From cd0cc187bea673b175b6d95e92499684459de48d Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 5 Dec 2019 20:15:41 +0800 Subject: [PATCH 001/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0mongodb=20oplog=20rea?= =?UTF-8?q?der=E6=8F=92=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/config/AbstractConfig.java | 4 + .../flinkx/constants/PluginNameConstrant.java | 1 + .../com/dtstack/flinkx/reader/DataReader.java | 3 + .../com/dtstack/flinkx/writer/DataWriter.java | 3 + flinkx-mongodb/flinkx-mongodb-core/pom.xml | 4 +- .../flinkx/mongodb/MongodbClientUtil.java | 133 +++++++++ .../dtstack/flinkx/mongodb/MongodbConfig.java | 282 ++++++++++++++++++ .../flinkx/mongodb/MongodbConfigKeys.java | 60 ---- .../dtstack/flinkx/mongodb/MongodbUtil.java | 127 -------- .../flinkx-mongodb-oplog-reader/pom.xml | 51 ++++ .../oplog/reader/MongodbEventHandler.java | 77 +++++ .../oplog/reader/MongodbOperation.java | 55 ++++ .../oplog/reader/MongodbOplogInputFormat.java | 200 +++++++++++++ .../MongodbOplogInputFormatBuilder.java | 45 +++ .../oplog/reader/MongodbOplogReader.java | 60 ++++ .../mongodb/reader/MongodbInputFormat.java | 55 +--- .../reader/MongodbInputFormatBuilder.java | 34 +-- .../flinkx/mongodb/reader/MongodbReader.java | 49 +-- .../mongodb/writer/MongodbOutputFormat.java | 47 +-- .../writer/MongodbOutputFormatBuilder.java | 49 +-- .../flinkx/mongodb/writer/MongodbWriter.java | 51 +--- flinkx-mongodb/pom.xml | 1 + .../stream/writer/StreamOutputFormat.java | 9 - flinkx-test/pom.xml | 6 + .../com/dtstack/flinkx/test/LocalTest.java | 21 +- .../resources/dev_test_job/oplog_stream.json | 40 +++ 26 files changed, 1024 insertions(+), 443 deletions(-) create mode 100644 flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java create mode 100644 flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java delete mode 100644 flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java create mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java create mode 100644 flinkx-test/src/main/resources/dev_test_job/oplog_stream.json diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java index a0a12f2689..e38d99a4dd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java @@ -44,6 +44,10 @@ public AbstractConfig(Map map) { } } + public Map getAll(){ + return internalMap; + } + public void setVal(String key, Object value) { internalMap.put(key, value); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/PluginNameConstrant.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/PluginNameConstrant.java index f573a0af63..1f24a562fa 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/PluginNameConstrant.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/PluginNameConstrant.java @@ -31,6 +31,7 @@ public class PluginNameConstrant { public static final String HBASE_READER = "hbasereader"; public static final String HDFS_READER = "hdfsreader"; public static final String MONGODB_READER = "mongodbreader"; + public static final String MONGODB_OPLOG_READER = "mongodboplogreader"; public static final String MYSQLD_READER = "mysqldreader"; public static final String MYSQL_READER = "mysqlreader"; public static final String ODPS_READER = "odpsreader"; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java index fb47727518..f5dc53b20e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReader.java @@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.functions.source.DtInputFormatSourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.codehaus.jackson.map.ObjectMapper; import java.util.ArrayList; import java.util.List; @@ -61,6 +62,8 @@ public abstract class DataReader { */ protected Map hadoopConfig; + protected static ObjectMapper objectMapper = new ObjectMapper(); + public int getNumPartitions() { return numPartitions; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriter.java index 6c8fa2101c..e8ccacf69c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriter.java @@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.functions.sink.DtOutputFormatSinkFunction; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.codehaus.jackson.map.ObjectMapper; import java.util.ArrayList; import java.util.List; @@ -58,6 +59,8 @@ public abstract class DataWriter { protected List srcCols = new ArrayList<>(); + protected static ObjectMapper objectMapper = new ObjectMapper(); + public List getSrcCols() { return srcCols; } diff --git a/flinkx-mongodb/flinkx-mongodb-core/pom.xml b/flinkx-mongodb/flinkx-mongodb-core/pom.xml index 82a3d2386b..403570c7a4 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-core/pom.xml @@ -16,13 +16,13 @@ org.mongodb mongodb-driver - 3.4.1 + 3.11.1 org.mongodb bson - 3.4.1 + 3.11.1 \ No newline at end of file diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java new file mode 100644 index 0000000000..150a3fe699 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.mongodb; + +import com.dtstack.flinkx.util.TelnetUtil; +import com.mongodb.*; +import com.mongodb.client.MongoCursor; +import org.apache.commons.lang.StringUtils; +import org.bson.Document; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbClientUtil { + + private static final Logger LOG = LoggerFactory.getLogger(MongodbClientUtil.class); + + private static final String HOST_SPLIT_REGEX = ",\\s*"; + + private static Pattern HOST_PORT_PATTERN = Pattern.compile("(?.*):(?\\d+)*"); + + private static final Integer DEFAULT_PORT = 27017; + + public static MongoClient getClient(MongodbConfig config){ + if(LOG.isDebugEnabled()){ + LOG.debug("连接配置:{}", config); + } + + if (StringUtils.isNotEmpty(config.getUrl())) { + return getClientWithUrl(config); + } else { + return getClientWithHostPort(config); + } + } + + public static void close(MongoClient mongoClient, MongoCursor cursor){ + if (cursor != null){ + LOG.info("Start close mongodb cursor"); + cursor.close(); + LOG.info("Close mongodb cursor successfully"); + } + + if (mongoClient != null){ + LOG.info("Start close mongodb client"); + mongoClient.close(); + LOG.info("Close mongodb client successfully"); + } + } + + private static MongoClient getClientWithHostPort(MongodbConfig config){ + MongoClientOptions options = getOption(config.getMongodbConfig()); + List serverAddress = getServerAddress(config.getHostPorts()); + + if (StringUtils.isEmpty(config.getUsername())) { + return new MongoClient(serverAddress,options); + } + + MongoCredential credential = MongoCredential.createCredential(config.getUsername(), config.getDatabase(), config.getPassword().toCharArray()) + .withMechanism(AuthenticationMechanism.fromMechanismName(config.getAuthenticationMechanism())); + + return new MongoClient(serverAddress,credential,options); + } + + private static MongoClient getClientWithUrl(MongodbConfig config){ + MongoClientURI clientURI = new MongoClientURI(config.getUrl()); + config.setDatabase(clientURI.getDatabase()); + return new MongoClient(clientURI); + } + + private static MongoClientOptions getOption(MongodbConfig.ConnectionConfig connectionConfig){ + MongoClientOptions.Builder build = new MongoClientOptions.Builder(); + build.connectionsPerHost(connectionConfig.getConnectionsPerHost()); + build.threadsAllowedToBlockForConnectionMultiplier(connectionConfig.getThreadsForConnectionMultiplier()); + build.connectTimeout(connectionConfig.getConnectionTimeout()); + build.maxWaitTime(connectionConfig.getMaxWaitTime()); + build.socketTimeout(connectionConfig.getSocketTimeout()); + build.writeConcern(WriteConcern.UNACKNOWLEDGED); + return build.build(); + } + + /** + * parse server address from hostPorts string + */ + private static List getServerAddress(String hostPorts) { + List addresses = new ArrayList<>(); + + for (String hostPort : hostPorts.split(HOST_SPLIT_REGEX)) { + if(hostPort.length() == 0){ + continue; + } + + Matcher matcher = HOST_PORT_PATTERN.matcher(hostPort); + if(matcher.find()){ + String host = matcher.group("host"); + String portStr = matcher.group("port"); + int port = portStr == null ? DEFAULT_PORT : Integer.parseInt(portStr); + + TelnetUtil.telnet(host,port); + + ServerAddress serverAddress = new ServerAddress(host,port); + addresses.add(serverAddress); + } + } + + return addresses; + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java new file mode 100644 index 0000000000..2af0a66f5f --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.mongodb; + +import java.io.Serializable; +import java.util.List; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbConfig implements Serializable { + + private String hostPorts; + + private String url; + + private String username; + + private String password; + + private String authenticationMechanism; + + private String database; + + private String collectionName; + + private String filter; + + private int fetchSize; + + private String writeMode; + + private String replaceKey; + + private List monitorDatabases; + + private List monitorCollections; + + private String clusterMode; + + private int startLocation; + + private boolean excludeDocId; + + private ConnectionConfig mongodbConfig = new ConnectionConfig(); + + public class ConnectionConfig implements Serializable{ + private int connectionsPerHost = 100; + + private int threadsForConnectionMultiplier = 100; + + private int connectionTimeout = 10000; + + private int maxWaitTime = 5000; + + private int socketTimeout = 0; + + public int getConnectionsPerHost() { + return connectionsPerHost; + } + + public void setConnectionsPerHost(int connectionsPerHost) { + this.connectionsPerHost = connectionsPerHost; + } + + public int getThreadsForConnectionMultiplier() { + return threadsForConnectionMultiplier; + } + + public void setThreadsForConnectionMultiplier(int threadsForConnectionMultiplier) { + this.threadsForConnectionMultiplier = threadsForConnectionMultiplier; + } + + public int getConnectionTimeout() { + return connectionTimeout; + } + + public void setConnectionTimeout(int connectionTimeout) { + this.connectionTimeout = connectionTimeout; + } + + public int getMaxWaitTime() { + return maxWaitTime; + } + + public void setMaxWaitTime(int maxWaitTime) { + this.maxWaitTime = maxWaitTime; + } + + public int getSocketTimeout() { + return socketTimeout; + } + + public void setSocketTimeout(int socketTimeout) { + this.socketTimeout = socketTimeout; + } + + @Override + public String toString() { + return "ConnectionConfig{" + + "connectionsPerHost=" + connectionsPerHost + + ", threadsForConnectionMultiplier=" + threadsForConnectionMultiplier + + ", connectionTimeout=" + connectionTimeout + + ", maxWaitTime=" + maxWaitTime + + ", socketTimeout=" + socketTimeout + + '}'; + } + } + + public boolean getExcludeDocId() { + return excludeDocId; + } + + public void setExcludeDocId(boolean excludeDocId) { + this.excludeDocId = excludeDocId; + } + + public int getStartLocation() { + return startLocation; + } + + public void setStartLocation(int startLocation) { + this.startLocation = startLocation; + } + + public String getClusterMode() { + return clusterMode; + } + + public void setClusterMode(String clusterMode) { + this.clusterMode = clusterMode; + } + + public List getMonitorDatabases() { + return monitorDatabases; + } + + public void setMonitorDatabases(List monitorDatabases) { + this.monitorDatabases = monitorDatabases; + } + + public List getMonitorCollections() { + return monitorCollections; + } + + public void setMonitorCollections(List monitorCollections) { + this.monitorCollections = monitorCollections; + } + + public String getAuthenticationMechanism() { + return authenticationMechanism; + } + + public void setAuthenticationMechanism(String authenticationMechanism) { + this.authenticationMechanism = authenticationMechanism; + } + + public String getHostPorts() { + return hostPorts; + } + + public void setHostPorts(String hostPorts) { + this.hostPorts = hostPorts; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getCollectionName() { + return collectionName; + } + + public void setCollectionName(String collectionName) { + this.collectionName = collectionName; + } + + public String getFilter() { + return filter; + } + + public void setFilter(String filter) { + this.filter = filter; + } + + public int getFetchSize() { + return fetchSize; + } + + public void setFetchSize(int fetchSize) { + this.fetchSize = fetchSize; + } + + public String getWriteMode() { + return writeMode; + } + + public void setWriteMode(String writeMode) { + this.writeMode = writeMode; + } + + public String getReplaceKey() { + return replaceKey; + } + + public void setReplaceKey(String replaceKey) { + this.replaceKey = replaceKey; + } + + public ConnectionConfig getMongodbConfig() { + return mongodbConfig; + } + + public void setMongodbConfig(ConnectionConfig mongodbConfig) { + this.mongodbConfig = mongodbConfig; + } + + @Override + public String toString() { + // TODO 密码脱敏 + return "MongodbConfig{" + + "hostPorts='" + hostPorts + '\'' + + ", url='" + url + '\'' + + ", username='" + username + '\'' + + ", password='" + "******" + '\'' + + ", authenticationMechanism='" + authenticationMechanism + '\'' + + ", database='" + database + '\'' + + ", collectionName='" + collectionName + '\'' + + ", filter='" + filter + '\'' + + ", fetchSize=" + fetchSize + + ", writeMode='" + writeMode + '\'' + + ", replaceKey='" + replaceKey + '\'' + + ", mongodbConfig=" + mongodbConfig + + '}'; + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java deleted file mode 100644 index 760052591b..0000000000 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfigKeys.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.mongodb; - -/** - * Configuration Keys for mongodb plugin - * - * @Company: www.dtstack.com - * @author jiangbo - */ -public class MongodbConfigKeys { - - public final static String KEY_HOST_PORTS = "hostPorts"; - - public final static String KEY_USERNAME = "username"; - - public final static String KEY_PASSWORD = "password"; - - public final static String KEY_URL = "url"; - - public final static String KEY_DATABASE = "database"; - - public final static String KEY_COLLECTION = "collectionName"; - - public final static String KEY_FILTER = "filter"; - - public final static String KEY_FETCH_SIZE = "fetchSize"; - - public final static String KEY_MODE = "writeMode"; - - public final static String KEY_REPLACE_KEY = "replaceKey"; - - public final static String KEY_MONGODB_CONFIG = "mongodbConfig"; - - public final static String KEY_CONNECTIONS_PERHOST = "connectionsPerHost"; - - public final static String KEY_THREADS_FOR_CONNECTION_MULTIPLIER = "threadsForConnectionMultiplier"; - - public final static String KEY_CONNECTION_TIMEOUT = "connectionTimeout"; - - public final static String KEY_MAX_WAIT_TIME = "maxWaitTime"; - - public final static String KEY_SOCKET_TIMEOUT = "socketTimeout"; -} diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java index 2cad2ec418..9de249d582 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java @@ -22,10 +22,6 @@ import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.TelnetUtil; -import com.mongodb.*; -import com.mongodb.client.MongoCursor; -import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.bson.Document; @@ -35,14 +31,9 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.text.SimpleDateFormat; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; /** * Utilities for mongodb database connection and data format conversion @@ -54,70 +45,6 @@ public class MongodbUtil { private static final Logger LOG = LoggerFactory.getLogger(MongodbUtil.class); - private static final String HOST_SPLIT_REGEX = ",\\s*"; - - private static Pattern HOST_PORT_PATTERN = Pattern.compile("(?.*):(?\\d+)*"); - - private static final Integer DEFAULT_PORT = 27017; - - private static final Integer DEFAULT_CONNECTIONS_PER_HOST = 100; - - private static final Integer DEFAULT_THREADS_FOR_CONNECTION_MULTIPLIER = 100; - - private static final Integer DEFAULT_CONNECT_TIMEOUT = 10 * 1000; - - private static final Integer DEFAULT_MAX_WAIT_TIME = 5 * 1000; - - private static final Integer DEFAULT_SOCKET_TIMEOUT = 0; - - /** - * Get mongo client - * @param mongodbConfig - * @return MongoClient - */ - public static MongoClient getMongoClient(Map mongodbConfig){ - MongoClient mongoClient; - try{ - MongoClientOptions options = getOption(mongodbConfig); - List serverAddress = getServerAddress(MapUtils.getString(mongodbConfig, KEY_HOST_PORTS)); - String username = MapUtils.getString(mongodbConfig, KEY_USERNAME); - String password = MapUtils.getString(mongodbConfig, KEY_PASSWORD); - String database = MapUtils.getString(mongodbConfig, KEY_DATABASE); - String url = MapUtils.getString(mongodbConfig, KEY_URL); - - if(StringUtils.isNotBlank(url)){ - mongoClient = new MongoClient(new MongoClientURI(url)); - }else if(StringUtils.isEmpty(username)){ - mongoClient = new MongoClient(serverAddress,options); - } else { - MongoCredential credential = MongoCredential.createScramSha1Credential(username, database, password.toCharArray()); - List credentials = new ArrayList<>(); - credentials.add(credential); - - mongoClient = new MongoClient(serverAddress,credentials,options); - } - - LOG.info("Get mongodb client successful"); - return mongoClient; - }catch (Exception e){ - throw new RuntimeException(e); - } - } - - public static void close(MongoClient mongoClient, MongoCursor cursor){ - if (cursor != null){ - LOG.info("Start close mongodb cursor"); - cursor.close(); - LOG.info("Close mongodb cursor successfully"); - } - - if (mongoClient != null){ - LOG.info("Start close mongodb client"); - mongoClient.close(); - LOG.info("Close mongodb client successfully"); - } - } - public static Document convertRowToDoc(Row row,List columns) throws WriteRecordException { Document doc = new Document(); for (int i = 0; i < columns.size(); i++) { @@ -145,58 +72,4 @@ private static Object convertField(Object val,MetaColumn column){ return val; } - - /** - * parse server address from hostPorts string - */ - private static List getServerAddress(String hostPorts) { - List addresses = new ArrayList<>(); - - for (String hostPort : hostPorts.split(HOST_SPLIT_REGEX)) { - if(hostPort.length() == 0){ - continue; - } - - Matcher matcher = HOST_PORT_PATTERN.matcher(hostPort); - if(matcher.find()){ - String host = matcher.group("host"); - String portStr = matcher.group("port"); - int port = portStr == null ? DEFAULT_PORT : Integer.parseInt(portStr); - - TelnetUtil.telnet(host,port); - - ServerAddress serverAddress = new ServerAddress(host,port); - addresses.add(serverAddress); - } - } - - return addresses; - } - - private static MongoClientOptions getOption(Map mongodbConfig){ - MongoClientOptions.Builder build = new MongoClientOptions.Builder(); - - int connectionsPerHost = MapUtils.getIntValue(mongodbConfig, KEY_CONNECTIONS_PERHOST, DEFAULT_CONNECTIONS_PER_HOST); - LOG.info("Mongodb config -- connectionsPerHost:" + connectionsPerHost); - build.connectionsPerHost(connectionsPerHost); - - int threadsForConnectionMultiplier = MapUtils.getIntValue(mongodbConfig, KEY_THREADS_FOR_CONNECTION_MULTIPLIER, DEFAULT_THREADS_FOR_CONNECTION_MULTIPLIER); - LOG.info("Mongodb config -- threadsForConnectionMultiplier:" + threadsForConnectionMultiplier); - build.threadsAllowedToBlockForConnectionMultiplier(threadsForConnectionMultiplier); - - int connectionTimeout = MapUtils.getIntValue(mongodbConfig, KEY_CONNECTION_TIMEOUT, DEFAULT_CONNECT_TIMEOUT); - LOG.info("Mongodb config -- connectionTimeout:" + connectionTimeout); - build.connectTimeout(connectionTimeout); - - int maxWaitTime = MapUtils.getIntValue(mongodbConfig, KEY_MAX_WAIT_TIME, DEFAULT_MAX_WAIT_TIME); - LOG.info("Mongodb config -- maxWaitTime:" + maxWaitTime); - build.maxWaitTime(maxWaitTime); - - int socketTimeout = MapUtils.getIntValue(mongodbConfig, KEY_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); - LOG.info("Mongodb config -- socketTimeout:" + socketTimeout); - build.maxWaitTime(socketTimeout); - - build.writeConcern(WriteConcern.UNACKNOWLEDGED); - return build.build(); - } } diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml new file mode 100644 index 0000000000..f38c4665c6 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml @@ -0,0 +1,51 @@ + + + + flinkx-mongodb + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-mongodb-oplog-reader + + + + flinkx-mongodb-core + com.dtstack.flinkx + 1.6 + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java new file mode 100644 index 0000000000..e0737e9c2a --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.mongodb.oplog.reader; + +import org.apache.flink.types.Row; +import org.bson.BsonTimestamp; +import org.bson.Document; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbEventHandler { + + public final static String EVENT_KEY_OP = "op"; + public final static String EVENT_KEY_NS = "ns"; + public final static String EVENT_KEY_TS = "ts"; + public final static String EVENT_KEY_DATA = "o"; + + public static Row handleEvent(final Document event, AtomicLong offset, boolean excludeDocId){ + MongodbOperation mongodbOperation = MongodbOperation.getByInternalNames(event.getString(EVENT_KEY_OP)); + Map eventMap = new LinkedHashMap<>(); + eventMap.put("type", mongodbOperation.name()); + + parseDbAndCollection(event, eventMap); + + BsonTimestamp timestamp = event.get(EVENT_KEY_TS, BsonTimestamp.class); + eventMap.put("ts", timestamp.getValue()); + eventMap.put("ingestion", System.nanoTime()); + + final Document data = (Document)event.get(EVENT_KEY_DATA); + Set keys = data.keySet(); + if(excludeDocId){ + keys.remove("_id"); + } + + for (String key : keys) { + eventMap.put("after_" + key, data.get(key)); + } + + for (String key : keys) { + eventMap.put("before_" + key, null); + } + + offset.set(timestamp.getValue()); + return Row.of(eventMap); + } + + private static void parseDbAndCollection(final Document event, Map eventMap){ + String dbCollection = event.getString(EVENT_KEY_NS); + String[] split = dbCollection.split("\\."); + eventMap.put("schema", split[0]); + eventMap.put("table", split[1]); + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java new file mode 100644 index 0000000000..5206630f79 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java @@ -0,0 +1,55 @@ +package com.dtstack.mongodb.oplog.reader; + +import java.util.ArrayList; +import java.util.List; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public enum MongodbOperation { + + /** + * 插入 + */ + INSERT("i"), + + /** + * 更新 + */ + UPDATE("u"), + + /** + * 删除 + */ + DELETE("d"); + + private String internalName; + + MongodbOperation(String internalName) { + this.internalName = internalName; + } + + public String getInternalName() { + return internalName; + } + + public static List internalNames(){ + List names = new ArrayList<>(); + for (MongodbOperation value : MongodbOperation.values()) { + names.add(value.getInternalName()); + } + + return names; + } + + public static MongodbOperation getByInternalNames(String name){ + for (MongodbOperation value : MongodbOperation.values()) { + if (value.getInternalName().equalsIgnoreCase(name)){ + return value; + } + } + + throw new RuntimeException("不支持的操作类型:" + name); + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java new file mode 100644 index 0000000000..2489794b17 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.mongodb.oplog.reader; + +import com.dtstack.flinkx.inputformat.RichInputFormat; +import com.dtstack.flinkx.mongodb.MongodbClientUtil; +import com.dtstack.flinkx.mongodb.MongodbConfig; +import com.dtstack.flinkx.restore.FormatState; +import com.mongodb.CursorType; +import com.mongodb.MongoClient; +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Filters; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.bson.conversions.Bson; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbOplogInputFormat extends RichInputFormat { + + private final static String OPLOG_DB = "local"; + private final static String REPLICA_SET_COLLECTION = "oplog.rs"; + private final static String MASTER_SLAVE_COLLECTION = "oplog.$main"; + + protected MongodbConfig mongodbConfig; + + private transient MongoClient client; + + private transient MongoCursor cursor; + + private AtomicLong offset = new AtomicLong(); + + @Override + public void configure(Configuration parameters) { + // do nothing + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + initOffset(); + + client = MongodbClientUtil.getClient(mongodbConfig); + + + MongoCollection oplog = getOplogCollection(); + FindIterable results = oplog.find(buildFilter()) + .sort(new Document("$natural", 1)) + .oplogReplay(true) + .cursorType(CursorType.TailableAwait); + + cursor = results.iterator(); + } + + /** + * 在 master/slave 结构下, oplog 位于local.oplog.$main + * 在 Replca set 结构下, oplog 位于local.oplog.rs + */ + private MongoCollection getOplogCollection(){ + if ("REPLICA_SET".equalsIgnoreCase(mongodbConfig.getClusterMode())) { + return client.getDatabase(OPLOG_DB).getCollection(REPLICA_SET_COLLECTION); + } else if("MASTER_SLAVE".equalsIgnoreCase(mongodbConfig.getClusterMode())){ + return client.getDatabase(OPLOG_DB).getCollection(MASTER_SLAVE_COLLECTION); + } else { + throw new RuntimeException(""); + } + } + + private void initOffset(){ + BsonTimestamp startLocation = new BsonTimestamp(mongodbConfig.getStartLocation(), 0); + if (formatState != null && formatState.getState() != null) { + offset.set(Long.valueOf(formatState.getState().toString())); + long state = (Long)formatState.getState(); + if (startLocation.compareTo(new BsonTimestamp(state)) > 0) { + offset.set(mongodbConfig.getStartLocation()); + } else { + offset.set(state); + } + } else { + offset.set(startLocation.getValue()); + } + } + + private Bson buildFilter(){ + List filters = new ArrayList<>(); + + // 设置读取位置 + filters.add(Filters.gt(MongodbEventHandler.EVENT_KEY_TS, new BsonTimestamp(offset.get()))); + + // + filters.add(Filters.exists("fromMigrate", false)); + + // 过滤db和collection + String pattern = buildPattern(); + if (pattern != null) { + filters.add(Filters.regex(MongodbEventHandler.EVENT_KEY_NS, pattern)); + } + + // 过滤系统日志 + filters.add(Filters.ne(MongodbEventHandler.EVENT_KEY_NS, "config.system.sessions")); + + // 过滤操作类型 + filters.add(Filters.in(MongodbEventHandler.EVENT_KEY_OP, MongodbOperation.internalNames())); + + return Filters.and(filters); + } + + private String buildPattern() { + if (CollectionUtils.isEmpty(mongodbConfig.getMonitorDatabases()) && CollectionUtils.isEmpty(mongodbConfig.getMonitorCollections())){ + return null; + } + + StringBuilder pattern = new StringBuilder(); + if(CollectionUtils.isNotEmpty(mongodbConfig.getMonitorDatabases())){ + mongodbConfig.getMonitorDatabases().removeIf(StringUtils::isEmpty); + if(CollectionUtils.isNotEmpty(mongodbConfig.getMonitorDatabases())){ + String databasePattern = StringUtils.join(mongodbConfig.getMonitorDatabases(), "|"); + pattern.append("(").append(databasePattern).append(")"); + } else { + pattern.append(".*"); + } + } + + pattern.append("\\."); + + if(CollectionUtils.isNotEmpty(mongodbConfig.getMonitorCollections())){ + mongodbConfig.getMonitorCollections().removeIf(String::isEmpty); + if(CollectionUtils.isNotEmpty(mongodbConfig.getMonitorCollections())){ + String collectionPattern = StringUtils.join(mongodbConfig.getMonitorCollections(), "|"); + pattern.append("(").append(collectionPattern).append(")"); + } else { + pattern.append(".*"); + } + } + + return pattern.toString(); + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + return MongodbEventHandler.handleEvent(cursor.next(), offset, mongodbConfig.getExcludeDocId()); + } + + @Override + public FormatState getFormatState() { + super.getFormatState(); + + if (formatState != null){ + formatState.setState(offset.get()); + } + + return formatState; + } + + @Override + protected void closeInternal() throws IOException { + MongodbClientUtil.close(client, cursor); + } + + @Override + public InputSplit[] createInputSplits(int minNumSplits) throws IOException { + return new InputSplit[]{new GenericInputSplit(1,1)}; + } + + @Override + public boolean reachedEnd() throws IOException { + return !cursor.hasNext(); + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java new file mode 100644 index 0000000000..3519e2fe35 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.mongodb.oplog.reader; + +import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; +import com.dtstack.flinkx.mongodb.MongodbConfig; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbOplogInputFormatBuilder extends RichInputFormatBuilder { + + private MongodbOplogInputFormat format; + + public MongodbOplogInputFormatBuilder() { + super.format = this.format = new MongodbOplogInputFormat(); + } + + public void setMongodbConfig(MongodbConfig mongodbConfig){ + format.mongodbConfig = mongodbConfig; + } + + @Override + protected void checkFormat() { + + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java new file mode 100644 index 0000000000..e44d20aa67 --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.mongodb.oplog.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.mongodb.MongodbConfig; +import com.dtstack.flinkx.reader.DataReader; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +/** + * @author jiangbo + * @date 2019/12/5 + */ +public class MongodbOplogReader extends DataReader { + + private MongodbConfig mongodbConfig; + + public MongodbOplogReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + try { + mongodbConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), MongodbConfig.class); + } catch (Exception e) { + throw new RuntimeException("解析mongodb配置出错:", e); + } + } + + @Override + public DataStream readData() { + MongodbOplogInputFormatBuilder builder = new MongodbOplogInputFormatBuilder(); + builder.setMonitorUrls(monitorUrls); + builder.setBytes(bytes); + builder.setRestoreConfig(restoreConfig); + + builder.setMongodbConfig(mongodbConfig); + + return createInput(builder.finish()); + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index f28df20567..8f3d39308c 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -19,13 +19,12 @@ package com.dtstack.flinkx.mongodb.reader; import com.dtstack.flinkx.inputformat.RichInputFormat; -import com.dtstack.flinkx.mongodb.MongodbConfigKeys; -import com.dtstack.flinkx.mongodb.MongodbUtil; +import com.dtstack.flinkx.mongodb.MongodbClientUtil; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; import com.mongodb.BasicDBObject; import com.mongodb.MongoClient; -import com.mongodb.MongoClientURI; import com.mongodb.client.FindIterable; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; @@ -48,26 +47,12 @@ */ public class MongodbInputFormat extends RichInputFormat { - protected String hostPorts; - - protected String username; - - protected String password; - - protected String database; - - protected String collectionName; - protected List metaColumns; - protected String filterJson; - - protected Map mongodbConfig; - - protected int fetchSize; - private Bson filter; + protected MongodbConfig mongodbConfig; + private transient MongoCursor cursor; private transient MongoClient client; @@ -82,13 +67,9 @@ protected void openInternal(InputSplit inputSplit) throws IOException { MongodbInputSplit split = (MongodbInputSplit) inputSplit; FindIterable findIterable; - client = MongodbUtil.getMongoClient(mongodbConfig); - if(StringUtils.isBlank(database)){ - String url = (String) mongodbConfig.get(MongodbConfigKeys.KEY_URL); - database = new MongoClientURI(url).getDatabase(); - } - MongoDatabase db = client.getDatabase(database); - MongoCollection collection = db.getCollection(collectionName); + client = MongodbClientUtil.getClient(mongodbConfig); + MongoDatabase db = client.getDatabase(mongodbConfig.getDatabase()); + MongoCollection collection = db.getCollection(mongodbConfig.getCollectionName()); if(filter == null){ findIterable = collection.find(); @@ -98,7 +79,7 @@ protected void openInternal(InputSplit inputSplit) throws IOException { findIterable = findIterable.skip(split.getSkip()) .limit(split.getLimit()) - .batchSize(fetchSize); + .batchSize(mongodbConfig.getFetchSize()); cursor = findIterable.iterator(); } @@ -139,7 +120,7 @@ public Row nextRecordInternal(Row row) throws IOException { @Override protected void closeInternal() throws IOException { - MongodbUtil.close(client, cursor); + MongodbClientUtil.close(client, cursor); } @Override @@ -148,13 +129,9 @@ public InputSplit[] createInputSplits(int minNumSplits) throws IOException { MongoClient client = null; try { - client = MongodbUtil.getMongoClient(mongodbConfig); - if(StringUtils.isBlank(database)){ - String url = (String) mongodbConfig.get(MongodbConfigKeys.KEY_URL); - database = new MongoClientURI(url).getDatabase(); - } - MongoDatabase db = client.getDatabase(database); - MongoCollection collection = db.getCollection(collectionName); + client = MongodbClientUtil.getClient(mongodbConfig); + MongoDatabase db = client.getDatabase(mongodbConfig.getDatabase()); + MongoCollection collection = db.getCollection(mongodbConfig.getCollectionName()); long docNum = filter == null ? collection.count() : collection.count(filter); if(docNum <= minNumSplits){ @@ -171,9 +148,9 @@ public InputSplit[] createInputSplits(int minNumSplits) throws IOException { splits.add(new MongodbInputSplit((int)(size * minNumSplits), (int)(docNum - size * minNumSplits))); } } catch (Exception e){ - LOG.error("{}", e); + LOG.error("创建分片失败:", e); } finally { - MongodbUtil.close(client, null); + MongodbClientUtil.close(client, null); } return splits.toArray(new MongodbInputSplit[splits.size()]); @@ -185,8 +162,8 @@ public boolean reachedEnd() throws IOException { } private void buildFilter(){ - if(StringUtils.isNotEmpty(filterJson)){ - filter = BasicDBObject.parse(filterJson); + if(StringUtils.isNotEmpty(mongodbConfig.getFilter())){ + filter = BasicDBObject.parse(mongodbConfig.getFilter()); } } } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java index 5ec5c7f748..fde7b456f6 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java @@ -19,10 +19,10 @@ package com.dtstack.flinkx.mongodb.reader; import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; -import java.util.Map; /** * The builder for mongodb reader plugin @@ -38,45 +38,17 @@ public MongodbInputFormatBuilder() { super.format = format = new MongodbInputFormat(); } - public void setHostPorts(String hostPorts){ - format.hostPorts = hostPorts; - } - - public void setUsername(String username){ - format.username = username; - } - - public void setPassword(String password){ - format.password = password; - } - - public void setDatabase(String database){ - format.database = database; - } - - public void setCollection(String collection){ - format.collectionName = collection; - } - public void setMetaColumns(List metaColumns){ format.metaColumns = metaColumns; } - public void setMongodbConfig(Map mongodbConfig){ + public void setMongodbConfig(MongodbConfig mongodbConfig){ format.mongodbConfig = mongodbConfig; } - public void setFetchSize(int fetchSize){ - format.fetchSize = fetchSize; - } - - public void setFilter(String filter){ - format.filterJson = filter; - } - @Override protected void checkFormat() { - if(format.collectionName == null){ + if(format.mongodbConfig.getCollectionName() == null){ throw new IllegalArgumentException("No collection supplied"); } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java index 85a52ac470..e1591e6d3f 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbReader.java @@ -20,17 +20,14 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.DataReader; import com.dtstack.flinkx.reader.MetaColumn; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -import java.util.HashMap; import java.util.List; -import java.util.Map; - -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; /** * The Reader plugin for mongodb database @@ -40,60 +37,28 @@ */ public class MongodbReader extends DataReader { - protected String hostPorts; - - protected String username; - - protected String password; - - protected String url; - - protected String database; - - protected String collection; - private List metaColumns; - protected String filter; - - protected Map mongodbConfig; - - protected int fetchSize; + private MongodbConfig mongodbConfig; public MongodbReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - hostPorts = readerConfig.getParameter().getStringVal(KEY_HOST_PORTS); - username = readerConfig.getParameter().getStringVal(KEY_USERNAME); - password = readerConfig.getParameter().getStringVal(KEY_PASSWORD); - url = readerConfig.getParameter().getStringVal(KEY_URL); - database = readerConfig.getParameter().getStringVal(KEY_DATABASE); - collection = readerConfig.getParameter().getStringVal(KEY_COLLECTION); - filter = readerConfig.getParameter().getStringVal(KEY_FILTER); - fetchSize = readerConfig.getParameter().getIntVal(KEY_FETCH_SIZE, 100); metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); - mongodbConfig = (Map)readerConfig.getParameter().getVal(KEY_MONGODB_CONFIG, new HashMap<>()); - mongodbConfig.put(KEY_HOST_PORTS, hostPorts); - mongodbConfig.put(KEY_USERNAME, username); - mongodbConfig.put(KEY_PASSWORD, password); - mongodbConfig.put(KEY_URL, url); - mongodbConfig.put(KEY_DATABASE, database); + try { + mongodbConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), MongodbConfig.class); + } catch (Exception e) { + throw new RuntimeException("解析mongodb配置出错:", e); + } } @Override public DataStream readData() { MongodbInputFormatBuilder builder = new MongodbInputFormatBuilder(); - builder.setHostPorts(hostPorts); - builder.setUsername(username); - builder.setPassword(password); - builder.setDatabase(database); - builder.setCollection(collection); - builder.setFilter(filter); builder.setMetaColumns(metaColumns); builder.setMongodbConfig(mongodbConfig); - builder.setFetchSize(fetchSize); builder.setMonitorUrls(monitorUrls); builder.setBytes(bytes); diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java index 34101da24a..b5a1b5d841 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormat.java @@ -19,16 +19,15 @@ package com.dtstack.flinkx.mongodb.writer; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.mongodb.MongodbConfigKeys; +import com.dtstack.flinkx.mongodb.MongodbClientUtil; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.mongodb.MongodbUtil; import com.dtstack.flinkx.outputformat.RichOutputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.writer.WriteMode; import com.mongodb.MongoClient; -import com.mongodb.MongoClientURI; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; -import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; import org.bson.Document; @@ -36,7 +35,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * OutputFormat for mongodb writer plugin @@ -46,27 +44,13 @@ */ public class MongodbOutputFormat extends RichOutputFormat { - protected String hostPorts; - - protected String username; - - protected String password; - - protected String database; - - protected String collectionName; - protected List columns; - protected String replaceKey; - - protected String mode; - private transient MongoCollection collection; private transient MongoClient client; - protected Map mongodbConfig; + protected MongodbConfig mongodbConfig; @Override public void configure(Configuration parameters) { @@ -75,13 +59,9 @@ public void configure(Configuration parameters) { @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { - client = MongodbUtil.getMongoClient(mongodbConfig); - if(StringUtils.isBlank(database)){ - String url = (String) mongodbConfig.get(MongodbConfigKeys.KEY_URL); - database = new MongoClientURI(url).getDatabase(); - } - MongoDatabase db = client.getDatabase(database); - collection = db.getCollection(collectionName); + client = MongodbClientUtil.getClient(mongodbConfig); + MongoDatabase db = client.getDatabase(mongodbConfig.getDatabase()); + collection = db.getCollection(mongodbConfig.getCollectionName()); } @Override @@ -89,10 +69,11 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { try { Document doc = MongodbUtil.convertRowToDoc(row,columns); - if(WriteMode.INSERT.getMode().equals(mode)){ + if(WriteMode.INSERT.getMode().equals(mongodbConfig.getWriteMode())){ collection.insertOne(doc); - } else if(WriteMode.REPLACE.getMode().equals(mode) || WriteMode.UPDATE.getMode().equals(mode)){ - Document filter = new Document(replaceKey,doc.get(replaceKey)); + } else if(WriteMode.REPLACE.getMode().equals(mongodbConfig.getWriteMode()) + || WriteMode.UPDATE.getMode().equals(mongodbConfig.getWriteMode())){ + Document filter = new Document(mongodbConfig.getReplaceKey(), doc.get(mongodbConfig.getReplaceKey())); collection.findOneAndReplace(filter,doc); } } catch (Exception e){ @@ -107,17 +88,17 @@ protected void writeMultipleRecordsInternal() throws Exception { documents.add(MongodbUtil.convertRowToDoc(row,columns)); } - if(WriteMode.INSERT.getMode().equals(mode)){ + if(WriteMode.INSERT.getMode().equals(mongodbConfig.getWriteMode())){ collection.insertMany(documents); - } else if(WriteMode.UPDATE.getMode().equals(mode)) { + } else if(WriteMode.UPDATE.getMode().equals(mongodbConfig.getWriteMode())) { throw new RuntimeException("Does not support batch update documents"); - } else if(WriteMode.REPLACE.getMode().equals(mode)){ + } else if(WriteMode.REPLACE.getMode().equals(mongodbConfig.getWriteMode())){ throw new RuntimeException("Does not support batch replace documents"); } } @Override public void closeInternal() throws IOException { - MongodbUtil.close(client, null); + MongodbClientUtil.close(client, null); } } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java index 7d1fed0a12..0ba22c3da7 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbOutputFormatBuilder.java @@ -18,15 +18,12 @@ package com.dtstack.flinkx.mongodb.writer; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.outputformat.RichOutputFormatBuilder; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.writer.WriteMode; import org.apache.commons.lang.StringUtils; -import java.util.List; -import java.util.Map; - - /** * The builder for mongodb writer plugin * @@ -41,57 +38,25 @@ public MongodbOutputFormatBuilder() { super.format = format = new MongodbOutputFormat(); } - public void setHostPorts(String hostPorts){ - format.hostPorts = hostPorts; - } - - public void setUsername(String username){ - format.username = username; - } - - public void setPassword(String password){ - format.password = password; - } - - public void setDatabase(String database){ - format.database = database; - } - - public void setCollection(String collection){ - format.collectionName = collection; - } - - public void setColumns(List columns){ - format.columns = columns; - } - - public void setMode(String mode){ - format.mode = mode; - } - - public void setReplaceKey(String replaceKey){ - format.replaceKey = replaceKey; - } - - - public void setMongodbConfig(Map mongodbConfig){ + public void setMongodbConfig(MongodbConfig mongodbConfig){ format.mongodbConfig = mongodbConfig; } @Override protected void checkFormat() { - if(format.collectionName == null){ + if(format.mongodbConfig.getCollectionName() == null){ throw new IllegalArgumentException("No collection supplied"); } - if(WriteMode.REPLACE.getMode().equals(format.mode) || WriteMode.UPDATE.getMode().equals(format.mode)){ - if(StringUtils.isEmpty(format.replaceKey)){ + if(WriteMode.REPLACE.getMode().equals(format.mongodbConfig.getWriteMode()) + || WriteMode.UPDATE.getMode().equals(format.mongodbConfig.getWriteMode())){ + if(StringUtils.isEmpty(format.mongodbConfig.getReplaceKey())){ throw new IllegalArgumentException("ReplaceKey cannot be empty when the write mode is replace"); } boolean columnContainsReplaceKey = false; for (MetaColumn column : format.columns) { - if (column.getName().equalsIgnoreCase(format.replaceKey)) { + if (column.getName().equalsIgnoreCase(format.mongodbConfig.getReplaceKey())) { columnContainsReplaceKey = true; break; } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java index 5626c5d56d..e7eca2581f 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java +++ b/flinkx-mongodb/flinkx-mongodb-writer/src/main/java/com/dtstack/flinkx/mongodb/writer/MongodbWriter.java @@ -20,19 +20,14 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.writer.DataWriter; -import com.dtstack.flinkx.writer.WriteMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; -import java.util.HashMap; import java.util.List; -import java.util.Map; - -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.*; -import static com.dtstack.flinkx.mongodb.MongodbConfigKeys.KEY_COLLECTION; /** * The writer plugin for mongodb database @@ -42,59 +37,27 @@ */ public class MongodbWriter extends DataWriter { - protected String hostPorts; - - protected String username; - - protected String password; - - protected String database; - - protected String url; - - protected String collection; - protected List columns; - protected String replaceKey; - - protected Map mongodbConfig; + protected MongodbConfig mongodbConfig; public MongodbWriter(DataTransferConfig config) { super(config); WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); - hostPorts = writerConfig.getParameter().getStringVal(KEY_HOST_PORTS); - username = writerConfig.getParameter().getStringVal(KEY_USERNAME); - password = writerConfig.getParameter().getStringVal(KEY_PASSWORD); - url = writerConfig.getParameter().getStringVal(KEY_URL); - database = writerConfig.getParameter().getStringVal(KEY_DATABASE); - collection = writerConfig.getParameter().getStringVal(KEY_COLLECTION); - mode = writerConfig.getParameter().getStringVal(KEY_MODE, WriteMode.INSERT.getMode()); - replaceKey = writerConfig.getParameter().getStringVal(KEY_REPLACE_KEY); - columns = MetaColumn.getMetaColumns(writerConfig.getParameter().getColumn()); - mongodbConfig = (Map)writerConfig.getParameter().getVal(KEY_MONGODB_CONFIG, new HashMap<>()); - mongodbConfig.put(KEY_HOST_PORTS, hostPorts); - mongodbConfig.put(KEY_USERNAME, username); - mongodbConfig.put(KEY_PASSWORD, password); - mongodbConfig.put(KEY_URL, url); - mongodbConfig.put(KEY_DATABASE, database); + try { + mongodbConfig = objectMapper.readValue(objectMapper.writeValueAsString(writerConfig.getParameter().getAll()), MongodbConfig.class); + } catch (Exception e) { + throw new RuntimeException("解析mongodb配置出错:", e); + } } @Override public DataStreamSink writeData(DataStream dataSet) { MongodbOutputFormatBuilder builder = new MongodbOutputFormatBuilder(); - builder.setHostPorts(hostPorts); - builder.setUsername(username); - builder.setPassword(password); - builder.setDatabase(database); - builder.setCollection(collection); - builder.setMode(mode); - builder.setColumns(columns); - builder.setReplaceKey(replaceKey); builder.setMongodbConfig(mongodbConfig); builder.setMonitorUrls(monitorUrls); diff --git a/flinkx-mongodb/pom.xml b/flinkx-mongodb/pom.xml index d8a418734c..876dc99887 100644 --- a/flinkx-mongodb/pom.xml +++ b/flinkx-mongodb/pom.xml @@ -14,6 +14,7 @@ flinkx-mongodb-core flinkx-mongodb-reader + flinkx-mongodb-oplog-reader flinkx-mongodb-writer diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java index 2257e87ab6..98f1e95c0d 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java @@ -44,10 +44,6 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { if (print) { System.out.println(String.format("subTaskIndex[%s]:%s", taskNumber, row)); } - - if (restoreConfig.isRestore()) { - formatState.setState(row.getField(restoreConfig.getRestoreColumnIndex())); - } } @Override @@ -57,10 +53,5 @@ protected void writeMultipleRecordsInternal() throws Exception { System.out.println(row); } } - - if (restoreConfig.isRestore()) { - Row row = rows.get(rows.size() - 1); - formatState.setState(row.getField(restoreConfig.getRestoreColumnIndex())); - } } } diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index effaf52bd3..4b2c9753d2 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -362,6 +362,12 @@ flinkx-polardb-writer 1.6 + + + com.dtstack.flinkx + flinkx-mongodb-oplog-reader + 1.6 + diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index e8e47f3efd..09be200bd2 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -18,8 +18,6 @@ package com.dtstack.flinkx.test; -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.binlog.reader.BinlogReader; import com.dtstack.flinkx.carbondata.reader.CarbondataReader; @@ -73,6 +71,7 @@ import com.dtstack.flinkx.stream.writer.StreamWriter; import com.dtstack.flinkx.util.ResultPrintUtil; import com.dtstack.flinkx.writer.DataWriter; +import com.dtstack.mongodb.oplog.reader.MongodbOplogReader; import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -113,12 +112,12 @@ public class LocalTest { public static Configuration conf = new Configuration(); public static void main(String[] args) throws Exception{ - setLogLevel(Level.INFO.toString()); - Properties confProperties = new Properties(); -// confProperties.put("flink.checkpoint.interval", "10000"); -// confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); + confProperties.put("flink.checkpoint.interval", "10000"); // + conf.setString("state.backend", "filesystem"); + conf.setString("state.checkpoints.dir", "file:///D:/tmp/flinkx_checkpoint"); + // conf.setString("metrics.reporter.promgateway.class","org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"); // conf.setString("metrics.reporter.promgateway.host","172.16.10.204"); // conf.setString("metrics.reporter.promgateway.port","9091"); @@ -126,18 +125,11 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\hdfs_stream.json"; + String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\oplog_stream.json"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); } - private static void setLogLevel(String level){ - LoggerContext loggerContext= (LoggerContext) LoggerFactory.getILoggerFactory(); - //设置全局日志级别 - ch.qos.logback.classic.Logger logger=loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel(level)); - } - public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savepointPath) throws Exception{ String jobContent = readJob(jobFile); return runJob(jobContent, confProperties, savepointPath); @@ -208,6 +200,7 @@ private static DataReader buildDataReader(DataTransferConfig config, StreamExecu case PluginNameConstrant.KUDU_READER : reader = new KuduReader(config, env); break; case PluginNameConstrant.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; case PluginNameConstrant.POLARDB_READER : reader = new PolardbReader(config, env); break; + case PluginNameConstrant.MONGODB_OPLOG_READER : reader = new MongodbOplogReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } diff --git a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json new file mode 100644 index 0000000000..e1cd579497 --- /dev/null +++ b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json @@ -0,0 +1,40 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "mongodboplogreader", + "parameter": { + "hostPorts": "172.16.8.190:30001,172.16.8.190:30002,172.16.8.190:30003", + "username": "root", + "password": "123456x", + "database": "admin", + "clusterMode": "REPLICA_SET", + "authenticationMechanism": "SCRAM-SHA-256", + "monitorDatabases": ["jiangbo"], + "excludeDocId": false + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 1048576 + }, + "errorLimit": { + "record": 100 + }, + "restore" : { + "isRestore" : true, + "isStream" : true + } + } + } +} \ No newline at end of file From 2c87a093d2df9bf359e51d7e6fd40803073e1522 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 6 Dec 2019 12:33:42 +0800 Subject: [PATCH 002/136] =?UTF-8?q?=E5=A4=84=E7=90=86mongo=E9=9B=86?= =?UTF-8?q?=E7=BE=A4=E8=8A=82=E7=82=B9=E6=8C=82=E6=8E=89=E7=9A=84=E6=83=85?= =?UTF-8?q?=E5=86=B5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/mongodb/MongodbClientUtil.java | 2 -- .../oplog/reader/MongodbOplogInputFormat.java | 19 +++++++++++++++---- .../resources/dev_test_job/oplog_stream.json | 2 +- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java index 150a3fe699..8e6bf0fd51 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java @@ -121,8 +121,6 @@ private static List getServerAddress(String hostPorts) { String portStr = matcher.group("port"); int port = portStr == null ? DEFAULT_PORT : Integer.parseInt(portStr); - TelnetUtil.telnet(host,port); - ServerAddress serverAddress = new ServerAddress(host,port); addresses.add(serverAddress); } diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java index 2489794b17..25765d9cb1 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java @@ -62,6 +62,8 @@ public class MongodbOplogInputFormat extends RichInputFormat { private AtomicLong offset = new AtomicLong(); + private InputSplit inputSplit; + @Override public void configure(Configuration parameters) { // do nothing @@ -69,11 +71,10 @@ public void configure(Configuration parameters) { @Override protected void openInternal(InputSplit inputSplit) throws IOException { + this.inputSplit = inputSplit; initOffset(); client = MongodbClientUtil.getClient(mongodbConfig); - - MongoCollection oplog = getOplogCollection(); FindIterable results = oplog.find(buildFilter()) .sort(new Document("$natural", 1)) @@ -93,7 +94,7 @@ private MongoCollection getOplogCollection(){ } else if("MASTER_SLAVE".equalsIgnoreCase(mongodbConfig.getClusterMode())){ return client.getDatabase(OPLOG_DB).getCollection(MASTER_SLAVE_COLLECTION); } else { - throw new RuntimeException(""); + throw new RuntimeException("集群模式不支持:" + mongodbConfig.getClusterMode()); } } @@ -195,6 +196,16 @@ public InputSplit[] createInputSplits(int minNumSplits) throws IOException { @Override public boolean reachedEnd() throws IOException { - return !cursor.hasNext(); + try { + return !cursor.hasNext(); + } catch (Exception e) { + // 这里出现异常可能是因为集群里某个节点挂了,所以不退出程序,调用openInternal方法重新连接,并从offset处开始同步数据, + // 如果集群有问题,在openInternal方法里结束进程 + LOG.warn("获取数据异常,可能是某个节点出问题了,程序将自动重新选择节点连接", e); + closeInternal(); + openInternal(inputSplit); + + return false; + } } } diff --git a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json index e1cd579497..7bfddb333f 100644 --- a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json +++ b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json @@ -7,7 +7,7 @@ "parameter": { "hostPorts": "172.16.8.190:30001,172.16.8.190:30002,172.16.8.190:30003", "username": "root", - "password": "123456x", + "password": "123456", "database": "admin", "clusterMode": "REPLICA_SET", "authenticationMechanism": "SCRAM-SHA-256", From d411a89666121313684275ac293524afe66c757a Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 13 Dec 2019 21:40:51 +0800 Subject: [PATCH 003/136] =?UTF-8?q?=E5=AE=8C=E6=88=90check?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-pgwal/flinkx-pgwal-core/pom.xml | 22 ++ .../flinkx/pgwal/PgRelicationSlot.java | 161 ++++++++++++ .../dtstack/flinkx/pgwal/PgWalConfigKeys.java | 51 ++++ .../com/dtstack/flinkx/pgwal/PgWalUtil.java | 229 ++++++++++++++++++ flinkx-pgwal/flinkx-pgwal-reader/pom.xml | 69 ++++++ .../flinkx/pgwal/format/PgWalInputFormat.java | 136 +++++++++++ .../pgwal/format/PgWalInputFormatBuilder.java | 113 +++++++++ .../flinkx/pgwal/reader/PgWalReader.java | 88 +++++++ flinkx-pgwal/pom.xml | 29 +++ pom.xml | 71 +++--- 10 files changed, 935 insertions(+), 34 deletions(-) create mode 100644 flinkx-pgwal/flinkx-pgwal-core/pom.xml create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/pom.xml create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java create mode 100644 flinkx-pgwal/pom.xml diff --git a/flinkx-pgwal/flinkx-pgwal-core/pom.xml b/flinkx-pgwal/flinkx-pgwal-core/pom.xml new file mode 100644 index 0000000000..a8c6798938 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/pom.xml @@ -0,0 +1,22 @@ + + + + flinkx-pgwal + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-pgwal-core + + + + org.postgresql + postgresql + 42.2.8 + + + + \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java new file mode 100644 index 0000000000..f002f50d6d --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgRelicationSlot { + private String slotName; + private String plugin; + private String slotType; + private Integer datoid; + private String database; + private String temporary; + private String active; + private Integer activePid; + private String xmin; + private String catalogXmin; + private String restartLsn; + private String confirmedFlushLsn; + + public boolean isActive(){ + return "t".equalsIgnoreCase(active); + } + + public boolean isNotActive(){ + return !isActive(); + } + + public String getSlotName() { + return slotName; + } + + public void setSlotName(String slotName) { + this.slotName = slotName; + } + + public String getPlugin() { + return plugin; + } + + public void setPlugin(String plugin) { + this.plugin = plugin; + } + + public String getSlotType() { + return slotType; + } + + public void setSlotType(String slotType) { + this.slotType = slotType; + } + + public Integer getDatoid() { + return datoid; + } + + public void setDatoid(Integer datoid) { + this.datoid = datoid; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTemporary() { + return temporary; + } + + public void setTemporary(String temporary) { + this.temporary = temporary; + } + + public String getActive() { + return active; + } + + public void setActive(String active) { + this.active = active; + } + + public Integer getActivePid() { + return activePid; + } + + public void setActivePid(Integer activePid) { + this.activePid = activePid; + } + + public String getXmin() { + return xmin; + } + + public void setXmin(String xmin) { + this.xmin = xmin; + } + + public String getCatalogXmin() { + return catalogXmin; + } + + public void setCatalogXmin(String catalogXmin) { + this.catalogXmin = catalogXmin; + } + + public String getRestartLsn() { + return restartLsn; + } + + public void setRestartLsn(String restartLsn) { + this.restartLsn = restartLsn; + } + + public String getConfirmedFlushLsn() { + return confirmedFlushLsn; + } + + public void setConfirmedFlushLsn(String confirmedFlushLsn) { + this.confirmedFlushLsn = confirmedFlushLsn; + } + + @Override + public String toString() { + return "PgRelicationSlots{" + + "slotName='" + slotName + '\'' + + ", plugin='" + plugin + '\'' + + ", slotType='" + slotType + '\'' + + ", datoid=" + datoid + + ", database='" + database + '\'' + + ", temporary='" + temporary + '\'' + + ", active='" + active + '\'' + + ", activePid='" + activePid + '\'' + + ", xmin='" + xmin + '\'' + + ", catalogXmin='" + catalogXmin + '\'' + + ", restartLsn='" + restartLsn + '\'' + + ", conFirmedFlushLsn='" + confirmedFlushLsn + '\'' + + '}'; + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java new file mode 100644 index 0000000000..c0d3f0a03f --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalConfigKeys { + public static final String KEY_USER_NAME = "username"; + + public static final String KEY_PASSWORD = "password"; + + public static final String KEY_URL = "url"; + + public final static String KEY_DATABASE_NAME = "databaseName"; + + public final static String KEY_CATALOG = "cat"; + + public final static String KEY_PAVING_DATA = "pavingData"; + + public final static String KEY_TABLE_LIST = "tableList"; + + public final static String KEY_STATUS_INTERVAL = "statusInterval"; + + public final static String KEY_LSN = "lsn"; + + public final static String KEY_SLOT_NAME = "slotName"; + + public final static String KEY_ALLOW_CREATE_SLOT = "allowCreateSlot"; + + public final static String KEY_TEMPORARY = "temporary"; +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java new file mode 100644 index 0000000000..eaafefd8b5 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import akka.stream.impl.FlowNames; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.TelnetUtil; +import org.postgresql.PGProperty; +import org.postgresql.core.ServerVersion; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.replication.ReplicationSlotInfo; +import org.postgresql.replication.fluent.logical.ChainedLogicalCreateSlotBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalUtil { + + private static final Logger LOG = LoggerFactory.getLogger(PgWalUtil.class); + + public static final String DRIVER = "org.postgresql.Driver"; + public static final String SLOT_PRE = "flinkx_"; + + public static final String QUERY_LEVEL = "show wal_level;"; + public static final String QUERY_MAX_SLOT = "show max_replication_slots;"; + public static final String QUERY_SLOT = "SELECT * FROM pg_replication_slots;"; + public static final String QUERY_TABLE_REPLICA_IDENTITY = "SELECT relreplident FROM pg_catalog.pg_class c LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid WHERE n.nspname='%s' and c.relname='%s';"; + public static final String UPDATE_REPLICA_IDENTITY = "ALTER TABLE %s REPLICA IDENTITY FULL;"; + + public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCreateSlot, String slotName, List tableList) throws Exception{ + ResultSet resultSet; + PgRelicationSlot availableSlot = null; + + //1. check postgres version + // this Judge maybe not need? + if (!conn.haveMinimumServerVersion(ServerVersion.v10)){ + String version = conn.getDBVersionNumber(); + LOG.error("postgres version must > 10, current = [{}]", version); + throw new UnsupportedOperationException("postgres version must >= 10, current = " + version); + } + + //2. check postgres wal_level + resultSet = conn.execSQLQuery(QUERY_LEVEL); + resultSet.next(); + String wal_level = resultSet.getString(1); + if(!"logical".equalsIgnoreCase(wal_level)){ + LOG.error("postgres wal_level must be logical, current = [{}]", wal_level); + throw new UnsupportedOperationException("postgres wal_level must be logical, current = " + wal_level); + } + + //3.check postgres slot + resultSet = conn.execSQLQuery(QUERY_MAX_SLOT); + resultSet.next(); + int maxSlot = resultSet.getInt(1); + int slotCount = 0; + resultSet = conn.execSQLQuery(QUERY_SLOT); + while(resultSet.next()){ + PgRelicationSlot slot = new PgRelicationSlot(); + String name = resultSet.getString(1); + slot.setSlotName(name); + slot.setPlugin(resultSet.getString(2)); + slot.setSlotType(resultSet.getString(3)); + slot.setDatoid(resultSet.getInt(4)); + slot.setDatabase(resultSet.getString(5)); + slot.setTemporary(resultSet.getString(6)); + slot.setActive(resultSet.getString(7)); + slot.setActivePid(resultSet.getInt(8)); + slot.setXmin(resultSet.getString(9)); + slot.setRestartLsn(resultSet.getString(10)); + slot.setConfirmedFlushLsn(resultSet.getString(11)); + + if(name.equalsIgnoreCase(slotName) && slot.isNotActive()){ + availableSlot = slot; + break; + } + slotCount++; + } + + if(availableSlot == null){ + if(!allowCreateSlot){ + String msg = String.format("there is no available slot named [%s], please check whether slotName[%s] is correct, or set allowCreateSlot = true", slotName, slotName); + LOG.error(msg); + throw new UnsupportedOperationException(msg); + }else if(slotCount >= maxSlot){ + LOG.error("the number of slot reaches max_replication_slots[{}], please turn up max_replication_slots or remove unused slot", maxSlot); + throw new UnsupportedOperationException("the number of slot reaches max_replication_slots[" + maxSlot + "], please turn up max_replication_slots or remove unused slot"); + } + } + + //4.check table replica identity + for (String table : tableList) { + //schema.tableName + String[] tables = table.split("."); + resultSet = conn.execSQLQuery(String.format(QUERY_TABLE_REPLICA_IDENTITY, tables[0], tables[1])); + resultSet.next(); + String identity = parseReplicaIdentity(resultSet.getString(1)); + if(!"full".equals(identity)){ + LOG.warn("update {} replica identity, set {} to full", table, identity); + conn.createStatement().execute(String.format(UPDATE_REPLICA_IDENTITY, table)); + } + } + + return availableSlot; + } + + public static PgRelicationSlot createSlot(PgConnection conn, String slotName, boolean temporary) throws SQLException{ + PgRelicationSlot slot = new PgRelicationSlot(); + ChainedLogicalCreateSlotBuilder builder = conn.getReplicationAPI() + .createReplicationSlot() + .logical() + .withSlotName(slotName) + .withOutputPlugin("pgoutput"); + if(temporary){ + builder.withTemporaryOption(); + } + ReplicationSlotInfo replicationSlotInfo = builder.make(); + return slot; + } + + public static String parseReplicaIdentity(String s) { + switch (s) { + case "n": + return "nothing"; + case "d": + return "default"; + case "i" : + return "index"; + case "f" : + return "full"; + default: + return "unknown"; + } + } + + /** + * 获取jdbc连接(超时10S) + * @param url url + * @param username 账号 + * @param password 密码 + * @return + * @throws SQLException + */ + public static PgConnection getConnection(String url, String username, String password) throws SQLException { + Connection dbConn; + ClassUtil.forName(PgWalUtil.DRIVER, PgWalUtil.class.getClassLoader()); + Properties props = new Properties(); + PGProperty.USER.set(props, username); + PGProperty.PASSWORD.set(props, password); + PGProperty.REPLICATION.set(props, "database"); + PGProperty.PREFER_QUERY_MODE.set(props, "simple"); + //postgres version must > 10 + PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "10"); + synchronized (ClassUtil.lock_str) { + DriverManager.setLoginTimeout(10); + + // telnet + TelnetUtil.telnet(url); + dbConn = DriverManager.getConnection(url, props); + } + + return dbConn.unwrap(PgConnection.class); + } + + /** + * 关闭连接资源 + * + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection + * @param commit + */ + public static void closeDBResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { + if (null != rs) { + try { + rs.close(); + } catch (SQLException e) { + LOG.warn("Close resultSet error: {}", ExceptionUtil.getErrorMessage(e)); + } + } + + if (null != stmt) { + try { + stmt.close(); + } catch (SQLException e) { + LOG.warn("Close statement error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + + if (null != conn) { + try { + if (commit && !conn.isClosed()) { + conn.commit(); + } + + conn.close(); + } catch (SQLException e) { + LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + } + +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml new file mode 100644 index 0000000000..d2630bba2c --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -0,0 +1,69 @@ + + + + flinkx-pgwal + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-pgwal-reader + + + + com.dtstack.flinkx + flinkx-pgwal-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java new file mode 100644 index 0000000000..a6649b91cc --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.format; + +import com.dtstack.flinkx.inputformat.RichInputFormat; +import com.dtstack.flinkx.pgwal.PgRelicationSlot; +import com.dtstack.flinkx.pgwal.PgWalUtil; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.postgresql.PGConnection; +import org.postgresql.PGProperty; +import org.postgresql.jdbc.PgConnection; + +import java.io.IOException; +import java.sql.Connection; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.SynchronousQueue; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalInputFormat extends RichInputFormat { + protected String username; + protected String password; + protected String url; + protected String databaseName; + protected boolean pavingData = false; + protected List tableList; + protected String cat; + protected Integer statusInterval; + protected String lsn; + protected String slotName; + protected boolean allowCreateSlot; + protected boolean temporary; + + private PgConnection conn; + + private transient BlockingQueue> queue; + private transient ExecutorService executor; + private volatile boolean running = false; + + @Override + public void configure(Configuration parameters) { + executor = Executors.newFixedThreadPool(1); + queue = new SynchronousQueue<>(true); + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + if (inputSplit.getSplitNumber() != 0) { + LOG.info("PgWalInputFormat openInternal split number:{} abort...", inputSplit.getSplitNumber()); + return; + } + LOG.info("PgWalInputFormat openInternal split number:{} start...", inputSplit.getSplitNumber()); + try { + conn = PgWalUtil.getConnection(url, username, password); + conn.setAutoCommit(false); + if(StringUtils.isBlank(slotName)){ + slotName = PgWalUtil.SLOT_PRE + jobId; + } + PgRelicationSlot availableSlot = PgWalUtil.checkPostgres(conn, allowCreateSlot, slotName, tableList); + + }catch (Exception e){ + LOG.error("PgWalInputFormat open() failed, e = {}", ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException("PgWalInputFormat open() failed, e = " + ExceptionUtil.getErrorMessage(e)); + } + LOG.info("PgWalInputFormat[{}]open: end", jobName); + + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + try { + Map map = queue.take(); + if(map.size() == 1){ + throw new IOException((String) map.get("e")); + }else{ + row = Row.of(map); + } + } catch (InterruptedException e) { + LOG.error("takeEvent interrupted error:{}", ExceptionUtil.getErrorMessage(e)); + } + return row; + + } + + @Override + protected void closeInternal() throws IOException { + if (running) { + executor.shutdownNow(); + running = false; + LOG.warn("shutdown SqlServerCdcListener......"); + } + + } + + + + @Override + public InputSplit[] createInputSplits(int minNumSplits) throws IOException { + return new InputSplit[0]; + } + + @Override + public boolean reachedEnd() throws IOException { + return false; + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java new file mode 100644 index 0000000000..2a0ab5efc3 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with format work for additional information + * regarding copyright ownership. The ASF licenses format file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use format file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.format; + +import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalInputFormatBuilder extends RichInputFormatBuilder { + + protected PgWalInputFormat format; + + public PgWalInputFormatBuilder() { + super.format = this.format = new PgWalInputFormat(); + } + + public void setUsername(String username) { + format.username = username; + } + + public void setPassword(String password) { + format.password = password; + } + + public void setUrl(String url) { + format.url = url; + } + + public void setDatabaseName(String databaseName) { + format.databaseName = databaseName; + } + + public void setPavingData(boolean pavingData) { + format.pavingData = pavingData; + } + + public void setTableList(List tableList) { + format.tableList = tableList; + } + + public void setCat(String cat) { + format.cat = cat; + } + + public void setStatusInterval(Integer statusInterval) { + format.statusInterval = statusInterval; + } + + public void setLsn(String lsn) { + format.lsn = lsn; + } + + public void setAllowCreateSlot(Boolean allowCreateSlot) { + format.allowCreateSlot = allowCreateSlot; + } + + public void setSlotName(String slotName) { + format.slotName = slotName; + } + + public void setTemporary(Boolean temporary) { + format.temporary = temporary; + } + + @Override + protected void checkFormat() { + if (StringUtils.isBlank(format.username)) { + throw new IllegalArgumentException("No username supplied"); + } + if (StringUtils.isBlank(format.password)) { + throw new IllegalArgumentException("No password supplied"); + } + if (StringUtils.isBlank(format.url)) { + throw new IllegalArgumentException("No url supplied"); + } + if (StringUtils.isBlank(format.databaseName)) { + throw new IllegalArgumentException("No databaseName supplied"); + } + if (CollectionUtils.isEmpty(format.tableList)) { + throw new IllegalArgumentException("No tableList supplied"); + } + if (StringUtils.isBlank(format.cat)) { + throw new IllegalArgumentException("No cat supplied"); + } + if(!format.allowCreateSlot && StringUtils.isBlank(format.slotName)){ + throw new IllegalArgumentException("slotName can not be null if allowCreateSlot is false"); + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java new file mode 100644 index 0000000000..c1d22d3f29 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.pgwal.PgWalConfigKeys; +import com.dtstack.flinkx.pgwal.format.PgWalInputFormatBuilder; +import com.dtstack.flinkx.reader.DataReader; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalReader extends DataReader { + private String username; + private String password; + private String url; + private String databaseName; + private String cat; + private boolean pavingData; + private List tableList; + private Integer statusInterval; + private String lsn; + private String slotName; + private boolean allowCreateSlot; + private boolean temporary; + + @SuppressWarnings("unchecked") + protected PgWalReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + username = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_USER_NAME); + password = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_PASSWORD); + url = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_URL); + databaseName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_DATABASE_NAME); + cat = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_CATALOG); + pavingData = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_PAVING_DATA, false); + tableList = (List) readerConfig.getParameter().getVal(PgWalConfigKeys.KEY_TABLE_LIST); + statusInterval = readerConfig.getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 10000); + lsn = readerConfig.getStringVal(PgWalConfigKeys.KEY_LSN); + slotName = readerConfig.getStringVal(PgWalConfigKeys.KEY_SLOT_NAME); + allowCreateSlot = readerConfig.getBooleanVal(PgWalConfigKeys.KEY_ALLOW_CREATE_SLOT, true); + temporary = readerConfig.getBooleanVal(PgWalConfigKeys.KEY_TEMPORARY, true); + } + + @Override + public DataStream readData() { + PgWalInputFormatBuilder builder = new PgWalInputFormatBuilder(); + builder.setUsername(username); + builder.setPassword(password); + builder.setUrl(url); + builder.setDatabaseName(databaseName); + builder.setCat(cat); + builder.setPavingData(pavingData); + builder.setTableList(tableList); + builder.setRestoreConfig(restoreConfig); + builder.setStatusInterval(statusInterval); + builder.setLsn(lsn); + builder.setSlotName(slotName); + builder.setAllowCreateSlot(allowCreateSlot); + builder.setTemporary(temporary); + return createInput(builder.finish(), "pgwalreader"); + } +} diff --git a/flinkx-pgwal/pom.xml b/flinkx-pgwal/pom.xml new file mode 100644 index 0000000000..b701ba13ca --- /dev/null +++ b/flinkx-pgwal/pom.xml @@ -0,0 +1,29 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + pom + + flinkx-pgwal + + + flinkx-pgwal-core + flinkx-pgwal-reader + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 132d4056cc..df318d098c 100644 --- a/pom.xml +++ b/pom.xml @@ -11,34 +11,44 @@ flinkx-core - flinkx-rdb - flinkx-es - flinkx-ftp - flinkx-odps - flinkx-examples - flinkx-mysql - flinkx-hbase - flinkx-oracle - flinkx-sqlserver - flinkx-redis - flinkx-mongodb - flinkx-postgresql - flinkx-stream - flinkx-carbondata - flinkx-hdfs - flinkx-hive - flinkx-db2 - flinkx-test - flinkx-gbase - flinkx-binlog - flinkx-kafka09 - flinkx-kafka10 - flinkx-kafka11 - flinkx-kafka - flinkx-kudu + flinkx-launcher - flinkx-clickhouse - flinkx-polardb + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + flinkx-pgwal + + @@ -65,13 +75,6 @@ provided - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - provided - - org.apache.hadoop hadoop-yarn-common From 576cafec9fb09e53221f44fc6fb6f3bc9afd8a46 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 16 Dec 2019 16:41:54 +0800 Subject: [PATCH 004/136] =?UTF-8?q?=E6=96=B0=E5=A2=9EpgWal=E5=AE=9E?= =?UTF-8?q?=E6=97=B6=E9=87=87=E9=9B=86=E6=8F=92=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/options/Options.java | 35 ++- flinkx-examples/examples/pgwal_to_stream.json | 42 +++ flinkx-launcher/pom.xml | 43 ++- .../com/dtstack/flinkx/launcher/Launcher.java | 24 +- .../perJob/PerJobClusterClientBuilder.java | 54 ++-- .../launcher/perJob/PerJobSubmitter.java | 4 +- .../com/dtstack/flinkx/pgwal/PgDecoder.java | 275 ++++++++++++++++++ .../flinkx/pgwal/PgMessageTypeEnum.java | 51 ++++ .../com/dtstack/flinkx/pgwal/PgWalUtil.java | 63 ++-- .../java/com/dtstack/flinkx/pgwal/Table.java | 124 ++++++++ .../flinkx/pgwal/format/PgWalInputFormat.java | 87 +++++- .../pgwal/format/PgWalInputFormatBuilder.java | 2 +- .../flinkx/pgwal/listener/PgWalListener.java | 148 ++++++++++ .../flinkx/pgwal/reader/PgwalReader.java | 88 ++++++ flinkx-test/pom.xml | 6 + .../com/dtstack/flinkx/test/LocalTest.java | 4 +- .../flinkx/test/PluginNameConstrant.java | 1 + pom.xml | 58 ++-- 18 files changed, 995 insertions(+), 114 deletions(-) create mode 100644 flinkx-examples/examples/pgwal_to_stream.json create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index a2b5ae2786..671c69977c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -28,22 +28,22 @@ */ public class Options { - @OptionRequired(description = "Running mode") - private String mode = ClusterMode.local.name(); + @OptionRequired(description = "Running mode") + private String mode = ClusterMode.local.name(); - @OptionRequired(required = true,description = "Job config") - private String job; + @OptionRequired(required = true, description = "Job config") + private String job; - @OptionRequired(description = "Monitor Addresses") - private String monitor; + @OptionRequired(description = "Monitor Addresses") + private String monitor; - @OptionRequired(description = "Job unique id") - private String jobid = "Flink Job"; + @OptionRequired(description = "Job unique id") + private String jobid = "Flink Job"; @OptionRequired(description = "Flink configuration directory") private String flinkconf; - @OptionRequired(required = true,description = "env properties") + @OptionRequired(required = true, description = "env properties") private String pluginRoot; @OptionRequired(description = "Yarn and Hadoop configuration directory") @@ -70,6 +70,9 @@ public class Options { @OptionRequired(description = "savepoint path") private String s; + @OptionRequired(description = "plugin load mode, by classpath or shipfile") + private String pluginLoadMode = "classpath"; + public String getS() { return s; } @@ -173,4 +176,12 @@ public String getFlinkLibJar() { public void setFlinkLibJar(String flinkLibJar) { this.flinkLibJar = flinkLibJar; } + + public String getPluginLoadMode() { + return pluginLoadMode; + } + + public void setPluginLoadMode(String pluginLoadMode) { + this.pluginLoadMode = pluginLoadMode; + } } diff --git a/flinkx-examples/examples/pgwal_to_stream.json b/flinkx-examples/examples/pgwal_to_stream.json new file mode 100644 index 0000000000..86924263e5 --- /dev/null +++ b/flinkx-examples/examples/pgwal_to_stream.json @@ -0,0 +1,42 @@ +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "username" : "postgres", + "password" : "abc123", + "url" : "jdbc:postgresql://172.16.8.122:5432/postgres", + "databaseName" : "postgres", + "cat" : "update,insert,delete", + "tableList" : [ + "changepk.test_table" + ], + "statusInterval" : 10000, + "lsn" : 0, + "slotName" : "", + "allowCreateSlot" : true, + "temporary" : true, + "pavingData" : true + }, + "name" : "pgwalreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true + }, + "errorLimit" : { }, + "speed" : { + "bytes" : -1048576, + "channel" : 1 + } + } + } +} \ No newline at end of file diff --git a/flinkx-launcher/pom.xml b/flinkx-launcher/pom.xml index 9c752548ab..5cb498c3e0 100644 --- a/flinkx-launcher/pom.xml +++ b/flinkx-launcher/pom.xml @@ -40,20 +40,45 @@ commons-codec 1.9 + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + + org.apache.hadoop hadoop-common - 2.7.3.2.6.4.91-3 + ${hadoop.version} + + + + org.apache.hadoop + hadoop-yarn-common + ${hadoop.version} + + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop.version} org.apache.hadoop hadoop-mapreduce-client-core - 2.7.3.2.6.4.91-3 + ${hadoop.version} org.apache.hadoop - hadoop-hdfs - 2.7.3.2.6.4.91-3 + hadoop-yarn-api + ${hadoop.version} + + + org.apache.avro + avro + 1.8.2 @@ -79,7 +104,7 @@ - reference.conf + reference.conf @@ -87,19 +112,19 @@ - core-site.xml + core-site.xml - yarn-default.xml + yarn-default.xml - mapred-default.xml + mapred-default.xml - mapred-site.xml + mapred-site.xml diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index acf2d38c65..46bd2d3936 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,6 +18,8 @@ package com.dtstack.flinkx.launcher; +import ch.qos.logback.classic.Level; +import ch.qos.logback.classic.LoggerContext; import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.enums.ClusterMode; @@ -35,6 +37,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.Preconditions; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileInputStream; @@ -84,6 +87,7 @@ private static List analyzeUserClasspath(String content, String pluginRoot) } public static void main(String[] args) throws Exception { + setLogLevel(Level.INFO.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); String mode = launcherOptions.getMode(); @@ -135,17 +139,6 @@ public static void main(String[] args) throws Exception { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration conf = GlobalConfiguration.loadConfiguration(flinkConfDir); JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, conf, Integer.parseInt(launcherOptions.getParallelism())); - - File[] jars = new File(launcherOptions.getFlinkLibJar()).listFiles(); - if(jars != null){ - for (File jar : jars) { - URL url = jar.toURI().toURL(); - if(!url.toString().contains("flink-dist")){ - jobGraph.addJar(new Path(url.toString())); - } - } - } - PerJobSubmitter.submit(launcherOptions, jobGraph); } } @@ -186,4 +179,11 @@ private static String readJob(String job) { throw new RuntimeException(e); } } + + private static void setLogLevel(String level){ + LoggerContext loggerContext= (LoggerContext) LoggerFactory.getILoggerFactory(); + //设置全局日志级别 + ch.qos.logback.classic.Logger logger=loggerContext.getLogger("root"); + logger.setLevel(Level.toLevel(level)); + } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index 3eb4be9f73..8f607d497b 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + *

* http://www.apache.org/licenses/LICENSE-2.0 - * + *

* Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -18,9 +18,12 @@ package com.dtstack.flinkx.launcher.perJob; import com.dtstack.flinkx.launcher.YarnConfLoader; +import com.dtstack.flinkx.options.Options; import com.google.common.base.Strings; import org.apache.commons.lang.StringUtils; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.fs.Path; @@ -32,7 +35,7 @@ import java.io.File; import java.net.MalformedURLException; import java.net.URL; -import java.util.Properties; +import java.util.*; /** * Date: 2019/09/11 @@ -49,7 +52,7 @@ public class PerJobClusterClientBuilder { * init yarnClient * @param yarnConfDir the path of yarnconf */ - public void init(String yarnConfDir){ + public void init(String yarnConfDir) { if (Strings.isNullOrEmpty(yarnConfDir)) { throw new RuntimeException("param:[yarnconf] is required !"); } @@ -62,38 +65,49 @@ public void init(String yarnConfDir){ /** * create a yarn cluster descriptor which is used to start the application master - * @param confProp taskParams - * @param flinkJarPath the path of flink jar lib - * @param queue queue name + * @param confProp taskParams + * @param options LauncherOptions + * @param jobGraph JobGraph * @return * @throws MalformedURLException */ - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { - if(StringUtils.isNotBlank(flinkJarPath)){ - if(!new File(flinkJarPath).exists()){ + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options options, JobGraph jobGraph) throws MalformedURLException { + String flinkJarPath = options.getFlinkLibJar(); + if (StringUtils.isNotBlank(flinkJarPath)) { + if (!new File(flinkJarPath).exists()) { throw new IllegalArgumentException("The Flink jar path is not exist"); } - }else{ + } else { throw new IllegalArgumentException("The Flink jar path is null"); } Configuration conf = new Configuration(); confProp.forEach((key, value) -> conf.setString(key.toString(), value.toString())); - AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(conf, yarnConf, ".", yarnClient, false); + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(conf, yarnConf, options.getFlinkconf(), yarnClient, false); + List shipFiles = new ArrayList<>(); File[] jars = new File(flinkJarPath).listFiles(); - if(jars != null){ + if (jars != null) { for (File jar : jars) { - URL url = jar.toURI().toURL(); - if(url.toString().contains("flink-dist")){ - descriptor.setLocalJarPath(new Path(url.toString())); - break; + if (jar.toURI().toURL().toString().contains("flink-dist")) { + descriptor.setLocalJarPath(new Path(jar.toURI().toURL().toString())); + } else { + shipFiles.add(jar); + } + } + } + if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + shipFiles.add(new File(tmp.getValue().filePath)); } } } - if(StringUtils.isNotBlank(queue)){ - descriptor.setQueue(queue); + if (StringUtils.isNotBlank(options.getQueue())) { + descriptor.setQueue(options.getQueue()); } + descriptor.addShipFiles(shipFiles); return descriptor; } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java index 7696cbb27b..8bc3e78c13 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -52,7 +52,7 @@ public static String submit(Options options, JobGraph jobGraph) throws Exception PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); perJobClusterClientBuilder.init(options.getYarnconf()); - AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options.getFlinkLibJar(), options.getQueue()); + AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options, jobGraph); ClusterClient clusterClient = descriptor.deployJobCluster(clusterSpecification, jobGraph, true); String applicationId = clusterClient.getClusterId().toString(); String flinkJobId = jobGraph.getJobID().toString(); diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java new file mode 100644 index 0000000000..13d895196b --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import com.dtstack.flinkx.reader.MetaColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.sql.SQLException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * reference to https://github.com/debezium/debezium & http://www.postgres.cn/docs/10/protocol-logicalrep-message-formats.html + * + * @author tudou + */ +public class PgDecoder { + private static final Logger LOG = LoggerFactory.getLogger(PgDecoder.class); + + private static Instant PG_EPOCH = LocalDate.of(2000, 1, 1).atStartOfDay().toInstant(ZoneOffset.UTC); + + private Map tableMap = new HashMap<>(64); + private Map pgTypeMap; + private volatile long currentLsn; + private volatile long ts; + + public PgDecoder(Map pgTypeMap) { + this.pgTypeMap = pgTypeMap; + } + + public Table decode(ByteBuffer buffer) throws SQLException { + Table table = new Table(); + PgMessageTypeEnum type = PgMessageTypeEnum.forType((char) buffer.get()); + switch (type) { + case BEGIN: + //Byte1('B') 将消息标识为开始消息 + handleBeginMessage(buffer); + break; + case COMMIT: + //Byte1('C') 将消息标识为提交消息 + handleCommitMessage(buffer); + break; + case RELATION: + //Byte1('R') 将消息标识为关系消息 + handleRelationMessage(buffer); + break; + case INSERT: + //Byte1('I') 将消息标识为插入消息 + table = decodeInsert(buffer); + break; + case UPDATE: + //Byte1('U') 将消息标识为更新消息 + table = decodeUpdate(buffer); + break; + case DELETE: + //Byte1('D') 将消息标识为删除消息 + table = decodeDelete(buffer); + break; + default: + break; + } + table.setType(type); + return table; + } + + private void handleBeginMessage(ByteBuffer buffer) { + //Int64 事务的结束LSN + long lsn = buffer.getLong(); + //Int64 提交事务的时间戳。自PostgreSQL纪元(2000-01-01)以来的数值是微秒数 + Instant plus = PG_EPOCH.plus(buffer.getLong(), ChronoUnit.MICROS); + //Int32 事务的Xid + int anInt = buffer.getInt(); + currentLsn = lsn; + ts = plus.toEpochMilli(); + LOG.trace("handleBeginMessage result = { lsn = {}, plus = {}, anInt = {}}", lsn, plus, anInt); + } + + private void handleCommitMessage(ByteBuffer buffer) { + if(LOG.isTraceEnabled()){ + //Int8 标志;目前未使用(必须为0) + int flags = buffer.get(); + //Int64 提交的LSN + long lsn = buffer.getLong(); + //Int64 事务的结束LSN + long endLsn = buffer.getLong(); + //Int64 提交事务的时间戳。自PostgreSQL纪元(2000-01-01)以来的数值是微秒数 + Instant commitTimestamp = PG_EPOCH.plus(buffer.getLong(), ChronoUnit.MICROS); + LOG.trace("handleCommitMessage result = { flags = {}, lsn = {}, endLsn = {}, commitTimestamp = {}}", flags, lsn, endLsn, commitTimestamp); + } + } + + private void handleRelationMessage(ByteBuffer buffer) throws SQLException { + //Int32 关系的ID + int relationId = buffer.getInt(); + //String 命名空间(pg_catalog的空字符串) + String schemaName = readString(buffer); + //String 关系名称 + String tableName = readString(buffer); + //Int8 该关系的副本标识设置(与pg_class 中的relreplident相同) + int replicaIdentityId = buffer.get(); + //Int16 列数 + short columnCount = buffer.getShort(); + LOG.debug("handleRelationMessage result = { schemaName = {}, tableName = {}}", schemaName, tableName); + if(!tableMap.containsKey(relationId)){ + List columnList = new ArrayList<>(columnCount); + for (int i = 0; i < columnCount; i++) { + //Int8 列的标志。当前可以是0表示没有标记或1表示将列标记为关键字的一部分 + byte flags = buffer.get(); + //String 列的名称 + String name = unquoteIdentifierPart(readString(buffer)); + //Int32 列的数据类型的ID + String type = pgTypeMap.get(buffer.getInt()); + MetaColumn metaColumn = new MetaColumn(); + metaColumn.setIndex(i); + metaColumn.setName(name); + metaColumn.setType(type); + columnList.add(metaColumn); + //Int32 列的类型修饰符(atttypmod) + int attypmod = buffer.getInt(); + } + Table table = new Table(schemaName, tableName, columnList); + tableMap.put(relationId, table); + } + } + + private Table decodeInsert(ByteBuffer buffer) { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + //Byte1('N') 将以下TupleData消息标识为新元组 + char tupleType = (char) buffer.get(); + //TupleData TupleData消息部分表示新元组的内容 + Object[] newData = resolveColumnsFromStreamTupleData(buffer); + Table table = tableMap.get(relationId); + table.setOldData(new Object[newData.length]); + table.setNewData(newData); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + private Table decodeUpdate(ByteBuffer buffer) throws SQLException { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + Table table = tableMap.get(relationId); + //Byte1('K') 将以下TupleData子消息标识为键。该字段是可选的, 并且只有在更新改变了REPLICA IDENTITY索引一部分的任何一列中的数据时才存在 + //Byte1('O') 将以下TupleData子消息标识为旧元组。此字段是可选的, 并且仅当发生更新的表的REPLICA IDENTITY设置为FULL时才存在 + //更新消息可以包含'K'消息部分或者'O'消息部分或者都不包含它们,但不同时包括它们两者 + char tupleType = (char) buffer.get(); + if ('O' == tupleType || 'K' == tupleType) { + //TupleData TupleData消息部分表示旧元组或主键的内容。 只有在前面的'O'或'K'部分存在时才存在 + Object[] oldData = resolveColumnsFromStreamTupleData(buffer); + table.setOldData(oldData); + // Read the 'N' tuple type + // This is necessary so the stream position is accurate for resolving the column tuple data + //Byte1('N') 将以下TupleData消息标识为新元组 + tupleType = (char) buffer.get(); + } + //TupleData TupleData消息部分表示新元组的内容 + Object[] newData = resolveColumnsFromStreamTupleData(buffer); + table.setNewData(newData); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + private Table decodeDelete(ByteBuffer buffer) throws SQLException { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + Table table = tableMap.get(relationId); + //Byte1('K') 将以下TupleData子消息标识为键。 如果发生删除的表使用索引作为REPLICA IDENTITY,则此字段存在 + //Byte1('O') 将以下TupleData消息标识为旧元组。 如果发生删除的表的REPLICA IDENTITY设置为FULL,则此字段存在 + //删除消息可能包含'K'消息部分或'O'消息部分,但不会同时包含这两个部分 + char tupleType = (char) buffer.get(); + //TupleData TupleData消息部分,表示旧元组或主键的内容,具体取决于前一个字段 + Object[] oldData = resolveColumnsFromStreamTupleData(buffer); + table.setOldData(oldData); + table.setNewData(new Object[oldData.length]); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + + private Object[] resolveColumnsFromStreamTupleData(ByteBuffer buffer) { + //Int16 列数 + short numberOfColumns = buffer.getShort(); + Object[] data = new Object[numberOfColumns]; + for (int i = 0; i < numberOfColumns; i++) { + + //Byte1('n') 将数据标识为NULL值 + //Byte1('u') 识别未更改的TOASTed值(实际值未发送) + //Byte1('t') 将数据标识为文本格式的值 + char type = (char) buffer.get(); + if (type == 't') { + data[i] = readColumnValueAsString(buffer); + } else if (type == 'u') { + data[i] = null; + } else if (type == 'n') { + data[i] = null; + } + } + return data; + } + + private static String readColumnValueAsString(ByteBuffer buffer) { + //Int32 列值的长度 + int length = buffer.getInt(); + byte[] value = new byte[length]; + //Byte(n) 该列的值,以文本格式显示。n是上面的长度 + buffer.get(value, 0, length); + return new String(value); + } + + private static String readString(ByteBuffer buffer) { + StringBuilder sb = new StringBuilder(); + byte b = 0; + while ((b = buffer.get()) != 0) { + sb.append((char) b); + } + return sb.toString(); + } + + public static String unquoteIdentifierPart(String identifierPart) { + if (identifierPart == null || identifierPart.length() < 2) { + return identifierPart; + } + + Character quotingChar = deriveQuotingChar(identifierPart); + if (quotingChar != null) { + identifierPart = identifierPart.substring(1, identifierPart.length() - 1); + identifierPart = identifierPart.replace(quotingChar.toString() + quotingChar.toString(), quotingChar.toString()); + } + + return identifierPart; + } + + private static Character deriveQuotingChar(String identifierPart) { + char first = identifierPart.charAt(0); + char last = identifierPart.charAt(identifierPart.length() - 1); + + if (first == last && (first == '"' || first == '\'' || first == '`')) { + return first; + } + + return null; + } + + +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java new file mode 100644 index 0000000000..c12a614149 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * reference to https://github.com/debezium/debezium & http://www.postgres.cn/docs/10/protocol-logicalrep-message-formats.html + * + * @author tudou + */ +public enum PgMessageTypeEnum { + RELATION, + BEGIN, + COMMIT, + INSERT, + UPDATE, + DELETE, + TYPE, + ORIGIN; + + public static PgMessageTypeEnum forType(char type) { + switch (type) { + case 'R': return RELATION; + case 'B': return BEGIN; + case 'C': return COMMIT; + case 'I': return INSERT; + case 'U': return UPDATE; + case 'D': return DELETE; + case 'Y': return TYPE; + case 'O': return ORIGIN; + default: throw new IllegalArgumentException("Unsupported message type: " + type); + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java index eaafefd8b5..4dfb415795 100644 --- a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java @@ -17,7 +17,6 @@ */ package com.dtstack.flinkx.pgwal; -import akka.stream.impl.FlowNames; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.TelnetUtil; @@ -47,12 +46,16 @@ public class PgWalUtil { public static final String DRIVER = "org.postgresql.Driver"; public static final String SLOT_PRE = "flinkx_"; + public static final String PUBLICATION_NAME = "dtstack_flinkx"; - public static final String QUERY_LEVEL = "show wal_level;"; - public static final String QUERY_MAX_SLOT = "show max_replication_slots;"; + public static final String QUERY_LEVEL = "SHOW wal_level;"; + public static final String QUERY_MAX_SLOT = "SHOW max_replication_slots;"; public static final String QUERY_SLOT = "SELECT * FROM pg_replication_slots;"; public static final String QUERY_TABLE_REPLICA_IDENTITY = "SELECT relreplident FROM pg_catalog.pg_class c LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid WHERE n.nspname='%s' and c.relname='%s';"; public static final String UPDATE_REPLICA_IDENTITY = "ALTER TABLE %s REPLICA IDENTITY FULL;"; + public static final String QUERY_PUBLICATION = "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s';"; + public static final String CREATE_PUBLICATION = "CREATE PUBLICATION %s FOR ALL TABLES;"; + public static final String QUERY_TYPES = "SELECT t.oid AS oid, t.typname AS name FROM pg_catalog.pg_type t JOIN pg_catalog.pg_namespace n ON (t.typnamespace = n.oid) WHERE n.nspname != 'pg_toast' AND t.typcategory <> 'A';"; public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCreateSlot, String slotName, List tableList) throws Exception{ ResultSet resultSet; @@ -83,20 +86,21 @@ public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCre resultSet = conn.execSQLQuery(QUERY_SLOT); while(resultSet.next()){ PgRelicationSlot slot = new PgRelicationSlot(); - String name = resultSet.getString(1); + String name = resultSet.getString("slot_name"); slot.setSlotName(name); - slot.setPlugin(resultSet.getString(2)); - slot.setSlotType(resultSet.getString(3)); - slot.setDatoid(resultSet.getInt(4)); - slot.setDatabase(resultSet.getString(5)); - slot.setTemporary(resultSet.getString(6)); - slot.setActive(resultSet.getString(7)); - slot.setActivePid(resultSet.getInt(8)); - slot.setXmin(resultSet.getString(9)); - slot.setRestartLsn(resultSet.getString(10)); - slot.setConfirmedFlushLsn(resultSet.getString(11)); + slot.setActive(resultSet.getString("active")); if(name.equalsIgnoreCase(slotName) && slot.isNotActive()){ + slot.setPlugin(resultSet.getString("plugin")); + slot.setSlotType(resultSet.getString("slot_type")); + slot.setDatoid(resultSet.getInt("datoid")); + slot.setDatabase(resultSet.getString("database")); + slot.setTemporary(resultSet.getString("temporary")); + slot.setActivePid(resultSet.getInt("active_pid")); + slot.setXmin(resultSet.getString("xmin")); + slot.setCatalogXmin(resultSet.getString("catalog_xmin")); + slot.setRestartLsn(resultSet.getString("restart_lsn")); + slot.setConfirmedFlushLsn(resultSet.getString("confirmed_flush_lsn")); availableSlot = slot; break; } @@ -117,7 +121,7 @@ public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCre //4.check table replica identity for (String table : tableList) { //schema.tableName - String[] tables = table.split("."); + String[] tables = table.split("\\."); resultSet = conn.execSQLQuery(String.format(QUERY_TABLE_REPLICA_IDENTITY, tables[0], tables[1])); resultSet.next(); String identity = parseReplicaIdentity(resultSet.getString(1)); @@ -127,11 +131,20 @@ public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCre } } + //5.check publication + resultSet = conn.execSQLQuery(String.format(QUERY_PUBLICATION, PUBLICATION_NAME)); + resultSet.next(); + long count = resultSet.getLong(1); + if(count == 0L){ + LOG.warn("no publication named [{}] existed, flinkx will create one", PUBLICATION_NAME); + conn.createStatement().execute(String.format(CREATE_PUBLICATION, PUBLICATION_NAME)); + } + + closeDBResources(resultSet, null, null, false); return availableSlot; } public static PgRelicationSlot createSlot(PgConnection conn, String slotName, boolean temporary) throws SQLException{ - PgRelicationSlot slot = new PgRelicationSlot(); ChainedLogicalCreateSlotBuilder builder = conn.getReplicationAPI() .createReplicationSlot() .logical() @@ -141,9 +154,25 @@ public static PgRelicationSlot createSlot(PgConnection conn, String slotName, bo builder.withTemporaryOption(); } ReplicationSlotInfo replicationSlotInfo = builder.make(); + PgRelicationSlot slot = new PgRelicationSlot(); + slot.setSlotName(slotName); + slot.setConfirmedFlushLsn(replicationSlotInfo.getConsistentPoint().asString()); + slot.setPlugin(replicationSlotInfo.getOutputPlugin()); return slot; } + public static Map queryTypes(PgConnection conn) throws SQLException{ + Map map = new HashMap<>(512); + ResultSet resultSet = conn.execSQLQuery(QUERY_TYPES); + while (resultSet.next()){ + int oid = (int) resultSet.getLong("oid"); + String typeName = resultSet.getString("name"); + map.put(oid, typeName); + } + closeDBResources(resultSet, null, null, false); + return map; + } + public static String parseReplicaIdentity(String s) { switch (s) { case "n": @@ -179,7 +208,6 @@ public static PgConnection getConnection(String url, String username, String pas PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "10"); synchronized (ClassUtil.lock_str) { DriverManager.setLoginTimeout(10); - // telnet TelnetUtil.telnet(url); dbConn = DriverManager.getConnection(url, props); @@ -218,7 +246,6 @@ public static void closeDBResources(ResultSet rs, Statement stmt, Connection con if (commit && !conn.isClosed()) { conn.commit(); } - conn.close(); } catch (SQLException e) { LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java new file mode 100644 index 0000000000..442ae67718 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import com.dtstack.flinkx.reader.MetaColumn; +import org.postgresql.replication.LogSequenceNumber; + +import java.util.List; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * @author tudou + */ +public class Table { + private String id; + private String schema; + private String table; + private List columnList; + private Object[] oldData; + private Object[] newData; + private PgMessageTypeEnum type; + + private long currentLsn; + private long ts; + + public Table(String schema, String table, List columnList) { + this.schema = schema; + this.table = table; + this.columnList = columnList; + this.id = schema + "." + table; + } + + public Table() { + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public List getColumnList() { + return columnList; + } + + public void setColumnList(List columnList) { + this.columnList = columnList; + } + + public Object[] getOldData() { + return oldData; + } + + public void setOldData(Object[] oldData) { + this.oldData = oldData; + } + + public Object[] getNewData() { + return newData; + } + + public void setNewData(Object[] newData) { + this.newData = newData; + } + + public PgMessageTypeEnum getType() { + return type; + } + + public void setType(PgMessageTypeEnum type) { + this.type = type; + } + + public long getCurrentLsn() { + return currentLsn; + } + + public void setCurrentLsn(long currentLsn) { + this.currentLsn = currentLsn; + } + + public long getTs() { + return ts; + } + + public void setTs(long ts) { + this.ts = ts; + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java index a6649b91cc..8555ae2d6c 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -21,21 +21,20 @@ import com.dtstack.flinkx.inputformat.RichInputFormat; import com.dtstack.flinkx.pgwal.PgRelicationSlot; import com.dtstack.flinkx.pgwal.PgWalUtil; -import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.pgwal.listener.PgWalListener; +import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.postgresql.PGConnection; -import org.postgresql.PGProperty; import org.postgresql.jdbc.PgConnection; +import org.postgresql.replication.LogSequenceNumber; import java.io.IOException; -import java.sql.Connection; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -56,12 +55,13 @@ public class PgWalInputFormat extends RichInputFormat { protected List tableList; protected String cat; protected Integer statusInterval; - protected String lsn; + protected Long lsn; protected String slotName; protected boolean allowCreateSlot; protected boolean temporary; private PgConnection conn; + private volatile long startLsn; private transient BlockingQueue> queue; private transient ExecutorService executor; @@ -82,12 +82,21 @@ protected void openInternal(InputSplit inputSplit) throws IOException { LOG.info("PgWalInputFormat openInternal split number:{} start...", inputSplit.getSplitNumber()); try { conn = PgWalUtil.getConnection(url, username, password); - conn.setAutoCommit(false); if(StringUtils.isBlank(slotName)){ slotName = PgWalUtil.SLOT_PRE + jobId; } PgRelicationSlot availableSlot = PgWalUtil.checkPostgres(conn, allowCreateSlot, slotName, tableList); + if(availableSlot == null){ + PgWalUtil.createSlot(conn, slotName, temporary); + } + if(lsn != 0){ + startLsn = lsn; + }else if(formatState != null && formatState.getState() != null){ + startLsn = (long)formatState.getState(); + } + executor.submit(new PgWalListener(this)); + running = true; }catch (Exception e){ LOG.error("PgWalInputFormat open() failed, e = {}", ExceptionUtil.getErrorMessage(e)); throw new RuntimeException("PgWalInputFormat open() failed, e = " + ExceptionUtil.getErrorMessage(e)); @@ -103,6 +112,7 @@ protected Row nextRecordInternal(Row row) throws IOException { if(map.size() == 1){ throw new IOException((String) map.get("e")); }else{ + startLsn = (long) map.get("lsn"); row = Row.of(map); } } catch (InterruptedException e) { @@ -112,6 +122,20 @@ protected Row nextRecordInternal(Row row) throws IOException { } + @Override + public FormatState getFormatState() { + if (!restoreConfig.isRestore()) { + LOG.info("return null for formatState"); + return null; + } + + super.getFormatState(); + if (formatState != null) { + formatState.setState(startLsn); + } + return formatState; + } + @Override protected void closeInternal() throws IOException { if (running) { @@ -122,15 +146,58 @@ protected void closeInternal() throws IOException { } - - @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - return new InputSplit[0]; + InputSplit[] splits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new GenericInputSplit(i, minNumSplits); + } + return splits; } @Override public boolean reachedEnd() throws IOException { return false; } + + public void processEvent(Map event) { + try { + queue.put(event); + } catch (InterruptedException e) { + LOG.error("takeEvent interrupted event:{} error:{}", event, ExceptionUtil.getErrorMessage(e)); + } + } + + + public boolean isPavingData() { + return pavingData; + } + + public List getTableList() { + return tableList; + } + + public String getCat() { + return cat; + } + + public Integer getStatusInterval() { + return statusInterval; + } + + public String getSlotName() { + return slotName; + } + + public PgConnection getConn() { + return conn; + } + + public long getStartLsn() { + return startLsn; + } + + public boolean isRunning() { + return running; + } } diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java index 2a0ab5efc3..e062690308 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java @@ -70,7 +70,7 @@ public void setStatusInterval(Integer statusInterval) { format.statusInterval = statusInterval; } - public void setLsn(String lsn) { + public void setLsn(Long lsn) { format.lsn = lsn; } diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java new file mode 100644 index 0000000000..8e5efb743f --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.listener; + +import com.dtstack.flinkx.pgwal.PgDecoder; +import com.dtstack.flinkx.pgwal.PgWalUtil; +import com.dtstack.flinkx.pgwal.Table; +import com.dtstack.flinkx.pgwal.format.PgWalInputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.replication.PGReplicationStream; +import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalListener implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(PgWalListener.class); + private static Gson gson = new Gson(); + + private PgWalInputFormat format; + private PgConnection conn; + private Set tableSet; + private Set cat; + private boolean pavingData; + + private PGReplicationStream stream; + private PgDecoder decoder; + + public PgWalListener(PgWalInputFormat format) { + this.format = format; + this.conn = format.getConn(); + this.tableSet = new HashSet<>(format.getTableList()); + this.cat = new HashSet<>(); + for (String type : format.getCat().split(",")) { + cat.add(type.toLowerCase()); + } + this.pavingData = format.isPavingData(); + } + + public void init() throws Exception{ + decoder = new PgDecoder(PgWalUtil.queryTypes(conn)); + ChainedLogicalStreamBuilder builder = conn.getReplicationAPI() + .replicationStream() + .logical() + .withSlotName(format.getSlotName()) + //协议版本。当前仅支持版本1 + .withSlotOption("proto_version", "1") + //逗号分隔的要订阅的发布名称列表(接收更改)。 单个发布名称被视为标准对象名称,并可根据需要引用 + .withSlotOption("publication_names", PgWalUtil.PUBLICATION_NAME) + .withStatusInterval(format.getStatusInterval(), TimeUnit.MILLISECONDS); + long lsn = format.getStartLsn(); + if(lsn != 0){ + builder.withStartPosition(LogSequenceNumber.valueOf(lsn)); + } + stream = builder.start(); + TimeUnit.SECONDS.sleep(1); + stream.forceUpdateStatus(); + LOG.info("init PGReplicationStream successfully..."); + } + + @Override + public void run() { + LOG.info("PgWalListener start running....."); + try { + init(); + while (format.isRunning()) { + ByteBuffer buffer = stream.readPending(); + if (buffer == null) { + continue; + } + Table table = decoder.decode(buffer); + if(StringUtils.isBlank(table.getId())){ + continue; + } + String type = table.getType().name().toLowerCase(); + if(!cat.contains(type)){ + continue; + } + if(!tableSet.contains(table.getId())){ + continue; + } + LOG.trace("table = {}",gson.toJson(table)); + Map map = new LinkedHashMap<>(); + map.put("type", type); + map.put("schema", table.getSchema()); + map.put("table", table.getTable()); + map.put("lsn", table.getCurrentLsn()); + map.put("ts", table.getTs()); + map.put("ingestion", System.nanoTime()); + if(pavingData){ + int i = 0; + for (MetaColumn column : table.getColumnList()) { + map.put("before_" + column.getName(), table.getOldData()[i]); + map.put("after_" + column.getName(), table.getNewData()[i]); + i++; + } + }else { + Map before = new LinkedHashMap<>(); + Map after = new LinkedHashMap<>(); + int i = 0; + for (MetaColumn column : table.getColumnList()) { + before.put(column.getName(), table.getOldData()[i]); + after.put(column.getName(), table.getNewData()[i]); + i++; + } + map.put("before", before); + map.put("after", after); + } + format.processEvent(map); + } + }catch (Exception e){ + String errorMessage = ExceptionUtil.getErrorMessage(e); + LOG.error(errorMessage); + format.processEvent(Collections.singletonMap("e", errorMessage)); + + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java new file mode 100644 index 0000000000..1b8cd773ce --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.pgwal.PgWalConfigKeys; +import com.dtstack.flinkx.pgwal.format.PgWalInputFormatBuilder; +import com.dtstack.flinkx.reader.DataReader; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgwalReader extends DataReader { + private String username; + private String password; + private String url; + private String databaseName; + private String cat; + private boolean pavingData; + private List tableList; + private Integer statusInterval; + private Long lsn; + private String slotName; + private boolean allowCreateSlot; + private boolean temporary; + + @SuppressWarnings("unchecked") + public PgwalReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + username = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_USER_NAME); + password = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_PASSWORD); + url = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_URL); + databaseName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_DATABASE_NAME); + cat = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_CATALOG); + pavingData = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_PAVING_DATA, false); + tableList = (List) readerConfig.getParameter().getVal(PgWalConfigKeys.KEY_TABLE_LIST); + statusInterval = readerConfig.getParameter().getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 10000); + lsn = readerConfig.getParameter().getLongVal(PgWalConfigKeys.KEY_LSN, 0); + slotName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_SLOT_NAME); + allowCreateSlot = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_ALLOW_CREATE_SLOT, true); + temporary = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_TEMPORARY, true); + } + + @Override + public DataStream readData() { + PgWalInputFormatBuilder builder = new PgWalInputFormatBuilder(); + builder.setUsername(username); + builder.setPassword(password); + builder.setUrl(url); + builder.setDatabaseName(databaseName); + builder.setCat(cat); + builder.setPavingData(pavingData); + builder.setTableList(tableList); + builder.setRestoreConfig(restoreConfig); + builder.setStatusInterval(statusInterval); + builder.setLsn(lsn); + builder.setSlotName(slotName); + builder.setAllowCreateSlot(allowCreateSlot); + builder.setTemporary(temporary); + return createInput(builder.finish(), "pgwalreader"); + } +} diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index effaf52bd3..3952080f47 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -362,6 +362,12 @@ flinkx-polardb-writer 1.6 + + + com.dtstack.flinkx + flinkx-pgwal-reader + 1.6 + diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index ca0be0783e..0b82dc17dc 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -60,6 +60,7 @@ import com.dtstack.flinkx.odps.writer.OdpsWriter; import com.dtstack.flinkx.oracle.reader.OracleReader; import com.dtstack.flinkx.oracle.writer.OracleWriter; +import com.dtstack.flinkx.pgwal.reader.PgwalReader; import com.dtstack.flinkx.polardb.reader.PolardbReader; import com.dtstack.flinkx.polardb.writer.PolardbWriter; import com.dtstack.flinkx.postgresql.reader.PostgresqlReader; @@ -125,7 +126,7 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\hdfs_stream.json"; + String jobPath = "/Users/tudou/Library/Preferences/IntelliJIdea2019.2/scratches/json/pgWal/pgwal_to_stream.json"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); } @@ -207,6 +208,7 @@ private static DataReader buildDataReader(DataTransferConfig config, StreamExecu case PluginNameConstrant.KUDU_READER : reader = new KuduReader(config, env); break; case PluginNameConstrant.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; case PluginNameConstrant.POLARDB_READER : reader = new PolardbReader(config, env); break; + case PluginNameConstrant.PGWAL_READER : reader = new PgwalReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java index d30bb39574..9a53b58f0f 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java @@ -46,6 +46,7 @@ public class PluginNameConstrant { public static final String KAFKA_READER = "kafkareader"; public static final String CLICKHOUSE_READER = "clickhousereader"; public static final String POLARDB_READER = "polardbreader"; + public static final String PGWAL_READER = "pgwalreader"; public static final String STREAM_WRITER = "streamwriter"; diff --git a/pom.xml b/pom.xml index df318d098c..28305c3a70 100644 --- a/pom.xml +++ b/pom.xml @@ -13,39 +13,39 @@ flinkx-core flinkx-launcher - - - + flinkx-test + flinkx-examples + flinkx-stream - - - - - - - - - - - - - - - - - - - - - + flinkx-rdb + flinkx-mysql + flinkx-polardb + flinkx-oracle + flinkx-sqlserver + flinkx-postgresql + flinkx-db2 + flinkx-gbase + flinkx-clickhouse + + flinkx-hdfs + flinkx-hive + flinkx-es + flinkx-ftp + flinkx-odps + flinkx-hbase + flinkx-carbondata + flinkx-kudu + + flinkx-redis + flinkx-mongodb - - - - - + flinkx-binlog + flinkx-kafka09 + flinkx-kafka10 + flinkx-kafka11 + flinkx-kafka flinkx-pgwal From 234fedaa9a964dd412ded2994af4108006ccf528 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 16 Dec 2019 18:56:37 +0800 Subject: [PATCH 005/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 1 + docs/pgwalreader.md | 184 ++++++++++++++++++ .../flinkx/pgwal/reader/PgwalReader.java | 2 +- 3 files changed, 186 insertions(+), 1 deletion(-) create mode 100644 docs/pgwalreader.md diff --git a/README.md b/README.md index 45c163103b..672b9888ef 100644 --- a/README.md +++ b/README.md @@ -234,6 +234,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [MySQL binlog读取插件](docs/binlog.md) * [KafKa读取插件](docs/kafkareader.md) * [Kudu读取插件](docs/kudureader.md) +* [PostgreSQL WAL实时采集插件](docs/pgwalreader.md) ### 5.2 写入插件 diff --git a/docs/pgwalreader.md b/docs/pgwalreader.md new file mode 100644 index 0000000000..56fbadbc84 --- /dev/null +++ b/docs/pgwalreader.md @@ -0,0 +1,184 @@ +# PostgreSQL WAL读取插件(*reader) + +## 1. 配置样例 + +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "username" : "postgres", + "password" : "abc123", + "url" : "jdbc:postgresql://172.16.8.122:5432/postgres", + "databaseName" : "postgres", + "cat" : "update,insert,delete", + "tableList" : [ + "changepk.test_table" + ], + "statusInterval" : 10000, + "lsn" : 0, + "slotName" : "", + "allowCreateSlot" : true, + "temporary" : true, + "pavingData" : true + }, + "name" : "pgwalreader" + }, + "writer": { + + } + }] + }, + "setting": { + + } +} +``` +## 2. 使用说明 + 1、PostgreSQL数据库版本至少为10.0及以上 + + 2、预写日志级别(wal_level)必须为logical + + 3、该插件基于PostgreSQL逻辑复制及逻辑解码功能实现的,因此PostgreSQL账户至少拥有replication权限,若允许创建slot,则至少拥有超级管理员权限 + + 4、详细原理请参见[PostgreSQL官方文档](http://postgres.cn/docs/10/index.html) + +## 3. 参数说明 + +* **name** + + * 描述:插件名,此处填写插件名称。 + + * 必选:是 + + * 默认值:无 + +* **jdbcUrl** + + * 描述:PostgreSQL数据库的jdbc连接字符串,参考文档:[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) + + * 必选:是 + + * 默认值:无 + +* **username** + + * 描述:数据源的用户名 + + * 必选:是 + + * 默认值:无 + +* **password** + + * 描述:数据源指定用户名的密码 + + * 必选:是 + + * 默认值:无 + +* **tableList** + + * 描述:需要解析的数据表,格式为schema.table + + * 必选:否 + + * 默认值:无 + +* **cat** + + * 描述:需要解析的数据更新类型,包括insert、update、delete三种。 + + * 注意:以英文逗号分割的格式填写。 + + * 必选:是 + + * 默认值:无 + +* **statusInterval** + + * 描述:复制期间,数据库和使用者定期交换ping消息。如果数据库或客户端在配置的超时时间内未收到ping消息,则复制被视为已停止,并且将引发异常,并且数据库将释放资源。在PostgreSQL中,ping超时由属性wal_sender_timeout配置(默认= 60秒)。可以将pgjdc中的复制流配置为在需要时或按时间间隔发送反馈(ping)。建议比配置的wal_sender_timeout更频繁地向数据库发送反馈(ping)。在生产环境中,我使用等于wal_sender_timeout / 3的值。它避免了网络潜在的问题,并且可以在不因超时而断开连接的情况下传输更改。 + + * 必选:否 + + * 默认值:2000 + +* **lsn** + + * 描述:要读取PostgreSQL WAL日志序列号的开始位置。 + + * 必选:否 + + * 默认值:0 + +* **slotName** + + * 描述:复制槽名称,根据该值去寻找或创建复制槽 + + * 注意:当allowCreateSlot为false时,该值不能为空 + + * 必选:否 + + * 默认值:无 + +* **allowCreateSlot** + + * 描述:是否允许创建复制槽 + + * 必选:否 + + * 默认值:true + +* **temporary** + + * 描述:复制槽是否为临时性的,true:是;false:否。 + + * 必选:否 + + * 默认值:true + +* **pavingData** + + * 描述:是否将解析出的json数据拍平 + + * 示例:假设解析的表为tb1,schema为dbo,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: + + ```json + { + "type":"update", + "schema":"dbo", + "table":"customers", + "lsn":207967352, + "ts": 1576487525488, + "ingestion":1475129582923642, + "before_id":1, + "after_id":2 + } + ``` + + pavingData为false时: + + ```json + { + "type":"update", + "schema":"dbo", + "table":"customers", + "lsn":207967352, + "ts": 1576487525488, + "ingestion":1481628798880038, + "before":{ + "id":1 + }, + "after":{ + "id":2 + } + } + ``` + + 其中:ts是数据库中数据的变更时间,ingestion是插件解析这条数据的纳秒时间,lsn是该数据变更的日志序列号 + + * 必选:否 + + * 默认值:false + diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java index 1b8cd773ce..3f9c86a020 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java @@ -60,7 +60,7 @@ public PgwalReader(DataTransferConfig config, StreamExecutionEnvironment env) { cat = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_CATALOG); pavingData = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_PAVING_DATA, false); tableList = (List) readerConfig.getParameter().getVal(PgWalConfigKeys.KEY_TABLE_LIST); - statusInterval = readerConfig.getParameter().getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 10000); + statusInterval = readerConfig.getParameter().getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 20000); lsn = readerConfig.getParameter().getLongVal(PgWalConfigKeys.KEY_LSN, 0); slotName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_SLOT_NAME); allowCreateSlot = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_ALLOW_CREATE_SLOT, true); From 495e644000be8ac8223710090ab3169d09d1534c Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 17 Dec 2019 10:06:55 +0800 Subject: [PATCH 006/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9reader?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/pgwal/reader/PgWalReader.java | 88 ------------------- 1 file changed, 88 deletions(-) delete mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java deleted file mode 100644 index c1d22d3f29..0000000000 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgWalReader.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.pgwal.reader; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.ReaderConfig; -import com.dtstack.flinkx.pgwal.PgWalConfigKeys; -import com.dtstack.flinkx.pgwal.format.PgWalInputFormatBuilder; -import com.dtstack.flinkx.reader.DataReader; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.types.Row; - -import java.util.List; - -/** - * Date: 2019/12/13 - * Company: www.dtstack.com - * - * @author tudou - */ -public class PgWalReader extends DataReader { - private String username; - private String password; - private String url; - private String databaseName; - private String cat; - private boolean pavingData; - private List tableList; - private Integer statusInterval; - private String lsn; - private String slotName; - private boolean allowCreateSlot; - private boolean temporary; - - @SuppressWarnings("unchecked") - protected PgWalReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - username = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_USER_NAME); - password = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_PASSWORD); - url = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_URL); - databaseName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_DATABASE_NAME); - cat = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_CATALOG); - pavingData = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_PAVING_DATA, false); - tableList = (List) readerConfig.getParameter().getVal(PgWalConfigKeys.KEY_TABLE_LIST); - statusInterval = readerConfig.getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 10000); - lsn = readerConfig.getStringVal(PgWalConfigKeys.KEY_LSN); - slotName = readerConfig.getStringVal(PgWalConfigKeys.KEY_SLOT_NAME); - allowCreateSlot = readerConfig.getBooleanVal(PgWalConfigKeys.KEY_ALLOW_CREATE_SLOT, true); - temporary = readerConfig.getBooleanVal(PgWalConfigKeys.KEY_TEMPORARY, true); - } - - @Override - public DataStream readData() { - PgWalInputFormatBuilder builder = new PgWalInputFormatBuilder(); - builder.setUsername(username); - builder.setPassword(password); - builder.setUrl(url); - builder.setDatabaseName(databaseName); - builder.setCat(cat); - builder.setPavingData(pavingData); - builder.setTableList(tableList); - builder.setRestoreConfig(restoreConfig); - builder.setStatusInterval(statusInterval); - builder.setLsn(lsn); - builder.setSlotName(slotName); - builder.setAllowCreateSlot(allowCreateSlot); - builder.setTemporary(temporary); - return createInput(builder.finish(), "pgwalreader"); - } -} From 22d3029da58095e679c61c9783fa1c69a2f68e1a Mon Sep 17 00:00:00 2001 From: wuhui <438949560@qq.com> Date: Thu, 19 Dec 2019 11:37:58 +0800 Subject: [PATCH 007/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dgit=5Fbranch=E7=9A=84?= =?UTF-8?q?bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-binlog/flinkx-binlog-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-carbondata-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-carbondata-writer/pom.xml | 21 ++++++++++++++++++- .../flinkx-cassandra-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-cassandra-writer/pom.xml | 21 ++++++++++++++++++- .../flinkx-clickhouse-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-clickhouse-writer/pom.xml | 21 ++++++++++++++++++- flinkx-core/pom.xml | 21 ++++++++++++++++++- flinkx-db2/flinkx-db2-reader/pom.xml | 21 ++++++++++++++++++- flinkx-db2/flinkx-db2-writer/pom.xml | 21 ++++++++++++++++++- flinkx-es/flinkx-es-reader/pom.xml | 21 ++++++++++++++++++- flinkx-es/flinkx-es-writer/pom.xml | 21 ++++++++++++++++++- flinkx-ftp/flinkx-ftp-reader/pom.xml | 21 ++++++++++++++++++- flinkx-ftp/flinkx-ftp-writer/pom.xml | 21 ++++++++++++++++++- flinkx-gbase/flinkx-gbase-reader/pom.xml | 21 ++++++++++++++++++- flinkx-gbase/flinkx-gbase-writer/pom.xml | 21 ++++++++++++++++++- flinkx-hbase/flinkx-hbase-reader/pom.xml | 21 ++++++++++++++++++- flinkx-hbase/flinkx-hbase-writer/pom.xml | 21 ++++++++++++++++++- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 21 ++++++++++++++++++- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 21 ++++++++++++++++++- flinkx-hive/flinkx-hive-writer/pom.xml | 21 ++++++++++++++++++- flinkx-kafka/flinkx-kafka-reader/pom.xml | 21 ++++++++++++++++++- flinkx-kafka/flinkx-kafka-writer/pom.xml | 21 ++++++++++++++++++- flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 21 ++++++++++++++++++- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 21 ++++++++++++++++++- flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 21 ++++++++++++++++++- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 21 ++++++++++++++++++- flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 21 ++++++++++++++++++- flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 21 ++++++++++++++++++- flinkx-kudu/flinkx-kudu-reader/pom.xml | 21 ++++++++++++++++++- flinkx-kudu/flinkx-kudu-writer/pom.xml | 21 ++++++++++++++++++- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 21 ++++++++++++++++++- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 21 ++++++++++++++++++- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 21 ++++++++++++++++++- flinkx-mysql/flinkx-mysql-reader/pom.xml | 21 ++++++++++++++++++- flinkx-mysql/flinkx-mysql-writer/pom.xml | 21 ++++++++++++++++++- flinkx-odps/flinkx-odps-reader/pom.xml | 21 ++++++++++++++++++- flinkx-odps/flinkx-odps-writer/pom.xml | 21 ++++++++++++++++++- flinkx-oracle/flinkx-oracle-reader/pom.xml | 21 ++++++++++++++++++- flinkx-oracle/flinkx-oracle-writer/pom.xml | 21 ++++++++++++++++++- flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 21 ++++++++++++++++++- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 21 ++++++++++++++++++- flinkx-polardb/flinkx-polardb-dreader/pom.xml | 21 ++++++++++++++++++- flinkx-polardb/flinkx-polardb-reader/pom.xml | 21 ++++++++++++++++++- flinkx-polardb/flinkx-polardb-writer/pom.xml | 21 ++++++++++++++++++- .../flinkx-postgresql-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-postgresql-writer/pom.xml | 21 ++++++++++++++++++- flinkx-rdb/flinkx-rdb-core/pom.xml | 21 ++++++++++++++++++- flinkx-rdb/flinkx-rdb-reader/pom.xml | 21 ++++++++++++++++++- flinkx-rdb/flinkx-rdb-writer/pom.xml | 21 ++++++++++++++++++- flinkx-redis/flinkx-redis-writer/pom.xml | 21 ++++++++++++++++++- flinkx-saphana/flinkx-saphana-reader/pom.xml | 21 ++++++++++++++++++- flinkx-saphana/flinkx-saphana-writer/pom.xml | 21 ++++++++++++++++++- .../flinkx-sqlserver-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-sqlserver-writer/pom.xml | 21 ++++++++++++++++++- flinkx-stream/flinkx-stream-reader/pom.xml | 21 ++++++++++++++++++- flinkx-stream/flinkx-stream-writer/pom.xml | 21 ++++++++++++++++++- .../flinkx-teradata-reader/pom.xml | 21 ++++++++++++++++++- .../flinkx-teradata-writer/pom.xml | 21 ++++++++++++++++++- 59 files changed, 1180 insertions(+), 59 deletions(-) diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index d83803de0b..b66b1763b6 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-binlog-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -67,7 +86,7 @@ + tofile="${basedir}/../../plugins/binlogreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index aafacd41e4..745e658df1 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-carbondata-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -89,7 +108,7 @@ + tofile="${basedir}/../../plugins/carbondatareader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index e25400046f..fbb753c19d 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-carbondata-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + @@ -92,7 +111,7 @@ + tofile="${basedir}/../../plugins/carbondatawriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index 760f2b245a..87fdc3a59b 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-cassandra-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + flinkx-cassandra-core @@ -84,7 +103,7 @@ + tofile="${basedir}/../../plugins/cassandrareader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index b49206c20d..e6f3c9fedd 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-cassandra-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + flinkx-cassandra-core @@ -84,7 +103,7 @@ + tofile="${basedir}/../../plugins/cassandrawriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 83498a0f46..5612533733 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-clickhouse-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -79,7 +98,7 @@ + tofile="${basedir}/../../plugins/clickhousereader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 1ade5d5b53..827dfefd09 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -10,6 +10,25 @@ 4.0.0 flinkx-clickhouse-writer + + + + dev + + ${package.name} + + + true + + + + + release + + release + + + com.dtstack.flinkx @@ -78,7 +97,7 @@ + tofile="${basedir}/../../plugins/clickhousewriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index efdef83991..9ca940db42 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -16,6 +16,25 @@ ${basedir}/../dev + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + @@ -226,7 +245,7 @@ + tofile="${basedir}/../plugins/flinkx-${project.version}-${package.name}.jar" /> diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 69bfcf8375..308214bc89 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-db2-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/db2reader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 926b6a913a..e2e21ccd7a 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-db2-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/db2writer/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 896139a4ac..4a79bd74d5 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-es-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -77,7 +96,7 @@ + tofile="${basedir}/../../plugins/esreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 07810d0273..44fc10cf22 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-es-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -78,7 +97,7 @@ + tofile="${basedir}/../../plugins/eswriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 4ec609b227..7220295099 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -29,6 +29,25 @@ under the License. flinkx-ftp-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -95,7 +114,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpreader/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index bf67ab37f5..0b9da3aed2 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -29,6 +29,25 @@ under the License. flinkx-ftp-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -96,7 +115,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 709c3135d6..3929ebd1d5 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-gbase-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/gbasereader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index bd729afa05..f08d51679f 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-gbase-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/gbasewriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index c8bf471609..0168798c6d 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-hbase-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/hbasereader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index 3fbffa9eaa..b8a7405634 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-hbase-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -79,7 +98,7 @@ + tofile="${basedir}/../../plugins/hbasewriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index 7eaf9a1f5d..4dc670de03 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -33,6 +33,25 @@ under the License. 1.1.1 + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -122,7 +141,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfsreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index ed7a3c4e20..9da5618b4d 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -33,6 +33,25 @@ under the License. 1.1.1 + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -123,7 +142,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfswriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 5838ca27bd..934b88dedc 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -33,6 +33,25 @@ under the License. 1.1.1 + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -117,7 +136,7 @@ under the License. + tofile="${basedir}/../../plugins/hivewriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index 7699899ec8..d11f27dfb3 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -57,7 +76,7 @@ + tofile="${basedir}/../../plugins/kafkareader/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index 54df306260..322aa0438c 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -57,7 +76,7 @@ + tofile="${basedir}/../../plugins/kafkawriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index 173f4f8733..cd837a3823 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka09-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -76,7 +95,7 @@ + tofile="${basedir}/../../plugins/kafka09reader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index 47fe35c6e7..fa5388537f 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka09-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -76,7 +95,7 @@ + tofile="${basedir}/../../plugins/kafka09writer/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 63a290bbc4..3aaef5c20a 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka10-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -86,7 +105,7 @@ + tofile="${basedir}/../../plugins/kafka10reader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index 67c7d083a0..980d894acb 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka10-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -86,7 +105,7 @@ + tofile="${basedir}/../../plugins/kafka10writer/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index 0067e69a46..005d4bfd13 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka11-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -86,7 +105,7 @@ + tofile="${basedir}/../../plugins/kafka11reader/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index 654e033a5b..83377e8a33 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-kafka11-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -86,7 +105,7 @@ + tofile="${basedir}/../../plugins/kafka11writer/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index e9dbab3e62..b9f7f0461b 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-kudu-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -77,7 +96,7 @@ + tofile="${basedir}/../../plugins/kudureader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index 00c38bfcc8..ad18666160 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-kudu-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -77,7 +96,7 @@ + tofile="${basedir}/../../plugins/kuduwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 282db0dda9..0c619147df 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-mongodb-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + flinkx-mongodb-core @@ -84,7 +103,7 @@ + tofile="${basedir}/../../plugins/mongodbreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 5d86666550..5f29eea84e 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-mongodb-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + flinkx-mongodb-core @@ -84,7 +103,7 @@ + tofile="${basedir}/../../plugins/mongodbwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index e9253b66bc..f454098a29 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -11,6 +11,25 @@ flinkx-mysql-dreader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/mysqldreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index b23815b5c4..7e99496671 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-mysql-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/mysqlreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 4ada5dfa13..9ac6ad473d 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-mysql-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/mysqlwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 2475779dde..325c89bb0c 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-odps-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -74,7 +93,7 @@ + tofile="${basedir}/../../plugins/odpsreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index 8dd867c766..ed4792abb3 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-odps-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -79,7 +98,7 @@ + tofile="${basedir}/../../plugins/odpswriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 95520f0672..0cea76d018 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-oracle-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/oraclereader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index 5581282b41..f351dffb9a 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-oracle-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/oraclewriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index b26f507ab8..e86a03ae6f 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-phoenix-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/phoenixreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index 0862d3b7f0..3b2e162b3b 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-phoenix-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/phoenixwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index a759cb1aeb..3945dabfe6 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -11,6 +11,25 @@ flinkx-polardb-dreader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -74,7 +93,7 @@ + tofile="${basedir}/../../plugins/polardbdreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 59e5c51ae9..41a16f01ea 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-polardb-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -74,7 +93,7 @@ + tofile="${basedir}/../../plugins/polardbreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index b2bedda02a..00f985cc75 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-polardb-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -74,7 +93,7 @@ + tofile="${basedir}/../../plugins/polardbwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index 280e6e3c54..a72d22c37a 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-postgresql-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/postgresqlreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 8141a88b24..e525990250 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-postgresql-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/postgresqlwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-rdb/flinkx-rdb-core/pom.xml b/flinkx-rdb/flinkx-rdb-core/pom.xml index 3f5dfa4381..dd85947710 100644 --- a/flinkx-rdb/flinkx-rdb-core/pom.xml +++ b/flinkx-rdb/flinkx-rdb-core/pom.xml @@ -11,6 +11,25 @@ jar flinkx-rdb-core + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + @@ -32,7 +51,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-rdb/flinkx-rdb-reader/pom.xml b/flinkx-rdb/flinkx-rdb-reader/pom.xml index 89ac9f26e5..90746ad5e5 100644 --- a/flinkx-rdb/flinkx-rdb-reader/pom.xml +++ b/flinkx-rdb/flinkx-rdb-reader/pom.xml @@ -11,6 +11,25 @@ jar flinkx-rdb-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -41,7 +60,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-rdb/flinkx-rdb-writer/pom.xml b/flinkx-rdb/flinkx-rdb-writer/pom.xml index 9aa8b5ed27..6b03b475dc 100644 --- a/flinkx-rdb/flinkx-rdb-writer/pom.xml +++ b/flinkx-rdb/flinkx-rdb-writer/pom.xml @@ -11,6 +11,25 @@ jar flinkx-rdb-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -41,7 +60,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${project.version}-${package.name}.jar"/> diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 6f4b0b2ac7..e4ba1feae2 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-redis-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -84,7 +103,7 @@ + tofile="${basedir}/../../plugins/rediswriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index 78b80386e2..b5aceaddff 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-saphana-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/saphanareader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index d304759fb6..925b47ab5f 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-saphana-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/saphanawriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index 4924cb94ea..d32884f8e5 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-sqlserver-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -90,7 +109,7 @@ + tofile="${basedir}/../../plugins/sqlserverreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index 171d65cb67..7048bfbb4e 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-sqlserver-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -91,7 +110,7 @@ + tofile="${basedir}/../../plugins/sqlserverwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index 25a83749de..6ede84f510 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-stream-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.github.jsonzou @@ -73,7 +92,7 @@ + tofile="${basedir}/../../plugins/streamreader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index ae915a9b40..d6fbca5e7a 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-stream-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + @@ -69,7 +88,7 @@ + tofile="${basedir}/../../plugins/streamwriter/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml index fb1d4d04b6..2f572d102a 100644 --- a/flinkx-teradata/flinkx-teradata-reader/pom.xml +++ b/flinkx-teradata/flinkx-teradata-reader/pom.xml @@ -11,6 +11,25 @@ flinkx-teradata-reader + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -96,7 +115,7 @@ + tofile="${basedir}/../../plugins/teradatareader/${project.name}-${project.version}-${package.name}.jar" /> diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml index 315d7aa161..4101bf741e 100644 --- a/flinkx-teradata/flinkx-teradata-writer/pom.xml +++ b/flinkx-teradata/flinkx-teradata-writer/pom.xml @@ -11,6 +11,25 @@ flinkx-teradata-writer + + + dev + + ${package.name} + + + true + + + + + release + + release + + + + com.dtstack.flinkx @@ -96,7 +115,7 @@ + tofile="${basedir}/../../plugins/teradatawriter/${project.name}-${project.version}-${package.name}.jar" /> From 89edb3dd096068b93225cf6e739213430b134612 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 23 Dec 2019 14:44:46 +0800 Subject: [PATCH 008/136] =?UTF-8?q?[reader=E5=92=8Cwriter=E7=9A=84?= =?UTF-8?q?=E5=B9=B6=E5=8F=91=E6=95=B0=E5=8F=AF=E5=88=86=E5=88=AB=E9=85=8D?= =?UTF-8?q?=E7=BD=AE=EF=BC=8C=E6=98=AF=E5=90=A6reblance=E5=8F=AF=E9=85=8D?= =?UTF-8?q?=E7=BD=AE][21253][21127]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 19 ++++++---- flinkx-core/pom.xml | 6 ++++ .../main/java/com/dtstack/flinkx/Main.java | 15 +++++--- .../dtstack/flinkx/config/SpeedConfig.java | 26 +++++++++++--- .../flinkx/stream/reader/StreamReader.java | 2 +- .../com/dtstack/flinkx/test/LocalTest.java | 17 +++++---- .../dev_test_job/stream_template.json | 36 ++++--------------- 7 files changed, 68 insertions(+), 53 deletions(-) diff --git a/README.md b/README.md index 45c163103b..200d12e8b8 100644 --- a/README.md +++ b/README.md @@ -132,14 +132,20 @@ setting包括speed、errorLimit和dirty三部分,分别描述限速、错误 #### 4.1.1 speed ``` - "speed": { - "channel": 3, - "bytes": 0 - } +"speed": { + "bytes": 1048576, + "channel": 2, + "rebalance": false, + "readerChannel": 1, + "writerChannel": 1 +} ``` -* channel: 任务并发数 -* bytes: 每秒字节数,默认为 Long.MAX_VALUE +* channel:任务并发数 +* readerChannel:reader的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为reader的并发数。 +* writerChannel:writer的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为writer的并发数。 +* rebalance:此参数配置为true时将强制对reader的数据做Rebalance,不配置此参数或者配置为false时,程序会根据reader和writer的通道数选择是否Rebalance,reader和writer的通道数一致时不使用Reblance,通道数不一致时使用Reblance。 +* bytes::每秒字节数,默认为 Long.MAX_VALUE #### 4.1.2 errorLimit @@ -235,7 +241,6 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [KafKa读取插件](docs/kafkareader.md) * [Kudu读取插件](docs/kudureader.md) - ### 5.2 写入插件 * [关系数据库写入插件](docs/rdbwriter.md) diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 85b373f49b..26dfafad67 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -49,6 +49,12 @@ ${flink.version} + + org.apache.flink + flink-runtime-web_2.11 + ${flink.version} + + org.apache.flink diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 746aff3a3a..69742ff1d4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -21,6 +21,7 @@ import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.classloader.ClassLoaderManager; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.SpeedConfig; import com.dtstack.flinkx.constants.ConfigConstrant; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.reader.DataReader; @@ -37,6 +38,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -90,17 +92,20 @@ public static void main(String[] args) throws Exception { env = openCheckpointConf(env, confProperties); - env.setParallelism(config.getJob().getSetting().getSpeed().getChannel()); + SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); + + env.setParallelism(speedConfig.getChannel()); env.setRestartStrategy(RestartStrategies.noRestart()); DataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); + dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); - dataStream = new DataStream<>(dataStream.getExecutionEnvironment(), - new PartitionTransformation<>(dataStream.getTransformation(), - new DTRebalancePartitioner<>())); + if (speedConfig.isRebalance()) { + dataStream = dataStream.rebalance(); + } DataWriter dataWriter = DataWriterFactory.getDataWriter(config); - dataWriter.writeData(dataStream); + dataWriter.writeData(dataStream).setParallelism(speedConfig.getWriterChannel()); if(env instanceof MyLocalStreamEnvironment) { if(StringUtils.isNotEmpty(savepointPath)){ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/SpeedConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/SpeedConfig.java index b46b01444f..f7a31043c3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/SpeedConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/SpeedConfig.java @@ -31,9 +31,13 @@ public class SpeedConfig extends AbstractConfig { public static final String KEY_BYTES = "bytes"; public static final String KEY_NUM_CHANNELS = "channel"; + public static final String KEY_NUM_READER_CHANNELS = "readerChannel"; + public static final String KEY_NUM_WRITER_CHANNELS = "writerChannel"; + public static final String KEY_REBALANCE = "rebalance"; public static final long DEFAULT_SPEED_BYTES = Long.MAX_VALUE; - public static final int DEFAULT_NUM_CHANNALS = 1; + public static final int DEFAULT_NUM_CHANNELS = 1; + public static final int DEFAULT_NUM_READER_WRITER_CHANNEL = -1; public SpeedConfig(Map map) { super(map); @@ -41,8 +45,11 @@ public SpeedConfig(Map map) { public static SpeedConfig defaultConfig(){ Map map = new HashMap<>(2); - map.put("bytes",DEFAULT_SPEED_BYTES); - map.put("channel",DEFAULT_NUM_CHANNALS); + map.put(KEY_BYTES, DEFAULT_SPEED_BYTES); + map.put(KEY_NUM_CHANNELS, DEFAULT_NUM_CHANNELS); + map.put(KEY_NUM_READER_CHANNELS, DEFAULT_NUM_READER_WRITER_CHANNEL); + map.put(KEY_NUM_WRITER_CHANNELS, DEFAULT_NUM_READER_WRITER_CHANNEL); + map.put(KEY_REBALANCE, false); return new SpeedConfig(map); } @@ -55,11 +62,22 @@ public void setBytes(long bytes) { } public int getChannel() { - return getIntVal(KEY_NUM_CHANNELS, DEFAULT_NUM_CHANNALS); + return getIntVal(KEY_NUM_CHANNELS, DEFAULT_NUM_CHANNELS); + } + + public int getReaderChannel(){ + return getIntVal(KEY_NUM_READER_CHANNELS, DEFAULT_NUM_READER_WRITER_CHANNEL); + } + + public int getWriterChannel(){ + return getIntVal(KEY_NUM_WRITER_CHANNELS, DEFAULT_NUM_READER_WRITER_CHANNEL); } public void setChannel(int channel) { setIntVal(KEY_NUM_CHANNELS, channel); } + public boolean isRebalance() { + return getBooleanVal(KEY_REBALANCE, false); + } } diff --git a/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/StreamReader.java b/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/StreamReader.java index 778682d56c..2cdeb62d9a 100644 --- a/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/StreamReader.java +++ b/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/StreamReader.java @@ -51,7 +51,7 @@ public StreamReader(DataTransferConfig config, StreamExecutionEnvironment env) { List list = (List)readerConfig.getParameter().getVal("sliceRecordCount"); if(CollectionUtils.isNotEmpty(list)){ for (Object item : list) { - sliceRecordCount.add(Long.valueOf(item.toString())); + sliceRecordCount.add(Double.valueOf(item.toString()).longValue()); } } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index bdcea70968..f611274ecb 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -27,6 +27,7 @@ import com.dtstack.flinkx.clickhouse.reader.ClickhouseReader; import com.dtstack.flinkx.clickhouse.writer.ClickhouseWriter; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.SpeedConfig; import com.dtstack.flinkx.constants.ConfigConstrant; import com.dtstack.flinkx.db2.reader.Db2Reader; import com.dtstack.flinkx.db2.writer.Db2Writer; @@ -80,6 +81,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.transformations.PartitionTransformation; @@ -124,8 +126,9 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.jobName","108job"); // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); + conf.setString("rest.bind-port", "8888"); - String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\stream_hdfs.json"; + String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\stream_template.json"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); } @@ -144,23 +147,25 @@ public static JobExecutionResult runJob(File jobFile, Properties confProperties, public static JobExecutionResult runJob(String job, Properties confProperties, String savepointPath) throws Exception{ DataTransferConfig config = DataTransferConfig.parse(job); + SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); openCheckpointConf(env, confProperties); - env.setParallelism(config.getJob().getSetting().getSpeed().getChannel()); + env.setParallelism(speedConfig.getChannel()); env.setRestartStrategy(RestartStrategies.noRestart()); DataReader reader = buildDataReader(config, env); DataStream dataStream = reader.readData(); + dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); - dataStream = new DataStream<>(dataStream.getExecutionEnvironment(), - new PartitionTransformation<>(dataStream.getTransformation(), - new DTRebalancePartitioner<>())); + if (speedConfig.isRebalance()) { + dataStream = dataStream.rebalance(); + } DataWriter writer = buildDataWriter(config); - writer.writeData(dataStream); + writer.writeData(dataStream).setParallelism(speedConfig.getWriterChannel()); if(StringUtils.isNotEmpty(savepointPath)){ env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); diff --git a/flinkx-test/src/main/resources/dev_test_job/stream_template.json b/flinkx-test/src/main/resources/dev_test_job/stream_template.json index 5ead6d562c..4091dfa6ca 100644 --- a/flinkx-test/src/main/resources/dev_test_job/stream_template.json +++ b/flinkx-test/src/main/resources/dev_test_job/stream_template.json @@ -4,7 +4,7 @@ { "reader": { "parameter": { - "sliceRecordCount": ["1000"], + "sliceRecordCount": [1000,100], "column": [ { "name": "name", @@ -29,34 +29,7 @@ }, "writer": { "parameter": { - "print": true, - "column": [ - { - "name": "cf:name", - "type": "string" - }, - { - "name": "cf:age", - "type": "int" - }, - { - "name": "cf:father", - "type": "string" - }, - { - "name": "cf:adress", - "type": "string" - } - ], - "rowkeyColumn": "md5($(cf:name)_$(cf:father))", - "writeBufferSize": 0, - "table": "md5_test", - "sourceIds": [ - 371 - ], - "hbaseConfig": { - "hbase.zookeeper.quorum": "172.16.8.193:2181" - } + "print": true }, "name": "streamwriter" } @@ -68,7 +41,10 @@ }, "speed": { "bytes": 1048576, - "channel": 1 + "channel": 2, + "rebalance": false, + "readerChannel": 1, + "writerChannel": 1 } } } From 4ebbacfb7cee8a88292214dc3860e93c6b05c55d Mon Sep 17 00:00:00 2001 From: jiangbo Date: Mon, 23 Dec 2019 14:48:37 +0800 Subject: [PATCH 009/136] =?UTF-8?q?=E5=88=A0=E9=99=A4DtRebalancePartitione?= =?UTF-8?q?r?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../partitioner/DTRebalancePartitioner.java | 49 ------------------- 1 file changed, 49 deletions(-) delete mode 100644 flinkx-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/DTRebalancePartitioner.java diff --git a/flinkx-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/DTRebalancePartitioner.java b/flinkx-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/DTRebalancePartitioner.java deleted file mode 100644 index 7708512292..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/DTRebalancePartitioner.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.runtime.partitioner; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.plugable.SerializationDelegate; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.types.Row; - -/** - * Rewrite the [DTRebalancePartitioner] to distribute data based on the channel specified in the data - * - * @param Type of the elements in the Stream being rebalanced - */ -@Internal -public class DTRebalancePartitioner extends StreamPartitioner { - private static final long serialVersionUID = 1L; - - @Override - public int selectChannel(SerializationDelegate> streamRecordSerializationDelegate) { - Row row = (Row) streamRecordSerializationDelegate.getInstance().getValue(); - return (Integer)row.getField(row.getArity() - 1); - } - - @Override - public StreamPartitioner copy() { - return this; - } - - @Override - public String toString() { - return "REBALANCE"; - } -} From 3bacaa16ccf23f22e916db97c942040ebdb1fbbd Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 17 Jan 2020 14:49:41 +0800 Subject: [PATCH 010/136] =?UTF-8?q?=E5=8E=BB=E6=8E=89=E5=A4=9A=E4=BD=99?= =?UTF-8?q?=E7=9A=84profiles?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-binlog/flinkx-binlog-reader/pom.xml | 19 ------------------- .../flinkx-carbondata-reader/pom.xml | 19 ------------------- .../flinkx-carbondata-writer/pom.xml | 19 ------------------- .../flinkx-cassandra-reader/pom.xml | 19 ------------------- .../flinkx-cassandra-writer/pom.xml | 19 ------------------- .../flinkx-clickhouse-reader/pom.xml | 19 ------------------- .../flinkx-clickhouse-writer/pom.xml | 18 ------------------ flinkx-core/pom.xml | 19 ------------------- flinkx-db2/flinkx-db2-reader/pom.xml | 19 ------------------- flinkx-db2/flinkx-db2-writer/pom.xml | 19 ------------------- flinkx-es/flinkx-es-reader/pom.xml | 19 ------------------- flinkx-es/flinkx-es-writer/pom.xml | 19 ------------------- flinkx-ftp/flinkx-ftp-reader/pom.xml | 19 ------------------- flinkx-ftp/flinkx-ftp-writer/pom.xml | 19 ------------------- flinkx-gbase/flinkx-gbase-reader/pom.xml | 19 ------------------- flinkx-gbase/flinkx-gbase-writer/pom.xml | 19 ------------------- flinkx-hbase/flinkx-hbase-reader/pom.xml | 19 ------------------- flinkx-hbase/flinkx-hbase-writer/pom.xml | 19 ------------------- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 19 ------------------- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 19 ------------------- flinkx-hive/flinkx-hive-writer/pom.xml | 19 ------------------- flinkx-kafka/flinkx-kafka-reader/pom.xml | 19 ------------------- flinkx-kafka/flinkx-kafka-writer/pom.xml | 19 ------------------- flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 19 ------------------- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 19 ------------------- flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 19 ------------------- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 19 ------------------- flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 19 ------------------- flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 19 ------------------- flinkx-kudu/flinkx-kudu-reader/pom.xml | 19 ------------------- flinkx-kudu/flinkx-kudu-writer/pom.xml | 19 ------------------- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 19 ------------------- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 19 ------------------- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 19 ------------------- flinkx-mysql/flinkx-mysql-reader/pom.xml | 19 ------------------- flinkx-mysql/flinkx-mysql-writer/pom.xml | 19 ------------------- flinkx-odps/flinkx-odps-reader/pom.xml | 19 ------------------- flinkx-odps/flinkx-odps-writer/pom.xml | 19 ------------------- flinkx-oracle/flinkx-oracle-reader/pom.xml | 19 ------------------- flinkx-oracle/flinkx-oracle-writer/pom.xml | 19 ------------------- flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 19 ------------------- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 19 ------------------- flinkx-polardb/flinkx-polardb-dreader/pom.xml | 19 ------------------- flinkx-polardb/flinkx-polardb-reader/pom.xml | 19 ------------------- flinkx-polardb/flinkx-polardb-writer/pom.xml | 19 ------------------- .../flinkx-postgresql-reader/pom.xml | 19 ------------------- .../flinkx-postgresql-writer/pom.xml | 19 ------------------- flinkx-rdb/flinkx-rdb-core/pom.xml | 19 ------------------- flinkx-rdb/flinkx-rdb-reader/pom.xml | 19 ------------------- flinkx-rdb/flinkx-rdb-writer/pom.xml | 19 ------------------- flinkx-redis/flinkx-redis-writer/pom.xml | 19 ------------------- flinkx-saphana/flinkx-saphana-reader/pom.xml | 19 ------------------- flinkx-saphana/flinkx-saphana-writer/pom.xml | 19 ------------------- .../flinkx-sqlserver-reader/pom.xml | 19 ------------------- .../flinkx-sqlserver-writer/pom.xml | 19 ------------------- flinkx-stream/flinkx-stream-reader/pom.xml | 19 ------------------- flinkx-stream/flinkx-stream-writer/pom.xml | 19 ------------------- .../flinkx-teradata-reader/pom.xml | 19 ------------------- .../flinkx-teradata-writer/pom.xml | 19 ------------------- pom.xml | 19 +++++++++++++++++++ 60 files changed, 19 insertions(+), 1120 deletions(-) diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index b66b1763b6..92d2bdfff1 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-binlog-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index 745e658df1..049e4f4d96 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-carbondata-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index fbb753c19d..9f7b556146 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-carbondata-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index 87fdc3a59b..2189e1fe07 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-cassandra-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - flinkx-cassandra-core diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index e6f3c9fedd..aaa690b80a 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-cassandra-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - flinkx-cassandra-core diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 5612533733..59f71729b8 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-clickhouse-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 827dfefd09..6f02526c41 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -11,24 +11,6 @@ flinkx-clickhouse-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - com.dtstack.flinkx diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 9ca940db42..fb12988d0a 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -16,25 +16,6 @@ ${basedir}/../dev - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 308214bc89..b218614f3d 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-db2-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index e2e21ccd7a..16439de9e9 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-db2-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 4a79bd74d5..87a9e3ac2f 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-es-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 44fc10cf22..52582620ba 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-es-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 7220295099..e6c22653f6 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -29,25 +29,6 @@ under the License. flinkx-ftp-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 0b9da3aed2..c30c8c36f8 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -29,25 +29,6 @@ under the License. flinkx-ftp-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 3929ebd1d5..130a443848 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-gbase-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index f08d51679f..b358851a8a 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-gbase-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index 0168798c6d..eabd9bc5fc 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-hbase-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index b8a7405634..e42ba78911 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-hbase-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index 4dc670de03..22290ecd32 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -33,25 +33,6 @@ under the License. 1.1.1 - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 9da5618b4d..724f008d39 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -33,25 +33,6 @@ under the License. 1.1.1 - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 934b88dedc..8d72d77a91 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -33,25 +33,6 @@ under the License. 1.1.1 - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index d11f27dfb3..54524522db 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index 322aa0438c..dcecdc9a8c 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index cd837a3823..b796ff0e54 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka09-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index fa5388537f..c9f84fd189 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka09-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 3aaef5c20a..215692f520 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka10-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index 980d894acb..10e2b5a9d5 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka10-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index 005d4bfd13..f8f128f115 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka11-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index 83377e8a33..8c7dd607fb 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-kafka11-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index b9f7f0461b..aaf24473f6 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-kudu-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index ad18666160..1694c091cb 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-kudu-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 0c619147df..634ec94d3d 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-mongodb-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - flinkx-mongodb-core diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 5f29eea84e..fe75c7d638 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-mongodb-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - flinkx-mongodb-core diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index f454098a29..22749ca413 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -11,25 +11,6 @@ flinkx-mysql-dreader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 7e99496671..93ef066baf 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-mysql-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 9ac6ad473d..74144d03da 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-mysql-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 325c89bb0c..502a3e49c6 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-odps-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index ed4792abb3..fff64aa370 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-odps-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 0cea76d018..f6266b7790 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-oracle-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index f351dffb9a..4225dfe259 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-oracle-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index e86a03ae6f..25c7e13f35 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-phoenix-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index 3b2e162b3b..1d5c2d519e 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-phoenix-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index 3945dabfe6..b8c6dd6cc0 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -11,25 +11,6 @@ flinkx-polardb-dreader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 41a16f01ea..a8a4349fe5 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-polardb-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index 00f985cc75..f0994e59e2 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-polardb-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index a72d22c37a..068c9dbca4 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-postgresql-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index e525990250..ec0f4f5d7d 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-postgresql-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-rdb/flinkx-rdb-core/pom.xml b/flinkx-rdb/flinkx-rdb-core/pom.xml index dd85947710..2229658781 100644 --- a/flinkx-rdb/flinkx-rdb-core/pom.xml +++ b/flinkx-rdb/flinkx-rdb-core/pom.xml @@ -11,25 +11,6 @@ jar flinkx-rdb-core - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - diff --git a/flinkx-rdb/flinkx-rdb-reader/pom.xml b/flinkx-rdb/flinkx-rdb-reader/pom.xml index 90746ad5e5..fc9ab2ab04 100644 --- a/flinkx-rdb/flinkx-rdb-reader/pom.xml +++ b/flinkx-rdb/flinkx-rdb-reader/pom.xml @@ -11,25 +11,6 @@ jar flinkx-rdb-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-rdb/flinkx-rdb-writer/pom.xml b/flinkx-rdb/flinkx-rdb-writer/pom.xml index 6b03b475dc..fcbfbe90be 100644 --- a/flinkx-rdb/flinkx-rdb-writer/pom.xml +++ b/flinkx-rdb/flinkx-rdb-writer/pom.xml @@ -11,25 +11,6 @@ jar flinkx-rdb-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index e4ba1feae2..724afc3255 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-redis-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index b5aceaddff..c707bbf2ea 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-saphana-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 925b47ab5f..1ded9f5ae9 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-saphana-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index d32884f8e5..123f212527 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-sqlserver-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index 7048bfbb4e..e04a6800d8 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-sqlserver-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index 6ede84f510..d31208c4e5 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-stream-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.github.jsonzou diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index d6fbca5e7a..25c9e4ce01 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-stream-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml index 2f572d102a..d829918213 100644 --- a/flinkx-teradata/flinkx-teradata-reader/pom.xml +++ b/flinkx-teradata/flinkx-teradata-reader/pom.xml @@ -11,25 +11,6 @@ flinkx-teradata-reader - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml index 4101bf741e..c9c8d937e2 100644 --- a/flinkx-teradata/flinkx-teradata-writer/pom.xml +++ b/flinkx-teradata/flinkx-teradata-writer/pom.xml @@ -11,25 +11,6 @@ flinkx-teradata-writer - - - dev - - ${package.name} - - - true - - - - - release - - release - - - - com.dtstack.flinkx diff --git a/pom.xml b/pom.xml index 4dafd7b4a0..0f640a0366 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,25 @@ + + + dev + + ${git.branch} + + + true + + + + + release + + release + + + + From 874fe97e362284282ce54a022ea8a12089719f3f Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 17 Jan 2020 15:52:36 +0800 Subject: [PATCH 011/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=8C=85=E5=90=8D?= =?UTF-8?q?=E7=A7=B0=E7=94=9F=E6=88=90=E8=A7=84=E5=88=99?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-binlog/flinkx-binlog-reader/pom.xml | 2 +- flinkx-carbondata/flinkx-carbondata-reader/pom.xml | 2 +- flinkx-carbondata/flinkx-carbondata-writer/pom.xml | 2 +- flinkx-cassandra/flinkx-cassandra-reader/pom.xml | 2 +- flinkx-cassandra/flinkx-cassandra-writer/pom.xml | 2 +- flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml | 2 +- flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml | 2 +- flinkx-core/pom.xml | 2 +- flinkx-db2/flinkx-db2-reader/pom.xml | 2 +- flinkx-db2/flinkx-db2-writer/pom.xml | 2 +- flinkx-es/flinkx-es-reader/pom.xml | 2 +- flinkx-es/flinkx-es-writer/pom.xml | 2 +- flinkx-ftp/flinkx-ftp-reader/pom.xml | 2 +- flinkx-ftp/flinkx-ftp-writer/pom.xml | 2 +- flinkx-gbase/flinkx-gbase-reader/pom.xml | 2 +- flinkx-gbase/flinkx-gbase-writer/pom.xml | 2 +- flinkx-hbase/flinkx-hbase-reader/pom.xml | 2 +- flinkx-hbase/flinkx-hbase-writer/pom.xml | 2 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 2 +- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 2 +- flinkx-hive/flinkx-hive-writer/pom.xml | 2 +- flinkx-kafka/flinkx-kafka-reader/pom.xml | 2 +- flinkx-kafka/flinkx-kafka-writer/pom.xml | 2 +- flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 2 +- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 2 +- flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 2 +- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 2 +- flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 2 +- flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 2 +- flinkx-kudu/flinkx-kudu-reader/pom.xml | 2 +- flinkx-kudu/flinkx-kudu-writer/pom.xml | 2 +- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 2 +- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-reader/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-writer/pom.xml | 2 +- flinkx-odps/flinkx-odps-reader/pom.xml | 2 +- flinkx-odps/flinkx-odps-writer/pom.xml | 2 +- flinkx-oracle/flinkx-oracle-reader/pom.xml | 2 +- flinkx-oracle/flinkx-oracle-writer/pom.xml | 2 +- flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 2 +- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 2 +- flinkx-polardb/flinkx-polardb-dreader/pom.xml | 2 +- flinkx-polardb/flinkx-polardb-reader/pom.xml | 2 +- flinkx-polardb/flinkx-polardb-writer/pom.xml | 2 +- flinkx-postgresql/flinkx-postgresql-reader/pom.xml | 2 +- flinkx-postgresql/flinkx-postgresql-writer/pom.xml | 2 +- flinkx-rdb/flinkx-rdb-core/pom.xml | 2 +- flinkx-rdb/flinkx-rdb-reader/pom.xml | 2 +- flinkx-rdb/flinkx-rdb-writer/pom.xml | 2 +- flinkx-redis/flinkx-redis-writer/pom.xml | 2 +- flinkx-saphana/flinkx-saphana-reader/pom.xml | 2 +- flinkx-saphana/flinkx-saphana-writer/pom.xml | 2 +- flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml | 2 +- flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml | 2 +- flinkx-stream/flinkx-stream-reader/pom.xml | 2 +- flinkx-stream/flinkx-stream-writer/pom.xml | 2 +- flinkx-teradata/flinkx-teradata-reader/pom.xml | 2 +- flinkx-teradata/flinkx-teradata-writer/pom.xml | 2 +- pom.xml | 3 ++- 60 files changed, 61 insertions(+), 60 deletions(-) diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index 92d2bdfff1..8b9da7db64 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -67,7 +67,7 @@ + tofile="${basedir}/../../plugins/binlogreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index 049e4f4d96..111fc621c3 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -89,7 +89,7 @@ + tofile="${basedir}/../../plugins/carbondatareader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index 9f7b556146..d902b905c6 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -92,7 +92,7 @@ + tofile="${basedir}/../../plugins/carbondatawriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index 2189e1fe07..feca24f0f9 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -84,7 +84,7 @@ + tofile="${basedir}/../../plugins/cassandrareader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index aaa690b80a..565ef3ca4f 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -84,7 +84,7 @@ + tofile="${basedir}/../../plugins/cassandrawriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 59f71729b8..01d798e35d 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/clickhousereader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 6f02526c41..94e72635a6 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/clickhousewriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 7925159670..1f58cadfc5 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -227,7 +227,7 @@ + tofile="${basedir}/../plugins/flinkx-${package.name}.jar" /> diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index b218614f3d..6a6f40a2c7 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/db2reader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 16439de9e9..2ff75d22e0 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/db2writer/${project.name}-${package.name}.jar" /> diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 87a9e3ac2f..227dc25d10 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/esreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 52582620ba..1cdfab73fb 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -78,7 +78,7 @@ + tofile="${basedir}/../../plugins/eswriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index e6c22653f6..1771055c60 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -95,7 +95,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpreader/${project.name}-${package.name}.jar"/> diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index c30c8c36f8..61d8ffaede 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -96,7 +96,7 @@ under the License. + tofile="${basedir}/../../plugins/ftpwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 130a443848..3082049696 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/gbasereader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index b358851a8a..da6f21b4a2 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/gbasewriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index eabd9bc5fc..17a89795f9 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/hbasereader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index e42ba78911..4ffdceb800 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/hbasewriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index 22290ecd32..f6d0dbdcc8 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -122,7 +122,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfsreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 724f008d39..a94f74c9f9 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -123,7 +123,7 @@ under the License. + tofile="${basedir}/../../plugins/hdfswriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 8d72d77a91..087e91c179 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -117,7 +117,7 @@ under the License. + tofile="${basedir}/../../plugins/hivewriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index 54524522db..8c7db9a4ca 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -57,7 +57,7 @@ + tofile="${basedir}/../../plugins/kafkareader/${project.name}-${package.name}.jar"/> diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index dcecdc9a8c..de09a7ff33 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -57,7 +57,7 @@ + tofile="${basedir}/../../plugins/kafkawriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index b796ff0e54..3fc1bc8ed2 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -76,7 +76,7 @@ + tofile="${basedir}/../../plugins/kafka09reader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index c9f84fd189..09773a6367 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -76,7 +76,7 @@ + tofile="${basedir}/../../plugins/kafka09writer/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 215692f520..3f5737c23a 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -86,7 +86,7 @@ + tofile="${basedir}/../../plugins/kafka10reader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index 10e2b5a9d5..d262c81ba1 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -86,7 +86,7 @@ + tofile="${basedir}/../../plugins/kafka10writer/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index f8f128f115..6b3f443be8 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -86,7 +86,7 @@ + tofile="${basedir}/../../plugins/kafka11reader/${project.name}-${package.name}.jar"/> diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index 8c7dd607fb..a0e8344146 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -86,7 +86,7 @@ + tofile="${basedir}/../../plugins/kafka11writer/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index aaf24473f6..c8a99fea22 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kudureader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index 1694c091cb..742fae63ab 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kuduwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 634ec94d3d..09814f6c3f 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -84,7 +84,7 @@ + tofile="${basedir}/../../plugins/mongodbreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index fe75c7d638..1d81ef36ef 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -84,7 +84,7 @@ + tofile="${basedir}/../../plugins/mongodbwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index 22749ca413..047e273350 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/mysqldreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 93ef066baf..8856cdf063 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/mysqlreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 74144d03da..82e3aa55c4 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/mysqlwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 502a3e49c6..866fd622fe 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -74,7 +74,7 @@ + tofile="${basedir}/../../plugins/odpsreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index fff64aa370..b268bf43b1 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/odpswriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index f6266b7790..52ff1880be 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/oraclereader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index 4225dfe259..c6fc7b4e2b 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/oraclewriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index 25c7e13f35..3222b8b594 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/phoenixreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index 1d5c2d519e..6bd30a5c65 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/phoenixwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index b8c6dd6cc0..2ca84a4c71 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -74,7 +74,7 @@ + tofile="${basedir}/../../plugins/polardbdreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index a8a4349fe5..8c8befae80 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -74,7 +74,7 @@ + tofile="${basedir}/../../plugins/polardbreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index f0994e59e2..7e78f53d22 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -74,7 +74,7 @@ + tofile="${basedir}/../../plugins/polardbwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index 068c9dbca4..f103f55d85 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/postgresqlreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index ec0f4f5d7d..2264b0dddb 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/postgresqlwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-rdb/flinkx-rdb-core/pom.xml b/flinkx-rdb/flinkx-rdb-core/pom.xml index 2229658781..fefc94f1e1 100644 --- a/flinkx-rdb/flinkx-rdb-core/pom.xml +++ b/flinkx-rdb/flinkx-rdb-core/pom.xml @@ -32,7 +32,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${package.name}.jar"/> diff --git a/flinkx-rdb/flinkx-rdb-reader/pom.xml b/flinkx-rdb/flinkx-rdb-reader/pom.xml index fc9ab2ab04..82c83d5493 100644 --- a/flinkx-rdb/flinkx-rdb-reader/pom.xml +++ b/flinkx-rdb/flinkx-rdb-reader/pom.xml @@ -41,7 +41,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${package.name}.jar"/> diff --git a/flinkx-rdb/flinkx-rdb-writer/pom.xml b/flinkx-rdb/flinkx-rdb-writer/pom.xml index fcbfbe90be..e286d72d92 100644 --- a/flinkx-rdb/flinkx-rdb-writer/pom.xml +++ b/flinkx-rdb/flinkx-rdb-writer/pom.xml @@ -41,7 +41,7 @@ + tofile="${basedir}/../../plugins/common/${project.name}-${package.name}.jar"/> diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 724afc3255..4a92436c6b 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -84,7 +84,7 @@ + tofile="${basedir}/../../plugins/rediswriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index c707bbf2ea..ba9e5c4375 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/saphanareader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 1ded9f5ae9..8c19fad040 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/saphanawriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index 123f212527..7300b9251f 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../plugins/sqlserverreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index e04a6800d8..e4f6ed9b36 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/sqlserverwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index d31208c4e5..d2dfa853cc 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -73,7 +73,7 @@ + tofile="${basedir}/../../plugins/streamreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index 25c9e4ce01..f106065763 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -69,7 +69,7 @@ + tofile="${basedir}/../../plugins/streamwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml index d829918213..44cc030d1f 100644 --- a/flinkx-teradata/flinkx-teradata-reader/pom.xml +++ b/flinkx-teradata/flinkx-teradata-reader/pom.xml @@ -96,7 +96,7 @@ + tofile="${basedir}/../../plugins/teradatareader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml index c9c8d937e2..bd44533374 100644 --- a/flinkx-teradata/flinkx-teradata-writer/pom.xml +++ b/flinkx-teradata/flinkx-teradata-writer/pom.xml @@ -96,7 +96,7 @@ + tofile="${basedir}/../../plugins/teradatawriter/${project.name}-${package.name}.jar" /> diff --git a/pom.xml b/pom.xml index 0f640a0366..72128c0879 100644 --- a/pom.xml +++ b/pom.xml @@ -51,6 +51,7 @@ 1.8.1 2.7.3 ${basedir}/dev + 1.8.4_rc0 @@ -128,7 +129,7 @@ release - release + ${projectVersion} From a9fdb052d6a8452ad49b5a49321770e4f53205dc Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 17 Jan 2020 16:47:30 +0800 Subject: [PATCH 012/136] =?UTF-8?q?=E6=89=93=E5=8C=85=E5=89=8Dinstall?= =?UTF-8?q?=E9=83=A8=E5=88=86=E9=A9=B1=E5=8A=A8=E5=8C=85?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- bin/install_jars.bat | 6 ++++++ bin/install_jars.sh | 11 +++++++++++ flinkx-core/pom.xml | 17 +++++++++++++++++ flinkx-teradata/flinkx-teradata-core/pom.xml | 2 ++ pom.xml | 1 + 5 files changed, 37 insertions(+) create mode 100644 bin/install_jars.bat create mode 100644 bin/install_jars.sh diff --git a/bin/install_jars.bat b/bin/install_jars.bat new file mode 100644 index 0000000000..140dd108f8 --- /dev/null +++ b/bin/install_jars.bat @@ -0,0 +1,6 @@ +call mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=../jars/db2jcc-3.72.44.jar + +call mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=../jars/ojdbc8-12.2.0.1.jar + +call mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar + diff --git a/bin/install_jars.sh b/bin/install_jars.sh new file mode 100644 index 0000000000..a43cd9a38c --- /dev/null +++ b/bin/install_jars.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +## db2 driver +mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=../jars/db2jcc-3.72.44.jar + +## oracle driver +mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=../jars/ojdbc8-12.2.0.1.jar + +## gbase driver +mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar + diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 1f58cadfc5..d4b7a3c96f 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -161,6 +161,23 @@ + + exec-maven-plugin + org.codehaus.mojo + + + Version Calculation + generate-sources + + exec + + + ${basedir}/../bin/install_jars.${scriptType} + + + + + org.apache.maven.plugins maven-compiler-plugin diff --git a/flinkx-teradata/flinkx-teradata-core/pom.xml b/flinkx-teradata/flinkx-teradata-core/pom.xml index 2af2575d7e..51ef88e633 100644 --- a/flinkx-teradata/flinkx-teradata-core/pom.xml +++ b/flinkx-teradata/flinkx-teradata-core/pom.xml @@ -11,5 +11,7 @@ flinkx-teradata-core + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 72128c0879..a4648a2b8e 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ 2.7.3 ${basedir}/dev 1.8.4_rc0 + sh From fd45c8e2d2d3822f3696caf0543f709df9abcd66 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Sat, 18 Jan 2020 16:06:30 +0800 Subject: [PATCH 013/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E5=AE=89=E8=A3=85?= =?UTF-8?q?=E4=BE=9D=E8=B5=96=E8=84=9A=E6=9C=AC?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-launcher/pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flinkx-launcher/pom.xml b/flinkx-launcher/pom.xml index 9c752548ab..d77e50835b 100644 --- a/flinkx-launcher/pom.xml +++ b/flinkx-launcher/pom.xml @@ -43,17 +43,17 @@ org.apache.hadoop hadoop-common - 2.7.3.2.6.4.91-3 + 2.7.3 org.apache.hadoop hadoop-mapreduce-client-core - 2.7.3.2.6.4.91-3 + 2.7.3 org.apache.hadoop hadoop-hdfs - 2.7.3.2.6.4.91-3 + 2.7.3 @@ -63,7 +63,7 @@ org.apache.maven.plugins maven-shade-plugin - 3.0.0 + 3.1.0 package From ebd755ac64e64ecc34e0c6532d0c3c90948631d1 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Sat, 18 Jan 2020 17:12:37 +0800 Subject: [PATCH 014/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 6 +- docs/carbondatareader.md | 162 ++++++----- docs/carbondatawriter.md | 274 +++++++----------- docs/rdbreader.md | 4 +- docs/rdbwriter.md | 10 +- .../main/java/com/dtstack/flinkx/Main.java | 2 - .../com/dtstack/flinkx/test/LocalTest.java | 2 - 7 files changed, 201 insertions(+), 259 deletions(-) diff --git a/README.md b/README.md index f86c810f83..363155418c 100644 --- a/README.md +++ b/README.md @@ -227,7 +227,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 ### 5.1 读取插件 -* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase)](docs/rdbreader.md) +* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbreader.md) * [分库分表读取插件](docs/rdbdreader.md) * [HDFS读取插件](docs/hdfsreader.md) * [HBase读取插件](docs/hbasereader.md) @@ -240,10 +240,11 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [MySQL binlog读取插件](docs/binlog.md) * [KafKa读取插件](docs/kafkareader.md) * [Kudu读取插件](docs/kudureader.md) +* [Cassandra读取插件](docs/cassandrareader.md) ### 5.2 写入插件 -* [关系数据库写入插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase)](docs/rdbwriter.md) +* [关系数据库写入插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbwriter.md) * [HDFS写入插件](docs/hdfswriter.md) * [HBase写入插件](docs/hbasewriter.md) * [Elasticsearch写入插件](docs/eswriter.md) @@ -256,6 +257,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [Kafka写入插件](docs/kafkawriter.md) * [Hive写入插件](docs/hivewriter.md) * [Kudu写入插件](docs/kuduwriter.md) +* [Cassandra写入插件](docs/cassandrawriter.md) [断点续传和实时采集功能介绍](docs/restore.md) diff --git a/docs/carbondatareader.md b/docs/carbondatareader.md index 66c873a57c..9911572b43 100644 --- a/docs/carbondatareader.md +++ b/docs/carbondatareader.md @@ -69,104 +69,108 @@ ] } } - - ``` ## 2. 参数说明 * **name** - - * 描述:插件名,此处只能填carbondatareader,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
+ + * 描述:插件名,此处只能填carbondatareader,否则Flinkx将无法正常加载该插件包。 + * 必选:是
+ + * 默认值:无
* **path** - - * 描述:carbondata表的存储路径 - - * 必选:是
- - * 默认值:无
+ + * 描述:carbondata表的存储路径 + + * 必选:是
+ + * 默认值:无
* **table** - - * 描述:carbondata表名
- - * 必选:否
- - * 默认值:无
+ + * 描述:carbondata表名
+ + * 必选:否
+ + * 默认值:无
* **database** - - * 描述:carbondata库名
- - * 必选:否
- - * 默认值:无
+ + * 描述:carbondata库名
+ + * 必选:否
+ + * 默认值:无
* **filter** - - * 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value
- - col为列名;
- op为关系运算符,包括=,>,>=,<,<=;
- value为字面值,如1234, "ssss"
- - * 必选:否
- - * 默认值:无
- + + * 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value
+ + col为列名;
+ + op为关系运算符,包括=,>,>=,<,<=;
+ + value为字面值,如1234, "ssss"
+ + * 必选:否
+ + * 默认值:无
* **column** + + * 描述:所配置的表中需要同步的字段集合。 + + 字段包括表字段和常量字段,
+ + 表字段的格式: + + ``` + { + "name": "col1", + "type": "string" + } + ``` + + 常量字段的格式: + + { + + "value": "12345", + + "type": "string" + + } - * 描述:所配置的表中需要同步的字段集合。 - 字段包括表字段和常量字段,
- 表字段的格式: - ``` - { - "name": "col1", - "type": "string" - } - ``` - - 常量字段的格式: - { - "value": "12345", - "type": "string" - } - - - * 必选:是
- - * 默认值:无
+ * 必选:是
+ + * 默认值:无
* **hadoopConfig** - - * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
- - ``` - { - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", - "dfs.nameservices": "ns1", - "fs.defaultFS": "hdfs://ns1" - } - } - ``` + + * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
+ + ``` + { + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", + "dfs.nameservices": "ns1", + "fs.defaultFS": "hdfs://ns1" + } + } + ``` * **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。
- - * 必选:是
- - * 默认值:无
- + + * 描述:Hadoop hdfs文件系统namenode节点地址。
+ + * 必选:是
+ + * 默认值:无
## 3. 数据类型 @@ -186,10 +190,8 @@ * DATE * TIMESTAMP - 不支持如下数据类型 * arrays: ARRAY * structs: STRUCT * maps: MAP - diff --git a/docs/carbondatawriter.md b/docs/carbondatawriter.md index 123bb15c72..4fd4dfa89e 100644 --- a/docs/carbondatawriter.md +++ b/docs/carbondatawriter.md @@ -1,176 +1,118 @@ -# Carbondata写入插件(carbondatawriter) +# Cassandra写入插件(cassandrareader) ## 1. 配置样例 -``` - +```json { - "job": { + "job": { + "content": [ + { + "reader": { + + }, + "writer": { + "name": "cassandrawriter", + "parameter": { + "host": "101.37.175.174", + "keyspace": "tp", + "table": "emp", + "column": ["emp_id", "emp_name", "emp_city", "emp_phone", "emp_sal"] + } + } + } + ], "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "root", - "password": "111111", - "column": [ - "v", - "id" - ], - "connection": [ - { - "table": [ - "tt" - ], - "jdbcUrl": [ - "jdbc:mysql://rdos1:3306/hyf?useCursorFetch=true" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "carbondatawriter", - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/carbon.store1/sb/sb500", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", - "dfs.nameservices": "ns1" - }, - "defaultFS": "hdfs://ns1", - "table": "sb500", - "database": "sb", - "writeMode": "overwrite", - "column": [ - "a", - "b" - ], - "batchSize": 1 - } - } - } - ] - } + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 10000, + "percentage": 100 + } + } + } } - - ``` ## 2. 参数说明 -* **name** - - * 描述:插件名,此处只能填carbondatawriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- * 默认值:无
- - -* **path** - - * 描述:carbondata表的存储路径 - - * 必选:是
- - * 默认值:无
- -* **table** - - * 描述:carbondata表名
- - * 必选:否
- - * 默认值:无
- -* **database** - - * 描述:carbondata库名
- - * 必选:否
- - * 默认值:无
- -* **filter** - - * 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value
- - col为列名;
- op为关系运算符,包括=,>,>=,<,<=;
- value为字面值,如1234, "ssss"
- - * 必选:否
- - * 默认值:无
- - -* **column** - - * 描述:所配置的表中需要同步的字段名列表 - - * 必选:是
- - * 默认值:无
- -* **hadoopConfig** - - * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
- - ``` - { - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", - "dfs.nameservices": "ns1", - "fs.defaultFS": "hdfs://ns1" - } - } - ``` - -* **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。
- - * 必选:是
- - * 默认值:无
- - -## 3. 数据类型 - -支持如下数据类型 - -* SMALLINT -* INT/INTEGER -* BIGINT -* DOUBLE -* DECIMAL -* FLOAT -* BYTE -* BOOLEAN -* STRING -* CHAR -* VARCHAR -* DATE -* TIMESTAMP - - -不支持如下数据类型 - -* arrays: ARRAY -* structs: STRUCT -* maps: MAP \ No newline at end of file +- **host** + + - 描述:数据库地址 + + - 必选:是 + + - 默认值:无 + +- **port** + + - 描述:端口 + + - 必选:否 + + - 默认值:9042 + +- **username** + + - 描述:用户名 + + - 必选:否 + + - 默认值:无 + +- **password** + + - 描述:密码 + + - 必选:否 + + - 默认值:无 + +- **column** + + - 描述:要读取的字段 + + - 必选:否 + + - 默认值:无 + +- **keyspace** + + - 描述:需要同步的表所在的keyspace + + - 必选:是 + + - 默认值:无 + +- **table** + + - 描述:要查询的表 + + - 必选:是 + + - 默认值:无 + +- **asyncWrite** + + - 描述:是否同步写入数据。 + + - 必选:否 + + - 默认值:false + +- **batchSize** + + - 描述:一次批量提交的记录数大小 + + - 必选:否 + + - 默认值:1 + +- **consistancyLevel** + + - 描述:数据一致性级别。可选ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY|TWO|THREE|LOCAL_ONE + + - 必选:否 + + - 默认值:无 diff --git a/docs/rdbreader.md b/docs/rdbreader.md index 30b7a9bbe1..86fa5af78c 100644 --- a/docs/rdbreader.md +++ b/docs/rdbreader.md @@ -47,7 +47,7 @@ } }], "setting": { - + } } } @@ -57,7 +57,7 @@ * **name** - * 描述:插件名,此处填写插件名称,当前支持的关系数据库插件包括:mysqlreader,oraclereader,sqlserverreader,postgresqlreader,db2reader,gbasereader, clickhousereader, polardbreader。 + * 描述:插件名,此处填写插件名称,当前支持的关系数据库插件包括:mysqlreader,oraclereader,sqlserverreader,postgresqlreader,db2reader,gbasereader, clickhousereader, polardbreader,teradatareader,saphanareader,phoenixreader。 * 必选:是 * 默认值:无 diff --git a/docs/rdbwriter.md b/docs/rdbwriter.md index d402f96470..b1cc7f5b8b 100644 --- a/docs/rdbwriter.md +++ b/docs/rdbwriter.md @@ -38,7 +38,7 @@ * **name** - * 描述:插件名,此处可填写:mysqlwriter,oraclewriter,sqlserverwriter,postgresqlwriter,db2writer,gbasewriter,clickhousewriter, polardbwriter。 + * 描述:插件名,此处可填写:mysqlwriter,oraclewriter,sqlserverwriter,postgresqlwriter,db2writer,gbasewriter,clickhousewriter, polardbwriter,teradatawriter,saphanawriter,phoenixwriter。 * 必选:是 @@ -134,18 +134,18 @@ * 默认值:无 - * **insertSqlMode** - + * **insertSqlMode** + * 描述:控制写入数据到目标表采用 `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 * 注意: - + * 此参数只针对PostgreSQL写入插件有效 * 目前该参数值固定传入 `copy`,否则抛出提示为`not support insertSqlMode`的`RuntimeException` * 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 - + * 必选:否 * 默认值:无 diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 69742ff1d4..48bdc978d9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -42,8 +42,6 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import org.apache.flink.streaming.runtime.partitioner.DTRebalancePartitioner; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 4c8bdd210e..7177bea621 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -86,8 +86,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import org.apache.flink.streaming.runtime.partitioner.DTRebalancePartitioner; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From d552b189607359f8c4aac57064fe7748cbe8c232 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Sat, 18 Jan 2020 17:21:32 +0800 Subject: [PATCH 015/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/cassandrareader.md | 134 ++++++++++++++++++++++++++++++++++++++++ docs/cassandrawriter.md | 0 2 files changed, 134 insertions(+) create mode 100644 docs/cassandrareader.md create mode 100644 docs/cassandrawriter.md diff --git a/docs/cassandrareader.md b/docs/cassandrareader.md new file mode 100644 index 0000000000..f3d24702d5 --- /dev/null +++ b/docs/cassandrareader.md @@ -0,0 +1,134 @@ +# Cassandra读取插件(cassandrareader) + +## 1. 配置样例 + +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "host": "127.0.0.1", + "port":9042, + "username":"username", + "password":"password", + "useSSL":false, + + "keyspace": "tp", + "table": "emp", + "column": [ + "emp_id", + "emp_city", + "emp_name", + "emp_phone", + "emp_sal" + ], + "allowFiltering":false, + "where":null, + "connectionsPerHost":8, + "maxPendingPerConnection":128, + "consistancyLevel":null + + }, + "name": "cassandrareader" + }, + "writer": { + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 10000, + "percentage": 100 + } + } + } +} +``` + +## 2. 参数说明 + +- **host** + + - 描述:数据库地址 + + - 必选:是 + + - 默认值:无 + +- **port** + + - 描述:端口 + + - 必选:否 + + - 默认值:9042 + +- **username** + + - 描述:用户名 + + - 必选:否 + + - 默认值:无 + +- **password** + + - 描述:密码 + + - 必选:否 + + - 默认值:无 + +- **column** + + - 描述:要读取的字段 + + - 必选:否 + + - 默认值:无 + +- **keyspace** + + - 描述:需要同步的表所在的keyspace + + - 必选:是 + + - 默认值:无 + +- **table** + + - 描述:要查询的表 + + - 必选:是 + + - 默认值:无 + +- **allowFiltering** + + - 描述:是否在服务端过滤数据。 + + - 必选:否 + + - 默认值:false + +- **where** + + - 描述:过滤条件where之后的表达式 + + - 必选:否 + + - 默认值:无 + +- **consistancyLevel** + + - 描述:数据一致性级别。可选ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY|TWO|THREE|LOCAL_ONE + + - 必选:否 + + - 默认值:无 diff --git a/docs/cassandrawriter.md b/docs/cassandrawriter.md new file mode 100644 index 0000000000..e69de29bb2 From 593ccecca3a9b8fa4d46c8c91c2602028072e764 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Sat, 18 Jan 2020 17:30:42 +0800 Subject: [PATCH 016/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=89=93=E5=8C=85?= =?UTF-8?q?=E7=BC=96=E8=AF=91=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 363155418c..a5311261b9 100644 --- a/README.md +++ b/README.md @@ -36,8 +36,13 @@ 进入项目根目录,使用maven打包: +windows平台 ``` -mvn clean package -Dmaven.test.skip +mvn clean package -DskipTests -Prelease -DscriptType=bat +``` +unix平台 +``` +mvn clean package -DskipTests -Prelease -DscriptType=sh ``` 打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 From cb5bb5514ce06ef58b312a2417c421aeab3cf0af Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 13 Mar 2020 19:57:27 +0800 Subject: [PATCH 017/136] merge --- README.md | 2 - docs/logminer.md | 189 ---- docs/sqlservercdc.md | 188 ---- flinkx-examples/examples/sqlserver_kafka.json | 43 - .../flinkx-oraclelogminer-reader/pom.xml | 101 -- .../oraclelogminer/format/LogMinerConfig.java | 169 --- .../format/OracleLogMinerInputFormat.java | 404 -------- .../OracleLogMinerInputFormatBuilder.java | 46 - .../reader/OraclelogminerReader.java | 74 -- .../oraclelogminer/util/LogMinerUtil.java | 290 ------ flinkx-oraclelogminer/pom.xml | 26 - flinkx-saphana/flinkx-saphana-writer/pom.xml | 101 -- .../flinkx-sqlservercdc-core/pom.xml | 22 - .../flinkx/sqlservercdc/ChangeTable.java | 117 --- .../sqlservercdc/ChangeTablePointer.java | 84 -- .../com/dtstack/flinkx/sqlservercdc/Lsn.java | 142 --- .../sqlservercdc/SqlServerCdcConfigKeys.java | 45 - .../flinkx/sqlservercdc/SqlServerCdcUtil.java | 310 ------ .../flinkx/sqlservercdc/SqlserverCdcEnum.java | 91 -- .../dtstack/flinkx/sqlservercdc/TableId.java | 184 ---- .../flinkx/sqlservercdc/TxLogPosition.java | 101 -- .../flinkx-sqlservercdc-reader/pom.xml | 69 -- .../format/SqlserverCdcInputFormat.java | 208 ---- .../SqlserverCdcInputFormatBuilder.java | 98 -- .../listener/SqlServerCdcListener.java | 204 ---- .../reader/SqlservercdcReader.java | 79 -- flinkx-sqlservercdc/pom.xml | 29 - .../flinkx-teradata-reader/pom.xml | 108 -- .../flinkx-teradata-writer/pom.xml | 108 -- flinkx-test/pom.xml | 11 - .../com/dtstack/flinkx/test/LocalTest.java | 4 - .../flinkx/test/PluginNameConstrant.java | 2 - .../dev_test_job/logminer_stream.json | 63 -- .../dev_test_job/stream_template.json | 38 +- .../flinkx/test/annotation/Plugin.java | 39 - .../flinkx/test/annotation/PluginType.java | 10 - .../flinkx/test/annotation/TestCase.java | 16 - .../dtstack/flinkx/test/core/BaseTest.java | 130 --- .../test/core/DefaultParameterReplace.java | 72 -- .../dtstack/flinkx/test/core/FileUtil.java | 63 -- .../flinkx/test/core/ParameterReplace.java | 10 - .../test/core/ReaderUnitTestLauncher.java | 152 --- .../flinkx/test/core/UnitTestLauncher.java | 148 --- .../test/core/WriterUnitTestLauncher.java | 138 --- .../flinkx/test/core/result/BaseResult.java | 43 - .../flinkx/test/core/result/ReaderResult.java | 47 - .../flinkx/test/core/result/WriterResult.java | 34 - .../flinkx/test/core/source/DataSource.java | 16 - .../test/core/source/DataSourceFactory.java | 46 - .../test/core/source/HdfsDataSource.java | 42 - .../test/core/source/HiveDataSource.java | 40 - .../test/core/source/StaticDataSource.java | 49 - .../source/embedded/EmbeddedHDFSService.java | 71 -- .../source/embedded/EmbeddedHiveService.java | 119 --- .../embedded/hive/AbstractHiveServer.java | 101 -- .../core/source/embedded/hive/HiveServer.java | 38 - .../embedded/hive/InternalHiveServer.java | 52 - .../hive/InternalMetaStoreServer.java | 85 -- .../flinkx/test/hdfs/BaseHdfsTest.java | 59 -- .../test/hdfs/reader/HdfsReaderTest.java | 39 - .../test/hdfs/writer/HdfsWriterTest.java | 123 --- .../dtstack/flinkx/test/mysql/MySqlTest.java | 41 - .../test/mysql/reader/MySqlReaderTest.java | 232 ----- .../test/rdb/JdbcReaderParameterReplace.java | 70 -- .../com/dtstack/flinkx/test/rdb/JdbcTest.java | 113 -- .../test/stream/reader/StreamReaderTest.java | 92 -- .../test/stream/writer/StreamWriterTest.java | 87 -- .../flinkx/test/util/TestCaseDocImporter.java | 31 - .../apache/hadoop/io/nativeio/NativeIO.java | 967 ------------------ .../resources/testCase/dataSource/hdfs.json | 14 - .../resources/testCase/dataSource/mysql.json | 22 - .../resources/testCase/dataSource/source.json | 49 - .../resources/testCase/dataSource/stream.json | 14 - .../testCase/hdfs/writer/numberTest.json | 94 -- .../testCase/mysql/reader/customSqlTest.json | 44 - .../testCase/mysql/reader/filterTest.json | 44 - .../mysql/reader/incrementUseMaxFunc.json | 45 - .../reader/incrementWithStartLocation.json | 45 - .../reader/incrementWithoutStartLocation.json | 44 - .../testCase/mysql/reader/numberTest.json | 39 - .../stream/reader/multiChannelNumberTest.json | 40 - .../reader/singleChannelNumberTest.json | 40 - .../writer/singleChannelNumberTest.json | 30 - pom.xml | 2 - 84 files changed, 1 insertion(+), 7730 deletions(-) delete mode 100644 docs/logminer.md delete mode 100644 docs/sqlservercdc.md delete mode 100644 flinkx-examples/examples/sqlserver_kafka.json delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java delete mode 100644 flinkx-oraclelogminer/pom.xml delete mode 100644 flinkx-saphana/flinkx-saphana-writer/pom.xml delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/pom.xml delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTable.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTablePointer.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/Lsn.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcConfigKeys.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlserverCdcEnum.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TableId.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TxLogPosition.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormat.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormatBuilder.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java delete mode 100644 flinkx-sqlservercdc/pom.xml delete mode 100644 flinkx-teradata/flinkx-teradata-reader/pom.xml delete mode 100644 flinkx-teradata/flinkx-teradata-writer/pom.xml delete mode 100644 flinkx-test/src/main/resources/dev_test_job/logminer_stream.json delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/Plugin.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/PluginType.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/TestCase.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/BaseTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/DefaultParameterReplace.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/FileUtil.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ParameterReplace.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ReaderUnitTestLauncher.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/UnitTestLauncher.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/WriterUnitTestLauncher.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/BaseResult.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/ReaderResult.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/WriterResult.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSource.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSourceFactory.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HdfsDataSource.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HiveDataSource.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/StaticDataSource.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHDFSService.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHiveService.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/AbstractHiveServer.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/HiveServer.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalHiveServer.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalMetaStoreServer.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/BaseHdfsTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/reader/HdfsReaderTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/writer/HdfsWriterTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/MySqlTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/reader/MySqlReaderTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcReaderParameterReplace.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/reader/StreamReaderTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/writer/StreamWriterTest.java delete mode 100644 flinkx-test/src/test/java/com/dtstack/flinkx/test/util/TestCaseDocImporter.java delete mode 100644 flinkx-test/src/test/java/org/apache/hadoop/io/nativeio/NativeIO.java delete mode 100644 flinkx-test/src/test/resources/testCase/dataSource/hdfs.json delete mode 100644 flinkx-test/src/test/resources/testCase/dataSource/mysql.json delete mode 100644 flinkx-test/src/test/resources/testCase/dataSource/source.json delete mode 100644 flinkx-test/src/test/resources/testCase/dataSource/stream.json delete mode 100644 flinkx-test/src/test/resources/testCase/hdfs/writer/numberTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/customSqlTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/filterTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/incrementUseMaxFunc.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithStartLocation.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithoutStartLocation.json delete mode 100644 flinkx-test/src/test/resources/testCase/mysql/reader/numberTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/stream/reader/multiChannelNumberTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/stream/reader/singleChannelNumberTest.json delete mode 100644 flinkx-test/src/test/resources/testCase/stream/writer/singleChannelNumberTest.json diff --git a/README.md b/README.md index 87b66b06b0..be38aa2e6a 100644 --- a/README.md +++ b/README.md @@ -323,8 +323,6 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [Kudu读取插件](docs/kudureader.md) * [Cassandra读取插件](docs/cassandrareader.md) * [Emqx读取插件](docs/emqxreader.md) -* [Oracle实时采集插件](docs/logminer.md) -* [SqlServerCdc实时采集插件](docs/sqlservercdc.md) ### 5.2 写入插件 diff --git a/docs/logminer.md b/docs/logminer.md deleted file mode 100644 index 70b855af58..0000000000 --- a/docs/logminer.md +++ /dev/null @@ -1,189 +0,0 @@ -# Kudu读取插件(kudureader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "jdbcUrl": "jdbc:oracle:thin:@127.0.0.1:1521:xe", - "username": "kminer", - "password": "kminerpass", - "listenerTables": "SCHEMA1.*", - "listenerOperations": "UPDATE,INSERT,DELETE", - - "startSCN": "482165", - "readPosition": "current", - "startTime": 1576540477000, - "pavingData": true, - "queryTimeout": 20000 - }, - "name": "oraclelogminerreader" - }, - "writer": { - "parameter": { - "print": true - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,oraclelogminerreader。 - - * 必选:是 - - * 默认值:无 - -* **jdbcUrl** - - * 描述:Oracle连接url。 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述: 用户名。 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述: 密码。 - - * 必选:是 - - * 默认值:无 - -* **listenerTables** - - - 描述: 要监听的schema和table,使用正则进行过滤。 - - - 必选:否 - - - 默认值:无 - -* **queryTimeout** - - - 描述: 查询超时时间 - - - 必选:否 - - - 默认值:无 - -* **listenerOperations** - - - 描述: 要监听的事件,可多选:UPDATE、INSERT、DELETE - - - 必选:否 - - - 默认值:UPDATE,INSERT,DELETE - -* **readPosition** - - * 描述:读取位置类型,可选:all、current、time、scn - - * 必选:否 - - * 默认值:current - - * 类型说明: - - * all:读取全部的事件; - - * current:从任务开始运行时采集事件; - - * time:从指定的时间开始采集,使用此类型时需要提供时间配置"startTime",格式为毫秒级的时间戳; - - * scn:从指定的scn开始采集,使用此类型时需要提供scn配置"scn"; - -* **pavingData** - - * 描述:是否将解析出的json数据拍平 - - * 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: - - ```json - { - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before_id":1, - "after_id":2 - } - ``` - - pavingData为false时: - - ```json - { - "message":{ - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before_id":{ - "id":1 - }, - "after_id":{ - "id":2 - } - } - } - ``` - - 其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 - - * 必选:否 - - * 默认值:false - -## 3.数据库配置 - -数据库必须处于archivelog模式,并且必须启用补充日志记录,配置过程如下: - -```sql - sqlplus / as sysdba - SQL>shutdown immediate - SQL>startup mount - SQL>alter database archivelog; - SQL>alter database open; - SQL>alter database add supplemental log data (all) columns; -``` - -连接数据库的用户必须有对应的权限,如果用户是 DBA角色,则不用进行设置,配置过程如下: - -```sql - create role logmnr_role; - grant create session to logmnr_role; - grant execute_catalog_role,select any transaction ,select any dictionary to logmnr_role; - create user kminer identified by kminerpass; - grant logmnr_role to kminer; - alter user kminer quota unlimited on users; -``` diff --git a/docs/sqlservercdc.md b/docs/sqlservercdc.md deleted file mode 100644 index 3bf155f3a4..0000000000 --- a/docs/sqlservercdc.md +++ /dev/null @@ -1,188 +0,0 @@ -# SqlServer CDC读取插件(*reader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [{ - "reader" : { - "parameter" : { - "username" : "sa", - "password" : "Password!", - "url" : "jdbc:sqlserver://kudu4:1433;databaseName=testDB", - "databaseName" : "testDB", - "tableList" : [ - "dbo.customers", - "dbo.orders" - ], - "cat" : "insert,update", - "pavingData" : true, - "pollInterval" : 1000, - "lsn" : "00000032:00002040:0005" - }, - "name" : "sqlservercdcreader" - }, - "writer": { - - } - }] - }, - "setting": { - - } -} -``` -## 2. 使用说明 - -使用该插件前,需要对数据库及表启用SqlServerCDC功能。 - - * 1、查询SqlServer数据库版本 - * SqlServer自2008版本开始支持CDC(变更数据捕获)功能,但若想使用SqlServer CDC实时采集插件需SqlServer版本为2017及以上 -```sql - SELECT @@VERSION -``` - * 2、查询当前用户权限,必须为 sysadmin 固定服务器角色的成员才允许对数据库启用CDC(变更数据捕获)功能 -```sql - exec sp_helpsrvrolemember 'sysadmin' -``` - * 3、查询数据库是否已经启用CDC(变更数据捕获)功能 - * 0:未启用;1:启用 -```sql - select is_cdc_enabled, name from sys.databases where name = 'name' -``` - * 4、对数据库数据库启用CDC(变更数据捕获)功能 -```sql - EXEC sys.sp_cdc_enable_db -``` - * 5、对表启用CDC(变更数据捕获)功能 - * source_schema:表所在的schema名称 - * source_name:表名 - * role_name:访问控制角色名称,此处为null不设置访问控制 - * supports_net_changes:是否为捕获实例生成一个净更改函数,0:否;1:是 -```sql - EXEC sys.sp_cdc_enable_table - @source_schema = 'dbo', - @source_name = 'test', - @role_name = NULL, - @supports_net_changes = 0; -``` - * 6、查询表是否已经启用CDC(变更数据捕获)功能 - * 0:未启用;1:启用 -```sql - select name,is_tracked_by_cdc from sys.tables where name = 'test'; -``` - -具体启用过程请参照SqlServer[官方文档](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-ver15) - -## 3. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称。 - - * 必选:是 - - * 默认值:无 - -* **jdbcUrl** - - * 描述:SqlServer数据库的jdbc连接字符串,参考文档:[SqlServer官方文档](https://docs.microsoft.com/zh-cn/sql/connect/jdbc/overview-of-the-jdbc-driver?view=sql-server-2017) - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **tableList** - - * 描述:需要解析的数据表,表必须已启用CDC,格式为schema.table - - * 必选:否 - - * 默认值:无 - -* **cat** - - * 描述:需要解析的数据更新类型,包括insert、update、delete三种。 - - * 注意:以英文逗号分割的格式填写。 - - * 必选:是 - - * 默认值:无 - -* **pollInterval** - - * 描述:监听拉取SqlServer CDC数据库间隔时间。 - - * 注意:该值越小,采集延迟时间越小,给数据库的访问压力越大。 - - * 必选:否 - - * 默认值:1000 - -* **lsn** - - * 描述:要读取SqlServer CDC日志序列号的开始位置。 - - * 注意:该值越小,采集延迟时间越小,给数据库的访问压力越大。 - - * 必选:否 - -* **pavingData** - - * 描述:是否将解析出的json数据拍平 - - * 示例:假设解析的表为tb1,schema为dbo,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: - - ```json - { - "type":"update", - "schema":"dbo", - "table":"customers", - "lsn":"00000032:00002038:0005", - "ingestion":1475129582923642, - "before_id":1, - "after_id":2 - } - ``` - - pavingData为false时: - - ```json - { - "type":"update", - "schema":"dbo", - "table":"customers", - "lsn":"00000032:00004a38:0007", - "ingestion":1481628798880038, - "before":{ - "id":1 - }, - "after":{ - "id":2 - } - } - ``` - - 其中:ingestion是插件解析这条数据的纳秒时间,lsn是该数据变更的日志序列号 - - * 必选:否 - - * 默认值:false \ No newline at end of file diff --git a/flinkx-examples/examples/sqlserver_kafka.json b/flinkx-examples/examples/sqlserver_kafka.json deleted file mode 100644 index 2b49eae799..0000000000 --- a/flinkx-examples/examples/sqlserver_kafka.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "username" : "sa", - "password" : "Password!", - "url" : "jdbc:sqlserver://kudu4:1433;databaseName=testDB", - "databaseName" : "testDB", - "cat" : "insert,update", - "tableList" : [ - "dbo.customers", - "dbo.orders" - ], - "pavingData" : true - }, - "name" : "sqlservercdcreader" - }, - "writer" : { - "parameter" : { - "producerSettings" : { - "zookeeper.connect" : "kudu1:2181/kafka100", - "bootstrap.servers" : "kudu5:9092" - }, - "topic" : "tudou" - }, - "name" : "kafkawriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { - }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml deleted file mode 100644 index 81611c2e6a..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - flinkx-oraclelogminer - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-oraclelogminer-reader - - - - com.github.noraui - ojdbc8 - 12.2.0.1 - - - - com.github.jsqlparser - jsqlparser - 1.2 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.oracleReader.io.netty - - - com.google.guava - shade.oracleReader.com.google - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java deleted file mode 100644 index 2dd82536e6..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.oraclelogminer.format; - -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.Serializable; -import java.util.List; - -/** - * @author jiangbo - * @date 2019/12/14 - */ -public class LogMinerConfig implements Serializable { - - private String driverName = "oracle.jdbc.driver.OracleDriver"; - - private String jdbcUrl; - - private String username; - - private String password; - - private int fetchSize = 1; - - private String listenerTables; - - private String cat = "UPDATE,INSERT,DELETE"; - - /** - * 读取位置: all, current, time, scn - */ - private String readPosition = "current"; - - /** - * 毫秒级时间戳 - */ - private long startTime = 0; - - @JsonProperty("startSCN") - private String startScn = ""; - - private boolean pavingData = false; - - private List table; - - private Long queryTimeout; - - public Long getQueryTimeout() { - return queryTimeout; - } - - public void setQueryTimeout(Long queryTimeout) { - this.queryTimeout = queryTimeout; - } - - public List getTable() { - return table; - } - - public void setTable(List table) { - this.table = table; - } - - public String getReadPosition() { - return readPosition; - } - - public void setReadPosition(String readPosition) { - this.readPosition = readPosition; - } - - public long getStartTime() { - return startTime; - } - - public void setStartTime(long startTime) { - this.startTime = startTime; - } - - public String getStartScn() { - return startScn; - } - - public void setStartScn(String startScn) { - this.startScn = startScn; - } - - public boolean getPavingData() { - return pavingData; - } - - public void setPavingData(boolean pavingData) { - this.pavingData = pavingData; - } - - public String getCat() { - return cat; - } - - public void setCat(String cat) { - this.cat = cat; - } - - public String getListenerTables() { - return listenerTables; - } - - public void setListenerTables(String listenerTables) { - this.listenerTables = listenerTables; - } - - public int getFetchSize() { - return fetchSize; - } - - public void setFetchSize(int fetchSize) { - this.fetchSize = fetchSize; - } - - public String getDriverName() { - return driverName; - } - - public void setDriverName(String driverName) { - this.driverName = driverName; - } - - public String getJdbcUrl() { - return jdbcUrl; - } - - public void setJdbcUrl(String jdbcUrl) { - this.jdbcUrl = jdbcUrl; - } - - public String getUsername() { - return username; - } - - public void setUsername(String username) { - this.username = username; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } -} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java deleted file mode 100644 index 05d2a23204..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java +++ /dev/null @@ -1,404 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.oraclelogminer.format; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.oraclelogminer.util.LogMinerUtil; -import com.dtstack.flinkx.restore.FormatState; -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.RetryUtil; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang.time.DateFormatUtils; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.types.Row; - -import java.io.IOException; -import java.sql.*; -import java.util.concurrent.Callable; - -/** - * @author jiangbo - * @date 2019/12/14 - * - * 名词说明: - * SCN 即系统改变号(System Change Number) - */ -public class OracleLogMinerInputFormat extends BaseRichInputFormat { - - public LogMinerConfig logMinerConfig; - - private transient Connection connection; - - private transient CallableStatement logMinerStartStmt; - - private transient PreparedStatement logMinerSelectStmt; - - private transient ResultSet logMinerData; - - private Long offsetScn; - - private Long scnCopy; - - private boolean skipRecord = true; - - private static final int RETRY_TIMES = 3; - - private static final int SLEEP_TIME = 2000; - - @Override - protected InputSplit[] createInputSplitsInternal(int i) throws Exception { - return new InputSplit[]{new GenericInputSplit(1,1)}; - } - - @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - - try { - ClassUtil.forName(logMinerConfig.getDriverName(), getClass().getClassLoader()); - - connection = RetryUtil.executeWithRetry(new Callable() { - @Override - public Connection call() throws Exception { - return DriverManager.getConnection(logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), logMinerConfig.getPassword()); - } - }, RETRY_TIMES, SLEEP_TIME,false); - - LOG.info("获取连接成功,url:{}, username:{}", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername()); - } catch (Exception e){ - LOG.error("获取连接失败,url:{}, username:{}", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername()); - throw new RuntimeException(e); - } - } - - @Override - protected void openInternal(InputSplit inputSplit) throws IOException { - initOffset(); - - startLogMiner(); - startSelectData(); - } - - private void initOffset(){ - if(formatState != null && formatState.getState() != null){ - offsetScn = Long.parseLong(formatState.getState().toString()); - } else { - offsetScn = 0L; - } - - // 恢复位置不为0,则获取上一次读取的日志文件的起始位置开始读取 - if(offsetScn != 0L){ - scnCopy = offsetScn; - offsetScn = getLogFileStartPositionByScn(scnCopy); - return; - } - - // 恢复位置为0,则根据配置项进行处理 - if(ReadPosition.ALL.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ - skipRecord = false; - // 获取最开始的scn - offsetScn = getMinScn(); - } else if(ReadPosition.CURRENT.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ - skipRecord = false; - offsetScn = getCurrentScn(); - } else if(ReadPosition.TIME.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ - skipRecord = false; - - // 根据指定的时间获取对应时间段的日志文件的起始位置 - if (logMinerConfig.getStartTime() == 0) { - throw new RuntimeException("读取模式为[time]时必须指定[startTime]"); - } - - offsetScn = getLogFileStartPositionByTime(logMinerConfig.getStartTime()); - } else if(ReadPosition.SCN.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ - scnCopy = Long.parseLong(logMinerConfig.getStartScn()); - - // 根据指定的scn获取对应日志文件的起始位置 - if(StringUtils.isEmpty(logMinerConfig.getStartScn())){ - throw new RuntimeException("读取模式为[scn]时必须指定[startSCN]"); - } - - offsetScn = getLogFileStartPositionByScn(Long.parseLong(logMinerConfig.getStartScn())); - } else { - throw new RuntimeException("不支持的读取模式:" + logMinerConfig.getReadPosition()); - } - } - - private Long getMinScn(){ - Long minScn = null; - PreparedStatement minScnStmt = null; - ResultSet minScnResultSet = null; - - try { - minScnStmt = connection.prepareCall(LogMinerUtil.SQL_GET_LOG_FILE_START_POSITION); - LogMinerUtil.configStatement(minScnStmt, logMinerConfig); - - minScnResultSet = minScnStmt.executeQuery(); - while(minScnResultSet.next()){ - minScn = minScnResultSet.getLong(LogMinerUtil.KEY_FIRST_CHANGE); - } - - return minScn; - } catch (SQLException e) { - LOG.error("获取最早归档日志起始位置出错", e); - throw new RuntimeException(e); - } finally { - closeResources(minScnResultSet, minScnStmt, null); - } - } - - private Long getCurrentScn() { - Long currentScn = null; - CallableStatement currentScnStmt = null; - ResultSet currentScnResultSet = null; - - try { - currentScnStmt = connection.prepareCall(LogMinerUtil.SQL_GET_CURRENT_SCN); - LogMinerUtil.configStatement(currentScnStmt, logMinerConfig); - - currentScnResultSet = currentScnStmt.executeQuery(); - while(currentScnResultSet.next()){ - currentScn = currentScnResultSet.getLong(LogMinerUtil.KEY_CURRENT_SCN); - } - - return currentScn; - } catch (SQLException e) { - LOG.error("获取当前的SCN出错:", e); - throw new RuntimeException(e); - } finally { - closeResources(currentScnResultSet, currentScnStmt, null); - } - } - - /** - * oracle会把把重做日志分文件存储,每个文件都有 "FIRST_CHANGE" 和 "NEXT_CHANGE" 标识范围, - * 这里需要根据给定scn找到对应的日志文件,并获取这个文件的 "FIRST_CHANGE",然后从位置 "FIRST_CHANGE" 开始读取, - * 在[FIRST_CHANGE,scn] 范围内的数据需要跳过。 - * - * 视图说明: - * v$archived_log 视图存储已经归档的日志文件 - * v$log 视图存储未归档的日志文件 - */ - private Long getLogFileStartPositionByScn(Long scn) { - Long logFileFirstChange = null; - PreparedStatement lastLogFileStmt = null; - ResultSet lastLogFileResultSet = null; - - try { - lastLogFileStmt = connection.prepareCall(LogMinerUtil.SQL_GET_LOG_FILE_START_POSITION_BY_SCN); - LogMinerUtil.configStatement(lastLogFileStmt, logMinerConfig); - - lastLogFileStmt.setLong(1, scn); - lastLogFileStmt.setLong(2, scn); - lastLogFileResultSet = lastLogFileStmt.executeQuery(); - while(lastLogFileResultSet.next()){ - logFileFirstChange = lastLogFileResultSet.getLong(LogMinerUtil.KEY_FIRST_CHANGE); - } - - return logFileFirstChange; - } catch (SQLException e) { - LOG.error("根据scn:[{}]获取指定归档日志起始位置出错", scn, e); - throw new RuntimeException(e); - } finally { - closeResources(lastLogFileResultSet, lastLogFileStmt, null); - } - } - - private Long getLogFileStartPositionByTime(Long time) { - Long logFileFirstChange = null; - - PreparedStatement lastLogFileStmt = null; - ResultSet lastLogFileResultSet = null; - - try { - String timeStr = DateFormatUtils.format(time, "yyyy-MM-dd HH:mm:ss"); - - lastLogFileStmt = connection.prepareCall(LogMinerUtil.SQL_GET_LOG_FILE_START_POSITION_BY_TIME); - LogMinerUtil.configStatement(lastLogFileStmt, logMinerConfig); - - lastLogFileStmt.setString(1, timeStr); - lastLogFileStmt.setString(2, timeStr); - lastLogFileStmt.setString(3, timeStr); - lastLogFileResultSet = lastLogFileStmt.executeQuery(); - while(lastLogFileResultSet.next()){ - logFileFirstChange = lastLogFileResultSet.getLong(LogMinerUtil.KEY_FIRST_CHANGE); - } - - return logFileFirstChange; - } catch (SQLException e) { - LOG.error("根据时间:[{}]获取指定归档日志起始位置出错", time, e); - throw new RuntimeException(e); - } finally { - closeResources(lastLogFileResultSet, lastLogFileStmt, null); - } - } - - private void startLogMiner(){ - try { - logMinerStartStmt = connection.prepareCall(LogMinerUtil.SQL_START_LOGMINER); - LogMinerUtil.configStatement(logMinerStartStmt, logMinerConfig); - - logMinerStartStmt.setLong(1, offsetScn); - logMinerStartStmt.execute(); - - LOG.info("启动Log miner成功,offset:{}, sql:{}", offsetScn, LogMinerUtil.SQL_START_LOGMINER); - } catch (SQLException e){ - LOG.error("启动Log miner失败,offset:{}, sql:{}", offsetScn, LogMinerUtil.SQL_START_LOGMINER); - throw new RuntimeException(e); - } - } - - private void startSelectData() { - String logMinerSelectSql = LogMinerUtil.buildSelectSql(logMinerConfig.getCat(), logMinerConfig.getListenerTables()); - try { - logMinerSelectStmt = connection.prepareStatement(logMinerSelectSql); - LogMinerUtil.configStatement(logMinerSelectStmt, logMinerConfig); - - logMinerSelectStmt.setFetchSize(logMinerConfig.getFetchSize()); - logMinerSelectStmt.setLong(1, offsetScn); - logMinerData = logMinerSelectStmt.executeQuery(); - - LOG.info("查询Log miner数据,sql:{}, offset:{}", logMinerSelectSql, offsetScn); - } catch (SQLException e) { - LOG.error("查询Log miner数据出错,sql:{}", logMinerSelectSql); - throw new RuntimeException(e); - } - } - - @Override - protected Row nextRecordInternal(Row row) throws IOException { - String sqlLog = null; - try { - while (logMinerData.next()) { - Long scn = logMinerData.getLong(LogMinerUtil.KEY_SCN); - - // 用CSF来判断一条sql是在当前这一行结束,sql超过4000 字节,会处理成多行 - boolean isSqlNotEnd = logMinerData.getBoolean(LogMinerUtil.KEY_CSF); - - if (skipRecord){ - if (scn > scnCopy && !isSqlNotEnd){ - skipRecord = false; - continue; - } - - LOG.debug("Skipping data with scn :{}", scn); - continue; - } - - StringBuilder sqlRedo = new StringBuilder(logMinerData.getString(LogMinerUtil.KEY_SQL_REDO)); - if(LogMinerUtil.isCreateTemporaryTableSql(sqlRedo.toString())){ - continue; - } - - while(isSqlNotEnd){ - logMinerData.next(); - sqlRedo.append(logMinerData.getString(LogMinerUtil.KEY_SQL_REDO)); - isSqlNotEnd = logMinerData.getBoolean(LogMinerUtil.KEY_CSF); - } - - sqlLog = sqlRedo.toString(); - row = LogMinerUtil.parseSql(logMinerData, sqlRedo.toString(), logMinerConfig.getPavingData()); - - offsetScn = scn; - return row; - } - } catch (Exception e) { - LOG.error("解析数据出错,sql:{}, error:{}", sqlLog, e); - throw new RuntimeException(e); - } - - throw new RuntimeException("获取不到下一条数据,程序自动失败"); - } - - @Override - public FormatState getFormatState() { - super.getFormatState(); - - if (formatState != null && offsetScn != null) { - formatState.setState(offsetScn.toString()); - } - return formatState; - } - - @Override - public boolean reachedEnd() throws IOException { - return false; - } - - @Override - protected void closeInternal() throws IOException { - closeResources(logMinerData, logMinerSelectStmt, connection); - closeResources(null, logMinerStartStmt, null); - } - - private void closeResources(ResultSet rs, Statement stmt, Connection conn) { - if (null != rs) { - try { - rs.close(); - } catch (SQLException e) { - LOG.warn("Close resultSet error: {}", ExceptionUtil.getErrorMessage(e)); - throw new RuntimeException(e); - } - } - - if (null != stmt) { - try { - stmt.close(); - } catch (SQLException e) { - LOG.warn("Close statement error:{}", ExceptionUtil.getErrorMessage(e)); - throw new RuntimeException(e); - } - } - - if (null != conn) { - try { - conn.close(); - } catch (SQLException e) { - LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); - throw new RuntimeException(e); - } - } - } - - enum ReadPosition{ - - /** - * 全量读取 - */ - ALL, - - /** - * 从任务运行时读取 - */ - CURRENT, - - /** - * 从给定的时间读取 - */ - TIME, - - /** - * 从指定的SCN开始读取 - */ - SCN - } -} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java deleted file mode 100644 index 2209ef1f01..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.oraclelogminer.reader; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; -import com.dtstack.flinkx.oraclelogminer.format.LogMinerConfig; -import com.dtstack.flinkx.oraclelogminer.format.OracleLogMinerInputFormat; - -/** - * @author jiangbo - * @date 2019/12/16 - */ -public class OracleLogMinerInputFormatBuilder extends BaseRichInputFormatBuilder { - - private OracleLogMinerInputFormat format; - - public OracleLogMinerInputFormatBuilder() { - super.format = format = new OracleLogMinerInputFormat(); - } - - public void setLogMinerConfig(LogMinerConfig logMinerConfig){ - format.logMinerConfig = logMinerConfig; - } - - @Override - protected void checkFormat() { - - } -} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java deleted file mode 100644 index be40af13cc..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.oraclelogminer.reader; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.ReaderConfig; -import com.dtstack.flinkx.oraclelogminer.format.LogMinerConfig; -import com.dtstack.flinkx.reader.BaseDataReader; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.types.Row; - -/** - * @author jiangbo - * @date 2019/12/14 - */ -public class OraclelogminerReader extends BaseDataReader { - - private LogMinerConfig logMinerConfig; - - public OraclelogminerReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - - ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - - try { - logMinerConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), LogMinerConfig.class); - } catch (Exception e) { - throw new RuntimeException("解析mongodb配置出错:", e); - } - - buildTableListenerRegex(); - } - - private void buildTableListenerRegex(){ - if (CollectionUtils.isEmpty(logMinerConfig.getTable())) { - return; - } - - String tableListener = StringUtils.join(logMinerConfig.getTable(), ","); - logMinerConfig.setListenerTables(tableListener); - } - - @Override - public DataStream readData() { - OracleLogMinerInputFormatBuilder builder = new OracleLogMinerInputFormatBuilder(); - builder.setLogMinerConfig(logMinerConfig); - builder.setRestoreConfig(restoreConfig); - - builder.setMonitorUrls(monitorUrls); - builder.setBytes(bytes); - - return createInput(builder.finish()); - } -} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java deleted file mode 100644 index dc930ab921..0000000000 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.oraclelogminer.util; - -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.enums.ColumnType; -import com.dtstack.flinkx.oraclelogminer.format.LogMinerConfig; -import com.dtstack.flinkx.util.SnowflakeIdWorker; -import net.sf.jsqlparser.JSQLParserException; -import net.sf.jsqlparser.expression.Expression; -import net.sf.jsqlparser.expression.ExpressionVisitorAdapter; -import net.sf.jsqlparser.expression.operators.relational.EqualsTo; -import net.sf.jsqlparser.expression.operators.relational.ExpressionList; -import net.sf.jsqlparser.parser.CCJSqlParserUtil; -import net.sf.jsqlparser.schema.Column; -import net.sf.jsqlparser.statement.Statement; -import net.sf.jsqlparser.statement.delete.Delete; -import net.sf.jsqlparser.statement.insert.Insert; -import net.sf.jsqlparser.statement.update.Update; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.types.Row; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Timestamp; -import java.util.*; - -/** - * @author jiangbo - * @date 2019/12/14 - */ -public class LogMinerUtil { - - public final static String KEY_SEG_OWNER = "SEG_OWNER"; - public final static String KEY_TABLE_NAME = "TABLE_NAME"; - public final static String KEY_OPERATION = "OPERATION"; - public final static String KEY_TIMESTAMP = "TIMESTAMP"; - public final static String KEY_SQL_REDO = "SQL_REDO"; - public final static String KEY_CSF = "CSF"; - public final static String KEY_SCN = "SCN"; - public final static String KEY_CURRENT_SCN = "CURRENT_SCN"; - public final static String KEY_FIRST_CHANGE = "FIRST_CHANGE#"; - - /** - * OPTIONS参数说明: - * DBMS_LOGMNR.SKIP_CORRUPTION - 跳过出错的redlog - * DBMS_LOGMNR.NO_SQL_DELIMITER - 不使用 ';'分割redo sql - * DBMS_LOGMNR.NO_ROWID_IN_STMT - 默认情况下,用于UPDATE和DELETE操作的SQL_REDO和SQL_UNDO语句在where子句中包含“ ROWID =”。 - * 但是,这对于想要重新执行SQL语句的应用程序是不方便的。设置此选项后,“ ROWID”不会放置在重构语句的末尾 - * DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG - 使用在线字典 - * DBMS_LOGMNR.CONTINUOUS_MINE - 需要在生成重做日志的同一实例中使用日志 - * DBMS_LOGMNR.COMMITTED_DATA_ONLY - 指定此选项时,LogMiner将属于同一事务的所有DML操作分组在一起。事务按提交顺序返回。 - * DBMS_LOGMNR.STRING_LITERALS_IN_STMT - 默认情况下,格式化格式化的SQL语句时,SQL_REDO和SQL_UNDO语句会使用数据库会话的NLS设置 - * 例如NLS_DATE_FORMAT,NLS_NUMERIC_CHARACTERS等)。使用此选项,将使用ANSI / ISO字符串文字格式对重构的SQL语句进行格式化。 - */ - public final static String SQL_START_LOGMINER = "" + - "BEGIN DBMS_LOGMNR.START_LOGMNR(" + - "STARTSCN => ?," + - "OPTIONS => DBMS_LOGMNR.SKIP_CORRUPTION " + - "+ DBMS_LOGMNR.NO_SQL_DELIMITER " + - "+ DBMS_LOGMNR.NO_ROWID_IN_STMT " + - "+ DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG " + - "+ DBMS_LOGMNR.CONTINUOUS_MINE " + - "+ DBMS_LOGMNR.COMMITTED_DATA_ONLY " + - "+ DBMS_LOGMNR.STRING_LITERALS_IN_STMT" + - ");" + - "END;"; - - public final static String SQL_SELECT_DATA = "SELECT SCN, COMMIT_SCN, TIMESTAMP, OPERATION, SEG_OWNER, TABLE_NAME, SQL_REDO, ROW_ID, CSF " + - " FROM v$logmnr_contents " + - " WHERE SCN >=?"; - - public final static String SQL_GET_CURRENT_SCN = "select min(CURRENT_SCN) CURRENT_SCN from gv$database"; - - public final static String SQL_GET_LOG_FILE_START_POSITION = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log union select FIRST_CHANGE# from v$archived_log where standby_dest='NO')"; - - public final static String SQL_GET_LOG_FILE_START_POSITION_BY_SCN = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where ? between FIRST_CHANGE# and NEXT_CHANGE# union select FIRST_CHANGE# from v$archived_log where ? between FIRST_CHANGE# and NEXT_CHANGE# and standby_dest='NO')"; - - public final static String SQL_GET_LOG_FILE_START_POSITION_BY_TIME = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') between FIRST_TIME and NVL(NEXT_TIME, TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS')) union select FIRST_CHANGE# from v$archived_log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') between FIRST_TIME and NEXT_TIME and standby_dest='NO')"; - - private final static List SUPPORTED_OPERATIONS = Arrays.asList("UPDATE", "INSERT", "DELETE"); - - public static List EXCLUDE_SCHEMAS = Arrays.asList("SYS"); - - public static SnowflakeIdWorker idWorker = new SnowflakeIdWorker(1, 1); - - public static String buildSelectSql(String listenerOptions, String listenerTables){ - StringBuilder sqlBuilder = new StringBuilder(SQL_SELECT_DATA); - - if (StringUtils.isNotEmpty(listenerOptions)) { - sqlBuilder.append(" and ").append(buildOperationFilter(listenerOptions)); - } - - if (StringUtils.isNotEmpty(listenerTables)) { - sqlBuilder.append(" and ").append(buildSchemaTableFilter(listenerTables)); - } else { - sqlBuilder.append(" and ").append(buildExcludeSchemaFilter()); - } - - return sqlBuilder.toString(); - } - - private static String buildOperationFilter(String listenerOptions){ - List standardOperations = new ArrayList<>(); - - String[] operations = listenerOptions.split(","); - for (String operation : operations) { - if (!SUPPORTED_OPERATIONS.contains(operation.toUpperCase())) { - throw new RuntimeException("不支持的操作类型:" + operation); - } - - standardOperations.add(String.format("'%s'", operation.toUpperCase())); - } - - return String.format("OPERATION in (%s) ", StringUtils.join(standardOperations, ",")); - } - - private static String buildExcludeSchemaFilter(){ - List filters = new ArrayList<>(); - for (String excludeSchema : EXCLUDE_SCHEMAS) { - filters.add(String.format("SEG_OWNER != '%s'", excludeSchema)); - } - - return String.format("(%s)", StringUtils.join(filters, " and ")); - } - - private static String buildSchemaTableFilter(String listenerTables){ - List filters = new ArrayList<>(); - - String[] tableWithSchemas = listenerTables.split(","); - for (String tableWithSchema : tableWithSchemas){ - List tables = Arrays.asList(tableWithSchema.split("\\.")); - if ("*".equals(tables.get(0))) { - throw new IllegalArgumentException("必须指定要采集的schema:" + tableWithSchema); - } - - StringBuilder tableFilterBuilder = new StringBuilder(); - tableFilterBuilder.append(String.format("SEG_OWNER='%s'", tables.get(0))); - - if(!"*".equals(tables.get(1))){ - tableFilterBuilder.append(" and ").append(String.format("TABLE_NAME='%s'", tables.get(1))); - } - - filters.add(String.format("(%s)", tableFilterBuilder.toString())); - } - - return String.format("(%s)", StringUtils.join(filters, " or ")); - } - - public static Row parseSql(ResultSet logMinerData, String sqlRedo, boolean pavingData) throws JSQLParserException, SQLException { - String schema = logMinerData.getString(KEY_SEG_OWNER); - String tableName = logMinerData.getString(KEY_TABLE_NAME); - String operation = logMinerData.getString(KEY_OPERATION); - Timestamp timestamp = logMinerData.getTimestamp(KEY_TIMESTAMP); - - final Map message = new LinkedHashMap<>(); - message.put("type", operation); - message.put("schema", schema); - message.put("table", tableName); - message.put("ts", idWorker.nextId()); - - String sqlRedo2=sqlRedo.replace("IS NULL", "= NULL"); - Statement stmt = CCJSqlParserUtil.parse(sqlRedo2); - LinkedHashMap afterDataMap = new LinkedHashMap<>(); - LinkedHashMap beforeDataMap = new LinkedHashMap<>(); - - if (stmt instanceof Insert){ - parseInsertStmt((Insert) stmt, beforeDataMap, afterDataMap); - }else if (stmt instanceof Update){ - parseUpdateStmt((Update) stmt, beforeDataMap, afterDataMap); - }else if (stmt instanceof Delete){ - parseDeleteStmt((Delete) stmt, beforeDataMap, afterDataMap); - } - - if (pavingData) { - afterDataMap.forEach((key, val) -> { - message.put("after_" + key, val); - }); - - beforeDataMap.forEach((key, val) -> { - message.put("before_" + key, val); - }); - - return Row.of(message); - } else { - message.put("before", beforeDataMap); - message.put("after", afterDataMap); - Map event = new HashMap<>(1); - event.put("message", message); - - return Row.of(event); - } - } - - private static void parseInsertStmt(Insert insert, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap){ - for (Column column : insert.getColumns()){ - afterDataMap.put(cleanString(column.getColumnName()), null); - } - - ExpressionList eList = (ExpressionList) insert.getItemsList(); - List valueList = eList.getExpressions(); - int i =0; - for (String key : afterDataMap.keySet()){ - String value = cleanString(valueList.get(i).toString()); - afterDataMap.put(key, value); - beforeDataMap.put(key, null); - i++; - } - } - - private static void parseUpdateStmt(Update update, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap){ - for (Column c : update.getColumns()){ - afterDataMap.put(cleanString(c.getColumnName()), null); - } - - Iterator iterator = update.getExpressions().iterator(); - - for (String key : afterDataMap.keySet()){ - Object o = iterator.next(); - String value = cleanString(o.toString()); - afterDataMap.put(key, value); - } - - update.getWhere().accept(new ExpressionVisitorAdapter() { - @Override - public void visit(final EqualsTo expr){ - String col = cleanString(expr.getLeftExpression().toString()); - if(afterDataMap.containsKey(col)){ - String value = cleanString(expr.getRightExpression().toString()); - beforeDataMap.put(col, value); - } - } - }); - } - - private static void parseDeleteStmt(Delete delete, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap){ - delete.getWhere().accept(new ExpressionVisitorAdapter(){ - @Override - public void visit(final EqualsTo expr){ - String col = cleanString(expr.getLeftExpression().toString()); - String value = cleanString(expr.getRightExpression().toString()); - beforeDataMap.put(col, value); - afterDataMap.put(col, null); - } - }); - } - - private static String cleanString(String str) { - if (str.startsWith(ColumnType.TIMESTAMP.name().toUpperCase())) { - str = str.replace("TIMESTAMP ", ""); - } - - if (str.startsWith(ConstantValue.SINGLE_QUOTE_MARK_SYMBOL) && str.endsWith(ConstantValue.SINGLE_QUOTE_MARK_SYMBOL)) { - str = str.substring(1, str.length() - 1); - } - - if (str.startsWith(ConstantValue.DOUBLE_QUOTE_MARK_SYMBOL) && str.endsWith(ConstantValue.DOUBLE_QUOTE_MARK_SYMBOL)) { - str = str.substring(1, str.length() - 1); - } - - return str.replace("IS NULL","= NULL").trim(); - } - - public static boolean isCreateTemporaryTableSql(String sql) { - return sql.contains("temporary tables"); - } - - public static void configStatement(java.sql.Statement statement, LogMinerConfig config) throws SQLException{ - if (config.getQueryTimeout() != null) { - statement.setQueryTimeout(config.getQueryTimeout().intValue()); - } - } -} \ No newline at end of file diff --git a/flinkx-oraclelogminer/pom.xml b/flinkx-oraclelogminer/pom.xml deleted file mode 100644 index b737df24ab..0000000000 --- a/flinkx-oraclelogminer/pom.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-oraclelogminer - pom - - flinkx-oraclelogminer-reader - - - - - com.dtstack.flinkx - flinkx-core - 1.6 - provided - - - \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml deleted file mode 100644 index 8c19fad040..0000000000 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - flinkx-saphana - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-saphana-writer - - - - com.dtstack.flinkx - flinkx-saphana-core - 1.6 - - - com.dtstack.flinkx - flinkx-rdb-writer - 1.6 - provided - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.saphanawriter.io.netty - - - com.google - shade.saphanawriter.com.google - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/pom.xml b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/pom.xml deleted file mode 100644 index eb0a6aa124..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/pom.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - flinkx-sqlservercdc - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-sqlservercdc-core - - - - com.microsoft.sqlserver - mssql-jdbc - 7.2.2.jre8 - - - - \ No newline at end of file diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTable.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTable.java deleted file mode 100644 index ed29f2607e..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTable.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package com.dtstack.flinkx.sqlservercdc; - -import java.util.List; -import java.util.Objects; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - *

- * this class is copied from (https://github.com/debezium/debezium). - * but there are some different from the origin. - * - * @author tudou - */ -public class ChangeTable { - - private static final String CDC_SCHEMA = "cdc"; - private final String captureInstance; - private final TableId sourceTableId; - private final TableId changeTableId; - private final Lsn startLsn; - private Lsn stopLsn; - private List columnList; - private final int changeTableObjectId; - - public ChangeTable(TableId sourceTableId, String captureInstance, int changeTableObjectId, Lsn startLsn, Lsn stopLsn, List columnList) { - super(); - this.sourceTableId = sourceTableId; - this.captureInstance = captureInstance; - this.changeTableObjectId = changeTableObjectId; - this.startLsn = startLsn; - this.stopLsn = stopLsn; - this.columnList = columnList; - this.changeTableId = sourceTableId != null ? new TableId(sourceTableId.getCatalogName(), CDC_SCHEMA, captureInstance + "_CT") : null; - } - - public ChangeTable(String captureInstance, int changeTableObjectId, Lsn startLsn, Lsn stopLsn, List columnList) { - this(null, captureInstance, changeTableObjectId, startLsn, stopLsn, columnList); - } - - public String getCaptureInstance() { - return captureInstance; - } - - public Lsn getStartLsn() { - return startLsn; - } - - public Lsn getStopLsn() { - return stopLsn; - } - - public void setStopLsn(Lsn stopLsn) { - this.stopLsn = stopLsn; - } - - public TableId getSourceTableId() { - return sourceTableId; - } - - public List getColumnList() { - return columnList; - } - - public void setColumnList(List columnList) { - this.columnList = columnList; - } - - public TableId getChangeTableId() { - return changeTableId; - } - - public int getChangeTableObjectId() { - return changeTableObjectId; - } - - @Override - public String toString() { - return "ChangeTable{" + - "captureInstance='" + captureInstance + '\'' + - ", sourceTableId=" + sourceTableId + - ", changeTableId=" + changeTableId + - ", startLsn=" + startLsn + - ", stopLsn=" + stopLsn + - ", columnList=" + columnList + - ", changeTableObjectId=" + changeTableObjectId + - '}'; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ChangeTable that = (ChangeTable) o; - return changeTableObjectId == that.changeTableObjectId && - Objects.equals(captureInstance, that.captureInstance) && - Objects.equals(sourceTableId, that.sourceTableId) && - Objects.equals(changeTableId, that.changeTableId) && - Objects.equals(startLsn, that.startLsn) && - Objects.equals(stopLsn, that.stopLsn) && - Objects.equals(columnList, that.columnList); - } - - @Override - public int hashCode() { - return Objects.hash(captureInstance, sourceTableId, changeTableId, startLsn, stopLsn, columnList, changeTableObjectId); - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTablePointer.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTablePointer.java deleted file mode 100644 index 4b180188a6..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/ChangeTablePointer.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package com.dtstack.flinkx.sqlservercdc; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.ResultSet; -import java.sql.SQLException; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - *

- * this class is copied from (https://github.com/debezium/debezium). - * - * @author tudou - */ -public class ChangeTablePointer { - private static final Logger LOG = LoggerFactory.getLogger(ChangeTablePointer.class); - - private static final int COL_COMMIT_LSN = 1; - private static final int COL_ROW_LSN = 2; - private static final int COL_OPERATION = 3; - private static final int COL_DATA = 5; - - private final ChangeTable changeTable; - private final ResultSet resultSet; - private boolean completed = false; - private TxLogPosition currentChangePosition; - - public ChangeTablePointer(ChangeTable changeTable, ResultSet resultSet) { - this.changeTable = changeTable; - this.resultSet = resultSet; - } - - public ChangeTable getChangeTable() { - return changeTable; - } - - public TxLogPosition getChangePosition() throws SQLException { - return currentChangePosition; - } - - public int getOperation() throws SQLException { - return resultSet.getInt(COL_OPERATION); - } - - public Object[] getData() throws SQLException { - final int dataColumnCount = resultSet.getMetaData().getColumnCount() - (COL_DATA - 1); - final Object[] data = new Object[dataColumnCount]; - for (int i = 0; i < dataColumnCount; i++) { - data[i] = resultSet.getObject(COL_DATA + i); - } - return data; - } - - public boolean next() throws SQLException { - completed = !resultSet.next(); - currentChangePosition = completed ? TxLogPosition.NULL : TxLogPosition.valueOf(Lsn.valueOf(resultSet.getBytes(COL_COMMIT_LSN)), Lsn.valueOf(resultSet.getBytes(COL_ROW_LSN))); - if (completed) { - LOG.debug("Closing result set of change tables for table {}", changeTable); - resultSet.close(); - } - return !completed; - } - - public boolean isCompleted() { - return completed; - } - - public int compareTo(ChangeTablePointer o) throws SQLException { - return getChangePosition().compareTo(o.getChangePosition()); - } - - @Override - public String toString() { - return "ChangeTablePointer [changeTable=" + changeTable + ", resultSet=" + resultSet + ", completed=" - + completed + ", currentChangePosition=" + currentChangePosition + "]"; - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/Lsn.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/Lsn.java deleted file mode 100644 index a390fb8ddf..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/Lsn.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package com.dtstack.flinkx.sqlservercdc; - -import com.dtstack.flinkx.util.StringUtil; - -import java.util.Arrays; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - *

- * this class is copied from (https://github.com/debezium/debezium). - * but there are some different from the origin. - * - * @author tudou - */ -public class Lsn implements Comparable { - - private static final String NULL_STRING = "NULL"; - - public static final Lsn NULL = new Lsn(null); - - private final byte[] binary; - private int[] unsignedBinary; - - private String string; - - private Lsn(byte[] binary) { - this.binary = binary; - } - - public byte[] getBinary() { - return binary; - } - - public boolean isAvailable() { - return binary != null; - } - - private int[] getUnsignedBinary() { - if (unsignedBinary != null || binary == null) { - return unsignedBinary; - } - - unsignedBinary = new int[binary.length]; - for (int i = 0; i < binary.length; i++) { - unsignedBinary[i] = Byte.toUnsignedInt(binary[i]); - } - return unsignedBinary; - } - - @Override - public String toString() { - if (string != null) { - return string; - } - final StringBuilder sb = new StringBuilder(); - if (binary == null) { - return NULL_STRING; - } - final int[] unsigned = getUnsignedBinary(); - for (int i = 0; i < unsigned.length; i++) { - final String byteStr = Integer.toHexString(unsigned[i]); - if (byteStr.length() == 1) { - sb.append('0'); - } - sb.append(byteStr); - if (i == 3 || i == 7) { - sb.append(':'); - } - } - string = sb.toString(); - return string; - } - - public static Lsn valueOf(String lsnString) { - return (lsnString == null || NULL_STRING.equals(lsnString)) ? NULL : new Lsn(StringUtil.hexStringToByteArray(lsnString.replace(":", ""))); - } - - public static Lsn valueOf(byte[] lsnBinary) { - return (lsnBinary == null) ? NULL : new Lsn(lsnBinary); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + Arrays.hashCode(binary); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - Lsn other = (Lsn) obj; - if (!Arrays.equals(binary, other.binary)) { - return false; - } - return true; - } - - @Override - public int compareTo(Lsn o) { - if (this == o) { - return 0; - } - if (!this.isAvailable()) { - if (!o.isAvailable()) { - return 0; - } - return -1; - } - if (!o.isAvailable()) { - return 1; - } - final int[] thisU = getUnsignedBinary(); - final int[] thatU = o.getUnsignedBinary(); - for (int i = 0; i < thisU.length; i++) { - final int diff = thisU[i] - thatU[i]; - if (diff != 0) { - return diff; - } - } - return 0; - } - - public boolean isBetween(Lsn from, Lsn to) { - return this.compareTo(from) >= 0 && this.compareTo(to) < 0; - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcConfigKeys.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcConfigKeys.java deleted file mode 100644 index 7945d21a7d..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcConfigKeys.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - * - * @author tudou - */ -public class SqlServerCdcConfigKeys { - - public static final String KEY_USER_NAME = "username"; - - public static final String KEY_PASSWORD = "password"; - - public static final String KEY_URL = "url"; - - public final static String KEY_DATABASE_NAME = "databaseName"; - - public final static String KEY_CATALOG = "cat"; - - public final static String KEY_PAVING_DATA = "pavingData"; - - public final static String KEY_TABLE_LIST = "tableList"; - - public final static String KEY_POLL_INTERVAL = "pollInterval"; - - public final static String KEY_LSN = "lsn"; -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java deleted file mode 100644 index 68f3624c63..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java +++ /dev/null @@ -1,310 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc; - -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.TelnetUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedReader; -import java.sql.*; -import java.util.*; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - * - * @author tudou - */ -public class SqlServerCdcUtil { - private static final Logger LOG = LoggerFactory.getLogger(SqlServerCdcUtil.class); - - public static final String DRIVER = "com.microsoft.sqlserver.jdbc.SQLServerDriver"; - public static Pattern p = Pattern.compile("\\[(.*?)]"); - - private static final String STATEMENTS_PLACEHOLDER = "#"; - private static final String CHECK_CDC_DATABASE = "select 1 from sys.databases where name='%s' AND is_cdc_enabled=1"; - private static final String CHECK_CDC_TABLE = "select sys.schemas.name+'.'+sys.tables.name from sys.tables, sys.schemas where sys.tables.is_tracked_by_cdc = 1 and sys.tables.schema_id = sys.schemas.schema_id;"; - private static final String GET_LIST_OF_CDC_ENABLED_TABLES = "EXEC sys.sp_cdc_help_change_data_capture"; - private static final String GET_MAX_LSN = "SELECT sys.fn_cdc_get_max_lsn()"; - private static final String INCREMENT_LSN = "SELECT sys.fn_cdc_increment_lsn(?)"; - private static final String GET_ALL_CHANGES_FOR_TABLE = "SELECT * FROM cdc.[fn_cdc_get_all_changes_#](ISNULL(?,sys.fn_cdc_get_min_lsn('#')), ?, N'all update old')"; - - public static void changeDatabase(Connection conn, String databaseName) throws SQLException { - conn.createStatement().execute(" use " + databaseName); - } - - public static boolean checkEnabledCdcDatabase(Connection conn, String databaseName) throws SQLException { - Statement statement = null; - ResultSet rs = null; - boolean ret; - try { - statement = conn.createStatement(); - rs = statement.executeQuery(String.format(CHECK_CDC_DATABASE, databaseName)); - ret = rs.next(); - } catch (SQLException e) { - LOG.error("error to query {} Enabled CDC or not, sql = {}, e = {}", databaseName, String.format(CHECK_CDC_DATABASE, databaseName), ExceptionUtil.getErrorMessage(e)); - throw e; - } finally { - closeDbResources(rs, statement, null, false); - } - return ret; - } - - public static Set checkUnEnabledCdcTables(Connection conn, Collection tableSet) throws SQLException { - Statement statement = null; - ResultSet rs = null; - CopyOnWriteArraySet unEnabledCdcTables = new CopyOnWriteArraySet<>(tableSet); - try { - statement = conn.createStatement(); - rs = statement.executeQuery(CHECK_CDC_TABLE); - while (rs.next()) { - String tableName = rs.getString(1); - unEnabledCdcTables.remove(tableName); - } - } catch (SQLException e) { - LOG.error("error to query UnEnabled CDC Tables, sql = {}, e = {}", CHECK_CDC_TABLE, ExceptionUtil.getErrorMessage(e)); - throw e; - } finally { - closeDbResources(rs, statement, null, false); - } - return unEnabledCdcTables; - } - - public static Set queryChangeTableSet(Connection conn, String databaseName) throws SQLException { - Statement statement = null; - ResultSet rs = null; - Set changeTableSet = new HashSet<>(); - try { - statement = conn.createStatement(); - rs = statement.executeQuery(GET_LIST_OF_CDC_ENABLED_TABLES); - while (rs.next()) { - String column = rs.getString(15); - Matcher m = p.matcher(column); - List columnList = new ArrayList<>(); - while(m.find()){ - columnList.add(m.group(1)); - } - changeTableSet.add( - new ChangeTable( - new TableId(databaseName, rs.getString(1), rs.getString(2)), - rs.getString(3), - rs.getInt(4), - Lsn.valueOf(rs.getBytes(6)), - Lsn.valueOf(rs.getBytes(7)), - columnList - ) - ); - } - } catch (SQLException e) { - LOG.error("error to query change table set, e = {}", ExceptionUtil.getErrorMessage(e)); - throw e; - } finally { - closeDbResources(rs, statement, null, false); - } - return changeTableSet; - } - - public static Lsn getMaxLsn(Connection conn) throws SQLException { - Statement statement = null; - ResultSet rs = null; - Lsn lsn = null; - try { - statement = conn.createStatement(); - rs = statement.executeQuery(GET_MAX_LSN); - rs.next(); - lsn = Lsn.valueOf(rs.getBytes(1)); - } catch (SQLException e) { - LOG.error("error to query change table set, e = {}", ExceptionUtil.getErrorMessage(e)); - throw e; - } finally { - closeDbResources(rs, statement, null, false); - } - return lsn; - } - - public static ChangeTable[] getCdcTablesToQuery(Connection conn, String databaseName, List tableList) throws SQLException { - Set cdcEnabledTableSet = SqlServerCdcUtil.queryChangeTableSet(conn, databaseName); - - if (cdcEnabledTableSet.isEmpty()) { - LOG.error("No table has enabled CDC or security constraints prevents getting the list of change tables"); - } - - Map> whitelistedCdcEnabledTables = cdcEnabledTableSet.stream() - .filter(changeTable -> { - String tableName = changeTable.getSourceTableId().getSchemaName() + "." + changeTable.getSourceTableId().getTableName(); - return tableList.contains(tableName); - }) - .collect(Collectors.groupingBy(ChangeTable::getSourceTableId)); - - List changeTableList = new ArrayList<>(); - for (List captures : whitelistedCdcEnabledTables.values()) { - ChangeTable currentTable = captures.get(0); - if (captures.size() > 1) { - ChangeTable futureTable; - if (captures.get(0).getStartLsn().compareTo(captures.get(1).getStartLsn()) < 0) { - futureTable = captures.get(1); - } else { - currentTable = captures.get(1); - futureTable = captures.get(0); - } - currentTable.setStopLsn(futureTable.getStartLsn()); - changeTableList.add(futureTable); - LOG.info("Multiple capture instances present for the same table: {} and {}", currentTable, futureTable); - } - changeTableList.add(currentTable); - } - - return changeTableList.toArray(new ChangeTable[0]); - } - - public static Lsn incrementLsn(Connection conn, Lsn lsn) throws SQLException { - PreparedStatement ps = null; - ResultSet rs = null; - Lsn ret; - try { - ps = conn.prepareStatement(INCREMENT_LSN); - ps.setBytes(1, lsn.getBinary()); - rs = ps.executeQuery(); - rs.next(); - ret = Lsn.valueOf(rs.getBytes(1)); - } catch (SQLException e) { - LOG.error("error to query increment lsn, e = {}", ExceptionUtil.getErrorMessage(e)); - throw e; - } finally { - closeDbResources(rs, ps, null, false); - } - return ret; - } - - public static ResultSet[] getChangesForTables(Connection conn, ChangeTable[] changeTables, Lsn intervalFromLsn, Lsn intervalToLsn) throws SQLException { - ResultSet[] resultSets = new ResultSet[changeTables.length]; - String sql; - int idx = 0; - try { - for (ChangeTable changeTable : changeTables) { - sql = GET_ALL_CHANGES_FOR_TABLE.replace(STATEMENTS_PLACEHOLDER, changeTable.getCaptureInstance()); - Lsn fromLsn = changeTable.getStartLsn().compareTo(intervalFromLsn) > 0 ? changeTable.getStartLsn() : intervalFromLsn; - - //notice : statement is not closed, there maybe have problem. - PreparedStatement statement = conn.prepareStatement(sql); - statement.setBytes(1, fromLsn.getBinary()); - statement.setBytes(2, intervalToLsn.getBinary()); - ResultSet rs = statement.executeQuery(); - resultSets[idx] = rs; - idx++; - } - } catch (Exception e) { - LOG.error("error to getChangesForTables, e = {}", ExceptionUtil.getErrorMessage(e)); - throw e; - } - return resultSets; - } - - /** - * clob转string - * @param obj clob - * @return - * @throws Exception - */ - public static Object clobToString(Object obj) throws Exception{ - if(obj instanceof Clob){ - Clob clob = (Clob)obj; - BufferedReader bf = new BufferedReader(clob.getCharacterStream()); - StringBuilder stringBuilder = new StringBuilder(); - String line; - while ((line = bf.readLine()) != null){ - stringBuilder.append(line); - } - bf.close(); - return stringBuilder.toString(); - } else { - return obj; - } - } - - /** - * 获取jdbc连接(超时10S) - * @param url url - * @param username 账号 - * @param password 密码 - * @return - * @throws SQLException - */ - public static Connection getConnection(String url, String username, String password) throws SQLException { - Connection dbConn; - synchronized (ClassUtil.LOCK_STR){ - DriverManager.setLoginTimeout(10); - - // telnet - TelnetUtil.telnet(url); - - if (username == null) { - dbConn = DriverManager.getConnection(url); - } else { - dbConn = DriverManager.getConnection(url, username, password); - } - } - - return dbConn; - } - - - /** - * 关闭连接资源 - * @param rs ResultSet - * @param stmt Statement - * @param conn Connection - * @param commit - */ - public static void closeDbResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { - if (null != rs) { - try { - rs.close(); - } catch (SQLException e) { - LOG.warn("Close resultSet error: {}", ExceptionUtil.getErrorMessage(e)); - } - } - - if (null != stmt) { - try { - stmt.close(); - } catch (SQLException e) { - LOG.warn("Close statement error:{}", ExceptionUtil.getErrorMessage(e)); - } - } - - if (null != conn) { - try { - if(commit && !conn.isClosed()){ - conn.commit(); - } - - conn.close(); - } catch (SQLException e) { - LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); - } - } - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlserverCdcEnum.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlserverCdcEnum.java deleted file mode 100644 index b4fa84fb8b..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlserverCdcEnum.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Objects; -import java.util.Set; - -/** - * Date: 2019/12/05 - * Company: www.dtstack.com - * - * @author tudou - */ -public enum SqlserverCdcEnum { - /** - * 操作未知 - */ - UNKNOWN(-1, "unknown"), - /** - * 删除操作 - */ - DELETE(1, "delete"), - /** - * 插入操作 - */ - INSERT(2, "insert"), - /** - * 更新前操作 - */ - UPDATE_BEFORE(3, "update_before"), - /** - * 更新后操作 - */ - UPDATE_AFTER(4, "update_after"); - - public int code; - public String name; - - SqlserverCdcEnum(int code, String name) { - this.code = code; - this.name = name; - } - - public static SqlserverCdcEnum getEnum(String name){ - switch (name.toLowerCase()){ - case "delete": return DELETE; - case "insert": return INSERT; - case "update_before": return UPDATE_BEFORE; - case "update_after": return UPDATE_AFTER; - default: return UNKNOWN; - } - } - - public static SqlserverCdcEnum getEnum(int code){ - switch (code){ - case 1: return DELETE; - case 2: return INSERT; - case 3: return UPDATE_BEFORE; - case 4: return UPDATE_AFTER; - default: return UNKNOWN; - } - } - - public static Set transform(String name){ - if(Objects.equals(name, "update")){ - Set set = new HashSet<>(); - set.add(UPDATE_BEFORE.code); - set.add(UPDATE_AFTER.code); - return set; - }else{ - return Collections.singleton(getEnum(name).code); - } - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TableId.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TableId.java deleted file mode 100644 index dfd0a0f1c7..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TableId.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package com.dtstack.flinkx.sqlservercdc; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - *

- * this class is copied from (https://github.com/debezium/debezium). - * but there are some different from the origin. - * - * @author tudou - */ -public class TableId implements Comparable { - - /** - * Parse the supplied string, extracting up to the first 3 parts into a TableID. - * - * @param parts the parts of the identifier; may not be null - * @param numParts the number of parts to use for the table identifier - * @param useCatalogBeforeSchema {@code true} if the parsed string contains only 2 items and the first should be used as - * the catalog and the second as the table name, or {@code false} if the first should be used as the schema and the - * second as the table name - * @return the table ID, or null if it could not be parsed - */ - protected static TableId parse(String[] parts, int numParts, boolean useCatalogBeforeSchema) { - if (numParts == 0) { - return null; - } - if (numParts == 1) { - // table only - return new TableId(null, null, parts[0]); - } - if (numParts == 2) { - if (useCatalogBeforeSchema) { - // catalog & table only - return new TableId(parts[0], null, parts[1]); - } - // schema & table only - return new TableId(null, parts[0], parts[1]); - } - // catalog, schema & table - return new TableId(parts[0], parts[1], parts[2]); - } - - private final String catalogName; - private final String schemaName; - private final String tableName; - private final String id; - - /** - * Create a new table identifier. - * - * @param catalogName the name of the database catalog that contains the table; may be null if the JDBC driver does not - * show a schema for this table - * @param schemaName the name of the database schema that contains the table; may be null if the JDBC driver does not - * show a schema for this table - * @param tableName the name of the table; may not be null - */ - public TableId(String catalogName, String schemaName, String tableName) { - this.catalogName = catalogName; - this.schemaName = schemaName; - this.tableName = tableName; - assert this.tableName != null; - this.id = tableId(this.catalogName, this.schemaName, this.tableName); - } - - public String getCatalogName() { - return catalogName; - } - - public String getSchemaName() { - return schemaName; - } - - public String getTableName() { - return tableName; - } - - @Override - public int compareTo(TableId that) { - if (this == that) { - return 0; - } - return this.id.compareTo(that.id); - } - - public int compareToIgnoreCase(TableId that) { - if (this == that) { - return 0; - } - return this.id.compareToIgnoreCase(that.id); - } - - @Override - public int hashCode() { - return id.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof TableId) { - return this.compareTo((TableId) obj) == 0; - } - return false; - } - - @Override - public String toString() { - return id; - } - - /** - * Returns a dot-separated String representation of this identifier, quoting all - * name parts with the {@code "} char. - */ - public String toDoubleQuotedString() { - return toQuotedString('"'); - } - - /** - * Returns a dot-separated String representation of this identifier, quoting all - * name parts with the given quoting char. - */ - public String toQuotedString(char quotingChar) { - StringBuilder quoted = new StringBuilder(); - - if (catalogName != null && !catalogName.isEmpty()) { - quoted.append(quote(catalogName, quotingChar)).append("."); - } - - if (schemaName != null && !schemaName.isEmpty()) { - quoted.append(quote(schemaName, quotingChar)).append("."); - } - - quoted.append(quote(tableName, quotingChar)); - - return quoted.toString(); - } - - private static String tableId(String catalog, String schema, String table) { - if (catalog == null || catalog.length() == 0) { - if (schema == null || schema.length() == 0) { - return table; - } - return schema + "." + table; - } - if (schema == null || schema.length() == 0) { - return catalog + "." + table; - } - return catalog + "." + schema + "." + table; - } - - /** - * Quotes the given identifier part, e.g. schema or table name. - */ - private static String quote(String identifierPart, char quotingChar) { - if (identifierPart == null) { - return null; - } - - if (identifierPart.isEmpty()) { - return String.valueOf(quotingChar) + quotingChar; - } - - if (identifierPart.charAt(0) != quotingChar && identifierPart.charAt(identifierPart.length() - 1) != quotingChar) { - identifierPart = identifierPart.replace(quotingChar + "", repeat(quotingChar)); - identifierPart = quotingChar + identifierPart + quotingChar; - } - - return identifierPart; - } - - private static String repeat(char quotingChar) { - return String.valueOf(quotingChar) + quotingChar; - } - - public TableId toLowercase() { - return new TableId(catalogName, schemaName, tableName.toLowerCase()); - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TxLogPosition.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TxLogPosition.java deleted file mode 100644 index 4fcb7d69a2..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/TxLogPosition.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package com.dtstack.flinkx.sqlservercdc; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - *

- * this class is copied from (https://github.com/debezium/debezium). - * but there are some different from the origin. - * - * @author tudou - */ -public class TxLogPosition implements Comparable{ - public static final TxLogPosition NULL = new TxLogPosition(null, null); - private final Lsn commitLsn; - private final Lsn inTxLsn; - - private TxLogPosition(Lsn commitLsn, Lsn inTxLsn) { - this.commitLsn = commitLsn; - this.inTxLsn = inTxLsn; - } - - public Lsn getCommitLsn() { - return commitLsn; - } - - public Lsn getInTxLsn() { - return inTxLsn; - } - - @Override - public String toString() { - return this == NULL ? "NULL" : commitLsn + "(" + inTxLsn + ")"; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((commitLsn == null) ? 0 : commitLsn.hashCode()); - result = prime * result + ((inTxLsn == null) ? 0 : inTxLsn.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj){ - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - TxLogPosition other = (TxLogPosition) obj; - if (commitLsn == null) { - if (other.commitLsn != null) { - return false; - } - } - else if (!commitLsn.equals(other.commitLsn)) { - return false; - } - if (inTxLsn == null) { - if (other.inTxLsn != null) { - return false; - } - } - else if (!inTxLsn.equals(other.inTxLsn)) { - return false; - } - return true; - } - - @Override - public int compareTo(TxLogPosition o) { - final int comparison = commitLsn.compareTo(o.getCommitLsn()); - return comparison == 0 ? inTxLsn.compareTo(o.inTxLsn) : comparison; - } - - public static TxLogPosition valueOf(Lsn commitLsn, Lsn inTxLsn) { - return commitLsn == null && inTxLsn == null ? NULL - : new TxLogPosition( - commitLsn == null ? Lsn.NULL : commitLsn, - inTxLsn == null ? Lsn.NULL : inTxLsn - ); - } - - public static TxLogPosition valueOf(Lsn commitLsn) { - return valueOf(commitLsn, Lsn.NULL); - } - - public boolean isAvailable() { - return inTxLsn != null && commitLsn != null; - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml deleted file mode 100644 index 08aa96dc6f..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml +++ /dev/null @@ -1,69 +0,0 @@ - - - - flinkx-sqlservercdc - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-sqlservercdc-reader - - - - com.dtstack.flinkx - flinkx-sqlservercdc-core - 1.6 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormat.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormat.java deleted file mode 100644 index 63aa6fba93..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormat.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc.format; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.restore.FormatState; -import com.dtstack.flinkx.sqlservercdc.Lsn; -import com.dtstack.flinkx.sqlservercdc.SqlServerCdcUtil; -import com.dtstack.flinkx.sqlservercdc.TxLogPosition; -import com.dtstack.flinkx.sqlservercdc.listener.SqlServerCdcListener; -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.types.Row; - -import java.io.IOException; -import java.sql.Connection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.*; - -import static com.dtstack.flinkx.sqlservercdc.SqlServerCdcUtil.DRIVER; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - * - * @author tudou - */ -public class SqlserverCdcInputFormat extends BaseRichInputFormat { - protected String username; - protected String password; - protected String url; - protected String databaseName; - protected boolean pavingData = false; - protected List tableList; - protected String cat; - protected long pollInterval; - protected String lsn; - - private Connection conn; - private TxLogPosition logPosition; - - private transient BlockingQueue> queue; - private transient ExecutorService executor; - private volatile boolean running = false; - - @Override - protected void openInternal(InputSplit inputSplit) { - ThreadFactory namedThreadFactory = new ThreadFactoryBuilder().setNameFormat("cdcListener-pool-%d").build(); - executor = new ThreadPoolExecutor(1, 1, - 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(1024), namedThreadFactory, new ThreadPoolExecutor.AbortPolicy()); - queue = new SynchronousQueue<>(false); - - if (inputSplit.getSplitNumber() != 0) { - LOG.info("sqlServer cdc openInternal split number:{} abort...", inputSplit.getSplitNumber()); - return; - } - - LOG.info("sqlServer cdc openInternal split number:{} start...", inputSplit.getSplitNumber()); - try { - ClassUtil.forName(DRIVER, getClass().getClassLoader()); - conn = SqlServerCdcUtil.getConnection(url, username, password); - conn.setAutoCommit(false); - SqlServerCdcUtil.changeDatabase(conn, databaseName); - if(!SqlServerCdcUtil.checkEnabledCdcDatabase(conn, databaseName)){ - LOG.error("{} is not enable sqlServer CDC", databaseName); - throw new UnsupportedOperationException(databaseName + " is not enable sqlServer CDC "); - } - - Set unEnabledCdcTables = SqlServerCdcUtil.checkUnEnabledCdcTables(conn, tableList); - if(CollectionUtils.isNotEmpty(unEnabledCdcTables)){ - String tables = unEnabledCdcTables.toString(); - LOG.error("{} is not enable sqlServer CDC", tables); - throw new UnsupportedOperationException(tables + " is not enable sqlServer CDC "); - } - if(StringUtils.isNotBlank(lsn)){ - logPosition = TxLogPosition.valueOf(Lsn.valueOf(lsn)); - }else if(formatState != null && formatState.getState() != null){ - logPosition = (TxLogPosition)formatState.getState(); - }else{ - logPosition = TxLogPosition.valueOf(SqlServerCdcUtil.getMaxLsn(conn)); - } - - executor.submit(new SqlServerCdcListener(this)); - running = true; - } catch (Exception e) { - LOG.error("SqlserverCdcInputFormat open() failed, e = {}", ExceptionUtil.getErrorMessage(e)); - throw new RuntimeException("SqlserverCdcInputFormat open() failed, e = " + ExceptionUtil.getErrorMessage(e)); - } - - LOG.info("SqlserverCdcInputFormat[{}]open: end", jobName); - } - - @Override - protected Row nextRecordInternal(Row row) throws IOException { - try { - Map map = queue.take(); - if(map.size() == 1){ - throw new IOException((String) map.get("e")); - }else{ - row = Row.of(map); - } - } catch (InterruptedException e) { - LOG.error("takeEvent interrupted error:{}", ExceptionUtil.getErrorMessage(e)); - } - return row; - } - - @Override - protected void closeInternal(){ - if (running) { - executor.shutdownNow(); - running = false; - LOG.warn("shutdown SqlServerCdcListener......"); - } - } - - - @Override - public InputSplit[] createInputSplitsInternal(int minNumSplits) { - InputSplit[] splits = new InputSplit[minNumSplits]; - for (int i = 0; i < minNumSplits; i++) { - splits[i] = new GenericInputSplit(i, minNumSplits); - } - return splits; - } - - @Override - public boolean reachedEnd() { - return false; - } - - @Override - public FormatState getFormatState() { - if (!restoreConfig.isRestore()) { - LOG.info("return null for formatState"); - return null; - } - - super.getFormatState(); - if (formatState != null) { - formatState.setState(logPosition); - } - return formatState; - } - - public void processEvent(Map event) { - try { - queue.put(event); - } catch (InterruptedException e) { - LOG.error("takeEvent interrupted event:{} error:{}", event, ExceptionUtil.getErrorMessage(e)); - } - } - - public String getDatabaseName() { - return databaseName; - } - - public boolean isPavingData() { - return pavingData; - } - - public List getTableList() { - return tableList; - } - - public String getCat() { - return cat; - } - - public long getPollInterval() { - return pollInterval; - } - - public Connection getConn() { - return conn; - } - - public void setLogPosition(TxLogPosition logPosition) { - this.logPosition = logPosition; - } - - public TxLogPosition getLogPosition() { - return logPosition; - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormatBuilder.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormatBuilder.java deleted file mode 100644 index a0cd8b6a35..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/format/SqlserverCdcInputFormatBuilder.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc.format; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang.StringUtils; - -import java.util.List; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - * - * @author tudou - */ -public class SqlserverCdcInputFormatBuilder extends BaseRichInputFormatBuilder { - - protected SqlserverCdcInputFormat format; - - public SqlserverCdcInputFormatBuilder(){ - super.format = this.format = new SqlserverCdcInputFormat(); - } - - public void setUsername(String username) { - format.username = username; - } - - public void setPassword(String password) { - format.password = password; - } - - public void setUrl(String url) { - format.url = url; - } - - public void setDatabaseName(String databaseName) { - format.databaseName = databaseName; - } - - public void setPavingData(boolean pavingData) { - format.pavingData = pavingData; - } - - public void setTable(List table) { - format.tableList = table; - } - - public void setCat(String cat) { - format.cat = cat; - } - - public void setPollInterval(long pollInterval) { - format.pollInterval = pollInterval; - } - - public void setLsn(String lsn) { - format.lsn = lsn; - } - - - @Override - protected void checkFormat() { - if (StringUtils.isBlank(format.username)) { - throw new IllegalArgumentException("No username supplied"); - } - if (StringUtils.isBlank(format.password)) { - throw new IllegalArgumentException("No password supplied"); - } - if (StringUtils.isBlank(format.url)) { - throw new IllegalArgumentException("No url supplied"); - } - if (StringUtils.isBlank(format.databaseName)) { - throw new IllegalArgumentException("No databaseName supplied"); - } - if (CollectionUtils.isEmpty(format.tableList)) { - throw new IllegalArgumentException("No tableList supplied"); - } - if (StringUtils.isBlank(format.cat)) { - throw new IllegalArgumentException("No cat supplied"); - } - } -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java deleted file mode 100644 index 83b63c642b..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc.listener; - -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.sqlservercdc.*; -import com.dtstack.flinkx.sqlservercdc.format.SqlserverCdcInputFormat; -import com.dtstack.flinkx.util.Clock; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.Metronome; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.time.Duration; -import java.time.temporal.ChronoUnit; -import java.util.*; - -/** - * Date: 2019/12/04 - * Company: www.dtstack.com - * - * some code in run() are copied from (https://github.com/debezium/debezium). - * - * @author tudou - */ -public class SqlServerCdcListener implements Runnable{ - private static final Logger LOG = LoggerFactory.getLogger(SqlServerCdcListener.class); - - private SqlserverCdcInputFormat format; - private TxLogPosition logPosition; - private ChangeTable[] tablesSlot; - private Connection conn; - private List tableList; - private Set cat; - private boolean pavingData; - private Duration pollInterval; - - public SqlServerCdcListener(SqlserverCdcInputFormat format) throws SQLException { - this.format = format; - this.conn = format.getConn(); - this.logPosition = format.getLogPosition(); - this.tableList = format.getTableList(); - this.cat = new HashSet<>(); - for (String type : format.getCat().split(ConstantValue.COMMA_SYMBOL)) { - cat.addAll(SqlserverCdcEnum.transform(type)); - } - this.pavingData = format.isPavingData(); - this.tablesSlot = SqlServerCdcUtil.getCdcTablesToQuery(conn, format.getDatabaseName(), tableList); - this.pollInterval = Duration.of(format.getPollInterval(), ChronoUnit.MILLIS); - } - - @Override - public void run() { - LOG.info("SqlServerCdcListener start running....."); - try { - Metronome metronome = Metronome.sleeper(pollInterval, Clock.system()); - while (true){ - - Lsn currentMaxLsn = SqlServerCdcUtil.getMaxLsn(conn); - - // Shouldn't happen if the agent is running, but it is better to guard against such situation - if (!currentMaxLsn.isAvailable()) { - LOG.warn("No maximum LSN recorded in the database; please ensure that the SQL Server Agent is running"); - metronome.pause(); - continue; - } - - // There is no change in the database - if (currentMaxLsn.equals(logPosition.getCommitLsn())) { - metronome.pause(); - continue; - } - - // Reading interval is inclusive so we need to move LSN forward but not for first - // run as TX might not be streamed completely - Lsn fromLsn; - if(logPosition.getCommitLsn().isAvailable()){ - fromLsn = SqlServerCdcUtil.incrementLsn(conn, logPosition.getCommitLsn()); - }else { - fromLsn = logPosition.getCommitLsn(); - } - - ResultSet[] resultSets = SqlServerCdcUtil.getChangesForTables(conn, tablesSlot, fromLsn, currentMaxLsn); - int tableCount = resultSets.length; - final ChangeTablePointer[] changeTables = new ChangeTablePointer[tableCount]; - for (int i = 0; i < tableCount; i++) { - changeTables[i] = new ChangeTablePointer(tablesSlot[i], resultSets[i]); - changeTables[i].next(); - } - for (;;) { - ChangeTablePointer tableWithSmallestLsn = null; - for (ChangeTablePointer changeTable: changeTables) { - if (changeTable.isCompleted()) { - continue; - } - if (tableWithSmallestLsn == null || changeTable.compareTo(tableWithSmallestLsn) < 0) { - tableWithSmallestLsn = changeTable; - } - } - if (tableWithSmallestLsn == null) { - break; - } - - if (!(tableWithSmallestLsn.getChangePosition().isAvailable() && tableWithSmallestLsn.getChangePosition().getInTxLsn().isAvailable())) { - LOG.error("Skipping change {} as its LSN is NULL which is not expected", tableWithSmallestLsn); - tableWithSmallestLsn.next(); - continue; - } - // After restart for changes that were executed before the last committed offset - if (tableWithSmallestLsn.getChangePosition().compareTo(logPosition) < 0) { - LOG.info("Skipping change {} as its position is smaller than the last recorded position {}", tableWithSmallestLsn, logPosition); - tableWithSmallestLsn.next(); - continue; - } - ChangeTable changeTable = tableWithSmallestLsn.getChangeTable(); - if (changeTable.getStopLsn().isAvailable() && - changeTable.getStopLsn().compareTo(tableWithSmallestLsn.getChangePosition().getCommitLsn()) <= 0) { - LOG.debug("Skipping table change {} as its stop LSN is smaller than the last recorded LSN {}", tableWithSmallestLsn, tableWithSmallestLsn.getChangePosition()); - tableWithSmallestLsn.next(); - continue; - } - int operation = tableWithSmallestLsn.getOperation(); - if(!cat.contains(operation)){ - tableWithSmallestLsn.next(); - continue; - } - TableId tableId = changeTable.getSourceTableId(); - Object[] data = tableWithSmallestLsn.getData(); - - if (operation == SqlserverCdcEnum.UPDATE_BEFORE.code) { - if (!tableWithSmallestLsn.next() || tableWithSmallestLsn.getOperation() != SqlserverCdcEnum.UPDATE_AFTER.code) { - throw new IllegalStateException("The update before event at " + tableWithSmallestLsn.getChangePosition() + " for table " + tableId + " was not followed by after event"); - } - } - Object[] dataNext; - if(operation == SqlserverCdcEnum.UPDATE_BEFORE.code){ - dataNext = tableWithSmallestLsn.getData(); - }else if(operation == SqlserverCdcEnum.DELETE.code){ - dataNext = data; - data = new Object[dataNext.length]; - }else{ - dataNext = new Object[data.length]; - } - Map map = new LinkedHashMap<>(); - map.put("type", SqlserverCdcEnum.getEnum(operation).name.split("_")[0]); - map.put("schema", tableId.getSchemaName()); - map.put("table", tableId.getTableName()); - map.put("lsn", tableWithSmallestLsn.getChangePosition().getCommitLsn().toString()); - map.put("ingestion", System.nanoTime()); - if(pavingData){ - int i = 0; - for (String column : changeTable.getColumnList()) { - map.put("before_" + column, SqlServerCdcUtil.clobToString(dataNext[i])); - map.put("after_" + column, SqlServerCdcUtil.clobToString(data[i])); - i++; - } - }else{ - Map before = new LinkedHashMap<>(); - Map after = new LinkedHashMap<>(); - int i = 0; - for (String column : changeTable.getColumnList()) { - before.put(column, SqlServerCdcUtil.clobToString(data[i])); - after.put(column, SqlServerCdcUtil.clobToString(dataNext[i])); - i++; - } - map.put("before", before); - map.put("after", after); - } - format.processEvent(map); - format.setLogPosition(tableWithSmallestLsn.getChangePosition()); - tableWithSmallestLsn.next(); - } - - LOG.debug("currentMaxLsn = {}", logPosition); - logPosition = TxLogPosition.valueOf(currentMaxLsn); - conn.rollback(); - } - }catch (Exception e){ - String errorMessage = ExceptionUtil.getErrorMessage(e); - LOG.error(errorMessage); - format.processEvent(Collections.singletonMap("e", errorMessage)); - } - } - - -} diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java deleted file mode 100644 index 6b0ef6d226..0000000000 --- a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.sqlservercdc.reader; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.ReaderConfig; -import com.dtstack.flinkx.reader.BaseDataReader; -import com.dtstack.flinkx.sqlservercdc.SqlServerCdcConfigKeys; -import com.dtstack.flinkx.sqlservercdc.format.SqlserverCdcInputFormatBuilder; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.types.Row; - -import java.util.List; - -/** - * Date: 2019/12/03 - * Company: www.dtstack.com - * - * @author tudou - */ -public class SqlservercdcReader extends BaseDataReader { - private String username; - private String password; - private String url; - private String databaseName; - private String cat; - private boolean pavingData; - private List tableList; - private Long pollInterval; - private String lsn; - - @SuppressWarnings("unchecked") - public SqlservercdcReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - username = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_USER_NAME); - password = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_PASSWORD); - url = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_URL); - databaseName = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_DATABASE_NAME); - cat = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_CATALOG); - pavingData = readerConfig.getParameter().getBooleanVal(SqlServerCdcConfigKeys.KEY_PAVING_DATA, false); - tableList = (List) readerConfig.getParameter().getVal(SqlServerCdcConfigKeys.KEY_TABLE_LIST); - pollInterval = readerConfig.getLongVal(SqlServerCdcConfigKeys.KEY_POLL_INTERVAL, 1000); - lsn = readerConfig.getParameter().getStringVal(SqlServerCdcConfigKeys.KEY_LSN); - } - - @Override - public DataStream readData() { - SqlserverCdcInputFormatBuilder builder = new SqlserverCdcInputFormatBuilder(); - builder.setUsername(username); - builder.setPassword(password); - builder.setUrl(url); - builder.setDatabaseName(databaseName); - builder.setCat(cat); - builder.setPavingData(pavingData); - builder.setTable(tableList); - builder.setRestoreConfig(restoreConfig); - builder.setPollInterval(pollInterval); - builder.setLsn(lsn); - - return createInput(builder.finish(), "sqlserverdcreader"); - } -} diff --git a/flinkx-sqlservercdc/pom.xml b/flinkx-sqlservercdc/pom.xml deleted file mode 100644 index d75358ea7c..0000000000 --- a/flinkx-sqlservercdc/pom.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - pom - - flinkx-sqlservercdc - - - flinkx-sqlservercdc-core - flinkx-sqlservercdc-reader - - - - - com.dtstack.flinkx - flinkx-core - 1.6 - provided - - - - \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml deleted file mode 100644 index 44cc030d1f..0000000000 --- a/flinkx-teradata/flinkx-teradata-reader/pom.xml +++ /dev/null @@ -1,108 +0,0 @@ - - - - flinkx-teradata - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-teradata-reader - - - - com.dtstack.flinkx - flinkx-teradata-core - 1.6 - - - com.dtstack.flinkx - flinkx-rdb-reader - 1.6 - provided - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.teradatareader.io.netty - - - com.google - shade.teradatareader.com.google - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml deleted file mode 100644 index bd44533374..0000000000 --- a/flinkx-teradata/flinkx-teradata-writer/pom.xml +++ /dev/null @@ -1,108 +0,0 @@ - - - - flinkx-teradata - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-teradata-writer - - - - com.dtstack.flinkx - flinkx-teradata-core - 1.6 - - - com.dtstack.flinkx - flinkx-rdb-writer - 1.6 - provided - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.teradatawriter.io.netty - - - com.google - shade.teradatawriter.com.google - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index 63a0652344..02be833871 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -381,17 +381,6 @@ 1.6 - - com.dtstack.flinkx - flinkx-oraclelogminer-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-sqlservercdc-reader - 1.6 - com.dtstack.flinkx flinkx-phoenix-reader diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index f3f76cba63..7b6a1b237a 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -61,7 +61,6 @@ import com.dtstack.flinkx.odps.writer.OdpsWriter; import com.dtstack.flinkx.oracle.reader.OracleReader; import com.dtstack.flinkx.oracle.writer.OracleWriter; -import com.dtstack.flinkx.oraclelogminer.reader.OraclelogminerReader; import com.dtstack.flinkx.phoenix.reader.PhoenixReader; import com.dtstack.flinkx.phoenix.writer.PhoenixWriter; import com.dtstack.flinkx.polardb.reader.PolardbReader; @@ -72,7 +71,6 @@ import com.dtstack.flinkx.redis.writer.RedisWriter; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; -import com.dtstack.flinkx.sqlservercdc.reader.SqlservercdcReader; import com.dtstack.flinkx.stream.reader.StreamReader; import com.dtstack.flinkx.stream.writer.StreamWriter; import com.dtstack.flinkx.util.ResultPrintUtil; @@ -207,9 +205,7 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstrant.KUDU_READER : reader = new KuduReader(config, env); break; case PluginNameConstrant.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; case PluginNameConstrant.POLARDB_READER : reader = new PolardbReader(config, env); break; - case PluginNameConstrant.ORACLE_LOG_MINER_READER : reader = new OraclelogminerReader(config, env); break; case PluginNameConstrant.PHOENIX_READER : reader = new PhoenixReader(config, env); break; - case PluginNameConstrant.SQLSERVER_CDC_READER : reader = new SqlservercdcReader(config, env); break; case PluginNameConstrant.EMQX_READER : reader = new EmqxReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java index 457cf28638..1885a81196 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java @@ -46,10 +46,8 @@ public class PluginNameConstrant { public static final String KAFKA_READER = "kafkareader"; public static final String CLICKHOUSE_READER = "clickhousereader"; public static final String POLARDB_READER = "polardbreader"; - public static final String ORACLE_LOG_MINER_READER = "oraclelogminerreader"; public static final String PHOENIX_READER = "phoenixreader"; public static final String EMQX_READER = "emqxreader"; - public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; public static final String STREAM_WRITER = "streamwriter"; public static final String CARBONDATA_WRITER = "carbondatawriter"; diff --git a/flinkx-test/src/main/resources/dev_test_job/logminer_stream.json b/flinkx-test/src/main/resources/dev_test_job/logminer_stream.json deleted file mode 100644 index f491e85bc2..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/logminer_stream.json +++ /dev/null @@ -1,63 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "JIANGBO", - "startSCN" : "999660", - "password" : "oracle", - "cat" : "insert", - "jdbcUrl" : "jdbc:oracle:thin:@172.16.8.178:1522:xe", - "readPosition" : "scn", - "pavingData" : true, - "table" : [ "JIANGBO.TB_SOURCE_2" ], - "username" : "system" - }, - "name" : "oraclelogminerreader", - "type" : 2 - }, - "writer" : { - "parameter" : { - "fileName" : "pt", - "writeMode" : "append", - "fieldDelimiter" : "\u0001", - "partitionType" : "DAY", - "path" : "", - "analyticalRules" : "stream_${schema}_${table}", - "password" : "", - "tablesColumn" : "{\"TB_SOURCE_2\":[{\"type\":\"NUMBER(8,0)\",\"key\":\"before_ID\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"NUMBER(8,0)\",\"key\":\"after_ID\"},{\"type\":\"VARCHAR2\",\"key\":\"before_NAME\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR2\",\"key\":\"after_NAME\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", - "partition" : "pt", - "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "dfs.namenode.rpc-address.ns1.nn2" : "172.16.10.204:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "172.16.10.195:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - "jdbcUrl" : "jdbc:hive2://172.16.10.195:10000/shier_test", - "defaultFS" : "hdfs://ns1", - "fileType" : "text", - "charsetName" : "utf-8", - "username" : "", - "bufferSize" : 10485760 - }, - "name" : "hivewriter", - "type" : 7 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/stream_template.json b/flinkx-test/src/main/resources/dev_test_job/stream_template.json index 7284af3cac..7555ac87c7 100644 --- a/flinkx-test/src/main/resources/dev_test_job/stream_template.json +++ b/flinkx-test/src/main/resources/dev_test_job/stream_template.json @@ -1,37 +1 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "password" : "Dtstack#2020", - "port" : 10219, - "cat" : "insert,update,delete", - "host" : "cdb-lp88g9sa.bj.tencentcdb.com", - "jdbcUrl" : "jdbc:mysql://cdb-lp88g9sa.bj.tencentcdb.com:10219/binlog_test", - "start" : { }, - "pavingData" : false, - "table" : [ "binlog_test" ], - "username" : "root" - }, - "name" : "binlogreader" - }, - "writer" : { - "parameter" : { - - }, - "name" : "streamwriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file +{"job":{"content":[{"reader":{"parameter":{"password":"DT@Stack#123","customSql":"","column":[{"name":"id","type":"INT","key":"id"},{"name":"name","type":"VARCHAR","key":"name"},{"name":"length","type":"BIGINT","key":"length"},{"name":"salary","type":"DOUBLE","key":"salary"},{"name":"birthday","type":"DATETIME","key":"birthday"}],"connection":[{"sourceId":13,"password":"DT@Stack#123","jdbcUrl":["jdbc:mysql://172.16.101.136:3306/db_dtinsight_test"],"type":1,"table":["t_dtinsight_test"],"username":"drpeco"}],"sourceIds":[13],"username":"drpeco"},"name":"mysqlreader"},"writer":{"parameter":{"fileName":"pt=20200308","column":[{"name":"id","index":0,"type":"bigint","key":"id"},{"name":"name","index":1,"type":"string","key":"name"},{"name":"length","index":2,"type":"bigint","key":"length"},{"name":"salary","index":3,"type":"double","key":"salary"},{"name":"birthday","index":4,"type":"string","key":"birthday"}],"writeMode":"overwrite","fieldDelimiter":"\u0001","encoding":"utf-8","fullColumnName":["id","name","length","salary","birthday"],"path":"hdfs://ns1/dtInsight/hive/warehouse/dt_batch.db/t_dtinsight_test0305","partition":"pt=20200308","hadoopConfig":{"fs.defaultFS":"hdfs://ns1","hadoop.proxyuser.admin.groups":"*","dfs.replication":"3","dfs.ha.fencing.methods":"sshfence","dfs.client.failover.proxy.provider.ns1":"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider","dfs.ha.fencing.ssh.private-key-files":"~/.ssh/id_rsa","fs.hdfs.impl.disable.cache":true,"dfs.nameservices":"ns1","dfs.safemode.threshold.pct":"0.5","dfs.ha.namenodes.ns1":"nn1,nn2","dfs.namenode.name.dir":"file:/data/hadoop/hdfs/name","dfs.journalnode.rpc-address":"0.0.0.0:8485","fs.trash.interval":"14400","dfs.journalnode.http-address":"0.0.0.0:8480","dfs.namenode.rpc-address.ns1.nn2":"172.16.101.136:9000", "dfs.namenode.rpc-address.ns1.nn1":"172.16.100.216:9000","dfs.datanode.data.dir":"file:/data/hadoop/hdfs/data","dfs.namenode.shared.edits.dir":"qjournal://172.16.100.216:8485;172.16.101.136:8485/namenode-ha-data","openKerberos":false,"fs.hdfs.impl":"org.apache.hadoop.hdfs.DistributedFileSystem","hadoop.tmp.dir":"/data/hadoop_admin","dfs.journalnode.edits.dir":"/data/hadoop/hdfs/journal","dfs.namenode.http-address.ns1.nn2":"172.16.101.136:50070","dfs.namenode.datanode.registration.ip-hostname-check":"false","dfs.namenode.http-address.ns1.nn1":"172.16.100.216:50070","hadoop.proxyuser.admin.hosts":"*","md5zip":"62f8b316522645169c47e154cd1277ff","dfs.ha.automatic-failover.enabled":"true"},"defaultFS":"hdfs://ns1","connection":[{"jdbcUrl":"jdbc:hive2://172.16.101.227:10000/dt_batch","table":["t_dtinsight_test0305"]}],"fileType":"orc","sourceIds":[11],"fullColumnType":["bigint","string","bigint","double","string"]},"name":"hdfswriter"}}],"setting":{"restore":{"maxRowNumForCheckpoint":0,"isRestore":false,"restoreColumnName":"","restoreColumnIndex":0},"errorLimit":{"record":100},"speed":{"bytes":0,"channel":1}}}} \ No newline at end of file diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/Plugin.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/Plugin.java deleted file mode 100644 index e0829ef303..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/Plugin.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.dtstack.flinkx.test.annotation; - -/** - * @author jiangbo - * @date 2020/3/6 - */ -public enum Plugin { - - STREAM, - BINLOG, - CARBONDATA, - CLICKHOUSE, - DB2, - DRUID, - EMQX, - ES, - FTP, - GBASE, - HBASE, - KAFKA, - KAFKA09, - KAFKA10, - KAFKA11, - KUDU, - MONGODB, - MYSQL, - ODPS, - ORACLE, - ORACLE_LOGMINER, - OSS, - PHOENIX, - POLARDB, - POSTGRESQL, - REDIS, - SAPHANA, - SQLSERVER, - SQLSERVER_CDC, - TERADATA -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/PluginType.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/PluginType.java deleted file mode 100644 index 281b22c8e8..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/PluginType.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.dtstack.flinkx.test.annotation; - -/** - * @author jiangbo - * @date 2020/3/6 - */ -public enum PluginType { - - READER, WRITER -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/TestCase.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/TestCase.java deleted file mode 100644 index 3fa79c40fe..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/annotation/TestCase.java +++ /dev/null @@ -1,16 +0,0 @@ -package com.dtstack.flinkx.test.annotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Target; - -/** - * @author jiangbo - * @date 2020/3/6 - */ -@Target(ElementType.TYPE) -public @interface TestCase { - - public Plugin plugin(); - - public PluginType type(); -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/BaseTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/BaseTest.java deleted file mode 100644 index fde86078f8..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/BaseTest.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core; - -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.source.DataSource; -import com.dtstack.flinkx.test.core.source.DataSourceFactory; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.SkipException; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; - -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * @author jiangbo - * @date 2020/2/11 - */ -public abstract class BaseTest { - - protected static final Logger LOG = LoggerFactory.getLogger(BaseTest.class); - - public static final String DEFAULT_DATA_SOURCE_CONFIG_PATH = "/src/test/resources/testCase/dataSource"; - - public static final String KEY_USER_DIR = "user.dir"; - - public static final String KEY_DATA_SOURCE_CONFIG_PATH = "configPath"; - - protected AtomicBoolean isDataSourceValid = new AtomicBoolean(false); - - protected JSONObject connectionConfig; - - protected JSONObject actionBeforeTest; - - protected JSONObject actionAfterTest; - - @BeforeClass - public void actionBeforeClass() { - // 准备数据源 - prepareDataSource(); - - // 检查数据源有效性 - if (isDataSourceValid()) { - try { - prepareData(); - isDataSourceValid.set(true); - } catch (Exception e) { - LOG.warn("准备数据出错:", e); - } - } - } - - private void prepareDataSource() { - String configPath = getDataSourceConfigPath(); - JSONObject dataSourceConfig = FileUtil.readJson(configPath); - DataSource dataSource = DataSourceFactory.getDataSource(dataSourceConfig, getDataSourceName()); - connectionConfig = dataSource.prepare(); - } - - private String getDataSourceConfigPath() { - String configPath = System.getProperty(KEY_DATA_SOURCE_CONFIG_PATH); - if (StringUtils.isNotEmpty(configPath)) { - return configPath + "/source.json"; - } else { - String userDir = System.getProperty(KEY_USER_DIR); - return userDir + DEFAULT_DATA_SOURCE_CONFIG_PATH + "/source.json"; - } - } - - private void prepareData() throws Exception{ - readActionConfig(); - prepareDataInternal(); - } - - private void readActionConfig() { - String userDir = System.getProperty(KEY_USER_DIR); - String path = String.format("%s%s/%s.json", userDir, DEFAULT_DATA_SOURCE_CONFIG_PATH, getDataSourceName()); - JSONObject actionJson = FileUtil.readJson(path); - actionBeforeTest = actionJson.getJSONObject("actionBeforeTest"); - actionAfterTest = actionJson.getJSONObject("actionAfterTest"); - } - - @BeforeMethod - public void beforeMethod() { - if (!isDataSourceValid.get()) { - throw new SkipException(getDataSourceName() + "数据源无效,将跳过测试."); - } - } - - @AfterClass - public void actionAfterClass() { - if (!isDataSourceValid.get()) { - return; - } - - try { - cleanData(); - } catch (Exception e) { - LOG.warn("清里数据出错:", e); - } - } - - protected abstract String getDataSourceName(); - - protected abstract boolean isDataSourceValid(); - - protected abstract void prepareDataInternal() throws Exception; - - protected abstract void cleanData() throws Exception; -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/DefaultParameterReplace.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/DefaultParameterReplace.java deleted file mode 100644 index 67c6d25fa9..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/DefaultParameterReplace.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core; - -import com.alibaba.fastjson.JSONObject; -import org.apache.commons.lang.StringUtils; - -/** - * @author jiangbo - * @date 2020/2/20 - */ -public class DefaultParameterReplace implements ParameterReplace { - - private JSONObject connectionConfig; - - private String pluginType; - - private String key; - - public DefaultParameterReplace(JSONObject connectionConfig, String pluginType) { - this.connectionConfig = connectionConfig; - this.pluginType = pluginType; - } - - public DefaultParameterReplace(JSONObject connectionConfig, String pluginType, String key) { - this.connectionConfig = connectionConfig; - this.pluginType = pluginType; - this.key = key; - } - - @Override - public String replaceParameter(String job) { - if (connectionConfig == null || connectionConfig.isEmpty()) { - return job; - } - - JSONObject jobJson = JSONObject.parseObject(job); - - if (StringUtils.isEmpty(key)) { - jobJson.getJSONObject("job") - .getJSONArray("content") - .getJSONObject(0).getJSONObject(pluginType) - .getJSONObject("parameter") - .putAll(connectionConfig); - } else { - jobJson.getJSONObject("job") - .getJSONArray("content") - .getJSONObject(0).getJSONObject(pluginType) - .getJSONObject("parameter") - .put(key, connectionConfig); - } - - return jobJson.toJSONString(); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/FileUtil.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/FileUtil.java deleted file mode 100644 index 7d8c23b92d..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/FileUtil.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core; - -import com.alibaba.fastjson.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.nio.charset.StandardCharsets; - -/** - * @author jiangbo - * @date 2020/2/11 - */ -public class FileUtil { - - public static Logger LOG = LoggerFactory.getLogger(FileUtil.class); - - public static JSONObject readJson(String filePath) { - return JSONObject.parseObject(readFile(filePath)); - } - - public static String readFile(String filePath) { - try { - File file = new File(filePath); - if (!file.exists()) { - throw new RuntimeException("文件[" + filePath + "]不存在"); - } - - if (file.isDirectory()) { - throw new RuntimeException("[" + filePath + "]是目录,无法读取"); - } - - FileInputStream in = new FileInputStream(file); - byte[] fileContent = new byte[(int) file.length()]; - in.read(fileContent); - return new String(fileContent, StandardCharsets.UTF_8); - } catch (Exception e){ - LOG.warn("读取文件出错:", e); - } - - return ""; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ParameterReplace.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ParameterReplace.java deleted file mode 100644 index 2710548aaa..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ParameterReplace.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.dtstack.flinkx.test.core; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public interface ParameterReplace { - - String replaceParameter(String job); -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ReaderUnitTestLauncher.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ReaderUnitTestLauncher.java deleted file mode 100644 index eb6a7117e5..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/ReaderUnitTestLauncher.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core; - -import com.dtstack.flink.api.java.MyLocalStreamEnvironment; -import com.dtstack.flinkx.binlog.reader.BinlogReader; -import com.dtstack.flinkx.carbondata.reader.CarbondataReader; -import com.dtstack.flinkx.clickhouse.reader.ClickhouseReader; -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.SpeedConfig; -import com.dtstack.flinkx.db2.reader.Db2Reader; -import com.dtstack.flinkx.es.reader.EsReader; -import com.dtstack.flinkx.ftp.reader.FtpReader; -import com.dtstack.flinkx.gbase.reader.GbaseReader; -import com.dtstack.flinkx.hbase.reader.HbaseReader; -import com.dtstack.flinkx.hdfs.reader.HdfsReader; -import com.dtstack.flinkx.kafka.reader.KafkaReader; -import com.dtstack.flinkx.kafka09.reader.Kafka09Reader; -import com.dtstack.flinkx.kafka10.reader.Kafka10Reader; -import com.dtstack.flinkx.kafka11.reader.Kafka11Reader; -import com.dtstack.flinkx.kudu.reader.KuduReader; -import com.dtstack.flinkx.mongodb.reader.MongodbReader; -import com.dtstack.flinkx.mysql.reader.MysqlReader; -import com.dtstack.flinkx.mysqld.reader.MysqldReader; -import com.dtstack.flinkx.odps.reader.OdpsReader; -import com.dtstack.flinkx.oracle.reader.OracleReader; -import com.dtstack.flinkx.oraclelogminer.reader.OraclelogminerReader; -import com.dtstack.flinkx.polardb.reader.PolardbReader; -import com.dtstack.flinkx.postgresql.reader.PostgresqlReader; -import com.dtstack.flinkx.reader.BaseDataReader; -import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; -import com.dtstack.flinkx.stream.reader.StreamReader; -import com.dtstack.flinkx.test.PluginNameConstrant; -import com.dtstack.flinkx.test.core.result.ReaderResult; -import com.dtstack.flinkx.test.core.result.BaseResult; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.types.Row; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -/** - * @author jiangbo - */ -public class ReaderUnitTestLauncher extends UnitTestLauncher { - - private static List resultData; - - public ReaderUnitTestLauncher() { - resultData = Collections.synchronizedList(new ArrayList<>()); - } - - @Override - public String pluginType() { - return "reader"; - } - - @Override - public BaseResult runJob() throws Exception{ - String job = readJob(); - - for (ParameterReplace parameterReplace : parameterReplaces) { - job = parameterReplace.replaceParameter(job); - } - - job = replaceChannel(job); - - DataTransferConfig config = DataTransferConfig.parse(job); - SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); - - MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); - - openCheckpointConf(env); - - env.setParallelism(speedConfig.getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); - - BaseDataReader reader = buildDataReader(config, env); - DataStream dataStream = reader.readData(); - - dataStream.addSink(new SinkFunction() { - @Override - public void invoke(Row value, Context context) throws Exception { - resultData.add(value); - } - }); - - if(StringUtils.isNotEmpty(savepointPath)){ - env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); - } - - JobExecutionResult executionResult = env.execute(); - return new ReaderResult(executionResult, resultData); - } - - private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env){ - String readerName = config.getJob().getContent().get(0).getReader().getName(); - BaseDataReader reader ; - switch (readerName){ - case PluginNameConstrant.STREAM_READER : reader = new StreamReader(config, env); break; - case PluginNameConstrant.CARBONDATA_READER : reader = new CarbondataReader(config, env); break; - case PluginNameConstrant.ORACLE_READER : reader = new OracleReader(config, env); break; - case PluginNameConstrant.POSTGRESQL_READER : reader = new PostgresqlReader(config, env); break; - case PluginNameConstrant.SQLSERVER_READER : reader = new SqlserverReader(config, env); break; - case PluginNameConstrant.MYSQLD_READER : reader = new MysqldReader(config, env); break; - case PluginNameConstrant.MYSQL_READER : reader = new MysqlReader(config, env); break; - case PluginNameConstrant.DB2_READER : reader = new Db2Reader(config, env); break; - case PluginNameConstrant.GBASE_READER : reader = new GbaseReader(config, env); break; - case PluginNameConstrant.ES_READER : reader = new EsReader(config, env); break; - case PluginNameConstrant.FTP_READER : reader = new FtpReader(config, env); break; - case PluginNameConstrant.HBASE_READER : reader = new HbaseReader(config, env); break; - case PluginNameConstrant.HDFS_READER : reader = new HdfsReader(config, env); break; - case PluginNameConstrant.MONGODB_READER : reader = new MongodbReader(config, env); break; - case PluginNameConstrant.ODPS_READER : reader = new OdpsReader(config, env); break; - case PluginNameConstrant.BINLOG_READER : reader = new BinlogReader(config, env); break; - case PluginNameConstrant.KAFKA09_READER : reader = new Kafka09Reader(config, env); break; - case PluginNameConstrant.KAFKA10_READER : reader = new Kafka10Reader(config, env); break; - case PluginNameConstrant.KAFKA11_READER : reader = new Kafka11Reader(config, env); break; - case PluginNameConstrant.KAFKA_READER : reader = new KafkaReader(config, env); break; - case PluginNameConstrant.KUDU_READER : reader = new KuduReader(config, env); break; - case PluginNameConstrant.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; - case PluginNameConstrant.POLARDB_READER : reader = new PolardbReader(config, env); break; - case PluginNameConstrant.ORACLE_LOG_MINER_READER : reader = new OraclelogminerReader(config, env); break; - default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); - } - - return reader; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/UnitTestLauncher.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/UnitTestLauncher.java deleted file mode 100644 index 583403dfa5..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/UnitTestLauncher.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core; - -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.constants.ConfigConstant; -import com.dtstack.flinkx.test.core.result.BaseResult; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.LinkedList; -import java.util.concurrent.TimeUnit; - -/** - * @author jiangbo - */ -public abstract class UnitTestLauncher { - - public static Logger LOG = LoggerFactory.getLogger(UnitTestLauncher.class); - - private static final int FAILURE_RATE = 3; - - private static final int FAILURE_INTERVAL = 6; - - private static final int DELAY_INTERVAL = 10; - - protected String pluginName; - - protected String testCaseName; - - protected Configuration conf = new Configuration(); - - protected String savepointPath; - - protected LinkedList parameterReplaces = new LinkedList<>(); - - protected int channel = 1; - - public abstract BaseResult runJob() throws Exception; - - public abstract String pluginType(); - - public UnitTestLauncher withChannel(int channel) { - this.channel = channel; - return this; - } - - public UnitTestLauncher withParameterReplace(ParameterReplace parameterReplace) { - parameterReplaces.add(parameterReplace); - return this; - } - - public UnitTestLauncher withPluginName(String pluginName) { - this.pluginName = pluginName; - return this; - } - - public UnitTestLauncher withTestCaseName(String testCaseName) { - this.testCaseName = testCaseName; - return this; - } - - public UnitTestLauncher withConf(Configuration conf) { - this.conf.addAll(conf); - return this; - } - - public UnitTestLauncher withSavepointPath(String savepointPath) { - this.savepointPath = savepointPath; - return this; - } - - protected String readJob() { - String userDir = System.getProperty("user.dir"); - String jobFilePath = String.format("%s/src/test/resources/testCase/%s/%s/%s.json", userDir, pluginName, pluginType(), testCaseName); - return FileUtil.readFile(jobFilePath); - } - - protected String replaceChannel(String job) { - JSONObject jobJson = JSONObject.parseObject(job); - jobJson.getJSONObject("job") - .getJSONObject("setting") - .getJSONObject("speed") - .put("channel", channel); - - return jobJson.toJSONString(); - } - - protected void openCheckpointConf(StreamExecutionEnvironment env){ - if(conf == null){ - return; - } - - if(!conf.containsKey(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY)){ - return; - }else{ - long interval = conf.getLong(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY, -1); - if (interval == -1) { - return; - } - - env.enableCheckpointing(interval); - LOG.info("Open checkpoint with interval:" + interval); - } - - String checkpointTimeoutStr = conf.getString(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY, null); - if(checkpointTimeoutStr != null){ - long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); - //checkpoints have to complete within one min,or are discard - env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); - - LOG.info("Set checkpoint timeout:" + checkpointTimeout); - } - - env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); - env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); - env.getCheckpointConfig().enableExternalizedCheckpoints( - CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); - - env.setRestartStrategy(RestartStrategies.failureRateRestart( - FAILURE_RATE, - Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), - Time.of(DELAY_INTERVAL, TimeUnit.SECONDS) - )); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/WriterUnitTestLauncher.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/WriterUnitTestLauncher.java deleted file mode 100644 index 4d7acc7e0d..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/WriterUnitTestLauncher.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core; - -import com.dtstack.flink.api.java.MyLocalStreamEnvironment; -import com.dtstack.flinkx.carbondata.writer.CarbondataWriter; -import com.dtstack.flinkx.clickhouse.writer.ClickhouseWriter; -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.SpeedConfig; -import com.dtstack.flinkx.db2.writer.Db2Writer; -import com.dtstack.flinkx.es.writer.EsWriter; -import com.dtstack.flinkx.ftp.writer.FtpWriter; -import com.dtstack.flinkx.gbase.writer.GbaseWriter; -import com.dtstack.flinkx.hbase.writer.HbaseWriter; -import com.dtstack.flinkx.hdfs.writer.HdfsWriter; -import com.dtstack.flinkx.hive.writer.HiveWriter; -import com.dtstack.flinkx.kafka.writer.KafkaWriter; -import com.dtstack.flinkx.kafka09.writer.Kafka09Writer; -import com.dtstack.flinkx.kafka10.writer.Kafka10Writer; -import com.dtstack.flinkx.kafka11.writer.Kafka11Writer; -import com.dtstack.flinkx.kudu.writer.KuduWriter; -import com.dtstack.flinkx.mongodb.writer.MongodbWriter; -import com.dtstack.flinkx.mysql.writer.MysqlWriter; -import com.dtstack.flinkx.odps.writer.OdpsWriter; -import com.dtstack.flinkx.oracle.writer.OracleWriter; -import com.dtstack.flinkx.polardb.writer.PolardbWriter; -import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; -import com.dtstack.flinkx.redis.writer.RedisWriter; -import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; -import com.dtstack.flinkx.stream.writer.StreamWriter; -import com.dtstack.flinkx.test.PluginNameConstrant; -import com.dtstack.flinkx.test.core.result.BaseResult; -import com.dtstack.flinkx.test.core.result.WriterResult; -import com.dtstack.flinkx.writer.BaseDataWriter; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.types.Row; - -/** - * @author jiangbo - */ -public class WriterUnitTestLauncher extends UnitTestLauncher { - - private Row[] records; - - public WriterUnitTestLauncher withRecords(Row[] records) { - this.records = records; - return this; - } - - @Override - public String pluginType() { - return "writer"; - } - - @Override - public BaseResult runJob() throws Exception{ - String job = readJob(); - - for (ParameterReplace parameterReplace : parameterReplaces) { - job = parameterReplace.replaceParameter(job); - } - - DataTransferConfig config = DataTransferConfig.parse(job); - SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); - - MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); - - openCheckpointConf(env); - - env.setParallelism(speedConfig.getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); - - DataStream dataStream = env.fromElements(records); - - BaseDataWriter writer = buildDataWriter(config); - writer.writeData(dataStream); - - if(StringUtils.isNotEmpty(savepointPath)){ - env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); - } - - JobExecutionResult executionResult = env.execute(); - return new WriterResult(executionResult); - } - - private static BaseDataWriter buildDataWriter(DataTransferConfig config){ - String writerName = config.getJob().getContent().get(0).getWriter().getName(); - BaseDataWriter writer; - switch (writerName){ - case PluginNameConstrant.STREAM_WRITER : writer = new StreamWriter(config); break; - case PluginNameConstrant.CARBONDATA_WRITER : writer = new CarbondataWriter(config); break; - case PluginNameConstrant.MYSQL_WRITER : writer = new MysqlWriter(config); break; - case PluginNameConstrant.SQLSERVER_WRITER : writer = new SqlserverWriter(config); break; - case PluginNameConstrant.ORACLE_WRITER : writer = new OracleWriter(config); break; - case PluginNameConstrant.POSTGRESQL_WRITER : writer = new PostgresqlWriter(config); break; - case PluginNameConstrant.DB2_WRITER : writer = new Db2Writer(config); break; - case PluginNameConstrant.GBASE_WRITER : writer = new GbaseWriter(config); break; - case PluginNameConstrant.ES_WRITER : writer = new EsWriter(config); break; - case PluginNameConstrant.FTP_WRITER : writer = new FtpWriter(config); break; - case PluginNameConstrant.HBASE_WRITER : writer = new HbaseWriter(config); break; - case PluginNameConstrant.HDFS_WRITER : writer = new HdfsWriter(config); break; - case PluginNameConstrant.MONGODB_WRITER : writer = new MongodbWriter(config); break; - case PluginNameConstrant.ODPS_WRITER : writer = new OdpsWriter(config); break; - case PluginNameConstrant.REDIS_WRITER : writer = new RedisWriter(config); break; - case PluginNameConstrant.HIVE_WRITER : writer = new HiveWriter(config); break; - case PluginNameConstrant.KAFKA09_WRITER : writer = new Kafka09Writer(config); break; - case PluginNameConstrant.KAFKA10_WRITER : writer = new Kafka10Writer(config); break; - case PluginNameConstrant.KAFKA11_WRITER : writer = new Kafka11Writer(config); break; - case PluginNameConstrant.KUDU_WRITER : writer = new KuduWriter(config); break; - case PluginNameConstrant.CLICKHOUSE_WRITER : writer = new ClickhouseWriter(config); break; - case PluginNameConstrant.POLARDB_WRITER : writer = new PolardbWriter(config); break; - case PluginNameConstrant.KAFKA_WRITER : writer = new KafkaWriter(config); break; - default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); - } - - return writer; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/BaseResult.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/BaseResult.java deleted file mode 100644 index 65d6a1509d..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/BaseResult.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.result; - -import org.apache.flink.api.common.JobExecutionResult; - -/** - * @author jiangbo - * @date 2020/2/8 - */ -public abstract class BaseResult { - - JobExecutionResult executionResult; - - public BaseResult(JobExecutionResult executionResult) { - this.executionResult = executionResult; - } - - public JobExecutionResult getExecutionResult() { - return executionResult; - } - - public void setExecutionResult(JobExecutionResult executionResult) { - this.executionResult = executionResult; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/ReaderResult.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/ReaderResult.java deleted file mode 100644 index 03272c2a44..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/ReaderResult.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.result; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.types.Row; - -import java.util.List; - -/** - * @author jiangbo - * @date 2020/2/8 - */ -public class ReaderResult extends BaseResult { - - List resultData; - - public ReaderResult(JobExecutionResult executionResult, List resultData) { - super(executionResult); - this.resultData = resultData; - } - - public List getResultData() { - return resultData; - } - - public void setResultData(List resultData) { - this.resultData = resultData; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/WriterResult.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/WriterResult.java deleted file mode 100644 index 008b02c509..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/result/WriterResult.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.result; - -import org.apache.flink.api.common.JobExecutionResult; - - -/** - * @author jiangbo - * @date 2020/2/8 - */ -public class WriterResult extends BaseResult { - - public WriterResult(JobExecutionResult executionResult) { - super(executionResult); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSource.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSource.java deleted file mode 100644 index 4da35415b7..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSource.java +++ /dev/null @@ -1,16 +0,0 @@ -package com.dtstack.flinkx.test.core.source; - -import com.alibaba.fastjson.JSONObject; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public interface DataSource { - - /** - * 准备数据库 - * @return 返回数据库连接信息 - */ - JSONObject prepare(); -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSourceFactory.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSourceFactory.java deleted file mode 100644 index b7c0ab455f..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/DataSourceFactory.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source; - -import com.alibaba.fastjson.JSONObject; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public class DataSourceFactory { - - public static DataSource getDataSource(JSONObject config, String dataSourceName) { - JSONObject singleSourceConfig = config.getJSONObject(dataSourceName); - - String initType = singleSourceConfig.getString("initType"); - if ("static".equalsIgnoreCase(initType)) { - return new StaticDataSource(singleSourceConfig); - } - - if ("hdfs".equalsIgnoreCase(dataSourceName)) { - return new HdfsDataSource(singleSourceConfig); - } else if ("hive".equalsIgnoreCase(dataSourceName)){ - return new HiveDataSource(singleSourceConfig); - } else { - return new StaticDataSource(singleSourceConfig); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HdfsDataSource.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HdfsDataSource.java deleted file mode 100644 index be2efaf012..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HdfsDataSource.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source; - -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.source.embedded.EmbeddedHDFSService; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public class HdfsDataSource implements DataSource { - - private JSONObject config; - - public HdfsDataSource(JSONObject config) { - this.config = config; - } - - @Override - public JSONObject prepare() { - EmbeddedHDFSService hdfsService = new EmbeddedHDFSService(); - return hdfsService.startService(); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HiveDataSource.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HiveDataSource.java deleted file mode 100644 index d42f2f7407..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/HiveDataSource.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source; - -import com.alibaba.fastjson.JSONObject; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public class HiveDataSource implements DataSource { - - private JSONObject config; - - public HiveDataSource(JSONObject config) { - this.config = config; - } - - @Override - public JSONObject prepare() { - return new JSONObject(); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/StaticDataSource.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/StaticDataSource.java deleted file mode 100644 index e5b75a2bd0..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/StaticDataSource.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source; - -import com.alibaba.fastjson.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public class StaticDataSource implements DataSource { - - public static Logger LOG = LoggerFactory.getLogger(StaticDataSource.class); - - private JSONObject config; - - public StaticDataSource(JSONObject config) { - this.config = config; - } - - @Override - public JSONObject prepare() { - if (config == null || config.isEmpty()) { - LOG.warn("[config]为null或者为空,返回空JSON对象"); - return new JSONObject(); - } - - return config; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHDFSService.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHDFSService.java deleted file mode 100644 index e4c4fde580..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHDFSService.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source.embedded; - -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.source.HdfsDataSource; -import org.apache.hadoop.hdfs.HdfsConfiguration; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.test.PathUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; - -/** - * 创建启动HDFS服务 - * - * @author jiangbo - * @date 2020/2/29 - */ -public class EmbeddedHDFSService { - - public static Logger LOG = LoggerFactory.getLogger(HdfsDataSource.class); - - public EmbeddedHDFSService() { - - } - - public JSONObject startService() { - JSONObject hadoopConfig = new JSONObject(); - hadoopConfig.put("fs.hdfs.impl.disable.cache", true); - hadoopConfig.put("hadoop.user.name", System.getProperty("user.name")); - - try { - HdfsConfiguration hadoopConf = new HdfsConfiguration(); - - File testDataPath = new File(PathUtils.getTestDir(this.getClass()), "miniclusters"); - File testDataCluster1 = new File(testDataPath, "cluster1"); - String c1Path = testDataCluster1.getAbsolutePath(); - hadoopConf.set("hdfs.minidfs.basedir", c1Path); - - MiniDFSCluster miniDFS = new MiniDFSCluster.Builder(hadoopConf).build(); - - String defaultFs = miniDFS.getFileSystem().getUri().toString(); - LOG.warn("启动HDFS成功:{}", defaultFs); - - hadoopConfig.put("fs.default.name", defaultFs); - } catch (Exception e) { - LOG.warn("启动HDFS服务失败", e); - } - - return hadoopConfig; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHiveService.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHiveService.java deleted file mode 100644 index c82ab24608..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/EmbeddedHiveService.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.core.source.embedded; - -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.source.embedded.hive.InternalHiveServer; -import com.dtstack.flinkx.test.core.source.embedded.hive.InternalMetaStoreServer; -import org.apache.hadoop.hive.conf.HiveConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * 启动Hive服务 - * TODO 未完成 引入的hive包版本不统一,hive server无法启动 - * - * @author jiangbo - * @date 2020/2/29 - */ -public class EmbeddedHiveService { - - public static Logger LOG = LoggerFactory.getLogger(EmbeddedHiveService.class); - - public EmbeddedHiveService() { - } - - public static void main(String[] args) { - LoggerContext loggerContext= (LoggerContext) LoggerFactory.getILoggerFactory(); - //设置全局日志级别 - ch.qos.logback.classic.Logger logger=loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel("error")); - - EmbeddedHiveService hiveService = new EmbeddedHiveService(); - hiveService.startService(); - } - - public JSONObject startService() { - JSONObject hadoopConfig = new JSONObject(); - - try { - EmbeddedHDFSService hdfsService = new EmbeddedHDFSService(); - hadoopConfig = hdfsService.startService(); - } catch (Exception e) { - LOG.warn("启动HDFS服务失败", e); - return hadoopConfig; - } - - HiveConf hiveConf = new HiveConf(); - hiveConf.set("javax.jdo.option.ConnectionURL", "jdbc:derby:;databaseName=metastore_db;create=true"); - hiveConf.set("javax.jdo.option.ConnectionDriverName", "org.apache.derby.jdbc.EmbeddedDriver"); - hiveConf.set("hive.metastore.local", "true"); - hiveConf.set("hive.metastore.warehouse.dir", "/user/hive/warehouse"); - - hadoopConfig.forEach((key, val) -> { - if (null != val) { - hiveConf.set(key, val.toString()); - } - }); - - startMetaStore(hiveConf); - startHiveServer2(hiveConf); - - return hadoopConfig; - } - - private void startMetaStore(HiveConf hiveConf){ - hiveConf.set("hive.metastore.uris", "thrift://127.0.0.1:9083"); - - InternalMetaStoreServer metaStore = null; - try { - metaStore = new InternalMetaStoreServer(hiveConf); - metaStore.start(); - } catch (Exception e) { - LOG.error("", e); - if (metaStore != null) { - try { - metaStore.shutdown(); - } catch (Exception ex) { - LOG.error("", ex); - } - } - } - } - - private void startHiveServer2(HiveConf hiveConf){ - InternalHiveServer server = null; - try{ - server = new InternalHiveServer(hiveConf); - server.start(); - } catch (Exception e){ - LOG.error("", e); - if (server != null){ - try { - server.shutdown(); - } catch (Exception ex) { - LOG.error("", ex); - } - } - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/AbstractHiveServer.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/AbstractHiveServer.java deleted file mode 100644 index 625aa2d3c8..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/AbstractHiveServer.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core.source.embedded.hive; - -import com.google.common.base.Strings; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.concurrent.TimeoutException; - -/** - * @author jiangbo - * @date 2020/3/2 - */ -public abstract class AbstractHiveServer implements HiveServer { - - private static final Logger LOGGER = LoggerFactory.getLogger(AbstractHiveServer.class); - - private static final String LINK_FAILURE_SQL_STATE = "08S01"; - - private final Configuration configuration; - private final String hostname; - private final int port; - - public AbstractHiveServer(Configuration configuration, String hostname, - int port) { - this.configuration = configuration; - this.hostname = hostname; - this.port = port; - LOGGER.info("Hive Server2 configured on host: " + hostname + " port:" + port); - } - - @Override - public String getProperty(String key) { - return configuration.get(key); - } - - @Override - public String getURL() { - return "jdbc:hive2://" + hostname + ":" + port + "/default"; - } - - public Connection createConnection(String user, String password) throws Exception{ - String url = getURL(); - DriverManager.setLoginTimeout(0); - Connection connection = DriverManager.getConnection(url, user, password); - return connection; - } - - protected static String getHostname(HiveConf hiveConf) { - return hiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.toString(), "localhost").trim(); - } - protected static int getPort(HiveConf hiveConf) { - return Integer.parseInt(hiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.toString(), "10000").trim()); - } - protected static void waitForStartup(HiveServer hiveServer) throws Exception { - int waitTime = 0; - long startupTimeout = 1000L * 10L; - do { - Thread.sleep(500L); - waitTime += 500L; - if (waitTime > startupTimeout) { - throw new TimeoutException("Couldn't access new HiveServer: " + hiveServer.getURL()); - } - try { - DriverManager.setLoginTimeout(30); - Connection connection = DriverManager.getConnection(hiveServer.getURL(), "root", "abc123"); - connection.close(); - break; - } catch (SQLException e) { - e.printStackTrace(); - String state = Strings.nullToEmpty(e.getSQLState()).trim(); - if (!state.equalsIgnoreCase(LINK_FAILURE_SQL_STATE)) { - throw e; - } - } - } while (true); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/HiveServer.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/HiveServer.java deleted file mode 100644 index ddfaace356..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/HiveServer.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core.source.embedded.hive; - -import java.sql.Connection; - -/** - * @author jiangbo - * @date 2020/3/2 - */ -public interface HiveServer { - - void start() throws Exception; - - void shutdown() throws Exception; - - String getURL(); - - String getProperty(String key); - - Connection createConnection(String user, String password) throws Exception; -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalHiveServer.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalHiveServer.java deleted file mode 100644 index 05262a4469..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalHiveServer.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core.source.embedded.hive; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hive.service.server.HiveServer2; - -/** - * @author jiangbo - * @date 2020/3/2 - */ -public class InternalHiveServer extends AbstractHiveServer { - - private final HiveServer2 hiveServer2; - private final HiveConf conf; - - public InternalHiveServer(HiveConf conf) throws Exception { - super(conf, getHostname(conf), getPort(conf)); - hiveServer2 = new HiveServer2(); - this.conf = conf; - } - - @Override - public synchronized void start() throws Exception { - hiveServer2.init(conf); - hiveServer2.start(); - waitForStartup(this); - } - - @Override - public synchronized void shutdown() throws Exception { - if (hiveServer2 != null) { - hiveServer2.stop(); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalMetaStoreServer.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalMetaStoreServer.java deleted file mode 100644 index 878c78fc8f..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/core/source/embedded/hive/InternalMetaStoreServer.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test.core.source.embedded.hive; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStore; -import org.apache.hadoop.hive.shims.ShimLoader; - -import java.net.URI; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - * @author jiangbo - * @date 2020/3/2 - */ -public class InternalMetaStoreServer extends AbstractHiveServer { - - private final HiveConf conf; - - private ExecutorService metaStoreExecutor = Executors.newSingleThreadExecutor(); - - public InternalMetaStoreServer(HiveConf conf) throws Exception { - super(conf, getMetastoreHostname(conf), getMetastorePort(conf)); - this.conf = conf; - } - - @Override - public String getURL() { - return "jdbc:hive2://"; - } - - @Override - public void start() throws Exception { - startMetastore(); - } - - @Override - public void shutdown() throws Exception { - metaStoreExecutor.shutdown(); - } - - // async metaStore startup since Hive doesn't have that option - private void startMetastore() throws Exception { - Callable metastoreService = new Callable() { - public Void call() throws Exception { - try { - HiveMetaStore.startMetaStore(getMetastorePort(conf), - ShimLoader.getHadoopThriftAuthBridge(), conf); - } catch (Throwable e) { - throw new Exception("Error starting metastore", e); - } - return null; - } - }; - metaStoreExecutor.submit(metastoreService); - } - - private static String getMetastoreHostname(Configuration conf) - throws Exception { - return new URI(conf.get(HiveConf.ConfVars.METASTOREURIS.varname)).getHost(); - } - - private static int getMetastorePort(Configuration conf) throws Exception { - return new URI(conf.get(HiveConf.ConfVars.METASTOREURIS.varname)).getPort(); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/BaseHdfsTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/BaseHdfsTest.java deleted file mode 100644 index 393bb9257e..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/BaseHdfsTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.hdfs; - -import com.dtstack.flinkx.test.core.BaseTest; -import com.dtstack.flinkx.util.FileSystemUtil; -import org.apache.hadoop.fs.FileSystem; - -/** - * @author jiangbo - * @date 2020/2/29 - */ -public abstract class BaseHdfsTest extends BaseTest { - - public static final String PLUGIN_NAME = "hdfs"; - - protected FileSystem fs; - - @Override - protected String getDataSourceName() { - return PLUGIN_NAME; - } - - @Override - protected boolean isDataSourceValid() { - try { - fs = FileSystemUtil.getFileSystem(connectionConfig, null); - return true; - } catch (Exception e) { - LOG.warn("校验HDFS数据源失败", e); - } - - return false; - } - - @Override - protected void cleanData() throws Exception { - if (null != fs) { - fs.close(); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/reader/HdfsReaderTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/reader/HdfsReaderTest.java deleted file mode 100644 index 4dbaedc851..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/reader/HdfsReaderTest.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.hdfs.reader; - -import com.dtstack.flinkx.test.hdfs.BaseHdfsTest; - -/** - * @author jiangbo - * @date 2020/2/29 - */ -public class HdfsReaderTest extends BaseHdfsTest { - - @Override - protected void prepareDataInternal() throws Exception { - // do nothing - } - - @Override - protected void cleanData() throws Exception { - - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/writer/HdfsWriterTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/writer/HdfsWriterTest.java deleted file mode 100644 index dd9b539be0..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/hdfs/writer/HdfsWriterTest.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.hdfs.writer; - -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.DefaultParameterReplace; -import com.dtstack.flinkx.test.core.WriterUnitTestLauncher; -import com.dtstack.flinkx.test.core.result.WriterResult; -import com.dtstack.flinkx.test.hdfs.BaseHdfsTest; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.flink.types.Row; -import org.junit.Assert; -import org.testng.annotations.Test; - -import java.sql.Timestamp; -import java.util.Date; - -/** - * @author jiangbo - * @date 2020/2/29 - */ -public class HdfsWriterTest extends BaseHdfsTest { - - @Override - protected void prepareDataInternal() throws Exception { - // do nothing - } - - @Test - public void singleChannelNumberTextFileTest() { - try { - int number = 20; - Row[] records = getTextRecords(number); - - JSONObject parameter = new JSONObject(); - parameter.put("fileName", "fileType=text"); - parameter.put("fileType", "text"); - parameter.put("defaultFS", connectionConfig.getString("fs.default.name")); - - WriterResult result = (WriterResult) new WriterUnitTestLauncher() - .withRecords(records) - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new DefaultParameterReplace(connectionConfig, "writer", "hadoopConfig")) - .withParameterReplace(new DefaultParameterReplace(parameter, "writer")) - .withTestCaseName("numberTest") - .runJob(); - - long numberWriter = result.getExecutionResult().getAccumulatorResult("numWrite"); - Assert.assertEquals(number, numberWriter); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelNumberTextFileTest() { - try { - int number = 20; - Row[] records = getTextRecords(number); - - JSONObject parameter = new JSONObject(); - parameter.put("fileName", "fileType=text"); - parameter.put("fileType", "text"); - parameter.put("defaultFS", connectionConfig.getString("fs.default.name")); - - WriterResult result = (WriterResult) new WriterUnitTestLauncher() - .withRecords(records) - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new DefaultParameterReplace(connectionConfig, "writer", "hadoopConfig")) - .withParameterReplace(new DefaultParameterReplace(parameter, "writer")) - .withTestCaseName("numberTest") - .withChannel(2) - .runJob(); - - long numberWriter = result.getExecutionResult().getAccumulatorResult("numWrite"); - Assert.assertEquals(number, numberWriter); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - private Row[] getTextRecords(int number){ - Row[] records = new Row[number]; - for (int i = 0; i < number; i++) { - Row record = new Row(3); - record.setField(0, i); - record.setField(0, (short)i); - record.setField(0, i); - record.setField(0, (long)i); - record.setField(0, 2.34); - record.setField(0, 4.342); - record.setField(0, 45.234); - record.setField(0, "string_" + i); - record.setField(0, "varchar_" + i); - record.setField(0, "char_" + i); - record.setField(0, true); - record.setField(0, new Date()); - record.setField(0, new Timestamp(System.currentTimeMillis())); - record.setField(0, new Byte[3]); - - records[i] = record; - } - - return records; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/MySqlTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/MySqlTest.java deleted file mode 100644 index 3d003a3af0..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/MySqlTest.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.mysql; - -import com.dtstack.flinkx.test.rdb.JdbcTest; - -/** - * @author jiangbo - * @date 2020/2/8 - */ -public abstract class MySqlTest extends JdbcTest { - - public static final String PLUGIN_NAME = "mysql"; - - @Override - protected String getDriverName() { - return "com.mysql.jdbc.Driver"; - } - - @Override - protected String getDataSourceName() { - return PLUGIN_NAME; - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/reader/MySqlReaderTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/reader/MySqlReaderTest.java deleted file mode 100644 index c5f61850d6..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/mysql/reader/MySqlReaderTest.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.mysql.reader; - -import com.dtstack.flinkx.test.core.ReaderUnitTestLauncher; -import com.dtstack.flinkx.test.core.result.ReaderResult; -import com.dtstack.flinkx.test.mysql.MySqlTest; -import com.dtstack.flinkx.test.rdb.JdbcReaderParameterReplace; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.testng.Assert; -import org.testng.annotations.Test; - -/** - * @author jiangbo - * @date 2020/2/8 - */ -public class MySqlReaderTest extends MySqlTest { - - @Override - protected String getPluginType() { - return "reader"; - } - - /** - * 全字段单通道读取条数验证 - */ - @Test - public void singleChannelNumberTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("numberTest") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 20); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - /** - * 全字段多通道读取条数验证 - */ - @Test - public void multiChannelNumberTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("numberTest") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 20); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void singleChannelFilterTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("filterTest") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 15); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelFilterTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("filterTest") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 15); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void singleChannelCustomSqlTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("customSqlTest") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 17); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelCustomSqlTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("customSqlTest") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 17); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void singleChannelIncrementWithoutStartLocation() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementWithoutStartLocation") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 20); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelIncrementWithoutStartLocation() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementWithoutStartLocation") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 20); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void singleChannelIncrementWithStartLocation() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementWithStartLocation") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 10); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelIncrementWithStartLocation() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementWithStartLocation") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 10); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void singleChannelIncrementUseMaxFunc() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementUseMaxFunc") - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 19); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelIncrementUseMaxFunc() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withParameterReplace(new JdbcReaderParameterReplace(connectionConfig)) - .withTestCaseName("incrementUseMaxFunc") - .withChannel(2) - .runJob(); - - Assert.assertEquals(readerResult.getResultData().size(), 19); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcReaderParameterReplace.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcReaderParameterReplace.java deleted file mode 100644 index 5da6e513c5..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcReaderParameterReplace.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.rdb; - -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; -import com.dtstack.flinkx.test.core.ParameterReplace; - -/** - * @author jiangbo - * @date 2020/2/19 - */ -public class JdbcReaderParameterReplace implements ParameterReplace { - - private JSONObject connectionConfig; - - public JdbcReaderParameterReplace(JSONObject connectionConfig) { - this.connectionConfig = connectionConfig; - } - - @Override - public String replaceParameter(String job) { - if (connectionConfig == null || connectionConfig.isEmpty()) { - return job; - } - - JSONObject jobJson = JSONObject.parseObject(job); - - JSONArray table = jobJson.getJSONObject("job") - .getJSONArray("content") - .getJSONObject(0).getJSONObject("reader") - .getJSONObject("parameter").getJSONArray("table"); - - connectionConfig.put("table", table); - - JSONArray connections = new JSONArray(); - connections.add(connectionConfig); - - jobJson.getJSONObject("job") - .getJSONArray("content") - .getJSONObject(0).getJSONObject("reader") - .getJSONObject("parameter") - .put("connection", connections); - - jobJson.getJSONObject("job") - .getJSONArray("content") - .getJSONObject(0).getJSONObject("reader") - .getJSONObject("parameter") - .putAll(connectionConfig); - - return jobJson.toJSONString(); - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcTest.java deleted file mode 100644 index 056bf4e4b1..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/rdb/JdbcTest.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.rdb; - -import com.alibaba.fastjson.JSONArray; -import com.dtstack.flinkx.test.core.BaseTest; -import com.dtstack.flinkx.util.ClassUtil; -import org.apache.commons.lang.StringUtils; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.Statement; - -/** - * @author jiangbo - * @date 2020/2/11 - */ -public abstract class JdbcTest extends BaseTest { - - public static final String KEY_JDBC_URL = "jdbcUrl"; - public static final String KEY_USERNAME = "username"; - public static final String KEY_PASSWORD = "password"; - public static final String KEY_INIT_SQL = "initSql"; - - protected Connection connection; - - @Override - protected boolean isDataSourceValid() { -// try { -// ClassUtil.forName(getDriverName()); -// String jdbcUrl = connectionConfig.getJSONArray(KEY_JDBC_URL).getString(0); -// String username = connectionConfig.getString(KEY_USERNAME); -// String password = connectionConfig.getString(KEY_PASSWORD); -// connection = DriverManager.getConnection(jdbcUrl, username, password); -// return true; -// } catch (Exception e) { -// LOG.error("获取数据源连接出错:", e); -// } - - return false; - } - - @Override - protected void prepareDataInternal() throws Exception { - JSONArray sqlArray = actionBeforeTest.getJSONArray(getPluginType()); - executeBatchSql(sqlArray); - } - - private void executeBatchSql(JSONArray initSqlArray) throws Exception { - if (initSqlArray == null || initSqlArray.isEmpty()) { - return; - } - - Statement statement; - try { - statement = connection.createStatement(); - } catch (Exception e) { - throw new RuntimeException("创建[statement]出错:", e); - } - - for (Object sql : initSqlArray) { - if (sql == null || StringUtils.isEmpty(sql.toString())) { - continue; - } - - try { - statement.execute(sql.toString()); - } catch (Exception e) { - statement.close(); - LOG.error("执行sql[{}]出错", sql); - throw new RuntimeException(e); - } - } - } - - @Override - protected void cleanData() throws Exception { - JSONArray sqlArray = actionAfterTest.getJSONArray(getPluginType()); - executeBatchSql(sqlArray); - closeConnection(); - } - - private void closeConnection() { - try { - if (connection != null) { - connection.close(); - } - } catch (Exception e) { - LOG.warn("关闭数据源连接出错:", e); - } - } - - protected abstract String getDriverName(); - - protected abstract String getPluginType(); -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/reader/StreamReaderTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/reader/StreamReaderTest.java deleted file mode 100644 index 9721d468b2..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/reader/StreamReaderTest.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.stream.reader; - -import com.dtstack.flinkx.test.annotation.Plugin; -import com.dtstack.flinkx.test.annotation.PluginType; -import com.dtstack.flinkx.test.annotation.TestCase; -import com.dtstack.flinkx.test.core.BaseTest; -import com.dtstack.flinkx.test.core.ReaderUnitTestLauncher; -import com.dtstack.flinkx.test.core.result.ReaderResult; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.testng.Assert; -import org.testng.annotations.Test; - -/** - * - * - * @author jiangbo - * @date 2020/2/8 - */ -@TestCase(plugin = Plugin.STREAM, type = PluginType.READER) -public class StreamReaderTest extends BaseTest { - - public static final String PLUGIN_NAME = "stream"; - - @Override - protected String getDataSourceName() { - return PLUGIN_NAME; - } - - @Override - protected boolean isDataSourceValid() { - // stream插件不对接数据源 - return true; - } - - @Override - protected void prepareDataInternal() { - // 不需要准备数据 - } - - @Override - protected void cleanData() { - // 不需要清里数据 - } - - @Test - public void singleChannelNumberTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withTestCaseName("singleChannelNumberTest") - .runJob(); - - Assert.assertEquals(50, readerResult.getResultData().size()); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } - - @Test - public void multiChannelNumberTest() { - try { - ReaderResult readerResult = (ReaderResult) new ReaderUnitTestLauncher() - .withPluginName(PLUGIN_NAME) - .withTestCaseName("multiChannelNumberTest") - .withChannel(2) - .runJob(); - - Assert.assertEquals(80, readerResult.getResultData().size()); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/writer/StreamWriterTest.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/writer/StreamWriterTest.java deleted file mode 100644 index a402f5af68..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/stream/writer/StreamWriterTest.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.stream.writer; - -import com.dtstack.flinkx.test.annotation.Plugin; -import com.dtstack.flinkx.test.annotation.PluginType; -import com.dtstack.flinkx.test.annotation.TestCase; -import com.dtstack.flinkx.test.core.BaseTest; -import com.dtstack.flinkx.test.core.WriterUnitTestLauncher; -import com.dtstack.flinkx.test.core.result.WriterResult; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.flink.types.Row; -import org.junit.Assert; -import org.testng.annotations.Test; - -/** - * @author jiangbo - * @date 2020/2/20 - */ -@TestCase(plugin = Plugin.STREAM, type = PluginType.WRITER) -public class StreamWriterTest extends BaseTest { - - public static final String PLUGIN_NAME = "stream"; - - @Override - protected String getDataSourceName() { - return PLUGIN_NAME; - } - - @Override - protected boolean isDataSourceValid() { - return true; - } - - @Override - protected void prepareDataInternal() throws Exception { - - } - - @Override - protected void cleanData() throws Exception { - - } - - @Test - public void singleChannelNumberTest() { - try { - Row[] records = new Row[20]; - for (int i = 0; i < 20; i++) { - Row record = new Row(3); - record.setField(0, "col1"); - record.setField(1, "col2"); - record.setField(2, "col3"); - - records[i] = record; - } - - WriterResult result = (WriterResult) new WriterUnitTestLauncher() - .withRecords(records) - .withPluginName(PLUGIN_NAME) - .withTestCaseName("singleChannelNumberTest") - .runJob(); - - long numberWriter = result.getExecutionResult().getAccumulatorResult("numWrite"); - Assert.assertEquals(20, numberWriter); - } catch (Exception e) { - Assert.fail(ExceptionUtil.getErrorMessage(e)); - } - } -} diff --git a/flinkx-test/src/test/java/com/dtstack/flinkx/test/util/TestCaseDocImporter.java b/flinkx-test/src/test/java/com/dtstack/flinkx/test/util/TestCaseDocImporter.java deleted file mode 100644 index 77ed60b8e4..0000000000 --- a/flinkx-test/src/test/java/com/dtstack/flinkx/test/util/TestCaseDocImporter.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.test.util; - -/** - * @author jiangbo - * @date 2020/3/6 - */ -public class TestCaseDocImporter{ - - public static void main(String[] args) { - - } -} diff --git a/flinkx-test/src/test/java/org/apache/hadoop/io/nativeio/NativeIO.java b/flinkx-test/src/test/java/org/apache/hadoop/io/nativeio/NativeIO.java deleted file mode 100644 index d38b2d7009..0000000000 --- a/flinkx-test/src/test/java/org/apache/hadoop/io/nativeio/NativeIO.java +++ /dev/null @@ -1,967 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.io.nativeio; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.HardLink; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException; -import org.apache.hadoop.util.NativeCodeLoader; -import org.apache.hadoop.util.PerformanceAdvisory; -import org.apache.hadoop.util.Shell; -import sun.misc.Unsafe; - -import java.io.*; -import java.lang.reflect.Field; -import java.nio.ByteBuffer; -import java.nio.MappedByteBuffer; -import java.nio.channels.FileChannel; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * JNI wrappers for various native IO-related calls not available in Java. - * These functions should generally be used alongside a fallback to another - * more portable mechanism. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class NativeIO { - public static class POSIX { - // Flags for open() call from bits/fcntl.h - public static final int O_RDONLY = 00; - public static final int O_WRONLY = 01; - public static final int O_RDWR = 02; - public static final int O_CREAT = 0100; - public static final int O_EXCL = 0200; - public static final int O_NOCTTY = 0400; - public static final int O_TRUNC = 01000; - public static final int O_APPEND = 02000; - public static final int O_NONBLOCK = 04000; - public static final int O_SYNC = 010000; - public static final int O_ASYNC = 020000; - public static final int O_FSYNC = O_SYNC; - public static final int O_NDELAY = O_NONBLOCK; - - // Flags for posix_fadvise() from bits/fcntl.h - /* No further special treatment. */ - public static final int POSIX_FADV_NORMAL = 0; - /* Expect random page references. */ - public static final int POSIX_FADV_RANDOM = 1; - /* Expect sequential page references. */ - public static final int POSIX_FADV_SEQUENTIAL = 2; - /* Will need these pages. */ - public static final int POSIX_FADV_WILLNEED = 3; - /* Don't need these pages. */ - public static final int POSIX_FADV_DONTNEED = 4; - /* Data will be accessed once. */ - public static final int POSIX_FADV_NOREUSE = 5; - - - /* Wait upon writeout of all pages - in the range before performing the - write. */ - public static final int SYNC_FILE_RANGE_WAIT_BEFORE = 1; - /* Initiate writeout of all those - dirty pages in the range which are - not presently under writeback. */ - public static final int SYNC_FILE_RANGE_WRITE = 2; - - /* Wait upon writeout of all pages in - the range after performing the - write. */ - public static final int SYNC_FILE_RANGE_WAIT_AFTER = 4; - - private static final Log LOG = LogFactory.getLog(org.apache.hadoop.io.nativeio.NativeIO.class); - - private static boolean nativeLoaded = false; - private static boolean fadvisePossible = true; - private static boolean syncFileRangePossible = true; - - static final String WORKAROUND_NON_THREADSAFE_CALLS_KEY = - "hadoop.workaround.non.threadsafe.getpwuid"; - static final boolean WORKAROUND_NON_THREADSAFE_CALLS_DEFAULT = true; - - private static long cacheTimeout = -1; - - private static CacheManipulator cacheManipulator = new CacheManipulator(); - - public static CacheManipulator getCacheManipulator() { - return cacheManipulator; - } - - public static void setCacheManipulator(CacheManipulator cacheManipulator) { - POSIX.cacheManipulator = cacheManipulator; - } - - /** - * Used to manipulate the operating system cache. - */ - @VisibleForTesting - public static class CacheManipulator { - public void mlock(String identifier, ByteBuffer buffer, - long len) throws IOException { - POSIX.mlock(buffer, len); - } - - public long getMemlockLimit() { - return org.apache.hadoop.io.nativeio.NativeIO.getMemlockLimit(); - } - - public long getOperatingSystemPageSize() { - return org.apache.hadoop.io.nativeio.NativeIO.getOperatingSystemPageSize(); - } - - public void posixFadviseIfPossible(String identifier, - FileDescriptor fd, long offset, long len, int flags) - throws NativeIOException { - POSIX.posixFadviseIfPossible(identifier, fd, offset, - len, flags); - } - - public boolean verifyCanMlock() { - return org.apache.hadoop.io.nativeio.NativeIO.isAvailable(); - } - } - - /** - * A CacheManipulator used for testing which does not actually call mlock. - * This allows many tests to be run even when the operating system does not - * allow mlock, or only allows limited mlocking. - */ - @VisibleForTesting - public static class NoMlockCacheManipulator extends CacheManipulator { - public void mlock(String identifier, ByteBuffer buffer, - long len) throws IOException { - LOG.info("mlocking " + identifier); - } - - public long getMemlockLimit() { - return 1125899906842624L; - } - - public long getOperatingSystemPageSize() { - return 4096; - } - - public boolean verifyCanMlock() { - return true; - } - } - - static { - if (NativeCodeLoader.isNativeCodeLoaded()) { - try { - Configuration conf = new Configuration(); - workaroundNonThreadSafePasswdCalls = conf.getBoolean( - WORKAROUND_NON_THREADSAFE_CALLS_KEY, - WORKAROUND_NON_THREADSAFE_CALLS_DEFAULT); - - initNative(); - nativeLoaded = true; - - cacheTimeout = conf.getLong( - CommonConfigurationKeys.HADOOP_SECURITY_UID_NAME_CACHE_TIMEOUT_KEY, - CommonConfigurationKeys.HADOOP_SECURITY_UID_NAME_CACHE_TIMEOUT_DEFAULT) * - 1000; - LOG.debug("Initialized cache for IDs to User/Group mapping with a " + - " cache timeout of " + cacheTimeout/1000 + " seconds."); - - } catch (Throwable t) { - // This can happen if the user has an older version of libhadoop.so - // installed - in this case we can continue without native IO - // after warning - PerformanceAdvisory.LOG.debug("Unable to initialize NativeIO libraries", t); - } - } - } - - /** - * Return true if the JNI-based native IO extensions are available. - */ - public static boolean isAvailable() { - return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded; - } - - private static void assertCodeLoaded() throws IOException { - if (!isAvailable()) { - throw new IOException("NativeIO was not loaded"); - } - } - - /** Wrapper around open(2) */ - public static native FileDescriptor open(String path, int flags, int mode) throws IOException; - /** Wrapper around fstat(2) */ - private static native Stat fstat(FileDescriptor fd) throws IOException; - - /** Native chmod implementation. On UNIX, it is a wrapper around chmod(2) */ - private static native void chmodImpl(String path, int mode) throws IOException; - - public static void chmod(String path, int mode) throws IOException { - if (!Shell.WINDOWS) { - chmodImpl(path, mode); - } else { - try { - chmodImpl(path, mode); - } catch (NativeIOException nioe) { - if (nioe.getErrorCode() == 3) { - throw new NativeIOException("No such file or directory", - Errno.ENOENT); - } else { - LOG.warn(String.format("NativeIO.chmod error (%d): %s", - nioe.getErrorCode(), nioe.getMessage())); - throw new NativeIOException("Unknown error", Errno.UNKNOWN); - } - } - } - } - - /** Wrapper around posix_fadvise(2) */ - static native void posix_fadvise( - FileDescriptor fd, long offset, long len, int flags) throws NativeIOException; - - /** Wrapper around sync_file_range(2) */ - static native void sync_file_range( - FileDescriptor fd, long offset, long nbytes, int flags) throws NativeIOException; - - /** - * Call posix_fadvise on the given file descriptor. See the manpage - * for this syscall for more information. On systems where this - * call is not available, does nothing. - * - * @throws NativeIOException if there is an error with the syscall - */ - static void posixFadviseIfPossible(String identifier, - FileDescriptor fd, long offset, long len, int flags) - throws NativeIOException { - if (nativeLoaded && fadvisePossible) { - try { - posix_fadvise(fd, offset, len, flags); - } catch (UnsupportedOperationException uoe) { - fadvisePossible = false; - } catch (UnsatisfiedLinkError ule) { - fadvisePossible = false; - } - } - } - - /** - * Call sync_file_range on the given file descriptor. See the manpage - * for this syscall for more information. On systems where this - * call is not available, does nothing. - * - * @throws NativeIOException if there is an error with the syscall - */ - public static void syncFileRangeIfPossible( - FileDescriptor fd, long offset, long nbytes, int flags) - throws NativeIOException { - if (nativeLoaded && syncFileRangePossible) { - try { - sync_file_range(fd, offset, nbytes, flags); - } catch (UnsupportedOperationException uoe) { - syncFileRangePossible = false; - } catch (UnsatisfiedLinkError ule) { - syncFileRangePossible = false; - } - } - } - - static native void mlock_native( - ByteBuffer buffer, long len) throws NativeIOException; - - /** - * Locks the provided direct ByteBuffer into memory, preventing it from - * swapping out. After a buffer is locked, future accesses will not incur - * a page fault. - * - * See the mlock(2) man page for more information. - * - * @throws NativeIOException - */ - static void mlock(ByteBuffer buffer, long len) - throws IOException { - assertCodeLoaded(); - if (!buffer.isDirect()) { - throw new IOException("Cannot mlock a non-direct ByteBuffer"); - } - mlock_native(buffer, len); - } - - /** - * Unmaps the block from memory. See munmap(2). - * - * There isn't any portable way to unmap a memory region in Java. - * So we use the sun.nio method here. - * Note that unmapping a memory region could cause crashes if code - * continues to reference the unmapped code. However, if we don't - * manually unmap the memory, we are dependent on the finalizer to - * do it, and we have no idea when the finalizer will run. - * - * @param buffer The buffer to unmap. - */ - public static void munmap(MappedByteBuffer buffer) { - if (buffer instanceof sun.nio.ch.DirectBuffer) { - sun.misc.Cleaner cleaner = - ((sun.nio.ch.DirectBuffer)buffer).cleaner(); - cleaner.clean(); - } - } - - /** Linux only methods used for getOwner() implementation */ - private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException; - private static native String getUserName(long uid) throws IOException; - - /** - * Result type of the fstat call - */ - public static class Stat { - private int ownerId, groupId; - private String owner, group; - private int mode; - - // Mode constants - public static final int S_IFMT = 0170000; /* type of file */ - public static final int S_IFIFO = 0010000; /* named pipe (fifo) */ - public static final int S_IFCHR = 0020000; /* character special */ - public static final int S_IFDIR = 0040000; /* directory */ - public static final int S_IFBLK = 0060000; /* block special */ - public static final int S_IFREG = 0100000; /* regular */ - public static final int S_IFLNK = 0120000; /* symbolic link */ - public static final int S_IFSOCK = 0140000; /* socket */ - public static final int S_IFWHT = 0160000; /* whiteout */ - public static final int S_ISUID = 0004000; /* set user id on execution */ - public static final int S_ISGID = 0002000; /* set group id on execution */ - public static final int S_ISVTX = 0001000; /* save swapped text even after use */ - public static final int S_IRUSR = 0000400; /* read permission, owner */ - public static final int S_IWUSR = 0000200; /* write permission, owner */ - public static final int S_IXUSR = 0000100; /* execute/search permission, owner */ - - Stat(int ownerId, int groupId, int mode) { - this.ownerId = ownerId; - this.groupId = groupId; - this.mode = mode; - } - - Stat(String owner, String group, int mode) { - if (!Shell.WINDOWS) { - this.owner = owner; - } else { - this.owner = stripDomain(owner); - } - if (!Shell.WINDOWS) { - this.group = group; - } else { - this.group = stripDomain(group); - } - this.mode = mode; - } - - @Override - public String toString() { - return "Stat(owner='" + owner + "', group='" + group + "'" + - ", mode=" + mode + ")"; - } - - public String getOwner() { - return owner; - } - public String getGroup() { - return group; - } - public int getMode() { - return mode; - } - } - - /** - * Returns the file stat for a file descriptor. - * - * @param fd file descriptor. - * @return the file descriptor file stat. - * @throws IOException thrown if there was an IO error while obtaining the file stat. - */ - public static Stat getFstat(FileDescriptor fd) throws IOException { - Stat stat = null; - if (!Shell.WINDOWS) { - stat = fstat(fd); - stat.owner = getName(IdCache.USER, stat.ownerId); - stat.group = getName(IdCache.GROUP, stat.groupId); - } else { - try { - stat = fstat(fd); - } catch (NativeIOException nioe) { - if (nioe.getErrorCode() == 6) { - throw new NativeIOException("The handle is invalid.", - Errno.EBADF); - } else { - LOG.warn(String.format("NativeIO.getFstat error (%d): %s", - nioe.getErrorCode(), nioe.getMessage())); - throw new NativeIOException("Unknown error", Errno.UNKNOWN); - } - } - } - return stat; - } - - private static String getName(IdCache domain, int id) throws IOException { - Map idNameCache = (domain == IdCache.USER) - ? USER_ID_NAME_CACHE : GROUP_ID_NAME_CACHE; - String name; - CachedName cachedName = idNameCache.get(id); - long now = System.currentTimeMillis(); - if (cachedName != null && (cachedName.timestamp + cacheTimeout) > now) { - name = cachedName.name; - } else { - name = (domain == IdCache.USER) ? getUserName(id) : getGroupName(id); - if (LOG.isDebugEnabled()) { - String type = (domain == IdCache.USER) ? "UserName" : "GroupName"; - LOG.debug("Got " + type + " " + name + " for ID " + id + - " from the native implementation"); - } - cachedName = new CachedName(name, now); - idNameCache.put(id, cachedName); - } - return name; - } - - static native String getUserName(int uid) throws IOException; - static native String getGroupName(int uid) throws IOException; - - private static class CachedName { - final long timestamp; - final String name; - - public CachedName(String name, long timestamp) { - this.name = name; - this.timestamp = timestamp; - } - } - - private static final Map USER_ID_NAME_CACHE = - new ConcurrentHashMap(); - - private static final Map GROUP_ID_NAME_CACHE = - new ConcurrentHashMap(); - - private enum IdCache { USER, GROUP } - - public final static int MMAP_PROT_READ = 0x1; - public final static int MMAP_PROT_WRITE = 0x2; - public final static int MMAP_PROT_EXEC = 0x4; - - public static native long mmap(FileDescriptor fd, int prot, - boolean shared, long length) throws IOException; - - public static native void munmap(long addr, long length) - throws IOException; - } - - private static boolean workaroundNonThreadSafePasswdCalls = false; - - - public static class Windows { - // Flags for CreateFile() call on Windows - public static final long GENERIC_READ = 0x80000000L; - public static final long GENERIC_WRITE = 0x40000000L; - - public static final long FILE_SHARE_READ = 0x00000001L; - public static final long FILE_SHARE_WRITE = 0x00000002L; - public static final long FILE_SHARE_DELETE = 0x00000004L; - - public static final long CREATE_NEW = 1; - public static final long CREATE_ALWAYS = 2; - public static final long OPEN_EXISTING = 3; - public static final long OPEN_ALWAYS = 4; - public static final long TRUNCATE_EXISTING = 5; - - public static final long FILE_BEGIN = 0; - public static final long FILE_CURRENT = 1; - public static final long FILE_END = 2; - - public static final long FILE_ATTRIBUTE_NORMAL = 0x00000080L; - - /** - * Create a directory with permissions set to the specified mode. By setting - * permissions at creation time, we avoid issues related to the user lacking - * WRITE_DAC rights on subsequent chmod calls. One example where this can - * occur is writing to an SMB share where the user does not have Full Control - * rights, and therefore WRITE_DAC is denied. - * - * @param path directory to create - * @param mode permissions of new directory - * @throws IOException if there is an I/O error - */ - public static void createDirectoryWithMode(File path, int mode) - throws IOException { - createDirectoryWithMode0(path.getAbsolutePath(), mode); - } - - /** Wrapper around CreateDirectory() on Windows */ - private static native void createDirectoryWithMode0(String path, int mode) - throws NativeIOException; - - /** Wrapper around CreateFile() on Windows */ - public static native FileDescriptor createFile(String path, - long desiredAccess, long shareMode, long creationDisposition) - throws IOException; - - /** - * Create a file for write with permissions set to the specified mode. By - * setting permissions at creation time, we avoid issues related to the user - * lacking WRITE_DAC rights on subsequent chmod calls. One example where - * this can occur is writing to an SMB share where the user does not have - * Full Control rights, and therefore WRITE_DAC is denied. - * - * This method mimics the semantics implemented by the JDK in - * {@link FileOutputStream}. The file is opened for truncate or - * append, the sharing mode allows other readers and writers, and paths - * longer than MAX_PATH are supported. (See io_util_md.c in the JDK.) - * - * @param path file to create - * @param append if true, then open file for append - * @param mode permissions of new directory - * @return FileOutputStream of opened file - * @throws IOException if there is an I/O error - */ - public static FileOutputStream createFileOutputStreamWithMode(File path, - boolean append, int mode) throws IOException { - long desiredAccess = GENERIC_WRITE; - long shareMode = FILE_SHARE_READ | FILE_SHARE_WRITE; - long creationDisposition = append ? OPEN_ALWAYS : CREATE_ALWAYS; - return new FileOutputStream(createFileWithMode0(path.getAbsolutePath(), - desiredAccess, shareMode, creationDisposition, mode)); - } - - /** Wrapper around CreateFile() with security descriptor on Windows */ - private static native FileDescriptor createFileWithMode0(String path, - long desiredAccess, long shareMode, long creationDisposition, int mode) - throws NativeIOException; - - /** Wrapper around SetFilePointer() on Windows */ - public static native long setFilePointer(FileDescriptor fd, - long distanceToMove, long moveMethod) throws IOException; - - /** Windows only methods used for getOwner() implementation */ - private static native String getOwner(FileDescriptor fd) throws IOException; - - /** Supported list of Windows access right flags */ - public static enum AccessRight { - ACCESS_READ (0x0001), // FILE_READ_DATA - ACCESS_WRITE (0x0002), // FILE_WRITE_DATA - ACCESS_EXECUTE (0x0020); // FILE_EXECUTE - - private final int accessRight; - AccessRight(int access) { - accessRight = access; - } - - public int accessRight() { - return accessRight; - } - }; - - /** Windows only method used to check if the current process has requested - * access rights on the given path. */ - private static native boolean access0(String path, int requestedAccess); - - /** - * Checks whether the current process has desired access rights on - * the given path. - * - * Longer term this native function can be substituted with JDK7 - * function Files#isReadable, isWritable, isExecutable. - * - * @param path input path - * @param desiredAccess ACCESS_READ, ACCESS_WRITE or ACCESS_EXECUTE - * @return true if access is allowed - * @throws IOException I/O exception on error - */ - public static boolean access(String path, AccessRight desiredAccess) - throws IOException { - return true; -// return access0(path, desiredAccess.accessRight()); - } - - /** - * Extends both the minimum and maximum working set size of the current - * process. This method gets the current minimum and maximum working set - * size, adds the requested amount to each and then sets the minimum and - * maximum working set size to the new values. Controlling the working set - * size of the process also controls the amount of memory it can lock. - * - * @param delta amount to increment minimum and maximum working set size - * @throws IOException for any error - * @see POSIX#mlock(ByteBuffer, long) - */ - public static native void extendWorkingSetSize(long delta) throws IOException; - - static { - if (NativeCodeLoader.isNativeCodeLoaded()) { - try { - initNative(); - nativeLoaded = true; - } catch (Throwable t) { - // This can happen if the user has an older version of libhadoop.so - // installed - in this case we can continue without native IO - // after warning - PerformanceAdvisory.LOG.debug("Unable to initialize NativeIO libraries", t); - } - } - } - } - - private static final Log LOG = LogFactory.getLog(org.apache.hadoop.io.nativeio.NativeIO.class); - - private static boolean nativeLoaded = false; - - static { - if (NativeCodeLoader.isNativeCodeLoaded()) { - try { - initNative(); - nativeLoaded = true; - } catch (Throwable t) { - // This can happen if the user has an older version of libhadoop.so - // installed - in this case we can continue without native IO - // after warning - PerformanceAdvisory.LOG.debug("Unable to initialize NativeIO libraries", t); - } - } - } - - /** - * Return true if the JNI-based native IO extensions are available. - */ - public static boolean isAvailable() { - return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded; - } - - /** Initialize the JNI method ID and class ID cache */ - private static native void initNative(); - - /** - * Get the maximum number of bytes that can be locked into memory at any - * given point. - * - * @return 0 if no bytes can be locked into memory; - * Long.MAX_VALUE if there is no limit; - * The number of bytes that can be locked into memory otherwise. - */ - static long getMemlockLimit() { - return isAvailable() ? getMemlockLimit0() : 0; - } - - private static native long getMemlockLimit0(); - - /** - * @return the operating system's page size. - */ - static long getOperatingSystemPageSize() { - try { - Field f = Unsafe.class.getDeclaredField("theUnsafe"); - f.setAccessible(true); - Unsafe unsafe = (Unsafe)f.get(null); - return unsafe.pageSize(); - } catch (Throwable e) { - LOG.warn("Unable to get operating system page size. Guessing 4096.", e); - return 4096; - } - } - - private static class CachedUid { - final long timestamp; - final String username; - public CachedUid(String username, long timestamp) { - this.timestamp = timestamp; - this.username = username; - } - } - private static final Map uidCache = - new ConcurrentHashMap(); - private static long cacheTimeout; - private static boolean initialized = false; - - /** - * The Windows logon name has two part, NetBIOS domain name and - * user account name, of the format DOMAIN\UserName. This method - * will remove the domain part of the full logon name. - * - * @param Fthe full principal name containing the domain - * @return name with domain removed - */ - private static String stripDomain(String name) { - int i = name.indexOf('\\'); - if (i != -1) - name = name.substring(i + 1); - return name; - } - - public static String getOwner(FileDescriptor fd) throws IOException { - ensureInitialized(); - if (Shell.WINDOWS) { - String owner = Windows.getOwner(fd); - owner = stripDomain(owner); - return owner; - } else { - long uid = POSIX.getUIDforFDOwnerforOwner(fd); - CachedUid cUid = uidCache.get(uid); - long now = System.currentTimeMillis(); - if (cUid != null && (cUid.timestamp + cacheTimeout) > now) { - return cUid.username; - } - String user = POSIX.getUserName(uid); - LOG.info("Got UserName " + user + " for UID " + uid - + " from the native implementation"); - cUid = new CachedUid(user, now); - uidCache.put(uid, cUid); - return user; - } - } - - /** - * Create a FileInputStream that shares delete permission on the - * file opened, i.e. other process can delete the file the - * FileInputStream is reading. Only Windows implementation uses - * the native interface. - */ - public static FileInputStream getShareDeleteFileInputStream(File f) - throws IOException { - if (!Shell.WINDOWS) { - // On Linux the default FileInputStream shares delete permission - // on the file opened. - // - return new FileInputStream(f); - } else { - // Use Windows native interface to create a FileInputStream that - // shares delete permission on the file opened. - // - FileDescriptor fd = Windows.createFile( - f.getAbsolutePath(), - Windows.GENERIC_READ, - Windows.FILE_SHARE_READ | - Windows.FILE_SHARE_WRITE | - Windows.FILE_SHARE_DELETE, - Windows.OPEN_EXISTING); - return new FileInputStream(fd); - } - } - - /** - * Create a FileInputStream that shares delete permission on the - * file opened at a given offset, i.e. other process can delete - * the file the FileInputStream is reading. Only Windows implementation - * uses the native interface. - */ - public static FileInputStream getShareDeleteFileInputStream(File f, long seekOffset) - throws IOException { - if (!Shell.WINDOWS) { - RandomAccessFile rf = new RandomAccessFile(f, "r"); - if (seekOffset > 0) { - rf.seek(seekOffset); - } - return new FileInputStream(rf.getFD()); - } else { - // Use Windows native interface to create a FileInputStream that - // shares delete permission on the file opened, and set it to the - // given offset. - // - FileDescriptor fd = Windows.createFile( - f.getAbsolutePath(), - Windows.GENERIC_READ, - Windows.FILE_SHARE_READ | - Windows.FILE_SHARE_WRITE | - Windows.FILE_SHARE_DELETE, - Windows.OPEN_EXISTING); - if (seekOffset > 0) - Windows.setFilePointer(fd, seekOffset, Windows.FILE_BEGIN); - return new FileInputStream(fd); - } - } - - /** - * Create the specified File for write access, ensuring that it does not exist. - * @param f the file that we want to create - * @param permissions we want to have on the file (if security is enabled) - * - * @throws AlreadyExistsException if the file already exists - * @throws IOException if any other error occurred - */ - public static FileOutputStream getCreateForWriteFileOutputStream(File f, int permissions) - throws IOException { - if (!Shell.WINDOWS) { - // Use the native wrapper around open(2) - try { - FileDescriptor fd = POSIX.open(f.getAbsolutePath(), - POSIX.O_WRONLY | POSIX.O_CREAT - | POSIX.O_EXCL, permissions); - return new FileOutputStream(fd); - } catch (NativeIOException nioe) { - if (nioe.getErrno() == Errno.EEXIST) { - throw new AlreadyExistsException(nioe); - } - throw nioe; - } - } else { - // Use the Windows native APIs to create equivalent FileOutputStream - try { - FileDescriptor fd = Windows.createFile(f.getCanonicalPath(), - Windows.GENERIC_WRITE, - Windows.FILE_SHARE_DELETE - | Windows.FILE_SHARE_READ - | Windows.FILE_SHARE_WRITE, - Windows.CREATE_NEW); - POSIX.chmod(f.getCanonicalPath(), permissions); - return new FileOutputStream(fd); - } catch (NativeIOException nioe) { - if (nioe.getErrorCode() == 80) { - // ERROR_FILE_EXISTS - // 80 (0x50) - // The file exists - throw new AlreadyExistsException(nioe); - } - throw nioe; - } - } - } - - private synchronized static void ensureInitialized() { - if (!initialized) { - cacheTimeout = - new Configuration().getLong("hadoop.security.uid.cache.secs", - 4*60*60) * 1000; - LOG.info("Initialized cache for UID to User mapping with a cache" + - " timeout of " + cacheTimeout/1000 + " seconds."); - initialized = true; - } - } - - /** - * A version of renameTo that throws a descriptive exception when it fails. - * - * @param src The source path - * @param dst The destination path - * - * @throws NativeIOException On failure. - */ - public static void renameTo(File src, File dst) - throws IOException { - if (!nativeLoaded) { - if (!src.renameTo(dst)) { - throw new IOException("renameTo(src=" + src + ", dst=" + - dst + ") failed."); - } - } else { - renameTo0(src.getAbsolutePath(), dst.getAbsolutePath()); - } - } - - public static void link(File src, File dst) throws IOException { - if (!nativeLoaded) { - HardLink.createHardLink(src, dst); - } else { - link0(src.getAbsolutePath(), dst.getAbsolutePath()); - } - } - - /** - * A version of renameTo that throws a descriptive exception when it fails. - * - * @param src The source path - * @param dst The destination path - * - * @throws NativeIOException On failure. - */ - private static native void renameTo0(String src, String dst) - throws NativeIOException; - - private static native void link0(String src, String dst) - throws NativeIOException; - - /** - * Unbuffered file copy from src to dst without tainting OS buffer cache - * - * In POSIX platform: - * It uses FileChannel#transferTo() which internally attempts - * unbuffered IO on OS with native sendfile64() support and falls back to - * buffered IO otherwise. - * - * It minimizes the number of FileChannel#transferTo call by passing the the - * src file size directly instead of a smaller size as the 3rd parameter. - * This saves the number of sendfile64() system call when native sendfile64() - * is supported. In the two fall back cases where sendfile is not supported, - * FileChannle#transferTo already has its own batching of size 8 MB and 8 KB, - * respectively. - * - * In Windows Platform: - * It uses its own native wrapper of CopyFileEx with COPY_FILE_NO_BUFFERING - * flag, which is supported on Windows Server 2008 and above. - * - * Ideally, we should use FileChannel#transferTo() across both POSIX and Windows - * platform. Unfortunately, the wrapper(Java_sun_nio_ch_FileChannelImpl_transferTo0) - * used by FileChannel#transferTo for unbuffered IO is not implemented on Windows. - * Based on OpenJDK 6/7/8 source code, Java_sun_nio_ch_FileChannelImpl_transferTo0 - * on Windows simply returns IOS_UNSUPPORTED. - * - * Note: This simple native wrapper does minimal parameter checking before copy and - * consistency check (e.g., size) after copy. - * It is recommended to use wrapper function like - * the Storage#nativeCopyFileUnbuffered() function in hadoop-hdfs with pre/post copy - * checks. - * - * @param src The source path - * @param dst The destination path - * @throws IOException - */ - public static void copyFileUnbuffered(File src, File dst) throws IOException { - if (nativeLoaded && Shell.WINDOWS) { - copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath()); - } else { - FileInputStream fis = null; - FileOutputStream fos = null; - FileChannel input = null; - FileChannel output = null; - try { - fis = new FileInputStream(src); - fos = new FileOutputStream(dst); - input = fis.getChannel(); - output = fos.getChannel(); - long remaining = input.size(); - long position = 0; - long transferred = 0; - while (remaining > 0) { - transferred = input.transferTo(position, remaining, output); - remaining -= transferred; - position += transferred; - } - } finally { - IOUtils.cleanup(LOG, output); - IOUtils.cleanup(LOG, fos); - IOUtils.cleanup(LOG, input); - IOUtils.cleanup(LOG, fis); - } - } - } - - private static native void copyFileUnbuffered0(String src, String dst) - throws NativeIOException; -} diff --git a/flinkx-test/src/test/resources/testCase/dataSource/hdfs.json b/flinkx-test/src/test/resources/testCase/dataSource/hdfs.json deleted file mode 100644 index 18cbfe7046..0000000000 --- a/flinkx-test/src/test/resources/testCase/dataSource/hdfs.json +++ /dev/null @@ -1,14 +0,0 @@ -{ - "actionBeforeTest":{ - "reader": [ - ], - "writer": [ - ] - }, - "actionAfterTest" : { - "reader": [ - ], - "writer": [ - ] - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/dataSource/mysql.json b/flinkx-test/src/test/resources/testCase/dataSource/mysql.json deleted file mode 100644 index e04502499a..0000000000 --- a/flinkx-test/src/test/resources/testCase/dataSource/mysql.json +++ /dev/null @@ -1,22 +0,0 @@ -{ - "actionBeforeTest":{ - "reader": [ - "drop table if exists `number_test`", - "CREATE TABLE `number_test` (\n `bigint_col` bigint(20) DEFAULT NULL,\n `bigint_unsigned_col` bigint(20) unsigned DEFAULT NULL,\n `binary_col` binary(1) DEFAULT NULL,\n `bit_col` bit(1) DEFAULT NULL,\n `blob_col` blob,\n `bool_col` tinyint(1) DEFAULT NULL,\n `char_col` char(10) DEFAULT NULL,\n `date_col` date DEFAULT NULL,\n `datetime_col` datetime DEFAULT NULL,\n `decimal_col` decimal(10,2) DEFAULT NULL,\n `double_col` double DEFAULT NULL,\n `double_precision_col` double DEFAULT NULL,\n `float_col` float DEFAULT NULL,\n `int_unsigned_col` int(10) unsigned DEFAULT NULL,\n `integer_col` int(11) DEFAULT NULL,\n `integer_unsigned_col` int(10) unsigned DEFAULT NULL,\n `long_varbinary_col` mediumblob,\n `long_varchar_col` mediumtext,\n `long_blob_col` longblob,\n `long_text_col` longtext,\n `medium_blob_col` mediumblob,\n `medium_int_col` mediumint(9) DEFAULT NULL,\n `medium_int_unsigned_col` mediumint(8) unsigned DEFAULT NULL,\n `medium_text_col` mediumtext,\n `numberic_col` decimal(10,0) DEFAULT NULL,\n `real_col` double DEFAULT NULL,\n `small_int_col` smallint(6) DEFAULT NULL,\n `small_int_unsigned_col` smallint(5) unsigned DEFAULT NULL,\n `text_col` text,\n `time_col` time DEFAULT NULL,\n `timestamp_col` timestamp NULL DEFAULT NULL,\n `tiny_blob_col` tinyblob,\n `tiny_int_col` tinyint(4) DEFAULT NULL,\n `tiny_int_unsigned_col` tinyint(3) unsigned DEFAULT NULL,\n `tiny_text_col` tinytext,\n `year_col` year(4) DEFAULT NULL,\n `json_col` json DEFAULT NULL,\n `varchar_col` varchar(10) DEFAULT NULL,\n `varbinary_col` varbinary(10) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8;\n", - "INSERT INTO `number_test` (bigint_col,bigint_unsigned_col,binary_col,bit_col,blob_col,bool_col,char_col,date_col,datetime_col,decimal_col,double_col,double_precision_col,float_col,int_unsigned_col,integer_col,integer_unsigned_col,long_varbinary_col,long_varchar_col,long_blob_col,long_text_col,medium_blob_col,medium_int_col,medium_int_unsigned_col,medium_text_col,numberic_col,real_col,small_int_col,small_int_unsigned_col,text_col,time_col,timestamp_col,tiny_blob_col,tiny_int_col,tiny_int_unsigned_col,tiny_text_col,year_col,json_col,varchar_col,varbinary_col) VALUES \n(149,1,NULL,0,NULL,1,'xsxsa',NULL,NULL,12.35,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL)\n,(129,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(197,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(199,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(2,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(13,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(59,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(57,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(107,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(162,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(87,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(151,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(91,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(4,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(148,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(125,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(180,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(123,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(77,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL)\n,(14,1,NULL,0,NULL,1,'xsxsa','2020-02-21','2020-02-22 07:45:55.000',12.35,234.4554,1232.3432,123.342,1,1,1,NULL,'sadsadsadxsas',NULL,'dedchebsd',NULL,1,1,'edsadcdcds',1231,1,1,1,'dxasxsa','07:45:44','2020-02-22 07:46:12.000',NULL,1,1,'sadsa',2020,'{\"key\": \"val\"}','asdwasd',NULL);", - "drop table if exists `filter_test`", - "CREATE TABLE `filter_test` (\n\tid INT NOT NULL AUTO_INCREMENT,\n\tname varchar(100) NULL,\n\tPRIMARY key (id)\n)\nENGINE=InnoDB\nDEFAULT CHARSET=utf8\nCOLLATE=utf8_general_ci;", - "INSERT INTO flinkx_unit.filter_test (name) VALUES \n('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')\n,('name1')" - ], - "writer": [ - ] - }, - "actionAfterTest" : { - "reader": [ - "drop table if exists `number_test`", - "drop table if exists `filter_test`" - ], - "writer": [ - ] - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/dataSource/source.json b/flinkx-test/src/test/resources/testCase/dataSource/source.json deleted file mode 100644 index 0649bcd959..0000000000 --- a/flinkx-test/src/test/resources/testCase/dataSource/source.json +++ /dev/null @@ -1,49 +0,0 @@ -{ - "mysql": { - "initType": "static", - "jdbcUrl": [ - "jdbc:mysql://172.16.8.109:3306/flinkx_unit" - ], - "username": "dtstack", - "password": "abc123" - }, - "oracle": { - "initType": "static", - "jdbcUrl": "jdbc:mysql://172.16.8.109:3306/flinkx_unit", - "username": "dtstack", - "password": "abc123" - }, - "postgreSql": { - "initType": "static", - "jdbcUrl": "jdbc:mysql://172.16.8.109:3306/flinkx_unit", - "username": "dtstack", - "password": "abc123" - }, - "sqlServer": { - "initType": "static", - "jdbcUrl": "jdbc:mysql://172.16.8.109:3306/flinkx_unit", - "username": "dtstack", - "password": "abc123" - }, - "db2": { - "initType": "static", - "jdbcUrl": "jdbc:mysql://172.16.8.109:3306/flinkx_unit", - "username": "dtstack", - "password": "abc123" - }, - "gbase": { - "initType": "static", - "jdbcUrl": "jdbc:mysql://172.16.8.109:3306/flinkx_unit", - "username": "dtstack", - "password": "abc123" - }, - "hdfs": { - "initType": "autoCreate" - }, - "hive": { - "initType": "autoCreate" - }, - "stream": { - "initType": "static" - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/dataSource/stream.json b/flinkx-test/src/test/resources/testCase/dataSource/stream.json deleted file mode 100644 index 18cbfe7046..0000000000 --- a/flinkx-test/src/test/resources/testCase/dataSource/stream.json +++ /dev/null @@ -1,14 +0,0 @@ -{ - "actionBeforeTest":{ - "reader": [ - ], - "writer": [ - ] - }, - "actionAfterTest" : { - "reader": [ - ], - "writer": [ - ] - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/hdfs/writer/numberTest.json b/flinkx-test/src/test/resources/testCase/hdfs/writer/numberTest.json deleted file mode 100644 index cf1ff17dda..0000000000 --- a/flinkx-test/src/test/resources/testCase/hdfs/writer/numberTest.json +++ /dev/null @@ -1,94 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - - }, - "name" : "streamreader" - }, - "writer" : { - "parameter" : { - "fileName" : "fileType=text", - "column" : [ { - "name" : "tinyint", - "index" : 0, - "type" : "tinyint" - }, { - "name" : "smallint", - "index" : 1, - "type" : "smallint" - }, { - "name" : "int", - "index" : 2, - "type" : "int" - }, { - "name" : "bigint", - "index" : 3, - "type" : "bigint" - }, { - "name" : "float", - "index" : 4, - "type" : "float" - }, { - "name" : "double", - "index" : 5, - "type" : "double" - }, { - "name" : "decimal", - "index" : 6, - "type" : "decimal(5,3)" - }, { - "name" : "string", - "index" : 7, - "type" : "string" - }, { - "name" : "varchar", - "index" : 8, - "type" : "varchar" - }, { - "name" : "char", - "index" : 9, - "type" : "char" - }, { - "name" : "boolean", - "index" : 10, - "type" : "boolean" - }, { - "name" : "date", - "index" : 11, - "type" : "date" - }, { - "name" : "timestamp", - "index" : 12, - "type" : "timestamp" - }], - "writeMode" : "overwrite", - "fieldDelimiter" : "\u0001", - "encoding" : "utf-8", - "path" : "/flinkx", - "defaultFS" : null, - "fileType" : "text", - "fullColumnName" : ["tinyint", "smallint", "int", "bigint", "float", "double", "decimal(5,3)", "string", "varchar", "char", "boolean", "date", "timestamp"], - "fullColumnType" : ["tinyint", "smallint", "int", "bigint", "float", "double", "decimal(5,3)", "string", "varchar", "char", "boolean", "date", "timestamp"] - }, - "name" : "hdfswriter" - } - } ], - "setting" : { - "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 - }, - "speed" : { - "bytes" : 0, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/customSqlTest.json b/flinkx-test/src/test/resources/testCase/mysql/reader/customSqlTest.json deleted file mode 100644 index 2d7ac98b2f..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/customSqlTest.json +++ /dev/null @@ -1,44 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "customSql": "select id,name from filter_test where id > 3", - "splitPk": "id", - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "table": [ - "filter_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/filterTest.json b/flinkx-test/src/test/resources/testCase/mysql/reader/filterTest.json deleted file mode 100644 index 0a29742aa9..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/filterTest.json +++ /dev/null @@ -1,44 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "splitPk": "id", - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "where": "id > 5", - "table": [ - "filter_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementUseMaxFunc.json b/flinkx-test/src/test/resources/testCase/mysql/reader/incrementUseMaxFunc.json deleted file mode 100644 index 7e6c0d546a..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementUseMaxFunc.json +++ /dev/null @@ -1,45 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "increColumn": "id", - "useMaxFunc": true, - "splitPk": "id", - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "table": [ - "filter_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithStartLocation.json b/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithStartLocation.json deleted file mode 100644 index c0a91e2c79..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithStartLocation.json +++ /dev/null @@ -1,45 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "increColumn": "id", - "startLocation": "10", - "splitPk": "id", - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "table": [ - "filter_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithoutStartLocation.json b/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithoutStartLocation.json deleted file mode 100644 index b07a594d8a..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/incrementWithoutStartLocation.json +++ /dev/null @@ -1,44 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "increColumn": "id", - "splitPk": "id", - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "table": [ - "filter_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/mysql/reader/numberTest.json b/flinkx-test/src/test/resources/testCase/mysql/reader/numberTest.json deleted file mode 100644 index 281049d50d..0000000000 --- a/flinkx-test/src/test/resources/testCase/mysql/reader/numberTest.json +++ /dev/null @@ -1,39 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "splitPk":"bigint_col", - "column": [ - { - "name": "bigint_col", - "type": "long" - } - ], - "table": [ - "number_test" - ] - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/stream/reader/multiChannelNumberTest.json b/flinkx-test/src/test/resources/testCase/stream/reader/multiChannelNumberTest.json deleted file mode 100644 index 7bc197d132..0000000000 --- a/flinkx-test/src/test/resources/testCase/stream/reader/multiChannelNumberTest.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "sliceRecordCount":[50,30] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/stream/reader/singleChannelNumberTest.json b/flinkx-test/src/test/resources/testCase/stream/reader/singleChannelNumberTest.json deleted file mode 100644 index 5696bd0246..0000000000 --- a/flinkx-test/src/test/resources/testCase/stream/reader/singleChannelNumberTest.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "sliceRecordCount":[50] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/test/resources/testCase/stream/writer/singleChannelNumberTest.json b/flinkx-test/src/test/resources/testCase/stream/writer/singleChannelNumberTest.json deleted file mode 100644 index 3e7569c3b8..0000000000 --- a/flinkx-test/src/test/resources/testCase/stream/writer/singleChannelNumberTest.json +++ /dev/null @@ -1,30 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "print":false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 0, - "percentage": 0 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 3d7b02fd14..0d36173434 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,6 @@ flinkx-db2 flinkx-gbase flinkx-clickhouse - flinkx-oraclelogminer flinkx-hdfs flinkx-hive @@ -49,7 +48,6 @@ flinkx-kafka10 flinkx-kafka11 flinkx-kafka - flinkx-sqlservercdc flinkx-emqx From b4c0d87bf1dca04d6cc5a8adf567593219a9312f Mon Sep 17 00:00:00 2001 From: jiangbo Date: Fri, 13 Mar 2020 21:09:39 +0800 Subject: [PATCH 018/136] merge --- .../FlinkxRemovePartitionTransformation.md | 86 ----- .../flinkx-cassandra-core/pom.xml | 22 ++ .../flinkx/cassandra/CassandraConfigKeys.java | 46 +++ .../flinkx/cassandra/CassandraUtil.java | 305 ++++++++++++++++++ .../flinkx/cassandra/TestCassandraUtil.java | 36 +++ .../flinkx-cassandra-reader/pom.xml | 96 ++++++ .../cassandra/reader/CassandraConstants.java | 12 + .../reader/CassandraInputFormat.java | 242 ++++++++++++++ .../reader/CassandraInputFormatBuilder.java | 68 ++++ .../cassandra/reader/CassandraInputSplit.java | 62 ++++ .../cassandra/reader/CassandraReader.java | 122 +++++++ .../flinkx-cassandra-writer/pom.xml | 95 ++++++ .../writer/CassandraOutputFormat.java | 158 +++++++++ .../writer/CassandraOutputFormatBuilder.java | 56 ++++ .../cassandra/writer/CassandraWriter.java | 106 ++++++ flinkx-cassandra/pom.xml | 28 ++ .../dtstack/flinkx/enums/EDatabaseType.java | 2 + flinkx-saphana/flinkx-saphana-core/pom.xml | 21 ++ .../flinkx/saphana/SaphanaDatabaseMeta.java | 131 ++++++++ flinkx-saphana/flinkx-saphana-reader/pom.xml | 102 ++++++ .../saphana/format/SaphanaInputFormat.java | 130 ++++++++ .../flinkx/saphana/reader/SaphanaReader.java | 49 +++ flinkx-saphana/flinkx-saphana-writer/pom.xml | 101 ++++++ .../saphana/format/SaphanaOutputFormat.java | 28 ++ .../flinkx/saphana/writer/SaphanaWriter.java | 48 +++ flinkx-saphana/pom.xml | 35 ++ flinkx-teradata/flinkx-teradata-core/pom.xml | 15 + .../flinkx/teradata/TeradataDatabaseMeta.java | 126 ++++++++ .../dtstack/flinkx/teradata/util/DBUtil.java | 83 +++++ .../flinkx-teradata-reader/pom.xml | 108 +++++++ .../teradata/format/TeradataInputFormat.java | 132 ++++++++ .../teradata/reader/TeradataReader.java | 46 +++ .../flinkx-teradata-writer/pom.xml | 108 +++++++ .../teradata/format/TeradataOutputFormat.java | 79 +++++ .../teradata/writer/TeradataWriter.java | 44 +++ flinkx-teradata/pom.xml | 36 +++ pom.xml | 3 + 37 files changed, 2881 insertions(+), 86 deletions(-) delete mode 100644 docs/FLXIP/FlinkxRemovePartitionTransformation.md create mode 100644 flinkx-cassandra/flinkx-cassandra-core/pom.xml create mode 100644 flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java create mode 100644 flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java create mode 100644 flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/pom.xml create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormatBuilder.java create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputSplit.java create mode 100644 flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraReader.java create mode 100644 flinkx-cassandra/flinkx-cassandra-writer/pom.xml create mode 100644 flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java create mode 100644 flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java create mode 100644 flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java create mode 100644 flinkx-cassandra/pom.xml create mode 100644 flinkx-saphana/flinkx-saphana-core/pom.xml create mode 100644 flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java create mode 100644 flinkx-saphana/flinkx-saphana-reader/pom.xml create mode 100644 flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java create mode 100644 flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/reader/SaphanaReader.java create mode 100644 flinkx-saphana/flinkx-saphana-writer/pom.xml create mode 100644 flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaOutputFormat.java create mode 100644 flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/writer/SaphanaWriter.java create mode 100644 flinkx-saphana/pom.xml create mode 100644 flinkx-teradata/flinkx-teradata-core/pom.xml create mode 100644 flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java create mode 100644 flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java create mode 100644 flinkx-teradata/flinkx-teradata-reader/pom.xml create mode 100644 flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java create mode 100644 flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/TeradataReader.java create mode 100644 flinkx-teradata/flinkx-teradata-writer/pom.xml create mode 100644 flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java create mode 100644 flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/writer/TeradataWriter.java create mode 100644 flinkx-teradata/pom.xml diff --git a/docs/FLXIP/FlinkxRemovePartitionTransformation.md b/docs/FLXIP/FlinkxRemovePartitionTransformation.md deleted file mode 100644 index 894fd67a7c..0000000000 --- a/docs/FLXIP/FlinkxRemovePartitionTransformation.md +++ /dev/null @@ -1,86 +0,0 @@ - -## Flinkx 关于移除 PartitionTransformation -1. Main程序在SourceTransformation与SinkTransformation之间加入了PartitionTransformation(Partitioner:DTRebalancePartitioner) - - ```java - ... - - DataReader dataReader = DataReaderFactory.getDataReader(config, env); - DataStream dataStream = dataReader.readData(); - - dataStream = new DataStream<>(dataStream.getExecutionEnvironment(), - new PartitionTransformation<>(dataStream.getTransformation(), - new DTRebalancePartitioner<>())); - - DataWriter dataWriter = DataWriterFactory.getDataWriter(config); - dataWriter.writeData(dataStream); - - ...//省略前后代码 - - ``` -2. 在构建StreamGraph时,SinkTransformation与SourceTransformation会形成2个StreamNode(每个StreamNode包含一个算子Operator)与1个StreamEdge,因为DTRebalancePartitioner的原因,StreamEdge的Partition属性不等于ForwardPartitioner -3. 在构建JobGraph是,Flink 通过 StreamingJobGraphGenerator#isChainable 判断 Operator 能否被chain到同一个 JobVertex 中 - - ```java - - public static boolean isChainable(StreamEdge edge, StreamGraph streamGraph) { - StreamNode upStreamVertex = streamGraph.getSourceVertex(edge); - StreamNode downStreamVertex = streamGraph.getTargetVertex(edge); - - StreamOperator headOperator = upStreamVertex.getOperator(); - StreamOperator outOperator = downStreamVertex.getOperator(); - - return downStreamVertex.getInEdges().size() == 1 - && outOperator != null - && headOperator != null - && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) - && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS - && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || - headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) - && (edge.getPartitioner() instanceof ForwardPartitioner) - && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() - && streamGraph.isChainingEnabled(); - } - - ``` - -4. 显然,此时因为Partitioner条件不满足,source与sink两个Operator算子不会被chain在同一个JobVertex中 -5. 紧接着任务提交到JobManager中执行,在构建ExecutionGraph时,两个Operator算子最后就会生成2个ExecutionJobVertex,再根据Operator的 parallelism 生成对应数量的 ExecutionVertex,一个ExecutionVertex对应一个最终执行的Task任务 -6. 最差的情况,因为Operator无法被chain的原因,同一个job的两个Task任务有可能在不同节点的TaskExecutor的Slot上执行,此时数据传输的效率最低。 - -##### 结论:Main函数因为历史原因加入了 Partition,之前有一些累加器的数值采集的是两个vertx之间的数据,如果只有一个vertx的情况下,累加器的数值为空。
目前 Flinkx 经过几番迭代后,根据自测结果,程序移除 Partition 并不影响 Accumulator 的统计数值。
如果程序没有其他对 Partition 的依赖项,就可以移除 PartitionTransformation 让算子chain。 - - -```java - -//测试代码 -@Test -public void testFlinkxTransformationn() throws Exception { - - Set set = new HashSet<>(1); - set.add(1L); - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - CollectionInputFormat inputFormat = new CollectionInputFormat(set, LongSerializer.INSTANCE); - TypeInformation typeInfo = TypeInformation.of(Long.class); - InputFormatSourceFunction inputFormatSourceFunction = new InputFormatSourceFunction(inputFormat, typeInfo); - DataStream sourceStream = env.addSource(inputFormatSourceFunction, "source", typeInfo); - - - // sourceStream = new DataStream<>(sourceStream.getExecutionEnvironment(), - // new PartitionTransformation<>(sourceStream.getTransformation(), - // new RebalancePartitioner<>())); - - - PrintingOutputFormat printingOutputFormat = new PrintingOutputFormat(); - OutputFormatSinkFunction outputFormatSinkFunction = new OutputFormatSinkFunction(printingOutputFormat); - - sourceStream.addSink(outputFormatSinkFunction); - - - // assertEquals(2, env.getStreamGraph().getJobGraph().getNumberOfVertices()); - assertEquals(1, env.getStreamGraph().getJobGraph().getNumberOfVertices()); -} -``` - -Ps:SlotSharingGroup 与 CoLocationGroup 属性可优化算子子任务的分配方式 diff --git a/flinkx-cassandra/flinkx-cassandra-core/pom.xml b/flinkx-cassandra/flinkx-cassandra-core/pom.xml new file mode 100644 index 0000000000..6dc4304d67 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-core/pom.xml @@ -0,0 +1,22 @@ + + + + flinkx-cassandra + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-cassandra-core + + + + com.datastax.cassandra + cassandra-driver-core + 3.0.0 + + + + \ No newline at end of file diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java new file mode 100644 index 0000000000..072a2b0096 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java @@ -0,0 +1,46 @@ +package com.dtstack.flinkx.cassandra; + +/** + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraConfigKeys { + /** + * 未填写,默认是9042 + */ + public final static String KEY_HOST = "host"; + + public final static String KEY_PORT = "port"; + + public final static String KEY_USERNAME = "username"; + + public final static String KEY_PASSWORD = "password"; + + public final static String KEY_USE_SSL = "useSSL"; + + public final static String KEY_KEY_SPACE = "keyspace"; + + public final static String KEY_TABLE = "table"; + + public final static String KEY_COLUMN = "column"; + + public final static String KEY_WHERE = "where"; + + public final static String KEY_ALLOW_FILTERING = "allowFiltering"; + + public final static String KEY_CONSITANCY_LEVEL = "consistancyLevel"; + + public final static String KEY_ASYNC_WRITE = "asyncWrite"; + + public final static String KEY_CONNECTION_PER_HOST = "connectionsPerHost"; + + public final static String KEY_MAX_PENDING_CONNECTION = "maxPendingPerConnection"; + + /** + * 异步写入的批次大小,默认1(不异步写入) + */ + public final static String KEY_BATCH_SIZE = "batchSize"; + + public final static String KEY_CASSANDRA_CONFIG = "cassandraConfig"; +} diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java new file mode 100644 index 0000000000..fa89243336 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.cassandra; + +import com.datastax.driver.core.*; +import com.datastax.driver.core.LocalDate; +import com.google.common.base.Preconditions; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.sql.Time; +import java.util.Date; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; + +/** + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraUtil { + private static final Logger LOG = LoggerFactory.getLogger(CassandraUtil.class); + + /** + * 获取cassandraSession + * @param cassandraConfig cassandra配置 + * @param clusterName 集群名称,可以空字符串,用于重新连接使用 + * @return cassandraSession + */ + public static Session getSession(Map cassandraConfig, String clusterName) { + Session cassandraSession; + try { + String keySpace = MapUtils.getString(cassandraConfig, KEY_KEY_SPACE); + + Preconditions.checkNotNull(keySpace, "keySpace must not null"); + + // 获取集群 + Cluster cluster = getCluster(cassandraConfig, clusterName); + + // 创建session + cassandraSession = cluster.connect(keySpace); + LOG.info("Get cassandra session successful"); + return cassandraSession; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * 获取Cluster + * @param cassandraConfig cassandra配置 + * @param clusterName 集群名称 + * @return 返回Cluster实例 + */ + private static Cluster getCluster(Map cassandraConfig, String clusterName) { + Cluster cassandraCluster; + try { + String username = MapUtils.getString(cassandraConfig, KEY_USERNAME); + String password = MapUtils.getString(cassandraConfig, KEY_PASSWORD); + Integer port = MapUtils.getInteger(cassandraConfig, KEY_PORT); + String hosts = MapUtils.getString(cassandraConfig, KEY_HOST); + boolean useSSL = MapUtils.getBooleanValue(cassandraConfig, KEY_USE_SSL); + int connectionsPerHost = MapUtils.getIntValue(cassandraConfig, KEY_CONNECTION_PER_HOST, 8); + int maxPendingPerConnection = MapUtils.getIntValue(cassandraConfig, KEY_MAX_PENDING_CONNECTION, 128); + + Preconditions.checkNotNull(hosts, "url must not null"); + + // 创建集群 + Cluster.Builder builder = Cluster.builder().addContactPoints(hosts.split(",")).withPort(port); + builder = StringUtils.isNotEmpty(clusterName) ? builder.withClusterName(clusterName) : builder; + builder = useSSL ? builder.withSSL() : builder; + if ((username != null) && !username.isEmpty()) { + builder = builder.withCredentials(username, password); + } + + PoolingOptions poolingOptions = new PoolingOptions() + .setConnectionsPerHost(HostDistance.LOCAL, connectionsPerHost, connectionsPerHost) + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxPendingPerConnection) + .setNewConnectionThreshold(HostDistance.LOCAL, 100); + + cassandraCluster = builder.withPoolingOptions(poolingOptions).build(); + + LOG.info("Get cassandra cluster successful"); + return cassandraCluster; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * 关闭集群与会话 + * @param session 会话实例 + */ + public static void close(Session session){ + Cluster cluster = null; + if (session != null){ + LOG.info("Start close cassandra session"); + cluster = session.getCluster(); + session.close(); + LOG.info("Close cassandra session successfully"); + } + + if (cluster != null){ + LOG.info("Start close cassandra cluster"); + cluster.close(); + LOG.info("Close cassandra cluster successfully"); + } + } + + /** + * 从cassandra中获取数据 + * @param row 一行数据 + * @param type 字段类型 + * @param columnName 字段名 + * @return 返回该字段对应的值 + */ + public static Object getData(Row row, DataType type, String columnName) { + Object value = null; + + try { + if (type == DataType.bigint()) { + value = row.getLong(columnName); + } else if (type == DataType.cboolean()) { + value = row.getBool(columnName); + } else if (type == DataType.blob()) { + value = row.getBytes(columnName).array(); + } else if (type == DataType.timestamp()) { + value = row.getTimestamp(columnName); + } else if (type == DataType.decimal()) { + value = row.getDecimal(columnName); + } else if (type == DataType.cfloat()) { + value = row.getFloat(columnName); + } else if (type == DataType.inet()) { + value = row.getInet(columnName).getHostAddress(); + } else if (type == DataType.cint()) { + value = row.getInt(columnName); + } else if (type == DataType.varchar()) { + value = row.getString(columnName); + } else if (type == DataType.uuid() || type == DataType.timeuuid()) { + value = row.getUUID(columnName).toString(); + } else if (type == DataType.varint()) { + value = row.getVarint(columnName); + } else if (type == DataType.cdouble()) { + value = row.getDouble(columnName); + } else if (type == DataType.text()) { + value = row.getString(columnName); + } else if (type == DataType.ascii()) { + value = row.getString(columnName); + } else if (type == DataType.smallint()) { + value = row.getShort(columnName); + } else if (type == DataType.tinyint()) { + value = row.getByte(columnName); + } else if (type == DataType.date()) { + value = row.getDate(columnName).getMillisSinceEpoch(); + } else if (type == DataType.time()) { + value = row.getTime(columnName); + } + } catch (Exception e) { + LOG.info("获取'{}'值发生异常:{}", columnName, e); + } + + if (value == null) { + LOG.info("Column '{}' Type({}) get cassandra data is NULL.", columnName, type); + } + return value; + } + + + /** + * 对象转byte[] + * @param obj 对象实例 + * @param 对象类型 + * @return Optional + */ + private static Optional objectToBytes(T obj){ + byte[] bytes = null; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + ObjectOutputStream sOut; + try { + sOut = new ObjectOutputStream(out); + sOut.writeObject(obj); + sOut.flush(); + bytes= out.toByteArray(); + } catch (IOException e) { + e.printStackTrace(); + } + return Optional.ofNullable(bytes); + } + + /** + * 设置值到对应的pos上 + * @param ps preStatement + * @param pos 位置 + * @param sqlType cql类型 + * @param value 值 + * @throws RuntimeException 对于不支持的数据类型,抛出异常 + */ + public static void bindColumn(BoundStatement ps, int pos, DataType sqlType, Object value) throws Exception { + if (value != null) { + switch (sqlType.getName()) { + case ASCII: + case TEXT: + case VARCHAR: + ps.setString(pos, (String) value); + break; + + case BLOB: + ps.setBytes(pos, ByteBuffer.wrap(objectToBytes(value).orElseGet(() -> new byte[8]))); + break; + + case BOOLEAN: + ps.setBool(pos, (Boolean) value); + break; + + case TINYINT: + ps.setByte(pos, ((Integer)value).byteValue()); + break; + + case SMALLINT: + ps.setShort(pos, ((Integer)value).shortValue()); + break; + + case INT: + ps.setInt(pos, (Integer)value); + break; + + case BIGINT: + ps.setLong(pos, (Long)value); + break; + + case VARINT: + ps.setVarint(pos, BigInteger.valueOf((Long) value)); + break; + + case FLOAT: + ps.setFloat(pos, (Float)value); + break; + + case DOUBLE: + ps.setDouble(pos, (Double) value); + break; + + case DECIMAL: + ps.setDecimal(pos, (BigDecimal) value); + break; + + case DATE: + ps.setDate(pos, LocalDate.fromMillisSinceEpoch(((Date)value).getTime())); + break; + + case TIME: + ps.setTime(pos, ((Time) value).getTime()); + break; + + case TIMESTAMP: + ps.setTimestamp(pos, (Date) value); + break; + + case UUID: + case TIMEUUID: + ps.setUUID(pos, UUID.fromString((String) value)); + break; + + case INET: + ps.setInet(pos, InetAddress.getByName((String) value)); + break; + + default: + throw new RuntimeException("暂不支持该数据类型: " + sqlType); + + } + } else { + ps.setToNull(pos); + } + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java new file mode 100644 index 0000000000..179e420193 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java @@ -0,0 +1,36 @@ +package com.dtstack.flinkx.cassandra; + +import com.datastax.driver.core.*; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; + +public class TestCassandraUtil { + public static void main(String[] args) { + Map configMap = new HashMap<>(); + configMap.put(KEY_HOST, "101.37.175.174"); + configMap.put(KEY_KEY_SPACE, "tp"); + + Session session = CassandraUtil.getSession(configMap, ""); + String query = "SELECT * FROM emp"; + session.execute(query); + ResultSet result = session.execute(query); + Iterator iterator = result.iterator(); + while (iterator.hasNext()) { + Row row = iterator.next(); + ColumnDefinitions columnDefinitions = row.getColumnDefinitions(); + List definitions = columnDefinitions.asList(); + for (ColumnDefinitions.Definition definition : definitions) { + Object value = CassandraUtil.getData(row, definition.getType(), definition.getName()); + System.out.println(value); + } + System.out.println(); + } + + CassandraUtil.close(session); + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml new file mode 100644 index 0000000000..feca24f0f9 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -0,0 +1,96 @@ + + + + flinkx-cassandra + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-cassandra-reader + + + + flinkx-cassandra-core + com.dtstack.flinkx + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.cassandrareader.io.netty + + + com.google + shade.cassandrareader.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java new file mode 100644 index 0000000000..cd8da1f553 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java @@ -0,0 +1,12 @@ +package com.dtstack.flinkx.cassandra.reader; + +/** + * + * @Company: www.dtstack.com + * @author wuhui + */ +class CassandraConstants { + final static String TOKEN = "token("; + final static String RANDOM_PARTITIONER = "RandomPartitioner"; + final static String MURMUR3_PARTITIONER = "Murmur3Partitioner"; +} diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java new file mode 100644 index 0000000000..8d3aea49f6 --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.cassandra.reader; + +import com.datastax.driver.core.*; +import com.dtstack.flinkx.cassandra.CassandraUtil; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.google.common.base.Preconditions; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Read plugin for reading static data + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraInputFormat extends BaseRichInputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(CassandraInputFormat.class); + + protected String table; + + protected String whereString; + + protected String consistancyLevel; + + protected boolean allowFiltering; + + protected String keySpace; + + protected List columnMeta; + + protected Map cassandraConfig; + + protected transient Session session; + + protected transient Iterator cursor; + + @Override + protected void openInternal(InputSplit inputSplit) { + CassandraInputSplit split = (CassandraInputSplit) inputSplit; + + Preconditions.checkNotNull(table, "table must not null"); + session = CassandraUtil.getSession(cassandraConfig, ""); + + ConsistencyLevel cl; + if (consistancyLevel != null && !consistancyLevel.isEmpty()) { + cl = ConsistencyLevel.valueOf(consistancyLevel); + } else { + cl = ConsistencyLevel.LOCAL_QUORUM; + } + + String queryString = getQueryString(split); + LOG.info("查询SQL: {}", queryString); + LOG.info("split: {}, {}", split.getMinToken(), split.getMaxToken()); + ResultSet resultSet = session.execute(new SimpleStatement(queryString) + .setConsistencyLevel(cl)); + cursor = resultSet.all().iterator(); + } + + @Override + public Row nextRecordInternal(Row row) { + com.datastax.driver.core.Row cqlRow = cursor.next(); + ColumnDefinitions columnDefinitions = cqlRow.getColumnDefinitions(); + row = new Row(cqlRow.getColumnDefinitions().size()); + List definitions = columnDefinitions.asList(); + + for (int i = 0; i < definitions.size(); i++) { + Object value = CassandraUtil.getData(cqlRow, definitions.get(i).getType(), definitions.get(i).getName()); + row.setField(i, value); + } + LOG.info(row.toString()); + + return row; + } + + @Override + protected void closeInternal() { + CassandraUtil.close(session); + } + + @Override + public InputSplit[] createInputSplitsInternal(int minNumSplits) { + ArrayList splits = new ArrayList<>(); + + try { + Preconditions.checkNotNull(table, "table must not null"); + return splitJob(minNumSplits, splits); + } catch (Exception e){ + throw new RuntimeException(e); + } finally { + CassandraUtil.close(session); + } + } + + /** + * 分割任务 + * @param minNumSplits 分片数 + * @param splits 分片列表 + * @return 返回InputSplit[] + */ + private InputSplit[] splitJob(int minNumSplits, ArrayList splits) { + if(minNumSplits <= 1) { + splits.add(new CassandraInputSplit()); + return splits.toArray(new CassandraInputSplit[splits.size()]); + } + + if(whereString != null && whereString.toLowerCase().contains(CassandraConstants.TOKEN)) { + splits.add(new CassandraInputSplit()); + return splits.toArray(new CassandraInputSplit[splits.size()]); + } + Session session = CassandraUtil.getSession(cassandraConfig, ""); + String partitioner = session.getCluster().getMetadata().getPartitioner(); + if( partitioner.endsWith(CassandraConstants.RANDOM_PARTITIONER)) { + BigDecimal minToken = BigDecimal.valueOf(-1); + BigDecimal maxToken = new BigDecimal(new BigInteger("2").pow(127)); + BigDecimal step = maxToken.subtract(minToken) + .divide(BigDecimal.valueOf(minNumSplits),2, BigDecimal.ROUND_HALF_EVEN); + for ( int i = 0; i < minNumSplits; i++ ) { + BigInteger l = minToken.add(step.multiply(BigDecimal.valueOf(i))).toBigInteger(); + BigInteger r = minToken.add(step.multiply(BigDecimal.valueOf(i+1))).toBigInteger(); + if( i == minNumSplits - 1 ) { + r = maxToken.toBigInteger(); + } + splits.add(new CassandraInputSplit(l.toString(), r.toString())); + } + } + else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { + BigDecimal minToken = BigDecimal.valueOf(Long.MIN_VALUE); + BigDecimal maxToken = BigDecimal.valueOf(Long.MAX_VALUE); + BigDecimal step = maxToken.subtract(minToken) + .divide(BigDecimal.valueOf(minNumSplits),2, BigDecimal.ROUND_HALF_EVEN); + for ( int i = 0; i < minNumSplits; i++ ) { + long l = minToken.add(step.multiply(BigDecimal.valueOf(i))).longValue(); + long r = minToken.add(step.multiply(BigDecimal.valueOf(i+1))).longValue(); + if( i == minNumSplits - 1 ) { + r = maxToken.longValue(); + } + splits.add(new CassandraInputSplit(String.valueOf(l), String.valueOf(r))); + } + } + else { + splits.add(new CassandraInputSplit()); + } + return splits.toArray(new CassandraInputSplit[splits.size()]); + } + + /** + * 拼接查询语句 + * @param inputSplit 分片 + * @return 返回查询语句 + */ + private String getQueryString(CassandraInputSplit inputSplit) { + StringBuilder columns = new StringBuilder(); + if (columnMeta == null) { + columns.append("*"); + } else { + for(MetaColumn column : columnMeta) { + if(columns.length() > 0 ) { + columns.append(","); + } + columns.append(column.getName()); + } + } + + StringBuilder where = new StringBuilder(); + + if( whereString != null && !whereString.isEmpty() ) { + where.append(whereString); + } + String minToken = inputSplit.getMinToken(); + String maxToken = inputSplit.getMaxToken(); + if(minToken !=null || maxToken !=null) { + LOG.info("range:" + minToken + "~" + maxToken); + List pks = session.getCluster().getMetadata().getKeyspace(keySpace).getTable(table) + .getPartitionKey(); + StringBuilder sb = new StringBuilder(); + for(ColumnMetadata pk : pks) { + if( sb.length() > 0 ) { + sb.append(","); + } + sb.append(pk.getName()); + } + String s = sb.toString(); + if (minToken != null && !minToken.isEmpty()) { + if( where.length() > 0 ){ + where.append(" AND "); + } + where.append("token(").append(s).append(")").append(" > ").append(minToken); + } + if (maxToken != null && !maxToken.isEmpty()) { + if( where.length() > 0 ){ + where.append(" AND "); + } + where.append("token(").append(s).append(")").append(" <= ").append(maxToken); + } + } + + StringBuilder select = new StringBuilder(); + select.append("SELECT ").append(columns.toString()).append(" FROM ").append(table); + if( where.length() > 0 ){ + select.append(" where ").append(where.toString()); + } + if(allowFiltering) { + select.append(" ALLOW FILTERING"); + } + select.append(";"); + return select.toString(); + } + + @Override + public boolean reachedEnd() { + return !cursor.hasNext(); + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormatBuilder.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormatBuilder.java new file mode 100644 index 0000000000..15da72c0af --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormatBuilder.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.cassandra.reader; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.reader.MetaColumn; +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Map; + +/** + * The builder for cassandra reader plugin + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraInputFormatBuilder extends BaseRichInputFormatBuilder { + + private CassandraInputFormat format; + + public CassandraInputFormatBuilder() { + super.format = format = new CassandraInputFormat(); + } + + public void setTable(String table){ + format.table = table; + } + + public void setWhere(String where) {format.whereString = where;} + + public void setConsistancyLevel(String consistancyLevel) {format.consistancyLevel = consistancyLevel;} + + public void setAllowFiltering(boolean allowFiltering) {format.allowFiltering = allowFiltering;} + + public void setKeySpace(String keySpace) {format.keySpace = keySpace;} + + public void setColumn(List column) {format.columnMeta = column;} + + public void setCassandraConfig(Map cassandraConfig){ + format.cassandraConfig = cassandraConfig; + } + + @Override + protected void checkFormat() { + Preconditions.checkNotNull(format.table, "table must not null"); + + if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ + throw new UnsupportedOperationException("This plugin not support restore from failed state"); + } + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputSplit.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputSplit.java new file mode 100644 index 0000000000..630ed907ea --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputSplit.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.cassandra.reader; + +import org.apache.flink.core.io.InputSplit; + +/** + * The split for cassandra Reader plugin + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraInputSplit implements InputSplit { + + private String minToken; + + private String maxToken; + + public CassandraInputSplit(){} + + public CassandraInputSplit(String minToken, String maxToken) { + this.minToken = minToken; + this.maxToken = maxToken; + } + + public String getMinToken() { + return minToken; + } + + public void setMinToken(String minToken) { + this.minToken = minToken; + } + + public String getMaxToken() { + return maxToken; + } + + public void setMaxToken(String maxToken) { + this.maxToken = maxToken; + } + + @Override + public int getSplitNumber() { + return 0; + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraReader.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraReader.java new file mode 100644 index 0000000000..92e6aad4ea --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraReader.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.cassandra.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; + +/** + * The Reader plugin for cassandra + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraReader extends BaseDataReader { + + protected String host; + + protected Integer port; + + protected String username; + + protected String password; + + protected boolean useSSL; + + protected String keySpace; + + protected String table; + + protected List column; + + protected String where; + + protected boolean allowFiltering; + + protected String consistancyLevel; + + protected int connectionsPerHost; + + protected int maxPendingPerConnection; + + protected Map cassandraConfig; + + + public CassandraReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + host = readerConfig.getParameter().getStringVal(KEY_HOST); + port = readerConfig.getParameter().getIntVal(KEY_PORT, 9042); + username = readerConfig.getParameter().getStringVal(KEY_USERNAME); + password = readerConfig.getParameter().getStringVal(KEY_PASSWORD); + useSSL = readerConfig.getParameter().getBooleanVal(KEY_USE_SSL, false); + column = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); + where = readerConfig.getParameter().getStringVal(KEY_WHERE); + keySpace = readerConfig.getParameter().getStringVal(KEY_KEY_SPACE); + table = readerConfig.getParameter().getStringVal(KEY_TABLE); + allowFiltering = readerConfig.getParameter().getBooleanVal(KEY_ALLOW_FILTERING, false); + connectionsPerHost = readerConfig.getParameter().getIntVal(KEY_CONNECTION_PER_HOST, 8); + maxPendingPerConnection = readerConfig.getParameter().getIntVal(KEY_MAX_PENDING_CONNECTION, 128); + consistancyLevel = readerConfig.getParameter().getStringVal(KEY_CONSITANCY_LEVEL); + + cassandraConfig = (Map)readerConfig.getParameter().getVal(KEY_CASSANDRA_CONFIG, new HashMap<>()); + cassandraConfig.put(KEY_HOST, host); + cassandraConfig.put(KEY_PORT, port); + cassandraConfig.put(KEY_USERNAME, username); + cassandraConfig.put(KEY_PASSWORD, password); + cassandraConfig.put(KEY_USE_SSL, useSSL); + cassandraConfig.put(KEY_COLUMN, column); + cassandraConfig.put(KEY_WHERE, where); + cassandraConfig.put(KEY_KEY_SPACE, keySpace); + cassandraConfig.put(KEY_TABLE, table); + cassandraConfig.put(KEY_ALLOW_FILTERING, allowFiltering); + cassandraConfig.put(KEY_CONNECTION_PER_HOST, connectionsPerHost); + cassandraConfig.put(KEY_MAX_PENDING_CONNECTION, maxPendingPerConnection); + cassandraConfig.put(KEY_CONSITANCY_LEVEL, consistancyLevel); + } + + @Override + public DataStream readData() { + CassandraInputFormatBuilder builder = new CassandraInputFormatBuilder(); + builder.setTable(table); + builder.setWhere(where); + builder.setConsistancyLevel(consistancyLevel); + builder.setAllowFiltering(allowFiltering); + builder.setKeySpace(keySpace); + builder.setColumn(column); + builder.setCassandraConfig(cassandraConfig); + + builder.setMonitorUrls(monitorUrls); + builder.setBytes(bytes); + + return createInput(builder.finish(),"cassandrareader"); + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml new file mode 100644 index 0000000000..565ef3ca4f --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -0,0 +1,95 @@ + + + + flinkx-cassandra + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-cassandra-writer + + + + flinkx-cassandra-core + com.dtstack.flinkx + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.cassandrawriter.io.netty + + + com.google + shade.cassandrawriter.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java new file mode 100644 index 0000000000..f16eca03ab --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java @@ -0,0 +1,158 @@ +package com.dtstack.flinkx.cassandra.writer; + +import com.datastax.driver.core.*; +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.dtstack.flinkx.cassandra.CassandraUtil; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.google.common.base.Preconditions; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * write plugin for writing static data + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraOutputFormat extends BaseRichOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class); + + protected Long batchSize; + + protected List columnMeta; + + protected boolean asyncWrite; + + protected String keySpace; + + protected String table; + + protected List columnTypes; + + protected String consistancyLevel; + + protected PreparedStatement pstmt; + + protected List unConfirmedWrite; + + protected List bufferedWrite; + + protected Map cassandraConfig; + + protected transient Session session; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + Preconditions.checkNotNull(keySpace, "keySpace must not null!"); + Preconditions.checkNotNull(table, "table must not null"); + + LOG.info("taskNumber: {}, numTasks: {}", taskNumber, numTasks); + session = CassandraUtil.getSession(cassandraConfig, ""); + TableMetadata metadata = session.getCluster().getMetadata().getKeyspace(keySpace).getTable(table); + + columnTypes = new ArrayList<>(columnMeta.size()); + Insert insertStmt = QueryBuilder.insertInto(table); + + for (MetaColumn column : columnMeta) { + insertStmt.value(column.getName(), QueryBuilder.bindMarker()); + ColumnMetadata col = metadata.getColumn(column.getName()); + if (col == null) { + throw new RuntimeException("未找到列名" + column.getName()); + } + columnTypes.add(col.getType()); + } + + if (consistancyLevel != null && !consistancyLevel.isEmpty()) { + insertStmt.setConsistencyLevel(ConsistencyLevel.valueOf(consistancyLevel)); + } else { + insertStmt.setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM); + } + + pstmt = session.prepare(insertStmt); + + if(batchSize > 1) { + if(asyncWrite) { + unConfirmedWrite = new ArrayList<>(); + } else { + bufferedWrite = new ArrayList<>(); + } + } + } + + @Override + protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + // cassandra支持对重复主键的值覆盖,无需判断writeMode + BoundStatement boundStatement = pstmt.bind(); + for (int i = 0; i < columnMeta.size(); i++) { + Object value = row.getField(i); + try { + CassandraUtil.bindColumn(boundStatement, i, columnTypes.get(i), value); + } catch (Exception e) { + // 包装异常 + throw new WriteRecordException("类型转换失败", e.getCause(), i, row); + } + } + LOG.info("insertSql: {}" + boundStatement); + session.execute(boundStatement); + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + if (batchSize > 1) { + BoundStatement boundStatement = pstmt.bind(); + for (Row row : rows) { + for (int i = 0; i < columnMeta.size(); i++) { + Object value = row.getField(i); + CassandraUtil.bindColumn(boundStatement, i, columnTypes.get(i), value); + } + LOG.info("insertSql: {}" + boundStatement); + if(asyncWrite) { + unConfirmedWrite.add(session.executeAsync(boundStatement)); + if (unConfirmedWrite.size() >= batchSize) { + for (ResultSetFuture write : unConfirmedWrite) { + write.getUninterruptibly(10000, TimeUnit.MILLISECONDS); + } + unConfirmedWrite.clear(); + } + } else { + bufferedWrite.add(boundStatement); + if(bufferedWrite.size() >= batchSize) { + BatchStatement batchStatement = new BatchStatement(BatchStatement.Type.UNLOGGED); + batchStatement.addAll(bufferedWrite); + session.execute(batchStatement); + bufferedWrite.clear(); + } + } + } + + // 检查是否还有数据未写出去 + if(unConfirmedWrite != null && unConfirmedWrite.size() > 0) { + for(ResultSetFuture write : unConfirmedWrite) { + write.getUninterruptibly(10000, TimeUnit.MILLISECONDS); + } + unConfirmedWrite.clear(); + } + if(bufferedWrite !=null && bufferedWrite.size() > 0) { + BatchStatement batchStatement = new BatchStatement(BatchStatement.Type.UNLOGGED); + batchStatement.addAll(bufferedWrite); + session.execute(batchStatement); + bufferedWrite.clear(); + } + } + } + + @Override + public void closeInternal() throws IOException { + CassandraUtil.close(session); + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java new file mode 100644 index 0000000000..4446f1633f --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java @@ -0,0 +1,56 @@ +package com.dtstack.flinkx.cassandra.writer; + +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; +import com.dtstack.flinkx.reader.MetaColumn; + +import java.util.List; +import java.util.Map; + +/** + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private CassandraOutputFormat format; + + public CassandraOutputFormatBuilder() { + super.format = format = new CassandraOutputFormat(); + } + + public void setBatchSize(Long batchSize) { + format.batchSize = batchSize; + } + + public void setColumn(List column) { + format.columnMeta = column; + } + + public void setAsyncWrite(boolean asyncWrite) { + format.asyncWrite = asyncWrite; + } + + public void setKeySpace(String keySpace) { + format.keySpace = keySpace; + } + + public void setTable(String table) { + format.table = table; + } + + public void setConsistancyLevel(String consistancyLevel) { + format.consistancyLevel = consistancyLevel; + } + + public void setCassandraConfig(Map cassandraConfig){ + format.cassandraConfig = cassandraConfig; + } + + @Override + protected void checkFormat() { + if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ + throw new UnsupportedOperationException("This plugin not support restore from failed state"); + } + } +} diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java new file mode 100644 index 0000000000..2e2f0bbc9c --- /dev/null +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java @@ -0,0 +1,106 @@ +package com.dtstack.flinkx.cassandra.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_CONSITANCY_LEVEL; +import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_MAX_PENDING_CONNECTION; + +/** + * + * @Company: www.dtstack.com + * @author wuhui + */ +public class CassandraWriter extends BaseDataWriter { + + protected String host; + + protected Integer port; + + protected String username; + + protected String password; + + protected boolean useSSL; + + protected String keySpace; + + protected String table; + + protected List column; + + protected String consistancyLevel; + + protected int connectionsPerHost; + + protected int maxPendingPerConnection; + + protected boolean asyncWrite; + + protected Long batchSize; + + protected Map cassandraConfig; + + + public CassandraWriter(DataTransferConfig config) { + super(config); + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + host = writerConfig.getParameter().getStringVal(KEY_HOST); + port = writerConfig.getParameter().getIntVal(KEY_PORT, 9042); + username = writerConfig.getParameter().getStringVal(KEY_USERNAME); + password = writerConfig.getParameter().getStringVal(KEY_PASSWORD); + useSSL = writerConfig.getParameter().getBooleanVal(KEY_USE_SSL, false); + column = MetaColumn.getMetaColumns(writerConfig.getParameter().getColumn()); + keySpace = writerConfig.getParameter().getStringVal(KEY_KEY_SPACE); + table = writerConfig.getParameter().getStringVal(KEY_TABLE); + connectionsPerHost = writerConfig.getParameter().getIntVal(KEY_CONNECTION_PER_HOST, 8); + maxPendingPerConnection = writerConfig.getParameter().getIntVal(KEY_MAX_PENDING_CONNECTION, 128); + asyncWrite = writerConfig.getParameter().getBooleanVal(KEY_ASYNC_WRITE, false); + batchSize = writerConfig.getParameter().getLongVal(KEY_BATCH_SIZE, 1); + consistancyLevel = writerConfig.getParameter().getStringVal(KEY_CONSITANCY_LEVEL); + + cassandraConfig = (Map)writerConfig.getParameter().getVal(KEY_CASSANDRA_CONFIG, new HashMap<>()); + cassandraConfig.put(KEY_HOST, host); + cassandraConfig.put(KEY_PORT, port); + cassandraConfig.put(KEY_USERNAME, username); + cassandraConfig.put(KEY_PASSWORD, password); + cassandraConfig.put(KEY_USE_SSL, useSSL); + cassandraConfig.put(KEY_COLUMN, column); + cassandraConfig.put(KEY_KEY_SPACE, keySpace); + cassandraConfig.put(KEY_TABLE, table); + cassandraConfig.put(KEY_CONNECTION_PER_HOST, connectionsPerHost); + cassandraConfig.put(KEY_MAX_PENDING_CONNECTION, maxPendingPerConnection); + cassandraConfig.put(KEY_ASYNC_WRITE, asyncWrite); + cassandraConfig.put(KEY_BATCH_SIZE, batchSize); + cassandraConfig.put(KEY_CONSITANCY_LEVEL, consistancyLevel); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + CassandraOutputFormatBuilder builder = new CassandraOutputFormatBuilder(); + builder.setKeySpace(keySpace); + builder.setTable(table); + builder.setColumn(column); + builder.setAsyncWrite(asyncWrite); + builder.setBatchSize(batchSize); + builder.setConsistancyLevel(consistancyLevel); + builder.setCassandraConfig(cassandraConfig); + + builder.setMonitorUrls(monitorUrls); + builder.setErrors(errors); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + return createOutput(dataSet, builder.finish(), "cassandrawriter"); + } +} diff --git a/flinkx-cassandra/pom.xml b/flinkx-cassandra/pom.xml new file mode 100644 index 0000000000..d788a4bea1 --- /dev/null +++ b/flinkx-cassandra/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-cassandra + pom + + flinkx-cassandra-core + flinkx-cassandra-reader + flinkx-cassandra-writer + + + + + flinkx-core + com.dtstack.flinkx + 1.6 + provided + + + \ No newline at end of file diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java index cb52cf4271..a5be666998 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java @@ -37,6 +37,8 @@ public enum EDatabaseType { MongoDB, Redis, ES, + SapHana, + TeraData, /** * contains ftp and sftp diff --git a/flinkx-saphana/flinkx-saphana-core/pom.xml b/flinkx-saphana/flinkx-saphana-core/pom.xml new file mode 100644 index 0000000000..bed09b714f --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-core/pom.xml @@ -0,0 +1,21 @@ + + + + flinkx-saphana + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-saphana-core + + + + com.sap.cloud.db.jdbc + ngdbc + 2.4.51 + + + \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java new file mode 100644 index 0000000000..8fd3137074 --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.saphana; + + +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * The class of SapHana database prototype + * + * Company: www.dtstack.com + * @author wuhui + */ +public class SaphanaDatabaseMeta extends BaseDatabaseMeta { + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.SapHana; + } + + @Override + public String getDriverClass() { + return "com.sap.db.jdbc.Driver"; + } + + @Override + public String getSqlQueryFields(String tableName) { + return "SELECT * FROM " + tableName + " LIMIT 0"; + } + + @Override + public String getSqlQueryColumnFields(List column, String table) { + return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; + } + + @Override + public String getStartQuote() { + return "\""; + } + + @Override + public String getEndQuote() { + return "\""; + } + + @Override + public String quoteValue(String value, String column) { + return String.format("\"%s\" as %s",value,column); + } + + @Override + public String getReplaceStatement(List column, List fullColumn, String table, Map> updateKey) { + return "REPLACE INTO " + quoteTable(table) + + " (" + quoteColumns(column) + ") values " + + makeValues(column.size()); + } + + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + return "INSERT INTO " + quoteTable(table) + + " (" + quoteColumns(column) + ") values " + + makeValues(column.size()) + + " ON DUPLICATE KEY UPDATE " + + makeUpdatePart(column); + } + + private String makeUpdatePart (List column) { + List updateList = new ArrayList<>(); + for(String col : column) { + String quotedCol = quoteColumn(col); + updateList.add(quotedCol + "=values(" + quotedCol + ")"); + } + return StringUtils.join(updateList, ","); + } + + @Override + public String getSplitFilter(String columnName) { + return String.format("%s mod ${N} = ${M}", getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName){ + return String.format("%s.%s mod ${N} = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getRowNumColumn(String orderBy) { + throw new RuntimeException("Not support row_number function"); + } + + private String makeValues(int nCols) { + return "(" + StringUtils.repeat("?", ",", nCols) + ")"; + } + + @Override + protected String makeValues(List column) { + throw new UnsupportedOperationException(); + } + + @Override + public int getFetchSize(){ + return 1000; + } + + @Override + public int getQueryTimeout(){ + return 1000; + } +} diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml new file mode 100644 index 0000000000..ba9e5c4375 --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -0,0 +1,102 @@ + + + + flinkx-saphana + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-saphana-reader + + + + com.dtstack.flinkx + flinkx-saphana-core + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.saphanareader.io.netty + + + com.google + shade.saphanareader.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java new file mode 100644 index 0000000000..0bac77a5d7 --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.saphana.format; + +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ClassUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; + +import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class SaphanaInputFormat extends JdbcInputFormat { + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + try { + LOG.info(inputSplit.toString()); + + ClassUtil.forName(driverName, getClass().getClassLoader()); + + if (incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc()){ + getMaxValue(inputSplit); + } + + initMetric(inputSplit); + + if(!canReadData(inputSplit)){ + LOG.warn("Not read data when the start location are equal to end location"); + + hasNext = false; + return; + } + + dbConn = DbUtil.getConnection(dbUrl, username, password); + + // 部分驱动需要关闭事务自动提交,fetchSize参数才会起作用 + dbConn.setAutoCommit(false); + + Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + // value must be >=0 and <= this.getMaxRows() + statement.setFetchSize(0); + + statement.setQueryTimeout(queryTimeOut); + String querySql = buildQuerySql(inputSplit); + resultSet = statement.executeQuery(querySql); + columnCount = resultSet.getMetaData().getColumnCount(); + + boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); + if(splitWithRowCol){ + columnCount = columnCount-1; + } + + hasNext = resultSet.next(); + + if (StringUtils.isEmpty(customSql)){ + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); + } else { + descColumnTypeList = new ArrayList<>(); + for (MetaColumn metaColumn : metaColumns) { + descColumnTypeList.add(metaColumn.getName()); + } + } + + } catch (SQLException se) { + throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); + } + + LOG.info("JdbcInputFormat[{}]open: end", jobName); + } + + @Override + public Row nextRecordInternal(Row row) throws IOException { + if (!hasNext) { + return null; + } + row = new Row(columnCount); + + try { + for (int pos = 0; pos < row.getArity(); pos++) { + Object obj = resultSet.getObject(pos + 1); + if(obj != null) { + if(CollectionUtils.isNotEmpty(descColumnTypeList)) { + String columnType = descColumnTypeList.get(pos); + if("tinyint".equalsIgnoreCase(columnType)) { + if(obj instanceof Boolean) { + obj = ((Boolean) obj ? 1 : 0); + } + } + } + obj = clobToString(obj); + } + + row.setField(pos, obj); + } + return super.nextRecordInternal(row); + }catch (Exception e) { + throw new IOException("Couldn't read data - " + e.getMessage(), e); + } + } + +} diff --git a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/reader/SaphanaReader.java b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/reader/SaphanaReader.java new file mode 100644 index 0000000000..4ca1e0497a --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/reader/SaphanaReader.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.saphana.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.saphana.SaphanaDatabaseMeta; +import com.dtstack.flinkx.saphana.format.SaphanaInputFormat; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.Collections; + +/** + * SapHana reader plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class SaphanaReader extends JdbcDataReader { + + public SaphanaReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + setDatabaseInterface(new SaphanaDatabaseMeta()); + dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return new JdbcInputFormatBuilder(new SaphanaInputFormat()); + } +} diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml new file mode 100644 index 0000000000..8c19fad040 --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -0,0 +1,101 @@ + + + + flinkx-saphana + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-saphana-writer + + + + com.dtstack.flinkx + flinkx-saphana-core + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.saphanawriter.io.netty + + + com.google + shade.saphanawriter.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaOutputFormat.java b/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaOutputFormat.java new file mode 100644 index 0000000000..664bd7a80f --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaOutputFormat.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.saphana.format; + +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class SaphanaOutputFormat extends JdbcOutputFormat { +} diff --git a/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/writer/SaphanaWriter.java b/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/writer/SaphanaWriter.java new file mode 100644 index 0000000000..e7a63d59e4 --- /dev/null +++ b/flinkx-saphana/flinkx-saphana-writer/src/main/java/com/dtstack/flinkx/saphana/writer/SaphanaWriter.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.saphana.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.saphana.SaphanaDatabaseMeta; +import com.dtstack.flinkx.saphana.format.SaphanaOutputFormat; + +import java.util.Collections; + +/** + * SpaHana writer plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class SaphanaWriter extends JdbcDataWriter { + + public SaphanaWriter(DataTransferConfig config) { + super(config); + setDatabaseInterface(new SaphanaDatabaseMeta()); + dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); + } + + @Override + protected JdbcOutputFormatBuilder getBuilder() { + return new JdbcOutputFormatBuilder(new SaphanaOutputFormat()); + } +} diff --git a/flinkx-saphana/pom.xml b/flinkx-saphana/pom.xml new file mode 100644 index 0000000000..a9ecb0501e --- /dev/null +++ b/flinkx-saphana/pom.xml @@ -0,0 +1,35 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-saphana + pom + + flinkx-saphana-core + flinkx-saphana-reader + flinkx-saphana-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-core/pom.xml b/flinkx-teradata/flinkx-teradata-core/pom.xml new file mode 100644 index 0000000000..2af2575d7e --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-core/pom.xml @@ -0,0 +1,15 @@ + + + + flinkx-teradata + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-teradata-core + + + \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java new file mode 100644 index 0000000000..0f6535497b --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.teradata; + + +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * The class of TeraData database prototype + * + * Company: www.dtstack.com + * @author wuhui + */ +public class TeradataDatabaseMeta extends BaseDatabaseMeta { + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.TeraData; + } + + @Override + public String getDriverClass() { + return "com.teradata.jdbc.TeraDriver"; + } + + @Override + public String getSqlQueryFields(String tableName) { + return "SELECT * FROM " + tableName + " QUALIFY SUM(1) OVER (ROWS UNBOUNDED PRECEDING) BETWEEN 0 AND 0"; + } + + @Override + public String getSqlQueryColumnFields(List column, String table) { + return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " QUALIFY SUM(1) OVER (ROWS UNBOUNDED" + + " PRECEDING) BETWEEN 0 AND 0"; + } + + @Override + public String getStartQuote() { + return "\""; + } + + @Override + public String getEndQuote() { + return "\""; + } + + @Override + public String quoteValue(String value, String column) { + return String.format("\"%s\" as %s",value,column); + } + + @Override + public String getReplaceStatement(List column, List fullColumn, String table, Map> updateKey) { + throw new UnsupportedOperationException(); + } + + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + throw new UnsupportedOperationException(); + } + + private String makeUpdatePart (List column) { + List updateList = new ArrayList<>(); + for(String col : column) { + String quotedCol = quoteColumn(col); + updateList.add(quotedCol + "=values(" + quotedCol + ")"); + } + return StringUtils.join(updateList, ","); + } + + @Override + public String getSplitFilter(String columnName) { + return String.format("%s mod ${N} = ${M}", getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName){ + return String.format("%s.%s mod ${N} = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getRowNumColumn(String orderBy) { + throw new RuntimeException("Not support row_number function"); + } + + private String makeValues(int nCols) { + return "(" + StringUtils.repeat("?", ",", nCols) + ")"; + } + + @Override + protected String makeValues(List column) { + throw new UnsupportedOperationException(); + } + + @Override + public int getFetchSize(){ + return 1000; + } + + @Override + public int getQueryTimeout(){ + return 1000; + } +} diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java new file mode 100644 index 0000000000..08e90eed83 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java @@ -0,0 +1,83 @@ +package com.dtstack.flinkx.teradata.util; + +import com.dtstack.flinkx.rdb.DatabaseInterface; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ClassUtil; + +import java.sql.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * @author wuhui + */ +public class DBUtil { + /** + * 获取数据库连接,不使用DBUtil里的getConnection为了避免Telnet,因为jdbc4与jdbc3不同 + * @param url 连接url + * @param username 用户名 + * @param password 密码 + * @return 返回connection + * @throws SQLException 连接失败抛出异常 + */ + public static Connection getConnection(String url, String username, String password) throws SQLException { + Connection dbConn; + synchronized (ClassUtil.LOCK_STR){ + DriverManager.setLoginTimeout(10); + + if (username == null) { + dbConn = DriverManager.getConnection(url); + } else { + dbConn = DriverManager.getConnection(url, username, password); + } + } + + return dbConn; + } + + /** + * 获取表列名类型列表 + * @param dbURL jdbc url + * @param username 数据库账号 + * @param password 数据库密码 + * @param databaseInterface DatabaseInterface + * @param table 表名 + * @param metaColumns MetaColumn列表 + * @return + */ + public static List analyzeTable(String dbURL, String username, String password, DatabaseInterface databaseInterface, + String table, List metaColumns) { + List ret = new ArrayList<>(metaColumns.size()); + Connection dbConn = null; + Statement stmt = null; + ResultSet rs = null; + try { + dbConn = getConnection(dbURL, username, password); + stmt = dbConn.createStatement(); + rs = stmt.executeQuery(databaseInterface.getSqlQueryFields(databaseInterface.quoteTable(table))); + ResultSetMetaData rd = rs.getMetaData(); + + Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); + for(int i = 0; i < rd.getColumnCount(); ++i) { + nameTypeMap.put(rd.getColumnName(i+1),rd.getColumnTypeName(i+1)); + } + + for (MetaColumn metaColumn : metaColumns) { + if(metaColumn.getValue() != null){ + ret.add("string"); + } else { + ret.add(nameTypeMap.get(metaColumn.getName())); + } + } + } catch (SQLException e) { + throw new RuntimeException(e); + } finally { + DbUtil.closeDbResources(rs, stmt, dbConn, false); + } + + return ret; + } +} diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml new file mode 100644 index 0000000000..44cc030d1f --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-reader/pom.xml @@ -0,0 +1,108 @@ + + + + flinkx-teradata + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-teradata-reader + + + + com.dtstack.flinkx + flinkx-teradata-core + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.teradatareader.io.netty + + + com.google + shade.teradatareader.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java new file mode 100644 index 0000000000..cb0f896f43 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.teradata.format; + +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.teradata.util.DBUtil; +import com.dtstack.flinkx.util.ClassUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.*; +import java.util.ArrayList; + +import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class TeradataInputFormat extends JdbcInputFormat { + private static final Logger LOG = LoggerFactory.getLogger(TeradataInputFormat.class); + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + try { + LOG.info(inputSplit.toString()); + + ClassUtil.forName(driverName, getClass().getClassLoader()); + + if (incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc()){ + getMaxValue(inputSplit); + } + + initMetric(inputSplit); + + if(!canReadData(inputSplit)){ + LOG.warn("Not read data when the start location are equal to end location"); + + hasNext = false; + return; + } + + dbConn = DBUtil.getConnection(dbUrl, username, password); + + // 部分驱动需要关闭事务自动提交,fetchSize参数才会起作用 + dbConn.setAutoCommit(false); + + Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + + statement.setFetchSize(0); + + statement.setQueryTimeout(queryTimeOut); + String querySql = buildQuerySql(inputSplit); + resultSet = statement.executeQuery(querySql); + columnCount = resultSet.getMetaData().getColumnCount(); + + boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); + if(splitWithRowCol){ + columnCount = columnCount-1; + } + + hasNext = resultSet.next(); + + if (StringUtils.isEmpty(customSql)){ + descColumnTypeList = DBUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); + } else { + descColumnTypeList = new ArrayList<>(); + for (MetaColumn metaColumn : metaColumns) { + descColumnTypeList.add(metaColumn.getName()); + } + } + + } catch (SQLException se) { + throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); + } + + LOG.info("JdbcInputFormat[{}]open: end", jobName); + } + + @Override + public Row nextRecordInternal(Row row) throws IOException { + if (!hasNext) { + return null; + } + row = new Row(columnCount); + + try { + for (int pos = 0; pos < row.getArity(); pos++) { + Object obj = resultSet.getObject(pos + 1); + if(obj != null) { + if(CollectionUtils.isNotEmpty(descColumnTypeList)) { + String columnType = descColumnTypeList.get(pos); + if("byteint".equalsIgnoreCase(columnType)) { + if(obj instanceof Boolean) { + obj = ((Boolean) obj ? 1 : 0); + } + } + } + obj = clobToString(obj); + } + + row.setField(pos, obj); + } + return super.nextRecordInternal(row); + }catch (Exception e) { + throw new IOException("Couldn't read data - " + e.getMessage(), e); + } + } + +} diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/TeradataReader.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/TeradataReader.java new file mode 100644 index 0000000000..e5bbdd8e97 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/TeradataReader.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.teradata.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import com.dtstack.flinkx.teradata.TeradataDatabaseMeta; +import com.dtstack.flinkx.teradata.format.TeradataInputFormat; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + + +/** + * SapHana reader plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class TeradataReader extends JdbcDataReader { + + public TeradataReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + setDatabaseInterface(new TeradataDatabaseMeta()); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return new JdbcInputFormatBuilder(new TeradataInputFormat()); + } +} diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml new file mode 100644 index 0000000000..bd44533374 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-writer/pom.xml @@ -0,0 +1,108 @@ + + + + flinkx-teradata + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-teradata-writer + + + + com.dtstack.flinkx + flinkx-teradata-core + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.teradatawriter.io.netty + + + com.google + shade.teradatawriter.com.google + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java new file mode 100644 index 0000000000..dd3757fa06 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.teradata.format; + +import com.dtstack.flinkx.enums.EWriteMode; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; +import com.dtstack.flinkx.util.ClassUtil; +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; + +import static com.dtstack.flinkx.teradata.util.DBUtil.getConnection; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class TeradataOutputFormat extends JdbcOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(TeradataOutputFormat.class); + @Override + protected void openInternal(int taskNumber, int numTasks) { + try { + ClassUtil.forName(driverName, getClass().getClassLoader()); + dbConn = getConnection(dbUrl, username, password); + + if (restoreConfig.isRestore()){ + dbConn.setAutoCommit(false); + } + + if(CollectionUtils.isEmpty(fullColumn)) { + fullColumn = probeFullColumns(table, dbConn); + } + + if (!EWriteMode.INSERT.name().equalsIgnoreCase(mode)){ + if(updateKey == null || updateKey.size() == 0) { + updateKey = probePrimaryKeys(table, dbConn); + } + } + + if(fullColumnType == null) { + fullColumnType = analyzeTable(); + } + + for(String col : column) { + for (int i = 0; i < fullColumn.size(); i++) { + if (col.equalsIgnoreCase(fullColumn.get(i))){ + columnType.add(fullColumnType.get(i)); + break; + } + } + } + + preparedStatement = prepareTemplates(); + readyCheckpoint = false; + + LOG.info("subTask[{}}] wait finished", taskNumber); + } catch (SQLException sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } + } +} diff --git a/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/writer/TeradataWriter.java b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/writer/TeradataWriter.java new file mode 100644 index 0000000000..e69facae73 --- /dev/null +++ b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/writer/TeradataWriter.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.teradata.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.teradata.TeradataDatabaseMeta; +import com.dtstack.flinkx.teradata.format.TeradataOutputFormat; + +/** + * Teradata writer plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class TeradataWriter extends JdbcDataWriter { + + public TeradataWriter(DataTransferConfig config) { + super(config); + setDatabaseInterface(new TeradataDatabaseMeta()); + } + + @Override + protected JdbcOutputFormatBuilder getBuilder() { + return new JdbcOutputFormatBuilder(new TeradataOutputFormat()); + } +} diff --git a/flinkx-teradata/pom.xml b/flinkx-teradata/pom.xml new file mode 100644 index 0000000000..9bfb00d6d4 --- /dev/null +++ b/flinkx-teradata/pom.xml @@ -0,0 +1,36 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-teradata + pom + + flinkx-teradata-reader + flinkx-teradata-core + flinkx-teradata-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 0d36173434..7bfcd3cfad 100644 --- a/pom.xml +++ b/pom.xml @@ -27,6 +27,9 @@ flinkx-db2 flinkx-gbase flinkx-clickhouse + flinkx-saphana + flinkx-teradata + flinkx-cassandra flinkx-hdfs flinkx-hive From fe6d97f1b7651f5c7be254171b102513d4ea0494 Mon Sep 17 00:00:00 2001 From: jiangbo <245730400@qq.com> Date: Tue, 17 Mar 2020 10:41:33 +0800 Subject: [PATCH 019/136] Update README.md --- README.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index be38aa2e6a..22d2c0d661 100644 --- a/README.md +++ b/README.md @@ -357,6 +357,11 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 | 1.5.x | 1.5.4 | | 1.8.x | 1.8.1 | -## 7.招聘信息 +## 7.交流群 +

+ +
+ +## 8.招聘信息 1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 From 9d9496313594f044015eaf0a323f5fc2337ef423 Mon Sep 17 00:00:00 2001 From: jiangbo <245730400@qq.com> Date: Thu, 19 Mar 2020 16:06:45 +0800 Subject: [PATCH 020/136] Add files via upload --- docs/images/ding.jpg | Bin 0 -> 73843 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/images/ding.jpg diff --git a/docs/images/ding.jpg b/docs/images/ding.jpg new file mode 100644 index 0000000000000000000000000000000000000000..605928bae9b06b8f6ea7da5e4417749cd63da6d7 GIT binary patch literal 73843 zcmeFZcUV)~)-N1TEFdUd0a3byCM6&ss7Mb*APF5L(wj;*6dS!s54{ISLg+<6K1gdgB^4Dc90ewmbv^b$E4DH#O?`Bm!MwA55o)Lcx=^tbuB1@H26^YIExC`${A z%8T*xJ=A_AucD@@r70w>XQKDysj`Nq`Y$3x6ciNHl+>KGw4CZ7K9KtVb~^nGpuBWG z^_<-~B31w~CDA!bqSFQd6M;$Mb40)RJxdoZo+r6PC?nj}y#^p6K6mZ{>4l5uNY7n7 zM?&Zm@wxLP7bvMNQnLs`dueFtg+w)8Iz<-JJu-0!k6;ykoA#xyk4-C+okQlaoF)YR zzW!O!CJ5#jm7XyzQv97$%+&ea41qKQq0N7!vt|hbBR)rR{sQ412PNSSfS8bo2rd32 z5m6GeoD)3%h(v>mn$_fG_}kM_z?E}^N|fg)0S^GhZ~h|l|89=8Y!Z(mRppqS9LSLe zkEI`tz5axrI|Y2EN|j$VOHC-D*)KFlQBkJ>h3`+A%I$JtHX8g}#u)nahN()$V=^zO z@Ln&7BWD=ffPEcGb6VM^yz)tP#u8|2DhHE^jEMU+kRij-0i>A{AP zqiV_>m)Cod%v8Zy4F#0aPgYL>8=Ut!L`x(U)bb^St=9dqB$UP>vCdyzBZ5KYfFYQT z1C8R789cm_k+ni^K%OVxxFB%&o#4`O;wgZh`is}}@7&CsYQQ}<=9H1{?=SDuum8^6 z_@9c|g&dz)m`Tw%2U9f|LV_b)@%M%m2tO%8Se<&hE&r8 z9-r4l+oSQL2U_WZ-QEqABIZ_|RhSLK$s(BpZ>-gaoHC$nv?)hmzTbAfHOI<}dfkTs zTYeU&fLDUkcjhR%Pqv=bX%c$)l+`0`pqFQX<$q(53^p@Y&BuehP`Q1NboNh8HhUlV3&P1BVwjY9216Sein z8<$ySVW<^s&d2!9^XgHO4-Q*DTvqtEnqoR@TquF+Gh7oRv}5fdz65gk668heU{Fb` z;pCDQv1j7D;LM7qU8GHlDT7ufWkg2AhmiNcD_Uk16@$_+l_zx_e6bOWS6(eG>3I}& zbT!#t=Yz<(mvS70SfB>otv?^`QR)|1N%0qt!7hINIIM+nfGY9_u?V$n4zKb%dV=$& zgg)OE9UL66jBbbu*AYj(8#)#>D#zr_w4&{~J`d^sY#AWE5;fbzv}&oiU;>W#0hc2p zYokp?Klyq)YPi!cSuu74E^$~ekzcBsOVJ1*Pj$%Z%1*>6R8uXVy|v;xnu@VB8;8ZCmo0`Of-ogEuSJ$iAPGvtBdB5fj(J%S3+kFUrWx9~$D5&D2Sr4cTl)I7r6u zj+>W$rxk!y+xH}QWmr8fDz$lhe?L;Zs=29v%dd$JV^LompcZe8nzYD5NG z8?U~s=%+7z%WY>{$|*yW7})G|ov$LNg$HryG=Wmy?dxuXtQ0ORtp-srnJMtVSy?mk z`qA1YiSn(Tta?mJ0i&6gFZA6LH`1F*C1F2i%a{W+QXAt^Rpmc0%ymk$_02i(M?ot1 zKdI>Fa?P=PN3TW4Wj^}gAz}&*pxbTGFUezi5OB@s{)jc!0)@I=di}`u-Qi?YK6bR6 zKaw>-kXscyp%bR$e+ko4Fqxthgl zBS+cHaVYa_D^ryDHyI_jJc#2Ba;j56db!n5-i)giLKw0h{(4sP9Gs6<)4Vq`Qj>8m zi+MlPCSgYj zi-k#(KjH#r@v^KU{#4Yo7i2K15-?qjmvT1Hwjh^~OM!}R>5YVutn}fis#XaT4zzXW zfTTgpmKH}|xxEPQr@Y-hHk*PY# zq2F01s8oo}0NH7$BTcoz^yihm=8OjOS%y)?|BP5WaX9<3Gbc=`TOs$7s&Afb1>NkR zm6UXG$~RUwfpOD*oDWsD6sfYB*~SODBR(B9@VcC#b&?_&E4Vz=%Qm#Q#34Y(B$f%Q zr2(&3P$H@n)j2^AzHg-*l+%S#4eRE4gmCUdxmvLC`KD&?@>I3cCteB8Hf5hkYn=jM zGh&i`BcVqdC)4f>)l&C_Y<5d=})MSysPP4y8oiw{jSlEB-qf-nq{>)C5ZXFw_8z| z@b}?Hofr94R0m<5m4k2&o7MZk7zaeeU|r&ZG;>zt_{M`z?=%;%kX;D(3tdW+n)8rY zq<@ejCBw(0Rfu{(QTmuQjuXWgTpW1~E(;C0e8ALLtH!wFV!g_x)h|+sip30AjbzvG z2Fpvca&*he3-r3;$X%&N3ySbCZOjtu{`$Kh^6rL^S`DGuI&B--&B#~jttra=rvRc; zz}r-%djq6na~$!ZAq>Lt4baM+YEQF6g*AH_T8Uy|Hh~+@ipVzE2Kb@+5yv;n)5P9J z-r;{s-i!HQD`(9U9~LodtY;&_{6_D=X2B=(jk`Q_aI9>MX`>9;4f2i8fy4f5V!(qc zn#&!&Y8-Q1bMs?b{yT4J3n?oy`+YR*hgQW(6Lgi+)iN`GWUxcw4;CL+^Xu<1s6?Ft zhJ`(sUe8%2S=kyeKGISej_iJiCTE$p6){xCdevH`)C{elY^;}scJvj*r0(SD2);0B zK=S*7=oPG~85&D6I!7$G_xVE$zY?*e^GZe>GZ$4S`&>A`?I!e{j`z+>+FS%`IGkQr z7(IMl253)%p^Y?d!ty0npO;NnmHQ!flhNi9mBCF>svWA>}x$tCL8dY#OIrsTc-Uh(YL1n zHxswZ2hG2qyqMjpkkeHc8Eq{g0NQbEsaXgW8Z=t0j<2jYdSx1w+#VbS-vn7?vUE1b zzrT?m7#5x@CnpDi)~cHbw5lJ!7n?+$z~b9qTiw|SWWS8CokZ|(YAlbBjXA!k5aH8BjWX8{g8gyojxq2Rq| zbvd_^m6dN1i^mS?!Z58uy&d2tr{(r}9b5&=OwCR$jfkL}Mt(i8+!KAk$ig zx9+bAll$gX$&Igx(IeE$-bXP$KHO&}r8A37{-9ah=NPG`ARjoO4diIWX=!}-VKIBw zsQAR}PA{vZmM#Sq=GfnE>fw7nk^-}7AsS9go9lQRt-@=aI7Pf{X_*(3!s7rf@1a6y z9ylpYYxVxYGYgm8Tkp%Sz3NDYed}rtZi#S%%Q{Bv2YI%Y@7!WmU8(&P@!jwM94?ww z%A@}QvoU*)wG!&GP&t;u`i1FXABU*;yIB6UJA58vN!I8L`Rt~HLCLoRflq@ zF=ZBeRne~7Lew=`UmXO;d8EM1&8C~*GRtkImohiALn}0#U&|giJU2X^MFwv4sf7;8 zN%!;t9q%*CBbY2dLWlK&(9-qh4|B~vV4kcb3ccJ~ArqPR>RiSmCRWwb5$2@;BAS1J zd_Ps>oH*IyF89I?3raN!xSq2nbx@^<=bM~!e~A2q&kah-o5OxxhCi+`=S}?x7Xar4 zP|{u#J{o)BF(vDyXt_}DBqp$JfJWSUhzAfo{3rShT>4~^;- zcBQt_8whXZ0V$Kmaf+A_o!C4DyWYc_RswiwEF2syw1xp3X@x8%JAw+y`YtK*j56IVvKQVCO{#hfmPw4A7i zdECxFEM`uv+Eg=}gN;jil$q0DT8G+UM00d7@W~+C@VWi~3t7FEa(bHGC&R zyPG}ZyCR6f_4h^V#DytV;NKkG*|8g1tkt+3T}*Rl=T6ju9<84+xdt|rn9XRVBtx6t zD5%;uw70z}Dz6jVp&|_z5o49+L4uK_Mw#!hW=qA6-&3Ced3A?vw&&O?=Nu*wQw}Su z#{Tj}L%K~Pt!OvicwO({M(5=YF@thvv_xz5jUtg`LB-{r? zlyofOCk}nF&TuaFN_&twRvJ8CmU-*4YdGl|C@L^hiLO(~tS3*QI;4VfvgDA=XjD)B zi4~@WUoJT`Ff8F1zf<9UziqIsCqqx7O^CJPsi0$|Ag$Ac7xrKX5g&7yJ91z#a!4kr zmnHhjT$i^QlPTg(3vEfRsec+L5?p9_nTM*KGAD4Jjc!{Gk3k;N6mp`{%aBxhvAbY7 zTi$CoeYfH1mOWZRrOLRJl5kG*o0R6mJGS+3iK7`ruXd~Z`7&Gt`zHc!RV3h7Q5fQj zipEOBz{T#mz+vDVj2$GWKZ+h&>Fw0OGQzS z3>8Q2C^)Y-PM*4AS*ZYTcMc=jbaNME(A+)_)By#7f`^`kecuqXMa+v;AwJ40< zJ%{2=OXb7q=6QyEU!byY(}0^~Sa z9B6u`Zo>JM*`B?`++0nmT?j%y8H>+Ng4fn%GB12lR5jT7M+k4L;|iD6?^ia8cc&@KH6)Fuc+Yk} z;;x}E9fMRab8O5h!0e!=UN!1~p*n0ubG2G0fWm}Q%DC5cdXsS^iA`vbwHm}n3O`@M zRTqeNX9|x)(QD`agxLC6$8k_OxEL|rySCo2`a@f9JYIl9SENL>6G!F1%38Ju7H`La zu&-$jRn4E^gn^FaSuG8XJVZm)Ylf7!R(EfYWvSR~xnE${e|Bflu9}5ya%Wiuh&+Ol zCHl@z`j#AkG1TOh_vhO1hIYR3H4-Q1Te^A1w0Z~&>`&XkvTT3z$C&=WVF!ZCl*DNonlG5o7dR0~Bgd zs}82^eI@~{k8&&kiffv@swwWvErwDi(w`SVUEJNX>doyk<+cnH{Oos`a;>ML+Opab z{@tm04|V@M4}FM9MS%gP0u6!^)-VRRqgw7xH@s`~A^IrbKikc>W>jl!`FJmAUSjV4 zxhr7f`?Mt}1yT|S70#*waA=UhqX<*i%o>h1(UMDPuZ&a^&1_X|C zqVaEu$O)0P88AaCC!2H#F^2cdn~hf6#XvNtn`2-{%(oj&E$srxpW8%i>whpeocf<- z@(f}zw-h00!NoNH#oIp*VqM{c8;vY!m8R}1Sjc?VCJ&x&f)`E%B8iL#wqh}%ehQIQ zC%ny5_#^FA7UvtwIK!KgQgrw0uYZIt2fYof$*hubp;Kk!azb<1D(%%EfQY1RofNpu zA*8;|U^X|2+33RsLz7$ucKN4pP>8h=(&do6T{^DlxKvj#vjxw&U~X|+RCvzO=JHJx zr8ZqcwaxaE$vLX7N6As&6tDV`T9es3NVVVtC$U3=$@zX7hJ7~C!29C}b^0;KsYV!E@t z!c*xJUqa_4?OBjvVvY{b2Bv$WQPN|+OR<$EGJY4RO&&cnsfQa3mh#K0!>|7Z*-x+pT*MyxucWiCRMQVFjLPy*&T|6$1v#fP@9b|j zz+cO0$$foUY>PsdTna2$m^Z-<@eJr+U9Fja`vWVRoRHe0s;>S5Ssy?}CD<0YuarLP zuu^UBQUqo*`k>UBm}_+%XIsx1Lu!O~DRms+H#YfU7l>{jVF(t_daUd1&i2KRd>FZy z5#CY@Kgegf4}3s8`x>Hf2W0@1$6NH8$ zdJaY0v0$7)f{~{tvCCuVk$SO%9wFSr z7?L7}PT(p*6AHF=C{N-vU!jUV!2v2^YPDNGxJsqB56O~g2&@b7a+PjQ`p;Kj5kP_5 zoHaSW4>IAlv!CiZzsOYr=QptuF)Krl_*y?PG%!3)cw&A7bfDRCf-ufk*hBcVk33Kc zpf4{Sc8X_Ga(k{uFFN4cl)hx6{Wxj!uwzBTl0|p<`6Unf#vsZUb=IuYjcdl5wDdnv z=)Bj;_s5-AZ5a1K=YwpVBlVtT^U+^%d~L=#;E<1{LbW8@?9we^i%WE97$D;JZYhl~ z7V`J)A@HMA6YNcx0VloeiIiE`E72tRY_q$kA!Q zkwb8Mfe|4S2 z$=n@>1rDd<)@^4eR#weU0q;G6gDha*u(Bst-BTYOd!M&YSU0p9t$q8+q_(v^RWoNt zZ~R-qBkM8-0mD{Ax-$kFPVPxsYLu%_8^CsV(Sv0_{jo8MJggrl6_8P0Qde@<+Tkv{ z<<;l_(kmhgAWDI0%A2@R(2{<*SOPPpM(e>tKMFlf-dDriR6H}FHXH1_`anHrm5FHEs#q>{;rHd>0gd0Q?}&CGO#0ZH^VU1SSIEjKbaW^)@UZ z`(46htfpw;t@3-i&WD-Zg15*y;z*X(JIFZpoClT~r0$S~>;nj_QgA zsd~rDo&qiTk*9Ks7%MufZ4(9Gd$Hm3-ciP;r9|sQ9Ez1(WHX#1ExK#5i zi{vT+d80CIiod0P5Fko>>Eg(uh-)yB)s8#Y`VTai{N-&MtdsQW^ zddCfXVc3|O^ukV*se9lHaWO_)Hf_VHm6HSQ;pd{vbSyUtMeW!=C~zm%f|*d=)y(fKM+I| zaicuxFTUwV~-?F*AZ$&C;=p_p83ruM1 z%ZDc7)Op8$^nQot;OoZ1DW$Y2gjeUZ`93Tr5hkQWSK{$ILdr6%&~GxPmqK6_ANuma zCFz`{hoqwj{7{b5Pd%R7LP|ZU!n=X7$+M7}?OGXU>p{Y%f2I$-SFNJQq&YDdyFDWX z)sQ48;x)C7UQ1kf^Rq72b+A1L8vCHbG zfb%omv+HV5G(*vBd@y=P6{=`=1ogTQ(Pw&Kt=b)z0gpot7HF^B7&)FjJOmdC;W#IJ z?oRlw(O)ms3$Er&qC?m)TT>Z*WDL;8`-mx)f-uy@cb`?52-@qqE#_If^ADyF7MLuo zZLD%AK1i|;KbgU%@d&~dL-1HEtmN>@NC+5H(?41<_UMLA`IkuR$rf@$uOR`P61jh& zTJWN?asy8qSKoX7*@YeP)`8WXpX)iW3-Sq10>nhN>>(l4j#;@<32UmK&M?pqLS zMF6P`^g%khWR7AyROAww!;vqHW=+M)Rpfz-gmEn7Ioma)XRh0NPe4(8@%w-#Bdt9+ zVr4*kO+6q8T(Ct`B~yCms$&`mQ;=GvX4k$_JBYLv#zEuXTFy?_2hcukdWR{PVS8!l zRRUSokDM|p*5IRM>xXMo)?kI`n+b(@1)dpu=++S{RGdlU_gonJG2yspm9Ryv4~o(_VM0j zKdaSC9kED1W~#3gY#M7uC6MZ?7XEkRh{hg^}u5 z0_od7>B4WuGV!FAcK?T6ZNXtztLOa}sf~dP7;IiB?qjnDQ-kRjqIKAKrFq_=|Fqc$ z2f~hgg~6_RqZ~NTL!?6bLf3hNZ4ocK)mr>^g{tX2L3ny;-v`B)w*tD964qBm!_qSE z*gbn*fp#z|N9#}Ln_t-L#9u5T*1fr)^^~`9ztAvEJD&o3T+aCP4<;{u ziytc0+s%kaJEWr)vljs#791Q4C^%TP&Q?c|Y5~-mrb0@doZWM~I^ymSja>D2=JEX%DK26YT?7a&c+>aT)yVA4U9&s*iaQk}%1*i0 z1JJA9c*vp36yWIdxZ7ryWL~37Lgj}Z0KQu{o=?W^hgQV3ndSf&E^HEF9Lb4VmKDu? zW>T0-n9-!Oa%tX5{UsTq{lQYSqh5$wCYbQ#k)uW@EDr^aItEyx2jv4)lKmt(fb$AI zUi7@$d_(E!i_m4LSSmo;gPF90y~&*U6{8QUiHf$f@j&VUK+po+2cj!1?kmWm406|G z)3BEo!UJV9Awhg9QGS$_$-}DEk~G5r5L90hoU8gdE(g3|Ohn5N8%?QYlu&vK*ve`$ zI)B5G?>*n;urv@qVc*Z1YuK-cz$H3&q*B!iQ(T1WnsAV}=tX$e)Rz4aietGDr}~yn zz{~y|WeygH81TeYsu22h(Z45-nQW>NJVJ>X;@C3^N>df_2~X3?HtAAo<-NBV4Grz8 zw(oU8=~tU?{u0B#B*7WQwg&%&2BhzTq#o8eJpLGg<Re0z_i4Mq9wZzw~lpcRB*1G;A?gGPO@1{aDn)0f7 zRsu*Ph+AA(>YPwqbM7ZGRF|S;AkZKuQ?s|TQ@PARfSX~NLl(mLeIQPs?gZ4KRy@8=`yGA?dmqM65~@h??%7N z!P*tegXug;&g;9l$>QOEk;+-Y)!`jCr3UGLK>%V^W(+dn%RJyYq z6~*p!!?)=@1-zD7ie7ctz!mLN?55(@>0eE_lGhi(+5ZklP_c8x}NF(2X6r zC?8UfV#~FfqzR8UznMYIx7bi2!s>F%UC#WgVT{jRG^w5^0ny~_I5Lm@#FVohmpi|W zQ$i+2MaMfWH@QRoaf-#S4tpt|p9C82#gkqUZAz=LJ`aqFqLT@rMU&bu0&}owXfO7| zCs8eBFSni}lVXrDvcAFQ=D=$H$~EPHmzilJKPx>lbrt)zatd!ZcrRrsIAGG34@i;D zh!#-Kc=4do{iA+hTpmR}tj1wq!)e07s?pV&g(u8Zd`t>B$TpYO_@+_zJ*8eQINV;r zMnvRLT#`6MSS^5$hk{+#RYo_a*)h5uCt#iBy(+eSk6N?DpQ=A>T9o%frbuVCL8vS^ z+?iv-hq6M4GMmo>@68bX_^q;x%EDImWWKT8$=8n)s+zO2Q@pP*r7~AIC)~f@-p}R_P?l#D-xbRP6GlBGPOF>qY~H7Wl2E(qJ0~Bna)mRj~a}*6{six9O!1z8%kPpn6|7 z_(reo0)hsT^pq}kbTpZM5`n1TIr>qaHm87A)(HJsvjOM%u~{3O!f_Dw_=5Bv*I?=-C9J}gUQLpTY%I!4%F&+oqxni1`>B8=cAMXo{S~5MItQGjqKJe8C8AvS}7cBj>(RJz>0q&x{=yS<(jg_wniv ziqtJIWAoMW4|WqCJ?ztU!v*k(=9GWEyWklZ@Ua`K%AO=DqQCCK#Jh)-VTf0u*W^n5 z5|maPrr~su&V}6ovOgrg^G_1{&;BKrRM-6&eetnQWZ(x6mkxFn#U?STsn3}B^{RyB z!>bhU#V&*{bA|><-n1TKCuZ zWF1PiWo-0I)A@#22`A0+3bI*2;k8lqPD(CQ{Ui(@UmUiLB_afnl}pqW<>2J~ z7HzJcR2(+7oJz4ILhea*%4RmIJ{MKk~CENrcTEpwrdo2I6!-R#NTsG^}V%0FBgAh_t-xJS>(1 zD28`T95exaHev{@PXA&Vr4xeIIr82^&ooicpkKNM$VMg z0Le19VJj;Bohigda|fy)epSBGD}l<(EP%m8Qx%Q)V(;Tvjm2MJr**dXEG8XLx@=)g zPAgq{?CIN-%^ukoM1J!fn>Gz$cIVuh)9TwKLpV}D8uCnYR#x?rUn+>Ozt)j~%F+@O zzeqUFmp|&;Foa(UNrW}M+j$SV8b@0LUKQr(<9RB`R*@GNqN0qipnZ7oFT(xrg8zU7 zzQzdLzr(^P5Ol{-<5Qe;5C^O#Y7=#DBmv|GUb6OW^MkxKLDTX$4{G zX6Q5*-Y|SV6d15V~M^t?NSrj98>1)5B1BE-HTGpq4tMuD%x&H5#UZCZ~vFS!m z6SfT(uSdrgh*xSE`N$gU#{cfZIWVl$NkU}dJX+oS*|frRr_?AtEJ{xO31h0f_@kl( z?R~e}Oo#VoqWPQOUn^3*kf**QssUREoiGb5u`lN21ja`Jq=h42u$#_wc0LdzCWCG` zx0SQHQ!6pgU1+m2o}M`gW1s5lvoj+t7XQLsJhHeC9d=Z=_Z~v&`P;uar>k2+gQ0q) zmpk}O%IoS##n6QF5#Fc-yDbcRy^%ZG79=7rzZ;J?Q8GeCQgp5oZyn)3nb>_(0M&R4zzqZ-lpNu0ONMhmr3D z+sLCvYwH`2xklB-MkXH4w=i_td{}&&h6^VX}IQ)!_+e%;lZxw$2ZU6tAA#LU*yyft9 z>!hs4y-HdP5EBrYt7O=PdVT4zCwybV2&*)Ru>F3$Ginq^=$ ze}z;g)Xmn43G-nSY3`6!v~w&N53EK!OaNHpI9y1QQJX(w1zC$3gs({ESG&6dM&H$( zkx}Hu;i5`bgd&Q1Q#U>C6(g-0y|BpP^WLbf!oKbGjvp_buJKrzSm0=1a?NGUO$&<* zyTRdveGjx$s@7D&D=y614T1ON6o1OBJ{AAj9x>>a7qfOW!UHHrycW3FHtLq6#pXSI}j>Ig+$d0Q_U=Fyo zS?;1~-Jcnu-0|rBm+v!O#@V!^dFm$DW|OfcrZp7fBpzfJ&y@7L>^=P8mh$?VnA*I6 zi6K}&j*&3CeGlRmWRQ{)z8Y|qf*y$I+o}1sSfPyq0$)1Kr`Wi@)oMH#I&8CBeGXnS zoJ%%OS_xQpKn(tP`3zyKyuth9TxyFs^nkf~Iwe4uB~E)0vFGb`TN9NKp697wL_g2N z+SxKBjT<5RL1al&10x*B97(q4e@9}+%e#}F$9_~Y{)y8(;SUqr;VL2Zpu5VxW0=9Q z=`uqN4GREcR9fWRjefdfndX)m2LXhA+sNJ7EX;gHLImi}2u8rY>y`%bN^d!9zUj72FUPmgS)rYH`*(@w=ScJI+X@3sMHeMwk z;+wEp6+6n+EOXjutoodf%T@taPd9}PYiCan@I*###?A}Dfz>#d61kPTlmZa7f(PUS zAxoY-pNcyo#W+CPAg1UWQruqI_M$Yr2@tF3R_^`cz7M9KU=5D?fjhFxJtXl&n3CvW zIs~$Q{%J#Chh@jz^_rcv)pOP`U9130}vwecCQrecS9ydQG4_DH-OFq`U9NRNfE3p`x_SL!F)%Ypnj}*Iq z^r86?)1jS&33P+voZ?kHcz`-LrTp*w64#QvKcnTK4jmOTzGzQe^Is{uUm|*1c^%&E_h^jYAL?Fw6zV~2 z6%iWC<@$Qckn6r~<*JP&WO{4Uq|wVx*Jp3SVb_JK3kGx-)tYzN^OpfKE(Rx!^$;*u}^=Gn7#xoW8Z=HD~s5h7aj}c_v~z3IZ}!{6BcpwZ=vW? zF>+ha+tzjDmXur{XQSYq$qN=+t|}qh{H@tnQiairgMt3VKdzb7c{%rN7Q$8{)@ypi znx)6oVxjK!q3>8|Z488SBc}Fi%Y7-`|POfJ<0FqplN^@ z2_^KNz}9)`vgk4p5mO?A?dh~egbxX@rBJ;8?VSOPD~vMlEwPEja1lk7PVxQN0M8{? zV=JlIve19$m&+TfF``Zf<>4_t7Hy5{=4~EblO$#TVLR*@Zriqzlhys{e|P$U#G;!H z*144+rkXQWn!+ajP*BM2n-YKdf`88a_6S8?JWR!>fM>7sPlo^I2$B53e`164_=Pd6 ze$srmmpux9>i$jNFLLpw8CjPnzV6d2fzNV^v@yD`N|J|bqpf z>01bd9?#RlTYK+70x+fgs!zLG>vam?7EG)Bs$d6P`|ev134I$adX#&r_{AJ4evM{R z_aj+PZc%$@=5FIr=CddstrD#-=HImSND`dj$Pj8}IFLm8FXdi0NYQnYgJkm&&hXqgY#SJPShDI+T1q2pJ)Tl% zKE;=C<{7*LUB*Vam+tjNYAx;y#^<^_s(^bvbG)4Z>?02{TVZyr%8;iEPv?zR??qQ? zu9T!%Y`L42IqYvtlF*P1MUL?mBb&Xx|CYuR_MeY3TYBvnl(kalQ|B3u?nRG6VaGhf z@&uJI{^mNk&7C5$3-|BbFuOc5E8clsfmT~5#qbkmVWe;a6p-GXle>a-X_jBC5tF}(~ z3QDEg^UQ&4%l<_EBt{S;X*nH@i_cJ+>cSmZwq^Fa&d*9)e<%iJWM>2O#TqJdL*f)i z5;i?Al&!4SYPw$=AUbQX&swHPy_aM|V9N^h+uWl+&8`Vl{96;6ijyi@EJulZSZolh z{xP=+RHEB&0yx3p)D@MoC8Ld3aq&kAHIJ8nK$=+8?oNk?l|C@Yzbg zORYaE%~MMXFaC>N32XWR;faSLyZ=sd1d;w7RFVD}tuyfyo@GgQF?)Utk6r(a25+1e zf7SlCiAZNI<&g3C^=|vUGZ^p`_8Y7z8&nGYAxTSWt$+1liUg8>MKKP(vv_v<3{z?5 z^Ia+WVI@`gnp^uX@bowJ07zMf27q_%%+#2G-u>4{{Mht{yJTf|Ip)(!1M}EV9218Q z_nx222;ut^laI^vUq%v^%*hEV`v*v;@E|ZzxFyHMBYKcc8uR`ZsZ4#MZ)jB#W|QZQ zlOJ_onJMzkM18=Lz7Y7c{gn8pMIQQ|18rBUq=s!=9LJQ{OC$x;y}M%+)P%RG`EB%& ztXzeFV^=|U3iF_y{Jp3pq$7R7yk#xnnCPpAo8*lVzAa^WND3Z?T?rGyK~MoX@Uf*a zle+RqgwZ)yB8}NIUl&50!p27}&c`7@vS%2`#93;;CM)z10QfzBA(haCr?3fAiwkoJ z+#pn8=d7?_gGhUvk?NcpAKia3HQJVVPsQdRlEGI{O8Hy8`D^e>J@((%V5FiRZ!Udq z6JHJ#D15k7;ONH$bxGU^UHcXatWtArcG;A;f9BOt?#;A@*^=L?UTwxDCM49?Wj#}< z@9dJ*HXL!g^*6+4+K%T!rOnS}bKWjG+^i3rM;-g8I;D!&|4H)upJd!_GE=;?gJT08 zBrbqXj>o!!eX^(pPL_$Xf5+>;&87gz**c^1owRm}#OqYi%D*-i-IW_#0qw4LBXO7;9d z`#k03CNe0)t~jU`N5wO~S80e@5PuFewodSK)GoN%;~1sfsKHvZ95r8hQa#pv|0Q(P zu|~4|ZSU+^1kc%@!Ims|xIUNGSf=j3`@>PVm@>|DA1>z@ankri@D1SKd${c1Xy*)@ zUifPSKaTrrV4qXBtz>}`J|7{zxhQ$&`ri;t81TQ2eqt{DS+qu*-xM#Z{iF7;C;lbT zhUyZw(rSlJ0k*|~f2}N#JpD}@)&AQ-zXjcKH&M-zD~-MvmT7L(tRk+_={+Z>HWoBR z)H$xDHPY{OuAG7!y}DFOG!_~TaCD`??>c>bvsNEy?Jn{Bv&*;4hcvt3SuV3RxNK}x ztalg1cEi3keg`o?<_+NCJsbBizGtES^O-ZT0DQgKONHXER{h@H6lIrsS9Wyd>kR{lsN6@jvoU>=PCQcKFAgUJ!d1 z6;T(Jr6(|(Yr?NwcO-Y5k@dydXdYW*kgW8t8MQ=%vqLB5$$zgCNWjh?MK!35f13TI zJ>f+stH3wmpY`Z3xB~!O|GqGbN;jY9oST+j*{op&*{?#SJ?vCQg_0#*qnZDll(Rgv zxAT_&hf0!%3i~M_zg|yU6Zw4nOCIer@$t*qs{8!o_p-8!vyLv>OGy6;5RZG9CK^Ff zNn%ryFJ|JCyAn2Ueq6eCQS7Cvx`paK#khw#2Pf*qW}IcuOGn3G*w^n8250^t6*@0_ z`SM8YtJYUa{<$@)D!zH(LyapAXLG+)h{vCXaJ`VFhDPCWV6)(wesmr@agwIh4EW-vj1x%Z>*14 zq(tSW*Z+F$avwZIHj2~FQ`d|H0L}pE-%u`L&XBDtbCBhpb<_zjfU&52V{PXo z)>ss+_#$?px^F}F*CugQPTI#a-TnVC@V8NR?FjiWgwKqQso{E9N}bV1qXFv)2SijX z;NqWNCxoTy&zS$8nCEx5&SBCuR!7?)7MfRV4^#X^=X!FIlva5FBpXY|F{1%fwJ)|$ z+U;aFjX$OX=xeig^{u(y9oP3C@|6-k(>CJ|_#eETc|6o>|NjS35``#Bo9t`$tt27) zGK_r**+#@zDn%U0UfFlaZtQ!=PGzm^iHa;)lhEGzT{AwP;dFHFbANxo-`5}ap-`++S`@KK^`UvF4frU4_LQ>RN`Y6Bl3PM;m0 zxGMjMHgS~Xa#cPeR3i9er`Y`RSOIOW(!PJ*{BXO0`T%s(2YsEo(ZK;d81#7<;nI6Ea9w{Z<%OGE9?(3H9f6+qQN-XthNC}I@Wlmh2I-`WCcUDDtH^JI zpIA5wc89_@6e7QOq`5B~-U_eLgwTgr3_3cVTiKt~6xT5Ow#A#@@sn$$xuI<|e03o| z4S)D_G%7q_#kt1q`YN+ZmjOy7tImE=@*%w4fwR*YX-D3L&`CpO>>olR2YAdz`v#}J z?R%XB#fFGSRc}}iIK34CeH_V>md1_NiF-*n3~v9(y1IjW$?(_aJ<^V&&7$CZVtbB)yZi9e&zDII!bJIBjQh?q~uzff8c4!35HPYeTSq;l0b zc0qdQqejM634GVPBy3lyS9LaD1%<|_600D&6~bsGhR&lMjKfj- z3;cojw<3J}-E$gRbD<+BY?>a21e=umamD7smPCx5y63Z*QaH#&+Uvk0`qh;37$4I9 zsBKZxD)anO%@3tIyNey2(4^RL4_ec2T~)A82H7M`rDQy**lc*Y6LieV*OOimjuGU! z@LN0eHgw$}%5(}~`KH=?sJ?WTGN(eMj`}p#dm{MXZVvx;uW@udIZ8gStd$t;joD+@ z<9Y~_IlG>aikJl{slWLlK7NTeX&~nfGri@W!<0#FNevpj;-;7zQLzYanO|6ni_`zg z??mYN{SWs%8*`--$OgM$yat^aAK>F=eNZsbXQsu= zil>-&%{8E_DTppU8&6c6<`+bcQnZhP7l zU0WKdm&_WW2S#G=9eT_UiNXQQwOs))ZWu-AmC_9}NXJfaM;(mR9_TzGDd54Be4kdm z)7y^&Zpguo$Etm`!riT$l>VOHhmh5hr^#Ol(Q9Pqp>F zN{8B+J6cW_;Vc%-10ErRQ}u7PVNeii@G7>qw)13MQx(l5rNm zovOCr)XmQb7QLkobtAV3Wy8P|S7c5acCJp;qS_Z&m?H-~LSvg=hByIp9*MT>~F3_$sZ9Vg^MYno8d19~_$rO1ymBK664V z)0+M0;~eo5Pr^@(qiYrBF|S>82q9(~Z)k}!mO)Sj#K=ZVu{8Yc;HLi#uV8^8O_28H{ze8E_LLh1LNeyV4}nqA|{ zf@@BU4_tnzsJNv84Wk%74RB&yKNt+kSkjy_8dJX7!*Rx-KA^X@ zR(6aVa91VW_Z@ZWMXpyWb>zkhUUpiHwk=2|2n;s7(&C+?M`G!uMw4zDlIsas@>t|0 z6xLCD_jXCHJX)=feq}iAb`XJ}=DCcc0m{618gLj-1JJeQfh!OZiGKl~+=-iiB_mUw zh>EEN-sGHEYfm)h%Dri8WdIlXy}|IhI`vXSl~E5niOy|I2&JRi|r0`>E;WyLWt zJ&Jy}Z6B3S+G$~eNbX>%(^M1{3>Y5l3VxZG^+W1X?*$Ch5Qb{P)wu^XONN|}jM$el zQ)Qow@i9cPakOWmQJBibtedq-i!<+`8FbamhnF9sOCR7Fq@PMb z0Yk1kBelk&_PvJe)~QWzJ{M09xJ}fW{s6IPwixu7#)}biL~LYiXxBt*>zXmxCTn-a zZQGpUSOCKmAQ;9o!_UD}Gx%|t2zV-Z3FY#gCJ(zdH32f!N20Y7;YkEK=1ocElE-s> zi5;y;08A%+S3}jW))tkgL{+LtolhPEhLvu<;AK{5)5RLmiCZs%6FuDTT0HV5BmfE-R7Ww7@bEJ93dNpSj3-^bpS9eybTsTX-_GVu8 zNX!NyKq_lbi))GfRZJlp>?}S9A0B);ys;HGz7zWAP%&ZSG6Hei?I~(+CmcN5aStAd z|1e)3@H*Bem!o=!1UUrAcfw1&8oPs(JAjk zC}Mo|lG6?A4`_rXgr!)ySeygyZqG8UAlBn@erf}AMCH)C-$qqAh{dFTu~Z&6IN<%1 zisyr?Q$s>R`)T-j%#WLVt-2u4WrR@p0zEEs$_vM&dg)IQKBR(%749{ZX6SI$L?t1l z3;q6sS1W5NbdYAbo^A2)B%rbIRGet{?+BC5V|Z(CRs)w^571$L+IGVVgxoWxaEKlg$ zo1@nmQi7X)^d zbGtFyVfX@Eod(SKP0j1QSHc1V!}FRY)gF_Hc$6&GAy-lJ zaubv5ofoG*yndL=yOCdDd&bw?2rC~Jmf=#D?AK)L8s?v*&+{{+sE*%#c217re|x^| zo+9h#Hhlzeri(uee!Q{ZXI{q@$`caFnuA{FJI}0Y}`S;H0_|)~#>zczXpwl&GAi@jM&cEyd7cbWXgSixfq2 zi9K2c8qYOJ69bEhdrCSg7j1g`Z}2Lg;@fI5V z6)i*-k62;VnY?Z!m#aOQiUv!w-(E3b6^B+G5HK%qhDmP|e5U=U#2B1M{K@Z8#)ed@X=}c07{b~4kKxK(_rc5*Ogz+&rS<7MIG4Hey*(v36@ z72UlL%P0G@_$h=L$@o2-SSo+Mywd=|k?E5UGr9)(DFqnC4tqET&(}K1z;S*e=gE4@ zlig7Uk%DQ8H+Fei1_BRBjAwt`?_!TcK{T-%VbsrA`lif2@h!QgR-p7pstzAImw8-o z?qW*OsJKg#w{%>ck82VRxk%DvN(2Ts4Aow(C`GA+8tbbD(BBuqBeWXfXgYzl%t;U2 zNmS?3W=h*Fo$2G9hN3}yjI0NiZ?SVa=@pE4*{MFcrF&E{uGE3bY>!gW_#P8pbhT68 zoGjN*3GAZe>q9ReL#>@jf!M{J(AJFp!o#$Dj8yziF2OH<9=+hwrf=W#dFhqW3$a*V zoiYkRq#u>j>0Q@cBPo?`S5xz=4z4}O>hxplTR{$LTXd;+Hs2uk9KSA)i@Qe4#<`s0 zu~EK4Ig0csynl)d%`LPN8M*K1+1t~l%o2qTM#)NnRV)WCL6~6bxC5&#dx5%&r5^I6 z9gypE1yM0p7iyI2K1)TvG{8=1&PaCOXz=)$e$4)6=bwn007?YB*P*8URhmWnrQZv5CH2O+kQ2J}vO` z?l6ZK(u858{gwWCWF*}#LALYLheszp@|Zlw($=1lcIY#hCM85U3jERa$T!GxqPtb^ zb@z~q^joye^QaFm9~PD7eSbG%OMip6J74Ykk@dJsE$*K$)gpb(K0u=@0inE;zXtzF zhM+kzwRi!PhPW^r&QkaETINWhYs?yIJ@+j9@0-)hFpV2ZU69CkJ^aEWt)d;1Jxn&) zJmHC)&r<$OIcgIJ2#&ftgXW>y#(txO>!F!(xoRrys!eyOOnWXUIVTK>It>N~d2%Vk zk7Qbj^;SW5_NS_9s6Gm#G3~w};X>Vm8Ox=nUqjx7=?%G!`t+s6qOe=NIn&h(|rI&o@;znrJhK!&7!dm!(d z1(OpK)wy2ta|*}4Lwq#idcIqoblJ&KIiF87tV+)h;n-^RCD)`XY*B?KAIM&cq)OBZ zp)5J3)Kzy|_4x)zyyKYB|AFJ9+pC7rTIWMVyYHwNsMRNObZgErJsA9_-l_;dWo3w^ z^;=VG(|(rq|xj8=lNWoy~*I*cN! zP^eZ?lyk(KG(L=krmZb8u%ICkl#Y1R-<{6&w(6>hwTF|QeR(~lFqs%#NQl^~`{>wZ zB9gbkg?-7u3KZT=&d2X#<*>1sVVh!SDg9O=#8@q`C{MBoBll~DyTXry*`!qD`fAoA zFKMa{<%QMy4gt0U?veK*qf7f=Tm_LF!TXI4=2Wcm4heNC!Bwd=9Gf`^8e#56@%}fG ztVO+l)b_XptrgXN>)3f0LGwLMwK2XTh{|Sx5O+t|nx3`GF8p5e;kYzOlPZ&x3icgL>?Lpd@@pD%6a#peZk~Ga zU`bJGK2p6iYr+`*{{8CE37IL;f;t=|gs^f{EV#8qGY{0!NvoguryMl|V$0dNh0{Z? z%Ah<*TqDBZnJc9Y?^_)e;QUg)yA@9S(Edt<_AvU~^+O>*zCuKWju9EDo2*1zdbfAK zone(UNu-fkU{1q%TJg4NWL)qFYF}Jh|P&6erpYdV) zr-H+-PU_@dU!Ua>0$pvXI@Ll&z3-fI3UGn@h{JnRXUsi#)>zi=pM!>&kru2yreO|I$XJmM3Z+pR(r9)kp8eEl()~8dd1cq-(y`1OI$%z)J zyD@&+dH)4m^fBudbUd`FX)?N8XEI`mmAs@mQg$`|ADVUkfEK70vH<`Y%JWe2`5MkD zZgYxZ{-yz9lg|>?Qkfz)BdDYuj>Ne*28hYSGeEIQ90SzJmCYt%UnTBesQ3{7H6WEf z+qx+9Y5$H3|LvH`DtbY@5mbr7GfAR&Lc&E&o*p*HB)z*g*ERRBhZ7C%WH|j6Gp9O0 zO}og`Aj_xGY0hvjXV(*PlrfKSdi!!sXhf4=5L1Z6KyOV2S0*u>L_qJWG1&n?TTNPp za)xh;(iI$O8!qY(8NB%sLMU%|M&Ug(NKtF9R5U~>((T=scapct=;_os^SsN~!<~1| z16u5T1f+ldLvE=Rr*7^@TjHCjcara3OiNR-;wDIAnyNT-HB~%?sF3EnZsVP%ZTbA7 zRChtnPr1lA;R8QTUb)oPUB85Jm?wU^!}cH1j&A~jP~#^kv<2f z`fn9jcTUD;IIAM_8jP9gy*0UX`mc$VMjvE|d-3Ts6|3p)p#J(sHkj<+{sRsPk=?~b z2^h+;x{H>sf9Ryz;$UfO(0x&Gy@NlFhy&7j(EF0KN8iIf8`Cz*z&1W*J$XdEuzo=w zxF5=0A%&xzf!d-3kb*!^CF7$I+y4&;2O9ENe-_vs6UKD&ZA)UUBD45wa6FA8NAN7^ zaT!Zpo)`mp|9I8U6?$?gBrwqgt9j5i7e&@`;)2Yg6Jz~|Q>)j{28GaSd#Pefuh8`o zGA8_0f6qvKIS3hLh83e3eONci_FYI}wd<`MXC0DnO`_&|^<;r7&C$|IAj!$^z)NzL z@sgacK$3F|4*2i52PddT>YTBBr@gR9d4b*PZ1DH_G|K=lphNMx^_`GGsRtNRInN8R z9J~f4JJX;&5W8M6TqJK+M@GCwL_}JK&l-i3-k4(8=|%JAR6`jTLgw5gU+mI!I3h^? zY-9$lU{);I3TGQIJcLM0*ji|D*; z`bK_){wVZykKvZ{?)&@iSU2-;%^?8~cs^x&iuYC!xRISPxuaLC!@AQJW3bo6bF! ztcljiX_K$hmTLS?>N6TOVA7A#y|9}3HTgjsc5)2dIave+ z{EE<1fTdu>0F%S>#Esf;sMJv#tIxqPuBlRjpCKh?fpwmUJm2#$U4v<%Pm!Ptj&wXhQ5RVDeLLmO>;I~DbaD1?^z0>G6d9#hxnK#4B)6YdFS_=O}xb5CU{~AI? zE+mz12gN)pBWOOPDcabV2%e((Hu$r!QDi3+OV#j_+Pwjv$Y!^^3{AVtTH@1wIQ*C* zX#~@q(5ptIJFWRly5kI^V`BW6Zd$Y@p`PFRq3;oKcvH7ychTBYG%bzyd+R~xd&zDm zc~Lc!0A;-X9Rlbxhr{y!Ug}{D3ldLUHOO6fp&9?OtHX08rG!c6r$XBhkAzH;!P@JP zFg2C;|II%61s;#(ugxi%ePVxow^fs-E+o1D?pvQ7Iyy( z9yc(u;>irqQQ^o84B3c~#4lkW#|(W}qs=5FnnRsgGdrM*Q;hp)RxLO!)s(M0+NrWz%=q|A_#SD+KB568UbO1r$Dy z_u`V`{`fwBf83I7h;7U(Rkq0yZVr|5Y@*7gDW^BwvfFl@^=0<*uhuje@BQEVUJH;v zr!G_+#on_p^o5g`_#;zz$2>%nvCdzvoS0}3%LjEgs;^0t+rlK5ewam!X0Y4nvPMQ% z`N5Ni7y`udvgy9QUtnt_BB}J4^XM7ul{3>Y;}KR53Yx2WRkh~9NCi)?77tWzuz1zdAGyS2AQUFxAzLuO4dtILuW^m0(eml{7qqT5>qp8LD(n$9xET_meuE zic;U=Qxt^+K_Iq8`bVk^Pi{$QqBd4GAz{{Z8*Uaa6Py=)b0RMa>#q?je2iWWg?qBm z>AWI&^Kw71pTcRpA+o6pnE-H62yFl^2^`>h4FK0!_{+%X;p|$4?BjBNQA*5ST46Lu zA@0WRMtQx_G4a_x@5tkDd$wH>&0yXC_J<3Oj+l}~6kT{1o!fIjJIbJ>I9HnhX2@k6YoeNR*dIk^NfBEN}@LOb7au8ITgyS@n_W;0cRljeNnJU+) z?ddD~wfK2df!ru8DM!8IN#5}@J~JgRh6*}@`ms8x;N&COo_Q~GC|OG-qvlap1~~C` z*L>HXGqVm5$?d=m%vhH(8yVWcn;c~J|Ft&DkXoq3<0z9OaH zt4&RFJY19|x1d}K!F6_I#qiOPelM(Gb7Th!Hk>Gm9XR6a z;kxL6nZK4eVbAC7(>DU36{k-F!~WV2fL5SSiyiiHErI1wec~`2VL;FIRSFeGB}y{= zq0B4R)}5yfs*_x*yW~DUC(Bo?ef_>sa#{+7I2F_bK6OM2EB!z)0XZSz`6buK+IfvSun$c~Ds|wFDVw^4F@+-+)^6 zMqvd~Nv7(Fwb3-QroFf^=^z<$`o^N5MeSaxZ`GYSq3{=z*}%(J(cP@zlu^>S`@t8( z!`gAh>2B-=0>T}=;Yq?XwEv@cLwnW6q5@+9aLLorZ) zC||@B)Ebso9O5cQLUaNWH}5s1hOKO@-;%Y(SI{gVCSuN6<|&8imm`rq4B_x=6zHdA#ikX%9{ zhBD+DehGd-<^eRdD&O>ryTSVT{ST4xlqUwQ6D+~oWq0u7aG48T)z}~NOTpz4QJRF3pY_*W10CBbry=5jD^QiZ}^%-`?@Ix42*H$p^!aL!q z*A=nM^k5Iuh6t<%Y`J@%IFC00^R>7^L14Rz7g>4N;>Z;~BaaXhL$lzHfMV+#I`gf1 zNkZlaDL4Fnv@C^1)F{ zoL>?QE|}rgJ%4k5AsR52f!=XEWQntsEDW6j(lh(3*9kIYF611fcjwaDUJnKw$adDy3fy2 zK^O!U_%*s-k&^89)T8@(F>>JBttH7G_Q0ar0@;;+uh`q!A^&i{_F$!jJ0&1&~kteZA>z%OFNZd zR44(3;Ykvo;xbc5jkc5aft3Ge8=}v%gcI7nUveXe1ln&tBu_NBJAJ+-w1;Y~g*s=l z;ask4AU*ujEdC6AmrQRjCL-xp&3SmH4oH6kYrGDaoQwORvnA->p{M8Zf$;fK9uhHP+}F zD0LQ6SdPPDKDxt%oABN18LEw0hThcbepE;qudp6j%d_sCUbCL|53ulDCrNKIz7ULY zavad06J?ay?;jvZZF*T6H5GxNBh%%k?Hw#QhSU7n?zF^<{@$A-x%R?~hw9GeKDGIQ zjlbRRcE9FXi4-=`ZputoY)NTLw5*w5c6d;VAPA{koF{8`*gJ3*PXoW$pEPUD>v?+f zk^LFqBk?hH=I#UY=I^=s_zC^v`v$S2h3&(ePNUW!K}KeF=gV0f)r??6%XR)xA#B%i zW$~4(Gx4*IqL1nJNrvCc#K1;&W5fa=mht*U6@6G|MNb7CeaaC5zk7t8Z$UzZ_r2TG z0_UOUsDhO534wT>s(KydjY8!I2QFE=V0pK%%C^TVoiEyVljAGwY0Qw8JQC8hw8!!y zz6}iqnjom0U@qAfM7f+M#jGcJ2R&MVG0CY5P=^&PBuWSpcZl6f?V0I>iO6|*q5H%l zZIK3;WlMz{yDZ18P@Nw4ko(_5%;>R_56{JL6?K*R+-B?aGp%=vXATxLBo7)MCDNQy zl=H9|$WX><)7~to;lye0&5e#x!H)w(ZbI_6K5HMM;$ePV;iOm1uWQ!QPLM*8!O1T6 z>abS~l^+X(mM^1ad%olqL6(M_g3cw_sanlk@+%~XurP}^blb-P10O<@sp(JsY~raZ zUIZ81uWQ!Xl@6=m%nI{#P859%%_@3bb@gcp7m6)#O6B@U8um{Fa`6!@A%L^x)VwnUSE;R<0do|<8sz&&#*izLW+ zP~dulfRhT_Qc;B1`F1HQ*-zoblyt&-lKfD9cv3=zC|8i7(udU6+cQF}66on_spmR! z30b}9E0JGnE}1VKdQit}estGaZOm` zX8*Cpi^Ya%RB#Du`O)Z6SuFOPLzh$)j0l*LcEI^6vXStT@XGuZM|YEM#;Ul!=RDEq z8YvGF08T+khk}Cc4zzpC&&%^8Hr0)mt-sEItR8|S1;~>ihmmPqEO$duidaPuAB)~a zoyD8b66;69^ZD!n3dTb4YPy?!WmB97HX|e&wWfhf8)5YikZ5!KzEs+d*nNXrh_+0u zfJL=KO#$phmKQ`Z=AK`l?}9Nin9!27PsGVyx{2kUJH$ z3rM*ilpte8!as^zeWh|7dhJoJktt?v4xecq=k zqpz2Kg0RUa!u?y8ZnPWaVwuTre|*~M@hbDnqWVR;xz1yR2xK`nmJ-7>% zuLY5b*h=2+DG|`DPz@xKuWTH-i4mYHYEwP$J|tPJ4(DvVI>Hb?=5Rmyf5*_IohK%Jq zz1tH6n@1vz)99Uy}{EkGG0O7Z|9>D3m zTNSMTLL%ncmT~`FS%nOdFe9`6bT3{!_j+g;ty~BP3SW; zG%C)Wtruw-4@~vaHca)RslNip0pC7H6ZLZ(1p=ov@ma`+efB}70W>y**Hxx0SeN7# zJnUPlMop?U6AK=mN zuR0{QIi!cdryx`>vgJjWQ6(GZUU~jNPMFGR@!+K)&u_{%AoR47jxbL81xJD3UF7Lo zo{Mg8|E%W9AV_C6SZlZk11N9}4wroJsoJOYItKX$EF&J-J<_%rmASnGt@@x7< z7%IhDC|>w9-R%vORx?1G)BabbWhjiYM8tZ=o{TUaLBs&>i|_O1!R$NO#7>wR>Sta_0v+l|9Nib5nPcVrwaT zo&{x1HJq^~STz!B!tF>z0zX79Odrk<(IRlYV&wUO{OP{beEbP5)hU@ks-&z5082}V z3n`G>Kb3Lp6i{%;-x`uea<4@?L8`n~OM<{_zk3D_Lj!|E9EJv_oG0~P)#Wa*SPwuR zNXS&+${Ce_bLA}CcOLJ`ISX7lBZrk>yG@38ID_Ng%IR=R;x|9%3QQwoDwt_+c0KZI zDjDxivu_J9{W*IKq5lv}* z8+58=>=}@_{_voSd7_?$*C&&@Vp^dy zd6484ynSKbhJ7K|#VEHXX;7n{c$WAydLg4CTlqqKeArIEiIzn9zF?5`LH!EiaTCp( zlYu~Ti+wgVkz0vhvComP4ak=Oo!FcRW?DG!ELJh(K-UZZctd+TVE@Z(*k*e0(!gc= z!w3Tt$kcLD<2SRINIYYT{l02ur;^;=7_eccQ}^hb<=nI1g)mUvifyAm#?nTQIb8jW z=*-lViPw$6chU!%<3@`rIqF}aW!SMwWUVxmK1F2}h~8(>ai-d8=BQa=U44PnDVxXe z3Qg^QeB2HG{WdN&tO$P$m2iLvxNeNl6L zv_H6B#E#E(EZNhnE->>Xe4^L`<3bo3Or{d;D6eGcdGey;=4gI!smMX9YllE7v7BU; zOsOqSTmQ*1P@mvt{4ZrwJEe{6m1yWA?sIW>?K}U-)!^lTlg5B(7`Rfx>Tuiz8xq_P z85urU5yRXU=l)Ke3Uprc;ZC~Xnu9-J5oBI;0#N_7z*c#@h$tBPpy#*u_}jF++iFdR zyjMO5z>ks2&5ZmNp*;tCAoAni2s?u88R($Q(X9;yhK>^_fF>%hK(mn$A97=O8Uwk< zrZ$F75f?}+mR-l&5CSOKeEV6|>-#E?;fHJ@#Ms3u^3I|PGxlTEGA?!lvlV0D{+D|y z5bZ-SQp@P~-}Fy`y?S`us%b}@Actd|Nf_};0N}g;Nog1l(0-kHe+mQnIx6F>08bGe zfQ3)+A5;+3-VjcD!jD{7Z+yHS10#R7C52SgJ3vGiqs5&fM?Xq=3$`` z)Z4Q@s@eW1Dgl(x1AddZ5_%&yFT=1<3(X0SlNzt89o-5WAP04%Dh0x5!>_X7f1)z+curQ zPxW^kK6}&MmqZLWU8I9m?{=~OBKN3)HV)j1CqX?yeV()}QWZ>}n$&3JsIYik3bC>K z=c4|UR*o&v077bkT+rs_Mk%A9F70llKNru*P@GCuT%@g98nw3pT28G7M185h-MdH$ zCiA=#RUO|mx`y^U+uJ=ao++^zF0A>^xI~QQ`ySgjX>&2H6$f2@?pwuE7m&s;mk$SC zA7D(0MJl2UT=!YZ;e=7wP{3SvPKy=xF5I7-m!qiyYQ{axm~7p(?`?H{02NQdU^bG4 z?}`5kVNJr)4Op=?y^Dl@J0+3e_9m0t>w}D8sf{6nX;gfIqkta}+V}(cP>Jbfanm*j z*eUG*mFo2It``bH!i=o8%K=%Ow?Q>Ujsj)hz3pzhdU2|+GeP9v+2;Kh^$LbKa+Q4u zo){oqd%H~!_F@TZ#amEq7bf+YNwsnL#fXe#4?_|p0Xf1Dq?~JVlHEvz*<|!BUDbEl z&jfQLF3g*Dx9?H7PN&5z<6OYZsTqN|B2$Yai&e!`O5~h;h-UP)(W>g6KL&u|;i$I@ zL>;Wp20keU1!+gtB?ZojUQ+lb+X@3J7*1`~G5rWo&6W!szMH!edQES=y4WZ(RB^KXyuT$88_Q=j3Sxn1g-b_pVnU~>lqMs%>fS+c2c3jm^ zAwZCfm18-!@IZwTt^?Q+@Pz6G7vU_}`T%$}z$^{rU@1cv1?qu)5?qY+tmrn5rhku2 zH2AH%Adx%_V^voyv{LY*8V^fh3F)CdN`*MbO6*w}7O*o2+)a+H2%@nBd9f1GeYE4C zIP|IPTfA^ow?Knpxsf2zBwe;L~d{lQ`qL1tGY_1nt~g z=<_~4{lypZ4In5QAp0n{IV=5KGXOd)JcTpphovelZ06lgxgYMWoSZn@)O^}DctL+G zcz*4^1^oD==isB6wq%Fyt^p>k@Y!Wc^)qi4oG>J)#rn19o^}s|;1xu6P*?(r*E2&a zi0GDQ0QGX37%NBh9rP=pzSA|gOwrW()faG2&oAvIXhj2HDU&CR7C)=?sCSQ#>j5QdSv$6pNwxjjb7GPk(jN(s@&sWp7dagDSWJ*oVCOuXn5)u`J=zJ2jasqks) zFUInSy%~URu-Ht!X$}5X)chzfW~o&O?keIYf<|C_#6iONPqwrSsZ`f0mdEjp0SOa0 z?Q<@M<=VbJrrHhb&J2=l+26i>z#l{gj8XFG-N0OW?o62f5uDp#UC;sv242KfuwfwA zOHFZN&tBjbV4md1D4g`zB-hg`6&HL*Yz7X~FkYyL^by^}oJU>2Dd0gdcGkGC--sao zVJ8S*VBp|PbN=rdA#KPC9p~QY$hV!tSmoyFcJ(SgBJ`z4T{b)_L|o+izVj7kpG@*F z9OsX0^V#f#t$+hY4S!6DJwxV&WwYUe@_Ov2Q>VeESCQtXUpj+_c zpdM#6z2*TL@Vbom^B02)DdgK;DNZf#IEv?&0znHW6E6&AwN@2icd_xzAP!a z7I6=PPd1RcsAS%c4|x$pn7fSb_-}^%>wTYoOYRqRZ^^tp}!hM z605<145TIuF@S0;HB+BSL&c=`Tvxu@jWZy(QQQF}YaNwEY-VrsOK-3IA;xTW;MJPf zhVJMz6s`Ueu(O=t222T!(h{eK&;LChWx`slj%+ec_MeY7-5#nN;vNmGZ+;689r+27 z{%tgF_p1pmVbNFZHgOlhZ8s8G@*AH`w#~_kPkk1J8~u46&^&CAuDW@(_Z!Y9e~8%fkr6}l=XU+>82$v8Ny{n^#Ylwvrg zg#w;8+|tl)BUYK>v|}v;q{KEz;<**5%=JxO>z`BuL`Nz-O7%LI&FJ_T#mOHBhwJv(Y|bN(|K@AI!$MYXnrweU1KTf@MXp zj~J0Tag#|DN}KM=Bh7OqFN1%y`*#ei@ilsbzZd37r^5xnNHz@QMPRWC3x8K0a^z7}NVW*u$jgUKg;VUw zI`?~&uL$gBZgP?@1+hzgTf6^S?q;Q%v35Q(mG&n~_P?wuP3d@2tg@{NH|^epQ}DEdlS>};C{2t~^iMuLM7 zEG)RdQ!I2@pcTa_$ex}TBZDzp5SNx?PmK9NrU?svc;?{=MK}A|U1f6eJG*3PTyT+t zX#uro_Z7sR8r_gNcS(D^mW%pWMNfZl-FeG@9MFJ8#}7B*D23O|fDZJ5Wto_Jljx*oB-?d=@ITK-kF zg*!&I9DHaA4su+K*9sTo4WX+2@r+XlR*7aEfFtX?Wx7DO6x+!5RvvG;hNB!cwTPDp z#P`1_2A=EW7bQmoTO}2~<*MVkAzJH`vyGPJI0L-Q^ve1VN}lw6-2eGG`&+ZS!Ygr9B8%|W(i4m$VATwzYn0Kr9gQpdq<8; zro{#3ZJ7Ku>ZY!)Hh&V&brQDw8&dG~p|}LcdtuW%A|xTbu?L0=C+5i8Z#niqoqA~H zdHSYZiyvy(Yl^sMQ)QyLz4(f9`8WP+xGGk;AaIT4fb>Q&od;+cm&zaR#hc6iCnL50 zR%Zf`q#u`id;2a;*B1hG2Dd#b44uS5+GdIYpgy=ehTRYU)|DzI=p zi0TPz$PX5~*ys#Jtp$$UTjA%gHf&m*cwG41Y>vp_kH5>ZbORt!iqd0vqUFU^=~S9o ztTGr%URk!EkGr50{2(c4-ng6y4HBXQgcWdA=nR)hHr?T%FtmYmJIO*2N+2&SNfL_0;YeEfirn$C}-d)QbAL1@; zJ$>X%bY$!uvXtsu@_~6lPf{vs&wNzN>mSgTsMOKKl`6CAh0@i-pdoL=J}J#sjlUcG zNicHXAY*mS^Zitg1pAtM@cew7lG@ZhNhOp0C`g=f)i?2WU|995Iz}&9LxsPo@}B(2 zt0>b;W{a>f0`9jVRF3)@JQc5pceuem;r6N#ouW?J7Z(M5RNFfr*|=C*dKP>t7&G-T zaL8oDgQ4w2Q>XBv|J)8Sv*-DpI=Ckwj6BVG4;Vg0PX4cz(QCI6X=GVk*PK!X} zsLY1fI9lN~j*I#p;RKE2iBhN@<8HCRm^#MiD2eMhtz)7RzTy-hm$?WEe!oaGp0m(< zTLZQ19lZ)B4EK*)1a?KS3tZ}c`vbE7Df0{Pm5j1!_A|Gb2Z#;Xql{Ov*AQiA2>KKg zdK;&c_)<)#neCn#&D2Z+Pcp;^$%hd}Y_b9qs3UtHxKO3VvJ|cHj*?Cj3iOOafoNxX zDuO8oU3A`4WA29iMx3MEs$%b|%hAHKV)HTUqxBW+*CG2qUj^c|U*q!I6=Gtj+_Wax z-)%E6_)hdHR)OE~XuEQ&ftJPe$O50bvS;S}dYp0;lNL6xVt@4wP?3r{?yu+lO5@0H zC)STAYoPSm=D*p#7h=s5lAc?wYz+gv3nUGc>c%~ zvRhj2wYUG4Ta_RjdT0BM)0zQY(BHH}A1^C`YEl<~PYD&8ip`r@WszsmtWj74O0_(+ z<6QPdh2(ol;0DT46vogoQ!BMvca2*1J`=&l{x}zqyM%b^#6I#~(FhC-49Z3PAS>+r*2(_bzSG7TmLaP?fgudI;H5m0XBM( zg-MYRMs}C()~UfNmNbrG8|lAjxZsi!HYqGD@)-6EluW6P%cSkm?v92XcKqi`hFC^( z7qp}d;7q*ePcR0Y?iiRUh&B)A+wVljmgr_OBFeCu=-zqF$=t}-jF6KZ1w)?S81PZ1 zA_$g?Tg=xt*Zf~HT`8@%(>NhhU>iJ&olxbvO!YCNLA`mQ>on;S;W$IU?h0Jhud_;o zbX-2-A~NQh! zRRRUFMQ8p+p)z220qIxlzrh;51Ipd231B6<`uC;V;Iw;yB-X5Q zJ%sy*Tn4U-(<$|s`(;IBbLRBOd&kAC zXZvU|m=AATeTHOMmDF96>Off@Y@)69tkOVDB1hgQ%2BclI0h4to|LA?4ODf}T&?n` z|5?E&f&ld|9tt@^_q9@DD(tC@FqU}nltdvKWPyjvRyFm&qOIFdOB&xT|GgxafHACb zzGb44smLpSHbg7o8gndO)}@Iyvm_!&$RlDlQ>{1EN1H{Jh+nsR$tc=%@G8ekUU@pc zS55nvXs3Joye|7mxRlt|lEYvAR_$!>5YFCsquSZ>wGD+mp=jg2Ed?ZqfDal5tzBM^ zGnZrw2@$6@z9DYfXz=Fmqvm!MwuPmDFu}@vY19{nS^vdTqOAzAK zoJa2d3a>g~3f}{XzLLpEMStg_X?zkQvEiW`rO;@ii90$LM)UjZuQS^0xyW;X-tpah zH7K#Ch4>wVjDzu@9ivkY1LB`3!@)d4to(#KyyEUJgq?R|1`1(;v5Jwo3WL z_hc}0E^qiMp}y#eD-f&uuk=CsFHk?-9EC$p$Jh+A%k2H8CO!@#|6{o-4`Jz3qy8u~ z4ph*r+FGlw?Ss{LJzZoLzHnJAn#UxT+w0aO;>72XxP&=e>9u&LO$Fcc(OBdq`=AIZxuK}#MX}Cx{i*i?ra0; z%docato~z0H6}`>!n{^C28FJv)zmaUYeQablyd`X70z4PT5_Z-kcPJG#K(x0NNPi` z;dn}u*!8r7Q-qApoX=1P^Urgf=HRi;E&V8rS++XsGg!~86gVX68BKRdP7%gz6axcj z8@it^Q3K!B-%CE?7_5K)o@N+u^xt`c{1{0`v83tiQ(m1^Y^)yUN zaU#Aw(KkeEvLD@e@-b-d&RsuJcuIgvVbP#}0BoXZP6*wS_aIXhb4G67(e$H1@7-zT zdY7@t@l#<)pTeXn>0lLH7btb$S+Ha0Ij2+$Hv2tA> zGJ0j7PIUhOTA#uOq<|5VP)XNd z0Y_DOcau9`Ro&ZNQTqKEFJV_c3iucjRc&xY@$*-21>YSBDWE14 zPaR`~&LKF{^_A^Vs^VtFw?{o;c3dc&v0Vnv*zTDt&e#rI)_O8G6?=g1(pL8NJn2!0 zO2|2W{0ydIvHO&hc?sng=sfU-uL9WcX=`)a1N*pU?}=jfKg!>GR+qQwVqV>=uvrlT znv9LI*s1&1{uUemsNb5SQGkZ)_=`5i*8&h#$})?)Rr zY}%sUQ{RI5S<+fDS(&JlJ~F+Sl9{Y=6_qk(yB5vXwPRU2(pAcbWYUOdU6~>EKXmZ9<4c0a_ZIr zSeSSX&fw5uGTw&YXBB|M)L)7-K{S08EM`It$OM=GgH-l^QJ&C6W8~@s+QA51zUkz_ z=SkC2Q(zH@%I@v&t)kHddMj-@t`N+h2q3r;*)qNA{*8&H%cgxO)y+&JEE~>zF&imz zksI&$I8;E{zFFoI5yP7?aE5`ABD%=Qejf89@H(6jL13Q$lwq-ZvtE#48zI+^ zxR_8t7rA9}-AyP&+V|IFblo&$$j1B3J#a#46tE5xX4md1qnOF&sM&5Q(Wi?yTq&lWb zNX*Ra5E&Y2DdOpAbA;wFGr#Uy=DgSA7eYlUDEW!mZkfUNJTEr`5VEtCh3x5(gRuz? z(8cT50&0z$2js0aVs~X`;9LW98`-VdSsKK`1<$s)ciu4iMs5fj`n_TO4#~B;)w*w` zRM1lM^p;_k&+8tLdEz2yi5AFpQZ3?%ghe{~z`f6SH71=S9rU5NeYBB@h$5ei3#tZm z@OfaE4@KEY%V5n9tT*C5vE%E1tI*o2DswxPSlC#u+jE4E34=Oodhp~CD&DEDF6P+Db?Oz^J(nl|u4DE>N&ozXCG$Y6mZkbRectPnb;C{` zH`T@YzEW@!Bs?_UkU(w9*6dyYzMPj;9Nvf%r+ympK_K8Wrf_P{b;5fWcR1Lwd&IzS zUcp2ePZPVv4j+sws{23fy?0bo+xzYtQBgo}CkRNf&?Pjf0tzYu(vc3K2}lb?sY*~p zMd>KL_ny#u6_g@fO6Z_c1u3EeB1P^BvI2X5aqs=T=iEEaIOCT;9F8TDWUjg9n(KYu z=lO7dQnH8#MNAzxGHZ0BL`n|}z?aIoU(w@l?+8y*_6C-TKcP(iV$y*?e&bUlGXHzy zlQ=RT%H}8&aWu#BJ=d~v^oVQ*)*%4AH~zw{S@XfM#d%r4Bqnky{tUlzIFHX}!kWKQ z_C#iK-N|LbDQXxC9jB&wm5t7V0GQ9{S-%tcFnu20M11Uiti0{A6u)pf700n*k}vx6 za;Xy)SMx!)o|y^Az@eH9DasGUY5oZQ)5ihLe;%j#uM-jNZfJjZ36O%~`B$FZMSnk= zT_xo2AR8cd-Xt0y=a|b*plzCkosUUbBytwM*V@h@ucQue?J&7syWFa8mfVK7<|QiF zyHx`|G^DdUSR73>MCN30cd3IdL05e3l;v-C0ct3XbJnm$AH3+C9jrLA2s{t>%+V-OfroqI=z6Pvg>?1mkF*co zn~QI4YI*hKhHmZHHBBSrMadagpXLdCKRlp@|6-i^k>=+HYT&fNkH7*Y(rbuf?&I9l zds?&WUX>vYF0vOiG}S;Lj@dvxC~Y;RXqWW{ue;$tr*mFF!qsLbGO=_iFE(n5o2DNa z*lld0mS(DDG^jwn>Om4v^@{4cMZInNaRE|e&Fk!o(Db2y(xU%T0(f9-e#4r=-c4T0 zYa3Dp<=%!=;ca}zyWpW5@OycHqNE{2KoI$au)S;bP^Ec5M6_0Ttk;9tyPAf@Kokqc z4!%skwhbEjGu`g(CD6S@u+0^z+5N5ojQY=mh5hN%uX?Z%0+!21sPdJy!s+FZ46x=# zpado!8e++VjtynRwcKl1dU&+cgAkLu-mF~@YI_HdfG54bu_JyGcI1p2qeQT;cP`%@ zC`HKyg*Auw$(TJB*r(v{)d=|ya+`w%ei?9s@Xn~dO;u-38|BHlUn$|)TcLs`cLm*K zRS;cDL12!+KSIKT!rWWC6%I}b)LKFi!2&hG1CmnHz4KF!By8%$F*b_g9PSvvvJ9k3YU+DdTWHneL}cr^Us>We z9jN!$wrR&eoZ2A+aE!sOgx|39@P^K|%->~Us)3~~0}Kdvn^Nzh*}QD6FfToB%Hk=gGnehB&1S2TWsC$xw_rg7@YV z2`bI-sz*r&hMW}c7Y?f+W3~w{pFgL)v7>p0tnAk7_VLBwO-Xy~YJc}*t?>Gw#2Xpp z>PX$lS201)@^03(3-{|rhED2)&9kI}2@{9qK*6#szsPykM03nXlJ=>L=?ImepVR}E z5xaLoVZA9Ruqk*K4tBOouAmT2Vlj(jSWS98pDF7bc{Xo3uQCRfRnJj4gGy&t7=DX< z2n*ScSo#$JmnOg4`#|`*P4-G`p~DZERO2T!7qwR2WuV~83AUhSr?%@O%gIt_kZR}> zde{EyLqS=?>vr~#Er?{_yD*WZW%^^F2$X9Fa+^qeSVEs+^c=6kgV<`495+4fpRVeN7vIsop=qAb>7KLWGWWc7|9$&JfHMp zeR@-G{gO~leOviYq$N8HzX3c43g^!d<%RRUm=uNKt={Xb4u05gyT>pMYRrcH4Rr#j<;U}Ni5u5RbhPvav2 zS#I&|6qbn9AV6Bw10bFjBUIo>E$KrmHf7WYg4s-#6M;G^nnz36Fh^?_G$% zZ#iMHf5~h8djs5&*QSaZw~rHtJ~?xE)&50ipF#HQ`tj0(B!9`TvyC)<6Biij6%`&8 zhEpF_J$owh)Q*RdcNO0Xa~o;U6}t1G)=Eg;@+7t_3#scFTgL;|#_4v-$dAI0iR~Dm z-t1J7Kiu5uGxERB9i^w5R-YdteJ3qN|GQijO>YAdoi(Bve12CQn%V0LFKwQiz$RJT zr!?h1CPxKa$;CfyD!ku=`dVZ2TyiU}tgp|%UDo&BF6-;>lxp2d?F@KM<5a@%PR7gb zqC66IWj1A^Ncp3J3dXO}v>hMa@FUZ>a1_Uy@^$QEi3(GwD*@qQY;ybiS0Y_^G|twF z{g^$}!#nwvu?+hqfPVKzTB2hve5kOUVirswrcIOtT9RqP%G%v0Bov3->haxH9O1WK zUZS?t3SeQp=H5=WRQH`xW%G0q!MqqbuU)~tduoLzSDnim&1WmP#dQBt!mrr?1-7QJ zut%K7FK`mlQ5v|Ow3bTQ^fU&Qjayfq6Plj3)8`k;d^sm?C(J#6MeHfm@>9r3oaRR% zZsBWH8<1sX#=zhL`}+Nyi=jfgBw*)xk!-5}MqXI?7w^gqyKNi0xA5-fY%kvDXZQ$= z#Ec=dpTs|;iP`?%K1atJn!DcLG#_-Z<&zOql&r4b+ z7Ew^;FERl!hdU9JF*Kiw_=vY4Le0p{^}EsIGx)LXb2*`mi;ZdO>wXaRjzN&9)7o~k@ z^6?Eg^Wm-OYOE=^lvJS8u$`>F7Vxo#y?dFgat5S&vHuIB8ozV^ql5#F`!!kBZubuT zvgZ=cs%QXaAkN~AU0lOQu8ZTCu*fW{>#u>u;YI9}X#Qu7GK^i~$R|~nQyPz$NB zxZll2RrU;K{9wayt32Wq=BDA7xhwEuwC=iai&L_MPS2I((jELAv-XAX=c+|S+$w%B z8$!c{wl{Wl)-#SRcGHz!pp{Jy2+gI50qSW9giM)qwL{VVJ;t|n3iZn&#b9zN-EYmE z_)84U8^m)u!lFw*iiRnfjO>l+Mk0L^wc5vP2OU~hoJ;(^)UStr4O zzdNVF!3MiOVnMl+4zwm(-5S(Ctk}v?FMCY5bc24rU0#4eWJ02hTc>wSdnYv2mb@wR z9zPHU0EZ?tH;f*4vbh-GUp~=oH&Yr4u2qMRTaMN~v4r0l<;mRk5~@r{tk9=k5PZ$Z z03b=Fl`Q*bJmA6@6x=ED=HC@w4r*%yTrurf{zz%+4ee?GE9Te-=o422O+Z4B59qfU zBe%3rEpEsS(GfoQOooZ&bvt7~oDA?3@7OYXh5ds})sDoHfF}`AABwOdO_!*$upBFJ zrrDNH+yE2IZgbRb)f{5UE&yb%(R1SXW;xrDYvr5LRpypp;hUfezo*T$wuU?P_rfq{ zXYVvIx%X?=+40RVe@|&Idthz>Dj4yMthha{%!8dzm9z{9I*bALvK)UdN7Yo(KTs9L z;{7pV9Lg?e_>JkX*~yR_lsQxt3R@8Rryb4 zV=}d00&(E601rENOrj&~eI{meXK>;f&kpo*Y-&7v5sFiL`aY~#JAT)5P0=qDa63?z zqKz2A=~K8QKG|07hScDB1B(BK;HX93AK>in8E$~ial+WtCKM*gT3JMnG$6kX9ZzivYdO#*Z zA_|AbYZVC3MO#SfET2T907crjwOEn z`Y(U%ptck+l4JGT#?F!LfqXF7DH1&J2!AF|DA<7==75RrZ_LGerLQ`vciO@juS_05jO>+!jW#26b&(AUE&?6R& zNyv0>HMiX+SbZ*^1n5fd-43x}03H=r1^`NtviKZ^%4@fw4N=9&c<;$Oe01~R7mQc> z#B$2VX}l^Y-?$Tk>3ISl2a!E*JaKN+(Z$nwYB6F-%~ba z<_Y2sKLNkG`##vJV^{t?aE#**iMf%AfIH!5i529E(!7vsO-3WicCjD=Qkf*vl9>tchmTcnpKkT3!%;ZjOv5K=2DG)#TK} z*0UAcyEMU(CSLu|iAN*`DtW1o>pPML25B0%oql~1^Q_h@^b13IZ`)bV3-IL&aLAGlrojk6DONOokf3Bz4;kH7HbFsrbS5GIN2 zl%`9K+|~ai9C7BVMgLWCBC+~kKql?~3u$Pn_5IA#eJ=A5(y`)F0D<^HAO-LOJ{9YC zfhnE8ihjfum$vCvlgz6S0~J;1r?ySvJY)2af?~Wsh@7@m_%=V3f=FWt1B% zAORN0xjANVus@jLd(Z+96@bgEOPj?c{?&)o+^HUiu?oFScZc0ATad~=OUmBMW>4VR zP7}-P|I7e9cnN=6onPh^FF0+(%AzZ`>1pcQ^faOw(8Mdp4_-m>>i9MdS=~@=43KDN zai0JHf${PAkdyb^uCnZlz|{k`X$o&6WBUNIw>C}KqB9`y&v-uCIh|1`9@ z&{oOq?y8n zPe33_|8Je2qyk{%L2dkem;}6P;OBL*f9TB*o5$n__Tqegrhudl`25JXeSX(LWz#L= zz`rBR{hcjtU)3C76SDRydLasYF5n1c1q%TEIw}DwD*dv9{j7fFj*8>I6LMCQ_y}*5 z0jshE-iK#F$PiehCq74&vjZ&}-}R#?eho9TP^0Yb4dckQZvlFp>~kte2m1z-|M(Y| z(1Z4VpNp4;f&6(%1Dqkq$#Dj?!Vq_;X+BJ{ime5pox#li)Qmlx$iTpWyf%4}jlflr zKe?Odm2Z#n5GrN*CH7$C?jHUoSiQFR8};QZB@uYRNxuO@(?Wk^!MUk%7S9x8r;G2p zWsVEI>#)CJPl_|a;XF@s1m|&&5~MNv0nfz8{(cz$^Q`Pr+FmEC+%nGx2*XDNj=-wP zn4LmaattFuG_W5|7}%R5_}AnIM%!&Q;9E@aqsy!`+o&;Q-h7qHk^%BS+wP*73-Z|b zECdz3q@sV}*oaFgZQB9A68upGD3-D9^&k+mH7y4vYGjVJVNEDGtkxrZgr=q?lvKD2 zi5o!!(NjQE)oc=zHK8E6QM8-yz@ws!Jfeb*mS(OOLkxiIuLy z*|Zt%0&P!&V*-~q3%-Yk_^?VOsIZ{1WdctwvXUCT@`|?4Efe5uF5w4;x&t41a}_!>1T{1J$k5g%jTx7+D_dx<;?}RbaZU=~q<6IW z7LW<@E_m`wTxHPq<#dI=7ms@&dxGbU>`skx#oIuRwK)zyM=L-JOo$GQj@v6+ys|_k9Hj_@h2p6&}uhP#Gdbgzxa`{x7 ze$C~p?aJ&F65i`*)yTfJ@3#euWiyn0kTZA2uq?4}Tr7c%lgt5;2aT;vlj%Z z_@yjrhj;LB=xPI>=m+2>0c9b;`gZN>{<0Hf-^{23-FI-|^r!uAk9#K~GZX%{Q8(-eF(VyA6zM6s5dReTQ~v@fWHD{AYq=vKyD$OrU3Vx zm3cxrD zDrqjzmc}#u$!=IviQh87E^^+8jVfFkaLyR;>@boS$nk)&NB1$Wg;3b2alONv%@%cq z_g7cgKJ2b#blsf70&G5yTN@tv#6)>!RTqzc-jO5j!9WyY{DlB~@Jr1#6U5O7E`|l`PpzLvq0fM&+ z`53e0edx|CuSu8kV5fB#9VlIp5}6HBXh|Kj!%KoB_J2tlb#AL63jpQM)?RyBVB6z> z#fg$`3=+4_*?3XRZ>z>S0px}C;Lq&_t{;&Gg%hE{h(eP~PM?KK515;1{^%@h=lH1| zr4_z z+))-oPUFcV((|``Cwek6Kgy z{<%f&CMeQ@5so4f2ljPKf(WWwFxWHs$}Z6EuD0+GVI59|FrXCqgoci=XmLzJstkio z$Dm+*-~{^pZ0tmykgXQi|FOqze}g%j%nuzFgh{FUbY&I85l+4H6A;Fs1C#-kz5 z1WrR!G81dVpTNjh<@!hlvvjhRRDvLog!!j}36#BRW8wi3E~m~s$~S>*O?A2rGlq<{WV50l)iMQ=8=65I-zJ9vO>~pX2?qGPIziPE0d}tFOT?Prp}Ayd>c;#a z`L>3~la@m$&X$!^9N!OFKf!l3=} z285~_&7auLOYH7t!g-}Giqut!^>Cd%{Ku~A%?}QTC(d3=9l=vT4&L5RWR&9lTWRrs zmy_%iH#3j~t9W7<^X-MP0Nr*FY+6r~i#mCFSN}_ou8InU{u63)2a%MFA%M{fh?GXO zO*OanV*{v|o>ccG&yhamWw$7TnXnc@|$Kya(mj(^9Y z1$ch^)j%454|0IDX<0+|xrK^N5-r~m-j%V}m=v1|WpglDa}RS4%&7&yc-#;F754An zYGVIr2MdXDVd3b3(OP?) z0?4njm}9N^r?4Jkyx;ezf+idPCtSWChi~_`N#+;33u>1oXJ;tuFcm{HRGG7vpYZBW zzO>7ra6q}eN$`k=i|u6kHK4n+WM)O>Mge2=7i~XkoEoQ(U7_*<+qwxtm?$*>G0rhU zMjI!_(P_6Vja36;9OEQtaXol^C$7>q|3!99({MFZExs<&KU?cZlS}UK{Lx4ooh3?T zP`vzayP~mjcuY|7X$q5?+l}35c12-Rcy+tKt$-ePGtt< zbZlwtN7k&{m)s!lwVLNhM0H2Moi~!iolmk^K)+lkJ-Q420=#@#!8jyTerNnP63PNV zLWT9;Fabx5po1Yaa721wFbxs`+!{9qCF+lp0u~j^Ic9vLi5pPlmmTP?ruGCy|Yw;-e%fHDI(bU2h5E;$rH!9@>(gb&iLlzFvTxY=F@-`F5T?S4>? zlW*9oA&;Fad2IN9`!U=}*#EUxn%}z&F^3{Y6bA+kAYlM%qIK*StEar;XgI-Z(5FJT zBa}^|_9b?4aQ#&|(u^k|0kw)fjuA=Zyj2g5sT2w@*#M>zpjqIUN*xgVv)7)EjYnX5 zjVusHwoHrJ3AZG8WuyMd8Y#e*9|>EqFNq?iR6$dS3`e_b6KU`ou^VC{v#X%4UD==^ z5rD*F2N5S|CFMSlLlebuE;3JG^Rm1Wb@Q+bXa$nyNCCiDB)q@CSVA~3*8848X*U8^ z3oDM0MD(2>UQM3dZONB7B+4hMa|%Nt?068=ZJ8G)Tw$E_L<_(0OTFpO080q&^-sBd z-!5`g4k}q0G^>BG&f-J~;NHjc{OD!D7mIlsvw=TfcB?=r^^UM$*HXw7T{$*d&hQYO z&O!=KsIQaaty!!fo)!OzfF=k&3x$oWVXZ5~-$A<=C^J$1L4YTNF;S{C$!=q(!aH%; zsXB#|w?EA@T{%ErSWGFqogxUJ%cip4zenly4Jz&3AwwYljf3a$t;BF|pI_0Y zfU{=JtA=T}1 zoc37>HQd8Z>KKC0Szf$0`gPH;-qkVl^|DSjK;1eDQt_P6f!3)TTa(`Sj z!{uOFGciv9C_?`J@L0@)*@v2j@!+bkn>u+gDV^^#`o_e^u{;28$b>2?5BzqiOKWFa z`0HZW@6UtUhypBn#mvvd{)ex2^`K~}t=>Zc)eO>b0-mPj-~od=$<1p)vuyd*o<3_{ zLjS)VJV+cQ6ObgcC~I}B^_c92(C|Eajx0JSy$y17W8SME=v#p4)XYHRm*{Mlf-kV; zx$C<3g~jrt*jGKP(}Ua?>Eg5%LN+7}un&^wB%=bNpI%^BY_36JI<==hmnTt@<2{U?y=I#R9 z4`rv{n2m7)P8`QNcgT4(HiM=$?oh^+w^!=?YyH}Anp9$>KKSLCK9K6syj3xmqqb0q z@LlfTf|!2?EHL&VDC{7FFf_~LoMS{gl}f#u$5V_Y0qvW}+57Le@D$%CWJYy{$p zCz6vd4Ikx{$}2$+$6%7z-SC6+KY#h35%`}G*lz?5pd8%@7Wey?|4)u>xC}P1znic` zgrU*X5|^d33erXBgzWx^akAy9tH5C6ma&eDt2trOqjZ%&wZ}%CHAMA$>Rm>Nqn1ua z@}VR3`ePGCDV>mfIX-;Uvp>zyQ7#<$%BGCXO4oo;A|yM{a>^R{P}+i^{fZd*=lkv^ zMq3a&mf+RdZ=(K}HqOw-EzQ0C`c?SOw+;Wx*szDM1vu5>LhMA24Qq^i($tST+#z>P zBP1xts&Gt@&atD9RbV~XySQsag^=zUq4Y)Og&2 z(CWrdgCks9kg);6{I_b939=qAhW8Y4SAX^>8N41hb#;j&sSO!-jf)>`w;vZ)$|X2s z#U9*`P3H_k%ZV{4h^tF04`QBeii9bL20<*QX2~MLwjfL66KFnCJL6|TME2TtGaXfT zPhJx1x6(ed(O8EKhdLvdk*DGVQq?WGI$*Y^vmAstwe0f+`!R7$X=t`|o;}&E}3D)xod+{p0Ji0q2o@q=JF>Pzh z-HFT|eU<`wg}##h=2=~;Vr>YDbHRSpjE7adFJO>gM?w7wAGFBJT!z4KkxN@bSL*?) z5qa}yCMSn)f!849<~h~VvQJ}oguJQN_fumxz1YeLIF*IxkiZKfI~9Y^b5}! ze|<`p)DXFx+0O3)I=PY$`0j3wWIdepNVVC5R21cG%XPE~9E; z9ug8wS?0dwnRUP9xnXMRbKUl2wZtS83kF6|gFBLQ8+`<+g|n+ z*ETCoH^$|=II=72%=XxFB==_Ygi1URG!yHW^<1t2phsEePe{!k(`sf8Hp`#1HD#V5 zlg^7Gq}9r!Xtyt7DKO(I&S!B9l97GlLM6DWHJm=%HQV_B(LnS5F*n-`|Ja#Rjf9uhCHnCnq_F2LN7-Y5Aiy^l%A%f2WLlerN;Br5^u$b`qp z(kNVe=Icr++BcFwxgoMpS{fOCP?Fn8%*`QI$Yg zeQ4D4n3oI34;$$@4leX$diOs*tXr6C#+rOa>50nIi>y>axp7>xaik6+D`_zeYlkr* zey8bJy3tu${!)qM>dLU?3fZ+6rIGKS7|5dq6L>Bk&IS5QPTfSK-+L)@SN8Q=qNWm+daW#Zr8;Q-hOVL{GVJsyi)>v7A_{96=*C^)3UwzG>k6C8#Jf zJflUrAsL%;h8O)}golKHvkA4hcWmKY#qrxDqw*(Ep?Copc21Y8qJ~(qz-y-u_2&1D&@;q^6NsEbh%+^tz zsP7g;XZZ_OB4x#Gzvh{I)L+qaNsud>*Xg7ZPp!ITy%LCEMn6$a%lsB!OdhZBO6#~z zdA0<$da%kXX>TTZR|tG}S*IRR?V^Q+lTbeNo~78se`QiXcoG(*9w<4KZ* zJz9J`v|6F=FIdhHOa@EVd-P>CzR2KBGLBxG*bu%eFT!;Axfi{~bz#_N>!yeOx(c!f z4nDUCB@OamT62&3WUKi&S=ye|j`4Pd(4=JNlIvIXt6oZD{VIhs{q|O&Mm%qCNub{M zmN@zf7lz+iC0{`@PO;Jrg?pXq7_v*uSyI`8JUZ;yA4WPIHeETo z`krFECFJEt#VyE-ukEV7ar0cc*><(jW=qP5jKPuxaSd~h`$;J?a?~^SNfj`c@}MuP zQQzMFxSs|DYMyymH-2yRUh^LQh>%Fyo*4)@tsVGzXXluUwZsRdl#-kI`4F(_6ewlmSG(%sgR zaaQh`Y!06NpC+CS<&^o@a`|u)h$J>VB4bERae>;bdP{k9WB3ZIR~TFaW_lt94c4KT zWh%yga4*{ZB)?0>9k>NK!aaK7&h&ip(x=vW?oH3QaFg*S=( z+7agpE*ZdL4s>+#eSZ2h-4kQSP@$18fHYUVlkvh7+2b5BogMM4-Xq~f0L7pU61AE9 zDp*#PDTbx_ez{!vqyzu6Fp+u9Ww-G`!M+z;kk$g{GwDh)?L#eM`Sy2MpnlK23?;6Q zEK%t-eC$LKHU%$sbhfK9Spef$EUs~)T*AuH?wWPqaDAzG@k-#~Q?mq|Movs<=J{t? z{?q0QRftB%Xb)2(RZ6B!4W^@SeV1Roj;@y)>qIfkmWJv$Ppb&JaxMSg7!1#tX zQfzA_ub0uJ`y^0)*7kdyS&};NcIqAX*v4gz;dl1JJ&t|Dw^QbsDX^us@bjb6wf8mi za*c}U!o$=3v0fQZ&O-Mf8+}r0akVbLn$w59|9&>xa?G(A*v;h z#(P3ZuW4S8CHuKXl2SPnjl?}}4Hnj7T8a`>4)$3`%N59hQ$>ZS6pO&Fud1+)NyLWj z05#PN8qTgNZlQqUhcU__1M^_P`Ab5a_swS?eb!9UpjGEb8!WEN!4g{|G8E($)YI;~ zZRrfrhBw`l_47GpBe$@eg!*};xusnyzGwCiv|Ug^fwBh47)Rl;DJphdfDZQYL9?{g zhF@ZU2Rl3L6fZ)Ym_dZ@lA`BAl}L_y*G;*%N=-u-uhp|9c9xp6mdtOI@Iy`DMGWjJAL9LG*ySx-v71# z%hSQt`HQ|$M|0z~eo($-hqhlYyKRShhrXZbu4Ri<|KV%>lKE<@O^o>n(Ujq$eC!s) z_1YV!Z=*bl30JZQ`j>=mC1%QU+e7O)tV2(Js>1NNKVDTiFVjEOKsTzG1PX}rX?a&)4b%UaqpI4Q{6~wZMf%q} zQ^*iD&pbG}e?7I%f#!bepFU6C^tl2(1v;|37S59ivgkv=8DM@)rH0Y8q_LQwJ*V#FmcX%Vj zTOJMyKB=zwsdnksw}e2~C8Zz3F8S|Q{Ng_^Y(YYfQ(v04imM|3d6(e4wH`whv5f~Y zB8H7b^v1L2Qdcc)c?1%NBUSrd3JaU>r%1Ww+wpdNc9c4`@RTT1U838`o!8Rxbyg9f z0^0ipEu|@gBXggC>p4^r!jjUldJ7`B1$lg6^8L?`d|QxD)x)Yi2{R^dS&f5M$|YSx%k7*6Q2$VI(Kiv{{YH zdyBC~`rO*@4)s(>9TT#3LbaUldQ%XXlaCSiSTaoJ;6$ZHTcp9ZAU_MkSETY_HtI{w z9vvBtuI+rMvZf=UGkseSJ(H?rX$E*PZFKLL9RvD8PQO$FLb$X@M#NF^ZC!}fP|wNf z0K`&NhWC2*vN_WPveO~E?crF%79`*h`=nKtn#d*o7h#@$vHo86!#%Z!F-bJTkq=O? zZ)5=$nN?&j0>2j68kcd~zYj-=@Jsfh(UI8q>a*C$Tl_w!nS?q=Tnt~kd#*lL)5(^Y zuOhv6Q^J3#P_U>qG%R#Yr3+b7Or;vL`Ei+b3*t^%6=^N=RhbpuNf(g%KE~>B&w<#~ zw1LbYgVHDNW*#l_>JLm0E2b&Fn2=PyY{@e%b6GQk)5+GTrTK$~O!K=87b=~1u|)=& zTac%sL%&(@j|?n%4KzBLGV>Uz!Sl{a-8ai7J23^nK`m^fHJ7IvDs_8!6@SRQt90`{c!(#1al#(h~_~jy$a(iR349LZhuxqk83is*5JX9r!TRYANS*d zz8K}J0@AaWYXNDHY?XH>jc#~`wx3a+&>?rNn#{SVEJ)x zW=StYRYwz>e(Q0|y%Db#-$`DKNDLeqL9KntJ5zm)2P)=MJ)8cD)APjug?FQ`Wn_70 z6N)X%%gh3L%B#y-atp#9ja_Akt@in9C;wGx_3Qm=zlusYWF{;#idj`cuuMNauFG!i zC*j}|vvH+*D~nJOIyC2+1N)}e+)7{*>J+bbe}XI@45_I<`;w*ZG)u-R{9)6`$85;fZMGnGC7$o}LW50%&P7uW)Bd%9tPxe3SBB z;aln2(1PdTkNc&4y@Pyh_-D%;5(MTiL!qz<3BXK2%9E5INDf;H%TGD?B1z1MNp~nD z=e57%TK842h^NgObVKLAD10}R%V`gM(tteeA{u3CPeyl_`MaY5mj4xfo+pOX7l}p% zj`2JjasBGE1+pxiEj7j`@3Sbc9zc! zs(5;aQv`eqFW*FvdJ@oUJkip*7hvAR)%OJ{(8+?B8yTptg6eE?8kkHyAoCI+X4*i` zGMO^8Nm;2$TWboG2+k`i9C9P&Nx-yff8t9JL`jj@r=y2q;melT*ls0m9)dDUHtyu6 zdJjfLE6tY-ue7YHnjD+vUyfN3hC_7>DKP{oQa+579K!11()K6M*m1E6Kq*9B^VN)6 z8?Q!2hCM%}zi@2wzR=xGI@dHfs)v7MP&zXnLcm+>UGPxSjB3+#fvM(bl%)WRf|y0I zh#|{qtYspT)R>xr%!5Hchs++kQz=oIt=eVj*6VTq!j{N+NRx@e!pF6ldh>x(Iy zchAIMGq>kqlt2&mcXf{Qvd)>}o$Kl`X|JDf%LjPz3G<^HyvO-TLqX&&mcUX2H6mRE{NjAVpN+_JJSg0P8edWk= zY?NP?uAAvxaEThcgE~sWnA_d~jV_zT$Yy63%+R!`vFG+0b*P;pGuAx{`))efH~*1) zQ(sPpjP~^RJU`3Wk}_p3O|C3avxHQJe8-tJiD9^P>XTH)fETO@f-8~+DsYw*;4-x+ z!l+D|GTH{;vQLtetFz))U*oJ280nkef_xYh_Cqmk9mxl;ok-nT1=^;{i-cxf|? z+mX}-%-Zu?YOcLWI>d35Cg_=?s(ub#dMHL4osgzYtYcps$@L42BEi%As~yH~GqD$1 zrgZ%^1SlxlJ2OgeR|W~EPhoUD(=+T6*IJzs4!%LrGs436$Q}wCid(9?*oY@FZI|c% P#l7wSmLGYzHSqrcWYozo literal 0 HcmV?d00001 From 73992a660cbcd140ac5e8d44533ee01ff09f1b6a Mon Sep 17 00:00:00 2001 From: jiangbo <245730400@qq.com> Date: Thu, 19 Mar 2020 16:08:02 +0800 Subject: [PATCH 021/136] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 22d2c0d661..2c8850b470 100644 --- a/README.md +++ b/README.md @@ -359,7 +359,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 ## 7.交流群
- +
## 8.招聘信息 From 533067f192313117fb5093fc94500fde1b78cd90 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 23 Mar 2020 10:50:38 +0800 Subject: [PATCH 022/136] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2c8850b470..d27424d27c 100644 --- a/README.md +++ b/README.md @@ -357,7 +357,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 | 1.5.x | 1.5.4 | | 1.8.x | 1.8.1 | -## 7.交流群 +## 7.钉钉交流群(可以搜索群号30537511,也可以扫描下面的二维码)
From 13aac4839df9422c1bce0b8cccc6632cce6a6ded Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 23 Mar 2020 11:07:22 +0800 Subject: [PATCH 023/136] Update README.md --- README.md | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index d27424d27c..c2730838d3 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,11 @@ -# FlinkX +## 7.技术交流 + * 招聘大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com + + * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群 +
+ +
+ ## 1 什么是FlinkX @@ -356,12 +363,3 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 | ----- | ------- | | 1.5.x | 1.5.4 | | 1.8.x | 1.8.1 | - -## 7.钉钉交流群(可以搜索群号30537511,也可以扫描下面的二维码) -
- -
- -## 8.招聘信息 - - 1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 From 41c46b408a4877da69d7690f09bb9beec95c85aa Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 23 Mar 2020 11:19:08 +0800 Subject: [PATCH 024/136] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c2730838d3..cf0383f614 100644 --- a/README.md +++ b/README.md @@ -3,7 +3,7 @@ * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群
- +
From 0c21a5df654146ad7be54d8f9bfce2cf215ba304 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 23 Mar 2020 11:19:59 +0800 Subject: [PATCH 025/136] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index cf0383f614..9ca081bb2c 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -## 7.技术交流 +## 技术交流 * 招聘大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群 From b002162cb8364929e8b04d91e8b330aac0df57f7 Mon Sep 17 00:00:00 2001 From: pierrexiong Date: Wed, 25 Mar 2020 13:48:56 +0800 Subject: [PATCH 026/136] A simple pulsar writer --- docs/pulsarwriter.md | 76 +++++++++++++ flinkx-pulsar/flinkx-pulsar-writer/pom.xml | 72 ++++++++++++ .../flinkx/pulsar/writer/Constants.java | 13 +++ .../pulsar/writer/PulsarOutputFormat.java | 104 ++++++++++++++++++ .../writer/PulsarOutputFormatBuilder.java | 39 +++++++ .../flinkx/pulsar/writer/PulsarWriter.java | 43 ++++++++ .../flinkx/pulsar/writer/decoder/IDecode.java | 38 +++++++ .../pulsar/writer/decoder/JsonDecoder.java | 54 +++++++++ .../pulsar/writer/decoder/PlainDecoder.java | 36 ++++++ flinkx-pulsar/pom.xml | 38 +++++++ pom.xml | 19 ++-- 11 files changed, 523 insertions(+), 9 deletions(-) create mode 100644 docs/pulsarwriter.md create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/pom.xml create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java create mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java create mode 100644 flinkx-pulsar/pom.xml diff --git a/docs/pulsarwriter.md b/docs/pulsarwriter.md new file mode 100644 index 0000000000..413e1e0f30 --- /dev/null +++ b/docs/pulsarwriter.md @@ -0,0 +1,76 @@ +# Pulsar写入插件(**writer) + +## 1. 配置样例 + +```json +{ + "job": { + "content": [ + { + "reader": { + + }, + "writer": { + "parameter": { + "producerSettings" : { + "producerName":"test-producer" + }, + "topic" : "pulsar_test", + "pulsarServiceUrl" : "pulsar://127.0.0.1:6650" + + }, + "name": "pulsarwriter" + } + } + ], + "setting": { + "errorLimit": { + "record": 1 + }, + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,pulsarwriter。 + + * 必选:是 + + * 默认值:无 + +* **topic** + + * 描述:topic。 + + * 必选:是 + + * 默认值:无 + + +* **pulsarServiceUrl** + + * 描述:pulsar地址列表 + + * 必选:是 + + * 默认值:无 + + + +* **producerSettings** + + * 描述:pulsar生产者配置 + + * 必选:是 + + * 默认值:无 + +参考: https://pulsar.apache.org/docs/en/client-libraries-java/#configure-producer \ No newline at end of file diff --git a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml new file mode 100644 index 0000000000..72c8493094 --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml @@ -0,0 +1,72 @@ + + + + flinkx-pulsar + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-pulsar-writer + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java new file mode 100644 index 0000000000..b12d1f8ece --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java @@ -0,0 +1,13 @@ +package com.dtstack.flinkx.pulsar.writer; + +/** + * @author: pierre + * @create: 2020/3/21 + */ +public class Constants { + public static final String KEY_TOPIC = "topic"; + public static final String KEY_PULSAR_SERVICE_URL = "pulsarServiceUrl"; + public static final String KEY_PRODUCER_SETTINGS = "producerSettings"; + public static final String KEY_TABLE_FIELDS = "tableFields"; + public static final String KEY_TOKEN = "token"; +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java new file mode 100644 index 0000000000..55aeee7dda --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java @@ -0,0 +1,104 @@ +package com.dtstack.flinkx.pulsar.writer; + +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.pulsar.writer.decoder.JsonDecoder; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.flink.types.Row; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.codehaus.jackson.map.ObjectMapper; + +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * @author: pierre + * @create: 2020/3/21 + */ +public class PulsarOutputFormat extends BaseRichOutputFormat { + + private transient Producer producer; + + protected String topic; + protected String pulsarServiceUrl; + protected String token; + protected Map producerSettings; + + protected List tableFields; + protected static JsonDecoder jsonDecoder = new JsonDecoder(); + protected static ObjectMapper objectMapper = new ObjectMapper(); + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + PulsarClient client; + + if (null != token) { + client = PulsarClient.builder() + .serviceUrl(pulsarServiceUrl) + .authentication(AuthenticationFactory.token(token)) + .build(); + } else { + client = PulsarClient.builder() + .serviceUrl(pulsarServiceUrl) + .build(); + } + // pulsar-client 2.4.0 loadConf有bug + producer = client.newProducer(Schema.STRING) + .topic(topic) + .loadConf(producerSettings) + .create(); + } + + @Override + protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + // copy from kafka-writer + try { + Map map; + int arity = row.getArity(); + if (tableFields != null && tableFields.size() >= arity) { + map = new LinkedHashMap<>((arity << 2) / 3); + for (int i = 0; i < arity; i++) { + map.put(tableFields.get(i), org.apache.flink.util.StringUtils.arrayAwareToString(row.getField(i))); + } + } else { + if (arity == 1) { + Object obj = row.getField(0); + if (obj instanceof Map) { + map = (Map) obj; + } else if (obj instanceof String) { + map = jsonDecoder.decode(obj.toString()); + } else { + map = Collections.singletonMap("message", row.toString()); + } + } else { + map = Collections.singletonMap("message", row.toString()); + } + } + emit(map); + } catch (Throwable e) { + LOG.error("pulsar writeSingleRecordInternal error:{}", ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(e.getMessage(), e); + } + } + + protected void emit(Map event) throws IOException { + producer.send(objectMapper.writeValueAsString(event)); + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void closeInternal() throws IOException { + LOG.warn("pulsar output closeInternal."); + producer.close(); + } +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java new file mode 100644 index 0000000000..637a97e67a --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java @@ -0,0 +1,39 @@ +package com.dtstack.flinkx.pulsar.writer; + +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; + +import java.util.Map; + +/** + * @author: pierre + * @create: 2020/3/21 + */ +public class PulsarOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private PulsarOutputFormat format; + + public PulsarOutputFormatBuilder() { + super.format = format = new PulsarOutputFormat(); + } + + public void setTopic(String topic) { + format.topic = topic; + } + + public void setToken(String token) { + format.token = token; + } + + public void setPulsarServiceUrl(String pulsarServiceUrl) { + format.pulsarServiceUrl = pulsarServiceUrl; + } + + public void setProducerSettings(Map producerSettings) { + format.producerSettings = producerSettings; + } + + @Override + protected void checkFormat() { + + } +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java new file mode 100644 index 0000000000..ae84b857f1 --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java @@ -0,0 +1,43 @@ +package com.dtstack.flinkx.pulsar.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.writer.BaseDataWriter; +import static com.dtstack.flinkx.pulsar.writer.Constants.*; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; + + +/** + * @author: pierre + * @create: 2020/3/21 + */ +public class PulsarWriter extends BaseDataWriter { + protected String topic; + protected String token; + protected String pulsarServiceUrl; + protected List tableFields; + protected Map producerSettings; + + public PulsarWriter(DataTransferConfig config){ + super(config); + topic = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal(KEY_TOPIC); + token = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal(KEY_TOKEN); + pulsarServiceUrl = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal(KEY_PULSAR_SERVICE_URL); + producerSettings = (Map) config.getJob().getContent().get(0).getWriter().getParameter().getVal(KEY_PRODUCER_SETTINGS); + tableFields = (List)config.getJob().getContent().get(0).getWriter().getParameter().getVal(KEY_TABLE_FIELDS); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + PulsarOutputFormatBuilder builder = new PulsarOutputFormatBuilder(); + builder.setTopic(topic); + builder.setPulsarServiceUrl(pulsarServiceUrl); + builder.setProducerSettings(producerSettings); + builder.setToken(token); + return createOutput(dataSet, builder.finish()); + } +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java new file mode 100644 index 0000000000..80ffe47276 --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pulsar.writer.decoder; + +import java.util.Map; + +/** + * Date: 2019/11/21 + * Company: www.dtstack.com + * + * @author tudou + */ +public interface IDecode { + + /** + * 解码给定的数据 + * + * @param message 待解码的内容 + * @return 解码后的数据 + */ + Map decode(String message); + +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java new file mode 100644 index 0000000000..5efc3cf298 --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pulsar.writer.decoder; + +import org.codehaus.jackson.map.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Map; + +/** + * Date: 2019/11/21 + * Company: www.dtstack.com + * + * @author tudou + */ +public class JsonDecoder implements IDecode { + private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); + + private static ObjectMapper objectMapper = new ObjectMapper(); + + private static final String KEY_MESSAGE = "message"; + + @Override + @SuppressWarnings("unchecked") + public Map decode(final String message) { + try { + Map event = objectMapper.readValue(message, Map.class); + if (!event.containsKey(KEY_MESSAGE)) { + event.put(KEY_MESSAGE, message); + } + return event; + } catch (Exception e) { + LOG.error(e.getMessage()); + return Collections.singletonMap(KEY_MESSAGE, message); + } + } +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java new file mode 100644 index 0000000000..74961c83c1 --- /dev/null +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pulsar.writer.decoder; + +import java.util.Collections; +import java.util.Map; + +/** + * Date: 2019/11/21 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PlainDecoder implements IDecode { + + @Override + public Map decode(final String message) { + return Collections.singletonMap("message", message); + } + +} diff --git a/flinkx-pulsar/pom.xml b/flinkx-pulsar/pom.xml new file mode 100644 index 0000000000..09ae5b675a --- /dev/null +++ b/flinkx-pulsar/pom.xml @@ -0,0 +1,38 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-pulsar + pom + + flinkx-pulsar-writer + + + + 2.5.0 + + + + + org.apache.pulsar + pulsar-client + ${pulsar.version} + compile + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7bfcd3cfad..dd8fb0c585 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,7 @@ flinkx-kafka flinkx-emqx + flinkx-pulsar @@ -136,15 +137,15 @@ - - dev - - ${git.branch} - - - true - - + + + + + + + + + release From ab6399dc36cbfee38c95fab35b41bcd90f80ef43 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 15:10:07 +0800 Subject: [PATCH 027/136] add doc --- README.md | 395 +++++++------------------------------------------- README_CH.md | 74 ++++++++++ README_OLD.md | 365 ++++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 491 insertions(+), 343 deletions(-) create mode 100644 README_CH.md create mode 100644 README_OLD.md diff --git a/README.md b/README.md index 9ca081bb2c..1f10f74837 100644 --- a/README.md +++ b/README.md @@ -1,365 +1,74 @@ -## 技术交流 - * 招聘大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com - - * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群 -

- -
- - -## 1 什么是FlinkX - -* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线数据同步框架,实现了多种异构数据源之间高效的数据迁移。** - -不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。 - -
- -
- -## 2 工作原理 - -在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,工作原理如下图: - -
- -
- -## 3 快速起步 - -### 3.1 运行模式 - -* 单机模式:对应Flink集群的单机模式 -* standalone模式:对应Flink集群的分布式模式 -* yarn模式:对应Flink集群的yarn模式 -* yarnPer模式: 对应Flink集群的Per-job模式 - - -### 3.2 执行环境 - -* Java: JDK8及以上 -* Flink集群: 1.4及以上(单机模式不需要安装Flink集群) -* 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 - -### 3.3 打包 +FlinkX +============ -进入项目根目录,使用maven打包: +[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) -windows平台 -``` -mvn clean package -DskipTests -Prelease -DscriptType=bat -``` -unix平台 -``` -mvn clean package -DskipTests -Prelease -DscriptType=sh -``` +English | [中文](README_CH.md) -打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 +# Communication -### 3.4 启动 +- We are recruiting **Big data platform development engineers**.If you want more information about the position, please add WeChat ID [**ysqwhiletrue**] or email your resume to [sishu@dtstack.com](mailto:sishu@dtstack.com). -#### 3.4.1 命令行参数选项 - -* **model** - - * 描述:执行模式,也就是flink集群的工作模式 - * local: 本地模式 - * standalone: 独立部署模式的flink集群 - * yarn: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster" - * yarnPer: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" - * 必选:否 - * 默认值:local - -* **job** - - * 描述:数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息。 - * 必选:是 - * 默认值:无 - -* **pluginRoot** - - * 描述:插件根目录地址,也就是打包后产生的pluginRoot目录。 - * 必选:是 - * 默认值:无 - -* **flinkconf** - - * 描述:flink配置文件所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/conf - * 必选:否 - * 默认值:无 - -* **yarnconf** +- We use [DingTalk](https://www.dingtalk.com/) to communicate,You can search the group number [**30537511**] or scan the QR code below to join the communication group - * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop - * 必选:否 - * 默认值:无 - -* **flinkLibJar** - - * 描述:flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/lib - * 必选:否 - * 默认值:无 - -* **confProp** - - * 描述:flink相关参数,如{\"flink.checkpoint.interval\":200000} - * 必选:否 - * 默认值:无 - -* **queue** - - * 描述:yarn队列,如default - * 必选:否 - * 默认值:无 - -* **pluginLoadMode** - - * 描述:yarnPer模式插件加载方式: - * classpath:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快 - * shipfile:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 - * 必选:否 - * 默认值:classpath - -#### 3.4.2 启动数据同步任务 - -* **以本地模式启动数据同步任务** - -``` -bin/flinkx -mode local \ - -job /Users/softfly/company/flink-data-transfer/jobs/task_to_run.json \ - -plugin /Users/softfly/company/flink-data-transfer/plugins \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以standalone模式启动数据同步任务** - -``` -bin/flinkx -mode standalone \ - -job /Users/softfly/company/flink-data-transfer/jobs/oracle_to_oracle.json \ - -plugin /Users/softfly/company/flink-data-transfer/plugins \ - -flinkconf /hadoop/flink-1.4.0/conf \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以yarn模式启动数据同步任务** - -``` -bin/flinkx -mode yarn \ - -job /Users/softfly/company/flinkx/jobs/mysql_to_mysql.json \ - -plugin /opt/dtstack/flinkplugin/syncplugin \ - -flinkconf /opt/dtstack/myconf/conf \ - -yarnconf /opt/dtstack/myconf/hadoop \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以perjob模式启动数据同步任务** - -``` -bin/flinkx -mode yarnPer \ - -job /test.json \ - -pluginRoot /opt/dtstack/syncplugin \ - -flinkconf /opt/dtstack/flink-1.8.1/conf \ - -yarnconf /opt/dtstack/hadoop-2.7.3/etc/hadoop \ - -flinkLibJar /opt/dtstack/flink-1.8.1/lib \ - -confProp {\"flink.checkpoint.interval\":200000} \ - -queue c -pluginLoadMode classpath -``` - -## 4 数据同步任务模版 - -从最高空俯视,一个数据同步的构成很简单,如下: - -``` -{ - "job": { - "setting": {...}, - "content": [...] - } -} -``` - -数据同步任务包括一个job元素,而这个元素包括setting和content两部分。 - -* setting: 用于配置限速、错误控制和脏数据管理 -* content: 用于配置具体任务信息,包括从哪里来(Reader插件信息),到哪里去(Writer插件信息) - -### 4.1 setting - -``` - "setting": { - "speed": {...}, - "errorLimit": {...}, - "dirty": {...} - } -``` - -setting包括speed、errorLimit和dirty三部分,分别描述限速、错误控制和脏数据管理的配置信息 - -#### 4.1.1 speed - -``` -"speed": { - "bytes": 1048576, - "channel": 2, - "rebalance": false, - "readerChannel": 1, - "writerChannel": 1 -} -``` - -* channel:任务并发数 -* readerChannel:reader的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为reader的并发数。 -* writerChannel:writer的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为writer的并发数。 -* rebalance:此参数配置为true时将强制对reader的数据做Rebalance,不配置此参数或者配置为false时,程序会根据reader和writer的通道数选择是否Rebalance,reader和writer的通道数一致时不使用Reblance,通道数不一致时使用Reblance。 -* bytes::每秒字节数,默认为 Long.MAX_VALUE - -#### 4.1.2 errorLimit - -``` - "errorLimit": { - "record": 10000, - "percentage": 100 - } -``` - -* record: 出错记录数超过record设置的条数时,任务标记为失败 -* percentage: 当出错记录数超过percentage百分数时,任务标记为失败 - -#### 4.1.3 dirty - -``` - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } -``` - -* path: 脏数据存放路径 -* hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置) - -#### 4.1.4 restore - -``` -"restore": { - - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - } -``` - -restore配置请参考[断点续传](docs/restore.md) - -#### 4.1.5 log - -``` -"log" : { - "isLogger": true, - "level" : "warn", - "path" : "/opt/log/", - "pattern":"" - } -``` -* isLogger: 日志是否保存到磁盘, `true`: 是; `false`(默认): 否; -* level: 日志输出级别, `trace`, `debug`, `info`(默认), `warn`, `error`; -* path: 日志保存路径, 默认为`/tmp/dtstack/flinkx/`, 日志名称为当前flink任务的jobID,如: `97501729f8c44c260d889d099968cc74.log` -* pattern: 日志输出格式 - * log4j默认格式为: `%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n`; - * logback默认格式为: `%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n` - -注意:该日志记录功能只会记录`com.dtstack`包下的输出日志, 如需变更,可修改类参数`DtLogger.LOGGER_NAME`。 - -### 4.2 content +
+ +
-``` - "content": [ - { - "reader": { - "name": "...", - "parameter": { - ... - } - }, - "writer": { - "name": "...", - "parameter": { - ... - } - } - } - ] -``` +# Introduction -* reader: 用于读取数据的插件的信息 -* writer: 用于写入数据的插件的信息 +FlinkX is a data synchronization tool based on Flink. FlinkX can collect static data, such as MySQL, HDFS, etc, as well as real-time changing data, such as MySQL binlog, Kafka, etc. FlinkX currently includes the following features: -reader和writer包括name和parameter,分别表示插件名称和插件参数 +- Most plugins support concurrent reading and writing of data, which can greatly improve the speed of reading and writing; -### 4.3 数据同步任务例子 +- Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [失败恢复](docs/restore.md) -详见flinkx-examples子工程 +- The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/rdbreader.md) -## 5. 数据同步插件 +- Some databases support opening Kerberos security authentication; [Kerberos](docs/kerberos.md) -### 5.1 读取插件 +- You can limit the reading speed of Reader plugins and reduce the impact on business databases; -* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbreader.md) -* [分库分表读取插件](docs/rdbdreader.md) -* [HDFS读取插件](docs/hdfsreader.md) -* [HBase读取插件](docs/hbasereader.md) -* [Elasticsearch读取插件](docs/esreader.md) -* [Ftp读取插件](docs/ftpreader.md) -* [Odps读取插件](docs/odpsreader.md) -* [MongoDB读取插件](docs/mongodbreader.md) -* [Stream读取插件](docs/streamreader.md) -* [Carbondata读取插件](docs/carbondatareader.md) -* [MySQL binlog读取插件](docs/binlog.md) -* [KafKa读取插件](docs/kafkareader.md) -* [Kudu读取插件](docs/kudureader.md) -* [Cassandra读取插件](docs/cassandrareader.md) -* [Emqx读取插件](docs/emqxreader.md) +- 可以记录writer插件写数据时产生的脏数据; -### 5.2 写入插件 +- 可以限制脏数据的最大数量; -* [关系数据库写入插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbwriter.md) -* [HDFS写入插件](docs/hdfswriter.md) -* [HBase写入插件](docs/hbasewriter.md) -* [Elasticsearch写入插件](docs/eswriter.md) -* [Ftp写入插件](docs/ftpwriter.md) -* [Odps写入插件](docs/odpswriter.md) -* [MongoDB写入插件](docs/mongodbwriter.md) -* [Redis写入插件](docs/rediswriter.md) -* [Stream写入插件](docs/streamwriter.md) -* [Carbondata写入插件](docs/carbondatawriter.md) -* [Kafka写入插件](docs/kafkawriter.md) -* [Hive写入插件](docs/hivewriter.md) -* [Kudu写入插件](docs/kuduwriter.md) -* [Cassandra写入插件](docs/cassandrawriter.md) -* [Emqx写入插件](docs/emqxwriter.md) +- 支持多种运行模式; -[断点续传和实时采集功能介绍](docs/restore.md) +FlinkX supports rich data source types such as SQL databases, NoSQL databases, and so on -[数据源开启Kerberos](docs/kerberos.md) +| | Database Type | Reader | Writer | +|:----------------------:|:-------------:|:------:|:------:| +| Batch Synchronization | MySQL | √ | √ | +| | Oracle | √ | √ | +| | SqlServer | √ | √ | +| | PostgreSQL | √ | √ | +| | DB2 | √ | √ | +| | GBase | √ | √ | +| | ClickHouse | √ | √ | +| | PolarDB | √ | √ | +| | SAP Hana | √ | √ | +| | Teradata | √ | √ | +| | Phoenix | √ | √ | +| | Cassandra | √ | √ | +| | ODPS | √ | √ | +| | HBase | √ | √ | +| | MongoDB | √ | √ | +| | Kudu | √ | √ | +| | ElasticSearch | √ | √ | +| | FTP | √ | √ | +| | HDFS | √ | √ | +| | Carbondata | √ | √ | +| | Redis | √ | | +| | Hive | | √ | +| Stream Synchronization | Kafka | √ | √ | +| | EMQX | √ | √ | +| | MySQL Binlog | √ | | -[统计指标说明](docs/statistics.md) +# Documentation -## 6.版本说明 +[Documentation](https://github.com/DTStack/flinkx/wiki) | [Old Documentation](README_OLD.md) - 1.flinkx的分支版本跟flink的版本对应,比如:flinkx v1.5.0 对应 flink1.5.0,版本说明: +# License -| 插件版本 | flink版本 | -| ----- | ------- | -| 1.5.x | 1.5.4 | -| 1.8.x | 1.8.1 | +FlinkX is under the Apache 2.0 license. See the [LICENSE](http://www.apache.org/licenses/LICENSE-2.0) file for details. diff --git a/README_CH.md b/README_CH.md new file mode 100644 index 0000000000..3cc37d26d3 --- /dev/null +++ b/README_CH.md @@ -0,0 +1,74 @@ +FlinkX +============ + +[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) + +[English](README.md) | 中文 + +# 技术交流 + +- 招聘**大数据平台开发工程师**,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) + +- 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群 + +
+ +
+ +# 介绍 + +FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集静态的数据,比如MySQL,HDFS等,也可以采集实时变化的数据,比如MySQL binlog,Kafka等。FlinkX目前包含下面这些特性: + +- 大部分插件支持并发读写数据,可以大幅度提高读写速度; + +- 部分插件支持失败恢复的功能,可以从失败的位置恢复任务,节约运行时间;[失败恢复](docs/restore.md) + +- 关系数据库的读取支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/rdbreader.md) + +- 部分数据库支持开启Kerberos安全认证;[Kerberos](docs/kerberos.md) + +- 可以限制reader的读取速度,降低对业务数据库的影响; + +- 可以记录writer插件写数据时产生的脏数据; + +- 可以限制脏数据的最大数量; + +- 支持多种运行模式; + +FlinkX目前支持下面这些数据库: + +| | 数据源类型 | Reader | Writer | +|:----:|:-------------:|:------:|:------:| +| 离线同步 | MySQL | √ | √ | +| | Oracle | √ | √ | +| | SqlServer | √ | √ | +| | PostgreSQL | √ | √ | +| | DB2 | √ | √ | +| | GBase | √ | √ | +| | ClickHouse | √ | √ | +| | PolarDB | √ | √ | +| | SAP Hana | √ | √ | +| | Teradata | √ | √ | +| | Phoenix | √ | √ | +| | Cassandra | √ | √ | +| | ODPS | √ | √ | +| | HBase | √ | √ | +| | MongoDB | √ | √ | +| | Kudu | √ | √ | +| | ElasticSearch | √ | √ | +| | FTP | √ | √ | +| | HDFS | √ | √ | +| | Carbondata | √ | √ | +| | Redis | √ | | +| | Hive | | √ | +| 实时采集 | Kafka | √ | √ | +| | EMQX | √ | √ | +| | MySQL Binlog | √ | | + +# 参考文档 + +[参考文档](https://github.com/DTStack/flinkx/wiki) | [旧文档](README_OLD.md) + +# License + +FlinkX is under the Apache 2.0 license. See the [LICENSE](http://www.apache.org/licenses/LICENSE-2.0) file for details. diff --git a/README_OLD.md b/README_OLD.md new file mode 100644 index 0000000000..9ca081bb2c --- /dev/null +++ b/README_OLD.md @@ -0,0 +1,365 @@ +## 技术交流 + * 招聘大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com + + * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群 +
+ +
+ + +## 1 什么是FlinkX + +* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线数据同步框架,实现了多种异构数据源之间高效的数据迁移。** + +不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。 + +
+ +
+ +## 2 工作原理 + +在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,工作原理如下图: + +
+ +
+ +## 3 快速起步 + +### 3.1 运行模式 + +* 单机模式:对应Flink集群的单机模式 +* standalone模式:对应Flink集群的分布式模式 +* yarn模式:对应Flink集群的yarn模式 +* yarnPer模式: 对应Flink集群的Per-job模式 + + +### 3.2 执行环境 + +* Java: JDK8及以上 +* Flink集群: 1.4及以上(单机模式不需要安装Flink集群) +* 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 + +### 3.3 打包 + +进入项目根目录,使用maven打包: + +windows平台 +``` +mvn clean package -DskipTests -Prelease -DscriptType=bat +``` +unix平台 +``` +mvn clean package -DskipTests -Prelease -DscriptType=sh +``` + +打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 + +### 3.4 启动 + +#### 3.4.1 命令行参数选项 + +* **model** + + * 描述:执行模式,也就是flink集群的工作模式 + * local: 本地模式 + * standalone: 独立部署模式的flink集群 + * yarn: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster" + * yarnPer: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" + * 必选:否 + * 默认值:local + +* **job** + + * 描述:数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息。 + * 必选:是 + * 默认值:无 + +* **pluginRoot** + + * 描述:插件根目录地址,也就是打包后产生的pluginRoot目录。 + * 必选:是 + * 默认值:无 + +* **flinkconf** + + * 描述:flink配置文件所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/conf + * 必选:否 + * 默认值:无 + +* **yarnconf** + + * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop + * 必选:否 + * 默认值:无 + +* **flinkLibJar** + + * 描述:flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/lib + * 必选:否 + * 默认值:无 + +* **confProp** + + * 描述:flink相关参数,如{\"flink.checkpoint.interval\":200000} + * 必选:否 + * 默认值:无 + +* **queue** + + * 描述:yarn队列,如default + * 必选:否 + * 默认值:无 + +* **pluginLoadMode** + + * 描述:yarnPer模式插件加载方式: + * classpath:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快 + * shipfile:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 + * 必选:否 + * 默认值:classpath + +#### 3.4.2 启动数据同步任务 + +* **以本地模式启动数据同步任务** + +``` +bin/flinkx -mode local \ + -job /Users/softfly/company/flink-data-transfer/jobs/task_to_run.json \ + -plugin /Users/softfly/company/flink-data-transfer/plugins \ + -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* +``` + +* **以standalone模式启动数据同步任务** + +``` +bin/flinkx -mode standalone \ + -job /Users/softfly/company/flink-data-transfer/jobs/oracle_to_oracle.json \ + -plugin /Users/softfly/company/flink-data-transfer/plugins \ + -flinkconf /hadoop/flink-1.4.0/conf \ + -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* +``` + +* **以yarn模式启动数据同步任务** + +``` +bin/flinkx -mode yarn \ + -job /Users/softfly/company/flinkx/jobs/mysql_to_mysql.json \ + -plugin /opt/dtstack/flinkplugin/syncplugin \ + -flinkconf /opt/dtstack/myconf/conf \ + -yarnconf /opt/dtstack/myconf/hadoop \ + -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* +``` + +* **以perjob模式启动数据同步任务** + +``` +bin/flinkx -mode yarnPer \ + -job /test.json \ + -pluginRoot /opt/dtstack/syncplugin \ + -flinkconf /opt/dtstack/flink-1.8.1/conf \ + -yarnconf /opt/dtstack/hadoop-2.7.3/etc/hadoop \ + -flinkLibJar /opt/dtstack/flink-1.8.1/lib \ + -confProp {\"flink.checkpoint.interval\":200000} \ + -queue c -pluginLoadMode classpath +``` + +## 4 数据同步任务模版 + +从最高空俯视,一个数据同步的构成很简单,如下: + +``` +{ + "job": { + "setting": {...}, + "content": [...] + } +} +``` + +数据同步任务包括一个job元素,而这个元素包括setting和content两部分。 + +* setting: 用于配置限速、错误控制和脏数据管理 +* content: 用于配置具体任务信息,包括从哪里来(Reader插件信息),到哪里去(Writer插件信息) + +### 4.1 setting + +``` + "setting": { + "speed": {...}, + "errorLimit": {...}, + "dirty": {...} + } +``` + +setting包括speed、errorLimit和dirty三部分,分别描述限速、错误控制和脏数据管理的配置信息 + +#### 4.1.1 speed + +``` +"speed": { + "bytes": 1048576, + "channel": 2, + "rebalance": false, + "readerChannel": 1, + "writerChannel": 1 +} +``` + +* channel:任务并发数 +* readerChannel:reader的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为reader的并发数。 +* writerChannel:writer的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为writer的并发数。 +* rebalance:此参数配置为true时将强制对reader的数据做Rebalance,不配置此参数或者配置为false时,程序会根据reader和writer的通道数选择是否Rebalance,reader和writer的通道数一致时不使用Reblance,通道数不一致时使用Reblance。 +* bytes::每秒字节数,默认为 Long.MAX_VALUE + +#### 4.1.2 errorLimit + +``` + "errorLimit": { + "record": 10000, + "percentage": 100 + } +``` + +* record: 出错记录数超过record设置的条数时,任务标记为失败 +* percentage: 当出错记录数超过percentage百分数时,任务标记为失败 + +#### 4.1.3 dirty + +``` + "dirty": { + "path": "/tmp", + "hadoopConfig": { + "fs.default.name": "hdfs://ns1", + "dfs.nameservices": "ns1", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", + "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", + "dfs.ha.automatic-failover.enabled": "true", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "fs.hdfs.impl.disable.cache": "true" + } + } +``` + +* path: 脏数据存放路径 +* hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置) + +#### 4.1.4 restore + +``` +"restore": { + + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } +``` + +restore配置请参考[断点续传](docs/restore.md) + +#### 4.1.5 log + +``` +"log" : { + "isLogger": true, + "level" : "warn", + "path" : "/opt/log/", + "pattern":"" + } +``` +* isLogger: 日志是否保存到磁盘, `true`: 是; `false`(默认): 否; +* level: 日志输出级别, `trace`, `debug`, `info`(默认), `warn`, `error`; +* path: 日志保存路径, 默认为`/tmp/dtstack/flinkx/`, 日志名称为当前flink任务的jobID,如: `97501729f8c44c260d889d099968cc74.log` +* pattern: 日志输出格式 + * log4j默认格式为: `%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n`; + * logback默认格式为: `%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n` + +注意:该日志记录功能只会记录`com.dtstack`包下的输出日志, 如需变更,可修改类参数`DtLogger.LOGGER_NAME`。 + +### 4.2 content + +``` + "content": [ + { + "reader": { + "name": "...", + "parameter": { + ... + } + }, + "writer": { + "name": "...", + "parameter": { + ... + } + } + } + ] +``` + +* reader: 用于读取数据的插件的信息 +* writer: 用于写入数据的插件的信息 + +reader和writer包括name和parameter,分别表示插件名称和插件参数 + +### 4.3 数据同步任务例子 + +详见flinkx-examples子工程 + +## 5. 数据同步插件 + +### 5.1 读取插件 + +* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbreader.md) +* [分库分表读取插件](docs/rdbdreader.md) +* [HDFS读取插件](docs/hdfsreader.md) +* [HBase读取插件](docs/hbasereader.md) +* [Elasticsearch读取插件](docs/esreader.md) +* [Ftp读取插件](docs/ftpreader.md) +* [Odps读取插件](docs/odpsreader.md) +* [MongoDB读取插件](docs/mongodbreader.md) +* [Stream读取插件](docs/streamreader.md) +* [Carbondata读取插件](docs/carbondatareader.md) +* [MySQL binlog读取插件](docs/binlog.md) +* [KafKa读取插件](docs/kafkareader.md) +* [Kudu读取插件](docs/kudureader.md) +* [Cassandra读取插件](docs/cassandrareader.md) +* [Emqx读取插件](docs/emqxreader.md) + +### 5.2 写入插件 + +* [关系数据库写入插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbwriter.md) +* [HDFS写入插件](docs/hdfswriter.md) +* [HBase写入插件](docs/hbasewriter.md) +* [Elasticsearch写入插件](docs/eswriter.md) +* [Ftp写入插件](docs/ftpwriter.md) +* [Odps写入插件](docs/odpswriter.md) +* [MongoDB写入插件](docs/mongodbwriter.md) +* [Redis写入插件](docs/rediswriter.md) +* [Stream写入插件](docs/streamwriter.md) +* [Carbondata写入插件](docs/carbondatawriter.md) +* [Kafka写入插件](docs/kafkawriter.md) +* [Hive写入插件](docs/hivewriter.md) +* [Kudu写入插件](docs/kuduwriter.md) +* [Cassandra写入插件](docs/cassandrawriter.md) +* [Emqx写入插件](docs/emqxwriter.md) + +[断点续传和实时采集功能介绍](docs/restore.md) + +[数据源开启Kerberos](docs/kerberos.md) + +[统计指标说明](docs/statistics.md) + +## 6.版本说明 + + 1.flinkx的分支版本跟flink的版本对应,比如:flinkx v1.5.0 对应 flink1.5.0,版本说明: + +| 插件版本 | flink版本 | +| ----- | ------- | +| 1.5.x | 1.5.4 | +| 1.8.x | 1.8.1 | From 88be7c04b27e6a1ade0eb6c27475f510704ea9c8 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 16:05:31 +0800 Subject: [PATCH 028/136] add docs --- README.md | 12 +- README_CH.md | 2 +- README_OLD.md | 6 +- flinkx-core/pom.xml | 1 + .../com/dtstack/flinkx/dm/DmDatabaseMeta.java | 4 +- .../flinkx/dm/format/DmInputFormat.java | 10 +- .../flinkx/hive/writer/HiveOutputFormat.java | 4 +- .../oracle/format/OracleOutputFormat.java | 2 +- .../oraclelogminer/util/LogMinerUtil.java | 0 flinkx-test/pom.xml | 442 ------------------ .../com/dtstack/flinkx/test/LocalTest.java | 304 ------------ .../flinkx/test/PluginNameConstrant.java | 80 ---- .../resources/dev_test_job/es_stream.json | 76 --- .../dev_test_job/gbase_template.json | 74 --- .../resources/dev_test_job/hdfs_stream.json | 64 --- .../dev_test_job/kudu_reader_template.json | 46 -- .../dev_test_job/kudu_writer_template.json | 59 --- .../resources/dev_test_job/perJob_test.json | 35 -- .../resources/dev_test_job/stream_es.json | 76 --- .../resources/dev_test_job/stream_hdfs.json | 247 ---------- .../dev_test_job/stream_template.json | 1 - pom.xml | 44 +- 22 files changed, 22 insertions(+), 1567 deletions(-) delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java delete mode 100644 flinkx-test/pom.xml delete mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java delete mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java delete mode 100644 flinkx-test/src/main/resources/dev_test_job/es_stream.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/gbase_template.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/hdfs_stream.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/kudu_reader_template.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/kudu_writer_template.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/perJob_test.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/stream_es.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/stream_hdfs.json delete mode 100644 flinkx-test/src/main/resources/dev_test_job/stream_template.json diff --git a/README.md b/README.md index 1f10f74837..c4f16a5536 100644 --- a/README.md +++ b/README.md @@ -21,21 +21,21 @@ FlinkX is a data synchronization tool based on Flink. FlinkX can collect static - Most plugins support concurrent reading and writing of data, which can greatly improve the speed of reading and writing; -- Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [失败恢复](docs/restore.md) +- Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [Failure Recovery](docs/restore.md) - The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/rdbreader.md) - Some databases support opening Kerberos security authentication; [Kerberos](docs/kerberos.md) -- You can limit the reading speed of Reader plugins and reduce the impact on business databases; +- Limit the reading speed of Reader plugins and reduce the impact on business databases; -- 可以记录writer插件写数据时产生的脏数据; +- Save the dirty data when writing data; -- 可以限制脏数据的最大数量; +- Limit the maximum number of dirty data; -- 支持多种运行模式; +- Multiple running modes: Local,Standalone,Yarn Session,Yarn Per; -FlinkX supports rich data source types such as SQL databases, NoSQL databases, and so on +The following databases are currently supported: | | Database Type | Reader | Writer | |:----------------------:|:-------------:|:------:|:------:| diff --git a/README_CH.md b/README_CH.md index 3cc37d26d3..2433d48876 100644 --- a/README_CH.md +++ b/README_CH.md @@ -23,7 +23,7 @@ FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集 - 部分插件支持失败恢复的功能,可以从失败的位置恢复任务,节约运行时间;[失败恢复](docs/restore.md) -- 关系数据库的读取支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/rdbreader.md) +- 关系数据库的Reader插件支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/rdbreader.md) - 部分数据库支持开启Kerberos安全认证;[Kerberos](docs/kerberos.md) diff --git a/README_OLD.md b/README_OLD.md index 9ca081bb2c..bb267a0010 100644 --- a/README_OLD.md +++ b/README_OLD.md @@ -47,11 +47,11 @@ windows平台 ``` -mvn clean package -DskipTests -Prelease -DscriptType=bat +mvn clean package -DskipTests -DscriptType=bat ``` unix平台 ``` -mvn clean package -DskipTests -Prelease -DscriptType=sh +mvn clean package -DskipTests -DscriptType=sh ``` 打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 @@ -315,7 +315,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 ### 5.1 读取插件 -* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbreader.md) +* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix,达梦)](docs/rdbreader.md) * [分库分表读取插件](docs/rdbdreader.md) * [HDFS读取插件](docs/hdfsreader.md) * [HBase读取插件](docs/hbasereader.md) diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index ce08bcc3e4..08d35429c9 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -169,6 +169,7 @@ exec-maven-plugin org.codehaus.mojo + 1.6.0 Version Calculation diff --git a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java index 4247c793b3..3c4d2883dc 100644 --- a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java +++ b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java @@ -66,12 +66,12 @@ public String getDriverClass() { } @Override - public String getSQLQueryFields(String tableName) { + public String getSqlQueryFields(String tableName) { return "SELECT * FROM " + tableName + " LIMIT 1"; } @Override - public String getSQLQueryColumnFields(List column, String table) { + public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 1"; } diff --git a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java index 8d08d9d886..8a114752f1 100644 --- a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java +++ b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java @@ -19,13 +19,13 @@ import com.dtstack.flinkx.enums.ColumnType; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; -import com.dtstack.flinkx.rdb.util.DBUtil; +import com.dtstack.flinkx.rdb.util.DbUtil; import org.apache.flink.types.Row; import java.io.IOException; import java.sql.Timestamp; -import static com.dtstack.flinkx.rdb.util.DBUtil.clobToString; +import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; /** * Date: 2020/03/18 @@ -69,9 +69,9 @@ public Row nextRecordInternal(Row row) throws IOException { @Override protected String getTimeStr(Long location, String incrementColType){ String timeStr; - Timestamp ts = new Timestamp(DBUtil.getMillis(location)); - ts.setNanos(DBUtil.getNanos(location)); - timeStr = DBUtil.getNanosTimeStr(ts.toString()); + Timestamp ts = new Timestamp(DbUtil.getMillis(location)); + ts.setNanos(DbUtil.getNanos(location)); + timeStr = DbUtil.getNanosTimeStr(ts.toString()); if(ColumnType.TIMESTAMP.name().equals(incrementColType)){ timeStr = String.format("TO_TIMESTAMP('%s','YYYY-MM-DD HH24:MI:SS:FF6')",timeStr); diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java index aa180b0437..217163c261 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java @@ -251,14 +251,14 @@ private Pair getHdfsOutputFormat(String tablePa return new Pair(outputFormat, tableInfo); } - private BaseHdfsOutputFormat createHdfsOutputFormat(TableInfo tableInfo, String path) { + private BaseHdfsOutputFormat createHdfsOutputFormat(TableInfo tableInfo, String path, String hiveTablePath) { try { HdfsOutputFormatBuilder hdfsOutputFormatBuilder = this.getHdfsOutputFormatBuilder(); hdfsOutputFormatBuilder.setPath(path); hdfsOutputFormatBuilder.setColumnNames(tableInfo.getColumns()); hdfsOutputFormatBuilder.setColumnTypes(tableInfo.getColumnTypes()); - BaseHdfsOutputFormat outputFormat = (HdfsOutputFormat) hdfsOutputFormatBuilder.finish(); + BaseHdfsOutputFormat outputFormat = (BaseHdfsOutputFormat) hdfsOutputFormatBuilder.finish(); outputFormat.setFormatId(hiveTablePath); outputFormat.setDirtyDataManager(dirtyDataManager); outputFormat.setErrorLimiter(errorLimiter); diff --git a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java index 7b3e34ba7f..ed77a77ea4 100644 --- a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java +++ b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java @@ -79,7 +79,7 @@ protected List probeFullColumns(String table, Connection dbConn) throws @Override protected Map> probePrimaryKeys(String table, Connection dbConn) throws SQLException { Map> map = new HashMap<>(16); - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_ORACLE_INDEX_SQL,table)); + PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); ResultSet rs = ps.executeQuery(); while(rs.next()) { diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/LogMinerUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml deleted file mode 100644 index 9d8440190e..0000000000 --- a/flinkx-test/pom.xml +++ /dev/null @@ -1,442 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-test - - - 1.8.1 - 0.3.0 - - - - - - ch.qos.logback - logback-classic - 1.1.7 - - - - com.google.guava - guava - 19.0 - - - - ch.qos.logback - logback-classic - 1.1.7 - - - - com.google.code.gson - gson - 2.7 - - - - hadoop-mapreduce-client-core - org.apache.hadoop - ${hadoop.version} - - - - hadoop-common - org.apache.hadoop - ${hadoop.version} - - - - hadoop-hdfs - org.apache.hadoop - ${hadoop.version} - - - - org.apache.flink - flink-metrics-core - 1.8.1 - - - - org.apache.flink - flink-metrics-prometheus_2.11 - ${flink.version} - - - - io.prometheus - simpleclient - ${prometheus.version} - - - - io.prometheus - simpleclient_httpserver - ${prometheus.version} - - - - io.prometheus - simpleclient_pushgateway - ${prometheus.version} - - - - com.dtstack.flinkx - flinkx-core - 1.6 - - - com.dtstack.flinkx - flinkx-stream-reader - 1.6 - - - com.dtstack.flinkx - flinkx-stream-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-carbondata-reader - 1.6 - - - com.dtstack.flinkx - flinkx-carbondata-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-db2-reader - 1.6 - - - com.dtstack.flinkx - flinkx-db2-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-es-reader - 1.6 - - - com.dtstack.flinkx - flinkx-es-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-ftp-reader - 1.6 - - - com.dtstack.flinkx - flinkx-ftp-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-hbase-reader - 1.6 - - - com.dtstack.flinkx - flinkx-hbase-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-hdfs-reader - 1.6 - - - com.dtstack.flinkx - flinkx-hdfs-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-mongodb-reader - 1.6 - - - com.dtstack.flinkx - flinkx-mongodb-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-mysql-reader - 1.6 - - - com.dtstack.flinkx - flinkx-mysql-dreader - 1.6 - - - com.dtstack.flinkx - flinkx-mysql-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-odps-reader - 1.6 - - - com.dtstack.flinkx - flinkx-odps-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-oracle-reader - 1.6 - - - com.dtstack.flinkx - flinkx-oracle-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-postgresql-reader - 1.6 - - - com.dtstack.flinkx - flinkx-postgresql-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-sqlserver-reader - 1.6 - - - com.dtstack.flinkx - flinkx-sqlserver-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-redis-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-rdb-core - 1.6 - - - - com.dtstack.flinkx - flinkx-rdb-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-rdb-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-gbase-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-gbase-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-binlog-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-hive-writer - 1.6 - - - - org.apache.hive - hive-serde - 2.1.0 - - - - com.dtstack.flinkx - flinkx-kafka11-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka11-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka10-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka10-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka09-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka09-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-kudu-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kudu-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-clickhouse-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-clickhouse-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-polardb-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-polardb-writer - 1.6 - - - - com.dtstack.flinkx - flinkx-phoenix-reader - 1.6 - - - com.dtstack.flinkx - flinkx-phoenix-writer - 1.6 - - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - test-jar - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - test-jar - - - - org.apache.derby - derby - 10.14.2.0 - test - - - com.dtstack.flinkx - flinkx-emqx-reader - 1.6 - compile - - - com.dtstack.flinkx - flinkx-emqx-writer - 1.6 - compile - - - - com.dtstack.flinkx - flinkx-dm-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-dm-writer - 1.6 - - - - - \ No newline at end of file diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java deleted file mode 100644 index 796d868222..0000000000 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ /dev/null @@ -1,304 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test; - -import com.dtstack.flink.api.java.MyLocalStreamEnvironment; -import com.dtstack.flinkx.binlog.reader.BinlogReader; -import com.dtstack.flinkx.carbondata.reader.CarbondataReader; -import com.dtstack.flinkx.carbondata.writer.CarbondataWriter; -import com.dtstack.flinkx.clickhouse.reader.ClickhouseReader; -import com.dtstack.flinkx.clickhouse.writer.ClickhouseWriter; -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.SpeedConfig; -import com.dtstack.flinkx.constants.ConfigConstant; -import com.dtstack.flinkx.db2.reader.Db2Reader; -import com.dtstack.flinkx.db2.writer.Db2Writer; -import com.dtstack.flinkx.dm.reader.DmReader; -import com.dtstack.flinkx.dm.writer.DmWriter; -import com.dtstack.flinkx.emqx.reader.EmqxReader; -import com.dtstack.flinkx.emqx.writer.EmqxWriter; -import com.dtstack.flinkx.es.reader.EsReader; -import com.dtstack.flinkx.es.writer.EsWriter; -import com.dtstack.flinkx.ftp.reader.FtpReader; -import com.dtstack.flinkx.ftp.writer.FtpWriter; -import com.dtstack.flinkx.gbase.reader.GbaseReader; -import com.dtstack.flinkx.gbase.writer.GbaseWriter; -import com.dtstack.flinkx.hbase.reader.HbaseReader; -import com.dtstack.flinkx.hbase.writer.HbaseWriter; -import com.dtstack.flinkx.hdfs.reader.HdfsReader; -import com.dtstack.flinkx.hdfs.writer.HdfsWriter; -import com.dtstack.flinkx.hive.writer.HiveWriter; -import com.dtstack.flinkx.kafka.reader.KafkaReader; -import com.dtstack.flinkx.kafka.writer.KafkaWriter; -import com.dtstack.flinkx.kafka09.reader.Kafka09Reader; -import com.dtstack.flinkx.kafka09.writer.Kafka09Writer; -import com.dtstack.flinkx.kafka10.reader.Kafka10Reader; -import com.dtstack.flinkx.kafka10.writer.Kafka10Writer; -import com.dtstack.flinkx.kafka11.reader.Kafka11Reader; -import com.dtstack.flinkx.kafka11.writer.Kafka11Writer; -import com.dtstack.flinkx.kudu.reader.KuduReader; -import com.dtstack.flinkx.kudu.writer.KuduWriter; -import com.dtstack.flinkx.mongodb.reader.MongodbReader; -import com.dtstack.flinkx.mongodb.writer.MongodbWriter; -import com.dtstack.flinkx.mysql.reader.MysqlReader; -import com.dtstack.flinkx.mysql.writer.MysqlWriter; -import com.dtstack.flinkx.mysqld.reader.MysqldReader; -import com.dtstack.flinkx.odps.reader.OdpsReader; -import com.dtstack.flinkx.odps.writer.OdpsWriter; -import com.dtstack.flinkx.oracle.reader.OracleReader; -import com.dtstack.flinkx.oracle.writer.OracleWriter; -import com.dtstack.flinkx.phoenix.reader.PhoenixReader; -import com.dtstack.flinkx.phoenix.writer.PhoenixWriter; -import com.dtstack.flinkx.polardb.reader.PolardbReader; -import com.dtstack.flinkx.polardb.writer.PolardbWriter; -import com.dtstack.flinkx.postgresql.reader.PostgresqlReader; -import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; -import com.dtstack.flinkx.reader.BaseDataReader; -import com.dtstack.flinkx.redis.writer.RedisWriter; -import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; -import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; -import com.dtstack.flinkx.stream.reader.StreamReader; -import com.dtstack.flinkx.stream.writer.StreamWriter; -import com.dtstack.flinkx.util.ResultPrintUtil; -import com.dtstack.flinkx.writer.BaseDataWriter; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.state.filesystem.FsStateBackend; -import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import com.dtstack.flinkx.streaming.runtime.partitioner.CustomPartitioner; -import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.nio.charset.StandardCharsets; -import java.util.Properties; -import java.util.concurrent.TimeUnit; - -/** - * @author jiangbo - */ -public class LocalTest { - - public static Logger LOG = LoggerFactory.getLogger(LocalTest.class); - - private static final int FAILURE_RATE = 3; - - private static final int FAILURE_INTERVAL = 6; - - private static final int DELAY_INTERVAL = 10; - - public static final String TEST_RESOURCE_DIR = "flinkx-examples/examples/"; - - public static Configuration conf = new Configuration(); - - public static void main(String[] args) throws Exception{ - Properties confProperties = new Properties(); -// confProperties.put("flink.checkpoint.interval", "60000"); -// confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); -// -// conf.setString("metrics.reporter.promgateway.class","org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"); -// conf.setString("metrics.reporter.promgateway.host","127.0.0.1"); -// conf.setString("metrics.reporter.promgateway.port","9091"); -// conf.setString("metrics.reporter.promgateway.jobName","108job"); -// conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); -// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - conf.setString("rest.bind-port", "8888"); - - String jobPath = "D:\\project\\dt-center-flinkx\\flinkx-test\\src\\main\\resources\\dev_test_job\\stream_template.json"; - JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); - ResultPrintUtil.printResult(result); - } - - public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savepointPath) throws Exception{ - String jobContent = readJob(jobFile); - return runJob(jobContent, confProperties, savepointPath); - } - - public static JobExecutionResult runJob(String job, Properties confProperties, String savepointPath) throws Exception{ - DataTransferConfig config = DataTransferConfig.parse(job); - SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); - - MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); - - openCheckpointConf(env, confProperties); - - env.setParallelism(speedConfig.getChannel()); - - if (needRestart(config)) { - env.setRestartStrategy(RestartStrategies.fixedDelayRestart( - 10, - Time.of(10, TimeUnit.SECONDS) - )); - } - - BaseDataReader reader = buildDataReader(config, env); - DataStream dataStream = reader.readData(); - dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); - - if (speedConfig.isRebalance()) { - dataStream = dataStream.rebalance(); - } - - BaseDataWriter writer = buildDataWriter(config); - writer.writeData(dataStream).setParallelism(speedConfig.getWriterChannel()); - - if(StringUtils.isNotEmpty(savepointPath)){ - env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); - } - - return env.execute(); - } - - private static boolean needRestart(DataTransferConfig config){ - return config.getJob().getSetting().getRestoreConfig().isRestore(); - } - - private static String readJob(File file) { - try { - FileInputStream in = new FileInputStream(file); - byte[] fileContent = new byte[(int) file.length()]; - in.read(fileContent); - return new String(fileContent, StandardCharsets.UTF_8); - } catch (Exception e){ - throw new RuntimeException(e); - } - } - - private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env){ - String readerName = config.getJob().getContent().get(0).getReader().getName(); - BaseDataReader reader ; - switch (readerName){ - case PluginNameConstrant.STREAM_READER : reader = new StreamReader(config, env); break; - case PluginNameConstrant.CARBONDATA_READER : reader = new CarbondataReader(config, env); break; - case PluginNameConstrant.ORACLE_READER : reader = new OracleReader(config, env); break; - case PluginNameConstrant.POSTGRESQL_READER : reader = new PostgresqlReader(config, env); break; - case PluginNameConstrant.SQLSERVER_READER : reader = new SqlserverReader(config, env); break; - case PluginNameConstrant.MYSQLD_READER : reader = new MysqldReader(config, env); break; - case PluginNameConstrant.MYSQL_READER : reader = new MysqlReader(config, env); break; - case PluginNameConstrant.DB2_READER : reader = new Db2Reader(config, env); break; - case PluginNameConstrant.GBASE_READER : reader = new GbaseReader(config, env); break; - case PluginNameConstrant.ES_READER : reader = new EsReader(config, env); break; - case PluginNameConstrant.FTP_READER : reader = new FtpReader(config, env); break; - case PluginNameConstrant.HBASE_READER : reader = new HbaseReader(config, env); break; - case PluginNameConstrant.HDFS_READER : reader = new HdfsReader(config, env); break; - case PluginNameConstrant.MONGODB_READER : reader = new MongodbReader(config, env); break; - case PluginNameConstrant.ODPS_READER : reader = new OdpsReader(config, env); break; - case PluginNameConstrant.BINLOG_READER : reader = new BinlogReader(config, env); break; - case PluginNameConstrant.KAFKA09_READER : reader = new Kafka09Reader(config, env); break; - case PluginNameConstrant.KAFKA10_READER : reader = new Kafka10Reader(config, env); break; - case PluginNameConstrant.KAFKA11_READER : reader = new Kafka11Reader(config, env); break; - case PluginNameConstrant.KAFKA_READER : reader = new KafkaReader(config, env); break; - case PluginNameConstrant.KUDU_READER : reader = new KuduReader(config, env); break; - case PluginNameConstrant.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; - case PluginNameConstrant.POLARDB_READER : reader = new PolardbReader(config, env); break; - case PluginNameConstrant.PHOENIX_READER : reader = new PhoenixReader(config, env); break; - case PluginNameConstrant.EMQX_READER : reader = new EmqxReader(config, env); break; - case PluginNameConstrant.DM_READER : reader = new DmReader(config, env); break; - default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); - } - - return reader; - } - - private static BaseDataWriter buildDataWriter(DataTransferConfig config){ - String writerName = config.getJob().getContent().get(0).getWriter().getName(); - BaseDataWriter writer; - switch (writerName){ - case PluginNameConstrant.STREAM_WRITER : writer = new StreamWriter(config); break; - case PluginNameConstrant.CARBONDATA_WRITER : writer = new CarbondataWriter(config); break; - case PluginNameConstrant.MYSQL_WRITER : writer = new MysqlWriter(config); break; - case PluginNameConstrant.SQLSERVER_WRITER : writer = new SqlserverWriter(config); break; - case PluginNameConstrant.ORACLE_WRITER : writer = new OracleWriter(config); break; - case PluginNameConstrant.POSTGRESQL_WRITER : writer = new PostgresqlWriter(config); break; - case PluginNameConstrant.DB2_WRITER : writer = new Db2Writer(config); break; - case PluginNameConstrant.GBASE_WRITER : writer = new GbaseWriter(config); break; - case PluginNameConstrant.ES_WRITER : writer = new EsWriter(config); break; - case PluginNameConstrant.FTP_WRITER : writer = new FtpWriter(config); break; - case PluginNameConstrant.HBASE_WRITER : writer = new HbaseWriter(config); break; - case PluginNameConstrant.HDFS_WRITER : writer = new HdfsWriter(config); break; - case PluginNameConstrant.MONGODB_WRITER : writer = new MongodbWriter(config); break; - case PluginNameConstrant.ODPS_WRITER : writer = new OdpsWriter(config); break; - case PluginNameConstrant.REDIS_WRITER : writer = new RedisWriter(config); break; - case PluginNameConstrant.HIVE_WRITER : writer = new HiveWriter(config); break; - case PluginNameConstrant.KAFKA09_WRITER : writer = new Kafka09Writer(config); break; - case PluginNameConstrant.KAFKA10_WRITER : writer = new Kafka10Writer(config); break; - case PluginNameConstrant.KAFKA11_WRITER : writer = new Kafka11Writer(config); break; - case PluginNameConstrant.KUDU_WRITER : writer = new KuduWriter(config); break; - case PluginNameConstrant.CLICKHOUSE_WRITER : writer = new ClickhouseWriter(config); break; - case PluginNameConstrant.POLARDB_WRITER : writer = new PolardbWriter(config); break; - case PluginNameConstrant.KAFKA_WRITER : writer = new KafkaWriter(config); break; - case PluginNameConstrant.PHOENIX_WRITER : writer = new PhoenixWriter(config); break; - case PluginNameConstrant.EMQX_WRITER : writer = new EmqxWriter(config); break; - case PluginNameConstrant.DM_WRITER : writer = new DmWriter(config); break; - default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); - } - - return writer; - } - - private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties){ - if(properties == null){ - return; - } - - if(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ - return; - }else{ - long interval = Long.parseLong(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY).trim()); - - //start checkpoint every ${interval} - env.enableCheckpointing(interval); - - LOG.info("Open checkpoint with interval:" + interval); - } - - String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); - if(checkpointTimeoutStr != null){ - long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); - //checkpoints have to complete within one min,or are discard - env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); - - LOG.info("Set checkpoint timeout:" + checkpointTimeout); - } - - env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); - env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); - env.getCheckpointConfig().enableExternalizedCheckpoints( - CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); - - env.setStateBackend(new FsStateBackend(new Path("file:///tmp/flinkx_checkpoint"))); - env.setRestartStrategy(RestartStrategies.failureRateRestart( - FAILURE_RATE, - Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), - Time.of(DELAY_INTERVAL, TimeUnit.SECONDS) - )); - } -} diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java deleted file mode 100644 index 99a02f546f..0000000000 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.test; - -/** - * @author jiangbo - */ -public class PluginNameConstrant { - - public static final String STREAM_READER = "streamreader"; - public static final String CARBONDATA_READER = "carbondatareader"; - public static final String DB2_READER = "db2reader"; - public static final String ES_READER = "esreader"; - public static final String FTP_READER = "ftpreader"; - public static final String HBASE_READER = "hbasereader"; - public static final String HDFS_READER = "hdfsreader"; - public static final String MONGODB_READER = "mongodbreader"; - public static final String MYSQLD_READER = "mysqldreader"; - public static final String MYSQL_READER = "mysqlreader"; - public static final String ODPS_READER = "odpsreader"; - public static final String ORACLE_READER = "oraclereader"; - public static final String POSTGRESQL_READER = "postgresqlreader"; - public static final String SQLSERVER_READER = "sqlserverreader"; - public static final String GBASE_READER = "gbasereader"; - public static final String KUDU_READER = "kudureader"; - public static final String BINLOG_READER = "binlogreader"; - public static final String KAFKA09_READER = "kafka09reader"; - public static final String KAFKA10_READER = "kafka10reader"; - public static final String KAFKA11_READER = "kafka11reader"; - public static final String KAFKA_READER = "kafkareader"; - public static final String CLICKHOUSE_READER = "clickhousereader"; - public static final String POLARDB_READER = "polardbreader"; - public static final String PHOENIX_READER = "phoenixreader"; - public static final String EMQX_READER = "emqxreader"; - public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; - public static final String DM_READER = "dmreader"; - - public static final String STREAM_WRITER = "streamwriter"; - public static final String CARBONDATA_WRITER = "carbondatawriter"; - public static final String DB2_WRITER = "db2writer"; - public static final String ES_WRITER = "eswriter"; - public static final String FTP_WRITER = "ftpwriter"; - public static final String HBASE_WRITER = "hbasewriter"; - public static final String HDFS_WRITER = "hdfswriter"; - public static final String MONGODB_WRITER = "mongodbwriter"; - public static final String MYSQL_WRITER = "mysqlwriter"; - public static final String ODPS_WRITER = "odpswriter"; - public static final String ORACLE_WRITER = "oraclewriter"; - public static final String POSTGRESQL_WRITER = "postgresqlwriter"; - public static final String REDIS_WRITER = "rediswriter"; - public static final String SQLSERVER_WRITER = "sqlserverwriter"; - public static final String GBASE_WRITER = "gbasewriter"; - public static final String KUDU_WRITER = "kuduwriter"; - public static final String HIVE_WRITER = "hivewriter"; - public static final String KAFKA09_WRITER = "kafka09writer"; - public static final String KAFKA10_WRITER = "kafka10writer"; - public static final String KAFKA11_WRITER = "kafka11writer"; - public static final String KAFKA_WRITER = "kafkawriter"; - public static final String CLICKHOUSE_WRITER = "clickhousewriter"; - public static final String POLARDB_WRITER = "polardbwriter"; - public static final String PHOENIX_WRITER = "phoenixwriter"; - public static final String EMQX_WRITER = "emqxwriter"; - public static final String DM_WRITER = "dmwriter"; -} diff --git a/flinkx-test/src/main/resources/dev_test_job/es_stream.json b/flinkx-test/src/main/resources/dev_test_job/es_stream.json deleted file mode 100644 index 965a842684..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/es_stream.json +++ /dev/null @@ -1,76 +0,0 @@ -{ - "job": { - "content": [ - { - "writer": { - "name": "mysqlwriter", - "parameter": { - "print": true, - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "name1", - "type": "string" - } - ], - "writeMode": "insert", - "connection": [ - { - "password": "123456", - "jdbcUrl": [ - "jdbc:mysql://172.16.8.193:3306/jiangbo" - ], - "table": [ - "es_writer" - ], - "username": "root" - } - ], - "password": "123456", - "username": "root" - } - }, - "reader": { - "name": "esreader", - "parameter": { - "address": "172.16.10.190:9200,172.16.10.193:9200,172.16.10.147:9200", - "index": "scroll_test", - "type": "test", - "batchSize": 1000, - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "name1", - "type": "string" - } - ] - } - } - } - ], - "setting": { - "speed": { - "channel": 4, - "bytes": 0 - }, - "errorLimit": { - "record": 0, - "percentage": 20 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/gbase_template.json b/flinkx-test/src/main/resources/dev_test_job/gbase_template.json deleted file mode 100644 index a7229e67f6..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/gbase_template.json +++ /dev/null @@ -1,74 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "test", - "password" : "DT@Stack#123", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.10.65:3306/test", - "host" : "172.16.10.65", - "start" : { }, - "table" : [ "myresult" ], - "pavingData" : true, - "username" : "drpeco" - }, - "name" : "binlogreader", - "type" : 1 - }, - "writer" : { - "parameter" : { - "fileName" : "pt", - "writeMode" : "append", - "fieldDelimiter" : "\u0001", - "partitionType" : "HOUR", - "path" : "", - "analyticalRules" : "stream_${schema}_${table}", - "password" : "", - "tablesColumn" : "{\"myresult\":[{\"type\":\"VARCHAR\",\"key\":\"before_pv\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_pv\"},{\"type\":\"VARCHAR\",\"key\":\"before_channel\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_channel\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", - "partition" : "pt", - "hadoopConfig" : { - "fs.defaultFS" : "hdfs://cdh01:8020", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem", - "hive.server2.authentication": "kerberos", - "hive.server2.authentication.kerberos.principal":"hive/cdh02@DTSTACK.COM", - "hive.server2.authentication.kerberos.keytab":"C:\\Users\\24573\\Desktop\\cdh02\\hive.keytab", - "hive.metastore.kerberos.principal":"hive/cdh02@DTSTACK.COM", - "hive.metastore.kerberos.keytab.file":"C:\\Users\\24573\\Desktop\\cdh02\\hive.keytab", - "java.security.krb5.conf":"C:\\Users\\24573\\Desktop\\cdh02\\krb5.conf", - "useLocalFile": true, - "remoteDir" : "/opt/dtstack/DTApp/Sftp/CONSOLE_12", - "sftpConf" : { - "path" : "/opt/dtstack/DTApp/Sftp", - "password" : "abc123", - "port" : "22", - "openKerberos" : "false", - "host" : "172.16.10.69", - "username" : "root" - } - }, - "jdbcUrl" : "jdbc:hive2://cdh02:10000/xq_hadoop;principal=hive/cdh02@DTSTACK.COM", - "defaultFS" : "hdfs://cdh01:8020", - "fileType" : "text", - "charsetName" : "utf-8", - "username" : "", - "bufferSize" : 1048576 - }, - "name" : "hivewriter", - "type" : 7 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/hdfs_stream.json b/flinkx-test/src/main/resources/dev_test_job/hdfs_stream.json deleted file mode 100644 index cb3ebabe1f..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/hdfs_stream.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "groupId" : "default", - "topic" : "t_shier", - "consumerSettings" : { - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name" : "kafka11reader", - "type" : 14 - }, - "writer" : { - "parameter" : { - "fileName" : "pt", - "writeMode" : "append", - "fieldDelimiter" : "\u0001", - "partitionType" : "DAY", - "path" : "", - "password" : "Wscabc123..@", - "tablesColumn" : "{\"tb_shier_text1\":[{\"type\":\"int\",\"key\":\"id\",\"comment\":\"\"},{\"type\":\"string\",\"key\":\"name\",\"comment\":\"\"}]}", - "partition" : "pt", - "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "hadoop.user.name" : "root", - "dfs.namenode.rpc-address.ns1.nn2" : "172.16.100.213:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "172.16.101.196:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - "jdbcUrl" : "jdbc:hive2://172.16.101.252:10000/hzfhbjzh_pub", - "defaultFS" : "hdfs://ns1", - "table" : "tb_shier_text1", - "fileType" : "text", - "charsetName" : "utf-8", - "username" : "root", - "bufferSize" : 10485760 - }, - "name" : "hivewriter", - "type" : 7 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { - }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/kudu_reader_template.json b/flinkx-test/src/main/resources/dev_test_job/kudu_reader_template.json deleted file mode 100644 index 9a2bd1ae10..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/kudu_reader_template.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "long" - }, - { - "name": "user_id", - "type": "long" - }, - { - "name": "name", - "type": "string" - } - ], - "masterAddresses": "impala1:7051,impala2:7051,impala3:7051", - "table": "kudu_range_table", - "readMode": "read_latest", - "filter": "" - }, - "name": "kudureader" - }, - "writer": { - "parameter": { - "print": false - }, - "name": "streamwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/kudu_writer_template.json b/flinkx-test/src/main/resources/dev_test_job/kudu_writer_template.json deleted file mode 100644 index 6960765f86..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/kudu_writer_template.json +++ /dev/null @@ -1,59 +0,0 @@ -{ - "job": { - "content": [ - { - "writer": { - "parameter": { - "column": [ - { - "name": "id", - "type": "long" - }, - { - "name": "user_id", - "type": "long" - }, - { - "name": "name", - "type": "string" - } - ], - "masterAddresses": "impala1:7051,impala2:7051,impala3:7051", - "table": "kudu_range_table_write", - "writeMode": "insert" - }, - "name": "kuduwriter" - }, - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "long" - }, - { - "name": "user_id", - "type": "long" - }, - { - "name": "name", - "type": "string" - } - ], - "sliceRecordCount": ["10000"] - }, - "name": "streamreader" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/perJob_test.json b/flinkx-test/src/main/resources/dev_test_job/perJob_test.json deleted file mode 100644 index 15d8bdc860..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/perJob_test.json +++ /dev/null @@ -1,35 +0,0 @@ -{ - "job": { - "content": [ - { - "writer": { - "parameter": { - "print": false - }, - "name": "streamwriter" - }, - "reader": { - "parameter": { - "column": [ - { - "name": "id", - "type": "id" - } - ], - "sliceRecordCount": ["10000"] - }, - "name": "streamreader" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} diff --git a/flinkx-test/src/main/resources/dev_test_job/stream_es.json b/flinkx-test/src/main/resources/dev_test_job/stream_es.json deleted file mode 100644 index 563bd6dd6a..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/stream_es.json +++ /dev/null @@ -1,76 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "name1", - "type": "string" - } - ], - "splitPk": null, - "connection": [ - { - "password": "123456", - "jdbcUrl": [ - "jdbc:mysql://172.16.8.193:3306/jiangbo" - ], - "table": [ - "es_reader" - ], - "username": "root" - } - ], - "password": "123456", - "username": "root" - } - }, - "writer": { - "name": "eswriter", - "parameter": { - "address": "172.16.10.190:9200,172.16.10.193:9200,172.16.10.147:9200", - "index": "scroll_test", - "type": "test", - "bulkAction": 100, - "idColumn": [], - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - }, - { - "name": "name1", - "type": "string" - } - ] - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 0, - "percentage": 20 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/stream_hdfs.json b/flinkx-test/src/main/resources/dev_test_job/stream_hdfs.json deleted file mode 100644 index af18d419d0..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/stream_hdfs.json +++ /dev/null @@ -1,247 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "column" : [ { - "name" : "id", - "type" : "string" - }, { - "name" : "deptno", - "type" : "INT" - }], - "sliceRecordCount": ["1000"], - "print": true - }, - "name" : "streamreader" - }, - "writer" : { - "parameter" : { - "fileName" : "pt=1", - "column" : [ { - "name" : "id", - "index" : 0, - "type" : "string", - "key" : "id" - }, { - "name" : "deptno", - "index" : 1, - "type" : "int", - "key" : "deptno" - }], - "writeMode" : "overwrite", - "fieldDelimiter" : "\u0001", - "encoding" : "utf-8", - "fullColumnName" : [ "id", "deptno"], - "path" : "hdfs://cdh3.cdhsite:8020/jiangbo/orc", - "partition" : "pt=1", - "hadoopConfig" : { - "java.security.krb5.conf": "krb5.conf", - "hive.exec.reducers.bytes.per.reducer" : "67108864", - "hive.optimize.reducededuplication" : "true", - "hadoop.proxyuser.mapred.hosts" : "*", - "yarn.application.classpath" : "$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*", - "dfs.replication" : "3", - "hive.exec.copyfile.maxsize" : "33554432", - "hive.vectorized.execution.enabled" : "true", - "hive.metastore.sasl.enabled" : "true", - "yarn.admin.acl" : "*", - "dfs.ha.fencing.ssh.private-key-files" : "~/.ssh/id_rsa", - "yarn.scheduler.increment-allocation-vcores" : "1", - "hadoop.security.auth_to_local" : "DEFAULT", - "yarn.nodemanager.remote-app-log-dir-suffix" : "logs", - "hive.limit.pushdown.memory.usage" : "0.1", - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "yarn.resourcemanager.webapp.address" : "cdh3.cdhsite:8088", - "dfs.journalnode.rpc-address" : "0.0.0.0:8485", - "hive.merge.mapredfiles" : "false", - "yarn.scheduler.maximum-allocation-vcores" : "8", - "dfs.namenode.rpc-address.ns1.nn2" : "node2:9000", - "hive.map.aggr" : "true", - "spark.dynamicAllocation.minExecutors" : "1", - "dfs.namenode.rpc-address.ns1.nn1" : "node1:9000", - "spark.yarn.executor.memoryOverhead" : "100", - "hadoop.proxyuser.HTTP.hosts" : "*", - "hive.smbjoin.cache.rows" : "10000", - "hadoop.proxyuser.httpfs.hosts" : "*", - "dfs.encrypt.data.transfer.cipher.suites" : "AES/CTR/NoPadding", - "hive.auto.convert.join" : "true", - "hive.merge.mapfiles" : "true", - "hive.server2.authentication.kerberos.principal" : "hive/cdh3.cdhsite@DTSTACK.COM", - "dfs.namenode.acls.enabled" : "false", - "yarn.scheduler.increment-allocation-mb" : "512", - "dfs.encrypt.data.transfer.cipher.key.bitlength" : "256", - "yarn.resourcemanager.nm.liveness-monitor.interval-ms" : "1000", - "remoteDir" : "/home/admin//RDOS_1304", - "hadoop.proxyuser.mapred.groups" : "*", - "hive.metastore.execute.setugi" : "true", - "hadoop.security.group.mapping" : "org.apache.hadoop.security.ShellBasedUnixGroupsMapping", - "net.topology.script.file.name" : "/etc/hadoop/conf.cloudera.yarn/topology.py", - "hadoop.ssl.keystores.factory.class" : "org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory", - "hive.zookeeper.quorum" : "cdh2", - "principalFile" : "/home/admin/app/dt-center-ide/kerberosConf/RDOS_1304/yijing.keytab", - "spark.dynamicAllocation.enabled" : "true", - "hive.metastore.kerberos.principal" : "hive/_HOST@DTSTACK.COM", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "yarn.resourcemanager.admin.address" : "cdh3.cdhsite:8033", - "yarn.resourcemanager.resource-tracker.address" : "cdh3.cdhsite:8031", - "hive.merge.sparkfiles" : "true", - "dfs.domain.socket.path" : "/var/run/hdfs-sockets/dn", - "dfs.datanode.kerberos.principal" : "hdfs/_HOST@DTSTACK.COM", - "hadoop.ssl.enabled" : "false", - "hadoop.proxyuser.hdfs.groups" : "*", - "dfs.namenode.name.dir" : "file:///data/dfs/nn", - "dfs.client.use.datanode.hostname" : "false", - "hive.compute.query.using.stats" : "false", - "dfs.namenode.kerberos.principal" : "hdfs/_HOST@DTSTACK.COM", - "fs.trash.interval" : "1", - "spark.dynamicAllocation.initialExecutors" : "1", - "yarn.scheduler.minimum-allocation-vcores" : "1", - "dfs.client.domain.socket.data.traffic" : "false", - "yarn.resourcemanager.admin.client.thread-count" : "1", - "dfs.block.access.token.enable" : "true", - "hadoop.proxyuser.hive.hosts" : "*", - "spark.executor.memory" : "599837900", - "yarn.resourcemanager.scheduler.class" : "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler", - "hive.vectorized.groupby.checkinterval" : "4096", - "dfs.blocksize" : "134217728", - "hadoop.security.instrumentation.requires.admin" : "false", - "hive.map.aggr.hash.percentmemory" : "0.5", - "io.file.buffer.size" : "65536", - "yarn.resourcemanager.container.liveness-monitor.interval-ms" : "600000", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem", - "hadoop.proxyuser.oozie.hosts" : "*", - "hadoop.tmp.dir" : "/data/hadoop_${user.name}", - "dfs.journalnode.edits.dir" : "/opt/dtstack/hadoop/journal", - "md5zip" : "789edf8a8ba7d370c8e8860aefec8c98", - "hadoop.proxyuser.oozie.groups" : "*", - "yarn.resourcemanager.scheduler.address" : "cdh3.cdhsite:8030", - "hadoop.proxyuser.yarn.hosts" : "*", - "hive.vectorized.groupby.flush.percent" : "0.1", - "dfs.namenode.servicerpc-address" : "cdh3.cdhsite:8022", - "hive.metastore.client.socket.timeout" : "300", - "hive.optimize.sort.dynamic.partition" : "false", - "hadoop.ssl.require.client.cert" : "false", - "mapred.reduce.tasks" : "-1", - "yarn.resourcemanager.address" : "cdh3.cdhsite:8032", - "dfs.client.read.shortcircuit.skip.checksum" : "false", - "dfs.namenode.kerberos.principal.pattern" : "*", - "hive.vectorized.execution.reduce.enabled" : "false", - "yarn.resourcemanager.resource-tracker.client.thread-count" : "50", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : true, - "dfs.safemode.threshold.pct" : "0.5", - "hive.fetch.task.conversion" : "minimal", - "hive.server2.authentication" : "kerberos", - "yarn.acl.enable" : "true", - "dfs.journalnode.http-address" : "0.0.0.0:8480", - "hadoop.security.authorization" : "true", - "yarn.nm.liveness-monitor.expiry-interval-ms" : "600000", - "hive.metastore.warehouse.dir" : "/user/hive/warehouse", - "yarn.am.liveness-monitor.expiry-interval-ms" : "600000", - "hive.execution.engine" : "mr", - "hadoop.proxyuser.httpfs.groups" : "*", - "spark.driver.memory" : "966367641", - "hive.cbo.enable" : "false", - "yarn.resourcemanager.client.thread-count" : "50", - "spark.yarn.driver.memoryOverhead" : "102", - "hadoop.security.authentication" : "kerberos", - "openKerberos" : false, - "hadoop.proxyuser.hdfs.hosts" : "*", - "hive.optimize.reducededuplication.min.reducer" : "4", - "dfs.client.use.legacy.blockreader" : "false", - "hadoop.proxyuser.hue.hosts" : "*", - "hive.metastore.uris" : "thrift://cdh3:9083", - "hadoop.proxyuser.yarn.groups" : "*", - "yarn.resourcemanager.am.max-attempts" : "2", - "yarn.resourcemanager.max-completed-applications" : "10000", - "dfs.namenode.kerberos.internal.spnego.principal" : "HTTP/_HOST@DTSTACK.COM", - "hadoop.proxyuser.admin.hosts" : "*", - "hadoop.proxyuser.hue.groups" : "*", - "io.compression.codecs" : "org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec", - "hadoop.proxyuser.hive.groups" : "*", - "hive.cluster.delegation.token.store.class" : "org.apache.hadoop.hive.thrift.MemoryTokenStore", - "hive.server2.authentication.kerberos.keytab" : "/data/hive/conf/hive.keytab", - "hive.zookeeper.client.port" : "2181", - "fs.defaultFS" : "hdfs://cdh3.cdhsite:8020", - "hadoop.ssl.client.conf" : "ssl-client.xml", - "hadoop.proxyuser.admin.groups" : "*", - "dfs.ha.fencing.methods" : "sshfence", - "yarn.resourcemanager.scheduler.client.thread-count" : "50", - "hadoop.proxyuser.flume.hosts" : "*", - "hive.server2.logging.operation.log.location" : "/var/log/hive/operation_logs", - "sftpConf" : { - "path" : "/home/admin/", - "password" : "abc123", - "port" : "22", - "auth" : "1", - "host" : "172.16.8.193", - "username" : "admin" - }, - "hive.merge.size.per.task" : "268435456", - "dfs.https.port" : "50470", - "principal" : "yijing@DTSTACK.COM", - "dfs.encrypt.data.transfer.algorithm" : "3des", - "dfs.client.read.shortcircuit" : "false", - "hive.merge.smallfiles.avgsize" : "16777216", - "dfs.namenode.http-address" : "cdh3.cdhsite:50070", - "dfs.datanode.hdfs-blocks-metadata.enabled" : "true", - "hive.exec.reducers.max" : "1099", - "hive.fetch.task.conversion.threshold" : "268435456", - "hadoop.ssl.server.conf" : "ssl-server.xml", - "yarn.scheduler.minimum-allocation-mb" : "1024", - "yarn.resourcemanager.principal" : "yarn/_HOST@DTSTACK.COM", - "hive.auto.convert.join.noconditionaltask.size" : "20971520", - "hive.server2.logging.operation.enabled" : "true", - "dfs.https.address" : "cdh3.cdhsite:50470", - "dfs.namenode.shared.edits.dir" : "qjournal://node1:8485;node2:8485;node3:8485/namenode-ha-data", - "hive.support.concurrency" : "true", - "yarn.resourcemanager.amliveliness-monitor.interval-ms" : "1000", - "yarn.nodemanager.remote-app-log-dir" : "/tmp/logs", - "spark.executor.cores" : "4", - "yarn.scheduler.maximum-allocation-mb" : "5616", - "hive.warehouse.subdir.inherit.perms" : "true", - "spark.dynamicAllocation.maxExecutors" : "2147483647", - "hive.optimize.bucketmapjoin.sortedmerge" : "false", - "yarn.resourcemanager.webapp.https.address" : "cdh3.cdhsite:8090", - "hive.server2.enable.doAs" : "true", - "dfs.namenode.http-address.ns1.nn2" : "node2:50070", - "hadoop.rpc.protection" : "authentication", - "dfs.namenode.http-address.ns1.nn1" : "node1:50070", - "fs.permissions.umask-mode" : "022", - "spark.shuffle.service.enabled" : "true", - "dfs.nfs.kerberos.principal" : "hdfs/_HOST@DTSTACK.COM", - "hadoop.proxyuser.flume.groups" : "*", - "hive.zookeeper.namespace" : "hive_zookeeper_namespace_hive", - "dfs.ha.automatic-failover.enabled" : "true", - "hadoop.proxyuser.HTTP.groups" : "*" - }, - "defaultFS" : "hdfs://cdh3.cdhsite:8020", - "connection" : [ { - "jdbcUrl" : "jdbc:hive2://cdh3:10000/kerberos_xq;principal=hive/cdh3.cdhsite@DTSTACK.COM", - "table" : [ "employee_0" ] - } ], - "fileType" : "orc", - "sourceIds" : [ 1304 ], - "fullColumnType" : [ "string", "int"] - }, - "name" : "hdfswriter" - } - } ], - "setting" : { - "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 - }, - "speed" : { - "bytes" : 0, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-test/src/main/resources/dev_test_job/stream_template.json b/flinkx-test/src/main/resources/dev_test_job/stream_template.json deleted file mode 100644 index 7555ac87c7..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/stream_template.json +++ /dev/null @@ -1 +0,0 @@ -{"job":{"content":[{"reader":{"parameter":{"password":"DT@Stack#123","customSql":"","column":[{"name":"id","type":"INT","key":"id"},{"name":"name","type":"VARCHAR","key":"name"},{"name":"length","type":"BIGINT","key":"length"},{"name":"salary","type":"DOUBLE","key":"salary"},{"name":"birthday","type":"DATETIME","key":"birthday"}],"connection":[{"sourceId":13,"password":"DT@Stack#123","jdbcUrl":["jdbc:mysql://172.16.101.136:3306/db_dtinsight_test"],"type":1,"table":["t_dtinsight_test"],"username":"drpeco"}],"sourceIds":[13],"username":"drpeco"},"name":"mysqlreader"},"writer":{"parameter":{"fileName":"pt=20200308","column":[{"name":"id","index":0,"type":"bigint","key":"id"},{"name":"name","index":1,"type":"string","key":"name"},{"name":"length","index":2,"type":"bigint","key":"length"},{"name":"salary","index":3,"type":"double","key":"salary"},{"name":"birthday","index":4,"type":"string","key":"birthday"}],"writeMode":"overwrite","fieldDelimiter":"\u0001","encoding":"utf-8","fullColumnName":["id","name","length","salary","birthday"],"path":"hdfs://ns1/dtInsight/hive/warehouse/dt_batch.db/t_dtinsight_test0305","partition":"pt=20200308","hadoopConfig":{"fs.defaultFS":"hdfs://ns1","hadoop.proxyuser.admin.groups":"*","dfs.replication":"3","dfs.ha.fencing.methods":"sshfence","dfs.client.failover.proxy.provider.ns1":"org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider","dfs.ha.fencing.ssh.private-key-files":"~/.ssh/id_rsa","fs.hdfs.impl.disable.cache":true,"dfs.nameservices":"ns1","dfs.safemode.threshold.pct":"0.5","dfs.ha.namenodes.ns1":"nn1,nn2","dfs.namenode.name.dir":"file:/data/hadoop/hdfs/name","dfs.journalnode.rpc-address":"0.0.0.0:8485","fs.trash.interval":"14400","dfs.journalnode.http-address":"0.0.0.0:8480","dfs.namenode.rpc-address.ns1.nn2":"172.16.101.136:9000", "dfs.namenode.rpc-address.ns1.nn1":"172.16.100.216:9000","dfs.datanode.data.dir":"file:/data/hadoop/hdfs/data","dfs.namenode.shared.edits.dir":"qjournal://172.16.100.216:8485;172.16.101.136:8485/namenode-ha-data","openKerberos":false,"fs.hdfs.impl":"org.apache.hadoop.hdfs.DistributedFileSystem","hadoop.tmp.dir":"/data/hadoop_admin","dfs.journalnode.edits.dir":"/data/hadoop/hdfs/journal","dfs.namenode.http-address.ns1.nn2":"172.16.101.136:50070","dfs.namenode.datanode.registration.ip-hostname-check":"false","dfs.namenode.http-address.ns1.nn1":"172.16.100.216:50070","hadoop.proxyuser.admin.hosts":"*","md5zip":"62f8b316522645169c47e154cd1277ff","dfs.ha.automatic-failover.enabled":"true"},"defaultFS":"hdfs://ns1","connection":[{"jdbcUrl":"jdbc:hive2://172.16.101.227:10000/dt_batch","table":["t_dtinsight_test0305"]}],"fileType":"orc","sourceIds":[11],"fullColumnType":["bigint","string","bigint","double","string"]},"name":"hdfswriter"}}],"setting":{"restore":{"maxRowNumForCheckpoint":0,"isRestore":false,"restoreColumnName":"","restoreColumnIndex":0},"errorLimit":{"record":100},"speed":{"bytes":0,"channel":1}}}} \ No newline at end of file diff --git a/pom.xml b/pom.xml index ffa2cd5a5a..1b01439433 100644 --- a/pom.xml +++ b/pom.xml @@ -13,7 +13,6 @@ flinkx-core flinkx-launcher - flinkx-test flinkx-examples flinkx-stream @@ -65,7 +64,7 @@ 2.7.3 4.5.3 ${basedir}/dev - 1.8.4_rc0 + release_1.8.5 sh
@@ -137,25 +136,6 @@ - - - - - - - - - - - - - release - - ${projectVersion} - - - - @@ -188,28 +168,6 @@ flinkx-java-docs - - pl.project13.maven - git-commit-id-plugin - 2.2.6 - - - - revision - - - - - yyyy.MM.dd HH:mm:ss - true - true - - false - -dirty - false - - - From 78090d7b802f03a502f55aae356f9e9ec2a22971 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 16:27:33 +0800 Subject: [PATCH 029/136] merge release --- .../main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java | 2 -- .../mongodb/oplog/reader/MongodbOplogInputFormat.java | 7 +------ 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java index 6015ab1427..425dc21aa4 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java @@ -41,8 +41,6 @@ */ public class MongodbUtil { - private static final Logger LOG = LoggerFactory.getLogger(MongodbUtil.class); - public static Document convertRowToDoc(Row row,List columns) throws WriteRecordException { Document doc = new Document(); for (int i = 0; i < columns.size(); i++) { diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java index 25765d9cb1..a6f4fb7662 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java @@ -64,11 +64,6 @@ public class MongodbOplogInputFormat extends RichInputFormat { private InputSplit inputSplit; - @Override - public void configure(Configuration parameters) { - // do nothing - } - @Override protected void openInternal(InputSplit inputSplit) throws IOException { this.inputSplit = inputSplit; @@ -190,7 +185,7 @@ protected void closeInternal() throws IOException { } @Override - public InputSplit[] createInputSplits(int minNumSplits) throws IOException { + public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { return new InputSplit[]{new GenericInputSplit(1,1)}; } From 9b4b4d6f4f0efaa1942683ce5355adb11c9bd92b Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 17:37:35 +0800 Subject: [PATCH 030/136] add docs --- README.md | 1 + docs/mongodb_oplog.md | 175 ++++++++++++++++++ .../dtstack/flinkx/mongodb/MongodbConfig.java | 30 ++- .../oplog/reader/MongodbEventHandler.java | 41 ++-- .../oplog/reader/MongodbOperation.java | 20 +- .../oplog/reader/MongodbOplogInputFormat.java | 8 +- .../resources/dev_test_job/oplog_stream.json | 4 +- 7 files changed, 257 insertions(+), 22 deletions(-) create mode 100644 docs/mongodb_oplog.md diff --git a/README.md b/README.md index 6baa51f0f0..e0a80d7c33 100644 --- a/README.md +++ b/README.md @@ -291,6 +291,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [Emqx读取插件](docs/emqxreader.md) * [Oracle实时采集插件](docs/logminer.md) * [SqlServerCdc实时采集插件](docs/sqlservercdc.md) +* [MongoDB实时采集插件](docs/mongodb_oplog.md) ### 5.2 写入插件 diff --git a/docs/mongodb_oplog.md b/docs/mongodb_oplog.md new file mode 100644 index 0000000000..7c8c62a410 --- /dev/null +++ b/docs/mongodb_oplog.md @@ -0,0 +1,175 @@ +# Mongodb实时采集插件(mongodboplogreader) + +## 1. 配置样例 + +``` +{ + "job": { + "content": [ + { + "reader": { + "name": "mongodboplogreader", + "parameter": { + "hostPorts": "127.0.0.1:30001,127.0.0.1:30002,127.0.0.1:30003", + "username": "root", + "password": "123456", + "database": "admin", + "clusterMode": "REPLICA_SET", + "authenticationMechanism": "SCRAM-SHA-256", + "monitorDatabases": ["test"], + "monitorCollections":[], + "operateType":["insert","update","delete"], + "pavingData":true, + "excludeDocId": false + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 1048576 + }, + "errorLimit": { + "record": 100 + }, + "restore" : { + "isRestore" : true, + "isStream" : true + } + } + } +} +``` + +## 2. 参数说明 + +* **name** + + * 描述:插件名,此处填写插件名称,oraclelogminerreader。 + + * 必选:是 + + * 默认值:无 + +* **hostPorts** + + * 描述:Mongodb集群地址。 + + * 必选:是 + + * 默认值:无 + +* **username** + + * 描述: 用户名。 + + * 必选:是 + + * 默认值:无 + +* **password** + + * 描述: 密码。 + + * 必选:是 + + * 默认值:无 + +* **authenticationMechanism** + + - 描述: 认证机制,可选:GSSAPI、PLAIN、MONGODB-X509、MONGODB-CR、SCRAM-SHA-1、SCRAM-SHA-256 + + - 必选:否 + + - 默认值:无 + +* **clusterMode** + + - 描述: 集群模式,可选:REPLICA_SET、MASTER_SLAVE + + - 必选:是 + + - 默认值:无 + +* **monitorDatabases** + + - 描述: 要监听的库 + + - 必选:否 + + - 默认值:无 + +* **monitorCollections** + + * 描述:要监听的集合 + + * 必选:否 + + * 默认值:无 + +* **operateType** + + * 描述:要监听的操作类型,可选:insert、update、delete + + * 必选:否 + + * 默认值:无 + +* **excludeDocId** + + * 描述:是否排除_id字段 + + * 必选:否 + + * 默认值:false + +* **pavingData** + + * 描述:是否将解析出的json数据拍平 + + * 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: + + ```json + { + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before_id":1, + "after_id":2 + } + ``` + + pavingData为false时: + + ```json + { + "message":{ + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before_id":{ + "id":1 + }, + "after_id":{ + "id":2 + } + } + } + ``` + + 其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 + + * 必选:否 + + * 默认值:false \ No newline at end of file diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java index 2af0a66f5f..41bd012452 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbConfig.java @@ -54,6 +54,10 @@ public class MongodbConfig implements Serializable { private List monitorCollections; + private List operateType; + + private boolean pavingData; + private String clusterMode; private int startLocation; @@ -261,14 +265,29 @@ public void setMongodbConfig(ConnectionConfig mongodbConfig) { this.mongodbConfig = mongodbConfig; } + public List getOperateType() { + return operateType; + } + + public void setOperateType(List operateType) { + this.operateType = operateType; + } + + public boolean getPavingData() { + return pavingData; + } + + public void setPavingData(boolean pavingData) { + this.pavingData = pavingData; + } + @Override public String toString() { - // TODO 密码脱敏 return "MongodbConfig{" + "hostPorts='" + hostPorts + '\'' + ", url='" + url + '\'' + ", username='" + username + '\'' + - ", password='" + "******" + '\'' + + ", password='******" + '\'' + ", authenticationMechanism='" + authenticationMechanism + '\'' + ", database='" + database + '\'' + ", collectionName='" + collectionName + '\'' + @@ -276,6 +295,13 @@ public String toString() { ", fetchSize=" + fetchSize + ", writeMode='" + writeMode + '\'' + ", replaceKey='" + replaceKey + '\'' + + ", monitorDatabases=" + monitorDatabases + + ", monitorCollections=" + monitorCollections + + ", operateType=" + operateType + + ", pavingData=" + pavingData + + ", clusterMode='" + clusterMode + '\'' + + ", startLocation=" + startLocation + + ", excludeDocId=" + excludeDocId + ", mongodbConfig=" + mongodbConfig + '}'; } diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java index e0737e9c2a..914b07c812 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java @@ -19,13 +19,12 @@ package com.dtstack.mongodb.oplog.reader; +import com.dtstack.flinkx.util.SnowflakeIdWorker; import org.apache.flink.types.Row; import org.bson.BsonTimestamp; import org.bson.Document; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.atomic.AtomicLong; /** @@ -39,7 +38,9 @@ public class MongodbEventHandler { public final static String EVENT_KEY_TS = "ts"; public final static String EVENT_KEY_DATA = "o"; - public static Row handleEvent(final Document event, AtomicLong offset, boolean excludeDocId){ + private static SnowflakeIdWorker idWorker = new SnowflakeIdWorker(1, 1); + + public static Row handleEvent(final Document event, AtomicLong offset, boolean excludeDocId, boolean pavingData){ MongodbOperation mongodbOperation = MongodbOperation.getByInternalNames(event.getString(EVENT_KEY_OP)); Map eventMap = new LinkedHashMap<>(); eventMap.put("type", mongodbOperation.name()); @@ -47,8 +48,7 @@ public static Row handleEvent(final Document event, AtomicLong offset, boolean e parseDbAndCollection(event, eventMap); BsonTimestamp timestamp = event.get(EVENT_KEY_TS, BsonTimestamp.class); - eventMap.put("ts", timestamp.getValue()); - eventMap.put("ingestion", System.nanoTime()); + eventMap.put("ts", idWorker.nextId()); final Document data = (Document)event.get(EVENT_KEY_DATA); Set keys = data.keySet(); @@ -56,18 +56,37 @@ public static Row handleEvent(final Document event, AtomicLong offset, boolean e keys.remove("_id"); } - for (String key : keys) { - eventMap.put("after_" + key, data.get(key)); - } + if (pavingData) { + for (String key : keys) { + eventMap.put("after_" + key, data.get(key)); + } - for (String key : keys) { - eventMap.put("before_" + key, null); + for (String key : keys) { + eventMap.put("before_" + key, null); + } + } else { + eventMap.put("before", processColumnList(keys, data, true)); + eventMap.put("after", processColumnList(keys, data, false)); + eventMap = Collections.singletonMap("message", eventMap); } offset.set(timestamp.getValue()); return Row.of(eventMap); } + private static Map processColumnList(Set keys, Document data, boolean valueNull) { + Map map = new HashMap<>(keys.size()); + for (String key : keys) { + if (valueNull) { + map.put(key, null); + } else { + map.put(key, data.get(key)); + } + } + + return map; + } + private static void parseDbAndCollection(final Document event, Map eventMap){ String dbCollection = event.getString(EVENT_KEY_NS); String[] split = dbCollection.split("\\."); diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java index 5206630f79..c5f80add50 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java @@ -1,5 +1,6 @@ package com.dtstack.mongodb.oplog.reader; + import java.util.ArrayList; import java.util.List; @@ -34,13 +35,24 @@ public String getInternalName() { return internalName; } - public static List internalNames(){ - List names = new ArrayList<>(); + public static List getInternalNames(List names) { + List internalNames = new ArrayList(names.size()); + for (String name : names) { + MongodbOperation operation = getByName(name); + internalNames.add(operation.getInternalName()); + } + + return internalNames; + } + + public static MongodbOperation getByName(String name) { for (MongodbOperation value : MongodbOperation.values()) { - names.add(value.getInternalName()); + if (value.name().equalsIgnoreCase(name)){ + return value; + } } - return names; + throw new RuntimeException("不支持的操作类型:" + name); } public static MongodbOperation getByInternalNames(String name){ diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java index a6f4fb7662..d2f103171d 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java @@ -31,7 +31,6 @@ import com.mongodb.client.model.Filters; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -127,7 +126,10 @@ private Bson buildFilter(){ filters.add(Filters.ne(MongodbEventHandler.EVENT_KEY_NS, "config.system.sessions")); // 过滤操作类型 - filters.add(Filters.in(MongodbEventHandler.EVENT_KEY_OP, MongodbOperation.internalNames())); + if(CollectionUtils.isNotEmpty(mongodbConfig.getOperateType())) { + List operateTypes = MongodbOperation.getInternalNames(mongodbConfig.getOperateType()); + filters.add(Filters.in(MongodbEventHandler.EVENT_KEY_OP, operateTypes)); + } return Filters.and(filters); } @@ -165,7 +167,7 @@ private String buildPattern() { @Override protected Row nextRecordInternal(Row row) throws IOException { - return MongodbEventHandler.handleEvent(cursor.next(), offset, mongodbConfig.getExcludeDocId()); + return MongodbEventHandler.handleEvent(cursor.next(), offset, mongodbConfig.getExcludeDocId(), mongodbConfig.getPavingData()); } @Override diff --git a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json index 7bfddb333f..bd01fd412c 100644 --- a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json +++ b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json @@ -5,13 +5,13 @@ "reader": { "name": "mongodboplogreader", "parameter": { - "hostPorts": "172.16.8.190:30001,172.16.8.190:30002,172.16.8.190:30003", + "hostPorts": "127.0.0.1:30001,127.0.0.1:30002,127.0.0.1:30003", "username": "root", "password": "123456", "database": "admin", "clusterMode": "REPLICA_SET", "authenticationMechanism": "SCRAM-SHA-256", - "monitorDatabases": ["jiangbo"], + "monitorDatabases": ["test"], "excludeDocId": false } }, From f08d4d0b0fb4c39db7c690997752b1919eb07e3c Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 18:08:34 +0800 Subject: [PATCH 031/136] Merge branch 'feature_1.8_oplogs' into github_release_1.8.5 --- README.md | 57 ++++++++++--------- README_CH.md | 56 +++++++++--------- .../oplog/reader/MongodbOplogInputFormat.java | 4 +- .../MongodbOplogInputFormatBuilder.java | 4 +- .../oplog/reader/MongodbOplogReader.java | 4 +- 5 files changed, 64 insertions(+), 61 deletions(-) diff --git a/README.md b/README.md index 08c026b0f0..a7a1f53c47 100644 --- a/README.md +++ b/README.md @@ -37,34 +37,35 @@ FlinkX is a data synchronization tool based on Flink. FlinkX can collect static The following databases are currently supported: -| | Database Type | Reader | Writer | -|:----------------------:|:-------------:|:------:|:------:| -| Batch Synchronization | MySQL | √ | √ | -| | Oracle | √ | √ | -| | SqlServer | √ | √ | -| | PostgreSQL | √ | √ | -| | DB2 | √ | √ | -| | GBase | √ | √ | -| | ClickHouse | √ | √ | -| | PolarDB | √ | √ | -| | SAP Hana | √ | √ | -| | Teradata | √ | √ | -| | Phoenix | √ | √ | -| | Cassandra | √ | √ | -| | ODPS | √ | √ | -| | HBase | √ | √ | -| | MongoDB | √ | √ | -| | Kudu | √ | √ | -| | ElasticSearch | √ | √ | -| | FTP | √ | √ | -| | HDFS | √ | √ | -| | Carbondata | √ | √ | -| | Redis | √ | | -| | Hive | | √ | -| Stream Synchronization | Kafka | √ | √ | -| | EMQX | √ | √ | -| | MySQL Binlog | √ | | -| | MongoDB Oplog | √ | | +| | Database Type | Reader | Writer | +|:----------------------:|:-------------:|:-------------------------------:|:-------------------------------:| +| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | +| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | +| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | +| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | +| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | +| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | +| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | +| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | +| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | +| | Redis | | [doc](docs/rediswriter.md) | +| | Hive | | [doc](docs/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | +| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/binlog.md) | | +| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | # Documentation diff --git a/README_CH.md b/README_CH.md index 2433d48876..a993e56562 100644 --- a/README_CH.md +++ b/README_CH.md @@ -37,33 +37,35 @@ FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集 FlinkX目前支持下面这些数据库: -| | 数据源类型 | Reader | Writer | -|:----:|:-------------:|:------:|:------:| -| 离线同步 | MySQL | √ | √ | -| | Oracle | √ | √ | -| | SqlServer | √ | √ | -| | PostgreSQL | √ | √ | -| | DB2 | √ | √ | -| | GBase | √ | √ | -| | ClickHouse | √ | √ | -| | PolarDB | √ | √ | -| | SAP Hana | √ | √ | -| | Teradata | √ | √ | -| | Phoenix | √ | √ | -| | Cassandra | √ | √ | -| | ODPS | √ | √ | -| | HBase | √ | √ | -| | MongoDB | √ | √ | -| | Kudu | √ | √ | -| | ElasticSearch | √ | √ | -| | FTP | √ | √ | -| | HDFS | √ | √ | -| | Carbondata | √ | √ | -| | Redis | √ | | -| | Hive | | √ | -| 实时采集 | Kafka | √ | √ | -| | EMQX | √ | √ | -| | MySQL Binlog | √ | | +| | Database Type | Reader | Writer | +|:----------------------:|:-------------:|:-------------------------------:|:-------------------------------:| +| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | +| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | +| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | +| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | +| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | +| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | +| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | +| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | +| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | +| | Redis | | [doc](docs/rediswriter.md) | +| | Hive | | [doc](docs/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | +| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/binlog.md) | | +| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | # 参考文档 diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java index d2f103171d..a1461eef51 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java @@ -19,7 +19,7 @@ package com.dtstack.mongodb.oplog.reader; -import com.dtstack.flinkx.inputformat.RichInputFormat; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.mongodb.MongodbClientUtil; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.restore.FormatState; @@ -47,7 +47,7 @@ * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogInputFormat extends RichInputFormat { +public class MongodbOplogInputFormat extends BaseRichInputFormat { private final static String OPLOG_DB = "local"; private final static String REPLICA_SET_COLLECTION = "oplog.rs"; diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java index 3519e2fe35..790b6e407b 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java @@ -19,14 +19,14 @@ package com.dtstack.mongodb.oplog.reader; -import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.mongodb.MongodbConfig; /** * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogInputFormatBuilder extends RichInputFormatBuilder { +public class MongodbOplogInputFormatBuilder extends BaseRichInputFormatBuilder { private MongodbOplogInputFormat format; diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java index e44d20aa67..e3873636ea 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java @@ -22,7 +22,7 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.mongodb.MongodbConfig; -import com.dtstack.flinkx.reader.DataReader; +import com.dtstack.flinkx.reader.BaseDataReader; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; @@ -31,7 +31,7 @@ * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogReader extends DataReader { +public class MongodbOplogReader extends BaseDataReader { private MongodbConfig mongodbConfig; From f7e346f764f6819a555eb5c60e819cc3aa508613 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 19:36:28 +0800 Subject: [PATCH 032/136] Merge branch 'feature_1.8_oplogs' into github_release_1.8.5 --- .../com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java index 8555ae2d6c..37846ffba0 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -25,12 +25,10 @@ import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang.StringUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.postgresql.jdbc.PgConnection; -import org.postgresql.replication.LogSequenceNumber; import java.io.IOException; import java.util.List; @@ -68,7 +66,7 @@ public class PgWalInputFormat extends RichInputFormat { private volatile boolean running = false; @Override - public void configure(Configuration parameters) { + public void openInputFormat() throws IOException{ executor = Executors.newFixedThreadPool(1); queue = new SynchronousQueue<>(true); } @@ -147,7 +145,7 @@ protected void closeInternal() throws IOException { } @Override - public InputSplit[] createInputSplits(int minNumSplits) throws IOException { + public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { InputSplit[] splits = new InputSplit[minNumSplits]; for (int i = 0; i < minNumSplits; i++) { splits[i] = new GenericInputSplit(i, minNumSplits); From 0699fff4126e758464fc7f32ec43168cf972cd85 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 19:52:59 +0800 Subject: [PATCH 033/136] Merge branch 'feature_1.8_oplogs' into github_release_1.8.5 --- README.md | 59 +++++++++--------- README_CH.md | 59 +++++++++--------- bin/install_jars.bat | 2 + bin/install_jars.sh | 2 + .../com/dtstack/flinkx/pgwal/PgWalUtil.java | 2 +- .../flinkx/pgwal/format/PgWalInputFormat.java | 4 +- .../pgwal/format/PgWalInputFormatBuilder.java | 4 +- .../flinkx/pgwal/reader/PgwalReader.java | 4 +- flinkx-test/pom.xml | 0 .../com/dtstack/flinkx/test/LocalTest.java | 0 .../flinkx/test/PluginNameConstrant.java | 0 .../resources/dev_test_job/oplog_stream.json | 40 ------------ jars/Dm7JdbcDriver18.jar | Bin 0 -> 1395146 bytes jars/readme.md | 8 ++- 14 files changed, 78 insertions(+), 106 deletions(-) delete mode 100644 flinkx-test/pom.xml delete mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java delete mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java delete mode 100644 flinkx-test/src/main/resources/dev_test_job/oplog_stream.json create mode 100644 jars/Dm7JdbcDriver18.jar diff --git a/README.md b/README.md index a7a1f53c47..0ab6323025 100644 --- a/README.md +++ b/README.md @@ -37,35 +37,36 @@ FlinkX is a data synchronization tool based on Flink. FlinkX can collect static The following databases are currently supported: -| | Database Type | Reader | Writer | -|:----------------------:|:-------------:|:-------------------------------:|:-------------------------------:| -| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | -| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | -| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | -| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | -| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | -| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | -| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | -| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | -| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | -| | Redis | | [doc](docs/rediswriter.md) | -| | Hive | | [doc](docs/hivewriter.md) | -| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | -| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | -| | MySQL Binlog | [doc](docs/binlog.md) | | -| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | +| | Database Type | Reader | Writer | +|:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| +| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | +| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | +| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | +| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | +| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | +| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | +| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | +| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | +| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | +| | Redis | | [doc](docs/rediswriter.md) | +| | Hive | | [doc](docs/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | +| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/binlog.md) | | +| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | +| | PostgreSQL WAL | [doc](docs/pgwalreader.md) | | # Documentation diff --git a/README_CH.md b/README_CH.md index a993e56562..6a8ccd289b 100644 --- a/README_CH.md +++ b/README_CH.md @@ -37,35 +37,36 @@ FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集 FlinkX目前支持下面这些数据库: -| | Database Type | Reader | Writer | -|:----------------------:|:-------------:|:-------------------------------:|:-------------------------------:| -| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | -| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | -| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | -| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | -| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | -| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | -| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | -| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | -| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | -| | Redis | | [doc](docs/rediswriter.md) | -| | Hive | | [doc](docs/hivewriter.md) | -| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | -| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | -| | MySQL Binlog | [doc](docs/binlog.md) | | -| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | +| | Database Type | Reader | Writer | +|:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| +| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | +| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | +| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | +| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | +| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | +| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | +| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | +| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | +| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | +| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | +| | Redis | | [doc](docs/rediswriter.md) | +| | Hive | | [doc](docs/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | +| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/binlog.md) | | +| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | +| | PostgreSQL WAL | [doc](docs/pgwalreader.md) | | # 参考文档 diff --git a/bin/install_jars.bat b/bin/install_jars.bat index 140dd108f8..c3910d5ff5 100644 --- a/bin/install_jars.bat +++ b/bin/install_jars.bat @@ -4,3 +4,5 @@ call mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -D call mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar +call mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar + diff --git a/bin/install_jars.sh b/bin/install_jars.sh index a43cd9a38c..cac9471eed 100644 --- a/bin/install_jars.sh +++ b/bin/install_jars.sh @@ -9,3 +9,5 @@ mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversi ## gbase driver mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar +## dm driver +mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java index 4dfb415795..bb325f5d93 100644 --- a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java @@ -206,7 +206,7 @@ public static PgConnection getConnection(String url, String username, String pas PGProperty.PREFER_QUERY_MODE.set(props, "simple"); //postgres version must > 10 PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "10"); - synchronized (ClassUtil.lock_str) { + synchronized (ClassUtil.LOCK_STR) { DriverManager.setLoginTimeout(10); // telnet TelnetUtil.telnet(url); diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java index 37846ffba0..e75b38a087 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx.pgwal.format; -import com.dtstack.flinkx.inputformat.RichInputFormat; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.pgwal.PgRelicationSlot; import com.dtstack.flinkx.pgwal.PgWalUtil; import com.dtstack.flinkx.pgwal.listener.PgWalListener; @@ -44,7 +44,7 @@ * * @author tudou */ -public class PgWalInputFormat extends RichInputFormat { +public class PgWalInputFormat extends BaseRichInputFormat { protected String username; protected String password; protected String url; diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java index e062690308..c32f225e55 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx.pgwal.format; -import com.dtstack.flinkx.inputformat.RichInputFormatBuilder; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -30,7 +30,7 @@ * * @author tudou */ -public class PgWalInputFormatBuilder extends RichInputFormatBuilder { +public class PgWalInputFormatBuilder extends BaseRichInputFormatBuilder { protected PgWalInputFormat format; diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java index 3f9c86a020..5dcfaf888c 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java @@ -22,7 +22,7 @@ import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.pgwal.PgWalConfigKeys; import com.dtstack.flinkx.pgwal.format.PgWalInputFormatBuilder; -import com.dtstack.flinkx.reader.DataReader; +import com.dtstack.flinkx.reader.BaseDataReader; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; @@ -35,7 +35,7 @@ * * @author tudou */ -public class PgwalReader extends DataReader { +public class PgwalReader extends BaseDataReader { private String username; private String password; private String url; diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstrant.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json b/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json deleted file mode 100644 index bd01fd412c..0000000000 --- a/flinkx-test/src/main/resources/dev_test_job/oplog_stream.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "name": "mongodboplogreader", - "parameter": { - "hostPorts": "127.0.0.1:30001,127.0.0.1:30002,127.0.0.1:30003", - "username": "root", - "password": "123456", - "database": "admin", - "clusterMode": "REPLICA_SET", - "authenticationMechanism": "SCRAM-SHA-256", - "monitorDatabases": ["test"], - "excludeDocId": false - } - }, - "writer": { - "name": "streamwriter", - "parameter": { - "print": true - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 1048576 - }, - "errorLimit": { - "record": 100 - }, - "restore" : { - "isRestore" : true, - "isStream" : true - } - } - } -} \ No newline at end of file diff --git a/jars/Dm7JdbcDriver18.jar b/jars/Dm7JdbcDriver18.jar new file mode 100644 index 0000000000000000000000000000000000000000..8e6e9f59b3468d5dd4f3e4c105b63d3bcf7cf9a9 GIT binary patch literal 1395146 zcma%i19WD=vTkhK{9{gR+qP{xnQ&rEj6b$*JDDUC+sVYX_2%4j?|J8~``%q|ukO9N zdVO8HR##QCwvsG31RMwm3Zho#@GyoC=s+Kr;bLv%cjKAFPw(SLmZXtMYy)`$}_?fw}nFA=Bq<-Avb1orNq*Wl}%&xC2`4Qo6xhR_;r z^KE_B8_HQF=!wrkZ)l3cu4RDpSm_+_`>Z@5N&LJvAIJ%Wu5hB|F68;}ujP~#S=4$TZVQp&e z%Jg4&nE$}1(N5uD{l($@&7}WVyp6GkF{7)q9iy0wwTHQj?AYRjjI_*>EX_E*EW@1Z zm^&zI+rO0e-}Masm#LZkzrOn40e@}d9{?LO6Vv|zh5TRWLo0{Of&}2 z-1;2+8Oq=VN#4d}T@2k(-j8}OV^@@VeE9M=x9ikP_FeAF!`nJC2(Q<9c(ie|vyv4j zLgP$bHMudiM{JL&>%6z=lK{cnuCbQG9l62rwL|0enic%KF$``O(}+Zs3-=Cu zAQ5%FexX?1_{(ku!L1b@DPJJksiMfs@RN%eD2(S}+bSN0GfsBKQx;`zRpJwQXPCW= zxsVDG$4dMDJeRCq;3L~7mJq9#s}$zNuFl_ptl8(7M)@kJpg1O0PH|K@gj>`Zc5gr% zwAvsMTyT(*pUSG1n*wu__=(p_xsm+ml2&kM^1JxETkt*25m3Z>Fc6iNdoKYbY#n1v zt)22IloVhd32V* zlbc$0th)BBUv*wF1q6J72QoaX34;$}hN}2fFz1)T$9YQ_z?M5gB{ibqbe%=yj&2de0Ydao4O#RJ?jrg0tb7K` zGaz>jcNmv%8r^M8N$y&0$^my6%$n{@tN|oH2EO1-SAlB!-Mr683#B@>l|z2sql(Q& z5>)Yb{Y$V1k&`Yt&$h!`Iff@qiVnuq@A)3lQCLG5cJRr&E=xEq1^Rp*Wo@wFYWiw{ z8|F%G>kVxRj-8^xSY^|^1f9PLmv>V7Yl^EZZ00b;28iy+Wz2gs=XM$okPrQU)OEV* z2&AF}=9A^ugK^wLRQBPPSoIq&0H7&yY}{ zVo2B-t)t4bSl9+d)1>&t%pbC2iZ(Exl-<{OqVgl`PRr1qW5q2lQR5ivj^o$y-TDp4 zlG|_J#HY)=gknN;zDpgZ1o=#kYpgzPXY(5LdISTZLWd>Q_#(n>EFNuDTUe*Eqhd%I zW~+9*M1~8!e(y2U=ZencXxtR#RimJyGFajPscjPZ4~3Tr9NM`%wJ?00M9H{H zPw?E8`lAsdcgi+Iy{mcr7Ci12T~S&z84ifp6WkM#sGD!t`_?sCP(@wYRyFJTn(nE)Y48P*kw&T>k zgdD$ObvsPFu&?O69Gx>ibt9vYhmbVJ|BPbekHO+S&#Eq=h2r?L@SsF+(h;&9N;%G1`=iu9^AG4fo~$x7CM@3 zsXTI;{Aq~FIuo_HB6Xg5{GA_oyc$l;5T${DV$O=Y7KOMHhhOxKMEL6-(o3mdV)i>> z&!oN>7Tn@KK}Vt?xi?XHZP5&+wFHSw;_2oOR+`0fcK@O=z;Sj-Dp5#8S2c-|b{zwDSt@jGhOJZlX#Qm^ zsI5kwT6m9@UJ2@%>X-FeZ4@|ewwze@D!!=J?o!>( z?47?urkxZ7kJ7m&6-YD^NZrVN2mjxT)PL+73=llfQ7J({x~oAzX#a21mYIpDv7Mc< ziJiHso3Wd@y}5(iKj*M!edsI<@uh&p=Ag~$RY(YcB*i(}Zz-i86m4ytB(f62#-4%0 zh+K`b%@~niJ$^Mo~PT5zO59N_%B2ODHaQD3Ysg)W|fo9Y*UlN&Q1qpPs!Bf zcT1$c7v z4od9V;@e~*SvalC^i?-8ad5DSRb<-W%>Y-F>*ou*S`AGN}k6k*X2NW-jdtMPQOLy&~yM$$>|7{jp&P#0s-?=x14 z+tY9qf4kVBSLd3m)7K6eRF)uMg(N^~#Bp18`Q;hHn!MUV>sYi!5p^Q=O|y?L{mPQ= z?JuW!dtLczTLfMoIf_Jb2m)Bim(_7^Uako>-y%f^yOwU^IXOxo1z1J?h@%me0rC`U zhHwOw+(pA!!s*&8^VCSr{^dESA+dvw0PRZ*)W`m>ajPOcP#dW%9Z;S7oB`O1AVTualJQ)fZl6v1nvRxarlN|ww0Ghov0bDeDPT9B0= zP|2J3eORwpX?=&e`np%Ur95K+U4iI3VuJpi*0#?Spy0cKL%@LtI^6S1R1tyr#4K7P z!m3e<8BH|k{XC5*Q`OZZ$nHZ|Xk9({&kZq+{Rx_(Cwz*_701uwnTeZD_=GJq?M za>yHjvMGL$ckKyB$3oD?p`sdJ9o2waflk3d{h}evN!1nOtxM0IWvkPncp-Ty)U^I) z82Tr#7PS1Dp4?;CkN*3-HHd`t<#5fp7N!q%sLi?7?`^EkE&heN<;mR}zoY2QXK3sZ zOjFdZ@IaRb-2H8T+HrMY?xpTzi+9#INAg6MD>N7~?Lf87dcvm3>#{~u1Wg)N>@vw; zZ`1E!JvR~uSXaKt+WX%wp%>0x>uj8s!jjMxLwLfhz+gSQRez1Pv)m0yBiu2oG&&#yU%tLi195SY)dkJeLkX3J3mka0i<5vdgFTo zBKmhPs3iu5_jptfv)3N$_mz24`Idss#iDIwOd_*s)e397zi177~)m{KDl zlUi+fN}Q30EJ=(Q_8B@>Ft*cUYhcrx{ve_pe*)hALIC}m`(4#nN?zRRL&;D|<{KOv zeN%ItVi8XQcB^tkBgM!K)lBCM)8{9R#S?vCZfK(gJok99>6 zoBCtj5ILjrr)FR6LiZ~u)pWm^ z5V~(W!mp{Q0|%RTy>Z2hJ#C>YVi_c>M>I`4t*n(V}f9#=DhA{gBF8E<-%XKerxy(&HBdC~%( z{&<%jq}Z2UHdbN?jWqPP$^>CC!-GNa#pL5n(i)U4ayj8RTRHl-7%iP=<8);=Uu>HR zL7h7ARUCUYazYOYUYZ0FPer)Xtu_>Is*ws?+j?(+UhY`o0!JLJt=8&>S?|fY!D@f7 zyiqvwajiNHVn;ROFdaxp&6;Ez$e#ow?Rl4T;ge;&f!XFq)kR+ zi79<}`}>Ef$`%w?sfhNA{YiZ@Hf#@x6Bj;S^}Sh|*|QMV$zbgSCwTync}k=otDbx& z(z@IVm9O^tX0Cwh$4R&iFPnwbGb7KdQbAUZH-98i4U@u?zVC9Jh3 zpkk4ic;-f;dB$YJ@bZa?P_e$){C2lETYqW(p|oV^&>#V7x-#ApqkQq*LNgn}t#r%; zjIs_3V#=LM;MpyFPy;?)Bz%}-NAb45)K|`VE)>|2_;~pCSd5p71cLMLM%s$L_ zHMiVR*x0DQ0CO;YYC&&x)i34UIAgq6`3~07@V4#}xU9DjJH_X~y17x`8_(%475CBd z^B01qcXj;i!`jcpW&iU4>cYLT(N=|#BF}BG?9IHufQ_(zmv>G`r+_Nt|~!gn6WR8?W6X zXnQ8Wyh&>Uh)g;T=FsL%D!O8?ue`O zJS%w33~}s+xZ$m|rPsMC>(Y^SQ~ZOo-`_Ie`31k!+4f!Dybg0KWvD?3ExI+SSGsPZ zvv6?YNHf_IXgB2v^PI}>RIO+1UU%zP8%NOBqEOGKlk6`4U7zfFee5nQDdbYgQ~FHd z2s5!uN{g^sp@-&J;jX^=SV@QS}IM=UMa$J5qV1od?hr+=9%iYs=;|E zlM0A!t2s|VMizo_%22X>-NGbGX zC1XT#a4V6OS15`FBgOgQO@L=cokYQUlSK3M!A8-Lh*Hr)#=bP8?ov;il-67GPrhP2 zZ@n~5V9lt1!5mJ9TVk=Zo9T7A0BxQ=s9nZM$s}+lwM+==I+B_&=&kmB%-O6pJw``M zb*-nZvTfc8+e|)R1=l^1o={@Y5-f$*X_H)obMPCk{tWT+6Rx0dRqLmbg6Jl6zp@lTUf5}UUC?PGP)kv%832qyLX&CHO-X`yeRlsxboR{S@tmXCPrM9(Ln z#D%8y%TmJ9AH6ebC{Zx~I_f;^`w|OdTFkwu;F&2)W3|72hH*YDWYqk&iN&u)X%xT} zDE!NR$E<2L2@`Cj-wS{21>{G?8Km3~={8~4cYM4)l1z8V`FMcmWM8hdnL&CGA%&ir(hks> z=LFQaW8%pWIsh)%&B_|f-3Z6N3xI2!J+4&!Qt@%=4y40+MoBvir(BFm7h>B2S^dExX8D z-V`D;t#sl~+r1OTYpKD#_$l5;cU_^6?`lhodX3d(FPr)6edu%)(hr39qMt|>|6CF7 z8Arew9{}}c2?wp6@_`B3+keFv!MkFJO&@K( z9zvVSv$Bm=6$tBFH@eXOS!0yF>flhhFE=vpi@Y>n>}=T|9{gbXp&&{GLL&p2K2M1B#AdoRSTdEkvhsJ|8C2EGLe!hGGmJoGasCaXx zv+k~J0W+MdGeV|w55+P~xf~kF*#k4hXIIU1S-BipLC}Lgi%4q zLK8__y=Xd1f9b8Y6@|>jVy_idTM5vyEeJ47&oP0}Kf&$MtRBMF`PK!T5+x_6Mt*FY zWR<+uCMYdxZrKWlHWa@K`Cvt#1=5;K9Lv>CbO|+y2TvctACE$w;+dkpBy^f-(?6rW zOYt%*tr5hsOT?yCTmq?sDj@}d;m#BXsCv@emvEFBnkt*{W1HD>ys>|jh>ajrdT->? zE^rd7m-Ok)F*t~E98omTe@_i^YQXu^OQwi_n6#6OP)eFCBD~Veu$T^0dlpLl96h+B z?4eELEPbaBeq!j?OzNhYQBv>D>PgSHu!R(4RF3t9u9!NYVm_`>Rg0SVc|?_9tXSfo zk>k9Vc}zRL7NySgb0NIGH}jkcwXx76qO)|V$fJ+w!ntF|vJTyzij}opR46l>!i&&% zO8>%cw{Dm_ICo;Yk(_J{D>{us!ZP*ETp$^B2Qt`k2YZ}3nbF6Afp+2myInV23?a;L zr1sZATB(kmbXx}xT68Rs&sv6U4F~gG25+W2GqrHs70+anPV)~XnM%?pIq4>Wmg0Fa z$&HW%L-z0*(g%xw_RjKk>@BZ>{SV&yOlN~$gKPPBsFMO#2!o6M6BJ4hT!nZBXFsRU z69T-A%pixzmI^HWZ2dw@!lPC=+q%W@qE7v5-Sat(#5)Dt-Z=5vN4xOVjT+$)t(? z$(DXpuwhL5Y@AhbbFJ;xv!Mvk{oRw$nh)vYxx3Q`x0cSOC0*ptcLc3t+KandNob@B zWY)tQz9#ayI)m(b?+3@Jqdcd7r3Bwow^#b<1}_p zo%GElnl_}Dh~{P351bnP$-*@bpWN#8VwxxmMvYx|fTKLn3Rm0NG`_17{*EQdwy|8i zEV$$meouRWl{R*98j>YTb8z#j__Vmv)y5!3n!CAn@20aqsX+Ufo~e*;Z?++|#s{dt z;Od)Tw)dYsOaCP9^5&JLZ^CCs%U%n58W%(AU-xF;xCg@h3KFfqK{$qC&8J(sZ^akk zTp<)n?I6cYDkhpi*JXuU_oAk&lVgC7+M6kD)4bwBkV;JW0b+Y|wWlHlr5btq@@hKrG!cnkamB9iyrgpuzdZpn2>kX(bWFz`W(8c;Tb(}C2QxoFTc#TN= zsS|Ro9Fd`%skY#XSY|t;i>v%oRhbi4NijHSvaGbM%oATpF=T$S%(AR(8XtOi)ndU_ zZQ*B3LryNo0%QCIlbc3$Q$GhE5;iKIV$jzn*9v3k1j@CGl$Uu$y9iP*-7amTh$bDJ zwmu@9{SSGI1;O$%3aEw2y5Sg~q$r-*Kp(md3yE%h@odYiz;(Vd^PHM4*--wam6Y{e zgH>l+5~4t(a2Y2Bqu`{SAP@-T3J6X}%40}m{3jOjKi3frY4uw_z|N|%&drI!srC&@ zu+Q1-W@Khk1j}c3=l!Pql3(dWD)E~v5oA(kpf0pi%$YI%9*xRSXu07x8C(Y^Ajduz zjB{eY95x3s84qdhsNdOn%j05nO)=|k1-HuUhnQXvsz8*_bD&)VaV5qW^OIit7;R330d}Mq-uSuj%n`Nz8cfF^nMc zz;jefZdz$3zzJa>z%UVm*?7*(Dl1eiu{*&m7X*+EkhbI_z@Z{PD@c?RO%w%>cS{5C zYfXayy44bgrCi(s;{rqy?@d3z((2#^&0%dMN68Uff&8h&nqg~J*=KXkER8vSQ+D?npx4I8W z@JrZs1>HvV(vM)$nv25El$CzlY;5RlB)Y!!a9KRJtku znvR3SVa9an`)zs`LftH6-VT$}u~LM=n@Cvfwqw+?@{5yfDo*32g={+;G=^(qRe6>p zeWF~S@JFr0fiKca4su^>?LaHBF4hmn_VFjk+wZu6j&?l66!s7+`3k zN>7$0filur<)A59qyoHZnUTuCNL=;`u~io&mOKTsNs9$_TAP%=&d3^Sjq<>S*}N2} z)33l*m{U}cV<%K?QVLXFrg0w0nm{OnPp5efDYK;hORXc6MXy4!p03JdeLMOFZbQgB z9plr(W3bJUC$TDSu@J4k5P+++TsQH%ERC+LT^nj4DV+u?w9E#9wzU-Wfz2Zu*S#=` zXDtxVcsWzqQAwFDw!z^B3S?Ct2tvY8#z_a-w_zF+Z5;+e(z#eQV_zY1h5%=8raM{Y zSy85mr^MOVIa&5pR;G(s*2>}tm<*|ansf2QqspW4lh7{@n|2Xo^C-p#G26P%NZUb8 z*;(#`psF7QN8!syaXUs$-@$xLUcFYVeD*cx;PXx8w-ma-w4AOiQ)QJbLwPW7tg0<5 zlf*Imp52XV1pZOdzW4NuDZ4?K@ZxpjKde-OGPlHoGD5f~@(z@4b(>yk-on{)ze8`%6 z|6LMRQ%E99mS3&bkoXCf`E|x$L6dJ!k2O}<4hg+}4{OH( z<8&NylZMACea!3(B?SuOeC*Zq6FAfOM6C;623sP(rghEmj5K=Gg}{HKY5cN6sZ_%3 z{?;gr(%iU8(pg@1h6r_nMG`Q$rZ3K}_o=f9mbFxsLwgzmrR}Tx{ddEoy0RhrueEhN zt5o;+sMliQrG8;E!h(WNi+Fb@6zyot1;5^^_#vN%Zg#%DBAk4=sCVP|Tthp9+Ny55 zTH6c4Kr`YLGq6Mgv_P|fy1{s1z3O5p-aBkz;ZIZsSa_bAS~i^RAlhx(ecesC9`7K% zPIirhPJ|am$52~hOYgy|Ed|xWj+Zu~Bm8AkNq#mHl0;FpM9qZ#=03n3z2jwCT9F#X zYA8Su#}lh%*QpaD!mL6>Wd`*Yo(FQJhu}c2hy)N`0U(bzGc%I!L0+JmS5G{Va!F6v zw=7i@fG?76mr;oZ&;{Fom1?JKiMFwe=7yS1sv+${;>$%MYov&CyVHa_Rb}icO)n}< z&*y$sztBD~j39;+KY-XKP}(;WoXf4g|GAiH*SurRwjPN#DGsSWfVM}FjQxinb7 z9>jhIP*sr^OM83=HDk*DQbAjWC&lD746;@vt*I`YnL_Q-p9wXy2((kjbc;n3V|>m8 zz2}l|sDSh7LK9_to&oV8qkjk_s!;fov=H=~;S9zrU9m(11gNVYtExZOB8b#3@0V5N zm6qpi;|fjB6#Xq*R^$od32|DnsHx91s?QiA_dscCY5Us?v`uQho#U4qmUo()+N zhYu69(XSiqFk8PadQ01;gMq(=_{deU6m?9L0q* zzLequ;f=1Vj=6}^Ma@=moV)A;D?nPhr5L3H4F{?}oOS z_wzWXf#crwcXN62v&@V7odLpU5?XxY}qc)lCmqt|Dc1Z(KnNxJQb?_VMT9fxjkWK7>^|LkA*%Bkqq#4D@{d}2(KeCxlU0l4 zBP92%x9jDchaW|omYwD*k~o#>VWEmiZbrx6T}gjQxtMmHh8{(^Sc=8-`SXI_#D3h* z<1TI}I6|FkffNUhxk+o#>?9K-{ygbmb<0SfMsh5itOHRX!OFcuzw%>U#enh(Lv-xs zpQ1jUpHL^SqiS(B`(>~z55Ijxl0eUPSIS-JCF69+!;?U4`HQ6kADsBAT^7hCMM01< zzHIml#{wRl%Dgwoq6q3i2v|NAeFUTey7FAAB;wdXxz0bmq?0~$E`K9t76xqiXdiHa zax1X8u8h?#RRZ^y-S{6PbZg5AO36k${x*JE^bt*Z`+HwE12j?fX~Yv3nDAmvZ4XHe0@&`bVK?eF?7PijAgr zF`mJ6k~F-=R%7GUfMNqlWFs^22G0dnl+pOtBrjq+Zf(BU2%U;At3fxg0=wjo|H6F` z2vCCysL=uRK{wzxJ&;VfFeae;34ue$vc34Kopg1oF5R7k$*WvUB~RNP^k#PSy-3U>VTqbK;OZav>`FXG7&F|LWyj6#Xe-f<2>#)+!Y(7yXMl zkf>1FO& zv5A+p!AqyYFIJLs;4e6ZMW*UAP$5OoE`x*&GCuKec&t)9LBuA>ZUt~1BP1sf1~$oV z4RAgM!gJ+*Kxj4oBPVueKob&n;zNn?MCsk2)ykY>HjtZEJ&rYDkDh!~&cAIxeR%SS zy~aNmhX9nVwL8x^xgM=Xjzd?#1Vp0Ozko>t_Z{?Q zL0CiNy~AKr!a@jBcvqu0AN22g;pxeN0*Si(_z@UJL!-a%Hfarw`4o4@s*Z0zikbnW z)ZC9!-glUEDH1X(Ldk%9~hXa7hn1VMeoi-pHc9zySLAY@1Rnt4v* za{CEo8T#rVodM^`@p&Kcn^m{_Vy`pXP6F_KcDF-BuSyVZNo$xz^^C*X@(W4h=3Ztb zJsdf5zI=XNPD*d!63!0;Re8>0z$M_)skGE;zaU>J54(0120|D>X zkif?Pmq*-vOg28dp*R7ATu(|YpWFk~mq>x5s>Zcl8a1zT<}b&lE@E*rh!0fZfLl;` zJ2+0;p2%Np;{oEox8(}?ZoWZm7rfw*CH)FL=7DeTN77~67BPdE6rJ%4A#|84fwTa* zh{(%ENZcwyJTwQ{Boyw%2Xt%SISWh7^uVl5lblQf9pPDmFC+6p5EHiwQrNz`R$1g= zq&{bvp#7y2)Oz=~O2)ge_c1RCy~Z|+UYG}(_P2IB_pyiq_Y3pOoDlw@LaaDeruN&t zNF&)xb|5mn+v0Ok4fVd@4k5Bj0yM5dQR!(`!Si*|Gi zH;J^2)sO$^-Rz#s%%UI-W7>c?;b1u=-;c zh$Y-hGkkAH=rK0=D}{vHkuKgRAYdH7<4BPg6vtP6nuaFsu9tSZ0i)y-N-S<7VbHM2 zV(-ojBFEBZ{=VaTHnf`&^=llF7cQ)<>>cGos9zGts45vADRYTDeCr7292eor{;dp#&}H_9 zVaF}NIV~WwKaYY2>O-NpiwMz2bmYRbL%4T3Pfxl+*t1+tG2cb;jXKR}4u2t&xMEK% zq(U%J=bqf(ZRvo)Xmz_ z;hzy@)w-~L_`{0Z|$`2t@n=otOvXru> zD1;51w3wMJ+JL_!&?3QC!^q1M5@iA@%t?HZPfqP3i$q;TPf$XtGuf-kDp`}G%)%zk zP+-U3cI^@|{5=CzIgtp1MU_)OojTYt6q$|Qn5h-!n4xq}SN@(5p2(33IkRnH zY7ghNXiwz{62{7l|8@#s!Hp~}4V#PHvhq45c$v+L@J#Y*9H_VA1LjQjY4Td5h7^p5 zX$}RVBDByT0ANy>W%{hjOm=e3jWHK5S*?{}nQ_RCa1bn+tGMxgHa$LhNF);GPig%S z_I4#Bh+=@FD7@da+2~mE%}K(7?1yF_asdlLT{1Jx_j+1@%nsje%b+}~hQL9|E3Jk^ z2+9W>w}9CgqeY>|h9@N$WX)uGEgK{>Wqk;+{k``$LqYJG-yredROnM%$q}YOLM|qc z<Z7?Rk&?U=3j=^k6fBhFW})gk6EPV5WAJo7xYGzQ_;FaLJ$J zpT!8dArf^q($Z0~_5^e@qeW#rvdw-Ak=I&aRt2yql`}w*C(Jw|cM^s}9aY0gzDBT_ z_vN%PuPVQ>uYWHBLV?CIW2YpsxqZue%EUE%M*EN|sUp-G+F6E;Tz%!eE zd=Xnh%tDBS=EVFS)^7zJV1BM)z(P+Uh!{oyu%l@>_ZT$V*OnpZgb*{z$Sh=8 z;AJN15u}$yH&g2~<`y0qfCHe{YgtlvWkEf{Ma=iXjrPe29;P_aCq&J(4MGB}H=L+Q z+{p~s7Nr#R*=h{?;DFKSO!JOm1$LB#M={jgxXCkdC{6%^gBBg^p(DH@}9meG>N+P( zpwb|@c&WEx!Qhtau;>IFhs%SFsuT zmN91`5$i+9lxaXW+;Wt7|LV)i`28!HXgSFVwX?A1P(yD7$Yq?_bR~3)b;|Ir9)-jTK?^G*%MZ#BLwZ4(hVTV%%~a<3rs?R=rtE`;4f^ zJ3PsjFi&Gf3am36KS_2d;F8rZak(Xo_fMSt@z=7!Z>c~7N{loQ3}mn)gccrWqNEJT zfW8R7nq&vPMEf$YvBJ1orI7vzUtHMR@hv;}Oq^luF#$$85guRFd3C7X3cSiVMe=o* z?eSK^n|{99f}-jm2YqIYSCp$Tja-{SfVFgo`!NvyE)H`P2QS)ql?-yWYucp&Yz?6^Mx<|xxKCO{(1Q&Osx-+TY^5W zswfzr<8L<8f=lLYRUQ+^rVp8fQD0*UpiyS3Ryc+_Vy-Wa9w#!fsl7bBZ*w_2}!OGLGTW!J^K-A+@A?9i*w=cN%{f)#2q z%Xp_Ih6&b^VuZfBR0grUJBO#ZF-AFyfLYzszg3+R{069G7`nf|W6Hv!sfEx86N!`!Qp=m!RW(dfLllTq1apTAa ztmOwnq#;9m&zt^K(B-?}Y8*<+rmEi(uhTEjD#q~48ppX9WS3n>^)u{9{1b8g_vtgaY*5<2XSzs zwyAKQBb)N6>rzWN6ObyQEY7+U2)>vAHaj@8e~W@Uj6#fQQcKytdXr%YGfRi z?^dOoMyoeio0UW_`FBR7$C4L!62fM*(QX|Nhd4#0mw|5l8_=c#dHXb~NqK_N(LCRG zFgvRr)c9etRu#ADvTk{#G<|xU*U0*b*uZthiW63DJdV?kWGUTY4p%%U6S{3b>wp?n zI3Z$uzuQ0%dC^ue9whTu&wFCu$|k97%NFE3tsesq1%yoDhPeqyR}#*twOj0?O6T&q z#`3yF^=ydyHn?zX3(610JlX8FiZdqA%tq_kp?GtIW0#wOMx!7XW*um!)oJ|-zq@lO zeS9J6xwZS~S2m4G3wZ5p=o{b~n$Oul3s+BESkJ%tSdg~HmAf%9-eF}9krD2mFPyQ2 zWDgn@gfUy<%gyus4S4n-3fXK9{}fTOk`eac=nQ0YA`SW8j5m{rnHTv-KoxTx143po zvr6W8-?DDve(e`GRw73Y$_!XF)SywFJt>>Z4U_ru%tGxXr#qp#uqv{n$!Ujbbzk{< znkQR!CPb4cv<1Ss(`JS85d&CipGsq+?nRfrP{c;*kBw1>?!Vi_4ohHt9IBRAB;TFPSHM9N+^jSqj35S$un zjft1HY}s^$J`(%#M9k*eetd& zkab7yuEp$24IqurZxbL{*Q?~qjmU{^Q=$vy%7$`Ztrj)X!?e;l)Vj7`+1Z$H9DwP> zm}QAH;o2-ZD1hZeqi4ZX|J^%S{|m*5UC$J+;>`k*_lssng1WSniP~seMDc4N9m*5; z+_oG0OawLIBQxtuKv!R*9q98(zT@xJr%_*cBnkSne=pr|lE`ux+xLrV2NLL;5qW%h zK(d1gyk$<}(6t&8$PLuoW%r>iDQ!I&BVqF0MtJ2OmOe-R8x*uvN+n$JoSYL@K>N%L z<$G8;fJwCbbI-ge&+)iC*oY4I_tNq#6gW>}p2W8ma>Zs_W-av}>J0F1p(zn?kep{P zcN!1FBC~CLc4V#2!9VklPi2dp)7WA-9*^;?rmgmjU8n?vuW#5)ARaINj-z2F7YP8C zfnG12B2;w9Z51GYoYhD*;f2>9dHaRt>K)xACL~XfS&L+zGl<$^g|(S+qzNkEp~@9T z{~4_MEroG}9cd2t@nka~cE#ZO!U`g28d=;lp@!Q!%ORJoh!7%IBKu%f{<_P{%iR#T zMT@iBG3+1(m_gZ7dEbJU1ho|8WYlE67K-_^EhKBWuqWwuiHPx^AtJ zeC?V~ocs~9D}&nlhuvOv*W?bol6eK;Ed3+-Fq`~%^S3)GE+$N*2k9xo;44LCS1mSk zjur}ZV>bM|K5LEyA~v1AdGu44$Q@Z&AvfS;w3O!$zxW;K2|tIum6`sCcL71H(~;|Z zO=gwJi}MScbM39LsX(k*dmXFN9penIEO3py#cg8;QUY-GE zx^Ufk*JVw(zL^T@!XcwKe);PUMwanUw*qn#?@qLbT;rveo2pp#!ek{65r}BthTncK z?3GzSz0msq=+`92&P3wmE1E5L#XqD-iY(fZobxcZ>!8ZeY7gLB++GeZ39cW!q0PbF z-b@+Kll#+9ASJ`4alNgc-qqG{gjk!G=OH$bZj$}Qq($W?$p62q?J3FesJqU7iY zll2K-e~Wpu7lNdE$$m+Yp_Kgkx~^8KS=tWKbSgbLLVLjEc#c)uXetEu3rjy!sl7^u zp2V_k@~=IaehVRJe*IWQJ5F2TKlFo;FYfP9A%;Kwqf<>ZXAcXODGI+^HzeV|$V*wN zn6I1fQcj<%Cpj#MPP08icBHu2O686_J#Nx-3uU223{4yp?TE+v zPD-3JF=Z~!@ayXE3Gba~{-z;?qt)OtDJ4gkRv1!*p^7T4)rS}Ba(e}JAN<{41>9ld zaOtp^Acjb0@@a%YUF~4F2z#9TnAVK_C(LI&AJlAgKi#^uYY<*R&M0JnTA0-8JeG9W z!9-2ZnlwwKAjLB}23}}0KU-x~dRPwTJNN1JDA~=W+s}SOA;d~JyW^iD%*)Wv#J{nx zY<|O;%f|~&8paEE2F#ogV5yAvFOEE?PvcXxsW zcXxujyBxo}x-kvy$XVAe)EZ8C;HbLp*AWthtAhQ8-Yn0@iN6bD zO|UKvN&La=7TVL#UYW7%iFFJ*Gj}EbJfa6cydBIc&l8?(4Z7 z!oW)J(ADbmyC;0u&2{l(6TaqRs0sV)8jg%BS05WN#S~b{17TXH1azK1cuS=4Eq{60 zr@P8Ck;FNDu$M|i#Vyrxe7QGCYD70b%7c@IaaMTleE+}wat!oLUq>b7or?T;GieI2ZqWzRq?=!d-5z_w1mg{e-4?IT|#= zRAqg)Ukr)90ojQxMgNRdMPl1)FW}I0pOa~oQgx|3kYqQV>ZQ0BQC#sNQ;xvmqUOQT z$D}dU2h#8(*TOeUK(Rnm-|1&m(V6$|I^ zVx4d)&mkPnjXf#!WMVVAr8DA@3}c3>iW8~6JA~H^27RQ$>zAhnT4EKl$1jWkF)GK8 zyaLH|-lf_)4ehR}Et6BK&*}ocFO^0XiGeSLIZvD1~Q4GXSk^!#&w<;dgaxGo`UcNjXD4^gf$I@xc0h zOb%%IOIPxXCHoi4F6O24u-uqGhd>BFXm(p$i+nufs*TH|gMQnM)X+Uns`N8?)IZ2&E?M4!D6Sa3g32b88ArT); zV@XXtsEW`jrMy+CN$fBt^R$ zm?C^4wthxTJmP}YM>ySbl+oxao=;xyNoqWb4%!fC9&ZHj#awXQ`XfNt_L|#Agum(T> zApe^M{i_>Bd?B2PkhS}jY}+x3ZJRO6)P*f^Wwj9NkWaioZS|H@>;9FffcR&8tNh!NV81-)|NPDHy3Cae(E%?&9rq2$kzEKzi` z$SGOGSAbU2&m?F;3Awu|Q=$nBwCULz_62+l43<*@(ovc;Rk(q<4cm-A+OeD>O35|& zsmIwn%~`Hgytm>9Z@n%ZdOBJ7$dGdTE~GB9fg^U~m!h;Ezwc!GEvmn)+77fL&R z;4A#){z7nJQ5eDCBKz6Nmm@WuO5=%cr!*dgrS<36lp&kd$uiF&D{5jk^Dk9{jO}DvLF- zC$h%Emj&dsA-r-*Agd9jc5AOV-vQWFX4SKyqFK<|eK#T#4(HVcy;@3R0_WQf$3p5y z-hhmP;&2ih0#Ki%YlBH((!I9H58vT!#z@B{&4=-y6mQ>IKL`y!@9{iFBU|_?H!IF6tG%W3>oB;&`RA=GlmA9IgD-djfVLf?zB}t?>nJHXH zQ*S}kHhHQ(nJH1XS7$-gK1r%FsV7>O%w-WJaNM353q5)$QL7l>gmiGJnWO|uYYU=r zQE&W@FDda-tG{~lHt5)x>xx0c_Btd4C&G?qSYLOhEf=_CvNV2H0F~M zFitF=0K2{m)p2}7Dz_}z)8@X>yfRPG`izx<7$O6G!J&>^)J|=%Dp)Ndj&o`OvtR+a zAO&{ko_}be4{Qu=pek;_7@l>F&RHs9i-OA>?G&T5;x`)oovp)XQoa zL4Y(>Z@3$ULdAv40cE%2e}+opYqHu`ukmK(k`L>8QVXRICAV~D&vHC|Ia=orF~{H) zD-K`EUJF>`xiuF4DmFnlRp-JNL%ZFD zP?HFRazBcea6`;wgsy^K?d8c`rHc^zz#ZSR46-%C4QunG#OAG{=B0J0!ct9>Ycx)6 zn=UDirm>}E4$G#D^4%%^!%;0oxy+k*!U^pir~ii*uPX0U2jTcJD+r0q-!ih_tKb=6pUzf@eeuzFpLo|Bvv_CsB3E?DV zk+OEp-sMK=XLG`!ySeCaT&|ys<+qYXAQ=@d$Ns2?A)*Zw&i)pA=axEPKsxA@`ENMU z6eZDOmUy&rB((Titp1wq;J=6CYMDoS052U?$_d*BDK1;(#F#bD>pNEz33Q1o9}Q?wtt1WN zJl{V(AQR&%bYx`>W$(}Rb!bzqxvz*-)Eb3*qj6M`;0F}5GFAzwAQX1rcR8ngUchWX z9dVcup;>C+m~8n*gt&8z>dU*^h9_tH6Sdd$8Fkc!a5kRSYi>pep?|w5rz|jQu_aDU zi#?=#&s->euy4-{gXJQ>jcQN1ICl#8dPZA5-XX2?0-+be!S;!1v)n|mzsBoUQ(4Lhd`*0RDn9r8r zM-{Ox0t^fSf}(5Jn9OO+1N&uJZ1nS+e=Dz?(a%os)t$OU@7kRjSf{I~={E-GFSo?4 zx0%SSZr?oP&MGDQ3sucjQk<-PiI!?T@-YM7o)XR! zVX&4y-qaQD->2@F+Lj@_l)=iKq%s&3l3;RQ?j@_7Oq>rd$C1YM|WR_FEC4|58idpm{T z@{(WIFBq-f!K3F;;>xu)t&=eppSTdnrd1zV%PO9A;xI?y#V(NwQhx2&opSC_$tJz> z?rrPJ$<~qetg!<>Qrpdy;dbMyBlI+!*uyrT8BJf ze*M8W*#DDe#B)2Uhm{!yrhX0vhVK6z4Hh?aHZ(GHGLpm;UFKdl@s_N}1FKg$$<8JHY_{;sWYV6Uq(Y&qw zBom8kY!w{IT9A?G^tstZK_;+t9nIyS=Afu(N2Lj(c~IO&;;lD%@;>dzsHu8ou?fy^ zH2$yE^lZ$X>;mZU-#DD6m&{!pxvdICJ)Y|KXdKlRCljnk&DejV6(nvKm`IUBMn5h_ z0E=ZySW|1OhY+IlMl@%yXGsR_2LGzp5RRp_5i+LmOt4 zYIA1%WhCP%dVw(vgSj=`LaCCOlj^XBol90#tqJ&zMrZAZQ@ij>hC8J;&Z+@^*&l7E zZ2i(CM;sF&hEQO-)W6b$e(~pTGzRNHf3<2@8%oV~%-^=M4MxH%cAaUd#j=jB3hG|$ zjjWTI@{Vdk9LsUiEb`k*m$gp?V2!w;PbEX;m^{<_HfLD61zIZhzU*iM7 z{QbU$ZDpssUH#QeGF0E&Z9nKLRoz}@+5*ll=eimUmm?|wKHX({hL*82FEgW*oCN)ZE*Czo#e+7w%5t9(!iwurABV$bsQ8YUu>a>!-^ofKh{c;!3);q zIL3Qwsj^^SZjL|cUCg$iGf-Yo8Iuf6V_JfOWhkw-_6vSCjA*-_IQ2wlIqKn)e-!7|OC_mZNQT$uLK76utifH3T{Xt7~B=-eSl)g0J+jAY{% zPCyX!L-m_=*Sf;Evy38Al{_*b<|izaTu^vFlS~Bq(&99jcJ0?rm)c(37O`vkp`jzVYUmUT;*34sZ6|nmEq#um zPfd=Z3?QN68=Uw|Ny~8j(VJ~PBNXsY!(d=iAV~!io^O46mFZE>SIx?$rdHLW&V`yI zWTqyuN(D>dgKIp%4woWn8vFH|{$G^J7HF%s`zGtHia`>N*FIcr~A;^~B zDqKX^fHi$aZI#J!k9HE#)6(~gU$SZ|gFA;xQL8fm;7pZH?nHsrvds+dn3^g({vPJ4j;TkK%sKEYoJ2`5vyCMhCV^ zb?ln4Q1ZUO;|812UB^LZ(Obnureo4;H-ood+;*gvl*RN~h3-CDR*N+z^7l!9*#$q` z#xWW{{be8G@UDHlo?Yz`8KRQvVB(;ePaTn~SKrTqds2UR5Bg%C*10dfn1TD5zgt#1 zzQG?mdFJ61>>MGrZr2E`#{AHSdn)l3atQIcZCd|u{D8aQwsX?0Wi%YfEGJhnXvUkt zlxFh!@t9_Cg)`FqD96CcWN@oedpZ3Qt;&O-hI&6v)42T|>9C}!c-lxzm!+9WsReymMZ{wu9Z(TGnS zGco`C4d%Y<5`up8@VJ+__-V;?!OFAnm-hP>SNc?3(Rg0``(KwY35wU>1^2A9B}Zs< z2gefev6LtdTpKKDE;ka6$tYqG)QqelS3HC3s9^J&D8+xB!kSG zDCPiOCCdv4AQnzgE50!rkW3j?AGcTKDQ@dJT=?6!h36Z?DO}Fz$qxYJl&hpM9V_-Zm|h;IM7(q{MowBB<&j6 zm$@J*-CxnQOp&~)A%d-Hqf=xlo)uqHiQKesnukUv$}&} zA+ibXJo7U0TbTqLo{Jkb@rKogwMsw?t~rw)Ps4l$M@=)<8BZ4mqzydjBLv;9wGZGo zKVTGKh-;(tC(~RLEZcE27dTj?;&ioc05#31thFl)zzA{oXG{)t;3VrCJszs8 zX<}!=JojEQR8(bl%K6|e6KgBL`dZ1RBSI+B4D=LwzFXH|B;MNxyvesCdk=L`dH+zR zO^VGdGs!}DKE4FG%P5!OMkM2z>d@SIZR`P+-0MGo#<_CYxWazw_1A}a7WM4tpX%UF zpUxPmyWPHA)bW*(Vot|va1D2f-qwYCp1QQf%{flb6~(%$J#hBQ{U@y1h0|5ZC?p|K z^7K0=5wrAHVEYED>K5U~e3IMG$;XZzHmdE)A+#=g#!h}_DF;%zQJJ#Lv>`w*wP+cG zQ~33WiJ6CpVAnq$_)ZDIPvm5DlU7EF-{-d{wLiKOoi9_d@()oY{^k(&=v!kL5{tt+ zk>79w5_%v??|)6fcPPk1@nt!G+A^df+3TJ0>&~18STUU=QAO~xp;-o((`THXhXx4c znn2ttrh}Q?kw~9ey2xU^YjD(%)vNApB7_j0EelXuVhax=^zp<#+q~TnpYKsoJzMYq z^sFeo6cJmyL0$UH%aYOqwF7pqSbLRSZ)#(tSxx4#wSQ5Ut`N_8h8Kfe+Q9pxl_G0= zUF%FdD9H6ww9dGqA{*-gZ+XYGy)t9y$~vi80u~~N2CW$0AD=0a$=y<@zyxD)a=Re? zI#b>$fsDc81Dn>>Pl5=qES~hSb61|tdkZpk zvpJUZO}A4 z)4AL2^Fq?Nr#4^|W;F4y;=+pWj!Ot+4d`|5taJpQ=EFTkHtCn2zK_{wd-PRxE2b&&QKHg=|(3dSGOEPw<>2SBGCK5tn>%6Eq`jNlfn&G7hNBRnK`R}DO2BZ&yYrw zr0JCBoBC=vl+`o=xYCO=J69&SnFGEuXCyTe>ouI_GxEo2gT=7K7Qf z=nuV{v|+}wgCtz;&)8!nZDK!~r-*SDKPHGQ1D8`HeqO^`XNr9eFf$wYEI4*PlQSGdM6r#vRLt!QbUxu9=yi>ll!1#PPF}#?eWx)&w`|;c+gf+amC}2 zQmca@PP^92F}l1HwQff#%lq+P!cLiFznP26@%;T(C_S-Zg1gSjxMtlX%=4$rMZx8zRhkve;kDO9hE0W=( zgCL88u5kU;qyWc$rz~TK(lAG+v(Q~+?zg5aU5^8v0x|1h#^E7ue$=`@lrvsi+Lc4Hw9g>V4oh?I};@aFOw&#mOd@pdi=~wkAOSq zdICg_y3`5Ff@XMC`8w~_KVl@aS|2f#xzUV+MBw{kB209 z#AW|NG-q1uLD}>jT+)SGDu~AsoC;UGbTA>QggF^<{Fa->98-3{m949dE~V1YF;MHC zMqBa+`NOmw%!worr4qj95^m~X@gQ^sb%}`+z{v8IZk$(Wn|s2ph}_U)9XK!}o?{&1 z#RJ%tpE#xWo7$ZJ{KS%4&S7il`=UFmXB$^>C^&ICPr6|Pvs3b0>8(hQzK4@-7DR@n zF=6dBNQ`+ho-I%&fK~nXh6YbwYfr-CtH|^xO08cSkmIq!-^QXREZG8jf>BVwnb#|= z_0)HsxGyYr{I$2(77EFJuVIPauqJ)d0uRsuOD>`ZVvF%)jIZyTBmz5#oD-P(gOaW6J4xhEtGGU=jh*3bveKH`R+x8lCB zy?oF;ew_lv_^G9PB+J`y7Aw_{tbp4yY2t!lhwg2Lq%|XFqSZO*kG-#e)D5Ixpy{gJrLCC0|Hl+??Xb;rOo+F#8dg!8_ZgF zl%)?02_vuVBYEKBlRkTP!g|!~l-+)E%m|Kr96G&5R zF!UeHg&PZ%a|ixL|6S2hyPpXBVw{VC=o7Xg5eJ{9D><)tc^l1#Xyqn|pd68!E#uqPy0_KG@`R%KK-*zkroo8^>>vA7m*p3^1 z*Exu4m^p@VT5k4)C(y*;`juYWUZ2q4*oGvFv1i*?#wHWIhI@-#J01>@(>lGszhM3V z=qJr_O{eov>|i(NyMxiXfZX-`OOUvAm@lW;&uwlj+ZvTuX>n=4DCnA!W;0#3!mRcQ zv1@f6E#UYc=4BD{2WZ?@n1rf1_vOHcl9YloI2$kOC97=8#$S)kFralbjnc$JhWw=E z!RO0m%`R8N)(F+;r}E?s`va$d(fH)YaL_BNH%K4GCR3a%oIxe6y?_O=L0X3MC*YXa z6SV$iAydgfSh_2+rQy0g;Djs%IF-<) zq*OJBb6t?5{fJ_qdubH(maFg}Y{0N{1N(Z^%@tf=?tfFIszF;NT6GQSLXQ)55&c-? zsO<^6u-VrT`FFPAI>?`dDdZhIB-rKv37$`H2tBUG6swf!MbUr;iR!?{5+nq- za3C@Wor*Lg#TDj`tuW=AVg>WS6C zJgnD!51d!cw^tB(VYi72sFPr!*4JLlBvtD%%fFpA4ntb40vdf<)9N^Sg?+94A1X*U;_gAeO?&oTf>8((Cv^S z-V#(~;n`Rm#KJzo8TWX^CcS>Bsw`MRG;B8D%_|T>JNAk~Euz(Wbn5cG^_EB-5d-;Q zHsM}cnegTmbi%d4DHo%PfZmAC>FK_I+XxV~^h!1Mc{}))yC;;-q zaObcCvEPhMy}^3_!Y0ARLMxcod<^a4p}oBe`|us~_Tk=vt}a{w>V24A;D_jK@?*EC zW`Xg=X~Wpz%O*A#=ndqbK4(&lfL;fVkRrV7C%Wjfs?Fs%1a@u+jvJlU2c(`$r|bQAOCu`&PVF(%N45n`O_&Yd!3G`8N5GE|o+Xw>L;?TY=h81AITXpFQ$17Q&Q z6=_6};GgyL<$H#<5dj~pP~JX@VoR8*EP`)L+k!vK5;A!b8_E5{Ld9QijNt{GdkJ@9 z$l!Wa)M|kK&sP0r8`R+xv}gKb-{B%Pjeh=(d7M=nq^$XLx7N!1_j z%av>lfQ*UL1_YphJ0+!86NKRfr2AE+5-P?WGjfgo*RHeJuMk{LNu={oZ*2g*Qx|&@ ztRH7?(-!GpX;?qeAstND4_jjp8$*+=rd_2F)*C?mi4e8PDH|!Gng}2kijCX8DAO-i<)T9;` zX3!XA?C(eY1I>_}$dUh?C9shZ+tZGK2KJ5Deb_;<+4MKc^HVlz3|?c@I^3J)kj=hb z&TvSec7JNNr33}ZvMs@$zgj@3mq}N>$~TNU`tr2ei&s@2qYR2-rhjk~hbG0wnEBqQ zb*4I~TFO7hPEoN535IM4k^{IE>7!BUBtB4S_d~MLF3Q^jzl?Ha+d|PQIV%#z_`G{h zn3nIX$JGgWg8h!Tzp9w(*spvjTe{DCE>}GKyE0zuN25Bvvn+uZ2H7dg+AkqN8k2Fx zQ+QHzks*Tp=Q`1EmF|dmy0+lcV@}`}*Z^s$cfwVRC{0v9g|*G)9^?~d2mxyKzhukB zC0qG$-g)Gi@_$@%_L!u4jB>}X_d3K3ORJ4i7uMcvC60=yjp__Y zN=jbs3Boi{+CZCVsfXkmwq_WCMAT35^d=xr&G`8{4dHlhrUYjJw?vC`TonaGKLLO2CU$b-2MK*koTE_txhmddSGX8U# z7`qt!EW(&{L1nQABKWQ>F1e-A&y%f-jZF!9CaoI|G&0E%3UGyBCuGD%gjLHTF1bh` z)35ZgyG+>8%P3YeCTYShmG>P^2hb-i;??OLWS9JYqR$iAekXZdT@qMpx(d~?!M_vh z{SxG*?Pn2UIu~$Ds|O#tMa@`?Sx0kTkr@?QHG(^liA>TGY(&iH^x1;tzOlPw0%4dK zZi%78knp2?)retsJ9TAC$?HD+@XE)!#(YY0KGzMlPScnVCoe{KVNr-xBa_)a``Yaj z3J*+9$`gVGy@52FyWb{WDf@KH@qOv-Q?&zbsc=OSZAG5cBAPmA?>Sf52ZC<%+8EFH zZ|YXqX#&T3mRX)@-?bn+>_+nYYG-|?h(#?+d7PxF*o^MqX7YF~S9^?>=?&{!bR2!_ z1WvmO`usi}=L(bK{Y*}HY{&;MLE4K==FN`lIk&8H?Go2l3x>P0tmeS8B@L}ei1^O< z%-&tXL0rYvO%_M`mGHXo?P%df{S>wd^9LWwMUhA;{s5%HT`Z?@87GW`wn$^G<3!&0 zhvz0s<)S^uMdB3DXnXHgUKK^MfCaeUmgot8>`wdok+wM(-v|%$#bRbu|97fx=Hcgb zs7-4Eh(6JcBlUiLv-9~fl7${5Yj$b}{su)@%+!%m63oMF{XqZ0z{$hy`%%{#-MIps ztkO&K?cf(sI~BG~7q>}|)kW4I=f&H5bOpb1kZjI8$}Q|}7PWWJcPoQD|FKr zddnq&&G^i~#rXcD4E$xWd0a^dC%Uf=|G zp{o5ctCr*rjW%)W^NZr*)3|s8*I@P?Sr~rCJL|8i-crRCly4idA7N{(?oStR4td!T zC4!IfkKD{fPj4CeK7Yy9BLmWEUL>&@HX!-nT3SC6pEZEhPZK;j9X!dgAUxMCS(VTF zBX#_qE%U;7!L@9N+trI(aOS6WqUl|ibw`CZh#UHEJTFXE*=U&ff7s3CO}CWhFoay9X?yx z2nK&3#zMYFLyV$;4P{ulr)0Gv!g8sDJH(Gl6U~iL7tZhp*8*n-QJ~7KRxDU9EpUg8 z`1OI}8O8p*pYlAVXPQx<>&7!+z##?-^b_l$GnPv=+#yy}i@&&Tg?|fNUIi6codI># z`h4Pq3Ovt{ZPx`21$^Xs8SeRfaJsjejf5>}%bQ;Y#|3c1f0V@33ur z$o|ECwS5tWy-|qbg^2r<`9FTh8>0bd(Ph6zgVhtDEZB-~eb8m;nRyhbJ?o(e)=fLy zAySlQjQGpDEkfQ}z9RbqEqb3z)MK!?Ds47B+o3j=YJq==ByENc<*-Ach60aj6P zR*aGb(lfg#&>#2`73rqW>Wo&_QM_s5ws@WK8_Ri+VEOl}2AYgk6p}9i;&N)jC&A!+ z`(#+_YN*zs!gQCNnp8WnD|Md!R+Zmi8*k_24RLc$}>oOR9%><3mPjuV}vk+ z74?`V&ZH`Q5~BE1KKJ%r2?f^!N_1go1Hg6m`i1^lF<@iq8CVo3Je!pl76dG8%4S6v zK17#&i7&z2ZE9k`h(abz@DvXX!ch!0melTvD|^_thV~-J&b0)1S-S z4{em5F-3t6vmGj9xj^6!t)d6QIPZ8J~3dsymt{zypJp!yA2c5@OtTe`M7xe|f-f&A!>Rk&UU<-D#>jS`KwAtqm z5TGFWHp-qlah87S8}<9gqY*M*Ig$mO!9$_-_Ei;O)NxwlX+y3U(6aPZ8Ogz_S}q^CQV(Pil~ukEbhi^4%sJ}+ zm4LVr>J-c_6?Na}nvY5e;W70ajsfR`5|2vBW0(t{>}p zIQnSFt@HZrS~M#gz?OdN`k6iIZC&&XdL};fzeBZ*i9OEr{2Ri#hnSJ;Cbu%oI*BOP z4su7TmkM*()ju+ro~yhWrNT$+*>b2@?Wm|}E)JFSmNL2!N&R7{?^L()tDJeYE52M! z+RC5>BRZ5qI?}CA8TsV2m zl;CL`MGn?ObgbDz&mcE3v=wvwqiJ(7H7dRNqJT&q_av)_+8+Nt0VEuPw(jY_5(- z5i;}GZ;O)`L*+N`#Y*>Rwnc4Os!8{cZ)0Uc7L+Qv8ID{V_vcZTaC`B%dG$@^-mF)L z_L+QrO+2)$LxIBIH&e=h&S*Cvpux>amt;*0=;a1 z))9KA!c{#~Vm7DWHWcsXQr8xLRIh3`c(Ww(-^r@+ZpPaBn2mCsgmEqPwpFZ`4@{I= zig^@|E`g>fAvpP~n1oy&SW=&vz%ML1~4(dxVVHnRF z;xx@=4C*U09J%Ps#ouWB=qP9z@+B(A-;kU`vNKY3_~+@ecekQ?`$Yfm`D~xGdnxS4 ztAg8aKd6J^CFalgST++aZN@`=e2JfAgENg9Gkf?qY~hbeus1l8K4Q7=Tyj34iZ7^G zH+gV}G*MRBitIux=mkzugz~wl2#Mj~f6^*s?e>f%C}_jJ9Df_qpm(vF7tyPA>XqG9 zV_`V@Kk8Il^P-PXV0C!WHx8wC84v!qDmXuj)OUvH{7L=M5vx8RK7A(CXzA$CPKoTa z37A?B-KM_5{4006o~nAA)$@*>nsjH-4TC*=C(d+Cf6%FYK5)^Tu@iZN@ zryleWUI%B%Ce|xAm1Woqo9ecso66M=r^$0ps*M0$jwzK%ylx1ZLY}{)`?TRMab?c@ z1Y*SHuZBZe#M8omO1+s&PaV5Y5zBf;1HcafVZqbzPkzGjxcv#ZhEsR|OCY}`hU{Z1 z01T@i7W}W>sze$KViqR-ZMdp16>vi4cXm#vH|h{SgQ;-Xn?DMis&SukluZ+khuTPp z_Y6jGaY!pI4Ihd<>NaKUB|I+DzSxy@vJ@+Xq>*FAaF%}e2i)`~s2NVt$Jg;@)*&Xu z&;Ubw6C5s0MpFUd5gK9u;wm*KHXtv1rZYCRu0niFI1Nc85aSns9{Ee01G?F00w5g4 z``zqL?TQ}A%as{CPHnwL1MG|@EARZ12f1A|?@t&L&DRP7WPDNJ#Y?aPPI+A9U$I%T z0$aXjI+IgJQvmS>5}3R5ZKbD7FY<@kiXi}90)-Zogxz2H)>~e?^-%MLBM+)-yb0Rr>;<{i2||ZY z=~-_-=(Uud`t882Pur8HQ2#v{qCNkCplnMtK;W~2RwU3GI3;!gApW88gbnlrWM0!! zzd!&lRGHV`s9!PwV9mO)-~3@C6W`N$Xf z;VI6Pz$La!PJ)C}3LxJ=LZQJF<-1L_hkTIIl16X?STOv?1(EyR&Q6TRvmI-j_OJGlN z{sugZp(*dpPXH|JKvQE;N364;T{Dh@|%{) zA7j7))IHtdr|+Lgz`=t1)Q^;FZXhptW-xpSksvvcW+34_R-W5iEM7j6W>fr4G6V3! zoau~6J?Rx(yXDorm?D0LIYyOx>S`j-FRuTD+Bn}Oq__C39}f$Ry7&1kg1`&$W*PP` zKXKh^6N%+o$VGl6t0g_K1(4}XM?FdnoEbNyz`FYMI$a%FX=MDU{qcz-O4}Cn5L?zf)uguYNJDYDk0pX@TDw_M{yu;C z$8A~ueS%Tb8f%V}s!Sm52B*>`dH@9!YYb1AVdF%@V%%hY;xN9nnOXm*ov~}vPDd%j zx>2*n%iq|IC5}XT(I+&1buS!OP=CQEv+4=X@Ghk1G;+TC^dfUr{??#w)eUp=2g8cP zf`~jzt9A;%QW%0^1jzs^8(+i=6xRhuy;}Q z4>W1tf%USResjq4T-oXsf9oE@@(Ui7 z{(Vyfiw)rrecDgt;n142!p*X3OG?9s{-cdg0|WOxG~wGLMB;<&s&>KFr5WC4{8T69 z_!{7v#=t5!ugRW&n)U9s7HMC#+$o4=>hf$W6MT7kYTbg(&puW{q}bkWQW1FtMlUnqAK>-Aw0hFYMm3OGLiNI|9(rc|ejbkLPJBY~hy}Rv-q{RF} z`k&nVR$RY!MIU8!<0_bI3S>ocKvf^(oS>Dy@~Dz@$gxm7&e!|QHTb%xDGriJ&-i|< z3&xpuUH7aWBx6@SE5}=>snN6SymF)Jyn>wb>3{m4VjZRPD#N0HY;O1ryVh8$X|a0J zcMm=U4axU)Y2|bErbF#RR4g_H81K;$sLLworquU1Wf70ZgBQLpt7oaHp#OmQCJn4E{a_$^;o(9UtOjEgbfnox)Yl zP7ZZ!XxhD{{S}Z+Q4H3~Xup|l;$EtY+V{hXEGaZqRlJa~nwzUsb*^x<@jc7jvZnrp z*1R{8wOvO7nni~f$=r-}{gmEux1+UkZPsE|$Xj2D*>+ue)~)K&6JF^^+Wl^6-{>9h zqPGeu9>peS#jamG;Lz>YXXH0jr7J(+l)w9wuhYJUVKK1@2OQikK=l9TTO7#BhI^Qrlde4TTK}l?}<&k*(4Q+AMeAt^=fd zjvcJQ8~z_enG!+@_^M%r=W|zXn5mti;f56IQ$ZQzOV(qb|2f^7 zt?jQwIk>si`ZC=`1bC2HTl|oJ`2BmXdrcbu7;UKgH2Zij)BB(Rc3}fmnEt>)xzR2y zV0nPdF5BZoOyE5w+Nk9yGmrgx0dtV|3qQcHq)lWI{5C+j^3fJ zGplm{43GBV2_v}v4g2*KezIH;E6JDf`K9Rabq%BwBXG}whv|CO*R)E#o3U%8M+j2I zTD#OQZ1g^`{x;3gcP+~%pe%cKJj1xln(kGpeD<7qobi;Ou!}>qaJ3*yR*^8++@`_D1BfoGt+lL>d7clkmf99QVG6of`t+wb7mGQKIJbiXvz;cmX3+( zSHe=YD#2P!3saEBmbR1;J)29oAa%(i40UW2Uz_PoY0-q0kQ=u%&6;A*jOf$;3S*_} zXY(y5zn@Drcc(e8G-|vl9^dXj!Lqv7Azf|?Q{CA|)m?6xFk2-S@DFTCiyK?8(VMSY zEzVdknVXsuDQEeL9?nlP2tCYme9uT3QwZc7H`{73Y5nPpuNEghZF;Fh)yXxeB9h?l zS{>YC>g_B&+rS|9V+2}3Ljd05!)EGVFPumq3_rjcb)j_Z{j7Y2QCjR0QDH~2ghntT zdKKxlAEcf<#F=bnYztE*r@B?vz1AUDI7|xwhxB9RE73U^wyc0Y0jwC?GTGZR4i?tS zxcH_hrYW?db|zJHkuBjD<_b?kgW(gUgu0zgD{}8T!le6T=)0lrUdkF~RHQP{^OlTG7GR{oWxSQgm zN~1@;Zof!+?I5*|`m&+RPNzeTQ)ud9A|8Ew@L8@~3hddK{MEKflq+AHqNJkHxk9_r z5-3+_=*pf#I2r$RvH-<$MG^oMP<&B3gr*7G9LDWw>a#n>Eqa*{=JxU5fK#!s#jJKx z6%cn*3S;QfbX#Z>*OhHJ+kcA8$Tvo9w8*noj1Fj@MC`?vd;HZ=lSZy`O#TZm!dd%e zUvcXqx1$zoUZLWQ=kGNsufM^L@L-qZbcox>54@s=Caqx!8`PtargcOs6KoKU(ahQgg?KA+dk2!D|15u5XO-Brol14mA1Z}@R51+&pPI32bSnZJY}}Nk!jqa3 zR5P2$M291?HVqd&(k@!Yd%;mmYJLL}_OVsXKh5X?=qsEDzVQHHksr3|@_M);Ogve2 z!it+|yhyM(_zjuan95Z2)9leM`MO|nx!p(%!N0^QBXhQHNdyn-P3Sh}Mx_CvzTY+H z;9OlXOAZ_J(=Cxc6*OpSMQbl`tBFrVX#Ry|@l7;2nZp)lgmQJqq&hTn$&r1*im9iK z8B_go#3=`(i~1XjK#0A@#*QqEl-5q?h0y7JlXtx7TMg#?^5_{fb>>71*kV1)Ff>1~ zrOc$wZA_YD+`h<9nNS3#-JmP$ObCu}pH3}o8y_A$DUQ*}_htdk_2hxZL9lHmtOy!? zayDfT)9-6k#Frg? zW{-J3;qgzgkI3(=Q?j9ZR#l~C5c^Rr#bXt%az_LcE_OQV|784m58{t^PODjAM`p|$ zFV*ZgyQ9tNW0Y-PpXs%H_)LmE`xUv`u9Mz71OC=JUrM0{e|S#eD&k%@K5!`T73TY1 zBX$EmE$co>C_OqRiTdxx?-hVpY!=-9+ogS=HYux&qf&_Yt0fGd|Q-p~*-k&W7?f$7f_f2iw4FxM|Yo zeVTH1;4qQ{q6+6hKiWWPd?;aM!Spjfs7`-;>b@FYw%p3P{S%En1T&{-GlhD(*l3xc<#+A@3}nhjSK)&L&VjtIZh6eCBz7usmE) zU#m^06h2->m`Q1TZSz@t_wW?$iUFLtMXR=9Sjye_ zo9+ca+g744TIT`_nOI+6P+6$H9oVBZOMh375Zdm-9*K3}s;aU4`pIurk1a+Zb8mNH zzdn?#U%Jdk2NsnaT#E$2zQP0FBWN$jbj$MWqNx(En`2u&rd#O!a_nhKk?h?^zrmgq zJ9(ZXRY-PkbFQr{^M$GM=#ZT9u^$xEkRIGDhUY}ytkxp@qiz9KY;PU@ZA*J+hWC8? zv*_bLa~W^`i1-YRy~cmus-1le0=}bbue|%#b)1nAvl-l(jhC1YL~*hGyWQJ1{p=X} z)Z?Vrliyu2UJv^iO-UHfxkHQ(x3j6{yJB-K>0*{{}BoO-a>Z;BR>gT`N?JBQ+H}HZ30ktFluQ~2YMgSu_GZ!NtWNpibU4mN`RI*B_?Q_{;MPR5CCLh^;-K$IA4r)K+_kt_> z1~^-Cg+uOo9Aw`0TkMAp;cqnTBnB-fn^>_p-`z1dpT&KDzO?ND=M29nH#6f+Vyfc4 zU9^4f*Q($|BB0N zbih<001+*3(af1+C2KUIt3A19yHK+&KWD;wSZNRwZxhpZc9BFApCAq$57A=Dr4B3g zuw*@0k3_49>5aqtAS;wnz3yDB@&f;%0)WT*-C-^o(jjXSJz&IikGb7QruRz*3*`VuSUW8vY>ivpDc37EeMj46B4J)G_MKu(N3%HNvqFa) zJMW(Yibqnxi|0B&)ZEBs+!>1c(xg{w9cPu>Ub-RiXU?Xko!!>4I~K(Cx8Z|!WT`0R zxyCn+<=)idzGZ(e=al}C9RjVGTuV@#nyfp0&!A&_OHEg|%iz{IcIV0Ng~AHSi3#eI zMGuQA!Q${Ez>6lgzyj{&UIFc;5|^3<@9vP_+Az$1YHIE->A|^tsk;*QSSEkWJFIZ* z4repv-Iw+Fj;KX=saIm70?MY=R(a`RVuI!`vqOM1jKeR87woaK*TI{$%%q!ha#34n zGZf4I_@(RQOW$dvz0;&WZ!{5BC%?_0tcj2N3HE;Drmtd$MEP8kT97=|@TAV#|Inw5 z0$;}8JY9+5el3K9_$IVW%z=5`l^Gy%)!Y^&3jeuzB9M#wiYiaKMt)B+d31V0e)o4< zfwKxotpwZc|a*DR?J@joktTFFjme-1J5@R7g(WZ z!3QWD=n_4dwRqJ+2w8{-S^PSe0eUiyxHnK9>LUQ^!5dtvzPyM9IycHUaS1ZdfZZ7e z`ASn$oT5Lg=yHln5)f8Y#!VkvHSFbYfa{{8#rGbHa~GdrFmpU`mRi4}ZgC zzL*2NDBxCyx-4bFuBgC%r3|nh zB9=(jcKv(T{$%^5D?SS4ukpUMOV;#_0VolU+xPVmx+3J54a@P|-+YH6{8)by53YHohvYG>>G);?;GU-0;4 z=WX-n(4(0aCBBt|oFvlHyP335m7_Q@DKCxySb~&8j6?i)Cui#~PckN$;jT{98;^{C zgolpS4sVF!?XQsNlv61I0pLJ}ar zA~jWB6n7}qbVY%{Dze?Rl|$TUx_H)BQ4Gu#uMUYFEy+X1>yGubLUp_=YgR!CJQdTJ zHWmIPC%WKhI8pIHT&VT6+;H69AXSymViZ&A0$1*{8Jn^bJy&cwbPPmIG=&%PJc25H zdX|EA9kvloIZQ4EGtSjox+=MRSPxKTVO6Mdhp3+fC@L1u%~-{9n@vX9{P!RfLcl_L zRU`&Zwg@C3O9GOa=8{Rxs;b=Fx5PxRIaN6{%WAFyH7z@QPe;Kr!bR&jAskvmva>ik zixVhAxm33fTQ^CPQ&PtZWr3W3feWJpD%eg@6X)jEN|sF=RTDyjm3yXWV=(eaIRfYn z%hswH^yXDl2gLGEEa9j(T#o1p&x|odq!yMIBtc`tDi@>#er3m{8%wt=j8mB}4lSo5xOQ<*Cgu-T^7${6d+N*21CC zPb;UxRc%nN!1bG8x>FZiQKnh#8#>rEn?hCQXv6Dee{&x~fLBaK@aL@XYr=b^3#)G@ zHRU1HQ3qil)?CTJL~zU98lpa=kS~XCsa%xyHX3G$TKmab6IF9MSugspYo=`i2W$c> zgWv3OkyRo8$XCqdw9ZRsx){1@jAl6)&RC!g?r+W#kX`~+uY!F5moHtwNsz15tNEKX zYlFfc5GiU(*y`Celdlp4n#54mv3_;(Cuth2e3*wxAK@vSEqbajx8o8*Kq30s+~B-I9(V0D2IX!8vnhx zC8scDd}FUrG;#POtH3!Qo5_`F0@*(yB6VciY=ql->_$m)qX$KAZ!FO@a=VseU%FND zVb+bu^q`r_U~L1q zgvUk1a9akwNWTO3mjV>k3{s4jw}}s|+t%yhvcN9==-uHH)0aNa?n)P-^aKnmU+LrU zVe`FdYnl4BvV!*oV&;E1#i z(qUlg2{!hDHSsIAf`8Fj>9@nnFLhp(BEyoF7M>(aEyj?a(4}Z&JMspO75LwQS&6t~ zr1YA{DW!1E?Hh}hCG0qgHEUHl-!gTDWl6Q^lJhZs-dc<~R9~?M`%<@S=2^{bUVIe$ zF@}~6Y1i}$4oiY8CkmtEqVI$)a&J&_7xL;1o2whwFn^0CNS+iG2%W&|jMkr*w<3&H)&4Aqf$6vp({c2a>Rt9waWo{ejIv4mBj7HHt@y=jFG85I zEhMjG^gWXqE7^nTDAz#m!?+Yf@1kwG3Kr zB4dHvH`X_0<)FXeYp7j8&yL}cjYQ?j9-m|#p|E)ZGFF8lrRT_uJw1>q?l=z`GMz1p z$WrjDSj?cg1aOFNdWqc-Sjz!o+dRi4bv8=7maKElrLgbUqbyQc=nyDBEk+j-|F<#BlU zU9s$L9)l%?QU3PN0X)1d*h^!O?*LPkKYWQVEB-t#eYLRL@*BB46ijf2HM@ot@Ng=k zFlE`nFNBvCd7AIYlNBd?&y*b-Y_Z_ST#hklkH)eU%X-4%MqLgs(oHUw0}*{n%jK$6 zPCv}mfqtXmbXh2`iP*HmXNt~Pd*T9CMBTYQo{%i+ugU(W%j{ST2$_txx%c_;Y5gwGk*A7qES zLl@qiK>h`vY`1=rT}tj9(p}6l^AkQhYLgeIPQkyM&KWABep461IMMtw9`HlxZtqAI zUFXKrP$ewcsBc0U{Da^98z$FI<6;8MgWD%f zla%yGN%{Y#k3BaRsivn4A@AeT3mWa+hSE8tpPG3dFCnbG{WE`csX2 zDB@}~E$;eT$TpD}N7`SwdSvf5Osr#=3CG8lHBAH#pO}4(w+fZPqMrSo-gA0|IHKJo z(JA%6!k0LoYGwt}4WzxTIY@*2sDG3Om$;{U4Gkitq8iYk@D+Ce=87S~hfB?H`_cwh2ta7wVmD7tTRS&TNNc zksj$wZho6z)F2>m|EC4H0lkkqu$y3r@ zQ3KOWBSUnvoL4+eQ_(6V+sh~n|Hctapta8^r1=5~iT_EW37VgzBtJi7et(5lpf zN5w^-x~wUnDc4-52g$YC(&qS@-HpzTue{Ecb3`F3Y~BHSEp2fzrEk^cU)1AKcXMq~ zSw{0{whe3sx-PFwuqS$nM8+Lc7_S+Y*#ucf1iLP;wy0lHKEoiFV|pwy#gFv8$6^F-Ce`i#O!c zQJ!I72j`T~R5L-X-C{xjqRf=WS`<}&EV=&vv!QokrW;Cm@C8zawyK3^8;fz+h@6I0+gUA+UOkXJ!3kT-`tQYhu6sx5nqe37=|ClFExc->0sJ* zem1pF#J(e|aAG6E2LhGIxMHB1z?{EWTXI7QNQjbGnyMhN1t=s(R%G>=4{T8qxVB1W z9`$ofd6DM7w&g{W)!wP_P5H^K4vW>|45FSF1<(=aQ-1X@x~ZfVguA12OJ~tgnR8RM zDl@CxEi6kzE=puc??8y#gJ_|Z< zGtgrY@B|z9gsXEe@C1W);aZ4sb{HZ~Y$DDjdKjWUK=3wj!X8zCEvt}yIWHd(Ej{Wf z#&`wU*Jx}brr8xCTnbE6UF@J88bV4}4At<3eRHJV55c|}_kGf623!U2PQ?$H@ zkAYnpV*5ic_^vx>zqeA7B;NHV1tB2NDwwJQKFvP0%PJn`OL80KMU-8)ROp8}`uhmu zYv(B>=!E|wj5lj=lNl!MC8IF1`~rJV?NCdDrtnMXcfKHP#RoX&R&>E_zSKyM7-Ln% z2YTDEPT}pA&&n64*Nz#ze>o@G@>dxyLT18gK6Fi)JDhyC>>YJL_Z`F=tB}4D(AzlB zTZVu#&|5n0aX!$S5b!Z8=&|#=2=K8irWA%u3Y|Rgu?^@k?)&N*PoMM2v{tE#b7zp* zTh>wv*_0b)J7cz3q02WJx>dbba!KIT0K z&Ay%U0ljJP*S!Qt-F$2tx`)R5VPBK#w=kiavj(MKjUX_fJZtU8aO z+C23XjeH}hF4YFli8q97mf#P8vRuefHqu=n3kaO^P^ds0t_hEwfOlN-G0>@$I>0 zjB6Lv+;BdC$@qBG{@*iP@QGZ7WP5kJR;o zp&*b*h;pGcWy0u7R#N*)V=3XA(d^6m)rIO5nUcUJlUk!Gu8Wt<@(Jc=vTnr8{!U@IZ+P+5u1G4l>>ntV=m=r$)3^ zKBGs$?InmgXIw?kG9D}6IORa>r`c-A$;V6DGgHt8{-hAJuUf(6hyxFsb6arMZ6Uir z+o*8%N1F58jQ0Xu*rDopUO>Pct)UPAI6soT zk&X#KI`oKMyc4!S0r4~mUEPd;z9d}>kMXa~{Ox3-7qsy&=D@+z)OCNz(?Ej21@ zb#IVNvaL&0O3{4Blq6G(Kp$sTCmPcZd{DEm6Uuxgk;(^FVYO-($R524SM3l8r+Kp8 zRIDoneR_i!*q|utK~cR)&9EjcmurbvEo&@~IZA=Ckq5+3gHzjh6t@K4Ch9hN&YZ>5 zoGtfPKSJ;{Q}XY8 zj1PN^4}4rx$@Ig+AQC}+!k`mMp}4xjD|CqBkEed-E7Sawk$~|YQiQ49aK}9{lD`f* z!V){1Wu<+QZ|-HYl4K{5dAZy+(FV_Mis{me%oDHLn-|c_xd7!08@sINgtk(Av*<#` z3lE+k)IDX&USWeN9CaeZ^o{wr3@j}uO@ps>l`a?6}=wwzZj z$h%06|5K~-x0-E#5VS0j{S5#^S7hIckPnFX!-7wg-zZ!F|`?hKdOrB_gYz0Ktp^nO>J=uh$e(;#jV7WHZ3qH_O z&kaNe_7*kIyE)m?HXbaHKC*m4F-Ykf&B3rFHrKFb1wIgXge}>Qim8;y+O}!~n8+u3 ze?dVF5QPR4Pr`;(5{+9)G37@E1*5gf{{iV>?#0{Kmh$4_H#=3 zC`&N;$Hr*l#=rB&rlYI5JLSg)yo!d_%{qAHQ?NY)5G-4;Qd-azBNIexY3fM+>F-MM zvleFl=uba)@^d(s_W4))x^9i*yanIuCP2SShz+5K;`8t~hc52I7zge5>qP!Gl|P_7 zcpIC@onkx}9B>w}?b&-gYH9L8qVYTREqQA#f18}hoqAjk0-zDFol4~1!r1Y!8PYp0 zzdXr*tX*B**55oX-TKzCG%XT{yB!wc2ia~1S;vb{Z626w3_(Usf$INu0?F^Gvp0_j z>cv?=2>i6`Cv;Su4NmcYPhPFidxEl;A;A9Rl?9qyKuSfr700O4+uZF;Hb4fx_lm^t&`rKsJh;; zPgqfU#xXD}@2=F?sFUtZeW3C#>}w=da$`-|0Wp&-ymnP`o7UK%$d8%yGZ10W+qmtM zhieht!1tR}@Qn-6n}yxSl8^+J;6puR4pn5+ga$bm~y4FqHnM)9YMk8fnW$mVvn zjMOr^idancE-z#?n>9X2+jU%JpUrnHKG3%OT7BI}G1Jsq=YpY^wyxSiVQ?uiHu6?C zqUAJi*7;*?+j9aIdZ_dK(cBV~r0t^Vrn0r)8pvPYCy9rvTNtZkc~vtnaK%_M)`AN6 zg|WHkeY?8J_bQ*J@een@h494>dvOmRn1W{n|E*IXcV!swU7%}_Ym?j7_4HD6UFBTU zrKqeKU0ssq7ztzHDGr|{3u%*GGlHjBDd?tPt)qJt;Xf3M5$N|%3a9Yq|DQM|O)#`N z9tRLm))NpA$^T4>RxxvSwRKT7`@i^Pi!Y>)+Q{?IbdR@cIG6|&8L+T0Jakl06i5+S z00eL<5X^d7y$}<)yJ(YpgAu6r9eg!dwW9Cyqu9vz^$C9iKP(N$_t|!<`&{l$CS6}& z{>!-O?evTH>=QSu^W)(jqTnpn){GZrI(E+Ayd4&g?`y4p|02h-F-8u9$C{_}*tyd* z*SB|(Z05`o9itYSP;MUnEf>NMt>m)KS_l^8e~P1vf>FoV5D|{2kE*-y_wej;dE09{ zYS=MySMZLeV-yt}TW><-h9zfZV)p6UoQ(Jub{A(iI2xDXZx)_4Wvg58ld}2|g5;!F zIGV?@wa*62MZ~P4*V<;|=nrDnoL5U5YS3LY*VhH8H#Z@ijsjtUkP@0$@@awl5bYfg zfh!6cI_QzBY;@LPDmzC_7B&#ZLNu?opFMNXP1J4{pEU=gsqJ)-Y-$|z>g(}UMvLg} zca3B6;37?&I7h7kB~<`4&Gp@7KCBS1iKBwOeM@g0S=FQch39x^yH;&aQrzrp?d(m- zz{g*p`$ zjHaqJeWFwY$x%0Pif5LJ9wjodgD>42dtvY8m1V2 z)cXq0m937p(@&nSZhVYv-WcP@Tm9QQOvrPv|C$u|Y@V@`FS1D)=u{ApB2_XzD7TJXo{DE{3zq_&D%e(IMZ!L?bP}0!yVfp zSEmlFt7Q#dm2W9X!;xo2a~tvj3~i7(h&6e2GE|8CNZ8Hk;dm>)x?4TEKsN&lWWjmU z@F77(t+BZ)mn-H`oQZsWtD?2yGbo|mSa`ulH_+sGV{saG7w{puApv|MdBp3P*L1(T zlBHs~cNcthopqK|o$-S5XxBZjeSUXk%f#}`T7{czVr&*u4gQNjjt1+>&=bkdO_z&> zCH;k2yMrs?hz}+bR=-Exn|sR*C4$)`+}1TimSS1Z}S?}za~ZNMY5mvu|q_o z8@}%t1NLTZ3t{`F8{$op9{~t^DP?I$grgfi?$Bn) zRN^rypWc=*BJ)N8KqZwm^0D==PKOEi$^!(VgHdRMDW*(4>++rKI0jkMuOWtt#-o76 z`{YVPGc|OA<)vu?#iQ#7I13ZBO)KDc)UlBBYE4!xQqPU+i6ihFK7BPs#l}x7$D7$V zm|23KNtiYc_*(wJ?l9ExE3E^Ll7SY=L{29ZPg>gTceeItxkrXsvOBX-UV(q3GJOJ) zC&IN3;$(wu<|y*_Z6B@F+nExVr5u-HZ^xg(6$lIf>U&nLPtFpVa-0^UuP_!dvGq@P z-2g6~C&j&NRf<>#r^VN}riBf+lm3{({U^#eLdU~bCtB_utzwy(%p6XRgYE}di=)=G z^=w9nGK=(S%xtIL*W9MB#?o}d??~0bJFIuyF%Kceki@S9%Wm{X@_)DH2n1e=7ueI2 zuziYnAw=0s^d!$;O%-_9@W}O9@!nkHDYPC0_P!vnbt!DKjkU9_mU2;tAVtkG0xyol z9+>)KYmNieqEEvV&PV}rF)fmX)RMXKChPA=%(7dHgRMQQ&`;m^IN)UhIB_D|26jm+l(@SgUm{v9(9(%LaBCKiw}z`9nF8P;gOr`IjtVH zx~hB&ab!bad#E!6fwlQb5MEBI#!q+^bI{6^V^9N<#&HA#cz!VJM{MLiPrhJ#=~%zE z>*^C7WIPgFgW?BF^@1nxng_KiA8#pNzqu~K*Db(HXW}8u_=WDFGkE6Hi>0M&xCt%; zg9t-F&>*`)V==AV3UR>lMc~wDIACzdDB9>-?QGgpj>BODa_0)3p?D(~slaU}y3eb@ zT*lm*9Ug|j1wrh11Nm!TuxE}UGBM-XCAWa#mgZ2p6H8tMf*BGB%0l_pdl{{*bu8yd zoWb+Mt-)D5@(VXn8lIgRo<}`R{v{5_a|n;!GQm4@!0rwb7qMijCh6Mw3pt-&n}*pt zQ$r<A!uOk&Xzu)`iMvu!qc0|Z~7$mX`Cp62L%)r@_Z=){cR zN)uV}LGwt;GHZ4Y?-h_H`IuG9ymW^Bp}%Pi7^w*v>#WCU#h!j4(|kEQWER^v0!2x+5Q9lFNKrY3?n_j(yf#UoNbIPZ7;|CC8KBfs)M>! z_q%Gpuh6h(NRQ8Q_U1q=td)x&#JZ~sTH{C%{3rTnM#}&OZ&1mejvS6DE7YC&w&K7} z)4MDN=eBn~ah53*nVD{){=iIq^RslmvW@n$#uD>`>8!uq{iINiB9l}T7)E`)!)OXp z%4D}WFxn8=1kJ*1H;00go+lcw7z~43;>A`jnst$wSF}bM;tHqOf@8epA|V$%Rv8!; zr>v8jHl9ZNk+OF)uB)39se^)>o4-K|aEU)+iQlKg;MkGw!j|zxP(6 zmEo%}iz#qTy;n$Be_LqQVet-mjbP=(D)WEgn2j~FgnX6d|V=)c;+bj!H zjFE;Wp~n&S*X1hW=|;0tn`v~wdSIRL!;Ma|CcKhB*^EN#jm%yyKzchPBUW0S$Kq`G z#^Cs$@!HvCu#-$q&J4F~7Pt07sp{!brIdgjZI_E-(5yqF9ZK>R2I;$t(KL}d&resM zmfI2mLky{u_75*klI>AwHU~|N7Fo>?ek^eV_k*T!UN#p+40i^XCENpQzepAH<|dX+ z-nEv&yoWg(w28ne^fjCni{@loGkui2RTZN3Hc5>7G?r#vvReQ7@G`eM%TCM~6sZE1 z70Xz~$zIxUCbdV!@m`TwyZE4{EnVSit&sjo3c2_w@T^|u2B$2g1m-v*Wg(Z6w3}i$;l z{fsYg`Cpn_iPzXyb=%=KyOSNLHgH<5uP`4u6yHc#`T&}&`aAWvw913HCy2P`j;5cwNZ z_5&|nh`XUxov}(B`nB{3@uwPAK6axCIHe9{9J=m&ib?3^6KkYe)=v?efnDywK>E;1 zA49djUebe(k&lHB<3e9;)BBqA(;QQ>Oz9hR>6_=()60pY;=(q&nN>Yd&cy1$r#x16 z>>jy=wY|g&J|A?ozYtQ+dfS{PlyKy?L}aU@dK-Fa0vh>T2e_(WHpA$R6rP=&Y6kLB zQM41Mss+XoLJ8m^#dKEm^s`2n$Fn-LR~qC*q0*5uQm@+_(+DGTBnify)vaide?0}} zWq3mOKsd6qSgce><3L*Z9%lO6ZlVMBgvly6=k5W3UjFa(yMw>jlTyZnP(qdvcrdsd{Oqton(m zqTBrBxJ61oU4CZLqX(9}_L@J4enP=Bqfd*mOYzWrg|9{dIP6h>aq$|T-aUCd zeV2kT$q_}y!|<8IlLWiB8a{?JIdmgH(Ni^l83ILa1gu{2!k+VExyk3#uY#%kXy;s@ zBsfHMb~q)tprWTi>zoC!rBpiqF$=L)gN+0wS3SQ|eaza8U#2oh~$Xf_dMx|O~#?p=e9LTFW zGgpr(9}Je6dc_E09NjdJ4MnZQ7@v#5ZFDG%dJe?#qKh{4AE(hdgg?Pm_p!E6HW)t3 zv7(o9<66y5_?5|=&?_#<;MM+Va7(k%U7rhZB3>NrQ2YZcBHpN$<%++ENX%C6 z5q!-^H07O(Vm+YysGXq3dx*wgk+dI@M5bJ>iiS&WXW8VBm)^`KbQp^rqxM+snnUXpdVRY37fh^=M8mNrR3D_RbVMC#XC^FZ?^n+ zrU3#A8zxgjGrrKAG zyq&LzA_q%A1Fepy%gUEF%CGIk`xvdVZ6L`GybmTV`)}R-kbm>!4$E`F7*F+_N$b0| znN{8IABEb+W3ORwd2)xr6ZJ?$!LJB6Vw)Z^AxD9FmwDCpWk7am^HcZfnsiEth+O)@ zo-L0SP>W%=kG$JJxr!muh8zdH?^)nthm}f~OhDsl52gfyUd$Z&`;XF?wXQ{@v}jRwDl0KLQTAizAgv{9=A1K@^Zj@nQT_GxUFS#-n@P z#maS%^o}ft64)HdOqFqp*f(j2EqGiY8h{(PdJxU4Y0Q3i9H1mYip!p{A1nqFTrYNk zbI01EdQ}sc!6y71vqSoK{?FtZH0u6|fIrLz6P%2G+50W*)3nbl#u)l4cAknwW{KJz zIp>ZNlAkCX)V>d^lynFb^z{haJ7C;}&x#4j;=Pv!)eH8=Qxv23C+PeLOpu^>$B6oM z8@4x(>UH839am;w3)6>N{C=s22TI@|NUF^l`!|t1|=YU|L)#B*kBu)s!{ts{&j4YO;}%X|Lgb~1{o2yH}i*Jc^x|xKpJnPFp&@ju@*F5oai4h zj(VPt0PnF>Gy`i_lZOJq$`zJ3szeZ$7Ybnl1;>wAGjtwRh*`*vRf^?>O|y3%TIerq zhug#4Oklls9$vUs$O7*TwFUIcLe1!$zJkmIDQq{8hS_ak-dhQD)E?`dGo`(OGE5)@ zg+s#l+=NQb!~o70rqn!UVE#;rN8wzS3dDpj>^qu;FrqLP%i1x^^zoU<=T#jm+@o0i z@%O*Sd?s@*!lW0FgCWI1E)>CaUph}{q~1@G7>UoI8fKRR#9jxCQQglt`4Ii9;5mPo zBTGi_^JfB|J(3uXPyOo}kS}kyuVaNzkhZ_mXY{D%1SZsS5AqYp92}5+$1q21M^DGv z4IOqE5|~g&wnun^owKpGBr&ituwQ?(d_{sE^gTy!tq~S zvAI5i#nfPoMnlxLR10Hi=c^3GZ3c%n40xc5b{U_4r6|w-a^p1eG4+WloCSUD)m&OzdRGw(VqM+qP|66HjbA*|8=zCicX(ZRh0qzN%Ao-m151|IxjA^+N65 zd)>AAzJ3?wk5eS^Nljm+KVGk*>un;7%>6ia3;kebR&eK*b7Tj$@wNdOFf9a5Pa&lV z(2?xs(%SVyvQCW(sEeQ&6ZAvpk8H+5%6|2}>yU$d%L1=9{gD%y6jjt*vgkYs%?2vJ z^BAf`m~w5vIm%5~{x zBmNQ`FCeonONA4HUSrWF*!20gSY(p*TMvtIXO`mP9Ib31-_oHj8Z+WxRJyzFE7?%f z6;+$u$iF=Qrufy{0qcUI-#8}4HWl3@S}?MjQ&NlWAx}tKJZ|ySo2*k7F|t{A5j11f zZ_VaaH_i3eF2{;~ylDznUc0d{CZ=B0_&R;H{>D>BSJPt8ak;~fpe9Zm4Gxpeav)T3 z2ClrP9X6Lt_#=mMf?%_7PJsE>Gu$t&*zw|h;}L;38Q9}o)wb$^kIA*H4hg&<^l;q0 z%Wn!I3nOn+a~f2HFj^-BZHa@N&wD$p$dU0ePWoGR>lu^j@CHa@z86=ReLu48QU}Gk zn0dd4q^{wH^+vAE24o(XS|=N5Qdr$(-4a-JJp2;#i=v7L%p$_ZKldCN$80K}QQY%{ zCuke=V89En_!5ehQn0A2Qz_=wk(0-kn7wq>_vRm$$YpN)-_&nFb|+aH8Ce3KQxU~O z?mBD2rA%ZN<=-Y}DPhz|j|QArtD#*V-8zOU=#Jm`2kvsy*sisL779*KF}oJ|1>q2`=lnzeo3mej z$z3|f7-|{Y#fc*?)}6Y0T*91rqbnTWA^)HCn}x!Y1tVwo`lnW)d>i&2XT#6MtOw93 zJ0f@F?d^bFltmBkPwnOlaND=Hb6wv49`5+4`ZcW-{E)8)3-FEXr{D>S_p0g|t?`P= z7o29aPuA8|-2wN%0b%M#wMHF#0t44eUDtUi3+(Ck$fX^5c3XXq`UB#WFOML4*g6p8 zPRt`6U~JuZuALZctpr>SKOVD$Qgv43ZP{`AFO3ESqh`cNSA+L$vnw4CqqqC}A}917wJ&^GZJ~hpSzEFrHpUcbML3OH_*wYs0(Mf15BDmH zXZbUSx$;%0`CnJDJwc<$MO^q^8oP+lV`&jiN7;Z&F$Y05zn+vc5 z{p~a#P@7=+;Deue_|@WerKqur`3#1`K&3r^TOpVmicXcdp0HBfxBKw4ykb$Vq*9zx ze(spON>OfiRDN!$px)4wdQom;W`2!oK7$1&P>HcKJb&}MyiXlW9Q8r@jn{Zd1Rha8 z6w&<((H+~aISyi|RhY1I24Z+d4Y}XnmHoG!^iFNb^BH>o)`-1w(Wb1DfLi|M)TLt4 zrj$~EdVWnR}8QHyEMEv!-cp4g^h zk$papkxEg)%%M_gi+G^sCJ4+8+s7d0)q&+4M}SH(tubyMNSI`_^alW>UAE93ut$LJ z$V)P}d6Z&3=h-eFxD1%@|4Q3MQi2X3ebbJ4MG(U;6YR?Is}=ts6w4_P)SL8IF8Y*J z(n0^8gTVr@M``}uO+rSzoEW} zsTzupxJ{)T?2Ww+8I@ao_4AXf;w3RoC_-rt^->f!s8kK8Rgv^dv)?@xqS!g^_QNSZ z#@Y|Ms82g5+@f24%mY70W(c|EgI4p96w`ViRTmGUP>-jfD$J@&ft=-#R^3V3i#gP% z+6z%xR~4yl&|-dKN;MsoEp@&~+FE%sxF`BJiUZX1gv!7!_Iz;^Dw$0<2^+x$WuP_n zyk<;+IUcrkRRZ1gst&A1EhuMhzZ_UAc!?t#J0M3ColhFptqyYD6c#;ASnl9ovSPR* zu(4Q?mHjvNJ5yAOdC6FuYj|Lht6~x0A%+dus5hT34l`AdFCe*Kzk4U5*reK9B*i;Q zZTP|+YW2(3B~&+GiE|j4*oCH8xcLmaEgrTF4!W(fkSxs-x-A*DElzc0k3C;LUlB^C zKz=chCsdeSx|eO`n_T)V`q=Sdv07UaZr zXxfn`Hp$C6smpy2tN3*}DVm_-mdwEwo?Sj}m`Jm{iB_@?p68EbQ|Ng31tRKtDV}HT z6bf*jXC7%~B(gpBs57Okedf7)w8Pb6t)K)fisE?fuM7ccn)BNtoCM!U2-#@tOaDW8Yi>UE~#^1Yk|lsF7YQUnRF3r zysu@F@1VdU(aW#IPEayjHp%WfsqTFvG@11Mo@7(fc;9NF+Zef9`Y#4iXW-heP?@>1 zkN^eETX^+!i&eS>dF?#0XF4!V|Auh$zJM?;v}cZ!n-Ad^n!s8XxbDe!5BzQcsx8Is zHMtN685%-;j)iSj_%Tj|XGEntY;MH7gtNPYa@@!Sg%fQ}A^pnsgk@JehjE(2qfEpi zxC5?`gQJ}CW!)O4evEk48yjyAtPQIBCZbO_8XYrde-3GONpHo(Ccy^zkine17=0YF zgJe^__|8(H-KlAD2ZTia9s=#C^CGew zq$}2%3aX~newmKdkLdC5hGJVg9H-^XQ$ijpZA#sCxR7%jfA&lbk-S)g(r~#9_4NtOWhUmY`#gL<$$@~07 zZ3h3}Nd4F)#Z?k9gZ9Z0P(A!PhNupcaIzJKlvAg*j6N}~4$@o4n9t&s4@nc#2skmOuq9bju(0Mp+(=e| zJ@9WDVm%s8xN?+ws>Te(TI^mU3i-Mb;1UqXs|M>)47siZu!rcaPVQ&7?wu@kL;_pK zpsv36tsVHSeKZGg z#2H}8in=b3$9d3=;t>k+4st5Mc_4|xQOmDkV^AxalTizeRxR@zQcI-i(pE0v`c}GB z-WxA?rqoN){f!G0_AuSl+h(O_Kcxq-=g7^9boI9C#dt%B`RN3;&$a5rv_}>C zm$fSzt>X$;BRx#LL063^Jx)Qzus^mseZ%vJdYcA9>LFYc7kamPW$YT1U=8ZC zEmrEh+W;Ow#ROIjJIgg4jjVda4rM_rofGWU2-qSzEigwQNqvdwIxF5d$zR|l`Ij^o zxXB2yam2+#aIpVg3EqZ>Q5dw68CYB15(>|i@X{VDR}@5wG|WO9oWa`MB9NCuCgLE- zeI6yg=NF8|B^M(~4q006#@n*^&(7>?NV9-Tnt$ILE^}KJMiHQt_#^(0jiqr?`UE^K zja428!rWUs-S~PLD7(Z;*`#iix85PY9Y%WSsWK-7eLNyneAD-lT_a|^t4dnCmavpQ zMB>A#vWJv6P$6`Z#o2Q8Kx73BFVBmnfQD+`Lm9}=Qzmf8A^K40tFEY;Ph;SsGu&?a zILmC5v8I%WA80C}_VP*wK5@a$wi1+h4o>z)!(Pr|ql=0VohL=_FUAB7D8c=anCVH@ zkzO&bJ0oynRzvc)5g@oBvLO(zH17iVaB1ZhrazO6%B)%PX+Y#0cx;NVDx$Uy7hYsO z)QrIpX0$@L3mo%NSrHz}f?7`D9D;wEbcD;0l=6p6lpDj?O~5O@A)5G^o~+=MNh{!A zPFQ@i*_SP>1Vtt7hofOI#fDS>jIuBftmw|Rn#u>N02&C)2?&Jw0Jqw7UcXxG!o7Z! zckP;J8aZ{UNQ9AFlKkk_g6+oHw<-3;1m7ahVF_CGeSX!K=xIy7vftKOC@0kncSs`G z@*R7kyXH%S*fQ^WEmCjYjSIR;npz@Ewy2c#ph2G+Mf) zd*(Q6nsy&+fx1RG^L=CEpR-|%rrPT3<-R<+ai8G0Jx zs9Ey98IhM*mT9B7qLK1AqFL#`Htei>l*v>5)(jRG#d2I$NAUAz-9e+u+*u>SR@ z;=s4Xf=@V*eQE<|wv?Z<0Kx%`PD{c@m(WBhda121KPTMRFJkwe{xb z3)tTcBVhHUcQ9#xed_Nfh6bl9&HO{p7m)thpg5xsxF>-)ric{5fYlZfs#Od8q`0vc z7Agn-J=X!q53mfh7ZF;b7W`TSxsVXb8->SNiETu-JbmSy%QNn#*7*`|)~FdLz1M8M zH%ZlbjRaIn@BRuMVUHhshyj#~?}n7X^lOr&AkO7elJF`3y<)ZAHzi0qWNXXyaj_HE z^;7b1>h3w!HQO`L(>|x#Q$De&=B3ks@oq8a1fZd$1Ge6ulxL~9E7<;9D!Xl=vXKd0 z5e$5{tO}7JIPUAhxC79z+NGB8J~k6D^GD)w7wGwK((;hyulMmS!?~~zPbqlt%2H=k z$C?Y*^iHe7G|TBk_l6ZB=|sa zo(K{>60yuGaqE1^C8>ncis51_Q`u7|%JB=QG{A)-`>JbHGvo|Uu_fz*wz2{RUv4r9 zet|eoB1spCSd_FqBTBIr;6#sl&bDcX1hq)fU@VKz13yUg&^mCcC|`hD@TgMe$}*}= ztimOFaQv?&i9W3rk=b9m+A13y6g@_Il#DbC5)!vLWp!$F<^p08z9afI6j9m6oNF1U z1nt4Us)Gf}C}|Z5u_cmkk_o0$`p~Ek56bR=fDU(^*?a_D*vVPs~O}T|6yz$HS=${fmPFgC4FJTTL3cNcd{799 zaGU`$R)&r1ylg8Wnyf}k7!Ok=_AGGE8v*g{z{50jkX0tag#4ftmrK2zJNv_ApC!^H z6$Ux>8SjIKBuynYJn%buKz!Tq*q4qpNLA&7bQL|$FkETuy~!5WTAA)9U6NdHC_YtM zKqm>Fnqhzr_7`)*{`W1nKQX&rfZGI>vD`&qt- z{~WF{L5H0GdsHn-v0FiVNCML(c| zh$Z;$gEee~4>g>QIM#=HxVg7J=eyD-uXE3!z!S4LmZ5&(gngR45 z$S<{(9|FCy+M&)~@Em{E_U%lqp*ql)dkYTxBstrH#B#E8t$XR*&XEt^a2!1U;No8Z zB7JxNZ2jH640ktp>=paH0u|*oYE6Hks7_&=RTzz0H`UOqg2JcFj|2Cl=&ngKKva zUzYj)Lg1u1agMFkTt5-kwmb3N=TJ+F;*@>xpvg14~FVP}e!ceawS+l*`HZQ>hx)%a2EZA2m zqMG4nrcw((D#YJn3iZG&X^Sm;DrbR|z4}XFnd`pqX*d@SQSMOr8`L6=>7m>xq|cZ$ zkZ5ZGwfUI1UyS);j}zN5W{_GM+K_Fy%V&AF=C)#;>}De_5Mqe6pWVcV5JwaIPE z6o|lgoh$mwnXmB%hUL`Gr0Rldga*jTWTr`DRi9Gt0^TzH9r#$LNI zwre)J7vL#?53}M#vxl2R6mU>zi%<{S^xm|G5g4@M84c4C^8sroZ;^ubND=t(VvSIH zzDpj9@eII+IdQz-rw^bz^k7byvz^}|@v^=jKrV6V&YEBM`I|p!=~4|6iVB7a86=iZ zq$T70s3c$WCe=<1!~WKMz_+QW?%r8Ogketwq`W;4 zTHj`@xe#woAQlMT8tntu49~~`muB^_=@>IIS4E1qC!Wf3_hq6FG%MJ+yA4g-FYWq9 z*HM3HI9v?qxdMQcd&S1&_bY^Zau|D}s(}$K-TE}uMA9nKjrV=i#ll>6TJieMD8D&6 zE%O`R^PoGxwV|76$^Ye2*RLnURaKe})8Gp5F#;%-d}dgd^&Hk2Ejl#|HDn{y z#fHEBt`k!9rW7189>}oDvCR2(@B`NLRzwvcX>AmLvHXv1NK5blUxrSZ%sX<7$%MUC zg4-hYA7B|%BT4I2)N?vYz>g@;?jR7p^FS)JS69J2Vu`2WT(Vd0u{FfSwtarVzbJFf z?%lWMK~32t?9vGg$#{%}w*)0pvY1%vQeFkKKcI%)?eMFyr}da5XV8osEF@3AOXVpE}HzE2enj;sjV1e z(B3qGxfD8_-W4G3wgn?y*r2Z`Q!gLaGCqw)193$QfSHdxp^B zG-(Go3y>~M!7CL>?~Hxm8_^r5SBx9T;cDOw&!q0o>KJKm^?JLkapv^k9rX22nj@Gg zoMg)H^@etwIG=lBcXRfm86TqyU-jk(Ezq+;g%d(rx4rHza0c~Gdh8LN?kjhp)y7ER z3@Wy6FD;`+`tOKQEn2a4*I*QlJ|a-<91Z>Lsx{iLDGJ;=bORl4zd|j!I2qWS>sAiU zug!Ri9ql_rV$j5{-Q>nj?uBN3R{d>H7$a{ti>rZtC1lrCtEXf0$A-3eaad6&&s0gr z#`XJT6;n~i;B+ySovfp~3B~$b3ugC%81cRe)^7W{_@t=wrLvHE!~)LMaH%68VrQqk zkXozmBv@_xwrCEl-coBiD96yEym0{3RG(3gZYk>Mu)Qu~%(W~^>AL^2J`?X(w`zDF zixA&h-K_w)7m(G`SE!ru(^k+aE~Eww>EJ2So|B1c>kuzQX{L=CtJ!(^3s)IwEvTD$ zs4Hq!R8%Jy*_5Fmj#o5$3)!X6B1}o8G}Kh{E9$Jdb~gvItgWshT37bC+v!o1)9X&= zTPH5anxV7OLb+Qn>>s;i6lbzjmC<@Y=JuWDbk>Ts2D4bz^X6YAh{M(XCh@QS(FDCD zO+haXGCjhktw^vQ;$Fib4(?rD!9b?1J3rdKjO6*Ku^=WGJ+<&xR4^}DOz*!^OORaZ zb~lt2){vH>6XX!+PxZg>GqO;b`3Y2=UqOl^JMjnHcwoEt-LWjFgb;(XY=gP}*b&uP zT7*LzwCJTKoE7>O8x2E+n15-H>ek2Fl;7qyy8S$uT&fz>A3_#$2FY~1{+vsPHqqob zd9qLT2uk}i<_+~fal4bJ){liM-@ZNja^6uLdTUqI=nfncdRG2S)k+cKWCnGRVJhTZ zTYD=BhA#=uCPUu`?ZFWjqVqq~*5ku(n2x|}3PT9}Qc94w_?u$!qptl`6KSj>Hqm^y*wxKfh@Us)(jNr)ic`0mY=gWuyU%Pzn>fvg$SsHPm#h9k4AF?(_qBuOgz%E!-}%Ao6zKaXZA7F%B99QF z$~kD|OW0c~Mrb%=K5@E-lqV^OmvCa?;Q6Y0f}&v8Kq8xe&~tyAwtgD%QKRfr(~OWR z3*|QFCcxYR(Dz}G_ea@+T`fwQiKeB83;OUEUH)|Olj4TMn}^mS;lUVXgWOfpLOn#d zYR;2tAH>ns&Us;XWF;}VhJqM)cxzDKRcllQtmqMMeGYh_WO1CQP)O;R{Hvk-)AiQS zx%n|S1_1J_=FWdB}nj2jMQoTfdvPoC=1nm8tcb0Vc+J5+2+{Qqy9J)bFR(|d2 z3|_*62aJ^zIXJ;AWvSvs8TKB}@T+e5BK&`~W$_*JyM<67AhP)XD+BZYYRmoyhB(c} z6Hg6$_^VOMWhEDt#tyb1Red#HE?yE&eTA)wf*!J^#n!e&?ezSTEW_QMV`8EiA)kte zMjx~d7#KAoR;>)s78C5WLBERHy@)pW*tW?JL`ZCoJ2JV=XqGr5|e9T zv_bWEYT2rw!jyY8--2`BZRtd?NrGIBGFuLKsm0yaL)JxKl^ph?&1k9Y8efkrxh}s6 zL{jRBY?Ofrcx`F%QNoqx-}2Ordg%oO_&*4y6!66!xpdktL3&Of@>OM~xN$5!0!vsj zDp9p2tpHpp(b>j{UYsm;yEFo2*}*>-Kgf*Jo>BK!*j|}uBF8Ob>Foc;_vHsYRF9cW zVs}C19ArE!f5uuXW;jYYtn6-HfTLxI=+2_U4|s+IAylL2WJ9uxpk|tzDgoII=YRap z=uT}||Es&I z!i#8vtKkl8`|X8q!jhAUcW|)j;$g|C;KI=tJ@GM(_mW?v#pkk%*k4t`WABzLZ*W-#L9fv2@10QMyVc!d+-jrYtf0 z2}pl{!?)bbb22okN+7ECD2r+L|AW3skoR5gHOth&9MJAe{edE_oyC1-0x#0~6DKuw z4LQ$rxj)=z(9xG2!3kuhHK>}~ZCCqd-YaO+J z1zPEf>>2m8AZNgsx#EK*zX3Okn_T6jC7hM1H4Sg(--o3e!6*;~c*Z zn;(zaJ!KAa(mq{d*(8ZWHO2#^Hzoml`Q^i&xkH1X(;%1RLV9DT9(r7^jmV9ykUF^&PGjBd$DbP7*yjD@DUAyK*+LUCh;tcT zzMAb>cuf}TKDdd9M{IAlES+~g%4h^var>e#Gs#)PF3g#6^{5{7nJ16=20 z8ni-EC-}6DZJYlkS4KAfiB-`NRG88Z!7lx(a-V|%jGo?pU^#>AJFV7Yzm?sSw#8Vn zoh{HOSBawr>bLz{Jgn9f9){-s=&>1&@P#KlrdC)Ee zkrNBEV{VB1I~?h&v;s!zd?X54FYH$cuc0p;(2qpOdt&TY1BiPggkufUV_MkbA{=%< z*$OM`Hn3MUgySXC<51XR>X`c;;*z}8;z0faP(FGQPTKsfImxdz=hrp#nKJg|+=P@v zz|589pA!$Wl5acnt$%_1v?68bGe_+CTWJN|V6WgYnV*U^Cw8#MaW`nI%vm#euwQvF zo;8S9ScQ}KN7?HbqyosK9&97{&+Pspdp%-*2yc0N(CYo;1jq(oO>C+JauSVm{Lt$^ zX^gvm_Y^$PKZCXXoTli6D7*H!d5LT94fy;YD}6DZa~L`}2*|;==>Wn1q9S+jkTU)M z3D<7QItpk)fX~tmea+P9?-%)S&Pr<}s%^9p>w|(7Skl(>Pz`pRX7v|i?tSs;&p@Q^ zbS$*r0-$JeATdbd1|JgL5f>o3?i4c>e(?Eu*lkJ)7zlCA>pC9)zTdAxP_=LSuNO@s>k)j6(tui zn3b=TJgB1*d?U?{#*((1@e-MQQ8ruo6=kO-VGy zq`kcb*ltW|&LJR_X0t3#R&i9`)tqifTpUzcQs#BuzZ_rfB#Hhc;FmD2Mz@K-M4zAS z$(0uy+ds(MHJC+qSQDe#Y>am76k8SBj}B!2CqR<;N^BH=90twvn@M~|c$P~n5~P}I z-0k0GHdVjJQD0g16~l~tp(QQ@Pl>?XipZJh6Js6HwO7j?FSN(kk2R4u0tUN;-}s{E zf}9YD&cZ)M$2;f=;+rD}XjR=K!Z3k1P{qy(fRdRQ^2{u@p=;zlPcneSd3((+%sq)v zMa}FcQh8|(``ii7D_)6T^7I?bH<@7OYvtiDPolBL_KRL0fQPX#O2FMTJbv;9^DD^&arv+WkwjIi#bCC8+pw(WI;P!M)32$)rnN zk#uJ z4p#KE%(c0hZP>S7(1Hl?uQ!9=L|KLn;9V52qCyP<@YoP#H(CqU4~&;wYAB zlH*xz`^pY8u2Z;&1s&jF>QA(~fntM5G-`+*$sT*x#nnTM2Mh7e-4M+$dUAEGE_a;Z!E$nJ=g5`uccr$=- z6@p+TD2xMed>y(vJD-q`s3tOy#D|?+Ds9P;y9b!|43J;~{FXD^23F_~_O2k?BUm#1 z>Wf&H-3{$iq&CEYx7a3dxOG;a1nMV4!9SwTK*1`Wlp0bob#ye6BWrEjoIoS*)@?1- zk6^;gIouhEsHx|TG?W&Hc%0${RP{iGp!;)$EL{;M@n1$Jss#<4AX8 z(@TWVUcC;rz|05m?*Hv8m-S~qp2adVNRT_~6#^LiWG~@5v=c)9e8ND+{PGP73=M3i zlMzYH-?(ZK5kYHz$KSCGB%?#A1ElWYP?8^;WGr%xW-PWk_z-c7x0q`)&CIUi{p88_ z$&K}G3nc9w&i^^eX=kqv6BBk-qRd^1Q;C@n5g5o}seqEe;iZXz5>p&n^eN;u6)7Y==7a2CMOHFRomvr$>9sT}skGl0-4fGF6;}l>mb*1Ljt_qHU0Eo41OU z%4KA-YZU0gI$4!L7=NOx1fvuY5goR`EsGklGW~^6K4KoRk=w*x>ZDybY^TIE>@4~*Eo0wX9=q?J}C3oBe%UKo-d)0a|P|0w#(o73pi>EYLMMKHx+-T%p)2QFB%a+AW)C-3FNK)N+3{JW`@DO@FtidkIjj1T zsS0?_8Pd<}_=Y)lC}E|#evx)Cq$F+>?8K@Y$kdTy6`2m>1#<_GudJAxjKPZI0=wL z9InrWv!aI$9)DXEIH1-7}~yO!&qexF={f|_qSTVpP7khmJEd%2hmQV z*ka+vYRz6n8$06`oQ|kD>h^Ku<=Li(<;L6o<=#Ke6ni#>nJe7Z}L|Gu|!v zhkO9k5LRJ*Ke3dFMylbjDd?RcKO`O^jvpI;HjN>(sv*^$Ncn8ZjtIm%3?ge-(a8WS z#GgPyPC&G_1C5hub6|?-@-T_)?+G+yl1yb2(My2U7#Ju@gm%6%4iBC_UH&knG=VJ+l{kF?JTQ+`AM8SchIE$<0eH zBINuorQoM1eY(y2m9%(5;Hz;lF^(GcCG))*;kPmEi%-zaA+h0nruEm>dpmx?8!|{I z=M~&ef(H5}`~^!$1uI)Wk_QNecGAUIdJoa&%KfRQY@`F81gq*KXbbO-8!X2FXVnms zUc!RJlho2VCa%6AWW|py=#Z-xKkmbgiwspF1$~vs`PU~zJPTY`YT|W?X|rDoMXx?o zHdBzXY`3KI0Uw%T{FVExRX$E zG5)vY4;hnX4fJAIz`4sKL`pC77`*CEa%rITx8iZ!!l1sE{qbs$wli6&uiI#A4o1^Z zPL?A|Yu)nP=TF=3B;-$`<+$YUl_5yc$2K(FNM!dq_HJ&OG5F$enYFbX49*BbKkem0 zH1qXWhD#rQGRq(I?ql0u;ks=yj^Fuut7vezcQKqML(&g229y#q9=Kr|6wE-b%-W>B z+_owTa|i?6RXgk5-f0nr&Gmx;-`O(qnXt7&?)`(%&Yq!Y<%>l$F|Ac)S8ac zs#`>fj3l#eUKxSifR>Nd5}|^!T0%+z{FqxO(ovJq zpEXcmOn4MLC8i;yf5!3`lwK=M&ls>ehBqlvM(%LW`r0q39)|O0$_qUG)65pOI-@eC z=4%DrWlWM+kr(Hxj#+!z8G|Q zU$9$ha+p^od8r7WM)l>$Bkh5$pC0qiQV0l?~LJJfELANMrt(oZWq2LGLVvJYo2pmuEi|fztZ3A6Tuly=4M;d?fWKU zV2WPdG)=*{-N$c}Lc}@p5MSwT3L`s_o(|@bkC|>CZ1^?}#r0#Uz{7Qixf5Cdc!NZV z-h;``O$vF{ulJsBS|R?Z1Su*n0GC%C2@n5c${0jfTC{A|cFX1_AV0Ed?E(4}$oXXI zW8uS%`xIM@-nfao5Eq_LuuMzxmKsENbu$djt@7iS?|4A_ zQj5uFa_+whOY*)LH% zf4qb8{4NMN<+JVkmBG7?e*;C~5w4B$JNyeFI%>gkze}e*wH9$moeoKqME{d1Ky@!} z+uF1c@P75qtP9POpyd@aGP+U+{~RmuAK->swn5^Brybm7wSf?pB4XAd7v)_ms*WV8 zZi>NxF2Xzq?dIjd!O(Us`g0D%#mFBBvC-Q2udIjHLg-h_xPV(s{U}@y^Sq@meVdy#jc68ivRJ;))tdgBG0RBCO zmuR7g>G(Au2Zc_)=ov9qD`n^1l6gap9OSZ(!gY1!3sYQ#);}ALXbY=6dlE^O`ldsI zFGpNJ{c6xfF1&TmarR=Nt@t&>6gs*o+G|)qM}5$!X7-S~3yC(gV!R3lozlAf=a^vx zN`4Ezu*~t*Sh43Yq_;ASKWsk0B(Kx}Zqg>f>jLTTYts+u2ODhonc9OoD18Ot8t?+J z68L@tw{0?yANAmnjH7F?B3HqDS2GrIy*>8X5KL&F`2wKt?C=%WH8@u}&z0gw^%%^& z*V=k29q6al%NW2ifuiGSfA*&P#@NftUwnH^-f(!y047D`+~1_m5!0CY%S;!*KH1chRPOPy!qu@?DMxnx!SmGinvvyb9D+Efz~54m zk0>$_4WLS@gLqXrV8o+Dm>=OKaU+RMrC@W__E`+JQ9?b$rOsOvC7%*zMZXS@5*-nb zykHCFT2=%oBl@?go1cRt*(ibWVatzJ+?6YfHVDva0kv6am=DN*qmBlQ!A9F01hx z#@0{;hc(H2y{`3fD#bg?z?io*x1zW@8A;)nPe**=jJn%eU7t{l584Cc8exFZ1C$Mc zE{(9yn7;O~&m>Tw24i6E=IGb#zD01YSlyF-qFbY2-IFV#m;*@Mme)mOz1%DoXnODh zM?~A0T7r*a9CD(BSf(zLLuQW~G~vY#s0|Iy<5i!9rYUJ6N+X@hQBL{@UQ;&npd+tX zV#I#n3FxK<*!$|_BZcQy@ekh*11P~!lC*mXHxX@8LRXDx6BKU5P=jVYY@u<8YpR^# z{jW>yea>{{n1P`yovfGj)R%Qzg=WlKe527q%77UJ?Y|Z14c2pJ3!8~F;|t$T_Z7n( z(#>&&J3ryB%0uvD`Rx7dvSZrgVsh1Q=GGDT-Xg#0pA{5~>t~^z z3kw7Imf1veVVo2H#4`9urC3Bxvi?w;JX5X?xrCFc)65BtPY$I;rS@9w{QN$A;L3@ z*N$GyipIDuYWy;hJQ#1Hk~P~u3VoSLuD5#7_(dxJ%P$n&S$Krvp`>%VlyAAIUa%l7 zJvhU=Yp82)&5Ff?MpcFsS)npbS>b!n2ux|M@PqVYg%U&K4`s%wg`4@oWT4^TCWW5) z?w5x|lw$ZG;+(#(RF<7yux8XGqjE0w#APry^p^NT^WD_YIzNHS77(hIngLGr0rY|0 z)OuibD$wWl@b}b0W19xuElI|V&iEf67RcbaYg#`wt{&>?ZKw4jlkBn3DgEZ=k7133hH1f)U2-}`jPV_Ucn6$?#)o41zr3@cSNY$TcGaH znXK*#HrAioZ>D~dd2|0t%?+KD_PA(tgfW0U_uTF|3E~}FJwJrgn%FG)D#ByZWqA5A zt>(G%G$x14nrW=#=p{WQ`biX9rfJGoKbx!(va*IujT&uCuNU&hS(1e}g3@wthBr?* z6lQ(8WXmad;f))ob0-Yx(Ffn=Y5?*Q-C>=~JlMJ4T3GyX5soDDfTr9<(ajWk$jTI> zxEJ_6^PY*0pK>~j5`GaBHO%f`XBIlB4|_bIFill{XaUmFZuKY1lj28P>&KJM%d}y3 zwsXYDgpY0!%&Nzd(~hEJe5xQY${y%{Q_=UW7>4eH6p~{IPhI6F+bz7wi}s#r9Sw+GH?!a%RFrc zueBR!CdLxEE;`h0b;%D;r2c*B(#iAhC6(S&o|V8b@AZ!1kDk^+9kav~-h)^CO^W^3 z(2kyZyK3Z-=%>z()YT-Gumj1<-7=<=nVHq`nZl>zyW``Lr)L+fY@_Orb>sI=l)CT! zLmvKA5kmt1xa9nawlw{>wA<*~H%X3H9{abcrgNUmwp!v`+KL7{Vt1NDc3LH93PiHp zG!HLbRy6%rYTXqz_a*!ui92#e2<$^4qISmH;@o37Tff!d!KK(mIecfF-x_Pfgu`j| zXN$up+Rvg4wY~yB-W{V4KZ$SrJ1B>i>FN(0CvJ%`*R}M1R;6_%mM^WFRfx~>yXeun zcrW#^&-G$g&uz1@-%l>zY^tj8z^na57B?)Y13j?Xbj}qG_rU&wVY_N~F5a1Gb4}uA zI;$Bhr3dTRf;Rb`9KZCNN$4`_i*=Prb}Jm(TwFNaIvXo=-EGT(NrlH(;?f$}UJs;Z zyRv?jA40kUsBue*5eF?Cd*&DR?LYy%R@>t~=pTOST@c&L%LDO0wrUJUZ(4q82YJvj zEOXcQty@$pc4fy7pNX#Pa_e+uPA%42AUB__($}Qj@4DA`PzZeMin_PuX(GCQ*SzrX zX!C&IDA^h!?LSk~%@(TKnlc>a(^^|$Ksq|xBiZ^(#-j#sL;-+z9zXZ21R^=&!xAiUEDpdzoE zuKw(e)8Dj*WZZN58xZ`nCK&3ST9eeV6+7C^Kidx`yEm+PY-$Jum`w90nRQxaQLead zBe*>ut}el&_=o&R6mv;MxcCZlyuU+M5PnrkKh9lyy$};!;>&Z0Vix-A7o&MDSzk*Q z*e2PT>wQ`t9K^Rz`HP!rYjmgM`VqW%ZxZh7NNREukE3tI8o%y+mr5kQcTlY5G>g79 zC8VeX6tZko5+qF6e29MPb>3hofFDl%7_xl7=YHR+d7%3+=*8shUL~dWl<0X_6ds5@ zamBCoLtI*4VKsR9Qu88Iz33kZdT3JUBbDnTRqKQ6!EsSv!8`jh`ONk0TlUA-j9In3 zvqymSq#I<_WMg;CVpXI7KGOnXlDiLYc8ZM2^TVGtGQr$A8*^koR;uMK501{6kpV ziT6e{sS?7Cpb6~*F(Pg*Ey}H%c#Bnw!iO%hdJSylG7J~W4KZxz`w_WIL#bY&6I$vY z8om8t{m&kda>dC1$h7yln+GqfqXcHLJPW8=^Ql@bTS6gaEf6pNDADsr$2u;k-26Yu z>-mElNV!j#0L(Q61lj-L2C=lUce8L|5i@bMko;fyUpvT@0etr6;8NZkp@5s2zViTF zmMMUk3nygHamLCm`z+Ui7dysL%adkRk7T~~f`Ys_QP!I!6ldPY2_!~uQ_M5(gMK2k zW38^~{V}(FS!4d|)xYn-$Z6YWVY{1se*0wyeRB$5pw4+qu=IOtGy9sWpGET~ho#0@ z>B53l`%&QQnILyW)6)vkOH6L13enXXzju#Ltt;}t}VQV}b#Q~fv`3jrTcwaim!>0coKDrF zy`XU_k^6?u)%|gOEd2OUved=288WXPWmY?nrjP7KM4?qS37%SwJ{PK0+umaJdi$S+ za7Xn!xAeSShc46hxG3=laIpqWx*4H68@k-ujh8)v)xr@p;Rx z{kJO8C{YHfDPBD;Xd`wH)oa5Mw$iq^??-MTJuWS&r*Y{q_del&pbUE*X){&mH2yRp z+?`ad7iFWmb0sM7k@A$Jo0ghJ|4JVMSpH#K0h7Atk{PaKaWm2^q_e4BK5o`+tYJv1Z*77w`Ti93liuG4Ii5g5utq%J`hFa?((^0Lx^<-!`v4!!Mwt_f=nmEEkqD1DO zS#I5(jp-D*qu*qX5{_G#4yHPhpD{gqI$a8ua#uPHZdVO4+lG~@N=LOPMKT?fmi&w| z&wcvE&u&&XQryz?-O_5O*t)(RvM7!_k>1KBqMN?ZH2q~>2WF?Z3K;LJen}Lj%T{V5 zIbMk6%QJwz9(=r`E>0lnZ6#4ux(pr-yyT)!5iT;!O;^BJ;T$;K;_Rg>wO<@B{R?Kk z5Q`jfF^MTj`-kjiB(__M?dSFyHF?BDGrF9~x}c`9koRnssie^1H909|Y0f=djVqP1 zz`>r?RFHyk((dVqrhCfd#T<@_PTg_~;%(J@_(MOvK~&0-bN_b)WfpD0LH&$MjXE6o zJY5l;FyK9pKI}c;WTfPcb|#|~)0M^-O*PV$@CChM|K#ov(FVFyyW=etTlRc>lS<2Z zw8wmlGIh3bFhs#}%5AWhK`V2?(GxqRF+GhM?7)qNqHWo-f7S>R-Y~hs(c-J(}2o2Mimv#=$cnaD3X(u*nF`s>^9 zElO&Ng1N{dM;g1vejKS$V$941Qm~HlOYos%(?z9`LyUtoORhCq zcd0xkQL@4cuB{u-47Y{`58sOg5-f+=K3&|tJGDhE_o+b(; zzCGsl*c3pi>ru=N6f2HznQ|VaL<$#sqM|1}abOyvNia%@3OD5;%cG~jj}Bsl@gn`! zS^T@{kKkvjtC+j5O4TC5{~PrIop%Qo6%{?w%HiUew#A-#3DR8RgaRI@-mv5IFMXU= z?~%P0+>N^~?qlngZHs!Fy-xoYTdd6VgE`GcqtD&CdeLQ5Dw2K@k*AQ+LkOwQkR#^6 zidF1mmWADXxnGEdI!U z*e+fQ;-7VVCzUsX91=|RWYgca&3L=0?i<=B%pP zJ3O@g%Sf_+)f6fzEll0~i}a#p#Xe%&ahij*T|Ct}uoqixK;Vv~Gf^U#i0vfg$8pIz zVyX6I*TD=sbAO^K2{Y(Tb=w2zXw@dzdmBTU8Cc~SzuQiw25+6pqo`DTdkoI0-HtPq zCBKE9H_IG>q`;#+2N1>Lyr)ae2R$ISMDCbS}-dZ+(V$!Dqh6OOWem!&aH)$w z?dhN1vecEZ23FU2Baq1<&Y&5UdHYqpcqP{)E(&G(#j|c>EUxT=T=Z8puY*fKjjEIW^-A-sZ}v+qLJ%wB+0Hx9@oLBW#bI}rh;yhCrdXr;WfW2YoNcx$eo*}(l)`hHmaZmfBV zEJ;JRqy?T<{ZouOvvq>L(=)k#g8o$L*Fh_l7RF_JtI)!j8dD3dZgn-$Wd2}=L#*?S zuR6-nYdNRUXZK~>Z~6i2{DshfUGRJh>dVKWeZaO(Gt{sWO+b@w}5{e7gu~Yp=0={Pv#djkr97d zWU&F0zKxp>hTy!xH8#chL$|_HJfljGwGta4{T zkBo*0xl7gIH;XidjLL`JU&E=Y?3D(oSCSfx;`kgpg0Gh)Z%b_r`HXeyXRNe}H_Jv% zRp>*<;>;5#mz(CLZD9AT*R zvMuf=&n{Hi53kRp9`VJiV12%-VMsx+i+RH5{XlEbZLCH;qG)9~bQHZ+{Vwdm+HO&e zgJLu5`xQ%%A16<9S_a%6fWv}+rP-XPu5@~UCn5Rs=xEy&DmvqU)$1`Y(t|0r>_V+ z(nXbg=_MdQF78Rnex1HL5T~C99ZxF1I3m-Ed35#kCQtG-r4ZZDtK`}eYkrEy7Qxl>Q zIQnsuEm}9G6 zS7Y6`H^C-pvP~((FP6bn_IUTpwscy2w2zBJcIhN-0H1T$<#@iswxve%@M!#C>b^w> z4W*&q2r#ex{PA3ITi4t&hnJY1G4)8$ApYA;rcmn|N9ma0n43TBaY}&4tY)xy$L>n7 zDYhs=Ux0;XsJ*p4RLPPm_lTeRNnR2+=VMhOD~qn3Z|5nkc*%#(xQ%r!jFm67fr%)n zAv1RiZ%9}6pE2X51rJY*NRBl%yyrWSmgB6`HXm2&=%y%>BeVoH8Ye@m9OD1^&|opibng46Dx z$!cm5TPVHcfR_?Of$#|KIj(!uP#wshqz=O*sxOQQbCr zAUkKUjmEO_eo(p=Z$&Mfgx@7q5yX!hx7ngm7d9{lj6tKQuRpeGQcIY=dqR3Kqo-{4 z=lm7()!b7YWjQJOW^>?(qG&6UC?(2MCGVFYY}4Pr>Edq?Roi~_y?@dz{^=;@lFhSF z=2fgIXQd+3yXgs*PxkyW+SO8yZ$G^+NMr%3((+fJu3_NSFng=}_1p3}j9VLF;z)D% zpY5lKR|it$ca1v+4t^Y8M~MIyhK@inLs7c6Cg$#t+$9~n8@XI`6CqnC!>!2LOi%1@ zbXT=BWfbLB7UwM~o%idqiv;nukXuwj`_Nb#c*ZW!BGf0_&s?pW`%}9|52CR=&>1>I zS&&_V2UEa7)AWq|hn1&@7lHKLzeo?iLWHCWcBgygbA?sNXB=J5ayIp7vSj?Tg}k&! zqb-;t3~p1szC;)#Z8R3Yk126e6-_$|v(#}LMH!LXk2l)Zp8SJ%l82tSaM0vQ8@&GY z!_P(ZRpuYTv)R5ekXIz`Nk86JOd|wGfab#QsNUc%;G4{8YqIcSmiGwn(j>G@ODVBy zo(0dQ#NU&2v$zvQWv2nQs+$pvq~?cre;u|ON;v7?zu;~LUJy4~W;L?Tn-? z*M_RNVy~5@XQEcfDU(huKjkmUE0xq{qvH-7&Zhi`ZB7)Az|p#GVxplR`U5W$*X_2d zvBvJ6+)1xjiZ1(Le~8amgv?-AI&Jn<-3eem%Y(atyV~}n!INBYnmkMTv?*8D;F~s* zzg#|<$sErHdzmkGVHJBBIBDL&_^`ag~EJ5>w<)gwf4(71^8| zBf&kzejENXCQf8iOUj$U#<+h;4a#`W8oj`5OJm|dgSt-71`0uCrDx4&{p^%~53im9 zeAlvui?)LM5E3(EQhsEXmh!T%4NXyg^%w?mU3sD%SOYVr=ikBSLYlmWTffb(sTTj=xX=yp4hqC5$qrx(coI&nlE!OcUnt*8I56n}I z*&BAwDXj)}c20%xdV_v;UD8}uuN($wo+RhcC`4PEwOs3xTBYVh??2Gk=&kd0u7`)_ zl2;%Lu({-GYUMMgdRU|1qnO6dU(P?LTR%BV8^Atyl1uzoiEHJ1nR{VPJ(4Z-0mm)# z&hl+5Yy&*yyk}S1bmRJli6Y;qTvoC7q8KtXOrrdPjwvjP9Puv3WU7RXwKWO~7#Z7p zmTVV!(wZVf1=yk_d-*Jv7+LxZISC8r>dhAQ#Vt4eQnPb-#Krx(Ed?yU)eWs9dD+!x zFe3kWJHZP`wHWWcOW-a??2q4gyFq{oJw@j&P!pw$FVPlK-p(^B6_pi6!v1v+39H$M@U6R?lz|GR;RrQ4#X5~i=v+)+45j9TIK}3^_g|(id!*+=168h$~Ke-p^p37U@^o4ZoR!=R-pg3()|G_+eSeEh0$dTlysmT@A-hThF1l7GCRe zR9#oQe(;aU2M7Us+*J#@0ntq ziHhX9GG(Bj6+&8*btlm(m@iVhQ_A9u-Az`7se95?bJPm`mS=%FHhN+bTS1{YYHB2R zQ>@A*c+gZz;%4xy3-MJ3NPwLk@J};=DJv>9UELp_wX&@JaaC(PVJ?RPH zj3{l>ESLc3ytS886*YUj)1~}keXe24xVY;nsYfm>@m%h#>Fm1xoy>~3T|g8cg&R&? zSeIowq`4oGA$-35zO#;@#%HZ}N~|Q^F;?vqouZpO?Jq$@4Lx6@0i3r|+-Q--M{5OL zCS6s-s?}ANhBfC_5PC{XHaBqZ`N_^A8^Xu9U`Uos;5Rz5`hE0h-^E1e>s0rQ4V(K+==tH75Cv(Ds6795`tn}DrNPyvn8CH9r=DjRDrlH$KNl9r-_bA& zi+Iu_q#WfxQe5&ii0s&@@Hf1INL2Us(-UD^+xZf4|M(xi3!XI-t*Ro)r*KWzT`t=r znZ&~Rm5NFYHQi@^)EJ;WOa2SNTx&~P+gdGWoaNBt?RwGu%(f;nxh`|mm2Hb?p0scA z3oB+%o~|@mcw)m$HdfJY&Bk2^dLjzFR6(rd9~$F5Qo@R@&^ufbyYxG(b!Cj4&^x9R zJuu`3A`PFPsD#{)PyAYmRDwTBs^CW}*0*&5A@spBpiQSH4_b5TeoIW*iY|&O81K)v zK_uNkgZ|o%TNjp7<8|QSvR=GmjBcJuziIIXQMgl9kGf9hK+gWx;+gu&S!(e>%}dvL zUb9o0;tic7A_(z+FrsyM8VB!o5+;Tm1c4iCUBB2V`=2WR(oZ>v0`kY>s1Sr`c=$Q7 z=$0{(jc~4$2y*algWShg5|$_PL$>~2=6L$+!5#YG?qKW5ZP;r~uC@!Y{CJ217G$4>YF*HA{KUmtw*2edIAxfxJ@ zlvX#l*zV1vMl;>xkoYyO)k-eT>53KSi;@`_}SAL;< zM`$TT5R4+A4Th~Lj%Kl@60Of)(&c3MVe=X>V4B4`E^XY=6Wdv{GOqYnA>BUQpYmHe z-jC$T%wsHXBW&%0;wV9pA@7FG-xO?B3tiu4kFG4K@{TOEgdM4W;vP`j2UX9*42#KN z;3HFg?o^R8HTHWfKI*cU1c=Q)Ze-4saz6`$N(^gtGyauKkPi^>47jy#9_Iw~aX$<| zg=8}+yi0&acIw2snx4wVliEHOw99=5eZ6ioynZwO1V{>}(e++R)NN2aw}?#%-?#iNRaBUiaLyxS&lR86>2NAklbvyl>0P<(@WZGkg*c$EIY@WYWV%B%WlyN2HvG zt8oj(Co~nkg!bG#WuMUT9K#Ij1Vw$=wEgTQ=|778n8yA%#|&_}J|-{gNfQEJY`c1- z|GUyhG@L}k`w;43GnGO%|MmMQQY0R8w85TS^b2GoZ&c(5OG0GXAg)+t z+2-cL%nx*d+KM5QeWhVHavx>e#`Urru7m)+vADGLDT}H>HHA^Wpd4w+IukoY7c7Ac zKv}&SVq;v6BxqrcAW#qo#DO?JF#UCO?VLC`dt#bm^y%CrP#*|FUi${qkzPYHN(FSK z8(oLsOlz1J8s#~z)S8yg`=M(77V1H9X9)pPFbz+NMC&0i*Mee)8MO?yH_?M!gB$e# zdO)Z`nWqB;)|i9O!!>>pS=X9sVVj3ukdJbm@3Db(qx7XPLShp&Ei}m^5gUhBc>&lk z4ZyrJ(Ph-?N^Q)feVN#`P><*>X$*J_JP~S}0MDCP8?y`yh$tB5^7~gSj8vfVrX|E3qo8plU1VDOU(D zX21+I_~{%L#*5tg-&dd^%BY-iY)^7e44|zk4I7LGG=v;wfb_z3_UD@295WOLtw7!I z0}W@kfgqkn9!&7mITMT*gAbAD_Vl(W$g|NC6TEdU1mnfxLk75l2SdX=0bVdfFQJ4z zA@jNdXSU7<0+oe5#jE1$PS( z58>vU2bt)v!C&&Ay81dSu-*9?%oE(7Th!U(taH7K_ICZNc_np zkXR8!@h>5e_#3F`$rl(0X$}b(!7?W|f*CapX51TrY)l4B{Br?nVwh8jkb*Q|bWjhP z2#1eh{J#S15E@hSU=X78{?Wav1D6ZRhLw_(6tx8T#D%?Ro%z&6Hz@M5OMbYt{%}DP zNpt3lNYW0;n<0Zkh=TbS_WBjbOg-xDj|fP^GKY@Yfy3{e_;(n12Q?2bod9_Ar2s$& z+Czm7e$zh=2?VuAqF+x1+C!Q%05%}j&=8`2{)j?{Zp6mGy8XNLN*}C`>RSsZtOcEi z`3oLn+zLk5V1P^!WcOqktUrPp9CO1Czy-<&$6S90(&n*QbM-L+YiUMNyJbN#h7^&g z3?|V@=}`jt@zBrh-Dn67hhkN5J5r#};qceafd9t{;DNkB2o_A-kq1d#>ml4&EADf= zYtR1u{(2n<`WLe^o`*1+gm7c7XiELA{>~Y^IV?jwkk^+d1G0j=u?OZtkGev7kpgCp zDIfM(z${-!e?G}Uc+rU(4m-hz4vN2`y(0k)5l5>aytqU!VRSr$t5^R1XNm@Q=k`Kk z@Z&jwouI%|FkZ+33;;^P1E=FF2yhOJ7hV7lfKvG2>X4Fj|LXYjKn5m*0y6_I;X(%u zUnyRT0)1lukAcEu@L=rMq(G2$NC2U(ADrfq)xRE?C&mwA0OkC?(XJ8Ys2?ldW1qfn zQ~&{}4g*~EYWI0_Oh8BgHpq!iu*hkGW=BP^!$|mv+4(6q_~GSd3>`WJ2mc9Dp{;f^Z}E&H(gR0PXzUfw~b0e$7py{V4)r_6iYmP4<)r@x%cLfCSgU zJP~~e06rkVd4ZtSm=B`izap=_pHe>5qzziEI&+u{tHuV_cCLkQ(cdltf*lX!6(|S; z5{KkNDJDZ3#hg8r2F3+zN~y_JPeo{sKr}*t%(xPLRz4%DwjLBh6m(0;Iy*2Gx)BPL z5=e}W5QWmRw2l}E;#Jmzey)b}TMiVq1Lkq%&k)X9F^%dXXRp}OK#Ez^XgE_RqS35w44lb;7I|u+4-_K`d15y5Wj?dY4wNycnrwBX1FkX;e5CWym zs|~6yV8Q&s`0KO)O8!N9R#RMjHZ>yXP>GrKP)kU$uVxo;yvUX7l%lqQtAX;6E);;c zBdlUH&d<7)GAq4UwN;#eujNAt;IS*Fuh@bbEfQTe`?OI@Rwk|=aADosw%Jmi zF6F|ECW#J|LENZY-*X1BULQO3rH3N0SH+y_*Y}E^)dH*wT z6Mi%e;)xw#1PQJR^hE{Lg@o@wfU^R9kwH$-HzdGI_>&!2uuq^b8HfzxMjYr5Ga3*1 zga{CU1zQEa5`&B&ZsdUeu%mU5PpE)iSgKe83nJhbmLROTnl9UkP0O!nNlRSC;ML*= ztRVS(p#R_f$28^VClj(EAW+DlAprls59Ple6me@48^`||Q6~N1vvG!Q0%oRkvi4_R|z_Gn|U%I=|3S;LV*A`tem z)J~Zo`;x9_(%^?Bhi$fjbuwgT#k@(gHeXSnkCBr)z9(Y3&<@nHs&rr!A}Ff|y4rr{ zpIb{&ptp}XT-lIsTFpuyr3#z+elT*O+!Vk<>m#(KqF;a2G?_}*l)C>Yj3 z#8EQ_P}4!dudk24SJs{hE{+T#$#He=V+j24U;SDg1I{RipZe%+I zP0w79_htS*jZ%~=N#f{S$D?j?4P7E#U}uVI^@_5Oeyi0z-~HzD(h}>i?Uni71!&zl`Q2)95K^5m1=9+hRZ=fRC@Wfv zs|T90boH$!s46U`Hg+Yzx^A<)@z?33i2a|u(%=w;$)soKd9pJD;$*Bd2@4Wd<2#)A6mmc>wKcjS7_FT)<;{>_0=8P1k2 zGVDG0+KE)<=18l;>hm*l`*pZqcv453+sgPWqy_5zz9pd4qtRpKe|>#SWnVKdOv)OA z+=$12V0dOCbGKqvT@|$23iO;fnr3t%Yy+~_D=g>E@)%IL#^N=vJ3b?-TNcsTW^MK+4ioLuOVL%pyTO$M;}FyXpJ{9{dQ=&>b#sFqMq;lB9tWh|5tI z`Hw7v5J<_fG+)%7ELUMYRU5Pp$Jj21lI3tyw1O1K$%%CZ=I~C<%H$xQ3wj6-lk?M| zTE~Zf^@=Htw%%iGIlHq#+GvL|EJUI4VEZI5Q6Xii(3p0fuAQ?Re^{oOjTUlDm|d(X z>qAi-m^$KPT-TP@Ptg~ou?!O!qMjo?kcD07vxLK4G_AG#)@(ehZ$+X+&VS*Qp1AI- zYVIke))TIZoUsk#TInCr{IK`HX6q!N$%Z2$KM%xja&K|xrV%*aRj#ZzeC{Q>-2+Y_ zg*oWtmIzI4L>-YJWvLM<_~d$G&UCTyI@BkzV{Nc4EN5z`+se7uf*$&(?eUnFJSra5 z8F5_W1tHE2AM*QMKEV3b8+lWta1RJRi;H<&%X*rEcy_jIND6L+ul=>t=F`LnyP2j? zl1Z_;&g#OX*-FCRNUI0uC%OlHPZEzh#xo7a-_A`SL*yEsj%2W539)k!^_%GJu_MG& z=JZ0}ER0#|wQ#tb(4bRv9_=6K>sMCKOiwvYFFr9w%IcW04s_?}3MbzP;Kf&H2TqMO zxfc>9JxUY#rvC=~-Zpkx;dNU(&70bWeIWHQs$-`(_ebSE#NqC^5w=qcA;avA$4xDF zLT{;9P>Q;+LL#C&@tfCmFY}t+*wODOO{nqZvHMzR1pN~1r)KvlzwAK1 z*^>GrPNL9`kx{pO8U}gQmiXu!6~3pj3fkDqfn7FYNJSh4ThL2rXjMwI{FYhyEpJrG zl%MI}?t-*Ef!zh^1tm{-4;0*WjLR!XE~X*ClR6a5)$`|}7ufrzl!AN|wdMINUr3i7 zI4^R26nyPcm)Pcjd0A4TCTC`+HK|IV=`mm>g!^^C-fBybK(#vy_+;j3U z!F%Wsm%F~5hhX`&#l1pD%&2NjqbrDADPt$cK-SW;wNZf9ht_8w!{Co9T&cygMuTh^ zQ?-NF`Jxfx&5BqU$)+EJMH#a_6`twrmbPWo`D8#rs=S_-PbPO=P1k~VvPf0*rS5Xk z3rOlwLt8ILz)t6*CFx_{cjNOl@YSfCr+$gE=S=Q5E$1XMxM;+lBae5{$X$*t`I)oW zd9}`ed)J{STbsZcsh8#~D|NGT;p3uR2`*hf7Vd~Z1E7a^@kH`jl#BsGbT$Oeo@w>v$OSOF^vd*bs?f{ zWAOZw;%L6o+gA%{4Mls~`K*QAu**^PQmeNkeoUmelAQB7e*zb7>TiBb`JiQwCFKj22r_x-*Zi!LT^9gC|@}*_?Mlwj`jmLs}xIYV4_6nJDI3YWern`Tp`eIx_cy; zKlVk%DexxK1(CoXIr(EQkPtH*nMn1$`rXW^>FWTe#+-n&U-(y;tB{+x8&Q&-Afo0O zO<|}*vqS7_)7hBSSEx-^n@i;d`Zf7W*RPLZ{kokF?)IfGxN~p1Ca5JVy^@_hjP8|A z0|D^kJmXx&_wdU8#U}`yG`1Umx3kSfgspnXJ{V|47~+%qcSN5;I-ej4J^?^iHbpoa zg%x@`smxhSW^2u*aSeJF7Q|N*wgcoO!{|uv+m!qPlQ>i>yGLQ>8r>ak)gOT+cYz^7d<}Ob6 zP7hu02iFSKb)4@@B^48#gX92OtLlWR>zv_Kuujd&g*@EbE#v>hSjiA-{cA;$Hd^71 zf{!trQJ0cS?`_)hL?mLhRD*FmQ<=Yu(GWdGm}b=dSGNG6QVHN{D`QC8k+2 zt**xee??wu5pI`64#KIOW`yEcz|Zt-rfTWyYawsv#?a=5TcHju(Gs>;qKz&P6-|(S zQP5o4|KlEIhwSWdy54xn?=uA+p2#z-S@SDmudFAeG%(NBvN} zNj#q|%}I2~5!Vdb%B)IuzxJ5m3M;=*p31DscA=b?kSJBv2jDg!f0yM~7!b;Fo++mu zq^lwcn)$jbfaif!%`RP0K4HD3m#b=mul7Y*Hq4H>-LmUQBT3anGtAf%cEz#w=_@IR z*>|3pLZ)A))dHSWO$?PMU&8~fJcPC43fWh3cXfBytF{d!9MSk_ez(jnneJDAr^hbE z{bV7UDvWH#ElZ??kku^>?b^CCRlMwuRC$p(U!2)uBfBH(pJ1%2*O7xbCNP@f0!$IW z=1mgZ$G!;mm5vDD#Z;^oi+eICz64we^MEe$`igOK(1CEIGubApH~{{g1{`Jnxi)?4McINK}^xSVh*|Hv}W14;&V=vf5 zotrx7J50LGVU^hTC{kH!1)j05q;M^tsTY3dYJWfI44g zPB_Mz_IRS~f zu_oY5Rh#B!KOCB*_{74whqnF|Prr_EUdRI0K83I~Tjqpl*IR+s4_W^siH_?+1s%jt z((`5hfw_JA%lbC_3Vd-**|#j-W>d$!aI|l-!f-1QVq;~y&y%1$`_msi3iL>0ak%oH zM|bioTD#Ai_wHMz0!1EL=oaNR&I_t^HCZq_A!m9k1`=;l&TkT1rAp*i^W6%wMbZQo zj-CFnD>M2b?DL$##V$?k1fRVdRRe<0JWYQ6``1s%FKXNy-vSmk6ikEn{v!MogAuxhe%J7W_^*!(yFPq4axfZaKG2-opAN z`8lH0%oLUj|6S$HUEZ`v9n1s2*kRr*<^R|Yh@>cy&sZhb^n`UM)e;pL zeESJ|@256`3xrJQgfG8c5;{|dyp7u1-X)Y;_>G|#qbo;nfkV5aqoyaSP%g7@By}&1 z2cj|PN)@ZP5Ez@Gl~G@>j$QNv8dP8EO%YI)NzR^JG&_Yu?kVf>KM7HdOoa&ei6}4F zwkl4ActV`Jzo-JZ+wm(-oR^9TUmXQ0N}1tX>yW) z=gC}=lJ!N#NM^J4RMAllw(SF>&F;M88mPyvW5ST0t}95lmT0TgtRdeiG}mEnU(mgr zSqFcwdmrAJn2IkpUnSb3(j#NkPZ;&bpO(3zOh-3>3YEEI+wE(Y7;oag&xXZJr#djZ z9?Dnw?~4@ZzZrkvAsWgDKZ15G?#_k1J#dA50&@*0Vwq&{dM7Qb#v=^;xf04FrGS3Q ze(EN!X9A~3On`_HQ9m(1RZ6Ts`99n@$Y*>a!*bb+RtSJcuxl}b5P>W%A(KqEG}fj8 zOA<_{9ca^_{Xv%68M7x>E#4*zr!MF@`ni*?sn(lp3eV6Bd$Dct<77gPB&)0-$|_)E z@gEWr9?3x-?cZMb(~Z*?_kIzT-4Be#_{DhSmhm9+e^D@o8P<68AY$gqWzTWrAjv&s zWf6^D_0_oh{CeQQU#{|<@}1MY!0@S!->`p1(dO~VOM;FXqrNm}03{Eozcds!tgjWQ-Id%@}6`bC}gRB%uDts@Oe#^;4rJVKm=1xzJI49{?hG=Sh-cR*SpARWlfCymX`7b(JFaieLz4pQDo|83pI zRS>IwIVbT$D6FJ2s6JN1tsmc|AI?o|P=_F_f(d8;KJjzC2Yq-!zA)rzNupGj6e`CJ z`zU!ib;{YJh!xrf@tAs9bC-NC$eC?SbB*YcxCJ9n!j$x#5_2I za=uc$4telapzfwY`WfWugi{)%wJq7A$6K|^l4+8jrQ4U|Omp_jh;x9Ij|uJ(%Tv~= zkI-?!e-6dZJ6eYfYm^&??ln!)z|{W~u)C?CGf&p2S2u}4U_9N{m1VC;-JTXKaBQfV zcj4H9N!bLo`O<|xtHsIxEQ~t;)x*GAQcI7>L>PY)zSG8Y{f)($Om<7zZk!V6^y|S6 zdkk)apOo%=)0HxVf84FJG`wM8T4dyt>_a!E#B9dHbSR>stiUk3gEnT9i*R|cNwd_8 zixUHwQ2Xsy`NHY9KOVxv1Jx7ylxh9;#d<_TS6@2|ZTf4i)k=lzp(g0OL*hZuutb&M z93|(DyQ_(BM|))Dw|tQYrb?H#$!%9u=J18;z6U{Xl_qm2*ahF5%F43LW~1Qd#Mi?m zVQcRq5t=P^f$&me_t&PMJJ)ez4nDAUCGbV}QP!{UIMRm`CyJiY)Zd(^Z)%#+&bG!j z_<)LQy{gZz=ViE4$+;7e(x0Q<>3iKA4KHe&P4L9e{{e$Qe7~C}IPuVK#LC}AUE$i= zaAUP_z@6Z}&3#v0*Vbkm=z)8{eXskz!r4#_dhZ5V0c;`5+uQPETD(ZYu01p_;opvM z92YhV{*QqBLHDC|iVK?s-^aoIg!@Uo#x-OZ7&h~uyxF!+`7}kSo(1>Q?q#Iwd2qi# zGk)H8+JrBk*C}VEs?|0X)EdMq;C|8l-@321^A?cOYv6v}{e~WE2+*e0wkB*^ZS!YU zt+shd`_=Z_MEIS$ueM8BMJK#RPI%w4+AdK>nz5mIFV~r9@2Gg*%NyjSHD7!T?ho9b z)OFq~GK1tl1NZ0dFZAR&1BZTAcjSwi?yo8Q!tRPAhj8p$#W&=MZ>zcDs5aB+o$tvz zKhOq#1qJOQgSoS(RU8?llbvI+oaTvN!2P59*Sbaq1_Q|VcX0pV{!@<(IAT3AD2VEj zLH?1#)|OLFr&=LJ; zL_Iwfwx|xia$?dG^7IBzA5UK~Jr{V5D1FW{pNSH0DRyjF&tegI+wuvx7;lr*BZJ5z zgR7Z<9a%sJ3?&B)<8!Wr;ZSMq8(-?%T+!&fKCEy2iS`KatmRp!sw*06FpS<<7d#D~ zQTib|Bhk_JZK617L7sqQUneprtWQoDQ%wR=1$L10SmL}PPd2jUBhNvR*eOM56C$)} zH907i7(zyy6QeD7!iha44_{2vl~&Rw&UYT0>M_3MGb=pXfM+YuwhBvj_wXhxq9e8g z&-R`PBD&T|4KF>oX`j)ux&5u=sG>8SL`)}Fb+MZR14wKKBDNzB4T_-;lGkWH<3qkm zcI0Bp7WYg8Pm^bQHLGNYHjv*;@XVsiK(vF1I(B)wxXfuBYBNEdQ%$WwdpR?~`bph1_{sL1%KOO5i3 z8RB^!c+U2m{}0rtu!zJj1kXjDi}e~6q{`H&mc+D2)zqciQ4)7tM(+5x!d7&PI~<`v z7hORvx>El6T_-xKYx}wr{zFc5d6o$^RVafZmWPP#OF!yJmq;>4_r0m zQvXk>#Q&VYb41|zdikH@eUb3KB>Yb~fKvbST)P^)PX}c^uY%`g&uiW0{U&1b26*12 zo8kh(`y97(A9I**6U=w&pLCm(rxKmd z!1KB13q5!cz@;sGY9-$1_`fFn-_*~ zRp#AD{_F15y!R-u4G`O4{oi{O$N`ZHTi<)+(aX0G^%P5|!IQ10jE{(K9J~>4kNN~p z&Wxf9dV#k$eP>bIWYpO?9if_R$n+yJ{p&MANo^qZfr#BZi02JSB<$k6d?rzS>QBqf zBg(_TJJ`F{KT&^qqv(U-;2q&zM~?*awAL z&73)Ol|KGSRWORGV08DcDN8e^I$59UWK5|wWjU|8J9*-2yhvI!jWJ*Ae8MY*VV?%@nq*yLSz2JVEul1=a7C{93%Kd|B1lYVg50^1(LUFt!>v zjz^AcV#Qkoxt(=@b2Yp`3DwQnn|Rd1n(m6Vts+7@hY>x${z)&W`R{;yAQG5xBd&P0=ae~m$ojj z^n(o2n$4dE*r^^>tDO-bwps9Iyg7v}?(VwH+eH3d;9crnrbp*_YU`Bw(pLUpgZwN@ zY3Cal?@x>msOBt-GZRSgAR>4$KfNL&An&`{`G*=d!J3KQVcR@>36F`vUNu>%H(Fcw;IoBFBrtdx`f_ebZBr zD)YuvJBZS4Z%lP}Tu$z|;{WK4sqUhy$VFFI^~O~5R5fo*btha)PFPw0H>Ntj>xthD z);Fda^m@H9)#=wl zeg28gubD;P+zsA)y!Yx~6fkfs^Wi1q8Ki~XITjRL^Z>c&L4_Use>ET!-1acJ?Gc`L zSiK^=_P*B5%W&Nwy5kM-zUh5SUxsr=C6?iZBWjf{ z!;LK8A(rpfcNuOZ^*)jMz_<)A;2v3AliB3rR1VF~>?+)ATubJ$xNE`t33xyBe%f7` z$4&eIo%eI_e&PL6+{-NIA^)J+yjteECQ19?l)oX$-*(qrCpU{u_@12bgM4O_7IHp7 zOB8C=`!jLc{L*j9Yl6bn;MGT*+CApJJ-|2Lw`X@n zoC`FTdxQFoFJbdNVNO$WR8V#)c=a#7^l{#J&$%xNzJXh3jEkt~;Z!t5&yJ*Lo%HNz zdbZ5YY2r*@J)Bm!VBL2-_>S?N(EU!!n?#l;fqNa_$>p4;-@3kMy$DP9GPx5@BPX1$ zu>JoDC)mJ1$DB!yIg7tEt4@jZEwJHw^Xg*=aTUAoJn)_EJHNYPh)Kc?7lM0B-$n8m zVv>Cfp&iVso};*`-FF%IF82MqyEsk}9IpWPMBkNij{N#|$!@y1BAp=q2Ygp~KP5fa zf_ob6ZOTUsage8ak7zT^4d7epTh(12(Pow#!9CY^liVZXQgtas$;lnx8t~mL2HD%d zy}KA>@1SRVkiCnZEwgJy+NqaX12fBI>c0EHcen5U?hVXLHj(Iq;9lZ;s8nEPMfnJ! zV?ABuXaT570oz7_#Z+JcjzNdZ9S942e)mD-2i{Rd0dC;zF zY1gRxUIE`rqAR`%?nAv_(zDm;*|+rUO?vi&-LRZL!BH2F8LuMuy#v0teDD4P9xGTy zw(o=cRNn`5uc@;#wY*;`)~BroJ|^m)bl3Y8xmo1?8FBwy{(ePTNMSSUNnT_B6|w(X zVf+45>^1t|68-P^Wtk33*$u#2R%|^6V8dnKzMsJNz3=DlAAoh6=#*cZ?)Dhlow>?b9W!&r|MV&PJ7e^2no z{Jpw=g_t*qEc<}_b${P-D@6UK@OrEeOJ5Z3A3#nR*!?TSQZUdlgUK;N%B>JfUnOqa za$K-jiyf#ID}|c<)7@|!RMcX>&#N3;NOw1C8fg} z1*Cy&{?|Cpm9~Ucl+EB`wx@ydB1AbYk;B4km7y#e;Y{#P^UpH9wyKyhYC^)Z!Qafn z;z))8M(+Z+g~>&kY;sB}o1+~d|2*)|@z0{7yMli=DjLC}1|bZb3WZw|>6T<02AXp_ zTCo-6NW<>npD#4*3I4rE16_=7l~ho5nWJbu0w`-dqQ3QCQR{n8_tc zU=jFRg+R*W>eggivNMThNN6AMFA+%Ez~7#CgD#@5UWI~-Qt8%4tnK!+!Zs;WNs@&< z-3R2~5BzC=)-=&oTy0X_$N<}A7hP9GcWfB3hFbU!(-I4lZ6;SO5|vf9!SOD(KlqpV z52&%h@g8;%_z$Mu@8>u<1Vi61+>Af5avvAw3;kYu|6$-i#DBOcQmco{i%j_^dwMHu zqp|~*4CB#+@fg!?Ocmpo4AM}&{+rUR$>lgrnq6&7THB#ohg(v9h|x*lKi+?`DWa*L z*RA%jQJo6@)BL9^EUtZA!N^M%fyp%~Y;%Y8C!u^6Q9j%BWtgffQNr+CVt5|^RE-_@ zzNWf0rdP?c;2pSHa=E_$BJiK@zqpz#_-YJdBVN}oge#@5#ODG0|0V{PR};cjWf5!d zO7JxJuaf73*?%}dp2c6>Tx1-tA&%Eplf~~qKUT_EyAyVX;-&E2K=@WwbHdJv0R%`l z$6GptDUZ&7Gx%@xpG>v02K=|u=+PtpEe^Bb?Id_M3EoM9*a_oi#r(a^gj(jS{Pzs} z_k#Z}|9z%&p7ousgeCOPb_(0Dg4s$(_z)p{*t98F(`2Os+QAfV8NWqPMYzR-mM6gf zsQ<~jhg%fJcxIA;XUIS!{c~PXqu{A?RUoH3<^3;$|2h9lbw@stpzh&hUBywnN+@2d zbzQ}cCaqDPmUp*l_Qmax%WXu285p(5!fu!Kf0qJhiu z{pw1?k#(eYMP(gSN8O)L52#-U)d8(k*ralRoVs=hfgGx^y=wvK)RQ-vw(tyh(N_^) z(dF}j5fE4_uuk3Mt4Rd*PABmdM=^>}jIK4l;zl1-*g%tqrX)LCmP|@zlP#U8Oq#51 z0D<)b424~%Dy&bjaC2&Z+Bpqu41o=W+}YNkO-ppF!s~S9d|(R*Y#P|I?p~il4tUau z*Ex!92*tLwdYv16(K_Spm}zTGEKIefI#-aX2@u$hyOMo}1Gq_2Mm_sP$*6JT_ z^gGj=5cJRVWM=}0rG#AqNYQjIt{BuF=1cPtbUp-j2`s240mP{h|77(5g$*_lkYud} z_JY73{?CZ-FZ9oa^v|#9pMP2h04+zgroDOn?)AW82($*4bmQLqjC^eBbyv1GU%=Q# z7~AW&H(!7R*8L7WkG0$+ZOhpjmS!Q43FK;vD1pIMg`RQYy4x%{N<=*H`PDpOKxt3Ho>bbf6Sf&*UU zq3PBLonLw_{~Wo)@I$-UI&cXDE(}~+&F9D~F@zj2=p9=@l#|DI1>w8Wl=)s$jFX46 zN0Cc;+D+gZ2wcsVMS+zNxQ;gI0{Xqw)`(L}%C9Nn)GwnBtcJk#fg8Iq;?ztb`DA4g zX943H!gy=_BF+M&c55h!g{2W1vN+U6rp&!)&(^;dZKmf zNly+f|1Qwzay_M#@Vx+mX9F+R_AXGa9EYfvA@EAzzxq|J2E4->M#*#*iWhaa{IQC_ z>kxP~kRykjPXBz1{<)m~d5(2MMjWiDX;(jg4RGK+2)rG5zZ-Y;g(akMaX0Mh>oh(l z8lU{byZSn{t18>o*HM2#sK2b=uD%ZF`r3E(b&%f@$nUy$S6>Hyi*+n890i$DYlGKs zvkm+Vfgb|Dn668%f2-HbAuF8z=66TaFA(~J2>q$BrT=vMFOYngf0wGFwyNU~Du_Sm zGF|#sb5AHhd#Zf4U`4kf2fYN>S6jAV1s0LQi$#;biVk`;6AVExzys%C7=jTBob#U$ z;9qhx2iyGYOV#Ta!;Qef9uSNMdv??MMK^@R-*CQuG2-h>`1;jn{bGdkzUe3fxip<> z$+RZ*o%7%z2oB&mgux*Y97;KaJ+-Tvt&(ezBqb3Bhm+)a))x!;!wuy-mO1j&%5gue z3&9b=hT1xojduTJWvD5fNu_hithS&Ft`EV{{2WDa0|<`IpQF%4Ok7jB3Hf6tRCHqq zZsg4+xAlOQ-be`rY)Fmv}cktea_9X|(xvxVd5L2#G+Q5vhHDHB%w zvdLI|^C7sa(6T!O_aH4jbS+JVN;E)ECzo^RdqZ$f0ev9^Tk_E9w@Zi$An4TAd!lxYZN2xXXGtDToIS6HZm`K^U7U_~-ly_Ch{8xE4!j6fXF~9_;92$FgOJ$5 z>Ny93=LXLcRcj(s`rd`S3I{tFa*h`g$BSy*!H`qB1SzQ}nOc~}%y_1z@&hH}iR@zQ z;N=j!GupvjN~dmY$CO#^|EV;-O73&HnFvc^sXm*gX_ja zT*Vb!O+0S=M|eyXJZ>f)BODxqDbcoWCDgaocnqdQQ@R5&s^>X43sb6z+)W_wsqrjK zsTOe`p}eeGlv)FLkWfC<9VoT>euPk7V~Tl1?=&7|tF}=~ZwL=Q0l`OuPj*kEmRLlR zPebq-x*5EuKK2;dIjkT#ndgbj3w5a=Irf(jyLyx9kV2k<>BLm^6G#5k6DTqi3JcvwP^-XY}kc zdiDi9dx)NWMbEA=#aSAJw#-5j{T6~>3z*+S@CSk!F2K}eej*tT zZzaPY0>bsAX(6apQvQTk<;RG`?dzc!gunH+6d2WsHf zA#fvWbfAW%0kNnbOO`O&VYYHRH^n5i%i}}qLugcJOl>Rbw*uQ+|~v&3z4b6J4m6kX#~v#ZBA`D0#e|lVavzGX0ff$j(AlB$>+QoS0`-Lux`@64PzVV%xMf|ap}UmOT~?zD zH87VW7l^UJaouLjS3^1GSm)S_rMA&mVy9Kn}L7`!sAf5Vln{ zx=(|0BSN9?dlrqK;xJ$3{?jJ!H4wTfbZfo+r_J8CL+B3r=t5zP6llvjPy=^2fxD+h z2WnXELo9CkvZvL7lEZSL2O)I7h$9$`E|^7JY*^b|eg^YkMWm!5Iwy-&}a$M06IACaPu>-NUXNbOTZ=H_TWCnLOCzr<&v zTEC`eyjs5{xsYTROF#EFxyp()Wz}%HKSJoc&`%0WbVJoztRb^sAoMF;4PMyi(kc1U z`Jc)SLrr#E{v4nQ(bOCr1c;m4Yqn(>1l5CaH9yc?TZ!5R|l3DAo^^ihcY@{B6a7%jFT zCTHkNl={^>V^nSqISn9ApI6pHOjR(L@D8b84>5Us7@~8Z<{;nG)}6x%-iR9WJ#D== zlJK1;2_E_Rt;!SU3xhlzV-gHT1FQ?{)jM&%$P%&{1F!*%HHMX(hW*vwh2jqK(q-H> zCT^S5n3pc2vKdlwuas2`&*y?I0XFCPT(C93xO~!}E+UD4w6kEf#6N})v%q!$+rakS z7yszCkm5vuM*1L2#7L3IR1nV?u}>lFJJct|qKI>1zz z(LEc$W@AWe7QoK*`Io}Xk>}%ZbF&Ek9D+Z$E}NT0Xm>%h?)xNtKs!)Y`JzsIb_Nyz z>sVsc<266x$(Aj-Em3dQ0Xeq!l`rJ(+mK01X z*r_xC-=BaVP`8~*1I~jGr~3_49&-V%c4VmX`^(Ly3}20bLjewk!|Hv1xs5&KcLczZ z^tG5Crh3qbmA#bQp2so7N=VM?CYKIlQlPXaiBr!o7tRoE!Hl4xNf zm&{*Jv{$07ZB{nZl5FkDCUXh3ILo-$=L(x(;2p8Vy%`^3R}k!8ilPZ^Vu9yOD39=Z5RgAzsL<20bBrQ5dKR5 zE+ts~1uTg?2J^+5yE-$CnfCTn=afXM4O#GW61fR&ZF=2O`aWC^aG9`kCBRi=XI--6 zpfon*BCF(^Ny$a2bc!<98xvhvXihef&Lwy%yHH7HuK~DPm|Y2Q9hq&QFLgL6j+12> zyG@JJnQXGPDQzgr$*uypURb>m;3l##F^fzxH$AZ&14hd3Z%Vhcb+z(jIqtA~ z0PYeFyAR-ga@b(fY+FWVVR15@%+fK3osugiv@J`l$jwf6c4gE2Yy><6@PM%J2*9Ix z3-)CcHbi2fb3!K$$qT!97R=$!D#6Kqn4>*2<6=6n1g#)!V&T(fQ8+ zJPXh1x6&Hjxo7DQan@=bgOyy%l4N^gR-zM~lE#>=b45pzZp77}^h)IMm>#bqYY7u^ zM&rw$WV$o8D3#1AEZ3b4qG`Tu1Nbk%OYo|~PX51Y0Cw20!e0k?1Kw0vU+o(S#a^3H zWxd-NRhUPctUS4v3>|Ovr@Qi#>Ihj)HsarHnZ?b?js%Vc8HBWz!$6`K6jjIj0PnyD zrkf0^jw~b=(5W8*d`x#c_tEZlE+R0r1kX9iEWKcY&j3Dw4@lh?0AG^2xJez!DQr*) zP1tG2RphDSru!|xSMZ&}cBz}MiksjM06)4e6{|}V_y#GrwiLF_ru3o=24)?}Y-fsp zeGGm9_zAuveZK+xPWlFD`XmMHS{nbP(kvvAYD+E_djA6W1D-;?VFqD^#>RZDT7WLzhZ715iGR)!Q%r;okf#=>b?LD7 zATJKuzcqR3Af!J_?32mPOJ;Mt*EwC^(rwS-wIMt-JiI>Z+a{{1bs#*Fu8z=#JsXH6 z*`(6CC_x+r;dR5qN%wjXUSFo$P(Zeb zJUM4A&4_(DTxUux@wzou6d9NR;q8QhMhH(L1N}4uMdGFPT55F%2u~Jzn;<;3Sg)y0 zr8A8X%va@f2=64kJrlyS44EA*#J8N2%qqxm@EFB0YFYnAI zs4?idR6PD!2;sd&mA67TX&4A3^0M6*`g%cETH6-VqpBqkUL>NbeIUFqMOD3pR*Nh) zutnMvQ-C$NlpSs%1L5}Y0Xnop0O1t^=0OlX*w6rsf;N2lr$s4l?&g*AeA|_C_%H|`5rk{-g_W?uMJ*J-X<_IN{ZJfr1${{-zOA51mTAbfl;xz13%2s-r$5EgYY9l?Gq4w z(#8+kVnezd=5qOkGauETf$&p8<#P~zo>cZSsMJME_*MVfSfPcz1mPEi(pMn-UyENY z^7cr1g%*^)4&hgY(l;UemKmu*)P9W6-@ci_z60U^3BB(@_zd`tSQrWXuWzHVuoctzL0p(v1{)4NGFo-B-looOO{v=*= z8oFu34H1`6>xGEV+@H+S_FCuWlIGSQ2|~m#v;suJX02vvn>lhuS9>y>YB2_hkr+fG zJfw^CfJjfnj<8kOW=D8d7H9hG3N4t6^npk(9?V7hL8LzgbJ~uuLCyx9-X{t{a%2!h z2Jj#`G6W(+4Lh4wVeGUW#y+Ko$FcNpYsRy`=e$0+SS)yR$znIby#REX?E zowK>&5eSX%^trH;LPSgWHx^_5t#>(UFg+B*r+6D zN>#0O4P&>Kip)n8kp-qN+H~hy%5L2h*#mtL*;8R#J6a{#tcax3P;)YoZCN4;PR_L? zL}N%mWG_*LEf8s?D%7T-VsV=d1|eb^*%w13DU_xlvJWZkoxexkBx~37yE>9tFyZ#ST=c4=A#x1$s+d8U#&v+L z9{lAI-MOo=pn|S>0z{6DoT#u%{;#$<9D_w~pA3;xBBzS1euFdjs&doAscFg^5OpdI z>L`VA5*>Fow%{`%a=K{2XG7#1YQb7iS0ruoBAr?h?dg!R^C5DsXm1xn%0&S zEHWK)C(X%q^A6gACflZU8AL9Q{QLiK2h9!}`tb^gTp77azwBH%s@lyn8#`F#(n^^< zjl6RWM6MPMWF>gtGTqlaGhRBUnL~90lA)B?awi{-MKyQJ_O=2{> z6(YCMXsE4?&7!j5Rb0Vs%;_VAcS7WLxQ!Iv4Uv2D`G3W-vd}b@(>A{*(5Fzce5G|i zMDFD|d65Sp@(|_Z_2f5RnJwh&;^oDC^6}v}IZ-D<&syL2k@+ z@Ip^QS#)(X^tSlCwgrZq(UAY;VG$k!Jb5P8RlwMbm% zRkPrZsl|Mm`#wb86-qyZ$VYjv7R$;Su~}*wKkXIy6e1rBg`Y#@3sTstNZ~X(^;Nu% z%5*NlhCWG8Po0;_rNo|Dr zE-GNJTeLun?%yHutI+)?ME)}CE-E0K$ui5*GKWMJi2ThJM^%WrNiof2GHDyFGFL=> z5cLSH0f+_-S|xsvYLz)58ir^{XpKTNX3#3}KyjLI4!d_UhfN0OCVXCa`9))9v?oO4 zytzetL$psmP1wGS!Zs>rfxlF1wbhJI;8cp9z;h8J7XZaftio2j6EM+ zTFPf5h>jI@Hi77-WP-VQ~KJmB9Y)wyIsYpV!R$WjwTcMz6#g6K4|w6@Wb zyr8u9r9~!XGaUtEGa))%7~2`5v-R39Q&KjD<;zS=bha$f#wW5c7oyF=!Y&Zql`Ql% zSTIT}Y!h3p`Bc`XG(Dc4NT)iw^bDft0*LM=Z0rHiJ;}y81{?Neomu2>R|>~jII9Fi z_YxLcAlhng(OOnvTR5<&uiq_J7eh2DtfnBk4_O^0wJIxLs)ZO!<~OVp7;T$!pZ>< zJurV5OfE0mDlJH+ag|3=X7mt<9wbyA3em$zWe<}|E~v244i7Ym6HU=0QET)l)8!pi zpJ)Olq9z@R(<}ZNgg=Bt9gZYZoDQv5fxa`Vl%fIL)5WPr1xeTKJCMfG!pp?>3 zYV4VmTEsJv#U@nVr&=->cf*wsyOA_f8{;qyXq9y-YGWrTcUlNvHh3ISf!@W*Q zm*O-nwQo|qH)(96Bf4biO^CiOEd39nZ(BK8OUTkb3bO4fq-}1T??Ln(q5cDien{#E znbcc~l+r&P2fy~Nc5~PI1fm}a{hvYfbF2QMBBe$MyA|J85dA`!_y(fiT1N=^n$JEe zDZT3Z0ixdtTR%bcXKPeaYTakwTc&rlb*4Jnk~2G!X|c`9P5lPZUxcYYAo{1Zw@3<= z8u#+zrX%Vx2GPH`iI@vws&(8eE>p60G-=DU8S_HSEo}H97AUG6Nh#S5QC^;=k=9uW zY%EDF-#G>l3km}fh(*c30D}Qbfl@xwHY>%%mETj2^?+DRsP6@_-c}zKmnpT{qskJA zU%3e_bnZt#wjb*Uu|C4g0Ei8=uJ&vSDr{rtwq|GA+7>2S_9bgWAT~%?8wRnp$l8cf z*78M5^&2WwNNcjv8v(Jkg~5>!Ti4oe?8}xKqh+OxBu7K6LD*a$Vq>ghbg9y%#%L|b zFwQ%%4I#FHu(UD6HnEP;))KP4WnI=dIL9`J*rr17mJr)2zqed0TMF;2Opb@7$yTwk zNqV<|*w#Ytc!+Ih#cM6$OuJaTllF-a+g@m&1hL8X+C_;{V~RfV;Hs&Gf26a8X5ZM3 z5StP`iY!cp*iN*DF^(y+QVI*T82{zKra2KQHWOmg`5tU+XNb+FJy@wtL#a~xR=T{F zU=y1Qv1Xxu7l`dj>ifymi&dF@`)N)rO?F_D5aas-h>87>*d7qu)7pLtvZclmEu%)d z;Sx(gOl*I|S|HYH9Y<`7IYCN@4| z8Hjb{FH~_y=)sqvV{nQOwI+qGW5+_e7fLvY(DbHkH%r|7?hzDU6&8vGa;ro4jDDILcUL^72$pj71kh z?0jM7Vu)R0jiZXolx$xH=_bMA-w?Z0sJ{YYR~EG|a{;HG+PNzs0(hQJ8T${!t`at` zh1kkco-!0H^_IbIlO=Wo#I6&TRzvJY>stn!y%qcZFFQHQ->#6YTOf9mn0ao6*ljfP z82i6%(NcT=)-}ML5W8K_xf^2lSoi+Tv43%%T3&8$TAOw5hnUEsh&>3ghw`5?H;Ks- zGE4#!c=|%@QHY5|irC{2dm^8ZVU?84xLx|IS3|-`>}iNSDV+H%#GcD1j1g}HdV&InJ)THhWz5QV!iwhgxHr5`z-cVH5Z>rHKO0X zf!Mc(Q$|*JYnLW5J=2=Bd)FbWJ5J(9hqwQH$<*Bn`}vGM@Zv- zh45ksYLMnTvb6Cp zYZbN`|B8Vr%-fb&AY{@ZWAwDKPg5Yli@r?k2Fk~x z_~sDbB)&y81FBRbQrHUOTho9VH&GDSMwWm-DV6JJORNxwCgr_E)(kj{@em&u->#Zo zB3FqtCO~{5b?&}K8YYaT>S9`AA&)-elOf(Hs&)s6?`Ujq1v%pfP>WUN8_7-(ZxWiO zLwp8liWxPD_EAcaHq6I&hWO0*cBH5o;&X_jNs+b%Q$|r9)h-a9D^TqQ@%eYncB~5rW~o3D;w=K`Vu&v>xUV2)2XapS=qc&h z7vd?Qryb&H(i1ftYtZhlDOQupP3Y>(G^Jaze5qbZ3+MAd7UG$Bu3jyCKH=|z_)?k& zdhmI`0#8bTUi~yD)3m|4lsjVu#Fxc$r0f8QA6QUk5;T4$-l(jw;Tb;!;s*&mheG@? z(j(^7Mtwt5GIFGH+ICH{ZN)CxRA+vIJ`&=G3uQ+`{Fr<;sZ81inQ^v19^%J}cGky6 z+K{&%KN;dD3e%^8GKNe`S`^N?{2W={#2FAjO%yr{;%8H#sF;~`DQR!WPu-;AJcyqo zR1C2}X>`y<5Whf3Ujp$<3*IicrAOgTfW2D8kEWab^yVe5S+YHD(bbu18(X+U(f<0t z_!SVpEPkcJIxB47^#KsSI)05W{vT>7asv>*Hoj7vJJJ2Io$a|3XI2&FZ`IT*tiLTa zg{fUxoFm*ReqyMFP}K-}5X*TrNn5CcVhGQ46M$tveuoYy3`yjjwn}GOI$ccSHQ1_`PE8kN|Qw zpd2=AIqLkLs{HF61zVD1W1Po>5Wg?}P&HrgD5D#RJp%DZJzt1;PKH_j^^Q_{n*``5 z2=tTHe7&Qbazy)upMWZ>9f7|@z+c90ECIfN zBbR?1tfX$ccJnGxd99j{gH>)0xxGQ$-sE>iIxB<=5Dc&AD$CZ=9mi%1##&aR{SL(6 zioaXc7K}`zx>NjpWJW_(usPF}ZAsGh97bmbZ5t3fwid&okTLw<#XQW5|MKC#T+EOE z2Uw8*9HPHnm*DS!h4Ie_KD`x7DERm6(YLbb=sQ__w_91yTUf8r>Mg7f{n3yAaU&aW z5sQt!nf1Dj4ZWSM<6^tq$ksbI|7fguG@AZ(JKI!YSa>x@vnBqy0a|8k3UjmVS(Hs= zz1SqypY6zouqHNw?Zh@>)7chmCO+L6|C`OWLTM1F*w$VydTZETtMQM8R#=^=ybED1!}?v0&$D@*ezq z5ZT|_-QWEP3-@>zd!wjUS& z=@9lW)Z@j5PWapfe7yy}L$SiG_}b-FboeU#<0vbzTM@=>=%hQ)8TV9KHOH~zoq{-l zK%8s^@enG17(qOWCZ4Pk#Oc_=9D+ELK%8R*@jNPj0YSWkCSI))#0Bg^rywpO5SLm( z{127CjUe7d6CYLy;tF=9QxI1Xh-<7MK1bzWAc(Kf#CKJLxZZKc88mneyI~do>qaY< zU$OE3h8+i6Job%p9p@Hp8XCmb;7_GG&K215t@tiNc^bBcaeSWso<~@;zlZSkcCTT# zto6VbFdgeK4J-Y05B}uFG6Ohj zPXTpbf%;yY#VZ+(?ia(+Ll&}I=F)2=&>@x(c|n1r%mt4mH5XqD=ol= zGmkQY1(cC2ri`w#>Ak>SbPD1nYXASWdS^ofu@QpU1VL<3C5SiJn@&NzMIhd>g4h;8 zj7JdLBZx^=gZQ8b#0s03>%&#_=OOd%}WPC9#32acPy}&gl^_C+J)s226JYhk@d)As1aT6A zIIT($Q6+|D3LyCUQHhrV(}O(H+v<^X5X`v<=6nQmag|{DD+8Q*WFUbUVg+#pg18bv zT#X=BRtaKl$3wX?oT_CVYqi{nAZ|hsw;+hys|3;D9KyS%KR2<$cnV=WjWC|YQS`-fqbSZf!BV5BvZeE3UD=9;b!D8jg}jD< zUPnN0BA|E50vca>SXZ{gS;{G(2^`QQE1-`M(8mbqQv~#79RL+?Hk@S>-K|eSozk>Q z`(c_D*$)^i{fM#B&uI7ea#5XzEFa&AsBTT^s1DK2EJC{yhaeh+MsT#={@zDeyuVkV z?d@5k%vz=Y*ldYMT`uNzsVwC3u%0e2>y3Z+cLh1}9feLE`3N2P2p#zd9r*|iIsHVD zDQ&F{;T*gL4*dm(c{nvYZ34UUCa}QbXjhzhTs>I8)r-Yl{c3Ik^OU_R36+(-d8n+k zSWyi{XAVPWu8ocySuRx8oj6EYgyBJnP#N(pagJ~B8YQ)g|J7#2HwN)-fcQ2gNV8vi)c3Tr9CT86Mzu&8T)gmy6N<@#5Z(ZgvKcp8i!O`Si98K=W_Wl5lCXcXz zuE)x54}GqW*$?4QrN-=sF=TyrRFh56ZwR4;n$SA|q=SHfgeIXQNRtlIr6?dBM0$@% zRX|GUy@NCTtAKG}12=b730&Fn0`@6R=sb`9X$$)r4tPxb8GTLE?A2ocyg4qoJ<(xOdoI zjLhecq>g$&kx{vp4usaErYBogr0%Lve4)R3RFkE zoIiNlQD$}TDwUjW(X3qKYpyZ&yZ_bZr7LJE=S-I%wW>PjJK(A%EdHF>RkhP3lvLTO z7xLpLp)(w4MjIN(N4z)kfcaQx{ct6aj&>EG{m~y&l(_i}d*D|GX9=IsO#nDYi1R8I zAfXy!mWsP|(gsusxv$OwxbbDPt)NUaUm}vOdTtb<7-tsPBB;Qepwp~cN(tF{9?=@k zR9=x#QBzNySdmlF*(*FC4(Nr=Sn*e8L8iqb#KV=R(g~~5&z^36zbeKbY;tXlk|e07 z38jC9SI-!ho?zvpD(nwS1BJaPqZm@#w3_A=E~vDk;Yi~ov}Xl{*HvW53twTaW`n2j zE;LQVP}rQ%o;S=S-pUY`77iI=GYFgQl1!@u*>mux^+JcBgd04d@Z<6~GGUT+>cW|= z4?U%_+a6jz4t`}f`&v(18D7GM60J4L((_FCIoasK>0~o=o^o9SHq@sTt$70`F8f%w zu#<9fgitl;&IYbz>VL})b2?GKoryOLm|^SKg)Z`;eLeBczwhzFITZ*9HFPH(1Mw7AQ`Skk^ z%4ehx3ta2y0~3a~Hl=)e4UHU>=W8{*de;pDg4sGcO$MrJ7hY*Iw^>*0hQwU^<{mAw z(FDA*vhbxlR)$)RU?1sy7A?IWrK%$#^K<-m{wBR&HUw`j+;~W~^82gT?Yp27nAWqm z!6=(Ej2fu!ei&UOtWO`j;AE^Zzi-e`JA#uy` zby1{tr#92xa$nNCWuHg7^Yp+;teLu34s5?`KB-n!udy&skgCb(AK=$EUPeW@X{o?V z@{KT+QS{U*j1Z`&!_Ij*^+k#!Y-R2BEMYtkIxMB)&S4@Ff zc#^Gn1CNWT(Sn3Fq+y>)<>w4IkFmTTp=<8i7~LHgRWs*WR*LW}j2ZWADi5 zKtL9;{T()8L2Qst6Zz{1Sda>&lN~8n0XlB;gakv7odPf!zSGxp0!zoA(zuP82@Tvk z{EAy0PoH?3`7p_dL_k&PWT7fTG=G2F2Fl=mCtY7}dS#Ha4n1oEnz4q)$y?b%e?$>F>w%j5OssCywS!*X z5rvR0b>W+Yy2N$$vDLajf4l}ggobMJ5j#85TlG#h#)L5E3HstwrAi|Pv#gya8;vG% zc8%-y2imi_XioOfZ{8HwS!A13b~-fS`~ool0tm*Vb0&3n92@rSzEo;tSpa{%p@zEz z9yE|vc5UQM_{f*+M`>jyX#JwhYVtKHiQ`*sh~on^i4ZiEQ5*9EHP~yv5;}u{PKs9B z8KEw^^8It~aew_J5_s3;L0Cx~DyVFA#Zikp2>jLbq@T$$w7+S!i_19lhB%Ztri)C~ ziZ1C*yu4L^3f{xA*nX}GJ*I>0a_P!Gxnfi6tk_yDdE=70(oL@jpN9|cJlAnMq!hDE zJi2n$nT9Z+;#r~%1&}8+tOeBlEXJxDA#8+IQQ4p&57H-#GIUE;J}og9M>X@6Zq>?g zekkUQa1(#J*OS9=b{HQX#W%71Teu4>Ji}T^(4z|-3Gfc>K`FS4VdXb!>^5zw)I3XG=W>#X}j*i%qX2F<1)6uT{l zS;#|bjKJ(w_NcI}+?@fFidAON@%^yZtnvZRIFa|g%k&_F1C&>0vd7e6=M3@z?_fdL z4Zi$Nx0XDU0*tIa>>OAA`Yr6%UZlp96In1>z|8%EgrQzV)*fs2UTb_j!%v@Kv|)2i zJ3CP^c6B9xIQmPYBe2XmKSvA0DyXNet8K^GBs4WKIlpK=mXLSzyPDs@M zcQ^n0aGX9FdFcUunItb+nJ7*27n>C7V=AR~T=rKz4@Ibdp-G%6>u)Vx{^FP`kNv?3fHiOU zyw6+!k1M#27shyjE`f7$ritnza4A7cc**U&q1t?W~ENm+mx`z%mdF`wJgo007{I#+r#L3YE)H@ z?S3^?^?YM`_mm7a?tqY*MB;v!7qB1GaMDy3Yv*b#@r8rym|=ajQnmr;OC zBR8!uPd?NH2;~o)Jg{Hp`;lpa_d8T^qPZ3m=1o>^b-&ySr`#$JLA|$J9fm7Ua4`|5 zNiLPl70i^kVh1TvxW*kfkM}ZSo1bDg2pJS^^wI`pQF&yaNRH@iO2KD(Tc8+Cw_9LI zPkk-X)EoJ!3+++myq90rDyAzYTW~R1ZU<4G|%;QVBhN zBp6D!?1)nm)Fud(5XM5__UPCXV53uS);@(Jdi&P+ zg58Jl)ono4k0V2#_Mh#9?F)>yDM^ZBA9Y(5T}F?}h24UGYv%>`^c;;uWReOjIFA<_ z9Ws*mgx?8NtH;ryJmToKJQM91Mbr0(?0r7z)Dx33_T&#+B$+;ZJ@pc8MO$H~m^MN$ zpLPBsK1!4Z1HWEQyNAgR7kW! za;&h;bv|CC{U*aQ>azfPJ?Rym(o_0vzIMrC9XvmSi8#?UN;+v3_Q~rP^qqw|yH2&! z5seSWtHNOnF!M->{!grt-wdzqd2D#frxod9=wlo{RBJPzKG{#4x|ks49A8Tq*T3dM zYqGtH_cDXMZ%%r}nCCTk!);z**}6_jeO>|6Zq2jFxYkXV`qev&gVA#H>62R^#jq$~ zcFGNo%DaOPwCzw9*ENFF4@--LnLkj`MzkE=e^U6!j*d>{@ zm~N{P($QFUl-^`uzxXo)^#T^l63=^dEr>o;Ni|;)n@mAf4`<7FwXl!^R zlJ$%uiiF=aP&u>+a@Ggy$?Y=Ua|jGyt8`679g`L=ZeR-!x~4g$aqIJ*2Bw_IYIX{A zC`txI6O|^%oRb;o@J>bW*}Z*PBp@z8=}~-%8Z4{lLNzW`;yReaEu-s`54PViquznvIKGxf7FNr9rxTd$ihFD<1!owflEs&MJv zSv<^vTZd3`$<9!fb@bAObBWN%{NyY#X2I=GWIZbicbQ2*jK)YVXJ7Q1cWv&_lO-RM z=SCLp2}0%cO5$ULB2Q!(y8NCT5&eC=L4CmcGK+B6s5U5$#rMIRxffgR4uof)RV8cO zM(8j^MgEwN2Xgu{j4P6m5yjTiqyvS*He;T7Y*FHDk!RFT{+imiN|aIm&&*M)`Erh% z$9c_QABz;7myT*v*j_j-(pbD|vQm20jhqXEw%~Di5UZ@wBtb9Wy_8d4IWY^fqMu^N z!bG?mZT`PLl7F*g3&tE*0^ScersmvxfUmdbZr^~iG92XH_=K`=5pW2-Jyhhl^LszA zB`lTx^MjHKw(Rt#?rVpUtt1O}v8eZ|4+TChw*W`PH|lr)X9{5Pvoruk+@V~ojp&r(lk~zJFtVTyC1gUEmybaa$>Qs->|cA>fZB@5ts^o zL$o@$W8~8Jsn26h(AeTtw-HYj9LOKqrOsCSsnx8uvX0M}*zki{Ek2$Bekl=R3h>kz zMsqBb zs8Ik>2I(kM$Og{uF9#8qbT_p0`s!PHl6iSl$wWE~h(R3FBI!J+%ANR8G6e%Kw^&rz zG)@CcZXCd6Dr=N*6@C!WHwX??mJbFr;O3eGcm=lk6Gxw~QXNDTe+O%p>Dz!wGs6w7 zvYz_ZOwstrVk(7>bw~_ApCtEbE9K9ZbKkN?tFiPiRQfo%F#rQ#X~sdsA4YOFZZ91c z!5d|K?(RmMa$FtR5UM{PAZm`CW5M|vym_l9j@wjt;<%-f2tk~eRnmE)DtD3sMG+Xm zq@Y_xnsUMefR1X26z-&^uYyDDZ5uxir3L^fgl~BkW0X%rLRIWo{EO=n&`aj zPvs!|QAkJ>j@Z-xp^~qOe&ZlwAQ*ie!UYsF@m0`{)%K%&Ai~S02f@1`&rJsm10Ayu zBC^wl+QI1pPZhWr@dWW5!H7YCX|)8~hl#ZntpP!9Dg}8bfGP3l2*71B6N0dw14j;m zm32oShBV>+mPWLBaZ@?PtZMOCXz{!aBLjFra(e;H6InWy+ngKCwI#eYBv&ea&t81z zPT<$X(_6n=7oDdk`GB!W^J3{O)VN+%*R$43x+DAG@yKjZ@A8iNre-daS12{E{WZc5 zoYO9rvKVnqv*ruy91;3x#AT2;$p68fTWHWcWqqx@QZ1&dFKsL}+{8tG$W~M?y1X)3 zGdabe01cbfS7zbQEdRjcqGkW7C2e@a+9I=ul6W>QMVz|ZX?DN~KY8Zl+wN08lC`?T ze#Pn|ElVE{QsvRE-%`HXqdEs*h6#3_5Oy9UJ5K~VPjMT!qtyz^Ls9nO$BGj{qBr!p zp8zQYrSA{Qon8LXAG)kZ{M6MU4Y9y2r9qr|aZmn4ORh&bOAUg@cqPniLoxu9RJp0G zlsne{b(dX_*uvKOVGdcqE0slLd2>(pq9u_^4{z4~qX&k(0sdw~1bT6c{rHcbqi&>9 zoVTWcen^sn)^xgFaK(peBoQKhmmn}U(`3)6>dkny~(5{26>RJP? zyc=iD4(!iM&g>Tl=nq|}teu7bsApxZL9t~EzjNj@>rToH*);n7hxU!qY5fOd=GGNY zZuP=mQa246S^Jmqg{AK$4cRBCSRIel_58{F>_DhVuugRKZrnn_$`5KB1Ts|fjog=D z-h962x@tp9oQrh(Fm=KOR}P*2^$f<$`qJe z2>beL@%yb`rGTeGs>#yq>#du==LXaMp&gm2$Af|8uEaOeqs`$RvcLCp4*`L+TZl(n!5WRU@COT- z!kLcWbGj~hD-{z@bP8oayyS}CIg(Dc5WG%)!N50nDQn4RJf!svqXpORvUd9A8K(vLf*l!L@o=PB=sQI|h>w29trs z1dwSN9>z_sWaUxZxGib}*$``)U*=6;Syp=+;`NbZ-ayS){0=&BC&N8?ds?D?7+IuN z6P~_0@gV4N{*Jd!P4pX%$;{kayd9UHlRJlz>8ELy6yU!`FS;Nw@5XqHKI z&}2kU1_hCLUhDV7mPD(4nm>~7o^I;G6T1C?m=A8*p?cPuV9iD|wa-599H#G^lTHMI zrw3~-l&PO7QqL(+&-t&q7%jP{cP87s-yNxJQ zkUy;LxKxGfCHDj-m#r+Kqm`wp9+v2of~R~9mZ&P`%O5#-@1P0iLZP2I7%+5j%B3e1 zZurJ_gj!>KrAa1u;$`AOl^0fmT|G>Ax_HVucwa%~A^s_Bo-XxwWb%9kww~sGUe2!& zUyIo8D>NN^{v6f78sZLvT&s+-~4H{=%HiK{}_N!@GmnK^iXI zH4DX5yZKPM;lH+I!rZ@}p+0$=>$E|iCrvLZ^(f>_(b!)kp+JJT+aaxhJihuC-b!WY zG5BX|#lnv8GJAGw>N0}-!BvEX{vG{q#aveW8@H`!c{$+a*l2K`CTp(imeZfbW3wui zM}MXU5T+cIupGjRdy@k*rAe+j97Oz2o?8y?RFHJ@8V=^1qjGdN1>g?7lC&56|W{;vN6^*`6n z&oJX!;M7lHN&~xi2bo6h}$(bW;(|+I<1Slnm{X*oZhYO~yxz}Fdu5Tq64ts?o(Ej0#9X-bF;k10E6+%Rrt@maRH>~d_tB-l zc!hU1ayTVeMXweSdKF=m9ei-juXNthB0D}d|8;n)+pOn2S=EZ2kz4JBijk?T!S)9Q z{AzHK6CazkebhUzO}6>NqyzT6W13AqMV+g|x5W+Lbz)p!u$`tfR9eIn!rb?Su}5ahg*m-^KJ)`o~$T> zWf_4{OOWvw>@~(B$lp#h1#>kf0+W7B|3H|qQ!&)$lD_e}CY$yYz3w~BJsIn|s}rxM zo7C`{9zSLbquT;jrKLzjid66T^{h0VyN(ua46S>P?@yo-mI6pN^=Ni%hISv+I(~f2 zqdhg{zZ45;^Jq}Ae1AMa7%Ez0)q3$@LQt*h!dejEL^S>tITs0S(Xdj& zbiPR_sx3n5v?>Q~r{>zk4w%y=wF$HPS}Ps&Am=!R*_)FMEx+_X?K}CDBYG7VCSXUq zs)~0qT2_HC+G=5IXe7AdGj%nxBXKv&!-HG=SvRrhebIwMQMpvU@%oz7o#ZjPSS~9u zJC=_uJ^i+;ij$z#1i5Mfj)L?tx$p791{qT(V zmDdB~@7rP<{bEVNzq2DN2!s!DPjQ4lLdJ|soTQ3x)jdjK1Y-m{Z0Y#y7kuE4)o1e$FKQ-vy@T0}4(_BN+$%aU*>~Ta@ zIf^Oeit8h`)kPL@>H%qyT>?ORdf^ugBJ+Z`zdrHG*gE=IzL{Q7kg)fx0&z2=SlwUW zQ5_u}RNcnr6(xMVa{)za)#OZWlVb1AT5`sB;D4|CV@1-6FI;fW6=wV>CA+<0R9Ws> zTh|ZLK8r#RgfEuU``LeadHlo2)Fdh|s$5$@7GUczBv%*b&GHz{on~zc#ifFlBNLuE zXo~`21Zd5Z&co6p-BT#uFCuE6fN*bjgMH4v|GMO)*B zV)B1~c4?)|uuoqncpC7CY35PA-g<(ce#*P_IkusVKM!i8A6zYKrSfqpegz82<1Yda zXs)r&>epnNgf0q*F$K_nTe2bdPNw5~hD&PE3APa{=Zokf;;79htbE?prE*+7YkwiX zMEC5z(GPWX{)8?*SgjGn*kGRePk5KedBI$m6Hh4@q7i^~e{kmo&p)g?97Ifi1G^OF zasj@8bwooJ#AIJnivu79ISL~by|^2GeTtUO`{JQ+#fx=o3jhqbqwsm~ zev?}VO_(#;cko!G?VKg97=3OTV0biZ6gsGT5E1`Pb@s98q$ePQcr+k{W=CraxlZM) zP_kaICxBN3OvWGe2%(yGrc~QPt;_f-(5^+?*BzA#nZ|8^!Xm|gfheRLj&t~o)VE?# zquxZ9%(-%apWso8dGLJg+?v)_Dr(f6ND4G+9TE@lQt(xf5cvC<^3q6aiw!ReZ;J^* z3uvvB&Qne8Z>Lq!cC&syEKh!!xikmdB=c-LbZQg+#87!_*8V)9Hsk4Wuezw;_gI#` z%52oozQ_xuA62)~YrK;&Q@m3#Q;Kvf(sX3P&xTgPV`lt-^%YhVTJ`3r0apvt-cq8E zf3qF!Ptgx*xRZjPGjAy?oL{qkF)FoliqIdNc2pmH*LA{H3nZ+pT`Xvd} zT`*1CbaBKqF>D8S5|A5(_46MIjE3>feMxFeoMfbC`;t_`33m-WeJB;?E9L`IzJPwc zrv#4K0x3DbT7hGV9QlQfRK8WosfJPwspjf*U%IwPRV>AZ0rQPd$G^8EOOzM3H;m_< zQ4?K~8*y8_&mKN^u%|a=7#n482Ze1bw>XJ1VN4>h_cqqvg)Z|3sc^_%W;o)(^1d86}6=h z6#!Gx#Gd!`K=4%K!xgP9(ibhtyO~rC^tpn78{nwxKLF;;(&cepVx~p`YwPrT2MM@; z*KD-Hqtd!WvtvsQ11ogoTwEvY8O%s^wRSc zZLDwF)G7@}={OKw(&kARDT$u@(2iw6!BFp(A(dG-#EV#j*_YnG0yWL^*z6}b)&@ffh3ebpG`yg!mP z3Xsg}rb-6tu>C_Yup-H7CTAV=Zo@<_#hQIy^BPx&BbFV#f0>n~gBHC%zykSPKzX(aU{9}4*)fTXHCRV&(So=N&Qa<8} z$)%5h9sPD+Drd^kOs3Q?NWPEax~$IWDAw(G$McKrI~np5@!i>nV)istT!1>_LO&-! zD&=p)HSU01Si4Z?Gezi7?QKFEcN*_sQ2h%ii2{^F1sV{;u45dhmCK!Goz-ie;<;Xa z9UrEGD~6vdgp-lm#z%XUaM<~=@9(<-_`?}0x8g9-xE9ahNw|RN%d^FUr)B9+*X>QU z^wJ6X0)%N`DPLPNUzB;{`HEI0XKt$!UE<`f1FArdjR*FoF6O;iqo$I1-c(p1ixl7j z+@hTaS2a~{YB`3ZM(v3%adVjg`#?48RY+(=lX0^L_wRQBai6nNPpm_3fm=KXZg1{} zua}Iz3Y~*9NNHZ|`@V^DNs!Br1&!?Hhy6xkpeR1rpIl_BEB;lrCI|FfD;*Wb>@e?dfvR3&eQWyaNyw~ z%5@B}!BL;g8U?F=sEV-<~Db7)Eo8)AIACW%SRskK;p)ve({Y*Ezk zl(13^=uaPN@S05UwWed#b*n4N>s@&{d+kfYJp+*8F1G(-(`5~~MBmzXov2Ow1u5jm zE@34U=&$D+Ip6dMQeGR~LFBJsPzCw%i?EUu==7I~f3GRUlOrbWw-uLxvG6>Sy^t8V z8`IUwCLB3@4C34Ksl8(q;iIN;PUaW*p)f^39Fhb_`YNn%NM@C zHbnb?#GO?2S&pdh0G&%ohLw@<-|3p}6YS4xcP0^R^#o)~Vd_%8@PJ@o&|JmZgI@XR z=#!8RJjaKK)3a1vH>x{D$L=u?g*vJRP;L>R2-tuXc-?FAJUs3J@YuYR!Zh$VC1Mxw zvrsy3ZAqp*@|vRWN%sY>10(!b_XTrgyM`?r?NZQ|%x5cxAS!n9tmr+06#yhUjekuT z)^F@M0FFz`pBs5ipz(JcSsU(|D{s$hH7#W#Ey4G3L+8m84oQRha|d2 z#!%!_D9hllfa{|kY>i>L)>_2-(Hb=oMWM@Q@=%rO74Lb}qJ;x=L^e6rqFz zErIHOWYyJH5X(hAhn)bDXpvzI(71E*DTzYGMzEy5;yWB&Cgl4eYEz=bPXb~M?J7@1 zgPwSTC*Ire;pow)m3QgmZirjGe1UEsUX-aS+IXRnS;8WG7V5{G*vA0fKZovfDyhae z;wWjrOmW5xIr52v>%CGkiWpr|;}KrKeBz8Eewgg2fULO9!e;PuiO(fs8qg<;Y()Eb zTCQ%F540V-xM5){xF~o3Qwd$xvbUGI7>6Yi90$7h^kpI`IM>Kf3L9bNSUejh2xZ9V z`K02J!!?qP!iG-y34skhL^O2tyvOWlv-`FpHn3cWwP3t3$1AjM^sw8Qu+K&~HCJWa z*toQ3+V`&P0}|kUQUq|WBw^IHt^iy?YsEvV@SJ6>h|HG2R7Vj@Z-azWB$C5B-!JNr zWk2!m6xQ3S8KP-rR{iTb#Mt^lIr?Cuvy_y9R?Omz!A0Rn>{W@^F!7u+$y&j29ZYnDoQP)VpOzp8`=Im)rTU!5FD}w@cTa{MG~Y zqn=oAqcH4QenDf`-Z!+=ehHD$kDedrCDaG!JQ~bMw4*lWpnrL1CREP<0I$Tpa)7EG zRr*CT{Tn0N*@A=kG{g9_MA}#X{hyQa*dFp50UKQadkLz!q-for*XG8nhHIOYS?nh{-IqE$BHbxCbpHfv9RBx6x0 z!}9w;Z{5;&isrzb2Ra|8ipg{+JMphNNsFeno&TOu2J47CTTq7>$!qE>Yfd6GC#iP0 z^=AU2JwNK~AN-j*c(hYV;1KvPhQ;=8U(2W8wu89|yE(wI%#JRlbXj-7`xn3JnkB=K zjnpm93(Xy}eB1L)&~-90UJP%PE^j0#KU36qs-Yb}JfJ=B)qeh`t)%qQ_Fo7dvLmk( zjDI%zA8XI|4m+9&9}D=H2ic#?)q&8ARn>eF5Px$Jv>fy|z`j#s263tt>djX%;Y|I1waH^^2`%- zc(Ljo=Rgjl&tc=T5nc)G^Q#J9YIlA*xW$~2Xs>5?r-n=2Mg0rAFUgQ^a>8o}f0Kk( zAdcwj@$8svA96Gf({>;3a_KMMC*%FI;76-d7K=Xw0afttHcV1-R=Y z>5U!JlKgrr1*C?!CJ)OELx-(BPmG+LTZcJoCLIQqvtGev-J-hai5x&EUG}N+4ktX+ zt?!EdAL{zZZ&xw1>}8@X7!dr!(h#YG1^o2g8hsbN%W3e8mf(o7Wj1)ygb>r?MiW_2 zhbmeU;PCogaZrup@G4K{L!jSA5$43rFyxdP&fjf9iZP1Z(VI_SG(;3iG*Tz~>zP^a zRFXO{V}U9YR*Z_=nlJ1!WWA3~8MAj3aWbsZ&Y7fLvDd=jbbBZXonJR6Rs0_5)@9*h(WWQ`3 zZXED~8@L&CV6L7UWZ;ap8JuewFnm`YEicQoGm^&wKBLu1m?}1uGm;rtiS{AEbW1HpwRMqgURlle z?WkjBCszhhyk&-6yye#Zms-|b3$ioy(uq6^+%xT;lQ=K)Q>^5Un`&|#6~8jFEr5<` zCeenIXdf(Esq!=9@~vTy>Rem8>NEeB^Q5~h#(&x7<&qBA3a@5}C09&mrHG1tefxOd zJX3fNgyv>@6z6rP(z=;+Ajj??es&YGM72xbdhsTTWzvrt3qUP|oN}Rjm#>X-9SEP{ z<@Xzp4B74}z6>PUrL-sRuUVEfe*HP=Yjh@Fyhsw?QE;M(;QT^|*A>+Lb5?o}Go;Pf z@pGnMucC?9>om1a#$2MyU-*Ouy9mq1Acd^4f3OpQgAA%=d5A;5s{D$kTT+QqNovxag;V?TeTqxPHi3_`?3M)x9Pr@$JtlaVmkNI62l}AR z*RLE%LMX906ptq_7*OKN|A_}&M5DY~8mm9-Q=jyb>%BE31gL2TX|wAmI_ zBrO*i4A_iz`x90#fnx2IL^x_8yp>hp^}jA2UoiNT9WtaITvOeGQ8_)5AZ*|}h-{N5 zPZGy_(Q z{$^1$Knf+ggaU6iyI~?dJ4U@jT0IMvet3O23}}dyqU~;>!xXvQ5BHQr@#n1m@G3a^ z@q1>dZrkUjIqt5had2Phb7mFJOkuv+6NSEFM!f!j&MQp5bm?npbf^3^!9VTt7d9U_ za{SV*;3&=2Xd?}K#XU>dnQ*0Q;(>0VKNhhJVw*9zatC$<7(9TdA8X9Z6Whfa9>+&16{vVHi(dXIjRUXo{A-?88{MpDH@}STmMrgR#>Un}l zJRfA2`2Hp8W0|M~>YG~!2&+x7GO{IUako!-KwNGA2XCgV5njn@B^b%a6UWCB%~yJ) zP<{96=PNL?ZF68h-Ir#1UA&cYh>>b*4g0*QT5~+Y@(2<5mkrKJWYEoFNi9=^nOSxw z++=jaV%*py2vE6r$V7H;_J>29iDWN-os8cTN^f07ZIztt;~WR)pohZRN(5B$++Tzq z&w3uoUA$~o*Y5ba9oU8Z3)sasChuAx*yHE|3f(a++3~jZ-go<3qtiOiX_T9_aPw{} zA(c5Kl{q4nIX0C!ER~tdYrU{lkp-(e@AMDxfAl^$!2hw$+=bMwjgLAbzIV&8_(ON`xvpRDmRC zKDD$sbH+lylqlw+JoldWt4PW(O6Z61=3Z>Rm$`yL?t#GY&HHy>o-hO@7j*@0<=-mN zdWW>IljUbh0k=e4ghYPu|KKMQKj81^t5VP`f&N@1+!qkog#1*F@G&WMH;{N9QQxYw z+Ogv_mofQ2*kc$L>%Qx`K$N;qWi{?!<4W=lw06jB3w9t&hN!-!2|nN^jw7*IP!oL@ z-$2yf%xbl&2EU=r(;qw~waj*KU8;KkkjJWuQ;VKTtai3u=2W2QXt)6!hYE zFn{-NfB96bUxW&cSWM1RVbcPeLh3nDRm;x&`Z?u{Q9avteJIoHn*cwuTnw3;Pei>a z%6w^$Z1YOttn7g4IjcbOCU=KSw2p%i+`o6078B$~79o1{faPvj&-p~J`yL9~&YveZR4kS4DEd}}#>ee*B4S9I20yVli+Mv?YQGZ>3NLP1JT* zFiTYP`qxB}QTu)c@2U(PZP(`knIxn79PRtb>^zlM*~wQ;5p-e#o~v6-89j5(yuN4 zS(ZpKKim)g`S&XDqxiXN*VmJ8i(i>u+(t$!WdtsMb^4t#zE6O?<&Bk0?&{IWTMjZ7Z=*u$ieAf~Ao$-&ay zM1!Xkk542X$X%(7b+~$sA9PZuD7-~!eI;*&8tCgLtBEA5fs@tf<2XG+S&tJUax=A+ z4L~c@Fg{#ebB>kr`Cw&fJhV4h{^O^2<)li?upOK+9gdazax2*px$ma22d?uaA%dZf zRnTLeOb#FS5RN~`^&EeAtK3M$3xp?QcZL-5g(uMt&-Hz8Im9b^KGgHc%qHpGsCk3l z{*@wAoc47~bF2xscBG2R(+zY_wxNp|87K|i^2X1bp)W0(aW`)1)Tr(4God1rj_Fy> zR~8{r7GY98mW)xbE^0w2FS|+01R>{#V(x4Ol*uM3UR>beT58zmCKaoBwsIKl++8JD zFzgu-`cQ;rk^G{HkV^lmqUrEhSJe1XfckWEKrqK(#(Qe#7!J?+ODa`;m0I5Ttyn8P zCJ~R?ao4)@;d^&9(ThXpH=Gy#rrCib#*2SreeWGbWg$I_InuL5H`(|62(P3A@{}2R zF@p&M+-yWTXmW%Afek05LY$q*{qOhqo~N_aMRv_N$T);E{)$l?lHwZqo+4jNc@Jbm z3-J&6Q3KUY?Uelfcvv^|VZSac9>*Ps>=b9`!%0%y*Vgc-l-ycS(!E>KwnXQ2#L5Ml zujT}akLFaf-E3$4)?WOY(-AEf$QpcE*C%Hl-FdUCzHgb+5G)o|J4V-|VoS(aV#fQRbzRGgh@C%TAO`k{f`MZ<*BUZMv%6&E8B13hyNObPc8tT7Ph!*h%sTTk%ywFr0=bJe4QhkOw#N>lQFo zUT&FVb9HD7o7TmxG($h;t^8cpVjxNM$MM^9@#58qyJxmiQRDcB|7A7j)yYM*v%ZUG zDLiDDIF_sySa*8TuXcuQcy1>)9k-QE@FbIz$d#caiqKdDGU!0Ic73_0)7i-|2xgn->ROXa~9H3PK%0y=(O z7LLhLsli*}hE3ztJ?5~%`QNVlh--W_D`IaNY5YAQaYLKg>sIU%28j+iCq}L^K>5$1 z{G4dM7&tLn5_TWpZo*+hDrZv_ah}fp3dfxh*@ob8UVuXCLRsr{6`Z+{bl0mQlIh}o zU2`H>l3Lm%~m3 z(1jos5L6RLI646?z!tBgL=UsT&u4;s~iimp@P_k(3Oxb4}FVs!^c?`dy0MzD+U~sLhdGz zwMQJ46g?`?(~vZBG!RDPqy%pfYC+w9<{>G&M!{__3E>Y>Be026Q7YZy{5X;6{%xJ4 zBs*@1s{ZY)O;hbrnv2lN`$2lnN7_rVm<(>WzW07%Y4ec7(ATV|=<9Fo=drPp=oK5X zdQia2G;TMpBMc>+@*-32@ypxi41d95caO#HUWnaE#tr8fd41|h7plJ?PIQ_{<0m}j z>uHY{cCCd3MD@>&Jupdv=6ldnjqBh?=A_8$n&KzGf=Mp)Hr^_~#Ev0e``MU711aAA z!(RnMac!>qxQw4qvo+sXTC`6pO?>+6Tpb(3U~cThT?!me$)t>iKhZQc4#>`cG=@=N zHNhz>#NeKic+3E?(jypl)r>jv@q<0p+M?mx2W)kmp*(olg`@iOoe)_%n4=`i`xam9 zEgwQ3FJB5GF3!&2zPRku!N!)MqZ9~Z0E|J9T&7U1yW7Whq7O@CK4c#ho^I8_Pi)LY z%$%Jz;sPbEqHjLMZ>3doNO7$q4Fi6R%w}{J{ZwQ7qG)Xnd1@3fNy^6SoCy^!{Bw;Y zqsXUJ{*9N<3b_fTtAvgjoIg3S*hnvwmi&MwQ+0|%IK?Ri%fv)3iOT%Z(41J91~|OW z?%5%KFvY5ScCWNE>J8DPUS2lvj6`rJb`zC66?o3rUNgDGI)R__4nGH*VNFI_eLUUl znd{~i{-EG0Lgg=RzFGX1A$zNrQZvO^l^+}Fqliu;Yd+8_gWUl|gh1qeWL-u$K5#C$W_H+$!N$x+_CWbt4Q> zeke?ONcKm-d|F6g$W{&1 zJ%ZI;UxBFvITAt#L<_^HnmmY+@UIDQK{kvI8V>8j6@enZF+evjptSeU$}#7xSjaLB z4=odifXFM`~wR(X~K#rYKC=vMw@w-tKPKI|#abJ#|_gp>57`SLZA5DZm@YQdf% zUWXjLR6?I{tlU!qDx(2VVkH>}!(mdJ$0^5qPP7J_Ro25O2z$yIoTLC1 z!qufgo*?wckvu&KJbY|1dP<})3*3Af$e9?s`Z0&P2J$0sz6eA!#F0ozBMy0TjU=L2 zp^kY?fK`i*mpu%VDG@OVStE+q!mHOP^@DL3hl~J;*fI|q%uNa z0_ixM;K;$Ykcaew8RFD2Aalgof1YwksUfwf>hO?Nh-)JBAp_L>0($ruc|cW1iws0y zVf$JYCG8+I0mTYO3>=K+h27$>w5uqw1}RyBk42Q9;jJ+I1L_N)B8jXbY6<}gQ1Z~% zAT;HM)<7Bc@`)w2#rA>m6X#Z^Et?BlP~AP_bY8Z!S&F@(%q21}-=UO7swk(1%bn3` zk!!Zc8rGk;p3;GrhteS3m0(>$UM*WS%5x|sC&-+LgN+;_gp)*$q>4lG&cZ%fh7O4v zC_M3xO5~2DR0rr%Ajchx9}LB7`-b)X8mzuM=6;~pGnD^UAFr4}xCW@J!C`Y>&L%tJ zJf7WxAfFB53K-Mpuwj$4Ns2I5fV$$2F(8`|4Kql!b=AVh*kJZC3uHcI3kMy|0NuVu z8c>8)e@{9R`KQAO(IpJfYbW>5g^loMUV`x;uOKi-C><%$VFZoGORa)wNnal$gFFmIfV_mav{z!F4jby>vF7Mr#T4hp2$9p zTZelr%rOT>0~yS- z!V0?&bf+hbtC4KGf67s>#8LtS4mj_2A@L1nV&sp)XmC8i*bIFLd%k@iXZhVY9A68gD)!L^+g) zOzA}&(kduIQvuGC&km^p=we6pm79MR6rrjRq4UoP0P3d4nITZBs7uvWmBIS2b&)+959wLGK+;x~T25Cm6@{p6VAL<1Q{B?3a` zAxH>pl2w$fgOs3P-B@<4Qo%viV51@SHEs9g@{=QY6rrM_=QPM!gkwPcjCnk(u7!er zAriac!egfeX8|o6TKwR_LC&&-ulZRSiLdOfng7A%c$KU~7QL2C#Ah;&YL2BW zLN6zweyuYsO9F(5vx8)DYRM3c`i0!$Y{e}2$Gy6}uxS>v+$%o)`nI_n>p{wjEi1;Y zqvb63Qa9esdv(CUi^9A}lMhGS?<=uFb^7(yi;f?fo9{l%VRaB8j>0OR*#Gk%kKGYy z+$2IJ7ObHC?;k(r#cCq#x-igwbUz)TZ7aI4k;oF#;v-PDj1c(4aB_Y`53#BJ3ssVy*#X4wiL;w*Q0#k8Rda|neu8qrNKCdIatj#`asu zSWnNz`gt&VCCL6gD*0rjzOLQ#&BeBF&c_WklqLPKPoM0BgX}S~lZPFMRDlE`C$ju}0(5^-1US79|T=I%g_8Sf4>+RL7$KF#xNY|#kS zV?cdOQjgrz`?kbW;(uUWUmX-j7k!DlYw+OiZb1gunZe!N-Q6Mhz=ylLy9Afu z8r&hcLx4b*-)_}b?LXVqUDZ7`)l)t9zWeU&d)_&5V5YmF-YY){2>tt+@sc=Gt4H~r zIMf3WLA2}>`N0cQ!6`od5GwM$-%&)cr0W)LzRLz3ND{maT@mJ(6eC8NsG~4$R*LKa zTOlld4f~Mnm}CHTm?Xi5a)vnv6^sD=km(3yg9b*U-GS%if@PpTTl_9v64-fijz9=n zY-zy+lrr>hlHiYW63FiD&bulygoeS@*Gp}${9l14dn15S9HO?BI+AVwjZ@X_K&sDZ6Cd-UN;2UMz1Hxv-++| zX^}pE8lxHEm+paIaq1YqRZ6qOxi$XFu0(JsGlL>S{7oZn9BliupbO6(x!@<1 z0sQYWM<8qnou7eYlT6FR73>4l*T2~lO8gZu;*w62$h_3ZwAq=QYlAft15kn)w1R!8 zG`QauTLm*P1#DcuMNz8YfEK}3FzNvQil~zSN1$rw#BVrTSJB`z*aso;c#p7Gq~O=d zuR@twn>YpSP`)GuUhZKCD8UF5Uxga8Hi-(@pn4JKK2r+-oH*b!0GKOZ!2*qfzadyB zInK$23DhMIv*=(fj zZ|o15b0O}tWwZ;kwLEbo#+z^ld6^t>T{pb#GcEdp%kg`S{wBmhH!IC z&Dbq%?A~4(5bJ2*;4%gq6)lL=GQ0_(h~F&^g?8`?JXL+`|54IB3BkK+vA@n-S%9>ks0V0AEgF?nCK`1dj0l<7F~{LK1I30{o(a#* zHfcQGZnQ^DzeTd8H)DeJP}8jPjvDurpIp|fee8pMW5sfFG-+x60)@1+uL?pwO^iM` zc;hr^@Ub%sOa%dw0p@Pj*Khr;`sQ<6koY;_#Ne`i*04{J)Tm$cI~ zquaE->`via3~4Wg{?(nh8N_DpfA6yw7u#3ci$PoWP?Y8hYs7gCXs2tNLV?%EkID!9 zQbm={rv45+rMe>fG1-(yUDf!7=Gt9TSX!5NBWS0Z*?qLcPDB*pV|s%zYiQWVJkr#S za<=4~0;Y$=ksZtj(um`;U{q&| zTS`#3l@l9-mpw){dk9(KHPsNxw4#zRM+qbIM-q8jp*7y4qg(`dZ^fCVwT_WkH54IE z<~M%Qp@t)xv}tpi-E_=2U5DoKde8P(TRzPsr}^|UXFVpIL|Pv%I9pA)Z=Fu={zH?x zN5ra;nJH!8Ht|$>P|e{00wB)3@1}BNN=K|l{azi6N7R6cFK~V=0L#jA%`Oo=k21=% z=wV&Udfsgw*Z9Qm^te7S9S==yxhc@*H~^W1aY9&iM-*DM37*I~a{+i>oE;XI(sEs` zqnJ-dS_5qd68?mhvw479yo#bP%@EQ}w@6D3cWXhqt<_OAd>n=Grp`T|Q z;n3&4OIa(t`sl$^WdZ3C>r(l96I79C6SA;*;`YByVTi2F%(=Xm2BoN4(kMwtzcwcW zsJWkFj;094!8dj8jpJn&5Ee=gw91K8Rv)}jcU_4I1@LSVSAkl9z;a%6Ewc0mbc7?_ zp{yg^x?~K{?u$}g7{zz}$k4Avbb~)45?*0^nyN>SO5TW?)}eu6bjP){emCY`Uv10A zjal#=2mZ9u>x^Lq;u3fJ!-l_M+d!XfSW@3nN(G7nERSP)K)AtL60BeBFr1(@5s4Od zAlCTF=~d2ZCA@eRaxD;hJM>b%gjone&IZzQ^*q${PId~%S`;Po1&*o_d9b60No`p8 zoT6;@zO@<&;FwUmR7Tm$OLEb^iVmdVTn7JAWn0VqUwAM&o1WuLw>7C8Ih%+7kZG_j zmV!?wu*EeA<@w534|xC9S^!S@ zmaVk%Kba6xU;j-#mW=D|+C%qe(!%&g6dT}JEOa3 zF4sz}u^fGz6MLL9ngsIEQYkJ?v{#N$)Fi}*5mbCaVL;#&i%MCX=xk#u8kC)(C%ij^ zu?|z$20&}05=vn36iKR~I>T2N3|2B#hy&OF0JcxU1g4rh4s|SF@$U38j1@^*eO3!Y zQ;CYdtq^^QIYV?_ps>d_L)J_AXHMPs$~}wI zgTbfNm=nl2E0-1;RH!ivx})*gE>ysdn^TYVx3rW0o;uS~P6fqbUeS0A94tUz|HIyL zYY}+?YKeb=z-_J4UHE(Oc4yMp1AcHBI_K5LF8`CM-eF3pZ;~L%onzh6^(A}^NTh6i zImQTR&1Ib4kw{uCfN@pdTI*3`sA@$%h&Aa#~^h!TvQ@n1m9 z0lUUok5X4;rcRn#rVc*wJv#Bd1LfzHhNUK1d=!|rI>t8SF9WTmsbIyMD#TwluuTJd z`P&`yOD=uCM8i*ge8nHWe_)G#MB93ZM*r79T$Nje05lN+{J@K3EFmuVn^(4s^*yZ) zClqE#o?WZ}Zz;PR?I%)O)z{y6NfPSgi74}u!Dza?C#^1|*(}&8yfBL(+aHZzmU_4? zV3Y{;fGdA89F;KhPU}uDOrXUr%ji9yY?sVOna7Wa%c}|y&`-l6A6QFJ3jP+BX#mU5 z36Q3sI0g@4(uJeB-pBbwh(l$cbw?_N7=74Y zRwZQd(=Q{FAck}lNyHkVUl-DeB+a5>}JEFo(7I2nu!P#O9>>-_OtxS6Lf zRvbR~he`~{S#mOIR)v$z_le||t~IVKr`| zVv~J)QV`UcR7tSR3zlEoj+9emf!L~wtN0z}Xf%I+p4wnspVn!$ed<}G-ZPR4LN20c z*}0vx=oM~lMMt|f?;zR1x3QG_5pc&6b9Gz=V8;mbv5cT=#&t3tKL&f5{TH!5#oFUd z4-uhHJR{@9tX{34TzO)uxm?a_ccv8|Tqd?vi~Iu7BjW0nGxPR$z?u`$3r(sHwZROx znWX~M!Gwhw-0N?rG3khKG8CXwoMTs(h04OYt{q4r5Kcu^(F+j|NpE}cm)#It@b24CZRh*O(EVmMqPSzM^pMqvt!O2p=OozuUALC-X281jn` zRIcXo-1pmx0sd|cKJ*WU!>$D~ZF0SYx2i36Vv{Z0Z^BD{$td#=h7096SG{ts^Pyq$ zp@7))-jY^sRS1dZ%2iiQIwfmq9cks)id^av5uI0(c;Q&CC@xuU^Ge)hYQka~^DH+z z(z>rY9KdfoBej8+KhaVec`yW&ItkR{&^KF0U z+XfGMIW(oI^z*5qb&#QV6nCszHaQ9(v}P7vdoDe*ofu|M@I5rU=bJMvp%|3(dPQQ! zScFh-jbT->k)sC%b9r$Ke*vFB_~j!Q*_|KDF<;C7DuAJs1L2edA(ZEer}{R2Ud$m; z7D_%a%04NSZgK9-E$Oww6RTWVVqsZgKam@AE!6#Mg12OHB8cycys{E~P8! z;e!fl{}oNSP5s1pC|Q7K5Urd-ruPHeR93v5SGwauP;KJKhr0gUP-9+D zyI16`<`8g=|IvefJ6;29xiOgRndq^yiZsFc&+_KSpvFFwt&OSgTsVHmLd*-&vTc*Lb|nH0ydR5A79; zUhq&IdTit@8SPGN%ENl^I`q9t~HC814`>W#se3PY?PkPDfm`%OFBKV}TSbd-d70Y<* zrG4o|F-iu9_%3~EG}l%$-aI|n(ua({8ey7CM=yP;j+MEz_qA14+>!4`k~p}%oY%k8%3W{b4|jzf0y zeHNZ{HyKFVI(*~kPbe)WrKx;N+EK~i8SvEb!=!t6qb>^59gVan;p9qlx6q=adf7(2 zj7N7en~fI>GQC8z-~PM;vq;G@Z|SGfw0*F5&3`vZ|E)GmLkz|ua|ZxyVODxe-f~Zu z?c5jKWaBtQef(LSn1CnuNu~QDB_mW??rK4WMQ=@$-ch1`<*IbIhBq+ZB;5s1zQ)RQ zc@JF?&y#wwXm-g8cP~TemgE{u_i}>ZvUS1)lqERlk-Y&?yg5aw9?BTA=psSg08btBqmQZvQ?`#4?meCp@1W>%1B?+VQ;n9w9^;x`kc_kA}L z_jXEwxoo_7K`}Gdyv&H&wOnCXURBXbwfsWZfI*{5wz$< zpYGV2X=tmowti46gq!h|!(6>?ZoWuC-$;nu8c8}*Atq>Xgg+9rD@|}%IAk$D3;^3J zs_L3i6kKdwU|5;zb>Za&$dPTBlT7z=iE`FJdhwSJ>BDz9Cm7Q*O)hXD0T}@cfhW>x z7E- zf3{sM`iENvUMS_$DT2z>98$6>`AWavuM;6@Z{$y8OI01OHy8n*aGw<4l8uarX9IJ$ z2_)QNC=suDvTKQMSasjY09mB1>#+#O2J|jhvNp%ew&lXc%l}l*isVzpe&sX1iN3|< zp2oi7eAy3mM4$b@z*95gig(Kf$WNJ)ts}L8jQ>S)(4oTL>!Vrxb8dvcC|(~=N=Uoa zMq96u6KP{>F|?0;V$c_sj?6+;zSUOqqk^W5H)^5XT*`}$1N5_DKERI@0j)y1^x4<} z1X<_$!N$L9co^HcOM)Jc=#O@lHEnd5Nm=Z0>t4Y+>CPQ|!p!QUS zWOC?0s{%C&=%~0RNQnDaH63qugMHU_+hUpSM%$RpTeuuEhm{&#ECC^71sH#)}dvp2cK10NFZ!2W`*}}B&>(?mRhsz$N*&orr}hY+?@siUD5I#nMI)zi-W)|I`#H7R~$zbBCKN{RTgT_NptCmA+q zd{IajhBMh{(uB__OsfcNUD`ZS124m|X_qbIo#5{l(N-Qq|~^r+5W?Ws{^g|ewe5yBK)ci8sj30wBk%bU8;fAG2ps@ zZy5^obFdPKkJV6tt-aWWTw04sJ7hEQBh{_cX})B(VCF<^Ri_S>1<~X_bt-9Fr~YYv zU6)YBA%XsBASByc_yL10y!I!}@@aFh%43B2(ghHX{FZ=zw z?MX@JEvATXYfLR~o@ZV296&65aZrQBvHk2AD|O+er~Tg=oMDAKRuTxGQvEmKpQ zJS)BO6vb}F3#3CJueqNg)x5h7o!kCkr2QALnj(##Iv&-;@+!f5<7JF;O@D0fr4vcqv-rsq$)L)tNLT8A_=FmOoZ$&G`7PK}hSb>sQfo=MJnmx|r1Cnz( zdZY;vxXyJxHr{p%x6!S9?IHceAwhWfjjnQI^9GIJ5%xfn)+W2Kp7Z*JQ42NZEf(!l z&$Yy#mYV52pW5RG5^mmz`%RgmXB|06vO{z2)<_O0?IYA*{&P4=uX1N+#ojULe8&oW zlrGGLJ|oz)@S({#4s~Hukh!n3i z=!@QdCTBpcM{6mt@T3{@&~TdmRo?=!a(O%GObCYd1H4l!->zQV7}FVHZTZ*Hz1RI5f&+qVu)F#VT#j9xpMKtlp- z!ax{bRdy9|HG)>zAh;pflG!FxG%v@6#?M3F`mSc^qJ^MWY!ODVOE zPEl+XHi(Ro*?GX=8vu?KUy zCyxh3LaFiuNVkEOAgWK2qbvm`k}Qfh`Avr96z=RBf+Dd!3vz=>Ooi5f6Rvx7Tqc{% zE;7c?F4YjU9R`X=!T2s!_k2fQw*@6)fDDO8VpQr+Nxid!Y!a%sh_J>oYrkfJ;!m#4 zILh9d`eKX|;4@Kl49GFg*525}-Obt#{3`C#q2rv7=~`P~yop-*rVrVem2$K#Jk#*l z6TTw}o}e%M$$a{=MJ*N}UL8>R*HsTheA%7T#qMe}kTi5(-JMbu9n<%Vrw*wUs`Cp# zsD%FtJmaoWhG?yt{gcF2(++>Hq@@HgrD;48X+}ZhCBLf2xLDd^3yLrEG}glt*Td+% zPlub;yxayMQ%1oSv(sFSlztbf@yQ)ma2U<8vwA%kKtt(5?zHFUN|G&_N`!souppcJ z08eG^1$w@`>aOKGpOF=D*J##mXUeh0-+Y#5|D|!#aDiCyM$Xc$ntqZxazX1*+Lot? zsd!DgC|#(M2!6(RHq+dYKcWBPil-I_nQZ0ltkNCo7TU5$Y#+z?_8@|nf^ z<#;wj7xF$u!n~kqd&ReAjm6I)L?~KCQciZKUmq-dB;z0WCw5^zsCXa{vZAWml_P*V@W4<7#{l@=BKq!>Myjjz|ZGLO1uaAOBiB z_|}IoOOQ!*53nW^E8YyA$Iv=`PugNU*Of^F6D$zOD<8bVHs>D{gC>oHm^PqnEJ|q0 zHp!AyOPtE@cJAKG4C``GFUeGEkm*yXPIG@#!^1xZl^UC9BCO1|FTIz(b^q;_8_?&I zGL7{gcrUJVUY2J0OMmu3j>_*Dt&S<-Z0G2tU^0xg*)JJ2{+D?2Ew?LZbj3uS{0@)o zpQ&8ukwwxZ_Spw)De*CH40wn{Qk-n@7GVayk@ZcKA-|Y3i~)^1ja@UcYXwflSdM>D zkzra^P?gQ*OlTj5*k+x0UkEspxS#}Rd5G5L)VE?xuvFPwG+)KMaU(gF<+WKW zGGPx2sXZylqih;aEGpF%q5~NDvFt{EIgHjrw~rlu^gz&^sSCR8L=s(PK#pH5;SKo4 zS9xM2cI+1P41qw8D*P{fj}4?mbU4)*o!^{Hl*E#*|Js~Oywj_7n< zIeukT8ir(3E_c4tC}c=?+Ek=fO7Cx2pB~MLt@zls{1$P!r}>UgHlGC#$)09qaVejW zq>l7nH%kuKDCB&MT3^SxWlh#}$!1%0Z!fDV0+WbTm%3{OK~L*Sb~2%PuB7mUxK%2o zu-UT}>toSo@=1M;;}TXSJ8EEap>=J?M(vN0?ev>$mk*QBr3=;aq})^;McxX zPj9Vh^$*_jLyNk?9(0Ia;+>K#_tI>1IAKRLkEd}*Xwy@nzM^3lt13?vZiU`dMYccs zQ7eeecFnyyKRGf4o{f=rYgCb;HrMs-*rGBc3uTj_xU~8(+0s#L(m_(EOqBRG#=RX; zayJBQB<@FCN~+ZP-n_q@CpU^~>qweDpy-8uL?+P*XtJTzRM%aakJYSNWH{2EB+H!E zN^3{89Z>=LFCTI#h{m{%>CmZU7HWkEYbfM!mEW8$GJ3i4m6U8p=oU8Pl{~Bhw*?`> z)FjQ22|gQ_6XvM}14$b~@RdHqGHa!h0-8&os?;N^nc;wI(g31dtZ@noZIRJpB0gm+ zY^i4S^5LsfWCwC3xaAIdT;fAKV2|o1=*APpNRurS&Dj)BCTFx{CFP7BBKG9E|D&m+WHZaE(vpBg-CDrFAy|b#kN~&Olsu-Y? zeoo2_J!S9dxWjbkz6s&%^9iIUMdL;@`Jx}`6w%?YEGi{~hK4)KR09u9PP%r)`#QSSNc&duucM!Rp9KQhG*QM-HFex!sdv?;@hA!+@Cf zDz}QwD2U%G*JGsLNJXeq1N;>(kOhD z3~P9UzOT};hog!W*G`bPi6%3~ts0b!AO$kTEg#4||50H3_y5o@9rnWUUf?mA@7dfi z@}4^?ql`H?t>V4eZ&1lkELIrGZ*15fo&U6S;_Ic(VOiGP0M}41eIRNwqTdvGq%1V3 zU-DE^NI!457~w0VtxLVluD58=MCXQ*eckB|lQ?Atk8lBrCwOzl!<%l)9E+ zO5IiLjPd!wY8<=tV(hq@q7Pkan3W4NM;LL;1m6ikF*9lpC5sQ@i=f-bFqAdY*RxAR z**l|<|ASr@G|N$QnP;h5SErly7g3gR1%b87Gv|R9sE+B7Aud`himNChZc>O=0$Vm7 zLlWdu2Ark&=_^m46l&F_&VGvf=TPY^GgdY0$@8!=%02KPllsxKr5rRVa+Ya9o=L^Y zz*3r*Naal+DaDy5E10&#O>t8BWid+*Om3@<nAum}w7$h!`!%tV53x># z?#oz)mP6|@eTHub-p3j#o6+yY6xddFF@RuHwK!9#qq-zew!XN0{8IC%F8d=aSApNS zFVW{P@>ptKA!WU!R-|GjfuHpi|M~d~JZiIs3}fT`m+!5&)1l??e=G&!!Bk7Q1m(L# zCBteYA-NipEET-T=5UBqBA}bEad8ZsGNOLe_cb&K_K~kHgLK5bcuW z4*6v3j%b{ApehP2F}07s+tl#kJ7r3?frJhscve$Uhi;UdYJ#ivf#XbY9Go7LDXY+= z`{bRkcp9)p0vAn@P)24MO4{(N%|0?X=wsLnd@&cT z-TUO1Bud}yjHe7}-VR;3F38gZzQWm^W+=z_oUtV@jYdpQgHhSM+W?7lW6IitTT5=` z(gs=I^wXl-OXE5)&$-47v`1l=+%VHkzSEwTGfP&M#k*c$iC{{waCV0_kr(EXG1xVg z);xq)8>6YRqmEEHT;ADv&K}Ik9ra=U^hxn0pKd30$q?WU`@-Tq+CusspgrTZbdQ(L zEt=>G$;&FMAq<&(*@`=QkMqr&HQcc!I_2!eYFQ0E(a;Fc+lsL0Q4wW7iOc<5WG4EL zZ$u@CBK@B4u_xr{%UEC9i8Kb=cNs}}|A2F!#7~;T`obQ}iW4%36{*Ce2yR#%^q~gD z0U|;xJ!mkf;sgKzUjDea-W`|0dl8I&@E`bW6sz2S>QbPR;M{83l=G=Fv5}br7(<|~ zT&R}Ti4~CYoOl>2>kB1!K(njX2fys^aP}Bft{kn6`knj)Lfo)L#U!=pHpm#>=+9mZ zhiAcjAIXISgWAp8KD_sDqWxUvwxGl!zJ9gY*^23@nsse?EW#9@k5F^Yo$f-VJm4(F z+JS_Id6NOK4#NS+>r~!s%iDOh}B_E*X#uN7??t+-fH8CIQazH zHcJj-P%Rt8!pqjUIDpLv(l(UsBxAUqy?Xv7&hCU5RUHE;SKk$`{!-f;dDjK*qn_ke zfs+?&7RXt!_xySz`ekA9jA@dk+`s*zk>;H^>$*6WHuHJbUu^PuG7ox!Un{(Q)NJ|= zl{v7036`r9nijLn{~zg-0%l#pgdx+-(9+XcUC|Uagu=9Mkl`J#jGr||&i1_M*C?G<*B+r&lw*B)WEkU@lprQW!4hydw<+{HD2 zRsmsE_>TU$OlMy{ix+%I{d7pz-<&e!eUhb$?9GrSEw#YXmtV54@@T)NnI*+T8wAc1 zcJQ?-vW%o+;vq(Jp9W z&yuw_!)bZcJ|(@fAVJ8Fr>ty$2qMv@d`Qx93jGCyXK!z(l;CO^LHM(^*F&zRkR4$^ zeICDWW^;X29T}G@EX?vR-cKD2{4-S@ee(E(k6NH(g30o!F{HF|b!hDNa_-NtjtCC4 z^W4%LPBk?paQxJWhx@=vb9}94xaXLpfYd8hqB7pdW=%pq4OPq-Qk{pVeZK>I!2Z-T`IW>6bNZ4t zR%##a5X)(33zKioyCV3ATqmFzc!(H%x7E-35#_F{zavbuZ9V9Uosc`iW1ZN-gBc?e z%~<*V-2T|R|1#@c`9vJo9Zu>k#^-t*M;7YJ|1Pf;*C~XTEzaze#C8$zn;^FXKO#jy z-0o{H8`D7U)5d!*@W;xTf zb+t?g>)US%@W<9gpG}*w9O*s5A64}}n@?Z3V>}Y$3OX>m6R7xzu!U&m&jw1r%ucOd zm*;Df;TFI0cf9HcpyYof-0dT!-o`f`uS9@2u4N?qXLgE1dj| zHI^&qdyfZVzaX0<(SMsl&HWlSbAr9AfT4bH+0m-DC+zp7m4pZ{+;L$TwbpyM4iR4n zRkW7xi31m*H~FH9J&r#rjwp?B23Y(w(zhTqU`w9-V3?n&F{-UbaccImYtn86zKWV`6$IE%_-)s0q<4m9}ZP1FIDLrlBe;aqF)%6!byIbR|yQ;UrhZG%_j_pfBd< z3tQll8yP-vL@&y&ycwUlmSk2(YsNH#5(-!D2m_B*0|6?4gbzqb)1rMTom_4`uFNu> z;%k7h42@+;A@{*k^jRb^+2fhQFTr7z)q{=*9<9JnOdx3nXbgN+ASD|OZ#O3%JFt1I zYwgISj4qpo7h}T_?JBfEdbO;Kos#JdRA}(4pFw>prR&=odv*(F#{oy6DCk~ev?v+L za`L~viRkzl%7wAX5nStg6LkTgKn`W|UCWvj13PKu$G2Yl0Vuco6y;>3!>K)>9VcoS z%VW&&7^0cOC8K;xi)x)S3w{DlLbEOmq@N}7!jMZjETxLRsEVG`JcLyvtaQ0mX5mqN z>5-?qYo(x$?v_LjjBQgHYTwF>=kbZf1KJ+^`EH*{V+GoC-@6E{&v81z4;?){w3?H3p5!!W9A2 zCc~y~uqS>+t`BU;y2Q`kVdTe^$=5CUh>dUuG@{h{Ks#yps0^kG`^prIdyF;LA_+_YKkD8CCrtzs{5)4;2 zsWk2rijs(;N`T*?4$hNdeBPgL_I`*?ZmY9uhscsX8y?4h#x_Q_j!9#tG*aN5zS6cS4|bI}mjjU`whENweo~(n57G~w zz+2OgHWrqxFQ2aSK>p$5_azmL2HpO~X4V|1(Dmv7XqCqr?WpaM>_ruIIoopn18^2e zh#kyRYrg2^K1>c}QWh6XaG8R#cPekSaE{thK#KUlMVsMT*U<7tt^evNYGi%s88UQJY3fN|za8%`R< zv0Ol7OfIs+#%}xU{jc(O*%JMexa>jm~W6WuSR(t;rsg zhB7>PhFP`^y-f*k^++tJxyx_=ojyl_mYBvv7I+Q39)}@5k7T(iI0a zHWpG7O$OYRURhq61K3*ZRT=ESW5&%cmE)qT`;dMgU~hYf~LyWtf)`OL&-mc{E0v@fPTy$%k6HIrU!d|u9CRP z8FXU>2ZqdrLpgYoSk`}a1+`Qd=nf^@%iFDJL>WWCntn&3w3R3q`j;~~s!W_7Kd#|= zp%ByOqU|k^ef<=*a>o=CO1!-vf*7ppHMw4r zn>>uOPslU!c0dJx=&CxFLOy?}DV!79SK!C+#zqByY`fR=W*;R^>S{grnW`QkTBm%} zR|fAw@_vcT(y_2~FW zo`YyY%-gqK)I-JkmpM`AaA^Qtc8&{ZgU&@Kd^bZf${6@cvHk#*q5b zqRNl@syD&6{w^<6mVX!f4~V`N%LUFO5j^5|OU9QTi2AtwyElyz^)lQKL=3Pci+zf9 z^(t`n;xdhKp+9WSYSmOfXFLnod}~>zKX5~0I4<6^ zRiwqw1K{VizlwJ}1^$M&Dd3!vV32Rh@Ig#?{fP55y6Lm0)|rZSqtjWcdM2)V_WhVB z(j}?o*G%Ul)heUj)?0due0Ip*>sVNGu3>m4XL=4a+0$N;&bHU8TGiS(Gm%KVF?+dB z$!gu){s+89+*ZB+Z43VxV}1OFhgR8KoIO#*a{b?(EJ_*|aJ9bjnfp43h>P*Rx-z>4 zo=L@fUWHLL7X;Xnr*&=u)ZB448J3z>kMCq>`(-3Ze#HP)u1q~N{P z#mDI>LhpR-oGUTPv-mWnaTeNr;PQ=hTPj53E#N9<<8B+eZ(VNBdHWLX2!AbD-4xC& zX+>h%xN-Q*Bo1kS;^95Z``G%(r8T5mx2mx4e@CD^%#<@#J;rBw#$g(Xot=mM@>tr$ z!}~Ak6#;A%MB3a^?t&$a4h`xne6c27;wM;uqlr3iL`Wl+4;b<?kbq#g!35xtlXAmm{R^V@?rV6BbUuj{`l82Z==y-BJYQ; z?cAdYU0M>|I>nR3KEWmaMgG;}IoI4AkPvT64#)|*s=cA1@Ti4veGkE+D4V_gVHl`7Sz_v z|GYhs6?0M%dy0=KdyXMn`19MJeYfj^qXcQhw9)S3Q9rydWO(tUzc&x5w3l26x$tTp zGq9+dzFqgJv%O!oTn_kayFlFFf0h122(P2{N=g3DVExb_Cr9Vc{*(wVNe*T#!UZ6J zBmh)Bk_nGwHV%$*EorfJQLl}sJU|^!qEU7iALU2wqXCF4aK<{8_XWfoDErLVO}MJr ztTUzer@ivMJT+%#E9~d|ROUV-i#{V$*LKWWk;(p6YPDa-j9+%AueUz4w^Gc>!EYA-9tzcbp)qt@-DhyTpTgL7DtW3^aOwJ15 zdQTDFky32|WWwy1OMjRbY2bH=77Ef_QPM=?UR54*bKiLLD&@l3OXxM!O~g;H{v=708|4{QIu)JQUb@RhAq@UBnc6>_dZ=A*^Z0mItS1V??{q@fbu!;E4S zhGL}mYrMNgkx_OV|J_mi-LXaNj_rJN>xZ4s)Xm_|g=@8)EwG(E z-jN@-UQk3Wv2+N#hPR25e_uz~QCxb?5lEhS<|5B6q?l1v^>06+TmP$?&Y1Nk^w_)a zN8O&Cp*VL)vhb`kGBdA-cFPoG%6Sdk$rhM5w9u#NOX_~c1QWx8`u2F zN=#7+Wc_v%Fqjm4jQ8z{@i8Dg(L+;AGv*xD*-3tfaOs)I@T#Pb<>wnQsniFo(|hD= z->{x=l~dZ%JGoHM0_XN0>c@aM{&Wk+@3HOgQ&?W=eWzMyZxq`lrt)_FYTg<>;G8GQ zy(dISnOGd=Kk0yeO8=wRs4$e{z(`6mclte0f86WGsI121$exTw><%&eMv?2Qf@QrOCb6~BI0U@0 zU3F3h=l9w)CKD)naWqZI#=KQ+bcxb9D;`%pj;{FEkzKq5)!Ev`y$^eUUgAnTb6&?% zhCk2Yo6<%?>+OV(^^Z}p%DfSM>4f)F{QZ-b}CsH+53`ZWbUUpLfO8qfjRDAmw8pKV*k=cJ6QoFRn zhTkYgzAX6eNVV8z;hFhB5iU)pKh=$_<`PvTfE_(_TQPt9hO)?&AUPp>a@#j}(^A$? z6_#(5Q>2;VA}{F->1uRf)K!W&@ME;w^XjFH#B);&#pO zb1*G7A*zH0d-+G)U#&xO4i3UfqB}-X4xeYWf6Eoayl6xzsGOBkAHL&gaq$ZCrGzdFq5xg(@yX z>4|!L2$M#c8ZE#=?z~?CzNP1=Zueh1c*7f|);s&v=(*nv)7wJUC`x?hg8L$fV)?D$ zYZAUg7m0W}A+=z(p$26`K1X;Js>GI9=?CU}K7OjP#+ph)32WH9Q4xyKQL?c{CY|c)-SIfmAAeh9xsSZ~xiOMvA= z7|^4DTyc`W^e^sTy=ei;dZdCaIU_6+kPovWwCq+zckqx!b8ZTGsaQ&Y1%?-&!2{&P zX}tUkLD9C8Q5qk{Czv)}KpZnY&?8A)Skj%lKXSyQtVPLwz(z}CJwbNs7D-QP9XFAj z^VZWx0#v!}TJr2fU`;Xda(-A6_Eg-G^VI&FUaVqc*r~01^)-oEf)g&WP522xs+EM7 zEMYQ{;Wv8p>n*4vQ=8Z9<~Md~GMLd9Ud=z|n}Aby7|?erqF}Y~BCSk}&uWA; zd!VQUcuF-4(%?d2%(g;xPfo#*rw28TKs4uQ`G`u^3pUnt97eegBOMv54D_aI_|%eT zZVh@*1q~8(>lnUYGqE^KOErhrL8s}ajVbY>`|k0bbexS8y80Elp5J90*e;DAIr%q>uUbtJ(l^doDm*Vt8-wc(s1m_P zsHe3P@Liw@!eO_HrEN&$>9?o5h^g%pj3c-z(KQ=1{O^PiT=NIc;j4%JL@*kG?GuR5 z8jQTPAkR~lyTI2~`DYKXrFn^gmT=0ST&@QidA}nkU9>UZZR=qgUc*p;-SRk;w;x-?)QI1%AG{xccD;L z&lAf8dD|g|1t;7kVRhC#XOarDg@pLcpgJE3KK@Bw!r0jVMb%pX#o2prpbLw;YjO7i zOL2EA6lk%<-Q8_*FD}Ij#S2~BT?+-u;$GanxL^9+-<>=EnP;9&vXf-<&U;Rhv&lJz zuSowX*Ye4?ykyZ4i(v}&IW5)jvl|QVivJbzG21#nnNw0DH``+cIntLBeO{DHJ)esP z6`Qift5ic-r|Nq=`M}zqqp)YW*wc1$M4tXB`je~CDg80=TSspsM_y!&R63}z+y#O! zva}HSc;Uhl^XqZ+#xmUfU~l2iCxp+@j)h5dq3|6{9e263ze0HTJjGo3KSJF^IBm|d>iaUT zK`}m|liKRBfhDG&=zCR@ia?lIowEDcI;LDx`s8Q$%HB(Dsh#mnUzTQI_1w#3$*K1e zwQO!^R4-AJHqyIcL9 zf3p8l&flY5gh=)Z-jU4s!#7kK+TjRi=85_bT5R1wnrW*u6D~~W+iFy+^;rdF6|f0T zK4`6OT8~=D7F}w~qBx{XD}E-^ zffKdcB~)#=p!saT3#+21sjhd zI`FSw*vb|6%HI2^_sQAd-{|-b7gGexki*z$WxZlx-4We?TCVsq@)Zd&9@v*gzsQJ~ zfvVN0^W^i2FjZS~vw^5&*3#S^peA;#pS`Hy*fvmmLs^w|=0*xCr<8F@y^@h8&Ou!` zQgZN*mKHnQx_jHvGE4@&t$YUT()r#FIg~d;iJXifA1*>|jJvvNe6&zNp4()68R3mu z^1Tz5E(OkO&ild=2Pw?Y3Sp%&-gdiyoC(BWnh$`jAUtEm zY-gtC)@UoN_NbX$J1Qm^5=LqLoOobIAhqU-;zmULVLA=u zT)y{r=Ue?l3^KGJxFRm^$1=xfUPNfdvbq})Rc8_slrLF+z?;T7R_jiPt2rLYu@||` zhO)k{DWZJb?q(LnzA$(RxP*>2o7KUmWg`8jS8A!Q4SD&`7Ccb^73Old{i!_;4HoZF ze(=M&tM=74;kmaKnW4AN8$*Sccx(|RAod3E`!#3zGgcIiH++{P^Z}%>#zN+efsA@@ zOR1k~kJb>bO~0YKZmyB@X#R8EnF72f2maa*GPz3bi((YQA=-N;R)T)elG`2y>Xv^LRxD%2pq5KQDNw=wH13RXK0cFaQN zxqRl5e*BJqJJbX^BTSaDPEkAHn~B(2i&wCcVm6b_cbHw3;l%&HSo4Xi(f9(yd)oyj%cl`tBZ{KeFlf09`yv5Z*p3rnJv1>qr}4CSL-k7HFEs(F+5*U zxiX^|7Gg^vy~T=+E_Qu8donmxp&!5YnE6Le`W5RTrcmj|esq$7d}h0$Gr@?@=KnJjfO#^Aet3x=~_I0s~M2jR6Q#v)oX5X=G;FzHZ6 zJS1im1zcauvdlOT>e#?oYIT_H*6@I0VOF#Q-qp0&v|eCVnSUhb;rxy&?^>np|2s)C zoUFi}M_Pz<>)Sg`n}x7io4YtZiRYmW@qkf2jt@)}?mvI1#4{B9b+cKBtM!rf_+yuI zim>VXTpM1dA?uR6mWAgYB3n#UpfMxs8Di%-_%^c45@)8;#WoX&fXN-0`X{b7yzA)Q zP#Q;Bp@s5Dx_-y}J3llHCh2{gWO~H@8b<}Jd^r24CZ?=jXR=EUO-GVQOKEGY!F+MH z&cmPK8!M@YICy;pmZPvfR303e7L?#1KitpWC?;G?@ayX@80WorNoTPy7oL_Uw4FM2L^q0kiekNyURFQ)*z-JYd~}@loaDHOQfqgCb4dx`Fw<&{zN8kd!O+0c zQ-F^c+HO6i?QohPY*AoIrU7=tr(^UAuUGYK)SW7(Co0|~7qrFii!6JHleXl>LCHle z$w|HV_60{o8<#ucRXDv)GY8_y+HzI6c0HrInnEhEjODNsr{Rjefirae3T#9M9j+hQ zHg_VO%p=yX#9uPSJI_((pXNbkgm{nwnUXtXFiRnn`Qi2ylw*u;GU$v+Er2)ae3_%j zB4Fwf^h_-XVVZLXm&+4oLW@uDnYoDCcW)j7Y~;DKqD6Gt^u+u>_9epkT-{#=k{9?K z-}>>-%_F@HNcX$UtXWbxW``~t{C1!l8EMgdEOX`0`n{SgCQNFaxt*lu=Lz{_Fzn0& z*S~rC21gX`Ik(W52lXH!5!0DZ>OQe!y)$5~lYkVx3txc^oRIRR2s2M1q=7|4c2O3b z4(O6lMa&pLH1^vYYx=ry9xx}htF`@G^fLZ zY4zGjVFxsY9}6WbP7R8t@?6zd#^5n!NlG8#6b8AKnrJsD%6^L z3k^8smRK>63a-&eJS1!O6)g(@CLYQ)JCR!l(JlM%Ca!$~Y`pW{c>mU!98$gVuI1Z6 zqMZ>CYzAP*SgtT{ZpHw&wY0TNRl5N6`B9dmb@N=I%c0#qmZ*il1Z5PO83H_c^25Dj zN_`NZ;$a&t>O{{-1vxEePU^_ZK<6tOlyz*NaYA>%c(U%9aE1sHk~mzJII5b6bb9Yy z+Lj=0vu{m|HcR&p4Tyd*ph*b{D`9aRB{-W^YDiKSFO2e5KLkc6HIGe`qbx#-az)}u zo9!mJM^qSkqK$?efC%lxI=MhvzF}N`jTx|)?fe=5_ld$8=CvM;4T!_;krb~z&POKN z%+DT@Y^&)ozv{rg?bgBg(YyTE6r^~H2`gX$1DtO{^tn$$KVvCpb(Xa@#EJ|lc)>o?-FTv5a>t9j1vS192p9l?NW_f4@;G2#i` z$M(jgCLA>`yQL|LApPEJ|0QIB2eHlq&2oewMpctJeb7E4$#kk#{m9)_jx#3Lrf7b_ zruny9x~Un&bTil#A7Y9hY+CoqaJ9n{HRTfAZ!^B25@aJ1WIJS=5)(_;OwsfVOV;qJ zvUu9yW^PN@pykig=IijO+jFbiBd#@&FcTrfa3HqZ2Ahr)Y-j@pJ}s8a4}@zTp}AT> z8UM1<4M15W@rRGl9`(>47jrQzuc`aKU_$07yK6km@$hMRzg%_Ky#0c_Taih$q`GxA z07}x9=*%pPZvT~P#GjrYZEh+74G0FKIuFqcY{3CO;i1f8VYQHUFTx`%QN8&Ka2^6W z@F&d?|B?A4a%aBJlOUnr-V#{x!s{zVZE6R`u_p6Wq4o#`&iY1^UaFuKjur{y- z3g>EImknM*XPOJxdIoR#b}K-Ul_jD25O5Q)fH>^=;n zo&y>WU@dL=7LmBF$Wy-lMr4)YD;<`hD}lsA7V5bv38)nWTm{e>2ejk?T{{5A9VlDA zq(fI|iH9HmRdaR~Rm}S(3oVV`@@*v8@+BX-qD(vtpq@jPfO0{=Q~*ONxE3)u*C*t& z`!_>Zh&t!R4z3;Q7%h49Tx2jZhJ=sP!`3c5dTlKuEBhbNT)gj!rQ}(D4GZv4DD@a= zISW1NmjjReG=4u~10GO^u6Rk2iPVBu_eZM$^QY!;7s%!2&y2CJ5e(8l#A{5m=M_B+PCeZ4>a#CAvMd5gz1;l(wVVY{J#1iz^}5t{04Iui8`=un$ApautQDk z?XmtnPuv5HWL+!|wf_M1sYng3EpJ&x^4ll1bxX`m0@|I0CiBun=#hK36Ve(#K8ZG5 zTsCT4HfCHl`t2BSkON`xBVd!&;fAA&wZ!R8>~68%DH_5t1t@Wwcp(bbr3_FevbV|z z&X`R~I&xiGhC=3CdEyNeEvsMUfmNgDMC$vRKW1C@ap$&n_-i9{tGi^Kw>=6qFJk~L zUjVoZ02XZ~<>3^)&9=uhs`i5$$by)TNz}YmXGbDH2byz8)S#%_J3%T5nTdcg0}h%; zWk z6@vEr7;iqH(p21=?BsYeNah?w$BDE?h{IGbnY)u#;*e$6R`iD(-!;!PKA^gy;J7_~)VvD0+Y!=uVL3G87z{rCL}=aO$`@@eN(e(@&L0CM+xrqFBWCi9 z41HzPMW!1TQbpEjb@SpjfU7!Lb{9PHB6h6*{BW0e#7Vu-AyNAPF~0#0Qz9p?U}qi! zl0^~d{Yc*WVJKV+Bp#u*(T73|5QgC)38cgRtjQj48FWu+B}O96$AQC!@X1f;3fDY| zM-G4^3+jamiP|~1YJP~h6Ws70q~s2>40Bp*#N^7a!#cXH$kj3tm0d~ZK=iSfwj|ud zJ}jc3-H@Z`vr{mI4i&IZ3^5`=E~5Ht3hvPjV1Bo|mW$>w2$S?UDA;0E^7e%i@F0me zi-%arfTp8NP@T?Jafp|9599}XPGyMQF$}iN(<_Jpvl*k27qQ@{-=Mk9mmav0|+3~>a&|r4o z4Sb`c2onFb2sW$;Hnf8nwgEb>;p;u&e233_Cqe>l;PVf> zw4r=2oc(Pff<$2qYfk~vQ=wp$@KsVM9b8M{u+$uP(Jv2?@`#HDDE`M#^AWh9h!HQyEHm1T(@UV6 zPi?xHuF(AdhhOvF9Y@yz zgko6Qw*qth$A=b8R8SW`mu3G(%jM<%{cKfdtXP3@HiTWw;@7BL-FK?pcdL*KeaOYn z|5zAu;Rm^RfLzG`SHdK_&VCo*n8VOjrG3{y+Ui@C@YZkFe(#3+-Q|yB>%Fd!h-dQd z3;n3VRY~a=a^ri7<|k|z>;-#cPYtqP^_oJ$d^&9;zi%`pyGJq0p@(88KV%{e%bEUNgK_V55D6X;CqbvpY)(u{{Z@iLkv|RpLk&vhqowWA#dc_vIH?vG}b{ovqf&lf^T1hs~UsMed@Lo?>?01KBVqGG+sKT z`OUE-+8;gY^Izp()fceRGOTngIp=y87 ztyhoX+Q-1lfmGMq~fF;s)^$b<70MZfgL+Pfly9SQ=D5cB7h zC8~npUMH#!L!fVmV_S540g11{@Upew1czw7d&4$tQuRG1NBIZ8j7!%bKvV$A<$Y9~ zSCDK;?_ zxpsfWlFldUmQu|(ONv&T@Wq*|Ew7A<3KlMR3^IGY`c`|x4{_fF=sTj2_F^*7HM>C8 z)OnJ~-!r<=4YNBEPz$90n;V+X8w;012V`2>zC^mpwfJ!qF847+xh{hyW3y4C#^8x_ zRR`TZnzKIsUfY2;y%?5qgmZ@e_YfSDLO-1J&=$USYWA}^vkXE6u4zH z=Y=$Y2_23j!>SeMh*EY#iyD;XaSN939elze|L4xazWvr`O2JNpO z$l{nlbMkI^ed=EgaGQG6zhnWm0T7oDQLZjgt`0cc*B)(9{ zk=k2_yg!b3ccj@6f*!uc9V2nx|DGwz2p3mE&Lp`Gmy(Zi6hwShmIlQm0KTy)Se+0AO~`g10u-5C!WYQ%qR~woI_7Sl^L}E4@%PNW9?igUXvMd zsRpj8gSf~WMA~%q>qeoeYGJ>cGtHXcjHMSB{N2l6jAz@GX&0e2?55w5mWkuKNxg_7 z=>h_?9)>9g%?L2JGq$w7+ z^Mz!MoTjg-mrTrsvmUL-5wcc#M=d~*nuV~9fw;7@kK>Pv;~#w)xmvKy2KiF~!)QkW z`vGBoDWjWzked9FVSdC}!Cv{K-fH1%Xox+8Xd!F_2-?gjP<0$I%}HZlFry7ZsWr}< zFqHR`G}cmGTo|_=X`*~(S6UTgF_~N!F0*iJsD%X(ICG)L($`y{)^N|jwE%eTV>jn% zF6xp1#X(mip!W;IRLkj*%88%zG}UV%Hutf7S{)MMS_qo6Bw};^WFV5o8WQtwp--EY zPq@&DbbUNUou@g-41ycfrdq?#&S+S_izUzQLID|at=RBWKIE)zd6Ns`X0xZ2lhX0d={NU|m>n1l(ip6vVf#$h zbj;RrjD05|2w$ItCL0g1AhuoD-B)S#X-$+o6Yia&SUZ#~@%2sK&ocxVIn~0Z6105p zf&M@Rj>lO)r0>-B>i-vsE1qHS>+YCgK%CxkG89ai$LPBSK)#yiq7rGh5xcNK*Y+l~ zL*h}rrjQT!dCDGlpY*-2j~twF;jsER3C}WcMPuBXXM#6cPV~CIsh+o#>5JVEyyorD{=@2JQBvZs#TUCb2wWw_w4Z_JRcw0Hs z8cO{6_V3}za`XL!RJ8W(DEeNIc=`D-*G{m+d>BPtd*Lsiu(392p&`U2SI7@aV>`h7 zCpddMK%)VmQ4Zq3(cLKA-AH7-)W>PR;hO-mZ->uL#C+F;^=|g(z1tm~@{h+Pj=IAH z)wKk2KsC|X&-k?jzfkcLVE#Zi)#h4)!&-vN+0TI33wb<_X{v?yzG1CfpLOiDmBU(> zWUYCr>#3;g#i`u^)b2Rc?r*8xNvPcgsNE6%<1KNbnt9F^I-(`Mgs7P*$9^G7XCYMZ zskw89CZKSt`|3UNoG?20)ZV$nt^4XX(wy+ki*!HyysI2L(B2ZaB4Z`WgS*?4wA)j# z+Y_^~PQ9bD1Zvxt$y@hW#-vR7j*hZvTJ9A*dz zdqza*z(DDM|Bv-Bq>h}KXg_@w0aqDOLiu~4haHFl9q7;-%H}^0kcr4OU#j54CuH!c zrgO)5_mvpp+(SFbTuP1W?<={QLd@#B1~aoGlH>XLnu8i{Pw=^!8J^*4VrSlIu(r5F zJVuBw7Wul$p6-0>#nYhyGDFi@!ik<`RGa)+E~hO!+KG_rzAi# zj56)9m)oM|%vw7rhSY`LR9}W3h6y_3R5pbZ1^wlDr~--4b7R8xOxtkAjIDMs)auy# zrru<)N1saio;&BBVv=0Z^xXxTzgc}f0VKnC5jw0Ah++)N)H;)n#aNpoe$%~sh^yDl zIU21V&LH*C6PFB#5|13l2#~^uD;S2r2qdh9s6jxSEL4;+K$s;VVYiFbg%qxysZ1H; z9E?{#N&Z*M^bU--EAeuTP2rz%<)SsawN>}!HYTL!y(Qzxj_e+tOHv9HERR^#3zYp3 z4DPKQnocn=b?!m5&qeiLyjH{hs{7E?etWGJ;zSH_(uVwFj_xL5>kj1ZCakq3^ff#9 z+bp=wx7~9XXAafqM(fAaZT&E;**Z+YC%3fXAZX3u*AHjo0)Sbh=N7J8E4)Wu#C!*& zmu2|p7Q#n2^xNG)S-EO?wCnMeIF#+r#3Jw z!2pN@0kO7kteLz|?_l z^2(4oQ-?eb#Q{V;t>#E(H;J+zRqV^a*12p&cUK?#Bssy2Ain&%Wt)QcpxPflrKY<> zezm>ADI_G8KNfaZG7YgmE7n4S&@!*+QU9t{=%lUYSvk`K!*`u6QIW}tmik<>uANDw z-pP%B$81$(h|){t3uGF^exbG9Zx>Bp|EM;!B>wU3A?md42y%_R&qHitQx`_PFXG>8 zZ5pQ_3hOJRtz#$2KhN)A2NXj~s}b!_E3_BaRa!o6^219$Bke22k<7y-uAUByyf{~Y zsK5{HRmc+c3yyO;W{^Ei$R00b&keGt4B2CYyvDWI(}(N{|HlusH1~DgP|S)*Yw?s} zI^U@4aUP#AQ5dI87eJ;9C=&#b34)giN{Lp#CFqeYL3QMjU;8I3^xiRHLnlSJ*T&b0 zIiEePk-nXrxTi#Zo~57`S}rL>O}!)(N-U6)=aJfzkwY~Ae1pD*(t?yRNoZl=Tp>jH z5+dV*+`?t!aUYf?H>?!Nn!+JupmZj^R2d;u~8=8?{n zw?s!D1{Z{Ap`{g(1=@vw|CU^~N%8*Md(-&U{t5ge0_Big%M4>aiE0ag4h4?AAdWo) zP9>#;J|==b5Mc=sr4b3G(ar_IY>+^unl~P0+f&jm>~~`0^sdTQXb@5jG*)e+fqLkaArM79-$>%1Yj{WAqvbX)Tq z4?XPwO#4#$pAOJkxoQy|`O(FrtR$!C%_Cwnvq{B!#E0t?nmFi>sjH+9RiVgB6pEh3ACl zgwDs^j3E)9AhFfc(W6nF?0Yv=@mx+_tgBmZ^q<+p!pcvfk&%4Wh(c!MxHyEcZer{0 zSTvng(~c}%?rd3W)F6XKEyc8z-pgwG{m%-lW{oHE_p{va*Z7d1sPkNxQ2a^Fo3_iD z+|=hdX^$(ex;GAYs@QdsSpNr7SuffbRx)FeqCT@ewUrYhUo06paC1e=L-&fgURyy{FYxkb4?+#BzY&Kpo|Sl0Zu{flNF z+X0&;S^}eWcEjgQU1o251Np)M=#@QAlx>q?i|e6!&L~p=YtzPZ{q0 zAaZI0E45Eqv#h&m7Z&$ZVrSspV<^I`fT?s}?!m*eO^9W=1PACv3afDWQsu^#lV0u@ z;6BkjPKH0)HIyrUi#W#@udYMF%5rc%8c(O;creMNk$lAemKRorRQO66*}GHOCZhKy zu?b+b0Drv#U>@2G4G5<|5(QJvnpvt3hb$V#941ExLMz~QeD{q{m9uV~BK%E8;kEV~ zF;9;H$r)8+tLBaTuhfe7ipj;|+OAi-!603sgHi>iClJewS_bqUy*RlhB$vs!!sJL z9tqhkiyM-!wT5TTiMpCcXR1oaW>D4K8bZLLGEhoJ?S#aNw8S5@+G>dvGh4QpEqbcZ z+13X3+deC&6t}C=4p=!+kTA;K-?33~CH@MScqa|MM-{={wyxNybs%eo(FX}n zN}LaHc1lm|r+7y=hfOWhtF_#-tNwKE0`gcp0$=1WFhg779pBV1qx=Z^)O&Tvp!X6& za>b?e{rMy9H7)KEWNxGq=bd6rAi80dQcGaP!1tVvE#!J#kGfargP2OZqeAo+kd8)V_sJ?)3wjX2eZF*th*r#egzPc0r@=m)DP z=>^Nj0vT--j7g9AjDrLtHy=-~)$mGQWNELuG%&jqE1$i{P85ZPjI;(=zHd13ILBc9 zO)zajw{J5rUOI+9bc|f^6rOcyDjcs~a`2}(Wv}+5jytsx=RdWZUJHJUgF8Y8mA^iTSvsVj1h0T6W+IWDQTlrk z=_LJ9BIoOqh`{*4Rubw|FtX<^;@}?B5cQst@;Lgv*aQo;kcCVsWRCzZm-d^cin~(u zd(w&hBOW14hu5RzoQ+~a#hn(P$MKR2I1BM@qu*P#?YW+sf|Ya1r3EJ= z*0}}mACY7zyR=|ru>H$cEW6c!M;x<8B#eGyGcC1Yd}_sTaLj0F9_`!q1@COSvT^)5NA9%`w+a6?&lF;vq;nk zU9(8ch|&$%f84XrA}3T?{&1SFsAqdL;lnw_>NM);P|*iY#*z7@TC)xgGg0mGu-=CB z=qE+Sng*2vHQq1cz==8)Q?9;?%2e{a3}^&#QR%wuzQg&w%6Y>c&o8651HyF$iT^@gZqOOW#GJL=9| zzMST=SWJ)+Ty0JhtV(!nAH$O|g`x~M#o`V7PTyfqdQD#19p>zuae#pa2`{EowU}>8 zLk~H<6H{edQE#>OcM>}cr@mHwB$DePDi#wzLAF5RT}gY@n&@50H(49(MnApO z=hcYjm~D1I7v{JG)a#PI(O5hRQtFBH$g)a8W}nofwh+P<1J#=ekYp8bmqtRqErZ=L z9sioUgxyiQ<%d#sX zH5M1u<4v!THi+7(3d>%2*+v3S!F^=HXv+rQ_ts{gNF$YD(+I`z|(W)Iny))Bc2( zCpLN%uS)e(LV48arc%ZXnaob0H6>(6lJ){8Nf&cOT4o2Awhk}-qs$H}ZC%z-<++~u z?7bKZ)+zpFz%QxkUz|dRsWRgrd3S6%^Ol4 zZSrMNbUF#Z<~wLdZd9$*7;aW90%SjlYSeYmvgt5!_eh#zV5J|e;n!FdxhW1t4 zuo#5uD_S_GYSnbS6Ih&KtF^D#3G6%-@RKDn4zfIdPQz#u53-V4 zTehDGR3oRs-P^teaq+w_IR%1cI#PtR*F7b%w91L|F{ll4nt^$PHa&2lggL8^h;jJd z4H-^2^XIt6&s9fC`RKd;P3Dm@dMfNYgkj@R^gMdRo?RteGu<;E^{-qrJ-kKOW(H>j zWn78tJyYgdXRY@Hp?{GFevTI)~IBHy(2-sri9v+bj4PZ2^3*hWj(*z?&&87JNL(dH%ycP0oO z^V!&`XVFDZ(BxP4(N=Gt*Z$H!*oYS%JQN}^ErdGJrFqeMW^5sFrF-^_6w{fNt`$eE z(5-}IC?>2uDDJvWwn#lGdg`Mc`UhQ$o@~)Sd`M$j2y&r*oC$`Vq=_p@MJ|yB1Wm1_ ziTAJ#uWU|VpJZ6qxQuE&Bu$juL(pK4}RXAruVoP4;d;# zR1+xTG?`y{x`GCa+!F7ely`2bKwnl9TKT(zHh0fZ^&;<{)bV${#Dh3?Zs-!Jijm)# z1d7XH-$vg(k%+EYl_lV>bVZ%G9K2|kslDxb`Th9Jbn##`dRhv~`>&OU{l`nFSl5x2 zbLQ$(i0I>^E6Rm~0;E3@LCG}geH;zaFbZn&QKI@VCTdDZqVxWc4$-8>({9F7?_+QF z<(-B2gZ$isZ1Xdy{ia57OcJd%q!JBs4W z?NnwUE~^I4er5=Xnr7nI@7<)3#-e-*;je0FX!GZvi}Ir=xEK*-i}F*nHF0){PxH^) zJ{I zv=Yf%Eiws}cnQ2Q>fuu1c+(w)bKVvV`R`@|lcJbE zC3?BToG}K>NY+Ut1>`okBb;+&1qL@hQqTgE-$>@rdd!wjf2E-rSTq;K z^jk=;TU$}{;0nJ|d*-y+itEwBE;a^Ub3)pLlnkb#x8^?x!bKDf^ODg9zbAS?Pu@0_Yw4fnY_N4@m55Xh zx?-mRKX4IFsX3KSsZpun*X8xM%b31P%#5eE#22edB*B`OYD~^_ptod;_W6Mty}!GD zskuA1I^!YJ1^0_@A>Xp|Gs$1Zk@_Zjp)H5x3oM=ohFoqo^MtLic6r}FOrj481y}2`OsT#eYZjk^ zq)AQ?THNepbd3n86;;&oTV&lNeU}@B_{zFhUU7-LCyn4LH@)UEYV_A$ddY86@px2O zamq0y(wYo(Aj}LEsbpD&T9;v3dcgzf4c=+1%!|Z%vGETtMwq)@u9W0d52e!&su$TO z2k;%M;lB(;3h%gJkwgmK*7DI<9Prp^K=xU6!R3dE8Yv@7LC3DV5}f;NU|SlIF&aZH3^%pcR5GuV<5 zlp*bg2%3|6C2dJJKG2*5FL9P#LCbGD5V{tOU2Af=2yQGMwhTW;YwZ}m^SVq7-|@6Q z3ESbcPM31y2i%^YRy|alQZBLqYLF{V z*%jH)*(`?5TlP0nH)7VzDNG$2Do=^BTBozvoJ>J>IIKOT>yWd&3hqc=$%iTnJAVOr zOdjTsF~($AsC&}X8*CYJwKz?}HU{(@FOR#T0?|^xlQ<7k8W~3Gip=s=WKHX8*NTG(74nJbD zXeKcOYNb!J*?~41B=fjoYd#tkh6F5+pD)yDp9P@Asx(ek%?cJ&bx{k@zGbam--`Nu zuOEdLg$6!tYSf~th@Hs}!oHu(Khnu16|pwxo5fno4iZftL%KLE$lbovWE>IrH0tC* zWk;IU6{$JNcJ9NidI)3X@r1i20kP zamcZ{V=1(?xj>o|3*mUY&S6Z(N9B&c9zG$yiZHJBSn)K2#h_LH@>ocXN zVgHoer$PU!G1v`Vd-3SS*>P{pSxo(OGlyq&Xeq$lzOilYwUU-r{~c&g`egW_O>D`?Wt>3|codqLs11-<|g(l?gdT zUG~G3oxt}Sy_0m6zeS0azL#?@eQZ8bZ*5;#=9G3J4IWVd%dNX&aTYq0hK}ffe-B!~ zbMCJ3OxSU@X^l2GmFL*4P8dTm+awmlR8wRIr>kSvvS?nd5GKURit`MU3`e%*=9O4& zDz?3x8Lic9=En2vq@&kKCUebvbDtZ=8V>E$M!m>KMl}k`-gV6=Td5;~#!2-TW29D{H)6;}l@WLH;lf%^#db zVZve~)5*iQ!8O65a(kv8>Fx&(`}H!g(-95nO0ijEMo=n})z5=5aL*xGqo?tVl_9a- zJLDNIz{6W$7W<|Ew>X7Gj)fMZdxuO|2V~Kg2tux~BlkMFaU5CTwJ6?)IWQ$<>pVK7 zo0wGn(lyc7eeB@GpAQV2$BI2jOD{gw%0Bo9eW}uWK9NP!@2gF-VehYm%~kibx=z_q z`GT%QjS%%Zg+{f70 z5S@K&ijC-y_FrY_JdNf)3Rvue3q(CtcAQ6^d}l8{@uVKpCsv^PCh>2z+KPy?yH8u0 zw0}ZH{++DiMK%)1NZJi0>oot4GHVm&k3EKK9SGTxw+@W>x9NFRXz}Bfo&^f5 z9$miQx%mFwoIcyU{JZxP)yqKrnIlbHN5+jOz}1KR}E#Tt*mkT#(qah^0uLQnLd}4 zt#?FiDeKQMGUVwE{yL5%s)a-f4k+QINY#10oU_T?5d(v20&(v8tCbx;V-Zktjk z|MC-_9L*wZ^b;oIAWo~0c`+w+QT{N<(a^H#$>gFhwVaOE)E2!I+JEf0(TFkm5ViEw ze+;KXMem61!{N9O#cqA#ou+Vqu(EN6Q9L73RulddzjIJZRHyuQm*V#6x4)}{PZX>#2COfltSet>NH>{mvxvL2dNW;HjA27cQ>ZWZ zYd(E~=kxJyW1It&E#i5_;=>c6^vf^H*M?T52a@#DYM=eyy(H-?7lt)$%@B}~gFzT_)fmJq`j0Mq(jLK9gv$Qnt47cUDAh7n?k0+KEug&R|_mh*|X zOhhBwPC0mrw3Xnc%!-DogzSK=Im2G*ccdj6NVDJz)>t?Uyv3Kb&T1{kY$InQ;U@UH z^x;L%swnt^JvIXyjj-hWw;&P)T@g4{Rgt6S~a2aQPM3q*|HT0veW`gNl7=TbgV920#YlDq|)!*ufO;Gz5n-|_nbMObMBs*xp(i(Jo9{>`OI9f zJ(nZ@co@nI+#QR03Q~n#P(fLd*Y^;D;E)IuH8><1C009bUK*itaeVKN7X*8Th3Fwi zVB7FAs=L~yJQ?_5+&$5p(L>pcpR#5gfNq*fCZIq`gncFi1x1xPPHTKMQU=P@EGzSTk1$+#1#L<=kVIjSuN^zmq zKy|ed_T3QsaBR>f@>5QAyP^Rez-D<0D=tIv3=&iDBmgDA+NPaBf68Hc z%dG361=t`>SQ`%14*7!s!HPCF$f__D2AClRMFwU#qeXZCRsPZX>-;-E-q;!Dj}P@m zN)aKj!TB+$m*D(Zdufn>d;rDi}+0|Rgx55Jb+>%gYXe8X!8sCI0~f+?hXf9m>B5f)%Z}(4A3qJ zc0mR$D11hY5J2Ys*UphBU$BnCKb>yT8T7rpwdijZ1a)EGUG!*`bCoQdoW3wE3b%_* zwGjf$OmR%Z@wlnianHFE!Q9F&0ts=y7pVHF4bAm}#;2+JiD=FB664a~aSvR`OPC_= z_*o1jgpHNhG+S`frj+0su1Ybdc2}^ts=GHPlvc(^WJ%gQvPE?-Ba^{O8)<#_)orzs zwzz1>lp%vJNcEnuljXrRmBawP^qxz(yRgmTMDJ!j+kzxC> z5z}q0m~Y9SqKT`Yi&+|nswE`F6IXiMH|i&HJi(+HGU-ad53zyO6Tantw?^n{c!JBY z87<-;B}~tgkA9#O442A}oPAKm0U1;m*DgEe)Vuhlcab){PRSEWn<1Mw(u3?xuIXI( zk#U##ZY0#qZu4-a=vYAY3ZLiB1MZMIQdx2RQq4+n^Lyzr_4A8{it9EtE7i^K#m8@8 z=pP8~WLL;G?`qy{eh(ZOSy{XuEr?s>x|L+^)=VTzV74SavK~%)!Q;w$Q20Ab-M#PT zxj@!&9F?Wn?cT}CVLf+9IsBrvEoV)`oMCVi54z?AUGt>8aB`W^-_jkYoPB-rr2qD@ z_L2V|XwhAnw%faciVsg@Oxhp2xMgLz&nJ#PG`H-xb#_(89FhvLB#=a||#6^3^vMH2MzO~9IoLS$ptGjZqi;6aj z+tlA`jPdVXnDJ7(A&+UEY$v`N%fTb|y7{A>EuRF%;WXaaKF1_Bn z!Aa?@xjSm2<#I9_Zuiu22rUh!R#jtyR_go2JeKtmkdS=4GKWU;kZpy1V1Zg%rI^!P z>9xdSx|KuT@&0l-Mt*{r)AgHxH=w$L=XxGJyv1j#!m7MNIp9sQ>Sm=%`tcp5#s#wK zpa+tqe78!rg6Qkca~R=Wk6~-Be)~1RSS;Kt`bu#4japB8OYpPuTTj21i17!zZ^tI& zkc@!aHN#qErtp5dyO&e^to%}_!xqbx+RK(H$nC*KC5~U4`XVVaVn|S28}coYD%Gaq ztGhnMGfO8{P7SMOl6O+G4T|);YaJKF-uJJKo&qO%&NzONB0_CH)s6JW9;5D+!bEAl z>ZU|FqfPcxD~ywjr&dHKB2T}cr}*1>3e%aJ_{*gfjbJK3J4SvPY!WbcrSkN|^YjpW z%PBuLW9rJ_=}F@0p_u6?Kkm1zJbp@bKT(U z2}O_UTT{(z2Idgn_!XPJL-)0on7`HRA-cDrmB(tcyX|3IcGpEYk|EzIdZ?L$ z`>9d`5ktNuY{$~u4Z*e>;V6l5idU!72ix%sSJXFH z<*kAsIe#0loaf3T+6$`G>xV0fO2?8c7s9F6#Ou9E$6Tq`;cB-^Y@xsi?wTQ+G)i}h znP9I3dfw+vO8(IG<775fthM80Tjlr-rKXf?E-UUt{1kMz+wL^}_za_sUsFWauA+Go zW?1}{O@haqoW`J-<>$L};91MLxZ|DpEQ%m-NSy?aic`@VoS(-MTZR2UBoY#x?G@*q z4wd3J>d0corflUmDp(p(^8usORP5CJ3`M)2K}a(+*(A}+YKCSk^Z9BhieUwo`Bb%s z{?&$(AHlDcHL`RN`x97MHLYH~C6fG2tzLsbubXZ??E|`7Q|svwRI?&Y*+REN*&@4A z*&+*NF_odgWf{rN?eysE+2gX;kCI?As%ygXXs}JnbklRtE3hdMb?QWeZMo-dfkhT^ z`P-Z8!?2of`FHe4i=1Dfs@ka^${no}q5SiD)EbNBym3p>K`S(k-I{RcsglO-QMB%& zXL5`4{f>BR^^$oOVW6*A);2jYiY#tXeYFSJJi?D}5>_1_H7Gd+m|A>zT zz<%sGL2u+wL_>QSz{}An>DqBZBY9yhbzuceXfo{CJwzEY2pho+mWf0WfV-nmytgrL z3ZFed_yRQojHMBRdO`$#FYoynYK103J%xawK;}lFNN!^ckk>>ASj)(%N z&Az~DwKw^asldPtwz@0gHKC2$?kmSd2}69dSatVZW3&@ClWJLc6BAg}ZEp%KOCQ*J}d#rs7-(Z5(mkJ%F%vRCkVX6wn^@wlFsIR|TgIt3B#!f!hZWSlK$%GpnMsZp^dVD6x) zH)Aq;Ft0!KP~L~m)>qDl(^iwo41a#<4r|_4zBv$e&#v)ddcMM=S~PSt$#jt*z-2;)$McSgPz% z^@*}JQ`NZw7bgqWYY(YF3F)8~N4W+9s4r}u2*CibTYD1Fp?m{1pypkW0!@A~=M3^K zhqW1~H@v`vMk0fV5Msz60)X9iN24Uc#EPB|pp1pjuo1d|=JzoI<;IQ>A_9lRqIAKx z5vX*y%9WxgJ~R~W+Nk!rF_}{JN*!-;v=|f6OaZ6jDg(p|E#Qz)`dZw}F411{sMt)N zLI7Cs98X0y#&FO=#ew#A1eQB-jF3`rxW5u6?d|NA7gz{39L2#15Z?O2{|N6nazoMJ zrwBBI2W-sk$S@mL`T+V3=@nl37@ChvSDd7R9v}@B42ZOK!A_kj4W?KKT6RPlAnM8s zlS1W@gfaGnpvXcRe8gL@ikvVlR1lfFDFXH80p}Y}NL6AWl)&F2Q9)>~D+o#}1hiZb zxKvRX6Pi%i)(0@XO4!RW?%hLEHyFRPZdG1&aa{ zx4Y}eFQ!@Qx4E_{~Obr|1sU#aTjCx24(dIBTOjpw=ob-;2F6S zDGvor$b&5i(AxTt&s7SyB=V7dI@W`z@4&|)FLa&^Ey@+X$ZOtJK|VH#0h2_*5C|;=3($6g z7!#r@oY8^--0$%+$(yqVqXLppiU4UBW5W)bxCRO8A^YS2C+0DT1H1-v#MEX1sUzWX zHW3_v8&L2OcA=$y5{(UdhHQ*3r3dX7a)*~n1D*_}!Xz0~2koWqh(HC5kU=Hz3l&VL z8}c-)6d&pY-hoxblww1d(Y_1`R04b%n9T7r1yeh4w>q(x4t#y*`^_(^U^DY)3>_^+ z(e(^YkO}3h5Xp!PUf588u1&fdV*Ah$q`?`Yk!H8&Jm1otsmCo!Sdj{$7sk&UpWwD< z5h^Y*8+)lw!&YE**Yq&i50c^Zi+FCY#dxni6TORk}*mypy02DWerL?EO;zRaUIBw2J$pxFN zBPz}UdrgbQhn=hgf9R!U!kjzs`ev=LuS+oy-pH|76gK#woTns+0MJ(s>Wy^c1D z&!z8A7eANAMqjRqd^F(EhX=9Rf9|1$Z#l;0N22_}Gl~r~&@*HwF5qK{DxBa0?(+N^ zA5@NU78E{`WsQ9a;0Go&@1I8{gI*U{4o8EC!haw#|4RgV5B>Bu*gGPy3reT}>=`a1 z3;8%4?Nr5}oho_HM}Wut_>5K?Eg)1%U@8I`v4Cmgni!$3o^%$DVgrRL04UOPVb20A z( zkpE9K_5fHe?SVEv=n12ZLE&&@j4V|Su{HOb=B@=Hei1!KJ?urRYn7E|s(rK!;RM1E59= zToZ)?L7xM!7LeS+QKDeCSQIh%AR28V?D0UF z7xw?SL8r>3JeGDLq*M(P8e8}h@1G{(h!#fh05ECDeA7$}Ei3dNa0cZWBGb7XM?f%t zJZM(o`~w6paxV-JMw;as08J#QVIO@-R6FjaCvQ*#-~Nb!a04$#qJqKa3IK8h76MB? zvB;o4G0Fd^0ZIgVzys#V02$FDgD?@kXnprJI3ye`j1&S6vrSOLI`aBHqSmr_8;~AU z)Qi6?LFz)~Pe?#_?*w(&M0C*T@6HEz%`KoDxX^UZ)j>zRIQ*tfde^~@nc=I6?ew{(d=))OfH}ke5n3m~L3n_T)08Ing!87tF zgrM*8z&s+La6%5E118{je5hsyNLU00s9&=%e|jj;)-m>ypf_?26i^G`OUpo0p92~W zJ{8D>$I%`b=8Vwf{}D$p5g7kCAhE)U2xvv_qL?2uzoC=W2| zKPgS&pGb;{FbCHF8kiy=ast!@7>x>Gw;0qz@I(0qO0-Cd{ZAw%N9&_4vtZ*2m_IO` zMqz_^Vg9%P7Wh8^`~pBE)%Yip0$2d(qg^TuDp&|pc7y>SppqaGkev?ztZzoEXjz5+ zSWpDgCEA`2WEuEg{^uk>iu5L=$}j+?0pPqCpr>8nLZy+NI0#EHfc}aAg&c7HE{F># zjwmd4geEOoKXd-4pS_nq;RCJw!+U^L2!cD$A4h&p;sIM4PjnVxpt%Mf!U~xSnG(H4 zt7?pniz@I>Ny(HgM`-|)XgON&B)s2J4)ZrsZ>YWi7N^zxa^A+fP3kAQn*Jk>t}KO) zMA{&bJF-u4z3JDZ2i-Lh_8kxu5fS*W)goLm4Ff>?3#6?=8j&_X$RF7l%fU!nD?P~c zwhSL?p#g0XfdPC&7-RwCjs=v_O|%Lw3CchkDo#@T%h!SrmDYg%0l_LF(P)nrN*{r- z6@jpTRkTnF0)*249`g^tWaQKvtS|v39Dw@^q*e?t@HrHPsi2O4QVPUFQvZ9x5ICSH zvm-Ewz%>ylTClw$pp*hI5soqiqtLSe0qiAi(rmsB>ECQ%dgPVEBymE|{l4-IkD%-T zV+4TTSFwQ*8dwN8gZTemX{}-I;KjnWf9{*6XUUWH8GViZ_lu_?OZG&f-)$POb2#f> za#}1(@nyK?r|~q;HoshnGO(h^*_Ha{%QFEFk_7)GBbsQ82e~q9*RnKafc9Cm&47r2 zI9F8{9BWCkMP+E`c~GPrm9I25&tU-J)X$nh(f@yL3B0%!rsw?@$p9!b0HC~Ehxmhh z9F4*OheQFQDPTCk?-fr7K%B4(`u{Tj_N9{Ef=@D>mZ?Z@S;eb^nFk(N4ePvmL#_Z% zsS{_G(;)Yn5Rn(i!26=#f4&ffyjwG`f7ihX%n-7`4DrWCAw7z-Pt8l=iM~4o`U(Q5 z>`|g|>@1m4FYv87xOIOoc#?SW9E419ABIdhqVp4jTnhosJP@n_b0pEm07avva|#eU z(r>>Fr~RG<3{yL?3F%Y~l5mIy`p1-kMOA1;(}*?ZY+l8P2OOzy_w0J;>#PM|F?ofg zkulcuCR?`C^xUXh$oCf?kectvok}zn7G zhE3eWiy{zl^^uYziyl-d{hK!xd!>GwRcZ}EOM-sYx1;GeA89k{Ly!g@k#wMK!}(s6 zBYc}B(H_|#Fkt^sl-kCVTU1zmV)Jyba%;X}Yu=<|XTBM9)=*O6{#|C!=(~?rnjY56 zdIO;}726Z04vc!N6uziE;-QBMJ`}bsa?QB)i_9IE^^$i4^&l!D-X-prr4NamX)OP& zH|54OMZ}yvB67yECz<%fQ2JUrVxEU~#s*Y1c$W~9QO#ogOsDG@^lnaKC_j4d8Oym8 zSs+%;ulP9vr9JjmQPMhXr9CQ^=8K-V7m`%A`*{#5uu?O9y%_T%-aP-%xk58ReGBs< z(LBS@a;%GhE#oz{Z8Xo2gakY9Ox)f*+XBUNEL$9ANy2&gq3(!1x}omqy}%)2MW2lx zCf@Okb~av54d9bDin&qiDL2^jctass&GBAR18Y&TOjiW;1}QhZ0Oo6-9(@?wWPZ>3 z(!@e6Q0O8;*2Qws>l4J=gugqW3i7CofN%6~quo43zplNdTlU|wFU8dt>~--0c)Em{ z<$a{_>Vz4seQyOhnkfye;c#{pbsuZzVVfsviOnx4O_$3SzSxfgNq*MA#O9(yYlhv} zyg1q@;;)C-MNhPmz~)zt?;*1v=MH5?ZVrBphj4ycQ8|~)zR`KzV=#XRI+skl`DWeI zFn{>uTvB!S=HSq_N4Dj_H{7nrjN8hkiteXJOYaAj;tQSEL9>({OW(3S^(n+_?QxCA zE0gEUsYA#$+1TqE;2b`wRec`Ac4N9#&-Z-7UH|PvS=xWP=r$;E;B!jj6PV)mNzvg` zpk|d@oJUWr$cdRXMWvb?5n&e_b6pD?Coy_OqfZ8sHb)X6!L9X}3s}(5e7v{ASJzAl z3ec=NlwDint4n(QmZpMk^LS3vKQK)vaxsToRo?R<$cFn}gZjY5r(A8Jfyt-e!|jSF zIrBLIqOY5~cRW`aUK$^JL=0N@TDJhckqoi7_Ebf&nDc+x)Mr7*lWW`Z_o6&lJlRv* z@N=Bw##~s02fKz^LvUInNB!GRfHL(Hc1|aZHJr=$(9u*%wr(6k39s9-hn}zqwS4iA zOl?aTdcq;3`6DrLx7Rxmy4_A-p*&!6K})=(jG)iHa`(TNg%dxfsAMma>W`bF8aJlF z(tw}C88_y{(lF39w8ih}Vaq>lfvQip=dq{(g83cXUnY+dw06dgR-z5Ix#fTz1;5!_ zf2yo%xArlo`qAlj=f~y|RlCzpm7{wqzO;Rp^KQbQ6~ufq`tndQ5x|}(3a|U2jvOn} zg{hi14{kUZ-qd0Hn+Z{@yr!?mN${n5C zw+P~h6;9cetQ1d)lvd_kEx|B+RH7uSvJi~mj4FM_7Ou6X!;Q0bcP`QG(u-0X+Jmpvb&bF^i)`&c4 zS#3@F?8@d=>`oWCINzGo>ni28{x;|L9n85JDgCpajpDl@ge1Te^2-2XK-Lik$yHIe zR8zN%T3}D#C`sS2`_F2Lb<9j_@0oI_vdmiSCMqTSX?Te!c``u>#+yhHky`Ny^Xu;#z5`JBFG}uRJIw!0 z%FO)rxhZedURoBXvu9^}q;q?2{dV@&Z3pfAeAQT$I&P3p>-rf?p;G&*k*3&0aN*tf z?jzShFpkQiU_h^SF;zZ|baCK}T_umSL1nKzN7L+O;|GNWL*$_1PW)y9v> zjz$Gxr-IcxVL3}d-V`1F(bD4t?bH_J-m(C9ubJ zN$DfE99r7_b^QzDl| zhKy|_tZ5*QW9cq7%^P%{&z*Ud2Ye>x=PY?1RVH4z`P8{;KN%-`uT;}9M-#t%nqK~> zNI!0bX@^knnGzMLcWgO%5d*1_g5jB!cin8RVc`wg>ML)f*3n*81slHSK>wBaSSzcB zSyb-6z-5ewF4G5u8} znhn-wHZ_Q!p)B$a&zT0IhQ^^AYNz>_dJyl3y+yf=N7XMv`}ZDc)-0O{OO~qf4+{xq z9_iXtB(V4UWpB_rK&nUe+Vv%9YFW%?)u`89DnQ?u%^FIQ-D_A(f?f6hH&yGSTmv^T z;rcEO)*;0&1*ctRYTRp{E;@W~)@@lC#u7jTYx>($Of}cug3GjXw(BS=D=gWSN+0(|Era=V<_Lajc+e{5Ve{Lob+0CAq`# z)Hm(Dhv^($aG~VZ#MVb)*?6r#C1onQuTKe8zFVI@R@oI?OQsQf)ks#HCAbhb&LNic z?PyQ!H{DVTzq6|wYw=775k*1UjH!*+(4;AF1iDY6&#O~x6-mK`=yAzM`vYJSl@7sx zS!q^|j?yV;ua@tNewLjWVT6~eRNnTv@MQ3q)*n1IoDbVF#j&Bp6Io+kR)UkkL=z@s zUMsClaqYt=%2ThYVui%Y&6I9_ZWoT}5!;3Qe2(IJB`dgkGiJRb{-Q@{^=80&hb^H( z_QUU6TfsA&go>UYrdmr=RS6X~?FHs4d(Fbidk!S5nR~2w!y%B&J%Q|D3`ky8+O=&L zB?peCo=2RQ4siw$N5Ap(4+|=7u`iC#ht0zx+JWtI+MtLl&YJ06G#7D^DCyA(I~}t| z3>Q|BZJgHKbm5pF@N6v=m0u|s7)IC}F#01v&vfx2ixE?nBao94skZ0s9KPv08!X%X zFS5+9?|>(VACP2?x8G@vUrtVfy_rod$aYm4+bOCUlq&I?Xp5#3z-Jt$U2(?MIYc3E zCyUs7_u_Aj>oNo8`b~EjNoMtjG5bD9Jt|~|q)8bbLti z0#^@B`igS;O23S64j6*{2?fXFgg?L?95Nv0S4uW@C#xQ9v9v284mi*ak%?2{*Y zjmeq@Q-UJOt4oumxx3N|Ah*N2wxsKE8)bY&#e6+ysiNcmM>5vyo@m+NJxP6@?>+P6 z4agZf)|{t2J2!*TlCgaB&9kF@x?tPJ^l~PDW}jecAA7AewLLhyJvh33iEztUt|k0} z*1LmfY<8(?ks!yo^Cf;!E%W8WB1KaAm@Ql-Rc49%<0Ni; z-OHyhkxE5*=Sq&3URepXzI)VGGQQl@R+ogj)K)#8{z`-+51(}FOtjuAZ3#TIx%`9{K-6bA*6tuBbvJ! z&c&`EQKl3@$!x@}=Vtl-w--_E_sJG-_0yy5xCMU=N)kO5C3R97W{vQn2YM1pG^BP4 z=@@#MIS8;e1@FG6YSH~H6Biku;C_{rFTK2?-1}_oC2}4H5^5dk3hD3Q1{!tgCFvSj z=|UA++fSwwgZ0~6eftMkHkYY#zSGyvZ6%!c^QU=lX$|jeIM(y18;cC@sMkBG8NV5G zQa6@`7^&7@rd3HmL_Q;IT^+6K-%F5Zv}w&SQw{IzIo8LdR;98zQx4N6O}u1telkp} zThEbF<*7FF03tFD6mXY(shY=DsjxC_S05s6P~Ywkt>R{LrX7ar)bq2w6b4Qju(eRB zw}(`Xs?TUayrXl57Z&?$Zya@}de5rwUGjJj#z;#`iE3pNO*K8(mEcy(#+e9;@QzF` z`@v;l;KuSoOjBT>J>Bf~$_(SQ>$Hph#~y=kVXgOhD?zZ8D}!&UM_C{I37O4^zO_$F zqqaZkmyQx~yXAx_Q_T?+I{fDO4`A=`oFrP%+oMJ8o{)i5F&MG_Zy}BX3y5YaNVYs@Hxw^r)>Po)3jk~%1aB~}lwlc7@5M7V1mF^LT zo$c>u;2x6t6|07ol}{@x^(!lND=Q5vs`H18bBB!oSsT;CKNmXZSA@x8d!>azrX+N3ac0K5;L7E(rZQuU*ZCpk$seRRg@>r^=2uyWMKCy|1woov$|} zZu-xJUs?8^X0=#`R?v|Qy{ngLetr70GfpFoKvJF3_VINHIKV0UDE~VraD$$@e zfO?78Y4|qJeyF#QZu=&BGrc453psyH0n!S^oA@18HK1Fg=>*i>F#EIv?>GKU2C=n| z?jh2uL~CRkarc?pAn;g`K`VNrl(k7;r`ii{{H~vg6&FtS5jGCp-hsDE^Tg$*EZSZ0qzW-* zBh3%k2G5%&mZ!3cK4O;kc!?6-cldPrlt}MEMHx{!I`OGVSIfU^l)hh=v`H4HcyU85 zj$Ls>6A4lc?TR#RG^+GvUumspG8%`b%7+1o9_HGI zrizEAyY7e39i2ndDk~H$!$x}b>F1A|Z>H<79;@YF==XII{j%Ay#?+{;Jx`jhmb|G9 zUL^gaf6H?60%4^;(o?zrBfobKYhgCQuS==BSqQINNyM$sIn5Aind&rIx#3)SD(q^I z8~o^%C*A)fZ{fSzEK`|NWXz0F=tqtvo=2Yx^z$19p;(%(oUTtI-iMnVe7q^xZIm$q zkKg)E+^&?~^hB<>E!@R!xQZ?9SuEUL{oOCg*)O{ZRlB+yT0fU<+TIVXyJg0yF?U|W z7Ss5#EfWL}ZLKKOEVA)hJrqjr`Wx8(`_R(otyS1LRb_20##KzrIcqpzeQ-Y=;{ABN zZFec}G%xenON*@ii)icrBxCWDJ5$DXHnXjIm%_@oL7`sM%3FIYjMY{Z*7z2ia~lk^ z*7(w2C@*mgwhFWgL)GnhoV5pnCRd`~FQ_X^PMh$|S!V3qt!NeU8fg_)=4Ng=5tSmR zUd5{uQP?=O+QOUDbEm|I9iC_R_@CAw7x54tQ z!~u_MsDI$iy4_fstOg7?w#pz`%vZ>2Sfh5iRxiWf=H1)H`Rbkc_J(lA{p%Uss)T%N z$RBZY0g$vg z^}n4nI5IQDI_F5PUOvC8sC?VC1~M;jB&}&wi`PlWgf~7?*GSU7~2(>7}K4 zk5`Xp0#ZV^7+9JuMLgSF8oLX-)y$fm>ejb3xGKFUh%3lsUv+F#+l&{iu7;OVp`#B> z108dWnW7!(;o{YCepD3II&PK*o)=0%vCd8_+daOpy1?U(+fNV_SXlN#y81EoxnOk% z8{A04!<13qFkU-j1Adl%-AL==R1&vu=NN1ueX)d^*cv{}BoQq*8a^cEA3S~EqPcih zocI1_%Yo=pZ{gMD_|wbrx0mC%m*akb)o`J_0TYHlOCYCH1rJOnbNKy8_UV{Re!m^+ zjT~vtm)>`nzFfJuybBJQ$fdn`qb>c0*7*izwzn<2l_~ntdXoR13+)Y9g39L^H&iFu z;=x@1gvdU5g;owXS{3h5a9^I9##y4(nfcQYbUBq=P`KJzF+4*3>T1X=?5sE3p~NMN#AP*u z@9C2E;nv`3nYjhif`94vz1!+9SVfKOl-tsAe8qByC7#j>%Lf?~jaOu5^2-b9GUW?jmlLdpzy^A5RV3C0JZ}jV>jO+Vkc;aIzO8XNm1c#7Ybn_&b*& z2aX-tl;>)PjlSp#8i&hR2?J(#`6K1)TAw~}{&cYL{c_C)F7tZHJIs4l3{3xO>5Qh~ zEA<#YPQ^rRHdXg){F@oGU4F%KeCShlZA~DuQo)lDS^&GCfyx#F28gjFLdVHqAPSJz zK*SoZBE0eh-XktrTCOvt4^s9ahk@^OW}My;2Tc^8z_gqExp;5cdl0WjOLSk1rz z;_ZO#4-9O1laVDX(pv3xT@C@7$#jz|2cS}1#l-(AbJePI`%nSa+7=|wV-Qf^c4gz0 zmOq}DJkm%hI%V5V&rdXY=cTKg#^7?8m%r+6bJ?+-RFSm1bJAt;G8kCS4c_)A&tGP! z2ofx%R&xrVF~x%9SaP{eCcvNO$);NJpoRX!y`~*akDGW4M%lq*VW*S$rBjV~joQqp z(|ky(CF`?oOtyl$r(I(1JFFHHR~oaYw|$ObQ*e@NU70#(H6eIw`)HZ`Sj{acpwRyk zH(s4l`{Mwz_8W`#ulqg^9VK!=NjNSMhNZd?V6n5bU^c2F=%GZ{=-_5x82zrmQi}UR ziJ{`q*T;&PFZbSr_)I0pr1~|T3pDe?O?PMVLzqiAY$x&N2^D+Iy=5179S+jZ1-gsdzwM5%{n?Hhl1#E+ByX9X7 z8z%}%kMk9O`J;NyAlcp7Ox3C=Wn)4*I4GEP`VT&D4B~-fLK)*^y?GXRXRz_7M`sF9 zyc(OoV}y6+r;X{hBv(gX`VxP@wAscMX5%u8>S0Q;G`LH1d%wgaSgv;A*Y;rJ6Kn0O z51EC1KmWe>z4EgYAE?wy@C(-H>}ImWiJ#5t+r>S)B>z+!Kikr`OZqA{ewJ*_dum-5 z1JzDZORm^93KU5)iGFx=8SR(HWZ6Sen+8c+jhs z{!uzu*-o?1gUAL@`=qhO3g{7+}2)66F2}V@$GiXxQ2!>3tUshx#Crz zkxJs=lJh6YqaC-cZ6Qng+<{7!NIyF^buE>5kA$@#si_*B$4HHguZdpVBcGqKCircV z*GjET&#WJav~C}L6ghRLsr4B2JDX;)q>ne@Tr*PW)QC4>KjOiv5X^rk*h`Y*4< z(IG=7<63Ay6M5D+Nt=ZAjMai&VhXSwLkOEi>_xjBVJZVnZwYiiSV)EE+raGn>5fZ) zw`iBdA{9)cU|FW=I?27c;i$rR@E*6TINjl7>|MS0O~FRb;b^GUozd zw{ZVB3RK}k6#vKUSYT5GG0O8*KJE$Ihq7KNJk-gne8LmT4`pAZ=ureq)(ktsT2!8I zZ~px3VRW;=lN_X94U?R(OL*eDz@q9n&f+k^QVUnMR8Gjn+Bq}4eqy>I(m3?Y@Y>vT zL&|jHta0dq;rr)|gT|qT+$i^|{^|kRf$*f>ty7lV)xA=b;1>_g?qQ+xjW^uuE7JSN zegs4(qFZidFPE9IX#PO@CE-uj`L{At`b#E+ZL&|T11$!E-$@ZS9ufUdvWTJs^IbhK zV-R-n-&=$4;h2uf6zk2K@gp-t+U0)1w>ZV!Wd_&Gmnnb6n~KXOo@nHntX@C*rn`6qFWo2CyO7mU& zbll{_mhY=ee7m0L_z>P;FS^pj>@DA?<2|cKN}+|46gU1c1m>c{+&VcMAU^1Z8+0QO zH2g>XrzLcw47^{Weo(9)5;vm?8pZ?->pDUT{C}_sxCEqis`a0B(Tt?&W`XXgViIfF zrPfoF9D-t}{@Cue+6toV-#*Gc!ZelDa3A=Nj(|_J>hy_T73$T>$`G5ZJLaD)I=+P; z-P z1T8&9;!Nz@mF(=*73`bg_9)U?1;^131=*^d3uLwDVRgMOwIh35?wt=?DQf3q{Q1|t z^CtZ)YLz=BC~EQ7rZfD!SuMN9cT_rg;!UVbr~4|Yk9M>>ouuE-MwAXZhrccy-7RjB z7}!QBg$w&W|h|)Cfn0xoRzvc3?XH#Q{D)I;ml&}e;EsSNO zOP6hD)wqfMSlyY(Sk>9wdF?K2)jrZ``c?RsnitBv-n*L3VVMosSe1P;G#4b~gJ~-U zsmp;sIkEr4Z*h^ow4rI@TPc3g@wM*n!FW;2iuxv>yaRXFja$G~`80ZD#c+MSk|E|Nz9k$zox zb7p62}x zt~#KTjYE2+)ovHUFD=xQ?$rEe82DsfL+VWOtHpvO^I|DF|4YwgV#crCE1xFpq9uZ{ z<6dVxqzZ__ze&ZHR*AlSu*&%0fRpD`oa&M{{3;1wT0Qz!KKge1XvaB8lIk)l(xhgo z?URv%1a)(J+qY=$x%wbB=22at`s)%`<1k9wXm!qk`X_l_!$NAwRZQF2R6JdCmRVE> z-`bu!@9ce#uGY&ZkX*_Zh#ix5^c%6uTHKo3oN+{M))%q!@Dg|VsUw_NvtK&mHm6=! zDQ&0NrB77ZJ}@W&&Lw-f&EVU{ci`2=N%Xya&x1~PaWj7sHfp!2Zf+ddcg)UNDYsyk zfqn%qqvNR5TdJ&uOn9bU4%4RFXgYp86zlkrhCLWtIp^D!q|@;uf!3?7N+W*w<3k%` zU5#e!uNV(%RPY-DH1&)b@6VdF+n;3}xYQ0ijW}rJ0k_pY^w6ah1#asQP?ol8$JhP? zcgG84Yg0bnu;uX!(h68$14(|LHf&V8Xf#hVOR^Sy)0Xxh_Eem&TDV9Gn?>*N=BgaY zZ$lAFczzU{4+oBVEy4Qu3XTcU3fv|0X&h;p3SaI=M^xeCzN6E}QpONyWwKQ6r|Mk8 z4^Y-~$Dr)IzZSt15kc^-_2>O}YZ0w$5nONY)F)GXCUhW?s;#nl;NJ}pv}G3 zOpKoa5#E>*VnX-ER8cBBNA}oZZH*27QM;npVHbq=T6P)k`Sc0h>zuGou>Z60usP5* zN%fDZ+Mi1S|MmiXbEQ84gw@Kn+w$$RiePE2KLP0nXOEV4-Itb}RZo|8wf7hAHT!AO z*S4&a)l&8ChW72!tTny!t7f&Vyl1A^IYVCi>t0|g`dVOm)b-x#L?_WJERzgB*JM@n zDBe;_o=yB5!N?{N0GuI@+j z_D;vvP6ytM!vyyqv+WU4F<%mMtFKMeHE(`iW`1$EN*o-pjJuqh)7e)gZ2ZeLwMikJ z{AlO~FACFewsvDv3{@POk{t=jxXQBqb<0=!qu1JQcSM@}7m$0dB)IqbVTauMRl_MymBy2W8t9o}Q=7O&~E(|sL99p|3 zmxd3-rM-{kz=YNyZRIA(Xr0EUXn?5Zc7-$p2ha+55@brPapL+IT!j>exht$kl#n3ONHD~95cH2RCG@)9#qUTYMdxO2EQxZ#v$4T8H9pk(4p)4U~G9$7W7tr zY{YMg;tTHk&;mtA;=XWnIAh^6Dk!kQ`yRp(nM;hwMjvtX3^?1UwCWo;Bq0p-=Q%Xe z9oBXangGkbkB)VYwdVms8-ciX8l*Wkuq%A5N*Jn0mN*s~J@I%(SSY6kJKQc~TS8p~ zk?J)`^y&xK%ThD8MUaVBsXy@2gqCOzQ!mQTLpU{tLO4aInpS*2t@t*q{Q0y(Fn1U@ ze|R-_sJ!=6uyZjK68qz5UVbI!0NQLb+dFZ znqs~WeP0O5b1E44p$|b+m*icD4})65ITStSO~<&AaD_B%#1Xo;6n+EOv%t2=H6}%f zfw0!$@KQP;4!#vQDPVvwiHi2)Ub(b=->mYfXK5`)%Yj=}aw`nz~g z3-{gE*;s>;NUvitdfq$3onF)d#OZ*WeoT2{_=jfw7-!tD`4eML**ZX{vN;gAa7h@$ zRCpSICHP;W)2>wtgpF@Jpw%`mIiL74n<{BEYias>aN02oYN~ZF)|U@jSz!+ypmT`gQi4!7t${gfLrg5RF`xrr3#= zkT3y^cbn5B!8)G^DKe4ZDtppo`u#ZxOU46Tt;`20EI~2t0`g8p%kiAxnDW*I(z9v0 zq8!|Da|IAdu^HaDYZO>U#h1LO19yDs(UNl9K|G(*?O>@YRS~x8w8Bm$l_d7J0FoL~ zCt4oEO|lc?J6040j?!`5#WzA6pKf)B^QuqprfKo9rYKGx@{eFFREiEg)nOtY(T}7d%JA1bL7Po`CloHw|Z$jq|Pk1+}S*Zf`Sw^Yo|c za92K;?_bVYTy|dt;x&7Js-7K1_0T-WGHYNmQ_yxvns!kXF*A18=5zkAoY|C}*%a;Z zLt```FY}DU{>^y(a-p4T+2fTGz}oG{Roh*36@g!_;vch7>Dj}WVvTfW1dDh?vh^&} zI92u2=C7B{g9=;*&Wde_3GXKXFTu;d~MJ0cS5`8C8orGpj%1Q+|XcjfZnNdt70(6u4hu ziOtlZR^DFGx5&CnMlD*k(tq85)8%$sG)DZeq(fgX#@ircqawb{kvMhzBR$J^#@+%buAuoBB?$y~cXx-y-QC?C7Wd$g;O?%$ z-QC@S?&6-{1b2r=zVF_8|68x>)~lNB-L37Ooipe3OrJj8zoto@hOm%fZh2)eFwM3y zKrxt8MNg4nnc;0+MNyc?CqpM~KqJGK{3g0fAhCa=T|sJIzd-(P_i#H{4!Q32(_X0|?#+9g>-#(3_#8a8zhOJTJL!e5b+<7No%RIu z@!NVsitR?*gpJAFA|*uOk`d>JG5$DF2^|~7CCmFsmBubEIF9m&!SP5Twi~G(Iu^>) z%qp%J_j73`{G%ReP3uzah&5yP8&AP4za%sGcnP%{@#ZP*=oUX>apLAl%k=2>Q|0RM z?HqdTENYdGcTjVR@jufV(iXqpJIvS~`cxqBR#+c=L(h6dhd$#V-jA~3eMKv_1!NP6Xyu{m zXysM6R=$6pcTCGy*Oav6Y>rbuh(RWszG?Q(`G;TNI%X&Pd zGzzj}Z>itgXN;q4;6aSu5n$47KU?TIMaQ6b4^|u{TlNm`AT%swDf8((`IiJ z)gfEP_|LW;!EnVNHEnI

R7nRjrlF0cGlV2&&e?wCrW()kX85QgbB4#9xPZLwo=; zH>*7y#wv`$&fAf)bevmqNlKDiYRTduAc=HP1PH|xfKfy`1k{dZ)h*(ZY^7$Jj|Syb z{71s)b_9sQiWfAiZR$qHiWfF}G<3*d>c%O$69HNoI?To~YmzpKK2y^FKo9c%xw-;2 z&1s}OrjTh>Or2hY9|uV!n&r@O9s^te23J+=C>_BBQ^JQ}94k4w!d%8%Rd|5{9U$^N zpokHq2h>spYJCA@;TVYJ8$izzn7WaNd_x!JLNCmsY0@(#0H;dw6qSb)DXDLv*7yfP z!^4~Z5_gKyNu$L{?zaYSTHoSWvZ^bB#rTj(rNHh zh0eoH#EeZ#RWV0%RkOF=qFzhHf|2esBYH@OpJw54V|CjWJ5rn@NBRHbBUr(;`S zr(*(l1vNF8BS+4+{#Y=?YUr&OK%y-qOS_i8m}kmo zhYh2J5XLvCx$8X@;*8Vk1iypGi#@Wq<@}7^D%SQIvSzX#E1gjNJ34Vu#@=^9f*GoQhUCSLQp#>|^fnC;0FVTEWS2ngfW!bKJ*xHufs|AC$uKC03 z9A#gA%O&1}CQ`eF@$o8XTw3G(Juk!1GHu#wVUo=F*ixekoJh&#OuDJhh)9)DY|?o{ z?S4xcT5^XxM(pl)L1jQThU>pVtNI+cK-iQESs8A8OQtOWS?Ljt{t%-AZ5#bh%h%QC zfNcr~Kp=erf*!5#m*On@_;kCVPk3YwYfdV5L`baX$_YoN8%_)K`T#Ki@ES-y0|h8v@jEOe{X zs32j72kJ-%Dp2dPD|bc%4HTO!(W)mR7BN{Bn>ZzL88vSG;ve!Y16FnH1D>=U12Po? zI%3}d(1piGN?hPPhXEkdt2NQ^Kl2{X5!2(M44EzE;Yd1Ki-+Vpi30V(e#JH3@g{rWLkre5&d7xp*!-#OX8U|_3)u~C zN{hzV<<0O*!V4z(#T;)u(5*QS+BNoMnv?LFeRba;-$ z@r#-jG`%5YeIm*y4hPK)9byL~p%fa0gKAkv4xQ9TCp9FY(ceE-pZs5?h9jE1w^yy3 zID}-9VG$tI_}IwtP2udj(Crp3)ZtBa@8zt28~F+BhbTdJsUZ>-(LywcbfG-0KEKou zp@3cTj$*v2HPPX2(X@q_TGK}__+#l*O8V@>D|-5G>3qKYCu#7(8-ozj{FOs2$&6Lj zTAy|04$+1df@6-SY=uuQR6wuc)Ij7Hsbu@m7-V7!&PsR!Aq76G%Mcbh)iMatM&n=! zrFj-_o-X_|YeiA=A2r$e4?#>*_3z3ars~qs-el_H(TXIJ!!Wm`lEXt~j&`h9^b&$) z<7UX2NXX7I~V$s-8*;v1}P*~17nq1R_U4I$V|vcb~zW=`#aCc z9F}uxhYGVFu9JziX@O^Qr4*D`l2nPSzD0RcI>FMJ{WnV@pd(TLHS2F@;+va>K#s(B z>FYKbi3UJy7S8Gszk{h&h;46z@dgFc^AyPxrS^5vn19wy%M7)4tK7#TbjSDi9oR-X z6<@Q`xT>|*$aZsA6>?Mct{QbO))Jn3OCn&)y5b$2+Z>#GOS7UYI^)fdnrks;XB^RC zwNZ{1`Lj}sd0wV=HVLL${Lni1Eo_!9Sd+X7EB^q+o9G|Of9vqpg*52)gESD3bJPmJ zEmd&4ox<3v#Iet>bS!?@EUN64(Atu`?3U09cqRrd%pt5~ey}STbXRWHya^ikQMW&Z z*SxVBysAxp1JwzZR-Th;8O<)-&o1;ve{B5~%$YSrC3j8GJ$Ml=w`iw9hd$x{-w&n2 z{X-A2!~KH}J;L93-j9uVuntEWFfI7s=Gj@ zZbHVQp#+*@yD(ma+kk;SV~%COT1WS4%h~+OB4IV<21t{aPnE=$nkN=e&-36Ki!ko$ z7XMjF>ZreEIRSUB?_j@@X3cvKKOqg?3lKG9sCG%q8#M#cU?XUU$SBGhWS|PeB*AZY ziQIPjluX^deCOI`_E%|7S_LC7v#f8nn=avnsEu7U4nigW@8(@-Zo)cNd3947(spNF zcSHc)pj%o?Y%X|Hg@TZvn_|xSGc4WLvOH}9N(`fb%Q{B4;7!U3DU_1`)h>wcFk2;l zRk`UE$rDS3{XN$|Uz+VvQV)^bdltv_KJHRVHVZG}4Gm%y<>gUKWEhoZk&IYi4Rruu zMXOQf*(Z=kK8r&+su1t^6(9_My`6)t{FdIc@GTv!n{eNbcM_6U9v8}u=`me8EJOU0 zLdl@MbXa@%=Z<(J^bZNVn~09V-HH&|=9*!^%|Z{>9E7U|^*o zVC&W_U>m)edkYY-#cXcFYv8!4YQCvD7+Z|GsdB%mN*i0`^EZ_%OD*8hWc|R9mQ{&;I~}#-S#!+_4n@i%_Zj^_j;4S zeECMtiteRok@Id#g004LAr*e1#;smu{$5r7-h`MSjF~))8PbMJ>fvjx=wVlhd8x=n zErk)tL8#1OwYd!;gd}&YxW*sm-1u@WjS+6HoR~@MBH7dX+u!iSo$-pp*#B+S2X?q- zm-?uC-i8u@VHOQ`@wo~_P_J9)*or1NryRVJ5vtO$%N>WE#r~yGe%$q2;%6(XXkM9m zcKVXR7(Kjo6!-9!iV!9c0;nld6a&OaZv&^oMA`shPASeQx(hKT6TTii_9b34oIiLB ze@-Odr3ffmM3!l-yycZo2da`?k^r=k-*mtX_THl12A6S(QwtCMnqR>#@JB)XUWQTd z10_WiKr@2A3h!o!{Rw1OeN09d9&3Q@EI%G#FvP)I8|8$QCVwyv6NAy1v{!MB5j+yU zhIa(`vae%df2bF-lv!}$+t#MPdGg`Fs^G}_jLs_^}y6-hql1r+< zrB8q`90RF*1LBgZKm4qBl6gap(mW!9X5iCA5OQ;KOnp zD~X8$f?0L(dmJoaDA1&tv_r`;Tobjx3Kparol7pxj8cHjs;d;viBj;93LzZMIf7Gg zs}ct6)s^;p{$4?c!r7<`EMeUn1m5w)Q8llMoQH;MT9ryD_1?>H1c6tj=# zbhu=gAhA`YE}$oco57RQsnF%XV*~}Cha4YurTW1Ec(`5Z!@HJkgi_uO^c%i-XG~b7 zmY6>O`!@nRlrmIG1E7)1YwO8ylK)J-eKD_d@JYt~4QZBmepQ6E6Az_0=r;kGY=E57 zfyvP+8twk=r)Q><|_Kt#ihsGU8)y0VOlnRp4F^nO1oOL0)wlwV4t2l;@-2r zG~+eX;OSxg^#zHJwyMO5#qGazoLXE>F2lzS(w}Ov+wF9Vv6UmAE{{VoHDn!g{!n5n zq(B?uE{4T`Coqw_@boH?yIj1a)dVG>OkLbZOOQD?>W_XXpvwHE^wS>($1Op(i!wJO zKKE4`XNE$~sXzYEIsgh>$mWzNpsH`}*O}K27e0Oc!vR^FufD}n;fS!nh0yr%%eeZM z^rT|`(}=R4*>T}hi{$THZ|z#Hoj>kah6lenKGm{yF%48A)Lvg?1}{B&8iX%Ncq)`5 z*V>~d@2;72w~UrkF#kAl8M9PP)KX4d>UW2t^rEEn!qnGNOI%V-TvADN{aZ-kA&ydi zKal8$5*I!-d@SU-Avv^^pfzV(Nif-^r>cJ#KnECpx@3_?VUK4iZ)G>yisuOJI04g{jh#)R17r_es>$lR55$59lqZF~eXx zLi1YD7NmyT_-=$1$uUl`J38~)1?ZJCn2M{$I&+_Gm6WxM)!3y@^*yADE42z2Xv=f8 z6k^pBKGx^pikIYybq!*TM+-|C1y{; zzT>5*i>JjpCD)%XOjS$wmtNYH$IN4~!r#>8y~tX+_O~sMr)+7Uykd$yi1<>|NIpXC zlnvoGD9az~G&xq3P~%?-Qcr|^$F6i`Yz*h>f>c zgkEv{U?oaDG0q-yAz>G>44wR(0nMhF?Zr7p5YI#EDz^Vaz8twh?1Kw4^~5Rrzc?7^ zB=n(nDCWcQw;@%K4R7&?!5`nKKY9bflwjvNnM+2@+>o79j7f@c;x(v|@O|W2li*1U^GyZ`oJ|{ zen70*L76Lw-65D)lK5aXG{*~Q5t~=Xj9U98ljm#Ssvw!+=Me9Wt&!MY8z$U&Wfh5S zT7EZ7%I;z$=_XC>;u=o5PxV~J+P9y3q6w?dVQqdXB#8MYe*Gbs`BcEyo>$ zOFDWSLwKt{kkKlZcr6IAEvDL_2EG>jx{W^mYmITmEAPi;yY@=Q)g91a09QEt0!`6O z_ptt9|HR_}=8livt+&zr^nU5&UhwXj@6Jc)`Sr{5EBW&){IlQp*SC~c!9M?Ytn=4t z(B`?Mz36>sR?Vy6kUz0yZ(QA$6y?T-Fxv(q<-~?~!-m4}t^<{33zLUv*ZmrXWv>VH z-P+#P`A2PpPW#36)6}ZswM(w%UB^>>uhtDo>m$Wf&X2-i(;K?rb^D^i&T6-BrLx!_ znY$fx!vl*)4#W*>HNPD?13ORQPQFe2b|@|P=rlm=w2vX#Z<&4lcKGPt$xgYVaPR7o zpzguhdq&{DvGL%Z_2{nm;I8){y0-m#{0;Y7eaCdGuZ^2^c2}u#epeyIkhSAgLCA_}uUU_Yu*sH_IgaXd4T2G4`2(cf=rf%T$ZYg=GZYTUL3M@~m0SdO@7VEzD^2G1v=)Fbf z-pMVn6BYWkXipBhjcG5BhjF&40erc{=74u1qhT+W2m-2a_fE7^oK@b#;craFz3mvJve|_^R(|^Lf@(2DF1E~S%N{Q`_W9?PS%N_@G1o?PfbMeX6L*B zd9c4_{Qqc$K|eanbBKPPV-^V&qWP7?`RiLeRm@4Gsl3kX=U5XfQ(Nartk#m9Y9-yR z6ushs{o$?i;Ga2)UU|X(0@l4hnzrOu9Z64JP4ckSR*|`Q3&>BUDt{ug<(7$_qqRkv zt3&vi57$}1y@e@xHaj~Rzu;vqE-xik3-Y~_EV@bTB{1$*_Z3rW-my*eGXCMiygE;$ ztMFhM*kBy(JU3C->yz$@&wY*GZ6{9k?O}4;q88Y=FYK0SMVOP|=125d$IPeO4Q^1}H^S#DCGPDUz2doDG?dQeklT-EVb z)JypWVtgK><#pap$KO=`n!j_-yWs9@szv+Iv*?erJODyx=12_V{e1=cUZWA*YE%E5 zaZudx;12ujcsh1H563&#G5~e3ni$}zni#mx5RqejPA$xLM#sRyw{2WMN1PptR9+Lt zD_kRG=$2(POr$E2$VSW*FX38hO`q{v^wQPrD6`JlN|%2eYwlw>&VJt-aL3Ov=F9nQ zz}t_6cj41 zk!QGelJ?D5(K$Z#lG-&=jBR;NTJa8GspuT8dCBD(NiP0WCVvA+zV>p&J2&`its3PT z>1vv(77fTXShFTJ!nVH0q1c#HJ;^nvy=Y1l%v}U;%^8Mz#)fq_N8dFiu;ya$%*`o| z_<$`wpBwb~%<44$;!pFj5M+HgP*J1Z%q2LAzw z!2lIQEFl5ZV(f64pxNkC3|L6%saS(YUf5dQ_iL{b7W`WWX6?g!HsAEkYdA& zoF$_^2cl(qHK@7aY&CA8~y zW=1!PS=YpfaZH8m3;jcw^8J9EvlIk)2!CK$ejpE_d>{|C*G^YhD?$R6!oce|(3!*k zAg0iOL?M&nAnIjOgG^kBF4KtblVD+a`8dSy4dZ)VqG z*7zmszPR<^%YHPmjF2Jm?xXneuA`G8_mHQS9d$sAwT9KEhm7;GlS9i|zVZD>-{l_>%1cF>$s zrd3N=lXFr+rGDGi0RM~bNZvVSV3TXI&a^`=_5vUQx+?1`R9$`^eT_l?0JBX0SuboB zGV&<~9ZxF<#7`t*lM7R7ow3eP$6rgF}zstA*8KATUCLfsnKOvG`~^t zcN_+o3s$|BrFlG5*gX9!tnZS61Lq{b08F zPkc%v6B6J*!~Ne(k<}9aL*BDnI>KE8Q()=|MzxVN+0u*v_`i?uKLVI~QU5vg|Nh~n zcLLgR1)X=M!YZ~19#k)q_IL3p<+7 zR6`J@^NJLQ@1M))Dq-=pDu|>z`Wz)Z{M^dGB4&4I*+E^vw3SY(fcB)rCCo5(=J&(r zSmr;=@S{skzF$T?oKC`ae7NrhFU58VBJO|==Y*j1(9RNIyj{l{F%*&ttJ2cK*caB z*iG<_Tip3FC+Ollc~qG@w4W6t_K%9wOO>nnBW0i)0Zdlx){RSas2Kx6>3}mHLFu6LR|2a+R-#zf z-#Eyl!8F`&vnjDFlEa64=QpJC=Et4wK38*zU5lP|KN#XI_ZLRU>4SU{cKK+}a%dtp zuZxR48s19*Depr4d2B21jgS7nc8k7FM&dp7+ANpFzqh_9+XtnBXP4)+F& z&TmSfk{N&F#D5?Kd)5ngK>LTcdX0MwyklHj;tsbuH0$s_`sWt4^yE>8Fnh5s+>fjl z?ck3WZr}L5R;0fK(9GS3@qT@qT`j5#efhE5|8nmvvx4>hUI%{vNO%1q!}G)7azj8I{%$`zl|rt3}0jLpvCsQXXf zt{mz;vBRf)YTLT+BbGX);XJ#?o{b-hr4qYal)g~$BxdW5awqEN1I}~>NB;OBq!qU@ zBq5x;toGY{t|Q1Q9g@d)_RbrzOrhyAJTyRm!;zXT&NL&C*#yQpQ|wV=mg%cHT?%Yf z2J{vC^cDN^q|Ohn4c!aZ3fp9QqW{t}ksY{_t!DE~gCka^???&i-E2MWH#+^jB~SrT zous*mi19FSGai$;;};~@u0pV>uzl4dYiyWAw7CR3g%4hOwj8t0eiQlmul4({2sv3R zRsOcl25eaqyp#9n|Aqfb$YH<|p86yPqw{j!^c302OFRd+e!^ScnzmvkJNI2n&GGum z+Di6%FQudy_J_5#XuDR;`w!wC`vfDJn||UwL%xeA;Z+}@WV>_4z@{)(3u;Yb^dco# z5L1<4+GdyG_AA-0YFRuVKXkWjkYRDZT)ylVt-R*Mmp+DOKjE|GK_Sr^LDvP(TU@ul zacJgC;!^pPw=w7&f`>g}XV8cK!Dl3HLZbKVZqPrUF*}5Q^1*E6v4T>JBuW&Pco1^_ z(y?946GP7q;iC|D`6g&S#uWMan~hI9lC?j|m|7kGa-=$Y*;h^95{zz+tcaIwLVM+gWA zw@(ld)F1EW_RKcsre@5R)^=_dF3j?dR;m_ejxOdDitcuH;wEk;OlEc_uC5_U^@@Wm z7+p)D6v6SS1AQ=opKva~h(oHu!5EmZ${wSv!wJXIv4a7uq^uX8d@%+h;P3~)oj!uk zxo3X<-2o8krpyE1owRGTHLwB&XitkQyt=#63R?CqyB3}@`G`nKqUqlTGWIRU&mv{v zq+pm+RL&w<_<0Oa0(A_2L5kUxEDKBH?EM<^>ZB|3+YLz#mNA6?$WW{}>%sbRUxhcI z^5z&1FM6fisyT+gz|*+Jd|<#U>TMg{#6FcJ$%#r8{Ms9eww+hc{eA1x2C0!?=fRlA zq-$+v;a6hy>m9ri~GgZxq|#@egT;tq`2jEv(}Khw73|2caVwmq{X@DLCY*bopz z|CibO@1(_O!+NSOV7@t`u_f(VI1dOziiS#LP0EK$K#^iV6>tLJDNw}4GbZ^5els{* z2GW@37iijpbnRrC+f>XnFtDjGbTRGCHf*qmbt{eR7j)Nb?w@q#-|l<3=$XrWz0NmW z&v_1n4qomz*#)mJe@K1G4n(%2K`Da`&RT@_epRpy$`2;8wTZfOKY9Z)7>gwcFL-Z# zHBuYHKKcmfHmH3@|13c`v9=0X(i~I5-EBsKVRK38JN0?^@=$oP7M^?cigA3UE>{~H zad-L~wQ0X}UiRtzuI?D#r^Y;(>G6a zKRi9(!elHwWVlV0Wm+H|SeFerqkWVOE!{!*0rzF<3J{fj8zptwt1KpfCtc-9M&g5L=EJYu4xF_ddOw{XQ6pp+2A8DhY1_!0>@L zX&>)X=vqymY9{1w{(xUa5Y3^H_hkU#!5-pU zx26P$;4`7(cr?AOHYXn8gMfn;pPS34Fv6j@oY2uK*#?gKQFGrU?OXx=g2tU(*`8Ry+zV8sU)R)VhO@=P<;j zBlyK(wcz^%p8ZHEq)$?)mpz@qzbw1KnQS)qKt7t8AP3tjip0n7=(89NM zG8U>?d?diHOD4qDZBAL3-sX?YqbSP!^LUP)B-%P&W!0DFKR%!5;zGD~ytzUF>oMo0 zNhyZxHVtelYtGE1BinE{4krzN{B1|&b~(Xb`Q`-Y{f3Ch+oXoLp*xhUIh_i~<{Fio zczr7Lu)CYFi&2P{{75Ip( z=w8ePNbLrg_;N95RLs1(5+2{G6x8l%^+rNVtI*rWu;Ipcga|tbm9DR@M%(l8_7p7; zbMcG6KI$~?ST$IwR(yNOzL=UZLTi5l#9#9nB#e)43Vb7=(lWc0Y=_SX((){IGGC3b zD6_Dp&eE>Etk7=mdY;5uYFNBu_o~2VPOrCzUsE2lzX)YGJKy63@+`8xQS}G!OkG7@ zN_f%&odpukuC;%APrTLUOKpkWCn7@L#1pp1?}Q+nCc4iD)3(Xq}2*#F3zH$5g|vpWd0w{j10t zl8?n!$=QNpC;g~F(kXsx8CSM>g0j%Lp3Khtcd91roKBXotR$zNz>6mBJ%upXb&q94 zC>j>IVmRr6VV_L7TvO(IwHxn>I)aMo(7pCO%ir}}Kd1KYXO@MHHsm>WtPxZS<anSfzglinKsjXp z8GzIpS&;yeerM+)8zWD*E=-NzsQOfC%xMD3)sFwMaE2>VtrbgXg}4)SBe-@+DHtwV zpu#TBtwH_JX_$m9vct9{H6 zS3*1Z)lS;}_w!?OwKR8Ba>kmF7x?%hBl8V)9(UI5DOKi*8k7b#T`#z228mr_eB(_> zN~;Of&6WsyG)BrdLrLn2g|VA2)i-QlR}%Q}hL z8gBQ@TYc|;EQn`3y}XE!feH7{CRdcmOP7jDguLGSiVm-eu$P9|@P`Jy5rYu!@Hzme z@9VaK%=qvN-2m~?%0r^LbN)6Ao^)#5GAg{ii*tV%jWu_mb-_t6o4*zaErAt!(OO=ixUoCvIN51_O!^P>UNHdFZJm6g2k!C zTbrv+nf01S*4&PI-NRsfHeQOiD=&6$k#nauVN0$QScck+|z<*}DBNKyyOuhT+2@%0n`It!*|oGj=U~ z$#Kg!tTqC^hb}_BWkvhp?zYd9Jcg9$Mx}eLVs8(24TLDvC<>>Ak5IFLgOW4U;(EZKy72=^jEjB{ zHNQY(QAp@~r~>s=2D{Z{ShV_Bp4Hc#p_m(5=H$WHFH{qBY81A_1EfJFDYQ%~ZoMK! zzc6rVI#!HDtKFyCf2$V$ZeA?j3rvwTu&t&agDUl!A6G5K5|19dk-FlK!I8pB<8B|~ z2}c`+B3Rz@?c`EBfT6nbqL*rQM&9bzJ5;oEf27O{B?`o;KCJM=2)kBSL7Qj`IDHW!yRw# z7^F@7&}r&9~Mgd)jUp-e`nO#Q+;_zY(`z z zh{;B<4byMO9u?P{#6RzQ$?rx72Zziz??kRQaVjo1_yxblK}YKzp{YpAPWD3hoJtEZZF1p=t57gsHv3Sy{%r>I3owHG5Q(|dKDgQG07k#Kk+4Es}(xM zH9Ad$+@@QmH@j6ERWr-zv=ctBc<~=>g}CtY?19#gmF4TUpK*`9;P%eKSWJjt)-};m z-}~3GWXIl<^au3nF+M_(7Yf+mO}_=)=bHA))A1OR&g9=;a>3WV`PMC@8oy8fEpD(R zFdaVY@B>ay;7xFaYIV6UR!qXZB#nx~NsHh~U+40ZG-Cqiq8-boc~|jTFuSGP4YiCDHQ+jM|Gf!g^9Veqn){h%YSWdKNSPH z0b>k*!%~Nuk1ShNwT=r>@=S3L87@%_X#*TlaqRP!*6c~dmX-6rJEAm)&^Ltzjg+vU zF7B-ZuRPc3=f~T-Prq>xadAUq1z1N~ta0yS@gw?h#QAyU*7paxiJ$`44ti{(ge6l& zh_9s_+SYwoY9{OJ$i5C=4ut!p8kgOw5ewvx)a>!*qUW$lSKq~xu3g)cnuX*iMylnT zY7F8a`Trx_?>xj(wh*t4DcG}?-|dKynYWkC#sX0&{L@Z z%YeJV2vSD~wD%EQyQ!1|bt@)it*dfz`A9F;#;}prMcV~#IK=59ofrJ5H~K`OfhDx> zN)ZC9>IF&`Wh2Ll)FMCR2b6yvl9Yd5<(tRn47!7R&DcheExMkES~npdA8NRZ0*s9; ztiK^-1(3GFia%QmW~9%eu97WTMPGqcIK8~nc~K#ky*#kK z)E9l1k67mpP?G(NcP*isv4D<;$6l#L-H=2Z(LBN>KT|ZPFfN9cJo{C4KpXBz4ZKv9 z9Mnra$toz^d{Hx1ZqB`}$JK;n^-d?qI?*j6)=1fVTS3v{`O<50ei zK}dd06zQ|m`clT$U^1^-)dNM2iWU=3YEB=6(EQzAW;c_15@@fxgg~b=JjTM7jfO}& zBOOTPqua;XY?XAhkW^@H%O*=s{z)xYTFQsb!q@O~Ty!**8BGM0>owacyWZL=*XP58 z;bck1s6=P;K*`M)qAN^y!-Jr)5P;3R>3X&kf%{hB2|Mpc~rkXWh83@#^X1c5|M!F2!Is?V+rnXFqNik z+H6~vf+@t9*hdWcf_}L?QWa)Sdp-{+)X@cx2f`cXoVYzGpPJDQ(3_8i^x;y`48+o9 zqfcx<((Vr7%eQ!F4oQI2feQQ{I}1B))Q*Lt0>*DrKXiFo8^_ZT8&9PK)o5o#B`%mV zl0zq8+@O>2VG35s0h(I_V|l@+*&bgwu~{}<*zj*RM#lONy*rvH5Zq0jUlPrVa_CoI z3g~%TDt;=reDbOhR2_jV6$YEjE{$ncfi>As-d1bSnd5MFbYUe<7!iM;H04 z|90xev_7xlYrZ}h)jW6b`K#awhwC*h4+|7!ket@fqH;71PCy&owVV1rZt}@4Wdb}R zFn&e6hJ0Z&8EDj1kH~B8c1a{fUN3UZ$dh&gNnsgjrTdPphNc6Dx}jErfdxoH@ymrf#^Xv8WxAgfY{t zWXNm%zEtpiX@%c5rA#fcrUkIMy&k^3E=??dVA@TUN0CC^Cjwf@#cS=;W4ScMs=&kt*0CskF`yK2rDsR+brp8d)$DleyZ z#X5k+I2fmEW25%+)>(DqOp>h*KH^~W3r#4!VR_||%pGOvj>9csX8VQvP+x>_ zy)wEa$5Es%?&y-c`p2@gJG4v`zXPLbMbfZrF~hOpe)gMidHnu0@B(MdSNp7dMMQp{ zgxq&QRguBlX&s@F!N9VyDaH*wu--t-dd2z3t?Y+%VifHwD;NaZ4l}8|jASuqP_p(1 zUb2V6sDECKf}cHW${g|U^eHIeX`5qZ()+}@OJKm4H_O&&h#Hc`O7XXD$xh;cxn8;s zln>|6P0Pe{mGgJl0%8&r^J#;~U6(KO0jTpz4z3Q&a4m;F@yfE-*5$A0`ydMwboK__ z$Y8rG%CME9TDfXr8>E5G)D!qW3DrVCC{IbBBm}T^4mYra3-Ah)LpQMCqsAn0V^sEM z1-NwzeeCu`;HyIY@(Zh>HgU+M>zS~=Q8eIJ+Urj!x6ueO=0ykI8aGjUcaZrx<3Rb{ zS~k&F!-HW6bZnbyuLBcwAo~~CSM!jZW_h8lfEzsXv>{{VzMt?`rGM?~r^dnU^@U-s zIKXn*FN{)bLWZ&MVlZ+jtA|cE8CRG@Bb5w(iDAtKEk?%{6%u3yPt#mT59$Rpck?mb zg6nB6+ll-Z-qw|wgM8Fc)!!81vSvLCx}b@sh8YegauI(2a~oW#S#jVxyu zOKnw7t%JU>)8xo7L_&?V1W+l=%#?|K;5@^C?{RsfJ1P_d+BkBPmE_s>s2FZlB}{`q zj=IdOchW#H`zd3^zWROV@Hc1ucSbbHKX;~XRQ3#F+|$NHh1W-)WxuwwvP|xA2s4Wr zl&^4_jc;l2NQSdk;^BeK0x6@JBrxswD#(1s>X8eh4ttWL?<%Qtrd%ZN2WQQIGMu3; z>6C3_DXL+p7AIhUOMO)TCJ(}>@0D@nLUDsc_Ok|@;pCBAvzCILTQU7qEBH(5tw+L^ zrT^9L(k1rHii@oymS~%*N{w-F2Fq_?FIw_t=7q+(K#I*f*-y-HOGGf1XzGylMqAc< z7wt6WQr}QOp{;l)T`Lft^2Sp#K^a-u0z4E-VwHs8SKN?F|L%;}A()R$z3l zs1lSk`>8^8K%rj9_}HVH!+4``{<!Mh8SuKJmN=wH=klQm1I0r(3OA^d@3O?h3RnUmq{zNaYOPMFP{9G&AnL;;65nYu-&g4axLH||%u-yxP(k>s zQ9a$Npd!`-P04AH&+>Kj+tlhsG!AJ)qF!w2Z|ipc3BhRs)`#CQF!h^3F1-VXm4^&} z<*iKO4^HZrg>e01I+Iem?zJ=5@S}TVw!r9?RUZ{0rF5|y;o_+Vgmy*;_ou$C!TiRYP z&1;~4F^Ksz#@Y<&iL}sYE+7bho*i!$n_3UuHK`oKvT>YTYDz@Uf8x*sIMHM5FQ#(X zn#tmUG+~zU&hivwuoJ}#wgzf-jN}gm;J(7jA+X2+imn zG>H;UjAA=hz>hTnYnzU>6`^G8lwQhben%|g^UX|w2uP}hu4NE;nYZIaa_&f-t{N?o zfX(@mgQ7={!?kLut4rQ$3sy|!;1EoAdY!V)6mmSbLz9&#jxfi5hyz5(B-Puy>&2b( zAa8mX?wVS2Q7QC;N}T^>91qLCc!gx5_#Cl>?Wj7BIL4W&%)AKw{`8`YY~ zXv%96>WgV@rUWx4N4pZN>O=lk*J3#mCh#0Q#eF5Gv!YDOWG#z!-a9-b zYB*KIp~_~&n`e5&&BVM=6l4b)K>%81&ciJ-VdSQc7olA?QCVCS%CfQNlWr`2QpA?L zTC9Vdi(u;1-INCv>E|-U3SXC$ve6HsO%gX5JhEeGBfG+V5OP(f*X+#ixr3W71=l;rJ zhgn+4RVocN`iwT>^z5N5@X33_&(JNy9l7ecp>YE&Y|}z5i4SLYTx!x?$o=|?JSfvi z8LAm>Cu5*u-B8BzsU`eMJlWCarsb71wTg%ckZG)|RFU{qcT@wWivnbflp7RqkIuUq z&_|9mhPegeF1O^LRG&)tqvzbC{UTYW=R~Hf(Z;-}e!a6C=^a7qVVpN|GNEGS_-HS; z&8$nx?rR*+ekmRO@itjRTx>+MQj?92?Cma9Bt&HaqdCFX|Lzp~R9_qHK=93Uib?o5hCEC|#vIk+L46_*I zGn~m=p(=XY3pO>cNx+OB3XEj(2<$pT?FB3gZVkCiDy|3DkwCU>@1k@T((=|{o@bJx*RCQsqE+Lp;<7&n7CQp&4x#jcO%TuDY>z%;f^r0=`k^S-U)M9x0ARwN(K^_?ltW%3+(o)7tWZbdY1VP9uQW{$Pxr*)Nf z=^SShlNZQ~-O{=|2!LUl^XLS9TF2drPEa}znWL6!^1d)bw6^;KOUvo$neuWbFO~Q6 zam><<6RrW9f8!pN%OGBUGFeIWx>+oa@~m`ACM%nyef-V|<9CH!odaIQ;Ee11Yh z$OD_7h9ILYmB_cti2E6{oW-vsTi+!+bEbSVlW&l3@y^zF zBtXd^8{o!xc6DNj^+ZiY8lHmyZ)ftY@*O@jJO=_48=ATC4YQJoR7+mTcQUyXmE2n_ zwJ5WmcZPeId>0nzXD^V+CUlW=70ScsGxsz3UikqZ@+_Sis`&akAEt&1JHD=N-y? z#N-cfs2XG|Gc(s5Z%e?02NS4x5=bS@*q9}M%H)snfP$Jx>m&I98&A=(SJrVj$FG?D zIVN34S_)+Q6f?~`mv5Q;Z!E54!I{|lPtNd*hNGzr+#DU;k4*kv{;zkho?}Ew8$?8+ zZ~T?XKO_9!VkyHnDiTu&7O1lou(TK@#1t7#6mBzKiaZ4`qC}XYAxX`g=JYSl-OmSG zSfLA3Vt8<;M^PR;cn+t7LlEo~lBSSC52kcO3WJJE+ZKTjtkFpf>te7{$CRE*A4+Ca zaKgf}C_KVZCR6^J30K5{3)i11eU$-$gexZK*fMY}PS3|%S_Od`#JzFe(UdKiGEmvl zf99qG8Ee82zuSfq3jga~ycxfxvyi#wWKvHbp=5dWM78IU8S-w;lwrz9w`_j@w9H&3 z;R_}E)l352aVuj{gmG>+*p)%7mRNN^u55!?x2+WG9G7!X(0HaLSIOdrun6k4n5}$U(r*Zf+1)sKc zJhF)tcm~Fj>Ut8Ml%nfq8dG*xVC31Qd`DtXF&8?lxiZQuqzfH3mMk;~eBXfYLE3@N z(&|~IwxzJ(0_!oFGM6cHlzEhl@hc+V(Qq-xMm#=M6?D^dbc>iWUuh}>-84i;r@4(i zJSmWp+0}r!+`zaGQZ?lb3_R+vhikq)Jq>gsyJ4H7t-?*jV68o zQ6R1d^7lI%qq)=L=E}YGo@YG5J*0Q zicLQLj*?^z@LCqYr^i!?bW1$neGfw!4!3PBRDm)~KbtZ5=WUMvaGy^`n&gR;A0h33UtdyVbUp5P*`K>&%}x ztF~RtLu)9)9ZcDzbOy7FNvKMCE+%a_tGf{7-DT+6krE|c+Hv(3PfO{oV_A(*8?f&~ z*!SC3iGz>=+&LI{6Z5nB!6mmbVo_pMki4ssJ_cwS4doH0JgEHDEv3o_5lXt+U%5Hm zMVa*tEme2AoT?$qlT3L`c`A_8&6BY?UHzq7KhB@EsN#&ma%Mu8=y|3*qx`Mx8HJ4w zC6NuSsl3r@!jQ>r#I4?8h@5(D>jQ>qMd}?wvfqn!SD8I&(|HE;} zc_7gb%2#=EfkX{6y~UK*mA3;;E(|KR$pwE`^~nXN>OQ%6A7%I;(Bz^7Wyd?U42$I2 zME~OJwpN?l zSF6g;y(_vqt7OKad7f1HnJGUizfdyAIoU1?Dv zw1VtdIZ!VJwL3!X;reS={)yoa2x7R6N%D?ui7yExYTeWG+d`o}EYvI1my(8Rbadlv zb%4_uC3~L?>(Ft-Ds*_$g zpsf(lFt@8s9jm!ETfPlfBkfH?up?P$cxY5b+M8A_?QuY35zx4Dv^ULdd^ffT1Dq)J zBldC*cUu-3AKK3KM5#YQLFQpjn$RKbh>#|_&BL7QQ8LU?g+=?9H4T;udu1rcBo^8! zG`R}Fl64QWnt_lV?i7T(d-;Ko6`ivenEu-gUin4n&~z5sBQ&E*gV*%h3|=PKJrV5e zV1t)|%~|XEo@LFo&iC$6kVY122+ggM){Py^jWt2qUdBswW1Ra zD2{+umJ^-KaEnfi4eT1JZx*6m#X`-Y)fK64R<+c}0j)(q2bH5f++255AKzTBk@E0p zSt!FosZd)*%4?{W@;IOz0%|Wuc@1u3o4)y9Bkkd_vCsw<>IiMDNPBatr9BSlFa&gX zIobpM!yUEfkem=YiiM5{aVLBy6}uny)#UDFvb9S?HM1@2j-tn7^E?+Uto0 zxF;gqlY*@$=FvH?MwmGh)fkgqwItl-6grKCP7a;!e@ym8xH^5;@n&Fvd=^6fqyN6^ zM8_>^$nR9v-Uj(=mqDQOSm^A~`PH(4c_Cvhqif4;ypFt&+ja4z;`ae=@d|)w*TswR z`T+PZD)&#Q+>0xZ^1UQh+Ijj%kD}7%X$27>CF^TF|KT?wIHA4dH{k65rU&P4aV1*r zD%*xc4Kz`8Pi2wdT9n|rfS$@kSj9P-S0iq(S+^|$VDF_jp{8ywdur$$66o`i;yJjg=#g&wL>n4ZsVt9F=Pfcsa3`)IH*-ORjmRDC}4 zZgs77U%37_^dt*C7J7=3y{e&nGxM(eWXRHbvjBb;0YB%K4C#xEThli(18cN6(1hq; zWTEFnf3Ik9ps8Am102xH2flZ^&bqW4jv4{Es4U`Sc*jRsSIk5Pb6f^9Sy*nW$R1{3dR zi1+jAZ8R7%40i1wIX#^d*DUl!=-;*r_p3pgk|X|C-K8MgcPQKUw#!g! z#V*K66Yn-(5amY{<-cwxP5e@#r1dxILIvc%Ao5>rw_H_;y4y^Gr~1}(CXr9l1^HB& zsYF$(Fq0@`{#J7)VTGz8)NuJTi2||_#R|_RRwr`+ZhnXL_u*=jkffhqQW;YtYL`k9 zlKAPR+IU*GLwRaG~=r-jeh5pV^g%Ao2aF{j+aoW z+c9;5x_#yHI^9Y@1)z z>QbgQse4zlVp~kNR#^!mCu5vjh9WGlBrCyz@NXr9F}fXqGOVbk40GK3d@WO_3jVZk z&x_j3)C1L)>bPkO*;Pn(Z+u#VPb=Ky)icm*EpOo(Ej7i|q}o~?dChyZ4*4zcJFbgx znc7gMgKfV+Qwvs(+X0rVv}!X_>33_X9Zb!t>na(kX1I@3MxJtQ1ki8ZQV(J3diBss z1mIV3IV^eU*Ki?^Bba&^E{r($H9aG?HbNS{@@S?WsUA~_kotvLj`$sNQq|*`daUke z{(w&_+$?P5q}Foy7H-f|PiE=~>M4~E-y+@9kS-Vb418MQ76F^Npygk~;U8S_I>^~f zJySi$?HZ1tl-=62al@8+K2y(CyWrD>_|(JASLJgvYpxUPWmf7XOub0Gv`RX$+-_By z70hB@Ml|RbD2~JE>V=9zh3waJN?oW^`A_AMSacf7nN+qK}p%ndP;g(psWIJjqx0x z=)J|%H|Qiv>I&aiCRSy#iCM|4UgBM*zKtdN7fR^A>j;Cn{`+M3Yenh@OnncF^|cfe ze!x-vn5iFP84H+}L@Lpq08;qK1`-+LFnuS%v{ri12M(=@;@SQb>_FcrXaqUx7S z{Y?FelI=ZI=~SPRSTl##7+;CMf7;Co8?+bX7x)SsbyI)G)PJkryZr*6wY06d6)tQm z|Hp(&3SIC!=B;Mxf0_Cp^(VI<6E2HfkROY+ws$9ZzXCjsxcz8rsoJK2jh{m5XIHRw z7rXqQ1@@sAVH*>K(Lzj~PV0%_d$|p`L8u1FCo49z6V^V6wXa(~nKz9f z*~S*CrJ(gkXs-9Y`5*!TiqMj6JMh|IrVZ3oeA*J9pv&~Im*?w^``5~i=P$O?hB0l3 zHk^{Fm3VREhP%cpIdK!*NCY>El5NWH;Rcr8c5UW&^Q#<8;9#wdW7=qKe4t>0*WU)} zV_YXR0KR))WxRO#nYIJdw$*kFgm+q{fh-5GGXmJfPar!@3~!8_fbxH5x};eAQaEil zrcKhOxc$zw2Lan%q5%}6Lzs#XfYw%?nn8{OZK^5CHK<7pXuHr&o5i&0+MX4}fYzJc zsu2U);mtvKd-;z6t!V7Mae8gtasFx#Z9dZ)wFMQr$IzCa z)~;~2gtIwqAEqtQ_VqDaY7pbg!jDDUb|uQkQEU$cwm;LBX$MfUofoj;>S6=b6CG;S z0Q2{wX)BqwLTj$D0UDR09b8qkj_X_w60AZAR@)BtR*^&Xx|IodEdoBMyl!P+DgSqp zE#&+vG>!N&ZoiXUCPCXWkmZWJDhE1v6P=c0TAS7$DA1WvX`sUatVaMF`~*6{Z-7lN z_MeMKVuvzqBYL?$#pR5jNY{>F+F@A49y1x)DOJ`mURI|a&9o!6V*+V-R;3!|0FFlh zzxSizS>hlxJCkaOuS}+r?HiHOiA?(g>aM%3ke(Q$ox-$}aJ{fH#C)(i9Y|?s_(9wt zjsbcCPL=J5H@|6TGwn?6oIrM*Q>j)sfb$W+1%9+L$F^#{y72q*LAFusftAX_qtYQf*Isx&oi}61%gJid3@|Uh|$g?HZ*4F z2(i=KA2?rYXCLA7DO0u6ZQ;Tp?QW)R*6yiPx8_wtwTNK(n-cd$9bTpTugXDUy9i6W zAC>q(@Ce#V!c48^1+G^{om)eW72c7hJ%ZZ)t8JCNijr5&J6I4uj)s`msxW8q zYGA3VccIc($7#=?MxOOOMe@R5$=L4e3bo^_$Dq>Y{{pJ+MeiWPmt6MXonHx8{tq~Td zqhxv|N;U6Gt7H+?`5rgzO9c3p_ads}{cZN9YNX*-c?ONwqiNqV?Q8A3Kr`s3N@vg< zzz+!EM?W*@CUFhPOkb;PW6i36Ga(M4{lc{WYQK7KtT_@W>Kpk?&MM=D%0H?H^kO_$6~2^dPyr3OpphE&yBGB*KtN5DPGTQWDW z)N;w(#Mc|~)m6S^ZeZ?H)UJ*E?3J}_+~yrz;d&PC6Yg)jPO=h`l&t%$n|VQ&K`6^$ z+pjKHn~m*ekBS?70s0VxK9rIj%WU;|RF!0*m33)bcsL>*;hlw6&She0UlQ?}0hleXS*?*|O%tGg){#mg#9NGd-2bC0g+FY!;q{pLWIqk16BRLKdEnpDdPZ&8$l-&P?Yy7I_I5v+yF6uwE?IVAaFkEW89u zS@@(A8#te3EWD2(Tw6A=E}7}TH2?7aEW8{)Tb9eMdLk&ca!wW0ARoi(};ZS9gY3=rejIye`WI7VZdd z3^GG3rdQz%(HY=j2=H(pGejpm1Bx_jP4I82fqtE?rwjj%g^vgy9VF@8g=n>t&K=np zt^GIze|(Unb2qBL33~m~iSP-C{=}-G=QmJPlrmqUnVf=HPxX;9@7gLVN|`U!2+lxg zXZlE)cei=l8?l`e6dSi_f6-4^&!*I(u{z!z7`}jo&kbJ~ys;nO)4E_gfgxECq^ z#m9noQEgikZ74r~M5DU7{rW|V;RjjxzVJgqa{KuOixuVe8xa1A5FYiB+s`=`H%@*Y zq`Jz{??4Pc$-<9?p9-R!X%#ESfbc9rc+Q7%ILD%B*fN^Y<(~Vf>8y5aL{5n2;hfnXx7t#WK_Lu}qQj^}{-~Qgq=q(um3;azv|4(dI3rRZ^-2 zUKHWQ%2KL{rcaSljgunP4T1FQK_lH*ghjds34vx*90C~-dLe|~K0=@wjvm4 z(eECO^kb2_NPQ6P%&J&B284kKVUQ2)aE`r1?KD)H+)yl*^Ls8}}! zgy9Hbgb&>`IC?AnSV?u2qYrA4(JZocWK0m{%&Ay8288hlVS*3kaE@Dw%5gZM5!sGK zw&7!HWCwhDAD<@T6OK`lo$=`_haPGi+^E%(z2%PA$gV81OJujomh9)5<69_gL=%xc zfUYk2G<o14;xVu&q zYvj+Y)U172ulJ3tV3GYI2UgfVEZ6#0H!G-^hcWClBkYzSSwW5@tJj)s%3X~p*Ob?q zExN2IlgCwc2O+4Gk4zpHO%=sAJk`Crsg4EjiYehb?lu-_jT{^#)lsBYVN#$IxON2B z;Ug)q7)!A}dCEd$1B)7nDkz&vD|VKbL21<*%&#zva!5y19O$eYb(H`5a931 z8m|j@id=6#_q`S?XW}K$k>gn8=*aQGSI&gXpR2GgR0Qz^gm|Lwbs+&!QNuM-;Hw+$ z>o-tGPGymkBBuq3_M0k>_6-PUB80PiMEgz7jW3^E9yy0a{ur5$Pv_y&Vu!d!&o8gl zDo^2#>&Qhca(?7bm96racUxC!mB#|_QiOL|S*tuIntCUt8p+Moy7EwS6W+YZ6(@D%YoF4Ch>Wj1SbS}Zo zz5&7oOo(hqH>WyUc$x;6?Qs@)6lE)}w>ptdWHIS;uf)cwsrB)VxkZWgj%=D|fkvKU zkteX;V266`Q`@0$uk7Ia$(K~F4nK2xU-N!z(3kFUePde!?^iS5^p58hYZU|n9J)v*YdiK8 z*qIjtM$Np*BCkc>qGT7J-o2O#C0mpbVsRqNS7hI1k++f0z!GJlKkr>NL5mx~hb;15 z|798q%vWFdDZdvh7KJ`F`YIEb=k3bT%`;!h8WhNTw32^$MS} z$fxKldNlKowhlPJod~b{HH&3l+c`4=;l(pP~pSN!)Z@=fGF zZWlB8Bt*$jM@lksl&IQ*!lhJ7`oguzD# zxTwaWN;K?0@NMvdnM|87k$#LtBS^nEX5&>vUyNhHiAUk2)4sJy^6G zR-52hEwL&NR@S`qAt5X3?GqbA)4Et`MsMuu5MRt@ElPD5F1%_QNWUo=UHh z*TEnb9e~0NaW85iyd{ec#?o6lm7ckwEtA!QP#Anp`Cs`CH!lz!#-c-_!)=Goe4g;*DB+iZL}IxiVCkk zfknrmUWzBgcq614j4tfRSXPXp+p*|2u1%q#wa^r>@Qy6HJ+g2F!+&=Khx`9wxVy|X z+>P$Sq7$Q&Y8>vm@TWxWNGA{O0H+;`x3y(68O5Tx(8Npbc!%J)ej^zCc!H4RAVB&x_u2+o5`Znxqo-5W{gePGz*nxv*@gnm7re~ zaoUST8(cj-Rx*RVLY;Xm+UQ*;*O1;bo9SqqhBu@}7qaMltm8OUv)1A57i%nL(M7(v z%+IzYvO1T&S#*i3cemEjS8K4!G8Wy(7qG>~+D>$T7F~`t9D_byBOe*^-g+R59#B#a z^BVDjhMQS5UWx@*Qla=N7Huij%FJ~*JC`Jju67;R3M(gAG{vH8O9`RJMt1TU7HutM z4D*_nLX|9wwqX^=@zYv`Yqo<$bEOz8&QCT93^uUnx>DBGm-DRc<4_jeSgOX7TtYD3 z5iEL`tCN`5(VEC6n{7kJ?^yImbpDQ$RhWdef{V!e{$pA6X!QM##7#R}#n-(^L<=1Dj zEnHm}vFHV^VFHxp{lmE3B6im_>m4?4-^*F_Ql#rRkJpQG{#UZ-pK;OP(BCGq zaE&6a0UXh5ZCmIy;pO66niBDB^BTS4%v^I^*Y^!9dYx;esTa0&`fQ*;Zer0}uvaZ1oEWmu*qSE`t(o+s(kFZmX3a zW6{65E^?y5aLRxMpJdU;(PsUe3ND$kIQKuCS1fOEHPEvx`c(9}8qX_S_*2r?QCW)` z5!nkY`n;>=^U(tqe2GP0#Db1NpE0Mf44tpA=s%qMxqdf|b#o5O|C2>u#Z9&1Q8Sq{ z<`Yx#Hoq(nd){Et*KnJwctl9%@a)k1bc&C~Z?ou|&Rc}3Of&8fD+52oDk*pT01j(*6Z?_-tXkO!;qy;nXxp4vV$%_aSWML%-qVXa__ z`x%S=%UKM2mD$PFJjVNyML);##kR7SZ{(?9(SNh(SA3(osCcG*4On)Dxt+Q+nM)e; zjPF?V8#I_(r3TZ?mP$Xc==Y^689%>kuk;g({)m-|-AQJB+7a8YEc&x+4JTkRiN%O( z4JTBw5R1uJqj-ll2kk161Konpf%k<~73>(XFpH_E>}|d3@imwhp>~YLBF_85dhNN% z4T%;9u3cG-mEvmFbL85C#k!T^x}>emQP$op*0U5>vz{Z@zARRUwZ|6M?r6g`r`*(e zO%1zjH#6Oy-Po8|mqG^Al9X-%{w^>jY0 zx+RMZ#;V)6SHxeEVeb)*xt29eoL_-jj!j%)5*3Db4@!omc>Tz zQQr*g1RC41*fyngW~GszbH>5(VeK7RYdhR(Q%vcvUvjim?|Z zYI{b1C?#|jJH(#Zcx!7si%j=qv6(2xxKefaEoq6CrjC^dcOxYPuZ3aV@p|Vv5OZrYqlq#)3w+Qu`i45UAo1hBb?s>ZEH>@sq1S$7F&jO ziuVcfb@oiO?`cUz;}*_KMoZ%d^0=?SlRIEx)xx>@Hd+ENf>N3qxu9-2LYi=Bv- zxAR|lQ6`mI8E-xatDnYVrJ z*yT71$C|Cb5$l7FeStZ44U1icGY|Lh)~snPw#Ke!v1_r4yF#1d>k@6CPrYy5#9}v; zZg6N znXyM%>=7@{NW->~5_^Kh9`jJJ8Phdj)^}Sxo6H#+dYZ+aEbU9Jm1k~9=JaOy9E&~U zp;=442A$apEcU#AqzgOZ`l!aYG-5BY*o&p-g|30#D=hYp(yhoTafr`jvHCw*>{aI# z35V*O;TtUWnse@VzQ&TY(Lvv4F}^NVyxL|fI+ZVR#olAFcbwgEhq*FhS<8!k$YSq1 zJ19I1(3okCr(i0RSYXU+vC1bb#uvqkcN=U~xSXG{7+($B+8m&5n|S$`{P&sk-@h(D zIkt$j+wTd+zGAV@V_&-^$NHl~Nsl(MAgIqDK}tfrGT6sMA(;-mDx~~_5XC%so?q+- z7W*#tquW6Ur&^Q@Esb$*rX>+%>g%Luj`0^3`)}-5x2dmVHBXSnsLCY92+|(mUEEkj zXCcO9Y%PPGt7H?+Nh3Ljg&9+kdVj~FP0%Aa#Ta7|EH<#XSTdK7N3md6#+VDeWUeW> zI-O{#mgaHMf%Rmp8|zh$=9gJ7DX|J`Nhk`^nTu$;b!OI=vEHnoTW9u12Rd_BEa=ji ziwRPq>Iv+W40Bcm|7!X+1U;Csfe6|~cY1}OeZLqY?^kRnV_UGT-1@##Ezm&;fHRW0 zwp4s0X7mJUg-)j7IJahO7#r!<3LUGJ)?{ORB_E*K7{*4S+P5fCIB%lyjE(gwoA)Z) zGByDbZ{<-oPkRT(wnN&39n0HX%1(^!h@|^F7ByYUB*u2eV$Ns_$y8W&H^wHrb6=d# zal)c|Fg68?4k<30%uVfR&orc)v&N5gRi_1TKQx1}scdFBEnt}oMqK-7N~Cd4$bZ*? z&1P&Ck{;+-x-j))dok941s!@oGG|O55~+=QXOr!DcRi1>MpxQazrK*M`DiW2f|kut zwwSR+NYTMmFaYqRkvz@48C!xy9U4l0tiWQ+7~97cqg9yw8C#A8dls?}S2XL7l(d&= zYdurlbxWN8;!fc-k^Au)7H8}Lw$gTnr@Bli3AY%OD5-a4LW$gwO;MsGd17?EIn%=~ z)ELjMPUz{582z#&W2@ymC>c;FmDrGI?r2Z&l6=D_$nSgsgmUjD{ z4_k3)0nTXj`E8KfBMS5#(oQ(OEMpm#bGtpFbp0T=M3kb~phH@Rkk-5165&B0$b~$m zNG>-(9fF_^b-R$qi$ahqmrBxfGv^Tq^GLTVm&zsy@e>Svcyy|zbkhtl9TMpp5*)+W zQS4aP>5xM$@Wn;k@w<+IUyVS?E|f%@G9B6G1kQjd=~gH)9tEd_HGs#_=ewA8)u&Kk z@tQ0Q0Jwgb$dVyIOeO^W*Pn!l3Qtj@kudxlf&Y&ZBHaal#R!9PUEu8+UYd|GJ?WjK z>uscW5Ba0pNS}L1zkN56KATDZ+sNQeWXn6ri1SJ0PBONWY%q7Ffd@`CWBopC_1pnKKm!_lPkAWuF?crYv|At0yCOhBCOYX>_VvQ!Ekj1<{ z(+iPFWHJ|E8?viGU>N^Gh;k>{?H04X%HQr|@}^`OnQqir4sTG@9%Dd;8SuXb_}{!v zvZ%nmCom?Y85kyj*(%bVtmdS))GHb=@M&*gLiy9aWEs?^Kz=&>yB|V=zxL-WFNZfF z)GcsYF$Mn9`~X?qNs<#4eA7DU0@4dH)G3`Lql3v7bUPT#Mo0<>v;vz9=|c_%edowl zq#g9Tjsx34CvO59Lpls#W1z$`&VB@0Pc{I}9#DHDSW4j*(FpySjHYOXLmZGd8OrK{ zKo8Zi9bQmWSN`!(pnf>m=m?9Fx|mqHm{__PSSUG){H};q9O@}~yBz%hId0zQvEw$A z-;cXRV0SFoWicZ3W(bt60*N_^Y&IBkje>#g> zpbNBggcjn^{#XRm(OGKrWu707L~LDk8+o zpxWg?>N0RXe}=Ahg{1}8nGowth;=5!I&vu)4y!I?EV9=Y&FsKd)pRTeP^fGSH^U2kv3_4CX=rst=-JqRNbu$p{1S8xD@9rwY zpm}h*X%*A7d4pbN81(ugm>0qq94ZDFFzeJ#asvX@0H}sQH4*58Q28O~YY#%ddzkbi zk8q%ibTTH;0VdD^CeQ&U&;cgU0R~Wf!ZxS^RUNX%A>Ulm==EUqTS^=KDX91~5Pk|O zKLbX8HjvS8BexeB{SHrPou#3@1Qq`Q&|ZSdF9Wn!0ztcr+$})6hurJYbN})Hd7wnk zeG~qA3o5(`J@9Slx$gw*xesvw_dql8?jDaIB=_#|G5qxjRQVX1^uOTUr&a785Ap7C z4|&X2_jnwEx_6In;ID6?dwc^8`8((y-`AvjJjA=lQ(go4)A~UEtT>SWLPEfsK{ViZ zAb*Pdt%%hD#z5Ycx2YGLc~On1G(x)2sAUxIYVzu8^6F|TqAT=>fACTKW#|(k*A4In zY4iquuTDWJ{}1A-@aVCd$?Jt+iW`G=135uqv=`}0>qsx!*Mgz9iJ`ZNp*J-9Q+N>7 z%?P3}bb{9L6I6y0Z$noQ@!SZ{XrSRdig}mo;=_s1xuC^tCZE(PH{^TT=k?0{P)&ta zQ4f_ZQ6C)yO4<@Y4k3N$P*P8akpXlB8A`XdxP;**;Nd3V;RaxQ(pyhg=zCuSbOl;W zl>8gop^_J3Uk$jT5Z@LFahKt_wl;;p@$|ciArvEoVuWtUd$P~#L%JCKfrVX+2_VLH zq&wZ7^rJg~2ouR5x-%I@cd_)=txYkuHpST56k}_%6KrjawA~GXLS&F3P$)0Z^1MJl z_z(yk@{i>TG!+Dz0Rqhgfo6d~vq7LaAkbcc1ky(lKLY*iQ=nhU6{raWS_%T~4Fc^0 z0xbi9_5*?T4e z0sVRniP0qKLDxdpOp)F+P5RS}WuO{pN-@xsVxS>KjD%?yFoq00KFkrDJ-tas+!cI> zi1-$xr;p??yJJtMy&j-_uz!O!Hq(AElooaN0l?Lcem&UW5I}w?bo9f>KzaljOphYN z>F+F^eWZzgq=|l{X@il54SIkM27>cYfU?lp_tWJVOt&cQvFMguj;-<@i;nQ;u}%OX zP6m&43V5tjL6Fm-F`Wq>>#QIi3ycwV_gEmr=+gax3jrRBj>-225W`o0fDxWv4E^B} z=nt2I2$w^DxB~jam6drcE|7bF7);0e5C}RrolveoH-bR7fIzo`K$}3I+d-gC5NLB{ z9?Mgp?R*NfeYpbN4+1>|0>NB^J^}(g3IaV20zFZg$MO(pBHgJ-poKsWGrFzG zVzBc(IHngM;CRszSj9|mF%w*jPJwOAY8*gU6e0eT zfp`l|rjhA~42y-EaK2Z5^%w4{=d6d_%uDCsT5NFS*S=__?3gQV^j&|8?Gw=h9( zVSw%mpmB(hVX7#AUaEs$OAjio<`ja?=nW(5*)XJjfOr6jNdrj_X%OipZ2`!Kkp9xp zGSu9KFHyOvc^wR)CHu?T%24G`JR2t{mXAXr47Dg;5kQ0r?l z?v3s%51PN3o+GTyOS=;$?O~ZVS*MA4;#~~l@MKs{qCMS2Q*_bJcNJ|0h&B^Mn+2ln z38KyREn0s%z~I?twBKIi9wm{{QX^Px9<9QBUO+mXFPFkWN@|wZxKgMFx$}yevin;KgYwn^qP%;IcZk?pm zE02=CIMm;90qKVS-VIv#%LSwtd_Ei=AD*O)!Pk#r!DziQ=(tY$luc95*)*jaYa()% zngM@;bdy$*I%zc-FD1#&(poY_N|EVOn#_?7CiA5%SuEwqa;crfCF`!qN>k32hMXbr z9lD&|NE3|P*mCR0T>2vYyAgwe)_x)5HF_V_XlJ(=Xdy=PTUrfxq<)BoXN-PPva>(xe;BvC#I8b8(7!sQMHSp{8A-(k(^GCjitFJ)pMJ4mYTDgh=pIfO?u2 zRPKZj>PmO0-9^f00n|TwKwSefPJ0*cH5MJr*isJWJa8}<0PKsv!TiYr-O<5tw<1~G z3f<48!9_#?E`tVlxo3mxVl=p=rfQ~ePXgoXXplk{sa^#=?rMu#RFkS|Q0+<2&U-$( z18~Nezhev^B437o{tvyvJn+MH!-$c4gHWVL3Wbv1gCKqnJe85j^kdkpL9PtPB8__geyFN~nc2pFw7>ogg{6r?vsq{w@`xR)%iVIK*e?d?A6^y9bKV!F&dPqHW zGs94?@;S+fd!&i`ZkBeQJGN8WMTQYn+7+5Sm0=W@cApd)1Wg;hOs@|;MxyXA>uU0T zeds}Y-}z+s`p^UPzDa7m+9@@3Z@GdDAA66~xNp5GcaL{U^X`-uZI+hylJ@D8mUT)8 zbV~87$#r9~Fg#oaG^CdEA)2j+hPCQyayq_PYk6@bzDQYKr13=>8rj%hQX40-AHK=z zZ`%2rIrwIs{$>MzvopRqM1OM_e=~xk?7r)r(h-0UiT|!s`aOW`KBZGSSx0)TOb68| zm;LfHNPHUn@621wrgaq=1={?QXiYvtagRqji2ay?lm_a{fl1Ie-SAo6>8FgZ=$lAJ9MA?L|M$))lza+N%s zTq}d|2Gopde?8X+>3 zY$u%$O;zR1ayaSw3+by#6Dj?Pbk(GzW6&C*(fvrm-Me@10sr?SbS@zJCHczm>|f}O zj4T3Q!ad0}?v|ws#tg~3lTdx=QL+P?AzB{-RsRY7i7NUNwLX;hCqu9a^)K}G?Vu+R zc^ZkxGe~!NCaIHWlL7J^vZdTehRbuoAoIyK@&dAxyaLZ%v>???QTylD<=d*Fx-|Auv$FB1kf zc{#Yd{VW4p*n|=`$NP)vB?h$>#`Lld0#)hHozhhpu-@7!T?;z7PDm({4}|6!2W_k* zz2#=hh-}$AUO(6F%|xM(@tP9=~nFUkCJA1+;jnS zLA8$pP8j~WV-rKjMZ zXY+qQ+bR7G{`q^s$b1}?SA&t)kZ$r?F!DiUh}=p>%4srQ&X9?68`)jXlG$=Q7$tfw7;lqah9?D-vz8lPCF4n> z^a{MwxZ(Q~#B&#LGk+mBHe)sqlM~=#|(zKZ<%Y(jYcY^`FB<&s|?Z?ZY-MfhPe5=!t>qO$u z`F9v{KW+pn*F)@4QiK7b>R<9L!eF_0ug^w%OHqQ?eAC{_3->S)_dC96kML?TTZ_0q z@=be`7w*v_?k{}P9t-iHT%Zk+AyASh6!X|d#Q&Xd9@|5FSrY$_BJH1i)85(34eTQ5 zMv;8ep6o??SCMwuH|;6%?xniwpiX%Yk$ATR9s-h94TOyvdgLJz4HiLAx| ztFdLUT4E@0DaLN zj@;+M6EbR+TRiNR5Si`)5>72c!Zk*tPeODrva(882NE76GTRd*oLxplqDRjtYno4a zshsg_nQbD|c_88ZG9=6zEi(sml#-sQL+sUyeS4;yyurg|Y!rzv4-iBg=GnOp7qzt# zwAEY&t0N6oM|n2A--)b}z-nz7td237-m&s=4#Vm3oMpl$-R}!CQ=MfSuspa7mM6+5 z5m`Q&nDDOI8zTS(ue5R3%QgHldIV87Q=Z^9pMNWs2kaDCEXm#KUbuGG|)fBH~sUy=wBexe7$8gM#ienU_nV#F{buLPIrSqVE3+ypPenXYQJE2+5c7fqobS=yr=QX#{xtJ~ zGo0_ThqAJ~K~uxMFOoYg^$GQK@~zprTTW6)oGervack-_cjO!lJEmybluLjSIrB*v%Bm3ajk-S4DkRpeP z9HfONempnWIz>SFhtzA;cO?F5m-tx!MV*Vk z@&~$89cDj5 zuM%v)Rr*m~k1+P=ps2AEQ4?<-%dF7Z+I zhGP(HiFa7{OZ`=9wHn=``BcWQgrg?5(FDy>VQ{AQBf3gke6VoEAe#6GoG-3P&R{&1 zXliXjB8wc(<%EV@1Y9flhO1CpF?`!dxeX#q9crXgn_HoRSI`|w?(WLw2W8UPB@OMWx zs?Uso$(=}Ae+wdx9dUpUcmsbUY4#Op>Nr_^v^;olt|Dp1{U?{|uM;?3QB_XlQ$*I7(0gIPz@q$Hdn zy@)3yakmcj_u|lc^(Z9O$VtUWuG<$V{i+y$>LY$u{3cl++j%&NGMT7% zTb#1r)vAGwtaST}_;~>wOr6wN9g*tUN$>U`bthfT)6nP@D$(xyngm1>e`GViHuuO< zcs@A?w9Qe?c{lkf z0DPbu_}P#*uu`zNH~MH9y&muUBW(SlY~vBBE~;}bKA*3x_o=}ikoq4(0``I~R}3__ zuK}-RCY)mUyoGm-PWtj#bZZ)sX2M#*N*-xTcs)0okPs$PxSwK{ z%|;>*xYK%G1k0`nXJbyBX$~wo(-|IhuKE6MUy(h2`g0C5(*j_XSeWN+i}4rZRw@HA zQj%(U6`ZO;v%>RatWL=3T+)v-030eTiZc+j2#?8evZ4qs#mdAsESgnFnqB`z^RW$z zX4Q&we8LdVTq`xIHp#L|0PbwStVY8rJf+A8JLZ?c7Gsbwu3&(rKw-c^robNYj6z{9 ze+f|hi{tq(PM^O7h4IfS-x39W36P9!MKsHb6mKpC2FmIYdgeaCJZ*fGRew-8D{LMg z$?6dm=vgMiCAEZ{r8%j4Fc38Fo4BCx(bGBry<@yqo^xcrj4#;bIAO4i;l?_-l*BEaDQqW1k0xcSV9*HoGILg_w8bf%>K?X1unIO zuX8{muL3|rB9^CF{!1dSgH7DR{;fnh6!9^Eg0^YmdqNpm%dBYDgNQXKH5ugI2&bz z?d(_giNgIOl`gjOC|d@E|8T5)CETh27dJ4gS;X7}l9>+D>}+r;7kB8ZMsx-AP=-U0 zPUylz(I5dUkLVoPqe8LGY0`YDSYr;|Am`qR|Hk`WO~vx<|PG+Ejsq=`AA} zvFwRb7UbS5!PY+%2=eU-p5}~Txg*|{P(G%Fke86f*~NrnYUc?8k_SEOA~XGjca*GH zDu%MfM7Wj-Aas;GwMXLJA#3tt4amKM+qxo|fgsIB`Vz~q;wfDW^i_Sa>6tEoEj}~~ z>-1ugkEH3D;y+y8W5O{`;*39p)SYj|lW&K97FLK76&+;4@w3tmPoF;S4M>U$RVYi7 ze@yrWTl%*Tn3cf;K9nIE@cwHq@CrHkOT1(O2VR#u!daXmBO4I=pj-aLh3hEubcYau zd!=1TQqj5Q&p6n(^b^pnYU08p#tcIrwBMO=bG=wtGWZ%OdY&_n<5dH|)=HY~he9I| zYSIowA{_#LvwajlrBuv*r(?-7jGR#`O4}61UagGch2ASY+JIjRk)c&m7w33QuNPu3 z!n`7e_F$`EHr}D!6dBwTD>ybTQ2^W6CxDy^TjI*&DRf7}DVJi~{M)gYV(|r%ykBQS z@(7Xtf{8Gqi0}$s3sfngg6|y8liX96`fh3}zea%}LLL!eRzPB<82ZL5D{n{f2myTI zrGC};PskqCf`tqaA}c-XMj$JILsM%07*P9Aa;r|FC(;Y&k^AmQLG@W9M8CsD|IN0Y z0SJ21sOC`t_zFn<5-u>;Y*~Rt9_12ZLK_*q5o30%f!qWcTojT@?jJW~}_0X6&=6!1F@Z_QLWfXdnO|=!! zrZ=cYOJyWnfDweB^e6wX;YR0ncl=^)ir^dN;En7G<6ydiu=x}$A9z%6p+c~XJS4@l zN?(g6ha3IUpm#A6-d4TZ5snuQZSSK#dRMQAKwgNN@idi(0{t4@WDHHT4O{fsy6<28YT`e9bH?3y8gfxW>^?%^8y=SO{TW%d z;dIK9K>z@68bochZ)r|M{yK9Rh-hVfQ#bP{)#)pQD z*PIBQe+H1=s7>=5xqb2CEDSxnS)dF^({Kq)teFC*!M|6QTghp zrg<@_d>&R_M~j1n2Ofq)xl+GFeGx{)C;-9m9@!9`Q3T;T!Y_mZP@9cnqt; zc!g#pu>v`rbJ7V+CSL>t1@ssii*JuOB-d)ysu!*ZUkL^`1iX`Ze_{-gy0U|c!y~{a zdi;^k$mbVv+o&?e1IL5=9HK|~_#>*3kCl+s^H-As(d7|C;)Qzu!K4Qpg~?keOmygRRTc3S{tqc{$|zv2aZ&|zF>lHk zMz0}A6Zz;kg5Q$_@s*tSDxzLubZ+6X)AjfuFulOZ8OgAUi923`eyNH3DZl%m=$`A# z|CJs5<9O&bTfeL?&m&mXTZCyyYq{g{ESUH-W#Gf0_K70ksba|asJHF-#BX`7u3hC> zMSIHwBffo&Ilo?%aTr8B z=)X+FC-$Vg<`?5_Zp0OFP4r+9kN7Y83mqYf`z4CK!DcU-x9OsHy3u`mQ@ z56IFUSlCScN{T;s&owX!OxYTX`$~1+@LCArontgkXlW?tZHzpa&wLsh=+5Q>--f&{ zs#LKF;85yVP>@ZcWAWjFtz$WT3Vg)iwt&Exl_)~qN=mTfK<2@-JDH|h^5gi?^xpmO z!;fPk?>B5Y1rioc`iA9p6A&FT729_6@(4YCjCFX1G$x>V2SSDE_mTh@oV%e5v za>R*E@qTC8&5_w(if4*Ia3ZZO;CkS^3lLrESJnOAnL71_jAvItKp<6@IH848I{t(w zZJm!xbDDI$&s0U& zq-!=Lw@)jUI*jsH+NUmFgr ztODumAJ=wJI6uX4baaTLvmxBsy68;NO|3JF%HcKQOvX&+94L|YWUC3O=|%H)6a*%_ z)gGj{ArUJ*PKiCztkJ!(h9QK#(yv0M#kCr*yqWL)M2W|dx=CfuBD$DE1TC<|epKBIl z1@qfq?X0(YZofkR@q2R_drDgGCAcA73XN_Dkgp10iZ^4C=W4Z`sIFIXVro@>|z@-NluenFk-E0ITP zuJVD5hwMn;&zLO?ApHo)KE{uyX(Ix-h4U&0Xvb=n#p!}-Qp9MUGX}XNzWXqDQR0Q4 z9B}c;#t*45_hB6vM!emz<>x73-}*GKy?rxvNzB+vZ+s=`>dW8{7%?LrJ^IQdK49`e z+;)AT7ZgiG(kHHw=kpidoCG(Brg#K1jQP=QoRHQ$-Uu|6$Zs%BcqVpN#xjfL#$++s z2dui#+;i)=4bioLPwj*)wO>+E_cTNFdBLUBp+<4f<5?|Q(@#5C%01M!NF7Mb(&5nI zMzL<@^^{8XuTLxGooH5grZIDHrci1}Q+_n2m-R>@8nwFB@Z0vfD9#S=KFMN z8{$IOD0EEAaoY+xJyS|_C4ia>zr{MEpsTY-TRh|GgkiPJpBw%`w=z(U@rk%}k~$Ed z*G^D5c4btQ^qJ(|t8&SQK1=v^N~3+qMgpap`o`V~;~LIwBZB*ipB zUndNd^(l`bIrejO;WuW47y@Iq+hAA%9Ss%`6DQrVmccYTu6rUZK_|?d>WOT_H>Zp4 z@$0NBT&|Pi9(=QNE1)5E>|;Iz1CF()wH7FyXE!z`Os6eC(k7$mF1_K~nB_@J(4KMK zoom0Ldl9=l3EsRHhM9k@S?uGIriRHv(*ka}lHs%$f|=rCQ%jYxdwU>=hw`KlK;7<1 zrCrmLRy#=e8l&tdOpI3>7-i2{V2edBOG`jl=CIU+muo4n)*Z%K2s+bwgcoi10|#ze z6snK!Yg;xpx6;-A^2Bbp0@lt#Ez{?PVxE_pF|s-10WNKTr`nba+OWqnBQIyVlK#a$ zZY7(s_Ji&N2m7kFgQcA?!POffyO*si&pD6JSZe_&JWuI(k6vci#j?v^<0xiE>Xv86 zuoX~>2I3hEC5Z^S8evw?C}OJZ(SmlbP-1-fvgkR4q5Mr+B|Zoj8ott#v|FF3+E}h# zI2Bc2oZ`=gMb9kGI&gLm#lq&?E&KvAJFx_INEE-H1jvT13#0!Mf_f7!E=;4}6=XiC z3q|l0<25w$%q<7MJnm}M66}JcdJReQTWrW5gKR={J1+t?>pXIcwp(fM#F{0}P@NuP z7J*pkUyBMir~_?D(``qj&+Q3P{IsWfO-?!JbysJ!t=o#?Qld-EP3vudtzq^S-AQfAy%*n`xMk%vriJwBsPSHfrap!o zdvnw5jz4Hlr0UAevWr2KI__PW8?GyK^gh0`iFza(sAiv~W*%7`2msxilZNP0Q`4?i z(CUg8qK)=|%bH#A;LhK%K@avj{6F#qNMcyktD}h4CxRLlFXd*C|HWheY3d#f%F6WV zxY$B$nd3EqNI61Z42lN6Y+mS9(;ap97**s(HMzCeHtC4g_E=Wrh7CTr^~YBg7XI0$ zFBGk%dS98wH*Ot2&v|)V5eL_cyb0E8tn!g4~pXd$DZ&DJG{D`gWn&)t8E- zP%JksxG}m$XVbu5Ei~Q5Y-&u-I$A7xG-*@u8cqoS*4lL;fV^kCKz*v$iZp%8wJ$Z} z4mx?(OYxJL60m#v;cZJim49-l^yo$3HGGZ0M(>mFBk1-DHp#HsL<|4w?nC^F?Donu z`IpyY&w8j=eoA1Up}?L<9O$O=h#=bD1r9vr&OUvdqk4@?d#^6sXaT7L{EJ@MK#Fvm zbiF%_?#0qdI3SC}cM{Vs+09bwN4OaMIFRB92n)xDyUFTdDB&H|v|j9y)8ug~<{h@_ zlh(Z6eN}7?=PPe(AMqO{#{}Z;i{`-40J#r+a|5EV@ED2e$1rR_-2rbk-6oe>Tw!nn5_cYNc*d& z-2d#jD|_n{^;T`h<9^3;zNEADeEsMW(^idat6(%Os{EhXz1%0Gaq<5Db3CI3NsMgO z3K#ERLRGRXVDx<$>zCstp8PufmD0s25SEkvHd`E@G z*VYHM&Jk?zc~fhIP4&ple$IP7TD-)RNY^UQ1HU;8L+wB@x|ThCE2jt3vs|a@N{#U}`)={?7k_Yb3~HT5y--fH%nv}& zsl;oM5oET!6l<#pM^F(CzU|ha`&-NWw?49XTY{~Csh#`79RZ=D&EOu+=nr@CDq?ZB zw#W0%eU6ih{0vS{oOfA97OQveqKm{8x_dS#rk(f(u5>lDkvX25;aB0k3^v|f#=$<8 zOg)~dDWQwro5-1)h3&LyT(luokILvM-|>^8$s>16tk<;oe!EmXrD=Y=?Tv29YIGx` zn0sT!R(2z^e@Db0Y-pe#4&GgQw54BQ0)Idw6lqsS27g2&vs*_+YR1>T31;Jt({__b zrqR)mqoP@pN9g|$q&g79fOr5%kTm=FGaR%K^4y(D|1XP{4B{@wvwYEy?laRq5 z*VyRpuwZ8L=s7yN-XmT-41%!@V@IJ75@#b31Sn$8G}uc7;@Mb+v5Rxn_F^1simTQH z73tDze%5ZccoLVfR%0SKYAbdObImFy-(xT0U-On*I0M32wI z_)3Ze`pWv)yqbXzE}RB}a3jk3=Ud6y{761(9|oR+c=SlbK9L#x`6GuUF%z)NT`{eA z(F9nvn9W!TS8dzHK>A9>?9+oKzcqt7iGV#>#*-p>uEbGWIigH3Sz4)N z4=D7-x|rOd*)avpjHmUf+oQ}F>_`T>v$OeZO}c;c^b9c%`V$?034Sd=PW!MeS+OXO zKj6>u$N~6=+}zIkSU!$fPw35y{ptevhtd2wvc{X}7n$)+G`bxM+B137jESXmqOer{ zo$!uedL{hWYVs)HjkC9x7nu#uP<&MZ|7Sr#jyd5uZxUZ9GxG-vRONI$m5axUS;`1N zjc{_)fjC*w%(`$oA0A+xtXyNOR%nr|9%pjI!ohGznTOKsHyTIMr8gbN>Wn$2>5Wd(f{g~#0!fWdPJ)fn)As#q2#G!F)Aqw_v57tM)Ao~Vl4P%N%&*v^ z0+>LFN+`*n5vLwMSLVvQ24OQAPoE2>eKYy+)%gXz&BAs4y%YY$J} zkj{w|a~Hp`` zAR#xuGPxT9qW=M^K4SR6fcyNJe@#XSLu_yIn?^6X)a^&)Ma`DLA9jbo&*%FM4kqlDq0@?j7jwhv!Z zK321X2?%D~fWchR1T-=JOMSXGw6UDnCpUvXwXvM!m4Nw`V>B-i)E5bC#(s4NL~}}1 zeVKWN|023@d=Q&P@W@v9WB8GP`>~0C1x9;`{)-SE&HJ(~!skgB!2~p&x zu!sdH^3$MpT4>6S;DW8b>eY~~A5dx0eMY4DYhR;Wz|h>0PqHyRY5Ze5&k?U6-4?IR zGq#l*k&jyKTE2_v`3~HFU>I5S#VC8G80n(IkiApSIL=1DDpI4H(KOC-Q0b0VX=F1D zwd$3=VUTIfYFs7O{c7qNU8^Vi4^e9h;{2L0@ycxz(8D<9*El@vSAwsZQ?81Qd1q+) zy5@>cOb-t5{`+o1PiW(gR?s2bmu^^d!pyBU zRDW$W4VAo5|JpmL{2F}iTV1r0$4a-Gu1WDudHz10zBIhfTHMpv|739E-L!@b$ zoDC1S)ee7ZZ4hn49bYc9cZ}rdm=PK+W%;d>{Wd5xTF&y@G5c*&h~g3pCG-2_*P~KD zo}*@ZbA3l-!RhPuSJN4FKptl+Vd?vxxv%qsCH9?yo6)b2dbmcbQC3iC;x7Is{wHRf zs3!pg+>BL;j#0^9ym5R*Go2ziG>A+zt9c#c`7E6A9I>qyVNOcy@f^9W7I98Wyz!P| zmJ8*w2Dh|IiY1PTT$Lp$Q7=t(0)mm7p1EIpO*Wvp0=90_N# zW=zc;o}GcyVE#s8Kb(EvqHhF*rb1rQUxHOd>I6$z#>R-U1PerliPDJpvl&cT9D$_s z#k5|}&9&N<6aCPix0@i^`Ke%&M=F2UIzlV$OT$P6!+vk_0L{0qCRcqcdKb4@!pdr& zLtOMCsL~J-dt$Yl^TzuzTY4`7g6{`PNrXvB>duj^Yzg>^KOMxRy@g~nCXW6@r5uX+ zt6VlR<2)D`n-*&nK2V-VkWN{jDmE^{2DDKet?0=pn`!>{D10`IyyMmGK$#&Ohg^*?afSG ztsLw{jZH1h{zsY8R1IAhRdw_acW-Gt4tjceVkOItNAT~+D0D$dfD(4vAZ^8mN)sP)n<7S@<;@VjEx%i%Ui~IU5^8MWPZ1R)u4-a$N1jvEf=?;E2 z7p<$?{%@~S4q$D2)x_Z7wUKI1wxpOy7Oa_o-<+ut4}e80{ya{WGeUZSA}D3*hN9^z z%_*CO=?a`x#W81LH8{51mC=@lL>Ni8rTL`M5KP%5kJx^dKQZA;I+IdJ;ZP5(ot5PW zJm-+sGkGl-(nfXw4i5Ah-2(6QM2_1o$n}kr!v$t?C1A4jDT!Kvfe9fdv|5V?{EA3{ zauFs;;7df}AEfv)lTNSYqCeOZ!>xx25@+-Nx>}h;%+BD|<_L0p)%zL-aA6N`1$94x zg9>hbK4bXF2*>wtjXCzPZ0(Y2vB|ip%vqe3St&Zu^jbK>uz@QsU@2NrU*J?BVoL{A z5-=1RATQQ59dJ;asuj8deixzJmo}9q-C2FAQR@_&tOV!H6($t$O zn*1bwI@cJnSukT^DfpkoxkKIL2+0N-4whV(*}&mEdPOqlAnl8f{XDReTY)vkXLO0~ z5lQdAm;1-63DL7*it9ZsIftIHQ$2kZ#BTP|aubora!cngzmJ+OIuYxt{1G;sD)JuX zfvs=vo;rEONwciJeu{cz{GiGLUMJlh*DSph2&Q=E zmnSoQPdm!lI3o%|VH2fe2(zGA+6P z)~Bt39X6;_<=mkuCQ#N5O9Q2Cintkk(z@pB8{;WoUOLOBb>hUqhiga0Fc4H(Vc|7QBY#DKDhM38@WS)8va*sWB5r3T&vF=HS8>|B`3f+FbpQ}oRTM9I&-`$_}MW>;ua_uuiFs!rg zq7E3khgiCYoHD{CmbgK_*HML?c7_KEU|0;17pZa?pxU4yKCg%BdQ6hdT$Doktm}X1vJpG@px4Ikz?)f)Ne}XIC2A3gst`s> z$H)!+p$@L6PIZWwZAZLC5Whv>inf4hi3-XqVS%*Fz1=uy@rn1pFUfwm6FhnRTxphc z{`KM2{&|&mbC#;CQK#=7&%QnK1OsQcPiT-aD0^voUk=f{5a?A1?~+h8x?)RX8EgsG;_Qvn+7)w0#fH0n< z`kzF0K0_;QB1JOuk*E*NZ5KXOb{kIvs+hCD*|3x%|krD{(ye7hx6HY-K|M^*=GHW=XC-5m`0Gh!rS}T+hJu0 zYe<#P`1!w7$}SkaEjqfdA0{Pz-MBdu-)VHeMXwRMatX1|2SVJw@MJLFW_XZJK)&H zIBhvfBa0uy`tAIiWh}mDadU(pfzGB}vP31@KSkjp!&$ZR)Kjt5#=wQ!Md2cs5^$8Y6F3{WwA3rR&VH=QhYJg_+~s>2F<|`DB}{ z`yaM;TV0)~%6L~_5VKiT8UKV@vrCIWPvZJfB!g83#;L!0;NZBC!WbFf@i=G>U%gv2 zu<*sD-S7ZrMFftAkhC@sOR*&`c2G^8Ek@k8>HsDWHss&yhV!zUCg-g(V=BXEjGKV5 zGuJXp?|fqrV|6iGZnkD=pAPMs@;|w`qW&ANt15ZkR+_HN?kZRRTpMdO?tU8?ok1_p$8v_3~8F`@QNepq+X~-2%)x=LczvbCc?# zaJI#ub}x9&46cVf0#I6%7i?B_b+_NEO0|44`{Ve36T=uzbg07?J0Z zUvX*$3An89LrlYC@O%%x=?(wfc~Z2l4oIH-9W&MSn}E@jn9Mxf(8Lmpe#i%#hK%Qx zY}SdFMR!3Kw0A6pbETAp!*DR`!r0GsdHo3Oy(h{4?<#qN{4Ar4E|hTE2pI`^Z#R}q z{sR7?`S)C*<-ZSVwhA>wCWTu-Ql75#9l9U2(dx}N{vTFYMXTs$tjpao^M=^V$jeNq zFHm;SmdLm}Bzl)LG=gvO$bz02`xvAg8BTF5$r1FVA5jsSm0c1SZhsw+HP1f9>6Rh) z$UX)f&MTi}0q#x)-=eNaR3+MCykN1a2S->0DxuS>2ai|;%9YryAgzhUpuWN-wi9;F zuLvejX;-aR86W?`%}xG-t)p8_Fc;(0qXp*@5WE?CUs%7x{dbRevihha`O6MXz=MG? z{-5oT(wCDgX6$OL>fq*V`afM`^)2Tm3G|OY&rY_Oi&^CJBGh~J?)XK>mR8AZkf%Ktvh@>`=N?i?>d`5;2SPefUG5mo_+TB ztV{{cp-9Ak`r)lK%8x8LI;)%ylsX5A#Xp^xU0;tnV28BMS`X0P(M7LXF4hO&heF>z z3PLepdCqanG4;8|69-7b>U@oVLWFU0EuNa5G6L&5);24$KAH^Nq%7(VS6nT`|JmIT z;B*B;xd#_My}(L&X>Jv~;2UFa#b45eIVO_>DvWHBtvS*nO|Hq z_{z#Xflh-C9FdbAW-rcdZN^908QeMBScxO$591=UYQJaLu^6yOnKXXE0-htR!ND0P zy%Bbn<M&;GYYv8$}wYk2p653Xg zT}ms~Z0bhoA06CwoG@N*(IYf(8hlP=nFGY3OigeAS(=k(%nUydn~6E&4^oRu9l5>{ z*Iff3T!$fSM>TxTq~>i+S1&;XKTp$AXj^$|BsDY^IUx3c)>l4ZR5qdbSxfA9Dtnwm z!O~YI$@Q)QEkzy*5s3}`Hcq))^l>QsohXWej|#6k?4@7TOH+GVSo-sfygQuwd&V%@ zV;(G4KiZ190i~Umdw~B~S0I<^_ds(S`z^TJ=bqST zC2S2TEbrc@>8O$!#>B8*1lT)uO zBQN|S15uu=Q=BCNqIEqjG^I5dBoA=PJlw79X*j`UQHHYwM1f^v{Zxzm#meJduJO^)O!CNouzYS2K& zK{@ec+^o{h0?J3rrbqLo=#(dGnC9x-+24I~h);Oj(x4SrHM?^{i7>c#MkWi-!@32^ z|JlFjq{uBT9G>JAcH=hD^N-zfXe^`mIUG#4I@P&5GAyupEPVDOS~$7F84fpp5Nj)j zebnYa72RMsm^iE4D`-kfl9wX>Q-91iITyl`gGyd!RCfzDKpr51!2|TCh@5@^voqYc zRU$u0_xsQyE-X*ZCcV`!kyZ$wC^`Ni{I-+J1K`GHsL0js^x5H8VvdnLAAKVIcl)ML z`CK*%2?iF71O`U(zq4=uKZ~aE>Vl?@F2E_D+(w7_vaj9{Ae4nUD66JY-CD|lx}KDx zK@*V;fYDvtYUtXl+<$%lgYyZKvw%a-%5uO}Eco2~0rf!;v(wWBB!8tkaLuxv<=^q> zb-C;~xim!be~*obTk>#<$k*`gG9Z^$XrLC-7Q8AWjJM1=KjgrYJis7goUp-P2 z|1(+HF&L%N^5sq~kJWOj9NslcBU~K$F3TQdy(L zEndgvbs-HeC}W%B&a&RxtBQrNc+AtGDx89OIIt30P^ZcnEwiHHHWRaCAQ4QT*StUF zw)gf*?wu2+WbvIleX2+av1jXt*Ay7(41f5~bw#LME1tY}n7)H+QV)JA(43s~;qw+?hv<)LE zSlS|-A{W_4Bh+v58wXD0vb|J_U7v&VtaZ5a6NdCXyO9uO#c-<)X*Sq(t?T$v3{OG?!qd*Pb5#D6kb;#u_%gTJ4U(ne?cg z)-`L`)@PTkEum?+eYUFKzxJDT3IA{(p zdXn3}1GAL)#wP{Q0KdkxE}|Pbyu>lj%4b(jfanFgv$>e}mg(*9N^LsmDK@{B;2s`! z1NY}!ewcgjpB<&gfFE-^<@(=V;&@*al!(GSf`9M)9F6@xY|kfoV-pYUw7jr7@KC#e zW)OKGFGs+DW=|(s1eJfiPYH4JdldgIMD{ZKBsUt>iB?6+?DdSPexnuTTqZY;1aY|;~M zZ-bD0=;~Rx?Z4xJ&SfIiE|at;EJjDj+>AUcnJ}eOo@aGIb*Lg9>xENcW*(#2`s_?y z9ZYZCScY)JCn8G|K1&@4Yq<%ov)pI9M}=oGy>t;{In48S0cl@I$nd66zg@oCjVfrH zw00mf9C~#yZM@sE*Hu5u&C)4RYV_v{@;u?<%KCE1zI@m_e=P<&@5XyK`jEQ%uh}O_ zLyf1^HM197_h0&JBH}03l761Ye7FJLk(XgE^RV zukxLtI-D8yaLWV(8Ez$`5=l1qJoRR}Xgfm`vCNvLG24Y6jURb<(Yo7M>jFY^Po{o{ z@eemWf3$2jJ|iAFA|4?9fNF*HVlm3)A}Ggaa2(5O*%7+g4#V{h|Ley#q_5^PCj=N+ zIusZf)&Cg;B#cd69h|-XN8qfErn@?iHjIAJ%D3dduwuDoG3$qlFBDTsFum4 zm!Fn@SnS@%oz#$Q$x0b+H}mLem@#@~q$GJfO3rXL8m4oCy5MI{1jOM)AIchSoL`A$ z2OmG$8_uKvWhq?BIfQ~dj)30Wmp_uX*OF_JI$~!Jc?C3Vi(S$`?w7j42SqAGGnfimk;BZtgLPPwcvH}G5;&*! zg5QV<6nvCOou%$3l-yw9N~h@RdSn5Azrp zi(Z@Zu-terhU449J6~JSBb+@-b?NteH<>7P}#A_RUxI zV?ci&=c1Hl{)GDP9+(v#p-1}HKNsCE4~+7Er+okUt~3Muzkgo-Z{5>4S4GoA|I~-k zhL*)ZPk_cCw z#Rg4+8M4?SGMt-|C)rt4z#=>##=1hjiUx|31v(ZuL7?!Ltj~$L8QFK6RCXGHAq6S-2UH zMkG~}vQz1&dC))HpyYQ#-E?d4Lo<#73L4k*CRMdl8C z_;c}^QQWYV0Z*CBsh?y`EW`XLX#=imG9f>I&aRG@$rD7Y?$S>s%PN2fZq)R)#&(_R zYq0r@p=9RCi%@3Zf7&Z%gs+i`2xSIngtbC4IrWFq>y60ob@=cZhb_vfRS!Ja6Nn*( z{bW(p>eu+ODr4f@AAYgfxhh-;+I9HH!c~-(R8M^xB24}V%8c5WEhL*t-&mHbG{ejR z#b;T|9&SXugpJ&9jn;(oktyY;+(jA&6}$gFL<&t_t4KK@Q86-)}{Hz0Y zmGBRaq1|Ad2-T!VLPX@`=X80{dlL`|&nCuyG#G&!(%KBY8q~0+Ib3kOSJ$_%ox=I- zBc5)1q9{a8$DmH)*h?jza;UU7$JEoJ-QdK_?qxB zFFRIiC(aP0|D95toXatL4kWxqDqx!vls(d>{e(5m+cE@!ejyw^A+rSd>Ao*OR;?c< zZF8G_n=GhD{A~V99d8m!Dyxg?WU?2BFM((ZxzuqJRkj01mV01c!aVEQP0oU##iH1) z+}txBMIDEeFZli0eNO3{RNC(3^zLbqfr(uVY`D$;#n@TKRt`j2I?T+R4l^?|Gcz+M zFU-u$%uJmZPKTK~%-CUO=FaqJw5!?CO1qL}xh$3c+$xt`{?0i(?sf+4C@}AirD+Da zL#Rk*$V%Pf%|;47MV1Zy5SX)&v}rVQpNd0Dkol1FazyX!HJP5G<#3D z=eIn+K0-Qg__TcgRV(p_=a7?Bh}=!9aAFa562ErIGa{&$mHMfRC%pU|iIz@YTh z?d?UnV5pU-t%<*8Rlql@NiACVA4NerL_^NxU&&V z+*r}AWcqElFEm@RT4Oe1T1VT|b*QXsOlv#Z+S)Yq^z>SD>sx(mSCsl(ul*m6r@Ofx zEN?5>m^^vDn+vT7pG_8rwwJSU`6XB?;|>7YHr5nof=dk4~;Gi@{kUj~riVGhV_*@&QAD`)KzP z8~Aa*RM3Bc_yO&25*=wE=(I!(H)Ex|&_{6@I}D~m-f!Hr{Ms5FKw!AkC-KajWlq$k z&R>MU4hZJjMSd@h#i4#e#K+;*73=9WHCC<`rHM09Ub#ko=QaBh=)L7tQtBToDiJt* zg5dNT40zYGqt`D=c464OenLdqsVap94&VZTFlGg-^HOajW?rAQl^Il39 zyf~^s>(mx~+EKk2R#Eul2eO@i0m!sVF%DJD90C;!PVAMOfQph9HP-VxufSJTwaa+U zoJ>FZrQLN*#MeOYn<96(Nd04aQ6+J1&Zcr5oC&sH7y<9~f{jG^ty^j2Wl2OhD<)m2*nepbpUaQwb3Z&ca64 zvXOSs9-w{v_eiw|i%k5J2+U>K*+=M36rvsBBTfA`Z#TI{KyE@L=42TgG4e-cD-apA zk?8WAv3%TPDtJdSuk~nKlCKS@#Uh!|?jGNw9%(T*iou_badB=T=Odc!A2UdQ`;j*4 zhI5GjqQ*Mwh0mxRXAdDs-VBkzQY`a2N1Uw z)((0{KXVjFn^R+4Q}F7kcYyGZ4+K#W+|?Ux&g{y=vzW`#pChw3z(zv(?N#ky%fMv* zxrAWiE!W6|_bFP2zcX&4ml-K*1F=lN9)CY@Y5UmGg@|uoUDhI0)`52&oKV znAv5z6~~HYaZ4QnT=cYza*$RWM2456!4$}CB$4Ikoiu>aRVJd+m*0&sQefy*Xf|$# z>!%nlx5tWr*>cheq{6uyH^+p99gPlO`Ao+}48S3ILjG5lf=iU5xVp}tB9e;pXCdaFR|hw6wWFr8}k)klo$Q>TMF@Fq^XJ{4YI45<&+rk3wN@msMsY6UM71Y8?$sQN?r*#9wmo zFgBS43KIMjMccHd-XFr%SZnE^2j+>CHV?gbvWVY69fQ~|`GvH}do^hGV&|O7e|bsI zM|&s&%`1p@X0@?f7lAs$sUL_=BwFFtcQ4Uh74%bt32C!taOhS617sB}3xE)3c85J_ z6AkGLE{KrnWOnr!K2&@gZK%VbrPfN01^bhM`xN$-C7~BQk#`T(A>C9$HoOOzD8kaI zEo$23Q_3fjUXI3WJ`c-h2HGOk3Q05l>wq?{-{g^;fIc-ya~&TrN#c-e&7u}#gO-nh zt-qj^KMwy($j1#o)Q8UPqH}0hX8(i+>H!eiPAV1-K0rZ-5zTPAEvPi7i<}Gdg3Em# zX2nvIy1hY;SrB@nLdtGF)$T(#zh>SN!f7vf_m}kUFIs5!uW}>vG>1$}+jZ0@o#s{1 z%P$y1_rb(@X(;_Xk?t=DgYI66aI#C0i)O4P@RaKfEI6rp8&A|Lu3c=!S01RY;@k?9 zJ1!f{<@9~GC5fd*ENsbP+D*v5i1z&3T#T#v^R8KJO41t?uiWIQmFi#CzER!VF@5R7 zJ2!iYimzfZm+Ya`eNs&g?5Y(+`;O&l{@ynVe>rx;ctaKC&w=WDM`m0L0 zS)*|p?K|k(D|rO8pEVnua`*Yt%xr8a>?5hiC8)Z^Ig$2`#Ug%eK6$d?vpYAj;sRgb zHZtUpAx-Im%0Y3>=Sk5vGQ$@y2X2LxLb;TmByJ4)AQQ=Hdyj`%D-yqbjDSp?hz?8` zgpXmKC!nKv2Y+1@XmJ>+R_@yUvZN?4A z4_1avC32=1HK%R!?D;hrdSsuyx()TuxtJVuBqfwA(2EE> zVOth)bcy4Jp`e%rUVY;El=VTQwrKp7js}o$LOIa#$KA39UvhJ=A7X^c=P=XZ`=_i@rmZYot#+UU} zurspSI#J9s-cd~)q8giCx73j+62L$IUf3uTt`{w$ZGM1T_%*DL#NVQOJC}0^n%OPL zhsI8sA7(ALX{V#aPX9x1{yU_n0H}Wjd^4|wL^wV%r?cbjP<&Di$FW*wkSO!Br6P@& z!B%NmkEVvQFm*6Vfc&xOr_wOvC{$Uxy-T@4g3UR}viSiX-OakfPMV8fj#|Rn&dI*C zVmDs75p1g0VA<$_i3CKFr+C7+P;A@CB0{VTV;>peU0H~E@e%a$&zo{%}jY1 zvsk#Ha*X&^%Clol;nI@ogOm}g=}gmtYwk{=n!Nn2KGh#qA?0ucIHxVJ3m|rVIeLL7b9!4vm=3K{+t}|a%0mF@R%|;(7>Wg2X_>B$?-F*`hz50KzduFiO-)V znyk0F$wX5`X@^qP?Cgw>-Hv{;22~tp-SKb98{+Gk zsZtY};ajEvB#l+{4@j4u_QGbUqnum!#(Eu5M?%?0N?I>MjmHt{&yx!0YVlhG6>{#ZE8j}fas zOFb9RLl`s`nEx`c(SQZkMfxZwCJ-eDCNy{Y%H78fc`Jo=k2$tT%apzM)dDjoggXT60)l_+d9^2FNJ77m_AUxdZ9N{V^gsfF2>5TzUPgXBv3zuwz?->6?ugn5h0<^ zKCvDcTP>I* z4ZAv0yXN$~U_;Y0xZpBsLoPl0)uY0fOSAmn{Y6|h%%`rSJ>-$O?pV`nyIZ2nCTkUV zIIukH?8x$qsWsW_OqRP^MSdi>1 zs(UjOlGkDy6#kS5NwWIXZZADXCjS#D3Z68MHm9%Mw|XjH{?mrN?)= za_=&we>g*^(Dm{?mu9_?PhWkAIDER*ZM?T@NoKaVij;Lwf4d{#no;=07s>rXS{I#7 zk!)X&^zrez?;d6=7l{Qu-r5mzR9WN48E5*q5BKlAgck!@y5!e6d7|t}(=#n>QvZ@y zeDA$fYN|v}AZaCrBwYkaYb5QduXx27vw>CnDQ{b9#(`kQGtd>xb5MGMc-Ks8>rsin3%)N}HP&*S`r*VL*5bTNRM z|6gX|ZL4B{M+r5~yZRpqj>D=h{!vfaK?yAV(`+B-?R$ji>#0c9oUlwvy$Z%K@1GGjCb|Cl=7v${zHX*2B)jbKA=L9Ig);lF(i*|loPMv6P z@Qg|0%_7=4X;S;(tdu<~Zv|HrI#GOxD3BRX1b*VSQ}swIIWqmx3HFsth4`losrWly zy3y~xAVrboG4v-#I|auEYaQY6BoCoapY)x6CvhmV0qIHQMwn!&tj}25zU&=R?CS^; z^%yeBF}X_=mp_?NuMTPC^iWZY*}%C?JcgSW-H6R5ks9uCS=2tVuMpKyXKQOjLM}V( zG{wJ|SCFLWnAm-(I|}naxrTe18M2Wct)rKD8D|S-j3=jUk`HHeluI%sc)!pR)`Z!9 zYKK+RFFT3uRqYIxVvX>?eB1fZY(p67YKeWf^hh$dHq@suiz@EkK13ucoFrk=vD=$QQIi<-D6L`TQ z+%6A@9=pL}5g|%-G5>~OOvV5Z-RNIKv^z>S!p<9qiAw97lsDVe9#N)W5YeiNN=ozH z0G)gENHaFoguo{IFvSVr8;gR{b-<|xtIMb$)km}k){^E2gBFyJB&YuUQ}|K92BJoJ zWi7f5rM!ecO@A>+-hLO3KFF&97;V6gG}GbUi-IzQg>r(~*uK%?UG85RW9TB&LL5OP zA$W-g`zkbspg@fS45Wj*_29tesEOae@X_>NQpP2@gdsAIEIcw5V9?-GBeSVVxoRli z2qLQKi^g#G^545PO!?eSm;eyJT|t=C4g5q=9W#Q8s}Rypz>WKuiKf(|iZ(YKm1kmq zb$)ZsKJy36RQ^fUZq!gW8FP<6&h;fhc+Rzo_L70iF-*%;%uo||U7XknOy1X0K$RDg ztdt4Ej8WtQ*~C-+tIG^Obbw=l7IQ3N#KQ@uA~nX4kF)j-Glo2MSp}RC!MleDjR5Md z1pXaOS$jAohVmx0{EcLm4m#X{^BrnbeE+{Z7_>feq{7OZ+WlL|b^wf{8&}DFm@)OY z`O*TIb^<3kpT0Yu?7hIJd{{S<`m z-1YO@fWj{#<`jj{^XRAS0OK4|Cf7S~kbtV18a13WE-nJB`*n%>W449bY+w*AI7p)5C+f}r`h$MQ2OHfM+ zb=_n11Nyf!TB#fuRNrlpeqMu?_WG=4o5SG@D}cEIJ}eUcpN z69AFBZLYazI~-?~G2$p47S1Zk=o63e zr@HYcf?@N3*EZOJRwAi#ySETyFF5Ivbe^# z_rcs#_`azDZ1SLstS~S9a4#w=68$&$J7|pkL>_pRLD~e7sJfyf2usP6C9hgP};VCx)OO3N7p}~Sg#3e#r_sE&vP=%$c)g~FrbHlheuusNx+L~Ix zu#N%ml+~jc5h$A;(u#?!dzNa;Fa}F!ngZs$h|<_Ws+e%D2WAIUhDf#T9FnWPX%nqU zctnS~ySavx=D(8|Ww*_nFF4)6xTdS~8BGpvWiB)0+|pTzxLLUzLZ~S_Ks6Irwwy9q zZngY((Qn*!^t^P^F)`HI36fmJc{F|1WVQ!4Z%ycMdjOIPU90vio;(OX2SbLmoI*UC zC+|3HRgw*qx2#(d+O|4{eTF8XEVIeC%(z-0{VyaPdP~Pmb{07~iKk0(^if`gGSa2& zbpGiqNJZ9hwK6h{Gg-QYBO2E6#k>Z8MX`^(f>3|Og|G=Om*zJZU_a*xMTF6AG4Q!SBv4#I+>y1 zc@?m!PV;8lFhH7<5>MXTRHl9`h>jo7u*bF_R@zx&*EZt>&HMI-ydlWATeen6h{02h zNepPm+EX)AT3NBmS;9zz^r(aMk`1%}nkre&|C%!Bk|y=uW{4y!&jdiY1z<*##T&i< znne+Cc9YRR!|V_s;xA#i)F64js;#zKF0JZhMyIgEG-eg^(phn0?V|D5B(EMXqESIg z+bXZ##%0wMO*@l+wWea=5{lf1EY5^WjH3&ZvXKcFL#MJ?xe{JGGG;4kZuwO7Br2>Z z-#eZL)82g(xGZh&z}mDNWuKorvqCt&l0Uw(J-)(deqm_W<~D6^b>WAw<7eN9?{F9N zMXI%#Q|Q$AGArqiQ!Qj~cLwp~f;D1`7^GZ{`E>K+NeW?OvQ40F8AiHt}2X1?mAveYX1T}6*DJ;%*Wq=454 z*2Sd zQ+jy8_jBTVBBO7VlxO}Tps#;KO#hM3PtMlvk4hHJqE#i21mGk>#{V>@+#h`dxKM~q z8JL(Rz*VdfbZ*DHcv$2a7N7TKrH;`Z46#SZAdlrOH0(ctvE=E|4x+z4ha#Nq8AubCkbMEqA;2O7rDX zEt_(m;*Jz~)lJJ3+s6o_(g@E%}m?g1EYTAk(_G*V&RMS&;Dk;3wb>w>AK{bM*U!`%HFLN zY_z2ti%{>2(rwAHm%-j{o1d-ocm9{P{~k0JG5fK$dSBt)`>(U!PuULsfpOvHRiD+& zM@ad-vr6oLH}{$j^s_)uI_WFPyBHL4yMU3-dDj~i+`d2RY41R(53%u!5H!~@s=E8*`~?W$wd9>2+wK5I(fmD z+#_5Oo58M%p3B4x5|z0K{@ z4*L@#EM>k#68P)J-yvA*HYY6R17BlFkT>mHKkn~7%byvMgrF69|MvQh3mDC1Xe=jC#yFRO)4D?U1u=9{$V{L3Qm6zQ}1TE_^dEzW15W&h^_hx zPKsFEQYa<6eYw>4qdbqIcJ`unZZIL-lQYoSWDzlF(Fe+xLTZ*o%2{!3(`cOYSTQST zsDw*d5s#_ajFVN7>t(^~G`HsJr$XS*H;3%qwwGJ+Ke9MAmCxl&|lL6;MHW9FtZ1T?IiY>wl zIgp3qS0k6Q&(Y78ZH_4t$`OgN^W;_~G8siOy7T06#hN-uV{}#`sS8k*w6(@R7m(!A zk*w8L`td9n>a;rN2=EaqaF!^DmlRlvvP~J|mgO5U1e9`21=8^4i$#OkN3!l^T1Bl2 zH}v+KTT%s&^zr4orxloj_Cxe^_wSe1ig2h&zgPFm#MwfzXwAx!18Q1}bVl{EB;aFh zn^m4IIiurJh}fgC6&7MC*JCKphN$n%()3~2(n0d79*1ycVP@1BBP?WbW;98Ginw9Z zlOAztgedBa99hQz1I)~eiuK}bOWZV@3LbonX?E;>oZr;AY##d%#S)Ho_&>{}CN$O^ zhTy%5qtE&85R?Tc^S9k1l%Pk*y1Db8pCZuz0OC1I3)OR}3*hgex!nJ$Ks7~qy^qbF zQ)s++{(WgQ7*E7eTtlYm^;d7fSxh?MU>R`=Q!uLBU>@Lp8UOp;#BMSJ>CJrw81r71 zd}mQi2~=@U?_z%U#D(c{!ySFV`~6SP;*!^O2h(+iBJUwbLmB5h28hqYh* zoH^YOO9)k4R0%U`bxMgQ>yI?0(D3_RZXqD5P;{q~aKDm+Hu80A zv~lvoMUc4fFV;yco=zT9aiT??m<~GvVrh4MZp1-g?fJgn6Y3wVlMBgL9>Vyw1&oti zF2hnF?|H(*gJFu_V){?^(Ng#~@s31#0Q%xe|OvQzwkw~=x8J*d3uU&}k+D$=Tzc_F9Z>s7RCH$wlk1`!!1 zq2UH=ah-;Qhl|sX3hut;{xB~#@i%1FI}*$AMCM|D1S;x69GtA5Y|52vw-&vN!~G69 zOfv;;$Z7H=F#af`6g7B#2DP?Jf;Pt37Pov|+hm_VJq}+A%->ODU7VE(;g)#dj3b6% zrxQrIm-kpZ*6an+r;d$|jL>-ioP9>0QIR2)AlOGo8l(o-oGhI>n@{THmA_@kK7(8Q z&Zt94$Fy-ys+fzk&Q3Kx@kxyq7RBzV#QoOxPh7Y{KfSjIJP%+3aa>GzzgkjN0|}Jj zcSKt+y^(5>zt&uFXtzo)DJM03C0`A%P6|~8QqQ3C67H0Dvp~udbgNvJ23!&*m={?f z6+PuSvK$g-4)d0F=BEil<*&I|Cy-$%omL+geq<3u+$X366yduL7NZgcPEI|J3zB$) z`ba9k`96&sf}c+#jwDc40oY3^lp=h_XrKr=PcJ#FFEK+Wi4PL(u-w3m_QLQqsFyzZ z)ZF@Ikz!#dxEFD+;LEEN<(3HVxM4eQCd1L&80st2>tBg% zo-;iU@6V42yEvP8Xb8SDb2!`?4q*e_XpQrDyqc+-{(EjAE?UEc1B9ry(k$%9C2jsj z_iPC1pn<#SWSuW@g%A-_)fZiNxMEEKb7s;U;8Lg3rXO0@`3^o%bPF-)rZ5Nm*+d5*m`xDvg znCoQQNIe^-8*(DxhGybUj?6E-F&ag5FPew{=q95j6?+T(syqJ{m9gZ)32*CgI})@F zL=GUoF~e?gfkO+u}}S36Ct$)n0uks*vVe;|gSFBTrR&OmFNJglVF z)pVGlN&PX!fn)^a9og`tGr){_G?i(s^+OTblxc{`QEjZgw(t2LieveV0VJz`1uOpl zE0`9*#_fL>ELlNmP#7g*=cqP>Iw}ONHMjj(H8oSwntGKLX18Yyp z`%Zj+!0TBG*9x8`!g?3@(5h5u@G}HvD0p4Vx5BZifbUVFi1VZ3&#t6a z7S}b9A%t%hF0As$sInf615d&sIzl*6;4#^9VAJvrh}ixTpl&yhM7P=l#}siDgXW)s zenbdGM4TnhzU|y$LuE+?BIgKzL~i;j z0LHb$py3Z&8KY!`7Xtn29i~0=AqC;!mEF~eqDZib)6DmgwVsW6qk<{j!;;`>0QCWc-r5BPwjw2|GD0?;j=rk!++dZYFhOn?wglG+>)JFK4mUU zEKOaKf)Yp_saU~0y`A_Gb%IJ8Q5#RGY?xn@$7Ap(PA(H$eUnM;AVf}EiVAAE@_^!a zX5asqU`V9LBTDojAo3L;AY}jd2_|c9YWCmBqH+UhKlPCnA+{Gvxs>1}We%9)Dw1OA z*$`C>w2}dov>#E3ndZS16O5OR)TTEE#G1gO64L=j_~(0!&1U|t5<$M%YqdL_J3gyp z$K0MS*Sn9N9k!vg@4%U<>}*%d;~ByBt6(V*V`Sf7$lRg%x_pw1*7=L9tZ0U4h{KXA z?kzLfao+`4h6@ByZ~c~YI~1|9jxgb1sKgkoRoc5omqxs-omB$KbdZBZWo91aC|CB0 z6M;`o@*r)JA+g}7kXh2iL&9!Tj<95Ia`ZSzEI7LveVo~#@NPq3f1f?R^PYuW>%eI| zTo_WO_7fp_!AMFr8qL}!Sw@KOXW_;1LEkhns840S*X8>EFynRA2oN}4=tspUrz2+0FuTuy)iJVO#%|FN zDr(e1JO?*|jen^Su$tqFp+LiPhb46l57IW`cm#JcG81kVwVFZL zjKh?Zke&0Mn-}3&_L~T1Mf((r{&aYeDh0CyP2kKt@6;CR2YpxSLw`v&sq`H>L`7_V$ws@VbIxUrRwRm09mgF{+GuekwdnxG`&IEaM+Qz? zg8JNTtNd|E2^UBzr-%_SmQJ6n^U(mLC!D4nbQnS1qL#(-71mdq(J{Iz(NBvMsk&oq<*U9OZ9@RV*B- z=LRbUK$HlwcPb!c)v9A6mqopE9pi^+m(ei_GB{}aT8Szt4zxJVZ}}w$p`nZbLnzl{ zaS`sCl2o4MAoV4G%ZtD(v@h*y!Qg=PP`AkTr$-MM;a{KLM(WbM+k&np0bcWniaYEe z^`%_4m-JT`aHAna8JYrDVLCt^hyS{t%P$)Odh>u*OIdfss#ajOJ*wX6vS8q$R6#8Cx zH!kYo5W&F)2FqqhKCwf+z+bggb9t-f;A-#=r42xt;Jk?bM)yR2g}WNXA>^&}tFk-bRrQy6oO|ewbrE9zsxk3UA!eu0DD z418oI$eClyI~5Qo44X$-rthhN5)X`J72}?~QR5ahxd>qc!-vVIMuGNeH4zgv|9irHV(dg+at7bjMXgpz_N|Gqvz7 z5PY9&D}0(f46A#I)5O`bF1k0Kpnd%f(LCgIsix}F5!p3>ConUxwt+%fFk23GF!)-$ z1m&wyIEaHO)BeP)Dm>6inzSM*u-WWiU`X1H#)2#1aWhlu8W|XBbaE<+>#73~7da}L z6*Zu81J+YJ!He!jgu>GC8OA7^KXz}V?T(Cbs4l|uS<;y$ zt9dp>fPd&U3Ng+)HItv(s)F0Rz-K2o`NmN>kc*nPpI3WXUQx z5FLmjU6YK#wJA)c%-KV)eI%EEO%BT2CAxJj;#Stl0`u1yvqL8)LTIrcq?|3S72{UG z^1wEhtK3D@gUDcQi++E*U;+a8nkh0cmxiNHs0Y{*lFnCL8FtRezO8_7vO$#S3Z&l zKS(FmH`GW1zW- z8y!KWHW+(!pGa}s_HkkY+kr6+hlODeYQbnP@|zroOzuDx>y`&!lzB-axwv`_ufiqJ`!Lau{laSP0G7nd%-#ES?katf+ zw}$phds>cOGmSktMG?iGXGr4L@9slh37hFOj~HXBrqndL1r&fP9`zBzfTBeujc}`~ z%7VEpuN07NT?cMXr;NJ3V1W_kQs1K>g}G5s#gvfZC(5O$M>0UEwVY-i>b+BJgc1GH z)FXP|H3LKcK7xp(T#sVy=n_stsWmH2s~q1QFr_?d2ho*caI}^LI2xa#9s{%JHUutT z>RQ~EGobr70cur*B^=T05xYdzOK^XS?3j}{)Ksdr!?k5>#|;(a^(q4?w(y)OedW6i zfSc5fZUt>9#v2A9+afj<0-@{%otbWDj!Q)4a+9P0_r-Evhac{`g=CTTFVNDpoX6uN z?iVHt_0X<|NxaWEXKQ3U0BMa6=<1Dwe;47K;&3nK2?VMa1r+tZIU@17UxiXve`Rc6 z2&M%X$7SCPNsoMk2X?mzlZ!(el8}1srwY|aR%I}aER-!~3ROqjrEzzRnb|!_ur0Ra zCto?k|By|m76+yl^wfJ0TfUQ#h)3Dx%1lSH^U|OlnGFaU6Ys-hq6d~Bl>1SxQL)QR z*3I^K+@CqOGCo_(M z`2~d+vW}l39z8Q7wF~rsZ)yeBTk3e0x~vQzY;WmSXI3sq14g$H$@&%2qwxT=ET_H4 zl0?jg#4X98gXi6B^!_D2S^bF^7 z_uv?;V1)knPuh(}R%(mDfw5GfON=6otP(rOLg9a=k}(-8kRz+`hEji=l4Gb&u+8ux zgo5&npu_6mVjMC2Sl@b(qFSh_18R2F#!GmCZ>)HL44KU=D3)E3q(3@E>5l*8mSW}Tr!gtsCE<({ zMTLtmQ;zEdqnIKeoKSUG zW!E+3#qjnkdCaOdH01@K_ba*1s#Z4T#m5ZHx*47Q(;uz z*8{NL(+^lJf_tc?c4M%m>zmvLbKi{UC$&@de{;WXi~G0+!Rmy?%-J!%%wF;U&G;2Q zSvN94MNK7Yg=R?gS`X0FDjkdNQM=tdnqlkHyY;2+$pq{#ZEfJhje?P!b+85$?q1MH zy_DrMB%|4J5i>qY-6+I{(b&+u1SuE<~46fCoOjx0n-Ji46l z)K3wat4!<5Wq>#xQbV_X>r9bckmeOVjk-+k%wFFP$#o-wGs8>rw(^y-&dt@?aGdxe zu83-`N{`I!p{PQCu>7v^0}Nl=Sd6T2KGF&)8Wn1OBHw1Jx8g}+KXwx`gh2vM8Wh^ z9!F^Y^CFz{Y~95Ta>eWlnI7KbH{F=Zjx5*LNm6q z2OY-<)Um2Pwh89K&$`@e^Q(Q+zi3lVnC0>cOEfXplyl2#V=e5IZLW>NxL82hRPI(J zMJ1E;MMZ~zybTORVtV5z*^PKrE1Cc)lc#b8y>{2YOb0{yM)V9I?NTvl&ZJD}R!s}k zTU{{1JEi|L&tEKFPJA6M7gch!1QUxr~BlD6j9amk|*E8md^#)@JTi4+p z&>|i^gWbJ81L^Yx&MpviHucv!hj&`l@jq81vQ?H0jmRG>;&99GUa_94r^28&cjXv4 z^t)ued7|-PZMW#uOXR!o0j%LjShoy9@uG_mpX0M-b8eu0x31LeV*Q)}g5gN$aSyT7 zZXL!JeiY#e7v@;uR4?SFyPDm(t-pM8BAMOR8aa#AFOQ z)ZxxNQ7t&i@q=;X%x~j~PACz2H{wxmH$4EEK_uCj?p2;5@q*V{+GvZV2PZYl!i0Y* z@Mrsq@pU=jRdju4Q(gBPo)G-KW=OYA{=;6`##~MQ!vr_g4dGztv~H%$Ol68ftu*hr z00ba22QnA!WldB2BHW$nQ>GXZM-_eQi@mB$Q$U)eniN@$y6k+j3}O3oOQG6K@xsMK zz*HcIBuKVmn2JUPU|*7?-fKne*0w@^7B#CDVMX%@7QV4W)6gQ5QrQiQFP8qA2{ddm zlr6ZWL`xUBeiW_{`KgyWI~zr7(d2aJ$fgNznmy%SuE?x*Tmn>-wH!W!x)uM{He6s0l0xF$?mWEsnI$40`H&SuZR6G1raa?UlQe15XUWbl)ni@f?D=`(zj6QGyMA}jESs1wn6Iz<$u!RajW8d%|SSxQ)vw} z1(3CgmeTAE+y$fM7J_ee530~jH%IuhG#4bDP|A+Q&wFHu(s8_W7lYt9 z!q7s4&reWE^?`hkoi!nV{ACC+eQSq+_{&g{$a>f(cgBPt^42AW`?sCI?1H60?dLh- zTxq8z_+zi$@iyr9CSoHf5*R)nsw>=O)c1KkKn(H`JgGQ8$T17ek_pnHo|nK^TNh`Ev-x5o?gRKy!M* zoI@_(7SIWd``*Squ>X8v6d(FIbgy1-RdW;iH@67?D_U%s7I^bBVJJ#L`Qt}iPAllM zbzO1bn91yzaUsp}&^7dB`Is^FiUUInmmo~nln=^tu7;oq_|nR7u^fuX*dHAIqO`zXl+ zh(Ks;3E~m?@hM)+^g+_C84)7AvjPP( zUX|I#9fw;O1AbtPWW}h)hFe0m6GCAWKlAx#Je1#F@boDE72{>0>e}nb9?I~|JaMK>#rq49O@)SJx-^CxkV)e3(vydZwFZ2A3 zCSdZ~eH8(G!ssCa)JOKH@((Z%y_UhpW>XISxwQTm19E>>xORsVIQj0riUI6Hh2jYi z+!9`r4|a&S|Bxef-a8!w-4m3O4%FD7>Cy-US4O5(%3S?|35bDvn`2ZlUintORCZeCv!3Dj=oCTLy_+Ddam zbRb9fFyx6DyoxquO3esjRGiuh9YJazg}Ia~FGAcTD0w+?uKvh({@Z=gabs9WfjT{( zdulqc*UAUt!)VgNz9?s>h8g@jRCSJ!1G;^ zDMdnzf+e&JCVJgG_%=qqBmctHN$}4uNv+WepCUf(%385b8KP4k8@I)O*4XSj*;70J zLPP3wPI3?g>GJ_jUEKO{gy$iYe2CxAb#0tL+-0~2;MiL7#rk~C!xT~km) zEcrXjsoL48l-w-#IGL%`Sau-2k5unDw_UY1G;F4X&w@Rrn@Qx}CntPoGugscA^^IO za`TtWODW3QO&036dv>h+EDtMT^a6se%wOy#CsM>p*(tI1IKukRwtY7xg}2>w8RG7C zgvPgXCH$i^BZr1a5o%%I$s@c(K{95cws(TGVoX^b5FaxGx-~zX;SlCDh9J~B!e=v* zDl3z$L979GtkCQ_^aSJB#f^JuDNTyv?Q%PB%r?8Vas#Pyw;*xd4#su0aE#fEuIIoo ziRPry;Ehvi(y|!t*zIzW{C^?`37J-g!I;zy*7vA!m`?eIoouMNBl_P*GhhOS&;mOk zKBRmTFg{I4K8>J{>K-|$k8Xr+<)Dx99vP^Qmzx}O%TRN!>)w)c4>xndPV7O3wFA34EKM=n$7=fKr|V}v}%Hp{t7^ze*{*?D)( zJM&}qsvo;@eUT9vm6;WpSrJk7#eubT9@;e>;TaL^5ZlQQ`Bnt)_9xh(x>GIkEjr#U z8Q7sEKHtJoRwWMK#U+SqAwtU%2)3^C0aSTNt((InqP_QekXBWS`T`FB&hu(bA4p4@ zC5!KRrJtuI2zNBM3Al$C)LR#bE45oM^m7rybJ%B7?wgQrVLZ&Rmb0?g^Hfq%(~mILQ_qeT-$Cf-YtL@I{Hp=^~`nVx{F-s75*=M|8gS>A8E~N$j%;;Sm_8jg`14*#htFViY6?HoG;#e)KI~QBg zk*ILc&~@8zsr=t>qR0W9#ZNX1e%O0(`ka2yiH zPK|iQst0I6mmF!%nUcEI5fi3VfA%i+Evl=No^+^n`WTYV_GW$>QwjdGUnv7`D*Ei%Y%Z zF^C{Jfi!6I;#3RjB#tFNEi>=Vtw^aV9dQyt(*M51Yi6MuadH4XDT}d5MY%02sQL~U zcXcpbR)9a$wdAKdf~>t8+^v`s7fs2EPtEY%8A7Z7>mOgV-)J4tf{l4e z9VUNe9$@4ccpqf!`VTgmNdR>RpzALCUyb04G%~QwOP@j>kg?@ar7-o^GY@^i(v0cd zy9eR!-twUJrJ&Ec|ROHY(l0u;yDC$DeYK6GU>co7b@+>>NhaI>UG z*Xx*a&(&ha=E85#zb6r0^u03xLmim8g=lkB=zJI>n;6qOlutC;ez2o@crHXLp3$a$ zwDp)b7B>uG`pM(fn1F=en>OhB?W?1+&1iC zjJyMcPzZnQ9f8V&w)s#9#*F(1sQ|i3K!q^ygzH~`37)9Y?Z?(}+i!MdtoEdzbyO$s zprq^L*{z=cC5fRCaKsVl7PX`Gmn5nw0F@(Loai0Tziw+Ig_-9gvDJAi=Ed>5?sled zPegrt9U1S18r2KkP>wR5VI3@AIS?HdP8=qP)^5mKPE-AEYoeiAa2#6_qtCVR^G4Le zpVVtTSIcVKe4eigpx)%duFC0Nba(IwG$;vP8hSla>srO;TQ_Uh{ZLNcSiCey-_BMv zk5yD>gGP%GjT`(0-q7T|1ZOuig?4oJ6c$A1@LHVgq& zu|$HXpJBq%%@kUp7M)}djUHrhNYRG)6{7zr{_M&;_z=p$h*8c-!tkMRa%s(uqqpr@ zG7(NYG0t`Bzzsar8n2ixePty%JmEMb1UpQ9r6oCB;V_B+;=ufQ4Emgbz(LGQ*_LrU zz4766VVwrHROx&KrG50)26g|CvAE&Z`oQ+lnT*PP?|g4ssjJAkIlESs}Y{{{H1tJ2V`xugTdFc*iU8*QKHh<(8p+Vo%i_BbK*{Wtac)%_PCOObmfL?!3+MQ@0Zsb*V6gr zuMgH&XZyF0$*Wwg6It1eWOAagqstLE&KEEBrPk1t0*wX)SG{+$bxyohsX<@hrE>+lQK{_tHgK~BbpXMwOmxgPQQN>m*-w4oh zf}2P(fQG*-hr9R0`z&X7xM8)VN;1HTznhfBUS?5BEeTxb7{K0b>hi-4&9#0iZp(a?D z>*B^ucVkQ0z$Q?f?&-9~J+!#COy+VOGUCFBa&D$Tz#u9U%QoJa8UCeNj%v(F-9 zM+Z)Ufb~|~b9Qp>sYsW0c-k0mI4A51HEt<2tJ@P2a(SZQ3SOdBR|l+;vU3^H5kSb@ zuZv%l$+@=1`090gN4;89=D0HI*&Ovl1NEa7cl*NdtUQEw*`0G?T6W;9Jn(MF()=9+ zsxGg^$d9#QdA;-V__O2<2X(PyXYz#vp2sq_2fK3fD1N|Q5&DQcfOah#QfVi2VMmFq~Xupp<-)!2mPylxB)G1wC ze^jDAM!x{4cM{$KpC1?vM(-$2!66*f`+mmT`dkNDNzVZSi9~_2!p!Vxe{m!fRbHK@ z2SE)N`0pxrdF}ZSeoft7;J}KI4DjIpCz28<=m@+Ttj>h`&-y|II}GIHf^cI;agRrR zwA@(jN2dYgBm?}P6qTAN~KRj&RTJp1xp0{S1=L=Mk8CCm^hV z+5?OB6xoy}-=ajJ<&%4>hZrj`|5uSxBwlH)rcZhEwBLR)2DOOB()zGeA!|At7-WC> zj$@3uT;AL{asE17)n{O@gZ60RhcNn49c;_=80|c4lq03|T@}=$ zAByzNp>fVMB6l&KLphsts8j=@U07`_^TmHHBLq2bo#d{^;~b@@x; z(AyX&=}+(1U0w7JwS(XA0Idg(uom2GOJ+>@H6Gaiu+ zyuY(%wA`{2_s4ebAk)u`ExNSBnIEL*)lGSRSsQx2Wz0!79^$S{E5=jos!hk3EefmR|I93-ZM1=F-ZXSvr=_!zIATi(D|E zcd06VFSchP9zd=9f%c{zki27e>;Nl#BaVm zufcIRb)GfPIiB3P|5_Ttq2aD-9id_YkC+M0&iwYDMcM~y-nfDqe(5m+Yt<3#=aoJ?fbN$%z5{ z3U>_+hff3R^HuSo)I$#7nczK*$JPEEB1EDU=I>O1cDum0gC`6SvVjGT0fZ;6oc(TzinZaXe;H!N@c#T@p1(E5yHyvI_NuAjg_<>g zPDOBy%`WiTWh;)ivY2_X#Jkt1D18pAl{)6VPMuX?vv}j*?RNVVMEQ+n&4&H3(Oinw zO2c)>VsvBsmYKZ(z46ZZ`&WXY&$|^sf6DQN8Y+)zQSroUc#vk9ACGZS@t}2jloed~ z$}??Fcj6SskYnolnMw@+TCJM&0( z$MyLNJizIJFor?G~}N zdJZ*QcUBjU+Y;Kb{N6XY$x(GX!Id6loGaCun)s6PqEC>n&tZqjn;@1JJ%tFLvWJ;# zj|;>F@Fa=V7JS3q)4DbLG~VItl6X_Z>Xe%k-zP7ypZAGHfru2po_Fl&?W86F%h5fr zS3;pZez?z$BHG}eLzXX^7V0gO#K6PGkbtlmrK2~2;RcbEc|#)ksXhPPsGWO>2Rcaj zSf4u7G%oN1wRI@Ks)S~(>x8A{TZ5kreqaZ)A3o>NA-Hvf&LH*L~Mi1%(K-6Hc@ zj{)p>4I%WE&r? zgn=tujmn)F*7GKB4bc}eyMtFmKJ}^sxv{vGPcMQ)*&9Eu*Me{Xo|_~2yvr7dB#IrD zyR8HOe1_pIo2v4A;>2u}FL%1&vCt4WXOxl=62do_YVEfhkJMt*sH7Chpl*x^DYuCD zR0UH+TH6ADW~E-?Pg%wj<1W_=4s7mwyFx^{~gEI?tfw({>Qh- z(uNMY|H`WHAJGo$Rb4)#9iqLT8zeLQLIL|S>+MSj@55wOF3Auqo<#!lg`7`3@5=C+ z5Uke!f!trmR#$PAuXlL5L@lnt8^X}Ga#_7pv#xy3taPm|wV~lz_2p#ZKJ8)sPx2p^ z_w^f!H{r2ni9G9JS3jv|%%{G&zNlo^nl^SvB%t1s_E$&2RInWZmP8hM>y*NxwO8mFt^g zf-99AVa$+ew(?Z zAH~buk}1L;wJVA62~moGnTMaW!59)^YnL){Z^b|TuEp{n1+ocKTekS;oRAx}2pzCB z!rzws6t+q=LTHhlnN_mrJj{DWfr)i9si{>})VJl%kbwb@O?Q>5X1P;~Oi{A%)Ksoh ziv*N}w_NQ})vpi^GMK=cmQ1guNWfthc+zIbk1_5IjaxY`ZR5&uaA$-XA48BYuo$y7 zXoOX&u-fY@x6wxrtrWHlg^6XEN=u!M$k zAjDBJoLOUpw4bEcI7^fmi1x?!S?a{8R6M@1p^2QSIb9P4MSLW`tL40IpmdHY526#l zl$7Og{%TMblq4jV0sb$mrnT4UM_e?~FjZg|9*VP3Zp89ZAg#0bsev-qLax)gZjV(6 zw*@7MU$hVAYfA6hXitgdl0k2v8`-^j7k$+0=$fH7O zh))?=*2YAINkMk(-X5jM zQ}d<#UcHivARKz@>AE^{Teu)oP9|=L&rg->6_~+gcp}BzQ`d$aMV&YTKn9BGibLw` z3#yHf#2rh`O<>%peio|-GS!=~4Zw38voQMRxR`oTwk|sQyV;)!2w)@6V|MG8Ez9M% zX4=fGHcry(qY<#$ ziJ5w%stDzMNLnsrGlf%|_Jxg_3Wti?oh)tT%{q+bka$Z!Ou$a+vH5 zB7u~rhq0+Aaj^)JsIVhRHuI>^%-XKR9&up9Ao=g+Nf#@=Sh~ zTizWmak(5zZp4y4VsBt#u`E48ejt-CW&ILT|o`l8G;!1g-`cFhB7J?0#A3R(FyzG*Rb=)L* zkanXNvjw&<9K!{e4NtoZgYk{R>i&(~7y5FJ-VcqI-%<2kZtgv2yN&-0 zvhN^HfqhXEIj9Rdw@9;E{mRip!w3(OxPkn5v28MSMot>yaaGq1R zp>?|hl0CraU3e7oKwQyJbDQCYx1gZcB7?f+YKuJ|X1Yj-T`gayAA8tmR1L!x%-tl9 z+mC*QYlULHX>`S71rxV9-+9zTi_|~Q_pJe9+18CP;>;&P4P!ABub;b&^+Nd`U*QA? zwSz%@BM%3wvvgbDJ*!-WOd015muk8$-@qOUTc^_%mxL7S2AnLOwkrc=F5W+2{gpujwdb&irkL zN=xY0!?Nen`0jPZ_73YKH^t~8lJe~+#LsewJ2)$qRF?8^I(I-c1O5N^Y(l$AiHCH`6f%FDy7ZeFlJOwqW zTVLM?PRDq9BHjqA39fZEj)K}g|Lf}U9(T3(A-=e?-46u6*&;eUSDudHlii!j+#3hK z{Y1Ry_SDlpMt?yJeuF@~pYznSwhMbB)AL&L^Prs*lf7T+^<3A5dMWLFPl0-I^jth> zs_qa!wRWlcWZ)5k^Rm+W4upD9;XP@ev|QUDyMN$XY<$txKJ9=1Qtfrv<=L1Sf8?N% z?1lW&!M(`JdkRl?8V4uof2p&vX>6$eOC_*GD1`DxXXBLOUZrtyB&4m|&!aL^Dyzu= zlrd&e)TQd;rU&0%pFr?bpL_A)MOSrgkpk&O2=U(0bDn#<;k>?=r_b+AJ`sw)?T(O= z^?e54wkkui>|n1tVYw`S9#&J%(B+S}BT?U|Y;fqS8F#_hmGQ4%Vz(ICzkXSWg);{& zqM-fme*MPRA-he5O?8RKtH=DS?^xUdGDLG<ayo??msZ;(%GWX}bbOp1b zSXo%FUWVRAqtQJVw=`GZ<4*?GW7HF(YW9%wOH?~)7FCk?2F`3*Nhb_h@VyDAmNK)p!|tmFjwxPe1$77YtX0 z^;*R(A`2Jk%MIPvCPvjYjh*O2&d91?xcb(sjP7Y(Q~ucLS!0J=W95u3nE~?e**$w$ znMj`dH>7L z6$Av}^Ii7S^X|WY7@z+PEa}ZY-R|fOJ}unOoE1)ncK^y2PXB)s5&qvqy88Nt_V(6x z|3HoZe^cw3T7A0I{sS%Y-)Z&!gV{g8(f;4y_VyP4Kmz}F5<`3ae}E$V4Ql3UVXF5J z*w0*)|E*^WYvX@Fe&!VUZ={W`o&7)R<1^{||A6hR|37t)^f$bn{a*>;|3Q?$>%!ha z*Wqtaj(_V%f7kS1P_)+W(cu)BYdi z{JY!f|6=+-#QOICKPxb`{QL9$ckw^hn*aUC{tpSv{z-tpKNLp?)4%uSe`odg-tiao zFMj_suKyhm`!~^k>;8ilKa)=Vs}1(H|HK{U6ZBtN{qMS<|C)iYf~CNNK2x;He`5Zg zKQPg1zJM^Y8k|m zY0(dhWzoS%Dz#BXc`m43G~YU!_>K%wi5k2fCDP;51^s!AQ&Dgb_hTz9i69l`DtFVsYZ;x;zd0 zbUWSO5bcQOZ&y%G$m3Nq)YR~EFp{!pJ?|!`U7s=-+V+R2?(FxZbIdmsO`5N>A$sG% zME-0nvxEg%p6lN+!}8PNdpwJ*)dHb4M~~-2ty1+a2ujWqjwqWrJ~Ep^asjsZ>01##AiIc9!N(K1u?Fi%5NtLp!Tx)8pq@NHn~ zgy&9W#iif|@EEQLYxKM{DIR^Y%6L`4RcN3o zT&Nz;%q4W`cN^J8iT>#VUtn$94c(p?p(9pV7$-W$GzK>Q4vf!9YpaOQ-90$`L^BCN z3~n|&JTs^wBmh1v$G%#s+Q1l|AJ%&K2>ujJcTy%3NjnWmi4ihRi4pu)X)mk&-J+}s{zNx<~ z-5nc6s28@cGI0Z`Lee6!t&Hafkz=h~J4}-={<1P-=w(T7YAkjZQ8I@R$C5{IyTJqz zq>OxT{yXGJi@1mty(mc{bglYsHQ=X-XV&sL^7ttb1_i!jN1h^{pA~umN(<@HbaZX;8tCI^$5A=G=iMyF zC_c`0z}PV#tmB+zSEF)N>jWv5H)<$!O^zZ9F(9GZ?>ed`OBalGuP+US9$)dD8iW0m z01E7=6kKoG_yv?uo?uXlFsfaGFo2KHec1iLt=&GgXdBc1r3gbe@Albdc4%eGW?$0( z?pA8G!EUvWsmQnvy~R=HaGHns3JNOciOFh$V32G3>6($pBY>5$F4Ml${>lK`4j^#9 zsVA9Dt8cDr+1W^Xz^U-a&G^n2>c!<}+ZH|*rIq>t@?R6^f6QeE>`f2JgdiaB0{=Ld z{f}YwADf7M8voivZ2NfIU;l|UXrK)RN*p)@2@VP#Y(Rt}<{RS;+GoKm4p|FeE~C!U zn=2^~u7{}hT@13U-bPoLGGTLHzMrd5UujoQY%tTVzjLf!zHUEQjpbw;lJmI8JUC!p zwV$|NS+L)^`Eb|s58l82NwJhv0QDl#2+qkN>33FrpW^dwnJxC6Yf>AVfJ5B)+R}7m zUTb+}eM7UQKwq+mxqnv;`y6qAI=Rkw=_L6!#YbmQ5WBg}fC5TUL9mH;wywUC1l82^ z;t-xOp9RjL;i1Ah(!Z}YWmcbii*m*VofSe^ zmQ9)27RnPRISsG;tUpv*io#LnqzP=U|I2HPAD9LNqyB<<^?5e#u)V@L=Dzu!75$Sb z^}guD54c-r)gXNx!GS4de7)g^X@`_Ar)_FDP#*C<+`-zko^6lipmi1NKKF}F<&H5? zAZUPMq|R;s*^n_U>H?Gks$&BWFN?U)mZdx2P=Hp7O2zXnRRQ+fhuZI{{uvp_h;P_;g z_czDs01zTJk;+qA%l5pkU-k-Uia^t^(F_jS>MIzh-b4Y!TQZ2W6pfFC^O7_sR7GG3 z3122bpPiwiF_n+w|(eJ zuStEo4}Is{U9L04|NGZI;L<1AK3Z>3OQ9p%7Va)2(!wN8l*<87?yO+0cg@D)o0aZy zKnP8!V^e*G@^gf*mVr{&Cbh(vZDpe5gMnz(j;?M#a)S_@ZavqmBd8*ahoXzQMvB(4 z&eWt~fYZMC2 zp11^)Awks{d}IKWzoYg^{>h>1CTt)F5sK-c6qavj2qYbFoe=1WaxAgwUAqv5$L4Zm z^o)v2jvdO11B_Ax*^rW=ByE(g+Bv~J#IcABK;EAhTITGd3gk55-;%>R;NUb=Vi2iQ zxn9B4j%rX?AudDOPVqq4LCd)O*|n$%eXLL+P(E%_bfb4PN$1~|#fV#vkA;rb&XK*@ zqZKNrd~B@y7ZNB14U$N(DuWqB_)A+|x>KXhHJeL(uc;^4ktN@(S+Yg3NMhCQN?;GNUUoUcX`7sfd}X3EPZE|RoZBz0!5 z3WNZUw&`FJBM}oL9GiAdt%ir4CV)Wi+1U+Lp8nb*%E*}qLp0pnBF)$TtM<~1h*1*; z8D~CXNxqUOod^@qLPni^n(2Dvc1r;1>xj};jkbOPVysC7tr;2U91u?HOJ{qUsP)YF zCRZRv(*9Y^JS+;kAt)PK!8LZUai z04NFi{sPQ~z~z^Oro#uR83To`q1Se~GGju_u?vFWJ<&!41OcY(O$Wp`pLNG+PN`xT zo!>{1#FOk$eM;6HLF>7^oJSqWMnEgA1?z70Y=-2nB}uRBj&6UVG!gKd`*WBK9msl# z7XHDV3*Q?=f3G2>0%85`_N)kY6{RpUMf8UM7l%lt!yK4_Qn2|TJxB~Rh%vL zBxe13%uYbsALNerb)q^|(q|q-J-Z@3&I!ofC|A$e9xAit3>e9kpmxTft|%FX^m?SS zM#DS{)puWi`+=MTfn-!Zl)Yavo6n|g$_k4i#&=cdzTal`Ui&j>A~$pbQ_#Xv`=&t#PsLBWfkZA&eqKnD8+_|AQVyQEe!>%kLx!kf&9unfj-cz?XI%c1w zqu!pS5Ly?kOLZ&;+4o`p!_r!8r0RfzPcP=#laDc$ZK>~!9DG7W9?<5 zBOzq7n*gU-&st5Fs1hhj;_0z!=xm}^6niy;^_`u6T2%=8tJ{SFA*7Zfs?Y3(M2T8^5t z_K>rMl3ia9Ex`yM@uI+5z}kAhAC!^Lq|(-zf-ZVNx3`RM@jU$c0QIFz{Pvy0K@O3@ zLp69+xlK`KIGBp*Gvu3O9TM^sc&LIP6EakC@}Ot@+Q!*lkAO2r!Aoy1#vgb3+@nh& z{j+E_rMzi-7E>He_sq#&!|lblobBMy&!F5-O-)obm@a*N+Mt>C{(2qTdz_EnnB0eZ zoIg>Kx@`@CjXgXix)kEjz1^{7VJE_Z6mYfbw~{fb316zIb-4ETVqdEm>XS??_RN$? znU>u{X?wIh{ilTw@cTl8YiUK&gnurYXPpICpamWs1~>To=P1y6n+YMEA~jR>c`s3? z&@BTxQ-Ct!hMqBVS!V`1{6o%MY^Q#=#PH_iY5_Q2T%N+T0_!j_iG#(Xqyy zZYI=)&!lWg+sgd?RG71h-X>?sPF$dMeLc}TODY9pKr9sYl^3r-qb(M0pTf?ZT_s@vCIbGBbNhFytgqSEaNUOp1gJylTN)~0vdsP&$1 zgDFpeBjol-``#A25tti69Mc#3#TYM8>z)2c!J7$5V~P!Pg^I9kXU`debZS^!oO{CS z`r1v;Ou_k=LR3ij^~ZMqQg?9QVo6T=R+Zer#-CT74?H>jX_r~Y6cgE~q}B8*DwiIJ zF4q*pIb#!IDqxWibVxz2%j|BO8JdWb-KI9hy2O=`1c}=wYBPR+rwIP*NmT5Y6o0R_ zen>8gon&c6N^v|bu&5tgigy&6@B!-M{C39!;tThUs2ME$($T*q#h@w-Qb_oXFb4*M zu(~Bs@;BkjA@b#0K(z|J-B2$>7%(;s0L>=Z5Cu>S%%oC5FwC(vsH0}5Q+*Xbd*U?X zd8KBy8Mv=s6&PA9;Qrj06c}vC9L#(lHJpJ^(&^UMUkKMtX=8Ctk{IYI?Qc$Dyc{-Z z_iY53g?&b!=7O+>E&7(Wx71bJP{v@L8DSrDbZRBWLO*cO;u>ziEcOe$tT)-!ruFM< zMFFl^+~PGnJ4#TL2>LE@-d1O^aMCyTQP!*V;!72dZ<}cPS@<`{tsFAS^M~v*OG+?~ zv{^sE^kFGApkRL?o@sxD(KgDin$D~0&a;UY-*QOFrgZ;uBVe!LUO}N5p>7!MmD#U2 zV{(;|df$ji&$GaKRXxEpUK4&py8}A%8)*1P47jrzC+5l~xSZ&hpt5L!Q)FF8d<>GG zt|ZCdk|b~obDV>J++foPu{svBj39n)!M;J&Jkz;`GT04i{%w_+l<4tvK+S~pjD>c? zCbT3q6s*}rP)RY{_+;m-1EXOB3K2UB^8n{ylMO?hSJuxkPI5uo&jZ?}WwX0fhXi}v zWERBLL2gP+`dblYPdNJ#0(hXw7*csy?hj(YQ539IxkE@SENs&qa5=Oj4^ubkn>b@qCHxq%hb(^!$vPHSv#`PBkLkIsTHPc-zX)} z_RE@?{0MH6NC}rJC|EMYSBE6oNEUyo(;HnGsJf$RJlrg1IF14P_lBG%RtCL}%mQ=C zNJR<4##6%PSms8=nFNo~X0@-!UAe{^cJ$7wBOWp@w*C*%cCEXNikC^}AVG68w6FPo zT^rImRlfpX-v|#uf?vO%?$brNfgoQYLAT>b;rC~_>3qi2J_mM!ocjb3#D=SGqlzhq zw<3i>=6^Ms(j)T)72@e?3Vd2}AbYRZw=7wu5JJi>QkaA3`wfA~AyCoD&?nQB$jPf< zE~98}n_<+Y6M_#CNbffe)wQJ>?IlBe!R2dmLm9I!zW5O(?1B~=G^A8UDVrmZ{|Bq! zTH*Wf3gqwr&4uzQ`s6&6NZNr)#*AD>gOUdM10Y?&WfL`-Rq$Abex>a@{ldX=wet~A z`ivv*{Q+=c$#n7x)x-^(!wr(d?JLI%`r+%3JWogmp0L9kT;heBV5kXV0k4Ui+B+DB zA5dwDlIii@+Q`!A-WxReaOlMz#3>eqcoN3WROx8Vi7&qhim5VB4wn`Rx!K3-J%bww zzKqNAt%VKh(i2s@gJVB83v&t=m_x)loeoY0I@L&>HDwjcpPhn7?9|^B=b=s?o0V- zXEAceCk}6PJGY~CxJ+}@X-RU5ZqHT z4B6_uACWo4spovN%!H0$K}AAL_8#J`iF+K*TkxpBu2p5ifuy}_ z&f)O_)l7gXiRy{yipDJiUo=FOuLEVM1AFT5E95+x4sv_ju!94&RDQ6g92sBe3IM+( zBrnVMn(IPGOz<)iZ$Q6Z3t)HJL#c{cE%++h_}gSsbL--8e}|RBK$#$^QM<|hoc~GS zJEX999P+%Uri4Ea?{N)h^mCa6G3zDu$N2h|k!;Y-#&un}vd}mjxoG?BLaSifkzT(C zr7Ou}t7@1f!THM6&+l2baD$1QTL@*ucJ^5b{9Rud-*B~XK-wur$=Alev%E%0P$M7#Q)XO02l0f zuVi8nv{_^ZX>e}k+Py-vUjsLCi_6@nFqx^j&zEMTH{Q@L8iFn(zUdv$WuVhhYD*n? zQ212k5WK|jBPfJjmnrVbaB>z>Vn1T{K$^vmm!IrU^6;abe~pToR%*Z|8hICul&`9{ zPHvwI0@dPWHR;^;Y2xCIvUTi*y{7Wr2j@5AcA47~+`J*&9Mtn1(^YBa=AeDMq+u7> zxl6R*9S@sk&*ed|*)hM_5!GoK;G`9Bnie;A#idl3{Y84~lr5l^pqr87>|)Y6{s%^J zj<%=_P}GX#_gPI-m{s=-+8>LnY)$?nP#L0aIKH3cQk7-Xkpf>s$5+}OIv>Mr@LmsG zuP@;fCewAhC*&>z^gXrp9tgeLy`!Pmc=^usHp6v-*a6B)kj+JFyWfhORrg_b-Dng& zcQ^0`EdBl7pe>#G`k^ROFX}ck&9*a3E?d2)iUt+DJ2div1gVd1s1ujXy3gWxaeZ&khPld(Lz{5L$c7Y%+%5QEls`~SmX~- zTu8fgOFgCdi;<0Fkq;P&N)MZ~w9LT3On&I6K;dyu#c@xv0|QgVT7!oS1i(a$_&(b6 zoj>TgRsMm1)pstz8L$AsTeBV8Yc_6y4rv#+;P(?2Wh+Yj9NzSt4(^ECH&6X3ahJ7i zVa}WNq{DmAyhlN?tHNMAdIgjYf7}-)!>fs{)5=K;&oKczA_>4wcQz8lny`XzEg256 zSHTpo{5H(YZ{go1q_DP~7LdA%>UqA9bcRB@dp3zc!rg^_J zVyWrvSCG}1G>8+QQ4i6@Nbx_P>J;kjDa2F7b_J!&lXNtENj^hu?N|2s!Cp`MnFvHm z@=)@JCH<06+Nc!?JxGWs5psDNp+mK>)`WrtZgO0LD=kl7Y;WsATm}&>@2oPHih9G1 zSZ%tj%nHPAHH1lMd9a;D-)`eru6KEOsO3#@uZqlG!`HAn^i(1GA0b-dDN9lFdW!0M z3MB&S;-|H%OkdfOVP@Priht+*S{ObPXx^5oW?uu^O^hGiJK)`EaUJXiGdG|aG|ok@ zI3bfQ9aG^<+*p*-0=Z`L%nN!)?QW&T6Y-p`>%M=QVKl6zJ%l$(E%M>0Gq6}H{QK+~+~kT!?uW#thB=UIV-cK#+^ zV_5*FL9@5A%x{CVzaVUH$&K!lknEqB3`(o`o|}F4L;VykZ&k_A)AW$6AHcgI6DUr= zPjXC&za8#pA_Y5@^h}<}IpxW5k((#VP9JX!cQvY;F@{FRB4Bcfo|RN-llyzZY2;by z6yB&+II$~W59k1upy}ga+#o6G4Bf9uVLR#Rzi+m8{wwnVx|xw(2BB@$BjvyfIptU$8us`7`!1egvf~`aMW)iJc3A&&>3?#Be0kk1_DUv>F#H= z(k^~uv5^DOU@hn~p^$&ZnXukIY>@k}E4)dWIxu(&8%vN}SGdYZRL@rO!byyrUPWr1 zu){dymh;C~ZH3P`DkP*vCq1i76&p2e;N9AgTOI!4UWPEFp3hj?B?_06ww|%<9pY)) z$cetS1jAua{L`+&)8A@5y`wO%6O>^155T3C3N&g`H4OCmOQYwRKupV`ydu@Ep}>!w zkwG>Dv)*a+hBuJFH$*$%y^R*%`49Z^1f**(<@E~cQDP(i^Xdx;rAh5c!~gza7C&H`PfxW;Z-2h5SurwoZuerk{wLccNki6_V)>B> z0l{_x>EQ=3xpCm&!SqzLiks8o>6o~3uueSBJa#CXY>JdNIfezCpEiQ2-eGu#$uuEB zm`wf0oGHKou-{V86$agX$mgG)Kka%;L`-2Yt?odAF|?i|-tPrZHbJbUMobY$xaz6B z=MNpZ&OIToN&&=8lDAkrfV6Dc+ZJ;7`?gVI*PGd0RB%qaGoNS-ZMuh__dN_+!Q}<% z0`(P^(esd3s2DJGOShX;!RZ`QisTHW7jJ*&dXNY16(JqZZSUGXySBU=78_%e$Hb9s zsd{hNz0&nqBXy(-sp0s!MRt3n-G?W4%%@%8Gkc{2I*dkNdF(y6lAdrzpG@`ltyvE8 z$zbMIA~WhWSP1wHMX{+h&p_==A$Au+O#)RwEhn<|Ea~D+Z;F0t@JZGODCN=?NY;N{ zE~3r#ShnLRQl8Q+55%80J7O#cX3Ws#T$tV6np>i^Rj^LJsFQdUm2VlG>B-E@+a;*< zJ}o~t&0WIf3x_|F=Y4Vb&87TvjAcG<4mCE_Izgp2&NE+R>6L0eevYfc>P0myvO7X& zwux#4BZu;b5K%=tOHv$BWWiKvh3a<=yYDA`J3N(Use=`kBF}BOR#PIiS80Wga&7ab z#dIo?Gvt8h_7>B!9SsA=tpc-lWsxuRX?$ziSC+4)eccmEjI${uYJSx@j6LcHuE$p(1vEec1k2kG2xj)1i!bajKGiJMFb*+&VOH4GmGM zt-dw&THuX5aX4>vQaJwYm3fkNl$P@i>Wdcn9DPQm&y&W55yxSrIO`u+$(Vf~@bzNV zE}R!^l|Lf(&#E44R!N!i&g~Oc38jEHOn-io_@({)OpltzL09>u8B38vGuT7RS%vr82peNKCNm|ZJasC8)eaH5ge8h z<#?`(OUA>S!*kaWSa8V=1u6sD0ivc7KuNHLx@sB&P0+N^%U~Nh*#NjU$8bV079{h9HCSE2Ck%S`lZ#ceS-Vr^mHS& z4=VdhP|+FJIt4=38=FSn zy8tCL2}tO_myaF5oM)P-)I`o|1ugvf67Yepb;Fr7r$L#z-jJkN{U?bm>JnPor1tO| zX4tj(yN6760txRs+)XZRfb007$*=6~+w!?0;YiQhZ(*1{LvsPlERy5qF*>5($=f}T*nigT~&Uhqe2&+Eb6HCVJh zCK0y_g)*IxmC9qvKY$6XBAV8W$K{yE6`#cAu25+^Pg|@>-#8*FZ+X;b=oRmzKAM*R zt(iXN^FlJ^oTDY5>Yi@`177o#ToJ|&3JpSzpj7Ty6GTxiUu(9Tb>obuTEmTh|G*r# ze8K#FfXDr^4Gz4MvOD?7B0iB#${jiFz}u7|;mMu(^%hI1AX9YYN~?0B#%5K=ZdC_; z4t{tv)+BkGJp!Ey$ZYDf%^oCyHeltMVhTF&#`KQzq?8>2x|v+ zjmU>uFZocBxn{08zcEG~D3isGTUvm_!I8slGsVuo2Dp`bh5J;*;+Jh)u>Ow?p|p^m zS3;2wVH2Y`c8Bl)UCo01TjSaAeHbses0&Z8R>zIOsIE-iw;w02m=9j)a*NeP_m4Iu zirXUh59hQ-{#|(+VqzaMp9-t^2Y1yk=t?X%)vsze|5%9RoEJkKKh}iuEeCHn`ah0@ zsL^L}90IT^?S-!7JHC*S?hiS{@dREOo+8a~T#$3NVWhSZ-aA6(9@lsL-szOeV!qP9 zqKnhNgu9hox2n2W&>%_8SU_K^5kpAsst*D42gCun0iT~3O7@t<5FHw>%E56?b?>~6 z1jN<;9AxKVR3cwsAG?f`bFn=XSM|E0XjH3Sga4@4%$yW5M+wKgWpq`r*_iJIAbdZM z|C4UG&yUYGhXetk{$|6(|1Ui3KS-_rorR+6AZ739=BncAY-VIf`hWcvvv;#Ib2f5y zaQ@%$L#3LIipn?0#V*D96Yv|85?EGJLIkmgPWKBg90#G~w`C6o(kMpIC@ddW?aES@ zYlhYK+f4G5Ys!=>_jjjZ3JptU`1BzcDf86e>|vdUNH;nefFD}FtnX9V*!@`9sPXxB zynGA-*pCZj4up?z_2g$lMWUcx#u-^bD%dAyarac!hCPk8Abt{_!gs;9D%4>g{o_tg zhd(~a3Dp7Cc)tWrlTmlEdefNdh3$a|gkP|-o02Q)JAXo;Q;f@UclG2JZ8 zSb=SotPbx1Jh(cO5Cx{4Ky4V=2oYh+OsyrQyA%k1EzM`i-I6rXSip9sNPmfFJaGXY z9Z`)oI}tRmrpW5)h9QJ!Ig0gOA4x9@)@c}VUKjxDt$uL{xl6F1*Mj6kG(a5 zJqE|7;`juFSBfhDsVgCyk?#d`63(aCMTV0F1=NP*_J!oM6s1p)g&McT*0)V^<(zsB zy=Rx5CTQ!|LtnDZ#_ge-JQ+?0hSEiam*#Av3H}c`%RO9ut2l!LUb==yaCG0jcqrny zrSQN{x>$O`?O1+Y)kIucZF$N~GgOTpKy^xD)tTIu79nMzeVKulf!dOiK6XxD&xkiD z$8#HDe@m5pTkeHtOd!+KbUDFq!u*y}U2@hIf50DwEWqD!nEz}GH~Vayd&%F$H8sR6$Nh!2z04AkW0__I?s;K|F6L;9ayT7FFm zpIOTak}SsSbJ&tvs}&PCFRH&@ju`$PB>P@knyV9hzAFYYvyHxhYZe`)=$FIMSfuE4 z^*)Tf7HY^-zFo~a>)BPN#R2^m*mB~wJzC$>Z^v7IK(8mwqUjE955P`MFPx%{}7q2RLP%pj%}3FCQeVkjqQItSxXA(uSu`S1-B&UJfZf0 zN{2cA408eoP|Sp~;8pQiN(gW}VjGD%MSCA!s)0D66ob%ZF1b>cCfxFAbBi!{a5vV6 zEPU=n+(mb(S+A|#*GH` zEg{MNySJW#hcK7o1s>oV)hQQCg?A7T^$`;0(drWfz zaC=K~bU^g0{aZZM92rQ@pa33_*Q5kb)53ZOkr=auju4}Wtl#e#O>^4b6gsQfUi|Ns z`a3Q-3A*|2mvs=6zSgmiZQQTx`Kr62J9OV2FX@%xTl^%>$vdE24q)4bR+<5oTOsIe z1>+y>KM)!JIQHb$s2!Y8OCNNnB(~Ab;}!rzi~pe12ip+VJ$e4t&~wT_Y)gJ=M|^RS zU?HY}VQBSlfr|$Ki4cU{3j)fsq%|~wanggg*FN~f?9{L({}$VH^RI=UR1GtFYbDMN zwkCVnb7n92`AUi2Gd{_i^G_o^G3Lcbock zL%ef)(OYnL#~PJS7^%a1&rcfF#kKJX0tguaQ_{aWcJP7GXDSC z$^Xxut-NkOFZh!$p+L$;IGr0pdb}_Y%&MfuGx%398hUBsV4+YPbr$XU0kQV}0XA66JDuc`A_4a<<14LoAEJx7toy6hHiGO#nbeQ$XdCuGk+$)G)Xje!7@YaUmM_WwmqW|1k10Cg*i%#cusI=Erm8 zP>1)kBw=dzYQX}dILNJU>7a;0pjg@KE_UraSR;yMJ$3e3jXPeOG_J`o->A(w({bS~ zSs1gl!XY;l5j-HFCY6IkE@&o|(eMF)t)yi^$vZjT61=EsZC+R1F=C<1Qqk>H zniMCWi$Z~9yxwn@Z_-5Je#I*b%)M>|+H+)Xk0-rbqUS%L?ac7W-n4IZg3&jz&HVrU z0R2A-5Md)1Gok+@2l?NRP@xv$H}jbIwaqcg%gp-g&Iea=QQZ#d6 z#IA@C^H{kx<;18&2o-iGMvl@@hemy3e(o2|65f5<0T`N*V%e4I&NpoSAVc?MZf!GJ zXXmLhw5?Ld!iW^J&Yyguhg7@vT^iud1G4Q$q#2ehs;4M@17}F6&CD}P8x2$PuA=de z?#l3LR0kob=O_zxQbwYr7;xe^eRfY-nm{9+aI^f6-32wm z;M&@sQAd>leN?1SQESr;dsvEit8e5N@H-P`u>W!tE7>f=GrfgbdDY{03rPm-qlP#R zYnO)4BM)F@9mSJa`p!u=naTc-(mJ^>39o8S2l zLW~_-R0|~~c?Q~yKOxc9Rnk4HEcR-Z5iSZxyqEG%+E@L=vd>1aNokx6xUvZ4_VQ44 z8L{v!*%0exB(CKpTR64(8`E{#y@!G6ELiq_Hndr?;0*n)sN?9=Q0W@DlPTzbwJO~>xr*lyEI2u|*Msr$xphw)39ov>A;)D^|K z*YcRp`Y~)99l3&3ha0S*)0ag_+TC33?{bleMt@Dh$v6}2bPY3dd=}k_qpqVJA-2`^ z7G#@kWI_a z7fy{GOD^ya)QI$WP_z2JLE;IvyNBwqeUVO)Fw`58&ir+^N`~0p&h9&iDW6LaAW0A? zuQ8k%Io!mo^@!L#UG=&3(n=^tCkG<#XnWQbC8 z9Jp5;Cel2wCDTRsQ(|}a_G$RU+jZ(rFn=In3~~2!=(IpKb91N;;ZNvYc=V4CtkXsh zW9@#<1|jwPwSar1apo|()gU(BD1ea)scau}5v_IE=VF8g?;vT8Q=1oDbXxgblH{zMoCEV&qCs;zo@U0F; zbylFVWq5x`{-n}>MUfS#Q4CzY6HJgSi?NU5(rVzwh+63_3ICYsOGw$XP3!X9zumbS zGAt0O-kZmWWb6wg+@~PVGFrN1%N-S}L{Mdj9X6T3{32J<0ouk4yPJ1WW?Z69dPkLS zGA5QMo#zqCNlA#)i_+E4#c3f&Kqj=AK=S>?n(;`vg1wQWwC<<7`gfoU!=1ApG`ckQ zAeo3OC{&wMA|d%B*R?hOPH+k*k$zRx1IW5dn3(geOoRsuj%J`v6;Y9_K*H zko^2|Vuq?Y3*Un;d7ORR<G20Hbrn|IVOk+x~Xw!YskI+ z)T&|HIwzu47Z?g^298)|#%H!1%?#a4f(=luf@{(+Bj=NlG+zkQ$?G+h$Y64GT8bw7 zS8pn=AV-=aNqfssshuWMpON!SCmSDJc#|7K@$+HV1droZqsm%)i!T;KzQ*)Vs|a8_ z%?`yboq4g%*tsFmXxBpXM8#{&rtE`%&6G|1qQ#O$Ov0|{hlk=hWz7w(|CLJrB8-Yw z^gu%jp12aDvb%>_5g7JtJv?1UxB(o;S|7trhC+;ziJV$rimc4J2yNuhXktCFN*uvl znlv46ET-=nE8D0l>ifdP5BDATiELoA%PQ+Sn3(d#`oQ~&kpz{GX!cn>XN)^JnM(*U>RtJK~~*SbX&<5unS zupKB=SP1rChicKcdgOI z@+WsginBtzGi*%qV_v_fIs)*R+&p)okG9K-&u`_5GcD#{;QJsZbC@uox z;oF-f+`+t64-+D1qHyDsiU}gW4B$KBg@1$5IK!l9@64jn7YNz~1b*}xY3~Wb(NHOI zZEjsaWkZkxgyDt zQ-ML9wFA77Nq8^%vj8LkDN)?-j#rSRf37lVkG&`5>kIme>j9z3jl~ED>!U3Bk0|r5`H1TzZ;H*i zh<;bP*oXIw7uE^h+C=E~1s@0&BxBW54VPF)DFDzL={dudjnM4?*K=>_9>{h#V7qg= z68!Acs$7vWlqZ##rzICwiD8_s9j&hRr)KT^JdEVq}k*ah$qrf{tf5oOxlwFu}7-MR+M4G-9dhWziZo=p)@&&*3)h!=3stqIs3 zk2^(>Eu;4U2+w4Y>qCeg5^m_^IS$*SZA=N`0>hXLau35OKmj5H%mtw(?I8xECGUyP0IA*)2m`B$23drF z`DrL750Ydk0@{)S;s&89_q&k`6u=aM0BXnwiUV5{2$Jp$_VGLe(uA_(491F_!$r|k zE&>jNYRCgh3=w_=@j}?aj`kY@G#+9_=(iphs3tZN0u zhFYSIGumVy<=~Yi6q0c&lOp0JNauo`j zeN+m0U%@wo)35>oihtn`8U=(ZLlSQq$pjv!uz>F3O=n6smIO|+Qzkpm#~v!V5b4A( z99OHdQq@-n8^Vm&kf7=A$-nKmD|;nFX7W-tpFyt7r!H@ML9ZQML`GeL+@Vp=LLB_u z7uU*VI$ps)&ei#I^9!iX)CYx!J<4H)izQj)(y-?yfaXc8=yMZsMR$xecb)G_JJl!3 z7ShvNVS4J#Yig?*;-&tJB8-b9o>|u~$%`tUYX7ht?&!V%-WY@>$3SHUs_M5GoGN{# zkh2tOrRq}HvuEo_Z9&FH4@ zrc2ck213Vc&8i6ko>`VF5F2vDMTV~GUgM*r8OkRzDgZ)a&PYH#XqYn83YfEk6;(RqDa%1_j>R_4R)&?o z@cNdRP4RU*Hy!Mk1`P&)J$`qq)i}Fm5_4={fNa?PS@h0=V2VzeEx}wN`}M}66||Zn z^=@uNN>gVQN?45RQqz+Xg|j~qO@ebW3QC=m)c@*a71K^T_z5ReIs(h2Je5!j^E%Xu zWk*Y3D4=e+j1EqALS5}TFWqJ9{&rb;{%a}5YW&WJt{uk6<4$>*x%#Vtf9R!Pxg?h< z5J9A)Gt~1j*4bFHQhQZFL#()sQ)gLhK`mV;Sv!PDd)ZnyQ)W5MRE(D-%bDF@1{TN|Ky}^WpCu{_219>m1^5AsA_1R_6h8+S~(Ds zLe{xrOaeKeph|P<>2qs>lmc@Yxx`zf$?h_BI_?~WgA`xze!HXC`mRtG3(QNqv(R6Z z!!FnMP3eSyJ-VYF=D%0|f!}+9EBHQJf z3EFDa;C)SZc88RWt+cyjCQibD8k%cm{{ZvCk$j~mQe%1}CP|Lc% z+Ih=UZAGA6|7h1E{$s_#0ck| zV8Yn&hS!-8m=GwZaX2*U3h9Q?-SGQ4duOKY(jC+)9_d0|>srp;>O07(?W<-63&v1u zL&j%F)Wod&3XY+w#T#)%Z#7d%2s^1*@j1^j0 zwo8y<+8HiQxa+WpB(-XM_h(6jpfT@>&^))wm8=%)^vo!&T7t)!e0v!0I1ab|E!-0E z=c!YUqcCXlOFKAoWm7Bh{B-PnFoZ{O7ak_%C+F(wUQ^wIR8XH~)&Rbp9!?o+JDj1!`FLk6}zahMwCU82@_N{e`{$>v6UG(VFXkwA@Wk z0-2+P2j3`RJX))^EVLjxO#bi2Xwp1mSbAN#!|bXn*TWQy*WSC-Jy3-yy66CunKlgy z8Fm`2A55gEv~d@w@q(DILll#Cn}d`Wst@0~2){vmM5LdXKSBBWbx>zcv9GSdcgm4< zwb5MJE@MEh+BscA*N6jsSVXLg#8YiWb(;YjB@klKJnzhINYR{gDWrFOsi(GLmnax` zAHJ(thp$I{noL(VHsaVqmGj6+ZzMNtqIiz-1=p3VfD>TUt___Y;*E~}ysP~q<` z`#|i+j<$4pjy9)pcE!Uc|T|}m^kxt|;VRMt4FbcO3KU{~*#zJxMJrhYoH-*vuYe*AA z`Uk4*o$fyzFw{k8wfG18VWXN@Vk7|Qt6=n79d;TQ8X0;aS+uZ}jDr3WpGXI&4jb}` zcqxAB2bhV)VCg1F_2P^BgniF4@s#Z977^|u5A|PX$&Sc~LFnw$u`n>#hr*Gk$spY= zIH3W@x3H{;-6f3~_HYEKA6vO8S;_e5b~WkW@t58$OMOw6UgW~W++pQ*VEJ{g=Qc}X zXqE0hFmTHCDQ^>Fcm@7>@M3ok#F)hl7w(7-W2niY)6Vj zc@gz<^xL-b=}SVnka0HgKprz*XJKj1rXlfQ#|xGo*Hz+oj1r$E3;5g%&o% z1BK(8T-U6?q;M|4jFOUF&uERz=O7#4!hioJ&O`YtHxL;FWi<~pri(&852c|F`b7*! zc@-Y{w*9V8&X7<_mFT~1dbQzm_4-x+EiS*D4G?g}>Y*|Y4CM`6bF8rU;lwI~hn@>O z67>y>la)ZZZ4;z4El`dy)l*?bwr%jYbcCZ#b(A0_kh~&TCD}nB+j9zh{hSyw4Nm|5 z_~;Y5ip3>r?WRew{y7ugG$t7WQ3Qy*_sHDNC>|k=Y`_k5wBc>j5wyjil)S!To6t*i z);Qa<)C*%vjD9FWm9ifnYj?#w(JU)0!AtKMbVGp zi5SJg;9s3t)xtu-xYCO9f7)}URhgi;1{w#D+BZB528q{^{EWr{Gy4>!{CSB$N^;lC zb}H)-Re%NYr4xZWxoi@sXG1K`xKmam2+N7i0oI$O&n;agasnwtiiAPqyNsI7K69X?(fDpk^JjMxa4Fid-*8N?kb zXTO=rZYl4-$ZhbLM1C5p;+g@+fqsopp#7Gol!mAJZRy(p-x73m)j=nRf&8gn#G1Oc z-b@y6#Ws621Xr=(Sk%*D;%V<%vQL{KmrV7D4=WQ@ZY`tY#?z`?%V1yjDAic*S}}yf zJsqzRCB0@bcSo_uC+(YwW>9Rx_c382v!SbYcWzn8DuSQ$h*OrQe(-d3FY@~c@_c2SK zTGQjwj(Zj$^p?l;3^UHfgc>a(T9JNtV8}CAaA-soyEv^GGCojoP12m)u4t6Rj?`)q zl9ek=jrqvlRxBAXxk9YMm_q`S^{PyB7c**V=wb0|&En`cw@)a*p4P{JnJ9Hx~U@xz>l{+~)v3!d9sG0XC6TVZ08ID+tr-HW*sJ9pI zM457RAiqN$=w9qhp4O$Ww<}+j-m4O0p z9BUdlRho)YB8hQ2Sx+JNks&=is7sY10ea(zF&j#JtrgLYKYsGA^Sqfk`->&-_&XSU zJ_CNlk{!$gyK3w-Zt64Pj1w8ZWSaU`HJf|PZ~4Sg0Rr`YW{#@fMjU?DA#0UVzE~}00W?9kV zsA*7Kn^w~M+(VP8adD@tIo>*^u@Gf5cv^z!2bzNK5^Zy%g!f*ct3f=nZua3O`u?VA zx)lIQxfFdX5s}74LTE=`%XgcOWoMrIDhYcBZv0$jY9oKY-kN;SCk*zP%U8d@vo$tf zRtDdo$8TawvWyG)JO-!zpn%ql-{8@lqbnQ~(EufdXLfY<7NpB|OF|jEvSE`vwITlE z`E(^#h&g_7V4%QGZ*x?%V!VT_N!3gbb-1Oz9(c@2-a02K6MI85rThywiD{V)vXaK< z6;@nvClZzUs09=-ZpqOVRBg>En{c^9Tg(mELDo;;b&J=P$WtTee%n9D0GIAJnIMaS6jU=wD_-SdUjswYu9J$puiD-nvLA(?vX-X*zf}%L4PcyCNJ?J^pxAw0W|U z*#xSsLXHbS-Fdq7Vyoj9laS8~HRjaGx)594AV?@YY-GWtO1*&k7wm?S#4N#-00{q7 z{$&}=QW*OX9agwyQND;n!#r%+aS=Z}CK0m+mg6eyYvFhIxY(NR4apnuCjJ-@5r2Y= zpS_h)7aV8y`ou<9ENZ!>^e@~O`$*d`bCaJRG0e6;#{jfzHXTS$AK&7x)_$n(6kH&3 z9hCF1buc@pYsOT{MoD>MchBGYbPTCXU6D?B1ybuU$0|;(LRh_!AmZuO=z+abl~-U| zgd{n^h^Z9)whp8I;No5Qm7dit1Hb)K^EIH6BGdi@)xr7&Gao1u0SFj(bt+F{YZ2(IohUp-4Vd{qYLyz+?C|W0~HbG}e6r9glri-3cNf6j_R!Qu%6|J}4 zH(CYs_T?;RAD{9ncRV+a7o}xI{wJG z8B&=ho!M26DBH`J{I2I%5MnPLYgd!O4A z2)n3>zHa48yI%8Dw|y1v9JtKL^TY23Kiu%|fDf_`A?F~{9c=GPSCyNwaMu<8N*=s3 z-gmuzpL?bBcF!kXSbeku*W@K&lXAhvTc6}74#uQaFwKp=NAaaCFl_O*cMFntP9<@s z2dX6OnZQFDgnOz+L9OrjCC;lo&9Uq=`~l*Cn02q8`V{P*&$|cymVnL_C7$S@wf^Q3 zLa|1`Jm=*GeDQu4!aZ@}uB>sz!3*u6dsXS8@p+%Ib`!8fY=AE_>CmYoBgyRkwDH9= zcCdJ3iXz*$&Z~la4F@n?dwXN5gK%0aUo9B zYg8sDu~)1xDtg+t=xZn!hsZ(JAKU{AOW659v9=HQd!`O0#}vlFT$b^AmxOdvVq9R^ z^sW0-lX|J9cT{t|@utYxpsd4PW+^9ZfAoK--l$9|6s9)FAOG~O%Vo%q97k`kFRRN5 z@nwnu???rjO4-z5AEAJ=LHCB4n`i~GU?LH3#-8HoIrEGik6qI^wBd(X&uH#v`819l zD-qOd@dlQczT!ByB)$6bjbDXnwb>hbwaq3?%V9TXShW@gnb85BOMLwqo}%~LW9#Vs zs~;nUI;O>6kQW|I2>OkblJk+RALWtw+(OXW$_Nu$Tod28!pkMJd=?vBSQvi$>vJ}v zT(U!5|6uM+$ZC(exDp4y5@NYh!|+VpUed-mm`^&lBH4~eG1jUHt|#1G1ztGxNPGtq z??gwh1l=}mti*!NbvWfTI3NGCM_HZ1&pCsZyceQgQ%HkuaSv~v?QhD^`J2l=o=0xE zM&}SRR7oDA6DzjQL>-X98d`b++edJ?<{`P?jyls4jeS-#+ah<`sW;1IojA^}bBE5z zbB*%`BeClh3eEF;yWF$XV6D5vfXzF=sU4I46_FyroE4Xzus1Cczm z>A$DnS|hykbNGpFOziVytW^z7H~quefXSpLfrPw}sA z`3sJdOAhqz!Xo-XNb3i?s7c`=Q^`SNB9eI_KAcL=_*%y`PMOj$N;S&t3@*+91t*bFvqm!*aFI|S>t=3Q-jnKEM1lkx#`wN2-HxIwEN<9er?$bN!fSryY42$grd<;Ii zK;K2sG}kWd)HKk%iJe!gA;ODWBOQLNgM3Z2ym-VX1f}SZJc!~5<_x;AS=u7w#FS7s zdZGhWrZq7V2=_n9ItYRt?Bb=Xg7hc?@+oAQ%Uan|oq#jw}} z1;AGIIo#V~k})r=CQ*E8iu?C67gKmZ%a!QNKHd-{!SSz87kGB>jK)924vH;Fvyk>v zk$pnB%f}imysu#l-Yq%$_nglW#$ggJsx6_SsTA2Fjg$eLd~5zn6XHssiv&OCO^c$l z2E3OvM#qRS?y59{sq*0NgE1HLFk}XeStH3b8O~~?GNSFB;Eau_ya%0isizeEZR#sh zd!-@S(nhgQ!}zF|ysYh$tb_177P8RlPh$!>MJ&r;?L{k7`a54Wr!5wA0jw}g3U{dY zU-18(AllG(ic1FnriWnuhldj{*Z%-OG86PvP&KfIzrunmV52%HOD%K38WE``iJKa6 zB5{N%f99|QQ;i*djG`oIxu0G^9d?~roFr~JndoFVEtInS1No;2m|oqqXHQ>Ha*1=9 zy1#oDIQDw_cz?SN0MQ*%KxxA&vVI@aw`(t3Xz4$e+gEJFn`A}PuPke_)0W;HQF3^N zhQ0L|-Esfh`!{6~VKUVu+>D$9d+**{PLAli9br$3^){B4TgCR&tY-7a;hehPaJ9nO zFF_}b^L?nLzI+v-_3nKk-9mCvt=KTFZ49wFn2wolqr427csz-5gt1{5snz(aj3#mY zuqPR;Ukh6#ZwM3=S$BL-B3;*^MC#6;bRRDiyP$nT-{yoHd>!Qpo|%6L<}@4yJ|+`C zac#z}STK%%xH5RE_(XS=jc%AtHTWPnTFF<~j%4}UMlPviqRwhz^S&-wYfkP85{MOV zN|}b(tYJ#U`~b&PsjphT#$UWQWkx=JCHLG~U(U+s1q-Lwm_QSPM<2yCIRnBhW9%=!0i%=|O>gS_SsSJ-W z+Zhp!x#Y;Ir*#PZ>X?QpWi=Y9&e1YoQvyvB7rpA9`PWSUxQQ=4b9~y3I4XAw=mgJN zca`SiD(KrpTBHVa!ttX5-gE|wy(y-(eo9bW$==IYH1!#7OHFF`mIX8GaUB#eh!#{b zWsIU}&y9#Y14Ut7DQ-Qr*M)^EW5dAyLAySJzYA?U4ccm|H5R~52)_6oib4&QATxy< zAYGm)&3B7Zgs6+#csxPM^A*ZrDnDd9#)zOHjOc?@b~Z52%0`|Q`(?<3(-+BNAhV_U z0a#WoB)@)8xkrPpWyR{{ihN+*t|a`<&O_%^2!5}ZCY+nsM#tJ~O5-Bpll_k_f(63r~RI#Flm6 zsUbh^%|@Pj1Z2>5dHV1neMLIjyC2Bzjx&@4oj4CEP{IhF@s>>BKyQ~D>yFAKjI5h5Kh$tV;>d`(SexS09pXO3hMUBwKL zBnRYoKQnntdVG>$$bU)%Xq^n}{u9~*!xh~kUF31CHQ3;uBj74AfjWN)q;A1qprjRO ztFLTfv+Dr;@2|P??>q1oC=igdZ>Q1!3Fjs9|L#Gk+jyWZqi-(bc-3GbKplt{KscuT z8mFfYA&W#7Rv>2Fxb`i|kPDEa!`m<}|tvX}%L*rREc3 zUuOsx7yYVCiEIn?{d&@FJ1wU8e6o)h1wLP!r$I6Xpvk$>pvDs~8l7u_bn9_^`i~CW zLI>*qD)gwij5HDJ_VpW6Y;0w*;3(X=neZX2x*NR2dnwiY405Yyc(Gb|fqW`dm|j~R z($4<5SU+e|M0xgu$wIcE&o(UJ-MG};Rm{d}#}m4mi*k<8xFvMAW5m(L=XgvSICz!? zWb*bG>k~Ju7L$c(`^XT)8Zf~XF6^Mj7 z)p#;Wys{!1`u6RFH-tr}%xDn|{~m}? zO}b7Pw3%4Dc|zq@Ba`#F#+ZY9Kba%jCd*MdHrZ?F>mfhZKJZ?Z@r`Ni!mpTA#ciYe zLFx#n0H14tMj+Q5)5y3YlhYPAIdPF}NxvSb7nzYmWh<4eT8WBqJp>~z=c{5gEpkD} z8$}e-(&DZ>2drl#H(Mc5w24cA2lKI8da?#UyhexAa}GDiki{%wVB$Y%4}YX&_)ZBE zV4f}3|9Zl(A7ctxefZFP|FP2Qa`d)-wy?h?R^aWwLYmIsY%bT4Rl^v8xl)t0&mcD5 z_8rm|Mg>2gqEs^=UD;w)!r@1>J%rZYwX4%9erGDRooz`fbNrIm$-L3snI`GFrgwK) zg29M0?ZCxUhhHwne6?!C8IO%mU}xcldBl|tmw)p+W0wfRkI!Ti>)N zyMhi#E~iYZ(JIxcr3ES$ZL})YcQSS#XnP)= z$LUqTQ!^l_({(`TBPHRUg^s?fEUNuWHF^j2H{A<^lkgxm+D*b*6$ys5Nc?ro>mxPs zwbEzz5!~^S$C`W&y|@^1=v?!&jbXe-VS3T(W1(+4*hC}Kr1w_rEiGjER=hK)u)2SA zP)x2`<;>5&B8uBV+f$}(y<$O!BhceU2DRI>#eS)8q*T)rvzs`SZa-I5k7!>uT`0|g zWK8C7^*-O|vV-Yo)6sbpGs%2)KGqV$c}Mq%XnJF!v;49uts`;BLU$?}wL)b{pK=UG z=y$MyHoWu{mj=Q^gZ^)dU}iw=jLF9&F_MG9h`?6o8^g8PKlYPAu};2?#hG5dzhkyJ zJSH0w3g$cfKTM&nNm98cE)8C5!DDayB-4H3&xS1zs9f*axqq_~hzSt>QsEf*3S-rk zMpg7CL*sssmK>n;LomoYU^IjBh_zw9*Ns{-Np-`cGv)6~z9z1^@{8P0hHg3E?$Nkn zJH1$Tq=1PpaFBka$8t7>f4-QPEgwn85lB)>{O_ zJ`a{;8+6&UjwTaDqiY z*MigATyf6?+C8bz`akDdzG2!(lD--`T@|Ci*7%git89{l>q?VddiS^#7loD0c|U5r zHa2+>?0LS7A*8&`b=zY7J00?%b%bEaDyKy<0mCLJ;m6v#!fS_8B9k2#&+NUo!-<}q zT|`w%&0*;ea5R;nQgBxw z52m0Z4t9>VW}d2Ej{o`9t8VvSqfmeQ1b3^p!a}sM^52lM33E5pGfC10h4C0xGJ$zZ za&>axQcV~3lwzV{f=Wuk*b;c%EH6}3mY|tVW0)DY#V;gy+3l-ERyd)#gzT#>S6tJ6 zQ{#LdU$57(-5?jeevoEh@5ZGwubQMdkycGhb_EQlMVX9qM07ZdbmNR7o8s2gr3M74 zsdGGfnPAm|_v3i&tTfj|HGSlQvG!t+IDM{-^yDHWRb#TUtD&ppinCdg6c;}N5kE0k zSYzNk+hbC${y?iyqm_UW1k78KL8zBeN`wf?amFn{vy_J011#eT=oK;9^R%5+Sx}YT z6ZL(VjbQO$7)T&V+nKA@DTt3BOE$CR&k<^83Hht_=7h0%;t&umRqW)rxJnE6b2G>` zeLIu$-+&uE$}_)ehHB$-e`F&ZFGEm=q24x_TgWo==?8DN4R!W3eIOP(0+#^ z&MqlM2F6CLl^7?;ZcvBLu$i1^7L`s$TaQN5&%dNa(ydVE$l5F!lBc5mG~w?6T*Bc% zE*;jRkCIU?%rDxcg_asayQ9o!vBsPZ)_VT?*9!Nl#!B={ks#qxaL2#YPQ4bMPBw;C zP{Z6<*u4HiHyNqDR->0dy`cuzvcQ<|X`V>$JxUEmj{X0t`U)gIlhC>Z+ zfGE%!(?c>A$6Pk&GBHQ*-_6KO=@l|`vzi$08l)=b@@TW*xw9cpSSa^2cL_fAR(#35 zZ5X~LWTqzYq>oIdyn-7_|v?0@W#1gE52;FukBGks=RYGR%z^pwH6<`-iN}+rr@q5J8 z-3`Ogz9*xbk5h(Mb7Ryo7}Lh8qhl&G;158k?XmK{T{p8$D%0g{s!TJ+8}e8)W^g>? zcagnWe1T5tbYjn$a7IR*TyI!(qhz`ViBO<-fJ*7w5M%Cdq9u%D5ivYuuF@E<-~;F5 zDd$Mdkb6n$mYyrUhSEs11B6->4Ofe$XBn9?h9HwZ)a1dBV zD_sZHoEp|(lRGM2g?{8U-*8y4-+W2X^zr$L{;xYlwOwxevX&4APkLFsD5dGyb0kb0 zZlDZ$XihWsPM>fuUUkg|dpQ+#jZgP6c_c-e=?HypnzWo+x0_PN`Q z9}Em7(ift1g^gS`6~g#C9f>^Q!US?_)wxqLR&+~Toacx1dyE>SOZh#t>=6F_N6?g* zT3xDNF=J|o^9nU`2#UV@n9yk4*$CM*zJS)c?SBFT#|U^&EO&_9F-M>&9q9fd zsZ`PZjYWXSHqp6f!Sma3JoOIhf-XU1pWd~UN3ck*-x$~@&5g|k6M|Zdrf$ed!<6KE zG6sLT*VzF%#qXoS8^Uf#*g}@AOJ_M5Fw@87XnjMa_!L3l99>V75$5-by`h(9)=GDT z7Qr%a6E9IK9j(lCM&XH^*VU4&piSRRs$}|Q?nIuUV0D8%L|`c9_`^htqk7es&%k7} z*N(bHj?KtM(wlCm2AQx%VF>Moq{MkQL77K78_AiOpTPd4Y?`yoO zYKG!NM5HC3`eGL@IP*ga^0t{6hRrJ*qiIl%k!Os`1{ijlZT%c@Lh-L$(D@+o#Q1K-i#LyId}xy zJaOCag#W$7loU!~I)2B9kbY-ju>TKA%y$N%nX8rEf7F*o4MbPqVygev#54zsZyzb6 z@aNA$VP#*oN#MR-JH-HH4`KIz}VDIBUOoX zG%Gf(m(R2-l{M@(pH}Sn2t20k19&f2t1euwIash!);2G>}{f zdqMSB+?s%k1#!h)t^ zg0#NY%Z41eyM=eoV21M}BvTFw5_S64`x-k_H9Op$l5`j-BTZLqtuZ)FPKPAPY5&Gt z(1w#4r$PmGTAMs_f6~f}=Mk(h_6~4*LN=A1xe)*j;{zjt;%j}kXJ;L<(2!iujiL0Ir?PaCRWHQH@fYr z%0AZ1`V`S5Vd&Q0ec5QQ_5P9{#6UY{ENUKllm&PvU7UnJmB`IO(PO^cp)OTvr4ru% zypu&$8{;3`sIFx|)dNb0yf209cPbVB-4+$3GPOl2)_5M;kK|YvMYN_8$+f$Q=@9w*lWLT{eiS( ztWjj-asaAELzUW{0UDYeGb3yq>E!TaY!aGgXmJx@?T<^W?W?v*nrvBXHSJD!;eI^3 zXx1jDXqK&D4!XcEh6nj&MwruLwy=~s6623I*e;Bux9)O-)3hG&g8S6qEaSp3Nx~|- zw9_d?r(Y4E+Bp6(^d&FVoQzPb_0)ZB1@h=qrj;f8)PT`ZD>|rxcIZOZ+*NDyk99|? zIT(r%kq+J)@@YeuyC)7Q{p@PBN;9aA_Kk{AZ(w5H3sLVN!T}g$N4isDO)gGCt+}*1 zZ~Y($p55zsVwnjbuoa&W#8g{S+20*FT2tGVO^%>@;Z>=oXw_>iE-h`7s=Q#26O13t z0+_3Zp^i3?#YkhC&Ako?5Hfp=QpR*pU%T!^KGDqrWc}_TH7Cme6+;G;9!#cqRA9zZ&ZsMEveZc{vWU*+ z&7{`$o!YML5e&wdY}2GJF2$Ng?C$)PqjYqolM%tvb*E~OyYNwWr>V~LQLSb$ zLdP7WRYGojdk{wN;fW~Bql`AQ!&PBF>ms0Uf6`%`0!-6fPip~C6yr$cv1#?w0~+F! zS~&8kl_u$B?oFB8@r3YxxEm}CTw9?NkPnXqLGL?c!DzOOg2Dfouxc`vG7Hx8) zTBqYA7br^+Xe*SK$Rp?Ew_){kKb!E!B;k z)r~xAd`5eup{@0OTR1Y!L;1T|OI;=0scLohs&M}E15_DDsiksbeUfB}{*zq) zY%`^eq#R=!7jhvTf8UKX5i|Q{E9TRN8hp{G;XMobP&Ud z7G~2FOXe#xOHDm&0$3AO2^8ZS+YMN5{W|DVijL={GBP5l@lo?NZg*xZ@x4X0uH;Z1 zsLo{xYUSG+`PNcp8_m%jZG0%Z5Z#Ka@~59?6rg7BE=-YK&dF{qEyTUFW@|;4e6t!C z+DO`1i}JQO-Ylz?gr-RIg|Pmf?bC@QYi%(-FgRe@QaBEArr0tVSEGu3Ko`9!7rdi^N$Uw5H=POkR~Wr(J-(o>Y^~v5)2ymFMA0@Qm5rJj0(@XI9z_@V;<4T z^N+ENj^xY7d*1mgLFOPd0##msC3H`m2nK`1!tIl{!&!aD>vOqHG!LAXlzT}!15F5R zvcGu)<8j%>(pd7$h}x}$z}$JzI{b3AN{*wHEdcNBe%e8BX1NxYCF@ZwLEjRR+wNAl z3+(T3&t;{-Yb^_tD0?W6d9+v)^}Yq3Wl3`#*b&rkH<^1;B!ddloM6x~=ut-5bikM> zPSv!0ndf1HY2j-lxK%4x^rprb>g-GVg`#Jjl(t7JC^ z{dmR3rFU{hp)(`zGLO+!X&I06OPA7-FKI;!@nGy$fj6{^I1}lQ>+#0rJpUD;*gtuR z^rr&nB-m@5DY+mG@GhyBiCjWo=g0}2vVU4Ep3x_(>ZMdk0T;ZZYcXT)UT^k@jqKX; z1BVCY%iQZ?75;Pwag>v~){?whb&XB{HHtY;s>hxSgv@TxA?3I0P?Ev38{qs2d--k{ zh^zZuUAtQ#yRcH33ng*8uR3FRRdcVJm{sW#`3gRCrafMNk(CkL-puaWBwV0VbCKI& zAW>?8c|>jV{`F*s$24Be)C2a%qu-x^F3?ASb^3*VGiDdS?$MJD3leUmt{bcR4AG^8 z!L*w-MccBYC%Qa`>VX$@5@3zKHPER$9(4{{g}Zj9{NZzD4f>M|JQL>{n`;#$cy)@) zuMX5yTkN{3P^nvlMo`G@U@9-6FwD^=npIpY98*K2&I40}M>`phgGF0|r5)VE4F>H- z5ye&&rCL?QS=8`%;ohD5>*E@Cgq{&>d;jvA*!-Rf`rs9j|I!Qf>(7KQn&htf$U6L8 ziT0?aYg!JzUb$;t-t8~C8z!&toG~yjVy0_bM*J@Vndc;{Ua~s0u7r?lPj}dyQ6)km z2AUxUdr?XZQ-s632*rxs7R&y7jWN!KCeL~ZXY~8bhI#RrO{(L64kn{weoN`nW8)94 zi6`rh9h;fX=JZqQ0Ta2UzilE>5G0LpO~S*d4hX4}Z@g&ax>aus`NLt5UDbHgT#S>P z%1vKW!(DwJmos%Xx1X?CNZt^-3qym$M?uN&+9)ZB`q&+mcGN}8_Gz8cTGb>-!i^|< zWCGNR`p8m%lphRAW%N=Y;313W?>ttloka*VB0K_;8{HuHwVVywr-yvlB^Q8K5r8iK za~F*34Vh!`P2!g7+;;O4*oBXn%=C`yB3PDBKB?KW4vPSY@m8hX6`aSuMg4kNeEeb~ z+AAJ=&X%D|^>%UO4MCUh^nwZi8{JpN_`5JFc3y?O4gK{T&04^-aO#p}&l>Kun9S^H zI zP^}^QKBkB^Yasz5M_!j=lOLi8PT(78dVY?OTi5?u%rb2=U4JgsTtKyYRehjdYIdV7 z7Vz)_?}__NOC$z|3sR@jvY18W(q~beUNbG?9VLAfQP3VZD1&cY)iMf6t#vFYfhp;u zt3oa7*pJvczGlJS6t=9rnNr9{lWO1f8JB?XAn08f{Z=UivE#t8p2aIWmJk1_F0mO~ zoaUVI7hWr{eSAPTErh~>gpwq5tRUKe6~F#~ay$XNBq8WH&hjN%H}HZQym{2QMJK&h zD|wGs!|#_d+T)1Dc?!;Ri&f8DU9@dY#&x?@ck_Xx7tux}+W~qzS*4p^?{iwi9!3FQ zGNccU!n-QxjUF;e9F5`dk#T#!F=37whLobDak;XjWRXPmc&s&}RMVLD3c+2W-syp= zc`TiHKI|~^KONO!4CAQt_yNYKG2!f9y7v zoBVABFFxeJvMxsC;ty_00HgGzIR`b}Fp|4OCuQ~U-fb)Fh{mLcTy|{EXaXlVzDcR^ z>%L;_{FWAl_-UkU&y7Q}h(_VIusdLj(73ozuLq_8x8X^Oddb=AU7M7SUgorAPMRa? zD%N2JluUj2Ox^f!WRCi1tFSAOMSMeOs1|d-1dSuSlXEowYS*~8P`7vGHY3-`nIcVD zb#mBKqfpH_!y@EHE}s_Z7B(ks3i&~za*NwCYw<_=t7=of^ei^2`KfeMf2y0%*^wgM zUd4IwvYIC~wI?b@5+AhdK)&5_bTZnv`_}$FcO9?P9LPZKiW}^Rcr+U6md1u0WB16u5;6(daH=&F!LA0uuh7zznBMeDnDJ4cRU>5-*j4tX?0Ny8*oPhR#)X83T zUofJR@g{#}$<@r^OPPtoXCfbsk z@fx$V0%v`j>s7}QcbDVjM#AUAlx-L2@xTORI@tVsB4^sv;e7j{g-{bqK3A22R$21c z$w1yki3E?SwE1D~7oLE){*t69RAq@8HCQibuyIQWc={ffD7c`Ii*8IFRwcA1BD#78 zEA@wwIb}JQX+uRROhfTSrb)C#G{M%#NGz8DD!9ko7t%ky-<(x#P;0Et6Sw4p?9TU^ zvN_4Y1CXoWtkM=t>6+~pWob$$r<$lB0BBb6@rT)w%yI_avU_P-Z79k zjY_vVqjV5M>B+9LZ5FGd=mP0PXz;8wv)0&H%WA4+e;dDK!h9nmLK0ITbkoSnbVL5p zem@TMW@6RS!Fj3q1WRfVlGs?9^d|gyWUg558`i%PIJEO#t-z`HcDF%U%sn3uFxDYD zV#U&0fH~bg#YC38<=BV9H*eENK$$kv=~-K*nx_2l7`aBy&~F5Es59#LO%D2`QS!wp z?)>q0wU^g--lS#dOfNQa+hJ_)x{i~A+58<&BrZ%(UQ>$g(xmOolE!|@-juHoDIt0$ z%l>Fp6Dw`WO(M5N8E#5yHT0m&xD(VxMJb1YF-l1}3oWakIzTm=#jBi}Oi^x?0{x{> zi{sVWLQYe-p#h$c`8U)+rNXGjd=2|)d=ueIg8}FE*X%hFc#2*22=l+0=7h|?qROO) z;De@2e{N<)wEgyGx&}8C$c^o&HE6~&|K-XC8IKoKb)p0S`D@WL)2R;YSW7g40T<<( z^aQrrSUoo~Yn3Sd1d&EpQTI>2$>GR??~dZ@LC{B{6$zfBM>ac(Y0-sSkI}k)G2}ZM zsVajqvW629lTNZ;lSq3hM=5w9x2`EejW!uQlaa;1F8|cdXI~We7@d!74{;SewyTY1 zG2cpFGU3QvM}dfa{#McJCmGCHdaabKEIw;NMszD}lNR=0t;H2CwsH=`r+uDYwj{Y8 zQ>m$H^#`V3GTiFd$$nAmu;d4DUR8eFwASilI%n+gH`X0ThuA|hUYDn7LsC3lxp(lI z=~s@|(ABiMTJF}q{%#&e+%8*55r5xQ8fmA7OkR)kcF|9*NbF$kt;7(tk_(Qh-<~L1 zIiBDj-LKzjU9qOrc;$B8-CjATzc!xCs1OjmJrOF;*jnYTvyCc?*M+7!l((d!z4w=6 zcyYT4?thVg>miqv2MVxV^ar)r#uiWqHy)z`9~}a)$ds6nrHL z*}8)6McKE^Jf`AK$+$=2_SuQRg7y}W1CC2CywSy9Q$_DJqV5Jz2Lr#H6FEF46H54_ zp%gI-+65&o!AHlYlNB~^DG(sY)%dLVIby6B7pt~91I#Kq?A~m^=^tyNCkuy=B}__v zv%Y?Ljpr=Mxf9w#9K*jltv&NjViITdta!9C*p7SuY;6*QJ9))Z;aD|Si9UC<@DV7a z1N8=KHQLrN(JFBtPz_pyBi*2I;+XMPL5p_eb{3tm{rs!0GTcA!7(|N47V8djX0K!B z>eeMyh=2U0+c&M9oXe5RzUQ9$kQn1wG;&sR*Fvhp_KWj8#*b2kvjSJg6adZ(<+OEH zQC6{Lntg`z1@fQbU=h*jLfm&)6BG3R>c-;-Hok-IDqTH{o!d*InKm8fUK#e(MNM&W zuinW{^$0snUF+K<%FO=hPi1>gy>TCHJe}{yboqRubp-G0*i8u%GKd|4 z-+>;_n6gC$dF**H_T1UJ}Vz%al1+P&*{|$RSdsk~lboXn8g?Nd05ScYD-ATYOf1Si#rOlM3#_ z6nga2Qj(Gshghe5mSMx%h(`?DL*=j&!R^$jcZ-`skVJxaj6r0?I$Db3?v#dZwDH6` zmQ11?$en$j^q*dPXCZ^X#XoLZ@rht{noI_%?WogI6;(vwcHp!N>W;xxR-7@SgiKx{ z9!v97Q8ONIzXLNI*fCwy$nlr5|7(PYc;enHZHD-E-v+wmz$zo*3(#qmfwiGG+sb$ zwrb`?&HLCs-u{zGn^!B}Dmio6$JC3W6EZ;9dof=@t$Q&iFYqo?{4C~k-tIKedp4dtiYwms>>*YXakbo(#ccby zS8GMo>V;U|0*0;OP@%FXZWneY&TTA9?=i?bR$Ofm=UoG_4Cr9v4k#VItj9Fxt zKV}H;IU;74fQUlM$d|JQEm7*UI>&<~+N|Nu;I^Hy3o2t4^VSa{jSC{9PGPWBh)DiWx3qdZ}g0K$*CCZV8&!f?(^?xh7>~ zr7sD}>jFiK)%bLK2_7#+l19NWo49UDgDb>GT8YX8^8nCdm2pmrlbD@8s-4&%;WKYpr&Gl-AlwWj_m73223K{xg9) zKYmQiFpUW;zhbi{bC_4n~x`H~zyao&>%?$%$dNV{z3Cuiv(neH-EgGD0 zU|PiXBJ)2cm zOP6p8Eeba1@&Kw+zE#(-EBcqESWutYpTV4gk|3n2cANcmC)tK2lHL@E)PIxr$3d&Q znuS`5e?9g_7>sTQ*14ch8|b;FkMU}AsN5bY0d5LT`c)YGNbm2Mij2A1gcGg2*`Sf! z&3|fQ+UsGEJc|p(Q8V6 zlhLnpU9%-W9X6C`=*G&Cy_mAe@jUfk!N6by5jHjUvJb9=QSVQyN2PDW?|XYo@hZ6w8-?-CGH=d{UpR{i{BS z)1W1t`ArR{21}`}cxK)e3$GDw^)Q{|q1tFya9^iG@UPB?HsHrM=pQt!Z5g!LqF=AY z@UN!hAMIox0BsM|t1a&H+>|t}6A538E9V)Qc1;HrMt%$g#1B9U-vb&R(OZcSFeQ)< z1bZZTkel~-R{iFU$zBmSe1yN(C;Lfb!u9hra4KcvIW&6h9gWU$VXl;Z_K7)*N|6k8 znuGXYtlPl)O#0f%TY6KpyN{FihLXY~{r}a=T(LMwr2Xwy7yq{Ku>a4n#n{2cz)D#8 zzaU0g#}-Efm1i%e#hz7`Z@0jFmepLmHVY9Yp-H|inBAtZq=+UQt#seKBRe>X^{l21 zKJeEwK)n49_f2?zj9@00%U&QVf!p#;mgiNb=jA`9&%G%0XhDty2~ z2J{>p9&Fcr_(On;WvGscXESxcwSERWequ6Z+e zrQN?7dn?-^c+D zjfGty;Zo9~pspF$dKvZ;fw|HT>n}Lad>Vj}IRLgbG9TVlm81gW)@`gUrNCr^R^8Ff z#CH6bYS+9S6JeY4yC0rVNmts;N9sR147aqBT~ujMZ#o5w26TH(hH4NSMZvhch_ofq z<1yn#&(JURMGPXR9}aP@VO{04-d6vvcV26cKSVeaEm8Sc;3vyrT$w7~%~-omHRs## z^Wc452p@2-DP_w{!S_l2)vW4-(^#d{Y6+D>CFSIu?@yp|O-3eje;j47x4XFfkfY6; z4h~8mC&fCJ7xVg&NnYHtaRL$Tvk(1#|FYIf>8AeshJCAkSFC5EY`@12a8Na@heFjZ z`bvh)psp4?0iUD>k(OA3o9qQ7Xo3Uyh9nR8h!fcGB$ax-q!+&4GIPE}^`$7&vw4} zrXstXR1-$hS4jNDB`oPc$QZ(Ymoknk9Nb#^Z9)1^Ht;I^w!aPX<44K2n)Lr?Vl`y` zYhslw(I1vT)f=ma#&QCS#JLn(Tcd1coA(yrLpyJ{U2Kf}C3 zhY99|>B+n+405f_{?d>1Id(dn>^REwJiMIJ)9V6F?L*0x~mxl_? zLFSyjkl0DdH6C_zd7n1GwnH0Hzzs=pdx$0YvcV?2Qj&tK0+7uFCd95Y=?F~GYUN%E z;619eVWKYj>R>>gQ9+fx^Cv*gc+cP7zC&=gCl|2h^T2wAcS~w3O;bA=)Io&(fQB%F z;R(#m(OmSD3purDH=%k6i}az#BqBTugkI( z=7MG+dx`c?4`K;|rEhSDbjTK8toscoxE3WcqL*xZP43txV_n;~WzxW#tPAxHB$6V?-+QoPPjwLBmJ&Yy9D>zpt- z`Zo3;oKi>s2xx{N_ChK|Hcbm-Ge0r-gdMVo7%kf2L{W0cG6?vh=GZ7cBv|PC|1cCT zM4n)NrJoaIThE=l{^MabJMoYFj`jjVgu6wn4q6Mqn z8{U>bm3QfrJ#3_-EcxYELkY+rZ0qFz2#k%C&DxzA35r%d{;4Jb%$$^{yEcyqG!qg; zg?f6y!fSgN+4|3^IciUq=J@8&N#7j$e{!xNZEIutU-Oc9_P<1eZxW5%v;wgvPiu~f zct+CCYT39~C=iG^D+t;U{GDR!8BaaJI7Vy3s@x9yp>H~G8|`JEd1KkNZ9P~UIr;qS zSxjTAzDD`g>W6U=uk8Fl3Z+*dC4gic!elxB`$Juw=LC ze`yz*!^;|bBU(qNbJ(P`^$wCr4>t{`t1MW8l|RXS=3pH}8kvep%h&CwD`sZbLr9#p zj24LRnP6>h0af$7(@DLJUO+jBK7O!(UZ8U5`RZw;@y&)x)zSj}1-uwypE<+{d^OF2 zbCzFa*aI1n8#GZr5<1gE6kJmDUsE0;AnYD;wh5-MG}43t*wi*GAi7MY4Ble^HHaY) z`7A9A+}|TPGmr6a$mNv{j z+3|>!1A3}B z4|a<*^Mn|LQ8Fc344k#*l0-VQ9*h6QC8E=sp#`itM@()@Y7H8Z4K*{C-)UMiawKJ+ z7E$j3DkNGY2Gc!vtwa46v1A*Q30#JwjIM0Th0)m>7r6W6&twYuubMg^`E=$ADN>!w z3l5nHP#*LS6NkImb!z_1*$tTs>mk0Tm=#SYgqcY3cGu>ZPlKj?dS(q6KtmAP{NQ;z zmMH#%g6bSUD3C4!Nb_fc5RLEJRldc5^$w8V{sVC%6dC+vj?N8<%e?kED8ByIT%e&@KxJ9K~fNRMSFKYAR7IN`Th zy!_FzV}Xjw zT$!CpcrnL@U1pWJIMhyrmAiXyP#UhfW{RY@bifTXAhCNO;?~a@LWY#uv|+`EY}1 zidLrDkCdU!jIZ1uLex!XzJe<`ulIY|oh@fn^O`arXR?Q7eH3#gM>*_lU}W2SUZRvw zUKjKG{q`_d$U8haAzE%*<~3{Z314vk2}GY3bDGZIKy>{-1fuWGkpJ2OWhHKX0}<*_ zR|?o{;50}wzJKn!8IIzpqA#i?`6x0W1qG_V+27U8KoaWu^WDqf@CT8tir49rd8{EU zJ$Lhu|E*Vg-SlATy`#L#O*`5O))CXs$74NSay(^DJ$)1J&&H>-s~@z4DEx4J+oQpC zo$(`-RO<&Q$^Qyhs%+Arr%YIeYORcQ_Xgc=NPc`N374$TJQ87^iaIN~4q_@@us<%-FX>?mKVvpAg~*Hl zl^J7UvlRxCFv|EOF0hIuqVO{bjd5=6hTr}DTX3XEq6Z0E+-HV_ogo}=t%lQV-Pti5 z)s9YVtA%;tmQ7$6!o}qN{oR)8-NYJ%IVOybEjd(EW3dpLsKf4!IELCL3C<=8C#UN> z7Qq+lD@$qyckhu*rxmg`(g27D+d&+GuuDxO%);3mCI%YVvqeHeV9K5rm7iAC>q}5M z)5YV8$Ico8>QKtQ5Yj=rT7L!RZX5pSgS{XzP7@SMYT-0rO9l*%xdn0VRIMxmY;&MT zPK?g1CPr|;fWn*t1^4O3ZOBQFUlj6TzE34Q&inBLu|y|dTI?n#dICD5jE=L83tutU z>+yJTmFiIc$h~7>?WyR<0;ugz6Uk+!1{y0mdrrt#i9jh`WMO1{M*G9@mevgS&8e?xJB%akEY^b((@0=IQMi zUgE;JA)@`iZq($cUduua^4Nm1BeNT4$k=aCXe`WT2{!m~tRZ@X;O?zr6I_ss&D8QS zc|0uR%@0QiT^_`IdmLwSW?tbyIspwcbwG@bw&QAFandjN_f9bq$GMG54nOszqBaqk(V28nnDi<)78S|#yhMp!ITWYh*m`2~Xkr>SVhR!oYZ3_{@w+)O zI6i*ty{f=$i`Z;R3>7hr!l-d?m)!Ik*a0zxRJ>|TT!CSp`sHSccd1>6?c` z*oUh6RPT97CA{)r?;x3E;XIL?bgEU*`Na`sJV7jmCpIXv9{;eOtp535j%*v^$>^w)x2X&XHP0RB`K%pnO{b)SReKU=N z>+`O9>+^>l96mWDly)i0{D)>?*&?%;R&9Dn>jkE&(n6U^$3mS|W4r0fWL=jMB2vq{ zErEMwAzwHm<^+~Zp9V5yF@_5mA(g633_+wBE)gagyB1^mt7x0T%!PHO;VDQ0NqJcs zSGx6ZhixRgr>GP^4Ua;vhM&5Q>us{}spj0o4y{aij>-cl)}kUk1${>~z{288Nr}cb ziPMw;QhI=aCLuF67AU6IWPRZ##h|qL;)LGb*+P26B-1_Xq>M*T`npFtdBnl`pypxK0GhoNT>kW zT$``S$3(wr^0KFO2QeOEHG)@)kuO@TD9>iW>0?wy-3K3ho7yGUPZGf$EniW-R>@`T4`Q7e@ zN1%eG(bExb=KRO8$%r{=27>?QpE0%w(y_nZJ%!1TCnFPm(1dHM|Q61@b71qu(V*O69BKy?Yx1scw%3tA-u1oz$@6oE3z1R-t0t*! zPvG6P{BT)sI<+KL7S=y;{*&~WAabyTd@pFRvHoXE+V5^O^Z$N-HL5|mDKDUX(lLz? z_YnowfF*_h;ZKD`>Go$unlLK$lThxR<=1jgbC3O1&vAz=&Su?cQ8_JB)*?Tn+WNA@1=8m{om92-Pae#QMRZ1ab+9) zPj&LZUqCSQ!2HjNoyl0f3M~&I?nNLm4HMqo+27bW$$^AV7b^2X)&(v;9y^#!#?|aU z*^IMXV3L9!{h>k5@MPd@XFRQ4Ym7vn5T;G4xh<2CX^V_M)fU=WKrVybmalt*YO%S= zZwzq^N|8N8Z<9i$O4+eT_M4Jo8zf?)k9RPLaECJE~21rwseL?Wa!=^x=Qsr4#1cX6`I_r)L zB>i#(AH91rOZ|lpoXoYVYej8FH}?q-eo^lZl26I5XfEj}Gu8!NQc_hEHN|8_Djd`u zla~dV9Vx}$qzY>@M}9D*kbT;2z(tO_u@P16d;1NzKaR86%ZvhXEDHuTI%{lR@UVxN zy#G_k$dV8z%LVm0$gjkRWh#;xayK!d26h#}cm+5G=}EYt=h?fjuG^W=751 z{_|AtM&k%y71X>-PzW(1HAc2fWf8BTHZ}O!_-{*v86lfKygMFv+EkU>tfKzKKuC^6 z0FU>*9rigWHfK%zL415Ne6yj-w(3pXee`Q_3|^Awel#4mQ`#IdQ*u*EDb%3x2BMk! zjrWN)+$6sZ6N{B-Xc-~1)1yT2L|lsvlhxxNe^Lq(g^knOU*j@SqQjA=NOXDP!*}jK zVgAYAE{aDaLcmmrUx}V58MMUcg0;}?hfpV&84^rc)*yoH9jsy!AkA3ifN@1;!nr!k zv1@o?0nvqDy-@8?SOs0gv7I$4QD%#7&qpX07Y3diex*Pv7oAByz-x_Lz_VzFj~(|* zvxuOmUP0!$U-Je`dDwyHi;9iFvIS2|*a%(A;Q$`7gST@- z^l^E9Qs*Q3mO~Lc{{dCBywmecPkDWhsQQ-_%RJvLi0f4?hfqBzK?Ock}*i@zw+*Cx4Pt|n)qhs zY0bRbJx^|)rd34+BO(3?%djdZFK^gBf3QcT7&b)fDA2=0upsF0y8G%%c?s#8E!>`A z8a3Yjd=~^d87xEMy^H5^7RQaCqtgRD>ReUUYbWyddt$(g3j5NbbdrNGVs?t-SEewD z%aSfW6u4Mmk6u zFG&McJ95uV898at%o#2p^%VS6PT-g@jWG6w^dr5J9GQcFye}^r zqk3cPbkCJ9uR9NL{9K{oYcHO#q&M}_7g?Ze_zl1p9u3Xa$9tvHci92^2HtESn{QTb zyUorxrq9&v!w!$Y79h12Mx@Gn!8^4BX0J@gyM(s-pMvbaC*$sDwR`l~qMS<%WG@EG zQ)Nl@yg$Xuw192GOz}2sLFiRlY|$N;Xp)y;ZRortVsr$3i8EcdCE1d7&Ofb6uZXFE zhn7`BX*}F-ryJy1^5!d+V^1^Y9M0K)R6|ZdL_tf4!kEM#q!lZRMTo4^%@!if*>RX^ zaf=PWiEx$g@0b#C!RG)sw0pbi(HK(g}@T>Fi?5w4tr14G_ zKbbvynK5Y_ETV2k2pv8LR+zm_i-ww&}DbmC^#gw?Z z*t9j#s|D?|0Woo%?R;+Am%JuN(O{rWQ%T*L=;)+%A0xfkzZJP_Es2jGrM?;Hzs%>v z%6%kT60k7G!{66oH`1_Llb%yiUSgOsUWi@oa@IAHWE62R8E$_CgD;!72c|y7AAA_S z{HDRno9Jus2pq>NoB(Q2morkE8mr41Cs`{|(-~GC)N7wL+(-`8dHl#xq%I$6HvEx- zKiqH$ozysV@(LC29&eOAYO#`pwH+=!w&^$3A=5H-)W;(gGKF;M-rGSBy{>0m0_S(veVysA)CIDV$(O8aN9Bs7kWmD>TBa zopF`mC`4^>ek(rPQ4drnY#>Z}uGS8g$cmb~Y7EvTI@HsHSBt$iHIX*Q4yCBy8$_qX zk&xY%V&@~pGMX%Z1aSqJJw>eH3onvVJuU`E^nn?lQFSZaiECsgM{cgo5yO;PlSY&u zwxAc8qogoElwg2r(F@;l-Mr`4C?= z(q7=B{zoA3t#Dy?28(-@A#?~e!sopJpKpq+XOXf@!jM)UTh-wpDnPtv-h`)AG*sywcLnwk2}>prv%ANFQ36?#_X$@(T-TrF2O(+^sq)V zQ~+f7gGR*~?qbC9y7gk=?bq-mj1R&B>IiD~b|;e0)?afrgUyCEsa3D47g~Jt&pubP zFi*OMmxPT2vdY$e8q&5S{8h*Mg_mDmFM-nWEB-fx{-uQBCxp=}?~f70=;3%G$I3X+ zUxxrl7LSB#$Tt7yFE}8oVTNkgxwrzj(N)y?6dz#Laz^+y4)1 zj-k1Yk+B1vu(g1qqJW0r|NixV%Knm6wA4_(JAUB_pu+aR!B{|4Zj!<17gZYs%%OhD zL@mj;PcO=ng2Ixfu`}gPzYT2C-PeAA@-`q`XeQOYZd>TR*#=_cFXj5egb+FqVjITW; zy}@V5?T$5I%9=p@lvG!4A;3KHR$j4((p5x*-bsckj-xe(-CIqQ$Lm~~t{#U5!LUQl z{jq`(2;h9cz?w`d>)Z{B64|m8c=VXTHS3!GVhlMA#c#0~=d&6A1L;`#ic{$N)3xhC zBKP}oU0^n8T)G=0Dk|uAjgAo2Dp)%1oOsNn3TEW+a99?$%ShO7GDu_b4I8+Nm?+0_ z|3MkX2IhRUW6ePoT{hS`XRKj&S4Puhn2)saD0R`_++IucRE!jCp&cOBbFUTIcD5KO z|Hf8cGo2O-4Pcsr?SF*$vyKB?*V3{Nt0DB!#&&? z6&9YPe%Zrq93w3cvKVzp^91}(?)E!w-LI-497Mu}BP6JK(rChNdrIYJ;yI8GA?-8n zqg6kU@LN2-y>`eJs`U5K=(E4CMb%E6jF~JyJ=hEWr{^$7n%kVSvW+ zj=)Nwv2qxx{8+)_)`Ivt*BzRU*;)E>4PqoXBJKBct@A$`IRIR}{;~lCyTww;l}>RL z4%+lUX0UrZRn5u*KrE-6Xh0Rs9+>#V;`Ljh;o!H!E)g959!he*C+s@PM3tnG(D~H0 zJK__VOC^ng#qzglyAR_z{%tnH5G@72pnx|$ENX=IL$imoMgL)avy#(z zgVku{(qTXnUEFmt+aA`jUlC(9Tu!r~cPT&)p;)ZmeX}J0)^ECq4?uKvPiK`?W zP3Uq0I4Kbnd)?n#$T{~6@FNKoODZdQ!^r#i5qJsglXg_o^JS;P;Ni#zn;H) z558|5CIgffexgrbyGK(eh3)q<+#?QJ5fEz`>mG44q<==wyqrXz zUG^?P0_4c*y$ePyT*!|iy$Bc#pO!prM3kN0UM>b?k9WR5pm2qeL+sL1^wAll38J-* z2Y4pM&m>ujIcs62#s&F*?*2O7BC}^fL9`Y(kt-DE*LgTsXeQ5F!JiUBfN#Ynsh){L02riBd6(T5xU^@-qL1r&0pHkmf-R|jB?Il=g)Y`l!C>+FietDrlSzhNUU zMYu|YsOG$?L@Rp5Z^}>v)SIx~^6*_4w1>C%_viPiagzj63jN16Ce!ay7rrO%%MLQR zPOn?kKT|1l5=VqQCq|UMDZ3;4&)4m&gu(RzVtL0M*=;v?LjumT0vN)=6OHn&{;48el|F$vXpLijv9tSs@G? zoUV*Hb1i-Z2j|T}qR(O>K|`po0iCJPbj^v1Xp5w1=yDR3ncDirSKDHC`+{bfLrdtx zQkTR!%o3YY=)sz0+Ah1-35;1;OO$f4wK%1GW|uIv)UFfK3l)))q+J8Lu`dC|DImd8 z@jG=Xq+A^>d}xgX{&S&CNwk`}W4jL*&JV9cF9y}cgeZ)L#yIWwq*7`OI5z14r|3`f z3&J*W10+G=d>oG|L?&noxccU(XGu|``eE3$8sT+f3xl8$_N5pS_0zrqZ1j%7WpZJ= zuwchlSkF810F2;My_o|1F}r^VDGZ#ZIfXU2aQE@cy9SP`R}PAZXctRV?LVU;DZf*w z%4#I3=VP(9h<_u8FF;SkS7TQV2cchT2DK!J z?7$bhbO=hhmw{5PM?q)Qq){N+tWz74QRcBx65}TNvuML)M;c$otBsB;PMd2gY{)t% zG>pl5&e4Vzrv$%f*r}8=l3ZJqK46aJG%{G+kvu4mlB1@!!v$3Z#>ufsln#)L0ic%x z=ExApY?SJT#B2LaYjDrBGH1t5U#ddKY>@$^#BiD!f`H3!0~X}}nR)QjpJ zmBGdemoNL^v@7f9Gi_YO)`8>+bM^k$szEsRC@8VBFfQd1@9`GY%)GBRZeHzEVf`BD z>RmLH=r#&1XtqH6RTFBg4Is|AQUMsRi2w?Yot@1PfVbWIo)@5|0`0mB;6{De)b-&= zt{Pe1pb_=Ksz+KyHx?PdO3?N@DfcYA-6v*RAv*eKL8Mm0&gnKt&fJ)==6lsYniaZvW#!`y`-#sxV9mP76(qOkBif)h^@x?h z>Aol@Pl;j^o&Ow(fMAyDnOyD=sQg{TGSrG!8l|el*g9Kx&*MbJjF%lPwf?x%vc44( z^y=RmfSKqyIX;p!V=)?>-g3Xs%^8jYE#ei&A(L*hIDKcT{t&z>MT)2Bw#-qRkk}-F zEBun!n{oq*^J^!1CNi11EAKir^XI2vv-yhgLN$dKAckof-ZS#D2Dl~fHv5m8@datZ zmcmPmnJj@rQtpti@LIV<%IHcAnr~39WpT2i zbdBi81juzo@6@$p)iP4FfEg{pyWmdBGg>*b^<%9DwqGbzzb&flUNg0z8YcwNp+4@w z9rSm$cOTg{1WrKfKrhyJZ1k-#&_bUtlkTXCjeJH0ya-Cd#<{9taV z8m(j-p#++a*Y9+%fO7k2?#iy%(d0*h_=KIQzz*)OLP<>u!yiiYje(89j)6=p+=V8H zbG=CcsRS|Gkmt;9&zT|*n!Sq(?oFW8+5?PUfETouKZzmnFyOjtGO}8ZLTaeD zcN`_Uxr}ECsQ1TwAu%8y8wC3Id4PzP`vBbj)}LU#KdT`@RC?i`hhcw}{i^Nx+Z_44 zG5G`J?s&*c8A#Bf-M<}ttdZI^UegOjpHDtW~Zx;bKU z{R8`kfO$uk^pd`+_hUZ(+1iyzDS8X3fzdu)=>K8t9h)ur98y(%T zZQD-Awr$(CZQDu59sP4^rfSalP0iGKa=n4;!TPSf_WJCuFy&z3CjB1~1Tuq`wP|tM z)62V4i5_9|fmMa&tpNztY`sa%$cES_KvYxiYTT3~K9#9xpA} zbFs1Oz%*AyNHW*Ds>P2uu1yzj9QIpx!pwq&B}J%lrHIb+rmMdBnw^aea5uxAm?N)) zynfG2ipXLQ-i77nZRj^6P?#;$=J$bo|^M7@BgRfRbe`x!#E1Rq&M z*}ptU&+vcUDdnXk_*DPVh10%KtAh6kf64NK67SZHy#cJe5eyJ^^5avVy#AWpbEIaT zpYF6i!E&U%cjUEl`ODs)+nHfMT5El#uN!|PnPmv3w-4Un9WQ%tNO9|PIiPepvao4C zQ-9?h{XB8s`hy;6PxLya=l;WXGxAHKA#9@&w%fZ^`~E=A8kU$A*swf>`8m91S8C#l z7$4rzBwRjJg{!DYj;CzFoF#SZ5ash8_(&ptFCFdr_5L5xFVkVSM>Glu$QTs}2+jX@ z^!wjlOaEnk|8KLa4x^{MwDdV`WU_sD^aF-LK+R~y2vh`|7}9u*%wK>YEKc^oh)EKX zjM?6#-(<^GOLL_aO)F4}Rt3g>po-2}SEH$@`KhMq<6>udd3k4t?(<~x*<{QJ^hc`e z4EIU4@Q!EU(li98DM$8d<$Fi z3uJx(E`v5@w`sTcbJ-VFv^F}&ZX%Gikb7$K^*u{+1eU;~bXD(XNEY8Z1<6inKg}w+ ztfr`ljo|wC9bl~LDH?|k!o~J42lDIw>})8UJU6q5G^1K!tBKg~8%6PpU}lRf)uVSV zdL)lgsO%WfkYmCG8O#qJTeU=U4`HZ#{%ct-z zuMOQt{6U>BjqT$_=(0^wNzUq*^0kk1jse9$j}CSXeBUufU6f=}0dZ57?>Es}dLC*r&?rJpxHJlNqlKVsf3M5UDR3L*9GaR=2&|kXqSadu`(j!3|ceTRl;PM>roY-nFXj0tOXr}oZ^B_?XI3QzZzy{5h9SRP#?fB!!nsD_Nc=- zzswcqB{3XHIKnp)_r6Y3FO`svndw*vr2heCz~h91suvCjKR8F+a;2u=pZ38b&w@E)LmMdoBS@0(>Qb1lTAXbpGJWnZV9a$Uj2BomqNjB z)^|8IeGBlh4H^=GkXz1c$ZhxNDs0thn>x8cyDkl>fUzm!dJf(H$I1$mPT^bSf0jzv z=FsJjeXD&fWA%KVsKU^eds{mtFcQ_OJ|G)(7dXRY2WP|TO;xS}(Pd3#VcfrEz3Z~s z2QRa4uF+p;LAYhS`z91GFHMiJX#?guQc~eVKI#d-NltAfy=Wmz_#{Bm353+;7+ugD zh-0#yT$$FR$qOtPO5Z4RffZl0%ANLWs%Ra2L&rhtZ%l+{_4MZRWf>P#$FiSiAxv z4Br{tvZiKhO5T%X9Yd)(KDfGaSMC3N3*JFP1%aDj^;+hivj^7jBkagPh-4NE?%pQB z_KAx!`IBV&A?6kip8urOC{u42$;bR@W=Ut~xG3(AtEaaf)CE(w9#UNV5&oVkqz`ST z`!Cb19gOHk=n8pX!`&XmqhzSAxl+k@v>fvnTz#vM5Nc|y7v9faeFTw-fq31GNe1rL z-)PksYrZVE?w!eQH`O-ChdDpr)nnTUI$8r{A21+4!+Y$*oQI=`6Mvel?$s;y|3&_^ z*~KT4yOB|&IsIb2bv8gZrg9BL7;{o3Fe(PeMq-&*!fHHc<<`#wZkgNwz5~9@{c>wE zE{KzHkF&w!zM+$-VL(664%e7+;jcXR!M*b;JhimP$G{6w=grj7o#y8$`lecu!OY`cxrkAS_T4 zLdbq9+^kuw;f2G8WJ*m&v4%5W^>Oap*{erjIY`)=3w;3wa;o4JXOw_A{;Wx>1Vtlv zPEr`U(goyredjQ+XZ`P<8k=9OuM1M4D9g{Wl<0 zP2ll8(TfJKE$Qjt8i6m~{s^}6Cz^9-0M?)Sk_nV;Bscp*Dl1Z`>1~dbyvu)8;y5V( z=7sXsXR}!}>O-FqE+kCkNU9mp;-|2hjo{Fzr|*|IF8v0>-Hh(S&Ej$&D^XzHa3x7X z;gRgpc=1WjS2Y+`pk7Hor$R1J;_c6 zpg^v_jeL+Yp0XItzR?!O+r5+7m#7Ro6mLqRAOD28whynua@rW&Y~JV$O*ti_Nd<^6 zw@OJ?N>Di0clftCnO><`w&3pMFLr{ckl;+6kgmm%N_L@wCzcTL)`XLgzfk1VCYAy( z2$MiqHi+&;GHS=kA)_2yu^`tjR!|Y@kiNT>f#^Q=SWPNJu^Q_RrTU~j1BE$3E`4#H zT)lAC;Zz{DgDrHga-Sf*RVYOSOlGo-O)xRBw%G%1!aT=5m!m(K?1-{iDfOhNR{1IE z?Z&Q~IXgmvS0%)?CECIAk57NIH^E-rn52INYMpi^u~Us(S5xWc419h0>mtn=k~C7$ zHQlaf1MSUobO!|%Tl)b--vg3SN$A72=s3myZ?yyqPMNAMMv3Vs)#6QHMg$&_%=t*F zVC}zbg_9WD4>5vt59%aOiG5?WOcQNE$MtwuAl;)tBhf<3D5I60SU~LxX*bKn5c}(~ zSs3YZO@yz{aseg%FITN3v-1r*=K*=R^c}?EigPRy?(E_~z#Y2hIf1`)03@&UOd39y z%#Y54pJeU?QY8_m5P>?0sO1SQvCV27xjIaz#vZ{6-7dC9=9BNr@^pzgm=A!i=?l@0 z^nzs%!^6*1$)|;fVnN%vjpN|?c2_i`Q$?&U0{rtei-@WBWLvS`x*@L6Z&TGiK!4N^ z?8~Uq!M_fFc?bS@6y(odaTMwFrb-3t75U`cP5gcKS6wxV&T=xOdKl+ot9Dp_nPU?m zPM{j)zWgQ;sd!-4y2LTu9|JVx8@`j^0cDFku@@SD6Zw=?ytn#mt{Rn4WgkvzN>yBD z&t+0-U!1>jgj(alj{spo6C@rby37IB@X8ssk{)Q_gNGc~Ev=hv8F7|%FQBR)DPaLh2RlMbVF*l;5T2wIgap2Y(IU!j=QaIbtuyBZPu ztw2A0E;`|3Ro}bo2z1og{;_hMJS1|q*@rtD|x89p9x4$Z5^7Y#|!nBq@ z##rWu=g>g&%zn>M>V&T}10tL~QSvT{az5IGLWqWP?hl|^s4+*xj+a`XOJ*2?&qfHD zRzQP2Zf7gj)TzliDV5K-a4T90CI5Cck$}r6hIVjUrufnC<3>hs8NSY#crn}ob1180 zlAjLk&UQ?}9QOU02*|~vNuVK=OnJTNt?_L&ih|^q%KlcyGHiy|4?NLLTebY>brBQz zBdj%gt)U}PU9C3!)yR{Bs2+D+D4sZ-RfEV-H#GLcecGfNIGrVfM=oow`XL+BGlnAT zUSqI$(j<&-TD@&^U@aq8NC6v^W-Vke2QHGoV6>7O+ZB?A*_yq%Q#RVP6i@1wP8y;x z9FXYubG0Y+JHV+|kHuVUnKe7ZXrpVJxb0YHx}(chn2#ZSwnz8cVO(#OI-BMmg!?sJ z_kDnRpuD}YvVy#sT>PPe3pX#j+?dX`_sUzYOboA0jy)CYImqkeqtBT)DeV~=X2p>IS+&nX zshJ_dZ886urA(g1Sd}*CtMl$-7zuguAd_=wxW(YKm%{C+BXY$SF1g&F{)?mIfm4&n z^%wV!@)dzhQW2(B;;)n+X}kt#qLd4aJXuuO&Q%BFdSVMUr)acJ=kh>-JZ9ouqE2T5 zaDSA_etilpXOQyKr%YM`k($~?m<^(g+YS;$`#nKur(6qsT76gf^9sqUy8;nEnZsk2 zban*P`qwF{f2aLx%aFge_lq?hTA_ubWEgXW$VJ~M zi%eX`m1oK5)nQ5LSF9rbQSx-YuK%NsMU*2%$_>@|fUXCF+z;I+y&%x;bAJhE%dOFA z++&x_E_T<|ZIO3fmCm;|9Yi}Y2Df|N7(|>bk`HtK}i0vcY_L4Zf%bf1d zKWqPzX>#zJ1<|({~nyf9$Q;Z-3guZwVN;nj&u3s z7RH++r%HUjizOV7q-A_VWo_Numd6(BAAiF+W_@`yVZ%9iyp6tTg!C=EBZnO&`fg6S zrj{V{9sLrm%RI|_#))}r+VDRAOERx$;b+;th6#EKVH<~k#6mYa^_ezMn+1<99VCma zNI500lrGSl_$x!ms))C%y?)uwTGAPp$|DEC4s)xBEmv{KAY5GH2my;IP7V(2Yg3hM2dY zKYcc>X_zu7Zxkd!rDY;4)gmbbuBoHA_J^+E?s1g+`O3&C(0fvjPf2%csxW`sZ6Xa0 zong;V9^eDI*F<(QMRwZhZ7-Mk+7q6J7J^arVfagg4<4{bNSV@L?Rnf8az-Fn;3CWs zR1_EiZd%fgbAv~y`LBQOTB9aTK{foaTN7>e@7Of{Xgg6o*{kE`7P%#j-?62vj5q;O z^jTBZ9lVAP*p-DvDq0II+7eaf)HH-gs$sY8e41 zqa|#3%1xKkj$f5G8HOZw>Hn{HN}?jc}wWiPbAO$^*vMr{7HhR@x%i? zEt2cWL4Q3W zz?-8HMKDq}5^N{RF;Xt1Zy?TT2(gY>ibrB$nPfYYzA9Z4PVnnx7?!hc93@2Z zS}>WsF8B`KoTcKK(^g$KH?dm;;sclZ6u;BjduU^MMeX-aZqjEk$OpEd&yEdortL(3 z5gYuoC0a-gR5fPD%r#`C(^{*MigqaLmW{D4EKZe0yfZ$fWi;cJkSbG zM7Y&ZYI)h43e+ts<+K5u&B zJ#+80seXOl?AU_1!RCO^77`{?50!kQqe01x%7?1CF%UUQj?xmnA*{K^yi=zrA9?JzZ0cJo5l+(*D;(=fz$N>{`SH*sguS*QT@5fumBbfmVPPZlc8dq*ipM;#GRNBhsIbCY6M$q9I zp3Ry$%Se-GMXwA6`})q+YGe;WKDG>b*iW%0sxeZ!$T}#ZB?>VYedZZiQEO^k}>hGoILx%Q-WFpyp<#Etv^XtTkAgVG-XD zBRFL$X&01UP@OA6bGkS+KkbCd)jC(RnlHZN;~GWs=0v%n3srui#f@;wpc{MiSbL8< zkPQCpP=nolYt+OK2#t!y-wlB$p5NE_Uf2Vm#a?v0<;pEn&9IP_IgXIPL^ZJxt`a{R%P>(een6jgGS-={1Hv;s{JSv;K@=_t zRauYl?J`k+uGx!l3;1O}t`nV&_O%}Bb+Nb^`GNk{-o8@HzlW+|eJ}yzE6lfuZ8kdd z1WEbq>iFPAR7l!i$NE!SX!7<0l{h7>h>%Zb_ns&x1>V=QmRB!mnIZf8^pXC{r+=JC z1C9Ayjb6pvJ0+Q0DB;^S)Gccj=9H^?=+7@!wXj;`ZhNe(C=%Edq-*on89+ESuh{Cq z8Y>4iynHkY|pKU`y#%aLg@0 zhdgZG4c}HCmT9@;g{^x)2IR6q`sIhM{>tfpYUz zZ{Y$(l}j_9@6+~}2TiL#vub=X4zb}}!RdU$BXmgKL%zrm+^V5aKaB_BZ+dDCx2*m+ z-|7r-Gp#MJbGpLwYlxMSVSeIjhui?Rd)EEFpnVTPW>8}dH$~zFieLDH5;*$0v4$x- z|ERiQ{XQ@z^9uKiMlELsHw^$ug?(Z!2TMjnl1Hg=f_RJ9I)Hs5ZR57d!G7tp~Cw}Dz`rBj;v4O^~)%eM`YP}imOPmu66fJUf`-C;RS@K)=gI2>q&)kv+>Eb4h<6m*cE@EZq?6dMrk|Fqd5JrX|y)o`a zxV$m2AG9kMT;VEi8SkE$`f*fB@8_x``H^|~d2fbt%HR|iifYkuzVewm|MCCNrV+-0 z1jgAn%LwXQ#{YksiT_VLWov6?tZ(z*l{`ts@>|KVeB`XmtzTo=m^$=J2xgTC%N00-Dvu-yam<00XU;!8(5L zPcH*ZlT`f>#GksU`g(5D)%!0}5iF+9TaMfJI@!;hH`m->uSXdme17aKRw%7i%Cb|5 z;bP^5p8s;SGnl`(H7(2Xfz)YoRGL$(E<=l!SYPVG9lhPITl;XJdjl#{f1_b>{+y&d zaV%%nC{smqm}t9BD)ue#Hdrioj^%U5VxpSiH%gm;Tc0LE6VljeZ5jheTmu0GRQgVfUihz??&VthQK$7y__WOLgL$kO6b9Kb7$>oAwL~qp%qj;Dm zGR7*8wFhx~f$DDzQ<*?4+lUi~u+E$6{FP#*zIQ)|*u>QsFwhl8MEe^1Y{A*&-l;q1 zKJ7eFY^XaQcuIfmn_}v*U13no(=t6SWvp^6m$7nLxJ@KoJe@vAj*EO(aa1Kl4?!6qKTY&sKQpY9NW_hvTFgzcAx5hwZ-?HXD`~7wh}MGYy3f z0HZOqD@bN_+DG@uVVy)eoF`PsHaYtLO%t#@tw2%;T3}2aS3drphZCfyjAIs&_4()_g3Qe_^O$8-aHe&Th0X zs|IWWiRIB_s2!4djG+wx%$+A^x#V?ndjysZ9@i&RtI6wH|21UH$15^$Hw zU;^etj%MOJa~`>McjNrpP6);aH#){&eLQ>>d(5C6?5EQ!B4OnA{v3^%3>FprIF2|U zIGWC0oAlQ_*~xB@Gk=fd56`KxWTa2-QFMDT)?zp7uiPpU9cYW_rD-UNrc|uk!Zq3{ z?torVd{i#h9!pONRaZN)FJyFcBg#X|s&AM!l-|<&8!rfVlxq#=!A*+P7nWUqHx_Pj zAu1`l`<^d+uN)=O{b!WfpKdl=zu05&rQV?<7Dv?<)E8>YJ&c2EaPOMl98~??B=?Qq zgL90Lp3V{)Y4W!A!*t)|YVnH13 z$Cncu?Fk+bQoHqSi;$0&C4(_AI;?!SLu8-+b`D?I zr*2X6WFl@+FAZCZo8fd)-vCRA&Y2bzl^ex%km`}-QlL9-M(g{4fMfk z^{ZmnTV{eF%p6a%yp> zW+ykvPraUI65rCEK!^4lc`)&XEFxS7Og98i3cC-I5z6Q|HI8-UC4($@ zwD~yEO-#5C06Bo8L9rz4&dJD^X8u=1=UOkhj=I!MF8(f^j;kT}lOFvSr4K@(JzTUN zzAmSIVqC8*$KU=qYy z1mKIUPpU3sh0=NU9*gH^=qzp@g_c`ngHjjYa_<}a`1$sdBaFry(fYzdoX56WTrk$by37L)-Y1CqAjPVU3u007kX<}?T+B~GwK8P*lY)f zCuYon&94JY&s&0t1mheWlL+o1g71OPzCt5C@+bk|@+_RliO@X+FuT(4;R;51PFw_VMQ@<> z@`{55>ZNvRj)cRXwmIPE54v{^mhTwW9tEmMlybGVMfm7J(U!vY4%z1(I= zlMFm&OGp9?;Qpa3|K)-8ML^9^SD*R+EzCwE_%}-~=>KJ1h#ga+zjfoO11dgW0-%Y* zr$JnxdH&^g0q_JP!SO|72*fX)j^(Nw*2h;kD9F+qKQ1-R}cq4fJxz+&T;8(5%Wc9Y% z-@;mS;H?6xA&aD}4^`IF4G1;})MO(16C}lqjSSX+m%U>^=ow^D&LKX zWrmCTN->b%a3Up->by&C9qHpaldLO7wP1kmQ0@BX8MBuk{~(%?4^i8zW^m}`8h(Is z@2^=AmGCG(t{aUnRXlyj4>(G|iI^p@TL&;{F;^-U4If*@)Uc0pvo(#My2I1^4Z_cw zm8x-+=#5V#34eDhlT^k^ErQK{O^=a4eW#nL#!JRJQG^Zp&<~M25CP$d<7-Ybnsm&w zF|qKwe|o3iZsu=Md@fDsGl+mLMb~OP_KYcgDD|Exr|P?b5e$-igY!B(l!XY zK>`zQV#S~JA)Qtxf97zPJj%R92TCpK7ONjWSmA5fTgrzgXDwykjN}8H_V0H%gKQJjBE@W^QmsDx-BF;qmW0q}q3PlgZPGQ9qW|G!U=I$Sf!V zNls@koP8*3YE&}IGS#_zr>Sq^OS97;U7f$!r6vZ9G#&rO=mW#l*Xo(u4Q1xmspKXW zx?!ODmsSNvo~D{+jV14XAD&g5%foV2Ai+3oL$TU4$>PD;vqidhqJm6uwWR)>ml^nd zOZTc}ZX`yPDN=6llU-tEKoj|VCkS!^fw!jRZL-=-t2qGip7`!m4Bh)LVPf)NZO{yp ze_#fzm1Nd4C(lK7@fOZYLN}A7opn>g_KUZTQ+Uje8|f6_ENc{ z*$8}%uEa-8juPRd&%J-;&5tul{{BV@z9%^oQS-=Qz=_PnS(EHe6TU~7p3+=dcI%kH zR$|_}cn?tB0GXmZGI5kr)duVjQFbwOq85DB8o)Ph!o4WRO?)QrSp$(0C%t)R{-@3q z56WE*KCRzXND{+1K^A(yjJ3gFe9tk*wh8}Rl4f?oi}#&h;C7F-d|ODT$}9ARHT>5n zD0jTKuQR+#FXBll$=K51sjmMgcajUpz-Guu&Xn zg+)BUtGEznW!V!o5+A4i)gINwq+(#`XH&cy7q~Jep7DvuK_N)I#K3e&IqQ zk5qX*l#ZD0;7>nlrJ4Ur8ixuRSr$#>hJnbkexG$pU#yPq$nB%SR$ibdmZlvJaYV7JKT_`N?P^=h+y}IYB$9qYZE2tr$)nt80cX(N zXUfB?M<;-LzM}e9MP67v`N2HZIr1II!Vx-0dm52AJUchq1zHUC7qR~wV*CqBjqa9G zZI9rBMsIiANO)_S?J9WuRCzyqswIJx=UriXiyeN;tWjLF;oEh5&VXvMA~8$#{+qWC zIEm`uD{W2fVf^{NZnardxa!TVG%s!1?jbjYFR^@0n!0Ff5)~-QC26L|I5Eoprkq9Z z=m$Ot3B&_lAsYP;zm{bPxw(j!#2b2d`*(PJ%G@2hALc0D$i5n+{|pjNBPv9m_kMlK zj7|KpvGj^m7=b^3k5ul(NCb^U?$d3hIyp+Ay3hFf@jo@0fz^yi^IMZ=|F=WwZ%zL1 zf*hryY4>eqKbPa1;vMYLS)_tO@Dr0_*ZbVc$QW^Em`5bN_4`2uF{XELU4|K(%%!uIkeZwDDV8G2o#jWnCB0G`oL zJ74c7JzihNGZ|m*k5qooyDePpEF=*}i%xZV#o-~QKRJ@rCmyXz;;B}X@{fw^k0vus zjTa*jdO~<@scWF&{lQ#@o)FAiN%2D zL*vw9%*xkJTdXDBzSh}x90Qi`kbf@$n^L(^Xkv1f)Tg9yx4>-@y0S@(NVEOG}ZF9&7xgON!;v1!@p(l;)6Ik$6r;a~b2hQk^ zh-3(i$y9IVo4|)cK-#k}Sg+JS4T@+Qq_d9r4LjBHc_bsp7SRR1#`eZmwj+JqDMv|N z^MVeP{~~p*6SzU=s3R$-QU>57w<20i<1LiB6C-F_s!t7bUl68DIGYgW=#n=NmdRj> z-|G9v=ta7~J-9DYOtA=$OXSGGZN@-xz=SUcF)ICqJOvh+A=T5$3NvSGOVK)@Yv7|Z z$mep+jQt|{Ni(>ypZ2JwDID>);tve2il-3wx}qM6QYNTa(>S~FdEu2TygK@?GCY3K zjCxD&BLJM!$k&A}r@Gcjq*tn3n0uPtX5?PlVl^^Wj#6rJLk*a1aZz(QGMlTIWRocH zRU^Ay3a>18MGC@niEAyd)Uxieg9Ly-KBlc%0qrD=@njOw89j~2Gvj=3lb{! zZDof|Z)>CP0nM&Lv7DFf(bTuajEN~b)EUHppxcwCp;33LFk+%jtAX}Hk8!MDD#lsSBu%u0 z;U&Cvrd%{sT^BuEP5`}k3ulv(Gj>?IE2uiyL3v@zE{+O|H>_-CTt)d8-RpYCyRYPA za%gH>q^$B?u;-t@6Q#$;-seEclXCXfRHk}9IG^5H!f)G zUw(2sY1<@1;zxhM(jmE>in4bcw;OqeXOSjsJ06qMkzSbq5aus)>IH8e~+++j^~Ggq6B;pw%n+v2p02d+xO@s0BHQ|dUpRNP-7wZ9;O z$9vWgyr7iP1-)XcY|VcJ2$x=dh3Y)`>&hR*ctmrFAuMilckz;j6PDy?lTgm>T zBW|yBX08Hbw75EYL$EkiS{Id8QsOrK^{VY=IYGCv-gwW%xR~a3!z9@gDbO=m^;R(@y!{>9kxTl?u%lDvHD<6fCO`{ zAdlv?jAq(x@NZlVw z>ng4bc5=Ew*Kd+U6-qGkE2K53uVu(jizU3YAqcE~#Mh`iveCU4C)zmjg_ zZ?0<5xQJTiary#%?}o&agu=MBtMsRr*w?zegY2d$qf*yR7&N*R0^5mQ{HPNLyc8Gy zt&4!?Auhui>ZafFmFs@2XxT39@y{83?wYuvAA8=ZZA7&jh;x z*AdGmKH_8h=GqQSjAnd0Ww*N*-n>_^9AAU>&&k7g2(r)dlN~qi7jEg39GY<7yH{JX zrydIPtssr$sWX#tyQKA8itRq5yI6qDFT^s`seM9HxS<*fb-bS-5 zJn*s;R`idHum9+fBD2JiOnwWs*!P+8|4n)<{QpXiFMO3%(Y49%yB$H@V+%3z1kClB z`l2D=u0V>&{9Oom*24sQ*}9k=6J$~+Wp1=k0uU{d!ejDR|A90G&@UPkHDToft-QT& z8@~K}19#pDs=aM7K_v^vY_EA;bGn|_JZqevIKN(QJlR2b0(!(}3u9?y3Wwp^yvp_3 zRCJ4Y%(-pq6y=V>j{>d=|B?TbO46!%Mi!LU-|Bks)`rk-E(tN`=25H{LtoSnzHK&W zJLSu(C1jknJ3u4n?GU>5GgPG=p)=B+Xd=I?=gzFKzQR+4egeWa`-AKQkVOpT&6c4F zt@J20h|Lt#zK}o68}qdDj2B(7yw;|rZxg9`Qcra#O1T@G0jC10;90O0H$@fp_%e{= zNl$A^U`d;FH3a9*{)>qk(xtW4SKMheWnzfFfT@D(J3pGvV+3zIJ zC`nwYdlT%|K8hvvJ3X#%9vHu*IiU_VNV}zxo@5ELGXfr16Co8M(^MHT3gEeF%+38&vghFfo;A<>T}*8(9{7$H!wt z6^sd3K2n5CRl#NT8I zX}X_QE^&qJF%M#Q0igRFewku;5Nv7j+IY?*JYFuLyC0&wQH)VU!)s~a&xV{k9Sp{< zCW5ol6ygkdwI*>}uAZge!@Z^4gKhJb<78QqW6w-_eNaf8R^-`2OaCn@7)zW1k~3iC z<26-1tJ@Gq>(#DHDnbRCo0qe4f2#-EeigWB$3^GmxzOEz4`Hx8g+w-1PW5$Nfs|V1r}A+8kK% z4yc{L_CCSVzpw<<9CY+AGjjVbZV>rEX=ZcGcbMta?7so{dpUz$b}B{!0!3XXUGeWC zi$);GEDATnzgFOju$VpFkElue85Qk4XxZr#s}F~dN>f{GG3~CZ<(Ap_RoZFXs-%t2 zDh{)eDotC^URy@#3d#)|3fOSZiUN7w0y1@7SzzbN+N=aD*x{(Hb4DLz7tYc>Sw62L zouXSF`v7c-USrKT*-@-us`k7AmOOno({jyUp4-(k&;-7- zFT^&nQhWq&;$qoVGE+9wsZb;Oh_MU66IzT_ z=WaHIpY>rNEioPZwnE?~>BzkxIgf*bXE1w0W(R$9nb#cA#h4htvM$REC2=P>yfT<& z)uv6cul(r#nT7Ugia^PD$y|C_R`yEOR5*_lo>n!lYicgLrE?LPOE)*p?D`vRv}0AK z6vex~_2eK|2NYXVR}J+%(VBixd*L815V2;AkPrGUOnA{99m@aJw$iGrkN#aO%=9qt zMuTe;*n+F`RLUiWvAA(b`B#i}QQB=Z9yLZ)eA@rzq>&+q$ZEapw*t3RoA zT+xM5(yb;Te7*iooz@&xQ!mbT%r@q~K9Zl{B%Imk13&&@L$hkL9f+CUpIu)CaI@6uh)i{F%%642!)l-}_RzU7@?-#+i&d1?cdEzU_=G5-RBqIpOA{q{UX+ zDGFVhRl2~`EmxOWzA&Zm##D>Jbh{vk{hshWe>pZ@Xg$)f>P}$ukRcX$tv)F;+2a&J)t$hEt=DO(%c9 z(%IA*h^zN{_nRl!H;(XY*ap295B->Xx?&LJQdOJhBZas^WD$?&=5GII zGW33n+sK^zyL@Z^T+D6kVr&7vBigN)ZLGTBp!pZ-Fp^Npx_`tAWs&9Co9Q4?K%z#B zt{s}FMpZ9;U%P$>(;c-KjrhLi3;edxQ-YUnag~V+M%#dIgh}=LFCSNSW?=RwD=7x{ zB-%2#rYdh6rxB$0Iy7%h=K0SAdPaxa=O1~4NyTt~$O4&L=g(#~LY9!ZNV&YUh3>8I z%wtAdM)8;fw?)JkOCsZV27fP<7JT+c{+vv;Bic3`1~Ykwbaqz4z1e}}qzw+9I)R4a z7zf3Gc~%h8+CQ#9nx}Hc>6ahi(8}DE-DN>MZ%BReE2|ffX`Mr&+yBkSfHMU;5QjL4 zqPP#1%rcrZy(>H})W{vo-%zMuJsa91|LcfLlc&}|%eqU|Bk-cht#|P3tw!g;_w7IS z%F3}TW_Z5?X}q8RyO)88{(o3_;KhI4eEqVfAeFk2yF1MnMn;@*4NVZH_trvZEs@uhwW=@a#1*BgaEdR_0 z{D5z|rBs_YMW6gIe2p}JzDUa~%k8c)J8k>1)jZfeS!pj$6T18lk2hBiqKR~5ssolN z=G&@=^`a?<5`+DqP^-@S5e56+Y)tABr~B+?Vr5v zkqoBgf_x~cVC9ep`URSm+aM;Zm4KSyR_z(yqBXBC0kZ`l(oK{jDf~gwk+X{d=l&(D zUqE?4%u2B~(G}Xm--}C78+JM51mc;G@_aRt95{$p>62PHyuOvMhecZ?n?b}2QIOgM zI{*vlZ`hLVU~5x+WCmo5LMio0kI8&trFAuOtd;9$aHx~u%&3Fsens|HD-F0@60DQ@ z`z5g83JYG$rAdtbk#v$-X&hpNzZ!(2$~H~LNS==sXgTRBMenPxiq|w63F65WA-mzv zH-X>P#`_3ZhrW3?+FWCD9qIRpk9f)AqY=trvlC?Fj*^M}v6Q z@y46CYbBzBQ(nDZxg}c7a1Oa{{Sfs~VN3rxLmR40#mWwI_5hkE8ci0omBP|VlVxvJ zLl1ucJ%M)%rWaVonICbmVSf#-lq&`Bz!_F2MQj~NB}QAi!Cw~oY08+OpjMB8UQ%(p zBgRAUW{Q(dm70+|tzhmoFiF4$B^879c_XKPYYFbfDNIqzIWgF+cyf_sS1qyaX-BV3d zs&QfjeDhB+tl=?k5wHhw31J&_S=#aBL?HuO|2zXbM~Eo6so^!j&M5g^_FLLfyYwI4pA#$Iqd)7|))Pm*$uyUn-F4E&w zwnyzn%jG%9A~z~Nge_WOG&Nn3Hsq)gdGwYcrIK7)qSwx0f?YhyyTIYOZS#x0HpHZY z@Y6!Ve(+xPs2UFV4&^l$<=gv(T}B1|ik-L#DaB@hT!xs@qRwA(MfyfW|Gw{~(y=XG|D|J-bk2HXpw63-^iIX2;__^tr*kGQ zFfJ$NJCdHfOxZYM3BegWikAK7jp|_m*CN=bdf+$LcDpgG_MIOJqF706r@vY0lkH`9 z7#j-D&VVg`PavUgL5~7FL~(31lx-fb@*9TB3<}3SOe`RCxZ-YiSiY7Et&=-=l0FHO z+vunDLldrS=KPgfDCRQp^p74eeYs!?F^<;Xc1YPLW1`h5b)v&6e~Bu4c_byJ4Vc3e ziu0uu$t8f&RG~b`8iQ{TXla#XnMKhFgDtoSRz5g4m?6^RzY(MLe;y3lL*aiaamf0D z?Vn`_HVMN2aIm4a_jQ!MU?H;+&|G%10GQhDN!`!%x4^xF-hTc}W3@%+<9a`T!98v% z?4+ZunV2}C9)BaN6U)#PwlK6Vum#)RNRtEW48osnH!LpeNJ;>eI zeXDwRCYV5*QF5(=>Jms>vh1!%V!M8NW02K`5?l>&R%4_^O5=>G+9R#B%Izb0w&3sV z|B)GFLgxf;;z#5O@w<2}Ut;i>7Ux~75@lKNWtTgjH ze}Vs>prf53CffCHSe?=L4RX%^#fr0twUL3cxwWyQlfJdxe;+Gn0o5I`MUg&-n;Yjn z?r6Egtx>pB3US5-abgod>NC}HN#wq3uwW)l8k!dp8pf_yS^}-0so0KWo&SfjcWllq zYPUwa|M73*UVd25`WrdH$|MT;m?+2uVjxZ>&z=nuBZ^~k> z`n;UP+)=3w7f374BFvcb`~Nm-bTJC zs}DGp%*f2lQ>6=vqZucs0Ib4>seF+%89^wFxxv7Tye0&ZrSXb{4u9vUAcV?;>p?NT z((1{`>e4AG(vDGhq1x=DAj=^p54&ceJ#KTztFW719uSdJT7~vwPcv7Rp0&yaWH+A7 zdol53%gTo&!QP*u$uZm=+Cb}IfjpcSVz)3H6`eIzoa4H)m*<1mFt?hN1>?bX;WJ5q zcOe>EWGv)3E-C*4Fh6vPW4VVVjyXuTPgD(clvfcP!xO`}nIMSZ)DS&K6_*i0MHzWp z=s6M86G!{5G7yc|KN+>cxIPGly4@!)(=lVwS(z_WPBXcLSOH~t&@I6qQDERr!li|x zkTW8z3%xA)ABqFZUS(MjcouOguU&m~AJN z1yskBZ`;jxG7@3vm*{T-DRrz-dZ!GRcp-YsG( zwt#`+A2dBE1#h*IfoZqQ;0NguvLBU2Dk^wT zCCf6-Bz0g6CJ>du8rD@RZ6m0rg09!9GGy4$1}AV7Y4cHwj>;C?@;JXK?PymQZGFF5 zwEi*~Wu*O;V0lQMtF|0clbD!0V-$ljH_y$paBW8FdhYG)EaE2#oQh;Ow;UMfA^n>T zQ{-}MBgmYg&3$zMF+EINE1Cc5wY+BUlM`4)xM~iJ&0{=mM+~aX*SqRjJ7cBB@Q1EYDqgdXpGz2%bUpnQj0< zs}@_T3^>T?tlpLJ8RC$XU!jkrJIBAw61%95l4a`_mbiGkqCKCiqA8mKms#*6gRok^iTg(BjHE0u5NV_M)(A&~6@3+Qg2%5*b$Y~i` zoW{V`z}_0>$ABBxJq^U$+r%pRgF{2K1D3bRcBPMnWaYR-s1NDL8$zlh?=gUhOIds$ zomgll53?&q2-)HvmzVq;3;rX7&rhmT1`@oa8$+4d2gqS7*y%2as`SXQt{^89#U71e z8UeM34GLo1{1-y6`j~$kNJH?0^(Z5!Z?Z(uxWw(@e-^vnLtMwEDy6TO8)v}@CepINpw;8?|puz86vrloF=#KDQ5fm zIV|7tNH@l@($JDWJwMeM@1Z9@=T20$`$5Y55x8Hqn>`47&q7LZ>g5$r`p$)!o?FoV z+p5-+lk)d^ygkgH8h%EKHp9GLA!+G}&J8-0u365JB0vg}- z7tL@;ToQc@SMbtYf@8<=`OOjyeF~_gj!B~i!&E*ojy`eCgrC`2jAuPaR5iC*ili+A z_5umZFunI+Pc(tYH^LdpR(idY)P}+EvUyQ4A2lp>$Z;K8mf<2^ z7$o@4BT?4MdvcPL>_|DkkN$F=ed1c!@MepwOV@3pq&5T*B!XfP6d3`k9~2oGc^^E2 zLLYG_!_gTbO%(8v=lt)%py$hJ+I#2E9r|w?MYRUm{$n2|{=}KHyY6EvJ$1{;dc6he zA!l0Zd zDYi)QF}zW(oCbYxS3UJ@D5Bru0bXEe*?CMd!`pMl)Aio0*hTqdCTH3O8clLsb#%)e zz|tZ^nd6i8#qU|?vRzC==#oN5N^hPGJiwy54GBgaM)$WmI9zK%nb+_{v$)hm%o2$8 zVdywqaXdXs*>*Z8j z%j>9Ivn=0L6lY<7dMm`^S~M4z#M$SGqgUfU7mzerhjNotUm`RhNa`s5!oK- zDlklw-m8OFonKmQlvL7HtmE0lTD%*0YV*)oJEf-Zs0N~8f+Nx|QlUG8#NegX(WJMl z-LXIl$B+(|e-b`etFp`eJ=T~-{7z!V)f4jmil)#F_ddm}HJY%c?!@OPehaSCIE=HNqU{m zNrK|XdL(pCe`r&xyuqJ6*C;h+N=|j*~N(+Sb3?;gccC&o5;r~2{(*Yua(v`4A?#1denVE1SW+D*Q`ui*G|JutR&Ck> zr+&p8F>DlF)gCq(rJBY|;Q;o6?7bj$&Ag`eXBI;1=1kK7>gAw367qYTfD&rl_?83h z3h@qByM;vzT(XNU6rE|>Lyg>)M@pbp8x#u7#Dm1`tt20n3B{2eI0e~SA6zcaC1Y%S zTd8)TKT<7@XsZk@N<3|R*;*;UV-LXW6L8&6?nte9Q9yb}RH|_RtBqTpvS-)}1mOQj zGcu>d{FoX1xXdi$t8 zqW*G{|1#bov-hCFhKM`Vn9;%HxVRQ zz6_|*^P_g5^>T`s=L!9S{xBG~su{yxxYXJ=ZbuVfFzTh?g~8xQC`Zfw<3c zSkoHsOqMBGbKgPekV`>ES*JYyBh^GqZrSdBGzATBB5OLogv0@&>X3q$q&4KcA0UWm z6sb1>#PHe=@vkS(-2%VjbyS+#G?2U-w`29{M!~By@zwDL$3b~YtR__NH;dS1^*v&D z&FnGP0X-9aUxslb$4(I~FA7{FLpyoI z9n`5kOBIxjAQ&!e_DT>zH+p_r+|!lz_Y2AVJx4xipzT$St6H?2aJ8myd*plRt=Pt| z!+)s%cBErDtPgx+``5QQM@VofILW#6k2!}NC2YTNcRud?iX+2N`l)bZSOxf?TkK$5 z1E~E-(qIw}P}Dx;6SWUtRn)SRP3;do(EQ_7$(;%#%g9oqv#a;Hq-@K&dp-F-bmUF8 zo9T27hWw9sd0G6mrg3~3fR8%0IsANhvyf2m4rm3vnR^|fvGxEe7v#~L;7_k@Fe%;$ z`fO2-+`XVfF8$52_~D24h-rWWwzp$yTqK9XkDFV(J9+~+f_w~ud%G`8-`C*YyaIwe z7D5B#9|m_X*4s-vx77bq9oRg{u#=7YFE~jWC{peg0vGuO=p^rAh^?cNWkbEd(uJ5e z8)s>91pvK}|B09mBX%sO--vnkjhO$NJaBOvdn1?s0hs?su@h@%{cS#pQlU`7O%t_L z8IvM`if?sFWN+LTtym~dx%jPjUaM~HjVD!gU7JHL2ovli1Iku1Tg*XU5;~uXiY>Na z#O)GFQOYuT+s6p|f1cfpKoo^<(GKd4+<)0m1Wx{sX6NYA{@>>_&kz2{%mDHLy%1%m z#`v~Ga2Ce*&HLsn@K_!B;~b~{=fn(U|2ejx=3Fcb^wqz9kmubXd!O=a(C{#qu?Ukn zciEOxs-t<2myu^{A-aYTZe#S2MV`GJOr*1$C+L?@!WKyuLYjtGzg1dqvZ#-2?Pn)3 zYy)C{t}sW5l%AA(nlx~wJNN`_K>kCB^8(kV_EUrbLnc_2tW?g04&IL_V284aZ->qS zsmZl9gP2jbh#swskweZ}3hjrBY+PJNkEq3&sUt~GPx>YK62=&Azp4+g3iXe#ELA+G(WZYCsePX!FHz&#jmuX0mFG0j1P|>-}oDEghOB%$J`YzG{p3_Jy zp*8ed)q3`Kyqr0j>e%^O^%{A|Fjh9(EoO#BIi0KgJ}y5H+u_5d7noW}f@9tSKmj88 zz{lZmGSqu=ft>+!PrQ)xMljMB`oWYmSA#9Qc(X7gcR_L)Z^;6#s$5PymG>uJVcx^1 z1BEm~j^a-;0Be;8pWMwDw`>u02gLp3U3PQq6EdrzI5ugdJpuNOg!h|z14(wblafBa z6jbsA_Pk`C%y499Bz+{9Y8=OZX)P`{A*VTpa7WsC`i$L~#2n1XbpOVPQ0UK5##lD4 zjXr+E_cQ$VrkYs3lKF+FWyCF163Rz6{DTHc%Uy%$C!`O>jU~7Zv$P&1Pwo@zWgSe+ znAyL+Rt_rMVKdd(M{A>$a&)%U<-+jMVs-esP??XlZv_ax>A8)crsbuw-OX-ow8FZ_J68g-Px?>4c2xHG60YtRl`F2ioPfcNkrVrFb}k*=D+zp3vFstm_TM zszwo4Nvb+}Q}asr+vG7x>@?J+xKw)Eqd}_J5~H0}5IbY>&850KKEfE2w);mPaE^>w zv^TXZgtw|C$cULcIyWC_aB2@#4Dqr>7ZZ0CCMdGr%cB+5O?9+M;IzlgT&)IUy4~g- zwh6+t?f-DQAC52qfzBSbLw8kn);O1+BH{tXF~N?-iY`();0LPYaQm@1FiI(|d;VjM zjGiHPt3)0Tc7e>JKe{XW>G`d*P03W~4^lXVH$?ndY5~aVw&j5O$MXs20d1zf_$wEWacVKPB==TpNuXc}0j32X06WE64YJWL~-BVRA z(jddeTUO3Pc|&#tj9a8F0O-vfWBU~z&;9*-VYRLP!Ytjkw7yj3e8;O`(|1#h_{_xb z&g^?*jQfl-(sg#h*A4mR0FCvw5eLZlnjxAvc7d&9oUc3Jti8A1=a%2M^%Lxl!u5+G z`bb6kix+wa8ND0l^M!@5$ACGs<00~ubK~DPD4^8W8?N3}tB8lz9bygRrTk&1^jGLy zlutmCPjLo8nHnQg>PteRDG$y9-^ zSka(aO-J8e*jiE(juTPdUbnQ|qU~POs;*)_?;ri)cbN(0K7s+lwKlCc@J^+r)e?2%hh zPJ{6koA$9t!a$Ef+vJ=Y<7G4K#ln|Y64HbwU5wjg>v~P{pJFZ@y#~bs3%kxC zn1L802v}HXL8JDc2;I!smVGDDGDhbdyiw z$!KX%inWX#-NuYb?hBRh#sspDMu7wT24&WU$mBGzdr~8sInhT6@`vLzn9~JM5y`HT z_s`}u3#iue+AeMUcyVMH!_3g0Gtg0&CD55}il1L_9xZ`oB-U*xB|=-+sL=S7x0WOr z(#ZhKa?yrB0h1x?tWIv)mvOx=x3S3p2>3)S0zkWlu4xmj=>2+(ODH_pzD&(4Rb%2N zWt=z$^lCqee3TB}zTh$>z7>5`O}Lu5f}A=#zK%h95*`8@Bcy)F{xexfjHq9GfPK*B zulX=d@7Cq<#L@%w7anP$BcF^W8yb>@rN-)ILGS_LOcJ>7)LpLANrL*3*Zau2O|o$A z#dnA1fpmryW=4dOS&M5r2n>h1grx^K+$eBZWPo)}p93|uiX21_jS4?gBopeIBSh>> z)kS)wLqfoilko2W3lVv8m8gs_>oP27(*dK2 z=0{37Pten&B)RhUgUc?Z$aNMGRNuF%Xln3o z^~Z-yT)eknXQnk|t4Tm2A@vf{_m}Y7U}ARqNx_T*=lsnEF5NaW(aNS>k!n?8q(nuug0mWk6RzA7%vrEUHpH#b02nUX7f6$l$}n-N{vCZyj*#Ae8NZkg`hqaDY^>(~54O{mvQ4t1N_=EOQ#7Pff1X_QgJ zpAh(cK`uP_6YFx8wQV_~)p*fdbEmodZ0vp!KPLwX@rUhwJxmRKw?IKb<6v6E2f4Xl z)fddZgH}BEJq!_hfd~8_xqmrEODImTJU|qfO(Sfk5}@^)L#n8>8zeux*9AFx2nGpm zvk_+QOYtpe{u=GZdE{P~Wu-0E6Ie-KJb1px?-?AeoO#Uk$$(APh?8O{jOr+Ej*qXE{$vJ zbQkY3EKfmGpqY!%21zZ6%?KuC0saR9ldxeta5Xi*=5x5fK)LOr%B6%6*FBLdh`5h` z(Rv%vc4W7L;VewBILed*tL)>4twUEMp}|I{V*i92xHZJbF}#>UR^?q9w5UprOza@{ zv@g{)hJT_e( zwvg#d8DLpq^847o(5A-vFB_jI8u(=)#Y3(ajXGuG=^I%4AtD?_7nSyMjIi6I`Yp*wY~+Vk~b%g2IHtk9wm0F$5fv?GFc|!8C$g#Jey#e+GHjtRDSr8+0_ikl$wv z>)~@o3ky*mtB=oxYBS{Q1q+Y5WmmFRh}ko`ernfe<9u*_aVKr8FZ{eOxJ382^?QxZ zXRiuo)@|FK+3@X z4=2Qx=?2lC|Lw?ZasF{$X1%y`ARC_;C$&W|c`&oaRw+Guxz=ZBJG6e~_t`m(ci(>G zi$f3PX-CTkid1)YR6K)sm*uzM&GziKYvXZJJwGovdIAr=S!%Eds@8Kxlt-EIFO^>x z7dkzFW_K(2E*V@t_LU-f5bc8FvJ?0?0ZmbtQU9+|PY0KXn&J`G9Hb3nZ5{f)Y52Ur z2Bq|f{k0N&owUFWVT)35dkPGEAJ+I6a87!31IjZ{RC5@WH$+r2;fE!}jcTCnw`U`q zc@Pfvnz)u98c=`s@8?e)DNS{jcECPJb`TsVhDL+QW?)M@!pWXcb{xeTbkFeGOFcfp zJ~5nc6{zsfx3~@~5~2HDUtKvUz3eZZ+MyZfPsHw;i{vo$tuO$)NpW5tLy`{3)b341845dBq(AWIQ1?l8k{BF7g>=&<7OG}r-)b>W*YGutV z+vo{~1njYAz7lGLn_*oXhU%lv4C?yh=cvnbu@4*L_Qay6at4&MVeiG7%BI^xfgMtM z$oHE;DQf+S@gG&xsUvGmaMNjazeifMPzeS+m?lvvC7$*QB&Ga^*kst#rnST(v>J1s z%=9=_s{=S>Mw|4jm%`I@MQq`9arV;Gtl2({2es{#%sEAINH*%ISe6aO_VvvlCHYpo zTujm((O~&78<0rE8Wt630}B-^YiK<>;o@w6?o726xfPfM{2O(WmnOVc144_ZZHw^)!r`W3PqtuO% zDt00p(-k9@klJI1s10hKO3dCWKC)77gv`po)JaF>xI;+Ok=(6Ge!P##2I3Vz=3M&T z45+kcz3)VENPhG-uK1?e0((~z-tNk=BS#h`d264|esu`ssyE$6XU3eUCes;#>MQ=b zZCnp;TO{wUvFqbGH}9@$J>YvH+}4=c6?x+3SH9Urd7^i#M7X2kdy~EDqt{4q>i8Hx zz%J|NR;%l0luGfbbb3U$Rq{%9tYno?uG{h=ed>f7s`X88y7RUUyzF<$W)1*%?l8`Ww7ivCv2JtQCHyHCxpCFI#_iRpM@Q zbx&2i!1L3epJtBJ4k0tm&nN%Wc`%ga_=@oDJh=THpBVmEqtk!zjWOyM{}o&Jr8kjk zbHv_|#2I6XlbKwcc&`XeC3UqZXG&HRm??48xo9c%9d^@Q+gw$&NgzP(}IU=LIzO9H8Nwt8>k*o^QF*DMT58v;As`0 ztmjL}+c{WM(JoizJ=S6aHc5l;+}2bcy;4Ly1ssM8Uc~D(7D5$>6v7z=uz7AY#LB8i zrE=Jy(xmpScUxnpAOV|}xCvX|4&3w%b^e7OU?EymEqzwbkZ#sTu@>(mLy;wt;2~;K zG~jQ6T;-aXhm~2oh#jIHm`B7!A`t~u%8EtFJWfJmV1kBhq?l;EQP+DaeLEX1wV$~0 zI_mW4HRUY{({SXD!3rVs7;7?ND)9U1K(e&M6 zK-#tpqmkQ)i0V6S{*J)Tl;hLMYm$>}$B!N*dfL_+a~`=Q@#o8ykhU7}Bq=qH45Qn^ z+%sYK%Lmkk)Kmwkweq$+xs9otN^?uAZ>>P9F3w0^E=MRYc3k?Z8_#r?%jXqtQ_UeP zo4Ib?V=fDgxX84ssM|Ig*6(^3TN>OXVh(Q68V(7_?Z8C32@)4s)RgSsT8fQI;|>y; zca;WK0BSm^X5Ebo_(_Xh^Hc3TBl!@A{&3>{ltJz7Ngh0j2T-T$pj8rkx=p!z8X7}y zg;XZWXf4t@i0)IyGD$C#zXRzzgq)fpQpm=RoM=oKP-~0$J&93<;5QkR`WRL0c>aYe zWi-3QeFuUkcp&DTDA8(}44V!S7@y}P3YD#JT*ViGLpigu6c>q45zR^or++Rh)N4N<4ea}uECyFIg1L~9|WrqDDDE-31BU{otn4Y0X zg^F!H?%}a?P3GD_a}_uJlvns3nTTk;Y3))yHfk9gTiNQ(v@DuQ@Q9necUsvb$9Y_i z?xe2J%rYj|sT>+h?G=REA}GJntbObtt%{^qCVdP|vlW&wn^F-Ss49_UT(#T}(fzXV zr0c#UDxl6`bYeT{9fV)4>M4H{9x%(Uau)srq&Iq(E_dIM(E{j}(lk`QOu9W>hgn(z zQJIo)jEANR%nptKIxErfHHQSM)85t&*4^c1Y8<5gP0tOOlA*6vIAmiPu$`lg_s-gC(K@V1HZaKwiZg>DKD)pI`pb0ZY`h! zkH~6Ht?wkQ06i7e$iKiJrJSUy(6(&xL@gT2&}h?{vJBCxRD`s)B6c+$0BPLcsqkP z=EwW$+}+^b4QPHQW|og zHHSVA7MH|%=*e#}7gh?cBaW!7)@JM>RoE!sHoUhN$wgtNdR>Ei!EcfJ;XTTr>yYuu zx+Pv&_S}O7zHZAo-!(@H!*m-Ay;u6%}2W z_XbqF%E0+3-RVcU=ls196FtxaH1yf-RvYKW641m>EFIe4Q+$oIIkH4Ui&9rZ^jANt=A5q_SDk# zB+J_iRfjvCfS%!*rrO;k>R=wtvOklbIn^ipyb)cTA2-9!t21Mndz!cd;QYSZhDST* zDLa6ty4xuyG*6%3B(Ed}ME(PgAK)(sbsj(bN1*-%gxxAc?;Ci4eI1)^4P3mEV;aNB zHlUjjhc=#yiPZ-U$}bMuxdogTgk_8GH-&{?Sw15&_1rHOkPN@z zcuN}jv^0dF4nx9tlaoTY7__L&nBje1{!jgjel3<(<~O!xe!KeqH~JSTI~O-m)&F25 z)qlC8jQm$BY&9)Ha2g{>7MP?UG8i}qX9cW0@dO08QP#Mlb{CzVl(mjH0>GVV*p{%s zGV^%uW#TcRwZL^Fn7ujSpp;JTjgzlTpt5XdQo6Cr+2mhy$|R+#~?6SSsyK&A=vA+Ee8@4_PN zZr`OqtYAo65N;6c>oHWCr>1Oe0-H5$5vr5k*Rx3@hzj_*8ITeL9dltuEQVkM8 z_QP)Dv7x6U`eY}G6_&T6g+(>9h{&rEZg4**CdvgbQura9_T=771;7({4 z@|~^y#wQINAm2#N9@7VjRt3^H0EhFoc<)?e%}myVR{-g|%MRIvZ_H%4{qcdlK7NuF zQ2Z}m5KKwj7)E?eKEq941O)`g&!yK-#+a4RncDcPOlaCX}C^DS72A3Qy1@5mL`>$9Kq+0s-GWQ2- zNy%BUU>)n3lM#aSYMlNB*zxj*qaO$PUow%60)0`0046|@uK56w)sayx-h zOr=WmjzU0Rf+HG5aWSB}7RN%Nj6azH9V*SBv4rWoao$3ddt==3&FFbO7FSGq%{ymoZ-6C`ES!IQsd4Q zude3$m@OLR6{4rYsEdgO$iTL=AO@f|QSa2O81+cBkwK|zoSJv@!E`-SosyeCF@oa; zo9l@p+gPU0xKdGUo?Xq#O=K~!XL7%>+W8^VsZIV~XvRAx(XVyni6Jlafop&I3eLA& z&<0E60NP6m7v<)ZKe|xu2~%8TB1F5-!R=RLw4#le_?fEK0_GRRF{TcPK@GfINt#yk)mzO64Y!641(^pq((}aIujDQ+pIBEIq)x25|?atq*qB zQsp*a4I}r+(+nHEc)phVklN+@$S`wu<=uq6@X^SNQlxEkn+?tGj_8HiOl=T;(rgNC zq0TP;dIurcy=i>X|+m5;H^Gj7nC*w4}T+y^UOT@hq~JVk{x(c(gXgYbU)vy5V~Ssk+85C z{wE0Z*X?$sr!{9^T+r9>G3HJK8E(a1{-TQuT|w2x1|q@LGut4_*1ynK&9-gCq^$tO zzISf;kZzyrq&c_6S%#f1D^9#$XKdLC9kBGq?zD$Vt?~Xt11Ye>!U5Q!DaNK`2Qw&w zyMDs|eh3K++7zt@U_S5+oCI;k7J1-0X77EPq3Vt98Q~+07~M@Xg-P7f;tx5W?zMUb zW^G9MWRbo9tV0wKLblIWD0tDrNF(>+xMc+_B|ln9(@DRIR_@(cCay~DayWPIjI#~j=Q-F z5^qZl0#ItSi6x&tK#YxcAlser-l=A{XTB`|%>;iI2udQf(eHJLIv|Y<^t?Ed&5_G( z1x55PJJQTk750Cy#F7AU zHnog-BrP#jIKzHn8iD5;gNr++VD?B(erGOEzJhbys9E?&sm*1!x3N0Dd>^M_a39DW zHMpnmD6kt9?;kTwsRE?yj@uCEl;G`&mbxQx;n!07wtkb`kR6F4uT=VgXs$wr3lfVH z$Bt5JZ>kCM-cm9jZ*K0iyC(4Pnya+byDBl+OHY(YIwuqde~QKB-JYXPmJj>7UXc&? zA|K+XC8iS$}>{}btPB>o%e73O)XIi)X~rPf82 z=gXO5bPflb?@5$M1Oj!_Rs}V`u|BrgRRd>)oz+r=3S4`8mYWXMqE8UVBHv9eVD_Ff z8yL5LJ5^1})3Ak}Q?%9fvfaV*(zTP5@%8cYx$z_RzotICQi^Cdv#W_2s4k)2pX^cJBp)?sRe&6{v4Nx~Wo z=2jpk(#P)1rv|1xkBGiX0>=#po08P29?+WU9BMh+!Ng&COy3FmnV@Q=x{r97GnKL6 zbcTip$vKwHWHLyjfN_xFeAX!^-fgh^^OcP?g?eDCysxW#GHCnl3r23GxIi}`V zO3X)cU3R3oq(cRshqxYYBgg~M5pM+Z7-gUM7BSX(mDz|}#G1UD1Nn25D(Dqe{?tPg zm^<|cjQbXf#ndK4&tNWH@s{G~RN)tCUnQ7D&47Z6bf!GCi&PRlhFD+8HB-Qd^>yVB z)EcRsL@TKdKy*S6XJ; z(B`sgYMI3O$Kt;5NR0;T=v#D~i$_=#T1lk2w8%_05-aGHYctoWh1P28WgOi?3?6ZnK3T_xYMZhsy5A@q1*Um!fBV}xP>-S zy+ty`er=CREEY!k4~*6rlT}-@b19kmfbM9URTXy+reV}bZhEWsn*$$kX-ex=`U~@Q zmovJPYM%mG+DsDbv@}N>E45S0fU@tZpVKk;85%3o;vuAVQo2NytsxLDUS5U_2Q5Q% z=WAAENhyIY#w*Q7xGtOR;~17E_^ZJBb#lN5kCtD2E}*t{dK4!WCgt+!+y$u~78 zqv2?i0mcV(hcpPUiXto9SvK{yK++4?V#nK-xliV!GcsY_<%xFy))RDNwQ4D=@E{}i zq}rb$n|0)-PR*Q3x}iK+rD|lUBM{l?(%7hSFRB>^FI2rbl!$kcHu*e8qiu01r!bdM z*P7nt-eMC-re~YvJI9!MXj0oe{6L=__{z1tc=yZ8M$jHheHX${3LBkztOrD)>5VEr zG8W>y_w)3%C3NdTNb-5T)$G>?x_v^;G^u-Z3k4`kRB<7R(B|Nc>xVq>pB((Kn$j-b zrZMeHFm9`<8Gf5>(AI1x6_V=QS2!8ojT1&Yj$wdV_ekOV-$X9X>Gweh(d~M7`+WtF zn}ZqJtg3@O^Hq%XF1=vmgr^|(qeHomZ>d@|TIFyi(8AkVE24i|4aJP&Qup<7 zhXj%naCCE;WtofzFg~6z>#k29J4UoRv-+KWEQ#ugu6VC>JQX@4?&qqs)7K{8cE|^2 zuVnRmm!-V9>vSuSPFh)iHPZVB*OB8;&e#GUVc9t>Tg7JJ7Pgm zEpCp!V7~yse~N^~VK`}bHs3asga5h+Bw8ev!uOx_xq<=ZCOmpXg0rkZMTjpA`d3i; zN%rz7aJ&CR+yBw46HgO9h3!Z)PF2_?{Konz99@A1ynNPRscfJaY!iq=BM3>mG^_!k z`7cp%^N;SV7wqQVkxC2KO`P*~@9)EP0OI$Xpfc18OJE?bf-n3;b3!A+ExRoMT>59N zDa^_p_8i|()zn3`b36@KblyJPa5~vrBY&V9e%eK4V_c)ny|wm0TU(1QxPZ>@)omwF z@Luh~*X_3aRbq~xFYd~=aMwDuJX2rO=Uo|J&zL6NIk$Ag*gE|L+ApDhXfzXc_LgpX zM{mU4QyI%rI(z@9jC9BO!s%b}nr8HIzWjy`PCvrZFNaV%9$F;-&IGhcvV`X`Ol3kff?^vT^%`aY1D>X=JGfn}rg~UrpAp;nWs$ir#o<2otF7o~-Wv+do zDQf}!T~g5)W7)0#a?$H#5!xi{^9 zkCT&TR`s2|YZsA#zWR@%)nx8Je4Jj6Shplg(I!n89!6B(t&_;;*Y&{82mG<$4*iLF|wO%fhIkdPz4&F@W_*7(PcCZz?<+@;f!X2ZlY_EKi_ z_s$KfhxQ4id$}4Br{=A5kMIIyYXU`t2~5gdX|5F|ZVeL(QV8{oe49H0!aRBk>HrgV zQwSn?K;?X3RuPkXc1M(ad+yW>51KRiMRYAF!9p(xwCL~-LXRF>R#H}k{-C~t>j|OQ9C)oVGmEuqx zDavwje-xW`IFV6aRrB?yNzq(%35>+!+P_dKNPHYyuN@NWde{lzD`Szf2YpP^oG?R$ z4zF%b%bP3=GO=&ua78#vWpS#qpt7=LnWc}Yq!Cl35XR;iHXynwixi5vV2+$Z6(vKL zLD~v4QVqMx5j`op_5K9y()p@6tEep_!i6kLwmbQ@>fLJ_q|kwiBF3AM}Ea zJy~n&Zy=vW(XXw^6vHGtsu*aZCyQ+RX4LMxLTJw1E;)Tm4jJLP3o=@PErnSI2@1_d zX2#K=4mwmZkyxEaJ0{u~RLM%bnbo(&-#O;CGleE{Ii<&q>OFNG;r4Jr_bn(EmL=Gx zvt@G#VfbbdrwxrFLtrjoI|n=R@>zr!ZVtfnTli=r2kQg_*I5tCdADb}39tD@Nv*SH zJ<9s)Wwf_W$~9`e#N>}xY?0ORe-1EL^7c2R6ys)w^Uf8+jPyl!;2S2;b>n4Q(MdN+ zTFiU$X*Wm!?Ohb?4M(n{CjRgCv|TqVU7~hD@+IGl}zhyr!CSUGkt}EqNo#%6Y~91r;NsDD!O7NL>X>Uafe* z@2GNC&fa+a6EYr4lw+{yrj%`w_hXKV*Pr*ms#EOYxX^~2sG}eA*E!&MrT<=?haH;u z5o&bpz2WO49K5T#IAB?Q{17l5N=It!Ni+G~kf~kmG#$t07nqkwviI1sVMzfnL*TS_ zbE(vI+8oQX?hY_uycGixXijbO)Y54%GYL!1j-)wzD;cg6)hI5ut~!!aroio;^Z3=P zHd~_XaG%OSOOO>?$Y+0mVDyn2ql^8cG&Bbn5T1gw=mF_pT+%Ab$-^{hsolwYH^DIF zQ{HoR&K!yMulf$-L+vyJr!^X8kM1MW?*oc{8r!h4g@mR^b zq@oeAAm1~zLni{}Uw90i7l>LMa;4#2fJ{Bh7*;>4a8so2EN5YHhz9EUk8g!!VlCSt z(q#0ik{U_LNwww?)H(U%wBYXLwsC z^GgY!CsZ}ck4c}=j!-%vcVRiEIDn8A*Zc)Aq@X87Eff=J4_8cggEM6LTS+<5rCot; zSEd_&(_U<5_et4$DPfNKhXw@h{qPW_eV~NRMBR`a!fh*5`)6aI>nS=zV7tI|e3r_*+k=EV_x%(FNPIvniI@Aj!fXko-xrQk83 zM-Db4HLJD*4O?KP1A-Xo4p5{O@QU+5{LNv-W#zA{_MnQY)QQo4qRRF}h4s?bat~Ku z5mLrb9(NH10rL&ao^FknC6>1H#d(ffi`p;LG-PWpnP#I}K3xSzz0oK76&zjrw9|n$ zhj}Z?{UxLr6AwcDKN$Ax>No2#4u%XMDJCA8igm2hgC?}umOgQpop?(Nad(!=2d*4( zd(cMhdBJ5eusi9KjG=%KKRuI%`(SB@&hAO2IAGQ#O=ff-$$0 zwdED?kz1CvJl+O>mFHjBIK#Kr0mtmi5+JT>35uk=p*zr}otOjM5NfYqkCGiQ`1_dg zvTFNI$1W5_Uf?+UR>z;(y2Cn`ZXJMocNBkj7L%7dkXJ;vyHclLbqHE|8Z+m+xQMr!u-lPZX9czjk4=N^AaLj1KBes(WHzoHMIC`EZxzp%#mf=8GjkN zWm~WxeGmBxxf|7UmtUkZ^834+KI-8AVeFl{GmEx1+N!W(+qR90ZQHhOzHw5qZQHhO z+qRRF?_8a|+qu}St>3W5oO3)+?~(hFvRxee@v=Oxc}&rPKF3z^V4V1)7g^UIT|eLW z>(nY9uJf zSGKp_IQ(CO)g<1&8m?H|D*H-deI-hn787aLNrm2J?|O^1FRNKB85xEk(abmsqM(+5 zKY7rX?&GY}v`P_0T%~AKq=Kqj3=d-oR~lYjK!8v8aU$E-V_CBl0)_jJvgvdwn}MONX+Z_9SgoP#EP- zM#1TNgME(HDQU2z{k&Zz@>-7J$&)Sy*%K7NmMLlSM6jPP-snnLP>?qCr|fF5vOGT` zreFx$b<0;ly!zcFQdK$KY!?P7^?9z++0qCI^9N=Nu_gOVv$7p9IGW3@nT@nMJMKG3?Ha|moO`96>d07P|ritOZ;mzmk0 z009F19emd24mSY-6juZ?dRJQc0m;Z(U@Xy62(KJuJ0=)aY4SIu9$|xyFujPw)J_n# z3{JdBqD>lV%q7N0$jGm>6s$P@5lOt;b^ zScu>={3^ef@&97$mov-%c7lQ=k*kuMA&Ed6ITMcSXI2Cqga*afz3K*dv|8>1TYVit zRN?kMO6V;@i;{Y!&y24`>mXD_^27-UoDKhSUzOrk;d`-#Afd7R5c&fhG@`-h&cbRj{$fD(GLfqS2bhUQ&wFBOabPbw&i7xTZRX!q3?V6U&(y_5lobVq8K%d7e;7wFim08Fe9vNu!GlMs>nxB<;1h*CN;ojA8~zzHy`AE;KY z52gWDt=QTh?RQ9GeF$=KsDnOPa@SaQ>i!>`{OHV*VeJ zX0pysa{3Pc{|z0wR1ypw{NLcjU=St$Ch+AGW+8>7flLyl`xW>GM<*m1=~KgON&hri z$jg;GpBk_xXb|F&#c;sdQz)Qt%~e>0g6Y_t&|;H(_o5? zTs0@19Yw+uKY{2*qC0H$gp`jw-EOKNnY3q%=wL zSEZK&!#n1I>y4H6yNNHyAuzS9dbz+&2%-fAXCCyGhwp7-3yr zwi#jW9D%~kv^VNj#INsgsh6d$m%52C9(lQ&+cwCPpcm;ck52Q&PgleZvh zk>QUSsfsS5s6lwTA)m!da75A4KF_J(3~m+E7YlCVwX(z` z0`T+4C#8BZ6VT}{XQht5EpmjSlrsCqgT#grjqoLfaE|mh6rPmRjM@*KbLu5S1P+eV zkf;7KJFA_e0Iv;!EA!yvBkzCGP;Z!ztg3@~UI6ru8%?uWN-tcX&J=XA%lCFO)^X+` z%zC_Kd}M*#7y(C_<&|nvg@cqD*RhTFSY-&9Zvr(nz*N(VA-f ziRfoT=INIsI3VL;XJWyQ^m#UWxi#eq?6j;md6eJmI5j%i16#J=DS)ABOaArOBgMjA zeu?br80}muL}_V?r54LowNDX&@Z+A&jf~iRB^(Gr zPjche|KhKg*ILxiK~fwMj<8aj59Xyt2p>Z>O2-a0*o79w-CX8Cme8Zw8Jb&1>n-Dl7 zGD=A0oFz4@6u^B+QO*z-=EC{75nao5l(Hesd&^`^mP2+mLXntf0cw_)n=5`$y%b)InD+qG?3F(YR!ny9y(p@RBzuW_1nX0RV-+t%PE17VfChjq# zZEP6s=vc{@%#5OFJJ%TtFEtHKvpqj*1O?xVf;B;!7WTUXFr?KP^bP9ySYqD1k7wj zuBeRmM$vJZZdyX`osTP%56PgpQ(#hoM$g-@i^p--xB0??-GyDp1YuwEKl#2_Uli811tIbrB9-5|OfKY6Oj zV*ijC-kJr$+IOs<3|_x%Tvo@ztGqMvbVP1ry>NWtqqt`4z(76e#@8?3pCzTH8j_Oa z&k;)N7l>F{#Gncjxb;k?3tdjXl%x6XK0dT3h7xm~LBKJkG2F0yYz{>r)yG;5Ko&pj zJt%Zk(&uw*ZP{oz8AwaqFEtc;#MP$V@1;(qNfoQl<2KKwVv^iMl|$%<@@p`Zt%}Eurv!hAhIafxwvgY|IK#T0;i# z(BJSx-~aS=6>Au>UqQSr%Z9_#>h}X=zI&$Z!*BGsBQzk|-w>y)&hfuj7VNPJFR708 zH!vUr4LQ2cIH;zsdQ_m>Rq@JqPG%!mN)F6c!0CaAIH<5ey8AbJ<^>O&@K<8=HSacf zIz2$#Ni87qFBGFeatDy*CvX>H7mr5DAVHdUcl^%C26H*(;s$dWrN~FB=g^?!zQo2b zf_GD-%A*I+r6}f96PHy<{zm58#^u{$Oo!%E3h7$sa7MKGS;El}?d-89YD5<%nJ*yD z%~@J?bv!_dw#kXk1rTV+cIFQsn8zw&{86#(m-nw#mqHN^fmLz>qGJh+O&x6nt0dQB zIzmzY`8dOa(nKVnA$QqDjpahZx&t7&vZ91`-)ZKM!O`|CoV@;3?gKh;kEX@816>D%M~ z8*J?)KVGQEv3%QkYJYsCro9M`ndel$hs@v3cP9n8AQF!7&mcKZ{q3<|=hxkUr)u{J zxl9W1rdp#~`nl)$Rj49+wP6}^Oc^^&?Sca|J^ z>`+vz$-LfGvq#ynxP+=0aB;*iySed!m(+l0%-YLyZz ztP&qn4g0YXc%|hKgoUM+R(i2*sM3*Iezx<=_9*)tQ85E-E$2nrF}vvn+zmG1UBY9M zSl2_&++^%8tW|iS0dbIX)PfeOIUW_+FWvmu2U|f&;VgT3QS7mbix5`~HlE*H{tQ=@ zE%F;Q1FnP10vp6tXnUS}Mg*+k)7G|lMo{ZPx%{rE>2K`gu;Quf^($J44#`2>v{5-v zT;{tn+UvwrJzy9Q2_MOSPAF!M^^Sov#@>x|QZo!~}pGGjX9)8kUYWIg$=O%|XE{NyEQ2}t15ZL00K&wH#xgSsRcZcL} zT)}3h(@bR!e2GszOPOX-TILb8;BdWLZ4%>M&exzgxZ(*lY_CVug*$Y=XS7p}U5^>Q zkf7r101Z77E^RpvT#=Lvdc4^JQ}q=NOdqE7^8%qFpj{5A#ieLSA83gMRS=0R?Sp9n zbhty)Z^&8q$gHZe`f1##=;`o_B~`ifK2BJ66hu2+Bzf+S4N6 z7nxa$hW7~;{X&$PaOI0@(6~`zNzmuKs%uob%9{3^8-Z<0GB4C2-Rr2b$=c0!6**;HP9rh$N-Zydq$C)Jgz18Am>v70 zoX?CtluRc))}EL$8Ins+C#FeFBIAnA9Tyo#=5?y9@D&`*lF*ec<*W$F1GcDr2;7d+ zA0>y{`r?5T4(7GQiM(CvVRFi}cxq=Gyz<@u)NB)(lf?nQN z1JvN4a{jA@S}QUeU?+kS6*})pC%N>Q5>R zW*El;Ba~B<<_vV{zrCzRS`!oYRTQE%#aC@;dIw+fU5*@pUS)$4xGk#pqx@-!dn6hn zN!L*X4_V63=2tCs*_#<@Td+b#EKqI1UfW>~;d1m(Lz|m+@wgsS1v;~G%9rijltBUI z5|Q#m^g2-k3}^TgO8GM-B50ja(y!wz=G0=>x3t=^LJg`c1}PQ-B=t5y zX;HIfg<{LS?%H_f5jcNu^p~{BOxr1!H|9# z+4^zI^^hIsE+!ruZKbJ7l#yZ7HJ3~ry)D7(N*^JwI+T^{Ds2>RCa0l|!_j7Q08@8B zuM;MfwcWPDTbMG3m0wi`Wo9rMf`g6T!AnE9pt7D-+mO9c62JsOfpR~uRKc|_M@g#cOfWh_CZdKtwO6JFi5 zU?zG+0-&lICf52hw2gubGb1g^&L)5g<8-k13NP0(I?LI{DJ{;#OYuXdyURa+2VpSM zSc~t!;}=vwukCW#N2H`qW`lB%HR9;3bz?x-P4rOHkb&H(HfKAXQI^c&k$|D+1;jpMvA0?^{kv0947Gr9b7tp=-ou9G(ThT` zBz=y+0Uff63kiwk5SWSt+HUkg^ze%H9+bje&LYB&xgO;@fq#7mcM5VU7-~^0(2z8g zq>eIUYX#>-U`uBc$agL#LMeY)h~`PTA0+2dH~1OEgHS_rb&8F}i1Gz1lrJph02B#!H}?D3OrB~j+FR;ZKq zk1m^WZlUK&SIuF}-Kb-UKMMW3;|xlukx%P&3|>ugZ)7PKGOb+Ji~N66 z;q}38P%^)ntEl&bT%AdLNBVasU0}LfLkSK$3%*eE91r7YT?;9~TnuDRgg}$3V5u~d z#Yc%>GaOIHXd5lZt3vMDY)}Eba=8mdBipBYKGF4mU?crYa@||hUy%hXTaDoWa2W0& zI!Gf27rbTw0Ag;$V**b$yzk)|048ma!V+#ls&F*U)mp*B5ijL@>vp$*sEI^lQkpS^ z5RD5pMvYc5;juk06vAy7~F|ejxCG`sx@OOYO$-w3B z4I{%a4IZ(X5@he_#4l=%1C=;qZC! zvj&{(otiMCCrtC$K0m`}GuW4=6dTSwTk+j~1)M(B*Kp4@XgdMGoMJvfL-v+iKKG`` z|64FHL$*$Za#~Q8zrYsjg=h-_WK>^JaH6Val+zihO=bcPRg6<{yBv&ze7X5nh|=DV z+{;HmKUU-2nWmIhW*oJdV?EcKuap-?fQ5+T9rSeQD57GL95FdyQUtnfm7`Z@=ry;? zUi6)5J47K!-lw1p21TBw!UXGj92`vLsEIuk`s`oj9FeXW1{g z^ugQj>_ftrJw}z*n2qTRiZPU1?4#$5cIon@C4x!*8o)Gc&K~eX+>iHuA7ZSne|Jn4g9T+e_vPN zo@V-jVtyGu=|gaeud2c~e%>*Dh@)8EapwHs@(RLkP${;G%8Yav+Y*RIrP#zY886ga zreCyKdjPu2*v&QG=J)T^(YR%|HaGY_e8IAs+j_XsYDnD&%8+iu1NX1Q1mapD(#IhH z@HR!pq#ub#H(v)dpT-8R%&s@!WlOveGNBHA7JZA`683DZ@Aa^`}TKwkoE z;XKZ`ct2YAcA4l9~ zFpK~)7rafHH!Xk9u=;^x5||eI1AyL6s|WA2WZ{icN~t9+Lu1EtV-YM&un{AB1|K@3 zbaY2QtOkbBYtJb{#2rxw^eMXO^6JGfG{B1J{@6X9@75U$f?1#MF%C><3fG#8}laz36~LaUP0x6N-T^Sy>|lLajB^ZnWox z-Ymh%^_?v%x&GxtIY12i-L=XzcnvGN6k(4DUA zig(>XR1pPUlS%zxzBFeV`oN|BSJ&XsE2{CHp6WpUi?_GUE!z18Nu_7HCL8*dU1u-@5zJ7k6|C_)VS%G_Uge zt)ugwZn^D!f>)A`iODOJN3iYg;p&BSw?&jbTP+l~O%@2-mZJUUDv^#;i$2XN)Q?!) z?)|h7S8W`LR#d8(bBn{pkDG*ptwfqzU`^56svSUO#^j6@glfC9WXoD$4OOtA1T5xb z6;*-(^ayfQfdd8G?ec|24!0HPKf6Zq>bqEqDtS7KBZys%N5M%p{f!?9%j;0jGFH)H2T?i&)oS8jILuR#}%@a4A{PUmwl3$GV*C_$S|om0bmg;t{6b{#gr ztfTR2q02!GR2D7yLj+I4T83reQ2zv@Q+hj}Xo)W#!JA1FYA}wz423!~^Z~}_5NZ5( zAxuG@;xGJhVMnn#yD2-IcM_v{tt1gwhx`&lc7hH$Stx<3e$@&?gfqs1+3_nOz|~`=c!8bi%1LWNDtFkfVq^UwkT&t zaV@mANM?q#s}r>=L&KGFyZ|hQN;vUi%2BbE?7cK64L`L;u&jW(Rqn%LLESWI2IQR~ z7}lzwolDayP|HaZmC#DY7X=o@PaSwv&mhr}P?antP;!=LXtjSQaG>-gY5z{NNEN9K z(a|oFzbLeYVhQ(tu=5wO^7A2O&I){RiLQ!)e|@Hox}bl)DB+32I!NfpT-6UFGOUfM zGlZ?9S}TIdlBUScFj~&QcxEB&jO^frYe#u@)s-t2I4T(9}AE&LJP!1 zwaPU>I1LD~bqA?IY?BKkh93RTEfZm2i|%QQR%e}eNAP{7N|~Y@uIj1LH~%}oe48UI zHr-5r(rB9F-OKaZ*7rE0`}=dB!0-6B6F}-uH0a<^q(&bv+|LHc7^^sZYf=Tx6D3(!*k7t2Dp6}WRF>^(3})EAsH~Q~ zutDnhdP(KDk5FhJfWR*y2#M)gE>)?a*(5_HrL#ejNaoO-|G0)Heq6)acverv*0MQt z@4gaeXn$1xc3(gZWOGOk#T7z;7~Iym>5ca!Dl~b1AF<+pZ zq%fF76}XlsOlVKB9vRi(;{5a(19Y$#?D%$On3hWR;SGQ^i>SZjb}R7kvfQg;3hY&gNG(-Mn7CF51b99!%aSfg2*u*?2CLCRH zrD&ZQdwFsBXr8%aV;oh?Dv2CM7oA~cJ69V9nCWS*i3(h198IbaDbo5Th}I1xi3nNf zu|Kva=)=xKkC{&Tc3aKl8TGXn;R?uiZ{(K1lpAcfO@QX4XT!;=IeXx1yPtMHWw+9S zWfN~^BjrL0YC3wsfw)D^h5XM^%v#o+zGW)ogCP^qJ=1A}zn`tx9k-^=PcbDnWErzd zth`xb*xK@(z)BuM^&nLF&b;x(8PRQl+aIm14b{~q-l)Id8s=lAZ6oUPRq7y_Q5VoV-{ zmBuFJ!2YW97EYf2zgK*|X`328Msa&;q&%dDFTv4tw~5Ff4aQqao|P^F*DB#^sy0@W zIPSWY6-gaO8S7Q%Q7d;f+g*2HXYqV1!=b-Nx=l1Xo?ez-So}!8fO7|Sw`cFDDoKT2 zqDy8kAFPH+HMlHvW@^=QgAQtCq4lsd&ee`jh(tAz){k&=Xb{}|aPCmNq1T6Q_!7ol zk(%No+L1eSG}S_mY5CYxxm9IdV}j6#{!ncP57@3OFLs80mgX>{A?OawDwXJGvlp^l z%ol5z6}QFE8Q}len#BwdPVxSQf_?X$#GK^zzAnFH96M5Ao|&D_M1y?~ zoIr-yF*gKRmX7S25yMAuYYhxcN1eSX`Mkee^~ziv`_jd+`ys>bLDO|m@1=+UvDqU( zvlDfeH>P0@(Dv~0*Vg_*!U=x@MK%XbFxHRDu-bd0FaX6c#bq@aIq^vd(;`$AH%KsV zq`yO_v>VyLI6yx~y@wt*v}0+132FMkK`WIzmZe02zK1Be@)PU9&eq|%_lgTMyGPvk zAtgMK-$ygAC%$Ge_g8WqC*8<1FRY@#(C*&rVltB~yc6fj5{c{>o(w}!)@CQK|GKV1 zfF|gF7LKHJ5U~u~we-rL!%S9<$}`i_p$pHmcP}n68dmO|9E%$O;ovNiNKX4!F}~gW zuZUC%rk6X&Pm68jf22< zv6e96s0_FdiPX#}I<;s_T8eEryIKnaE~5E|YDR%9K@iMm{#%gz-`tSd*o;4h3A&pe zV0h|>I^9p-4=oD~XY6-fwe&4c@q~TZdYt8W#j(xteC+z|^9l3oy5ACW4FpW0aYCuo z%t)fG7;gO$WB!)9J&Qu)G49mJp;2JarK5s(Zicqz-WQ|636#)P!+D$wzlMsV&1heZ zsY{{MZG!=;%`gcv>nfF{{wkBUO|aD7|4F~y=toniL>L*`1Cy_^$<3)vl{S^4Q;`OQ zC2B2L9eo~MIOJcmS*}*qb2a--luzJq*Z*jd#dn4SK3b?`0+efH!42vE!)++W-0yTH^W0TnM zE9BH$c?>&?R^w2~q=GSR#Z4S5=6^XTcj`X z{)DSlivW_*C$Gqq(;`i5BcdNutzh7>k(eh)-rpv#TtS&|*ussKO`7}kIk)Wai~?^z zA92_&Lmjj2JZd!qre~658I2@+x_Y`(t|j_?Vblt`{Z>B=XegzwT!N`ewOF(8{x@cF z3*2*k0EweQc9lmzNVm=$#?#S~#C(?WB(=n6C>gQL-fxxhddI^WNPkCEr_PgY^={w4N6K^^;y*s0%%<;M)3)*&r_w`H0#%`SV ziZ^tZ#7j1FF7-_&N~;72kD3U@@^{K#8H&-P(sMfHSwj3OlRJS@2V?p|8`!ys?f zj@Y#Ih>kBoe95|Z44V9|GxiVaSy4`faU3zCB)I)#sV=ikdS9SnbbA;8F7?L8#$8;Mj83o1*U z3Wq7Cigb0AhNY%Mr?wBmysczMXmY5zjDkRT_aKS*@obsOK7k)Z(|6_u^oOsCsVpm7tvO<-rU3 zPW902?}l5v&;oEDK(y*eBf-#i3f-pA(MoLl>~P7^fnGq>`LXarWE-FD?h1k6KDQmSM( z;Oo*Gs>Ph18jR=^w1C#Ym-~b2!Us3iiF@J|+MMq6vw%y?HI7FN^J=}_&lAbxVdcf8+Qw(T0_4lF6g6PU< z0~_HRVQ$zYO5O(bM6aK^NG;F^+MBbi`!dmXx5h0dn`hqEh2E_BY~#jyH#p`33rxk6 z#dygk<8>g861Q|!XEc9zrkWYJ)EMF>BckOtPUrt!7$B0kQOY}yO@^0t`uj$hx2gG8 zB;KtkOt!QYtTIbZio-rPY?|UqY44UaOGS&|RtQN~K`YB*x-_UOS48lugl%hU0Lp^9 zhWjfT^O=;RJz!-Z7rHe;r5TPSF|K&$Tvdr$keY5oacjyLO=f-#7 z^92d&(KP9d?&Axl_{Nnx$oD4TldSt&0O6Zo-K->(ygRc-h;1sgvg2NbA1Jl?f40RE)u6mGloP+RC-&1*wdRUWyxVN4#L|RT_wDzao>#5M8J=6svfXd5NWWCX=LAxBJ{Iier<4d+ z6T;E~nWQ0}N{vmiG{U%|I-E}MmDe}ypj3EuBGFd$7+q|73$6<3e#?3B z;dg?-JRd@}T3)`$wPzBLliujYO%*^TL(rC3(#cO1x^fgKgjsWV5@*#bW(K?U;x;h_ zcM17uS9EFaZ2ps8C2+G#{ z%$eVl)6f*)YawR z@k#<_{iIM>po{m07+PWF z=vrK1S0_sT+F-Wwu>3%Vnq-M}n>&?dU-%@H3DAp{HT?|X0%SCgtbxtRauG(=qJ08d zHAb+$@&;hy1t~CduX*#Fb)M%2kDU^A+j_qOnZx2DjW@N;7b`Qt*w_gV9OHd1f~9G->{ENfJe6K67tVct&cl{M zquQ7NrE(q(Wo5lF1!~%3aU5P^H#RfGPZYHj_Igvc^&d{d*bH->0t*IrE~pmwZCTre z^mQnG=rlfi!fb^+Ls3TH7Rn?^#vs>8tstm6s7(ML>i58zO`?%F`%W#TH< zTbbLzKhL1w#v)W<^Q_MY^Q#mrLZ`IhR5lPn#hbFrYA=6K6yqzf);bpPhePhDOckjn zl*+;F@I&FDWw6NZWu{?z0r^S?TotC2?zMI;Vrl&qJysgDO34p7zZDOZ-JPGnIrTa( zVvgXWaAcgI=4S3@hxLdYBtz>ny$MZPo@Q*E>h0>%WWZ&5!Tu5frakWijW|CuH7q9_ zf*yJV?C!z14m!@$mZBqp$?n{rJ;92oCOMO5t@m3oE`OeDfBN+9u*bOEa8dDSWJnt+aG!-?kDV6ujq)iRKT4rpFH zg=x`_g}t0BVb^WigK+o*52qo927KR zjc|F=6Q-+SbOt0BRJska`pn|{PuScB`^K5#95h`La1wgIcYHDAV9C&!;mTQ!AGrdz z_u&>i#`ymRt7^_V>aw^+tzzyjJ@BPxO2G|>nCN9Om@eGg@r_~o;utSnpj9Je*9=fM z*kVV)9U20sT1{=iPfwHS+w!T2cz+Hw(m*g6;AHH&>DQP`#Qg)g{o^a-X;o@x>C_B_ zidmWD<_Q@5Y0=(JkimY|_-1L1ivI7qnfgmtRFRQhrg8q0J@-BwbmoKVY^T2gjg zMQ5{dk?a_mQT{jHcVr408H`3<+*>BUT0dbO1<&Ffaul*uk!Eo^zEsJ+Ln1Kt{t!qf zLDaQz@}VcK?TN#j%X})XsMY=1DX2hRh2nZ~4KR^Ng0qB%*(#P6QQ^tC_3Jz<+5mu} zZfI25VD`FvwQ6p#(N(%un&N1pBDtcbDt(z?B@to@btujeB@v*^cHQ(fAp||1z(pg; z5SKiWk6?3+k!6~$z=UUJI_x?J_)F}(bpGIywo^u8VpNa%`!b5t^g+w2uwYXxq$w-c z%GY;`*6dKB(~6}I=GYv6hQZOx{&3N@ie-*MJLpIh={2uBa}7VKY)^a6+NixFF=I2! z$kO+_admT$6V>Tq{rrL0ho_jUf#&ufd2;`4dPYbCi_tUMhWb==aYkx(&<2F@iINi@ z-03>=LMqNAiFM^2^Yp~APX1U(eab)XUt{bqLM|p=`1j|>GVp7aiH?axTDMb?tk}U} z9z!*JzLfbaZ}argW%fdUye4vMGP)OG8w(3CNfSxb{4iM;8#gooOo?%lF^Cy3>`XlM zBhsc+}ha)?`jKN02A6J9~=~UA3B`jLqoOle`$;RD>E0F z8AEB_@$=Cp-v>^b^(-8hR<`GLq!yncKuz93^&bNmpZ{XB(uPgIXJ!fUg3 zdXX87zGMEr%Ila@xeOUIG*?SSCS%*q83E{1wzUQ+O&Buc$LmI>{ep_LZC(y|T zbhN|mFSuQBmF$r)Rh?_qmE$8{vc&=LgvTv^YB2(03&SZxePA3QQDaI=(PCOC+ zfGzsd*6bsjUJ8UK8;rysFvo!uQ7%*X%_Co zL$4!Nc=4V>tqXnuRs#O$Des4s?a6tauDQTKWEw%VN zC8c~$lFW6Z%A66?u@&}iA3Br6Ga{p$bquj5S67$QpeD(CS@SP>_e-;nK>?c$sjH~v znR-|}8g2IvA(|caj@r;*7~vLfI?B+(WGwoPsHh_$4p1)1Fe zep$>e)8E;AH#h8P2DsWQIy*h8LYg0HwPfyv>rAm z^?lmx)ShQ2_q28|8z9~z#Ok(j-?X0YvpdzRJ2a%1MA~fE@!!T?&Q4lWU2t3$XtXVb9BP)Ai?$~z6?!ireZBb(I+E@M9WDZS z9tVBqqG#bFy@qpQMtNPOB*+j5kCWk;PN||5g)9qcS|8-+dU#ZCJnF}7RS67~{Al+x z2#_D`p!qE@?YBA?yGY)2x;vgH7VjxUHu}*miM3u_LpL^3JKk<_weKWe@Y@3+H_$v2 z%agym^mzKP?a+LN{`iI!>kf4izNEO>XHyS+Dg^h}WzbBmy2)YQLZ}DJj%j#F(C%_Q zJ8%;+a8q8oNj2>`&IJx1az={d>89qt1Z)h1?DKF_yx%5t1Xg!F@08#9a1*)iuRGA; z8HC*Y^ugcmgFEDU3FD5)9*eo@$T)aEPG+a5z8SpN{Ls?rr^}H0 z_!TG)e5kOcv(H)bU0sCGvkJ?G%;DLN6`rvv!ClSNp2>gWIHfa$;i#Ls4IvTne7bW} zkLHd(ZuEB=jAHrJ0o0U00_~s5s-MGH!-_vBg*xAwQGSy3DzWO+7=W5sri?>p3@~9# zqto1_95t>?(=~Q@-WWc*h(P!EfIh$}fhH>iS;6Jc@1>2sBmsp0B`Xnj)!j#N@K906mYa_tx}i&umcG4j26F&f*@1W&?4 zvX9nvBi>(ynclT~})L?qTB>BFVY(iGpf>7($T~ znfDVJ*YKFn6k%5bgCqxTYLsXKTT_gJ011vq_RW#W(qT!CM>mFIm7V&zbM(5jLUHM{ zeCX#<*kUnjkskqua9MK3_K^B7@rQ6u{(rl3{KFklMV!zvTC#zPVARx8LVF&P;vfGN zwlroT$s7FHg1!B0!Ty)#>Hj&?|D$=TqUowAjQk}tPioHoAfhQoM~OxNk!*(2iz+|Y z01vagkVm5-D^qkyAuhvy+%V%I3?eMtQHR>~LRv0~=R{Mg%~SKRhej))+&YN+gCoe8(8n@G2I212yy>Y&>aNNXOTc?!s#4(i*h3lDuvF-k4_+ zRjN14LU8Smou||L+~#8uD@h0R_;St&sG9Mf3j}7nGD$ZcAQ)!QW_h$;)QwU&e~h# zG6UmxUGUZ6rz;WOAcn2tmR??iwTwi>mc)L*n5?R^61-vwvZv4@3JTdsRmFc2=1iTq z%R}SlNS?Ch?MvE}XUPhu&rd)<=XM}NOkj|uxlB>Wj54SVC*5F;y&b-xIHUK34aPhj zACyJLqEE%d)U9c`n8`rJae&c@*GbI7(~+x6b3L6_d65SaQ~QvjwMgN}ZXA9&$qx^; zJ9Pv!08AXHq!S5;DcxC_NU`PROIp+$3($EA_nD_=BrQ2xD@-bstZciZFnS zr)Nyldzj z(*u_~V5*T>5(hsa-;D!EC#9n|yhKu%zNH$Jo{0USEdtIdv$oVH1;ZY8ahmY^lin## zsGa&Mgd;8-BA7C`{+qr}lS)SwE#7mP3GxSzwWAHw_HUICvEIus$fGk<-P?##lZRyA_h$a%AfMTD0nHV$KQOJN?(!m=eQN$rMr z$*}Dml)Tt<S;9I$|2of`28xYXW+T3OXB zeta{83f&sawcFHP`To0CYlpMEZ}ry-d=<7HN8dB7vb){%_b{4W-Lfy-tlehzO1HuD zYKRLQ&0eF<_g2j|+7qu@PjGg*3$|Q|>y0Z}*otsNyM0RV75Y2B&_9FUSi?M0warxR zvI(=Q6d8i3MRlIA9CKRvA^}9%m{EoSOSXHj;%ZJpz;_F-u^WryWcDz;MH;`48n^UJ zRi6p(TyxoMJ3Iayn% zHsk(E#+S-TN7bF@uA}0Kl5k%a9dE+%1|aIjtksIu!t+sD053h(41R0({jXqz!#~3J zil342`ZE$({)aB?|BVDkCw*(X|LM0&#d*iNmn#^2%J(g`k0xm{vsbf&Y=E{IRokF zGTn8Y@!Tdqkj$ zd`p%*b#Vf5kOxfr*SVZ=HDE`++{nEX^J$22ktv#`lJ(H>7+andJ9gRwKo*{N5OuVX z_3{t!#=}KN_$NnR6gU8L$+RMlok@8D!gOW?B?u6OvCa%B_%j-8b=9t=)v5r%so!B` zqJn~}kKtK>7eTDCb7D>ye>JZX1WeY`l}J<=ebJQ9Kqh0%g2KZ-R2%7UW*u1GT8Ev>B73G(3e6I>MEmnEs-5KQxwGSQBT8)h*M<|O?5k#h_r zB+yQ|3ij=s%lBj@vS(*uIEzM6VC`WNq_}Johm0GjG14Eg#}aC%&rSv%AO0V@zA-qH z=<73_*tTukwr$(?L{Dtnn%EQLiS1-!O>8?m@Ba70?pyDE=&tIn+qb$-o$gcjo*#?{ za>`tEX!y#>C@dadH+O5$AI`D+T^0<|tv3VrvT=E(((%PbR7o)Moey1=AgEkuA>Jw- zE)94Mg(ZPMlIe|t8z_j)OmyoQi!HYqBM_i|f-^LLc2#Y@BL zv2mr?zz5~R6INl~FNPiSa=s^ySJq1~8=U66)V(Y;eura9EkMRBUk3hwdR58_rZQ}i zbg}=6MXyHVd4tMIAskLEDF+aiSbbQ3kWoS)2ENujl=u(F#s*e{JZYSoo!z+(!)Tw=0;JM3=r>wu$M5tCL*Y8nQJb9q)hWKODC?Z;t6hwik0 ziM(w(pkF2^4B6k55Bf>_)S&Em|p@HSTtDnD@ZCs}k7&D2tt)r$7aiLTNpYW%HV ze>6TZW@R$It`<*2PMW@webiz^ILNt;|MR2zI94HP7P-$?Wi*I5bL}2)u+LdD0=fW& zVrwdoL3>ok&H~Gwzqkd$LaGq45C_lSfT}}EmXk6&=X%`gg-{nQ*e{atbZhKAm^0t0 zq7E_tQvL05LdHl4E&7g^|7_zubE!VON@SZY=LKVfm-o1_q)X%+@s2JKDxw4-$aKd` znz9%nm7J%OJO7A#&?WW2C%$*4&gUOg^7LxgVCdf%X+WJrW7^H;v6N$8 z4zygzCHegNV;Q`I%tEScZD@L1_+pinVcbB*N(^JgbHw)V%DR4s_>qa!>>@`rSjfyj zg^I?BWnXd;Tv$g;_>QogV#x*ni*{Q68!w2tr^Yi zP2Ak#wBGg5S8)O(4Av}g&?8M%YSzd*=^N;1t59ssK|NCblEz?jL=jj*VQX8iX6b}; zI^Zs5a-IsDeg?KGIN45tv9zupMtsS97AftxS?Ztw1cM{8c#i#_uRnXQZ?3<4y@-Ul z1mTNh2WBorO&R&hsG8KW>+s@JPMA8INgy!Tr);!Z#TA<^QA^a^09Uk?Zt(BFs_@9x zLy8J%K7W!Pte0~vRvSxYnaA2~(H~BU=Q6@DwtW~HCzDpH47D>O1Xt;}LOO0pL4@^KgM3cELlUIq$E0WdvEcC&pZVTvRM z3}wu49QSiMoc`A9Slz|z6-5~%Jptw>v^7pt{U5BFIYW!~57sIQ(?`fNx_5`$IL*P` z>XCbmhN*6-r?aar*u>|bNtf=2Vz9#&+Ja85PHTc(JyWVBmD;SV+= z@xLSa@I|N1A98^#s@AQuj#YG$ZisN{gOGBD2%r^&+^;RKhwsh{M0FIg~S}tahtD3dbH?yZNW1cV2PIXb%|Z3FWROVQRX@YYNUX}GH^;o$*4F6QqZ^vFEcK=W_xK}gXf zla<9RdB_7=#-jXitwIIIk>!V17`;Uc!~!iB=xOgH7xpr7D=jWDWy^^TbKK9buZmYZy%s=Hb*^&=Tg$ zkfgnGpSG^i$Hx`wF5Qm4NyDl{kNI_MN1n$RPY39#z#t|3itnfEd)^Q>3%J4Qlg_YF z?~a^Bv21fGv*x~Xo@cP-NurofW^7Y}MPaIX0W;KWP*~C&nWIw-`;)3l!^tS}7#+>D zuiSN6}c&M@TFa*dk$<6bs1_|g#)x^K;L zh3z}fxmEU?df9uPg;nB@1f+ki3SB$;2$RilOS+r(d*8@3a^StQnW3w`naHryEgzGk znP;!@+EsFhSzzm}%2A{A<+X;n3X|?Kk;aG2gE9+z#3`8=-T=cU%pRvo7Y$hGILXFL zvrsn~>z|XE^;f&|9;{b?EeV*({IS?tKX02`J3>}Hj|=>YA#Go!VS(=1!z|4eZFF#I zY=zc}q4@xepy3_v$GrViHOI$k71151U!1ISPb2RdjUk)1H2FsscM^JBidp-PYILf2 zSc44(w=g%b*oW}DkC8?irlMn9$D#giyf*~|@TRsWcx6|DdB3n0i zLJ3@<4*$j;tuj`HillEcrM6y0)*?3(P_)$UvfRcadTsw2vn7SP;lAV*?t$uX%y8Y{ zH*}Zft8OGPh>XD_d_~%G!b=-E%& zBX`zN_31X-E#xTg@EUnqfPo;&2f%`qQHDcvnoQa!h!D3>Kt7D_j>CdE%X^%kHuD*A zrFEaD7o2nzxh3(2cqB&+@8#~wQVt>OEY%5Ctz9F!`+0L~tR=@hhuYm2JqtCkOj}3y zB52AmfHW^V>ZETC!+0P^D(LkpHM>K-cTWDE9+Cq+IVK>igNooUDCRfv!VxcN9@*j2 zvX4(VEb;b@|8LKNU&86f>8OtH9shAjJdWTo^DHg83-{Tpa=aLF^9yIM7!lssPSo{3 z@p2CT1i%k!H)NrHBn$Y!FrxaHF(q?x*G&`~dxtljMJCOg>U;V3jj7_Fk3^a`R(q=q zIA&>)29vvvFcdkhe?{T1FMms3%B6np^NzKqGuRtrMAtx%x#&7fOn9u*@Y|ki4;w&M zYwx8g;hfrY>kOWu4b1e<7{dhNoce;?TFQL0dKguNY5fjN`*!VaW8`(fNNpLHzA1pr z|1wzl!@064^nyWb1o2Jo8piOQydd=Q-)eJ{%u5j{7zhXk1PBPx|F1U7I9fXWr#^cm zPdY9tBLlwC_!JL*d&nk>;+{b2HW;Y<)44#E5RFD}jS=NEP;WaZS45M#3x}K?0wbp- z#|!`=gk?D%>PY0xam!-6pXKWNJifm61937=S2nhOl6)#W_Vt|g;Gaz}dk@IoYyqCT zz_5yW?E1T06rnv`x_RyT!Gv-urqKOkh)0Lk3xeCRr8Xgp?cJrCFvNYHOySI*m(1wz zl17Cze5Z@}V1&=s4H6}!Z+*vNVB6B-K*#Xpsu{w$cf?5 zhHk@%e$x|l1U`(rL=^v7a|0Mq!M+H@qchF*+Iadx=1yy6`5HKq#L+f>a<@{a1S0bfH`g>WTnW#=MXf()b(4m8 zkd!E(vf@bIshy?7>Y_xtachX@s5ke36i*4G??F0s$MoPQPCLmF^K}rvdw+qHJ{HKK zC1R<&HGq{;^D0*`&F?W7A}B;bFhs`!O|*$HlPI?c%*y zo@CIKZZ+z8f$P+yI$+Ej1ok~(Tn8?m{>tsvwS7!7HZc44LvC$3m z9`*jdY1<-0qqn0R%8t$oN%4yQx#BZn%V_?i`mg+)9)gXvjRM1qfJSjdX#lbv4KL_W zQ^3pZ)+6FrUSck2l|79S4f>Yp_#}_vD)Om-gw7?)s8@`DVU_9?E^E(UIfJ73fcNKX zt(6OX5w8^q_XlkTK0^#l;dz-8A3>|F>LH65q2(W2{bIp8`HM{7+SckQYaD{!)xoN5;Pgl?RW(p;+s-l1Lda zvbq;T8kS-6I5rkNL%%AP^eW%w{yKh%R3#;{kUZe)HS+jGo!68q0g(HITAwSp4KC-N z^XP2(hoT^`8u(P;p6Fb$0G~o*vD7ETeKmDbBH1eycvN(3kd8boM1?NJzy~|@d$XrK zL@Z(pGstJGT5^kekWf%3AG>BiP?(H|^u_&!R!=b9#Jdvh*jr+L>kmIida`7&waER( z7mc*eaODx}qL*3B|DB7IIS$1mkcvd?k3H{xFCy6QIK_-2G!$h~Um8VZIYh{$wxo8C zQoiK&CKs^2N~GXv;s44~gIv6#IfgOf*(+Mdem}JTdLS8F%qBc5-Zc^VniW`qzwX+~ zk~oC?wpUrtQjT3r#JHk_9k&Mi_G_eD$+bemwJ3a5l8olC+Hu~CU!f;n#cj?qNa)3_1#^c2R}0jZ~Zf!e;SzM2u; z*)_`yFmy};Iu27R!uObH#FQ24o_V=LZ(9DS(=uS+Fk(&l+ZFC=Xcq_)!-fd^GEZ~j z@kHDPMrzg;tucDvRF;bw6d9!k@6C9*zXBYIrPc>ZDCA0!L&1&Ukk6D@%Ck`Ex+Sn^ zK^mU?8I2P-1mQYH2bag2FW5aF_gX=@l*#RBtOib_Qps;$yeT#Eux_gIY%&#SuE-BW za~)cxa>(5bT31%AnjUxfhTo3g`S%d?9-Iu{c?eeeC$wGSjSpK@?i@B}FknnOK7f%O z+a5ZxB#24H-I}&s*lt`+I-7cvlW12DDF2C4?<|b?k3x_h6`r<_n%G0Cg zC0r3-MMf^ej$DDgOCbL7ikq1|i9l=>h3vB3v|gC5+z_IC?ir)#I2f4sOTMD48I}>@ zSH)3h)0hFZ+Qh5kGD5DXwS4>+1r#yMX{FVHzTM``s5dnx`{=EhVq@~G6kJi(0>jM3 zjFqoDd^@QuYOZV&qNe~G8~H4Q1{oaRS*KX_$wfg)jSozDTe94gm}`3K925(d&d(0T z$S^!{dNPC)PpD~l`I1eMgq@T1Z%OeBOL6%Lc!g4TAWdp0$@GBzXWkIpm`)FTTo&rU zDLfg}P<$ft9JwqPutzxo3yyA+jiw=uqn)LF_sBSc$Xl_z@P1LYv0fXOTCzxb5Sj&= z7>l!VnnBgl3Ks{an=Fdcx=GPz+OO?t`|yny4{@jI&4G1W^bsH=4p)ZbwB;zsYE0-# zpSgNHSCl(GFmW@5fS)NO#f)ReXqA>H&pk^Xj$D814Bwg6-If1aTXf5()N+P6&zO50 zDlq~$vy!QMc7-_Za4Vw0fdMbNLgd_q&hr~`;?J@x+N@U0L8Nisabmp!b`BBe-l zBsoC%Cx4BUgd;!HkSsps%p%1hOJ)JUp|L9})2EO-#}Z6V`dQ2w<8zY-rr2VeA=hh7ypBO}P&BsxgOoj|%rx+%U$()8 zgdE>nEJl9vQS%+YXVW*tke<=LJfLV57N-x%rY+@i0lWTBo&Co;ftuXy`JsGoMd`+T zvE(urXdWG1c#sxJz%2LIgjoXAWE(thuxx3n$Y`ApoHDL;NQOb&s-`c);mU3+LcIbz z>Vq6whF)lD*XV7Dnz1fXwB{(CP0*9-7VlYtJ?=)MkCw+@7&^_}f&)!rccr|@8Hev} zyUNW{54&IFv3#N`neF0X+=FJaT4j9p?8CX7nTJv8kP)MDXn8!u!3n98_2$jojxA)s z+BnapX=W5|`eOH%p&FJNJX@^49UKUk0C^dsINpc{VjjFHUc|H#@Idv=>z!}bvveF1 zBciqzOa-%nq=q5qFok^-EpU z4mBoLY}SjMRI1&}ZLGy?nd1wbgVHW9rg$`Gkn=hBQ>CZS5dhpg!-w*G^6=(WO|r8b z1k_xEk^v=MD$ebZT-WkZnsigXT#;#bFAdxI{6Z_aN|py5L9Hff9F7xd<*rGbH4VfK ze0;qA-$gb-fHaE2}w6p;-wl!!_(wQU*N%BK5o@TlvqOBEjg`UkD ztMDv>SvYieNX8vIb9dUjiR4?JmQ9zP{+do%<8>c_v-V{klqbQmzFXLz-m$kgAL}>r zxKd)AG=a+l<811P^0q8eP0is=S-XnkqR#Wu(NumcT_31?dH%R=IE zz^x{g4QbB)HcH=h$r)l+Fim)#k*{1689Ngs6Ik+6#vRAXn&0A&SG|_hB@uBJBlD7b zW&{cUYyT{G&jp$lB+WJb06&N)%OcZej_^rPyGU?|wRVZedMDj4?7Xfr?y9sac9WK+ zKI4w^i=wc966H+OcC5nAb&PHE2 z;fsBK&Y$QtP=S1_Oub-`&>O33(xjQq5uia-PO*fPmmq!9)EE|I9Oa!8o$~A*eI$q3 z?gE?|^jgtaL&Q3Mtb}VEg*WeZMwo-PFGDUcl<^PyZ2jUWm0kW}r@$K3gu0OGNv-)JjtqH4N@=`>8}7!)QL4En$4T|x zkSiN7Dvv?u_Hfz7)Mz>lpA!b|O!w6!iNH{3sj-z!A#d@g!Bgdn==hNuE@ySH47w5< z#TL%etC#rGhy9NpY=OhZ1H|3b6A;6V{&e$1PF!laz;t%w2n%fdn(K)aJZbMdqkA)l zB2BP>g>WoW6CgJWq8JO}@*f#q1?@I!G!#+P?#&4BiiV8veVlkYUbIt@eqSGsqS!CGY-9A zt~mWbM@xR0GMr#F(m_8MM*+{t(Ym&Vg-SvJsM3CyPEr-l z0Z+3SS$yuh-jueCZ&pB?iM#YHt>BYdU-W#!&^^w%Z5L0&h{MhRFaRSfp|Bd|J~lU~ zEus{)VBp6(2sY#rMmBFs)o$_^Ex2^YIV3e$RSe=ws#h_BDI+r@>-2@W`qA@~ovHM4=ljhF9-+g1vlmBkv(v%;;HK^D zXs>5E&aZl7)HAlAh~6(WM!K%iQ-dqs9%lom(dr16*Q0bWp-!(_3{hc>UWPF>`mIiB z!7Z!khpew0Mt!hzuprHnO2AvUT19c;cr%5M0i*^_K;xOUu}oE05({|z;;JxO(mcR3 zqv}DXbOhi81Z|;+TL0jE+^gNlz41R8#6>`o$7cjxLAo?Ngr_eZ#Vhp%lEKBN2bR>KYN&nxmAcev-IjULjDdWP{aZ^JG0!a~}4+5(av z%0+huzsvfzUF5dKL#Fh|ElTeX>A9SZ@(@o;c?ljJbyc~B-fr1lj3YvaW+BY)Jb@?k8ltoJuzdlERj7ef@r58%^ZZr^AwL-z))ljiVSUL zB+Wu9Zp=5V*e`Ng!BFXDTFgRvt?M&zUkGdscoiX z3%Rv>#*RpYZVP_3PW$G$_g-tjI|cI+4QW?tLSI_pyZ76T_WQd+kpJ;Xe68K$6JII^ z%0{j6=~aQR!FMk&)#7fb#sxHK)F_&{Wugh3F3p9+hUhf4AHfNA&?jmrUV zi-k8ZxrvhAV5PoB-P8Vl4E;UyDI(_r$;9-b`+|^YtBoIfORy=>epI^o6Xm_6lCmv6 z_E@qF{(2kGaiZNxZ^YD*{-8f^Z)CGrP%g5Kx2?9`WU)H-`)XQ(`QzweJiV#jMpLae z{NZWb^4<6y+4o(&IiNBC+M=?APcpiz-3k8>4!qMQr|1A{n~k^L_sxvTAb|T+Tp*Ta z1>(Kj^8Lp|7`l8hC`J`={jBoJ?E%R)PZ)TmPt-@{bX*%z~Zh*Hp+>NP^5aTfT8!v9ey(L}OzvRzX`6n2uu|K^zsbo5*Ge|K;8w4}t z*WIY+^(Om*fGlA_agR4VM??A?l;xqQP}7kvTCL1OT-I0Nnk=}Ir8n%2Iv^*wf`y;p zcle&>E+@bWq#cewWlyJ{0DksIKJZ5WV}~Db49?VmIpU3SV+!C4&S3UOz8Qcg0`sQ^ zMt~B{+^Ftaz!QP`lLMtdiKcH@`X_M=LA#Up%!0L8x^cY`fKH%-@p}isO2#j>{dPEl zC@&811rYwMzzPs6>?^Pt)?9E;?5iJFCcm-PEMC-i1p)W{kbsze3czIl`I^5^e{W>P z&plXYAs~2vC_EBqy_Glk%?6Gj!pjbJ9azE44fBm8fD6>g(i_4X4yYGAYwV5wA`h4a z+a9^`+$Dfd04u+70KOySLSUZSIY9(}lea_u&Yb1r0<39)tB|>;?26v-k_% zykqx*ynMnNLHnZsvA~T6Z_4^JSp0czps>G)Zz8b2m~W)8zo>6Uv3tQ@WC22m{un?l za3kOi)lZ@Eo7VmU(-+>|bDUn#7fFB^pA7B8H;8#uk-FU{~fFdsYs zA*2sTKp(U}7SIbkC~D8|XW!IKPyaKEztD{q_80BVDt7tf69^CqOlar^*q^}sL49Ke zFNF930aOAL8opWVpJo1_zj1^AH;)IP5A;I;U?UV5Y_guvH_UuF%ET)Fe%mhbejWL= zgK@WGmicsW4fg>K2!j3~E4tpn`C_?I3i~R64fr0o2?G0$-OKnHG;_oBGidx~vj2Pd zrv9gq$&1FW5!(me?#{ot-mkICKR=$~zd>IT;J;yB`f$F8c7FBDAs!9Vbs2;0fWIkyr9u&~uC}9E= zM2&P66vvcIG7pn*^d8z3r;7<@a~k58-A$xo8cHF0aJUMx6) zWA{t=MNV~dTuLnk1ydE3q0I&K# z+9qWX_l+tb1C(gsCaGVF#h?9#1)d4YaNvQANrf)uJ%m>?)+$Yxpxp@EKlkL?doKY^ z{JkCawN;+zPSgW6Mar;N5%bms)~*2b%9I-}7wz_xd=_VU`q1yEAC70R62{kcbuC@9{9!;NBX}bt-5}BD;0`!ES-loNl7wMB09%r< z-rF#m-_XGr+o#*q=a*>yFYo;5Zo_}>b?&<+04^{mQ*X8#XrSKD_UNw} zw{-0S{`#`VR)d$AQEbVrQtj>P#An+4=Y4(kF6%=-!Ajt8pYq$65sR* zT4M(olLsE%Ms)k{YJ# z!iDlt-U={oKXMzseOQcY7*#7{(5qvX;QDJ2Rhhu2upT>zm_2 zjyy2$pmJ7q#wsqNWZo}i{vG9|0iL>JN|<+QVINC}-WQfE`8KBfi^JZ!L*Cw>%dk^t zOnCQB&Au^r2!7=aUg-_(VrzVQV==yy`FfK%TM4@9YismJJq6h6yDgfAOon+*9w(mPq*HlkJ;z+)!25U!?VS|D$x6 z)th3!*7fMVxFOgjd~8}irpJBm2#SpLX~cTfVqZs@b~~XR2*|`fo3m`I4SBPfb{m*> z>#%I+Vl&mxChn})2$sAJU75Tk&92uNmiP{P;z)kkHNV9@Np&_Pq_`j|KS#?L6_ zth#X*T&S`ZG)#`)uOB_)-@$FjUF~pwJO1>aKH5QW-u}~_U-I(z5r6plXXdO3OM#fF z;A+h4Ch^hzrKzA2+Wq)BUv$NG##jxuA0EpmgimGZ8<&8$>_)5i9V26$R_Y5kM2D{aKC(wN3Z&?r1X0GnoGj?KgvQ zca3rPKIC6Ob@cAfvU81g?-hJ<{aM@xClQ1d^~H$!X^=d}H6Xi@v7Vtb(TksYtw$5{ z=Gd&5;~e4PcZT@xD-611InNSQV)bbkW*jUZrKRrONl zv^L2gm*b9qI(Tir6=Ts85@G&rKmXmf-V`Vr@UsuQZ}H{>bJl(L^~GV}UMuEnbtPRl zd`HLoD>A}c2q?zY`F=|e_U-&38d2P*aWQm{#4iP!62JT+fZa#6|7`y8wClwXxO&h| z(1&;PfjZmVFV(iuzP;?Y1332N_u74>&*!`KN1Q#r^#`5By4?&vcH}o2bVJH#I_SWf z4e-7iy~p4;8ouY?N80s4o)z;Z9JwDeKw3*cY~4uMaT4OW7JScozo_5Am>p5H1I^WH z3Cz~D>Pfw;%}~QQUtaq8*tL}SSh4iuymiTDWk*%x{BfzFNi|2EmZNx~s`eKT$1hzq ztE#N3OIt--c?RhoF^_W3Dlyfr<%K0e4|Usm-6USATE)ETc%@4IcL|62_V54JM=kn`M0@6hf0z&vdb;86&C0t#dT>sMx8_x3cS0DZ} zT0rH;=Lb9t^HX*NBLPK4y&{%HB@Ipl!4nn^pFSejY5IfUM*;zXjtnWDngdRP8lWP$ixczM?M07V57&M#ZV|kR7#WuRy z>+|FTf*rr(A%ZFT3|J{1MtCeL2D}+lmr|fgU|6Ia+-h$^dWKyL50niQOEa<Lyb? z!%?7gSA}6_>JUi(NWlf0ky6$Tb#U0TRs$<~4zX?SRcFPgVI`<&9`%N>%B5W#d%TnT zJ%h^2LyIPw`d4Jy&vXl5H zR8vt7Ql}VFv{{IXPhg-%>++;9E8~85B7A;{aDHW-cKwyB3cjKQ*8`xJ@g#IkBfc~p z8bhPyM>=4_Y+jP+A@pl0;`fQWQk`~!XL@>M**-}vJ%&T1^#KO<&%{N-SgEQGlQ-(= zG9p53h){z>B{q*LsQ_h=aOf*ahVcPX_^Cc=Bt~&q_%a^Ojo2_WnfX#M3Fr>WS|`0V zLPbudrG1z1rXRkUqBVgoKKdnl`Esm6TCydqA)|ie7;-)}k}-|AC0L`dxcMlU3TRO~1WU!BXN#=j|kt&TCPJ=Y3tWE4^wAa~Z8=Zf{s2 zZ9Aw>6q28h^zTDW7@L#qnheQKN40@EB&ErBv(zVl#w89 zy&bQ07)OL$N8xS6Od*>Oc7|+rcDG?hb*zMIx7C<0#jG%L3rR{X?}rWJU!+oK0Ju?% zQuDUzK9OT6l&-Iv-j=Q?ug*&@Zw?xo*(9&3X6xLGR!9DlJ-@FV*)I{?kvtC(puxbi zElEPWi(inG*oL(iKJhnv2&O=Bv`?AGMi{rmNj@!Qkg=IZ_Hjrevz5kpYkRtx$Wp zC2UXc0c}d33^xdS)w9S)|&W3ma+04M4;ly1gmlQWp`Ngil4W-K)VL$FP3?jU`vRt!!j9 z;`BtzP9Pem&rnH|GKHxX2`(7pmycX*)`+33fEXSygsk;Q{uu|9sBq99)= z3W*ni49w1A^>A;|pfF7$p52`FIU3S@FUS8a?p(EkS+Z41%S)3Mn>MPe2OK|j|Dbb* zHEEbn&?XnXrf2YI>@VBeDQ=#-@Dwj8`ef?uD}RPD%UC`QC+tg`+_wJ^E$|w$6x82o z3o6L!(wJ{kXLY;zO6HSQZ29R7qXBf zH)B~(Hm75={!2ATA`trJkjv;!A4HQ8Jyoc~f0;UbT$&lZ@(<*jjnM-3J+3d+J&g_g zb~pmt{Uk4>pabfbiiogpqe{|RPu5iF#;lCk;o%Kmg(8|sq|=@x|3OKX!mR(j&aGgg zLrZV#Ihj>PL3C{`hZtk4&}g2CdDbvu%w1f_RXgD9svUz_zq;QTR`m84e&O8Y?^HBy zs1apC8%WP~cy=p5SQZ}bXw2XbI_)WF(V=bv#INc~2DKB?j3AGi{F%z#RO4dSI+!ZF z9wXc?mRo3|-L-=dGOj<_^hgVwu|BSq2fAoIiM9yrnEW`tp8J1UjYJ!zF%%A(&a}HQ z>*JDF^hO+Qv%jKCjEx}r)E|a`Uw0m(!Z47-biaML!yfKc3R*>MtbXiOI%E1b;G^Wl zl%|p&$}C+D#qB>Xbg_jZ|&n89%8C`%3c1@il}C5RfGTLo}mMi{Zsf?TnK(a8(bOB z9LzeVog9sK+UEFz2k+f}r&`=f>FzSC9vkj_Nv``XCaOoHcG*fDu>k}lo2qi(Kxzeb z&3F?~l2ZW%=;mP zm;3KN{g21rx$a%kkz>nn1jzdAcAJ&bjKgod3x7>lZ+J)n7NmeO_7L>H`11uAg55F? z6?l(Q*Af~nES`oD_%L-#gp|Vwf4rNKoCe54;b0lJey1OAO;Q2ET48Lvs8S`Fecoceh{L1^3-yx{ zTh_@#TJVHgUDOMhqP_+-zc?8b{PK4q)hltZ2${u@Mxz(*yBw3eMrxKZ3?~fB$}>VGO#D@EaOmGk?ZT;B@+@7zLF9@75bKHGn=DLxZp2 zXI-$5=j}%M6MG=9J5SjYFRWIj7GXp%ug6(S-YIfQr~CSVd!`IVUY)tF{epc@bS~46 zY6?lYKa7+TXV59s@&H{yM@lhzBBT`2$QCG-mkIUY?1M;v-zA5O;IQFmTtmExxID_Ve5`cf=vq>iMbuDK!rVBfF zG%MrB2DLauGo>->#>uF8xdsjb8WW|}>(l!v51r^^y%_xX<}*^h_ z5n_C$KWeX1f({hF1G(SGB1-Fye+5k4=1%*QjlQNHx@9_<2I}8+OmS6Hga|RbdQ4p( zau=le6W(_%UBi8KAKqu}OuKvu-$72b|6+pq5*x3#>WzC%py&&J^{hL_7v_G8ao+0 zk@7d=OGR=!t5C5g<_Ft&$rO_7XMr4{*;nEl(?DNhciby*bElr%Xz&9if3%(=fh{O` z<5Thu;$F%PMS5(QzKUGOhac;*b3au!I_)|$nl!DfhTnviu&Sl`&Oq%}#v-e3>0zF1 za#&II#JH%Q#9YpV2ksMIByA8D^9@hbie__YFby!L0ZXTYZ4A9t)({;Y446}ct=7Rd zhwhOlfpWB@*io))NY&HEv8N88w?LqS1T4&|t~9AyQ)}r?@M;3ECxqoX($b0m$xbxq zIyjcpkm;^?bpNt#JwGPVdSifu}r zu8Cz&Wl49XqANkL$St!sv4|~_Z%FOZ#<8PDWBBW+If666k=~TH_>{?Oq%}crfk&rQ z#nGv1LbIXsny;c)R(4_h_ji$eV`^SaE^}&Cx+@Z$5}s^p*~LN{VLh&FT-oO2gdKnt zsOstFFpE7wFZYKGy_3R2v5$fP$b0O@9dTv-K&3Ysyt-^ib;Ii-LzZ|iVKalZv5cC> z?e6-Nm)m@Oybl%}zlWf-V8H~31;#Ddk4XCr&Szh~v70vs*CtG|iBQ&nF_Mx8u~7)y z29$4dGl8NH%bOE#!?+4YzX(Y)Umb8P_S;3?gqj^R^aj6*@Y(}?L4PvAycnHv27wwB z!3Re=Q1^zYj}XHSm%6p&j@8?9;fGe*b>R-F?@xK*#2@Y(RC__dpPc}p>3r^<)`)sj zR;ONYdvjLfDq3zq@66hU9yhJ@-|KAbh1W#vLFp&~V|17X<#g(kKOwtwJ#YgvA9+5; zj>3pGI)O^|kT=l|VBNOW!K?N$yB>P@+YPm$1%EsTXB{GT6*ii0z;Cw`g$gZ}y#~C?-2{JnyvzkgUu@hYeG0t9e8T^k-8FmX;XY^>7)3<-SWbcq zayv}AgbMOPAEb&`Z93Rhlj8XRvcz4`zf3iP%>icC5f&AC3- zZckKtKRvBzcykZN(~A~)5DaEO{=8Oli=BA3CB~JEc7XNFw3>4j&vqbu&Ip)4n^7H` zbWi!jUNmPO_~U3CpPL6obs*}iM;PeqtWEzNzN*u25@@Wb7CP#H z9ZxrATJ`SU)KK)KHQa+Wc3K5%dR!Y_4hb-TX_{;-uw!`gav<%(sCT~{0IdTzCjti7 znQ@PKBwxs7Ui%V2btxtV;KAu%n-rb@iJ-7JnE;9vIOe%lVXM;A)(4H*K z9~bT7RI@79f1$!JqVG?Vjl3|$FC%wuY5Er_{1W@Y#tF`*E|*+akWspbal&Dk37KwL zpR5ZzZWW$zExPMOfw)y_E9Yd_^b0NRqc)R{4LQwWST90F&*y)EO1o0+UJ=N*$5C`q z0@8Z@yztLGL)>Hcg8Rd**`>AX{;b=C*ErHcV7KKrPB@fxMdh6S4)N!hWj7G=eA6lU z6Jf@)V1(z2&0GlBaaDiIn*e_7>(4#A2zl&RN6ZI;+DFK@yOo@UZ*p@f zvO;0J3mb1UnOh8g%;zxYG9GKjz~lEDyONrA!^RU(N)m`t$R{#wKCUp|8EiN<(7$+1 zyr5d!e+45m9Kx)})9ay??C~*wH=E*v*ytu3Hw>5SPQ!j)NzRLbxh9qO1)66VPjumD z_^u{zgl4Hm|%=rUwYV@_#Sx z6xfq&9}oq1eP|`+#Xi5;IkyKf{t5p*x)b~Snvroj^LxTB=r^19Tu|~w%uC`ohX2~PT2sXh%J+u%*tb;Eb;?eJH#AlhLBUtiOW3T@ zXJI6LvIlx@e<43=e-&{1jZDz9ADJ`JqXZClxGZkLN6dM?Y`r@t#Ci003+}4h{yWq- zi1pe=pZ}AbbCwP}*$b?MX>8)3nY1ynH!)GS`+_+Txu-}_ACht4_Y0~_83bk3t^G~( z>;`(5wf`w(T*iVMsd)SL8jO7qNy$H$%iA93FDwlYedT4a5+Va%)tlu9tB&nZl5m09 za}la78+4w8a>H1KnBEw|fAAyo16f;Wndg04vNz~9+=IZ7?qU6@W(9AEJ72=#&hF$2 z=f}`pwrIQf8nLMoqS@>BCxJco$XrL{iOztzE&3f#Vg5oaE?lHdVWbb;S{~9+g36$^ zJZTaGa319!0#*DVv_&B0bM;lsaT9KZl(lI0v5Mv*L7SyY1H{yWd=JTbAG5F}XecFU zuNh}UM-PgXd9|6Hs_Y=3Z+tnX*>Wt9)a8)A@CF?-aHh@&2J8`8Y?_s`v_E4QFO&|p zG-3T7Y%x_}AuNs-VTUdyZ}ci%P=-JRlul>wvQvJWZ7V0$JjXU;eL&<2Tu8zCt|c7Q zfzjR3^;;bJk#Wq+&jV=QFn{A0&k?OZluJ4fdlOwPo8kR@J{NEnCK8}&IvM8|ps|1W zuy_G^{X_E`%Ed_c`{~>yo;5kgGxPHvHvV7$&6}F7PtE4_5-4@c(o-Gr{hjQ<q&-Udi^IPe*NdZ9=`xaISmR{HpF`lDh~+Y77z*en@8KtM?U z$6Z(xcM~N`Neg$g|3f$pNHtJG6F~>;n7mK75vPWfp^JpK5f39Nzp@7#ud6esvTr-K z+tCECv`BoX8fHRM8oq#ksEl}7W?-(?a`9bHWw{+?Jzrea?+68gdB<)M10YmeV?K0h zZIA3Ufb6x3RDDkFaV5`CnydT>gH+7Emj3K_$CAzYLcj8LEVYP@4cm6wc)^OPnKvB+ z*6lo%bn6)a9}d-Ol`N{5&}u~0g(bCh3&s7YSb4tVwyO3V*>P8h66M=DQo`8HHQe;A?=n5x1V^!iy{Jo2Eb!g=JH9UCv8NRFeLELu3sR&kX73hsoU z+HX_p)&llVbnHJ|+Rx#@&4DM2XjGfhC2QS?3@nZ608{l@hTulkN%k2!$u5%Pnc{lC z^+LMSg>U&Sbm%OBz80hfsoJeVE0g@5V{W{s*VP23g?9+^}N34l5}?n1Na6@FP4zZ@o1q6?V3cjMG{RI9D%s04wO41C`Onw1)=b28NI(w)(xIj1sD+f$9?@(GPnMCW z4_<|-G{B-^>TEgU~j zAE=9DJ|;-YH7(d7vXa>>2qNCgi)x;i*8z$mkXSX#n24a1h_xMGx<%YI<--9&e#U*b zu$a`ckrbem2%{)=-H9L4hKtG% z*WQkKTqaB!BoK%QMObjcM3}pU{|!2M-vQKxF@i>kdr ztzo+y6`yL=uVf2OsYFm+T|KK+@~Bv;=HKQ*mnQI0WUbljd_A6HXa2CRJhT12 z;qC9e^SSSz=Y`#e_R@%fhK((*yth?EQh$3Y!HUkpoN>FJNZ)=tP-HlBefpa#U z*qybB0+);LymVWH4C6Z0PYh1B%6>X(1lnb>P?yih4m%jps`>n94GfV@A2e zVqs}fP)LH#jm<)ztr*FGq}Ed4=fE}_o^tWBCN@pi)XN*hbjz7-Trt=X%}B*?E@!7u zZfK_UKoSk2+@5-PToVnsp{2;}U_yzVY-Y5sv@q*hTsqomu8j4@pLRsBAB+Sbv7Vs1mv0E-M2jH=d za^Ix10>W>>!EARbjVU@bb>c+)x)4b8ew)W6N3_IYVrFSs>DgycaeU|QCIAe@0P1X@ z^vMcvJ?RhZ(O%~030!ViMX+ymS$MN8B#~D^0?6GrMFd+y~y-x|z8O--!s{1=MMW8H8fo%>6U;2~lj3Jy96^Fu;<7M)#2s zS1MLvh=IbgMN=q^d)T(nxU@DarLm5;)Ig?HPu{QM=$=fD5-2#p-<+&KhWm|Imz8SG zRfBtPgGJ?+qLLe$D}TmG+Jm85Uz|adPQ5Tv=gUb;ea=$oz+btyE=`?|SCW>kNj?}_(pn1~RLiS&rLJjUn5O#bH4Yl!e$)Q=?cGwjl#8)W{k3vW=7QOIs&IX|Wz?qZ$9z9f#@>R#xE zX6P!13vV0>D>jA^yzQ~PyF7() z^bU!2`(l}JWfV;OW*RmsvHjcIcw0xEA`rRr|8#jL6Tr;deutJ@+RAAzuZvmn4#Y6$ zBYTB9jNj-uCUJ7GWJqDDAZe$?X#x(|BQ;29AK=RaJ~T#QUZAY`%Gk>dO%~XR%~qo3 zC0rLIMnkFniWUZoBHM_LquOQIe3hl}IAI&!m_L+8rHPrE%L{25DcwX^S;)7ChSgoO zQ80XstpLoCXXMr7iVZQ1r(GWH_K7dTqEe-b@(#g}X^2#O#oX|fuAV}sBR8B~{6u0^ zYm1GeN5n`LriSWHLAA&JDIe8{_xV06WfbN@uZN_RdAQ#%Ix>g8gbA8Lk z$+i3UnRAk~kjvJ8<~T!B6WBx5{x$NdaD}1~&7!J>9T(X`Ulp*Nu5RO>THI#;PH)Xk z5Ubg}F%1Fg%9+r%^x}f6?h~&14P?7%DUNtEwA==*KB2upxjlk*c#g-re0WYRr9QU2 z{85`+Y28jC=x)U34|dC#x9_KVD%q6lAfz%mEob@Lot}~eyM~6Qt4rf$%QW=wuC9~^ zoIQ-sa)BE-kgoopMW_v$RM)>oCyUH=+W+Ft7V3;!r+M^+oc(>`P`7VbfCXskKOc5No zo8-wlVp@To7<>#-ze20gbBW-FGk%j4#JUsMv@-lUJ2f_GQGqZ{hVY0caD;kKb%|qAH{u2*7M}}ApeMS z((x0EU#rrzEtrnIOZyB`VZI`#<2LbM!M@1*KQyz2cdXauUnR;2AxWmHqRuws?2oeBz7)I6+PpY>3nW1;n$d(2eIn_d5Jo=0QO0# zgl_jrr>aV>46Z10G*CopQ(_~_J2)>Ije1*11hMz2;lSmE?(&qeS})gV9h!H^hV>sX zl&Bs=Y*)^%4j(@dZ{v?LTL%ysWmJ2Yn-g+@0pr(;MtH$V3E^jRcFxUB6+m{%~@ zNLzP;S$6_6Hou=~t7tZS4sq`Mci??idW`Oc2#SRJ8pfj@bcr9VLNg%Aw7#M}kkOBn zgt41GYM_?VDNe6P3+T^U=;Gyo<%m2~p>T~~Y1?{C9smoZPx>Wv-zwXENYmI*#FuMF zL7(1^xAxz&EA_{($p<*}Pkrc2%OaTKL_zaPGuADc*?9{`*ojPkGW)L-qkI3Oo1L=T z8`f`x%Ov(eohjTyQgpe5nVcGE z@`&H$E8kO$LZ=n9ekX3R1Bzy*ObHy5am=-Q&-4jW=DD!V4ox&5g^A``%dmn&Dpat& zGcj_%${hlm2m}Zs)RfW32$;0!iD{w&myjj4U+9>H+FGF?!&Wd?$`>F24XsOS{du*{UVdw9TWf7??M{Z; z%AW7n>zvRZkkrj5{zv-zmObBz|I^J~%3prx)ClzQtgYhoiB1SBH&=HstB%f(KtV?I zO3QPtY8;&KLLm;W^gFlLAAREOb8kEIsYbV7Fq`U+cfKzrKWE5slNujx)RG1t-am^u zfx4jaExtMA6=i5;Wf$?ExWcmtP&WuBvYkw1_z~+W`ZXYX91_JCCm8Z)QKNtC-+~F{ z`E0UC>@cC;OdDL7v9Kdb53s``pKLhuXfY!q{g~u3$yXl@?&7Cjm&}&=&hCusXfa|& zmErroG9}dqbqRr~4)UOr6rjX=x21D}jKs9Eun*xudoO7Hd$C^%$>n3*Z0NACq9XtH zu)PK2;KF{*@AZ2aThuXN$_CZ8(qn$|>CEnN-tVkaVY5+u2azTj1b;`enar{OB+W?4t#+liWKOn-{iA zD~!5UwHjb5Md2T+)(9ZsNBYgDWZ7htm86z(vHv9qMa$gH>l9of1F{d3R+Ipn$t2a% zvJ+(SP4q05AQ=l#RzSc6-#g+?N=1NGi zX=>4u!)h1M0I4cZ92x3ntt;|r!o}c~6mnfrJcWD>#_jBdy#ZPa+EwCvj;yB87Um4# zbV=TH)RhmL+h$r!bB=~#c0kJ_{8}MGqq4S4H719)H2XG2(4G#j-Y1JLu*(1udBkWw z`@|928QA>Xwn0;=WqxNRk{SpG@S2O;u$l>xX8L8J9S2ytS=SmkIaoQw`4-HgY&pc@ zLloE=nt01K3)Sz=U8TCQZCcY71=BX<@O0+IDvmz{h$BUV)&qqTb?~&+t33wMS>Qy1 zle^D-HZ?Y~RK-~azKtWmR4dK>y*%j+j0jf+qIWDN3wrAi7vQq_z^r;||Hy;JX5QS8 zO1My^^MKtP=@+yAy~h?oWOr0CrHiwJfU^&I@dWMJLEeHCse}FUZhq?CD$Y7JGe8^k zfqa#1S(Ngx{NahRWAeH5Y}_r(jYa*Cugb=eJWQ#*i$r#3KoSG{%gpSFFXxQ3c)i^Q zC(N?!I#BkQO$DP4db746j->79kmDe{gB1bSg`GlU`2yH5g*PDd z`NJXv#R%erxywtZXt?(3iw&v$-0|*&Cu}cKl0=MzDHD)5`E*~@oUm;LE5_|ElGk@V%*bi zC%ZHJJguB&Ue`j!fuu(CJrb|*xkaHka`+ke8cMTRiN8jd31FW`-hX%IF`luwfQzpY zb!ooOSKaO8isb!T{$!Tjtj@kKmdr~#^6cjm`TEk3w&z5KV}CR5$9)4irq}YZd=&oB zC3XHDEM}PJ%C?)vbB;>sJyXsFCunZgxJUC^k(~{=M`fjZ(cIdiJ2qNSKeI-Uf*V8^ z-nKDmHPh}AX8#V8+$k2PMMLSiV#0cBk^Gk23EVcK{FQcwhOr7SnJd_UDOh@Ixn>a# zF+@s()n6s(Pm2C9_k0Fn=S9TgEKyl5IlHKX@TpMxK{@_`sftWCd@G;CV-cr30zWLv zQ#m8iMlilAie`UWxB1|FdKF6HacUX~&9KNI`-awSbx<=cE<78A2UjTD%XHTIz|)4z z)x(DyAT!aBwI2s=0(nchyi()OdM5v1o)5vwQ}0Z{s$zZbMWN|`Zo0lB^VUgOXNk9M zRrzo7K_2iWS}O^kA(?X2&xH3as9K}Y>{h0~%Lu+b#zx`m!73REv#4Fte_=X`3-FT? z80^1f@&w@{*Xv@yT;1|eEefZ&GvSaAYKYnCG?Zn~q6H?-Pn0yxT_CASO3Y;EccAv{GHx`;={I2HWT|}2{<6k4z zc)+-`CqD180w>;u4yRKfDrByLym1EyI-Esz>`PyMvY}8_fFRQXUd$Pix(dgp&NM3z zOR6ix^Wg7wZ){?by=x8dGSx!kofA=VhGXe`j6!fkyU>zgJH1Rp6YKTAW>q@E(F!9W zzpr)ksTO_-!&W2Wq8pD$Dmewc$k7zN=#x`=Ee$?y?jA2=G>K%GIu(rpHLL-eP>|>= z0>sal)(msTrsfV~&bF8_@*MITlU`KVudhA@feCob3^zD9W?aVhtX7TU3*fR*{OHBx+3k~ulkIZs0Z#C9I?|PU7a~kA7l?O-Ww2s0_=kL{y_z^t#K=VpCRz?c`qadk z#$1FM&>15hW7C{I*Vf4OnwEn1$IR%dDwo^NRe5L9*djQC3$0W{e+=;&Hs7(nBjUGN zo_E_~sj8f7P3n%nnb;#4i#Hb8d#LX$(v_}+V8VukdwbvVf%Uvyb%hft__CXiRrc9D ze8=>x^RDgnr(A_6Nl&Bui|yYO9)I^MY1r-XmZDF=ckvv~dVNK;Rliv-OfK_7@+%qD zBsf?Le$7s9#NJxD)ZKsXFr4&kB2v6ch{GS-DqR(N%;`lDKmd@Ls`wRzC5fHK{Muwg zR#P)i5EmliR!F{G8A`7^{qIV8?EmB;i7>%X!v-*lI3&T<=r!@=Uo*UCkn zAy%Un5$o4pF^L3Mf8B!X=j*AS>`OC_)jTVNJoIgNmllF1dCXxD){7x&6rxzkDV(_b zbx%*d=q>~FK=VzN|>u-yT;DqRa2WL;t&812)~gVYorZb|IST zG5&5*^Jq^?*rI~&x6F1M-aSm=Lc-)d7#Q z2H64J2~`@=SwZyy;#{h6FVlaEkC>K2OfVsk+%kd!BTF-jl`a0Ue|94&Y;X|HfM}HQ zDj!vXK#GFe?xfb1Szz7p%?NOtDjPFgJ4Y_704B;di%QGS>{LOq%%ECUTzF2btZdm- z>vbTG!j)Q|o-hS*B#6yC+a$|mzbh7R$tVlI0#E^}6v{j>Jnl*F_MKN3WT$Rf>DBF{@Pnv$%vUF_A%v1o1 zA(C>G0ExeKI#~#!CCp*mVLY8z&0;FdD&lDZg*Tl$3EXsRKfR`8DPi-3$|IJ}u(hF+76P(55n9P8hDA+N4U6g@$aOZ>Rk&|LT|edO zBK0lk37#7t=5=B`k-(m0VDrt+!#6;V0%zc~umT*RI<>!unSxSSF}Uyt!M&?&4|>ei zqQ1>#yah1;SUFU6sE0|$R}p}dkm~YBW=8Q2t_D3LU6}k^l8yAM@o0c5%hh!w@r&6! zeBLJS7~KYG7?<4=eYz!Ef2702WBf~Y));GeJ;5h!a`vNCfiAjE8)m?xseXqbS~B6b znC4eH@DH4@G$AS8#)9f~s5GzraDEobXN?zdp>I`MNw|;E9g{OZainjHd_S&xa#+{NjiBV-whm>)AEl z;WA$c2Z3s#YG}(Z`{jmo^I*|-VVLeQe~igl+Di+OSbV@tyjyZip3^9Kdqxb)7s>fzEJC@)k({t@S6LNxdU#07@eZ9#VnMA zYUEJ5bq|_gK(h5vFQPEwEvdZFRt#*V(y(7?)@h%ZjxA_JfDpnsJ=d4e7=` zZGZg!)#0S6OEm{9&}$Q5exp5fQ~FKf*9z48%VJAT$cqAQu5;e`&&aH`WA?y}+B#g5 zW-F*)Tia8FiiMX>;jcb#@Tl(OJ6q;+Z>(27wT? zXOnE>(itY;As$u$`7R=vVMp+GisMY@mGfHxa)VoH$zw{<1AgKGPE?hkh!-eL{6G#a z1j4ha0uznsa8+`}3`ie<6>Yt`8dOnpPKlV`vXlkVG|m zLw(c}87(hFQr?7txo}ht6#Ih|z37$fpcYWE>3T)nqEEpk?sS595WaxeV$+hithpgspMS0Vfw80yaXNVt{%6h`y-K3wQQID>J5+o$_Adjn&hdym5(wuKKN}eb5Yk z=|GUb_(62|IIV{icK9LW4W%!akiI>iP)+srisBo(9{U)K{cV3^wl895GKrUpps7z2 z=GX2PyXf+6`YG4%k`tzb@br6>ERD z27;MYGkVbtS9RMY5ocwJ{7z^q0mH$Or3Wq6H&Gns`vp zpY8{!i7l)43+1L&Z0+oPjAk79I68FM<`Dx-ft5Huu9izh2Ntx%RcVI)EcSuc9?-n? z$cXzS*%#M{r{CWo?X^N)wTmWm9B3h_4FI$H=woMiPp&i{-b8dKVBHDj@;I3C#ApjT z`9H2YV%59=Zp7l&d))Iw&!ypesy_bng9=zdVRk~WJ2cB|5fAiCvWE6!*lt+=_h>c6 zDKZCzTFh6DEneZ?o~}zuSx%8<%#TmN>9@$>RQUsj|Da6Mx7T~Waq>l$r+CnpqTBma zusCh^%H*V3>K1;Nafsf|xpY?X3SLbKuAb0vSJK3*Q|4r4pCw_0p_d-Sb>asJoWBH z0@+P5*ySIq>q0c#d^X(T$4zPNHiY%Y=%7t`rnosj{Ni6-;n(||TazL#LC*_=U3vH? z7iW?^fw{*xXVyL_-F*;EKH$v)9!y6F`XT61$6RNgbW{ZveCnrV`mxFweb$^7aHqG0 zN56aOfw--O_1??FX_|3wO33+UBY_O3o#zjpcsbe(UsCEbKVZ!||LN$Sa)QJjRoIf~ z+!xnRc?KQ7Q%}O=@W9}O&Rl~spzfVfNi&+R0YYbl|IV1o@!~#re{BhyuXlQo6`XVe z9Y9zR`$ycp;Aakm109K%1;q=S#t$;y1R8GF7Fe`EN(j|wGK8+z+tERwNVU>0Do9kS zQ6zXaK^_x+o7=PN^hv>q2KubykM37nNLSHu#fuuw5ZQ3CSpE$RmJfQ84}1zQiy&BG zUD4zzwxE@q*GMX?Clxi0lr=`e%4DYK>&?Dkrfh_@VOSq{XVMQ+UrrHf-Z~Pu)WUM< z#$x|T{8mdN8TSpa^7VW?g3{BzpmdA5d($%4`(#e5)wwKvWKSD}_;kOKxfLVeJikTj z_<7T8dj&c9gmq0GxAd!}5EO!yPd7P|dxUorxFzyk9`RRo()a6P#Yy}b_~4x5k_B)I zv3R1&okB_M!_pni&P(x2SHA3&XY@+oyht}?%+4==n0BP*1!jF>hrZ%8#rI0K-nO2( zeZhB!=M@Ei@No|?SR{M9PuY8w3P3V90GVM~SkAdpE5!lwdLb zpaXp!l2%UX=DPX$ym$sgYpW?Z=BG6mPR*J0H@&hBn{bhvP`b4J25c!TG*yF&!pnZ| zYb{f4n~!-1)&6yi+tDMtx0AfR$|L*>l}Tyu-lEF)_||wfWKo}-{XvcUYM-y$3ufB; z$MQpb?#;YnzjqG#3kUagXRq*Ek`8mDuhx_l#Fk2D{l%A^-q!7j7i{M2vySVRR(tBr zNLlcmq5m3kUWd@8HH@Go$;6MeKRzwqp%6VX$K2X) z{twv%WQ4qgXZQiBeGcq7w=&l$X7K-N36i!WGI%jVmhjn zE)%3HiC!wf!g?Mq=N&ox^J1CfXfLG7y(0MqZ5#P`m9UT|`Y%s7ig`l6)-}5TVZhL@ za%CK{8eZWPh2Y^z>lw?>PP-l_?9PV{)i0?lYOO24a>fXQcM2gY$5N;1x)SYBiN-Rh z1NU%6NTae$XIaDm?oc(hNrr9S)C!NS932%%jo3LwR830Z>Mbb3)jG(xdu~QxCCI7Y zN^e0;A4H$+1qBmg(!yf8u z`s7t!{#HNz?p*x!#%1*;vPd%X0Lwu}-eWwPN9bl`3aIKr2(=C@y~f(LC8%`z*pq+AUR~eNyLj z3~`=09&o{~E>|FZ;tcM@m6Kc|eqVy}sx9ZP7iZ!LAF;_)K|TpQqtf{bx@<3-m#P1N z81n|I>O~`8Np@w_m7?&}iW z5xre-UCOfzacbw90AJyKn$VFJS1#hx%Po_qB7TbG8iNd(D_<5rf3ifuoRr!~)y!Kd zWz(J=+FFks)L0=iIV+f=S-QL+T2!GkPN&WWwlvI*$#vMf5(CiB$Y}~8a}d#Zp4aJf zk?4d%)whbWSDCK6bn)_=j|j729$9&dX$Z+wzMvG4Vf-sTLwVDJq^emNnxvpQELYC9 z?n7c#;3r*17H`wc&d8s2A5q#F2h!(_c3vY0{mktlq(sv9#&XMOZqf3FOQ#^ORPqKw zr+yM}zt|+j2b1@%YNf7UWWD^if3LM%+c(^;{Dhei<_{yxpmK`!)BvkdmXiiu_$>1& zvBkS7cH_}uqxQ5)h1H&LcJbl&cXBYF z1YtSYu)C08xT|(01H?BM+*b_QMFq1$uFfYa>MmN0?Hs_kN-J2iB0e+sg3bf%P(7&) zixpEQrpqzz;xSaMCS5e0{{o4}*W07YAlT^-bmbSxd;aJ4OyJkLfcWosuxf=`g&eOG zwaT6Ol=JuqEB_#|?@+hOUXXVGxjL^LJJ$4VQ%Y_N&eOWR(xR~NI4p8L1DTnGf$?Ni zxzb_>X0lK~3o7gz=VpCLJAv{vcdQo%uUvEu9rPiKc4~)XjocFCqk?w1SiNA8MXFj{ z#iK-VO^A<_KncPh=yvD2ulXe^tHCm>G}wFt>jsb9LiYHIYHoa4#SIsh0yos636;N6 z&gR<7T$x&~Pl&QL+|jUJPAQJ#dxx9UfgVIE7NO@2M0S8#$RFKXj7XL}s%$P@ZZTGg znr*_OfC-R_(`XUOl%#_UK4 zb_Tp`%M<*Ix-2I!s0e8emb!#SCH@?&80@Z_p?WA^0c6lNouuSFd!n(Yk$~*azkEe^ zkgt{xWc^pNw@>!qNt)U1=Qn%2v)|XmNEP*&n_?tK*w&Uw^D?3%UesdVsq?9hWQ-J| zN4T;`7c1PdN1)V7zAK`Q&_rvE$T{5aHM2)_e6kmdYgIi5Y}|qir7wK5vkgJm>pu92z3LOf-XEMZM;yd9xyyCVXO!A^Lp zpW@LrA9p{)U?AmsBBA{Ml0?NF$X^jikI8DjD4*NDJ&QfM`I0)vTp-1~`UCyvG@jhi z)2Tl&5D?TaARzMpb&u6n#L~gS)JfFN*vZr3|KPI~C*=kNQTYay>mVr+zz9w&hKEL) z!@weNA~P*k>oQZJP(4fN$UjoZu~9I7lB7+$lqF(HMp)ZfoNu^Ja=qVQAHRTiQBydD zVNau(kv*l0>REU~nV7HEu6nxYy>TiR(FK;iQEg4RypwlFy{UCbLg0Me|I z&|3C4P_d-V=n$-(A}`Su8{v3m76 z>>0Fdz|m3-Hw}aCOaMlL5q3qe+|>(_jrZ}nKaaCAW}dD-kI?$4o@A$xeh9bTyX{j? zIun@j#GPC=5uN-9v%k>guJ=n1(j|~-VNdy*eX$=dt=;=HWfE$b^g)UFNhDO_xIxE9 z#ttk^4x?oEwU6o&M7Mw2k>~S88u3pV;)4ZM2^8j%Y$k-E1RX`)i(rHkpQ@_5^dkZH zVk79{Z!7#g0VOjZD%&Y2P`FQK{Ry}X1x`y4!?}>nf&+*@El2T9sX89l}(E{~k zubfoT+VX1kTdMZI%Jk`I9oj zSHVdp`sP_;N)&?*%a-NBlU5m}EXgzEn`q^t9s5Iuq?IA0*qAV}kT#oImR){8LF=~x zD@)!)_Dt2stX9kstXgHMGa5EBO=V}OeE{$FDE{Q4_4mjn6`~D&T0ol_40TKT1 z<&$+bS28tr`=6?5!T6{wXME3enWvKiC<3vPfl^J7V=!S{0Sd4ZQ1J0&0nm0794s8; zeZ2eKKUu4-7IZF}t<~11t*+MP8Z>VH9n?#L%c8ceYFF#w8}8OSTWZhqzAjr={3g9O zn^`;@?);ev1R2gV?mgenygRQmeUtWluJ}MJQJI7uptRHI9SCOP%v)_TCOhNNDzks_L(zFJPA!fx)c zvT+=l`rDJYf-wzca+Pk45r%AxBX+F0*YngL=Zl0HauHN#D`$rwdl9A;Aq?F@5?yFH z;Zn*PNGK`xN65@lp){7Bg@Kz~v~+1dor(Uj4+XhzxSv?-%zkv5l1~8}Pjrj%7ZVK_ zeFW48PmsU!DQZwUyNa^;)D9v&Y-}3qpbl0{U#B4_q$RLz%MF^ynhLdCMP?!mS|`mQ zC1pUHVaTerA?P{+ENqe%XiIG05@pOlI+bUw%F<@zw2o9tlrfS_aj@cx?`o{nv1}ia z$Rt`~HD0w-+px~{Rhb^KW(nn?VNiB7GWtlER!qKtl1m7N=?EH}ZmS5@SW%x1yyn;9 zIPAl7AaIhmG%9Sma7FTlkZV6pv~bdLuHw|ttH^*SLvC&K=`8a;(@aH2{dl=INkzMI zY5&Z;ZrgmH2nmWA&g&ceo$H89j%+7~U9Xo9p{DR zp@ex8Y_&flG(q7bC3=huS1o46(4D0RCsSNmT0EtiA8ND`lT`Gl8TpM8Ye}8C*MJrt z@svRP0B{?WDUq)@R~|>S(*30!?{kSTaD2h|F&lL9y2$aD5U+*y9rQX@0ZLXD^CX+> z7!~t?OKRj_Jwh?fqr;dbY~c-qFQ&N=3bzI?+9ON|oHF@?E>=9O4M_{u3O z2lhZf61xU29h8Umd(hn?!f{?Kk%tP<*qJQ`y-pvs@o14sdLL7%$<<|&9cqlu3KB~! zG)S1|SiKod#>zC7=&lXwO#cZZhl{!9n)uqQ3uU;uV|gJe!LdsQ(}H3)P(J&*@j#Z9plqnNl-?L;x%kS@A?z zA=VL8yFQ&+(-{i$am`&2RI@fFTE+2%F$F9P7Z9NwUc`K8G zeq?CosK|Cu8z&bMzQ7YM-AutAu{0tQ%Yzl_Uesl62lfG;SUGemL%rF`h_b`_#Dvj4 zv)SZ3OWNed`b4e0b|F!f_V71aEV`m9ZG49$Hdprk)HX>IX(C-(4BA>AOHx&(%t|2X z*24xD3qSl&N9s|fV(t>Ni7@R$BaZQ~>O#UQNzMII&LtTY$Fs9LP_kUwHq}GL<-Ue= zGKVeZVKP9)H14l-z&@cQd;_uiRE38#dpprGm9R~|sq9X1zDuXv6cbh;v*=*bK32xG zNLG~*UQ?M+woN&%2Ly;+|ALC?z-CN>%mPwN9!rVSd|Bz(i$v7U7@A3KzA;_e=c7$* zmJ+oo{k6sr54g9PEq0~E${sg|tCUwQaZUjV*hRyZkUZ5YkGZ(rnCV%h%aw{)PCmXI z0(2l*;B`*9on>88qAmV)@6SM(wt|P^s}=2({@$M-=@SVbY3Oexq~bIIFQbECs!cE2 zZUk*`@QO*9$_n!fGhy1&I^E|{F<8@H;_W_Ncs#Zj$sQ%?=tJZUMlt5C`IwM7W|p~c9WYnz-ueRVj-tl;G0oY#S-h8K7$A6(jg*%fTnZEm2d zyhK3j-VONeyjzHk$Zkp|m6bl4$!^ev8%J@jy0e*iIr>2kpSG<6C{koRNas*gCU{>{ z70t5iZeY}{*q`%(E$)RW>iadVBDl}Zd)gH!H6t7v3^Q+rd^UYMH$!=F4sJvbh zA=7mb<|cB{tQHJTANYRk2XT52yXht5--FPrC@*y2|e zfZI7Q6^88lAy+|e8qz5oQj!csvvHljgd#@spltutw@%We&GfIBq`8dKIz`)D*o|6j z2V#Bt$yAe^Mj=Q@PRg=s@1dPiFBF3ti#XoxdQe<86zdz7<0m9-cI+I@`L2M$JzO@l z5s2^&JmATB5F!!iojljT8&O3(tfkj6Qyq zt&*iP{5{YDGWNpS49FwmJ$})tPrhkpCX|>M@!o6^IEM6}L&r-70R!1<90WOU4H!T^ z#r+7xIj4e9Sq+2UP#8|>O3B+zKP&$RB7Ed&xMHPT)w@y{BnG^y`Jjh`Tm)mJ zzIP1eOL@oNT3c`g^+Pt|MdN6X;~rIpnA5NXy=iC*aBN0&OOg2EPb(YS)vaALw=0}> zY38+?XhNsxAzzK&oBST(zbd)GKz)|f?sZc$V5!0k5uXS2CB*%+ zAH`xGM_lhc4){B@xav~@9C7XSZqvb!P-?YmYPH?>1~~@xsSO(MC;e}v(J2P?3-<LNlI-gwoM)P; zyj>hAJch#wY58Fl9WhuV;sq<)Aolk;n-hMD>=Tn}+*oe+kc0KJ-Tay{^Z4xivr5Se&$ zkamJdZ9Cs`iFXPyl?}3Am*b7IZ4YJMvpDpCSn0(w7;Z*UneI(WWYDb&7jmabAOp<}t3m!ay@W;WSS~ z-eBDFYnycNq(+E!iuGMzxJ};QWR3IOGKzzOe*&(Zo_w)tt>Ks5D9H_ZMc8pWs*Uzp6Um(8y4HHiIW9^IMv*V}*h zO*fji`{F-+*E{f!;pD}C@XkLJMlA_y#eU8CQ8lhR*w8=P^c#D0TW|XMU&E#TNtCd% ztKw~l=94-GHjg~rMQ86plRV?PGXgO1&h8My!BA_%e(!jsz44GU0u=Dh{;E3TC5^_D zy>sihsm9#aWu@lRjl(4k))Rf|XJ__ZCDzg3hRdBs29ofLjf{=4#yHo{XRd!aF)w;* z9|)WW86qcZV-39{lJ-Wct#y|Ihj(^hY3>`E2mV7`wUN%-cB$W^xc)Gs{(O_)C~>_h z2fdL;-&E>ssWW}Eo2nv)ThGy8Spp+b-|t{qdFHSFyR+&3+qUb2k|Vx>Hp;T z<&*Syl7(?5P?7YHiaqc`!TcdLJQG3Yh)MitN7QqLji&b-D?6f0CmlU46Bh41?J^@DCsH-1oQfD*yqp~RtT&cHo(p1$T#9=aQB1ce$=#g9dG*Z| zoSa~S&HYpQwU5DnZJ1cFNZ)XY+_sH`Zcar&NPX6s#7#5)W>%-@o}JS?d=xvca``v1 zKj0J_!`(MmRQSs;>{@1gcY`7YxAyA1o!RzsTIJr$Q@%E3s2j}&MHB0~Y3$5HQL2_U zEk?;V)R1!}ilv9bz^(#P7_Ki`2+i;}X|<1&v?vd1J+|Uruzr=8Dk$iig!IYu3x4np zVdluYxc2q7KH7$_iisjC_;O7yZHLvD^9DNLx(EC3WP|n;ngK1iNMR z9k(_4x^qpIE&WO6q-WUpx2ttm519-zXu~ouRtWlDYqRP_^HL9LTF zLu?c*{}3DY@(8$&IKdD@3{L6`%GJ|c_6oWDeC=mnz=#}Ev8Ox}ZCCwTu`V`)WGw5WOi=!-hGUsj{#Wr62z4(TNECn6a$aFECZ8RP+MG)QR>3C0kzDj`{N z77@(ssgMk)Xc#3y(7*fe*Ad5+D^}H@RZ-1Xe{7dm-YhZB%yq3WI$QPM+f{pN<8Srk` z)!A!ywUS^$tqK|{v<fxRQ?$BR4;l>HgYjGHqR0M!v-4tSsPT1E93X%*sFOcUu%( z2y3RB!ifuplJXh)1lL!m`nbcvkC_uHtyh>8v>epv#}q1riBOM#g6F9<`lPCvs1HQ~ z;U=(9O{>yii3~iT@xUtTEK|i7M!nSHk0aCLvtl$ z!930x1dqecU?J^L%Lw%Xb+VUOLI1LUgrrkll!}&BHw0hG*C?x_N8hqg+0)q{7-%Pi znpHPME#qRu$+TgFpfj-3K|3DcRiWLLeu*e|vCo!;gi)~oM>9`sM0^oOyJNi_rq&T` zy_7@{qJTtERdW&Bro4h9M;|Tpo#e0;o?i(?&f-d0IbT`HjGZ?V2B%IfT9d3Sx-ZH5;Ln+)J^VAAHrs8NuIp3vE8icS0)@;Fr}*&00NT_Sb%g1%=K>dB~Dg5hw)zxv-hy{851DeSVGwd^0-oPxMSt}D%p~p)TkXJM%YPY1P zZxU|W%tt4U86CpoE+r>S0Nd;}}!D*6AH$b@?RKSdv@J(XlA7AUs-UUrU z5^fdrciHANubbeEkqV!p(!`L26(qsnOtnZpVN^3#D zv{TQGHtSL$hKpdxOQB@@nZ$*^9-C6tafs_elVRk2sE`&MlcRLQ$>e_+An#V2)!)OY zKWR5;eK5~Jukg@$7sOWRf5??e2;@%AS^sw}V)LFT$5kajYI9U1%} z9ZBuOlhakKQGsI7u*G4m?KEg?t`L$T#261^3ARFgHrl(s0E-a#cvn}H05v?fdTH1- z7j}9YYmf4nW_=I-IpJ=f#TA#8I|p?K!}?Tpt$2sf!!x(PW{;K3x`yU$bQv-DCku)< zASdC0SbspFIW?p>8rrg)*i!a6?=zIx-M8{V@#+cY&z8Xd}OHHFQNY z+(Mo2;|AvXyr})hpeptLkHnMwbycLA;_F{eYZDf*>Sc&vf7x6CN=nLFMuf^u$Jhis3kv?$jC^{9%700)wF6L0g`DW%?$e zu@ad>RB5NbS9pAJ7iwuxBYXcgH75T#{T?dl0t33yqFR-X^El{5C(@w%TX#0=c9*MN z`~OP7Ji6dps?hDnTwTgI5<5RD@Cj@2t@32XZe>^2H2wxC24v zv{u30xHHfdczF@Uw(?R`>BRB$u%ex{GqpDC{0$Ey4@wghepF;z#|($?YT2{i?=c#< z3EZEnwzbuulMLgM{~Mb)Sk{TOZb!5t-E40 z2Qg!UeTeOhaehcxDOj73jw@<1lG>jW0Y}uTa2vNY2KaId{{H?UEYpx14s5lkGu2_R zV064F0>sUkU|fV>#ZEK-_vQ-YyZ4q$ucUgpk|C|KDc^2~Prm_y)JC-?{P?4PuN1fF zRN|EKwP24%64-moFLv5khpQD~Bv3&g=Z|EI{m18Z9?YrS|lEH$CMQG>LcP-Ixzf~?v?2>T09R^> z*=g?T-=TEwRt^CDq}>SbMU6u`zfbKh`t;HN?N>{ScxY4*v`R2AM8B{gL5wPHMT|o> z-zl-t2nOt^U^Fl?)-gB%z}c+PXl%A#sM{G)n(rc5 z1sIL^x5Xpq2LdL0^*z51Z;FnGL;Zjl=LB7jFs7tmzr9Xy0hCt#vF@zO^S}zI-oR}{ zM?9A;ElWV_mnZ{}nnU%P%5ci&E~KVYW*8M=^I{w$d-K-NbBDyIIBd1Xx?Z$nq0(78zu zQ)JGj;k5wuGL^Uw?b3W29Tne1l@Lr6*CtmDRQ3f*){U_JG$BtAla8E~fSnpAvOpub ze2u5ha9d`LOT3`ia6@Kxs@Tm5way6U2{Kr~Ni=2-MqU(J?U+?!!5$Da954or%B=TG z+CKz7$9;?)naQM%*316DKJW!BqO+9OA_rR$B%SaxwJ!mf(fKae0bs5W#shffUnqyi zh^M|IyMJCW9{Z7I!DaHpVh9^Zcip+bxBX_g0cPyJmCZa)Ke2&%Ph!+1*nE^VsR169 zxpA#S@Saf;cM&WFej0PJSoblMKX<~E*CSOhd9e4rB_ex>(0H$$#z=SPo!jKfIG0TM zgY#)Ziar2SWl52ESNW~anIjgn&Uql|i{#J)<@6ywwC_pT$2M86>aoo-f)sUxlq;%T z)^Wfp^mSXeYS;+jP5wfAh3ET+(3XAV1)I58>A#u1@`<_r7G)aDnQQdnU--t(9yXEQ zyF2A{RrN&qP2%@pI)_sGN=uvyvX`UWG3cRPZ`}mZnEx7a0D#YxzJ)qaPdDUhhRmjd z*Ll%j_E#lNM7eWY^Hbw6?Iz7q4GNCSy#1Nfhaf@aPTdR@H-)F|s|Z}_PXydBp97X> zYjeygiygUVUpBSs?j09Y{F5d0$^ccKH;qhiR3y6qN=XlqTmtRvz!GYW6iQ!=&~l0( zqThK2?;#mhHe2uu(-iNk1jlt|G85Qzr#+Nksc#$8@;srhMcoODdm#h!xsF5|f-3-r z(?{L)YFU11C#N-qa4~2tN=D6kaYq-~PaI*L&gC}%`qEU$YaNRt6tWC`c0ilkWYMwdnH+g1sxz9wL55&r>9+pUbl4Hspa!sT5Vjg=N2yi3vRB_hG%K+11U(|#(|FXuqBb( zAh55h(6A7S>0`2N>I|eg$M81@ko~u1wB4@QUbXOtO6vr^zyxRrWEq$^6=XsUfuwed zF3if$j0&bf6(oI4IY3Fnv7$LNkONZlmJUiv89VR?A>UbdGj~euSV_89WmH-_CH5VJ ztDUCFlc%qKVs5Tlh;+kfb;ami-sqOim=eevhB2kNSLqe2i~B(~G_{BgWrioMO=V>t zD7!$n=1DgHwY+=MG=1_}H**L4&61bK3AS~Wzk3q~R!nD_upgqyR4=TBtZ(eKCbD}y z&sq4fUIW6RhitppmAkO&7{38I&$ggw`AQKB71@j$)(Q`uJkCx2K560M(z$R?I`M`s zv93dxs{@>*;iEOuf!R}QnEVzt(f@8)!Ba-g0y2dj*Q$I>HCSzUhR_AE%MWQ@JHkcm zfI{XDK{z_Ejcg4Pd8KSL8Q;}9t7Z8ruVQ3Ai)L>n_P5_8WbeFnVz?GbwlABdR<%gt z{{}G;Rd4^18x4Md&6)ibvVEIu=3gc|%Ynf<^g;6Bv0iNClxr!5&HrFg-6%TY+8efXzN=j5(lXn=2t8|Q2qC6%J5F*8Ul;;}E~;UeSja~0+D1Ta98) z1d*6AQN<7UijP5F6xyFA4hz&U!8MDa%!W7^$TM4}blL_fLW37_wEi=vThwB`sNAYt z(PE9zSMQHMg0)tJR#vdN*I$q3d?MW}Ex4V1P_SNv3cWiu6!hMpmoqE=w9$?}z!$ZS zKWWEEk_@H}nB8?&XN$oJxyO@Sd~=GpBLVI~EjcP;tihbY=){XPCX{mEi%ah%nF{bU z<4Kf&?j>yMWuMnCO-k;SG$Ws11gm1(rm12Ep*>KORL~Is-QGQ?tc|RoDA1Q^D~AHh zbZ9GA#QB7Y(3e_D8EVtZ{=3v0FhCU%GB~o3zGRkHYB8W2QN*>D7Og1Gx|M5y!J$Nh z-As4O0h8u;!b=XpK5tBxOaoei986ILpL8Z_)ZC9nZe6>_r4{D(KAc%%GwG&DpVpFW zOKcy@CKaY%dcqzECxqnYS-k#6tyL!@w5oP9RGT5pEPp;o#Tkq(9|tG$Q8^#_$SiU{ zrkd~H`1ii>>+}e`mskrLsqi9KyB()kjJ415Y+{yb3(RW%Y8D@xKCtMi{rSS_YUN_R zW~E7owsA(V!F)ZXUm&jwI_4t%87Vg7#7$M-^;_)zHf=X=A>!^E+SQf+q|c0ZC{0#m zFpxSROf)uZ_t!q2=8!0> zu@L;o4>Q74YP6?+C(z;sAwS7ut8rjsYdQQl@`Nv& zr&y=zzjb_-mVuFW>~7*f)u$k5v~60+ix|W&z2^F&A~!ecb#v}@b6~~sEIO@i2yPc}n z6|UEHuGd9A;C>=klicIR3&q})cw-RsoLKfo3hc(3?hem(*kyX|MW@t_aXj{+m*xp; znF!A(#sjf&bfMeg$$gn(!xU>TOL~-R{T^`3wxBJIgRH2kN6X5cU70p`__#We=^DZ? zQJ@`B!g&UUGqC)`)xWX|ji>!GDCQNt*iQmdGHCSMzkFF>7Hh7}ExvJSrO7*s=TPP5U(ECZN?5 zxjGp)5cqz;Ruw0(<49Cj)_c;H&r^}~PEH!T?3HZdprbgndktEpTeFkx&XqgJnc@jq zGIcY9IyGaMor=h;ozn-{{t%xmIR~uebBfBwHa@~#sCu$bUWC#xT$rNQ0CZyIMX};F z-squyQ?L^6>MGL{D(lQGz?;R3JcA8DL1Kd)BqI{k;cX7C_uL7#F5hwSW_S6gT5m(c zNwI>@sFLVy2t>l{irE$iXGzPcvs(=O6X$Szhcz^<1o-%jbeapf+&4mP91EyjEwED4 zrb^SpJTz*W&l`pPCy#$o6PrEd0C6MfeL}i?&urQ6ke^AZZN5Y7j{BCJEMcgFp*=b4 z&&`Ms=kU%!eRJR8gj5UIz%bT{4WCA1N`@;huamu1#Zv5svt+3;ONtJpJt)L=7OHGbp(a^g( zeu55bOioPNO;dmJbu^?#AN=(bwOI;K^(1c*HN07S1S_hgHZ$9B5NBbQ-lJ1n_Dd}7 z8hVyF3vgzt#4|($B?aW#kNJpwh;){cBrkOK-F%CwtopUHBeho z0b6E~r}e;EfcBJ){WOSnQ5&E^HWkIw`G|q?>xj$j!%mcpqVU5v9?75Xjx1EGI!L%yq=X6;%s77V|Kx)pB};T#Iv zlU_A9*GP-veA6mWS+HDOAdEnr4r;|s3xd9(TYfIYsB}<|rq=5#JtwXe1qegVdhHb+7DUITqPVN(d!rK9L$Rlw7JI*F%L#~yx9^x-TKd2P9 zZDbhuc2NITmT;?7Kd^_?9EIfUwP+)FeXbR%DZWCZIZ|T?!qr~GWi)m1pcFmfH6qdV z-0w<4YOgrGk2~5{#?TKMy0x43_5_{e`xE*70pb1O-{VP1&umV_X^k7`GnZf1iPKuJ zYL9395Kh1z{Wm+X>hW6mT+V~BzZz5R)fwc$h6$j?IL#!80#oh8LZ3DV^eRm&aJ#N?iWBdwRA+nOw~`7>dLh zy?lGH^IEU&Aaowk8PxtGL2jQ25rHM!Dq`DSA!ov7L$Z*&OA<@+T|m|khUU|(Dg1>< zx%4w`RLoVX=LW;b4ADpysJZ)w+%1AxYi8K-qBQ=83n~ zS*p>NNw5$jILTTjbZrmi^9beok0FU}@n_#xWx28FaGANJOa3dHtW|m~enC}o{}o9l zO#o*Jhh@u|pU?nqp_$r{H9T3MU+^EJh@Axs+nS2+*Z-#v-EpVN03ZecKvVtS^r2RU z`hp6-0x3&92Ztul|E`0seSc(gde8s@0R|KKT0$lS)}qq`27<)uLP)IE4gX^Z>?J}n zCZVFyT5N7!u4<~%wWw1s3#V3xsZ*6RZCYAax|r{3npnANaQ=GtO8R-o_Bb()mqdNk z`HXXLxM@4-ym+8F*>|w3I+#Z{-33d7olTw|k^b3W1Vp327!?*j{NQoe+>=)F%EH>O^Rvu|epR zWfc|oI&5pe)Ssw;X#pk58gfu|-{G~8w7Hb7o36?^`<*07n67_yg=uiHUuCuT%3q8w z%-wKM^+8}~K||tKzpUyZXpk>&3RTHAi+mQ#2}yfIcVtfU$D_dL>B1t*L>ps*Nrcxt zDru?G3Fp-eOX_yswU}Jc%0xbvgrRK%WBm`c-{K#7F#G7xtQsl}mq%5{$TT#}a3 zAMxeu1}TaHc1g8WFSi2*YVcwp%Pk#_=L2!tsewemoSKVBQNuG(5-qW#E;Nn55u{CZ zDJoQf#mIlm9cfnfE%rB{kri+{o2DkvRHYFhX(T=$&bM<-(w{W?EVzudgBOtx>0 zY!?vV@>5cz!C2~=cLTywejQm8L2SP6GVDI#=*z4EpI5K6s6S6*9)$T^@7}8mMNCZe z#5HTaZ6KexPA59aYL7R+m=?*jx~NvDgkr_3WA>Ei(O`+pSQhZikhY$EnU&hy#b{7I zb~-7*R(NSP|HpWLiSMS^lnRj@w{?w*VasM7kl&Qo(7cOkQqyRD$ua8c(R~kWBEJO2 zT#Y%IY};zyMcl`0%}s((kfojxC+soSz3U$VELXZ*?iAux%T?C9wAovm9S z3deq2v@MtaWsup|Z2UQ{Yx4?BKHAy`Ny4vlP>@m}sSA7ew)4wgj6-Pu;8Z!mIsrt# zaZnAmj&eOie=3z=&~vdcW)7Qt!Lf9Wv6bQ7iiOB)1DsJAc$&P;qQ^*YWU9(83!3qE zb*GVXzR|>p35Lp$&kV2IhrNGpH~>5TeVFiq>VBn})9kXnrKHa<>}lR4_8)v@8Nim* zH-K02^}~D@@H8@c`Xl=o=H@!M?>!I~#*x?5v+<|~7@~SJaT*F=LU0py?Ci4MZF)KY zJ@F-EJ-~Ji)~0Dp+9RD8|3_6&5-juAr|WyqB}BVR6zFhw#JSg8;(%t zcBrtB`{EQI!G2qy$*R6WWmOe(8!_t60Fwh6c?E1}Ova~t2V_FGfQ)%8 zLgwPj53jCa6kfjq2{&~y>#)C$AuQ}_5hIjqFOFJ={@k&xfkmT|I54dyk|`o$V5o4x zs$)zo84-`ce1kw6d1499cz{jnW9BT6z(Te=&_*3$&Qn zrsevOV!9kc*}twRXYj;ViYC`H=Hh<6Zep9y(L!7aHdaK4$L6Oh6=5MNUgk5CM~4cC zdHCg^gT27v{9E663`K+ilVTBw*P?$l3pJGpMUblq8#>q*mNF#nDd#qn|Nhv01%FRS z^js|&l<}taQYCnc_K502jvVX-NY%lqdXjaaHFMDGHvdpx_0Z1ZO1&qbr9ti$sqJ7s z7N>k?DhI#RK_q+&_%xlLBR90)Yh96iy3MpXoTM_OD~Qnh2Mi~QATl1186T7uE=FTq zv_x59r_RR_z!&xNnG3)EBO0*7hnn8OVb|57m$%?W6$d5x*RV3U9|w!U8hkYR6aa?i zoN=jFgRQ$>+Zgq)G!blu@}kM6VdoIQ1aPrX0WdqciPh#5u<0Qz3VqU&Zp1lY8Y49E zB5yi++E9sUxs@vuhS3VATVbWJ4QU(H+@L+zS>gS=Fo7Oe}82wB?8^;h_z{7^s>$n(<{R znDBV9KYmbgLT*BlNfS^FRR`0orL#g5oBBj>^DVI_HoZ|Rp^j~S6I)3; zJ5B&&%VQ(5#o!J==0pI5h$i0G5Yz`QO>>gi83MEpb}D@yumh$!F&M93-J^NTh>H{@ z$O9F=X8=n-bjrfm>g9n;d-v$=OsiMYL8?0zF-D}(Y<#(yRhR}vM~SmsNEKSDJCzb3 zeDC;}hguxd-^41#EkaUza-Ps8F2wRN_r~3*rcv!c~E_8?4Gr$z*gF{x|2?P0xqgB*A5S6!gAeU3uC3jD#m@- zdMoCl{(I)%XV?K(@QxjC2r#Ag%wtAYC5StL!&?b6Y|!5 zo2t1JbP}~nTgygmlLbk59S)o9nPv5n8hta7k?V48hUL&D%`>Pl!JrZU!U@w6zm)(- zv}i~r9V=Ot183%HME^>xCH5DJ_TCmosBJyH$u5-=FvbZ<9zts^YPjZ=61LisK~8J+ z@>MmaRX1HI2)d9o;8-rH30H;)F3CeSpK_+GB6F>6bUTQ8K%u3Yh$+*yne>7VwW7@rRxbf;7VK-8B>wYZd z!Jds<<1=~4`FikPO9mx7< zt31lVQKogSwGxiP{zOlR2~7v))F_aXn91LoN%y4PAst=WyiBz@7H^Ar5?9_`sBWRc zSbE(jaz@`c?dk+Sp5HvMl1u=55mD7JIFS~!Msw_wuNtI8j93ZU=#_rkIAc|#dl62E zc_73%er91hW5%p?7wph>ecRPa{vOc*wt65=kP$m^s9aJ#?U2bOkHTb6*Raba#IUw% z(Y%^(8PX0m5ckfMaeJHSqGb4*JR~jDIP0giVz~O3AOn5tnjiO^OceQK#&dDRh%U2G z;X4$s?5PxAIX9-&d-tedipxugu@Tfei)dxTmz9u+-EDo(dMM_oXlP@?mR@RRCi*0Y zEsbZ)6;nE&(px^gOm;7c+XGObS?8p6RiEBY4(J`-L(TJ;1t_h+IuUu3_UYw~*cr&l zk;|g>3@@np9Gz4@2rucac6cB$Tp8kTTa*5oz6tYqDgia$H5WyHqEet1%T$8ME+hEI zNK24mdo6kq%se|$u42@SqQbm+Mjd-M=Z?1X^WjljqQ=brQJ%6voApiQ)1rc-Qfi`# zW)vpbY%NLk*hcD`_B-qP;mT^(kONuUew%w(8@-I>lQ;EK^c(3?0Mq;CDFm~dv6?rs zdBc1Zm-D&2!$SfCwP)enLItokkYh|QUw-Hiz7$6ej_spk>}S_!y{N$(w&p?we5N~C zN#t|l@aO19!b!)Y%6o1jE**6^6geOYJV8HM0j#bUBkNl>i$I}HzS*k~0$sX~DbY(6 zEMK$Oo>FF<&)vrMTX^9OeH6}gs|J%67Teyu1T^s!akh~kgy>CS$(elYLNU2alDR)w zF8?pIpinHnVIgbXA=A8K!QyFRrn}93j&mUfo4nmC|?u}&>X}rZjlinKviReK( z8O_|9X&&K1W}g)00x44^N$RJKH*_Dr6i8_W7rW@TToQYsas`lrlz86_hAozL7>Fs8 z52m8I5k^=c)4?Hh(w>x!nz*Q)EwV^Jlm*jw>+FY*H{=>F$cGzfEeUl*79x?*X%p!@;0Ej1aEZhR zd(T3uJ2adKG#l#_xQM2S8W!bF=3QRZF-J$Pl046)Sov#oqwVTsPsQt{EJfW=Vj8Z! zH))mEbcd%ymxVkbEO3o16C=sM*2ZB(IA-BzBy*qHP4xVObx!D>Y|H32;iGrLJte9H zxilrHHSQ|wDOO+GD8U$#PDM~9-OXYx(nOV2lG8`X1*h=OzJc@W?%%Dk&x_G#pWuu<5cVecGKPS zyff&hiMLi*uD=NXqUUr98l!N<*b5F~t!8yUoC*iTT)XlG#SfxtS7C7oEFLF-Vzf(!r_K-$>~1qsC0thZdi;f zS@mxWaS;kRybayb#}as;z*=4d;gi?4F9Q7}?Pw!_>XvCa{Mh}qX{9g}vI?mx@|4%0 zBI^Wjj}rK9-e%X5V#^a2#mnQ*`J$-Q6TRQ6Z`SGtt>>m+2D*jgufKvHP>Un58_y@@WxJo1R?()it{};iB zAhuIY*)@`rOm79b*CG__s_98P-D8B(HS3bxlUpT$PB86-4t%W?pOL2vQ^O63qmF< z8vg9OLGigfsRE?1$@#7RIRa!5>|spC`GlF9|3kaVxQ^uW0AP=PIcd=b=lr>=<#}%+ zgsj-M&$yyY1!MCS+s=j6&kR7ndB$l?`niQYOiso(dp@u#!Lfi~G`p7#d@&-&F3qsi9j%FF)aOVv;u9Qj1|x=JH_h7lJ3%Ro#0u)ytM&@&)hD z2gA>h-$?}Iqjry+wg5X;=?cA`QAMUx>T{1sDz&r3UxZre>Hacg73=%mlD|DkdWMnL zv;Xe7{h@^cKOavsj1q%#ikSfqAR_t4i{nz6#b@N1(uYrjFlsyFPx6owJF1=~#Y9wC zk5F>B#k2PtN^tuOXpdxRNRH-Rxz#g{Vxf5{Zygbnt8yV5iL4Irq>9PiJAI4S6#k`wbLd3 z$}hw%{rM`)zvm0N5VK2Z8a7*vMowIp;YHWA0)>&jEM6w zl)JpSz_^-`H0PYfvi+k)>$9g$oePI`4wxiEk;OD_GL08d-)t}i*Q$Qv&_W^|Je=<^ zB?u+QzEJLtC(ZA{EK_;!$CN;hsfdl6o;ot1R zXy8~NgJVaYu_ePXQQ#tv+Bd%4N!cg8OOA}-yPGaC)zm6O(Awh|K(TJCyhY=eF|}vC zf^!#d>fGXYfjHjiCO(C0pT-&4S#V#9FvVY16H9V$bWIK03gZ#tX@_LA4i?=6;;;_t z+XgGQDRFzGpX~nkP|qh}9XA!BIn>8Jj>?5t4b#aj_^`iRO#A6(n-oa(Ztt|ok{QIJ z(xv2=@zH*!8}r2__?=tPno&)Sm_k*jX-_2S;#XXu8FPl^1*W(afc`SC+8%R%=1`mT zk=Hc?jDGD$gd@H%Es#NoJ5|@jn0#+}B{|ZW=)C>H^440-i+h7hgB#;lo{)OSX8O+X z$_5&q4S*}-55Eh{{l)Uyiuis#03KmA?yX_PGZ@;K<)&+*?WMH`{Vp){2Dz}SCIRdTaC_&bfVGZxD>xX0I#}1GdwQ4}fXKa9L`cC8ggJawY z0q+j!`3;c#$bj;D^eJBHU5fz};vLY~S!iJO6`eg--B*5R^AZ?XQhq1eDg$(;>^-Z6 z!_qjTxq9*T%^Dqw;-;CKztt}ZQWxMEJyLi)gPGggX*OIgAp&MLA8Vb-%%>^IbgXl~810;0f6?|I|agE5&Ph_n@)YJ$TQ3D88V<*G_Z!7SqVfn#1`3ju) zDoS=MBm0Ia`bKwnn*D=q^Gzc~b%9-1=`vWqjX2@1K_%wdNT;dlg`+8` z*#x6&7-mF2Xp1~epzr33b7}GnuLm0Q86cuTW+5&_F1T-~2NoLAY z`gG?1^}5H6it5y-@?L-2I@K?XKVcx~jBrW=YbUeFp>#d%v?FYs{Q=?DcX{SG^5pOL zrmD{q>F8}Ua)jr`9wRwjBB#lRpjxsqc{~n5N#KQZviXITjFKrxwcv z2q@^NRUx)Jb4Wq&wBn?1cEz3k@u_7(79A}eI{4Tx?@iYf5Um{nMhhf(yjOOJQ~%WM z-IzDny2-#PZ}5B@)j3>y)i|@zCgk4#w)uv=*l3=~6>9HBl3N`nGQsFo-rfNgtu@lV zHM#hp9h?IiJAne^6KmP+SaGA!{x-)*p1*f~;w$$7tB~gVT`=}dTpgmfhF4SmRw%bW zNGyw>wI7&`dE85+ZHEjVs|b-_#aee!!@>r_);f1L+rcL;h4OZ{J(qOlFASdnFZs*K zyMtFw-n?!v6j5(-?56p3u}-K9w;(%zi(iuG^OnxZJ9KpH37HaLn*CQFfT3a>$|FGR zH^St3{tu3XPsf3ER~v2yZjoi%yo|MG#)XTP+V#YfU%Jac&dE>1- zBz8f|Z7{s;D?rvj5-tFHFtG~^o>jLRrl4=MEIA_{1N0{a%e&gmUrQHFIs_pC*3k>& zlz{lW+>Q({GctDYR}wTUcpLEaqTcy<>rP#8rkF*YS~vJw^zGcC#ctZB9vP*io+6JUK3qjor0iG5T*S$ zsPFb;hos-)UX?>zz2;vP-;Du|&8ovOE@2D(GVE&-0D{WN##j;HXn2DpJeec2FRTv~ zE%ska{Ln&66jbJ68VXIp=G18GMA!Nlr63U4K$ExaMbUqIXM66*eS={g}L-|5ZwHnOEZ ziRyGNv_Cl8@8Ip>KVmEQIj8R40oF&n3ni~o?eIOk{XR<=?fffSq*xMMdLG2pt?4Ni zuvxn(Vp1@ZKE5hIg<7FdFAo%(4ubG6=c;>txVV725!J={2q<4~a(2|mOPX*Wo#7M! z7yD%^^TL`D9)Rw&tq3_&`u{2mQ-2*MNO&hroz*qnQBkjHuxnTW?h0esH^??;Y?Z-{+}{5q^UG}ZL)PnglN)rx>Q z(Cz+&?X7$C)?DH)`cUIQDy%KKPFz|*tOp{z2=0KDrtzjuGJi+>ziFhk-@LzWqX(Gw z$h7io*j;vAo7%SJrbJJH)Qroz;rH)M0a*q&zuX724?6j@+Mg`Lea03bbn? zYPj92iBgNI=r1nKBnvqC#k7yh9HTaUcJR#sL^^vScpXEFGxuXG8^*eYV@O{M&WU{T zftoXY!r`AhT;p#9e8vBa=5vV5k6ItbQ;_Vai z5VDhE?h|uJ%BUz52~C?Zg#T8|H3}WgQbbxTs;*bYrj$Z17Wyw1lgNiqB<-UJ zeWr7+&ryO}FTZxFti~lzeO!0`Cp$O!4n}DRKeDJi{iHOhQ+CNVsXg*Sk=mxuwM~5h z)*Smr?tbtg6Z{K>A_29mGRLjrSj?IGSV(LJR3fNzE)`&# zvqeBhtIl)OuRU#HFgzy@aY$=(Ye)wWFA68;^tdE^;=&5LpUYJ zIw#=7sFiPBgmTVOuF$f8`Xpzr>>`W&B1|UHsI4C@tN$rTofzb-T;tml)GXV~$CoUd z9vAlsu@(!y|3**nPshs#)0xz5kU@7)X5k*fD-ZgV=G(}d^+!bKD^bCW=6d{Q>(k>9 z9)3gVL9F#zpQ;&+-^=c1QEW>F=gH0Viyxx|C!hQrt?`?VS~*8X0zLW8hfp`J`<+n$ z^`c_IdAxQKP7FsiIEf*$0sXlbN@T-KKlaBTFQp8dXrJoOOy=iu&Tx0Y9eq+Xa=UuaS}G-6aMfd_&r@WQdfWK^)zhzDK05 z#oYAlmPB&Ovr$9p0sKqy^Q^5NydJVxL{$+PrAQK*)%r7jp#XsAto56-kQ)JqZ70vu z1Qp`M8atgpn=;NO-nw+)EYNMHO7j?i9bp71>Q3gHiG+$r>8D#H%7P9{M9Qn1O%^u| zW3BtMbE9<+^xa~!AitJw`}rqP1cqAYVyNdz^}_zm(ly!OSE>|$58Ef+_|=u21)eLe zCaB2IR*5??E`r_TO&VF>2+)DQwhJM3NXD4eZKw))8$ceRNs!N|iUf(EtX35VwW=6Z ziV5DEhYXpQyjCbfF9xlnmdAN-m3d^LNY^y6G+PHKh7MS=i!ah z)0QBF>;>5sBaVz=0SQD=OwWBgF9SR9vyumRNqgEdp*bZON(slt*~OZl9EYz@LY#v$ zjc&FIeBa!Tya8p6GF@Fm@3ptBf}uQotURTxSSC_-os$(x6R9s8U*$XT8g+$uHqzY} z5O0EX-Ny%c`=8cL6(&!XX?g%rQJmxxtr9m3L+%naQt#r=NW@3BNyAURp2W*GJyN>T z+gJgH=v0+<+=#2jiKisSGr@p*eQZDx-K16x^wY)$ms1FzOE`wLQ1c$y+LtYBY8%$H zG}3vv1o*&r;z0wZWjreZ^$}WH(~@ECAutK9KDfu!7Zw@hvzXE=T6zYmDFuund}bLs4Q@9i(_z?wgESG?cOi$2Tn!XN=jr z5Rh$EQs~WTJq{@(4V7`^u>iw&K9|lYI7CcLx`>qf>IkoeF{fF9KQb}jXvuM_%Q0ml}Qqb25I&x6~vOy)*Y@4h{qY-35Fty;M>pBi?u zgdL1$ivHG&s6003Hlg|+_+xW!Ok-i6=E(MHJ3!O`AK z-^ua+xog$jJQJVNet6QxCJpbw!~}e8g9XPx%TyLYVq-xdU?Gix$njtkCNm6m8Gd`I zu(m~&&Xt-lF<_kU2gR7ueTmrLTi@Zayqj8cn{@-1kI;m+H}H$YjAhh;4UG!ySp>U;4TU7I=BUQcXt?Ikl;4B z1$T$AsoGEb?$-W;bE-~PbzP_XzWnT{-*@=o^an^qAc7JD&g{MaBogTkSp4?XpXXx? z_q&qlGkDdpnHtx&2ooe92{WQ$f~wYIn>17ng?Im~C5quyv1R^7)cc~ymxz>392N|9 z;k`14wj0N@+9aTS56;Zvo5 zH_^G)PA(B=U47z`lz-TQ#E2HWs^(D;@NROvRoLltg*WZ6gMJ=nr;q^So+keQxtQ9k zy$f4TEjC2CNCp}byRJ| zj;tF@>XJL!6WI>?1~X4QKInGyZC+r6QRN1w(!=~b$k*s<8;G$s{4)w4{Eh=cYwnk+ zjLf`l(WJ1fjhA%P;j*F#OQUbmXBeHD@VqIZk zFZ2r^P)^7Hf%+7`u8NE0z;y*^o;M!iZTs)=rq8RNJBQnR;d|hJq|q5L`nD1aXSXs_ zi?gy$M7UF?EE$5-30x0mlE!LsbC=G{92Q-%G>E!o(dF^ze;6tNS)aLEFcLwbpB|6C0~C&&sqyAnSUKtjUdjswvFN zUWR5<1Ha)6M@65t%ide#T!jufcwS6;n=ggC-6hfMRToD0pH~)cvK;0Z_}?P~b_e6i znk>u32&MmQ+^e*e`nVQ+Z|JLlTbtN|6R(#ZXY5pKC4F(-XMa77p9p}OU8EIT5Q0g; z=QWRAv;Q!55S4Z=PpR4A6zKFhY9UMI`dYrj{bIYb#qz{dvou6R?zdo%+ag$UpQmC2 z|EX`^+t}NsAbDuk&7hx4q~WN-6^A#&)ZP-={I3hz4P8)gPb$+=F7dB< z1AGg1q9VK5@V#M9w~@VpWG!E+UuE;$;R?%m3NP*-8r3I`VZWk zu@={w`)YgNI*(TX7xS@>IVNK=vng-b;ZM$aRS3P-+a}ykt}M*2+~MBk@o!Z@KNP~O z>=(%6Ypmx8PiG6r37;A|>H7S-Q;gqm#Ju)wLj6x{nemmP+me+2>+A0WX#_Wv%69j* zeeJ!3&43mLU6lrzRJP@wo!xIQ!vc~XK1Tc3pD?o6m++BC!IomK)U_iKN#*IQWc zwLkpa73?9SrTVAm0m%BxbKL_W7>xQ#7WM%#2;{SPy=dpXFWQ0rz2S$Yo@}-DKs#vl z(!YE4Rq(i}hXc92vMD!%sNgRK@tY6WhZ!?{iQ!>*v0J3aaxqVNh0iU%s3CPsVa~sW zhaO=_4pL@q;9j0Ly+GeC%=iwH972xN^N!_YX;hkK?9d*vj5BF8pqxJ3BIhyJ6C5-; zx+Djl4>Cou+POWV1jvB%{wEVKi&iO-dcGTuki|p%@sqoRwX7K@%xJuQe)Zk~bCYGf zf%W_`W6NtHCY%?IK8NxynbvY*2WLVCG4B?*nTWZsI)zu*Bm?%SJ~k#Hu|ph>&^0S( z8k70-d;t=Q6rX?ik$W32U?d z2sdih+ZMt&9%nerQ~&y++2D_kKSC`PFdke|K6G~X2%apI)xk_S#u$nDlU!(8J+_d+ z8XX#AE~lYw1Icu4HJC13rYm^m;QJxb7_*AOc?Px5rX}Km*u5;U# zqS-^3REHo;A}Jc{Xsc|2b^z*ey6`o!N_%)|nahgP$ZPWyhigQXt<2V;^!RtF{Py7@ z={%yyVU}lUs#R)t@rOv{LF#p_8q)ByVVZY=$yh+)o_Vre2ZS(o;Zn)Z!TsLMr+OyQ zs6xokL904Gk`Rl02hnesx{H3UpG*tw?zd(}9DkEtRkU9NWqPrlgI^q7J6T8Ur>Z$Z z9Rk;qXdd(M!x*xTEUwlH(_b9>`dXR$YYG#jZ2K+3vxpl`V>SojIIp9gGV6ZSpKocz zv+m+m68gpyv+up*{I@Rt_}Ui;rP=ueBn2MkR)xIW>h6SURz8gS<5i>oBv1Go3Qp7c zk`y7?v?Un)ow8E7sQdOhX4;3aHOexCnB;n!}N-#TAn-qyJoC4QxGvJ9> z*NEh9fvTU~+GQ|r8a~C;U^=(09*ZzP;fQi6BougiS3pj3tXMgcu~mcBMVnN^U8nU2 zTS3$c9@(**#Z+@9S^f0dvZ-#<5=<2f*Y?pROuE^}$lDn$#8+7!=vR1R!0FtHGBDXp z<1c}SCz`;&gEN+X#ES4Y75R@ zvdes=z6gxVLfjv;VnX$5hAUKhD@<_y)5B7n2!VQ`%6iBeOtZ8{)C~BhW!2wxmkLyt zsF@7v4Fns~)z3jdLn=x`CN)E#xSbRhOJvbju$-8uMZ#cgP0xx9D`yD?{vK8w=J7~L zd+cwY(2KsD8by@Qa|`iE%6Nu)Cfpy>uMW5RrIX5fYmJ3_lG^U3mD%T15) z`qy+Y&(@=}HLx4!IgdWm!avQSa*P*)dMrHn2A`T~WC5dTHG~FoQ_UV>C7B^r$yq-4 z%ghGk_`ehwL%zQUOq(%`L2vN4jc=%O5&-^M)`Cn%aGBbnL9YGD8WJp~5xV{84Wyot z=WP;bvu4KL?Yfb)gFu&xsn?e|xv-!~C&pdGW z5x#tNCY1~g_*LbO-wyf??;xPLWEY6OGShW-^z;3#@!enbSEpl1SwGj$oMLLikooxr z5GY;&%@LT_UK&SjjqIm~%f5%K;R6K^9+Ba|5gFoIt_wQD?jn4%i0shwLf8L)% zouE|H66|1NpxNulxvE`2o4PlaiY$<`CwK+v#9bX)5{a zp6qa08G1#AHIQyxpMriqwkD-)fvi28K>)O&^hisT!v>W_B{^c5Oyogo}iPT~KFu zj=wm*`5P46W`{W6zOK?XVR@VBG4`2cd4v+>NKtK!g4)cw?nwyoe^B&wE9|~g+q`FD zzLRP11h|3`Y%P9~lNV&;kWfXiw6@E-5VktNR^12Kv7FTniOrr6LIJp}AOD@4ERs)y zIu)8~UWZa%7DL9|$;|fdL$0o_hp49ytR$YsKY4z^bK+(lf7P;vaT1uWcEuCC{mT{P zg|&vhotLC>T*AreDL^`q!c5u<2e*+g{LsT(^d#^>2F3XKxY+g3rUvs-+Xz zTRLN&M21>cu(>v1G;v$@IMK2>Rj?(KcaaW96$t}6SY_nO9(G|P9&_AXb2#yDG4WWJ ziK|{7xgqLmjk4)W`f7X5&kps&9U{Gc7$67GM$M}&Wk*3hNL;`kc2PKo_joAug!v^w z9Lfj`W(YAo*TmaRY5bw!=EXlZUWcIISIYVFN5vydaQ$4FL6TaW;gZ~QWAen3DcNPXTLwBy zy6?i6-NH?tDs42&W@THdss&Eeq0FswCyOa0DDy=u;ToE;vqvnTgq(8HBNy(ypmv;6 zIoX&Zs5dR2{LwSC9I%?m;=rshw)^>P8S;@es7T9{E>(2_v%Eu7wshpfF{`{9pK zA&EC)@sM?G(p-q{Er03pzf;hdFy*Ci<2hx=Zu!Tgjq4rREt!#ysnlnAC(}Fy0YZ3@QR>EPHnNG+ zYvYv+p&s4AZA}z^cHzj<_&vvg#wUJJ(*f=AGsFkxP^$D@-i|DKV~IDTnAd$l|G1g5 zeP3Z#z8#-u663W0yp1mU6AqNS8NJjhzizX35MPeXPKj%LEIbL5gAKXgk8qyYq$&Ik7Kc_Vk)MHVmvpBmq+HtU`N81ec+gU zx@0Y+mdayHg}&x)#CJ#j=Wh`oJyh|Q#=y+_V|L;adVmCw9}7DfQU3=OC>^uhEGzJ< zIBOKWdDScUs+muOUPv@&AWNUQ@vM6z{cC@Cd}1D#lejtloKuB3kseRu4}(k&!^>=v zLDP9tVgnoBuE5?_Av;f??1F18nyqNlczk-}E;tWkJ$Az{@*_oV5qm7nB4gyPQ8G`m zLwMx(tKf9X(Qk$=bGRp%=RZ=V?ps!C4_Ls1Tl8l#_W!t_FJmM(2gaN`*_Fn>S=TLar?BZBKFtdAvLG4hY~hKs z04akx>vy^A^MWb<_Jp*FbcN@sbcm1mAw;Vvwv5JW3X4Y<-p59K&9tODib6~#yjxtg zXZ-4kc4$wB=DbanwQnHIADGAC)eFvl-96?1aI1lxgVb0pAe zAk4DtG<{~eJK@;wJCJ5wF;fW$nZ@_Cj;C4$Oh9kQdY(nYUVjb-L_XilBB~o@^b+Ff z10tPA3_HRC$9}RCb%3BRKmJd=WD$ggQjh%U(#OS$a0KI^Exz+@^p7-$HG7mBdBA%2UH5@LoNCR*T?o#0ixxX z=w=FuPX#dYpE1=i$$xfBrrRIS!M%VF%%5}JE*0l2lLs-Hv@Zl6^8z0B*0(l~CLzCs zpHY9#U*mt3hgFulTR2x9-t~ymBwX^4}ar$!^b!;ueoi+ytQzI>$wT8n3y4iUCqgQHvU4%n#n*m3@H||c) z8WdLZ3t5s%-R!Fu0ptg{Ec)EY$wTVSz0k#SN>SYq;K(}%F#p9pm&;LAE=hwu z^+&{4kQ%O%{`Xcb`0+R8$4QS%kB(hacgBz=(^GY?rqTY}(qI<38^>0R4-CEXSxq<` zzf}I@!mD6r!nC)__XCk#`Xv;eP*pi}y_EfMbc>?Qv#6}sH^ic)7^0v`3%DvXOuYF| zsddpjjHfX45z$hASomICh)#O=jVSmF62rObLo4L;VcBvc&+X-BdtC#v-QN|_6cRi) zl5Y1LxP50S*u8RYJbUU)+BkF1I~NrkJkTJ{=oQvCJ@XTHmV!kUDVX@W9GIUqgX42I z@x-R3K={nCha?iZL-O39hqOFB@F(>bq9OP1l&KP|i`m7si$qI5rHsTg)+a4>I7yYH zUesa0n8!)G(TNMkwbDqcG2t_N!k?!|2q?Sy=YE&vU-E}>QUu)-@#e>Q$C+N>Odfn( z1_Lsi&5PZR>}_R2ze#&pxi*#7Ml}Oi8DA%T0cx`6EQ>g}m6B0*7V9_=UE(sZ_WDy59xtzhXY4 z`3;nSEF;c%?l;Q;^wM+{TuI#L)I6F!>PtH%M7`)Ytr3#W?q;`=lUP%a#`e`?}rEem#Ftr9i>A^+P6}v8I zLpNtPR;Ydrs4BZM#=;|;8vA(VI{;zoZRI}=fpeEhmr#r;!mUL*%!t>$kVn$)2m{h_TZ^+wZ2w{%6@6I}*!`Sgo|2b34 z{4U!iKuJB#tA0hvui*~FGjnpJLV6=^d?#@0c6cBr1iMu!Fww{O`GdV#29tj{9g759 zCc~N=nRybU$x&5b6JKQL5kU^VwrI=AZ?$dx6s#%!&X1IuBADrhuVbyiLo3vNQF5rs zA5nQwIN)&;GLg;qWHRvM__@RmAyI4X=TR!D-<@Eis7*jLSDMhNM(+Lb^dl&zVd=Sd z*k_CFV2Gz>h`(h(3nbxz?h&$5+3R0BKtXwgI%B4P)`x`OY*OXA+bA}g~jhw32^2`CQ!(7lke;a==#>VN{EnqEowuOTm

<-q(rCeB2*H(*CiCe?!n+ zh+1PxRw&bUsAeZ`eb79knMoF87E z7k{c*t>EzbtSjJVelQuYHT|MhT>`ZNpQPgh%icEgW2>Wdr%PWl z66sI9Yxol$BM>k8>CWiWFAqh0WTl8(MOYz z-vdYIE_7$tDGF!_f*}I3@szrYXuq=a&n_~$zY$;hEhcVQ{QAD*+j)}qO4iU%Y-yo? z6(+Ia%AVUjz57bq1BrymeReGeDEfX}af?^#}2W-}LBkw-4P^arih8v0Wl(vY62@@z(u?c*UADz&I zulUi&dQVV@DbAeMxXQ=dH>9z_8ximjt?B@a}n|b2fn2nU4Yca zFmMyenJC0kE+sA7=A%}E@-)WM)n;yyFGu`ek!;AQ`kD!Ds*MxO&+!_}xVObbh_3MY zEE}2!N+@14gU^maW9|DWG=og82cAy_-gvJf)K=h(BJfEyQ@JAECM)DF1%&FUn2bVM5~)^6mr_B z(G!r8SoY|j23bGS?0jtMIK0X`WmBxO1f)Xn11+`E;~A|vvX#1{fzu_~x9mDBs%%=T zC&(khCTe*)N`o)-bVZ@J7^$UV|MOq9fp!xI1pDdJ5aR!*@znqTZ5*urXX6{EwV^?* zh4ZnIY&q0ek~lY~$xy)R2$OO*--=Yi`dOYkEX~xRBKp;Z7!R~j-?d{QP40g+j&DPf zd(1z?jM3XH2-42IRwMmj0K4&Fs={Pso^ATg2=uv4U*`Bd%=88R#qGy+kmVm_H%$RS z!b|DVlomZMI>)0L3l5a3ba$~!PX$V~@7qH^Nq`$WE9n#}wG^lAAcl_w`mRkF1j_q= zKQpe5Sn}H7$3W${LS^og0hyZwjHB3a;onAi+hci2QuEn!ppMeo8kHBT)8WCMBo>Ic z9WoK9*WN&|TWN78P;hp^f|)=N7Z%f;KaGo3RD;zXnyJ%*WfQsYV8n{63I0b%&1@Gi zRHadrnf`pw2);{GOy4oT_`x9oXk8w;^~@8hf|@zsrEf6~8L z@b~;^MxNd!k;op>!K)yP@m;-&$xP??8H2^fen*N`6U>i=T4O0%qaSXX`>Ap8Ju%qK zkP05ITj8o!=Rxu2OQa1p9HE?Hl5-}dCxI1m?JiN6oG4xMYalvm&$YEM&oCe;P-0-6 ziq|Z?0fHORnC61Qh}1?{_)5eqYmu^J`~H+ zpn+->e*Ut}5qE00c);=MqbLnOzYA|!Vd81h-WedMhPY>B&Lq^e@y$$_WMW5z#pE3db{|=g4=c26UqJPoQKOz~{yD~+0rumT74Bjy-rn<5nsf9fSD_U) zksi^l$Q=T?!bMhY*$vHN0r)>dcMRTl3?aF67kSHN)wzB zmk-Hr?vB~C$E?Kz1Izr^CLrLJWCio@ixdp(5x|^p%+lxNJ4AwqP!<}&Rg&tRNdg5p zT_%41H=*J*R@ot*oH{e-@^H-ScH^NbO;ePb3RUhDzCMlyNn963wj(FiWm$*8*`_62As7B7#@P$ z4Q6w4Z@DMO;RW_IBA;nY+a8gQHD>SDF5HGA4_u$FMzv+?f;J4 zsnJ97PB~}^oUx?i%6x0I{#nmZMT7kvhZe>RGtr72mIe_HCxb?8yx)>;QZd-2&Advx zq^1x}zm1KNwFUS5OY}0^gR4Q;=0j`SvVGm7=T@6vm#x9EU+)iVPEaiJBOs6dx$nc^ zp8)tV=Qi*5-RJsVMGRxx{J9yQZJ@Yyy0gcTwZ_Adb!SOiD6_h&u&Jix`5VpR+D@1O zyPZ=zy&*mtc4;Lbkbv-A7L11f7|GIK5@2Z%KOv84sQWha?T6af>K#Ygn&`cNu*3iTp7b;3{N&5 zj8L*hgwF?{45pLj!S;TdmY8$Z?|cKEFHVj54VK>&)?0%y_A zM6!XUEP1!#{-PEvCv0Cy{r#+zyEn=sGASK-Bv#H^3r$%`GO{PLeU31G*rH3{8yNh@ z#KWj+qeMjRYbC73)DG2k=kIW6B)<4}Gx+4!B9AE@p2P6BjF1lF1?1A+MNhmF=cN2< z>Ky2orfff~(fD<(L1C2iZS|}SX?dSpvCIq6xJGIAcRP0yTbfGfF%) zA{~vHN>z_NReZ-Q^M-uRvcg=4lHhoaaC5)iz$~kZFst?)#A2)-!VxVME}8l_mMaH; ziDxuQkv|TKbMv*$&S6w8#x$!07;QC(8nfC4TDp#AXx7(Pgk0!^=z6+kLqU^Q+CoG< z!AB~R`kflB(ZJ^V%}6?&Y;^8*qaDc=!c|i<>RPN6xBcj}gHf+PDq4$D!c1JyZ5giG z2lFP=?`mxZtBP^0rp%8TalGJcjU8PBg-bnOa?I6mQe%giwnG-O+yqOAz7%dE#^9vvIK57zqk# zWFoPF(AiRek2Q9+2ns`M-}NCEv5rF{Xk%70#J67*Ma>xR^%%)ZAGYX5D%pV56jethh{)hsk|?(p-0tbzf})hSw#C%7T2&RcEfDP>vZm`i@81vpDF;G`p2Ytn~-j=KRI$_hQbFgKYC^6mC!4j-|Xf z-~w1;Fj-DCAy8n(Qrx%+*inYkmNQtxi@&d$ib*<%{$}3W;E2Aa)!31oAgP@s)Xk&7 zjw9+!si?bZ>hn{pWB}LNXOg`$jlVe!I zj83kQ95;hTRZqDC$(ygfY|7N|4@9kyASwEBxvs9{)<+#+3h#~#(G>FK+U~rcz~n92 z3Ipu9MXh&Sgm#xK;U?6g@Nh`h@%!r~61?1$EmTP#bRqRlAEhsd}<2f4h{kh(A0fop?EsRmd;`}v`~<|_ct+@G8zchBHL_d5ZUkK#XZrnW>_N*;~U zaworSSp2nDQ!i5WD^}E@N7ry)jTz*hnPUxfpKQ>2XDi|*-@XO1e*3S4DXV{nqyE6dU%RdWt*yS_B`FUxP`bUx=}GSM!R@ zNJ{c*c0Ya9P7m{=$J#Jr>|(7;d%bi>vN_z>t7FMR!{JO?IuofjT01*^v+c)VeM1zR zPP9`gWQ2~(;c7jTtxXEjEN$jD{jl4y(GU05J9UhuS6_QXLFt>EbyXQW&I>UyctQ(2 zmAo01?1m^8Q=NRrMn35qNP;I1RH5i>hemu@*#GTTIC+1f{(Cm^t6aT-2hVuKUilbw zOe4fvPE`APl~;%$9NlNv{#S{;D>Bhst%_bm!jJJ2Q3>Ir8e5zfc7)?DmA!0Kerx2g zstZkyKRua(yjCrzGZ$BJ?$;$7xlHM*k^=G>w8NFA=Rj1PY__Veo3T`tv6`RjJrNJJ-jjy-PiXtVO$&eH{>8m`@5yPZu zb31cgGGpXW7$v#mM_TAQJ&&5@88sperELe6;8-e4mwi? zAImaE0D0OIT+b2wkzSCGXCii;WddH%39yPqFBe>v`69ifk2QBEWaX{T(oSCeim-v9 zILD&YS{Lhe|68f!^Ae4Hz9Cj>>6r5oh;yJlw$Y z?7L>HOOdp*kOUTdQo|NOvrC2`Yg)~~d zaH@H?VU7u?FKEN&@GIa7^7;5aT6$T4wRzuHOWgh;;~Yc`=pRtg67A4qo-D+xtbCZL z3BHyVJWG(WqXy?*pi#VXH6P9IS zy+NBxev~6uS+{O;Z$rVun|-d1&S3S`@RJnT!&KP(njpB^^_%OkhldhJFg!4TF-B%x zkweCp&E_>4cU9h;W(%3GTuA0QxaMfIS?U7dl+r1xsmjPJZT@=3OYaq++BeHHCc`4Y z)V=G&g-Mg)gO^m)#UNv!Mj~GzZyMR!<>uB=~%H@p}pihQO>%=vg98# zu62m_qL;{(ABK`ra(#E9ws;vtvew5I?4wb2lge<#S>Mp7g;)VC9)&!$b|3WBk|(hJYb7}3Me(qo&HZo$0!!Z+_z zU`oUjkU_M6yPFKkxZEviEwKEg5N|rcI9Zwo5gTg24Y)!;sIMe97F}vQnh=fME{dwl zp7UPM=M?|%JF9p{B-2z8OnL!4O2sQxFY6AjzraWBuzvM~27hSoiq$tmkjBbX(yH_Z z!#X_aMSE8Roo=%9`pb4sU6spF$TmXZT?l}bN4 z*Jk)CR zEukLt1)C#Xv&w@+bYqUNi_+9<9aQG=MD94#;(oLjrGCfCl-)6`_&TuzBJJsZo9P^3 z^KOwMR^DEpQAgb*l?t{?ebOEc*_2NHE8{RG2at%y-E!7&zmW9lWe7Z7aFZSEjeqtp zuXwTefN1* zNdw`kqYR^ycDr{k7=cXm*8I0)XBof?eG5-4` zdTu)?ZWtmIxoxx}WBGtxksUcH-kr`M^7GY&;BqL$JB8z(04x?Aag8J#tg`d&!z+6P zJqP5B_I*+MY9lZPZlA#ipm|NhPiL(4*KNrILcS6%2hQ^E)-g(1wK8N~fRVJOG#@|p zLL;i6;O3dVJ0Ha@tg^a07sZKtpOthYF50^DyQ9Mt5=cQya5a41RD{J{LYZpN-rcCp zF)=`MheqOEApBn%+4xZdk%_5uU!>9#vdA^D@H1Zn@zyv>ZyGAmnmX!mbFZ?Yp#W-5 zgM`ELy@3{bh!err9C6rg|4WvN$Auhezc{2ZnTDp~-e&SsdzcbJHleK1z9m1Hk^lmc zma`xkHRYqHw!ro_qEQs@#uR)C7=;&hCW0$=?G13N>r+Wc2^NEgq?pSg<6yx6`>)z- zk(*UK_ycCRffwrD_l$4vDQ~xK({H$JboP_r%l6$z!)tnn;VQ#NBAvf+JO1HjZyIMH ztpBFeL5hNIzugb>k0-ws)qHTos|WWy^~!y-U{uI0 zArKk*|1HMABNtniB?dQr*iBxJzBv*7Am}&25r1TKSl1?YK1KXtB1YLbrg4l^I+Ba$ zR6hs8`+=5C9ju<5HaqYcN%8O(Qb2|yJbl?gu&JYdDZ+m_9tE!}R0%MKL*vv;B3Wj} zRNaG;voXZw=6m)KhDf)L4hENLpJ-QCqw(%10h)J+fWh`x%&QbXWRmYSF3MH^iATdak5VSr@woHczq1*c zTrF`|Y%1|xfH{Lf_u%Hq%b^T`leCZ=-Rwsq=PoZHs?L)Q1QDz&Sg!aNXwDj_@u$Y2 zKPP_Q?H{5U?ZZdn;t}bP_i~`ti`-@B?ByUDu3J$ zA7zTowX6IfL#w>d8C^ujti*sX;}{VBesmq(z!H%K;%h7y;6y*LL>*A+ySI`b=9fuf zvQ{}8x{Ff8IN$+v1T#{y&w>JcPvfi z%`vmxk1uRE$RHUG6Ed(>w+pIuYFGz>F@p}LO3bla`FtI@CkgF+dwFx? z9rr~A%hY~iTl#|E>#UX@(G?yY=4Cxip;1wDD^huJ)m&qTw?5-cY3f^t(`m$v9G5YY zH`n^VY61qU4%+!Jp6*|ATwmB=-XNo-8cfZ*OeiV#C)%h|&CEUhqN2>p?@=yP&Imv7 zVM(1a@9>W=2q3cGk6B?cvCeyDdVnm;MsE$)+&Dtnw#-%n@V#8mC2m~F2pnlm~U zjq76CAe7iFytn9~%PN&BNlBAcn4xk~&8SlR+}nnsLnbq#+hJA>b(aUO@@fNxDBE<> zN)O_jp#s_$a*zN+FB|<$H+3>+37PnPWTi%V$E8GEanf4^<{p60O6*ei@4D!Q#$|g6 zG$n+k6Z_Msi3#S&rcSb6Dq2#V${@cSm|SR4c`KX93Bm-)3&H- zeo)-LB~{=!UFxSnN|M>Tjr(@l|oLMuy&G1X519qm}kbj41z>N(#DF(cw&m>-&6=0!-|W+Z10GO+Nu6aNM4-;(+1)vn*Km_0p< zCQ<1Z-!gjghh1#xw8pkuZUSa)MR&IfSaBJ3(rs!tPfK^Za#URhEz787c5H$z4#Z@Gjs{^>0(rwtWpGg!tVttTyq0A8zZ4kkWZm3ViDD9 z4_^Q0P|i{aZuC0ir%#VQpFYw3zxbV$tE-c%q?I)Q2yh2D0sl9|3-d?lHk7*mxPaiI^Vuj{Y-o;1${2968cS(Ax(t!WPPo^_+ zWy+6(?knQQWoD+(qtIjT+rcg6xYu-Mpmnd!;pB!8+3SR3s{8?1v!(H}!iG)w zeF3DR=Y28`Q4z2jS+)5Zt5YXx@8LG0pk1rqVSS)i)mr_>&I&xA{;SJ>hY+IE<8Jo2 zKd+P1SgOCu1dQEPc49ZG(KJ|`@QiEvw*X-MT*;#Wnc=sQ3MEpm(e?~h{*f3Df6NYu zOn5}2&xA>Gp0^tF&f zPemrqcgs?+Nq-eLxxuVzLo#omeD?fq?V18_r?PtZZ@mb?{az5OxTK91v!6IONi2B` z%hbxgBW;K`RSkyu{AjvGKCKsdDi7V>EWfnuoQWCnyu!5%e@c~Oe8Z+Kz@A+odcHYZ z{snGv0;bptj2A})h`p&1hIq=gG{z{Rtt}IjPBp0gSg{=r;PMu$H~O+lQA5wt*`vRz z4{xPcXXKO_d{KX2JZ{^a%AHh^$4hrB`tdt@eFG`;NL&OEuOz{k8(osR7#HNdaye2O zt0;UFUX-V}5(S{eRzDF~oB$@*ERx64`B{@>Uwe~Mv(4Q)P*<#`+F2mk3NLlAA`PuL&IPD712*w=Y|^pst~Tav{-^#}hQdL~sg^Pjtj~3XyX(|tdClaE*vX>$UAEF zS4Yp7<0{nJgy`g!1^*>q$UzBAG4Drc$U4?Mg&+x;Nvt@@GXEADUmA^f^6#2sUvtvxfA0cUQxo#_`iymUvOFfR!3gchUI zeOJ-VEZ{Ya8v|NyEYtcMI{_lOY&g9}x?G-3-kJD7;-y%QOTSg#0*Ay_x^9P4y8}7S z#SGikF0@Pmdq?p2@VB%5S^Aa54E)W9=eEYUO0Q9U?4gT#a>O=NgIKW>pQ_e+9(to+ zR@~?kT&EN6|J<&?dvrhXfi+%4xgllqGd|-b&ZT=>OD_ldi3jnoqcB^!^nAvgu^?&D zu|?--&cWL{ZSnF|U-M&FtrF zf2!e9ha&yZ!52pC*CmPvd1wW&Rb%65a4$hwkfX7tO7 zZT*%E2o$3x6m@}a)f@77udd5@;e+SW&M%TLb*gK`@r34pWh;&KI~Ad-QsKP{MdW!1 zS;sMXtC4lxN&oeCl^4fJl|#~R2bTp(V>l2zmcl;V%Jae{z}9}mQ0%!k6MRvY-G}Z9 zN*n{^QPmcT#*c_iLG|?`${|X-MZx3KhXd}9Ur_iQmoX51wzrLY}b1) zK}u@WAD}@sU-_}9(|&d!dFQFK4+n^p|8u`Yk&8}CBcF;y&z1>Ov~37MG~xA@7Bd1N zXJ~acFV_}KZG3_0)98vQe2ub$kQvx=TX25RlP>J7w4YDH?y$~O=0yj?3YV9*GiKMatP8UsVIS@5gjQARkuSvUtvcme!tcAT7g|?2 zZFC*?LcB9`byxobgBv0U&J`3kRa)r{%}-rq$d>ytN%q9S8N(wxwZE${=$cl08TOF? zg&y%-VJIjXYq|Bi1(~%dgN#$fhZO1eH*fXGY8kR!D|3xZ(@kKV`TgpqqcuAvfOhuz z8HxSgP_Zb9;-I&3barM)YlY5Pb&Cn6$2mXlB)Rc*#2*-lYdxS&)b+u?LyLOlq_Ss6%;xFWt-Ds ztZgM~qt#m=!%|54I9>IgW7gaSFP;GvQCk@9BRysZYel=-a4F_gq8#*j!J|g4h9m<+ zdXXzDKI-7b?#NbFn09UIz28G zf_0nRr%&uAptLC9H~AhGL4=Sco_9UN(x|LV=R3^$?``e}i>)e+q6`nE-$gjyDs`eJ zUn^%doJp1v{%D@!yQ#;;(Bso3_{XkL(s&VeH_NKIdCyZ`tVO0oY>lw^0K35c>Y+8V z`exJky*NcvgRbEm-J^@P?6fMg6~3s)Rfj`iYK0PEvB$N~>_n|wMGf9WI48c&39L(F zaRYL@8`Gd>At&Y_57f8BzoK7wEjs#d2D@N(>X)lQS6-g=VPPoATWKpL8|e)X8_vS# zxcW*CLp4E!p(gWC;*$o^S!kVuuWEVm$J{iFPpvCe16j6SRN-9==QC{B8+5umlbvBj zNW>+0Dko|&dTFN7FK2SzCHn@d;^@h?Ijx+-BikH^WgXFvH3UtF>1{d=rmpTFpE^>` z!i0n$DrcK0r>!o^2IxuWxaHrxzpK?;0y~Gx!mc6X{p8<{E)eCK+6XRWporT{hXKLr>{$TPXkPmgvz%lTU}oZL0-4V|34j ziGJL=;P2%MJTD#ox`YoiQbN1i78`S@{eyw_XC4RXoouVH`e>(idwB-fn48nZiNIlr zklLxQ;H&!Z^~Ow~l7scTiA99nF%6xHu+2PTktpzMQ^BoWmzc-8>rUeWH^L;%U5>!M zo6dSw;^_V`!+>&9g}4=9O^HsmnD9;q*P0Yp=}9Am!4Nl(s07u`&bWgVwQA3E-{EPnp=WaHM;kS; zwaUPkYsoPy;7ENPpTu#^!sF^um?<6Sfk(H4BV^J(FB@4Ws7Gjm?`OezX-5pw`*}xD zO%Fq(v;{P_BhYgx5)3wo^R-y3Sb<|D0`x0tM+dPNpSDKR4p^IeGU;hdb<_3?E>3-- z8jrMcj~`9`f?Jbt^L|`(pAc%~5vO_wGFMAmh9 zG+bWIe}iWKGO(Pg<*T0Wm32(v*#j@zT)+_Z&;7}?N05XYFWtAp z)`aZCYC{L=QbV>~IW{&NjBvmbrCSm3wX4mFgg*TKmDR4!r|yVc$gO%yGNYvbCOK;7i zh~RsY`J7?K-!OV+s81mW!q|P&N!dCk>4syEQyV>c#k5w28LJm=qU$u%9) zI&+?lBi=$A#7ar@t`+Rv`i~~wu8l7nD)i)3hR-<<+_LCEmD9Ikzz}PWPP%@KHi#-0 z#3N=*9U|Dk+iYLhLgx5m>TuwX!{G8;j#lQ!`Q#hzG zIHPxpPoGtyrudzEGMpRux z^ho4Pk1DahD>2uPU?D$OrEYvW@SCcjiz5xPUxpat^5`tT`dEMI{xp9|`Ek8ZQ2>lG za;cWZqShl!wnrokd~MA4L5uxyQ?hsRExO1#Qor2q>RSyIE*3P#xV{>3FZGF#-p}38 zrAbh6NS?HH_&T9`Fng()W=+;&XJ2{JoUvcCHw+k|i6k{Qq9pr`Bsm1*Q(?`#)V-!s zbuf)-2&adE-L_H|khhH_HE`aV?8R)s^BJFoU?!R7{!9hk%7(dkar07$>YO4>%<8ZG zQh=wW##xYk`@6DnsQ=e=hd*JK?P<#QMXca^7t=#abT_+q5wW-Dg8};ZRMX?2kU?$_ z7Q|AT`Sp|V<55OWQqb*G6W3+}+ejo|0m7FJPx)f9a^$h_! zO82wUW&@8nRG9?ha+fUHIZEs0+EC=*VIKC=XlL3i+H{panv{hqW(Zi?A=BKLlmN0I z7I)uN*nSr$p)DnM3$eT_JV)aK>YcM=jHpDc(nLRz*`%34Wd4j!@HEJro1d$_7Hbc}Ka527 z0s+`e)r!L-6aMCYEn+AaLWh>;OVMpf<4^7aITq*!I2j%}T}9gt$+0r)!&kuet{pCe zpGACfP@LTeOCOt7$Nkm&K_}rFrVrskCy4H%s|;8kUD4AeN5ps$n>EdX>R7`&Z8Oa| z4CB2PvK%Z+K_^|y#51xfN}mCw1IZ&_C|{2Ze!d=WFjCrc?TQ@PcQ6-j5yB1HcfKP{ z%{+y9?{7r^dCu3x?WnUTP~dS#H*H(XrDuH%4*Y8qYAL(({GRnNo15Ow#q^_9-dkI@ z2~E&mk@NEOry04&_SM6d?D-gxp?`PRI;t>PU#->$zD!!ch({M47wma_(Lc)61!+gx zv8?-_85#dH@Q!rtv%)t>`IhBDOph`siRPCD>Ao|LzJ<-X%Xt)Ipv1_K&o!rB6Sc{% z&p#xT%G*p-SiZ4cAd*URXRm51pXcSCXGk7Od1c!B(8ucsArc)Lz>6+*-{QCM9+Zy3 zM}v71GX{|j5l)cZp?{#yt8STzHgXTNW^7${IHH7-*&g8LVH6cu6qOo7q57N=E zfT9rScvMrOB`rBg(PUW#=)$4T-zrz$LwAv6D!M5FJLPpqCZId9>R3tYPu#c?rWWs^ z)#xN0{7VnKc+XtWvQsu3duYmZHXzQ9-$TF8bkZZDMX%ZBzS-Ve8E>QY@6O0*ah*mm zuzc>XyGrUa&Ubra9Y)rFjnRv!!1f-Q)D&Wr%s*xmAsnK>3ss@!fTr zh_`lQiEF}?wIG*vHR3czxLNzpIeo!70i$3DHJOK)PK7;9UooCeFq%P)Q=5DBwVn5G zXKmb{OI|w(#&hO-SDZ{#=8yG$%0IA!xeS}S(lArR0K4Vb9P?UA%k_`F>9S3NGpShSd*+^Qntk{( z*UP8N_HI}^cx^H>&Vi}0Q!@fAT+Vkj@lV79-v`i6XTJ-@{>((231@ENdun}}Q`5tq zz7%fa^Fr`YxwF35<;YwEV6e9`$&0?54PLlyS0BA`|8Uz_SiF0LZQAfi%KI=0UaOHk zgwBJQ7ARP3f6aNg&?uQ6paO5wO(@h@ql|bh zd&lZt?2kwJh5ooaavmmCl^4&-4ru!*Xuc$ z6g^pGSq!5$6LQA#qh%X4lH@tvLAK|Kf8Dg~Q@d9?g%WLNk7fU)GMFnf{+fbaTMi%9 z()En~?kM$aY7j^=xyt-J9r9*8U!E1QHSJMw>Zd$Ato9FEeQnw~{pTLLl6~e68;6p2 z<-6(h`~I!p&`CONs#nmwA5luhR7X9C)UHRKo}3@|I={e88j@q>i=!LU zT@@L=Iatx`xnWd%;iwbuD8OlfHPN76yUOo-4WdBuV>WE*0oI3En{m^`>?b%W#8SIC zy_^}d_-gqKEt-4>59$RVFy6wJrrjRBx%yeHN^bcp?|&B0m^c z(C)6Jvw}L^9*9Vy?`#EbT=F)jFOs99ntDwP)VcTg{%&yQQ$W~mT zyiaR*jyW4LrkJcxr#?ZjU!q*67rh18C1lxMWlS$N!k1pm&iW)J5@asFPeonE>PKp= zyf*=R6&Ckt{PA_qgZ4F$+q-N&a(fz36HdiCd>)^Y`f%W6+qW0u(^BvHMm4TEbdIJJ zaQ0hP%<4L4_86!ieKxGt^ z^Rv&<`SeI3{2$=76E7zLK_+Da04ToxcqwJ!5o3YCOZLa(%bglO`J?=f4>ZK3%4}{!0Q(Uj$8?Rm# zpJ93FMfHx&?t62|$#|`3B;q!xQCUv>_`$tbXe_;|Du0>)@KR3max5mB`_1 zH>Q8P5LKfNW4h!8W0APe4*}*86Qp7+98I?5R!SZ4HMqkhjHi@fNG!Y}_>}41BMwFy zoAiN64n~e9J$f=mOMMN-aKlQ`gWDNvQ~l0C`duNDY7XVArIf?kjl2csNA229P0UKX z){AJ1i+?6Z;bfS4De-l0ph$)hQif)|y|7s}iV_4`c%!^C$4_V7!j>gZ(_>nqO1o7v zSH{T~ZIO0=aPKw<;A?5-I20^hpK_q2>NxTQc~z%)&kTCo=~#~sgr9hwQ)#K^%7B^O zOU0{mHQLEP$7{apQa5zb5!UJQ^)%QLKT&P@HQP_Z(|*5hdQ?&J)bAy{h2kLpM%?G! zxO>!1=*(s87ck&gF0WI^)Hdoh?_~X;zNNxhpjU%TLd3?4Xz8wJ`58;@D&a?e zCR=~GAF5(PFv;btEdzcYfQCWAV=att-+i#fRHr_srcBCv8rCMK8Tn^JOFG)x7XI@) z_={0JHq&?tgmw!3!Me;!gz&U4wikTpdiX#iI84N90Tdd6693DAA7}nE_!$PxRl=mX zQ9uDt>-#)G^D72HNZ$CEi(|I&jQfkh!-O7X)2Aj-hR0gjt&FTF&8=cunsYB#Oxu2| z)w)G{vhgeI?Hq2}oE<^MCJk3uDR3?WTvO@Qv|En6?WpJeVEs&fUHHRiEogK)5UG*r zFWN%jA96{?4}{k9e^^tr@3OXKwNX>fT&hKXCB@WaO9EMEf)O&urhE--;IG{OY;ahN73o*UJ> z?p{$NbGq-1n_5h|>5BV>&GxU*03LIz#;(O)X|6wxY673vqjZjvP8*b~Y&67g*=zZw zC5DoZ&QCjL;Jo`8-z>kAU8{-uwbMTq%auqTzqdw!Q%T~|@XmdUmimj8`d4D;eN)rv z+e{&fySM1wA4~JnTybw*YsS5Wz#FKBq{!FRO&EFbTTdTOoL?#5Mh{9>hvaeim3cn@ zf7thUBIrh|OgZ@_T2_g%#eKerp{rS_W3sV+XHBI+$JeY`tCNuJd^AC)fs~r(GYY4+ zB5^)bZ(R!8VI(w$75mT2O(}n(5a+2vr{h=yf zI3>!$Fht*8(Femph-84wyF;)*nRLm43ZZc2zEv>(nMTfiK7Hyn_DkO|rka&M{M0?h#2 z2XW6iaE{DJ000@n1koD-80k+#?nMW(4yJ({3Ik5V-MId~? zA^9uCE)&{29N+`xoffbHvjg?&I=BP!PPu0f>YoL059pHxdiU>8y@KyFnC%~QZRcu!u<{GXWZx6D+3oVRGX)6g}(C>}rhYAn~$@|vLwD%V9NaX_p2n_k$ zp-Bg*1|X66Py>F1AVCWR0aOFy$-9w()PsuPe!P1YAVl)OjQ%lXKTJSFNFIU@{@(GY zat^Sgp9*cA`K0&jzL4?r8#-zD6m=%?8e00aP# zJ)!_AyJjI}fr_NBKD$`Rek6csnD_ugIS(GdM2I4&KnehEHx1d324EGU36`e-H0!rO z7C`gB*lP+fQ~+Z1SHb2XbcX?W`j1fskbpjeSRkMCl5{s3))35xW^XEhiOL5OkP>1D z>w~eU32G=0TonP=shB^9ciFszaZr++HM}|I}TtD_5%!<7UB>2YBSge`p&So1G*y) zeC&5e>xBjy3_gN>HVAYV8ukMeNHXXT;g=4`3;3Mu7Q21O?{v&J{ur-@6z_FGA63I2 z>Hz-~U?1Fj9qNbXz=!(&2Q9Eq=G74LLo@QDYWIWo{~}}D^nrcQ?fXN%YYud%sTrv~`L3PQbl#8E_G zsl(FkM$|)%I`=p*Qe^DhtKB>H@UsUBU zYVMbp6U#5t*ncL>FO%b!DegZnU|(L)zP!YOeipU*eiXny7^OdPkRWn` zAo2iF#5hsJE|S<^BryOq=`A!V5FF(R9OX4I;x!QFUC2cRRMm&r3zT^Tx)AF>$nph~ z0%-)0bKvA|0#7LV(TO7J-$|p8#LB=wxxC=%`{47mMxSXW=u)6iGwxR!_bYu2_GKsf zN{4=xVubp%yU=_&oceO8|JlC4lmCipe!1#|`l7miQER_kxsZKPvjRS;Ti?E@m|tG) zAwEt2%>RRsU(oFfUjLW-^FR1suHP3V`Y-p#fAGIt$1j-oU#``E@GIAa?yDXqkk2MO zqkh4<{~+xb-1w^9_rE^2{RcnaA^7~`7kmTxf&^d9z536i$(KjL|LSr6f_?wxeomE7 z_+LF(Umm-@y1oVf)t~>`i1al|UNB$2hQ53ef_`-h>#LMM_&eX8KgdTCpf6wrH5W9O zq8kyo6S4yS5e|UbjSV>tnE1cFO!ik$`S*Y@!TbrL-vDU#Cve>N07`$MA)?=Zkt~7W znXjM-{z6HD`Flu~x6uE8AOre7o8K0A>KmXC@cFm`{qq8n1NQmg2Pu67^@2;^hCVUt z|0f`n2O;-bH}H`OGz|XWxaouXu-W%VcwZU*zKg9G3n3+#h>uS5A*>Hpx^|6IQINxT|@f2>4&*zA7b-u!=nOtovM z?r9k%crK+)jqSgJ_+o5#!6s3n-2hr7Ls2;bbk?m>!mE?ty|%VnuxxoJ`oA9!J+JH( zvI7GFf%yzy@u>;`Vrs`^ZE9@7WNQ4w#Kg?mnMuOz^Jcf$4>LRapU(f|Y^chV3YsuF zKLKTk+-QiHl$e;9kQDN08+9@sp}LN$4p=B19Y_4=H%hj3w%}Ae|G`*YuIdVrYNtJD zAoP2TM;8?sJe+xD*>%y&rt?+Kj=%pqm;+ukk14i9cU;FMPwgpB>-jA;tET5gYpUHg zY%n!+Y=QsAcEe07K{tyb_r_+d^NCqGzT`?GaeWS_#dt)-k7?dP^mjz8Bd0OvWsl_M z16RV&%cOw??(;~mK-i(-e;sA#S>M)GP8+86VWy7+Jp2uFb!+cDyg} zPM)^M(D8gjyXCR|7PCt`1kli7Y|YXwuymuot`o1lZ1pG}=AS|P%K>iEKC6U0pD3s8 zht8(PFhUaf)+$}OhQ!)^mePi3^0EiAyCrTb@N4L6VzSxaIv3_dQK^5m$=rOq^g4i!8J%`dei)z1 zm)jxDfu#M-EW`yh0akfxrB{rdOrabZyC~8@JHmcz4Y#Gr$=nt#VnS2`{;enRML+_H zIalY;j*>!67I^#p;rU*k2~;B|v)K1ASbLZzcE5>1DdjGD>?@^9wYl@!T&DvNclRG! zsF|i)JF1%Og`YdEniUNF+#{6Kywa?tzvG`_GY)?S3V0FyFqrVE9%p#{pMnv{{1Z7I zhXny?LjeIH{a?*id1nh%GZR-QD;LlIbF!vr>bj^c;k-ZSIfdQxG)p5#t!wDpX=-Y2 zEVVqZLleC$KDO>`vr&>mR7_0zKL}V{e|)@O_g&-&z8>e;fau&LKxGNZ8&7+)vfw^a zN|@ZVXH0oQJ5Bsx6ZlbbtBQ*cHaf9tI-M0~mbl#|shD}w9l*DWb=K1+$cidJZp62b z^*abU+*rd_D!;2Q~masqd@8yN%^@VLTxaVMkMf3}syT^aPXC2@lg0qXW%e(u~R|4OG2PK=d9V}Zz z5a0e#R?vBAO~njV(S~H#{^@iL`n(nhYxt|bQ86CZI;}yok&c0`p=KrxudsE3Dc>w^ z%@@>&l3v*$m$B}%6}-c*Pp`eXf5siFu6@&Ne;EE#AxGP%*nd?)RS+k{`J*;Ss0EQq z=5M*gm>n~_Ev8miA_*5~fyS7|CZkegS-D-#@u*VBIg+&RhHG+>NZi%$hEnF+6Rl_& zEL`k`U16@wdR5V;{v$;yp>Qk)u&N|;fofy?vNi&`^x8g*B;0j~Pr)uE?}Oz<4}tTU z*epkT5*ZWo?dC8ZcQ$vv;<|6?D^0A|bbU2~r^B3gUf6E+i}V3rN9}T}Wujea2gn^@Vl~x#205 zwDsAXCWpwhCoIAXq{DdQg(?=-6wn1yEB~`LTB_6DZ0ir{w|lFI;o)ZCwAM*jwOmi$ zBEgFDV$0#pqt%fFi94?bnhL5 z&1Q|4tB{*E;kyugE}5V79gHz<9K&hQkZMOaPpJvIW1u?ARFwQk(}5}&!U~(tNlMoc zT{qkmt$PXO%ZwbWf6yDrbfT=hjM-_ih!|q)8pTZ>q?)BeUg3`bXF-oR;^YukNqXT(sdOiPPCX6QVzx@nVciJ6AnzsW| zUT23LMtBsg$ChEWP9s#5Gc2vsQO!8A%#jka=^Ae22K96*#k?j0=u&0Va=ojh5S8D# zP6>qvG(%h;^1$QYjwfB{T_|q-?2o=9t~1Ay7P^`x}_yrDYU+tSWM+PGC5+qMd27 zHmvkO61Zz&W5e}_9)Lig{p)<8eP2~X?Wg9>rVS{+g-%i_2RlpUawoiM+^OUms(AR4 zabw1W(-D!W#>8sLsABqG?=~tQ69Oc~*jNlqh#5VfM=F{wA*(&SfxNNLsD9|HPVgAK zy-eN0~1qpLgm3FG$s;=3uzY%P~03y!1v)FW6Ue z5cJwD=5K1P$Me1D0M&0|I_Wrz9j&*)CCc2~by}ML48Yh*mcg0kx*0yCvW8J{Q5oPb z&X9LHZAv+U5f46@wl5a;tLG^-vhdcS;pfzJOjN&Ace-r~okWa#w`yp;6kc0Ib~8)% z=P6lpYKl49gHAS!vdyVzXp{b_rZ8(ywuACa6Re%u{Z%Ng<7d}9mEOIcSd+GH`-MS%-;>?*aKnbfXD8I6mp(%R-8H1=#d)Cff>jhmK5)y(TOE7%0d zHn9qmGDDIbvnPdm_AaVFnZ2k#qdX=rjsBdB9VMK^{ZvnW$R6 z$QsIFE6)-)!h|+=T)~H`*dz6PNe>Sra8J7Qmf8}yMPThptL~LEsB?T3Xx#;0-rT82 z_!vg?iII4PHvrw%y|lzYY=+0;6(jHgc1Pif5?_;rmeObo_rJWPV&IX?yHLiBy+J^GP(h2Nvn<@xLF(#^(>@vYK8c5Ad&N6paOQ zgNWvKiE|Gzp0ltAou`wJ9np6vW#?4SAIlFRtvz7l8!u(H<0>hgFwrW1jL(y=?WaUN zQ>1YgK7qMU)LS<*X}cn?6&o)}XSsc!R)(!{!;%!0i=&BSck&CPTpaZ|Q+_%y=Lj zI8bVi`9z#Gh^LvB zGz_OxU8uXGoay3|7Q)Eu-X_8^)Ub$&9vs``@q>^F$zT9)o>IY&JN#515n9r|H)_EN z9RF>fBbifElgHfA5%DD%O8BAF9b-$lyKKxN=+RfG1An!mbT8hsBR&0CLg zJE_{I(9(~&-h?mNP1haIEB*o>VExX(sz7-#Z@7|0HIg(%3?~f%CSO60U;JxkX;dW^ z#H97UlbyCDQK@VRXLIrsKE%j;89ZBpzj_@?HHj0n1?He~FXdcdY%iS%1naYn#%VQ7 z(OCj=ISEDz`^y17&lH`S<1-;NGiO`xgDf46V@{`n=;{UY-jp*`!z zH`}t@rNeaMdIbLY7SM?mLy_c zqD-h=Z;aQy>(a2UN0xI_K9UFsdzFT__&&%AI@LA_C97_1=Vx3w@Rf{_&gkDJhu|w& z%72c!!0Zfv1Tf&aQjc-TcgG_^w8~3KdT9_3fl@g%v#6Fpz) zT)=B^6RfvV^vouX7uI2TJn4$eD9Yp;=N0KDe;9AAKygl^t!=Jk;UTEo_x`KxBe8#= z4}M`~Ss!s_WYEWe3wqsV3Lk8+YkmKFT5d)D=6>Fbj%D0-p9Wk{ka{3w1$0Yg{j(M zC=1j#Xm}LHs7`jx2)cP(8qm85>voG>2c zScqMVX5)B9Xi92zPJe&)5;vSl7^lzFMLRzJ)f1h=NBW2yO^uW0uFMrJaF1tWl`}NX zeEvL>(Ch#HxYNYVTis1r?fEy$lRX^l4<8W8?}(Wt+9J#%!M3i%NR~R{+jZ@m(_w2U zhc&I!uEYgu%DrMU~jY;kZ_57Zu8 zeL)?E+}_6OT}VzFWC9-n#a+@W_Aqu=C}2CnKEgMWG46vH-jVh{F?yzIkyXk1MHjqg zT-03?=6>;wy2)Fm7l>wp;@a*f%=1m1$>p??JZ3TuT5sF}h3Xj*coBQ3iZ&6~kRE`w zob+ei&O*&_b~C6B4T7x~WKcerjJM9g&e`dM4IJ@HckNod$9n`%12id`K5?+_nu3dO zv9Lx>rQ+H7g2i{(z41ifX;*8O2K<`QFN#_Qwu?5-@gTZ}GRutwhvrrw&r8HkS19~Y zwwI=N9O6O!9#2&Q?4EOb6rqWX_!5|)D~9rzr*^QXboXNq&SM2_y82%KCpFLX9Iu>5 zJrIznP7n~P|K*RMnWL+jvx}jMnX|*^?%n@T@@)2nHd0ex26P;zWNZq7ePjNm1c8#M zB!++rL4ppeAC#XTCe12DJ`s`;LK;Mc-BS}cuw7rdR0vvGZTym_}R@G%7*57Hkvi-PC?^;UHrmZzkz&C%V-s-lh| zPaZZGTWc9biA#4RfrilVEVc+bL3|uCJ~<9pmwH>+Yob$((BNRLN2;o*v%((}`bNl_ z7nqG*6$d3OBoqkA89d8oeOAT>!0#Q}=%_6pG!TH} ztv`+prG0Y}6(wduzP-BGpYP)EqL6fEPMMl2d0sVEe7?^?SL9e-QTyq;Zfs;__>7Jg z&1BMYYha?Pst`df*&!c3_}8|TvG^t6mdv8~ET}&}D+t-;5mVTEECc)B)&XhQG?Iwo z$)QlXaNSm$>TkST?7wEVb)+bE)e*~@BP;Ce8tk=J#7YQ^MK{kvg3!;bCv9yAv?oX} zdx$ouXeO=*j@im&HY5yI`9W`1lb>3&)s;5oS}J(jGI#j-M$ECh%j7aucx28~6&8U( zBGhL_4L4%q6s-$ARt+WQ4MXVP%7l5B$jygSmlqW|y$cHv)tD2*ChglIwDbw=wsoZy zxwQsJ78?yWQe}(oF{X-aa7j6C78@mLxX~rpPCD7w%geH{W4V^;D=MPcNzuCy2wBPG zXAWa;GM`IKf|#L3c^$BW2?f-$aBZx6&WZpU1Kiwt;LaM=j!ycb2e`#41DBGlyE*y! z=#;Du4zVRRmO3^LR%^dlb+%0*RV_U`+PiNp@s-1gBh^hTuu6^7$YhYdy9^7H1u-L- zb2IasNaIB0hNBl4QO2STJ03|Mz{;04?MXo&nclhj`ze}<69=^qHq$sZRFs{hX`Glba z3jlza>Kg#odoW5}PB5G_iwmBbc2OWQScbDBpEX z#wvT;B?p9Tu|j?(c!dC0IEBES{9h~G_@bP))#Z{dEZoL`n4US_QvIyuBr8Tx&Dw|9*+CPW7_>`h7d{o)BxYupqhFFs)qo5S=LzL z0RdrLZO3sD+(TZ-l_7+7bo*ovb`^c)Q2e2MQ-^Q|ZM9A&%QNs9xSv5;2v`c0+4+tk zA_N6ngw6&WgRdIxTY($cA?_Pe0|X9K3)qN!4WEZjceu+cPzS^S9LN?9Z9(3V%a!v3M? zHbQ8I!7Y3S zhHii|7j3^D@|7uZ28kH$=GIgEY??s+ zuIx&;N!zT7`!Fz?g;%4~Vb4i8KY<6g9n#j^kwmC3%P{=GD$j`aK9(^pW>4n?6JzqsNq`PA&< z0lTd1Ve(?)8ykJqHhg#aAa{lYe+B<)pj^Tsv+kYaBG55gM}HF@=#$!Y(_XK|+q;s4 z{0Pibda&74*Yr|HgN^*w>CRkj5&uzWgq6{*E&Im{dkRN2<_{})`*cUb^O+XGRjzdM z^)%^-vib08&9w5g8EEp1m2!AU{u2pNpgE<2SXJgeN32blDnVwW7C+^9qgHG1osa=v z;qTii7v*ZS^B5J>iTr{J7M-}Ib{2lWzJV?xf)?zWI;1J}YIxANnD$5xgsH+Co`g}e z=G&i9p<3?xs_|%5^9?zd5*6+OS!<<@C|r z(KlaA{~S`{vBQCa%s_eU)r0>PwvdTe^C<$;@MGOgI>qaKTLx?nW(1MkkJMo%4d8!4X9@uJ+3?JYDoeY z%p;?Xj-^!vL!MODR>HYNcy+4~D#CV@_SF^98lOLGD}>e^!T|CS%0IZBKt5(%-_kgg zf2NYuoGF_foMcu>_k0HunRV%>)y zpYo{-&cPO(>?g<&2uii9-N|@ag!#1v1Ky;0Sdf>Ajayz^K_+k<{E%ExGP(gd4$pSNH|qQN%Pqjv-2xxI~9Z z_Li$!qFqRusHLizP_7+Cugs2(v;AtHH!XpwNiXco%?`~PJ*7nYFTV>M^=&-kL^J7R z1v~n_Dt3v?$Z3t1weCY$UpYVLgvlO}SE|bR<_v@{vc`FJqP-6NP7+Oh9C%1qrIf*_ ze1RUu7#)%HvPL)bl#M4poVl+)zfk&owtj-0mJ=pJE(7;dE0~L$QM4|o#=?U4;u|}; z+e(g_E=J9b?FdxAA0l5FStx&-PY?>N@Plq^l;5nTEqE75K|-SJXzT;~uy~aIG-slo zo~kC6xSqPF3T@cx(l5i4f-BS!#`~TqSOkO+o;3Ygkn&r&k=lQ-#%_b$tG=t4BekUq z3R=7zsg~K=EpnO-H5WSJ1;;E(*R7PCujS8Ts-mLw+n--Zm#fJuC9AHyS1SP_A=dmV zE%-cn_>!YPl`bTfH)a0O$B)t#-Dm~b!a$*YtSA|*A+xuD)9^(6CaWyog_btHUS91{ zKub(-t+^*S16v#k%jH|dr_oDrC1gPsr^EVynpwqa|5Z2SSGL-7A#gA)_t>gX9lH>W z?$ae*5hoOk`|j7e^@D+cN5+qAgpfEqfmxudqCwC11flu` z%LDfULq;f+{<>ORb*)q|O3NTSX)YCpcT_Xj>1{^BlJh)99S@7l(D#~EHX0jtemPfH zJ4l&|bWRzY<`_SPhT{ps-w&I`I2dTGRql;DbPuG`!-O@^)+P4_8W$KUFwHS?IUPwo zrK{&OWL#T(^VoIeB8vDW7QTgH`)usfSy4UWbOrU8Pp5~clQ|c5BijWDUaf?%pq<^M z%F%8xv4C7-zPFxaN6sp39IBFnf*Y!lcR5|*q2dM~vu-`C zYf~b+hqSGK)qlKNkY3ap=2*ROUAvNeYge(Fuez7{=d0w|HGTCYsW9iX z(8abm*E_id=f2~OyeqT{uW+tbS6hBhc>WyVM=@yh+iW$SYq7IC#Sy>!$PHu_-PlL_ z5x-(9SofyFD)V95i6A$e_5F?(4%x!u$76|eQjYs2Y8J;$L-dW20SNcWw2}Q>XJyR4 z*|OpAo5Eq{MNmeZ)c;9fmcF2onfvhw3me*nzWb6#Lk~V3YN{B%V~=}+$$$1s!5HM z%oIn4LToCtQ7bB9H54svW0tMNZnvY3tzKC7Li@4A_V3%T8l}}XByW1v@CMh(q$Juv z9f-CKZ{94;!3I$zGYr5nc6JPBhD*6CZ%3s|% zuiJLea5A^zo#UGS(sH!_ zt{cgdDZ`6%P+i+Vs-lZ0%DDc6H50Pj^*`veNWMu3PHa9E1;=|!SW4?9EN?K%5Y=jC$7Li=c%+}pX za-$?gUilTqFn2~Aya&Bxe^{#~9>_XUhO*B$?;%mU%z+9;zU6{lYz+=VpbV&QBqxlu z&T~;r;EUQ=n<6!}%kacwnLKNXbT18%UpJ*&-Ar%uiOvkJhNVW%#qNqK>3F{jGA?7)TYvt4d9%rOM!)v3-CT zNkdDF1pMB#s}zr~p1+gv$l1TIXzG|w_~lENI=o*+>bO&NK7ogBnBdgKif55axR5M$ z9t%;W7u1iUtS3-df~%hZfzYIKqfQV+s^BKm) zMPit3xG8xG98snP-Mg+|TtZ{m-xDLdjFa;qF8oKuhVf7Y2y>|>hiMQ=>$^^AOy_R|2eZ)V8+T=NB|EFW2x3>$xn`JN7= z>?xxueOCot(S`I=n|6`p#ptd_E!Kkqk*?&=^R+f%Uvc7 zwyFJfhbw%qopDGWE&6lzQrgavv&^uVe4b68VWES(TJ@3VdrLX+ftVxqCA~S#ixM)G zcR*LFKK&cM2-e&62Yd1+ZQTNS*uv*7^u7%u(X2&P>oc3?gem_rV9%?7sJE{RcIF{I@1XrMkOXM6cVk6?S>#*1ox**K8 z8wEEAsj;zsOIei4a5uAwJ$iaNTD-z&r)4SbGn3jPyP6cDKj_RNqRuSKNzLp9%HO#= zEDZn1KHDH0h|jYqTB^P+fns~RB+&~^tqMftieCl*FKOWLccS~rP@wW2o=b?z)(wySp zZ^wuRK>yI~xL*%+n6-C|Kq3tPoi>h1)X!_~lBCL9z;hAU#x4vrH1?5W%i7vVv}06x zo0oNKW;gwCr<$RZBKbPVGcjIpr)g`)U0*6zCWQ-dqPKhNWS&ub>KTX^e^CQdxiHpn z$!JLaXNg-L(yqhR#t*dY#8SUp{}SK|dAzFusJz@w5{Sm8VoUg+IZr4Oov_a5Y=s>t z+P5Q^)#npy)iY!wY{N>HW$cQjwpLJDTC~q-q3**_;(-K-vz_vfa)(*4;fDS&8xjai zgRus5La>zWYt*%BbtFG-@h1TyWn7e6rkKjgYig>xvCDd@keR>kEq4JD%fqLmmZS-R!L(GesB z|J!J7Pj1DG?J9(~``&7;tmI-|1&|C$K4J7C9wbFTkdiAeRfP_?e2I4yPX`sO4e@#? z%`LADzfyfx^$HQ-7Eu<&EyzQCqd|;uF~K#91U2@T{3eD+j6ffhgc>{yS`JUPv}A{< z_hvaRBKhs+$Ay`y7fVDwc8qSQK%rD*fiZTaH2z4{pU59kH$$R6{72g*`c=qM1kXmo zW;mARpGHxV&Um(?0|#Zk%>lt|sQ7jeO>7aF+OPX2J||MD;L($9-W4p0#I$R^`k5d^ zo4!Ae5v8H8mXh?7vHTdQYT9HzdY%{itfflrt$>j<>!x^!v zh%7BSg&}TP-pzKQLL7vmipoX9uz!zR@MwnWPH3!C%Fi9hb?|W)7b#1{L?>tLICR6Z zqO(Javxqy-72d6UP2_!5)oc>i?s*l1!>m<|%_b#h@GH$}CP$z|lle!iDVB?~_ovNk za*SDFenFfo8|9KO^X!|FANc+`lF26Z3x<##8QtEL&DB^F;azt`(Jzs}!$IJTSHqj> zHHc&z&X9Us&tk~ppPNcyq(E`u4F2J*yK+G}2f;b7_}IwPYejDcS8hp%AX;$(3tzgNFeES$mn6WM;Ekj@HSWR{HMk~CERz~b&aF_pUV3hj(8=G#KhJnDNyNm ze0m~dtuAW<8w+;?j?ls2kIUWFgeyV&bZ1EkWLL&Y&22x?6=O&F2SrVHiiWTM4*+IB znZM`^HM^=d8i~(MWv)7Ph9)SRu&Gg_or(LEXc61B^_pR}r5&#ozuHpTM^AcB5e@O6 z5)V=E(WGS8)a%vqvEmUW9;MTRtb|vN3!(OpEAfPH{UY%i%hVB1b?nbhjMUbW9%<#z ziYLX>N<1T;H7Wl-Kbw>jV%h`u;L%3BR>LW-?RO!+Ry?o73*z@}&0>VVk>d>J>=D4f zaz>pmkFw__C0-V5rskdm6&WjaqDDkFv3q?Y8 zl+?Tr5^7BzIW^nIwYQ(C%wOu;O1vZ9MW7o~wN;Q7X;#lNWWKe_k2)i^-&f)TQhG!M zz6(I*e^TPl`V?Vl>u?oTeXPV^{1e&o$jWE~C6js?cH&bdJ|k62Sroi$aC4S0$0VhGT|Dwb$ivQ_jb)PKb39>-Y zB&dqjk~mq7-6d0!6X-gdG`Updp6aBMmd~ddn2?T=t_&DV;8L3E>OIKyX+?mU?iDp6 z-j`WQs?;$-Vqyd}@@$>_j3g;LDcM=(_~I(_Ai>D*$lfutC%jDC)8fj=ikMGJ?4o2> z{WcO>=-Qr`Ys?tfDMLzjr+igMzcy4)(n3!t^s{nHGyZ45>i+=dzG$N457ZRz_-3Y{WcP_dI-KED_&QO8y@C8kb0tid>4u6B)2_)HaE#WhQ3^Z_N80 zR&=#4!K-!_TupWeXB{~8JbElkn3$2?I$uO-zw$68$EE_;KH;_sTw|ldl^mZ6w`Vp= zh1;){rAkgDCevK+vkUf}tmG5|Y`eZG<9w%_s$^Lz=%3jpZJgr_)0Lc|ZLLIxR-*2+ zl$@PPrD!!pqN0fc-K3ppgd>%l>sv2Jr7CcQc}gCY3cqKLp#6+!H7!;X(E=rp_QUT} zOdTEN(HIMrTtpky>P~am>?IjWRUW70@xGW%E8%P9VkIL~?j}-{w^LRrS)Iyi$&e=* z$8p!nB}y)(?GQz$M3Jmf@&sB(r~`o4=)r51tfLm)+O~+U#s++>j49bbOHx`Q>Uy)R zxRME-LP|a3Mw(QZcPVH=O-iox(;wQ%nw@gBl52=7TM+`T2_59rj@Bx9QaVp3B?*jX z*(uj4d5U(aS`86xOdT#>D^FGOG(U^kaJk=ky^?3p+FOA%^LT~$#wU7twmip?zs*b& zY{#5Ro=YgAXq54AEYSaU{o>#3LTvlIuTL*rKSn^WkHlgW?K=U%<+st6r4}54ngrSh zJ@ZO}EWqJkYtK1~HyqAF6Whj~`X=ipH+sifCNX`C07 z{5_3BVo4`rQQ=EUzTD9ZrXwk4Wq_5LuGs;|>9E3B>IbxK*73f__-B%>h~nt~F~4UI zCxyz(1ke*Qm_B%q=L)~hgGLa(qvX4(Kom}+x78{TMF%qILF<<$>^jeq$o6vqyk+TE z7qoU_FmBJmnhq*)rL&i4{iS)}Px2#2{<)*~RWe#C`7tStPM&RwR(9Um zH8YVpKl)Z>SmjUhuS$MS-Mz!sujIVVnZ2=&G!4R1rv|XbDTZE31cJ>f7I0)oI9Up7G}2O$uBq zJm>(N?rD6X0J36f(-L!tN7)2MVP)&{ytyg89weGHY0?IF4XGzBS>mzUT1rm#Svvi! z^B}8gW0d}^k8@2zL}p1>-9%?_m|RG0{_pnq&1lH>*+?C~0@l+SjJiW7x0oqRvSv54 zinfy)F^Ozp^7;4MiYc?n}Q4?IipDQJH!01Yc?*J+Dr3){nbQ z^z^3f2+cEMm2?t9XHP`2hghQ=s~A18JvLR=p`>~Pb(AGdHYIf|QPvnzn^>xS2`!$i zvC0}pTGI8kBsCqctO?ZAsUxqcteVOuDr*vzxpfsJ@lD|rWgS6w34>M>VY<>~S{srE-HYd)2C zPL&%Ip6ZTP)-hx^1k8GkrHc0Z)JWn8Vya+m3S1}v_5ykzwP{>N{>NkFy-Zj(WML3D zu~=CVGJ3O(^q5i;t*u79VNI2?s;NeKaz(?0DwZf~DOKd8D-bB<1}HVkIw1q}v+Jv( z8W>Kk4+(pj2Ok@)`jr&z3q(q>cDV+iLqb;Hb@oY{^6<4ozh}+9Pqdv-RwL}t zPgN_GwaO#OI60nFwnkYedb>eOE0g$6Qr5{-qfAYj_D)gOZ!#(j>CuNeJR2sJQkLG= zs9%j~+}D^P(}LC+)>)2qX6DgJt2UGXJb_*aja9ZvrD;8oUQ-&>`yxs9Mq6bD+UF~4 zg9ko+7&hDc&OksXh*apDyxdms_-{!+WAc4?QyonTT-`=xZSucOLdP6Lj8uN9vMwX~ z)=7n#%g*sW;gGS?y(pW#fQeExO7%A$VoR3bLYCCT6OFUu^df~1nAX$ntEu(h0;Y`i ze~q%X(*AWIMGImoy;fP*QE5=Wk)ip3if>TXjecVzPO0=JW!>zT&ha=v#8WV&4K>D_ z>dohl4*#EENZ`x$UJlC~_5U}hrqMp{Qr6vc8s^E!c`@@{4=uXtVsQ$`kPU?f)?U8! zVeEnXl$9iVpu68+fAk>3mbc<$^m}Tb-Xx|s%&6xB%6c&M=Cog7h{t=`5T9q5+iZ`p zOrzAZ_mb8g&U`iGhLoplwNy;Br}c!gp45QA8V!nB<~?d_hq9j5#~(iLRMly#;8|ro zrvs!Zp3*19)(gt|y$&h*<^IbW4%v+neZYvOr?LL6SCsW?DpHyGb%@N*k_DlU^}4d& z@ZY`jUQ4AwBo6VGvi^{IXDxFGbi!-sMeIJ-JIZ=D^)eDtS)|r{m;rxlC_J#!+NrER zb~s2=6{#23`T%iv)VwJ}52f+`q^v*F5evQg)aLVA*2lQN$wy5bI(;sU`-!qXO?fx1 zzLrtf;AK`=e^u7!=}>cBL)_cg-<0)bDh{04w^iU22l!f9-?R#xmIonfeXFeRtnVpg zzAO@-;Jv;!(Y5}`fQ<->&ep$_=x+U+vcD$oK6lHp{^Mt6=~{hiVg1Nf;g9u;vUX`N zm|k4g;kF=5vzf9vg^@bzZ}rTH$EusE=<8vwjSgBjF9K)2doj#>z#s^x**4`J(VJ`d zJSARNzLF9NS4L{5#AD{crKlZc{Fpbn^vOU_e?h~0;eMx`t?W)j#95k%{m+Dqv2&E2 zOP@$`Y7+XhYSgx?vb*WmHc~GYne1z->#pn``UD5d_59{FGS?2%`DgiY{am{@-OV|! zAHv_xS9Tw}uQ}&RCz-W+OD7|>i8k9`+56c8C|p=wovupf)wStT*6~Otv&tNuWkt^mwZrKy<$&NkAuWj8~*;Dkw^^uj{zdcpiW&RE)%`=g3d%Ch` z_+|rzj`gVtmd{f5Y)W79cc~-oSbn6k=TbSbmx+v1={#j0MWfKrWeHS0RJcIdNBi?3 zKv0)r;X-9E(w|i_M@gpeIAtGC@B?Fze6fn`@Q z*uTRg%yyp%!X8m&FR>}+)>Pl19des8O3^~V*=v-2g1y`bu-0>-FRN#0zX}~%JKIJu z*ZT`MM1w5fpzIYk_RTggFV#j@XM9d&VUS;IH!8cyUg>>~MdutCy6rc3j|YZi9JpjU znehUo4)m;)L5lZ8&y<*qT}*_e(IZdqMbE})oq74l)efz3@e=yQSTB9qLp0vjd$izt z?A{7pdgIl^4mf%6p&1=e<`{1%lSt8HHxP8}voqhZZ+nIuw9m3DN&cLx?DPD0kE^|R zYU~ZlzJLmyrG|~E>>@-42fEv+>`nG1Mh4AFZG=SrLaM$@*_Yd!O?6ocC{$fe)mJL} zDypW}`#tDU@zu({hKflJ&PxN9sH=e zMo=H_SL%EWtlOdNr~Ox~GK!y7_H(pQee{~vF;umHeijnYUR3r=#KSo{SewCG#8R&) z`&IijGeWC0ioS#Ql&C-~l~tA5K5tLWv2mrhQ^PU-lViWlV8EWjj^;ddWzhbE{Vs{G z_ga6WY&VA)drY#>?~2-gRQCJ!2L`0x!K}`4-s@s0+0OpcJ}rA$(EiZ=h?f2FKBe^R z8RXs(DvYP)f2Qoe+MgRjNiBc>j$h+zv;Ls{H~TBc{xb9HtGjEg?62)_v{PDJYmzxf zrazpj;rOkxzq7yhG+re-CKdBcR@Jmc`lS_0z%>WGNG~3-WXCg4!jaK4a|-CqkV-Fd zENUn(Lp0Kd8nwHlC2$2LZ|vEbGILvvl{&G*xGL$s|LfQKhAf)qy$!xPqC?sAM)jDz z966E1Up-u(u1EAp!&gU!A6-*zzRuRuk;<`jN|rY+`8{|cjgaLyPQY#7thtCk(v6sfIQQWK3+b!X+|__E`)j{<6pyb5SpsnGcBebh<02M+b{H-y(3lZ6GFf&tAKS1vRdzu0Y$IK9SH! zZZ5A};lDng0qwo`r)7@Q9}R-Ow;#~GAy3XWZg6f(Oz z`@>28p#*kAjT+Oz%vtHxrF!|_U)p1OSVr!e-w~lg%Pdq*5iPTK#xlM6)}W_b?S1~x z8K#`!)GoiHcB^&f9i4VL2Ra8i&d3Z?qTNE3b8uP<>H(@9rd(rKe%_} z4H;UXs{syzEc{ib0baB58ir2L8C&Jxy~LVa8b;iUJ-Ps}_X}E}Yk^2YkHRG6Z-%S_ zD+vRd!KJbRST@Kj8{(A>!Lp&7A&1IF6v!kT)C}2FF@k;$D!}UE7AV;aj%)@ga{47< z{DPeECiEM$g5-V(l@n%+uonH{PMSj?RBnk5tj}yfXwm!Pcu}L_- z#0|SoGdt`iq55k0CG2)Np#^FmgA+;uve?Nl2s!yU!Qyd+xRSUi4(8^-LvBeHUW(O& zu=1GuVRcFNgRth9oa*~wZAqtaAei?sP#g>euYf>qj}A*l4^=;ySyY<50smPE9s|tJ6tVW4@h(3D*Q=1hqGwf zYumKalNGD5+aVzEDNE#sl=qO@tHwYtyxf?Vgj-4w3}Hv)=Zx3w3PdhEix1T;a95f> zeu2aBcPvuCcnCl#Qo}^(iKLYeQ(+*KK@m)c17Q{%3Ujd4k=SA`)*c1ZU_Q)-W1s>S zLIkf-I2M+{aZnG(Ljo4VDyW8&u=EUA0vq83xD=MdHmHR=u*F>vgM0A$0IYzAAr8+# zBm5qk;AL0|Z(+~(U=4f>C&Je_!cTBAV{i&{;560=PG`O03|0hZv5{~NI~2}iGvRzT z7cO8&!^Nx`HnIe4Vyod&whlJ4GvG?L0j^?Kp)R@^wzA*BjqCxqi9H23vsdAE_D5U| z@x^1%l_(zWh5LxF@Ggn?VfwBGwj-pLv6Z;m?Z|lbtN|W`hY(&Tu}WP1LOuw&p6 zcoaG3QkIXSJ&v5Tjdek2JOnQG`x>5rClR_Y!@KYl>_E7`1+T-?*dq(~`V#JcJM!em zSo17AhrIbU_IVy&fKIsMD-kj;LTB9f25k8#0*2g z1(Wdk*~QNKpe)3P5b@unBz%>GZ|Mn}V8K5_P7C}SZ+_kk-Kkklj^B*I`e07Ty&5|E zLs>XfC?L1O#E`|eLbe>23+s#RW^h7Qv7I@go-fHnNtPjKWcI8U7A%qJnQzPs*Lwx5<{{b^A)=}gAzE`voURiuO0p?*QK2r`1yTFEjrjq1b3y6P}4erbu5D@l{McqtBG@d+ita4@BO zvb1=YTD&V1(oBS%u=5DYakZV*xln9}?6i!H(wkb6v5)_6Z#`6yPe_LhPp>*{V)eGgmPm7SVor+bag+6<#oN_Xgk?6=3z)BWsR zQk|$xNSUzna1Ixh1dc=xPBRpkt^+iCg=qx{p$kiF^Gs9fXNOj8oB`XkE?CU=UxASs7pKUda&x&Ct{$_(@xe#F8kP$*~80!h+SQsXt^qt7^ zU<&I4Q(0e_jw~`0Sz;~{{6aPiD%pXsoE;1aHVRHe`Ft8HK@Bw)Hn4H15yqnynuywG z3TmA))H*ZacWe$kfLiNeRu0dydGInj3f^D~;9b;WJ5iUskJnGwBKVda%N%wbQ>=n@ zVU@_FRcr{WVF$Al*jTolO=b10oW-zJ1FK_kwvIKjb6FF+oULRxuvP3Pwwm3^PGpa= zwd`fA{g9o){?1NgKd{r;Pi#GB>`WeHXYt^=-{0ZI$8WQc9Yhhx3dqm z;E)J~ty+^>C{}B=UbIn&Mzkh%P{S=mo&6|^hBx3Sc8eA|@1n&)IyDHNpp`;8H4DB) z!9luGAyoRZTS*JkYZ~k}8ikz;z1Z#S4is=dz_IL3(#JfARt?P&;76kfyo=q9E9gZZ zWAzN_%W)kxUI!D!qnNqJSf^IrZDAA>$r3BCAZ&@eu%(G`Y~F1t<Zj?TD( zT%5Tl&N>LK4uaD-Gp-LOyz}6McMoVd(Orv%6MY2oK}L2)PZX%MUJC{6!)Q6Qwb)g5vM4tCNun?K) zrDoKdIj+8OsrO4X=Bubqjp?3bZ{YJEyrJLqhJKeC{V~Ztr0YjX_6c48nq+^Y>(@#4 zEnWYSWdADc%KlT3Ga|`;-q^xfl53k{rWi+J*RwyQu|LoRkff!@t(SD0UxX$F3T0$? zo`sLQXT7TAE3eg zXB1%{p<(?oT*m$a*RW5ZnSBZ^>~j=bU%;d6Z}24h5}spU!E5aAD7wCdKeF%PQ}z$| zlKm6DWB*3$}1}H{f6hW-|{}}Jl+q5Qh#haQlLnuyPd>~G+RE2;@pU@PjN3pNqmtV#)sjZjkeLdd^p}YY#MCmBS_q^-@=Lf zKoVc=^TRyx!AsMu&L`vZ z)U=4}MB;+a$B7m|FBG6OQFk+h4?pP3r}G&&MVQazv%Gyw#Yw4^();M`wBCXlbun(C z5>-W2#;7VKEER5VyB>8;dm)HUEZ6NFBLve!2dS7Z1)DF+7}iNG*71-X=0_sj+6>DG z(KP&fv}^=X`D4&WTO53Wu_5_FV?*+V`Ch?sUcqtLd~smY6_8!fjaMakl#CH7!&|)6 zU71*?;MEv;kSX6Q|=PNkpqA&Cx74?P$f z%ry9^C@MB$O+gX1hDkZ&ll(07vv7oS-H_c13+vFc0Uk#zG(k3B2_e1;sdEkN$JfFj zei97h>);^%8#s)g3RC#$FpHl7NAojbF+Urk{9K5kx^LtgU=_azPU4&3e0~{R!Z*Vf zekEMXuYzX21&yw2;2ypW?&H_P1N;VfoZk#D^IMT`Zqw+QYQUYs`g_4Oy=53_Nq;Wd68@XAi{T6WSll#}S{@@EiooG>jeekQd{M$R9iy7V;gq zvH<$OGx;;xA6$WbNJ64N$e+d4w#7gR{?blQLa&OIILNMDMX}3D0!5GW=Ql&B=rR60 zc{WM@dxH2Q#cp9ye;IOH_$w4h*e_%k66iz=e`7IuQ)u=eq_L!hzr9$a?yw}^xf!|? z;_bOP*_z0a8O)>IhUN{tKLx=*LlN;g1o_{P_P<0S`W3?c8-)AcVIcn&hVXyDNd5zi z;{S%p{70D2e?}u85qGoKw6=HBT_pX zk=oe^Z)ait1;Vi{<|(vHf#*AV7t5hZ{$;U6AsbwoU4&$DRx=zCppO^=`-wsrB8I{UF&xH-5l|`)g4v=N=7~e$XfYZViLp?LZE8g+B*a8m zDJH|2;t049X50&9j$hAZL7ujKCEh+L>|;Xme&Lo_g7ANjSLa~e(2$e!^K7rcHT~GnXq|IT z;fP=;a5eNSNQyvSC_pYqi%=y&yk9&vY&|GCAJbLj-Y>d_oTTX1RrE}XUMT$OHXm=% zmKVK}qVIOGABP1GiowV1YmfUyK}i;w`l4{X&4?Pe?V1_3wu|8mu7*k)=6*4<#HONy z7?ilp&@b$67o!-u*-;dX_B~=O4z(a`ql~00R*q+I#jc&ymx4fIVv?QSA|~Tn&=V4U z&BQrtTEtZCtO|{z6A|wqHXt=$2wCDH=p;5G6>oyR;!-$3Tm}VVGg9%DaHzNnDfw!o zMyPN!$WE#I5iG_I*>_0UwCF;4^VI1Mxc+5cjfNyzh;E z;%IR{8-v$z;(@f)&BpsQfp)bYsT}V+!&o~_Ovm+Ru|vcRP2FL3fS4&}(f{ctTG;>5 z)O`S4FJ^1%J`^@0?+|sD!f(ZqI%qHrP8D-?&|nAHqFl_w**}B%aIZLuaxWnl=@cz? zvA{E#%?wB53_<#cYda}60`JLgYkN>srtCpc zi8k69Nf9jwSsRmLSwX*~SZ-`SJwm-VLcO*R#R{*u(JO8==Ac-WmgjS!8{+mczRvF*4f1=KE7B>ev(KRixV}bsO%Fhhl2E#+IAA;c7!5b4=Q>F zq2CNv(IdrD6dOM75weOx_Q-qI#tvn z&M!u$@2VRVQ4MnM#$q?*qQx3Q1-O{B;1zIk>#^(G4H9TSNFWs8Tj8)?NI^v;rY;e; z6(I#(zTV-5Npa;Bkc*>)tzIo+Yaxz&qaIU_(IRec*3$T?6zk2YTDO z{R-&h-QI~a=-Z^YJJtHVkdwyT1{i>t`vt|$F36PvdPoZfNCyVV030ZTFjgukli4s= zc7{rs3rl4esF&T~6d8ikWKY;2!*H4G4b3tiZkPSwZn+=aF9*POIS3w+2f*WU2s|qb z;Z->d-jWBxM{*>5A`gZyajqZaDEL{9W?khN)?1EZ`^#||B5AOu2NFqxwd!TM%u~2n zW)v>n{n-#Y-VrTX(24cZ6l6m#>!B&gK}9OWeOe?{LO<_r6PlYpY2nn23fJ6qgO4<& z1>r|1^zOp+1^Bi!XFq_gyCUrF@chwPqFAyLt6(U3D(E8y=-7bHF3@GcyO1Q8C2SYA zh-8V4o`@52OnA*9?eM_aL|Ruvp8}$-q2}DKBv2R*B*nuXxrME#iLf7~_Ky(}Ve`Vr z#gm)S)hv8WJed?bLV=LeBAzRDnxTa14=L2v{IEi_=9A(@RIK6PBRj;ayd*1}RS-ty zn@1g9FLtN}slz{nWKIlaX4vi~-b;!<(qtdrisbTy4iErr#QW`d-Q-;_LCBB9>t){a zW_%`o-h$Wf@Org(y~ew4#p`i+y%w)HF5G~BYw+(zpaVaEXU*8~99_dU(*H*VbO8DD z#^Nkw5VSXw;*(}RDrY$wrF=x;Q1TT$2d}X3UY=|bf1TdUdt`RKMH@fqIBVE@?myIv zvkvv+tV1czO4CqB@nur{eZ9k)*>ma9$c$0z9Z85%KBBURhO;Ty>5$p$G_%cq%T8X) zPJYYIXfSk6itnjRH?7Z!h2%ACDYhb5|3D;y?x@3ybLjcU%@9EA=x2J8NeP8WLrE#< z$JRY$Fk~9G$n0j;&x{t{>u7r(2i^|b0%xQ0+|4k?-%bv0CzSJ0up}sobDJTA6qHMK zvfJ6S*E{a|To%d=`5R&t7qNAXfTmXIKLKr9)L7_Yr z4wlEkq4Ic`AS+NRE{0jM5{{KsC>f(rBbT9MJONh7I#?}ZuwJf!^JE+@#Ooz80hh~0 zxL&SADY*(IuBV8)9+H=_$K~bhDR~8ZUS7#wlv~(qi1s(-R`!A1#{Mj?Wq*-3urK6| z?0ebFevmhDOWw@04YX$dmGA{*-)$zk=7- zV`ByPielAAH zzo8WUQXD3~7Dvc$#5BC0C%+YoQ)HtNZt_nYC=*dxptZ=ckd6B7A=Cz|aqUG~i`)+T$)Tw6f;f6lIb6I3 zS?B@|(q?p!Ee9n>pf1c}-$Q@Yhv?pM3vwidqO*A)?0Yblb>hQOBOiigo%ur4#l={b z!*4=5Gqx~VA*Kst!ul8Mx2Ana?rVJ5!>6D z)C(yY0ZZ;GNXjwe@;8H9n4jb~7vkf%q?|z4NvIL%dPGuA!^=!__$23;!zVe1>dKRH zK3$JVO0>bnhq~SXYGC-ZK_4p*23dVzsMQw^ zvHC-aH2_Mj{b8DQ032@(fhwy28m&TDWetNmazmIs`7WM!_}KXt=>Dfjg`* z8OHZrDJS|aO&d(#nE;Fhx5_gSzXIr7mfR_c5Pws&3zxIMm@`Azk&hzP3&AwQprVjF zGAYlpu7OV8En1V=XhWbKfMS-dcUj0SdSu(KcUVc5C|1pIvM5HOU5|EPVF*>)&u5Hs z*IT^1+fAN_GE)Ucxspu89#pacOFB_A_?%jyX}K9th^vt}LH00K-V6h@7 z7kQr24$a&a{`Z)?7+KXu_I_Ay)C`{mLp2+@P!1Jx6Q{druuNW}6UoNFIr37SeKQW$ zYlBq5BxsbE>64@&eXg|~@;}3SiVDAxhsc+Ok7%DuklN}|7;;8FjTo2cYdN#g=R%|- z@-1|`7^&W@X4ou>1C)J(sCL#1-rY%|+L7dQbf=OH+Rx1=Ok*P%3T#I9!%?$_p~oUG z_ikBIZfS;b)M4wUtJ7I5@>(y8r69k%qp2b2^&s88#k)u5y*&Sn?oQcX;u2V5T?#eUW~jHWM4Gq?)>~VUUAMx8);8E?T?e;X*TenR z4M+<&!E-p~Yu4=<5%@P!Z2N}1Uf!T-;thGDcj*6ymt1I}VkZ1k6z>HTP}_CX>sd_+ zHoB$88FgR+oU19pfzhy0Q$henppqv_2%@`tjTglW%bVoQ?eO2C=?ma9eHN!pS6P&l zx3utAbUv;*`=FvbORW3?O2$PS5+(9ki^pN%Jti=P>siG6^Ki8F0xYtA zk22szlmRb6+eoAazU;YRBZD3;%W`>c17|KEcLt@q(&EPu!P2tKtw z)_9+5V4Mm)e3pupd?sI!ui}bA@VbZhKJdIJt_Q+X`kc#xnQ(}FO}>t}x)FwWcVYR4 ze6w97WsG-->l75~^Q_c%b?&zv$~^Z>k)h)s(n_?K+70=VMb?|Ic6C|$0GI&kT%-8_L0!po(Em*qoAif9}4h# zuzd`SwHIXoZk#^urGMk(d)`6AI4$6~z9WyDgy=Z!txN1vtbK2a*%Zm0nqflF*JGHl z{Nt}aG`5$3Z7<#Pp|N~le&C&Oufl<8LHld2LUR5vDgQ#(&yw;Bx_*_Ee=q2r)8^<~ zN3iThgv?4%_A2ORuZBGPL>OqV)l>H~gQm!6fA1I^@8tLLA2>=_{uAD9x3>~+Z?@iB zS}YoxI6Groc4}L8YFj_Zf3*`EVXP-nwiL~ue~*%dxU*3<-v2}i#B$fD0NMm-yA^ep zXH%vl1hqDAex1dQ7=*FuyZHDZkry)?(J#e4p7KYrUr3$+Tl2+%!?smvMBa*bb z6n2+MD-_O3lkqr|d|+=tY+QgKx)2HHA|#xR(A(YweeFwOkbOBEVqXEJ_LVTzz8YrP z*T6h`D;#fcgKGP_jB~lAsjV)xdLrx~*zOH8bYd3DSv2qbjE%#+jE!vb0!5D7L8^m&JL3CJ+{|4WnHt%CriRt; zSL5@cJ>wHC&)qFA(9!aq#^pT=^u}jL%iHf)FYnF0EN?(6Po&Jt6EVx{os-+q^8SR& z`v{l!amKLODGtf@6Gg0nzn*R2HM-TEH3;H<@3~}5H2cRIe(Sp=j zeu3UHt+xQ3iiZ$LJ)w)vgOIr1f?o{=ryJN#XfI$Ww2D%q&(cc^=Zx2hYSMq`n?^DlOM8| zy^TaaOz(}oZ1%>&gif8JIP_lYV0xv-8h!I)pS!eJzX{3ubPEbR;qZGUwkAMJE`I${o;A8 zTF>)~7vxo=u1B*CiMV%8=k~bO=i^Q(NM|y*&JobnnF`&V>2QEE0}ga%L#ZBv9QIdfIFPU(Bf3V4yPJkc9z0UXBm9rEQjx$TKL(Cv4GRS zl#^gRoyIiHpjz@@+X?%sk%g^=-U(vZT7>?b0GH)@d3#~&IP~n=Dz^dJgaBA;XhtKN zESoP=sc^*>q1(&5v=InJ1q3E0l2*lf$?3__PnE6eqF9$7vKJR20Ivr{m;CBxutIjR zL#^~HF85i>GTm6M@SM|;PR~Gra3=J2&VoM9ZxN8^K!I}}9O9f0hdLL)MCU@7;#>?z zIvZi0a|zTsmmyF$XIN2dQ?$J{m214#T8<#$`mWZ?HD2KXr#CD{w{`~t_5he?)oUxN z7CKuoV?|jF)(Vd=9@c3qz}tAf-`z36?ug|d7Hb%y3u4^J~>rB*(;x_ z%crKwr+Venb@}vEITH0)yu>%dMyejIt4F7*M|<54*X4(&$`AL-n{;_os=Uc7U#-hm zr^;7*kuY8>@UzaLhN97_XKd&uU)FV*AxdFMN864+k$aQXkZqDs6(76MO zoV(#L=N_2k+=mEE!m-YFMB)R`=sW~#ok!qY=TYR2$Kg8X3AojH3LbQJz?051@VfIX zyyrX*UpOxycf81i^AgK)US-{#*V0BK>w#Xb;fx-d3lMQE$2(gOTc=s4x09SxJ;|Aa z4wSWCzaU1E?(^H*h|u>T;Cz@NLUU4NmSdf5{Z@<6bI_mGq{2_}MCg{32rX%vAi09#xIcevTm;^x3}ZZ5p;c7u=I5PaqK zgk5f!1>D{&$IWA1-9D_B+n43L{n~p7+=1#}sPRA2SPFf&2 zo9goi)NY@kHx7N>1nlRo zgoE5wFxEX0X1QyTsMbM^dkQ4n(_p=OIug>EaHV?|TLw-Hpab`Hi?qFU@4 z?<6g3U1e=)Cl#_iM=p_)3R(bAx}j!bI9HK?o9u+x_oV_e63f$PM5Dsm9L|6k<+(b zym=+$BObp(7W)Pq_glo{cZkP-!a(;2#Nm&KyZ^u>_ZOJw?t)_j9BKms8Ur@04>)jM zAP83lvf%naHryNN1P=vr;E6yk>glGteDA4)lcY17Y|n(3`119_tq9%lZcT zr*m#uzZA>#OR-EptspsCgJBQfVe5A5j&|1IrCz4v;MS}#ssm0)T6cDrX)94jGy=oG z4vc_6U?k)O4uYe@v|Tk?|Dz37`3b?|~zU@FMKG+fOL$PLWQ2(5NW#Z$VZLaSY@`{jY!N^VI- zrsjH)DKk^=q_rJUesD)KEx?(M!I>7}Ovh$e$r&?Q4|yS~4qoWoOZ+0R6v4WecbTk5 ztVg|Nj|6UBu`kH!M7d0DLUV!Ij1ZsejTE*XOZ7j{>+cje9d%!k(cNE25JIVNBZX1S zS80;g<861c8UeMY)t6}ufP!!j_z14XmQuSUh}@@;0Ig>T_|LZm`pIAgPJv+HROlQy zEhFhXCk6DJ6nb;47d3k6?j_FyeiIR4X1w16KfMZPy_~dOqwAYV>urhG_qIZIVOQ(@ zr1fDl^tRF^AL)`nVV5KDhI;%ZX?=#5FHEL_^_9s~u)az!d?ba_180F7I0rf8eCQe2 z0Q&_lfP%nOO`+^kO7kFq7!hhhWw$#Kgq9g$1 z`ilakfl(G^zB~GDd@;5estKrn&l$EA#_12~py}H~C$d}rpyYAaWR91(R2Fu;lybk+ z4ATsGy4E$ddcdUhgYLGTv%KEhc5V4D#(vQH5lip4eo9(Dm!=opVgtP;g2Np~>t7NM z{2y!}KXU_S#5Oy)@_*dHD64;WYtvQGnfwJ?P4@M^#+$J8$34%e%ALk8Cz!+1-F3Uz-m>>S8tt z1O!aOL1@?{ z=IgB7JkF}j6KtV*l2w~;vPaF+T`aG)+<+$a8d<9>DpD@CM4he4gd%HriVFL2vq!k8 zDVJ~dknL6vKDLab%7b$l4soIUl}tE&^$`Nw<-}T^id)>HV#0G8xo?w9K*es@1pXX3 z$zVeb@U)x9Q=7EmvBzO{Ebviv9k^rL<2e?5l+^9?9^t3_CkHM&abVN@&tN9rKY-Vc$>L3eg}ihi;!zxA_%w)39}W7 z%r+Q}491}gW}yyuAsGjVunq-mKo2~QUU&-qumc0|0tTTG(_s&l`!EAu!Av-aJ>V$z zgf}qO#V7t$2Yj4z;KPJtPPp?R*rB}ATu=m)__ZqA(NoZy_fxOmeZYBt8SV@{wI;2( z`#9>YM2BSVhv*hv9os2?>SYmcK% z`0N|m?!8P434h8Ws&rGOp77fD<*tOcvsnr?b4at3#7@%FNv0Ah6M&6QlC2Oj!$GMV z>FR01=@!3^c+(202D{f>PdruQ>0WaK@l;J}_nI4tH?5FVE*X4*7f>x9VewJ6=x&SO zY*Rjqm)exq;^S>9Wv#_0hJ<^OyAFmZmTvJfMc9L2@yXNy?F^F4;^ib-J$Mw)MXi+76Me&b2(4kr5UFBwc(FYVSt&sq z;1JO9MnKGm02V+d7C~Pe0ohm#d3X~fa5UuO7#N9TX$qFWMEoh-hNW;Pj)VC)9;$Hy z`~iOswO9sE;3RksC(|_iB~8C_c!|m;{5AXyr@(7i0f%uq9LL*f8dgFJ&VcuDCY-@L z;4_>B=W#ZCi+4dQ{!ac2D`wzamWK0K6z^u&;5{q{?_~+Rj}_qkY%DHd6LBG%gp1iU zT*7AIQdWfzusNiN^YB5o7+12DxQeaEhuLO)ggt|6*e+bl_TV~p5ZALK_!#>qZeVZY zM)oe&biHGIra{;3nWSUewr$&X?%1~7v2EM7)4?6vw(WG%Nhi}Y@0^+QoO9l9*XO$S zZ||zLs@A`eO?#jpICITcTNb}j?fn455Uph6@0^n!p=A56-LMlnXJz`JV-G9XFax~V z#NRGr7?{~~@l)}|W;l7BJh@0{@7_sY)1>RsrwCF}8Obib{wM9r{6-^a-UtXt&?{DZ zG06!|y4DG3;PsJi-1!u2(-xW?`Xgu7S|$wl^!P5G|!1E?10oz zIu#0V;MyNqscvQB?Z?9K)u%i)`ap3{`v%fh=2w+3HIFfWN(TZj%xl|3pqV8?M-CGv zk_C)oBBRZl`7ubezXJTgUER28<%fPa`4oY+VcVhs8~Gsw7HyC$7i@wVWJ104_;g8@~7AnzCdyqbI8A=i(g{Fsjg-WPnCto^gqDCr#nS)kN ze~C&RLq(DMf>?GLGj8xOu~(+^#IA64CR#O0T7tOPeebdA$ywCZ%XXcTh(VkvlXU)a zXC`&zls#fd+%8J|Bt>g@9okw$=h;z`{gyQO3)vT7%qK;fzTzx`y*uwL!o0ige9-hp zcq#g&ilA3fNQ08fOt=l!cOfY9A$URXnS*pT5%_M@$p zls=&GXfX&#hKkDP4U3ULtA4$i1y%i*{+P<>nkfSNTFSZb@-9r#e5-4ldGO(?ixo(^ zXicscd=RQA1(WwBa_$y9e?eZflrxW{B!<`zA@*P|Z30KK#HSp5Hw5m7M0xbVj3M+> z_7mCSa7;O@VBqf?4gXI|V}yLNkOR2&;4@>q0EC`mjN*FV>P$LY|G!0my8q|PxVeoa;q(RMS zLc0*?7W-nVEe6tS4HWYT@z^{}omz=!le;LvYx;b%y-3jOWbDeaNKk+Kf6#n!VcY15nNlC~i;7%&=qOoAVuo3!3u+QmajUut3 zU|0xYM;MezDF?V8*r?IWnZ@M8bq+N%877VE-!MK6WLQS%Ff+RW*REcfcbf zbsCB62(L6>*M@tTItc{qU_}RY2NNQ~XR%8I184>>?t^faTZhWB9`(uI6g8|i2-!P@ zfJ}5VJnWk}#&;t=%cNx(G7Lw=0&;CMr2hSZ? z%e`iRB}v}Tqo>ZPz$Xp9c;|N-%@D@bpCp734cbtkO@yo?ke!ltp|(-87#vO|>&xa) zjK54%$P2!Ok!Xpod?>;t6L{x0g5)9^eZ2Ar`xP7*+_pMwJYM0R^0te~!EG6$tJPYp zQaU9F|8}FA89}*m1m<;?(D^8OejBvzL*-q?ofK0^$)tF%|OfAxfY={GvsxG6TtiFCduJp{qToVOkfgrfpGR) z>gY8abd0Nhdp+sw<`M#qO`eRcl4&7JrCL$0Zh;GWHG7VOKb=1?=A6vaOTEMNDsIpN zx|bz$b@Q+3%qW74?jTrb>Z17uYj_&XOSHZH2Wjn{Ngi%`;sgiq_j73m=m^Gr6j9~y z7V)LAQ`{KDUw()2S10VlqAsNqY$VYrs5jBc) z;h8K$)RZ!h#7hh8GuSi*!`SBAr5=1ZQLLNTaHoMF|A{^H=+<;``iDLI`tjcZL_EbD z-5uQi2SCJK?M_b37(HNjr-#o|cPn>IwgY;`rYEf5 zBSFa@b|`^`TDBPaLC!Fk{9G(hI8fbv*7ppx24VLp_v};9^5fxY%M%1EDTbJwS-VcB zW+mEGgRcFkbEaIwhks|ivO`Y@OoO<~MyR=tqo?0)r>eGFQLfN)=eJG!x7%)@s+<2< z9U2uZ*ygsM^+P~XYL-BmywOmNZ=DP%hweB-Vy1(-J^}0@%egc(^~Sa+dMal)GrOce zi}?I-rBFDv4>)Rz!A{;*rG{SMZhdVRSk*?jX#>N0|CmpNx-Ol88Zq0)chM*W>6Nd3 z1Oa8pD(r#fuiymEHt=^6u@(Dd{RjO3TLb+8Z4cd%Us%Fw^RTg}LUBG_O(OT{V+x}^ zeX)-YEaL9y)#b!MIAZj}hq74Ns^o}xvFB0g8fhh?`OoNK3l}3S8Mf{ibfNGp%9oZ{ zfgwTGJ(bCMIh;RG9lc`D%F#L$R%{&W7yrU?w;Xtz04UcBE&`--_^?Xn_`PF2 zcsrluxjuy-RJaWth~ZSP5f*~(QcMelg(9NZNbBFbF$4Die z6KavpnVN#-ydtN(g6O=WYOGA*8<>NT#*fI#k0<2;lnf<3&GyxUbZ|*y%tgp2tWL>@ z707&|8X+w@3FE3dvmNm)iUDG7Oa)eK|_WT!ylIPKNr;Z0N4Odjf3^uGyaw z&IfF)lplEiS*=N*)uzS&0G$ocARrX~r=)*1Z+rXy;@yawyPE#Lm`-(jd0h?c0E6>= z)m!bZ*uikW8?nNkde;yhY}N~`8aL%tMBq%4oNrNBK%>i)WN|DITOsySyBP>1vN6eZ z&W<%3U`@dZ%*xDsn=|}kWnp>9p!{OJ^u4|52IJU0&3)>h^A?)(_UZrlSTF^dJF=me zgsy8!wPoZaa33jevr;$Ia|_E*qfs&;skiL=W73Ms8EmSkyQn9_AaI47m%mg3Y5Z&L zjHnksRz3`}Z+BpPUfqoKn2dU`qTHA)M|v)w?E1xQF(y2nbF|G&F+*7*QEWKhNl7|p zWIeRe(acSk7>2dThFpz#y@tY)Z6g##Ng^G~4GhBTTauDbC~Xp`-N11D;W}{~nZ6j~ zPv2)_V2F5mp+lyah$^Q9jQ$emHHJsF%L}S^4&L!(t7gK6sqaFK3-xI!8M%_t@}k#X zH+4m_Y%jrAV0}Hp5%soQ^TL2;V0|r?l(uJm)mb#gK~#snsEh^3`Dlil+;~h@iYZX! zhW_B2o6F0|T4?cbE2_eOv(y-ZQRFy>gF7#s#`2E#oyWv{c8e|axb6R0ub+tVlpE@R zc=8fPk$Zqm|JLjGY>Cr*e9>Nl^fzt)!*+WFrC}>$#-f?kKwN;RA7#$M0lgr;@}@U7 zgLz!a-mJD(K6`ej%?X3BUnv8;T&ZjBD4iJ?Ja&_Wj2v8$WQ-~K*MU-&IcZhOrOH|W zZ=pP;+enYHnqsWYPW0S+-U5gg6Z*RU-I$Y8!`Mi^PqA!wBp>Sg-7tLQVBv1+#C21R zgNVL9Qo7G*HX2^gJg0JP?F|zXRpGa5VXU%CID6czvQvPJuxfvI6iSV1FHdm_O^!cA zsNSeK0m!P_ig~snoK{xb%`nW$I|bwzrtuqC)_=k>k6q~q4P?iWaqJU2eowx0llk9l z50O}IKxU6DUzQ@Y##}@yCqz{p(YYx&9K13r2CVWKh((>+| zWr6U}O&u^Ske)P@W*^&w?=<=nR7*SKJjiz9#V1Y3qVd&M@FklRGZ0Pmh>HQ~6 z{3r7eP}igou-`x4=I&gvvb!(tL&dKHzq6!Ry!`(&diwd}ja*TW^bS z2?`?#j^g+Qw{%7r)V91}&K{ur!rn6i**oO#Kl^|Rv;ynDkT*&C6#|@7b;*sWAo@8$8XW&nu3=c=4-<`-vOB(31SaW3*r%s;aP`T z&jKcTM+N@9g82;>p=BIAWg)!5Iu6xJ5{uAy$n;h@E+tWBK2nYiv2ngr@smZyz z+A&GG+u4b`JK33gX&T#En;E-VJ6K8@o4Pr={C~VxI6qwtobLkmD}|*VsEv?5*^V>^ z>qRI$`1MIh-42JHMUUkAk{|e76ZXDq$CTt;Wn%YH#g){wm&nK{IHZ%|>n()X_f+a-^0x30~Ho^L=OEcfara&U%!3J-$OgG-D(o5QCOTo67$tDRmZqO3%8w^gDn&<|hRt_*4NdkY7t6_%5)<(H;QpC{oL35Q|c z#84sB2vP3CT(3d4l~-E4azNCPpK!o+LO%ZxS-b>;Yx$rN$fX<$D$na8P^~JJo%pQ} zpw(c%;Y}nip-YO@eQ@Fjv@Y3*cd4OlNg4O3`KT6t;t#Xe#(~B3>PT2mMRMNAj-vT2 zvOcT3>UaF~y;9az(w5wor@4ooZlj6wQq#3GmKx*_x?F6>L}5VI=E<)lBb1N$;5s2w z74K0tTJhT4u~0=CgL)~&jJ`}Y5bxFAUzXY2;!wL!Ymt?Mt)GJ1D(Y%LIBGJ3T;AJN!@x4&48V|YsVcjT zC)^97VN-jQm0@rkssb41==TO)qK>lEGURsQx58417UgleDO3c4bwwg>qMzURS zV6Yzdc37}}!JEF}2>Slb{piQ-4TcH^CriZ{#C*8SETl}Db8;&j-7{6e9Waur$X1rL zjF)L(l2XDyBTh)Hq;HfB!?*$ud?pd)*eFJ|R`ikT3=O7H%Ukcq$)H{z`AX3eT9&06@OL?it;?(nndBba!BXWQ`C^kUr8$AeKyIyn&r}QFN%v2sM70%_d11KPL{N8YMCtymx=WFaQ{pkIX#5@FCH|EYH8ENrp7~t zzW^6C)~W|&o<4W1_Hky!D(;_CUC42^&t6TQ^dqCs?1t;43DWRdm7S`DNjL{qdgZw$ zrFLMe#{=k`lZSg#6c_RtgmRttil9w^%YNCBo(Dog?mv2=Edw}9!Zs!z>~vlY1JS7B z&5sx;-Oe2KDh~mEKhsrQnF%%y`R~x(FHC9cGmLtC5!ed8PtP7#T2(Q?`I~UF&qX0*gVbi*;g-0LhWMw_4yr?73K8pIYh|Czhbp{3XQ{ z$%G4(k^#(6dpDW9!Xdj zS#Q<5BRVWmpMt?9;!SE5E;5JMeL;v4adJcFo1v_<#+(M)+c6bBxsFlP$6M-)9Rmo% zi4s6-6OX+(na^UFX=-Dc|K98DYn_9i#;z7poR9@QvXA_q-;vP`2P(-(HVdtc!(Gkj z!DY#Zpyago-2FxSFy-bCI;s!kljhUUq3O%oSmX~K>V9~8#!hA(7LWU!@>iPE??BvV zbW?0rd8~X3-0%`exJSWNa+;41+5lp-4fBx}J%@kis{~68{s^L$bnGey$eO@*3G!}V z{Wll|yJ;>*cu`x2_TfaU|=XMVTO(ZqFBjeiLBy55ZHK9R;JOkg=(dBDax2Fn-l z^j`ITO$KWw(>`!42G$@`9C^C0(hG)3Ee;SfBqxnr7#%iT6c%7aGR~vK6g@b7h&8kJrDVdhv#NpKe4^CAU(iHV+J)nj3^a+(QN%qfsHcr&o zwh3W%7O6>*n`zPj;T=qA-u&diGT=J=4WmgMLry}26MWFiOSl8JAdIe;g4G^hflUZp zyp1+J>V%%+!{;e?YBy2|h2wpW6K}8n%j~n+E}|~Pc_EG$=|Z%Q0RlwUgdV zGnS&q0yhd6zhbha<~{_{(g=BUVaENNrso>m5F}0&KD~v5`p8p2=<a9pL3i$9&pA!mp7yNaW3(pyE1n1pBH1zrU#}q9#35#q!5*E;h!k#%NaD zb+-CJ3T`Np$M7nOV1{Em2_c%-6w*m1?i2v3)tUP?3|tShG=muQq099s#`cdh*7BXw z3y|v$#YLD&mwkn8cjfF5e0{&${ci_VZt0r=LcM`A#;@N-4GKpCVV#jm5udX&BRdLA z&CYY9+xvcyYfHsDXFXU5C7s%m{CJ9d-TBOGSFH0McCx`_N#^4`O11dnD&K?i& z2aou+7o9~pw$5whbPslXy3bX}E4_sQg-Yt~@TE`v*K_?|%nNNv9ubJVXf)oL)$*m}A&AN0ozO_eUsOP==ZggAqREwfv!MDl^hM z(lBz^zjN$}@S88r@>|bkXUmpt=4NgJHl3#=f1R~g!#fBqT{IkiK4naMTHKEeLy+;e z7mtm(>65wvqJkg-{LrbkT)Lho}R@o9#J4c@N?P2MQ3#&I0bU8ETs;3 zX!+LV|LKuvbNzM_lkav<5iSi$bNoI+9gsFi6o152^0t*P=5$*GRU`F~05X=X(!1$k zDgE^mW_@aCV}Uq>t+<$kZ23hIjPjJ;g%IP(M-mtu%e`?n7dbL~m`c_PL;r)YiS+D1 za`q<{k8coX?|AcHFf9X8bf%?Itt^6ug^0B#$yDP$CzF=UpqrsE!UKyMH63tg>a)M7 zc4!sR0GsN;E|W5CiH)lR30z&=Y z?Tr8_x<&S^;5sIqENArAtgr1a`eT^LT|}^*!eI%C9YuJBG!#Jt^FBiSOXbdo272X z=AkydAQmY>39p;3(M5L3GM6U$Ri{gI)hRCaCpQ)HCCgK}uQD;REfEIvs}2rXFbJ*& zwH373J_N=KGz`5`mbYnTCLY%{iao#Wyw&7l+jXTy&SPIMj$4Jil#$%!#vmmG`p)DW zE+-j3a?Q#)k?(BaqaZg{^TTtM`ia_)RoLt4)<{u^k`MEIBt!sEu+ZfYoWM`m6%} z;){B7!+0YB1NFwBarKAWDVz8b6EaSg4wxyzMHir%ExiVYOb=%q`IefJ6$M5sOe=kQ znOmF0h^8HyHm7x`zx;Ar3@dz)K+_XP`6*sUTok=8v7izSjtD#f>?Q;!zlO2|RW~^9 z)98qZgN=9IhMnR5{9Pji;gl|HH-tgzR>qF z8pFz^!tUzuv$<)_B;=+9r8oF5>?=z~1!rRP4{7Mor~)Fe4{}XJ-Egg28S8o}2_|l| zxKsfPo_fQPbZuXIoU%7tTEqips<1;UhOL(fAYRoVv`3-g2uM71Ww*UmJ@9PhVbS_A2lCylqm;) z8Y{utIGzW%sh3*wqHZ%pL4<*sBmM{0=|r5 z_>CrzOgjt)Bjs`F5v;{yslI$Jc0GI65v)AgdOER<+I|i@ISaxkKjsUws>4heqKjwo zsWG36)W$-u`Xd~iQ*ap0vRZi4_yMy*7nhz_Ba-GeWq#BW8S#bkGwL^V^pW1MPBODU zN|1t_xo$ZEY($q3mbjaYlQ!tE?X~a?7T*NXdm6U36(|&1>UMAnoW`4e> z(wK-%2Y*)0cKkk?U-|x0*gwh&Lb}=mPdyrGpVgM+ZW@~R(6q%#ldLR{0n%N=d-!oa zlg2iXVS?ehWCFIMb=f6mF115rdvNWo=nN+DviY7E zdrdHI+}DjEqbs4y6`#4AW;__`Ay=?IodwNI5;Xi+KB$psyDRu6G~GeC>wURLk%kHhIRh+fRpcnDh16xz3;hfS>y4^#DT9mdR zrJYfkunP4J{L?frIdg$(APZh1(hoHXrti1&h0TXr^bwhBk89%_>czgSGc@Iw>}P=f-6mJU@@6 z#UnJi1T}>(gmX4?hS73LLxJkmQ}X3;yRXYh8f!|ow8UbHaK1jZVT#Ei*A(ElO*Z>u zuy|tps77QluOv5SWt+Gfe}t>r8E?Ezc_Y_L>4!h*HK9S69&tc`?iBPeo~G~&_S^-F zsCP(k;9eT3Q;a>ppjR4!-1mCGx@&sF8hh>UH-KwU0>to>o)#+3(*!)(>(#BlWum$# zquSPl8d6j675_%5Z`%)VXIg)tTP|p<C*W+WJpIYOt8&kIg1$f6L{0UNDG$Bx)0s_vu5GN3u@hy zrXahqY+#6E-)UdW-Qwg8ys}?*DF%{F@=t#tvq#R{!RSn4|qavBk~YOy1hn&D`OC*dcMs3QAkT z$o?tVu{){r_Y11BTcv3>!J!a`kkz5eC{P{vsweYXn}jG(W0E={Z@SV9()e$npVYwG zC;&M%v2B5+#aUmTx7Hd~28iZDvkFP=$d6w{5lKW5w9(qOhF8mXZo&F#Bi$8V^b#LK++Wzp!3;XzgOr_|*g+*cfHAzHO25t7wHF#5#NR*3?BtMp9o`?Ok>cs9h%zXn&>eRxb~>5Px?W7_pn zwB~abhI?w0Punow(oM?lG9Arh}yzDo8YN z*gsh=)m$7Bj;ac#@g>~xRlOb76~oU0)m`a-c_2}P!qomm<> zy)8HkO>;0R?vgOSZ(HHr!ht2JzW@BG-r|-X3ayty0_hBo6<1L zvFeOaa|7I`3~H5Tp)dQgXiNbCYhe{PPA)K4498=RQ^?o;_Xu>@xO1<`Zkx&-OyU<= zO1p0K<)&m0#wvo=ud$VuPXkF>=L{}LCIBI6kxf+XP*s|p57A>C`EpfDU}lQ)jg852P^ z*_t+*f1idor6FH0QmmiQk1;|u>6$jy46*%nRf5O8Z%k;^M-)Z+K-B$BfO*zPXa~;I zBSsm6B;U?cA3&#V2v-+K+noOw^go;1NA&Xo@~^oO{Lj_?&k|!2bF_1}cliI^ zG56#thb3Y3@b9R+oFmF(1FNo5VVStl5!_`632AASMCwb@92?YxU^+wcEs;yUCXguT zC+^da_`kLOsrU>|m+*r-~C5e6Ov$&{!nUxy)|Z}m z!P~Z4LMYs={{48`8GRL1!A|DEt=U*Ca3NCSF@P;vFkdDcAgE{R{giKHX`PV!5|Z79 zd16{YSn93wVqq#8=(h#&p1M3&<8zck+7kQZguwvh(?VRCd^a2i#ExUf^;d=`yTy7? zTx3KaV~)^jB-XTMyJf59t6b{{HCbBQw$l5@wU4Zx2WaX01E5MNINYP%Rjn2`%btEV zTWltfywW5Oc4k-?sCq`3jPMfg=*=i^2rSyrhKT^uXvTLy|DOK}72BR`wR4A@Tt#Z_ zcPOQ$-RJ^U-Tlc=4QcHd0$*hr2sJ(of2=|x?TdHZh-?;GqDwsY{lo=>bIkf0r?zx~8U&Grs#jPzHRa-;|Auc(P#PD2}3( z7a7S#{k!DL3wQ<9b*!*s6h=iav^~gM$pkH^q6L?=J4@NOGG@(hwE69IE9Eee61EPm zgL1>nlXy5kB9NmQRl@xIbfzwYebEEtKM6_T2d`qcgY|inBZK} z$EOk(kNDB@kxFD4&nH!uh|u4@!bPcUI3|kP2^8QmojziwKkkNwT+GksLNseHId5}t5aMCVN7<%q zTh=Lq47SB(DAD+yJc7^Kp|_{2HICRte=q4h8~Id_?f*Ja9wKBaci9L= z+N#vfYjv$J?jgO!t&sw6Z{ZCQpHP`h2_pdGo3djin!3e|d~_2~;8Rh7?KnatqMno6 zu{u^u@eo`tkQvp{+E?6`Xu2`&EZXKoP0Hj5a^l{WFs_`H^K=*!4C--4x=Er{<{n#m zCBezFLA}9!Ju-G-*pr>LZ^X)%X8&URl8LU zf*g9{$V^kK!FQAZ7)?)D8*`%b1UciA53oyFQKvOUTr}Rbo6>UVAx_F_=bu*(Fc&1- zxDW84*@VfOXh*B=gr(?RL_n11BE;~MmF$_!^B&oALl9uOAHh{818 zF#BYN>!WuuG|n9Ql3Ox$8znn~{y4rA>~ z|EV+0f<2>fjP@!V-2xEujq-4nt)i`DztB=}6o&W({S2mc@4MgXw!>%Nhr6V4*NRYW zuYO~)vWTH{L)$DhhwPuRc%ErWw2u9WITgHD@>1{Ue8HX$MOU$)c!Y@CkSj@*paoTD z4gQ99JO}u4d#X4KHvN|Uh_X9UDibsf=~HgWQ(3pbt+?w@D-AjbAk-9@!YuXjx5@3N0p&#`pK#) zBGQ@5uPt?OJxB>}B$rj7i=huOgG49fiOcQEKD=wEoCWnZ1tYWxH>ugl-f5WQG^_99`TC}q`tEhyK+?Nyht9X-Vq?p6 z+F*HQugqkjzSqu*J1)#YxcTffOJ7-JT`$htmya)YPN)T7QKq#aljj2iF<1)5S(pxiT58FDA%%Y+gouhaJn)O$kxqt ze#@_5Gw0R$lpKAl(qZG~I_BmoHNFi;cxs>$=DN**KhvY-eix6^$7juWxWYRHQ{9n5 zDDjzX;=Q8{tZ`4|ij(++oVg;_nn$Y)$tBHOA|!TPU>nE#g|G#J%H0r@%M#pc3JG&b zT#!7ggOuorF-}7!tqk=(1nsg3->eDsp6%BE0OG9_)awYYQxl@c9_(iXd9f-IK%7S; zAUsQ*h!jV>!#l1dnV4|jB+@0N^|ev?31dFJFOhIN^3PY#iNLbFz&VZ+a=lA+hb?V3 zmTh?2n_{Nbk9B?9-XC_^i4ckv^b2BA5Tsx8e@k)udb|(%-gq26a~=d)tP1(G2Y-*Q zp8bD5#dyDce$qSD0@QnI+5QUs=lsmaE*c0V0s#qC0|6oV-{+@;gP5JQxr3XCo14pj z$&Y0K7%z2+$A-%+a;74Z!~=C?aCB^CejN<-AK(b!U@)MdC{z>V@baw2yP@M0q+l_J z{=r^!UPQv8{F-_g`VV?Yz3zQ;8d?iG8d~2PJC5}k0UtNN|Go<$-JDP~C_| z)S$A;+NP2uHW9DIeZMTl1aZrQd^HmcR(ELO zJTy|Fpw2-t^nwk7rKgjp$!K(u61U#Bw~Q6Vrnb%ifKngD05jG_<{a6P3y@#T);1+w zT*oKGl3k{!cBGFQGX-|B(GV}A=VKhi;zw^LT9TZ+mN?u2yL@4 zk=DYhm=4UFE@_$oxV`g=?BwMmCJQu0Ewp4)&@_?DT%(5v4Mg{2d~s5XVs7yfH>Xy`$!mneSg^d(dM8+lzDHj7!?WTzuzX5T~ z)I+CMa8X)Pxk48(v#i*lOsG{Mfr|ph(PZT+MmDUWpm5NcZ&=8g3az1*_N=v~monc6 zdzD?XfYYQVw$QDU1Rzu$wda_6XgZZFUhN7h$EupgX88v%snk;)hDY|vFyA;0Ee)Nf z_mnSXN#>4{cRp{-YHd8ShJhV`Dj~;*kkWO+k3%>2H94As3PmG77Gqo+m8SmrFBji( z|91+7^mv;Q{tSx+(XMnaoDrD(bVxG&Q3H8c7#?{pSVc0*as(Q?O18fV&5*MgpF>#u zcw;BPHH4Z@pEfVTo|n3B(ubqRu$0&bUop}jA>3ddhv;3Ih9F$ zWAoEx|3W_*R?<-8l|xb`aEH}$@T{h$p*5tupJ>lG*w7R1dw%>ugCjLt{5$y-6GNfP zXh*6m4Nwc^65#|L(|p|L4K;A^(sLyT;KgGNU}BpO#AK;B=@=$h&hvBaO!~YqE>|QVK6@(hj_e`dMp#Q;k_NUi6aCB9f7%x; z&RZ6CB}H}b)>P3L`jIGi+gIymGx0f@Y(0W8!Zt}qzdraC4hEwTYTYh7e#Fon!)d_U z$XyIYq0JxSdteNcFi^05wR!w3i<|9biCZB!7QDm0RQShou)w+ucS1qu9^5~j05Z;g zkkIEK^Mv}zT65kI`41w)i)ILfFOS3deYvD}Jbn_Q%jFA1wg~Go&yQcgAKz(E`A>V0 z0b#i+n(FFDec|&Ae4Mmz?#<5i(V z$bGeMRr3kwz8BQhg-X&Qiu4+H1q+Xs;_6}kBKEnR)w{onFFK2BhQ1YtD@<#H`^z}) zu&vY1iO~I2A2dmXl#k+7)|zUGDB$Eno-P}xf z)(-5se=3BTLZWyFZ;0p|2*L)B&exXmJP(@v)D$MXn1NZk6e#E|BGIPPp?Xav5Nl6% zr{t|4sJj4cpU?! zor&8zP*pUG%`aaL?ghgSjW|9qa=D@`PF3D7f{yr7x=sK5DRDFi1QUjr#g8ZB4q%RY zjD7ByQ*KcVpJ^?mQ?e}uk|l!a>?SL>^5`jcY0lYooXWehGaWDRZ7-n=}tGL!N840f6+CJ*@_-yMc0 zEF`a@TvaASddTD>zm?yz6H}u`DWcCXZ4I-xp)xbMF5ILLtc+E*A9)}+Mw0UjCCD`U zR??izpR^b(LG^4YWP_vTP?>15Co#84W^27#U$RWPU(#K)vKI zaahBj?*U{}Wrh2VLblmomF_6*m!j*T$uNZ<6_y%$qS^zK#I1CpALUq+i@BHa#ubnF z?%w1jMlIq3CsayN*nX6Kbvg7aODUD{qk6zg_g6~kuojWUeUBVY{)_sH>hF;fx`OuM^noF2 zU~wuzphbd+)2h`k+0QJgk2n)jIHkB{I-Qfl%1OG)IFP7cDtIK^cV(Pusnh0@e^sH= z&#H5cc}52|CQds9@exZZQ@LZuuXy7^$*~}FYt-bb@S6&nQ$5kimZ#3qXBirB7L~d{a^cWR zP-y~-?8J_f-N%Yl;n_Us>GDMg7+7+t6{vBv++jNe!?Ⓢ~E#Hlj2I zXTS-7rj86*LP=6gjfJv3Se&Y$vq{ju?c@wvg>rq)X@bYmgni(OkoGQ zj8R}^OwDooOWhrA@+J?rLrbeT_5uKEJTlR<0LFJzrY4D5T)dLH>`xQ3=%{ZUIe9&` zRPO;5>6#+DmPY&Wf~XD5Kk2t(DE1&2n0?980VP0x={Z1b&$ywun8?b!Flo=2O@>b% ziquc`x1LII7n!c(KBCQF%bfOH{gZlp!)Pn;vKVHW_Ravx+gryB)Sz|Up$iR zL44JwCQfbdcH1i=JG?b6RdatE{HYsAs?vS@zn1_ir~P2Dx1!lD z1?iPdUfPl9xTBe2$5VD!AtQ>wS@LN1+ajeR)kvGF-QjUNvv>A*qCV3IrlO-cyI5hJ z0>?^`M@gqnG@b^eV-I+Tj58eH?@ue#67KKG2nFr>C)sb1$s+!_tNvW&t`#IHXO(-T0qpV=6GOt{WTq4Ed+1nPn zA$N2n^;00iH|dC4t#KRD&X8r9S9setPd<&aUM^ue7blm$O7xj`E~5w3!jx(+u*5C8 z!r4U9G?h>2_#s_!)w>~8h1wo*rDoSrx+wul^S!ehw4h18PM$(;FX>SKAw9i$@vO4P z3*$}I{;wF_3#mWxNW`H(gCbWuY89>KWs()tY%%uqFJ}Y-&JI)H*`Q#?A)1JeD{a)0`To#n5Vsi{AC zie0_*rG>1k@|K()PN|)i-)>IaN|1<J{ncv zpN~WFoETN`18I=l-;kUqixhdLX+jivwylLk`^dh-OLvhx``J|cKHkR&Xy?i?r3M$C z`NMGyl{O_r_hA0zkAz4BT7?~j>~vEBjNKZ~ftuKO^t6n;O$4JIqSo_kuqp2>VMHgN znX?I#MqKy{uo`*=BMBQKfppU+WBdVmMAv|MSdxGS7!L-!FlMc*ca(#%Vk53St-ARE zdsBG&)9jf93yES0fsE=imlwUN{F=@Q8x8&8mA1iLP3^fhguxpp5j?#|Y&*L!I>N$t zn9k_@wG_sn)`JQThC=siN!zuURa@e^8DyXMLJscaXTfMUf$V4AcsFm@i@I2L&b()i z6mr)CZsvp+*WIA9|7SexaLC17IDQni|f*u z`_j?OB2}B48y>%TqF_mx>5PBD%A#t^vo}NDHY5{nDU}g@tc>N!GjMfr#Nwg>V3Sk!%i20- z!Q=-tpqJfMEAMyP{emL9+l~d9j6xIh~xSXHqf$AQA=X3{t{j+^=jbFmhO22?p+?{FXY5|h& zr~U!&cau|JApG`m0UXyy#vsF&LgUIF;?`%wFu~XFea7z$rv#Tffi^hL?Vgiw_#RUv zm-|S8CM@XdI#CW2B9Hg3we!-kg7XxyY?#*x8Hl$1d(BP`?%<;r;~Otxh-<;Gichr= z|4&f-$QOciUx3 zt>_8G5L7&Rv3d4^h+9(@7eRqgn4d4M4Z@;PiNcJtAB)PpOT1wLRI4-(Pq545rPs8l z;W8+tP%C+4CS5YK(by03RIlLxmW%u6iwoaz0o8?N6D+gxaViSIoXc!_g$t0&hIYzE z;vrKjAX6Bv%3>)ityENFda1ZAA_}E6G>vM(78KlcG}`&7wZ~a*bt`{thFzB}SFM~i z08kpRLP|$1%V(A}R(0Vfu+cPDdEuJ0yL1?>24Xent+FD4%kyq4&fUSBR#r{Hm~`iU z2z8qCm*oE!Tjv~{Nz|?Ti6^%8#Hg z3lYm@sI}%dUf`|!vC<3|z4%F!_6i(59J>DN3tU$Vdx&TF_!ur}VN1@=bk;fN+D@%+ zV_m1U8{;Q`L(tf5&|UVBU)LQv@Jdr|^xGU)UeJT%%SnhzkN9pJ7cZp$wp$G4 z^GJIuMm^1sy*!o7S@;Bg&?)U@Lf9p<3W^C_7|Uc6{|X+@qgLLM&pl(23CPPoa7Ug9 zE2SRYgs|+NPhR+h!YgYUrK=6@O7;_!CF7U0pB@-be(07zgt;O8fMYVYe4%su)xg&o zBf5jh4{k7ITE-7%FhIK!4!`3X(~s}LnNqnc--)o*KVH58X}@oGgCrny$moIn-0%GY z*QqG*`NIeMGt96T=7lgturItHnK+VwmalKPfX|22OVty_M~3Cg0fF9!QQ?;pnh%LZ zf0_wV3dc=KMBlOrW{U0Z^&aVqgr5AN>rGA@@h3@cz+V&M59;m!|H$jz?L8`5|Ij?& z(3|L5u8*^G*+Ze1_-mz?Nq2&j!d-V_!ffA06kq38E@+JI(35)(m5g43ZSC}PzPvGb zmT7kio$m9pPN;j|XX8M7t`zUO#%sijy?3x^pB|&$>j6@Kfv1P(KU<$*9s&N4Pxr); z5FdpO3IE6mE|^O0FbMYYf(JesxRE@r; z4(xIxN+|O{mkxMT1&pdG49jpMTJ~nBErD`4cIB)}oOq_jxMQl78euZ;uNJcX9gloy z>R-c6?t|A%1ZUpk&;I6(zJ4a<`_rAIvF{pw|A+_ox2(&$cJ8`$-?{D!5K8@S*dq2K zl~_pbNe>(GQA8e34pn7OvKf^QukT2N&^vF8k_#p*6Bbu+$IXxa$K?B1Syiu1dMrm# zfK7x6!wOD`Svl5{F`-`2BW_gNf^x`?VKH$=us7R{95Oay&nBlj|ln9@B>om&Lzx{-wt!PPfUIBM7kG6H}k*f8Xv zGSZG_Z+-yxA21&{YPbkxyt>~<=!v-sP%fpXdUEsaAy~VtIw5rVOa`!jZ)r2vIGRwh zcD`>QQ0-uG!`23ybml(I#wx{efp9kwO82Xq(tyEu8#1utmng#>tR;4(C4X&f)-<_g zIap3)fpL7E+LD#y$Wx)Oq_IpxFG27-RzF4s+Px@sU~(4NTLP!Mn4kNUHjv01U>IcR z2x9FHz}6!F20M@n$VK0OXP7d@MWEtEfckft!G}N0H+p?*e&B94JY3%CHTJz(MplA) zJPK;?mW<&2D7xORjvF9Bh1w%>1Tz>ZO|D6z3U2ZfZB%*WB!F6h)DsRD3uT{UZX35< z(*bf@p>FCfkw%ol@PQw3Q(I_BUci{05Td1z8k$pB5sR+7LAMrWU3`@Dq>|!C*(v#P zu6^lmC0k;J;qR{Yr+aA&I1;8)TRC)!ouL`C{Ze* zR+%-hY<9X|1n2(Z{O9hN+O(B za6v_dk|0g7Am!T>i;9ry4VKCkH46htb!3fzNsy{rkX`~wAtFZr$fT*FW+kAJTOg@0 z8A#iLbhsmuKwK>D36OY=!h>ju9}&~NE`9(>23p0?+!T8OTv4*dMZ`P<`g^pQK|B(% zF|M{|iJ%d^Mu|VaBh^qqGRUMVwvbfHe~N~A$4E?gj*VJXj?5hDRrC}}CuE+P+}aMD z|3RF7zbsKj#428~=6$8%edUIQOEEYqJ~2BhhDZyQ=*kA|B{mBsZHny`VXuoa=@XFN z$ELCl$(v<>Mlvj9GxjK*7*VzNg^Ma%FyykTYy&NRw`bey06xd{;-ND4OPp~TW)6wq z)edohBV}n$TxpXMU20j6bA2j6%C3#QUui%j5ME)ioI85^yj0QBPn!8nu@-a?A(2>A zQ}K=AMke-&2v=Ho47^LmG1Bir`C~_&d@6n{t3uN@Vm1MWyw9}b{AuFBc zML?=dx;NI_NImJS$~z-f9!T9DK1#|pEe~V{@eKy+*@x>D&H|4*XkwXFqB1Tmy0b=D z#16EONqq@AtXa`u+aqIG(6oP2qB)^w2#Lw?M-c3wMB+y-v_&T4My^ zBBZ#*GGoR1fuyDQk@PI+upN;LXrMY65LWnz_MaTJYf3Z@G;NzL=|>XKgn3QKva<?!+}=6 zDYo_X;Fgpo9g#kAVgk1S))kV5Ef3D0jyZ_FhWL>mvp!!j`l_Os-vhv@%vop4h`eDz z{y>uqbU!JU`a9y~Z^HicSGPeDr$Eq%6E}bu|7MXqzvYn7^?E9}?ngXnI+L`i1k}CD9lN4+e(%UR{ zx4el_=Nm3l5aW3nDQ4MVLx@wLCQfsj8ZztVbuCVu_@YM|)6oL9PQ*>?!hfU%&h^6} z_Jfu2SRyJj$0S`oP26VZ=w(>#Hv8JY%jh}y{7sFE^_g}=YpAUq&5esv#|liB5gi*Y zR}{R@o-^&kvr}Dr-57)*>ElmtJ(sl@mzHQk8eV}%x}bvj9{IM~g}?D=Ih#Z-YlDp9 zrJG+8+(2GM+*AGw>0?x`9PEgxXd)LP*OQB!qv146i-f5nak523y>-3K;5bpk?LTu~eIrNi;`#gRSpN|G>=BmGsF*|p%P6X^r`7zf>!Q5xskvD92l1>6iEm1(PL}Eye8V8bB zWPWe|D|^3GRL%eP1q0)M!N3InJ9{st=ILNA4YaWT@46vXSw(bZj8CAUYS^!26*!T~ zk#Us-upw*|tRN|QVN-l8;E0PwRq&-88)3vJ_`fhdE_(J`Ozc0+f@2<6IgSA(2HCt`%*DY54PNgDPisGBl0EH{wGxy}a1gD3FGZeKtTPl&u~BOc%F}j_n%ZcL`w08`0d3?3VW}iu z_j&CFMO(NHPjeS|?JuK(D9TuY`*9)aVHq`}mwX0~ZJTa-)9gfDvy&~BJS{;TN@crt z%MD!>rKh#FH6{06o%zx1CMw6Y|AHk5(<-OxE3^2puEl7xhtQCRT{9h%KOG(CUaUEA ziMwCM_UMwt=tPdcS-(dlK$^*JC(w%b04C!R1RL^3h=k*&2uOmihQBw)`_YKtNZ=Kk zsegFH>_&~u#ovB=Clnt(;oF4hVDDSx*M!Gbk%vvps8J*EgM|9cA};N@mWWJPbT|ho z&PsA99re9j2+e*-T@PN$E;0NaEvQf1KwFEdY^Jk!`n6T$1CkO-ppa4#N~>Zf(BqVC zu9QSg1cz$cm5RvC@;4Vqjtit6mC+F*B>-TZPm(Ihk}9VHt!{}mY>KHxX?KT6kpNhi zlcZX*Mr;&Gx%ziDjj)|Nb50+;ADok~sdEj)WM`!UVVVeRar*!8Ea(OB4jS47*hF&) zYQGQ?9AH?*KT0>eTPO?n9`|&Gu1KpF-w+f&b za`7xxkS2BRk;i=tw-jRem+5~JFO~{I(EkDGF~8=_7x?nc%#O*%%*2$*>_14cn4PM* zlbgBI|1)h=*1w=)lpk4bu#R$Gvh(Z=30r0RBJu%Puu8%<>KHV%Qo_C`vWMK1t|==c z3U{zRh{ym`lG|RG8|8@A=JS$B_>sA*3yJj^p60ut8>LV+4dQQ*OJzcK7|P-Uz2x8U8-CCI_yE}enkrc3UM3KtJr%knRIE#>nfRGI9C zuP)i>%aUB_sKnfMTIg}5Cul~Db4kiI6j%WPK{Q1kO=W;sU49*vgusbHb13Z*gn8HX z`ndPalEW^pOv2ebTf7|ffyP%JIhii!bt<*L68ln>2j7F?SY&KP&LnpuHd?%51F_pO zH0kR7BtDh-N>M3gK%)(UiHT~;#+go!=LVIf1x-u}JnKE5eDVPhEV1&B+xy>Re5-`| z?D|`n>hTu_{U=m_ZNAe@aX+t4F3LRlHo&XUJQEF0G0=QmFeqMj1{F-vy*W3e!7f=z zoYG5N!xRJ#(VJ>*w^=IdN$zx z9+B=I`~%(cQIz%>ufRxYlf)C~O4OKzV(ayB5``4KlUPS3rPjQd zfi@|+yHIP?22=V)Zx?XfOsZ*O-Jkhe_;L>7;D5m=6Jp;*G1uCo^l$%Q`hZk44N)70 z@xh@p)7)5lhxni0IA&w|AH1)k(qonX-@0I@>Siix548Bd>q3eTtj`zF<*)vn<;gS( z&lC0y3>8IODuB9_w15N_Wh77<5M)J=D2?-lbW!f2Tti&HLt+lvVkm<~P{dJ~I&TD) zZ1rm&bu{(kAN5`}AHAK;Jnq;34uqfjsc$ZBSJc$hTyYKD`wKL{e?v-hKL?<^aWrEx z*y(I&EgZmeV5PJyGTPI_HgTQjlbA6*GCpH9d#&q6RbVqca=4M=%Q)Uaq8$+U86|0OTUXD#QRBv_WLV1LIE zM~CRb0Hm<-auBbs9l%$Up9`WpdJBr}cC#_}%F5L8imjbo4@@u@J~?}6#Zvd;(|XtO zm+fe6q?00~dOA|rB)_09Y;kXHS$gM7Tc|!WYHWq> zl%MYkDF7X_@6wdGX@Jir?oz6zz6D^kI%*gUMxzL2c0G0kLKMqNW!S?z43rkPAD5yuc*B2c%% zS%c6}iX@kQjAuwaBd(FPwYar#wm*shI)4jd3l6M_lX=2b6gk)4nn4@4k9scUeU+^Z z)x%G#`k<^V=BQg)$J5>Z4ndo(DZs&5!rAh+{x@e{DjHOHL}t}l;z~^+{~Vl%M8P}| zlA$EWCaCTz3lZXkjugdV&qUg7lLXe6Hu#gG~7-Z13nOdhgTmh$CA3F%e&C$ zLwNKEC(v9elGDwJezxo3G>T<-ykV3dtZ_ygf=c$>I#~8s`m_!O>(l{Ua=30;mnSko$d?PSnpCCnvpa?_Qg(K=rw`14BA*=yISG6 z8-`D13+EAwJzF7U#&9tD{vEXym3H+^T9%;^dK5<8Tqnnt;&L#f5*@I>+1;Z-LWla+ zdJwn=DlYwkb-$NG(!g?xW-14LRSm5zHA9v%e{DhBaK|DfcH8_eXPapFdf`;&x0MjS zUB)PlBCGP+)%C?OFsq`%S(K|I*tYP=vwRPpc=uY^-yGM^841Siix8A%I!!S0C#H+P zGv z#6^eJX+D<2F^hc*f)%Y^i>5?qY-7rDNKp4)4B1V6?*aDrX>@SOb1&?D4uyb>lp;>Q zSPhR;o1f~V;u9LMtKkic@01^y0y`nSrJTZa*s~H#ahcN>$nr1W1zJ`d4VOj?kMFJW zG83RrH*VO_1;UxC!YJyiD5hz$srw8wZjfIxIazHCv8GL$K=2HoT$DcJ5UO(5Rgsn) z6K`p-+iY~Y^=K*t7N1uAiW6q-n6p0NxPnDz)%ZNzP!Zpo7HarVLvpokQhFUH<;}uakg1(&9 zrDC_cp$8CU++HrvSWy=`SGBU;fHomJ)GXrU^p|DL<0Z|UBmJfR=)tM}=MtqMGvRPA z0t7Nu@L<8>Okp@0?(qH82b5`s)gF2FPx056{$JFFra{Bk4-TqGQN3#>tCm$WWa;Xu>S^km z?jQu|DeG}K8{EnjW0bRw7Y+RRsbI4g-8r_tWNRuzu-Xg9DnDAaml>9LHdmHRi(aV> z#wo=Pj^jaKhik4rG*35%XJdme6D4aQzP_BP6`q`K_96-k7J$%}z)@>C8E@IaAXi=1 z7oK}jwh^{=7plA)EyoXf-VtEv>JJY@3<>95NS-XCsmxgv${SwQ@1$frVq9gAHXahn z?3su<%Q#d=OmXzhlK;8;WFae3#4@88ytI_IoHAi!bX5q=;p39(NB{ddkY4;ve==<4B`S1>JFCpDjMEt>xN4p6OmY|G$h+ zC6O=R1D{Or@fCA*3J&gkK6%QmpImV;`Vh-WdkY>wZFy;Jb(a@i`3)S`axDVR1hfKv8(sd| zn%}>gI$LT=eBL_KKltQ3qYsZ8KDJQTdOM^wtX+c-S=_?Et~5?K2irxci`W@omzF%8 zqpmT}+}g&t_5g<7TKCARi#E^^sB6ugJG=MW$zbKFfcb_5&Lc3t6;zG->!8q1-3qte z5E;CfJbnhK&D-^>tsq;$t4UAxNK7*?Fg|fxW3sZ$3XtCf#F}V6xZFc%`rX!K?8OpV zg%;hG1@2DD(ia9T@XLoJ5Ep-sW-h%m1VO7X_c_7(rM=*%pLS>gehS`$h)4fWB};I(FJM+XJg8 zEb$lUT31SPS3h{;go0_>JZsP*?(ZV6T)U7~Q_1tFvp4dsjv}+WdOlA7>!iY=^pv4t zs)u(Wc~5oBYD~=Kier|XJ!~HQEWqMjtEj=pOAPapBb%(}hQhGoEgRv#t@rPhzcXIi z*Bp5vFat5WjUu_kU?pJvtoJlukX?lrw@x2h!>NB$92U&hH7Bh9n;*2l(VYuTpd<_= z?9Y;)yESxSmJR)qw|ohX6s(BYSiwY?6FG`7Jhiub44?Z_o@$L0WNWpLP8nZt#S=XV-k`_*{j{#r^I=!&Q1ZyLb{9=R&NbCbZ(t z5rIE@_w9t>AKV=JK5%4M@XJbfJ#SfZhm5yZ7 zwU4CvFritEL5obV_X5nE-M2KZU(}w6!QdhlDIbObt|w*2rV;CT zQ~lSqt4I_|NBOcX&*fCJhx)ecCR0hK>N&mmw7CX77o4(5mXuhwP_XY9@fD}%hlci4 z^!3nUGvE}(16r_Hhs=|?o)m^N3bS9}RP|_T3UF1U*l>*#gO3`M_^1TqGwmX5rH+e= zwosN)XF{?!QMeMo3lW92ON6H%7deWX!`dQ|O<>!bNq~{7V+whQ^?{_|92MD`L*Rh{Dz-sFN;;j#ROE36prFa&EDfRTmP^bs)}SEl;HTabM1?Jpm*#=$yIB z!`pAfqffH-h`0H^K&cT{#98?|G&FV5pQ@gE{QLW>tH3CtcPo%XpHYOQ8fR)UA>Q8X zK@m#@S>&2|)G&^{LG-+NC#=j>et?qZln zoDE&4u+9e)$~razbWoTF2n{6U+(=?j64lzgh*CS3*rci{+6l~Ydm2|kye-L@MH?>p z2-T=_cCv|Ze*xxeTG&K97toYpdLZ0tW|mng*hKWy=RGM%A>N{z3zs)2i~5KhIZBBm z-acD7jayq}BPAi+LgkTCdm?~5d-g)@mV1sS!;uvA{)H(5XM1Zwndy0A97&!6h%!A6XY`eVP)I+-6??RIe-}43q7jWWrZ<{n&=CFBo_jMzPhx zUgBLbPJHDK72f4w(MBI=Qy11Edr@hwZ~vWl;3m%{XS$$D_*u`=y(;|m+VPhk{w4O2 z&WM;%GtPd;beuerjC;l!J$&r`eS>5^+4n^rZd-cZg0^v#YmY3g`sahI%NxYy?}=De zsh^&%F~V1yOLc9Pt3*MD?ZqR*BRrXVuOUlk$Cd35Pv2e@cuLJb{*dR&p2bcwXlRr* z(*?<1`X5l_*b#OHWxIV8dOQoHthr_H>59LWZ=w1Io;ybP47P4toZsz~xs6dp2Mb@* zewo4?I;0EZXYwbrO7YBj$}*KQs9?=VFo)`@>~VoxI@?vzM6Yb&m4;zh9PW0qAeZv% z-NhDXC&+C0*ob6A6EIR`-F@GZqyRfSrYTW-&BfZYb^lNxnhYu-A-FR%V7^rjF?9N! zg1O3WCIR>8Hrc|nc%%|$c6hJ^qO9%jqc`L2R&i!{YJM4oDEciYf1Y3SBnS6-C)~4T zU##K!amgQrZ&@6{lcW8~`1DT4vx~9zJSA@3`p5JG;=dnV4Z6fb1xuMuIlt}n zexe?)gQ`X)(dpI~&}{B&ikdWm`4}2onR|8IS-Kk*W9zR`q}3^`>bfd!PT;b944+Aa z+Fhb8abvzFk}dTccwu0O@ZU^@Z@djEVEz78jIm2;nnR~oOf;SPUa*ccHj+rDw7Z=K zc2`U^zCSjOf3|djJvn#lBQ=c+%NX$bX9Xz{mRX6OY%6(Z8@qJ`UpTcfTpJB3?q3^} zIfYCLE-*^2Jt(_r=f4%-E)%HntT7I+9BOsYH;fM?XS09W1k(<$KHR$T^f6Mc9g=rs z|4fryIRr6_pSokn#h$ve&zAPFs4*3542!ThmOvOG-M&nl^W~fMxlx~oX*b|o*lsF; zbQcj9+lL>8?2zeC!jdBVoSqhdTG^Wmi1Ivl(LgIV_W`s5rXir!q{4NW+mY*yBy(#c z{%@%%I8CmqH8{H8#;!kO=H0)kTuuPFD`BQafB4g7V+O)|?IaSl72wx+D+zZ83Z%^y zft7=h8b=9ft2o#S51LeF)GpYmSb}5Rpq=qb1Y+)ygaxrJ%hK2c>uL43* zLXz^6bgD?lY_wo4=Z>`BY)$_)3PLh8X}yT;#aEDZWSb4@l-NjMbHwcAeWfsWN|4b{2%^cj7 zf46XKrwVG<6Tt7|XMC1$_?A{H=75rBFLoouxE-~_%xD<3)2Amu+$Rd(I#{(P7YM#1 zqC)6Fw0YCo*!`Ta5DWhMXZpgLMh9K$#ub2e}gmSMn)Ss(2a5v9& zZv>iR{Twcc5~=aHsk`K^0R*9wdWb=mO5CJZbLR-0+qq?!%-i&5h`HLZ45KM!1pQbx zSr&8XM)wXJ5+E76q0+ucQmw;hXkmi#jvO+;>vN-eVvQ!0=`#So&=A53{s=v}58U6E%UAA;}eXmM^!Xy0Yu<$I)e`k-CtDoy^C(nb`9k{c^Vvk~EH zy!J(_%7~xdXpKUjuZp9h42!QK9gw!xSHdx^Xh)#5ZIY&qi5jR1cNHDg!436nKBS-7 zPf$w>w1tZ`(A;>!NZeN2FoL@>#AqdhdL|sw&k1_Z-E%A6`CQEGKYliySLMeL6v@I8 zN(bNHOxE9oIIN58zvyxR`x(};BjAB|6|T4omRtvg90XJU#rKTP#9H=?wh^`LDn#tO z*Ae=M6S1HWVM+Fr?Rl(_X!I8DA*|3ahU)KO(b|U&i{ZFMm+m^=!t#?q@zaF&sovpu zWGWbYAJ}+E)P!0dx_Kyc!%mMA??5rYJlTlf6llOQS(o;XuZiD8VTEZnms)4|z|{^9 zbUc}}ciIx~Zr_N)ofqDaeXPJ;-++g@PX*(*^q)3;2t@8+pZA99n#3Dk|GB31x_RIG zWtN@i<9nNCyF9>M@$ZTGGm?1-Gpy5Rc|b&T4qz9RT5F98>Pr^*e*WZZz|BtTg!aTm zN9db1ckWJ8{27qA-J@LbPLlCvx;8HmN=q9bK26Q`B0{4q^bI!mr1A|h?37gTf$V0$ zDDwn4n|lhp6$fxvuFvZ)x}-AE8M077^v2onH#^8u$U9-K1Fj0;oKDhvFc6i+WzDD_ zY)*2#57p7(@PvpZQ4nr9h_7n8kWw7g@kQrG*-k`EP6wVUW=TSF5Tt|7QToS>KqJ+w zYF;v-`HsV!_l&>S&tzPDa3QMS(gnLTb#G-e026kiKO&vT&0m^qA|gVY-Y)H&GjYd{ z&M;};Du#x!p`@biev?SKFJc4PxjBlY{d(Hn_Vo%^9n=~rx_ zDzu(&uoLR_>axHuDz{d0@HQzXThMQU4d~2uh|4qAx!G05Je~hRb5V0>Kvia->e-8~ z&QYkOqJinAXr>~{G$$-~$2z*%qoi6ypE_HnlMxz%{z>HacZxl)z%xR^Pi>w|hnfSv zL%?i_CW>RlBuD7T?mhy=_bEbQY`wpRaKp;nq%eu1-Wh2!j`NN2;&(27P6Zy}NS#7- z1RD-b-y6w=&S*sW#g2w$rAG04YlZP#JLhPz)G+g@Pu77((QHdqu=Wy^%+fdbrMV^| zN7Sc?ahhs`4)!C%oDt-q*PhU3#SH)IJ>-4m8q4r}p0&asd9~L%wLjFQ?xFs{+RFUH z&%pz#A=*cAc4p$&$g)4I{sXe2y z$R3?J^O~gcTE`zWicEpPKx%UWG2q19NTgBa_X->X$KjG^B^0R~Fd=q!r@jKy7|!W> z#KXe6cFfvzgl*hMvj`tjI1>2PMo4svQR`b#g6~3!6s`NiL@~L5ZtRfg_Gj{*O7cAa z-Yd1ZYH~4AJ(mn9%BfSb(@a^++%1zd<+iQ1Ou{#r0=Zqch$mhJRzCm!rNO3=ZDj0K~aqGvFUTxf& zP)U7oINTp5XW^YWiy8=(flc>E)VUL#0X4W z$F-Zql;~p+P-URy-*YzKah(|^>u##H2ZpL|*6||Pdk+fkHDN|QVZiI>?Bqvz366hQ z?vb`hdhYcZw}okM(=MeOID%^^XXkJ5v|(Bto;&Gv*Ti~jeaMRV;wbDX2vUjB+}bW^ zxJV%mupj*cac@!l*i+<14mbAbj;3b}<&Yq{c4Ay3wI-3}l3qH4Csi?2dYxc84&VLe z$J~xE5I|$S6s(b#_oUVI1~}Bi-xcSNGD!s!Q$N0N6%z@HGTuV>+cQIc z2B6+hVe|{&gnvg@4t8=8{c%>8FeYv#^lhFbf-*Zc!$H8J4>KL{oCTZH#D66Og8F^D zw8|dkVi3Xz87b`9@w-njIi@7(o)xs_CZurYt*ADwS8~36QKb4^%5GTg7&A_es<#!)-J~+c= zi=8YTj_%^co%OZImzoxg?fJDEUporFeEkUjdB#5E=JQ+AOXv@=j_|R`m-44?ekDJG z46Ua3)7GM2Ql2>ds(pjDt@8F;*EC;pp3wbj|3(Q~`R~Lt68^g!>&?^my}{NL_&mak z>FEgDBd!tom{$*Z*V~6|i^J;Y(6_2#-F2G8{C)0W+@^qHU1I^-)nSxdrUjkyDw?&| zV6N8Si^ZLD?mF6G`s%NX<=VEMTziQlICLr89zw8;JQj2JJ~rcAylEpiUZRU`iXb>! zM(tkp?HN5QIYx9S5u6EV#JHqIC|}Vkp8zomHz!SEyfdhs@|cjE4XKgb4axp=T%)3I zB1cGHRhu+;mvYbjElS_pB8zkDA&YbDVTyD0XDsRP$Dp{%6E9~RMA*b&dju=cBc?by zDHIPC&{jK@jIi;APB}EGF(COWbWtuPW;!~RH|gFY#X2E0`L#uXb$T2H?HkLj-Sf?F zsTO1MjiW%kM+5Q!Up(9|Nvp&jrTR_6t2-(8y5!hfEUh~uw{^yu>xUqDYGq-6Y*Pit zU*S`b4_Th{?3W{jrTMAlxStCS3bT@Z(q9@RDJ{tP!*2!Fb)$ddCc?Yb)=#&{V9+L^ zyf^p`0Tu1muE?n{BYpY%fisFkQsHo-7Xbu3AzmkY^H%b}Fpv{<&YH_+5fKcF{PnkZ z-l~(PMw>ZvLb*X{_C(kK2>9wUJ)SpQ0f$+g+=9c0%j|C7ucg8!Lm%!pAH1u)5D>h3Kkv0qej8*BX?vWpkWL&9W+;> zsR_|>#MNQs> zwIK_jxinzw?&#KI`e7ZDz^xsKAsoVu*|DiaAwYC}-=?K?rId=QzZ zWlJ}^oCKI2&{von5lLxJlz@yBgp<;U|Y(@D{_2Aa>iQ3H+`iM?CMDgI= zMvCF#TE>?y8|-9%c;Y1f`D-l1t{C9A06Pei9~nQoErH)-r@JbO zd27y-!svI6!d03TxfdSNip}_r2K_1-=XPwpka@km2^HMZ7xv2{Cod=zz8)}gn_YcC3~?{dD7BUK{z^cN2;h4(jv_uCWbY~wU_pSd!MRz@>~ ziCJYO*|GPdt_kkgQsLVvw9INXeCg2Pn^TvM2tqi2bMj|Rkd71TcFnDNPYRSH0}K~^ zOSD#8GLZ!6CI(J8K;Ct+B`}N%ReYCng#-Bh1Te%2wNOH2Tz@~HvIOWR1!jv0;i-HZ zZ3t{K(M5=e0E^s6gDhG_9bs~biU0?ltwUV}k*os}5sGpGAXUEYUSlvPFiHaA$sx3_ zz8@f4{wLxI4#>^`m#RTLP}93)=7^_cI^l>DdLp^t;%EvX{@S$gfVGoCp?qpULGUa) zzFx^ZfzERRZl-f{e9Sai6CNSWMBxCQPj_82s3TWq0Cyg6M?Q$v9E1Zu@H-(mz~7&M zG@c=yg{0DwbyO({`Xx#Q#IsQ$ukRxRGyxB-Wao=+s7BZzTAxiZjqQDg}OmHGhFO*dfh>drK1sme9J2GqQ6WDPK+jjxNL-aJ0eY}zVzxSjBp{eD2h%_ryg-?= zy7=o}8%`SStehN@>q7hlGfzU4*d0q$6oXh*6OdX&x>k7q(ui8h(DCH0)P2@z1YaDJ5P2S9 z#q(!<=HRy$j_ouDIs|aygxQGh7};s{Ymr^rw-?{~zux_0Thj3SOs&2Xd2^S((+Z!{ zf?{8?!n3U!&>Hz?8k|ISDWBT|YxWUIb7Y@>IqS~DSlu#J{(3yfzw1qn2U>W@tnW2L5%;H86RM02>5tM8+cU^6ez${D4 zuMy5V-SJNGa+IuW1iV$mQ^p|`FN+<-I@ZA?J_ohnxKzW=HCfzP$pRF0;Y6D>a7tXs zg>Z_e-t%p^XoluqxU4ush68(y;g|)G0-2`4q=Ne#(kDo{3)OkTV#dw;`CROH0`;T;iiN#SybOQXc7-kps28S!S=OK&(%n!A zYV-@fDc0o=ev7B-4aSMU@3sbp0y zu9Paal`6*nI!2GqM~+&BKrMl4?TD0C?!97{HIONk!)lpww1?Cwlu0t~We@JGj8bG$KtI>9&S7${pa)4=IX82qBY zArtnG_i&~KYoq&*X-P+4H@!@ag^f@+pEiKK9 zmgZWSp@!0Pv`I=8lmPk4v_DBe@e51 zeSX{o76d@yyHK z*>?Pm6KoccppPFrh~6z#Fu$CJkl_hVsbILMwM-e__PTH@8v8g|xxPHYg71naxHR`$ zCvLotKG1p}##T4823T#cx~`DR{JW(4_tnQ5fzE>V=2>+mur}3qtK~`K{^aDO9sd3N z%+~y$HrUtZlGG?#+&+r@9jXen)or9F1TXg9d5izv8QB+@5Kx}=WI5VxWpB?fmfq0T zZ*?#Za@aUOKfjkKfVC%IU8cfzbvzbgWCcK_JaEH6XFjB{QCo78FJz%KDvyO0)n+!$4U<)t367HGWjjrW!aM4v zEDpu9Fm}T#9>NM?{YTEQfB0k-FG{+4{EryoxNzfTmPlk=nA9Hs zH{0SZIu48ztOcudq^ylr1<@ZGux4XWc3FiirqaRLZ4#oty3MM?a2;gIm4ju|GCA5@ zPi+tl2PpJp;vd3HVy>A}q#lr^;%DZhtaL?$S1BxOmzhx!PMu|qlqcw z`>As>CIlWZ8UHYk_`YP=6Pk>N-hEO8T=!Bc&NQK=kGpx%0lmzlgbC{&5xM{(7(93H z54`75MOd_pvSRtbBwlyiB|_5!^<$i@!09dYp(`y#>)AhHfM|94_nVr<8N@dtx}-U$G15A977v)Swt~8x@kDbinf( zn%&jadZk{v!UWy$;NQ;6V*}5&Z7|Pc>|SC6h1}H4qfCmn@thK5SqB*#0c^*6j)>t! znLH5n4&<|`0HRA4Lx35f{NGcou;6B8Jbvj&1Am$Og?bZ)yo7`3Xcg>Jl1hfm7$}j- z{zUzTY4lNWo}8GW53*uKV-a0r)A6ZPSlpkMCC!a1xyehZkO?_Uxf5;*I(%uDolPwp zf`ZjQQtTvkyzwX+5`LeTR2n7~;Ve@2m9@zJg_CBikhf93%qaIXGh7suY<6C8reQSg zklIT6IVlcX%y0q1Qhd%17&ay9^O^2MP4?7N;f)h#s`$Zp>>ajNnrW+v60NhQC5eAl zeoU*^2g-O8DCAjUgkf9|;_qdl^WJ*wjCoD*N#XF6$!TxNP15>p z2}!rYTzX(L<~G($ruBo|i9fsZEu{*TJ-srFC8ZkLy*z1vLA0OU3=7EP=kZYcr_(iT zp-46rwP~dJd{u|4bUN)k?do(F32U)6U*xl;O_mS~$?THvTZcIgmh*dpI5D+gz$g-p zhA!uG*{D1)yx9EWWtZi<_o!=(N&iVdlYor@w2meG&%5=OmHVAY^2K^M_vGFNZ> zw}0=bAXMd@OT zY2pe+a4{ls(XV`A2KEcKay(`gX~BI<5_3zz^O-mTvM$E(V!xyF28)1%IVPY(5Zsyo zf1yz5aZme2HJTA1-kI)WvFL{~!Svc+`xe|a_L`F38f6FjE6NnBed1Xb%WSMYJTa-h zh*ZsEC$|_Pe_9pQb8anng|QP9Y<|6W>fH z&Wr++XsOfJ?i<>jM;br;Al`D}cCxJ$SlCnKeC&_!0RkR(?TmIiqE-rT1dy-3-*#h{ z78Y#ueXTW@zpohDh+9kQpj_U?8RcpK1@(#T>)R8c9~pcXQ1 zl;o?nmwvKv+C8^lm(zzpvNWCFh#KbZWLdmZPne{KP&wGr9_Ckk4Aa?8VuIzBl`70+ zC+g0eOpJ$(a1NaBos4+O%Q(kzai!dbVTMc^Do!Jb+{18r9aaqHF2E8m0J|UNqeCth zvm`*yh_kLovKcmRigb1jhtZ|28G1CPp4Sd#I^s(53e(Jw-z?j-CTdmL1{INh1E()0 z2+2xzJuGU{(VI_78%sR2CF>+9iHjMQ;T%77m%-MZM$s(g04kEB&e_$}wLcdIOTiU7 zj+v&PC{H;Q{{4=UNkK&p(;K**XPQ{5S_hkCkapP60Q_QvZ%}h(72}^WM!ztIZy4q| zj5L?Sq<4%|KOVz5mgLRXiF2<1`Nx=49g0ysia-QY2=<*b9AJgyhWr0r_g8JA!b^5S zOVgcVBBm5<{@_X&*BR>I^CBVzvJ_f3r9=&-L}zovG|D3i@5({5e{b2MWoHhOzKUIT z6+lZPH3-E7`y-IDeg>k3iIPFh0HUjZPN4^ZS@2>+;%m~_oAF(WWUs|QJT*t~E(o16 z{pV%HF6302mo^KvK;BR5DU#ifGgKu{m0abis24-&Sn59uIqGu2CJnW3pW(sCGW^Nr zL99vdb5^wx8rAM$%(Z*F&tKHa9W6_CRFX8SH*fU8g3_;?!4>a_B3%S%>Ry}>n)DIBA zKK#OUt*2Hubngqj(&?;ea|3<9~ zMy1C>!r>}8bJSX_+A#G$GAN6X6ox!ne$62Du5$E}aB}cs+h>WmkS2c+3g1DnM(Y^Z zJI#bNG{f@$@KmAjS2}9SLhtlWVcy*mhHv`}UBwMufjZyhdiwc|X?{SZM)mH=+%nhJ zv1(HP3w7H^Ws6dnCl?Tnd!ekYnPvJ8R;z3^fzYo<(^uXo6)-MgJ$~HKzN?O1b*uW& z`3{)l8}pX0ZS`XP|DAaqhC55nR)(BP$1m-F!^1uIw5cjuu zkQT*v3`Hv9j~_(;r>M}>_*-vfFX`;y=J?+olG?iq>MAz>YGQYTrY%!!P(Q#E%vRb> zH(WHr&N70Shz71dwm%+YWkFoOVTGphkEj%mWs=psWrjs&3g1Q%;#(eTQMFyk6k9e^ z65-p)1EQtR)v|UXIW}pGtHJJPk4w+$=ku?xTZJE;H<+LRkW7|FP)ydv(|~BYB1nW8 z^AgEz-K)A2jC+UykaGrYmzCcpM-8B(rHm6zjD{ttWU*Ra0r1rH$`5uHd*Nznz~efh z^S111E&ur>kith4((=e1r`<-Eb>Py3ROqvciDo+bntHbc`_*oCC_COo$&?p|-wIV@~xUiX=&XV~C3 zQjl{_;s1Dg<`xCrwmsHW13EIis7d0B^n4ELge^AdWC zg*)431P+gT^E`&2BRR!uxXf@5IpDo1 zWuS4U?m4o{%9Ufl+*q(j z%^1U0fU(X)+f2{ z9~!nSfrV2*7a1nAeXNl*XLSKa7wmw45}ao(AcP2#j)C%1ojJAPS*r1)f0hqn+H#bS zHKF-q$WoGN`WGom`U0a~>`&w8|Jsx_!760FF!YR)m`JHVE7zf$G30U@8D$(UVnpel zvGAjcLtw&8T#c|(!fc8MKTXH)~fD}Xr)iYzrT#$ZF9 z!2HZehLzf8z3+;!P%CAwmODOF-YhsSIH9`^{s|gz%$Lx(*NKw^tzBHBaONe^K$Wo= zTiMldK#-AmCJn-v=Bd33Wga23Een9n998!=m zQALhe4O~qI5-^T2G3TCsk<@^N{|+_w1R0}Go}L7}OI5ATC{dKh z3E|H-MaD*y{h-}YF3a-U*<>3rmrZS;%>INDcyUNYB6Y>4h@V>_In?_ivuJ8Z;Xi$Y zzQwR1UJW|L==`o}&MRun?zllJ5wZOEKA23du)F$0sQBaHzLfJ$zJ8^@U=^G*WxH1! ze@B34;lO;MXO@O$^d8RP>-Vky5{`cS6=L}XGNU&bbUQ>}6sB!2D(BYvKsP|J7|7>$ ztolW=b#=_UYIO4z^)FMb1#uS0Qm;Gm{HS?1KoANyR1@!k`pDq_9RDo97fEJ;x)#;w z35UrP%qOvTe?cxVG(r20P3{~N7Wg!wmV?o(Q3IiGRd9jlhfz>c*kq9BFtflq;Y0tx zNfB1u(g?>_t$JEWmGnxEQ>pPSF^OX^w}-3pL2PKI_1N0zn5Ge0#)!teig?`ucK zR5E`f;Bw3YFXq~y+K?aA(C73o*Dx`sNQ0v3>f55h`Ec_uKl&@h@(Ad~Nrbet9m(FB z>e=KAT7TFqD>!PjknH{pX72r#;hBe34DO?*-7}lQY@Mb&D};iE(+1j_o$_{j?BaRA zc0YM1+dDL^NapO64dj!)Xs7cjh<+q2M!&Pz=akk>t44u@=78 z$|PJRkniAHl^BoC=QGTc`({)D#9-9Qlh#RZbVI)D+iSQZ_J;b(uJ@_;tBhs4<~RA( z78xE(1d9pBO^vW@cL$4!cU8>_p8~(kjM5Xg%zZJ?n^oU1q(|!17rns~8{5Yx`x*fZ zO9wr48rHKj5k|@HhGqLkMh&JZMJ4YKd|6Cnus8e}+Vu*hVW3rQuYvTovaC-tjBycM z0-|VkiQh6(`57jsjcRv0NS93&EBL;so|*>`P)xXd=bu`(qa%Dv8DK0RR(bo+jeZRg z=k7SD+1k=y2lkef=EV~{yw}2t;$q+eU zb|eTc=@f`p8VD)0ZzKfr9R~(W@HJOt$X{ z{5N{DPeBnDf^ojBxW1mev3}?*qHcS-JWq4jkC&@gL^mi9>>X{cp^LwP?$2+Jd=Z06 zU)Vt?dSryVi?ruVRnT^vSW4a!4x>OSIeDy>=*;5m*maY5g;D=JVG(gjY&D2+CLeM| zR(x?PzKV0j`jJ8aKDi~_AA2qseE=zE_%7F#VqOft#hS~0{K>s6w-=^7mV0NV1jWa` zF9$ciNc#YzLg1kpN3?C0)0*?I+e9g=kwm1DdUuXAHAkY3fBU z%g>>XVe_@RB5Z4QtI$xlp+&zO4nd`VdTqAz&pD2CDrO0~YE_G^68@WL>`KD8YLP!_ zA4d?}ea9ANt4>nQ84s?-Z%VPiiN6Z{TQKY*u!jB(* z-w`PP7f%RNV?`rpBRh2?+yCCZY5Mr;O1k|sZ?0_C&|o|VwidIEjM2(Apcitbm_&bs zldrVY1shLN7&mKItIuCeRQ?G^U07lUD(xFvOWB6-I6$ka6&xHzkf}x;f z2#Lsm7C-#7%ePy&p3<#$udVezj_YQ1InH9;<#suqO82{a9A)^SS!n%vLq&%t#RiY~ zAejMCV3JTm9})0x=KJ99U+B0Opc)k}fIc1QO3?6dxh9oiw+h%3bf;pEE#R|2~!u^g7g|GO*ksn`*t#$4Dli~85b_NVzKDh;Zc_PCgNn9lx?r31#AkP z>n=WqOsWxz)?ij(Ps@_0v(RMm2x~jqe!hihxj!+@3yL2q5ILkhq6he-XI?LIYa95l zlO}n4(=}^mv2*hev_(OLCClQf;7Y|dHsL0>082slyx@o*VVt|iPo-SPkqAeb`^YmBp>I1J0zk46qIZFjoMo zoaie&g$I`;z{f0Xaa$Ze&{cPYV?Gl*GLD)KVcL7f6cY9$u>vcfvb2d))_*Q9I!KrpK)!0vV}#gZ&1KKbzFrw+yWZB?a*P=7W}WY$YU| zF7_BjEh0pJ8^uEPdZ?Gbw*y4-r1Yv~wFUg~P*{ zj99Te#m1^P=IS;EuU$EnSS(bTM(8~JErSd>H#Fv`>E_jbtfq* zMop`=H^{_`8NCT6j5~oG5iZ#d4yoM(Ar;7y4?~ntE`&r6WKf#F60u%gXDVF1bpbMD zJEgAbF&|lTC*^+DTeVkueVfu(f||V5rB4}LLa5SmO0II++6R#!{FKDV!OSMl5R@zy%xO}a_Fyd+byoz6!_Hzk3fAAKbWQ0M7M%L*>Z}si!IaVl z8PB*K>n808yX=nMjR%rnw>-zJW7apqQAKcbMKuHO>>`H&V zE0065&rX+j$Ve0frR!4bQ{x|#jGtk%5-_=?w$=b7E3narolmGH8w#mv<&&!ITWc$= zFJV(@79S@LXadqRR6nk(xGPN>|0aohtpBkDx=L3tN_HAlDMeJ3%Xt-M#M*ja1v8Sc zvoN};8I?OnL*C?7K$TfH_thS=T5(05W|zyYzWlY6xv}deN5P*SQjwOWPc~8GnUIY@ z8cfLSTr^sT+70IB`v>%#5R2M(5dVv)M8jQ7%bpV});dhCJK-{!C^6qB60S3l8%LfZ z3Bxad&!>ERnj)TKA9cwL%eS4fe&FmgwIJX6hogJq+p&1`{_+;Yr)!RYv0*go@)pIX zYso(K6hnF7)Tuh=n#N~pAJ95!=Z|4;Tbs{Mr~j%4T_$5k=xA}n7bxt8P#Fk*GDnrW zd}oC)!yneXV3V-Jb6TA`3-M~3JWKJ~G;R^)+cIv^>C1D7nm8-}>I%_)Qm3=>7r}cb zU((yhv_QlAebzdf^ER_+I1gUm+m)ePtAJlq`b%~33B1V$nO-wgt;rK!lD?_#Pm5iF znhvrUq=kDv^0sD7w`Ivpb9lTB)OmcLcTxS%>wbwYOou_@n#EF$Mg6``Wb?|RH#UL+ z`~CR#jlIiv#7?M_ZKHzeQkf?}xnjJZi!BmaB zA@?W(`5GS!JBZv7U>~)qd6o9%68lg$4=rcfe2hUAcAO{NuqEbbQm0Pj%>#i zDhwWYLq-D+tU)canwth8<_kn*yWtE$%ti7Y2}^Yxa>TE1XV{kl{iX6EMX;(g=*Bkb zmaHpTdUN8XJNli4LAT26jGs9$dfyhC+n0&!sxoFint?;TOI4>B(EzmNf6fdhpDoFe;scnDbLzMu{&BOaI{XjXd<0 zR^sVgszSLa?E?BRK_SWpi zC+h)i&wo~zZ?C+MS^-_)G(Aqi>&$UuF?!asX3&B^^M+dicgC~j05<3IrQn3(Da>(& zHJb6<#aq|8#bQXHbm1MXk%kaIhyR7mion%7Wsl7Ye29-Qx8DLteuv^4%TIW#zJJdp zx*=M`Ka>-vNqYpgnlsz@hub%@ zTf~?vPVyoP?KtMps56;+(1%0w7chc*D(?||>+SQ+1&GYYlcZQVj=(no@Q?1MY#TQK znmKSj}et!dss0DxPS#xT2f8p*vq$SD#nD z=;&x95r7H|Y~4GmtL0PplTvED*|HOqPuc;gBolQoVdJm%P};@iL>_z;25CLU;X1sU zFzKHCUN8~_rezO!wf(A<62#lnCTHel8Af)UoZ8#I1kTSq;&rNM5aog6hJIS7UcR({ zvj-MuLUMr5VO7vMM=_7OvAh$FuzwETH=2)>r#8=g%Juh>l|B&zA7yPXXStid(snVe zMr5OBa}U#$QkL~YF>P5V8qE1bxf^Hvon?_UbxmNtkh-j47^ka@?5lE}F!hVUzsepV zA3N05d;csutiKgM!`w=NC-k>XLU{0UzF;!oRvpfwVFGhl5r77rFkjJHqu3=;T9#ac z51h%bmvlyN6)P2%?YQyV@7UYk zTwfi#hI12_a>uZhMjTi5h}3ON-@bgDjea*zAgE9P!h(>O~2OZDrK z<_bbwRWiuKN3D1TGvTXz_jH{@mng(5 zO^15xHz)jGHiGVi$5LaQE^`;20p*}|syIh9NrlZkm5-&K>nj`S-9eAAtrC1P+4Qlz zn)qb543YoJ}AEtv2N76u!$rgt}CzDC*8 zw6ppjey$q68GPi@cda}b3UyKHM++RzIeNr~3aJmCYrzAazlJ&^0J zw|(v3AqJuVHBEwQE`$&Cl)ZVe&0GS(Sf!fOHLF;AnYCVZO>=IyhIQ^3#74aUes5Kd;`Ro;f zA4SDIzg+!FuF#!m1tYBBRR@$;yoHsI21!&YEkAX{Z28^Tohi*S)A$c%J@L^j#nnr7 zhvPh=;4Dub`iFS*RaydZI~6Pty+o@fd#1Esco0-vsL3&y{rSrB30`{}eoXsbGbqWD z>(u3tv{tP zRg&8EH_Y&Yu`IQKArVSUJuEKfMp`_Q)lgj>EbElq84^KgScl6&uA)YVQ)NWlka#MC z?px53%QakF*|41RN$0NbF840?^UQOufx-`6VQ#Qop_McB4fV+?&8N94JlE=#d}gMO zL7&f3Kp|_3C=^sxOAMt%vB^U$?W6i8ha?E?c37Mlaa1VCNv!qP2=D|KMD|^N7JHja z$4JSjo*JobyY&L#S*sk)o6&Zd^|2SwLwAiRNz7QA^IL&{ypZSs zDWyHMy^{6%fVbwCuoz}8(z2|s0Mn6ZmJP_ynyv&}xcdm=KsEE34m++vm@+D9PXfr5 z_`AM1Df)4h^i?VIhq&;ZE^^r2bY~L@WEF8}G3fdc$${&(7?l?^=v8RiLN<;y>bi(% zNmZqxC5rl8g@9C5mg*nut$m<}yM-(buiWKDDg0mmc$1f0HF}B9A#&>d7f9jwpb^hx{!U_LZf7o`pD6`jO4EQ?waU0>*@Hj zkI0)&>3-!XB|w}tm56QJ;Tic`8|SbyrYHG%{rq6Hp4mqkDcQKX1h~9 zYXK_*-s(T9(fdSACRB4>vge-n9scVrmz~<(-CsZ2;-=wHKv1h=-50Dgv?tFpGp;xc zSj_XFqI0ku;H&MpMmH*tBh0+9i@+mKmbEwC=nB_cz?`!OTQ;|xbf(1hh!(jYqRqRH zEJ`+0(k-Fj@E^D22U)Y!@A+vVK~CW}>$?E~rtJ-kc}w95k|+l>5{hF1aB8OUf7X*> z&<8=lx819@7v&evF2SGU&TFiio?~+8SB;cz^*8<8Pn}3>U`^jsS~2TXCmttyx{US3 z5wrqSr38KWxI_aV^U5*cijr}K7wyo!;OeQfR5?N{)|TRNm1$y?YyK*!tsXT^($JJ0 z*9ZDwTY?maPjuF-a$UM_=U6ni?4+x&HtI&3>ZGLl$s^$%;r9Z-WRH*u;%Uk~A zhyAJUsSx-GIaO2+Ltwnj`Jhx4I81fBtO-sys-qQ6^AgQ(lS=hM)^ta$Fl-2rsON}y z+()5%wI^S}SuqLCh(=J0!nYn2eEdNCpNlf`vuJPmH{NHz@lN!gc>k{j+050<`G2F{ zO~p>(8}$Mn=Q|xR(abvrVoI&$a$xAgHvz=siQrI3Ju@}kGl|W59mWrWufnL%5riK< z|4|*dI81V6B$7Ypw4QL!a<{&}Uq2I)Y~*Fbkz-9~x?c|1;<6__$86MJopL<4w<0o% z!W1r4`))VeB^ayL=@4-4Igi?2=K7sGNmTV)8nf*LVN+k?dSe!Ab(*ykTgN3>xB7BH z>U9rNeeO8Z{Yq78(B-x$TGC6ZOfG_sVe3dGl}N%VrlPN+j$sL4uU6uyGiRSjpsDhT zxhrD9anME(>}uNF;$7S<_%dA#iFes<3?_s?|7u*eXj<#?q2$FXo%O@!1%}#jWGOI^ zrQ*hfB4x4!*wO6{_LZ#J%WsyH((RnJ7~Nh8K*lBelDjYHK!ku?iHc6rIUfEDKb@d z9~Gb^!pB+_>}l#{aWp*yDl5q7fR5~o!!^3+Czb`&uhC!hWcLyWcS0LhswkwzkvrlJpU({@$|)g7a*Qj;4kdi{$eUmdJNqX8K@ocW-MxY40cxCIGRFq!-_;Bo-&*&@|ZRnT)|=s zW)brOkrk5=zBo)4O!NlmsTF*Y6;a}5O=1@%;>~)I8%m*N*)^E1X{c%T?l#@0p8TysScy5L(DAJ(S4lV8z;B zk?m$WvKsbnEIs5+u|;tYJ_;THAw=1&WIOYGTH6ieOJOvoOg(d`W15foRXgGR9=@mg))TksBfg|j0b~^Z|MfChH!qc2F2i=ZJ=~9{xGk=p zu6w`hE1wW?M0lU>Bb|pfy@s>esj4%%m4Vdzca(4zBU&p$(IV3}nK8(z%BRR|dWYI% zH*M}jBrTF4=VVyy?=@5@F3hJAI~DzTS=+YIbXuI)PU5`e=#1h{cpVtJQ2moAjkeAd zTAPI(Dn&p;l}eQe`hMCxZRd)#o?#n(_u)pUTH#jk`?w><3Rp>Gcax*~P90n;5mVu# z{3IUqP|LYmxsajUFwp|Fn3Kvd5(QX+_Hp>GTzNQYf2R)3TRsdN2E1&KUDlsszp_-o zxav9jHs69>(5yexh${H84-1qW<%l2q=@ln^$%tu#NI>HCeO=0_t9Lu;+O~mtPZddo zz}Y^Sp790&ZXTD0okWu(=FHr{{FNP-+eZhy!HkYNRcPaK2J?6!b!nLcQl!-=0cOet z;;*LS8ov*^qxwi(tec4UaT9F|2GA0j$56clUI-VdR_@-3Q;@S7GijgAC45O9=|gY| z@9D@c8vv|}7^#eb8Xj^FPE%A}H=0_PU!=dQQWw7y_<~iBuB-V^(Qg4wusm-tO8$wZ z`%z+w0oE){YTDEY1M}x#Wlp}^Z=VQ0G$o>Koi+gTAEYvb#x_xfw;HF_(J3YL-YAP- z#&5|qwD=b=LqGQm z4vC|EHc$mXuJ`pJ1L)L2M4tJk{5HJSFP2bDBsQ6LQIEq2(x%(D5V2gyj@+~!O+%05 zf5DG`uBlz&d=|GHwv+)3aCMS0Xxg$bNk0oSvWQ6mER3fY+W(N+P5~jNT19{<4InXNbxdeUb6=$ELbaUy|kS3{I zE?b48oCv0knx|C0bko)Tp4lza=PZkdbM)WIRwf%@0m#^UoRxL&xBL3*x>n)yIlucy z$pI=nL%`sEH~snkc6~~l(Xe(0T;nd20{f!hIlCuV#ftcQA^o^zMX8z6GY@GB@=Z}e zcc_rt$)1|Vh+L8P1{hoA?uDFYQAI=TJ4=LV6d%ycOQV1ieflB&(4M9(XPFypgu0s} zUdkHKVb^jP^X&dO%Ck_;E~YDHvm0-#N|!C;_Bd=T8_ix~RLwt#f^-i)@kTeOTjrQr z9jCqGWyLBFNIYHR2Fd56JB)iwFG-ka8k4Owi|Fsh2AJ`@CNaKr@~G2i@V2rxI3kEk zg3-J-{UPpOuxyxhH1NII6IRv=HFGO^SH=c$tR_nL@N3X1xSf_MLP=i-Q3)J|pli|^ zR1SC&UHtrYAWfVT>$-I_7!$!Wk}uzfZ>y<|5J2i?NtM(&&O^g2GN>A`SK1s#sDhM? zg8L%o*iP@95vx>|(n*2;GdjK#17H7z2Fo>fo=MbjNDFUW*?Kqkw9$`*N?O=VOQ zkAkUzHGu`=>Cqkc7<>MzD6-p?E<+cG9rdo&}P2y9+-)2Ehd&h?C;)^|J za>O~0b?vD7`5lClZ)DffMpK$&n=v1g66}D7RFK&8^gkFb6UvPOUE&V zMH^81DJ|6FlOH>KK(fNUK|yX~<5=@KLC(YQ=YwX-s%O|rcHvfxODOh{gVmrfH1bJT zHHg6=J5J9`5pB=16Maom_sm*6brqyLsVSFUu37U8V1ac$dR{ikvQ5d*7`pP9%2HL( zz7GR`cW4@=VmU!hM?&{#$JxuRGPe39C0icf{Zmh56)>StD%|hB<&Le)1ZafUVe{=| z3-wigox|t38ghq#BW$>)>qLsdH)cc6sla)4N^Sbs1}T))mCf|m80}W3x3gQ)mb#Z- z-6=irpDL{IHP_#m(7d~O=kX^Z71c;sPLiTm1XxoNEHU*ZCI#){#|*D2*pcR>jtybW zb=GC(G^+|QupET0_tevFx08=~tWN1&;x3qI{3!y~%bV?EfD{bBZogXq)0C`9>e zsASY8Ewq#(;W4#qDCMs>2zx<}KrL^p7e+xW|B&xu6C=Ak;6d1f*7|tsu#;T{lNCxT z5?r}O^B>5|^D7#YiXcj+;VV4QvKnm&W(^)YyUB_@xn(WodB}V~;#W6CUxq)N8j{|d zQ%hMOyUc=zX*FbjOs6r&O3y9$$1=fkD#w<7QaWLo=zyGiuR~^;er0>)QQEmWFYuk7 zw?YvHo&vRUuUvP}8dJouh!g54b*zX#S+DBFVr@pUkQrTEu%m+q-f;3(c-y8LGh&Pl z_Z^D%NMTFNzzwCIA>oj5W*%WmP5~$`HsUi^z+pdG*+&WUvU6MuRj4 zaR4=CXR0KXW<{<-ZGSLu?>e0%9*`+aF6-yG&?)*Lq*~*j&Z%+JT8P*DjFy0ZwO(Gw z`sehZ^8R}tce2w7Yj(Hg5SlCNJo~2Xc`zT+NlfZO%&*Lsk|N~EaDrtvh|8K% zc^7XgfLG)#Z~y*v!Zx<8x==1afKVE8^pYQA)ZESBdjFu@sZCBmvPHH#gU(z(=OcD> zmOg$GIZl#c$qnR2xVKgABx`0Hjo@|O@^|A2<8%2F%*>%CFPeLo?(1o9rF9w7aICUo zH8@t`Y?>YE#qA&wnOw=JYOf@HbbOyqN7U*Z&B&I3tO)fN?kFHgN1*Emtc$w) z(Qmnq)*oXHi9#4gPSlfG`cQ|qcU3qLQXz>nbK-5oR|4I0B=w0xGjx{)$09^F<4IT$ z;j2#b3yDsvx==-}Wbw26E*y;(Z1<7e7Zuw2ffj_yeoKPoe+YMp-6Lvu#( zZW+b#Ubuz%ZKjtX66v1X)DSeHFEpt@~{DYmoi V4CLsUgxv>;g?xCt5Cfe0&6eX?%iOWL}AS?dDWx63&3Lh z75`vNUeQqAoLI*QtGYAWI0<=EywY&FhD5a)Qwwu3%BkJ_yM>UmDFt!@(s2=^D*2oq z(lJdN(s47z7yIt!36x8|8uUfp2D}+Tm204-`3>|R3C7M2Y2DEt?VVEdrWn<(i{aH; z2qYOqgy*+kUd$&$@TB#mt`0LxUc1fseHw1!{tpUBw(l(RVXpD%^KH+C;jm}8z0<}F z{z^=C@TQ+1P2==_$+a)>t9(D>l8>-SCRIdIN;TChYdGEnLgl#YCn#=%Z^qPA!_%#^ z>D?>cdM7Kd)ao((o4%D5H8Hj{n00=-W5#~Sv3=xp+o=tP2BAn_?NBDH)U}2Jx}oH4 zD73F}{nolM>{&_x-N8j0?jHtm^dCJTCmsjpd?M^m2WEUDkI)DUwfpW}F}yb#Gh-kg z(1`jNX1cVNU-NAmZ2JOEaPHyhd=3XZnHKkT0J?sL`YhD5bzPG+QaXfm+p@SkU0nO$*?+i|7CNuCmX(~n+!wEBZC`=YUqxqINLu#6pDp6hBeYNN6> z$Xgl*8Z@`|re5aGX7{O`m^TG-5?B7Q24f_$IR^XW8@2kj#qR8KW8?egcdK)6l14cC zz?raRV|IT40-?=h`I%fHILZsW|K#@O$Mcg~SE#eKUd-w(#K+RNH_d2CsnQ_0XA+*- z&`!w=Uj=eQ2bZct!bFcxltJHXZb;ov@I^q9?l0W`3);YMbZN zxLFtt`@zdN21#c?uRON{6UEM9+YN`qS06Jl&UWqE5M{OSrnC!!U~0LO=+ruSh^Snm zY~u^T%Fb4Un{>oA0wAn>lsB`p6Irj7<}~oY7=MFvFC-7Npc?a_zTuzYxIU2b?c0p= z{Kp7!F=~0}&Gs)_Bf0cMNZvz8#!{=%&Si>&cGngVgcUbs40mWkXEtH(ADU-c=Mx+S z-0x=tin0d%vs8*TN%LiGZ(;rk0pO+$b5@^8ZgejBVEwnh_h&I9>(yMU>k(l`I%w_C zUStNSAF_}c=FB3fl}y~i9Ihm$6(ge*R9}!{Im%K6nnp81Og2&&aqh}2ObmZ(#|<<~ zm@$;ul(N`!*lJ&rH+H{wMPqMd4$B_VkZ=^cdk*BFIm23d^!lu&Tm9mIr##pe!B(L% zv6jZZKFBT3@M=>OfzoHTOEYIUv3&gp`BEpwiipYPGOyf_{!7)F!Ycg_h7*Ww0_ha3xD`YD?-Hk{CXf4kc}t%@(Vo+yPEqMud{g z+{#^{4$M0`bJU?yHnW^Q*2f=^$2bG-;uA!@RQK|IOA@Efj7Y2>%|PhKm52OkBi!g#ZbLVMV(@2BiaR%weeJZ$Y_EM*NNR{-#h6A~R>=7t!A*EI_ z@+I_-_!zbsF+v7g={!R6kQ$E!wG=6~CK9NL$e2C898(*dN}>EGjDqGXsi-&T4F-H-#emw#nJ_!Mx3P_pG z7IrLi5*^l=CLza1#}Id%&Xy~0{!LRN1=7|pox-@Y0=Tkr!VtVHiRrk%i>(3|iu1m5 zjEbDLSWKOe1JlBR6f7PYH~MN=(nT?NlV<2eA}Ln28i_o+6?Jo;cK?+G#`>M0O8%`QpQzHpN#jEFEKYK7*AFnC)XKqGr0DnX(Day_xYU?nQhA^-mq* z<{I}zD=l0c)jfufJJ?!0nW%EOOnPX^6;VHeGc%e=%VmaDc{1yh>1@KdQAe0_Xd}Jk zi|Em}1{Df-Fiy+IIc}~IM$MtTTF?z_S+ee?1P-sie8FqLWcE{EY`u1^&Bujx*Zig@ z&h%J;{Sw5dN4%-_u_A~0R;LRm3q1v{tG-Qa?!UZJybJUDM$JJP$;zn!1=(|Hidf%J($Jtr^l0*#&H?0J z*P|YcE=r+F#HviHlXNkzoQ?T4pVT#7B86MW2MnRI!8$2I*eoGuu>R%1N%~~9_nS(d zB$nM#1YPw#Hqc;63$^A>z7V49ReA7uC(2Ln@ddrrbG-E1Ln07oFXa9FQ$l`^*+4Uq zk*(O5aV31nXomGjX7X5HQ^Ta%@ownGyW|)nhV?*ex?QvPR$MP2*OfY+3Ld;kY2C;6 zmy^rRmwhx_4WY$!PJhcv#V7*u5S^P%)ai(CNfd$~Y(1v_iDF3|f*)=Xz;3zQFDqtz z$*vdq<#X7XVye-=jS?-K0S zsXHgX%@dx@V;LKPkNpC-XNU@tC|!-fdcbr36y6WjWE2u%TEN_jcRUs&sqYbH?G2Vv z1O1mA^7&ix6R75Ib^4&l1yslePHNWZb&NK^Z@>7OtvPE|t=8cotrIq3EMg*5hL5(zR-sRDRGTPq}_N^lACAJ+?aM(tM!cI^s=2{HyaI3>kTt z_?H2tJc2aheA@;g3`Nl84e77cB)AF0Ue{1h4aE)0JQ5tzfANhp;rQ6m7D4kt@U)Fi z#L=|98G~({aOoaqkrovn1Lx8dlw=)fXz2{%SS)VEQZ7s$#R4v=2;!6#{fZuGR!f+> zzst{?soN+p9NhNrG~sK0Lt8B2>97`?pEWAq%PB6p**&x1mt5^KfScW#WiQ zlBh^2!2>(|KeAfK)9?gGMVjA%p5POnIp%j7OWQQGTV~8^6_d9nasuk=H#v?Z{NIzy z4Bsp|pmS3Fh-NOT#(2Y4LS z*DIja$CDtkHJ_HFx!0_|SspC_j}zf_|ux%%y%@ z))S^>6I+grI0sFTc$H`QusE7T6ADql@#UNS|f)zl@}g&Z!w+1 zzx>>1xJW%n!?2?%m~B}DARwAifz%8&P-H|!&PrpbehinfHG|*4q?Tsn_K{aW3pawR z9ozvo5*2C#9~yKo6QtD6qI$WX7FHN^q_!kIz@BZ|(Zk-JxmsM2)=oo{UJ?5Ng2nOb z`CZo%@8s@?jfq9Lrth5@Yo8qnm5<|_9ZZ$3vsPmZv67TqK$YUC0)1S_F$ao-P?EIwDo_Nbzp)bilV1H<#vQU zF*@fa3|VeOY*PxcWW)lYi{M+&kmYKMk|TSzwdfa9O7sEVZJn5mb6Yp@PArV4K#9R} z!^{1V!8CDeSz z$9;wr$(C?dq~^>nq!~tuEW%+MS(^ znTd_a_b1=4mvK+td^gWMM+3qHi@+OkIAig*N_qJyC<#mGaD|8uGn^#EB)!FN3fMwa zWe#tTr_z@8Hc$W9ZNWK9D@VSi=^8++fLoRBpPIrf$rX(aiP|iks;KiZtt~kwKw_o3 zoP%6h4fXQ->5DI?ga6sNHb4pzt0xn!NyRF87e;=7*0Mq8b5WZne}eyE!6IfU07#SY zIWy`qC!i;GOo*!{NC5KM44wwK87I_gXj-=L)R_}kj7mQ$c~RO-K5P9}_bkup7KSHI zbuMJFoSN7sM8|}>k;5o`%3|VB3Bujt3#l`OOW-t3MZ{oNJ3c)ZpcVV^c@l4=1*3`A z+LpQ8q@r?NAg-h@?nUQWK>MIj9K#o)!WmysWUbnY;X*Z{2|j|^JuZQf@ORQOwEF3< zRF6srOJk(g<*|0NpafLvT5b9^-GE@ko}<%#8dtz*vqrXn(ys6LA5Cr>|Gyol7p&)- z{w}tBK1S#|b?yz_iL7!pZycza=+6tJT&GYWFS@;Nx^v-&zjPMlcED?7TU=npwbW+u zjRk69)mKV_C3fTqQh)2Dg1#yAzA0cG0=67wHmdpgrhji?)Y(c0aRbBxxS^VOpq-t1 z+t=DjtCJ0|Ed_-7zR00ny?WcX-ZZdMjE*VPV*|I+K{qh#Ts1r9>1!`l=xYPFl0i3c z>s-zB+gxh`k0@7js8@M_&U2z|OKd39%JqAu1GmuYt{Q!Rik)j7P${MQ&t`piFzWD{ zJyHXAm)@WgYIR}_`f*@YVoXtJtDsP-dHQ~%fjnq+coWw7v$V=}f}mbR&`-*}o|Hx{ zL>=ONkti3Z)T_C`%ehSL8h~Q`lt7+LP%mD6o?MMZvv-t9vo7}0Z>!!nE73^^ui*VS zdBd+1(tk_-kCvuUL$39Ks9Lq~ci)J0UkQ7utE7>7jDvnm`d#UJsq4j&dKD?JX3LD% zAXRQvD+_gu9nyW{DAs;x7yP}P@d>6IfWWn7nxGH(-^<{b$gxdglwYHSUtvJ!1CxwL zu5J6N@D}e5y>ILTwRKb|J%-`m7QJtaZ`@P0O;o5oj04XM;5Vs1J~U4hL-@%VUcJvnuZ8dfGeCUgQ}|4y&knX=@!6zEbdoLr7{hU0Zpjwm)uQwT=>-n=y_B1&~u)b zL``jgkmV*H^ZN@#^ZQG~CQw(%b-@l4n*^QE)79W5^Vx~?#Nh2_o(Q-vU%v==rr{Thy=ZtQ>6Prtbfi-D8@w;l*9s*pMeae>R#MUJq>H%rIODPLxn%HjyJ~jEjmkX$yc$b@CPuS{f zbOvcJ8K2D68uM)gzKN&j0F7!LF$tp_avjj0wH$gKxzJ5M+C@BlDNoFA=xe222}h6#^)h3?JCb1R1on#QlD&kl@%be}X2F(jwx|LFakztKPzN!B_^nOK(cl22i5C z52-wWl_5=$BR!y%DOSre&v)dFH|1f-T?aKZ%k+zU|7JlVe_?xrQ0ADvY4c2ox^oiXF>5O{UqJXrh1f@OAKNLR-2#(I(jZ&~F_XIO}^$i4zv z%u&S;aT(ibG|Lm_lt=B}>C6HCfZU z|9aOb$XA(5J_WEXUpPu5t^DY;j2<`CRrLfvWf9;ZPDz-aaOkX_y>zZx7cF!rjoGW+ z$MprnoiL;1DJ@-=IWJ$3hND1dz9qo9bl=Tz!xG;zM{}Lw`C&bv*P7f~E7-k>iJ^OHfs2=~Y6{Pw!5qtbFqbjXQsP4X!pW=9N{(r`@j4 zWCf^P?{mngoT)HtVHrEA6#J^8<_M0Hshs6EnzOC6thL7-U+;su)Lbr>fN6hr&W=PK zrfqoQHdl7!W&+~Xo}kaUF&N?S1lOSY_=*w$Vvnl)NaW)RCyU-0Nvlnd_t?b`l=%JD z=q=+;3XB!TJ@U)zUgd+XdlYXTN2-)^OV6D~?tVoUdgsTDU+@cp-?^16#if15R}b-Q zEQcw7$yTqrm^CA{KpnIwc3ewG{ixEnG@!buNS!Gmu`5VI!s@UGu_W9pB7gGP5T3^c zdnr)t=&=yr$T>`B2BTP0_BbQbk@TO*u58C8M=d@-(rO=FtfuUL;I2g=kJhCJq11{@ zt(n!#s^f03?Y%dzHNCNY8d~P289by@!&WvgllzmCn?HH`bKEetRa$$xt9*99%RL!M zFOZ-WrOyXbWq&xgS8Z0fJEP6gWKnjxRyx9Zr#;Rndtq@L4^+` zhLmfBgI8LHfamat4=sfu_^7duqOv2lL1kYx>4WD%m@o0kL%Bg}!a?i&sJ4ET`(U{& z>T%&93@Wq;e{+UUa>4f+TN35A_`jY zW2m`5HHcP2Xo#pSQ7(pT^3M@?xA7;aapAE03iS%y#HxB){oeO%D2DII6S^*P~`60C|`2~yFOf!y=RID z{DRnsOvuc6bHT@De3-$PcGNe&(TR0(;m7lQ3UcUvsg;_I&7hC($h(7-*-=^|mQ6)+ z@-exhFWrkot_zXhweYuvz_;AA9Hl|bVs&G*w+=DP?sDaE8^Y0q(HXMT$D`d8-;Cfm zfd12)-Vrf0dm&Zi(|hgt?+hTM-G`_@Km04w9~RJm<9`2V-BZNjz@vVLJ1i>P0eKbe;;>r9EqVNd^)O`y3!m&!DwGO+$FBHMZ zA_d%V3Dz@xOpS%vaWgG$zu1FxpgBR{0)s^ZW-Rgsf1`Fu*66WNj_pTKGG%MlN%qFB z2F|68#c#hNI@|Nd6Ym&2%TUlxqNi+>9-m zXppAz%!RTLaWI=%9J23DY}h!{wG)!9 zs*~;tQI)pAqT(hKgjOZAbO`ofxf$#m1H4+{00A&ST4l8ZckTWkY%7KE9FxJHb*BBH zbpAKC)qk$Fgsr`cvy!u;iGj`kVR-(>YKQy>v(xu#*``W4z=U4j7#AW|_K%XX@hJEX z(r5t^1Z1vwi;mdX`cw_^7c4U08@eMb#tBJ-e5R}k8;PY8V>=Z3FJry|j zrJo-bo(&TW8mZJRW-lZun4sw#I}ewuR1T4P*wB}X%k?j;0r#)@tXJIMKI>Hus0!M1 z@q;R%=DxmSpop{dzmo}<|9!P%zi;^jTPY7s_m4F}4tEjB7x zS%fOKs@21>M{E68c$8n=Tc&A$zSZE*Xo#?KTRWjOlzqNx#qJFJt{ zQ;sK6`Wb{yCDeqgwx1Fswu`N7{+Tcf=U4elbyl7P#oN2KTRSV6b`Noe+Y~vdP)cd0W)uSbV@Ad5?GPgvUKgF^fQRM zHF-O3B9Jdfj4XY}%TGNU$2FK|lg%L+1jQn;N-Q9Hz-5rl!wfNk!L%rfP52^G8n%!T z$X6+Rq$GtiNZW?w#Dqq2fL#G<%S_t+{RQ?~aw1JgMFrhe>LdN(Ul5=3af7&~bm*$Y zM4q-3m%ulCiF$;xwgq_v62OdHHextkvZZ49XhMgoBGm(o=t zLMP!ZF1~u7_DaH%I3LbR@sHvN1&+aKRSBh>OF_=L1t;GANO60?DY*OU5MwdyV05ke z8;&3-vngpb8i=z&+Dn2Ekpf3+YXVRIod^-w5z3cB*+MAzjpq?&ZNnN;oh0Rr|95QzRO?eIT=plD)XY~uJ|FyQlFF`)YoM8Oyr>OV2yEkP0$ ziGakLVA~P<7cfwR@Esa~{|QZ&G5Y^7FbAZxarl(u^#g$8ET{MTm6q#YlIpUA!}Pjt zQ(K{_p>gpIKVYHvLRqQ!)el>6@%@)k};#VDC*_QdfS5^15xLQrxK)ZP;cQ zXs74zY{60Y^aVjZW2{mNIgUMD=%?=)?s0pYCJK0LRym#Bi`m59ik37`se@Csj_X-$77bsE6?XJA->b3Q(MsY?W7pQKDZR=Kz@ctvs812f{j`sps zQD1ApB{~~`4IMk)4metM;kn4avadzFIGJMOG-x|Ii)xSLE)l7;Rm_OZP_j0e>*O|N z#-R9XZ)F$f<$}fXSPZ@?p@W9l!ZO)kbPF~$`zP`eme3k*fjL=8DYm0J)Z8APRmQ2k zlrVq-KNRK7yeNi=xENYsMO?@3>Ye3lGG&09DNmzMgcKo()-76h zoWf05Y=^{S9wD_Pw~9q@=!DA}M}R5P1_n#Rczx72fiTr#ivK74UHpH8GYDT}WJd=F zv<6oCZ^lSD3i!o%BRiMGCnrX1FYpq6u_lPi`Mrd1GTdpUpNpl{~xhyQ48z;+h`T@qy3DmfT9ELSqKw?R(Dk3hTzxj`51gMsgu#kc`J^dfV!{*)*HyM{TAXs4g%I zn+JxljU;C*q<{kZtj#by{jaw8bw$R0XpljBdNQ>|M-_6v;zYYS@sqO-1BP~HMUAEw z4D{Ybx-)G=Mp15fSW5mGdHJ+B$xj*lRZ*F;Gx1auf1oUekn?~aq!$$q>>zD6#A;#1s=jwY+g*1 z9oTh4IQl9?(lkLOdWRRyR0j)}g<_8nKZ;VWsNkG#P_HE-R+hj0F9V4_1`{k!rG^|+ zv|q<10Tw1mLq*MTNl}1dTMAUh0K#}eL}EfJbbKZzZ-7%pCiM0~=Q*3m&17l!2v?cp zUmC$rLzDe-tG@^}VzcExQkmpS&%o`ynFRC+@z(lzP8-m2b!BO#$s#wHLTd=|eTJYN zPL}c9jZlJ4^s##{A0mBNIm|3m5vA&@d$iZptuk1kKh}y&yp=HL-z*hdVJlABOU$;Q zE=)8*(zxA055@k>01FIdCNY+$7d(%;fnPbkq`NFV<|`3~Cdxr!+Ys&(LZ2w>Hte#w z69#RTD5!Y)H!OD65sC9v@_IM9L9Vi`v%qc#hlF^$Nl!Qo$X&wUEUPTu_=aj2?Soyt z>afTTZ!0%9NR2%ZOF?2yZx@jSDzS(I?Eg-fRjva)s8_w2AczI2z0r$x;->J=cE8tm z+KM2M!kZyvSVPEeC(*F_K>#?5QtWM=T7q;uKnW13mnDL@i{|4m zZ0qLs!p_|2vCB96*xCV;pe<>#9W=BH4z;!qBAFAFhbvevlFc{4ncN!KUElws7iPse zthEH;*DsA9@f_U$U%vPOjJ2KNe;k*J)uG*!mePHDn4ZUS5or17#nXN@lLawEUbTV^ zVI3kx7l6$2vq&>08q=poFd?I;tyob%0$jTS%`KveHS6>VBLt0#n^ij&ZO%<9FD)-E zo0l}J$Dax=yuSPxnIyn2@2TM@vp7<(y?5SyTLzeNxmiJWLCWv=AxnfitX4#{HY^Lv zU*6O!o+hu%t$frPmVit39TSE`yZ)gi)kIarEtb%oRxE9R#t#2Dl$ajeB-9v-*$5dm z6cVRoklW{C$qh2ygN`Uym)lVjxe72Qgg2Y)j|odzdFnn=2Keon*wo|kBtGRO6*!@@ zX0N1aA+w{C6f*GPP_XR|xc4#ph}b&m(igumAeMk9LP;-pDo%+c0Hhh4`+AL~^V@Oj zVaP#MQ~*Ua;su4S0%{~OTag(jK?M4}z_t3?M%;*8r7aySDVm5DmywWU%w!uxI+aA_ ztzX&M&Ze9P7CDq)VZ%vv<*K&wR%dN(F|H7%KCG^Ruf+7R#T(}O=jk6g&ugV>R`ED5 zS;KeceWxgo-wCAl)p5>`^O|z0KC!dtsUtL9JabIc=ElwXmP?4@i@K`TBKq-wkS(5c zSb!9sii(f0H4nS4{Hq4pi$YWzRwVN2t_l`bAK&SY(+ROoy|7rxf~NYJ$Z8O$fXB90 zO7lVdxZ`;LNXW;~4;=L~kCv?b1{_fwh|x++ps*#n#uc5Rj^htPPjLQeKQT`5 zi3`g89p%#WiYN{$v#hyhvghlT6bW0iZhkH2Cdz$;g{`y*?mo2w>yEWsRkge}2GtKi zoTSc4zSDyE0$#o9V*^FaZJ+CHVJ7)<_w=Af(iw(*Aj&{&A)d)Ty((Y%=T6g%KEn8f zLs+_C^?nc`ZV9{~Nj*O$n@4#LaZ9q5SITBYH+gSZ3f=+A6nUapCztr31MF3ox-GgV zQ;*_CBi%$=IH?&Zp|WJ;l#bs)pLc(ncLnp6a8bI@lPO|RKHUayne~hfabV1Y#W)-F z=XspM3o&;6rMv|%#funpH`(Qs3W=C@B7J6g!&2(48O0i1rfa|E};Q`R{|ZRvr{_r zx!K$J0;=|iQdry~l% zr+f@!<_c0kqeCV-3Jp(y>}M|5C)E##aY2bFK7(c>mY9Rh)$Jogp{z(m6%SJ(^YrI% z$sf{sMd^`A@e8roB%Px*@nIXcZ|^OD-jlAcPCt!MF|AIFOADEvtn>G$pvgH%3l(tb za=eL)ZaCOF?Au8`m8=^^?YAq?;OcVAP7F!6=LUUie}I2t>>iWE`rHWR2lr%snB+wl zs%)O_x_OvS*8^jqXAlIX)uAi7ImsGe@l8}cdbT=X4cE7DMfOl$GNr{r^8kuB8+I-SlDT)+EUm~`3{~0 zGM@7R9>2(8c8hd4`fR1~zNJ!FI59KepJ zooz%wds1O;530;}&;z@<0KII!<3T~}546O`3|DZvRm9f^R>f=gwpe+hZx55zlQkb4Z9&XQOqqLZmF# zk1nEX_20cn_a~nuvVvpTGk)NaowTx#@tS&B&I4ggxd#taTpe6|}PaYj9E zsTcDTZC>MHej4Z?+L|8_W5!O%ok6YHo?ujO4r<%k%Nn^mhCli>qi?LA4&eXx4cZrJ zyLa#m-rK^W9}01UjCUC}LqA)ybw*8bdR0H-yL#i;-%&zun9ckGbN30N?i*itd_&!C zaCm#6*@Mw<(L0mkcs%V{!PwW0Z;QV}F-zgLKbVWb$JSMcr@C-Xqn+dsS@CxxbD27# zZb@BiWztIxKay8Nk$gjSHuh>Dv~Gg4W9wG#^ukV=bpXhAu* zyAB}Ua0``~w?}(=c6~RLF>kOko-qNpoF%)k{r705jTUA78`7NDjN;qG&ezCU@C@+y zw>Dnkiyt9!3q6IPxp_f|xhNsJ4mScv`}Y!j_TH@uk`7_ggQg4)xge{Jasva5TLlGM zV05XmuJB5I2{JLYD(I|Kcz2<6!dm;!=}UO=!oDNTmPPnS>ckjI9&4 z^WZvdge{7rSJ}P;MlO1D2g+&}`}ESV(qFujzXFYy<9x6Ora6RsN*ZhbDwKRiW7?aQ z%{xCGc)8SY0Id+rYq7(j737_%-uGu<4b_(b`O0-(?g@K>SJX^rQ2 z`cuevl5YDDTu~&IIAopO7 zU3eR97C!JSZ~%T^quXPDWt9Dktw7Fp0Mmv#al~5ncVmv$4Q=s47%l=Nf!&-iVU)-L ztgY>Kn)CyRAuAZH;kaR6nNBv<#nbBVj@A`Nw?CetbL7QYodQgBqQ} zWSIxluJ7lS(qM+O4`V# zb?_nWt66!tw8vXBI?9kNgLdIkTS|npP?Mpyc=yqE4!hg6RgL_d=Ox<+p$9%S-NZ6@ zX`{NJV{lIsu|TOY7n0+7U#^q!n9Pp;g`j5;#{;wZjw*|`XIRh!ROXH{qymx#E#yAQl~YWrSj>bb2kBR8R(y*}YdA)adIO zg+_HCl0svV(o{+jrzWk^hiOrT!KoLBBKU`=Llg-|(m`SDjOrUXecugd$>Xf7M}9*)7^Vd~{zifylwdt@ z>&T5B(L71%=lPNt`k8xo=Q)qvdpGN0av4H>Kmy&XlX>m(RC~&m@&&ZL3O}NOXHjX@ zw8G7ne@;&2CXYgq8+*gqgi6hxMuO@1CQ)l&xylp(8WATFd4h;^GLZ zY2TL}!vaN$Tu{wVr!D^XFiE6uuRSB=(EI|TvXlAQ9&%fu5|s@s=Q#>FTj$O!pS@=4 z1xtq$vh^2`E7qb&RUs|`$mxE%O;nN!s!V8TV8eY0a9ldh|PjTu@k>j(YY7 zdC}0I6uyuHo1VU_Zb!4`L&|WqL&`2ZC6&CBifnn3HiNg3EMi;1jf+$4YFmNHXR1|- zdIHK@u`Xw{B#T?E+BW^LmQ1#aTJ|mN@bG2tw{>rnPxAFZk1iK#8#lq8d3mz9ql7*L z?|9`0lY~_Aw!P&k=bZLz&o~X7ItdYs<>J{5T1A)|c#{b{f2D2VU`H1frV>0M4whOB zDsVx4nTado=D&eWdzja_*X&+s-gMAiwemRCKwO3w>Egy*eO6~D)7UHYD4lH}GHSZR zPp^!A_)$5FVhx=yM%XTh9kyI48ksK}Xd2VjRc(8$3tg&ghpb+BVUO(dIfys$fc^cu zk0p!jCEL~|VhR;6dXY!sA}~bNxO|1n0Y;W!(?@oXciz8eklMv?d2*k>o-4>{a8gdx zhf{Fqv>aKyC+vS8P-OP9H)v_IT{X-nWY*3a(bRAsT7ngNGpk88Xv<7HqE0%o5F{6V z>GAW3#p)7>(V^KhbFB1UC21HPmNqhIB-*5kb!wZ%hD%p^z)j=CuNCSXz^CPJ=b!gz z%`Pt<5|ob#6kwO9$tvD?&i1!~OAsnRzi6UALT_4M)~uG&Dc7v7ruva$>hie1``0xY?R^>7 z$>zrK{mt`peKez!GxPa!Lime)&d}|bnZ2Nv*-$$mWjPWM?-xX;tgx`Qtay2OaszOF z$spPeuU$Y7ud;;X>b2BvWjH*)3-z|)f!@G8thu+nQ4L;kQzjWTnmv}bYt8Q{G?EQ- zqX7nF1vIj(BG$lGK{Ae(Bdc@}f5k>>r5BDw|>S5oAwhQrs9mI?)*OW0? z%GMQjFwxt9NyZCy+q;R8Xyaaxp;bi)sMc}KpTK%>{b<;{_3!D23xtS9v+Y;IfypZm z;=?GrCo`g|Y}e7L0v*9Uf`1_Pv<&#A*d^3*T|~9CA{&&TVJ|>|cvq94v9QY6%7(Gk z0Sf*g-GML5R#sQjX!Q5;co|jJMn&dpI>~cXXl9}eaZw;=h{mm%cwQCAzGM+R?^VOl)*=@MBNA`a!$R@U?!)lIoPKC z)Zkgs15?^HAS&`P17_89iXHr_S;ehIk>=AW`fwCuw>Q51KeH z-Lt<wqAi1Qb=Jc0&M_U}mJNZFAZ?Id^hqP{!&1k8 zCAF&tF67~Asaq4EbHUVC2KK?_vJpgS$|iG?+$2(fAK6B}99B_|pZ(2Ctm>60+6c-I z&sq`oK!jWtYB5Li6z(`tB-L|%B7gJ#?oiI7N?(pMd=s;qP452Py~i5a`b8$7>S_%-mH~1ST6DufA2xeN?hY z$SiHLQ1&ea*)Tv4x-j&q(QaGYNtaKb7e3Mxufx7tR6%PyDNGt?kL2pWfRT(RU<^*v zle9jcic9Ui61|T8;hNTof*K#y#WUcQX!;iFsmsm&%lKOtzU;e zJZ>Pq%->C4-wlT7BB-*MfWD3fhw>62D>n2omrMR(o$j zjEYpyRo1o&&reAzBqWzDLe%&}{I@}r4+Sa{<$^HP`8xmvUy;vj@J*wYar{+1P_C@; zZ8W>MNUdK1k~YXMzdK6i(;?fpQ-v%RSVqz%j18yeVY2sRb5@1^MKI9IfNmN#oB55& zB`Dq7HVy*yQZho3{973BF!zh9?^~SZ^P^Syk7;4ub{*XVVQEZZ1=Om^t^TQ3x}NFvXONm;W{|2_m;Kjibk;G}YO^JlU#(>U~- zS~JV6wzO$5+_S4gj!RXADKJwlNtg13C@wih#1J;G&pve$vSCjQbF4Ga05{`nl6ZYlVTaq;Pht)Sdk&w zSQM98`y(1OO|!bhTlOa&N-F(E9X3@`vUO`z3Ln+B;VOqHp2`4nqQi16p%{zA4%?)M#U? z7*aM1afmPB2=jt_RAoit0%n;|PVaWsbSXo>tbV%Ha11KaUY$QsRMY7ewOM%D+)79k zLZPpC7%|l%s=XemIec?&U&KYz4dq6Jl0L#)MSYB#8DXoDzKwVSjPbPF^N%s~oo96Y z6cSe$))AZ)ny#p%I_*Lyzu|EMv`A-97Qw2hMouR+Q-wOITBouhdi{j(5T!!Bq=xxu z6l-CIjN?=#sViuQ=%RH&MiGei=+D+k>Pk~j4tDpW^*qh3F&%@nF>Z_G_zx27-WY6r ze7fu(PTN1%2pgl8hzuuT8O_^}11jFa3Kqwu4N^ej7GcG5ctsNu%(FIRN``9*FG7OJ zHcdU(8r=}od{x>c9Uzjg4~1Y3?mBUm=fbx!-7}(!(PYky8A6rf^D}-BPrnBV8=guy z%&8s=W<)heq$EojoC=zgpk8dXcBm;OK36*0KDQxlWE7FN2D$L?pnzc0s9OE6a>Yfx z@tmB*1;RV*P+aqHh&@cfhsK_24}Qq2dFEi}%x7QZI=`BmFfHc2W*F%cvzO??am3If zuBi^%6VNoe74fA-9;MJf_~17&CuN3M#t;th&U_h1N)>MjLplVVhW^#|HSO~CI+}mN zddaI37jjs|XV8zEQYt~+(cx?;deN9%2EYM~Cwh<%undnFDH5DXNd*l}J&8>%4?+?& zfw26S$}I;ti;;WglZ>Lz%VJU9x|;8l&O6Dg8y7tF#0K)e-+4%U=||_yO2G=%`@}%* z>j|Uj{r`-GH5fb)1>+a*ebOmb`MP7TNPv-tHgRmCAA)^V2b~wks;iV9LQgV$3w`Zb zftd_{^>yxl+dGz|^_JB}x0tNCHT8J5kBLq>Qr;5Vl+-Z&Q3xSrFXUPBmxv&vG$_tq zLg-qvfPa)uY`8BAjshsou>N5PlN4`C_RNupz@ki$;3`K?7cmQT=395|Tw`aH+*FrV zl!WX^v`)7mUNP1wdLxhX=o_>NizorCypKOO6+Ol}eFwr9e@0ORC5*YEu8^z%8aPe8 z*{fs0$!PR&srde)4GP5s{R$$bKPWVZ)b7=HynZ{id7X%Ic%6v!{Tsb%8J{|@*_^`|yIHeNx}9+d+u-taOV^3JA#lbB(ji$~>dnWURW?;K|SvCBro*Ur!j0j6{o< zva~P_luL0d;oYDpGgS}-)~Gq0AIbrCphU_W8#nTcV>3}#rjes7!&JboWKiy3qvw2~ zCD}urD%JxC*Z~Vt?cqYQP{{h*n#@$=8Fq3ZnoKp`2$XltpcO|AV}rSnOeWPe3)V8z zJpvg)^oo=>>j-GX6kTng2nDv8ImR6(6tWYX9?30erv%)qbff|Koab0HAB|m8ms(12 zfan5ahzXKje<~BkdZJ*QV{4AN5K@pdm-~FWdE@0Qg7*r_>H(gj@YJ z!&Xc8#s*@I4mL!n_tG|f=m2f~5ux4R)s7_SAL{r<_#@HdUNfz=XUia4FNOA*%by6C zO~PDt`S}t|^%t^;+;Zz3&;jR^ftiGtncfIN5y{M;+zEZ@hM@G-`g(8vgHX2DS0uhD zSb`sa5O3%+%5CN;+e)1jet=D07XldCwjl_z+(9I*f)--7#YcOAxERx`bke{&TRuC9;Wq&`wV@RZ%C;4;IX zGF0M>G+A__^Sh9G;&$T5WT0oRAS_!q#5(Bi0K%kx^84@NUQIjA%&ymcSZI^xV)g?4 z&E?2ZQ!^wn*Y>UMN%;0RpW;nn0ZTt8M)RR{E0Qm(FA89(a8?CJ7UVA?J@!>x%SRO# z@O4h_g{%(-ur*%J5FBKPip|Xr1i$&d++K~rD$1JQt8BIyy?#P$fy0R!pQ`&kg}7_d z4hBNjo|Iu@u72{mRL^3;HR^1X=#`^sIms_h%z)x)l`s2bw)yq$yeL|&TR!oRxC?`% zwYErUppZAMt_1z?w{MY%oQ)=<|DcU#M*753#9Fi7+@rQJ%qI$|Qmxw;+AP(SuE@HW%lx8(zU9eJ2RRoh4wS z?>GGdsN>Om{hIFmVjWMa5(m1Ecd&(Gu$T%QBI?mL2#};}EL<|>kXRZ4^ ztHPD{-R3{-`o4<|4*6}$uXmdRsqq4{-(!6e*7JUli7Ex>AMoIY7Q&CT#EmyWEdS3St{1}-yYe87YP^N01?+Gcf};*% zVLiT3ZlIC>NF(o#THXV#j2l`x2b6Ngfu1+u7;!TH?sCv7u+*7P`i>h$32uPV6K5R= znrHvWp={85;(>=g-S;>qsvqWDc2v=C)m4D@D=nV9baw|!4n>yZ*L^0yo=g$`lGl}e zd)uKPYK71j+|}z>C_g&DkjI?>CXw(rOC35R<4&y$}U8`W9PNo2CN7+N6` zM$OMa$ZI+3N5H7CHTj(kNA88#;0g#+u2wXPogz~aaJhUJ1L=?fkM<~lFAhnyaKe10)BMipz=e1yiwd1Bjr z{l_w`ppkc!78tCtJAR_!Z=PF$3m&?se4MWT2{B?9!kieXfF6AQfe`%p968aOv2bqS ziBe*oT`!`$_6hdm$vr|B+}t^$j{wp`qy1!Q%vC#Gj_b4!=YvZ4mB<5{u$@$(FKt#T zuV?Q2fyMDotxz1n8<;R3@*Eqfz|L><;<)YS0QdDF_>);Wp$962Pf6h(PRKVCdye)v z_BT|bTy|#z&X}D(#<2YNz46dHGxNICV~zrQ8}F}3Rj?NmPof?NH{=)6Q3phDjGTKallYU50kj-Y20(CqreHIZ&!R|7py2CxYd>HMv` z5kl|DD2}G+W9gP&5+Et>A))ShF_IIXcjY62l5k)g-k=Bqt}P;2;=~O{bBD(si@T3Y z^DX7VaJn^(tcaC~h?zxkF83sl9;eKHZ`Ohouayy^|GBsa=MkDf2?Ch+PJ>YGK-l`G z48Js4rGQ30Ploypq*8oQg%8;$-Qi2`{WQ--LvTSZjDoq2Ogcg}w+rAI=-CBr-cfi5 zeRuQy8iAwVVC_rcVDWj|C#h5oYFhM2OIqa{krbug1FptU$uR0tiy!6{{V*Dd^@*9` z&O-&_h5U90eAWU7hAGuXc zky0-WL-=UQg1a+HWDTy)y3mo(rN8%VF*#iFx9$(m*a^jTD>of>(S7F!LdBVc@=pt) zM=tAaC&Pm_>%+_J{liHu(VJEH7iYmYDdd~Is=fCK@xuu|@mp!QFUlO>Id1keIr3}^ zWn~K#uz6Oqd3wxi9LxAxq zum&0=nEG44z7!dXdIK|DwP3MIzNQIi=9sLgg)4xDjNO<|02y%eV7yik!xcbIK3U*j(O<6=Q(`ky&EjU&J9V4 zQ#>wzs6dnxZ9$yRaHrn!AG)g^8!Q~8vL>dLc@#YU6&^njDfs-DWRcu!I5itgs;W|= zViTX7Jv5Exbi{Frbvz>-v%raRvCJJ`H9!EKJ5UWeVvrS`J%N<^-o?nZlcfR5h#fUK zOxRdIYAwiUY7Nb)!(@tHAGKigQ;uo0G*%6o{72A;Wz5RP!Q|TZr;%zk`IMn!JZnwS z*p-q|D|T|0N=vOIud%yivPNcQx`xTj6^!wm-xSDrIbgC~QQy?oOl$2Ca*|dtODcc4 z&3J6Om|)yC{5{3*bSxVh?%X5Zpz9ado>P%aRH$~oWQ{zjS~#v`#~--pHX}Tfx-$hp zz1Fxf`lwG2iSau|CC5Nulr>z;)0D{T@Bl0$3@-}h#>p9YW>$c1)8$1`=4{4VyW>1Wa zT!E9iBBo;TYgT{gu}#J=fU>gZ4FNM*Yv7|7LdH)FjPD5PJ{f`KJl5kEh-fl#YramV zJA{qb32Skq7hV{)K`3b|>5N*Vct5WMluQBAbsG)*4Ua)M-f%6j;sI;kN~U4z`?_ki zlR3<0s^OE7@oVPBs=<1yA~7maeuH8O=j%s8*$zdG7)E#;*YftIp95} zMRHtDZ;aWj+}D+JTn}ISq_sjewwt?+lhyvRYfbEXL^2TR*p|!jTD#G&4{bT|?_s;Z* ze=`4HjGa@ECPBBZr)}G|ZQHhO+tc>fn6~Ynwr$(C?e1xv{$oe%6Z=H$h*Oo3S9MW& zk*ij%T<`OK*QU;WQQh-K@d}9KNFGL`wq9vW^`gx-v5;E$X|+lbAQp!b&WunRT6+^DZwP z0m;&Was`ALbRAZ~Un}Wk7{Se~wE^iGcWNyB8Ct?NXjBp2>WY(z7Clh?!;+oEpLOB^ z8GCaBzU_dzma{Rnu#xAGnbm}gS0I^*RhGerg55@=Lff>B_YQIKmstqD89@PW67#sD zej2oN0{)Se=$jq(m!T{0I5UV0rx3!AxBCZOI$4>7E0YZ2iee&io&>@x8L6raK9Zj_@ITn2uhb0^Y^)=SGx`9oCJ*hc3RR9=as-2a$WO29tRg!l>8s?hz1eE3 zB^ji46^CkL8>-a~Dt3wNQ^ziV8oAQbjm6(~)`RSHgY0Z%RtvWL0%$r_3mg08meCV2 zc1l)Q>%?YG zwmj@NVo~>x0Xi#l)_6udnh%1`-KB;@Ch+T_3hI=HDU|otlz)ra1Y!1Rp6U(qkRkRS zV7EJ&3-DeyHhLQ$p}jf0({r_Ycm}!E&2e+{C1b(Hfu4p5B^D$;*j-HNVH$m53}=03ULH_%7o?ch|{$AK%D zi%ZeirYt*i==J;%yAM+ASreTZK7D2Awhc~y0v|h>27xVPfI~6L8$)G^GLf*pw$a-j zM-PP`mj1}uDB>&lxL9AlEW&c?Qcys&-09?2tZ+BAK?uP&lrS)xim&9qjbBS&>=1tM zLExkE#cL(dBe*we2t64>?VsQ8hCH3J+X=GdN5PrxQ8<1B6z9Sq+o$MW;zFRxMG#~y zn|ZT|5IYSrd+Cds!u6wvH*K3IZ>RG?>cK>+dc3O-tT`^|+)`=JH7MzT-SdUEyXC+x zcW*Me`hZR&FVY{Yk&gctR=`q2F<$cwyERi&ncCE>2s=alFTn)Q1CyNELD*<$JdWbc zyuE@8%%JdAZh2U5wuN`RrJ(Af@PnMGppLT1!m|HSZMuauc&n0qe8M9F=qRk%m!cG8{SRU4&2dZN8SL-3 zbHg6?Ia$pd=1ux!9B(460dVh`0rS6q3`A)ZQg`YS+&Gb2@WZ|y6@Q#s#NP1gYGy?j zj%@aPXa3xpi?kI4%{#X(hIBe^*&fkUxxYacvcJVZ-qGPMd^GnyfV8s(ggc^zMu+xrt9&9S z^06*bsn=;jLxojfyN1k<7MjrF`v{O%s<6J%0yv$j20*toJ)W4bFCGa}6>95r?y}9- z){6dG81@Qb)?N~QNe1{cjuY(6dal$>aJ^vCmV$OqRu4G@U?EdRB2 zZ2i|~xbkb%11_?$jf{PBDj0Tdz8>*GH|T7m^=zmf#!T5zJgfURYy^ ziN)s^`akXjfm{xLoav32Z+9WPNqhO_)hW;=`+|>9@ToYG{ZGQ3PFTKi8aZcp?NJ-y zb7^FsZMUgA^r(b=Y;blBpjKUy{9d&XI$Ssuy&=UT>Ok;R2=#VpqgWf<5WNF7KRhKJ zTqW(xeze-n6}8plo`S-BHsOXoh-jUpC6PMk1pe`zh^^+5JADAqQfA#6;?B;ppY(Y~ zkZ)J#xeIqYbd9+PZxuD!e};WTcrx5w9V5U@vn;xTahdpO^|w};P213O z!2j%GF^f6sx%gpd+ED(lG98ismg!V${dBDyHGSv$SR7Ap>edUAGfJoZ0_zuq{b5DX z3Tu)QktIr!=E41$B#j|uaWW6eYw}AjUR~z5n@bVP)!wkIr29wOS5Ws<-lW>Tx;}Ls zU9H|>+0r z(-Wz$q)E{hrM$M18BvxsQ7a(9ilu0WJitK>y_PJfZ0Gv3&$WG`h5IAqk%5w-;aD-Z z5;HjiSethy=0;=ROJ&KD8snvVGS4mN=r>xq!}K)~W5uS=Dw*d08VZ3)-|rEbAM{MZ zPKz%@ZzC~qaw{VFBYV}jD}iN@Pm{7IC2VFk4X5EMF@Gl3@K7-V(N78k-pR~85rj|@ zIhJOem!)`T->(H(5Tw}en<;VD+UV@gPlv%ah*Ia~$9Gk?NS5$?orX!WhID2hS+swD zE!)nlb6_P*SmV8VDm&K2H@`Ka(=^otTDY-HVME!IlWqzjBcJwsjcwuV*EM$}jkqqF z=9VFnb(W;Lt=m*cvP{Eb1@f>;(U7y=*-g|z{@3Ym>#$dqb3Whxj8;@s*77uWnyOcr zTiq%!3=ER-#7pDa%EQT)Z0)LOrMEW=&s98Ehx0R*8qKo-@xBx7Y4W~bh}peA(8C#k z?B~^%gu7^~8f&H5Gw^gw@HW&-jqA!VCg7^}dU2QnOy1FGrGU=o3=Qdwgz*g;Z~3cC zScXutrG(}FLTrp+#6FcynQ*Lmx2^Vjvm*KO+4&Z~IPum~U6>vr!{9Lfk=tA-!IeCw zZ>m*t&*ZbyDJ*Yx{ji4Lk!zjoHSb@n${|jNyhw5*?KrC)Wb}IG1f;;%fmm z-87Car<$Cn@L`~8?fu54<@K#1>UtEr-tA5legtwBgIQwZS^Gkl$Y+|W0EZSwv}5Ma zoaSlO`cf_wDbWp&jMYdRSluZ=hMKBR(T`wEeuI8V*J2N(HAS&E#nNPLy*>2qaNHMu zbZ`9kNX`?H?`m(rA(?gub+UAQhiCP@yCfa8h-n6t}sTyyI@z>4zcX|8$Y-!|A+H_|10z z&~A-N>k*@<=zl?7Vl0fhDH*muzD zwga?U^{O-|n@uh_Mwlx*Nc0NMc<6l9WB9esoJEMYRyHza&5Y+3gls7IE%qKxEk>sY z-A$9I7D(2!Jj8>YNo~hTk^_BOAu;*%HZX1g!^+N;3{myhWRsJ@U|wiW9NnaxPi9JL zIjc4acrQAhzrLp=(a)}8$qGU$e04@-xs0jECFo!!*B{Iip9@5%nP3q1Xxty~ipk2I z%*|g%;~GhH(hh$R1-Fp-A9x(vo~i{qF{PQt36>-dhCSDE&QZ7b4x7(@3MHj2>b_f- zk(aF@Q$+=v8+B}n(gvs#wO5+mXdlC&BD6#MBgA~XTv=5IW7b4}b?=}+c6L#-B*=ZD zBTH}P=3W*GAG5`bpV^O_dAE?_5L`0S2FmH@Hd5;_(zc{PcF1JM2IwtvBX4$n`XAmC zQ@$Cfgo`LAO!oh+J?Z<((Y}xbu4!X^4)#mbD(K#nqWg#Cy5xd7*YyaZ#ClwGtN}17l#m-4?KYfgC_Mn=Nk|2j@ zyoVxg)UUq<)=!Bdnn4E-<7UuDq5w!3$CDRRw7LFsGCCz7n<3U-^wyulKDJ?}S8i(HkPRteAI)pJ+Rx@7M;$EH zp0hOxA|Go7g%Z`k#gpgz9uET?(b^F((>LE)^daSE@nk<4uyzxlxi3+{9x(s}*9HGjTG{ zL?!r#i@(e_X@%!biD6HX8#k$qVrbeTKDN))tcnBeqIOD#b*cbLbzHgR!Eh{OlSe-z z$^ul@vR$a*9wf^%qid=tomD5~B&eCUCOaF`I^k6EiAV@~9n2T7eN$kP$ZxYzF z^K#r;`KfO7bm4>6wfh})@}T&rtQW-0exhhr$VBK=lYXbwyl@56X;*2gEzd%_W}H~G zB~!S_kFM_WQu6s4_vj6PcYZIsRIZ$vqU}ngXDCt{l`XDTU3bA$F-m7mr!7R2J$s4` z6&vLt-#bq(|w~`=`bjD)=>V!6h0wu)AB<#DZ=_h&5XU+BXJ!P~N3j zp4(a(uX`|03PYzcr#0r;k*1#;FS&K69C%w)H>F1kN8Ii2^gk>?V-!8>IdN;Vu*+~a zPNxl2965|R#H>b2A6?LL<(@ybxa0!d+d)cZ0L8s85N#l@ z{{|Ss4RA@=aqF@An_e&_Y;36sC0A4=5qFeene{J=hX?=dA||6eS-dhc<7g%z873$i zrM{^()*QYa@~|_{IAbm}7REnXW0vUD$($-oI7H|Q;x3FB^kj-?HQ56$_v+Q68^b=P z8b2DUO`%BwvgOSWLYST=4I+SYFlBgx^XQF{{=mwIg$ipYV+Mci&dxZ+8W1J(>{yTgd9Kl?)u zx)=Ny;kZDciKX61v;Bq(K`sD4gendsZ8EOq$BFORY0IC_esIJk>&9GS!OYi~LxsHL zqz`{|B>rjAfk8*#A|osV<;aPOT|0T51eXP@Y91ky`c1s0kCLD*S^4V8Ufr$|pEoRy zy`_CTZg1GU*s|g-rl)O~vJKs+rtn^7rtBV@1d~xTdonN%ug)OIyvkQ0WQmjB5cH+3 z0Oa)%$;Y(j5~nwxq}gvd+Ru9OcR9pl{x?Jc!hSQzE&c$WYtiOS7fi_xj{D1i3cmZq za;*%)ds3U8f9j6i#>Zc5h294@&woyLPMcfUjDS6u;8uoRtlk75UQLr%J!bn2WcN3} z=~0dA7qJ2Ro_BRFj6~Y<^goptq-$<4k6aSpz~A@K7lgQ1r)@4L@fu zQ&}v}kQj=5JH4hH{tVUX_BM|2T7d68H3pgHSciqd-}7;7<6XtW{=f=B$-olh*tV+#KOci?lTM2E&B`ugazg}&eKLq2)l7F=5LvaA7#03JK29$q8oUl!$G1Y z*!N*dQzcyHaWoU}WY21^*aMiI6FMu}maaUYvRb@9u4$_*P=tDKxI>hC-Vviffm| z3QH7?YWh}jC~=`N-l9JaKr=={r~JXE3TmlXw*Jx?F%YZ_40)#sxwY)LARc_;_pG3G z;qfc9wi2K*pSX<8=IS|3REEcK_1c3%fAaU7F*NUeR@WZ2V3vG>z5IjuSWa;a+=&k- z%ETNC1!SJZm=IwuBumn+*&~dN<2e|RM0^n@_{Zq(AqPad`&=i#D-mlcc^Dw?ukJhd zsmu38ECozW!+S%!oPt)IS;6Q0(Nx$G$MJ(;@(u$zeQ%A*8G<;4ct!lg^f}|&loS{$ zJxy&5^$!(!)Y%l7Ihc6#;S~(%CG3hI@`j3=oA3i!E1*9c;dy*Q%Njk5%xlE;RMTht(NKg4Qf z)d<5Rr2`n#rYo_bams0i#;~-hm8=|awzHGJ>tcsNw82Y8mdhuMPDM+nUTlZtSLg#@ z*(7rLT>_}+YkJ~p8)M1JOw&HYDqG^zi=jZB6uFwmrQ*~2&nR8+*;cD#`GOmoR*kC0 ztzOkvzf7)LMWtaMh`KTvx;7lTHXYJ`ppVKPIGQ_Uy)vl?y6nJBsJxH#)!s1_G0&Iu z004))+nKT{NT`Qu^!Y)TT7C`V=4ok9GpgiGzYJ9VL(G(auDEFR33Vqicp~pUr(lwM zN2PAcWKwu%HF#kun}feiMkti;fQwznvs}Iw&2-0Tw#pT$S&rksvcH_|{%zCb;F``qzWow%unxnN6FKGeQ_3mYJ{h?&($`?Ad+5MI>{tW`%@KzT_P(v{2 zycE(JtgN!83v!)rVX|>`1Xqq%c)=PQhydeQpPt4{|Z^rNaWsM zW1RntnPL@pV0x0o-Sz*1nq zq0wL$?MlN_?%htl(39GxSY@q#FG(-XrQb_eC^woXBD66^!ruJ~=0Wq?qtp0qa?H0R zA3Ilbg!aT-vA#h9+#_G`DXgOWzGCotnXIMb-&{GbcB=j8^Vtzw_iu$JbvYn{7SF6q zzXG{XYwZ2T9#opd z2Y9q5A)IO8HOC+^7)i-0P0u=vqZLDe;ya0SlYmjdvju!wk8#*D^|5u0QTH?V^vZTG z@D*x`r9)lhSLhNh#1nVE+KIl72T6fX!mOO_3e2rg7h=Z~v2IQ3?!iCM%au>gJcwJZ zPg~!d8T`3m%-Pa?B0q5aC5!zKewp+uYn?f*hV03Vt0I$TwAZrnPKG z-P1ni&Aj^3z&H6nM?=7y`!VFsUw$!gR|BT@R9EchN{;L&NB1S$9S&ffZ~j$ zr}ovTzgi-EsOMIJOocBQ0OTecOTgxJ3bES4Ko67|5{Qjp{`4g?d-HYJL;rkbS7sz6 z5lkYx5#hYy*w3S9iy6el4T3Hni&qO7M&!Kd>=!ih3=Co+9-|qHYWk$Q;02a0?mD~C z8JwKxhr9pCc&ghjuQFJ~e%?DW4Q98fz}{u)!Rn24J(4fMJef4H+&kjb>!$lb0UUSC z>gza&2e(qkhOhVWublDIH|;Q-In3+p9E8&wym5S|&p|W?!4g|X5Kms%kMJ`-2zt{# zK0Hzeb}hKy=fbm1USV`<$KU7=2t||eRl!}rbO%bDShV3YhfOkv|FFgCXt3Mv@!g*r zEwtJ#c*?U(aovvz^YopX1aiJD3$@&i-`}Z}G=c{&&PBXTe>0 z-}t9fx@~^DN>zVIx=Ys05eCog2k@i>fx&uj-SS7)o`}@0dM{n z(kNL~>&I^wldmP){Zx_iy}%5uF`7|X4ArqZBS|&3&k#~!!JSe>8o~nn5#d$Gmxahk zigeB+An9&XhZbo(cwut$#(#tBq=V!0>HQtym&@n{Bvg#5`{b2$xve^{#m@6222Zkb z_i+lpr@WyLH*4Uuoy`t2-iqxjXd^QUXjrtyB0W^&ReLuIr@fwF6{Kitb~_3$x$gOR zLGmNE&dFAk9bpWB)A$q>vUJ{t3DR{azfew_&q)*!O%gS`{nOl`)YRvuT6xO6=rpBz zJ>!VWIpBCPh;9S}vje(?C@LvbhIJl2!Z(p0h_5Mu0hN^Q;OL~ji{|N-TO8E;8&GuH zj5Wt3o0=GF_1;A1_Afkdt@c=dM#$)w{hAmdtCPsrjbW|A83-m{O5m=sF@)#H91u0wav)Op5raqZZl{T<9CyazpJ{4)UCX$JQeh;LGJ5!GTm1DOQ<=%yDt3)jX z9!>Yg$X2hl@sN(T$M%?RRn_jJzvh25dT{;uqHAG>BI|f)r2h{wppz#a!2zWQN!&PI zVt!gLQVV;fdlG6teH95g{Sv92<)9H>)sLYEqzafjZEboTlzUDrfaBjOo(8AisReox zn5Y8wR@phLm_yX%e2f|9)I7^T0JaK~e-(JH4fC`Sl1!A--!V>) zVa~}6Gk}aqI6SX%6R<+rS3wl5!J0@`Tm{pO7Z|n!=!_s(&X+=TaSB~ddDTh4|3gL< zFN<{E``HgVKfb#^jIUp&b_~|0#wHA=|Cfj-;$Z7$Xa8UOAy#ce1;yj1%MJ>BKp5DW zij-120kN0T(#a|6~4wRjJ>ZTVwV#ZF&WljO<1LEI!_ z#hVavZUeeGtoUS$w(9-eqs=J|TG&$MYipkRo!1IxD&CU99p62<9`aI-u;o%iU_6`5 z6}t(BFCOw-N)-8LWNEr(p*|F(tfP(v%2gSrmmxAiJc;AfCZV#rbL4Uu7sS@YDLGMA zRCO)zc>j#48TvZ-gpI;;IAv?HVWRkL;}4%ax`W6SjFqG|F_l@`8tZx65>X!iS%42@ z8$tbLnSdgr=$z)c;||1`8{LtGD8wKc;@Q*ahygi`SEQ=S&>of0)O8z&wAB?mLy@`| z7NYPTPp5t(3NZt9k`#8M{s#UF9l*KiH592&oL|&SL?1$;O%9+);Segdd5HQdC<{R+xz$O=o@*X~c|f*h305tgo>9>a5)I4p(qqxG z_l|4~!S0iy&XpNaFDU16HR!L6VF^M>-90qdj)3&EFv~(2A$C8=FgBg^!g7diu+G8a zQaW^09mb@Gq?aCt(3t)+6Z<1SFvpamDH*nx9gTqIdWq*uR}Br(Ofa0vs+0v9Kz_1J z8PnEHVTz?WbTFa`m@?0>6A6SA?|9YyktH=4 z7WO^;=5Q3s1oE@Pd9Xww>?kFU9R);L>0bMnY< zVVf2dYA=}1LFP?Z!P{TohT{EVcf`*@p8Eo~91G!|R^!YYT_SiyU;^}eHQ!)m5+f4K zXe=)Sqk3z~0#sEy1~@NS7e|jlWMV`3SWOCf@|3<*Fs=E?Od(GA^Ud+zr^dcf^$ZI8 zTb^OVOT89fLH$hl^t#<}@emojm!0}s)LGYs8-SJg;E{Y7)LxMKmC*Z~KqPxvl^#nG zJ~0`4pNK+Q%FGBB-caRnco7OrU?x?D;nJb1OhyMejpg@BBdYA2dwM26hYQ{hHcme$ zqgJ_mdc=Aa!296=-T-g_ITNjZmIv?k%e%&0CmtaBb9*W)>)1ZNF)k0nVr<&$ulR(q z@3gSR!5e8hB9mYE@)V{3D>HnA<}wH4mn|cNF@}DWW2WVccEpRfToT;&c=+vVTSdKK z3JGs7E;xDGj1qCjpr_t2)FdObM&u#3(+rYB0j%Aiuui0rtd`}B?Z7#+0_aOVozxRr z=kf78w|P`$r21kGHZ^(Y$tmxfKNl)I98hAp=#A;*#jSm)aYod)uwZ@+PgYZ!dwMcgSi* zMAciQ($Z?H6+a#34cEm0_7!-h=lf(v{U)yLHcXrEZ&dHY5SPNyrfyl2s8|istw&qO zgt{w&TkFi@zuy3VNo#)Nu`RQI0h5F8pdzw2PQLbe{Kmv2URET16q31+dBFjdT4mw| zu_PlbJZ{&0a(_VsZ6{Mav%}zrfE^|c8+yj( z0bIQz9e^2U9T}k?ycBwrbiq!V@Ua*M6^ur?j53Yjx=hX`FgqnlDgl(_e{O%-q>3}7 zL#K$C#!o*rJw3fAHH4mzx6im=;&+W`a_9gaf2+0fH8Kk46<1a4muzh=SYSH}draPn zparg6E~}GRja3}5E!sr|yfH`qIIZh)QWvrhwSGo~ z-4daKO(%N#MVcClm{k!TxD#_JG1~@V!|ozF`{+fY3RrWF5`U>NNgQjKV z71Aql+K8^pzi}tpmg6o7cg2yIhKuj8t*j*IRWeNF!>8OVPwAa= z-xgR%0>i`i5~=@MTBB&-hIh_AJ40<`H?QKx>}n`Mz@B)P7Q@c-h@&LVgO?kgS15*u z1=Mk&uc?xAH-BzTPd+^v*$ak7irW}EM2>6tK2`6mo` z3NsONF!QQbF=;gn-F$?!$cTc}-H@l@4>AuN57}HSySal5F0<2(R$&{q46u}ar?a;5lUqQXKNbxxZdKq=F(M-E=rw4_eV= z(7{`b$7z>j(^NHZVi~0y8?eT;X=OUYX_E4=|B5%~xO&ldeIwn0{*d!ZMlq7lR3h2* zP_<9L1>VkPcOV2ALbpjiP1Y3zynS{ButmeJ+T=t=6_1|*o7batEH55Mtv}O=mnA3G z^>iJ_2p`w;(aH01E5pbq7yjr0{IM5;nSZpzpS91ouf0wD(JPYlE!l)ezqx?v5=%5ZYV-O@IsEM zODSN#cUEbOFo#kS%1jeaO(Pq!&ofYEBDLQyY7KSShyNSl(@KtSD9_t(mw&6ds)>vZ zr!>;;c&A&n%d&Uig0v5FhiVXoX%OkwiHh?o?;DakIP$8+J5sZc@hiR4 zi-T*dzJfU=w6KkGk8X+obQycAvu9;T;}7Fe`I<6mWd@a5za!o3ecKQhf&gZj_sQ@- zBb%K0SDX%tv4I3F@TBhZH=Y> zT%(gWSvOg6F7s+nDeu=EH`h06cj39$0g%5+%3|$zVWoN6$?+EEmify`uyqL{N-564F@ir#J}y507+(ngNzBHA zz1H3W#7zaBdRvbfDi1KgaWpT_8=(i&;tlRw7V!)lS_b2mwoqVZh>8VTDlj4znloJ@ zN8n?}KDx84))0a19Gl{?Ke<93qC7FGYP z?65GgE2{KSYUCU2UD?Ko6)TwH)8m7Mrj4L|99e#T-4F%>-p0LzefRM$1Oji!TF0%z z2>aNZmV!%gES-y>T-iLhfpR5MvzQW+{QIm(ihdgZ`a?AV_x^bJ7o?rXu|t@=C>}!wD&)M9#Q0kh zdN{?WgeRbIvlkc(BkYemjxs<-2Aw8H)_cVH09m)&*w`POz1*wIPmEuk7lJc@-)MP8 z$XwOI(A?c&;lx@?-U_6Fm`b{Qfbc^567CIxpBD7vnR7Cuv(@SAG-2%vNw~va1H<;T z&)ZN?rycOVE*S3HFLu7Rif8#)$ZM(-HUf-mCT=~}cSc&Dv>5hLx7vs1Lh65Q=qV>E zoc+AhtGVb7JFO!;DwdG4UWiY$BwFZy7>`t$_?X6EWgYP%ZXJ>xrp>cwsVo;SJvJP* zZHa1gJY#e3L6mtS`i>o6P?z)ex)Q`H$SwXJKfVWOj)ftq!@qdgmqg8N@Q&9qh6&m; zw58*BdvU9R57{Wh>8PVGHM5daagno=`79_|l8eqb2dz*Lgl|e=sIE-!@zzqgDi|LA zmy*){wirefT+$?E75J}(vuS9*JtHN<3(l>ciB0H8$I@Ez) zQqZT0DeE!8wn_kwf&w-GWLn)U*ci%+6^kZxRli99H{YNXkL^=b84qn&(e-cv3PCRS+(y zri~flIlDy{GPzi zt$Q9+eveO!yHc7W{rGV1Z?ywAY_A|$q4h$vh>zhxBT`I0+QG!etaCVI5S1JCgs@*# zeJ-`2PXJYirjq+oK){?KChGi_0UIVf$>wcK2tvlW{oAgXXZmYtGmq;+juBoc6GP7%G-ds;U*4$H{%`Nk%Z9$lW%`JRqB~6Fn(XLY83XT`c%cf6ttWro!|@0h?%^cla@XQsB=L5k3r0Nh@gt=A=d} z+;6Os1Lg@~SP8T_ugxt(U|5=#(+n*Q{8*3wS&GCb8SD2;kBSx5YiUgm@!B7YUn<+M z9-DE-^nO{nu-MjQTERwL7SJgX>XVZ~*wAL?BU(ZZqxvionKfHwxHb6F#=$u?e1x1% z{1_qf1x^d9nt<6-Vn5plu%h?Y!>ZSBkJM`tK@EGW3fT(8qBS5VHOa^Ld=SB4ho&ct z8k4ZHCmwF*5w3|*;}_S>IkM7!Xa~g5!X(Eot#TsM6_(S2_mNvx#UvM&5DxX`x#Viv z8%#-%FG`WmSR`Xt8ZTiw8RntEuYQ7)63?P(#eEK^ z7JIn-4zd0%x3KD}Y!!?&4VF=1x%FMoKk z5nXD@Y?~R^imVCiB`i*i7V{apU_8%zp~JbS4Z~CTLJ)PUca^zcQ2jy}mfm7bNRphV z*F=r-AZ=^?y=voc7~8{PT1RT%j2d`Jc70omeh3p2b#P`j}#FP?;H>TdPrbPKWU0633 zZ|b8xx^keY&e;6(QY>bJZCME49-RnjsqU7MG`WLlemO$ucXnmnSy}h$Vq9cX2j?w6`9f`EE>9J8!UX>i3T5JCr+twCQ zW4x-^`p^L5U|-q^CrGI%Lx4w%z8~;@5yNf0<}ZyiQXKNDly((G%E#tIgmANOl`$_F zo7^%B#t)2s)s{dgGE8G{y6wd5C5N%o(mSHIrz5G})=AAR+*!9_@>n|Yo7DUlp2hCO z2k(`t{Pe8oFGGnuzQZ_hX+PAbRQlLQx|eY2;JSIsJUliHD=if{6TLIRczQ7)#T|K9mR&Ke6y$fj4-pYum$o|6*ZTu`F}&G0pv6#vUCi& z$;{d@VGk<24yaY!&M6Gmn! z&~L~Ie3JOC&ExinNs}5D^&UFl2j0y*E9)pQhha1F?^7PZ3vg66w_}zmNg`E>!Hq^} zX@&S@!Oi=|h(8slz!Dhy93>1vv|dZuo?*4DTw#Izt-^AbR&%YYH%?zT_~N$d z-X|C`0~W^kazE8f80ve@lm@*1!_pm;rRZkM8<0zQM`d#8YJ8--& zpPyy!9*=R^T=JX>18%f%Vjr~@810{PMRze{WGf6z)d^MdkO@)}x7cYc&f1VF!5OGs zo`s5Q#?ptmlq4dJC?m6ZslbelG;$$w%t@OMG2O^fSy=h?HNA>* zg#Aev%QiNjM7|kWT1OLpjJBd!5vtO$^RJNn8GT>rj_Wc3u37!3P}K;8G{{v>*aQmm z@nj7h-r7J12Kps;&f^uPHD_?4+QcxQ`D`>ncziZ|oyZpV%T%SFoMC)bNon?p!xSoq zHv{RRA6J1HiV=beORrxazqaB8cx#=&yb+m<2kzD2*Mk&JAllzxqgf|*S)_>@IQOn? zJ_z|yV!5$#1&|DGY@av0G5BVTD{;($cP zkexaa##HxvPK17iH+CiQew=Lq*=_Gx65_d7YsJbJNB~t$m5^z)({J&}2WxrYLTUN% z(G^51s(2W8>*F@0Iq{ZuJ?c?O&OxDweSQh8Xf}hxsx#18p1(({$|h&gT!yVXjkUdU zi_+Ma>XUZm6`Gm=Ic6#3-v^G#AWi~*A&J9L<)&|O;W$~xmTt`*$!kKR4K)uT1Rjmu z5d*eYy2y(tNj@SlZwffK_JKceF&Q)zX8!8mJ9y!_;IL`Qit>48MYGc36=gGgt-inZ zrjV#5VoCu%lIjG7cjlVH9z?Mo37AKte=AL(yb4K4<19OM#4r08x#j!di-3;af6Vi^ zhkabJq>g`u7nK>1K%EA!tS&5;DjeWIW|?13N;fJ+KOC_7^JWWpesT1Ozfjg zA?p0?*SZ9|{<3A!_U1Nv{N5~bbFgxcL+Qk*-~pA2{eFb+&vHJ0JXD!^p?zD^##QIl zT2`Ll)MzWs^>6h9J$7f@L8g6}d*hh!7k0Q9}__;`b77DB6+K(oTj1zUIYp%xSb`=jE( z7B&y|mz;-1!7s4{iEvH{5K$dvpt)hqQVjM73AQgxyEw1)SYc%x*GdMrYAtTsx`C6_ z#>zM`VP+H|=`*QVfuVr72;`QF`KTwjz^6dOv$lVJW7SB^0gJB-Au$Tqf)ih)Ap!HR z4XhiwmCx3jlv{{wMRCkk3s`)9*uzS&q!YEMBU9dzCh;+jiF6I0czBT%^%942-0osE zDL#I2m`ZG4?FGf}1Yq)xGi)@4r7 zgQi^$(qN9t%JV~6$pqP-b)kdfbu7(+1|TOISbrFN(3nalfmwNtYBgshv^zRz5K?1! zsH1;}eQ!+l$l20v;8=d}j9XhIq%EG^ZsG=KCGH#XMe2N6|38_gyM8;59Mz-EQkblPmwqHLn_I|B#ZK6%ypF3H!qpXJ3=x(jzDr##X+r5#f41nM&_8emw`zn z=Yz;5ubSb`fSzjoL4p~BOd)Rbp@N)LC71|J>VQSkD5PYg8(!tWU#Px-u-iz2R>D2Z zhMr~xv|QzPdWZ^>;AJI+>o_ilgz*`%AoN`Sy&sQ~&WhrxpAN_FRmL*$C9%7x<%*;` z51=|ZuV2>A{rLl|U}I+cdJ4RjWaPtFNSbkuK7*CcDx#Phq$uelkK=^IsB1I#+S(U; zlA|jgQ!&$|Gq>6|gg!lTw=FR6gW$$D29U!exg-$5j+kh9K+r=6V}0uYJ8)So9u|pn zj(2f!!RnYa<~FX3@~q*K=++6MgwLz$<_oqqSkO%avTu+mm9@sk;+`YH=MK?hpR0=D zb@_ zqrTOt9%>hKp~6PM$Mz5okk*N#WE7+Vfv@wwX=XGffyY9q$FvSo%dv!vxN`XX3!6v@)ST0ksJJNHXXxe8DM)%OIZ@n+<@+QykaU@kOqbmqoh({5>wB z=pCfWcx7Xl|K!N9QYJfRd5&Bd#Ikqd4u*P9%by=~0Ojr*YL3(al8Jrl2?0+ejhI^# zO=PhmP;Z(2R->2CDvWgbi9TEuY~IGnFI+pR=CH0W2f$7VI=ZK_ET!}rt*epd>VuG{ zYpA%FX2XD0#{Z< z*GC{t4%b9<@{pt>qVO_SrL;T=xNqiiR-+7OwBvezFK*FX>6S}e_J_l(Cf^%zcwZII z2`V_S>Pm|r3dQOX;}-<~dQY1~joW>E+~eD4Y9tRwOrwDODTU6Ua8AudUpy6&=OAMy zz+bU?0%=sL|3zH`*DdvhNGigxu{wZ;_?JO7N;+ZClFt+JRlQk=7*Js7XIJFl=0QnI zf)n@3BHw+j5shPgo+&-X=BbYofW(l8&#D`o;W&~{jtUO*&7k93E@@Ndt3ynFcqz{U zIr2LVkfqq~{`G(8I>*>vgRb4zwr$(CyX#%swr$((F8;M`+qP}nyWMS1pZA=c3 zGMP-){bA<*u#%aTb^We?Var2pUJxk3w)YJbdWn$QygVZ8X;l_>Q#%`^dgh87I;)E6 zGCfMa)QZeBCkr-4(i*6b3GUXlv5#>$~{r*u9U+ySX48UAaf=vjQlqbjjLaB~d@u?yTcwOqip zi(-3isbv@8k%`(czEV19q9_J5@Om(sd!9)7zBQHrO-Hrx9N#mC(ne_`Q=4-iUxG2q zDWPKmh$7Cz-$#n2jII9U@|~7;eojZa2ZzMG`vA<8Fq|FDn6eC8GM}&c$joKPEGI^I zSiGX#3a^H&f3=hxDc03gLrWM(S^#Js&Jr>ZgMUG<-w=(FB7am!M&`RPopY>W#}rDF zGTEKKD`f(<_Pi!#lvBWE=++;E*eDR&?A4;!i$f6bP27uCJX4U%I?QFWSsX8hsFe|i zmQ{bTo2Ia6vE(hyefXqV6-#1U17&eIP5?t9)!k>Z1j=G!PlYvMwr^dcYN+Jh^B9p- z_!j?Cu1Jlfvai2EvIHyb$LH+*5rb<6+ow`vziULrpk_AAeq}lVF*j0LbZrLXta`)_ z=OIsE9Y;@7a&AzA(P;iASXd$yi?K+(Wr72&ro*G=n^cw7`&MuHtDjIgZ%+_MF5c}6 zA-;&v?#?3@+X5F$WRzMXW=PKOVlcg?+E$w*H~r20O?h3(j1+sIhG5FHb=5%UW5cz8 zS)x@buVLLZk_z=cjrs4&cn__Jge)y3lB9LvkJFy^1iy5>f@OdlRr!RGoMr(XIr*G$ zCr1mu7h4SR_5~57(j$hPW=X;g{7b@pj2Jy|%axLS%Iw5%zv3zv+Um$Tx?;7%PK{=5 zZW3}kR)D`_pi@doRT>OhDBVI$Z-(TtpPVHr%))Q_CI-`Bwsf5whmKj!zr|EEfJ}223bz2B{~44 zLoJu~;$0$~6l?Zlo^v!vW4Xj1QL>V|6w1q!c)=*cOQGhOWtzRKjhfw3N*2`TR$X`6 z&V3yp+__6YUaefA|KQIR;;Sl!{nFaGvt%oOOS(2$%SyAchiZkOpsH&<#F#4#Y)?V10ItT4lW&Nw_nCV3K(~gYbp^Fke1u{yZ)VB-ZvpnPKFGdCLL4QM z1fs=6@B*oA>k3xn-V(`xH9-42!%6K6n*uIjSr)Kko3U4hr_ip)oOW`X(xsr)#A4)%idoAFVZ*V6HrGlc7P*g7>(zs~oM+tJ z;s!l>n+Wpz^o{IwUa%A%-m3n{W>r@iwLj@fvBq>&LJ6xOPXn*A_c`MRTVdzIF(p*3 z<;3NKYtJ7#76dumSF~M@H{%E0Y==)Ht8WE4Pi2BQ1oBDOft;DJ*IR$}Eh_d9AYnwu`8!@B-*nde!ZdwM+Bx z@mK#&QSPeGg<|tuHG+(1t~mkzue80zaOQMd{+(!ctTOgQYzr85q~9k~mU>viDQmeG2@9ok>~^9>isyt{;cva~(a) z=c$)>FFtW%bOcHp7BBL!uzH^$qbO$%&sxHKCbTT@i6#9R(;^V!sph44!=v#*riVNn zFM;MnJ^mN##8G4~OHQ)*AY$(~66I%0`?gbeo>T;ANf-4A!t1;?q7Df&NxyS`WKxlt z--sxlcCHi?DieC?48ZU0tNIZODD^K+A;XD_R^OGz{xi5?XUY}M?0QsoPCGn0&99#l z7kV<=wZvv)+4_@P-oVd2KKclbJlAet1MNzs3_WIlcV>f#t_>084x1vEu-7Z~cCVgC zjgBC#)V_k10wgfMRj454?L`+)A?BfMB}+ z(M^U6=hM-$M!C?yqMRHZ1oNb4-S~n0a_f_;y!+se1P{|GYqnE%!__n;F$8?KpyINA$gk@6~62UUxB za6uPFzGg)2TMk+KG)sFEZ1zvvE#9UB#2*s$r#Nb?lc3OF$Oqb+#4fg{reW!l8%(Zp zQbDx?U76(T+wdf95>aZ>o9{)-g%X!E*nVV`=MP`GZvnbO_7^Ts?J7Xk{ZF1w!<{9x z#3C-10oGeSJ$&K>V9rkRi2Lf%2SRLQa@hHnZi`W9bQywo;R zY#rKwx?&5e5ZY($1S9H*XCSUPA|b*~8#->oCTo01znEbsZ_YDBhRJf~sI%wV0&a#L zw|E}^&lbfKIWrG4MOirTJgZzh~shdw>H{_EkrFW(mNc7gKV9^==7)Tt{h zs6KzlzIMlB<1r}TR)3(^=1I9FF{nfl)1Yp z?G(7^R%&3|1_bF6^f4e_hlJC1@$Sz@O>pt)Eh8!@T@xmg>BPT)uO)IdD$f}KqTKd5 zEi7k5DjT5b$A&G)?S!Kp7_I*4mK76&-W1It3-^8%`MGkJv#-rc{Q?*t0mOifUeyew5~>XeDfCkmpqp@>gU0V zV6RH$=rP-(k)Bj;e*H~IuLfjiZtc*|ki-B;IAzEzy^&J4u)1`R6*D51f?e659m$ogbBJ>Aj*`r^Q-mKCK`MVV+YjU8ua%(Eisw-}l4R2qtyFkwr zQDf`?2BsfLUm2zIY4h*f^}6z)>HwK5SHlZZCuNpmWc>&o{qrgyiYu!g9pcdR1j9ic zu5KN)$_?@D4s-G%xylKJ1){F9Z?Y!L@QMd(ye5493Ic1?E`-L7b;@vaPtgq)FU)Cq z;wGSQZF{c2MKod?_7ZtnzmY4u|J@K3H{fxNq3zE$qV}3yTLyo?)g5Hf(&C-u3tZP& zPN@2t^PRv4tk0BQ2%{ril~(Q(pf};=S03g+jEzd@w#>qP z@|Cen#qYS_-rs8I>Zz=Y>7rXI=bqmYMUjHg-#gQ3O(%^fJ}p(b)fQz#3!4w?6I{GT zl-7%vAb+b+FL5fLI`W!t)_hh%UFDX&bceKB)+)Ma#;tB1 z#ci%3N!Rl(Xnr~BQYc^0JzAmluU1z(Y9k!-Qd-}tP+@PaSe~w~At`QO)-B>_^@`G& zuI8>Gac8|KJQCqF)BwJ^-81_R0ePW_F|;2?9bdt9Ip=?IM|?uuZuPBY(CM0R9`bi+-?6j2u6M4R z6-{%0UUh{{__zsY>ox_gyydT_cCMRhG#2OTDxsLGpEOz@Ty=1DCTc}+Z|?mn?B}o% z;tx~U?zbDJ_jBK@=Y9N>Xucc277pJ#^KkrjO|+XnzYH(vfj$ zUvqARohuA6$oa29Tl%Sx;A|*n-3_}Qko{x+9W!GJtVe*niZO7bW8iBYW#ZYOQcF&f z-acJh+^3#b$|n=vu_qG)SZTaC_OB07>~j#ExJN<{6!thvQ^+#1nJ+2KBWD#}e;k9^ z;dNNrr=;ZiqvDh#GdfS#fKPr+l)fbYBrOh^xZ$K$<&CPrq}SgYPe$Y65JyEovSJFD zZw?NR$O)s8BFKc`sXlB|C&mWIgs^@B2u419+5MD#rd?A;RScYGteHJj$ruVcmVc^g zmKt(ZpqfTD4W3|rA99iq=${~HbN2P@K&-=?DIE{KjuGh1t-U_*pctNO$QNG$<$}TF zg6PF5`1oTDl@gDz8c+!$NjzvOPM8mJ`9Y;TwsQr0V!&#c8fZmj--A9tchp@`u2@wq zD}^!g6geTbRpx*l6BMYMs?q?8Vvk`HdJ)9oE{Jmp&F%-n{%CnY`7#JEm_zyA1avXs zU78zLxdqdZI@0NzW)G^xsJeu;g}-TWYJ;{oL~#mZ=*J}M1`W3KLG?_rv?rd-krr#_ z#mhsxAU^{uus~#MWz42>sn1^#Q|1Nj?F=p6!>4-Yk`c(;B_zi`Ruy@~5{V9J_m8R% z81%rv{6$tAh!-;=3>^~K2%1zuO)rPQM#4nn!cVu9xH6dfb7B=uWWGK7(^B7_i+@Bv(}O?3gzSSx82G!Zwom4|@AV3u z>(Dr%@v2Wj2s!V?6w1qH7(u`g7Ea=a3w^5UC{Q0LkHMdj7b1wZ8_3RXduq%5$Px*gGqKONQ4ysebGdV zkbpqgy}fRBVgAYG?L25zze|6w%z;lKx!OKu!tl@z<)jd2WRXkgke8SfsK?NK0=;~Q zk0T7qi3-`oIg?g4@bLksBK=HAq!FqtYHw)!0iOmDLx6-MG95DJAd4e(6yX z@cUb2zMMMtA`S!cZok4d=(+&x-%AtD4Zbn9uHcy0|WDDF2V2`M?b%8{TP9eFNY3Wv2v9m4!nD{xglF7l`RDE z0|qK|_dzblcyXxFED^38eKQj>K?nIyX-}C(PE+agLGa8@o3?meCZko z9YuiM>tT>U0PH){%n|a@lVrXSpIQE7{@Fn))$kb6bgo^Ies&aaU0jN0 z?Z9%#=lq<3{JDJe**%m`=x|RA=vT~1#le_-uI^WRxaSP!3_ZLfg==*l5= zho(0ikOOTl2_H!0d%`nhK@Eys>j(2uz0@a!wt7H=(Sxx zRboo;uHWP)fOcEp@mqcTc(=8*5)^fkb%M^BpdkN*u1!J6af3lnKN>`^5W=K}Di+{D z7^e;z1}%UkrwlXcYQnIlVsMGozc)I|0cTF$ve6yr#v$ynyP?{3~Pl? z?R=AmpruLG{pW`uhs6vxPymUQ@T6vMFH2wMg|+I$6(fu;@t^~CEY!aZ%=9D4=Ej-` za^i6~mc64k7p`&QK%+kGb2tXUD%62GOpSK1oe6U~ZdD9CMy>w`M3)YLe88x-W?om=$ifFMwl3W?bD)YIn&G&qFN6RR8V))hv0+<;{sVfD8v7L2A0HwX z<~d~B6E{#_n_^B@4nvhg1{KVN&^;TBDW3`<>&d88>T?ea?Sn&Zc8v7pixK*382L0{ zq{iEWVipLshxJ=9o;?W>$7xv@5sBKlGPf-kGilfc`gCDb(&DAil-f&R>3D@On-Rmm z6!n1q9pJ#e>zL0#o&MU`HsCFA*Vzp596&4bgw(bYuIzmsQXSKBaIw;9uE zSwQAw5Qo4QEZU_s$bU*OU(0K77b~AYGVi5*1^M7#V-s*(INoyzoKl$p&>$AQ?>r+A z1!UO@Ad(TMRoHQv;fp<3^6eGA4NUNiI-T$Uq{0ol-{5lAeqnE5z`a)@hZi1ZX)$rv z57hXs{TKfx%cd1;Ij=@r2yXaOpTa)Sb7vNmsu=!}U zrWfLh)p{VwOOyLVuC!B1wIS^n>~%ujJ4v@X7ARi%!WMCc{eB*lOjiWNree4mvHI`ap)@eBCbq8k(gXHXfpw00 z0I5e}N>q%(Xit?^c_-xH9~&BE9iij;0UKH|EpGVeVSycO>b60i?m8m|$A!l#>JI3E z!-gbE^s{b3NtD%44^3pZEOh&f@YSsN)ek9~F(9!f+93N9CA+^^@wNjmQ%9Q~uXjzS zcvxb@izFuPG2htf-AAe;4n<~hYuqaMh?DFzIV-ch!Y9eB5pY9*ym(NcIrB#0GlMyK zJX8U(=FHZpd!P>YbUxppQ8Be%Ae_}R@H>$cd$Rs85iVxf<%(SW58D;olX*O|yGl{- zc?Yg^GrqJdBg>s{!tF+6^fuz4Zl^(vHK$4^!H~E=m{}V^cySG7-r_Sa*GMyDoGU-L zdAlE?dV5aTrS7c8p96^GY)&HN_@u%x*%A$ayx2d}-#Q~jOv4zE2l%!!*KK;DN8fQ{ zy@8kiDA3w=!i-isnKn$PcjB0lag2$?{{AzxwD=bTA}LW9J*`4>P@t=DkOp^)2|S+o zN`y1qevoWEU%oZ1Z^g!#x1=M?p5l~F#){mO}#^# zH(pcQ=pc+cH{Nkgz_Q1+gF=Y$Rx}HoXn47Drd6KlX+=ehQ_{(CQsEdCtHfy5c@xdli6WB6OV>f$#c9@yLC6{8w;?<` zhb^n$oqM^gOSF`VP!gBO64R0l9cqP0ngaGtNFF>}xN|mj8aq^VLU!EYxY`H6-yD+lpPVVPLA!NXe#CLw?%s!8 z7j~i^yVVB38zryB43*HQ&s!GJa!9S7N_BjP+VskiZvIWO!Te*=ITUI05E1-gMYh>2 z)pD6&!`UbGD#tYA3#-y@AN-?5ZV*T|@ue7lmrZ)kAiG3PkD4SKhZ#P$6v1LjB5G&bq?cR9+Eqy(8b)7k zu}S~%b!Zm^%l$AObMrTwKRM@B^PsFrgKD2+EPO_Bx_heHEFak`obHdFv!5>aI{bwE{2%3wE>;Yxp`7B%!}duFcAF{eC}fJ{!%VVm+_0pg4dLoElTj$q!0Lp1;9;~@4AlDGdpoV+s>U? z3WoM^nAI^3SSj$vn2*+%l$$wwCBTD$n-IInTZAi}0qY>L4@!Yk_C=#HG|hPWpw_EJ zvFj{geyT_`izUeAjSn{)pcxF(eqfJadAf8I>oI;J<|XecwR04uj{nfl4`F9I+=QCW zsE2D1Up42fc~ESd5HIepfCf$1plLE|rpR|1aKepg=|TYar~^gHkJ%44OOEa4fXbVb z_1Av!vGbn_uVanV#BCI(7)i7tzuK%qGdTv|YVbunv1rv@i4Sk+{HxqT#Dlje^U_`w zVVWh~#$`pCx9)-M#)XBgPnD5SSF~AAIS+%tm@Q2=#L`80CDpRZ^V{HfN-_-j8C9%s zA96*zfd51AQ~H38XcBBM~=j~?+AA&na)2{ z>AG22k4P|$$ew9dDZ9sr*;sf5oPGV5L$;h#qSZ&fBTqe{*uUzNQsgI66 z8_@66-=dnL?;1apP%QCT3$C8Dj9h-EHmC46vq}}e-JBWC$Q?ePAILcw$3Pen_l4L2 zR}dfSu_t!hsjA9(Eir2^uy_uqVfLvl*)y*LkihI?7o=;>t6|_NYV@~vSq4@u>w~Ch z{;wRgLnEm}h`!vUFx6TKmNh@?0pR~B(eBQsxg19Suc^Yn5 zMV8M#fnE2EV=q3|$gvM07F2eul(`L&o{_%A{$lfQ$k{5j4b`IX%Ok2ETAbv&Mvvi}=AWkFK zXUDwc4d`;%DHP~yyV5jWp?l95rb#gEM==7V7Jb@QeYiCDN)V`$3)K;{?X)zumFM^f znmfo$L%a(hg7wW=pEdBRXOgSiaP2mBT4MRa5Ta0I95_`6%w za`l*ATc$Rl^`IV5rz|H)X!ak+f!zJwQ=xO+e=@eLTNa>Ge=6oj$V*7^lWZtl!(T0s zb~vb>-3GAiktn=GnRYkf!@UaYmNsFnyK(n5Ztsy^a!edORx*^e%_VFury^_t*!7{Q zh87(20@R^#n+56Xo?mQ~i7HW3t@9*d@obS(UdQsLy-^cBrf&f(EXq+rn_8c_!m zTv;ryIp74h>xx&~A&=WOnhnw?T@=)&PUAjL`Q*lTuK|-bZw*mC#e6j?9BLP<6wKSr z@CX$&4s!U46w@jdvAE4pqd8hd2Z%T>$8kr8%>no-j-%%cYI_x4XC1k(5oF{=HTcC$4rBr;OQ2$4D+dq@z2vc)^y=_>C}rymw#M1b>;g( zV<+IwqfdkYPDDGpw@Vb zX3B9ol~J_K@Fqz|$lTz(h^iP^s918mDLYw0_d75Qx3+taI|#wu?m|7U^4U|mOjDhM zM&T&qlHf2Yp(uu!aKA6mH(<;r_D?o}EDp(TeGR(!=j*gBEw0rwaT3vQ#0I6!zizjR z=|u3JoUt2@+`5Z?(Og$sxZKxd^vf=7Q({tgrl7+SD5m!l9!zL;oVE71IZbhlRr{s*_k#bl3UVJ#)Vm1JK4#e} z)&UtlJ^W=7S~Gng7UK(R6aGQAifYJ7Y}x+-dCe9 zfyuS}q0Y7WL87zwiK`Q{I@!h8uXa1>0C-hx8~)(2dHuq_dHe!(^Z15F^wHi;>5bt&1l(49e=K2TkZT^!FwskTotw;SN3qr+Xk zp#eRKuqo@6y#Db+yI>^R{2+I_R%e>{fREbiHT1uIb*Ot4l)54`h$|k_ z(Yrjvm_f)aeb6T7#*%lcBa?KBm#n=^<+xZqxi_wi>l!mN_F<63_pETrod2sHhR>Kn zX=EZ+lwD(gYeZI*ZKoX8(&Tc*IHrF=x0l>68vi{e#x z@2uN~ponvJuck9xpp`u-7wuSb-K%5j(M*^k|{k31#k zA1A+(V6q8tHy1k*7rU8&nc=WqFX^!V3SI1+M9dUpmpfEs~Omf7`4>Gr1v|YKtIo+w1m8W%m-gDzWyOle~;{#5S z0H>@!S|(uten#1U0+{&o>y|l8&H2XDm1k#NQDD=c(03-(*kzAJ^%@GM zM#sDBbK?(O1&<#wXKS96)<^}cT%bG(xMzIH(D=N!Cdb?bkCclxtvF|B)WfTpZXfGK z+kPu%t&L%UnW|9SnjgZ3YW0tHJr7-mt=~zO$hpmh+zZIlm@Hapy&L5m808!cpv!tJ z1^4N!>)X8>S9WUcq&&4IVyCLSJgCS3Y3w=AZK2cKj5*KcxS6lGvwx(dJoanz+(`4> zRG>WeIA>j{&;ZV>vne;R)9aYCE@Me^f&%3%V8AP!%g$`b46mCDf~(kRa16?8K0W|K ztdtp)Cl?Fl)siCeV{T?b25gGc7bHo&6K<@R~ z6A@4M%-NhfqNPWwxyY$GFApw&qd+AQ8eg@(&>5w`*$KjB3k9IFoDBS5kxW!q6?=+O z)#vqjZrFKlUQnK3mr_;y?0!pCUpJQ-N1dwJlEiXQe&-zh`C2pkKw*z@rP_SFKXs;) zjSOK|RWd}Jmn^VjKCIZ^Y0|`td}?R6siU51qU0S1P(0sp6ToR%U{zV8l|;0cR$zYc zaN`{_{d{Kq27gyryA3}(B0Zg~rTp`^S*DQiBt+8?mJP0i^s-u+Hy9 zizL;JF>Di#i8+0Se&K^x{f0RkA0*|4)6n=fb>;*E_>JIfpW4*rTg9vwx%4`zTwl|B zf)^fA3QhkkIL-Z?e>Ih%PB|Yq$B4StL?!a>D|HTahuy4S&fuy(OuDiavnZL649JNQ zbT5XD2?CG;-S>2vTF#apoJk)E zHt0C1Cj&=ht9uY>=26`UL}rpwfGL0h?ERxT#n{wo?$5JY$6jGkN8DQa#BbGUSrZeR zbSHZUp4qH_-9)Aq6N_RYV%e@m3bsS-+E`CSZTCkJRZ%K%~zjC1cy##X8bk0zP(PyXaC zjtK*nq4-L}Fe}7*mbzZRjLF=^O;vUHhrWa*C;|@^8@#6(U2+TYgt2nb4%*g|>NzX2y#9Ls@oFOz2%c^;fZzN4|7=Mj zpmT^TL4knSA%K9W|9iNCn4ycIil>9AoT-c9f7bs8Gp3?%p{?qqATJga~GJd3M3k=x3J#tnC1pvy^vkjoD9 z6c+?jN*|FC$PdeBw5nzuzEOJ>$5vS=4PVz6(Ly3efR=eHtk!Ix)CI@nz1WCtFq?U#S5mWPrzT{R)SZ*Q)TC0iIJE={44co z@hME)A_sezGDv4jsC`FC6zG*Dt(vxDM@>|)@89A$YmVufsQaFO4U3i)Tx3iKqc+pJ zV~S|2VTT{@IlB3NJ4ogNFMkbynn-&&HIuM_i`Q%1`#0=uwwcXXS;DtwL z-4uQFI|DbvbxY8$pj)~)rImTEq?vhgeUUP?ipJ%Khl)t*%iDkdO6VyDZeRQu^-^mz z##vJDNNJxiiOwwJw$|P5GVtuc-Ig}sz9o|lC>+ zGd`|x2M56B7SB%eG*ZhUB1kQb2tTqs2)sY~a%X1Ck?jsA{>w+o{MX3!qHwyrH(2_> zu!iD}`;(U|j4{zLUDGF)+6^%1{;#kzI=L-;F?$5gZwT9sKRVYjPIq=+eX8-2#7`3X z^&aM{Pu#6L1hTCDvID7Yphh1ZCT;zhojGPz45c=@d+x7M7?!xQ>;Q^o(~#pDNjPo~ zNJIZ-7|dOlX7=+=#E#4p$-mp_`Y>*N1I9xis=C(k9ax4~-@VCN6qgn?9ht&EwmK`y4`@hR2LA->jGPQf^v|RX%wQq#~FHWVX)r2d~7zoG}9~g-4 zf6jO#L+AgS@juw8|DE(p-OxU`!{|(3X3Y(&3tB7tcr&3;&@HIs^{6OPL7)wEl)p^I z>uo5B!B*>2hawq$Lco4eho*h>%!0v$bY?@lx8D{XQz^#%{zXmysyZ|9dZJZb)Eo)o zv-_Q^U**Zm`eAl5&1-JvPWXKj=M9v{QcFsR2IAe~KCQ9a(_t&&tv@T6HrEnHu2jdpWX#4XEC>HXl;2RcL`f5+w*kN!#>yIE|V5fa2Ql z>gcRDl2qUfj8uwl5s?75F%Lz#G3M{$`V0%~@45N93Q|-!HBfI298Xq4>hC(J z>UtwOxHB_poI=LV=+dq9*+FhCU#;(-nfYo8G?2Kwm>~yIdn^Jdu+1&Q) zMbdfd#xyG2&JewhHQ6kF_B4THTwdqt;!CwKO4}hczqB^D9KQLK$db`I$!`jIEsHq? z#0hj;RZ&twaFKLd$vMfr9<%D>q!^`M)bDF3xG%V1YprgCAEl))SAXK$a*03jB-&hC zYf~mJuq|xoY^7jX;Gdll<}FK+RFf@nuL&lY1Ur{HVK%?i7HW!@SH1OhkhNUc;R{*S z65ug0+tbhvcWHW#lwD08c1uqTmC0K4CPfy{w>v9cyS{ z1i*{%TtTvf*_a+rgA3)UK~kQfX%&eQw^21|gp(kMyE-^il>~>2KJ$2RFXaWti^lSx zIhSdMBjah+3`!y3=M7RJ;Fk?rAmA5?LWle14`LxyuNcyWZ@6NO=Ckv}oX+v!Q#e?3 zBXl~JDMJc|cet(rM7w!pj}&teyd5lr!lm&x$_7mkY!?g_5b!HRsl)xsMX|&Eibc7@ z{i+Ai5I!Zs%kg-$!hyp5N(b2x@GA#h5b$dU;Sd02qTu23EBQ&Hr58-eqNSHi(W2%T zb(rBZD~23X`E%_&SJU~Au500pk$0BTu8MudK71%E7zR-a_pnUM)b9~i;Jv&_)*}xOpdMBzATx0a zesH4`jo0+#bKN{!kgpM$zGZ9xv@?a5VXe-jAG!_?pBSwNvmjeUN8KjX2!dN)tgUUl zySrV9E8+6K;V(&hPF2JvMp;sZ3yn?$S8*@1rc)$5MK3)Epk@)m3nP6qzcyUe1*xkb zGV`{Yw_z)TE{0id`+@BHhuTa z2l<~4@=E1i{l@;Db{si4ZsOk3+p=v=);E#}_tU!y+WDsCADuJf*ErXA$J`0Ni_ZAkCacgDm8jJr- zxg8cK79Twcb83}v;TT;!&z3uN=IAboRu}5PN+pj^B(X7Ad5Qu#Anx%wV0VnL9KoG8 zo`crnK1vv7U(QcEaS$aziMKQ%+v!<-f{Bfhh`Q$Yoq38nPj!A>8N>TPWVEpA?ldZ49$g1=9W{A=RA1YX`C^(y3K+n_+{t!QN3UG4jM<_c$F!U8N5 zNKwo~UC5HPJ3kA?kt^FzY)WE&m2_3O4SQySlM%;hV{?rs`+H?=aZ8wBeX4aKyeL~G zPgIyDe6PeTc1q;sk38+n$~ayFr<0X6<2zd!phR;!F_rv%d?T|m*)zqzDUd*~EYbIV zLVtwjFMAL{qOqCy@zy!IbE*2w(%9FUho?Zf1bD`PbpvQoZk(1=o~RJ?gpEM--#%O( zhaw2b<>s1OX=Wy^UMVe0$=32E_O;wAbGNz{&}#%W*~v1_nB%sue!UB<*Hu>=k#BTY zjUs$osUI~rS!x3FCn&$gOJ+iZuPAy!IabFU&h=20S~|i@9=UYo(X55{8I1&DSPhGg zXkp?{yMvmr{2NPlNTif}1xC6Avl~}YX>GS)?r`sVS0h9)45X&8B9A)qAMI@<1O{jo z&Y^VjfxwHBvm_x4C9b9-0fI<&2$A^`G2=F39aN~{%L|g&SiA~*D>z81IE;zDULvG5 z>7HWS+0`*Cn^nT7Q7ck-*N7k122#8cBnW%|h=D88N8kh*T3IAn+Fb5Pn3OB3ZF@<1 zjh(mD%BA|Fn~5=0tNkPsUR?C3$D1)m0LSv1zP1sEW4 zLxV4kM6s(S?TdUMHWKJ09Mo|sEb!ey6smrqDee=@_=*K52K?VMe98!prh>t@0et~9 z7;5C$fCkZ;N^q=7xOIKIkZVyrg9afKxTD@3CNH`cTFlVV7MKXF2$xQ5-fnaJZXQuE z7V$s!{fm-N;BNxLP#mT8!|rY{n$PHgNoK(|*@SwIAU0vPFDct9=nuTe(1dx5QIh)1y9XdE ziX|%LJDLS7=!FLfFWiG@vN!lh4__{fGFS$C1c@vQ%=up`e=kXx{d6&FBbRERhJs?{ z;6425y6E=e4GHf!XU@ZcC~-=HB+e2T`>sYdJWvs4Z{vv{-_EE0CUV}DM$V{IQA8{r zq$^XFL{f0s#peAWrJ*DosJJ#pxZ-#NNGtnuHR=#p2iNkkD3-7eO!VnDr)(SvR^D`I3fHsRA2xqhZ7vT&$E#}BE^M#$0>0!@JC&p zu&SI{mzduQd6aQp_y=nUsc3`2Aohi12?CtWGK6NKqeO@Rjv5QxWo8ts9NHZ%209%% z2E5@Ab*P=dM=u9G;eW3|hi7FIxg<77-^3cY zLA0a4IDpgyrewLaOo=pK3q4gSiJ*^R-!~vtUH0Rg5rUj+O27iV^Nov^y!1&mhV6>U z0VewJ<-vxcg`8KYi)F77SQss_##O$MDFY}CR??9`EY6=>sxF2V(jD)KYuV_xJ6V3Q zKGJ}b2wsj9VnQ3u1xtbqjrpE}3mHhHLZ{Kfh~k8>8i2(e;L_bcepUe`z9kYq(>Db- z-4XHy>uiUEUPG#i1T%(~O{B!dGH0O|x~EJlFe zFA$i(-7iLklFt(?hkgL1C?O0G*OV>mv51H=W7CN?;A*KuP8`*=RyxYzF*$i>+6k9N zlg1MY(+W|qq7wdyCP^qOC6|NyaSp}^4CAm4Q}PUv!-fZ@`NnczjLR!1jr><__L-mF z<>g8g$BFx=Fvd+O1(?j3WIzS-ma=;oNpM3oO_-JShfnL0OuWjs&R zszP!H18jRsFyV>?`N;(27AAHiqqjVN*rb_6#hil$o=^wtq#>vw)IU~bz1bQ(u%H7>UUe|)eWZzR>4dI!qd>1&yxXFV2hTQ9^hCWG z{+{R26-tgKJVD?IrVz16#8^@tC3v(b5}_2?2`R2S%+W*{6tyFB52gf|Id*j=UsZpo z-*o>D1)&cqx3Zd5y7?J?zK(t;7Jd*&3QnoeIWsfRl+>u!O6g| zEzvNxfTx&$&sci75F_#!wp}mLVA&DJ<5zKJZ176HD6`$M9jkuAu@nA{z4i65#*M7Y ztJ^6p_P!jbZ#COMp>;>KE$Ec;mfU-F{|xZH2;(J|2BwYusm$Thc2kwo;_4!8WlV~y zheTatVRJ6zmWyU_k8_>i${+_6$)`_#7nQ3hr`Nkh)ioQR1ffb{B`TZRkdS#^IH<%m zho*Pel;Jk?^V2gquQqk-{QiE~OUf}jzKUcaDW8})sRoq&#q}XB(r4n02cd-Q@{a^> z(kO9pY5K-|a=F36P=D6IUduGmxPp(Pu#Bg@y|fd+n4Mu4<7OlE5DDwrV&oxejDW3X zK({-;nZZ`z<$P+IZ>e9r>rzW3PznSfqN9@@rrpNZ4c5hirMt4ez9^{tEIGfD$%Gv# zDq{~IwrJSav4}x_Ha5ALa(ETmJ+W{yZa}5YFpEiFAnH=EF|B(*K0j_=(@ztWK54A$ zt~EArZu8c)cGqrzrEG3#t}bj0W+?ZJjl>ZfiuUF?!-{oOv^fb%LhJA3aR{0+bzvr( zZ(&$p-iY8|8gnxl0^|E<;|Fe=LV7AV+_~`A)HHV%x+gr=N;}2!NUWX(+4Z7h zP=K?Kv%91D-DMg$HA&~;wV)}*Z4!}6`;x0a=BKk`@O zG+uNN2R-wD(lM^<1uu-htT#kwdTp*_Y*7U>$tGEQznerlDhG{RT$dlaJ$Av@2zVd? zM#x^Gs$X7QpGskC+9f=cB?ZynmDEBLZ}O2C@okPPfv(L$iP4FF)hVUlBt;&S*m0=d zG+h#@Zb{&}=+m>Ut?n+a653o|o|j%O>$=$5y%JlqJ-N8m*x7CxWb$f-Ur#Do&Iqwt zw*8{kak=lTaaDGebf>+*O*!dVnXzc_zV74MMBerP=0}N9&y!%R{=-n#Q<1zncByk# zkFXtj!^Jk9R)Df)vB3NHgc%3U&sYK&Wt|Cl7#rLukiD_TP+!6XO)5GeQj*Tr)1*a* z?^a>oEQQ1Uw~CqW*cn&4v9qbLc(B5>X;Oer$P~HphIi5@b%%JzCJ*Fh@-l$G@*}n$ zo0Zbu$^59aqCqM19y9Bqv7FRB)Z`l4I(FJMnK+D1ZX(FsyE` z?$EWNHCq2uQg(JH@79-_%BMT7&bC9rqxz4dC%WxazFLTc1u}3s(t+)N0jfY%zZ`1x z&zLgRUl(K%S4)hCV~Mia<^Cx%=)zcIB!+cpsOJwR{Dwo&LL+EDcOMedY1EQl%t`{9 zq7yio=33QZv5vpAA&enpbGW9mlolIT0i$4MTEWaz!K}1`SplP)4cMy&40-&O=`GSF zA5C$g6^O39?P$C@_|pj(_p$ z#h97!FIyCD2>UU#rcF(%Dy40g;;o}lFoLlfeuDg;Gl;yKY}1r+v>)_tMng@RfCDIJRMiwenl9O@n2kt{zsk>vj#-f!O_@lS zCgTk?;Rb*8ip*I;5`_ZkvqH1xqkq@NH6t~kj4<`B188T`PIghgGqN4LS~ufGjn?6y z-fyBSR-+}7$WsZd&w&%`gQ4KUuoXM)Bu7poa1SR=U1QDv6Q-oALt=g+i53db35@wm zSCsl2DNyOBtIvM_On(?B-GYN>5Lj%X%1p*$SSiJQnzA?giM&$UD68lYHD&IQr7abnLeF)GPBHrfc(o9Y<}+sp=ud>n?N4X7ht($^N|@g z!3a@XcF{8IyscPFU}_FRr70MY?)C$sr71CVDSk-6uv$G<`#}lQlm-Gzm?+TY#6%7LlMvS+h3&gadXGWS^V{2wY4|nm~=p6<{ zdk8FJ6iw8&EDSfch!N7Y+cINlOrbUUSq5K~oCa2Ic=tC zqBR8tWF?#KpT1E#QJOM{Gl^Lf>l?zM2;Ie*NXL2zfoAhqtDETd09|MQ@k2$w_5kyc zemdNIM28V@VwTtwz>LEQ>}MWK4UV)f7;UMEEQ~ZJtQ*bh^$~Q-EBd35@a`RnCO=4x zQ!vWd>L_$5Dl3>x4Aw;GE~xTb0!7SOqfEfI0WtF!l*BCSsVT={d`8=<3D-6UqbA9t z8w*%|0!DW7w#3qCi%AkRIZtab((A2tI_X^BB)*!j{G;Y2A{ZKW|(HA0V|K{Qk0EEHO!$MrY| zwUJPZFmdOg8~>B}0U5b$fC<`vO*xOiIA&wn$faA5auPJ<0&@-2`=%P~N>eUE1!L3@ z{ojJJKD=YPb2d7VH3x&c2n%;u+ptv!6XrbFqAe<$1lXgk!ZX;l4lBFqk(350rF1RVZ zOIPlurqo4EX}u|w`6I0;L`hTbLp~GY$H&5C2@4(|FpEcQe1bFMHM4I_ix^3%9({U*xV4_>yw1M{KMX>6dk^SY#IIW zA|Wn#ky*ldoBNEkRL1RsW128iG7Y<-R1`Ev!pmdFX)azp; zOv@3JOUvUlGYnEVTHDeP<_9Svv4zxI%cz2-=sv{44ME0zn({RAlVPDWUQMm4aK^|A za`A@O30%ggTqd*YS*4~(O<6gL9X&0Uz}{PrZDB(&HHBqO4e7Uv>p6Q@R_wFSoH^yZ z9o83CfRqu6qO86l(#-Ov(KQ~wak)?B+?nO&%#(YA9*Q#*M~cyyf-nh*>Dq{X={exJ z^0}roZDFU+O3}x#_%VDr6;4*uSnHwN-p0j%Y08}Je7%&JR=I#L za<=wj?kCCzy7D2-btexVZ0+pltG@K8sHS|3F0Z6?q$c3!Oh{*B=yf_S>{Td>p+%*X zP@mFe+aH4@wM-X{EI|XKDW4Ni=nCK$1Z;lV##i*wfuE+0D%!v|1k_T*j2^Th8qy+t zN1)T(7UN^yjHH%4M*Tr&oQ!V3r`NLkHBpPKr=*J2jAZ3=-y{$I3@_o8A! z*5}@4UYp?S;*2?I8N@o8EgvJopSqh>!eY^BC?K%z=fa;&bSHu2&)*v5c3UBV6+c5$ zVx8Vr^z+b-qtBw7=jWkYMr(|2KDkONb|98lRAND;@M7tiHlZ_YBMEfHkRwnA*_RAo95#SQBhkffLXQWas|e}dzF7KYi;N2&)hrL8**-8c8)hx&dl+KoQF`E z>kYGV9l}02-Y{FU?TJ1X^Tsq=8Ajje+?Azv7E{VmvrR!4fEpyt7CwG~50J0+zDPf^xHj zS23?kvsDuqlj%hpZu5NpystctrP;y+;=2&Urgf>NDHs(!sjQkgpPy$iVjpZ|5rK=P z-pYtL(;~`jtW7Z*jLBPf>Ynn9n>cIzKs$WC#TuJtTY^ky5x8r%IwURm>|wxxJGhMoz_~$!oSKt=X0#ZPPYQG}{WEl?r+_+e-A>hT0EF zVVCRSEN6(N{8Zbf6?fYz0tXGX_3`8Jv&DE@3&sg4`mkg8nynSRsv$N%eu{Cm5m+_E z{>M)>EP99xvr?Gpa=m6-OCUJRM#vDYXAPi(KzNv~kex%rJGueYBIUF+4A|2y0wegQ zNUW)@h3-wY@ZAy3)ESqcmitAT?Fa&Uuv_`U^;CW&DQodsvmGhdu7{oEomjlqY)4^t zj0L(-oGmYBZOGxVtmMQP3v)*?JT-!&3RBB z6tSH{pB@~_OlPca1%_B0FJNm}d@0<)m%{l4Es%lIg=q5GjIjr`&@aCjIXzGsiUt#j z(m-XPvOH}MM6+$B2Vc_kJFUt(gLhxb)*tDFiuW9a_|}zXyBz&M){th~M$a2?mOwR8 z9hOHCW@)yokiMy+a6=@ND!XQg+h%IE>xQ}w%l<&m0`W5A*&tCm7qQ+*$HR+_r`n9u z6}FoRjFRi+!3J7EmLcX_Q@4u3nWoYm^b8O$MLn4s#_g%T3jLWXyORdEysSFXDDtQ6 zt=aBQAw!up&Ux7Wm_kM$yIpAX9n0=Zl~tpHHg21SUikwkC$elsV_25V(QFT;MvPwR z3e#;^_(-ZSfgO*gN)hK{=JsO21}W~-kQh&-CILy6UX0^@D%G9ZXgny9VvEnDCcP%! z#2h?Uw2x-{bBb6EVS|cDtmnkSw{ef(I3_!@z+q9huREllSmTH$Z0Q6fVuH0A?m@$Ur2r{9}SJ*@H%6XBzA zxp_0Qq5{kQnJNpVmEnZ_D>a*;riMg3)gufo{%zurcwADq5~t+d)NQmR9E_O?-=~FX zUWU}Os2x7Upp2U$y!e>DsmXASW|eEUPfyYC2hwc^1Y{`NTeLM%*eXMT@-zmS^rU?9_Q>8vK^}(*l0_?0#vvBrJM#YBAje&f3A?Hu=XdUQ+{) zLl}w7VzC3KsiV_2dUkEoL>+@7Dwgk4-YhE_N5Eey9a}Nye7CL2@J%#z0zHA7UP;g& zs~J2ioR?G`v8`utt*D@>D7W)c)wr)HJRou6!LmcbMbgxX1j0E zpU0+Kc;Y>5a$UhB@@gr8svjltnp%$9JN*<(dRVVcMYAweSg)zm2<&Su!i~*4Bf*+F zgFpk{-*5g22qLAgMiLv= z>RbYI4faY&P6Ovyqa;nNm`HOrn=fyt%x8T&Eg9^fEZGTEg_ntZM#d z*3|2V2*h^v8J`1&I3jkv8qw9o^f@XwTTSdwCX(Is>CBCE|67gfY8`#gm2;rSc-InT z#<;H5lkfObx+ub3t$mx9BsZfBH(>br!!lgzAmwVKPUfgh1jbptOHRIQOH-E-*t68% z+V7vpg2ah|A(!1Wbp=tnf4=py?8C!9@7h_$s@c!BUN-!=e)6@mtW~ogyHRk!orSfoVhg zS5;Qt6?zso(BmM+(u{CcOx@?3qHM^fQbrG}QcnJqLlrZbuX-_pT6Xvh?R2DOdCQXU ziZXE%g7kG${?sMie+#6XtMaPKavW$I5+l9G*~CCoP`L|q#<0qU1kuzE8qB9wtc+DY z{_vB!0oAp5sp)X)T2s3SoSvqI(M?|#sbN`W9gI?lIjL1u76`LiseO%oOgAyj#xl}% zLIp#UDSeoiCVlipmTf)sIVi2$c#E|SZ7Tkvr->mZ!ZYa`ep^>#qV#Uy&kHx3-&u}U z>ktA>yWn~Lphdz7+>GYyhtvbzqo0YU?l1y#cIg}{PSD}#RAVxD+v+M&Lk0Cko0}F}6tQQ3_W{M1~6;tmZ_FCermPpVcZ} zjxC&lUcI$~9hw!>l&hwmm9}-B5uT#oT|{5&nR@F#s(^EIlD+yR=BV?00uvkAspOjJ z@^G}Nk`a7fhzeiU%d8~z#l-~nlXcjyal)C`PE#+kJGD#FmBCg5BTCH$sim(=G2Tfl z;+R;Iw86CIYL{eav&#q^nyN}?B5oFZ>2fvI*R(V=MZ;yhE`51rBpzcr0*iu;VOAPQ zd1%q0sQVhGKfH;VQ5b=7^b!n5lwnBdue!rE3YCW2spA&1R1wEr+XE*+SH$XARn$i__bW znu#Gsz{$#JB|uHRp1>Bfwh}K2rDLl#AVcqp`vPW%NSO8v8y8CmEt#Qo(v=U_A>w>F z&7UqtnCdpA;0*+7Gdj5|E-2HU80jt<-UH3i_q>G~-f~l~nI6{E?I}$z&7Dz&diGf5 zHUjYsUp`ka-oTgO&b+4Hfi@<4+tc6^tZ^5CUmIN&o17^&F+0rWY;3x!w6@fgpK-VB z1bZrB$Xb+DnU=qYz*QM-ND<4zn1)dJ%6XVgWPbfaXwH&ZPL-7y?`(s zJo1d8fP{6R2=6Dbz+4UgjB06$Ob$5JK#Vt{sSgr3Cap6jC1BQlhNiVpK%#*((xmcC z#NnnWT@7KLP(+&gFo802#mv44vmzXlM+sc=|5e2!NJj_txJmU0PNuOl@0)oMS<*SQ zXu4^#(}w(H$}g3@5xU}O0&6o$@aEJ3h3Qr*jW(63mIS|{H$_V`#v~g16M=a{C?-Dd z!!9sr>a%o_JU0c!@fs@Xc>Z=468xFV0ALR?*X?|#WR%&%zs3m$qw~Pk*st4C_S3SJR?5fkJ%XZCG zr}p|y=^NK)`NnS{XQvfeJCsAZ5o+6{!9Mon;2=8EknyP9x3A^O3|A)Z& zX|d?g-bIQ7Qm(boABJb0GpckiT%a>K;qMU`ZQ+CqMe_lH`gBh`)0~>=88Fo`6V6OX z+U7?D4l(;2(y+=t4reQWq_eK6pAcxz#by7Ypu@ta7sUIFz}SpMkaVZ|zXXy$v(1qf z=gX8!YUv7V>etxYVU5nWDWkPZUh#Jrq7Ua43xi_^0!QwWZ#?OzbdK!`ffKB~O%pHd z1jGJR&YfMQpEvkH=Pr=Z*=+6=^J&i`Fw2TP4cE&$-qY+Ffr^}?KD&cJ!16AuxQ^YS zvjZCz{*&EBV7DLepESFhz$rh%ahdefdoI{1FM-?sf9tu}iwR8g=i<6(_K^hk-#G_n zXxBxvk4Bcu-GQO|3?cT0-6VT)dRvmfZ+@~jzU=Z_VKn~pC7v# zY4^(pYcm_1NZ^a?#mJp;ZgN?K-09`Vp%xy0V8wAK796vk9L+w7kS4Cxc5bgjGtn$q9(ZD#wP1m+Ioa16hNuGz~7T)V4FQ;A(% zo62oYGFGSTQwZ5V(_b+5UaUCCti5-6(I{c1(=&TLtY@*a>qWcZD(nK=1XyTjY4oQP zm_Mx1{~?~LNFmsjRTIrVlfWH6&Uhz&g!RsSs?+VaX5WjzrB>!UW^KY_Ij%qqX~45B zcb4H!v)is-ex5C4yo*?5B*Yn3U0jP-$-CC{^d&picBkr#45Pld7WJLI=9d;bdQDIM zIM<{VtuYnZd-+KQdoMrAU|%Dy*YIolEHHgWH`2YPN3(|sxm_IVpu9z=Xu9cTN=BaP zO@&6D@%)7;4-Hnhfj+f_6sPY`%qmDU)utEF&(5LwbGmZ};|;OvBB^ns{m zb3r1^9=@Qpjl8B9{YV$idazM^pwM&!b^%M+qm1I+k490jsWINdzJ|xX=w=#09BZ~% z5vMK^i_|sMWudAIF1O6jc=cmuzUi%xCTf_UyyV{#F4OD_2~_8B9L4B7leg64!}bV0 zn#NAorJkVU@5QUAG#)OgkXbXcvu0&y?UkK1J3DJmcGlk6S^H#X&CSZHtjNyF?tNu; z?<>u{X9ogPRFKE}Qcr){mk{!y`BV}+;A47aYq|{3(hR9cmj!bLAsuNl!A_;52}$a3 zR?ze?ZMt-@yVB()B_(XnAtf`}qQV^DYyrv?8|JWOiV_o`rdVWYQ@DkF3X;~!kg5z> zVkF2AuQZ{e71E_FO}1EZny^{FWp&8ct7eE-F!NDo&0a@fN1D4JMbmgu8tc0F!ZP&| z;!2sjN-gT3Fgi5y zMt3DnU_#378T2SO^virSKJ=gq{cs6247+}Q`~d6oeLEQI7&8V{+*V}grQ@hbOkmo^2X^!up|6!!`O^!5sb*hAU?l6tEZL2Tv^sqaw7msm?jL#V&f?9acJw=9 zS!fxru7P-q7{&(t{7f`|%dfEl5Q)+cPSP(4@~50NJ9TfE z{^9R6)$D8OTN#6L3)J30ne(SO*DR+ERr&@3qq3cLQP`C#45dFFj=e!qnU-1!&p5Rv zvJC#HXNzkG6PW!&$KR|UW3l(s#qAspyP4;ys#3EL5;(|o^iE0w`}{#lL8MrpIlYWw zDUf1o|Bz3pYxYA3)c!QM2;>cRcN{!OIij_I5FV*SCsWHTyX=6us^{yDO&>_~Til72PMls&3vU|q7W z-vW;)u|OqCEU98N{PoYzE`>FQRcyDTw z{cUtRr#x!hj5#y-ZL|=%=?nrIGis>(9rm;A=j7YZM%oQ=gKj^URMIDllLPb_V|v5+ zy8QzCg@^_7Y6P2MPl_Z+|CM=_$ zs%2&Q_FK#&P0^KFXovk~2;q-?hi<>kemh2&{l?3TxCP&>Y-M!FNturXfa~pd>h`KW{2k7O?{Z_c;hkEkv18vM|*y5C{H&;B4zx&0xu zj%8N8pk17g)FV(dAU^kCyuDRfN#ESHrX$(OKA<<;CJZ+6Nr zh|+KE@eZ3minuU2urS%3>}wT=8N^pI2YZu!{0&?7moO{QR3C1T;-T69isVdX33hxH zc_Wocd5|s!Q0nrh@;FZR3T@5)2IXDpY-#p4kq1mZmS%s8^19(%Y4*2`7L82a)8GK= zjMak!6zRX|2SzZ65cj-GUyVX9O+xa%tU?1S&Hf=)Nq?q8vww_vnO^y)D5o+2((Iq( z6+==JuGQ>cQ2LnNf6e|C0e7bFui3vL;K_PHj%NQKR?aj_n*DpEi0Pd+S{|Xz$XD6W z*SKkUHu^?8(<^MWJUiZHdUK7Imrr|=@y;65+^v#oX#$aL&ldG`wzsYpA7XW`Ghfk9 z2U^#oCG`6$t6Te%b*<}r`ozO64j_3h9bTh~K-7g7XIs5b=*S_A)o;Lc)RbeedJ-wULKHe$5QCQ#8)5$()JTI3I&SXBd`bpksJ+Fk0 zOvQR#THaWUD1`I4E3fF|YqWXe3GBI3SW30L-EefJLt@&UK25MwuldV*H)!4-1PCUR z36$<63N5cRwVl4x?eSNEichZ1jcy5n_Mh+TPC?_lPFh|afu%q4vrbk6h!cqZ z1Rr_|%1=CLc}r1!nZNL)c7|})gIPvk*$^-F+c~6CU@SiV zl(&LFWSEcXRHVM76Ea@+C-7OU3gxXtM|BweNnM6jx~*8zw7eFvUSp`hR`ESKcBhut zMqdEMmM`*>^k9zZ(=xn;)Td?mbtLs^8N}4uXMDEMk?0xhYfIL4(BNwv_1z8sBstJZ zVf6Q>Q$GN92Lkks|8|%K3gm+Z0A45n2c`-*K?nM;5L_%R5~*9H9+7%Q>Jw?PNJoft zq)11JbhJoIL^?*KV?{bnq~k?8L8QBh)GyNAMLJQWdx&(BNGFSQPmz|2v`nPsBAp`A zsUi)Cbec$~i*$xaD@0l;(wQQiCDOe_IveJ&bZ?RFBhtAdohQ=yBHdS{RpR~yEWH)y zXFu2!uA-MM^s<#+w$aNZy&rjMnWiUU$JneH4`HehBF0P^r&^Ir@BEbGQuDBLnDvrZz#Vg>= zQOn_9qgvs;QS0I3(f@$|j(#7$fd&GQAbHS8JkSKUFnMAefSj?NvI#;vP;$}Bh1=jJ zEV~^u>ATwQpd5qTOiPe9>a_-i@>os>!>036fsMdjIpH|W635+ePZo}u0*A(@?2k8O zU^>Z!Ni#6r|HGIb$i#H9C8mch#+8BTk<8Kk!4lJB!%@hViQ^$l3O$|^m0z{Q@kCDj z_`nj!leyr)|5iAjvY4Y5fnzeK?9;cwpUwlPxVV9*e>o5Gyf5G2eI+%4BheI5hAt$< zFoKMN-AD;cBx6m@PBxl-&7#>S(3~L#)OOf{_x0Q0jp)?d;cWuvLDf|Cc6e8Tx`6k? z?eOt}@yZs^1Ge$o;frmc1~B`Z?eM=j>IfAJT!9g)`it!ZXc6x&QBvKULMYeA?7eZ&>LiMaDGfqWks5%rN# zbpanK*-pmBrro2q&r!F*=xHO=nYWSMaJsgE8!t%WX!ZLM>P6rG+eaqV*=U^*ud^qs z6CR{DmTq<4tiqg66Oop#S9-`}+jv}LppwBA(n z%>n%Lk@@Vmiv8}#Z;npAfxed#PJz=&P*CF_^fiDSfb+8woa8_#A_pN=TXB9?!z9uU z)5#jBBpomZuNRO`s3u*oi1eVYJs6ruKO8^?p@$p-zs3Ah$f1b&FgSx84(E^~;C^x> zqw)fzInqy1S%rH^2xpS^j`9U|Qz(k_wqi1c8Q_KS2-q=$%f zvq%pU>ER+hLZnBsbUQgJHNpmrsX#dwoXQ2@Q!a$D%0;k;axqLrQ#DJu6y__xhmdj^ zEKx3pgmML}RJK8zawTj~u7ZB$YB)@}298p$g%gzP;0)z@xBzXL;Yq?)K`keoMUE!F zr8FkL8=m^dWK#bDOX?ptJeK1#vAk@F<;3B!oRo>>V@oWj43Fj13@kRy63gksV>u%e zOSvVMvxdhqDHF?lODt#SB4svfNj)bg9M+P0ZcaEhTH-h_CmhFF;y6Di92Z#QAQxof zu+7FC6)P>ZEg0usxfS*udkxTk`{1Abd;AA9_Gj)fOLdX1&bt9D@6M!N+K z>~_#>x52))Kf(gry%4lL3`=c~z-rs0&~JMTj_9n@<{gV{h z{zd9-ACU&z$E3;jIay}=f~>NAMYgI6xm2~0%hi0cO)VhTsZR2xI)Z$yjwJt6$B^&U z@k*Y$o8njZP$sIAl*wwDQmRf-rmF#Eky@cFRx6b{b+!^$o0+&w;esHS9+aizLUd*c zgDyfy0Y!m`lCma=)IIhVF zN6-?-wK?Hvu*7j)PB_{vaa^Ah4r{0254rG%{a8yJH{{fh^DS}Qm=lg`EOFeF6OOwq zaon5}j%O@!+%mlDd25F2X@A4g_1rN$mfJG1Si9tR43Fi`3@mwyC6>E$;gY-&mQ=YX zH-|j0)Uqdk%w>M^=34gT-kfkOvBYs-PB>Os;@vDk(M|f%xQMdu*C6D zPB<>N#9_64Ff@)GmN*{C3CClWI3CRjhxM}Aup1az^J2YhmeU4Cp7pZXup1beIP$HR z&4%5;$ilIQC61?Z!ZF7Z$J06Ch*;uyCMO&#EphxQCmdatIR2ayj$c{gcs3^-r&!{6 zE+-sYEpa@b6ONlLalDWdj)yIA{3RzGFI(bxF((}FS>kw!M253;FK1xU@+>XgUoEht zZ>Ex0GBJ&@+Ds*X8?HYd_;n_h>6ZO@!vf1RfyL(DKNE@Bb+#Eu{yr>{%#9f>VTt6; z+?;s*Vz0mP%zf5Z3Vk}t%-lw0gr+w!BOxO6cs!TBMP2@ zQ3ZcC?M9Wz#YHF~pR>h9LjG&f@p+=-ZfNML9^8F3V z0*^p}Pr-)r1Gg#mdFnLddK$g9r#4RK<(1_5lmfbCNdcz>3Mnw9MBSkj&C4&zE6JAy zUI`Sl8oUqZgHE$X@|)~Cl+p9F6769qrnzJJ9VME)bG!t0lfdp0S4rM>a@1|`20aqu zK~@@rgXUz~2;-4V3Gg_U!5GI180%OCQymAvY)1>s#dN--6&5;HL&VVrQB3O{?a<&z zLKCJd9BW~iWiPS&lqrm82m~ZBT>=#nm??q1C@?}* zw?YxUoI@|Sfp>&z--=U)Is359YQ{g4ry_4er%a`@hg?Bt4=c>$sOZETv@F0YQl_e# z5=gJOuesthUXgNJY|6CsiVMsYX~zD>?Azt+OW`;jCgH3f3%cVta5;|0**^|OI8K03 zjuVj!PJjuHQ=rswDpWX5LoPTSsvT#*BF9;<)NwX6I?jOu9Oprs<3gPIi=oG{1vWai z;_P3Fv;TWI(Qzf5;kX*kbzFlSa4mAc^~eE#Kn}PC?s9C0`y4yqQO9lYl;aNgi{nms z-SJ2Gm*YMYM|@^*#Aiyi;E2zZkl={Vlp4kn6biE=Lpr>}_`r_bJd*K29`eCRrItaD zB2>;GMOmmU%H#+&j!fxRSft_03!l4fU;mS>VQJCn4$Atf!pBwy&s zc|uR-TT5E1;=bmJ*^-v3xWHVoL}fM0Fr_Ae>2`DF3)3oN#>Lr6S%@WxK?23{5fsZu zQ5+vbF?<}w@Cg*dCs7QaLNR5$KT<3$D1g2|A3bn%iDDs7KH2Ky!MzK4`DuqtJQtX_llsNZL{LV?rWapmBbZ4nD z#~Dx-IHxON=M1IJS)nvIXDTb5vy?XH-b&KBkFpNaPUk$PQhLEjh1=Mg-7HlKLPDh= zBvcAQTbYre5LGUA(3LK3WXRJ(x#W{CgmTGO{6e|pD|HB6!b}iRE>(V?sb*~VqXNrJlafZvMT5E@L1$7Lh%0gh91g?_6H736T z8>Ly-abeOa{0CX@Mh;}?CQa%l&8(X=shc#Zn>49YG^v|3shc#Zo6L1KN#IZk{4%S~ zuO#qm3H(L^M@!&$5;#@@$4lTu37jl}n=8?~~V@ACkA7pO6onpOR0VpQA4LucA7?LS4W&x)0@2WXSUv=`I8hM|&k9MdcuQB&A58~trMYCfN@10R)ss5kh6A zJf=LJDcQ<$Y^exmYw-^WuTObmUcO*1jq%dxc?I-DH^v1!$PCH{d-;^7*zYs^6!$y_ zGbn$ijPwY~e8IzTD7~Y^Ar65nFUX3N)2hs6l+{EL-WE!~4Woy>D0lM;+h7u{vs-$d z7t`vbm%S{reab8IbY$c3c`$}m(Pfp_L>1kq{B2$#E6W!>b5P7W{V5ccTuc&N+m*kk zIjZc$x_%tE_2a>>p9qunGhv#3Hq6$~g(m$1SfyVGYxRraSNc{sO1~7&(=UUI^~>Q7 z{R+5SzY6ZxuZGw4>)~_#2KZXPkr4f6GFsn3#_M;GDxGg9pUY>@cy4I1NjCyxx~ay{|=4AVv6@RLt}TK+zur zhmJZ-e+0(rkKqJ74*O!dKc*r5NvPAGf@PSl)cL-;pLf#m;8!ss5yZNZ5rm$E`qmiK zGs6$+f2l!zzjK4Kk>Lh4k3Ko5zkq}K3l8cf9MsD&TK_8!>{T4tYcNB99S8R+4(@dv z+#5K!zvJNEz`^~)dT=(I9Gp$%gR|LhV(G(ZLEK!JuPGNWIuFyTLOU!h%!8$vHWg~HqR;_tg*t31biql59usL!F-UU?^A#y+PBP}a zcqbX!HX=3Wqjqx6OMZ}{3rB%cI2s&!3m#`J>jYGOrUEsg)KOqU*LFJ zaXhPGR$)6%*cylxt~2$y-sp3^%`f^KV~Upc`8nlz!K>$CZTdjc=k$>*pKbS;&$h=r zb?U?LPum>(Hn?=^2>VQ*Z8DpnJ=2)2l$s~i{74lJqKDTnXZ6RX)t_oqTFfg=v#2z~ zs5F^Zs$`YY*?g9K{L4;8*r(oe$@i-Q+ikP+E`v&1Z5po@z~~Wp#h5AE=xl7-4J1X_ z82FH|U7!|rgI3rJg@uQMr|<|ERd^&!D*O#f!BNmycr zRt#6MC%x=IZ1ZgMvsQMG6HlQ8?}Sf~UH08!TM#ur^=6aLx20tUMJ81oQ!|~2F|?zF z*W=*-0Qs0Y3vU2_;f*l6@MhBjd(e2?V-7RF1UneA=-?u;s_*vMYOLF1E<@oH*xr*S z{BrqOMPo1V*A9<=;c)ohM*RQz0sIk*K=d}YVg^d_PZVp~7E2&1ftUp9Wz1-6@Yxzu z!KAG@RgelMZObKaKxPnVJ5T}#N#GY*byiEDT>@*e>a3H%dI`t~)+QrZn^9Rtur?XN z+GGT4+mwY_MzA&+!P;ctYLkJhO$M$u8MxYH;A%4h*P%jC+wn34wVjw2g4$$YYdci} zr(1=fwlgzAP}|wrA*fu6@h*MY?5T19l%L1cuhm5g$&CSx$2 z;A$tOt|X~&ts!$=Ye}_h9a-Y)AdRl|WR+_JS>x&?J+3aY$<?Z6vq4Hj}$uhmwa}hmogUzalTXjwElmjwbK8eoNkW9Yg->I+lFrI#J1Y zooos}i;eKJ*a$z1jqtNr1`7!B@YAiRMzHXi?OYZt%p`Bx&SSyCeDbmgKQA-F&%0rn z?R*w~zJuZCB`jD_lnsV0UMJLKfLqlIq*b+DgufixVnCO0?U>Ra^nq${odKg=XF{3l z43icK80tJ=`#ra-N>W+FTUA!$SXAVWjY1Q|rt#)46U}2IY0FF_r}=DGu;o;r?MlAd zEj}OQlw9d z^iLvvR;161^e-ZPNu+-j>8m1rU8HY_^i7ezCDOM=`i@Bd!%|MV_l%kQKt9fG`zS?9 zto;c~(;j@h8m8dOih`mwp7KJ|Jt^(<{eU37KT&=tZFBn0u!XC$`57SRYA85gN zbt{aWYTIu6uY2+qDCAI!*i9d=wpVR~@l(ec#p6VAo;pW8TKRt5h2Q^kglUg>I$axf zy#zMbUy+orLZRz**xmI8?BRL~Cc9AZT>pXnUGE_|-+{%hk4*0@s58W*j=iaXKsiie zehu|{!nUuOh}mGV?Hg1WXnbwoW}9MVJ?1#X%2AF5|B5k`4W&8NR@^|EtEi!1UwS(NnnHo zMrHIyEy?W7Sd+ObpbJrH=E`V(yaaZWKx%_K(}3|u$BL@JQ?vjk6zvDoih{6LQ8mmj z3PE*I7?u>(LSxZFIIw6DB#R=@RkRp37A=7zilT6AQ5~FC6od1N;_&;TdbqY|DcoAr z0QVFn;NGG}cmUH!ikjiEqGj+Trhh700WTLF0B;tpg#Q#B2%i?Mf^UirB5Khuh+fo6 zJVk9}Oi??TShSW*Em}wRD(WCrMH?7Dt>GMCwfdy)Ay=Q&Nyh3EHz*!5R~VEDBp?jR zG%{WolqK-JFeo}KV8){Wh&tKg)WcsehiWZ#_=Av!TQgkJ(E4m5}F1iLb7hMZS7F`F& z7u^7-7u|^bauf2)t;jF8A-~*?{Bj5K$(_g_cOie=jr?&ByjygyX{-B7LyvrE9DexH zIQ;OXa3tLDjyU|_hnLuF+fm;EQWu>^H!tbLYh1q+Y=coOW4pRUYi_aK_VR?g{* zY4CS7`)wbl-WrK|g#CQjFCG87)ROq*^pf?dL3A36^OThwneo|gaGId#TPQ60AB=Gm znB=y>RJRH<-FBGm)?hz(0Yuz7#N36j%B;V~Px_cto!##=Y=`JNz?lKa>bdh^1W3iLz z>JVu%R7;bgTAB>i(qyQXCbgTLrl3#{vWXoE$s)Cvsg^NhzIrfYv8mut`xuMuM@9;0 zCfL+|req>S7f_7kvnbR7%iue8faKBb6Lru+1?3@3P3nb|msCb`RnmN;Aaq)`1^Zam zG4oSag%pEjWN(t$Y;M)exn(OFOYQ%_{SC$ryE+3)G>_e?>rJfT)I=+z7(kY4`;m-}t- zVp{C}7s|uGVV3(pu)zH;)VSY+#qRf^-u(eIyFY{j-5Q#jiF865Ba z98Pur7tVHn0T<%Em$<)TbY8?MW6=3}gU;6*biQ8DxfsqDDs&PYFI4CyVxuJwX@7P$ z?Xebj&)&#%&s4L@w3}3>%3U(%pygAQsR~8O^C)C^YUt?r)N`0+Lf||JTp)pqI3T-8z zJs|?%X#kHW0TVpSpvxr^GlNFIh+)Eenm!mjv%{xeodx$jwG`@zag_dN0EIn z-Pdz8S>X9C31S-d97AHBW63hlab$((cyb`7zwn$uIy@(m4W5%oH>Q1_lgVM8Q^?Vt zQ_1fu=EcJ{8Iw|lECjJ@NWsc zD}nbV@Sy}gmcXYHcuWFMNZ@k`d?A6aB=C&{{wIO&C2+R@>@otsCs1PP*>ZPWYQw93VLTk2F_AuFGxcS~s}h z)~?Y0%^lr=UV86cPqwesPs+C)PmuYpUJ{(!_@UE(FzDguXt(l4S+8adm?fTWzM>?a zvG~SmU|pjL>pO>IK)3fs`L7@duQ_5Vp0WhhJBi|u&-=P0-9M+mpaQU>roeKM(Auuqt9=ilB5ne zvY*X_xEI37W0PdcI3osSA{T1KMqFh{vlyc20t zb4{qm+9<8xMX$PJ2Scsjm9PwIG-64tDUeEyuFiU6x_g7v3z!8=<{`bhr3@J6funkM z*Ke;y)9@nHi?$_ldXvc8Wd(zMjSe*QI^1W<#cla;3AOY>)bF%~Ynx?-!0GQI_4XXN z>|uBdt;OPbl*Zr7lz8w0Ech&eueIeszuruYdR5lDH8B`9jjyw%ZCGN>q(=T)h6gOU(W=ZM-F& z?;cgkeI)kjH8PGk=I5-WYWso&(y+k&(!>w-AvaJTkaO6s;} z#~{E*y;lc*u8zYN!Uoy|X7ys>QgO;80P;k{HC0snr+C<+@*cl_tashCR($ieX;s52 zU;|a%n5JbSO2c?0WK+=CKGycH(k{N-!6wL}cP9Q3CsO)8-Pr@S%?AIP)s-OZgtwSy zx3~F=kQR?N8but~y8OZ2I%ZluU!FYBG=dt9KQNJ9 z9tbRgpW63OmadpyGr${#a=0LsfWCO=7+D;;B~hIW&?WI4xxWp>lZ2l&#R6+T$~XoIe%7>Zvk6TWCJ`SNCydrJqL0lB=?UG z7~C4$uy&|(fO3(+_bLy}@1*aY-fG-1frr}7!vIY^9LaeB~x8PZokb`Mv9Yc=cnNhO1`MK$&S}Qj9t4Vc_E7QxRisAt z(t4tzS_^IkB*)&qQ|^)zQE&ftPEU*tX+21wr(fN~F+7A=nhY))A5puKDvA86I%2^k?kh~f@I$H|(2_=0osrYTP*PGsuu_CGwRCB-@O5k-lj26q^BVyN z3B&P2LoR4hMqkD=G+05XFXvw7J2Z@FltJR;hE^99jVR1TBkN&M=GW9iW}sE(C)a~< zU|Qy<*5iPoW9kLf!%IX<(~Yb_Cq_}z4X=SUMqkU>DD_z!5|+{NgmjOP!@KZ6v9;zMwkzt8BP&!1Lp^q_a zw+B+i)IESbM8aW`IaEJ{_OZ-T4zrgTXPKc~j)Ib~J}gk!Z7V`T+a(hoF+L_aj93_X zlR4MZQtk*#+1!RE@0KorW^yK9Gq1bNW>|ZbLBH}Wg(kVwzD??%fTy2P;8z8Ui#9_1 z0~-_NQYD-k`NBlgvFOhiig-vrp3`$rJ@qc@=bn zsm3J5V~Z|_iUx-Y|Nn`<$^At6#6&sR|HR*9gtC-^vJ`{tzvFL$QA=GxN!@9!ke5|a zm1TiP6?3ko_`6k&!%e-o2j7CR0$z5^dE7y^bt94UJM( z3J6w|j1ozWQq2id%B(7nj9TZ6YUcxZ45o(ZIVq)zR(@7fKl7QOjuR>~gP9POlg_#I zEa2{`SN%A4RFSmDyrnES!zoa@ajid!Q^ItcRDlkSSSp#ge8KH#FB!jP_wtU6F^sKyxYN~a(_108>6`b! zjXEL$>@1Q=GI0Ka@^1{rqWf~^! zghjRVbu^)6&ZVeHN!C$YM`N2~UI}dKyKdmz7O@@zY(U)RYv1X3K*1Jn+Y@!e;FfX| zx#q}!@3|%Mvf&(qCq_GLb1+dkTTt@ctgaaJwzRxjT8j3zxWrprhEKp6!&_=lQsq!0 z8jk$fuYMf;=&XJyJ=8RNikCZA&yHzZYF}tvuAXT?aps57w?>^}WoeD2DNUp)O{FQ# zs0r{J+KA6`w$(*JNwSq!(YM#dgJ=!)dtdJnFwhX}6CnJC1q3_uhrgrijy6Gto~{Gy zj_&3Nv>APo99@#Iv=sLR_FZK|kzmqzDLH+Tu5P<*T~6(7=L+_V#9AFr?&uJ0(BQ7? z{l;T$&=9Hk`wT={CO|Zc{o3|+=?pZA1I7|nyUsp{dmO4To;)2*3Q3W1>3;tek^82O=FAELzcn zW3jF+5S^MqMQoj#p~>jOV~H-U@YK}4Nk_Mg!N%7ve9s&x-^oVpEK^mC23<$2nxj6m z`zIYx4s9qBZE)vv4>JK)KFSVwjz#IYNE?5CC(5Y2&AcR=Q2HhylKEIY$Zf#Jb5c1O z-r!{?Ql&ZVy40HxLnk<`0#EpJ9;wpL_(k9}S>90s4>_)OIDJ+Rw9k1$wji8g_$RcK z1w^)#oT2e2$t(%keJl@*$#Zlq>DhhQ4`iI8(|f9JzukpeWC2==^(R~jYl@Mbr(6k3 z5A}12UgeKD$_c!W?1#VqbkK!sDG<+klUC8l7SSmj)y35Nm5eb7Cv-~4N7HM(rUxdJ zQ&1+1=H*~ga4h;51gOO@egvP0nVcLU&A;1tis< z=SWltn-Z6yu??C9Ox3UFUaMj-jGdK79+tgax;K)R6*U{dEL>OQu=^mFKGpI+{#X|A z3}aLxs>O2zGb@|c|ISrZrFUd(=h>J}FLzf3dqf~trmjm18B>eHQsOtOQ7ZIp#38Jt zI2m7y;l4031T~YwHNYOxuf#{0W{+J~F}i0u<3p*bHe_uJur%5j(O$~PG;EF$uLvrx zU2BkBmMa@}Hv_qZx*%Q_IvcoJbgSuJ+pNf9j(k|cH3DuFUeR2Wv&d!kfvtC!?8wv( zUox8`cIzb8ZRsSk%xAXFkyz+^lFP1+Z2TiFkiI*iZVJs0VYCG-6Hbr{-MOH_Fw$Uu zU#QCRx`zUDF@mN99l!}@QlXHJH~CO3O4rC>6s{1iP4=d<&zseeV3}K8NTlnkoFXS% zWXM{Db)kh;r^SR$d+8l05lE7jZ0LeV{*g(+jtSSe;^hv^xPpcnLPt@t54o~DcBuM# z;34V#TAV#DJ1$D7OT3o(ustEsA8V9fM5Mz?m=-a2FP{Rs?7Cz{u))__O{WVVt8pN?2+`3KgdQNsyP zul*p~4d~r=5$ue*o&!9jO|Vk)_>aGt!~T97rA_;*{=J`db<|gpHpc)L$u$Ud=JL%V z{8VDVzzKXZG>A1oY6lIvp4lC-?AMkY-xf^x3ykc9RX|n;`k%XG+wUtyw4Dy{?Nvq>)Qf{D#~IH;~KB& zwXg)yJy(R4*pX_e;JVs3tUulL2ZobP*>!bb7vP3aDNpYi)I)={s2;+K9~pD&c^+iekj$4%_Kvu{+* zr=gR!BhYGQzf31kS;0cJqo9q)AVGC;w;%UklXc0Zjxr$bCInBwPrM~p12*Ajvt*+0 zkEl28kTMcb-nc(eYvFwfs(%%dYlVu_uIsLBu2xVDmQ%x&bO{u^@slqi*&3Qno1US0 z+lw+~s0fM?Bi+u2hU~Gi2=TMkh9UuC$IgccbU@nPv_)snuF$o)4^?K*ZqT{83QJ9& zUb3=sUwO_C(Y?6`Ia3eb87zUjWEs9z;*5=D9R#ZhYv96;!wR;Q5s-C-x^rQld^#9r zU^@Ilt5K3?7FPwOFWE+Ci45Ky*?F-J&|f`7<^l+KlhCSCo(0&xgUtA4fsbba5H~Tt z*zF_g1wcl_RhUpR9#QiMapAb%K;|Pxpp<>#LmtBE5_+jL=5hDLzahK&NFx`!ARpf> z3!b}RE8j5%@Jjn+zDegjN(S)1QSnoJgAWZnBVly zP);7#)pV_(-aFUQX8TJFHnVy|soy@=6s$Jy+Sk%&2RGlnzSwpB0^bF`n148V22l4A z3x4QBQV)f`0eePJ51D#EOdpEbVrd75y+CUYDD4q-gE`+YwnkSEMG5!uS&Z%HpbOz+RJle`AiOpBlT-vq z0wTch7bkyVd}^BR+=yg~kV`@nw5hnoB2uH0U{0l8jM5CTNi8ZiUY4`OONG)17wU3yNmFWN*76z<4U&v@m z$_*D60_MX&@E!pPataGJ3bbpUkLv0g9g8Kf903VxYQllD;64o@nJB*uecC8zr(ln^9>7_U~b83ln9??ohZGL=|3&lkkDyuB{= zuINNJRYRJ94X7~3$TV~UOm8V#kHao8w+Z~KG4HBo!WcFdXW3^eJm5fk3h;pLYTrBZ{$gaeD=6lKOL_|)GVBrmtU`&R~IO-F9OH~rlMzz4w3wv ztTv(4>u=0*o9BOA_P6XRW@w80Jf5Gies;#t*_8_|>;)VFen-+o|3WPYh(lJjORFlW zH(k5}7xIY=+{2UUFG+dGpo&pw)%Mg~6EKo{#h^vP)OhW`3IXX4Xbxl3H6FQ?#Cu4O z)p*b83rrlBSu`bKHF)=;{QG^^M8K=*_MAuz91q@gF=F(4`FP`-d<)|ZZfZdCoYdED zOnSF743R(5ZWI|G<`K&-`fLa(B>wck@=-@20y>rwP}&t0Q|>GHyLPai3oM%k?e<`^ zDSPPt@9)u=Zm_u}QT9w8SKk688|Z`Tz@>Hx8A5X`ax>~KAJY1QsG;>-S(8NbG@MPG zrAz^)4>~?+xpHu17Nwz1unQwp^iHCT`#7enekZ-)M$rJ4rd<@ zpC-Zz`bi|%4IBwm?#r1hh^1r&ifKlDG5n#6KWpz-HDqIn!g?-(hOHby8ttCi40q06 zgVB?1LfwuZ$!%YF>;*wd4o=l#4{%AUor1x@LQn(|N|KXf&-uga=lz9X=1wh32+! z_7Ae-&o?XAz%vk)W}j-qLJ}fHalFCX-S*Teqd6^$^-ZgpE~=U(YL*>}@lC58k9tIM zaw%V%AO3N*FK{)kfIj>aaQ}vHU%+Zy8Fr9moYF`g|L0(-E!k$SwbZt0UXbrt)Z5J<1&OEM@(a6m5v8Sp8*oII41D{Wv8u%EqB?2kz+owKg2P zJ$0#HkxI_1`1x+XmjZhEmm7Rsy)RV2p?=DtDrcZEt!>i0H9^a~$P_VSUH~^s$12a1 zvi$*V#SomO05fw=GRvn)A3*b-sHI;BJY+tcEn6%fDs((RpHtE_W{%zKe>Xb$DB_{HC61X|1#ftRx0Oqoc)$5zpDZnp^J39q!YXts;WDgcn>aUP)0H za1Lm_8i-wtAruF)2$G*VZkr;Gw24o^1FxGRjz0my2l}rfRFNtu!U_;NK@Vh6_=O@U z0&#jg7vGx+L!4olnIF@(k{{M85c-HB2Oc#=9DM?NJoukhFczs?I0e-Ni)>kU?x3RM z8}P9CXv#G&m`1bP>dwHm{7tmyT-1UXbk|y*H>g_mF=iuJw~W0U{J2>;R5oBZZ&o8f zH)21?3$)R4Q#&FABR3BsjU}+v>ts5E+T?%_1F@H78tQP}a0IPeJ)tNaGg*(C20>=j50UMGa?abZ6?tC2a!d zRPpUh4d+s9XY+)EM@c9a)F$mgER`0$X^4a{zh?)gNGeW;I>294(f!G!kI(Ma@$=B~0> zvC@i5!iDEdu8x!OFfSdNwiV!S>$_F5%tucZ2x- zuL{KYK!#dLG%@5Ftr4xv4p($WtBYt#GKwzwIAL{Z*1vD8%VWM}tvr1M`xE?KL;PPE zhQ*Q^<<3=N+pkL7Dx2`cY19*PgwWQCUl}~aok?P06iDG0vEt*G8yjoIMM6tviUPec zR{Hfx0t5W*crJ^l2dt$0l+CA}YO7P1AU(|ClS~EfQOhidp`xma*v?~=iESC_#?nla z-)3;ok~PLawlW9Q8D;40C_F0z4!b!+=}}rxlgy8y=gdhr_~o}uyF_|48SbX%qG{J- zEg~TJt*{ zE>yu=QAd0UfF3gVu#}w_ggq{px##w{EXdIo>;8v*Y5Ae;4P&dsxGIvqnLz5eoZ{nu zYe~B(!t8jRF$J-;1Z=MCe}#v~`3R6s=)bi9^n2)J>HozCXH*GUl>_nUks#5W|@((kFP+?^ll2=%R4WUUsm zw+i_?jwhc<=S`VbhnSY;6Wkqe)WB%#wT+NqwmKbKE1($sk-vt4{Ucn`FeByDJT=o9 z&ozymE_Mn6cyS!m(ivc4Q-l6-6Y~7Y1peWW0V>IPw9OXE%J1_P%DznsniR!w%nC(w zwkI^5P*mohh1!#hrZa{oQ|!C!uRFLtAJ>-vIHT^EQb3M^fy}`9e03y9aIt6i8O(wC ze3njTZ0qq+m*4!DR6vbjUxeQoc2y*jczO^*27rL7o-x=NWE%X*dxOduFvy+`PGYv< zZ=aw-m3%$|v5JP)OlVNSg+?eXm4k%eX1*}8B(WkheP#-_7&n)Thd?DaBn|?qPi|K^QKc^RrQ{r}~$nT54yt&+uwE+(P7Dx1hqJc&G+`8vRD z|4PpZvSG;7AgCRUKbSh|@xhU(a-HcLlfJ5T(w4yy4?6XQIQYAxS4$5D0&?#lM8bU} znp2BZvZE|y(+yVl2fqWE$QqXh!txgMM{tRASM-B*7NMA@fk`VQW0Vl;Dkx)^^pTg3 zG9h6qMTa1;Rl6$ij%!rtvz)Bm1@&Ep9?4WtFYfs*hLAc|j(@Gk*@_1CHRbWfJ6+zp zQR!R@p{tBTaS_X1jbmq`U+UARqQK8ks6H}2YkKt?*z2mYg`=(p;4hp-<8U4htqXZgo3 z&*f>+R2f2sDs&e%gzG0qvBFvpVrE*zhNyuUC5A-3Al>e%pZXPht`5gj`EH|1uJ>PD zKw1bs*PzC!|APXZP9!g7Gj#vpZ3<@)0M8g~?4nrq*DKDiFBp4Q!&itLRSFk4Z3z(ri2F=(y zH~E3K`8uZ9+*_sFbcm`t<&TW&;OfyZl-qrbr-n6!;Uh*d=_nJ{C`5o#1q^tU+MlGc zDsD7F(wzZu%#qvaf&_7~CU$!aM3yZ0v|N|ix?)`}-4EqNd}5`Y9@^z{Lk&$wC&cwH z?C_|_nV2;@uCmr3gtySF~j7a2}@HVlk+mzs63+C3N=(($@(;SEc0;Z{LmCqn16;wOC#{i^fWxk#%njDQ@0Ls@|y0hiSyY&N~&ic_}N2YvL~s<1IyNlkK}i zDv8>>F~d7uNS!>*?X#W`IhtuC=NIB_T@K#b^NY0wRh4$b$W>kUJsrA)qwq}O z18VMQ*%%{vDXmqdM%XBM*pYR}%WfEw^MW@vT%f$fR_uQi3JMcp3c>#!-R$oIfYqZ8 zhV8K{fWr??2K7-|LF~Mr1pnZH&@zC4t4oKPy&_<9R@Lkm6K_lC8fH5v zky&87(vw?olS3q_gir%#!dzJVI~tQ~qguVf?Cv4AvLa*7{{7cGZ>po=$4RRt#imz~&Qa#+{5Tt2zNsYOINrK`nHzqBB_(v-{O|A8JmTZAGMV(PQ2eO*~TM z;`HdNA94DqqTSyT>G0Ks;i}UW0izwft$G)X`Re`-qqN5zatefUs984nIWdWFSmo4) z5uWY!K`@s|$b_ocW6jE1)UBC*oVipkb#GB#uLs$QNkX4%G>jBLB&h_-ME12fALrDR z#HCAys9aldld0vM%k%Sx=PM2G4I*W~6r_K)?9tKV6Uu6cw(+Kp+Zi0Og}$3%f#f~L z?gv|n6apN)b5IN!>XG83qegEwy5f>@x^l`H<@}L4-2y=pYgC4C*HVj+s=v|=afs&D zF0nweqOWo`UoPV9mfPVfs>y|_5hFPl{%o=;k!+f(cptuZOCUm`%2?DbqdQ1~X!OD= zvCRaFkk2TrFGMoyt9BTQuTYF>Jyj>BTgTH5_<{CL-dx zbMe@{l~g=di6p{$IAyLgt>i@9%pCBwr|9KDIT>}6moMw^M%h6ycVkY~YG2DVKIHV4 zNw%PFhhp)nK}J7!%BfzMpzKdE!4PYhgHw6C=G(HB(T_3;&mE=7)#?>Tum1K|3iC6k z#=o5OFW~BLv9F)-T(#Eu;NCn;T=+&`NPTaFc!MLG4DmlLU_+iy916m)xCz83;uP7$hFv5zRPrhK#3@J)8gX)Xiw&0%@D5cYaR~>T zr(TKBfhU=J4bNMXt&4@w?ivwb!+#00myikXKH!E|p2IyD%$<`I{1L$TC|#W6O%M6b zG(V%275q6wda^Yv1@zw9U;Lno+{7O^Ctyof2Kkbc>^9jt(%_l(yb0_9Y$6~m?~JPY zm?@p;8#7q%tV=d7bynu6JaHgVK5o=1Y2CpwPIqHSowiuRsMM@%w<_n>{4|u(G1K9FLiVxGe>fChsmTU4{HZ^>y^wXJ;6oRM~$rnjp*IQcA_dKK7ZKqEJ@J zr?stk7ALedy5fZUU~yM zDRbrNd+$lPO8INBrN&SZ{_^rx_OADhdAZhV3>jU_?mz+to#*N#MY%DAATQK{P2vYQ za7HZRT`tuEQQ`?WRtkzgbZ26zt{3Mma6OeqIX>46IaW3o4mmcg$`wZm*YFlpovZ{R zalz-02k~JnQ~%~tXqcOvq8%?HIXe}K4pz`Nu2~~Z9jleG$gz)rHwt61RTaYe@W$X> z49de~;yOu=(sPN=osavmBWe}^&STyy>h{I}Vv(jdLXeQz7>h?WY`Fsy_Lr@3mmlJv z`9L2u!3kotw7EnV@CPyO*oBZV{a>a+ur&Ooywfx2_|Ui4Ayu zNH;H5d)WxyxBnX*h1Fa2t83ab6QV}`-}F2v-}iSM&04CbNCW>Ivw z&0Ik48P3dwyANxM*BF`7L6IicsQWxu(N`0QFlnE%;z&zV-~YJe*y+n8x=#~4z`(bs zvoKWknZ^7g@$-V~gKsV)AqI-;BPP)1g~o;M_-e*UJ-sAI2Vt&|gKUy;w<~IO?xyPe zg=YJH4_ot51hzZ)$@3*N_<~Hk$XCP0#>}2R*=X6>;^c+5-C)mPYaB_$|8OO@BPO9t zXKOW+9Jjms{`a8VsR9)!~{80*b&!w>f zB|=?1=7)d&Uw6;)hZ?3<&MT|zkIyq}CsAHzOKtP$tmV%l8e3If>k@~CPa~7d!E-<@ z!A%G*AzZ{D0?x!iLKdmBIdc|X`9ZW1Tll%O!Ck~&>`O|z1BeW}yFFpRzzu6S8`=W4 zWKEtp-j8ASyk4ly`;YZBz7twV_Q={b3L%wn1nF;$9fOyw_qx;fD;@0!P^|k6!U$ z^Fn*O*&+tdL~(T!3&rl@B@e}s{tiZJbq66*Z6&q^D7xi3f8om*=G?}MTjFDKL>Wi{ zcKO+tC@3ZbLwKwVj<};Jxa$PuxbWGYc7>Fh{P4~RUUmgIs{v>#QhN^Z5Y8R_kv@Z$ zA7`{7F(RdXJ)wVIVsIdgK#~249gF_k${6;L&K+Y)A9PAKtbfNU30)dLBy5421iMz> zK{kER6}Uh}6L+92(IK|rnMCB~p3EdU8n!z#H{q`n4Y!=3|HY?{SN zoiZ2nkU!jW{4rNmRcAW>czlLqtRA6?ZL~hqS(d7gL#s>Z$U9tYiuDcpMmy-+-PGl0 zD|X{{a{_s(t}z2Y=s3}VZ5?W;J=l6*VaPb3*`O*-oR3+PfYHpR?b5!{{e>}ZuFG?z zWg~aoQ2PO_@zUz8ve~ONaGk@lV)w~lHchdgTK??(@dC!VnmMszq8Vv8 zIqTyZoOHB{PBa;N7Vb9nIu*XhaO%KQHms&*y9+3)(G}^ekA7-@4ZEXXuO({j`^mKf zvwJj5(K1}^1#T_cQjoGsnbj@qXv-M6+V)e!Q6K%>3&l(6NOi`hGW?vYu;4Z_iO8!O zbIBqZMC-i7WTcA`B`6x97^m$8Q{~naWL;a{< zdO2uI`DQl;QQZHMZ?oLjBK&1TOy^rv&FO{nqZB{zRX7M(B72-DWZoH(QF5oH_;14C zjnR6yxTQDA8MOB@AGfuwr2OpW^*Qa(vNBq+dmDrYp>}mf`R>EkXmBU)v@sBmHbYTVLO_Ftqfq z0qDnAV;4fB*&1e6)m5vpjEuOUtL3!JN|o;)~w%eiZ{{|RG7uA zZ$g)$Q<{qFPYfTaZjz&9oJV_y6cgFhKpLxzf@hSx*AK~%V$wf~Hd|;xISCeQW|=BM zGUv}_O*ly-cHWGt{ctQs-D=jjt0}NTxzhh9Z(gdBUQ*Q+kCU4{o$l1Maj+U7e9N5^ zok=M0YFto`6<;h(V2En;qV6cKM8rWUd4RZy8E#KO7WgSfkvRb;SI%h?5<)QSk=gDR zN8uf1v>QBQ4rV$gk;cPT7j4Mc!pO$KQQEENmNOVA;qPMShJ3MiM~<{treH=2F51l3 z$!3vc%_>?g@Jn`wygpPbo%4y|2!?g2Qaz&+?G_Ma!$88=GgCSPje&uLvA0q>LlZ@0 zfnb`NubLwhjT9hd)3W>g+n)wbihd!f>}QBoTBe?0;xiF9Ef5O#X@{YRBgJ4v2{tbs zR7IuClH1VGuRLZCrX<9wteczyWD-mP?U?->ZYOq;$?(I-UVAz#7!GD z(0B*gHmoU6hmv5AsuML~XFpJ`Smso$*^|3cIgqum)e_QwGEp}_ihOR&a5Na zCOYC9O}$VWP*xlW#tkIJ#KtRJG`WkHct18rk{(rHPO??r`qY}xpz3QzxK(M-xerXC z+Alka|1NAJAgQ~r**lm&!$H<3mDf)WDIj^~>26N0;alA6q{o}J#G&lu@4Q5jS;pO> zDA~05d|NWso}&!zM>Sd%YEuj(pd>`WGCyd;D7}F}zNXB%0NS1E z%vzK@Vj-wk@|rwlv}uu|R@SHWJi{TOQvMwkR{E@hi>dIEY+ZfHPGZLK;pmn}-D_E8JiqOPy5pJ-7VEtv>Op$=bZ z!n?g7ZjM}KU0$iJQg3$hHr(FO@Lz&(Kyh_(DmBDDU#@S-2?`AjS~Dl9wKA)+tkBf1 z6g(zR`=*jZB%v7PG3Z z5V6Kb8+leyhlQtE8A6sQ^A2~Tw zB`;aGE=t(SxIjgu=5D5&|HZZ5j4KeRs!3w0Z{3VvU#zCSG+#$Tjlx!UsoB4y^4G1L z`0bg+OWC=ku51{y2;718p~Xkw)$n!WPRc@2n>Iuvo8=wSg(;lhoEUjLvQu@d*g6zP zSI&vaUf-CXyGUbGV2`5$s|_a>1_UU5t|((;Ls*xqS$u8?RWxeaIyXl%t4YGlZ?m2& zau*vrd+cFgt{!+}nQdc9YG=4awW`Hh)xirwAtzjgk+v{fIfYCWw3$SCU~78BjG;Lb z!r3edteg*T^-L05aVu=XyJo5{u}Js%En9TWjIbeDRBVf&1;+^2O2k!DA?9n~T(7QP z+LbRTwMH{}Jh_>As!}DJT_6QXiniFRXS)Oxy}(U_Ue9gB>s;PB!g4OLQ=#$Q6Fo6) zY8w_gilnmjoH!jh6>~JdU~Vg`n6Gza;PHfQezhgvGdO{TB4#2$7s2hjgpjr0%A^q+ ziU#EItE5Sk-n5WGjBv<9vmsGdwq)v4{5FqD_$)ZfSLK zu|^dVkyG0Kqa7ryVOAJVF_%yyZgnL^AN56jF3DPgh*6MbWfYRpBn@mb7Keifu0d@X z{pg_pap(&%KpyHH-Y-{}n3jWu$N&>GUTmGsI=89qyQde2;juGfu>^LwjpcQ? zvMSwflMkKkbW`K~F3|q@0XhCD8ND2Sd>ZJz2=+xpiUzqIJkhSk1!j>Rc^FH*I+HP* zBB!a=^qMgSyr(lUkvf&Qzewi6j(7f%1v~tpSHFc{a@X&)(J9PiZ;4vS+PoLiB*<2? zS_s3TUXD+aFsIIR!pKWmwUrrS;$XI5TIRV}=G&^)r%XSY6m{Au0?xnF>SFuOUID}x z4-xEQ>}D$GAF+rf=8Sq4p~=dLd$eNLB|!+#^i(Jp673*g&nc5_X+9qakaK8%i3$Z~ zNU!HMeW_w^ES=5Zj`U_&9+`Y=0n1Q;6IrF2IM9L5TyM#`+TqReN9HD*o^a$f#@Dpk!=rrr~A%01XmOwYY_y&}yHft*V0*S=&sBbZS1kUC_4X zATcbLMk>?;*)K1f95)k%yFAu2G?lGju4gK*EZEW?UUce?I!{7W*E_8Pdm0!`$)yek z*i;|f;BRRRZFZhcW31wixSk)}Xtz3ru{?bg&0={%d&PFxHQZ)}0(^w+yuB~s6^hvF zH-*qdan1636z>-3`FD~bZ=KyY#60h#C3g+u`AJv3L9qknJB}j-=_&c;9j5X*z_W-; z&J!}B1CseTvTcXOIJhED_#-vdAI!CHaKoDI@IW!V!=7YE)7~9dsypSS80|21I!278 zJ`|5JB}nb*ZwyOnHM`o1tN=w8h-QIbEEm!dloaaw*wzu3aHPjTb4;UiV{J%J&YcJ_ zmaH+Lxwb$_Nw`8!6@BtcEjjk2qd1tSu7snE6?x3u;;}5dEtAhdF-DmFqcsDZvdBki z<_AXcD)GWl`y)r+eN|>(pVF{*I6S^59)<%1`UnN=_d0lK=kK!NF80 z^0Ut7r_c7u)A;6zr%JbMw8|~HxKsnU!iX1=jaCS36`K#wh@0(BayuQ78diNJgr^CI zIWkl&S2!AAwa%G3)*BbJJ>|R3KO-Xk8fA))(&Vul8e++i5Om#2DE*Kw9sYPDitLh2 z^3oNOW`M*n9P*|iJ!x2n1XFd+wJk6vET;_K7*N~XK`2H)vN)K-&uZK3QrL0mY;#Vv z%z!!yzT*v`B%QoMO*Y2Cn3f>EY!tnus-!fIdsv1LfqI2wPLn%Fo)1Rwk6Edq^BjKX zT1QSm(U?I-YClfLfeK`MbWKw>OGmuK8o%>)(DRzOK2M)|D@JhE?Vm(>F}PPsCjE8& zr9AB@^mSw(N>fcQS+uA!4{5iL`R0PI0=V+-JB?TmoM}7um9kG*f#GcE`%be0I7T%d z3z--;PGgyzy_~G!xE+aiCbG3Q|D;f+OSsg~BnSk(hC;g0ZLkPl_Nir`+X@aJ64Fkx zRhxiiQZ%hj2+UWw#r`8`(!_FFQE+0n-pWAi^vBk=Q`Fqe#mHH%SN@il_zhXnZBc40 zEEgO5lP!adQTvhmwsr2*%*OijnY(R+VTE1G6lJ*4^AoaV}MRF?5AA$u>?MQfb;3a{=n z$oH2y>h`*!&h?1P(U5}BIO{MSHR1Ga5BlNk5#QU80zs36*C+9xgRz7-XS=)x+_8Kk8mwrBfVTa zEU$FBJ0{>yb*)i~dm66L*gBjqv|ZfM)wU0$oFS+;gdYA2;ghG*dyHJcxPKj%nOrk% zgGH4=e}dm_k;sy zvE`|`2--d(PVRxs|1!-6XTt@1^@@6;H}5DV{(&7{Vi!&a$1I^+?Q~R_ zhkjNwa5R|(f)(o1j;eR&CLgq-83B}doOC`MpYEe4e0E!7x=h~=mu&^hQ1)2*%%{`8 z(`fpVpnL@*bdRR}Y}RS9E*~q^|KP!%-$s52oD=Y|emXzUL_RULq}jxtjS`ZKTUhjt zgM%FrwkxX3L-)TZRUn|{{S7W6p}UsJM<}UR7TGV5>PDtYOx7QEfgP_KfSO3Bh$|(F za9K*_R#veQ1NUCLB)1>{N#c#gD1S-Loq`Fjdecuzd$s495btr2T^s#L9KC0wuhLrO z&8cHGV%I+D&Y!5TYbisskBnGsDPtQ&*`V>HCowvX!56r&Z%FB^A3q;kz|J8FRN-Nm zp%y?<4fPhZX;YS@@^K^bqZ%I|gCLM?fKR|edMKb5cc9mPu&X~Z>1#+nkO1= z(;(5y&I%2Z8hU1c;51^ei8p`C6H@aQweOW09aIP(H6o$CYrG>)4>UEV(J{(X$@~z| zygY{H>y@%&H>!GIh*=a)&hHVEC=EirPXhF(0ty5M3f6KBNQzw^_8N#P$tj-hwIVB~ zZL5*gKIw2^QLg8@7~fuJs*=GhehiulBz`Op03G~d+BB9n4XL$L%;x-mbbVutC{d8^ z%-peU+qP}%j&0A}v2EM7ZQHhO+wEzUplkTccoe$a5i$-(~S(J8ts=)64oK3NX9T!QWtya zBJ?K^kw+ERjW8q&VNWD@ipL<8CI1|cCsht$)k&+uRPV-%#BlNS|3 zQt*?AQ2aj9Ihq0TquSJiz7U`B`Vp#ysY}>SdMU9$y^G1o8*v&YO}g6e944s^TYEinwNVwSWo#A$DW->5YxVLe>_EFV3j7#f*TR=qW@^Zkz6ZeokQYap{Q8mVQ zWOIBufU%s`bYyYNDjwOr?4LWxpn-@^l%YVaNiR}8fPL0mG#}>PhD${?eN5t&M{QxHhkM(TtTb-;OR9BYEfzU zr%eN@fMrKju4+f}Y6sE20xzn&x{}Te-WxIIR5-nWO$r z2o!TOf++1ttP-YuLfdB$YlZ7^KcjJ*SHs@6&@>hUwS^fF!m16KEL97bYBj#2wJQ5t3d8?EDli)M=PZLb_QJCU`{6B=v8#Y zfVNQAvWahlEjcy3ho?@IDuNJ5*Ji=``5+Fky^mn)loKghru^nc9p)0yv$&H?d(k zg}hvA+Rz%Dh+Gc5g&mO6vE72Wic>a|Y4R@j1O9xtiVG|{%#7YEgSaD5u0?Q9H)UFv zX7i7Bz}3m?!HNFLHzxIGA;eC6f7tx}uP!<6ffX1+v1bKpid9Or>ZAygH z(qTitJ+*}T$op#(+~_2xA>8P1md{|~z`BElDDCC4h}%NDEt#BKhvu|)zjA!WLWA*u z0Gw6LCo!Rw>{+Bwqr{5N=dr%wFak8@Zk08y)eJCuTs zGZAi!(iazH&^ajzt_*v#O7uV{DFjTJmsi5Q2Qwr@1wBVq#8ndIKwZT@Wf9m@Q)Uou z5#%)~Jc$U%N-9HB`$TYlm&uw6^G+}%xGI}&RX?rWyT`G_Dp;OHm!aAyasZ>|d}$=v(yfDn&MYDQmIrb_A01uU6kH zNGBxK(u@$VJB!$eqIGE~N}IwRY3grNbe#1TxNZj)>$Ubdv5AK?E!SOg_GBCi1k z06_bNy^#FxgGD(f$Nzy+c2b#9QZhy6!E;NE<)zFeoCD5Lu@p}%ZPHUv2oMa=qtv4= zYU(DAd8Hqd(#55mm|i_A&Y*2KD}h4IvR{Z|`m_r#xN(EVdLx$5o99WvkTP?O=(ggm*A`Y$R5!y9&z?&&@eQ)z+emvEGV!i%Pj{H%AQ~ikyN{V&T!ObnKw4 zw*t~(KWoSpC=6!{Bq?E){#$;d8c1khpV&zqoWXIZq7d!FT~jyy_^Gkc~1=A zmk4O?mp!kiOxE3~Kz$M=12&+L=x;W_5O(8*Xl^u_zH!v=CgJ*$s@-fenNpw3 zWo#0W%s-Nvz!ZV8&~((^T;}$2NvUwD{65TE57HmUuuV<}M+r0qr@fJImtU40iB|&h zY&A}hRK3C`-j8>BI;0Nh<4Bq&DN)*=_|+ttbm$K~$DHtZK;%cHS*Cc@`4nn5-ymVy zomsQ|=e_UqDhiFKLa$8#dN-e1UQi>&QEM0_ZD5TYz6tZnt-(qlbD$2t-A-kM1$rUR z!VM7#3OO{Ivb7AKceoU9QBPCIH;D4?qBWYq4|Tz5DkxJ)9X|9bcVrP(v)qhLC#H-a zgdtjvI3c5wfoeG+jw^w)*e;@-PUFbmMXoGywB25Fq$HITlyK*89Ri?7px6A-93|*b(G*)3S9JL6AIS zT~qLtEZH?|N!LKAKMn!K_IQ){I~}fOkQyHm{EYA%%Q;IJK7OY+ zg%VBDPqeuX=>oneuUHiYw*3J9_loBc%MHQ+1^^%h2LM3xzgIkYeFuGOV@G2L8DmF% zVSPt^U4wuBb*`eS`A>5R<=dH9v%d36IQ(j`PNteYUYx#=pCH=6Q-Xgr8HviduTjQ{ z!#=to;-6z;pbo_>kPJm*5UP-pI1o&V{5Z5KlUMG-StFd+ndQfFimd7Tyw9w2RaZ=m zkwma6?#+aTWzAF7#`cNa&-WA=!1ZtluNC5I&J-Mt@j_Z&rLhiuWnP#ks~4eh(S5MB z+Cp$vy7*ez!dX0=j$fBdV4BZ)n6aVEa)_!Qj#L>`S>F8HrhEV`pwG7>D-K;I1>~bF z%iL(CzTdJm1D|1BryBffeP=P5f^_1)HpI9Cpgzj9&@#tq_|SiSd%?XOv9($B~~b1UpHV5HhPw z7CJmoOdNIzJ+7O|Na`xdOi{)#w@{~Cc6QCBnFtim;~!SH-x;*&Rp2Yl0Y__(qfa!e zQ_HI)7`13tjwLA3()pr`dggB0Z1ft3%^Y|unkJVIoxqF`-2n`FX&C^Ggl2G98ch#D zsUO52PdDs`5aWUhoT>d$p9kE=i00OLnj<$Z|6&s8*aIMCxEerS8XHzbgqS>CRPl?u zfWj6>Ekvbmh3-+8OpgE|s@S8Uf8_{si&Pw18UFQ%NQ;r=4kljm9^*LH&0l5s%YP3z zF`7z((kJ05u|Gj6J}90&!w>^|8w@$17nV3^o~*YY*vt@CDUA67W(R8OrDl~4l$=Cg zC#=sbbujelFx^_H-xzo>o*;2Zd{2IAp@%y%!kBWEFOm{dZ6f>WjvC~Kp^i7LWX+6% z54F~0w+ii)_=^l}4VB`&os^N^Je?+>D(`hD>))53Z=USWXS`JjtEpoTIdW>np0_Vg)rzo|W8W?fjktpDrny5d3EYoO&ERFl!fRPV=(*|#XURTNRW{Im zGG7*JFT=)^d%Nod0BVgy^L*u967LEegu5k73bF6b-1lhb(kd__?J_dD+#B`B<3qlk zikXh)u`3I$~62PR}FCX_yXB{@cB9-k&vSiCA-B3S>PP zIb$YZ5zSyTkt5aF*R(3psa~$o?%2o=hL+2p93TLGXaoTe+9dU2$=9Z!&ATb1kif{oibwki~WGPt_@5)mG!X z@1vU;ZrC>_k3wDecBv_f7dNzUSH&uZwfd0F_Vv-#Rp%DoQxzd%SAjp5T6fMNpJMNI z@g1+R9$DupQSKk$a|A{gpbZAGjvjru@93{rG9J?(!nZ9~UvjZJw2;$9S{RU=0UpR( z@M{t;!K&3WnZ{zQjAO9S3UzE5!-+br9S^vw&g} z!Cv%dkq=7rh4bhaw}uc(SK{R!e}FjWZQqz;zw#tmr(*%h1#}SS&fyfVYPt_{(8%0~ zd58HsW7IKFU6W$J5wE$PGg%d8thP#9M)OzBiIOh&Y#-C0 zQ=RIFas3rwSk?tu5Tf@rHTTSh))fV&(?h3Q4&*!rifDs~l6Q41J;~tz1glc7C?lAc z53l;8AX_=|)syQ@IOlVV+U=c_g6>6idiuQGpFm}1-nMpC)&5VMGE3ll&gfn1@)nm;rX3N~n+2}NJclP-m{q4GM=k3TIl!xP*vVaUZXtrS0szkt zv&&l-3dg7G4OuZ5R+Cb0uicnfFu@s>+b)#GV4ZZ{+lqE&Pr(D*vHqLW;#Z!hSv(6x zhSPm4M&sjy{Zepzqdfb2Q-Az^6*1Wb51DQ+S)No+O{h>7hbxb74JCzNYkEX zP7>T~-<|&c^{QxZFYnm7y|j(N1@&e49irNSk}WQtR8^IgPgj^%Z|z!6QK+32CF;bY8;zNl_AV{XTv?9Hn-?G1zOoWJFI0 z3{tbBxw`7R`)-Mabzg2#$GFF>y^W4T6qRi5Jsv&L)?qEIPSkvP^^+_5y zxrOYnyR*~{s>}3yRJftFs>DEBjNLWP%d8h$5dcZsj`fR^vT4Vv5&}BKj10P?j1-dZkE|{2wi04d)GQK$6EA!} zI6EN5aTla(&nOf^d@Sx5<0KZsYv)oAK|YmoQF0PzX-_K_;uA0E5X6~}Dq(UQ&8^0m zoYx0{F6=PC0qxUc+n!p=13o5T4z#UmseXoLzDv9`1Bd9zCTW7{o?m{a)cx1iQVwhR zOe;`Os(EQ$U36oTx38-dQI3TG)u2X%(^RcYlnfp02*L0`+HDTURCjAOUj&lTAMg~s?s(5w<}t7@$(qtN#=VB zc-;X<)i;`)nT1XDLqmg7!D*XR7m7!-EC6jBe%K;IolQAF4g{MnpdCpvM5Pw9OFCdB z24xl%13IS2|CI2&Lp7wSHdH{+@ZZ(>I;gGo+B+v@%|ae7yt1i`iv2*jD5!beY>`-k z#4{<*@}Fp97a40N2Uf!bxUHjK3MXE1r&)0_@`!TRS^^$x>FhXuQvxNY74V}fkLm|B z+ol0MAH9cgN_B=XyHE(z9<|kmK@&^nckU%^MzorA)KBifnjLI88#|&A3v;WLp=U;t))f z{tU*U(Fm6heDlVh35resTm3(rQ#+{5a3TOd>TKIcZDFr=NG^2O_KWc0vY|KY+6d3* ztJ7e^bQ|d(!_pU69LBz@VC!0`ppIDx;pzU;91YMjfe*D)OY1ujiscXU=~lUr->Z0Z zso2b1d!cC@lLG?qRXbuCV2OP2U;2 zXSc&5CX;6ksgwk%l93shqpL3_WcOWRsUTYDYER|y#xryxbN0x_BP)XwS4vi_={l0Ov74Vz7tlg#kxIS$qO8IeAt z73@GLCQ=3Nb~+}8$>2JbKF3&c)i7Y4{P4%0Kt z#=$PZ?X!TX8%m}YGF@PHd9xu+mN6x@$_c|I9#?tsRn<7pP(sR*S zSc!~Wn((O6Ws%fvlPgA1Ln_h`mU}rS_rGJJnlwVh(-78w&DxQyx!3r}Y(K6qiFBW@ zyW8ThT-P)>Ci@-BmN(|40>5jLV$RtyD&W9NEGfcC83)X+)%CGAnWhYu6K=qXU|&)c zjY>TauhM71n3AFADy5By*AIoN{8dEvv)U*+YC`Y&=z5Y7CKWSnvm(ZXi?72=ZF~=! zug8=0cgv%mBg$tHH8osg4bdD$I%ub9A0XBH}EdvT13OwqjAXephz5Da`7*J?(R1}nKW5po}FwKdMB}2uU z3^Avu=ar#T;3b+2HJ0rt55!yQ%W4S}rOGrJW8qG$A;anI>%DVR!L^9>32w}5qYNy3 zn==!&QLY>+rcml>`g@}0(s&jnz?6?gCJ|1~m|R=vHiyL;ld98~`$dOUnCb~A%zFi>Z{)%$44COg6cMZ} zJl1>t<|vd8n$Q!B>0V_qFq|8stQcw&f(h<9@_9#pD4sCX7Kk-fyX_S8+s3EsvCF9+ z*;-UT!fx>#omsg?mc1%?e2N1VS+m9GPt^IvGsv|95}cZHHs{-{DyC`m(DFNKhUat9 zXdD|9of*>hNe-D)P!ZE6s5P01beIiofjSbk^BVZ-sp9aM4XKNBV-0>?5=99E*HAk& zOjl~H*-0u{?|a;I7DeJ!G@}uy(=&Uk4cSUB-6!%Z!8kkOn(}T&kf(2kLcOYO{|QrC z4Z*T6v@~QmIx}^Em<}%)d2XX>%z+m#UK!LBs3}2hnpO}fJ|UW$HegjLql-AWOC;hBCRK=IJ>k)h~Vg@`Y+Qgv9hkbM{TmOa8U;_XebN@!QjoW={(ow6_|S;R+b0x<02SN zO-!*id>K&#f{LtVF=8rJqaThg!v+qo^cCv8Pp)(3jN=;9wXg6`}av?naK6o7?6CV64#PBJ(EddF&SajmgW^Tl|c}3Nb~hG z`NJINT2=`}T&N)OO3nI1Jzo}}WZkaeYR<=6=RFMwDwprzI!Q)J7cFou{b~0Y*vh-1 z4l`ymiBgp;bWaZCATivUp%eOGYy(a*D*9pkWSf7@y!Y$`?seN&L<^|bRg(3nn#T(W!qc0Irdf7(P0R55_(B zi&?YAAi>#Q>MM^(w}`CG=y0sIXTvmIO^}ovGsp40C#b%F_XXSHj(bsbn^}c2X6`!U z4tptnbN9mWqOqlBM3oJNCo$`cnN}+FN?ohs$RT4@r=mMH@U>918%W}oIr*Z9kd3{Q zZ<3f-Fx1-Wt3-v-PZ$SdL|?lKWyVaUV)g{M%#RK}^(3DW{3r%4bY3b4>63w%iX>q^ z*Et@TTXaQ*NUVB5NQAvG$ux%?EUw^Yi*_QkFtCpk(&`*~2z@p?w`#}U_pF>AQHb6g z7>|n*7-HBNJ&W$Hv~{A-=6sEnD}*Ig05fhx-)D9(PLk87&*ySio6BcSEA{H*W%}i} zNr~2-lzO^j8z@;vOf%;{5K%Fp0vcIF2AY(6I%D6Doud}1i-$q*rW{fcyqSy-;c~Hv z){9W;_Tsm2)r)#XTzlqk1f(ErIpk{!J9-2fPcaV+N~frJyJO>E!c@Y_1!#lrm|zY{ zzAh?PH>5k51|yWgOs(tI+sogyuHo<1=|hS{Twum43oO z-|rL9TxdSFzYq{DaOJ2sYaTk(Ga?R{;b_0%-Om|l zqd73JWTtIThKNvr5$MtdwVep=fG6oKI=w^!M~*u{fBl8v&<;CbdD*>U^qs`mhqb7OhkgWzz8HCO`tn2F^p9b$K*2~LQC=S8+4719mc?x9lj>NLG`bO`x^%ggF58~~ znXdIxli}=4UU$}4R$55Q{E&XBF7K)b+tL-_vOqz@NBuV!&Y$dY4=EdyG*dB))7#LD zpHUyxH#z9}f|o^d29X)HF42>oTir8$(SkcK6T^QJ1!zq%{eM6D>jz|Kh~ep%K3i#h z8!xJ@^L{mCVVpAN#(`IY@INVz)WxTZbU<5a_j=9BrJm4|fM;onVS()ac zNI}m(ZC9hvse{W6#3#381zRfB(9{jk0gYDuP##l{pZSDE9pqIbkR0P^FpBg3?i@a? z7))~XtZB+zi&Q|;a?dh$b|srnAydpn^f^$vuujlDLrUG&QQq%|enyTTOVf(>(W*om z$0CqqD9og~4xuw-Gs|${uf=W6XGWmN=0hTx3ozCTo;r36<0&z6EIfzNw%n%#@n#~q zhtNVALhw3;m7Si#MA*R4N{{hGsOWHR$D(f!Zmv{XRzgNR9;WwDhVPG^o>!8u&_Gtr z*3rl7L zf~;?n1P$84LpiqyDsde{WUNKXXT=v-e`;hSK3Q5`T56tc5Y=X$Bh{Q) zLYRSXHpyaJ*e)+ER&J83)FA>+8j~UOuU8m_@lJ$4o8sKz_b6YIj6QFZwBTO zw4+IF9~C{P*3A4pE2yL+TncICh(GT-XN7o6ARgdUFPIfWaR+yRVtT{h)v^P>|6lrNy)DlYWxbjJt zNmh$M=*!Xo*8j(Yvk~4UR}qeNfPTHICp5%u*}B6&z|)wL?{FEUdT;ows2tt3iD^nF z(yY`rBpX%5;~@R(s9Ets08zg>rKi<0=4pmiQL<51k}X4;i#*vpe=H>G{1*&_L9|s9 z?IqMBps%?I5tPj3#;QFxs?VL~xDPptCpQAj96O;#ZM9WlI0v@Z#}tpxJJWM_pnX-W zVi#5#Cc?5*f*%dXgd72JX6#7M{WFvnw`ioRAJ@>eA`g$JFk<8sJTHT|-?gSOi8XrY zpxxDf{qdG%7V%5aP#^>QCZYWmN#v<9jn7iS9wF&V9ZjrEI@m|8X8QtlCe&2rE(QJr zZc9Jbg{3(~cXlwd|hg4 zvlP;A&@*QtLqin_5 zzR>;VB9X~}DCS-Es~Xx5LJoSe|4_t{XCjVe0|u2(dBo2+2kyXt0}))x8WZok7MiXa zhx94Y;4_tU{geUq?$VbC=6)n|8u1-RR4K?zhO$=}obOL+ECCA4UqM+=ZnKH+? z8iKdfq{h0RhQj<3b8k_|LBtx+nh_jc0TE<-CB#V@z!Sf-D9lE5j~s-XfYE@c1(=W? z`d8M71y1+NO2)qt2RhWO?GSQpeKXfXaK@$8M(P{l=(`eiWuC3onfwGvn>vhJRppW3 zO_!sivFS&rlGTj!YGyDStk|hrD$-#>Vq>a@xHC15Y@&aMDG#8hFo)xC+mFPGjT@O| z6RF53$;!q9XK`9@Dcc$6!XqPj02{MCwbP`9g5G=I%F|0TE2%(~RE~ z+N+W0rNHjt%eis7wbSybn9Qk3Q^O0@{m!SClMFQCq|G2G#5U3@s7Q<&3ghl6Gm%5( z)eW_YSSXj0-tKYW+HhNjx&5j$%c-EsWZ>3F8wJ{`sp~NlJu0Y+Wfzhbs!&KT*kM(4 z^QrEuAPmMS|0W$l2RnD*Xtu3aQ7cg@2#rf#B#zw35}c~!Aw;kcs})f#F~>dj(-*Ea zqI0aQ8s%JDKoZ{T!E?|HgPCH2;OW*9_Q4? z7~IZd{FeWLJv-#d$P0~Xg6(+*NK&QW@XXWfM+$EyRU~i zRYsTg=wiP-t8{MQ;Dt$^(rn?L(l%1IPZ?vO;B4&2Z)nJMLw2J@hd#}{n*r>iZ&&OK z%E{D;$=8d}iSD1-etb{(+*Vo!a=(z!Nyv-oEaul;qg82EI<+Qy>*{&q(dPYo8b#L0 z$T>?_^G7eY6IPHH;QZDG9D2xEy`FU+jBIy{Z8Qt1ro_X6Q7I8w49v)5)qC_V^ou^8 z_nzBBf6|*hPLgbPMEHHQB!&XF8xQjI8u>j{b}8c2hSY2F`4CiZS}1-+@iQ|l9&j^e zGb^TE$9G;kJYFmMQ#-2D>JOY2n48_7$dD^~2%BJ_RluNHD3gIKdbn%XsBJ~u(`9>Mrh;w4hOYBp7||<+^s+D1BtlC#T^TG(f(VAg&1K&F5*e&E3OVe z@_TDw6YKmVQY7A0ly`@hAuPnJoYWdG%#Y8zJzlkC=ehb)TKR2-xc=82&dEY()+CQLw z6gu}J`@*DqDJB6$q=O8K`v{W(N|_9#h?(JDKk-|V%;{JFJ-@`Or2V|BItk6ez*^Mv6?)J5QOKQF(Q;0vL-p1Z-)!|ha zuZ!i0N}Go`w9DanAX^JQc}UZS+Twg z*CAX!!8S}dV`4aZjEV&Gw`brseqGUH$6dm~1{Wm4_!6_izt{EAhZ^_MvqE!Qu^-kU z>2YKEy-x*svLY7ObEl3C_iC}ij<1E4t|aKuWPoF>5x1ru4H#b!t|Ot2p)pQat z8DMP1I`s+>>$Thl+(a7iuwv-?6+7d(PAbsB^pXGEW`0dYD=DEo9V^d+tAM~5rf4=V zUL9Z`!X2h~rkAY?u@C2nL~)-Y3oipv6Qublq~anFIJaOklPELA_T>@CMl2e&%mveg z8mpAOUsy3SeOLf{mfCALbyS3Q*^BkGe)=_xBGNA7!04b{kG?joMcgpZi1rQs!7+-EXS^9;6be69r#-wuAA<{S>P%2=hSpE!QjtAJ8+D(yc^%zKga{v+nO!Lz zNdJR8zYqKmtFQ&&uKS_!{z9zuupE*1OU@oXnQ1ND{XhhqzCAC+(YbamH!*laMS>*Q zRdDw4)_B53pUg`%%cGq^85vRFuDF0P;THh6Xya8#e5GfqE%zk<3X-r0r;0O4>7|9Z z@rE8i2drovmL5=yLHT2*c8m2Z(1S*3nQZqSveJ2UNA8 zi_|X+dy!RgWgv%GGGR7}k3>NOGGTAV4tehIv0XLsNy|jz%;;XJz&d3CblMr6Kb;3M z%uzy2XdTvF@jT0{n>_@0Yo;#ORfxP<4=#89CD(Q>;*S;IGzw1a1Z=LZ8^G6CAN-#j ztNJPZeDmGRW{(0j?$Gg~zEh!%52$B#JEqprCe&C?D9+s)qiC@ryezv{Rdk`hWw`F# zs-p%%a!Q2;m-`g;=wH@=ZyP}Os3HRT!;1I=xL`pze!$Pfj|Osi4cw4qwWhhgUmye5 zruqarXmuxf@i+mP&;H1=Y^hB_9=OpW;f0HNBMNtWd7CA3qx1IX-oJN5w+>jcCuePP zy%T`0MSpZ`Pg?ZOG*!7RwDo`nFuvIFEaKRl_u2dTV^0Cb=G-mJeV z{$hnGzZd}cTzc=Ed5K=YkxVF%0FO`gG`MC4=5bZcV*oDJ`r!)sjLSYb%|fIZ=%@Mb>a87p9{P=Ey6|l)djH8&Qai4xpQ&-i7;vI~~UD+kw5&+jFn5mjbi=mE2KlT%+V{2(UH}2~cyi z!GczW<&YGEq~>Mooz@lZO;(M()I&pPvExx~Z-impJ&?lFPhlc2a;5LB#(V?sNEX6M zJOW*P)?2H>E?6o<7Es33>$pz}xTHls&GrOC5dX*Be|=yVYh5j_R4SQH)cfpM>-8op zJW0`pnVi_J0Q(4i**3r!CI(v@5RZ2P=sbu^VMQHW*Q+xecMmyTRkLsAhU4DAMqlKH zRKLbI*U%M2bZm{y@Y4^!zk`n09l-mey*jxa@Nf0Qj^qQN+7z!J=aQ&3#tRy3)o0iK z8KQIW8%`)hZtu^t?rWkKC|~W&9{YzsNiCha?K28iZ}%|#J{D`R*92+(jT*v7_LMPS zJ;)_lbLLmS#cGWe%SRMT?ZG89?4{Q9DoQ4if&bMmq~!_hGrdc_Slf8^9_=&G%Btq( z)b$DI*xbaUeS6J_S3GIJTd)9_Q$VK@3?(reU#6CBjtc~qpqJg9)%!n1w%ZbdEx>Vs2>PF zu-wy$pAkKO_0Ce)>CbYo4?%7kva*z%xmXANIHa57S0^rS6nHrjCn-D#M>CNFpbH*8 z7+ldY#s(5oqjZgb=&|_B4*%39n&fC6Id*Lz>D4T4`aiIX@LI9DN{Cx_##DH2=whb@O5KcXIp0M@t8ujBpfIoKeKT7?pKURG> zSAAX@W}@hFCOV_NaP)R^g4Vh^W8FNQ=kCc_aaAw#7YPC)NC0oj-JenqhfnkLgcTF%ObfHcH$C8^dOcL zc?f%#B8Z%Pyuk^U2!(IL#1I4vQUD3v=KL}6P52zywGi!aSB zOUZ65Nl|#*$`O?2>%K~-F(;?%jS!0$^)FOL2p;5l=aYu`$kW~$31 z>|rx}e5_1UUJlk|lvz7tK_ zI|`44<*lS9j6L)~=m^F*op z1T%Q!GZ!`vPI0kP#25)~1-jI|DEu;LMVu})D?>^D9Nr%yYXn~~Sz+va0 z5QP=74L3&S`0;3pVDk5<`-Un%f)wIk*h8rZ!kwDmcn zd?_#gGw%ZD!2L+L08S@4tL0XKOtN?WhQF4%dfjTN!G80KM#P0?!)0!g44`~~iD?^MpsFJBMD#R`Ioe1l@VH&t`8mRlJ z(F#zf5GGF`Ojr!olfqJ)N>ZQHb#JYcm<-0*N2tZ9HRC6Lv)585sTB{NmT7im%JFBd zlO&5K*-@Xk1RKYMU&40NLr%zq9+?c@Ll53d58hJ`-v3%7sSe)z|HnF71B|V8V;eB3 zh{1cz!F!Ft`*!wPE$TYyn7M_FV(sSR`D7t?x~TlAX*-@M{)$t?zrQD7-s{jy$Sj?1dC4NEa zB0$^{*?+gJLKOW!)t*nf(#HY8cG{%;lqNV=K90L{2U$pUCw$RtV~cBuzHz7LwsH{F(`DTw!&vIMLdV zds)757BoTuWVx|}JWbR_loy{S%~;%0hr$r>sUMI}*q>8NYf}2iXuOdE|4MKaa*ADfMFN2{G_k=d+?n$~~QO5?Q(;Bj0>z{y0$oYG}Xjae4GABLP1~XjzqD zp`v)0i$sOr9@GA!sAwb#R(k;G;z-=81iuM%Xk>w{)GG>cz51Kgf1`oQjt#-V)Dfau zfdqR~=FVR;ba$FR&V;aa2>cow*oD?#(|>nb$nJ!wH3EKf$X*%{i=@@GKJL!0?Vwy6~wwmN!SQlb^7w8L2ReGStx?{_LRzEUVdZ zL5Q#-B+{jxWtmpL(Sq!UCgMg%`kWaVWO1SDuuxcw>V*NjY9E4?3IO|Oj&$jBcHhw$dA+_EIAl#MDezubXRCr$c-8>ajRnU_v3v6e|{&sd@6&-x-MVo`{ZoYN4Sj8rL+8oOa5ea zeuLGx@AzpTfjegV{04l)asK$@{Bdw|{`_L`#Ac^ggbVx3)#yHOvqb59W>>meL1k05 zjcz#NmDweLYgPq<%}E}O%fM=GR>hUR==yJH=L`YI(}TE**r>$?oTj9U&i)^? zh6-twK(^OsGVA&^MD0#={`(!`E*Cd-ugk(*vmV_avy`nlIAt*(I}XOY5<~n)Cuo1i z5p?uZ@U3f1qbKPe7@VaCIW0qi@2?V|Pq$b__3fu9lb~cgTls zsluby6Lmm}3egmypJU`wfa4C3tvErkyX5{l(HOD&sSsZ{4f4fnu}9>=2+qJM4UmK3pO=4z@6&Y!7aM|FpX%6H@157dK!|?hdX8=iZ@5mSeTnt{F z;eM2$!OA3i5~RBsQswzjcICvP2eW23>Zk@)NH(PZD>9^EirM7%LotQ?7_pa>z!w!7 zcS;V#IbtNmp3q!r%1a_9wHM8C^FF7wmEWgxVAk`r>K!^GSYLVH=iuzkyaET#g&Id< z^g9pu=*RBNLpM@QY+ZPFitw&>M+gyqFl4?`(7h8EVgOPPu8b1+fv%Q!kQoe!W6{UT zWI@xF0%Ij>CBCA>XucZAYU6@QW4^5umoxWB37#Cb{f1;DZAtbq| z$7B?^DIl+wreD~BK98Kq`xz4Q^;9@H5O-_#VeVTOcT>p0_4*NGtD+X}pS8(xPySe-K~#tM-4*3?bEp`#S&)0YPmU};chq;S z`waXlW?s$nW^tSG&MwJ{%#e-0b(B@RB6=*yX?g7!Fp%au`sgccN=SUbvb6@qm?c5K zxP~Ump`@SLfG2HN+$U_xiOkNV-~11M(h1gI@kE~7#iS4N#7^yo8&Ko69bqHuU^1v` zTEVW039fp;GUFfh=2<4d3#Xhv*qwLHgYas|4qo3)3wnI;5np&b@enuXJnl4_a1rFJN7~52@IPaOybU2qTy%fY2M-s=w$*3d7@jBnGTk z(CzF_OJUxhPcR{co~F6I1sO2C7-qM=c)bzp*@H9`fKO5^Q!;gZn(d#uW<)=y%&GHKOW!R zLh;$5`0ZsuQvP8+I-&}u{{w|~oc7l`u}VG6{6jLWNd9j+WFZSmKdJ%xJTpRmp=6+F z8glU(arIgY(uF-`fM*)TuBQRBN%PM~7B4va-(SI6mfOk()bH>Ww9Kd5`UZ$?2iAcO zSw7Fe6}n6;q57~I>8lz)sUh&PFV5E4MQ?!l|HIfjFlPb<>$-6!o+MvvTNB$hCbn(c zwr$(CZQHh;GyCp+Pu1S{o~l#T>nC)tw|n(^-a>EiPnZVu0eFai&j?|Rn_4(~YFR3I zYdQg>JyjjHDu>~5h;RBd8XH#Kshh&xrOOlTb^I6oG-n&C)%$4Z?-OP`Ea&k-Q=VgZ-ia37&^v?MZ2J-yP|m^54tXCL0dgp zYO@nL|9&ONckx7b;r40gT+%ksdl>15v$2=c@opoNJ>e zP(#mt#F__iJ(6f<(h(xi<%#-WytC=}=}oH$x?)&A{MqzY(Pf(f*efbZ8Bp!3ZK`8Z zJVCA=h?)}Njle5jX46X{E*$MI-=AiymbbMwF-DW1$%{eo7EBNrDna211Z5!*G`OD# zmNwA9oRqK$@d)1Wju70Vz{l0tg?a`E9%^6{?(F@30}dILWfSrqCb)-zk2|mppwta1 z5A1DP2(~D(BIcBCO-&s$-4CfiVPW2LygWwscv8g zpVi3$QjGDjrI zUv8QPD;5ZD_o2Trl|lx#I2!U*;-+E!wgA<~!5Jz^fff>k%)t76K_mlN;ewBLwX1w1 z@X(6pUj8Cj-wvs;6r!8^PJdYTAy9la$6{JwH7k;_q2WrV9*Eaxb$Bi-qixko^w@IS&K2jWI5PK3|L4l$V`7Sd~Q=$@Gu z!Us7;C`>K535s$lA)B92KsEVB0&RILY8hA zjjPSRZx0hZ9R}P!%1^Jlt}cKK3jm!>n-cH_eK$g`U7SHh=VeNd2T&QWlLT!R)a)Vy z2{6lTg%%txdNL+LUa7~pllrqjM2fK@kXVF`q#4nkVL0ft?|x<-q5lYIJ6SEM5I=^v zudvc;*DjD~z33vC<`z`}_iqFUt3$L`AiZ}e_nW4mL2LfCEw8|&)Z$S+QHS(aDMg5G zJ24HP9CT)3#(Q2M%n`nrDib^Yq=0_*ES>gy`ku!W0DTkRtq0=*6tvuz=2WhEkOafG>+ zsgv~Q{D<26nUDvw<0NnqP1*f7Bk1em*0yyaY;7qJ>p{Q@qL-k`W<)Xpsac_#?p?{bxW zn&3K{{)rhfu=ByIJ6YG@ADZ|w9{XbPz+E4*>k4k@#hM*>k`=S-0tdb07<^)hboqiI zZQJ%!tA5d~z9{&OIBtnJE``YU6+#q8cq`Ee`gmn&{ur+_=bRaT8l+aIbBO}3?;md$ zQ|JosrY#<{2y}nK4t;wAyWodqQZ5(%wP7}7`vAOJ0B9W8tR0Di2_H^Cjtque$6qiF zeL!luDx1}>Pgdgiw`IGb`>ky!r}kFMb_x|bU`0Qg$r+8FQgT~A?7+u`v)i%JXJmab zC!=$&eaF@0u`Rgs5{C9N&>q6Ywm!Hsxib)3XP952pR-y~xfvW^LjG7B4OtMHLei$> z_EX*|OJ1bn{UDOw<9~--kaU={#^%Pn_(HzU*2*Ia^8=z;U6WgKp>cZg4qx>NsdOMq zzxu>z*yim=xMIhKQ&53|8IXh-Vm<|o#Y|>7r)k?%+y|GA)oj7NYfOV^y%4XPvW#^- ztFWtGg~wI;6ar6!18;$0t27aqa{G>{y|W0>-_z808^;3RS*hP8gKJw5p%yrXHCSbXYg-;}NN9wz3}=Z$F@5_-hVE)Gld&u}lW}|r&di2v*xr|R1j)|K6F8(x%23sE$GIer~cR;@h*cO1eQU@kB zRm^K)fm3KYo7Kqx7Hm=@D8{?jPjrQ0L<*jMzCy2w=Z2GVk5@bPe;C7f8CQNQHromH zA=}@9&!tDzMf*P5#k)c}WGji4m&vQe;l+-oIP0@QVb^c^V`~-y1T8#)U2e1AD{|lz6mZi&q_pU#{X>{6`R9k!2AkZ zwIU1QqN&O@+nGSKS_@3YJf;6>-DtL23mC!7C(vts$yTm|))08vI|3D+LRW@71W+8C z*REGXcnIx6*D5PdKP8fh0!ey0cT;dIG)b+lQTZZ|Qj#yCcKC8!6HA>^>@&(8ldyt< zjD2;VX)hCQV99591eO@cA~s&MLm{s`Eo%p`{JW|QDH+#=x9#?cm*xrSwybqZb(Eb> zq*BNp$=$>x8q56#gS8IdxS$n!b~P)w0XMdu6Tya%IxNcYDZ+D!j_ zHL{NBMygj5Y>kX>DO(7J6BEh0a9yET9sZx)x%cp^;{57>u}To>jJSvj?YV?A^4gfg z>KMapQ;LAAZ2fks@;lX;yzO8tAzl%>OHPQlu#x6qbM-T(^DTtLo%5!G;VrddxW8^G zv20SHk-_>08#Hey4lPBR0wC&}3M*k*A32U6w{389h6s%*fy;ho_=>%MCWdR2V8>(W z8#HX4jIEhB2zVVW@5C|@IQ)~J{@tyaykm5;32_`fDKdZvAV)V!{x{+6?;Ym%C*KCwY9uR%898# zc{a>1WF17g;f2bj@%5hzH(t;qvSXG+({_%hf6vb5Q$n)BRjsb zTxp51gXsec1j-3YuC_tAD$I1B8hZ<`0w`{_X~SV*BbvNAil7-BLBI#&(j& z3-fh^U+1#3DjygS<~_Z?u2`{mNA`#_Yl>Uh6I;Ql?Wk&puYBRG$t;~ZbTc#pvaZ2g zd#eWcT?jwHFS_?-{9iV^_OM*=hPTQH?EIV9=FoWlp63ED{Xx}MI!}T)^!J9DTFs*N zqmd|>4>b2cfT-6=e6oi+^>75H-XBH#`cw~mYh&!)n~tBO=)JBjfd@4u5BB9XRDfuR3cHz5N7;r;)U z9Tbf842}NFWg$z&!!~gVxiku}5bz*{NTRt6+D^r%ls8kL#|J&*_`YkE5;`?@v-+aMvNo6anFUvU=`0#2DiGu`FYj`eL%V zy0J)-$w~y=p!uu1tU9qj$aRj;+0jbH0Tb#XYAd5<5b1*>EN1mmpKn`T$L&g$lKjFb zk(xXiQZCHTPR`>PMl|n>O4M5@Yn3Lg#H)3jHt-s)%`6JYH_=8B>7;XyMh(UsJ`SWM zyU4N{la2ka#Znw8F+~qnbxJvB5YN5~b{6S1jLH;=wq;$BC#Oa{YdVr7jZaPrpVqIGZ!>JO!n(R4TVnSUn!Z-S-A8gD`|E;JmnT zjGeVeH;^Ydl>=)j66gbL4XXX9?3(;L+KXxUYAWDd)BV z0Cixz`^Qs}J0DRGvb$5II+J!^YGa=Ke&6tgs%0QLS(8r-rT#XAG zQLy?)oZ7Z3BHPI_yst)&azJ$wCrEnZYPkeZXsU6Os$^sg!MN1O6&SKzWK*{g^Ov`< z2p2HR<5|n6a%ob`DU-zkZ-zVLSu2veJhM93U3a^gWENgB*>X19e^icLm1tBy9VJjC zYtSyFL?zsw35{Fc9>}y?=PehuiG@z8%KmmCIv!vNGBc=Rp7m=L$eozfsHN28##~M* z_h0pY;&DXEi3^l2vYsYhaAUj(#0U&);~_?=ro%fL}aH8r{&DZ^3d9-i$-ey{2x7s?_40Zw9#0duR?XJZxtQouHIIJAul$65QqbOZ) z*2^UIv=ro05pSw_7@w&EQ{hw;W0qRqVA^+)D%tF}+0&J`o#?%tT=_?sD( zRwcLGO#5A#GiB>C)Kzu;+bL0MJq=h7XD8`Y3<2+Ho)5QHsMR2g3hyWR$78n`lTZ~4 zsbn#jZj0jv-CTrvFJ{!SqFG$1K_$9(X4GX~Sa1QU*wtPzaP_f@6ELenPZAnk3eESX zPM}K_!dFKXlU2u1;FEdn;)^8B?O`UU^xxFRO(|9E0Zu2{b8|v(lYdZk!Mk$!AB_=f z+6MJ-iE8vBn^LD@;@ngtEoxz%RD+644{eZ_4dBx2k~h_2oHu`}72N8Q3A2#lNc1sbVSHj;W zbKS(G#>>Tv5y@{I7$on;=e=^C{nUIr`t&eGGX{-u~ zFIRP^^qxVE3vS#WE|rAP=VdEqMc~9g;@q^Ul$S z|0-Kmcs&%ZI$La9t+O~qYf$@Sd(gCS_*WzCooRkgo$Tb>S{W^2JZ9DpYkN$t9ZW0Q zyRKQYFY}*qXo-^^oE!diO!yIh_d-Avj{`R&t_@Jg?s?e<-W<`q!SI_zgiIr9&0#KW zd<3c(DlTH|wZ)!N*p4z=uz@zr22tIX#@L(R)Y7I#j?*gfSv(elk|E9LW-0@8+1lNEI(X#xgsqIoYo*~4X@=TItl59d0YUi5ncJN;&vqJ~x8lk$->hyiNBm53dxnVus>XP_$E)9?>ud-&VG`MRuYZLH91_xMOSrJz1j7RQkAeaDS@i z@YN|5y`!4y*4}15T%w<$SmP=&e8`L+a$r?~le^DVqvQ2Fis zUYj@<)|k%VouJ@r5^d5bYa1lVQ%|e+>XxwwiaKCHYvelhyW5C?OFw~@un?;gdw}RC zU@suW(+lH#W62HHznZiRjFwNDAED|st#qRyjc>bfg^ z>*d0ZxWh-fVuwDUr)cTJ@Z~0Lhc|Z1hq&WMsQ4@G^40h9JNELMO6>u0?TcaU3*r74 z`7=86`}a@(&Yg{C_)qLsPWj!dl=MG=RjpYr(Y9oHB>$ik23=ui9`I)|t6a&c6slZQ zTFK2*C}(_8n_2CtB9^1FI7;`$ci{M3uZtxJFzv+;gJGL#H>}#$M2`HX9 zw#K?B58>Qa$42-FDmSEXipzq8^cl{VkSY(!Y%0`;e;QK?G7OZf`f-;^=T!CHZgS`T zGi(avXHZZvGcJcuijE}TFH?SC{=0a@?LH6-1qA}q{D%f3`#*_C2PaEMMI%Q+Jp)st z{{(_XDOosd2m!b)IGyuL3#HSGANe!dY3h;9)wc@^tG1HcQ7EDWjqw=`4c!tbN8{sG z@;rC(?g%D-r|yCB`r-{eUXHu1QS{-Ox!hcHzic|(oagxbykqrRQKQ8ordl;oX;c;K zn341~OHm?hR2|n}J_{_xmLI>-83D8VEof{vt3Y*`Hr#i%k$C;()7CxjRfjncbDxrf zj&w>YG1L$VzrAL7jq3V(GIrs|&l%(u?rQceg#jR)1yx6yK6{9hT|*y!vK`u*U9nZF zV?VFQiL_w}`Kbe1Z!*yzdSWFiB2^h%Wd**r#uQ8ZC%I~17V7hApmG%-yxW0l+{_Xc7K*_W+Yj~kMpxXed+(P=)# z^VNI|XS!p^fX7@IhIlqk5&b#hi-#|s;weAp9-?0K2Xje2ra<)_aeKH{tcNjEFTrMX@8dDiJ z{lIzQ4BLd?$9)m)GEQld<6tPMW|bwG*CFfW;3P4{NGUOtu(K#;u4W_q+t_QlU?}Rj zRR&^`70adFO&^g}GOi z$fl&{7B+>nY}X_Ztv0ma?h8Fc2 zFK&Ht)}(ynyG;FbfFKjjK7=k^xQ`^)VVIxK`{m3lko`0AIxpnfzYt(aS8pC*sS^4& zg|&K4I;Sd&!8}x+%1p2Djlxsoe?j-;+6wsY!kFUozwVqnc9u-z2k+$$zGEk^S?CE5}R0cSUNtHox=d*@@jdkN3y+gn7)W%M!9 zVZ7J`$G)a>j*&CRzQ+bq8n^Od7}w^en0(rFFUT(vsTlW^r@*yhshpwx7_uF$vctLO zWDzF;k3ru_WUTy1wGUJTD`*&f);Jfkb0jNw>yE0*4v|1#I2Pd2+kBnQB?v8Do>*KF zx%`c1ZAT_8rdyX(#I?%BL;I0m)HeD)L`EjC1U_h6a)Wa{I?r8 zhPY{1mScxjj$qjO5%6Hz)h(Yo>(ql3sk56LCp_C`m_Wm7c^F5^Uw6snwNnqsYP|8l zNUKpp>EE26ziK7&Uco#dad{gzyMHZk@_>NK-wSYWI(ktl(Vy8Bz_G*y8jUaD9du8> zwTO%-P&3V`%l>)vT~>G`Nat4`TF%FMUK@3PhS&h=F_AO^s7Fg)P8!er`il zIyca)XDS~F=sw9rHcq$RVPv!CJ7*nk8mb4%2M37uT}0`Lxb*Aivoe_I79F{QSJWV) zjCg8vG(2LJVR9BxNV6ARsEex>og^xF7YdzSCK`LkxjLZRNDN}3oA|~TS=<^@h_;Z+ z(&>r9Fq|0;#x`2(r-)O@)&L2s64HThtDG^r9`8jMG~qv#PoUJ}#*>ote*OZE_;u?6 zf}gNVXgFN>1VPPzONv5^$aF_CjJTN$<}xr<2R0O{E06R`GJvG_<`au1=qJ}GOd{nM z=51){TIMsAme0Et$^xMKC~0$i2GK-hIMKI(+U=yD5%GrF`-!U8@Dct{=g^gW&F%~Y zZD$f8QBJ0DM0F@9Tc0f~w74j@gUvaH^o(b8J0;Nk7d^h=<28D^ECiDp|HD^oKelLH zaB_8C8tYuJ1`%K_bRV}$JD(nMY5b%c)SDSCd@Pj0TDfU+C#_%EX7`h&Qq&P8E1_s+ z&xcE4$Wv*Et1ARLg8Ev}`E@k;kP%G(y#xkl7I*C57P(g5c~NCgTuhLu(|;sv7yrqa|n zc*_gSoRIq5SMJ8-aaXeqQE{d@xaW%Eh)Q+?LH?R$+uM=ZD8%Yu>m?F|V$L5k?vX2cB<%m$WxhYl7Nf+Lg3K1c=fgd#wC%A9S^2KCd zM?Mnk8t_|CVv0Ko&)a#kk|kN{V53&4{TlXU(MyjXgZC{8$66;DoziR~_E7YPemb-b zbqP*Qm2)Jgp>R@Cc5(xDYZ9?At-?p(n$5-(hlk^nHcRu93XIicqzu--EU3M;Wz*jET4#z{+DK|4f@nM&}HRi=|0;d3r^U1 zz?v&%n*13pn7>kr8@>EOUCGk>aK8rAy6}sxuzHL&Jih{Y+7%2zw_u$Zi$12+FQX8G z%k|w_f3>73mTIje97wG4cD3^r)pK0FXkmwrtFOu*d`E`2^tVO6x%(Ryr}GeM>wAeu;n-gS;s2UJJDF z!@s?ZxrZ7YAw*|#gR}h>K%D`7XP80ck*>Pd`D=j$#jhOW#u0($vCI2K1Br5FJK6@l z<_45unYw|@4uVk-s^FXr1$0T%_uGjg0sm~3yla>{iON!(+fpi z9I)GfD%lYO^1O0;U-1XFrcCt)s~iNuT7AI~EcrxQF4R~)8KU6ja{^3{CA}wW-;pt+ z7|78U1v^r*b4!%m6=oGChO@<(%1!<-b7G~>p8;-prp_{Qt_0QBK!2aZ1|n6BZ;s9p zIg5yOof3v9_46mJ1( zsHz!}gGk5c2bzxG=MoE&FK{`gj*UT&enSjdz*7diw1yvMYA&}Y{D4zOu` zmoWf`-ZPZSUogywW;3ul^r8F?dQn0#bJHpW{+%hQbc&Wz8j2_Rb-~b%PeMK5z2;)y#_+W6C;#aiC_{d_e}Y!NX~!on zygG7VlpSt@*raJy5wiQdZw6jnAS$*)(fl53o;o7=AoV0^J!L`Uaw@Js8l9cK1A`3? zTg7i}j70()&Fsviaa89!4$GI+6iQZ;!Y82fSSdHs1p6q3;`D_gvgMQNRyn{UA2qif zvcRKH4y&^k3;lVwavqA;(XYg-F$!J|*+~+M1xt(?E1d=_U9U|~u%U67K-3a$WWhON z$wkWI+^@8q`ZMkWUKoT{9-9kv6*%%plqF9{Z(Gkz%PlpE9p9xd|7{Kc5>1~Z2ew); ziwf$)u_fNcviXF1?GEG8HRRPb#w{uenjP7J*gfI*B%;hD34=+0{1lP4SoEqQ$06*J z({kHdxanQiGOB8JrX^K1O1|iHyXBm9=;0UdHj$Ok=^gGYJB|F6OT%wZGuKvZirWOr zLnZ{dX*&B&smz$zf$xN~;1-=3EiO3&xT-R;hLIS^Id|amA9I2xU7<*Lt^+rt z(a$mkAU@KY_sdV>2#Vl|lE%zkIU>zH04hC)hljOD4~!btRK-U9Bi;l=V-Hy{&vBwP5U%s14}w<+Q;zQ+9(>;?is6NW5F- z&4|x2uzE$1b#X!VbI+v06F!BpiK{y7Sqq1*?zeIvC0X zO3z-Ap7!&t72k2a#CfJe0ZkMa9PHh_?LCD_d*_k#{7y46l*&sw;~r%a{3cg+aH{bA z^yF}U1RPo0l4rS2cwC;o9zujPEZ~u60u;xK&pr{1W_C(-DSZA$5V zL}flf*M4NzC4ch29&J)T@RVvXl;Y@Y@h9U7AIE=T)QMr_5hNrcKI#S}B7Qmr09cDo zLo2)5@J#?jWRKfX*~k4mQK88pXNZM@zw>^`jk!jbtU|F;5<@o@Ki7#__(4qEeRdh` z!yj%*)A!`*FBMa1O!^pd9;kipD5h;^|K7IFxCMuM$pBoLcO8;S;d79|yM(x7S|%L1 zPM_Ok6Uch3o{Tuyj}<(vHzZo>=JOrU(vhLQbKqI8JGIAAFi|3e8#SqmlL!&W9!MJb zUriIp^Eu=rGmih5BNi=^j@x2u%X|z%x9e0E^2qBI)bHf~zft%Ht?ehWupV9boIw0Y zJ@%op&>mSorJ^eQou1R2zsn7MXET1BdqsXtU%}14Oh}En%n>4$01UTOXmu}Q28!~p z+!==;YYKVJyfn!;iZ}nE-2v>YQloS+<7w*Pgi~XAc+C1}_v&rU7qi2c32s{C#e^I)-wxaXBvRrtIGqWBam+r=cJ16la! z&81FU7-ljdME*jTa9tclt5NK-fU7R&UYPZ5kYDb@x9Xsh0r&{CKjP79&92)3#~SaC zo|5eBI+nBL5eUbeWXEZ7X-@KUQFt#ehFWWNs6jCd;@~UYS!)9ZFl+2+z z!C`s~6Pe}Gz?%bepUbytb|cSQ{J3I}xVo&G6xGO_dJeBHt`mW$OtuCW{70OJFIN2o zWy7pxouZZ6^G3~&_QZFrhwJ$-f-H7==@a#;=+1yJ_8^P!EenmL-jK&*KSHDQZcgEy z9g~w5nR|`t&*-DjKWnEZJn*(^_LEPRcfZ<(3T$ufR=Gq-_MyrFvI=rs(oy>uhog4u zblDAA0v*zi)`RY#9NZ0d2whYKHHg5DeAxn2M^L+na^FM?)&^g07}#jGnDnZZ1GLUr z5jNz)4_~(+l=rPueb1vlIucVS79|uHW{S(GbLTn5Wfi08Ks$RN`r)*YsiNMM2H`wK zpFlq{bD}C|2Hh@`e=XdI0d$JdY(W{PROBQz3VCdB=Gvm{>=COTph@pESd}!SmDr05 zROoeKym2=w|8p?O53~(;^>2>x{rms6QE~pSS%!**t&$0bclV_@i<*oS2pCfR-aH6t zM`BPjzI4A;TVmk`eqgWwW_(v>Z}60LdkUYaB#I=JK9)N_RdeFv@+mBvb1-$%9~7Ia zKL#kGI40OWZxcS}t{7SJp4&H;EPmActCN`xJ1s+b3Q#V@*Cf-gvzd|Ev+> z`nTeMb$o~c&%*I@1XO1mT)&~N8Wqx1+Xp0Upo0-qxdmm=p(CzMOXcFFu%V6K0pabR z&_$WTL>Wj7;7j9XZHB0HZ6`}KzosG}&^I4M>HIXa)u8O-lI5T;tryqzLv`_qyO(Da&9oyWivKe?l$w%ps~6O7NyW2B^vi=dj62bHIlmyU zPI2;@*$(fzL?-ntD@^ci5(w1Xv`J0NYG1KGgV7A!0GR4}5R|Jv*Kx3OO$bFoOEsV= zUIV&V$akZgC6zp20-TqHvrb%38JHkr)xT;GHB!h`CZOs`mA-qYg?rH^XkjYKW8%~+ zBTU)vC{)eP?lm?H2=~~tr?e^kwB!cxYeU)-1VspC{LZ`(~3`dK!6%9SV2i^y&ieSR{}C&S&k`AJWOqV5IPav~)i$nN@1P({b8|#I! z{tA6fLj5xE4k}{bmKyP7tN{!9P_um!3VCLqUGJhWjBek}J$L@WsI2yN`bt!2cLWMe zW-kz#L;;~p(}qq~eGgaQW=>0MVe`1(%yTV_j$-kUJH&slHc8@*<)ei-0m}Zp9H900 z9q46CvrRjb;qI@v^N{WzOtjLyfR6aSS`wyC>wS{UCdMUYPofu5Xd1I~@C-kuglHjB zjAk*KDRY>nx}$oO2(g5K8h$dmW|{4XLOX$P)lN?i!J4$r@u&9q@Og_QcWq)E-yYJO z$wBIE69?x|ZMg4cx6TRLy!8Wg(WuR1w?($i?036KA5k0a(GM=K+pW%(`&M@j?MQR# zy>|r=Fs^2*O<}ClIenhL?u~$)LNp90OGp*0*-;dP71-%c%;_CA+T)wt6Qt`qSLQ%! zW%tbk)gry69N`o60n$jmP%MbXH?6ND6;b^pFi_0 ziY0bb0jnZmjq+v)@`S*XqMSvw0R%Vw=3D~z9Cg4QW=)U?1rfwfM4?`6qY#{vZ&1_P ze7{asMT3@|SPIYAojeIJ`WzSxhLSL21b#@`w~wf1fLT{4Y~S!~8R-KPYrn?C#($4`=; zNT57+c@Z=E^AGabQGa2YVe@iq{;eTm^$p6S)Cwi5U$wqboOGLoBKGpju%S_3Pv4VpTvIC%i)w8MI~@DB!)m?g7_4Zl-33M0+} zm}7)|iyaT%l~y}dw&COqavw$UgOR>Z(2;lgT&F}(c3S*;v!^*E%&sRz2qQk5PL9y) zD@J84qKI{bTy=yPMx1zIoPgMje6USDbJtWe(=h2L9OXwwWzoana}|Xcs9WJ@aw=*6 zNvIO$(o_4&uZAKSF&rbRxy|OqtiJh;MEC4ThE7$JjGO^;tCXSHAIu$eXZB@|<)}V7 z>GI=aIO$-GcbD>(YnsU=|CNl3&9?ngR+jaYVYn8-w0R^$v7=z zvYvkBM=jDdJMhwPjD10wjl1NMlM#2y;) zcgib=caY1ih5Hv=mzU7Q&c4a(R^qe|H{I98qk=?#POme~D=gz(U}p4ZQMkotaF_o$gsQ?Ukb%Y7VDQ z#0sxz4UAPqr;5sSMP&4XLO1d2QSwD`iWv?0a~Y71Vonb<%56Dy_ij@6xzl0&2QVqI zJt|Ds^KDSYZNq?jXC`~G`Op_|`QUKY(CFXHc$W>s&xxm$X*~81Dm( zJQb;0*eZ^od_POnM|QF#(`1BPv9Vayq6A>!)73)hCWr{5*Sa>662}!q(8Sh6UNh?= z#W$WYS}hcsEu=GriIc|+i^E016=4)x8J|l1G^cfYJHJlZkS~5JMtojai9net0U7(mjfMy>l&bquye-p#iQji&6!lnM0}=s%$Na zISm$$%>biBN_$)l)f0&v!0;OmC@Ov=N84Bvi!^Fzk3&cMjp$b|WNfICj6dY-D^ zALr-y`~`^0mQq0I4KPA#Ql;HBv)JhY0|-L#j<+q5*MR|BZxP6cQCe?N)Gb;!Nhk^0 zUmzp#cQ~a8MQxcUl==cxF{LrwcUDJpv$rfRD#DDB?jFBgS%z+8N>(sskItn{%X23zR7hPGB88NLqSzXR z*C9a$zqG?XY)^PR3>!6x#kmxhTjM!$$0NHEle1gPMjQvE)PzEWJ=OOS z<+v!f=Y`~DaWKf1CoSf1eXRj?UN!aRy!?t>H4)N)pkDUc3DK|YH;KJc-_ z4U#84FKTwx=X)W2I21kvY021@5eUlO6XqnK_qxyPmecyhPBgF@MHe3&)GbnE9e9cD zGgLM`XwfUF0?obVW)~|ZG?q%Za9G1St#ir*xyljx6=o%Nzy6YjZ%VSIiA3#Sm?H=q zxf24SzhyJuIy^Y z_M-@Mxvj<_UD3$xrcp2lojW8lrbPuwAz9Y70^;q9#Jho_b}9ISE=%mVaCy1a%$+f1 zNMig3fKwk>n+$6UsS%4SRW+v0n8lM}_$Us7e{Yh4qzSyp%RxALgxgb*y*(pclgvf@-&lKP*ZZ(F-&?HqJo0CPgSkl|(Hc%4c5EU&CF}-_u^wKjXPCgpE@n zdvZIwQj|DX-j*~>3EmMLsM&pxvO5?~lW6Ds?Tuh@O}H?=>!oHnQE4&j5qi{Crdru>(2OH^q-65DdSDn7IoAtgwS;x&>TEl+ zVn)1~uX$ds(!h)37Jbf%{%SF>iR&s{-AJ~ZnVUIKtugS5rg5m;sXK~Jn*KCD`N}oG zbGjthc=x$jMObrhr^DYZ3Yl5hN{t!T0_g2>>1h`VY4pcS3SV67}6J3T7Nh5a#z zTAxFT#4Es^)~X=XUtKdCfIZ(ta{xKQX<*~oiaV)~cgbWE_EF#o zgSZ2%51MUKW#=LMoInd~9ob*lw9EeH=uYfX9$F{36$nXZI7#aBLhoHV%C?w**IeOta(ri4m(U-5*uo^vt+V;m znldR{wUyLN30@oB!J3Rrmz8scE%-{71D(_*`PD3K-K=a~!Rw%F*iVKoNWkb4n^?mM z%eO>t5vb#vSS2%k4B&gyv4O^#dz6(SxqWg9ZyNiBx|#Lx-W^zLngeIbFut zO{O#19j4RLGD^N%e+Yos`sbFwp#ylD;0BUcXsa=lH#A%Rta4e}3tsJq>y|ck7yVhS zH?0edjxv*JDJbLs)KX?H$9HJ`9$r#xrKJ^~><%RqrV6E@MDf)x_%6*pdLBT~2O)*DG?0A_KRez^ibhyhvt&P*=}D zyj9(hI6&fK1txH_AXi&lTn~L!g;J;`UR7wQG*@$=my774iE@@a*2@4WChZ>|7aK!t zNy0TNO;gfnYawgSDXc7pCM95@GULy;L$B5S{zp#2`PBrb?hnV`zrGT^I4;H>4I!|{ zEFBdE`@Z&bVs!C4qw9{&s zz0x56Rim8Kt;2R>C}cAfjC$>q<6kfp%ay$^j&IS#i;XO-kXRPWqo36xU1icwO@2d8 zPfj!zzb(A*1CuT!F)acr;#S}yY_T)aBR~l9f!*de73LbD)tI9E8l3IK58_E7Fr2fB z2`_^yf_>$yv|%pY6nEBV$%oZu$@)GpQJbnug&}5OnIJLqHIe=>G5=Oy7MPEy3ctD^ zNi)nZtRrb_j4yO>TaL4^otE{VkZ70fhpK`oGcdHLa>Td`b)YD zfm&FGiM488cs}SLkjWtGkYq|4t{pICX-n?eQLC=?GS#;&_TWT1khgpwkg|S~*TdO^KnP(NSL9&B)HpNc`^@wq=$z zEyGa>V2YeBUCYmuRM=Oj-gqXHtIS+Mk7D_p!Aa!oP&ieMy&&Iy4z|!#elD`UKAuuY zO^KtRew92!$taO+WQlpXNlG{O_;8wk8HITXqARzb8g4vy6BL$`Su7YJF}2)_phfYw zil4)=V37;1T@XQnW)>gxAJ@_q?$7@Mj6ie0p%)dC5jdej*zL98jO&s z)fyZr(Vn8gNSQiKgQH~W85)d|ojgl}(Qq`S&e7l)7(=PG8jOW;lsZp?@v_Bw4JN>` zl)6xZi8AkE4T>a`OEj1)QB{H>9gK08#y#~`| zYO@BVkb=_?*O*{=Ys2DTbCq1<(n_0Z0`+qO&2{vyXY!V1b)gP)yLqK0)wDF&G8u)x zG}tJFpu~&lZqr~J+%5|%B5UZtoZJMKH4kiu9WMAgVvk!{gFC5HE?mm$1%53QY^jW{ zO#^b`E>0x0USq{W)8QVRMz*V=7j{F~1$*r64>M2=+F&ovW$b)(b;C)7{Wl;-3&y(O zeg@g`mPQ2I$S$6cmy6r4$&Z?Zbv<69t6_%vK|$V z^}=Itzy*&ZsU1|&;2><3!-@oyiMhTVp48we%J$G`H?3-HZE(RD7mOo#&uZ`-Jdadk zSxE>R2_czJMK5ab5;dcu$+YkitXDL66<)*4z~T_1Z6RUIDWL}2;SCM`MX-2daCHl1 zyrse0@Q#E~xTb~76Gu*E?t(%B|DFc#!v_Yjvim)(-PcP*-~X+_M+Dzn7pg$xXhviG z8ZUeTpSj>u1_g07JuNo+iw2*=7r3Y`rCEaci&h5eTd^y1auyWYM7M_q`>M~SE*gMinHA z$fP!T8aN(@n5r?(9Fn`T4z`o=gl-KP12Y~^Ck*e zR6xDUd>Tt;s1dy(linR<%M?rBsq5QWn#TOBD-P7sc6zkGz2=p~*dbvN!xvFtj5NX0@BdFCeI>Bt_2lL(Acq5kX#ok zGrMC{Hc4ZJRF+g1DsI#ZNK7Vctk~$N%$~7i2?_=^I!Yrhf7lB{{F121(!O$WsV{)jnQhC$L52uab>ISZOtxFF^$~wiwXF zb_mDK#cC7L@g{4YP%Y755)@KunFb#4dRYxyuCbF?y_~%0C86g~gOl*mL?)qC^EEUi zsC7$9N^<|4D}w7CYt&d1TOqZDzMRX|b`I&1W{ri2Kfb7fC9jZ*S~Ylpig=ySXpS1G zXp-8wN~{iz%TO@ZyGF0KQKr!8Ywuz6W%!1I)3F+#FNs4d`&am_BZc1~TVsJSnDjpb zp%b)LW9!&?k#GYU#AJ(<%+D=B^F-uLs;{St7ijE4;++@4)|@257G8ESyTrx*hUQ7k zjG0_0Vl*1NlwF1&lBQ;BfQQ$RS;wZc zYczH(=?ESo^@JH4G`5k}zNBWHN-Kj?>Y7O#%w*SV>;}?6uDZ~qdSs7W9jRY4*%po6 zBt#|D(zJpIxJ6@IDWgkGeK63hlNKWIw`pvfn2@3b!L15}%3JH}X?ktfU;ycM#&&4z zP9d~{eEA-e88I^@iO-N`XRobVq`veDcLsU+qLMnP#V-K+XV)ZVRmTKj)rsft| zqmpVorm;uZn?^CdXJ7^5X0UOW=9OaZHHhNdy};KTQo>EP3B0pmNwBa5>uuTQ{L0q+higU zL#Na?1$2f@p|XE#?0xo;VCWPn1(TavrC*CxCR3HqH1;w3TvVAXJ+;ERrIUj-bqxU& zoSr!)5ou`%v@9DIEfW&+wZ^_+-=L+M(_t<49s9w>zK<-SmU4~#$iAeIAqoRJ-!x%D z@~PGaYmCUT6{^rERm9bIh}!^2SpA!y5ZtLC-uj$66{I zSJgGdG^RXitZ+?Xio?=alu9FplDN>3P2W8Rf+!GRJeuNEyx2V}7+FPHw39+Lb#*xQ z!9WAKYY}L5O~XnWSG2as5&_PqDM?DQ$*;E5k*3e=#yV1HGLM?2X-Wz;OBRFP7-(J- zElbxFzmkk&taR6u9!dsI^d)tTwbIJ&Kt3tpp=6TK-yU(%@CKwcm^JBoz)XYw;TR2flW#`?#gFtbB(>r45?`r}A`6$Ux+ z>+8~rjgBR0buq~8xRF&Tu~i`nrADdABxfQug``T87Gbrqx6v*f)NXe}1dgQcML9yp zN|~-HrEs*|2hg)!3a#Q+$_iWS>T83|Ze=E_m%^EIOXn3%o#j=^*qa2VQd6oZVCGyF zAVrP&${bCpR_0<>V8x1HBidfQ>=sHL>{x0sUsD#)g6M8(l7;BfD93Bc38cZhgv_L) zr=gsvDFFq2{1j<4EHZ5!@tkR7Jl&_%YD!R9BF!6fg&sN6Oe~7zizR`0Sr%*wNK31? znU}^|6)7L^m)<6w(=?)G5ullqop6Me#$Aokamk<_p`D)-D-Sh_PUdP{V!7Dot6ftdT|u8P;Wi z6((IRR%px;;(4m3oTi*EHLZkDf__9Zd0-`$>Ndz2lv=Bei=7;%oTB!m}{Ekd3}LnFWo zH0454xn1f)<+IC*D@ux5gwy|+amH zxKdNDBIlC4C*9<9E7xF)(i!E)&6!kDSW-TzVxCvIPTA;EHnj7RC1IMfNy)hK-)c2eOr&$bX-LbGj?mgr9mPhw>3upMcIvS%o-BSlzqgHRuZfSHRU0aZL+VV zTwkZ`*OW(;M{$(os9P3X9Zc?gxi>7sr0!<&2!z{_Mk~WI^ulXeg3S>l$!|82C~!@R z8VBYHO?g~-l8m~TVi9raWL1km!S4K%pd280&*DO^JSSu59oVON9Yc++j8jW`Thft4 zTrX+L%cNKR?MfEaNfniZ?&T8fcWFS@eoa$er`oBe{77ANMMYZ?WEKsSx0H8W%GPQ*?=)Lf!5k7%4C`iMTTCfi5cU<5Fk01RulD#=o>I#fM^AdDaglLZ8H zkyCUYxA&6mr#7|GF-F%9yM=j<(KCJb&Y4rPKn#q|Bk9umBaICwfT|?N__2B;>B_ci zBs>92Bi(;ZjyBR=*dErF(kGziUG4qWR0*S<>uPYv;v~rEsyc-x$Z;lQ?WV*uLYCfv zSL?*3a%s!5j!}r#866Z-Sz2T;quwu~lbJEgOS%b=zAlI&ezyau$7Q5)RaRsE$#eM z8QfJDX{w^C_EidHmBpIsQN8w6DrJ?RrlzWC_EmJLSf;5x)C~J7(`1#CG_|+d$G*xG zS*1}^2dD$>t4x+vPS(@{b%?z)-_fL|b}gDZQauWtsSewKujHFcsosWU#=Pw(?I zwMeC~iMn1>OV!Et7NpI*q9zw4Q&E&oaJ6`r|HYPRoc8WVw^rSdcgZ+~>cbSSlO||;>=x#~p5osgVfppxdsdq)_ zKp#TJH0srznz~E92b~towv23qt+81yb+`GXj`gG^ItOZ4Q`^+NXpe~l4l|a-hpqR* zt=^B+WEW(+)dwhXY&L;=NK;E-t6SYqJIZp5>Ka>`uo5~TY^Ym(jDeD!<5myg$meDY zq#=BL0!wqfaG&~=rarCy6IDH$o+|>)LDP|d@wU9QF`Q@0rnT)1Xs`OL`n*eh&ORz* z*F;laP+yesr3lC92uL5)@EspsMd3z7FKg;6q~Jx{R&f>x1(`}-)700=8152W9js|> z2})_Cl7DGzv)H9uL9l4t(K<;IFU5^5Vz1I0LSb$Q$WX6G%nX@UR|pGsg|vAyJE6~* zaFr+5Qww>(D{(d&1}IKFxB7`OA!TMq6Vk1IhT~4M@&zKz9*@%ER=+~2)0?@~ZzvzR zg~{*GlFDASEZ7{(*3TekH!aETN0a&oO(~(t;i(l)fUY$DqpAOu`+Mf8?oxlj*&jRA zmMkIL_-B~qR)0q+om^a6TvePsrD8@|HfF1T(9Au|pdi7SOW?-3IU}O0rxsTfXBW+` zs3jp^Xlgu3=coysUBJ@uPUyXGpTf58J%4h#-5mc z;`~)huo;ckd1a!3vW6mz9aOM4Av%Oc9bK#OGx(XPA3`z2Os?vXn8{U=nA(;Oh`|bp z6CGhftbfATPz)c8ieNk*4Nq;E=dgSZM1$XQTmAwKb= z<_U!9#R2j9ra&!1okT~%)RnXnq{t0+-6|??Z9r#-Z_)TobPTJD5gC^;^N08?8sExq zmE>bdRqKlSptx?5ozv+aE^R!%#kXnvc76xN|0xd7pd|kO!nFBukHc2Z&n17tXK zK|BIYvn)~Ju$MncR{`?rqRC;Z`%jHO!=IHF3J!6IW?pCGk26=f0h=n@2-u7KWfy-5 z6|LDcY-$XaH?>qojA_G?*7z%AZ6~il%|Hk7Yfh?a4$v`TrJ3%yh;47wOmw)vG!b}X zCDj$;UZVqDi4yGNA^x_;-x24{X~m3$cij9vbj#%lz-Im8XB24lvS8zs=rLdN4L+b# ziEl+bKH342QKHBAM;iZF98D*iK+l(kP5b+#PN`mThOKDTi-eEtRqusa_`|={_*eXE zxz}FOsO!CMHn9p_D2RfyW#yQ=pw1yHe=nDt{6mWnGhT+o# zUI5HX0u6t=U|Inmrjj8A=)Y7*6XkyBD$3HKo5=4DJ;XBudg3pqE2H9>CE9ViI4a8f z=y0<2{J#3LpZ@HxKL?1XLf1@1{XsBTJaZsdJo6x5)GN^Qhrm#S-eH(OT(skKb5xX% z5Kl(8M#Xca{ya)AAEnbZ8jcpvV_=MUj)ido=6Jo{1UObaCyIB4E*yyWBE9`&{aLI( zr|8e)^ygIlS)xCu>CfrGh`ddCjy-V0F2TYZ@pmH*Fitrfu&pS9x1-2yM-khBLUk94 z&`uPOy*Ob1#NmDxhxBw^QSXTj5{ z>rq`gU<=Zx$pL_y;ASBN@4xDog+%viO4 zT;^u*Wm45JJRx$a!Re2~(+Ww7{_;K#Qnvx;d*S2CZ7>+~&+WY0mz8TBEZwn=`gql*!>4BP6R(_*MV-0n z@FbGib%_H*ihKahQW4r1l+@pga_D2c!V7VkFnA41UnkuWXHg#`1}%| z->|VvVdGdbKK-nK^<~*?D4WQRWRrv(OqJuR%fVRq1;sywZsWr<@Pm+&I(QtX!?7qs zt8jY#gwuko9{DShK&}|;8x44k9-mvl+NGx!y!^H`C+Dz zhQX(kbO=7Zy7G{kso(%RBd;LLy5xzWWT{j@g14Kclj2hMupUBhu%1FwuwKY+)|;x# z53_92L8NNffIRmeHb_cru>cTeIn-7r^QpZ|4kZ{ed4vuLlOqX)OpYc%68;!cmNGeh z0}RaD&3Ym&Tf{8Dyonp23#MwrtY{;+ILeApyP|9g-wG*tk#sF7=G`!71B@jZ>dH#4 z+K8Om$EMF~V`Y0-IVNY$+s7*A<+ZV@Hdeid%|~qWd6+GvXPBKRr{OFy|29KOo=7~% zYWA?8Itpz}aCRIdv#HRXl|Uah4F8IE5|1xv>z=V<+IeSOnLz6JZ+* z;CxsN*max-OW`F}hjU;#W2}yGb`s8kde)0IqWrI5$FP%GDQjj`EW{SDl~`*v{;pxC zu~XSK_@4;iJBPi(*0K-r`7Jx2{lqR{zp#tgA@(;hhbEx(ql8CvDf@%1 zM0nuvq@!ZU^MirmF6<2JjAnOX8rKte{r{R>k(bYUwXrj! zeA-Ob9^3H0(7f0Lud(aJ*k#MHFvjlni0ol%wLY?C?Ch9P+zVtj{a+l#g8VF-QQZ8$ za1^)TDBdJSF;9+#X%tCsFk?IwV?0$cp6A4j=MR|1bryd4KRBN090!yZN8Nf{chzR_ z=5ZY6bE~OhD=jn80l$l!Ja)_f!U4Yx2k~|^&F(<6?C0DcjoK!g7;q~B_+tHKxqHWt~kp4jF=iz1Dzf`uuyd)Y-O0_qNYJ9Pu} z!p9oVRnll*?hVi@R(6Q;8=-(2t?~v5Q>3{hNEA?Z6sex#JJhHFz_vg?)c^<~$ z8h#vm8D_Cpp_aW44eSkA!QO&1*gI&mzYFWw2WYo{441IaU?ckiwz4l_C;J-qvTxvi z_B}kuenbuM6MV$}3!kx{(N_5leqg`DuV^iB1wA%}v2KdOvJ}n+Dh@V8aj{`a7dBc+ zVaF)`=-SQ3>VoqF(sY$ryA`@39-}3?O7{}(!p>J605T-Y!|dwKXvxXsb^BpEn%vvq z5~}gAG%z=96ds>$n+{CGAQfOY9DcMt*uMxaI<*Ta&Y|=Gm(mkbm0r+C$%1}LHVjq< z!#E`uW+-`3t>nWA%0O7A41<-*aJWbr0h^U0;WlL?+^HOGa;)!)IM#GdD27)|kDr{1 zw4Shm99B+xGe%oV4NJXaWUVuzU#MO#EPICLbF%EnjYgraE5qz&lWrFBoGcl3s{j`5 z_+ko_DKoL-72sB8Ll>nQ(v-Q-U6~KPm4(n>IUaJ96JUt42u3I;!e}Lc45@)4B?!}$ zCCHdMs8W{0Jf$9vR~lil(gaJDlc7-wL95aNrzk5;j5*a{%gWwVNXu=5o zVMXu_BKR&2f&-}I5=HQY1%Zk#zUt`|fkNMBh(Yi&rm6mEa*BCUyI--FJr!AcZh$Pg zdToNP`(W3+d)PC6_EMO=(#HPva72M@5K$<(gly`pK_fbezMf|E^)!Q-Z{q@xC^O%^ z)56T7keQ?7jMn?OvLuS&!@Dd9$`HYfI0!ywpCpXn)4MGQf{0*g90XtBT4+TOKm=q5 zr{`sc*;nO54uXqq1J96Nt}y%7GVj&1z^R^NGM>{7B1sA4=aLQh~VDN5zsaGL=h;e6~U8;;ORIBoQf-91a2#W z_YuK|aS$XaTEYmrSP*a(9NZBHLCT*%kZMJciwN@LAn^YQ1YNBNrXhmTI8w9=ZAmMF z2|5CAUS57!=@wRcMj;EkjxRAm^%_vUXek{n7ecg`;zI3h*^J<)#ew<&T3~UZ_O(L2 zA`a9?l>YId4zNPKJr2|ZN=AIBgRD>=iUaj+B_}@A+&e8$UycLyeY9lbLM^z<0`>Da zP(N0N#)mrW?)Fg8JMA3m7s?T~6N22;Bnh=wSQ%jvSVvD&5G%<5o1}xql&-;1Z=?Ah zm+Ygg@Q;W?_V3E*_)w3wLOrf?sHmaG#D_Z83ibFnP@T&7_)sTUp@!l>O;RSthdRj$ z^~yLfDPMI15b+o=e>(OH<3oSa_@e6RruO@HCV|X)L zV|_^&DMXke@f3iwMB<$qIGkNfup*IoMt3t}U~Lp>fQ33~b9XD#4nki_?NiF;`IQ-M z$}E{*5msi0m3d)hp}udXXRiw@i!3=}gG;H=;YEAfllC^8Lx4MnfzNpa6gZEB3C@vF z`Mvr*lKBaCRA*1agTq=8s4JPFz`DkoYPTpy)p{kT? zA#5rVRsu=R>5%CxgPFdNeoMI6lXNyUEx)|cq zCF0o7w^$HN6tFl=qMRW@acP)GdqYEYlK_?olK>tp*ZaExdJYN03q!)nnR}IU_A2Yq z3li4vPOPwQ1H3&%>~`M>nn?J!z|(sDi}os)Y=j5ZQGAHLAs(bnZE}Xzk02~#91#;F z!*NjGgA6NI(2IlZ^ea~r(y($ZdQcgTuyXw<+92$)8Pdc~VGs0@=rM|CIK1gRtlYFS z!$Cxyn>dE81_nQ2xXpldhlC~D7912?rNn&%P!Lw`Bwngw1e)nQkaG>rk5eGkc`6R_ zX()kbzzFBrFx`0$%yX`Vh0b*lbe<2b&h>Di^8&ckc@bRhyaYBoFN0g1m%|?C6|m2F z6+Gs=77jW$!E4Uz;Y;Uc_|CZnesI#qR&Hf3=WQ&-c?a`5?=fl5hDKZpHdNWE z>_S;cWm#e`Tt(S$Q|=M_c&$*O+^g)y#qu>6r|eO}=y$h8L{YTdOL_oB@xJ(K?0%~p zz8FW1^{}yQz5qo8mXD{I$I6rmV%v6XblWzpJWz&jC*lwl3ELo>-XGoo zN&0(=d3X1bEM*IM(PyI>t>tsq9s)1x)8YQBD-}{c_jgGuUf%vL2x(Of_uXT+;n~E z%+eR4<+PcRYTB4^w2S>q6nhXw*n}d>Kw-1_0vG5J_URIZekuf~%DFPKrANKRb_^|T z7wlEuq)j!Q25%4xQz8+t%@%^hn&)~333?8Et{0%I>t*QUdKHGc-Y^k3-5~Hi8&jlA z=RtQFD%*hAP$R!zt%tLtwh66W;Q9tUuJ0h(^*wZR{b)kc-9Ykz4U&m^h}f4GRz5Vg zS}Z5h+)i+~UC_ntHo^57a6h);?r_77BAP8;nmg0hhWf~Ml(^X<`4gigHq&T-OdM)! zM+6;p0{1ZB?jykGJ`%dRM?oL=(J<6K7Dl_r!!hoOwwB2!ur-FjTAje5(LFpg6(Vx6 zfwl>Zpr^iDx1OIEr9}*$yAo-c4FlZOFvva6ggdgZG5R|VR@}-X$UHhQxQ}pWZX~v> z6>N!R6EQ>6B}d(nxf!0yBujEJO6~}mitgau0DbaMhpp51++=ak8(dGjhkQq-JVJK^ z%rL27*Tg zL7ca8uSG#P58UqcD3}*P5BJ}oxBD^}?7jkp^GX=*z6M6RuY+0cjWEx>2~Kif4-M|k zu*$s!&UD`d=euu)OWe1>)$Uv2M)z&7&Akooa^DUQy6=F;-P_@ydj~w}-U%1HZZNgTwB9%3)Kp z;eL{>b3et_yPsy)xSwI0+|RNt?&sJx_X}*N`$g8~ewp3pemxp4VEYW~Y@hO}Sm-W+ zv10d=Y^=|0wtQ~YPc9v6H|xb9E)RGE>&N%HffgL)b9F~ADU_SV`9IQTeIQ%;S||P+ z8(o+BvJVT(^GZdzJj)0k-5is9l_6zF_CHK9}_Rqy2&!>$O ziY8V4{6^@Gk56xe6nuPkBlYe)v*t;WrI>xbzI`(wx~2&A>j@#1t&rwf2^pT%Nad+8 z+;cjN@|*=@J!hK?nsLSej8lHl&7@H_ir5keU;`}Xf#)hjc(tts@}o_c&(XU)N_2S> z$b+TIPvUH}Ol}qK5T!Ok67_M=wulOjcD6S;D4v~&`yOz6?nPu1l(j7L49!Eq666n#f0#Oe&*6(hI zs3#%n<+MTcHX?c_0YtJ#z12Q8h@4{gQy|Kom))lJbJ-yJ2@(A#0YrlZq50d9g8l5f zusUyofqa`B$c%P<6)`}q99-H=NY-XS53K?+wJPYZ%{I|Bz@Te@LDvACu4LqgP8S-K zZ^h76Xo(D~3#?pNj$AmY^XM>Il*h+FF0&p8P*1ReJ_qIV+<1oqOxEOy2_Okrk!(aH zo8lueEXkS#kknd{+=EE&O`s7f~>Pa!gZY?kYtC|*MugKv-?CBKZ}O{FcC&L@dXqcsx}<@(i$4J*5u>p=UQY_b zYRHNt50T`@M`Dw?`G{me0!SjJUos^5QWEH&T+%hul73kNLUOqk-C*$gauN`dY9S;xmw1#z zXg7EP*lQvd7FMsy+YKSHO>?0P1viD&Yi|G_9f9|&*NM~eZR$q9dP6DxD-(x=)Eng? zA^k%Q>P>sqtzq>J%*WrbGOygW)(%UpFDAzXFK$t1ok?!IrwyC&U-MwGE zcitHJ)rHmRj2;BN*O$$#K)zDPZezXc+7Q13B8o1ftd zt51$`W9QXT9&x{3@42Ua&ksOyhFg3eBEuP0pIPf+=^jbV^BHbOSbZtO9adkhrr$Tj zRe8E;X}P)T8wcDs9^AeOknEcX>AoW9<121)h>i0F*mz$JTjHx_EAZLs zTguMz)vU-+@Xw+T|H;aQD zF4hW*)eo?qi=744>WAvT!OhM?h*avqN%4`*q}YS7JV?pi=1C#0lw4#pDL#%&ii9RZ zX+o2sOr5~SIBrvP*Kw-ZSh5cd__DfLMFGFF!9IAX*z$w0~;56Sg zaHj8CSnInEF7$1LTYcBV?Y&bl+~a9@AI(!fd0jjcxJmWw-k7XLtDavAcW^2mxOx#Yz|ODz-of zGlNsuQ9_s%IE@`Cgjt32S&n`d=}b0Uh_wUOvV1WmoNNZ1sEhS{m@UN`h51LDtB0@3 z&+Wh;hA@ZKpV|vF$*%hIyf*dM(tN-ATUb3*oscZstADF3m*=B|91^l}s5*a$_!SK~ zP{UY35jDbG8N+c7?1SBOa&vyF`bDbx4R7P>7)ORfY-@98h6A-M_s|aH7<@SZNhmHn z37cg&A8z-)3;7X-ckus+jGO|!QAVCc8F>z6y zDt+(38sB@c&i6iCI~@A$rh_k7>O zN4_87bKg(!mG8gsEx!MR@4w>vA>YqJ7Q}}>Ao8J)f$#>`co(Fy68_G8LKfD*I-V?) z-MuiKrwC;ihN(OideHtYPqR5JLoO_>`u#fC*^Q36B?^$-4R~@7v|xHd*W@h7Ozs1H zllz)Bj`~G5j=0|j?*x#WPnv$lq$$k1T8`)>k3tNiAt~9sEv6Yz(`}$u>AmS0*{MSj z5WV>u^@g`w;JQ)xD0vp*sKj2(#$HrIM)F(}uAT<2ZgFvm&lFe?T5;7Ou9Lu%+yMUM zCd75J9WLhQ-D7ZBPlV649K=gT)s=jK?G9v1$0vlLPPZ6S3l55tzsmhP^D+zcW5}v$ zB(@>=+rgW>-DEqO&-$&=qdckdfNJ#MsmKurPk9d>yHzzFm@sS|dwEZ0?b9QmCO>Kl z^f(NJ4jmz*50l3LU9X3bCPvOZuiF6mrsL20iRHBRCzf}nc0WsQ_sheMypQF$ck&zH zNPZKv^hNkLn8)57>TFO_GC@u0)%l<(Ej%|?Q3lzh151Z&DN`xvDGwI3f5AL@}- zsAtB3I+(}qHFnB?YmfogcFq9$Jbet%i*)aHFzJn>%Ta?khU^?74&URTjxHDy=41Bp z3G?t}VjDNzwh>2VQo`U&X#;P{eW0a004XUCL0Zaw=$7&bWTrf7a)<{T1}CqMLGU<;izYCh*f60ZQJAaqVE!1684E@^(jDj#XYh2ixK7C6QIbaB ztcKKLaHdW{&P+wlOhe9;B4=hGXJ$fXY6aw^R>IKKDmXHAHjGKFHW^6c4f4kuoEmR% zYBUe1w8fAE>3lJ!I1GhkUL(9g>RfFC+zVQ{*ODmr_);EwL8-IZnR*kbsW*3^ZX;{V zQXU&v?-b}h1p0tI(8vx7U(Qd80XhXh^pPV_OJphaWrX>Pi4WX>$&F#)s+-pfJ}l=A ze+v4S2>mO2=mwXg&>MMde7@ZVUP|l+UK$5?nj`)O9$%rOI5~d&jW=77^a5{MR{Z#z zj--XhM(5l0Bh{uKBe5SxCD4zRI*L^Z^kcOZNh$WDEP;MVB&YD$*nB6WomLBKS}*}w zI!%Xu`k#V+8bUuk0a=pJ&){cvJ_ef27_7xHSl7WAL`-E5KZ~EOqlo#Un#~xTYejMa zj=_Z;m`dvm%@aX#9zVbHesrW8iEp4pW-ZX+7=X`o4}WLGx*cCLie;=AvbLs3{SfQMy357j!WACGw@lJ zcDLy|S+&urYD0vp#T`CUz}5VE@o_aDl<*ti7qTk&X66C;B~3CuslEteSg!G>?KU}m z5k0Q3b?iJVmsl;h$mD>Hd=x=y-2l#X$BtebX~PyJL^YLTyAl?%Td2BSlHJ?Hoe`A2!S#Fi&b+>R5TGM^40R5qW8dyaMn8bk@ep)LdlXXB9)q4~ zk3&}46VNa1DHxpgG~}hd0Atc#gu=9!p)Bncs7!keW~aRcb!qQHW7_+$I_*PPoAwc0 zk@g8(oA!A$rUo000olm!)dysw_{N=r?v%=Ri^Cv;jZ_(w=_wVi<$Hv|Fa|Pt7*kFt z1V3-1@G<&1ck#RRfx8+@$Rru2j^g4&(dp?tKYusho6f_0--)BR@%84#*}oS$$k|w{2a3UNVVVmIr8YO*ihtK^Dqh`pEI|Hk%WTr`@rQ-w&_PIxi%7p zqJj8=B^$s>^V;~q{lH4|>EmehGe4icvX)O@S<9ynqtVZNDyYhrUs;3N1CUA{d~Ng# z)8YfhB6CTYKe3-bNgr(!pDOQ_FM0ix{48BQzT}HqPv_IUntXYWCZB@TA~bM?I({3R zPJnvp(E%9>pPxlh4Ejp=+39>luMr0zGha^|d13zCe&vSBEK+*%d*acW=jt`j)oVV# zU)d(V!W{XY_)Wd|iC2#ACFAI4Zv%H3D#{noU#CxM^<-K6MV-$VqBjHnuAun4gV)~! z$1elA`ZIA1dqHo17WDV`h8%w%7~;={5&nK~jDG+e=N|;+{v257&xacS5IEUC6i)LW z0cZP1z%Pw45Np}L z)b0@b9b>TWPy5Tk;WvNI%Jdy${w87~`RWbN8=9OMn*0RIp?iO&ohDaaHX0Y|6Rby` z@rg2Dp8mzL_E%;kJaZ(ohXL<(n~MN*b;yIF)7GIzKbg6yBRnED=szBk{3k#c|A{7i zgLHhH-Dz`aV5ZK}t~Q@Vv&4Y?L4>>%{Qf!BTX??1&ng7ulIuP!21Mk{shKK0&tZTxexDhsm0{EG~ZmcTcWJCq2@f0E%pStAnZ7uur2sb zdws(rggG;uk}zjCB;>yt&KUZf?dg-!b1Tw!8`8H8S$sP#+jqb)|8`SQX1GD)aD&F- z293jY8Yzzs;c}#Qu)Z;`&>t!?QR_5Q>&S3OY8_-z{g&Z~q1ItX?OjOi-AL_Dq;?ll zdk<23?_Z%-9LI@L>!8mc+nnsZ)CiSEc2qOzqor-?#)5-rwPd#OKk|5+1JDi2YI7)2 zatD#A>Tg4vXfL?^_o0%yALaM~oGlNc;&>SP`1j+ic_d!A)F6GZPCAhp8!p|Xo2$7A zkl76r+)C)tewiZkx-5#;rSUU9dl}yLIRA z*uYpyjs`ya^KsIXb7bz&E$hXq*#{e!8AT#B$tP%}eTMS?1+Lm(#EwzMpyE<(9m1_7;)67PhZY{ZmVHm0b8Cmwi)7kCJK01pswLGi!` z@KKo4twEYNa-^D`dvD)9ea^kN=AVE6`WwK@*p6UK;8LS)?lo$4v*Fl>(lK-GMs3$R zl>1$~<31@6(n!$QFq9sy!(ig@%AA2pEoRNvkIt z+byff(8mnj$$nTZ2(xmL%mSgivY`0C{3gclY{+-B1Md4UrhpDktY zCY$bnpr%)(uiFM1lv`cjKl3}$7)MTD-IR! zx{lpxuIXL(^L_3{*Xu32W4oIIfmFJ}<6GT^j1lKq0}(_85=FZsO9$C$C;tF}$i!e&gZ_oFY! z^4vAa-7fbhY+lg_I8`7p=BIVOph)oQ>Fj;R_x(CApkO^9?h0Jb?3CQcx^I;c?HxY0 z|FU&j@5m2yo;b=@uZX`$B5VzPOW23eDu@wV8L1{q(1{(DOM->fBhF0%&s4f=I;&8p z)Ant)o?bb#2uSmc9& zZL_@sWK~)~9MuHza3ey%5ii~&LSmI-sHfNPQL0zi=)w2JgZ~`%EU{eRO5e$I;8~~) zoR=A7i_dTCudb~Cp(cX0dvOfYqP(0*vEAN#(J7M9I$Nw^{?x1@F;!e}c_P+>rWLPW$Q zL}T*J&^0-`W}cvj;gCACZF}leB>7&2ASvW9A?UH{J(r6*HLC_Xi5Rh4)pei0eVi>V zIkRAPzc&rHJ>E|VcYS-X^MU_H0)V)L`O1MUnM$kLo_ES4#+J$(1KSw&c>cCk6>(g~ zf9m)TTKS8LfrblzcI~b1Ed_CIsOaoE)i*F#2geGRU&psX%G^N1E!kwu=&3XnngCEB zYEy*wAC;I;ULZ;=y+H8=f_Xvb^wdt240Mlcf?9yq}M>s8lv zx>=})9JMVr3T}j?HV<T<1|$BJG0 zCrdcyAXW{fv!1W(?C$|!8}ttyU&48cR{;F|eVcc%Y^#!Am!T3#5Ri`cwtII3|i_KcsIK7-r z$YU50aIfSR30jFzx|2bYl{QK>1E0$3>!8tri^Tb|hl@1qh_O$>dg$TODoYV=H`b8r zVAy9~C3+`bls5Jc5q&`0Zh(i{0;EOFQALI{XF3cr#O~Q=J^LD(NH0nA%i7j74|Lse zRcahUZNwgmF12-^?>K)z>vvGS*jD3YS*b)mGNjixMpJ43wKHB^@KUvD`N= zMYyjFbu|qPly!Cc8@u~l0J%1$%A;B6C+@N|muEAzwbhyNcGXo{Dw-{g0S==>F*s^> zn=cibbfn0m)K&iVRX>G-JbA*(ix2KS&CM>?*b^-=Pq_Z}E72bI#J3rA=&zi|>fV); zXAr=#@R^A0aAJ!atV;_{^vpz<5K32Ypb#PiZF6i!NXM$SUBIEnQ3#gIKt6)oK}Ae) zI%il)ASG3+Mgw53ZmGbOS^l#Sav*s+HE@}B?Q`r0X08qcL8=F8HsJ77p{lEQwZF=m z8$th(;~FbLRi`dB@R=6vbDRfkuF0@)nbxoRW&?CB?$N*}rxqDFOxt$)4npCZI*tQ) zE$)ASU1!%;A!b;0*RhnW+XKvmD9pN=UQ9vjNljiT$KHE^)eJANtXAM5;+V$w%?6xW z++%@Bg~uP%e+nIqC2d0)>Ew~J3Y1DXogz4|D?Dpdqw62vEM!ypF)qtR7zW^TjAJ1> zSh0NgK9~*=He!NWrIs>3FhqAj43vdake50jQLA8Z&y|n^E0TLUhUg z76pqZ1We?*PXt*;@B8-z3spMjvoSb$At2b^^C`4N?;Y5asbrcp)Uh3adm&SkTdqpz zmORmo^DikU|84THBQJVy2F5O|L6c1)u)4Mz6p=Cetg6@UCEv!k1o=Uc+ zi}P2=5pVn&4TxbygrLTg__v@`s~lZLy+`3u{Y*fxJnY&vtF-3IWnx5onzeJqvD8+j zHz*;w<&J!BaOiaVi%g1RmgI=2Z;Cbg+X!(ScLJ#V>IW^QmJN!DL4P0u#ruv7WO0}h zDh)$?s?h5)`%MIr?`7X*LRI14ael&+A$-mY8UGMjphaiXd>!A2rRC}ohLQJyhx+9% z=(0%=F>tVx>~%DjWB6tf z(k6VaR{<|xg02y#5Rue241F>EmVYGani9nytmGb8AxbwM=(xv6XvBLlfw3udyredd z)*69ugsDq5ZPeo+YLXz=sbmV(i)63We}dzP`tRDMR}9M`&6UnTaf*Qo$1}y(yn`Ph zhQOs!CotU^dzv!87yllH=Lq@y#{uED*NAQ?em+eV8c*Rbo570V0S2-Ph*@ht*I~1- zoiH-Zf{acerVoi$$Q(T1lU~!$;AlY!jVFaXqzEwv3Ks}#r(@p$YaaFY7kDE!@lXJjRpe_ z#}O{izG?JUGVeZc?E8YZ4nvUfS1jTJf;WDa$(%!)E4R+BV-xMl&Y}fQLt*8{lz2bv zSl$%nwgU776SLZU$&^qI0$}Hx4m$biCt{U%DCZsKZnXbgViG92V;D$-d<4 zE`YtoADPw|U`!_4o;r~Vm(^`0@!H!79|N?361tkaV*W{bQnbc_W|h`^+1Qbsd#K#L zVq`gH8s`uJ-BQ@X);_doJXZ}BDQX;AN9LG^tRFt9=u#+iwT63*XC0@Dk}j=}nTVtf zZp)I1C^4-@Or0+&IIH7CGfZZ)^1$8fckOrKU~Mcc80K6v7!sC}fTe4liR&t;MoN2W zdIOOwB|M{y!K{@w6QuiD(H^v(xNv7$1K%ET9YJ#gF;(riHtrv!TkZ+fF)vxEXVn-B zh3;9ycv%-MwH&{jfn}H129-cD!N9t4g2sZ}G+CCCuvVN{=6D6~LGC{-_Zi`w@sZU; zh(!l>w>&_IbB*&3MX$NBj-zu^lQ;q`LKtY)XuFx1T)kIb{$Ks_>@!AihhlW@8F4n^ zE~2r2fRwKjDyglu6q-cnTw8?hicocbAEQpm^I`v8HL{g=mE&*wS>wPCZjy)iM=g9t zF*ORxI_AK+f#DH}E=>{X4H|>Tx=&33p3LJ)McnY`gUfsEmLG)G&6jMnVaXXe1j!$lqn7Uc#|)5tAs- zXT^v&iVa-F_A(?np}k#+>(cfz{~~#du5B8O1K$S?<}+JjMdm$vEDyprZWK%I$MG5) z++0S9;yl~j*r$npTDe}cs5gr25Jc|4cI_Dhla5m2j&lQ8f`XJgrLi2|DVw6@Xop0y z-*GC|8mP>hnHM8$NI*8qYPtaTnuyoz6b;H1V#yYvjJo9r9ouBe6~Z?kpQDXJ*Z|@v z25-_}6PI%-<&qDZzKS=CLGd*?C`Q(NRvB8>n^h-8fS5f+-=m{h!{-rmS@|65m5v58 z`Qr1>V8HA`G4!<@AP;>P1%GF$a8eo?LZ?AyaRYtHy~nh1KRLVe`y8`Jesyp_XA8g| z`khbr2PL`Pnvkx1PZ0Zh9=jUt0q8M;P@d>)^Vn zq)UvYEy&-zu>smPs~(9=Ep~CU-}xe?i0wc()L#-fM=o>G$V=DHpUbp=iA!=)E|a<0 z2uJhK+R2mH#TB?;g2id}fJd=}ti6ib7z%<1mJG*qFY;}*b1|>SmbCI3;%=N-TFm1=_n=7b=sD)ggb*mAUQ&RK??( zeWpERkdCJH3?4Yqfzsqla5>I}7&4P`?=Gvur6t%F*jzEV8wt^Tkl zI^=#mW8=hX?QJjOIE!m*^$ubm_8eZ+1wlH<9qWA!b{aOTg=Jg zaOQVz+4pEKu1_ih{xS|N6O!V6?o+zlLq{XHL(q-_CK zJt2H%6l0lK?B!{OQp+V>Y1;MGA21#jy+O zs{1;jhDd~3v4?==4{@qe)~oc15Wzp6)*IvyhLW40@_!yMU-d2Sr>n9>TZ@`Opi!0m zdIuZGV5PzImx8eQPr>GgV`B$TK8IRAr;OCJ7W!?mp6OY~C7 zbRtBnSQlI0`+oRi$Z8r-(94Fl(AY{Z-5$Q-egXd!PEJ)LWnp~$xwP}+4HP>xh2UuQm-H%s^**rmpRUC#=d0x{K3 zB$tkJx01QGIdp|plqt;+I-}Z9v$L3D_zOe0AIl%& zP}Y7(?4KI>S~@Kd9j99ID4{_1_x;-DFs- z)5_KyLclLkO^EdD%x^(mo85|L>(+X2-D|J?iVLgESI1yKakZHzt&i>>HcsB~Kuahj zxI9`49%=E*grkhmL_#qxtAj7}C*Sa_nQ8_^W=jZ=%HK(wE~xP-nX3sB%WvsASJ(_< zHkK6bW+;pXH%f$nOexl|oCIkI_6YyZSCM_3e^0e-&fE3u*}~ZJ=eAJ9X!=Gx8CNBE zZhn%*L1K}^bq?)%W=h7d^73Zn_-JIQ6SffKlP<{=EaC~oF2+#$=FBye8gU=FSXNPd z%oA}UD68lY%L$AQZR1Kt8pJSg%2W_J3LO+C1aQTI3d4$`@XT<0$Mq$9OyNB%+(SsD zRSR2&1}4nn(U&=DKl|qCde_q;Eo*D-kZ#IhYt~daDgb7&CMx4XiLHfp66;(VAO(zq zM6@G#Wf=^^C&^>%WCjJHP|H1q;xlkJ^@`~UnlBza5y<&b%w>D;`PnX#qM$sKB9JA_7Fv z8ejRs{bodEp*`BQ&FZLLsSvVxtpFnr{0!?edy!LWR&-SW>%K}QEHLX^YYG#3od+6! z7S27pq~(PPG6S0dCKMH}!aq7@F?{2jbh;S2(Xc<=890Y>muvo+Uy}+Obm_Pz;90;~ z^u$|jeMTdSmbPrBWyEvs=PK)K7E9uL8L(j?7<4)#F)_|wpvfgolJS#$h~?t5K^wJY zni3eBZ`qf$p%#;F%1i2$Wt(p5XsMl}T_lCjGs=(n4bUql8hf2nNOfsr#~k4^Z`?&r z>QT3x;FUA7=Miv=6kEz9H=rXUyOq!hWfi8}n8?WZ*#lLfK7{5`tFP9^zCan|zRm*N z6K_@jEm5cW$XV&0Kq|?SM9mn1RYFN9U20Kan5U!b76D**1IM{Qsj#uKu5DkJZxc#C ztBwd+$_4mD7bs<&C^nM{GL5VrMppK&Wr$n&Ek5`DXR``TFEI%v4iH<6u1&~}DB6lK z-KCe7qxm3EO zKT(mKvy>wnwfWkL(V0JCVC2~b0zbq;+$VtZ=vExz?TqEd)_rwwERTP);xajZKWTeX#V!=IYt zyb7*8m`?#t#pq-6xk&l_yF(j7>5=z%i7JM*90mgM&@72@<>yI#T3Aol(uCvEm=7-y zpa|fEv0pW+aJ~hhaSxnb4hYV^H)$Wv`(DQ-UqQdSg6Dh$7=7uXc99OiF)Oai`9vB6 z+!QmYGXg)Pu)rK`(v8R$KAh1Oui>BLP5@Po*b(+uG86INO;?;uiE3U%f)-OzMw4r5-@~w;Z*;R8g={QScef za|l8auKoqhZRjO`NTG8vHPp~?j~e=eOlgTZXffy@godNVxC2PAnPlD&~IeQ25MiTi%Y*h<3{z~|1?T4h=&N*@lp znhIcfVe6F1gT*~S*L%ruV(o#!1+Z^`Gz|0KL+sFi3!xx63^Nkx<{)8E=#s_hwM+4Z z+a(*(Y;+?KOBqA<^%-Am}cQGa1ijhtD@`~+YALl^mK3nS5VF%1D17d)OKRK{NfB01(y0|fw5 zI=WIAdSy-P@m}yNql$4|M}Oi(5m2R*?{k+TD(~{yRuRiqiBY_JfVO;fa>$ z<@>D^j?_C`n&aG7Qvyz#Jza<~C1iJgKy>TcWKMX4V%U0}{`RKD@?<5CP_0_5L{5xw z5s83YJi-a%D}t8-0aIUtCneI81cC&%0RuRQZV)SgSZ0uu`ZrT)YbUk2J|>9IMH8{U zzKz)gqRT#!Ywg&q?BH0;?3PlrUsDVVy)tF%N3_GXffSwLem>Crq%rEq_Uv}EnyN}#rLjgjOsMQ0%;(#pR7_CJ!T81EoUf$A1cprHI`ShZ$CglirALjIof?*t zbXbjma-t_+|AW_0$!~N`RBy>7lWpYCtH2R*@y1xRH4yb_CC#MmKD}aAv8E-?At(x2 z$*=94np{)JY7^z}1*rVd{wyqo^ZV@>#VJo57=*?mjQvIsCvHlCz5T=+B}I>{GpirF zfBn^Ja+x!&{SN}$hZ2ceU_5y`^&D8@L#>zk;=>cW?< z_26^WTg0VajomtmAVoclaw`22DmKL7lF2G}&1m%F@|uvBSRiU|Os22j1}(0x?M{2W zQA>4Ot*%XGH61$yKOFZ!bTD`s_gvjY%*?>X17h_0@FALbji~;@Dj2^J?Y$BAy$!3h z^GE+X+vvf^6?|(_^(Xc}$L^yRM^CkvPdPs_EnHiv|OKrVxHMd^@1LR&31r#=f;6900v0^-A%FQ*2R8sl67?xq(zhpXj$ATgdO>@{#UdTNeci zFx}i{kE(P;1(W|X%UbpnA}@tvH@Y!eE->w-F(naBj+K4~jW6nFqp*4TyaX2pzJtRNA1G!;IrEEAL^s1Ov&S*^b4>QhuUkLjjhJs|1!3g(? zGkA|4xS*#n?>g}eyw>bL3h7;(?X-GVq0XI@_rCHGT3&wN4Mh|qvlOXEK%sM-;5!x5 z`*~^EzpeTc4!te52TJYwQF3Ml=Py6JNs_Gmoe>F&kg z>~;H4)|x1K-0OpvsL3IMxElAA2dr&7gk8G7R6FE!F^FGx>pRxmlPEbNoF;sbUuF{r z6(R_Xp~ikRaVN_~Ve>)GeiU-YEeB=>2&x};RA;V60tK+EJt*n-AB`cJ!k~sg^gTEs ze#~z7IGsU^!R|c>48vWTfE{?QH=kTz+PU2j*SoQups$Uc{q6`EEJu`NTiLMO8!JL`aI)QlN?ZL4Bh5}WI34TgQ`|96ze9!j6j&BkNFgHu zR}9F5HX}%rK8i7x@5UgU0vU#?BBn1{I2cH%KXD3?hF$VTG{dh}lweZ{?_u!(wf28q z=!>SAi*&OSuC6n*>u^XlWw;0|$jxFl>sUPB*zto|0^ON5t5{_>%x!t%n>$q%OlC~X zd`Ji7O!zFy==|mf4p@SM*7lseOzwY)tVqDV#eG%Ib82ZG7C47t{Qv3Vs-E+P zf6oLf&zPW*!=2XWF!k;(^3yV7yJJ%w!i>DgNMUO7GTC7HZV6{t=rl78B;>RA%}$y9 z1s^R>+C9zu!XF`{a|fW-g)dXl^l**kOY75Y>XmIW30xIcu4|v#$OuUM@eINxByUF^bMh>{n8COdAj!=0ngtVmyJ)C{*MHEfJFe z0DFMRB{K+Rd5}m^QfT7l3hy<-{)?Gtn%=F_l1a9WE?Wg2-B6hLjx!b$C|dQ7pdi96eZW3BW{V;*PGQi` zhhWfz){_P&S@U%1oZiLko<*E6-Q$sKUazQc(R0Moz>8Ih5j&-pe=iUl5jK@{am(LW za7aVT_;&adUq;QQZV^(RCfd#d`IzR-q$kZ`L~56v(Ab){c(YHiJ>Stvx=may+cYqh z)w%C-UQruFXKs!XvdIzRX#ASe9C|L4()n;SKYPioj%$YZ&@RV3FBy#dZ%5t5cg?}! z_qG+}8fCddc>hr3y1Uq?*C-s^GvJLI*%I;Jun4pE;ICt-W~?ik3Y#NB_P(INs6bQ` zVoiszW|j=0$MB!kQwu`JFxYWrij4n!E;0-T-W{!}V|<^_XwBAjr9=1U$29w2r&P5|&1HeZ4F=mz?<*{Z=D5_1~IG(^WN@zSkQeTn5mZAg^lxHxd>-~oatRps{X!1ZS92%%3jDK zhsy8w%IYrc==X5N79br_dOWBi`R%~&F&EM84Ob8}%3$FxzRHQ}>gpx^*uVJ9!Nk1n zf9zt8np@{%3DPs#OJ*_Peg;<`f<6WfA_AK^5NwANB)=1g#`XpA&*hg>Zb_kJ2@X(G zf$IvZ--fZBFra1mD#;NBiJTB)Rq?@*PmwYSZvv;L=`*1hv@jc1s=NxdJ#wv3D-N-k z3%=i9Xd(y+dBcQEEjQ|TLolc#4DwI4+*1kg{y94!Kk(~7(OU`gLEI%8tx z!&$x1%js!q!UOz>X_{m02aluIj?QnnDvqNb#M4=_d6l$gz2F<%E2SKypx4s^d9!Hx zz+9_MwR={~)jCQHxhD`_v`#^$V-pe~QO_k!zTmd+EWIAz?#=W0zvygMbibitWmjdD zaD|nbi&ZoSg^b{BdWdT~Dkoy7a&1UT`Wr^0P}aJRa1$=pN|2XmlEg8&u%}2;5Qm z3V+Z%Dm#XcE-^ML9KtJB{~k2@aQ2k$!8%sG4f4{QdR>Cv7kwh(sdo%xUXsRDC9;U` zPJLGW)ynVAL9fiK#k#?l7Hx48a<*RBwCckxQHIE>4f6cYCg2RtM>f9>u7{}mPEeJF z3&7X|uPCtCL*hJBEsDhhzTl85|m8;A%(66aHtL2S4KRYE!a6z z303BI6!CoZhfb|xF6n&}4gW#ts+Xg3G9Ombymc<}3GUAXQO^+dSq)YGBA1ltDi6NQ zM)23G5Dr|Qxtph=9UYqep#24x_7O2nF+)!oNhIS&lwOJ^V`UREvUW&L}{UuiG zAW`ceR?9)Eo`qV)2(!5g9H4@w3Fuk)cbc}a{Sp;QT5LL>U-Twomz*;}CjjO!a{MBX zmMqb2Zp~8QaU*JRCux>1MIf}FXnGzapA5z$ zNnWXs2H(%GJ-ujG$qs9yNweY@0`P%t*6R-suJ<%bo31Tg&9vQ8!rc6SLtDvIc8yDo ztOL3qwwdmoUFp2r9Pypk2|Qc-oci{VkWQ%o>-BkvJ`)#tBb;J|yh`X7TCe^sd55r2 z{vG!CuYgR-iwN$y$Pe-_7N^S~5H#FoxEwT`gY@sC6rvHrk(Nl5O39Wbz>Jmb-Ha6O z@%ltE-FxP(|B3%AR-tkiqKg1qSZfzc(ulFkAzLBuqa5jKlJ5)< zSjScw8<-PT$QR*;4?O$gxe{EWbo_}|-uVL^t7kZ*vaX5b@=aS+yXru?&Cp4g&oCsE zgh!T*(v7eWI;yP^KNtUyuf*>gmhZAq$E?IoF8#LiO5k8@(D0v+F_~W@0DDFjj`nG_ zj)}{;uxXQT#mki1zTFKsH$5>cCb(Q4>)j({voJR*VV4tgVx^g6jG_E^g74VtZSGE> zF@YCltQc+<+{zltj*y8B6vF1aQ0WR$w9O5x%v_`?9sHw~79GI~f(|78Sdi-MKWW>7Y|Y8|?>59YH9p!m6+YSp^<*yx z98BvXlNfHS5(Z^*_eexJnS#wHYcv5wH%gYI>Q9DMYC?D{`1NlNPOF^O|U_J@+@U)TKo&miwO| z-x-GF*W_`F?Sq8hP%3U2L?UO}Tpvak)x?P=ibbmP7o;CbfjNJf-@b z!AM>r8IV0gXPeA<10W}&s!s!+v{hWGe02b%4xwj@#cC~G^lv-Jnhh;LrH-I}Z7w*6 zjd7=reL(cxPs~=~aNQIG(9^#-f<3AYuyT6;++kz{=IvPp5S$2^Mb zdCiz>YhBPCZ)JT+);_9QPf(f?FCrU#P@0`&G=gY5Mli4rTA2^KiYq+LmQH4&K9p)} ztZ}yo^vnuFq{|IWj| z+YsKizsZMh0}3|=8+G=wrF@m;&h*|>G0Jco^rzCQDe&9ka3>QJgDw(a!|f8=ra(#Ho`z|Aj`((S$mA zk|;1=O*L2Wj#)LMH6(JEB~q7YQVDnOW1s02gfcA_%eJWhIZFBg9<`A8(ej(V6o6I_ap_* z+9!4~HuxMM#zJ<=xa1WX=OZz1UROL;y@LDPtDv5gJZ&w3D}3pBd?v|9HI%`y0_2e! zvPSw(AOkppQzu#{s=V)DsK(4tU1s>t9D&YcQn4TV%H zR1!1+VsJAAr9&v{v>rC-R@L+O34YSagVz|fS;hKP;yDn)$Q{m@FFt=i_Ei?C>EV!q zRu?J&E78FzxD(4jY&8%TLkrIv2v*CIA5>e@f9|c)_jPv4?^5J zD91tumA|gWl5n1At>#0(?X$dp*~~8>U$4V!PT>KqXA<${3oU~>ZSl3G#ovvUi{9;Cn@~8GSn7=v6gDL7wsh0`glLhK1k2T`sCXh4rD>T+jvw3~- zfWA|&T)CPTF2uABJ9z-4Rw7oCe(crq$?b3g%iF#I!%bMX{i*pN&vo?`h2*=#bn;RT z9<@pWnl4_N$kpvd({T|O1DvzFo_wHL2`AALZt{nKSpX|bAUeyh9#(zF5EegJnGdhr z+OSfQWYUGPR@l5wFX-ILx=?Q)hR!oAQLP~q@;8dv7tecg0JmO+Au#8q<{`Wg5wm&Fe!+xflC}ih!kZO^_KG>LYWhZsS#c&jC2B^_y5$Hu0_9VM7SjtWcED&lT83%8pC*da_aP*# zNv|kBrCIpCsZRzMTCfSRoNJOxi+Vqy_CFvj9th6QVJ4|SGzn9ZY$4hzBb0u9`g%7h zZiOZlgbq3G`pfoI>t$Z$AQNe;F8hwB_}Rt}3}=X~j`~koZ}fFLx5t@JgNd~_;rc#7 z>)l`vd+U(i+qhe?{&DZN(1W!CVI4ddB7u_r@m7r-tGe=>#&50@H7UVdHp7lH!@Ki; zBk)yjKH%I-)4g;b>&9eSoZQi!oy6)fdroK%IivIlpUOEgqq+&Hfg}z$!QfL`_B0rG zBp7!uhl7s82RV%!=W1MNIVw%>siZ_X_`?iM)aXfw$VrJl#Ak4<*C-vlk>s|j>z}yw z?xmQe?OL^6!;7D_^5U1QlBMliEa`lk1yk9K_>}w9w<+dkGAnoKv2<-e33h4T^Tf;D zR*lLs^`4QBWdDq^3gK=3j_^|lIu{3mVt;xj<)r%wJEVPQ5dJsS>DvPNSN}Zqk*L8w z3hZKONl4!HZ3yCnGw;>9ep6NIeOa*6=I&7mv71i5NLEL19G0r030r|gH)CkGl@5&g zFD4S?)OFsXAlB_bVYiwP1P_Z9ugja3`Zs&kbexmksqr%i1sw7fjy$z|KzjW@_n~Zi zCOw1w3V|mGv2-u8u|6t}JE4Q9G+K7+BOzH@a8i4aXGLD*u?qRivg6iax_4M^!|PAM z|2ezQC3WR<0SpB6@E>_Y`M*#%-VUZ#E{+b?KIUpJUjG-p30IzWSQbL#KiW_@?DSrf zFGv%zPL&!FX#{l_4OgQM7gu@>pw6T6ARDiP*&l(3h8YP#JSahW+oC`jL}}@Ko5}LM zwGH_B`~e+cg_}|W^9it;_g`=M*4l$;@<;x>Jr+IIAD@;AfZH4QG_M4i?Q*@^fThgd zyxZ|UAP3PcYLN+fAL@0nSVBe00BMUO{qqby^;7WQJ_s!PKU~s8h5aeXa&zD`-?9Rs zRY9LV|4o|6SY=igQ7c^{ma9Xx%3Cd9+viYOT8)d5i~gmk#@^yI7Q(cOGe_&`h1Rzj zamG}u=Qr?~Gr1f|x#O;)Sa%`6_xjq0?sb5lh(rzq@?o=x4p0mo;w3J^t?v^ERC>;J6Tc`bA`{~kp%UW-!!FJjDUr-Sl_PJQ zYFWnd6|z?0uB+Ys-I|*PetdGC0Xv!}|KMEoUaDkgAEO8}`d{qIKNKqY=}_%$qb0(O zT3W09a{kY3(GT1eE5`%^disy{!2kcx7Bz1t^Zy6W(t-9{8GQ+K3}}%|k$>FIJx(Da zf;IvH9x@i<+$6W#xE?b@1ul}6!hnhhI~5Wdg}u{722UhPKSbX40nT3ytZUbySAVFN zr6%Va*m2|v`d{t=i5|1{ z<4m(O>okQXk(E5XFlFiT@G-WkwevoRp}gycCbF_D53#6Cj-H0_ktGotn8+%!p}(k? zaVdq`J`yzh8W>7w%j$u(d1^%pkEZIzGM0lwRM_{609dW4)C0$-wHMiWRmIJ*tZ5pw zX&Uwz)l>cJ6l553tGQxZ#U!S+6`XgD?NGFP%mmf)MNsOL#S)GC-etDbj>NfS<2oBB zM>q~U3lBaE`^tG0^gD@+*Y!qIXscoozY6O-sU&cGN(hExNs>Agaf(BW(!}{^u8PJD zB5dvGSlIVq_)lTlrpZ+HC4X=b%9bfAy2pb2h|`H0vGW*5+rU)~a!6@r;o=!IlyGxn z0g{*K>?&{pQrXCu+VUt=Fl~F$r4rS_6IeJMJBL(~5Y&WD62W#e7BcE((@FKW zu;P&ju?uPGs9Z{t#mMRh#>B8W${&-+C-!zPkmT?ek^C3bF3p5_1+At%OlU@w=G={q zjr24=&UURvZlbyr%$8b8Do<7xhX5195mPvtgjp03Vk~rCy^;osm2>psly_zf4k^&I zF*aym_*~ksBU4{{ggL|rg;a=oGHp?zghw_GpJR?s)aLA{?EMl)eCxe*imVL1^q=zr0H zM^*Sd7E|)ah(wtaAiJ-#H$=%%OAj7Rt0*Cfak$fb=591dfM z=a@$NaGr!!j=q#3Yu@OHOxQj{+ia4PC6SxMS9BGh#0H90eR0nNY9A>K<5O^Rs-GgQ zsJEK-X-Q_8TlBij>_AmonXY5CH4e(Xi^lGi^oV=}OgkfBY^ zbV?kCVZ=lj7$?!C#^YoUm+qWZac+x$#+&>5#AmK!Rv;6wneoHCSkQTytf5rbKS9$o zEGL+mO>jjcJ+dHRdhPSiq&g-RGbK=va0EJx$AN{7k=GSw8N zbXSU0P=5|lw`Z|KNC8Tv0%&maC96mHG6yo9X-aA45hi;*N>51sGJ1Cnju>$t(vaLj zROjq>6AMYaU^(hvt(4`Y{7u^I9P=dD0d8`|vR7E?D*0)~D5L(=yJT!VlYFWlsoP{R z-f1n>buA?|C5{N@a^z+7P(9^;`Lp5rBP;OlSCX#ltosgqXAPV&e_@}2XOm7l8f@IIID8$IV4E^ zpr*`+{R3{ub+9kWsE^=2m~oFjS<-*)cys;IaDI6kz1I2~SI{a zKL)+O)CHyX4=3-QaI5#Y1htQn7q)p!m5~y!m`ox%uJQy861u%U0fv`sZrv7-Zs#U` zKSO7hDUf2fqrZl6tEIh-bzyhQAybxymB4*Dh{8DoZ95@((Um8Si9{aj;wr&ya<%2+ zFu+HRQpG{8ijlm(Yi;iGCt>c@DiJkRPPj%(tR=;))f9#qPhZe<5QCXyHvB@0yQbHD zHrq;*R|MeXt2Di!L1jCt!ptlySBmyL0#N&{1z~1g4<|J$XjJQ1ycqFQ)uTx#?ih&< zFTdB4b50c-&r+m5B$}j>UPX2N7LM^Gr@cUpjP}(3*-SiTtBg z=|)0cLoLy8V7rO9#40tg=wGY z=Ap`x;|LXvhb$e&Kq6w2rw{k&GQF+6P2AUax_7lN zKmV2hGe53EiSAZ+bJ@RKI~dtg2T#gdV(vaiS_07r9NfRDP;_u#qhd%m&zQzHFUIxN zz>)60ifeV-?76d~X8pO2a{}cS;nmY4rI**I=FCG$D?=6T28}N@r~9ApR&lXpDJe}z z1~lpb)gynhL2Wo+V{l%Qs9$bKs#7y`1(E_gfDKF75E}ZLBG-CIn=Ri!f@He9AKIIs z2n3ZOjO?Kr#n4bm&R$L+=Kg_#FZ&(~l|c3#43$8#b0ka_I*}6q9Y&dbA00L=*O_}y zhMF(eITTibly5V%45gWU?>59L=S2a~rtm`n(4gkadnG^@$i1VXz9Q9g0C0v3P+yVr zZHL;z&axT{obsUODe;&9K10H&2*?gh099ztY$sGeBQz(f12bS5ijfHr9)_89e-Vb6 zbB{9A&jf%1lu_)+cx6F%AUkjZr~oGvJt6lpsJBEtQ(^N&JJv%r(4N}w%l3n|bbg;@ z&~He$ZTBVfOZ$!&hPHsCEDVAAmN-L4z&r%1Kw-@wov zEd5W@WEdlfU(+ZdW7Ec+O6)g7OYX1BbU=OM#*xhQr>6GTlg;!;ebe=?PHdoo<-`0H z{cHE&VL=Z4k9WzaV29S%6Gd#`p54QI9memD#lyTV-H&(8sGwuZ``HbK0G!sB`(NmN zVXsK&ed%{iik{T_2Ku*_X_r7x?b|np>F<5hi~e<%z=h&pf$8txF_+)|ww>QQy0^go zlOMvyo!`*(e}NM&zlgfGfgR?*zI6ZgZ8v{O)4v6I%zihY5&Yo@C?`?LvP4)OD9su9 zK8W~C1CjE34H3RjSFIn>^~4)Sk;qyQ59XUCztX?_^n|(F(jG(XzX-gDnO7OYNl(x+ zn1Kw!GAiXm`iBJAy<4w=7g%!pTjcNw4e6nOIA+2`uEIXbp&5s1ZHB7IkbA*Dx=n$a zAKm`}m7ZM90Xv=Yc7(NjDN2s9v5e%Qcz# z^xbYDOo9t4TW30Po>2QzDSC@lPcxSfT&d)F`H1c*q4Z#D;;F^3KDzt!mSS|eZ*;nA z?JBDL^PTd}=eU>)>PWkd>vZ_^Q0u0o&e?d{HN@Y}T(I1D&|Xfyx%d17+6#M!-J;DC zPr(WRvsd`(T}deVv71svRQ45YiyZZ?;upCwT_GrYwYCXVx=F0&r&wqC=^>H^zYp&g z4rX7Cq1~kX05!vJqAY9)I(~vw%T*Kj!^?4crPf#E!t>{@imsew%|Jx9^ojd#;ktr) zM!xjR6iD@!*bA&O{Yq+;gNq9KN`*4NBB8K*n_oQ&X*n!7y35AMP+d|nB|1adnTh_) zB&BqAzIrF-9_A2QS6H2qg7VM^FBLXeEqFwL7ZhkC?iBzTZqRfCNYR9pFJAs0E+1n4 z5agYeAq4Z@%?q0J8M+MKD72 z3+;V2-&HJFu>~;IhY~vRcU@>tJlK7X?MM}bo-LgLJ~Bfw52$v;2z4<5Ts>&A4>I20 zNAa`~fmm$C6^P{VBdJ7a$*8gbQlF(LJi_rwX4T~3k_qOkz@#H?@r3`-CrC7I1S%Il zM<>tXsZ6j)NDo`&qH9oG2DfRP%OZa|hz79f`EAJgWc0);R`e z8hmL#wr$(CZQHh!i6*vf+nE>>=ZS6Wi7~M~?`-||!|v8w)zx)>?YmEPbzkQ?ztcik zFvNfM=Ss;NHBi!H1G3>@=az35+3NlG!gd5nZu> zBmB2s{FwNs}lz0X&%pJQ%pYDA;TzSNNtnqA!rnPQ7}Ef}9?B z%*F<8RBO``uE*WhF)pcT>U@t0y|-uyRtxrQL?P+$Zix*><4xQH^;6V6Wa$ zzVcmhLU9xNiS1Y3UG;q$W=(1?Hfr6y|4=U>oG;OE9$B?pNnG*nL#-L;q4Ja#5OyJL z>-`-T2hQP#-$`ZRM}ErdxXfu%X_w11>co{6P?;8!@bIJfoT2ov=z>SCB;9GOwgqxy zWU=y3%9Mz0p7;jpy&y++yB zd{irsqU7e9X4%*DzfXC|&Hh@MpEAFfy&~j`XIyAE;GkzUe`%H(Ox6F_pqKnTPkZt7 zQSSRiq>kFIA-V3BBh}zWpm;S^;~)P1Ep=6E7Pt(@%~STGeH9|Kqf2SD>{O$@S#7su zsKLCTUM-|pTBABw##i+3kkVEzdB+e8ztHt%TT%Z3y<*vwrB_lY`*i2aw-Vl^^dmv# zdX>FOsY%sxjlLERO~Ma^YSEmYWfiKS`S_MkAJn-C>6Kp}@}@=UHK{&AS%=$ibakL# zhuLr99cbT<)n{%EcySOUFvADZ@Ixmw#gDK1*qEP|raA&JJW(r}v^h)L(`v_b?E$g0nGoC=&OQkwf{)MrC z_=oi!iOUbPH)2XmX^LnZ$=_?1!!xA>G#Tg(s&B@YeE}9AlMl3%M9k_3tZafpN)?mj z_tlfrUXvbgjWU^ztJ_@CNg;wTzb{E~#)u-|+Mz3&r4`@I%O}MjfMedzy7g>ZM}FJ{ z>u2jG?Y_-triCh>uhVHWROrc4ay6{ki6yVZsILCj_&0Dk&b5mXiT!stmghw6S>KX zl`8+(Nh#wh>P&AG@mNmxhE(^u0`1v!KV~<~?sy?zaAao!p19B_BDH%*GA5TmVmjn; z)A<10>fI5O>5rc%ZOAI!H>|D9(-Nmd3QXJa)!Ape@!8$1)>wPtf8zRj_TRDa{UL{* z^35JG5!bj!&+vH-3YX6CF>mk{{UtM0Z;}5bj8VN8v|Za7l-v;68{3xL*xB!e?Q0p? z3TDcR4{UPRMWEMJ5?g5yDf`987J|2HX5}IZsedlEvEpj zW*}eet8bGkhGKK@pKWz-l_&ka>*EJJ7?Igq#okBh0BrR*_HC*p zXg?guu9uaDyIW;nL%48_7b_0dHW4Pas-ci6^+y6D-XWCJdS(W_niTHagV@^Kvb{EL z>GLE41_q;3%~?a5sSaBLr29nIK}XgE$n8;aV;QWg`{1kdl#dDMPSt)#O;TzwmAZxF zxg)0ksoEB?q)*qcGLSr$qT{Y+ZF0+v;@G~|ZW(VKB)zjq9`A15XjI~dxdRoMYL5^XSJ*A+DA_4{lVNd^=@$pmt@k)=+V%A*0;RMTg4CUSaJ}WAo@|HE`WtHZy5RxgO-iZjv$pZ z_=_aeP!Xy-0ihjtcA-6KfotgZa)I>K+p>|!6_q2sLCFqWrY*QdC!s}WzC~yExXVk* zegHVP?aPqX6=|pBmblbUjO+P=ApW*f1yeukPw2#a9?kGOTOC>E>T{BNgkfAj)+ZYo zGv!5{R&A?&I#T^2ZT+NNx~e4CV)-PLWXsGXN`Lu1Pdain0m$1vb2B=DsUL_*cyjqX zHh0mu%xqC}q=&mE%}-OwSvmg9Ki7cXuHHo_b)eCXs;w7|OSjTc70#KSAaDDIp?7gX z3W$B|;}PgMEt)f_p|YF@18q&K^%CC(SFo6`Uir?iu9~Muxt|J89ERMC9JAZ$4c4;$ zkYHsO8?Yela@{|VRLu$eO0Bc@BfF+_^h&;bD_{k+TQTorUO_A{H_ncA%ev41P1X+3 zFDU8YX>(Hr(^m(FZHoobEgEZ7Uf0<_?qi19DEi79FSK}4fN{Rw(OW4}X3ty5li$-LV{Rq~!4Ja$dlrcBdzWt)E z`-+`nKVz%DxC|cgC+BIQkLo&iuI*5+WuB zg8F5x9U2h2AWBZ%)HUulh_3LhlBlbA<`{9uU=`%U7cv*J6N zG5X@l?~bQuv>O@vQjYJDGUoVoQH^I}e6aC{x;J=shejWaJjrg%Q@w?A0c%a!pANWs zSopb9}Gq_(Xa zE809YOGiG93KJ|Hm@(lmQxOxmWL`ST!~H}WKj2q37r^abD}u?#XvY60 z{@((BM7E%46&MJ}7X%0h{{Im8?R*_<&6q_k94y_~+5RK%U#ZJ0qpMTg*fckax6gR{!-bt@hEm$9!H)} zcZlPU?@9`XzsLrSgWu_LOW}Rt-}9ON?bX-a`1yz%L~4421pL&4>eZ6=YArXrPQH*!t9%<@z*XT48d!|Xr%;UhV6^FH?% zKS1xDPM&D)@;;1VN9uTQ>Au;6`tKlphS?!xYP;5VxmPBAFH@p5fZizggEAkIqp{xH zmZLH5xk)8QCBkae6y9zjMyi#yg*eg$L=eUVM>MYJ--=LD&xR2__Nk=ZdE!1`_IUx` z;pEBguY(|`7hsT z|1Ze@CuQ(I0WM)_SOP)e5VFir*Q~;d0w9sp@weGRf9gQPYi^4~k;O zBD|FzI^iNrA;Hnu>HRRNY&|JTTkHyTlM^P?P_fWClcD@&Wp1qC)JY8Ib|wLe-0es9 z#FN{4P@Ko+;MK?fjVpd^%#jOm(6F4Fry#VzEjHAyX0{0T=0R7%^^9w0+_ey`F3dq= zQ^3@3;)~j~q04C5mbVg99p{UQw8C3LeKhSb9u= zuHfW#SoF&M40hMCNqK-gxr3*FjYs?(MAEBD-Bb{4lB_kAu{MwJ=*lC_XLgpMZbdkc zgIx5Q{)@NH`TJbb1R0vf|Nm^OV9bWUH5~}Zfa3q}-1_fr{m*{3`9XW9X|CS#zBsZ^ zz$<`>Pa!}^PKJ?5l>7=(1CtOFK_i1bvq-@uNdlvVsfxn6jENyrBn{)NZLT$=;%YSY z@~U5->Z0rV<-hX-m2{8yrB^-b)&FnH-HCr77^pBNcB*N_+3|eT={d&P z8R-Fm62+SOYG;@g4vfeeJaWAHs|ri`&{X$Ig{)L#j64qa(YlS_;o&~)@U2MiqAm|~ zr%KcGbVmz2Hn4cPIvLSS5G{>1Czsn3aiAeWmSU!;LozO=-_!266%SjOZJh9Q*iiUl z$C{{N-w*eRKZ}D?j~dxg3~ED?ySgsk!(nz#kEws@tgC42l0#!iknQ8P{boU)4yev4#RmUdF@Yb+5`1rqO5Rr0U|fOX0JqfTw|%my&6mCjTOG zriYiuWj&dOiw+mU4u3C?vWTug$B-jfOygx2ou|ga&O;ffw9gUdR6eMSZiRiUIG(6_ zxoBc4J6J~HF`4)2-4OdiKTe-F`jHCrvFD_;SPUBQr#-)bZz0I0#VD6 z#sAZDP{C(ly>X+(de+L8Fg=x8nI;*GkC zG(X8%0sF)pDvHfj$y7F^IgabcFVxCt#uWD9Uw&lsgbvdDA?Od*N;ymmu#!HhAs(OG zVrbh(DIqut0@+#GET*2(Jr`jBS)a1j>qTH*$T=kY9AgnEIzBeUA9`(BUSFH#%Wd7k zOB1AcvZ}NJRcd!_JVP}!sd%`ANuhYz^SWst$U)qHeW4W9V;D^@BnF?w^5594l?quH zy})YOa$1QK&8{lzsZxCuX45_A6O~g62fCN)@7>`muoUIzKRKv`6CZ8Ggbpo&OeiZz z822mUtsvna$;1<`I_o^tg&?1^MmSB8`Ggr4x7V|7b@Z-B_^6>R;J+f8324p%6N{= zbMi`6-D4gC>?RFGg&%yFC%d&f?>S44P;51i)w50z<sMxyY)b zM?>?0*KL*~&eGA=xus(wnN8qJ%ILJ`p;t3{@lD^$S(7zOqBjd@Xv*gOh_A0A6W(Sa z62)h0&6?$_=2FD$VHrX+*B?)}v>NyM=jzNe~ z%XOM?pNk=mL{Ga|N{U-LDpom_!+(=F4^^beW&lHeCWGN#l3s*N*2qwW=Wp_1w=Nu; zl@iGD$D6Q6?<1?(kQq7|G!`|Nu&FI2=@M4|Iz}X_s2&c7Ozv8r@1V4VqH**MaFnd7;nC2Z=%lC!f>LI~Z`Z%Y zh<`25DH{mzPFeF`ji?3MMHF=G<3al_MA#FaDRUV>PvQ7ZMEo%o6mM8}K~H0U%t6fL z?}c{g1O)H4?#n&D`3{v#z8m{eQ}QgT)zokn=8Gn^0@@*m_mf*}9UyEm66<Ti;7p_f z@u`PP3ymD_0)zb7JSM(+S`ZswNWUAu+k_jwo9t02BRPC=UMG#qH>pH7GwYl-^(ObP zJ^Cy#ziVqfwDPx4P98x2skKS_-fZ7nnHc}WO<8-T)BxUi)Otv3eN;sl-^raR%XSEh zMIDB3a1`B?9~W|ul_sAZnP(zGadv)?Yx_b(+;RO zvl5bCCXJRb>WmE>!2P;JWOVDF9x>08U*+)>f721C*GV6!PH0t`8c8X3V40dp#qvl= zHFX)9ER=hj`$*LsQ@5iYu2l>TZ7e0P)M845N#F9}bJ_0`#3v8YNobRc$-1b*tB|v4 z_{WX5{d(Rd+qSEbRr5LrDwzKDc9;|chx;8UMOrSTqeP3lCR}31k!av(4$igsKK0wh zETGy!gdD71$$>&wgQ?Ec+^*R*Um?z%Ym3B??EsU@Xkru~s%ok3W(6MYT8^el`*5L$ z&&5~a){BUbvzf@f?oDnVR==H%0aCjLh3`D7Z(#-;54*{<4tg-Z-Jaoc3V!mCRlDg@ zVVR~ItaWbLx8jlzG|*L!L)9Fv#*jlfoJioOy<-K;-W&3Utw>({A#~5YXR@F)$pCfs zij1$9-O}XUx*Fqm?2%ub5`Thi=ppvndi`0^O}WDxTbx26PAO9olV;?lWHSC^BT;hv z_ehFXpUQGX_yDhVRbNR;mI1ad)rMiI2)Ap2`;ydRI>u_#I^nvn9AW*s{b6WhzIJJq z0r;`P&pgg>HylMOE-MCeDsiyHw6YeZ#`Xxve?L-Q(EiX}CR>C%!)=kr=o1ouHqr#{ zUrmU^eqn&>POZe=Xr_*N7fwEQlW`?e6Pn)33m*Jzd;Sex^t2y7uY9uwvp7aC;8hy>K)a@*f@$oH|RMm?r zY{APCM0AGml25UO?!{GoX^U`O+lgG?`V}a%$8`2tOTru8IMpDYsP^U?^@nx$-drOu z@JDI4k$Ag9)s@pq5?R%1wQnxUx;(Y!fc~&EdIMjq;j~GXNONVK&i|0uB5I zK&`oEyt;MPam>FnqkZSJ^DHM#hM!;o(IwI_ z3_cc*8Q(?(9W#6JabOMpX`qCBoF|btEF@1vvlh_^uGm%_M{+UKGO}UD zw?l^mHw97UJ}VeJu~9lFgUUXvAdm40NUZ{G&!U_q^+)EV!B(=CU&*AVZjJ#o}1iNZN@`Vw~LI zVBkz^D6O&bVo~L?*PvBRE`!39!lN3a=Ysg#02|6gdW@=s$B_J~15BYs-)o-4T4Ad_ z5J-d^8rIWiuq;vdURRx${NrE{9ZwhG>A_(z0sM$d-$sy zL0Q+criaY5I6O}`UB0cX1#^{aRa2ru!_CLKw$=IH-Lu$-lQX$XRzqcN%3-=X$P@#U z_`VJCJp`0WnDQ=_PP5LR7D#oA0Z>nM$-QNg_0|S-Q|=Q1mQwzC7P_hT1z{??a)EsY zg_G^dm2dw1GjnGgdzoP$us4}uFVq4pOP`?mCx!F2{pkGr;QTYf)k~kE`On08VaTL@ zHkD6c`KwbGt-Fm%pF{;=i6nj&XaQ{dr75=yVb;|9l_)|~uydrJ%M_m+0A*!Bp!h4U z?5hr7>_^JYtI`W^%aZ@>g-0Q5{;2XLM}S)8U25vB0ie_Z z-e-qlmH}9!3Du?w6{d#GQT4FE{{+4|;0LZ7@ssTQ(=&(&gRbY^+wd8!g~#Y*lR@ z*Yt|C(Q9<8wbg3;EVg4-yBeu6tlyN^=$2~BtnsY3V^()ewOgxxcKgvsuko|ot|6q> z@@KqVgXYzbn!mm}%v<=&eB3q9R3njyaQ(V%9aWyCcFvktR4YzeS6C}f;f~F{iU;E3 z+Pg?wvd#P;w_5i(kD<5)Y5wzm!mXFVSXZbvObf54ps$Xks2)^08Do~`F=jF`Q-^n* zW)$iv0u|Itsko)Oc4S{TF-K-8k=BcyI1A>ZRkn>0-|xfkEGT=g14j z+VA9#Yl=hHk^V2B_ygvT9R_^g*~0_w<8O0RkvcL;6;VZIYVYo{gC{BTazG}w$}bD0 zo8$>Uh5o|S2|vk`d%^l!^6aTOl!{?KvERdBmvHb+(3p0;K)*|P=AlD3?r+<8SlH`| zq{!tA2%;cru*8Ay6lmZ%jX)|$p?nZCgg`;>HO(h$64mIHVl<$T5*P!R3g+;~^y1`uS;lpl&>NmKaz=7W7?YP=fh!f^<2@-BE+4qU#Ub& zD2@&n9B*wggp|elqX62nXWXiV$vAND$B2l$(`2{ zZjV9SB5=D9g0A2eKt>m^HgZ??xG_c+@D70AHkACWr%udPF{;-9okIlGHmtTOyg_0& z1-vicng+ueV()y)EIN;|#&l=sC`)O~S>xRNO|7a2dnfc2O$GWR!B^ikO5s+hQRFMw zSKmI0@p@=)wv6^TQ20A&40Gf=*ppyhH^@trZWqz$Q_$jom?Y>0fYXy0<|V8!$~8Vj zGhMdWhe9leW}>PP^qEjS`&ocyk|ntkkDPDH;1-r1@s-2h7r$VfxFjg?%_J2-E_$tG zVML-BWJCaCQ?5#YGa`z_K@k%?Oz1|V3GRo1;D^wqwuMg&Ei42YT16PG44~~K9#!Rs zEUfq`08aVJpaFEhX9DG8jU+o|3Mspg30dTZjE@ly8gl9EIv2Jjvi|+#F)I8a758#Z z@Ajxg^L!D^MbLx}W#JmBy*4Gq1H#QL97Ubl;kd{;^%vfbK96KgjgYGIBOQq;KvRpMGr^Zf3;{7|dGzQEl$bk0`vq?X zcx(l|KE&P}`O)kZTzs|aHHN&Kp6d3=PxvXHvaWqEa+p7I#3~|I$!kD;=HWvUVv)3T zW zJC=XEG!-Z5n@zJi+5s?X$TqilBScfQYo53FG2}0fUAxWU*O9=F2JXTdt!g%yGUN!4 z+NV3xIE8~XG~q%m@%%FiQTqP4u$WFgAErG=<>v;bLqPV#rA_o$Pn!$y>ys-MN)48h$YkxD7_}JXW(8@ zeJ+)^;u?>U(2`V_nBYKIS26)|D8R!3#cL@JL_;*`Z3Pb8O%zBEbHtFe8HAtM&j4B} z*<@yyFpRaqh6BD=%s?EGj)@TJay*;IIcy%I3!qID+35)s=c;Rbo-ILxGBhysCoVg@)O&$%8{p>u#)wiuUjp=nci`%MezJ6=e^nE z*+f*TeHp>Fu3B+D+4TDIl|f}OB6J#|*CDxfTGJ{$Jvnnyrs|z!*Z32vE1JEi5C=JQ z7z2l@0=54E$PiK)?U%(2mjVF1#Sn{6z!sJg%sDV67Bk6GWC*iO>OaNvGwB8e}pQ1MODNYMsQh4S+MI z_W>bbJCR>;c}aL%MqO2~+&H%X9cb1&gx0esp3W9EQWQ<+5^X1g$yp}YMiFUg(fef4 zi)hh{Js{GA77?YH#K;sjabT83$3zO>7gyH-J2}i(g_1c$=1Qr1yV9s;0FE8~2k0ic z5~_pD9gT0&(g~|K47JYa6!CMo{x)eHLOGi8wr(AcImYp}xswnn>gqOQ9rPx~>b2G; zIMfXLQ}Hu{;nnN5zVjcV_>_milUny-&Qi{9pk53!O>w#0j}IW5kHv1glH<>IH?ng#_1!v+(;c=Lw2$&zm^+fP=*W~f ze!)I%zE$<$*j6WSu9_bYw!}oouom0v2~*&>L~D`=9(^N1EF-057>Shim(dNEgP4mT zf}a>KQ|W@gG-nUS^nHXoN7P&Im-?IU>Dt_2rubunnuG3wO}|-p?f@%8<06zIPzye> zr=r-*NiasAn5fJ2Hw)4i$mM})L538PAfM-To6qXU(3!8JsVJXBiNZbbR8}f@Vh7_% zys3eZg(fplvw^TPHe<2iM)@-YU;D8X*`x2aPFZ26&f%bLPEwWCZV1R{X`Hc{B2k&e zvEG;h2S0dZ34-<9}zC zE$l1N_X8Ik7B+2sO?J9>=FubwVIMZlA_-8M{Gc9v_^z$0axZ|~92ZgiWt4_$B;F5d zuA05PSE3}*%RsKwonUJz1c;sjZ7&uX#X6f^n82PY)Z5p%CIs|~(_FZn5*ge~Rm7&d z5=2$%HdqTpaZ|b=ls(<(^g;p*u5r6fgi-~V78kkb-Zp_QJko1!2r1oxgmWipj?-O4 zovz4E_ju&1;dJ8K3C?Wu_Abfy!FxSXnvPLDQkakg{o1Fi^Iq)H0Ua0ngmHG!x5fxY zATuBhRsfY5$Vz*stg>=|Mm;@-?GDg@kx)zzxhAW}rFa$n*TVb2RSxp?{HIIdDlySK z)IjZWUu8Z_-~tptxGfbZ?I=TLUJ=7eg^az6RZCX=Z4uuEK3hze#(5!B6W}QmZx<@7 zk(|S3zAMXFO+NTh1`fp|g0R2k5Yfxp#Hv5pEXc$bZCIN~dSSU?JKy9x^P0uc8zDZi z5bgNQs8Rp(iD9uvD=KN227Q7R+-NBof9VD2+%5Q1|om`b*T48<~jwdTi1l;O#!PQFqtlOG2**%ral@Me?IaaF}mZCza#} zf44U$&D~_(ykv_RI2LW=BE)O+(<#Rs;B%xhsLTiFwrMdf z)q?1-bq99nv2K^ognzW5dF#Lf4mZymjp-w?+u*zn;pJ6lJhSk*zri)E_?zxPcFB9U zaXO7+ltZgsk}(REZfiASTB;_C(jniKd5bmLDU{fciYVy|EUcbI)4n>DbAiw&LN_Gz{>WQv7$C>GeB&(CeKh?rgIlhvChNl z7=}KycF<_xA-rk%m{`bm+dKOdI^MHoB@x-QMUe&3*)PW4jxuVn|Jb2Ub zNX3cz#7UVCF#d{41c+Qg{iV{yn)UgYd=@_`kR7wS4{@5dP={WhZ>SH!m|rMky^EMW z+mm|Yg^4X=XI}FI+%XXOwbc`+OfZR}d(d@+C{DSmitC@lkhGZEwLfh~{k4@(qs^BV zZ}xa>zdi|oSe+g9VRf*;zU>oLc(Qr>=IO`k+Ul{Z^)qbQfjO`2D%5ndR2FvX_SH4w zaH^HFdszDx>-d7Hxb(KzxD-V=VQEy`FpaN_XMkt(E@~I-D@xY@iD(1NDMluueM_{e zM@Np1e9pT}L6DGWAN*s|!e9fTdi#~opPXd+D}c9|aPKI$b$_b~Gr#uU+!`-u|0JQp z9O?(hViAHpw2sF7`Iy2D-2_ysm;Ag1zT$BH$b2TwBlSf<_R%xlNMT2`?hOFsO3-Ze zql}9rH!n<==J93uSgeh&=KIl4125sclnxzQ`Z3w$0fj{#m(G_XCy7{%-`< z6$)<;`O#$pK&T$PV2M>0*(jF%o6}Eu<6nP~zU5H&Dwt1^joV1qV}r7^7B3OO6-cnVY z;v=AEmS~~@(o-I5aaRer6eUkjCJ}sH2 z-;;7~Fps*HDS4ut8rWS4{e;5y;Sb)oNaF}N`PcHin?#@b`~e$qmC$uc`4{H|O{_(41LrYvV*#WO z=Y(4XD!vf_c<1n0|I)!#MY3lf4}*U<0%7o%idp(|oRV+2BJ(^_6xOWR&0%{FR;tT> zaGw*_-?p*=yxuw#@JP4E`sR!Gh2lXq)dJ>NBx7#OiGonq4l?^)Ei4}E<Bh3lGXV2X6XDez;o>pwr8Y*^)Ty=mfCS6;jqNW!ILqML ztaca?aWwe62RCySHOm(R?<;xbf64qO)=bPI`>2SmC(MVn^X+O9gm&7CH#z5joXoJK zp&DmZjvPqAKNdwtH08U)8h1Co9@cV6wBWV{wJ-z_XQa7w%iuCU1*_ubZhEu$?SlPT zId3pAlop(>O_NbHVf^d7E#s&m$M1N)p0!?wd65=PT0;{1N_^%(4*|Bxx^%F%cD!@Q z|IST(<)v^bxOt^*-Y{b^wC?{o{y~waka0@@^M%>?v^Gv=L~Z;5WpUv=X!7L?VliSfO|rQ=($_53dx{p-J{Rba%+M4P`o%0b?2>w z9W*YjcT9535+c4l0x}cFqDyvi0%8OaDps zOMn%`tvosWUrU6cKzwTWB^*itsrFmE$|xhLgfsKJ4DSXMaC0F81PxTfDD*JKt5eHY zH4jtb1m-~%=n_}M!{>+4!`zY4zuF(D)Suc{k=P?N9xfmL8XoXCWtx361ET51m_mE7 z?g`cg)`V-F28?N{>d?HEeT%K0A`PAOc?!eu$2!5AGf~Gj_jsW~MZXw~Z%z*`Vfo@y`~mlK;#{{2bdE;5`mBYm(kq05J!H zAt!by1rZokIJ3?J&z2%Z;EEwJNdwrJ3t`}%ivLCpOP9hmFby;9MGe-EhishDf$+jl zzwK0opTnI3VwR%ra%Bwl$HVVOxzhT^P0c$SL~o2f@hs>hzKtsK>~-R1;&%YUmw$0N zjo8O4uA4cGe#F(kW;G!evN;W%#&6wmnnU;5UJk#+VF}n=0_NhiK1W|e=i`5dPpy-8 z;W%2fsj?2(VC#;8n|0n*;W+*S;o>F6(n|drXD>i&iL;g%_SRzHD!`_X!yeH)R6c+^ zoI9X9v~-QX%V~YK%|nijceUTc>3Gg1v>$Vb;^Sg^%xa5;!I=bq-Iq+RAgkiP2IUbx zO{~&A46vYfiDEBB%Jv|R!Chy{E*e>2foC2gd;B24wzowJ%-&bQi)kicT6})%wZfec zq~?sUI9;C%>9>QLvaZ8B{n6xe1y9s&|8Y2JiMoxKX3hDrR6wf zgFlJUxA3!dnpD+fV;0ni!!~5RIg=?rAE!OEU(=1?;>TFIk&A)dS?BWkSa||3Q`=wH zYNvkg^55oAP4l@2$dT11vZOo$1;VMDkmO*vX^U!iW@y)TCurQKYj1JF7T!Cx<;=9W zf7+^CoX`mS7^R(GnpF~8gU&h2#g*6Buy7=vpRjO&TS;a;($7yq8%xkGId`ma+(O~D zqoP^nDp$6Ci1KmH#jCH_bu0m4E2E~J%V$AWa_0rCs0G}Hx6(_9%-M$fjT?y2xkL;c zOSfq3gljp5F8LeaYq>_-HqU^qX`#-g%UG{0+}EE=kuNy`+t$xRr_-RXO+I7+wn)Ry z)2U_G+`N2vfqZYGW%4`RxA+T&Qe7CIVvkAL2jx1=1W=ZVg;52~Y`F2t zF-G^U*BmvhYZJ)7M$Nm=7==4?BDx_1VR;}=fSwDPc6%B$L%EVrqFk&~)SoqU1>*EB zmRpob#}A#KM478h@02)n^0e$p8em@0nRnSGn3O}>B(dEe9^n(oqwb0_^sbk<;Roqzc-%U(nL@T3mw#3)jXTv2~R^GqS_O| zR4uoPt<~8~(iNw`H5QIk<`!sZPuJKE!dLbADp;?-esN^t&u-tDdf;=;D3jjt*>#+IKeJ1zVAzd^KYX`luuI_da_m;^c4%2eoKyHr4iEp8}2P zHfxZ6s;@ji&Qbu(t8-k;rHEEHPAc%)?o4@G5R${4pmUnNIMxTtdpa7(NiFWoj{SMU z!Lu?QD(23IkqgN(mj_AuK%iqY8)0chYG5-POx1$(i+e6y@x-8Wh#_3?L}1;1mYSZI z*obv%-@gS+h@Jd4|922W_}@KnO`Dt*o7}eOQ=7OzEZL+KI#IbA2+f+c91qv7cLqAc zPbq)5B&PmVxvzD_0N2dYvFZbka)U&)*%cQkd7u_6B3k$sjAB}Jyl&7t7g2uIq2>izGxh`4 zM$lg~_$Ea@kwL8Db0IQ zlYNA0wCLB??E@eWNIX+L=oLBLrWJ3{Xp@2}rn~SzPW)Y}e?vR5Yyq*@a~=oJD>{?7 zMhxevREq_+sZh1{=sP7PH1L{57&JaNrM6E_`$7ZWg7+G&Um3?yD?I4yY##Z4W}SF> z%+#v8iiJl%jHbw_QDl{V^zuP2nBId|`sZk$o|4{6rw!fi4Y6gl$#M;(7&*%H#AE4gzOQddh76`7t1L=l3!3)WdmW9%J_vcWllM9- zx#M=;ds}q*f?x$Zy8_Dmf7Q|Flh2+iWsU9P9d&|@b9jnOQVFMddqWh7;`}}QNtnl# ztgO}ow7hSqwBLj-@saA&$wb=gda5U`^)3#g$m}0anxD<`uqn2Ddv=dm^sD2bk>egT z8kWEIt)-Bq>%ZlP9Ha{@k2+*vH%Hj4vzP~bkM9ipC{uB+NiUd9%UZnbhr)9Qb5|0W z`NoLRP&||zJKT5rgaYBc-zxHdVIGPx6LtJnZ(#4}P&cYnE&uS?HWwswd!#sZ$!ujm zl@2%$4^W2q_Ku+2l?vdv8k~njC}(L+X;9*9%58OmPPS3)>eAku#xRF2e`!%l^qGH~}e zl@CZ~AnxfAy)MW_4rzH|XhG>^An&Y-T-qZ7f>%ul6kwNZS6%XK;s*@uIU?%RSsdaA z>FbwVdVlckTI2`k);D#|;3YDyQuK^Xj(FPB37H=6yRUkDrgDNCH^0Heon$23fDn;ufxGl)nHVwnk74FMjL6hotdvh2Wa(1xgO+h$Zm-|M=>&Q9MBQ(Lg{{$^SPv=RdY%Stlzo z2WNA8_y0IldwHO}HC7hc-T&gg4L0f0H8ubq)^ZlNJMC0!(b?2CaAiD!!Z;x zexZPlBnU}Dz!%G4G=YMmGDL+SO8SROZMAQ}C{3QByApngXg%r{O-PhMy!GPc??SQxpteS^g1 z<<;F#6Z|V4V7$x}&67=n_U(O)JaNx_UPAF>H{HEppWSS9N$cZbmm)@fDvZ-ozHH=-cbl6O(5+DOKkHicSWs6U#b-xY+Sv9j&G0;G}^x;hi{L%vU-2 zS30SvG~dcD>MujSf_ATOn2ZVI)jta7?ELW@A|VFo5kn$Pr!weFuAan>fN~R?q{R8p z7fpK+qNA8_I zS7r7*_Iq^cEQ=}CwAmd0(2&?#y|fU4udDz^U@}3Ltuzb zJmQY1T2$LG+)bwNpTTOb=6Ou^IcL|q^_id@1CDsxzEFkZ5a>>k;g@*erpzytK5xIo~isc9%anYI3X8SPU zx~h`Zj4Md!UHru#F|a#feuqF#bfZXa2Y(6u;G0dlVudZ3?7xE$d_5CQzH-^cE};nN zgA<~GIu1=+VIBK~&v$!Hs9CDl%;z9(m4uIeFO!7sTWRDoW4 zHml$jU6R^)Oh8R1ih9k}!voeWY2u~dK$6me$Q&;++s$=}n!Uyj#+*1{lX?Oh zeuEyYEkRsQTs34dg36kZ20(9Fo2gJ$%9n|b#6AlG`P5K{Q#~q^a0NS!4 za!^ZsSGu|GQeY$IaHr0@DV51{@iO1wS*d1yf^Ar;Ykkx$)^HeMiERYSOso!*{FOeW z8*|E$V9I%He3nrA@syfg{vKguNmWmxz_X8QP#-u4T8j9RBR zHuEQhhc^p7Zn_6tpPur$l6NKhBjpw-RN12_{7&4WL59>|hQ=W7PdHC4Z)xtN z;WoNew7N0cVvM^+5@!#-L8=NB(Ar}t2_ze1Bv*6(UP8MEL;kI$D>7M^aYyKJ&2<5i zSGT(Yq8MV&EUpRVpKMgZx@odo+8@cV7L%N2Da^cb5O;l&-HNz>9rz2;Eap&h734;n zLL7H%zs8xN0r8yU&8(>=yXj^c!{KpbWXlsX^p$YU52o5mbpY@%o@gFtGlR_lEqW#hv z-`XhDptv-X{F~*wxFIXRu)hY>W;e!8vWDKe$&GjpXcYH{E1%8RNtih znWR-6vcUdysf6-Qqd@lkDJSAn%1i#`j8EL;=U)0qSVo0t11Eq3rKx6o zl)$87&NA!CYpvLyW{#zcc@<#ud5qgAKd9#!$&sLVTOZ7qw&gp5MtB0ewRbWF6G!&Y zi6+ym?x7yR3QJ8HuOdl8MU%C`eU z<9(8tQggg6m9A=IYB&L95d;3spfLN1+@{%}b4o}0j_y0rdOX`gD9!>ztqvY;x^eZ& z^UQ!pqP&^Uf%$^kB;cF2bxEuO(4pvtO9G#i^f$DNTy#4klxLg?W-01!Z~oi{z8q{^ ztaP-wI(2rRt-}vF{k%e66Blc5xJ*ploxr$BH!H&5W`_=rfh8m7UGLd|fVHh2GkIQ{ z7al+WQv7)HP<04Tuwz8VNU$|Y^p-tlrb&K+UTEE7_ory0D!q@JuJHaxZC^4r{6^{Y zpK;*g+h7uG_zL}5DZCrn7xwfO{HHf|KhMi~YkoH(7>gNO+el5@O(r($?SGkobPVWw z+WCA|HyqRVX&+&Pcl*wkQ^&V;%5#csKtvhN0J;(j4~v8}G_{8k{atf?d~P3TqZ8I1 zp(w7zRxLTluYOmdGdf@C{RDKFF*R@W0~%wXWf-j2*4>$BkygS!(aifS$e3mDj0GV; zcGjY`d=6%Pjl7~rdbWI8*<*a~^oZ>m$=be{k!HYe?51(UDBW^ThNHhs8h2@i)Y(S4 zGp2iU?e42a1%#;Tfwzw`Z}=VPVZLT`4c;@|!JG!uKbD;mwuh6Q4!F4y@Lm!%2FqS* zJ)+SEdM{z~^;KSy>VUn}kb4DrZ)PQfLNh=k8c^c(N^nFdZi=Nt@^4vj_}^@)YJm)0 zs&GWXA=@DnuZN@X{a~sTgtV0J8+q66!Tr=ASMD)m3!<_fIy@I7}EiU09 z8Nj_FYHrCc;Y1tYmW$D=yCJd;HO@h;OuM@QwEggl*)wIm88V*@#c(3|a6&@Sx+hrx zamDa$3g|k*%$EkO*lpAJOBv@ql82yDh4XIM6$5}<*!%p&8qBjnwB>-3TgLmGsVmrL znAt;$LPg&D1f^5@7W?cu9BA=_O}!F!#Dp0JO&q_zd>TBuJ%cEvP?G!>w*n{LsCboa zvm^cSB)|~k3{<;>2h@;6)egszHOs>& z1rG3uBhocmGTpE(v@(fpoAcKqYLrF#39Cp&qu_R}JP@X%>Ku}QsJcCQ4lzLZyaS^S zd5C=_59-0lJcq81AKY%*m1ql;ZLI2bE2~>sJF;z@ZeP%~O}o%~fa@s1ruH@erHwm3 zU2Ip6=k89k8d1CIw0-Cd#D(JW_M3#Nmz?;Zv4pHv&TWtKC`fSMC9*MErTq03V*v<( zD%fK_6M&tfpjn=BNP2YsF@g!9iDGeE#TdX>wi-EA!_Qwew+SafYoofbDLD%FSX+y~ zNLjT_J$mridd=FI7N@Kj*$3R&BTz#CEwfE7jv)~(JH|AuA!`N8!VHxmKCh8gp~jB8 zNB@#&G!D0UMNK$>dA(pLlKu#0hmhCYOI8+J%&glce~T!nW>^T;(DQMD ztW#h1(z?w5%UK8*bll?2nYEz>Y($uzB*j|Kt?W~0BjXR(X#WT7xSRq{4zKQHu+lax zJ|ii}SzZi2gW?{}^PWpaL#|sF*DX(NNX(5T-c3b@?oH1&?$-cD_KkIIJpwN%vW+dw z-h+q%ijExYv10$6wIFbkC0LysQbQYSmdSv&ew}B5rNhfvq`F%+{x{RfZ!cKPcX@JS zukb(gNym31cH09H4w!!w<#QZ}%P4&nm_}po)EfvFV_Sx#>VUEFfTQtP@yW3`r{TF1 zM0Y(7vf%eZ*VpYm{w(V>fx7Y|T2b69?b&O67+hlqP_um(yl(jJ*)8Om2Y5)$+TOr> zhoNKx?q4=obA0zkb6@4|7N7ZD2Oa%GecR9`Rtx-Neqc-yX2mBJ>bZZt&&ecn*-)e} z90%KqDq)R0w9L=;)Wd>qaoRP-xdmv_mm%ecD6{}s{ z?wl)PLHM)u<~(rY`b*gL5pJ7T>Pn&f7<+e1L~X_)Nb)VMQ}#tRGroWL5yNfn1yPWX zKjQrgtCy`ty>ulo zrwKHVS{AU-`6d*SwACqdD4OChOETBnkXjHBnldJ);pKJPC{ku!nx4{c%dC zX3$mRL`&<4>OrP}nyI@JAjp4i$uB_;YC}K<0bwQr0m1t}F(*Fe#)NE*jQ^S%TT_E^ zQyQqPEah8~n*7526}6X)0#-^emoN_u>N^CCCIifOKOC|Y7>L@hC@L``qMy;x(bdr) zv~qvINuu?j-+9>Uo*z6@u8%o(p zqoF+AYlyKYiZZi7Vz<~Vz(EY1Qb)DJ0k5Iwm=XLsgks&u*(w*q6~IAgDQBb7h@!n( z8k;wUK-H^EGdCZoZbz-4rGpMTWuUFhfv12l%DHS{kbI$0+@2V!Bd?&NI%W?T_M#X) zL6=8`LBC?Vm6O5^Thw@ypdKw7Jz@H7({vtWL3|4LGC#Mwt=!rTr9DAn?&hxSwtG7o zYbPyGo$o(UkENrHQ%(WD5Kqln$nwnDxJpR45-jjb_ILBIV0c#Z3L6DZ&qb3*dU91= zn;Z|4^krId)Ak!*ty%l0Q;+Ie+Xq2o*FXewJxaR8;JTcq?T-zEpZBh z&Dt2*(lO`JfBzyc#kCb`O(jZDD#jH{IZm!Srb-nOb|xYXNi|Ln_~`{U<%XzeSt^M< zd48WYi>D^9%~jBN0n@tgG-cl-Ou?>?;{Q6p#wMlm{Z=W+biRgNK18Vq7ME%sHN45B z#bFh4o?#-e z3fFK1-tJmG^cC8wROE6BB>`E9FqsxU1HTDukO4Hifw!vV{Ew(5r<1^s~)u?$X zKv%nkfv&M_20}-C$6fpS#Qj_)qDn%rVPdI%9)c?cQe;Z|@;ozVZhMG=3>LkQPT!)@ zx!huURXbx%AU)m0F!px~^};LV5mjt@(-cL_;3qQ&@N=%ljL9dkOuBa2TmpQ5;u<`r`)sI-+TIX;fS82%mN7%+P zIXGkX@;QEw)zZt@^f-)hw4jeM$=FOdIHa!z@5OVp&;lE{C$sK{o@Tc=e7OckimF#B zWzNU1OYE=BX0~ZbpiLb6mX-ifslGrzIx(0uAOh*BGMY8Y2-Gs zN<5nuuW95iJ3@Pnv;OYI-f{C}^OPMLIu;Vv>@64rkyY~JOj4nLhg&2@Sl3r1*6RlS zs&ROH4eG6lV;0c;u#nEs-DBP-fZZbi!^{r@tDik9_y&dVTt{H}vt$MM$>>Q09G=gF zd|JF4nh{Qpj{HICn(VOF=q^_Tlg- zr@3&a<|TEH0qtHq&eekUFTGn~>P4KZ5DQ0$m_`{}cBFEnns9{uF%J9NBSyL?*jq|J z{3yW^KQ+%Z3Z*z^q1YAKgp~S?JS_VSJYau^Na=RE7yYInOcq!#$6IXTn1ZzQe#YPx z=P>&Umdl?MC*^mF3W*)ug6(;?cS1U-EyfY|K45Kn(;%|UmyckV<>a}ByE&<3&h}Ng zv>qp$O}Z6VPeNnbC7J#s@`@@HuAWb-6YE#?QYkY(Y#)I87u~$a(Va5CD#6ad!a`p7 za3&zR<8d{P#tXOENzZlaUACwDudO?2>vUvXz0h3u_VzFgJB26K1k@FM@vmhe21hyw ziNoa~H}9K)3bR~1@RICetI{*O7-RX=+r2vNw)^6i3;urCH8kep)P#FUYm{zMd zS!L8yMRTXoRc944>*H90)RwAxe-D(TZe#N?UIaAN@@(6sbkkKJK=Rz_Nk&t_pn?I8 zS{jaCJnHj1pw=BMhF0%>mRby191{DKr5Ed*>|{*5@`@bh2Xl^W)>%S2_194De+*zo2cJ{`Uzveo=Gs<6wZ^=}X zEFdAXi+<$jaU+%}Szhz0U4QDJQg4V`btdo7?uaD+U4so1D)ejkl{G@U(m1d-V!C$% zeovKwrz?o)K_Ymx4C}oz*h+0^$&4iH^?bX?^+vn`z}C%MjeZ`4Umc44+xrr973pOY z^brpbShYp;2x#n?-!F1Vzyu?EMPHqRm6ZBM#3%8|?hcvYX zu@qo@38vB=t_DuOMeK&1)JyHg+aA``ebS7x-VMDay*la*2x__%af99na=lV?Bk1~$ zbm_Sy2|*A8>01P`Bu2YsXn~X^=lU2wv&GB>*O`5@DS3?aazy+X2D^#diuEo{dPy6! z<(UcwZH9xF|B-2Ezne${gn!zALyCIqQ$>(yD|o6IDhhAl$G+bHmu&H9LonU|wle{w zjMWU^`U!vgwgu?+)BF29PvNQEysWEzIJ(+DeJ@K)qtO9Dm)VvJa_$Rss!!$XQ9 z$S%m3JJ`fOv_-HB!(gRQS(k*yd!1^~nEUVC3e~=whOSg6HRDfVAS`K~Zt}@)w`@C~ zlH42U)(}i#LNh3`Y(u7og4e5X$Vv7HXn2kt&F^HW+_tjhBRU-5uFMk`K zhqa4mL!_S})5~N-xM@gH%hry{Yxop3UpZ*M%+#*dhD<-LVj!m(PP&Y*R=O5WWKAC2 zP*g4T0w*)4Gw5)}nqKbhuUJ3TGR3Y|fGRVu%_03@P(f$kx{hpmm|>rIZkqbt@ix<~ z#RqY6k-k6ZS*~3$D-gK@1@ok#SvV^|^$5G$5T|Yg*F+!?+3@gwTH{c3jh-Jex zxMQ-T(7urzj)VheA6^5|gtNHCwqvqtV>J*Gxz^n0hDgAAzeTsh?6vv`o5j%CM;`v5ajyTXXF8|ob z3AMpU-{U$lcIoZN=%c3-PLRsJt>cL9W3(Rb%TCbmn~cyr}XnXU?#}wvu-C+xq8U{%t;}ZG={&iZSPe_HH#{jZ= z;=wxf0Ydizcj(nahSvJXSajqO(>0D($`_?87Fuy->1iSKP z*&GrzlZ@JT7K(D7SyU1u4tYAzt3ts!U2$23JG8UAIuvXw{^S5z6yf}=!l{ZFJ|z^a zJb2f)&7Y(Un}!Z)9=YXVaP?GGmN20_upOt-IGP$6m>R;kI-{s}+=c(7Rrv zNDkmW#mr zfQN>Se|pT9Wqeh!2TUdM9W=yd2e!~aXfQOICjrbx^1VH>m3f`Q+@5GFrRd_sKb~IE zNf>GEhJMjtnqILGn$ir-Qc1e)!wU9}-%{!^j{4 zNn~xA&>NY63SUUbN3zkZbn#ktPGA|P&&KNkxXJEU-L3-8(4uf%d1fp8Sb`x1!k!-}8DaBGRg$+mQuay)fdOuS{{f zzqp3y@b_g7@iDnWB%{7CA3qyg3$rcsbudbohjxNDig=-klN}tfPV8_@Vb?RcTF$yo z&wW#o`zG-s^NA8=n?Y<%JjrD|i;f&CMLxTu@My>5@cCw|U}A@w>H92`dW;cr^}uv! zq2v>8qH$dOC{u<{x-oVGOMnwGeUQfDANA5A3siHN;i> zdvo}U%eXI4JgbvKKZZPw9LHy4H%>~HjYD*w0@~p3Zy1iNdGBP?(;HL3Td${(%-{|b zV>CTF&LjB|-cX@2#d4@80#=^Z@ujYXhztDlSzr>q zmtvExQ=bmzM3N!KF3+za@MeyVP^f8+wkGOcpDH@5K~?CiBzjU3%l6BwHC%ITV%cNC zD%aYH;4FXQmSs(LY9`>8hxJ$6Ers{QazK4J;xuZnkZhRBZM>P7YTMJcX{r3h0pr`?X8(0G$s^lsQ4h)T!lnTa53#w~HUshw z^31}A0o^0;E8;U$+nFH)dT-+U*`jR_??R$C#hYJA8nE641=EWfP~LfW`L+#wPG84z z1D4gE1*i)N8ko59m1p&>q41~aibQDzrI#f@m$|+#YYUp)_09o4(^-euS<;6>oJs3j zUcf{*%6>7|{Zw^NzcuUgnf4Z9TEUh`4-8;fNxFL!uw}omVV}r91kDAP zpE%$n+=Bl(%)#Tr1pjOrt0{82^=-%Q+QBPkCT|XlgY@Tqick$fkE<5)lj(jDiRA z>jL5TZXq_9a3e^6bwZdlVw>W6i?w5a-0b>kpGLZ7>*?I< z3}zs7KixDtkkj#CwZ+loeD%rxVpf;;@e^`nPTCBm92#q5l*WqM27|@f5$(iXUg@$5 z?WviC1bNI2x%dsckY!`5}m!<2tj;xM@rrPGUnP z^mlQqOB{D%bjU@Z!}yp*--k0k7YPhY|CrS~9$Krgv9U~T%W3E=G1S-gmW<17KfVuc zO&oB>pD-wboh5q=W)`1~=a9~ocZf2##|f)b9yG^tARj2fYboVG5VLVf7bRpTmmw)o zo^2>h0as9a(jQbz~UzAXFpl`@%w(n=Hs>882CH9EPl zHVm3GX5>=jRp#Z{ND95Q(uR$Or~qGX@pSKW!%DwYmu9wUZLAIk^aaC1>sD+_%HzQe z>=bAir!UwSM){UKHl_MnJ}!=THN0|1`to2)S!x#<6&!oKTN`vu7+jK;%yKSISeI!W zx2CGN*bi~5F0i}WOM(Xjj`zAC&5pn-RTjB)xT=boWwiSpN$3n6B|L}B6(0%NIpbdl zhu8gOBS*l&04pRabc)*1@yva|%CG6YNu zXBU9YQ;M`h=C~OHvsDd_4dd|u`-@{U8%!XexWkF?MhjL5G%nSK)%Qwq~bK_C}$J4KX?yqD)#j znRO%z^+z!N$ts50@?BrvFFqaeET0sIfI$9$9`I%<49d2ex+f<<*f``FE*DO!3Cmao zKz1z%ovE26^&qBPazIHiab`&($vD}-%JjCH`S;|p{?PBqbNz$rjFmnYs}SB(p2i!` zrH-X2k0nSe%=_x_CAxF{Bs+0)q_&8&b>34xs~eP&Ct`KN4#aOq-SLPLN1O40`99xI zMMB=3BMJu%JOzD0^Yvo@Zn?3{HFQ%|I#bH zU*Efkk3Mrkljn_U=!GiQ#v(KHT)MN0A`^Z0w4;SwMDEsfMWj8*)I+=Jl?e{b!^|MtjeR% za`%>I5Q)H0Bcq9zDX5_VieO8j<#S z`vH3Kv-PmDP0KKAb)~a0@nR3qgYI*Oz&7$s8DM9?U>pp>b8C- zp30kqs}$N*t@b&K!gxZc+-_P;J{)7(C+YwLZnV(EZ1nB0s?uMJsa3Us65I9yX%UML zxj7!msgK$i)_0DGo>h}C+DeECdarDWI%uDfTY-!?@7%kJZm)f`=)aLpm{p0d=^qI; zKWaiky-RZV>=o8rMoIC+I~0YRsSRcj#_qo>HLdwRe4FSs=D|GiqxfZmCYE z#vBRo_4)SjLzF{}FkGWJObswNm>`1^pN>HFTd)fm(ecFbKnow_?2zZ} zJ#jRD286Gc5Epf8Y(M{?5lJxiN(-lZ5UNz^1}ZHQD!@f^HA&}#s^oFY8==3d-b7$% z2N2l?12t5sX27CV7RhE1Ei=kKANl!`Z}jHk+S%oeA^(QAgrhglHSJzK-Vd`1I8f)2 z6`4vL(k`A97&~|tvn_7y+w+1S4ytw;79v&JxO8oMCWEElPEffuxZOlaKhV6!L!37* zc5Kn4cOWgdZDS->-QlX-f)u^gi0GDi)D3Cz1n04FawcxmJ9Bvxu?Ys!60OlZP*)@ zVVyu=m0J&{Hw<)*TWOMhNADEJw5tC5x^n0%sPp@9cs1P^fg^c;7ApeY=QP};C!$Rv29IMu_!d+Cpba&! zbBhp3*;9bHNihGjO3VqZ0vJ%iXC^%H$&>F80m#P(FTn41>uMSXAILf^jm8Q4F7&8R z#x=LW*u(Ic^m5?I00tn@E*qv$%Uk;vlegt7^$jU3s;?(7TFnlxd`GBN*okKx#wKbw z^fJ9anreFF@CmJ+N7=-S0Ut5pa8SJ3^Bg|rM$h`8Ou!EGm425t>JSnBm3}&6tz-WQVY!iGgI<&d*wlA>9j?Im zSRWU?_Bkf27idcK_Vtz#&kOHew%3CyA5O#<^ak5Cc9?$a3D5RiI7@6u*NGVGkSa+|r zxSn#Y@EYUlzR||L$gMyMmkHSe<(|u|7Rm?}oLb7c|A~KWl|E?bQjA~f_6yOOk0yFK z=e``@BKR!p2S!eT!K3FoqBAy+#}3L%@AWwTG3ce~@j3|?7IcUp$a)R{aW>Djn0mAK z@-6t#jD-_Q+T$JZPR-2?4C$_>nkOLS8RC&SNSE%Ik!KEd>q3BME^cc}p6BCM*c}V$ z4*rojs*CX0oo7z@@#4}ga z8~5~TIIVHpj!3)sQjd%%jkP*o%~Ke%?C4RvnEN>&aEx)=g>HIP4LEi#-m&MM%c$n5 z4*5uh;2-lwIX!U*#CY_yK)O?LbF)EW?bZfg{UFTE8)UF{i-14Qi~iz$TnKpa%mr~p zuv`2u0)Nge0(y?KS@~ZCdsP=1vf0wF4eCFMi5)P9;xDoPA^zYW;{Vk9Cy>90GwdHo z{+;ttG`uLmUL`>N9}1#A6nOrDGz09Ph~dvKet`XdBr!MZl75)`hgIw#uHgS9eVgyG z6y0kes|f!^0$w^kkp5XQ4(y)|;Lm}dA)9gi+O9TjK+d85A`1Ur_!k@GJhSFX$g<@aI*be~I}I{0z;& zZ)pDq+y@%TKhdP2{WqZh%lQ*=nx!A;-$wqsUHcg2Y#)4{Kg5WC|J$@Q?n^JGe|PTR zSmDoA!~O?MiN9g~OMLYi3;g+uFz7i{coAsxzai7N0}B6HeK6o}m4Ey7Um*VG{ORtG z7D4~jyR_YFV5_tNFTfASRY1GK*)>txHG%wb1aTePd{_N&5M;CFe-WsMlaJAlv1#*P zFKM?#`_!s#A6!>S0f1v&o z`ake{Hhu6t{s8@NP2tadG5-H>xiuVKaL*- zrsfOGKgHku)1(iV{L8-o^3R8K&i~;}?vLL6vo;&#KfQ{*3jFv&`Y}Il(SywzK!x-++vdc*A;Z)NCobF?xe}vD$(sv> zg6@IWmv#6SXstptqac|l*PZKi7EK9q+hh%TPtOgMjsw0_s;FJT()Dj=XC z_2G0DG7>ru9(?t@O;(~z$|%L9BPDr->W=I$dk#zXEz&8AYFA8(&BOk@%7}u-u5xOh zgHjZTG*-h1bjTohJ$EBWL^XIv@Qk^(T_UAHHGGF`Ygf$N-C88rgdL=Qf} zxiFGo5{nmpQIG96)~h2OiTR?60(=st5YPPb6nh(6EEbFQO@TsO{`68SA%AX40Z96d zK;~h(rZT!eG>y?qW2{JAQg%(*hYT-F!R2~bTlq9^ZbQsGB6xX%=2s8%)*uE_ z^iccd8}ohZ;9L{ixwzhdkp7~eXbw0aS0s_n*_c*ph%YSBoxR;eD~0FF#`XZuFk&(* z{4JvF@@1kHY5KHYT`A)pc0#-=ujS~461y#M-#C6|1iNYabzHoj@MEb&a>EY{V{hT` z9n4>pzdOvt?`f6dn1d7dc?8BB;_7)G@VVw2W+YtwEc(WZ>8j}G57vYbo=}S!fyNfc z4Ph#G&+#f^2h*wkKU3ja&8@d6xGFB~Ehsgta_{x0RSH7E29i_TuvE9 zKxGczviOq|FZy)kT$;<(RZnLPhm5o%<&pQn^oSYTnT|xB0`Z$r#t#p2ttVa%}hK7-|nB{Xs(eu`TixFh)X;1b&jG_}lb-Y=w5?j-fZ@X#2OVUpYB(T&j3L5Oj zwn)lot*af-8_~VOI`ITS?225bAJZ4T_@$?EoYeD!Au*D>cFIjjWv#+|Sv;e%sT8Z} z6z7GV^7f@cgQ>z-2=V)!?b;fYY+f9&BuxjKCnZ#sV(4fzRX@Tkfq={=o zb8U`kMdanto-O1>D)3fy@}rzy5i;!Xa3O-C&+~N#)YqxCbZR0D(KrLeMJgXM1G;WP z9H?Z_=2sIXYh8a1M_#v~1Ziy|S*I^chn=>p4%PJr7%V4i`mldkiF1Bb!2Gj-uoo;!NYzojnVC zX_ZlTH>xFLG^)Y^9$f15GUqrXRUR#6MAGR?6GR$Uv$FAv+PMZX$PoHe@MP_@Psl?} zO9R!=s(Z?8uv5}@2208j);=yTf~e>=Y>}ZlYiQO}yoY%S6g*19&K}l6ZqzOWUd&;( z(FE=1Wk^j2l%1LB9lD*L(J%7TDMFGKlevFMi*ES18T+JiaRAZGJyLCtO(}DAuLUph zHrA&U`8{Grj~3gP+l38}*yA@FJi$676ptR1q^quKFRA8#5tu=~#i-u1n;~?FoZTW? z=kQOfOxhN?nnQPpCC%u%zdidUr9^Yg`4+VuDYhovL{Izh-5lI9;b%7;Q9u7goEd80 z_bNp@Q`Io0%yWoWeg#n;pb|rZYWlmpVYxWZ4kIqU6})Mj>0KzEYm+G<{Lw|2y9i4RC*|3JeGH^e@@{N<*gVj z`B>q1jNqK^p^0C`XYitHK}=s47huK8YG>i`OH3I`Xl`6DFcm#Qu`U2r+1V~O6*awl zAF51=i9?e1)1-2E0RD0DCTs&uvp6l%eW_x=MOi{mMk$+{+&naW>ENdGJX(9f0tED& zMn>7PQ;fwT_?!oCqI2nT_Cc|<2a#pRa&FqZN{3EHN!y&dM;w=?t>VEU+)}3m!Y}sH z&Q?ur!th3_947T-#qh@6#7hpUtgx3!Fl{!*m*}fP0Z<(;b}zPU>uI5KIG!9r;p!5 zt`qOMeonEt+oSq0N1}>fW9F;}*Jl2FaR*8`dnb1+KeC+7=fdkdqZb_L6wZ2RxFQ`P z@dkqCdRqjaS==HtI8}w)oM0O4E7hDK$gj$!@=y?mxHPgn96AT1*`G|@1zN_svbY}4 zQ${VVp6n)Yv%CsKqbS%E2D^?PW(S2e`o~smM(G;e(=G~T+e+f$5KnU>kXyI8Mz^-> zT9&*i(UCT^yl(8==Jw(XE3>VfNfG2#<%3#-2TLrEI2z)p=I@j$Y_i>t$$y7%hf80dC z1stAX@@B{PWsjtnIY7VWm&L`<_Ku`fnLlU#xX7vV znNzzAP^5h<{#@x1X0yY3C36SAR+U*0GDm-ee22$dsFQ6v7karnsgc4hT@8IFsc}EhvLo6sz!g+IJRM)-iGtRtnA4cY|&UXPmdFr$c8<>4@ey; zSiHswduj`sqB@|US#0Mds5Bfw#=BvJ6Ls(y_Xp@x{gngQ+{MbTyi)qwcAI_G)zo5hyfVV{IB zQgZA;d;wZeEk)i0U0Uum)gz0PF!3#cbG^pf zq1eu(lp&J(xuS4`sznQMC}h&$5;Z~6wlVPfgX9h~lkDP#<;lU47@f*{&Xc_hZv75K_*I@ zyDF`s0TjK7w2uVMUIv@PS;!dL+Gf@HAQ1XLAIj3JDxB@Zfq;Z#{EvsSf25oHN7RQ3 zRX8WKJ@ohU@l%qKK$>r0gTe+e5-3Dp8H&GJ0pQd>!I%j75vsGb!O>W`?3f4<6AMEM zmt=n;rl<<|VHWlShN{ZKvWB9eeqJHE=-9ksXjWxqSy5Bu+XX-KZz&XW&I+2ir!H>S z^OtpC7CRk)7evE1?O%n(CGnkEu#9@@WJO zDvEMvm{MUKnPt^X-!F8?Wv61pi zngNBA|yuBNoR*#>8~Lg5wu_|vX7-lgNHT@Dz`LpLExv+ zheK)RGbv6EV~WyuXTU2qvnfUrF6+1>YP6Aa?>|78cgqv_j*#mdOGbvwRH`)I&q9=^ zAwp!MpHnj+z-IByH&3{tPTWFHR|PT^Ol$9mE6#byyo2?vqV)7$n8^La2zr`cLv1Jk&W4bUuNGlQ})L&)`%T2FM|~Ip#5!bwxlm zH->hzTsx}bTnLpKjZ%ZyPJ;#+DRE(lrj_m|EQip;vLF87`k}`@CR(FY?z6&lX70db zXGgmsY(~f6H$nTdFz#-83b?%Q-M+9ya579f&i139RJFtA6V4=z4pd6jf;oF9*w72A zVXYs2;B81^IhVMOy-d<2s*vPY$TQ+PF129T`1TCxK&&5(@bc0f2*_PK zrJ-KR$d9A~p!6FB!~kEwD|*|dsmS^)hD=<#INRgXNYeMhSy&wE)BI!X64#aeZ{*Pj z{BIb0(R(hLY$*<>Ec7{J63YuWul?{>V9a}}=}{tUkJLXRH~SK%?rGv((l-3K`c2A} zzIUp0#fxVCMPL@QoeAR*1&Y7Gpo@Yve31ZF_>7}lnlzPjjNo=)iO%*G5qZW`x!w?x zyqXc`>bLz*hUOu#F82NTJI7s&>wz`dJDhL~0arp5oFviVb|ZcCwp+lll05{yL!KK; z*)yg_4BE4r?8Jhgwk?L)ywy^WGAIQa8w|Yoh~?hev6Q zWl;+$t%y*QTBuR*Es>f5JO^WQG|Bo)Xeb+h(5-g*NCvJ zEn1A_ekIvFLQBJ_hY4ZVe9mJyDVVU|1vmj_^~LHDTkT=?JSa!$V$>vQOHlfz>3Y4P zSYcmKRj|p^EBB*S#vC${6zGC%roP^wIF+tKP?EO?Cmi(DUF? zCyl%sV2}*cje8dv-lJfUOuZ^#qV-!ViGk0@TBce(*C74h8Pws>O58m>bPogKXubR zVMAN+lV7;Xqr%TxT3~Lt70A%0YA#X!<%}}tItA+T@6BIVE1N_1NZKmAyqAV z+QJ*Ej4YR%zh3MMwuPYD(?d%?P)aZ2G!fWmNqDp7u)+O&opd9{B?k!G z)o1rW$sPGdS8DcGwnl4i_5&H4%r&h`&*tg}hnjQ4v8+LBj_jwn0!^HUBD!-~`~@8M z_No=;6HhI>_B{${jymulZx8>)BbdqRvwh)A%+T~x4&CCFu3hZ?uGyu+*aj3Rp9yER ziA9-CK4W&WtAHDy~k%Yv-5^?Lk%QF^w+##PR!xUM-APohw&^4mEewj@;~9 z4GnS%T#E$AjHREtE1@Cyl+^h5+QVp5_#?TkP`n25nH{K__pb9tqI_1G=hgJT7RxeE z2f7+Cfo3&Iw@h#JPv z%9FK8eg^i5r^qWi>a!@gv-Ih6B5{HnRk%wYr29UvXZP>2MEG9EZZMHv`y^d~?S?_@*MUM|$GB+3C?V+-6?Y@IU9rTwNo&D;>uVJTdX((5SB^+Yr zj|j$F+Em@1yS=t0uots=te@UW`(VExyHbxvtp-hxN}8%Ob?Pi35;7H(YM!H1 z$T@@1&xtD($QHGh$mlg*1&x>sD!(mB^+;#hbk!{&@P{^}rO_EE1Brd{k3XfLEs+OiN+mKH_VPLc^ zbR*ja3d!DOFl$x{NgG?Dy`=YJs}`0@Z320WcguTtxs3_(2pbDQJni-w+|JC+?PZnk0RwlD~K2Z-$cK+af;c2+%a7=R^ zO_CCN_Qum0_#TZj-Tk+IllP?h8XCL6G=C`FJrLDfS~u1Eo6BGdebms_h6k`LD z=Mc1qpBc@&!XD^(XZ#Ko5`Z}&#f>l{{pEXk}?t0eSBKr3Hg2l09>@S4iv@kwRg!NVvJ`frK+$_vQ^M z2&XYSUQ8<`&PkD|zd-jOsxPw; zK9yE&&nmOU)AR$r0r$yZtW4m)q4$4D-lFGfo^TsHwElwl#_t1@qy~g9aYj)w_&ao? zFI@D;eemibcZYmW2z5@9ER1st@gub?e8SYS(q-fP=jO+L@s&Mf=g-&!3f9rI?|}W9 z`eC?oQ(XGlP9%;dRK4uC3H~GB7`$XWYGd&M23JyFe(gbu8-`o`nSc86jT845>>p8M zyvfe;0mv_2GtQhR-n1v)ysQ6&V>*-nDD)+~e?5r)DTs-e>`lEhng>o8Z~l$}Tl7f` z+-5Bfz@?@pww7`sLUox*K-xnTIu0fq5GIQ2l!QzAM9=7Hwy~BXo;!=;I`M-4NBVHP z`ULquBnSxA&q$T@f0?3&$2SjK^Z$}ToNaB0qk|P~ZTkd;mkmupfs37CCU%Kb%7r7v z6-LL7XuMa0J_fUZ5Arwj`y^I5$K*f1 z(R!R;ugVfdCMT@PI zdcB9sTRG6Zs$`V%-(_i}=a1tREmGfO(jOUhHjaM~I=q%)478(0Dvn_`D!WP>ER=2a zaD*l)O>DCs(}H!Xk?$Nh7T-*L<+mT1wfm znJZ0YLsF}0Tk0Sx$frxbi4gu)JNQS+fy(oB)RoY@eh`;fJpP@7_bVm+d|6f4oQPQq zt`;*x*5D;!n~=cQemWfHB)~$K6EjyS>?Qk`ulOMO6*TLsFl!0y*ve9TH7M_GR9^!f;w>S6D|{p?p{WMP ziffY=drVuw+CWH6PM+(oMsVu3n^_u2*yv*k$es0AP`WtF|8iqFj&gB3O2MW+!9u1S z^zqUZgn2+B9I{G$w{M=aBx%Ff_0 zv-G2dr*oZwrkC);ryAX8{75!98v#1SH2_Y(DmlWu+1pW)6zFIWMJ`X^aJL6p5Feje zhN}FSAElDECp|nf7ORsFF`yc^CH-^eFBXR6P8OqlCn>Xrn5yXq{)U;x{pZ{!iCn4z z4^LKcrLVKhF^nriB%p6>Aot0=&1AuZT3CGqLJ3|#Kr6X~<9i%6>T1CEvPzg720bBF zvyKz~L?;XsErRg}%xHU&k;Td|!;F-CI(C&iwL}(-cx{5wJF3K$fc&z8up0&}pY*Yq zTot!AvE)YeAxBJFd*idSPrZ_iYGTS}**h~Cdsu3Q3Ly=%oD&f76}qBM&M-gbGG_-St8 zME%f<6&~LvvayXH#uGFPPw&_wV6Ot0b%QZ6aQwPF1GvPUkLJS*nRRGwLXdH9kIO(On>uS)k}UE*0A}!fuW6|I4ePmu{{MKt2-7rcB3|N0Mdjrus|b} zcgAb#7}7guV6uJ4z^a#|A9=v>6;%ahhudF88fCz_CprmMgzar%Q3Om0O5zSOyRF_T z-LnQ3xf6gRH?lZP!Hl3rUp8>UMyK1#@E^s&Q*?0nhOsU5`{F&VGF%YPo0O9pSO>K0 zBLeQ-Ap4xJ0vhyTYMQ@v?*6?OtoU>%zem;zy#)8%Skosy4q2#|Eu?DNWeYP8M=@&w z@Q1z6dG+N(cci&k4>YZ_4atIbwQ9-yfS}3dc!JH(sk2@SmjenvyO3`vuC}2KVS|i? z2xPg_Jg?ZJ@O{31qbBKn9ffD(Zxwg4Kw7z!e zCEjU%da7w}=)_I}Bfd4=KuTY(WNV%&-~5Wv0)C7?VLt|lR&sN=5 z&C~3;wgS$U8SY>w$UL?}5du7$tVL6&_8HXQbusKb;>1{RqRS$Q1-DxQ(&YWZlyj;l0l zunxRe$uZc5wmqL`n84*i2R4uAIrh0a0(k` zDuGczpU>-;dA=b;!Z+biZ|Dp*1l?K0!p-yc5Bb*M)rZ?Wt2K66#*hvJ=?xDNqHi}h(l>sDw9HpB95=|eaj*9xgxZmb>w&MVGE)A zY&s-JrNPvoEK5jO6ZXIr-A}c}5IjB#=r~iY$X8`OyMhSJ!8yIB_@t)mD zn}7<~^+>9--h&5{38*}LT7I`EIA)`gLo~c7Cy0erV-F8#GPMm6WckXy2DzIZ$fyDF zTyXx>;u0>JeVu_iOr{ikKtD8ZHTY8z@vq%$Uf-{f!Mwj;)?nX3_pBm_>(-B*pXd8D z|7iiFzvh0Qul8R}l<#X_ROF2&`4&?cC{(tU&KTu+=6&XCn;yzMtl2D-5axHzxYM!_ z7|vC0mYE#tTNLuiI(TcyNh@Jrf@HJRIDg1$tA5`v$yTe;v=D)_uQztZgL*yVkTlTU z*gx5b#==8}tt7fIesWD8kbc@cst&ZoY}5_DY1anAHH6!&wDgyg4e|;nApo(XK$JpA zvz5A_2q2NSV2EHylCzr38TL*IxeGGefrk6YnmqPr0#`P2c~w&KKgoks$rrU51#nqk z%!r;jnff4Fq)-cfe1iFh{ttN7ACw7eJ;IRmWqn>hG24-pZ|z?a&A16{RqE+A-t~LS z?iHjq4H(>Pw;1d_rN@Hn43VH z2}tkT1aSF4TTV|`^d-2C;1V{cNN+sO0Jv!K(VU2r5_aHn@#ow?zky*7uRkhO5A+G? zT#4HkeH!7Fz&xO_CWc(>N5SLhU1?a1I9oRhGKTmqIgQe}%IX0#hTE$$vwwaJF%6YV z*<1{98&V)MmCEzbW-iL6tf|A*#hGU*F2*mB447VvT@l-wIZe?F-N`55H?FmZV)-ob zJ3|_+ZijCyv)&bLA;2@WeP^oUjV$sW#g4rF4fxL@t`A6TC zxF~7)-LPga3DrbDyq3RWV3*qHi{G@Rr`ixysAD};T3uBvY?jB)IcuK3W=w%lcB@CY zMzd;MX2JCg;+z5XvFXCr_Q4tdVE=m*mry|k-S=7C`Z)FOfh zKe4^JgbK2G!=ZOhiHj|#FRD)i62#ea*cL2EV5C%ss5M$gDq5DR6lV?9Cvm5_Ia^x- zSq&U^rym1%pLg#5(%#%%_y;`yyAGNce%d0~VO06({{w;XP{Pa-tCp_IU%!?$WfU-%W$D(pv~K@6ZDcUE0sp^bXIbjg)$Amqf_sWJ&0Oc62ul43mS+tQ!;z^Ing&D&g{v=lp*TM|pn7zMQZ@gnoq zaXa+`AR`5?|AsAak+W8w(EdcaOGEkyH=^Q$I>5H)M?l6->QwGHFg5m7erJTPI$>GK zsU~r5JTr@1x-_nA?a5z5&a%8&T+MDLcy}1e zSWBbi&*9e{YgH*nB_kyeL9E01HZsme^c|$o7BC3Tvvkgn!9o>J5x`c(2vp`hgAq_u z0dSmQD;9iH6#z!zFO&QNa8|*4YNsdbXy6fWy6gLve=*cHo>o$vscodIWoELkTD)6~ zoD*BZXWg(#7vo=!CR)}s8a4Vo|F-M;^U7OWx^ca*@K=Fz9Es6k0dY0szW?y{I7xEQ z)&M1iR=cHAO6WGSC)NaEx>$~%UA~y!^0dS^29HJG*wlX}69im^951U}Q<&ZGevK62 zu@67|%F03~!i@A{pZ8vufCS5F`F@`LVxR{*osWB(K>hx@;bLi9 z`TMZ)U)WE6;D7h|1I!#?;smwhO`qlgft{@Anx%uaqT&w z$j&`M#zX7H{kV|$5-s+T7wi2JlAY^fd+fn4(JtvMZY?4Y5~KODBEi_CjKb;BpVK5A zOT~*^(kk^FHr)05*78a8!t3|Y4QK)-UrDnCjLSQFH;O%9Wp{3KQ=Us39$fR)J5-S9sFV~&6JMJoH<%jPGaDaM=Z zcO<@rqEbErdcjfOjmmCMqkWcKCoa5Z-Y12Ex3l*|XN zQG0AF)AsIWx`NFll!2QDoVNN-5=l1 z5zY{P%X+z&pUJK)4i^ljz2+7?YqE&6<-k=(e=!N`G@ zF9K3yfW~=L=e)u_fK*6Ot-+TZhwi_w(le3h^@@Jep>d7N;!dWPEPVZWq;0(Jo1c{(Hc4XtsHbsprXGK=DYfZ z`*5<<`LorreklZ5qg`N8M|dyU4&WbfZurK8^SazVvg~X#(=JAPbRT2C@{!6lTGW+) zNr*{{&r(e0@>J-q+;Sr=kV_4cQ{W&cNLS39mP=90xFpY^*-&dqv#_3Iaho)vR!J68 zyQP|RN~KjOq{v~}Flvdmh_V1$s7x|V8mIPGkiE$kGHmFy6j^jw$hHg~K}>?ERI-Ta z_+{OijtEk%OFNA z0vouvA+shJCfKZS#-R*Tf-WROKm+9TFp>+i4dhzr#W28>x(lNYs&<&r1xpv%3GQOZ zuL;rtrZYszlyeAv|LTz=^<{gs=|l?Ae0>&$t2b1Ms3D zWWy?Z)?nmkoOie&_}~!HVUs=Q1)3`SkJ%Fqo{#Ml4X=-@6Ajcot~4%|J)RFjG;o-% zX?7+J>R338X)a^}8dVt89y~5W2)QgzcrJD_l0XR80)_`vffydA9tbBNLK%iO)M?Mc z1xXfuTMR83l0!@-8P+d!$0`F&)G=7g9Qy!{L`=1nh7`Q;IGKMJVdx z<;!Av=;LB2%`k*2s5CL|DoBkXO;u!IsDLUOFr1(mk}Hgc4QmFfb(q-YOExkK)ZI`( z6NDiO?q{H-FjTR}v4Dab`ok1a7l{R4CxmgB!v&{{&=5K^9A{5|0jdFdEu3gb-i1;Z zRy&+~kERKC3uQI*!4!HO!3i>N$f^mpEzDq8+l8o$WDD{d=GTzXo>LRaC4}HGOB2y0 z;!e`;K$`^qc1}$7>sBDiN%;y6ogjfir5^g=zy2lR|@`3C_ zKn=|r`jmwOGw`7pLpO%=4(07JgL!=5`;d2_|AeCrMek99*`HyEA&bK1hfep9!PL)4 z0e}I>-LOAHPkZoS%4fuPlpx5#u-&0QduU+VXXJOlW_AJBLgnA~xuumP4LkZQO=YL* zXd#*!{v=I(((1N8bz9$xrPvI4Mr{UWiT|h7^HsMUlrP1m$ulZ5)cbsPK4)dgGa570 z!&J9nmM_UC%QK2GFl+oDT9Kx{NOfC)y6sE((yd&@ahm)zZAPXFL%rnZ>WU@#ba_U# z|IwkUZcF{#H&vcdj-md)c9;h>^@FP0-aj|1Sh`iHIL?xvX3fYnVfa4}C@?Vpe`)n0 zn);B{ZBXjAu;oigvK2U=ha=C>{>t!w9w^q-7pray`&?16gd|^q^LZe;3~g10`v2E~ z%)kBL8JRyWkaYA%{(liv(LsxsKF0|`#*HGT!SElgm zh+d)Frf6O8m#|M^fqNoNWLs#cA@D=t|G8KU%;p0lghUFR6iz#ownq=<@_`dVE`**B z#~X^)3!1`GH=_aW{;Ifr`>_3jD3JbXU=J~a5tGeQ802&BwV#4krf3~G< zbM#kka&cQrxJhjzjHgK%oJ4MOeEbGNb7>)1+3yv#d6R=5zr9|~?hRAtfj4iLgwv8Q#x7d%{}?C=387}tgw{Y9?Fm%k`}1^v*j-bRjbWOGWccGy6=c; zTp<-s{7ZrQ~?TpJ^Boj@QaSTTf*Yg^RmKUjqIO`hopoXJ2=XE>q=hiO^^jY zjKq#k=J;rP?qpl0d)hPyl!)lP-~ozPzvK}GkFqK>LEMdPe@G@s+|Su36wCU~g4yGN z?0i5+x}qreebGWx1>H*^{46S_~BUX(q zeO4pDjz1i}YB>LvE%lt+$< z8jCBLofQjicXIZ;DXkgfhZb1A(V5}~bgOO#i2&>@vo)*?y%OCGW2dK-s@yfXlkI6F@s-yz+Rr zsG-w=ZMJDF!2U~(u)y6+$$e{Aduy`;{@$`}DgEsFi#Ox>Y)!6_ZdOdU$8XIwTQ<=A(l6V(EwqBij*DIeAWsJ;!es^SCjh<;uo%F@P z&+EsQ<WQOS^vYEfJQLrPL$>AR29fHE`=9Ig zB&{w4iHtz@B#`dala(9YqtBC+Pn>-dP)u6L`>Z`QKk*#*Ne_8Iv@APKk1o`K>o3M! z$M#e6ER(2V?Ju7z*I8i`sFPmcw8QRi7+=V<6Y8&sJtxtU`cvcQr^3DP_S@`LwZ1HA z+3`N6CvEM|p5xP^HAjSVIgGGkSyE)7U1^TccM?0a*;2*pP`>sAtGo0BC_B#Cea21= zYfmhbf;4|BqxQ3Z4JW}^n)I`lMgdnkASt2alU2o7WvxZJ+zAcRDRrGs-2L|U;dBA2 zF!s21zC5nG5oJns`h~#TO%%)Mb^Z|hj-6~}w7&PrgtT&6{4qP3akL{3*BadPONqN9 zxhwXGRpwfpeqAqLse9F7S)oU=`E^b!k~KZ zx!(}=uT66Ew!GJ`h=8CIi#@!ll&y{?6mEZNQq(~q;(tPfeJ*g$9qY{{SC6Eb_Pi@o zhV@R?hHyo}Xw3FrZF;D0;~iv7{y*zgoaT}E$I6)LS7l7{Gm27dPf1FD@%_^?GH*PJ zTGdT_Kg{C0tEfIpR-B%UnTY?D5Ol`K_jX-z{J4a(Z$DQ| z_eWxa<6)8@QEtXAr8(MmvTgT0R&SP*LvVJ-4W-0AG+F`K%V&Bu8i+N0WV9daLuvc@ zkSjboP}K62l>xq>HXr+BKNym=s4jp@ih7abxDwZLkr1bNmO<7)i{t9h6J85G$gl@& zzL>FNDym5f!dM9(P$QTs;n1r4U7f1Qx}x-;{h?*znfSZ1MzbTg7OqO_H(2@XOgYo z%pA`J>7}dF!Lq$J1eP~4NTP%u5vWp?Td%B0rMvQuwra#U zC|mLiI@k#=2e~UB(xja-*sU2^)aw7_5vspz`qX zlIkS_Dh=T>lXRR{6^-XJ)W1}^tR!QD&I@ost5@$^MXW_IsMnvk1OqMLD6~mU;Aj0m z2Z~JsP9M<^Z!{8zaFF^HGA^oe`#e#$RS{>GUc0C6uSd_^K;t)F584kIC_A1XRowv? zcqt_(Db*)wCbK2Q`s9G?uu)eD@a(pja?Oznz-gcUV@)UT=zHEj=>Lv*0TihZ#o!qoX6FhBR#yL?GrBG~s;@F_glPlEk1kwau`+Ar~@D8N-y{{eXUDWh$!292DW) zh?p%s4%gb?4}85pzi#ZE&ocS@dP56Q3$7reCOn~frRa}o@F{=NT>2gBA*u1veOc(R z8UOH7VxPTDB#05G>t&IbF_U}lIziZ`LC9c$M9dcrAbdCKzK-{tDmT&$?-b0LCSKVq z(Uy?2PJ=5rU;u=6ZLbyaa1s!Dfq%-|R@o2$1X;)fGb=7d1ReEDm-h^uE9xA|qV!gz zt|YEdT7D>Ji`it`787T{Xl`{##8*8o7@G~(X?k;Ha+fh-V>MnQNd0J<8=_KVso0-} zGZrK0&ESA_uRS+Mw9S&5gN zO
a|R;yj5TI#lm%bv2_oktvon<@eHDrm$@jbuy=i63_w|x*emnjKeeJzs}^&;h8G_eQ&MW=>*Qn!Y7#GG*jl-`<($Vn2?ml;7Y zluph>d{?G3x-4!EEe95URT)d=Z%yFM?CDqKaWv|;68eR1N8dO8C1Eu4#r%z-hPE(z z6I!;FAuRIkHPY>iJ$W>iLqOf&t|I%>rWr47mEl)_`ZXl9bLi3j`SkryBF5Zx0%+$$ zCjabqlKLou6)CYX(=5MBt>D6pk+`QXtx`@KWbM+KVyw{hI?7#$p(&&p)%pxz@D!Hd zHsz_vNhCNiWhtg|t);ilXlUI%Q%t=?U5ht0er7)*!yg{`!i1{AIEx8DPDmLM?)y7cO^X0_o z8Q0LZ&JVL{ZM2nN`j2r?ax|Ytj`3ke68@{emmtN&o{}YG4M84JivIaMDk~94+UoQ7 zpdo+EcZtNNJ5|cqLWssTkGfouiau^9uKt(BzH#u}v9vz&;me5oOFpvi4eFmQl8Ly1 z05=h-3Cw4KFMI>qwVhJ8Ui$urVqz@2@zd1F^T#K5*TLF8|9VTGUV<&fReo=!ey`m8 zw#72`klgn-P|evp{doI-mO4C`g?g^}{QQIfARsYQDBROtuUC{$A!t=Z4Ttpa3hm<#UWpZ z6l+%mpU3MX(9XwRN8segvEr-u->Qs5F7_*uR5dK;im9 zJBoF)F6(okIua8Z{pP+=ZRn-L<0wN54JPWcqUaGN^itoC#DpII0zLgEA8+(6DfBB7 zZ>}rq#&R5f6|vG0$01z65nANs+v6=25yyH<<}n?a3QGfo@=sJh0e z*}IGsN&ZQ06cg)SPC!y<(j=}^Lnzt)nvh1UB-!c};rsh1#^QylZp06`%zJ*hx6mz% z8Rb8TZJ~I`|0!bQR&kkFjxMQfy8mzL@2Y=_=sCIL04y#G(`;fwC}X zf|wg$Qb&$-*t*3(Se-qIc0<#-Z|dC*{TyqeS5j5>r#(tn`$X+vb=3-*QBT;|aBR`NJ{`Kt?Y`9Ko-#@y$m;Z#jMBx0 zv`-Bhr~&7q*e}>WXlH%&j1c8swQraYR)9Nn-()~MAKHmj>I4Q`q#8aL0PzA^ z942@shs0B#Xtm4Kb3zH*{$Q24Tif5>?I-07JXNat^=wk|vGe=oBX%>W`JYK7Zzs0P zTk$cg#%l*h#qRX(6_ZGq5|eCd`_|qM3tqgm?MrILWdW}_nQOmIZFIi#QZaCONYGNL z)wn;=H|>&Rh}l7jZ%uiG=M)V&9PCcCSnZ%P-Eks$q{(e1wkOa+5zo^KZ)pso)n zQj(sKeu;)qy|-r+277Z+3X!;&%}FSAGdEHKby(u;ANz3@vJ#sCm{S?o@p#}i1?jnW z;cNEFe>;L`3XO#kkeU2H52E<6MypQoiX-zFBLV{glCW;*9?aHb`1pB(Wn7WWpA#=$ zdHD#Z68xY}V^!xCOQpSCQRjV(@<&_C102H8=2LD#23ut)LP=hK#G*WrL*zq`^2}2-#Yt@mJQ&Iz5%J78*Ca-W?Ptf`dpX2N!KaIhMd6eOSqt1Ppgf&f>NqW?@b zdnNfGAH7CV(q+&bZElNx^Z@_;+BB09l?DxZy*8Uwdh5aVaiNX)_5VX{=8V=`1=(0S zJ6XUB#023Pm%dv%3^Tsh^v6>2#e&{t-$Qs^Nu8N>nS9aOlpB*cSfYiel9R&MVk_@U zzll;8h;HS4`w_Pt_%gMJVrO`JR_s4uORyzaVSj}GwCO4!I{d>WwJ2W0W{gYwfKntI zq+tpbRI5!*L+>pQ}Fa1H+97mHj}pP6MGW#m)q> z00RyvQI;m&xW~#yt6<>r-?f&<6-$VVN18>C{pWIz{2=R<;Pi*b>ectl{~;T`gw7zN zf(QZOg!TVMAJzY_Ay$I6y@%!s&R-VJNveHu@t>yOFo(2l@JZ=y&Y_!>e!{>>h^L_| zRiUKKC~>%Q$kAenFEKdcv~8N%uHc@H!mrk};mAXUEos+lpM%uN+Hva}1`Lg^=2_G( zJLiSF-8tn}8TRLd{fqhsKlpa}Per_MmI)vjZ-8hZlxK5cobBe0$JpzA$-es1cJ_jn zQehDv0Jf)$w=^T>qJ_0oph4$KAy4vP_igUa6|H@sG_>^kQ2ayxR89Td99;b9Yd^(2&hiV5KB%M{O4C^$AI!dXCqq)P%<@}qH5G0dOkyqU6ZI8y~axb zZkHAZ06ZTGpPuK3X5t+sBm%>2SQ#RIPXVXmuZ>_SZ-`d}<+a+|Y07$vryNkZxuPyE zIDU5$aNAzV%N)o-gF3X>{h4orXEuNQUcd1XpFOSc2dVknIFwc~`NMZ1a85*-8^3g9 z<<4|&FRz0@ySBw$Jgtd_+deJs387o~Sa zdsVV}x6uz*e-1VsXVI z^H7FvmI|g;&NG6@6wBIpn)dcrmAnkQ*bZu@>W)s~T$@DT0fl`9?>5e#qU;nJLC### zPJ1Ly)(q{g5H+5hb%@)a2BgNAnL4%gjaY%g3M^=yQ<6?C-fTEmhvMzG1otJj;*dSo zur_VTU7KUaUE*W7kYy&t0kM~w4kkj-F7+{v2j4tgf^7ro2-2#nVh}b_X7Z2}n}p~Y z3+TID={8szNNCIs^f`c?ODi{z9(}w7N`U6^TSu*L^n)1NGy<*#p)J*6bP6Z9?k|ib ztMXpJUVK3-{;SNW#~j%M&YF@bPTFr*ZI~QsnK_V%WF7ujj&YT5f$Q8Qqq$wM_-%sk zdTG{M3>b}g@kA);${XK$e>9_v;3-~a;+}d?!M*lU`7?q}#4g!gDS6!9>>e_07&bwy zoSoV0jLo;NAbMqwmRn3mu7(n`H%qKec{1xN@)Qw9g2R)U)0r!pfmlP}(HTm>?3mlp z)Flbc^_ZEL+Pw58oPgv%0M;%xtIWK_S0S`q-Pj)-ubDBHVK*dOSEJq&1>gjU58K*_ zG2yN(6t#)yT@k*6ejEcIyK>Lg0KS=m^uTgWz!(-gqgKeD{WlRIHd1}PY#(&A=JEnJ zjpekSpzE3#84bwmQ4D_gAaY!+(=4vjeHkI)EN1NKGMdEwiE^1Xl~|sp0FEp~hJtz| z9t!lbkwm#ASE53$8#?9c@QuQwrA!t5VROF2(L*>Zl_gOu(Uhftz*2dtJ1}R

fl zj*H{xWb7c82|orL61zWFlrdq~J(s_HlzcF?eB@f0?m20+$$v&F>bhmHxhU$6bhRYN zd9%M(#JJ_D4f5O&eN#4W>5Kzv%L9Y0&A7ujUi)LDcs~E19)$nQFn^Z9V%@idlD;a- znLEzIexzX!_%Vi=5fYm_nnxui?fZ5(>PGBc&ZJOh%&QiP)(n}@4_)n5OT!Y~Ufp?& zRGsLbP2RVHXG-dptT0JS2LH1xzla*I>*zdPDN88)Rx!&^w)-dbe5v7OstxWB zdkdI#awOOU_pf1=P*N~tmZ5z24E20v-w-uxc|f&7kmxa5*U>cJL1~o=151ow)&;L% zuQS!d-=+?O`b%;h=3s@Y{fC)d?(~{22Dn|RVJuxD zp#(+)<-Ih@DxsPdgZbSYyKUUzY5$7kKX&rKt8jEX%?QAH)s6O&!b!kz>4;iH5+7T) z{PwZiQ}>pA`9T)l(bH5!b z6Le>%#mRmowu^;F!g8>mxD-M0vX7IFuTK|vaC(n3H|XAZ)NQoct~Hw}*CktxTp~8t zU5XEghXkUy^~RC3l6ZNfa`PPVq^e$i=p`#C$Y9LF^@kUGW8Sl@F+3MsKd0<9a4c6v zcPli=S>teUp&%sqw{E!GAQ1vR@KRCzdCt=BU}J0_ttGav$z+a z;4J+r3gy`Tcz;t?RFM`eNQLJ*@f^Fq?GNE1qneGDW}7)dg@?90@~sBvp9iz#ouZ|j zqSd;H{n|$C9762;8l9GcO-Vai^2gxfxBd3q z8~2H_qyrkM=Y7P8;Pfxb>|?gZlu0*sK3h0;4X9d|6z|5?T3eUIItXuG?q1X?=m&C9 zzII4ni4Sv0tmhRyo=c8jqI5rjfnW$>cg!z29-pMmG z{CD(-#KayNeqY#bKHww%W8i{K+s)(am_&3lojp&}A?b|U3AEs~)Eaxh8`rfx0jE&R zo66q)0a@Y!#uaDmq4_~5C(HiAdbSY1NGw(vUww0Iusp)#Le9b&p<1%$dzuf$dMPHs z@Mofq8h_$KL|G-Ye<`Jpe$*RuGsfi_?wjO#rJx%+$nvN}n+c&9b?YAnS$btN)Bd^+Ur4FGs5h9U zmET_bydstk$6t4JXCN|v$A9{ico3Nq-`BSMX9aLboadn&6@2esp%7r{(-}+42Q~pd z@W|jjFM?4!PkI$UgbnS$?&IL%+#lS8-Rn-z3yw$COASqFUR@ib+RUq+e`|B|gr7NO zNJR>2Tv$)%1<5!cUZ-xRSA&d$P=C(b-yXh}su68xhZ399AjOMg6H3vsTXnOL4Z6R) zOZoQSi2irHTVY+~9v>0{qWANn`rm-VfBIokKIWD#9=6U-|A~1Aswk<9d_@n0uCS3( zMNg7;R>k5Or;LhQ94lXh2Esf4@(NdY{U-lqJoq?fEL?Pm%rc)j0bX%yv@fTC_s?uV zU~m9b*Vn-q!hI_FF(D*z_TV^A@K+rC&?z!YbVl*DX>o!#jx}I|Nd18xD?;VYzqokB zk5C;J%xMpru6odh&l%A)wD|_j!btY}7mR${ju{Eh+641p1Ahw|7dR2vlNG*0EIyv@ z9t*1RsNV58{|zrbf%qYGT*FJ()y-u5CxV2py_nf-AQ#7{fP}e@k?%RBntVG=46tb= zr3SfAQ#ob-!^2%b-PmIMq@A&S=Fk3b@z4JtKx%o|I{fDwtN^cg|U8d=XgTUm^iA%^@bt zUYWt_B?YYrkyi-Gyyj4@mIkzGQ*lw6fT&PXEU<#D#1sQO~dW`ky9f|mzqEggg z%2lENYe`$MqOSs=1<Y+t@m4e<8bLC41)M zwWG$kDhx;5nv)~0G>H?~oAZG;5yom5dVGsxaxH9|8^b0uzmqf6u`Elc7Z|JKnbS0s z6tP@Ql#@99V`GqVgllxTDy4)(XUEPDy4P;DhR#I-psjt?_~}yPbUJ=%*T%HEqL#_x zjB%Y>3Te4w7`2h?0@zSs6FX8ZrdVv14YVC9$T(dJ@s5VNFgkLwBRASgm4KM{BB}zI zQ9-#p6Jv%m<{ov4JLJq5DlBa5WEil6?vH&IigyVZ`K;ucd0pt-5#!1mY5)Mc@z<0` zkF>7k^^w(2^$N-9&2}xM3DPekLNsZ_Gwrf^5*h2R)iKj!iE1|%WLYSNGztKkf$Cwe zOcW+divlq@id?4-$xC@GwX2v`aTOHF94SP`4TnZkT5q1K_6lyZz{i@dE<)bLu-dMz z%^jv{E)+S|vCa~ZR*nyaRmF$wzJYNYAaO90EhVbU-;*n)*)EGQpNmbg+o8yq=Uqq? z^QUaXJdX>Yo+%|JgGPH_-w(^tzLiS1B*jd`VX9gEqHkvm(3OI){z(!VVXvSp6os}| z+@d0Ow@)O{opz*-_hukjJ*mTUAE;NA6eqb+-5iL;<=l$z!0V(}{B7yJBL6G5Kw0K9 zJi6?N-!}NhGj#exI&$QTBq5XMCsgM%P|4QZ###>-t4KTD-<9^G(c9B$IT2O61u;)* za=T7jMwKF`1<^opE3RJhAlGBdO3yP+(C=6T>q$Y_i&e#9Q|G(hFD0QW>s-ugFNLj_ zftYA8+lyo6t6?VB^8IeeRglMlD=R5c@>$BJFNuTv~6?`T3C!>0`MbUw_ zZ{<*qdybJcDm_I4ZC5Yc4ZBj3 z)2<~m>!>i0I49DUA(0Kq}jG8Aj|4pJloM(pf4q!xY-K$ zS(GI#(z5QyU$dukbHVZ_>;^^97u@=ACim%T=JG=Z$=qFF1-RvD@oV+)b&3R!i^SLe z5f@XUV~um=6qkaX@G=*#3gC?uKZf@zqe>qnqKyDvk586V6K75HVNcB;Xkv&+L>O^g zW#mxc*(=gY3o;f&`-}Igy0MN6#c?8!=;dGhme!(+yR-ZQwb6@=?Cjez|0=C$DwtSi z&W8$yBTh2s*NGr6(!z4(@CUKQuog`6AyXfXx&}z(e@EEWZa1u#pl3O@x1$GwPYPmj z5J~`*L*v!Dle~ZIuCXnq+;`knnxnjgSL`-k|1Y-QF-nxEOBO8Kwr$(CZQHnI+qP}n)-BtvTee+Ov%2T4{`$?&{ChI@ zjySP*MnLT7aQ$XQclsO99AVdEF~6l}Mq$NTK@N(#ycw||%$2xsi;W>`1i}dRpgn<{ zC~Zxs1~~Y_h4VqM7e#cUx{~8SJWyaCTJe#x4;syiO0O7l&N>tUhgg(mhoL2{RNzug;6`g(;JMrwUBy4XLKLA*}Q~b#$CD2vjBN0g)~7yf4=1!z$O073hVG>BFqqo$7SaZp}N`>`qn(cV}4o?1+{- zCpdBVDmiV<-*jxlQ}S@`&J%;busShSmE#`j8~pY|le>tw^5j`w%AsSA2Jg1qG-YyT zM`MXk!u$YCa`Qm=C$JV+vs**|Y{u*M)?6^Io$jh?fk%kN zR|G&08)z7{y%9bfl)}IZUR?WZi!IX^-@}O?*3W-nbB1ImfcY(F%zR3SN9LMOsFzPF z_m9%7@zNtlR6nxGjxtuZC{2`*zn%jPc($U>8>4UGK*U~|d(IvTlkv`6bj}pdJ7&b* zaYl-JQYX*S2K?IWT@;RihrL4RNjWR1{qkIoeyFaoF~~&CARg+o6zC_aWQM+_i&5l| zHd*6<(A&3xQXQ*GKY=GV9DcD58vfs%T?hL)R2R2Hw+pk0Vez5lj<|9gT}JBR0VA}8 zdhr`f?8jS(b%j;>ThTdYww0JT=TToKnZB3tQBQE}&7zpAQFFthU6ow`Rk42dK&m{b z*TB$nD6H*Q=e1S)Fu#5?{eJ#Ivp%TTAinjjya#O$djEjGCqaM1w!42CNmSA8Q+c~Z zpP8yZGHAUd^U%|;=o7i<@8CQ`-oB}BVpd;md{MO}BETgg!z6>s-nNJQ`=Z;{yt1yi zU#a_kM!a63Zm8k;6pHQ&p!V{qj7@^H$b=)q)z6P>}wZ)zHr)7p4t6=#V&&B|y4DPN!5ac6u4tr|;2G%-R<+s$e z1!3l1^sIA>U)X#Gal^0k4ye*N=Wb^#_E>Zk^`QHii%s}QdP`}h2`qygX zf7)!=*qhT!%GjHmn>zg$!;FcSl^bF}2+5P)5U^O7Ti|Y{3D*!>6|Gc>5Jixv{~N~` zD^txytNWrw!vPQ`w3doWSdO6gdghgh$v-=GvdQfa9qr=+A@)yNDa9iJVj^)wiAgL4 zK8~Qaag_#9b2>4ryKp3v;9y#Zvf|h^6EA56*&=sf6Yn?s$+PA}O@>KSPmOPQzO*b&PMId8Ea^0R^xpZ6eC3ZwaE?IW$&(%z~K(&Te*=FHt^8!bZ zH`EXi@dse$N~|S~?J8bIRPN>iLtt@#eiMG9*^dZ)03$3UNr|7}P;fIR?)ugPkNNI( zb|^|BWi~oRuWc))%t2u&rH&EMKLywBb}g;6VKW)>KU6~o-Sn>1&qjH>8!Sh+V|{Ld>u zha(G&rA#UCtlq)c#ov3JvHc7&AJ{Q}fJn_YAtEyE8^pK1074EJKNmWqZ!1DQyvgEk zJ&)fGvJj=RcwT6iVFt}8yv#}CJ=frPA`>=GFxV1oQnwU)QmnLdBF}rg8(d6YN-Py& zvEvP25lyZ1_OezYX>Y74{u--s9TG($9F}DY2(qD@gRb+OPg9>*hCTph9}QtEKM=a+ z9=}nVSUP4_bwvu*(K&TtTp4+zo97uj6-71udXPn(?WWwAil%;h#1@&vCMg=o$33X` zcm0*(Q%3w?QxCi`^nn-ciJC{B%R7j~^NL&twt{nvMdy%H+$TjULzTUFJK+7sm0VtR z;9b}{jCE!Fg2Dd;Z7A&-+w2(}xsu z4`Ff%q=~s<{L2~E7|FCi9Ab*G_wK9*@PP!a0du$wBWjMCH*`raF^oTkCytnS{<%W9 zy&g|dAN=oh5Q$1l&gl`>iS2M#@l2+s4{EYmpR<@Aec{L^f?R1A{gvqm%WB#zWsBUW z&<-CdC0wM&=|}ct{X4o+oU4wq>aX8_mT!pzX-&y*?~4Wf&r9}ym#>(iv5UQv$A7m# z6a->>*cmuf7T#k zQ6HgU@U`-2hMGPz?+QFm*9rtipRx-6ZOSPdY(?MK%soZBsXY)hSLQ04O}A=4)YPML z09p4~QJJwjtX~!?ItSi!HGomKp1j+&)5R}Db^(21$M}d3&<4!ojcb;VS73;#8RR3e znh`J*62$-pWS_mys8#8?2I6YYM6cJ=+OIc`NsU_F*croyJdIe8viu6Q>X`CDg};rTbWZK>PF1N*hoN*Io=%K3yTfpX zGNTkhheQRPf~LrBG+nNkOkauU!Ue0Pl>^J%G$jyjP4$R`*i) zt|J9mXf@BthB!h#UqJuymZJrhO|HwM91oa(kUJ zKyuSuK4wG>@Fo+7to9M@bqV$2XS6-gI3)%5bG0;_c@?SVFVQ*vd|3Q_H;-u7&01EG7vn;kTc2!F& z9|B}zB!(`v9aJHLOM!}xTq2>IB(j83-{e?f{vPp|-Z*h?4ibf0o4ms12%mC;z+863e7>QgIIL^n{RGHH!!d%(hbjWi z8AXANf}8s=ow#rsDf<%ahyg75v&C8&WSV2{Qhq)qB$ZM6Wo{YB@#U(Yt)Nvw?%2W- z4~d2Lc1L2%g?qC5W%MMV)#BfPErI}FL{XZv5Ej{t{1rbZ1MP6d;Twn%rG$3Maskw^ ze5%5)Zm3iHx$AL^cj#zxMD#g?R+>%#=t&Q;+zk{9!9lqp+C;y-`oDG%>YG# ztj**G%yj@G<`{85bFNNZv9S9s*$P8s&Gi4-hbG9Z^BsOEM8fad@c$2u_^%iB-)%_E zS{+#(WgESa*bqcyq*7H|1TtTw$|q23Xh@8hLa;`uMVJY)XhJ4i2I2DK%XgK}rlp1c z<5kI0z5Yr|*Vk(G{N+3HJ8Z*;To{6}HnAnKBhGnt-8|1{_NdO!?+dqI?K3}L%D{sM zmj@L;22@#%38NuV+-7Lr7L+f5m025eGxBV`+7w(^OYKJfU!!p+?VkMj{6Kc@BIpKj zS+gT;x^uJ&gIPvHw(UhJax})VB`=7xm zr!nOaOLh*>azN`4P%j>JEwu&>LV6uFzsgdTg}=UmVcSIw7*Grqnn8g8Z`>(x$78u2 zymb*}p-r>~Bdx1O6D+42I(k6_PCQdwJl0mIDwG-&%QOP|YEy%OJr@_=MZ;8Z!VIRJ%#VcB^2=ZH4e=}IuV!cqdWO>!9uj><5;2Zp@ zu19=G=r>lGIp(lYHPkgVn4fi(fZ@weh1iv6G%Aw&pZj-yIG5B05B5$ed5>Kv^O35= zgp>r}E7xKWta&n}Nse$kY9wMVm{`tEyKJ?yOdj8==i1E^_X~K~( zAVan@pN#4veZnOe&&(Duo$G|!vD~p53xlZO_VQH{|3!)!xbCS5Y`{tHDWVQ3nX^V< zCQ$AzYjJ1Uu9=Kn>Wt1G`XDp%9%7_UypCD`X8WngYqajL8!8BX7mBJ4g!eCKvX9_a zTpv=&jh9DJM#WEREODIZ&4`;CvILk0Kt(HKnV0yl^-xjhBNY=@5D?BO^i~3?zBJZ4I*!{F_%G`z;?X5D7q@;nFw_!= zM{%lBVBj7ojA+W-nmH2k+B}so&lLBMO092qlPvO<9We6DM8;WHWb>s{aPuSylnj~Q z7GDzEu*2o5-9=w2x+dHT>`qwh25bv=g@K0^&pT7)jzL$qr%s%j=@>k8&w0SKJL)Oj zG3gP0#Hy~*)YYAY08!mbC8mK%(U}g9qe0DU^=|ky73DNnk7T)P4`|C+)%HZX9l_q4 zQgT{8=0?6a=AN2ZFW^Ky*<=cy;6Zs^=(fBf;8VYb(|#W7qM{c&_>}VPk9WORM8X)| zkCUS2?K`hn$V|dHtoAgISx8ATPKVypVESEan==)8)fJ__9nsZOIYiCqSbRQ_a9RDk zOy9AWsB;w!k~@pF&#AXO|06FoX=X~ewBS4af`q)s4_Q0eMwSOsvtgV|Hy>ar@2-j~Ca^s3H3pu{Ye)eyRP3X)I)-AN4iXfy3oa z?3+rM#vpAsN*FI*j@^XZT=3dP7puL3 zJXY2tJ8?_b)q2FwUUVT70|8_IhFFJxCI=bjryTn{HV~OU87HHCyF@o&BwrOKlM>7) zaAl$Qi9MTh!x47fIz4+7S=pT`&iui>fhgJR*xd#$8RX}Lw1aOHH)txGE?_|`c;Q@7 zQy1u;YWJx5K331Q5Z7#i{I_n|+)}LmZD>4c_uu(0p~woy2l`imJ+8;EtKlceGIUF) zLwwClEA%C*b8MJ~blmn>-d@Z@m}>)QGAVysTAqnM(0Kr+ z&z7+bFg{a3^k(_Ptoy8<&LcB@N58%iH}Ag}%M&**xBuI=|1j}OPT_a&>gLxB@qaqH z{5N6#KYBrl#_!UVI_kF^QxYaw8W1p`k_XI<36rHes{&974-kk@pd)iGg0-u*yf!^<55 z@p>Ah=#alwSN zjRT>ZP|0Aorp})oibdty8Pg=JsnzXu>8T@GP=@OE){*5!f+}D=<3o=(A{g2yg8=ND z3Yznlj?+V#rqmr=L5QzBuW|w7d$O+x%nb`DF41bxt=lpGJsMNl#VQXL5~|TIJ3yy0 zCI;HtdPZka;I5F-)L|e*jSy0^Y(cSCv^jF1Mszq}2eEZk-wQPDq(Y4uXCo~qO&C}R zoN8n#-)`D;$jb{`6ZPsRm;5PGh~r_u8dV?0EGP< zlV6qf!paA#&HEmW*1Ibjt=|?4%dtzS7t;&vSCxyhY-{K?>E zDPXcmbJ(e=KkGEKv1|oJ9-yFJa?e7o=A7_BWZbDX#3EBj{HjJ;7!YDsC9y0aiK#lb zqKHsVLn01xvwTg45nI>PR9DwN5zAJ(aWTdx^eN$kFQTg;B&9U_=O@fj*)=aH*5zH{ zFv$cDw}*#*eVuj?4VMZD`*Oe+s_X1UR4}GwcG3^R>O~E53Gn4?Pw z$5FU1tpxC3Nxy+m7u{NNjmE6Ma*J94T=YDt!)YeLnF-Tz%WAi-#X#e9cs2DLvHV9nKaZ6e;XI~0f#@?Z^ya{eX3*t4Q-Xm~ zaYNp?D5=*Qk-p>B2ML>@c;7UFp@=V}$QH4Oj6(2FYJ+q7{o2yQ|go5XJkaPvem=!yYA%c zx3ye0Wt1(iZ(ii_bJB?s0lhK933a7t=PMCgHZ#S7Cj{mD4g&W+l0!ctzsQk<%WtppdYH4WTY zJD)m&3|v&`fdiWL+aq|Vg6WV2;$PHW;R1eHtZ?y(rK8c z5;SH@Td|n^4J~bc+lluIyCdI9@~WCk`=jxz$HiFd*9tUPv@l|woA`PL&ME9ELWPZ( z8aA3^#5(>c?uo*PFIt>=#lZN1LHh%94Uqcndr5zu9VOBC|Y9v?hv- zk@dbfVn`;<54T#>8sM2UB$TZ6*(n3e|Be`1onoP`nP=Z|>2DIoWq44|ivVC&qJZ_*vg| z-rlj*a*aQmIGiKi-uqZ@YetiTvrZ5(*+aiEZlPaw4o9j^68cF&IclD2Rf0HTmCG02 z&k9xS<*M*wv(BSgOXq|2El`{9mi8<12?-x(NSZy<-)th z6?uGn4@_T=;O}QlV|5Ord8cLYVP|DHZ}Qww8g~Pkrc1nvAmQy z`eB7hT$?`fU;Quz^TT1UmgBsU2bo&O7-qQz+IjaDW5<==t@V4Y@4kM#@cF-C_IrCT z18oIcLEkb&#n`lV9}%JD4$WU3qIBNA9w$0;^yXY&${bp*s}sbW$i!vUrs}%)rEG^& zKJte@%Wr(3*V+D*YX2;b<^03M`xlDrp*Uj0GYc|9Xsp*nE5<6O32cU~RCbrcBA1og zV=i*YqIV{iCTi$U{Ihwo!Lmt_b&w*4w?_7?d_+FtT4La~FV6ZMb(@0K`84O>hs}@H zv%@m~W<$f;+-<-DJld4Ej%|tY8vgDLBTI==r9-0zN25H$*Wz6wDyqc|16izUz2wAR zbWP+tD$7swL(33RiQXX>UkCLIJZm9&@Ih5DsT#eW)SG$Q*AP0x3j*>1ha_ky9Dj4l z7^?a;_uRp9m2T*#dQdsgGka54hvZcEU>>tyU!^Ku(iR4ux~89EN_N~z#)2y-9R8RI zfE@AwO?PMYhonoZfY#=;PPLZK}xc%{G!6|=)n!9 zGi7kp9Sfd{0Nx9sG@JrX5i>oWCNKjtfxT3_uBqL%=xXUT!@D3tU~U+GFN`U7b48Jz zG@bOS=Op{1>*rza>-Fu4+h2Pv6~O>w)ha`{MrDRYW%0nwtg1Anb@$=kZYUMOBpk2O zepyD9l`3OJSJ-TVb4ZPR4}%+YhjRk)2EqZ4U@NSHx= zwLO|n1m#f(GRROsklm96hAH}6xHcMt2uTbh^iwLDT?g6I1csPM!5E4p7=!5OP(G^# zi9N5XgsKE#a&^+Gr9889&1*FOOREioY;*ArHG?T$RhAk@_2r^;lp*_+!jerB<-HVj zYLAPwnhk3!e{Q5NL~jluqVsvCgJi`c9!qJTZ6cpnm2e*NFBGCAC?(At%$n<}lyu@v zb1-2_yUAhVh#t*;X)uiNPhO9@1xs2*mWXXtD7WAG5umW!P>;iY;tS)W;-48lb$hH< zW?F8cAzn~xjS@A@ckpXy^mEy!sk~Dop~7B84I~*b(=mqj zjH=4i5n9MN%wsyxw0Pt6c7kNBEvQizS!eVwc#u|EkuP$CE<(r7@K`B$AG0Gh8+stF1%%jxOk8!2s%^cUTX1=0v>{RPaQN?ZF`r~xWhBMl+E^9+Gft|2doMeBJZ; z{cr`+&!SglVRX8_6=830hjQo@tM1yk12TLE` zw$9ucQ>M;a@J4GgO0CCBEwf?<``R-!;h(SzyeiVDe@Z1PSw&Ezg`j^EcD7wuS;7Qo z_fJP@sR+-)T;ej!w>4SoRMw?1=7qwIyf`xvm-3-7_7p%Zfb5(tfP{dqoq|DvLAOpI zpe+y?M(2tAp)pSOfV4nm9Gi(K)ZjTc=4R&XQmO1(dzJnnMGz?Jr!y?xN#s!Foj}IIKfA@dw@gN z-aL-jTDOg84RsEfhD}s81+R%&T_!M!R+ozjM(&UvV3rUxXhUF#9NcQCv2J3^pDuWRr;aMPykO~tP+tZ69#x3_uL_P$oS>}w_1Hj&C zyE+00C*jx_Y}T!SiBoG>8LZY3okqsv?H9O2KB-YRMEof7)v`{zz&Bl%e;xZ?=UN^% zGfN918Y&v@0*HC|kijScdc!AMBFT;TSVl0X-tRRtTjJf=5!!Ys8j!gut!5~`#@xB? z{t2-?*99ZswsG={NAn!4I)F#v9xlt*SGhu=9Z|)ITDsCqkYV*RdnkIoCq+x%t2X1js5 z`XV!fu&&|xyO1h*W5&!0);cLQl#`)eZqWy1k8l>3YTa$;EYE=@XjCQM=?JrY8N zF|z7Z*pHqJ17VQlnxcK$;i>U~s&r1?ZQ9n(md5n#J9nF50zq0vAEnSB)Ues--Eo;g zdJ9rmScc_+DzRN!qep39EyA$&NiN2qjBde4rV{-HqZ4l!^jtWgb64zkL`M#RoL?=W zpGCBcVj)~*(TB@JXPBkr0GY?8uZdm-kaWOATkF`yzDuiOEp|*gV~D7ZC6P>H1@>GF ziJp7Bga#__+7-1pYu#>bqGft9Mb89-Ovf(q?ln#sT3Pv})HjlxRaSabpYsnVpJL_t zG02&xP~>LaFnmXmKNQJ|2>X=1qi(1Ls(v^~sRq28lOd|nl?0&2b@&ZS$C*l2- zv<>wdmwNvu>*}E?D)(z7)M%|IQD2sM9?}Leunv7vTV6r_p2y^}cB*Q3zcg40QK?Lo ztTD4=@u+TLYP6k)C+i0Y}TM5N&|EU z;qge7;X)QspJmE>RN*BjiQ>8LeCBfN)RCO#iY_|xDnUhkjIt!a_fEuN zR*KUS$8fwVItVgU+L#Oq&2{)WE(>&Id6*W(cWFiWk?8n8(S$Xz`}+Uxs+OLGyS)yB zdqscaQUD+**bljOlEwwQZ-Ss)q%QH}^E2Gq3#NOArkQn;1s4qbu;OyheMo*Jv(a$T*>(CENpl{f~h439A`28?ZN>bK}2`bHKz zxbB-=y>-XGo_6_J)_9%gf@K29O?*%ps|`UeOt_^DHOSkJ7{X*LM6ERj@FJJM8G;g* z(1~4f>cZKCx_y+eFm%vvw4)Y8D;`^KLhYogR79`R>;Y>bwWzI!+dVt}0{d{{KxKTeKkERYY6)%Fh41TpbIcK_DRI0|v|qA`DYh z1tX9kAW1+VAP6O~ki^H(2t&|0is(4v;G%B{h!(Xs5ku7M@m+t+?3Lfd3~_K#6~=b= zo$t7CAsqF*{04Pbc%5e}m&tv(T3Nf_t&%?@%`<#&2?th4OODKykVZv@sexNt39||} z3Bwj(!DGraNC)*n!_mU4(P3tGSy-00$4;nfq-StPL^P#ynW?C*tbkHqn1N1S2;XCT z3vjsE%?AfcCNVwSHyxWv72XXW(b7!)oyg2c$%O?JB*2P9B}Zw~aXgh}OSV|4S=$he zWX*^U6*WzXe4*@RS;}{Z4q!=6Rb8o&A~)^xZZ|Sc@qW7{(v2`b_cv%{J~^6|$;*hS zJtjPrLok(*&eLPzT_b$xLWn`vi z2Y(fH2VW>;mVsiZQGErZOdVv3>A0ii%qm{9DgcIG+cmER;>ED@ ze8!p^+Alq+1?9!E^Ze$TJ8YeXUJYsu&eHotw$N@rE6Ebz9+dU#1#PC@dc3d6-`+1( zw*%+JwDWwDLonNGfwlcmd3-0GdAU|w$4(1mYi;p*U^2)_$t=d-Cy8 zZa>SZvi;5pyZqkOwSL((`nV=$qtw6o0({hHKl|bjWDCpUYcH_vg9?Ig4b9@~Mr`KY zdYqrm&)F}HTMfvCVE1vtwvcZ>TjUn-8j=NPuNHSyPolaOa(uZ_b&R{Z+~bW^|M&0E zRff#W-b~L^j!#(!8k;q4NV9i0QBSEPjhOfwpCfI|*b4zl z58N14&XXRNh32Djjocx0Ls(8Za533{$Xp2Ebx4*M%tdx&EUi)?i>2N){^U%s6|NjR za)K&#LMp4oj7s{+FLi!JZlaVTiPB0!IoU?a`(sF?R)$qBG=Mv|NtJxjgxfP=sAG$1 zb8)_eqpd9_1U?T_ib2rPBQh%;0kY>p5_p3BKvkT1^Qy+*{A?%LsZ}2|1d)^R760?T?(InVz0q zuFurT5t^)YUIwFP&ZmvGEd6*M58$TI{4J#B38N;$8JOtez2X(m&2Z2zn^YbrGrhG? zKqnK4hV*OJ$)=^9b}}T@PmLNMX6{R2HN?MUW$oc(XxMuZAmQk7|?!J zOpHf?fn+%tal;Tz6_%AWLUCCGV;`Lc`k-xT1LA5+4hogODs4^2}E=8+OX4yFN>lvQ<= z^W4P?vLmxYAdc0Fvx|JEcy3Fh6sy&e21g}L(|T{G^~42l7tIzC@nfpGs+u5nbx8}w zFC=$d8lx=-tV(Ms z$>p~5JuoY;bfVrD0hXV=qmCDWU=z04dfIENp^O=nz@{gaTA^vNEMA(%3lG>@JyfaZ zl9-@0ldZ668_HEMU2PaMHIb{URoSFzv7~(AQ`$P1HUlz(>AX-?=&Gu<6wk&|SXF>+ zK3hfx{{B^n;Ro@yk{QfTWJcxwtFNP~uXj*aUHPC6auF##U) zZKk451En$pQ%40)$?bruc&XYzHlxzIz{#y3@R6wW`f4wamEIbmcJ(?_!PbWp59+#d zB+6RYl}}5oukBPx3s%vi+(}ZmBKL--M8kOgp9?4o)$GE>(vk`6;L}2l5G)~j2TU^x zh>0#hsZx)?W4Y?@3(^$k5Tj(0g|t&(c1bv}QvnX6D(I2PC;{_}6766ak#1YK!t1QL z8m_6XfmYRU0pE_$s5+mw6f!}S3yb4&hXSP*^RfU|Zad^o|4mnpb+ua9mg)6CW+q3; zX5aPWQnKK+Z9diOgapPZV%Cqyr0vYZ^QX|z#jPApm$#=sLKcxDFG`65r}YfS8;z{2 zYE01xq9kVvxGH*V55pVUS)H+<*{BwJP@z`r&?0v&FTb)1-@FLpcLvH zlAQ>t$PI3>H`GCS4VH*4s(xOpXnCR$#vXFq2+0rsRfE2seA=5zW1|)mx7G04s(3i<1zw)oe|Lop%2u7mRAd^i0p^ zE60UKs2f7DFGDVh)Gdl6?e{fVpr9qymVmtEJyac2LrbOq z-g&)j2{w6?ocW}HGcQT-HQx&Sp}sbbTCns#;A^3fE+Jdgr^z(?-TAkmF|0y4XT&MX zoof`b4awfeL5rIFJNfi?u=g@5Sm&akPh8eb%}%5!`+KM`qV>a1E4kwMTSg=2aY;tb(0ip+3(e^Ydz}gT|x%J8FbV z5j>v!-)@-p@xh`z%Wm8KI>_baRh3YwPxI3=cr@dmYTSca_>fO!vXm+h3kWrpv@ol$ zs4A;Z>Po9rQ0sd6WSF~o_2&7owhn0f88ir&W=vXNnlFNq^u~?DSSn&t;epc{m>LUC znI6`He@}#$&{T7O-=RLLMXu4))R!x_lex86Qc4Y~af}^lrWLqT7c8kSBMEJYbL1r; zVldnqD6&dVw-@;y=ZjbP<_2h?Xeq0j*~M{^$O@g{yt3ycL|ew3n3$mIj?fUy*UMV5 zC_7V@&&R{^O@Yso?D`_BEk5>L)>ymtj z+87cPh6QV(oP+QpA{A|H9iW*uC)3G*kem!<+UG^9ksLcm{O-PS-HJH8y%Hdsex}i5 z*UwyLY7IWE>0^heSryL30*q~$x7P4~U*7>uQK=F21@z|;~?EX#SnIX|j= zacvhmHaI+Jcw?-{u^7xC>&bSU+3EXxb$YISgzP&(!e58nlVmvX46 z?ZT8zGEuFIvSd3re^>9(^h@6@tHrbaOkPwim96w5+^Sp>8DmKmYRL?t7WAN*lAc!C zIAY2n>ugpWVmujZSTi4K?6OVg!>&}jaNVb+38K=lbLz9x!m>8A(Hh*hSwhh&)wNdx zr6IdYYZks|$6WQ<>I@MV^{Z|*RaVxrs5lpZySbik%_LK%A8T&~;XOMcg7bDGq=o&t zl*{U99)oQX_ma$~O@MUfEN1d$T%7y!e4o^Y!!l_?1F%~e`cJgw z9LT$%f)1lN5%@#|?uiw%V#^Tqc#KhDvTEbtU-e2!ulG^mCg9aq|&S(RyDC7Vj` ztI!YVx2o#0D3!4F{MAi7Se2GJ2GUWlTMF=Z2vmbulscu1iQ0@z61|C(Zc8CblU5Y8 zlEjDe{rqv%(!TZ*krEtTYr7<;L^?00wV4?o9z%wxJAmoY?*OAh=NnI2za~f(cI20e zw}gWk?p3a6LZ0OCyYPK(*ckLqST{f2J3@7bKcRB{ChFMgs-DNZA`G49`Qj7 zdy*hGpP^iMdu`qsyR?f6qG|*;v+uV#$$y6!L)#UR@!7F5gZ5~&GAB=W73bisUzlwY>kHSNQEfsM}_@%_}=EzJH;H}jw zo$&8)imuR4v5HS}D$>Puas%Uwme7^*h`T1qOsm`}Ni zPkQzJS$N*?(G~V0JMgaEt1aI7Me-2~@It)H3GlIrd9m*b;=W?Z-WccRtpf$%G2xYY zA)Z>;I-ccijnB*rckEPucP$XEw=>5^1pUUicU^GX6xemXrX6~bbL>kWa*n)boqLmV z=&xVjxtHI}xOXjM@yrvxkF7l1rrumK1IF(axJ5$W!&o|5TPN}K3ns^DU4nROw}g;U zF8&nVmJdGW-&PKq7oO%rQL9;Uoi`>ajg%9ysiin}M#riC$x*Z`_pl>m({tvzLAnnAyU?WGRF9Zv3wSm15Cvb{nN{KOE z3k)T7QHf`!t_=FjwiCq#v`2V#-wW7B;^I_Ut3(^g$&-)PVU!!VN?_u^Nc!cq^e^LG zP$CfbSs-J;7}55ywS<{L#*yLueZn%7{fq*@Mcsga6t(C^s7DuSe2 z4>n!c8UGVQ7g!E-Z}NF92fOCDBI`R%DJ%Ht*IMU7Wm5Z)(Zd)(|myK4~MMMSqT;U?O360NQg zCYzB-=9?BFhk0_tcg!3g)L4G8HXjEl>#YYrt{uK18@uxlhD+ao2tN^mzCrt6(3}sr zsTXYkcla8${GWZa>Me`AN73~VD^UGA0W~fUH6}d087o(%Pa$`1SNUE_ZHqpemWzKp zkM3F>?k()%vK@rAEvn#}*=-lcu&dgt7T0~7RnwyXb%x=d0`^>Q$aNjQ z6q4T>j_}()fTR3Uj)CT-oX1ZtsU6w6)6%+vGXw3Gpq>7Aem%q=vdE=yQR`RUrHru* zmT2;MPM!=Wjwmw+HdW_XywF`Z2BzqL9P~>Jq5CYp;Mo9v^gS0ar$|dFU;X`Xp7aE5 z^WXvPcg)Mo2SU6To@8NV(j2bZSw^y43E!4pyz%?@oBJ;C%#KDVaiOirF=YL zU=D^V9Kgi;Fd|5Ci;MgU_^anaY+1(~`#g}=uElZDKi+ZF+Oa8znTT5VRt3sS;SRMb zDXoyTfy<)}AcwKWKT@$ptMSzpHC19KROm@j(hSRlARXy6bS1SqV!nD}lsa-{4@z9X z_e~v2;M_CUzdymEqursmca&q#IOG%nu}?6&gh~}l!{?-dgw7=e8#tmqjEaF%4K3xY)R<{@wYo*_hRkU_+v{wr2uMsVX%kUYstW;Q7EhQx>4;` z{Y5Nu65&*gNqH{n`Rh8N>rz_VE(KWpoJG|l;32EJxM_z{dACYxtbpef_|9d?jfH+$Yeww<%&}G6dH>i;iABY(h^4l`ezv3 zK##7F&YO|c2^sc8`b$W|lQ*&LL6!x3 zU6GGBc2B?C++=s|&Ofg+V!P6zPXOF~*+*!8p}u=F-a5dZqY25u6bU)y*$cAFk>^bV zcGE`rSqB`q{!;HPoCTU5yYNItj8-71S6i%J;n_rapH}Cxm63VW4uMY)Cwq3+BXY*k zp5);V7n;cbcl(Nt$^mv@RIb;ND!qr+9o&|QIGdn00R=&A$T>PR;Wm_F&VPdu|jAF4h0 zLbh?ugc6=8uVOgGI(GO@I;32 z6@MS8zxsrGed3@8u^dN~NOXpqWcTi{r2h5EczK5$$VkZ9>lC4_F2#$@fY~jD4IVpvKMKkuJ^l$A+bZ zi`&*jWL2+Sz!26)h?~61JU6>&?e`7xIrpCa;hp zVRCTbl>-l~j8ML>g@=%58RMZq8pi@lzZCcXfwLiD-rE=mdb&<7ZY@2W(RD=2Ad2CN zbS}@rzywu)k(k}~2*1wopvZIl6pF3@!-p0@tC%K}Em;>Y+RCbMNH%9qiD6<_o}jof zv%)9|7>NiFfFr@cGQ3m;H_-wgIn-FsJiX2^xI8knR10iL5-)a@smEIaGzcR1v3w4o z5u*xE2E78;YGmG0r@T6p(N6BgHtswTeKTDH3W9 zQVTdoYl?MCw;=O$eTm2|MuZM49w8YHhdr_@9S8_cM-Jj{or0sAmN-SR)yQz`t{G+!9#H&L+jmF+BLPdcdfNE3*Y+N)a(B0NYd_iXSDE$ zFl*@+K9Nt+5eI=zAy_!ufdRP>fD7auGu{<22QCwGPzv2GXfU5Xf`m?k3Az~zB>x|} zKUz8#V?JiS5`qIWTj0)+BcQkLm*Am5li+ebwcy&Qu0CNd#~f$=L_Pt8^S@C0mzI2O zfjIx%#Wc8s3&GhyKG3gl_c=8Xfm3xB|COLzkS|onCoowcU7$|VGM>eCoKnAu|3xv_D!OzYWe>^Zk`mJ%&7tMT`IiNWx1n;hfllFi-Y>jq!f4)(e zsR5m<uh=}pU&tGs8U!LcBsaP4@})u$7oka zR8MLF_#dJL8=o!+W-V7PcZ<#nOKoNDAd#rHDAcwiF$y|!VvY$l?EsO|Dms0*Qh-x* zye;bp>AHWcnBk}B>?fRYk*3%mQn-qXeju+{%UqluEz@JiLVe{MWa<4XOwOyT*zp_;P3)p~AcseXEFa?9@a)$z>?78`N|OX^s-GFk zwmse%zu0lPd&rH0+bR(|mmpHSaC0_(>{cfSUUqfi#P!0~b2?+5cI}+Or`Vb&{vW>H zfl0F{$`Y;0O53(=+qP}n{4%rBwr$(CZQHE0`LbX1>z8 zz^*t-Fzk-eoHMR{gCA!8J=}c~kn38=-3s>h!Zg3WmuT8n-;uqcTq18~l*E(NQ^+35 ztHnu47;?Tkw8OwRkjqT|n9AT5MZ7YrQ0HRBJ44Z8ZMTpqRz(th!C_kj)BehFUg)eP$L zQa|1D4f*i?h;;IvGDDGOB5e0T=7c)QKlWz~W8F2Nk!r%3Z`08!bkQ*Oz2iuDk%RV) z>i7YKyoe!#X8Se!e1j+ZR*5^|HG>Kc>K~L}1Umaz7U9>%kuL(S2x7+ZFA`6P-^rgt z3>dvqW~qsefzA_i^N9yVYT0Bo93sP`+89)3W7GBWu1QeFY#8x8#%0pUQYLg71n=lz zCcXz!9J4P1ppgZ4&S+zKv?<&JCSy!G42)AtD%l5;$wWGzhvX~n9Lq@rBOS$~5Y>Wd zXPkPnt|$`})tDSpDs&|sw6Px56()Wcnj9l81(_G?-?pQaW}M zMb&mDK^I;c|K3EFTaiq-^WDV!nKchwR8`0N$4C(CZx8Y4l&*2b>>vff=i0^V^w3b?549IjJZ+Vm3LTJ@6L+P`U6I((DeI((C!J+Lk7%$rtubwxl;M}v5XWX+>X1sLc z={$53=)8E7YCK@a*m!!zTcm}$tvnCGOvzs3)--&qNyh?A>0Z;;^n7fK$LS{b?xdL1 zzNRJ?n>2{OEohT>Qd7pSCt2=7+f;E`>{5DCFOC@Qc>|_wf(@>ixsQ1Vthx== zk1s?cU$YU+d@P(9zD=E(dKeJ9jETL9<~A;v-rT`3OFaGz^Iy{&#iGgmvg?jJ*D;2^ z4URO}c07>2MiEVawlpey_B3jIRxrwRYlyqrvXH*HVq)x)!%FC)Ls#m&##rdY!(8YJ ziR8X!6ZL!MGKzjSGs=EeGxD1;0roCd*pFaHe%X&I>J~28T-hnTD?(@K&J)eD%dG29 zv1NWE#X@^A#X@^O#!Pt$7tZ#{`|@{V(9L{;O1dV)>nYYP?ZmP(%HYD@i%yTgu`PS$ z>8bWr#5}neusg;34fY~>m+>Yh)N&2*k^1hnc{RP=VEzTjv;*z=GcBa!z4k%-I#x)$ z+bfWCcF~rk=!b72wacUQOpmAQ_;A zSkraPLn( zhq-cwdPhmY}!4wxA)a>m_wv<%4@qi3oJ$!nD%cf{Uy>&w+iCW)o=a zmp?n^AFd0h*(9#~9$@X`WttxEl>Ay15#7dMVtkj5V@uipfEEeoFoi0$*FLGXa8f=cQ9VUjK4r0d2%h;( z=7peUk0PGtAWFfM?MT5?eW$%O*p!w+T5wj*iC6aqBf`mY$W)z^J*GBU`+$ntW%T-7 zmN`m0b?8`HQrE}Yi+DVyv+4hd^gk~TBzD1h)Q^WBKjHt3WcW{})&F{V{*$b$2?a=4 zTKndcST8)IY90dGBLwl&75@iAUIad0Tts6bl`+pMKx2U=i7z7=@4v($Vo&KGTAvpx zjeN#h0wk$Lj-SKHn&mWYvomzfN=CDLV$yEMnKk~Lc~5#4)H_r2`N(s^-SXo^;F|sX z`50vvlLHhhbn!3?PukRMR%K&jJ9cjW_J`ItxoWt5^V-S*1cms7A`^90l^dF0Gy zqTE*Z!V&_}bgGsmaRnuXWr>EOK6%pICWnA{Ql4b8cA1NZ>Aam&8+dBmQSef*Wd%oD zc(%A`1<84+TQvnibC7CwfWLM9{HeX1b0b6Ni_`3fw;x~GjapK&edZS70$vJL?EnU@ zRH-K48q0RuYD+RH-T+%>1NFAG-Q4etvWVgARbP3<9~%}$($d_@>2I6EmOH`{SQp>_ zZ-TNWS|8kj)h%NMaHE2iRrhkqSG6I>Fa?uEAFMW=ErHYpIL3lWjO>JnoJ+q3o z2Z2-?H8@>RJi6Jf3*mr4MZT~$6dnXhVGRoB?9o#iE=Rh&b@1BF6@n)!H5kL+iByZN zio1&!woH75C~B-AlV1ST6bTxq`>z%YbkYbBl7^?Y96HlHC}cgyV6IDZbF`v1On$#} zh&;DkANC~4ptFlaPn4Wgj3sw6vidbboI@B$>Kc)?IJz4|#a|}B>4v_s$*~j!8+h|P zX0Z1n21Z+BdPnULAjU_o_AI7e6my4g0D0(sUs4K8=^j2$I7FBbRx-yQE6OY!QDlPc zP{qWvI%nbPascGGf&MXaUnyaUM3{X8ktE!tz=~kkJsf0ZBaV!of`ln1 zzHp*~)+fbS+@{OTxeQd7b{@uIEBKlbNLmmftquR_WEauJumc246k>b@;S8ZZ$<0B? zrZ#`V`tgEG()|}y5E*rn@!(2fqyi2xRf=L;(F$CeK!}tAg<6EbH~p^fzi_hrSO5^hQs+_v{cUa(tAQP_tFo|ybZCK-6KZo(0|m2xv4Tvl6>k0* zbOJBAAKB>U?2r(y|9w}dl2wfPt@S#8Tg9S}^}D|gj&hYiJ7rT|RaebWtGlqZu&lkg zthK$qzOa1yZ`Ngt>K|N#-&NE25pM*4{6Q~K;|^@idX1$Doy z*B&~6e6si5Zs+M{qUw>UHKxnz#Tw;~uU%mtZB3+*5ZopDr-;Nc0DDs^uYTqrcM>WM ztEy?~HZA^LQ4dhAd&L0h!R~I0;t;i_HSVS*Jcj8%Y}}t12_grxqkMtcHZ%j-Y-uY* zX*-~=baJtIc-7+Q8y2Og0bGZGw|0kp8433pHSz|cwvpE0ESR|LLc+r*g(pmKmb)`V zZ|=M9ACALDapW{QuCC7W5;+0ZJO+$7om0M&Q3Q6gAqKcRc*hvD8HxVR6aaC*z|6LjR~D|oYnXpe=+oF}SEx-_iJyWeKs41ONYruIu_Q-`ft!rcbX;|h;sT;m&tV*M-_>3(F5a%02P}C=jW8aWU#5@A z6NNM9R@yWuE&)K5%y)o{BD3{6&>{hxGNTdz@&yyj8Be0aaY(iahxE9pku^WBVDjX5 z(LhsnwhRj?upya)iYb`0s*RZJFPC1aX&th}Am)GUZ4&3DgG?bBhCH>mn|CpPODd6P zF{%T%pbEzQZG|lmejF!kM(;>bp}1{+VV_-ZRn(za=6Ff}q&;bOw5vFEM5-j>(POW# zQ;{KmTXv=06$~@yG3@fYBR|%vzPTJncFmyqQZzMf z#YWd245(UEwU(uzt!HLNQ3)nZf~KgdV+QB9h)T*sX@+zPFQ{}VL;qp0C?2CNy57R~ zho@5~Il^~FlQdKInN(CE$VCi>DG*XsMj_&pQ5Hs~B*8))3G^nbj-tRBs;C+`l`1Z# z9YT(|tP)-fi$(-^U?&IDd$`kz+-Z~3A#j7IR`p-07XuwiLpmy}md6hX^IOuOJ^~yG zEikbzDWg)w#RJ&IPxUE@h4c1pijp-mgrZ?oBhV|tslsuhH9I(p@>ZMTv(mLZ$ehr70PY5 z^{Z+(1|0~Y91!cstL2jrnYOEoZOe#^8)lVxWCDr^be_We8WV?(6Bj!_tiQ&lrYYc! zbX{G~NqR4_e^+1%hOYJQ4Vb+@Wp8wx^&83EYT~0)sw+-8j!sTbO{QhU^Y{u}ZGt#C zDhaK|?RJJ9`XR|>|Le#x+m?+X`;ev~AEg~l%i$>;YSZZoaC3=(aq>t~qUYmd$}mCI z9#2h}iu6Dj_8WMH#$EULdTB6wm&^7wNKh900>@Z0#QKs@;c!Z0BJ1aHfBEX$;u0*y z7CjP`H+%mPXnYuhlRS-2MXjFXy@NgfGku31I6exjGkb4_#r=nK+Tk_0-+xtb@@$8- z>iW^&#WjPw!Dnv{HgjSO$NS1J`_&u0z4tCmoWWO)@UB#XoDW#|UZlH*HaSuOl;%lXRP4V)y_ z$vAYZ>j_s8j_r+I{TKm6NR4Ky=eEF8;F0kYtV57DPi1>*avO* z_G?qo{J-1QKcJeuRg?3C7>Ao8bAlk(dovEymT`g+XQJIX-upIu9Cr*B9C{oA-R|1c zPlA7LAJM3`{ZR|;D`Zh^m&$}2&r#Ieg9?g1`WJEO6**`Yl!j{MYzB;}j8US}yM_L8 z1MbfQF5?2&MPFuEXhGlxySz-0|kLB*Q0;xByIR zD2}6U$Wo8tgjQj{whCvb;wvi)>&iOIWqZ?+HA(c>XWKRMP+v{*EQ6NH$!;ACPFGcU zwag~c6@$q~5P^r#&T)TxX843Tr~`v+^V}OIRUjpM6h4%sUq<#UgAoou zh#JO-GcJ)B5yoRIvf*A)XTlX8Mts8??*{b^s}k$F+Ez>3{3@TRrgGwELoQIxTdkx{ zcc?HpL(~bh_vn+D%#&9F)m`T)Kd-cNhDnY2gAAtpcbNd)mX` z(R?+(z*xs|@+;!xO<@3M-YBN8y3eAJ_9rr!Zts}L z9(U0iZDC2`inr2Bx&TeNUBz3yr&pgbvkyH%pvVgpNruw~kJHQEOQ_iRtS<{@E8PoZ z=)7>sd$k@6;(T9MmwOm0i*|6fN~6f|DkZf{2t9$|%&$P*h7r5(si3t@E1?FFml$Fs z1|N_L(ZO<{?60E$?n4ZIYDd{upaA|P?$1{W>_s9lW#Ymx1m3Pu&ayA-6iFY$#6h#*+ovjv0}d!^Qrn)YrLUqfy&Wm z;WeAKs|yCV#SCy^{*9X+?wSruXOb5HS(|Mem=lhWdvtn;*yESEDqhu^w0ywYj?=QR z^@3|Ng!4e&EyvfFrQOFm2Vq^DQWvy62zRQpDTTMo`v#=N!r;!I5NVTs<}PnE3@d|+ zJCAHwVv68uF@7T?L*r_tsvlDf5o5LKE_l$FkPfq2(*`AH#d`4fm;iebdvJy&(rkXHKOE zWgOFdV5KPVG|!*7r|xrBU&U)H9& z(TCS8)=NiF9rc);`No;$SDq-C1V?6zBnT&ErN>o5t&B@HK#%{M!EeS2r*2ky(gVzJ zJ4Z;LK*703h;&T}`i`)zT{DEma3-jTAGpBcnVo`;C4dEIeulwK4jV!`*qNtt{pm4C zjpb5GH#!AX;6R(uitosug+ZJ+Q17IJEtpDHqdU#=fwuqTQ==I7MbX1r_h8s+Bw^wp zRs1%3x8Fvd0K4e>*8XNXE=v&g;~qo&@>0wO$6m~>hv(hOd{BntWqm$S;vz|X=z6h& zUz=CrssJ%#?04|!>3_hx$?wLAYuDMUK|4r@m34pi6y$5!Hb!o zFHRrGP$INXK|!2dt&tlODse@mB=`aUI53cSSwiuon@-u*xQ;6+5xNk3wk&B5HznSA zzyEuZ5pF4CRlG~WlSR9N_e!*nzAxU?6|QKQSoV{X+QDA~g(5U*lv|@k4p??S8C^1Z zOqpu21q);`vvQ%zaEAVXex+1i!*4e+JjancgpzeJ z?RwrL#7B(vTQ%ohDC^!G4vC&Y-H$(fLo4onBSbHtVjQ`;2a1oJF?R8&JtRC|;-H;! z4lj|BFa9GnY^{KKk1uR;^2iU1&42Sg|zvt0O z2M?4_LDMKyh{-gHdqe7fu^5N}7p;KskFcWz+5;c}&q?2V>SzP?=^00JlqA|TvC6t( zGOE~GqLT&bo`r6hHVv*++cIx@gYc8Ck!ehkKQE;+*H16;`ib`K7-$P0$CR9Ge@br> zUJ;U7!HmqV%)0obJ7MYP#fp=4Sdi6^&Kc#vEnA?bAY3D%WPH=-u2p`hNVq;QG*4~~ z$-A9X8cs6Q&|zXAP9G{anJ4?s!y8GCkP$84^a<@8g85c@ya;hnA4$~c@1(7_1*7wa z4$}hxOSke+M9|m;s>Vvx&;pf(`}K?SM|4i{zXG9lcGmx|>Riaq*4D)6KcYT_42;Z8 z^bDQ;6Vp!SMVvh>AV5%DCj$*ZZoqfv~< z#2pdMSE7D{NSd!$>BI-a=dx&Npy4+|SPaauHg2ipj;LFl)io6~Zb`1VpvmMszw=7v z7zdfWejZ+TeXrc}et&1wd|$2I|EiA9$Qnhcgos|k1BV4WA1Ro#_kLHmZ>7NuJM|ML z$F#Ty6Mf9vYej+;7|IO#_Z8Sf$B6)yYIO$^0s8{?b|EzP#><0x7b4{L5;D+^L@SV% z&Y_KW8xutMe&O|qz*8&+)Siw;9f>vJc2UP(1nb-kRVYOtlRk*_C2rWl$DMcLeg4>T z<@vAS(+(;bB%boBkdcB1*vh*&_r+%Kty!wl5)rw{&DsX%<`K@t?FssovxDoMqMj+m z$IrXorG}%7bkiSbP{&?!kUcQ=c&g*W*;%etgya=pbTW@PA@HUsTGV8JNf6+3PeQbb>5eGz1TkD^`)hNKG!gU{X#lKVqt; zSE<6_B6G}%7BSMF4x?HtS~Xj$hnvW5l9P}5ld(0YT{K&fk2o%!FVo{eWv&vGe)$ge zN8w&7l0HC6{+pI|%0Zm1G2o$GpzY-38+CEG=%*{x18Qj=dcY5g72Y5FwG{GMnBFZM z8M(mhR%qFR6yTrkrohcp7<}yIkPw`4i|gmqlZ_rqg-=^vneNr8$yvD9Wobc>a`iBB z5RzY)i!}hwiI97o=EZ!9P9XcZK?==gBtlW4J#q{R^COy)vN+$$sLe$R9D zrh~*_vBflxw^eulI2zdKD6IKEtVhc_y5;#jkILZgDzpgP^_$uhh0i2s$lmWS)G~TG zpScs04U_@tu%nqOw4QNOB}pHP#i(n-Ddc z97P70O#Ubg-qfD*J-{wxdC6&S^Qst?bueZu+!gfccDQ4f(fT$`Dlo!QWEDS2FKZKE zG#vVsiXE3e>YQF)^DO3yjko#1aJ#XNCn+r4AvN?Ay1JyJp81?7BrqLyCBO6r^tf(a zP;jXh1$c1vOW$EZTaAPVc6RM76r@jR$Q$|!)z}smlf%SglhUDl<={{2)R|RF&_kDM zbrKWjyvr|72-4ir#BkW`IPR7S$Y_3W9pf zOXb~xf5{BgAkR*G^zD+|RP?@E!F)&Wkb`FIUIkMFd!;3M0r#lL1GdAy_n|18a+%_D z-${p&JZDuc%Q^Eld{@yhl_ascA^mU90u~lJr4po!thZ!@% z==7FIx_G0ds!3r{LO)s=l*wQB9mADs-@a+}St#InJtURS7|G<6)mv^54~@3>4~b%~ zWUFTYzgRg;c+045>PKFPKlkHu)3OMpn-PVA>wx|F&z z;*B0-yV${i8qsRVkjoCDTLrJuP#r<4H2zbVU$Ii8P{}0CwOMi=cX#<^(ptFQD{z@B z*eYH{>N1WPAQP#823s!WnE-_+U-TS7yT&h8rs%a=b50m%#j?h%HQlUrK)W`nVuUKa zHBY?4QIsYeAN}~riE;cnpMO3%O$~h%kG05v`lOR91SvLIOUZ||IX>^cRwO@2DA;US zFJpo|@*WoDOY zQ63FJiyHkFKzcHzYmjo|r<6%bWgY!oa=L)61{N7u5Q1#PEkgsVPBx|Mi!r&F4!de# zLr7W~0k7yH>G70el;wz{%Fx(0DuA1*{+AJ%AEZmZYbft_*btzU8<5!FFKQsW!>cCHYoM7X1Atavt|CxGt!RYQ6j4&{gb8nOY^Cw_HpHQRuS-r+M zyvkBKl{!;2Q+Gm>>j~NZ#{cxDB`Z4gLm=x>xt86${h*|SeCv4pYD zu}?f2!=BPB$)XgC7LQ@(>@?f$_{;d{f$LRLS3|beSv05M{X1x>Y2gNMs6QQQRbA;^ z{{-D~I?wb5`w0#;#MIvEk@-jN^tHDfXo+{6W3kB)z~@eDl9u8F8eQ)ec^9)VK^7Vj zg1(?+K3s4*Y;<#9anKN~f+bCppc+-GmV=OnpfY-!i4A-kVtwRTgy%b$91z;ZBz35F z6dJJe+j5m9itj30SY%2AAb-OhO&-xOB)|75q-4p0F`>Ynd4-O@&%NWX>`syz`&o^;t86gkZ2Hi)6cx$=nq68wM!bI zCYDmy~ ztdCyO3POsC{&3R)&-izbo z$*F7Oq)AQ(S!UKb4a+6&pm|_lI_U$cl7*NApbdw6>k)5~pY@fOyEd=eWxc)+v{cs; z3sY-s`yZZ!dj7LXy#osW$Yw6wVGPDF=HoPD#cV>P1U+h225WC{6yl_M%LwuC?0&U~ zZ(GbAchH?|<`QHM(#eEltqzR4Yt4)?K@*lDkFu#9Ia^hw?R1LH*zPt{Og{XnuhtRa zORun132U|R6TV1p4jWDOFqZMrKGBhg^A2#ntl^9V`%4NJfCZxLm$&UQ1)5$%Tfr01 z@-ihr_COB?mmsdQYh;jCq^b5DujGgs=N(qcitQ_5=e7TG_}n@UnMCLjS?HI>@LSYc zl8JE}wOCMPuSz{4)?k24JtR#%s7^gI0c1jkEb^;sg% zxdDB9+R`z`-6UsmY|62!GbCu0W}ZZ-(g_H((o;elc=D3PXu{BD12V91e9YWrP*22Q^LyN}fa{?8pk#@E0PL$%d{;?pF1UN3o;cj&!D z%GmEZ6B)*{q{vt^S#=_k`nsBfj0#N37zJ!)a&+B7!z{|3LotC>aVCyxLA3fVJFepA zvzH!aK?F1YJ|k|lzWYd-ofc4au38MLv`t@5$N?JyNZ)4nd%T0^aX{QSGT`NqC7_Ji zwe>(2J7NIn@*24h8?9S=4EvFVdIn7nvD4bj+b;U&jK=R7sscTxLoDvMkEGs8S2H1A zgh`l(gOryP4J1d7L%(s$XnH_S973zs^4^Rx0f2a|q$IbE74I}9ERr@yy;94@N{hzU z9Y-W4sb=^f)I72kb2F2Uh*9A^Najrh>52lvSr55UNAJ~%>GaD07*>b=GGFV3A5HsA zl&314pH5br0#ZI@V_n6axKQWifm5UXcZ_>)l#+)V%wJrrenk>~MXDCYg#if7sxGg?I!*OgO^6OA5r ztSp)BFH9y%Lg_N1I_X5Sj*fNIWN=iDcM53-{Yf0Q){_*%P#aks_jOcec~!LkWY*`c z))XU-l4-DB*%=_TjdS9O*gkRpC&7dt{9PC4hnZCJV+P0Ye;>+4Tup49r7fJCO>9jZ z|9d!B-cbB;htmOD7d31JPn1%CL#Dc%ywo7UOv2Y^yJ7AucXN-2nre;4?eb} zQteyj5E3iLcItNKoq|svOXF~h&hvUP92E~Hh^>XDEx@&bOrM*U=i`pI_%%mqE{r7P zL}_+6)Nh--QK88y_ghOTx%!AHJqyX}n|{~!2s`+8(Ra~7vzVPtWD~W#M7N*hE_nbZ zcw2Hyg_&5_{h8u)#742q&cWNPJD)mnq&@I=usVzgMAV0PSR@yEGQDt9J9EB2!XO4F zYnvBtfww!c3J>T4;xIa)NjeGSL=d5qk(95)=Z+x@OnuQMCDl3~XrxIkg=hMWJ_XZw^phyaQr_;j42m5cwgY zgFX0w;MI$|IV9+8zwLt@cz3VMps3h`LOeo2=UyX8$8c(<#vGChYy`5-SZ|T__z&`{ zgddn8GQ?e&`t#7+1w+nqOr01*%2*@slTu?P)crG@5xrQ7C;i?Fx%^zNF8jJP8w;fi zB1v6H`LChfDcQbL*TWOP`r#eY)c*aSIw9acVvk}fRV!md{8-y+ja1tCIV1)mic4fL zkj*nB@v&~LN_*H3Ss2+Kr6<0>{%5VlpiLZN|I`}Ik8Kv?|Gn1ae=sp&17`yzI~PZz z{{;zC{wIqhfXri*f|?EzC-+-~e^8Nk84kH9bUXzW%l|jS7vqMjQgUO*>K0_TU$-CJ z9-Iuc4D;=80Qo`MmgYFE{;IL7so53BbGFy@0_yNTi%R?C+m(< znv}aWl8ySvOsn-}(3WHuwVg()qs&{@i@{uQ=wSHY8Ku3W)nB@Nvzcxp|Cl-bTt~<= zJRLTE9P+)BbW!20fU1Pqt8LphJnIaZ`+NiWowpmM-30LC(SIy3lP^0F#DM=%DmOxE zPo^f<=~9?P-6dz!%nOMPPtWdKx!`;eNzmbUui9)~zEzo}79 z7A#`kQOtIha7}mXOhY-3-%w19viXA{`RMyp`Ja@bzI9daQ-U6y2ZvjsM+)s0hZ7J3 zqt4`gBBRCAmgQsdFi1m22A3lW*J3(I>uaj&Ahl+)Gc?o-?YDszZIvMB66vkqb2DSc z1Z5lq(ub<1Ek)|M>#*u*1a50`vMemKmTo=5Dc1WmP@3Auz@MNW>V5nkd}$smROap# za_!V{xIhF-wg%+PH8xMi?`cqmHOk#64*}sq4Ppnp-CcUck>!9Oo|764itPH4^g+9))JhU~J(NV*R) z1u#K~PIo0Aqh`Zsr^BrgG#@;zj;9JV^CrMg)Ey1$!@p`=>Px}0pF)EH37B!(b$MO) zsn+I%z&61`hYi`dxh!cP(FoCVLXjXklPrVlnoa~~hyjM9ZvntIR@L2hUJ3-oqnHAkP1BfCo?CQGL%53<+BYkfnMqTjY^7h3>AwhEBNJ&8s<>KEFNR+~ugGf*TQHV1mpQZrFYsYA947*pi5>b%_6p(we<}hkH z9~3)t+4tGI>|5 z2X@7LHD4AlDkw5!K*O2FDl^!c8wod|vcX+gkey5KNUit20e3cg#NbI^fKr-ED633! z1>8Vd$1Sg!3(S~-uNFz8RMMiJGMvL6aBMY;J#~yIZ;>RAfHB}K{XekWlHd< zv7%Jwc@gc<9bBZ`KC%su^qz_XMX;Xf^lAEa7Y8}(>g%T&?kNu-uE@x>M-M+fG^UNF zI6^K%SnCbNOgN`_g)jz1I?3nqAZH`r3`PW(<>Pc=0-fFp?J~gdk~=m^1gK*0F!E6fu2$v2&48OvZX*s=!v$rqR1k)D z@m^ZA;-E+mG`L5P)7KD6I!CYAJq$} z4bY@4F;@}>>w+ZKnzfsAF(QO8V-F)(ria0Y2Byd0$`b&xuc>`XQKSY$-UkX>^ z0f*`Vam8$+i8)M9o15r7!HVQ~fzm~CxSfEhJAm;U@0nMm#~Dv@G-{fG#e|BCqsGcz zX!wN0%#gWkmr%oGqmj7ON?byoGc{MnZ_n0dLoFhi03(G(T4ETpO=PJWU&vQ99DBsaiidxG`}6 z9k0y-dWBg#Ww42O7>xk8Gj*KR_4aM)ha>qu-_U<_)ZRE??zRDKs{iA)-!Mw)q9Y=4lhq2YOV8 z)&8bkw!9WV^NvO*-c)Bs?BG;kyF-VwOW~uu{_1?wrR!tI{&a7(1DfPyGuD}KUrlmv z%PZ(p@uDj?Vc`aA9m(ArEY9CGqmX;g>mSGe?9KgE&|%KB)3ph`3Io^dkf7s?TwDdG zWxhm7dqdoOr$^su18vPUf;;WHYq2sx+h(v~hh0T*c69|3Xo&MeS#+HwLc9m@{_TSM zxs!iE06NXv+nxIC^)R=t(QsAr__y(Ce*K?jQ%L9>r!d{Nn4#Wm$6voxLc=e8vE3n$ z&c?qlYpLG|?E#Xrd@%)3q0>YHH-m`wMlj)Jg=Ktuhuoy292?_(9{y;WektI^T7IN` zXR7+mK9!Z+;*opKz;{xsJOWL_t^w4DEvkJu`)X=^C;sDIxzD&23s=i8SeJ)fpmd-V zDFn+e_^f;IR351;-2Oc1!@sdf93y)U*vCwHqLJ8R5VZBtX!R3qjDheP;(Vf*UZO|g ztW$e-5x?~9K9ESi48cCIH1_?VX zS;`~J%Pgg%xBE8~-LSWMxs;Qzw+3mlMWVNdyp(g)-g@5`tMxYf!L;{8T3SVOAZw;P z=0f0Z5k}C2(cLgwavUnSf6_-4bo$9&67Od$G2K3WuFBt_)E4x<{x{9UR9<}==1)+F zKk~0%#Q&#G>Azp^|9ZBg)I6LM)lj~9>L=-6&Gp5B5}QFO38a$^>WbjUlM`9lAqWV< z$`n&40e%u>OaqgFMa#{GBK#ZWLz)#;nkcjRKsKbS&Q3x@O|(@@?Tbg)-?#4ExHAed zB_liDJ1J>nhgifMuiLIWT|Hjc*IUnyXH#Z+ziNSeT@8Z!UG@;Z?cM1Q#&Hx71K*{y@;X%nfG$1VkYE%91RT3UCT)jr}{ zdq$*1{pUt?`ufP#xFOZFybSx|z>{1ugiPlngQPXo>|3H5Nr>L33j107B-9Of4sx^! z3sHk0@AU~WAw=f8#!CI$b@GgrJ-7K`E#x>Qu#jD^qe|wp`Vch9Wd;*0Y|Yi^`vqk8labB@|#@p%!yGYYRJCcOuU8XFyS(o^6b0( zAkl594MzKkrVztSsSu|;;%@XQlH25KIhD zR*iEpe1_Rgam`m+g*Nq6a_DU8a)*LYF!k!B+iJ&U+nTDpn3GK_Z8^{xkt{?fGB|G2w!(s=As8{FJl?o#0droBT3rMe_hzIK7r+VDH-c$;Zd2^~D(V{KTfN z(b3T9V>qbaoqq12Yfy|K*cTFTlC=WTOOmRr7^y)Og)5f1hz(nxxShwAsA_Q&w*s zkv5A#{9{{U`NmLCNrv^?^hAf-T#1;=B_Sn;3mH|;RN#_LuBnFvU2-HLM0>SL{=iy^ zklh>~7?*<0&oZBZ>~Z=Mi?J5WYfL*_b24k3o2vo$vcLB5NH&NCHmxTf!(c?*K%#y* z)JKUu)0t?RTilXIdXo2#gdc0dtokZ5LV-9`xI$2oTA-mu4nCj$E7Bl|jz|EXvZ z){@~~sh?4T!nFy4Q^9Z)+^usE7{oYo92g8YF(C@FMS~QXm=NlpO64SC8O#j5Oz)%o z9Q^q>jARCf)V8N^Aa=yF;{N?r8|$!UZfZ1 zf1pSZOvCDIpE-=|LyUhOG*oYz^3R;|&+fVDWaBau(fSfVWhF8<&sv*KhtbLcL6&&L z87_t$LmCjM<<4SbRm&ta&m<}iBxQ;_6N5d3N&{KeDr%@CMAe>#%f(Tm9mCQ|#OQD% z7!2FrrZTuy9KBCa)#rOe+l|#co5AXr_|@qR@B>beHLD~4iAZ?=T-d z70o*jI$B&1PApST%4g(ik4r8}LN~9e#)t+DYXbxd#r_?}R(GlUsiN<{ifv~u6JEM4 zPM66k?Y0${4ozFjt28wZTeP!|YWHpgKVD8>(ArWIY}@{t)71NuvYN>`BIg#=Pu-6| z=Zxk6hzSnPpo%xLTF2h^W{1s}eGXgrg(Y<i6ns%L91Fd+33?thI6q3Zc|n%!d^P~IGeF!`F<;e)fqLTf2lSa(`fb^v6C$yf zg}I}b3Is8Xa6R1%>eFVzvMxHN5+Xl zV`!M`cqBe2yBXyl0O970mJ5H!0F1w5_zd1dXHV`z`6;P8co1A$RDA!!TAQ_3^-plB ziw*;Rx&{x)5`Yf5-;`Rr@<30dvv{VI=qOo5dY-1|PxfkY zq!zBcVF6!}9Bao;sGcv6X1pFLJ#AIhGYZ={FH$R8qVn)ull=6APMnLN;h-CTJPhy% zq-t0q2XI7~cSV(I!`qDc-JY(6w4Dg!#@U8e z$rM{Hh=0<00zklLaka#C1V$tWj&}fNJccHg2RHWj%BBK3q`(pfJ@5dz@?UOGcSeh7 z0K}6($b-(H3ckzYHfm^?0~}?P+)~dak8WglU;P|9SJ&(jP-M1=U@$rtcS_$!u!^p& z#&@+ixR$$aA#3u~EKXRKi&i`gK9j-TVQe`z=?XuC`P-CMB0 ztX;6yZ~%+fg6?n!dkp?IxYi|H1<$?N9|(qJ5OSJhiy4DSUzDTK4+KYWqlgTS5MF(N!$v zyR_)*f$x7##_|F5(O*B4u|LeOUzGo+-s(TFClllUCS(0qhgGGj<)oy7{Nq+zC{i$X7%It?B?bWdWD-D$y~JoYHu+~ z*!`cVA0-D}l(Vp0Q4zsZVlNZIgam07SwC5R*_pF5BS8m9bL6>_D03D=QSVTcwI|^# z`z!co&{YSU!Tf|eCd?a}q2u&)-5hL*YgBghA!_2@=7XVRj0n=W7VY!Wm4#}WxGY5l zk^)h-{+CvXal0?f)i8f8_+40(+Lu^ob+@s$?eE{_H@VumocUOeL4YkHe| zgSo**I>|*kf=_C7a+&oz+;xRm$`?0eVUkMk@X6&{e&^@{&z7-X6V~?f+e`OiQOR36 zG2_w+5>kL$`=9Tog1w911&2=D<}0QfT}%`mVT}_b>Rt~0PSQea;X18i#FEKqXD|`? zp^*d+unV3#r8V~E337E{cdX0+ugF0pk~#J^+X10n{9PBX1XL*NF5f`vlEkGP5u(gD zO7?$>nyRKNT+DKx9yrdS>tc)a0QqR=R>U6sKYYDobYJzsP0NK3xkmr?I3Gy}JZq{mU|P5fXO%nDpWkV(shb9|&DFHM(@s`4TPGsu+}SDu>bj~l87@+=HEp6!*-LuWtJabmOZKo1 zE4GgjHiX{ke&+gh0~Sr8qTUv)LBGN{ohu4djyg&le+SuP+XMt&=J692rA7OGx8;?V zEe>TC)(CLoQbk(KhCLg$U9HJ5iV4yQ`n3kVR&^~PjxTb7U;WDn`|G5GPO@sNXNd?f zzXRez!Es7#PURC|Vx7^wbf_ZL^I_KGZyeV`6^LG><|Q+!9j2V8R;^i$=eb^gkB$y` znrVwP5s~5SI3v(Ubqnn!p$E+Fj~Z+KYSq!@P+HjE{DqiLrYS5J@k&lzaaB*YP4CiB z{qe2_(6s`1#TQpnnx(1GKsd*W;#S3$qOmBDjMSoH*D=Qg;31m$g?PdbYsrJQ$n$UY zV_i_#_Y1H5Jjxbv4}+7{e8JiAOvG!*MN9}~@}~IQ-R}{rwu$v}OmNmwZ+b_&crW9! zpQjsy(SJX6nsgKg9+A97OIRrNi;SDbDcU{8qA60JXnL||qnwz}zZ}hvV4|(0fD1ox zX}yMVb<*#GN+9-A_n5G*dW46he<4~YEOyF#TgJW^ zc)3sy++i-YynfYb#~f%gC@L?>7+Xq=TM0F54ex%7u9D8ls`PgVuW`d}-GZk52(kJg zo=YG{M@%-Zft_2qRkWfDx_UYp0trP1Fp^Ybakt)8tfp1O9JKHg)KRqjni|;MsPlwi zeKDgmn)0~7iPxo`7aYC?1_IKi~?b*Kf!Xe}c3NnUIjN@C(vhdyCUr#SY zojJO0=o!39y8K$G+C`)qQ~eXmtafSlQR9<%;A*RYgi_ZiHuJcC*LLJ}#Ps=o(|+|G zZ#adV9h){PbG@l~0bdvAlD7_+C`d5aZ z!9gZCXvQ6ia!j=uJeDPBl(q%r97NMYrl(6DMdnEsmUN8k-R1%Mn4cb;xOH>hDb6H_ z(_%s^qm%y6`R^%zEK@7???yT7?GDw0lX`$EnGt^6N3RdWA{K7*RNTEL5R(H}<5!fJRpxSoOB03)5_lN5}mK-Ca%SRHs) z!0)`%YBcgB0c(Y}wz!Upi|FMpl1CgC6j>SLNAnN!cIhdwHoDgfkjhiXO~XRT!-7%I zu2X&cWrvx|pbBbm1kVAmcEHlqO0{%qkO7rA_K`d6k-c(>K5XWXq}pkUG~1~*eJ$sp zCz;Q5EX|`BlHG|A=u|ERNJ~S5H`@!0U3OC=mb}|Za99#7Kv;zIL0hVN&dY)qr-G=v zuG0{0p{Ez-?>qdT{CL1h|6x z{?yYKGJ7LO1|iPE<7v6vTwm4{+;5oxAfz`9D7pG$%?OEqyNC{}F{sr^Kxi50qy2=H zElFRGzDBndJX9GTBO{zGwZ^`5hY7c%U%g}_%o@@Wo~>r|XlGE579CzZ@}tb*%CUjC zZc$RqwDcrluV{*njUO|Czxm|#m&I--zm#1c!lr9g>s21X65fgU?ZYmC_tn7#Y!|Uo zmNcn`y}>T|L(E1`FFLR0VE>pTc_g4AE(1rW_XfZlZ=omdnP-DtjOs}|#_6cXFu%BHbFsTsK&r*E z7wC^>I{wfWzQEZq@jB#elvgg#^e8TIPfyMIk&fSBQ9MDVojD|Qi!i4g-{2%X(MU?- zhBGxU+%fJ-Hn2F*jP>??W&kF*I!8)a`ugsi86tsLS{+y$jwwO5dDalMV?-KTKIo!2 z{43elxo%W1F{_wYE=nIHYli6@eoG6yO>HR~R}ce;6$Na7xLa~X`yjtHiQKz~m#xle znJYI=)(~eaw=7dvT4@!m3r}H2YuxG?(}-0#HQ6QCN4mp{8QHs-352rh zS`*p&GY!dBxX>k*p}vKp+)daS<|@`oSLZMAD8%uBtIPa7?n?wjabYZG~BW{1~0Lw?p-^a z_O1mp&j=JDP6G8ht!V0xuE33aX0`yC<*XSnT!rYfd3LQ@{EECkWrOMx6uC#MWX+<_ zv}|6ZMLBe_Fis$8T&#BIK0qNqD}?D@U>k&t2J*tVpcz%O`5> zL0$-`pqF;3@o!2^PE}cgDTYx&3-|dEg+yk2YVkl*9RY5o`Q_qY@cB( zLx;AMYdm7m)|vSfmg_49>l*gqGe3k9Uci7PGEsA#^5b&nOy0n)4KTmneDjSALPnh7 zmpkpQcNdG~$`Hxih?7JIVv+@7Av<(AyVXq)L5xe-+K9!no3(H;uYoXBLji^E(?ngZrxAQAivdf=l3K(13*lxcje4vWbp$ui#gM8@{H*Ic(Gy~}$DwHmVvOChq}8dL2>nEZjE^UIWyRo_l|B2n zfXz_n`0q1cbw=>TvcG(LzM0xESeY7|FxdYeb@pFX_ETj}5RFfVn(Bu#!NEI9#Vkhd zHt66Yq>WHdI7@P`pWT`ZT`&6yY3XkFEt86>+1vLI%Hj4k;X+PX@{EmF&d2jsj_%L5 zH*iC2EqMOmFk>=r)p5*SwPAXwrN&IR@OI9Ua5?Dp?sruqRwoOqD2wXUBqy{dB=QE_P*R$q{|fhS5S}9jlWC zv#{KKq0)_J2ez9%__VRM9opy^jV?PrZ1idAJ zv#>~-4t#y&*(T%d9|SYgv2^`ngH0tnaRJBr90Vk#eh_&_yHMC}AY8Zwb{Uk_cM^Hq zWOaXpQe5A68j0B?2agnC(nD0zOc z?c!!w!u)nQQCvkVJN=*)B&tR z#;P(nHpZLCW#Nm(*t~G>g0Lt=?7HWZjw;S^8u|OC9-bs4H9u^4bI=M7U(99ectjdR zA>{5Uo7UXBvzMB}MVpR->O`O_V29yfC8s^|7~H%s3pCiT#w+rFeh{LIBveIH$WGc2A!sM zPMsIWH$-6&X!5sjK8ZV%zJzhH_eO`8zgv%f^E_To8Kh`@TU2yq6kmpW^=5!8@TpKK zt)pG|SR6(KvZ_{A#Y=?fRf=l1!(J2sOBw=rjmjv3`(ngy2e%MCq!FN!7@x8Q-DU^| ze`YGK#sU*X16%`j^H+ZC>sXeI5=DAa=4cLqk>m3csrt|aQzv|=e;QKiSl+JmqZPH* z_>?a%p3?Za;Ey9&<*V-I`EwzvjiZL+HiNU_=6B|;Fx`g`5afzdnu6xGG9+A3DtS`8 z3!aD|4}|k9$QOq42umDV3t@8cGIhz0Bep4ty9vafbeimDwLzYQC0p?@7u~s{YD^$O zZ`#IFZiI^^Q$6UlC?M1DM&W%*1lD0g`$Xk}g`^1iHg$qyoWNaDb=al8`;Ahx^;Ceb zJJNbJR}6P(Xo0$g@#^;v;*J|7-85@9V4JCZQoiDand~fpCx1wz(P=iLU6Fqu(=BTC zv3fLjo1AI9hq1D+{{F8e04?C3+yM3MTPO0rsgD2M6aV-7{%3ED($H~2T|(c`Nal^l z>S3=A-33DtF~v0@>a`@@9bhmX?o&KpPL1u6#Ig3PP%<~|k;<5s&>l%BfTxRPx0zkE z%xDTEXO5oZ{6%K-;@y0rt@nO@s{MYV;UnFC6-l~Q7xiv5G2VJ~dAEABe)d?^b%Wt+ z<7VI|lS9p~>cdJ9wNNZaQlyh?wAjqfL54c~H(7|!rNVrAzOecLC8H<&OsyH0)CwDkv2li}fi;GX~vN9`NgbXu3JNpg$Hc{ zqjE@|dP^KqArES)XpvG5spbaFwk^|c8>S4ndL#Q1n;5sp+Nt*Hnl|^D$LeDvTjPt8 z{{giZWxn_{Q`)>T=-&-&mN*eJ z5;xg+(H15M=+aGz(2+@s$UNCtesntuyP7c?L0gB@lGnoh{NtN-BB_NRt{!LZI9c z#|#?tp!A-l!kFUHP`+{?plyQ#KYrC!+Gb`BgsH7Z%h0V7hJmCf8XRL z7X+Rx$W=ua@|Q2td`|a*2Xp^hvL{ejFmV4hG#v5qe8 zb1g5b4Y1xQ)__W?l0ua?tk^VWEN=BQQe#E49`v$=e|s4FcaoO3co|-*Ep-TXGIh4n zjn)df)jaaUsE6cBmdXv+%G?J@F!r2Id@vwls=S5hFQD!r^LTL%hes`?b1p$67exeLR$jD z5nTR`;)YZu;d_aN{) z#J zDG-caZL(0_^nvq1Umv%KTo@m6|FMZU> zC;f-b_NyE?p>C6Gr8|7N)v(fm@K&O8cV|;TI{d@sLbUBrx1PX}uguPw91vjMTocXYilqi|0=$7s3k_Lld8CtvCfUAfy z<~*;E3IXR&q==I_@)l@yc?EljdLUddiJ4aR8(M0Xs(70)j*1J7`sDrPAdD}YnT;pksecp2=A0`x_#ZYFFsD+}(b%o8KIaxt+|=JiQz* zbpMUg|CJp^i@G+5;65u7_?}i-UmYI?uh1b6oH_+{N^1=c47Q*uhE8=DtUf3a2DcD^y z5V|}dqew@9v33L1$wh{vX(kmHa1O-alt8Xt4%Ps!Jhb6#k!k5J6OG!eCQf&EuQmYZ z#>ODv| z6etC*qq-${dCGLjJ}gz?_CJbb`SNU2b+^(+5Vqn1!hD#+4GntU5t=Z%#FfyglZe?U z93H`?Da>K04RW0k_e%Sx>|sq{?@oF#T)3XL1S#5{HiqBucIVOvpPhr#73{tF z|8(#Tp4_umKbnXAg`3GD8{#6?Cpr|%<5w6snigH5ALXVx%d0V zZA^{D+yExdPIe9|00$RK6Tm-F@PFP-{!br_FU-3jh{WgEO?lf|<%nof&#Vtg5vkBQnj|*$2H6T9Syq^|3t7CBw+AlPvntc4V?Azf!{qXWHhQGR z2m@S1Qz?b*Vx{BGVNfxTaI9C}IZ1f2-@5(zjbzIa(Z#RddllHK9V;VXU&_0E)c{+A zY~!4%aEFoVu_xK*|eS={1RJJ7c5*T$89 zvFm>l`XI?Ag4a{>o*Gw?>XxKi_3hf^{+>Q)O}4N8eeM0H<_-T}OQ5J}Cdu$?Tml34 z<$v;jmVm6Ck*TnewUOk(*gv07TtIMDk!64*xA^6&=w*b9u7L~F$!kx71U$ECHKtj8XlrCeRRPlQHjf$)8t%#|)BEHl`92+H^j3Yn z(kMf4W5UgOJ>Q{(B}A%txK2mw2(TPo5bUYdjk>EBTuRNP7>m>3Rj(mIv#Pl&TbMMt z5QKuENg%bpKDds4IbRF=c*Eb}g-#cNMzmIC>U!2fXg#5oD2Jm@ls%eGzD88XvC0qX z3HIQ|EYixY@>j~#xuI&!`Q^0Zhuwi-=dk^kXB7N{w8ogX>p>qvnp&2l!Tr*(n=ZOuUBZy79$aEoBN`zXbbaicP8M?Fm9 zrYb@)Jg8cRAo5piBpLG8pv9@A2)xD1SeAyRq$p~5U)hyfLfODLCxM+SAv4ZH{R2f} zkqa3Gxo>ormzcG)-mnccbzkpy_;-*b6^IBcDt^d?CHC3<`5Cm(qn))oHy`GV-`8bALg}AGdFCz`YCd5hA$6mE{-N4%lVnc2BA9 zTnbSzQq$@@tox_hOjp^}jBbOg&F@&DS4)%=4aaR|f)(=iEk43}WR;o!ZXTwuxz)}dm!WiFmO)dV8rUX+GRf#18I7duH{fQ;yTmb}Y z9i*G(DiJdt6r zSKb{p&)F=yfox#jvh=p-nD2D)Tdmte-QhB{YK*aaxdm(`ESr~aF5Yykjyk>&XR{7V z@D-ei86as2K3C_t6y0Qbd6hAo4&Fo(&xqIb02rbzk}nBQQq*LOM~#rP#9d^pz&3t~ z)emYDMR8NpgPh0_ET2~nH4$}X5=yicZ*=bpM^fKaP-Kdxe$WBRKuoKBR~D=!w%Ejb zre0@DT*HpYiWKUEq!lNTjHK1jm8d920OLHL`0;Fie7ZjEsPVz$1NXpG!~_?!)QncW zE7vi2`GFn1%egdVk7M4F)vxj7fhVBQ710tn)>@Aw`wH|zsW<`(RSKh`6U%yB)_n%R zzj=qJJn*b~q!-@7(ecMncZ5E>g;n1vS@*$uIp`0`clh(&?ueHBx2 z_i%x((eO`xs(HInMxt%{ta@d_h7|LKsS6KFMu;@bhzgPyTH#-CY6zj3BBwPZI8Ai0 znc3qMQNX}}T}C2?$Pd4o56yNy@~Ra$3P@r)Si)uKn&_OVm<6Hl#dV4Q5;BtmdlXsZ~^MmT-FY z&6Fg=eNw@$EZN(vaQ_NZ)7$pF;gW_#(Wc(E$J5!iUHi8n} z_utQl2*6!ot+k^h2F+_V1asGIZBj8dCH=<6)%`td6?%`rZ2!Q5p+Dvd<&Y-V=uXu9 zhP=NabGbSZnBZYm&yIDqQJ(od=Q2NI6L#f|o}IEu4P<5w6TWRKvq1oa_IB`aflo-+WmNe( zrzh)xi(1Sz@}xE~qvn8$B&9r4JF?*B8^YN2iCDIOKFAt|mtaM0qRm+{?kKcj30}Vl zNVC?9MdBn&eVxNANSvXm7_>XLaH28CDKU~OreShi^rwwL>wrRo8z3*Y2bXm@en3DvHETFdj=*}!4feO3f%juESQ4bBFxAnpVHG>%pqrwniF1IH4DBqA~8B21Nei#KDu60XK zH(3@@dG=e~tfs}&Rm{YUQz%B7-^hVaL0ju_K}QkuSgPT5Sa$d;q$OrBAVW>Roj3lK#FQX) z%(iO3)7sj~b#g;k`c!+)!_R_zC%gwhuih?(^QI7ZdhbmzTB-JqHlTD<%W;aM7{=Ei zRA&lXf+B&{+~qS(EwYZDUnT=|SM9>Tg&UfNvA=qD{yvRYcHNUh?~W4;SH^gK#ZDVe zQI*|cmg6hn+Rz_Wo(?nC?NtM}O!h5|<3KUNCr|{!&*9-=ObBH*c_*8|C%8g%t6a+y zyvgE~rzTCmMVFpQfnNb5M@*z5M-7$j`RwvdXp1NSe#+Owy6>5;em2GNRRq~bJA;3p zwRIYss7HzSXm5pk|1e$|ja)y@Mlv}EypgO4QnQrUbQ+;dCqIiUxG{5RUbN2i4&Ldr zA9vK!Bgqw5r*FifP$z$_0pBM2;+V}ftjV(fT)~x2QKw#?>@A6H)e7~+gLtkMoHS?F zIk+C&K#3{plQ^Z*R2Z!+P`#rQ@CITRrtl^+Unzdi=J|s+01-p&gL@Ps))Eg%V>LQj zz1x)DN3+ZuKsQ`%yaqVej8zeb7R3LqhAhZ8T053dY$m^lfM~^bSx^;V*ne%vm*8lHM1c&RRAHCAXIaqAQxDD{l2{Eo;*4)m$(m-=uvzFTW@FBbsiz zG>^%bZ@vw)uq6cQ;U*Yfba6xtRifOC*Y}k#5*-^)SK_H^m>o};LkZ@7|D5O}gYDMG z%s!PfMpuNDIqXbTAZia}?kP|3k(7R3frR{;z=g2PlW=OW&}h61)#4g`^DJ95acsxo zlXVg!)}$e~#06q+_$PIQ!mDb*R~E--jvc>QOdt zKXskvRcLC@TZmxmoW{}a`W*3$`j_Qf!6lA&@(=?>F8hr3JHcye!FM=m~x+_W%x1lf-_t$shkkC zw6iVO(lYRfKKgr*@+&*|u793S=;G*FmN#zUaM^-Xk9cC!0={9C}aiL~lYpAy4=Y6}`$j<^e7w31Y& zsSY{}zn#gYMkK+QN*;_3dR`tb`T|Lb0!T`Nd`D8DzVxs*VufjA!-T z)D^=ge2-0P3=z?E5(9bBEf)#X-g2;$-kMZ_d?DY)=#hMJfzgz-xzlW%2fu%;;OoLM zXd4X;lpltxYfJW}AkCg`4R_@T(QtU9=Q;Sg$cZhoMd~EAL#B{QmkTKX>(( zIhyR){R^1CUM<`T!m|6yH$OR?QzVQQzsZ9*Wv1c8dtEUiJ6n|2!}Q@_YqCiU+d#qBlH7g3h7wQd;azDK zUZ78aRR@um$62cBpg0hWlr_=my}I4G!3KsjY4gH|P6p|ks*mD8q!8j3iJZSa{;Z^G zJ@7?3bfP|fJ5bT_rDabg@1Knxvd+(`!HKsue^L{NoEC({ z8B*Pou1bPm=iyj+W|Q>Q#!@u)S=*@VD}rBvzh6zIt{lxX+J=nNpa`~uHLd2RfB!H? z13evveF2kbAa@|)>{aAn=F^fah=E^n3XiP%0U3}BM?jZ_gRMYKz*~wT(aWKe=!hkm zZ;~WDPgEhoCz-dA1g>5*mqnTOJP918cbT;1pp)+c)WEx^;uJB`<^18YOyvDjc~mo5 ztJdV;fSr=)Bm>Ss*SYCT*%$5W&Y@qqz?i>EShxSX*f$yibDUwRX=WVO&#~?;Mbv6{__Psa6GRnog!XZb%bFL$-X6%qk2)mFH~UXlpeO+ zri?$&vV9-xHm9`8w`rleXJZt*30My9t^|p!_6XAra$0i4Q-)w|we-v!Rp(JL3?Xfs zLps#_Qi_UMbE8K%dyEVIWGw=PZ*g0A>gqUQok{~(MzOQ*3MCv2xY`>+oc4Yrwr%sB zBom0b|Mr#|*c9L6!fFz7rBJT9XO?XRio8~y6{ErD+7S}q=Io9W7i0Sfbgwn|9s@l)0inp1rm-c!=JC_BJ$0?Hs z^HwkKGYC@^6L27aJSmP`6Bx+s=NUV82lae9%^0@rdKts=09|wa0Dru+%a+jbIueT< z&krkWP!1U#`7J?-`^jSayW8PJuR95nF|d|8f5ZFEgYv@U?9v$i6`8{LYr7hMcj*Uz z!g?1U=lCY&VkWxT0!+0zeYn6Udmjex$U4Z|a6sBrNC~3SUCk&Jqf9&099(FBZ`bxF z5;XtqkK2%T+~Gftb;VVUmnbkGD~oN@WWV8Zs^g7HBj*6TJGiPT8Ma2lle3IZp#7B* z$#%Olhts^;$+YR%yA{-8j}n?odP9zAL!pv<(Dyw-$P!G=`V`&^p(L2~1aVUst;K@J=D!>G%dN`DF*7r3sAi5-P&?0dZ*) z3G+YNwQhODf*swWU6;A?^ocG3yPKq!Uq;WO>uUpaNRJ3WOvbRsuvXde;Q=q<@!|n5 z@$sVpFQM@iH@HvYLmJ~Q@=p!8;bHgjUdOpd>5X&fKL-x+qA z$C(1JZg?D&@BR{Y^i?CV$n8A3CFy8xvmI)zX#Dzc0@-oHok~jLxr3WgcxKX}E*VVq zDkF)3WY8b}D60QO7SMNZx*jWbb|i%}$<7{Cb-4!4&o#9fl`WDU=&i82~DxS&)xg|2-Hda~J!F=x0H0 zm?nr8MSBTw|AQc}A8G~1ZO``)j90TYAL*7Eo2i1N4y5B=9{d{u6i99F)3IY&TD4&s z^@8QnOM11t^(<-WF}E_R;mlx!@>bLK(mj|b**|@IEa8RlOv?2{K!vR{XQo3Hp^ohN zL*@NoB~0^oK-$1Sg4JrJVpQTj#=RHBMUIVFg*!uKgzljO)yB)rwQmA#rP@n)a5jZS zbAU=7RBL~o)`(i>cNlp(mou|--e!^L!I0&M(4>_t zvs(%W?I=HN!S8@ZLIBlr&Ppucm+Uj;$&e>oTTjJNqji-cRer4ZxFK_%p-8T{HK1VU1?-x4NOkHKSCqHgH!+?b7hbUUe1EFhBje7%JQ9n#W?V>eL z2d0VEh#6??Tho8ZeaDqrCwF0*wGgK2fg&`HM@uDNP&TbKv-ENjjIg5$cE%|h*C}5Z z8$|cWLMhOud>NOduAZvlWOd}xyMzI{?T-nGR=->%Vosb?>Yy!QdqbLGyI_mS`BpGdlAm6JtXQ$(5E+As*>_g;`brrt;~N4QIC*5| zi=^a!R$B_|c!t@`7I1jhv#mNn+gMdMoopR593xu1X0eRX(nd`4Ui8h-txLOt4u~%mz6XOl1It!V%YykSrEq zuYTZ1SrBKy>T!bE!K1O@(dr}R3RIcUl?7FySl+qOvzcm}N9K|?HvEVW!~Ub+FCE&d zd!LK<9l|b#ybi{{d?s3vJz>9oT-B^E5(@-}WmYAq(@nXb|Im(!8lP?()+#yoOM5at zQ{jHd&xE!=j|Gw&mX5OW%DHI7%b|1IONtcxvqgO6Y9P=o6KGns6jYPI5a@w}^E>5A z3@e@gKqM*BL$YcHiAf?OF8T?mO^8Yr+ib2ZGn1+SAn2I6+#h>Q*uYktlP=R7^OKag ziPz}*hkMuVRYaT5NJxtFPAbDVr7Vxs%{_B^{8-_d&m@?nO%cdp= z($UU_1RtT#qjv)tQzVDv#zMYh;Xq)rshfvH{}@2CKow^;&%)B@x^SoD=Y>>3+qOsM zdp_#jl#XO>UK<=@!as^Du#UO`vdHD7%XHS7QpM=MNS}@P^5xG?aw4u^fsg^UYS66C`CX8vGeZ?$w=Yh~EX{$GQV)fNvbme7;i`SC$$e2;4(3;FBV_Gv< zVbn*ZeU`k{NS}43hl_W4%4L}tHG*mi-lA#~u=8}^z*_~WJ}-m)a;>H4Tuf&zi%-Vz zWq32sxRx9V;p-Xj+OSxL4@z@)?(QIAN}(kb#MaIEy^yOdpW^)|XmZdlg5%xr6L2;) zr%_V+EB0ve&Z+>O+!qbOl&{q>kONCjUy`oT=^1y|kHiw7$`Z>)rtyUi_a@p;{VJwR zvr(igFC7ViN$3W<6F1x~vWeYRn`N_6*|j`_eQ*+z-v;I~znYyaE-{4v?tJTFfVqZ* zdHD=^H_eSv2e!~W#YL-d(!D&!1odZALRC~C#{G<&Wu7U=RbQWE4r4-vRqzD_4emCE zwCy$MxKv5VPnvSg?A}p?bFgP~hungXcD?!JTMZ(Ksie&^(IG6|A_IxK0<`e2Orj|m zkAjF+Ixwj7uXEH$cbd8Rd#t zb5T}a7&d^G2++78&&gfTsCrJy) zL8=iRxNP2y?l72UH^y9(7@pSVsgO|bmM6ew6ZGsJ8D*08YpG~7E|0W)W-br#?A>;q z(oz~-m9WUa=@Jhp*(Gnb+R!UNke|u!TK*?UjgF~O$MLDc(R`RH2@$1KF{OS3260)P zPjFQZ2laLOXkPM$%JRzU+#2K}k2y(Oz8L9!m{pgRLvO}nU-GFgGY}O+aZp} zr?70V3zq8^rFjtV{60wuAj7+P)ZCKvLR@MrOMlUnK*7-p0);vIfB!ZNo>T_?HSQx| z*Z&J^hcfKQ{oT{A7GW12E56)_>OkrZO&~#6Zuh8#sx{r|jw8%qJ0<=|YJFESi6l-1 z@!cSoy!?*_pIvD)S6M-Ij%rTK=4nAs`N$pqTUo04Tf$Y?CJSXMG#fh`EW0XlJ9mV^ z7>M=F6CbV6LGqS}+~oS;naa~dhQ){-_pxdSR=99(6!!Xr-G_)wkO5ikN4O5XUAw7% zh*nenav+#Hu1U9)17B3J8@_-)tY{Y5A@m1~hY~Zn8~5(E6C2yHmDTitC6dpO!N?qr zDBz8n&uR3DuR)E9tBMHq@p`G_Ve2dG=assVfvQ$pwfj^tuTHvE5vp1}-80Rov+o&Q zdPj{{C=64&aRWa0A3wT?7cZ?qDuy`2xUz1kGr|wZe}y6Zel7leUoxfNmrP0i{{ch( zn{vqj-2aPp>HWv8m&^02C2R5_u;g#ShCFYnfBRNRooCaLYVZU4x!Ie5-zZXVE}c3r z$E!MdWbEw|1@d?sRHJTx&XWHSaKIJP)=q7(ME64zN<5;JE>sPlPboVU(a(FxwS3v& zA?Vm7LaucAkKy8Ynq!yaCB^6Kw#c{2RA>>@5{0#}uzO8vB}x~ZQXtC=pm-Q!-1v_J zgq(Rc7rwY+j!Pe>87VDEiqw6STq>1B&M3h|Vx6U*PCV?8P;u-8U=_UVVn(%;F%<_B zY9)V=N=C3*f+EV_$n{W$D+`kkgzb)c%3R6tS+ed{?&8#xI-8vbo(6Cmv|H1IqQGW7 z4P1z}rawDJ<6OVrQ0z(>ni6V@`R^%ll{c-)|h z^(mTwNw;J;iw>aSfX8_GEft0$7%T>U#+6-*5tkI9cNAfDmd4HN+CNrnE*Cs7x%-wo zL*EHZoX$mMu=GI2Fl6zcI{-MXP}~Q#e5S5xHmkYaww5i~*i?5MSqzN1e6zFdBYFyx zU$F3pe?h}v28WdKBWC1#ss3$Z*A4Z}oc8aQ*!pO08eIdd683`W2+$RZyARNG0@4%@2Nz89W+0ag z6G;-m=Ib71!$||Jwa6Gw`DWKI;&=Dh!R82afNM~7Rh*s}kAX6d~u^`=bG69{uWReaf)MeTNSG&G;>lYK3xgt9EAac$=5|{3# zdr}#vZFTHf&7e!?Y9+%yKVL~H(!zmJZs^X9=l*_K=-cK9 zMl!~L==gxnE#sM#%=%C)?cd?YNbbXxl~u<1(K%R;kl3feB7XG8dRPgNcf z%ZQc0AU&Qw?OjnT{XW00Xu;3$>^pS0x7a&^f5VplNa6PMF%ScVdBZ|QX+vMda8P9# z2D2r?@rpzfnBNJQ`i~iMMARl7PWiy4yMAeR{ox^>DtaX@85GY-_YWrS7Lfb|jt#SS z$f9MdKI6z|H4xhf7->S|so4r8Z~9poVD~R*p^>vf&sdDyRk=ZVH>uw$U#W8xbNLsq z=M?YC)zFc$roXt9GK95n{Wm>bTOn zH-N7Us@^!M6DLmVUR>PU7@XitkIh$Y_@S~olIp~Vtaf>fBdZYF?sQXj=jj}b(OiZ? zC*1$7vxNp*7jkanJ+>~$=iWDM2+W0njM>>ux9>?CsK%QgXmeO}&w&YZjbPllhNoj7 z4dvp?J&E%Ev*sDzIy%uMXvUc@pU7UDK-@zju18G2-HS0z5Z!8UfGcARC%?Dy%IP<- z)w!8HgzpejY>An!o3(TAi(j`^hO=KFB#efvdlVc+S#r zS5RRPK;F8~QB0t-4YFV*2hCHl&wuwHi}4A+O!L8?Eu#BB-Wu%xHVZnt7`pr~a>kER z_kS9W#VQj@|Fr`-8zclE05L8s;6r%)2ZjKi5D^rGn(-&EhiXlhOxICq42Ctmeq4SJ zKOq>q%4rY$RWYU2TL?<149XDgE5mz+)7Q@Z`|ITcK0vigeb+dRVrfyTysXky+4$+^DYHPn#7HZ7!aVmEdu76S-eEj)BroFVEr43mgc$QXZn zGf^wFR{c2Uuyaa{YAnkJ!o#vr@0-~oW{H#Tm@B{*s8OD2KBhZ5q;3U zR}_r4@A^1{!9z#fXpH3Xk=8c{XKib4m!ZVYuttcDe<`7P`cX__oGC|#>KixXjCC&T z9_Buj5sWpvIAXB83%%QNkRH;IpPy{KrTWF!YhT^gOA2O+5igndbCscPA9Tpjb^?GD z+a=6clipJr<5r1}ze!iLRYy5zCr6Z_?C)SpOul=z{$4YX*TX*V3FMkR;4&(;hu#EV zV$Fp#ksYBIVO+bW`axJ&rzrk`-APkm7ZCz^A3G26Z|@=%i*mWS5$heaFu&!laM-$F zI|snfYC!MT6Y(Orq&}cv*u+L9a1l}~K>S1wQHt!-9K)Uj**wi|wm=^!W;}oahy`o7 z4aD!1)DX>CNfYITk$$YeL%`p+2i1@GqrFktGwvN;=-|zrT7+yAdNf{XE-z@o!J%I%g@7IlF{#-O_6B0*Da4r9^=iz6Wm%bi6hG<(N~(jhvfiIz zxP6`o1Gi)oSM6ZdLQ~&EUHCe$-c55WNOFQSNR4M3-#t7*q=EkIS=aGOm{@q92~6DO z<>g6qVR6*M-`-xm&09O;`ugfdi`Sr-9e*uZHr0!(3OfvtsHXF?h`TdjTGtl0dokd# z8%l89Gx67N>a6Lr)A}&xohVSVAJi4iOllJMHrhKh*hFgb2SJ3^+8L35f^HY~jPd=8 ztPaE8ID(yNX>0E?88`R0UTNleTfs91+Q$uNntH+^jGcpSf?jw)^BNQD`W_dKtmUPG zRcP-w9^t@n_1r2Obpuh>st_9Ok_XRr;GV@an|EhpknIgYvI>_Mq%PHzh@FwPO8_yQ z8x?<1DrH4pYmaxPLCrUmDcSit&GW&YM#B`8`0_}hWM%S!SIi%e?*LSJv1&k`g&+%> zES<@s8r({8MUXb@vv?K3J>!T0+A)&`btPAXBF^9l1ATxqh_f6{7TAdqpFC2do7eW~ zB#(fxJx6@m3+J3TWC;=k8b*}3Cu5)#4O5wd5dcXVMoz7tV$Os?F+5(Xih)BkTv4iu zC9P=Cf>aGtMy)_4`~kLvf>A8I8BF!9K zx(KPe^S}5A)8bkJeAA9cov6)*%7KUULMFnK@yvUzj%k@{8mu_jRL+yZ)c36Vxp;aV zPA1d+>hRbOb7s&Sqx+gW3>_PG3@e;+nJ7f!pKI75$}-qXmT+=Vq0nf-0^5G)s^Qx2 zSyE-D+E$&<9<^f047hY8QCF!3`Rc0q*tS)!%xBj3aBprP#)v1ofkP)~7eviV-egWb zX+>zWL}%K9EOl#rn2|l5WP^P;pxejOZ(%oyh5E@^Z}A}u;fBE zKEMyQQk?LZJTy298#$)p3I4v8^{qH4D4!Y3Gkb{^_XJn6RnWp=`sgm0&lyZgZohT; zg8UKj@_f7dbOw&W!Wlpo5$^BiPWzL|>dIq!3o8q6y!kk5_Vjd9Cp5V-?mN7zd6d#x zYpdby`0-Ya-pU%R=FXzq)!}NX{Gp|KkO{Wa-|DHV{dcb0f(BU-nJP9i^~+;>7qm-a zVHVRis2*eQXxorSD{|UnY`60aQx+37=s52Nv&-h8fU&qOE2XfmU|XkdJS!RWGfoBmpWK7y zUw(qWHDQAMhw0l>B$^*kXtSGJ)^JM$^!N0EU(rKe9cub&@9ZCKCfAcEr>yZJc1*Td zo02fiv=Z?&KzmGYTVSch+Yvp0zYTA-lQm_q-2mR;tkELbT%$K0aILYZ0+lm(TI6Qr zC0oE}hfUMsn?U@8wTb zqTf5ax?SR(%;@4-BptWJq;qWUzP<_oPLC<#L}zw(;+02E{nn92}6 zvx0_k9Ari6YyYtLl~Ohw-pRldKa&qc8%;bTAd5yMEPqo?H0x+G7N?64Si0L}o5JjU zU?wtZBdq(`9ZmQNPA5|7}>avEd zAAYeKoSC$Y^JSUQ&m7ZNA{O)7oHIshE)A1)%hY?;F_$!Aje_+|DChZx^j6t(Rt!Ss zFy&Pwz9IHvYtY-tdaTDLIK1~Hk+nQwOM z|G6}vCeQ{^=C{yN@L~w%Z>eQ1%P>Ez2YfhxZl^=n7|oQp=3Sx z%^J1bIO0KeDQ392Y6FkUgB(f+Dx-jPp1=3kJH*1iccjFzi#kk)mJ5X#N>d}me`ubL znJpa$#CzY7_cd*94170f>DDgD*JzQOTJ97Ylp;+Vn`^2}bk(m3nVnfeWShYaU)``g zkCUBnQQAF3T1uVW%#ex~5z z(fN<~%hGwoIHFjbLzY2}d{RVQ$#}Y8BwxhJbt82WBHkzTfr>GDa^X(t%TdX(iqnI`Yf+ zbm%K=k8s)}hB-$ZiGP@YZ$deVn{C9lZA)yol5II%`IggX`$-1zAV03lvk{4E zqu!t^?p-w^n^cdE*kYAQdpY7*X&YYdwn)w^O>);}8}79J!JxFsD$ifU!Pa;2m@~aI z*66kq6oy3-Of0n(8YAp+kC4+H%fT0TBGgRjQ+n9aX}J25?^e6(LY@UZsdVIi24s@B_)0r!fYGjQ(dEH-XzEgsJ%+fdq!<2W(2O0B~K!`1kq>z!Z4cWX~r2i zMuOAp^*TcdKZO-($Q^z`?Hot5j(d@meNlC(Z0cZ#gc5O{-x;BTr|`6Nk7)ESx?M!> zcN#lGwB3>F$8DK8?;=*Og-} z_nKN)ZXfj88C{+kcgND3{pj|{nFrmjv6UQYTleFP{P<^`bL$ys-(s4VItSvb zGxFlnhcFRgxqM3P?0)5Ut4dZxIUXutO3Bs1exJ-$OJrC$$8gyujs?2L@VJ5Yd9r1W zL2}!It@M@=GA^8buS|_|1opK-r_i<`GB(aZbDM$%Lc0}aJNYr(W^mr>)H_)S$U=<{ z1u0@#1n7nGE1a`FqVNLMmICD#!^THKLPj0RuQpAvI{AGL0!SXZAdjEy%SCZ{??10i zo22@ATM_XIokw^d`@0L+Z%+8_phexMDM3?<9{%2D`&Ctfx^ndG0 z*(n?D%AzR4JqywaC5d|TXo!jksNRYN6f4U>vJ=H6C|VRmWm!d`aS54Yl9r0Uf1`>h zAc!I+MHYlvh$0R(yU?c6zwZXU??&?79w$!t_a5cABxEtC)2!T+F7SH$_pF&e=cNC8 zoq4kVwHZ5;MX|u(%yEp32^ZpHsm$*CPlU$~FeyRACvhByH-#amVo`=YC+@1R5*zCK z)rS}{tAK{NybFth6J$n(1m9{+F;i54RUtHZPtN2RvE;?Ncs_{B`WdWgFN3d5CeLI% z31KQDoMuOyEL8+hOlC7KdDY$pgJuTJGHu{W3L7HgoJf=sDHP;w;x!K~2siMbAEmv9 zT(@;+N|wN)WHu^d<=`T^3tN-iS=nSN%J0v`VB4e^VF{sZ3ul57*&jZap|L1hKI<#% zb0bGUoG_;FEuvOdbVasuw$7Gwd+gfB@WN($Ra~+Nn@%8me=(^6SkBdLd zXhL?kNSg*8_F?3$35`xeWcU5+`-ZUow@-qjCNmv{v|eD?6$PD%V5RGC<5qYlWon>P zan81w6}y>n26XizYGp@d-X30v6h{Rk_zUM=VRSR?aM{Q4JRccz(+G5CBsx`&C1sA& zo}f{cyy#ot17?;(TU}f?^Ny^Y6OEaLIQSE)ZE^8bS;${I7o&1yX_iHl-9I8clXWn< zO&^6_FkjlNEG#ct$h(Y9KI}_6=lb3IP9dm1qV&}i?YbJG55A(dGp6v&AZ=CQCPRpH zRmZ54tZe92h8me44P67leLiX}r%y1#&s8)T;#Hb!Cb3L5RO90XK2ggz^;sG{2l2|MKXe{zj_&rTut||FBo#*pPR^9MvoEof>F^%HsdwSiA;H| z@qVQVM}>-o^$ky0cK=wg-Jrj?8h*@2ay>RozgqX%uvjMvL^ypR6a<%PHq%wNh8)-7 zGN_3^ZfY654_H~i*Slz|a|^GiH??D6ZMVTk(^8qyn*9EV>hI@gU6FT=7}b(=t{A<< zZ`s#_kLsD%v&a3NGqNZ9aEz=@#BEsB9@{pnZ%xE)T$OB4M&lNKhKu~0cv%|ED2T6{ zv(<5=0&c*k>P&d`O0=FglqP+ni_XO$P^*m~9oMgPS)6Wsdw|p72+I2!qgR$&kHkJ3 z*>k$Kr_FEb#5}azc}xDhtGWL8w}2**`J;h#GxkJ8mRoYjHD1d zF3Lt3BU7n5E6|Ii3|B3)Xk=@1NVRN(zlCPm@tU1s&7B~XdUey~KYW2@KBFu8zI&); zZ>wd{bk_h4aOpZMkPBeei-JimnJQvZjp#tva5us3+B2f;fUj=8lCIh_{j07v8hSnl zO{83f2z82~P?I~#Vval%{P-dBr83;vX8wG^kS!lF%fg?5DqCYFq@L6oF-Z$}=9;eNe%YU=2wL4#LIy+BqzI~m1 z&;|gxnSfxVtZ#j4b{3y}z`5TaNejE<7P4`$n}A>*PrriBxVO0`&}-YJ3O-X~C@o1& z5(S+Pz;DjU-|UjF)Pch^@>S)wwp*|&^Ht#2uy3nhm_`$Rd1{?VNZzESoLw?Wq}v1$ zw(VGBBLYwuNK{Df0tj{PrO}v{xWO}wGnXan6KqAwpYtOSejo-wH05!iVGwa@h=>GK zy~L=BfT8PpC{SjY`U*=-al2O2Ro3nExPKz$&_-3J*^d&$eU^@cm@giCS`g5`GzF9N$62DKgv zlR9lv01d*FzCo#(`Zun|-y|ZI721BWIHrT;2`25>hyH(Lk3nyTwRaK-5nI23{%2)0 z(J!Zi`GX(Y{`rvp=ff!V1Ey6lbTYIxbuo1!QPETo{ojR=>#Vfg00RQpuK&A)g=PMY zkY=5rk@z5@g=!H22&!d#hYK>a5~P^GMA;{qX@+QRoXKJB_<`LMV`O>T=&$gptX{juX!F0L4#2;GZj)FAJ)SM} z1wM40GLA{2w@iG5ii0B`EEEi+bHSFS{H|zZ$c_ev9u` z1gfDCj!pVDAP@cu2MxMP0I=JXQ?{=~`;TP8XN0e9SHF?MeOYs~P512X7(f{7iek^% zH(LJnA4uuJ7AsNdpK+!7dD#j6f5-O!3~Rise7^uf$PQwWg20>FZBS&mL;c;QWH=c@ zfynIOffAd75}Hjb*(WRfzyD%4X$dPya$HXHG(JpUPWyfP{Ny!@E%f5$Giqu)jIFl# z&^9d@+Y|8A$7y+iB>?($d>|%Jt*m&XPv(~a``Lwn*6Rw@ywLSTBf&g6!W~Vjnu5nl z2jKKybdgm(PqriBM8U56TOW`K?hD{ zkM>IHhOk04l}V3WHMhQJ*c`+x?(#22k8Ew^B*KaNzXS1ueHT58(_l(?i_)1H+0q*s zrx{O-eR_Q`4uBFqLq(Gi{+VJXkE$s~ULwX6R(UnCJM=G`Tv66Os2u-C-mf&nBOB@u zUyI;p34r_mbtC>uP5i&1-qk9y^2mxPysjWNe8lgGSNXTdor7ikkD7zU;;IxvQ3Yiq=CAG zHD%UFRF3Y5+NVSfy#NAlY`%JPpu)_iKZcN>`Is1w)+XdsBD5rO6pTu0WF`GTu+>RS;ielRJp&wOe^GDT254 zC`BGq^^OzW7Lha12VZ6rbBrlYG40xA(;{I=oBAp_?3<_JM3>D%=lBHYL>qSKD}m(P z{*6ZMXevvV<}|BRzs#C>+F^45Rb!WU@;lj$#a~t5;AP zV+0m?1$6pz3l4h-V3}LX^BxeFP$4fpf0?q5kmVyzjZk-n4Q1dOSWp}fPzyE!N!QxPD_|{}8CT+g~-sx0qcW5#` zC$EW5z?a&=`>VAj&%_?hJLXY1IhHr$uTu+rFxC=u;niC2r~O}WNani;A?ezE% zdS^TgIXk`apWGVwj{BrC2dq(xQ}HJpjPH=%Y4JEodt7jebF z#xtI_XrGnKZY>>VSSq5x(vVj_O&BA!+~5Dhruj=to6G%&UoQ4z)BJxKRQ_wy_;2dG znx#6jI))z|2#_H~7&nocs0*Ru1w^m{vhUg=D>4G?1Mi!Npw1QOXZy3PjN4UYn5Qf1Nq{%R&=*ZaSY6ubkZ_k(w~Bth zAX49|Y|g4kL!%IqF(@$-8lF05>~UvE4N|4Gp)h~tsQx|})U(ooYar(2O`nB9&*}3c ze`e%^S1R1q#6fV7vtAyHl3ZAB=cTT{=of_Hxcf|)r@W-RHI4&1UNn*8_6??V2~9~& z?WvH!4>1u7(X6;m{>YI477IK4J%*v+fHDFh!~IK8>&wSLogxcb6Q|q zl%{mb@5bS188v1aU1rjtlHCXy|Hg9KwebB4AY5h^s?0(~@Jfs)1+#GuGSs5OWYt`0 z^lSCdt}{?Ko;ILzesdDjx!PVVK?HjBhhy+KXS+y9NQeH3U`1sD6 z-4H*G&6D7ASj$Y@8 zm)T`)^_?!xO^I9NGHtw(ax=HPrIcqI>!q!|;kHbdXN}kC-%{-Tx-IxOPZq98Ssa`# zFR?Ovb!a4V>yp+lhG1(s9Pbh6S!R9oIe_nyQtO%itK!6hrXT%ktngc%r!_#>HuQm` z|LBfK?ue1={0lVm9%JT!#=O(}!Q)O|Ci>|scv@6O)EnKwvnuw|3J)?fpFZ|%tou*L zef!#!t2-FZ3k|OO5j!wZ*O`ebB`D6U#)DhrGp2!o$S|>DXipvO z-YJE?ef0;AMsX$ofeqHz-_?bt@1cL)sULE<@3CRGd;}cl%k6+bSUN!yH#uimbP2AA z7FdRAuccQtL{lfas`bXK(=KuM-uJ(fTQ4ziOSM3>NJ&Owh0fHpilL-;{)y@)zFb()f z3D}K;Axx9$H|a0-2qfR`58VF~Sx`d|2tyDyLWdXGu|E`1;6d27T}hquS)uo$Lr*`t zGqZE>)cRglQ&Uq>b3MlI@9%{F>yZg+Ux1)UeHo3mEK6H)ao7DLl8nA8UB(oL1A(bP zOKKAq&rXN|3vHR8C|Yw@?~ty-(YrE&?8A3yyTS;-VTE;Mmlhjf3Q^FJ@amo7L51~| z!*~tGAN}%HsJMQil+}+_{evAyg7lz zVMr**RKv!Lf^}xe2ELUwrX^_p`XhxggsS<*x=303fTaxa;+SNyx07yRZ6MnMV{OMU zGFonXeH_clD^!O5no5No?)AgcTT^q1$Z|b%e4kj`Zqi$K$Ix{$Pg$Yy;3UWofoz*^4;^g=-Yaw!yNaiJDp_2ScRs{6vR8rM4 zUeyd&wM1OCLaV|KRk!-0oN!?t!KNNl*tXdf2a>q}kd3UT+cdiuB2#(0SJEPjj%YX8 zqAknhjYl_kmdu^c zw)Qs1o!7Sq0{Im6MhK7-q%AoRbDw=Mo4c(SfIc#-uGna*Kx-6Cundt(`<^!ATVEFBM$q)25Dcb`P z&ywB@pw3QyZP}^OoQ~tTNum35KPhBOL{%+$OT^W|npek_aAW>1hvb2oau8D%6L&O= z7KFW8CesKl$dX3QHf0wfX0)Wfvfx%pMd52GDqmGU!xvmg`1P@jSB5OBdhHX=L8|vh zaI`_?zkGWy(-@NK$Uh8wf(k> z3Q~pe{Z_%7D%b%qTzscCxk>Ut`$PiRX4&wx7#uqRHYt4R;jBpOjPd=0T##4k5~fCf z0XTR+5v_lboYF1E2EUN*+qAim;U!LcMbOFc5*Pv?(UJWV9oUe!(pHJ~aLAz2J#>T^ zk+;%4g!%=@ycp`0``cw6&GscBgVH!6!%IoBMnv$*a^e}%!n-Wl(@07zu- z%y$WZd>I(yeN#x|G>qxN9+D_iBM{_TX^fFb?6E;@q+w>ph;V=Dv|bTP(kCaAl6|YQIf&(J*LznbKnGwn2ecGGMp!6-1 z;*>}MfQ#%r-Gb1NF;;U_qyj)H^O`1+T9*_KMy4gLF>L%P)0FJMh#W1`l;{B2R4l+* zvVMWNqQSj#klD|U?dM!p>98W>{85p7Fu+?8SIa zMF?heP#3VD>#8AX>46%KyPSr6eoM2N4@=UsUwR-8=Cs}~!q zGfH01o+Bz)B&C{$$i~}~S8g-eTs!Eunv1qqQ@WVt<<_b#l1TM~!r8Y8Ke+|m{o`61 zt;D7v{B1-@^G-Qcl-T<(MV71b{&j?sWYB(}X2Z-{Jv=eqj)5R@%gXku=uW&$Sx2fv{RS1XWTepuiWl17s zA;>G@mr7(sEWCP-3#ay~#sA+1|JI{@d*;F_oI_opLq3G);}leh%? zh&hv>2=HVtldQ40%NEMzdNjB-{dI$4>qO@FFH2(~ zGBbKYbL|v4s7*T_<+Zra*DBo=^kKqsCH&wR%Zl{f&$}M#z5187o^W9;3rdpblrH?j z6Zg_Dkw|Z-?Fuja#Yr2g88+Gn`(PLr>E>i@exNZ}U%PEUEF69}2!fUtcX2?~tov#r zS!qoe3BTo(_?$5E<5PCOaI+NjDIH4=wcRpuaHNfrwL=p6U)!MV=6EF*VtR%10Wy-i zgR!(VF|1skn-pb;q6$HicE3aLvjoO($ta3yjPlt9Br(dzDu7ZjMKgu6MGs(3>Wf~2 zMiK5KVbCA!rL}2{80p1BbDvr;(*AyNlzS@sAL$)yB;x8{nFf@ka+M0yF3P^b*$t#) zN|80Gxsz#G(G@_a4AagB4;ogVcT%n|9mNLH)l2=qNx_m{#ah;MfKYfX` zidl7}l5(|iy%OrjY`_ERyIwy`Fhvlh`e_j$T`H7c`jfORpETaya`QEFa3?5Qm^K=y zsCU;+zJaIJv6C=3Xk)>}d$16jeMAZoon=8Ovy}Zhksz2kL3EM1xIWCkY?NyXF>gA9 z;)M7cbqr~qKoYksvA}Q6Vy~tu$_2blsVRv_*MxvX0eP)|^myh6y1JcVR5IFhsMHa8 zKBpEz6L0&^OG*3Yx&Du=SxNO(S_*2?L)x>_A;Zd{m3{Dxvb3~f!K@|CeOt9KGjqy( zx$;^0U8kgOGmeI~il`@RWe^f4dhSHIs#@rYS}MS{A zgJa6S>5&51-dv?>D$7q%zoLqiyf6EQG#!Dbio_dXTAx7-Pm?k`l9bG*)a+Qhd*ERc z4Q8n-Ta_#hpE8>&UOIOwxt#j6n+EZ4$T&dq7>k2!`j-*4!baEB{@-MtiV(lqYzD^a z6CoW>M#Eo?9LKRg!#g<}oNuiv_=PfhA=NXwf@%%4<>Q_{?IBNvg1g7Y!vbl}U_?no{Hq zP9MqO(wua*!EoC{rcFah#p;cV8YXf&Ds1fYz}cN~BLih@oAc422+qz`!XdP~G^iay zV@BpsuA1e?Fn@M$$K0O~|MTh3 zHU8v`*=<0qmn48_e9z+7zdfP;=QEId^ZLN+|8FEW5k+enP`-WnE#atAR~<%A+QhQM zH@Jc{zh=!e1=6c#4Ogo#9(@ypa!v#fu|z$E&e~cbQ)ZWj@=5%gI9HymG@a~% zO5Bgr`O+d4jqR>up8wjlB!<)vwPz%Jr$8vrDtSNY_c`QMScB`+=Jfa`B7Mss zCY%$sbmam=l!oOXX3-MJKJ)THuD*CAu1#X?q)k7U6H~6a9_*M(5eEu4Y$@SEl*oG3 z@w)`^n1$fB&Xul;J0dQD`NpY@S))}{RRl8yx@5$>T~@rex#+h3Epllse&;LRN_*v z9u@nazF0drbD*9HCm3$8UPtGU01aWEXN@MA#k%D@X_j?5DXx{-n$W^nhs>O5g`NC@ zskCmcT}t}r1k%*{`eC-@cgvp^LaxYD8) zKO@!&x{gbzH#5yV%T&6~vj&CKE!Q>SRM8B3`FW;=zph`J9hZd@-5r;u6aTd2foYs_ zZS!a9XNSa(7eKe>FQ^N3r~^!l9*PQbyA6&c%v4ND0EZK8)Q-o zrD0x|x+D7~PZ$r;-%^ z6wz&Hnn#<7PiLCPV6=7H{GY3biu_F0vWulVIct!b;M7lD(%J?^d!uWZWku^)X6jiH zTG2c#UwNQkES67&)q?Ax!=6&Qm1ii?iKKn_xjt#yN2W*|bnR2=f>TD4A*Z$^E4o7K zt1Xog^hEr5hO2vBhatDNkXhHx-S;a8=6vm`jtz({!-6HiKcj+Xz%|RqmguP+@|N%^ zU!yB1Y%AG&%n3)NXo6mYQ%zU%E~`ib8+F5A&yTk4gh{rS}T3-23D}t^@wk z7hM-`h5tv}0D2QfUg3hS4fa_v_Nxv{58!&sR296kr|5zG5A3GT*dfUaFUak2ni#tOw+0pnBho2kWQY`vLBb2f{Q$F{KT|)T0Azgdmy5)#8*Y2?l;nu%(wMSB+!BV(!3h1|AE$!);qjuV%*w_(@anyK2Z?YgPpE^ifd@K$p6=l5}R@E7-C zbg>zvGb|q1AdkDS9h+qn`YvysREVx;3Nj3GsdBnxXU{0QEsLhqTiar(Yg%Ektl!2R zJoy(rvqvi(pK_`Er}ss4FXxV(u5V3rxeG^ry4@BjdtbaVjCc-USM;QI9MDV@{mlcM zD9Chx>(h{J4)p0hDT@3ucBGR&j0hj)IxMq2pt8ZpJTU9NSiLqt!5&mqsA?O+bzWH; z9<~AT`<|i>NbG^pYP2-PK3#C?tr3UB+&!)i9R7YWiYR@M{hiY*WNlHHCo0)4l*~}_ zDp+kvmJQ0FFNjx`Ql+@YT5-7ZXS z$A7CrF~YyAjvH74&>h!E1Db_L05q4GsFT#4G9ey0^|Z)|q-TL;%fZ5q`Y*n`}?L!7ZA^Nx&lm(MO2wYMh& zc(t|7$n6(9DCgwyc5K;pEW>uyt=X_P>h`!tXa5E!zCjl-aFo8P2l_z}O&^?3!d)iR zjj4~so^KR9^(V{$F%wfIMy-3|BJ470MaCA+jM){wr$(CZQItgIsLZ%wr$(CZBCohwr3irzw__L z?)lGdL`GCbMO9{HWJG1glh1wKqb{(0L+cf{T`@`OUt}?A=%2q_mgtIKVH%82g*o(%_-wP zWu`uW8ih$9i{P# z$bKcrjt_5u+$B5t0!cemaeYUc_z)UJc!hBsRdV4fQTcbrcHoL1zjuTqkffYXQ3bIa$#_^FkA@M22)Nj!NMXiAyl_^vWNlvEfssrxRoygyeMG+FW* z-HslJk*3mt8i>_)K;y!dlY<0@&KH6_2P@W1YC3(JP$epUTTjkD;P*2Mizz~!ga;&n z9zg+g5v_1!a%SO&4EGgiqI3gZtX)=&Vk<#T6j%8|GUS-5Fqn3VI}-VwN>&Ri@#5^s z5GS!N!h~*y@7OkS7mR8S2KOfR%x8GhZN@{Isk~CI(HXsk5eRiOzzm z7P>_V?$}JGrg{>eKU(vY>{X`eYV!g|q#rdaJdtYCTQBY60Ew;$+ybUM6`mai(;tS`2nbqk|4ev4hYG9)kMnu~uJ#ODgMIa&lv)V3sM66E9aIq%_s~OQ2NF%AT<@bY zP~=fX<^OKT&d<>f;OoIauFKS~X#!*u`_1SEez&4#o2BG$Xaa(gKgtIvXstk~%B*-U zjWkiZfxZsm(Sz+ndFE*4gKL<@E*|DduJ$J0hOZh8kux9t5u}MGVn4<)RO7BQKrQ;>_yd!`et+r89D6#vJxyY zWPTnJNwaM{TWZ=*u%Cf6B#nORu8r_|#a^|{Vl56?yJG}%7Kb;$cy7Tk=ucawVfTr} z9Cy|`>ZXUU7&M=GigmVg0bH_x>S`nGGwteZjx%TWpah64?e6jEg259xrlXp0o3X>s zB8xLaDE4~cM+StAlOoGmHLR;CVigS<7kb2!{HxO(%M}c+r8N=93#6%=5$!qRTOy>_ z&1fpqjTU@-nOF9Bt-g_0f~Nms7&hSmbY&fenGgrO4+yYn5XCaQuqY$Rlo@xz z;fA61P>4ipUZKxN#@zk}$d=u(@cJvwpZ$|)Dr35_qBT!1s}t$W#A-wpvWaR+VSI1R zcXwx#(dQ`Np2XLcgH5_}two49cP|rc;tz;)vc?dar~QP=I3Rc9G$kXC@Onq(&;_`u z$Pvz~yTY946OdvZIujh*QA}9F}^zqpTcX>5I-Y>ya>}^$F z#@f@qv^O?QKLtAK^d%^6v5|~egW*VG9^pWsT-@YaMU`zUuper}+Ek*E%m(6KruH}e zE7MDx&!Ids=?WQXmDM-ZFm=+SU_`^Z7>Kt1_`^hA+s(=H!)1cgMW+o)*m759ZMge8 zAy7M?y%oa@WEkf}IFkig7cW?T$6r2h3jWHgv|Y2B`D6-xO(mC6-mS;RKgN++K>gMz z!Y1!ZCOcyc*P}}JvyWWq-|^VSVd_cf`+a(?E)eTcj=Mm>(Z_I}X&=sjgP!R$?Y5=% zTtcHteZ^Dcrlc}UeFcf+CNv52CI>E~I0)4lcgT8IO1P8-wJ}Czet|mJan$gtY) z-qj4%6$A0r3h>!h#@BGsP{78C#AZ%v<00u8SH?G5+E?zo$oNi4`=$Y(QHy!;7X)1(h6rI{VczS;>ku&WB98!$WZF7n&%kRQ30E3R10 zu73gjz?%JH*16#CoAsko?)-$K+)22E=HVE$TP z-6ecNe|(j1(vAi}BhDg+GaRrmgO{;&=fqd##n8y9@)bfHuAOxJxC^0R;xKH&pe6RO zEpVioJ89jgx${M{!YK(pj7Ul%ZetU8}aL9?tMd{M@cIyR&|tniDPM0Z>t`=ys_ z$V0;45&Z}{s)uZc4{kEP5`#oxMYTgYOI_f*W#x3gjR}h|UCkolwOS<1mRJThU zs!V@zy~zwU4EqB=iz&sR;4S)ODf&0n8Q9I4yH$kit@#t%ET&5ZOmrLds zb~Be?ymOwufEK)YWM6dPyuHhO7;2T*gHr6`$b;K*W3N%Zc$Lkk&{XBgjRV6eg`YA~ z?j%dvB>bmHD1-D!(C>G;IrjdY)~VOujFF`T8W$@Cr7*2mV?60wccPXZerOJp~4 z(P}c7M0^I~?F4w8IP}v~=}^Zw_S0kO7+|5Nbd;we*@y|nNSAEXOES7tGy07kji5EH z1f|SEYVkNJO`}Yd2f2iAB58Q>cyr3ZsSW;x7{!Q`Y*c)UnSbHPt9rBx{i${y1i^(D z+L)^w`qzq8K*6iw3Axa`LZwal`D3{R&60h}J(;g-P2D|C*6A{8*~gbgEdsP^ttEBx@=NjANkGuFJwo;as8)J+Y~7 zzfEa>k2Al{HS4?aHka~>OlVRX^e<0xWlW$6H`wh8uL(*WcVrQ7rx+wVkMZ_CQ`I1s zY3@~AK9%)Ox?Xi*=~`SV;9s!9BCPdR8%S;%NILFYr>&@c7<1#2EHq&2@}`Q?4R*vG z&yTp^MEdYNdg}kM;?orOQfgR;^b4B>!8Uf|yz#$tJM`Y^e(8{YXIz$rbaFfPj(Pv@ zuBo#6M3P5nARsA1ARt8lmyCx0P07`g0pq1E(fUhCLwfN{eZ04Ric&@!R7BV!fkB$M z`bW+<^%A*)wS1>p=g@rj6*InU{r?k7736TM3LAv0D?gi4 zedv{-o87?K3Zj2?9|vIsRi+BJci7G*9uJJBgW%YrDl4a{FOO}A?f+>2T`VkXfgT_3 zTLN3K03C+^@mVTNuS|4i?F0|2JR!>fNdsF8%jq!;a1XME&vJ-ofNm+&XuwoL9L7ys z30Fj7j2H&RZl6z{gRLHvv`xzV8#g*?%+5=(h!RIJwAKbO*aWs8fv9F2aj2s1E5#k- zRABHEmu!DD?)ll68&eCb4XTKMq2aHwy-#sTB=dxtFcN{CYP(IZ4N1NnH!@Rdm~Q>J zwQi%ayB_nII0-bUUiwl*jgI2CG1Afwq0Jh%w<69HCMpl4@j&DA_+)=#Dsh}hg$*wp zANl;{siq!V2?cU2?5X3V3pK8i!8~C;3LR?-rn!d{9-@Wvp}YI?!FEa$dxRLfP$TQW zP34#Z?6>(TDLw-GsAxicE1h-0A3ajK6UdE*3)6vT>R1Um^NH+M<3Vbx$gbQqLU?wa zpY@E`j&b|LQxpqFD-Fi%Bs8$0o4mI($_wYWv3QW1ISXl_g$>edoPA!e2Pm(O6MrK{ z$imEiBL2{Ymr@B*Lr*KHAE#(8re<`BZo$W}2qP^p@l#&IwHY%xwNdOF-fCL+^{%YZ zIy4n|eW z_PT3C5B+G3p$$s9c-5v)=Pw%;rE!jAmrl@2@26%No~Zbi^FiYrita1hgEFokV^Hn= zWv5s~e-y)=)?@~=BsDdl#?Rhf=pAir$2kn--fIo z51SR-fNPguN@dB?nW{yfnE(N*sLTxCczT8fTUlL}gbp9B@AJ0=fU>$o16x^BW<-N+ zBfy~Iocy$0Y>a*c%T4+_-A;3zA<3l)u%IbM0%2BNriOv<5NfO89P^Dlb*_GL%LFv6 zs;LifUsQ+_UV)tVSt!c@Q5}%cHTSO?lTlY zn)-r;@CF{^*69?arYfUr*)1BC9t-=+F! zAEZ-tmXl%HRt&xlG-NmnISM%z5P1v*Kc?(N4HOy|D-ncRu@5RNR|+Yao#t6 zKg{A(%+if&&=S$g)TWS+X*Wr`*TsdCBKEZXsjpkC>f}!jQyov=txjIYb`L6fspt{2OBlc&tywoGpjTKQQ>g!D<3fEu;Tw^6}UdWh}BNvU@981yYs9?DRWv$QEQ@sZt#$4I7VUOJ$$!}N!`t4?_g zRGE)`)L95o@Bc;4kTy#x6vTw?7XRRJy_fl;wm1pkSCU_Mx9A5XC6hi7N46LVl}~~` z&}~WN;S72=5K%e}%~hFgs?K(=<{m&28t7x)&pRE*rhdh#ZCmBdX*N?6r9MT#X%~oQ zr|67u`yrT_pm}Y0Y8nGK_E(~Gd=gzSdpb;Zz}Hz|WLk4R-1L*&HTjg*wWehu) zui1`9vD{V{P{X~_km9G3P6NvfqhSAar$L(wws$$ONNRk!vq=nbNHB+PztJXy?YJyD0~XKl)>l_XfwgstMzH zYl|k@f{{od>s)-MuH9S4A9>0N6CmKnk?_#d>IHh|fa3jnxNrF?hhJqesm2l7)ZI)HPnnYOVo^3vxzgTL>B?Ny zLnNm<&rkpJO%I5)A>F=+NQ{g^vIOXoAgfF}X&1)Y_sG!1qZ7Jyh?lov2+&5CZ4wvs7f00( z$$^w3SCmoGG%va0cI|<-TLK5JG+LL8cKs7yOldnka@;708W^?-v&|a0404qB#9KE! zXsr&>J-OPUl48p<)yrpNwF5TNCa9o#lfs<+G%Uuc|HlN>VMRe+D9Z~UqIpPJ&$>OF|X z?Z`K73t#f7`^c^a*)sx-JLR7swJ(uZuNxk97gx(v9;ERMu{TA6E<}n*dfyp%{-8*7 zJgtD84JyLnq?{D$zr#b^B3QFyFn_6Q4J+_q2KZ}-m3P~VhnL(? z5*m|Uhnrr5N~`IQUb|5rs1M*kecT$M6VCEdw1rFm-8m036`DUqp*BSxap-?e+Z?d^ ziAD05zo!1`{c@3@2=`1*mqWZU`N~Xq(hkfQHxlLl6>QUmZe8})~9O9f7Umey}14%{ZWIOB>f>U8|WBMmK8{?BE~6n zjvRLtv;6&QOI;Q^JjxcV2;iM0x9sIAkS{ecO=}l5?NltKD-Va@?$ZTxaPzis#HF~N zqRWo^3u?FgWg}M4ns9b)XF4eauhKg;*||0qUpfi9?eVvRS!LJa5e(zBtpuR)~_p=_rhx443p>r_ofCdqp$9RGI+^m*Jl?pI$~dH=Xh)iB9^}d(`iv z!ghnyRg)T6xgZOlW5slLPe0x1fj%4; z6PoFfzDd?!-poXDB66`Fz3?T)RTB>az-^#r2Lk4-!zo}S@+aP6gS9qW8wuEygtn!r zH*2aOJ(gL9R1}B+u+(K{tH$GS^|hw|tqt{LnKm1%$|Xi=%!L>1->dv(Uej%k7Wba5 zeY-K0vm8Ua1?HA3!dl_}AZ{(gkG0%i(Ita5Qtb%Po)Ciuka$Gy=}Cn|@pXGD6x)r1g&YC7n0`eBv2o zVwHIFH&5eGIvHsi#K2IcN9{_UEDev9Lp!40j*rl#WB$3K7x9~+femdW={PC{kfD=j zFm=O(=%8!|@!`YAjLx5bN9nBMh54I0mVUFA=ifKx1(cEpx`J2Sf=nzBdjTvY{pKve z%&Fn-$r&^LM7*pc`pN@s!Ew%hh;q4z2q|OR1F6-uPO8m)jqR^p>*i|4c5BjAtCJVdopKxu`u=-RNC@`~e}ENeXTCp4 z1ttOSU;*>94nCwQOX7b6ZVgZMc)XzA+YLZcMX!-@Hbe6WR~Qr843f!Jx|UKx)}zan zOm2k0-VSt5X>x-_O!L5cwj`P2jiQJvsSkU9$m(pLGP#_lhq}xLEWL58cJN*3oAUO} z@V*#_f7bIn64lQNycG!9BYucn&MEPS_11AbJ|0TYMw~hT{AL081AvGf%5f9$w@I+# zJjk;tz$2STuZ0NYE%ApukO=um3JLg$YfcR3S`2f~2B$h3jBoR6oDs;lBp?iF4TQxk zUIa zB5|f%{3;HBM={_hA#rduLVN&=iH~{nZ?a%*V9^^PjspN*1Xw^weB3D&XCe~EAtG5% zeCP!7cz;MU0|r-qRei!? zEqwHc$6raKiDe=d>--Qnl09Y88Zhp_mCTRpfGyn(7ohokNn5}i`7P~&sG1#(enK+x zCfxu@!zdipq!ZPV1HT%0j0I*_&nQ_hhjPgZ-h~fk$AGxXi(mc3t&QhgU)w*>nNCpc zJ?;L4$dcDln>rgD?r)$Vbiwum)Aa{u!w*IDzF+JMAdh5;$W)O=iD}#!qNV=9xsL|; z-A37q^=j{xj?M+p^V>K6g)9h)!@3cS##b8^Zg941x@S9EwGzk|%?6~laGFJX@MU}S z=-F53*};H^%MILWgH!b{Oo+Zs3dRNqJC*t!8y5&zPF@JDqCMO$4}#TNqr>Gd6bde0 z2(F?%%(Mvy>W(3b-+;uf%MH!yGr@fP0H)=9XFP!oNM+$>HlQR?08gOI}2YqfJP<~Od-ZC!T=zigVK0XMc z{5`6&J);ypa&B~kG4OlK+X=k_@trWEhbv`JDRF?@|#(mvLf~j5meBOPx6XD{XZ}rq^@evF6EG&kCB$kr` zg(8A9na64e@^8)DdV*ApiF46(M}iG9y~Ni*rpSLs0e2yOI^Cp37*I!c2Xt?p$@d64 z)Vs<4$D56^y<}rkp2zlYx%%|GX%6?_xL$tZwTbwn*n6`M1;S+Cqq2LAHyl|+3Qu-# z<-ixg$5~(oaj{YzOe%Trn2eBby`D8A^(B)aYnISD13TB|jdRn}-w~_ymfx}?fNX0( zec$UiKH-r#7&yk6jbY!bx(%@QGQ~F7r6hCo^B`CX?OC>E0Q`V+=sg zDza|mXD*1;I1WxD3N%FmqTm+)g5dcLel`j~zbB^3BJM#MzThgfMNl0E&~u8c??#xj zAT_Ro(?}uL?M6u20#Px^jla7_nNTJCuBd;v?j$Z{17yW4UH~@ywZo!)OhCnj|aaQ-v! z*N2Aq_!Xe;)=EyEj&<`EF#B`XyRUaw=tuUUvSeJr#q@NA*>6kqhjsh}Y_A7v(!u8_ ze<2tM&swlKl!6%kV8LUht;hB+>m406lQH&yZdFMNW-)Tc>x;_52?eZf{w{oZI?XU`{F(K<@b=fS z-lvoSxauS(T>Hw_#a;)J@K_n(I%6`=hph~S<=Yn49x?7|%2EUT%_IE^l#5!fRuk?x z&3L<5CcB3!E@?P_ARQwuq`@cT)>rTvEWm?y$&Sx0u?ko7s9z<`7KMh{wh5PpLpu z7APJ#&LqH77ASl+E!%3L=jcyLWOI|l_psf;wmWSQ_?qYG_@kXD@S#xNxQTbeS&~4J zUAxtkYxt~;EvH2_^DI;^-_g;1njJOM0>@G;%hUz}4%4Ky0Oo-eW z`$?z`MN;vHyg-t@EeW5|PtqDIw=Ri(4A>W#_Duwhhi}#J6RhB0lzz!b1#PDK(%&qM z;r#@tgw^{CmM~0ypcn=|l6;8-1gq1A7TgNC8Xh>`J{x7}ymJHvNG+gt(S#c1!nAm3D#V!WT>9|j!`gOD+O6GotM^v%W%@}ZV^}E+DRri39smUt+Sb% zwM)yRAI>DxCy~(XVX0;{lBhpLt;*cwjHm98Fykce46~LmwZkx3VWOlQsJj~*iMr}H zqyIs8)D){3v0p1>B8Dr(GD` zXCl&p_J`u_Ax?$D6wMTh7%t)#{-vqAG)moC5WFwQga~V|KOd7TXKmK%XpY?@SNOg5 z#7$Sw|LG$NcS&w~#|~EM^^1VJjQ49)nh!TKCwLF}6M!$(heZ_?HGRi9S8B$djNMOM zVm{0)h0&!&RFZsYguyd3CE@q1hMG5mOktoxqogsSygg*q9I}m0xYtdCiey}e1T#7D z46YKEN19XIoE_wu_oz&P%f%G-T=(_SUmqOUXaaK8y^eL%F#W%g>w4)h`zGcpPhx!;qbcWfR5-e5U%vLd&a=?-!i?xM?7s!;6i zbYV4jb?2J!KwGu}5zmk1Z#YLK6@!l=)#%@ouJB_iSfSKcm;l)gG+Ev%_ju?nwMt1@ z0-jJVihwvHWv}wCwwyK&OJi#O^-=#bhRBQb!gmBKDaqIS-L01LKRUY=KSVntv$348 zw*8_x$V*v<91Jwzy1c}K9lU#YhWSvNraw4r7?PUR823oIciPAUf?uSyIAEK1adjsk zPH>CT5CC^7^9o(QA*B1H;-*i|0dtK5oSlT_jTfSHMuwNDD!&Htxf(4+FP7g_VbCis zkEB(JB2}VUHkYC-SiGKKxp24Nj-YyCJ5J>vb4AF6`Yh&~V*Ux%)Vhx*<_cC1`6Q|X zkA5WHQ87mgf3ed#<~#M=6-8b^~I-;{cXgj)-V%tc8buhJ}QIOYw; z*@UEm+LHz6gAqTRw`1j%TJc;D&4Q};G=a8T&EKwef8a{#3iWyOcKYGTvsz+Xr_(+c))aIp9>eKf~j-N5_cyL4=3fi8;4wzs+eN4&I|d+J!jPQvEZ(H zZ%TG8Tn*wwH4}t6c|gW~p5H=Uss>+J4f+gGcSPHtA!D_k36uNN zzT@p)nM2x$9szES7gKFtEWvXIBOq{=E(Lw}6E=P>uxYMdmG>F=xikB3iF>>H46kiz z!A2IhT3=AF7gb<$7e;0g2m4=a276CS?O6fw%7t;NJi|0RJ>?}Q9{6JEyw`-;#E4I5 zQjj#ER4%w>z~Z+^_2Sfx;bBW}q-`;28C1D7;`;P@q{ht785{mjlN*C+HSyct;e&qCxh5m`ToRF9eYG|V|X7ytK_hKX4hr~H&w?(&-jW( zf)pgD&XHE(oHvNrIqG?cQ<-5!bDMUDZJ3YVExzA-@VcA$(y%8KA0TIfIeY54OfZ$b zaAR{+C={A_FLk0HeoK|;4Vv%PXT*Jn=#$Drk~Ini$9UW&W}q_k-t_Q~W0196_ZL`@cU4mZTTE8%Vp~Al0XJ) zbK2cwi5#|u9V&-ivbmYsJTjYIk~nRW0#3H7survlCA6>NtLal`vzzA3Za%I)u6{Dk zeVsQ@{h&|iMfL+cB!0~JCiO)A@gp&`j%zq;Je1+X0vIWGarFxAWXD-~a5-IV+fl&+ zD&KqEw4Q}MD}k*LNuKU;43QYF71P(6P6|?!ic~o&zMNNagA?!T9#+hpY^~GM=4?0? z2(ixU@R+J+e`eKtg&|TJmapZznbQw(v{L2k7;&BAkg9X7jjbxQYV2N>l-VkiEZsIT zy8GMcWbbm9uLTHre2wglHX6!%#Sd!gCd-rgI1WQ4 zanHlXj8=3EQ~D)%&>pM(u$?tw2HslrvdRKJ$#k~+xKc@&>Z0`<;vgbm!zAfbgzK#9 zPJ6Jnx>!uZX1@pY0g^Qs>F1{dfGeN&xX!MoP_GS_hTxM$0giE`;j?YXLc$rhlY$;l@vV4z3%g#jxiPWybDr-P=2;JF1AqzKq`!>L)HV3cqte92m zUk4vnnO4jlzGo{#(q-dA@R?|a#Ts+nC=FBQCQU?g=7jNMvDMICRd!y={u9%Lr%X&sc9`i zwkb`8pFpGG_w31G7Y)I;f^Z`@?&JZ|wHh}!W?6b`+_}O!iDhLc#B45ra;$UbS`6%x z$Q;t?#&nUWluKg0JBdVV)0BKu!zJnxkW-ZF33HBLZ`K;#7W4vbnJj6x1rW;jFVVK6 zb}WrP1T8KJEMFwax!UG6=9?%Tyc0>Pw%`^VK3^E+-*Te++X?e%f;(MAw{9g7pB~5Y)oFAmM@O_($$8db5mS9@-u`QlcrA# zW2NCxXkuwK)#s^~Zai#M)g^x(YV0zma}@D)@s%vqG#ZdpR0;$ykp&HU%bohYZ(0eYAXjEkIC z+7Ni#Buy;iI);X$fMp@3yT&UVKh7YU`NnGvWp4x%$~}K%)lFu(?%y=ZPgtdU6g?<7u;8r~SI*SysB3*@mC9+aong3Fyi5$GMqCAl zY%hu`>Tp+&0hYznc%o%*YGlKS0f3JXa=me$4`aCbBq*!kAL*yU#kA{yJ`_j4Lwp&sc zO&(>pC=aRqcL%iH0XxK5JlQ;4hL~BNKZUG!zLbd->Pb7sDElNL~y z`Xm-7L$5!=;NRJZNO^s1-%BHj_Knh!csn#xCc;^9SXIpTsH)8-)Eys79-YX4MTG^m zr7MDUCqIfET1_WXKz;^$7nz?s!jS=t(qAb)qv2l0mV%#G1yeE>SaI-2>)N6F{~C5BJXGp)mtGQb~d3S0(;Cr=!$IYN-GCoKh2v2JFp*kq;VlY!V#)w7u_ z8w$gH;75^X$cfBq-tz0IQzJ+f8TIO>5yi>2YLqgu2dGZU8b#@gu_Gtq2Htil%M(XH zZTXGJrc(KYieUjWqUGZf`E1d_seGUB%ubmearq2JwsZ(_1!t3vO6BY7jPA3sVU@;! zwj4OQ;AvKi*2*ABTD5lFQ1##bK{P)@nU3dky#k^+6Ixu#bOT6+n6PNd#vrx@fQoB} z3YcCttQG~gclr|tD_59>Q?r$XsRejO;gTeLdzft?Lqr<8qF_66k}GwMD5Y?b3#eQb zMm-Cc#aa)dZws)hY`Ohgn9>v#1_DEryfUW65z>L#N^Qp{;{LYkPpq8JS3{!Ye%DG1 zEvl>O_naC?b{Jq=8bvM28}d3;w)^Ck!V@qT^70xbmpxxrZBHGkoMN@bZ!L{tZxDS)-+;?#)ioa14XC!v_d=_>3atzd3VTFVTE8bJ>P5i%v_mw9Mczhle#a?wx z#gk(EE1e^h7|%E@y0|Bft7%ILwWdg2V`&7=IT1JhK_1$;*Hki@q=ajB%=sF2+UOZ{ z4w&yuH2C-J^E~T``L4!8j|{bCp<)GWhQR8)h&l;OX~}e;apY`yYOhCD{cSekjqg#y zsxn9)if#ejD{}4p8)o6u`SYUD}TeH6)em+~R*6?}u=TFEyxd=30U! z;({7ebQNWUCoYz_$|_Jo%b;DY&oJUXwADBbgnb-rt7!F>BR+uDY;C3SOuP%0>bHj6 zjeK{MO<>Ef8WRR{JsSp7yw=R$nmTgpnX9U;E32-pN!2O#cGXj+c~TeUVN)q#ZQi&W z$zdKPCOxapC9B9QxSTc!SuJ5TxxbXQjvAkwFWaUM(NkB0j*%;Tk+jj~qKWydAK?SXC2*z48w!1uN-|K`?C=sBgBUH|wE|L7h6s-5jp zPpPNtP4-+*YyRE3yYBdt?XPmh5v7J6f(JL-xxKzv>wbjGmjl1zS5Gv{eaGohm&9Y5 zxV%3bSg$X%f1;7F?sa>mR|{00jf3zdll-vZjv1$u$h65{?ru!zQ%@G?nHs)0=o7SF zUnm)$aJ0{>hER9Wt8ZuajLTgVcMo^weV>NGn?mT~rvIP{G>tYsK;X&y0ZAf11N;CF+3$jmfX84iRE z3;;#qpwSsN;`7DM6AD_=XAj8_ln~YhAiS8CWD3NNy`lSo_}EAc+3;%wrREcIcs=M& zK7=vLOmH*dGuTIglLf#kDj{SK8s&}t6ueY^m?Pd3(sYS|)U-a~Y|IDAp@+}Xp>LfZ zsG+CX#7<$PmxPy_kF{72N;wsQ^DsrL9pPw4)X|T}Kj0ph-T}{Ixn2TsaL)Gpq1w~8 z5-Ri58-oRtL?m_R30?vCdp)eR{-Ac@@Jee)TF)T8chf-@5xAG@LqO&}y5<$erd0%X zACLBHSK9m6AsL3<>FGGm;Hn$`?k$1Th07&xq?VGgxCz%_LiArxx>b0Tc#g#bPZlDY zQ9pC^pMC!s8si@_#6M)&RDZ~Zx8|B?-ma%jr==#%a!ZcEhKZ##k65rErcZ7};Z0GV z4^i>IkxNjihdHPHUD%PHW=uQ50p0Ocj+EC#{S2@9qkb8OZQ*YyNDpL;e+2#H5A_Bf z@clj{UI|_946Y9@tn!F5343?7UK1rSy0d?A5so4GDDFR{3X>auB&~R$I$avyjIH){ z_T=e)T|Wm1nqhxc{2h9|nNUT6$xa4Y+Vmu<4bZ3cRg}??qekUtpfuCt2`NdKTvDa; zi&$&JMj7h7ZkB&=S)NCY3sIVUL&y_%AV+7xHxE~?S)D`K`nWhDEl4#c_k!doI)Mzw zZp9cWo1@|RxUVQM$q3ZNrK>;u1N5@$!G}pK9%>{;^&5uM2@?So9CY56_3=Yz%c#d< zIZHc0tTx82+rPnRhr76QNj!y~aOML03!{2S2zhSk7tzS)BDATba!)Kc6PUN(XI4$D z)Z`C?!~&i-(sl=JI99}49WDbmcoq7DpfhY7bE=R9ZVx{5Ae#d?igM0IzGSxn* zQ|Cj<@Wn?`vO-d+B`T;S+AAy? zkP+gS5b{n7^S1bfM&f^NcXA2c@%a(~B!gdDlNomc0-2J4yW}KBBqV(jqP>Gh{wX2v zh-xn!lvgk5_q$XifzqPAilV)eqP-&Jf#RZ{>2B{Z!B1+X*Lv`ebJ&@AY~W4_z`nh6 zqt98C#j?*YA9fS-9?GNo-k(7x1RrrpjRhlp&f1&=BXJAV5TpTAf(0b>(IMTh%&AEh zS%_aUkuBJ8WTP^_U6S{am_TW5IRbQ36$7KX~6q#^!JW`G<(G;R1!bn+)S# znGA~yZ-}A;u!KC(Z#4#)lq&k+@60N|8#G1N$y3%DP4vZd>eP#rY@Lgi3y3ih$JOyu zeYO~vocqLYLH;4hfys@@6YL8XD*1IoBTXs(E+ONuK0IG)v~v%sx&_9S=4G_YNU`-_ zW-+a?7&ePb{lp@3S&y~+wHAstlaHnFzoziqgDX?^28aJb;=$)bcgDGfmTf;2i|Jm8 zLdX$&NJ>b@!cm0<)>kmJWu?%<$i*M_;kRyC6KC~QqFPM$n4`Adcv=H_i^spf|Aqm6 z?lXX({l;YCa{pf<+G-ZA9yVtGgZJH{3*(n8(O7pb&$=rw2u%*piVy2bafAwl{*eF_ z1LYxxLQy^-$1!&PkDVoWw=#-4zq0rb2yq~FA*d|sumqNte*B7ti@>d>vX-a&wnmsr=GS?kKWqr%JcJcUcT*Dr)c|e7df&ZVu&dd$@yxsjd`Z1-B zrLJ|23-+YQiZRhKQ>oI;{fs;v-KKe+3#)U%o@zHAcNs;O0C#U29SAj373_+w8+Y$N z@mA-ln|nW`;sZ;EwvU#_2hHy}yhL{CGJ8Aonfs`f$Z}{ra;MH}&17xERaoqNdL?1x zQ+d;{Sle;=f%Kp>LS^)xQIe7JE`nu4dR-u*p=o6&*?C3=syfvPu@hUT={NK?mGK)A zCdpW{qm2IP?2)S1x_@F4HTI8EHB;5Cmyrski?_lkkrd65%rufUkI*&JQZyk{l)`D> z5dy-ys`nQ`e886gg_HL*D-afMZ0xw;r4L&Qy$}$FC#0oIgWBFa7(B-%DHa`XE1~nt z$do;^`<}^M?A~@cq3#Qv=_n^Y#nYB)LTyK2pX~4T{R_7NKkpQI59L~<3ZUfs99qmhWXPRe^JIf@bZp-R=ePj1Res8=qz zW&U7FM<(W=z$%??Mpo-kd{Z|bDOWv^Px-0Ao1>)W+)A1z)hQmkq_Rf!;KS&59s%jZLc>Q$g7`m7{ZXFGHX+ zY49L`7?^}b{!=l?TohdY(mH!Sp|IQ^8&3%3OU8ZqoITCi`*+SI+JHxATfxHFQ&)EH zEO?yFeP;&dhb87$78w2fgan+%^8MfJ6jdOK!705cDqY8f8D+?jVNT7(vT$sD*(haT z99}kBQV|6~9_x~<;eFNwRtjB=g1c?uh`i={$l6Vn>l;d%L(T-J9-(=pqMQxR0fK#% zUxMBSa2Y|(^`6Es#Q$N7f3+k=~X%SSqQ zDcVtpf+9OJE8SuqZSdw?C)>3~v{;`%@j3vp1%eW=U=KzaMkhAQ!!sVQ_0Pt&5bM^0 z?9LY!27IKTic4Z=g^PK=zQ&WBJq^p|&hp}_87)#pYJT#-!NHX*blGGnZk`1U4{ksr zCGXuS{A6^n+8vlmcPO+}Q8rQbj|%(LL$WCi2dGkS;8!>uUVEsP zWDvsWR_f*)+%kZ8(h10?;9iMbzHpzbObCQ^y&nt+j5+!d=RPmkvb6|%tAN@phi&TH zqX#Gv>CTn+jjoA^YzF$ z!(p}yT%S;22A$y?nD_gcOIVW_J`rXX1+wDRo$ji|eD3)$&!z>kepL`qcp8wuty796*B zs*r8EtV1?tk*K0l2J`lDhu`c zEAAU32~Dbl^HQ{+m7n4~YAQCM`Ipb%thBBuX-GA1SEp`C=v1#L5jy?vqQ{0{0cn0v zV1aCU)LD=kZm~iDlJZrTaL^aUz{u0s>ejo?gJbL`AGwBqn8}b36tZK_pV!W9&A$){ z>B?Y;a5w5Vx5Vhnd;$U5ZqC|!l!rYhuJ>Tp7*}pidrEp;<$9 zyB-j*AUOA|yrfoBB2OFa=MRxL&C2kFsG+dcIN9;bgol}>_8h>%%SFa zTIvu&`jv~t?MD#>Z92313QxJ{rA&Np{vF^p4EMB=EE?+TpA7ac&j=UyYEB$r2FSv@ z6PCilr_~h?#T z5v{XuC`ivdqxq0B`vq7#PdmYmmxuEqT_%Jyj?*19xf2fQNIIa)_pC$fMtAbj@B>)m zSG@794%cZR4~hK?=Uwr(9E)N{DU|lW+4Ar-zCzb#{0P{i{AA2hQ$QI-4n?_bK2fb4l6I}Dj(JMY^oLKqOnMd4d%7tSJkLYI;YHH-O@8> zp12PE(Wohkq3H03@dIQFY?i|hYFRvmx`DmGF|r^~^F{_}{4<}Qpe<)}2bJ)Yq(;o= zI$)Mh>TI1&k|9QL-wGTjrOqs7koi>DaUpFfd_1tR%$;NBwv{tdGb8ikQ|rYxS7cyf zP!V)I_P$*7E`SeraV)dyg{OPKPAJM12%C`I+S}BOsvcCD<v1EvikhSm%sgcpPpw?vBWua_`No z6A5!=v~_Lw4}B0oFtn;2f{G>z7^G}5kb>l-`+o>Kr&!UVAWa|Jwr!jD*tTsO_t>^= z+qP}nwlTM-XEHOLbS9~X^|BsT_D-erf92dGdu-nBB@zf#XKv}9JhS=;+n>@XzoA~r zDwjV}nI1Grp6t>%(<)xTkv#f^RqOvxmh_#8`(WP=hx3eoB&NWFrTYJwo_^RAb zSAVn0*bFR4_W;lijkDu;HgjI?5|gTN0eBT9DWh<;^iG)W!oq35(k#?jhiMXF59&J3 z&W9?GU}OMSYHB5E>_%WbjQs>iq58bi<^e&RnIK7-FLkJD?V6MP`UuEe2y}~|EU;_} z8W9~Yl54!Qho+|#oi9;RUTREQs#!c-}zK9uV)_?G? zpv9T}OHn;`^{~X5nlzK>M1S^0)N72>ht2`8aXv6Z?d79UP|4KI3^Qc0T*j4;!(rXZ zAWz5q_YVf#5`siFL=*zffRb^&q;?7JwH|?w74_@yca1Y{Z}Hy+4jdX7RU4w5aMX z$|Hp}?9Tl&SP7iN&w^Jp*Yc=Y5)dyETsGWg@+!qSe%=6|DiyY_I?kxuNdvUtf;6sR z?w5ppF@(4?jAF0(TD~MCJUZxR;*uWLE7t0ZfbR)}UW{6Ie7z?l`fLMT7)dC++Ix5K z1#r<4XrZ_@&?)B|MCOV@jG^a)Or%ytM8VgI!e$d1w&%XyS2AO6H?Y=hlITRGu-=8y zH5jZ6VT2j#Vyit5BDvXuEN141)g_28v%PCu`?2%JQoihT9?y|C2iuUH9cqSHJ)=K7 z)#TG2kE8~+Gh^}BFhu`dFb=QsG~h6ZE__J<;F|btGxB1?*V~~%gm!l5VM({$_rXV; z90(N2rz=s!d7-AFiT9n%%uz~FQqTmOL>K9xKqG}1*EK8LRJc^On=_KD4mhWw(lYKv zze{B9bVVv!_7p3$R`X$tgfdnbca$(L<1Zv}z_-=+n!g=-%R(MG;k1kr}Rw}W_R;dhN_uP5+Lya(tv z8|DWN#K#PT2jgxI!UJRP2$BWF!3rb`oP`Kz284$R7z^a>3c>?+$H~G55(}J#3m6+Z zgj}%lxzLVyC){@`a+e9>{_m0$+B-RY)aL}h)BYwd7hO?#Um6&N);5sBEwD0+RFmSk zk+Q3wyDGH`k0|pi1s?j|Y6QIk=S^XPX}QMx?>Qk~;k}8skyz8h)r+x;F%ZePGjR9` zg#wK;p$>{ZgK#xM`pd%9HsJ>ctCxYfR7tyX^!zZCE5*b;M}RhStoSing@xRe{7^S+ zQs0Qx`yiI_%5+G(F zXMX!}e*5O1!s$FdY*&4GTS$ffV+!oi9mM*~+yJAJvgM%`%9aNfw9dO=k`gjk+yr^^ zOGAL)LW8<&O8Q&UB^A6EAnEpFxPVTaI_*4*HdVa+@J3$e6P|{) z!@XV?4;ht{bJjB2b}>qdZ%cs#Dv0WH;0$o{gc>)_;BB7%P=8b2&(c0=9_Ss-YQ0(H zyU9)+bVH!wPi;uyH!swqeS8gp=mn8}Hbnop8V>n}Aog-o9>5z6;mbw5CDt#A{_e~& zO0Sc3o-?Sd%Y3d6Fq9dV2dUnq;8K@%0QYZ617DqdXN^6vcr+ef)xU61kc5BkwMy1J znQXjWg0F4hVJj*zlj;+N%kk%vNbt<54+0<=4X&~AIRQT_N6BVS-Vvnk{}QG)*(11z zHr#b0mpmZ=_0gs642~(p5`bTiOyEuAcOMmu?j3z1AZhM>R5Przb;RZXECWvQNzBgO zo{4xy=!RzOlRDxhSeDLV21Lw*Be;oy z+k}U*c<2&jp<-uV!wop$50{Hhym;Hrh?0e(62*d#S79t@NqyvEN1W&Y!X!tY=_A`F zzs^Ob8^P6oLw+F+Rfmh_yqoi|FU8ke@PDh)@j$nI_=I%!d6WU=H7HE+Acmx2O+8{e z20%838;?2;R;0beE0@0^jsHi4+3J;$?G0WRaC&;&t{Ica9yAqxTyI8w} z(XCIS88u2wb+c`mo9{>;VvrBD+VX?HX}3s#WsvA02mg&5LJw+Ibj41_qqgIx0u`PXEs!^RZceqcMe}AM7CB8&yN@a8%J5} zZ#GmJtdZ=yXAYZgOA<5<+m)`uEO2i;s&ITp4RgcB7ofGyd(!;bdA4+Du5V-9M51+X zVWn{iz(tFi7ws6^>!hqJZw_u!9jjKX}nt9T)vqm1> zfo4x1*Fn?i6_93wyYOBPTEpoNr3gP2#eFzyPpJui9e2j5-wE-CtGy!~dv(s@_2&d% z11+S`4Oz{S>;kkX>76i{MYO%VddEXeL4$e3CsK|pvEu2Y39sJ1Sz_kwq>N(d)C^*Fo zNsjohn4tK~IYc|ix841=#-3x*3-6@FgXN~FEcgFnQ?l8(St7OCW4HEejA!6w$;cf9 z(tb6;3^q122Wp(8(F9F8U1!kRCOPkt{0Kqg&or#jWE$h(8plT0_mAa-GLCN0ltgO1 z$*td8)Sb9%@YjSHX=P1!J+BH`?AsbKJ<#80Lu6i-!`l`0bU}V>!N}}LnccI2RDG6% zc-#Z|0DNpppbITRzav@(q|^>=JkJf&EMKV~K??0Zj0m(>&$E7+x#$I2@byNtV=r+5jzuDDafk=1~g?-qI^p8dhr((#ideIRH+t zreJI(L9~!cnOd(8M?NNp`<|1r2yDPtA}`37lhD z*fwb6bll+!QtQObm%UCvE89yv&SSosMqG0!ueh0H+MKFHq7yV2wX3l-KXkJlU$D+N z|2LnXLAg?5V!`y=W3kvm{AFR8xg^l9o?LTLy{H6#7uvXhOumjt&A%gXf^j6PM2&BC zIEz|cF6*G@XHU})KyX;+!ENwKV^skY`}uCisLVn%ZQ6qA)$+ivI2gQfeP$n?E9~GH z=gb#0^#Rj=F*Y+hpaZvv@bhokJ9N@Cy#iftROC1F^vNoMrfz6>m!R6c7DK$-nJiyK z+yfr(6#gRL4@mKP=@Xb_1^gQ`*Noo6>X+L`PH#T`8$Lxq?|#k~F8tB0!qQit*<&nE zitmB;5BtD4{lu1P-fsVL=zviB#CVmcsJr+c&rc~+D@o4K@4y&Hc_Si1Z(uv@(LBTJ zEkmcDtP*HS`VCCaT9^+B<<>G&^3Px6w1DK8X!sGo!)qUE8@EOt znum%LL^@{!YS+Z;`D*@zXG2sD9kDZy^0%Yp3|)pX&nm)y9>vU6hLI)1(0vt!xPWq@ zqhu_q1Y@0bczqs4Lq{npvIJw7b$G};LPAIBNL1yfjiGiHK}bJYI~{5X#$&Y8rKMj8Jm#(BlY=xlAExvA7_h z1})h+GEE{=81)*V(Y!THrWe`i4Ap_KoAhL1v;YhCX?SX&UsYkUTNoD}NfwV*ULXQg zn?PHe4y{2+djL;iRw^j878%w%)%?gSIModbzZHs~pFZP_)q%2?I^_=Ckn)!#1v0Ts z+ZkOg%;!Kdo~{N-H3ITdZav!y_&at~qW{ZF=!o{$&P3a2JE&O!Y z^FE`ETYD{xIA8TgoB=F66^L<+N>^a1O?-sFL0p6wndLbtLbQI)k{&hsATOS?Kv{b9 z%#u-PO5TBTYpE9{OcX2@>yDu=%Dk!MFAGw==&V?-^FIo=Xp|(egE|8NEHqe=Y=E-% z9-vt=uk7DiNq&VbCr8?Pr0rrT)q5Q!w_K}7;hv=$=slXg?V{Ju7Ahc0hWql z!MsY08khq3Ziv>n?C?xesznCk{+xeRNE(h(8D4C4Vx;AH`2sE^ab++c+B?h}P}H!B z-~XZ;ktW`Ypo9I_VZ;8naPz+kewHlNIRMo=7?LRth)+WyXOGBdm4-T}9 z`g3cXz%GpkF<8JD1}M@Xokm54g%FHDfRT`9P%b}ga1swAYdYd#2VjNX+^KVAIb5NG zs)Z_e9VD)!HLtn4@po~B&Q+eaDVn`ygKz`)yzY&vmaW<#CI4wOMQ zpK6{S%Kq<61M|D~)M6MA-&5a>?ZVHgimeJ~} z#3Fx~DW0*N9B&12S)TqO7|wZweB-Q(rzJNWyb`?2gf0_q=oku9v+}U!JcDX~P7ooZ z^eN$u&Il~!LO>ZXEL79jb7^D)7Ha9UW*kp{BO5pEywW9hoGU6pVfzOER*HZ0l##AP zX^$&boD2@oN0bVQGreq!%UQ0}g|laHJJyMtf)5KaS^R}jCFLi?6-=@jl$E!gf(I+l z4PXNGWDhf^N6cjoXfy1*$ZQri?hFp_NDq}vx7)~LLx<6c=HkP2kYCaAz=o4Q$k+~u?8M^rNjj$E`IR;B`pC>;Bmla^C=QP@B$b6&M)eVjiKSP9^38GD-)a&lVo(ceWftoiu-AZ!y*xqOVxm6 ze9^P=YB^gqQq@Vp=TF5#v~!t0D?{OJs`n1~R|A710cOb}U#xYB$A&D=0d1Jgg5I#Lm~AQsCuKS#9ZC=5$L+?37E|HrZMdCh zkvYYxW~xjIsE3X;JscN^2{%Lmb)_xVZKs8y)a^3O>cE6)v51@0$q*S_Zin=egI!F--!D$5i0?| zG*&a*$x`cp-P29&nPh29_UNy$#6mn+b^EQzeyFmecT4tRzii6fEeZd!$HN@BK<;MV z%OU?J>=b*J$#ASL@V(jHKm6jfTTyp_?(!Uh+^$ITU4x+PiXJxS@TS|~l?NbYDtUC; zVH!YJNSa_o$YpIHW;c%r>&KmJu&D@Sh%b&qP+mM6Re1=#lvp@BNOKeCdWtl`-g9Lr z=OJk?h%`1^-ShjG5lkJOiJisl&ysJiiA6nOLbDs^XQMC}1U8of1 zW2L>uQdn1zp_VM#CYD6+e!xm$txSNPW>69B^DErQS9l6!OUuQFkRLn`TjrF@CY#y? zTz79Q#3*R7+;J6ziLPGvQL)eeNDK79N*ygvPY+$HH#lO7Dh6A!FFShR zsgW{mV}^G@EgMuMr^(=AKxh-o5WY$nr)XvT%_;8)!i`E{sZ*NBxpERQnlkPr{*AbM z>{kw_nt_?|h7kA#_CLf@cH$AnpM4KXcvrvyXFyC8hm2nstNl_ACff}z_iCD&k28yj z^|kUC@x&=C&oh>B&E*N}(d)30@Km-hZlM6g{upw*s`&gG@%wsjCTOYkx9J<$=4Dk;gCA`DS=D(|pQG^NaQ8yNiN<8hEX~ za~{1zaNMr|^<#j!$w+q$mVJSien8op$c9<)dKmI6;-R!d3q3hEaMmk_O-yfabX}V2 zTjYi_zSBpd@)32`L3=``=@b6s5yw+ImiHo2Xo>t;T`BXA9Q;8iUl^9x{oiXhxk%B0k9ol={PwE5 zn}SxGWw7f~yCTA0zBG``v{Q-giL{BuU%KL69noJ;@Q$~y9)MKj0&084Ud)W2?5FI> zb6)(DSIeJnH2mWH+7Y^*>z8?W{1I07fF=ArPo7xpJ0ky2=zauw^mFTHO~j>Do>-W@ z*f+@hCxy(@d!?_aJh}l7cL;uRV|ZpfpZvdz9PVW@vx-Xfza(%y(kXu@ZgfxW^m; zx#TCj3&*^LAX_#~nLeodttwkp2{5t~yOLAD|s z+M*vf(}!*e(mLzo#`mnZqHX!S=!O6UaY&jzF)m(_3O*sX0*|~Cd*Kfi_^@*hUO#cV z{YKY#_;iuN!6Wi;og{tpZ+%5|S9+!=?xxOB1 z3l%KI190vqb)scJS%ZU(8ZuPnN3}!mah4sMfSV0(4>hT*s9uf!JmzuvyVN)2om?S$ zHqPPq;{ZLS2)abTPZv0?ExR{xO)sxk!)`hB?A%jG=9U+`5lLSJ5Ixhxaz!sSvM%H7 zXrQBTdfwZA+xPp%NIZK0{Zeo2gOZK!?xBqA@QA8H{|J-2hiiP9kZq_Tj8;bd_A>40 znN|HpGrqxlgZvu9{6cfz*^OLaJLj`E)2f{*cUPKE>g3P{yC%03L=|Ny3;e{-RUA5_ zdF2;ZM8_-wiP0`D3rmC^_3JGFm7$u@`c)K_Do3vXiSxTKc$=TjAApD&DY*+|YxQJ@ zQ*jkptDQ!m-Wryv=<>2Cm)3;(oR7!2|HD-vvV>5N#h37JCdu%8G z0JncER`UNY>r!KKu@G~#Fcvm&Hu%q!D_RZG8G9MEdlyTB*aBx$GKql%6gy$9{xA%~ zJ`!i)&jCnjO!h`U8Fn;Hv<+uNk$9m#1$RD;iIo&3+#VU_kTt1PWELn;5)dtjr{{*E ztkRi9$J_#_?B?QiQ_<_}SWF^?{$*DB@2>Cf_v?+D$KWsPM@4J-)dCJOtMeyv4GKc=V zG%zBu$w)$Y6t<85_`j-#7NULWH7!swauaRda3b%abe5@?OMt~>D3Or>fxNv&Un1MU zeB{FUT{6%mIo3Ld*0m{D%9%x)62?@IG!vrs%Fc#i_T1hG%mE*u41LMq;Yd2rqAbtF zfi(>t>w>QGo;CXnSB<&dqy452bqw~8=pc`R&UfiCX*NssfMRR(72%C3=aH!$ zr6c`jWxItsm|W5~2jY2R6$uC#M}zhdw5Em%=JgtByg~s&_wdV85Q5hxEA?=xYBK5J zK}4jXgL`oL3)s?4Qu*+(d3!1kf8eb%l(hHxW?SMt5pW(Ri%Qn2e$9rJH)-cFDpM*d z==}N;c~rMON7&p^&w+#qq~{YLZfMpaVQ-tb>~+#v-WGSF!uu4lau% z`0Q41M7D%TTiYbaWp(`Rj@Ug%BiBXwMq2IGPZ$=kZhNu|ctYbCD=7g}b43TXOsZyF z>1UMekm?={ojj}?qQ}P9Ov;+*JEE2CX&|x9)@Gv97!F*P!m+h2^+Vi-3O5v*gN{lc zOa?$2S-s6_WT_@cs9Rk7@PXl^0>j4RQ1F)S5JFvdqm2WjhkP+`R4UfQdzi#oF;5k5 zWW-~40;sWdImF#dY)n)kCeD{6#C#?D4&x(K^lXul#di`C)w_lUs;sC7O63hogLLVd zZZwFjV49u9*0D){Gq;IaHI5&fLb94}^mixUbzwj2a_yOVjOL+(puji;6Oi0R17jAi z?Wno=b=NO#M9HvAUF5W1QnNivmy?7^lc%Oy^<1KIh6q+E?_*pT$mt@P9z0JsY9C|D zSEJQxu~kwRUX0mr8B>Og(Nm*;r?SWoNmPeG9%hH*7d<;Tp&zCzp=K@+%Q`tJ4w_A< z$DyZ2OHHZobf#JZBq;J#%UMq}Qqmu*v;8>cgLau1mvyHy;|j;q21)|}R)@qg(_EpD z7wxffb<-g>h3BQQk}7@OwA}#~)uQ*1X=6M^g0jHvM=CP9De(KPf3t8bO^*$nJDNYPYShMw zx%B{D18}(?m=KyZ*W3(N5CwI0yyi8a`fQhbIoAfBR$tg+Vf%_`Kw7D~UK@A!=j z+n<`sfUwwZxIWFU;N@-+>JZsjb8gVa=F=B9cPuW@Aq|R|fhN(~N9Z(nVzdtaxfV@& zY)^A%b?gZWV`0>qWPqRk8q%66u{)lC;@f;XT!r+NXo(0{X$#qfXw|2^88}`W@OmS3 zHUKAbM*u<;8^tp=Uzh2%6HH-B?znW9g~5}= zasUN0w*fajl-zjSXH*?l;Sm}{e?Ya~Y|CIn5%stZN2M`+6xIjGgAk3x7=fibKgYIYg!|3>w4 zu)1WJfX}l|t`hpysrr+nz^v&<+;+;Tv(BpeOz4Vn)*9)9cF1+$#%p?~$OFxen20^+ zmsuaTs!eQbqH1UKVmoTmme{ssv)_}F#t>64i`kbNURo!rM`$S0lr^H}K<|(%SgAMo zcgO%!u31cD;(Fd#dH-@~F`T)a(&?n&`jgT8HS-Yo;w^mpw&GjsW#6BOWwk2`X&r)< z_-*8&y_4%vrhdz|KEGxmzH#kSArHBAhJM>&F7KqSa|ihjX5a6|dF&k7+QdviWd)qa@c`S##ir(FdY?P+P9rcr zj%-?@c>8(S|A0`KHgMRHkCMRxLdr$2ET|;H!ZLwHh0C%+di4ouGQ~7jl!9te%B6J} z5ary2%{OZ%d=?G9@NF%K?kt10SWi|<-O6b+)J^%b^>!FA{C!gU#V%F6=CbO=4wrtL zi|Vg?*OP%3mm7Mcu64bwMvleLAQuSr@E{fC2=LJvcstk7&gKmCDWd8u+|r)H%Z|}e zI{AEf=@>otIH}j#v|J;!Q`ITr$~_;`jL*t>qj7fN+Qf%@z%9Fwwn z;QBs7%$`eEEc+e(mpsImxceP4BPc&Amk*ig0WBjqKYB9<&7Zem^`O>a$1g?aJvBq# zm@vLC&8I^h2hbmant|LQ(=U;&@YiA9w@i#7xO&`QgxJGeL-HTmmxJB|k1x5dpuz)K z1(F>UI0to(VY?tpmksWywB=kgj#&51AE}T;x8$QbB4ztqK`UiaI8B@rf$lJMoD%(L z^{`V8#{*5q?E?mMTZB@^53hASt+B&|Y2}8xX;VC+dotZUaZz!peE;A05?JEYS??dlE94)>i{-zKFHH>pqoDr}=B4((PyaW>D_B8F zZa@IxCk2SMSaIl;dB`3>h9ZYlS3HVL7#<^P^txd_VIiUAuG1SwoaG+;tuU%_*?tco z^XjUrC*97p*O#vc;2*?GEF_;3gLBZII*VgJ99eGUkU=sF{QSk3C0W6yMkn+~K$NQi z=9nTp^^ny(AQ6%hoo#S8FTf*obg6<<|frS zoh`qD+2A+_Z7asCCG5&kRcc;LbVYmvSw@QK{!PS zm-K*{Yd{lVE%lG^HN}VopJ1OR5}n)1H;8U?tK}hN%WS^?r492wxC{CK0sycM2>?Ly z|NUV9D;rSJQbSfl`GEr=hanb67&au7??YfDwzS%$fC@vxMu-+x@Jc5cE(O6x%&ROyXyna5Oa&l z!uW(b@!?3Ax~emK=iI7Uz7|uuyWbAurrK6WFl2AOuFm+BV1W9J-ENZeiUHU5kY+$& z!!441DrGu+{K53v#k$SE_&11zQjdXlF;m|>CfXG9&!Ve;rMYM(7^p!=Oq0d^yCvsn z_NHjf5GIiYw_Q?0h*89=KJyVTlTL{F$8^&9Knu~UA{wriM+0_L^o?0T0rWN$95cyI z6{m8l>9wLiH&#quBBAxsuUwPzQ-NGi zv6x_zpk0;jLC#XC%SH=N&&4#BYCg%-=REodx3Ca>QZT-unff4vEJIVL%Qxtt*U%iU zlwC2~A!mt-bc`uzr~<4*ma+UeNuMc3%yssaK=>4DmLW$yy#v1wlygXa2;=OW)XI?@ z&|g%@rgDryOvIc<6qVes3tRmHgpp1d8Bc8vU){maH$NUms9=FW14DW_UE;JY>OT^& z=)(6cW8Rf%eI}r(7#%H7W~z@}2k9-v28t^l!B@Nw)W2^nMqvvukaC#n5lAVo%!3R) z{H@s zkf1~d4b&NqfkR~l9~;Zrv5yQs{#n$ z!-h@hpwf!awP1-HFi>_!vN+O_h%-lh@w12NjMQ*>_$;G5c6Re|BM)Q=N@<|El}LqL zI(F0>ty&KQnDnIgs4Pty`Xv;_N$0|Nwk%}IQWrLp_P{-ZLdwjYc?j5w+Q;Q&)cUzP z&uvgyGBMyKkPDat$LG4Ozld*=d9W_)=-YV+++V}hKq1tS+g4N;jRb343tVybO!d#k6T%nAswE(&;A-h4Rulm)-XLtDkw;oQLvBg#I_tU?T zD3oBhJ0A7a<#9zf8pN6wZdu`|}Al-C@N z<2%3Kx)Ry@YuiOPo>j3E; zP={lrN>~@{t}Zm%-O*MXa=6a_a+m$r)-@#c7WEqv06;lC008BG=QAyx>}=^JmE>dv z9UTok{?l<*Ye0J|A1D9jwkJw6wL=mzh>OPo`jaF=A|4Y4ENXTzhxTv}5blTXguC!Rqx74`m*tm4sxFqwQcDmWx znvyZBCF>>J`ouc@oc`^ZRpawL=>#xc#HHhaFmrxZwY$1FtBM&0zQ0iS)zc%vs@=he z460PC(OqsL!Gg7R>X)F{!dSy8yMYW^Rja*N!?=ng9p86!Ya-QnHa* zMQ~ILT?qxg|1pn~kGGvI2r*t@jeOUBDUjzt(UYE=pw@(#F^R9q!^V1WIU2Ygz!+#_ zS=yoy@S%ec;cAGJ!B|9q0?+*L;trT+^^d_5Rainehk0rODIk1jBhS8v&Brhh6#|1Y zaPE0XwAMymM}Yw^WDVHv_O>#sqsNN~GNO|vO3HpKYu=nTJnCe`RMO_it~VJ{2t`-Y zxZdAed27hgC#Yg+qX9QVD51aC{NsMu)xWukAL*5@%W|`g6!A|8SWS%_!t6XHjG4*N zLW=`$t%DaWS`f71m|3nmT9|+Lkr9!2){j&odZv1I*FU{+1L15t5scrcU1q3FUSi-L z9AlX^E|y7WVU9pt;vSv$LxXE(1Q&^Oc43%HBe$(muj)6o$cTTMW@!oe^wbC)MPDnv zof8-=ZT7yx_XFYT0ldTHo6e5(oYLIrX-oWCsAH6GLKWuyF+!VTUfNT{&ke- zdwyn5oO7Bv=)h5nfHv~!iGf3*ov~l;&@T5|`@1pFgq`cu`;n z2f)Y^*Mi!1Gpk4Wh?UyrhJEFgTy#h}+#;*CY@UrV%j$#3S_S@m15?hblG#XTv|O4F zl>N)(7FUYSqtM!;#Z^nB3x!Q3n zcj+X1IM&28xHBe-RDb;OZcW3D?W~20dP^ra5s&Pn%hTe+a9t7Kp~l)ZY*7xi|D=2z zlwC5KlzRR^hQo;_8mGow)+dsNKMm`wG5={4&7!-8e|82frxd9@&Kf{~nv-!J_4SN5yqJ zsJt?~535hh`mHifM1^t#<(xMWo9ehVZKRJDj@mddp0g4m{2ZcmJvqQ`TMf2D8i7k> zV!v#Ie`W4QmBLb`H;c6uUsoWW`SptdCk-|Ouf2eOvLE(#7k)L^o9G8nx;SyMg~A(X zb@2v|ec(Hk{{FBh!O}(x0~H(ZumU|5#Epd{IL$bbN;%z&V*#B1(TB@1w!hwym!{<2 zyJqN{W*R7@Td-q+p>fZK4kYfw{2PKD>L5J2=pI6Cp1NU|f-$4&V#^bG+i~|w42Pxc z!V`M>F0Ds^1xVsRKTbRUiC2vW%BgA-v-#NEAW#T<2yo~ zuse8f5Y+WV^|Ks&8_PhH3saN&asd8Dm@AKf+T8TU;p1#-*3L^VdjAiY@kuVdM3&w> zgf_+=PUblmk*L%R^*+lnN*>R;l2#*yq$gY;kkqJ{^eBb<8^gYS#>>NBg{#XQ(yHSg zDbGODnzejy^%e3;y^JxmaA8`!FCJd^SOo1A@<~_v?223&Ka64KO0lxEI~cyz&WHp> zzC5D5>*^rFmXe5ZXY7WaXlLI1F z6^NaLe3S$9B{E8W;5wszW~o|{sZ^E@0CH-^#W#G4vwi$*rDAEIHlWj%`RfeY3H;3q zsYXio)$Y9lk#v>q8}cD1+vdDLAvRx@7)I=P`NqvN9C|^L_$)y%=Ik%a6|I1+B8?Ee zd>`u`o_g`0e?@jlE#ywfT==?AzEMi{#1~E^9Yqxjm?N|)BD6Kt|-8FEw zT%J@S`CT7PWfrXqBkQ+ zJsJ9T#nWks1^gG&>Zze9Fj8^)dzE5UE)v9?_L2b9AW}PAYzFbfhmeEn+w?5j@Vgl`72HTD z9zcq0!+`$|!Q{!>7^{V8$yr#y!b9(2!E^Ye4~1cYMfAkV3Kz#^5juG6Aw$A-uz(%;NxqPf$s?v~4@z#5ZM1(qo~BVDLfmep-ia8V0ZqqR)mbU(KaTwr7`AmL>?MxK|D5vhvQUovdu?Pd z&RD+&&rI?Os0UADlA8m4;GbOt=UoMqCfNKth7rDe*Vw~d(1;l>j`sJG0_=)AvUmy3Q z!B>YR~*Pn8xvmN+TWJkxnw?5=#5$K@( zIv$yeS1?hmKYVQUJl$zg5LPr$_y^Lh_|SEjk#(@A`<5*BquYVmJe`qdZjV!VSAEfK zXzYhzTP077{c%f6p;4A^Je1Q*WL9949yFhi8oP;`RH1e55T~%f&XvQR8slCzLyxov z@77HwWcOUU%rKqW@jg+}q$gbD<5|h{`Be`VReM34e)k0mzl-JsIQ_!~sy}T(cE6*$ z$S4_q7^8JS^<@qK***h#UKz;t5mlF4)kR|U`S4WEy1@9Mw>ixRHAmRO-D(C+qN=tN z1Xt)-?S89hu9;}?r1x2 z65L=6?Lx`^5<;8D8~#v%U%sw``}i?^jg{;}K`Vpyi~w*D@56X!?}wwkTP6b?mqC&e z$iJ|C-Oz;kC_#P zf-|xJwCeK(6vnoe4E*f}Q^UxPp>&@Q;tlWh=fA&|G#VFvTq25Val+PV_J zy&Ew3I*rn%#|OBTO5679xQDV&C6@1PtVCEMkJ!b+XOzZ2U`5H$n$A#3o7RNR2@?ZX z4D{)M)rjp^!kEV#C&w%=Y{Dk6jFx8!EguYewtMFZ=Jy3g;EN@|kBd5o1ICgHIBg~X z%>*{|`mMxtL{B61bFqicbOaTmF&okq2fLnDPuR{oEno+a>FMw6f`y6%dtkz_UYpu6xQ=vp{f~42%YUE+T+!Q!LYZc0s0dz;mBVdB8j2+?a$4 zcJEUy&Kv}v(d01$DE-KBD^ip6%dJ^i)a8*C_;v^3Lyzp0b>!d}?A{o0j62gxr7^E+ zJkN&qf)!PWnM`1sT1DFt{y|dsRd&jlhZCIg^PZ%Oe)Fyb ziPS^BY2PKoo$~d5xFSSk85kp^=Q3Ky6&Gf2oXjz zHvp@Zt%@BVu(!P}I;4+jM=+lp>~Ceybus6^ZwY4D+ZUbSGRqZsz{^OVk`IGxb%QTj z?J&m1=K!Y~8G!jH7R;qMLdm@~r`%mSuz19kkbpKF1|+_@P|C1MfXZ|YOLog-1#n(o z??R(juBxz#FD9^hLmt}JyY!a72Z&wKPP)6XhCSQrry~YEN zsZN`AEvHV=`VonR%B0<+wcfL2<4~1bpP&tNt;|Ue=TA4CVDu~??534X5!I}Rg~s28 zt%b$bhsEZfU#JqfArj9{OdXSTMkFaZfyRnp<`lZOR9lC6%czKzO{gW4Bvz!xSjAng z%~ZMRXB}#0KJ%h^!B3`?(s6TTOEkiMYdE;|$aJV}N`{k5S7S!FT_KQ17BVu2i{uOw#U?wJg#UJ-0JmnKo5a=@IR`B+Bq)>3HX7)O+o@9v z^Gy83jnKip#WFRcm}$#(OIM|QN)99c98N^Wxz6|`gu;2*)XN{PpF2=5^WHF6G{}Fj z03M7$iO=Np&W7BXZAoraD@}f(J^6ka_rM;G;mpnrNtBuR_o(5*MTZYyrwz(JvK5C(I zTaJdZ{|965;M`mItb4|`o$T1=j&0kv?d;gLZQDDxZGK~8$DG_bb5GU%Rn45Kde<8pOKl7uGrQtnReiD4)x^#8%Lp_>M`iMZt3K;nHBwtRIW<+f*;6 zF*2tTl8V~R2~{bZO`__#K!=-X(L^puM!yl!*fewWUSZ&2s;o?SW`D+x3=`ZU#=){G zzDjpWZ!X%%7GA0u!=pu7v9!gowG{)ld9q%p3lGx*c*T@rqlKt;j-9C3c8GijchM{5 zGzz|ecK*FWw4?>3n!}zjY2JuWGNm9^%X2wmT|*-6o-#^xNz7WA=Pb+Y$NIuY(sbpy z102=0LB;9pxeb24Q*1tURDb+)%(SrS0^-c+agF@_nefl-lZ$I-aCj@eD9$SX)$e4h zE=JYO7?xN27`LFBu6o@f%03bys!}CLSLTvM&E%oYjhvUt8mRJ~@liUL)dDrQgl713 zH_S%1Yv*Qfo3;v(I#RI3?KF5g87`@9+PhG(*hB^G#b+Ns%T@0V%}TF3h2^N(yM880 z#k#4I=teY>^Io@j&?Bbkd4z_oxb;AzNU{sGPN`RG1i2#0VHE`t%3KhO1Wk2kpOj1@ ziP+E+oB4QS#n}SCjcR%{D#;Mo56)~ z@l%``qzfKR?|J5qhY8R14W3n?W3Ox-LK;|2Hf^b!l!Pqtl-x^_W@Y@R_QPf${l)X_ zldS{dxXGlJ?-!cZ?IcrY!!9^PIv1%GYBz-W>yzXdyCGSZA}X^m|5d%9($Dmw6N>7k z)%|(D(6bF(JTCq){dY3)*{tlCSi^)TcJrcL2eiLI5Ldda);9&p@A#dbf91Ui9j`!j zYCZ|LTjCsoo|xci~F&nlhQd5^F; zqYkPf?DiGe%ko4gEOpz#?K)F>NGr=WRfgL?zyHHutzY{K5EIwQ)L1O%KM6)C|(e+Rl zG!7gEG6I&sKTIgPja+tF&tUu_XMK0~gXR5wc2*xKcYjPY;?{)~mZHcI&5A^(yy=xk zQkHW*0;cP;&76T>)X5GY(ov3dj(R5;tXwd%-zAgbL5_J6>`J$&Ap^<0jMw|;Gf+Tx zz=}d~O3jR@G^4uE18P~qL<_~uOMU5I6zeNxsIsS!L+&Uwdl{_-E!7)&%<@Yy7m>>E zaS1O<dL# zVVk2h;&edRyT@Lk+*+sT-oY;d(q;-03Xz>)ZAxUypWa4MW!xC#M$cjiYtX zPf}@;SL#;b_LXVJFGz$8jbm%df>>(2SRLG&+euRo{bM}&lvV#C<(agXi{=gf`ln$l z4v$BhCju6$wPxzt2I2c1=i~nYqP0~#4OILjfyhr1{K%+4 zCbkS#CPu~#&OZ}B4F7xl=f5>nc_S-RV;536R~wuEN`n+7S^J+f@U=eqSp*JkDGf@8 z7l}rJG}xg^rc$9)MaczrG`BgThc15o`=zKY_mynR%O4d5G8XSeF(C+_LJ*xS z^a{N9NUrawB`FPynxf|+Ng1Nad=^W}ynuo>y_`cqETn+#z67g`!x`Ck@1Fo;vv|Q| z5O$mo`h^MA>**+5(ELbXEcS#Qk`#!$*Hd|U^aT1H)e*4l_SvU?YhC%Tr8JAWpw&c& zOp%^$dX&TU1Dq#`Hc~3M!5z@xO%_-N(Y*0$~tOy z&5>BkY{1E?ePGFS5Q5pH z!r{R(GUPTwy5~KBz|3;A>f}nIcKCw!d=-VQi(1>Oi>exp8n-*%&7Rcf_Z;sNvqTxv z@8^?$86D;`_uRezHvjZG58nOxzT^TrI2Md?oAu75k_jh+v)wT7tb)hbZ z=kTJE%|)(2$T}rq?4&rs=IiaKoeA5KXp0Y!0XA*~t!1oh9tDyTV~`CHNDV=INr!8B z+<%?6rWuTh5>9b79N|?$uhhn|sy^DST!cjY=&mp|iWl(WwmlBcR73Uk!Ghus!nY9& z6BLiCt+hqLS|{laXaDK01t!9FF&jN75!zm!)a_%48IUG$EC~1zWTcrmsgK+a@W2)! zBQfG;7QvoZg~OCcVz}#acbaQtjbV=2=<`|&tG55r!mYrJPZI%qc$?c`OrT*IzXKDY z^F4+ElM^i(PGH-Ab1;UUOr6~>8XqDJP~7cZ%DEam<6IoV1|jik&ImE8zg`OC5HVj0 z5q^|vH}_x)A(EN2r4X`9Hao1fwTnDf3$?boa9&*%Q$LCAv_ZS^NDXOXk|k=fH1=g} zHdHe<*-( z*bq#B9zKwUMnqHu>;9X?!HB5g!sG*|Wt2!S zg7wMoPcQ>EdM3fh#Cg}H0i5IFk!;om_Sy(Xu&`O9DJ8|Q@DPaHvl{44+7mggouM2E z*Ie8ki^7f| zbUFbvxsK{Me=KFNN(A~uabLmauC&F zPD6anhu_A$_+;4IDN*g?1JmsR(K5ot1s(Kg@M%nW%~Gb#_ldL@BkLaZ%L7;KgBZ!! zt_9@m+xQQLBhw(J3X z0bkoP4YxqBMPY7xP2Xb!2yPA#+?;N)VnY$33tjosl|o-*=$TIi;{o(&a`}^IYtu{v z^gB0(b}?nPnZ(mEm-6DS7xL(NDLx_S>3`SE0SyRJtsbt%oFd{IaiilRDr_9=iCY^v zIdnDK+5-l;&9`W|TT8DO1OrMZy`22&&H^3m8i`p4&@=U#) zeO-0e24)??w?O+_xxps9U54yhWmv88-IdmOtoONi!;L*X~LH z;CM`T#5p`?V6gn=L8%i@4r;m{bn^>QMQD+<3=h8|)9AVb4~pMq;aywi53_M|MA*`= z>NGv@?UCXrfvlgqK!(2|<#Um8{|-Yi_qn`;gt*_<%09PO@v3H@xG-t%wA}({jq;fg zOcKW!p1g?2*_)PA@#shVRV?E?@|qU;k(^2fuLHK7qggFtjFi6ZG(=EwgyxqYh>#6z zqeF45*Elj|_i;wTK{(B3jmo&w@#Ij7#$e82RaM#QRLNs2o*`lLnzow?_sO5ETb5fC z?~U~pTpm90(pT27Hckhj5o+c4Xv_xkz-DC5*SvrtA~3Bi^5-Up^0+nVai7noTU?7_ z^gdF8n{+|wANFxCkL$i0Z`r=yms%S^Ibk}#S0HXAR}I_WbTP?5BocbbrtxkZKa?95XokF=nnm zjW<>Y^`%Q7drOdFWm9M8YJZ;0;^xr&pn^a;HuhW=kcLB9r>3v6qpmacm%MsouEUbC zhB*^%Rhu$NstxT1ux=!=NX6E{06WZNY&gcJNU?>+i+-qmaD4Xj<-#a^($K@X8jz_OV7$B3Ux#~0|iMJ(#t_p-U z$x_8^Vq_Z+c=+s7L>HvNWGjjyu7*k)a1^a!P`-AqXt2}C=#hVA*3C@e>nX18&vn36 z%pvR_Z?1bwB3ff@bIK=-=%;DDpr7zWSZb8@bvAVnrpT-$T|1^(v=04l*hy7Ir$E_R z_ry1Ik6;}ZlePgothmfb{cawK^@>qNhz{5mzKxt5->-ETGs8tB&JNRIB;yBAOMk?x za9u^PJRlA=9EY!2(Du907pFLp$FZ~q?Qg-ff*;zAi8$X;Y67xV3ST5P>F!A6HsKqus4WTSxh&;7ld(H< zdi`u7!q)mMWLvUS9(Gx#%Q%IL9TlTp>_rf5MeR3lTez%+H@u2`QU@c5}#U81;H2^4hK9r#4(*0xYGY%{Yuz!_-4EF919y zG?_wuHqLfBU-(P>dUw%e8$(Uw0W&6PL__~l(`V@WiJ+Wq={)CA!iEzLt5lGF4N0eJ z^tEm|0h~uwqM)caTpJ1wEJx|?8a>#!iakx(EGI5f+tcZa)6MgY5>U~)=`KmSjUzaQ zyK*YAb$@!lgRZS=H`)NunLV%3e>U8jfPy&MyM~vcSjvfk`DNr*A+B+{4BlFzuL+di zV^i$QszFuPf)R(7rY#CT5lu%cn{5jID%}Achu0ktB#FKzK%SP2FY#%l9A><`Jx83}V_1KTp$iMu z6^+T`(jM=_ce0kjvs-+=v09bo+b-fy_@Bf+m7iaxT+{~E)NRC{h@{dN_ptj1xJ=|q z;*fgMVxmQz&E{ahlNdsb$2MF@$AgAaQfJJ%XS0!Qda8qMxg^<9M0WV9aN$C8ahk8B z6{`}Crv2cdUmlrqiKtOIwH)4SZlym)8tOoC8`=99YpdJi1+n$OxN|aWFJZPfdV*YN zb8W}(?(&Q{n|Cf}Tb&^Y-#C9{kS87vu1hmDM#W1z3!Vjj;ZjayVh*G zCoB_Jc|o~rsg4TX*;Dy3K@4USc6FxCeh(^4{3L_~h6^sK!y;4q8@8pH;Z!9oBKd>Z z{Z>v2aPXR*Q#_D1^46jVx_jP)&nR%$eO=g>6nDHgmQ zhnCt|%E=wec?#^S=1oe}YsUml+I`bJ4a1%}%j6cbwBA@~=H*WIEySqy1Ujj+@fqQu z-grIL;d$Pa@2qLZVIObJzx9f)#Qf^gKWLaMAs19~32aj{F*DCR%JNF{k2aqi*`3Meqf$~;9GXG^DLdQNU_m-X;X)bx61+nOlT7__g*v6> zco8BwE$s!S31@*O_+5*#BunU`E#Z41t}#lr)~$1iHA%3GbK;8%ReggrpKR9dig+XU zj)-1%noxxW4pPYRI(8d^-i@jK0RKHWL)w zhaORCsL-BeWpvm{ke3Ee-3rT@3Nh5b$OuNx=do2{>Z)zdR4iv+@%l1jSyRPK{eU=j$#*4PRwi6j9)yLRXMOSL!m&4lMGL;DR2q66^jy z6^+zqmP!mpX{_EK^B1|FA77*v+9Y%lrfN+$Hz*1`hMOc6k5^W?6Uf76GIQ!L&Nonn zB{g`HF{Bc?@gY8$ml+s7K0pv#%oVxOF=8GH^t!giBezu#qxyMKM4Xrt)Bh=1~qZF!IYD!R3RBdB?=lFGs)%9fgV(tgTz=pSQ8sl z%@iYtHbgbRjVL6$t;g-bn#sf(p8xysmU=3J^Gp|J`r?S^(>`TkO=rp}z>k$-E^A6Gz9LPc_yhZ|B96?#LJ9 zq}hPvd-g6W^ht?-VEXom13h%%Qr-r&#K4Gs1F-21<9fj|eF2NQrJ)@xD#kZy*KS*;`=5V8z&Ik{s;^rDxqEZ*ieLn;R(SzZ%+;@^s9lt zKrl*zJYxNX1gW9W_tjC1jZlzqLi|81_^?kLO&TlKafK2E9}EulDKI7;1~-Rcm?`Tq z9Z-VIwkSFwxD^tw3PoBU0Z{*z;FDZMbY#0?Lgs1@b&(ltVP#Rji#qY%eqHVOSLY7p z6h!jbWEp%kJnDoFMbH>b&>{j*i1B!*!H)bDR!-KYaaO!H9v_6mcniDk7xuNn9xaSN6#j9HRBX$AU_hmbKy z0TZ&X2y&yuFR@6Y=pXY~gW1~*qVab~kjViAVKoJE>&8-6bTyjJCz(V#-KR7&Z5Ee6aQJ&)kl zriv#EOx@nB4UEUFs?)UHP+lgU`OvkGK&)IUa;e&9Ye>g*5yj^Vam9;ONiY_@$bF55 zeU9A4%BU%UhE?Tcp@j>MMx_zV_Yb{ZhEVk=<*~@pr~I(e1A#tR33tG7RrJ zIb=;L&EuJ*I_2I3(^v+Dcw-Ok>3rT@me@@3!9>~@Sm-!yhcrho8(yt z^xN0NVUc!QZ{aebq)~pn88Q0QKDERLch-g5jMcxEpOLvKD2pY8Q!(~33H%X5l@O2l{gIPZyyr4xud%;h7WvTCy=gJ^_qZR>}{kEQ&obUzieemK?8kfLABPyB+4) z0XBQEB>w6%%{U8T{&#DfD#_;UjoOP?D|8LGSt;YQGD`_9e+?tKQk$`vpH@1~a}g=O zZK_rDZVcaR?nnA*k?N?3nWkUqDIx0DN|L@JO_&y4&ImW*HTv(Rsxc({f?f~YgJJG; z(w#w8uUr~FqFP^(%G*gWXGOA5C_bG($%(){{%>IV!tVPhkc>XX*;2~L2o|k;@25Er$;7*g zr}kEg*Mu26{zUJ5!{Rff@6caz#4)G8b)=$`aYm%@SxO?-naTDI*slAnvV*XpIv}&? z&0M(Fd}sPgxgfm-r)&i4J_e1BrbRt7qnsk5?m-;&Fq3aS2$E;r74^EyaFc0zAdlcyq%-6x<9?}( zB3Gh%MW2Tld19(?aF@r%6-}6pNJHhW<-v7`TmYo}V-{HlEPs2AXC!aW%Jo@xM=g3_ zmib_op?eg;%4y?elecq1<5DD`mB~0Nwn@+t7h|9*Nkdn_qQqzJ5wrD+R`!XW8#&c{ zGw{^7cy?OZbMWXoOXY?_FE0-*N{5%Z@*Y}7YDnvCw{|o9{^hPnL{6Lp*&>AbUa_vg zeyS;Q1;y0lA1FYv6-!^3&PX8hN3os|ztFOSpmWO~tpmEkiRJ7O!(^P$z`q+W=rCn0 zN&T0^RmA;yutleYsc!xgXM&uczQO||iXpnJxgy^faADYeTfdwJRvge0ZKT@eKUS`j z*S-j+GO*Zn@qXJ9kv9v?FK1=vr8B2fgW=3FiQO2u-qey4^RokEqrt)|*MyRjOP*JX z-W9*wF}@OKud@+^h=XHiQVT#7_Ur z)y(k=LHZ;I`!PU&_7K2+q93Z*OynIM(pF>#fg?uJkUixKp9y@?_Q#cW8Ytt}3+ZgD z)(j(QOaLa2wnjn%2%bWFCRJ)Avm9cLD~EoC znZM`sxo0b$T%A|%5xoD#0J*nC8`F>f=#9d@=e>8lDU$IEKe{KWP9`)?`5;>v*EcNs zfWJ=amEG$HrEef`@N|enRfKlTn+$5QWJg`~3Jl__O%A`y$+&d|L8Zm4MK=fO6+n}p zq_Vjxc~B|Gnh_7(2TlpNxdHok1HGK>fkFL(kA%4pD;F;zh7g&fOXSHT`Ku%f?j$&e zh==kwY(|+nGZZCS7ar861^c2)HY$A@wR0Qp-=NlngLZzq{T}j~H%S^9K`x>CZR&EU zzJst;7NoyCLGj1q{X|{}&v23k8=U6O3?UO;!}7~dgAv^}Yk;>JFFcZ`dXM5LKSDR= zjdOooQNBvjSmAH189Mk8k^=)CW(0DP5kYxIg4a}t?J>?%u@U@{P0fPn4hpDgWq-b0 zrm+W^KA>30)o3-zWY(xdz=l2H(U4-%RuNaj5Tmrw=TeEcq6& zxN6G_ZVv(^n9sjHyvwEi#+t(FeJ_{R5+j*f#i@QM(vH6M;=BV|6`cvem56QXuqy@V zd%VJ&T*lWs=8sSJQUQr z%PW!BXL-TbG^#3DVv4#ey4meb_@h7a>3(v_--VD)_n`w%vM`bWr>lrb*HRA}a=&@a zf?na@LSL>_>H^cQ&O`|KJ{CM z+^69EuJgL~1V|wSCMblXZgaHK-gE3g@tpcUnQ6n7k)g0=4qm@Akyw4FL(W`SZ>BrT ztQTEazP)42Ve*`wwTrS*LSw)r#-Vv+-KUw1&tl|G24J>wJy3m0ih-)bZdTX}+hNhmAU5NEG?(@Bfd(!DHr zO;M~2j$INJT37j9?}j6+AV(lExi@;*i*l?HP?k~eY91hUN)2bh3P zNz}d#q&otjW>Jy+{45xn8oSsG7rpXkSiM3>mPxqS62K{r8yupApk^&xuQn+fAFs9qWq{gQpb>}m~&+6AnWy~CGjlHri^*q(Y-0g-jG1Yyj$U33r}F#q<(h2oI7oVsfGqlUSbsU{E~7T+zXT% z4J@E*G96nwra0A1rD@Wvoif|U+59u-25O=9?nYgj=+|P~k_1w`4&L_x>XhGAL}563 zbXrp*vBosL{l5R5L0?a(#0%>hxX8`r%MaW2O0yNT!yFc;~WB}2jA)od^Ogo{fT z-Nc*?d8$+u6T|)WmoN;K_OaSX@pD06h`wPvjziOLez06l&`$rV)jn%Nr#r^+2=3QuC+LWV&JD0(@ zgNWmWk)Zd{N`U?hbW;Y1KwBr6j=>L-_sM^8@#5=we-1r`Y7wuP6mn(d`9lMnQt%$} zq|_YBqtYe}o9m$J_>uMg^b1_sV-Dp!0`e4+%$L{8yL z>6yWksV(R1ykR2RAjY5SEyLfj&bQ3usTi*XH&TV9u){jJ=Bd<1l@RZ^Evt#r{D&mP zgjE>xwCdut40YL+F!ZcqY|;By{__auUqtqcjCQG;sEY^Mt@7tlwe7gBN5OXa*%3Ei zMz=HJTV-F$r?bJ^U_Ly^M`RB%!VnTKg4lE5?pnyZR6YORQ{!8AUs}2I0}*@Q?$auH zq|ZmCkd|M9iEEYC zq$c=$M*S%2wad{^u)9#R8a!P;z$nLjQ+BYfDxWW?P76FtC#{)ywkfPjo^zX`y{i`e zY-_rI*P}%^K>4?a!ltsECtMn^>=yBAq6DJatWLGUY)W|L$!`<~KfREwC6;{zb$t+J z2JuL}fSosHxRo>#^&=Oj!`|xZRcq%Ea=+0m3Aq}=sb@jM~#AdQV)2dz!>U=yy?Nr=meH%N3)4B z=Lqfyk*26Ea7UKu;!hJ6K0n$hH{Uv*dAr8h*4Q;{IaolU0dGA&QZElPl@35(^iKcv zW75U1;N}Y`tWUXdapYP=%55V9@y@*;K?OnuSmX=q6SNXUs74Q16(j*sQDu{Swc+p9 zniK&kIfM0Z`<|cMMF6PNeh0kie2+K%@gLOx%73vkI_6q zq<7uW;Q1>QfvUW=Mw(JS#3#hdLzcl=Tl~&WXu#VQR_(5&R~P)jqqjowH&oIietI2$ z@b6EG_GFvAqqo6!r9aUw9)zuDaR-8*l01ckeFUG_m&wYv&ODF-&1xAdwP7y*59k{Q({0DFT9KeS2k?fx=;BrrQ8jJNSiR7m31 zH(eq{x)<|Q#&l`pQeuwashq9tIr9l9l5#cuAlBIZ0McH}TJ|>3uXOM8*qKSvC*4Y` zTS#aTZ!o`d4uB#HfXtr<4Py{rmbAhwb%pGaiNnFLE| zelHWga11t<-P912F*{5QSGviRaD!r`=!g$3%ZzYI1JXsz@R|+nFk%dhjy>(5?ugTt zdG!#&fzc%sH8M-(U104e|6}KXAQmRWFqf> zT8xN4y$aF6AFL&5#1@K5(S>b#;0%fknof$RbIWb}&=D87Z}W42)v|#R5Er>`=Q5sE zDrznoWA07w;Z0vN?iFNw#_h39Cd^f<(lV)cxR1lZf{5B@X|7mO;y$yn&P8LqYD;y( z^0XbYyzEMvbMm@wtsg(o{wT<615q>igBgIb8lzIpm>uG1hwtkoXEN6q3=dFII0H3x1``VO7qk1=kfOa za8-bne~&5z*fh>+Q|W%l&iSoJ{+2JFF@cco`oNl);L3GUZ@uuzCVi<#8r z@e_(cbXna+i*`E`Lm+B2e)RpW^QKH3?Pom=Tmnnk( zj1C%OLijc70U>yx8O@P9igv&_!>-rIH8QJc)*XyKqNrnoCmiF5XC0$1*{L^(K624M z?}17;qD`u6e=s`2?J?zn8HamQF#Q4)7K7+G^a3>+qkq7ApzLGu5lVX9_L6hQ{SkME zEr*i_TW7fHm{x_!<{Q~{>ESa01-G?{H2*tnyJ8Bt8tVzQ z`@pVvm}33P+#7ovkr%E>OCd>0{vFpMKlrC}L6gD+qlK&DV@kRBt_nf)xBff2vYZ$8 zRbGiTH{-B;th)t;`M(Tk=j_E)XB8{T0AJ#WZd-)E(I=aF?Y9E{8}4I?vy zjCD+!X%`F}N}|8F)?*~Q7y&iudFMD9oXRXzv^2w@00Hwbbuh?HcV&+6o2lTSe65XvWD zAu#Zx5aqKP@WJC-XYi+YFR^f{+r$qKePum~HR_DizhxUJDZ2#+xOr8EWXFVx5Ab=M7ja;k{x%GEf!59V>x6Q~! zuy6!PBOfab90w1Cjw+}pmb)zzOOo{t*oX2U10$iG(T(Y3!T#%C-qXRqNBlm%R`e;x z%T_P3vSkY#P(v7F#>;2?pP8-zW89M#s}gb@yEZy?!Fa!deV}2Km&ImF}3^Y9{Je>$hN= z=3B6oOv^0YE=ng&dF>#E;RzW}kG`+eOrT&#>&A+6pT*r}yLjESqWul^20<_+|BAme z)+gG~TWj=FU-@w8@!f5HWhno zQ@j6;ng+Cs$};vBy`~v+cON4ybH?v_LO`hNYGsRE%jB7p7tdpJSxedIqVn74^WUjx$zq3@?&se3 z>-U@IKi}@(?_uiS?}scvGov>;Sfz|yOKPoDOG`Q!^63$c9^&5Ktf`cnXmMf_R=DSB zwc&QH#A;I^GT=Al&QY2xR$}bQ%N(|inW!*a)40>=>)G{2FqLY*1|wVNw5sCKV}q-0 zVKQp=!%Oz;;xx4f=3_Z`)o4scX<0}KJ<}}8_AS<(c$I2@ly*Bh>6sZlq}X886qq7d zNI5z!(dXs)_#;7x5v*GyT4Xegn)^N4?1O=I7?Pg1h=99Wn#0zNhG}@Way5xRO)DWj zA&#GmRll{gs?*UM`Tr3iq7HqR<56at6Je4JWm@!sw`nB7M>p40K!gq=RgDQCG!9d< z$D3-GK=6#IVRZyo(_UFvHyY7Qm8zi}YNOb)YpZ88)bnr$WkkdnIx)JAW0OT}<>U}o zkT2{cSV+X5vQH~yBRkK843?r z*`HVlgn~rW%nM|b+zD_Jgo~gCc>KBNO^*m!ava;RutjBBls0r#gjcu3;1Z{dn`*U~ zN<2-)DIaA<5UI9>rpvOz*0bB&yp3uZ=nKK3xz{%RAy4;JkUMj=NTh30kZ_be7hcqD zqD9+&uAHD3RF9n!8CO@oA9&tyhMEsbwqn$ba&Q<)3IsTX{H7I%?Ce@TMTb2*uH_;qTz~lg%8u3^jaI>$5=~u~k_7K>>|D2B- zM%us&(?7sMyV|(ZV#JZa$93OBr?^uz9cfM;*8_s$Wdvg~vV3s~|C%FOhgD*hgwW+Z`;q4&*QBJZ6<4VRe}M1FI&0qjS*F2m>^Y0XaxxfFIf8IcSAP zOBszkArol8h{-Cd!vSh2K|FcbQyDEz^=@vcg?RP47G5gLR#xi0B>?LLSsX%}@kxIH za}I~xH@N47bC%#_Mo(yW-SPBYf_GLN@lwp=@$L-4C%l!h`1M#X3BwU8N708Mfg)(BA za6pc$w>KoPvH(4}RAtJUCT%98Oc`Mhy6Nd}kl~wNqQvB)bPgJ2(l(gB8vIxqAoz6R z8nl&g&dqYkjdG%O$X4eJDC??5OT&cQcf!}L{l-aCjaJaPpCbr%hjbn!j`Qp%NpFUP zun*yK-wxY4NbeYFD@Vl4fvVk4A4Q?I1RsWhV8M@tRT7vn^$Nfs-W`K*K_xe28P)DE zz&$1|?8kmGLTJZZ&^Zw4NYOjouD;tjduKB0cHaqdSVVA*d!}Ckje6|E!)AOGh$7Iy zKPOn>1sZarZw{L1fO&0)0B?`Iu_KDlS`e5Ls!_{HY#jkMRF*$V2o02*hL9YexSWxw ziLWu=$~+!F2`;J+&V^v^Ub^EVUyJHo?$+6gsE)S7fzP0-J4lDzv(s~5yx>(?Bk=pn zVVlAxqyd&)LZdB?3h z0a16fGCW~Oxn%=-CO0mL<9^WUjT<}Bs}8t&tI5+#F4r!UxT0289rPD^3)F>Dwlo6x zdXbaOU*~7O1|V&V7*>;{0qCAz@7QaOtTlHbZ*zc~VTj#hz2>8cdATCY20a_DgtUp8 zI0S!0=rfY*6?v?+G8~gWx2&xSQ>_FZ7LBmld{fX3Ca5`JKMDf#_h+I_chk73eB(qLRMZBiMt#mnjBb@| zpu2Y?61$7*ak|9+Ni`q_#Y7FMIT4gFAwgqZlpWF}5ht6Ph+#J8cG^ItWGAUY*wec- zvt0!8I_p*X$xsek?mO#xeR22=AK0@Q+}%7|uHtP6{OA28>wIf6`~COH&X9glZEzue z{XMYam(j;|pMD^JR4>Ap9m8WMRra!O*_4uftZfbbqnAxp?UMDHnrj8eYU^chMuL7no* z267A*a8 zEuV_iy=1M*MinA5jH7Ez*cxw_87Hc0EjkAd8f^;coN&{dJsyBxtVHcZ4$q3D5cR(f zNbwVQBa@z0XMU&q5~J}Z=}4=H9slR2>&%KhXYAnZSlE!H6P<@|H7NUeX-EH{y zA?MgE4ze$nQn$e#Zb5q0{A!vEVY13#(jozwMy3v_Y`FAFf z8DW-IZKv1#h>yz=RxOyA72XXp_E}dR7VhPCU#}ln-)6kQ+RKKQDsg2E zt=yVgZof;8em_aiCiWV(2g^J)J#Fog2*n)Xo>8hxy{imMJIGi7ZUC%J!MI^ zAd())gUgc9N(_eGUf`%pRq5q=W%auP8D2>!o>3^CxajBjDi!Vx5`V*n&wZDw5a;sw zmPlz=>7FNwUDN=ovIAi3ldUf2>u+mn7>?Z|WpJsVTs)3VT+ETXe+4H+Rh~SlO>B+9 zo3*Q=xWoZtX?r4Y!=tz4hzg-EvA$M~t1?IG?waR_mL>fB&~$`1zwvDYMVH)m%hQba zS1A5@1JWx_hu?b;&vcoIpGI4CiJ$7OR3YhKJ+&rX{Th$nVsalh|I?N3kL$66AtVqG zBGUh#`c>V@(&fLa26by^?PYY`(S{nx)rM+P@*#3ca@Rxa(ZFJ|h$O5)bfQ`#OdFeQ zq=^<6+sRZ{^s@=*lI~q#Edt*Vp`2vI*A-~E2oP+7U4s|kA0jUR8>_U=F#XJ>bEac; zwXvA|i|Fh$*LOwF`=;eNf$#eUk`TBZ+x}gFqMnV7FiTdJ+!%bb-Z;dF-u&ol&8B~1 z1VTJCY~-Ki=G40a15zd%MzqGrnA~V6h52+b-CiDtTG%LJ@=77l+6NxhG z>fEF`2)v}V-%fr%a=@D>jj0n?-zclh473e*W;-+J=&D`-TWboc1y2E1f}GHuq&8C+ z2_4RJNt0!lzJYGDQX(nyO)v|9`~kwZ=>P@vOt={}Cd5BIY?#Myk9 zfy>HjSfgV*KNl_5(Wxj5MF%n+;j)59JtLp_s%XqSxHbq)V`(xH?W5)APPnqkauz6X zqN=nHPA$i%L6XMOL>0itp`n_A%$+wQcXOphr*ykFP70fDw(qkx_i0P(`QrSEu_7HEpMZw`FvFD58 z(fhM#d~ZIZ+f|4>e$8)-dalw1vmw%LC^Hgm>Ohnwi`CngQ$UiogNOAJyq^>UjY~6? zdO(I{+(u%FbL=k!Ngf9QO1PMM#s!h7qr@PQ-wc%VFo<_4P3P~c$yu>H^qK zLiZ+cgfiHU=mb_}%{A(TY|_yf z<7mRONGuMX@;(!g&Woe~qi)7>!DCOszBq|`r?X*7EX5lXItA*s$m_724J^bDI%0fz z%qLP`Rj<3L#(fwV*Xha?MW=N(q16+@PHhaL^eiJ*R+z5OH5gzM9KZf*GC-v>I{v4Y zdotWj7`Nba)?urekZgI}uF7Gp>zn8y++rx$8l4PCP=!6V2B+k#+)`wh&ir3=y>oD^UGOd%+cqZ7j`ogi+qP|U$F^I3 zzjN!>xpnIN(Y1O_SIuNalIQV!#p7T&iHKrfbbDSvHb zI+<$07gTY{)*)JHz+=7@m#ivw|L0l9Jc4#2tJ5F14jSIe)bU0W|HJFHdB0s}%lQLt z{9W?$_DNyd-SyUyb~>;w`uhBpcKDOS$7WG-CAX%{FUo_`8#v&^}&@6{pZ)DG@ zywh5+6Ds{0&o4L-4>K7NWqMGqnwxhro=;IbAp4CYk);v`lY~_eRf@KE%Z)(MI>?y$ zD`*X#umCq4K{t$jH-cfiaCf&jcQNcT$}Q zOCB95n=&TAY_d(=5$Amgl=j~!v|AKC;~&~L#80~LJSFUmGp{w?y_vi{efi#QN*^i^9GS?akhsPq!lI`2Kyo1jh@0_Ubh+`i)iNie%e|_%{5T!n1te(FU#W(L@jM@}>dZf-= zqO`Id9I;&stJ4){tI@nV2G9j_2{wOjV4bp)P5maJ&)~R3Y ze@A|yZn>nR9#OIDXX_vSN|%C5)(_#jK*|=J z8S?3dc?YTyPpcv?KM4x6ChK%V?kNK!_yuZn=a+Fqt>gGtw+3YE`pKTJXBWjsHQ>?+ z+pHH9eh2rY=fB|iOC#%fa5a?mGlPd) z69|ao|5o2~Fm|+cb}%${q&IYQq&Kj&b#ioa(6^&?GBdU|rn5Hxp~G-=ywsA6S!YA( zzOI=8KoxuHjn`W>5%S|iOd?^LC!!fLYlF8fXTaCspoFe#tUH{@SN>GpoP~m0^ z0!YmhlnEs(=P2TpuU4mhzccO|?x(qG#>)9bkI#RY4gWemD=80OS~U+SuROHL`?ONb z<=pMJ05-Xmqpv5A=ihRAU+>~o(CNshj%ggiW@^j>8&HO)95MF$UoY=5Av91H8a+G^ zptmYLpvPCuL)BZS2UdZTm=`XVr*nK>r+s~i1*bM@$BKWCxjZ6Wm}cf4dk^sxs>e6v zL<@WH4Y5Auc|`-Xk}n2esq2~6y=!hPnQ3$6GJCAJO*weGIkFWm1e}hgM-Jbf35)63 zJ{YTt^}p6aiyym`nhJ@oA^Qqn&!+<3q0h;Dg}H3eH+);M<}w&i>B7q&eq}>RYnFvg z#=$m&X3>^BPS z5)AK2ea$j`$!1R^(^y)=&SNK$i7RSWrm3jMw=@T3v?T?(TWYqZ>8Yo-!1RlUKihLF z9+QiDEhKJ!oGL!fLn$U-wB?3Hs-i^2Y{t>_*+kZR=ly13wM49>BIST{GB!!T*0s@*!l-0sy<&`} zspu?OGh{H83{*jryS9oT)n}&}J2AmE#mFdyI=rWq_#7J_ca~`UKD35V&B~Fa}{4jt*8> zMWd1iS@~SCq4cJCICbcsEowb#@raEnuP_3wn5GRr%DHLNQnR_36rvQb-8$Ona(yrw zw$Ls|C{z^9aYo4DMk1i;agr*YpE2BMCm5})_7VkcG#5DBh-5Pnfa#V+bC8RaQ&udF zZhnCfOy%$rm<#DOqUUA{_unUbHvH$tI~sVn(#67mIrwi&%&8xK;qs(-Fq7SRxI|p% zqhgtb1s`_$_Y~4512CHAN5vZ)hiTY8AwU`FZRoSG*s>!EGno>LwvkAvzfRX5gNCCa zFc)Xj5vWyQ+{i8(_`)7X7&`+Ni z`z-DIf#4pW;n4X%2vBwZ2ZA1(CvcnK%0TzpdFWN>2K|vm7nI*ju0|uVx8!Ox<{p6b_Z87%xUwqSsaUx5-=v_dMq)wMDSBcM@C>B5r^RsC zHXkFp5x1^0EXty&NqqyeLssl>ijRe?yg0cm6DOF z#y22X^t2?GZ9W5IB4N5$dp-*yWI&P&oR~TPDP-A5(5XIaRdzO_e(>DqAonMf@t%7A z8e!?r#EY&r`#3_&G%JtZ)%$%OZ=t9LKLWD0no;9)D#Q_COh2VfH~WeT+q%}G883Uu zK55LBpMg(3UO1{X8nkiEbgQ6=n*tWhd6vJ!GM6Zl2<@LjrCbRc2ZQH2W$E@eL#n%n zt^x$^Rk6TXH@$nCwT+(n^YT=&@JB;)-}d5_ifoujWXU|kt7^N4-2ngKejcVX4%mEo z1kYuv(2XOLOIZ1)iyRT&#_YDwJf+g?1gE0&g?%eVhER8odnOzT+VXvvu1agfhXS<% zmr$=uY9Yr=v!y3aF-J2$&wxY;F!?*!3vVc{zfRU4#t*tXDAG=?`Ixv4nm;xY-v%2_ z={KHBZw+NgdipE6#Ov!&WN*K?D0`YZN^IWsRs3e_%f8n>JhKDTBdfd6{;^S;yE3}w zdAqIJELTu)Oa59lQaL>bSJ@^Y6VFT(!ELaSfULQsMMC|B&>9oL*`I?C zM~TEq(NF;q8pWYs=>wGk+}mQDo@wHh$ekPs1^H!Um;PXIAQvPM5JGKK_*+LBq;qZO zUxmu(_+ZTMirfBQ^`^gF%B#DB6n}0+p4^>cnIP+q(4UQy7oRB3{sNo8uSO5YFz=w( zbF`3y{J04epkRc>C019;fFoHdN!wI*rJr2mjpgQ*G>$r_;;FHAXDp4ozh9G95q3uX z9=AA;(MngedoOA;TX#-5awdN45jv$#a4x;fcfRnbAVUahe?`4jKptZ?P52{S8?%fb?&`k+MLWrUjWdbQ@D z#E-ATbxAH6LB|<{=qG0}m5l{EPf0HM*2`6m1td`(tSZ7DgLU16Dg2SHs0^ED#m=?J!FhtH9bZ$N#qrNIMKaQ^Eag6HehWyW0cXsnvs}l}ri%f}lvo>?qKau)73U)iJ(+S0TDvdp1 z122x6Dqgy{f4x zB=W*)Z_}HeC7*=q;us5^KN7X`{EqG*D<##Usi3$}Q5asf&VBu_nq=v#{>u)T(Hk>S zEa@aqD6R1AH^-%Ij~8z&ka(HJzwTE15Nl;}lrXSZKJwJCkDK@H&z8Px$nOH0jP|nl z`JH(uF8pao+h=J<4d0iWOlDkd)~)0aSG+gG!&A#7-s=p0JZW;0C*J!$U7lm<1m%G~ z!kG2nkZ&0D)3(q!#!e!%){xd@Atv`?=xOo3xrabge<~suXbVw z`rCNn`Mh46?dEaCyvdpRJJ10Pes^^m6Md>-yYQ&kEG$<-gx*H zok#6d^ED}g3f#xT&Q8+hW{N2CJBGu zX7St8#H{Y~>;p^--t}{SIyOGotm;84Z2ll#rE9T_K2h)_8*&(+P*yObUe@uDDZ5xz zk8HlZmoQswm3~*7o!Jei@Wd!tlR-S6 zc=tj>_M(o`6q#??&N)`!{;FYg-l;FRq4SJNn|y1Kh7U1Y5U3BLHN3hzM(x`!M%@#Z ze=IR$de@? zLzGJLF{R&V^gVI;wRpq{5#>;L!aT2#)8tnn&Mzn%8VLVb!t|Pw3s3bA`xCS}_7r9v zXw*GoTOJ*znuK{ONvt>P=HyGlmg900qwKNF+d74;&)y^X{!&Ta_RaME2#Z9MjsvZe zx~gRXqYekpYSGu*5x6ij%w$~|@pOmX-?Ynbq<})EU<9$ie@pmGh5j=_c8&2cG86*v_&mY89aGQ#G&z+yXGZ zTa3QnOHEgZc)Vvh27ZQtW4Lpa-up{eSjKUi4x=g3F)DhW=TbQ7_2E_@uiZIz?CkAV z7mCR^H!ty4OLJH~HShn@CH*jUuUWwW1Y_`f1Xl z$v|W`jaE06;B(M~M_0ZoLDo212e;eik0oLr7fD+p4)o_iGQ!$`Cu~wWfq%-uATg|`@{G@R|EeWd^SISK>8?nVN zte9zSnSX_kEzVEwF*Xu-SRTlUV)a~PvaA-}WHIEzcCA6;XtaH^-3Z(X>) zdh^Z^ry;Bwm-Py$J$W$2nWt7HPM*lWKQp-24|TWTTgPS7TegiSY-@#;IH+Z%IMbXz zuQqFW|4kp-U}_reO=lp#T-sV}G0B`4bC|Nm_|ek#)B%031C@G2WC8m;UvaS1j&P*b zn5O;V%J+;VfZ3_sKlC`5eMDz$mT^s?K{ezy?f})qy>64#D^F=yWYIJ)nX^uL#Du!n zBGa{OrkTz}jtXQQInSv~QzbtiW^>;M$qsjn0UonF2hXJa; zDR&k?jgGo4Hq(Era z;lV<1NWdgc-{OIb@_WZ+%97^NCw+`yR8wZsqg_#mtaq-VSZM%z8+BkiiOoX+U9EiK zCG9U@BIVGbv zk%W8m643WH|E${kO@KCoiKYpw7|W=)WM?u-js#;oKanz1JUF>#h2wV%MFevuDQ*QM z&!)?_eh{k+4RZo3DpbW#OGZ@3SF$>R84Bl`@7G&-3!ksSMm%#*8JLVq3b&FH1!WZ- zZWUq}jl5(1Z(o=41r4}Htlkug332+Eb0KB=xNk$W52H7^h^tUA8cE}4VDmg228@;C zVAZ!+;Potoo_n%whGWnV1rGg?paYIQ^DuiCkbhhbGXtM4Dg19@jBc8Grs2L`6zd!$ z!&_sb{5ZW_J`kS^XjJFcxZ7^RMYNu>k6+Vdx$;Li<61{H?zRy#Vo^v9-ZZ8>V zP4|2|MTUBmrh{>820?W3Z7w_yTsAE!#;NAN?{{Ng>DKbx>O6d}zU2~Y6Ev!nZ0xXv z0HQ!gT9tT8nw6=WDs1?R2|S7xsr z7GR|SJ4)Rq0GSZ~DbN{f`Fs{f5D=gV75w{sM(zhP28>Z;QCKS#1*2g!cz!89yj+QB z^DuBYaw3>oTEq}I=U!AK7&g-BVK~T0dCjo1XD}f!CWgFSR3yyO`XfgXIbckX!_&c_ z+95;E1ZXW$Ou&tF7?uz;8e0(;ky37P%$G|{vmdA8tbwk|>ogpUEvO(A)e$iTk$~Vbnw&@bSS{;wbsQ-4}OdtNy_Z$c%YcP5mX2Tjiv~XD7Ds9LUwHj8j2E1 zufJWEPmaGA1*}FuEL;moF*dt&z*&mVt@4sj1o2iFUW=wPRbC}e;9v5k_o^n?Fo z*D(oyw)6rl5I?=}3wpg!makVo1@q;|79<}&gL>L!9V};?Xs7oxU#ms0m*^;#{Lx2o z4|V}Mlq)k{@n332fdSAiBytKl$tWsP5ak_7b3s}!6j%&$o?UmQ%&~{BfM6{ul0Vz> zORx78uk8)-yH}0mhDLxN2qdm`cYV{8d5)D`2v9#vEp(ksx*t%04LPs0!HptL_Jv_^ z!V;5mH#@5tlr4xqygo@iD^Yd9%7;nm%#j_js{sZ)cwnF1YgTF_2GN<`j=d)s08#ig z(ic9JhR?0Dx8O1X9IlH8l@Oksh|7M0Cx+)()JF#dGKT{zczmgbzb$ zaQQ0Ai#!Tz|bwMQyMJ=iU2r!?b>M+BO(1 zZ)6X4gdrjXCqS?PB2VaKi1fVk6cP*$+jpU;vJ=e=0gOZ^g$7p%{wj`vHkG3Y-5|fx z7{l)dcwbl7D(ZD341gMv-Lsn9XdNU!1_nc$Io^>KH8{oy z@Eau8k6?yyV#8cXz8vXn9AXgw^oXu_M=9~4|AC&d2(0A1rRN`AAPuwY z!521fy8^|xLb7dQ^{S0lx(Pgs06HE33}4+vwDLopP{q{{+2=n^vWKJW8A}VHkFZ!w zmWUbP2iX(IyhafMp8?T21o}5dFpV6%>ok=AqhJtW`8Q?lXElnJI{!`q58;rJFec}3 zE~`A-y1d#1BozXLe}4N&E9{5SO+fV~ZMTd@WN@s+a~PEDOa-Kqsyb45T67$CBbSfF}RF#^So?d&}& zB{$&uQpaBtmCw?FA>cpy-RD6%ppH35V~wFC9p8z;$oGEz4y}Pw^MW2R{MF-o_)AsQ zF7Gg@ZoQ-V&Oz6yz9ldHfN6~p0mDER05A2}DZV8lczd1IseDd=>3e^zeoM8lXnu2i zk=ccv-V^dY@lb`uq3WKcV_KP}77lA0bjYVZntb~#X9YijK_tsW~UpO76Ix1JenJ0n?7t7!Tt>}cnGFC9NB zp$GNBx73@Sy3g=Wdb#`eJ{$W{sh`DsF7NdQLYFGChXCKY;xRIxm{Yt>9+?uUf7iY> z&t_Xcpm;Y(f4JkEAN+~<9q3|kK@OiB+?UPiU!`?j1%__@v%bH&XYmI})% z`KXQU6B1_)gK4Fezn1&&T4sJ_vTF!KSg`3=r_wO}6xZ9sP-+T!*YZ79I|`+8Q#(g8BnSnB62 zuS5`0^K>RwufKRq?x}=l;>Lt1JJluUO=dqR*>`y?dTU>7fV!ZDtL*`T|1t9sQiJ>qgk{Ewk!tdeP(f zv^^AyKCWq28Hs+0+1MtJX?S#FFyVH0vK?aL4gYVCVI=l@p~pZIE2F=RHZ#3HjW#o_ zzl}CCqd$mNt5jgU?R4G6+M^ji!^gU%SarSZbl#=Rv$IhbR{_<&P0K@ICh}U&+cIh%R7-HVI9(%_x%;fsq|S z#J^+44XG#+@jjvlsTGERc0=K6Mdn`#eE#19>Asp2z|*98+l((YdIkilos05Mos$3y z$6O#8R%Ejy;-VKDy1{(8!azCJ)&D+hST6)%7}VB?ZwfilWaWyu`oMu93D%vT^z2_0 zaO7J-UFYX~F7&rN(X?SxLp{HI@zkP42c62#@U=}sCS}_5XmiNf#wRTX)4ctK0_;nf z@JN>d9p=W1W7n@(Kru6Oko_eGR7|sg57E>|bB!4&|3&E#Fd5#$)^aXRN zk6Z-^YT&VLQP;-JTt75Ox5zA?8!m(O&dhkF*`m(;Kr7*AQx`$xjEY@;SrH10v0#b8 z{1qgTI10$JVPa7}C0Pg}m>M$R`F&7ib#7V_zi7o0!7QPEY^Y>R&>ynu#FM2x zoMB3dUS|j6v^6ru!jQ!#Tpv^0hJg*b2me9?bwDjnuy&tyG{MT*Zz7e2EL~rQv@EQ zD2V1P5MvwwU3gZ^o=5$tC-KkdtheQ-qdz@w^e_iJApO)6do{CKR`@6XWQC(`C7Bek zBU9HG>xCv@s?E<&*gU}C}&N{pX-4IUK^W5$`0V&w@bA{p}aPi9Zla&*`znvTv%ADVM ze#wOPkA6P#aU^`CMpPU>_TBFOYPKZhPL&SQ&*W4c|J!hbW{KJgrXGE)^4&D3ec2M4 zCQ}3J6K2>k9pQAMlo@*0uuhZ;jgYI+#70KllBNhqVfu>Rl&oAtF4&EPZ&$Lfs5LDF z;RgHOjIn1*gZGKd_l@p0Y@YOsDUUo=y)?kenG(*!kp61S0K2DyNm8*beH3yoJb-5C z9|77QBKr7ihy*sCX&-0htWSy1!NArak8sXoOj>Vj#N-!|>#Su>q(g6AaCg7rMd4&h z^l;MgYaZB#C1i`ORh<^fb3L2)Qg*eDGQ^iUX0#dai+^x3!QI!Q$tu`W`n! zU3%vaM$n}b1mm=iT}GiQ#kd>Kc#?UZ?ML zI#Tw}=8OgzJ~fQ`NLMEHzOUh3#=WmtrG3XHK*87$+f4(N__sBE<-x1=lGY3w%3k4L zu+2Xa!x@X~;W>1Nr;%>8%D;^`991C;YAm5$@b95h4`fU@=dffe;m{&c+Y^WYGNSYb zR=VY0bQlQw0gKckxRuF50pfVVW)Y)8EOqDw+V)fPs<@Y})V#V}t#Rv0)spVe6nR+a z(*>h5AK@3WB}^>5mo;Ut%#1xpN6Y;m8tJ--N6Qjl9EI(cU%%>kv*uPl8heV5mM1>a zGjw6^EzNznQ+3PF-*P*SdI0b9R8f<%@0J!mlM^*z7lJ6-$si<%6U?WL~$Ve>{NUzF#X0QrEYTQl(OK13jDc|=t!{e zz~(s>Fo{DTKaE&&Jg}i<{-MN&!N#PA>5ink>9%tk8R;I=;DF`b;D+&{T0C0R%WbYG4Dg1`{;#hCNL2 z`n_Zd&{y$H$v#-}*8UkHR79|(aEK@_IVCO_r$hdB2L&KR0w8Jz@U{mc`BAEoAVlnF z=PTgcdZk!KL=^OL|4ffqdo~3k|Kzy0cr$FX14Z+^9Lwe{yo*#D03f1oWRc zIkaawIRxVeCd@znO)x~)mYD)Rb#Op1OG#+R4NPbN`guXXg`FK%{O4Ek%nt>Y(T^ea z|26E|b|f_T&5Sk$|M`;Pyy+be(*dEU7$NG)%uRW4$j99|WJ4}Y*>8;3iAIF>VCT`B z8@KO{)roFE^kARTnj62(Q_zWCl-HS3n$?-|o6(U80Mn6C{aFS7lmAnnx(T{-z#;m1 zYlugO;@}4d2>%BM91st7Z5a=Cc)lM2>dK^}=*oonaRI_lMf#-W|(#-TEQ ze4L#~TW1RR^3DMx@x}oocYIuZgpS^n7@FP`xV zC0MRIM?&;}Gf<6UZDeL7+u3K~?Hn_q<|gem#94-tGSErdJ zf?iuldSH*m)5xsDY@4Gp%tod_G~pPFEPT)H^MaD>Q5tcHnQizBKR%FgI1L&vA55-P zoHxtGUmhP|y5`>y$g1gcOkDu&UQEIvx4zLntf73PdmTf~>pKa;0mpyy2qtVl?GOZm zXF|&e$Bs)i1#&I{S6Rz7OK~Mo*F)d!kCZJCh63wDpVE&Si_HhCE9W%APH`|mz)JKlf% zZskAeQEoZBw#Fko+|$hj%wIRSs|CMRHD}nF3hx>~{BxqY<^h;2EnEnKW=20IjrKAP z;1n^4@zz2#oPn_v5*DHSr{BYZv8*G~+AB#bRWwmEQ*XdD4d*E}$pZZ91Y>8=7PbQV z>2$bmeE%kwzSE?pr&24)D$y2G)=Hd%&Z)`;ox%J?I+r{L`yUT$_3c;_58sCmdpRjqHc-zJ zaOgI@C_3ms!R`EVr=9w`r%oeJD+B)6PHeE5zpzjR|M^hEB~8Xv@n3|}4x?0@A8$~9 zyn(Bg7{@>TRVvWr&lmbw3#6lxw-&7UNqV!#1v(Wn#n*^8s3Z)YXOt3lE4qX%D3!}gON?^~2*;O8x} zDk>hOkbE~%fX5!7V6pIA;KZbZS!3@YsEtiDet7mk)01;2!ZlWx+@Fwi*y&~of;c&% z2(#q~U5oqwq z)0*!cBi9ac1!5tXYGbGg`0^GH*sVj4SLz;O6Rl$wGM>$8+{;pOe5ylpYoo;YW0SUv zhki$APdd}(Dsk)I$vO^)S9im16*q@gcSGKKfp{b7#44?nOKhWbQ*`iy5Q4k2XN;O4 zQqL~%xpp?Xc@{I~yFLrdJdZF-p5p)fr=r#F2*+k6S!lN_u38Ibtz*CzI|DQNhvE3U zTDX4njSTVLJUxKv&xe1P7~3-_qMA2-&cN5pTH9+T!C(hU<;DIC)q(5%SPTdKl$2H} z(}oiA4WTN*{XvroCgZf%=WNQ{;(5U9cT6E#&7a*Z#1O~X0}3BPU39xu@6J}}ExjD- zOE8v^S_yS%M!jj9#9mGIxT-PIJi$htuhr_xMtZ*mhS+eeTr+pQ&%ya=Cn z;caf&GB^8j9ZawsHzhRn+oOLhq+~*}LY4F54M%&%@2oUVl}l(IfuTWR@MRo0>f#St zmEFx_H8b8Bgr{K7yW8EzUw))3VCm%dR<&K(*Qbh%DP z9d6iw_^(!8u)9SEBcnwoXkDpN>LH=cgCLD**ZHzo@z@FSQC}dHn6&Ll{)|$EmJ(PZ z>2EcaGMK_D4gFwoJ$#7CLBPDEIFcuv2jc3tj(8vcpS!_A%G*;5WSn%H3f`L8kC8$! zH1_NDGVkm8Xya!E$WEcTqh8%y5UUWy{y~jsXVAu_TP>sKC`pc9`EL1HcF#w1Ubaw;H%>3gcW6 zIphG4)w*honOmNisn--Dc^Nz_xg-5Iq9((^I>;#m8~UnorgO4TvF!3jguk8+PV$cu z`@q^|yK(rzUwD?+kt>?Kz&lsMhPWh+1nmcH9M}W!=B&QeOOl)jZ?lrArU`fHKMapi z9Kz74U4CPr9MH+&fp@iGz=^{E7`J~*T$GNwu4=i%82J67;my|tf^aV^`2HV29Hb+T z&L@4w^9L9#+QaXK1ef(RhwHU7@9&R%65ZArL;G#J@#I{CRw}G>(v}@BjE;2XPsaGX z2E=&!ZQ8#Oo*Ayfw9;~3<*TSfRwML}>%s7tp-h{o^LDk0^sKYR7@st{wR2>fFo!L5 zVa{C5x;weYgB~42HI2a<)?_oF0G$)aI`*R`MDsi|vlL8Xb(TAn4l20w!CrzazpF)% zcZWwQ?Nt3+-Z_?dg}Rb|hw`;7@195iV3@dX?vXldk^iG^b*1ROiXkUm-nmEYtZ;_B z?*=OhlH{ssq^ztWx67Yb65dXy-T zc8oU97ynSClwVo1o9IShaf-m?hjU0-v+>c499{D^hGY3L)IQ@tBzK9fQ+*D`{2Oyt zZlgKv96V-Qq$hP@DcSm_su^8_n$V-K~Hu!Llb z7S_IBwvs^QSh-Xa`D_wC7?a8I0m4556tnBiCH5r{PICt&>JqloJQ{NxFFlbWXY{Jd zwI<9|W%spJXTK$R<5%+R9!pZ6j&y64q_x{Xbkc0l>~B0+0$0)4X0TOj4b|(h(wLyF z_kQzg91d9;J|08me=XohW7vC#H^lJxV^#_C)?nqbkW-M!MXPD;cDtB)PjF-SEE5G& zicSWlBCr9&8At~2V)G{VT|p{H%DLh>6aVJTk<$C_>>o}eL?KVQG5GP~eg|a~G$V36 zNx{8w4`;`jdn-(3r$G<=*(Fxpn@8g9nn^sq^ck$NuA}l4i+(-T3N_^+JjMA=h*y8k zv~0GE{mto4Cz@HxCvjfOKzZUm7b5dW$J4>6U;4}3&>dr(N}ZGJ>6imZ7jma(<&{<} z;(p$pC$p}HBRcIy9nHtrW8*N$ zjlF>FGNI)RX#5%W_(Xa59r|cy-0ky~bn{T%QY5B!N*S#4p z{8URlRYFE(=S$Y`zsVS!^Ke>+FJ=rPB}!N>CRGfaIGU=Y$%+Dl7H+g3)1T-V#Kj>O z+6n8Od5sR5x@0OwZ%9uammXF2l~~$$jYVf{Gim3Hj`Zz|F1vtMqjkDJpk`$E+!!Oc z9=;B`fmQJ)=g5c_7rCQ)JRP15mZO-6v;^T;O6^U`Wx2^gQ8aA>;K4SaSFCGoS-- zMKCv*ZwGFZLw%hZn;@MywX0y!AVzqEVPBILMI4SC_~nca=TAInh>RIYKhX z)iE?x=rjV46-9Dt9k0FHB|3ocZ#fa$&SZ#l@eB=<=q&PS{GKn40`zifhB!{G8=$GcHOc z))vYi+KGjXE6&Mew@$nI_TUwc6psw0t1eBMy|EK#`rQ!ZX>Fc+%;rDc+zOoYNFZ(z)Yq>8cGdBHW!;(x>5sJ5--GLYDG zF|v7sI!xnzLVdENp%-PcxWjx8aNEr=Vr0TS@#MlBWzdSJK}WtmIwrhj8eC8sZ{0cP zsU=+UTqkI7efzr%vcQIf=m7J{d8Wdy2c|8Mq_RZr@%zsSbQ5g#uxsCzLn7X}QW)lF z-DO7ZAOa8cTaeiH3nYJ!#pWDb9LF28oDw)8C_j||;Vl)?0x=p2A&9>Auq$60NnKa_ z{jc1L_)EU5Z6NKcOCW&b*Hc48FZsDjqcq*!zHADSG%&`X3S<~y-Nwwvow^rxsW$iB zFM_n_@-nh}Sc3ru)L-ip#rM}#2b|0H<%vb9>&Co$=LW=Z+=^TMPpJ*Y_^*rqG^pG> z4=kRBCO_BRq$bIVJhkKAP1tvlm`8W#mbd!4sJGo)ZOfXjL*6>zTY+<=XTb&Nu5&4? zwT|TZs>}lPehc_{`k-UfkL9?SDu$4@GNqi@aE|os?EWaWBk61Pigc3hZACTk@I)aA zFtF$s!(GO=L<^VSrSZf$6@y7JiK5sBU|&qzO;cjhp2*|2(aC_RQW`}XpH_`Ct-}{7 z_d5-`&hIB#lURux<+l7Y4RL|P{Q5V?U?VH$&WutP<>($3@>B@fB`JF`5>Hm2oJmfU z6{aqGrJt-KRqUWNBfe_ck679IzIBMnVTTgd;3CaK@ZR!k;HExg1izgCivtkC>aEm%7L%$1jWV+*slq}_#kXA?sbyry4y2cj2R`ZDDr>SFIAc-) zb%qDj3SoI#LjT7pa0L5q37pkOP6waUH2I5NcKQ2`_j{z`t3Ttm1F+9~xL#g4uFdFk z#F)72o`DZ;zkMB_Kef^cqG_TJ=B-Q5k1}WSlo`?=&lzLLb7rXu5q=zOa&eX_UnMEP z7V!dTcgC=9zT>Uj-9FS!rTKs(J0gd>0j@wV*j@vSi29R#War-W4BY;HZ4@t$J5k(h z!NwwmIqScEWtgqn?zG>elKO|opIs=eMLYDR(oRqJ*w;D6L_@lvRNPDZ_$p@%3G zeNzdZsX%QS^V_Kz*W+-GbrIjb@3+v&g~3A7R8(sYu&4btJ9(`eu3Mqcrt=oJ0U0GKeI3GBa>y$JlMiEf3J6V{RKg z>{6dh6OL5tqY;a3F8j7tI79f{+zE)v8lq;t+j&i^WwBB?L3a{0&+OVs+;s=wHGe*O zj%lx$hqd|jUr(`DifI`ko-2G{WO4)bQOd&7f)6HY-J^3>Ry?)&A2dh3$q$@ZIngJG zY?-A4FG<%Ono0Ur_JQ5cs+`Tc&scKle9|hOo&`c-NiJY1Ubm#43<#Hri>gl~>CulX zz?X*cx(NH~34V6t!Y1)dhDsSrqXC=R2Wulf)$_PY71;5yKm18Owsux3+v|%WSdP{W zDxi<1a;sU~)W`x<|4-b%%arL^kdtj!0lGuaXBY6k-&59#on&`VZ28Z3$i2ho~`;*~oJdJ_P7R z#npB-3`*PyuH`)SE0zAYV~j}dB@eL@)|PhH!!5&BkG@$>eKh|^6r-kgvWTSj9nI8X z7~kgdV?$f0DsooW<6WGCUf7_jY5SiJA6dU>v9@KHWA02gg;fUgphp4=c;phKGzflMV@S}22z#Z9~jFxQ*b z5-`u+=0FMCP}SyJ*#e=Z?S3%;Vamio^h`{4X^KGfgq|H>BEZYIPfxlMy-I02dZv8` zcFuCrB{d&~_)|u^ZhNNPV z>7WPf?17Wyc+$toiM;T}-HhMaix|A&W+foUAV+}}d_{;rTpY@AYu(l1izBRW`pxLk z#VDeIA^W=fYOp^b!tU0-zlP}BW*N;SO^c;U`mB^*;wd7`Oqr0yDyCw3bp}`fv?KW|Mx}T&`ah6l;=yD zsc!GSh>%Zxl>OQq@>~Ul|5+RnH{FtZkv|d`ntmT@MCThaxnUmy6; zRcQYh&nasq=#KbB3-4~jq6#}{L<)HSv|(aXYcMK|i(?J#3z<}}HDk|npsSjo#`HV9 zzNb(7maoc^0bCnM}&HQtSf@a#L9;sBS8&51$FK1W>fqbVfq&h4;M zZx6rr(mNhp0d=DUh5Do8!;ZRwT%XIqU_ge-EoNzznetR^g$|(<0%C%96?V=4zQ^i@ zDb%YDJm}Qp=4mk$c^z;MVswzrY=Un7M73=qanJy%@@W%`H{Z8jzelliXi~37_Q=9+ ztb(tTP|g-Um;t*{NM;2)w1$41sXhrr3r>98s<^#`(dcm+fcFihB&z4`1do2Rr#%Eh z(HZ!9(hQ8i@=^q%yex;fGYNbR)ZwXZh2xwsKdTHGhBw%Ov%?5_eqdk)4=3eLNj|0F zU$Da9nbV#JzAT#VH?IYrxyDm9UUH$`IF&z;Jo}fbr!F-81vZ4Pm{&at&#g6>*zf;g z>>Pqci@GFzZQHhO+rIDG@7lI)+qP}nwryMe{_5ze=vmEntic|gI1zhg=9en2_*Y1B zw8Dve2a?(#2pxuR%?tS6S*X*2-;4K0Jyk=gS(~?T+aG_!*^BP*iq&{APNPhTd;0oo z3`Xm{joH4cSRp+c&vvwi;8IPzOF9yQTW?st?`Y^llJG63(FWHAoOy`Tm`^bh#ytL_u;}3JwvL z3w{FL@e@ZY|3e!~;~{(F@ppbye`n<}Lhn6Zv`PAV%y5nIevBvvl>gqJp!+ah6n+vm zmd2%HPTvJgQNb#vPk7e=q#;$qP8gzQIc9*@dz0?eQ067Xf4Nb);Hnz#HVs z`9%Ce_nt3AH*@n5s_#`p{1hls{^Pmdni-PVClSK@x%fqMdf?9vP=>W3kJ}OC-phA; zB*j-3_6@R3Uu`H`B=j;GC6ax@>UTD56mmb58oE#8??e2XV?P!1Lr`iA1^=Bd1Mx}1 zZ(wyl9W0JC0boB8V(5J@d8GX7_9+5`lTZk+1heG;wm!F*!xOwWhr9R<7Nq#8@1L`v z3qk!!a<3x@(M~)FFPn@wA3N`|qM7{Qv?-qqBAhuCMp7^MgwvmQ$2h( zUpF^`OmKS@2!gIN*+or;9nSSdz(w(>9nL(yij{UM!4C8FrI*aD?$*}UW^Yzzzt{Om zHYb*DSFc(t(J;A>^pu$qSNI1TVv!itt<#(QB!d^J`yRp|HnpKm1q*@**peP!UjwLF zyf`PRS-1#W3m{Nquy}k8(QHB!wipAh&olquK-v4}S*qjT4FkYbyR;i&rnGjywykG* zaC1AzCFCw6hBvCSoBO@Wygy}lAUjSh<-*uystVxR_*9g@EQj(P+r_wRRAbo}Zh>%` z+OiE^3edDKA#0sf-0gunH=*axU8K+E7W8veUQ^->#@a9e>7h9t0ad`{8K z8fwhbMEcdEdK2kcw_$tbW!dV_F6rjA>@Dstb!OL(bc-j}F7H?;Eg9z@SMTNg)O{e3 zW*xg_F#}RG8w2lOy76Ew?7yc2(FJcMWD44=_a%m8mRi~H*pH|;fL=k4v9K=^m()si zhPemUYBz*hzk@O0k3D0nnNjZOjuO&jf(x(O?`Zu+>0ft7D5Ue)@WFhE=QUgMkl2YF zVg9}=0=9hN{qS>xZD@KNR0@-)m3n<-Vx9xTqkBROUA4>U@oInbt~2>lJNSXb4s_H5 z3m>S4A?jyAt?QkswBmN79UEy0`pc0~wlJIEJ)2FC(JE1r^Zo9@3PyUl%z{er{LdUq zp(jS}ebgz1pzk!yaLH4u%=*J&n0%a^svm`rKj2z=M)q9JK18~3h><;*3fORZ)RrUG z4g4Kt>t>?`OxchYPB?}0G!%^&8n%+@$gcczZX|fKY;S0En`Z8V`8QE)EHXlutC@MP zu>N>@uZG4oHcTB_%@$D*UZACfJX_Obw=_6Kz($&wD{N0J)LE+2ewbwJUZ@PJp!Po- z`WyRwsww7=5c)zFY-V+$Y{GE+uXzV6<-TTS?!)?N&5^e5N~c7RimwdCNc>QO9eee% z^p#S5CZ(QknCnMmi9OV%00FEl&_9R=Y)aG$JvoHhcg7sQ!|ADrNgQ%;!;C=x@GoBN zP1PNLY0)E2BuE7rGCg~4ENAB$CoZLXS&bvVbL5zYS`wB2(O&R<_9PUr7bBHbKGZV} zRenU(rahosNAVnZ&hKF@LTK`GN0ZitKMVCvzMh%^sU*rhMAeI8?NCl`H2*?{6_tFw z>NnW_P*rKYA0#@!yfnss_TYNIO?xV3%ls^{Ul;rPRfC@vno+c3|=% z*Ck&Ki+j~*HVGGJ0>oJ<)e}8|K)8Luf`!b%UYdlvhslBuxro;iex}6F>8c1lxhph& zA;fw$(nw}5)$7F?SOz!S`BKpZS6ZNhQhOlmnNU*{j8d(kB+oEEy?V61qkS*xCbfo!V#5wEcHIFS$zV@dGu9#oe48X+=_I#=-bW?9_{qEb_1l$ zSGDTi`RCzJdR-f6)M`U8fzL-UIau199{Obe+#;_oqM{*U$Pti8xsEL7Xx_QfcxD`j zwi6bSAG|}+fw}$D1}S-?>)m1dhg1GkGOwSb2K)<=uJHZB6Sf+`2&T)-EDyxq(!@1+}}c?kK8tTr%h>OG|PlQ)Sg&iJ{TQ{?={V zgAyz98?q|!dy2mv%A^G(;TN4Rwd1H!ZHYHKM_W^uWGAFy=8J3WfP;QQzvO)FyWj?%NiYE z8KBvDQx(m&1Yzm8S6AKUgt-SHp4Bke^pDZx8KAEYkF+ZTPV>46d}nwhI5~~R-^8g) z_~QRE0>T!n1$@+LHb|6WPucv(!ia*K!aZ`>%P{?fpCzZt zq){VE5xrCcs|#Y*Oq!gJEUWE5jy+<4@VOU;%m(L+=iDZPFUrr3$^_v*@NddSsLH@U z&z>smsy-7JlQjqIlCPB50=<^JgYuVqCA1jN_ zrc`b;@m)x$LYg1LiXjB5t?oc7M^zS(lfy2U;#Y8Ec21LbVlz6lx5MK-abarcz}Qjf z3)sN6a8%=1xwcY?3;DFFq`!?qMNd$ETeZ9`+IVSjSO zcadn%Gf4XYe|*q)@J+lvePhpj-6G=2wTHcsQ_Rrb8UOT-s7XF%WYVdC3J^N`j>j`f zoN<~Eg4g?SdGz5}`~1xm=d-o>2e5(WnB_WDy+%e5zS|V}vL*sU0q(wpP4W)>9_TG&D?Q5*Ev;-glmT0{4H_rHODP*CMtow6{s}4;?;Z(2r9#$ z_%-4ldAbC(sq&?BPy4+HNK@~S7%#DynAU5w24nh>*x8Iy1Mz z2YQt^m&uB8_um!JZUbA#$_Dd-8LX?n03WY~>gixyF;{s2-U^nW$nQkf=-*TGQmim| zoAsmgMXS=0$@1VJ*%92HiTot1@VA;nfoL~feIf|TS9`4_rMTNQDVx*Hoao+^a zTiepsLR3MkxOTwb5qzSQq*I7?AVtB+%1CU7%=(6k3OfH7nka3o0V$aX)d#C zi|w5@o$_0&6;XR}%lS4-KE;56Q&H|p&CRDY$xTwN(O_~-o3A5G52R>6J7^9ctm*2T z+D6<5Tf7dDP z<{~baT4IV)CpvVT1D)W%(PUM2_?)1I&3}06V}|q%#F`yYoC455byOvL2Q+&z1dtN{rYc5{*40 z++euFk4%zp%vQ?h8+a*QRkx{Z;YGQl$T<_{zbKF^rjeklr*yqX(>%%>b`9|J_^jFl zV_9Q)c>i#TOmtC`h1PTrvQu_dn0u2f`5qVZ7IFm^!doPSU!}DZ5&~@i#MU|uPcH_ud9X_ zKj3B3g8C9&t1xF>=xDILtQ1V1#Sq*1l-M8b5665ajxEcI%_cGfhIB|Mb*Q$xYscN) z+4+$&^zzPuSvyUwxmy8GAy-+lU*P0D=}q5|l(W;3cgd0yoUx;J{a80~U+s9Sba;|{ zGxVz+>56xDQU3g(p`JZL=Cdj@pMAvCES1ZzN_M@LGJ?rK6vdyOs?t8JmQ7qJyH^eK zQYZ?|u%WyP=&0P)9~2wt0ozUIKW8^l2T%E0Yc3T$(v_XtgLXR~Y)OnM59;LWDa|MC z1f)@G+YZs^otNnT&aZKsEv)XlnDT$>iB{CI<;?hKjy!rGx$)xl;-Tvel%hTVimKw= zFZKq7^op_biab_23CEr+xBXgtd9%Xx_=#I%s+euNEYwjzNi3A19&}R=WSS`+*h5$7 zdaT7H)$Vu@b*HBNe19L-H@C8TD&q6??!n#Fr^21Rho-D?MBF>ljT2It*$P@6<^;5-q^pUo$I#a*6sxD zNEPmimF8W3s(JC|SXY+3wb%{aFY{#fk#bnO=-yrX8wy1@%?ae?L}9fl-oGzgLW3#NRI4zi2Wvs#j5z0C`3V&HHx8QfsB@5 zvM^e(%FlxlFzmWTWiISVocfuEvy}jgEb2sG=zSprg~(Ez0WMtrs58E?!LVDg#H#@0 zV!063cmqI)p zW(lxb&e_=MLP@m_&z!pM!8`3)S7c9()R+1IiNhzC7eGqjv*?FzKeRx+n_k&Md!(UX zS*TW++4$W&{wEY?5o0);Ju8mr>FY_qoxJ9TqwNQ93Y<0mb7{TV=^-K1TE0L5T|o`s zWp$&mMyCrsnfRK0%i#2=weVyGcTD+JiC3MpqpayGyb(*EggfUKW(0piJG=Cm?Xyuz zy&qb5NH@FQKfH0eJym*F57p=MZ&@e#yNqs1LSL-T-Ao-@nH}Nv;I>H`xU(CIq++z5 zi!!o@^S~>Oc;ZW$k>QLpP>6q2@MUNVF0P9H#i|=rra4fpp-ggmdU?Z7<_j|i_{JbA zLh~lYpIZ!HVw89Psr5hgJnY(5A!GM+lIhp+Af@g~NwerA`^9QQdg!=OrBtZSX)~I? zr~{k=C}eB#Ab(BRj5_!YQx5;ZEEHN-Q>&M0oiixk>!V2pH_M_+%7XL43dlT8sp5%7 z&KAxhQ(7tCsT$osRYHBFHtfie+vJdlGs;Up<6PcAVM!K`{-fYn4taCqEP>{&94eAW-dkO8Ew^yf?1rhC0_G23ai=R6VGj;!{!ow*-7}ne{_qP<&B|usBQVt21_e<}4aTWs@Bi#>=Kf4Z zbp$sZ@UU}q4Q zqm5z1Jp;+A+~lfejssw#nOnCnAp%4f`m{jhZbHwvNe;qiH`ZWzhuqJ~E}ev^IvP$e zO~rXQxcr#zH61REI&H-I;^a294LP~p=U0nT9?V$3Rm@g>hx0}dC$Ff0liaT^HnF2M z-Yo>s@Z0LWBrC`AB0Q?;weo*vDX1F5DhWO)$K`WI(aBxc#=HmD@_mqEp!ng4rM%0z zjwilqFm#w|pf8t_5-H-;YCo|u-p^b(`b2^sPhY1x2~yv(P@u2P@aST5ixLB_q%k3} zfM1jC3n6$J(}z0koktSC(9i3oM52d`t~mP z4u_YFO7$`1&OiSvuWS48g}Q=2qP(50d*t+jevapy%uyfaOpY1kyryLwF!Kcgw}4y0 zaly1{)v zx@x=2JrCUiQ8jF6)gJ-?YNJD0Vz*e6GFe?(lbo}jedE*ZFk;i9R;&A_=R1DAQb`-O zat{8zQ?;H5AS+tba`da&B8vS3U$>^d>z|$Pl4}j@5P$)E&w@r~2=Sv!-U*hy_fzm} z47YuYq)nL{BtliNl(u5K^OiD+j)a^kT9r`Z@EDNksfhX^flXp3#Y>vi3+%F>%M1sh z_@w2|2PXAo2FC6dICt0uCFe|@q!qjMxaSbJ(_|-h{cd5pWy8tnu9bk0p4shaylOQhNv!`T&%xC^QeG60R$3Ki~*{`ybCQ8?fzil9N|>Y9fH_R}F< zZk9jy@KGHu*HYf{vs0%KnfR7uB*s0P0a?RHj!28MMWZk%@vwLifz%DV4_4$8+LJ5$ z{#bllvyY`DZyE=JB{;ukbb!;+;S+B;JH|ZM4^Clg%RFLK5yg>bmWc*aM#*i6ZD1TT zea~LhIeNWbQ};iz+BN!-bQ0~`zx{1YwXfL!Rh4(Ex#3d~RXd(3D=(DE8nI=G+G*h; z?0zbXB#e^fa6$L!TlJQt+DOazWp^FAag+!>T6?lF(yefV_hJWL3yqgVr^@hIX=Tod zRH|n^;<20Ksi7rV~ThD(l&%S=`RY-=q&almbL6-%|L=M zfFKnDlPC9Dbhjh}5+<_+Rnb~=pwl{SKb`rH(S3+R=%ML$obqv3L(GHeVrewcNaSilzU)5JXR0AV@q%( zCf75aNb!E$q;qP1k9~jQ55FUGp8s0{X7di~{=je3U6OtNcHQL>dEnoL1z%@$rXte1 zx2?F>>&a=jCEpRKxJtTTggW>Y(?igSv#1GFJ6Si%xXd0%f(au?)y>0WH}vxNkkIkN zV(Qi7)s*yY_Yg_);OUC-NK+=O;~A?A3rRhTML7v<`fLY!VZz`?*OULu`sUoYX_QxW z#{qoY1%g!L1XB?J+&xiiY2)ZooktvK*U&u`$6P|(4`v8@Oz_tT2#Yqj#p}Q<;73^F zd#Z^#D#Z*2b9}c=rLd)^D{Z5HCLn>0_zIxPf+blG3}tg* zpU?f`%(q%1euvEBsW={*^3@JLjv~X4abbr^{W6JngK9mf0ED^ zs4G~JJON5ScXHh2>oCPv9pM!BIH9OWYqiEYJv z4M$^fL*Y&{DuGv)~jE<&1tfEpV5Ql z>SL}PBfoLc>=-wJE+asycppF=)svt%hO)ak6ap+2NbV(Gu5bxgluEA4Ti*Mjw#=B zQ@D~X`3&!g*7T>u)q1SiGB>=8-O#0<8u*vj)Qqd`g7^U34pIlo5EQ{R;fSQZl@f@7 z74RaE7B*3Y7wPE#8kEuNhjMC|#R$`&OV{HGszWBrtYN_@sI>W^Lb4hv&j!ojju|wT zZr+D-XNPo)1;`#}+Do(qEPrOJ0|?p>Mj%#UEo&}3HEPOs`CJIguXFm*HUug(U9D+5 z{Y7sgHgIQe~{Q4~H5eBpo^=So@86wmKH@@FeUV+n(tRh6AwLFQz)d0WlFph>fb}2WMxQWA$ zKKUaF+M-7nq4|f5eG@r&zP@G~gBwL9-^s*^U22edc})>;ZLRKE<#;p%NLfb$qd>AV z=Ch90A_ia~gU(OKcvMr^`_Ouhl?B*%J;YJCS3g8#}nuI-CDL%v&|Dx$?|MB`RP;9SOqNN3eqxC*wUDy++q;ZYX$-D1&Ytsu)r?dNeJuHFuwNMMp+6KGvwwm82)p!ap z-QQJSdP%0QDzlVueDB$Sz7{whn7Suk`j0>HJo=UoU>t86iB6Wm{M-u`q_oaaCIE}@ z{Od8z4t|$(q*0*wO|u{ipj>g?UycE{7a2&-XPRK_kS zZoZIPi)0s^zFpxfJ0OIjedyCD$&LF?8q>Hqjee+Ab%ibdT=I0Z;71+E&T0n=k zX7ChC5=tz)-x|&EzVvGBDK?zE1cJO@DwayZW_=^?$?)tyJPk^Rv4dR_W<*gCO%DU2|tA=Rd?~rgz$M( z6O^Dsy9_+UfEA{=tc1`gf(pRJL$b-5cjU!sZ`gun0Lig#osKw@@9kE1(*m1_dIr=o z`qYz2POrFRK<#<{<-Y8v7)20aHP-?``3lsQGlFGBEk`PUmlpAwEcK1ukgB zh*5pt*2Y>QbOpa1DJYh|iNgGl?pswCN`Lm-vSTR^z}7%7c7xur*uxosH7YKEZl@dU zwK&baKq^vgWKC;>fMj~uQ@ovk9`oXAX>=p1}W3ct| zX|=YJt1Mk6{eh=_+mJpU4=rGUUNpPB+n7j9i_@ze?l{BPRa;-G<2I6MDI-j32w*n@ zfWiKDgNEWszw+8OucFxEDo8j2j&kpL8zg8$@tcf1@aixFexZD+#bC6TX}izG7=sE_ z6Oj@^)~b}{GT{gdLX*Yy62w5`cNfAzLB%-}a8|gO(fUJ$emO3vC0qcWO2Z{uUdHm* z5Lp+*qcLlw?4OaZ)0v6lhi&U?_8VD3MoR*HFtIQr#Y+Qwf$15-f;oso)eH8)`qkNXf#ZWWPJo!o-*UVFRg;u5`Mhu1q#3+sbffUL&42#Y$kIV_70Cym+ z44Z?>&za~qYWFA3T_KEV2%AN#dX);BrkD~^9=$z(#=Bxr#q?`LU$)O%tppET_8IeAizdT z6Y<`N0n-Uzw}!dOK`9U7gFxpzPq@)O1FKtjGvyNbPLjCbktLjv6+9A=HDCzdvMjmw zEKmFHxep*OV8PT`%x|D77-%1=b)*v-&#B6iOg@J;rwI%6-}8Ln-@ZMzCn=qW?VZd<0pUNO~Z z{<}+C$eQ`+Ej}SSwIz1DzR1Ht8%-8!_t$?vF%y)n zAf8o+&pVVxSL$18`B1PN?`0U=**ixnEUwU4;K^CnUu29pYkxnlD#G_Tyzu?6M?mVG z^Sq4I-hj*_)e4+igmf*-wlkB&O5j;-917Y?3w$ybP1nfcEf8;Ung>M938lzvz_ij% zq!K5|G{%GB0m^TX^I#GrH@IGTIw_9-pM4~OBF3VatD|V@;!@p&827GWkwqkG6uP-9+TvMXDXxZBU|D^ELk~cLZFu~Iz35f7yP$qan!uZKi(vga~ zI;!MIDA3x80c0o*KSDc=RP9(EhFsrV369;Os9VZb`cRyc?o1gjRJ;8HwDF{vW_o&D z0&68`eYGJcwX>yrr!?k(V+j>6H!eUw4yMsUYYv9^tpvXaunRMVUMO=fWY)9hWb7c{ z&f7mOTfe+;P%Q#$y}@a+Dk54wz}Z=H4>eMGNi&eU#y_saK~e|Y_x@=SKt~lAEca$% zFTTA{)JIp(3c0nZIpT6E^iaI-^H<-;ObZ0%2YAZqIVu!dl%p@0;{ zzi##Q4CRzb>}GCoNf}AV284>*;WRIaP-80gfhy5D;FV^A!OWWa(dm^UmOMl? z_;O52Rv&3sC&z9pSK} zt9cgzz5_l5iD*L{{ZW{IQ2u#P-D%Sq9(pH2pq;NKFB?e6J*9%tf^f11-X^Z;i&H`m zd8@6uv4@>PWGGA2AcBZwFh`06?Y0KUG)Wa;SS0TF%NwSyq)7twTOV=1`F;CnD4ORU zkEN9Je{>Wob7DEVxx1EESLs);FMpih9=+V%-LJy0FS(!IFj`t_=N)@_J9=t< z3$6p0Y2ul~1qrkf|H-Y;H#nqT4Y)y@YG&Iw|ZnYvdODEdUuFRwQ{?=wuno+ z)`{(cJiWae({I5y(+ju42Ck{q5-;SlMSrU{&YNxVsAI(J_QBPHs*>C*B>diS}*&_WD&)rS> z4Cbr29trJt`e5YTj4r)A*Jr-Ig7+&sT*VY@yXm~Sb%XPpdL*lt+y)TT_`sN2Dm;caIIHSOW*y7q}u zk?uqQdPUyII}@6^)0U~hzX*VMz>xoT&81j*;jk`eE+pae>@0k@d-{Ycy9 zvHjTgFA2MS{T^yO-eZoM9K8%|K{yj*6JuCHIx zrSgxG&pvMg0z+Q^xZ~DwwxnyR)juBu*p5sDbL9+V;`19ZHT~7@yp##G?ef`NXUV$R zHk zyYO}F$X3;w>$31R(}qu(%&z9yq1*0h#Neu{25o%PHEpqDS=))RLcO=xy952x@5^aP z-!J6N)wah_8)lkHpXV`ac@m_h3ZG2bbl6b~jo-LGn0e>Gh8aoqG;ObkLUwUbW^>b- z*{t%(aAKOk61sSGo&GZrF>K!Xt3M0)s-6yyw^LQ!iGSHCudGG6+|5?OSA{+kxQer> zHC=PB`B>B7xq&MtC9k*R+|-%gVLnqU{y1n?-&QTF3*vc|X+4NNP4lB@U6o6Q?4HMd zL+Y$#E+Ic)Jc>^4Rc*aOC3n5YdWo<7cxC>wF<)V=eCa}>py5mBIo)DDtq2i>J7Kj`kJwzQ3Wo3ET;)?U^LwX zbNe%R!n^BP2evqU)8W#qDxNsH^Y&m|CFP@x4)}g2v{417;DSsm5rlZz*ef^!@bq?RU`MFVee^+q{J=omR-2FZ9pzQ6_ z)`aJ}(%;)F>cfRkpYm$B=%osBR=RlYKmwUsDOY$2-TD~O;(RH7h8z0wWy8GdtecCq zaCvFx?%JXr?bNeqN47$uZfOna*ehS1o@>A-8mD z(%9OWQ)6viHTD_3eWQvM;MSc>eK@jjUCHmVx)Xh@%MCC8BA4uTXinmP@|RtwERS|- z=@izD1*RuI_V{HqotEk(@19Xh<>VjV*y9F%)I>F@DswY>dUd5^?Kf@rmahc9!Q5Hz z>iS-zHPo_{X#0F&Wj&(#rG?Kw0p^w8_vfD2@eDfmIAKnM&THthl;B4q##0Bvu!ltW z6rKR9>dW9;n({Ubc}_1q9lMxL&cwxyOO?h&`T0v!f!~B&yqEb~iYqdvG+BZt%M?3M z3KOl<4?>HFI$a=b{%9TsTkvnRM*}k^?ufACvYjBg_;7Stn-$f$Bz-(W7=(YALYU6K z#o%DOXpe;a*cCC;CsDfH>kYTcRvI?@D4en3EcxSy*+Oyw8uRf3a!$YMBRE>UP% zyusy=1B>DC=W(H+BMCBU%=G1Yw-OD$qZp`EM<0PgwFz(Ybv#|F8LK@sHm1hSVZ$zt zfEb$s)p&^~yL+hWegn55cN21!L4dLAuv0>HaOm>MfqMtIdnaZC9Dzs-_?TEV`0SU)M)r#A4LM)c}s4rnN3bu)v5bh ze|>(sTonmVcR}ITM@0WU+x-iRRdsZ=VqPFIQ=iPdsrbf~{Rq}TIq5cV#&B36*@op8 zA>4`KElLq5>)bx1qQ`gp@7Ej6?53zg zMVM4%_gTJM=i&^RPKo4<1cnK?gZAEl4u2akVr#jP4!%9l+3_I4$(t5+lIGvL&ro0t zDSH`7z*}R8YUynt_mJI=8u!eROF~M|Fsz(hXbiz(8Q1=mbRi)&nuaZSij~jzxFMCL znLC`*v}VUE{~ehY#*;?!-(QOVQ3y^qVm&7V$3LIJ%4p8Z!QU$Q9(0;Bw9)^Ld*I(H zQ~&HDKINkIi~+;N=%9j^_E55*T@8qia_FK*-Rk4D4`2$R7R8?^X`s$f=xfBUOQOs zm2ej62M+Lv5jUqw1r~1DH-2=>5UgP-lp#41h2GTuzApq(wudyy zQgR!v_39fu&P|pI?vs)HAo9)SOinyu?dr%iVE3X2@c#!r@pHH{@q3MU!wcw+#7Q+` zf6FvgQ(YdgY8=xW;0yXqmO|b)^N>SGgG}m3PW2}|6HGbILJHs)A(g0!tU?VCuv9%i zKsQ-a@41r&5Ji;EzdhMVsI+{izr$^*h0*v(Lnt60{VboL@k3Eb7dV=QRR~8DfD%Cj z>F3(dglPa1x)s;QhVcKg1%uK{iBr=Y;b|*QUKLr&Vb)yA>TyX+;1b4aMUfD{=3G#8 zoWT~tlyev6xT~jC3!+SeQ$r!y1g-TS+mKT&52@^*3h?=1?+m6-rwB`(w^>S3Gup~o zG?IAyt0MaTCGt1EFJc~5FdijhM?YTK?5-wiF;IqA!3N-?n-Q!VJ&?pu@}ne9L!cwA z+@E@v5<)ki|E2~&=?p?JZh_AOP+Tl^|C1M@BWkxwQOvf*?G*c#@X9keutw@>iTc1T zq>Zy!7zjawW}_^;N5-b4ft#u?{u2tF&T)G^#-zXflhJW8ehwF!ZdnKKDpH+eNMZdm zvdShrt|%svI%fW7Pc-|BFO2JzI59#gwY-rz3%MB@utL2{ggzxKy&~*y4b=hs!?-$- z(f9^H^inNGL7N?t{Q99*7YHVzjeoATUp{ic?LKz;4rsj#LO~qSQh2WwU4atp=x3D8Lg6$2riR7K2fP%5>FI8Z;oSxa)i#4AB1>XX+HR}eT?~m(tQnw1 zx%5d$+d~pBe6-)cz0|F7j};PPfsP*y1p&P!;RT2rl$$6)U?<9(MIey8#@|Y%zSqBR z|4M*M{n#l7GEm%d$RnteWHSJ|1W?o|fU)GgSjinNC=*O}9aeA`Fld+;=hatAj2NS5 z1dP^l{D&O?xv(aYF)aRafxzAt>CGX^w~50kIs{z0!u*y&h}e}l#@VOk2#OhWf`Wz~ zBcq^{2>%=G2qz% z4Kstpl%Ph5&zBI)H3i&LJoMkE285Q#R*hA#opDUv84@rVMfs1yv7ixAZ>YOX#TgMW z9mW(Zx!fr-%8L^~KmWyGijyBE0)BQ9ScvOIO(u_9&vyq($IbnB5R?*d@jK-v+icY@ z1G||TDmsTg{~%#wwvabeWnxED z4@?c0YP3kM$%jbJ=sV|%P%v*;G1sJ0GxyotgXrsDUPYbMYgD)M!K%V()Mkm)5BSzC zETeOppy5f$I*!r+alNjrkejzV3(^3Yjh9gah2t$|gA5Q?H^`cAWR#!tw>Up{eV||g zX0-&Naq>zs$fZdz9PjaOF*ye#GD=FNAvU&`nJlE>X;MJjh-2aH>XA>u(F*qyN$r-5 z+kR%1Pca*YZjnI~ZGq`%u~Yh&FzU0Xb)gy~RFZh>i74`7AlQtwd^Xnrqm>tr zgb{_AENyp5CTWKz5ij9%jV9s7fs~~Z%O?4Onv04qBn0mTQ*k_0AA{bX=j!v13oMOI zSv4>h1$Ith=2s* zDNH1j{D9?X5GI0-S@})vk;B9&EE^+gZ#d~l5oVT3rs*=Fryb8XzQl`3S0fFFUF0Oy4%{mcdO#0UjgeOIK(Z8Ngldu+tmmZ-sgojcr0+hu|5AcUIZ>YNVxaVUid_;DzQ z2sp7Lseca9(L%|TB8*q~UtD_S@~xXjVdxko>Age4Y9Fuz5H_wtL_lI zGaBw-(aoGeb>a|za6~d*VMuQv%*y76`vu|(?D^=P?w=D&US|V(ul4dkpGb|%)-kSo z!u>N(a#?n-#*0>nP_=e-8hWFY`{U+YM;#JfeXsMQGGzDJ?~{rlzG{9;)&<$9FL!N! zPk3KSyh|P%FP8)dC#55%Q`o^xrF-eGxlgox{HLCXvw}Y_I+@k{oCoLod+ev(k6WH^ zkxx;y;g19WP|pbf!1urFrTri6_Fp@# zvyqdt!+&?vR&%dA9E>&GKB8u|Ga@buPl^H6bDjV)bOJU)J^hg&C84-RvV7{zU5H?- z`y4rsagi>rsM?}trvCnz^>z3xhmDrDS)WC0Cw>=7peOwjL6yj$K!Hs7I?T;7)6o2W zcCO`SwmuJ5Hs;w-YK|nl7MaY(={qlL832}6O2$!m%d!Zc8?&!4#9UZuQ5p1_^p?O; zkr_J``xd>)UFH9At0xr-k_ZGNl;VL;#->ll0!r;Dpt@7LUy^B+ZO_ihz zwgozYH;__OCx;I1y~9H8)kFtThS@^^2_eY7)S@T@`$~U?gg|4lft$@YT>kxfq}?k4 zWf=7$fs_-4Fe9r1hfm*ptIJybS1`zIxgeX8fmwbj{{!F(nAh(O@KX_15ZXi^ zbuq3DtaGxSzL_Y4nzAPM9)*zhy)!hTgIJXHuOEp5*^dJAfG{lx%rnAjh%mKUQkG6{Y1R^ z7PH#-l*kw56^!1Q+aOB9O-=@(fvQJ`HOKl_LQ&SIKsQNQr>&_WYhV1xuZgg0t~MHX zZwSI0*{DMR;cObu6{%=rb=b6zp4*HT87reMe-5hY{GhwZ$!Smo&t1-}*NJe34c#cw z`Up8v*RgI*fFr=?cjo%IBotQ4?uLOzYUcanxa3%drf!O~m5>2!pBm{+v$(QJk7%}A$% z4-ynxL7K*v; z2&3;|nh^KSY@QMu)uGUHS17z(7hcQ?ll%0H;7~!ONFqLxh_uS#bjly}g=cND9h@Da zbW%Ql7hOn#y|l>~w}(GtqKmM*fuFV^xxMeT9R95;oYT3XV@F?yfa1>_gU@Bq4EeyV zS6~t2N6aT(lRYrosbjnM4EqT9>WI?8OE3S-5{QW4;z<@|l4OaD2Y~R6bDm!DUal_X@SxEg=UQa>FIvUlSuRgUOd4}K&6xmgHny!dw(VqM+qRvFZB1<3wr$(CZD)3NZ??X>-L877tN%dPsq-9+ z^Pu1^iZ#yLyZiC$>+3V{b*x)-))MSkH17AjW;TiyKAgtFS-GE6a@{P8=eok}(;aq@ z>Nzm{+qJ$y>^vuHbG)bD_cMfu)3f?*r(bQSn|#cF1{Wz9nab>949ey%eZvG~d_o3X z{`holg}1~NLmGb1P6I`aRp*;J6g~=$mRqw8GTFcF!O5R zZtIVHo42Q2ubWq>PhcaFdfA|!%T_i2;@4&4At`sZ%F-!DeOx^6Q=M2D&%UX#Wz`Cs zwydg!?-5nLeH6l05d*a3GPrn7uPFG0O*vHSAIW!D=Ei~=Puq!~P7EA8lx}#o7I9A# zXXuo$lyv8PWY;uRjTaeo)i$wA{ywgr;{izHwQKsNARhhcRbZ=Vq+>vQsW+}}CSmDi zf9Qs7#eYh%F7G^-E(EaAu#k1MD5urT;XZ!${LEB2Q@&EV&yj0?AzI=zE@?_~OvJ6a znEjEJ;B3-XmN{)Q>=X6LvlU4%;S~wVje;ydfM(=Nml6zyYO4xgaTx1 zDwWs=+aV~@(EwWvw2`_NfwGx(=gbs_>xpybaE}{q*P{#}TH{aUtFNU~KBd}uthuo+ zn|#PgJ#l=ew7tjK70m@6;~0I#s9LiZekL^?;iuNpFEt}4aLihQRhZ5*-5CPbzX)?l z?>e$mRL0Y2pG^?i!#yqf)k78JK^q3MDK(kzIc5}?*eW*KqJe}bpDZ|x%`mHtnUl|T zJUWH@dgHy)d}c$o5CXK(uojCr`TZtzmfPuqmtS((3~<@$m!>5_WsxO;ribZpf?nD; zK38w*mP+SQ*pZivjP|gJuQ$HrWeffwVeM_r`z;O)AI|&Kn3T3&ywEaUL#(AoE^DF$ zKJ&a1*YfS5;GEjI>SQkyv}Mxv3(INqqHHY|;Uikj6t5eb>PuR;TqO<3)z9WGwu&mY z4Mt`~c@JnNPxW(+;-4;;k8`&%WMC9Z`g-HllG`_-=5lb*ZE%cd6{oRCSEcN#z|O3}<-qQF6n zC{PL}X~|WU!XBGjyf3Vm&Si1|$&K5!cqycWsnVX4iz&=usIGJqFwtp-iUbU4hf%pX zzbZeFn;vR0>(cdB6Q!J-!6sXH)9!~usG@Or9!X<4M!ZKKaEVJ@-uYAt}QOq=GGsDXGP zBLv+Df*p5vjf2+pTejqgFPgWEv3?5Mg2ECEPrJ7%xI8wAj8+m~zb!~+B!?1P!&Pr6 zEs|BsNHx3Ob8_dHdn4Lz0?NBF5SnF;Te##kUb?$7lr|RBG8)*lE04ky(8Vj!ImDKM zk<^B%u@q|onR@ZZB}()-$Bn9$SFp`r7JRo7#U;cpJU|%3mo4g(k(npYdFyMVKx#MZ z0)`YBhNkhwt8ok3fwX?k;(P=wR7=0m~U3S zldfR|6B?Tuzn=%o96w+-65kcm7?U@=;JaRnE;YOfB)r4WM=^WghoJFkX)0{sUvMnf zR}=s&VaA2Lb0O;&g{|J%^xy`oNG>(b=P2?pWom`9I=!2Fs~-{;F~GHUaP_I9rz3R{ zP01~yuF_hQBq7BoM&+AJkRCU4xo01FUqpzfQ1uPv7srY0J|(3VPQI$YVlLM4yE;8P zG#fU7b(K$eL!Oi7TUi9UZ@8ChWV`MEB-p9ZYw*lW%YN2zxtv+lOa zeNNZ4D_f@ms-Im~`#tAFb%!<-=IL`AR|9KOXaxZv)alA}_&CJ|( zYQ(V6jwW*omlP;`<&6=q0cR{TwY;xr^G`i0&t_sB&BnB08M7K8pUTL zmV*qHry()?9Ri$}VeYc< z{K}Ss?2LE+8IB-09H0M3M7~Z3zBK7zApANEk}HY|(oN&y*rxHw%?BGVJb)h%z)Aj` zHkpe$J*m(WqyQd<%S)Th7fIZa5nu!Y0~Of0kmB`t`5kEz=L_is)0eei(6!llE5Iad zGX=(th~o8-vkEDx4yF~Jl-^h23aJ)Jgk`9%WJAPw+}Q^LO->XdLVZV71okPcuv499 z&EH#i`fBRSU=gY+y1^dwk(`M9m`1lo+XAKS43?6~>$x22+SEW3bN3BkgOf_c-omZYA0k2 z)i)IF{(GcxWFX9Rzb-3reLG80Q`+`yrz`XsqL+FP1OB!-aER{fu)Ft#<^+wrq4 z0X$TppSapEq)zI)EhyHT%BQg>OR@09K}^YYQ3_;u>}c!z{vH@34F|oYP+{H`?LyD& z(M!hfQbC%;2*~sm4v~3-N+>p9)m~081DUj`y7A6H;$N!n6-$q z5?Yb1h(-)AFOMJH^`XejWNcXVpGlN(#p*!kk01QtAl;|JrzcTBP^tEj%+CyrLyu^s z)6Zxg_8LRr=tl-j|FWo&9R!2-LgEyZ*KFV*(|8tb9KN;E_S3G`2{ZW$=KV9w6qY)| zj`L|M^iS@vyYl15!NSWA-zzSDa5`XPo(`(jkKruPQBg_{&XHP4xLRHx`VHW|-o2-Y zEEZFm2hV;6fQ`GK#nM+f_97J;^$IU=6N_Ku--b?lVTE=}X`7K?WG*?ukKu|7a40I? zH8!lOCg(PJquX{A{6$knf!MnD!)VpS&(7%^ewO=dtg^I-n1Y>*R(Jp=N;P?i$S1{` zY~tU)jJzXlI6TuP8?O5r`uMvaG~eI&!>y25H)~z`w}jNTv}lbxON_`Wx|ruK?^(M@ ztFrsKk9rYgm_1JM9XKVPYw|qJ=@Q@NU`Z{l)UAJEDp&rZ6LU0V{0T=M+5FGDjqZry z+jj8{%}IEW*&o%OZ!R6^djuTd;=Pi`Pu4AFqcKjs8FYpY(_v&Pc!vY71s7##g0lcs zRTD|AG&2+N3_;SvFCj3sfyQ3Rc`lG$2ijgf35=BObVgBs*+p9y=cvoSKtT40BYw83 zJC~0L4&lgq?(=^^{*&Bt5S#JPfd&GiWd{P{{{NC&&K9P&|4VFXTiX|LVtnuP24=5F zEmp??mAf?&h|yHDF2tx+UZ>ZR8OtQ|&`)8F6&C*7TKB0;INXw9yNJe9JH%Xj-SD+| z9GyJw2G;(y4fa6r{mTXw6!yj)Q;YecjWRsR*a3g|Hh8+Hv3h)Sa&nnV*ATy|lY2Gq zck!@3HduetC(QGAK{mc#>w?%o6yxlJQl`&E5jUN^nSELJ&VF1}F@pB(dX=I7G!v1a z1{a%u;)?Jhb&Gf~E4#>Z$A^j7_DylA-~Mms$3V=-P|Xh#xiNU>$FM|64YitzAz9Sa zEw>R_Z$x&qU{+6W3wBiIts=h|hN-f?E?!wM$`+4OWN8L=KYi?aoR#iZoTAquwDz!w zVsFug$-vdY165g5g6195WJ71t{b!J6@{SLh5LmS0ufL@7SA-R~n_`}81hz&hnOSuU z9^J8p77QJ;{Jsr-I;q3^rSkpEA6^QV%!~zSRS3B^ZE71={5v~4AVJV;u--qEX#i42 zwfdc-`4L>Z9-~YX(5SaOXVs9{E5R%K)KB~teX)6qw(DE>HjkSmZ#3JnPL)aQuKk#c zNI^ZW;-juo7_WN~kecIKCJcIwZq zazzvGcwNGTjj?A(#kpT>XF9CXX3ev}`%7+c6yoxTr4zhK%QR@;zla;{hHdlhwT;!T z5z&nLcCd;NhX%Ve+D5WrjGv}#*aBp@FOcS%az$y4a#GI8cf^qMOv%>;F=bQ@eUiR> z3G83^TzkW9$^(-eXZ8++PwklLqz7TK8S~PK+uXvzzV=dgEcNdv3>TaSb9Wn!w~(0cgr9F zzXsl9!48q%FrZrm?i}GlGlRh~1hV{rDf0V9&=|y+R$m@^I4K2>IG!f=69kK~OBL5g zG))$uapJuKMU#Q#3aJC);%T9xIJ7Z29ZPt{M9e7i#VhzXhJYO!I1Laqu^B_?VS|O2 z!GwrlK~Cb9{lmplK}0fYL4(QW@^rBKc)Q%y$}l2iQSmo%P>UKc4K4OV?lBgJ?&BsT zh-J`6g_mSd{`)#;ykr=-q|c-@Zx)Sml<-0pSuh5RWycXM+7i2$nEdy2ODbx5S2*bc zrMjqshknSygudt^VtUBb;Sw&X{x{$MMiwS6x(ru&uGzGDBZXyr=;@*<$274f+mz(b z^NQ$tn^w0Q6)}qKt?r+2u#RHW0b}PA@bq@%W#rg}^wc6FP2D{hcok=mQ8t`^0(@(k zVuyfb+st@qP8<7c#pjF0t7l8E(==+uT!rva@d$}k%;H$Lc;(?U)C%LoQy9fI2BBO9 zTs4(;#tc?xpXvE&hAH%$-;_bY9}mx_2c^R<&<(~(jKpMmAyiGj0}A?2)m z=uuE&1%N95KHGf5<6D{A!{n3Kv^u+x7id{Q#jTDjWu@v{`6jN|(K;5zcvxG46!S``qRL{A4Y``a*L8~vwi4SgRD zzCd>(6f>y&<=XfX@<8qdiF9b+-R>oH`-7dIe=Q5+LL41J3ZFsUtTy3Z8?_LHQ z#^UEWo!%TeE90|zjIU8+gp!Mxyg7gU!Ipn>*Y=bmJm6G*0~=&67q)S02r+U^;w_Ed zPn6P}(vbiT#ukr)`qMwW%~yEGzDj02N0-})eXGv^~{bclO_Wr1rZCPlim2!Cmub#I>@edlaM{C zzn~2uYk@F&4UV&0aO1rm^ZPCCWC0P`lKz`C{$wGMeeC*nZ!WbH&FLhbxIfFIM1pZ% z8#`F+sCYaBDZEE$9R&ng`nm2gzsPL+N zC;LFzw|4ywhx{XWrW_&Um$*woZn;jT5LV}|nmVT6hR;9kj?_`Jn^rrfthRw$%cddw z@{R6v#-sGWpoi1Co!f^kd@1|ohX_0uA7cAxkMYvt2nDG8beW68)eU^_OEs6@VdrO? zmf+tACSL-Mh?mM6;qF*3UpEvFm*3bPx43HJDd(f9s4S3{m&TTpBn`i=Q#%t((`gYp zUp|j5Nbaupb?RSoyR6(#!gOG-Ij?5QVo=E?VsvLPxaQXStSik_Nptk@+crfJ@#&%Q%%x& z@N`gZ?0m>=}<5ZWoZ>RqiHwGF#-u=&Ufc*Ku=+D20@EPv8+3{_KmqncVCa~|7 z_Uk$q-17zXC;hHG($L>&5i@`;F@*gq-3T@)qu29|{`UA+3Bcp74le2OdHd_{`O`4- z)IZzl&Fj(h>#x7#uSNELKa06SvtQo~4Mlpj(6T(|6zuOTc?4gK-{U!36uD(bn5&XI(-7CX6b&8y zMmaqqiMxDsKWz6=BXJ?AcL>_#yQh-J+ePSh+J|Do?&q}T%`OiBNf^~dF$1-AFbCb( z#kQh^BIFEtU_6!avU!XPq%u;v*n*>L+V^Ec5iZ}uNC)-U-hT?~Hp^bHkwR-I5}ARA$= zzA3G1hq9J3`DRhSve-*q+j#>VR7?%}^k#+HVM$qAE>uf3>)4a#nF2-8S1Zn@k8f&$ zM*w(q+Xqr93I39&v)g>t2%l<3=W@Y}g>=~bMUZLse0sRryC`d{Srmj-SU_D3Vl}C> z=mv-Yx|R5)VKfP%E)I?}mTzk{PLl`?xo+&i4)wH;?S!g38otaD%yP?%3QO=byG1k& zaVK0~H{<1*fV#1pgi&+X&x^t*Qm-r*_P7Nyh|O!Rn|*feN+-HWeWX#zDkhY$`$+uOsOwR4kaU zD*G-LxV9pyVCCldTW1HN*c`Bs-C-$GrB=!`s<~pJutl#aU-k-+sCnN9$#lx$!;&Z( zSKQ`S-({rJ(Z50v3V-MyTOo6XH?pFPp>1G!z-wjtm)du4$eB09k1l)sk+#NQoz|ru zRia)i+s{NPLvAefPwybKVQtmOA<_3(J#S!Yr==ac(bsryvS^2Oql9z+FDeBxsloDo zL=0~+XVA8V6lAUphFfE~Zj0^&ziwKM6vtnJ6RLu}PgJ#9eGmCIFAL;s>-Sa@o?tyj z^RUgfA}h6x<|nW|VkoJGsT)g`eSxwoIS^gpB%V8{MJ3wYk{0cPl9o#M>P1yZJ(DAL zqZk6Ddw1b*aVD<8JP0t|#%B||s=FZxHK4Lm)YfAhxq z6-^=Vx@+A?zs@Xmq|_{7_HcerP7P`6f%u#uI#3qv{F7Mv06#Zg3zRhsJuMyixN4Af zS;bhDLJ#)A3s}wm1W}rr=(E`PP9*os(Lt8m#9iU0v zuZ2(+vCXfm@1bkIvH2E9II?I5UE{5SSmLTpo?l-@7>a8kI}KB~BM2PfQD#Gnv@Wp7 z)lW@gvKnOo2g!c0f&*@=<%AYC(EYC{Ab99s>eD1uji;1JOxL$f*R5=aNh=H+2Rt3x zy`BEu{*II*M8QRsrTlhnxbN~Wu($rDW|-pOp|)A6l%*tui(sAR4S3lcC%S}8e@K_z z%IK}Zo@x0)i4D6WD_+?a$jaGPh=IDzr4e1AbK2;#i2z>ijkrOpUAR08*2j^cZwLzxF?33||5k$$yf zKkA1@XMML7Y60BUNi1ED=B-W}3LJ47OW(tH7FBcbtz`5oLz?`VNw=d?u}&87wNBrL zqcn%fXnR@>ATtyo4LXFxpDbinDY@2MEu|7P4Aw`VUL()ShnE{Cd#>t*(YwlOyevB#BP-5UUH`@Pe9>mWlDzUO1j-zF@!0rLj)oJ%0ybt zXW8QRT=(V?UaUHnLPI7e4-ReGv3`_)%_VFwuTEIkbT|=1aZv}ZNQdn*o@8IuD~O$)zVefeK-VFx9HE#K*qobjzju<9WtGv!;Jw#g zQloYKpEqBW8)C9}l-}-qLU-R|NH6907c+|Y>0f_){eB+(Bun-_?f=g1DnZZ1;eCD+ z?23@pD7^oc`n=rzd2p5ep5%JZhoi#nv4**q9*j9wIM!taSOR)?sr8T|6HdHnbF5xk zIVR&wl_O9WbUvUB&eris7b+^DE{{61RZuBA_jJ`rN2Q0Ci+ptNqxQ1U+yk#E)+SQ# z(6nP3*Ed=@M&aQmIVo*LSwsd|1AVfnAVhR<`xRfF=|-WSSQ`x>#kLYP9+DGvQk>k~ zzP5GQr%_*gx2(zf-&Sep$#mmau$@P4LH;E~ey0A_4iPwGXk7;^-hVB8ISBsDHB7=1 zx`FeKGR(&8n5olptfiWb+BM>~TqK>0*gJLpp~E(_^gW>O#ZC#5+c|kbTSPuu_*$~{ z)ko%)WSC~yIzLnem`a-4x-o)UAdx05`8*IyS!U5uONF?lR}Mf zj}f&}+^zA{h;cBCB;+JU?qVI3;oi{hjI-N^XD2h+MO2Zq>yqMY3L$^r9P%7ki8Me# zYqZ@F%h4^iBHjo^k(Ug>v3ehUSU?``JGG$x(hrg$1E+8kXFB0v;f>Y4{OVSK+jKN;vk!h zvcE#(+?F2NQz!A-|5fj*b?p+6FhAysCJdJuXa<+Q_Yv%r5-gn$Ql5heyFjY-Z)k{w z1c1V>57<*D^QKebFAiwy?8NQ*7E)eJ6+gAHC*0Ju)irKO_PN1k(J zF5Ii-_D7BH?$J!;c!5<}&efTTZtt@`F*pcsrB6jl`qOZSmqLv!QA3=_c5k)|p3pnf zb1Ymv`GIbHe|FTq4B4m{o~1iIYA%*Koy_|ia?HX=gf(DNctakkc&^8cdw-2T$`>dq zy-lX8n(utS%m4KmHG@6zCncTbBLk$Ld0+$>tUoC&Z{By?1^adx@2ZHyd>7mRlo#Z< z4dXx@3>oU6DlgeH7L?SKTc%25+vQB^ImCf~Xb&Wl zvZVAl!FmyEqMyh&l;$3M`x8=i7>_t7#Pgx5c?{0PT055{*u0R?nXcvp?+4ls#pfTH zvucW^$4b0UHG^~PjEz(l6rn-KgrYkIKbFgQQYQ8Q>uB>3bYUr%a76W~=}IG#cN9j< z$R->@vGXNbG@hTsS`!erC*V(4$=Wdj3Yu7N--2*huv9KBa$zi@!JAKvI4$U#SMpHn zaa{oio0du?3%2YZ7M1hbK?7*6VaN#Ul!F?dJ2h;C9xC&xQe~48#$AJzpfx?D-eS*t zdS8t~7!ZG|c<6E;Qmr=?y(a7$No`vjR}0pE^Ntd(ID969!#_uvD(ld)PVidIPi?sK z!9Ze6{@jy^^>1w(#wmC`P`D_63MeOVAxdlKk0LH5 zgQI>uSn%JwAGA^XNFK*EGA4|>nvPO=@i|U1*h6pnMD~xJz9hMuDC;?Wyd8hsyNY)^ zf%A0IO*a({9VFCuSJ^9=Ne0P%-8S`q&9?sSlInDq+Zl3tg3>>mUliGQg91II*Z1`y z``d=%?CQeR-9gWc-9O_Ls8ON)6Jyj+@arS$ut1b_>{q5}?j5 zYG52!CaI7)IY~oC4ctGUe}3-A_uFZgPaQkr#?_D3_rE{OEoe8$yC^LGa@Uo{)8eg7 zzUi`+st1!wbENS8GW?=fzL>1Voq)7Ovn@&JMgyUkO35f??d>`6E865G zG5ornU_X)iG2P~m_DG359@_(WCLk=dsxwojc@R?eg@5={C&Hg^+G9-+V!c^qn&WsD z2@4$U8nS9qqZas;NwrOCQs_C8$ zBs*IK!n{wq+>0D{G8F;8wZBiY2&}1y$NLnzbryjhas&MJiUG2d!i1qPR}!o#`n}P8 z#N@fqF_C^fRQ~Da0V@yv^U!S~@o+{}t*j9AxN_>GYUT_>f&9jK!UaE8-dxKR&;^@= zi8$TNz!8Y2}1lhZ(%xCu_EDE5^-R zf=NWTN%oMApKR{gElg1-1##Hs_I6C-XhRP9?8x zzO*)MiCE@ng=)#1O~oNkYvW%(vV!NEkioY?A+8IGH-q~{$kb(0>DvnFoM|yxB9s#K zb&^pMhH5WYtJEo7!j)x^)%vgE;ia_~-F`7qmHFLeScuS~m}dPtA!*ZVTe>2GOdum) zW2{U5g`oJ~mRDNop}WJoqpU}I`Np)fLpMvtehRMjp;&ExoQk$!AW7m@u$ z(>QbZOM+exH8>YGYXyabhG+NpRBq*q{5$7;DqcgdistSoQ$8u^2A7RfRG)#oO=izLEj`E%YSSL)GKw5e7_VoA=Yy2je&0#CVa$e z$e`=l6PJ%nMf7m2m3Yh!$>_1;lPK;o{F>!49oa0+w6J*5xcQuf z>|IhIeG5CZn-)rn-ks|80M${CBz2dZoeTX+`nyD6OwwGLz1J{Vg~jThcnhz0(HxHr zr#Bz$Pk!p*n>IA0w`sw)@p0u1n6+uHs9e0Em>~l*ZOfWcDCu~noT;o$<84mJ?q>-F zc8)1BOikb{$tX_`7ye&-I#!qqG)HXZ;n|hROya- zm+_cd8AXtLiy+4=LQGS`kv`G8m3OB%5={Da#y+gCV|cN7274p~G72gav-}s`!ItDM zzCL)uSMA|WlC4hRb?@Kn9-U@or``8Rte4#t<<_jyi*z|9pZ6uHLM9?`5lD?GEB++` zluo_zwm`Ju%O%ZrE{vvD0MO_&%5~U9Pt!QOzD2xgF`M52#bRvc_68__h?u8&i&{E% zHJ4++kN{h2<4uDdPGmdspXxd1DBE6J{7|W6q$wC+2X2gLE6^Rn=aE27@#qNE;dSa3 zERIdcq>QM6^8wV*pM*7tK=G+-;3cj08DM)v+|3RTfjWFEr*HnPLWLREPLwMe>Lud~ z@7k4p)4n=vZ?=Yy25kaK;qU`IV?#Q?yqU(1hSD02{qiNf-RSnFoond1EtbpS-cJW# z5N1?!1q|c)e%4_pBXYd3xis}e6)ug9#ozd|4C-IO z?5-vC#ZlWvT<{yHqoWs<;S|k5!B{rBi?25ZhY@VMN)jwuR@bI|(y-0e(V0-(m$4L+ zP)?yd?NXw9z#N`-hy&ak8ewVoZCsXI{6f`je?XhWI44$1QuLKE$A^Cjq3C$XLjlW- zAxqgEA?F8&W95;K1>Lz4&+OyOZP9h8?);hSP(1bLZ$^IJUdn!U7btHAY3xKZf4no$ zav9q&G7-hi>tX%}@#tT(pnIwjQ>OVeE;9g@TJZ=U{#fu;Au`l}?tjSGj=EAQ8@7bW zLoNM1uRt!e6L9W)Hk(!Cp3%sj*^`gmVX85$mx;Ow>x+CXruAi~NhI4gtinxx!6_gz zF=9nFtekueuZ?`WA?v$bzk1oskfv)IUO_ady%wx}mok&-o2_cLZMV3*UBZi#X+0UN z#!}qcH`>DJtm{S$o}ikksOCdTOJ=&?27ghfw_X_$s5pY;WZIt%{VY4^C@QWI4AR7; z1|zqW$%)IJ2s?CjSX%VA#nOg!*g2;KXXus0x1<{ttz~S5 zVRMMaY5b22{*=uaGm;M6X0?{nW!2Y6h3#iYRi*{*F~@e{e@XW)O0JV`5kUt1-dwcOjXnW1ifm^PK zllO9EmF#x&)-`&SR<=#os+H3hIl?Y6n8y%7P9zA-6=f{8mA0OXrz2gvm#QR@)$G%B zE7;4K*2-A>*4g++?&TSG3zk!+wOwe|v1c8vfZ8s*GBpf3K!bVcKv)X>ijW)NQ7M!M zj>ctEZIfom(MoY?cdV&)aX|3&BGIF9+)K7h`BoOJ61o;*$-oN2+7)(&gAH(K8<8Fq zm$!oH>Y^;#kHnAU-U#k2IKLwb6ZtM2y0L=vgcLh0B|JD|brT-SkDEC?z7SRR#}kW9 zrnP5oG{vy>fT~q-2nS+B#^Tof&Fr+<%OF#u$u`a|%(KhDUDoqhmfhN3_0`)~_jU@X zWbipNp>h*9jIF8dmfd4_i#CZM!KMfOIKO!%d+uREB8j! z3AUJ(-^POE#!Pcq?10E-h?h^fCVlN3I>HCTo@*_VV)acd;ryGc5J#+(*BhqN88fYcYkN{FxSFS#Vhgnia_3Hn$QcL5D^HFSuSPSi&m25a`Q;=lm#-svT0b5GlJA z%+zwPT9wGuP|+1FSCk~Sym4CB6*IISSp;0HLyR6-x1JG?Aqdl?HK-L}=6jpI_RYCf ze7ZGuOCEzlUeZ=f4IOrq%` z%F@!)a8GGw=84RviY#~0w8gB(%~A`vnj3p}GY!;BiK(rY7T+Cv@))M&*sEDx>uCHO z>Q`Lfd(j+qr#9H>fHgP|yz~+5@LV6E)=(VgVdXyS&0;O+S;*G33?8 z76TuI7|cY;mZ9utt1xM!t6@7Kxu=_v;%95EE+tLeZAF;a@6qTcpv!{x#b)ag zup;Uz4saFGV78rNOu$ViO)0Ym!#{CKHhrjMrmW29 zq{AggNO!DunESK9Ad##+nD**dOM7lXgahUuTNHA*p+eBF^rSp>{)}? zz>QvV6;wD9f{T}fxWtVj(-{xUjSY=rpPUign2^ZJE}FeCI?%TJg13q5y)FP0n28!l zO?52-p~0myl<8SAQ#7ca8G9Atp1C2}P6(~-O-$)|stalwzD2aOk4ooYI z>f9SeN&xCg5=vxJGI)m5s$e^g%kq2ym(AupY9))24BToH&FQ>|Pd^rNOo{4ZI}OMm z-jvl?c1^)utmU$(-<)8C>t(5NG4i#E&lHY2+*hkX&{pw7 zxkk87v4MrOLTm#0-9$jlOfXYzu4il60+)qNBSxmiN;nscNyiZY;5Kmfy^v1cC5E4*d8OV9!iy! zW6z%#9Mi0~F(UkY!4u$-ax)rkkfenS*E#F334c?S*IkNN?ImMxk#YfCx#x?;cr7ek zG!NtXD z$CIkUzlTMiH;4O`dZ{W8nE}sCXy6far)yE% z0FP`53iGed7pJCryT|H~PbqSB+#gB>Yd13N5BOA>r<-pY9OZH#1kkpd7>m~GvP)Qt zjvU@av)Ylp=I-w=qTRK8$_+Oy6oN@|nw(a(!$B*bL9YWgmO-x)-bFr@u@vk_eE0fC zB5bJ2#bxJHixmT7STCo}l5Qr!Lnua3=)OQ{k6D;14>=#@Yn#%P)`hMLRx`Zck>yvv z0Sv=eaDtw=O&|$STcE)}MWjZxlrn!462=oC4ymsD%wui1tGqZQcVQirRlp{I8s z*iPPU?Rf=h7Om=L(BTDG)Y!oeqetRSe0u{%q!QiSx;zc~dSM0&dr@8-x1rEq5#a8# zm7;~h&x9V=2DY<=ynFiwlklKh+kq7U2{eGk0O+14=)la_4xqvKN`47CYv*G7b033N zTPtT;50(x8Tf24)c9pkcegOy_^Qop=XB;?>c;ULVl}gr&+trfNxFIuIcAF*&XH+f| zrvzbM%gMI^!!!1{i{Qn>ziIC3zqm?=#Xe*dvd+zn;9y;y4L%6e9l1JU)P&t_p6gqF_@uFo`~<(M5k6Ubwz}z1o}D_(My_aX>(TEgs#Tymh)BWbb&f4mFQt?4XCk_YSkr8;;tkQ&8AR_i*ii{tqyFixBd!nYk?~r{1CRBx z1*wy5`gI}FMJ>CsCY1Z+TokmvoidM_g^dgA0=xuEv{e*pC;!;0L$TvDGrW^f2Pxv> zf(1Jf%4u`QU-AK&Tq^!#G;kNG)_ei=?4HrA;rz>ru-jdKU5hoPF+Bi#lDkY|3q>7A zXV&2^zQZsin&t|0_F8n>#Bcm%Pj%B@$Xx6JA3?+>XxruDE~d^XpH+uI1PhqQ9kj@Z ztHTc?UytW~Wo&=jKn=^)#+pY5*QR!`}w{aBrz0_pw%bF2y ziWX{RzxF@6gA*mO%4`>(uE+&;^cz{WRv>?}_O+(#Rg6A-GN`SH_rVNT0t$owYWUV1 zKUi6-Nt^rv^x&jmD4M=QLfwNab1WNEu+@#qMK}eW$4LpevzVON1FOgE3*Vc{_nV$) z9=vs1Bj6Z(wFAeQEV32hGaZhJvziFv&?lCTmj0zXd(NJ78tcVMmiywY_4c z${kis6p@8xtuXP%cf6V|{6a?LZ7yOg{zAwwBsZEjf}^U1uO_i?Ok*6iYjsekbRzml zz?!yn%5b3$sDHtHsGlithI41b85h@cNq?}JdQ$3orXZvfUA#%isaH8#G*fKYv$921 zyoF=EhC`$Hx@H`-gMeKEktu+Sn}vNk^rR~Pmq!{xV zQUvsB#b%cvS}-Nbb-O~rHsY-V@c;?J%2#-Ez5=V^ueDHEg7& zSs%v?@AJKb%J;PdzMK4>02WTa1P!I?ZaQU*EtJ2rtWy1!`DLgv8cY zM?YF?@Eh(thQx3{t9qx5grrdnREIq*==k^crEr**bwr&^aiCf-aL-9n^RGft4gXCM z8AeLBK!~?Ae;&T*OxU+Ny~R(7qZlBF+ZVH&IY^4m3^B?~N4gu(qR$2TJjA}810i2S z{{^+mcv!}PPy}Lzcmz`J>1T~Jr^F0wtG0{D9|uOBDD}LeD;iw``hg!9mQn7lelIum zXCpZWLyc10-|tR?pUN3WUp&pSO}-7SdhasiKU-Sq(fv$scxhTN@=^R$iM{c*rLymVf~HD8x-Yd#PGin#5|#Z^fcL zRn;tNEiarHMMR zvXqKsmo9mI24q2#mSkux%(Xmd?5vQ*Z9!D^_8@ggH0?w!WTa`xdYcYh$LfAN4SPp0 zKPc+ucx0yO850xaxYCg95Pl1fDZR`4LB+fR<3crDW^JY!d$^58ewpjALa|6}$ciZq zcmfmhnilvbt*Jt*K`FNd=Dv3&ttyR>>cO67{-*O0Ej(s50x)uGtXo4^FCl5*Lm`*k z6B?2!eD+M){voY60k}>IjZy1B85;LGP_ZAGau9z|(5CQ5c(6IcDIb>% zbXly|qn3kRua3KI=MKh=g!l{pFvBqJYAa5EHH0xWpnSVQ9Scag zS-I$8ezU`d^IAX?ztVUG=*ot0ioSJ6kHtbbqbbOZ|C&$29~Gz!_SWx;g9&D#0Q81S zp0w%cw17rnT^F8K%4bGpBADSg*XW^^CyPZ^L@+5k5A7jBp%appiSBaVYL`n;vST;if7D5{cFC&E-0 z*BMCi9C_OjTp&QrzUNO0s)S~(APTssr^gwbk><$y86n^{yB^^G38y-Y0IRJU-_vN2 zg=x|jx9P5bVOnyj!3jp=8}2oi5M+pvaTt>>SN&Uz_LKwviX#WT7$_d4hR6S zEWhyURj;z^-dcX%_o%UK@Y0H82Te&924F;vRi6(PB;RG5*X4CipPdtKIWr%qLKPIb0aahcg|!6Rh{! z@u^Oxezs6t^iMV+ALYH#OQb1bTIZm2pa(A7kxAxj_Z-@jO_IwtC3T5qThBh`EqceBb|#a--E{wKAH zr`thf>cL8vW3NNmLtZ;;TdBOGthz;Wt4J%RJdrtNhtOXdq99@{w5DCgZLS`SCz|3%n0L}vmm z-KJw(9ox3kv2EMV7u)XGHakhjwr!hVZ1eAX@92%*WY21yv+4}0>a1ORQxGGQcnVZR z<||}Ih+0-KkQc&`F(O)6P{Z=s!&m8s8`*^Oqe}}Dkw~KieU-;UwIj$cXY zfbyix3ej_Ii3Z6Bi7N-eHoZ)f!Qh%)QCmG>!x>0`zCs=`kiZwQ3e*(bKh6>uzZAD) za_%!yaNN~rym;`&_naRFqCq>J*;R0$(QZxo(G9TvD>+$p5~mm)94p@E!#=A_?e8gN zZN1>>>gN5wP}||4x#>LP!!mWO+6(Mz_3gu-G^Nzb{27ew`eK zXv?$`U1d?_yIA;grk*vYwDDO5(ks`uD0jjS=r56KmNQf79>~*foGL^X=snS3=`x8h zu;l_w9~sFZZiHczc$jNOihDh0)y3t0!0;v!v6(1Co5N5yqABOg{`%vFIK_h^{I{56 z$<4A#(X77?y#bZeaVZ`nHK;O^WKek^FzTr?pQF+lqiCKpvyo`|ArSnM*sJa) z>+uPcxk(R-q+hEieDA$msmi|eGlK(Ox0^Y!Kx8L&Ifn5bdyTbY4pHs>1`p5XpKBao zAHGht&Fy)MzJ>5a)KXnhaGhP^KnUBlssl_u^H*!G+Ys^vP1J9C+UiTAW|I>+k*5UB z>C2VOUhDPyx2i2um5PUTYY0eA%^|mAM9oGK&kS_=71}uaidb-BNVkfCbY+p*LY=J@ znO~ZHcAQ)m_g-v^-R?IUhT7A+V=AWNZ|cW&R81=oXxE~mcbym>WIQYJl8l1No8JlZ z(@5xk&4*e{j{FkQbFcI%(1&{36Jnxo);}ZTj3p4db7_6v`Nyyq(g~sUdrKJ-!59;$ z_T3#EzJIp3+nx&cbW%5GPu;l53VaQYJ*uP}bX8@~_qBjGR~>}#UIEz0dkl=wv{^42 zI5k0OZ+=3S&;m{DkPn0-!PKdpsoRW#&dQAmvssANWpg|1MJ-48`mj4hP>gND#N`d2v0$n39Vg+%mx ziKIhwG!Ge?pyZn~M$XzE>C%K&5$d)Yz}>|yU$tanG70_bJ52$fPIG{CE}ix-l9?2r z%2h@(MCwm!@>m&XYA1Akb7}lFswibqhBX_;A=%;!rah8Jj2&fEGbPj#pY<(>5?t3w zddCWhocWDjEl0Dyl>_-~z6VT$X2vd&oF0BSi_JZJ!93CZhKW6I168rV{xXq*mwoSM zMOgr9Kb3ucC!OT3c7XHz!$&ch5DXtPD|VSFhL5Lpx4K9lxNk*Jkky;?D;Df`Kee}Q zh08dBkr$g8oH>42Je$a-e0sFPKfjUj6W|re&bg{65eY#N!w`wEP+mN^epNw0lzwIY zKcM~=-ZPLAvyv%MbqP5YqYhmK^&75{?S8 zr(3&QRbIm-0fa(Lz>Sk5>)jFg9fhLcX3ftGSNfzdWk}j>4h=wbp5JgD} z!_&9Tr#cLj#j7IdAC!E+h58f!#MhxAFCvxw7G{tJwI2hyrEah95ZyZW9_`5tbm|VlPY?4f82;t z5U9z``Q%v>4s66p3G6>exno5C3Key+tw@fDW%^u*C^xN$Nz&J*l)^J6zG6QsBEt?! zKz}d$m&~32By;CK$=vyGGT|z&SCy$B^%DK3l`|AqSw2q?^Dkas?tmu&EuuBN}K!Eub;jpOh zZ>_PQpejicyIz*k(OSm}%4p2LWk6}no02kE0hKe2O!|`Q6rIvyCZA5k(m+vERN9!$ zjM7rmG&u&!qh2RjC69zuxM015k#a~RDZOD+WeUPyXe~Z5vfo_szNMROqZ6#RXqF#- z2V76F+*nRr#Gd0qJw6{^$+F}7eq}NBe>3mN$AHX^SdwozcleVed#LsIY<^}=Hn4!r z$Gz?cDDXEW)zieeuHj)Hm)m2G=7$8f>k6#S8AXm!Zc&2Z>*ESiteA@IC;6YbSwj|N zX2w-HBxL^7Q{Eq0;(XH)$fgX+<1i(wdJSSio89PTlz(}@TjfuJR&CQ-feKB4Ou975 z5-b5H`YfI~f25dQ^a(av$s>g0F`CeYo9Ynv<|i5c>?-HGO}dtUBs9*cl&Lko&+ENLYX2XTij z;IupcG20Z_&xGQ>7F`)W1|_!JfjKKd-y2GwAE*86O1qb^DFD4WK+yU}mrbD0BB=H% z=xaH|MGXYB&my?`Dmb`+HCT_s!Jgs4kR2bM-G2A)@BumWmRrZpv3CvvL@uq(;Q^Gg zs|^pXLschSq!Bz&IEtNtH4VOwy^nI*Tn(H$1vLm|B0uX~{CEJ|*?6DvSTdWjRz1=d@QDd%Y%msDa zaoAYHH~>TKRD7onWP|%n>j&DXyH{spA*++>&i2p4NwS5L?_WwWwK8~Xl>0lr@#irt zbXX*;jJn%HVBoC$R5wP?Ggkv<8}Ai}*|p<6>w%LLdm_8J=f`Coj%|mZj28oAL!;+< zk`@c(ht>?gmW$#QtvQbE=iDAN&DxDnmgO7CS{{L6PPb?mAs+V9k7i%*ZpZ*HU#>WUyX=)jHkH_vizDg zck=9v=a^0n3o&(Zsef`krD+`vow2^)z^}{CisbdUpF(-A>=dF7VIP-L1>hO}P|Or* zLwM4z&fjv~L@9m{6cNVnm*m#|F8b|RK}(*RO9sAzOPMw@%Q}|qx8ZL z=5~Os=!fVua)0hBh~1Kc*HFH%KV-qXC6xN!SAcf@D5j{%ofoKb@W_4x3Vxn)vVX}^ zoJ!3-MBKM+m&AtoiS5{Hy@rED8jmE%Ue)w#ZYJ4^c6DHh-jR&qx_{lh#(`t>6mI)K z)zmOBd@CN%N^jdhuD#f8P_ z{I@EQcrBUe5{f_UP&HQJ35&0Om~Fl74)jqqU=vZ{#LgIt33xW1dcXa+_`KEnnaf8Q zR^;_F6_y~({#bfO;-iW1C*#8wNotqZ3o=2-~I`ySGM#B1J@!uk`Ju9U{+tkF< ze-l!z)+hBoN3H3njxN6#9vjlVW92SD(W7SRn|~S}88dYZHjo~qrHhsb6U4C;`!^@+BWJX-qr88J(J6U2$2ffBI6boxV?_QmHWvTwvC<()eRnOA(TG(SB5>7=0a289@Jhn&!4+7Jpj$~lv|HnPBq(JKI*@O4>;!=V2g_Cj)+WEe(* zGIXeGqz_9&rZs*{aBgaH-*0_8hp6Z;Wx5}6J>V+XAn-_&NzA7L{Hb8+A?NN6W0o=c zoie{C`LhpkRQS$kGteb%FXRgV^gdXz{UY^%%o z=IqBkzF$;#qP<{oSryDD(CP$U$r$$roKLD5Ju^||pdV~(N0PYJ)zed!8B&P-1LEh$$m#UJp@fy^M5#rq zv+&QB^8)sfJ{yxPO zqjC^is76~*|7pUy976ryX8>?S3{cc)sCZ0yf0*_%fGc8Aw0s%uAa<6&exu_D-6QRZFhmwhJ7kRlWO=B#hlro_o-D^sJ|C zk(u<~G>Gd$H>^2xXy=~LPfh~_^fH>sT5PeVLb+vH@!_&9?vttELe zz#3IjsMC}Py+3GSEr?)K5Pa?bSUzm9jWL1ngv4#2Rmn=M36UVH)zAN=3IU zs&pjhCFP!BW4gzh0*}&F-LZ|KKO;6}9fD9TEHB0<@uP5KWR&ucZ^WVWoiGnnU&1lI zAj}ni`SP3q#w-q;_0*8#tgdyn5T!Ba1iWdQl5xDD?#*DPFx&Slv^X zp3Sej@?T~v4^u`Jb{aXp;mGAC zNeLC0Kl#sQ7q&@ee4)lp;1hlzg~efa3j$sU!M|m}cL=9Gko^2f{QP;G6)c(u=U^{o zGD~_>=}bP-)-wr*vKT8EPU#^@^4_s?&NkQ;m(tEhHZb0L_{8GzH#YOG&vW(jz%)M& z#D}@@k8%zU&?SE;-l1=RB7-a8Q7_&EPa*=ns5=#nG|LlsI|SOeeR?{UzA6d)X9f6bqz87~B|iy(YE&h~Iaj zm!pSYoEqg7F)_;jYJP#gn14hw{@KK1<72V-EFX(rTmCI2edY0zy2=>n(7m%+%^i0W zsT>_aBtoMRmPJ}65z8xZvbOOk)_J2jJr^!jw|6pTd_^o^q$*sd}rFUlJyoU{`UM#?+||L?0N3phvR|p%Lgd-X}$>y1)%i^i|0a; z#P{!#2|D+mX-8lEv_SDifBd+P3_Z|Tidb`=p9hmUo%HfonY%4A4qxuD8RnCBmu+Lx zbUNMtZXGJpd+lM33sJO{vtFOyvXlWl3bhY3OfnwL!zDXnSQzSytt_tf2EO zhS#R^mr&- zW6NqFpqg+YX)ns?^~EArMm?Mqm%WE?Qw{Ocw*kYtvV?1rmx|7F`N{1`e5WsAQa6Vjjw!M`ADAKuR!rS> zeC*+Q|_5Y9rObEG4HE!5rF z2aWPAe4t=Zw>f=%XpZT15_PF^XC|Z|K*^k}{s1V+j}E^@Lq+|ZP_yyy!P2mVt*I^U z*m+oZj^j%d#=t?;81zi$T^+Ob{wDZUA6O$T&QJCCP-j${aNsa$&5_zO#bZC(#v-=gjSYAnd zzdoNhuE#>BtM~7Wp$zO(zZCmC2wl&qZ({RMk5tD8KQ(b;t0r*5%fhR$mSLiAlK$*9 zBPg^nK?wTh97&gzkTJ9SxqkU~L45J&j@pnDp8cRx8Qwn0a_D;g!iF|b5#*ABs?BWq zkivfA>7li|I5{V^x@0gt&2aymeA@1&u(%$XIieYfc4MYRFxx~<8PbqCy~y;#eLHBw zrVND?6kI<~BAt%o$!+@WwfS85!u6ea1$B4X$%y%3zY8ftK5(w~ZzITb`X-y>l-XZ_ z)RmvFwS&>w`b>d*^1}7Obel3wR0h9VVC2`;Ij=)T9K+~@LUO@|jd$5QnFWKKI&~BS z@?6gj!Dia)uTH?;!)M(>iH4VF3lWJKg<)+?9JdK7*X3Rr@keFm__QS1Qju;B2J1|C zJQj9ySe|JY6nCA)E3x*CD!W>V+1=+F=$?gqs_fKvPddlSSE_7&p4jJW*w@niu6_Eq87m1d@&0cSKK$HM`LW*< z9Ve!Q(&UkR#^I?R=(hLUtAZ7)FS_R~i%_Lsk<(aMgj{*v1qmNdU+iKFcE2?k`F|pP z=T#lloQK}OH|)zB0mSBd<91dF6ta1GIEU#;++ZC)U67j)sTK+eWG{!6Xj0 zQ!E6SWP0#(Nkyj|a;!8UASk{cM!47_BiB{}Qqq3fY&os6 z`|lJA98bmMS04Usbs|`|kZgL7-%k>o`$L`z`)9zw!~D?^oq>{wsdW(Homw=E>A|1D z)HGc#`lfo?>u0Z>MFDZ^K@T1TQgxNj@+TH2z=5eTSv667lnkQ;NU2D5{(R)e( z50vupt6o?NXoDwAF4w>|2$K^v&vCW4hs)Bse9d z*^;OhaGTvvWVE@Hp4{N3J#rZeWV2i5ZM972V8=S`jgzXCWotd)m=V(>?l&Vs}I5%A&Uf$rK7{}Y) z>-x8RU-;%E@C!YgVxPsxk*O}C1FB()IlKqq-#bq^bUG-BlJ7j*K z5n0z-Gi+(+zPSe@8^J*w^S(BIT`f>7)EnM$lIUNUNuB}cZS!y$Yg4|E; zkKePqTPqY;1pfw;@c63XgA{8yq0ZZVPpnHUZn=3-f_q^Z_aB9TgVe8KgQ^<}b3n7e zM=>saNq7+s&!a}sOA9d2vupSBR4?NhOL%CBQl}Hy2=hV*fjD_Vze_erj2_ax)vNkJ zW6Z^eTxrtO6H0TCcNgm6*67F>TgJ%>S4uNHk*FJjYCzgYihreZH<$dJB-OpP(zjS# zar8w0Txq5##)bbbfY_nM$dLnM z_yH2ZyTulAR0o}dRlbiLKpt4bY9H*N&~IN>bKWXZlyDvf^x?j4UmNa2^}U(sKgrOv zZ+tPEBV-9k)8fvJg}#T1FvKATUUI?@*tfrtZ(RD@lwC+5v*xj0_22geTsvibK{;UE z$6evkDBt-+KC4<$O>11?31~I$PVWK8|5<4ZsNG~J_uNeH)iurYhD%`Q-h_Fnf0#Z{ z6=w38d6hvO(?~9}8f9pd-<@NI#+(@JW8QdwLGC?&*mGk6$c(_XV#7>VuHPfwQf8(1 z+vCM7VxBEB6!P-%pNTo|ACnSJD=S`?0!pxM3C(xyo;wzWZKiJ;JSo2pQbqD8C zamh2ZO;-OP5}6Y%QY(tK$41PqTh3SCA(%7v%V_*juxzhGy|7xzU4FB-3ndzL8;w;A z=4dnIoulM4u&IZXR#+|?3M2djMJlm?@5!Gdx#`H2;pjR)3*Ww?!);r}ar8lR-Ds$- zcQ0I3w#UDH!w;GhP8*0SSxMsd!$IIWJW9d%W)Hc7P3GF7$l85XZ#HKtM1w1|`fAh( zUk!3E0H{3$DF)AUF{pj0N*H-}F{zr1nZImY!{$CXO1)xwheZC76=);< z5Oic2`D4YTYwGSH zR>pc}Mn4C)r5>-E@K0nZ=)mp-3}mT*_~xe`Mpz9ENvcho9<=6RnhjW&=YkGy-+h4k zalp7ZD#N*6!%`3&PBXiGSZK$QvI!i_9JzV3Hb;Zp-^?fU)Z?st38gIOCLL#uD{lfS zj%;EIG8^Sx5m2jlRcSI+q)iq8KiZE?P*fYjIZ78|&Py;|xYfsg3aM}x*kQ1|heO4C zmKisE`%HMzNL)@4VkXu29T49bRNkbuM7Yw3B6=skVNCqt#cD}_9^dMVp$fzvd%h7m zRXfvg0>FHB&DyI)AsWu1VOt26x7qC_Amfn>4hh#czJUvQb9FZLJcfc!z$(-P{wjtJ z|2i4ZT*E%<)n_tR)8<0N(OmZk$MAl zeX-Qzi0y)ncb6w_0}d6mA`zDxU#U-2>BQ1#Vy2uOH9fb1(F*V19hx#Ra~-yM2Dc1HS?cbT70ta=&&E!+SuAH^TiLI1@oYE;It2J!w zsHMUB--3PXa2^YgMOx9sR;e5F^iriQatd=2z9!oj)4=W)O^@8>d3IYvS11=xj0I~} zfOaa>gzWK%J~Lv`ZGIKv+f}}iCzuZ;Ru|n_{1N1a*K2c{`=5Tohh`H9{1)z}oSy{i za7X$lNAr!07*+9^A+iiwk){!EkTCcfQ78(c!m#mFHVr6zIopx=4O^2@>^s`D+wJT} zSUUBr3Yi8l2F;EfX%A?c)6u!4;tXCQTDvR#Cw_h2)#-3JkSQpE3X9Vg;xE8281 zlr(<2Dwjf4-^_t%)*PJT;LPnAOF;}brrDitS?jU{nDWl11{O8mHn(2`?xUahk$;pAQ#r$!N zp<*s7JC3HR;zg^#D#|Mz@H2Vpt(oy4XeXcDGDm`iX(gf*dMyhjwGkO9M~ca) zob%AA7P*&i!~7GfUTn*fq+ zl&xE;wR}`a7Bd7-t@C&-V+l9@vap2ay{sw1Sfa$E z+O+G!v@%#o#E~{j?Qs5>Knp!EF zva%0ShE-Nso7NJOBEXM^C(%to%fw0=XbOv>7B3izG)KF12^%kD)Te|dgx4(YylgItj7J@2ST`eya>I@J=@e0Y z9n~DsI3&h{#@&p^p$LH%bdev;)fyVv@k(SOjy>X(0O2^dPN(b=&&y&HZC${v2Rcsq z6oo!17Zw8j!`vrK!t_M@M^j+X<#-0w2O>@s7)-v|=2ZYao?>{{rrcXAl@JI&d6VH^ zJ36k;s+9U03UXDp>C2y5n!ii}9l#8s4LcB`7`QA7gG6$z3v{NB(4%I+2u});kq~=_ zQAN>^6SV_Za26JR_x|%V;$L$ln~f_Egn`~?U@ohv6T?f_|4Xl%sOA+|SCMAr!g3a- zV?;|{BoK7~XJoaf)L{x~-<z z#S0;=M@esDpY>aEm%vdYT8yjyD*c7NTSZ+@4(o63{EL~m)kTK8`;Y~aT2e|S$C>;f z*_c(e!wA}8;LFSGoyPbd<=il|A_4q9o3i@SKFb_6?)nY{d|}#)slSvC^;AWEOZZ8Y z2}g=?a3#9X@q#@X>Ixh;DCCMam1;*@eax%{ZyD z+IHSGXEW%>s=8tAW&y~b4kyO=m0ubLbz`#39jztp2#lida6RE`uN*&0saS)INv}=O z#Ij2#qZ}udU;3upOa}!FEiW;KN!{R^E{Dl56kF>SWf9#yQ)eXOvQsUpL83OZGjSLi zbVQ1w<5DZB;(izik0v)PNEv{670XU5vtYSK9I^w1uM`!DGx+3|N-OVf235=K$3coC zqQLtGwE|v`;xd20l`bnp|K_p|QuO~zjGPWX{5K|6RE>Zh$4mHEsvvWihH2?!nHw5QnR{pC5o)TI&$HBvL5-aSmb8 z8HxdE&TPT0Br87lY@gGeV3_5zRV)M@UCdVqC93aTsj2++M}{Ffr5?OO9?jsQciVAf z?VsQV@O0L;IGLr2mGQAzsA>^<<3&Ji1TD3MfRkaT>7Z+$QnaLBa^rO#s^d=IUAH%8 z^E4J;KZY{%My)6Q!}$REgDNw8S>OF6s;Mg=U?BlkYAc?OuFnG z1l<+MKY66%51f#bFj8=n}>lZyD=_cml8Yy+O zxg0wqvxa4mL!OJ7HXoml6mQ1;d)80oVa}BazPq{lCz;K89R)@1WHf((>*sKR07a%S zA5%v|p+!DRsdC`NJssy&j2jLsoKD*!D2Lu`RxiDeA^~&+2gk-sVI&2|vM{v1I4-t; zB?-V2Ya+(N-6!l;LfMjit$0m5lW1;CDVh}%j=%waBk3#kF@X;q@+OuYpsGXUS2g1@ zfZuEN6CW`%+OYG1mpiF(19sX@{ppWH#+S1U)}1h}SvsK`5rPVD$1f8#7xWNecgdtT zt_Q1$M$K&o7A4#TV{tKxvTCi-Ebh|w6U&n3;#xL_OD8QOv{)|4OjJP6W>Ldpg+;rf7_`5Was9&BpE&2At5o>LgM3QT%5ya}7O|)&ePb_I#^tW3-6}734zM4W1>_gA zdm;`!W?GMs@6z?lb~Cg^t!KAEOz9rtnrk0eNoHX~E9(wDCDUi)Vt2~YlyArT0%oj) zg8E@h0*yksvnn;j@U{2EvnE3^Gz?&`%3%Xzu6Kh0KGHs(nPn%polRONH{WxOpO$3-B?F{- zxX_BcXG5@jqyAO!dRgD%svP$KQ1~6Gd0%0dDG{^KKwbrPho74*)RRl}U!Uud&u(#2 ztx@xU+gu&@PfzB8EZOAf^oh31P4w`N-f&MunF294+_wyTQzt7&sYt6VORdtl|ZDxwI;HVl>>&lG3 z*I6$Qh6I&Aw?AIwN2Ew9qvg29W#55U4^ML{cahY8C?q@i65`;fR*S(j5+(FY%zGkZ zd5gT9N!(#w{ii2ZcsyP^)m7FzK1wmi0^S+jN@5r)Se9LoOBh>tMcS9P?Di&El+j$V z$+D*AVhGF3XJsgUZ{f*f46_!b2UM7Uqmm&e2H`*G1#$u!%{a*A&sip3+Ghnc>6ur_bbN1L9grv^jr>AxHib zf(Hw#;Xq6EmAjgGy(GCMafGkz9gL?7UNLgv#Q!CiK1R=O4=Gh!y=ujq6Z;?^k$UgwjkY;8(*4-Jr61yiQ|}SfZP7^KQ3GdgGiX zhAF1QX>gza_hT@e&$ORF4U=<*r+%q+wLrM7uKr{8+02}y=@?%LtMxq+R*ss)d%h=N0znt&AALW zN`;=k?+sx62ra(i5UZr|FLBr2n>APX=Nf}J5k8AZSoJd^0m(O^qfgQu|CApscP4oG z2uR&sBf=w8o*Ak_*8-||&Sc1XM#KfD<-JR4pIdh1=em&Mdf%&&WKS*Dzb-*bpPQ`y zGPeI6aM4QwHyC3NLagQWwC*vz(nluAo$Q}^c1$0Ma|f_yxzh^z*rqbT)45CQ;}Ja) z@{41o`KS+iu~?+ly*{x~5u^r30&wO% zxT$HKFHJB$X;q(n1g@$S8ucSSJhliLi3k6kiyO@CUTvg!dT2^zuPb9@_bFMr#`gBY zyR^SJO$fs>R;VTP!n4|4M#xq^LJc7@JgA^>@5mGIFIICuoO!-qJGd3V&pSOTLtLjU z_9?B}NIC{I@(FxxNr!gXNbS;}dob=^5hDr?+0Q(5YB&q7Xnh|L*5W}m*X43Qc^3-l zEl>Euk4LRN%$NK47}TwOY!W}Zsf;K%yxw88XaZCo{VG;Q8iD6sqVH#Yw{D)*r$=pR z7iD6vaXYI(EvZL+|L(jQmZWzVpVG9veNqLfAqHK^3{8Nhs@L9-)?U9nL5Jfc1VFGJ2n%x}L48zz?l>*DS!=JodurT-IU( z|A6TW;E4Wy{zL?O!C=dt_C7C$>82jIu|YaPIOu2`Q_`@?qzn@h2u=AI^%$AqRD;@u zC74HPL#f6{($||jn=Mn8#{dxc946+S|FK6>52QfE{Q}K@T!{g!28Wk?<8$r-SZbR0 zj@mz?sGFp>^}=~|(Dp@I5jNVXwA|e6gxRk$tH`>ee&qbgW;=oLIM~Rt>zaO%ZI8G4 zYy~%>M7ayl_DRWy=O0uJLfRb}{Ca#a72gJO&@aWw_|$8dVCHH$Ojl;q`G2-0sh_zn_2mc!H60!JfZEh8yHFe%~JH5v_l}?nFPOT>84T@NA9b~UXi%EW>Z0;z4IJKi z9A2&EUaF+kpV##YC${+?8Flb_+CjE>f*Cfqu9`G;TMYG(1|AGrxHrq59y&Z{7Ce#A zzfpBS{WWaPY%4j{bP23nYn8RDFbZju7O7;SCU ze(vvY@)49u;xBkC-e<7@=Fr~EZIx)*M7dAp(pDq*n&uQ&XQMp#bN z&mo`?*%F@ug5HQ<2i_--aJho&`}D84ra6yMf{mhycAtqTPI$k^*8kLYop zTvToABrg4{x3OivlDu(-?OvZqD1E3QYXlDgEIb(E6p$ zLF3}q!2{27RZY0GU1r|-{`tL3_sa3(2QSF20WHXF)8*f^C8VazDrWYXmJ!#1bLd7S zLJw8PjAMHRDd-H!k9oW5%SrQAuz3+Zr=z1%mgM_1l#TH??|= zowk)TFD;w7?{hFPY0#caeu^VkZ$U^U7$4@OllBtQYI9Dwo3%h$64(x zb-4g41%^!>iIq>G=@!+ttwbrWxY$?g%XnJuTVn*|R!A>APnQLx=M!=a4ZT0vcfI<0 zGxuXzD*VG$D?74mPsb2%N|z@iW*o}{U;5b#Rb#fQQ>oxC{23k)C|!wSIyt)=x)bNJ zhqU{j^=}B~Tc5ypmq(j8&ZoAK zci)&iul6x4PSd7xXIBmfP6Y%f_i+c+;D9UkZg*o{=Ez|7Q?HEtjU5FzmC*6l{g#;z z|1zSxyH5d@ZSm*Y^@MHtIBz(mbB3zKC+*ggtvSIi?pc~~POpKtxw1b!vB@iKetPE{ z3m;b@+aw~t1S?1R-K)_VXaD-U@42-}OfNY)r2KPW@*4ftEmAXx_Cz`@C*B=OooNQ@ z8J!od(BeODey>n!`2Tyum>RhlsW`Ygo0u_Jy4cxPYhF4mh+}_e zOT7~!f=L%P!Pt$2%onRj!P4U}TF+ad37?$t+3VUGxCxY9R!Ykn8|T|G*}a2Yh{o8!xtsJk@H zMJ;NpUCS@TS)MOcpC}G!=?BOohKR&wRbEy<>GNC|6-_2UjdT0QT_0yoE#YOi$LgzX zM>^6I7|CnMVd|+aDFULn)Pf#o&t$qIMA=lf9S`^BT*RuSuoT)~hP$uKV6}POI-|f1 z5M-PT?q7?omH`OiXz6EZ%^as0MRmXge$m6TONQmZ%M=2S0tQG_!kH@BOZioq(w-U^<$WEzQkK{&(5T(M@#{(bzJrjSr zQ9h!H)&AHyQ=}(nE^sUUxF6LqnXH5@o*_eJSUjFKsGeLlufj^hoB*s@RWwi%V!1Ym zP4e_7dsdoP3wC!WR}LG-V!KZ%{177Tn5G+s zJ3R6J3SKtEj8U1;b%GUzk^qrF=}>TR|9W-5e!ZV&?19?0x%Sz3-CM5mcHhVowBPYu zZ<4H|lWjlT6i&oE8vM)mN2*(ql6>DIHB?A2m6=Z>?I}?<)fO^=H&u!-#YoZuV3TyG zV4ft(>{^Zhk@y5Noa`H?)2de%q@FS2+N*0J??p$_%duwYTFlL>Au{cg%LU>}f>Ro; zMBX?wn{X{Ji@KZb6ck^%n7CT29=n+xURXkh3> zS0r-C>Tw2oqQAfg9ZPfZ;x5a zlD0z~0GBb=*~b)@E|cpak!;tPXr%>>Q3kAY(hX~w$Euj;k{W+>d~fY+e4&(xev~E+ z3Pf|S1(_1(^yD=;d^6XV-@AMw&#(LQTN!*`6J@9wMGGli^7gp;bu-tyj{RFLB#l1P z;!JFu`;(;ug^z2{)93HUoaD`EMk|^oBndr14Y2+x#NhpmTI*McY*FJ3ck%z7)-yVvvj|s?Vg~%RP{$`j^ zAE*9%G^IDAeRron>FDAQ$G#^UVLWvvm=~|51+j$<50x4ozT{86C5`i180KwQFE#XV zx6hx2(3v8@+ksxCf4pta#4F;;6anYI`%hE&w#<5uLM=a~KBh86cLJE){Uv`^w@SajqYt870c z4L9O0-^cTz1K#IW>31RnGh!Rc8(7(qcgPLQ#N}&Ey8qTil;&}>DJWMo$A4yPKu)~H zYZH`XX2|=>c1P5_`RmU7t8wBSb9ufTm8c3TSi-Yopcnzv3q%?50m;&bA%*lMVT|M{ z5DV0m6Cs78D2KQ26YpQ~d2m#l6I54G*&RpvlIPb$F75zfKacY94pj=rdJ1=&Kl2?H zsvzNdv`NdOyu@J>&ZXT!GW0o1#=EIbHjoVYV? zU@Fzbg0L?I5tJT@y2meDiYV~(o%F+~8*!;UISqZTr?7^T(xO@XobO7GCF2P+AmW>S zytK$**P0yLs*+po`B@832(`(07y5u+AtyX$l%1|m( zx3Vfr=bRfA5g9jDmArDZlaXCnAPIVcVT0bual$nAgoMGe8xM{#qVn9aX8H7OUv45~ zOD){(D}U?jUuQGNxF*6B5C8xT5&&TQ@62X`Mvi)Bmj7!`QAK&)bZ<%zc|TU#?7Oi^+E4dL;)~V;As?>I zDoLbN?P28)pA_b4hQH<>YyDA9k4q>qk4!uaWe2QT0ToM37?=uWczJ1;qx?bV!k|Eg zB-LKT>Ke_N2fvS^ZD=jtvozvke5IfBUOwvcgD67kq_sJdaY~> zXA$mC$AzVip}yK!^>J}&Il#=GzPUlXArJIFWQ!LoO1n)YPfzZw-G>U*HMr;vIv;@; zB(-a7Hg8h65!(KdG|Wjfo9**5{$8JbD1*~P*BSA1T!{9pI*@4$*%C~R^)LM$<-6a! zI*IlY&XjFCZ$(`4UgnMHFr@uX1}=kz8Ze2Xlh6!1`;+(ftMRc6 z?ic;f$&cM0*}T6%T;|24QvJkje;sRzdI@REC_zdt*AJ1_SFgJ zi6ST9hriqEk&p>+>&YSNyXWiPNdY6i~&?FTthu}iVaNFaF6Y% z{nj;CvDrzrb(!ukRyjl%jdCZlZS${84+ZJftpEl9iocES|IBnVD-#-HGkXU|9a}vU zBWhb~6QE*X)sHY)X%RSR4CwC*z=??p%75<>|6!2d9x(v-?o01`2kO8trU?1{@`N-D z{XT}W6;*Qp0HDwQVL<*7aO&SDe>w`OItthu={cI&SmP<0Ss4N7=xM$;T4s*jx|Qz{ zSpOKIU}Nm)qGxXe;4`$*H^LJ(vo~_FvA1x*6Hs8XYux)@Qyb_@Ops5JHL5Uy%K}9X zEhsxRb3=-m4tyX*2o#@ynz?TtsJ7+?B+&O~eh{Qt@eQeJp&G}WXgD!+_@^IeA|j$* zBi&`*PiCYIgv7z~_3751E9j6Cl=oJbJ6Zh>Y&4qFY{OrYv0)K0KHF}a6o|l5N9**y zN%8Y+{`zrvkOaCjY|Zbdmr#IrSl!A&G-PV6G?*y5+{T|4RPv#eNazN;6XPThXq9y8 z2}EX9gPWUOCz^&0(KcFD!Kp0Dd=gN;i6l$v;^Bqz1%JYAba;>--6@_vfotA6D+ljB zM1P}hwaUS)T>yQzZtn*&pI=Z9K>|L~+;`LVygOa7H86m%pt`$baJ!WCO=O?Dnf;^Q z()mG*V38RObW2=D#77LZ+@a9VdtGZco6D)^Hvyf*aP6p^8dXXC&T|M z&eV|jkUYJ~7(=6YqGEiE`rFMK3y)Y_yseNtF)?%p@kmqPx026?VZB>{d!)1*`*ajg zL!v492}D9GPKQ;F3lC$Y{KMjEL{XXK!R9hj!XyM}#FJfDj52h6Bmb%uh-^wZ)4Rjc zF{gS2(8MA%A~tBj#-rHX$KKRsztXYKXQJ~OQw{>;nwiv^^jRnFrib{;aZtk=H=HNf zb=}sn+a|0mKs;XoM4%_~l0*angO}~d8yoO`_6~G-rCgGnNBIxr0mOu)1*`aV{r?sE zi?cekX5X+k{I>sK3HE=5zK)@pgPy*nk>Ni%UO1x;4M0W#6(NY zBwjwnwm9*{50KB`3eeVViZ4MGze_F0GlVc)yEgLB!R0zvh}l4b0D5C@V1NN^3VPO2?t=LsfSc}a&Xv;}l$Ifd4Z5F} zBD_yYRy=pZ8id!fQp)=(M+wFNHj=JpkzGoViIaJVNDocesfx~jan;YfvN8C`t(K1D z6*p9 zC$A$5z^R+gL>6pF|FL~5YklM53x)&{0pbtqX3S7X@zR4G+mW`7Du$)&EaRXfyBDY^`4)D}aJ67y9$A+d zIS=3&C}-mC-74#7ZN>7D7}7sRe(VX*1>#%claELM*akXYmjT=U1@P+jaG)sxLS*>z z1iV{tf;^%rAJ}@Y<^OvCL#TaP)qaEbANo!EJe*BwT zf=J+hEW0+U@wf0I8IR>mx*(+KviXl zN6FKD-`x}g?xvn3q~BL*lDot6Wjv6uUvh!~7Xn6z@(P+QqALU2yhH9(p8H9~o=p?x)V(NZmI)lQ~?f**2k zm@EbP5x55fMcTCHTZV-p0S@V`rmoeW)KvOb{#xMphNVKwpQf($eY-_SIxN0=Guy z&=wMZR^h!AR3t_9wbF*Bx$f@dFj5)XQ$pU zSdIm|#w7XqMIt{)iaz-@XXk=8x0`FX-cop)=q#+}KhP9?Pga@7$`XzQy1WUQo^1?r zv_r68X{ni|#F|l<(Hf9x-8%k#w%| zIgP=K7!#4g`8#HnA$Kq9(+mc%4U5w-f}l%hs29#_bLt4ibEP_^ik4$I&~=D^SzYG?3sVf8#xQ-V^b^Xz$e~2 z@NnfD)d!v1wa5#*Xc-3W-s|wB+7v1D>Q^cfb9-uX#~5w$3IpA{-Dcwe=3<->{v-@an`Q&8?cA=LZ?r?5LNbS!$W z14tI}2i!ISpv&V>W7tlcBiaB^v#73N;j@MOZt($Dge1!;DNBt1?~(kX8PJOGN{4f7mjtZA=wm6dCaDkm#Xo?56i)kfLD&?3gr!Kdh&#M2=Xr2fPU z<>f^7&%lorMlwMo?pYq{)$O6GstZ@DsGC*am}NFwTl2cQ`2Ca>&$Fm}JwOy}$NH7& zakc*Tma+Aexb*~mgwDSrwmL zNgpZyh7&3aEG{-0Wu;rH(1*JNR?wDWT}=zdKl@XE(j!2aFTA#@}> z?3{~U3fS4AoiXijrq8_@ymRJYsXB#{*mfOjanD^XYofes)<7KMuu-HGqA9-)QV_IW z+542#(ru}XcVrL@iBsU|GOtZuUus;k4H4&_x-qT{f<fScjjq?0vA>Ae9I;D-89a8P!~xeHeA*( z206)Gvq+vwF07m=jD!w8fEOB}5apmnoFW}xnz6OkE-k%yZjNURWXmH3BxLGB77^rq z6^}xZ108EcP^Oxr_5=nN{3^goR09_xKEVAcEc$bP=k7 znW(lOFR-3)s{7DYT+B$rd#YFj}?17=Zn1T_|Z zHZ@dU50;o1$*;eQ+Z?H&M?GW#o}4i70Z|#h5COab44-I~pEzu*sG;oO;r=Om?c+GV zA5G8{lnfz=Ku|B7_*EJk0=wSs0a{!jz;#GYMFB*aR+|hd z=m)ePEk-%A?a$u8^MYdD7vBN2h{l}6m_}rJ8UTNvoZSmwMC}ugDBZNVO>Ea2A60T?k zwyvy!^^hjELwHzZGWwCdXaNr3HKrzV;yHo+dXRb7U*IdB`!LqW6JUR#^`*QLd;biN z!om9|vL4ki=gL#tpWf4nikXwL$o2jRCbw*z=*3=3<#s|^(y0ZXPAVA}4(tk#6}A!>{NaQj(PKJ*Dsm&BY>%Faj2Z{L4;~KKXZf`MEowqoH%o>gAN>_-xuw zsX(ETtPl+s*TmKqG0^bxoDGk%3nhcY8McgWY5f^Tye}4)W|w@>fyX0p-|?5u=U%%i zn+yB*VnRj*={-q5Mp>nkY1{nk!HuZ80OVM#AX7Pzojj@@5H~iv5`h|JL#SqX!%9Sm!Jg`t8F=H=$?vLlBX&fO! z<yi?x1SO9On2Kg)Fk%omvn0B(WBhnvStw+M zh-4KCc%Ff@wEirUdR-pQkIUZH6KC0$g|l8+tO|HXUb5~bo4F%tOr?to=Rlrb~_+scbzFq zcfB)ieNB0J9$spsvQ$Q+KX5;cA*MjjCXp5xQ_W&j%&B7UMGD`tXD}2H433SjLVbj_ zEF@8BQ1hFPzfOBU((ruZ#&Ak27{KfeB1Nnn)qA^f*jc9_SD)uA`_d-$4^P*4Ubuh1 zp*L=d5M@lLBXM|VO4?X5b)v9bxyJLsToFZXO;*OohDHPdXa8l`r+4JV`fxI3=oM9> z_3m-4<&Ch};Q|#qGFnr&A(l`FOMul+5jqNA^kvx`9XBG{z`y6FDyqE2We)qxH?QZ$4-I3%PbYSytrOTda`UvNop>;PCws`36QvrrMo zU~yf-OrtS);kPpKA~U`AY$SZN?dfs(x#ual@ax-6p^#gADuafZPJL4J9}6^0EdA~K zaz_{J?UAaZ)Gg*HV(jwe8a=M>Y5+FZdoocW3UT~K{Cli}K3d$akIwfUT+f=xC%)ce zAv5BiPd!c9x7RV*f+JyjzVh5tK+^WxJ$z}HX)T9Cn5r$d+*+hmWlcpVKgDBG#fl3(wNXqDqt8{%@O=XV9i80BMn-v_zp@qoezwn1?(Oeyzw^}P}*@3BS-Fp9| z(jIcP6bqEG*>aQ4VjaAhl7_!rpOU>ouc(Z*F3}syaHH&gRi|#ZWNM8A7U=v%1`B_) zzQ<#7P)9tWT^IRo>gQ@0`7Rm_BS1Kz(-JH3lXCldMj^xdg3=&JE~7+V;ppao#w*RD zN`00jS6XxiPSV%Ydpt=>hO_?Z)#~dL&2E=4lb-@1K+qj%$oH18Ol)HGQrr7;0tV}x zeKaFzy!AZ|Q?+%6LfOf3d$#ogF%b4Z7!3n`w7qVVJsrxq(V;tdPN&mYvqO zDzRn(NozQ|PQPEch*>(%V%pjhpN)r;S4x1;Nx_|t)7@r6uAFA`rgC}lvmun$m5mZ#5TAjbd6AGQ<>f4A%aF(N)3 z+$hoV^6FB%9Z#&;dTCw-gDpVH;r70Bw?qs|hXi*>WT$)-_Q)WB&r=B@IFp!sT zt+GtQY&xjp_&V+RiUV3mjF27?k+6ZS;%}&|qqyIPVu8Fy=H#uu=CZ@G=MLX%OFItZ zdw)6TnC!jrj78zTfa-jgG7*sK^kHYe&`T)W@LEU+P>fp)N7p``n8;2}PT1HKb7~8! z_y2Bp;o}~zE0mxOBdLMYj}x9RX_hK&X;3jo z^oO&s5AoQp%l3-&<#KClI?rv6o&OlzcB;XjvAZK~T%~;qNd{o^hT8C8Bzvi5cl2WWF)Mpw(N2J4T`v95H;jEV$`D*eBl-*eT`$`g9Z}L`} zNR{ZuO}l+; zaJw*VWNOe`zyU;km<)96E)Fsfk<~<-1wbh+iU=Jc2pJ@x8`vXCE4_y@FooA2t6x#8 z&8s@sQ(o;P;6+7^nhO$39^+|-6^}N37i+&;Lh30*3Ca~jcWN@Z8?IrUYi4|AJS&I! zDM*qC>*7w}u5fJwtXAv(Lk~LMK5GP76nSsQc8c~U6N+sICCAnk4S$A!7e)x!HA|#X z6X*!i3!x$xNhQn)I7_CfG>k%aNY)NY5Jwwjm*_c8L`{WUqW`_IyydZJt>xY8YNJBk z*pe95$7uTt#PDNvkWzD5JU$X%3oJo$9$X%h)26Q*SsJfGfZcX#8+juPPjW0PQy5Xh$%#7DqdgZ#!QC zRlYBuI>G2Qv2>erx2%@}w-1F5x+14rebB~=xY3ku<#EvU&RBuPg_&`&IZhg;z_W^kim1|38&W=1a;-jXO# zKvZc(g*XfgA0z7o|0i1169L(MWK==5$(rfR#udTa;m)|O(Z~PFN!N!#K?}I!sI3v( z=e1QPPK-^GQ@Q}FNS^2brXqr_uZ#PHY6_}?j@61p6M>XhqmLJVa6sI+AlFSDSwW*n zx5BtVRk!5ZgR;Tn@ItDp4VGDdVq*3_7D)eSN|n`kqsl8mJ#gB4>sBVAHVzD<+j5)c zYIaH#|AYPYEW@Kq#PVHpuyBetJFai5N_wN@>$&0B!&czge^8otiwQ zds;z`<^G6%S|nxETPHM;#S_w6p0qblTSkb5UfXn@B6F5DhowJBTmj%LJZoiikWVOaWV~oUh5>7@zMZ4rf(gj0_{P77x}@ zy(7J9+!R|@x+1Cdn9CyV`Tf-26k7&ECBLL(6`K0nw93@Q#v8N@S9?VAW7k-ZY--#N zr#eSs23h<}_TbPnEM_`(MXrqKO+ij9U)m^HhC z$Cs#024^KkjdQA^qbr^kzurE(SABqb76mvvQ;kmleO$#}9~6cDJ32`9ZU13h#oEZ# z@t<)OD&X1wS24lwI16+?5Fc_C+Mlx0H6avcp3+nbAg1i6k`c1UcPOs7z7$rTwrVr`dk7{}@0U zK&x1sS3a5p%9Mt{J7(fdF^&yG$bgg%JP4}o2}`{8K2h`KX{$`y6R@L4v{1nJ0+QQj zWo^-GWyKC{B;JCE2dGxosuRult1tRX#mg3`YlTXOZ#LWdqLBVTByjDJWVQ{ z=IKwTYVe$ixjXm_xy5~@IIvpx#Lf6?!IjetZ)lz7R?XaEr_=e4sH3uqP$i^iETJR( zD`>s^$Z#$Y9oS}bCXM2B>FJ(BIGr}y|>== zQmQ1$hA$c+-4N^uPN7qB0K#Nnb+lQUv2hci>m8X_5Dh*gHI+&~k_Mm3Hx!Q`5V52z ze8x1f^i+bB%C{`9ED#a#2X(RpX#v>v=3eKKcejEPYV%su+=*aXQRFwI zAb>ZX2rFi|k1hF&j#r6}A)aW*mKB9}^Xf^1&w!*^`^);DV*(EUm5T zJ40#Nyv8qhTSvJxeh~lSxe45IsFN~jELl8v1u#-OOrJE?LK$@EF169pYgG68zWm$S zL)+w?Cxj>XChiMJ2?sXo(Wb8qf-B&@*)xXDoOIh}k2L6c+3?fy1^3@0D%+<6|G&hH z=6l$GknOXzH*z+!adPY-Dd6a(s6;)MHlrxsFsGy)> z9$mBIqGeIFiQqor_&-5tV)eXY{qpkPrahq*Mb9v6BMu(|A%+8ruyX07fx_6>%WLyh6f+ zarI6cP9g=byS5xJ9vJa}p^uE$+T|t~!1vZoFz;Xs7#KeBsBnoT+F?Oz!L@1EHGIgH zhN<8z802{Z{Uq#u@szKsq6*lh7W9&!_8Q*$(7kawBaXk|iD9kvwhYvo&~Bu=;JSvj z#On|iCXQP}bu?A(ZVlzQxEm37g~+YdLcuHmq!LEos$s93Bo_FG5D*bS6+jtlXWkG^ z-A`dgLLg!BVTKL_A!yhImyRnH3`Bqn_9bUAnVXGRqz$w<*h1aXaohIq;kOYh=KA_A zW>4S7`rm`!zlhoy?6t$uLHw@1uI~23y)6{*EHK_gFF4Tw&zB6SJmsKt7CsiQY=7J= z(7wY@SHYhQw6@sbdcw$3e-?Q8YLS*gk&g1_2jVB*IfG&+))m1)ki7^6yg=#$I{Bt} zQR(qedtc0m55j~>VWY`k#7d#9Q-w+oUW8T#p{II_f;4c`>1b(RJ=$14B=uXZTYqZb$1r2MEco6w~nIvpH}s%gLS zm&MlOe*9oDeq54}Uc37kcB}N!!`xkD;MV1oZGW@g$ntt6FKU~U;ZdtEvs2PJUR=qo zPtEdbYDn{l{xeQYZ2IwFUMR)@r`rAgIZicer$0IvNy9+oLQg-zn!c!-;X*!K;r?PY zrKD+gwCt6nqkUW7S7dhBrv!E+pQ>ywY&adGj``Txuw1;etEfnhCaMgS&w- z=yBC7zQOSTACuVFv1W5LQ~@g|OTqe-m|59M_ijEs(Qh(G@*6-Ew`~#QASb>Dql29` z=WuN@$>tplf=-g11a7wJ{`KMcBvU;kQdO zi{gvg#Uk>TxD=hANXb0?-Dq-AyYK^nAh9B(v* z*|fd4^7GGOp>CH+D{9Nz?dL(~R`;{v<4$vo_nZ0SWlUyGL$dUr^6gt@Lkx%wJUS9i z#W9R>wVAP2%Q#|yhrBLJjjN?V_aa%FUo9!HRIn+?@&`mo(IoBg7x?o$#j2snz?YV* z=ng7jf~mGDE&B<$2nnaVN8oFLQvwC{jL zVWEW6__2HCkS#FPy)a7A0wrT5jQ5LX0R43YiPpq@#AwdQLgmCUd=*o-m=!!9R7THW zE2gDNB8Vd=oC(ecBA?l9I1EW#q~<0bm#(FVC3RZ>ozH(Zk0ey&>1x9JT0 z;qAP51%gKs*OR^d?Q?IMC#%cF=5^*{A%6n3Nn3RPdfnKdU#YrwA?>_n^pJf1f;Kfx z&=y+II#Gi3=RAo&5Hfi4$tQ{9ewW3~I;6>(Q-dL!@Zam}HJB1Ys*wzHZecGecoZ?M z(II}k&e&A((IfL}nK(U2Bs#EkI}l|c%-<5@N^yauM(&P-`7&u7Es#PT+Q7YmASFDVy zG}^W>q+sgUb^DL9<#Cef4%qiqEjqAZ5(?BIlj0516g32y_FUwHV)Q6xlFJk7eNY{N zZPFjT@(xKW(Wz)ub6%Boik>u7H(JCs(S}ZHzEKt7m;e)}OO`MBc6;Y0)VG(}J7q3& z$FGgEt&}Z(sp{|G?`occ2llv9hwU7X5yziQ3#@P=zmMj}e92rtlLY z0d;mFXTVH_QhpU4gZ=kpRSG)axE0Ypk+ps$GQ{wT)vwA1BojoeZNM%dWCPQ7$6wb6 zj-vj{-=x?~fOM`fAKKnj|Lx9Wt3JE?Xg46h83sEiw?J?P@rB`@h-4REEKXEz>7pZ` zy=9MJ>7pfCc-PrNzQ9kPmJ>ycU>5n$b*I<8S^Go7hPo45DvEfCnsDaZo<}imc4qd5 z0LcC$v!C>o3{y${4@XZ%ZjA3t@Q$oNJ7#V^!Y0H7c29!_8p|#)@(UF4jw&R#e7q5@ z5&9mE-xHy`&dk6-D4jnk3SZn`MYb+T6L<*)3Ynqe)#L@4Q9=~k_@e+Am@Y2@xtBTt zc6=}uC5OHu#)dy5)D;5dbrspoJ>nzE%D{0HeLcfxzA*iCpmVT~Fok$z*sML{i$RUt zI$>nHC}92$@du2cf}$vD_n7<#Blf-{p%iE>TI#;u1n9LVjD%*Q0%)m-*4ITk5JX@) zt(LoK{p-Z_nyUl{3_Vd2_OvlVf;Muv7W)>547O&;(VsM&)X8tB05u&SoxZYO^}%@Q z`#ke}$}m}KkPC`1+Fxop>B{(ee($~U=zppIm7M)?v*e;P+3h+rjIneMF5u35L%45E>>KW$d3CKzq}D{{$&ANCfAZIwS7%51 zXp$|K-JbHs(i)m68h7i-KcwXE<(A#+c<8BaLYG;L#q+LfwW&Ckor6YesA`199-xyl z#1vu?^***j;ym6n$%#%Mfns1@LfSq3ZBs(uh?q#){B8Q|dn0J8t{1pciIQhK)E zk}#>FA%o0^;>~iyBN>X&&adv-xLAPj42%{a&W|#xUVJfh;auM?^qUbsd=)9s8~_HZ z%o~tQfq?|2_)O>A^f21JeLi*R?Ci2?E8&PEdSrjdar8Eo#YR+Tulb9C{`eP4G=xF1 z3=rpiVo6XA=$z~#6CxOW2ZVkZ#Xc)HzqGGy3*8;aB=!ApRnBnCsm8(Tqlkg`+-4*7 z-vJ0jOj66x!-qrJ+$<#{8hKHFdwCf)M7N_5m@w$GA$~9cjOwnu*|*h4>5qPRIC66? zGok55K_^|dpVTdYsT)StAdSz?7VF)gq>)QPNHy&?8>q=SIpDVB`yjTbgOpsuzKs{q z>RNmXM*c6w>Mbx1TDpp{b+Im~wU^3u>`^@W*jTkPvsl~XU{Gc3`1o0IPJD@t+9~yx zu1nYsp<&9!L+`bvw3+BZhD%YQ&VLIvM$O5iouSfw?|N=qT20-osWo^sjc@mFJqL=T z={``r)J`wZ%V?z74W%%HkkZ`ema_%Me=p5b4oH}OXc(MftF;k{Viaf*(LI~p|78g` z>Y^GwM4w9|kx+5a&I1xPRO|_RwsipwW$jIQfghNX-n21`yONm4+_fFqr@#OM>Mq#< zn?Vta6p%j8t6Ss9u=#-P zRvo2Gi;<~rl}zNZ< zkL9$~ay}f<72O)%J71nLpMHK!jAQ95UCRB*oT}3@GE#x?v>66>X+L~U83+TI5Fl^o z$T18GO6E;##>Yolr=M2be1j$>i~*QF`I=5KJLF{@6+S~>$et`oPZsSK-6S}>T5VnB zvT>m6P(iI4ERFT=tRY`nJN={7e`oZD;$q&x{*}@@zx3KqzNtL%9k` zSIt*E*kW~dM5itq4A=d8hMuXBB81jbt`y1o_ypwh!LpNYZ^X3o)b3%081%Cpn^tqW zPQvKuFu%g)g!j-a4dA~z-hq!WVMTHpWaan#fIT0l-s^8SU$B-ii1W+VPpXN%Gq`Jo zldR;g{!k#HubM>m#YJus+AT4*_ea&AHibt)oj>Bhcawg17Z6iI%RBI--Z1Xz+*k#PoDe!RzZCm z8%GC6`+sWTg_n;D=5Xcl*DDI4$afN2ArY&&ISff%q$yEl<4Dy#X?>kr$x3Bkd^K>l zZoSZ`P?_fqjJDf%WrdiWW2*B=wYZqh#h4Zi792#EHl>ngX=y(B)J;y&h)JUO^kVLF z?(<6NsjaMtLqkJF%}H&e$-{YoYVEO=P}74@AM z^C{73wgvR+RJaDysP#gxrSe1@cF3g;d*SQ!y_GwC#=)HUWS!5%otiR+X{O%t$tn`CWf|&wlVbTQGeMT)w_54)cl&IvVNxD>amXWE6)cA(Ea!cYjqp;g4k5k z^hGsyj}o-qRhqJFNtc37sn3@S%+WcN_X~;K)z)uqnBXU(67cM};G|obH9vn^?HGoxyS$n!oO#KbsAXb{-#_ zd=z@fNMC3eFqHD}JrQ$(0tMz|4tovf8#|(4XfXTJFj4zoq?k)z`_+<^S9)3dbvB~& zf7@1C@YMh&dNoS!C^~_XaM66lKT@>bGgOMJI2!ZWn|oPI*G^Jjc~H7*#;zeAxa!c$ z-H6WQCiwwMLLEk^j1^COs8EKUTKjH7fYbNWfl~hVUw_eqo zOgHH|z+iUEvZL!=SWawHHj!N5#_bg-ZKxdU(_hexb!Hv6Q%g9~rsdy%9cAmWvh;m!}Wry4MD-(<{dJc3*EVARiv{RA>w8{`rvBV1gQd?h&31GIKPJ2td}I&~-C zSUopt3j6SG%P=+18XaZ5SUrn=P9nxb^B}pVS5i@pkEQi)_{nruJa_UOefogXKscj7 z0<`rz<3*-HrtdrcU(q2XUq9WR|D|E%vhLvqNZ*M__NbA##P#%f4tE}H^0Jvc4$trC z7?oIeVmHZe5NDYT=_O2y6*peqWAJ4VeO^l|w<@nfo-0IlmSXjt<_g+&;oeD4G4btdOa<$8C^MfSo zTNjm9QS3X#R>pgPXjA5?r1&cy2JimExh1rA;UmooV9Pa5%^}tq{YF|c^0@V(#*AIu zU(vqsRoBk(yn9~lLq6TTF52W?2Eq&?i8u8yZz=7wjfoy*oi?RAIJb@EjboA{2wp(Y z+nlXcbZ19gSY*A4Ym!iYx{46#_i(3@*UM|E2X1m*!+TC07k9vyPIn8^0IFtzX}{0g zhh7lP6plV}&ufagTz_8Vn%;?4Y_xK*kIX@1J>}G@)b!5l>z!XA9}1kcURiin#_27_Ndhjb4nMroCE8wC6=`O6P&?jN4vNXecFh9Ot;F# zXxLiEboFE~1zCry$iorymvWx-Vud`hX^G13U$nzCTkqNZDiY#orHr^SM=CD`8j`J3 z{f>U#!HiXBK%Q{pV3zy8CX($f!@FbUqa{+!Tcd--I`k3l=|CCr_7X*nnl=OZXb(Ga zKvvtZHPaG)r?6Xk6~RosmApE#a27pDU*iuAx*AeBoO(`{MV7|ZeA?)akeJ`=_@+Uv zX2g+yH&x0mQV}^FoGHK3Q((7B!CNu$kq1mjAg(Kwz+nhGFT){zwBQWWEpKsbtgUl! zM>vZRsmCWzfHeoos=M` zUUs^m@73oMmNp}(H9ne;*a0R>V?Z?aqZF!Zx*F4R~z1M6@ z%d^;nr{SIA2)Hh2%$?ZQzZY4dE<9Ai=S4*bE43q$DwCpl)7blEjgCKZw4`{=FZ$bw zElD-0Fx8k$W87gZWvw8IxEeDV?TwW*wW8{eaRjMhYTT+b7zp!#VUq6I%b6`1uX_mz z<9qA$%1je*B(KB5EVgMJRBw}dSJqEXSV9iU5QNH0{_65t(%(e;FeP^I$mP-wzk8vPifLA_k-e5#*UKhuYQ zYn-xK@@`TOpy6~5lGTtiju8D!Qh6`!QRhihPl5%C8by>eachG*R8d2MKN}~gJukUx zJvDL}|C4whQ(YJm%)B&5nAWBpjjg?8UDdHj0~kZ3IO#0=JHguIJ0n_1cfIcW1GaV< z3k?*T#t2bU^^duqkZz4B0~CL{Z;x1i58e#-i`2l2iHbT%^sTI%YEit3z|4rhfM>2>(lpgCo#@WiDvCDJ72?wu(lCP;c<1a z#n;jKWuB-B;kwd6ikvs^Nl8m)PC<=<$lTa6zv7l69lJNuiT{w4#|hJMDZ^^dQhTPp zJnG#J(=rFc^88&G4_EV7&E+J9weUS+I7=w%YmAANDiS>WaLo&=#FnwuWQa>YosINc zi%}nV`3CB+4CAMPhsJ!F+LQCD%luVybMpfm0i3}wlDtnl6F$YAXki_+ZFz+5tCt!7B{%8{c(V9!40GFVEW_$3m7GhdaoOE*fZPi>7# zY+I6(Cj~nVK}!xx1FmTz5^@RnY{KVH>@jGmeJA9hAU_*2$7-@xQXI%Lq@Aug0-rrr z-{GzFU&wd86k5}>6QSI{r;urRR9)st>B$Y;J^pg+T=z|`0F(cimh zjNe`;G1IOAKehF`9}|Iwu%KQeurZxRhu0o9&R&~eT;By<(2Y-&E|X(6Iq&M) zL~xjc{5)0LtJogUkGj&G9b75At4fXHqDz6fea>V)bFR^^0UO=MA_Dvsojb^i4!NSQ z(Z$sp7pYqE%`RcS`m+y9Esx^2(WZ+R$-rO@Z@EN&5xtYH%{soUvIdGNq^njMv8I9) z%aFlL^(J~B(5Yt(j-Z~8&u8Z|*#}9wDV`veKjfS(t4gzpl9VureCsH3e$5_GOI6FA z>3*tC3L*L|=RWH&GmE@{thIBL#g8I%A3SH3tXAKJJ|h_c^;*24PVg3d;zpSG>h=PB zxZ=9>vhU_L)`SDv7syVTXy;t|3wp&qGZc}5U_vgXCb~@oE^H7BlpV@H%gFVbhn;u< zRtgK$waM#}0&#_$K494N?-K9jvJnCAxZR)XaC=%QX`&_ARP*IR!k)1eT1HoCMk@q)HeTZhPb#f{wqkn%e&M?*`n!D+sQd zKkN0tU3H=(G&GbFg0Lgi))QQL9_))`cDCg4TO<+Yui5r8%vSc?*}wlP>?5}o#xZ8k zUC%(8HtB;1E@q&F@y6EZJ0GtwqJa>$RjiaP`aE$}{n(;`RyJoy&kv!YGY?dRF=nBR z4qH@K!bdyyN;o%@uUkw>SbynL=IngAn6}AS4$a;U-40F5&L7Ur-D!L72ysa`)e}Sr zmo)upIdHfaJt%&L6EPi?!n;o@;*9uEPl!+xJ!R zq#&#`0mHInvAyvphrRvg851$jHL*5_v;C&M+x2}P5X`e%SU;)r`+g3M1AyuNa|=vF z7CwDZfb2@GNTHbL6!3P~qaS(u0Bz!Uag034xE@&Fs)y>`j=Fe-!=eL-&ZJhdMUvp04hl2Pt`0gO|I2!23<6#@7Y)_nbx)=q}R zLkL1l@o<>G7(i45M4DJQl{IT5YMe-)py;@ol&4t`EAkFzi#ZN{5A8Xs8zZnDFi`Pi z-?n!Dk$#ER?)r9?*q&p_q@52KUJDUc}v^qz)Gl__Hnpb!5OmdIUkgtcAn==%hi1n0>eq>uQvtUiBAG`@@ zEq*PNv%1{q&A6my)vj;sOt%0Po-cq!APxK(m$YhX6&>F`Xz%8`tA0VDAeq~k`oPGv zqoDn(`2wDM6ys8`=>GzGK!v}KRX*ti{uu9uFZb;=8urbN&@hkl0GNl}#ofV;m0{4k04D8?2aaP5{ zp5s($Kua$RSEWW9yn!!?(~;hf5XX0BO23YVGXQ&% z|Dx{|4%3@#{2~QI_pGCf$lYbfWZYewourMe8XR}_NZsNS28+a+Gp@gZy{6G=2;@W?Z{NbfecmS7;vR_Af#Bh)A4WCRNv@G+M+b`Jb zS=*9EJ&?9*R0B~Qcg`cKun3uFu2T4HDgC@CfaZ2rOZkAUV^e97jdll+K%ZxQKPzrd?eV__(sQug{gT?`QwEY|8aMSU zNbU_EXY(-QATN)m*+k!ui%q+^XdbzA!X$+TqR|coj|k58vZtrkd8tfr31FHhVO@DD*K%^>Z4q&DV?M8TaudDdzwmIyIt|y9{Gj+> z?Qq2sJvdx(zyc0j@o>dLU+QqhJK7c+TJ~F3wlm!qbmW>q{TaiYeU)SlO3uTYgT6 z;Qv-}rtiN#I^I9}@zq1G%wR$`oCEZQ_U&YB+OIa%laaRMF_ml3Ht}hp_RW`=L!Kvx z$|8&2LbE}n#-D3Nss$ZS6vcH7<9ya@k%@!fGQZQ-7iW8$|3BUQ<=kv)9{R75{fT#7 z%LHZB8Si+N_uWDA*fJ}4rrw_~RXYl5{UMvoOL)aCEZ(FGBA!s4#xPSUp*>1-4J<09s{Z4Gh5J6d+ZJDjco5<9pw%+V3H#r$W4$xy-(RpU#cm@~8$}lC z!kdTdTivH$zyIC#`>+1+{^?hVyCrU}6uy7Iv^0$@-Ix?;xJ#e)%c__j!wAwVN@Uyp zpCdJdjNm))`apg42c%J~aDatD;$>MR(#+Tmx`Fwuc46GeE=M9Rzxpv?Rt`~-Q{#MR zQdC_>wPfdQUgq#Gm9kV*_HJR2_Qr_^kRDNd>-S8RS^a}r|Xmiv31{pe^#vuJ}+e7+JMedVP(c{M)5)Rgn?q9hMZd>Bdj&$3& z3jJ+70uh^2XW2Vz~u^+@VhMpp4f=f@) zz5^D_;A);C2Yne&(a2kXi*iFAAHV#wIz3V+r+csV_g?RN{0?3o{BZE<^kflZNnBfF zFx?=SODWU^7Ps)ShLXe8Tu-sxL1bTPg{~!QuUU^}fLI!a6+xMn%fkuA1KGAV+VU5=U+7`XVI4*+(2ikjAAvxvtB0S*HNT$|vdgIgI5v9YyQt z$PqB%bQGYev&}lXniR9_xxwfK8EPVyz7|c1rg@bu3_KD8++terPOPzEZg0C9&-Y&ZU)-Gj)|!c4*R92dT0Uq_ z$fCcYStTP#HNaM9#^0pj6;Y=wkC~*S==lLkn}jOZX9%OI_gb)Myl;1>ImpP6nbj7W z$9ftgw@f8?I?!n5i&|CyJ8ZCdmNBhG8-9713Mk9kiCtt}w#qTDU+g=HCg#D=3xwNh zTGuUy=``aFXIYv5l11a2m;zz)U~~F~4>r(uIvo?4VQ~HW3Z9x=pk zep2uZ!I)3mAuN9GguzV`-A`PUv3$4pPQ~`avum)?t`N`4G~ZC>;|vq_3;RM=b!l+w zzBO4fodSo9P75=1+P}2Hhu$=FmTJWZ{cAUv$Rl@`>zN}NmlgE>BxCoQpZ%=AQqE2cI==aQ>NxRr|hg+#qU! zsq#dq`CvVIG-0>*C58U_N09nVA6iN}i};1hOGC1J(vH5Nd*lrh%RC6GLC(ucjZ!u$ z+FgsdY`242F+D@@T1kVRxFDj^ZB4{eL1^6vFt>!;ZDd_8T;jfsz`0;&pg!X$B(CO| zJ~6$*)4B>o18B*4l_{!BIp!UVNd!bpH?D1!Ve~j9*0#H+&jZFMjBT#w(DDK#*xv9Oz1++G@^&B(THEv zK=E|bGG-~d@szv#ny&^Cqlg|U-4A{<1%)w-jWYri2NXhGFb=ou8ZWzrWhwD*fxzxH{?MMhjY$#3*qA&a&HrT>Ki=$!`BEL`+1Bt&TDqhil>+O#(rwARF8 zWCLCL)VXG{IUsnfWV+$iy}oXZzGN*f654FHW#xUu_KC>wHfN8o%A#?&Shwd>W+TzN z%5XkI1I8NqLDT3Yn!}K^%7-1+>#hnnuKuOb=zXP~R)NgS5yu-zFz1VmkSL=xSRd{M zRJ}mtfxPF=T}clIbqTY;0bf#|Z*xa+#oofT`fAfQR+lf~#umt(2$lFFvahL9!}TI3 zV!04x)9+rQ#XP!x0Fp~TP)YrOInrbMwPxzI73<-Ag#ucIv~k-ouzGb3`xh`gK4OH| zw{EiPx)_xzol)%02(4DF11L-yeQB%dvys?V&J>2`{VH%9H0(dTR-s2r9@47)*H za%cyJzQw1WH3mKPLGHqT8gKjchrlA1tWyvI(A~@Ivf5EbiAJ$5kpWhVptu|L#^Vmy zlh9uyCPvCdfv>LweWAV~b)6QCV__+X6l^f^v!kk_W@XggSI|SW}=}g3L9m7$ZIzcLhbZsX+^<ePJd`XrsoFmI7YPgrPofKMIFb6-Pm6^YhjOicX+36LC{~j$2owKm2@>&ez&^+oKlyA4&7uLgpeXJB=YL^PmX1)75m`TI7@Wmcc%D~7 zTqX+Z#`s+pgkWh59{fD7#;5s?o-?3M_S_m%I_hCZt#5r{B9{>rcP&{h%%|u!Fdt%^wp~`V(DXwB!C5pr(0q9CP;$@mU_Uga2}WIWuk>jAsNPr5aLoQMxpV3MFY7%g`g=uFWL)`cK&4ZUei zr)HiGHdMHy@5#{j2KV(nTjQ-)D2=ya?NeT)tcqe><NemE(o`3y|Fi#s#} zl~o*+9zY+em3LV9pJ@pc{eA|!_vdIh_qR$k-09KL%hSW-56Wa{x0zEtgUd(A(Pc8b z?*sv3K6oxH$|e+CA1kVKe(zWjy)0$lhz{Np^AUCRC(3jaU&PCDP$>=Dc3eIWkxG%a zU_+ON@GV=~8kJ=$)#9?WH#f($^*#uTb_T>{2L<`mSiAND<&O5E>W@`+GezwIYJLaW zu_A4F@W_nnkpj&!^FNTIU7S#b+RTTm9QK-Q|OK>u%0G6K#%{H4rOin=VXFd3n> z)kmP@-0djyUKl>OEyiK46=rgDZ}aRH9c(+b+pho^73|Ovuw&v4@58zC)(K9#U~!%9 zzLn8E%d#M!UXuTwO z_}dluOhv)N#2BX+ht+pCd~Hs~T5m~gE#i$f(%AJO-sOs0Lt83tXvI7oa9l&1wcU$N0R@J(hmzBBz(L2qC`DG4#aC@Ci6f#nt zj`Ls8?cuTs5kowN-^GOXKI8GR=qYWrtk-WLnYC1g6Kpfdl+!WFD`mais26tW;t(;V zZfwrfzg79>=C@&Rna&35E1%`(GOWr}aJPeN53ffKlElD+XWlF&gOgu-EkMLq@YmlB zBI{_-r;|D9v@*#OX@Q42%|1ihyyLj5*z1dQ`e(u$I&~mN`Wf|RAoK>K2LjtuXRVnk z4l!VXhtP&lu7#e&yFt_+*bNOToB_^}Q6TQ&Q=J zosf(l9m7#30&-P}@7zlJIlJot@bNsjrf&N>g9;e0Wa}6Z`m_0J&9s%MK`U6Y(8W(d z1`tdU24DI9MFZh`mtAXE53G2E5*&eh!(T{%AbrTCNF8!%oL*sfb?}_2#pHIK#8?3- zTJo1-(<9!OL@@D}5@V}4oMC4!e}$Fk6y!l_r*iMlo^v&g^+wQOwz^4}su1)SD&$rM zbV6kU7h#5U{Ekvjw@ml??1M$7XZm60rvaRtyS`lEm_@bz*?1JKHyO)Kl(p~3T2_Na z&nP)M#Yn+n(@^!a#?Wa|z{^8Dp{FoKQAy!c8(55ZK*rTai779krR(364CX(U0icc> z?Teom?btVJP(j)|}VpR$O$riONN1LWMr!qC?~o0lazD5kAeF z-I!%RdU>JW!k5||1M*W}wCnn3(2beA=snX+#Ovgf{o;oDcd3PHF&4$`B%AFw9DxU# z7S!j5H}MF3npEmJ2qX(*4%z1o?HeX(2^(-Sxyy9Ffq$7veCtzZxaef%OuXxl$hRJn3=pdgoAPZC*@u-6R=o0E@s10 zoiMVgR-L;T5DCg1Rhv91ej-TW$6*g|959_44G;OzU$@4l)r|#~X@wdq3*4jt7ggb{%DY7QS8=AJwRTZdRdGXS-aw+h{)s`sd|1zT$F|O{W=#a|*BAej z9of33qe|Z(-;^DFCpt29c0HEMbfhCpTu1uI#*N6#svOa+{^*`FUT#FTuIUJWvJsh% zf>0TGW`V0J{BJqkntMFABz%c;ApInxcFS6&J_%wwyhQz+M9|jMA-ZiE`eh zAyk`r5kryV;uqSHExTPLnJLs_u|(e5Qrdq=XZ`62e$B36J8`N1qM?wWUk=A}P?ca? zy}Zo8*coQ7R}JBmu4lNAK$=C0g44yDC<|E#pl^uLsI4tY=9s2-47Zs+P@q^;GId|2 zuS}GO`g-ddr)TFS6m%VmVCpb@mCdXX)wg)I-`Z zz4TqnuG1F1*VS-0I)Scs7@q<&PfHo2VP-|#Q9-4P;$3M7x)qWp>9JZ`C975Qp|G@^ z0tb4}&@!R-rPmw^v5Or*D9s0Wg*&6;4_se(qJ%3dpOC$8h)!HSqEjU-geK;eUS6R+ z=$La!g=i&ph=gWcL(uPcc@XOZvyz#Q+H>82|9EWIyi_W<(XV~OV~W(XJKUel{;WgE zv(Y}5^KH(`sYN@Tb_PRZJ)PkK@I_Rc4yQQwwg&w0$mZprK)KC_JSY$B5~%ZD>qJ{o ztAl0&PvL1WyeR{?%TQ8Zs;_`JB7HgP6OVL6K6ms}l zbF{G->i63~*zOactme%v-$#PyasSq7fkmrk$G{}3vBhQM90iSz8@=yFw_S?AU<=2d zsQKwKX_G{lM<+QGh93rM&_h1(Urhs$zn$|gK~Fp^aU}~CAM)E`US4f}5Z9WR`sNYW zk~b`SCDiz4%%BQrxLbfxlVmmh09tPDBVS{DurDnpAsnKMphI zBUgK)8gI*8HGY`enz&+_JQekMX3zaJsKxCXvGk8Dtqp&GF3dDxPniZ=p%$(j<5+uC z-Mm6;_mmycH!|bRqs65CN51_D^Q0BFXZSs_^0``=RKX zYzPHR!)!mNgwJdUhG0H71%1Hy4ofFrq`x3k=&mMSqBEj#>G|G!j+X5!DfIt{6}7xC zNG|2L3)0P`^$$L#f9cWolr^55uT2;06|hB$DOX$Z%e{23#T}BhZl(Fq`^eo|uVL$G z1>irrPc>X*-@M%Vb-A ztRsHg;$h=Kt0q*KZg$q7?psx!2P3t6>=E}Zny>dbsG5lvgkcubgV9xH1tUr|&&lD~ zHY0c$LB&Ao70I9u7`LndGG-zgwO1hhT?|A;`5NeIC-iG?^g9OkM(LoTHDD&Xu}&2q z7g`(VvnEiFV+GDJfk>MD>mhiy0EaPzNJ`(3Ol^ddhb+SxV!*{kFnO#Q@mp=m&1w(l zTzClBv<7`g=T$pLh-L)6fnsCrX>0%hLy>*eCpR2WbZ%#C2DO&*Fm$3*cxx{92AIeR z6nnRpOl{({-8LYldy%C;T8Ik7oUvXE^mc2(3;=H++Q`zrX(>M_C3QCaTg&T2xpaK@{yLbglDw77U#+cbjMk7*Suu}&QB5XXjYzy zcyzo;p@U1L%*OPJfs4`j`}XDR1rA8(7?l#!!jE`H@7IC~ie22apRIVQ6|k#tuFNJ( zSd+J6S9~O>Id{gV$(3O*qsFskERh1EmH~qvcr;8Eea>1$KQ<@M@YiH{>I}obpWyLo zd%Jv1iQkI6Rd$*E;?zQttg_5&L}BksWx=qRw&cP1G&lRF#T2#W)o=EjhMym- z@qepa(LX*qKKj$a>yL6p|5#6jTKbBAVzvj}YaI8AVweofyp)12_?==5O6G0nk$ogTc}KVg(!ieCnt-cGw% zn&7TQ5``nKx(sZ2yy}woG(+C7Y*`1m^r?93xj2t1_D57UiqV31VU;dkU(0oNdM;-E zp{M5oEcb*_e~4+rK#vA()2dK?oW11Cgp$!KxD>KYA!O8{)(UWSs+%;Yz-1cL}LP!h=@8o_}T@SrNd!1t?Yw{>)mar%`g5Oe~DRldn@b z>c}W^#P=uZTQ$zrGy4R%u1_aJ43)EUU#NjJ6sGj>xq0=g=?zVDl&^f~euSWugGYbC z&_K70!gp5SpTn#yMY0zYm3s*`oOR0ax?ZjDRE?W-b*d|yg+$Y6M?KlD0~1=x2_N$! zbx6{nmVX^)jt-J;(swMgb6BmnC~C+CinkzcsMqeOW$Q$Qt7_W(KpF9gqWVK8sPH^3 zP4Fsb1HpVPtS5^~NoA`@a^iP+@&U{wrZaVSzg$GERR)o$EvO}@kFl_DrnRap12ji)>6MZu+(;-w*LA>KTN8+ zq5tWxo>?`N8Zq@2>rz%&eOMSPkvT?Xjk7bVbH?WZv>p?!;HlrEfu5h!sF+*b-C*gL z#f)btNPFrcp~nje?-X;cZK@0Wk8jhcsv3X2;n)6jUg~^3?1sLnV)&L-dE%lofDn=4ei!C3%Khb%d94%$DLA32ci09}FM`{O| zpS!gpP<`eA235N5Gw|1wb5sCHy9LP!dM;)X zfbe@H5hNIQ8Z_Kx6FYTMr)-tMG)r+KwPB|=#CkKS@^MT9@2O5-ARE3^$_kxh#v4Mb zx@LZR=ZvJZt^Q%|nVO%4n+V*~;!VF_-`C=;ee3UVUljif{Vm?XAU!S)_c|9W<8@(v z7vIA&o)?ex!uJh4Ms05#I9$XVn&ToJiIvD7I*6?|Bmbc0;E`?w5-eH3v3eMiT}4`Y zsL-Qo?j-;}$)7GWeR}bLtn}%{=W}cSw^C~Pz2_^qlt4K>(-)|E;b zjqqrK?*6LCEnpB3ogT9~*i$?^nk@F-;60`agDvq%0-GX z2S{kRx@ILjQ&M}@>EGSG?16)55T$g=^$Xm-v}?MvmNqspUuu`TF|*=&1S4H+&6?o4 z{Um!=@i`A|oF?Vp6uVA@oKztlmIFrd6(MZ-rW9}js#P2)%gdO(o2Ibo%-mzVjw)oo~M_SuMbf2$bLd*6S5_+KkU zX4sXmhg4ra(7WMb5`q#7iSRACk{|KKItQ$bH_*JqI9&uCP=KYqHXpn*BiBm}P79zP zVamyTd07-w2bGP)r-*5Kb&I;wkI&VkFXoeJKJj8z7EUogmQSk@JSHdb^zjp#{PEZc zUH1Cm=m{8Oa4 zIZP&$7M`;j`z}1&Hwm7#2E*$&=77o!{k0=8 z{I}27rZ0)Q9%{hM9LZ#1H9TB*?DX?7XT9$Rjxai*!mfD)ZO3)t+zJsuoJ!vf!n0*L zbq$bN=0)+onVTdwYPUXg(ENCugOGMCAe!$o3h#(9)&RI!4giSJ?HJx%FzIYs2Yz1N zH0WgQ5`v}=2h!z3dmQ5VtewujFSd;1uaQI+=&_VJ@!RS(>68fdn3vQ=rBK|NL`t}E zgrB+3(NT4!h2FZ#MSyl7YB{^Ohu3i)n<&@3X+(+09_LuNiS^Wz<+^R0u60QT8P`*H z!Cl)}z?h(H2n~_78-iK7lIqv(ChB&2h|5S%%p-Fsi}tAtWAHgb=ZJ4Qeng5)y11=|ZrMl&R79TwD=u&ns@p z4SN(&k^UKZJxT98ZikW#M#-#o+S87FBZR=6>PuxtLV$4W>M_8^xrsmZuN{|FkR{|a z9BRGi9vr@CX!Szhr9`X#lFYJ6ECV&rkcAg%dJODrh>T};X1HALCL*m{fNI=@$pvl1 z8|a9^3hj_xnLBzh)bE*T$VNJJnG+Eq@c5^2XiYEn7!l1};)M(Lcl+ISY~!jB{sv(A@?W|mwbYl47~)b(?>H0{|g;V+GLY1racXV&+y{-3)$yV zt|luZ7dV?V25s>c@mPx`qACc32L@F*9vkpz3dpOSHvCA~huC z#If|<-*W*#03>Cnoxaa+b{mVhg|OM@PT^8%u7|;VfT4>a%RN+{}|Q?9K8b8U9|1-n(>unN;+1 zm2XN;T<2w4rFljl@eB&j_1C7#p|SNcskrR1PTsGV`P`~<5j)@c(&}W9 zUM`mKpEZ>A=HkNsUY2oXwNuXGrJb;d-`WZ32m5=O+C!;|I4dtC*j7ol;Z-Q|%OWYu z^O!T>cltqttxT3s)%nzQsSB=F=7mvgoo1QLfYg$IdwI1;3P7T)DpEjzyeKEV@6%b5mC3w!^6XXb)1&|TFaH8o4XgM`aO0^D zFN&(`b&_B7=E+5xC5MOncNnkcQ~v3ndW^WfaCGl)UO(=8_s+cFbaW;T4)Vjp z)X#$;JQxR|fApK5X6sG0b2wB_Bj&Ch>_F#71QJ{@IOv^W;s-mK zI;tmwz`$7#-^@0}v#hM5Y_nVnPI{YMKe>XM(ZN`zsC9dWD_%yYXDVHj%a@D%U7B6i zRnqJ+5zjhh6=T}fvw7rg(m`EyFi)>MT?+_j@@N^CrCW-#RKa+?PO^DxLC#VIVS8U* z0+zRxlzE<>B@Vc?BxiXxI66u)s8rcaR+G8b3T6RS2^IyHuC=_(gQROcp7{FtRmU75 zuN=&h<#Lc;>8TdUC163k)GMhHPCf4|x|uEWGI<3XI`kgr2=jH3T=Dk{_~nTVe)IPa z@UJK2m^kTGPD5J(XbGa#@2k&LFu9DR`PQiOVt8)JJ158RLK2B!*^UUZP(6Xqwf9{*RhMB1;y~N{$rjg4ZuQo}^zkW{Unfm=|u_@%wr$x#?UdFH?@n_wl8tF?K z3$)1%a%0>Kx4_knnkQ*S^{5)vqDoYUs!$E6@GeQ-ewP3bQH0*lZ#{un@-#12F)Z3@ zbu^zJUBlom(xR*$$JgOVq~(vVz6WwIO4!{e2-JrcKv925;sR5z;qOb}F4Fg*=j{|O zCKvH$2}I%sD?QIb?*hnT=}{XGvvdUsFob6rr#xRHJu1VSoscyj!>B@{M!b<3sXW2w z3V#iULzVDs6<;QGK7BsVme)8yp!3PIiyv}CT0~S#%i^o_lGuQU<>MLvkaL+=`6EVT z{G2V~>@uNQ*PrIpf5e2)n+?XJaMmA>8fNT3jC36aHBoHkTc>5Nzgs4UgRaT}!VEe1m%x$YZ_QF1jCTUrgbHxR zw{fva;{rQrm}HTz3w_?~Zmn|aCE4Jomq0G}RfL9Rz4;o>z9{yA$@&Hkyk0tw?s__5p0NgO_2?He&Kt~F?+xcbv5i1 z{Uco<1n$@c7FAsUN7)1ioyfVdEI9&YXh@-N((SUV5-Wk6!!%x_x{)g*A*f5DPjt^Nh;)z!YaSjLJJAep7OD2lSc z*F=c{WiJJ0J3yic?DuJT20VKOf9Q9hH$vUo0)+^fZ0rLPDR@4BBPc@*ui_=hdLUD- z*UM{`P~zf}js(yHA@xZ>vhZrE^E&()%@SpAW#-N*+B5nW% zC7mIxnehZj1dl*$3~#=D_~;)mUp##DB>YM|dVX^9FdTpNm3aK*`zNoSgr9vO9zXy2 zhw!tJc=E&J@WJQs=hc%R!|?<0FP?<=zZBm+KY0>9_(FXDEUFP{AP?D^wx{Pjru`0VfBLD{bb_x#|#c=hnx@N0nuzuN5pA%_>4 zn!!yuEAr*?1)+PWVXUbgBaI`M0oQB_W~l&)dV1vJIkV;Vwk$ISN1>G*9D%e=P1XWv zQ|#pBl?bS?q6DEFH%&StanBvo^k5ujEIZ3QM9qt3{>}^fPRiiACG9<;aWG8&?T@t1 zfTc?m6%P+%DZWnQkVggtW0=2#L*ucOW4Bu+(FQxQ$8@Kg(-dCyz*-L|DKddU) z66xHd9s_d|pt9lA3xQ|80SW;CV)etUNPu3vD3i@RpSGm=Cc3#mNmbIp0a5rEj)4k0 zHjq@Ls0DJ0RiZq7a0W|2j5HnvVYAU*165>WGrf{ z0-J!_(>UvA+iiIqjgYWlcNxPCIGPx^iy@ruN%3(F} zb#nk)bKZt7tYkQd*wp@D6Pd|6qyyHCJYD9o(vDQZTLXDdvrFcz0yqGHfkZ`0d+GDM z0BMK|_vHYoQXt~)S}P#LX=Zc*2qe-SDeofL3R02jsrP+yfg}Fcev_IBiL%w-f_^dpjXX{eHT*fR{^l&Ro{_MdV9KdSO?mzFENVgH zt2~|eMo6fpxlwco?8B0GRoJ>8)s{oh#X_pus-W+!-_yPm+&2x!>04x4VaEao5H6{A zr9tRA^&p_-j~Rvljy4@&tx19)-)9aWnetHpoXT@je}>MtIvWxs z&Gp8iZSKVb)K1jw*JGV(mO!;24XZBHwq7k1LbDHw@K*-l0J(vOS`reb0YK4MSs=$w zj{!nzlMd4)Iyg95m1)vDef@5D);|KGn9O`BWaq^YxZL95C2}MdnmcDUQMPdJG-|@A zut96#(`oCP*G$!9m%(VF)YC?mlXO<3GRG7ob?A>zEBH{LvK6Cias0^_y zL2edt61sfM60)xj-6ZRri`mmAqrTdE0`Rex+FH=`!e<(SoN8b;X6m(J15Z$Qj_Hi0 zf#S9CNXR11ssxiv>)*3B{d*c^x9HzfNB^Fh`gc^S`(>?v&${*RrPRNdTK`^7GU&aj zf8WV{vtDlC1c`2dbTm$XX#*nW{IoiQmCCCeF`cB23?+^-h-DMyX?7NwAKUFIoGM1b zaJ>Nu;S`#N59A^c$LmfL(dbsytM@Ek1Z6lWwwF0r;X9UhKR66?q%Thl8TdO)j~ z0jsFE&f{#7_4`3G%vk|=8pH6QHb_wPYkVd*pKcoWCiiz^^ad2q7>wu$M79wlM~D|Dg`Oh9X(SVSPEEkR5u*UJVA>WEog$>E-6Hf3%*{S8($!z252c^2V6 zBs<3R6R-tX7x^nP(aWYBZOSWTcpeRBT_VOW@z~hPtRay77@0cibCOJq*lhLuQLgVc;b{MA6ptc zYg#^48D%i_7ZzaD+0Z1gX^qpk{o9WqSlR#c*_WI?p?iqG`I%MR<$)nz|N_uVm1!3Bf$}~T(Nz3rOz`rmkNX&P_ z0im_!?eliSJds{YF$NF-{mmnF{7&-Ym`Ej7O!0Wikb0I8{y9y76v5x&A}*i5%U%>Y zEP8cKCTqF6?v!RUz zfJ7`XncF$zdU52zBB1=KnbQnF)ds|ZIa9@(ceAi59m-68aa-xB03!)3Vm2e*C7b#M z>?3r|h^H3q!Q&L5?Y%+04{$MyxOiCkBcMoWgn4qy#)ujz6p4+MNwL-R@GzCqJZvgN zt(O%#SfoS5V~`b6>Q-%D*d0XysNiLJuiVTQy;V{z@_Fx`*U!+t0cg^LbE)r1$E}{8 zhEaDGXIWnL5Nxu3*~>FoZ(XF&MwLTNVC>>$`hiSd6Ug@Bs!CStDw$)xQC)S9f?nw7 z`nd5km!3#?kf95~1Ed1ZZ*+W-bgW&@aFz(pwn`oqs5QBPZCs&yLt4QR$3GBsRxLs2 z4=@?5SlB^_#SYQ+XNNxsZtB#Q?T=P{on6z-8-uiY{M!>v%ISH^5-Y`$kPKGGp2(H0 z*08w=KB>trxD(d79oqW}_zxdAY&M2j9dIIf8JRk4EH6R6*`#w|X!hY5q=%Nje|d zatZVQOOtnM6^vYF+JC0n3eoqlL1w<3JQR3Dt1nIZ2v!9cMhgXK!yZBDo-mw`bTfpL zqytgok-G-a%+k%Xi?xqD@tb9}F z(pEMw5zY_p&@HR9<06e~KP20l*U}Hj392wsP;JYrTd+6LzHvC4>lQ_c=YVRB>;}`~bt#*=Xg)N7?AmJ*iP|QWurYnpDnw8&+|D>^KocmP(sX0n*r6rsZ zms~E=iwc!as;3*4k8@-yZs`t+5DPX@l?P+VfMus3MxUKU@-InenffBN3Us~B*U<5J z&vdn_7g?MsZp{#XO^D-3cgeC$P``4gCY*tDYfj=M=Z0-DBdHV}$r(`q0qR7Hp3(tK zLOe1Py(Dm&fF(zHC@=*Xv$e2WTvek?&Z0_qdPkt=C^D_q_N?;D%jKQ6nI{eokT$9& z{)+t)D5$YYuWOr_fHu!{RL#rJkE}Yvs&AYfXKI*;@Ab~}991BOPM`E`sHdm|x}L)B zKo&JK*Z_Tu+w=$(@Tjv^)y?xai>yUzliEw*d~54dVa#&7jTHuH*JWS}4o2*COcn(Y zY}^&73Fov)7xHO*nxBPvp!x1vPL@3MPF2#d$pJRS@DjP8-2AiM7V>XGe}M>)hHL>F zeML&Aqu>0~*YCo?*{8wl@>75KY4FKWAePbMbbQv29r7(UkbeJcvYBSJv&u#+MS*fA z5qKx=j@%9(k4sjY)h*&okYv{+94Z;-@3w}=ISR?q$OdZR^L+Uu;giqg@)Y%xKFY) ziZ^;#mVVutf-?c!B9+0W0%J;M1?V^_tacJ~H<0eU#HH@@^Vtb%1vc_ka$75VN!yNa zs>Q&qW>znyvyd$_qs%n>1JlyH2K@lA|FKD_W@u89+3g~S$bdNKAsnt{RMPQ{N?T%O z5&zR6$#H(A4%UY7HkB5~Mn=C196NKWgb_fg2=XnCHP5WE-e04OK7LY?aJ$AMO+} zCs*UzuNd8a`Rnu9pWHwF1Y{+$BE@6eXAzIvKQEp%R@S?skj}8!ePgd*dTI#^EKx9L8gDr{$5$pIG z?Gg0DmkCh$@Gy~Gm&Q-|_p~^hBEms~VS(ZwdJk+#q zVuq@uW$2L)q)@k#q<6S<$Cjl%f;%Thwa8>3(?sQs!g;_RfJ`C z)t<53$z!PDqr;PIX0}XcZ}&A(`w)%j@)GMMNkm{77O7pdV3F zpb{cqKU=Mmd5R8CMZS*Nj~*7i*8eH>1sSgsT0#lY#~nhpBUHC&a};%_W=?SMk||Ki z6Jk2uVFW|N+d>?mmCM}3mEMYuCbM-nA}u!+ zO&?XcYYVhn6zg(GsoM9zzM6o`x}M&se`1^du}vyk)&O?m2p1winN{uLlicBCl{bp9 zgDFw^KLLUq=xM1RC$O=$+pmvXGw{kPK!DC8%MZrb`^W+w$bH>aM(Rs^S!}!2joL^n ziOwi!p*mA*n-%dT(VYgU09XIAozp_(gxXFm18$BtbO95zZbP-eKdsyR%>ExZM_Lbs zHW(scY70Ue0A(&xiH|2PqJb(oLzQ|q&hr`#!j}Di5?71iDt_ z3I7em?S+4lg?|ZxV@q-xDrT-`I2akb^jB6PRf&LX*3hj1wzZ`rc-qqCIoRy{fZV{0 zWFrUwd1I-9r>5^Q^3tFBYxZcTuW5BO9*x3rAgpp@s~mhyQ(5lJx>{fN({t{aV-~u% zZMP%y_IrJU!f-tL8f{qG&1;G#M&J*KoviR)%7R^vt0DhD|K^JeIK;yn1lvd~lZy(N z?p=s74qKmq$^F=Nsv+OTctyHxYEvbl12$UK!qlN^s*jOsdt^14GLms){P8bnMGwsI zbvH(5cvaU!gk<{RyVq_ewF*G(77!)X77z>mSzz1>v5byzDBBrnY_=5M;z(?wBN(ho zJSVU6BiN`%r?3032WPM6egF0F^!5C#A58yydL(Ai(QmKc^^e3lI{NQVj>JVoo~+74 za+2tuk-SH99!rmMlif00IrR~iO{6!GK-C=IaU`>4bPqP>8ilL#Wj=d5Q9Xpp6?4Kc z{d}~3Kar56-|8ky$X9PB_m0g*Wl2D7D9SD1y_|`{Vi$g;T+Hs>;t`_^vm(JB2;W1cHAaaUIbFQC8bdv^TIWnk17TS53Tl4W!MKwrw z0x0lHYUx>ak+)q_()$4q_Ow(+->K&TLMfT4apQ-KO&?$Lnr?fjxd3KBnZHR_SOljh zZPdIGzo|?=PcF@qkIl-ZQMts*yjE?sd9TP7k(&$sS9JIu8w>rQ-mlsIVmpHI&nb}l ztOLd%q-?irFh{_dB#2ZOS#)rKzYY(H%;BgNrBfr86ADwI>V~DLVIF3R2&9?_z|Lo? zCx`^@L`|rmlu5OOrfNEaDICKIz?@A}MIy0>7MdgrDYB_bDH^rg@ePc6-a8%iLu}n`l8&Buu)~;75o~9!m1HueV^sSjrO}cP zaKgiBs#Kz<9#eYnEC@Gdi8@0SO@mM;in^M>@=oh`XIK|afn|tv+r&P=zjb~R&H0K4 zZ;`70gtFol&o4SnrpTipj!!H4fo^~@Z)P-rI^UVm0(yY&*gXBVhR0hR)LZJ%qmz## z8O7N2^m>R(*N;|ukaZ^1)!9e)1HAA>7q3)_F;<5~??;!7+GDQGxmbA}RsJi;L7WbR zYIluOkj2*MdCf|#79X)vEaAYG%2GI_H}-~|sRYeGTtul1O~g0O=rUO#&Yk2}gtU*N zdH?&TcFYlo(@mfa@TdHENH!Hs@mH6S*9Za5+wRIUeD1#M`axCR)fJ0T-Ze z_!}Vczo|hSUI-1?@O)SA?j>L5Ln7f`UgTRHl3bm>J+1H}Lrdamdgh;q8y3fyv04fK zlDG3N1)eFKHK*S)Vjt<*Voxf7~}uDao>Cw0{mqv~8$o$bUTFVYWrR>jM3bev9k zCBy1?G?ge@6F6Xrs&qyL<0-9mSYkPeoT0Y@ZgiP=yUF}`b$GZsF8r!bmxUI?x|>L~ z2mAUMAa@DH%6`nnR2PwI3r_1{x>{qDyR3?Gzd$uifj$k#_)+3V90XIZ?f?d&n(@>E znb-Nm0#|OMxDLIe0r{wGxZDNu{-uGW(al8)04A{7)@|t#j>=}j5MYi<%`Z($gKr5- z*(omzvfoU7iYQ@JsfGHE_`eK@ei{s@q>&FAPa)?=@sjeO__17R;s+MwJCKa+H)O?!y zfscSCQXdUU-bAW`qSOdc++^JVg+|)p$<)CGD5o|disW?BUR`jm0w~heTi_`PlL9E$ zmhdEkzLf1CG%=yzKG5KcY~DWnMDa>N6~iDJnApz@B&GWcjFteN8u(_J(5emqyc_y% zqUB&A&|SHKfH`E$nAPN%4}3@tSTJHdtItrCZRU=E3o+_52c*<{%?3DsU>6txV==h@ zsZvXzL_9j)7+Yu7Ujr-AGWuBfKt8wI!ruhPDc?lq{XkWMAe`gU`{mt0#T@}me4wz~ z?Hp-j(9-IjJ=C$@h*Md;k7%{!o@rff+-qiKTI}0*R_jvBts&PM4-mDMz>Tbtl(U$xGH*1VK!Uh?WNvx`;q zlBah87kW2hAKL$Xy`{d zS_a}0EhwhMv$vO|pbWhXBSr2+L4GF8Y?R_gYy0Y(*UYtfgYgJ;cOHRVf9*K9QqI?$ zE57+~nx91_{4=Sfs_qf$^ZW%W;fgv+jW*ysQ8$K%X;uQ4^esiaLr#52(G96OGF{X< z>cTLrV5Dz~hdr&E?9pI*FphL_i2CE_ey;q?ZP*l+{JYYLEy7j#J5 z5wdTI2aCz^ACG&>w0GPK2_6H|j|%uiN5>@|b6Tqc9@psc%U$wfq=CUo3F4odjYOxuwo7do0ihm3nbL*yZcS)Rm57GbRn83qRF zx}CyhRiZ>~{@PKf!H)hgtF=GQF#EGoRDYbP`lBy>U8>#BNZGa6-w3r<7_u?aU*Ex) z>&r_@uWLCqY8>bR48<_CX-QrJLj#demCS2Smf6EYzACR0 z(O*OUjI{M|UF5T*EGcRu-Ub>St1@WUlt25@QPUKpxTf@p*LFFb}#(s_Vut7?0Net?`OFJ9fPNtdVz5brvV?U z8P_qk(b7jv#W%12xT;m#A+NLJy?emCK1DmhflBvuxJ0jQ3Jm35eJ_Aj&USRpI=fL^ zUCOG zU=1*utI!_VR8E{ZTpP+psYYTaFg=a||jI9!osqET@g4eHhlN*+& zR&%F*)*lD|U7k(O0s3zzvLuD|l)ry^(X2BV_seE24(usEIA|!ZEMc7rs8a=YEea*- z^e}Q-#z3S_k#aCT+r@Nd*KV(|P3FX4jPcpxW(I}^)Iq~qT;-5~Af5x(s^Jo>!Ay>y z^#TH@&`vIwm;zP0H%+i3UxzCLIg5FtyYP+;PE7M^%*$Dvp1j6z(1K9Yn#ohndGZq@ zx`kG&)5eZ&;lc149U3gF8056ZT@a%@`#Uv9r2OAh>V=L<)}Z=#n}R)P?ZYNY!$#wKrTtxq$4oK2KfxLGVaF|+Idz!&<8c75PDd|($Nvxi$zd=Yowdd7D& z-;7p%*HhoqA$2{HPltOB`~V80Fy|Y$Yw2K66F`v z3iaKS&=Db_SVxo;Y5hhAr;TQkWPJ?9OJ)%s5t&6=3!xsUg=G~PdRoP;oEb%Q)Khha zOH1yu{2C$js$i`fhlR#_W=p)18Qn4z9{WxbLdW(J-Suhni9rc(P-tw&0}Vu z@{GW~e$aPn1QgoabxkA<9UM=l^Bn+qbf+;@+6NWGB^cpyO`)Xjf2V&}L@Bj4uf>(o zC3F>eVX6Q-^$|~I1QjAkr-~L%Pd7%xdc5e#HtR zVIpqPJk$ugOXj4fax!5&*G2f7wAeS$=DS5_@niTva+p8teFUv zEtyVo!^-yZB>xevr=C;xC+5`Plv^h&vX5k)hzjx2wR(Ijgj2gf>HMwZUnSzT0*Sr0 zg`g%3%1RK>)m@)&y9wKzkP5448hScO& zmE~0qC%6SDcjoo*e3te>JMgm;2n+4h1@H?C^>pQV{vMxx#XH4;bbygAxd)Z}jiG0l zVg=t+>Q^8(Q5zmn*n@erJUrYC@vPZyH)3YSHOw2xlj_K|jUyWG$Hmbc7)tlv^PB=H z%u$4ZKISrh0raLL8ALIrs}Kj6qr)AjnN2m{O>Pp5;f{-LrDi5iJ!P;Ibr6SGaGyZD z@fkEnmy3euJL94cUY4bx2fVZF=CysYy?GT9BAhjP!tgxJo`mAyUiZ24lKVfxgA;lt!OBfr5AF@mSB78~MT@zX3r%%lbXSRwKE=7@a^XcEm6G zpevZnbJhVD>PQo{PQhF^9MQg)_PnNMAFD`f2D?|X4nuiIwE9g8H74xqXxh5U`7;E zR>b7t5})R+bMzEEr;PKEOzkRa0#OQT-I{WuaO&waWw+!(;!(O|s?~^5#$n+sMb9nP zjczQGof?xvuQBMlMZ2J1Ce^P2T5`VXMBLC`wYId>+#$w3&(y6&3I^aODzt++26ZSo z^pzu>b%oKjT{8^JIZqj?&F~3IBCs{pl*nB_=k8+CZREOV(P#X+DDte`+hf18!+vA` z9lb8P9S9nsdTaLqORj33EcGsK2alb%-@#$uDE7wgy9?|(y(;2zov$}*(;w&-+W|0d z{2G>;qhwZ?M?n&;FwNKc*bd0e)2noDOu+%3%x+@@I*j((E1l%2gHe}x-Ln0vM=ve9 z`y$5Y8Z3i%vl5x*sTwLGZmUA(a^CO+9gloJQQG9cd#a)p-K@E!x25HuJ@wpK?Hisw znSCY6$G{>@`%1GTgLSALm?vslwzD0I&r)2$9^jzALNlG68l>i=cFgH+hu=u;n3!bY z!yMUJLqSv@(s-*F1m2e<3lD+TtsX7n0-HAQkGwualT%;?2L;(esTplEP;wMzQQ6N# zt;EMt$a6o#BNCOADJo#;Io`Bs0arTdhO;G}B`yWSOY~Z);NLot?lINcps5k|mE)SU zz%7zmRRaW3$9XPaVrgk)C&tMGybaM^B1g*9rm2O^=eQzOGScWV7qGm2ZSl~#c_QDu$jP> z$?w7}DE2_9UiQgpZF6?R5^De%{=-?Y_c%nA@5ELzsv}p)fy(ii z;-7Sy&kgfC9jj4XE1Kmh@kF}VX<>+XCU3I;urK)qbUKJR+L#TJ4wg@!?7Q}UMX!Yk zXn{2k(RD(wbjdY>yuso>!l%#OD$HyZYHyd!^=Wl?A1Qfep?E^1MGdIMRi5=b_%BUT zpx6I-{yy|%Y}DOp1L@E9_@r9yntQp`78mtSx?(x4tF(rH=U$@n@$Q|5AM37LpcK;i zieR0H1U(B#iYM1T)I;m$WaN@6$G2B>WKZP=#8;ANNnIjX&JH}6v9UdtQta7L;UpRp;YZVf35N=s3eCvM$%(j{8)fAEJHAc zqu)q!l`n(Kby9A`6tng@kV0TPbi8>}%~%s@M>8w+H;K4GP=pnU!}hv&%L$^8Zkj-u zIm(S6kSj#TC zwG#jg!@!KXG&`l3IA<|{ode}Fg7#3;bg08_-hCS>Z4r#Yh}6mfbMc)ANN>~U>JTyc zg*7Xi{3xS9Fxe0I_QL{b3VGqRX+3Q=&9R@VV^kUKx|K0;i}I#sjO6Ii2q_FTnDg+( z{;BO<3HjLI)C58ZyiXma*@?*91*kwj2Sm@k+H9G*-pXNz&aJ9ReP0oZPJ9fR5aB~R9hiUwr*;V&0?qAO9Qr7^lgjjO3^9`yVfy` zj#Ha^D+k%GVSkg0pc3v5M*|d$7NJCgGjd7Qml3I;RPfd};7qOe$HQFjx4#JJojdbZ zpt~UNXUC)e7|<1oydAtXG;Vn<(dNj%v9i!T$oI^!t$12WSB-`()V$=PHs;;~Q^Hg} zZ_(tIO;Xfh#&5Hk8qv%(OU`5Vh|ZB;xTQdDb#-s{(&9MT<@ACt=`?*UxDdHqy)sTZ{1?`?})E00W^>-0n{HPcX} zEA5IXISA(s9*!`wsdD8qb|XL)8;ajxAU{CaH$hgHv zsP+(|FL~puj$)|e%9Q!0n8{!vvqhX;s^7$(x*kL%wa~93Dc3OAM0pm)#ca{yWmF{p z-T?Td?~TBDp+q+7jL{)v^eI~*r{7d@UfYTzPd!CMF)Tqc<`;_SYa$8R34lBOKF!eG z`(PB-uHp6`Ov^{C?tC&&F=xHKD>GT>or-rPbW56GSASY$R?Uodb&47bb>z%+lJEw4eYQTql*SmHk^{==kg zQn4`%{z>sNGe5y@8r75mHggQHacyXqGB4FmF#X{B>beC z?y6bC?V+y2EUH`HxHFB!!cb&j8(H)ALS0l^fwI79^3}s{TN-Ss=ZwU#Vp2KFjY{9- zQoOu@^y>ZuVX>5>lVT;u%AyE8eb%^h?U+c@8g9Jbm294&e-VHZ8Ym4qUosjRs5TbT z4XQc3J#z|>C2c@jqAGe;Nwzv`MbXkh(ad%XXvNOb0)FOt1vC)TEw3AI!%4TW(XNY* z9Mp^`L)%**C9+@`imD4b`hgjHy%K|maTM{A%~0O(cyb%+H#LdK#HglD(B^^ok*jZ^ z$kaE@xOI9x+NR2fM-FF(g|FHOz(DoA)Y_@plx7iT5Y%LUBkU$#ZnE?yU6;M32(@S7@ z_WK!nhQPYic50iSpd}LrV|5?2VOFW6@*LzakVYwD> zbgYo21ihL`40Y)`@h;&!1U4UFV!E!$3sLd~6x_YUftT(Et!tZ+V8~0tk*G``);qw_ zaSgIOE&yE>kUKq3T_0r~>cpa}ZemM&FGr+EXLzPAxx zr)x!YYw#Q17J_M{hZ^44_>w#f8U0Te~l!zO?Q@Y8Pc|32NhOLgaCW9VyI_lnjpmhdVYZDH- zM__QCc8h*vY0ThnU493WZ7?EFv7l(W2G}R8rvPsjvBuMVlr~)tWc~MxqW(3E&+|h1 zb}0#Ye;h$oOp|@$+^ga5Ae0u@czEb1wi5m1ou<{SL(rO091|T(zWunfeOyYup}wG0 zmv6yEv|?x=b~}kp&h5s8f)BHO~3IdTxa zrNoiTh#%spR`_$3RI&V7F47AuF#$;n5F9Vl!+&L9U-M~j_B?!FUVsZ)-+A3BHBVdE zydInB49MkbhStja9@9Pt=efnhM2wI@G_%yH?7>E!Zwu#{-A#D0Hb?qXrz$8k1-cr=u6O z_0iYbeuzgMG5_*6^*Y7)F{^kDhe1ogn8t}Sy`dK19H81^g|AxH4=I5ZBlEc@Jk?pq zkj426lPRC&M1>w*fm5udNmlFXT7?+{jE8+jb`3a7tIh?Y7V5iWPACUR1?W*Dn5R8r zn*Kj>5M2&Hj$fiS7MKhe-xgrfBYW+LhG$M2m=5DvDjmfrkuT$6*dp`Us=jj@*vyl3 zPr9f=`MJgRG4R$WL=q;4$5*N^d`!%ylDsCT8vv_sB)nsNf8^e(3U> zWvXS-l<~`yK!P?e0Oz&bZZa{g3s}Zjl+D$CRxT}psYW%;RlrM_I!nV>||-m1^ee6I!Y2EW#huMC%un@!@vS z{;W5f1m#9IJ4+ZFz4R7tE9oS$?Y3sK%ZRGmYNJNVE`s| zPMX$GNox~DU6b2Qp)HWmCeIb{^Dz+Q@FvTDOiF-b8G1kDJw~No!AV0;$X~5+>-)*^ zc4*F8JYO~awqcaYu0 z!MaFSanV{~P$uy2yd@XA@Wd)=u;%*SOj0Q^(9{x1Z^=+M2cYC1$m0?ZfE#rT|6N8c zEXOdz2kbFM=LK+96rc^{R<$Ad`Qx-E3-@^U-=pV+%PD%F$`PDmL(x62pZ2|bXQ#+R zMFnu}`I(1d_V(qY#`_(tiE_~F_K9H1i6cdwW44=En_dFoE}jou)O3AN8-!g6a7{vB zsF=Z{BpFFXuCDSpkju$-nC{pn4UG-Xfi5DQ6ZH7m0+zyvrlZS2xdxcSa^6xxtmvuD ze3`_VchIUYQIngtrbf=)JT}?aSDV^g7K~dqZB5o5oOqza5%(j} zdgmp1V%=$}uz>U-ViiU+P%j7$$?UC+Zp1tU^s?Q0gk4r5;jHI{4KLvdE+0SH%QA!F zsSKwM-O99OxNsIJ2WdNFQLUD>L#q>U(JHFkxNR^frNG(=a$9e9!j3u^NIQxc_?S9p z0ZeVCqM+=(^0Tu#qp%$x)tTzTl=QQ@ResNCe}jqHB0V$KFjgiM8}Hd_mCREZy%A2v zuLHqSjGT4@bO-z!_e5OVb8GdzH%;(7E1j!c5)>>7-v->OmhyJ^LPOg=XhYV!!@Kr? ztVzhhSgE*zuw{?^*$&eSzwS(Wa2$*$cfe0LM%riy*>=c5#JCAEk)oezh`z+b4DT4g zA5;=Fp`Zq#4p=^W=<&AQZTAq`MTWvn$w4c)6X9$dbO)S3OrZN^ zx54mk(Au@0DJWUSIvj?bMD1WH!)5Thm3#4+qBt$5X`q-o7^(iA$CKSFf|cTnC|4rJ;(!&MjI zf+0HDBX1OW#CeTq8Ur&p(Ub;NeN z{ax)Fqe7Ne8%VdQT`_@{u7nl=*;SFYqW0dCqgkBUT$@~L&whJ4wzhP?uF0x-=MYWqM%GrX1#JfG@WTT`j?r;gC5?c(kB(T#l^X4u)_ zpW_qT(av@9awhk2&Dbx&YL7+wW4$OV8Fh&{rObDPv-YlfOw+$@s zp3DWFG}@i)n7rd&cGu598r_12Uu6qM!8oem3<=FVxLVJ*CeChNo6vdI5)cm&2)a$RUPR zk76E2_Z&T{p&6CIj&vKq&u9Q+)4C@L5VXF%ZA-JNTi2;}`wgf7pCZ8m^KVlLv(m^_ z3VY3S&C*;6>1zs%1F#MqfL9IGSQHO`&(n-v6lt0^xGE|G_q`)6V>)wT(W8yTj4mgg zS!o!FC#~!>+Lm^b=BVw0#5s#zEP5fCHue01_9v0MS-yI1fpk!7)H9!1>x^C07EegF zt?980izJfvI6<8!qgPZ*O_H=;m!BXl5-Qu|P&;F2q0_PscxEW^Co;%Yn_?gLM?=SN zvVYjS@6ZtdJ=p~*bMYE_1xm2XSZj>evqxumE*2Ps7lyqRW(BY z{sG?Fz&;C_|4CSOK#vEmT9O0zYLl1T$ChnZ+`w*2)*yB#Mc5z(J9+EkYN9LQMXS8Z zNfUkpL>KBkQnBen4`|q9-JT~_@%tb1cV+mQc*7H`GrMG)`TB8sg}!=99ouRwEqrAf z;4*?9kzk0UgKbp-!`9ZB{k+A5-j!>=oS8+i7q95SEa1nH=+keEFFsci>1LD8`_Y)v zptw@UU~-^>Aic?wch9nt-T?{lF}fHIl&V6RI5ItfQl3H0u9PYeR-R=*66)E&B;p{< zYCmQWuoEX6y_~q;c6SwYXG-U_P0m04?cw0pc<^B~`1;N3!CAmIy4SBi8Eez}O*#)t zd7rBi%B$*ed>xMT*GZmL3*-BjBrf1{Uc~QI7XFsWJalxG#Z?c}Y3xgDt~#J4MCBw2 z(^v$p(e}0gj#AnsC)kVhy=}h51D%b>%72>h504xd?5%*LSgfgJF4*pg9?0yECcn#C zdZ8|yT~miAV6-L6yH8JZ84s_w)*MRBrP>X-e4dLkh)*_iUSL8X)pJJrPiu8{p;7!L zIyOgBnS^2+q8 z?j$h>u*l!lATaO&$0>i-11;^jVddv|`1^I_KqNrtd9|S5blXWrg=p>0S31R(nto>Y z#w8_`qKE4(9WBXw6XzMMTpXs4^oBkPtT*O6^dk_wwy7PBxG@v_2)%#h>7D_v;q zmj?D8F^Kp01NW=4yWi3}2~Dv*nmrwoGh)Yz%Z3doNL6*qfj+Fa3v3Ae8{gG}J%irA zW&vbTo*_j=W)O(-)Xqj17h^Z!i#LC)`2*R!zrTH9XWOXdlbq^~p{BC1#F(= zKrQhG_0j3$ld~i8eT&Rdz#{u;gy91)!r^G5blJz*Bwq(G%i`?mnY zK}R~BZ+>OXO%LMao0u}X^0L!3UvUVgj**rz}*>{J-?bcIFismUFzI%=y zt;>7N56)>2NnVkY10%f4W{pi4_?g-{4^dReWVMex*@PUh65LAQrLNJfVwun0vRZ1M z#>@P&1CET@49#3D#8Pa;Tv}0wvMO?L3ejdR+nGB&r?6`xRUzt!za$k!Wx>53aI5#| zu)EhQ@qw=Bo{quNQ!nFAFbGT78;nPz_4`SW+@`}G{=Rpl)gNYGoy-V&=;4hqeZA3d z)D2pu?Xb^{IzVKgTFp!Y53LZ#ElP1sOh^I<(}uK~PW`OGgl8@6ZZpVt<2<+vSFH;v=MCYK9kT?jrMYpVcA zMM3B*9YAX608$F+a{Dx-MqXTDG$#P^6*;}oxsYV`Y0@6vj@LErTG#APt*fY z7JUHXl;IWLZb+(ag`$*ZW>!%QEnUKZm@?68&AU$Gf*wY2jJe}F)~HORU5)A;x+m6> z)pj`nonxvgqDUyiKsT6<|Ci8Y^e50{bSE?!4L`5ZL~8o})7Y~2p~U|@s*L^$RB3JT zv@KDxA!iZY?ZODeoZE(BET|nMwfiJWOl-awP4={YTumwvptO|djtgID*R75ws(Ba? zF!GMZ5-_K@wnXa&e(z=^8KsD1MgGn^NlSeBwxo>6s2fy}X^P6l0u z=AwZ7$Z=_bgIQ871Kx)@>02}U0%_wmXgXDRT=I@G6~K;YFUtkta_ZqdlScWaW^~kL zV3ZY?$d(djV2B-hTDdt4QUpOxcAB0=uqBptZ0Aiuymu5STX<7PVCab++rkwG)-5k3gW1$= zs6_QJRL#LJMIPw(7k6r(2kW%YiEZsKtl%JN)ZiT=>S+Z zw75%GVqSu{*&+TeJ0<3&8H-3>(QYLe93q-qarc$XB~J}9Hg=F4!=*xRvxMV)w~wxp zHe4GMyC3^BTTM0E8!ClAqB#hdc>R$v)R&|VQmOhfe3r1ar$xScnkLIRk4`=dp!)OW zc1LzWvA$ieibOVv(PT^o?xNQTMG?hPE>cg783!BMtpf~O7}xRsu+lAh$)MxO!>_|z z^pW_mof&X&{2-JNqZm!q9L{*R@9XDRlE3N8_rKgS^0iu7Ch$b8M6?WKF;|JoCp_xY z6r@N`XBfPUZ?tStE9Ca$!rPC9r=(I;s`DfAW+8)q#HeB7`@u0Q@uu=GG|y^-(^AAu zm23lw07HV_{V#7RX@dH}1E+92|9w+R?W6)kr2@H32d$IEC+ELSF9}NMlh|EK6%pQv zXOH8vj~Yk+PUGk<)jf{SLtl4;PC7DfP2l1SSsJb3(V5s%)oT8WJ2rorSNW@aZlL3x zn*Z{S&9_0fn2zyLGw-q2)s-7?P)7Q>OXL54pwL%;V)BD8_f6dD=IcKNtXzt zZI91Je`4}?0kQek=`LB)mZ7kl@<>6usG8+|6BT;gP%{&>61N2_rTc%*wLf7fj=BW# z{paiCzgmri2;EG0taOgk?3uhH;=2dPI%hw9eE78VTN^P=U}#%zdoGplsp7+EoG{fO z4C}-|v@He5+dc2N+-?s*w2)7`KELqG;259k&{3F0ZM}oROn4JHexjZQrZ9!vf-)K0iUFe%3SGDo?WDPsgG`i_%37MFleP~Ky_Kg zjsga9LQhhn&4Qbql9G|fwUF2XEsENaf*`DxQre)=w?nT(=FmhrM+`VBlt@#znFfRE z7H}A&_Kp-U76fg3pPVDemjL@9|B7sysemuy6y%|^AOtBqrWkNA9gs0<(U-VCiT6

6d!+(AG6tzxQ^x?)ela;Gh_KNk8XjVp7;UdDkeq7FJuJ zrWCn0oq1tCmC$S*I=!eN2#Sj#{vmrBO=W@a)X*orV9Qb};Ks5Y>5)=B%u#IGZn%0{81goyc0B@WW_9wR^K*mqkyq1Unr{iwPp;!m@_M`5#N1E)R)rY;zO)?EE8-le53@NKe|PRzDyU2UaZ8SmU0>0d-e_o(l1T;6)lHE@?-)3$N98lpX;M29+Q zEYjbiJ)}@djJZ+E`7rUQ(}1FH0Z1?9pHuX&svF|xef1}Y6c8hRLDiX8d!#a&CW3_L z>&Ce}urg}9kej2Ud6;Co-IAd9kQrbQHR@)ve*6~YA?8@sl9<)P5$(yA0;&NhaW`j@ zzF$s9VdBqq^rYGW1TWs+(buG_8G84()^*bHGKg9?LN>_JF`Yl zPrJrUjo|ZY+y?w=YXZ-;qNNQQurUsfADL!8C&=`EcDy(|TpVXp%7;VMR~dr^R1ee; zjOzHj9R1smk6|k;kK?*>+%LEmy+1~Lm30y*1Dr*)75i*@_LjAeG~F_mFQ^g(_=53R z%LcFL8x2cqWymduTmesyNdOGyX*ShQz`W!O{qpTb^>wBfm#onm!s1jz4Cah)#oL^9 z_XF6GWssU2r3~hEcPLW|hvbna}cb4BK6^ zAAW=XNY4nkoCsgRG?!bQ!q=rO>`3miBWMQ;WwleUTp|SJlj-S{zkgce+lui#d8U%_ z<2i+xz>d(3c`Bbl%L2S?lIdu!Ej-oOat6;)Y5j--^#ZNJ%b$Fr=aTm1u2$N~kaD)? ziCjB;)#8ic|7OZ1t!`nh+&A&8-pF9P71yo(vGA@_rd+bR91__Cpc$sxSquZ;8H;WQ zi*Dg%nr>v;&&WCEW-;(dbMzHZhsTrmq$^l99~cHHzxB-5k8d0L;4U>wiqh(Y{tVOk zfy+g#Vx08Vv-#oSh$5NkC$|6QnoxAn`URAOwDvTy3O7AYplJQ>fZMd~#;;nxm9g<% zm4#ft?e;)pLV#MUyAHBT^H4VOTr&#t^tGC^gAXiuOtCLnc^*oy2Md8ceB z&2Nw6YjhekE|NuNp2&v!+E|p?2DiE(+uyh-+TM(k@>3SXuONh@K;0w|cGu}pSyY3) zJySXM^$+&+OU0f7X~VBNcwd%R>BaRUe)lWWvRSPXFWH`rEm0?QQ|_Y?C_eSnXuKLD zJZce6gkQ)#=`4a`IR1S_NWJ3v zC2}~H;5whaO6`xalvfMY%f6P~SC~T(b{E2Gkzd4boD9adN7O6!N( zATq}`jMy$+24OeW6uZ~OEb|CznQ#d^Lu(lqXIr zR3UwF36D&&j#xA8ea(^BLfXiY7fRjETR#}@;>gT z5-*G)4SBSwKw4wXX8kxKdmp-`#3oYSgc{SAQ}i{IfDi0r@4xZBhdx4}`fI{xSZ!3*Vi1YpNoh*ZQzd_pBW6QN{AsB65lJW_cq{(I5_(4 zb=jBitmJ`0l_&43B0kz=v|HyrT*P>WD#G3co^`!6BaiwJkS53dGZ+KfucoW0>qH3Q zL^xhS_p5$qnZe%U#NQl452FwjJ+MO8=z4;fdKn>vt|t@P1D2Eeb@?n~Fs{TU6c_Pv zbQLg&@C~)AK@=l=RGbMoLzbw|ei>f@U3*A^iyjlVA+GQWsXaW{5UcIhUNlI%lxe(f zkZ|Rk@CZRq5td>n9=>{h0`&WOuv!h~b5A^a{`?=$o`l|G(VNeEC%x-yPdt11{0*`~ zX1O1qK6>-j{rg-H%gLnA;WT(ml?NvSnK&Lj_`)qX9^ZdZr^&?oR*RVU@}*hv)w4(c zc!>uO{5pB^^3}tW7a{B&@$-X6=GaV(568)upMEjI|G)p)r~a4YufF)|^S;4MefIVJ z;pf4?NEv@U1Ww8j{Jj|g(8kERnB#H!=%w$vslTk90v5Z_8^1%oEWre#N-zKEJHc5}w0X7LB ze*_$Z5f5SGw1bSU;sWpyC%gp|rtr@WxIy4rXLwwo7ZWvJn$Z}_Ny$gfOddJ)Gyon8 zrh4BjW4R|xIY3j4%AxF) zsY5cl4{EVv&{=sTMn*X$Xov^yvB*POc~)<5irjY(N}l40T@b|u_t=Dj3pzKa&X_`f zJUXVdnodsvDMHT+`cArY=FZVWNYBPOJ~7Hc^4X_Gfqvuobk4yVBk*h1JPK`Gksfxv zIf&nudu6M538-2kM{n4M>S!4H`kS@cD&B0V-W|Y@!)7a)yQ)l7*Z1a17uE5p_>3o( z_RZw5>_qj-Ky`%@Y9STyp)qd+ME|qTzxeX2uRRC#6@Hk)qylq28vH7oV7AApz?||$ znf5Tu`uQoFLEswKxom%gAya0B88K08zH5=UEE6R*TjTPF=g zrLnh@T5X-U`bwMFB&IU7oEp7_qdJqjklC^mGoO{8Q>XE8WNC)A6(b@0i4Dj${|&BO zV1$HpChrpZiTh{}H!17|t{X;6uVOZ>rvvs_Zkm#~YC=ENUB~e;R80xJnH^ii!@b{n z7q@0td)Hz_cGYtdk6;Qr2d~Rd{rUFs_TBd9?P`0nz25#3Oph*s8CeXI_sI;PG+6>u zFA3c8^#0j)i(;cq-hf~g{n}iyM8CE1$U0Hp$)Z0~(Jw#UTX&D4Xx2V*ti92t8KhnCWTx zL;M5ep$`FFIaKW%no_N3&GELbmTqX5MaVwMbpC}#WDX3j>q?>=6C--8ch>@;ZpALhTy>r z!9hh{;m$=thv4dy;jU!NpjH$6B(O+ze#c|MA>sMmP6?Z^=g#)eqR8#%@G!O#vE+7} zDuKo_7$!m2u^<{Adw2lTYitbzRAOGrZLu^QSaU!h9oaPP4(ovDKnr)#3|MVC$D?(W z)_bMW;W3qKZr7vEiu7Bzut4_u8wh7sELRNM%Hdnl|I6~VJVno`!Ge1_^BQkdF>G$o z*UJph+j)Vj6oZs*g6#x)gWa)CU)4biTX&s?SB0o-Uj(;8eKQiR=q>i;jwp@w1z#8& z!5Tn~Z-}JlK$3iqfJU-5K)ka6$Nb>8kgH(VkIucmADq?;*EGO z-if#3m3S;(iWBij{2(5R7vh=tPJ9qg#S`&d{4RbL-`7qYKi)}Y_q{`+zBlRI_l5u- zh+k}3K>AUNZ|!9Cr{U)&Uc^~~KH2|k7eM5FcJV|00=Ci*9AEX{b_w}HF97Kw10HLcng|Fd%0{#zR z{IAv!Ac=|)@=0elMF{ydGETI?|B0lE>Z1zPNTcIlfyel@AC2!r=3oFV1}tsg1y??C zaOD#NR|W{WPZ4VUF=uGJsiJ=!9{%gNn*J-SCU-h4D)Flr2LZ_a!LNgXkfV;qV;(dq z2+XtlSL;Nsgd?>g(Uy^X)9ByRruF`=gBtuK>X24Vi4~(BDu01loN)^bpF|9B@Z#5C z>U|R%k8Bt&k2p*x26xY-C(%7DA=8KoMfWJ!?!6ub>s8PH9=i{D(0d*7?_ib|vt=>> ziUm9c{P6nzFH;Yr`o5!g&yJ}P8SD9*qxjeh|Ce5Mz5B3xjfhcy@dzy!ZzLeF09!z$ zzu0oYvz7(V?!4gr+ZMcbI%zEUt?P}W+`rx##U$Og-q|07CJ209qM(_5)Ot6Ey?4F; ziV?cujU`oWiHV*xoGdM}{rB)kXsd}KcocUpdQO)TrLz=)sx2OOJM_r8xAo=pb&=*p zT3ws(_;jNNo}z82ajfp1DbQel3>bu=ewlk2eLJ<@6R?NLgh*8gqLGKOsw~Zx}idBa0gg&bN)Q`UvTvn1Q>ei03Go+wc5b~Bwe`z zoB;$wO~fW*!G9A>M-)t8-@6_tPL1VzXXf#2N;P~C-UBAy4blzeH)vE7o zzM^vDv*TBjS6B{sl*f1CMTkeR#C)LBFC)*!4^Nz6r~jmePx{dwjK3^$jB`qZ8LOyF zuVgf)QHYg$Ej0#w70>CKNtk+#hu`2aLxNy0gK>AG^gd5(oAPlU`Lc~;TIg(VmCon* zU>3CK?XVPV2vT?Mhsh#|=jJmaMuPKph}VC~l-fT6uy1jc(qrOfz*i>TO@EojVaXeh z3eg-|N|j;E=COr&X;+JF-_aNVFMq`Y_Fv)?@$PuE-M%^=p~R#&^Ec!H-9d8Xu%%%+mJKS489Xi~v( zo8Z5{6ps+-&__ATkvd|7065I;e*yW&(dScd&j4WP_n3frG;lgw^?5rx{o#z4VoksF zxrt+%$TfdNgDfZ&=l@NGw|-#$58ItXh?7Ba!c|or9{$><)ADTvoL1Ue$>Jew*%whw zW}iwDE2TQ_b+GAwi9mbCt;~{i>H9DG57B{Cd>8$5YEqQrXVz!q(Fa2)13C6r@pJV1;o-scc6;!gq+1fkK5Vz293Fl+{zN>X+05$) zU|d9O$3S_4{@y5%<1rz1LR{uUtq8}M@%2{+@+5Ttz3ukgs{FyMeBRYAjH&GgK9A;S zBe!u^!hFP6-)fdK1t4A#w+GdqqnztZ?C-4uFazqff#XN3mJLecW~h5)gFQJ0TFCon zJQTg{F{xH`-a{?t;oly7Js}QS&KoP({F2u1wMrSab`N_n5ewf((5gM1S*r%PPb5zY zZijz+ACD$p2*YrN0gLOy&GNyp4C+^h9D&5DRYTxOUT;w(7ty`Hwafzq7roKN2Wre+s?TDN{8H#FS>Pr7(%w91 zg9Q03!wZ4U4DONk?Y=%+H~NW@rM5aj2`9FJb7y@2cr)1m68Q&StCXcu%U^+k? zjxhhcAHC`O2l3(I<>4WdV@X*8ip*7qz#!k2Is^tBthi!V!TJ6b!y2}R$yhB%%EPih z^5v;vxn8FDpeRXvU0%8%^0nRUkuFE|dsevG^@(Q)E5tnpHuqc>L_ z^d)hlu1LT`u?FhA++3jK1jGq;5EN)f>;9WS5Y95?)%T~B^|!;uUMs}=zV{3JHB@M% z&rBPEy51`D;x10TRVcuFC~zctkBt^?GurQEy7|?ohWY-k7W+Gz=v||{0s$cWFY393 zR^vNArfy7Ye;n&J`%Ssuev{E7OUD)78#UY_v*QxIOYr4VV?_I+gJY9<_NYD{`G})v z;kStsjvIyp<^EVCXVQVdw!z&tp3Ew(-%jnxOMcVSu^684`{4fIPI4pc!3X!*U71mq+jm6U_CBmu#({RL#*56Bb)msE=@?4g z=F#|0axb*i_eX$%Rfu<)Kh5AI2+#(iRtxg*+OBruYaY!Ixy-LeejMeLQ*5`zapYtw z+%@bUw{3FgR%{K=s(C#Yk?d~QV=DsQje9Y-682@4R&~14iRVcH;_4 zr=}jgbvv)&;I6GGj;I+`!8&#wA2iox&UQAl={5&3%#ibTZ$%^A?{*yHvkJjq=HKPm z+`1jjvu;})KjD6n8Gb(2bsxIvgq7t*RHN^S?o4-Y+gdZ|y={?iwr6Y<0pGVSp2j?eq(A{ZaioXe^A#_P)yS;i|n z(+o;oOpvb?5YIDwfpQjOBvFZ&ZeWE*-_;-Q(CW=`+au3J!wHXE$*N#y7yL8xv!n2m zV^_8Z!^U>;%jOApm`p898?P=_4g^{jD>t@{#E`G zXfF0F6D!y;h)(n=pFzX7ebl394=LxeNay&M1xflvMyt?Mj|F%ly(oZGKZP+uwt8O; ze#*CaUdWr@C8xnutNPbt84@r2d^8ed%c&n@6C5oOr^mluCK)z$h3((hiR&bp(`Od1 zv0?msg5l=*ZF2g4m4g(F?{!f@z`D0s@&iQ}!e0nmX5ArEUZfNl<6Ajn&x*u4ePEAh z0N^ghT_l%g$PhB#*-;SqF`5~r2c#JJFLWHlRSqx#<)l=Ibg8GHhrdukv z%Al5x-g4qW{jyL!Y5av0Ettw^eBAEy>=7DYG0d}kgi40zQ9x`25mrt1THEjK2laj! z5VRljz5ANt@S+Y4sThk_xvyS0z)QA9 z5M4W>ZeWGab%N>Ypr&>FSbQphe4k}-Nq5REGgiCJOsikVodZ@^Nv~0Yf$E@27osU} z!r&(A6-HiM6|)z%pGG8pQ)3vLEdy^uj2S+&VCQr`CxJJCtdhYH2ke_S&(ubFNfi7e zwh8YA<$_3r{yR!?c1tHk)xnc?e4?HT)8iklg}-qqz(4Ep~Nj@A<2 zGdC;LFEGe=SQ|wA*7p~7HKpMWRo26Exw{pi)**ovr6t~jgJqX`A=Q~}LvL+;X)~KD zMHq^fEj?LiJ5ji7 zitpPfN`bkpvJ_`pain9x$X%-~cwanMSXf9_uP&gNKUHLBZnq(M3+QJ?2&)EWKK0H| zjSya0nUJojAA;D4wpdeXd$X#;LbkFtwtRC&fw=2d83i3%^$kcJ$n&ehaBdohsW{Mf z7wK4%eixT;Z0b{{#(WlqU^y7|1B>Y)F#NRkZ+7%Fp*`&S=dO7KpklGCu9pc1g|}*& zkCwF6z!30u1@)@QT~NKo@^O8P zPWXz0;sh%f!HzFM(JkDD2*ko245r*}z5PW>$-Ud@#0k>9$?~a&L9>`LGyQ+my=#}_ z#&szAJ-=caj>kX~WRs)FHvx_2l1Fl^mDtjeQh=Gy%>`yUG0ayC1dd zg#y_+O4kcwaCoB7gE=L*==5URuZV% zvC$md+-wmwtK>_S#>+>M1rAm9vADp4m+;I1d51s@;PoJKHT!N;=piD`V=PV584+Mu zqB9w+*#i{lq)TMJHv#WLn_xJ}xAhKp=B^Yzv8Nl_OKuLT{N4^9h=!rvSYS25Ah2f0WSWei^Xq%;QA zIKj)NHAAr9wG9zh{~a%~-Jv+#NiXg~_d4`0GHjac$BESg~#tsPe1=EyjuoD&I z3pBSukRpjbr5l3ROjeCLN!K>&$TkNkIun&c5ME^edWI%%1|YZ`{^e#DEb z7!7w0czTkniopcPqv9RqJ%B4+px6>5Fy6Ka&a5Y(=`p8<hl%Nyt@1c&}2QQ$`Cg%6CYhlJ`DxvInKSaf7uup^?_{%VNiay)wTuWnKos zn)x5cPPLULMr7Hpuw1NwqhZoP=M-lfuGG~SSp*K-mu!84NPiqu+i)nUd2NgR<%;brN z*|+IwEvuYaN3v%zlA+$kt&Xh1eQw_l!sd$qZV!Qwci9|`eh_9i&l(ij2u-;1F3$oHfrnd#xG)H1aZN~ajYQB0J| z0Z`4B%EqRiSem*;^a1F><^v)#_XRJURg&M7F;{l;mN{Yr>C+N+IHCX+I`|C{XouU; z@)frkl(K|}Gcv`TTv0VVcw6=aGQrV}VpF_U+jck<6TFiwvP^xVQpXPUEEl4858i5p zQ6zi%q5AWzq{LDl4KE|+Z$r=a&_h=8}+Jgfz1MP z)Ipv}I?!VOh8=9sMy=3ZXROENPN0@jIcfBiWbWg5U>t;ly^-|ms*$(|`?=8Sme3s$ zoDBo3o{oau%zGZ15Z?|@osoB!JokjS1$==ida#eftHe_)JEsK^yn6i)c%%8!QD4oD z{FEzW6#FHI&Mg#qjLEsz@Faujz^zg)1d#NEf3S>|dcJGPJSD@kgKyV%1NN zakNO`NARW-7WYHU7Pfy6zecKXE9fxGWJX+GU zFlYQ7wp(n}8>h3_Y0yfO8~7gWVwrn&tmf2Bv=WOo@~!Hd060dI4_d;bjUNf{*z{E1 z*}@SOrA6)=^@_Qteo5k5>{udN~}pVVVAZz+QqHq_=fw`mz0 zK=lkFbhsluFtzM(dyB{T|ZGQ2}i z28<7#P&n4sHBNZmY>cQ7CcM%LzE`7|A9meK?a_4fv?V9M3& z@4wGY>Te*XF#gB7*{nhqb!>v;*I?1GSnv36S-y*|2*$J{Cc4cv8P83@8>I-%;(_tw z#4LlDGO$T2-xLTp>HI3!CPDkWaBzfkY|nG&?UFIm}cn%~E0Ovc1b( zO?rlUS-4B;1?HwDSjRlW0`|b=Pu?iIEv+jTOJtMF_iH$VxvfiQHFjYv<}oAyQ693i zV?4JJ-6Vw%lzHL7ZR!`~{e7tn>2gfP-XeE~h|5q+9BX%FPgYDU(_Si*k@;H9K6&cb zBAsl2P&D-{T5z0)D8zB7YVUibfP0@~VW47a8w7RMJ%e&iZE~eI?95S6?6;59kJy$C z62^k0nB47}`)~GLTdhiTqjfe0V6kv4IPreaN_>MO5=xOwph=%-#7InZ#1M^*d3j?ZDg-0>;75O=M?uK zV4YmqCW=F5l{S#zehq-h*}HTQKJeN&OAL{f@;M4f%xB;tF-G^w{zSEqFj(ew`*h?7mdXwj-$1qONg%&dRCn?4*tTQefKBh?=*0%>B1|=DJgaLi+iPxvMYO zb3#lW-DXX#^9#V-Y=8ce3(*lD{=VscTdh`Q4Oi8`8*>}EP6!w%df*dfn_+8Efn&pT z0PILI^pTjJU@U@(sDk+94Si)ia0ZAdZ^2==(ep8H2M3Yva##1;z?gT|_!}AKBCdrS z{m2?Ke1es1y}^?xrExSQW94-4t9sINm|QeQ@SCFmOh1iuZStg4ZHzr#ezU&5dpy`4yq^PC~8=(AY2F1Bax9E0=$`9O7LsjK7)HI>=gdl>%rWJ0-6 z?~u+%BmIn609*E$L>bo!iy*YiB%0r8j5$|Kk!#rMj*TS`EEyoNkg~`L+u^v@H^hKf z#&M^jgL@q2cWIeIiD4}CeV^Y=953h{6Xi zb8ksh%J|uQd7V$^%kel_dsS|hx6FKa1|ge#J?huwf$alrAP3@lK&|q9pamXnx*p$O zzdsz_U2jJBhr=G8-X0DQ*DHLwu$b9?1GvAF<^xc&U#g2Q4OdDQfv@p-3vi0SBSbg> z><$2{Xv?8;mNr_p@u?O_*;3R;{N`(xO@+pUO!x4ha&oSVsv0JTw>N_i1|1-_{r;L` zz!h)Xct3uN)Sz9o8b4sPBfDrleviS9?4r&19UxeuT5Ol1JS?Tv29&I-*^h?-P+N(z z_N4g4E>>PUjoXf5v!Bq=R1`Ra2aLQxQPH{kk$r5WC5$2o5}3jR?n_-N}dk zPQLB$iL5JPp@Y#bC8xAv#7Yze?+; zj`0(y#dCUERWD|)wX+CXrkHjB!%OJ4lqY*KG*G?254gd+aDxY<)jB%P za7@WS{R<8&?0$tR;&st}&tB&}uWn2OxGHNEKSAU>#@bc2Je#vybp?zXsiHV^uioHMJBtv$*Eb8|~d+1=^z%&*I3NL%$*n|hpzjUbX-zEQeRcUR$ zEaQh=fxNpPhaHPy5WjdqK6v>r1WvLS3`KJtn4SU2XhfaXeTAsyO`$_^47pP{6&#MP z--V@7?Y~qC)x=#&&l^ao!HjOp!g)?jU2f~p-KM{8|H#XSN{ilj9*qx;YkKgNJ!r?# zM!V3Q{eW%3E%%wTUx>}N?K^Y!BQc9UdVx;98kpzQn!?l>-qh`Q>bJ;e>wAL2;Zdnu z=EP|9MHJQO|117-rw%ep9){88m@i648CstvKA-+zYK2hw)ZjKx^1s& z3w7&WUD&L&?$xb7L`Qvt%fiUqjky{Zcn&En(xNP^5oRnC<2Y4YTWzV)?&R*x>Ewk35^3As!VX?%m8GRO>_~p9S2yDtBLetXd~jtY!Q@hvjqw^O5AMdQnNx2kt#_hEm-Iw|3!G za#8(qU2Rq{qS&j2fv0vY-T|1aBu+BF!sO3?-Iu!uv5z$6cq9Mtuq_i0bg=8P*NpqU(FU91tN5l#^YLKw9D(Yh!C0QIW$BTh9@dc{rD=IaP=uIvgLg$}G%BX!keSN~o9qgDK!N-9J*By3Gn z)iI{WVWf#!4W5E6HOg?Fi0|{a?U?$o9J_1tqz0W>9RuMKy{5z>?|a&^ianN{3Ir)g zq1jURhA9-VjrL+!!%^_cjuW9&HU zWK$OJ%b`x3HI)v}j^6H#SHKXXUeMG)08^`)j`lfM3{&ojJx?| zi5Rw+E2rG$yC(bkEd#Y(cnZ$71;Mhb8mE_*Qn zcA(sKl;4U%29=ICe$Vq0Iu&_<7}|!`_4eBc`JpHzuh$-A53@;6toP*PB2+v zf|Y2C$*9e7V5>%X89Rlp{uc$3?O+|!iIan_Whoknm?_9z8+dU@hKgXYRhJM#hukn6 z!zd7^;KOi;0#W-PGxc5L*KSN57c{>g>beP@q{a(}@bnIGj$c-HcX_b=ai9UGr}5wn zzCE?FQt5XHuwG3azmu>5nM~Sh)m6WoCv!N=m@)7Pa301- zE%p}6xOBjrjlD|H3}c}`9Yte>jk21q`0&MMZV8LRD&ApqJQHLzr5I{xL-(r~kZ+GQ;R1X2Bt|Vt#&7@F9wrQxr4>&id+p z+dX_k*-qkChwvI&YJRDWQuWCB#lA9RxjO?6PkSQRrFSu~2-`|+iMAZ>1+!<51qVQ6 zf%IOTS|bc7q1Wk3xGM=^RoP4!#ix_S)B<`=TcmCz%xJGMp6FSNymiajOrsHuxc4yq zxDZ^p_YwUVE@3M>(mo_0MHRvbUO5Wb@xF&QS(AD=sI*Q)jq3PEPMc^?5@-@N$*RST z$j4oA05z&b`wc68*G!1*kev4C%McEJc1rF4g&xA`6L1E(x=i~CzT`AQ z<#3;A;0*g7nsh)yN3X|DZ1J_e+_XsD-lagKV>X^jbqhoa*o22!QitfU_5poc?X&GG zfuKE?8L)>Wu9o)bv792KcY_giJ%Wm65o`)yP+O7EBEU6TH%%H zHN(Z!4*$&kPwbW~`W5g^sR5yPZS~WMd3raka4Yght>t9s6NO~lOyo_nFuzHb|8eL8 z^@cu>TC=OErB)4fqJp_BtOxF9@EduhIue!b70@H0TbhM-d12bW^hr8E^@X0~Rqt(u zZ>6LD$2Gqm__(_^O_M=@Jmh$vUt7bHxBSFjH`4BP#c$OkMv}3{bY06^^=+;1+4lx8 z@{ulgS-XCpV5zxNd*edi-S zsq#uM1QY243HY=_ab-@XkYGr*y2M3))C<$?It)8LbX!BW=J_EASt#=VphW%}rKyMo zm5RWm?>|PLl$?tiI)r87stdH7XnD6dof>f>sDUzeIVJK3)}%+|xT?sw;MM9$4L8sHU$?Q!ls z|2P2oOM?3khYwnY4m9vp&C*?6CNNvr_m@HTWb9||BMU*Nt^0*wd$MG-ztHFzZ4);J zHg{vYpSJ%_w+Lg{Yv5duol(@iccwozIuvIKGhHfn@`65C1Zxm;c_B4rNOV>*b8ZOK zDJm^#IKJKN+h0?-u`jE8bR=-1O#<|zJdRJm6yqD+0;u1V1M%OSawi^}^TrKq=Df1P z)7_#XrVPZ7bIZgkz%A9$(A;9SQn;P}({(rRMx&&<>23{Q(87}#?BSb{c~zTNa~;Ql z?rVyqgfVTDoR*>@VIUu*AVcK17bEt`Ol~4OQKq~X*7fEoB%j=-T88ooY$PrKA8Zs} z8DP;dy4+$r-^L)IVU|v{LwY$4Cm3p0gZosL+YLDBS69bmdgRdXTKqZ?hgszGUREXh zopQ13W#&;bMq5bh?vNDwpS{%f2p z#ItU)ocHLxQtXdLl~L_G^U^`*6<^xd9VS_CEhXGWszhhmV>s2fy)^xcXoTF|hv5wE z_7`%q`!}6n0S>OG?mZVK$JkBXb1JoR!8je&r#m9cOj_Scwp9mn#Q1$}uOk;v4GC&T z^l{m<)75EMEE`3USWa473aJyg>eOCbsq{pZ(CjUmezqU}4xN1%=&XL`%<3OAv-+7c zds1fuGgG$J_GC$U^n$ z;gIO^Gz^Rl%YEp6V(_GJkR;2ZGExFIvZN1YE;8!;fVm|JUqzIQ?%}6JwMAYJi)w3E zRNG*LmdcjX)~=kk>d9$$dEu4|UH&MD%kY@vU8=8Si#{JWM1aP^d{GpPid#Bqx$Sh3gcE@6E#YMA{rP%&hge6081p;@SD9C0q9E7i{UD zwq3dP=?NW=eP~i2Ojuu8=b@_nTGj3B5cWe0mAb+42RG?BN4pp+vv{pNSk&-aS4t{3 z&XFQ%*$SVQN`l&+<3RKc`;=LF=)TWNPyB~^k~8OSldmlm%37nFRlacorY`f<3;C?M zdr>Y{m<>CgCfOBRS}Trol`Qtg_F7e0&6gN}(nLd&3`k|XOwua}jG&u199DUrOYy?~ zQjwKRPjIG!=`pn`cq(423XcjcUGhgku~w4$)UK~66g!8#r=7N6nQ-hn+3cyP>_@aj zoq5 z?mk*QDzMAk!nODpx4|YuuwljrO)_LyMQaslyBgWRaqX^;s2V-^@oS8$PdFnmm67v> z*SFSB`px$0jN4BD@+93J4zYhuo4&@je;R!KaOy9Qqp3}M<99W%J&#*2pveHzrZ6Wu zZ($d>)4KkZii(FmKBoIYdlb`cps(fJGmGndE@YGYhH;$_mh;ZCeW;A>gGL1k0~7(A zZ~Iy^3AN(`QA_v{+9SReHNi`6jRT!J*UV4a<$0!5yCx4(%TM3+Ic1^OEuIOB1AEZ- zJRIZaLh<{YPp3)D5VO;1Dl@>j^f(qTg_pSg2 zMmCdN^yJjG`gUAsxwIn4oZGh%GeJeTxwJ$hhK0vD210uBzNKdNr}q!n;qgcZTyln1 z0nZ)ng?FQP*;E+S+pJf_Ngow)dplQQ9E&t0f+aN!rxDtzUl{ngy3VI)$Bi1BiaC68 zG-NNDK=fu4ULoir;7s}Kfg_jbpWv(!hQ&VN7*U(}%D1-X#IC8T!RSMUG|VQiKKH#g zEeXB}H{vnu8VVWLDp$^_!y6t4Jw$l+_G>?1X~kn*sJ|IIIPv(ks9#?V{WJQcg_Jcz zowT(yC#K&ys%2s3%t;JE%`*q04c$bu!33&U(V+`pEW_njPk^>vwE|!pSfup1GDi}Z z7J20(w7&QZ5L(3s4eP#ncuPq*7B~DiY_H8z`i@_T9mCzV^_afLm)btU z{J!3%5By5LvmUPR<~-YcoxTTTv>oM}B*mzEccXltq!>|nJH|2$(Q`M(vaA0l=d?qa zh*RJ$S;V%aHQ%O=h&8`UGWe=~&KGIr)0TLXEUBUP!zVVR2wCPOfS-w{*ZJLe4;cA6 z-;OtIk5-*k@)D>fDSUZ7)!m;Rczl=d$3h%DX}Tj##@_)=KHgZpnaK`LZkun$ic(3R z^wmgsmaTQ7^?`K5(yXdt)4YoUfoM#$8HFV?R)|udm~WscyO^fz{7UgUD3SX&^;hk( z*p~Q?3VtX*0?vJnKckd8BSZ~&IMtDIur_-2ai|xCA=v`>$?;@9w*$)HL$fM27?4II z;m3Qpn3m)L36to|@8&M!=$7fS*=#CahPw)G)jf&ThczpR<-5e;i04zZB(y6qg~2Ro zh3YZA6*zD7o#e@MRku+~!1uvtQ&`~g4rl>dKYq0|KPmj8{`I{Hmcx%KU`~41EcY${ zY;fD?mmOwjr(EUq_fnuk?%*Tk6E0WZ2w2G09mGVQtif5TWSm zqL;<*>Di421LQbbw%2W&KMJ*CYyC{lboDk6Z^hJ6G>dC~+pDGD`$o6@n)((9*hiRZ zO4Tvslf}fbYg?gn6jWsW5PWCc`q2>+s<|$sUVlFzxVm8himtDMKEAj1IGRX`QhbrRZUy$Qf_t~?ZL!% zSO98N_A^>_Lmtk+If}(i`BxxD!>-(7W)`+c;c4${n)ug>?$pa8$+Y0AgM;lu6ob+q z>I%mE^(yZBfjfERjLoHx7xc>k<{b6g%27X7mpTU5({Rl%pmm+W@QtK&u+~gBVCnWY3DMpT?!$6Vtbtsaq`S<1nrnYIJYYI!4Ox|wx zfR>I+3O-RMk4I&XS4px{S$P-5;Cnkog}sVPSw9r`rMzfE`>NLTm?8lG(EQ-CtO{Gn zE5ay&ik2YkVkY)x=Acw>H$aEA?6P~egX5OHudtNjS`^#3H@1;Ofc7k}3T`T74gJ&9f>{Y)FbInR`s}bL?sZ0MZQqCfaWP}0^}W!p`&>`% z8h-ND@YM2;XL?jf2b=(NeD7pp>911P+ZPij4DZ?BaQ)po%cj;O!+}MN^hyJCJFWM9 z^FnxMUZPRFOGpZ-!Cec)Hs1H-L6!!PABMwUjdHZypmLp9z+~FQedG5uDmPu8zP&H zy`mu}5Y#IZj#H7UJ^Ux#N;IO=7#}@hsgn0GW~E{Et2XtSLU@N-lA>^bvdfOZh-Cw(NbATUKXQv;_iy1%du( z)q;M@%UiApcTJ$R*AxI;|HJ^RD78?Jldryi{l&LmC4vt7`$Iibj~F+GkK_Cwa7aT< z{_w@G2M6UCU27cFJ1iv6GMqSqp5Q4 zG%?t~3YOtpvUh}JJ-CbS%kF7C0C)|;iWzN49K^JZQxc}`J!dX;B3z8 z<(R?^S9awKd9NR(Mr!bwg}Gp4g1ZVEKoI&P9fl?^95Y9gP|*o}m+nkG_8{@95vM=5quQZ93!8jF3MSNOkrm4*Zm#p`A{s?# z>@aFZ5wHoQgc`*=`^^qDU+5Buy02sE8l{|B$bEfoDJio7tW7~mNCm1|W_=b^dtOpy zAiuorXjQsK1R+pwxgP#G0p^5vjJmY&xBx#0W|Q}9PU=OS1w&rXePk;V{1Yc=@FZAS zKy32XXvi#!1EYAdO%1fl%vc%pV#(-dG?i3oKbmKsLMuMg$x~>lBCT`io164bq8Rv{ zKZ})FeQg1{6K(s@(OfI#|KOQa+JCEYZ91q7o9uUDboivR?4DTrEU)c1>t~&1cRNm< zWupqVPqTLjV0n>%)Y_)L1X!qPWqFXKkHkq-E_ZTT=H?0OEmp?h7wfzJAcpipR0^vh zFHl&&uZfZ7WyD}M?zaQluuxNI6g8=8ro9N=bjf@?L%}|B^BDAgiO~vYdJXOG2PB9Z z4z8SwnjQD?E`YbAIhCT92%c~uNTc=S^d#$kW42A_LLJSf5b|C8ZAf3hI zQLINh&w3QjlWv7d(h|c6vn`Kn>_8FZQB1CC4A?}FxoLTu6QQZIEewh1{l%aal8>1wa(ji$Rw*b&kk#>wO)T$Y zF-{8x)Z_FxV#0&zcq}WqqQQHYt(m4T!E}2mYe}dq_m$DdqsM9ktp~K zd5iF25kfusp}NEJUt^8dyM-C%xI$kg?Ru%mt!#u)o~^hEb|YrtPl&JYsByGKmo`l_ zUV9jv{>T)~Qzt8!)-E$rC?z)O)ga@c&8GSAHCkB~OZUF$OaPM8B55t_$DDW<7sIsy zRd1{1UjVx03oV6HYUxmXE4g&YTvWE|Vmcqtfxx?&$aV%PDW%VDwfJ0TwHI25rY|31E&pR<|2K1+XKp&DCbTBWbE8#)CEzgDz zdeveh+Aa(p_no$r;fg(Rwi}F4M^l39sZDf*XPraY7F~;tkqE1eC_dL^D9sj8JKVEb zmj{BX;VEal{zp3&O1O%OgdsVYV9S1Jepl25viZP@;NBqXZ?@7mJaoi58j=XjTWz`O zMoNggy60C0{?L&wKlSo(z$hIjDwxEAMAi`#Jro^sE>V3<+_vqW>tOYH`K9E1@|FX3h1d>zBvF`$_LDy->47Wm!acE7P zY}F*Ma;MXtUY|uMho2?=P>2QCxk=p!X)kgOO9vHnDL0N;x*^9oR*hQwpiY=lA$52^ z=~m_9HR5}n60BlxbY<`I;IiPA0}HDup13vXwyX3}sbfuKG-2E*vYEE95Yj2_k_Dv@ zqx>#RF+k0FaWTuzLgL^U7lXLxlhD=Q52~3$HDKL(c|I}a4X3@?hc3TI^4<;;v_Kk$32>wNd}WpkbH zU%p&k=j%DuGhB$SJv^_gSJ%|gZhWE&^l@VD&;<3r4mKfmcx>B-MyE%HAvP#^SGrHb zt3VDSwMw|Q=~ZG^cW%6E_RF<29qVOj4u`c~qQhZ--MoCczFy)*rI)cd9Cmt14~NZl z@$%*FdX>x-o0kc6!g+`jkEYKb)uA)Y9n+QBqh=Gco7-cGHgLtRs(?88b+Pc=} zT->y`y_*Jlfg6Zk*~pp-EMz@6vV@tP8v5KxpY^=vK6f`Y@;Y8)$s-knpaxn&dxNUp zmoLgARa45goW;NwZfmRxR?f0flG9msc|)l`saUV4M>_Du=Xu&{ z(m*AoR-#N8q7`!)O#)lvy>`0SvxLMTY--W5BT@0hEfzoJjF3WVwnxjm-Q67Bjf4aG z1HkP@$8Bf18XIcZqA$lA$avh|?aFp7rbTU!{Bg^Z6d3$|b|L zGhMOA^0s~mGb=}ozVz)Rr_iJP>m#Ov61Q2(@?KD=so>|Sv&Okh7uwk+tSYrih=npI z!a6G`PT#8&N9TbS|7Le92kAyFFg9ZrRTbXK`MR-tTSRYEtv6K3IdC#G)ufCJN23Nt zgXEFDN1jj*jcssr;E87)R1tyqjwLH>; zPYrJqvnBPL-MhC%exozGRzPPv{>15Bytbz-HgjZ|x0nH9|%)p;5;>3}{_U z+HJ8c4~Gc*h@^-0B5v~bW^)^3I-;9B`~rwsuFKoJv7fBrHT;*DS+tm_wQM|^rt}{r z+@1UxsA2Me7S0(^)}#Ye)G~xYe+k_9)vZ|)FZ%?>bE`(PiQH-e9&8FGCuoDT{C1^@$KU3go zyWodr%a2R>cYj}hx8Iap3a4;W)#?FF*}p884Tpa>WnbNI*F{^kR1(3)UHx}_^QZCX zHp&0gUdDe~js7%Q{AqQWT){5hz)ECSxZI0B#r*nlmVS0j;CFiFIbLL{XaPU}G=Y|r z1-?iYe}cvdHUj+iO?flBO-sy4RALb8^5&mzF&$BVBqO`opPLapx%^W{_-gX`=a)20 z_3M@Xj7GaOk6QJyBHk}frsUNg?w}7$A2_2EVFIf@%U4D3>Dq#Q? zMs`x^nC|mO1XuJ-Yqr3x?#fm6Sgk%{#6tFCdFaaMNUG49-^|=MZ#VmrU%YZ&z@p;p zbjxe0CM~Z@Y_vrenWB@F5Ka%OxpbxSu+gNxh3<0?RA8L;PftM zx^})*F^qt1L`ymwfsP6DWw@|0U}gAk@_7=a%L!c#@}Y-tIjG4ucQM~R$@O>!7unvs zyF0#W^5{)8D#?2l7I#u?0A1#^G@p4(lj7YwLlNEv_TRWEZSd2s!OH2Q2Y@x_e;^9H zCiR&UzJc#w4m9EX!f`%IMOj9k@cA|^TqQ8CE1k(x_B40n`rd&8|=pu8z@5Ct$ zZZ8G$ckN~RI3OgWP8YwUD1ELyj2@J$J#koj#s?s$q|d}xdVCA zVc-jxI=2m^n>Gh$sTUNh;ZM`gI12za^4V>URQKJk!RU>ugPaSf1~bUKpdH3(=_LoP zfch&r5{WX(6_D`O*rN5Sn?kLr)I~<1fq6?g!bFFn9GrNqCeB=F9DoKxa^Z9@^4(2E z%d=?pPZSNQpGfv0*H4`P8^-rNi*Ph%AD#~RA$B*d2APtr1(0J;QASU3Xoy<$EY7tR zbIS`ygei@IV#e%mY7`nymGJ=Mk2qNXD`U}~5u5m+TZy-1u_AAYh8Bp`x-=k_rFwWF zYXRgMR|*EQ10aCQti?Zj;6rMQ58<^&!EDTMn>UL!g6k%Oa|`GjUTntAHT}MNwP;2; z{mh#2MhbD?$6O41LSI)gewAEyqXx#K0T31@#-zW%W7R7^D#T3IizW|nZR5qc6={Lq zYgri@>Cj%mHf%7=BE6R7+Mpvrv9q0fQQmmmGXyD19? z_SA|KSMllmpZ|%0MdR15Hdm zzs^^S&DB2JT*a@(`y{=wps`DSj^t#Ou2No#24J_jJ<3YrrT6z(OI}Cnn!cu$1-7T) z^=on*cDn$MWHtiEza3TM_OgO?eRbK6CEB4q5rpA7-T|4_jb>MHou}cKgD=MXRRME% zZnf6N|9m&VrOX!w>?(#EhE1$#rRE5PH|?!iv6fbBtpu?*&8-r;!moU7VLM8{;*qXU zM_*R#-oF!?vjwUqH&o&7aIh%5-J;F3K_d{Hep~wP4u#uCxth~fnK4y0=j~-zsih*d zl5}Z^u9hjyJsL)A;(fftv^qG^B`jfvDXMU33!6p+8ike&Ee~V$%PgKOJPE~x)?Vj^ zI8>exZ-^6S?lfb5Bx~V((ZtucG3E>3C^|=+=3w{ByQ(f%q#s-CvpAA|mBtKaVQ0(S zC-yKBbzEx1qZI#vhLTx<@TrXg2o4h9)s!_H+T4#UxnqHc^)>hLb4!`?2!L*S0&T+k+5cBVgDh9hKeO*<2oi?X2RvHZ9# zY|^q68p+7q?|NDNR60ME-tV@@BB`XD z)aC9=?L5IP{YO=PSe$r=kvv~F_uG9}uD(ZbN5p}-S2C@QreX{Pb3_#4WEABnY(N>n z-`j+3HstNR9gpWpmE#SX_JcbBd?P}=L1p!Qxgq6Am7p-9x=q-M8r!STgSSqGzpvyI zu{AQfipn-@!D|O*Kw~7Y;G_Tw9fk%hE?QU;I)|U+A|%HfA1_#yWTW|n#0w3Q2Jliw1XT6K2^^d|kLVWm==D;zP;q`fBT zF4&b560DqjS>Ba^zgEOAn*dqIbe%rZHlQ=r5>OvHUOLWg==0hDqK_#ihO>X=Mkkxr zV^1kJGVl`j6)J$)U2I)SXsr^iWeUKNOjelq$?Ba9*IaQ z(+8Ja_OemQtXW;tZq)=c3nkx{Te?bP3oCjyAtZxp;spUE*EBzQda zB061tyD6$#Md%|-IpCU5tgZS-xtkcHKqrj;$sr|wJaUXb%eRYII+P0$LB>_$p1VJJ zb>Ov!SRM&={*D|X(?Gkec~7JjIv#0+MAm9;-l7YOm}dRB1B*5z zV#XTI!X8?Cn^})q1x0KJV~4!Pj^znRcAI*Vg}%V75`O2My9(Vcx|SPe(yk?-z<;r- zvp!uypDHnh{i*CC`eC5s`t+#RO}83w!flK9rKgS%N5(3j&O5;o)w~*w5+&pBh@vdn z>-b@Wk7=p1#oXZYEl1Jnnw{UgqQmE6w%oa9%esq}B*MAXOM5GU2y+3Gekf#D z*18?3EK^YIcC;fN6^i|SS{CT4lwZmR(1!UVB&`@mmB zi4{g$)m2KE$3Vb|i|Ihf349UVAzOjLWXTGgpK#B0Xn|i{g#rxphQu*2?i%3PFhWZ- zn@)cYEVUM!WR4_6u@4fLMCXhY_JKU0%xpgRkuxMX8#4LY5#I?ZKSwc2F{xHcd;(XX zRTb?Pk9rYt5-13bI}SWwxPj*>E-5G7Ep+gE%C@OdFqC9ESe^Sm+&>;iE-ak94(N zx!n}Y@{bjOUuOWM>H|oXNIDk?{8#u} z;cXv%^>N#{13;$&UxlHjKw8x;W(7*i(JLi#(=5y5_b%b6Kbwu|!)4RP)h4&g^sO1EprcI3AbfW&GJ?ImSW%8i=N$fGEJ|{nI>-Riro`!gvff5cGF7 zn_ZW4V+?Yo>-eaYR^=f-jFqOQuEHIraz*d2jKs3U9 z>8V@%&Ay}Z;CffM&dD+LTN?go^7(bSsNz?!IajYTY+($)M>t*umWCPg@%#jnj;(Tc z5{`eA7JOdqw+D1_JODoj+h1(ZydDdYUeSezb+fDfhSxs4-{dz|{9Hv{X|hU@Sv#f1 z(z)t6c|96?nb9i#DsF)AK{NqqNO4Omz$>e2SJL>Qcn^In^BpH#UZJ(w-N;KQrpEAb&RUmDrV(u|6~w$dD7ANF;Kof4dwhQLW#PbFxoL` z=v)t+04!s3V>Fnabk%*?9uA)a=)bz$j^h2{uwlcw``qln)g3^;UVby=5 zdF2AhgI$M;@OSz)ZxqFrq;K-KqH{#_OkD#_ykm>FsM!|Fs(Xm+T~dc+XC0E^Ba$uw zwwHOs>>G6dei2zbJPg%v`VkzMtIv{jPu3!js&6#-2U7`B{jANNCuU^f`>eb9)`ot!2O?!H-&6zW_J9~y;pRZ>xU%pk5#uTmhHocjp z(_3T&mw=t*ZTu#^8CfKo6 z|Nhzf$LLN`io@{#sh33b_3!?b?4wXt+7<6G#p(jH8x(ncv&8#@;D|2(MBBB(-~|wd zK+!@SfVl#$kUP9h(8v+qKrMl)J4;{p6r>))<(tFdM}l7vg`-Ps7yx=J`x#QJKlI?rhsQl3BOE0l=pnKF3V~YZ=FEVSIY!Q zkltGf7<~og{9EMwk?<(s23}nk^8#*Qd|v18D9lrF`2Z)X7{@E*4PT+}%Y1p=%p0h; zgE|yNs=0g*N3$8nORluVz|1wJ9+mLG;^|MKa!)ZZ82vi5c`N}QW0xoq9>V~SiD5y4 zN(Kd7r@HFh4=VKbd(jy@aL2n*e81RSz9+8==zopR`^yJ>7MyYq*8t@rROj!}d^yeF zTkFed2H)CWA~v#As6doXr>Q=fZRqIS6}@w#VsbECCp-@M=S%XkDrP= zs^BrdYc_)nwH|NtqbQbLV75i?#I4PV+u7{Hj>s2G5^y-oK8KTk72lvM^(guI6`N1* zk{{~aO!g*g`dzEzZ+9#Ys8` zM%7wLheZ1YBWrtkil8_}*RXxA6*epK4u?w%J%;KVBy(oBu!bb5Cf-;w6in5z*&j10 z+N+`|v&3j^3D+?S?{5aPTjK-kNMD>+V zdz>SRiPP4}XyFZBw5opR%HSIYNEW{T@-LBvq%PvkhUTOsU1$G+sR@%Xstyw@qx_ zX2`_WYvtd0BI@ZorT^VNQB-r6ce0VD7?9_dP+|+LVM%Hf>=DH+oYL=bxDk7eRO~5k z0bw>hMfO8}x7cNCplp1cG)*n?N1pzuaMN602U~LD5PU0*GzL#{YmLUIcB_q-jK%7s zacHw+YFob0Dq;Q{8N&jOFgd(o2+YIbZHnpV-~r1pY+GjDb5U%4`xZF{^#Ohz>vP5L zNcMuCWseUTw z>L8Xph>vOzXs9Q-9s>(?W}Wtl{MtNWu={7P*xp;Qo+ResX_82h9vZ}_nHh4DQlojt z@h)^bmbq7q6tEgLawmD*+UITToxbF<*0Ur=Mc1c8ICf7Y*5(ZfH{Cg{OWR!&Zn?7? zr>12AWS%FtJYMB1*;T-NWZuNsn|}-PP7DwL9$)&OC{)c43R9E2uQ_8bptEuQ4V~TF zb1}9p^jP`mu?nKcY8X9ME_$5*!yOr#+K^VEwxVJHmyQMdTxzIxvSVGElW7gZ*;@0Z zH5z+2bsNVN`**8RZg&;II|hK_u7`e;`+GOQGkRi}=2WsqStj113q zYoWgb7DtE&Z|3l=#|WhJ%E^113TtBBY?hKw=5_@l5^%G%9NG`FJzchSY*9 z#8U`IJtwKTeb49-pza;rJi$EWbKH5@FskUc$z4|^I{FQ>bRJuz(04%ImSv;p^ceeH&GFv zaGtRIxWVCL|j_b$@i*hqelXtyR|8abBBSZUh!fzGEi z8vQu4PI#d9%&k2oRX;ClT!#REyT7~pG2%QW;3|wvt3Cnlv(KmKw@1Xek{@zz#auG4!d6zyg*dUd%@-}UQav8`ucSpXFV(|fy)C1$ot=jUoe zKUc4;@*d#+gnCj~X@vRrF3i7oVP4+G59$6%Fu$q7+M-W|d0*La4(1ig`@1UpQNe#k zr~k={`x78Ntbx(0{!O}Hus2Mb05nIVHggsHwqpPPYK1;|Yu%N*bEFL3W(|=mchyIZ zfmHW|x+u0+8f1(`4x9)_ivIxv$z@2nm@<@kP~ip|(pqw$l#yTBAlEY9F}t)OW@fDO zgTRGY*A_X$l!PkPH1`UVfJw7UadUe??)~{MEUmMwz%MKpI8lc?j%0R@`w6v!c1Q}G zaMUY%)GKe)ZE)1BGisBvs#vIXc5GEOWbMzf+e>3Lp_OJ`=cSr_#?m4in^D`#}QC&#`(URTZxv042Gw&CW9cyEf{0HSPRD=@$o z)r{TzV6Mf0HU1Ym4YK()Rw6qrSvEUZ;Bm3p;KpKzherTE3?OW;K~(IjVoc+KiKsY& z@JrGk)0;1=_W?Mv+TjMQajZ+Am3P)K5GF!^W=~aer>R5=YBj=U<)`FXdh-3Tg3sREPgs@mEk3pJieHo*!$%4 za5#!*uY1)Cy5GR%1dD}?NJ-9Nsrb5US9ES<8u|uZ({(IJryE%0RL6}Jm`%&X$wS+Z zHjM7uG{U*c39la2s&J}T8UB-2bQov@>2OJ?6(NAA#q09^Brxt`q!a6MMF-lgkP!s! zD2}sI@un9SojULgdFG8CtePlBAxz&W<`os|1Hon7O(7Z0bb1+pyRpn7x$5ngc?W&k ztJM!G+izIXV_Soblev44cwOTDh!2PET6f-CK7|1kz1T^dRcqub6CH_re}k7(JXKL? zPwtD2AD!_;0Z5_2jc$^Kz`jBFE#i|hU-nHxJ!jA)6-OE}mmwlAO zQ|hN=qTkoA0|nxWssm#n6k87>FjChu-ief01?L~S1ivqwX@TrlYF3? zl2a~))P9|&+A+4LP{4B)4ut$qBQw@<)u99Fqw104q5q?PVp6u^e4n_C6{itZT0)hZ z)D|I%mw%&$fwVmU$1>5=pZIx`gY>Py_y-gcfzVsEOyaK~uU z*xNJxl7~r`ITiRB{%lv}v~_TaBvWtN{H9Y0PWkVD-MZ9;$*~!G+fl~)WZ2U(!QBj& zBJiU79X}E@BS<{9Hp|&L=HLj_^&kP()j+lGLzE#TbGC4nLY71DBys-8<!j0=5f(MKbq8vvge5tLUy}?XD$CkIbWag6+YGZ1zIC%WU0aW|6a}ZjAMflbu{2 z`7hn%nH<2#x#Ok-8A^4uCq@r$qDV;*i3kECe>B$P?3Tpsd>q&6*MI2{tk*k=W3@v$ z6EssLX>+}%BFbZp8H0z;c)==pb*_Qwm6N?5H^<)T9Wbpq34`edGtpKQ>6;h!0&~(E z%!R5;rea8rrfHi6d>He+dc8XwitB1ZcP$1`gF#=f4~GWI_>N67%9ri7fXPB~)dD4W zRxRLIumwsCty@Spi@4G-Mph(G!5Wo~HSo~_rqhkM&|kUGKQk@EGvKgl03W$+IJl^N z!V#$~JOp&S)dCDY4&l=Tm(Rhxi}Onr7~d>CJ*D32PHB9f){!$_cUw+icW+OoiL<1q zu!ys)sRLHwJIWiPyWvj-zrhtfJ*`ih72R8yAzhO~P6}0<5^`d?TC70Hgn-+nN7sBR zZScfn=Uw1-@P8YgGP^k3|kM|F(vwK0aC_42dYQ<%<*;G70ay~hV_JuS8wsv6CwCX1X zO5>jm4Af@VPjlU+n?QPBSAX4?U$0^j>s7G3%wbe3%m=KNs7%#t%TYAupHYg-O9iyf zN6BZtD?a?1Y6#g*MZbeOaS~=w6t%A&SuU?z3Cr+!jGN+Zxj`n8e%#zOyZjf?NQN)7 z97Vsl6-7mKhyF;q22VxC*(gykwU$0EH zO^j{3Eoy+dM-?C2qIq#a)iS$U1|^W?o;)Q(3J1L6P==vJUW{kJa8;|!n|#Wun`LwV zwn29VB~Rm&6QnJ>D>iK@<;bLI$rV6y)pqi#`B1l!X&xt`-T*(aZMJ({wq zND3djk=y#{BJBv2zUK>-_a95}5vVu3u=y_cW~2Bb3<}2AK|k^(W}G|oy~>r?F+QVE zsOW)6hy=ynPB{>r=2E5?QCg;t*izO~4AZLp23Cx~OwFK0-C1h4ldI;TP=T(cQsU^N ztFm3f!WS3;&jQ~``PW!|Y+qQ2ad`_O$)y@u`{hzG3S+9CJ(p8;Wr_>4MEZ3A0WJZyMMXl^~WkvE>OytO^{C9ZUZ>kfQz0}7Gp9)rtN~2L+ znB&61QzCPodnafSS%kNgW*F=r5oBft=1_}h;AdWuz=xLrp`27q9j47l^wLOqFjasg zQP(x;kCsoxaP8gTD)Ib3|Z^3{_gbsE2RYnpJ+EyG>FQ2IAi%ZOF9K!d+Y zYcx&8n_$vgjRfT(Xv4W{JC!Uw!ce&Mdr{q3Zx#^}e_sG;f#JA5cKGWS&1Yc%<5`kb zH^5HYg(NDJYj6mib`%`PN2R9@_1JR(gs}!*Y?(gPNGjgzIqg$j8{oKYJvq z4h^pzNu-O1ZCx9!K-;Gih!}bCGfO6+SSPiqijtzwp@#&epf-mW9EmLAbJq9CAmlIQLvMju)O#_NG$!2y zYQ1h&CFdAwo?256>IB7SAos__KSc0>J6^6)fQvdQ;mJ~ z`!9#u;>EUFBgAbQle9F35*A`fLDjJWXbDIF)BYj{Oh(UNR&!sO?p?&fiYffoos$&2 z=?f*?xcEbt!TjV~%w5DR{Yh4{6e_`Eefk1_YMe9#BwQ#tIhPVjRDD}IHobG|J1Br} zB<$@;+fep}40HSDuuT&kp;bB{#V}ThERi@V@HlDXOCnF(cvmhnV$o2nIZ?t=k3Vx5 z4qI^QIAM4qPHoK#6H|1Z-M40VPRPxCi(`w;6gsBqGqvtbeaaac^fM z5v4~dkYzR`$?Z5$8-1LWUzoK@eL;yc-WQ; zb35oOn;@c7bR^6IMgs>I`FvOI;t($^8zqUIkSJ1L8phL4&O4Z<$KpJD?C`+mDG3|v z^vvZnXg^KLjSHuywb@!x;k4z-hWqei^NGmVhrhM^k>HrlK#0pSdpn?QbA)H}5)i+z+qJ^ZAd3C~ARYR?7qpr5eEp#C$WTg5Bt>Wdk zG>M=xR5WmXZTf};C#Ox(xjwNi-|ou0>SGqsceYZUUG2zs(-@?SLmYd%HMo=ZC_hXoevwXV^Y!kZwp``v(E=z?A`#e_&v0v%p8PJL@O1) zRL$+1yE`MFgwd=vB@;-NAP)U7mfbgIqMAMGnL)o<7MtaM0}cPa{O~(Cu%|ruc$;46 zUt>|0nx00H(I-dtgE+3K!oCvZjVCleA&c(WD+ZQmX0!$URd+bd)Gqrit!QfxV!vfp zIrau>+@+jP18xYOfwdS^x@w7uSytZGo^{L#)jT=*lFHe(WVc&8qul?+3!yRcUrf>B zU8%1d31VI4%9Ex=*&YAV7dQrDk=&^i>7pgS`O1m|UguIe#e95em&+&f>bjeEqfyc@ z7A+Jd%;RozN&_8Zkf}d6RgD+LkPoJq@6d<2-G~x(xkWCfPac&IhQ;C73#igIBJphY zoIvO_jg?~JR=8LQy#Ohwsz&7 z>~_V2ZaT5LF)gaSKyuSGO5jaTzpD(=Fa=`0uLNtB_)7+8N_v_bUQBv922}i_S4PgK z#^g6L*(al;xNa$lQZN5$*e4g3-s|O*pRa|CcnbRXRiijUzgdbWO7Gx`R=$w3vr%tC zfF1cA3iA_Z2F}{K7p_sOC@xL#PalnGg^L1dU;m_voy)>8?C@!(twM>z^&EHUlfE*qi!BhAWuHQnjO~)VfVE01*Ly~BQY&K@bun}O z4l%;8&Yx?~-yx7CpKh;$IA$H4#ruYw#@vfVhf=8W$B)`g=0Tifj&d)(shizBU1z`9 zHTS>6HNX^u>Y7sLA8CWJj}S()GdPtgYhnkAw{6^x_X^vfFnS)>*A)=6D#JB8HSQ;G zP(t>a=q&bwJRGcVFee51!b(ndJG4EbW^_Q{n;U5|AmAG;qTHT(A3O>xL`^km5xRX@ zmJ*pv!)sK+$ue*9i*x=llE%%u>4gSCQMcO3&EiodBZDIx_G{A+JI#m_Jq+=IXDxgJ`<&!iE8*vbctgiu0~n*!3QXulqk@E z!FlOw%Up~(1aBuf17tZuqtH-Ahv`&}0Y1V8N2}N&vw5C9j-ww;#JqDeRXG&BRcm2t zIeZ%Q*58``?xqS`OBhY71>LFDnJ2PB6wN0qMZX~x0Kh@DeE#CzN-cUN9M^M76hOZh z72zAZc?flb=lDsz5SqXchGUuGYOo9%J=Iqw{<0t1|& z_tJe(z4-Z)8`fsu3;dkI+Bt#Ar_BoSfYh_UT3Y8E*{bpj04 znux&4T>ushWFmg3bl8~ThK4smD<4CBkVagDN&QOJr4ek~Q_ObH4h5w_nw7-3?1G&?KTIl_`dF08hb9 z0lK}{M;m2$=$_mW<`u3f5GZsKH)Q%@UE_44nz}OGjV|j$uGs5nhDBp82Gn zU(;|)s7drLo1ik5gS9E-IC#TIV}XEg$ijp!2#s(FWqNd-e=q!i~++sJb^D=N*(JP#1dEH}TLe;T+ z960E~|N& zw~Vc_ z7mx}?(TGC!v4Ive6HT-Q`*=nudzy*u%D5WOP_w=Gn9z#Bt9*M@qIRRzi{q1SNhw+sm&?aV!RgezD>q!=hR%q zYe&rBzD?5ojdx+LT^SMGnVH<7z_4O)!&EeRBpI8fv*c#8h}1lxHr&I}ZM;wQ+>>-Y zK&_qXoY-twp0${OpX!aJw=}$?-SNz6BlxAt`ZVtN<3N0B-QE#}OwSsYgns*!fk}d@ zCyor2z7=ZnT8Fv!q)h?g5?%CFj%<9rV2EnR7^ky4-DDO4|oU+80I4(*7qr04gpkHJp*3iQS6gVklX z>o6_dj$*&5K4p4EC1Q&BzB_*k`AW!cnqx$OQ}!3D*ST~XC!OpA~E9Wm)_*b`?dlRibbFMennS6S1L5D4Eh` z(e;Z1LnhC3P!XUV{U@UQPnO3e5kzk8(%w`IBF$7Wk*k+R2gq9s?D z^{pijmA{c%Hk*~7rLYsRn@}oJVc4H(d((A6>yci*TAoon1aa>6T0`3 zoB=UqawS8}DXSYkj(w9`c+0k;uO%OeHk85>lat}Xkb90T5oNa-WkX`SgM@nEHofA( zv=o+2s#*bKmD)ojevMT5*i(yihU{oT66M};%@Hbmp}$eyB`4~tYZ@#2)Pz5xLcSA9T{V0Tj zUaT;LMQGNP6f-t?k?dHxseG;a8*YJYBdmcXX}fTP=kmrL4i$Re#DQ07Dv&eBOYLDIW@@zW@;78Gj=)k~GGq!gs1fsvMpD@lt}NCVQ25Cbq3?Z9e>+ETy8J z+%I4h05Y)uQmi3B*T;p>!J~Vx{!5}sW2$#W0Lpb&tp2=j)%+3g6^xmRs;gCVuSfNW zJZ#nj3oMNjR4m(z?9@=;1$B=0D#Ei!sjcc1H@eq`9j{TH3yWxtDV@EcfUR5owJGIV zG{KiInKHh>^x0A*tg-8@DVg5*AYmH1)oc17SJs>SIXaX$&$=(?)9<^74V?S3R3Q)X z@%r-dg`(GAL?iiZ6ur2p?(ujRbvJ);{M8j#wr%$d$|3w@ z!yg3T-O4F0>BO=7zv)PH{s;V~*ZKd;W|M=}16`|Ey0;{N*er0e++ty;fBVDl{ymac z-WPCXyu+)@PDg|J5^dvf`xA`?hwR@Sl89+zNi6t;IVKENSPiT61M1cjyK1!hYN?GsJ`q-_gx1p@=5D!udwLudgJ<&Y_JPcUg; z-02S0-dd->eIh=VdSAR1?*P~Wr1hjGoYdY8f!12znhzz^n^&Cs9eBnk44MMGKiO$m zgbo9zx(R*7XM(*f#_H1fm1MxQIQm8KYn?nZ--QRID~xoTBoJ_K``%bXBN$kzXN+O& zj6t@;?VJ+^4X+eA1gM4j&*v@_W)wX=N~{rFs{afw6^{1#YxT5Yc&ml>-1erKP(~(! zYH&Sv`B44AcD$}L#p6*RCh_qYpC3+eJg_vMFrtyS7^$->-WQxW_8E&o9}Ks2YB`ic zjI73~0X=Uy&Wy=l5Fo^r-SJO!tNa)Qtpk31tXe>u4?Uh7UkKXP1nq=~G&)l*?q;MA zZ9fjQ|M6Mv1A6nCN94?{S3yZgg)~A1Gy=))O^uuCHm!4hVCzs`AuUgBQNsLfL(de@ zfnyP?c*`#Rt~uS8FYA+}y(JI6nUTDK15jlgy6Tpg1r95DQ3^$)%rfn7VZb$#mr+~Z zSL4+~UEB{!PyBXF;3Q}xL-#0???=*q=W1}2@H@lo-YKwev=)(XVPhqyxd^S{_rSuS zJx9VC7p?+7cRQthXnQoJiJ{~Xm8`WnI@Vb0u!*sEypk9aKmh^HUB9 zE;a!PE#FjXYlF8;M(-xozK?+&W9P@ur&C+_O+H(2N@5BlP;S~%;GIhWp5|8k?IoVT zjV-Zul6a8|hs7`aVH-cP)R6TvNotf?J{sZ4EPAqur9|>l{O#-Szdt3;R$NP8PQn3W zv_xA(Z~=y|LuB3s7{1PC_>4*`N-bwZhwDoNEx6hqA{T`g9O|mscR#w1e4022%{r^n zy^H!4jQ_q_m4R!;c`w_st~RSsQ}(v|c%3&)^$>Unj>J!h;|Mm>N2NGr&7*=#qZm~W zZ|deZIHK=Gy?E?POfg;CLB6v$1K!Wx;T7GtCso9x5k@t9KZlW-b3itCC53d>4VtTE0;T)o|6 z&S#1Th*6*tz;tXrRDML63u@jEhpTupN^br%&fnbPud4*p*0mfxg)NN^hw5f_JBn^@ zqhu82(TIz2;HIGk*2$`ySGSxZ%knJb%Bo3v!d5U=FvnndiBXihhI~VQ+J&ontENWr zoK-`YLsT|96w6RtM(dtu**lDXWP!aUzfvy2eFA~>0ySDnhrd%!zD3T&G+fBVNF0oJ zIZ?prI^T`7JFqg=Ro5S>45-pRP);qb@F+iS-@ z**H1gb(>UXTu6delW+-z0PS@p_P2NLV`lC=Kyn}X(99etat93)T3ti_^tm;~7^J{+my=dyryk6%>G4#Rtz2#} z$kZh?7fKJO9S3|&j;zOrBv|WFik*WGX2g?I3K3ghOe7yHdQMFEh`t@W6m||@lD*SX zvD;ss$hA{P*%^eLgiPt5&!Sf8{@ zuM7kt6OXjK9yQ>R4d(ZW0?Q62JK&n%SyyQYyQJv_eNE~=#+OBOL0(@g4OBTf53d}D z(E^9jY7PUwK*XTcWxF)gr)3Z+rADlWYD})n>T=Ds$V%i(O~t~3=S8D04ZhD%ycZ`R zyrW57f=lg}X6CLSr4m{MFUI-=taK?8(5$S(Duw$f@(6|8S>lu67&KkiEQwP~j?%jd zc%bn+8mp8=5exNTtUqNHw5*JH#+@bQ{W;1;-)oxC&=RoVT>>`lG)w^ZL+N2Gj;jY7 z3Vxe3ujz^XCmCVVyO(70I~3p}jEr)d6cdBD*5g;2Z%NwE)#Q*#boRvNu)0t)H5`IC zw7XOw{iux$G&kY|BUhO0L?*3Z5;g|~4&!neGP@Kba5?J_hsNW*nO}YwAi@BL9CZDi z;Dt7QsR`d6lyPlbO2GIsCvVs5gs7R+*7e{Tgv>GSXl+1C2~?HA9cznmLfoNAl*-|b zDuFu&H>qP;h09r`*0MT!A%p5g<(AL}O%UQ6j@6lSbkLig6>H`hxwge+zmJf7kAF6FJX`e-XL3!xQx@eD~tr+W{97d_<57Ui=yW4S$i$U z&~GT3SoN?tNP`>mW^s4{so)_mjm^H4J_e!~Da~Xlh9k&)L$6#O!k7YhDKqL-f*6RgR_2k1FLbTRuavuh_4YP}FW(6_(LE z*JEq$mkETDa`J~u+r>GeEN#lZQAUB^_qNBod8k3N&4F2SWKQRwZxAy`Q+i8mEuwdUvfI2iB1Ws+y8JoqI8)>!JC8(Yb`k>g$FpqAN zn1nI^16N+CaC9G_=*-mXSIj?}WEt(|8ak1KaUV2NfQX}Hhxemu)sV1 zNnvtFYv#EP+2YI6BGp@T0g&m!HK@pg{(Jf=uDy<-m|Ab^*>&O$(d(du-gWOq)T1j@ zk*+Xlf_lVF$BIJaLSVh7)&eEmnn*W#pKQ!n*!Ts-Xd%k(k;6^*FrO0r@<8m+wEh)P zS?Cz3xh6w7_tC@h2p7$KYvVFtOv4Sl2xRt}liWfANC*J%C-CyeUw_VuHc9JR#%mAk z+S&r^5yrP$zCFm0dD{%5D))zKv z5EZ5x;6>+2K2rCvoZoKjfhgF&Z- z%p*F0Xtx~Ouv!?QsWVBcNFsGQaVenPN2l*kf(q0++Mp@b&cSNlF--Yd`3?F;LaB=` z)exHy%3JaujxcsEMG6@T^eBvtY!KvMLe8O#-Hm1$m9?S2zhC_v35qVeB0Qc;penCd z%^E#qRlnH_6dA;JELO2(T@l%Z#wI_343*@~4w(b(rM+-Jqegur3$PC4vSiZHl0ZqA zF-7=>_QP7}cyK6bj>oUK(ZLCpKSv2c%4F@FXJ&673weoymOX|3b5iMF9xS= zFwn=F-=cq0qy&?yU3xxXH9@nFqmHi_D+~FR;vPZfz31 z_Fl6VgX|>pBWa=o(VYDMhI9q#HPpYROHCz%%@!_<;%t0E9OP7db7__@4fJ$v(kmUP z?nF+Jc6-!~v592ku&KRGqaCkPc?u&vU6Ug(?XR33)}8Uipf=*xNxdYcV{!a>A#(Ga zwd98&6rlKG?{dQrTLWu*M4gi)6A#BYu!LkvP$w1_5^*x%{ZF`bNt5(~cB8ix`2-Yi zWkG2yJpuzt>v@(zvUMjUO&9Z}9Bd4$YCak$s%5#YJ`d!TOSYbY!KK`xy*Vuy53v#J zajn1_2F>gU(tLl~z&tZE{f;E_=J7Ej zubJpe9x%Sx$>qee$~J($*ND96snc)bUG$H4;h`jR?)<8aEGl{{R3&pG4h4f1 zNbyT9)9h-O`LsWQh6-`3tJjN8lchaLqs>Va@0bd1gs{K(64=+Bm#-*}`Cw+(E?jC` z-b(DGFcbnlK~g>WIC`s9NtX7-ss2EZhjL!C#4HFP?KBuZ%iwv>ME2&9isoS?)rTh1 zdk{z*>bt##emSV0A%@pnzcZ5L`sCYUT8^d~^SZaE@y*Rg`yWQs#a^gG~cN-BADc+FS^Yckdw4Cn+pBAY5;)%)zDQRp>lFk1C*ttu>&L#mnj)Hx^ zx$G+#Ou>i-1F&$W4`B$Ko@g@(ss*{FILwaYhM$HtNW|hR-uZox$*u8z{&S)WaNdtv znnDjv+`}~012WM2=Wr3V^hwVgK6%GE#6A47XAZyYn*-=0A?zaNq4Kv6`W@n>v}v~M zo57$N#Rc+ns;^fme`0?Y@emfOXAY?aKRfFw{WLYTg(}~saXk_{&{7S>Z<`ca#y|3w zpo*8^Pj(3=ZV8;9vs{9<4^mQKeRkGYJh2F$EEjG0R?X|d*s+%FQ2frBy>HTM`nIgY z@ICIWJAFxPe8C9bT2;?X-<}nEx6$z(CWps&*bePDS%neA--Li}YN0~WGEqamw_w>B z^_JvQ^bMqmtIsgVXwp^P8uAn3H@c^VZB4Wb?(i?v)e0O|4ezi8weVa%EQUk*SWb~z zTYF{L)BHxS_s>%EzIG}qEN6{60~`3$mUZdiZyUtc6jtbd3{x+sfzyuEnmFj3Rs=SL zJGH*UGzFj`E|)WxmreLWCc=Z65MzF(KdIum65HbGRHNoa;!Z0aNNW|yu(&-@Hod9! z{~b_#F&iY=ez|OyahON%a~Lj*-p(^)@cjO+$5NZ0_CLmNx0~kmAGTeqZZM<)*bAWv zskE%7&aLu`L*zn=Ir<{>U$Q-NpNxB9ipY0kO)Ne~o!evC?#QXw$u-%a3sR@)H8B1; zG*?m@MpMmAtiXN?(E*rd9w%TBy|K0nZo`!C_b%FYYFB;6VqI+8XUyUnCd-xgY>};3=bL;sega7%S>VAa3L~*~3|Ky;5Q4=OoUlRXOtwm-siOIK72)Jp{8KW2rPcDLY z;#fqQ`nQb%KpThwBuL=>E!3!1Q52rNv28-vfANyC1Zu>`HwU;&$t+;Qe2q|JPepAa z+ef)fLzLk87dn)R`KJ5kMG3^9T`5_w$Nzy21RBRST>~0g+r2O%C~S_kd@gduQZE=g zo5q$hU0{1+7Q1jP1Y>C6#fn#QkHB!?Qg4%0wJR&7DnhPSe+yp~L|%$o5T^#+A(Uq% z+iKcR_uVb5DJ4Z;h{OHgqcLcf2=UJ$>e)2E%$}|YwXO#NrjZO$ z*Z2;ztH50j&m+AXnts)n-E`Du#r8h=Wu5H`wZ7iq04fZz|GIiZt-e-k`mx@=zCwTV z*A<7Y2Y$AkP72Q4CPU=IIQzrg_eYO+k=X~Bv&Of>(QkxfrY#3c;+IqFL^(`}7{o+` zRA@zJB{4Q5GF7HW{B7)J^(zYc%7Sw@E8xrP=Joxmh8SeK9joX+$2I-KeWZ0GwL8Rrn9DYu|0!D9UeZd^y3=Yc zg=?G*Vhv_EFpM^zkvSLz-&$LOde8w8x<=wgkF4&?{A0-&N<>rJb?1`!rjGLAm?R+C z4}{=u^(`1H8KIVHw(UILzFl#(+HQJXfe9~tri&~FniJA%9KN^dh#f(FA7;xm&t@D3 z#JrK13CC%dZOY?afx(A<(wFoq;HYe9EW+A|Vq`LEv#icZk>BfrgF$yOLD78vBsvn% zJ+g2w)$3YQGBiy$4tQ+{iQw(Lhl318DAoL@vF?Hm?fa-xB|suesY?;%76 zAmtFmCZLVp-Y2)&9Ca#Gg9zdvI9Lzk1EcX0^(_23BDvxvQ+U5=TExLQ5_m%TsQp;) zn>ggx&=*ZlIDxHtLDhT=<3~c~hLN_&j7NlbkrbJxdi1BCHWhnE|6p7Gs(Y;=E_cEc z5_@qm8ZDQZ`GdfIHVq^?Lh6NQ#?VAhV3R8HefVo&#;SRC<`*>1;);Vq_jBqC&;PG(#bvvKtn9tEXM4z%713iTF=N(cIp^>#MuUg5)3oTW8;3O+g~=wLVh z*ntDM*%3kPevN=<)=+lh?fb_UPtT!^3ph3oG*ewwaX$=pC*BDb9dsMrelSF!r%z1Q za^vO=)cAONKg>S9IQzxF{^r-3#6{C8acc>B2#TL@c#LV0 zPEa7-LAt{L=C3B$@UC$2U4Rto5)_caluq|5%n&7xFf$8}kCWMfnVXP9Wbe$0x`#D& z`-A1Yyn1=CU=gWEtYD`ZN=#y`c63{r?wNT}R@Q{A4u?6$U*aAQV5hNQ3T;mDh3`L* zFO;raiqzM7IiGarzpj5Z!vB-WaQZL$$0zV3FQ@;na#xETPp6q0fsU0(S$Gl^4dr$$)EA_|*3_l(Vxw_XjyR-C~A_p0m^qJgVnb zZCW7M9DN041LySQ%_UsmJ9zY|DTu`N4$Z<{hEIJmM_iQdWIV+Z^2 zHf^2a^wdOpHd}EIF`%72Cu(eHCGrmH3*U+fpuXEjpbR_th7&}GQ8*I5b{7XjH&jU>Ki?(1)Oj^$^@RN z5n~mQ!@t|$81Cz$w{XvRYZphUS15*@!9!MUR5il;4vtY7qW3};&_^yPP`!}_*3C~w z<)WLWQ#*fHQVqjNXS_Xsn+Psla~rM<20{ib)h@7vm$SX)*^7e(cPbD=lU znRY1M{_Z|~;_(w7c`nx{-4v>N6u;FFgdiq7h3X89WT^XRz)1ED;2B9u%@CgL6S1ZT zF*-a7VCD8NLTW-#3kC-N56@v=3*pJI9Ew#fnHW-oJ4yfMo`TGZUk;sbR=vb5Ksj6yy1#=rhs#?CLOs@3m^R0`uy6Yrjbq5+>*<`-rUe%|QSiZR;xFUC(kI-e!<_iV&F1D8qk@df4S z;FMUN@tsB^=45GPmO+T?`AYYJ_7(BhS-1K5LCtUKYu(Cs-+ukM#tOR4HK42AHHPY5 z=MOt5)uu|dvXt2`5XzZ=gcsAZGstw`?skJbrGs+*iG|Mz5r?l@Ox@})V-~tf*5JFrQ(Mh@b&5edTJG|{($fG{Mj^9irKNQn`s8uCqus`4E3pIdpF9J`JDhVP`5 zxmsV}HP)tsMYTYH@-)pFK)k&kZ44FgO~2p_xVp$5i^c1SLFtb&fEKx39>@bIw%aQr z0teHOJ^6j%lho*6e(~)G>nMt8iC_R=?#94IAl&Eft@~^( zeSf`ezp3AJ0JCke#!z0}X7xKrThWbQ*Ik7_2TEcOP(eY~h^)wNt5s_go0%ip<~^mm z&DIun!`Um0ZP*-x5lhkLoaq;flif8Q&D7CK9|H`~N0a3A8nBBPr`LmKa#jf!R$>5F z_rfjUb&}F7>U)3H?eCJ6D=Zp!Ad%Rs1cZ#H1tL-msdHp!aHij}yWtdiqV9}-60cXT z6F2}`b$ctQbImRoeSf`2jA(*XT$?xqb-dz^*YE{drlf9rd;g5T z$l~U^kc3HEPzu8g`FWUtgoy@8L&urvo`ZPhx5B z(UNanmP6KW^!;}u-y7S9ax^sngjb~KU?07jk3m{6@O$SN1SQ?!yA*@8kgddAF7pN_ zkrhqfEU~9}lJwfnabnNNo++^hN_s}LYC8k52E9*WQxe2T;wbJH20%lFPu1FNk+>bR zf0C!r`PV*Is4R(Ba9#Zf%SYzSBgp&SiDxAFZq*whM7XP~8#0FQWC)&_4R^0!gI(7v zII7vjiDoi{H60rYAcTW=a(l0FZ6T=~6FXJ4X*>s%NMx`lyL2ohE)~$0aJe4n*>&u( zNBf2W&4ESdz+zYE)<^H8r-D@o(0gDeMXl3(q}2xmuwY<=RW4zXljoA*LSQ+#GGalZ zUZ9Gsk-?8>mjVu6H@{!VF9!;FXl)N-m|GEuxkf@+f-&di|Pr8*{KQ>05C} zAANRpHNQ*CAM|}`M2WiAVgMO~LGV#D^R*}ZKwp1rX9@eGYlhST1g_S%Letk982Uf# z@lwBr5ZsDxz2zTM8KD0 zS|K-+-8U-*iVW8ayU63Nm>;m)kl&$LLGl{PUShZ1n_;nnj13aa*-vG|xO7!r)hm4( zu71wS%`P05`&ex9;QEMx4nORbT5_zp80lbD$SQ5M49)GWrk3m-1s&L19KOj5lEBpa zymp^yym`*1?toDHr1QK#Ox1dOpEavuyE0MJT|9e~1SO3}v!^%e$BRsy2Te@B-?I6; z&FZxb*e>EZ9NFRxDGYLG=|Vxz^+?H(fz$+*OBf55CzF}D)-Ky6Iu)V7Z2|&5pDEL` zUSQhI6uPv?Si+st__36r#QKrCFl-{y+PQDDBpEmF%(@W`mypFh>}DONm;*#4kMw~ekE!4=b;e~x0d7o?@?`=zt@fz4l%t(M5O>T7&0}o6rIeJFbrw{iVY)@v4_M{z3Q5s#w=Q!8_`rf=Bvt zr_od_pR7_23l1qhJ_ZHwS^=dN7#f+Nd91p8d~6_Bq?~Jyb!>Mhwt9pnWz!^*SNYu^6n_w@;~W+%T_EhcQ% z^EkQPRIlJvooL2(-KkquZ^l5XCRBT=QQBzZC^{R!6anz)ai5^Gl1ku8teWs62jgK| z2@k@Uc06Edj%TQg#?H$n5(5r+p@)E+uqJf951PE!R5NqQ1&Xxc@K7il4gWYKggOoW zYHWZr&~s0UjlZB)+@Y79AsUVxAm*uMsA;P1-82We}=5)o(=o&?>( zQUt5pu>HN0=w>JP&MfA_Z_{^eDI0&urCK(ocqj9@y(xmm0Ieq4*W*de*64jKtiG=d z2dtls?MW{oAl3I>K5K=;ycJ!7Qrt0Y9=-=VlUCTEdTg^5f_nGK;Gx57p;U!;P3UJI z^Om)~rknd6A_h0iq$Y^`#O#IPG~1k-71!iAYtTXMlkrkU%CC&{8D$tB%|FhD^)UN* znHe+oK8w9H3ifD})G0dNnB^p;d~hNe6vQOY>4y@7#m0la3?z*tjF{zs>7hGV{Q5g9 zbMU9QU0*5V8|RC;q-Z!~7h+P*Uo7GA)SeHAV5jHzuLAxr`}tnlxoWfJrZ9{HnRAO#b_yLkkZ8iECiUp?UU-{GqvcF z60O4W$j|oGFS~f~?rXp#Fcn~p2ScYBp}a8Ru&5%k0VY>a26?o}9{!Nukb$L{GB!Y@ z|3d~(&C#LONyon?6bGv&?9?=bleR6TP{W*>3o>ei{Hnfzn&j$ToH2!U6xZ?8I#y4F z{0EQz_(|RlaGn$NFmW@@6*ZLdbfq)K9zg zGUq4rkag<0UQJ}kko8$>dTNfwR%7nC zKVQ^chq8mJP=)o#!-g|*lP3{?y;A{`Zrhjj8ww#G6ZmrYRWNYhng*J?fvh>6SL8Z> z&^v9cfCTz!#t?lLEchiW2i;{j_Tp}^MnC&0kcGFES`GXiNjH0kAE_1+LULI=;Bw(X zVE86r&vvsa4W0vPMbD1oGSzDjI4gTh*AXlFoM)jK!#5pxoj?5b?|=I0+fvt(qp5ZU zMhY|eN-RjGjq`fUCq2h5UpqkuEmTl|nn6+OMc)*>X;z)OPg#Sz_&tP>W36LA(yI>c zpU1~6n+eHu1#;<5uL(>Bw_ooN+J$Sz4U?CZ*(KIpmLPGuG#G1{XD%|*!|R_*OiOVB zAwH+S&Wftz7c8p(_txcmgAm(-wx0787IvK*{HaBX+<{7w@yDcVKvNI*=Gpd`G(?upiitLBwqP?4l8GYymwGW18wxNh{}{jpj@ZbXQf zj~!>B4wktOA{I4q3pN5QKeo9;aY$i8iXIo#FxM&bfYLmyBa4emwaCo8AqEILAua>U zI8auowBZ=^<>)=SW{vto?na>hfRG;;hVgT%=`zHk6OOgviYO#$Eiv2Qk+>=9F=}iy zo?mSt8}fQgygA5gC2v0W^3Zi8`O)V=h{Iz{ZT(IG5#rWRJqMf)y+k=kNHhl?bRS^MJwwFd%R#1aqZvL76)|+0?givBfIxRFKao># z#sP_G%$zRrv`u466!CbA33#t>-Gib)@y<$(yP#QM z-kj~>M&r-6t2dv+I9M*$>hV|wpN|AU`HN8+<=d2-Y7ov~%=m}-3NXc3(BdNzFzXS5 zwJcbM1=fSAQIrjGK6K>+iJ(9x`NKM4XkPEzE>xaFU-Qc9Mom`=j{%x6L|+8{3yuj< zze41%8P(Oext-UV%m;K*8i~W>W8;#Mtym^+5=ze<0`=9_`x6usSN&!UJNQ`NywIQKEeIDhb^%{nk3&_S5*|3@0h|(cZLxJW zfNbjV?VGXUS5{c#GUpZ3D^f_02@#6DO|M-Qe zY!E)EECLj-T=I6r@-0#+yWLC)-G-xP5Zx1~%_H)f-oNA%G=2;QJI?EyQl0S+G@SvpxHlvyN{33$UUB}pYIeiOxluT=9@Xn}eq8~RkWl(nEB0r-BEw%wg23>g!n5x1(n%@jJ7?`L#< z)*hn%I@+{wd%+6SL`qZZ4vOlwO&057X? zoQt#E%hy&BKMMo~3KZQesy(=EP=z8w|HYol&~Oj*JB~0SuFR+)=yX2&Ur2`>khpH2_LVCcEd{oU>yQX!N7Hy1L%Aes8rC-HSf{ZIKig(f^7+ z$9t0^ombI!=}{5GqtBb8&5hB0Y;;;x^YQxn$6v7RXp*0;!&7{*n@whmX;Mae>94=e zlF`S~G%xAd$I^cABA;Ir>B(so-QU=FG=Trze4ul9oiDO!T%~z7z>Y^zoK2&=I!%gb zl4n(s9xbZ8pb@@LW=UKo(+IwR*U%<w#P=`bDjC!8!;No- z8xKBgJ{UiI{QttM^-k}2kxg(Oz34p6ruq4R#yL(i7_lGy*6FOTM=#>+ZWT>)7(6Yb zqht~<$|Ne2&q)!_qIfe-Z+~>$FIY^Dogf1u&<_`0+))fSxX*D(4QNw<2GhoJL)m0zhg~btCK} znw-XIR*s_Hoy`Xi9z5#f=X;C!JTI!T>w5F?8a%fjZ*@M$MP~&Xe}IX?(vzY#Khy=gw775AYBc+5|w z3pc;rS5G%L=+W!Eh<>gv=1G1WeL0)G&8NxeEU8ZO>CXV=vVuuO@apIAU9`Hr?RW0y z&dMY&l7GSy{cdhY(yMzUllY-bXwvXWkxX!-dBRaVfk(KhvkSnE>Qqp~?P1g_VcU-K zFR-*(J|TRDVYt;upA0%Hu1EJ$pYY39v*Zlk=|lGxpHm~6=bLyQ<$!Z~h|&x;9_9{9 z!pG-DS|y#8f67}oQ~+?-2QK)lWnFybzy3--+~0pOi_7xxFd9B1Kt}U8g%i3m!N>RM z$#(SHl{7~*!Ct3udMf}v9QGK7n!u>(v>Zfb9-RZk;|y?T!SFwmO_NV{rzLc_e~2B@ zbC_Sr0PZrm>M9b&pW*z3NLc1)w4q7F_)$tHKwV&0ppV*6BYrWEX7fdP+R_LgYm*X* zOiLSju8jbO$(MJ>EzRVOHmveJIF3L*prr%oNS|t})3|(ho`r4r+4~}&Cq;GP`YX~i z9aR7~!e;`qmvjICXo#dhI)GH;ZZWEKRvZd>>R7_mCqR=2M4*!|6;a0(r5o}^e)JJf z5Kzf@QZ0ZgT|^%jfC-3gX_i*_F&;|nDY~0vr!nl;^e*&MB-NtGqAWR=zDCE{2tR2V zdJuU+67{z_D_8y`Porb#`Yf&{rwGn$adwmxy8CnZ39DM)UmG6o^mfJv(|ZS_%l^T1 z4Swt=uMX)2{JiY1kIH;e;7*1;m-AU#RoFG4;`u3@mvWATo(CwhyrPFV%E8{+`iVPt zpMc^n03QMEX>+~>!iq%y?YDdHqaNIR$=wpRBmabb1BwFW9{_3}$3+@F8f{S9-B(CN zWpCGi+CL}{7O!8uetq!e`NrYeW$pJ@ZuI9-nV!vOFnCN?2^>SXj&M2u_2JETbQ5GJ zB{h9tq&ZN5^w%W1lTMjdJgwrBXZT(GIfrNe*xRG4ObHyK;3-@?PafUh?DL7FtA#rU zlA52uKH|}Zt7-H8lZ|n7hDZUIVo@<+eTwV)Wsy!kz4|gQXmPkb<-Ok1{e$y^9}m}_ z_4j{zc6jgdP7fXo5AXH+U#&aX!e#?5Lw~?j%yK|%m@sr$BI!;(#3$6@)84`PS|9p6 zSl@XDy+1uzKiGVB>HGO#d+*)?yKw?jfR8`Nvvdk~YEcTT4+rDlt`5q>d-#F8Kf*hi z8bF6XU5+mYeV+HhexEj9+9l;Ao-d<3R+ZvI68|G9%=Hp8;L`VGiUDzw6oakCceG zln8h;xNd>GOo7M3!-?o$5f|Wx;YP&e5h0^b_o^6=FR&9`{i=%u?IZ^-Bgu&QLWnI& zI!VZRA$6PPXEHS0srhWW`wIF@kE5Qy)#K^(6^@8^W0JwrE-!fsBfdRx$XOmwyZFL= zwR}#L|TQPHq8WR0C#8M#N*ts7M+Zr z_8HKQpiM980yp^rX*fKDg(lN3(sbFHK}0AW@j`_CXzzxg)cz^1I7;wGI3HdXbp##c z87;a*u)-P7P7>Q)IEDI`^}!wD&dtT&KuKbZh)5!vuO~5|oYTWG$@&z7B zFA{exMOoxqJVj5Z_r;#v5GG74Zk zUSoJYC-8rKojn5?LDq*a%1`N><)Y*y`NG#P^-GXPHa9}}4$O|>@92=WBS2{PiN7vK z$(Ll}S18y^u=dyEX>~S>h%-_~B+DHu2js1sCzJFz6)p#^eg|kC5Fpf}zxFo|kv^@; zw|%9bHMGI;`Y0=W)r*TFzUWgE-ywFR&|T4aZ2K8yc$-fWe=K(N-Tua5G}a#sAR459&CAis5P01EkYlz{~L z0gtwu-6%f?8ud~IK7f>dHja$YNB`k8`Om(Pt~MwRBA_7ik^#sN^qkEI_|GZ;Da=k5 zl$P&jahkm&p1f!Hb_e$rgpuwr%7OuFuhS9P0+kZ59ohyFoOBmcyx&F2ieP&4ye!kB zS)x7AjNO!8JL-hM-^nDg(^{cL6C903PJeWsl4_>HNyQawl9Eyz2Pa7f z8ezMhQo**d^y^OW@0H9UAac!<0B4E!E*w0m$^0jYiu>PQ>`rw{?;i%3@q!q4z*K04 z+7D(yaGvLB!J4KfP%4BIqk}J+W;cbYfDXN2J$kB;fE0A*a z5F-Z3ov|HlZkfkOskY@>4Th(0Jke9QZ<)9L0(8sRYC$*OYk+eJTnl(T!^!CP%@xA8 zw2thF(})=j2UHou6Afr;S%($?FPVd06Y{zT(lOGSlJx=!Tw);VAhfF-Khowr0Cr)g zA52bxsEm-IaP9kl)!=P_9hEcJ?}RXN+<>F-0-D;^`E3q%sSPRWWFnY}GHrc+v3Hsm z)nri#k1oej^svE1TL`OR!a_F891s=|Y7im-h0Avh0{vv!Z{Eqi)xTh_H%yF{>ikAfPvKQ-QV7 zS>gBNXam2ka+=M!E*HtHy-tuc`n~2<|C%s)c?&}-i4np}L=C{IkMaI`_kHly`Ds3D zZ-^ING(p)eB2KHb0#}I@15oJ|%$2lY+stC_=)wJ--^I~g>g%p{Ci^GLheeNTUpNJy zCyPAh(`WYoK34JSF%jj75cEpD?|Yb zU!XRz1!Qo@u#Y|b%mNCVltv%$?IN84cVm>2ZM}&Q1tWWz&+wN4iRuq@5G4W6r&9(V zj3mext}E)%20QI`pZ5{O%H%V%c0z`>VMJcN$LG-?0 zyA)i=Ns*y?c?94{Ks>kGsIwbTKOySN{@^Fd{@0u9iit$penF^rI&a-J{kRjVUQp;i zOHn&Dn&w%;-)J4aM%?E`P`G#^AVn{r9c9(F%t-EIF?B)TlGg2$2Ul}fv@n3 zorsCb{>MWiCHsbgoW=7te2A_Eth;vRJ%(sV7Y1%|j^azVLhq#!A8@%(`yNi&^!gq4 zrx{qS2Hha-i#dvDB-KNJM}p1#yXUqiOVJ#6(ea zO=9UP$vC@zIKn^h_ndJRpU|HjRC?hxm3~gVl}8t9nP#sBL!vQ@E0Q}<%CAx~BXmN$ zPoOt*ue^#4Xd3EU%sc-m1T@p7HE7`kCJDJZ{KN5h-mdSleVQb5oDJ&8ws_-2NQxtRM|QU|triS`Km(>TKn>MV zp5vXY3NY!HTWqz@c2j1%#q`Q)prk9hNbxs+DoeVLC>BgqCAm!JgQh(-9H2QYN9@5L zJ(c_M3UI6Iw=N2q$u8e^q?6>xjU$Q08(fmR@Ar-w<; zV>f(LDhLH?!_8|<8+dIqixmV#Cwv!wyU=Y*%0M>=9GKtkn#g=*0%%T9=!s9W(IWt0-{K}kXe0sPr2taELV>1` zGx4n;%|d_;9y)xI7}iPzWn(>Byv)%H>WEH0Y*`C(hmt9M-zqk?OavP%&dC2*VAb?N5uge7Z&~AHL{4(n@*#-45wa6%^88vVQr z%u$X|{gReQqhi#o`&S<-@A~?&)XP~PMEFPUueue8bUzAa=N++KHneIG#`0zo0 zq%&Q0dUWbM9UkDJ zNm!1jqbwes(07^7J|{l{|KhFu1CX(V3>1d4#@M9EF0~m zNP3hkZ;l=eXs^_Gg}dSb#zN98DJuFFtsZdymF%&aOh>3nyik({`yW83fV{ms_n&)kBCkzAsjQBS3BfW)9FVe}i@K}Xr@!MWR7TMRcJg#`^ zWZVRpqIj6dciJ;9u)R}4N12i3tz|Iwu=KqV@1a};OD+7d5m`_~uf1}+743A0w^Ikr z{U~toWNiU0ha@!D=a^Kju2&B+!w$AD5aruX$%UTJH>i(2BH5q68i_Vj{MOOD_KIv? zu#I_y+5+zrNL&yQ1M;KIkHs(A`)tm|&vaC$=|nW~FtL%52poG$D{ns~{t@Lcbz&c& z?)p5L%^VfSMFCJ6NV{5tK4O-$%=ix!krg`OPCEvkC?$H;tBmzF2X5p%Yh5Y(Dgr#; zIl)UY%BpApHHm#)HIBM&uLtP1<(hW;*EY7jdL;;S0jQwA-W_+n2vc7kbI)QI?nQ|a zCU;*w9z2PDOv!1jfXzw1OcD_sfp`JktWX>~&Cz>X_U+D-$4|al3dWif|C5fj1;a5d z*ejm=NYf1Nr4(JJ=H8`%IX42of9mda*;t{v=v?s~@Cv#jmVg2GBbFiSAZdU%qRqe$ z`W{lUjCa8d==Mx9q*gJ`m{h5=3-Vk?|MEHB+Kc&szlFJ?>Xft&z%jry(N+a48@dP9 zI)#_GytcVp{ufe-2D^!3py;*LX=o8V5S!cW=^`;cqS-78G_d($ z&dL+jsSt{x_2d;cl2=A#z^H(+(l$V>@>%It$)z6l{6Z zH_LdZ(w&3YM9W6it8q_jy}fpR8?r_MpUNOAYnUuClFh>&B>9S-W5!@YKjAT?==DV zlfB<+KyJL(sniq;_-wnKyTW&2Ym3|s=T5n%JX-;{i_+?PRRC{~%k&`^ESj-$rUz{I zgrf9L_2wbJ5gx_y6GeLFm*8%41KtKxgG6zXpW(>V%tf{kkoAl2@{@D|!VplobG$2x z{LDoqmLPm!Od|eqAEPFn($Rz%h#Mac;dMhcbO5gZ{{#@R_N?B7;{>3|;nF4?sDO&5 zzVX=fem+Q~y$4BGo}(^HpwkxzbE}{8GI7H6y}O&yXc1$u|J>007e!u{!y|wcqURSN zorqeVcK)MDI?b*11^pqbv$;E!{|?b~ETBCcz{9@hqXQpC>|&Bl zX8B1u`WSzXM_E#>5gBna_ZkEW*ZQ7|gSaGgP%XD=xCUDBc-& zaXu{aF>uJItT)S?H7?VTV{GFNmzhkUwg^Ztx80Ar2(4~(D4edwXIn%DvJ~QoqcV90 z5=W(xPUI#lkw#nWLjwCs!fnaID)n-fPZzU#*PWSEU;*6}QX%T%+U^XZv zQlJzDzkPEbXs;V3h{Iq8H_&nLGef2|=nBZhSvl-lBY_XS8bl^ys_#$1G{MtW_hn-9 zb~rBi(&`7!r&k}0s{A{+!X&baFZIS_L(IuzCT0Db@kfYH*dw9on?vGCtAh?wcd{fA ztBBU<4f@Mj*ike#o+PO=%*kJ|<-Ej|hZ8;DmAwMBmm_lsB3^llx%5Q zToeQMwI^om5US7Nx)UK+=u%Pl^{{7!X_>Ho(AIh^gz_YeBH={B0AtUAT*W|H*{3(? zDhL14qmT#nyNAACvU}J&eOFVLHT^@=tN|Ckf(SY_$G{K{cKt!tgf?xSyp#D+=gvKE z%aN}Y_CAEya%F7krIJQF&hwD0@@Mgv9$Yo#RJcYH9mD^QqYXaBHDVP~DwGb;u_~T` za8-(9M9HFb-H(N+3N9DUAN<3#0ym2N1(hslh)kQXCGUot3UtDOz(p0MeGDT6R&8ow zPJZ|JFt|cd*MncwJv2AA>f-v0af;I^IGIX;Lr$6Sj{yz!yHC(Mbc63h%R?iAc+3N_ z9;Ei3w)=Rv#B*d>xA;3mQM)kQck6vi?-Yn9Ko}Zc;Fd{Av(MqN{yW@>LV_ek$x187 zLOmG-#E!P3)m8Xc)m?whT}B`Fi6-Af@?b0D&#KqafGP5Bh@BB7iAY?E(fpwE@7#F5 z@&afgx<*;FUS@KV7;RzBxE1o1Ig;c&YrnjiNHE8E1BJbQGw}OcGK?1d*RhP^ksoa7 z_4`v5aaPV^N&`afL}Dp(ekmj)blUl!o7kT^vr*d|0f8_XhR3K~C$yG*guWgEJTTNW zpe>DcvOhnX6vQiv4MBt2i8Z{EkDK^EFn_;>*RUfM+FY`WC`(E-1UMpWuMkf4qs%iZ zWO3K6So2)f1b_Wu_oeeg!WD~A45-E;oqG47Z{b>sV&nImW0ZvIlGUyNhZ^pNDy6?@ zc*^yXUFgXW58}?syvV=QVh`$xcM$zKPJ$YXR;)(015-dvA;zk#)c)7eSTVE-zY{SfPdLl zlq2p?N0!K)C4qVLdcCBHnnQDxsL#?A@hh$FmgcZ^R+#;2xi;|B03jR5J;5pR7$GTC z@UO_;7V)_`Qn$HBjux{|@SH8QqEUAHRI-a==<#hG8rWJJI+O?sx%oNgvXId7mw_0- zYY$eO(xUg%!-3c^Yispvvd$RI)-+*H;{(Fn!ihDf>R5)hbICm!p9PqRS>u)ent*E< z>~HZm8qi~QQz#Zd;ZHEtp+swERaztxx2BdAx1~hBOoUE3(5Y?oJHb9OQb|BML10%}^^?6^_FL7SEi&49Un&nsz3ZV+k z3IVqxBaQiA3p{F=B9_&@^dxJE%;M37#8KuLTqG&r06aj$zh>zU7R#&QuAx*$kL6JA zm?>V3e}GF2{V4k)?`_YJ+G zIe4d2_qFBFS3Xhf7z;-mRreae3&g4H+z{;oU6^13QfZgf#ViQ|Uz(5GqF_AUwrE;a zrEww(9q$$Ueezi+?bmbDwA*Lo*Rs`V6{R{?eIMo!#G3}x#;GuwDm!Oiwg37D&O-iD z(!ciHx?YeM8LECu$>{Nw&*-Xo#+27Xj{CThZB-MmreZ6dk2yL17tG1Q5a$vS0PHE7 zYEUa^(w=omP>jhO+NCovkdqOSMk#-pVe>~fQfow^*d5mVq_I*L$FzE->JZ_Mqs?o;I=W2xTlCIotA zX&<-G{U#$l$0sgjpqLLi`PzV$AS5BgvvUyeqVv3<@ugkM=+~!=W74_%N|prFFPR}y za~8JXl1JNeM&2REDVg=ufl=mm7GD5WMcIOow2Y6%(Uw#m#|0ehG6Yri`!du0DFCW& z&S@v(FkFx>W8`og(F)|9MEenw?qtC-NhIRL`}mtE3c4MtJP3?6t{5x)O6Ey}sh`2N zcQ9X;DOohy?Anm1N2Q3wIfXyLKk(Jwq{xk~!R23x^c7`V>N#Gi#@PB$7h&K0{KJES ziBLZw?Rl9dCwcTf0hCEiJM|KBE&-$L{3!oIq-gh5a9-B+GrNLbca|JiU7z5{tX9x^ zhJ9JP6#riJx)G*Fba!sbJP!fih5&T;CASAUwb0$ZM&4R@-?tgq>4JAS*)(ut|6)sN zGmeK*^DK2b!+4lBsLf%%=J|W29|d0bcFkXJ86)O!Phl zNxg(0n4)q(Nr^t7HWVGy|MBL#0Vm~6XDA>5O-c&%D0m?%Q_eiGo6Q%R{Qf&iRPHVm z7_-FuPSwjGk$x?wzInJ?qEj~JF$RwB+oLPmqoo_-*Td#*VxCsNR=T>d498U@0c} z?Q>#fSVLntUljAalu)CXu9q8YH>|vQ{*QmYef8JpA9nx#3iEM9PoG8)HfuZP^Z&c# z`Muw+`pvZYWH)DH-kewvG#tP?5btI3!BYnhl45)X0qjzL18;S z>2#W8muU&t_a)|_xI9BeGjOSwuyffZvIA&1gg<^+V3_SC2z+xCE;^Dr-~<&Q5oQuZ z6FqVt?DXmV$k78Eq)QZ))UosYe02VRDCCFl*C%E9V5q6&%Jn-o>%VbFhxP6s9DKRI zad1#!s`i8I;NbYMI|w{RTROe~O7ms?j-N$m#r^#%o8(h23(>_K@9^K~5=J>69$xPM z1fy+i4B>y{jl=$0*HKD1Nx+bk5W~)_cq^q-)RZ2lcypMc@H#{r$uEn%(p(NkI~3Qy zy%mI9;#L1piVNUzi+IW7_P$`e`)T;U~2nH6Z5QeY)o$ z*>unENQwnq@C3EHq>W_7a=vWqZ(5VjFpE2MaIlZ`-{1srdG~twthuA`s$Xvn=Ly>H zjC=d{e){Ti^@#-(AI$`@_@wbIGjTEG^tX&)Xmup zzs@c=wGay5Y9lHsK2>tzaX=kurKqmJbTMJRpE<+GBoZ&BXVj#UTLs>pSmi=2lkfLl z{qXV~_We4YRfu1*GThklHG4=q(Cgi}A+;C1J{;r2!%OVgW0A+&2ZQJ_P+lD2UjK4* z4?4l^(=*OtF(Q^{Lk<#|MlE47sH2?&jmD=`6$jg&w^Bb?{6wI0N-7M3gVq~CriLXU zI8#405owLq_>)_>E*R$D()cSplqYf~3skhlw;1Ezpye5LLVX;i86W0b^wisbw)QWE zfB2^DjOaYkrIN|9?Z4J|ANrL!xt;wR9;C*+wp-@~@H<3gYJ(e}Rs%W4(ptxgx-^G? zPBy_35{a**U*d$N77O0~-QAbnaqa6n*HSj}r|G` zH&u8c7uN*-&KN&-93BGrEBl#bCPXe4$m)!$3sNmRR4d|*f?GUF%buh;mZ%dlOd((T z{@tE0-2^@S%+&j5oDmZ4Qsl$d-Kn2KI^JUs#N?OD6W9{o5Whcu3rIg&11h`(25Z{i zA;>QSU2kW5|0e+bR}P4pL)1wP>p$)P^xNUu!EaO?NFE=YYwe*|ahc98hU7YmjLACi z)d0HB#H8%L#5_H49Pl4HQTAGE)=iQcNwC{g^5_yyn>>(WM{pT{WL;`|`Xx;ICBheO z)%X$)MU`jp1-k8pZ0rH827adnOEULG8{Izz}a@~b6$RST0RSJ==pryv2k z5rPe;8x$^3ESz-s+vecwtLE11TsF=XGC?EWBPO8eTkb~i64gHkGX1Ih!^jI>)H+R1 zQ2y&~e8G|Q5*|hyUp6=3zb(e{xA9w$Lh-=h)e96WAvIGcOrGGwrGk*O2xH^21mm$X zPMyMjlWPz*cyf@L1IO-@bVt+3Bs&_v9;00-!`3 zV0h~b*rgZHw8vwt>5`qHUY5qk9+{+zA5naMC zFCTAg4EW(+;OB$aulezd4asB6YT$SCDm_E~4(SScw2KKB_7-EV*$~POZE zSm!LO!wCYniS8|V+aVWwyUEY9X_2SXrw>OPQircXtuUN@fD*`T$HKn(`H*5_f7$rX zHPY#2w>pi?AEKrBja?4M#w(t)e-l56wMn*A`*%;nEOfxfF%*TpG-8;aD_noe;hk}_F|MA{+ z^e(W}JaO-|@Qi|>QFq3f0@wflS#M^*#b(%-=*Q&fQ|f$(|7hVEgrJGhSy8MtArPia zPy+>xVY8kCwzaL;K!cVX)2!jwY>M}pY}}074>X#@YTF((XhQ;*q-w<(LwcTciDFpy zs0BvXf#|wbcb1|)%7cw!HjR2f_O5pXwu$UEM}aMvvejvu;L|3P%=?#;at`!XodJy!LbzX& z!lZ?cW#m-OE`VY%qfz}u6@{6I%&mote@MqnsXn(azrbYB!=*$M3;N7Z0`N#6F+>Y4 z&}B4xrtEyfvM0Kk<7bhu+FaiKLefl0a0aFsFJh9xm}+5f(g$U^gAS-*|E`Wu)C#(PKE#Nr@Z^%_`Tt*slDEqx(%OyuHy{ zBKcQayv?)Tq@uy-t+ow1gPWi(&5lgncI=gOM0qu zMZjL<--c7L*sF@c#v!m(U8yHNj1(wDQ7r}z+2wh?xfEn&8#?xOrGQpEu42tfBME(s;^ zDcC`eF0u(T6jLNI3-Oxt<~uqMee9nK(yPKBb|}G` zY#HDZxqdhqCh#6mF$KU=x>!lylo~Z-9K;CTLr;A;sZqD@3yNbTP)aFzxqbpG$9(+{ z<6hrG@4{Vl_l@psKKjO0p2f)st3e}d;}9_Pv&Om&cTII0FcY26Xn_`xjA0;G1%XIl zy3ga3r|EIkbNS_bNP%zUgB4j2@eX!C$$hc00Bn+ssGBM_uFt`Y<89+t5fD>u1>6%Bi~F*C-i-;SzH9?%9&vXz%IP)((R=rQL_rKV$2AB zOIxB^lBC_|Eg)hXv4Axe=!#D%1tgVkKDyA{l8T8iK#Rx{J{7X}0`)k4N`y+3h=Lrs zqBCg3sgx);7_VCLmtC37c|M(%KK@T*0uhnWaZ^H*eSKfFc$9X-SnGDhFQWJ998KK# z4tHyYdNPdH^R-l4_gaTVY}D94=Y)Jj3Y{5@UfbO1l%=>H4?b|+xV>go9KBVS%gb(G zYw+-Mz2G}7*E=15Y{mmo77V=05#wdB8aPSjExmA>lRp%K@jQ>@o)3T^mq$ z6YOs4Z-rSxvHXDk8z?%&Vui)bxTxJp;z}vFyg$w{bfBbu4J>75?6+)Owh)Mu1p@Ji zQcqA}qVyO&itsT$YH{j3YCc0W?nVDP8kPB?m?XRCiiC_z;D*(YhSDy+zTKA3X9JwC zZHkNh9H%q(&&Yr}P)!}ur^*RCJ`VLV_+-nm2duP!rbiA|v3!av<}aP9k6{DrJm2y7yw6RX#CKewYb z{o8_pvr0+!+1H)SRYfAMp-Z?a5O4r@Ux{!V=372pMAf`EoXE zxiv0XPPk&6eeU=yDv;&%ez^DKO${S6mpIa&ZD@}X9ktw@AS8TF3Wg72oDK8?iA41g z7fHnhTey~IxxlCpxVv(8Lz1v2W>unLbm!sY2M_wftkp)`cG_K1Nj6On__W^=*rT+* z*56M3iO+6GN)Ob{ccZa#c48g`wgCN&H8c+GDZGBn8=0l(T-MQ-4z~UQMY1YyoK&7$ z7y|Pavm<4Z;tf`EY@^F7N}Mo8KVpG#zmH21U>_PC)-b@cCM-<~@u1kND$uNltcUh# z1=5zetapq>$E-ehXvzyaHdnS=bat>a_)l|#>1AkGA|onu)Wq12gsJ{}nw$%#uzQRK z1h;g`+1W&9$2kl{=kIMbM2@Fs9ym_T+JhQR8ot(Z$ZdG*m2e!Go&(d^d4>GcLkj-f zauEr-ygGJUGcie7Wj%BOX$>82UQ9py2Cb;i9xI^uJO<%+AV;Mq8OPUl^+ngADF_&v}r8&x6)dxAN^C>eQAJUg1p_$ zFDO=$<2@G|Gg(gqW38c51y&K|Ui92cQ7gkgGbwqf|i12qL>=!JCS(Pou(4l0j zbiug5X_%+gxJZc**vH*hkDc>Z-^Kfk0w-vZ*G+2QRI{g{3_Nk>4CJJ`hMic&*P&+F zHSX&5NkH^Aj$YaS_Tt{34oW9Lw3>^$_khn+*ioi7ubyu;^2mob1Baa+R4agv9;|oc zDD+;p+YgE)E5zET=+)j=N-Nset8ZVtB~P3M`;}uVaB|&}Vgx*JV$zB=Ho{xpU{IIz950~zc0=r$-P~BVM`)6D z2>cc{8cdo4EN0 zMowLY(H-Fj`{@3B7;ru4X%i-7YfHRVGpy4+69l%ISa!MQeHy(Yy3S9-o5cPtKaD@; zrlBGr&Oh+b*opSf4yEX=%062jlnC-xU|K;D?-RnL)Y=f$QIl?5@!4bmTqo#NsOU4Y7#YwL$2A2t=d} z!QFFVmJMjp=C)yPYfEd!Q@s&(AnsKopA@?l^n6%9U;k`ekvGF=pn$a73lP^_9X*vU0w;z`*fRjz~yQb~=O8FVQ zWdj^xZ=9uA61dy=-Ed4oeNS%072o0QDf@NkV?1^|=i+Kx2><)Hz31zqbW(f$pUwiK zdk@}{wQ&@z+^F`lH|88o$Q@&&Qc~=2?P9EE+uMEhWSw}fXmfM)s9%e~ch+Qb>4|$| z3KG7iyB8pb0)Xys_B-Oz!)cY+k;m7&j3XF^wNsDRpJ;(!&OMs3%t{?#*OWc~XP=#i zmLxm$GuMAYxt6s#@O3H)aqyPNrK~2-fhW&Zx_rq?1}urta7m|UiMN8|6U;#5W`m_} zZ^%;h;K4VKO-cg?K=#_Pu3K<&#Y+mQGT37$;6PsYr75Hs%SZ)bT>^K%1sqmb~=!Pkk74lHuk_%bPNXJ$Up) zO}?Q7#Lg+to^lqrNTcGn@1oqp58)4K)%9M7p80qzk2H5pO?2+f%DZmxg->cudOOdsO&jVJe?V7NVvC7qyVLkI$Eu(*IW6|seLLCUCv zvw9V(p1U5Bx|kc`Xj*!9fiLc*N2rb%LQ*$|oA=i}lKiwCeYG9k+h%VII$l0;DSM|Q zn0aw@8a3j&-ajV^Dx=5ITlUR)KA)HFWF~BM^TFPf?F&ybZQGP%kIh+NU8H(0Q_c!9 zcy1e|hjZs(bDLzO$G__yf;c#abaAQ(MY4uIg@@=jzV2!z5VxaCjz;wp9Ms#K3k=>r zYz^(*8Q;BodC)y*ISa2}M-N7i@DwDJCw00$)+89Tf4n*`{`cw+@XVJLVykq}|L?xMjMa1XtU5A!FspQt zY+29{wU(f$xYmy4oIuqE>0< zGA1gy=tW_%G@nwDAURzA`iN423SrqWyqeQp>4rW8@;F3K$4!Td>y+c(g>Fk@G5Vu9 zA7!{Nl$m)#+8MS{bv(gbezyErM50%DPJQj2~Vx~ z=TYS9LcRKu0*1u}tWSb2*ouNs;fy+yf8$W|81HFLND)ZHld&TtB!Sl4$?Uo?AC#+4%h=pnuyG6%*7cv1$7Mo zwC{5NhU;&0n6^mllM$RJlr}6vW``W&OYy2tfo;GvWR!D?-Nz7d(~=lh_AHLE&lYdv zmYXu3j}{pRm$0X@D1N1?8kWaDI#u%GgPDSm**73c+CEVfKNp-hoxPZ z5#zaLg*>9alpnylKw+7JlyQMBR7`s*NwX$u+V(qo8h=ix6cE*sRalZ_5$?{MDvjvC zu5gIoBkm6gr18UIdPCO%!&g@jBsK83eYC8Byx-UoExh*xV~;{{k@JF~v9w|OTQ8? zK9%S;LDuAE`ae3uoK=vr=N4@BxAA(loM|AsZQuQB^X_5Sq_18ksyB@;l23Xyl)QZ8 zvdUE~SP1y~wa-xK!~fa>p4}(HjJQBdUjsH2N9{)WEc+l)-z^*FmQcb)KfC2z7LElI zTrQFe>m@?id4h=6?2oBzoD!pSL@w#|OasutaS@-0oJVu#7lcd>VbnV8Y$<`W<)*g{ z2Gvc}U}|&t+t7;e(6}tha&Y!>#6#!&feFEb6J$h6WB;pVHMsrhxiNpr?@gFut%I#8 z7lPH`QE4FpzUuq`I1xakr=nMlxT|KGduFcamO$quESq$U_5!Na!oJF*_8?I2H~M&k zVn9DrLDORrVK}q19Jbmt25r;eir20jmXs|b2cyt@OLu@NC+K#Ih7YQwVUI2Tk>a;x zyLI)jLhlsaL>jZl>udM5pn?Mwc>=9QKHP!%F?=TVyy!Po&=@nw0$gIv4CiC&Vt z&&jbHzMBEsG!`79ofgi5*vPZ)q+ag9y+XGJ&q_}em=L@oU7I=2p=i)VY=H{nvhLb?%b9t@&IO|9pP8SJBvwsDi5^HO{tR3mzT zo+Ik^;Ln-!0WsM0LzQBk_%MQ6vL|X}I}unu1?UZsaa~ya`{lbga*fuJ?a~EsA_NHA zq=bPFiSQ5FJgd~LYS7VgV%@I8F#N+LqzYVTeDW*T!kOhc48o-ZV^;_G{~S%7=&h}5 z?D&N5J#q2Ep)j5)$o$T{I@^qaCof9p9^|+EKS zY^#=Q-3U$FEiY~=96u-k(U*Wy^?`Us7YuEL960L8J1TF{gRS7LiAS}pQm_ZcmFnt- zjd;)i`@=A<#o^%JrO5JC$KVAi^rs#zFg&ryecgL>e&Pej2-6AJIy|tFqSS>GsYtL* zj5BBFBix;SBnNb41Mu`NN>p^26b|F)(FSH3m1sw@){KLAM3=&N9Jx%!_{?(xW8f?C z3Hpuc-=rZ-KjS-qO7~e8{y;qzTx4Fx#*h!(wlVu4T~WH+sJ*GYU9)csAWQk8l#Ole z?O)flkFF%Otjkm`vy}|T=K3b4Ul2!g#F=z)H#?3BLchVp2UL9+U?as(Ine~Cw?elm zQc6h;0n7lLsf?#XKfs|iUDv}y#6|x55&exO_}fh=%=$$~^}>ysECOz+|UWt8d}6FTqsz8*$jFum;AJkP`@>PxnEVS2%DcJS*<{@_uB zei+=)e@L&sj=r=$|HgfCQG4?^x@dWPZ+#i|)grBHTMG^RmOK!>e-}XKcN#s{tJC4V zNJ@W{ZlxWkeqVLF?=LhLD3ZU;u2^5i2SH}2Xa+G_o-@TE;* zV@Ysdr?WkYDZdOd=tyL^4CKc8ra{|>2%$A|N_PAqSrfm5{jcf3Y+||F`@ppHJcbjD z5LLSVcvGFy9Z)bAz7}i=0F6=t$gXWxE3g1u;+uU z*u8P7AjuP>(y=(*oz~Ilfeg4PO2TX=FFk2z^UG?*;@7ifd1y$lVB(hfa+VGbwk@J5}Rt=RBev0+J- z4zg9>HaFywwhnhVIGY0txpGJ1>OTuZBsCwH`e?SD9y{S>eyjV}+;R^*)MW1a2y)@~ep*(< zBMR&itJ~d%Bf+X`QvcTb z&$u8O8Vhsai4%GIFu*nM_=#-`0L1Fwn2~S0dVSlUs4X{eIxA0|70g`Pv$cnNx9j1V zw@KGJ_^P!7p?*Vp?mbs;So*)HubxN~3R_Mu_Cqf{8l8H438gAHdMV zt`6Zmqe7b05LznI6``o%lk14&If$?diltR0dGSdwV2ur5RW_U%E8v9B<6>HCJA9}u zv1@6XAB(Ds1RiE5S={LcRF9o^e0AAaer<5Ef*`LtapGf=0(x2hHk5yLm54{pQ%|cB z)IT@rLcg4uQ%f!;NGJypl`eh3f0MG0l9BP>lssqvBD78sjbCjdFA_9Ab$Pzrr#{eL~Dhv;#x7AJ$Im`oozl1$1l-#jC-cs z5i%}~GDp&uZMVF3?+57g!w4Jmc*Q6zx7;}P9x0tCrLbKMn>KpXpJ ztuc%Sll;-+|E@aCzlfbp35Y#(CJ6<4 zsGguCx4_17Vp>Xm3xN2U`})};hYk*laTsa9&rxxoqMs2ber5@TGA7bF0g!U#VZ*M& zfBuL@n&ri2J_QQgYd*ysPd=spM85W2?ek6?YPkx7vYf_nTT;euuT>n zX*$Mh+usid5zObYe0m#S8fM?T_j}YhK5J`1H#SULP8IT>7NcLsvzb=NE@3lJJhS^j zN!JjY+($iGpasqb@oLl#tL*mreM?l>PYDA(DVN(oQyNR!IDicu2*_2Q2Cm~m%rrN}0x~j*j+=M)fq~XOjio;Abu{3TF{dQ6ohGTZ=hJI@E5KR9v}Fb7mV= zIP3}=tk6yKG%3(LN=McN^Kr)K6H(iGM6&b=tsj&?*I*TtTF_N`AA`yb~$G#P~PWi|veTzi(cbJujYYT9g0_pD*734#- zeIkkD*-~&+M~_dKvc0B;J`OUthDyb6K~_4`5CToBhm0szWk8EgOnOE$kawW-{4eed zop{Ar{kPjS_$vfJW>wpy5qoF7mkfTr_C4!s2*dBmF)QekPQR#3BXNM!h||pSSM}Vo zD<-|H(+DduXe}24A58HcsClxRIZaKqkQK=mixZ1DW30~HKWqC;3r4v>jAf#&MF5ss zH)xrqCqzw?3g-9s`d>L;V|BuKpQCd_N@)Voo;QR0#UImdmSYLlrJECKiOD4*m77`L z&803tPTi+7C1O4s2A&gSEoY|GwU)P*#-zwZt4d^QKPws0kaA6-U&TY~5I^BX#1BA9KkHs>e_QWo)@bv)J`fDiK+;9wfI4JeKRzPF6#Nx5-Wx=>J6yDg4nyz$&pZaWM6U zQF?ZkOjEe2u_!AQ5#ZT&huu-faRshcHSpdq9GiLFG;vvQ^TE}7hIv_PoZRffbRNvd z#Yj`cjQIFPoD9K@K)%i0&aJpX{AkD|9T5zF){)B~5WY6_oPP3p&X8R(lku~5%*3iG zLCU0EJijYWH+q)w{#)~%o;)YN-ie(oYH~)p6^@ok8uG; z_YUU$nv#r}lDh6;k7EK>S2@i-$?J+!A4D<#Q*WbC-nuEdG>~4rIfl zi=nTGMYk=@fMPQ_i!)cZ| ziH!0b}IN4Sz>A~aUW1h^BOBJr6cG=V=q@yQ2_Od1*Bh&Q&D;A=?r1y|3&13hG zBuhDCD{i|NHkJf0T$My&Dzqf2P1WZrk~+LG`D%H%zuGEP-T&-HXOz<|p3P5V>v7hT z;3G;Z&55N@N$R+)?wu}9s=LGWa{ml}_%QD%P5&89QAu6X1-|+Cp%*Ng#=S;1k{bPC z%)E(=tp;i=!Ma21zwfY9X#Kh^0)>(Y$iYc*(>Mh;)7aDiF!sPR3Aw2=YE|CYnN&Zz z?vlP;v8CG=nbI*^%P^U#;VHUyV(7NBrT4_EujxpB!S96eNMx%v*cGzk!G z;VH@SqThEONdQM|+%lpcBE(b8$^in*5G!~ls>STmF$SnFsYX?=m?10Qk2a*DrU<>{ zJaTg{L7}EW6lSDzeLFYM&2W(9mp?j0Q)TZ4b1$Bv+lv3WrqdYL)TM4V=)C=JE}4{H zk+SX@{guL8L>~lyuWd*7L+4_xtksO*3$$I*%+48|!vCZxZ~hNWdG?3#s=4Vp_usMS z!77Ktl&nTuf7aqo{x4eG+M3cW3j{y?ow>R%0-z_m1Y-ED=KGEqYwBqz+m!l>|j2k z*KskB66(ktOOza$6vfj*0usc-!)bAi4>qSQ-18FIsT}&<{Z^UG5Aw?gxcB{mH6$maW!qIs2mcSPNgiD2cKKs0d1Q?U3 z$k^_f8Fv6MmLAr_*~RM1B}=Z^QnZve&5Y;4lTsDnY>|u|&MAampiio#wTvl2rW=-g zQqU|#Z4oKQ9v}^)68aDmOn3sz!Rg~zf=o_&Y?a{BsZ29_EQ7svkn&MZ;>b}47b%)N zjv8+_W-Ya&W)p`|9_ZIXy#?&8a2H_eId{F5(vF2S>GRf*l&I;(8PYsj>XUNiavVu~ z4ob|@kyO~$+{7H#Lv6{s^KOx_Q6Fgs882<(s)6J=);a`{%laedqSHcpl+Ni=#`=I2 z&!H;aSgFPxMPPLu(lKNkB>KZsU?E!lfWln`WZRX0(1Yw~wM%pW(^R6XS#F6|yYL>0 zRH!q4HbH?W(%R2{ddP`Qwrt)UPjBrc2G7jJ7_h?RNaSvV!AYw=En`d^n9mdQZ_rkrCDC0yr}un*11GD znYx^V*KG`^ZQA>W@AT4AE|}J+zWlo4#F=2*CQ(zUh|_&zxmX@!`BKYUu9bE%uHr$9 zr#{UCmi;NYh`h(+n#6eMhB*hIA1Uk0U20ce#}^Km$Y}7-x5#Ng_K)6 zO0wiQt=P*6q3p}lF_f)M`&N7N2RFyu^vm|f+V5Na#LYjvgVt5^zjg9~|90J-b*YQ% zmR)+U&4F8{*Wv(+cwyars0-R|?*-d;x@9uI(93OkJ*|7tIcl0jv}pFjo^0E+zM9d`_RR;tfGDLO$_O*NZoTsh~*5w(do%N!?cL{(7K@Q z{Ek!a-MNa=`g1KU`n{^H;ZC{fbo?F#G2OSW?~TI{7BIpT?lB6AHd4)eAdZ1;3$)Hb zAWZ#We1q7lsdnYMp3SX$hGj9B7n?_`+Y-Kh?d^NB+YiH0SWdt%{_6%{+7+~v1{0!1 zvvu%?-!+wiLAD@SlfoB;->1MkBI5PlPz zVs<`Naj|wYPfoKHu28}L26mLW-LK}NhG&mIJIU0>pjDHXhuE0QwsTA-B@Ne86c5@tjn!A+3Xur{2(ro5> z3Gmagk#1qTwwzd=!LmWbM=nvM)K18Rh58786Yge?`n*H@X)@3%`#+!0K@{{b&;bId z&=`0mNtjUtu>W0?42R{hCP5oN-buH*?YMoDEz{b18d!m}_Lt#j`XK%I;IXzAWlQfOo^n`M; zyFaGwH%ANuE_hFzQ$>Kc@FTaK$~Kgu4oRd!sBmJ2>w(TgKlWQV^BOqTym?9IK>?h8 z1EpFXeJh1;F?Kb!i`4IFlljUv8{++en;!4{k6(FDJb8NoecC_l4+KdA=~BrRr7ctb zsO8E4;OaSsKR+eOoUzptWOmuP4zjI)y$-Honq67x$^M!B-D!;|5QFtt=sER5Sq6+Z zAURxPhm7``sZ*aAWDogsp(IlS9}ks*t66F02y2r>9Gy^J3AV)iN;FRAkcbHDKNc zQe8DQWhDK+P%`V-&yUwA&m=D5d&ya-Rs_vqxUXumjYBrxvv_dE1v1T7^?o$4%5~I@ zx=ysHH>S+^(T2JNpxZJ9?8a<$bhjN$gz^|sAeh57jEpKUW|8vSN;;|eaF%>dW=idP zWnYCTfe)G^Lw(-uTq-S$*^wctgF~}Y4Rpxo*~K3lH9X1T_DXYgo4?IG*i5wF$n@PN z0z3cy)%tD`2Z=3Bf^tkjLHdO=I3T*#J}U|M`1b7VBUg61Z&+n>9Hrfo3EG3}VY^IB zWFh+SR>G+d=4n3$sUCxxDM$bt?(K3lg*%*maolKaady!sQH8xUf$7ht8hAxH zpV!t>zEE^yKI+DN^ht$fI_(?sQQ={D4I2V%E-J6wcUz)7&od6|zh=W~EsedRY3#pH_2qtmW`z;c4 zmjH&|-;WNBTUUgsgE0Tu7_(blNwj=_Nt9Y(9xEIGp28k!ropz`NM{JL`$s%|@}X?( zl23cicpG-|k!-HbF51jR)X2DhINcpHjZR`Rb~r=Z+qZq`4g6O$XvqgFunlijsIywXI3j~v{ zj5&txMptuidR-;)-+&GJxx0X(?b>uiKRHkp2R{53#7QdKRlm2Kf&NE{ONwXb@r6_w zlEPUuwBtZRwNXQFxfvO(BbbHQ%2|Ki{JSuymsf|IYIBj=9sUHLE1SOV#Bzb-on(v#4Sj zLMbg#lf(_bQd{c&M_YoScBS9Ob|(R?4Rw0$eyvH{OAx>zo41oP+G#JrPOq)C@3p_p z75P6Ds1x!u!OQ2QAFV{8SXuxhp|pyX4`^wn9i#((f{C+KWJMdYD{-vXLW_8=rXSrk z3rmYC!=YP<8=sTWC+rKt6(y)x!Dp;ZBMWjF0aztknJHK$lckSsTUVFby#mm{3eQFp zqMh1GkVvFnRA#Wk>Qg$=K}(&SLEskrH8)oscOBLm3zxq?mbnLXmVk&jVxpLo>_x9$ z<(n%!>FF8rXNmB)QWV0~ajhLES>`kJYH|CpEad|kxO&$3MSg8=HFdSwO7My_J#!HU z>%r(PMpU;d?xd`_A8O)3|YNy0p~8fExdSspKDo`Eaj)dK6a2AfwSQk45nSP4G| z!xTAnTW!bJ+SskW%XPxvG-?YqHZDAm@8H~Hq%#ba@#K;9%#Wj!S$-7HNGqNtRM^52 zq$Py+TC%(RTIPG-T(H6^FT5pmB`9IY=YOYDOIi^Hyd(?m_7U;dk~Sl9IdJ`hW|9Q) zC5zVYMZ3sLqu`1x5P7uSTQ^=~>>B3peq}T3Jb^xUUcEAWag$^3YT(&RD*7xTniBgm zLa=czqq%!38mtKyDp_a>6LMeTGo|)PiJ#!05a%*mp z$>?*RO2+syw`m0IN#L7Dq0>}@*2Ryu_8qTS_HKr0;JvZh)CLWI1akz2e*8NpANF3{{EV#zNB5y``*|9gUl3t%?RcPr4 zEF>DcNK^^Rt^}8u=1C!049oYE#jO2Vv`f`EoN9u`T9UHg9+E)E4!6O5(wMR?Xxr=4 z`$jXf`_hHmxq5$a9e;bVJGHW4t^U=LGNtfeyRK|A(3M$P7^OcenZkSyWg)6iw4Uc} zsWJKkFPw5V)2L6*N}lbRgvx=H26B4d#rX8PWJ&^S)ey1sH#_gvov~&AUnhu;F(Wat zxabz0o}5<0Dj&|0E z8GKRwXY_*}c(onvN#qP~5hX{qDo`V#ndI+1++43$hK-JD%534dBnFj;JO^#?#v%AL zk(MjR9Olf@p$O#LL>N>FfP=2!zpCvPT#rdS@8gWJu|sPb*Wz|c?|PGUBfVYwtLBGGD$9% zXvK}k8^^^E>xGmOV>uSg&5&eBmA&1?OI}O)XXI2_8}hTyx&|}h3xJtd#_h9w$`wU} z3~rQywX@Ploi@<;k6woj)5M~A1zoNFDHlg{en2&p@eu8-aIMS6rHGt&mJF-(EWsk2 z=}E?7FTS9Ppk);wQHjV-OHobLH|B&6Rj=FaHw$TwKIUnLuQ(o>Kj7HHJ??TGy*LFf zC3(8}kRlFQ?ee!=u=ufee~d)(S!TZP*Zk%J=z#OSm#AX_a7le`a*fOBYpNw9{@{0 zw7>g#nN-Zyae@ojcd`e=j=Drz(LLgn$0Z3JBcyWqu$Y5IB7DdhKp^evkM z4DWQcvLC|aXdoE0?@P7A_alaIqKnt?*q+ax!$Fjy%IkIED@sxI7Yt3ls>5pc)zilv zb^`p2nh00ucikmN>5@m!JZEXJ+wloS&dMp!)O1qhWqw@Y)Yg-1xG2|W=@Et(K!ZmQ zzJ7TB$$#8`{P(@l<46B-|H&vW=U+NZY}z2i0D?a_g&T1ez@$8lBh2`-efQ4YXHVDT zXWcCe7L#?OmLmW^5~Rn}9S8v)A)ZdTiefLKL;{!C^71sn3;Oaronn26)~eJ4Alm<+ zTFTCHyK0{VSrD4_kXy?)bK(O`XCP|s``%RKqF?QvJ?V({T@J?IIWb$t+7*)6127Py z((Mb}msEBBJ=;OPq3g5_HTajol4az9mbAIWp6NG{z@^*1Y1;Ac+oM|)O49TcE!%0K zeY^dsm%2tmc}$$)5htfAvGimXCfS%_rmCBch=F9d*DfuU%Z$+qKpp=!5Jd5difvzq z@Yh!OmU=?BuJO(BF!YfLaDNEs? zV96%Q7$sL6ke%$J=Gt%+(!v~}7ji*Z2NzwBX9qmnaZWHKKT?V}jDW}lB0wg1qvOVdGT z+^+T|Lob^@lae}E%pO^Ia2k7t2MCdlSI3j6v(=8_h|b?-uQBv%yh4m!v6%C=0@li9 zWX4gBl!C>$k98$aFQic!D9i2X7O{@0ilav8B-tP_ZGcGi$_z^& zWC@&WVA`zp;Xfp$?{}Dz0dYyv()Nxe1O@(un}dZt?2O$>3B%>n)*;FDYX(c2UNDOC zfGo{oHWfg7iTkITB((LlseR{sbxAgp8XfhOcy#GGLHp=JirV{K?7MEc@{G)#g2pGN zA`ii!N0y0H%q)oL7l`PZHzfHH&Y16TPwSIWl1H*22J{29-D&A_sncyKi zk~v_32X==FOF`3ZEMUlY4xKgU{KJdz6sr_K2gicKX4P$z_RimTBYA*2RcJ`56c`|LYWQqz*GzEC#Mjw9NCf)~ulj{7vS-EG--b!QU%d_VZu zd5z*4DO1`&{@f(>p-Zufqbg(!y)rg=g&&8o;-UMH6IsVA>UdF*n-p&;-_3wI?aIlk z00=%L)~z926=Oo^8%oc-1y za}^!~Po(AM=g*O)5=Awd$_>RQ?!hO+qx0_i5H*h+DHJD6V#3D$2b$g2T&eLK%OHK0 zE1v0yBq8a@#aKjI$gPg7K)!80jGk18a3Bx(L#n|NvVZW%>NP3WlU3Quu464HZss(b zY*umB678^Sdvf(n8EHi@kQm2f%x5gU1|QKTivcnm)07J`ID&29Xuv;KrHeCLwbz^- zf~Yo?YmZEAYxi1#*yhIJaJAnvm0MnvmP@)r=O)+sJkhMl#OZmCt?iQev%)$yPA?Ak-XWkxg$@roFbu=ZCfH;u}w*>iLc zqkac8?m(LKBNNSKG22-uxXzrhP~h5G525=cW$}VT*T4vQ&ci<44lbkcWZ5^)@6Dd8 z)}D4{J&lg&MQcBGuO~~5m+$)vW@C?G=TG27W`6%EfYKqrNZRinqjkuoEQCh%*+A9D z=TfDD9rV0-;twM*)STuGdxTdupX_nw*6b?lL_3X*Kh6eTJXGf6V|AL`1CD>^ndAHL)}bsgwHA0;<-Q(8Jo#Xx_oTq{~X`pN_pQf z`Y!Ayj4BGb9Wz+ly{UPYan!J%?{egfbrpahSa?hcD$K5;bvj*gJreifZ^spYw^9F1bxRNn`%d@c^dYRhUSJ@0hT~ zttgGcHN2Z^ux-Gz4UG^p6xKZ&YS3g1mQLh@mrlc}a54swQ%RqK@34viGKMGObrLb^&mo_^WDJx zCD$VdlDp%%vLAJeq|9fZ$sU3!y5QfsE~VEc;gZi~w?A;-OuLbUnovK;%VRf~xZsxE zeB}BFrW+uJ>G1`2GcWQJ7^rORN_&Y8@k9QcmW8IEc|x|)uH-?<&oJ-Awx>N+8#1|l zsmSuX;xJ`}<6YW5naW0u5OM)eIzk|{ajA7|)IpFdwEMk&4{o=Y&@F5l2%G5)|9vkY zah{#{AELu?CbWS^3&#@ZDOMTFO_rR~(`5n+W0~dF(Bo}UD>P0C-&q9O=C>?LJJy6M zPC5gFsf3hCnq9KN=r7BWDS-F6fcGUO#fUIhgaG1@6eLpLnl4fGD9zpKv_Ib&Pjd4x zI&7p(cPnDxNtr{QpK1%15E@5P-KX$X%d++LNKvyUgVo>neAV)m6+ty!+enUL6trRF zGi}J25j0TX5_spgRUG;n^>(THBV-5MbY($AmEEi%;NBS5#S(|n58@$V!Nc$kE172j!^uj?T<_i(# zrk5JuO#W6f-Fs)peQEwr0??)Q!#@v|>)mn-{wK>cz|9I~3;dCA{)9?P0|Z z_-WlO)YC;JCb|mvLx{qmMKlHP27|W&o0fL1WWpitj>Gfut+}|ZB{T9P>CP*MNcQ-? zV)z?$h89OIo`s_f_J{~6@vU43#K$tTO3WDt_;-}8_WT;9Sd<_o?Pd^kXtKsYpNQ+L z9iQD&8g5BdS@H*yZ%>oi951-@8uw5xj&i&Y5%qwR7+?ynfqnYcvk-v;{)(W$-`NjJA7lk)XXpcUHjjcY>xt&r6Az<=@?#MDxcV)g5;Lcg$45s~CJ55?s%A3{*VTTx znij$Ry5nmqYy`791AZPjAO-0NsecFrOxi9sagGPWJWVG+KVG`cq)gP9*$j>2qZ&~* zOQMzR1j`sV_%oje?}zymn>Y*?^0mF6cW1N?`CwZ4ic-d+TZz z*O)DSj@-GR3aN|W8CWqK>kEe;L88%X@d#j`Ij2{M8uxT|!c5Tm{vJ-)aNkU$SO{z` zFqB>;;xHNx{kHVmIu#J#)GO0r6)fcrGV>j{`M!8bHR~gTw2CG*BiF+(aT#o zZdy2Le^%&#)Rr}<(Sp9?qJ_ye_!2PLhQY3l(8PxZhAZP5u-^t_>-IH4pupO29;cMj z5e-w>1!)-FVhrI^q*=Ac;Wtac!jo_leB5PLyS)OY{EEDOWpl;#6Ju`d1;!8AWrHM4 z%V_#-+hZeQ<}yu}yn9!!4Dexx<$ z(a9_>OB?`I6cu}rHhfW|=;=-4PuJyNbs1id?>>vsY51wHowI#+BBgQe zk{4@7(h8;JDF<4hmK8&>@!IU67sgSMpO=_4Dak&kMJ_J-E6A5@s~4tb{{HsjLwths z?se~V`%Z%xU)wV7_O*VP_ohr!2Mm`=1IWq7z zUb8FQ05=9*YFzviisB$8k+1G@M{_CD$p=bH*Dqf#d)exe0XV$Q(WERtu-9a_%ID+F z`IoKZSsqto44K->=kX-1F2NI>F+=~*h){_5jI^Svo#nupEz$q^ zb^0X{x2I{22NGtx9(31Bldspm0a9av?-Bz7ZSRQXw4NwG`L1|`l` zFl+KWlN3>uRg%0vIb4ZOSm~$9{clIB%lDg_boeq`3ENsm4!Na_qCqO|Ydcy8q+gfNp}6MRO>OvqwL$NP@4s_f>jL=I zc@CzK;WTl{WUx7MzMp0mNHE`OGYVq4CZ1wF?zT@v!7hh2qD+`&2ntC#?ExVU^D*q> zN%93bGODo~7Z`j4kAF;#{&$LcpmLFnFf&7R=h5cFM}F=)8o= zKmd@n8SX@3c%BJa8|+l6$!lt%;9?Gz6`n54ldJ(NtvoZj@uIh)E;^hbe{zl?$#zKm_+3UL5=GU5GHv@3x0N_n$KX@S6o)XU2uo?`Y9x;s zCbA3iEiumQvt=N|CEoygRi2YqMq1q^vo}j5G#^6be zawx#)2=gn^_din5IHk96Bwa1916IeB6_QrEj6$OH=*0|GZ_6*}EC#k|g!7WH3Qn>) z^25>P^K=Xiff1sA^pO52iK1jKPFJ`(#W_lKS^!P_ z5lAdGE?b(+;ZmJxXj;qrX@dHQSCYuVIUnllEGIa;`pg;QYHy2Vmg4pE287x)`<|S5 z8javQ?WMnR=Fn8aB)LiLU`(`}dKFiTGICYH;hG^ZPlcD!0_96*`@+4K_Ki)F9+uqD zr-=!(C^59vm5k@}U}$?mRo+Y2A~>7)y5t+%H0dk*XDZRxR|4a|^e@q!%{5k5J&18y zss2Cm-nG4LBU=;wEdC0Z&K{E%NlCt2ilQg6osOpyJJU{gx{s`6AQ6&~Pz0MGEjy9_ z`>cDdDu9%n?!EV%bN2K&B2cJ%tyTOC{xx}mf zQgOg2>I&^G7Vn^YKM!n8YX_yAlh#^`O(Th}v!P;Tb15~<^=m0X0<9L0#S)(Peq%-O z1e{dBF46^~mo0O*AU@^lO~YNMk9p?gCX;F4wu1fX`RnhHLx?2?DUr$U9}bjzv9+;wv4>s4X1(L!(p#>4=2lE;?$B1pYefp(a!! zE%3~1cg@+Q&IH;OO=8xYpidE}!QGPJt>fh} z5YzsE;ss7&7w$aXl!B+Q)9X^gte74Cs<=e>N0{ViZ@x#>4@`y_+-UGHvfjS*-Z z@2k;8^&4#Id{oWPwuGL?JK5FYi?fdn`H>*erT@VFsJbuHq{SOEM!hM zQQKxGT4xrOfty+PD;jsrW}@&ZbVdRgG5r{?n*<~ux?-5wQ?85s_`;xm5tofN*E13o zo=r*wsOpjydAb_-<{Ga0xa&yk3CQ4c+8=L3MqAzNexu z#3cj!SUSeLEHKLKlmaUe?e%gxidK{LdR~XeOQwMiAKmqSIi(n*eE~}BZfpPkvYfJx zhkTH8+z5{l5%&Bo;#{PmSpvhq!hRDu|k zu9g+I*`IV`cV`Z(Iw>$PhlL=To$B-yX6{Gq`PMhoZ(~LXX9U%IToYWrx<#gVA?1t% zFF?6mCE04=V&Q00-z18lQV%_ySC@O(ts2DCmOkX7+*g5v)*0o!OAaa^Yd)eFvL>I| zKNS%zI`q68k5Rd(=)X%&pi5u-q4Y?(A!DfXQ-TV>*0^UiGcyb_i4dXJ-=)%r5ZmIm>Q8c(`+qhH=4>+NQa}o4N(&K{V-$IsE@IYSzc}$AbUb!snuG zM~b-t$9=8NcQNLT4CP4g+uX-DAr3fQ)SNRqpg|+EYI(|J8IdS&%l1qfmKv4f^Xr|q z*V5*-JKU*9Z?eywV5Gb38(PZOvgrHXvchi5*N@UV6sRC=1D;Zj6ECZI>ypzvgbOYr zF-B5(Nm_}qf7PVL?yasgD)?oI@z3}MU~2Xtp-}&^f~dg@TcaPV`LCio)ZAXC6QH{; z8!zdIxkVB8IMUJ2EBKcL0j`f2&>CTa&=|C%rfFc#=uY^psxIKV1C9``fZIFwA3nGr zoVV)BeAc6K1Nv9y7Qb;GX}?n?yRiR0X1f$!yoy~71L4Ewhk*OwLF{~kYE|a5@K_?s z7EIWfnHi8qNrm-GJ%-Kz^+0RF=$Bex#gU1ip`$9OBxeh{2v=2?bTriit^AO#kO9vG z*~@dtsuWnI`<-3(j0w7H{a=DTL+8gDq-0>FxLrVRcB zkJYr2lQrasvT|wyoK$m$NBKlL6&J?T+Zenr^`T$ z>*6@Ox4pBIO|e44j@j67lIB-H_;>E#e{dhf{>Fl7e81c8d_TB8Vs*yT$(Fm3MNpup z_M!?Jgr$wxD8JbH=|!aYOW(>;GIzV-X_>t-7H7Rqk?j7rC}%xs@O$g_m}7dAUF^FX zEcO;wDl&9L64Vn6q!bmU9oHFycO@HjN+6ABBtrr!iKv3%PXRqx+dDi~??$M?3K8!G zlfRGv%5tup3ew9v{OKLq`e)@k!LMWouZwI#YGa-F=GD@Ax0rYUvLc^NybAmcq1OU8 zzDJaDCngFj71+I2B6;6sDRMLVD4ozSEqMHScp2-!hiGS%89KqEFy~;?Fhs{e&{T&h zi1{pd0-O~78%lUgt$6=Fr2XrL-lCFk4*HfKG&|w0pE;LsN#WsMzQ`W2kuyI%#Q<^G zAu=VDYDq}F&xUY&XJ`Atla`7&jUn}1;dAIJC1Y*&7%>|Q`1;z>`iGCXrjs0H_dh*TY^r%drB4cHAIcHo46zVh?@!oDRI3P_pyZ0?ecFd~Eqgce3hIy3NkXjD%<4 z*cDqIYE<{la^$Pf7_Ae;q!-vM#ubCf9s7PWR-E4-N7DWDDdq?X59Ew~Nk@{Me-QkM zL}T(jSimOo7zeDBn12wIHS|xR6(O%bs!OJx)IX>JKsY8WfyVdAL{3BRR>=vlX*h@F zn81g6x9})oJ_vg}!fzQfk7|i@>k9r*owC7Zzli}6ohkwDbKvJ~n=mKGQUS1#MfO<% zBx|m41<)z33V6XfTh26gUAD0;7scB1a>=DaE7-O~!uqh8aQt#yk7m`3Q$5N4JX@CI z%}rvYoXaL}3BR}oE$k8|Kx<>esM2?X)65M4cvrq);tNW!t)%Yi^grVH+h@X*&)GUCh1)x;BPHw^B(P+Ta3YzO=nb|5L|syRaj zqbm(#_35&R(LU^qJGJdcIVp6nU}I1LX*>c)082%nJ7rDwo#SG0iOR#M0=mNBo|+FW ze=#Z%;mt@ld6-}RZQ~nx{Hf%rper)J#Z(MXT5ngO zG20e!al(jk#Tb$D>3fm0r$c+iT`-VhteUs=OS1V!FA)jN07qBtWluUg4w+0+L=co{ zUJuPdWbr}i-<<3kZsZJGs>~%hs1lYNFmG33!qRFqn${LudVUJs0X-RPQ*qyzWQ)VpWaO3{pi3o%A)QDXDy;|*xR5LAmW7%~ z2#fUBk|ZFXh+CeVY^ge82$4#P){8hbm!+y>!7WoUEyum5s182uSva9l;0Kq%n-OvJ z!#RV0@t^_SLE?;#vovJ%>!ajg+%JRFkeBN8>e^j(8Z8uSeec`5pT><#Ap%O4gaBE$ zEeW6hjQ1@&=12k>S>IDyxzT8(<$ferc@{*VqXgHF&T^#s8xP2%qt}Ix(8C9~k8VH1 zj0QPu(ZXWGu};fe8E85tqSDE+2Hz;hsYfKcWKGnffbV}kuP%SqCCDX6d?479S_R!I z%*kOnhON~I0eCV(J;UQCkF-4blT-I7260wQ*;PDbNi-}blvcQ_e&QuBvaa|A>i?N8 zrhC8wkT^oP)EchI=^`F?$)$rCvyNPC2{}?}gp$t)tA4^X7Nnd@Ob?AKz!ayD3UIO} znzx8Hq%u4U0um%C2z^tk4;<8QjNSp_gug(87!rI&ckNk^F%U7(0V>uk5(i8=&5rrp z+=%d}%#qAqE9Jw8UozawpcssR&8|FtIrfY5BH_=xL~ssFIN5)RA;ew8xKld!Wk|5r zoTMzK@e$7YUNrvau9KcskEAW#bBOoOckdzJzQA0wBaGZ#oKb-0hLyl(c_cZI?>PdP z{@>Kt|K?qVp5@}mgHab{{UiVA-QL;WezJ|;On0+iY8d|_EB=Z9Z&$p=?{0OXSkX92 zEpZk_gkD;JTf(f56#ksWxkei_Hf*A0!POg&@Qz7IhaeSbVPOS+#2Ur2iYoaW(7fRZ zaZxhgiSv4d3LaE}=kp5XZ1R#N3s&(SStCt7_MuvKM9G6DWkpz2?!!3;vA}i#G-z7k z;_gz9yNI4rEV|;>g-?o$<6`U{podw`gIC#>`#_8R^#rwoJx&oW$~4^oXpmCbZx^HV zK=Rpre(&+K=U+W~G5qqyqvyk&os-ky(??%D89sRM;Nipj4Ghv+7~XnaX{dE> zJac>9?X|XZ+M!mISTKE@k|gpS0m1`2^Y+PM9ONX)kBF>63TxaBZp+lTSrGPBX+KmkW*@ zsV*_r+l1#m!78Dw4y=d5e7urlz@0Ja;?Vh7C2&N zHIDOIA}1H;mk0~%WZb$I3R_oMiRc@&g(%k}H1sZ?F3bxjtCn+Mq}~;^wE3rkHRb_g zf;JQ~uKO(-mmsR87}T*=Z0v+?!Nh6VsQbDvl{Nc$>`rKxjNxNkx4IkQCE^p^kim;r zqz;csKqW_Q$OOhgQxRZgNBCR1>ZGF)p|-w0?RBwhL)kJu|Mhx#O8^=l_H7z!00r9dMNAcdJw@L|^^@B8oMkZp zRd;r*53+Q5spZDOkh#cQPN6mTHJ5ca+v>qfOLKSSC3F8VATI^0EU}DNOI-s0*Y9qX*ix*$B ztkQUoTLfQ|vUb!xEnt~9Jcv)DcVyXi#G4wSK-L~k_zq&9$pbiRR5qM8$1`t8fVHGY0A8?16c>uD64r)BClEYDl!np*DecHW@H*0bvELcZX(~A^Edd zvs7wwlWfdx4fZ;I@xo~6XIzDX`OX)Hsm`6ASn3~!l;x+A$%G!C!<^S{s;I;__^_pk zDSP^@WDAOxzA?2@UYs?V+39;36=8|epW{Ksu)&&gNbo}$sjV*XL=_X=!8jU8^QfT5 zh8@@#=w32CoCF|va0+(5>|ai$(r6iP0nJVm4T0|9<080F#)vFByujs$G$ijsq7Uxy za>`v!1{Q0BNm|7N#@Wg0g>(ed?6~~OrZA%=rX)OjJU@abCrmOSpz}6JJX>%1Ed}&X z3UQCQRw9{%7uoCkJx?O(g18t8xFyFzCddd5xV&Jz3Up?kVh|l6zJ$GJMzv~g-pxMQ z46u})4Yya#BB3JC2aIz*Xfkns0H(ff#9Ss2Rs>)_uv`;?{-{Nt$E(ir9|D; zZLp@i7XlJ?v`zSKP^H$Z1AU0eUGiRoTU(j7M)(5;KrIe$N40ixCt{qcD*Lot0CEukm2p~hIWkI4m` zLiz}CTW_r(fK7TF*B$OI(a^QsQ11NPR5DuGoV7#RY6_J&J=L^zv&oX;nLcFCCaCxS zrkI=-sP7?L>C{d8;l@zyW@#7+%? z2Ie2OaER!6?gvUCM54sO0~;;Wlt1(a{co>&e_g z$H?IZ_e>@!`n+cUwiA%B(58!Zv9$!w4haURkcDL^nei3~NEIpAGiG+9F+%Sk`KCg(4s5IfhGQ$N2TD%j0r_sER#=7?3b4Y>^gQ zCYw}5JY2jgx)c++CTTmfunCZLY%B8Cx{$^Oqg+zI0W)Zcr7O&bB%cI}p}sZ09E-$~bC zULvCxHoLj+%d{J;l7WWgr}OecIxnM%F|XsK5RMxCS01_sb!6LL2tfF6JdSQQ{@96Z zKal^`r%3%Di}!rONwdy0;5q`J1gZbubsYawPUDuPu)GV)d6!~ratsBO_gc}l2lk8o z=84^>&umzY&t!x*Jj%R34qA>h$eR*?ef_as5OhrYu^uo+-QV(wmy?1tphvJFaoA^c zHj$Gq@@dyn7`IvOe3Cf)@_-Lea_wcB<*MZXrfSK5>EM^E4!)bO=-~6YFVk}$rm*j4 zGrgRzWzZ5GPn!Rk;F7RonC7;8SB^0qI7=-OEwkas^DA|c@zmIIg~tPyclKSaJMShG zDLQwyaYJ2)M56owR68ICB8IOd2W}dFpb2F)$XzmU%ko6j*N!3;infm{CucA5v!_qO zR%|IRhfSF(UW?1~YErmm=!BZ%^=u+Q$R(E?ph#)ZxM0E*l6!>fCgNTdGEuQ~&fB*8 z^UqfJ=byXH%WKb#&)H`@09Nh>iu!oVnIXZ8Vq&+j08EPJZ8&bSsES^x-t6e@K=~TU zCI1*dz_jw?tX%Xha^ST&upLXzae_I8U4pTWSOr>T%4%7o`v`sB zfksG@Dng=D%GeKsT~S6R=R`C*c>t>#coz0FIKb&9!o>I<)chhq!Xy- zDQ~7gj&%80A-;bS{w(Isu-%Boh!_mplH|xZfJ!876ox@H|QaxIXGy-~hsMFc-k_ zjqbKOFajmcyBAfxK<@Gaq3rl{kT%Up&YGRkO z7e>lKa;M{(^-Lh~SDmZEHAv2Qm>MfgDeV%Ven5`Q|B0uVq7;Ic6LprVb&;;2DJ5sg zt;n3Ex?W+m-RQ&3)-TgY!*|5E#tphCEqD;@D_J8ko2;Y~>}b32!_B~Ed)xcUqfV>5 z=BV2)Sbfy_?nRF+wRgLO19)}Wy~PHT{H zjPu)me6Mf3Z?pk0yqfgR4O+O@iB~_s_VK>JN4riYL!EvYFy0q@tKMKJ(kqm-c=*C? zYjOY~vGP+{cDxn!m1_Q9;zRvf-(W)9@b3Tv50p)SPUfK7$L`evSzSkHG zCd{G7J*$bMAu+%(REosIqEDDw*?ffZaHO>N96w+Y-IX$kXy7|^XdZGBn_3rC+wtA4 z#TC%|yvKrK)RrXQj_B*Ps5vH@hRlPPI|+)QCo1wI6mKrQ14@uSDiR_ zV$uT>SgEPBFU!jXWpQ0TRpk7=AaBFZ_JahFOfd>{)%*mdLQ3gJ+zEx_6qQg5kUaVg;Hkx2f z+2xJBB!Sk`w7ZTe4!1=kcVf+ABSv6_{-o6v3|nlST;D}Z6rFTvf_xat7+m5{^ZXd- z3z~TI`3G7Mk)Tu$I&~wS=N95?%U_pNdk=$0ShF75*rQ961yp^quG2r z(Jg^`eOACdQ1Y$xyFdqkuff2`tREx?p)RYcAZ!Zdsh`n#Iug8I`Df1r0*Ltn^ zS_kn}SSPM=l?dx4*oM3#CpQf_88`;N$q;(ksC@)C7QlVi{q!ZB_I>7>R&lX@b`g^j zgIYP$U3C7Xeu|AG)H}@sNI>+{u!mhwf(%6S5oUT<=V#B6v~~}T=1r!-8kDSeW}4}s zg~DD(if6O&tPabdg!&P&WGwij#cqb7Wbv?s4UAV%`F+)@Lx%=3A{J`v_K0`>WoX63MRyQ~T z-{u9qJ1jtDqk-f4)SiAhsVnhN7f(IOq_ibJElnV4X=m8NyaX;?nD9hF9{~Zl7|^cN zMv-X&l1!PsYkD7e2ueZu^wHC&3XUG5Q_Q}UPjcy*B& zMC_KtZRt){Kx#IFQ1VC4naQ4WE@2rrMuJ-H9uVU0qhZ)`j5^IPF}H}g(jo2^V)IZt z%I~x&o@{OJY;8Z?l4!`GTqr}>UPJY|976v{(KKX{a)@CN z!(Z}ud3`c3XNxUu>^fCxq9fI6*C|T1OxK*pbW-$V(fLt)@=MIir*8(?&USF&3C_~O zUc?wDq>v6y(JRU*XNTFUP1(3@s>b0?bo;R?8s_C23%WUOaA+Nqi}rC>tiBT?%$r0S zV)JgG65>|WY|U~oL|odn@>48({2RDAG%dXN{0V8!_#(Zl{Nr77Uue1AZl#TdyM6_^ zi^LRzBJ*4zk^-2#nyt9bIdTFy^QAlZ6* zxw&Gukrv@;Eu^HmaKMm9}9XX#1AdAsJT{eNjk;4x}v1n64ycFfc1wL=&f z(cYKu<(Q@-tCN}9!j{;ITU;AraKD><>~22;2U%0~>`iC32@fWnr_pp?YSUo{vk zDJ;14FIwDqF%2n_WLzY`2*eW@2^O-p9(T->81@fpO(*2qfxcHCq^D(FV_2&V{L}V> zs5c_Gdgu)!(Ml$RSY%KFYGBwf zi{;$}(7olI-*f8KAmjat60(7iR-KfrYoOKRO!Xo9@A!l6X|=z$27|OYX*$Y0viM3i>!I$1AUhQ7j&%Z%?Xfj(K;OO?Z=RW>n3i4F@i8 z;tx1y8@lQZ#hFMta4f;zWO!Q>Xwo-8%*oYura^^PfhER#Iy|R}Mp}NyIQkXph&F0;orPKqCNkCOb|ezNlrB&k4mg*@x@g~Nz!>!GAt!w3Uzs*AC> zE4b6fj5!YHlaz$fw4~P3rp={`V-N8U>(pI)oOcTdX{UuS%!_|6OM+HVHTwx6-%<{) zAb?-g4NY9is^7!Z%xYMc|QLD1s>ji1e@?b?mhY2o6(bp|8ei>D6ePl zJFO~E6hL389YvHe*cd$Cs^=#`8DF1Q%gLC`0E7^;B!p4>@UMUQmW%=So<7|I@%-t| z!+v8mu~J-2sP`r-+0Z5FO0Ls{$o#Lk6b9J?&zOV-X`P!oWDIQ_$67M2ub^a+2u(c- zl-F?0DRpvmOyrYS)uKStLk1mT8*ZJQ)OC_fWPn;__E?Ai&50t#9J5rXc?<{4Twc;b zDtI_`qb}g=mW%Qo21-HyU^VYR{gQ|=>xL}y-A*#eai&DLnO6|jwI@$1?6p;Ce1_Q; zJVW^=Kh5W5c2U;Dd4a3ON&wZ9o8Lx=_N64);;4f*9g2EZy*)bu-Kn$KF~YuPMzLJ9 zZy2c6Vn{qXVjP9Tj^d+11@kBwJpA0kp{oSiZYx}p)q=xlUVp^0XIE~Y9$>_Yl+ zk-ul~Ci;b$tl!M?6ZF{D-`{g~PX4sIgx6DkS5541kg2}QXY}Xk91hi2l=Vo8(S`d$ zF5GVCD9v}A>WkP@HgtN=vmgY46~!(n`8|GX8>w zC(bHJ!AZx}dp6@Piuo2}4|Ng(&1Ns4(E344rhN}$|6ilc*B|v&KF>bHEujYj7_r*- z3QNVFR~B-SNTKa^Kx*LiG`@b)5#n{?u~1SvZzkg(DXF+T?NYW99uLue?IcF3I+`)4SX38iT!`9tSb{A2g(6XUz)soM8nx9by zhIb?(4~HcD%@DV{iaG3KK*$FEA9||g7Tg{o5k!YGfv?L^euYx!d~7|(=G$h z_mJJz{;GT7u32=`A%k+?K>n z0_&mpyb|Cc!@=B{@mS=Ut*6Vbskoa{NrolrC1BrpFSC+`@am?R)xMmD?dlL(NuCir zZb7z^Ey0`@E-9am+ODWo1+tK+52bk+X22TKF z{%B*c{n=SD2TBHY>i6u+I+c&gnphJHC&*!&o)xsF%ju-b$D)FcW9GP(6=~toQhv+h z+D4s$0{B@$K_9BJ+lWeHigGeqVa6D^^xZ^_m=sw+W3bZ3Xg#zdihoL!fHtWY3c?mf z>yX1k%?09;a$+-=43Zf-U2`UTAs@*SQqN=^{a%#?HuQYiH6b`0HKQCc-eDgn#?|1z zZi1`P?UGZ$Lf{U3Agp=}E>iHb_|hE}?~4;@@^En!qT@FVR!wOxHgYjL+~JpjDs!LV z&AlV*H0OHDx+D4ltWqzyeEvXDkasVD{N2Tz9l6r^N{F;2?8EAZl&(tegrEm^fPOJf zb(C$4mz80{Yf1^tZ7u5}#CBjxjdvU+&aGIChvJJfrk`D3P9IFWRvsT_A$0Edk8wDh zP+Nj;Rl{C3EQ{M{Z%51 zq!Y)CYB@!71xz~!7yV14x7!8jO-K_=GEp??(T(Oc?i!7^p?-i1WdJsGYwpI3fuv(0 zE`@$e8@qNT&{Mo!>=E5H{9Gq?*QS`Cli$HJ=6%CE=S|>@?!K&;t^>Oy)#(=Bbu7;4 zm#Uoh;M|Zx=VQb{Ik)L>4f;7$ENTTd@RyxlP`1bSg&yg;1&M5B5qKb^!OGZ{dnTxq zE>@0m58z2t2K8I=^9GTW&p9)#f^r`ngW5)sdujLNYLA!T zPS`RCxiZLfUlQMaIqV!w)L6On6rx>veX0i>SkVVwkR*k-^|xHqQ$&f-Fc0Chfv+ToJ`8o56l=j z&d2dRH5<)4hAQE+&rp^}lBA4}tO#41)Z07&0+P?C;dHPdaJ^V*!j<xZ3#FPh+j^*Rz)T%^}nfA=LCD!ULlXTXx?mN=EiLgro<(2BB0$ z_gVCR0${rDSOWMq9zmlHL#vPk#tU5a;UDcVmjXLl(A$=wP3 zh*L}@H7G06eu;9j1l##BvmqKI2Za{BSj}{_GcsW#QFe~3W80J zX^1n@bo>gj;~Xc*rQBq9C`I_B0s(xk{*bXjDau}DR9ZT(jmJGS0uz#suw7Jgdqv5- z*q(+VfOXnE&9Eg|H6<0Rv-oiHjz4QjbJFELJ4-j=U9z$);+C+S3NrjHYDK=$t?6R# zMRv|@sO4qh8PH7ie87^ z#r@+bGK$+4y&qpU{JL@1udc_6`N{&P!8`%V$bT!-*5TP5l_nn@4SG^xtOEZ`OBnAyQl;%jDZrEr0?^aCJVhDn=5qysv}!fmb6t zE*BSh{p)V_Kj4l=A}0E3y+HQh#XWi<`umRsA=v4gc+@nk@NvpEP6v-qR#>DC&(X^C9lGB~R4wm~xdHR}DSVV>e z5$*U~<@2-PhaEj@ngO1nKeXhB1^R*wlFbk44T#?$LybfK1`@|BK1#}|K-~%@8|Mvm zQHO^z!XuuCCVCFvAEB7-qA$tNnP}iJ0(qWaVyeR$geOkYhWBFU(}fcOrtv!p_OkIR zvgewQ%nW3Tyf!|iNo}8-cAdwH#-IF*ebi@o%i&=kzeC*Y0j6fke!Tz8%rnphbWDHB zcv`3mq|1}GQDwJ@#LdGKPAIXLT@v-95Cl~kMvnIxUeHkS<9iSKE9bogOz+pF7cHu(BT%?MJAyCb^)Ed^u-Bdl z@&|4YCEnkMZT@Cap&;7}d&IG1`&lWuvp3?+!v@ucdLT6#$sH2B6gJFq-Ai<1lwKh) zx=a$X{-Ng(kD4kq{)}sl(l42`;-z{e6ZDnKY-)SxRdgFh9TwgJdH`P^>d&J_i=m5{ zJ7>0duKi?U?8T{~qQ&!E%uB?2KCZ?eRFhkbBd6G$F@llZ zEh7f9NqKY^Ne{l)`rIC3C5h3fpS2Yyc&xFsI#L`XYfmvVq*I5*ec4H3HI zQ14pwM4Ep6`n3=p@(j5w*jE+J0ft++g$-wgo`UZcH+iFan%?P<3X)IFl$0d>TX`i&4+{s=5*Ef$oHzE!{ zf0~U262ysga1))E<8fg&4~&m^@Z{0p_RhVBJ5Tzd2J&VArf*)2mnS9O@4PSm{ykLw zM7A~GV=rGkWlPKx5E;isq%~VPOvkd`d}-iyn+h$CIQP7b;K2P>-uv1V^EfdL&7(U}FB+;}js8l{;A(iAr z?Lg>(x0UDz&Sui=$IFY0VvOKtYdKJ-uqf1X5xu~&qNJ_6(`VcFwDR`5YOdej)A#za zSNG9@Zh1`h=XqBA6SK;YV?b3G8hmw(u}soj@3_JU<$T&~NiubKF+WmWpvJH9^csVW z-oSoyc@q@;z2{4aY#*w_2TI8~J{FdzoocjLr-XX+RXuNpDd%#L>E*Pc9Gl-FUnC>^9Mp>di}bXwA^jAs_)zHw$wGN-5|#~g@bGWx3y&tNLMFf^Ca4HNb0q5%8wy=viW?` zOf0Z-{|O}+*m?Zufs-SmX$!e;@Uc~jvqfFLW)kV_$wqXZfiiV`uMzrvw0H>){tCOi z1H*SSR!vW#eG%?hE05uEi;6nv0Q-0%?<;A8)04GY#w9VOr(H6jLp*ZG9Iv2r8)-Rz z2RPqP1N;hk+UP7xo)fZFs2YKcF1ty20#$M!4FnVSkUlzsI-KJ}poy)6X?A~wn_5a+ zquNqqbWDD@#@}gc?y}+|kUHHhA6rfK4&r*uT8w#tfgDI0dgBtI>?R+Ebe;D-C2FA% zpQnByNt{VbS9x+nUxWVxyFbJY^s{DiJYTnx`BTKYY!i;uwz$Gr6bDNin{IcT8sAK; zI;mzB+u)@rab{5TWWw7CCnZQvoctMU@wVP@{>{~{oaU1k-qb~JV4MP)Ly@ua(^>sZ z@1!Nwm#nQghQMfrv_o7dqBnQ;H47uXX2d)Jd)9!)*5dHe7Yg_a#opHX5&b}H zj*}cJjEh*-NBT*lRoS&^=8I=h90y4VoqcF=ymMI$2$7x(hV%uph3MUpxY=3t`rL1x z6!2IOLPc|Cfieghb?>2zc)Q;86*WDo=O+>+8Jpo-dh+aijvMpd{mp@d`5$lz5nKe3 z0@WfMF@mHOFEZ%m!|exn)7{^BaK9fNE9qBix?f?n^Iya^j&<{fk(RjkwqQ$H;NwNH z$p2h?*xzT4Lfu)yJp$a#{YOvup1$`a*k9#C=+3QM(G#D!OLyLBIl-{4gXDp+k;yhw zM-1o5`*M0RSz@>&QH5YCg9qpNoCB2ptbxHuN4~RD$OxuDCcF7$fn8>G9EOwu?>AwR zw3tuuRV;(DLiikQusYm%fdPVJTkG>1HP<-}TqIvs?7)qjPuK^Yjc7A+ zku(cI^vpEA6b0@T>td%^51`eFfL#Hu_@l*`w0ykB3(VNPSBch!Z9?4GG#Nt zPx!6xv@7Bsn@J0V#girOLLSZ5Kk(FEl#CX_eqb~q@FIVgm$bo?1?Fkc1qQY{te8yZ zt+vI70j1<;r9qOprg8WJ3XSw-h)BnYL6eXxgujK+je6Vc%}DFuFAzP(nBN=pH<}A3u5^o=V%Ke(T?W9*Rzfi6KTk@0!<}+NOeirM!ThIl@v= ztcxiy%o~KtU^VK-!CHWhpd^v$C^SUg0Fea*gBB{xXFoxfJwqeY7JUSg+&`BEs#h?4 zoAJGgjc$02M!v!%ZsLr$VO?O>bK=6O0Tb)InJl~y+P5TwEeUd~3}-DlHpF^;SqL(=xH94MGKIU~sYS^X1gLP1Q{yGg6Fk=xA5mFuS@sINP!~H<{M`Yo?Vfgke_o zkxgWC9PG|ozdMLbG^fM#f+RqIOx}mxPf>p)-OZ6 z)ln~$;dTH+`i|9p@s!ZB8_bCx=p2s7Q1YgQWN9ZK211DZ__YDtXK+?I_klVqtP;`F z{9F#`*O6ib2JCt#AL`|C0d%%tBn4R@uQy(4GNs`mkIhDT?f^sol6|9&G9a14loqmH z(7)%^WtPMKcUj>gjr+;te*uuq6tkZrNon`rv>4lRnuZ5QW%<;V{r#V%t_ZwHQk{s=-!B6fOyg$3eE}QbV7}cM41x7#ZvG(T?DBbpdTFX=`UV{G{U{C-)O)13JHPl zHdqvTbQ${QZW+yRtHS{73e_fR2xA=0+78X(BT52nsBE*<{zDtdHc!32dLM7y#4UOK({7x98ugZF?Nh9BzsqK3_X!*eC?>Ej=&P_z8#?4Pdz1l zV!qHrD>)I1`Q!rz!&K}>4Nw2{%(y-EUXM~3tjmD75f=@9Ld>_Nv!dXhKH^AFYa@3E z5291gPX?JP>VEG8muI$BEgX6^+^uw?@4$JzgTnk}H6JUKT#r$N#K2`>INYM{a)mo= zsQr8|Pzl32m=}@93!5P4B&EQAMlLLsZpl;u3uX57ov3W}Gd1A7;GJMRs(8bj=PbG^ zkTIO1%`fu#ubepD{0X@l!m6 z(a}ibID3>r6*QOHZOV=K&Jf0`v0qH{k7}`+)MBV#)W`yELsJ6Lx>gA6&jT<`+J3aj z>~R2YnpT$_X-ytn^Iyr$j=}TCTw9CLLg5BJGW=z99x>QL!DXJm{_ei+%e)Rah`7mc zqFy}v?#nlnB`EtgpPnrdmJ0DWFwQ2RBVS%#jxO(0Vxk}Zf@z8FZ~gdu2z3v?g;|&k z?+>|&A->wae}8AIIJwA&oPKC{%=L{h_gF+ZoJEmvE}NS++BX9%(R6!hmRaK?lN3hX zs^7T{UR=41CP|<}zxu3&&2MbNtu^jbFHf_$CmaNe$$giUN=n!tQ`r@)8dhE)`B-n+ z77n13N{Rg7wNF`PS2^>Z)!4s@t=-x(r_PIcR{Xw11x>Y5|em|gm$ZJ-~O zJP9Qwg*YCN`4(_dhVdrpD(EfngL4ehBt%DKFG+_Xu^aNBqn7WLlCCCAWGTe^e`EGlCV`^J$}luN^Y&D? z5WQ8<2=T*X2EaKY?=`E2#Wl4g^iG+M|Uh`OuNvIC04UlnkM|jr&emHicI7EZT{PbAx4hn+JtT} z2Es&tGXI!p*whd*R4IGzTm*!o1erP0sQ3P=rb!_~Du=nhr^dXfVf zNj~fiCa@BD_P6i89qP6^ndkL6OL7=K_+JopCA{f=V8;8WoO3^EKLc}i~A7cgxzn}cVwM=L9N&ECg+BrQ9Qu62HR8$J=;f$0OUSN?72wNDdziJYx%1?&{}jD|0@B1=i{>@+m3(T7;uuXVk$ENo&eavc%Y z`e%f~_*nsRBZ_OjqY33*d2imd%PkGBB}pr~>MfS$FKaqMv-S?0NkEodM1$V#ot=l< z+v;Atit^d8qA(&)?re%auA(>_633*JhV9qcB(OHpHDztkJv^{Abk)~uc^9@*QBnhF zp?xuIN?UjikDt~!^oEfWav>*Ng76y=3A%<$h(7tm-Lv90Gb77&8|p@Decdg$@`S9L zla;*5$D4|Gx^T{PX>!?{OU`y6*3x*tlaoVynGAd=Zg7wwup+z9V%G!3V`|R1d#F>6 zau{u%5PKjLQPXL=padnOjk`OMLT@Hv-Gu#Ee^bk5(6A}#%&qIL#PnVl1w~g?_g!(2 zz^k2{3WTv~z0<|1!C&f8j<(-3aY8Zohe5`U55dVGTcep^-O!huRU=1U_zYb>^Vxzi z?}DgRPT1`nQE)IOvNqf+ZAi3g!VqCF$+iz`iflRY(Oz*JQRk7E=vRgiN9T$oZQGSD zIC7t1uub0OEw2OF+@wo%I}nhNM5X}&&C`3XaG}k^&VKnTwaIK6_^)Qrs!HZ-6~z_@UG3Alth8^3@M7{_^U}Z|`D6yf=`t-4uKC_0&MF z2#phiHp%zDy3oz2Yur4km^xvv47zHjMYx_55pE zVW=iTYjQZ8JhLtdOOnF^|A|p(7Y++qje+J`kxsG#&{8nPfTy^jC*s&{j{_HBN};FX zH1$^wgT)N_L zVKvP|f(D#K!cy&#Ei#^@Tc@VHU%7U;#vITmCHV|xmh+t&Sw%+~1VtfGBu!;uup3-h zHJ_YQYI<5Z65Q*Ls?$@4%iwNd?#VTmu4-8j;(%Oh2qzc))dEhRz{9d;o@?ciDQ>*9 zM8A-Oh#n+ag@Wgbn^V3U*I)OJdA5_#Oo>I6_r=fwi*qB=7`qTehkOMo-N~$t)6=zW zSnUBPlws6GA8qH3(0~{PeAY0n^K%z3=;(rOsZ3pk5{Qa$rL&D7scSFHv~(oJqu@P8 zTdOVN{-7-eev`1nH(NvWcUwM4Oc@p{nm%_}Qc+s!a5DeEjU*v^hhg20gvMTb7vSMb zFN5eBSbGDcnGw9JMLRLpoz^l8?EmTXXw~ho@>=+064`ckKY>$sr?>swwu15UB#%St zw@>SW`QC+kG#WS}8B$IVy4H5U$#&$Jv3+Fcxo~+aV^`KSy%7&KV*x+1sZ)yh~)~K)(Tb;aAJA>iVPsg$+rg5#bNrNaFgvraf3pvCkB^s$1TvR-#OvPDvt~4u`67 zZS!g5ypTiJI?iw8_3(m=E^2I|AiNq)pH_=CXK=;oxGbmW631=(%nFA` zOe*`CHP8%g{5P~ASc%q1Szq74ty@xMN7w`lpLiKEV&)iWRAM#xWQ%Ohr~q`kpDfcC z#l zm&_$=>Jh!wN#Z%0|Y0xyEOsX1cZ=37dQwBXd3bbil7Flja9{zUdpJ6ub+RJ-OAwkM@*l)$lO*Gu}K0dW`^8>Geq{1@^J0M-|3D z-0^q0HLeSka7WB5ujY)>%=6Nzfn9_$iim;uwmLy&eLCNa#Y?bGQN7Ipt^oE zMPKC@V0ojvoHRpjb{sgrfWs8;SVtV-V)Th}N%}dmk>ak3y${LiOR@zirgSO&7*{N| zDEU34S11U6%oIo_(Cjd~w^(I}^CKf60QR8bSFct+k*#Eaw+I)=*v^+As7TdP1p2^# zCEY**Cq;H$(m`puUeR`ZmSEIoKMj58nwCuJA|bSykgyzxkI5%gZJ$UBEETTnsx6^Z zhF`oX#l&YFC2{gos=+VHYFRg`!uo+=e)9aS?AITuY;EuU)#dO#P1ge({+E|K>3rIT zEXB>XsI6NsJt~SE%vaqrR6)4VOl&BqB|fDgcHp!+Lv5zgxIt}AH&S5RE;vP#-8@pG z$1d^D8KT?}OX5T&OuA9l;ZLxuI4c_d6d9vA2vBv}rfYF=grO|0)gUF@;_&Y7j(n!V zC8`8%g_V22niULg_z-F#_M5hraS)Mi6JEqN8h_E<#KIMIPtl#i%^E8)f{%;i7=vzB zeHND(tA=>R9c8s|r7AF+MCVNr`VsM-qEfat-hqsW!64&ePU4-A+q;_V=G_q9praA8 zHr4~LH5Acj8aoMc50D+6a~n~9gtZJb?;i1lZzL6OfgyT7oAVZ|8Hc^0WTkh9*|vwZ z3k5f@0*vw9g=alf%+Q0)tlRJICEcZ(biE9Y)10Q9RC18$wHgwM24rJfePWuZw?~vB zEjnoaa~-Z)#^%MWr(ta0WMaXirO04mWI6#6 zZxti+l!QdCJO=IrI@H4UV0vCGnjfHm3Hs31XRGkpTOHReTk6|FlkijsH603!L zsq-cD+Rxc^;3uC*mO!mKk$npP!@%j0y9T4qPg^iwr@<%7R#t{EO%v9r3Nf&wD5jtl zfT%c`gU1*(ae|pJrStiuTznuRAV75tM7ap8tSF~U)Es1pZ-J4Zb9eIMoR5~8Bb3eh zt@l;f%UpNk@;BmWfW5;Xw|AZh0I;dhtept>ptOa-X@t(I@rDC+sng+0oBmJlXhT{y z>D5ob>Q2Z#6AHInG*nOxJ3@e))0=c#?<2QnVGOi-M(teVH~8A3o&Ufz*({|pJg=G` zl%zNeGsmt?wmUh$;hX%l_*&Fcy`Zt8*6=uDmuem1X`vn#|OxVk{~)zvhAcU3JHv*qGyHm}YwqUx$HXmPIUmxuhi~83TVEV?k@+PQ!}$a{4+rT-Or@^r5VwAMTW^wS^F0Qa|3GWM z1vTCRE+@Z&c|55mSEx@PU!Bjd%8Rora?`#-SRx#Ieg*tce$nss4&Pqx9&PpyfBNF+ zZvXAp7h7j&o0%{2GxpHOYNZS{xDeRCt$jpl4VA;7y4C=4Yg67FXQt@8A6J-M4~CjzI&L^L+MQJ`+oGHRF)y!)yok z)#uFKXBW%Kq6BKZe@Fki^9Am^&$qb17hO)uJ1Nczv?!ir@RLj92DGvu z??^HFi%ZbXvm&f=lv4VG3oh{7T0z2~mU8m96q{q2d!Y>POf5=(C!fISX#d8_LkXf5 zQj7ge+ot~AZ`QMzYy0yrT07B^u%t)BC+K)~X8}^zykrDzJQ#zFcNhE@4-2J+#RQ*t znjU7oSJj)7YF6}VCk(i<<&<)Re7cPVG0G(Y-W2t}edA z8^q<*TE*iA6-FYjsOd6(yyE%OC|b#&4E#05{)Cpb#lX+h>mlC4?tSb@AlZBWzc>7H z3iNLA6-9;_q8YzR)QIVna5UM!e@0rY$;tEfPMc$0(oYJ{{J*N3vGm^3Ww=CsEs7Zh z7fZZziO3d+x`IASnCfjf99JZK=}|}a9uhCZt=)^QC~pIdt`{)5$;NF`9gM@`b2 zIW0vm(w>z^eTBmg&IUmxX&qak4ol*HEb6+K!5;lm#q=zO01ha7Ez3kNeX|X}Bl*{Pd@E`)ZhB3;ncOCnw! zEh`e~j@8^eC8fM-bn%HtA&h7G53QRaEH)L))DdL0LTVW54i-?sxMA*)MvDON=q9&~ zqIEW!UQ)c6kEF$@n1sN%-k=w^_Df$tSaoX>EF~x+)sJ-gnmzUDasfE(x6VzmfH8dTX=UPw-yNZ!C2j+Y2-AfcCy_`aSVK~OdX7wcyI!^V2et%~OTUNYdkcdS^NxxO5 z$4VtJa*J(zty%C;7C-Q^g9giWB!=$p8UQ6HOT4h!BoeAR>{mlaar2BS z&XO}o1>21!^wyPP@9>Wyo~WZZfF}h9bmkMm?3)(^dWA`OT`Rp}bT%4r#mr;ba`wXw zT#A>dI@Y;?vMrhrq;KfQr(t4Fz|mkRu1t28uMT&|fPz(#a=?kk>3WIsnb<6x$L!Z2 zhA$9-Zi)a_H9}zIXr@8}>l2c5t~m{yMb8+QjTiSUv91H6+8DfFvnrb$Hq?AB*EZm*!9Gn#4Q02R-~;6(*j&h#chu>RrIUf(Ceybk(cXA-&*R0;uAE zKXEIo$FV1o z#p9m_rLauf@e*XEpcdv9h#Nqkm{dfj+kBASYBnsYO6dT*H1FFcaB`U=xAr!;hsFLg z7NZur&57BQJDZ!+(sEdm3s2b+J3LN=A7~a{F(1hqghPmn&VNM=yvDdkpt;srO-zk3 zGjLPD4TWg)zaY&p}~j|;jkfmD`cXb>#+$p>NzfYRfcs>{VP zX92W#`^om>ZApK=It&DJ8I23KKv7QclFpdXioos3N3kHr1dAdk{k%$bFZ3Ong)ldr^%g5V1$(p zm*G5i)^>>VxJfq-lih-(Ha$!_NyN7oQ%bTZUv;7Z;^wvC@f5jzYL%dpYKc(`D4i@9 zzCKFFM0?GQ2FxbdL^s0s!VptAL-v4;R5O?AcbDL1L$>WMBP5C zd~nXFY{S+RANtgADwT+G2Foj6*k5=bY{`MA`-098aGUe;c)6g2E_~J;#}4Nh&9@{r z`biE`AKi3ba-!Y}6m`$Lmtu=y&Y?h8U9FZFU$dEYx5zVAzu)V4#Z}~o6_GFBxH*oZ zXK?E6Aqtn!K|BZ{i@6+g-+y|3xWgecc3hp#_)DF|9*>CK)Y5);G+~AkQjloQ^LLa+ zRN(R`(Mln-ckDuYV&j3h;lf3zTPciNmHHw*e~OW11=ygYp96$L4~+(XS-x;D>Do(Z zuXyXbm?gTspH;IDJ`)abZkpmq61dlCK#+6L9l`Cwx*{ghUT70{!2lmt4zdZ(PBj@n zXQZZh08|CJnk?t71+XuJOh2chg=(+mDFbNK=CW5rYV(5Nm8E*YirEVdj;yxdKRSzy z;=4q3`HL3Dl?i~O;kqby*~YRpaf^6ll8`N`hpy$w(xfS8E=()KT-a=RJSk5Yfkczc zbN+dg{*|-snD;ad+kuuukJ5x`s18p)YHX_0^8GWF9ADP;5}n!OjkV#u7zdf)o9vnI zmY7PXf*%4uK_F+{7|uC5<>)?NvqxKYqCqjgnS1%f5IvuY16*J=W@g$%A zTI+RQ^X%fh;`|!(jH+;`kEIZ$*{GkId>B$r8rZR&z(NM#WR)Uh?wPNn46K-ODMZ&M zcPr2ONWgHk>>ll~=pGEqBYS9PJ1Z|NJsWKh7@fq6;atj5wToYOQeM~PcqH{eEutEK zRee{D%To*%g5yTozheA?Qng}sT2@738er06-`=^mbN}9`6XDjyGZMpIh4a)j#?#{p zmQdCGI1l#Q?;=9yp5-7eAs3a;VN_E}qCm0??%S6yo{q8~{n~=StU0)_3RNXjf`z9X zULD%57a&{~XCD~UR?{BW%Tt&b%n~gbC|~jh#ft@%yMafi{lhWfz~(YEJ7S$SJ*+4fAqChJ8M%E7E%_VM@x% ztq{DQaR%gJ+2|cSdswO)6Au8B7Jou9>#a8D&skQ!SFE*q>i+e0uzKr0v?1FI!d6Gr~2p$XG~NoXn+a*W#z!)%R7SJkRm z1JR50ES6%$0o8=r0nIY9zlJv=AGa!11bryYEqk!`qaJkw)n|9j$t7uv5L7g{7KU(b zJLv~)Z8$d?W!rhEynlFe>EJaGo~?o0h>MBGh*kw%x{=f`4N9+8=*M_@%!Yzpk)TlT zhv>DLu}+=77$p82QfVq#>R!V5s5Dp?AGSHhCHv^0(|`{!!!u&0$>i?po>B$PUuzD* zh%8xzZC})Fx4Y&=iKNfxQ5mn95w#0u&k^emwU#^z*~h zh?j>R6}4qht)x(aYs6kM8T~##^Ol_(_{oNF`PE!&*qdej_wT-iDrk52S2=WbKQ~39 z-Hn@B_Y3DWc_QX(o=26Odx=NF%NL*TKQZ1sizO+^oQ+24<@DDnCO(c_raJMdaKSMV z7tK@S1(~R5S0Y4Y~51^)O3{l z;ij1cw0pN_T?w}r)$oBBQkX?cxU`bgNr~}uSNUCN4D{aa}F>Y4dT74*y)qJL{@2KX(yCHI;03l ziHCNzowTIO;D5o!N>^p0xwO$w_lYfar(uu0cBvu*WvZXP!9@BiF#TMtQf?#qdH;T1 z0PFUyFhdM0m2gu4SG8bM+q%u|P1JjxmW>hsU@ZMdVp@nv{Vk#8aGKW#EPU)VyB065 zOgkF|J7TFe!B4IDg~Pz3XX;oL-N)%=7N9H#K|jaLdvKn?i|iMLu+}??@oX<<#5N4r zusEQYKXZa&c2~WiI?(~n**@mIBk4hVl%(O^aW(8w8>Ej$5JppNA8~VREVole?}Q$^ z3@=J@z8sImhj(1E(?_5ss`vo{Zp=y>9(~GEQGJ8t28aM=JVj;FGJS_K7HZHbz?^hh zIVnr2x{fo7Oa5Kw=v^d1HPiuxS5ZzliC@%jnwK5pRNdah}(DfxvxozHr{T zxWlyVZS@4v;GJKFL+68G%Q8Y~TAaZ)oaX|24Np1GCMLuaH+=*>RQG*mt0;#mgY@_n+?48*S5;+We}cXF%}Ta6=R~{g*H9fb70pRID(7F-b_f+}^D9j?i@5 zhsuG}&I<wZpQ-`f0!?qO8EixDI4 z+1_K%pnLb9-Xo@+J#a}I4J`8)+0#H-yt@*tMR#>2q>G9VezF}R8jy@LDcO2l2(O%7 zV5-@00!;2i`2oeIBMFZ)_KNqU&>N!v3BhUb&*Ct3|KX$E>>1QCPn_T#yTI{IP_={U zX$&bgkN%}VEpws@I2D))Qt8)GieI|a&}4B8;w2JZOqE7A7#p`;TnMb%qpio}_W6h+ z4_YaNUVq$jZ0cnwChs;`yR_nirgFqfs~WGU>#oWxsz%}~BKh(5gY74evmx5pSTv!5 zdtu5ZJ{Pa2nl#C5d!zVGsft7~DLu$6RNCzR#%2Y!GH9U^EGf@yR@E~Mu|dAI61G`SY8B$gMX!W&5;;=G|RH>+WX3s0bc^z|nhfLcvGy#S zRzlZWWR$JDAuRtc9LN(4FKY|3S7)WM0v+BwMDNv;Da+sYA++(RnvcJ7OkciZ=72ytgZ8EH(vz~CD#yHl#0mLwxuTE z^x)q)ay@pTuW%!dTyOl|YIxtQ;|l_t$y*EYBqU&s!?G6U!RR>ZmY%!WckAT2P4$m! ztfEf`FD+u!*^*=l`AXe+L@AHHIdtLjJrt^$!WvUO6mf^J0a@Z96vD>ge+HC8_3VXu zE{p5}!7e)WcKd}5hdi$F%dwKza=eZ{KLu|n4aCN z`==K_beZPCM*o4I$z8{}0qPIalU)fx!UycB>YsGk-KndoyCm?R?3ispP$+FA_~)cy za9&Ld-l$<=VsiNu1LS*-;JnuW>VJ4$_AlqMV|QXLR`%m_jG z91Ubx7%wPMEU0}N+a0XHmFi<$+jc*;ZQCW#n$UxXh;1aXr`B!9PEaw7XlbM&OOO;~ zORqu!QE4tbS{b75F1aWQ&F%ik>zfw9V!d55zZGa4I3z+h_%Gi@)auRl(Y0v}U36m{ zWjrt@Sarl0lBmCXetYP{RKT_8(biY8711oJT<2a(DaGmgDPJ+$oYioB?5b=1`e{#M zu5Adek%P7+*md0xJzq|fzBl8Q`Ed36?dueYcR?S)&30AokZhR{FmB#D0&$h#^6Ny8 zaQ*}7q)DX&0ptzv24NCw^67r|<_)?8<&)Vt@^f4C$J^fFPyM62Zw2SGn&l^m9u2SM z&$snm1O(+`4BlyJb_j$Nr2H#VG+rH7i$!&DH7QOPeM(A+Ze*9RLr6OYvmuZ@Y<`=f z!HY9Q)&eI$~b@OvutS*;X+f{`Jk)xH_q~V1~BjV;H6a3|$gMhLK{Quc-0f!L8xRVUZv8M*;wQ z0mQuI8R0(}>vefSu*6vea+`I3B6xcC|A8`uaf{ffpE1HMon-tNTAaPV(Z>h3X7Brl z`S7>@INFSAnHQ5+%Zp|mhnvHr{(ehU?qejfI+^>~@7Y(cpZ^2h&L-8#uicCVvbuhA z4vZqc)_s8hLycSv-ISLc|NVtLY*;G zD{wl%7QLAfAjPiwDoqzZ7P#?tvj^MTDj?Q1^MGji$+|yOGaS!fuo(XF6>U-c>s#2R z3|9)DfKbjZDD=^f#qqBt6@QKYd{_O3|GB8U#4^He6B3YT@Ts||WETuDN(Pr%g;ca8 z@9Ya{vI~Vs5HvDrRZ#VU413ay`o?1&7d0~lv;l)hS6?^&7r(Ua~xtuFut2W$6O2x9MO|=mhLI45;Z1s?i}6aM&GCG z!y>ys=W|(~7X=OV5>5l=m5Kb&#mQEVu+0F)%PyhSkV?_U+cp3kCYht+qwH9&Pu~Co zBtdH1ea;Vq=N3|t$lp@g<<;w=U6F;|Z*(@&vdPfgV%~~Rx>S=o6s#GKd~skk?p{M% znr=;cg}hPs{IInzs(PUa8n$FpSWGkI5KMC)K4=1tTK=`Q$;% z(J@vxyp|1SQ@?r#I_Hwc8KdeCvUFFlCVf|`Z(dnCFrP0lQkKBl?KT|zKOH}jSHJl+ zXW#%rFUOe$&kKgu_kfB{;xyq-j5$f7DXxgEDlD|b1_MrbO?*X!>JoYLG_s*zpmPR( z#O#7Q;%;}_d&A6;&ctvcmOE(X=wgH?erq+JIMvubp;30uq+0PcHJ;D}XnNvIr3l5zeTkKh$)JmuQ8?1Skx-Nn{|JZ;Un$GDb)x~VNaJp;J z2_;X(i5O;RIRU-eTkts4k*l1D8#^)j9uu`{O6Xoi&*b zLoYD@6GGw)?(3Ju9YklW=7b1jsGIEBhokD)X;a%H5gEFJozQK@>WgARQo39(Y=@>U zd2xm=*J=Eztp1@1%*?3t))v@brQEPG>c$CUCwKwuzQM7X5(0BivI8ec0vr^NauAkU zPFu<`t@Yg}8foE-!89ITg^p{N$d87xmT@5h4VE+EaiQbuUshN^ZagZY`GTB?r|i6# zP-0G{svIC(XT8mRJc`5pJ_qtfrRq_3`|iU; z5W&esm4Lc-KByxEz}|)7@9aFX08&7$zu_yMHGd;Ol)|KQi!2L5)gD^%cgNesU3z&M z?m4u>&+?k1*9LUe+?l<2O47?D<{1qfkti;sqT5)KXdR-6*v25DAFCevduP>sWqaP}|h^1mqZ zDZ2vT?#}3n%X&Co6yHDKNRMGF;AL#Ix!;nU(q+F*Zv|9)P-qRY|2QiSqMT-lxPOw7 zUZ3{5$a|cT@wD4_9{DCrxWMIZIXy4t<)YiN!bERqediS>v>PiNb-vIGgM!E5$}AXG z-EJbDMFk-UBV*D-z^Gj)+{M|YXXIVp)6+(eMs>hbi|?m?3qjHI_2b->&59TP@`Xkc z&8tfSqD06LfmgXxPNDL0e8(Lo8*s~^efV&j-Af&yc{-^M3cMYo+q2UvXPfv*0+Oz# zL*qbsiZn869_E&(7WP8-%ulQ%bv&yMrA}nJHM1HsqVXPi9k0FiHtwJ>C5CPiWEwYR zD#nYMVil7@{R|?gSxSLg&<^>;3x4m9^A-O&_kUnA{$IlcT(fi6?b@}Q)|4rewtdSe zt+H)Bi)dSNfQ(5Fo%%qCD(&4V-j_A%?UkrGo~1pL1$QSF+<0qxwTN^^S(q-%0wjFJ z9Z1`=l0wks>PZcMac3Iu<3rVP0dE6kt4MmO^ah7MoFer3m{JD1gtXC;BF}+MPH27L zQ@AxwbOgKR)E+PASZDH@_?}IYxGl$|RuX&ym-BF}}>_f>DHL>|&B<6ZGDj6ECQ8oxr`&p2Ow7s3jf*gs(C`hFvm)>uC~VBE&8E3}>2Y zH|+1%-(}-@e!3v`(xURm#ky-^xaXjRS1vFL;jxhPQ2hpvZ=Y|25&dE=P*0h?(+Bcej$U{z4?^hZ_sa#_$T_cw z@TzjR<>}1^X;_LR3RijoKt-0Wl_!v8jrFKT^5Gu zs$5}7(`Qsqsv?vZ^&v!^vB6{`?kBo6p|V;eRMZVXxqylx0GR48A#|Q=$D83&)0BnxbaPrQp&yun!Nke}?2dN97qT0#3^q1dO_?nvKqNg!PvBMYhw{8<18!IYzDU z2Xl|L?(6w-x+pISJ)rpNTmEED;>Em{uR8w%Ih8Oks@QILuGnH|9ia&Q{AlM;3r`I% z53MC{%HIk$?-Uo_z~c_dh_(tfy(vnr&JBoh)Ts4m{XqD|Dx&L*QF!=6cRwZ$QoyXy zs=q5(ZgDkQbBYItDMC9ZdTidIu~s(tcxllC-rRp$BA|hC3qfq_Y9n3X%qV4( z;1#1h6KVFKp6~L-Io>`!#I%+M4l&_HHbkc-?JkKPeS#h6g<|)_H35xmY_}F2ke`x# zNnw(7r@fxDF|1X?QPy9P73{0(eXQ;v7iQdRCNK4TVO*4Bwx6NVNIcfN5@N1I&O)Zh zM=9twgeov2%N{4vV?~H~KqHG(KjpMs2(|=rf9G&^e?ty=l99jUoOPVuxVRDaKJf;) zm3A_6UG1n;yB+zv0dM;N|FgLd^K8bc;|gf=F%0?wM6yYZ-qDozi@CS~-dL!NE*Fvy z@{;n(nUL>9R{o;v1?$Z=ZwUTmGc!+GagAVgL_-B2w)U_(!RL(^X^6cSsT7syW4{@KaSv!B{bs=4te!^ef}h9s z-u^4|7&%#ymgQrNsd*h$3+~!)FtCAHtLpR=#Y8oW48(a7oj(7$ETI|k`$Qt;&Wvc! zkQ&mTFWn*AHxB?NB`ws31$SziMU7qmh(~}GtaiL2zul!L9sVk6&PMyhawvMyBATT1GmAi$LzYP=40Nq70oeuG~Q_04j4?2QO5;L8-Y zu@me1AQeRss!3*1JzWN$8U@m04!<8ZeKF@a9B|^iX#F*ub7-s@QD>UYmd5|GK%wB{ z>;lYQ!IEZmWZuGjs8nCGnu*dSk9?x|5Z5%8*1gU|~o6z0$ z0q~BLa0)yI3B2R+z<9yQ!V=c2li8<{?=EbT_pk4x0Y2GdD09DNB>7qecv z36CN_*0tN(=$gF1IJ0jSH*Bq5=8G#Bti6AE5+6nY?$MzPc+&V6OCMV-WKt3NO9Qs9IyGj?z)Ei=7x~7i=-?XaP>PrLL6i@8vQ5msk7X z>z%Jzdq#g<($2x3gTJ>qIWONRa1;ZYS35eR4=Hji+YA#BPS}Wc3%<~R;5P-S%Bos2 z5AMPqjn|9G5|jioGUuLnhuC_EcSXj*V07f3Ocwch*s81_I=d|10jnX*cQ{d~kvD@5 z9>LeLjejL82j;l;%0+XJE!iN~)u(=iS=$Uh$ZG_5;@`jL{}K>byK9BdriJFV`73+L zmaWe~yPd?+_}bE$DZGB=47D~k)@*Ye3k!*HT$MfKgZh{wr(QgN_D|lk;Fu;&$InTa zl`+Nf=&=<4e|6!d`AGl-O*vE9D-7h7p5rQGgS5LoB4WSgaUVKPfcIndsQ<@Vn>RXk zvoLHY-tT_mg8w$w*{(S>^0WTEsjuT2FJCyKiO-VpTW}vD!FfCh4@wG%o1GUTs7+=R z)ERC}ieCu(<4S7^zoDriA0z?qAOH|i85!x}Tbx}H$ie6|p9OAhsn&Cj`bpH&o|k2J zAE>-sl&dTf$&4WLN|cA z+DKM}Mt?I6b#FRj{TK&->lt&`<Sf`Tq;g`-7`X)f=U+11sy4T08yV`-pzJ*b~uQgI?t8LY#n4tAAn`mFsA^at^96B zEp%95SU#4EVg(kSrF-c!$7>pX64AXtb405S`3~+UC<6yf=UopM{)})Eqb{4C9Z2pI zik&=<&iT?{;BR+X7MK;1-$pkc`TPWF71{2u=1cY-CQX$LD<&qr^WecZd$AGka@k|y zQ`K}Fx2mn*+1-2aFghRW`utsB>l4?_m_*p;z>>Z=>|0t_&7T{+65K(rK{9#E^=p}+ zWC6(o2#L~ARn`vwS1#ezGR?6-Z7wxztDQvHOsz7E%@u*85P=Cp!8JmqtqTu==VRSf0!Y6jI+y7n|a$ev?~?M)r9)jg_u+%}uen_aNZ z1A8O7kfPs3baz8vz@^K_RSS@vblyoqV6CM?M-J+{F5U-2Pd2t_&ur;yceAjdNr6V} z&KH;_oO}R)j>n`S2|JBqQ#Ac7TOCPk<69_07h9s3i29VpY9?P(DyLY7u#uIBx5oQT zRTn>#2AU8sWVWl9-IXhG@IMtbMp(5!tS++Md-v|&3$&=Fc2QN$^`Z!eR(vNLymPq1 zV)0I(vdG~xhtM}#Os0{Rszq2?%y9U4-)@tj=(kzc!2JCg*~B3n#^aX^PO>4_7+`b+ zgT1Hvea08b8gf2FwTlRnr>l#h*CLkvYkJYBK-~Lo=i&X9IG#1no;W74%`bYSlkaRX zAHdos+b2Sl(UaE-nrQ3Pke)i5Q zsg{-dAnd--Sn-&tJB|@cJ5}u>PiiV$RDBCvR}4K{Wm;zHZ3a7C8#HUeBEq)TqEd<5 zp+wCjJ8R^WBj)MCEf;?FmBH{#vwzVy9D_!0qk&6C|02xH2paL-z5ebVzvO3uG#3Dc zasnS?E_>5OnNdI|ZQ+0Ciy|X+%~gaMXswqXzZc=LW)D#XJtn7?*hdK}hZVn|&p7a) z;{yA%+9cAyn>v~|$fX{$#T;InCYUAf{1$(%&Gl>JQ234g@kgnoX`o?5WA% zpWen@$z#ZrP}pv<*U2xI+JOOp?>*M5&vyM9=s^58;YYfC9T+T1IDwrdheMD~@^ zbdLXIp@yxMWw}CN>?lKG-f8Y!#t8{k{KTmEoZ!mRc)m2~PS70scyW!Pu-Yg-*QY%c+mO8YEb3s2$*IbR$H+)I z<`epphR?K>8n!vwn^;=-FOP{uxboD}G^YX{sl5;N8E9dwE{$x^uMYJqa1{=GzgIgV7e`}BqtiwJZ}o-$oTxQyWF0zP;(W=nM?Kn z*dZqBHh=FOthkL2sPQ5YMvJQ%ftkgiKNCn2Lu_GVVyZqxKR7BaA;Hp<*wCM(>lBTZ zi^iw$7AK71tcQK}`{H+Un)y^yrT%v+`qbsTsy=w;b<7$xMQqG@gh%s~Icg0)&dq;L>nD{kNgbin5I@fbTU3Wn_hW zP|w3OB~~k0fJqpKg{f);Y6IYERI|}v1j14cu0CXRqUf*~x_+D)8VxSzdHW=m?A}LDhUGR|od;L8_PU*`P=D%wSbxYoR z;A_z&m5%-OkNfV!DBwLkk6Wl3FHRf=6DgMR*KRiLd-d>**p46m=>Ng{Ey)4-aypvK zUQqlgmvuSa;}(ZBS0($2&^4;tKEhV)M6$_X)P28DL{Xdzn%Jh}6!m1SIohX~2FeO^ zbq7E32QMBcr%{*l@dQqdT_1ZbmC(V0M^F&7%D`1jZ=eZdj!Kvi(qc*XDYM;6`X=(4 zBjZ<2dTCE_@XUO`>+1681!iJ3WgVl;sW^?QU8Vm=F$ssL{kL|6@a=%h38ZL?F3nFq zOLJrAiqk)uI;G@-K`htolJITw%JXKMECbQ3t~s@r+$HZ5gP}hqVFxOY*31_&HGfV@%-X#kUD<5#wB zyvsFemaAB|%WZ%&$;^u(giq9dynf?XPjl>!%)fRgSwQLy->ivfn04Bx(-ks5ovlj# zZ|FPyi+Iv$F~S{Ha`lTMTraB`Jv^IE&faC?(<%SahNDsYs(@^YG{E%NYR>;f|u(!@tn=m-r(zLRbo#TrN<~RlDm1ymh8~ zA|F3-_6i1RIGxPc-`3;_XU>n6P+FSoT=im#H(ZB(+hKN$XmSMlV}pmQWN3vX&?TY+ zn31$e!nwq3dg=On_JXYY;osFVYYR)XqkfD<(zfC_t$zGKWpgm7qw$8g#E=!$!Qb2 z84-v|`Ro{9oRD|ax~;^C7!KV`#7f$C@3u{NOQuQ7keM`;iepg;U(njb$!S`g$%P4a z>REC25}Jvg{iK)j^KBsbXO2mo0j3WtK`$4~13@crNJ@gsQHf~E!JYnH)*6ruJLnuA z{_Uju_Tc#c{_W&07tBZFd?7l6{@ugw@HqQ@c>;uc2n#27yT30EzZzU!^fI>i(#|E0 z?e_(OkKhzZCe-Zc#WNb;oo^p}bFY`(dAPgx4gK@Y-jK#hF(J$1teBE3;+ZfD^y;O) z5gi02ko!2A&+-4;$OH!UMSp9Xjw4)fNQ(i5#>eZ)I0!P`A&+!c#N2;W)jy8jeEjL? z(T|M6znJ@$ejofkpr=UuLgkNtzYeXOYz@2NQ@qAZ3=Z!M!&HPP!_M#Ht?tM0e*-~e z`#*U;r|_})jbgh7&W$PgRld9|#;uLF^y|Rs`tg7&xSJ7%H|Zo>8!*~;p@6(m&ExrM z%C?*3q8P8j76>=Rvbf4L)l8Xn5KE73F!-Y^W^*7FaE87t|1g3(WS8pQ>xyC%6*ENl zkY0rUZD&6mJ$do!C~zed?8iafOTbG&*&B;Jdh+Jz6*HYF(-(Ttr?gj`S&BnRVWx-1 z1kTw%%1g@3LKWWYk_<$~+}y@PukLr*27^#5_<@%MsZ0XjG zXz$BM6y_x+_4_L$x+vBSV;MVa!_Zh@jf@TJG%YUZNQd?VK>@w#+}YjP`R-wNpWWBs z9tc14>14*@bpAe&53Jv#Q8A?4Vl}3yKzk=r;+)@aL8%8AoZSq7FfsyiO|1vBw zBFY)FDHD}~bN597=OV%$(vKqOHk=qHn5-}&TCS^NJUIuOpIrorY!-W0Bly}GD^c;w zY>!yA=!;W_GXW&8)Ke>`CXLtmMRMh^u7)(>Er0pITOg1^iNQ>&OAIyT-k+qvRY~$Ky9eB#JK<@wfpm5iKB)*;qu2+Hs(>8}=a;TR= zm86W3_BU0Zu2lbu_cf(8376xNGE%$sTc77i#vfH|PY_)JIo4>8_zBfJfiEnNrWKFf zEsrN#TPKN}9G0By?4dgg(4TTCU~HB^5g#Hi7s-iRE>nC%1CnopkQ<2~Ym4%dN04e?Zr8Kv3N29akG#R ze?r}?A?T>~P3QLMlkkHYnMtay0Ge*rCKeAHX2MqzAvC2=eLIzLpErFd`>?hk4P}(T z!fkoqDY7K3ai!B0*ul+?Ev!4y^;QgeJbH55c5p!!*5BH2$qLK_!cQ7lC`uAyU*btu zkK7~8#+tEifZ646nP~&9f#i5zj@|){>3j-EXL*@*U`A&sq%mX%t|T9KTM@bRyN3_I zbvI(D`-s&Lk|!|;n+q4=fqHsm7J*tUv?hD2S&!fMCBo4E{1ij)iGQ8N+9iF<;_R4S zoiyV`209h1)RKpq({V@%NyA8?v8O*k7Dy?tWe;2vrf=3%rn!1HrO@&cx2zMw<tCc@P$`5v za~>ux7tHDwXA=~IFXf9m$1gBuhWDBgJIx)$>%T|DSJOlACvp*nLOuGeISw(;EcUum zJ3XpR`^nae$?|J&Q>$v*BgY`t2!nYcN56`_=j&1=A)e&e?G#p;$@q5uk3|@dVPeZdhOY7I>gpyNPu^=~M}iTe3E#~NJ&0>xzeGSRbKgJa zvrzxn0u<~_{+;&sp}N1fV`|lC)Gl*`NY>`7+Yie%c4LFmxy~`F`|~->$ZSoMYtYw5 zGZvOer??YoRM^*X&4`WYQRa}9p{deDGs~l2MS4U04u9A8ZE7XTnwja&~q}hFk@?W|SBx{-G3tz9CHaqyPQrH#P&LzH{(&rOOIB5BR(C5B7b< z-;ZS)^!E|g%L=YYZD2pca@Boh*?;8E#O$zCZqaM^*>9ssZyi;;`^{NTT8s{`B4Z1q z4c1d1LrgUV$ zXnA6&Pr?ygunUbofuq88u^7iEG~e+jXXBJ%oOFvMyU^42s+_PDLj&5e~jLbBvW}y zL)$dzv2>DJL45<)73#&nH9(aPtPZ6nPsj(&*VNWmzy18I?Z@m1P||gM@xNcccpiHR zLkd*N09Lvj1G z*i_IG+(;!ya5teVkkYb%Gs8^5chxCh_3>4;oJ`?~x8ux61P@gaZ3IFLxpvOPF*E!fKEIx+Jwt2an5^?wL@)%$Ltm0^G%IF# zy*VzPv>ELpEIM^l(Ma|rkw7Iu8)DMd3_?D~(qn3!+2F8=&ZqepuzRKYMyYQC^r|d& z&}X!x2(~Qoq;eUoN^#WeE4wLD6Lgs5K8XlAU>~T1otNZ{s;`o?;wB_&6KVP~eR`A=Q??`Q_C!15h8$mXmE}Xhmr;f*-AxKGYQuLkRM6NI-e}^e^AMiE3s{| zuG0-&)uK#VSP~!%RGt5L1nZid)w~1FB5pOSP1yWW9dvj~`Qc-J1bs_iUQ>{_eJsxi z87i3xgxgjs(VV=Jat!X@S@c5*a(gyfKy^ah~;cr)!YS5IcUX6|@QOw8=-~D!(cgH*H&x*Cb1A za}*(raN0?U)y3P#m^(~}`$?xyE0HZjZldfPz>(MCDhgL2+U@cVBU#`l^XiZq(&x`9 z^WA}~{!Q3>EbCF?N$pr}CMTXR0=pTVLIZmRz_kl0xp;KSitl!~-p_J`3+Pu*pdk5h zNjQew!htD!N8=kzj>A%*MYPDgCsx@kKK}$LkVRxOcJ4ZzL33ah!uSVBCJS?oyqv|e>35*7I6z5G0xbac}92}zcd#e;6ZFH=&$AN}u>gD=((_L`nmzG8M}^zWN(2rh~r zZb~>(J5tsmNF{`cv4D#lyugC6Qd%u>2*=zK*77*3KZaU)Zet%hEo$GI`+~K5OjJMBgNOgoO;OE9S@wbo^JJ8tu3$yXhVzO*iZu^%C_V;=i9PcLJ+8$Bu8%zv1O}H9?*+ zG3J`KHap0wtV2I+BQi+yytB#@`~>|*|KeMAwbpmQG}PX_VmEK{^o)L(ojc!tb8mOZ z0wz;TbgfXm!hrMPniHf%^;*iO&Yf=_K6o(9o*sQ0IJ=9Cxi`kLv-jI^nPs%`i&Ta%TC>#@$d!$wvjizN`3UX9T;sFpWDb3gWFcTtJ zlzkR@0K0>~uxtAAN8%qP=uSQq1+&+yKGebX8q}MihQUCAWC%2I5~L9^#V{{Unjmt_ zBfNtkgHRCfo!z~io%_^ZF*T_m^q6=*u4Q|GMPkws;M zq4-KXV#27u+*hdGwhVBOvNJCT%8`d8fH(|71}CkJ8qP+xe}MPbsduXBtO-)k(xvE)oPth#%e z1JG4ZVmsw4q5KICcLlA7h3RYF7TV?gm-BKsxetOYgm6hzFD}nry5x@oX z%Q{dgBymZ6ccTqeN;LVr$*Y4(us1*MWl5}Hn#S)D4CYQz(=tDk+m$+@V&|S@?ldjc zyyymnC6TTN3PfZy*jNp$Ud24kRoEaVXf!)o@dNmkGOc9L3{;f}0Pn~|zc4D_QC65% zqC_CskH6A>|4NKEdt!s%klb1_7R$cvD{=9eWJlWCN;AQFDcE9PoV6`kkYDBiYZ(az zUXmkoq+R?+IDS-2+2fW7ii$HFUHQl*H)b;^`{8%=)8C9<`(jv z`#jvR+dtFvibV}dxJ5>q3ROPQ7)h(fpZ^T( zS5s1}F%!$9E5bdP6>gZqB>TZSWBSwc%v|R6KI@4aA-@SXN$5grBR!hR^|-DIIYbfw zCKwVDqR=^7k^lq$M&#mEU5)d=?M zCE>4cP;d}zFI2NQYRQnndpMO}sDgL~FWKv5Kn1XU$)aY1*4sXifQG>_M;Za48Q(3E zy071U9T^EmP)kC=e!LaG-6&e?8>#wd- zyWV*8fYT!)8Bc5b|8rZPlBQ1JL~mAR;GP>yec5+!d>pLM5^r-pjw6UJhQj1l1o2MR zu648ByS-8ybkYe}x&LzGaVlcN^@`TcvO+++Y1G4fnsZEaBmw?Tb+mwfJG(bssl44C zGw2H_AzwExsd*cP+r3KdDVVj4;}Kh=l@e-}VUl4HlSkq>0d1zP-mSI0KnL4gEo%X5 z4wp1;F@+yz{`JlY90%o~uP?Ecb+THm#E;x1j;KctvNB2IyzqU)VT~>Eg{~p zIItbg-T34W#I!-pcF}b*HQX_hmQ>NH>y!NsY|DpjjiTte27JaVi4)t1rjK3zBI=e( zGEb*^*&d$=sfpc8GK(YIkkFQ5K0+L`i*gJ6Fe`W5xoVk?t3gQ_S2}m>+G}LIxWOT$ zKso~m;jt!)LN1Ye)s~kLRiBA7MB+Q3>&3YRc)r{}b4us#sG}gn+-1qdBk@PXWWh zb42>Hh9-#~n$R4w`v9@ukCWmAOW8K(E0&LRKe-yhitX%hHd=b{eRgkWNAI~6JAvQZ zxo_3~gSsb!1ao1FX?xjipXk~g=pCy;)zXEyDZ6Fex9JkAQYJ3gHH54Vr-2yV*q6Wk z_M7+!k>Qth_aK1XR=p=uEA=3rBrI_~Rsh`T8cuKGW^XzgonTat^G8@|WS%NhF zKPq^9M)cU&P?}&m@tPsH(@wl*+lkrks#f8~Z9>O^eUus!zN9A*D894E`e`A|ttz?Y*~iKkANIZwM#^7dn|x)%L8=TvPjLD@*H(zgVixh10aY=B3u)>UPl; ztRW1X+1I7ZnA~uUCH3kSC-aqB{-R06UYjanV?#|U_KqIt8aX<$>0a=4Spj4EHRs$Q zfueGy6rUdZr{ySUV`iz#+yK~2L-R-e+{KLfJvP^APfK^rRaY6IYfWagboQyHw%>BSgY4oAp4j&yE>R@Q8HakrCaNSndd2WS=J#;r zz@NM3fXc))jb*Ih@B;g@sc;Y^$!k1%+9G)p#R$}z`N9TV{F-Y;;tm`_QY{cDQ~_Pi zyeJzW@5MCgAhM73SsKXGI7sKs0ss#LJ-awf)#m9FG98IKb}Lmr`uImt&&x6|qZ^EO zVWhQnF^_gNiRrBFyqnHzQ~2P9#_jL~R3iV9V89_yMJwKuLm#yF$wxH}?#g~I9UBWD zbdE_@NqE@G?q=I?s2GQ$l#Yvo+0s(PM-$GM9?`n!n|&rvFoilY5|OY z9sio>lPgG|tRJNucre3_^U>7oa3k`kPS?3*-Qte0u1*JTlk1HP`q1L@kf*BhOpPZ? zM_(K84Ah*6*QY_?eCIU#>4?HIA#!AL?)(>$kXN$lRQIb!tEX#4-Vv$nm~5^D5k(;B zQ@F`DNF{RP^jRpVdA`&%SgjrOD(-xk~&~ zSBWeF+0Yi8O$ClWR6Y}}Q>6pxZvZIz%J(bL+vk)hXXMsOe2m1M&rYQn?BGh{jgk_H zWbR>WNs)3MzH<&nA$9NIdD%-PnXwt3 z%6aWEN(>O}1f#@L+3XQqvM9~R(y4=-c0x=<%sEe>tA`e@MxVb_MWGZLUEb?y2CW|b z4pjMSPT(x>g*3~CHn;YHq)8Hbr1*T1R!1)aNvvHsxJK*u)+oFc;fdWMgn&nP-n_(){a=+oaBgF4~8v0ZXWe$G1$bTyE)M%+M!tx+(_*Y5iqu9lXK2|r7hn&kJRc8oi+7otk4*l&{$A= zZG4)WFxy+>9D6E|jmi7bluQ{(w05-uMgjBT7;8Gu)2yp1?KxbSaP<)3A`c^&3|Rc| zz6<<6ziS&BpiSG*BRfnwAN%oT6YHsq+KuHt*63{qGw&rI_k_h^V8go zdzof08AeMd?3Woib2HwFSQcqd*~hD)?ka)4QzBG?zXfQ5)V0QLSbd^sfRdA#tMzG2JTK>>86 zqbU~lRavQxM5w5BqcngLmc?yA1nB2n99TONw`?u?^Uqd_CQ*utrdM^@wknZCV>g=W>-|<- z-leF-Ke-e(s-Yu8%Gae~wGpp3GA{=|q2{{LF|Cx>HI)tv%sVDlC2M#0SI#3{ zODp0OpjP(jiD(pk1v;u9CNLH7rdWLpx91!Bg zk(GL!r7r&E+VQjmkCC?!mI!9P%t71IZ%h2wY2$ah3pq19bO65mhd|)&JN4I}f*h4% z*uSl4A?X;(Jgo)R)p=1IPyQ($D2f;viD+QnsK8@JoN-1L{!Wl+IDpreR5h+|1kifk zx7ddkvGG-MrGpBbf}-keDHRkmL9nymWIrbgBUu~{(g~WB>`xWRzgSH8F`1CKE@z1k zPX}lyAAeNE{6L+vYlXWNUP#U$he880p2Fmw9!ZJ6G7=8=j8%&|jhqzeGtWOsMVu-6 zIhjY|oltuznW7&D8D^~C`tY~dXyT4hZP|8PwX5b*qqnrUa#?pb{mMu$bvJk5z0iv{ z#+taiPjM4Ho-D#{r_35pj~*bmt{ml4kUoP+>)hG}lb@bHQ z0DsMPIg0N0Y1tx%8bDkEiTD{W*$}c^Alykj!Ewg>%#Hm(UWq!(>=V={tqo1T+J!i# zuo^h1)8p2-b_jVhbrP_=g=OTbCLz|8i^X&t0TU$O*n)8MGvrG+=>C4#IXw8=$FI5{ ze?R>FaKOY^L*cog#=14$(NtuJz<6#27BhOL&DMrVDvxL$@ICC$i+M*fU&WlJVI>Rl z5pPGw^5~&X<3~v2ajezD;tF+j)`@uO{74WVJ!iYt2ng(+NzEIRP`_zT?7~=c%@Se} z6-}?&2#*_)U#jrJ+TL@D;6}(Y@Noi964go2TZgL&5Fd;q1rIzTyWG9#XiZMDyy6ym zf(QdamS~5+C3|oh{ZvEZ?Nq$2$@K|o6cw@aR!FlpH|^Dfp5pLr#bn> zO)RJRs4R;!NMmEBP2}R?S2V8D<9G-fY;Dy-vS5|R9r5(8Ja!JnkQ7}kH;3F{x+pcA zxPhh9qrq;+@eyg<{BIXOLM>AW2?-E>tlmjiSdHoiwaB{9#P`^itp)dRZJK4!x9xRl zzlg(|lPtqLAfjiIhYdrAb##u$LVPXd2D80UbQaEvfyWXvY+{tCozobyA!H2#S|^<> zNMW$U;wt3?CT;IjT+xMF)h3}^Aon>TAH%T;$tGjK;33dgqLbyae1^*kbv0MvlqQTR@Y? z{BezJRPT}oBEaTjSFuJ-jOSg9Fl;A==IX@TOBgzxYK~y0t;kQ0qE^PjK z`1P9e$&-_J`1y1!kVA+>{ATar&M+Liu#0Emo;mt3pN37kVE5&B0Q(EOe3lE!Y+z8{ ztJ!v3V52ZnQJ8Gbp+^n8nEgUOG{Qu($vh$#o!>ZTXo9P2J|g@D7F0u`Tr4ie_AbA{ z>?A+jddR!t0ei+2D*?6=VhvUvYSACi+3L=SR z$iMyk?5E{&E@4WX2RO-sLx#gj{W4Ft*Pcvhr~UH;8iV3|NzJ}_^Rm6By+MM9rglm3 zEQEhMsVyczfSVBb6$EEdetQ5ytO92EI!p$|`~=49bTXYRZ@LveohbwS7uW2DLrVH& zBglZl!7T!3C0|T$C~p%2F-XLso~N_R((kuf=WK+Hsl(!|Xh>vM0xm36QWn_0ihGgv zgMu;yaV_Fsz%V=3nEL5WwPCRmXc}+n$l&SG!_XFet@h6m7!KN%eMuYDoOOMZkdOrz zm^?BXb{-5e)eS=nKBS}lMD!aJu3}n_o(;T|%BrJL57SzcQC{Hyfs)CJtk4qR%AVNN z;Ph-0pquha>Im!&nUyy;lNv~HXoH6h`*qBQ(IBPsnUBRnbf@&6^pbM=Q%Kp{GWo3bsRo+ZB zgr@dO=Tvn@UZ_hFvh!_LRry_N;FV$QBz6smBn+7#uyfo1>(+YNL|j%k_{7_y+xIW# z;uH}Cm3R5lp1&MXJ~GB7QQYWWc9DnsQlMd{MPN6wAfyMjERf7l(_#dD%fkoXbe-ZY zuXUV1|U?>|;*%h`WLdhf}vC&m0bDD9*?rJdPg_F4JIenNR2Et7-q; ztT+wy0L%d};Rc+CeSSF~Al9Img=u@sw_QGd3(hi};I~i*4&Uy4c(}83e`j}h_ig9- z>$gX*UcGqr_W6r9Zy){Vqo>av{qXEa)ytD8{*I{7+vg7#*UA4xi|&RJIW0yjOM2lcXXOvufaiQx z9?^0f*)ehTuQjD_#FSZp%~)4cgY#Xy&Pju7f@YNzuR#Y$npVc8kx!}dGj(QAsipGG zBE4Bt9HPxZI1s$>7NPVM`7l|eif#GEoD3?$W7EBIP`OudI^bXMHz<<2tlj=v-09)| zgp;NTxZ@{kOd5-;FC3hsDr5$qb2z+0CIopxQQ8z`ZcP)cu%m zEFD~{p&OCWf$yb{Tyzbr<+&;!h^LC+MXz)2sBoZhf}%&v8sCn@`BL@fl&+Nm{60J{ zu6u;G5_SdrJI+5WC6Z(Ce>;c6WpQya%|8Yrcm44)2&&z$23%^kx`J0vex1F_FOELU zJ6Rj$JI7nwC*6aO-45AQ0hT*Q(XFiA?HrG`|NXyC?smUI;OeAJc{vt~6>KPja3!ts zpNisLNiB?K6HfRPp`+k}qS?x0&U84XP~4=%x3na6*l`q_Ai7l0>@jwZw&X;j;sf^v zk18LqN=!2B9Pn5Da9K(2iwM6F(XO@D8_dbW#7%21~u)}lqrXGQR`**%5xHlwl7ZZt(&}Gtj4;2eI*+$@= z{xl7O1V2oQz)ehaGBeZ)iF1WKz{Lu2lB=*KrWhG&yG#5H490h@Cipxk_A%`v`jCE5 z%5E{mh$ZptYyurW$zzBP<)s}VH!D*;a7RJI?YM>;+7yGYuQ# zim=Z+I%lpQVGejZkpELGyNjRO+v$0#Sl(se)y-_@jrXq}ES-98+%_Zs5!TObOMyn= zlJ1!8x8fEZWm^Y>?ND9u4lw8df)^cVPot#^PPewmrz25x$YtPahobMmgNYydjk1*j zjc{f`5~*CMJY- zXGbsYSkJM1cOs~sHn3iLmq(>sSV`S%vt049C#9Z-o5o!r@xnH|S= zWN092%FZp6({+;SrM>r>46OnsXfYeYh>}s0<$Y{@R7M-wEaP%KdQ!&~W37d9KaZA| z{XpC(ONmVrQzfcu%RL>h;GS{HX|X<4~)k5j2-be4N!16skwwORIS3DT*&*iCPshx20DfcKGmf$4NJ&eQR5k zp>k+*4B6X%W5Zn@-pzoiD(+xiMK4jw6v&ms>lC*irl}WeP?v+WKtGu;f9q7r- zLPTjb7%VNBuis7PaubczDq@5{yL7mYBrqcA#yVoWCW({YM=tQ;FVL0S2+MAaGEuTC z$Ji3#%%`K9ob9$inhZSm=qhhjrw8OkG4yzR)g}5;LeaA?jbfzwvPKZQFRG^1G|ivv z>LhBCkaT(iT8TdtRDm8=&rEtGSdjU`RbtkonX&i`?-ZGuT4%C|r6iS`leT)Ahlj>B znUmGkXl}!Ev?bw1F|c(gV$~@jZeN{yn_kMODnv+t)vR0R9|-oE0Q##0d;&`)o#*L+5|0&P!Ywb3lLhwRNuzpCmZDgIm~Nwpf4!5QJIWmj%r4CAy2ber|+xnUY`?V5@Dc#)T4o>%3EX_N`T`zQ-xa+EMhbAAY;3}*7$nn4xV)rN#f5^!q)K@vb zvW9NCj$BtJ!+E;My|)qXMA01VSB9sr>0d#IX=g4U7w8CaldYNzWtOi+3V{e@vv zWje9r;3s6HOp(q=DU8Dqy22V_JiVNZ$0VuHW6PyZKv+M~JMKISW^iwKed+aVZ}CXS z*FOIM^Oz*QG_S!-SOJby6cJNFA~qviAPZKZylR(*H{RM(m`4j^~Hf>{#g((saq{eJ1*jmBCAP-+UKj?HCBh z)LNJru_{~o26nWQN;vHg*P>sPy4}|`Ug_@kwv((jWtr$zxD=&XQg#%( zCx^fD;2H;mF1mB*qL`_nw$nX(pgR9f*5*&!@fL!0g(_;-Z>Ge`Me`Z%TuNwC&hin` z2N6gX1wfv`mmcElz1X9`zb+=HD@Ly6YxL<6#UY|1{!)zI6sTDHj>SKH{u@|1{z@PH zlf-JcWs6V^4{Hm(q!VPdaRi8OCvRx6L_rG#!7mO<_fK>8t<`L9B|2XwM(~yC`sD>_+LF zj~{hA-BVB<>g{Bd-v;NG^YWV{$5^7aca#ZeQ|JVm91Sbx!YXw zpY@Vv`En4(Eyn#Wt_Se>NF*v^Un&AsMQC9TC+Y{pr6hHY(D_KEfcWVKU;DV%`H!$o zq#bMoe#jn@)Jl98u13Dz4{4Qha-onL+BOc*C?>O&vq^w$aJ-yd`hm3S-=42ajn3Y~ zVZXAUHjOVn!W|${pA>&>9WuOAYSJ52bfaO;s|6w-qi_2BB<^ds_rH*h=|MN0BzWWS zch2-g9&M$r$##~{5IR|AedrK7$eE27VM7BFZUHa*l7uLBB<8QWX@B}Qip|*jPsZl( z%LMsMnX^7R+-c-tSr1Y>V3qH&J>)vJ?f;q2l{MXuUFpANoTMMuFoY&P>ba3+_uZ*) zB1U?#?dCFV(x2;VG}EZJ`Py-)?c*V=Xwep&6Upw`a=j90p=%dhZlu&?}`9YlLMQjAn=|ce*JTvrXUiDM>q}b)L)s zPkMVbP>Fd%NEpu>HaWB^RkGV!ph#s9f?$3fa>*cjpfN82-qvi-`$Umr8|`!!boTX` z6Kd_5DWv2Nu#V+5T>GtvTB{DU1NO#TMDSxqB!<1}R*OzIHbILX{%cPvo>*V%3b!xV ziK|5;K$MD7HqCQUV#UFqcQnwz_#H>Vsz9>pwOz$Uz_z5MY-7p(&bJcgFkvEWx-8IEf zQ;i%qbVuq^itDzG3=LalJ9B#r-aGvDmV3NrLds=6PbR~^5QBgnj-Kt>UR*qmy1SzsFDD(q|97_uVH$X)R^`ew&%A&_X z$ResjV*!}pGHdVdfP6t5q|Zo%qGnWM&AN#XwGYDrhwHqvyBLkTT!Dr~;&s@)tqo`7 zTjziqr?E%yYbxR$xtT+{(y$yZFA+^a8xzJYCz#qh(lUczNDsoIYhQ3r^QVUy0&SdG zkhF83*@~fK&?(Sa#z`~O5Z6P)<+bNmvIqH%F0z7y(6CS{NRE@lrT3bu za3~_Fcz*tY$Vg=yS2()HLa85uk1DC18`9_0PflFfmn<(OeTWA#$|uo+Q|WFq3%YK* z@ZTWOTbc9jY{>t%mRKCbN>Z^~94iHtV+bwM_Xpp#2vuS>IU|J5F}qAtBuSy5k_vML z-SearxT8=MN`taS3V}XbOs8*%^=W?hEI(f|#r(|1tS*?>9=peNnAIvILr62*sJ{;}qHSYnnsFqF+hJCmGz(65X?v0@DIwe-@$ zvgfL!yTnA2VBgoTllf>l#G?9Jb~5}(OxG}FdYvgfnL1a=VZv-S z(}OgFDo8ja1OE^Z#i+~jK)0ia z`Vzw=&FCSr6Yt&0>}B2=GM6L6@~!Jy!`wqYdxb8-K;p)9kPI=} zh0Xz&9yUSP5clcdt%FeScEf~h@U?Iddu{NGG(ABKzY5~&WCnBtE(@Vq-_1wkG4Y80 zywJj(w4u2wicB(jGKwqd% z8$?kq7sM~>zj8Q3v;dyuoNyz}mgg?DBu|7?f(UZzXv@ck*`r*7)+&|iUqD}KsNC$S z_6-M7LmdzeO9xJ`3L?pRVwsGzBsnyZSrBuQyn_&$Jcx*mP{m>(Wt>NQbez@#gLt)w z8Eu66CFx$|hTNuWZ(y?BfH>~$zY|jyy^8EUHICdrOtLAu7s}a*dmzabG?yrYTanDM z7R{j(Y&>sGce2h`TkA7HKpxaCxM&-T-Aw#5s;sXorV7seNX)|9{Z!v6mgrC;oViA+ z|9N_1=c-b;<{i4O9AbC3IePf4D8uW!^!LlB8ITnA_A5G9<5A(hZCj#laU+wQE75O^ zji%U0)X34(>)2nrWSgS;lW4DSkySZ9V{Yh$v4nPg4J3s-qp@+JOGy}_@e@F$%GjSJ zrU>S^sBXd^O3E9qMdgtSv$wr%tX|>`m2_M__Err(?fDl+2;bmrtcaZ(rh&vxY{S zdL$aJv{_Op#j36S@O^BZQ?gZRV^YaRkZEx}%pUCQ(B)i)i+gWJ(oblTFDmX0%=dS8 z)S&YNF3HDitNVVp|1hKLi!z8ZwoPg)%NX3N<;he2VeXTA;Adaap!}WhU+Ddv-oxzh z)R9&gcOy@I)iV$%uJ~_9aP$b-Foc!?Ili7*kLeOB%{*fMv zNFLeiCdsO4I$5WGO-221N-GKp?hKp{UeI5TJhPZNQMpvz(MeZWoE?!pFi==fsng;E z2??ab-RT7p>pZOc4$BGRZNw?nc39#M6^H{8aRM-o1*S7!{3!V+E!=YxCH>f?7t#Y# zV4pQ*!AxA)32=|7T51(UqCVK_DqfjqoYtl6#DU6QNme0@I^Mxgo$x|&V;4U>e$`_* zADEPPdOKNQDUiNlc?s79z#-F@KTN}+_#ynh$-W8m{o-|YFY9Pfe>Qn}Si%tA z-6R8c8|tPPTsNP@NZRZm!k}j-;4`judV0v`vnVGiv+6zW^{oT|kxDZ(Wgn4N3DWmA zrP&R0IH0j=a04OBGkR+v{8X!6zOi98K%jsbAfjVH-*vF9s7Y;O)dX!t&QR(C%ODS{ z#S5JuDJT=xu>y}@zh={U!wiHWwKHE*4CxI;F>)(Anw}wRX8tmEzdGj8X<1BH%ba~b zlUL`H50vH1XMdV;v@QFb^s-~?crS}RJLzVB2k0sKpwWd8sX__Y6M3TF;zM0t7ydv> zNN8hpEU}Ye1@j3LfiQ7_b)!`W(D(|DM6=I?r!?Tyeqz-`52;QTH%B*OfW;#TDIbD z^Sy6V5Hp1VBpigzNS_KMvmwfnhE-&uiF4`8D0A85InmnCEmiZm6&_db{FGx#s1OsC zm*g7LlV5xc>5&nkHQYswy|AQ(I4RC2BT!Z$_!R6d0I{8FD`+IP5ze~?vT-6Q;v~ov zB$e^^mEF4LDyYz?KSddQoh!CBg@!fX>pqDUN69h9XP%M_r4Ji+x!rQzC%Zth0o@Iv z-}G*>GOMara8?pvWo2?M4fUY(>5%spo9%jMP7yFOHw(0`J) zVd{6F{3fG0Ef!-*8}4fp<|e(hn!3c_2MRzM+;(EoNh)NH=1VKYk+0XNF*vVRr_06Y zO!uHpG~-`LYBknOmn{WQ$cF-$Z|+JY@pG{uUqfVCp}I`7&hPQFB_L^ejJ^!y=k@4( zw3uW#Ddh6x21_B6vKOY%CwMO?))L@c>o+vx+gWWDwwultcPmW0&5wlm3eA<@?e7{mPlIQisfV62cu-`aN#ulTo6S?F zZk3?~AvrpxX z$RzE=TEMj$2(c>V=*A!Xb0Dr98o-wr6Zq4LIf1k>4eA6=PN9-@Rl3k##IM-|SCzq- zH+Obs(l{jwfQq|3syGZcoJ4GepU8L|{)w?(Y{P_fr6P1ymUd&6((J9OPaR>UUR}FH z3#eCDf+I>cHZi@2)c2u@9rTS8+Ypctg0-G;c{w4)h;tVCanWafJ{g-g$} zF_}TfSx_sh;=;p>U#*llR87cG{z$|1zhmtUy%5FOe@=3O>vOKdTqiGt_||-K=VvmT z<%^%lP}}1lzq0xCQl5!5wFqp3l%s=2zuETKeA2R?#QLiNcD;;>4=oC>!3-Du11gn? zMRbVB>=_IOG#JRG1+It~HfYM-$a`+s zPSG0yUyQiT+RATvHjXvggf70xVYvPuy*rqF@fIbwZwBqrlUI*~_K{(#c2)nSS;<50Wm=8ski1XY;Hf)ZgO0`KHD(xJxMuDm4G41^UABPh($X9m#Id?yO zF79%aG%d{O1(6_^U2ioDQ@mWw$gf$#=tz<(2~_vay@&T6a(3MZgNJlN-2$V==({&7 zvWo(gy1at-$o1%kbD(?(;#faf2ZI9Ur%X>zi}B4drPmIFj%xrv-wj$sb@`$9g0j6%@8tZ&7A9SWU zaNo9;U*FntQe_gg?x=^=&G)aqiA4Tb>4yvQ2hQyqdiSXd@ztyUuZr<%nx_sS;hvGl z(Qkwwz!Fwa5(;0QQMU$25?=(4iQ7ixVCADQbmYt-2Qz3V(yZOj$#{ahmfM~qU=k?w z?9fD;k&`b^&TqPXO0q0HH9_H()aF&0Q`7{dDFe{} z2g9|nmM1Lz^$bc75TQByeP~~`pV!4=iu|h88IpMjYh;Whf=UdHH&13uWMyt#a&1$N zi*hxZk{Jcb(WS5@OaIC>(_$>83*`A+aD*yIMcAu9|0sQxsnIqiKRH9}=QG;a1hhk# zxGmOv0eyp33)xMfyy6qpGs^IOGNMenh=p-=;p7BzI6*qVOHd1<<5+z76GF34Aiw0Q zbp0#)*(29!WSSqH=aQ}CstA89iqloel?mxy$cX#dv*J2mkPU>dS4xsmQg#qlyTyj6 zk&=`C0t77VDWu1ula3+td)^N+S(8$HbEY5!(@=iFrx~fIrD2;g`kuBZ-BldKMLSi~ zdB{SSg&YM+b?Q0@VCRzeaKXOjv*K|vJD*O@NDo55IaE0?P%zoykaAah*xj_@ku0^J zn)`V5M0+UYg-$%b=#Q>c_TvYgtj)7Yfgq8s%RrN|gaO>oKB)&mD^S#!IBUQ4{{T=+ z0|W{H00;;GI)N-t8-W7{^8@b~@-#G9?8Sd#VX7dcKYaqMmPnzq-Zx7)MR zJd{LBY$Q@8sn~9t`|Z!n03ZPnpeV=b?(LS{jVW+27z_r3!C(OFJ!03rb-=pqVf%*F z!(cW~7q<;opETG!O;@A6J^bFDM9U_7AEZlf5rp1<%$8oTz(n?F@3Z$_%#pAY@%Oji z*smXs#fND86MewH`~%zDWB$!5N_+;1{v>5-#Ku1J#tR=`qxXJ%9S5n;rv8<;UZm`5 z9Zu393X{)1+k3R-^VuWDz6!>%7vGN9-~2J_wL9$lFWFTTv!DKJ?Z>wW(PHP(Vi8>j z;Y<*fFfS$wg3wP8gg-4&N9Ac{)tGxJyABo$Hi^R23qt0>`*kp#`Kf^dxnVHVmjn}r$w3J#OBxtr}6f&Mxgq`|H8b`Mp ztVk;YFTMC(hFJVqE!MLjEW)PVEKkB8EIJ@-M1mJ$4uth|ZO>cGx4S}~A5cpFD)8Z8?F5f^$doPy}3!3284Ji5jn z5&M7+GEe-N#Of;cmi~1Vzf*b!N8wGU&<;0W_}G(4#?!gaIGYrZG+6rVmnif_+Uv7d zuh};t`|_JtB1LWAdV{r~n&IM>twT&o*i{fGDGS0R^}>n7lB~y2fn`GF@-mZpLn&4^ z#KMDab{%^wg2YZT@m5p~COXY{4U7qswnoj=OZ|GIsr-5k^}rH%^MDp(lR4m)1g*R{ z@lT^g6xTt38^FhO9YZ1UY%Cz)tv65JHyQlX%abRVryYf_`~?ZdXWMF>z0agxq|SQ6 z>awL+{b7Us`WcELLZr(Td=**|xsgQBG{D6behyG4CK)i-r93mG`Ei0M*@)Fhh1Z(d zx5Q6V&}IpI%QN7!vxp|%!rxKTMr_9qTkB+}se>Yq&!c$hr8^4fa&NkK3y%qnQdNYW z&qvFEt(G?2Mh*q{Z?0^UbquO&a{Fxzs`*A{d*}jtwZ2loN{~$gWaSP(^$eg{9IaQc zZkOX|A?pRGnuYoR3K$<%Kmco-V4x-96qIT@PjYM*JAd`UohI8k_s96>r5EF$XRDb1 zy2U?#U5EV7g1)V1_~(_s!av_kQvCCE^dA4b@FzQOjB>Bmi-lgSEO!k-jpC@8@wK-d=g2FXa20aF``t`~JIEDd=mJ ztwgtYpj%J85Jv?_r5?_y!4UvzKq(w|p?i3*Ver?gQv2fd>u>-1>YJ}?O$9L~Hktg? zWZ!`{KmY!#uktsZj{kyb=Yips_+bLZ!h64=m%5tA@LA;_>W>duZOC?8uF_j+ups$< zI9mkCoQcfEZcYsP!3Uuh7`sQqPBX`t;CFTl0E7>-v{u0chMpp>vv8VYLR`I0ucJ~2 z!>5Lc8JIrgEf^%vVZ2eRN+~M9m)v4d03byLIA26wssT`n3IYx8-)bO~qJn(k-x$R} ziVE;-v6_2dua{)C5P+JZLY|>rjUn zi4=tlKS@-v9LW@=5VTJ9LLfyW0_-P)2uR5xVD^?vBteK29VCta7EEEjmxG8DMKmTc z%uAnpvA{RdQ*yw`+@HMr+RJ+FC{Hnfz7K<+*SzsZdP)vRhVysPI`0R0ih?dlxD}R{ zDN_toWP@v{C?$sy=C{Hzk&=T5Q(Xa(#=FYYQy9mi_}fUx(x{lGnaBn3LYg9?Fbn<) zTl!TR#Uw+cLYh(|FVTT5&@h?F;Bgp4)1Ope2ue{m|M7vb=bKl6rd|)d_rc5qB<*$L z$IoCko3_*FtLWN~Poek0{3w7q?9De<^;)o+!(dZuum>ls)n&i^rVXj|Y=teF-XAV6 z!__EadI}m~?W5ZC7KyKL3fOSAEfB!@c9YPjFA69h5aHNM>dJPA+a0cw71pK!x931t%%Vc+N)?~F>BTJJ`1395> zEAz1pklK_CVB57up=vsHKwd~TK@FXig>Qm%_W9?O_5jmrjRyXH5F}rFU)Q-sL0oWh zira&>#z=hbMAfQsTx_qY0v$LC3z4l{mm(>%$txt?W1Z$Ud*iNz-^07dT61z;jXT8pvWnv_X{1U{>`k{~{5B-9?le5n?wtzVE!RWNmj zNl{D{_Cv|KJVJhw%Dku#rG@f&6p34wR+FwOu0QV79dSlc*5AF>)w9<5$8>mYN>X0b zFV9;414X@U-MKJ6tFe{h(-*y<_S-*$;O_p1k^)6vw#Dx^zxJAg4;D%Y*Jfg9f`AZB zVbeDe0j8R&?*S34H>O7Y6z7i6sp&nQyf#gk_uMvVW~*SpA~*%8S;O@L z3X{JJdZe9u^9&TZOt!&X`u_FlIbgzBHWx=(bL<&tr_h{C_{`OgLep;1RdhUcr90q5 zRnj0|VoDwQD(mW?Q(6g7L0^y?Sm|HP2fk88$h3r!j?2jjS$TxL70Z7u8pvZVn;F%gG%kar$jlbmM zI2E@S;84C)Mrcg8q)w+iajGU>G}#-Tyr(8#!hd@*RkP5J3*-skf?;Fa17h?o)v0d+ z@=lX2{Lt3na-VA(AZF?O7&1IYXn5J(wQ-b_9W!eO&hcF=-xR6JCBiXSrJ`M%N>5pm zG&UMm_;#`$!>Cw?D)t*CnTxc1_{{x_uXLc`pe8W{nlFi%ZPwZ=wpZKUTJ2=D2+}%! zIJV*LS;Tb<^v8wH!*QabTrN?RaruH&rR@IzaM5{lOr4zn>cTJ5Bm8F%etsy!yO&{Q z_lXyX1(L1wb~U`6E%ELxUG#nwUc|XaL>K7omGPld3Gd$0Lhr}pC85~B-`-vD?oBuH zcT2ode(zh(xcm~<%6Ky!!wVoqbJXcn6XXO?g4pL>X^OZnzbo#L_`%ppw0;imE@-30Ni4xZ%DfGB{$IN>&C_slQ|M2bmV2Q)AX2|;o>)bbWIXz)rU_K0UCd) z_8zeFd6Zk^tlJs&I;$l`|MYtBt`Q_$XJs5FYt*}GGH=;rtEJX(triB8P5dN*uQ*-z zLQ$}i-8qxUP0BR-ezhtZLf%q*&s?O>UP6mL0n<1KS`~o4EdpUByd{Cw;a_M$Z)rGf z@N)xe3NYQCXb%3fH?R>cEjlDh_ZK1shOdTad)Nj(EDbggA6pu!Ja`8S5jN*Y#}%`A zQKw+7I$%XF%}?FzQ&Jo)_$Q4TZ!+;$Df6c@pW&kZi1ojH34+HwHj5%2tapuSa@+dq7P91wzqc;zIqVOO8$>v1du^qqU=nsp$mc zoZ!wbENKqYq7Hv~Lahm}57K#%w7EKz>X;J+ud=u&tXHA19J-6-N_OYvFJ<{lfEgay zitpydci(6nXlzA6FQcHRQjkar5+?<{oPypK6rdLX)i#9!X3~<3p!v^#7DYb*C^<^-nm|^IKKd%Gm2pCP7My%5Th(=o_Vd;Az zWp5%Kv5)x)43SRek_ard12lyopdAor48o1xq3es6-Qgson#9b{2%@cVn~N_nnY041 zXHFbn5~~D@>2+hfNkmHfAwH5J{@SsjFvk)a3G%T-pY)U4YRNHDCj)t&vt6#-h;!H? zI5f17M`09&h5X(!r^u1t6ZvI}X*fg$irxs&7BL8lvLG#=We~1Y6ryGETjEcma0=ga zN7DzJOT5BBnuSnQ^0;p24$T7)b-gWz<;+ze6#Q@QHH43HovK#H1JmlrnPYGbO8tkq zLGO=ZlLR@gD#l1|G%%Z22SC<-g4XN>4aS+%3J}3?s-$58AKV*;15LCz4NJ-o0bH!R z1i@h;e^>H$?Rlj(wpF^Th$)x*IS)a51WuN;{#HUB|R>3FhkVQUJ7?U8iu7`(;z{gY5~}A6BIR-8gOno}6r;09T>B zNec6fAY=>QTXDZyG7qj&+@`g-Wzhn+&B1gfWCQw|G^$pj-Me!=OxswLvdh5ufqyCE zE*kxmj8odD(5pnSi^GVPk=EQ6CM)aeJjV`vL`Aq`Z3UTszvD!p z@Qp+7iPCo24kAv7ODcC8up1S3hel90SKQui|vT;GJtK2Q5t4%tlbs#;T77bK*&P*ESZV;N=|u#1W1 zfaHQ3U}CM{O{O$?vREO2#HNyJ3*X_0dn)3d9dXZ!xL1z2uOjZ-5%;Z#`zB)Dh9plD z2s>4bPqUDbMgoi(Js33LpAkC20Q(4$3n)3|YLM>7!KIT$QZf@nl53W5!i%V_mXz09 znx1hawfB3Is^*nb#ge=^$epV>P0*DwEzI<=kK?py+G_|HWQYr>FM-q@)b-jAvPaZZ zL6E)sakBDYjvJ&4|754OOC4ob*bvBk__6Z@(*g5t?eQLhKdISOtFq0ip1{8r1^XaZ zXA}ya!=cBOznkH$s@BXv_ykhNVa$jWY@H05G0{RG)MXd7=PCA{|9p=BzNCNuLjS&| zf4`-F&*vnpB27~{O&1R{qwxXzq$heBGDPxlb z4P~Ik0HpKa#-G*$USklr4sN1U#F8p|5zsPG?demO9jNVt6YkX5qi;>NPW-=MC^IO0 zvu&02M?FxP)+6-relcD;N5)9(^gl6-1hB}0<$8H8=y?&$f)sbJX9Vid-pv-!O4V>r zSpF-wGIR-(Xw+I~X_m>js8ritiNYK`1Gm=Vg!MWl_K(s%$Ty1NX}JGPjQNGJTkKAg zg_erO3)O^@E=S=o`ts$s-+cG_+1IZ#^XQ9bukbjG=Vy$Y7k~BMd#@&Ouu5r=7am+) zOGuQNU!E4ZtTD=JzTd%o)Ae@x5%ybpZ}wYhtVDrm85m+lJs42}B2GjLt1w0_+5sy; zOoL7OLix&w79FWcdA`ujodwPEh*tdbrUZj6zhak`BcOOhjOe7`4>!za5D9@xxi}+w zic?n5!PzJw+K#Zubq8Fb9WlojM^W`3f4lh6YyI-@O@j)pJ?x^cr`&1|5GV3h+}09) z1QD*8h7leh@l(MEuS;WKRk7|Cq z>gP+WpGfuJadwI7;iWTMt~`#n8e?pWzW)cun2#Q0jAg22~%MvhkGeWn3sB8!n6Fvi?H(=!uOFUjA8)@ZCw78Iz!DKT| zT?!D-W10})+X*SvU3|B#6d@R2o#J5dZWy}5so>1ey~t|o-PaLGGJOEvIG+q zr`&arPUe}Em{{KzAcWv|(HGSCjZ&E*;*3ph?vWquO&gf#r@X7X`20X4JfUyR_vn{3p?kvR= z^ao1YS4u)%e6VfVWUk;RH?4RyTs2yQP)N^RDBIM+1*0LCVzW;u=V<-Qbc)Y$AlvyI z4E*CACHVYpK+4$ei3!H<449&z_&YI6ME#7hnGMKfikU^*7)C*LSa8fB)Zq`}+_7 z2!Pj!EpcJ@Mw4ML?MwX14!5D)!|!SLs3D+ zqr^8U)lt^zn-KPbCWER1Dh@K4d+{k?dzRL*OB|b!y~>dNYLLiQfFvTGJYl`US1hq! zr|+NRX1>Rd*`UE-)=AVrmO&9?U*_pSp6-w`^<<_#X4)@i8gMfDkY&GL^ce9Y)exsu zHHlIMp8bK2KnZ=z@bq1HtO&GUnb?Y#T_I+~YG5-mwP8!`SyTJg)PXix&0e$arN|zu z^I^@b5KqlVH36j@Y!Y9F(2sX6Lvu*Nz5^wspg2YM)MwXHM}}OXqf3%w&s2J7Kh%bL zH)Imfig1P&9f~bRe$3vXYe14z?#gPmui|KVD)83Li1lJ?;v>WZgqFOL2eiADO73+A zht9!pL)nN8WgR5Hd(?9bH=A)_haeolgvyoq2!gxRS9?9PXIDj%RlA(<`>(wION@1* z2y17O2XIn1E*r(fnZ_SEis@hfDMvIuWaQ^#{rIf)KVIvXPU{E^gT2d>H+$02vh6r0 zMjD~#9<-Imc&jNx?-kd2N5nTbY6Q&ouJ zyv_mSFid(y7a4qtgOgP)0@aqye^dZEe)k79omUyRS%c!)EX3@4D5>u(DSyk5(U{5m za-Aotf~*g%vdatoVOt5!##goD%a2CamB~2vbwC}UO6VJ-s_02HJrc#rz2-I^O?Zt6 zOf~ZEHjawQY%z=y;@#ct$Je8-GQQ$Un1dwh`nDSGnzCE15_~-Ack}L3r%nhXjH`Nx zs#m#&_f-wjUe+8gE22upg|6Yldp)bV!i@fAX7o2PY4CYJ3qSVsjEM&!5$@O2}N@+)ZwM(L=O0I{|mwv-qkdn;l9 z{?ifTg}0m-uyQKW16j7F2eDTr2^(v^^d^M#U6W1oai#IhCab^r16Hy8V?j>^vbC8nuBnl^9YL`4}OyI?^+v7p2p3X+KtZhX-QVK^JvtrQFtt8>%P;9m9 z;;~g8+5>m#$}}!;W5prBX?$xtUx}L^dP6X%7p{a^PhGJZEWEOX=7X^=^~M-xCo(-{ zJGn1_dlK-%LnV4`kPw&gF8tc4(WH0JZ|4^A$XZVT&ryoJT7z} z?W)>V3i%=InxJhIOFD|3&v7;hB-JbqRtwt9Ubo05O|&e7jC_7WhZ)~ofwVrg$T#gF zKs<-gFFPAi^~zE88ib|TXEIjbqZ;2FRlvs>CZfe$UKpe~1EeuBUIFu}!MK-ls3frfgYwb)M$t^)dIf5xYRH~Hgxc*wkX9He*&xlE>2 zoUMC1n!m*6W`>~5WSR!=gCvUiA{jwjdj@riCi|($er~cG_;($pfgh%ZwX>JkP*y6& zi%hRT>RV7fHV$SjKb!_$D2`;~L9fJxxlhzg@qSLFrYfcMP1;SgBX;5ki@GLOdwAHT zrC%$3TB6!!w2^tZwPfks(7HX|G0|&c|4yB^)T&a&g^SF0{t6>Cri)v)@?+!{1{X)b zF={)Ifi`;$yaVG0g@2eO0hPr(%j2 z*u*el=>|?CC~N#2spMCSbx|OmaKUJ`iag85)2w%hK~-S_#XNn~5P8Of7d%-%FF~Jo z>#(f?o#Um_S69I#z#HB8#sYv$)=QsjAHH%4iF}I}V(9LxD9VRY-TzZ+J#Vx+k>9Ou zF&&1{Tl-Sa?rve(bb~?wnR&rirg%+RvX0|uMweTA0f^3{qQb6fu zHr0dOv~%;~u+w3Wp{L+t&kepo+Vk^s{0VQT{Oc|LycJ*1&n;6kw@T{Vbm=6jT~?R# zv_4=UJw=Cm)o=6EtroCsd>8U6L_kWbjp0~-QQ-r}Tr=-uwgsCT3${O z9a{h4Q{sJY4|mTS#lxG`r*m^i*>0ryT!F%KYjaj1@k}KV7p%k_yc&&q)%u^JYTfg@ zSMFS^+@Yx4!FH9COsQPEduGbV&5j(Zzq@}Yz1`hk_#dIV@8F}tr@WbMK7r4vPq_JR z&4i71DD&FAr6p|N%)0SE9qTr;!OcQ9+qY)j7V~*A^<4yMy71++_d1HFiG~8G@+K+T zcTal5A)9;eX>RNa!v9q!fJlg}x%q23aIWNT%P zqDnLKd30wx!NIwk@WxquM-nj-JAA`_vrp6alzxLI0VMsI_07uNSrEz_o_lk*^y21g z9iOgCLKXVGO{^{Nni=DCy$T0S z*4jVp93Jc+_05G_e5O`hfFeW3ap7GH-5wz8A07?&2g5@fvWXQLCyC?Z$a;tjDBJJs zcRRzw?vZUPnH5)7Lr>!B1Fjwt*6j=i!``spx2+FUW!XTmtb4FO=pG&(9ef1KhJs~X zJXPzkGw6K`%k~A!THOK2e81Zt*uw6rGVMSzZ7@9O437@Y_|t7zcF0-QJviue`olpX zJYXA^9Z4zc9$~j047#=ksLHc$M>1{D>+IX=Se0MhZYE_%N1Z_jYH5pMtje-(Pw)(+ zV!zYhKeQpM+%x-8VV$TsXyN2E6PkA|J@eqm^Hunmw|*hcDc02*!3 zhau;nTT$2mS&?ZdwY?6i+5=Fw2mMY(QM0maC|GvT=?+0@AK8SXD$A%p0nhq}z^{%~ zX;fy|P%x~uzuy@g9QOOekKouqa%|8U4*Lg(1KTc}svJAw92@Q*4hGPiHdK`vHpqmm zdvpjL-P)O|@~kge);&Dv@Ar@PD~O5U&e3paGY_hAtSdOyMN~&HrdJ#QtXvxiu626{y~86=Kp(@j1Hm&OZ8$hOu#YB{ zdDf9U>voPnW;^?XiZ-pCYkkf(=$O5OqwayNLsey3M~Yg1cnF%d*Q+>cTA9|DOanZ_ zUiavr;yl30w0*&|gZ)nLsC#HbRh3~!Ld<~e{r$nU!K~>J?5T>{EC?KvP3izghx`2wjGfh(2Dpk$quxY= zQ*Zz9s5|IZpA2eT z_*dyR>vLQ^0S*wbaFbXe9M%9Ku?qN;Rw?^E)}mB_i}6JdYK#3)7T4>fL`m< z;Qd<|ZkiYd`Zn~h`RU?7GzismZ}QGOHs(5*rgJ>*{h`d-Xj59@Y**O&1}EtQac$Ku}mXiVQkJwD2kRxpwy5ci((BVy9k+2MkUlwXWBA4Qdqn zGM*gpB^ZK<=gygjHFM5dnJ!3bue=yW&99@W-%iq7AYhUtuf?%IHRz(r$_IA8_ebog z(^=gdv*^7aUoE2R5t|3osUIFI`;a8kBBo=uig*P^%o{^Z*QtNZereH3FC&JY#$y<< zC-1JjiQmGA`*E-c(py=IQ3x>bE`{_un+nNFQP07&AAp%Gf~nua9%VluG6Rk zFrw9Bq3>gm0Q36Cc~HC3j(Xd6RBiP%CGCDepft3%gwK5$9Yw1PBtfbCeO8R$_-QP=cp_JOLH1dS>-lOQQSO0RFwkpT| zkk-vjdXEKdEudG6VD)<%uY6wmL%lQgXZI+Y5^bw{=)(LjE{U5MTi#~dl;CFZ-K6jL zG(|tL`28V$pIrJ%didSM?^D~Ag*yEG(@5Xn|J|j}&DZiK+optjm%iW4H2lP}_wPRN z{vN`2kFvLlRxk;q4iSKPzX6X}JO|P_MN!95n-c=_iRB8cS4jnCxp%e%XOAN@GTW zU#&^Bh+^yrB=9_51y#{Fil;t(=&WwAlj*n%Z3)eRcTg5uq`)7MV+VD+-oj7u5>prr z01($ZN31`@kQlxi!hh}X-q?TvL9&CV9ff;2 zlP|-x&Oa4u?UH7y**9yMPp7!jz+8?#_LpFf*{rA1F5}4eK~{ZJBWy{}wzOzXwD*Vy zB|c*8d%P1S2tmui>~6hk!pkfUruY{JE&RLiZh@aN12q16F$H6hh8@U!9i(&C#vk=| zyN&uGok#qvYq8GWo1xhLQO$mZDv{uhHdu2p@qWy2HWi7Uh%P+jaUAA(7+CD%i};Y zBt|k4V$UbzEX|Br{AIUe+zhn(n~dRvAtCXxzdx)?rru=XoW>Zn`a&YK=aR9Vh2TzT z!wp%f$0(8@jAxD!bnkD!X=i{Jx%NcVsmL}_*A{T0lzmUK`n6P&E$qzUx*Z)jFCdc& zakF*CaE&}Se8w-mbiD={FEykM_84s^W^f{B#jcBpa-4WFWf7;z)k6NcU?V5WV5Y*& zUGtlRvl|NOcB~!1(e$7az6nnk0om_n_fbExD-VVTqd(-@LK9|Gh-i7xM=rdkiEi!k z2w0G6J)uSQv)ksGz!$T)OmMpfQR&D#M_KE@uc3>qyxVNTKw6sIAWJWXUcm4OFE)2hT)O^nV0CI2B%nvGb=}F7vjtpz86cYGq+a^gN;6%KCdI~ zO{#&t_geQA@qH_^QvNMqk7Evhf+Mp|a0w z=tgMba0r`QEUycneTy9s)W;%4B2u)XwA|GEd`f8H!k;uQ?B)fvy0&xb6|D+@o3yE5 zq+g{)|$prp{4&{T*S4;fGl|Dz7 z93$lIAUXBIiBEQSK2tXLj|f7A4;%=dd+&otzeRv5T-DuWxVk#+?l7_VyA)rS^aCt- zP)Myf=F*;phVBph&}g*1>GTw5`SqHs_gaJqkJ&DgO|xy7MF`i8%PoyyauN}H%LY^O zs2^}AF?!)?_NKy6a?!D;SIu?(=~hzQYZQFFP`Fw8hoa&kjf9Re$NV@~Pvi*9W+@*; zrE|@lRiw}!RxR;>YYFH|E-mrEswMDI)sfW&Bv{nKqIRAjr_{o+Q10Tfdfenwf|iqL zVCIPhTG7V=;={IV1v+etZGoNrEmWpgKZRN1^LCbGy#3BSWzPU02 zm4QB57)I+s4}u14$5K&?6M{B&c`K`lO0mO?Iev>-^S6( zn|XZY=@{LC)@G)rX3W;mJ50C>r$Mq>c()@MghFjg8uYPa$nD0APn{SjZ(PBF4Ly-S z;fUduJn}fwFRYqWyP)@91;6-|x5@HpD5=L|d?_!}m?a6vBo-{33N>>QCQy@6$3NHW z+050LisAi1k`yGL=jqZ8!&v$$wxUyLe}zuz@NkV0yku^hkJx^d(u!l6I)VY4+x-NG zT^c=6wl+4ceHzCH;L@DxPPEIK@(D1*w@Vl& zcZzFFwr5nOh)URmBc0Wrl(A9j$$}qqA^-XH_RuKe}>R{eg;nF_wn93ttzMKf156M>1eN~D*W7--BWG95?>Kqw5@93?>y-C|s{I!5*<7BjAo0W9KkL!N+Fb6S%1HoKyi8hzBANmV2 zPJGpZp<5`VZ(ipedD|J<)%gmhzu?T^(KNIpv_-k0_OxWFd*isDz7z$4l;J@2x`7M; zDZ?PvQ_3)K5FJU*ZzFJNdv@`a3ci7MYetF8eE;LtEE2f^WP9d_yc}c`TUOh%MO@Hw zN5071cVvvrekabz%$r%W`D7p@fC)0i;a0pywvqf@A#dWxD*JJ4K_$qspc0kIuw}nj z>TerPY8)$h)U+8!@oSa;vSH(mWy2N=J9Mf4Tq!?o2zi$+A65a_T6D&bPs$G4dZj2%L<87H%rxdM z$9j^+P40g!#DQJIkgvM_(M9|=U9R3z@tN1wcA6OZSzG)pH1yu{0&=JQj%zglAG*@l zf9=f@q|WVZEV}+5;=ne&tu@;GDy+KUx40%lp!TP{7B$O>b_Ix{9VZxkZD;Vp3ummp zn#UjqCR62n6ybP~9q!>sL8&Hfo+G*7f0cWyX(dg~D5nR44xT|w0Y#bu@1Qcmojun74?jTrQvn*-|7#?7xb25qVRNjLz~67CDxOZSN<~^rfuh z1%DX{Y7LVCcgpb3JAXNQy`z6c^ZYeJz}LUN`sQonvzOe46P}r(r}VGMdNRSFcl?_9 zkhsC0q+Ys)#<`itwt3H8+8GfSf`GN3ejG=!pf;liD9Fj7K>H#&C8tkPYX~XGuG5aJ2t!N;b0w{dZZB(wtHDVN%&NGg^xUypJ^FuDh5?7c1QX*&7 z6EIu1BT=v8g4QSdg$NNvW;l9)W9LcNu~%bTy!J5Pn0aXwx7UduKbv7xaGPe%-(1yeFJFCm1|#AV3d1Rqi94CqIlo*)B>i|EIpO_VBYcWZa6=mV)UgXGy_HTS-T;W!Q^bp>}}H%R3==N@7Y zakoN_oYlS7FHP2I9if6w<`m7ihE5PC6Bz#*ZMJPqGAn0YCe+^(@4uX~I_{|mQXgfT z{=D!faX{{E6fuK4Xw&5KCpd+&XWJ5MZ#2E( zwrE0{=6x5zz*CR$U(=R;=id_7-{_+qt)ibW%Ayegih8y1CVqYIa__Rf2S#bF{&ciw zto|ray9D$0?P{@3-oACh)i3uNPwSWM#?w*#@_M&X$G`RVqo|$Vhv5KSR)?>j|F;&0Wm?Xq1&h=+Kd`Ek5c>K&~({)*;U}<@$M8J z-xTz4GsNsV6ST*fv85P40G+3=URgc>Hl9P3mrNgiyYPbW8ye9#6~!SWNiOIMUK>pm zDr()fhB2XivXrH`fWmhGm@&A#uIxn7DV$;QEz`#nG#;P?lS*FwiwP2m6i?n9+rHOY z*Z%ljkhW^On?aY+FC328$3Zh;$}xc$aL|SePm-b8h$nHQiE=EP&_o&*-HgyV?pOlU z`OQhNX*CqCT_6TK1P?{a8)6ratarwDT15xZod^^q#smCuddN=KaT3MfiYbt|W5L`% z8`1^$hBF3#=v`+5Mqm2P)m1`d89Vf*5nd-msM^tW=*KUj$(ruy0X4^yry@nqcpabz z3BI*Or1^gbpkp8tCh-m6f1^V}^lXX0I4;(roq97G*+Cg=3GFoc9_Cc>sh7YEhvF^1 z{QBGPU)OR-z~~d-UZq}~n#~Iq6-=wcsTWTZHjbm~#8?#6xno1QK~a$P9)uLjDf8Kc zETM%!6VkR$j2HpB9ed%-=K&jzm08JEf}{ZN1!DgWe?Cj=w!n0mWfl9USoQBjRzlI9 zEMOi(Nv;KKEuY{N-b`^Cg(|Gj6O1-*e|3z$=eh}4jD|oDwv(^AiXfV-LbU{Ia~dkjJ=qVPqCcbRa@n? zk`<{y|ANS$eX`*wcURFykSyCYG*PJ z7E|m?rSmhAEU^R5%m-7p265OH^)YSt#frDaffrls|HCUpE{=IZB*^q36R$BsBR zdKCDXfPYdE-dnDChR-Wz_<0?PB6cd~2*-&iqu?nh9@qoo#vXGf@Mg4emxRf}TKi~# z$|Hk9J)|XQ;X8y_`+U*)-LM*GW)Xi}0Yb>8mdd8h>VOmN?pl)jPmol}drUaV6O;*) z)NwS*QveUZHVWtr-Jxa0@k?Yxunney7-OTw!drnoLnf=uK%~K+yt>a83sOj0ycB-+ zrc!IpX}U^Adwa7Wov+7G%jMon_z;J|FR}7AUPR-)rI)0Bymti(7iQzF zsh^@RaxdW|>=8ZUNBgII21*8G8uMd5*1=n(Uop+o_*P@MS}km>FH?UhSDf_?x*8!J-%(22>W{u3a}vnJ~?{Hb@1L6FXPvK#}3S+X@s)-O75MYiI!ONsFj#OyNu zf>%*DihE3FMvkt`(RJk*8##J1N6(eR8?kQrY_C}9{X17$GP~1cy+*ZK%PhT;EMt3? zK}nXa>-~fc$RyS(G2oN5@BZ?6v{-^~T|Cn1y?E0V{ycf#i)Z8doNjM@ z-(+54e$r$!y5KcvvMaDKn(SwodFiUxi)S=b!ftr9f9CDvn7yZ$_s4Wi^}OvJ zvpK!YZS|y2Y>z>86w?Ex3=;K|p3tRgCgSYorI*gzSBnTnH9TWPy;SvzutOSau^>-g z5voV5-i3747W-NKa0M^w$5j2uAsPmV3bNKfg;d>l@rNO6e_`U}JGlzfgh97YFl*6w!!Vk6j3izK3@CCs)Oxh7|WjY43 z9^*+e!=ce%XBy1R3NzMV##R{U^L07w`a%1%D6q|9WBO=7SoIMj8ckj_iki=fmcZg4Ao0=v+@r?n19g9 z#lv?HLkq$6Pje#}3LxuLnAoA-0Ul1F??Re^^u0j3hb>X;NSMLajevp?>5h(D^kNPH zqR@i6fI&9J&O2j|fpJgS{)mpd2HgXO6?~Yw5zPdmF(8U1qF%MUW05z8yfc}1P?~o` zUzlq$jM@!!r@_@kgGu-ay(Z^1nDt^F9!A^mLc(cPcV9x--KE2emv zw+J7>#8#`tzJPQ0nNcA>KViL2LjW^8^YFMFo=(-uzzt}*9GIPImah-i@@bm7aw#KtwK{%Hm8vUt z4dH3XN{KTIgwfFythhqRYMii^ik!WW41Hr{Hp&XZ`!A_u7P5>&|F(M5ZqIyL_u|_W ztqmalNah!!VtKgA7<;N4hsuNVN&A=LNWzZ z5EMkq)pKRl)T!QCQl-csd4a1?!P3`B*R+!?e#-ZnlMk$6!?OX0B`XC-%7AHz9|M}-k-LVyCn=~}8B^y)v)Q7eaLCC7C-Et({op4xE7p~lB z#QhU|0}nYRyh$7-3Ej&;8(<{#bugXzX~J*8SO@KU@>=jSNHT{7Zez>aM4UG=_rmF- zc;V7R$i{s(dgo3i(MnlJ#*;V~i|Crws4R+>B#;_QZo|~O(K>)wLe9?jk&DO__QB88 zk(Ude4&Zw5DP|yZ9wu2AigA|eXB%Q=K0ZPS;+$?KcqQVjAHquq-_D_3D3n(QB;#;|1{kQm*ALc6;9IE^Md z{D}xF)5KU@A|Z8$)|o^9`U|_h$!tnm7Hl%GCE1Cgg`|Cn_m?d24&1pT^4ZLJF@IB( z>3Z5$*o$Vp@)OK`<74d+?fc7e@h&YF*Rp;m;L-BN%>V`Y%}XU2QOo5CdC3w$1hSSP zlf5FxCQ~wWIY=qKtZ4yGz5|XtIh!5T3mni{1E5@98+y!5vDfe6(sDYP>wVc36RgM( zAD7I13N|fZ4s{9=easT){uRcps>9U=o2PidrKT!Svr4)~&_6O&(KI&{C9l$GwSqYd zf2^mPs3%}Lfv`yRw_0gR>HVMIfl;nR-j%CYpLM07KR2(Xs2`&haekjv*+rm2Fj1H(V*X~ z;)o8eA!gGA^}DnGKKt&~8Kct^?dsL0Ra;%ut4}B-W8P}n)GBVfuclcMvRt{^AV9;~ zwUP?#)|xh`5>=;w7qurB4jriqB?bfKt0GQNjZGTQYF6afiuQrhr5LAl|3@3M}k0W>Zfdkv~7=TFwX3n;{4KMh$7x5k7Kxp$d7YQq2b92qf> z8YIm+Pfur$E%0w*cH$m;{fQoEtKILWHuJw5a-)m zV{t{i)@sXhiKViVTaxoZ8nsPjXL+?Ib2$4i)Rw(3~cKg(bRhT6Q6G9NI*nWZB0` zp_=(3cvF70y?m@$H&vk6m?ay?mA1N{^(SES5^sv-0k5mYdKQFlv|KlWbYM~`3K7+Otkrqy16|SW#!5d3#Q`Jyk`krroD`n! zrC`_8FvYww_73b2q?sEpH*mnnWmFf0>YdTVrvHNP^DGgriG%2!QXGA6GOyEdwJ#BNiibv6 zwp3?#5LQE&^NF0=piQhNn;GRQz(u{2DNh~?4u2lAoDcBZt}SKzWSjLa63Yb6kP-xb zpom*TWEX)aIELGC;4qX-ycHhFJo9h3>lk-y&%L-fqr}H@c*8~O;_}U}AL@-qyHDHA z%gY}h{`RNKC+pl&dH%#&xSmimOJE-=+jKaWp9$~EH# ziKMgN9@H;>eEep&@$^Zf2Gq=g9br?TT8`Fff_HAF-ppQCf6)%Y(2u`({qiezqW#Q> zq|)&qGy3`86zQ>P5W*+lSES^ulN;2ZicW#Ti`B9crw7z%cUF?j^Ozceuox%frcS?K zI`<&u_XN2*fq4Y>JXp}BCkorLWvLRf9qUz9TV(H2#eGT0F9r6?oy#X+_x^Sn{&pER z9yKl>UlLa?pX>!FYk2k^3CKFc45g-n)VTnSAtF5xBfN;6j!>OI`D0LL>qQ!Xsm1ug z$-W=bh1GNom!_~Wud1!_coyp&OquM4el`QpRIOzb8tbZaYrxF3@kHlxe&IDj7Ds07 z#@+1rS&mh^j@M!J$}6V@=x4=pbDx-A_>^gnbeRjh0ohG-QYbL9$>Vpf1Kc}JNA1A( zAR((u!%EfAuyWQ*Uh{*gb@lZwqPinHq0?yoI63FG^|RvD8g*afv=- zTLiOE`8^W1#Wk6V3X_^DLTCw;gX<`M#}8LtMJeqw8PU}$zloxBN2I**B`>Sf)Y@i| zdi-`c1e`=sJPkt6IHFYE+_Dh~0y=QeS=xG~Yt0;oH{~4WU`o4cNW=k=3KI09DoCaT zg4wwY&?+c)HIM|nLQ=m=kn{yPy&N=6IS3oC$K!=wAHq+Ul5facm6#}e1!-#CG-*c* z?K#ec35zZJ7dWz2%RUdLQ(w=|FR}ekA2%WKWAXV}_9FjW44bGC-ubt<*pn1`7NK~) zSjUnCzW8LKfMy;8kR>m_Lw7@7c`_HHi)w9H=5&S^la$h*BKEmtSMe`X+!Up`=Ph6ekU)wsOPUA)NEv;`0yEeQGkb{6&Y1_c_(&p9PZKF zv~i4-AUE7;JoxRm%#JVVW_ER%A1H(%Ad^zjKd>o!$(F0^4hS6UZI`-AY+UzbqeF|| zdSc}#UD&7eZ2awjA*K|tbp zA5?BTzg{R$fX{V{?rpW>D&)E7yx~kh_Ps7l+VCO4PW-soYK3cQtUTd2@kAlnSSLHA z3fe|wPQ8)c$|G*R9?`7jc!#d=$(7wMRZLv>;J9N=+`6dw2T(2IMQuf<@sp$zDRrx8 zwT8JC|ABY>04;i=ypmVtH}0Crh$QG*zQ@RU85|O--VVg?)2ZruNyKLK&TBSP>*7QjRm15t_ot1TZOKo-*CtE$ZpMgi;-4I4&b=y zPUg3r_R94FJ8_M9mQlaZ!m61--Jg4Dmfkx*KZp`219^;MVE^GHevHB4mYg}&>PcxMdoeNEem||W&np)4^ zRVcDsMEF3bp5Q_|$bYwHHz)8>E|859dFNzpbyJc>x%L&Z@Vg>0hc&|ohlVangB{L3 z6y>7=kZ@6he@^)n$-ypE!JM^fEpKT19%H}hS;`)a|u7el-;F{s>cZC!_a#dqmCUsxuX#j-!R%qG#DU}}Fs8l0p)JirgKDImT2uVfjv4HQB==SV8 z#+4}3AL%y3^CL?KHudf)5+;XtUvxyaWmgo!Di<*F$vCZdT0AHs!&fv5Imyq2Ry!-7 zCFcZ)^7+m3#`KsnJY|I>JZjWU?`xXT6pQC_qZNQjI({x@PV@l!94Gri=ezZaJv+ZD zn2GF&2}cu!1NLnEhGToSFX)>?ixkI}%!g~l!r5T**pT1KLO}4b7h%ULwl)7&&Ezmm z9IXPl!0#d{lUz-7F^63@WP>LBpz^_` zx%d`mbXP$tBT|&mE^$#YZM?)dry%6#9Alz~#aXe6lfC(f#Ino7D6Y++q?K$!zVvK7 z(VEQ`(r;Cp8-n^(GIGD8iN$(Xm3-tyOeng1h%YM$pRKF|BjZ2HN-)UyRMvupu~_3> z-s>KpWFpqEopr80N?(z!Q>s`BjZ;;@6dLDuQZ$9u`2CuL_E+Df1ncA0`DKsT_Q!;f;C?l;L-l$eH zuhQLDEaW{HxgX{$ierZAj6~Dr<&5$^@VL{aRrH`FHS}GLS5`|Q7zz;pt5+;wYe>}Sbv1Hb zMdAu!r*L@v&>uBGU5zVO(Fykwf@*oyxbzKq0aVW#tiF;Zk66E$7ft_Q_bRx7DF{L8O)ot>8}rVK zQ;B@C;+9Ul6NnRtkz?VliQo%KKp6r-r3J$T?-9RAU;5#C6Tk6Tm5kNV%Evhw^!H`7 zh7sHv4Y=}V{@*jk5_yI1Ka_lzt1|r9SsR49`@z-oKyedk-a^?0L2<~_;Jt=TM0i)l z#`htRigBV8`r;K9OepoQ%H9N5`|-uki0@4>a(|(Oi*Cq`%OHAgXu)S=j9qv4Gzz8k z2x!7@U)MYPQq>c`D#MDt4ZymSs$>r|>B^FwRh)de%SqoNMa8>f4Dw9BQnnE<$~`=8 ztVXY)ZPhA}V?8rbj(QE1AcY9q4z3T)Y?Q0A+alX$U8DFgeLscL!kSo!!1cG7sH0Xl zac66Jdd0JNS|iySSIZX*MG%|lBeh+kdDGgiBv1{5Gccf&U9W>ymu#o6z+)`e$`6is z-y%iMirClxZSin2ivQRmdNTMM&w&d-*_vwnj=a1xA8mmMA|vV}+o7j-VY%M-Koh7D zrZ7{%BqR_lcp0`N&r{UD7$+-X(a*nm@k1@|er#B&8=Ft#9}hb05AJSqysCpi#x$p*w_2`0)!`h`7-7CL z*jfuT#wdR=^^_AK5qwSicJ4RiTFa4zT^FOSiPFc!geQ;p_+KlC8j&4@Eiy$%xWfHs zjWLPjh^%`4E#HHObXdcqWzch$sGsMHQNj9rFL>WA(ObtRa1Venk67~sj=?Wg(h!Dk zS8!}qzyl(259N3!E@%dIb```)s#$zk+<3xxX+0jSjGGl_bhl)Sa+(L70KBp@l768Z ze>V2;Dr)h+Mos-lp&ndrpmnKY6uQBRg^eH+hDp;>ijqb%e|^rK2mOHl%)<*%EvUu` z6~~<$!WjT|vZ0=z3KcBOQr%M`G7Co6a%;v0CFd0grg`Xi;79I7Gjd;fQ~%4bUc2&= z)bQ-&VUx9{dN!?1c}oGj68?@LVGD2UFY?{T`K)LaN7MDhXFa}w*0s%x!fi=)^AoyC zhN-WfJwN*jPn)vu5;4pQOD#zQPowLwT1}l+1aFIXzQ6hgbZxD0C5>9Q3lTgFQuq4o z@2{VIcSiT3zdHNs?DVyI*7-&r5EEK-vhpTb*rKLz5Aq z1kGW>lW>J<7bSR(A-FwMbM29ocaV z=`HuYa5I?@)ob25jbOYXWb#*0XmLdeA@(f(Z^!z+uhEZg4FZNcDur?7Bgiv}8M~Yk=-|IC$T(PAj(B<5V3-nk#YpNHDQ29hmf? z!WUWPa;@Dwnrn;D&fNAjN|mj12TUd7Mt$fG2%Kh$_TXL0)h$d0psZrud;CBF3efCqofouMDs z8$_2!WtB)S@9Ed~)i}IUKe4q8q|ziM(~ox%O9hGEL^uB|;<+`^_aj%d3|D>#33+0o z8yTyp#_iaL0|4f7xlxzXtV1qy=g=a_s@F-twF3s_lUxVsWS(~uf^@Nbf+tw>J~DE) ziLCVr>mHP=2Yo0L^A;m9JHH5K>924;%BL;RlmxIz14(RZh>>2x6K28`X)qz&WtM~} zapqAJ<;OB_V<|k6wn{OZ*o#q^8{)d8ECMos9+W^j=(a0v?gHr3TFV33I~r;~0V9S_ z-;^{%sGb^aIdZFjd@^6hdUOvOY>)Nqrc_p-GL(tD*{2OC5d1MrTU~uEZ=&814W z*=2|JYW`q={s262NLog(nZ?49vcmv0ZwWe>%UmUDmFikB=M!wer{= zy^ehrDMgBmmrXLYd2w92czTOgv?}~=NvTLt&k<$s4lq~DMnRL-)SBff zohCNcWodQ-q!&WL;^vR)!_wnS%t!ULo_!6=eSpA+<9!yMnE_5`oWV*L-F*d|H!9bz5ZbHjjlu&t_+Z^c&ioBGMJQ$rq}_ zXZwD)^Xug?1}?9lwdtl&SWCHT0hy?3bE-BK?eeH<(R_-YWQ6^FREH_xx`%$jg_e!M zqO4;m2_3!3y!TNsW%DQwegUT8eg*;qsN&>_S|L-TWzKfhHLgr@Iqv6POjBJPl6$3r z%&zrH)8AvgRZJ|C=F*7?_G?=nWv^epK6}Ytz5ed|)7RgBccu|T*Hw;eXTjyyd>40< z;6t*=0nFVR_+%SS?4N_>Orp1bo`}m%?4KvmI!xW4(Jlk%|#3tswn!;KAD;R5v z!xbA2Hwxxzw!)8v|gAGe|yztm{&cwQ%d5Re2yZj3x(V|{c5DZ%QCg8h8&tU!u zzcA;7BN8S=8edR}Cra=-@3_0QH#RJJ+Y=UN+v12W4nQJi@Jjd#s0+bg$C^!)FnM(4w0UOEb)lAh#l+apT_vNvfRP;AkR#i`a>| zlc32BAD8No5v}f^J-%yz%qpBPkYS~46v!Ndz?coE8RmN{l6i2I{-=LS*NRih(oa1t zVUorRNNA`O+MP|M2LKm^q0;fc+8%)<0;9^~b6~Nph(e;bn$5etX$FNG&UhuhkNBVR zqg_#zjC94q2iG^{`6RDJpOyl5JcQ3~vhmUIh7J&GAL&Qe|= zMiT}Y@em4cjts(?ym`Dh7pGC$;et_JRTTd$yfKA%dV#3bDsJV+M+7EU4rIm$ji;)? z=ZeHbwrqP$QGY#u2Sm(!1A##4B&$ot;NxLm0;L5 z(J+o~9O@gNGUz#y>-z;xsil~savVvjO z5|xR3HSsOER<8n?f|RY>q{>lhbgIT|zM#2W>`!blc&O~T*w%A#-87fCVsTI5%SJnz zGxJkr%zN+DCg3KCbz2{1-aWDMzNT*NmvPi1=8cnRu}*#U00chC9kBuZQqD+-c{;s| zE#^EfV8pts8!&4Z!4xJQ3ot~EnYlN8#Qt=4ivJz66`raT&PMDIat#4n`E726yWGgH zPGZ4%u~SwrvA{{y8zAM{iswFEFegqq=u*XUx^r~2e)u3*uA&(4SvYnaY@m}Z?@vG3 zK?5eE6}!;M4lX=obZf^ffYFG)1i8Q@m?h}#A@g@O-Y9syv*)BuP+pW=<8TL%!eeJ6 zmSpGdwG&&ssO=DCeJs;XPd2O{G9st0siofMMOP4FFN6o1p;T5#Nbs}{y|1yS&==^? zO&+Oo%%s?NRPn*q+i~9XhmYpr-EbpdQ8&c$k{02N6$Za?wxXV|^6at1>G~%{d|j6Z ziF?e}827`k?kWeYeoBFS_{z-0$BVbeczJrMxyAbhK%GWn+H!=`FIRVQ9V6h{RI_0U z;8xKR@}hy$HJzbGqc zi{%W7_&%4j$`>fg=f%rauF?xzP&d?DOd>M>yG=(AQAb;yK2^xb!jdbHQM!k?(#mDo z8Ep2BA!W2|g-4RGd6h@X#>cPpNK8II6ZcNbcPgH2)d=ip_mUAl_qHA?*>;~xOVtVd z)XPu!Y?>k&@nE(y75J!bAE!yn@NOsq2HNkvh3Q-8%LBLce!{d7+OnF<$9lW1C$~|) zLw|me|FkQw+aJ-1sCbCPHsa$t$1%%htdK}Yq^6D?c^y0Qh~A?SKP}lC+rRe?czu$I zw<`LBKS70Bu6@I0wdfxbm2116FsBGI_$)}&J>J5l`?R?GTYMQg_oAtAt(G0=dLF>^ zM#m{nnbc<-8i<^IG^&-N4mqN_+4YB23UO-{lYR@9yH#a(x2p26SMagAp}1rgh>z&r zW%1r+Vm9cE_`1z?aW6PNzpJ3R_+}L}`-GPtdvpCbX5aaq{c=QM zFm1A6n&gd9mF{eDQ@}pRwsKPEv1!d>?JA69{~!%dOLn9M9-r98+*Y?t@raWMMKEU; ztLyS0+9K2}(*xbLSqxswX6q~3x?|Hl!u7*ykNNc-s>>e_3T%=8_Q~TEPlAR&G5mj; zxpU>3!fd2n?Q;!LkKUPP03%1TNfi9?9gvf zOa8l5C@fxc+@oe`gXWcG&{U#pL?-2$U)QiK z(fi(f%3IDHP+V99)WnGl#~MN?$r0#D%4o|2{9ck5XYAANT)N=Y zjcmfNr4`5&ktJmKy8ALuD7^RLxed{UG!J?#uHfv5ZquH**L-}tCz$w(01n7l#W}$(wq4# zUmwQ+8xXww5Ps6;CB`x4YLpkuPc}Q*kz?p%(9~gZX&#Hlzr1XODJ>kNT5yuWm1&2N zkPCN9=n9Mfq$_DKCb7QvV{7-}9=DX?Tch+mJ~|H$2Z=f#wf@X{!dWA)S-`eyp<9fu zSjdy2sdd#uB{biMnSO`2%;hpmMY`lt(gj00@%Y3w70p_Y+KpJykhxsm0hveiVXAaN zP0yngowb=F+6t%KyI8~;glR!>OJD_tbRCY$7=du@#UPB)as{T;I9LP@xxgp4Djabz z@&N&ldGIJ*UU7vh`i`my2C2ZInO_Hf1Kfjl_R2M>O}o z&XTfuKt?^Pn9UTJBL|Ok66%BGS1L7rQH|bBP+QaF1jmc*3nOJ#sW$j%8c+@=&F*S#`1^d09iS1?;`grbXgOgS33w5rNE7cue4=I*mGPoVbg4Px zlS5hAESc8n#n5|VU!=El#^gq~F!y)(JMyzi@;&6z4d7%$@zjTmq8-yKuwxD7$?}*1 zJ;tFB{Gugwi$^54YvVTY=U@ofE)KSBP=83y`NAc8GE9N?bzF3mi8*f1io_1BNWXxJ zLOT3~xPKsd7p%U7&$WU@i7@K!njF%`6~<3+7Q4^AHY^x)f&9(+5X2j33q!M7uN@a-8r z_;yMUzP+Fa-_Gd4xAO!1kKX91y=fK?_Pc^V?HM7=nIM0z{xP$lv0@O&WNATuWq5bsM;&WGn` z0(A!%WG7`HSqr2LPhW_f-P2yim1jiXi{bg{b9#F|JU@6rZTkFNlJa67NT9dV^RwQ0 zpOgHe%Si^(`{x2RRHS1kYp4?*oe@9mS+yf}M) z&_5tj&N?s7hP>GG7taS&#IX0Gdq~s|&R(3opezUb&rZ*X7yIYW4yko(hi3<;r+wa# z2WKz16!i9AJbz&)<@B_3$ffaY|ESBw_Sy5p!|pzj@_hKb*CAf)KY#YTPtDRhcy>4> z4s@SCJA6j<=${^TPl-SM7l*@BO4~meoIRtoXCUcE|DU~gZ*SYi5{LhvPr-P)D`_Gt zw$pT`N}WEPB;9tmZTcqZ?DTQ{=Ak6oW+RCjk&5GK^WA^v1`q%Nkdp26vgJI}SR}3o zz`5aEp74@>`0>Yq1%eL`4jwZ+AAEku2lwEk2h=~z_WoZF4`^ohKYs93PSC+$|8}1m zKD__Y;R71z!N(u|^>1`W4nBKu|6?k5{{f7Sal+pY4*z<@e;qx3@LBJ)F%rOfC%un8 zIygFz8uwur&|jZ_1ZSK<@Zr-#YUSYZ{SSEr2Or)4>qqqN=-}|^Lwfh&V;Hh`y41<% zNBBQpq7HwZavT@n`WH-H%cQ?qA$|DJwrxK3A#=JDO^FOO@vDui9VdKC6>Rw6TYU<- zR}|9vX5zPPA&-Ox!UOj9ZVVSY2A~0H zJ~=!kKnS66X%-h6l`D{Fvi@N+nwb<>ZOV&&&!1$JuG(7%ac7G`9MqICFdW;cHk7>; zAY~@2Wwz)AZ3(B8ZU60GT`D;$Hxt-Y{e$5ln~DFq*X+Vu<1oF{u#RgZCqblIphsy^MS5>s?0FuCkhX?IGe8+%mZm?&vH zYu7Lptfx5!@7hA&6LfWS5=1t8la8!0>8yYLk7m+GU8s{=g(w$k-C;9O6h&5Z*a?S~ z&vaVBUg)7cVaU!~T9dt{+xCc6igoSMqr?lMQ&#nR<52IweH*EvB`IYv?h9=wOf4qI z$YJh-8820D&5b^gM#W4vXtIoDmLQ&K7sE%CN-U5(cF1ttHWVy(!;^Z|+EVO#(nlg< zu~GWIUV^zwH!JU^^)1_0VK5lS>{;h)uf+g=5QvQs4PM|0GGY1#1iUo>nR&YYH7jq1~5<0L#FobX0F0pK5^~oXheIa`jNBl)x-;mc(Rme5pRf90CQtb22x@|D=XN1jikV8bSd2Il*G6@P&z7 zA9EQI7fyO;lh-4A)i>J>i>4hPSQEM$VTSmGHgp+Bl_p{&A@L0wInQK?9Nw7qn|YO+ zLnr63^36~vg@;DjvXL#eP$fSF;`V+@K%PGQ5M(&isXu^!C-Cn_@NWwLy2e#q7PW?P zth6;hwFs6PoTf9xi;F-qWR}-1Y6=Ujk1#m7e0UhBQQ9?1UBeTNk3YPzYrN4l-h^tT z2Hi(U!wXd>eMpU1FWz}pM!^jVAHf?BMJLRVT`JJ^pj3(%t=Qi>x!W7c-{;2P6XWj` z{vLR22d8tdCj6VtVv?=<)H;gbBAOq5nf1F8dj3 zbllIg%ZMDM`gTK726(hl=0J(U<0XEt43R$>GlwM-q!v=_v_2I~A2+BeF$;ucPuTwP zX3PTNW~2lObZO5W2b{$%I9=7|9Y2mTy4*!Z*GqRTG5y(cdaHtZV1p5;wwBdnjoqlY z9_T0%3c5|A-4;Mb+-Qpwn(T%}4!yCXv$-#+dji}6Uug2)KJ-O=hPp@Iy6ZGwPKryZ zdlaa9-&;4GuP)M9O?I?DG+BI{M4dmlk;<~Jb>!4KiqxV;B`pqTLls*87i4mv{NcfY z&XaNoyk}GE504(WWnBb71EIyIeum72-0R=}=-?i7RcB*y6H6cvt%`(y z{({5vRk62;&2<_Z#k)_Ybo=jNC`K13uO85UJgdcw2++?pn&S+ab|E`a|CqE87PS>U zCxlFQJt-czHbYj^-*;%5C6>lr9c_n<{9MNPM{}Wd8VLzQgc%N-rLLN#zHgS|%)8uE zePq)zHUy(eQ|~p_`W0l#Pl#Y z%D7gSm*c-{O zMUfdED>3~oD}&lZ4v z`m+(8QUaL;3*bW$k9r#r%l(QjR85{7oQ7iM#NsIRC=P49-H;DRv|Y|m8f*A)w}x4V z5p8!1Ks1rLnl;~P>R*j3N~`Q6H1MmV1|3-y?N6ar4y_&!qdgubtRd}P-8(2ee?t6`JK$y-&fTK6aCb{q=p ztM;yJ6i^pMwf|*X6r%7*fT8X*393#Wv|orPtxC?ohwow!4W1aSk;5tcf`30c*u!wF zauN>@4!7*=xwo^mx3i-M5A4o9?6y4b6?!kc6DqwE0>R&&(9xEi{g=11^KQ_4@GFGg zKfDuq(+zs}x9sfi-p(f7p!d;xfu6!;99w*x+{bur>i3)5`;O$kF3y9HW_` z!u50;KxkfVHb$ViF6;X-rIZ|er6vMC&8FqTn-tQ#_*mP zV5Sb?;wpGmKv$X854IBQhn|&6A{eU62-8L`u13JFmi$&>;BN_YWEpP><_yKIFc&YM z7|h9+@y0-2LeC)FcLn;jz8Y~+DE0gpb68!}flCkc$@i*sLh(D^4f+fXRFK~L%MCq$ z`U^DwRX$m#YmjmNH=86!ql4tTFG(xG{U!ZPiCK78>!K>Et5rTtF)PYNw$3YDtpT;M zpL|}-=fx$KF`Om z!eDmGX`Yp1DKb^F2m^PU)0SdoYtMta)p^Ip;i z#8|K!x_(y`$viKsdl+h3Z4Kbo02b)+eLgQHuzl*{1e5tZ$+$oX-KVqUY+WpHYU@l| zC;>!#n>S0od;S&nf-@(J37jUC#r!$C84MdEf{T^{j#GMtdlom58jNbkZoEEg^8x_AAg@Kv zVNZ~yYlWmVGh**cN+desOQ%6MXSZr=otDrmpv{+r?BGAc7mUK01>^*F(gng%h zwW0WJDvCP>C1 z?me#7bMsIBM&a>{<05+}#p65Kl=PPg12wgxI0={c z06(qIDG$7cEn~cri4PO2`iy6&%}lFJ`JAFE;KQ*!hBbkLhs*qZb9M%ZXNF~nOY{|b zPLTcbNvxq48W!O)&$AM?eVIku0j!p^4ElxSkFv|`3uI+kKAX>lV)5^9;J^`Vg(w0`E+ij!+BAAF){>n57t3vRf#C(@HBC-##*On5Hw)uK@ly6$TJ* zCz_AHg%anW*o*ol&Yjp7c;R*~ai@_|7?_UuY2YZ6$raUvF*ws^$C`cKOW1)l#Gf_V z0H8=VLvr`k^@$ZxyrQh?Svtij3&nm8pzGssfVIubi)>aeKb?#Je(~%ZBL7P$c#*z= z>1`GZ;I0`0eqwAPAB72Cq-a5_e?FJc0{b*AipYoo25Odqt7EJ}EDoRa!eH(j9p6Z5 z50f)AiyGpAgKlu|TkQaY_!SU8yfn7nBSmCWDoW#M@kR_{-u&``FQ0M0<6c3&1emyV zQED8_z{ZV%ptk`6l?7LIs_d*WkC$6i(SWT_K(^!nsYOPDA$CT}?nG)x<`vq22H_%F zN#qSdzrdwHQxOUT1)urUxn;x4j^Si%k-0zmr{!W6=^!he}2A}f#+@Ns2eRF#7$cuJh` z0cT7Rt$4esS-ZGiHmRb>^j@7{VmZECR>eQ^?6N-^jfM&RCtsGR>8-0*7x}tw#2O5p z6A<2>Oya~*`fXk zt73opM*qat$IiX0>~o}WRC=5|pp~_Clxj%KliVYkWjDi&pWC_~#28DQ19f;sM4)L_ zJ2JW;#-o2^G;`sszKh!~v3!01Ll-T^d5OsXIb;9|4LsWiaEV<|%+$(IH)I zs(DUCa!u=gHc!t@aRYe;ko>>M(pgQ}0ISs~Sdq@!>dqjY$he9$O4M9w>|jDAO1jvR zOkQ=0Y!i^ir_#Q1iieV~D1N}c<~O0^o*G6E7q|U5 zr>hfDsONH}pXckcLWRH$J@@|fCnNg62g+y&4e~?-ZTN;6=93x`0lHspCcsZ12?sn? zBaFnMbmVc9Vo`}n@iuWwsK${6SL#tdlNZIDah-Z?H+Dh$R5xbY_0jjudTw^s z_+Be#>HAOb9@aA@@f;EN>+io{D66vBAM+k0@<^X1eOUl%LF34p61uG`yyu^dkZfqa zzs~-(0on^ln)xN@MttE3Neio_-hKmb9XdNSm>g!o-LWWH(l-{M^CV!aGFqWW$OgLD zOaWtLq-FE+FWiF&QU2ND(KMD$mNH+yOwWg?b*6tsSV}2)Y?KgW~#2}3hi+-0#)r=NB1$uG^Qf#6G!@< z6n~3&kAEuBp0sW9BCG!7^n*$p*WuYZ>WQI+Gz(cX4;4BqimJ6GK4==sT7PJksTBle zxFH}6Fj>(IgbJ~k-BO6w?0J1eeM1reJkRK=BU)QDbV3JERwHmhtGtJq|Kmy|}L(S1;~q$Iy!h2V^S7 zJ%t!+zQm+c!L6oMLx1P{J#U%U`O4U_{xYywf)-vNM(~x%*6U*ZH8bDFplb=~p@Tw{ z!b`nWmMwNEy#M}+4~mpxTMT?8Ram}6?I)^(NRqMZpP^eNN+16EN*^rRVhX)B_Oo9?G0VoPbw9BM)!aU4S5%*IU)!ffS-!j+R3DWcUk--diI^|E8(6)xxwJ~ykhn670uvr%CMLO?2G+@K8hUgNHK1vnP*iaZnKvC6Ki+^Z~2@E zt@%g3t~MYdzwqqtT0Yk+CsI0%PTiD!pz?gGE{d7KH!*?`!y&m1N2{;Pb&l%U^vCpV!j&3Gf^3DnC!6d>*7ZkIbW^G} z2imA`aG5Q>vBL+wl^o7UGUfWd&5Lsol%(FX=RF_c^|#93zJ%2q1MC8nCh=)RNnw!r z>d|QBFpu3_^NvQ85QO~gr<^sw)heM73v}vq;KRDYEuNa z{?n!ApWL1eSz^o=6pinXC2w}4^4klORc=K-(hyq@0Cyv z-*=RfRRg~7dHEteOV>Gn@8A3X`~6SvOfTTfXFmf|zAp0FU^MtkNrmp^1J3O#?$^b7 zgqNo!5>C{qX zGPs&=06Sah&s?Pr;}g?cjAK3I<+t-RUp^zDPnB|=%8(*q+GQ5+=zbDdaXaKY1#DCo zi&PpTGfIm?%mvCOEIRw7+rv6rWTb|rw0G9mH??gQzc?2BsI)Lm=Uqo#Vio7Np0I4T z5=83A+h%b`iy+0-+faF}9I(pcTTcyJU7%{n+h%!3eIVu4Tc3xsrB0CY#@l8UN5ddh zaZ2fDJcbOJ7CXw8)|i`3OdzVb&iw$M6D z%Q8P-B4l&q^P2EpHODNV8Soo&(q1u#wMsrwfYTn@EA-H&M}{Q@PFAOh+7Qn85w3M9 z57f*PIS7n^zuLpFM!*GV7?Ba^Ql4R`Z87iyLR)<_AfeV)3TQ}phhdZo4&e!13WbJ% zU)@1iL%;$uh{zD6ly4An1_E%9uVFYyffhK_rJNy1Ga8S1tE;d6x+YHZRyPU^oG=!e z&S_(7yvs4om(f1QHXBfolTw2^)WMnow+6t;M-8{NKh}ZK&dK^z_TKqp;D|8V7{eM!Uqt*dvUxDMNt3SQ<4JI<2QSm{ zuT=d#8Z=9i#A(YY^P4Z6g^j%0B3lSEfiXjDq?dal4%B2mp(ylnDiI(1V-2Xnvc^CE ztg9c3_)a0WxMyuDhFEjNH|jN+!8`MO>CzL`k<00EHw{kDSs{R{^r)9dn=j}w7*|gC zPG6i<>_LE9fnK6=ZRIW>Hkn)9^E2+CPx9gbE%5<|l#)sS-=1a`~ogMET z9qjeo6aqo`Yzvl>Z<(;iG570>9rrH2?NPmv!!NZ?4q6ZHTI3nx0iByCeDF^G#K1qN`W$-w%G9S?X!Gr? zI@dEbujXjRZai(0#ye0&2``QUMio)k8>N*S%#Y`xwG-xcPl=mRKSt?lHNV2o!{jr( zx*TbJ@2JGokR!CSsJp^smENd>Y{70NvkpSU24Or-;LQoMUn-kKigLL%R5EKi z&#No$OH2$kBrLVnqCD4kL-+vB`3#yNkEo6OahalBL6OY3pz;S*B<52?Cq?PZD>89b z%{?d3KyfzQ*QT#;QVcXiMbA)GTaD~?KKt*_{z(@YYFO}_Z!|{9i!4)gnWz*ws?4$~ zMfuro=DDPqtcOFt^W`~Or`RLUvYBxO*!#ZNtjU@dCQXAc;l)%t(s9OkbZy}~`56*j ztgSVqXvlMh)+oxV*Y1xjoYZg9?HO0ilX<#)U3>cg+{B7X+;3P9tn8eZ+M2IEzLQN= z70ZS>WRC~gqdtF9goX?E(2{VB`i4!uFXiCXlq@1@XIcG4jHU^!1UJ(;1D*VOuEMY< z96-WH}F%Pdg(=2IU9m{&~@BE^-5!+D+WI&I;gkKZ0wlf9qu@ zt3A1K;ZZl&7kCYcw-Dd+SWfqa*YDqCmsV%^4#_Q_Hs6R>^NnW!sB`Ug7yy;iBX}ym zboX@G?;v3BAkgUq_F)T*u?YKb7>481bzxB|b_^E^U@UDxpY#$t6ad&|k zm8*A%{HoCcnl%WDj*beY*PcnThczHX^Kd#q!$dDe_D~clrzIu!d7W}JbqTk8izMkFHxJFZ9cy?|y ze*C^pS1vBQc(@c*q zSWKS^RSaJ^Whejrs0z^wccJrn)>?>(*!EaVGP583Y7B`iUaSUDp zYSzl?4+R~gIp2^>kH$z5_KOSwxi$g|uS0G06fHswdvLfH+Ilnu9_1^8P|_CvB9!M^ z8n?DIn@{iFrlf5*8i25pcfRt3@o8H5BjV{<(A z;MEsfL-Zng1OHG=D0r8ZwWF%gk&%BahO4n}EK&yXtN1(kv)QvFE3Ra09qp-*we_c; zCd&RluQ*;F;`>kk)ZV_yZDX)X6CctA6Ic5Z?Pz@{kyNsV8I$93hL2z?Lg=>bv-*0G zmQRYRY+B39N@|M>@hrWf3l<tt%!|Q@EQFt7 zrk)<%lljHWri7;-Xwr>kL7#L{I?euS}5x(+z1`=lBP%|_LsZfj!*{&`G!GZfrB zyGT7zRP784=1EbJx-{F7a&c2cw)C;0D|4n;cuW}aJ78p5?SW*l(PK}edGxMb5Cj(Y zgfhLEJt`zd6W*T*f5HA+Qf`VUw z{ZQL2z!ONWsv63&n*(Q-+Am)ymN9Y8+Y;DfI8Tk2H8ahBd@2T5{4x*M?CJfIZdyQ*QBsvP);Mr`hJT1)m`^JuwVQJ05^H#8 z3L9JjCByBu0{Sfe4pyL^%dM<{mxqXs^`=gZ_pE44z1t$mY;qCL4TCwOr`;LC8Y{`F z+}eirZsXenPTkY)zzJGW`Yjh!L?n|g_~cd3)=Jo7HQ1q`{e0#MQU?M^j|lSu~=CqqTUfeAxxv`8%BK+^MJU|5j0}}7;jMpTS{@LdHQ7HzuZTzN9kIXP`QV!_5qdBu=_$%2RyXAe4Q2alw2ghJCVBU+ z&zL(xXY}ap^SJHk)p=MHOQHQ$P^%p99PiLzhD4QDCDV?V08~ z83DDv*UB4%wtF)zi_7Y9!W}kY!S6LPaa>Zm(O5&0mgZ2DmS|f8{Z7kT5|B~BAR^n; z&9T&FHx(UhbL-SRxPtn?0MLp2c^W`N28rzGw_}Y_j+C17LG6_2WtG^3SzPonO@rxj zqFs$V@jumU>(&j|lb9yYaByi>^wU||rQQ5rnoZu28w|(N@jAh%K0UrL>eCIl#NflL z>P!FK;o_tNeu*g6CCgL;!Tgf6);=HGQY*DQYgsuDwTiNnZX@RQ7g!@0WW!i3hea0O zS!VJWtWuNBotV^{B|;kOntMbJMqIJ6tTTI*b^LvQJbANffKCOxw|IM2+p0dim>AkI zBvKa5U{;R}Bwa=}CPNGz9dxV9RB#`HEa;N?kMk@+#H;-3*_si{f%d#odumV>*7sX|_ zZGk11zpFiA2G<>^+bZ|N4%-T)cw_tAQSREEw_D`C5g%HKU)e6BG5NO`x~z5EVj2Ly zpFP%fgS07!t^MIQ4=~^j#eQY`!$qEs9Ihh%H|21N3~)6N=NDM~V2U1Nk1 zF{&}#+mWM;Zf-@8>UQ|wn;`WbPJdkgVzy8VO@UAERf1|qE$#AAx7B_>^3iQCwYGlp zENLe}CcGcsvsBZ*@+~q;w`G4%GRrT$?rri(c-h~xu+qNlF;PUb5**yqVJBPww9rK> z9AY=hNTClsw6RItq-2buw-PgbEo;uS%wfpYP$hw@ZF?JA|Audn0~g3XHzbPqq(@I7 z@yX0^LMd^{b(g-Hag8{c5)GwJnLyc7Mn3l>U&O+u%!DEn#W|ADdAvX>P^LegG1-Y= zs%3y3##(uT2|@=0gvv{J?e1C?|3)x@{Sk9-tqXKyp**&}U>zLy#>lDbc7O#|>kMP* z;HcftBpSvygKCAc$HnjLjR7k#bv#mTe9?GJMv0+o-)mb9#W-e~Gq>VjLsdxjz9f^C zsCkxjA4be4JzO%36p@~1y;JRKVKqi_R@Lu~dlWLS-SW_#<4KRRaQ9C26|!KLyGid= z;BMG|qsxQFt@~>qtxF}ETadPUvP!FqlKj5RO=*>uqTgc4fc8rai^#l1qtSGjOnfsJ zq&zJa*eol}xN)lCoE_&kre9ueR(deugD!ryedpjh%*6?RG?4)l2+b8B=RjRk zieQ~=$ggdqwu_Aks7RPmX|AtA$IsIx&;V>5W6)Ja@fr>oYz+9{T^j7Y!-InZ{l>0V zX49TG!00eJJm}&d!_aWI()l`_$UYM7TPfqC^sZ-T`83BEDYVNb*)+wx@#va}MJOo% zqD;5d%Zq$ox71_g%AuXud$CN<$Q~kr9Z^?!*1pL` z#h@i<(3{uH1grzt2Jwfk_6nB(`aZ|KuFg~^FWM${kW(fJQ4zpdvd9(%=CGe5Ux70H zayTdH01aV>!PSA=kq#4C*0&bf0He}P8>C;11L4X zb@%V241=G4PErp9v{D^l)W0*)Wharcp%wBWeTDD^5X*w*3e(5)jG$J z`Hp3*jjwd7C;X97TJUl+=?FP&S5GB*WXwf%ap>px>Pnw@D(!n}`2-lf-IXjtF;PJt z`PU|$N8}M#%g{Vfby{RfZmP7{a&nBr1&0a|T)v#nHx$#g2W5Kxqt(Q!fkSmiV-`3m ze$3)J>S6edNEC}2$M|tkZSHOqZ8nG7WP>bul1@3R642Hm5KX-R83&wAF>4tH30M{R zvhp4y*kv`k6%WPcL_=Z)*WM=R#bTx>4q}W(BZOlHU%9a6+`~k z;k4x15O*?tz^GPbaC*MP3u#K+Mw(KV#eu9H{|W=j*V1G{ElnoU5-;#1(oz_Dd~+!$ z?{PMt!@0wQ2aAyp8~bsVuUP_(Ko?uI>3QRB(my1CJ0PL-HOOL;N*WIHyiU*63|vhV z@n4%9bMXScxu9UVfMYSU03cKf6s3eU;x&mXd(plR4$3cdqW;)FM14*<^^x!`rrwp> zn+TS&sssO=GB(BaS*?rNW{UIk%sGl-8|z%JtLLDzQ^ceIu81*eN!f8w2{x?&qZ3<$ z)V7C#?2Bd7@1HQXpzKMM(jRzJl#e+_s$UeKo^@mq8OPop z^Zsyl?$klF=ofvAjxxYKVoye6^vmkW7zX`n@5#qL`Ve2{38DOOmJ|wDPXYud|75d(yIJTCJ>d4n&xG(Y{5b1 z4NVvVAB>>LqE8SsP-g%hHYB30(>$ehf>XS8gU*{IHnbZawINn7f|1Ifv{m+_rx3rQ z|7zSiH;gOzTH#lF&&-CfzsybZjyJ}>3*#-33>Dv?sV8*#b-GHLQT_OF#z=MtQQ)rBv)F=0Gr)=@(uj6cg+FGpe48r#Tk z=z%dC5dC$Ekm+G%Zz3v6eIL*%Vu=}ekmtkdmB8(Znm<)(k03HN?-;_@3HYS5*|((4 z>a0*xk7u*Qqv&)A_R!qeP7m#7^|yAx>UE01=Z3CW8Qc9jZdi2?&{U3R(uYF=LL`R{ zEP;_P5EV}^^7#z5i^OfrILe)PFw7c;CJH)@Egvu{Ih!XfNA^cPvOg+iVbcig$M~Tw ziHImAUv1{~W8~y&4-HXoiSr0G6|`@`hjyBD*q$d;*RYb38V(}bzu)Ltd+*aLiX`wZ zJQHBp4?Q|RSQ6z95}l7~0?!*ab;pQ(UB9W70{{7+L4=~14OK1KV@_jFB99)%xm%Ft zf=vs5ON|U1p?nO|{zEf{G2`VF_`dT^DSU2xHCgU4l$BBZZ2J7vwS)JV4y=!AcHOLP zPb!SHJB2Ybx-$AWIS3pHr`mm|+F_(x*WhRPlprX=!pO8$`kvsW4q6ak>Y;ITFzv zYt3;xGKlOiHD19Aidbd~vFUiSRIsXOD8raq7Nhc13vsB|jYgElJ=E%VA@qcDO@U-SLjWuG;D406k=k z3Jfpc2DfYE6StB3gIhYMozLiR_lOE{q@&qR8@E2BThiXnOk}hD``2$@&hq`5TbJIn zz5st48iTvZ-#+^A03VLAeF$=GGkC10i$gpK1S*k*Kt`pF{0p)VN?7|1ZDf0=890B} z&7_*R-?0fAfu&A@z4t}2S(h}B(FOiFq=IOzxv4Vss|2ZFIV%Bb*3U5pTX2;#o__Dg z`n_-Td*AB!zTNNr?fQNHn|CW_={LWDcVb+2M484;zlnYn>ijLC%vOer(og?}xh$~Q)oAN@g;e@{gD_LlPxqWpiAC_j1+L^+S$O4LengW83M zQJYSba;0Y{Wz?;bTjevGf^E)y7trx zdo(oo9l`&-f&XTpKgrGk`vstH79n4T-k4$wWRfO(l1^WGFA& z_@=uUowQC1Ezi>M#3G}5bbEuMwA43Xv%_~gw=+a&=thQl=T?WAEobpPiv_N+V-e+` zb*{vSO_)%r7izD>&v#v)5hYSIqfON)Q@?$i_}wtjx@HxpbtS9hd2yakiSc^&ocwRj z=E~3X=bycO`09T4%@bAm*tJ4cbB%h3_=7zLL(V9ZUF&7`c9kwMw0m3JNpfV%)*1Pl zI!3^1dt-?O8Z!I$jumJVV`(OzG)Je+ch_O_U3?@q#B(yeZ?qCj6`&ww86tB#0bq9m zIzC(E+w??dZMqv{f-^SB#|-b7#2u!&S(o{nY;IqwX!B~zsNbJ*RJI4j1~Ke&V;jxC z$4pO^6Ke4^+6X%i!!ev)C2#UFud?Q8^|TobYO4G;jJP0enaeeK*m|;cu43`Rx4jdT zK546VyGN4h?A)=vQ%7?yeqWAYKG^5!@;q|`xUTW#Fv;Fl>(n2qe|l&{jO6k<)2nw8 zrP@)m+F@I@X+Z*{?u0`fddnT@a))&}&p3SUQ4-36jK#1iiCuxd6YJu#0eK4>B0vZs z(w|3B?Ds~8U~rv|LP(Ds%PQ^$3Z7U=RdC1>+2j-r?){;~)C6fsI|V5QAn{h3xqgA_ zX*EgLP5eYIC1aR`N*GD`3lE10v8Ur?J=EcP1T?;sh16*amVQ@jS$LQ7iWrMI&pja( zB8vj8v?U_IxsdxA@0|7sU!ld!e~T-b*UJd7y6Kf{4(dIgJU#D|P|UTR_kkKv`(B81 zLf!X)Oz327L`FYt*%|nBayJ79`NB#ZxQ`t|1M+{YPqC8n*-V0s*7xv5Se*V(;` zE5NoioyiBrYxHZw@Eqf$r-#w=Ik6FPR@FGp2ZaI9mD)q}lY6Hs|96VgLkzK`T!{PT zu`A#_F|$ZXQy~{DmO19F*=y8eq-dr6)Hi1r-s16bw;n`tz0B8TMc&EtsN5_V73D5<$K`d>*WO)}eAU+l;lFx6L&pJ8oU?cG=^dHrFqZ!n;e3 zIcpcjHj)Ddhq{LHfdmtL>=61zG#TM7ArgX1v<-Deux_AgIBH2Jj!fuPlGyvtt9Y_V z?s#xT-_M&~9I5byi&EZXGJkF1svO*%0%r+D`U%GgcuQ~4ZbY0BnkwYNmKQThy zz^fNkx~|lXm@4zgQznGxar55f)-AC!Khz=|b6|E)m$R0+L5a}J>;>xWWy{%KAX5#V zp&L9w*i%Ocut05Z0t{aAI-kDIFbceP?-8bYRTV3GBZj13E~|cY3Q)c`7%j86Revx_ zt7_fvQA6I~Ky^19ptXB9TpoH_t+O{<^>ijP>eAEMHD+`sQ|jE$wo@WzV|HjU8FYVZ zP_9pmRaASNzdWkHT;tSQu)z$n{hsX4CW_tSxWZ9sOv^KNfpB2X{53qe*3;-Nh&w;? z>~A@}!?RAyb{1Z@GhR3fdAlzDoi)#}Ea6!z?Yg~1Y|hd*llU^8>2*w6CwtUB*|tTs zDRW@DJ`5|yAE?QzwU)^=o6l=)rV1DDMH~2$84Visq0|Wf8!maXzXHfAsap zRF3-Ag*OZcH|Lvqs?r0N0bLNz=mYu`-AM34<`KrrXJ$~ypMOrw zEXsVNbhXNGg+H0)Z<2g=ycZaGqKA%*C&$4)-lJ`ymt&6=Z^WrWVOF|9=|{c0(K{3K z2`%)eNzyV#5ViAvXY0V&_SNS{pWK6H9`%xO((4Vn4Xh`1BFcs9u0B4k3N=|^J}(5` z-cUojb#T^MZU6>Q8{(jl;yA-E`x6?zDl%!xkV?Zs7`Uj+{5C z`HsXqOv{gUrR70Ve)NBgl=~^UpO9adj(f?tmx}-2AmV^+&#lga$CbOi%E¬gCR?DSfL1Ue=Tq!@%zz7S z3bNWHf6csnjGo!^dB*M>Y{40FIKamo@&}{rZH4>^HHIccY6|ID1uFmp@NEFkO|}9^ z7zlcOpHi1mAc*}4;^85N8Z&;Q!I96Lpe5;peSfYNy5233HLq>`bj<&Vsh@VIKA5o@NUI>!!Anx$$Yb>JWq`>yxs9` z0sei$nIYVR`7U<;ap*0_fAL06zdqT1>GBQ~1d(#_h68h;m9w1hFiwWTbR3Um4C<5v z>Vi*iJy78q-@7%w=NeajI3lT~sEtT0ypyzcVfcRISRif@tP$h#fXuyE!T0lTV7Y-i z`VJ&wm4E{Q^zunhg&_CL-`SC|*UZfOsg0;5<5 zMj=-wiPd)S>W18`F9X^Q%JB4Nq>zmEXcOn7oa`l>0`;W~{}tY1xO8|xzEL;~)8Pr_ z{|Qi&(XfBJE$fOk-h=lJ;(|9GeCHI>xXcS8=6jfTXdZPB*VSZU93}z=Wxc5xiezlf z&CBbwaaw-zZZ<2=v9$tz){HwcfZbjtwB%6+AJHl!N`Ul5`X(>dK~~Z7KtKd+FzO1o zg&9rd3=u9>>eRyb-Y%!BY*ygTkh;*Kq=^sq6`J5*`` zOP{kWGDr+(v2Thcb??gMLDX7P_BQFw5ZY?h;JfAkJ%CKSu@;dvpn0t3S=bds{O`+e z=V`v=yuWo9K$?RIw6J-RRZMpP#jOXd2BDH>AgoFu)+mZ>>oM4^wc>_QqYljcu@Q8y zWrA}jcR*m0i>kl{9hGSHKf=`2H6WV-;+ttv6$R#GjdDnl`)TfhcKrmN0(X-tl$JJK z5aT|PO*mc4jYLgY?;e%S*zhT_axi~teUcR)xA>r_UK%FqaqDyi4aeHO-DMG)eM*rK z!j9p(*ip1tBqb~9^id3w@!d@*NL`KDh;Y9um5Fm*>XT+E!2QwmczQ-&N!05_ zz_9hrOtk69&BzwJ+pLGm0)k>z>S$f`ua0sX-x~OsZE*Y#Y9M}xB+%!mBu^>eMO2l} z<&i6VjJJsWo}u~8fDgeSim7sR%}__!yUNO*ttbX8yuaKmCS=s%GzHKynhOtJ59a9C zG?+uHX@ghWr=+Z8LRqZ(d-N-t?X{+E2#0)BRtkWyzecV*n`22>DxaJwQ>_uxx!`N0}R0l+IUI4OnW6`Kb@lG5NKr{uhbK2-D z#P#4~#7e1SG@`y`>y8-dNVOfW!h(KrVp`%j#??622$*q^&t{m^SYu4@IJI6W2Q0s1 zSP6XOx-1Q-0SKYGNGl3T2;1LVM<0}m@j?Z7 z!8z3mGgR?#I|KkdZM%(=>QsY3rq?X6%~XLGEY@{m`F=V~CIix|vKRjT?x+WCIozi>#uTF#aay^`d|^#Bx|=e(n*zF1I7Z>Q zX&BFSa~A8p;z7xVBwRQ>MJvsES6yW3%(tuXTWnXwf@`Q?&Io>_38LW=2%8LU-W zZ5lCz=##z|3mJ0slPs@yfNb*>%r=yr7+&A69Z zbPRc&O6hiqv6f1mwxNE9FfF&C0k$O5n-d6k(eZW@x#IG8n_Y#PviEi;d?%>3zVB9W z=vaS9xK3W@UCmlsXW2o7wTK-DzY2pl92HBrX1-kaahi$vedAExWIYuS?1|0(MU=_S zrn_*P!YTgh##KvZHHmF?pMuhO?Mj9l=$`NrFg?b6VZ(aZfp<;L;0w0XzuQ~AeVFvv z6M}9V*yaml;$rcN{d!uJHP`O;Xp3(B-1LrenU6NBr>0fj(10=Zs*>!TuzJBOaeT;B zp`jKYNDK`%7~&p#+YaX~8W6J!KiE{~Rx~`36c)DUr+Lcn_b3;rzJ{F_y za=ES5Zj$@b%^qzvnvU1VF^k%3>Ps3L#y|9Zf9U%fn}WiyF+xg;Ge32{0kxp8(lwIu ztMB+LK)p?Ww?(vPEiD+9*JjEHpD#w6-Sw7ia*Rn%ak97<8cgvn?!s{vm^7d3IY%O3 zg(C|$>$#cRBl)y`J=T9BeEsI}Nf%XVL>Amm{;TCWH`f`5Z=Dx-ZB7eR7Mt}n13q*n zi4Mb8D4t;vdw?)#K|ldF4<3gkSVXBY_b|P|se92@l;3PvX?WTJ1~MXfxk}DETFGg0 z0sdyiH^*-A-@Q-AXz6JnKi`7*^I}USEf;iiDLl#1Y2(HQI>?~VnZayakr~wU`${YlX2PQVqHCI>5A1FT{_bxcM^DRN+i5>dy19_ zJbL3aq%?F`y+JYUhDEOU&BXf_x;mPZP*_chK=l53tC5SRu6-HZj!3Vm4DE{Te-~5R z68C$RKfffawY}p08r;_H?WOn78)2Ut-?Hy(k8Pn~F@#o+A7L9G0E|-`o$i*0^~RK| zzeZYZa%om>V`}kBHb-j`Zusn3uq^{#?Bw=uC{UVO{irIXj$UWc|Ns^)2l#nf{$ICUImJaUcRa2ZI%bz z`~_K;$yH$Kw=xl}C>x@w`v#kJ4{G~W=`LB>?^rdwwr`DkhSr;piT@Nb2ots9W*@x$ zyw!?TwoWUUnx~rnhN-CYG7uL`;;MQ(#&7;wbQA)$_?tifweJ-B*0^a&)6kwAjs>(t zh11y^lo-`z6B3gYn8vepu}D_MYO}KChFI*X<^tKfw&%&6NE^@^LHFyXt{afyu8w@7 z&zk}Ju8PzK+I&1H z9$(umsKQZQ;i#=bsrG)Gqke@((1}qnNWV{j`zNOQY+j`F$D&wfSNO+tvtDP*Dg6zr zGll)yJJseqy!3QVb>NyXr6y3%DP@7chV`MBNAxph+#^+wYAAKCN;w3evw>QhV;aP0 zI?vK&pWn4tKN_f&qUuLx^`osV&rW*tyd*$pAW2>WplU)UIBU5Nk)SU`i#e3q!dO7j z=Hn~-FK*V2kB_nAKDx4Yc9%@PYU}jRakI*(fkPXiMdYW<(fK}&lNTU_Ezj-OAZ314 zT&nQ4El$1oh;3Zm2vgvA#ju?@K57}B;$4IU%hQXY*Vu>lb^C2`R`99VDMFTjei4z3 z4_)^!_3~L;t*)_E@WH9eT*;uKM_CT_3-Lt5$YgT*j9ef34OWY8TO(Xze8<*IkQ~Ga z>!TinfiX^RypF*(sXxQujGy0{upXl(yt>S?Wi)2%bp>1i^?+u}uT?U7N$$w1)yhcy)=A(P26iL!; zzmyIH#m71z9S_uQHiB=*jBMVyF-^?}y#EV*D?)*&sHPeA{fB+VtW?z%wj06Bbsy6G&`Ensx)-oM3ln#3D|JJZg-#Y;XA7h)TqK4E0D?<%l~M{MSXPNKAXg1-oC;`RXE- zfMJAmYfSkWe)@(#@mdc2ch|nh*B9yt>@HAoE-ia&usXr6V~A$g9FI~*MB1l%jfxl9 zTY{$ho>B|IjZ1!r!KbVEYCb!OvgWgkSZh9ph^v68UnwE})5QHU_Utx7 zesU*;KbR0A`9r4C<7F0lj7tOhvs1;0_-#I{!vXeZAzOfM)W|x&EBj~l+FAnb#VDOh zzD&;N#iaH(nG|n3QJNdn`r?6**_%qX!9a<_`@Mm|zO}{?RF4XA+17TqHTpkwj%xXK zrvq|Qfv5{Y6iA%S6n4I+<2kv-FA3Kf@pLXSouk&@rv1OCz+4I=LnyjcLI;sH-WFge}Dn4KcngViZ1G zhJ)%PkZ-fWT{9RX%G4G++}PR7zILnT8q?IwZnyed^S2!~<@TE~vWfg!&$g{a9w75r zSVtZtTkFPc2yEpH8OYX(&z9>BRaYs*bX!01wizd)J{30<7F=4c?@aWGP4hqu%d}}q z1iL2S@*>`>(%rEP65so5o}Rs1%F-*oQVQr^7rV~~+3`K!3G=%*)?^WRWMPaW? zA2-U=b?jMzLz3;z8-^I&RDW0VPp9Z^(4;1N;rJQmvnzvzk8u=TCRUK3^R*p8 zcjbL{X6Sci;w=l9_Tn~c6NhmeD%&wMyXA>$+_2u)H>@X$Taa7UVRE8CF+5F95O}AY zi@Uv4a*FKrc|OIf+Hx~l)W??ei|6^W3^KU(W}{K4yPGgN@~Oe_OH)HPzxL}d3&$EOz?3EziI!cwqPW4uHQCDqwwMG>26QB()GNg(4 ztXW&|)XfGy!=b=$I!WV@zXRUvpfmnh+$z03`}NGn#@~Lj!QVe;iHxxv^uhnauxK>B zfok&~Z1JYcF*d8qy@wzHPHZYX?%YBB?qRb+?d3!WNZY7{is~b6;J;8=jr(d$7%Ez$ zK#dTCBbcs>`TQlab@DXQZ@7iO%Fe1Tg_R#_#NpUiI_!W9E4II%ppF09FrVRF6k5E; z^P-6;&@l(2hKEe3DDYSlmDQttd$E31oSl_f)r5d9xz6TAI&=GuWqdUw)CGFS5<)~g zm_+(LTW4kCdjPN8sB3i7WAfQ5TgHNt)DCp$_U=8XL1Q9u&GOYz4wxawOJK_28-Jk1r$7YL~&b})5WSiF3h>#2p9E`%|4F7VoygCW4S zrc}Gx!#Fs;<0xK=7P76A^FP*VC`f6c^XT?BE4EyeDV&Ho}3la zW^7)w{vE&r61*ZBuU?l53O)4#09G>NoT=CtTRFnDguup!j`PluGpK9dmi!{0W!<)< z1(Zm70fC4WsU-pwhYgRCHDW8;f^CHe&@ItCT}SYm6LqZ(^Sa3N0@uTfOf9TPI4zFP z8nk7%Cd{OaQ{?}uOAMfzqIt=>tdjhESzw@8+Ihex=F``a6?~qRnU$4pSv+UP$+wh- zB;XII@JCF7KVlL{&G#a|uX+2jlA*J3{_7T;gZe*@WV@{k2FZ^5$*f4w+mN7#c;quc zBcC(>(Ek$qS-x(@gxJQDp+0;P-JYfddrVFw@Lkf>XJP-1`jWsSTjFzhkAIp5PnrB8R{sE36mc0ZA@aWh(f)*bW+x=yo2}%rBX{c9P?pS?y z^`Y{u#ty#DJ@AY8H-(1eX(eh(;JM_84c-eizl&99gi3&&r_1wxuRbr^W>jt{vXUg= zZ`DoBLZLbF3oE7a<$1Cy%G{5STvb;$9oKJDxz))ahvJ#^I>sh!QwT%5 zP`2eg+J!+6s)#U+#%}ga#1Eow01n$Hv-}Omj6L2nqD1b~y6!#tgp@$=&!m`LJ#vnv z<^*xS>>oaae?zQb8O~^I-$7|yQ+lqgNU&A;3j?y(cJwVCWv*$gMaXHG2>KaEu#fMG zD~bynlpH{y=%HeGnuM|Rkj2QXn)sPE!d=`6)f!Wt8nIHoeD(sFmSXv`*ub_0F0LZ( zrC2^j;iZRKLf$neKlGgK_XC+x<(Qg>AkwE2mf;!}8a(XX4HBdt|3imzR9406+jX%@ z&r=B?p8WX#ZL;+hwA^+8e(3q+xyG3Hja#7d2a@Ub9<_|`lWNvh1fNgJRl2m!8Dm-w zl`0WgVcAe6)G+UJsU)lkM_9$hpjh;@6mVMaVK=QqzSllxo|a=h2z|8Hg#v3APuQDI zZMZ!sN>~N{Wh8SCaM7c^22y63I$vPIDfIij%;5~p3ZQ9A;E+kDdYJ=DiAH=Rhgc1S zw7!ige9+J4_BO$hj_JQc*)p}lFtK+F-~`lxlK!(H#5gKrlOmW$WSiiPyPCFY&+DkK zwS|^%2=`_4k;L8V_lS$|F=!Y^^n#-AcD@0r^>jHhIo2uX#7IGy%)DhvuuG_Ea z^q9YjiR}(1w{1Lr)PBY;idP;06y3Ce6aForG0ZNm@X&Kb{F^K_bB6mtyLKD2Rx-I}}Yx+_-&odi|g7z_w8 zNtvi{>u3@K%zcVc&R6L=UDT>eD%nY&Dq86(ottm{!j|gTDCQNj%FSYtuCH3lxnd=& zSPe1=g^2HF5k}Ikpr+`uSieq+CFO&F-OcIjvdTAPc-{AB4w0+*E)rDZ4&T2rEUs$N zSD_{PlU$MO6Rx>tqc*&E2tSjinvOO{#;^Yv91aO%85v7nHB_mARh0r-yVo^7nN5+Q-;n z{OB8sG~H$?qcu0$be`V?$fQ~4k%1%zPHP7G*q3$97kSm*=qJem395Fe6r;qz?J8v1 zLub!-#B~YLfasOzNNqquDT{fAYVLmT>GS8$o{tmUawA-7;Hsw})MHl~PACxel;dyL>h%Od}4gRaRvI2%1S^|Zm} zKeoI#Okl?jEG~agM|G3AV|kw`O#A|NlP`%_;J_PKv#CxEt_1lWUr4y#hEqbBSEF$H;!j3nLk2Kv3PiH7(Yj%eHXxB1JQiNETMY}JjkOiT;mY)T|+x=V4NO> ziL%fS6E(HBx2YMp-<)jnxp25yl>wWD5?tf-MMt|knF-Z)c-MjHklJ!nf;I=FzHyj# zeE8K#4_2{vs^1;TRecNGUWVR|Iwo*SKYiB!@V0I0S?_ zkiv&Fd0U!r2mN(FUNhh${*=)oXTY_GBGeqAS3@N?=!^4K-@``*aifh^|A+>S~@H?_K7z>SCN69jx9yOfItg{GyUyd-jd# zDqk+M^`2Z{@PEB}$=LNDjmXcw6d7I@ zB&&ARfFnEEBz{Dv3$NxP5sYAefA{<=>y<6^Ge&5N3HAGN{dYj_=W{qx=&=N8!HnNt zH+5WhG0-EaTgQs6txH0|_V(S@*|?#B@j>=Pj+ctl&DOfuRAd9uED1StPm!%Cyq*;SXgQQ9=%HES?mC!Zv4eRvyp$)rI1=6p_c6DO}v z2}zvRtaAJ+KhcZJfG72Jzi-4C>y0r>kK4fzXe-8xOxgyrV|eZ+qw!<yO2HffjFNy)9S1WXX)VXm7EUY%*wiJ^B)dr8abvIOFQzPlQ6 zxzK!kw{tI8A2KZ#i_KC{LW0{7wK?%%P`w}U=KkqFzkT}n<Lu?^eU75de))kZ*!?zjRP;vu?saSY`P0Wwzx)SJ$_QQK zuoCpDc|0QVX?%D?54jC-;ByP^yGNbNBQ^#bVHCPrc9o#!SI5@yC z6it0rnN52Q01uNx*Oa9uw5!d0PI^Rq-Qv_kWyP{ZJ5Y93$`fV5kFQ9FiMF1otZUQH zIvlUj0`Xwwbh9TH_izMEnLKdED-;K%=`(rW!={ zl+s=y*#@kel$68W8!XGQqf<3MHUVeQe4btaw?!;TR8!U{jFhSF`*2uSyt*KN2F}ox z6iZUeU!=>~JR=`;@@`G%nCJY8K{lh3jq451Qrl2EPWG6-?Zs4xvbDo7R8Omqr7;l1 zRy6jtyX(T$jltB_`yGj~(?7Li=*GLmRt?xYtrMBZ3^+mdJjT6_D(F?VPE-f+p#M?GW%$!1*jg-()tR{(()Yj z0{JqT6`RRCOa8Sfs+J)9_wKcg_~o-F&&EmmrhozVR|Sf!IiQ@YWRZfrnircAHPSQ8 zwDY4F6Om#QH@jN(QM40gt2454mz~qQo5c9(?W3p%kDTHmV_E$0QA)(qQ{&-i z9l_D6j>Q152w$FmQ_v=9Ny%WLjj{DJ46Z1WKmbvMN-I)jG)5(7Eo2SVYM#PoOtIoH zNy~Dxz%8AXoX=21kDGzGDc-aIiS5|OVN6T_*m4`O6an%R(h(x@(oqYnILB|rvR4=u zzEC7)FWV%3FzcPduNR>%y;M++^rFN)oRa2yxQ)R2iRhVf!p|+%@5*nsqpDUr(#`1qC7RK zE4X+qH12vg@Q&mcpY3fyyn|K_VexTsQWuD8YP4o*v<^|Db@yy+=Go(@R~9i^-*$EZyj_!H ztS~^IgH6Y?m1_7&?ap{+JERc=xiqsg$W{Fsw-49Z7KD!O=2uga8HWeTgJrmq%c8eI@r1*4n3R6d!WGITtEFru5~dd^TG7I}=^ z1x0gYSR@-n2~K2jNfCmnL0+a{d+L1a&QHoyLJIZWhx5(;O?esp99cXuGfk#W<2LG= z#*Ng~dkpWB-d=JyDQio_A4X`1xYs*{FL*1w*J})TFDzDc65*xiCM33IOHIXku&(b` z1+X_RG@XM{f)+yxDRL-Zc(IkqgrEtS7Mz83e61KTn(h7KGcllln* z&e3#w+Sp&uZ%9PSFBaJbi*3FhEN80=?M@nSSzC%ZAvv;mTtjRTho4-gWg?d#XPbBy zK=OAp=-2iMyktCWHcqOu>!m614`VASq;0mpgjG~F%d18}Jff2-*p5zcNMj2|P7TLS zCBmjM>>xiN&3pVC89QJ;p1SyBIJ8gU9fmFJQ(J0eJ~DvZ{T_>$l|PGmb3Fp_+k&YzMDE;G0pT0)sO1&e#Ka5pWEHy&w? z)ra4pRl3jWjz9N*ibA9@pISslFu%gxek^5u8Xce6VfgS0T?4Mg?H3R7F|;kfxeMI!qhyx9 z@r1kPFx=2Hu1@Jo|M&qbvM_;*G!HFtMj-^vK7vg}*Y+?=DEyQwG}9_CvaCw>0YD|* z$mbMuRq2lX+B5!49vsM?tcy!84G(yQYk;ainxxC{5@} z_CY5XbnR#E@0HhlWY1`ywTkOFFnoMKEZ(dE<>V2JX-w&E_NN#5yb-Fs>90cvHDA6= z&*6nOAh6#yqJZj(#ipV%c18U*T#?qH(W8DFu85II%jQtkTW|6mPBI=>_ITc%5ZOjz zbE@qzKh@|3J*-wR&9Wb_{c}2E)__ zsaM@how?j-*(u~5ty!1H>nTe<)U(+OT4iC2c*$$<*6(0rKkS7Dt~4_RI+Fv%CeJ~%lzjWAt$ zW*E&K6+ipwJU<5%Lulh+)N>Bi{}?`a4x>nDGe>s6;s1{_r5FU?qEVZ~AgD{?fPF(X z&I|@2Kx)bS?8Sm+c7#Ci zVV~Vu13ax(XLaO#4(iN1_86lG8W7B3PhhNkeBE!nKc;<-yu6P0e7vXr+LPT2wI4Ma z#SKCM-MMo&X1gEAdi~};Dj1=&_^6O3atWQoNSo{w(~?(Y={lu}zcN{(J^7QMWWF`x zu@Eb_7YS024+ZA81HBv`9Q?<_WL3av&DQ(GahGF?tPO85=IZhsXfOTmK2qZau>bi| zlk7E|)qNq>`nRIk^^0_t<2?*j>M}-ofTW=;G=^ik1YY5+SXTRI=^~$Bp~!HNt#dfx zcMTk15;TNRm%iu>D$wdI4ST#4^jmRI@UdqiQYot&CMU);yzdD|G#WWHJNHXg+0EFY+p zrmwSXwO?TB-9ggHE!BL;cmWoP)kq_}bVS2n4uLjIEd7X5*|beRfQ&p&Uh0{Eu=4t0 z2;-v|6-M_Ta5A&4B{8!NJdsE;tdj|CVB4c9;z{ERJH^Q{M#}FiN*n#l>oCq__)0^aw^D^a+61+$teVD_XN3BbSzkXl{$3f~?gGaaRN!7m0)T#FWeI2a&mVsLaqyUHnU|ChqIZa6)VUG;?~D=$ zeM8j{Wo&JpVE0eigSPFRq5{RrWHWy5fAm{_$1JKc6nok$}=>uxus5GvdOC zli}YdEUd&VyoiKxL>9mf^L#Za()Elo3;>;`l+?s%-=hKVMOvBx4MMvDpwij&aj{sy z3ibE+M*WMIUw^eXR7o4Fa3GKv=wD}7+n*$rMA~O0<(88hOxRFWdj1WFFr%vYZUqGP zaf%tf=`1|?@*jwrIFBkxn0WihKM|S5+(7m14XqJO=W#KMTHfMAs=dFBDIrQaeGS6L zw1W?JfZ^bqtN=G5f%K+iV^ep=386^6i{3>(>ygm2?PgR@+_MlL5zk{q2yh?!1$C>; z;r)0ae0H5l*KyweYx@{X=W(>p+pNoC{jJzo;0EmO?S7pp-Q42bZ&@9Dx=b1muG~j< zoxBl2w$bF!Y_A<2`21q9j2h_boYKG>C1t^P6*^EA{E3))gJy_$V-rSO+lob0X%}3F zj~rW$rJ7DUfgh3g5^(V=c1j|(Mz@92HQ5YL_^ylmtoonq3h+FIv4UkqW7xjKD>D-= zmXXj#uuQk1(1HOWF4*QRSr?a`>fs5gr_dhF$Oi%zG=sB$UR*jBz{a$jAG^(0jUD?o z0sK@8mWtcX%`F6EE+IBl` zxxK}zs7Emo=JIqlNImZ2NfmmGBdMLCcbS2j#T|QV54w;j-_wq-cPss_dC{^`dv-1%uRJulI*@T7kUJjjB+#!%qDCO@Qk`^c)?m6FgJE*mu~9-{ z9_DPYGs}ig=yK3~lz2hz8OxZTl=*DA%M}ar-U+zE3?_L{zqcme*FLLiL^pI^@wwia zwOgpqt;v}nxP&!n^&4q>b~?yvko9418Q!iZn?~3j4|G4_-DzU0UT2#^et!Z*fS;DQ^l{2$1WB1OB8hm~@!-hSZr6)3g^eYwef zWhfSLc(uts{)*hny3f|w0A`_F5&RwBq3G%Z(0qa^Y#UsM9yAEw81Fk+I!tY-~OPMqYQ*O-Wv{ymE~)Td@L$_F``N zgGFz_vDO^@gw#0N_fz+y**93>UVv;p?($?O#> z|JEG%3U81Z_R*U^=gnT`Fx9ZqlxqzU7?E*G+w7l*D@NNXa+hUp zH|ZgAq2~#*H%uP*Zlzls{bBo{sSgOl zDtv^)k#N+_S3>S>6zib<_KR&3^!*oA{CXoOv(TwTBWxroO735C;f!&k6Li5b5Fsg>8DWV@(#0aVqQF3@IE=4T%eMl@W+ z{dx^S*eqiyIgl>==-FovgLY`Z0~FOowr1nZX7bxQGAu{$_#`>78{yZ+ zXE=`g{9`{kOeD0z!$!k(XlvEQr*ol^v`A6btR?Mg$85ldj@c&E1$%ioUUTwg!{$ zU>HSUC0z5vdiG}f%C0hW9Ncu=(#ilsf9V!$O|k02Cqn4--ll7&LRXHuk5OCd<-Qzv`UR(2$JMZ{i`{(dhDT6%|NGwQSSkRR27`t#S^AO zg>q?5Boxj?8&!G&dl3$Mg>n;&66Gd0q~g$t+2X`|X);fCOqagLdI{6qd)`U_7*FtF zI#sU#pkoEqP#G01WW2pPVg%58cDekPk^)s%{rXJ8cq0mlu{zwrfvaiilJ+iWp5hM# zYP?=i9flF%Mo*rPbG(HHX(HXJ!ctCv0zSML0aw3`|sX^N&BqbQo;jI!Rqeoe{6MF$SP2wTwmLonTm#O*a}*6Y<+ z=nhfu*4aMB+uWp--yx$(BsyHE2uq7}7T0>}D5YU7O7Kg2Vmb9_lv8UkanXMYWLh$- zjOr+ljaOSXVd_;o94%v-T_XBjGa81Mu-7%2%{x}Era}(#F?KZ#+ z@4CI7-)-GxN1FklL*4y~4@2XPMMr|9qd~4@2t}EK3~c;G?#Ho zBibl4AodFt{Lft-!B0lIJ+hGZO;~T&MS1>&=>@q!lFx*9r{H7dgYY9BR?uSBo%;_C z{2`>-0Ue7R{xwi!Qp~OrYK5bkn7-LlGA_zydyg7<#3|adnh>eCB{WB%7drDL^}>H! z9ZjlbqW;*&4wGU@E@u4wHR@rjAtt%3#TUbDMF}K_sp*ArET3_EF@jzl1|J>@RIKwOkDnDk& z=t{c0eG}Zt;}$c}q-DkZx0c2OCL<*Zh`lHU1%UjJ4yP2@&YMFjv157reX0527;f5c zqf1~bab~*?^x3viJ6E8Z;pndeiSI63{GnI{wTw4BmM(JubzY^mU>}gvmK*yCvCdh9J;meL> z@DQze7Abxb1AJsxQj=$T0CrJ~u*&)W3EXT%;VOc$PDm@DM z>Tp}Sn8!WHbH7T%u44@G&xWR&?k_T!GNRR4u|Y)xtyyzkxG?)p_DhBwmM3aI09*`G z6gdhypEW%A4-L=y5}0CP1WDg)!fRu1pj}u3B^ggP6PRseM{t3sUUK|_ubdW}{Zp_C zdpE3ycX1$0I~o3MZgecDpzMeK!)WWOs;4M!65gqsGF$Wh$YzO0`=mB>je@O<&VV(L z>EFsF{a1`+BjL6BKnnQs3CBaK?L9dXT-oncGYU8ou$SP%oW}tsVBX*A1FE-uvviwc z@RoHx^p~9JX2nI4x zN#byBk<3eW=Eg9iCi8(AJZ7HKmx@6PNKo<>kA$qnBBj`1%kRl)6^xuF?a~J49W5XKAA4`x-Nub13V*-9qOHj;$%vFCImsll zr96ozvy*d^$y__Tn_Ndnhay{2#}u2lyJbm}$$x)PRR9`j6keL7>?G?xGqKo>mjX~I zR2AyQj?Iq}FEmNZ>(fPxie!EareFbM*jj9b##}wKVvy~|{U)b8VeL)eBpo=(WM|4v zFDzy>2Vl@=)(Fc3&0)l0H|jQ`drj^x0X&}VwEX6q19Y4he;i6rd21(7ept>$JUAkD zx!ZGY+l%dvQ*P-q2vF$mWq1&C5yTVd4)o{kiH0AZ>LG~lPZ1`xmwW2zZ;L6?tz{}& zfC^YCgeH(4!bqhA_e~gcs*`jB03a!A`9)Pt?}x?MU@NHJ*YjFO8wg)-bGe(26Z}Kv z0uhecG}@n87oO;DYSZt^*bp@Yhp;hQX&vf>_z)JP+9Qeuxjn(H?q1ci(>1~#@^2qQ zii3XB<~C%kn5F%ZV@=giDGt`+U3JnZ7ZB3oOfO-pU|G~z=WV{=0yZP&se$bR!rT$d z5$ML`c_&Rt-9n^iXn9{W^H7$n9>&tP^0U}cuxGR`y355x#l{;M&;aNxFr+6VbM*p@ z;~=}}w7O?O8LAi?v!+_ellqZ+c*|GUc{5dSBqV4({D`!v=2;F!^p*2V1632qisJ=t z6BEU{l8gXFK)Syo@gLYOO;02Ocot0cW^p7>_SMC@@mouP!Te=TPKX{8S4>F)dn6 z?9e{T#K~lxTchrY-_1vDPacKdC%2nrK8b{`H_fIDeT6<@Y&Ai`O?l!iTc7>@#bV)Y zCbp%0-3Y{yJc*d;qRQp6y`NXvOuy5SiF)SYzbsVW#(2G_HOQ>+BvUCzOX=&30z~Irpy}gRW%$dMZ>e>R?PsbN781o?p|!02!_J)*q zHf;m`*p}Sm5#gqfky_s~flKqr5W;ApPk6MnpQ?HYj>$jK@C4KXy%uc?Cj9UC0vzY^ zFY@;CopPTFh35W!Ie-bYbI)ep*hg%EyH?keSCFl!OFsWdEy44jLY4v7kf zT!q7su@Tz^U-YYQsAypOz$?pU{Bt)1lNnSPu0fO$jv$hdJJm69L$s$nM1kEcNQxSY zMPrz)tW;p#7}=e0dMazlYhDebaxq!SnD|m3eR4#k7%aHM`bg84RyrOEyv9%EQaSF4 z8GNN&SWkW0GC5_wJ6(8sWtI!3n~#SaPK%$N0CW(JPd-LCX4iAUq<;}ph6XgB0gBO% zB%q<)bRf|{@pS4^q(^kvR+LtSmCXv?3omr#q73K|3#pms%EmS=t_6|_b@n~Oqc0LZ z7ul!s@^X>KjIG-Viq5Ul?ZGsa6-^_?DU7SWjKnOa%TAxnEJhX<)8F4r!xB%zqGHS@ zfo2zMq8glp&@Uw|wXlt^ADaA$WBa_c?KSz9= z>=lkh;f2SJl5goTYaN+3EiQfnGQ-oy(ke(;o)D#8kKb+a-quA z3q03DPMiUJ^QW+0$53xQYSfxjI`-E|A)HhaD@tA^?M}`tb{g-LAE9lC5aq~51S1-P zAFL>j#!u!8OOznXDDjoZb3PA-hJ^=^Roj1rEELTO0+|=nRhKN5q- zEN>1g!RnrCLg2=KD$Mjaxp9wif9$`U?N(UkU@_RE!VO%-rS@Q+Bg!b$GA*TJing#e zLov>rS0f#NBfJPBm%Vg5@5%;hbki&M)+=EluxeYX%>e`5@0Ks)D zGbg9ONlQ9m5=D7sh?&q4(!`#S$jvH2iv+h-{#TLT+LXZ=LvjkpUI{yXj|f4VF$p7( zENy@B-K8g(He7pvHhr1HYc)ucT63^4CAhhevA>NjV%lPBdpyZzAY_I{V$o*ew~ z8SYPyC^mMC+fL-E`?I+aRiu1_aqz)yTBky({MqFhRj*8iouKJmTgw}WdZ7G49(f`( znvVpW35LLiAyQ8gTCb~B_KP~Hdgi!}8E+ZsvO9i9rJ7uW=~b44J>|1*bt@9j3C*k= zm-UHKrXTj9XPT09jB5_ld4*m?&bZ$(dxRnN=y5ye@dHn<+yGD|jaO-}5K|-4>4D3&(wy@nLg(<3l=!w)bsE$D%8U;yvZv7 z66dcLA-u4fqj@}OM8)R%h z+9xy4@|Nc^-T`ZiB4dQD?PzBuTYI<;Uz@65-4b7G>RdZ~E%Q$GFv+^s?&gz(r)I~t z>npUrJytkX0sMzzg>AhoaAePD!Y$K2n_JrkFZ^~{VHnXnVTB)ZdgZ=wB8d8)cgEfs zv;UqIev5PUo)u=F2rC?1fd^oPp}+0e*T3g;z0ZT`fghIB#cTPM;4-q6I45$SdkR^R zgS(6qsaY5L>_9u?aAjm~FTeTjzmKwm{ny$-vzflmvp?_c$C&Ien_Mijm(z=BRftnd zV4z@_aa@oa_2%LtuOO%MewEx-dX; zrjjI(LG$WSJTS2Tr&ceee@|tD?2zikoF>)LodiOZh!LGcslv# z_>0N^9WQULS5upl?HY}si}E8KOx72C+4OfMCeN21p<@vxgBVzDhTUj2@MzA_AJv+) zSNE$NnF4W2ivyW;%3eAWnTa!i}oAzM?xhpqUvJB`xrby;;;Z^Kv>WMUo^ zf6Ubof2tGH+>l)h0j0JLFWzryc0DB84B=&~fXqlXUZ z&`OB2Jhrsqkn3<}rEq*`JGCq|m6*85<)-;KTUH`9PoOPQ8bTUm)kd+*-UxJc`Vf=f z6Iq}JMs_d;+o@_Mb9t&YnD7-nRKyl9^7;SjLi{GLnqoFxI6bQUafN)waAe2QCGI2p zP(zk|j-xu#yFq1*HE$pNA=qetlbWllfITReu3ANB?20o=4y+6+=)7K{&0VI=cI;{G z8N0RdQgteXWGX2vje-R;_WhC#^dJXE`kQ_yK%Ha^^ap!S)=!S1dugD9(BVMkrpw|Q zQqgV%ETlUZ;wgs1pH@CGB^IF{9&=T^W^)g{D!ad@Rq+G6{Z)Aen@Jxqi*G8-9!>Q%=#w2E$XA8uvRV#%@1iRIDULOPpz!Oen9m`?O*l6nPQ;zZ z+*ZuQu8|=ZT7@(}&!>yo%|c*9S|T~~fW9r}%@v-{qU}j%Lw}QBpmEGEaVO%$g)b^&#TsgI%yD-KNv%?3R$Z)sYA~pIi zL!g^8LH;d=ZZ|eFg!g5gsw0{0L(yV?yNTxim^sAt+^(A=Tq;ZRJ5%JVVmYaqfN1&M zsWa&hoPB)W=9J)FD3u!H9VH--#1u$~3z^@k!LY#c;UKXw&t*d>7NM8?ZJ5o^j%I7c z5fyv@z(=w4#AdU@;HfWdEWD94$TA~@u6rf1wHzx;xa<5jU*0gZSIv6wG$5AZ9;nt! znUqx!3g~a(S8(}=Vpd*U)Oqv6Q6>%{`uj)xTkWt&Bxjm!v_SK8x+i-G>DM|1_mB;Fi8 zegO$;8@FsPjUK7vXsiwga|~OsAc`TcG`eu}!)#_`u~oAQ4ZOj^=vieL7|Db_SQ@jm zy{sWj-^sjusdqNC_%)2}NDUF+H2L+B#xyC`yoGHK2D1~&3iEcCo)~Fo-MN)E#6DkU z5Y5UTw|_DbvZc0-czYGN+D+P}4ih7`v+}+$OPJB?Jfox{NuvaOW)WKFVCf|%bCJW> znltR4QtUV>RD7(c5}S;swJ7DTh%I}yBA165^)Dc5ce!BY9b+TF?&+e+r*pxZU2$%- zYvhw7Xn3`ONddh4&U)G8%#}1|#KXqv*%7dRHJx1AYHeI}gH1o(Ug~D?MjAtUbDU6=5>M+nt${nLznTsl~ zueiIU4fExC#Z(kM1I8opPN1~-p?2OGiqXb2>9k3%4yv2?FJubI5j~ZsDpDPNkm|f@ zlw-C4k>TlMqa5ww^bNg>K6uu^NpZXrYKLe*XTo-}8Kjpy-obo->dOSaoD6pT9m0Q} zHF%X!i#o?U2mAYcTZi$6u(5gZrvDIjPM$p$72}{jZsuv@F-8E8OaPH5ppb*EB8`Xw z)ybf1#|)2mjW4#f$QKW=%J3LkT)4ve`_1)vUe$s)4!YANP+_vf&uXe5pPSet>rQ)S zL3^-cBU=hqg^|iTWr&0*|;UG>I76UpUcs69ZbrXWAFaDXoAh|2-=f!$0*mt6&w0W;_ zd035$M|3e2D`aA~4W>dZ>O0Y`%oaj|{Ys7-X2N%Y*pw?@dTD)?SJCo36@SQ^tf%;{ z#WGi6GUBgPus`yEu68q(U*#Mz6gyv_!qgMo_s)>L^e4v^BdtfM)ZRx(Z#@7;*Vu$1|`p|D#dG5aH!4EbXvcG2>CAM$LNf1 zfqy0;@C%!}Rj;?oK|gFy>+rn0u^t%vvaX^&t`}d5HXHr z3C29r>l$J@0f;p=Wc9Z-lX5!cpnd*?Ly(WMF62l2AdZZ`w`DA%Ef*nN9R;lWe>a6| zR)ipiT*=~R8RDLpimy#K0C-`rbOF>iB5|-aLoqp89#U3HQK%+W7O^I`DbR?)=J@Z7Yh`JdS7hW*-k_V%nl$NrA2UG;tFOS_8&FsoQ zXKfcklKG~*RhGgrkTJv?PGFWD*|RESVKJ>KAyTLjEZ=PemnJFBirP8wtkp=WX(5lf zII$GjyR6794i6&O6V{}Y#>3ht``iYBMh#l^NMEO{*BmG_@&-8SO1+T3No`7U{vV9Ljl-1!nP>HhXnXOMkp~sKUJIukJ%jXMa9UF zQ7SyZVKfdMfr52+jz<(B_fW$he4J%msJgeNWM`EWUc7osVBuWfT~%k~D8o;-9KfQS zIteUzJ`jR9T72MQdO6O@uQ*d;E-((1XU~eYW`%lBn-rxT5LGacWn?p7=F7aAHhBhR zRcK*nXZLUn@&y!f&S`W$r6~N}PllH+Meymk=^~RUm{Av4IVyAyzAl!JucvpRi)T^f zIR*lV#2mu1)4SxdVh`7uFXQh9ao~*jZ_=&fnHw3<%NpMv0m2Rd*XdedIt)F5r)d@paV z^NQ#;ci3CkEBhH-sXU{;)uL2k`Ika5L5)|HN8Yt^9MEE6qJv{O<1)j6pS|>it(XOJ zEU3^GbdzRCZsDjE@Zi?mHKK6{s!tX8uBVG2%u8CcS12@~l%o^~G8C zM15(>QZVhSkpY15T16+5WC)p{qRtk1T~ls_Cm4S?10^j)CC0C^ZZ?6K2pJP`7aolc zI|G53E*eX45Nmf>v}JBR8@GVy%(YQHjj~(VrjiNQRbjA_cqX7_5Swhm=6tt z)Qraqcz$E#%=#6D0z*nYb;3TuHk_i0@3v^J%9}=NHC@G>U@k*C9MKl(D;a-6c-Fer zTitsCYKUb5@ zL@~~DJnZXO-yRdTSxh_%Va6I*f#Z&%1^LLLtE}Y|zm?mL$ZEzN=6!KAWUPgbDD9G3 zaRAokBHvq-m&3tVFJ652;wbyEya9i1!VB84qAO4~Sv_Sx4PcBv4UoCI1!im72+j-6 zq;i>)gspgxJ&)?M;c8JdLpW1Ay=m@-R7m2VXXcS9 zUhrIvilHStc-5ThJ{q6F_)JWhKL}!)C8%zzT=Tq}x4Pv{+_k>Wfyv%F+#+qLBsQwBe18qM< zmzg&n78cfs6d-y~9v)kKNi)#%WK8TZ;j}S2856bhtVIsJ2@(s}X0XFv1lcW#XK&%Kc%h$y+ z$ICSgC5a~tnJrnRU6&kmnb~U{=)|XiYraf&P}sCM|IJophofI`+QZao%<8m&>EHgICpR(?VdG~fUFFcZ4=C_29 zbYaWz+x=!G`mpvR%yi9d${zbpb9Jzp^wBG!+;_`jpCM_wHPkYVT=3qjf>Y>>Y^x!i zw=D9^Ekjn=szmrGmyJ-Ktk~II`66nD&QYW|>M$Ldhq$jsd)}!aYH;YOI}lzCg}>vv zDA51*_dewOYGk&RK)5Cx0r1tS{bhqQ*a&DgiOV7&5P!3QHO&V)5#X#A(^>xCg&>KX zo~sSfIhI0jq{VH44dg)OGKG6}3=cW2v!cmv#Xk~mf6ilY5~}Osa+%f5{esWA36JG^ zghiFP8Rl()TidT7aFM*VwTnh=ee7VuW+x!Ufr@c?nUqUjV4yr_u{ybk`pM^iTe$Ur)BVLwJUQLfe^=%5G7}u~%{6d~FkPs)B3?~0Xk!`+MBcht z%;Apz)h|lT_d_ey{s@-$dwYAQp*a21qmRw$f?9MiCcR@$G}^iu(Q#KG#h;8cxv?<&sRD;j+O%PN~5QfHXX z(NQ2-S3G{_VCnFH0D+?`>SXC2V~*y+R_Q*c^ch~Q>PhY%)mu8E&vY`&_Kcn-ozW)A zlws!W8z1J~D{TjN8uC!H$ptwo%Q@|me((7qr@HI!>}#gGs9v_iJM68n9viV7TjYKF zvMjt~(IzYk?);QdC-ElOS`_XuGDVmW9}2llhkxg6gYnpi;n*q)B{M8Nu_u8+>4`ml z&Lk0WLX5+U@|F*y&t8dCePZ@q1lGY7VX6H-LHl|&kBC^J%lm{#iF}7kX9WmP5>@Rjv6&0klG9sa*^ zO{sPvdpcj?&{N@ns{k|{Oe=^BK;Edll}in5kz(`2@>aQ6=*lop;D%i{EE3W=@HvBs z#_@l|OTDH~SCc~T9-7A3{X7`0QBO+*W;b|^ERY`a-cVrspyLoxArqX)Ou;PLymEQ21vSDS1&Fyz~_qgTCe1i&{Z*aaM_V%#CRnF z1O_Ziczjoi3u*QYbPq4HikgDjEb}Z*b&l3a#J-=xbZ;5&p{|RQ$B-KD9O|yF1 zgvQn!7EON5FnH==O}`D(=YZw21^4v|tiNSaX?gO(c+r?&er=SIKL%eJqBM|WQ(%yDDRBp<2ig~3=3s_WLvJvMVHngT_z1AAD52m zp9v)Op9mDmhB{JP6lDq>@o50{I|p7~M`X-yJ6f}>>I0J#iUxyql`k$+SAGgDhd&yY zJqE8!X-9&~Ybz2~XNaQ&xa%aiZP^YNh)vq7^kD{m03-0q^aouC`Di%!7TQ<|83scP z)dTbaWvvH9qqm1R9>JYM#J6N^uLm3p5NC$MdC-wFSnR!-E{3g%wjuEP+*aPSES6E4 z7ldV+_{AP@AaEez;Jt$0^=hP=7lc$`*!8p#IMy}ymlFbzyB~jb98?9>I4|e6R)S2J zksy94z6}|_R#EMuEW2jMHb+1Oda*!M#bSZPVf|Npi|T>Vwm6`+rxe->eLjYg6pK)i z$3?Mx?a;*!sK~LuZ$)|;9H+`GP+g|KE`=Dw31z8x55 zf?{POD$3P&5V0z6Yq=v)EA|U9w>n_!fYi-kY{$%t9a=|5))6hnBJ0?sKq(uc=Bw(I z6pr4r2lu_5p#t`OFiQnamM9btoEx&;K&Pwfb*49~gR7XJR}^{aJ(_z(Z&n==EkQz0 zdR5+fh?KjP$6*v6jvEA+ogEyEFrxc1QYE)72nNqDv7kdJx`Wzh05%m2O#=^d>`+}0 z0r8Kd{3&7<#>8r>Hk1|iC)g5DuEL%2i8?|?Z@fm7;v6RSAv<=BlC1IsJ{ z5mY&^$RZ2vXdY=!jMPH>L)H#ITX+TMW7ZA?DY-qZygISgjpUd*7jOZtlRAZ(|9g#=tf{@fy+_k0EdfV*e z06IFM4kwe5&78_?G}P+YNkEe(H4?}Q;Z38m*>o|xS-`ZvDXQiMw~{C&PE-U}TKu_8 zQa%Hj&$LsNF0~{Ix;HSQrEU$0!@e3Y3aoPM%;X@RA{1cRaWj;*pYSg2ct@<_WwD%` zm*Ob6#*4I}j+%uN_NAczNPaMklxTi+_tcxlRL%>;@N^3dy!lBZN;0O-rLa_=GuVl+$mDE4KsHd$1Pg`rKL5&_5ontC+K^>~itpOiy zGHPfS@NkA1fC~^cB4BV%D4u4gBq>Hbv7OPNS3I^7bx}B&SXt8siK6H04IipZS-7~X z1yA}s4735^ZDN-s5#x?6+r_*MTahK*9*H3xVSJzMMUuhX7${lF*l}5YSJbR}5leHK z6J$gr6#~zD``@mOc~8hAzk40~uYYBaS`7tt@`c~?xRsok_-pW}e4(NYe-R2NY3IUI z6h2kA3_2^K^|3ClSLhs}!fB2G3NF}}8Yuv9a?C;yQoHGk1*!q6a9<`}^>DY)p|m>O z4X&?z)NL|;1&;Fw6C89((0S%-fHJ!S?5(F z;LX8t-sUkv^xX{bHEwAPSI~BIic@q+$&oE|I6eW4S7=lmc93-^voMC2lpp-UP;wVi z0m<~CzHFfrIHj2$@bwm&S4@+qi_DNZWx*|U`Y!sUOL9*Bqz`FbKnzboF{hz^W;d*u z3-2eza&^-{U#g)%hrBGqMTLDNqr#5t(0ulWtR1q^WftvQ2mRhPq~2A;6W%&!Iw)va zBTK9UU%#i1-q2|3+ToFvN2|4O1GdI5$!S|N*Lwna7lf}h>f%8#2sdNT^+=L(uLw4` z&zHYAp_<|GYWtO%TTpx?_^thd^lBc~bK^4nK=7RP9F_@hoCMp6vo?CFwDV@C%*jwR zoB_-xhAqsjyOyvWwhnHgbJP3_U+8&+A8@ZE8A%0sFmZ zha{(}B8zI-9A$c9aEGWGD;iL2%Fr?eW{|-U%AoxPxkiADjC)<)RI}Wf>_X|#J*V%77wuJi<|19VRZ{) zg!vu@5^BWufC*b*1c)ZQI3SP*(l)oZLsyV{27w@*x%ix2yM`9N2~+Xv6K71@ExURp z^?3IJ-gzh+=hHpxS=xvqaaDhk;70o-c^t3v`+0f0Odm-H8d-l2_WRKOHqPW_3Fi2| zsVe@-rBjfqa55!XOz#DXw5mYmdnFGnomYvwWux);Z@iJ_KM!r5q+CxxFf8U<#QX zXjPm}DGsR^E);$G~&><>MHXDAY^xe1dBno zkP6W7j4*J3?-t6gN?@)1RP63ZHblv6e*g=MeLk@fJok3-AF+6%YSRGUBXy{!)rUu; z$AOV8;Qlv7-6(5&{bCPnFsH>*?F`Zyj$&!ZRm08n64M7NBYyh1*OdRcTB+>pxJkbM z%V4C2#argD)ikICzJD>0g)l~qOhE2!A0Dyfdv~Z%5iQd?eIw|)A1GDn-mdh0x$@WX z$Ip{CVxk!$Wq&W{-cYA$Tg{KW^pC>HdRFE6@`nKSa^-*Ym!B7Je!%j99U;p67%PK< z&JUstE={D|kFwmozZ}J05#@Y`$Djda3hCLg@PKixP>H4(S!S zb5sE?O?*7pH8NZtA^~vPK&|{l6z4Pf82A9S#OQU^Ia3+`)K+KU1r)a?wYgMjyl<*n zNHukt(Z0&&WnLp!`SZ=SYu&|8X4JSJ2w5hrcH3E|{S`LwuLU{FJXL=TSN%XpUOBm} zxo8ZwRs;3I$33XQQcGQi%`_vF9G<`2g)SJR=kSCZlICXd4t2o|JJbd-u>*`&nO|hV zaHOEHHrRy70cf}gZHATyJ4PJgQDp6oMVq01BY|j#9wy(oXHr6egPV3$Xw;ODN21F9#RhrC6_$#nc z5Aq24a|l?*5lbS$FwOEWG(4tLLTWJFz-NKMp!sK{w{kw;PG^D9WXQn_{!YC#*Ui(@ zSk6=b9>s{(>#&SY&=Gfb^4CX z8%`$E@c~7n6cI4Y=I8{n&?bb>ceq)a`KCHTkH87KF~b}$A{z}J0fbTx9ox#HXuXT7 zye6pBNTO9hzhw+WvmVikQc}cOt>7|&2ImQ5%2l@PlM6O7iTMD0a$e_NYksbhc6Mmy zm^O7aT^{co?C(wOU8`gA#WN^S=$1aNzal(j-zaPUJIF~5##^o zL?-e144$4D=r~5Kwfg zzcpc4hjE6vCd3X93)y-2guxZte935|#2I;D%>{2x8ydt00VyuEAwz|kd%U{REx@mE z`JEQEci^|!J%Hc>F$%)(fR@RuPg`e)so!77h%wYYJ|>__PhlTF7|sO~4o19zJ5#(B z#D3}8G;6U+UP^z<`gz+8+^oFSUy7CB0}V^B0L&MUW>fGVGBaQ1;2kf9$j(6~3a2e^g?NUrxDG;@8j>D~v91!SJ}YCUxOLDi)van0cidKXz;UNHw^i(g6Cm)8=YVkk6s)a0J0u4?q&? zeEE^SSeMLb)FxVinVF0KQkLi?94t;}t*1QtkUU2@&d0F_UF=e zRceIs4#d*I`voDY^5|G`K?xaT54OV)%GM4OT8lO1o)uySChDvs23-?f!212wUFh%t zund#qNMnaB_5JfFpMCY$=b@$rHh%lve|_=gH(zD{@we|@ejh5>URPUN1TgAH%2^1j zv4_?(z)BvfkRk-Xdfhb(G#}mEv3)fatkIP8r;NqlM{%_|pRgq!A%#EScmqYf`fv#@ zhrrKRF!EZ8Z3rNY=!Nc>1pU#w(6g!26dAh3#4u6tz$z{C+n4!*55HtmtK?h17yes6 zN$2|AOnQ!*&b=Bxb$Dln3cB1J)`*_rD0Gk396@~vkHGC&%U?93P$+5+{ zR#wPTMhUrplVuw%!%HQjD5vmNY?JGBgrkClG}_-=+F!s~YI@ZeC8C!`3S7&L{86F_ z6gO3tg+|)FfqKLcu8Pa61uE2H*r}e@Qm+^GspmIJW7VTK$ z9NWf$XyY^uM4i9N)z4R#=hNY5`{V53qmReg;ggTX+5X%W_U> zDZnHT1Au^DR|e-4vJua8$ay*JX7vpUE(a#%*SuapUmyw`nswfZnQ4OHbbvW=^v}bQ zbX>yZiZc4OlsZKxMU}^Hh~;E?qQA%zhZ0ZZvE4?d?4j4u_1H<%6lQ zqC$HZpENElWz6^`1dW$7ry>(9BhlhjcaVube2MCGYecHAjUIIZSc2r$vNjUtih{57 zZ$TA|GE$q;7LrG)0xfK(29FFvKfN50=Z%HhU1(agc3}u1w;!=vMB2m16M$J`B+o&d zAgN-z7!g{U2f>9n3jX#m!tA-l2CJ(LFyUs|2N$A5QRlE|&2dwfvOkrAXKR#2L9Ua{ zh||7ZV14YjJIDO`v$cfMyD-dI>!KlSo-<~FK^zcB|Oe+}nnlD#d zFzN7y#ONA%8>S3wpcr_hRdm!P+*BoneZtL>VV`g@SV4t)nWY4M3OHpo2;HKSwY9tL zX<)RRq*!ffLee>61U_;#KT)|rb*Qf?(l%&+Womn&@N>+$sVB>eps#-jtgSILFDL9+ zB?4bk!j*Iza}Oi&rgsjVZP^#Wqe8U=aWyv*870K(ZLei?s9puEC?!{1S{IGc1nUc! zzawc$5?bh3NZ;@_h#JABSwhqNt6*xuq9Kg{JbuIlUjP~T?L5DlzA4JE*N0T57z$0A z-iQ^sRt;KJn=+}QywY?OltE6rzMZbvVDRRnwMPb_Mlb`>!Oh2{lQL%tG{+xwY{yz{ zm3S%9?dx!y?Z@N}c#;J1KsVrQnA+7EXt;$flek?1L8H2@tsg(iKVbJwnq!<{u}ndr zT&v=wc>D9bZmKet8R>yzs!JdS+Y6>tE#}SDQTAjXT*xrc)pSu@f^3k(`aTsx$$VZcFDDiGpFM$c z%D?Q>Pe1i{0{ws^TX%{DyvIyh>PiYzBn0lv%Y@=Ncbm& z_1R-prXQ%{L)A^>MCo-M<0R|iWgl-{2>0xpM(-U=Yo}_~dNuRd6B{`KjawRT8U%SY zh0}@Nd~Lppz?BXW2FknJ?Fy(wQ>;cFNaSaKo2ueqqCz792AVfOyOf14n=p}lG!fZ# zbLP>?C6EEQ1=R_gcQQ{vcZp@|k_x(NmEp&sTwiLTV0dtfVGOnx)sT1vv9Z@66?%A{ zJ!lSpbCO{$s7p5>!}99;>19np@q%-zGFBB8;Ze4~3Z+CblVHf26y)njIY#UKe3XIv z)u*_dWf7@J*>R7*mtu6V`@-E~*A!C+@b7pB$jMHJ(0l?T$S|<<@s1*<6m9@^w~umK zoxoErWr*nG9Speh(Wm2&J{y0$KmPb&{PE%V<0s>fKN^4h@%ZCU#vlKA{PCybgC}EA zos2;cYnk3K7B7-75&sV-@(=O$won zDb^)Y&Q8?_N-NL!7VIghXQV7GWGOh^#0r?FBS|iJ#3P(K^+n(`vL`c%2z=Co2#DUZ z{{U}poFt$JfuyRwWs8-3H#4iULA;}>mB^P#I*ev((d$UEk9BfZCgx zN|^K4coT70yI&I0w|4o_YOZ?Vc<@XqkfSQ_T7{^Au&F1B`(d%PI1p+DHO$`X< zRCINa%@r8<=2azRIYDf0>ekIM1H18+%Tt(w&wpT%j<7^8mC6iMa?r;BAiAXBY0`lN z11*#dU>;mVFYX~O)~ZCK>)S%qwE)>0bVAkJ8X4Cx{tWy!|3X^SUzo6~`a5cbg$S0> zbwsEno|K?0^_lXtMH{-iSBF3Co8o(?x9Jz;3kLj{yg`Et5{{L>06IjNNHxTt4Db{!Ii%M)=BIuqLL|PNGa$qSu6!R zTg?Aabrz$ML}xeW%~2?0-+teJLU#z*pO zLJvvN6l4$cw8tA0S&rVo_u)|}=$&(WZH?(j9n@VYk6d!x+9&BR=wuo2!^-@3-Ws(O zpTagW(`2)|ZZ(fY#)=$)qwbzfszds*oWQml<4~WI(*Aho5RV@_>p&rYGEfg)EXrGfYHy0V zfZ(G7HTE#xsOVF)E`&(vD^%ROG~$T7C@xWYTgn>>8TF{Ro_ZXnTS(jZNW`zH2%!=Y zy}g2xD=Q(xnNE!juBp^qemBck4a$ZnnS#jdTlg+08e3n9NjybFg};t>!0d%zWg+qJ zoQ~81Dg|#VN`Ul|&tf)30eBdmVuRM4Bu{$vQ4hcGUvPt`q-6_EV07BNwAt8`dzf5W z$L@4{^nl6&7mk7@72bgm_bLi(pvbk@(#0}Zq>Yj~xG3)Od4SLR=htTJ)BumJW2*8D zfe{PAT7I&2gXvw04co-x2FMvQ&|nW=#Uh8Qd5fhRqhQFOp27-=0ZFCiU(lOCVj_4b zJ3(7Kke3J^BT((9-h#i5v;6?6%4^Wo74?-;?PPRy!XIVEUZGL}Q+k$1?b;id%xSlUS)<>o5@ zWA@*$y@}Q*uyBeRgcT{s^pKJfLz-Ss?}ky?WUQ+RDMUJ&sYx8Gv5gIwM;sJ_I={A? z;NW5qs;Q>4*Z%?3(&$w!MPiU!gBJ}UR6u9QVp2d+L_^}gDI)H{GMxP9$lit84k?fD zaH%ciu?`((JiWxmM2YJw)GsB|%gCwh{BAFO_+yvlPvcw|a_E+Ptxsg}DIA&g$Oh0l%KbIB z!WA2=)FIC@jI$^E`)zh;pF$m;59s18DgwbnG$>NgEALo%C7n|gQ3aN(7FXT4NbE=FVI+~cx1AqN{4yTX$YohgQ^OHwKi9l|`woQiuxW6!=nNY!F|@%z9~y5h1JG+C^1&b{I=Ol&A^m zz^{OS?1@*o);?JCBF@muDi;g~B<-6nv7ijTR$jEWDUcUTe34tNDe)JG)g`qJv!@JH zAOVH{+QI7mgtOuRpFl@+LConD<+2(IGqj{{6}aj(@J$2%VZQ;a zOgz=@a&x^&mcX6rXXaGH@ak3xWsudZCi&f=;y*PM!>wXO2Sv?<=?u0QD(0e>^x6c0kniMJ=Wg(xNGh}q)A z73gT;c*R581uoNNA%L_HELG?_-8&oRU;=w|E6dn(p}L^S73oPuklOd4BwNKbYmW-j zvI%8j4r%)r&m#6#+Ie_Jr7abNOx6NrjE`bTbqf;=)TEHA&6BoGrTuQn@oBbef*0=4 zuI}xfZ-dq6cUnx<=cALL5RtzlH;s0Hpv4czzJOfwT(w<1n!r=YXv&T$V|uKv3Qs8}7=z`&B{|!KW4)ZzfMcf9 zo>E_s70O&$t@}-?bQm_Rd{0fKWEiEJ9teh4t!ct9mG$UX+ud0}Z9Y(`vI7Z{I(Y=N zgKPH)eE8@j#vliBP)GC+e@n5wC?*tFepQvTyas;PR8 zarTJ);F%e&sP|$cUv#-l^cI(!i_vD_XL23qv_U~Vx2H4Nny)iL)+$*!enKVb_Wk<4+3A+_Ncnlhmg}zD}WW zy5p++r(k%6kY`XBwrPLZ7##0QK?zO!XJ`(n79`6Cb?19rgB3MR)h2D_q6vyv9}@|)?@+$Q8C|&8kgD` zr|IzfYEVJqnrd(w%IYbZqce;ZtvOfeGK?=hN1Rp0W148ksgp!X{%+|6jAV?L=4N>y z97M&eh&_sm32yjyiPx2I`~NFz;MFgt+OZqlgQBZJfHk>**?)kiX)(*rZ<@x6vgWLU z8A0tuz9^^O&FfAlSzP)iNcIiu3(XErxd4#-#`ec{+cId9v~2_%%^FakZGo)~!jKvY z9D*Rd#8u#%eb3>T2JhDpJSCLpoc!#)3LQ5Ho@{tRiEP55V<;eA2J)T^VsN~(zk|PP zFu0-LDq!BTa5M)Blz{P!Itxpn3{EL%!0sRv^{CTQxD1waQIZmzt59Us*l9{`5h+7 z_$dRMVUCVD)e&r2eo-uOwz6OPfFmBj3%Siy446JGD8Y37UWfQMZbPY~>piQ#rWW3L z9H+BeCC1&;NK#g$O;N)9oJysmYA{Zav)?JvfVuEGRN1_!1qHvC5$96+Kl%4R9elF? zl>QqJV}+9wn`sQ^(+$!Su7LdpG#Vj>22Nv(pONSE7@3YEVm&tCZC=t@rXt$N+(6bl z>M^e)L7_v#iL#Xv1ux4!jHP!p9A_z+I58-<+81-48~titH0+1JP-toH6U5^RtB}{= zUk^6oU&ZjYzF>kA-AD$2sogqIxoC7FsAyfc4i`lt-v}~9t3ZCBWSkD5ZfH$H`Pu zt1DC_80d!-qocE4NY(HxcGY>(*^47^ZyQq>h(IBDZH zvXucWI!zfSpNj4F?c^jfMv8%3Ai$^05D*@;H|wLkRG3~uq+@Z?ANpc4sV9BGkWY+n z15k3P6k&aGpQ0%XC8zZS*fbxN+*C?eprjk=>X7_%Kavpq?I9P=o5n7J@jZIcW1yF0 z_oGM2s~sx2bGk)h4;*m(5UZ#0Oo4$;dCDmkXIbjRe^Z=nw&3PC;o5-ezdARxbndQy zLEGl66zxEi!7#_@=God)d94H(6_d}3a)=Qx&3*09Xt$dGP6CpVH!letu+rKEIleD% z&M}Kj!1SQ)#}3G^y6J}L=a$hk9j%$HXG%;v+8)#LNR}W`;W(o_ZhQFCZ|>TzePA}U zT`ZX-`k9qG@qZ=B)Uztjmw|{z_fFwJJfq9I9xY$q_h=c6TaH>&j}PP(#?;90#@moE zaj4<>{h%m3g0Ni_cNy$p{*Q~C{UD-tZk zef9Ur6y}>~&42nFC??P17j&sBqVp{GGLvR!<_-k_wHdYvaJSH@wAAt-GRIQ6M;cN8 zFCjFBS8L(NJ#Z~)Q&D?{GnlrS3kxKbBWe!QawP{?CU~sD+<}u-aJYAow6?n7eYdF}yG`9kn(}WR zNfO$$Z}H+InSIt62-vxcDJx=+Sh&1cS3Bv^yRQtqsQD^Q}3aVe!V1a z`~O_^+w?|yV72f406STN&FCrv${5?XHu^6`>psk~CXt>#GD=^n;#o~=rk30^k6R+| zb@yW0FeZDf@FkHO&!@91Lvlu|tFd6kLQ4K-o{?kLLdQJzLJMWW0b6${^3;xgA*TKC zV?ho4-0BX+XPto0;f|~`Z00I(p`dP06iMONCn9m0GNwFy%DcRdrN5R@j2+>VVh3k5 zX`Z~yGQMyJFr009#S^NnE>f847xm4xn6CR!bybF|ud0XY2dZC`WfL1amEme**x)Q& zThSP+vVCPrlgptreFauElmeM8-#HYlNE*6fk~l|M@R0gNW^t# zg+6#^B|0K_-T_<>xE3u~>>}WogrUC8J!bbm_pJnOnTviWb@*U6_fprJ6qo=SF$Nru_06Q_PCO6T@{Zl zR9E@naDbcnS$u#8>Ps9Cc3-Kyu}t@>eyICEeG=ae!plx&xazIrx^e@3DMUEs+|iWH zbfGdDWgljrLKv2gpC}w!-BzAuPd*wkZ9n7M@kjzHp-P} z@<0Omd?%x-{=oaRc8mS!FXr=_>#Rb$yhC?OC5W=dOZvE<+*d4MHP1Mm4-AJkdXSUs4ds5RD z`f^&~c}>Uj$Uzn+S)xF9ByQKxM|N(61v$UZKsVxyg(6U31{4Ua;Bld!E<~R)-W)l% zDoO+vo&sU$K<9Bnujk)z=;U8uaU^mA4UUB65qF)6CJ{1H8Y>c*$w`kl+-oUZ1cbh? z)abecP8^>C(>%B6ks8~C?nZM;!6l=_r|wgl9YwL& z#h7d$Sj8LM4oXfQ93>|k2yffKv0oVu4$o0(M;5`O=x_A z`QJIpcAD~LcD3WFs;k_IqRv4*nBFwy04^PfMICt3m9@>>cboZaT@{!za-~?Dv z{%%*>$R{CZGfAsz^5&Le*a;2ueYwg=qHZ z5vLj#8L>q-RJ}>nA07o!8jH8+Q>|IUtVp78S!rz1Le*pQu#-n6L@DLPRnTj~z+A~% zfvv&`?w^j#((P-xw320-Aem~J|se}$+Z5Pn{a-LcTCUXF=on*xZjo-A)$X|Z?BcB}Gs zEcxg$GRQuZ1=sHbF*yUqE=GMnjC@~tzEj(~#}Iy|Qo#a*5Dd^||CSxXiAIXWHfPJ2F?y)7S)}7KvHOFnKU0{nM** zE|w^-Ci22(-`|fW4?ojk{Ij46Bx-k5lRx{3L>sivtoiC^kGl;<9B4!40r#$ZRq>E1 zlrw%gvA=w^oMZ5TM{Nkcm{G6ZpWdm2&_@*14-B!DT3JTj!&i`T3s@r5{bNPhyFA#W;y%P`Kyniz)d(2#SDt=<+*B_FO?_ogB{QHnm5 z0s;raP0(IJHg(M3t zv3=B9TYH;?$EXr=3sFxyo8hv4>uq)%&%EO}US}K$e)gLUB$oI^4OI6M8HHrEb-qI$ zV&1uc57Z%c#3fins4PXU8_qk#`!pXNC@rs}b^P1C`sTcDD)&7BJ2`}euiuondG&k> zZt@rnes#yjfLBj@2u$QD@Yh8-ZNu{hY@<7)?p~K3>%hJJn)n_A$ptL^R%KEyes|2|d| z3F;kqu^YyX^=T+1)X>&m|EBb~zfV%ubD^~6r%PV+qCN7*vhK{`_4IWP#sUYcyR5zz z8wT_*b(xWZ5)xMCRaL@`;}UEX+Pf_=EbJ|ih!CDA88kG6Ew%lSBTagE61PxU81-gK zig)t+nuvbmUu%asSq;Evw>uTh)f9Ps;Pd6SpI)|8Q*KECeM;RT6$w zDhNv8Ee_0!iwje>)@(lDb>}Vg%3M?!4au9b$T8j`mpi8Lu z&9ggg>{RI`tP_G5SJ(;h=Gf&9qs=Y^&Iys0cIqb+|M1O~7Ep^d2?RnZ;`bg#GC_Oo ze|y*cPA0t!;GlT}5-P1XIlcw0e{ZE3>4A&(ta50SsXAhJI!3SlCTvZ4+r;4~!HYpl z?d`9%DM{y8oiv)=t)*13Ev$M2gpcO%&8r%Rg(%cVH#3`c`x-84nLq`Ls!QT{NefcU zpX(`T!5J!OLD?^(j0<^DRANhkdC1U6 z$4CKA>@!o!E0L3TFf!tDKG_b(?s2`?CNqHT!PW9`Fo?z1P$rjI(8C%_(yeGpNtvl? z5HCpz8r>$39I^bBl2PM$1H~C`fl*thsrQT;_xU}era^swj`e8wcgLvNHc10C0W$+5 z)hb{B7HnOk+f(P5EYvZ!ZUbQfE!{!WGTIJw5l2Y*r<)BNuRB!+duc4{wMTooD;tiW_> zQ~ox~U&lF5r7!Dv=HcS7E`DCQ9W1YggSLsmm~YvgE7gzLf`Cml!1)ssDF zocF2{d*@xtEVn995dH30UGkT8bz;n#a~vk;WQDD$iI6klJ9A_!&nQcs<@W%eW{^e+ z{xX&&nrl^(;*pzm8b=|%Lh!#KHN)!>P{PhyL=}2!85wH_9xg?4Zw#2vyYq0 zj49OPyYA`EAmg*qLVnveRX2+l8D<{`%Kn{%1*tE@1wn`eV)$BzM-U%Xua+}8hLuw9 zMn1@%J*K%i;pUbS(vuTlN)lu>YwezZZ1=EBR95@I5S&IR89X*2h9rN&`9qFXy&F`M zfh}s3Q4OWHgW1h@*a=f92qiJ|P!viUgb2;&3!(zk#Y$kz&2?TCvtYri{4V%i9Jpl_ zECFf8BTZbNm%$RbfHaW)2qP0-uU;x5fPB$WcKGrB>h5X4AQ33TD;(ii4FtA{kovy6 z(9H#EUXpZ#Ggih7qb~P=frZpMg9X$3O8SqG3s=yK!%P@#C5sM@56|6Qwi36 z7A_>2=wKNk7x>a`$^BJahYLUpfkN|&KEV7vh6&tko6~_G!1@;BB;RDsS#X}# z!L20f7@xo_fk8Vl*`2;K$a_}9L}`{d#zaI}7S2=J?l*-eYdMw@`VCpsn z+Xpcz2Ks{hIJN>I&XrI}{09{Q;$P4tbWuyDlLnl6>IslyNfsnm;0FAz6)ku*fYM=j zVxVZzMBZ8H->HS_&B!{Gh{0r)=_(_;A@ zh8XMx{vQ0DG=RMTEbQBsdX^+kqA-UC3pNWOI=Z_Tpdo>i>YV(vWN_N;&)Qsk(Qe9A zg`}qPFkc8s9_hzfv2;QbX;ot8*CtYJl}}k~<=RG1-8#sLW!-S#Z6w~zVAc%i+=ea| z^MfqVkiibb+Bu|lphcbCNZuzBTM67qSGSI(IhP9rpADA=-WO2o_7FiE5?mx}c5K?E zyid0V0+)t}Kv2_Z0VHV9rPy-=cA$u)P{(Op6fT#Gd$Av9`KswoGW1=bi?P__#rz$!c>yLU zje%j13}zGC2=GzHEO%;0C0_1RYJd%(II!UE&AQf!8{754731*u^PmvJ0lJ*g1_v2D zZ$qiNxqAPPrk0JZ?C!?v8t&n68(q|S^Su~mdD9H7gC1j*l*>23kO0ueqP=#L&;jBEgdE|_3Jp}BKD~#zJ=b-?NocN*-IGF^LNfVgmt7Elm61^ zLeHI+p_UM2wxw4QsU5j3wkCerLz#D7+z+6>PS*PioI=a*mVYg0H-e7^GX=l`wI?DO zzlmA=_kWkrqGCYnby9$XYQupV_F$a-Qp}Icv5%3xI5Ilmf|jdi3mSFE-g8qd=4uKO z5wWYK*g2U@v=$#n-Cco}w2aarf;p`4-C>V|k9*!IGG!)blnb3$y0dR`F z!V?^lc*5ZFPSi7A2PTrU}QlE)Z$~-f)UwPX0uJJR+G&t^9J3p23CIQ4ev-o z#wlRCk3uzYtJ^q#4yZ=Pmi1wj(UovYL@4KaW5d&>q zustMzy95k+1-hlNRw5ZD5?Ee~h00VSWXm3z0h4J~dMMPY7zDPLiLkdv(<-Yr`_RMD zdEy78fC*W^Kl(uH(oINX*6>Ez9;c8EFZGlPY;@O0EFs%Z%!0c$;n&0kA0EzmQTE)B8yWya^rc z-_h61Fw$3=!tBFWU{pXqPXz%fm&1XOC012l*AN({z5aN!WD{np|FE9kvaa~ zFpnF1ncInvN(VZJ@HAQJyd5U7|%-LY6-e;?t@libq0%UhYXV zzE=LgxA^KfGfPC;p&C-}KygCl%SU#MR(df4}x1!+!1Zjd)hzsRr30|*oD#|0W$}^gry^!pwG3I(&L&n>b#UP}a&999l5a%EFhpZc-CjS@~5Bp5JIDd`zK=$+RHRtK4mCUO>$5 z*8RxI4Izd*t!IK$u%`eC7QC6vulSv3F1Ba6|I|$}*W1r}!scXYcHU)%vgu zSQ=egAI01*)zfw`z+=oj3gkD#yuK)8mMuKc`by{UAkUe2I>8lk;ra{WK@Ca-y&70T z<^e;Rmt^*iXLnWJk_q?WPHWCR*wrLA&4M1ADwp6do8L9p`SJ!*xTfaYd&F2H1Y%T><%fR1A?Py`^NfJwFJVQ*cr^F@! zqjby>pcx;0Zm7=WB=+v*N!6pGs>@tZPG&vW?cKzB_-;MT?3Ref+Qb4L50Oa43_o6` z?gK=HOD`^5{!x&D=iCaY)v~k+(vl_dkL;<0x! zO?A|F@#yk_aWWibKHo#4EA(|XUC#2wmpA9<$f;u(j~>&T2yG@kUdhRIc9qXwpO<$# z9)cj~0hBlsqT>z(XT$r%g3>~JV7(5U%b9qG{sBuB2^qeoCuan?^Dgdh@-Ag zZ{_m3ys2{#Zysh%`T--61E>+qujM~z@41-6@}Tp(y@G)?f|M(iQyM4si^W2Z#d!U9 zF|*?R8a-`!7JEW~CI!kevd!fD`h0!{W}y{$ZhV-%&hwQx31>HT_Tl4@ORdwzPSn#n zo$L5LDuUmE&KJMU{ojCH`i*Of?TCdv;;8zRz5$&tDi{k;SJ6AT>RQ}R?`yK86s!{k zTQ3^8@)w&hq;ljn)B<3PmKZcjg%vC2mWH;cs&vSPc!Z9N?4%p2*2&Ty7*5>VN0*k}c;?X=Z0ZR$O z=w`KSBYe(%fpR*kLi-XS<i^%hfX6ueGdLy zjg0+ce!yD^CDnPmlk?|m^tMHSJoN-8&{ zv`d^V-`;F~ZIdP}YeCW~VMIt_c_wGaTRf8+45eeK$`CNcA-PA*2l;}6lwbppSXhP@ z;S8`1(!87T@k+p*$zk8vP%Epl&g_yBaJ#4gP{yl1LjraN6cYkgyLO{K*HV_@;oBW) zn-7e--Yp~P{9vTzBH!`pt;#~K9^eDTdS|k*x8>WzW_meqNBU3DH882sjVLaLpajif zQiDgL=E#9jGVxI9YY(WH{h_)PMD=5|DgP@qE23Upqpn@W(_+|5WX3ZF?CM+N+$?ST5N z$-!6*HQ>!ci4Sxd1C(W`LB|22bNTD7*L8}(9P2PHx?G3Mm56(^99B_Ns7Uf@vRt( zdR!0+y|P~PvZm3COm2+UWPCXoX}*9vJY8W5n3eA8re4^#+^tg zxTyD|f6#Kw*JYLS;hmpmZ;HA&FBU~3-jE8v_h3wMZBb6=6YOu35rxC3_St6t!982Y?g+&kU9&C}hV$;g_ajm{IV1pDBpN-{Z03jdsf zi7HQ%r=qL|c93`Sz3Y}An4-RTXf)oRPbXt}{UmtCkxI7d?_o*3h_P=T-5pe`S9 z<@~w_w{S-Hz7asB-SC^DYHp?riGU7nYwHZIOx_XqWZaBg!JSbB8nE3nH zRSQm?eMOmYbytK`o{nTE$J9%zNualvB-C}bpq`IFC@HJlGuVmY-DH5NN)w3wvU-)2 zWtShJb?FlmlF39W?5F|onN?Q;(e88dcsgYI_53bnWPtIdACxnB7bY8%6OmVFExO2Z_6#~g`2n2~Po7-vvkgBWpy6f^;D;!NhR_FK|X zN0hEe->@FV&G?iRUDjjx~B zXwVQ2@mYv43qi2GB|Egq-Fy+hW$V1>vYd-;FzcK1JT@&U+XAi&NAl`yNaDSRiuUbY zp;$A2W!Wh~d@rj%D!6ffS`iWJPZkVzxQ+w7g-yIKo_ef9H1PmYj~A6h-KS_tt^b}( z-lI|9{(cK+BqQ#nQzXW$elv=M+$A8>u}PNF@W~ZS&|0{stP&Ge%t!JEd((?1uZBJ& z`d(<+1EJ_}-^-`7tD$8LG8Rm0%7np#D*;Y+$R9%fAf3}_+BDU0fFEPAEX~#$VD~b{ zfL@=ET?5k2wZlH#BuGmwm=6?oQ|AvJ^j^U+wQ-Dwfig6Z--Pe6+ zzG(3qE@e#j3B~tTPvQ>8{_FP`z(1Q{K)@cX`)nNnHly3gXFucgwEo$T<{!XF9=8lY z8Xx&C2sSA$b1|U&?GtS5{QYk2K5v&^%f){;1pBv&0S`nPw>j7A&_A9eF+B9!_V{Q6w zeVs;?w3*VVzRmQvsU}mlgV+57nWI8t2}G_BG@u}-IubP?tbU-5s|&I z*%_c|KgrO(LQK5a(qa|h#}SGMli5|Vn46k}V7q8IFQ)|7ATxg0_u0(#mA~mo3a)e# z8o6n4uAWzUCC;Ji*xhp<^Y@3eSGVlU(Rd2DUmF_ty*A(um|ML=3wFzXOKg^Wi8t)D z6kx=CdzCL`yJ!^-mQv!)6pG~HkJ%I=PBhMVobf+v=bSNBgq2ZMW_3LrfJ4Z|qP(5l zqrXQsAh}}TF%9_syT5#Q1WE94ATXqS{yA4rV@Ojr0_X5zQ9|QAUAZ-Lu=!?%+4wHX zvgst@LuT$k-L+Na4G~4xvV^hj2QiMZHEJ-L{{ci&avN>uoOKtNI4$Cw99S*@2rqB{ zkEIVm7ibuv9*hVkzKX{l3ap%Qy5v<2{h-h*>YQ>G!-)0OOq-kf1?D07T&HCRk}_>5 zfE%rJd2u1=IJik1U>Kw$KNrwHIlBL zKL67pn6in@(wT)X)lvWu!)})Pfm#*?if8=o`p3pCCrNA{=y9CYuZvZdGOwdHoYzRY zwOU=dhH;qH7gQaKQn11qCc~=Bn`)MiiRvu{*+NRESTZpF?Zg;B-xoP`oD+<@nxdfb z?pcS5jz&BwN4>-4%>rCEt8$jtwIHI-_9aWlYSV_ASrIt4rVv8y}YQ}vjU+M#r*2P{Wna?U;>T@06 zGxW1M$PB}+h(G}DFPz0^Qyunch3xtHh7KS!>_tjT7h!<JgHEMQ8?)_A5s<^Z;1j6B-EGU{FaKm^zN(J;Rkyp=YvL zL8s+(2;u<0TH3M_0^^xv;biPBDdRz~8fPUGEfHiLXURk6CW6cfWXZ|2%RHnx+K)@j zLd1utp~+TO0vA_h(n+-!j zbs&o7t>N+}mPQRRskk6kE6f25k9C=(r&w1U85c@<8djD>PhgWOIJ*En9nexT%1IjG5@`l5Yzwm`q zn$^j*&nRxyxPx3bZKXvw$nT9y&{}p+$dE+Pd5%_~94@OY z47{O_?sn*?XOBJEYT_a+aRM|X$*uD z`v4rmyGa~f5#b7u#B?Ud{T{0cm?CPT!w0s|^>`MGfJ$7_Eeys0oZ$3V&iywoL2A*% zZplM$NvBayQGHQjdvG%>Qm3SH;&x~NM71fW^9+2?VWLPC?P`G5)cr6@9)NcZ)efUv z0TyrHPy=i2N-NOdu*@h;E%H1wH8Ju)mukX}g6-H+uwmnOSJm0owEk8!KVk`t7UZWl z5U+GPj|PF_AooS&w95l9N(-#+hg@fUt6r7=zni>nh66#X*reLAl)nC71()8kCWFJ@xI&4HZp>L?jd@qwX8fX?mD59SWvH1~DyxT6i7f{wVIPAHH3~4W zqLGM=2YO-NbCGUtpqE#k1PyIGY4YQ)RpQdFHG=pQHpaz9@T%*$)55FT)48g4u`mn= zlJ8(yFuzr$>$*Q z{m}Eh&b82Fhf!yn5`%DXF|=JY#vVl}G-ZLaO~Q-9(YyGuLcSiDP_RcwQjE@c*=eqDB#!DEGsK7XVH0W6tVT63D!aboF3vCGiA1O z;84PyX9m%jbyD$IU(hWjejr%b6M!{A*Am8W+r2yJ1+gxQcazGj)kiqDOb3!kei_~7 zxUCqB4rW-TFkUea5HnaAGKT_y2W&M@OjAw2Y7`EglaK+YH|LFRW$Cd1Wny6)YXxUx zSGmfT5AMb85NuP@DD1u9V6SP|B{3ngTSKXCWS#3Iv$Re zjp*&8_AQlu)qXr}ZOZMIn4&JHq_Q=l=$O~D=_>zvZh^$bSFc}`w=aV!YH*|E3u2n{ zS~JfWIbMo--DX^cNPE3Od3qFJK*PkIHGb_;5wy8ur&zTiXA%r7h#s`xfFO{Jq%TAG zluHUMq^M!As7L@T*#-+1eRIH4-LlE9mXM$0`xn*+mq!gpsU7Uvs06G4{)Mc8^NCnk z@kMf|_)?}(r5OE`7RR-_92ki_hN||zWrxz#T%`+v_+NHi@8Q6Ay`1QDZ8B1Bz()}s_UIe%bw2*^o78I=tM2l`m&tB=M5UyS5Qhh32) zqp8oC(7;FH&3yhGwzhaH^BTb>m|Q`pqJ(cb24UB5`8W}zAhotj=Vz|`N%k8klWE< zc2hUy^_j|zy=!zWTJjUXfg7*4t2PP7-F|<5y+6NHxwX$P6{6WpX5JyA@BM*iW6?t%cmd6S{0hRGSaonV%NL6?eWsn!6#*cF zrl(>QXSGd!!`T#pH1JmsxIp&$7-^BqTP_7;O6o?5517J3G+tw)a?WQrRb5s`8KCNp zF^r^~knWkHZS=koOMn)7`VvfOc4EmiagI0*s>hj!xp)aLxu0nFK?UkDf(n6K665+O zHrJ_Vw;l)l=UGBx&o)PBv0Mpa=zvQAPSAj12Q@7MW8lYmdA9?h3?$b#If$aYH6RNXQG@2O|lofx4AEhuLXQo1B{i!jXmuJ1UwHh6Up=# zMlCUZckr8_Uf>42tn*4`K_t@Yf$S_x!&hSk*+C$1c|wJlQ->c1_zIejC~{TyYiI%?Cu8b3TXH$VFLb|O!yl9 zYkY4$3ZwP4fn+ByI!EwWw?>p%>=oAy2P(VHo2zopoBC}q zb3_q3M_z)r88fyt_nz)d(#V58K4g3^{(S`vovPSGev^^8;M30RidqFh1} z-i-AM_kflHC_rVO%~e zQ;$EgCOrBOJEoPNV4L9<|9|%0zPpX%$QS-!pJGI77k~vqBpoL=lqj7m1ru7H8Dl~_G(AeVYPze})AQ)6SW5H66!R@AGO^VC-mGPmCJHkf z8lPp$N*kHA8<=%8E(=Vdwa&*HLTl|$xSF(e4=E`M=Ozi-6CBTiS~P(~=GGZn5e%<8 z%Wjd}mM~4ScGUXKQCdUhT4-qtI@eB0Fe1WJVPVU57tDWv{k%L1l9%9wqRdMrJ>!ck zuP~tuQqnp-#X>BM){~;&yC~o7I%;X_TrNOMfr6Tl&Nv6S_B~X`mWK}>Tr`vV^m_My z-bAgamru{Z1ye&?-PPl5ve>>J^=2*ZoNiMEjx7lGiKz{AXDQQWKx4(P=Yg%hMuTiy< z#2fDbu0f{3SW_kjp?Nc>Mw70sYOv#q!A#5fva;qiG#;d=&O3Db zIba0SI?3!+aL=lFle$z<|66RV08Y)dpyfPmQo)*GzrgSIPQ|_|-hU9cL;BnjX7#R4 zID)~o#o|xcu(BYB>F(9j_(0eKsFSzEh|FwrlMh&Lu1jYIfgTn+&;` zbSCe0pF71v>gICER^8Pic1?u-vCVhOjIg@J$pBgwedI{$S=deY-e|R`FP!Q|5V-+q zRo{dU3twJ`oFV+8+n|Z#6Z5F}9n1~BQajF4(fZ!7_}yRxa2QwB;%T|8`Y5^K5i6?| z;uiN@tbyMB!dPM8`OKv*jhBfH7Ef&HNs`~B$Gb(VwSL>3DR~9Cj~yCM_G6tE8_$D` zlDFW{ADTA2S+lSRhbaUCE9|74&#`^&fC{RRS9`~q#-uk!$hv==xa5VWRoG~)+#&5$ z2xG4$ZRM=)r(zq(;u3g3BXadBWF`7R7kkAL9G@VY;}vxGdvr;`CY$G;V~*Q5=pM$nxMimo1CI*UI? z=RA9fzS}2-?LPFgPKD6n(FFLr>fLI$oYrTv!(v>49m&H4-I|FZn;757B{YG7Jrgrz zZCIQ=QnJl;o}TV?#!R#zJJI}IG8v$r{1YvcNl8Lmka{;PXdi-I)(md6?Ha36(Vp0? z3XE_PY{=5D%qS9^)tJf9R{3+inpUDIA3TaYSCF6ptIDfIy{gho6Vutn3e0-`b9uJx zyL*~3b$qb%;A~jz^m;qFjTzVu72M&B7;GHeAYZc3tuzVRgohRw53xS8PZe=}oA2ap zT#f1?S(B>O+b?jutehdUvDn6nnxJPwHz;=f+ueO!;N8Q|-$iAty|vpNIlT$Zr}Zka zsY9KwdTd>))?qyZpt|1av?=kAi)L0`BO|?BsKUqi_qrScC}V%v z&BqS&r^k-8(m^MJJm`6|*4UW>Qe&5|))_mg;+%SwF9p=HZg;niEaC5|1w`spdx=s_ z?ZqtI%UErAP)V!~huD3J`-m4gH~>Gwb}n6=^!#f-#WR9?Kuk;#+#W(j@(l^|KJP*i zq)NX5Xmjp$ChZr+K3H}J)0!-MHDih+VWY*%3Z9&{p-3BFLcbj2OHZ1yNGRe2Vkvg3q+Ejx{v3t3ztuUrjAI0U_vMxuqJ2o#?9E zL_Th$ar(Ud;b9<{fTQbKKDbH%CEi~~9x z{st3c{@oQ&j7h8`0h7ef#}vw{C^#?bn>k?Z7rMd$jEVid8@&+T>}Hj^I6Cb6{TR1@ z-X{(`Oj)+HUNv*H(C{0j*3BO7BKtiu&?uJ8w4M}SJ>CEFvp+wxz7@Y$pY|Nr3fxXo zxK;Ye!-l2t;2ox6Vj1fLPxg#%c^(z_}CQY@(8L8gY%hhs} zxwGtQn+T8DAYuD_hAs{sviaN65D<(2|Abwng_2Fd7v;MjO!Qe}WX-3Jm$fW4%H0A}~^+{?_LaV3-DE3~wR$m3w`XS@!X$+a^3f z79MGK{LSNU)3Tnuz%y{iEo9j!QSSD44?SY_j4R9S7=*{mTLtILT;X7Hy+i09yZ*26 z=dElEwjrPAuHf&)inX@x!C&?gc?Q1hC2~8~+)MNsfY(i{-GCjsQ$4pGH2h!ovLdyY zrS>tX?)LU9@7KM4WzP8bM%BBOIyUtaCo6sq@z*d)Gp$ZnHqGwu;uP~{S!0he1qZ46 z7evE~(~gbu&oSy!L%vH22O)=R?hTA4P5~hVUd#~K2MpR3t@$IvliJ|4EMW@Xe{hfB zu4V~=8BzO$wAD~N4RZlans9a{D$eS+;6nsAIS`?i6PdCCyY_WdRh?Y~v?i)m5v3tc zq5EL+HE{bW=&ax}o!~MZtjqv$>QG{#<|^LcQDv%f5*wyqGN(*4mPNUAltPJUHr!9NSn#i5MFXz&bV@uW zP`JDj+^)7dyrgKjDApolg6_Qq*OenlhQZ5hE5^i>c{_d_60I?flGlxuGgM zvpkRHOTN)uy_{WaRtp7mGC*j(T~RClFHNK5*3j22?@K2kgcS_*;aUTqV%fVOR~ST; zHS?nNz^0kW1}m34>j$`8m8;98J1iH=vv6cEOx5x%x8#%2lcy0I5$loC*TGV00h$5|gYpwS$)%E+=D0U~#o@oK4Xf zfY&W1AFj*vfDh(2t7Bu;|F6lQ|Kv#h1OFS`??1V#{u!X<)xG`Uj=gri@*;Mb%jvbx zMq&4ULWckRyP$UvSia4jp7ahXlRD&gGQz?Pt~CccEveHv1FRNgi05~fbQ7p?Ib?Jg z-ptdr+Tr)hqV5*YzoR3NE-z>2(Z%$uWhoW8B?RphrV%Xrnr3=ZF4_p1=8B`EVUAHX zLqAqVlm!4*jnLyIo7BrVUF-U3sg58rx>RUVCN>q9=}nvQn}FpFk)^yO1T0DZ13OY? z60uySBw@IBBv-=uqFFUSrK6=f#Ienxk(J}Xw2*mpIa{9Brz?A@?x*P-flbXc93UB7 z2$;4S@p5)8P+1V;%g`LW*CdCq)sy`04cP!2{wzt3gE-BZX0 z@8b5_4~&9Z?E@t@w0yQ>&XbEN+mTO}S`qBNLcABh*WCP?iUl5r2XV z{Pqdy84gK{lb`_#lor-VZ@UzLp$nrX^H?&W58;(xfJRpD2m0%r_(~f#LCYFr^uE zTmwQ-Rkud_HC3k#sT$Xcio>^RZ=J0G;cN^5WV9^^$|}9{WW${S3IsUU6epGHQoK=< z^4C#$s`Jy?S+(fDFXn1zszAYyIn`MKM7Be?ZsCm(UMcaVS{+(3d`JUv$QzZU#~~#! z>O7WD5xjEhfyci093zx$_~%l|9fAH z8l*)F;bgH|TxT@HFg25xyaC@M?c)FU^2Lw+sk48R(h%-_-has5lHRqY4g8Wr@9sn{ zJW_(MQuHcQMwIpypHbn*a}2?2rg8wDj`%??EAI7C0mCt(`?&h_y6^N+Qo4=Hl@c2I zN#G8~|BEV;+mn==u(I^ZC{I^aeC+cUgIUHEIX>FOfti7$u39G zj_7r0|IXyn^PqhVmDkaaimR(iUu8 zRJpzg&*kq8MbTcUCQCfg$pERxFBygOD2JjLp|Vi?4_g}^4AyRMe6ny_92_i!Tym=w zy`aJobp!>#!BGxDL(owr$fbf-ZHY=Xh;CFa`8In=nAht3a#9uZW=;pvR3X*>6#ILl zy+4i)_V#~=Jq2&+^nRNO&8*Xab~c!L(kQ|Q+7Z>)9N>s4MT!vZmV@6@CCRr!nAu~Z zz+0ChGZoS^YhB8`RA*RJr;102Ed$+hR#eT1#Wxg*S0tZ{0Eazow1=p&>LXm;DeBeuk$#``c*iZ0TvG+f zh~iVObpl|K_{hzhl>gc0?-iwU5F~U zATCQD^h&qF0&4_g0e$5L4usDf-Qh3L;&>$2ML?URPAxQ0@EJw`?vFe6zEkJ#h7AYv zl-Iou?k9_4vLY)^zvF4iQrGE(zMZZ5la)H;V^IdUe3FBqgD25wI{p@l<5l@G*6}hy z*sF`&+`#A5q;#AHJ0x%l5vXZE+a+AyYRagFnA+WDNf4;sLTKrGlsvWQ`w)( z9n~~(rwM41F=N#xhddLwWzbI1aXN0iSU^zJs#%NkpVOEGE%5NSpV%2imG7Z5!`*1& z21IeDe+-B+)bGC%aX4>Y5=rzwQ+nvE`H5cf?&})a`)JhOOM@|;yX=Gqn3W0p6UZk+ zBkMr0AF%iviDgLQTGYHYnN2^8bN>^HZ&4|t45uGoa-u>uRJO^-nVCAngV#sk@Bx}V zsSre z1!KTHyzlcGIqB~HLk=KDA*D`>QVLsWd}YFBLQR673`c(uT8cr$>gA8+kA9<^Ha$tS z+eva5DTGpSXk>pbNXZT&Y+a)yXAaD9WG(qvcfbT^s0=#o(3o_cBa@#V9`>3_MV4_$ zIcbXE`=f0KPj>T8!_&c&3kyjed*5zzRYzgD?5cj`Oz^gr@Nac(=xO^+t&R0M4bRQi z2CT)OnyYo3bC)$@W`#zr&qF{iQqqt_C{lw4l9rV}=o3=gpG+ZPB*P21`Qx18=&EVz zyX8El@IDl%5~fY7lET~TR_I?(l@e_h5#Nh<{9|R3XTjn>nu7iz=LXwL`M zcVfCAf*ypfqmoDMVe9mny<-3bDbyq?`Z!>X`yK6S@|r|4!FG_kRr2FO5xyzX;d0;! zlT*z+1oWZ1mM6Eog9%wQG1LP*$@y4sCOVE~j`|Y0K#0L+>yUmrb|A)Q9;viW7;J_EygkFTUKrjo{gu)5w&e)hhi4reYD@X?= z(%)X8C#%_xjc6mrsKMB7JVE`z&f^zv9zCFvQgt@77jtmza`%9w8G?S+vX0YF3;{mU z-2%LaW_-EK-5Ib8tIP>feR~Qj@33oKw?k6MnHYmVmwMrpDubRd2ohnnhL*X=tz z8u+iVnY%iGGOQ!~(2NO3Lb^la1Xc`?67MnLkB4_Y3^*JmKwv-kdP~opuT_0N)F+E_ zaeY|)qdF-LMtjBIpQH3Z^%ea~aMLbyqie90t7qzjQ?J0(et>1{7Jq4`)8-0NY7-Jh zjK}Zz?5f^O%WJS3HE;O0;&O^7yGif$0ol$b-_`B8T9os7wcu*MtLAXsJR8>_YEcq1 ztnde;BH*9p$?^{LS<(5A&!0XketuDW|EK?Wrhap2C@6w^rZ#_3l6f`dkL6@S=Q(`I zB&AXDmPH>RCyvx8f#K_rLf_b*1ed1=>*Vt6P@T#YJ5t>%6yvMUW@uDmy(;jSi@_#W zi*m-$d*kL}4qN491pV03^We@Mhq>L;OasINGXY1g*&CC)vugD!#UFfj#=n&)^4*KS z7SE;?fIO&l<*GPu0F532_vp1>%#2_I!(uLH^?zSh1%8FufhFl@#A#U;ffPp8t8kvg zGF=B9&V{JF1q9mGAHX^a=a@hGITlDFa7Xw%H-~;r8@6g8?Yn+n4Rc5)ZswbH4p8EZ zj-Ez*fF>YhGMcr}ByrDG$9ss!So9G71dNH46p!{pKupyY{E37Ke-KIbj=3n+g`*7~ zop*P)_^CRpmjHwcX5pn?uk-2USv}L7@N_m3h^l{4tE1GPKVtDn`MW;mS` za80pk3uq^4Z3HLdcF?bGO^HnMVP;DJCv> zGf4txs7}rkdpSiSZT*UK0`oa06IAt6JcH^_R!)?T@j-M>-Fyr<-h*0tp&33hH{G|lEC!B9V{etp{Z4Q_HvkhL1D;}qmmqf6V1bA7Y*@7A zOw0|rG4Jt@L;4w<(@55yRLj+(F&s_Df_)7~D-ws`7v%4eMH**-Y7yT&@b}W)hpnPd zFx6@zd}s{y2Dm_THB%?CLQfJ@t$$_P5iftJ6p{D1<~VAe?2nY99f^Gp3$qn$k%Fj5 zhlunG8c6!pPD)`h-Kj|c!V3t>>}j!{3=UO@7=SS@V{QV?hGF8XGkx_}F^ir3_L|gSy*5dAut&NgN=nk$MH-nonQ8 z4EPH?ywg_{`DDCYig!tXkV_0c#4E+TuQ4BC;`Scl>VlBy6r{EH=mEdUi0je*7N54} zT||!<#XKqS&-HY+9F@0;NW(QcU^Gk#BrB+&LrX3pbsV}1+jWs@wCYJ7Ef&|{&W0}o zL_p9uR?&Ypm&g_Y>U0iqm035(I;vG;B?G;9G&un-RY5V05@r>c9p z%m#V--px|1dV`PYwkDK4>{eZCyf6q?U+ux1wLK3Q{ii$R~iFGqezu+>m8Al*9v z)aoBsXiK-i6s5!MWUkn|1bxFnz|Ardc*m5M{Gu-$Eamo@G={pX?$$BP*vtv7+s{Cc zM;)Q0Pl1m6y-svvvLYd9VI8Ppq<^&JR+ISrS@C^2ev?f`*fnsfY3m94g(8Bw+B@FX zXXUC{j4qee;!kI=9JHxd@2@YPKkFHHgp?77UoqjYODtVKspvGF5wEh$SQcnvI38&9 z=4d>LDM?Sk#~<=mz@XSJON(dm%``l1nCxE+wm!DO7Jg(UoBHq@vfqkXMXLZ^6m926 zNOliRi2V~_R>&60Xr_}FS|@V^B?hcNk%f%U;&qi+j=|><8?OzVvOt6Ge--W?{REzg zN|r9%CGXVe+rHT^eb+WCeDK?7(#)!k^p!fI3-c50yUz5a$V)Q3%@No-^b)nFN8*1- z78}|5tE&Flx2 za^1bIgAxhZ3|40wLE;7B;tq><=L?2qS)MnSQ?OCEDzBHIb;aCZ|3liWRdq3+rT~5j ztjW{WsgglFn5*Hb;LMaULf8GLe?`#4n`yEFmYa?8_BoANLf5(e_`%a6!ncd=pGIWA z!PLvjEzhe6LRvJ!8ifx-*3w1WPpg(#UmUmI8h5+frBMpFeGuEKcnl}J!c#5?G0Kti z8{@g<@^9)OQ@`u?ES;&{pwInwmR%Zi^7M* zK&NmeG_3+p!_Jj=9J5YCM8hD6GRak>y7TS&2PC*A63DZ@xTq%eno$L|4fcMHqC-Nj z9$2RmCbT$8-FDd4=VD&BUEuWYQ!|m;-WR=QUXJV4HH28) z?}<$BX4(Q>LV`Fmfk*}JbHfhq$sOG4`UG|*jlmk*k&ML+WZ$LK)OH|EP<_?-7_gU_v+v$N^ioUYYIW-Z$ido%6Y1;Hj9H-<5=&rwKqlk(5pFpoB(7USMQKzKHrtdwgEb&H{5= zS7EEu-L+=N`jfTz%V^B57!CI-bj@xNfwC@&kf$U8k0%K_B2)@sW zJ7`sG&?ha7`~^lR=&CoJ%sNmGF>%%JVqEp4mkfWXreK4Rl;YSapi`$vtJBKa?y3*6 z6W5c#ZQ}hRR5pC-5sBryZ43zR7wZoQZGazjNLVxI4a>mpW!j5{ z3#OHCw#4A8*e7yPPxUG35aV36IgTm%b{#Ci@UED@9otRAPOX<9r8UhCo$o0kI?_1d zdBVHj79x@7L06aA1a&MQk{MqCaUKy(>fD-Ok9&gvW=`h$P}GUU%y_YPl!y$ZOvHD`p+#QUst76POOsDl?G(wJMOsg%xW$WD`8ZT_k!XFB2*)Jkh zLsp|g?6bwH9Ufb(_!%BObB+&sTUs_X#bu|lHPU&~tUKOLxV35;)qis*N9`Aneaju6 z(*xkT-4D$mOX(()PCeG{H(+ch8-oWk?arTlEm)TmRpZO}0(4Pfn6K&d7EmNKmva=i z9+E^nr;r22LFHf*#d>nI(`Fat!9m>b?Zj63JCA$!lQaeoY7!F`{b=K3Ec9+?YA z)vOS_PxTj&sD}u)Wsues4}b(GiXFTy>nUpM&>NYI%@OyYE~oYXl+kXCVFR0SwOrCR z8@1|2hUtq6s-}{C29?zFb0AFf2_hx5dq+xtO zD2qF02AcZcDP`9hFXxF;{yukV%cV-(oT`LO$0;(`_zQ>lmrOP9_}@buRq*Sqs@eaO z(_Twy|5HvosZUQ|zk{@UnFQCF1aHgfB~kuf7ZS!`wc|G_@eZCFig?thMR8AS$zU3) zsW0aZAhb354ts7B(6n4c>w7lZaPE`>4<;pv97?Sw2HQ~4ORA1mXTOFbyT9rCFl6rw zzJpe?a3kL@tp!EhCn7WID6{y602x!=_UU~@l;OlBOW&(I5*V7z+am+Cq0>prYdN@r zdJWg0lHN;3mHe39S&5FR;gwaMRX6kgFM&sIgY(L55BaKp=`8Y`%h@lT>v*hWV2K*J zNgE$>RU*2G8*m^R&9;*>7T00rQovBe9u*BS-6`k&3hcJ`@G{?Dqr6+)<+}$c8f^v^ zl{#aLFTqs-ij(fs^@Aj!))@S4l?2F95`DxR#7o>9n!98&p~ly(Cg@HA0#xVM&X*of zGKWF>nhX+)aJ=mQDHniWv-y`R!ofOX1S8Vnjs)@3%>_XTqGK{~p-4Vsmy| z!B)J!3j_gVAd!b^{%o0^=1zzzUftQ!)}WOZi&jBkPI_QFM+|AN^e?tbb%0R9x=<&F zGxCdAM{@+C{zpEJY;pv!;?}ytR_>ZsmrOE$ZV+|KbuV=Zrc}JvH2`mH#;Z!JXXO;t zuB7^r6*idu5Zn9&O;M0RfVz7+?G0HY}tK+A_5k zbbk^Td6W_qUPndW)ue$wTV<6no1A>tVZQ(5eAkJ5aIgKhfb5K|kh~C-pi(Y_bvr2L z&ZVtg%~1(jd^W?7;BuWL3gd%6(y*($b^sqUhs3?4jggyNYzpiY=#;W9j}6+$@UKcT z>pO)i0*TtpNt00m)6M2Ow$f}v8)h`TrrOWz4snoEwKl41lQy$tl%c0K;Vvd9VRAMj zDBaWfO?%l+P{JbEn4p4YQ!(2ksNHQ7lr1~)esQ8C*l zC>Fr)$IV`oqOOxH>Shhk@ThB#q}CqhFzd(TQhC)xM=eZ-mED;3%z)e~Gc&bpkM_cb z-x@pH8tqB^Z1WY9&>aXy=2ogcGK@CP=!n-Q+4YU3zBE(a;`;DS10T9PiedD;f;|hL8}X5*Sk+_=#3-ls06s-+Qsz=x7_Lq23A?%qZMpy5gv5U zxf4wJ46tjJ8RHymX%}b(Gie=lKd3D5aoux>h6y!5wiHhhkt)U@gwGevyjra4Y6%v9 zHh^oG!C<))*ubP-&Q;}yg>#|Nn>Divv8zt-Dmkf$nOmo4FwkBwNIdzLkXF>Zl{{~_ z1NCgFS{{YNkfz%~hc_78h^iiZXCQ0>k$12y<&Df{BinidKA!Tk^?YRAXKm0`w&El! zP2e)c=BH(0xg8BlN0 zx1jeB17k`41@6-#HyhH;l4}QQxt~UAjmVy;TRJ;ljAZd6HCeq z88f9EkYVvs3ddpFb3UdzdH3VOavsCE73Dhi3a(`tpH_K>)*4;e%{v@m+4;SFjPrip zJj|hQb~w`7$=eKe2M_I=jCU3mD4A^kDdOLd38?4VxkJ| zESj4gx4U`gn3}yK`4TZ{)AH_C4%|i2<;;E8Kv-maUblSSrbSG6gNf_<4S)%aH;z7* zXK>qZ?+Sh^Cwd3TX@UVcAHh2I-k`QC9Cdcvt5<5iPowc2u#-@&yYa(C5 z$~BIQVLUPM934%G35a=E=zC+(y2W$n2EsJ=H;~j{>Ow~{`YpjG9a#}D)WVCYAtH@n zTF~1DX|*a>N{(7bKRK#HrL+reXKd55Cy0>P*iPHTg7LI(G;Nd0R?}!~YzG7IHDK8A z4TIAQU<^w$Bz+nkC#)O3&?*!M1|0w4$fHst%&_%o1l=Ox#Rf&1(e3joUGB&VonZ7A(_hJE$X;@qhi}PV2Kmiz|{eSa5Nl10Er$!p4&N?m} zlg5b_6>7BVSqR<&kF#g^m;sXG18r;<^IU=2eBPMO10zRUwtxLgv=_dC9Vu^mN+K(A zms?u+kDR7*$9FCl|JltRbJsSYM!Lgfm#^hpSF1x-o*i-o;dnLcV(jAXK!LA^%_BdfHHX2Hg39%iX9^Z-e z%mtWi_{`V+s*5Sv9#f%xM6t>z!cMDXSNUYL-Kuow(jft!4{6m7j-EL+T_qBj22 z23}YI3Sk;?Zv+$Di#(A)>i&@s6@gD&Sh_`K!#zin2RH3w|A-ltH=%w}PCmZTMVvo_Y7Yu1}Nr}eegAbB9`)Q>81{}ISRQeNpP zS~sX4%wpx-U*m7MiXBgDmFasLsYmo~IbJl=>Gy1edIcA>`+s=I(Fo6_XoS^oQJ6tG z%t|w~Y;GXVU;(Z2%rvnoZYt1ReIkK|2>**jgAz@3_3M^sa87;=A`N%si%cVIVmA?L zG9^S(4Jj&8Vok@ph?HPsT-ziq$ZZQf-Risrsb@drFQ+|sZFOOk%=hd@+gRPf~=yWf!N*>zxy6vE!>K4M#(U*Ps zw{Q``)BS7Mm`zu{?9B4caD%NmTSl}uqxIviwtqxF9xut%9I10=BCY4qt(?rw{#>-$ zt#mtVx0zY^rNj-|^4Wi*bH7*)8mc~F`_y%ZxzkOFWI_Cy-IAnkNV;r5zgUwuukHZc!cYW zDq%@{l0nU5NOh`$(`$g79GB!UDZw~KBbJiwZ%EDtkHd+X44`!WD;@XOjS>t;L<>wJrx%wFLM(_Y z1ooT=&UIK^EYF6Bob+;D&t_F59k_1yMmNPMq(`8dl4wHm`uqz9#a7Ho%>aY|2az5j zG*2o-(JUb%Z9QS6QJ5#u#Zy`;AELFaRwM+Y`qN@jUXSLBX4Sx|9WB*qwi-d$^1e!e z5iOT-0By+pYVWv63;Gs!6g@)KU%{;jOD8bqBZLj*sU2eB19^rBiB*`R<-CH}S-*yf znO)p`R2i{$cA-bc8BYA`1MA_uk7w#PX9aCQ0YHgx z*jH3uEyw)jqI^@?R;|t-uI+8S(NJk^kB}XmuK}%Q|ak!WE-QK(O0YSyqOG(W)45S<1oVP<7X=p z_=}x0ap&AGoDY^^qvo&|E*@R}JteFBZK@=I7f4WCUld%xRzwNbuS|D@|4?VoADXLb z@l@@tu59Y{8DKk_a^gY~n}%dSs%>KpYj|H=E~ba(%LnApv@=-2=b(6EU%xsyE)LbO zMr=oFCH5Lx$rX%LZ5C(Ibe(!*3$zW@fBoxU-b`XR_HtrApDvmUwSE?F9I+3Gd@?5aNp<^W%tLkE2PFHf6VNgesehpSpCk$JSD95*XUTzTZ`aK@o)FF(Q>ms$OVzJfO?iydm@BqRKhljuQeL}lkgIoGB-_TN%Jk(HgoweJPI5sJ>hKLa-F3l6 z{ick$2AiTczQvG1##lb+wFZZgVkH-Rl zVvsvNJId*`j?z@;*2W)?leL-L((a`}ba6GCw|unZMgEHRhhuvley=u7+XgGJa1~k_ zzqu^+Trrml_TUobG_~oEn?+?LCUr_#DS`Rr^B@2B56^yn@#EkQ3-w0MbJRt6kEldf zpRNqg1yIl6!q;$E>6>P#!Z%!yiG1T`+AQ?9ulDx#g73||97Ctz``+kxLBVkN;mmuZ zZ~TuJ&7_?2M^qC9pUZdkMg2dBYb1PrQ(d1lb9osTeV-s|3_fLz757qkdUN@=#Vb!_%&HP;;ryw?#9x!ZKVgUsQ{;Dn^vX z-cBd4os+^^!Xf4tP8lCqji(vU(5(e@iO6}7K@g+iKZ&ov4;TX{y;;Ca{*lhvM2WyV zoRex?UYbRFfy)jS>`8@m;4iJz6txUR>x2Tf8HV{BSEJ0&}imq}pG1&kX8}%>cbQ%?J6X`W7g_#Oc@|tZ=sn>9O zO3hW;Q)({Lo>KD(DP=4O>EkT5Wsqu~pDbn`_B%FokZ605ELV=2m_c%`ykyx@>RD8Z znMW^Kt`wTkfjq}yvRpwtHv@6HqGWza`g089?R^=@xndA+|H~ja;!*JIcmEi|f4>)Y z3o?Ags|IcBa#UQ-SX(KrX(-NX_oNx4CT67nct%noT$?DS@(-aqT+a7v6!X3AUsw)b z%#reVcBYz5s-EZ1{s223wfFu}aj^f8>QsOKE($4-I@LtIbw{jg*WyubJ=svbyNYWU zk9?XL#3W=B$1{v=3ChutJ|n6!iQ`=qfv}HEd0|lxii6^7CHDS4&Z*-vVk>vU&ZvyB znf~~^8oz<&UqBQkB^cC7N|-LHQtiZSw-K11ZMr@2Z6fV_QGyNbp;1TY8fao$%wBwG zg@!roxnQa(3kZLLfB@WWYEr7*AZT=9eoy*vZo1n&;(sQB^Y|oXUoCCo5ot zJ)ts}kD3ETN`(Q%K$xaSllm>i(K_0(^SevJSmC_EL&J^|vW@xf(GE`R2pp6Cwou2n zvywt>K&mkq(?XHdJvv=4|nMJeYC?T=|iv-1qPv3%6;M#_j~hq z56K1#?jsf3!8g78N%d(_^zgsPQB*D$7C7KGXv_7G54wq@nx%2tY{ghDZ$9JY@^1^Z zZ@%mI@VKk^0{D0Kmg)pm_4h{qYXE!!*R3jg=W%jI?)6C_Hh{ZoIIgH67FHU@yHIT} zK!NwQI5+D1N;)*9j9FFcTe?Hw2Fu}~baN>>K)M+z)vQL`_nKff8d4%1*(grxJVXI& zz`&&+g%@&~o72SS8=O<9wl`VpuQqG6v{E~?IH&xUnrj*h=VQO6C`<+4sZL3+3y;W! zR_E&Ak_wL+@}8KR;=~({qnJtmu1^zBjQ`7FVPd~3q4B<2oxmE9TdU$}p=KFS)P$4f zT}#!-;`Zk6io5m2yjiTu+3KNbK?~2J#}=vy=85)^PnQg=r=6LmHYngrMcf@wefJX+G#QP)dd&el$S?lRAt(1N?UFJ-E5x|$@(PIcEa z5ar_9$%oZcK5w)-wmexj)5}%Gd+zT3o}!$c2Wn%YC8$Ji^?&5?0os{2RW&b8)fsIG zh9LgcgnlAP9G5+9V_ETuohAJI+E>FaYP&W18S?@sk8!qmO;}L{I&(`EYJKF(_cnGf zn=mvGstilO3Jm}1OAw7<@ZT#;D9`AJd~$5bU@4TPEfw#2bCl)i|Er)RGQdWUx5 z;$4bbxJ!NKGAz7>ZNKkcRVQ!i)vhCRb*K;%!rCI~Y=0p82f~we3dKQ zWFsGAVga`pGqqG_jU387uupM4c|vOqW01rVIakf1`k6oW6MW)xP-I|`vTf=yTvZS{ zqm0iXFygFfkMHJ%hiiJ(Hsw5PVB{Qlq-t|?x+w4wBxY=Z6e^> zs#Yr|KAZuqkEcp@nnWjG-inoM4qOt_NNk%j3l0#JFNkNZUhcxm?P;4DVY^qKSJ(a2 zwl8!scd&O{wfrNNHM9iE!Mpb(uz&-KPC zD}A(gl5o|j<30BQ?wIEb#hMrC?zknjEELJzGr8PRVCYlnZO0IEQs^RZCC(3wzhUBr zcRo+F)j%vI=1$S!c%8D<7!3>YwPtR35-_NBk)s#RM=su5r$HlVf`l(w_rVER^v4|; zb$sq_-WnWy*B&}P4yVs#M26|_IMnf-3@9Q#<-JQsjZt{-vR1RVeYZkw&Wv?Ookh!( zw=)1HitQ6Q1<=~++SmG=TVDCMJtUnMJs#Dx#zT}{6)n96^SGSw`fBeuu1seJ(Kwx% zTQ8QNl?JwjeQwQAmuwFhHqkKS=I!)bbcEJV;;1W67dL8P<)%zR)o5w8I1?F5^eIY# zA-=~K8cyjc!9Oi@VIkH)0V22JtH@=*IfCK)q-;e!1QQN4hn2Sh0Yd2&M55+cQz3Sh@jt^1Db=ed0Ur71p* zjuJ3dXW_uOWY>*H;CI^;$?L8(C>?h9kca{F2538-BP3|faV?tpbh);X#VP;VtcO>{ zDTr6^z6*bWboGsrSnfL=1uLsz;k>%$S4y-MZpWgHGd-6;teYcJYcpY!vl|yqY#WS~k~RO<&_SMimkUh8!rft;xDz-(t>rl7NS$z*a5t7frRABu zhji~CjT)u(8qyJL+ixN}Vc>3Z8R=l%o>b@M+q&^@#+O(1YJAS~0E6_jnTYDVvFc@U z@P|Wjfl%&f7fl}`B?+0WxPvPkJ>dM+?9G7^POj@@QI&6UB%84r2xq$OMCPq@m+G;F zH*}E9xC6E=567D>qNc3l0bKmznJn~C*bACb=HK&aB9osji` zQId{2!)SS?wT3bwR1?{6hU-T7^S3e*j-U$?ji}x4Nv&CnY1TT5uULFfnngvOMm^&V zRaPKQC3RM|O_u%Bg-Z@lR5N~fIxzpW#G!4oKkCiL=6|cpsRY&UPM2e&V@fo#Lp_V+ z29-Xf)C`q&Ui>#D<-SPEiXncxm!L1zkuJowZNh3B`x{Bw9f?uZ`?>DK2vw)gcmWfR ztks0EcM*MHJ4DkpIhBkx4()9i)mv;R8{q+c?V)%D|Ky1~>SIPHA=%E!s<|628SW}W zexKEg$q-N%E`|)E4X%KI6WPVwpU)v@i+Unmpi;5%dyOtUs><;vMTTj)aCxGW(U`Nxtxy&ymOAT1j_? zPf^_&h+Fp3FFT=gF*qo5@!aRDajJ4*i;P;^#Z*YN5o)&$T0ft)ON_F*eM0c0+!T1@ zqkm}ZsYIcGC%eYbTZRA`Vt%sv?KeOlCtHJ; zXtVA%ZlL#@uOvJ#Y(3=4*pHbiIExW1~+s3xF^d;0{`{Ts zCtu4&EY~j_6a2BBug2;kt(3Ldyi*n3V%_jIhyp9D%CkQ48;pbi7K!RrtkAq`YH+p- zJ)Q?}>er|;YM?_~HUwk&2V<{#Sir#)V|o2hbjIwA&z8+jO^EUvOgg?F{f9;l^St}= zc>seze7{U6MBno--sDo)JCD@B3 ztswr+WuStt-BDs(hviBNXV@XY^(Oa6;U77a8F84%)x+J z@&D{%h_BwTP}Pa`MQ_%^!9c1po&$ESK}Gjk^dc~Qfy8;Y7+eNaIFNLFDV-5XTW+Ya z>$#zZr`0ng9XvPid)J`EL5$Mia|)UzB9vsy1;#;~fpM?Y!b@?6yGhP~L*cL?5y_0H z5LH-!+vU6b-*3mG$`r|XR9>kJLuCr?cuGUj*~A9o8sY<63r7!^);E8YmKJO#`F>TOey1b%d(i0pJv1Lk!R-;K%h@5|QwLO>>|NlxH8h zM1lwZSg)VifY$jdcW^lW4y(>o2eS|(KkyE2DH;u%SRcy+=9+T?#b57 zbH~BRYXQM4aw@2nJyKJCItW3$gAHAuxP-*R>!e+JvJF7i!1->}ll4U=mV^4H)kka) zk4v+Ya8&qW6q>`w>?kQQMKyQC6iAPPi(}zQ&GO!js4y39+|qJZUzCy2k$%>O2fLN& z)6aU-xF<|lW@^*O09h2Xwt)uiI7q%pp~;b=k6c@D$rV9A!nCx4wq<2HZt{7lW7(>C z247Gm%_SQT&rPR0I?OCHpmBP{A~6#~Xe6af_?nbuhs8JYiKK&`6GyboeuQq(;tPA$ zT(!bK^Q{?(p9mg$__oHI79H-!{-z58`+HTrTa}Bd)Nf+D2)ymJg5k;Z(zmg=*H=wq z|9Mq98-|VmV{|v=EM3n`SOB~i3ateY|Es$K0hC#y8lr8_QOkN9ISKQ6YdKCCkoDtV@wi||BLOtN)!)>&D9(w3T z3Ob8S`v3?nZBKF!T;d$6i)uulTPVY^$z@T|S1!LYW%1cam556|uBrxkW>R4*yv z0M$d3M={|TB!vKids*Z`-PnpQzWu7w&DBk=rrx|))9keCN7e=;cE*&AOc=^tt%D%~ z3BH_TSQd$~-Z*Og=yJ9^uTSH9n#g@(JMrno!1PysnD($KLeO@n4h;0TTRi{q=Vw3t z_@^I=zx@5jr$0Y`@#9N>WXa>B!*!f@HR=NkAW5WM#aj$FHurAqU~5%mEGSQYSjZ(Lg@6d@(iPxY(dCnS?{XhOus~Yx9C=vWC zIgI?w%L!Ocaca8Y;2$UD_|4fuNt%;g3=--q4k&IYwnJaA@9q8DL-$5l{c>WSq%Pml z-bfwp$%)+i)OlA4)|afx3jEZJEK6@Pnc5T_X%DrC?FlTL6?QE?MHEcc827dgK?WzL z^>~~aL}?&kyLQaNEhz-*e}Pcld2=u6*+URc^0s1bGvm+>LqLrrbJZ2Gz+svz4I8)cmh}J@Y9X zQxF6gkDFvyP%g|>Jp}@0_kbu8aLZY~gF!zY>)m#fL4UTnI^R92RWkpP&$!0CN7oow zLHVyGGpKY)zAaSnI#E*20Ew0vo#xj!KhmWe^R}W8N|){0OkQo6tB&-i2vWH7&`#NEsHIQV_)0P|N*P_r zEl3v8rN>^YBzviJefLv+&sj_*>vYVfQnM&jxUu0>*e1=7(qK?nP66PF&h=@?U+n#9 z!eWOR4nQypm$ZpYJoIiRFYrUVgq3!kQS9E`i8zb}6+c9X&_Aog*T>qR;<(IGJmxqK zvlwoJAWp}Z7}W*fLiI^E);E6%()fbyrTr z64zl$76_Rv-v+*}@%BrtM@84w;%zw%fT7sk$~g)cbg02Y6L`~znWAHc~ z^V7%O!{oMmL2{{+@UXk1m%5{_EX*qRI^IR&uy^v={T|%|-!mB!3;CWC(8$w!C6f2i zl$Ra*7uB>kz_m};GMS)oI#wxE$Co#`h#q+JP2oYE9Er$ACW8rC>gGv?4$51(w^y6A zNx{E9F#fsXGZ)q3tO836LV7|N^m=xNpN$faDPd@?AT_G`oezWlKmYG#wZNdKs<@*y zo;!Re`LwyXP~2-$m(%8KdFKw0$kjnPg^2Y>;Y}}2OnjI?vQnw3F*F`#YAyTH5yZ_s zZ8cIm9P|IPf}LvmdPGa2i-9Ms0l2WsQsJ2^X5 zDP+%zkX|#IMIR1gHG?33lD)!*u;Qm7T4@sd zdPA|pu1)?Dt$h!4LM9(1pl=fd1RoC$f{!}RQX$8Iu<@hB&t|&=`5Bj_NS_~@LW?Yo z>f4-;L!n8_B0?FOHLC)YONzo8wF|_q0lNrCJB&u`)3fBEjk*422XIqI*lA+}~nC*S0R7~uNPNi(aqUcBDqVV|enbll8N z>&3;5_S4`rp)dc?xSwzK3OC>kELi=3=Ys9DPDK;mZ6r=X}HSesXi<;-DG=2O@(e(&x$iD6b#(lW{} zlL*={YobBs)Nxl_ei9WU4zbQ5mSqMwZc;SBI=S2^!JgF@Qy5BR63ceFvtMBp2A!KpEDI)2|={5PsW}OzB^(;5o zEYWOxnOl6EV>z7_z$On*2O}e}cK6jV+Dgv(n%rB+wwZ8fNA*Nj(8WrOSfRF+;|jyp zC~^mhJuoo1z_kPpb?|#L-+O;N!38TEP}%kd!E{zlaaF1lDEJ}j^3=QK6r67aB-gbp zPK8Y_bz%#@_ocbbVzx@8a6}@zBelv~x!e{vO%9GDS+|nl3!!dzn@d#fBuM~Zd=nWE zMlMz6Sg#oSbc))Ct7h@0@Ggl>U`!+1dYjT+%_!=nfaoP`Ly?7vy8(65YIWOwinbld zIdIy9f>2HN=$sC$FH=e}Su1AMUY~5QD~gZ1-L>y_&hC-HEcjU4UF&Z5S!(#iackAR znfW{(9b5-9OFNa6+4QZe5uV6S5cN$pgmKDFQ5&=xV z)nPR$uyl`bmqk+bL-TnN0&{`-^n(HW_P;sAur@O(?9!&AEh@KBs%l8EI&BL40P7#C zrcHRlud@y5yc*#V;`>EIaPS8jg5~%o1E7|G?LYRSt!>-tNNjkVyzDl5eOa-8?R!oL zs`rGlXKgM2tm#;h$3iw&_sFJP>(6u)Rp z3i=XATe%wq|2p61Y2LUh_4s(0La?ff00=Bdd=@VuQY zVq{TZRoXe9O4-#=c;5naynDVRI<_awe94AC6U2e;gLCpC-hRLL_|XHMI1g}1)yfGD z(*sv{r|aBdI^QGeKj$Xro?REIFgaTIXItqA$<$%6HXWfsC*}8xH{GmXjCT}onywbu zEswUt^~!X&*DYz6X+RbC1FLjh4@w>FxPb`;>Q^t`1UhN1WPO!1WqeIi(2>;SggCr` zVtS1ZYEkA^Iw{8;yFsVW;dn*BN1YWX;aGcyP_fqZm!{O${rcQ>`mU(q6H3NZ+)P9A zI>0TdGXnJ&eYv0RbJ3S~6rkYwm_ighwB+T8I_dYI|H%jpc@?NHA8vNXvbkvJ zmU62%HEZp*n~h{Fh;C;z(=ur1(c7>`KpS$#tQZT=c0^fh`8h5y7$m7NRIagXRrJ_#^C1+l5W5&qvHgH@U4zxr(p--1%A+`a^xP zC>Pg<#XqW(;$XB_{QWte`{+`q=wE``x6t^vwQ71%&(vzJSHOPx+EEs5p*4vXOkatVZgFAN|ZhX(Z zHsk)ecWq1@F3kWDj82+&{FC=4`k|KQcQ5{0FanE8TCR%oW(q+U&Z`J)7E8id=3+^g zv--c)=7$Mj@_~u@dCM^dzq5J$&!=kkb8d7d zH9*(+7s6*eEd1Pnw!xwLhVHxtWf|%^DqbB2EnKL*vHGrDm2@eozk&P-FQM0yYc&A% zxWJ^#AiIN5*6RgjyT@9Umbg5f=vkr;)2p&9RPp@bSp$Z zbeyr*@t!tb)SP#BxA>_#tCx!Om$2!Sda=!?mnOX5z0v7x6cKFyqFS9dlkocI-l$&w zZCci|7sv;6s*n@H)Sg$010uDE=%={vg`dQmR*U|HrRy#dh67a68wi2l?_vOSt|ylA#T)NLBm}&YJo)D< zqJrZ=;rw6-clBo25jQSvKAZlJG%x^z-6 z%af@(T*#k6Y=`fO&vtv3sU5QKEhp>1;Lu8dN-{vX zU})Qs5E1muOi)IMKEZ6LP3>o#)NVU+ydmojw9kR*z-WHCJnvJ@>!SQ1n51F|6v6>Y z1d|sWpG8suXNBh@rxmzY_4kHc-e9DNWL&QLuc&3m!ze8$Y(*9_buNep*lCo)|4>5U z+Ziwh4^tE%wA&EQDAfcPo4gQ9T8O1uh@n`Z-pb37B};egP2N{!iZ<>de@UaW+g?rX z-xo=kqh4b~^O(1oVWhd5<7Alz0+%raB8SxpY7oLxz(ZmfoL6-StFNHK*8xX(tzN!% zTq>&NxSaE11`(v%-A^;-hq63dpP|ZF|qW^DUoWAn$Rw$3`8E)_QAK&*H999*q1V zO`c%CnmGL6dY?`My@7u0-n`G7^FHj(W<{AjdSB)Wx;@Sy?c=W8!Hz+-iuQvrb9d8) z)Ip&PPe93?#SXv$55R#u00*51;6RHdxb9rblM`HN%X&7hV7;*hXr^XzQKwbW+^4=n z5^juK)!@UztA4}r6JvPzhe>aJo;O!r0xc9tBx@*P7H?BJ^kb;&g69)$VaOU459b33 zj6SlA0f?8YdUh#nYuGz!Q*#SS?TLVBNNa<3j@*8cydX1!wK`HmwR&XXBh@b$BEpbm zLFHopYNf`&0hZO5Y2Wfy4#mBtsw7&$9?Qf6@tto*EHq-jP-3BQPH9@9q^-o_?;s}0 zFOgQLPcpGoGdQ;2j9jP;K?weg3C8F%I`J3oVTx#MVV@z7S%*@%pE9X%FK(GkshfFN2@p~BvE+B5B0&@s099OKqOP-mu;;s58(A9{pD^Jb~(UbkDIaL~{mVS#cY*Kh6JDzuQoSMXob{?q*)6c&av ztEQk=n)iFJG1A;?b%tH^#FJWQ3pl(663ZP}8TsG;J+7Pksgm8zgj*Dwt+QqoPdYRz zCr*|O`nEF@V**_&)kqHDLe&Njg;|sguVjUyOT@WYlDE)=z`{yceBTFxU`99bOIc?y zhNO}CTTF$wK-YJ>(e+V>sTZ1U3P{p5$nqWO`X&U!8$q5}D0;#LD4#&BTzprqlqfoY ziGqO+8(?S)jXHpjfs3d|Io6O3&E0^4b}OnOj z;|(JFMSXTwE&6KpotNNk)1zF#Zxjz!zjQ{*smCkLKe#E<>Sos3Nfv;A#`VK*3l+^aXozkL$ zkPOYjw5yp-uhoAlTr&pCCu z=MX-}FH|Qq-%-wjS%J9FSLI>?x4!doRiD&&9vdHhe{A2IDh4f5V4xo zl;sSX$?jPvxy}3Mf4?RehOS#gyB69CJr>FvU z(Lfa(Jd!#@e%#RNu$!n7v6NT;)eDB;zk0_bM_%qz0_;A1Ws!LxhCt8NdWo`Td#xZS zY@=3#d7rAUoYfbKBjB}srouK?nZkO@bP?2;t$hFUUw=@;VP;sa1G^&4ylQN4k8`A3 z|3u}Yw-{0$(fWPzgB1D)r)ldUm+`8H?dTn28Q?|Zz2xdV^f9Q}z>cLT)r71MNEB-5 zdrL)COjFw1@n&=8)k}b*;9a+lL%qR+T9nD;CDMn?JasdyJ`^+xCewYe z&8?U7X?cC9DxOsjdts2puoQ>u`sHxs>%(GibdW9vb<(SgDKh2N1pUsk#|<+Z?y|5I ztJiPQeEVpJj13T+PqPS|oz+B|9Mh-h_LEKufi$&?FgG{e2#x$@!c4~X5AM@--k`g}W;uJ5by~cj0MlKnjQ5UQBdmBWex^kW)%BT@GElAl(xR*wADb5z zM3{%A1!p07O;AGWTR!+CDkrB)lN>V{^WAgsf(REcrUZUZRN(QlVqnCZHFZk1R z`V+Wup|2twNFjnKTkV8(P-wk(Ico}Q2=B6?$FOs#Z6%=r)AZTSmz|=MTu^t+u~BbJ z>ncqY_o~>v9<3WQnQ=Y!B;8cP3=anH2nZr~VL?|bx|^AUJ91kz%?67ac6N`*nWX~Z zthO94n(1_x-Mx0SXbBEPM?1I@rbkK1?jg~zA_7$0@ujPHE>Qvr&OT8261d;&>}Y3i z2Y)kI!_m(E4&h{|zxMY2ZRhc$6~Ir3#le3btrm|#&T;dkgdQfaoq`1#`t?{RB5r!0 zBpUUfh1`b+)FAaQbS9A|fhUn1d+@N@v2yfUgHA;#*%}J@uO1Bk`VHWbxnT>i6t4+^ zPCBz}U1t*4l2}OS>q6TXDlJe!ya~CK%XpmkulA1Vln8tG&j`hEe2Kh~_l%Liqe50|v;RQx*G*X? z`#KyK^a=&Gu|K4L4~CpC(Aa73_GIVbK@V;UMSdvm6jYVhHt;f8`~%)6k;SlGUM3oq z!_#`PL`Uk@GAOGl14n3AHwVFsfM&n>7Mu+TqNq6q*Uxj_N|M4c7yJkh-KLhkTq~dh ztl3hclx@(V&M(3~>4%-!6Jt-(!!Ka)@9XC5-hA*MAL(Z5DRrhI_;TwLd=Z&y zee>;JaP#II0tnG4Z&dRsPXN$9=>)K`+#fa8%@n}3Ty>!^I>aajhPh%>AKU{pNLvF?GYu2`M z4?uUHa1F_uEL@icHamPjDxL6*Pa9p^#CjlfT%RwRakT^(p?I>{-a%!jv|#k0*cKqo zr!0awp_2)2aC*JsN%0bM_InTbCJ8@7cjnV_T=gIP*Ms*5!w(P6hJ{Kpz$;kUNTHsc zHU+^;;HcqM$xl#PpURZ4{~#McN43F6z+;Sh-jGqx8@0uT`PJLBLzNlBa~Iv_l_}%Z zn_~^Qp%tTP+h_gg!Nq^gz~O%PQ+4+2-CXVaYH*`U)QGAX8w`uq*4)5vEh)CMCVARW z{Em@Y)aSYra`^gBp(F9worh%L7*|+c`s-n(s#^gom2B-k87IwYg0|+<4A*Q1!(h4m zn<9$(UB9ZfNOS2&JMD^`-kX$tQr=2v?j#tR)*I-TkVd|>*=I7(08=hQ@r1u zzw60{=ELIK7@xj4h*Tw=yG9xj^v91aNQW4cs5|$#C5^MdOP>cgE;$&NoI8Kad+vq^ zXDft$Nu7`o+He%^>t3rD(B&WsYH&(4MoQKNM7?dM zD0Zxx`C+lUzqgk}i@(VY}0hP$lA~VR0QLW~o+Nd#Z7W_vl zpvW6%2DX_!ZSF;LdOvryX|7GzxFQYr{vKph{1?=P0CI6LkC`1C?M*H(F0LckO+#MB zh*jFe_BY+S{{5@{-|syX8{c9rcrz=(8(If;vwQmSY4Ze6wt~v(1_7SU{vT=3d|1;5Cx7qcIR)OlD61%+)j}Te##VMp=P0j9p zFen7SLJ8U{ySqkhcXe$hM+yZd(L@ezsseldjitBF8hk1tfiw_yc6GYtd2>0PsA2I+88??R-H3cNh1PfqsfY^9U)=<@o#4qt zx)m5d(OkB^I5U4sO{~YwEce}pi;xCN@eS}fr`HsXfHEk$IcvOv%EdxQb<&Y~or35q zp84^dstDu(>H>hg{2tjj5mJj3Rj3s^UM@Kul*Sh|!PHHgMY0t7NJnLW?dc1V3owBxnEZgP7@{iqcwR88lZ{XJoA-u+CmrELQ#p5>~j}lGAaV80LY8n z04DBh9h_oD81leODSJ(70!S376+|_5{3C^)*^o%wL?`8DldvEp5jXZ6aKq3b zP)t5J2vCsa&}ts~1FezRPJ9qGurnH1wgL3|I2abM>^9eJgow3^H<3+pCY}aW+GTu$SIv&OeHx71_(oSW3WZs|a0d>Aaf3q+teBGSsl}ZS2J2P3?04+m37}X>j0%kp4N5!E@Bv9FEYbi-yCStV zKM?OWEF6Zs(%V^Nb(?-(cbIhJrh)s%z?$8+t|VrREh%YMkFP86Bk_3F`9;kNbNy3Q zu?0P}z3}Kb_N~o42;14V+kf=%Kw6Obm@yeP+4yt2Irc%?%hfb?jsYY}6**@AnZ|s< z!H}ut=JA1kGt|-&pyQJf%lkCBxG1PLuE*_A(l&wz{n}}1jo3h+7w_2;jcg@q&~J!H zemyK_O9X^c^KXNEwxrg`QCgez$EPQ>+hj(nC10p%O=_|#D860W^x*4UkVuywxpT24 z+zb%9L6DN#2+vxMxO(KPDSLB}V$XHCXw&PZ&!QN#hnw1ekCwfMwrI&E7aqYQ17B~* z0DhC=$a+XKDVHDwErP3LRf16n#a7du64OS7os&apDj7%b;) zm6L(8HqfSq%H>vds&z(J+qe3u2KU$TQ^u!&T38T z zL3zDsbB7!;Ana<}JZv<{w>ohmdWy=)JwanQ{e%1=TiZG%eC(x4kwm@$y0HT^>`*B@ z%`6NiE6U8d2bTSJ5dE}ee)S{sd@n^CspN~e;eBnFzN2N4D(YwQ66npE#YH(y1eOZh z=2A!p53BHT>!qCC#@*Z+D>3L_ErmpKuIrqZ%L&BS3c7O`DZ|(l%ed)k9_0z_+ zG0kRUpZC%XA9w62=O-C^*G|}afU-hGa8cx`x$C$@ewH(YY!Jq{pJ9FMN34f*d%Td& zdU}fGQ(XAaVBd&;ZgAqpI@r!qIzf)Sm&4*sRn1?_tC{|+e|MTTcs_I+jbHC>vJKHc zEERG&ek86FruV*sKPNo*Ai#}UKiIf_w-Lv*xR&s*ZnNw(?l5R&4srV`=Q;7J8-kTuIVCTv&Z`!l_VuA3#wGXa-7oMb~7V)_Ic*4_d-nwwVn=n5!QD)I}UT*6ik>QcF zh#93$w|*pUSL94dJ&{oceQbr1H;xV(z)iJA>uAq#WZL7nu045OP%7GR;+nMS@hvqKVffp)x-J*_78TY?+ z*QJQB4ZbS*EtflEp-vXZr{v6Lv;ER>@wJr$mrX07Pqdmn9?neoiFwNn~9^$-Kh!q!7%@%XLZ(VGj1R(JaMt>`$ z1pJ4|oeMQ)^i(YV?UvXvW`U_8IXszVz4jb+58ocN#MYDu>#qCY#Vtt2I-t~_IFx&~ z5xQiK{1 zdv+u+ExI6DIDs(XPTOFafn1@fKN$@ZiRIuXM5__YLfhrMmGZ$F1R4WHu#MR^Fj!*! z4Z%p-xq&NnfV__ZhG>^*MdkoF{-#IONtU|NX5oS2Or^&VLA(HAsBj6Q zS1(P2lm@@2N_S&?sk~M^HcK)n-FFdj6Xo~clnA=<_Lz}f57xW?XesTsC1H@++dN-C zr5FtDlk}V0(Q0agcjJ9iq{a{wVwLHlldaUz4MKpv z^IFZ|UB4)u)TY?1m41~`>0w(@IRY^Z=s;iUW)ZZKK<3YRkRd3)GTEuBnhMz*|7M}APa>o|F{645qHKDlgyy|M@ z=v8ktK{SrJ0nw}94HA7`!LD^!>>ogwXdN;de_W^yx1Q~yPYec;K4}(c4xxS&%Vt_n zim(3HpWuJ`6CSZImnQkU-Q{^X0T9YPl?DDEvVL`Xdit>K8n0lb4Lrb3z$bk+*=3t* z(&3n)bhP63rAh@gp*rVbp(NIl-F(_Enw-aBq)+^DgD<~N&bXW^_n;cJfI?|%0?8o}0UQE|*t!=Z|Pbn^I5<)a5DkHwho!6U~|PY)aM z7gYbGn2)0c$|fPTQ7Q==ex^R>%g+~$iYb{dI=9Kjw!*cg2yHF8+?f}Jw|`K_96DiL z*OS7SWv_WDUljU|7XLmfvNyFwbXAL4`vF;P0OWdb3ty(iLo-;dUN*LtD6 z1#DZL7(0rm-L?Sa{)j}=>y!bWCv-s+$gq3>+JQ*li*kOnL;nWXLlD&xqkHlGML!IC zdfW>(X5N$e#-?)<54-lK!;XfJ-hwaf@#?1I1!YB}MQ4D=JtJ*Gh0uML{Xg+2^tXIf zV~!>4YN3y@6jbi6y{>uExzo~E14j^snhsL-IHl>ZbuEIuR%n;i$req{xrs$Pg+bbi z(4m>oqqG&8(|KS;!YLNpvtthVc2L`gWlJN10k=NYfJDTFyOw(*=7YJTx6nKgzC>z^ z;?RENL&@!oru#|^TY!fW><~b6AQ&OhyjdDM6K=Mk-+G7gpsO=WtGEOW7b&<}^;>E*AVSx?-+PTh z)N2$cMPIVc$WW+Z7t0+Oojq{wC=K@$L29*XKVi6#%9;(~rLP1zaq>uhn|qeovK$iw z?-BzY1)CMBTbp?`Q%RsEjF=k7Wz>Ulb+9VW`o1WnBq3C{H_I9(>ktXA8|)^Gg1hXB z4)d|aCIrr!0=o#&9+If8zcdDM&zM5|EnYsiupsYtTaZT`2ta7IDImEZ@p?YDAT4R# zglr6uPAf!6&b8F|2BXyzV$Ry1pBj7CX$c(Vi{2sZ!Sq(6{92<;2P#M$_5f2Ti{=w@(O&Ti;nZrjOcgweRoz!70~ZUGj!F-5_X#|g@y z?_|pUwXN%pS6c~da%02F5H0O@mg-H>l#0yY@>(y3v?zIjkDH`?LeC z!}A3_habioQfFT>`GlkSs(~pVT5SW})nbmT106#EFr|Pe z^WP8ed>HiU`V;#1;Oi|tcfMA&{!pJR%Ek3z@sH}HI2i2}e}9gwn(8b1mvCY$bkpl` zsU*YMd9|olKm-FUW4HKAGo3b9kdhn;moVq`teW6=e0Eibmi459&Y4&G#Z^5~luPz(NzC}(fj%k{{qgzJXT{GiitqpQAJ5cp&RR|; zDDEo$c~Py-8^rcP`FGY#s%3FI}iA)yYfL!9^}A zwB#x<%j9fPo)zVcbc>t<8p9W?s*a0#Iju^#@_=zRtyp0M7S&NTh|_Dvz%(70R?gU5 zs5M)YWCa7crRTw&JI>a1PqSD=;4<@lZt(ps+Mo4fv z{r}nf^7polB+=jVSHMapBqLH+oQt-cFY&SQ+t|+9&d!@p@yiF1kc5~7xCCff>+%2o z>gaAX&<7eM(S7+ItZU6O|H0rxgMyl8Id5BULpA(X@C>U=98wvigIr`m@2O^bz45b7_V6auM zfYEk&ityArvbu@Qc`T?rZ&p*a&nFk6DscU(Q8}`PmR~L=S3*YN0;ur}rg#jycgStx zpH^QO?vW>67lTFXD%Aqj@%n{So-xYR<%Y|*T{OVas${LU41uCjg(UpKSdFDXH=0Q< zQ!7GdZ337O*Q}t#g|z%saauM#R>kcTcsUVT0GPoGb0uy-6&0kRPNQUzX5LSipey9k z6GoU9lonC`*rU@HKMy0)u~y9BUV#c5zUEa;p}1yY!bLw#amK0HtEMoFZQY#KJVY)6 z2SIWuhA^Cfu%qZnOb#c4;T4!vO$Huq)jH?dU1gMt)=juRL;eA6JxBrU0%{%Gc#1Jf}r{?dv0XIc0%PfG1qNNcgVi{tC>V*=*-j+fl ztVrZ7r0j!zh4_-dgrrRZkZ$J6Fxg&8p&OA?CuYgK^`JLs?}SDS1du8>5P|k%dMXyc zRtESlef_*0qq5w!gp6P^k`Ac==hVbFHMnpM`9c69QL_HFPF?NmS*M(?GCFm1mC>o2 z8%{;U>}7CDgllMhDr+KIg9LItqJ0)-q@j}#Rjs;5*eklyQKUu@oSDUL#o!!(8?)YL zXtBjlDLv{E)@xZ{RH~^(Wu+L}D%JCW<+iQLGh^}#MqkA>MU}Irx}>PVp(}TOV|QL> z=`Vw$w5(=`7Ng0*>Ql=VY_rSCQpYR{OS@r2_g&?D!R6q;(*Q;eMtO8S%W-H~4jxGB zs!=pwF4~d_g|FDuqHrbejg2=XoL*Ej2H<%gh*2I#C-jHsa+ggPxgLhZX*yW{7#W|& z)hluUa07>PvB+o1-Yh@w_wGM^`sC?2x*KPxJYy8%eU9D-*bnr8I!z`qaCXszizx*k z6%sRz$KFch5lizG0ba@pivhw#vearTGnI6zk|k5r zzG5pA3TF!%m8$fGZ-+Mdbx)i=@Lx-t07Fxr*!L`!k=)F2e_bQqHO{;yli)EP?{ETcbqC<{QJh-76HBa`Zd`Og{8q3vdNvKP6VZ7`X6Bf2 zY!CNp3ECzvh{XsD3kJ!r&Nau3dH^;whi*hvLUgBlu>}ZrijgyTQVzf~HB+Dv!#ZC$ zPQ;i`#28M*CbJ4}?WZCWCbc6f<}$X~oY1*2Uxe7ls3SxP^;(FW`Uf%t0Agc?YObw_ zWsA7Vu!hmk&9=k_Xu z*4mYh&wa91nuK<_8U2xxcsaSiBNMFwl0sgc7RHdb!))uAWCWQuH7JV9 zt3gX2Je$KF_*|zO=We>el^>$LusC#q4?+s?*QzKZyek7-lEL%=$7=zLL#U18T9gi9 z%9my1r^Vqdr(UZM%}UU5U~4f%MAVR=TZR#Q}6uzSyg>3JkDQif86;9U|ak z?ZEp&r`i%_3_IB-k4W7VFd&0s2raMrPk4Yo?PQ($UAw+~#-Ya#YckUv3I&E=_N=*L{ z>k%Z=3(?!qvy*{z+C#Mo#gS<)0ZmI8#4!c8NYMyY!_m1sPOBiqJZ1^2X_ z;R?H@xS<9t{;*K$4~;7mJ>4#2hE?Y=277dQuc#K#BN*3-S3;nQSe6u~D6`#)xkd-O z=mY`2z@v$Q@b7`@j>9$0t`?w5YBl+YYqS?Y5|J+H)5bc63Pq~p+Mo!{o#ewn*MjOh z_M+j~9M86=yIx8{ey__dJ7Hh( zBwK&Mx$eij-i|#g2KHW;P4vN~4xCXcv9=6;S#tTOxYG5fQ@bHcwq{dnD5X&<1mRRx z*Y=1TPGk4z5SupOW`G%N#TiV-SjlG2N!gAq)1;om0*wQEgJx}ftAVKZ)tCY+J!vZG zo5HqL7o;z}oHzgCvt3)3SKAJG)K|bbfpjK8)ICu)Q0b)V!cao=Z5%{Q=?l|}m!hB# zokS6RRxPDAM32SU(veN~(8SW2 zPpS|K#y4b%DJ(UudSxokCa!kPZoBTcJ(;5Da!ux`D=8e~yaBZ|+teV6DyDZ&+U-_K zVf-qfCH?9`N&jq{iH1KnRc}8pT&~;#7re-@<=i*2(~WeRGKnViW=k1xq~~-@GqVZJ zz(jz>>J)Px&4AMWzUUtv9Bg)?p@|OD5G*yC=4HW2n?~g5*pu05jM$-!6FYZyMPSM1 zVKr5#W+#`|mw?F`UP1RbCq#ccjLZpaC2bOMewxipwVv&wv{Fnl74vAz-jO=XvL4rQ z%UEH6+9U2FE?(KGCqlNcU4c}iZ?I%7nyC$JP@D+EWBjOyBX{}ZC5`v;%j^Rnj-wFb zX!7tMFKy&ueWHxMHVw)s!lipiqg!pXDP>d(wq3et;>2%Mst8aigs2m^P@q_VGP4~A zWsw_N8I47@UsKx3dgG7PW=pRfXaJxsl$b?76zheDho5N%Vsggu-Sm*Mi*r^%PU+D8 z`A;O5A}*j7o>@dg(SodCRcSPCxaQ&mwUlN8FBKQdBVio%X8uZ#_Pgh5%GRtH1VCRszgp-Y3{q zoNNN@&|S*X$`VDqXwhi!Twou4nu#qaBAVO;-Tu-tQSgDxWM|;L7{9Steyxj%wAEC0 z07Vf$RB%;8Pe+sM)C_K_2voQCe3DZjAPn6UaU_Ulbl%_))m@^ZQJAaFLD7hdw8SnO z3U}_z&t*z{_HmkanoR*MiV_X(c~+b3W^?hb8RO+6H{!M7ihOLO79XpHl!q;sORxb+ zkrWHnQC?NHEbP%{iw?I&r`Em7rt1aqRQ0SuJmuFHHBzu~~o>tEcL9O7sr@>I#GN^;V9d2?y-vkEVG!>iF0 zmvaNY)}+lPLLwM5B_@Ms%!z?HuZw9mEh@*BX#z20Fbc%NURV8A znoae_oAiXXD&^)DK{Zrz=x%Ze?V8CFHvPk_%>O%0F8h4T^G%aUGpG8kV}2yw%Gr^> zA3y6BK6H)o zq}_F*qh*Gv;STNBh(4u@)`Mnj0|dCP&s}rG(m{=JK`pBY4Z~jpRV9u%BC^5S&K58A z-7R0$ZiD1YUN3XO?8u%nR_HTW>ouCD zubPSrm5|bra<=P*;@KH+VUb9j#=rUlOoRZ4fc>n!vnU+6{;tR+j*JPE1tg@p>Nntr zD0$mqHBaT^ES<+?(ti{6cqyOfi{6;OsTY0BRxY588tv>yM@C=JIOv;$Vbq(YSrQvb zaqXBi6;Wt|O$}O5W6N4R*}b6a%6DoCC-P{JcEddj*QRxwRFQ$VXeFZE0-E#3Q8Q|S zz9tB*jR}F5*421B;=+*c;h$vp1QB=n7;GO$nr1VU42@c7bjhu`#M!~%_ARGgM}rIN zu?Hj>Tk5-cQVLzpicBnNPnU*I4Li;F-1=i(K6mNtnc)ZJ6#r`#MW62qWF`OfB7q}R z<~gIV6+>Ivwk_OFc2ONUbXe;sc($sJQJrtg9*$`8a7v-0$?od9E-S+*hw7qEH$)HZ zv;%8U6qB(W0==PRffA2S=!-oG&31_hC;y-zqVINX&vVT6&~OsI&chy)LBTKC?u{x# z)H^!PsjXwJ;ag@IhyU*NpZIJ?rY{GQdt2D{bWS2BnrGZYE^{!zLmf4c&0fo0_%3V) zxc~MmSr`d&zpvlXnT#!Y;?j|l-jF!%MCedJ|FsA&0aB?3(s`Z;%Y&b^+)8a_^;=-T zOb{0I-uddYpM=f0z+vu;^LC8u)OPD~LygqBLcW>M^GXdX9BFs*4jQ}W8*-<&>ztb2 zZMM`%r^h>*WrpevI<}B~#SZ6oIo?_11!a2~NAYQq&&aL_1j#Yla?zC={dJj6%ZqVz zxOjal`rlE?XUypER+KN|Nm^b(p`Cqy$C_R(5(}ZRojby_C0YjnyjK9$sn00`TR`;0 z*ER;pvHf75O<=*z{Y4XtDMs{)^pd3L-r{x7WGY#5Zd6j@G}o#lx`jIFb#)q`y~pee zgeKT|%%2tQN(~IYHKEnP1sOBEQZFfJ$GnS|hs(xP>qVDYER+6(VjA8y$vrt&WMUSKR`pO5C~r)G(0R z>{M{CVOB(Y?GsaVti#;Ejv`uZ?4XyY(?2Oia2xIU+%F*RSd|G@;K)pa5#SP~;S44VDC1M-$YkzKwrJiHjdS|R`ZvF^faSiHVMZ?`QXay?^&P93I3kJvkFg#9{>ebB~l`t^5{4*%`RF1P= zm+t|_f-&Bb?%@;n-WbLw?)?+@z7hA{DIkf|6IgsjMK?&ztT|BBwuV`fd<6Y^hjr8r z>uViauX1x+o=*BQx8CY{5_f5FA56r_*f_(2E{z3s*}Anxm5`mZ*>bu2XY1JPx^!EV8p9IJ8KZ_w zZ8RG;TmA+Yme?@L+Hp=X=$luYf6Wb7P*8m8p;e|14;G&Rtk)MyK8l$<+Ju{={(18l zaT^_&Ky4I0cVRVEc0u7NL{gxc^ds{%$tYn$M}0R)A)|q*4&sSh%EBJY&*}U;;s^ln zmmO=dNM-kkKSUx#TLpqSL0+3+!e0)c6XwSSRIb5bt#$bRp3+QWT#y;8$IRL|9TwcW z&-j=Cn9;ZK5&jtn2r~&*2~XSdU5ilX!myQE7>(>mErN6_*!EHcq8UjWA7>8BMmkV* z*QDg1JR#z%lQ?<3br_MHDgje!qQG(llJ66FLf}l{xkqUU4^w%lGYNsWzI(GMvPoE# zZixgeVNXb(B-B58g8;y6aD|pB9`Kr3RZQ;JMe;0;%w>0m3QiErvxA?^zuUXhFGOt0ov1sAkmzUwgeZZi;DJ0Z(P$ zJ%7>{ah=ZZ=?8qe)i%XFnyqPCEN1Z)W)9AhTddZwYbxX;ZzE99^XXOPQRJZ%4w7(( z=l9Fc55GOQB__6Y>S{Ciq0E&;OQO8cy zwoAX-9zPmep@P%0L-AmOlWBT7YdhAZ7sFbIuGZAMIpwtm@3stV;vH4wJ!Pqx+a)eu zgkseqo=(&3{CMYJhkj$Q3He(r-5g}LG#k;a5iTR)(N~Ax<7%Nvl>jkfgDbNHY@-tW zXyBh^K-Z?`HpOVeJAf#c%Xon}hW}bs=V6T9Eg$2f)&ZY2-1M0`^H;Vn|9N$avn_%m zwXY8T#~DVMzV6yXZ~REF^+RVw`)ZdsADI&Z-|ve3Mk={>^EFEZM2>MRA@(VVq9Za&4{2$L9M&svj{~kC`K=U5eb~` zeb`d)AKqU3w5Vk`73iW|r-+VO zww@S$6TS6n2&}8%65pMRfJ&L6t1sPp@Gdwet+{u{HEE4rH|Io6*(oF0S5!ZdCNa^+ z^DB(zGW770A3lZdkU|=XwCjSXZah<@|JzKWXxt#aPseYXS4hg zE&6zIk^N;jxcA~TOO`T#{6+iYN_gFl+GPG zhd1A|t3HQ@9#VD3P*z!UFvTJQulE;uQNq7u02(is z`DGD-m{F84BS3u4&Jwzn6)dNMqLMGeUei+|h_6ukqIx@wt`f|aE`JxIm}m|T&smQp|rBc8kQ3`&}(bT_KEf)C_y0R+Dd`?tb0$L`Quy)Xo)tMX$k1monHZ%E0Up63$Vl4>((>!28jQ-)l!Mvb&vbik&2#3z09pKkblsMyVzPQ@|EiRxB z*n31TOiT?BX`%orW-Ha(csh+#gn1m~~n0$p!oiGwT5)=l&9- zVNEXLr9Kw5t`X+eS|(zb(BWR-W`n3IZ;SdQG}TSm_X}831!f%mp!8;uQ#Q?MU{d% zygYlF{Lc#MLcLz~)>ZFUq(xD)z3Q#2-hH4BOQT-()>V(hS)*R{)>ZGPsifk1z3Q#2 z-ecfx&3e^a_k1F6)vQ;&b=7->N|7h8$fvEUSH1PuTa}Te9=>(edpMD^r`N0Ay6Qcn z;9v%x@mqT(HY<{oDfUpAz+2u(`hJzXG>|T!Z=wNBd?3bB-zv19Zh1?{ZO~RqAbndg z#PPN=gwJh-XrkK+DGUfJxN7abTH@A8qcy#yuMU$VK5~WVT`f+7S|PNd;1r2k3z{Xl zP5^hj#v|*FMws~p4pOAfa1O?j-&)L8=V>N6+p}!Xc38}l@*#Vg}&SyMq%#ISOWS^v*ccSS-y_YlwX6gJte0%Ahy3o4IZ9 zkpzGu6TnnmLFo}T?(7+b25@z)!Z3kZ$C%@fB7l0$(?6|*ClId@C;F{ZfwTEh`f$Er z5hj)r-vpB)$_*aR((+2Q#VU{IISGk1slA#Xr3vZxpwx5tic0tSmH5jmv@rxBoeg9d z1rR`sz(^LAp$cG9{GZ8Hm0M;~E@!nD@vMa3)>VTt0t`yVG8RKUI0I1a`U}+Dx^W)y zeR81)?;Z;McLnf9TMjRn2|B(+y_gOR@#9qgF>xj>yaxLnAGSNPJic|JG(L@WcSQwZ z!lf1=QVYbIE!1&60PgKU{@T3+)DWp!wm#F6v#Hw-kd^h$V>rrUsY$|q#}iC#MCu=8 zuuu&TyRR3YPI@~5zBW2d5PzfQ1phU7P~;o5_`S4s$HdX#bk5@i?R$MktHNRz>l$d* z4h#C*oQgVw(aKrSVsOhRSoj6h6_D*Lh%;+7Uv1d>x?vZqpmKUYeQDTqsYCwl-3!0DIB2>^1j{y&elR zEJd1Huln@)*k9DwWx(N}u9Bj=px#bKcQJ-yivUx_eG$~DbI2DExr=<3?9KA?e((O% zr%#@al@4=4s#xmcX@c6+S&CZL-0bY03`Yp`F?1$sKMRoVfO_@SBwPCmgAvNfvEyT! zxxbyw^4R@#o@dFG^Xp`_TqfDXT>$2Kl6V@AXA2-3Ac`*2iL=mYnmNB+B(E*s@R3`C zh^VaqK65yE8qb(GId?qidHlL-m7+`*jaTF7&|N-WmHC5wIgd-Nygw$kT#4WK5{0YC zlMK$N5@;swS-n&{v}b$|(*li41@&!`7Dy9Y`o^W)0H;u59?za#%}?{0oF4;Vt&|%W zdu?f;tiZHVo^|7kH2o2gOv>S1TomcKJ-NJ8 zxdP!jO_opZM?xT$nXOOIcu$nk(Q7yAz36d*z{gPXqyn zz61~PZ+^%W7q3Eld8KK|d?+72=IMYNHja!Tp8!KXBpT;FADGTWh|Qs@-RsPF->jhq z;jLkhA4WX{E}fUJ&-uBCsklsMGm_Qq`5>DBc$rtH1Kq+$$kEDN&^phk@&M?~Ci72-gj;F6s)E0%= z=o!~?k!01k(NTPf;P&QalaC~`@0tR>>Qs@4jBros9SA`%1704K` z08w4TJVSwhUP9q1yaj%PD*p>|N{v0s_EL*kJV`F{8Rm8vRF^;Aa%vQNyern)#Ape% zolR`n!3bRJ;D9X~zF)&PtN#G5Hln{=mOo0fRhbm>3G$j`+?t*#8&T{wNSS(rxOvl4AZ&Z?^(?bLv;0!bCDSa4DHr+W{sqSM!Al=# zZp31CiN13{YqIHx+_p|B_-+7kjXsgrF632TNz9nKIn_m*Znt7PpP%NN&|}JbsQo7L z;3j_3Pg|9SwRDzVEoZgg`c|R`77@$9IxUb1CI2hDuyjIPCXpxldXfd9@ep60rjp4?|LcySC#3(`W8L+3E$M+w{cPw0IJp>|J?wVe@4#Ks{bDGKzC8lpsVjN zTEYs(20Ab#2eC+&NUt4wAO>FfG5CNlMJYI)iq+{nMYhBPv~qIr0VT>p=iMYpUwHvm z-Vdz+&tjk!~qoOo$h(I2L z@3gvIXtivNMUp+qC_|Ab&3a?4zsR8G#7dHz`V2Kf90v-KfQjkV!Z#`Nc==M_e{1lN zl67JZMeu(a>`RJcQTxi55|Zab86aI2c0yRCZjkn*#9#UDU8?gHPg5Gxaa8YsW@XJU zl*9FLbciul5GCX!cv85icJxK^*~H>7b#k43WvcFP=Nd;?53vU zd2+Y2-3Ia^Xy|g6MC!W0};^z}r z)VC$P!^e&}PRcDW_a68XWEr=J8{AcU@9?TbR-DyOKbrtLkvQUfMZvZOWMu;~_sE~1 z;U}`<`PrF^tm`W$7nviW-mvI0@OIK#dM#ZAI;Fo)ZG^9M(;jXd)r4Cmsa3M0f=}HV zMh6%xA)BRcE;B^qfoj22dPR7+_kTxS$NN!1yfY2UzkseosUxc5jV=K+K7lEQgY-!7 z3TIG9=}#E0x$F%k3a7UeCp>5NY{b%PG~iXzT%VR^D{2}|DHFYWJx!aycwG{szgKPQ zf(`WrT=gNc|83xF$HdwSh-=h9Fh`rl%zTJOJQtjmijARFSCJALmtkl!zTGr^wYD1II*L*gK>u;7KS<;IF+9t57Ijh?YkGlbngBbH78!pnEd|903Di%x zGy(sT`iU^zfhNS{9E136B8u0A>n@@-3f8`&I3}&A{32Lzw4Fy`21Pdt zL>MA>L&XPBRzUVamGTIJCdjRTHHk-;ceWzx=yJ$*WgI``wS=37B~&-51|t&Q#h{QG z%U?C4C6pfMcUrPp6_);~KvDVkr@wMA$&B3B(WnWRE>O4$6Cxy+-xy#g_P)+F8|Hgm z*=3zmYjkTf7Oe$X7Y?cgR5#YC1z2aEN#|`kGfVG+j}cQ?F*9plUnp3LRe4`G7#|RC z5Ng)vUa1y?zj}_tpUN@@xs?tq@}XMB=jW)2#F(x~Z5KiX${QPq|b5LXL-2La;miw^K8a*`k3Z&<_zvW|DI zSUpC&H3J8Za>4lvkyEk+mN#N)QT!aU7{&w=2w9b_uzXKqw5uwio?f$76~7Kt4di#u zGrX-}{pe{jiSaN3-e(T81@uE`P)$t^HO6@9YK#!u(M`g@T1CAkqOhalOGZb%u2xYX ztGn)M;p{GfWlw%i@p1Z*Aj(#uDk(?CWl1baYfu=qWeXkL(a)V65jU6-?(WK zfRly74_;Vf{7~T0ABs!1zPl6n@F^;rVSnEOXK{Mfnot5Cz6>uMN02wcDuwt{-K-6v zJ#f`9Iyz`d?zZU~;@mh^iqd)b*JznEQjoIuKiMP<#ZCB7h~eG*{Qcqduq>1MbrzBN5?|IrVd!x4f~}1VAKDIb&2NeNRz=r+6)#a(3~|6vn;+A za?8rshv1DxvLm@R!XPMaw8F=W^Z2ylebpaC|E|63xD%+$ZUM}(BjjU^&SwW}rO}Nz zF8U^+(=#-%U(_DkPZhC;_hbDXMFL`7AhmB{nRqvHpthkFldap9Y*3Eo@doowa=Ow& zh!90X%2nw*{7m~o85C}lZtVYU|9qXcomS~uOJ=q$yTNz8Ev+3HOxew4HZ}*bnI_wF zL-OU$OlGhZN)&lS8f#0WM`-YdTPdUUL}aQXlSrElq=bf%+2@OsUOtm!!e1xlYXf?6aw5Oa$dG+}sPq%oBjRdgaW zIVxuJ(%)&UYuEF!F0%KLAh-VOl=huBFfeq@v;kX=o9M*1%SBf;|;WzsD|b97@0b4in~DIWU(UzLVF!p#!H* z`qKz7Ym>|uGy7L zM}(>a=Bmhn38V~a$?{gD;?j+y!<&oOw|Y^B6yv=}WhG9tS(+syiYRgmjW*BIY*glp zF`y%W<(sB)^!axWe)#^Ud$$@rd?s=sx8{IWNygorJFaD`Lr1GjVb1l&O}bEtcLk%L zQ?5S~E`J85iaf7E)9t(XV#F44=Z;0(Io~~(s6b=__v#4gO-3^F=m&U`ISPEsXrfmTgz7DZ-@u3p}FGknL~4S44+O! zN_(JG@me+PKnK=8Jw$R_5!I=2r<0qq+KQ<<{8aU)XkdlL++=_P%YK&pIHsaP17p zH}!3}dX2+B+|J;=U^-!)rLd{WIoagKg9NH_K6CIQ2*1VkT*+kxd(jus?dr&t9`uFR zgI=gw>B5jJo)m7gS-ua;{M?h4&qJO8n)zL8E)VVDrc5Ax%v4+qMbKuQTShr*#Pa73 zh-W3T$i3Z$4G-J4^)O!ZPDCq4g$m+VIeK#~hO--`La4i#uXoec0A8Q=k)cvgOXkr~ zo4%K!dD?D68bJ&v!D8o5G>WvI@)bInAVFO$f!&pSy=U@VA81qT;+)OW=AdPW?s@!r zRE6nQQM#2fSeWw;a2UwdU@UL!9myp$T>g`|%@9FFtRxO>mO8{03l*{MdPIdf=TF(d zO_*4CHN)KrRm4fYn%U+Qzt<1-<{O#{6bCEoI(%<|Rh^AQUjLVPXo_h+veMVk*FhxXLOLm2!u#M>Vr(wJknWy>dQdQSg*3L;q+zP7jsw?>Z^Q)xFk*9oG{ELo~TzWh>~6S@2$3LWD5=ma}0zm*ZZK@5NmINde z*jYSJXICC@TtES|??pr!iHjy{+1yr&YeFtc)67PGBt>4w#Q>X(4Q#T9&+k8aX0sxf zU8c(_K9H*`E3yG?ybMayDj(bn3As`;Qzc?z^xs5_i?~QA{3S*W!K0AUI*Qitcf2GM zK%RzSlta}ti)Uo#Ip(1_T_^CNoT%9LURvCx1gA6fdN}4-ZvI;bF>BmNd{+8=UbJB; zpA>2rQy9x(5{w@7*G0kY=YmF7?cez(jOi@KB(j=dQ%Pkc-F*dlDexwZ)5Y>{d!j@V zU4DItf{R`lxcM(?wKa2ADD9dS*@Kkb!!%JCHmHEZ`-2(bfyb{@7;A}Iur7iVQO8s} zO1N_A`fx}_tvO9D;#aA|1*qy@eU7SLnjArh0(*VlzileS=wy%^>UOIFyc;PH0}Ueu zd$pGw#EH=l-^fq;`Ydv?bkGGq!A8{T`F2=xt7-3>hDwE_EXqk z7{h=9my*VLz{$<&bGte)iuA_yEO>&MA4gv=Uf&AQkEy4J|GAyc&!b{FA+D3VC;hvB zq!csp7(S?O1GjnW6fo%9n?w0p7FRkti}DZfAA1*d?HxXFM&6K<<^WAqah3%c-CXO6Ugg}s_VAn+pzVAW{m=2W4?nOwD1(f5R zqn$gqOLQ>n2x$q`rl`xuiQMhoG44xHR3KV0zP(>gtI|iJG|?5l-so|OeE_=UPCOF| z0aqui7bnoK>gwd0I)*VT-5m=w6jS??Z-?qBQ;&!oH*6$EY)f{F{9Vsx`Q;d`2d7CU zkiL2iIFc@kw73;%R*YzDfHsWBJN)ksAzusm?F69cU zL`e3>VSj9Dku9h*>L7Gg68D$1NiA%$7T#ae&3NR~mZ<;NEb`1{yqk|k?Ddz&xo9udzEHEONR?6Ktcl40Oa3Q*Oy;KM|NYJs@2=k(YJ$IFOI%#t8*vP ztf=^19fa}O4iN{aM-Q=B=o}s#9M~r=%@(S?<~&qu0MnsnW8|8R3e=ry81)gUA~s_v zS~a|%eWO|%YqiP13}colDMp*iW6Nxz`ipP|6qW zzPTDtRO|F^bR_KNdag~fTA=7c(KfBUHf;=jh79boVA#Wjc*39~DC@KO!x`z- z^uDo^HrZ5&5QO;|nZ*prIdRDPZxNHQ(1)~PKqHvOKe@qLTqCJywU)oAh^H*CRIjxP z3h4j`L?UF-7>37CEr0o?4wlH0Jm8G>*+WaNP|d5t>?|RyQP^VENrpQdoVrn%-r^UFczoP5S|9=GIAH|5cQjcT(ULndE&4s7**E2 z4Z~F3NoGJPnAQWpKKKm5N;P?OVlwEztrH>~6l_5gp$EzWS0X~qhKJm8M;K~NyYZ;} zFY!91ZyA!@7)YAn(BCI2-XndjQ^zA59qpm4%~47BH2bKW9+NhY{DhL(&}4qr1Z3I7 z)gHD}HL^Es#H?!ebALZAid7PQe)ESP8bgx~3z(7!k1Wo$(;HaMEO9D8N#7uK4|Ko( zIr@J*qs&RvXXJ|K7RPbZppqR>Quf~j&LgHj%&;rND!|DaA-U@F!YQ7dhq>1i-Xj7;}pG3 z#mKdnw_M5autGCXskPPkb+gr)q-1F|e(i1*D%e`J&lSRD+hn@c6|0aI)$~b(c}z&b zan{)QgWosJ3ap%Nji%|;m2awA$ZIIcLm3-(g^0Y#SX`h>C+%pHtF!K;YDR1rQ(UH) zTT9`xOHgoBPSOHiRiCtd*P<5GhhIlJ z4Lq_IyYzY2`25VmZ0e6E=Pb{NB$}H~$YPrlffv_*^IT@Nj*BVtdmL4357Uii{a{P| z2@sr}b$w*Vu-3q0%Xq}VUGT7bq!6S@|YA2cy~qZup;q^-9*`#>`F~# z*~(i8Lfm5ftMWTz+^>99s#KU1l!+tFM5u~|(Oem|J7kmorfni{(uULz%S7NuoB;fc z|Ke}@yF2~LGgz4t<~>`zrsj$=o-a)2o=S^bxnk3z?@@SA2WC`^tWiNkA7slyXXtTr ztI;?IjkrDa(~$JHN~ep#EUP1tOAOk`mDw)MxxZa%?mF5H7&A7x;kaFNu&#wkCofyi z^&mnncDt#;tTy^22i9fHfX^Jnt)g`n(nP<+2wJwI$j*tbYXv7%#8fc0zBl)#B%teV z(k?`_F44i@w#J$y!q@J z-jEK4=Gu}Q_BYMjJp&_+bZ=o>N1xS?ds6*bDLO}#tX*iT;O#Ef#%VkTj5#AZ^zaL1YY+=)vWaghxjM;qhbnD5r3Zx*kk z&%e9(^UVie3yUFg{m#$MV5C@TbY#2Uwi8Rtr}M6))w)nvKhvF;0~Cx^Y>O^*)lLrD z39gqkY-tnQqHV1ytp|3!F7Dpo%Cy5AJZB5{v(Qe9ODhX)MxVk3WNW9RR!Q^v8G4E}9LTE6 zT%_;ta!Dllyj-vBre|jf+FVBu(q-}>e{JQNSZy~?%J@H%E0pv3#iU%$YA@nhDSnG8svYPSzfpbo zvhf0C1hTPe6dt0|DpI(uy=idWpq6 zMo3?OKjw|-WsC&HxJb`RYL|^O2vhPXX8eY7RDcjLE##3D$h-B~UI+(tyJpIUqt<^O zt85;^yril**emk~>FZ?LKRg;>Mi1TUErbVDIN>ID__UeI+&rK;MN8@rVOB>pt9NQ< ziM3gUpY~ZD`DZoF-odQeEfX2$HqOb;?xq;qIVmH~fF$EPCz*RmgmeRyo{>NTEto1F zp{*Q7mY43pxHcIeKq2Pmo5h{-heDzKBzhWO%~L9Lbn~lk5Agqe-hC7L8%XB1=`Hm| z_Zk#xq{fNSlx_td=-?WsC_XUQ${FmI95Y24H@C-!Px*yC*}dn6#UPDvT59fYyJED4 zTEH*K(?a1&4SX%BN#!OV00sO9dA}4pH%({gUvfY(Iv#psG+#9eK&4fg6hvAMZMMiC{xL~2vx`vzCjZjs z1dVfat>avWP}Z>*|3>NUTw6Gzo z>ocYe`!ai75y>m6)pbb&I9ALt?vyYr-lm=ANWw5(N_@D;V>oWuOxYYr*4Y|ivHINo zuvwCI=?AfhWkiv$3gF>{JFK(MCtrUz1kw%v`(X$KwVHn-`A&y0`E+o_?VP9HerC#! z*G)m7qu-OicH42f^`!JIp3cppe&?Y&(EnLsJ{q4<YX!alYE@kVXDt)u}4m)=!L$5e{c^?W_e*V z;?f^?yxcZXu2PA=^M2!@hvfYez3fH-#P)a-rPHxCmxCdt@~ORw-nP$DcY*>E)hB}DeAXo>BFrb* zGiQ%67a;m=2yQYcS%ZZ14oDL|8;}}&)*?6ftV43}S)1(O(~#Tv{G-1%^&=sqzFqo1ZE3Y}X>Ot?1!}%@DJ6!KbcfkXn4nhV#9ni~PA}<|Tbi@z5)k3Mht9)2tm_a}zXZS1q zu_e1{f*M7KA4ZCsca?No&h(ZN+;l!#yPj-oC(Xv^qhe6k*yQdoJmy9UcN8{`jyG>Z z33T9uZ=h^xy_>4(6l%sJ6j4wAI*5MMg7ekCjBWl=);2~XcfrdG%p6jOjpR#b`fREY zAzu!{The{R#n4fE6Wov=Ihy0JKw57|ySwXP$@a8lAvcd(!>EzPoXXmWZ&s`*b-O zlh=zln_7ap;L{EH^b}rwodRzmA6_G*!taaV04ugA3fqn2do8AKQbBu)xzv^kra+0( zMLdn5w~N@?kCVf1htc78@W1be(b2&$A|r(#hS3j))u3ty0F|kNxNszKBEE=VRwv66 z1Y$Ct8`6F2KHaB^OYdQnloKl!i0qjw&YEZDt_!l#;6n-uG|uM+U^Jtl7(h9Ov4>S8 zJ0ln6Z`Ug5ngm|`R#gI9CiLv8gG#|b9UqDiO5&%yrtk%BSyOR9InK^kv)C?JlYKr9 zu%-eNk*8a1D5}M&?QS$*Rsg(GX~0D|?wk`eu+fcc{J1N^QvvEa>@ws-UG!U&IIU~i zR2DU?Sx-+97U&$he>a|7v}eQ=!D7_oi<4dBzN!i*4XvHRoYxo2KWBM7#ay0Nd$n3g zu|Sr@54B=tUXthFagFb*r#B`1)Q6s;<%KILl!&L(8c?StQ=W=|X7Oo6d~J@3vz!2C zu9g@=W16GvashK?oJGok2};^GHozKCzCbFegZLMmXaAH^+u(^ zz2!h;Yww*bp{w^{hsibezKVYGt4wYpbU+MDwpq&!_p6Y5%yHKD9&I|E4@l;V@`|Jd z>k-!;haVpsW|VbIW z403zUb;M0tyI?bzxf|l>wH2$lx>~Wm^^oXWrQAw|XbkWMqwPQc=>TV&ER}H$Pl-o- z^<78IJC@~8g1M`$28d%f46VUpby}3m{=v}K;tdGi}L2miRvl_c>r zy-G$9Lp1IvVHU^upP`zvaW#3@Axvm1$snBTsI4KFlrP+~6#o^!exd>zx>ueU1$a|bjX0$9GsQcTKXU{!eQF=`^t5g#k-Q%Rq>Dg7k zS{)zhK`Sag8O_tc_Zz{Q@ZP+e^ z2*?6VI8Yf`uZb?&!Gw(U9)i)kY0qSj&mM3?iFQt~WLuPRxhjSc9gTifX(#gzMMsZ< z2`Qel7RwGvRpnOSljLtQ7y zpo2v|OZH~@dB1o6>C-1q$Kuq9rYU>hh$zgQ1QLUPtS0ax*^3^Q(n*Jlprwh}w&SXo z!$|aJ9G$R@^`W$^?&0+GqPH)Cte!&fztxzJ&9tHz5R3yBGkU=^&oSaT2*a6%9fx552yMz(>|oy zInXFyFGh-lA_XcH&GO4+IU;KehL9F$*^L)K(fKlU-$U9vrbUm17thhmH2{xmQwuA1 zAEr|~QrNn-9gOUfG#+nGSw-x~2p0y6EC~Q_K#;%weNrnMV3XL|lxC|g8#}HS>tf|? zZwjZ^GA_g?>&USDMq^VAHb+NI9yg1IjgwTZqgA)_|a_jl=u}k0%&uUl2N^#A9 zh1aZf*%q?X0>+|?XC5fam-9zSwyGZ>?I*WV#z}J+8z~eTQaK1xi9Ab6skUHkspV_n zl!iI-cCZa~+l3x=H4J_0;d;;Fbf%FgSlf!}O_t028rWWh`h+g9tAo1Mb3J_y6y2_wPQxe{Z}A>)541eqaps1|{arq1E_Y;iCNA>(X(2n-yv46YwFwVu6MpY3y0_l7b9Et0P+V~2e;@OPs z*x08`@=k*)-yobkbdx+5=?6O-iMH9U(#6{{IZIz#?%|sSi`7|sL6IG*puBO(d72e^ zbSmX~T~02PRb|zI{}%`tsD3-xs=Hr!z!Eps${gd$Qxw~$$@%@)i@xacuH3S_qBZDS z+U6^V1A)~q(Za6s4n|dqCR;PK)&8mSr{K#)NEl_m(XRNoa^D z3@l5-jLy!2)CEYd7P1aVImssqk+Kux2jUys5LoZ;au>W=zSMQpe664jw6oG9*!W$0@| zbFXn7HKIEaE-?L9o`*O#*}q?&Kl)XNJ8bG4TxuMnp}iW)jLNjcoZD`FI-pH&p5x6U zhL;|C)GM|LXX-ZC%kb6-#uXQL5r_M|dN2N3%@@6(oAm}jzIbj8MiUz4OlVsEK{E`0 z(A>8xie;xron_gpwQ=r5MR_$#j(5~v42t(rpT~5X^y_GXeI_g!4EE0AY4R}ZAGO|h zyzyD$)}I}$jry+QQv0um9JsA4DNZ9Y0OH?X+sXBY}M9|@KZ z^8Uav2|bDJqHSL+>(D0S>2wRwEVIySmJSZTI{W4WiyYFOH-VUOg{MVwmu;x^(D=xeV75_C97b6_wyU|{p zoo%kVxH!6rw}h5Fi?m?y*@mKak-s7AW0qVY#RWj%V~`QVIwsFPw1#4-w|2e95zQnF z*EnSHrEibd57<7zxDy>7Q6@25n;#kO2;YtAZ%#XWa6pNNMH#Mm8w>py^`dy*Lzl5= zfhH1Tj11X!Zu>m=YPgZ*BDb+J4~ML66VcE2hs(=R-gAbmxiL zhyrQkC4`Iwao|DQ9#)@rJ>lv)K!XuLJ{vH{^^ROGk!U`oIZiVJ1L0j4p>}N&LIeam z0CeL3N*yq%fVcQ^i^Qe*c$jzn*=T>u$0-0`6Vu+xnAazwg~@kO>oy(mdm^g%e2^3F zoXGa8xB#xUHC6bv#A?%n&SNW=NrbjyuyrwB3#GL$WLQ8)uXbEWve7~I>AJ9bx3%j+ z%EJ61T^Wk&CG1<|N=tj~*NV<{>~`Pi1hZQi>YSF@pILs1N1*#o3=6>jcJB0c)%wlg zXzcgyvfxiiw4*Ph=S(i(7#2v$C#kg~pKe3l&!^i^&-3XvH01eo8|r>O-G+Xci}L9< z^wVu<^yxPA(`{(=XuNoG83_x+#t36EvG4d|m5iLTGR;;=beRI3AmMZY2X~A6r(oLf zB7vEBP5z`*T_ zf#DEV(HB4itJ6jEYh7c&D4?_k0=`XK5PZ()gs%)wD=DA-{SW1$OMgOQLFjX4z58+% z2)O#HrgwWcARDO4H@*^+&$VBT$>#&3$n~yW!&R9RcxQx>&z7#vs*b(eD>V5qZ?Nf2 zng-2EkR=bY2FQ0nr1fWGf~`Lr5byVP#VDV^t48^3g^tSSgS-iAKB2p<^{O&d5&MB0 zcKCeI3$%W`x=*H`pG-fuF)JXp@8D%tbh;`_%N?>~=JLzy(cGYgDV*(*n8(>wl%8if z3^zKR<&&4m6!khqlv9AlWf|>bhC{l-FK)C7(!5-bvso!rtT(Qmlt$1_=uswu>b8V9 z{A38fh9P`M!)7Bl$M!P$Q4B=N93Y|uG{b4hLNW)Pvyf?);!T?<8*L=Z<1kqcTTqn- z53#u=s+2tiY`uftgq%Y6N9|E7Z6U(+cD~q4i{EB(nmwUlddOW5h*`xC_c4BG5RgqTd@6 zB=^mp9y|q;AwS6;Uo?P~oqD}x*zp&VWBIm|){`wrP13``{AxYiPA-sIJ&7M_Sww_ zWSn@rH7#`WSuejjK5NELb#;P`WxM1)=oC_1?{&A#WbxFNPsf(-c(3-7qgu*O6t^=} z#kEPcGRrJ=iU6+nL9U^m?yy38ok-%(JDgfnR9WPUP!*{-NvnyHu?9mgEk%nM=v70b zt~ZJeyd74dWjm^T+vEMxG1tRrca$yuC{z`Pa9ff~H=lQM1=b;uwmy{2LRmK-zg1HQ zwXhQ*G~}Wc)G(HUn0CMuNq6f~OJuyVdWlCzxo7-FEU`wH8_J*K|is{6aXLrzN>S zbL?dcNGFX0FCjed%c$KN}6>XoH&Njz_kY?6#2gu~2kI z6KImH4^-%Tb;Gu)Pw7gj&3OUDD32wKmP&*wH0xm%WY6G$=7!*CmhWyKM+C0gc#J5!FHaLG%&4A`^6ua`qx(EcG*pXV9; zJ&ca9%~ev+kFVj!Ws>oaoA~8owd7ylz^}7qN`HR~e-|-uhxF?Jf2|OjT{llT2We5# zERq%}Q6FC?ujBb*2ErSE#~gGFsfdakQ>J0QxvW=4lZ!mZ1oHGzhhuN@tNjk^E#|(^ zH3j}s&a^GW^4IX|d4 z#FN^8())QsnX#!!)W{V<|A1a6Ou;(A0lPFS58+>WKp8{IoA~W?L<<} zh!y$P<12$FO>f_qfP?^cq|f-ckW2FZF4V&B9TAHe+a{DPAk}-2VC2*#S$A~ed9WbU zxWjdzmG@Ht&3@ymBC!qt8Wi$T%xswcYg zSW~)u6-5spKfnL<@lU@-4}O1q_xZyokDpav_p3^TCC^C_qvcS!7)CSHLb!6{*z)2O zfLfRlOkRH47J~A`_tLEFjh$qO8pgFg>5Y1It4wq_Sv8vOmg?Vl4o7r3)o9(R}= zIS@G1N}y21-ZYt{^LWN{blsEp06V+c^v&}uxw4Fxzw(hfu@!Jkh~9&+xfoh3xhGe@ zQ9eHVw`Y`@?_JCx%}(1Co~D^&evHO`PP4u>0~kZPNM46R`XzbogA~se7cpQ!vP>tT zAb-L-kGYNs+e!}W^Q_atcsk`J3`w+WrDmZ9b`5(E%;+_v7$Z)j0&>QrD@pq|Hdt@o znwA(O0~H}}EV_ z0?0Nr>WHN*CACH)avWnk!|-}n(KIb!vsiS}%}ZcTt2t;G5Qw*)I8;sdV3D5FFNs1| zVc-!N6IcsW<7Sf>cZ@UB)AcM(VCH!Q7=z5qi&(nut{}sdSqnIrQ0_-<<D;T{%5U?Ws-=yd0Q&pf1l-x39 zzOeU8FAxgsG%w4X{4E{8M1fFvw4Z&Q;|(7OZHcylJ%o19Qez%3&(loJ4b-Tj_%wQ= z6&;ZGNVFRrk;CfZbuX+LIcovUsQD8u9d>I;&Zjw{dV#k26lLXf;(WSAkrtOCGvrz) zn@45g3xYGL5k$^w{FFU}mv<3?$H_?x1>I%*wWFQWmU8HB4x|-bBsk3yT?8!|01% z289s`?Q+#HIZy?|Ust8zG%)wEdVQW0VIzsO%0=HA1RcqnByDbwQedYNvcK>g_a++# zThzr2HxwduSCj^~&qCR{+ld&wj7*g z933uRM@2qMr+VJzojYHCe)#RdEsuynJd2~L)nbRv)Z-oa%O|HOUS0=86V`ws#FMt} zgq5;)8XeYUll1qR9XH`@tMmS#!%V)&38%Vj03PNUh#q~(#`L&^3x-9c(r|CE`j| zl4!3YVnsT81&rug!jB36*wbE`l8Fye9D1Od-zwIKLeI& zgpjZw#YQDQz5gV#`R8hZ7s`L4ZP}8%dlaL@Uxv|l2M29Z0T=CaL_L2|k%(@oCdgY_ z%_0mfD8vPEYjOlFqYZJN;aNUA>o(1;_qyDVj&mSpIm@a0VUY_vOEXyEWpZ^7uo%Qe z^4aU}zzaGG2R=_#{ecR1^J&7wlpTUtBA21x@ODaF`_iy? zSBY**9lD0QZx~$f>DMNET6y-S1S~asJvu6T-MRRq7UieoJpTGaIgZ7VupLb+#bgRm zw9@0!wu;j#YM6JL1usQ_5GZ&+cF%|C(pOyMmomNy-oc=2-ZYPZaidQLl*L6pn>I&| zMv}nuBX=Bq8KqfTx9PO*96LjxuZfIV5tZ>NnFtx3fe#NwY%h&LVBS`IYA!yhd}TGf zS(ASY!y|^W_ooJ0{%Hbg79Qw5YJT+e<5&jR(W;LQjhQ)JrL(CGRY`4C&S%lFfskwP zLv{rpT}9%5MvN^=37RkBNm^cw5vv5pI0pR55|7f3o8T~MtmVTf1v^o^D)UK>?wut< z#*pH2N{)AcBB}Y=?d|Ma0gEPYSrD*>92^m7h-)s5NN@%R=DrCu(J)ZlK_y!CeQKYI zM`sBHNwgo&JJpfC)2wJ7`Cbrs3hYE72RmPEg10xI^Hu$%H$S`AbFx|%`I08HC*s{W2Cr5gwQ&are{&QAp?H>N2^GVA<|@Yf5{n($ zOYkg29xWzo38RL7P`l)Cv@O{qLe?%nuTIhZ*XhsfSfkU~ zY8k#njh!p!cqpjOCUX}MEy_K;G>Z2)yRsdL!YVwX50i%y<2!6Aq45o0F%c=*CMxJN zFICGvdeAv8D$V>1mwhFSHhRG`!b{*U?8GHkaBI)3He!9tIhB+1`$A5Q4+t=Ug`MC@ zIKN>D4U}yTNq;2S(G`_prXjEFjS0L>*K4DSN6kNa>P(C zEWzdqU=~;nk%M6w+8<^AAhbWN3M_m^Z5krYGHyEnb(K>XND@eHHSi-m{Ajq&0-rVY z7P3oQ(Ou}{TT_2w^v9*cFlw=)qD?cwxyVpCs}v0&|BDSvh$`Cx-U9dih2;>9g<+`4(!m)Ni0!=cemSt!vHZT zIz5j-1{S-|ybS4P|E_G(jRceyvvi}(zy(ZCSNVdznZWci4M0Bgc6&inKGW6tT@H;2Bg$Jt+4AChfX(nf_sVRh4qW--SZWH>KUwmW>$=)m!@K$)|Jt zP^L*LwxVcl-IfkZs2)AAbR}XNv`w{T__#D?_#oNYuF7m(hP%7kyuI%M7~5(i#H7>7 znF5R8<~`dJdtgo6B&{TuHIZoVw@z9v^LW{#S<^Dda!9{qnoKzt=H*3_MN|mhpbZ1% zy0PI9K0Y*_F5`0;T+)#kUGKyTQU^@yU1U=GJt5nH*n|DWGjg3{4C~I4_*L>Dja6rJ z>jv%2Xu;_B4AHiYY-0|uI{zT1loiffp4Hrb_3(a?XU&nyCBed|@3*8HA~yaxvchL{ z0}chj;7-$gV{&abX?K0X#J?2Zz}zD%$c>z8ENFrU zo|X&{=?At>B=`VVO7xOf=}u%RYeN>mOtdO^6u(aAt2yrlf&d~%ugXNrfr$D3+ydBf zFDpIJpcqCf*S#l|DezGlpZB2`w%CzSIY&y6W7^lJPoF#;W1sAWZS!f7eXGvbXOFSf zGl~8Qtf#1zvq?B2r&CG^%_v%1!dw%NY<=Gp7Y(`gaR^Rn}@R7*19fa=<}uJd2z ze`0o}T7+F4?7z=k*Hd{4WViXQ#8V~(aS@#Y!Tk27v1Ec7L=x~6iVN>(jTzLlmmpg9Rhx0ei!bHG*LZ9J|}kwh_O5-V1gNN@}c7~WYPBj*!f1t?<4cHqfLc$`H_ zbmUV6)k7d__+`$N@+q9nRPf;5&?lx!@+5g0s#>g0i*nfq2)n-A>;cBA_!!D#B`Wx3 z(U3TD`cb@o{5VSlthWQ(ki_ESKR;ol{rLB%zcQoU*dwkz*)nCFrfRdX+tY()aDF^W zvXw4KU@q|{vXb@%Ln>2JKPus6I_J!E&VlK~!iwRXyCpJ>Bw&@3`HRr}3aU6vLpmMH z-@swPB&naaRj27_&6sh-IL=|(wK;w}x#$N=PNA?IupuH%Cht0$IhAR2n7 zJSxoOw&1f+;GThXPEHf6hU`(9ui}^IkAAJ~Q+zp$u9A4ECBn970Lp3gTT6^;5!DG~ zAxj#$0L}nexQJO5^G@E)R~hGtJ?VXYh&Me)-{XJz@2?LWf4jfFFxx3}Okyd!ST>V^ zpuG;+9z7<`tzYU1Kq`lODpP7%KkNglX3aVmSY)r5r4s~zB!=eOEagr~tBGPpw zoRHL1186N+8YnLEW!Y>YxBs-wvYa$I6mgycrva zRd0+ofdP)*4s>whjRYst&HxsgV`Z3kD8Sbn(Zc8$L-0`VcXw}D#2zkoJ37P*5Ya2X zyHPN9D;T4Z4F-PMzln4FC@wGd&SrVO6fXyK(KSkR+H0Usq8|_v?M!(rq4uO4bA#y#{XJMDg^=7pm!VkS>i7;J)CmdSF|*|gL6=*7ib7qPSFm| z=n?yHg!s1048J34k=yOy%t z=R_{mqegAhNlG(^|K5&nevN&!rj-Psi60bYT?WXN``*?F^Z{Ue+ff}vnFX;oa!hbaAR67#$hO=`J_&0( zV~{<9E)Al;Mc>Jzs)CN&LZH7304grhEIf0e`9>rthP7g%Ax@$CnFIqfTfmz4p4XUY zYpP5_Ih>`(bRlBR;E5A)@422J9F}*WRl|DhjUcnd2SPb;K%w;S4j5AsIf=K{YwxPs z^r=ae)NB?DL@u^Wxw}NO2{Cx$J!AOFO#TSkr<|xoDHEB2Y|NPL|+j?y^CIGI7Obmgp*Z# zyo4rb>}&L#e}$;7b%i?_pu?e}>5s*$lM4I`Qa3AYGjh6^1cL;sCEZ?Wc}}T%!Iz=V zl&6*Qz|*S4sj*8Ns_lcO9X|V`qL0A=F^WxBP5Ee>tx@mLSF>L#~VaHNa}BKEl%i;p5TC*8PcG%4!s_!vawp(@WUoQhpB zr&JSLWY;%F*yj^`o8&LL{ zmxX_k3vNOb3M#B)w?Bac2j$|XO3J2v5;`T9hQp;#BJuA}I-xC9aj}TQ{Rhd8&C0;V zd#m2hruK?+A!1p*rJq*kg0=2c+#1i}$?8XmVNs>-)lwYNPdYGS(Y5Y)75 zdbg#Ziujxxn<$9v2%oby^Pr2Pn|ROIU`5pR9()Fhc*k@~QcD~&7RG>uM>pYL`;xN6 zLi}kz#%uD!BcwYwu=yXM@(5pyqnPxiBK5t6D%29CRoOnP&Z?6c>>*Btujr4f-VS0p%5%#$4&mkiUCa$ zWnowj$y&Ve@~C272rqk9E(Cam3!p6oHkLGw%3Fw%+3QZ>DWx*XskUoT`cdENRxQtn z(Tb2fl9RJnd*=f`N7rBG9E4Krwnw3bz>pQHSXaRt*%WqWg~P5Dj#?{Nc_5Ff(6+Q8 zJQ7fd<~fA;W4m3)_ti1Nzift}ALGg7K5K1f*0=O8Y~*j+jxCy{JN1K$-ifn|=NgYM zI=tx6!cd3@k#z2l712>g8v&4~n>gHjXB!VLB54)+WiNnMI#7U)WZ47i&rSSw2)~XT z=La2sP+c5(_?Jhc^sjT;gdP>4Ck0Jh!08Zr4$RZ+BR>X!IIYU$(?M|U2LW-4`WaA^ z)+0dJslZpdwo#T(q5&l1F1iY3or)!kZ{LxP9*8y3G9~gG%N7-@(I|RoD@r~B&^vWX z5G|!ZT@<>D8@+K%_udjMj*ZMSPVJ*}a;yTTXy7Y2WRynCzG#Bi*O!fXeN&7C*cOsZ z(~dL&L@PL1ut_EJ&_92R`trS_=IyAeB8o=4mQ#iZ3Udq^LyYD24dA4vz+lI4e)dH_ zuxH|E>>c#f;i!H7aQF~i&ueEuL=ZSDsz9qWQIaQsnFz--#d6#}X7+%7j4PI7pu?>= z3uaS99mIc}Vc;yhcb5WT;U5h@>94oq7Dm~CULKK~FS@;!Ik`ru3Dup@FOMi52o|No zmHq-)ZwFETum4&l%PS1Vvj5eVp3lC5Hh)b|m+|sy9Q{2xjgIyXqTe49&&TaW{Z)a{ zv1P|oI9Y&Ix=5C3i4HFVf-#C7QYMMaTXJw53_eK0F5#Ojo1>gxR4Vlj0W}Wz*=(fQ1JD8q3 zXP%SaqUYmK3P4UnY}$ITiD-^%}*SXf(PLLB|#|G|dr{aJctP#J0?`MGRMiReEv> zs7ct*=yM58T<+&lygXlFSTVeW!3YIn8x%Z4wbpq6dG^~fKgWm}wb#5M@JQaHyMU;V zSci5*mlrwwr0;`Tl_bN%e_wcw4=L-%{zW=XEP$nJ`A{~L!pBS$q&N$_&C((|1)l3A z<#7{psCFQdLkWAr(ZJqB!sHeR#INamk;aq*4o84QdB|)S|0&YTH*}nDfk^26jZf1V z2EPZ~IZv`=i5D}=c$(%EL5s1BnNgrvvIH(_U(Sv+U*{>q@d-4g*Ae>P!+*q-YM8M% zhee6cZPQ2VGRbGFd1ihkf4YMDG~_JwFzi)P=5tZ@qAVA`kYT=D^LbKU{4hBsfDccr*tA5wGj8R?C?N%V=tugPefg5A^xX+gsHj ziT2Q~=oJhIhD&7|L&f>}{pqh2_^Lj(d`b7MgrFtwa~yyPf`(;+P&K0doMHfPis$ z%-g>ZpLNSt*(}mt*Jl-K*Zz5t7o}J{+h=Vz**^2ul5N+}jCMZem}QEGX9u}QM>*c0 zU#7rDA=N|gFN!x~yNQdR!)C9yC=SzVwACcTS%Iwxq9Bh5z-oB_*8#!lTrF`RAgm*f znHn%YMIX&Ujhw;|>{-BXRUBmsM8_a%h{OrUiZ-EpMT=l1_sG~(A1Cvq-I8hi-D1Z0CK%rTKtDNjiyz#S_yRTkD4(BUSBNP zOJ8f?um2%74?fBHsR-0ttuKn9M%5MvfEY@y8D>POzyv=nt3nIMYdo^`JSyqu^RhHja zG%J!2sgmN2rKlt|RZ)0j44Bc&F-a?*nU*V8ib&7{6h#<$G^tb;hycWeDp>%Y=MqXWj-scXQSIxL77gJ zZlA$Q0fseBN{i2y9E1!#gQZ)Xz~rNig8xH)UjBsJXn#0g(24#k@p>w|T9Q@Cv&`qw z=Ljv+=FVsz6??Bilfy~`!MSop(`+-eb#}vtDTH1=rHG?4W3xhQt2=s8q~{qTOp215 z85x5&%Ai8W1*78v(2>6>gj~1{Dx_@J=xDVQPZlmEDT_ujgo^b3>vs@rBASVJbSRACeD*gBEcl3SD0H74@ zZhw%#neLwM*WIt*{gk_=S=FqO>X=97_Kcl49m`4R8h*zNdNnCoy^#&l5|~ZGS<>th zsV_m6IJB2J5BFN54gjFB=WW3-I$M6Nz(x zB>oW?+dI|G5NJSW^TM$}Z#=&PM^C{WVq2i`nAwmbvjpfA<9mj@b{H9RU7KW1=j?ja z)sRzRqmQIHDl!Y^!))kP)fLEX2ng|;OC20o7qfNayC&}J?ZPd-lKN=PLFRfdr3NTy zmF;7nkuh{F$O)|ah}7EhHA17;ON_!_XS=OCQR#Py(W`8^UghHkzjuCOi~3Sn8xIii zCrFwGun=g4N8B-77Z>U~sk0~;a8pjRUCnc+@+=BPsQdV=DiBe=Z z9!)3Li`Oh$g5pL(tQS9kG>MD<_V1*KD#LHzY%g!BO{}d{@I=~z*em*7Qf|fGz9wW8 zeM=gS*jp7mGVu0W0jO1dA9?#V(F&2bZw&#c|Mq;fs#nf@)$5z})nDrkNapBu=QVCX z@|qvRT?bgVT99H8MR=@(9~e13L@!_e$JhTT=}V#)_^ZG_2gRGRlu^czY`uZ)H7z$| zkOJAA7y!hf;9mko4+c&}v4)r=nJUs>R7usspo(U&Q^9NW`FwH8Hs7Ge!yRWaS_VxJ z=ktJtA{0fQ&z=%fDE|M-l(wXJ#-^)Ema>M`&y;q!-~*e3^4Zn9+aqmv$3S7|=@D4w)uF>);;5E6kpq z1}eie2GY0_x;4j!z|_2()wrNBMhq=U!ITW@Ccz#~v7DzM9O1nM((C#z=cO~A&18%K z#~wOlJ?x(Or2oqdpLJsg?d|K`X881DvhP0jRUBj26q*2J*O)YJ>!UPDxNPg1|%_yJ8Wk=>J;rvqP z#x$?-j*V#q;|G2tlTL`2?B-&O>@5q+0#joJRa{@0WBZMw#eh;7VD%r#u1y;h{pS5| ze`9{wRjgYt#R|+zve>v8FGN3RrNwdlh~=uco3*%#<-@C0HT6FrgVU3#i;7M!z%bU?IQO6CVY<6 z+NEp{9q=0l_^DvJhFN|P=VOB40vL2x1BsGYz{(O$C~iJ{mTY(KarVf0X+SGK>mNJh9wP>2^iwEC^~m@%IP zs50{c1bhT+9163@ck+Y`I04 zpf!krkv34c(9nuj2s+KK@2bt@S{g|J%8Z?=iNMxHr&??}!=xxH_I;qj?Rc?~no)(l zEVLBTZUhEq)mAd1O}!}Z4+5PZp{I`v9f)@hUuAW{!$FCXhlim=HLlBF4+6!%l)rxL9VUj( zDP6y}@E-W$tb+X9`AJGt0wI3YQ z!|q@NIynTR7jO_N$?csp5NO<0kRRU3e*F*%WSV4_MY{D+~f~H9r%2Auk-+~RN zs3HD$G|;l1X|pM7KSI8GC;6@p&u%@GJl25{ zK?2)IT_(89WELmNQd&SD3i>`Tk!0!0v}Xn-)-zj(dw$*zt6Fl5%B%>%59dwuOuuU((M*B}oIwBGf7|NsWBcoHyPA*fl zPpeyugk>fh^``Ff@jSiY?!mBiSB&Xr@Y_QW4uutgG{&%?O~{$C^7$I@$LNL;cL2=3Rk$(Sz1>G>2*h#COLU5c&$40Ulg*Y@7M>!T0-|!$OYH2#7KCLu9n2&EL1ugujQ-?3?X6q^|Es z`}^@_y@CwPa8rIMH?@BmPs`a@-~f&-eiQ1H8f zsF_`hZ=;wUQg|pC%=3CrzclpPmDU& z!*B87MKmW5h$|B#21lCaqkgX`NGpiWy*GhgENOK_eo7;Heg5X{cYpighXx~J$FI(x z&)-IJ34Lm)=O)<@^gB$H%UO1j^8Ep76ZFkn9t5}}RulBld-BheD**t>a5n7wG>v^LZOzTD2wVnO@OtYaWWx}I)uw5W66i*-* zrGaHJX<_KGf!rCYSz>1}m+r9Hr!@8qeJgMC8EU0g<-rKvghavT?qmszzkpH$x=GTp zf*M_lWsk!?9)W@4RZ;V|klU6PND+?ayEwcF7HE#n(#8as(sel_@BH?rNu3Rc#1VW4k0vBB{DOi*-UCC3m=g@&}GSrUX8gwB4QhT zX9barF+K2)X+`SByL$Cb86pe3BmO1&saTdP@MR#sTO8zLj(#bQ{_1`M&aogIsusnx z-a>Zb|Jv5EY=ic&nvDGIt8ZToi}7tO2HIWLfRQV45bq1fnm&Q-&On4uOYwaGHDEXu zK~utVko^$xP37%)-tF*02)f){3zvUNXtJTC-(vPsON7*Tf8;Ad;X_G9^FIWEB5tC=PSf^rW{h4haAtlx8MqnOw2p%0|+!~-MSzx+Y+ZF~;pA4tYMJVio zHOQt16}dev;m}&e4}FSm!?OFlHVAq$WFb5D6o>BaVcZy=zZnR`C$yst$`0#j<=VmJ~Nj2cC;lcmYbh{AZ|$5!;{f*%NZs zlUG->&R?H@bN>C`X%2j9CGlc-pyDHcm$%aG?}fykHN|4?S$G&D04rhy+NEZ7uhyr{ zxFHZGJJcldHBr0Ww^9mPii&0WV-3*>tJ51$stoqvkqES2&|^>e^7&|awS$q%p;}Hw zuob02TsM$ej!m<)@tF)u7_)t&h*Fqh#qFoP(ix>Uw$52Ta#ha!9Jg6edlN#9UCGS{FAc%LUI~;(c*;1Yy(s1q<<Iu|&;SNMPQya2I%62G8g**3@Qo2J4DBfW) z0`f^;`_3FkC}Oj(fR|tmk20f!hvKQ$p~r}NJy<_c7cdMs>=)Z5Wd`tSs8gYW`}-uM zEGp>~TD_l-C)dV`!_}PQ9#M5E>S>EiGahnjX<_Z3d0A#aG$uQ%wtpwu79;EKOxLE?k!73c{5XoUK(H zW?+QX#EK_+;>GWar>AcE)lh-tiWa$EXnI|+m$1LP5V}NuTfF~W&Y6Mdm=hzIB`#9< zqK#lwmISpUtxR$XVK^C$ExY`RJDD;;$(l1k`Gm$bP_kZ}8QZ4F6b@kD)M#`WJ0~dW z_O_$5nU7@b-GH&sPk)OU%S2}WFSU3LETldO=(X7P&26j>H~s(z zp1G9++jaLvXrzZ#T=N*@8QqzI0W#|vX>_!P+Dh*=^`K>JRErwlRX`EVrUMe&beA~G23KqNOUllr*-&LCM@P_}MhDwUtm<~M z5lj$}H7FiwDzuQraCJitsuCO&bS`@S`2Qs=5BP!&;YTyb znMJKLddG3^V!2NQT0N;w(s(pf(zXy*&87wkW(l8%=0j{YynL;E4Y3I*FV(7El3CJa z!;LA|84fL;;SOlFupdDHK6=0WFcSZWKR;YFIMQa1{?37JT5TS^VX&#OXK=_3mSP{1 zC3J&k6b$Sji2s4HO>uxwu#H4B9m=&Tn57&R&-5xcBhV*a+qO5#EUjvGTiN$tA_{b< zF-xsjag%A>I#^0L0>;kP#!yW0TtkCnC_5V+%WtL)kD)X%K8A7#17wVyQ?n>Qu${M! zvu)e9ZQHhO+qP}nw#~C`+sL^$sY-4tspJpL({%MW-D}oTE;o0I>^1sKFy41dIm3Lk zw!Nn!%idW8oZ2-7>vQC-iT3qCgoX7xCWABongVtw2iE5}07eHCvao@GDOpMRp|kuW zEo8+R6la7EbG$b?ApgMP%77TUI4O@l$vZkI4`)jcV}e&?K#nnFP`&65>B@>9>WEHc zh;5%Gl*+QhJMxg4R4MEW3mRDw|;itVnI&87d=vcMPbA; zeJH5~CzMd8XbTU6I0u%3zGuTXWf-g-T={onFr6%|jWP70fjVZTjW@NWLAxi1CKe+) zhDDuVsKsTlw9VbC=FB>FL2PJZFUa45&fJT$P8V+-zb#IYTcD5-Cx|@h85O@Y-Ih$e zS@o`_S2KfDdbu_{zhQ+YdA@f%TAvS@cy+I%B7%9wIH1qDvHXeQXD`d7MLl^(K~!P( z0_qAP^VrBmC4c^(Q0(&3vUNeU#Wly=6yBvLT*1jweR2G?L2vd6A&bgkNmcmQE07~^ln6lWS);6zv+E_{K05M z21&o|DvRJoi#!cN^gVzIGah;+F~O|gFhQdyGrGDVa1ET5uMvn^Ax`~o6ETX6K3rf1 zwjG-b;D#y#BbKp_3lP#}@V2oQtXAh%fN5L-XQ@{z`9ifAhD7JhiDe@D_;g+lGza+7 z;WQ)!_4+-zF5Ot*)V)5aOs(oOYw=C|-*-8(XK$~jyF4LAi_i01NUZ#!aKcg>TJT5~ zahypX177IN`-@OfQ>><%gbC95?Tg)(Fk}!|R#fwP;4)ik-){l^kQq~vxb-3sUE3Q1 z6znhUgJ-ABQSF!p@3%D~uo z)sf~Z)}v|YZ^&F%v$DA_`E_Ik)0Z|)KGY-ji4KhggE@QOiXYE++t6MQBHk~B*9~8@ z4G2=(0D=opP>tAx(my*?RDBNqD>mG8k!M@_iV<=8q&+AQ<3C#3L(xu&)qtI45xP0@ zcTaC7cjxoL5WJJ>AZP1`>RoF}da%bWtXYOVp`$J9p`Sfor7&NxlyLlyMjh)9^sYx$ z$0S%aNQ(+tNEXEA-|*wy%%`M{N%aN*bXT9&cDew@E(xw3)2 z2QWS1+0d5lTN(7dzrin0x@zz+meiCioI;I`Sz0YKG#>|5WDiK6GL^lZnPrO8%q57K z2_QSw&|eZo?6Hb4@l1)o9)-Vu&35Q3oiS6cJ(b2D^2eQJreqkY)PXgF!7qB?x9I=a zOexT?YJ_WyQNkrYjFq0RJ7PhqzPy!RT;x8il#(UL3-Z zh#H)ACpMd1n!aaDZnXQ2z|j8S%IJ& zsfReY9T@^)A!{&5Dgc}a6Y^+Y`zt^V^y?8_8xO&kAJMsMs;Q}IX=h5-Y~8?Lw(8lD zw-*mJ%2UDDL%IS>@6F?%<&Hbdpsj=BREp-G6<=05zi-Bv22$p}i0oZ+kAhNmNLdCR zncnkZKl(XH8}Rh9D<9^8YCfDy72}w}D62nykgXK;y@jxLFkO^gCYx?u6`DlY)|4w} z5_!p60y>QO_V8hczi^0xcXL!Q_QaHLYuXT0zyt8(4Z7aIGG3y2hz524;ftsGyx(88o%3gJe)_&2q>f4RX}PCkK2%R2Y(@MIVqqSO9eRGlWuVkC%Q&6o4e)BNim;0(4a+hXw*#IqGj3kix!Eo zVwa_N*A`TS`Ne>7!s)i?Y*vK!_qbP(lR9|rhRZqR@Ddk?YG0@vPAFx_;)5LyIzBMPGIijl${dKxcBt>BJc$cS?yD(;j4kHF&UKzA3OzLD*w zM5=W(T=w{g?u+7jzxlw;?e>0WXlgwY793!spN>uKjv_jcmgzsC!GxJ-7h6QCxjwsf*H14DPn#>_l^i2m_lI zhWmt!$)HOSeN0y$d;!upX?szku3|mbc9!6rWD{yBhzH=0SNaOW=255sUaJP@gq{L3 zywCF&)(%8y7c%+Y4;qndY)SsqMLZU*l<%h{?qm1?#7{7bvazJTnkERl}T{R{e z7bIctsyO`%v8Xu!Lh%pcA#4M}@|X!n9hzDIc`fcl3@1q=6KIQfpMFx-PQEi?S)j3! z{i1@NXgUJrs=*xZh2WsQe7+k6l+E1yG^j3IP1s5^l9Ldt5)*tF2@LOh1^3G6 z0dZ%5ZfZ_dkmnNk(ZLWm_~EA@jBmmY4~+0E5Hy7|{sp`sl5Uj-ise@^sA+xPluP1$ zyL9E#qtZWH6a(}vUA40UW#OCXF^B z3=t5ha%&05KMfpKj-Y%4v3GUy;|*gibbmYaWJ zIfWJqD?)+zI4<~PAnUNnQ8afs$@=_S@aZ>#=Is8j;4d_4k>gK@5XaK!XCpP7#lDB^ z1Cfy-Tj+0#!L_{?sBIh05S|H3rswj4%=ZVYtXhiS2?>h$paTc0-Gp_k5qn9J5AoI# zo0(25&uuhj*x0!ot-%ZmO#=&p(wT5m?Cf3Dqr~l)jZ0^V(zuQit{GnT(Oz{@Zxs6= ztd_S}w)T+G@x0HTelEuoVf{fm7npFp{a&+F)*&C%0MzRtSLTWI z?VP9wy4%&GA?GLL0%xMt|aL96W;{>CPor27zZwG zXO)>Xr7%nky#)DIw=LG0y(b!-S;2r2P^-%;)nHu9e4n%VVK)IA9AXCE21kbcSG9XM z!et3FUd!V4`U|pdlAC|=KX8z6Va1VaCKES8iicslHfo?l7(7kUyw{&4WQVJtK8ed# zjAxM~E?QDiGWAeqJn|jy4kQd;&67S(8ehU&dLINgac+^#K7w2F zNgW8Rx>hglxP`KY!0!GH)5#D`RqcsP zY~$lH(~|mN?qe!dgiS@a#8e}o+n-p8GQ8H_2xh)L=Ei3Qw@8TwC4EMAvnSv|LAO^h z6*r3U395>lW%g0et(Ma<6z%_Tmg!F44w2%<^ru|6<5-{vue-41r2g1gv5y;s{C$y8 z#_jkdjfl{=p73ZKzc&R|@K~QlA;fI(-Kac+-Y8ei%N`Ux8<&iu)x5Y7&k(2g_K;^L)vu(G=W*>ycjRXgJda!=r#=k?8jrc)H2v#Aj4IBmO+$&| z5jx@-9K)AzrViQdzjDZ~=+0q&pGR>Kxx()@UO#;+4vl^|aWbwq#g40aUfxq0V8MLJ z-kxEKmE z18$6R|9A967R+E~42;~!AnEMeo%W1 zQ%qhsl(Xm2e@RwoqZ?+neU6m-^2GE#s*@a;8+-hpwP8LV)&2kfxMV zF@TYH<8^ZiC#-amfcWV~GxRTj9Rg*P?Rmsov+7UHoq$n!!u2R(RUjXm=xVAiSm3Vl z*b+qNv{OIBLCp~%&izj#-{p|V+oTc4lv38%8V8cl=oHGo75)fB z82AWaC7u*&@=}Kbo zi0qcG2piQsW)aX}GujH$JzU4TWAK>h*aBL?#&7rqGSZ8BU*S$IF$6lExE9vEET+W` z!IcDTcnc6Kz2sJWpOwOL3cl3&fIMN!fxk&To zNpBTy)#OVM;7hDog_qV3IK9X%ws(g$b=|~?-Z$?ABXg7Uf$Vi0vE|VfD;>TpUv8;6 zxB;!{g2M3FW*;}EI$lM10Qf==aj&EmM4}DCw-)E6`t^nc;qQq?;vq%TlLTx$BfB0e z?SES4X`4DB9>-15Do|24nDND%@q#44BsgS8?^*vQG-EjyvsH=`XKu)fB_}&D742U3 z^R0Wa@Nmg;gkXQ}s@Vw=G{%y2?mS)ssRZ-J#JPRfYPz36t7j~u2i;do4;!w zfE;4+9myjW6!J&@LEFeoUv}r^?8yhP-(!o}n2<#$u}9~71t{!P{t&8}q#C%GnR?C? zB8fcIpyA)^2&@)p@EUPY&@9if2Jw!}7Gj!&pO^n?lM%^a+1~-vhY!+Hr55vP^S?yd z35}q#YUa9F$TH<79j*i?OQ7@B=l5fQSu`2+SqmN`AF(y0?(nk|lvCO@er$*Hi1=Qy z^l~V&-GPN0-o`LYl7(T%zH!kW4{Kb}5cpN}kRQe?Pzqc?859E%r@!=mf_?zO`RFZ( zNCIwgWoJF}RkE`jRM`tR!UNjw`LrW`7>Kzg1iqRn$lc`=-}%T23te{^uo@i{p#Nau zg&hN4mA#AXNXFMl-HQZZu~liV`C|CdnGfKLZ%5GUfis5Sbn4^WUinE0^ zt))|q#;@JhD9Wz}*cYF61OcH@Rw9!=pL%N0iU3&A$i>Q*Z$pefsvIIxq;jknSFQm* zf7ZxMBD;DTh)f#+#ylZ;Ifk`hXwg*Z1zp7{H@7S7jYeHiNe7d2(#YBryK zzE1fZs>PJ=nF4Kki9(*{&Dc+D@u|s{ExI!uk~fYxg-IU+PYWLp3y(^bh+8h4oNVY^ zE_`k;ug^=~Qr|a`r~U8KOHUgaodG%yu9#mhSNl7#H#{!x44mJG)YI{9SX*Dj4R;(| zEO=aOVVWFRTwheL`_1~2lFd0rl?;PMi@5*4BzOX6UlD9C)Q!CY4M4~wlr894;8uRwkp4hKe+f++6E{SrAqhDmyt;B~lz=k2mzuMbDBPELy{e7h9 z!~{d`P~}%@5$9(L&Si#LU(A4H7g)seG?a}<6a*D8sHtIwI0yLB^U-Q0U&8GC6?+B$?ytoj#P@gs%=n4a}T%cFyU%|I-vH$ZA z1V-E?4cQ{fh=Zo&l?Y`VxcM*-{FU!wdXG2yoc}6A^7*0Ib|kiG{yVkRr?6?=1>gP> z_N^%#dAHTu5WF<12ksik!IV@2d0N0@)P*cfZJmS8fU!%pe%nPmOMkAsxQy2T#eC|i zNv=cLBRI(m6jn}G7+mbY(Bt!9BukBLA5Nir-c_7mjR2=i7u^kS+?zLVV#_eiy*YU! zhPO+tev8a~>wYMI*m*S098Zu~b$%yjlD;XZ!$qIN*fPH;u(4t;9MEN*^ceslnH`#V}iolH+ig`^)g?w6(> z?q(;{^71-Ku8Djj=Ut00$VoLd(<*tIusen))0P!4bM5Y?(atDU{L~NEj$n$XQBZ*wC@`)heD9pFX$XRpHCrspGdvPy~%EwWC_)tS&v2yYchvI?WvguaK0>}#(Gtf z#DVs?i+}$8JqoGaOHXVndGw?9G)Wi+F2)`MJQgZSLpK6b{akPP&(%rH zWa76erEkW* z@n0x^1ua!m-Zm$qY04Bgmj&0>v`TGvFcqrs7zOBP5SbVcyV@BdAGk=Ms|_bwTP0W% z5bS!icvYpKI878ruTi7^B7ujTtvrpJ4>}18!B^zGC8!5^e1d6&wRUii#14?*>SH+-YYI& zv>W)l8_MN&flxL#le>~RLQ>-gm2s}3f_C5nk+sc1+lyG#TmJ<`8Cs2Orahf%ht+#8 z(xS(|FH}lB-AdBsWXFH4jc+~!W1@kv>#nV&AkG(R{1}WRQUOQzo4T6J@2&~7Q5TuG zwICA}?BU33O=fktrKzYovu6rAtaJKr@syz{wqMr%L4T1`4+K|h?$qkUeXejx^R z=Lowpz%atElqMqSkzTs>c~G~ki>C|E%)XtgBVSRgmz1#1idg^*T9RT--nt9SP3BNF zAi7W5uk_beLl}D@nr2LwWw}OyX=2_OGeWoLtzBs9f`Sm1E+Y(A$s@$zTRcwpGLeXv zMimJ^GtNP(YLDIV?rC%RX3)!Barx;Pththp_a0g3Np~Pr$(!qNHwVWPd-ygBxi)^% zCO&*xFTHst=bg!KKt+FQ6yDZtP}u=Fnw=O!omtJU3f_S9iI5JuCiI8Ke1ER_4**xd z+!T;{rBml6(LMIZG7McVj08_$3F>kmnQG2ghd&eKMFFB{Ql?B0kY8l$Pe45ON7xO~ z8Z263-tHgIR@H*c{J6gl1f1`5oZn{dl9JpX_7$Gy#w;%@O3i)ShE+H|ku%U>D2dn= zeAX-tdRiSSRx?1UC9Cjv40pRbmCm%#W$!(Sc2Ew~xWsRl$+}F()=p1UL<+my-I?28 z9q+z__S^K}c{@mU0- zJ+XUul|Mzsx>`qRF?f1K>cG+hQor;3Tg^w<7DR6-H}zj#qVzdXTKO#g07lVH`8R3I8FQF)nyj;<3$y>sdIp1_9jXr^c7qfF{Tz0h9l_-KdU9l2uK9+=03vBxN3)&& z_h3OR$4d^b*Khm%`Nnzx$1UV&KO!sC1&IZ{B84UAagehl5IR)rz0}~?Kv>AIN@Ow< zzfEL-!)%z^O4~Xkm<<8}GgFE_1qpVMxClc!q{c>eYt;S%RQ~mkLkD$5!guoy zzSO*a4`*k=>G-_neP63yzI$$>lQTS0P;^A_8v`&ZGgzPav}y|(8?Bdux$`EMt)3cW~cfwx^?qP8feW=A*)H1hswy3`ZPA%Cnd{oE@TP+w9iPR zpR^h0DNxmz1BkGZt4sDZr~{@50%`;+CzzvSH%8xqh4lAoW{UCevaMj=LKdT-FF;VnONtS*d^H_>ukg&_FNhKUs z82&JJ52it&EiYgw^dko4VnVl|-hB8j2ZRpH|vkWjMW&2*pv@}Vg z0aOKNE5B3#(O_`T|74?tt`jjI{TC7`eE~DsiP8mp1;$X)8d`l%#gZw9f7E%~krTA! zFXi=zK2KR8h4&AWci$d+)JXm(jENuVvCyW`s%{N6U3fw}@JrLumL zA+tpC$B>;a?-%F9Ad0uv#UpBKAgk+*>&`c~gx5wTy|u%Yv*s^1+>77t&)xmN&Jr zS^Flu1TCiP-JZ3bPnmM>HnJgvhw0q5N`BCJ7oqikHUOY#wLZ(I`to3eF)xY;LF@(6 zg#oUYB%7sjq&p5e`)KN**QF%qB=TyN?%YPxhDY0B$t>-T?Vrs)zqdV~pyhyYmsU}j zvx<@1$F6^eKV;O^1bkclPx~{Q@Qa-9uPi(SGx{${9W3Ncj_~8vF&_|IZg^jzj;hAw zUxW@W7S#XffUqOS(PkbL=5w@gVKR%EU^_cJRqQ#1Kk47r!sHx2-`xn=XIng6)$8LR z9rjD_S>V}tolfCe;Q9Un%=f-c*@;Pd%K2b-UwF=6N|ThzVsG8v4nZ*>w(mq121X6C z0%-1PB7y#iRMc<_ooOmegrRSD6$yf+F@S`)a?Feq<|mUX{3S}S?RDp9TyG^#E24OTwQz?CSo`s zG28?OzYZ=ZV_*RtcyI%uF)==oLB5++Q54ZW8u;j$z~B4+NdH$CF&6q6Azu4dVdt`J zDFT`8jDRTPk`(UN2l{d)>J-oIr}Ob)!Ak)K-JSVdQ6C1lP#j7+*t2<=-LN-l0f)+r zbcoBARz}o)8aoT-ZoQoiW7EOWk+IY0ROHhp<2R8X`%rOXpo;*l>pi4bP!?| z2Qn}q8H=Ajg=;Nt-<%)Dcm*FKf)HqQVGmtAmjD|so}5?=*aTk#jC0uHjh$L30Fur9 zYbq45PR+=6A^R0aL8rDCk|U8eZY0%Su5=HArSos4p@8VslRJ8lT`w~FfT^28w7SYY zIoPgI1-T&X?LIprevn-W5dWaeZiu6?I@m4?Xu}hWFZ@ix0ETy2F{`ZdpFHzNHk{H`+&sR??&WH?euKfWRUrtw3?daq3E;8Kcow>h)IkdZ05)gr75 zDBF|9L?Bf?bK8%wL)DCiA`lL4ud-*7F^tIxX|Q(;EWmtCI{qF>>MjfZ4G|I6<}PQS zx}njSli7C&r(JmWexj|6xdSYRo;$yU@D7tNSrPTw5Aj|Ccdi7_pn@nfbaejC(OUwZ zmD}y82T$M@b-ga{VdX6Jx2Cwu$6Ln`79Qk~mK%ise8B!1MnvAbx-sQ?KB?t1#pE*K z2dz_%cX0t0QL$~Z`=7#&=%btAiNr9;t#8)l%>!5mmSbN6IbK~Co%WBz(u=+XJ;TMq z^KAI}P5L7xPm$4*_SnaawS7bFfZ;}bmtxL?%ngwxN1h+bka3WDfr|#|3dSrO>hqNw}|B*UClg9<*)XI1-j@3vBTB>1sXR=i7R z#+ppvMq_sn(CD~u-95$gPkx)mLSJ<$^|9>S-sAdW0H}CNUPDrnaLtWwyF7D|T}`&- z&CXCwiPoR>Dp#b7s_4i{J-S+tq*bQ#+Bj$YHUs9Z4UAV~t;mpno*122e$cbLZ@)nQ z6;VIm>fk}Z0RRj!003zIzYtZ}#M$8A-0nX>^;$#A4oM8z7f$haXqlf4fjCs2(s-;w z5)mR?%Sfprvf6wdXAAlghlxv8kf=C0)rt-{^FQL>WafEo^!=_i@H;Hk;#-1g3}2)4llaJ%K@aU&ZRJ2Uh5Stg&8awj?|!kqHFr z4rzc%nFO1tCon_Eoz|tfzRXXJ@@-VcHevs}s7p-MPS3*oSO0jWXEZHJ$ii-i2{F#j zHsb|S*`PRC>O3}ySCqnyeW)d%(gMC;8tPt&B~i+Oacb!y2Z1Wh@r@#i9YxR`3f3*) z;uJ-ZAtfJz!L-hQwVoN7vEfg!YR*{2AUAr5V1YbdJ{AbIsqh_U3 z45~`Kn@4lrM>jsJ+In?0GVQp#*V^CouXoyb9pj2uZn6X$9|90+Oj=h=o6gXWvsIb4 zj-x~*xr2`4GmazqQn@&j!G_T_|FNK^M@>&>VWQ()pS#bqDR;~KZ;fNoyl`RjcGH|{ zYF%3b1nlnU$D}y*9`I~BSRKH0>&ymJLNBf{N>ii7Yt(b=Cp2RaJx2hTRnvGuszrWG z(Zl_kqP>*7-FNIe=!7-+Wwk8LB%}GE*uw;&1N~=%N$NGsV#Ew2aCl0IR1Dvbw%2>3 z!|&Q?3t7x9ZWo=e`F6+K$BD`7m5`eO1%v=($(FakSv%Ms8*LJd;uh*=m4P}YXqde}F@qsw zpBOECs#%h1vF+cg4qc7op8s>eW z2AjveeQzQI1*t&)5^lQ@!wg%asr8L!0m(?98OUT6phR9+B;7Ezp2>ITzA*CcY9aPj z!5pZ+nZoRIma4&`!mQBRL>W(0^2cejn{#tHG#oS6%-2rrm~FW)YdC%CyNxpzm=VOx z&uFzZdOnEMdwTDg7tCGWBR}6#yX?Va^DK)Kp~fC=f}c_@T4^5^roxqqT7{tOVLk=B zI_%fG4fbcZnehQ%PT}z)Jx#T)dd3HY*x}wh(zjKWuZJ1nYbn_G1f5tPG@Rgu>EylNd@1>4Xcu;DZV40QU*$wyp_>p`$I&TZyg+>=C>wuvn1 z&T@69?7G{xk3xYQi)MPuQ4;i;A`ibB-7=!IbSkz>Ki8=jF0-TgANO(q3ny%VkjckX z(>KDKyiVIv(01F?ZfdA`G`(CfOl`oYfRm=h5=bK$CLHho7y@^GH#>Ys zgj1|`DF=$mMniHUHHNodsrI*wn7HkGBJSZ1K1+tL&YmsE#go&~izkaOw2;-!(KQpv zEZbmh8WYavc)Xs1-#nVxV;g2f%WKs!ffXL$=f(ibyKeT_&+2?oHt3J;Xa4EpwdKrC z6*z!yZ(SDera|)XzY2Vfu#x^#W_z!`hpNr8E76`%?FOV&MUY@DCDXh|aZL>K&0$@T zC*MIwbBw@$JKE0#uV&yR)mb3W;}Vc))~FQ~$();hGYnD3#MOF8rdtDu6fSaApfvQ7 zts@0X=oC{ zkLs1kE*}qR$t)W|S~>qrKJO;(^Mf|ognpuvkd-4Wk57msB3mUuMzSeUAx^NV#x<$Q40=sq3RoGEbs ztYTH9OE2O2thP*j99zbvy06IZ7@n>(z)c&?Q{57+Hh6NK&^sH8s~O!}7nv(KOJ$;_ z_3zL;JjOOk`ZroClbB$LGG<9o#hc7iXv{JU2wgMHscg{iVZnAKN#=m@(NP4lAqS%X zQ7FMMx*Og?*D+KVK>NaHE9B>C3KdWQotd6Ys~P+Bs7yM(`=?PkhWJYpnWC^tZ6Z9Z z7Rd533d`o3br8-{1Pdn5=XE*hh3*P$IJo~ZlVDGYk5x-L)|_0eWxI@>m|6^kCkYI! zh;j7Zs;4-r3=j|LYgtxza0zION)Mof7RW{@4;Lg|HXK2arCx2X)R9zK?VZz@RwN5eM zkM{C9s1yfM?os!6nY#kOp9j?En&+$iM*8|x1`z02;TOYyVxn^t>c9^Q5H1-VBkbPu9mpEY`-BbYbZswcV|Gt0v0V#vr7qZhKvd7)_vqmj%QMTS`F@{Nig zu_TWZAS2L`obd?Dkw1W@J;@+gLl4UkOe2{d1x;Zjl5q<2?a2Ra3*X0adFU~YW_|36 zeiYY_yoeHy_aNR_N#&9wPhwY+q&FWfYH?59Vvh)afn7orcM4zzEdGXnO7M>@A1CW< z$YTgQL@F)O&PNV-BaxTPJV{sxZjMuM?g?=XJ28~ANs6|a`>t7^zSB{KU~#bpRXHh{ zL$MYAmf0?35T}A^Yk>Fy{I5lVjPh7U0|o$q1PuUy{eLZz|6Uv+Dq2cPYREru90Z7Q zWj^M4-ggCuF!0X> zv*gPc>Q*_6o35NWKZw8l=WTAWVNwM+q-J$KT9==zn!6|OySqGKcTl4`qO(aFe z&gA76bI~WxP0IAh*xJ3j7S2~>tdw_j_>#v?s>Sx)Nmy))3)Z|Wxf3yx<+jU( zrBKreU}TgbEgD0b=1{AT?F(9&;+P?io{Un3E%)A%b8u%N(dk-Il^z^QM>Q(q&f@99 zJ)&}DBb0NUUqTD&)bx9UvY=bPS>0H$>onol3iU&Mn(>W%0lsyiluabL%Q780tIF%REy zqLIQHX$$1=-Q9LW=sCcSIRv&$V}76rg;p+;k;Py;b=a&NfPjING!@o(4_>U<s1!VAY&9nU11GC zb&v=^ojQ&lY^c_ymV{Mru1~^nxNC6C7bK5oLNh~AAHw6(*jy$OOW=aa-hIx_Ie`r+ zB4dw&9SAGO$rkzRBa8Fea9-Qp8nUZd{;j2eqpcZ1z?@U{KSAMk-m08$%XH1KY*#(#QF*&4bf;L+~ z^FkzT;gZqP`F=-u@e+}Q@?Bk&Utnw!od{bEV-9W=oZqA_5MuG3xr^*!{YW6Y*&iDh z+T(o%o>_y$BG6-(>U?3asEojC?CbBVJY zgC|{rEzpVyjQCl~Ky?}{lItvz>MV|Ts%gY36Xb!pNo4OY>Q6U!!Dp8@He#TiI~Kv2 zBw3RdS5Ht%GP|R279}GHbHLfkcObAmJ6iTEb2GmyS2+Xc$0V3XSA?MK!y6>ackksw ziPt%%@R-uKf1uM+tOJkVShH*=0)83CnFu!8*BuJyf~CC{%n15saR^>V=V4|cV#p&b zA`2~0PX(!k!>z%PoF3@}V^N0hXme0Ec+}2bD0c%XSRcKRUsPNdKi7lciJLP8*YnM6& z>hPFs^7N5-k9W+Zr%w^pd^lobtonqB6jv?!EOoLYQp+Jzqge5yM2VsIw-;y7*8S!IyNHR}O@IrbU23nFJ?|7Wm08u7wEKPxQ%@pSCl>Dvf3Voh*Bx5jKNkA}1^~eM-`k9kosGSLqlJ^5?f-zVBpEp*1?13O zuAI15RC2t3{y=X5P|4yX=0<_Z^1@0r`Qh#6;@Pc4($`@YWumOLZy2P4%;%+8YM4eJ zz}p>~sPSr5pi~feKm_FSpkR!yBbs)} zPO3Ij(<#&e1=1{bsjVJ0Sb5Zwv$;BG=yP9XUxobSw<4;RyAA1;GG*f2 znzEgW^*c7*j!|{XmM6T=%UB&u27O8{(~B9PBd&1}Z_uuP8K^v5F}zi6mLFDqwro81 zn!7mEIB0_`_kX;f#N(Pq%>z{DDI4nU{0fK&#yP~HqXkFNXBVJdlFUENfzQVJCqVtu z&&zxetmtN1lX`=(S$YYw5_c2e{=^Slfo7}^(#_?|^ zcCRFdJz7ELu!>F*HwM`WtMd;iXy*^dLR?fM>YwRx?@l1L{snsl=1zW!sk05P0=F** z;RA7>s;(!DZ$H2=M(hQm(W+^O8$iKLR-WGt#e5RLq9Q>v|n8)Z0UlH(c# zJ(4~)x;uhR!3DT9T<~&OU>u!Clz_(LFT~iVA+FR zMq8uxkXYtmSvF@q_AmRm?15MylDT{}mRJQfBoCq)4i-(bwM_s|KEH*=7!?-GN$Rr0 z#Uk=QSOtr|N534!jdrc?=WJ~lSkLr1-L^X4RNef_e^+_kcAES3>%INu9-HfXI9dW| z9j`|d2}DVVNL@-mGNRd#W?pS{Gvpg1!uH%pP0oeOgKQffyuVKcCX>xF!WfwB;pxdC zD%d{^;>q!fbBlR;d4a*jyWrX+isv0zybtre`v=ERv!;Ieo zH90gbrOIZFAe3!i5Z1Pr!Y{Tc-o>_>m}_G@S{7DZU^0!2BqV#-jmr~n^HLyDf7P7w zRu|ZL8BNXjDj$H=tuE|1F~DJ#8-3FVV3ChnNoFFOX|Nu>Vq+>pL>Q38svbJIj?E~L zro_Y}dl88O88H%G;jkS&FR2Vn2Hi={P(TWNu9M-5vn9rWG&>7e?YELb;g`z$^dcRb zF4jzXDa-|MdJBaMaoqhX7w+c=4upwpNS=@hWBkgPFRlcJ_0a6_@T`!iZw08`MjU)7 zLdb%t7K46;L+)QMQ?+KOR{3Sf06*Fw>BOpX=L`D<0mG<2G1>U%1r!Q>)a z|BL2iw?h$R%84}vlab+iJYjo0(IdUiucxOcutqW51NO4VDj*~<9O4X3X|tVkKW<}M zYQ~5GEp{I3(+A^wnmqW-2~6xN$mgFHWjWhK(eETpr` z|3J@FbS*QeLU>Wyq|o%^L-KF9{z{>V*E9d=9DS9fm2)z`tuxBZgI|wGB5) z5zHtZNfF3+yr7o;`Rv(4LqJKzxG#+tga1a$Q=oyHBfV1`A(vldNEH$GrtF*pjw8sl za|Vh^SHnmWd^f{d-ES)%*WPE^nH=VOYj>AdnMNDRk?-ZgA4Lu67md~{pWo@jI+G!8 zNwWHV59?bFb!e~D+^p)ZjhmZXcUaLr(5t#∨z|Xx1?7Z?1av3e@{b*3fNvZ(Ynu=v9di16v4&}%PT0e}z37bCG2 ze~MR1iJ69F)2at}=Ld~Hc+K=Y?A*1Oo{;s3na7k4A*HFbK7r;DQ2I;ymVrr_Ycmgv ziS@=cizbZYR1;67@4-E@_FeJh;cMCXS=?K_{Isv}X8xmnY*&cuI-F$V>i~BIZ3}!$d zvtuv=M1oHQf6mg0Cs)}VQQY(g!oQv9K2+O`MP3vIi{#3X?8Lk`O%h?dj$?>_Mzx!- z+2^jFIc=GkcY5GA`44-wO5GWykxg+(0ZLl`$Zv8^O`RA`8E^#5w5feXh*8&)oI;FaAu_T~4dg2a*A7xqT4%*lqbnmCi+)KJe?d-vzF-6tOcR2$0Xkhk=urMDtNNj*yepWI8t#wO~C-}C#5B_c){Sc zN)b#k5;VI-t<7RJ)OisRxVgfrQ{Aeyc&1s6UTg@KR=|D;`_tYwAG5Em(awp>1N5je zR)4^da=tBEKP4x#C&!{VpCu|ZAd%L|88&M!6fH-h+{-Ts^sMW!s|0oFQ}@eOYg|N8Kk0Y+u;$1IGYX^h zdz!TSkqb|>L>>6^gzXrIORkh}oe)$f;Ps$x>(a#p2D32x&Gn#Mcy$y{j#(WUzbwH&G4cyi;(jmf}{>6uK`<;)7Vl12>Gx$h?U+l-fd zdk(sUt(FYKaGn{2^XcnYn#|+(zAIbhDy1>&bC5S3P{g#m^J5)kU#i1)tjer3$Ffj` zFl@eoXeNS1nY4S4$UNQ9<4^W#aH>)&D(yHJ9nYYu=(p-*#XGGf2*L2la77%ty1ao& z3(M)pJMiTuCz^84*wKbK2e+*wak>%l>R-AE0MBe4C&ju-; ztk2yxpy}7-LH-ouFO6CrR^~2gh$`DHbtEi3raazts6p5&gU-aZEUt;Er`p|^qgb^@ zF10iA8qn*f{@j||!ajCn7dcEGLG-_Td4C#W&z6O13Ch+bsFS<=aqaph_jE z15$9}j9}wz&I>r+SMcg;I2hSC* zM9%59NKwa=&=W{xDzJNM#|u5L~#+?6ZAlYCk! z2Pc0TMeGQ~s<690;nTss7b(^jpZ9Ft2!Z$!at0`ZUe&i2nrO54Z%b|{wckhjsW_J#`3%S!#}qv>%aS_^N@? z&VxKK`mAWrS0})is5%F*wl+4&na&Dtr5w0A{z%yU>U)o8sA?Jaec|Te&C)H%`8; zBk>JNeM#1J0aoiKukA-$)iVT>b$YhYRAbzu;M)h<3=~{3jOG#y9b?nnVZi*&Z`c+X zrNt-zY5)ZeeIB{Gh?kC$^&-WQuN9Z?Y^yTxP}3O~Kbbdb)&pM`AKmKJhwnVlsLXeK z$#~;@eU~GlWs+Pq$r@{MQOWlj(iIg1*J2^z;YrZ*H@H*b*xf+kx*wD3TP!ZaynB=0 zul5PPo>Ko15Nn_sb6;T-KBgxM z$XJ;MBC)Kh9o~86g>P`Qk22AY0laC0{$mvLB}Y`kwZO6J%QiUF1-6mRT6W0?%|db zo}Z?}9~-bt@o{fCR;&>qoBp+`Dk((&eJc*GYGS*0gSdfp3BZ42n7QNc?HkW}M8SXa zTgge+#IAKbj{g&pjz793ZfbTmL@dMQ&*a{#96zt;}AaRPxVr)ehTgft?5vpPxuF4&}7Aox?5F( z_f##YT&&uS3<}jC`MPQdse^}1YS(>|EvoO+#cD-U$9d8f+VnU>r9iyAj$QU@Ie+l8{VIeTC&xg;A&PkQx9o~4K0 zW_|k-8;(EdZdhsFqDK@lGN4H)!`N1XJ-)pX9Q!6Y-_5q8;Oj+lZ3={v`3Zgt<+9_q zAIENN8`69v#7gBQ+P=SdT;sK(uUUSPcCLrFP67M7RRxFSJ&p@?Rca-Lb2X~tE6XPc zp(C+8egAhPdVm}hZSfNZ`wto=`ad}rDMcYu)1M@m%l`tSsp>w8DXZw;xqf~hL0`WF zQ$$O!2=9I?f~%XsQybc92>B9;&o71QIiYLnV!$?}HV3mS$c@RCWl48*V^V zOeF&po>B1XIGhYrJ08H5U$cThJMcDD&+T17XyjDxmDMI}nA*pp(a+``T! z@nc@0G1zx@gA@8CMCj$?V1z;4Ou`v&2<~$Xl(XyvNJDanG_$W8b zYA(b(t3qw15kK6RP*xkI7C4%d;+L>h#3Q&4d#l`w#`jC`C*C|!qj%3)7DD!L0R zdl1(XEaWW%@e<41EuIQ&9DjWKqZxx=KVh!tG&N31@u|U=C~VCmitpvojV+u++R1Dc zhO&d{{n<=tL049HchWn=^qshLSv00y>h&^r3Y!(lu0jpV3D=>htkVhOjX0aSNatPH z(q!#qDLRLA;?>#3ynRgOWusWJqc<;P&^%Q~uxXBK!1S+^Y>D%-yCmk3~8K zW*n20_)5CUv1^Oq*eUE~O1+H5PZz9mu?tzo_)(@-5hH70Wb&B9Fc^0K|S`Y%g&mNi{Qa#%)&Cq z_~r};bH$2zh9(jW80!iX?;OcE1)qDM%G?nX4Bj@N1q`u#g3{hWOW#q;-WRC)UVVFM z5{FWyqucr{Px5j+!)wy(C{k}K(s$L1znIiK8R9=yipM2##ymTtv^@37=LrS#jm?%e zz65qg<_ub^I;WEFmb@-ci;6%X5992xg@{*=@R+|_T}>(GFW5vEA)1G62#yT({fWcL zbCHH^(MFrs5>!KsJJvn@b3s{2kW&YcO!RJ7THST$Ox3Srce$)BB0gin1CKka*xOqd)XolI4ucSXmrbU;#&{oT9ChN-*St&=Iznrd|zD?J{nMSwOIw{pz&O zrDbh4l7p+e@h5>y9@FB(7WYoQhoH7*ARI9gS@e7W*^nF3b3~V8*LpE*<;TJbw&2nU z15L^=O~?IRwbsguRUa$A!_lah2KlaqY6*Zog7mLce<=_M+iC`U1{P|x!d)7lP2UA< zEQ`_6!OZS9T3-|#yRL(2+kC$Nd)=RWjZcqNaFbt;!&u1HG4Qpo$|JO)o{= z-LuYEex*a$_re>6JsZS++QZ|$s!Hh$6j(8UaVQ|va6}Uv1fpPyu&G{ob%YZfSiNU% z`NST($AsIz^+ajAr~kwpIv9jrIcg&hOir5e!p=)l=A);*k5S#%(h5H%3pp)CC@+b0 zp_QMq?xd;B@4FYN%5y*#Q&%s`1AU-8SQ_ojX6z#sSQZWV{TY@K zs#zZTKudyokgY<0YkJ{Yqe~h_+^a$C1^XdoWtn=5ypCkjlHJ>q{9stuEm|THvyFU! zOnzKEce#MZ{Ec`o2qeh7F+io-*yM>-7rZrtA?Rb6qZ&v>Z=zc%wcN-_K%sCN+a06C zzN>`Ti4AxH)y?$^^rNWS^W(x#zfQmP3JdUEWxbjL+7mLCNAGc|%E(Xw{~j|_y&@Op zn1uz9cwuy`@Ya6FTyKEIrNvFT`Zpa`eP7?+Ex}uRrz>?x?f9hq>uyFcS8p*JOF!?> zfA*j@u?k_CjP^Tm)3b#fa&M$^*gop&qHA^$WtP=>z;AjH z)iJ)rp981zs=gtfXm%mMKG*xtct7CL?+D}tX!S6oV!Xk_L)%S|nZc-?ky6Ft z*f->Vzoh6CX^$|10Rh4P_+Eb)^ZzHY{y$7QQl=7S_GZpTt`5%sr3}TYZp!0opy@&G z&H&+>*vsbtF-LZcY(b|Z^@AB(7tpVzqN`bADNG~z$c8h^{oOkC|7BoVBJEX~(u`h! z@(lV{8L<0DKs5C)wB2jWOs;fpxZA9j#jV@Rx=;T5@om081XG{_Uj>Tg1cveEx*WMW zON%dEqMoYTa>7i}{%@7Q)?D+TK0ltmZab#?lH}@*zbx*3mq0 zxyYFfk;b15=HMm6pW+YqlLvp3V+)ml6Ok$+F`xBRR%V)W7y$@zlq7q($`4t!nTsHC zmKs^{QJ77YkZvFz-@8WsOGWy^K=2?pfT%Rq&ZX2Bp#r5KSo`0dC1q8#rGsVG)l=DC zYD5WiG|mF_xLGZ&Qi6DvEhS{?Ol{42zZD<)oJZ21P9|ZO5&S3OH!WWau7zP%^>4DQ zg@hpoL7r1}s%^mCw5Cp5^l1M^gvi^pF_dRDd2Y@IHWW|rRK_%S7XtJ`{M$J>e2rPX8um z!bYzMt2K$sI8uN6R$2$c1cksbhnC{@s)A>?_LgUnR`uBUg*X|d%rU;o5MwHP%b4=r zm?ZJp<%>|tGXk47X_&pio0$?w@T6gHz~uJ^YqZ(=R#JTgC0;S{<+||z!$Y@^ImpXt zUM8(VHC~M-raro(SpH}Y!;AHLN#*G@ecla@w66?DduCU9&H+&nBJf2 zuo9OUw_O;pYhU2YT-AH*tr0$(P)L1qKlGKEDr?E z-dUPaqD{`ib@KuL-zv9Q@j3AAorv?!Cnv2Ff!8Fd@JNI31b*$2jw2oA&aThM_I~ku zz&GQ&dPz|Ec^5-^TUA`Ms(71m$TRhi$er#G4#&v0SZUN9 z$K@9KHoOAE69~hi7EW%@eLrS(28qDY& z?e~uk@lL4Q|C-Tnkbz<^8*R;p(()+|nt_S``HT0RJI+v;PjsPH$D>2~Eg>IxO=nc% zEyL!GAm2FFLhUWtDc1vky6lQ;;eWT+|3`yE`WNg!7IS2vWUe*K}8LMgny;q{%InU6^sa&0%gA6L%fr~ zw!l>o=E%(4etntA-oDMBV+7JT%NR$-`L~q&wzDROirx;2PMNzuolXH4#EL%P_}2l?~IBzO11Smf#trW7SXBqur(RLYQx z1*GZ{1Mo3h;yFTNM{@_=MEA7gq}kS*@`;x{D-JFPXYz!po0gm=aGF`j%Gq}i9R&J6 ze+u7c4H4|(u4P*XAF}DJT&j!WaI)|`u&tQlL+jI9mX`%~$M1Q@*$|wDJEmhJuk(WZ z{|`4>=Y1OM*NYjEu~Jbt2&q(b%^<%NPT=2eH`5 zfWY0Q)2)w9y>pmIE%yM%SC84p9GAS^j<3J@QyRe4V}XcfI46`6q@)uAqGS%40Li6k zEdfvVrI_Kd!$eg+ezbK#E@Lt}_#J+6K7f&uYn+%1T{2ti{cvG>=E5Ft0`p>8;<6$# z`+~?yJw#1>^tJg$b0?dpFg+;WmPj!WU&V?Ncn``^Q86Ami%!AN>GXx$Tkg6rekzn@ zMO>B@X&NCuTVwb^Ftvz?9b0llImCgM%8K=(4SY7Yca}`LEN!y*WPQv{${&0dI?Cow zMVV%<8@p7?VQE5jOQOV9@=eGU=INMG;V7hwPb$>O|HIkNnI_ zeEcT)^+=zH58d))fo2;`j%f6$a%Gy^9!k{F3^Vfu;O0V}{-v5lty^|FA(H$VGtCUI zp?lqFE~}1~mY8{F_8<9Ceyyg85iCa@fV<8#H%)*vm=z^7uMRLY&5P zM9|5{LX_h$thua>TeXNYGjdbE0$*Mu!Q3sOLZwx!QW67imgY$d{xwuH+nYbowT_Zl zhdhlWPgwF%{yv_jgy(fB>aS+b!sa9+^>N#R*E9cc4Jmkpl2XB%MTQaC!C#sWH^Tw> z%Jk(hy5bPiOdYhYNmwX&EnO6Er9_`QhCA>fH?A%y4cC~CW|?QT_z^FCo~jb1Pc)~J zmjv=3+j$24Vh<1Q$hXi}%9KdswY`=?`4^%5EwdR8ng-ktiHC(dwPU*K!C>oCZuxWv z+@&P=+Cqvc8fX%}U`qbY0tvA|sz0oYmd#X*(kvOd{eJXbeOIPandNj$@%VR#W?feQ z8FX1UxPyqf_M!KoM*g#^VNF7Mqiz((j8N(2*x10f8_?gr#g5RCxn9DeVsvW4fesv3 z&^y&o(=QTwSr}@>O~66>Ml%RGSFpua|JRhbl(E_?o^Sw!;%ww;Neo#*@}AkP}&PiJwEhS&i6> zqNhnR?hOIynCPT9e>_}sV5mDu!nV2KP^l0+vUfgC-z3U{<|}LPTKzZkpBcEPJev_G z*l0&uSYs70Otl)7GbF2NtoCO0M+%@Zsu&5S;6vGdXgZ0Y1Gz?V2;M!DvtdWz5qq(Q zOq9IB_GR?NsVL*s@6t2OsJ6p&iaxma*u>q#BMW4H1^>gNSFqMSRp35`bc5h2hwSXZ zT)bn|e!vp_>d5?+O3Lse_&<o%?tEJIP|BYb>rpjXCFd*|&tc}ANS*wOy?n== z{0iQ6=2mtjn(J$AQM&KPda@oyFjS>Ej1Q^{)@ds5piSAUmvsM{gZp_d+5(U z(Eo)W{whRtvj6#C%+tio(bekzwwlEz z&DbygLK^!%++fZ~2@{s4PH9zC6i}oehDLuz`2`cj^+fs_+f<^%&GE3&KXr$cL*ldt zoZyL;00YSdWW3U_KvNPiH5>l@n4PH?@cH!*H1MPSa*$-{L*%<+7xt4*t+0>QAcRE{ zac!6GlhOg?okh**Xpo{BFH&O=QH<@*qlrpQg!^5-!ZMQz+1;#&TfkC~;t%VfFKwJF zO;I!Lpe&C=&)(bq$~~p5fA&FIaxHM{3?zg^KwqlK&y!yHfDmr}YrQTwXb*BMEy9l5K>qzfA2?d80 zsGQ{^TC-&Mk8fjAU`TdUgpV&E`9>8he~(U^BXjEI}bvtIyebkzPfHs;yX( zqDIk@z9m|2G|!Eb@;P~P0OC#Rjd1@oEEq*6ofP)euxUOem5U)>tUY7|Z?a7X$X#WVbc&{=bTK4+xrCs-aqwwdw zr=8_(H60r!qlipKI<>1$HRMH*myDpJZ9vgt@jeohX&YE+X--)H0f(XWVWWr7gCm=8!?t z_HuPW*zKiZ;b1Yh@~@_c^A+)CV`EIUwT_qH%Vmu)-h-**uW&f&KT#Hp;lUi}OFmqt za!u5=^)5e;Idua;(bQ}UYj5T{Q!XsTVPTUvQ>s~@V{$GP;-%U&<}|vL$IrVMhOci5MZ# zU&cXv?gIzVk@ZdqajsWDc%qONa;eC@s``UP7AbbqCP9rNyn`mF&1W1|l=aBEIERJC zX1#O*QqllRl5(;a`-Vl7Ja{}maN=NrM9=;~IXXBCn7D_p4B#L)qh^In54keRPzWGG zddcSj@6rUHw$$eH7>LOnR)_<4i&oWyOy)(q#fzde{`X6ms7t(5P1wcE+tXs&G#Ulz z0xmoj*;3Egh@k4vi}vOe%Q${B0fc-w@TU(O7hO;xBUH|1g}A0yv2<>8q!q@mOk_1S zZ6L=*jE$3m8US9ZGp`Br_e8t&rWGW}6yYtx2x?HGm;k$5_Wm#`zL+b%8}FI-nL-~h zU#j;wWeGgukh1J@Y*}h{vC;mSB3qHfBtxowfqtxr0SY!cl4T+wR*oAjWYj;^J$k%x zOr$)>*f3t2bAc$C6gky41BZTJCx*KMDN2NT%IYY(i*>Ev3C}RgZh9e=ZX(BAi&4}U zT`Xv#hDSO;)cbe$m6M`(UaguF3hg|sb_ z|NKp1dJ#qR;_}3Q_8aM$-=LKJ3%SBpPu)+()~ZHt6dh-Do|P1U^rLaXkp46ZXlD&H zN8>#1BcEx5@}_!fw0mBl2k55oz};P-U=CQ#^5W{*IS$Ff4bZ^R)xLyVS!O%Ls zZHte;q3$qQF<#@b$UP{|Sn*V}CI};O7Q2pt&r?*IqBWJ(-K@Fh@@ws@PR%>A7ku5x z=+$g-ibWj+qcwmh@Zl|nnptg0bLwWCHKazvLthBQRKk|}D_45AxNmh?7$n^7|}W{;Iw^@I;(pI#(hDDyb6DmgmR<8+0SeXOnc@ zf{kpC7xYwJ^4WnC4}PkG_QZyt8BSw%A9y$+41xpGc7g(vK1HZaT{9gH3P(@8Bzwr! zgv$7Fc3p~p!&DH$U+hjdcVkONxlARlO4{noS(v6E%Ant-Bo**b ztKZ?$4vUPLFQ;k*b-u5Dcauj3J01IdV3{77e{sq=gl{H&sn=SX1|O_KbZRei%!PV8 zIrT9}Wgb8C*YrOj%7*}8Mlf+n)73;d!igsH#VXA+eE5Uz46csA$($M{n8T1qCw+wK z@MW)%G`wqoHkSYv0<_l{^Zj!f2v;oFY+34d5gTQIjn2A%ZZ(OG-~9W5w1>Yh9Tt*C zhSxtIVVdewJ`amkSkC5GZnJHCFa(?ou zA!;kHGaCv&L8f3?U0&bx47w}sKgZa~7G28P{n62&M7|BDur~zCxgmSSh{Jap#=THS zDU!;W2?4r`!L9C$bRJKsi*Z(T=G1hj;#Sxz_`k8jTB}@kU#+lCZxQ3_X9OIXT}BWh zKh|QpG1%%F8Uymch{v5gO)53COqe(;ohIRTY)k?JzypoFMP=iYr*00ii_KqIK};y5 zDm8u5n9D%J`Qz=aflK^uhQoBUnSL6FJ)7S&o-zxBaTaU4u87LjQc(2ncqg@#M3_ z!9JzrN;L(x&;$>M5oVpo$Qozduibya6S5r+KoNHGkr^uzCFxQfPJ+jqez9N-LCfM! z&+6H!Mqw*+_xrFTam`T%QBD)p&+QjPJ?`s&g$>6jK!>y>)$CSl?W${jQ5{S;l(_@; zA|_uI_YyGkgwex?k9~csWTR9eRuxu@UZVusHYRX~Ep(4V-k`;7x82RQ5z|5r4lptA zKXO}FCdPzZL6e#yQZwpLnCY%PD?8FAa5e%+oEh zMzsVUMq*+Jq7sv66NNb+gCkzM&MYXsw5FVJ5ogJvT#w+4)!Zd3RkLC(e50Xm0m85v z`dbArcd#uDuv(tZ+Gt{@4o$|C^ows>(;5exp_Qk|c2}6~YH*BpJ4Yf!L>+DH(we0=?35Do6 zJv(b!Kp@#7<=J70G$XunLls9P0!NUJ$(n`1Db{hNglV?q)Wk1tpQzpFO7%pcLg9(p zY|xjicp8kK$TJ5nmgu+SFpTh>3SkDBB#t2n-BU^i(70h8oTF_=Er;?Dg{tBw7z5EP z0b7~B2O^x=#dO}SN9UbBBGkI+S~E7cVCIx{T1%J-ZEE|p=n*~VQk-gAv^1T@fz;Af z(D^>~5Vm$pwxGPj9$5;=iD6Q)L{xtj5G{~+3JvKN=vmflAs~=3lLrb$wo%^yLo?rs z^86;EkwF=F=Y`tZEhgv`Vc7=S`eng%J@T=e+EFiaY9a9kHOnzho913Gk_FFl02PJi zW49t51N+j3Yvdo^JgA3-iBW?xuy1oe$k-=%gL^^#GpFTMXsIb zPI9blS#^f^yrCvWmcQF2>I}CnOK{3y@?YxrEFXSj8V!Bgz}QFH$AUl6?&{aF z;|z>C$bo(WhVn)#&M#H=9yVRk2DPjKZdQk+$qQk&)4WVcxYoQ*OSpzGz226Nff^@; zIEt4VC!BYw#fcEKm*!MCh+K8Q>(uebSmfuqqnUVy`{$3{KYD*cCn{hjJ!svdh$hYJ zuRYr2*~wYlu7*2uP2{&0=#d}tT5{%kOh2&UC|Sf z&>q4$H!wMuy=71Z+SF)GyyBZDFQT3a_5r^ArPevZUcXVzIQh#nrnjX0Vw-0R=LXqm z!zIhS)QO|GC>J|Da-V6r0q6WQo%yd_OaP=qFYNlR2)NIPs-$%cS4E-VSaJf31|yjm zw^3!!BFuz|qrzmxA<8l06O-L5&^)RooNE8!4AZs7+UhhaTconGO`O)F%M{m9%8->? zef^s)8%Pm+XXFl0KH={Xg}dwW`mgMaXVz&SSX%!$ufF~$gP>*J{?}USg-!VU=1Vx( z`!5uJ+GqM8#6Ftmh&O)%j{EU)AGyT58#ac&IDUGs6T26-t3-p}gF*F_goA28yFh~n zi?D;zSxls0Yw}I)2~q1(=vPHF>VP;4CD6Gta9v@2hGLAI;ktbq)P44+^a9ZbDbgzz zC=KioOso(cxIu=$5PmA6+sm>Bqnlo0gFY@Wj$NHjswa2PINTb%MLDZ=6(BD6?jEb^x$ONtUyUy?S>i{&r zT1#G|Ervn(g&%( zceH_Ds}=D>qL+gitwr5461I0mU6;s&*a7_h`+}JKTV*$EYi^R6>jg?X>(@l zg=x@~)g>VVr=S8glonN_{f*be+G~X3Rr1H8Z9g^EELt%t1{r!W=#``ACBOMel_`+` z$q`HC$N)?wc1B|9d@-+g8n`cZ6;(E@I~+Jz_A?CU1H~r{oTwSFj)B0>R;7QO!bt^{ z(mpw*lTv@h+zq>JSt6}d7?+`9+UDz(A>I@YD@c4bNvmoNY2AGmu-0Nt9KPhW!?*VC zZ6f{>7EE^|)H-Ao6^#DTFmbG&=Z-2jNPN@kf>KAby+)Y{xZJ88#XYA%lrg?FMg8bF z=ekJ(VLSYcXDARs#0U@X!I%oLYm62#V8{cbm|^H^o#clliy|exP%hF<>ZZ8R(_7N% zwBp2R2zCDMW9i=!3G^zfdz*!JD)v)g**;Q7ZcA_iHUsnH4+rTxrrjytKVX?@5pu<}feB z>h4GPRC0?u^`>m^%(=Yx2*Mp6K7rOhnsASp7COfHqc1Wd4Vg-NX_eq|%grHdW(*m!7ki~KnIB*s?E6G=r zAm__mle%TgwVh^cbC`xebLgCV4EsNCHqZ%sJ+X)GIn|3nL5}LjNYkU2&sH&%@ zY31c-&Bdu{Sj`36X(2%;g)-imG3CIegkKcQ7`#Lb$)~bAzy*oMe zlxcefr@4#r8z*xR>+-m6&^sDAoH>2>F0)T?kGkD92F7c*iPup5bap+}zFDR1O*17d zB{`O7dgn@|$p^~zv8;Wdu2d6{bDQ*Q{`C&$`vSyHi%2{{P}VX@S_Y*%2HFUVX2eS2 z2?a0L9qur_Ru`ew`0_|}yWF!}YM_>;@X!nWVt`oQG3gl;UiWI6YHAGJg_LEe-p=BM4w_JEZz-QXv;l#Vt#wGu0SUT&|62(2KTO|!2)mNp4=W7+BlVK} zpRBNolB~*4AA*^jk)xxPy~TgAeSV5F@`Hv*IrO0*>c4({?kn&M6?5dV>&wbYR*o(? zerwbCmo%54?fwtJ7xYbW)NUk0^1#f)y@{8#i!k5e7HIa*asiTdF@PJJyS5{u)M4W_ z4L^FV3%My1ZWKpV*o-7E^gP}!M9PsfUR+sQX>Hv=tjgO6gqVxM6rutXBkiHIvNVuH zjR5>(Koq7#c$CI0|y4R(RWjgyJw@zgTxTG$j&MJ7mb!sZh+4*ILYp z$H-f8R8DlSZ}z3e4o{+|tMB|t3Mt1BJXw0)36|$zT4MG9W(DUDE>ldJ@1n`nYv!V@ zoN8M+^|S#Bi{LIO7`9ruM{pf;kI<}jokDwpx<9@r9YGkU7&gFu<_MaHK0*Jg(pYhKrH_fTFW%-+zvbWUNi&nM z^2$p+6?*;P-{rpbntPu;?SA{=zLDl>c50Lm7Q6F)Z;0@3?{5^Ul4)<|NsAi$`OGhk z&#Jmoxyn><;6yzy-&_cohN#HKmF?ZceaVv7EV`W3dY1)_deUXWj3|Ey&l3-cb;u5< zw3!7+sI2CNwO-4~ZW$*tQ)p1(Y_sIBB*}6^wlW=i6fYt>Tp^M#DUzxf*~sXqX!tDN zgz>}UcfAOy%%0XySR!DMB z6ceMMtw9OH5mZj@XHSZi^R@F5$Bef89G$buy+z zNbcesLG@=kR{>yE?x%mNgZ^5t5PGWW2L}9u+Pb#t0f7%y1$c_E;6=(p(*=`jDNcaW zY`Seo&&WQ?nKUUat@AH!+K>%zIV%`Af!XO6qvWg@(tLMPpNKY~& zY0Q}W2O(Te$SxyEunFwT_r}kD||5gD)8x(xi?Ynf2V0ms|Va}iuhMP=h2o!*8r>>5>0i&s!9x1SLL+4YO< zj#vaobQ4Cr{E{N#!Nr?U0~mD>;?qFXJ7c*Q^AeQ}*6=&etbG)+N3aw;iIM`?T;>H2 zW2#2z1rrs6?mNJ>aCF?N~YhI z&jJS0th#DQC`laqWRiL44Bez@n70@*YdmRur!RLTmrt8w@o|dJ=rWiKo08clKB-wH zq$wu1HjQINeleWd?CpLheP z8@Hs^?Qv@%xcq-1(5n6RDAk0S7fzF;q?$K)5CypAgd2(?}mKYl_%w$>VyBFk_6f5c-Fo1R3 zc4XTW@)r=MEULkI5o8pc`EPFUZ25KcP=jp&vP(`#Hv{)x?tz0g43D<%Kt@@&5*#67P& zi;q&dQRBK#I-+P<)*IrpL> zb)_(?jubR*Hv(UvF&R_cE=Vp!O~$H8*oc?D;FX$J$1r{Rq@ah!&Z5Q}9hKYH;;hq( zJ;%CUIC5nM9BHG%a9dh#oa0h$?@x7oxp(+h=-q4DZTaH81BAK!nRB!dr%@UGV9_U%eZw4%uFh zSj4+OgHSo>BaX(DsA)z`R8t@^NTQ3$95Nq+svA$yi&LIStyiLk z;^8BYGW@#vd~@c{#%ZVlr<(kqJY+7nAbT#q=}~>ISBus~E4y?R27h?TvtapfBK{~9 zleek7}<5=j<3wBR-N5TE?`}QB?WiA@%`? z)b}prH3wr2n$}gVHi)0BQr5|p?ef1l#3=L*hrApgX!K5`v83Jj!gnQ_aRJ(77r2oj zPt55AC^%uXT{JT+r&_T!1=y(EtRT&i+5!6OmJiYDCqtg=-gAj+KyOYM6mGqB=UX)} zUH)LcxvXF0AN&L=1=l)m?ZhS>KW}WuIz$?AdIX-f=h%kzoaI+Tu3t1OXdXLp8ED67 z4c!VU|9FSSFpkyy?&uW$z+PzOdW$y2d$4_pT*3I3jIPspew~I>*rOW%G2X)B#pUDvz z#S#hS0$?MBXE%*=ogrYM#dFXSs^3b`qL}^^#sHJMx9$?c0Tk()4kFw zS9&`rP}k3pCSTASX!-u}#Q0U@C_^U;)c*u9#ZtFeGs!aY=s53JE&5Yyg(%ilD>cXI~ITg|z?0RG=d z6PGx^lXd+QZ1;noEr2#O(EjgF@qja-$_|PFr=p1N>h~K^XWWJST+zEksNQ)WH1OG( z00R|!c)(@8V8BVnN7$66Tr(yy+u35evt=aO{Ffc^td0!VM~IxmCCEl-IHw+e;lLeF zKnw)xcRSBc`+ao;4}**CKB}EZQ!v<8u|1`6(x`5;s3JW4vHvW!ZWyAPp*)5(3)Qv^ zBk!Xw#Dg)1bMjpNIX=M;Rj zZ_I|-dXMg^fUcO3Ml^XU&YUZu>tpMXn@R0@a~l`Ax7h*MLbeINIl`ajq4M}En1j)MRkjZJYw6W4-VpTt^v+ejJOi>lWYQgs<~T=rq{iI< z*n;Y|6z<{K8LZds9=9{=WwFOQNbu;ZG)H_O4H@%g;bl1`N9c!`&$UA1!M4~6tO$_! z0jEq!3&f{8R1GBIk^T)qLpE70@J`T+SsTrJ%-$jLcnn9HTDM7^tD1^-!XsT0S;hw{ zHx&R8jYXB0zh>6-L`vTrb?QJKq?e-VkQMdGF|x=j<=W3Cz_$~T$0j?ExQFyqck~p6 zkV$9uHa%TD3Ot?DdpQ{zBGd>Y3^YDjMzJqR7SPSGZHDtHjjVKv zKbgTq72{EltT=ig8kvzCaIble5|9I;4n~32&mZzlm3U~&9KbIe^$|fc(iaAAhP)b3 z)fffvS&xbirGS+!>P`C{?{z|1`UXuRSz(Xq=>`H)sy-M8-iAM&+5O6F zW5hWDYKxK4ZJ!#AA*RXRfi~?BjmTt)2$!m1>ISkjVZw=pJkI?lU2@!T4Y4Bk)u{dw zcJheJ0u~Pp>9!E)74uW#;>e5fqKTaDScG`4H7nW0on{f{XAIq6N~t*{^+u_UGDR(x zPBf+B9D4^jwTtu{iuZYZly(m!IhLfBf$BnY1m`4$jJ(g3R!!h-7E|xU@-0%z8%l5K?<-UV!-;xb_zL4%}CFZp#U z!XGV5+&oVv^-7SHpaH0r&s0m_JjH2G`d72xC6cfuB3h_VsjpLZj1%i0W5umn2C&yw9%1h=c|L5)LGnJ<5HtHsbCV%6wjJTt&Y^CN*YgZrm8-e zi{d?|J#JhGPJP$`$!8^ik03O<%_q`440opBuQm^z^^?_@BZyY}OagrGcpG|Va~|do ziv4s>0T`?0eS~*X_8U|C(^TLiY6;SG3LJiMsP``khQ7aO`$X3Yb|(3Xc|MDx(;Ul6 zI5pe#r{}FlS%}gGxvwlhi>%!pZ4_J5aI*VQ)8dxTINJS>_gdW!0Tbs%AL@png{^_3hq9f3l8}Uigsr`c^N;djVqo*%@?VUi&Of-% zJWHyOCQ?!Xf?D!b-AJP028ZE&r=j-IP+-tUFfM0t+se<0i(=54Szf+x!)>`*-6Q?d zp}{nmF|CqU9JzNPMohxFwmsf07B}f1ES106teigj^JNEl^JNzzjl6i6<*ZJ3}_f@wa-pqV23AAv&_uM5|w3#V78d4wb#q-K74(0kCyFZK~W;!12P2Ed^&O=R$n1Paf~ z(fpx8WQ6XcIVTu=MsUCwHM*6AG$YL95a~^Ra}I4QXP;9dCZmNv} z?`BkO2x7TXJk1{4^%wJh0HvVtpssR1Vf)uYwUjU*IsH*WX@3g!pCfwzYvJ*q3suF| z!suro>HjI!6oo0&^?HFq{Dys?SMvRBttv>xklExT_ z`l9{YwQi63_*icA1%;6@*rY1VW<+rk0U5E-L1;w=bm4J4i-~#q7A1DZ zg!M~raVQSm6czf3Gy{>(@yR0$P}fj=>`F@{9M!S64<|ArFioaL2vXWH$5}<@$R7hv z6~s25Sv?i+1~EmMpCcDXu;wr=0N0NhW_r>Ko5FPwFz=O*FiWyWkWH~3A&rbHbMLM( zt)sgAfYj6tQ;d=70w7iGRaWmjrw(xQ3xFOKdNBi=j83Qvk*6Q=_boB0k02{< zFXWuirhSLKhJCU%dVA@tTCpM=cnXRkh`xO`7CupVw!o|$@bwbSA&9u~?Xt`J8wVDL z2-@5QJ0nI{N9XBxuB6-gzBy2egKW&Zpa14^N42k8Ao~d~1VH%@^NjxrcNVp3Rn63!e zrDiRk)nqI!3(VLEahejv^=}1@1Z$e znyYMVuOPJ7cP0aGNDch$VIQeF?s`5!enYCar>u5XQGS|0h_q_dm+7$fR9+O-e_C_~ z_}Syt#T>I$wm_<}2zNpu!6H$#DdjnHDH&JF-AD)<0ANVSQO0i>5RoC-bBI?N=&8+43*Xj=hZ%HB=jtK&g7!MzD`{P7`SnDV@;G{v6ci+|a*f8`= z#e34O$^TOMlWLI;q7tB_KLL$S?hmyBxeX{)rg04yTqyA9%Ay@WJ1_)%O6u}dL;UHwkoTh039hOr=wj~Gj(K2@L9*jB;OBn+@K z&Y^Vex{WU%KTvVVxjYeM3m?u^Haey(5sTlM|JyJDUuQ2jocIO2+Fo`!*|jw^zoMYk zvJwa6DXb%D#Kk8-zYRfYAv}dLb6=@mBHtTx2lHZqTK;OT-J;GT~nUQLsE$jkT&!Ag^5cNKqeN3zq0sj8Fk9e zMs0wh<5v<&n4ZTmjwOeNcUXfdLO5Ek#Q zFje;)C?ED^(K}thtDI)JdfrIj%t7;};$#VN&a}Ua*^0$XU~}4&;8upQC9f;EI~@-f zTb4{ymr1IL2ws>#XCnBCTkOT6+c74=8Wac6me7XJXR>9A&ESp%CWh0B)7}=T5|g;| zyLaiV+`;t@2nrqdUZJwmb{J&KIK5;qT_SPMMp_iWUw{QKn+wlR|?RLB_4?VLVtxlP~ zPY=aATp{juht#b|3O+HMK#X%qpz2*?K+H%Bx6iX^VnBr`xo%Uq7{_jU=) z)2lgPOSJV;jvIlQJl@>1bjRx>0&6=T;1${h(3m!vh=tP#mImx}Z^H#wB@0=bNV|q( z1pMnB5Vg$<G_!CKx}XeyWT%)Is1h|9XA zQIT3gwxy9xGi-fBb@&@WeP(?70=_7nDY(NB+~lpUuzTmQ{H{$4C+hQ>RjkAhT*^^0 zO|i3khcTQoaz-!f6DRP@#B%27?~1>PFd$)z5d;JEh>7|vS`p;s>q^tRtwC4FHKY?* z_*Et}FPs*^+!^LTi$=;@seM6*h!ib9jA{c-d>n#WjltezhNjXCYa-!aN-zai*vBz@ zovtI#vF9Ll#_*D$yJ&r$xtWP245UXH*1F?cmE%iG9!`NB8q zETN-+5Uig%HrLi?SJi1QWed8eRT~O7baj1 z#_W*{cCVbcP&+~!tk8AL=B)q*RP3>3;V5;A!pXpl*4KC=0bamw;>~GDH$zC9V3vJ| z9M}>l@l0myi&EGcMufF9o1?lK@@xp~E^xP{v|mbW=_ok*bpxsm6@zW14rs831PFPK zonj8`xoY_6qlt>I4ya8+NJ+LNMqziAKG=&)UMnq@V>z<@hM66Z<4UEO9rLtR#*y$I zJ9f-);^Gb?(3m_`oc@?{uGKnlLfs?hvg*|Sn%l>s$dMy;Ipq<-M7KL!5B!L7F!bZ_ z>qB9eSJO3yTV1DSc`Li~mhesJX%lrwAy3t+J@{<;*k~#36!`;?Bc$c2ZVdalG~{tA zF9C;^CFJG#n>kD*V?~W*EaRjK>`CR1VplNnmZDd}SuK!;gh|C0zfQ^38qPEt$s0*4 z(pejV$J5*K2amBR*))}3=pKSE(5LI;<6~EVo=Ztp_wS`Hl28tIzn8CJ{+!b3TvQGw zLJ}IyYy=b5*{N`A3k3yHW0|Nc3h=5h^f~gN9>r>@W3?d;4 zGZ)q{TbjZLCR@Q;7%B#wD$!L#zeTcfF-=>t`UzUY&Ku$ye6kafevd&B8-u&)BfA|^ zKljSMd?3F-z<`YqVk1(iv$F0BG$4r$>v9u=s)OE)4IScQqlDd+vj-X-l2b9A+iIB* zqK(-{%7Qk%VD}Qy`S+Y{ObHdROGoXMV4+2M-4mLyl2)YMBYM4Y&ndm)dO$gxk)O~==s1#o}yUSfvzfWSvkvR#1}30r?4s?)pWoPAz3{l*YD+|NDBL4 zDM!)fcAVlwGZ}Ix!HS-Uwojbqozi}`IW0RQ_Y1Ii6sabG^k}q7a-Eo8Dm7X;61#$E zdNKO*Fr9seWY{J@cJfN$rE2&W!#8)ZH{mC%R3)v;6S)t4q=P!}_|o20yH%Mfbvc@Z z#iW8im6xI>uGDe%!Z?-~%FZTHncE;%^nR9nm+Km9SUckw2QPa6CELc}-8F9@j~c$L zqAWdS9`Ms=@W?IJa}U3^?e()425k4B@}>R4hnEcRt{5*?imvKj4#^7S**n@V18%=> zrs?aC_>@!=xVSjbgiX=g`1eGUSi?dVdZfr z!FtV0L_xIK{#c^$^jLbr;AVk&LrrGc->h1pl z%H@=`C{AK!1&yXj74B1@e$pZ<7%r!)jRu$<&tG!A1OMwO>G${(aQy%hATa-b&A>lS zNzKvXry=<7W6Dv7a{p-tzRBQRn>yl2z`>0fhSNi31_|j10`P;?1QFx!=mQ6393ik+ zyBwU*1e%qjYF1V)t!xyuf>aBhL3mN&qs)Ko4mK;Bk}WH}-mR)Nn$0TjbK6~YSDg`_B6H{o??tUHJrjaNFuaw4e~k%PQ_g_;B;$SfT8>bKYo9> z>g6F{imf1yQ&7LU{`FBL5#cpR+C(|0K?5JDsERNdv^LC&1LbJMaTTYZe0Z`QSBeJO z<((QRXxZv4yes%|B}XS>czZ+YG~hR48x$$6BjC`qvI8ZenA0j326Cn5Y5_|LE;DoT zM0p>A6L4xFN8fJHy6v$>vluSN!_3eodLGbFY!r0UFl04jepqFvp<#O+a(MmnZ+Usmj0yu( zOCxuxaEUr&CI}ysE9;xYJBPPW``?5TfEHG@p`6oE8<}{pVS|8Wq(#_jYNe>)#x=ER zRA>`HRcT>*bY!Sg5uh9k#AKN#!a>>Ul=7rCHh)|$1$vf#vj&#ZEgJzxwBSA^#3uOT zn(lXMcS{RFr9LVr^GPHDqNJ+j*@tfjmqdrm`0YUp)oep2H@uHZQtN*vvwri;@Puh z0KbJNYl#_6$3}tU+>TyG#5}@Y?n$qM{+VU=`mFS>W+6Ovh3f%T`(i*{qwyahwV^d* z8izaZ%xBx7h$@XxN~z4~UEM^9+UyuR^LA0nj&fr;b_bd$aTACs+mF!`gC3m)5IwyQ!WiYPj z8AJ&R@#|#PY3mlPhL>Xj`vPbb@aa z?8H5*Y45y*G`L7aO?h*0X%dNsFXfaKQ7ToXx)N37qV z9`@^_1S>^{=#bt$n_$zz`l2EHWC6vzeK>4yEHBuuxCL)^d~9FIfz4-*g4<+^Nd~P} zMxu+omnG5k=}mDQCvL#`*XG)^a}|)ugJpN!fgU4%nIv52d4E`gm*lkmxi?2z9Ne89 z%$V^gY~kN17?!7q(7)pZ8=I<{uqJL(wVLyF6{At=l^U^fUwxlwD%>gDgB*+kLSXWHL zo%r{e^a8X<^A|w^MAe$P(>vNBkfRMm#*ZkGD`LI*cNXY*Z<<4M+>%ZlRIk^6Zi1$~ z;59FKp*`S1yf{)r3O=z5SJ4Q-j1>HrML3bZGXBw`9AYz5w?d>)N8d>oM#in#_; zOoinM57t__I3w5;b9(_4$Y*RCVDUO#P((aj@n!+%HBZvvh6jJF4=T8uA78Urd9~N{ zYpl@ibLb*Rp)XU{Kj=RUE>XlR zO->^evxpwf1#bx3g-ARx8TXBn>-tqjtFXuD|Lk3oIO(Cs1%GV_J&E%j_$gXBx}A@Jl<8_4BZLixs=op zNt`Jd=8fG%7Lwo8J-17}t5ZO}B+z!**Nj~&9s-c){s_gYhXN#_JU1+opAm>#JU84V zyoJLjxNRS**@raV%8YYOczh~DP24wzET6`c_8DV6He{pjp@p}tPa0)uBYwez;>dmL z9^NG`XM{+jH(C%arSz4bvgLjVXF8x)NXgb=3H9R%4&*zY^8CQ*e%wUFf9=3~AO z!*3Hn+Q-=&)uD?P_Qr^ffO8XwlIH9syuRTay#3WwaEOx`StvejG7he$r%|^xn^Tw_ zFI-5Bh@OKQ#?Z)|9j8?NzRSq+c3aw8oH8%aH>K%oeH$?_54+*ksv#l?4^qI(phzaNP*LzrBPobCowhlf!5n$ zZ2_-@{>3cnhgwu;`ot3Li%L0SEsg9%FSA^rIz7bAfG0HqSd9#|F!|=X@T*TfJ6KQ+ zJA`)eh9qQA6Dve=*L7EcoZ4jh?{Si_0nL_SHDFJbBJ)-)e@@1{yBw}3A35w%5Hd6D zJ`qQ{zc8#1wE;FbWR>6wqaJ=Le54cj2%}&_pnNqkauv7;Re`z;IW2#VdX#4aEKOmI zeDxrwfqsvA-tf~bJ9N%;xf}|#)git6#QWTpq{%aUg-f=krL4H08(bu!wAC5B&8rw7#5ag-M<7FRxf;FVbrf z&}OuTYZkH|2;;G<98{EAW-l%C+rxmXh=Iz}4|lSrD;Y&banj8r5<9u=O?(d8R+63E zoYoR*RpvL1{z%dqNY&{T!{7PcfB%Wk_K&>^>Z|?>%%8MSe3)OqNd9Ak__I{;-*ImL z1npGSQd1H`{w9S84lW`DPmOHzB!VC)%uksoHylA8R)t5obHoRW4o1q%mcRbe^;+1) zbHCz(bFP7GhSPaJ*8Eb?`7)io4-T%sGBlZ;&gJ>o`q=WCyPNCt`+~^_>U%#xSkKCg z+L4iw9f4HkM4Hm-MCmG@doejXea^Nuaq@iyzPP!7@}w@xVrq~|a*Lsuin9#Bk*u6W zc?PLmdUF!>mE@0x0B-6?PT=zL>u;x%=To=+8SkOX&ZT#%jAH=qCICoQdM%o6N`f=D z*mBdOVxru&DJsl(`)nj51fxA;w&n+rY6=wrfHPZy|XD}gguPsx1rR^Co1wR)hCyng>Y}gDZ#M6n#eIS*&kA+ zwV54elO%QrR^Zz-m%7#*<%v3I*S4VK(- zj0c8g$EARrJ5NQR)0496v|o%v0@RXvVeXDpB2LSVE70Dte!eA4je{g^&JLq-ARB6jlOm#SnX{71^bis~;z}?-D+3K_Z9q(!Crjs|;_Bs7 z6-?Qp^T!A=s*Ry+_jOw^aTKqB7Mned%Gw0BQoD>`RnS^c#Wtw%#N1gVZte_Z&hx-9vRayfM^RFLQf3m@6SBl8-ARcZ{xI zEKLW;(wzGT82J%;T2^gUB??nSPJ?yM2HCA8HJjs*e_HO@9?d60>QqVLYJ;j>2&^2? z#g=_(G&!0GSi}s4B=h5eoO8}fLi#ZW5dd*$!XRpbAlm5x2CB+%DN1>1y}R3hHyb%<6VstG0|5jb)|y;@-Goqx@kr1i0E_0jb&uf zp576+BJpz~w8nMHGS5daK{f`+W-FW&7oeG0l#RG_4$PD<-#R^iC<8P;YNO3bp^4x~ zHDz9&x0e0Z8&=;!dV&+VOrk`Lv=XKJ5v!H;W7!%8k2>l^Jd3mUN_3~Tn|%h|6GH?K^IU_< zC-M-`5WSxQ`XK;w5O^#LT_O4hv#%psdfzZK&dh5NGiBNtEvEsYyU7_aNB+35NZ=hy zhi;ZoVv$B-Q8d?5K#?k&My7amP(fKcV;{`1-4IC^HfS?jy@loznqN55HEfY$CPCvZ zwNSR+H+~w!JMe9a7pB4;Kj8_A>{@fUMPk<=ig+crK`IcPPyvy; zDI@Bbbsq6XFvt*)BmeCeWBxfJ<>}&(VM@_*P*UL+*ngMAv_QT$BwAtxZJj9BuyZ+( zOHoiHqe9|@{#bp>m`pgoHe~;7m~b!i8t6sx(cE^SXix4pQ*YXNHn zJi*yn*wNao+s4^hC&A|gq2x@2xFBonQROzXYE~fH*FcjRZnOc^w9%>auOi|w|Hv;G z3`XM(NGfUw-bul_Y+IVF_UFsQu0QNi=H+}6hzm-A{Bg`qx^#bW*A{JR{LU4()>D@{Ek;U}#m zj8Yk$QP9TbINPy$G=p;%A6!I2GUF9-Rh~G7$PjH=6m4+5)4kO8J~==$V2RN+n~nvg zw}s^sOOkBN64XgGyNY1&H7+AV4%hEJ=_`z{<8ppauR~9)r$4cf?)tpBg$;Ac*ay6* z5L-sQ>c~1EDU#4}MHu*OUeC&29$$WNuQP`iJlm`g;|$83qKzjtW=nb|#N&KWw}#}7 zE(4K{R=6rQ_03}=Yhu%gboGGnJt_GW1p7q2@!ZkUw8BoysAe%QO2!-G8qkw9<4o$( z?4i`@Y_@Y`iPu@bC>F-)*DRa4ArkA7vZU+?5|lnAu*e_vbA!Xi;$VYR5mvY?Gjh{0 zFeb*)Zv_q5$5$bVoJYoAg-!ItiQc|7S28mzQyz~m8g1)gBONwJp;>6_>MtFC61Y;9 z8UkXCxh=`ib*{}O#LRYRTM_K!tl14roGd~cyb3nYp3i_+jR~|Z48dF>(Hp9BQh9{` z#C9HCIiH?sO*x?S79LstDIcjKbv?o~`7Y;5uo zGHq=F)~{9-LLvS_5cf;KO|#_ywMF%per%7xwO(_m>aTzrSv@nn7vQ=c;#C&k*pl%E zbAx}5&Pg0A?~femA+HmjLc|Q{lHQK-8ZRPnG2V}uB& z%yI@sOBG3A%}6-T{ll(7GR_s3BDn#v8Xjh&my`e!a6T9L`f9Vi%k1dndTX=I&CaRQe^{xXCw`iw5mYxphXyTf%4(~MRCfs4iLy(6 zdvYAeR#$d~xaSjNcmCc9*BmNU`cv>ZQ(aU?!uypZ_S{`c|Lrhqr-WxmXHkXnOLEl8 zG|6!?Lcr50Pxxq!ba%wA-DFdCqy|7Ma7%&kZ;kBZMe10!pThv;t7l@zlYb9Iscpey zZAy$Hc2&BcM-2(VL1>s2cPdVo96Y@$|?19MIP*0jeJ&qU^ z%y-NS9fZFcPDXCSdS}sbgwh+KsA}^o^f9B1Parx6?-bb|gtXwD7a>wvf?LF^QfMqE zhD8s3k>#pO3n-t}!r(03*ENHlxw>fw(AHxa>my#*sJI_1 z>qc-gMf3QhcmrSrd@#F!HQG8yide9az`=M9{1GxjRYED&5rZU#H!Fi~{1;w{yB{=m zcx=P{5pp@$KfF2y())#)lvLsjT2`+z`287kM=8bDl~#Ce5CGzkzs=@-+J9}3rVNO3 zZVKbimNr>IEOV5Pc(nWXw;Xs|pTf97oxQi49uTg#r~9`MO=X?JV8=%w;B@DS_Ysh3Fn!_z24dC*^MAPRu9#in&sIQNf z5%r~5>3S5<@%#1^4oGclm9i3YsPQ=gdv!INqWyfS!?hC{d~-2lxkdO_=^i33jyGYk zAVautzJS`i{#GF=(ws2Ep{XtTBJ>~(2`BP+7*7zw#Y^}sC6p84MO<@85PnEtSJXn0 zcvZ@WpJ+q&5Nsw)kO|of#N`By`&&BnJSL9|skSP8fQq`pi#lmTbK`$WkYTer-WGLi z$)2k%AP2SK;y91PJovn#)ZR1#jyY!f_k52SgH%G)z{kkpY9H>RY#Kq>e_@bCr*{u*2`fGK)y zQvvEu%-cBO_JpII2=@E@l0G+ZXrC`ZQlGum3P1Rv;fWpFnYi`BlpVNchj6K%yEQ7_ zn2O#Gh;B;(-x{^vnCZSS)xk05fu%QvIUwO0fa0EP8DZlLUv3}SaEH8jboxb`-I;Mm z^&Q4fUd0DBP9iOfpGuS`D1kT>djv86#LIwOeIfQwVkgoIn`^qoUms+Xj1ZMDw^q8e6@JPvC zdc!3%tWSSU?tAiRgUrw9#0I;tG}+Q<=i(Sc;$ z(*X{ww^LB$XTCV~nYB+E!c?PYTW66^*BirUkaSoz^ui~1$FSvi5nVM{47eM|@wflcuxBooZb~m6CwlOwT7Z9?u zwf#W_|1TKWM^VP^2Mp|!Rm|X`-6mt7B|oSJMeHzes2z+9;>nK+7%6xV_}NfcYsfVW z3)vS=B`%JD?{M57Cpc}pJ!(m4?9tQpcsk?7^!@lUM)wP@zAnGN{wQp<;?{F@xB)k; z*K7NFL}9pxIh+WDB(#UApFS20@gbXaJ8x*WBO;J-Xtm;TAwzv4EFi_qfSeDy5Z`B5 z-fjbsh`^~|pleS4VuV*sL0BXxPzt5mKqp{UUMsP{04NLE3PP^yi1}v30KIs4!E@_| zs++e)ihjHH-sGCLv} zRSdKbXTA7%Zo#E4!M@zr#c&Iq5+tzz-2UI`O68|^zfJGCngMx>p5CjZH~3*x0Ch;$ zQIaiI&Rvrht&o|_cM8>WT=R2?i*8x*-CUT%LljP7M`{;!B4kxGKI4bMyT6fU*mCUJ zc}HlrJ_XsDEGwgL1YUERE-|`9%XA_c8^ylV_5Ct6mbjF*TbQ&sTyrXcNIf>lYQ3Z= ztKsFimi7`nPXCP|%nl-#G4)eO*gq=Re=H<&dUq7A+HxrbN1yn@V z)$Hta$62n|*>qkXpC6EVzbCj3pixL;uip5{vyhNn3RFkVO7a1QmTXHRn-DnoboMV; z?6P8ft5};Xk2%LH8CsV8TSr@Pfq(*^qaug$E<8z3rY!zt7OGT*$xM{Afx0CMfO|!J zLBTa9YdWjsN>jK`F>v5C>aB-VkWQofYJUA0igW{KqDlcF;e0-$jZ@KVUI?|HMEHG;%l`sAFJ94rm@TZ3`FhS{v z>ai_gEp^!ys;t+kp-sb#E0^oW<^ z1+wgO93-y5;O5D>4v}`0sbp9j-rn4uC%?0l&)%>qH@h&Mr)+m3unFQHgC1nKb{js<5Ik!b^PsOJU+3MevWaXov^oB3bOIl>wQbH4pQx-9b|| zC*BUAr8-~gSFxNG!%8p0?09><#u5hW#`<)V>VP|u9*69|0IH#KK*sV!V7~WJjEndV zVGHp4HvxQG@Pen=YwWVVTZe}me1oxLDJa9KNa81odykRtMc}1#S1N(7oC+i96i3)r z#@x`H17a6NoM&L#+I|XJyS0W6D@2WZs5a-TBk_Kxt9;*+R!A#&XobTzKsBW+33?pb za6FgVQa4NLov%Y==Es<3P$_ECGw2L`t=8Vqd#X9Wyp?zE<9oe|OXM@n8sJnBY)p2_ zt-hGm^O{r}m-x-ncsyH4P3Uo^c9+Um{F_}97@x(N0@RXYtHjrSF|9Jy&1Ia zX2z<7muN(0G;Y4j(@~zj{uc%&!g!&;6r#sB!G#&Lv7PO}Ho$i|;OC(Y@78ub`86|J zyt3$Nvu;vwRgikjnLw5%-2|@J?{2&oND#MGq-KZe5<`g7?N4iZm8Yh+jRMl;LK1({ zY89f&FUz+TRn}V))KgRLJqI&K?W&PXIVRmSg-4$Wbv5n;8t~4V54`dC{DH?W<{^{F z(g0cj%zJ!RR{bFTojxWeU4og8iX{u&f%{*q22J^^FYVrCu?_|4Gaxo~^nLqfJ$Hh< zNspn>+{Ega71bYTJH9toa$$-|5eYw#E-j!|NX{EJ4u4*2H0FnW=hMKNDIBquiT>G2 z9U>#K>F@xtN^Da@ZetAa2kO4lAsMrmniS~d36YPfhDgTSiq3_2mW&-%t_nI*kw$ zGQm~mKF!5b7>e%nCAQ*L`KJ|=s!9$4rAp=QA`O>EO{!XrzD)laYw(szgM1_^c{!VX zEg0zU_FuUBO#D*ew=Bz}I67$c4@lK-`OMWRbESz!C}q{6SQa_B;x=^8g`&>-J{AnE zI6c}kZk`++8;ce$y;WMPEWDeyyFDLV&uSMm7b+KcPcHI*l@%2K7JhL)vp$Pm>@|ax zyS@5G@RbswRk$ouDgP11C|y-DQYIzmQ4ZpikAypvXIeH=j>{~tjq zxZt>uxbV1uT2|u~cAH9#TQKUOsflzmXGP}IHr!U-u6sip0m`%^->rGeJQSEwL%v<` zR=7vXoTJ*UddogEm@z}SUGo;b2bd{YAIwBU5o10>jcLgvZ@iX20^_5Ryu*b%+sU3(&CIfO9QC1$+)sv5$(!*$|&Sg zmLJ0Wgu4v!rZj-T($d(XGH@Dw&;>B{Q2Chd8|gFh8!~Y3;<%*z9YpKL`NfZ zj6}+gxQs`pp7Q*UZrR5uL=oW!p`n&=lnN@ax@Q4UCXTIBL5}Sw)dN8|ci_sgSq?^y z9H@+mO`hbEle_oA>T$WtNu}CmY}e;arjq?^x{ET#innx@X~rk8T2=v(B1AliO}i(z zSGdUf0W#Y9>f=kaZJ8Q*WTnOV{o53?frf0g@ruXt&0^g4l)?vaEAmIH5bm( z5+OVY?T7ik@s9%<7F}aqqC4)|`JYV#HCO3Ll;4v{4oB~sJR=>D=kDW8)~0cEy+jgzk^TDm@(yA}ba&s7I7GO-km zMV~_iv?}svNvu0;%0N0hLuu~(9xdZR2UE%EJ`!f6ly6F~np9yrFUHWrjz3{M8F26p zO&onQ;gS4q`Mw`4RJ&tJcQjzFBQe%De~-*%F5i6T&uzBVSiH9-A==lJ#FP^D^uHW6 z{y%KJV{k4(+qN6872L6H+qP}nwrx9Ev2EM7ZQD*(w7xw1d8_vR-r7}DHPhWQHS?pV ztNS{StIq;eC2otNL26GHrDnn)))j}6rW)v3+oCDQjcABC;-vRs2c|!+M!3rv@$*9@ zzC$D$a6IaF`UqxBWuAM_r zl*8JQWIW;lfA=NHknJ!ks_H%Z9rr=FR# zChd~YQdu>R0}OwMNj9Qm$AclxlV^fgHnOU8VH%SiKp#lXe>g}CCrDn7H;qV#sg_XA z`mCtS#u{f9iqV$PsJ})-O)@AcR?WmM`;G&uh3AaF24?8+V8;BsX_v-E1!GXw#_do@ zD3@Q2v}O7s(ixirxb7gDHp?KxH_M=vaXMn&Gr8;b5qvLy%Mc8x!MmabTqp!VytQ49 z1LYyeOe9NQ%>?Eaw9w#nH4?YcLGK9PZG+jV?zv$2Q@3TY2(>^GbPB&@$9=L&v@h4>a^i3ZXQ3jLIyQE-0Y2Pc$X|;dH&HI^ zI8sHQS!)WmHW}Ojur_s%Jjz=lALSK(y8?ST#9nyYFEb^r2w9z>558VOUpreeF1>i! z!ih$3?+wu}Oz{iqT(4Va#WKGzMtF040v*4W(DhfjrxppmnLQQjaf#&`>eOEkAUo8` z) z3ajM`d-&U?L1D1OI??2TShQvBiSrb5w5347(bZC82T2EGhs#5a$>q3h^q^bp&klwm z&tQ_7M~M=p@HavZ4gIC`a4U&Z>ToN~WGl&}BI+@DtgS(M=Ed_Wv?7lx^K#sLNC^FE zI}r{mB`K#LXnkaM_A9~4eK}-$*=pyGlHNvlY$<7H5!?!c2ZqbGhMT*M&h|9;(Dx!tdiV?jJ0HBkGxT{D42x4Q%D-cYPW_t zFwc+-6z|u(cIE&XUORU{r*~!9iwDtP@ZybfwkOtTUoM#%jgZEEUm>q82Y`wW;6+pH zmUF5d4X0pbUf=I^8jkO`xuh!>B;@1%ht6gAH)e!H?CGtMN5h9`Z(+ZKUCjrChFha6 zZPH(I)n)Qkt0v8w<1z|5+B+MdnmSBhe!uVdm@l!X+03bbl}m4sJa03ga!?n4N54z<>6 z>TAuRR8g$3WTq}%S#D^|vHS?vay=fZ!bC_Q zLfA@$!>HedRu1WteXt=MC7&+g@J<@SAREhw9KHh)d6^?bo5D$(B2BMQs?CwS_#D&p zeBCpS;Y1p6;4WZB9aOz%i9nbheK7-Vn|E9KZqjY>*dNiuv2 zjF90mCs5BD_x#=4HI6<-Wjqwk*m*qsnBh6~eqGJAO9g6?Pe3pfcYW8aEyUUgxAOXm z^|f>{7Ke$z^_ykMac?1S9#Fa-{T5 z)2mgrT3u_F55sTbVVrQyd*c&L%ct=uKSz$>Uw7azh0@bI5hk7&bC zY@v-YDnva)HWldOGVE7Vh7*m{t<_;a4%gLY>Ly*bNL;XJ2W`^aXb|R*lKk7kq%Xo^ zEB2Xyzs>j#xP11=L!B1zG$D60@8oRF{Fee>r%VkAoD{tJn_(s-ekI++lXx(K>j~zu z50U4OMUYHBxT?h&{TIT!prOD5D{%{?HFuqqJh@yg{E@9f*P9-7LT;bULsBw=0+id> zhd7&*aqBX)@zNH$)N!tODjgCRUfmr|Ia|ao+Sr%;q|p&ou(B=w9|kDBKw7vn?b3FB z9^#_$$+>h9T#hOOj7;@@BtxE3^BDDuA|~7e1bBBx`o_B<*Ze-DkBIBldV$h34I(Oa zglDQ0UY+Jxi1C;`OB2B~-D!DY3sHyf$L52FDMJ@ihkeicm8pYK;L1<5|2vNPI-Sv3 z{R3YK{KTFR|J%i!9IWXC6cq(D{@b~#)gaxKmr=gy(#MGhi3p%I@CgiZrCBxp5DN<; z(i0F1g8VKsV0`rN*BF?>g14q>)@+VyrrD@KnbWMVZqB2ZFE6VeuBdKSwX$k%NwI8> zQtgy)Za&?5H9avgH8iy7p|iVjyy5uwceed>=i|-qh7VZ3#k}4i+W+8wHLUgTg_dfy zrG{CM2>vSi`6fqMOFMHaARUB5OJ`H3L!bO;gChR=)-6QP#&V5DegQ>kz=<360oJl# z-bg!o>GzS{nWUl&h^wEUy?$jrbFWm!eLG0zr58j?g>4Km6iL%J3Juzm+f#GxnccO< z_BI(lC)h|c1gOC@%G^quHv$W&yAP4a(d}wr#^fqyG#E2*X74W6U0G^kQ}6Dy(P{g@ zj|CNm6jR0p+tHP0?P{k~Oupn)aC7{;!jW_>>cx5!1vR}XX#dy)L|4X*QD~uMjDIo9 znL;B0y@mjRW1C93e&9|aMruPSUtheRvkHD}u_ZW=Ysz!HM-K1=u7#1sUuo2qy@C=Z zqD9F6xp*)FC|B5T(+T#)q9tWCv?t71RogweEqbxEq+I)5Kppp~egIVo8G;ZxJ_yxKz%C#kLb2lf8E}_~_YJH-EV*3m}SRuC1#*3w^Z8)gPqu>mF z#7A?xeN*zBiTt)lwjM7j!K*Mz!J}0i8k7*PrGp6R?rmv=8VDw6vU9oR(AiO*Bkv}! zg?RtY?P?7w!Y!4AFIx!ciGKYQX7q2m#=cEn9#idj~EiyENa!OzIWyC2^~h`#4zW zx>*y+W18(_66%mmVXfUfcy4i6q3$FEP zJ$?e;Y^~bkw6?kzoCLy^glM(>?)mO*P*$$>2^C;x3XdpAy z3dTK>uJ7XPdPdMG(vQ2vPh{&uy1FKGrP(?uC(<^TK#JEuG4g@36?)uoX9XdIonA}X z2I^&z#!M6Q=1gKFW68k>OWqds7&OXt2ET{~?Jd+_y=FDK-8LF(m+smX0lbpGx5+Nf zGB=p=5!k)y%`GY5=3FE(1IyzPtq$ihhMygSr)m$T08$LjWw8UFdxP%z>FivjK}$bv zu+c}Q9Ta0x+~C!n2}(`O$QIM|uDx%c)Yx9nk!?!5K8oUvmIO9ote}hY zTMXh?5P{4yO!X{=?<#(7pU^y_i;$fDi3(rH>^Gi)BQxh>!mzfVUVo|sQmb!YRW5Q` z!jRJ`z3?o4abdovNaR*JymgT!eu^-v*m5Xr=szNGKGQc2&3Ia7%Q13c*5n0Nvwwek zSYj)eP@zICTxtP)hnMPh(y^FOxkLSZrb+6hfL{IjnKbp%8ryQZGsa3S%fTq>R%TMM z80?y~{@)_=#A~L5lhkXdNY;zj3knGL7Tw;n)REWlv7+8x^Py_s6EXPrvP-khl>QDF zT)O>M-m+b?qYjGv?Tpjtrl~M2L;YNhb0n8k62=E}6cC>fpuS0w4w4XzM;e1CS`7&~ zOfzl9F)1+1pa#5n0b!TXy zOB|hJ@afn~662w0&UJ01}Q;SPjiwD)(;V){yR?kpKM z4)y7xEo{079)!7Rd8M+MsCrD}3|oZ0z*ndtk!kpmM~#bU*mK>qm>oq@c&VzA$O@FC z3;UQSf(-DT8pV2j>zT)_> z%@}hjpJ|1va^U{1VM-h;i%26aj)zj!l-~?mBpBF>2_`cs8wVbT_eR6)ONQPnGGbSM zuNA2Ht{RW(2^BxWco9-gcGfzP?f`cDS-v-m{4MPiNtD)~tFh(cSosnktRT0W4u)Jt za_0ipmIIU{0&kj>(4H3UkroPWzh{jJ&OsP6cZL0qLd-A=WY0w{3_N#)d_aD}^*D;x z&tY8?z$f!*AuH;P9ksfQJ)B5n?xu>@-vCy^NeD0(Ce4>08|O|&0-!8q1V~O5*r8;Z z6BFWRsgC~Ry_?uD2Uzh&(H0EATe zG4V)>W&wH|B>3ojO9KD-d8=K^t}W_|WZI zOynA}+wB-9;#qBUW%Xi5&lP5ML6Yn_B|A@9(%znd1V4NcE_U28QU}dMgbmZ4H^qQt zr6Bc9JtoFE;65d z5jhURdKB$WV4`hXa*(#Q zhcE%kx3?HSjgBcU5)gJz|03e>(ce}t&4I#TvtNHt^T|R;siBdiAoK8?)s}=fh5^oT zH{F@!e8YG?6?e%U?4{(ai@&LCo0gP>c-nx}IzS1_`D`q;7J1ME=E}pKs-&?Y1KcrS zY*#YAqmU@M#zm^;C$@2&AOKo5YTv8hjGl3E(vv2VVGGuGbX++H&-?`NcG=SGE}B-+`ynBBMvi-125=fS z+@{gHBxWyFGX}nX!${ff`Y&RS&ImpyoLG(&ywh8XPiNsfREx87zUJNR z;c6jpZh_LTf4?Piuvy+x?=lhk%_M|+Wgz|uYu?fAyZuiyvIH-WDynp!&f+hI5zJ-Pi^iZL%h%7b$Xv9IkHNqSML z#9eYkQCHh|7rTT9Y2tUrNMoxgZB?zp?ejk=1D{M=`SG0=tY7xDdSI#drP*}uJ<_yTcHXH+LjW>X#V0OU^wg46N;_LQ#Z)dT%{3>b+OA5H0*g)}o+HW@hych>ZA(78LCT&FxSWXSru_rpp(z;)OA|(uQC15am9HYsT+(tHc{1Q*l z>Ow*5s;XlnonRA(sjCJ<3i|KtVS9T;03Pac4w*5G45?jNah&x)jl+UXi))he!7hdf z)s`b7hV|vXFzC8w2Hoa_kOHgEPU4Z>s0Z373e?|p|M%j~?P~9Z?MG2f{81{f|38a6 zM@It>VG}1KM+wqGR{_ghd!((;s%G0U1eH>bgKS57kT#`iN zw7Nv|=+$lMp!jlBKcF+~eD!P?5J1E5M9bcCdfB2kUrl-|aAyExX*9{Vc1TD(C|(Y3 zR80aUe}rBDC$sroA4><_4J+4dw8>#@n^fW>@^N*}_*XeKm2akw9XOF`sK9kGhGD)>! zGkOf&A?6I@;-9&p{4-5N?h*hd-{00;9$TsCFtDGUM16>=xopGS86qC^?>*$)ZONBG zRYF0LC_*U;Z}QV&_F0@j#guB&E3FQNDH#*W{_q%Z6a)-@8JkT=5@4=wozikts>t26 zR_MwO!c27ppBr>DLG8g^Ig;cgRZ zCV4{dfl8;YC|1LpujPd?)ezLbIW1k z?t_N@BugWQ8#E{_&qON2;L~xM?q>^o0$)5rukXXS@$+hLe)(OAL+{ zn0&lAuO@vz{+jd*@~o{j-tcZ;FLQLGanFiKG@9F7IuBWBCy9RJB30ngyx;~oyQkQc zp|VeBk4j2IdYWBrzgDWj!HNp}jq`&&n~8|X3T#Vl)qHR@2t*0WtjdQFELW|dP%)b0g6lw+s$3b{ zKV#9Zeks*socw#uA% ziYfel%;Ifyz!DYM2W5$U%+9$B^7#|w^az=v9>@Tr>XF|t<#rXKe!W*Dqepfh^-3LW zQco~cHQ^tWj`M=u9{NAdp(#lCkmqlz&|}2uJ1M|nppEqUc)TPtWe;yKSTv^^9uS_w zo_ry+g^FnjwoUx`$DsyH%sq>Yv5<#oZ(ZBIFUdaCP5+>@vafr;>!-hM#XdC0rnIBd z&9Mk7l6mo`+LeTbg$N7w62sYB6!;c7otn_zFBsNDIh}&DU%GI!A1Wp&1#ec$A_)t1 z%KuIOFPO<>C@t_33kZl$7YK;x{~d7tL&OREKZDK>%k-m+x>!3anK=JXcuC7cUwQeN z_jQI?h?EYjp)Ld`!T7f@ya+%GHDw4axsK?-_{ZWxf^flRPtlOeh4da$))LS)^O+oqkU5? zH1N_gPzUF1;sX7+rS{1su)Y%hT0eQGkBPBQX|4jUe;^KvPJVpvV5=^x)^XgMDjWEWsLe|5XuBWYa3>bJT1k6Ws#uA#j*m_ zbBVWuOOv3_Ql(VJ6@8Evq#zlI*Jblrebw<#BWR++a`iG#LMK*e~_a8M_ zcQ9aLYi^DFb7t}|OpBbT6UK|u!H+@S-*%dpm5{Dgif722D4s&TZzl3VP4!fW@eVjw zP=7y{b#Q2Gz9)2-f<1fd%(x4fv0@lZnfBu&Uh>lbm%B7y<2 zszc%yLf)R`A%9H9MQkeyjt5Rl`(spf_%o?|u)+RlW4o*Set`p=#SX>Mt5m3>N zmalujE_8ufp2ue+UGof?DRG*E7RuA~RJQQRHkiZe&c5qicL#n@3Q>heoz5kx3a58KjRD zYpfZLhU*1x z5UK73UFgFPR?Ezr&~~6IAdoFvZaV{XYUcn;5hk@7G)bn=PIk)XNoDXsD&4#XYWWp9 z&P;-1Kp~WD+rpd39e-?vVAu-z7h}uHQRUspn?#nL5r2?KdSr$A4DBbW=(K*cf*BdN z#aiT!)SIBhI`W7@7RaV+sBH?$#sJmTe1qlovjDJ&visI|A=?d+Uez%S8GuZ~)zp4aON+ZOfq9B#jTCfdX;9yQ4LWlS(ofV?u zUjqbh=lDY6=9Km1hH5IYXj|24uO1ja5c~#EfeMlSNoqnO%3FPrW$|VEeN)hFd zBf0-mJ#%`GgOqda+giAb4{SCPeU=0HgTnv4lD2-10`koj(o=A|FTm8bfie#jTTw>` z-xqlBN*I8Z*9aDv2FTVyXcn!qrH_DRB9*SU4ax)E8Hn|V`l$$-!kT^gC$?3SkcJ@zG8&{_ zT_?Lhq}g17dL^%#LX$ZH6?CY>F|`eoST!V@Vq+x=RcOd~H@9Gzdfu=tZ7r_UT!Bh6 z8qw(qE_y_>Nu1?N7)X7OCKYisT;twNv{2EE#WOzY1(yMwx5*KO4*zc~LiI`OX3V^$ z2FVZ^1dR#(^bXpq-)=vC?tJ&H#HRHgyw=|N>_zI#R!i7QBL(=@Phwq`6O2pZWv99I z?~*PEYK;v=RpN3hSBP@e`SCnT1}ZR7+5vTz&1^@{BA@wd#B|KE4}X2_qV!DZs$>ls zhX`u(n9DVZb+!Q^u>Lh4 zR4Vf-`>A}}K@6(7x!_f!95Z94ox1KvG%A-E%`!hJ8bGiGgdvu(M*;(vbE@^JbgrI) zak>#mDXw4`CUMNkW$_!38jS6T>r{!`t3(VQ%aydN71fi*LbA+}Q`(zaWk}9p{;om< z>X2VW@z|xXpRA0;4RM4XI_$0-d6KM%@nBL_L-HN763ZvZJ;GY$7(D_LnHPp6Ij;hl8nkUJu9QVw57m`D)Kcz(qLw8Ntk(c=^uRucu@8512 zlG2x7BC)ckol>|mW=t(*j2`8T&QdOD(9KO;9-`?Pyr6*yk{(yEGL(p?ygX5%c_o}7 z5MdGRVjRJXseBc^O1Hv#K7@)Exj#u;t)**2uRTo$Wt9CKqLky7#0XKHl)+5woLE^} zo|J-mG|FX>-Vfv?=YoT>9aYu()k7&dFjMO_+Eh2;Kp|VdYu5UgOOuG*0(EE$XLn~_ zibh(~FA|`-W$5)BzaTv5+UDte|8@x=~CT;UT6;h34 zV7*+VY2(;CFk^bE&Ss&Qd8NE2x9+hbvy@41-8wL1hNjL|u9kVFur|9+SCu*SbA!@a zh;@)ll4kP{z~ai#Im9U^&H^pmyGwD!*pe`xmi&ci7JT;jV|#0+7CtXwTd(c3iv-n~ zsAJL{Lo9ZP& zMPx32n-29Y7%5zrb}{luivzV>PF5+ec@XOy%AnYq-YtEh8qczBIIPsYL`AkS)D-yY z@Onqqr+%3RL2Neoqkc`b*{dPR%RC^(@=Yy*Nl!;mwoQ^tNVOF_bJ|Q?LG|1a^5;a_ zP~}H;Ap=*Bn+f2`wL%{^f0*m-f+?OJ~UCztt;)-?cTS36B4o- zh1VZWS&k*`4WXVK-PDD!4v13WO+P`~blfPB9HZjSz?=OOGj^Mdf|}(imgdWxLvg=w zNT%NV8dh9K%qXv#2Htg1%)3c&_|x10N$eyPHh1hQvZ{(d6E!_n=7v{Z@dK2WdekBn zt>jSfy6*$L7XH^xSV$}+R4%#oG+d^OcfVNFDdz$-4?Uf0+euRY@My#~70#pK>7|`} z$eTYTM}j*hN3ogfS(qy?Ak5V_uw#HGYgb=;pYt;T3R%UKE7dRtOf_bp^bzBpN0K@X ze1uy~niB!2ds?b&*_zP}3C0Dg_L%9A46R*~lz88V`~%52+q*m6pZ&|nZswK2*9G=j^?58G$rt(I!Gd1l$e^nfRqU`?%QZ>nfGlRPico?6 z-|8HZp$q2TvH6OWg;1DMql@erk(HL5DW;3WmV&n=}SXfMrOOkYiVx@X$;Q@w>wCEjQ9L`%YwRc={4nB9$NPz zf(o}p0A^y?Xe6#Ic$rggcRxrCzD!yTW>^<{?!&DYFz>uE#iaE8J@2Nl4_u-7Ze>ZH zhF1*!VZKh}umM6*g_{dtPQxTHtO@YUlI;$ZUCr`_@P%w29KFf^^Q#<4?>i{@GC)sk zSEzewY+b0ZHIw~BNE|7vD_-(P(iacAl*N$+)&xbR#SOBwempwjN3R6FoHGJA7fu{dj=g9Y|!LeGG9g*LX;pjio&Mf*Rrh5)|S&li<;;LgD5| zOw!O}JIQw&k#}d9;5}xpW3Nept$^kr4!O+n@ukSE^RNmQvn>enM4@ZBS-P@11+W<5 z9yP*ip2IhtfIMn|IpxA))yd1}@){AvR9wdXLZngSm_oXc^k0c~(-**|RHI0R!K@&n z7B^z@9Dkf`(IgM!l8Z6(h@RTyYdbuXXTS1SSx6+$p{)puyaUe*>-IHQ=ZkPb@MEC$uY*2JHn6{rT|iCuC(wM)kCl6G)P zjN!_N3$Y1eZ>kWiTIcUn#9I|OZ@(yOvf8m(wBy2L8Z5@lHGpf{2Bu_M5ZVlGI8v>Q zZo+6{I@16;Qjl0^76FT=Iv;jH(@E)a}7u< z*MTX&JoAoaJ1)bUwyLe#xg=kUmK=LECUxnrf@D7Oj|I#*fN0q3i2F%Ri3enQMT{Igd!m!l_94 zp>}Z)DM&U<`PIQ={;YkP8K~`ctROVc#b3i)B$3 zOT$@35$Oi-k1${PVxWs&sgx%gKKeN^3y+ z_Ac-&SX*MIew~mMHJH*=mm#2s;HEuIY&rZIHf6M}qmuNCxkYkBGuB9+ZWy1|E~>)@ z^~smjE1Qq1Y_%V*Uq2>D*-3f2o!gBwck)u(-P)0AZ|}C-*iyH1Y~WQMdsl>{$|I-> zAE`5}(EqZ4!s2hX)`w$l6$-B*LU>4^Jt|BVr3zKg&Q8D?EnQ5l^QG^_QC&;G2Q~65mc}OzX(&%lE9cWcq%lW172|zlpu_ue_D{$N|4lziulhu-^es> zBy-A|cA5{=6fcVDY}h*V*dXeHaLbtfjq&!@ct^XK#fj&^Nn|R)ck2mb*!tEny&L!x z56($VaJz2WCXAC04?WwZpRH}n`qnbN8(``Sa>^#C9fzLnw-Xl=n_3q^mhlbqQut!- z<{<`_Nq2#CgciOWYSpB@sPmshyhwV6Bz{0&sF~ICyy+*Gbro;$^&)EX zRJ3Ot;NsM}ipR1Y!=)2&p|-|jqd+xP)@dUTXGGT`l5H*;;~F>N&RLc;RrbqTzHvc- zXD9l-3~&KC#yM3sI%2zfWky!dp4VkIoAa#yu@EP+XQ*p!qA~kETO-Q!hQ{;HxMjalsm=;1;^OkXbxr+mDuw9k{B!aJURCq6TQv@1$=GV~L*(k*)d%fSB_@rn&JGP8 zjMKaeipQg6Nu{$zan2kf`^7Uiq)-@jQznLR;?|<*G&M`nv_~m=bc)MgZ`6owwn@X^;u9I;sL*Pa4!4BdAou zOv6%Ov4j^t6a7pgT{F8@Rk>>51T-Zc$|LHIFq5)+L-LD9;bb$CA&T)I zsw2k@L8()x_UTL-r+Umg*LV!oDb9fPtUeVHU2p=L0-Y}d;3RdRZ_*rbSA6u5sw`2@ zVoj&?LR8kIdSOg~`S++=x@DvPa9=IyHL&SbM0ul$S6^~nIQyKAmfdD%B!RkQ4 zCBrwH{uSGxjg~q2uH`*GQ)(nhul*_}Hg+PK#veYdxoC!Cg-_LqgH{arMi-xT_UOto z8&n2UZ%MWhYy&rq68>w~kr*Z{H@fIfx)6uz33sUm_!>aoN`wCc-|LkI-J#j1 zGsa?U#2x4PM+!LACf=trO*J742A4ex4lY7w{ ztQ+P^rGp!5nInu8+Ek+W|Uj_F(_!}d( z)6r+@@G_D?XRJu`*H^Jq;j&=!5--AxrAQ`r=fZBLs3401%tSGN>or6BG&yM?dp5kS z_Gy{KXpY%UH?ne9>-9(p*eqXjoK%kb6N5W2pl8>mnXo)HSDHI zsQMp0{oK`guFFLtCZe$P<>uJRCpcQG-Z-} zwSK-XiJUX4pppE2rwpBSzv=DU{G=1gJJYkTHRg*0rzdsFdo|$JQ>#a{f+o7xm7jdA0OBrmnH_^V_UivAmC_)=_I+GyW7;}QO-x!uXMj}tU*z?=z`sCYb zhDR@_ShxMH*Rr{L-aqWTc?7XS%p>;%D(t)^6tQw;KZz61jnvcTlV%zR|9IoX!@qa1 zrT*?F#Mj~(21KZK2WQ^m$U@P-%8AeMX?%bp=8NJ+aJell9T|NHSnM0Rtr`zE~h4RV{SfG~te zHaYf9)M-e9`en^qskOSKR6WhTPBDQSEL7+!ry0mCNyxdf54Y%AV z7XDvo_WZ~djDCRD;+GPVzv{Z&zGdTl|IIB;#J^Kn!Rwonv_@#GLCud$L+_L)VH!DK z9&(DtQ>Q6nGoz;2`q?X%M9-X0MY&eYnXpjkg}J%BezuZ)gPL?OF?BOcG4+@u6mdgt@io9v_c`dSSlb~AfQPBAfO+~@INV({~^!+qZLZn{x}V|8dwV}|5qcN z)`WCdS#H*~y7jr`fcym+3}y%t_~SX|42;0c6Hp%O+d5a=A24mCGDQvwEXqnfLk2E7xhZTY@&o=WFLR@ph~G#eL=d zRnw~0WyWK_b2NvrAnDB@3QS4UlKSOQ$4IDsSp)49avIC&Qe#w_JEGF-cnp9hL0-t%wycJQ+JR{t znHjX?95bb5YXe3HK)}GS;L^r)5xI>o)gvYWFr-^DvvuNuj{YX99+4DAZsJlE|ocR^xlXyz>+V1 zElFKeA!Ma#-&!0~JxyNyz7e`2-Zq@QvnCIi&|Sc;h+O(f0=Gm$P2X74?rFeqs$lk) z?rscAb{_GhR?XQ`$B`wKAmd<84kbZY6c2k>_-D+SF5BSj0Tokk&b*i>o2j^gPGM(b zX=xoBTv=LG1I2dU$)(J%-Xf;tWFu6>jaZ?n3*2Mxm(F8MyCG%CP~Y6cxTU>sYgwB( zQX#*+CGS*S;}t{M(1gnPOd(OmuAYnKV6X>CA~t|bx+Wz=Bh_3Lnsi`1#<85gXpX7W znqgm5R=^DsYsGnZ)DkOGa;|JRe~$CCeC=e;C_ompk5Zk_d6$*4jdDT&&0NET!x?HV zGwo_l4_HzT5n88ZS&p+x7`)lDhj88KE;K!n4l|W67p3dZi4^bV zGApV5<+&{x_F#^$j-$3Ik3|vuU`&^#tj%p)v-3>?t1_?7V_hx;sx6wr;Y^KFTFw~; z(QdQ0cevd`8YXLBZW`vO6G{ZztjLmF!?9KHXRYR$6~Ojr4gBh+q8q1oImdz7U1VA2 z;L=9kB-)yDIp?{Zan&Llv~YRyh||H(#%22mV+7h^6eNT;?2?R?G>3zQ(?M)Y05D+G zkxbQc##rP(TWPqMsMuf57yrFuY|^H2IdO-6K8Nx|bO%kG?~Tu_D-_9fU0+J5*_q@M zV^x1Pif$%q0ZyB%a`0*~Q{!-^xd9`#ifm~M*;#?JD6o!banshiBcqQ9Xy(lX%`GqKKrgaAQ-f93DBOV0>`by|S`Ea|8XoEE4eXZz1QSrLOk8j; zvWu5d9M8ZUu!j7S5_?c)9=e1#os6xJ(F>FuSM%N$3$6upmQ%CyQq-@aAUg_6>kuO6 zGqWe`Jtp}Snt)EOsqHNbqvu3Ew==XSkS?uei_FU#2yZ!c*>I6i*98rH8a?@ zZ~(m6mD!#GP#pD2^2N-(Pfk23JU33Hs~Fc#eED7P{0o#$%`Q>m=BWe$4Tz8^I(;Wb z3~u4mLH;xGz$hl5C4Z*VZ5|>07nIGAQ)kN4K-fmsVd3_%_jEP`!;K|~E0kQoO@}oc1^7Y6#c3?>r-(QD=UhA} zhshmA)L~`ap~Psq$t|1ut7J}z*O%-JgktssYiCI=xsGrTRhqr*YwGM*FOZvrJ_okNLLa=$WrKV&G zUY+u(YpX5TyQtzsa=M20F#!^-9rDnG2zV))O^*#jd8n7@Zk($e_IYiYi`cdmLfE{` zEgD?+^J1i8tE8Fxs_bY`jRRM3eVPiV3KBH%RcpmbKAyp*M#Y1b(q%XnSwTUK9*$)s zRhz4O{HM(k4l>Txn_aN}qBu#q2Hk_F7D7@EhqnhAp(7`Co!pP$uxbv?(^&fGb5bHJ$`%lVx;7p`GDF6CU<9b&cH5P6Tz z>Z_U-Q_WaSj3pi8Wq)J4yp)HzpK$2PpDI*~7&>T36YnTlQ|m#B*^Qej z&ZMcEjV@TE?H~cdU{ns^WpF0(BCKb`V8|c@A`mW7m0V?Rag}%>o^frSP!~Lh7v1u5 zF_o{PJTx^qv7WhYo_b462Nz<6DHv+A10g>o%;nK!6G0~fi6fR0?8C*V*E5pzIHw4c zNsAin{RDRH)bq)4I^!=#`~cAp`z2bjPG3s;U@6C!g@_NxVZb;OndbQLuA%$(2g zNzq*)7KP=b86stiCCm-73p}G^_{ZNV#ALZ-EJ_c&rm=l`^DJsvtvGMc?PONkOSts1 z&Y(svopKMI9CHbFCK9y6*@$itD1&}dzuGj>UiGKa-^-&lV$O`Ulk92wTxe^=9C*Sm zZK5v&sGwJgrL#g@Ot1HVFLu;PP>eao9&oHjn-}@MT;Gw8u;ZOv=e+KPcA^0u9IQtg z@sIp7n*v_-Jdcm|&5H%q&aLT}GHa-M?ToA+1#d4Z^)DT375T80DVdNvW##vO80sko zIi($rUpo}JoWYm68C$HZVjSs$=9l}^F!MfTlm%#p0zlWgE9+2vA>?RH2cXjxbTWV!P_K5bCe$h7wQRElpcZGj#Z3r?D!dtZ?;8cly zt6m33cc$o12v?&t14gi)o!s+tjX6JikLXn22zmKvOh7FwDe%o%VdWxE@{>*OJNY7y z=gl(rOS|B=C{Ks>=SBsPG%rD>FaA_;i;6fpI&>Pd4;T$z8Nz%Y;fR=I^jE@_Od&+_ zr`U~ay>?)#3SB9*6_tQh@R$>BOd*&J<=+Uz7C3rXu^On7Z7W+f@q&x=@qaI{#ZFtN z+Fys{id|(v^4iJ%k|;)gmiuC2)QzqdPA{h2VaN@-)~_9D?+ty#1<6qtoO|Q6g{ETC z4WumQuZP0|(Cbfi%r&=G8Y9!vpF#jmdh7hgT40PmwvOuy;w>DsPC5W@yG{Lh4zAY?L^#QtCb7yx_p=GCXgncmcWTv6j^VN`2|sK@5LI zp+f94UhfEp(lk&Yy;9_NA6+j*ah>r>-5K>`9?l(=&9DA+e{iY4x_uvU2_Bf9ni54{I&=HY9L9^hoev!Pk>e4oV=Lu${)w!FcP4Gt zs~~nRa!+Z9DVr}$sl42^!uL5lpGmzN?zO@-+8c=+ zBbw4SDJoT%yUU0D&|HAb8F*5;<-So_xoNE{b1k^RLR7iio>Kjz!4-hzpdlas1vCA4 zhPQ`t4*NS{rn{V&|3g^uzmSED-Z+8W@+D%IoP)`8pw*)j-f=0H*%%ivp0qbp zlk*ry^5>9Zr-Lw)jtc~n;8x7Y(T9|qCzP9qB|iz6JgAkvxN8&R{#5bgbnp`C<$%;e z1y$KOazN+-1FHb!$LF~;qRiIjqO%G-w2GDn-d7t^&(mW`t(Z7qWP*p` z9et4@IMJD(5!+`#LIU=(9CJm)>17@bPg__{{=^;eMqh`zc^ns0xQX0zMI=B-T7hPR zszRChGv8DgL7UYeOp2}C$G6-EGB#=;Xm^1(j{99To5k2Ek?T6d=OiUz{o@GnD|!k)Di7xl#`=2)tP*iz)`fHGp`|Te42D z`IR^?7P2~&!W!AmoGOHKImdWVbmFHR!Sl5Da>HO#=_0{IjXmat^0*Qm0vT`xt(Kun zag^NEZCLobur72?`&)N1+>4K$wl@JS(m&3kijmvq)L)LLIpN?xpj2Y90&PK57=(48 z;?F@;1cYrsq!mCaG9+E#g(Gnaq&nK3Lvr`v{Q{XDS$gJTshj6;-7GCZtx^IUcBHqgTL-9{$Wj_MM7OoFKBtGKF?H1e585N&F+`zN%E7Or8?-WHTO2yV4p) zIDXM}#{Rae*E8E%#3G(#5KKLJ)2d~MvH4LIy>Xi4*r1#Rl5HX29`oSc}M4D|m z)9QS#9;D;s&J!t1YDheQ~ex{yx=wOp{+emusgPc2;m~HEZKA?o zInhx?Bj~VI*sed|@jnOP(4PuZYwfKSXQtXdG%(&;r5Zhr@p8afOzM$IFP}Y6E`N}MMSIGHNk>;;yy1VXgQbH1?D;lw z)(J~|xQTy&AA6-d`Zx~*Ic^Ts!1pr4y2~h$gV=X#- zw=sIcv3$tE5WD&d{hqfDy^f5Qv<{mvQ1e6yKm=*v^Jf*~cH<%3>yweRj-K}+GdF%! z4yoUEqQG++qE36Sh1m1zx0XS#hZm$z_Mg?p830k5^lHKqC7P?($@n@f^<;38E$>gF zOvlE9KU`jsUMBL%$sI9)83|4q4Un-5_fe>Q<`~*!LX3Bc_uZO_>l`DsIiUnAKi2&U zvH)wlWzO~NZ~3rKOf-MXt=Ti6+rocH|NA%+Aj%o`Zhzc%$Ba$%zZk!HO;x+MK07%F zEIe|azjJ%!3AqwKuT6HVba?QUwYf-IhYolZmGs8kBM2yJj2|?1(R>2_RWcy*&Vft0(c!Jlxjd=4CSdbm z6av5!FtqjjmQ)I+59DT~3v0PVakhE}st498379rczX~NOi~@4r0u2mr9i5-wf=UDR zQMm;pg&ofl$nph>H5(oWAHX&x4sis4ROeODT`1Eoq&j!sU}QM~?&nPa!Hcnd7g_-p za2SLgS-&agu1DwAE(`c_jceH^VP@h>Cx@rMX}bB*CBGx zVLvxXU_`)e2!Fd`r;nG4XwH$QiI3OZ72ks2yXM-bvCqSx$5HX%4NN!6Ac?`{;Dqh) zxGiPsY8r5F)1^2uIy&v>NIVh2j~P!oBE^->f0Q&nQy6`iB9k<-TvWb5#|f_`(u7w^ zzU)B))*hv#84}1~!r4tTafZnfh7{iM7{DFWgYnRjr8VX^l(F?Qdw4W**cVWKG*3ev zNvZcE^`5LDyAIV2Ai;inRhU4Gq3}^`b*w1?-^--)`pX3KH-pB5he?f5gW{uP2IUO; zXcYXARf9-29U~Uz0h&|J8ifN&S_Iz^(4?(hl%K($!T>F6xavUmE}~u5m$Wu@{D3!u zu{KTn0E$Oxiv~Kv{vIP4eaM|hVT%qsa?`w}U91+({@&NA4Ip`$I2+{yOgj)vp>VIb zJ^)Q2BU1Ssa^R5WuO0~(u!&r&QpA2=lD|u*!~R)LNrOM+@0R2#xlQI@=pK<*>36{M zlW^DLOdo)8r$ZtJAR0se1ywnu8$fv=oUiynRWJF$ zS+5KrY?u6Cy#4bcwpVu0-N!Wi9>6sO9RPk%)VF!S6fmpg&+vtZanNLx`J!9j`$FcK z_#)+4@WShv@j~UvcB(rp6vQ{&)*m+X-0znd$RvmAD_6P4$BN>ssnh>#u5ycStu5-_ z7CKo}jmkQV$Ka!*YQ|*SdTGBr=4zWWRnIvtZ`(B0Sex)!!{GeKjcX$8$Z5s&-t)qo zXYHD_>1v-wYJ$|gXR2j%oWDy8sp(`K{fd`o!GPgs>*}PhPY{9quVl!-8p&-FqVI$} z{a<*%Hs?m9wO35MQZXkcBcZU^Lsi$lnNQ#I3mVF%!erjKf8dIg9u3zwnXIjnY#2Zqpv9*+KOonV&*8Z6dCJuzAlYu=S zbzHs;@be{3zwjiu@9yMAsJzwy^ydp=s5FSJj=I5dP&Pnkr+UyDuI>?Z&ii(hx9#>2Ccia&msg)o$;#*R@L9#Y#mNxnA`hI?D zG)}9+E7MGM#cZkN&d->KbR7TqF{x=SdzmlzdAkVzP$P^FsS@}~BUxa{XZ}1m>ttP- zrvpf0bx|%R728;uX?n6neXwhfN%a0}y*=u(0M;;yeymVzM@gxv%+-;pa>RR1K3nnw z7MX(SPmh6g(2+zkHB)I4P%@K-serX3f(~fDM8Q<1eh;TZ&$ak^PX|84t||Uom-B^b zg{%HcL(vF-05E4aDO`CZaom4N!n^G9%Jz%!jf9QSlH~$`dga|Ar@6(zA9`AQdU_Q` zyd4qtcDi&v)vmQEpF?;>cfEkwX5(5QGC!Y@;RI^?ahguL^Rr{^^1><`0NUe1uRexv zoeRjY0WvJnv@P;>1TiMJEetzT>oeWtX907XHCyMC?in|jJm#zJ4ZKx%q;@CYc}R$j zbgz$#fP?(Pf&-)D-M)=mqo%+jeg(mil6$9I%De-*SN7G3Pk@NvVA_P`E8HtvkA&bT zAMjy|%5YB#81>QY8)2Bp^iXw=>97A4xtmsZKYIVveseE)Cq(-;)K1Bnd{G(vCEgL` zvvcYv$HGfZt$76;ezVQU=C`JKz}1Oy$8}uEWRz>ryr8muGTH8I9~{*@+t8k9+w1&- zcevnU7w!=&8`~Yt;s2*F5KgD$4#z@y=puNi>unF`tf!Y5`b#(~EE#g{Fbsxhf3_y& zKuQ7ai8g#JTOL#n2U}Guzh=smxiZ;}ykN&97m1la1s1?wp7c{PNVjog=5ZT}isHS6 z8-)@WcEfJo{=}vdi*^dB#(u!(&{@kx@WsY6StWeH^<3CiChRITeQ;h= z8VpI19K8O>I`<4&&%>xZwm58Zw&HEW$>Ydz=04~i>T121wC6w+;9eOT**fu~*yz9{ z((ZM!KUrqG&xPBO6}UeNR}(DDneCo$i`nY-^@lZM7*@~!32wb}nqh7J%85+@8VxTp zJG=U_6}%x~Eq%{rBARD)tXBv0%_s9YZ&tY|&99%km1R=1H#M zJQY4|&yHH7>1Nm68RI}HjKZ|UHBu_9cZ?>&)q?ZO0D^ty+p)A&9`A?#!n61dtx7P?N#%8){W4?%;NHq ztCWP6oHpfVDJv^GwVHvN99HH-oY2vNV!YYR!+?D>mTI&U-#BolCTSsY*~Fva6nu^b zKZl>D@J_{wy6`t8tu|zN8XOuJ?}R#MPI2cvZ6#lfy41XQifEGh&&*11<%V;GCN*C5 za)e5?47IvOtD5tlCsm-X8F%jo?fLfjMM$`B!q;kJajp5sX z;qZCc`Ad5Fp{z2ZS7rJGxkUwH*(RZKB^<#m7N>Xyb8yLF%CsflRY}N0o`D^}tvXwQ? z9PQTCJAK3T?0>>{;(PKfFzfN~m`Ch`)~B9$ArjiRxucG{iB+q$u)K4A6AN9m3BK=! zNW=}ft_!{ij=3-FAthmYv0|jV1>R`BkNn18E;-cqDoDgfL`tv}rSR zA{_1Xz=*d`WBCW8N+X-dOnUOW(w*E~X?7wz!_Fp#HFRW&P6fRPvi{Rtt8%^_4CLnR zIReDyJ6|a$8Ti*HQisbc8$`)01~%Z5)CSmosxW);2RDdxRwU!7yTlb!b zJ1liu6ENpSX$JNN7&aO%ylC1{v*wY;dWcDWExL5k58X}OO}Y*6I}m|h0UUKtx=A>3$N`J~%8}!xu+-p1J)_IwWQ*(PsfsO_7qD%r~1x}h(_~;qo>c#tV zqm~FNb{AOlI$NvAMRLLlCnG1@#<{=?ek~uFU|km~jS82oe*5kKX1@c59{ypo*L!2mkV+;v9R{DngYXtMTD*kn3Dg(u|jLOR+s8%r*0fz9%xa|oCAM{yiwGP^9hUHVQEa=4jLNs~52ZpBC z#Ki10$?)1}EMFIg!l2*NV=68?ca+-yZEK)l1;qA*uRJ#4*?~QigX&q^d|z2aqaKT( z?Sab+zZk>ZBKoBRc5)ef_M{dk9K%Bt)t7Pz05wi432rlo>Ux2c0Lk5Bh!2rqFMf5WY0g;Nyga4)L86r;usU^&C5N3 zgQgVy&2^@;({2?F1lbmOD(O(QSg`+w1^BalzO=r@s~dhatV@x%69P zgsP_&b7V;iPokqx90GJ0SP;-4uVRq0Um*|LA3RPu#>Bk1i7K~`WIm*GGl&nrjb>+6 zK6}tBFyg6o81G!irKP3`J=24n`LWoeMv>mz%$UY<6+-(0 zhP%%8JmnNzL^s!s<}e#=ELQ8fVHxZ8m_(1sRL;AS&kwdSOu zn#Pvt9yNQ*PZk5#J7U>y)P3Qpfz|fmzF7#Rrn=0Rj>KGhOugSy9u#zxEW=H_78uA z%1giP4g?k)hrI_#ujynho7koB2L#7(=!`?_QC%v1IfAFt9`%Q_O) zDM};QaMYG)(1rStD>@cLd^@B@dB$+MS0jJ>)qb*~pAeIzc2dTZy z*&9KSAHlr5Vf_~(rfIe2!d-dX&w(M7_fO@j17?QMh!V=S7tZm*l-yc#OVE&0>K%&n zlqQi^U$WU5wK-<@KO+ouRN-Qh0VR}ZPmFJcq*{ByUmF>}> zo8t#*RV$tt1z2kSJ>7;Y${o+H5Y8RQW}@|iuVG($7aH1S|2j@p5fExQ)m-?Y(v4rP zDkO=JFet08f|;L-x+wgswzvQv))Q0R>YyoO7R#ej-rA)Q*=hBdkmoFuO--zenzJJlrgM#npphd4*3LHjb*_BR;Mg1rcmsr*^ zG-fj@vU(HUU5Jw|@28U#i|vTyyE2Gz{~njTHG;A}Aou4G=j93czR}ctqc45f0=Q?X zfsz(v3#@m_I)V1NZ}~0ayi|^PS#B3G+1pRvaUCF;~JgT_>{zkgZ*_0D)Rn~S$}8d(`E z+|<12UQEUIoaEs4dPAGHM0LIdrNfK>y{H_q&nC&4X)PY=kp8a!>XUMKnT_gE7WKYod(CnIAZ^Y^yr|yZz z9oAR4;kYz_^6(BnBYJlN0YG2#(hYG(y*_fCBG9-s;{K?}Sr9f%`5Yld6fe&+`0RyC zi%qF3b<3!33Pj`!B)pgF(nv)e4=QF2KPXnnA#V(m=J6EGlR*7e^OaeRZVRK~$=kXemMFglo~F9gj~H=(J};}DINjFsI_evY*; z&lfI52<$wqn42Sy!vor~ZYsIH-tH%*pTzrm7K4<^58&u{H)0g(l^W&h6^Nwhj8f7G z$2i2Uw?0Vn+G`H0Y+F^e`uq_nFS!BpTMi=BJYXg0d2y{sAj7M|(N;+;{tOp4K6w_c z{9A}bYGzmB-Spg4qXJ|r;w+jtD|dc%?$KfXGivL{BemQ$apq=0&|eh>f61Mb3!yER8^8;I`WKkuxYUB=v56g2c<1*C|%TI zW0tKg?5AT9b#mAUJd+R_i>0EzvT|2r^_7SC| z`t+aLV3|pKxXI)iFn;h=@upR?OW@II){> z;d3!?IRBUV#S36`P@Cc}TLo^(M<_tfkXr%mN?41-R8x4EkoQtP$6VELKbbM56mZeU z@4Z=htN0COdz@ZEoPj+e0M5?CnL(VNJEs&DqxxZvbIvkCf`*PDn$)q5AH?D{cxWwK z1Ny!{XSd#X53D%`L9_c9KG8Jxv!Jv9otkb1LUMW6~s{G&Kc-UWW3ylEXEu@uuYdl z2tQshEpwL4Nd0Tb-ErCWd$oDD?b>y3@$dQD$MhDGU+PDWxBEZ8#U}ccQ*olaUBXR5 zmp7L<4_+DkM0q&SwlkR2}=p{$xP;nX*`BA=55xMmKQf*3_d{LoWyV0EBU_`T0jB zZ`Dguvi|HwDoC*yOh3+9PXBMSt(Eti61U=_32RIG(P2_>#l>&M)&|w#0?QMNQHv)( z<>P9#OE%WV4b{vOCwWawuGz!v+8*I0*&WouVGpL#Ibn}GqRrU@ACw{>%HSRI{gFfR zgg|)`xfZF^d|y_zMC`tI2XW3qoP!rk*U@CVa_pgo7f#zGIYK{ryRPB^saG25pB)kWKh%B=>KZdAih4+*O5+J>p zf!wNd_?w4jsOLA>f1bOmh)A+A|HWxv6fiNZ;ei9gc%sFc9=6}h{3DByiyyYNX?G<59fy4P1WQk6=y zbadb$>C%*bgiMsDSW8ujJq7XFNR7G^j55U*9Rk05PA;AQUS7i}jv!~b%=mci?0$RY zJ^*gU|EHxc{~phj-lf;h;7z%1nIOdb}zJv zJvVU}=4-V~z((|gK^C|(5XC|HvSe-^waiwb;-JBJocchDCkgwLtSq1-+#44OJ4jz( z;W$j4ML-8$ThwHs!|8zM7&GjkFGE(U`q(nSH#;voTE>)=u#^v*v;f^W+?o>&@nN73 z+`v4NH#EkCwO6R-^xlMOQ-%!;OpC=l$goZIX-2ru85nHo|Mb~2V)W78P_;87FnV!~ zLqdm6bkQg z&!lCu(=bt}X?f#3{%T496to5vzyil84xOQWHDuZ9`2D~%h)13MR$HQp9xbNowFaqL z&Z{V<*x5)-wu*EnLSgNo?^s+qn$dbPq{hlpsjJ0Go~tH~R^eGuo1XUNA2hMFzJKmW z9J-Y+zsxhR+q-OL&9yM9GR2&Oyl9(hk!2?w0VRgfgDkurya3~7%mJIs$eLCOE9MSY zfsy*$dC^!d5Ow;G{((61U+#LoVc5#uknU3j?*8nAdDbAh)O4m)>V={m*37@$cOZXx zZc8v$@xk>xQIJ^zlCOrcqeleTXABc$lZ4Bpo}ooqHRaHLIY>u&bo_c;;=YC1y^z;e zeBA$2{WH!&yGkK7CR$RVcib*|{JG0cH3*k9j!LzSTcl_LEi3%4`3aLV6tNAF)xH`w z_>$8M-JajF#TWtK0e=V;7Qdk#%Y-hv1z+W=***Nm$s4*m#hSZ~+q$hmF3RSsov0+q zHRg?;qC9myZL+-Uh50`gX3rK>&Ms?gPctCKs6P8b9 zB0izs?_ku2daNr21PvaGC1RgD&ZZAt4G@$Hy|e+czmddrn8d&cYmYbeDHwGu#1 zDXx}6pXqLRH?1@WZX=Uw$aEZS7wdV&!*pho&aR@6_ThA@xX!r!V5&xA1r=r36c$GT zi=U|J21JuYH?oatQ?a;nQPRBJFXzkoDe{Cs!>$e60OSiNF5RQ+O}w*+w7X9j0mSDk3(7fhj@inEs4uwh~n3QBxP zAD$HV;w$U$1!wZw4A;2>$4EUv)%1)jkPXn(3+N(N_3Wt11Fto55Q_B5*hQvhT836u z;&+$@!P3YU%kfyD1>({+L~{4>i)zeUE~kfw*wv9bGTdDk>`6El<>GsTx0xfan8LQ^ z%ZxKpb1N&Xv%wn7DhGb@9B)gr7bP?~d-kTYT{#=F02u+a9UU#B%meX0xN@@v748Ff z#qrK57*#Z89BK9?@hx_SQlu9VtpT4|v}W*>t-5>IrG1ky>hzvzfq5Ch&nn@BUrJzt zZGU_BkX2u>UQ+)F)JVJiWmkZU!dDO~Q|f)uV3?-g*!xAnQMzc1pm~FTy=5(Oink!{ z@p$`UeRsqidaRGmA|7onMqvzAo^{!*LGWu>kIS(e(|$fHk9_LggHEhTO@+~z3YkAD zkC4k(fe-mc%RB{3D7vW3z)h@|9{rG?nBRLB8oMi-^a}kst1{!Fckea9BRwUS{6GdI zD9uizInf>Bm%Y?%nsC}Uf~Y4iBFIRdhKyP`YTk-hwN6vD9;;}cs8DluXUudSpgKVm z%psYPVf2c0-XS~2o#2t<0Z(=baArh%AT;UCQlDXrBGAq+3OA3NVV)(CpuW!dTdFYg zPruJ|_8+6%Ez7nY`uO84Se%`wG$WP#z-D4V{dcx$;x&+g_jx~Ry-2m$F22y^yD@{> z-r0Y=%usdaSNqW~+7on>uUzLZVV*pd8O!7cYLl;wwcq1(mc0zTh9iU4&b3+d>r?

e0cU_e$jY>cHiYKa|XYQ&<{VVv=jmj(y-SqP1 zWevnirKpB6R7%IlI~)coqsH{$3KKYvEk(q|a%0}!2TLu_szkuUSeNjfqQbpk1rl}8 z$h#Wv&5wJH#8rhIzHz>=f2(|X0Qj-4cmRjCS6;)Ne+@88C&EP;OW zBkzd5Y%w0Syewd1P>xz#T07(PLr6!FITHYm+y8)v5E&`%-XLa^=`qLbG>s0^9w)@suAmb;x_%wu|v_w#C=S5d?we`uu&; zSEj{8X2pNmRFK(J{GyjYCC`;Dh7PL*e;Q?YqR|g6T4BxV%tX;ACD>XV@zXhTSRC1S zCi}zhDq}t)JTL0ruZVb82^F=?IGx}QJ?7-){u_KJleALT_k1$XcYizeYeg4LKtX z&E=~m+nsH3-}o#}sd43jW51)yGIWNqW(H1S!}c;say-t$`HA_5ANtRVfn*h$AE+S* zs?lh9v=^o^UgL3R^XEQ>f!$@temDP*`41K3_rW?O2*~=s{6xI}(M$exbNxTfWKQa< z2fF&dl-EV5(9mWDa10a(?W|PFKsX!v1@@R`ak%YxjeZ%{X4NG540)@l{6ILm^1ZK#31jtQPEJaRY_+`fA#1v zWT+3xsZGRqTsSQ!Zl!N$X--mY8&DA=L+Mkzht9*fT< z$aVa*VDq9mVXFwN3|{6pAn@{$Sv91sVytnWv=|9BW0Wa#$uI-bbCpyMBMTxtHcwf~ zIPfHVrf8WcEP>et8RsyFoNN+7h^Y~+(c7XWZNCKzW->E#$0t1FKPH+q?9lPJ-fvM_ z=T4K&8Y3;K3tU74*_n=3Tq*lUod-H|RZj zBdZPf)*Emy;~$~Z4=H6XS;_okz0Dj_j!Jq?%g+!bTik*`MgZ|yA50iEx30M-MNw_^ zAso674z?3#X(mpeC^LPYYyQqSP6G&1hi8Y(+h}n&d)rhTd28YaSj;OK)V2zrW3FyoDJ8QJm7}MH zy|}tsO55GsIcc zXDU#o&SxoD>4*$;CI$x}Tme#ZMjJ;OLRQKIs&I$3XpE|Dnm890u5E4xlf6PB0(4xA z$#FES)P(fp@#3d8+LW83^H<#t`iQ)<{Ix)L)qC)Df^Lv5X zUAUjM!B!JohYo)AA@=W!bLGwGWMIc}CE-@CxIh)Kq*y8+k@6XC=+2--?^XR~*H6griY}Os0QKI6XS_EDQe-1?{}Mu4|*kakRxr23?s&&f^x{T+W{# zo3aeeLB@RUOY!ga&P^o0#!So50YTzWlY7$=o^1UdY`(~USNh`C9D5Owfp_KTzqu$) zf8y>M9=%6=LpVCKKD;~4#4++VC7-)Xs9{0ikci#B!!%qr$@7^Pb-GH;q z-;~AYH18sG??tudc^1x4-h@Y_WW*knv#hTK(Y#|t;zTZ#mKk6BL}wp~3k zHifAvvZ<kZ+9%kE+Cbj%B~X$B7T!36Czsd-i3&g2(d?nkc8Y6*9YYYh9js*n0cY z{<;-~`3^0UlCVdfAoaZA$9UXs*GXJE|83WIpNGLC`t{*fQ<+l;0Zogbs37Kvxl5V>Jv>(#QMvH50gZ{O0oqN8hQ zfAw(Xay@OvirZ;txT|}Sz~S=u_3C@wCvOmNYiRfz4P<8MUEK)A>nryB6j*THiS(VB zY9vL1_u%GQkxz1Z;d|k?OlsI((`h~3DQt@%qRsjHnhfHR%xuX$)Y_iAbqM<$$#DP% zrZu#5B=tc88r5@q%N&e?Y~ocR3L7?^d<`EE6Qz3`MTFYGHCtzIQ;jZwFX|oC*0aOM zZYJf}?1_KO^vfn0Uu{EawXxFQzvz@F9RAl~V#m`?r}|^Jr{FGGhmpTYN(~8>{nr|% zKB!GmO%F=T-;91oW=?n!2HpK9Z)yh86&t=!P-;hpf$#`jEQ8n zvBD#URhHo)uXy{9eIY#72STVKuCNCjkgv8d($1A1rnvh)l@7MNFU50p}ku`VrGQ}A~EN)ivv9Al{MJ?MD z%?4QEWK9m@p*#kIXSc(tkioNf<`opK1BZ^HA$7>~xKlicmHV(0B^3}q zLr!t{X?^c1DD)96hA5FdGZht6v|g&Ac7uN~rG{T?I$%Xngs1mM36StsY(t;Uy|=Kx zDll#>N zCU&lcm2Fg*(S!aqP-667``1e4>~xqH!Et*s`>uIbx0>y8M>o>-VhFyK{k!|UW7aa^ z(my)cacr>zi>1>=1LDwstCb~cFBERuLOBV=>WM+x7usCHL7Iv|jxdkvB-|6^R~CRT zj+F!&|28V3xl-Y6;g8C$bf@^|d(jobrj_9x|PnR-2cMWHs8UC*pQl5lHswHUX?-Vi zF+IjDcRR@NOpR1Ubq)((T7WaOm5RcB`WX1fn)&mF`Z0sz zW5b?JS_Q~=<3h8`k&BIJQanVMB;*bQlL$dSG0ur>aji$gYVnXX!J8R{JX%pfbv|M* z&s69CogNQMiI_;fE`i!D3vyrtCybe*p(ZSAmtkBe0n_U@-SXT;Y5cXbXE z+Ep@Aq;ab1S2bXPr>Kh`t#o!6)f)9cT~-sDU4DfCOSLeKTV;6zCmxs+EExf{IY1U> z4-<7i+e(Xq%57hhL#V%WT*_@fq{A(25Sfgn?E~#*nzaGs>X+unLNI?H+E5f{wp*+x z3_55fFI3CCCH5l5a7l~khzA0kDXNXBRbWlwC9}X&e==aBAMq+Xm5kfS!3D3y;H&Ay zF;n_?`mqq_*hbz4M%Xl=Tt$xsO01dGa^&wzIw8tMN%-ztDVf1-1#4D#QVygq?%{!P ze2_VCMv+g-mAj@(b>vm@Dz+wpMGei7;$v+d_0>?MMUN(b-g?VcdV-5ORf-Fx^Afhl zD1|d10Wy?i_pn5{%;BH`J*xqH<6x>o#{aVFXd$&$NcpRx8WYibqkYr?rneg;cU&(f zS9pKN0Q8n`ZrR(FNiRJD99}5+OY~-j$v`Aka^XXm5(=Wz>%x z=av+YY!!JabE_43IdiiG{tlu-ac`2MLV0hHu%DvmU1)C(fsy%IJ#NW!?N(Cbt-&On z!;8;2=k^6Ha4jXkQDeJ~S|O;zNw;@X<_5q4OS+oDk2lJt&&(kmk=-JSkoD$CF%?DX znG-b2hj0^y{hA=zOa?9|QB3(%>^zlmcv)%3K?;hO{o)pzbWw84$WbC4Hud{usuPfr z(^%^4Cm<#`^VJsmpGmsNsHZ6dB`@MN*LSCMfA$%z(k(~OjFMcnT@qYcjs#)EDU>>? zq2O_8UfMY`@%TKRKwt72O!?!q=fMtZZjhFz#LnFDuzA-B^{8_D)W`d2{d$m@q^3-X zpGrKxAyTNKWI>hhs8q&f(WWU}SXw60#xUaZ5uG{71LO~3Sf{Zv$jBDO^CVU&;}=vD zJCmC`=mOMGe*~1MjEWUAQduO**pq2-Yxgkg#?)+?lMgbutw%=6K^b{9)&wVP!kQ|n zWSU=7Y_JtCrelf1#x!PA<+ZaCk`rs7QlS zm^|D{O$xqB+;mQyXGE6ZsU{zjEJELB4ZIAVr=?Vq2}E(Fko$8{Vn+eHQa&6uofh0x zj1*u7eY-d+mv^A-h0=2cQf&U4-yEc~yiG728Xw%6Q3B@Fktb3Pp>gycuLW6S;w)M9 z!OCWlnO_q%cY5W7cu#(kaZMOs?>rs~U&az-HNJT@F`G*mcT1Xdp9+qcYw`0RQ*c+m zp1@rU#DQGlDunSg;HjRtRaJIPT zpa7uq@(i2aQk^Ke)OzSB*BD1kx`B)~6NYx6T!0w|LBSu~h<+xcgn4I0eRlMqtj{g& z=E1t`Kj4{Nj^yi5=`#j4uEvG_!4&jww9nYjB$iqVEzcvIf?Cq(wpGpWnfL{|M8s!aQvaa) z(=6D2k7xMP&n$lFC+6?yb7Ak5-ayV^>JmhL!jL0Qy-hO8NJ9oCj09mRgln& zbh+S`!;633`$po_vLWnH_=TW{QyN*sfBpfVTLtF?!j$!L*EF#H=#jGcCWQQpTps%x zzjZ@i_}6mcJ}=(;D|at(@c?sxotFj~jhZKTpA^p&LI^=h^@w$maxtWNTw4N8_IZ+2 zNG=XtI;>7RQaWz*#C3v7x8%*6w&#Cwb&kQgMBTQIZQHhO+qP}nwr%a$wzFd!JJt@~ zIN8xj-E(f8d%s`Z)z!aNt(rZ1%(0%4WmB1PS*{}-l^v&}xKKPRl25VsJTv?8a>{&? zF3zIxz(OmBxp0Pu1dZ4@f~WX3Xe(y32hG)9o2N(+BnfG>&*evYW+up|TE8&p3YNv6 zyZSfs7LQ&4sLdr&A5O=&pH`zP)YojDdE*^%k@TCEAPAEl?|UX7_O#?1dNz;ncfZz0 zX<%?T+IB3|L!$4X@NDiGiZA|m9FDx-vvKQ}^vUNowYE+F<+nt_GWPKwMK7}AJVUtr zrrh%@(2PMhx^NFXNMbxh&jYZ|!YxuQgoY!tS8jzNJ3ds}%DV9uXLcVa{5Ip-;+jXf zZjkxm8fOH8v94FuyvPBPSI*9$t~)-C+%28np!`U9J5rA3(2daoqE|AG7R07-%mWhv zg4{PUJr1-|uX6W9lqB^-WU~mQ$X7vWTBlz6Sni-CS#JV7P=H7Lp^C)Qh_40pRkL$u z$>YeoJOi%$29$~hs+}t}g6qnS%A*^q8uW_Xg0N8p^yRZk^#G3pT2anXg@u$SK}>s) zz0n8}*B#3OGuRv2PZ~R1`UW(?RY;c?_lhZ4h6&o55nRYdJ`#;60J5eCM%lPq9O8G9 z+#JkFG5Mn=ovbyYcyn20ZNDO`IGYn8hY{%-m6mC#ORlfljkA*?&I?x;Wxg-l&Ez3L zzHf6ECFR8R=aRf)A)RHFJh!g>tOn%_0g(CyXgF?m=z;}lOEaM) z^;c*pDF1@>F;$<^Z`!Wp#Y9sGr#wMj_&5G%f#3KztdTU}9;C~7Djx)GOTWon0NC#Q z2qOwL8n`s{&TvMX0$73~3&^1A^TxF@4ahr|%C=_NJAc*5emM+pN@(bz_)DP_*Bhr4 z^~B(6Q#cm1(gaT})YUKlPB{9lum`nW4MtD8t&?Y9ZYEnbDqYs*v(-m!cdFtgsDZGb z=1)x_j)$g+j|{30vA#@3B8bd%lsP!#-a~eNF~R87Dv(9wg%>SzZ&AgyE42)ZC|s4? zP8b9COW*)@~p< zG*=Qjs^K1Phsqv2$&tvRWDq* zF-FzFU08NC(e(phz~Syf+{+-j7m~(-|3U=fpavhvHb8hI7;(t{mFa4%w2_c7SimnX zMPCqaPm&+{f~Rr#$$ZMiLm&a!1dH7#KX8hS4zW`J=aD-VVMrA_hi+z6u1u z3Kr=H7A*=MP`+Y*@P4Il0vzdZ1=;!yg)_$chx4wC zZ3vBVd}GX;EwwRt#jjtqIo?U9PvTCTaalW4(E%hHqthEy2J;dR>2`RCEi44)Cw}}j z`?^}>0~zwCzS``9qjSfS8){!nys?C)%dgZP2;a=PbAK!h_ir1sKS(_ozO!bP)ii33 z8+pouW;Lfayk*dN4Y_0Wf1+q0%ER@`YRzKN1Z_1)YQ;APF=#cL$fq(lR8Al~np|LY zE3V9F)Aa4inOZWAhoJ&}D!!QPV(IV-Piw z(qL87Wz8>AAPLO54MaY$7k&(p>0Z>w4^IG!`r%b2q4m!9MM=}}593b6btND7J5whl zU1%5Do^!odW+Fpahlp#9%?B!JQjzh?m^0z>>PUx9zk30xX^q6E;C!QM9K;28`Rv+P z@vMsGMZR8-kV=~}mE_Bj06p40jdy4K>kY(BbTE&fjlGZNxUT+qbb>f8n-6`+9v3cb ztiNxBob|iJiJ{F^3vSMIw=v5;2>mK14EFJI2ql5&p>2pb04QB=oT>w+2l?iY7G?~J za?NSyZ3G{1+77%<7Z?PSgq;r>%9JT7GoJ^_XlV)+0VMR1H0oH}g!Ro0cv~5w+TvZA zX}kv6UPILLp-hwhhMw5W^D0Qq)283R?N+Y*+$xIcenF&eMUcW=)YB6t1j6V zYz}_cmgdRE?Gq7>I;nNv11R;}Z-W_zS%=C9QB6+Xnfi0NKX!k_xa}S-Y@fLD^YVhdGzxHy-$YJq zfI5ZvqZe2p4pBd|dqwku445SjH*7k;;_?RL_@cyaENjj7N4H&h>x~)+5xz70+oaZ; z=#SLDBJWN4MCxeNbmH_yceE578@ZC_8`>Mj+}M7l@P+*Ty;oZHD6qTeTC`){GL{4QrLtbBc$mv$Ke`of8ybRs%XluXNq`F_slkL%;*x@Hw3%c z(4`BJP-j7e1&0Tw?I{>idqx@(a?kBe;wus16VCPwk9}!h)&Y!td0o~gX;!Hz9f^3b z%JMO9jJS`1{e{^m`bv)dmD!Z<6%Y5FnRGJ-+9pA37?L?FB>Q5>aLe7dvA>oNS*)~D z(d9+2UU3u9n-H@YA@K9`^R5avQeOXX?(}y2%mIgkb0VK#IdR^eki#>A7t?d}%mJU{ z%n5P7K7wR704XoMH)?7>Vyc+;f<%UEP&W9Ed4KOIt`~ak{_4WeFCk1pR1QFk5pE&G_8WI&?D`#+_PxOu4PFPz3bCwX?C!sr1)x2F)GY>^dH35$R=Hw}R3&j}M2s=f1|}2Xxyj}E zkw#8O1KE(Z;5keo0>Q$tT16wTCN@SSS9mo{mVH1*_)rc>-XOSbgSsLp~d|;^@e?mmZJZ{#^)(Yadh zpQQM+WD&f$o*qwGz2v>0(}oa$+hoLED4IZQme<7Jhz7yRACyW{f25d3w0`gu4NXQm$0=p-owMH7QNhhhJa_FM?rRQ=t)K%f)S{;%nQ>|B}MiYtW zGC4Ltk^aMtHZfJkQ_~U1Z9fAXVd|kJC13(JhPzdtEG%Dc^oEP z=(4@SW?pW*Z&I_+w6G6;KFknr5fb{I&uXUyrxhe#P~M>toT0>roLaq_EAG{2)!+Pk zj>-+3j0A9wO!lwLfXk=X&KN9GpzG<9}1UVNL}Q zG`q07xhyGYFo=Kkww@VYUBiUuqK^>%nV)Lj(t9#V6XO_jX4>(MMFhFKunp37YYw)G)thO%*$f2r+^ObD;%GG7peGmO8rWPb6-LGkaG!)~OB-<3UkF{ntr6;c`^3Hx z+No9c!M2R4haBf(slNm0jP;*{ej(ysbt9L*N{vjshF##CG`DW}k4&(B@DNa*W<<<0 z&0w_4)+>|eEheeY{xC>o)vHdc^~rJ%D%N|X=~u?&Gi}mq^B5~n7jbr3N{WnvcGssg zr>6_AVY!-l-*H?eO)a*0T7JR}oAFj1RED@5mHxrdf4ss-6p$c6=kW?9vOe>e7P1<| zwBi*R!h<3^2Ng@DbrW`0fopP0cp*6<6A&DoA?JT|BNtWVI?W&9Fc8H|V~GozEB(cb z`U}O~IF3YNv=-h-@{PK-psWQ-Ifz42OFJbzX z&NJR0t|+a@_4pCIY*X^gycMPb558A*v}?#*?#c{+y0gkfxy(xN;c4Spcc-iU0nhQc zr5`hDZ54^$Dr$Mhv<*hO+jS#z{(Vi$4vYj_|4d{$0=pEXQ2G-q-@(;16X(Ui#TCw!ghC)hs@ z(qH_SD8u< zxyNEltNP;g@vyo)?C++%kP3N|@re`}?W-EKc@V_{8RoR@vlhNGrukcrr_fE(!#0@D z=DW^szU286xPIRi3JL-Nls^oFBL{E%n63L%q~>M|2a<9$_c6UjQD+US&Qo*hsL#us z)ODBjLbg1@A;qDzX> zGg8l*YkcMiwDY1Y&>me}A-_d8A_Jfs4flo2CXPb(+hM?gyKvNy#iDA<<5L3Kb!IM4 zdo$?I;{^5Rti$6haPD_e|6QYxq-<@c2%GS*>qH`CvHhAVM}cOUDo-BM*diM!Ue0v&5x3ETDFNwwLdlk2e-?={l@b;)5=7LW zwRD~I)0jSSo0d^f$m5u8t~$g@0r9;;Z~3q(r9VoL6K5sX_zMj_;*f$DRc()!k~(Hx z&*)Tj=*%Y!mcmr5?X&9{oSNLuEWXU{C}%3q37s*&_LAAWCIzKEK1P+O z33x;XeJY~>6Chr~2zJL-pgcHrT?3qClQUm;V)fk|lW3aPT`_k>==AAE!rP#GQk}g_ zSzYcYO!yPzPU$we+N5`__fqM4y4lRH%ntL0q;RASZu$FBL5fBIxLq@ekNHW$T6^Eh zRJBtiWtjwqIONO>w=DbIg93Y#%2GY=2YeO7H`rOt%O^LP|3GtktPsRoTU_M_gqfwt zNij`5g5CF9$%2iy1Ep+C9KCR~bdIjE&n9#BBP2$=SBj{ompyu%#Gc0OmxBE|lCbpU zYcsr%e2AsBx8!#Fi@3OOZ{m3JJJqUP z%|U!#UnU*)k&H4>y&;c9r}#4MLanvjm#pWH#vMEN&~CvoGED#Use`~qwhGTcB`3h2 zTV~H`TvVd}TWKs2VEQ&xqxOh0?Qlc576LE0UU!dBqxHl#3< zc@Kx^+so3aKwl~U%;Aih5pCBiY=CfIHv>aJ;AJB0M+@BWztWYmE4j4~Grj9z20-4p z?-K?3;$4BBj{>cSp8t9bO~Z4!z9X*RuSF2GP_9qdSJr76xy$l2NKx$Spnb-GxLF(I z#Xm5HRB}h>6S}xkw($?M75qbjBGDLC1+3?~n;J!t{)zj# zA%E4E_OTYVgzz~{Ho5pLEStRI{!KIQTBLg+TuzKKScen;+~X55nb zgN~%G$Z-3aypq_W)O|$_Q_i}a9{R`C63ljb<@nn zI*7-PG{Ln=6FgduGX<^_53Uo9TT8-2jHx+~7b12g9J@G|EyWkY%4?w#dS|$PbWeYi zJ6{GD*N10RE|lvXXO!yK=#bY2BdhllfdH%LE+qm5!H)OKALxs?dFN9qTAVBr0NdaD zA1I+E(rMoTlm+sLw+nx$vl&j7!fKYnfvnf;MJ}@*wd z)wg;*&vB4Vv26xQcFi8}H-05(^DtpAHS7ZfY>NTIuEGf2{ToTX|D<#6JbNwoT5N-_ zWB}+NJMMEhVR^fBBU7A3p6d%e{BFdODeGC^WEGFL2LcXs zc89(fT&ZsEpn~UJlZCPUzt)kld&e|VC*Tnr_bD;gn>hC#TyNII*Lg>aP+#&rztL~s z!dvwsze#j2Gd;bPfaV|eo-58y8(9-IsOgu+P|jq+8FI|_OkQg5x^+zq-cHPa%`}0Y zzV+29=7BTj0eOGJuSNG4`ct@4#y)o-U=O_i$F~(IMFWfRr$w)U0s{I$p#0x+SgLL= z?xt>H=B}nL)=qAYF8|%D$E(|`;;N$sFqCbQIh2?6Le-^f#ClWP>DHuINaQH7Wks|H zQx%QR-jM6>TrKUwo4zZ4AO9c0PG55NuedW zLQP9eVU&-tN<8T&VD90v)w&=f_jMFlF?Gc7@(+=Oc(^m`mG9pLvHT3}{K2IoChJX~ zl)K&&h*jL=3kInu)O(|;BNo-VF(5~DY1H!KjWd9u^+w6T9ls@$kb+}8nIt{FTI;q0 zs(!j?#!G=oW^Nz{AKnC7FU6rAI8`yPc{{8eX}h*`ji*5(gdUoDlL8~J9m_0u)mG5y zH3HW^_P!8|t@z8QU05n^5HEb|x~+Mj-4Lq3H5PTnC_`;wn#D}CI%B};SVnXCpo*Di z5QpEpse_&MFm-mZM4B=NAzzh->Kfb0#G{SM+a&(V>r}r&bzqNQJc3xS_V7fMuk6#( zd#>Ksg5D^zv@H4>I=+J0L34w9Vx1~Hs5@g8GL6y=SoH{Sp42%`0I0_zVsP|d7EE_* zI)#hvH!DJX#r%w#C>rBRk|JonpNmZR(?gLQv1@1qP|La68+`j`4816B(&@~n57Iqq zhWg?gFH+2Z7%9tD4_1GeB|<1M)aT{Ar5sf?se9~sXR`h9sn8>k_6<^bfs^sQNZn!p z#+@!a^p}VuM?Ov>v=PJK4rj$|C@a5mMF7k;=_^UlBo)G=X`&LyCl35)_x-ZYDL#^8 z>#q?Jay2*8n6+Z=!VYXVG8KSVg>VMGG>u(!4v5vWq0so7iW`BDmcpz`ZAMw?Fl~h7~-k2pYB^xs`=lg@R=jL&bh7&q}VIAoX z9J<_pik2l7eR^MW?$pxtikKWA5f*wC-uBMpnR?UqIE8_jPxTHy65o826#FU6!vTX( zoDY%#=amdt&zM4J>lIx%A=h+<(VRBQr^zfCij3g&Cw&WfD2;=LFleCHs;<%19}yEE ze{vD1+6ngY)k&UnI>WW6PZAg(oiXH8&bX-;u%Y`*ek9dIrWHyr+B~9ljd~B;*3zm| z$R-l>m4)SAE=(&~J&WBpF~l?bNqdr{1cuis+~>L=j)qw=Hh!uS%E?=(h!!ePL^wu{ zt-l{R!pAwfO`h-`m-y$QEOm>iyPSb#T4l?1&Ydx2(zx?NkPjqwS zf2B-7K(uB+KXs;Dt0w6JNWXeVKi+=OD}n$3V$RGfPDRhDIS0 zLXklVNeO}w+aQAlOSO~ZfU49&NaKa|7=xg)RHCay3NZqMZRhoEuw<;TS$#ffYZTnHxT?d$dk6tfRj!U4sr15 zQt!^ca3~W8hM`}@<+3icq2+JE-7KpP>az`%xoWtzhZ-SFlERgDnfV6F2}zX{GjMcy zO1ik({Q_#r+1$Aph-*vt(BkN^)t|-xwt$9NjGX8URRbtPaZID=Fc{%Bf3qjJsa>~Kiou}{HSERV zQTf~*@HTFur>8*Ji(`!3^Px+T_+W4lX?U!4%I4uqjccJzD+L+sU|o#x?A?`##In-E zDmcz1HDbqTmO;EG52v8{^qpypGDk+A^ zs2YuE@`4r$GP$jBN|nwVO4Pe6%Epy~nJOy{*m#1S3LJ!qXOxCJ8%dP34J6S-&Y54J z&<^bj%!cF)M&)DWsvC~Mca-2-qrmxiLxUdz7i{bK=ZyU&-y$_+xFY!>NIJ

@$2r!uAF;nv@TGJy)C*kWl!D!^U@}(q2Gtx3M%ri3o{j8O{ zd-huws;#XyNVOIsPdLUp#Jn9nk1(u_mVjQEy?`iZu$@RIDG_dS5ka%mzNm0B{@01a z?{M**rkRkF#*j{In3SMo0cwYYXwiGpYqA6oEVJ^o=%s0EBIQpgS8Jhs6zi4@4&&_= zFPL!j+9Kq|s~rudA6v1K{h(p^g1ru3v6BQ9_$^ai_#su*R}c7ikZW4o(>e=s7FVX0 z6Sbbil=6unzmlP7-I#D&L!O{;As5~OIAg~?42OBEk$I7ivK2Gy6Af&AgsoKkh#Y;T z0hZW#c1fijEOhW(+#|AULue87`f~qKqSFvOlv$P|iEs>wiV{8y9ZXVJAF@M}Q6EZK z%;kbzpt*k#E~kzY?T!R5BDgDU^=I3(Jx_}O+GrCj#YgJ6MD8!}8tHBh;ZX401B82- z^p&C@+6VUZ_jUk^C-Z2PGf}gD=R}AvHY=%2(K}sdA-yyuhi35B&#sPMH6OZ%F>t_T z%W2H_QwT#MzYBg~jU69tsFidPgB+3Q~J=%EtW zHT~T=Atu>V*qD9ZC-yL!a<>uIpiT{VNax5RGkVssc)Pa=R0nUvUa1HHYpL&FxiNFj zpEM3LP{l1G)}arK+TzJ5_(kW1Q=lx-$wx!UAD}4nWk1wn9I7nV&oUT4uL}U-%vtdw zp&w~9Y~DBb3P-BUF~q2)$onTbw5EXsoo<25UL&0mo%103*D82@A0E{}^88)Wl->(< zk7UMrtamC#$i31l93_S%;w}OF)`jsX)ljE5Wc$W-96YBI%#x-xL9H%ukSmjkFNb!T9LO z`zYwulP?|JjDE5aG}Zv;#tgpAfY+@MP&s5z{0=UdfAoHi!=6yqfl`PCXk%B9r239qD;GcRlI zBQY#bHRz@g)6~`8Xm_*1`sC#VU>G)1OQIfyf4iZzpf9{mRRPY|MHH^|g7P2;Ob>wY zq%h2=Q>`GvpFRjFLqcru5=iCvZm1d$4wUVhbxcO`195VXCX3sHlrZl#S38+; zC$gjtG*1_c%1*`ZaAK3zVHnvD6-hZ6GSu>PbGeAysS|UF7USQzCP$->sS_kJsW9RW zWw13G(z@=Rqz{x9NI<6HA!tFrY&aE$Q0ej_mZ>jk+qO%l_$T670c5C(d^llOlgeQh z&C=PfW_~fH*_6E22++3D*7vcyT)2wNcgLkl<~glw16y4s&HGY6*Kd3|8Qs$~W)r;xSb4H~xNlMZ#C*)LNRzW-dcd2gu?Cd# z=`#6Nret#UK-{zO;l;s%s&tQiY=%*K_$r>0>xY?cm|CrIsRu-4sjB)`v^z$`D-@+P zc}7e1-^-uCcsZ!FiPJ2yKy_f5UvMzI>Izg`_!;ha+$!1xD#Fzo*d^!iv0SUPCqni9 z`mvRH5@FRgT-33)dsVq5dIfT?t|n_sL@KhsCqZIE6`(Q;TvoU_-sMMO>3gBplRT7t zw6Kp&+rxo%mdf4K<2NI|8v!Q0VWqc1QR~Gxo@Xq5NQ^B`^act@A9*~IOw#d2B?`I$ zi158A4IFpX8IkzYe?@=~*V(=3x!u9L>9ksU)gcaFYeKl>!8ttEbvqN9Y&9`s30`)r z4&=R9!>NT8^p3&@1Jk-`Kfh|8%tH;jytxCdz0Ct#X&fE zyl#l#@8oo%7yIV1MSMo!-j2P~`vs*|fT^%-fuGp#bngyJ>=>VE&b7nA1mYii=)4l$ z9M&=*k;onwX$%UFiQZ{WtJYkPzrR@~hWp+jzZ(@*_Os>1dme!wlFx4gm?h4f!_v5l z<@8WcoHMo(D21sz@>Oio^U`fD*|u;Q5zR{C1VjuAb~BYQ$GL!A?zCYGLmyV0xuEG< zf>3<@hK4amHmRY7gt=p9rZM`5OY*fas%sOFuh!{4aO+Ks9JO94`q6yxoAr?APxNs;-hN4AIq$m-zdE2ul zjombo3lUAekx0(LXFRNcT%zK{9^)*b6Hk&u%hW^g{{8$q2=R|nA&fD}{neO%N7)!p zdT)O>u`7}??~DQ6!FqX3w`kzj_*s2TbmgCHT}Z-n=fKkvnqMl{d~O4sSQIvMEu`|r zdZxI@u}58hZVcUUJ{epBuNloK@@#mJLq8A4K8Z3}!U-J*>R!R}09E=8qcwrfw9}t) zKCuyTfWb{UPemp9jvjclB_!J^CIS+N5lWv-q9W~k#R?$mIwxLEK(Y;wG&}wG7X&pA zvCSffdq*4x3jjmeiXbX-)~ndjUY&_A|ARjKr+P%gW#f}hY#?bnb3~6eUzXSo@2B! zN$#Tsav@GTPg}g6Mvh+#S78Ij_WSho>YK-_hqF>1SB*Umf9^WYkMYaTv=kZ6q$q}i z!f6%emZu>_UU)u@RC*pSBPco}YgA)i?KIwVo4H!>fwF*wxFJhh&nQ~#BmN5eu?bPy zveDcrIWgPu4@aw~1-got{uLwnG_{4+p%_HI7W+U*NPVNYIaG6;;1%iyHod)^dz5-$ zJGlbJFRrr+4=v-r0m!DT6DZ5tzsvuujfJtW+&*juy4+f4U{OT4+6R+MMA6l!>zKr0 zO}Dwo9>?(2)3=H>olmxnWn|I$usVoh-7V8AUwDB?lONt$4^5YEMT?-W(9R0 z`nT0AfdJUZo^6ewWcH^|6bVM*qmQWGDL}s%JL=bwFjO z)Vo~brM!vaTm`P4B|;pmWg9&`hMu-tl{9FmIl)UIv1OcDFzj5~DWhi^Q)7B4`ff$4 zrGC^#99@fy5i#l@+ zXt2xbq%>nY0@ILlECOd8CVQ2cO;kv8N71RP`f*U`1 zvw_TG6HSO%I`|mbhs&f-r#6W!~T1 zV#(2hS%;kU3hZOX43yb8F4>C5?Vu$2>|A((l8|CW|5Ho;~-_tsVg9ZU4)mAeN*-i{}UHML27JC_>w)$Z(^2@j0B2rV7Yx=8;pC7~rHOVO)M zoM0-cOo?E5&uzu6$>T>Mbz>&BT|ye&tB9kYu8g04bBRsc?BugV-UeSlEuA(z%u!`0w5_uy6wb)UU@# zDY_k0rCl_KpM{;(cZmyG+r^42->uZE zeYX)&9dh#o1wWWprTGQCSIK-S;hb{kW5e$^0xVSw z?jSF2++fy2`t#ZpOziw^3pahUM-|J~NQsBa-$|^o8W-n_sT>4bIZf=tH?+n;p9V*pE6BDiaCfeZsXL@$Kbu^rCob@e1H7yR z^G22EMLAupeY#3>>XNNZCj{u&QE;|l@r@p5hCJA_x8-p*0-SUn*v~ptzGEm}bU!de3C-P!_g2})8fuE@& zk0A5H)-ItTy4km&Zy%de2*q>Z`$gE&FszF0;IO0fHEIFH1Y<;+UK_q$?Ugf!quPz+llknVoi)WO#f5PLM&`4!lS zJ!$bra;2|0k3mSSxLDcRwz%cOYjO;?X)lrTdgvhHSDp6~pztki0BO1#NaploI_OEJ z?SOJ}4A06p5^q}t$=U3hbvW3BZRhSQI6SG?-tjb6wq*^V+VLwIKt|i)-#7qWEVjrc zz9Cs{GBg}8^QS0nZ-F72QCex=K%C#Wd++oZSw?dX9TD_{dZ5?2luleTa}s>b{N38B zdS3-YEY;Hj;lFrcDB0X4YkWb%zadZWP3#@~Gz*^05hdMR#TQG|A5Gsi^CcviV^1{^ zki**o72bfI-d8TUIuaZ3cY@~=*7HHR3HxO=p1Sf6?o&YaoBvm*s_Akn(T2dKcSz|< z-1xQf4zu;CK?+2?(@n&iEoCH^ktW`0FWN6^%nH+s%tQNI{!G8=(b?k(aGT z1AIrnBwnjmqa3Zm<}^NhZIQchOhY7?)ux++!(LOPlb?P-z3*{ODcOxUe0Ml5+p7z>JYAB z)7b?PR+;whE|!Z=^sFS?-coSJSP?w&N${)MvB+DGRxQqnT-p`Qk@)H;w_0L6>B;cB zUKMaJ7N}PJO7QuRVOsbj!qgb&W2t;=Lcxd3kYR(o7<|BHGRY&g8(u{YW-3jpaWiQ2 zw}}IP$ZA*zWAYIWW3n;j2!<>El5u&RwhW{}W&Z_-dg$*0P|J195;y7jQqsPcZBdgzyVTnaP}b=9?iXT zR#x1IIB^2fFXIfrkyI0wWOR8JeTK3(M*_y#A(k`oX_Tx}UMKiOY&@5+-8kQ3*xMm? z9#z2@^&vQ)Fm|81m@^BzmLo3zUB*#Lm=|cKN-mIoZ-ixvaLGo91%s+YL#~$|HW#%2 zsDqvYJE+PiXeY=#Nhzn=PPC=!Ecj;YGq)KEDiw`&BJG`?Q><(eYk{P_!y=eIz^^w5 zp4bqmog-{Z+oPL^hYC|NrxV^s0GMAXRlc*ENpv{}aEsmjZ#QPMJA*v6-N~<2nH(!m z4#=X~41`I_9P=!fG)3>9XN3$wP#jZ|+J^F!jDZ>AaRvK9d3dEc+h5p6@K(L{o7L42 zpNz*#f+|(_^ijBYL6)3c08B$aJI_n^09z)-9sm;)U(wa^%{EpgsR)mK@87Sp4kK{j zzkP`_sOYmDLciehZMU2*i>-)fC>>(yNCPd2eY0zF0MF_wt$OSxeBH_g;9ex)!<50% ztkR<~L!m82n9;g|eAo%aOg>2A<5gZX;d`)Dwa+6qFg1)rEcktWnXBNiWEgyZvf$Sin?hlsCS z1jGIychbfO%Q8)_cB@0Y*E_0iP?Ye)@w!)55}g2s@{LfarCqhpp;Cy4-Vj4d3=6jysmTTruP&`!?nA|<3-QUM$X#IAe$Z~|+FjcXOU=cGzDW>Wcj zX2R7S23(@Z)0_4)4#`u2JbRznM+(1IrqVZ0Bd!Ei9QHZopJS;YhGbn{(0Z>dZo`_z zy!{Y;ckWnmjzSdu(4=?tSwW7X0{xOzYwt>7goAh=NK<#DGV1~rtyidAu#-*umW>cE zBj&H=C4Z~X^6$Q0xRno2Ucts9D{bs9!9G2)>($)S(-^N*-Q1DmDADzML?P}Tt#kCj zNGqS=R;P@CzG6>RbjA|{x0F8Na7NSyW;@`&XS1Bd$?z~+=$zG*i47X%v7MiJ; zZVL~Xz49g+UGeog(KD8Jvw>}nQ>XBe3;Aad=fV^BiUIe5fO6-nGWRXa`2n2*z&sBf z7#89aljD+;dquB+fvn9@2U}P64?V{jiQ1OZ z-_wlioJIZ-jwm#mgD8ieUWuDjOOrlvOsH8)5Nk=sSxEn>`Z@RC>yb01W6idmFln_h zlNv;iMA2Hq`DR#*L79xWrIYkDKdN`fVhOD%zDSAtW0&_U*vnDo&*bcG_uc_=#RBr< zDdndDQnT$oHDv`u-lzc>?S2Fo$IFWl;KACY?qhfT7o+;|Je_KJO}54XytMuK#oPau zX~s|tq>%WDmB~bX7jb!_`xn$7BMVG`%3_0)#Gm+*K`5aJ(PG@P$XWK$oatuJM~sTG z5Pl+B@~E4Q!np`EVwD>u;v%L;wK~O*2Q8`{F}oUdk0ljZinxbcEFvwT8}*%Iyq|*( z(H`t-4$~B9QtLZOiji#a9^9B;@ovZgw*4G=JjazYLs$xJYnY#kE_UgA>DBKSf7W5S ze)WUXB`Tbl;9#9vM9}2U(2XdMSm;Kc%f_x?$y`mWGyAN~I{aHRMQGAZckw(V@*{pMwE=earg8ShaYGK@M`tCLMiUNXhD|_iA&$JHj6yjC zRFwrk?6W!J(4oK*QB6sKP8)Y=LDh_sbV{iqd1elxm!LSHsgIrzId?&{N9lAbh81bA zj2Ki4l4cCSnt=B*&V_uceM@gxYQs`*k=Bcm?Hh@K^v6pL0!xCV?CVQ{{1YQ90^>Lg zRJZ5OYcLxM@+I0_+_J>4GWzU;c66XeXOhjfREoszdNz)Td2Ii81$7yyT;iO87-H9~ z1&=@W2mOBcYVugse)|@lbY8J_D{6l-0m|SY?{G_k*RBF`%kOKGV|lqU%CYK?4!OCl zY+YP8#g|1&oUe)91G-pcELw3r6eOXO^y?@JgNvOjZ}abV z`x_~|HhWi0Dc4qC9&W4L>DZrlBe;$U&XRnezB6}x!gda#LR;U?&!=TmZ^lx*P~1u` zLB$U3#SX8<4k<|P6iDtENbVf|&&`O~A-~vRACoGd3|TU2eBizx{5g=b*L1ouFP}pSopHQz{olgn8jUE@@0BENa^v z6;1dNF@WF5|La*F4T}$qrsZ%6^Z?;sjy}4)9c0CKTnsBa+-88n2 zv^@-=MxJ&#F|(xB44&GR)UTpq(;j%HEVseZ2GCFxI$uLo89tZD^K;p?>Fmf})|ref zdjMLx$b1Fih;;EgZopq42KBJP=CmRJqSmkrmb!;wbvV65{~{Dx@gMNgV+XAME2(&z5VI%pScEV( zq(@(FbDLRWajxY+qqG@u?)Jc&Ta-8yp~tmtZStzf2&HYi>y(%tY10wdE0Q0U-%{Ht zksrC;_J6zZWFR{x33XwM;a zZ6>v%5d6G_cV3bY!{6$?psEW5$~G|k$5J$r>WT<5P9eb_e4$~90T zK!FNNbOh^=cvz`}xfdBlyB)#~hVW%cQpcqfOS(ytcDjU~#EcedlRV6hoESGuKnEUf z#l{o~A4k~D3T*Kasw9-^(SurSLW#)%6(IJ{qDIu1uB@CrCNI@rtO1y3(N^ejO;=oC zi$!WdV`0j2OiVz3m8ZxCbw$2Cw~&u#!PN>*mvg#9{x~-VpINT4xB!yj+Rw>0P*sWE zw-*J0NGp~4(@K7cFdsGY%}+Awg4f8nnB~x}rn$u|XM$);<>mot-{V8O>*Cn$8F1QY z#~;eyehMDsTx7RT1s@?P z{m?FQa=18A@=TWt+w`+NHno>UCWB5uVwEFV&w+x#AsAtFdv1$=L4OtFg$BQ`@zqj$TOLDJD4R@vbgQhg`VSYQT4L*UHkvc9Jl+co9OO&WJ z(t5;K8iRn?-3zXF>xc|gaaih_`uExwJZdYZ2}x$}Z;A(!+!q7C4s#S1yynL$>qEref^6+bB$9k*8{ddayt4H(Gdv=FpHlYF zE#>#xZiejOVR`ovu#+DL483thT|l`5N4uI`zyCoKX54M{~WDA}~ZyF%K0qE#e`Q#KbE zvq}HQk}|g1t==5emNM9WTJ=?dZ~b=efOqFt5B-Z{zH4Ybg=u11toCC8lXYI_B8G_V z7=@I^2_y64u`wpQ^}${&wkT``TB6=jTo+Im<*6s*?;1)W%r4Hyu~_{_CjGg~YLk29X>K0&Y6tJm+Lv-_-~~6=Uea z$!7ils3b8k94RoIe|I@B91$=a$dPkYgQy(XHcx`d;(FCwNfa*b5T+;O{zVBW6+&WJ ziy$US+Mi+-N2e$eEY6=Te^k6mhs#wURtma%a&p$;Sx#z?zE!sQ!EYMnh!7GQ#|2oS zQ_4RwxzOS7bhoDseHFRF9ci#5rzOLyrNH^aea`+|B8D`7dr*LKlbILyYs#1l`yWkQYgJ%H% zyobY)P=)X#7}ngx`VQ)Rb3Ab8Vm$ZYe!6hwF`VRj%CLsPH5*Zy;NWO&OP?uYAMfid zU7X zjUi*##X>)|-O)+xB}0Y)_R(_okqF$KeK7ggX=$irb&4?Kmn>Fgjlb*{9aeF!h+`8< z@#%?hoiD-U(hr2``#)y?6(ZH^aY@c3c=oh(8`XRth zvDp&N1RADlY#=F~dG`VXv-Qnx5Pz&AOT59k&>-&ynkmt>qnYAKd)I0ki~!&?AFx#| zoSTyOl<&jg|2X71E)-SZ%Ph51l*9`V|Ln?o^&*$rQJYoh$pAUAGK}-}i2^GQ%gp*J zq>55%+Nyjer6TJQyBeADTU-w5NIuVi+C6vXKpxy`G~o%E3zBd}zW7(p&mn)J%jV}4 z>}WK}e9D+^^Er9E1&+#AQD(?Zu_m-mQ)L)c`s(&!T0LRM9fAN9v~H~e;g@z6NLlYe z>gjo}Qe7)4bWo8AQ@*ixnnmKNd2spg-T9K1T0W7(640XDer(jz3bn5#`d+n^S286 zMRe{g)QWo=^KLGELIy`PYtTjpl~1YuA~UQq4%M{}Z_Q1B&o(h^au*yMH%Vh?rgris zIhxu?uKfEOjb~l+PvUR52j;%iWBL5WO6qopCtHo@7(-X#Lp{5IRWbfQ$kO&fsa85@ z!kx2$RhW*PYc(&j8tWW&UWBCq>B}qlnu(@H%@d?=c*CzjzMg-+dJwZtpFo2X^_0mX z&w6jluM%#&xyz3h%1r-m>qcUtRsVehQayDeNY2{t1U~m3!2*3o4fpBQ!a41vd@lE1 z9vI&iHKjZ^2VGvH?wIYQ^#>RRe#0gc2op=B3&(xtw{)nWz`bFWK9Hy}xQ(OhI_fKD z+5FamhL1}gCiXv*(qy9BIC8lP{MKLLIBkpV%&n8)y_8x7A0jWeKt#Ji@`_X^|4aF& zA*BT8kQC0M1k^&+N)i-bat!VaIi!&lE*y|l78+(~@PT8(2RyJT08vadE7-EK>JS1DbQ)^+utI>iw^NAYSk-)eH$$S{tAG59cD7{DA5^uwR`WB!Y)MYZuy8xLtLu^g}Zw8-q z%LG^&XX1e0c?W7v6UFe+ApL~6v?zH?^%&m0=Y!84L!^xe zx)Gw!+K~vlaYo#na>6}kE(_}6Sz15gAPdl{_j}&MkN8tV`jbNnL2_g9vVePzs=lfr z2+5fl1>i}q%`jauh8|G8PbZSvC-4%KOekrRNgQ}{L(m>+cE)uxrXJkVq=#~jdT$a@ z?&&$QTql9QmS#mLjG^9=Tlz_0?E4)`y_RK#1ICO#wWd)*vgl*tOTwJFRZPaXjgqn` zhbI;W$=U=blYE&0DG?2(ZJKg6R!71PMU_YwhbVRW>Jjwy(rjavcM+AyH_994_3D?Y zPy0M=s-I-37VWy#E(x@a#P`-Vg}UW(akNd|9)DZ|J~LetbgQNkF)d{d#nx%w+^&=@ zJLL5%w}==QIL4c_13QJi1og|V;x#O8My(nZO0RG|vY)#;g?Om!3h9o`wh~%S_1pAk z5^$d!7S)OD7D~o8o=DdIrjGJF`gf|-Bt92gJt8dt(<-2E2CnHS#Izn{}^Hz>Y5mkEs zHxC_OGHQP0{gwqcig}2AFV?2qO~zPob8$p_Fa8$Nsn$*X@2B~Q;|)Q`GgZ8cdMDv0 zZ=Leq_^oV{Q6N=doDojX*TOjtfa`MzT+s75JQ}xyZTycX;r>>jL?pl-aVTWm{5h3I ziHL2Ab^s2~Hq8;_5S@qHqPSr;z|@QdJ^U52@tcM2fVm+1E?{o^wg2?Dw}MR%cD5W7 zZ?Ty(Yf&DE{H0VG){xoW>$G$50Z+LPqwntNgfq8BtnNmB|0Kfj%zh8eZjH}Q1W&KWE$B6x-6yOhBgYtNY_sMyAUBV~D7dd7g7X-GNAD+2 zy6IWKRbKdA==8HBE6ZaX-(r{m+Z3rDuEAfN^1ffEVj@%SB3B={HP~{9>VkGK;a-6j zkHn5oq=u)6+0n*-dzg`wqlDs2;XbVd;9 zeL{CZajVKAqgtLN3@1Z1eNpx#mS1dLJ-Jv^nW}*3SbnSlp8U%qT6^F5dkr#Nmu+X#F|AqdVz$(UTGnp*w?3ktTi+eI_hp_3 z?ocpKqduZJTSMgX0AxJD*+C27u(MX_A60FQCF9MO>{wAw4;ed{C>ncqi3_c}A+|deBVt#}66hxOrRL&2?8#CcY#2biTJK*x%Dc|t$J>E3d{=nV0 znNJX>d$0oJc8VIX?x1RoG1ZU9;Mh}p8m)SR+;XdjOH>EE(C(*@q4P6jlN|z zJpsgCB=7=dlGJ!sUhWSdsDp3o2WdDFE>hMlcM3bz-iiV3KRds0USmynf z#@JDqlSx+E8pNn#7zKQC^3SqrbNy1;#)FGtaZY97b-Cfqd? z^p-km9d#3v?i$MVipKW$8meV9L|h@0`bx7KCTB?hRbR}rNMA|}5|6~=MX@5+{PtE; zU9+}OmHIyudJ8=qq9Z>u0Kbb^;PNM_zjnGf<{Cpw$xs(Rd39lVUVC%+W&c-4x2~bP zxk@IL`bJZOj$wb%R=F|#ktat$DWYoxg$qLAaK%Mo$kd`oF@C<|^j5z4z6n7(Bvy9r+#T39D^iijuHZTY6b; z=YQMxo_+G{9bC+HV9!oT5_Mb4$c3lG9W<#XmouuDubZE%^0?}|t6{{2C$cc*Q&i`+mk6pFMj3`chJ~Mr(_X zFj#ELhJ{n;s~VY**GQJMTVWDKwT%r~io+V0`&DI8%li7A55t_Bg0a)E1q=GaWWE@| zFY1)+k$-!FMBHtNdniR15NO9PzU zyn=E%8djhoa<3w}+cb=z=Gdq*IG80}gY%aH>r&G*-QG@d*HGak8bj6a8&l=VxMy+B z9zhtcJh*rO*_R@9f+Y*yQ=Dcq#=-wre0 z&+pML#ms^5VG@4`;9&Lsb`N;<6I-Hd6I||=u7&@uadJtJ?*|yOVZpnEwC5iI>y|LF z&S~H2wH-3e54#kk6_QTu@q_cNni~or)N-IitWV5Ow>5=(3$_#1+heV;4uyzxtdq)$ zz<15L^f6IJO7q{JEHAI)+cn-|;N@kDKnhn)YWy;2mYUF^-;CmKp7mCft$j?~)J9Y- zBW>477f1Pe zECEKPG^#_kJ_eSRI$MRh8m514HRo+YhLdlx*N)FzOP_$F0%asNnQ?LF3wwiVJytgx zxBB@v7!gWM__f`>RPCJKBoA<2XiOTY>c(rjSEp$j*o9rRp5K8Qo&2rZ-sPgjg~zLT zVogu#;?m0A&at>w`hulYN!)A>e=_0d=7vLHHLsJl+`Bz6m|~UfBlD|@bm7@Ly)AW( zY7hiQLCYQIq*aYOu+f%987H^tHmk+KLxKqY*FC6eg|3sqnoP!XV+%h<`e=sitiI z4IBf)ME#;@@|1u0DScnFlDO@DVh3D$VKLaENpQd4A(us2nPoWLET`kIKDPIgn4u^R~aDLl=L@>_YYR0|4J9et&ewz*chB>Dz6 zmn+O@>ai%~pFA~LJj<9OX{O=sgvRS?o5XgHlnh3fPMolc#bzlo@HQ6ql}t_l03STq zu<>>V`eVCI^bX;8s+s7; ztz_)HAXQ3kzlF|P)|AUrWZDcm+mkeZh=$eFXLbCh1h|MuFLBhM&d+Hzos=UcP z)xXR9TjdI@-7u3c#b^!}&jRz5JLsBp*vI7T?cs5@I*g(^n77A;$jS$VVaa|t#&agO zqr$_;Fx!LNMvHV~an~1Vt_Pt#*!XEhV+rQP6fYYVeO-E|tsJwQZ&laa5wy;KEs@PX zrJN(1NL?Y{shqdSmB(tHYj+Ij&|hO|pEsy3rr0(u5@4h4@@*FzMe)-)*mxANxZXb; z9gop-71eccZsKUqF&K>KT85E?%gE-H$$h^uholYzB~T-R8W065=&L=Q!0W5Jj9Ha4 zct|ZBX_m;kjXTJ5Z&OopnTDvC{W48axtA_q*$-Cb&mVPBP3??2chwhj8H=RKZCFc) z&(9%l*0bJnKqiJq~7#x5{IuvGmEoV?nbQB}I{OW7%mYA(ieenHT4 z*}`*p&7SWOiW-hK+&2>q?ZT}$*AJp0*Lg{1J{k9sEoG}OT|5uIVU<>NZKNgJGlNiB z-l&b<#tq+GTZze^kMXa$P2QxABg7u?BgVTpWH+yWU@&N_p;Mk65NDUa&bc}K?(Z@? zb?INU*HjND6Q{tN^oM4zQ^0sTlix38O@WKa=SZ948+PVM`UA6TeY>qGgY1drL>Nfh z%9d>*zc`FAZ%jL+lsbl$Ev(KAnAe{DpVQ(mzX&>0X(tW2vU>xqb$Yc3^Od}_=us)L z-F=@ApX{ODQ9=hzfO-5w4?Hd(Z__UCM|p~j!ERpHK-~{jH(%}h@4ixTe|~?yenF22 zxVGKJj>Pux@4dDpLVJBGn}2e}0G?2k@o}F?iO}z?HxJWovUoN^^=vvBxEb}6bx_+6 zB~TC;q6xhv`uh~f@O04yUUPy1b7i?&fBD{aK||)sbGaLS?n0sEdxsKcQic2_bTQUnq<&OpaCFj3} zG#N7MRn&?3>jq8qVsgZ3vWw9Gntv+m z1BLYt+*X3otNzN;Tlm7V_B&CQfY)KP%Pnkqre6G=)5}?ZcaCesgVA1kj*GHqmzUp3 zJLji3_s6%r7n|RIhG)MZaQ^@ea={EavLO2vptn&H^$F}<)os7*;oUWv zn!M-L-DVVL?W*HLuwz?&JF71|0NS&+al^frn|AXv8MqAaDD@14EaYTX!yZ-gJuEjt zD@B_;3D#cBHfK~YXof!Kp^z-{+%e7J|1tm)m}`ZBX$$^x{ukmru?L(dsXeDHu`Mw|z{fK}O6f0!(h?vnxto;1$*QNP zmDMX%QW*cq2qC9|q+NoGrrB8@5F$^o31S{JFUXkenh|H0<@bQ^bBw3&pN#ndPqBPv zmht(*no4bV288>cKa@E$J3oDFdw)F^9jgR`Y0COHK-UHVouC*{12d5H!hlqYMr#MS z(ADhOIL&(V+|ojxB4D0cVxAg7{zh*(1YAscq4I$UwzITpW0Vu$%pSuith}I?~Hp~ zdR1!DrhRpbCvuKg2yr)r=MsKV3acxTeZ^&oY}$88dHnCezJpNxjx-{)Ch@qSCrk~X z1~G);koRqcttkh)f>wpXvV&Gd!jOe8hpb(IoR$oe%5m!?w+@-j+F%{xVe3YNxdwu{ z#-fK{|DCCrQew1xCVpS+PA0qr%lo9OsM}k2UTll6bOly>QR=-gv|n#IZ@JC7+!S4G zORV_#*Li@zpObM^ym%QoM_#n+=&n>b=(*+L7GOHtc7Tx%^w(tXuMUo5jb8$PHqS(1 z*H3E1^|9U~YJPn1t86q*=1#T42~>Qq>ZVs^W-^|w^R(gt{#EEG;#CIA+J!33XbR92{{V|r*r)X!- z>A9-()dhEkqxa@)Lg z8h4Y%1n!XJHdQFGlRf(ePNvz^(`JuQn<-`1nJIg$FN`9yQDB#c6$7PV1<24&KAh9& zWAVwZOlSZ-(WHOlgp}0`C}U#V?}!G!BcxQ^ByJJzSw7)3m#&OmD$Eu8{^>eT4J!v2 z_PoQ#l{wgXYC0@YKYA`XR9^f-?-|Z zj91Vb=SVQPz>UOVEqS2~Jjrzus#kVJT$x#**lk!=+jzbv~#|N|x+b7|WDw;-#M#qPE8pBm{3>#)L6YxaIHsWfvPLB=~ zAJA+X$o3%{(_C{D?*SXPhhC#HY*FgAE3`WIj1+xjTmC8L0rj!Wb3G;t^C9$j6!pL;e0G4l<_5rI#IO=3H zndHH<^hVqfKOtGRD)cMWLtpKYRIH@cG2T56kXSytoc?AqruV zsa?tX@$Vp%arRuKxG^CIcIfX0Ts|G+MA*hL@b$xW(@=GPZ&E-gjpAb@`0mx9M)N$2gSixhMF`1wj$rHQR#L3X?@Cvu@U=dFpTYciCjUzu|sU z1(RyRVSNQfcPPnif~NeTss|5``!mENuK^<}=Izo`&dK8rNz!{ocPlX^)lz@AktMp$ zw7Y1R=vAf#hBZnzG|D!_3bA$!g7gMe4|E8+iaR{z+@3PdPtk8r)lvm(yzrfpM<&&n z{w}ZQO1a12$U>-O_k{n#j+$++iD|h|B~$q%So$Jf`eH8ghHo35Q8f6(6C1Xx9eoWP z89u6B|EEY@Tuw5@=h_16n1V{0T2iN==Q@|E$PZ@S$O}fLiRN2?p?&2JTfFr<^SG>V zN_v!qne=|)MLFA)ngys@y?~-k==_(8@$|6Y-M3B5VSt4$Q#P!8uRZHRgJOBdKh4^^ z<*7l}=9)!ESuGsHn)j%s6$HkDgI#eFf8vOq4I zN1yZ6WE_#uFb10|y688Jf3a5FX04xn-4}@T(Md#T39Y9R4d_OMA3NgI9{1@Paf4E8@upk=>S`W5XqETnK}yR2G|8@7A0y2{w4fVm z?Zj7{5lv;(HoAEluA>0dI-bHaF7McFB)NI5G>xeC+Wv5r-yO$b-#Jfrbhez!H|_~_ zpw-|86idMPv^IY*@>Ep7PrDIFwRPfix4}s8^@K=u%t4mE!93Xsk>Wv*>cJY0U=)Ha zT-TQ9I~N1vgqV_+UCQ3kdhz6bZTZAd?1`z;6-W(TvFN;1#+ib%BsMl8m-o}PD~6&+>hAC?C(a^e0Z;*IeP3By!~vq0V6Y>iK!O zq+!c5Gt7N@k!Uu-7Ec_fDRV_j{@9`8k-EiCvc%&Os1q?=8~t0Xvbo^JQ(;S033jp~ zTa}Rzp!i;9qwoi@uSDIi#J9K2Almze9Jv%gx?K0wW+m=E$u2tBBqLK!!1O8O5B zMP`j>_u&Xi$>$Et^9fgxZ7TO7{(mkZ9&^sh`u#)iKJooG-pkR{$m*pyM()Ja-Y zScIK}!QRo%-qg|A!qlls$0in+GvPZ6{8<2LG}OgEe44e{m{M>@nAF0EoOQmEjPlO# zk5ENs!n!Rx&(t;ch3^-in)jK;+O%PIZ^~#niPnu;VfWYa*W1U=soxzMhE~GWRLijm zotqTJko7%NN5@X*&vB-m=)ug*zNoG z-pkwZ)qr)Ab~WAJM8!nLE{C7H^!Pv^9gM;)KixQ)>=1+Oe0iL(sjzQ?lA5$otr2d@ zTwc@U{d;QYm?g*duNZG^^{d+OASNl5~ZEFL*~Bjyj^`PU&)K`g|* zyd{3X5Hfrd@Kf%YE@L^H9!BL}-}{Uz*)C8>&^j~+IAkS(X`RfH3X|Nk6!ex0Xm){h zLP3^!5#8+-XM#QxCu;s6+^b!&D6}aG-FAi`sMT2tuHY%Xp6R*;&BbpO&IwR?(N+X5 zgbDk6mblV2?jfwGKKSM}u)y$CEl&_-2-tparm_fOGxEGu;(c7;{-gz!P;vuWy9s6x zZ@a6}KVt&4>0vy`CnCh%SVS6(M@tTmen#BNZ zYpi4plgEcu)MWLM%At<1&_SdhI5u2m$X^4Rkgvbi6ik!wAk#G!e9fF_F%d%4rX?ct zs;U!>|~Bs_4<9BAnK*P|mq5nt{uW8&lm2lgm8 zGRL>M1^8l4~GD9iJK&?eGP$xJ~>rZvBgR zg+OJVMKh?WWpGWOB&MLxO{!D7*xE{AyXHhWJ})j3KX~Mj59HeF2@iBN<2hW+%Sgw> zg=9szJlH;KBEzGpb;LGlJ2Ul_*nCZ)tBG&G=nzV*ocytp*BWg0R+Yu5)%CIIp5veT zYr);Dfze$E;8OPJ5z>B<#=(!*JwNziCnuMG0{ev7B@u!Gmba}w2shm^O@TK5q(~FT zyjE=v0AG-iKz1qp8<+bc*!cG)8PL0Ro{`lM?>GqYaP(#_z$BZxMGh8imBWSGCJ9bv zr|i{ZRc7SIl8#y?N_LyhK^Mc-A0X6)iPD1)5hmIqV~OTdunY}dgg%dnX7%3{!#O@p zpgMR-0e3!cmlm{`AFn}@V3NcA>t9O_C2@dHGEm>@7a@VB8z8NtglbP<`BfOPTGX5) z0Ys9fly&JkNF3`d`#7g+0KuX`BUZGhvb$CyX+&D8kb!N0Mlyo&>kY8h$3wAJ9|>cA z%U)^$qH(9OB7%Moos4*ru!5cqFC}PQPLXqWo5C6cnxmozkyNPDLi5fQAP4y%vuj(MwaphKn@I^V60)NHnmZTj3H4&KI6RGxwe(&Ttdu7kTKABS?dO!f*d4MZoc4*|B<8C`{0 z8V}Lz85e)ZG&C!i7vF?pvdp5-Ei?0Gr#22k8mGG0m5adr3^Eu$T64&*yfS&Ktq}j| zS(jF27t&R6wiH2SnONXrf)`f^W9e?1Mo1vUWXVBBdT9>jfSD`)AwzJRvwjKXs4XkP zrP1H9LF*Yy@teZrISnnMA!npkvNMGpOE>G@F=?!e26fa>v;p`_EIgR-m`NvfH0qGG zjcN?BzIX`DmJXbUDyG=}`tP74&=7*F(qD1TpA3d|+{3@40Y9M_Jgo(ZuMGjZuFyd^ z6~<}K7s-WqK1twM)y|n{az#vjd&Od!$)Pa_o8#76bZ$K3H`!fS4MamqynBkR%bKZi zBgpAIY*_*c_s!bC!*YoIO$!U$Ak&n;XB^$C%A+pPcfeJn+F^BY9OH-}SkqOD3e2M^ z%5k*!QNQHVcb%h=rjw!Jy^OwW*HB{Jg*Fz}Hap^j5wmLKIbW@>7IhSRzYb$tyR$_M zI4>DM9%2)XQ0no3=Wq2FliYF7GeNou`Q2z+WrW@~)cd_Ie{ZHbrAf;%9)$4Y09=Wy zkEBdBA!MWbd81Gk6OM@Kh8PzvpfMeb8&8ipkLt3Be4b9q(gg>M79=NO+DZ&i2d?E{ z`o{2Xk|R$B8yBsA^@((XRwrzDl##0~o{-F38&mq+brH^Q+xYy7h*qxxbe6G3CJi}O zCStdC)2bRZ0U#?|iHda*1ljBz(B48Vbh#}`lx>%uPC;bC~h;qCmY!TdcG?C5=0Kjyw3r#DN^u{{WSeV3}#@Jyrvchfop>G4R4u~=BxW6d`N0qeErmv zD)9Oik&K2mWTqxi3K6onGR{vFe}{uiD74I^N@M#%AIqXH4_nk6ME{@+AQ>o{@wYv0 zP>a7$HHcO;LxqimLk!kZ5+RH=ocz^1++8Z7K&oh^Ql`{M#i+T)ZA!&0UMGxQqP1cq z328hK-cFAO$#gGSOl^}Oo7%u!8_0@p^PY{#6;IY4oq}BFz$WTdgz3#{pv&&vN;i+H z^upY8>{dZq?5GvgoV!g4n@@I}`C%XHCwnkzTwAhV2yoQ0xnZPA_-pGat14Cm6RJKz60s72L?+~8|Q^v!bId)H1#FS+Tt)hWQ$_~&GAGv z?V9z~Xj4M%%VQOWMG~g=yU)P%7cP zUMA}^kg39Dp1&OPZ^t$*Y~ffHBXYyN&?#5MGj(KOf|&1^&H~A$fWpGW_(Cs@Q+4F< zgbeR|uCS~uC&!0Vmf|oIixE+!rPit*q^U)LEYpzhIUnpG8gETI6tQw1a5M&?mE;lT zdq|``Wp$ug=OgH8!&j1DOVB1x!+fJE$*l*hR7T~@Q1DQBY0&f1jx3^ zZ-;w6S}KY^;83Cf`~g8)H%}OjG8cdy-*m;o8l7;vxU(F(+c7=lk=IEar2yk9v_W%l z6po`wEcRquiEhH$ijG`Vs)$E7{wt~YnRXl!Dv_Wkc9hL4Eb69vvd0=!xdw?s$o+`3 zpfwiu4)T{-N~Ef*)+3Yli-Gc|OI+4nVz1~Z)B(y8VtI|@N$AblrQ@$o3yD;r(<`gJ z2Y(Z!g*cKl^xm26fDxnC2$zI&G$@Ygj`{MlCRf%)k|Um`vY7HeW|u;@`SD&IbeNS> z1yb54-8iQc?%4(!R}=j$njYS6R+6b3;!Cp<#cqRUt%XZo$_8R#arf_*mY2%*BpomJ zm4-eHGIknvMH|jOJ#B@AdO`8a{C;sch!70|vQ#7Od3D&U)DvQs-o!RLM(>r8b=H163_xya z+a{Y6?q=;|ij%dTA+DBnq&2{Y7(isIuYzUsjPAo%L@$at0Few45qR0?>nLZM*w*() zeX6uFl|kV*;Bd0!@i5S>i;vrm3c9az!afAyZ>MC8p;Bn9a6|Z5uwTv{50;lo>A`xO zRjYR9CA)Q9X1WrI*fiN*BdQY*k|XZL`JpstYS}^;MNC;AcBey|7;DO(loPG>MtqY$i#h0VX~eV`~`k0JKb$w#U-AlpYWTp$y?O>6x}b@zy_uV#wTFP2>K zT2k~|cICa{%aCsm2+(9^mV5KEfMVp0Lz7hs6T;bPU+_Wz>+Q!Kqu~22fmwEF zJf2{l30Jc3luaK0c%@vZPW!BHDWrO~BVfbK!8tvCv;?D*-;{<2|7o-v7UuOR%4(Ah zh&uD3@=z3}T;rXs2D~HD9x(`bg`kq^CuP(os$tX1tM0n&4`owKzi!OEu-Gf!CLiHJtCc3&ykv!Ck$q#;d@Kj3U&!f9mI(`HZI5bVf%2PM75J2C zky1x(ch{%+hJAuhF${Me0sE$ul+qyF+H_({_I8NYH(8Sox5`eFlu@=RvZv#tGCV`n z;V1!$lhQl7NrH8WtrCA>j^soHjFKUa$|{U_-A1zuYR?f`KLU1f_0$C=dI zOC1rO)yXhx^pA8GX*|V@><0;$bV<~^uQjLk`+*fdjgGAc$)ZJpsgdA|lBqn78J;DO7lTs~d6&o-YcN2luG`RIh zO_yXgSL1`{ZMZI%RRaxwf&d}3E!6CTfR54!9+>RQ#BF;g275xQQONoLZZ9y|`z*kg z%{i*Y{wMulw$ImLgjlRnw&V6fUPV=FIB%T(Saxk2{!J!-neLv~DfqF0%?C#MldJ8h zlOZ3{UH3{bm_-Z+Ow2N~TAaf%w$fhodi;=h*>33#Y70x*!20{;>gA_S;JNAW9ni8V z$N~6CJS=fT6!jLlG0I&aM4LOtr?+Ht%*T(D0lAwA>QG7jW09q$QdA<(ad9=EPuzNH z`#cZsiA=Y>wGHD=v#44}WO~?L+*5r)()n2dB>?s!b|1F%Qfx+hNE%>7c+D;lX#aQX zjlGpc(O4x{d4s|jbB`sh?P5NF;djS_M&JDF%Bi4&x7VbW@x{Tvcmb3fL7U82REfmw zWg_d+Z|pQobsw6VD`O<6Ez& z{Iamuz`C8j&)h&nF;38E9MZ%GmPWVINOdzdUi)9h3~<#0VqwhdA}=t-QT(gz`Ag7l zS1`T6nNv8mto%dV>%MFxxAoYPP_+H?`QCIhotyR!SWoa8;|l6T%N*b?5{ixSBFR%J zko#0RqnTCkP8f`Y0S#GPwLoQ3u2!rDzOh)*cc7XNM%yi~WrkDSa&0)tufcXuXQ`MQ zVF`;bDxKZ`9qS4B6s%#2o7gpQK_t)5F>?Q5ZLeGWU%8`Gm_ z;9}uc)eNHM=SL646>M|;TWl>r0Cr$p7!QbyN1F$9MGbnYf6HcPb(C&`-B1}77}9gB zg9xjc#}qF>H&jEBc?TH*&EOowwaJsFn!XU0>p}TCmW5TI2AosuF3xcy_CQgtu2Rz^ z7JQ=T0iCxqFh27O!2iNd%O#bfhMc?80R6h(?#Sop@UQeBZS_RWqmMl#XF~e=>*h!& z;eCT}VL#N;h==1IB{Okt?T$?591AGbZ+u7^Jv297(=9j9?m{u7_(uO$NR%P9h#lwI zX+4^R%8(9WdoGtd6)r+~LAB?i z14lZ7Qd&9p%!zctioC(ac#l|0_!pP{>L}XAenM47KJ?c;PmVKg;r9J%YEg;=AUz}N zEAgM^M(1#@6P_v59wG?0PHBGYl6^9}(zh$@oyOi-KQZa|0AvFcbL*kVUB%t%UB+;W zAIC35WBpnQS6f?MUl-h79;kOhKACep!Io{df{${g-j3ZhFKK@@_|(*ficGd+MIM=5 zhHu1JRmhzIrQUE{yg6eKJ*lc8&n+a-71E|@zC|R-$1lb{;hle3wrN2G=i357a)=^T z3l2nWQ`Cyv&xrZbbhK7+JpC&)*qQ1QcxRIx~Md1E3uWK7U&h`-<7DJ19W6Q~XceZ0;)QyP0@ zu9VWJhZi=E4p$7K>PezhOwVS)B>cu6pYq{!oQut^ zdsTg7vy94KR@jWp#ZXG~4;971D-hdT60wtJzMO$IAbEfAlcjwgTYU8vslPHVA|vFl zU%Y%PF9}HRZn?bxQ~wS;XBiGYWSg#i&7y?$NhS-_NtK^DJA-8#t3911XDfzYmp}im zqoH9t#s~}t;3E7${_kKdYs;kQ0~QdFy$%o%>Azs@|8*__p1%Pi^8drXY|_=S!)@w;zGp|mL3{J&U!flEUc-*yJBNsYnpm})jHGPu2e2w8gopQPp< zngwNejiJrat$p@>`<{9=bl}U~+UBPlUKldre)HLk-!Xe4#a$QL=lSgIFNHzF6xo?E zSki~+;UVTdKXOaG@$KpK;rT}9H}QTSEVHMc-21^G=S?Z`mQ(~MkGsd0uGJayY_j@* zZOf)hSq6!iRGU1|i3b0gdp*9ts=A}9q(T&eGGb)bAYL`C5yMwIM60#rJfXBL13QJq z>^Jk0)uB+CjvqTUbbey$>cq{(77%EQnIOjv&#F0(UGr3nxwm)*GpA|8fC$%@!5X0S z9Tc=y=1S2*U0O7L{Qw!Hc2?R@1lQeU292-4L!^2Q$N6aV>iq7F=vf>s zl|@dLt+{&@0i#t+;E(D`vBH-uiUWfgrxmu zmvrxUKmy?sm6+w&m>DpeJs0c)-IO=zyB_hC@%q+qi*2nu(j-FJE$D{ zl|Vyd-^uB0w5gul;Q5ulP^tyY#;{xWZ)6oTSh5_2W7qut_LhiMQ+FtQL?7XK+`+m} z@3SlAL;4_&+9$N(G+fkAB?98ZYvvpzEQqdb)-^!+LroUFUp;;4pnS^yIn(STsR{Bh z50(C45d|RWYMOY&@gWpZGiAv|F+vv2kI=17^#%3cx$!(y%SCbnpF_8s33S;4q>sf*jlcQ;sznvP%Jy!c>%$RIBMK z5XDSskyA3Y>njEYB3u;`7Z+Z0p0A)|L4u`_+4gZ&y|(Q@`b*oUN%+Mk@~{clzecuK zW0;!C`P9_lqD>}<_4HF|$-0e4T*blLG>NrVI*P^C%!z<0n!s#f<3%8crK*`_AB`(J z^X~qg9VKH*!cvVFtXD(-xmH*6t-TXWxY?;A zEY-M!O>Z{`!{&Y=k7>A!wKufRuG>q3D|xn3I)URcVX09!73>kgn}UrK?^lch$DiT# z5mTDLXrp-~)0#};Mk!-fc4^hQaJ$EW#9> zc<5J`!^Acyh>>u1$NmNU#jLf+V5g<76l@t@JNzvchA%?0`TBpW8|eyyu0|trgTY&oT@W2C9QCu0)b_aCLuNQg~D82c`t&c?+$?Kd5ZT z4hjU9FM-}7C&5Bq2F3r-)ah9^$aYofo4J&z*D!5zZwgBE`&yUvw+xsEs2{vBw@w!-)I-?+c@N8$~cOUT!GkzlixhW7Edy9~96;X1|gW zeraL;HalwlcF%Lf+PgLu0IgF%Arz28<035P$J00q7G-MA<-GEkBFJIAtA!T``I~0cltOSvx_P_c`3&ZJAT04+~NEc>*T(FA2pAOYoXFUvpJg zBWY0(xvl80k%@QnpCBS%n&E$cjKY-_IT9BbiEHS2#5|sLkp}`{L+^8J(Oeqp`iT&D8V`% zpkbr<751%SPghal0SZC%i5+ctLQPTTfQ+qP}nwr$(CZQC|0&$&?* zajWs-wN_&`S7*fh#{7S0aU_Hlra*4{hoZE3rRryF-!2E3O>kJbvHY>-N^`|@c>2}h~&(5Gm(XS>(yOwV2Yqkyc11Kt~_0AR4F;Jc&B|SUNz2J*r zZ1l)=tBAp#7D?rq>RYRrSMjcCXI72ju+8b}$tk>u59Y^Fib^FTV!OJ_&@?zXS$%K#w(MT^$84t{>(%<`TPc0L8`Zp#FV~StEf|0Nk7eQ z$9taC49C?N1E4&!w-wUKxE@R!;LXO)EEwal~(0dZy=xwswvPS*@jJ~!N zOtoW&&hpIBOg!Kq?1nDN*KQC{Z!%i_tU5PY-zx=tPgIh%Bc_u=u}6@Or1=7N$y3Cs zhDDlrG-KYMeK}7hmNEP-$-Qp}VK(d}saMpCYO>Xe02=N@u#z`q{#zsh`5K(?kmv-1 z0Ub;dCT%Q{ikph6cITJ0Pu1RzB%-=V6bBUD*iQt6x<_x<%Ye2JH#5uiu!XXrPWP=( z;ScZUvO2Zk8wyU6DuH_ICQHDCf+pB-%g%F4cXKTDJo>Yc7gG37jLD}^JhCJNgOW_B z+?EWu{k5np_qN51<%7S91?q~eD$=#9S1znJFCdT({=8YtYaEX{atR{OBl6!ko3gj& z6K~%fdC?!Rgwr`L3<|TvRz5v_7@mYbqF;+<*S2f(wF|=_6UA+MGXeBjswI(B&~eWZ zbnDV+r`JON;A`Dw3;o=Po13xm1{DWWG0}oGf&&J(t;dN6TfrD3E$7ikZ-w^vw*j_H zn{^ZLi5e`cSAt+RbHD4fp)=vb(R637>tT~N_zgtHj8=mSD=G(@4JxGpTdVga0J#L; z3}JF#=F~o>fqHbK(w}NBAB4bn^u-ln6p0S#jZTYj$-Q*IEs90z3Nw;4pqo{{GMpZs zhh#8d!UpqlpRtvw9tadF2-C-N)>7}FnfijKSt{`qNsZ29Kavdoe!y5w1f$L(I_fH+ z1d(tR(z~XJncw=3GM3~<$%sl-9fo_HpG|Qf8&^RDkp{~23~uYd%eQ?CP=4C!_M2-- z$T!c1CW3M6#lNic0LYc0r98BaA}GWiXV90%-Ns2ISpGAF=p94DeJb2~9B>}$6NsAw z5tQxoHnUo>ei8N<$!`7Jq!^?JU%}Pir65KrG&#h~j(F@pq+8CgYFRsZyS@U;ywP9K zPL*UE>se6EcaewkI$}qwJ%~`9gRds2*#ACd+2CRLTaiC5g)EJFLUhe%XSzlEZ(UiD zU4liwL`ulvg}meCJ->%O_Kwb+KEv=MVn7X$o7L{y=t6ab1w-dsGSbV6c&~tzH(q!+4HmO5HwU zkqX4dSHqbap>NkfN(toOb&gC!$7TzDF6A(+Ws^24#u*q!LP5L+waJ|BV31P{o4`I1 zIZS*$IQhkm57)p!D5_?OBs5N1qi6n8cnR6b>UyVFb9o@wX5~c1j5BkNGNdpQ&WF1H zTR%5yoD%_g94)Z@LhHgqIjBnG>O?mI2l%aib9lTDk}AzFx()4v~l zd{wUotL)gTw-aboqlstK`uKr5XBn=Kx?T!2r|u6E=r}+?hU{P}PJTIOJQq4*LNgq0 zNVzuOV^p8*YlNT~v@7Rc0?n<;B^ZLwKGAa2gTDjPX|lzjNSyMy@YS{};w z@C{m41jXOi8uugYo)P=tfM$t?;Q#2GeE~5*0!eUV{m{<}VZu!Xj!0h^gG)J;3lOk~ zm_UuKJQ)3z5=`>;d1Q@QS6(&OV@!d~x!6qRLo<1ea%ebafB^Gkr3w8XTZ^I)zCj%3 z`Tf_rt-$s?O3?<=yWk$B2z!dYbOs!L_bDpK$>tpDe=7%>ppxH%=BlrYuaV;`ctp-d zPpL14@lCM15^N*$j}P<{Pz4z3n=6vOH1W4EoOM3~Z)1JF8bJ~t2a6}L2$}0BL=3bsk){3x(mmsR_6&utu>|4(_ zC5QBxb;z-}%<94!SXEo^sc)!}w|JB@MP`Au>B@Tj@Px~$1=7+wB1VY`pSHK7&>DZKby0zs9d zIiT^VynV!X6jLkteOz66!fhM`{&vN~4Vb)W&X}&Xy)g|Bn=(@;`pxfYf;@|iUnKgg~&ALPb>6gxG|(x0M)r&Jxy5pUkSXg zY;{4LmA}d@HX+z)&dOLo&(}UXKaWe^he0}QVS{p7I8Z3!bX!-s5bEa_ zvBfF^;AlNt`X15Gs!Hhj2$QcPg_q6z*p^@%kc%&S&#mx?O64?^R< zD+dn?bUfN18A&G37E!e%Jq_ss8BM&^&rW1Go>2fO3QOXQz<2(S`4Z6f5-evF>so2)nIj5q?P%}ol>(N0}8ws1iP zRwwlPN_U0})ePfB35+PV|33WD7YIx<6jKk{XW+MQ|NYLFkTqT?*^8R(- z;0UlC&&6&^GZ%zRf77py4AYCIaqlBBPunC}2*Mz7I>>MMP+Aw7A--_b_pM54qQ2yQ z(N864Jbq9X(|(_sK2Bz~Oq%G%%q@OPP>5Tx02WNWkYbH2SCXb5RXn{_LhS(M2zM#q zd4J-cveGQ!ClH5s<-s(Z1YaM@mXGSOo@GJM z%E=e?Y&J(n!%|6G;?sI840i|1zC#{0pFfsMwS(o%`@^1E7&36duW2b2nKZMS^!k-@)3j}w{T{U8Kqb5%|) zsWNGWqLWJW7-w?H!UF?;Wyv~rRwZu~H3GPXApLH#n^k5C0<=>F%RoCxt}ox8Uf)+5 z)-qla)*f#myS#Z*S@6dTckP;BCCQHFg?lKURq(OT@#p;lQVHA8D0hNH(y=V>`{>*} z&ld7n)z+9InUal3h>#^RWebQx62V7HXMOh}ISwv(X}rohW%+2r=4L-+|MWw)MsuLV zUnPh<5QpTW@+B$?_42;*#0`ug#NJ8YX(!?X8g%!ASm}G~4C^Dd>#CiuIu{~fjN*(H zn@+E6EuAIM`y=};&4Zk_R*3D1;*RpQK>_x;q?8BjDmZ^%+jIG}YIf#m>pWh;7k0T; zvD|lDSc^rPnC*S>y#(g7xqlclw7Y`aw4P!!KgfA+FkfZVX<#C940Q*wG z$LsBfT)t#(tZeay?B+Y7DTFHeI_d>2i(}{Wd~wQ?@b2u=`o4y}g+SGl_{JBp28#4v z-?0UmtM!Q%-5x9>y)qh(<8Kez{!mco^ z+gHx@+)T5jC#5&2RguERF)p%?gd0mI+S+8L2gvMh9OKTM5auJFYkH_i%VX7*qu$w+ zpn&#@g|m5M%#r*Kted$fPXQV*RV-`ey z!lr<0J2s?u`2J{SHJRt6wrAPybdBV}dpvXfRE}#95IUms?CKqRazY2a3m$q1ti|bp zhqz&@P%cKNpcJ+_=@eoo>MaDnT5@5{pkS~B42+am?pmO>0>vzw*|JzJ3Su#z&e$B-G z|7a$PD*rz-As(socK)xK)B%eA*Gy)E#u<`FVauYrz<~nW9!<>@7@61`ogXe1l$N#> zxETvu>k2?D!ZNF`tE(?mmL5`iPIGvFAisT{)2%L)z9fC_X#dGoncK0nd8cpo_`JSO zvF>En;jh?zf08plYj#?$(?Zv6)NDkIc*Tp(9q_>TU)sccuthMw*lj$6tzR0s7U}IvlAlX zy~`ByK?=zR|7pJ;TYNd}5l;qP7C2V)sRW}@$5oX`|F)p z@h<1Rs-tu|l=rEkSo`z`^6(}|VLT*!G-&1vB6Y8@7eloKXHz37yWD#H$h*O;x#~|? zAG{7rKCcdeuRBWr>!^#SyFJDFLrBaGxy;1H(C78F8G9Oa3H*YO6vzEWr8k-ZXXpcv zGA^*zzWyn=Yv6y(q_(Gb;)d4eedTB?WbEZO&-aKk2?uH7Apsu!KWJkAizZ^@E^K|) zq!&xoxHN~>?VhL9xnkT?Ty0CKr#q6?w7i>YG5z892!RC8NFJzz)!*uma>CMx{aitb zz1b4J-C~56%_EIQ90@pS+gw|k+*33+LP|a<5x}= z{zp#Qn7o%gam7({c@p1N!^Aa|rZ1k*#S73tXIpzm_-?snLl^HnF|x}B1_CDekdbAf zYXF8E0yP9<&W;Xib; zVf+!_ba3A*G{Mm{m;y&wnOQEQy#B~)*6*B;u{BD;#P7J1KSp|W=#+UwA>e+=s@;S$vqo2f+Jf|jSyzIx!|g=ps|0o`mT@MP7oj*%~8%4!P{{wuk zEBQ+&8r$)cz-1Lgb&HL#AL&mVGvPXT=|}iH)VRtdl>CU7w)S;sYto;0j1wf##CSff zdq8DBew9;6{AB$*-X&9DLKt2&Kk&s%g(Kw3^K4bDAI=2v;z#7`>b81^ZvW&%N6}=2 z?Y;lQCa}M3Qq=X)Vamzgde+vpDk^xMl4?0b`euOIb0+tA@_79Jv5DkG=5{0!>w|Ik zozTa7O1oPXEMwwI@g?J!=PjA_{K4oRM#JbbhORrCtl=CS51{xTIuqSmDnSMe0cfU$<3cz@&aha#O_r$0qDk-V z30wUZ%0Icg9pMq02p0qz5FmFJx?1Rlp1t$nN^0jQn$%fquOxnk&5RW3zic8rv)dMi zTHr(!Uvn<+Xkw?=xEBkR|2Sy0kAy>qrG&ZC-_MUXjxf%Dr*!B$>KMHni(C&%9o{)k ziuoW?uvlbjqoS`BjdvJIOp`BA16t4H{~tAR;JNR6s9M~q_<|aH=%l3rppkurBUHi6 zhCA=K`}yPtQ+l}iSh?>kCXlk5c_1!85F#7tWdW=f=mm?(08SDzcS}4uDlflf3sFR| zKlp%n2NeNlk!nCA4MBZfCduG%BzMNCx;(dBH6ta(NMZ{#?JD0a7yU6(nvu!Q2cUJZ z49B~BlHjQ2)ji^_uTb~YzEV#z7%2jP74M&a4(6UQWkT;|6r|g@($272!$sho78E5W zx0c|0)gEP4SgoKc=tuu2Tlq(R{BCdWxn^uu%?Cao0cIDu*fX@K-&_O$LnwyQEF;=b zLITlq58LfYKFI93;oVAE#$}2_VN3-dz)w70^F|6#G>#9~KI+IUu_xzB=URm`?Hny9 z+VykR4(2e?6~Ie`XLmb{DSfLFt}TQHiLW7eEQX$z`!!^PRTzMNA+PF}<+XCtk`mrq zptA;IJ7NNGy#E6yGcas9@1|q@sgT3Z6UE#$Lk60l0MY{t? zk>qRvhVNjL(tXpx-JB9HYVH9(+mny}uBOb2_y|C*?w)>s+FbIpZBCSg+q|%b>z*$p z4zqISF*$3*5DwT=QX)jbda+R{fn8wQ2u>c3`Ro}5!5NdXzZ)W>PyP!h&>J1| z{Yts=V1C6p?Fjxb)CHQsOkyh^Xi^yYn{6TUSAwyJe$a{8yw@aKhp#$8_&B0)sdfUy zwu5w0qD&(8s5VN&TQl`uK!A1G4zXQYsu7jf%kiQ->x6# zzIg(&Ny9Pam+>S7)nz#%7!PT^Jr$Z)tN(@>`(8RrW z7}bC~h+4deWzMylJGP+253Rk8Y@Ss&ZE{7$^{W12hiQOvEbNWal7b6Kn>&d~-{Z5Y ztf=4Y13-5tvbmlSAX@6IUqliu5@HM|28#mWQqd*rTMsb`sT4WZCi0akE`?92_OFLX zN6G3PO7pmO0x`9i8xtFmTS0^|C(7LA(nkqUxeBmtLF$q_3pa=y^;l>dvFRubZ=xCy zwTQufn#UWJ@Ty;;$XY}hHLIzBukmGxWRwHs0kV`rXJBKtVB@-4_P0~ia`5BH8bE2` zU_#t_@uLO{tzrF_PMZIRPI$aJ&Ea*ylowVjjvUyyX!3>#D>r9lMV$^DN)NXJ&z{ki zz(ztVn67(>J|3J`mOEsoEGU*4lxS(c<&KB)-IAQ~xafRU>U1CtmbU-|J$8gZ2;UEx z!X>GNov;hVy^Sw(R;!n#y3J>O?#zi)8JPaz*cX$IVD|ZHmnDTN=i9#dSp|E#_h><$3H|c(o%7gsoxL6usAdLBf zWKVL@S@ZkN%Bn_A5uLn}40LY1tTlYb$gypc3Qggg0R;jh9&!-M6PAh5<4G^H6#Mt@ z6t7*1Hl>@Wm!pP_dMwHUht(DZMApLm2T;heJuoUz1>gr}k*+x3?U*U>yyW@d425H! z`~HX@49-1(Dgkwf5j`6Ei93u;BqwJzlewCvYus_*aU5_c8HSyP!4-Ur0qx*6JpxpS z3S*_Z5HF(}ob*Ovw$SNf!XbUsQp6x6U{TT4i|-H+BZozZLOV9p2jRWZGr&(hwafZp zFY+#1^uOhr0zO~zY}mHpG8<=V23v_0kFP`04V~loUQkx!^fn~Q9QeB953e&qe+Dy~ zFY|`dDj<6<#J6IU3Ju3Z^(Z=LK6&9*rl}9bTPvGU586N3%uR8k@G@XPg$eTGvXnng z21ry5o0|J9k3{a|Lb5n^BR-W=cu^GmXwmJIh1feU%TP}Za|YC$!ZL|JzH7*xR9e<(GeHsfs<6N`X^jY%ak~u zOD99noK8AdP^+&_yB z=MLI+k2|8H6K&$?w1~A<@hD!xyYpJ{`z zK5?^0rDN6J=oo;6iT8pI%MU7dHB=SrY}<1-bUX0qg;Z~~<>vf}HZb7P`SH_@(J`ufBd;<*pfFd!gavj zuW;u$Vfsg~m=I1J3gk5)B*Z$jyS$~ielRQh+2{`>tKCZld;V)Eb`eWdC^jT5?@fFl z0Wr)<^i*!HJX6>9JKp}~sZ6y??n=6a*t(L@64M&fKyg9_i0DgQ`+ugJ%*V6Pf!WqP zf4(q1fe8`&oXQU{Vt#@L%#Y;-)71Td|akCsxj3(b@uDX^yL%2e{a(=S3R zf!WhM)nYUHvb0sGTl0^y%ZHY64mCIP(O1ss!7BFy+DGt3>B^_za`DfFzbjy=Zu95pWLjTJsnf`fa6{|F`|OdR(siW~TQ?e^5(}sY9kE8DC`u=G;EW9M%7uLbiWIqSq&YV( zq5UQk&Yb!P)z&JrOMSR}_}f3va}xirqx1kO`SNrRgz++WJXa>-8F#GnNq(L1*6~j7 zX`t{{iWX(FCOS+}X!Q1S0uTy=8}3SRl>I@jx)1-rbj1giCSn>iMKa!cF@~uyo)H-G zDx9>gxof)Q)rTp4+7;Y5dWQ8FZ7U4rB4m zG0Xad)A3{BX>C#L{6vJ}K8+R~OHnhF;2k!oYIjFj|SBr|$R&qSv40@}UIuTYz{p$i`-Blx}q7{AdPl$I5*8$Vx~$3qM4Zag-; zEnQ8<8Bx+FnCqgTF8$T6$#nZHb$FsX@yGIG2CE7dVF&J{=g$MJC#)q;9q4(6(M)I+ zV#}I10kp!Uzm9?f|KTIQwKvF#NR1x8%dV3V9tFm^zasDEvKUKoW2joj> zV)gCk`+kM@QdH|L37Ym)#1hl=2O|iQtP{pXf2bSI;yt-h_icvLa+aQ_Lc$@7gtCuK z;E+>&6u+%5r~9;g z&g+Mid%q9D){Bd3-^YOg5KUMo)@)VXZzWynq4i`fOe>l8+`sjnZ%c0`PTWRQbbBmv zYb4`^N>S@7Ea5Y96i-8<{Z}hFerrV5%I)^rBtNZ55WIlIBv4#6&cjf%}$al^`?E z67HBg+N57NUq1qXtOUVg!&USdaDzwC@h#wIo!Tn?-321w8ZojhlNn40wRq+*EM80^ zfzo%nR>Lqg#>WfFi*4l(DRemdGr%9Z0g#}Z#5t86BLh&q#A954Rl>3C1g;+QB z$woAhzCfBLMEDBF_k)tlFS7}|i9YRay`HBlp*mvJL;eds^t$rzSfN0ODf8~P(D}++ zSHX2f_6r)ScX!E=w9kZT3I1fp;by2lMs{)&QW6?=c+IU(zo^!BUl56sk!7WtwVUv|8xpV02jU@cw`o@n73~1& zuePhy4AFlLrNStRc~j#*hSK^ULs|Q;p$v=w1Aa7HLnn5h)p3eU;v-$q1ij$NMAJ{) z08L2?SOI;PZQ`|){j_tMfg~p^hRNW#I~bNq*bx(oglyw!&~?bM;6Q0KQPo*bp6yJW z!OtJZRj2636x%b9ETxJ%0V>D%nxIL>$QEVkb1d{io@4fRWnO~`EtmFi^N^+5jhrH#wfG4#8a*2z~`{*g`NH zq6?OB4Qyb&9|qXnzpj@E-qfEj)Gnwd$H|f zGY;AY7eEy6b$ZOJwajTs$7e?=08mww1N=F~7KS4iTz;!{T_&f#1q);U!cwH#ECVb7z`f+4e z(I`cBR?lmUNJ!ivt@$4L_`GUbh;gW!5+O6a!; z;&|!B;zil;Hb6SXYKgKmGs(+ERld<4W=_;KwX|sLh%DmP$$0Bm${3dTZcBX} z)Q}Sy3Wwsz-$4pS7a0!_FA9KYPSuOr7ls(XG9IT6?~kp$zkp+lQ-&cb?{rtuj~1;` zMZWYB)}(Ag&=}sAts4wFvZnFKd~_u7d7hUC8O5rTh}THk;z@>xbeUDZKKHf^{XpgP z)vXbX{dwLr5KB`!LvCv?<|x7oY$cUmOS@jx`?NIIhM%Y{7Fm5Bk%CCTbDVou$Jp_T zUy#qZXreKeDKE#XaKigE7CVcS=Q{mffba-Qp%OleTt{WULQxO3At|0P)5cLQ|2%%^idi9)S&>34( zlT@@88mJ)njq=o3Mf)C9`KT2K6+^iY$^Pr>B~?vTRn5<4Ru=!L-KQt7jR$BTy{)=w z8C|ZqV}7zdrhRU<3dnPK0CdYSE`vy9(auZNvCUzX&FWy=%@wC^Aexr{_Bk^*ay93v zF(+3@h%Eulm6p5hp<&c3tJNq9Cz@(31yxxrFonMWLX9$sE)BjNWM-~5HE+nn zFb}~Z<;wbQ42wn6t}%}e!!QGhvi;p9RH`|fft*M@W-1UOie$ZtAqDk>#SSpfth&&K zLH<)eQ0O93o}R*7JmUFBs198i2>Jbm_JAsqV8W_g4*%j*HlNT>z{kvB28an;oo!t? zAtny!Ru5&^mA-uii5-bySRd!9A`6lL$1hKWQ;NVLyfONl&kTBIk+uNE2uM4uQNjerFMsc+C zFpmFuGrupexNL5h48NRiDS`s%;9Y|cbHTQ{*Z1)Jp&UYTfZlB9f z+@y6H#w$274MvOX6k3sZbsJ|}a*YSK!D!J|%&=>PkMMZB)F>*CBI-?VDh<3Mi?(1o zx-gqtJ=7iZLCuBkj7E-y2BIQPc6<*A6TS9W_iM3E0K||=pF5{Ol)b1yq9BsAa({ub z9ojB0vWS9q0;#;&Ot@7riWEj!ji!M8sOUwNP&1+cev=$qJtqN#YApkztX(syT$53G)xUQu(!R^xqKlcXJ+e!I@}fcdc@3P zkcPMsd5TwSR-L*V62?-sM zi~EK?0=t!(PqC8YC920}fIpb1Lh@%C4Bb({APE9$CEx>k;O)Jmi6z?(dxe!Ul#rOV<_wJPz5_pCzMMGsSpa*VK zj`=?4VY@gp=Wla{&3 zfeUI{6GOEp3YSh6M;9i0-hz4c zSQq-I_GK(bI%sU+r!l4{3lVrBKP0pQC1YjgP|OBj@`vCc<`sEpklmXM9{oAW9}e|< z6WmWb1OXr9o5%nGtf9-r=o9vaeMHUVKw@12e}!>iy8R~>(SIuAX5tn7Tk5dTS}_1p zuxUoMQ^^oVBc;NfNPiZ^^yVw3cNOP$o@mVA+{Y+$Fa#yfl`1=QwPSEFQ#3+Sn3Ib< zFg9rLg++!AX#ukk!I1$SPwmjo1rNWZ-xC1kt^SYhJ7%M_(m@#ny@e7aA%R&U`oV{j z#N`ssKU01u{#0Z=P5hUxF&w1G5`gwzA8YeD)P!v9_wsyJt-xIh51_Y?GQznch-W%F z7~oS1PM}9TQ|w$&k}^VZ1wkvuo}%WE?LH{zfecD`!D<|3{XJx@KN#zd*9jOp+Tz3s zr#+kfht`rgS2T7DDosIZ&IZvzGHX$8XjqrS0?-jxA_+l8&IUn25&t%03i*{cd_`gp zko-B~)eBD9(`obzWP|D1QmSno=-^j07G?%m&m>cp(E%Lgs z=ZYnu@7Ao{&zJDrt8=nWXvR2g$;lF1V|2yb;z~$nn5X)U7lQ7QS|0A)6x?S9JQijG^RMaZ29;GG!x4jU;I)$T7V?CO)aBG$Ls*ijAt5wDw zqWJL-UTm~p%_A9J;&%VQJR}#9SLjzYq3a@Z{C2?^H(X3x@xGl3Y?lT7P!q(=s-bI| zpaum)A=*M#twznBiIDA3wf!efjl!c=bZP=f$YNSu7o?<5o+oe6J z^9Om)=6%*i+QS9Q1nSPDbt%gZn*=StoxK;ejxoaffk&S5BoR-VQtc<_!C>e(*vV7n&9)##u80YCiS_3gV|XK)hEn40a4;{x=yfiSDBxvY~`nxJ{ced7P$+MxUBF*jlfv z{RpZqwSQ4SgT*$yIAYLXS^GMGxK7O6&%YqwzRbUYaNFR0m#^0s{~#i-HUc%7CPL*b1YsKiigTO?h=KsIZMzG>7N@Vv!<`mmx&l+ z(TCGH7`ASZsZ4s*t@hXP;C0h)tUfqBzRZMwVp|2qV|1GNz)FQr_jo+VA!v+ek{;SV z7zwf2lZ8jSXYgnndh&*hAeS;|3$v6axfW-+OFmt4)13}VgItx~x zVx%9_Y8;?ik34wS8@$Uy6?L0!hHj{02Ns>6A(p=*_3|>#C(LBxK9$ALN!`Swl1ruY zq(Qo(N7klNxTRt@pX;9<&V|oXrJ2>{H&KRJDYGvV!!g<2V9P#)cyNS-W)2d97Q*t{ z<)W3AwFTwG;(T(`0&72F*;d?)f5^j~=u#DCWNvb1YIXt3Q0Lt6*RIs-ExXmHCP!mt zVU^z*y0W6q47~^IGYVmCX1xzubOsURUn`P9;f}Lh{rBZaf^vxfM^T7*Q*T>~b5vvc z4kcqnSBpq47xT1dx^7u;&-eh}L}c$LXcm<)w!ZthOQ@aMZ>o-|*a90U6mkcOD#qjEI^whQV38q4iwpgE(Ds(h`HbuR=x=2nIZMWZiU z0cIFf5WVDBsrq)0((-Qiyq`WVRvxUn&sAjmx)f6y2UQvYC^AhPzj`Q!yEXcl$}`n3 zXV;xF8FYqjchwf56c+21!g?}9{Y2agX|ye0+_8JDmt|AH@GfxKF;Uee3_WFh(+bQx zcw=b9*!<7g!U(8u$h_A zc$n5T>n(t{^qg2dHKH1BSWv+PQ5kdcc4X0`f4?A@w;v#whQ(RzZvpOnFEN9XzEsn& z!SZC2>tyeNtAB(&r$$G3p0exR-*P0&bBJadGix#hZ|XuqQAZqCk^SBVsD=>1MY69I zp#Y1!Ouv((Q@OapEbL|3>C3n_bePUR)5(EG@EQ#IHz}~4L=g5)q6gxP78TLR1#8&# zJDAZzq@Q(r@?GfCXTYMIW;l-&%YyjXA zH$7n6!>(fi_!zIU%_)u>b!6Roam)Fl`?DipJaH6z1O1MaG)*QXq>MS3uQ)#kk_$P zyHp>HghGKJrz#j3;|m1z;hO9c{&Maj@J zJ+IiJpoVd(J1>{!>{-8)I3~qNeA@E2jinyRQaj^lzKMRMx`sGg+HoOW-VLm$c(4-H zSr3YScNaYxURa*7L4^dh!Me+N!d$JTGTvzd>-Q7u#5rvJ?<%>8egknc?TIZ1`PG?l zM9s#)ddXR`Qr^WNo@@M2PWz}8yRUq{%*CL+b0$$V=(l19F<$iQ%KhZ)HM6SePjCV` ziiMWQ-5`DHs0q7F=I{e00w3&i-ld^ug8YX=kY3hIoy}>N7y!>n?akpNtMEiVr>lXw zV3)R1Rv&-9#FAEOS|4FPK?qh^SfdnXV`%eBf!EvDsit`J%ayiUvx>6&`-*v; z8#?v)3rWI)MZ@;zhdW_M)BSm@D42~0Xewsvm=3Zrp#+LnpoC_yxh6l8R){*;*Vi2m;v?C}kMO5wB)i1j-wA3v z2=EHKRGD*HS=7(Tv?#$Pd8SgSOkU)ub0VDL_qVqCq^l@4fvqs&B`-^S{s?!ls&nDy znmj3krtnk}D62||n~+yTId{nDfunwOM~zE$higdQUAQ?NunOH?*w5F+-jmvs@b4^|?8qB-@NFt2{4T3j>yO-ib%)zt zUw^90x1+&Z#vb-EIf$JIE{Zw0VW2A>=MJij)(S?}MbX8Z7$d*Q%-`@_w}>8! zE>7}<>I}&$>>PnalHpg$8HJzM8t3A|Rm9gyu>r-o7IFDVy>;#&21OXO3 zoIWzVG1?zC_*;Rt-}m|G{Sa+GxY1||r+@~q;82h;xONmY9AQO3H`}pqq13s?Z|WRd z*D-A^YF#q}Z0~j5yu_PIm3zofDxt?%baDL;)ll^vWiwOH27pLHGf<-W((xCu;|9qx z{SxeWqzN@-NS9>{C>%MgcTcGkd(chx2r~mj->RONHclwspxi#l$!6fkQf6!^8a|B5 zVNxnETLrFS9+o^c%hYX?%uhl@$(v(~G?-Oc#Es=98Y(rc14v8Z`fnhjh!o2>I?Ejf zEt#Qvey>on+a@o4NGvr|MZZ91tbdohwGCJt8xW#S>EW5pM=l0iQ`~VY=d>lMVWMw# zE#nqf2W)@HT%78zkl$$bRJEb}{XE8GZtL=zi)1;};B8;YD9=3c^{}xo)4Q3JP_A z&7HVd`OzB2b*$gaD^`uwZ6?j!lRv0lyP!6~#%BeQ=Bx|bHz>SiS*2ejr_f-CSpMCY ztiJ0AiZ238d2~EorTm@}4#u7K{hO0y{rj_)Ius*zvKme)(WH9^3Xg~-{bJ(NW9&X5 zi`&T|*Gp!_;;V)kNv;}5E+0r%ZOoQN#70seBUNn4H$23W!NsFT#A8xceFA8FV(5NC z`F;{@at>zJ6-LZVuO{8iPY||#0|SdUsILEzILOAJGmm9T(vX|cwYg5~OEI8dlS@+X zcZuRa*|q0^mwGQF=s}bWahzP+vy4JZkwb zSFd#}8q2Y+1$)PNn!qlcFn3UG>naUQHpCF?7H#;znBLdF$Ty=cleCaf)+3=d3k4}j zD%kxaN=$-SLn5r0UuJHmX>mxJ&1-k|NkR9BG4GaX;??#3Nu_xWl>CATnYD^I%_sAo zuF{=CK5p5d^e(9~xLPRTQk}7B&`j#;V+qtxdc4syb1wWI_PvM7`XNSh(H0H18S$95 zD*X6(06oU^93&v;76r>uX7u!u%i2p>0Xo4cQTUU(Ur~V1xYi0@`+ITQ)j_$tT`kdS zYu^*~2pYiV?@;hgPbyQGI5u;G6Q4G40<%_vf8-``AT*kUYHw%$*_(1=cY&mK@zen( zoA0CqSmCN)6Dj`z_1m2QeP?tIAT>=fL6MlM#~#G}Ritn++4payVEq3f?3;osVWMth zo)b)L+sTP-+qUgYY-7TSZ9AE0VrOF8wtc^Q@7w?KSMBbHy?1w2SM}Rkd#ym()O@kH zer$GJ(?u;gwS5+Kd84n4P#$)iZ6#s+(JnSk`5`{`IE-U}MtajsHSKI|rBNlyQaSEm zM%|o^VD?0pCpg{Q!~!_B66_vF?O3@tU**xRc1vK85s86%oR>a9Jm+HKOYEsmg~p+V zOZYQ~lYdFz)1#>w*dFCqMGE(rY&iT>n3!(c%Z|tET>R^T%954U{-+>1AS7meoe#u`FZQ*(5;MI<)#C7Qp~Y&Y`C^rXCiS`*avFW z6vukYMq$5~X0OvxUpT56}DifCZ&#Qr4=U8);S}!aD!Ys~d)7nIgGyxM!feY$HMKIs z8Y8drsE_&P`UG=CK1PGylht(wB|T>zpNonc5mk8g`{@r-*@~MXQ+@TM=zZc#%bMyD zc&U{gHrOv#W{0yF|BXpgXaf7sRy7|tiaRoAT3Uu+6Wz#w?tt)CTTa*DsYUa}sl_#a zdcP@g+Qh1V6C2gAhwFO!szk*K(NQgZQzbUoNV^l`sYd;MXFRMQ#ttU+iUrV>MVXXC zHV4B_<=i!i>XoK;X1^P8*vgnHwc5K}#x~|&5sw%cnX($dwELqltZN@Cm z!3@}D;Yx_<;7VGxlP!g1afJQ~AB}`QykBCEz!gb6Qo8iLuV{=OJ(5@t0Uz5-s78#d zZg!umURdH34|caP_=g0uJaoiWF(X*gDzJcUlRGSg4$z+zO5<3BW(({?hveH4Z_y0J z4RO`QwF%x0BX`K8mTy^njHwzRd29|;5c!~ zk?OGlZ-Vl!;#r9yA>2480_B)bYWo9`-W+Pd2_83nA$d+ap%<)%CILrp3@OaGNjgVf;c0!19BJhdDB8!jXx?@L+B+bh~ z71gm8FhT2fLYQXnMu?U3+{=(o5ZPiP{5ZRqDaiVA$O`N`p#HOp8f1+K_(2>PyD&TU zDblJejx^}=S&W6(9r|o*MpZ#DOOXuZufW}s&r`xc*Zf$NY{yf{2_f$l76C;Gd*$gQ+~E%WTCn?%1>kv%mfbPQ=6e^ z!@zA5qK#Mu5C-JZhe*WAoK;&B{XxY7rkxKaz@CoOy!Wzx9X7yZX6kOWPl};c)1h4a zIH+kCsU3|#B0bwxQFl~0U~Az=EikPaw6qAT4B=_&!;J5MkkETVV2v^@$s?)#&9 z5fWi{L5TpR&s4ZfAP;GJQx5)aooikQzf`;E9FZ&N(977)4)(qy4VfEUHX9U9+VRJY zos?Wq&efI1{j~`^faHxQ$;HhUFel*kFYp-ScA-4?ZKtkWid;p0CJYfXm61G!3Fn6O z1Ne#Hq1XOX-$~(-N3Iwld<{J$L5Y~DcN^%1VK5Ka1(a>^_zg-wWvd_^~ZVK0ddR-v#m7f=_?#7Z&BC2#=bZs$3;R^u+FfI4APA2!6w9jXlF{M?p{r4 z2q9O{SN+co^t}BlVfCUip(O~$9Oq{B_##qkxr7zKFgd>lMwSdR0RkhxYCPP`lFqbL zx)9^og3AB2ZyO2A+0$a)n98N_rC1^lS0NfiZ#tIyDLXOhbCtr0s@wYB>U+v#w5VW` zlXlv;PYD|Dn!W%o?rPG-#xVS%h$Zfob(Cpu&*e_Xn3fTB9*>n_ec|p;^n&cVT)!f4 zYr>L3_#V8xoh=Ztsc+L^8+;&SW}X^$VetB*tBuzdt1<#rlj2xctT^+-*!ZO6~<6@(mwS$0x;g79lP5 zKA(o`x-!w#UlA;5K?_H~S&bcfL+Az?G;7rQ!$y_U4_ssxOGLII(L zI3)J0qzN0nENnPUBTb{I@x0QyQprUAUv-QwTgcey*o-}@Z0jbzf-dk76x^ljVYehM zM|Yl7-_cOn67h#mp$IIqdr281Ij&EB%1k2x?*cxvtT{3R$F5di9~0sHj;lq$y#CV; zu*q*)Z-aoaN{7aicy}qhjuoBmxuql~c@}%-weVD3ED`)%*k@2^v|CpjWh5{o7e*>~ zuD(tO^Ugba!1mh*{0Y}*V8lR?IfWbB9{5D*6O#>bDB65foMgqNWw?pICIsfbx=vKp zhIa7`ctR5A7laR_y##+E@fjKsFqxqe-^>dSFsTn{3_z7@0#l@`PQimIHYbx12b>=K3S~9+n?JZgzij}k$=?SKrNsq@+l4rd~p5S$3{JWcLp`S zC2g8NF36#TfRBYTEy6U_WbH1wm1HwfsmK>1L~*A8GrgPRF3eDh5i1DYN>46rJ0!4$ zF(8$RfONyy)!FAW6FS<8Kw)DWk?D=gmvEf5$Yd)^LPHe4$m=BL81h_zngUKAYtQsc z>m8>{(71qhBKu5!)OgT!9Q>0=L0DjAN7O4wXTy=SVKS554U?Y<;Cz7- zP?<0bYC#P9IYVzlU|)YcR{PW^{qV>fk`R*M##qNNIznY+aCOn_)GrwVryW~0E6pdx`>Z+E^(WVukRjE=Q7G0=P zr1Zel;Gx$~VHF81wq?lg#x3shvdPn| z6&o@rP8f0K$Dvfjx31;zMD**E?_0yH{G~T&3p5xB8F$xQKE(kQ6X#q)WTsselV9Cc z+7qo|mYRz(9A4{mp%RU(=%zQ$*{?6G4xa-`#E!zJT zIQ5D)tAT!n8u|X5nO7?Zh|4T~;R5*wcBtka%s=Ep95u4|8RFdK*Am}exo=(C z<6c#|CWja;EH5oSKe}3Zf6rryRt z5Va|!o2ml(-nwJXw-;aof%u4%+K9X_6*qRXoL($PA!oU2>P^o-6IT~wU!uQ5JL|9U za{hFGH%A;}7WYi_0bf)-cvJAd3jvg+ZT-^z2gjZ|wl&(X^mPjJZdq9l7_OkHSor8f|fJAhocM=hCwS7jtVA$Fqo~~?@4ll zH_g0g>SlV#iG)|nCG+EQPJddpG>vRGG;k|LLzI{u4CFP9rVC*uRF*vA^H;{OE`&WE);rrLbVWZoqZ7JtHCdKD( zS0Ax5&@OlU4SI zMXw)NA>T4ZQa4ZkJ>^5F`upQd%FI+X-3nzAg>Py5OAohS^s^gEP!2=!KiihP4sWuh zqCgVXu%HGqll96;X_uL1i65)sCqnR^usKfC0O<2`sey&1F%<5~GdyXBbCby01t!7&8CbLU2t!fgE(0wK`~ph$ zU89ZQ?s2uJs?fTzHbaZ8bTt%=pZ}iDQ2yiemi_AHQpDGqAioThh)_`;GIwkSg0Si6cPyL=HiP~kTGDIrbr;%^aQ}tEtx&*~u+(o= zio4p9$ci4lRM`bHlk7B8$g=u9IeY8lDvHW2w!G}p_n#a}@ zB01OM+zxh1&J~TRH+o1LW}V_+>D_|LJ0Gi$67XkoX|%@_hB`&)F|4xyuq%*ungSlO z0WXfW0E0fPs?@0oS0id*u*4)wQ7^o9QG)N0SnKz^UacJUpb{E*)q-(RwhfsWVrm=4 zj5fJXKU-5zh0cD4v{xx0{7#g1U79MorjU7is5&`3bKgAGY0WkQC3Er3jVzFQ1Daw~)<~;OhoDK0{f!Bx zw#c+E&j+lu;^8ejE5C{<4_9sm3Gu=XzHmY|*#g@<*${p(k808i#T9i&JNwCtId8cU zT9PTJ{vQ=bzj|l&dmDm0G-S0IMmrixDQxcG-Iak_4kHVP(j7N9fkl&e`>6+~b>|rQ zF)HgM{>U}^TBP|kat;=L?B4T7DEFEt8(sqxJ?WqE@^UX?sS_H}$ADYD!|hiaWOrrL zV)M-oN?Ii$%gE;;*mTfB zg*3iku$n~HvJDHmYeg0PwL~XPs@S(6kRwQ*oGTA^kL{NGr|#Jg*pOT~P%DT$SDp>o z=)?^t2Oq-d_=JJr@)WL*9^#!y^yByV5Kj7gHEY2SlG67gf3}M4Z@_KQOH?%10*a)d zDpQyVQvBQR)1mwU$)Ry*1x<9Ll07g+b&46QyZCA~7f}Qn`G`6yp)Tyii%Bo}918{B z(rk!#ZYXmSfed3oF!5<$*7>hUHG*H@8RnyQKO)z@RHi=~$!3?MP6lpvl567i z;wKSC9U3DPJtTcRWE}t*bUuX|nOUF`2g?9`28OYUaf?G`w#r12JDr=fy<0 zbIZ{$mq3xa&s&mwPJYnh(3lEJchsg<>P}N96>8BjsS6>X7s7rw90#GlGnoD{a5dFxI46wAPe=KPU90JufMAkm*l0kIuI ziTaWYly*voza(*p=)eCfBvb|5w!Dex7kVq@5$VEQ*zAE$cS50&eLeS5yx-u-A&^AAN}gcW7?f@lpZIwm zY}{fK>nJ%|lCS2X}}b1G`azBZ*16Q0TIyt%#us&$%r(f1y=((w%M5 z0;GQTW}v@W!~oY)KRfZ8{&NUWqfulaZg$3G@dA2oSNCxAm^Wr(d+OxC*Z_PoV0KQL zr|>&u@S8-rSm2uv23Pa{c@=+OM7m`vHDp6Jq&z6*X7POrZL0byGjEL+{y9She~ui! zAv6@{z{sj$3CQ5v{)K0)x=)&s!Wred*{OEM_#4-ysePq8M#13lFq}`=&iR^E z*7LQ9q%HFDO5fs)Q4{A*%sJKPH|yr1j%Y5qgH2K9%rI%>@TSpd@=zXr*~pqeLCx?5 z=zwNV;f`XAKP_W#-F#GaX!DAV%-UWI(t-29dT&b&alFZbK z2SbWy{-Z)!DEfe2b9FR>&X1;o6$Ep2yRf7ceBJo!#x$}Vv^7)N1n0{#D*LbA{q~go zFUP+xH@hMtSf+1nMByzdDuW&r^eX>c!+q7R6j%R(mcHx4edPqU9P;aBU7h-;4c6p5 zAnzRSt_5CY=+6~S?0ebo)bDUaH-&bUxKfAz8jo_)gj^eGV2;wJ<@>jQe8!}&2CB~Y4uBI+7pDjc@igBLPIcB|~%c^2mx zO>kQ%y;sU(nMzD3ySQMr%{hd$M?7SRd*^yv$ld{pn=QGW-NwD6yXB-S&%Ad~>Kxd2 z2Y0kr6%~~d?Uh6o zkpvV(G+^h?jhzAe+~o@-+(GGi7vL9U0gBY(L!J3f4+5)`C?(kZdPt?Q`NoYzi~-PK z58^i{U;t6Qsf*Cqy?BAkPB`j4kU z-!27Z9n)+hhgr)8Hhg8+L-+bwm%E&yYs>!Mt!VRo9~VLi!Q3-rS!cP5p>I zCwBsHvfDK@3Gh#ByJl|=m~R4@ee~>G${zT$HyRf~R7&INu@`%4gHu7|PUPro<;~JP zOR%~?0`gGfOmCTQg7^V0{DMH%bOkW%M{*P@%vKR=l66S7ll@VV2H0`LL&9`xfFreAit8E@SBtmKrD0aceB76K8aV0>(0`=&6+SGa{u=+uMl@S)EQB?t=sYi(Bk{BaO zJwVg$kA)%gdN{_25+`W-2tg743!$F|VZUnoW|#LO&+C^TS6s+8Lch1+IS2ge5gWeY zZ?7c~t^&(;7`;%h8mtc~Wa-+pj{HlkYgFw!lx&==g#PpA3XacO_j9 z81qcQM1fv1hBR}7`{*6*O!w7_hzsAOmR!}A;`TJjh2tY#iJP`#~6NkWG z@|quQR#a}3;v0xJ)I1+O{uWHNL2mzQnQBkE68jO$i>?U zl*R#QBR_9Cs+pM(63``f8@TPR>iI-&20ccXJf|l&p)=l|R{w+z4?3vH;mqp@!LncE6<4;+U5DkJECTZ z*k5pYgE&hLFQ!8;(<>qcCSgM_dK}tF3C>%>+x%=e-0TmZO`|%saux!bv{#+SGA7$> z^vlWTvt5c3G$_yg!=JxLCzhpNIjuCdNyWc~-TfkZ&R~nrB#Un?sUr-XTR&h%SW4UT zN>7{H4UJtCJ@a}66N|vSIg>(!(oi%!Ro+v)GT)Vlo*E~O zV!4L%hk);287K7pUhfmM6ZDmSHY_rG>T8p|d0(!%G4%dV|NDf7@NSxb@n#oU`Nf&U z9owPxtAn5;1JYmoC&3>q|78X_2g$6p=`Q7?vbylEOagO)Q`_T9W;3Gm=zXnYjU~S; zfhAXIx(TW%Bq}cDC0o|TEhb9h@V!cc9dl}Yab>uRR#fwut;4zOBd$XyN&IFBZX zlFTCJX`vwP%AYH0aacO3xLlcTmiW)WN((#_!bnqSo-QOiX1w6N9v~E*k*KmXq_r!L zwZ~S-GpO#^nyi~Bt?1*n5en#{l>+Q6V;+~d>a9mLH*M=|N0cTB!G{Uc2i>kP%Swf% zIMnZ!pC(f0&(7q;3A2v$8F3Iov%yENno*k%G5?lV2RttxXaS}N{MTBKG62ZM^1sO? z2w8EoGp%&(k{TH}qOM|Qe?2u+6yNPp^ZIsj&+2GLSxXT-HK>?N)!0e!MlbWF=I_ZW z-er7ml9v$6W0p-c*ifK~}YkcU0W$&6-tXvZRL`~8OD`pJh zYek1?Fg1qLMIu{4Ape6aT^n>-{ZGQBLfnOhS0PdEx@i_kxpBWHusv9-*S06U5s?Su zsD%|ocCF!IL}&UF-P(cc#(aakBeBVxL~RCl)Wo)cZ0oqeJFG1$+!T6KPgM=_luAQd zX4`g8zZ}@s7GG=CO&x>V1;abVZ^wjavL)J3=gkXP=0;jOM{iu)r}iycti1Z^qox)c z=R9cHO6W@;d}kZZSPi|{1^(6)$DCp>@tOl>I{OEs2~Amns!ut+TD?)#{(~O$ii7AU z?@yldS7Fbx8>fQG%D+R4Qi~|d@$#QUS!5gOm}Q}QZd}tPzS=F3ov=@ED(l4v{etwU zuWG&i9kbymIflknXsp5})7Av;zLuIyFQXWRL8yd#L)}HD$mZGleeQ?LbKo5_LngM0Bc)#ZjB_%-sM(w+ zAJ^DaH}aqpH$3Lykec0!*x34}tr?*Cb0A3YaSa@%#biBPrs*avShZytJz#tnP-qvWVv-;?);95=;vj=hv13ZDbyyO>l zC?&L&Ul}WJ4m^YtHkaZlV~Whyvqgn(+Pngp`$Is#{{4#9OSowHy{T7WHTFo%QZDYO ziDTY|fqCeSMtv?{v`TjJmq?u8Leef_Khz(>a!i#uo8KO>i;SLDQcEqKxnhu)f~Mfg zqXEcb%E0M$jm*-f9`S2aqrmk?TU69EK8)m)#5K&F*glq z>apTg$kbKX4$(hR{yn#+Cu7O_$Xg6|vdybgW_j^`_=bP@#(($*eSF~Et!i537#7v| zL+-D%arh7Ge#ExD;5gp0ZS7g|t(WmdVo79tplf}AN=u-B-!?od6;7R*~u7C-JZIm z^iCWvi%Q#B`;wt6$E!1HrqlG}`+C12o#dmj2qO6<4GS`Os**NHJ)(7@j$1Adn6(6N z;Xo??#tl^}vVYY9@`xJ2_D0~2?%+56fxY{K9DV~9dz8VPi>EZ)l%MSx%H4%FxxZ-) ze+@q2#Z+rIU5`Vr_p3UT zo6u#7&#}pUWW`WVwmtA@Xc8;Yb+AAyc-C38xEKo$NCGPF?3Oj4ZWNW9$X0DAfQnx^bItYh@`f+ zB=UnMm2r%|`GM_ccFMHnOn$x|$?2et4-i&Jn zrF_wBies$TIwv`8zq@TP`}a%=XrkC&ZARlbEzf20Y-k1(d%EyBGS1k~iMjeL&{qx)5QX5Y+xJ^OpSBUwg{&Ip`G zBD3eI8KpWu$p9^{3WxVDqu<&N*ZNt<)y)$dSJ7qZ3U1|7QLV6awZvt3hnD%It%aoj zf@?pJPb8*ADf9vz9z>{jvHB7}RMMxIL^$7NU)<}{w!u*S(Xt0U`@=pg z+GDq)YwtPSay`V3Am@f?J;3P_NdBe8EFV{5PZfE1AIG^tlB@bF8>siW5LI(qMM8EVY7Z(f_Obf=phhAxkj} zuIw2~NDBKF{f}A6t)YMbT{Nb`h;^+YZB^orlbHJ~Wbvpwo4&WQ+>dgdrkiBR6bT4Q>~*tQkYm^T8069mQuP0g~YHfe>v)_koCSC6vdTCfTNeERoil z=DP?Qv7)6^#PevV$7Xl~oAlwl+5pQABfSXz$Xb@Lccbke7>K%mgeRqGO)GH?TebBJ zTM_RkMA*t1U0no#JkbM*(Wk$dWQ-!ILuO6F} zt4xy0(-Kc~zdrwzU%zsg{Y;DbqRnH1+wDGbd-#?_|u5@iK5{7h+TsIXzcH@Yzl?TjIp4T94 z*-H1TPkfG`R^=`1)`Ny)g)LyAI&Q${J%vg57hXvOE*9kHrZRk9iR~s@dw@9#LEc?$ zRhksk1SfB=?$p8h#D`varM@dr0mhsG+u^QHz4%?V(<>6P5Y21c#f{uhwoi}D?NyGj z4>2QS=W14iyG4-}i|!t0IX*4NT~=5dL~Pj_0v0YpvC|vS#t_TwsW|5T7fc*VW1by9^{Eu)mCgseMGZtoZYkIP|ltxjlS28Y7d-3!D@isGx3OR zPYD||x9e`t&LOQeI*z1U{b>T*DW^B(fU;b_KGbr7K~ZuQfvEw{TL zYcD7CeTBjowbgr9+XI4ewy_VkuKuwLLHE-R{6gu9wimMx6PhA9%qXK0ayE+v=npW2 z(7u4_(v3!JqR-s@SRwbk-wfn+V)}usT&#qP|HL!G4KC}BU7DJZ`D^?lL*r%k6ViRB1bes@^R# zdh{o=3Lnh=&^|Lpo%tROHJ0MQ>>g9S7GK2DO;-*1&*~OJo-`)SA0?d_GKFNu?Mb=4 zlSL6pxlm)9ttFJ{NbHdIPb#1VTuV8B_>VpFtcl6UKXL|fRbo`_&=zLnBm;(qZbax* z1L{K_x3m%^%fzBx@eZ#*U9s_omGM3dY6{<^twi*ZP+Jv06)~oJZ_;2o)IOaQ?~!Of zzGf)#?S=7yvv#CeTVngCz#B#rKV-=M_jF!AxIcTQcu%>MPxPQ{xLPL+58gb=r$3Z= ztIh&NT|;C0WuyC+k(aK64`D#r)*(}l+Wc(yvl|_uM4-z&# z{wcp)?95?3gdMw^Dmip)J+#gF-fY&7yQpSa`KFM z!fHN3#TpkdGbaaWECLNJ{-l!@d1_Mx={f3npt7zv6OdhYE5)7*e8k=*dkCd_QkzPV^cseekJj}WXWRHW{& zdf>jUS`^97m0@Nz5+!XBd5;D$Yas>Sy1=1NKu{-&vRqOJn;tN8X&|*_95_=V4vl)# zqke=SvN2^%Hh*lVXE<6O>AeRr>I-W}M}?53Dj2yynUSg&C62L-9vF`vaGAM4uyUhj zWQWRO4#;prFO5~q(3h=Z+c#k6*s*wjeGAs8ui|v~mbTz>)}#`PmW<8-%&aI`-OyRp z;sH}4;j~J&uK$Iq)g|d~m37~FZ4Z`d&)b+ONM>a((w>ebHhXX{GYYd&@yn}3GC~iH zAA~M32Vj6$-;cS9Fl?5($N^1oaYpDmzNI620Ag$3cRA9-981X~Z9I$js`i}%@GtEy z{?wk|#~DS<$6EJC2f}zKEc)zN#)pQ4F1h;anz_X56!MKeMCBwo{^$03XTA?cRsDji zq#Qr;O(*t7+zYEQDf+$7<+;B$WCkl7!J!Ox{Bo2F#+ennT%xT}rXJ$GSx8*l5T#=V zvu=)pDYjPhWOamB=XO{de{|dQB-&4P8PgO4w5fEXY1Da-3jM4~gG`pS-R5}e%(m%^ zV7dDzZFjhYxs96O$GKQRZhC(dMLYxS)Ga`c8KPSkdBEIZ z^^ICn@FI6>quQa!mm1x7jA}?*=cToCTX)$?u12d%lA5ZO`9xLr>ZpEt+y6XGC-u%L<3|ErKpZr|!j_KY587vv8f0P3SGb3igpP%d zoA|%Qz?i?2-$#uik!2Qj#E)29mfL8NJ2cD97wWn6k&;$4!OPk%w0ApHuDBwM41dd) z;Xq0BUFel$dX;0uW$F*3Kv;o)ZLo98_5ec^ODzK0v}uUn(*=@Vgq912v&w%%wb!?2 z%DRJvAM2P^dXP*H@sudH`^EO{*Zs620N zLEp?-b5Lgz<*TQPDC78n?AxOx0Pinf)R&J{?@?p3)$I&<-r{mb?Jo`$t?Kr4RVU)DHJ5Vpa1B0_TXrag|QJ}{EKXSpH%h!6eRH{onZ6srFVZUFpvt2fG=Za(XyL?*;I$y{-ifjt<<^V);!KfE+fbMXgySFpWv|}wbS>JVMow@OjXGe;4c-<6Ebu;P0+*S?o{l zImMByw0VRY8)P+%g8IL4ch@Ixf2qGSYX6Eu*X|UGh=sLDEy9b#_)SMg$1iMg((=qd zJ18g0E`OAth%SFAvqaWHD=UjFqbjp>M0$i4?J;)7B)*Q*G-)2 zL@6iK#yFh{FV8>sp1Rrh<|{0(`WSKOAMc_UpX)qyxpnVYtJ8E5(^_0%NnSyWojglC z$E=!(fPgMBH;DX7r&;NE6O*G}aJ{qyfwuOa1!;s2td~P;hzW8}HPB}x zyxTP=6ROU2Nnc)Z_)kQEJ2&QD>XbHA*9K^hfW665dUKuP|D8StG(;Tl#2 zqvI1bZYe`q`WbDMY>AzHhJ$qMCDA*Sn|LERP2Il5hU@Sc8`Q~fYFu~BymE~L8}Jkx zfy{@t!!|AUGR_voGA)+2OJkA4r>a$JNqc^oF)Xp~o^%Z75n2|i?j^H&n~QS`-#D#wFp|GQJ31tKN2ss|Mo>47wG>bdBY&3Ng(0oShpb&e{$LJ;)0kHH&khIEt;$BMWZ z3`hj=CffBnwatbPdbmj1G5;BD{$=OZP=>B&C(NiIHzqE(jHe$q*zUQc%3LL!Yt@aIDpED69=~XyJj~onE&FWH zh0uh1I}LZWA^WOV#Ph*cLoRMmBR411F_)%;K8z`#j%$AC7Zc2YYgAkfJ)&{ikVYDp zX0ZerqnZ`t53Di?*C@nHg2KGV$bp={?VT!f{GE5F@+p@0SAMV*?em@Nl#Q+h;JsW`DY)wQodPuWKvJ%K_! zLPT7Pqzo=7T~zBeFWiJ(sS5wl0YPmvptcx6g5d4H$e&-IG@oDGCT;!B5$*RVP8Z_% z$i>42-fYgl+uu=~_+q&Tr6ET+5k~m;wnTexx%RwBf!dp8P8&=~0e=&5VQcC60~8=7 z)po5g#{LdU#qIwbycNOaB2?lcM1NP|Ix_Cw*!Z3}N&j97H;wQ7c3n}LXAvx3uHM@f z9el>!^Z7)%;U^{JC--$E>j^0C9}&cQCwL)58xtSb89Z*-=&Ltqhrm+gLhVVo+JRi3 z>tG-Mp}vNSQ`V zH26$02jgZo_<|(ydGo!u(fzlcw`aoAr=BTuL7?J?%u@wJ#b5(-iFUau@KOgyu3uiz zP+fBt8c!fl>51HY-rTI$r!45M+K~W(N3d{03Q~U|s8+qpkNTt$Mc)JWgwQGfJwo=> zpKmZ8sQL<};Jh~up4Zm2K5P6&=OjIIfiWu|A6?$osVkmh@Q?|0gxdt8jQd^lccms^ zJJ0&7HK6V5G{ExhG(fE@%kR+R^F5vQ#w<)X4UOL4c(2G&UL8a^)|6=btT7a+7qRwk6{K zjkFh$4_%V?Bnmw9_0b*|QoOj);o`Up+go{_QC3i+1qs_C}&B zH{$K|#L7aC1`0tO^myn5Yz!CTn(*mRpn z!#AJRDO(uxiCfA+^Iw}9x4ibM$C4CPbB;j`m*qYaiMiTVqfN8Fj)h>a^-hj*VEwuz z2bdixVjmeBhg{dM1?6>4udG`r@A|35TLO`o#rF8fqE9IXr5^(9y-St$@VJT%yMsjM zc(u%oc-{KmdpG>%66BQlP%-$L^o_Xh5F<5QM!E%u+B^%l*8NFjQ zd(|>~^~&y?D2Qmo)vgT-+Ty`J)Q3jvB1zxVhm`z@XO!CpAAcKeZvvDVfjYuYcZ6bAJ4{wV%Ox>;IJO>6D-}3*ngI zygc*aY9&zhAj!Ix`?#(jm5#bmEqpxtMXnjwi-9`TunbGTN4j{rw|Jta#_~B`x<*U! zuU3j5u39g5yckvt&^U^g&8UR=HqkezM`V*0&aDrqKfQ6eO9^nwD?6SEQ~J88L*A8O ze3W364+84qmX$I2i|7py84Ms{r_Kj1?`gN3xz|9zhuQeRT5O3!ElZV~c3>sLEa;h# zWp3QLq9EQ=#E%ksfl0sBI5}UzO~g+CdZF&{U_mM7+DXWd8+zgTU|~V2Z1XhmO$VV| zaDq%lG{^rpn+{uixma{lo6oW&ycfOf`w^Ok2Fc9s!M#$rSC-rVGZ8PRFzxAePN zQ5KUT&^-<9?3gAWDGj5d;GeU|XNO-oz2rYxpnDJ6nJ0}EK?c$Sv_iRHNk028q=hhr z^1o&I?hVy8p1Y`SU=Q$ZVG(?#o1O$Sbm9cIBzZ)9JperV^XD^{sEOSY7E1bWfj@52 zReVp%uiS#neqL={4pW*LAG*(5^g1N-q!pu@Kohq%wna({dWUF6f zn5J3z-|}wrM$R=b9lWjR>W`?R-3FzM?^36pQm6hn*==I-h0y^4n%gf43|~f_YzWB| z2q#pvaC0{89LIn2inKV^o~LZ@xOme)i9j0~6UuI}KdKjkFxSeP92rL%yW$w}{99SG zwk~CZeF0`K7qDaFAWJF z-;>L3a{O~tNiLvv#U=lnDc6@I0e}_q3g5RJTPu8C`;1dra7Vf)`t_t|JZrKuBd%Kts|u_k7x6+ z|Epq=1joz^<#XJh)Fu8`wU6An1EG}X@VzxrNW;-OXWQZRd5F~yV2Z5^897wGnq=uc zcq$`dF1AFTb%+YNr%in6BdBqbROVDDXXtu5t66ql`J8!C_Hk7gT|5B#4T`WLyPVU662q~4{Ce`h9jtY=t z3<~5#WJ!v1LMFoo86Rk+X5?+$iN4`d0*az~79Xi&+)AuTn+oKHBuRDi|MeJUWW!RB z{h5e5_#1i93Hp`pthJRK`hO}r4|uGf;h5kr&afFRPGO&8jZDFM5e5dcXMZoGrl|!zgj!!iPIR>={9D)l-GU9(p9)^ zpFHI`t)NtaQ&03y@oY%xpOiUXPD{5T(=|9s8@RGIM91t%e(-r<%lFV1>BQeY-z!&T zC{;bEd+89qD$lV?XZEWqs`Lo4J1D$lmuAh7vVR)yVzC%>J&0#8W}=t)%N_RkBpVgV zSH%W0lz02t6VG~r7=-5ga0C2niJc1H!e1@@u9TAy-XR-3@y^n1YFPA z^o|9~aWXTgwVYNY-l$)^$$mBOnv9RoJpR5T$_Or6 zSbhuUdZ*!VSH?Kxh0HszOU56vjXQ-+1l{k9oG)@a)-S&Q-L;FKDHA^P@3;Pb-L zn4#$Ogw40-Sf5q`;AQVcFGLYdC7-xR}~rFE;Xj9lqllW z3taF%!M55ux$aCX;3_BJn|r^7mmzFjf>fZvV}CVubrnCUz@bautJ1q%EPDNNkjkYF zGmDKoNWYC7np2VgUh<0VmdH==P2QFDQ?JB~-j<%t4gYeB@})e{Jkrm8ahctI zVKkVnJUd_NHU2W)%C|2ff)%5|=ScUZemQnpiyG$zuM6Z;l0U+HF%vY?8?MId+y z5QzQoZ&_#~J&`Ub@aW;sS1f|{+FX0J_ehJYjz9A0QBKX}2*zvikTV~NH#9FN;y&QW zOO4W@NxW=EU{t1eL?A)y&8`=>5od7=6$ngah=rFW9IjcGmDxR-zMfJ&{dGCxt{1}Z zb{5M6AydEh8-e$11B)wN-e2py9_Z7#$QQjfthdLSPV&x(v2Cv>?n1P@(6g16>WZ+*Ljvtp6JUh&osE3O5Ju>afJWT`hZy4=b+Wo zG`(fKVLG2fxCG74E~~4N2fQmhD*Wx(d66EwRL9CniTX&ga4B`Msz$%bgY1Dv&e|>o z&6sxuUVR;>F}r%(H1~a_W*x(6*EabXQ)h#wr}PZrC8tQpkvI4Nie>#al$pUOG^Kx<;=T{O$}wO9fi-t_N? ztUE3F3j~Aq3+s|4YA@{ug|#|s?>NPBGG?n%#p>r2h+Poov)oIF+MRHpGxsCdAnBov zpgVWp>_8^blNm8X2S@Uh9P%vx3n3pY<272@tR&e62Fw%tOlGCI4DxWFM>(23Q*LB= z|0cw!?aC1$1<8(}x`f!au}fFFagQ0D>BeniooHtX;JC8BCtmK!pcdYQ+<{f$^vHzX zh>3gJi+X2IeBT9q%M(#%t82akfzaE7KRP&-Tppuc6wzPj={U@~Yr7;Q-C^m&u4D~y^da~JM=#<>uRKB3fm z;jP=+v-p9YitebrcG?TZXEm4Q2|6joVys5hTc6#%z^ZXuG~wA1lQJX0II4jM+yRcs z<3}E37j|mYuFew@SmT*n(|r};&P!51fz%OEBspkpZY^rlb^M%Tev+NIN%>X2^n0fI zTz2B7RJj~dwJq6!M@+4C@I)7j+Oj#1ysNM;P^j+9Mm@Lhx+E(;PePr`b$u?s-E!QL z(}4}8BHr&Pay7%#i|`(ap#V#dJ?D`+#j>ynbL)F7FCOCXDMywNjb=`kg%PjXjq{iD zE0~t*H9avC_@Hk3T-lFK-`ei(7qf#RB6iwR6??2RA0)L;sSZU9h+fugI66+9+e^qV zpUzv*q>%L*$F#MOc6Vf1i=tm|MYIZi)~)Qg2d*XdTm%m|q7E>qt*yOUzp{Sj&b|r0 z`LY?;m!yNwX0^_oE1Wo=ZzNuFv#r1Id&Rtx3u%U|L9_%2cG1V3!H8dgw_7nG+8`1h?|Z4Wqc^O3#OqK+VRsK8v~!xk6E1i z?(~MS`#5or#z!$?t(W%)!Uj9a&lu|Meid$1a5=E-rIPURrWzCQqIwXkaP5VvJ9CqS zlamp*g@py;lm&Vp`7_R_$G~h?Y*h8WDnnudsRwMv7_JFfQ8C_kGJr*~8g8dt-%&bK_Lp_3bB42VB!Rna%yN=CyR3z#CSRUYYm#Q*n;(4_x4R zA56{BU8YUJ<;rqaSYHV@EnF=qJGnl&@}gqsE_(4Q>IFG5jtE97L4(qRrMom{C)mq9 zi0i!EzX~79?0p@wcVJ~_#)DJ$9xOW{zm_U%1qf*w*6rJEAhnnG=(=}H^NyuAy*wA+ zU0vPV*E-98Q1EIMCF#Lh;~j!hC_3^0>5Ye7CL!|ain1BpFrnRV!1(bHU(d*<7n}nP<_trqo89 z{z${leFk@Fk~)qgk>=5zUNABdu~p?*)LBEESl!codg7MNIa`(4<1!Iu!fP}oHby#8 zyoz?0wYp>I&R--j*w9>eJ`mLxW~=H=jSME{l94Yss(fn$x1}yNX`RC**Qvq1a)zRG zfJ5`*jnns^#8s{mGf6jjCp$PMSX{CfKg#jmWZkjJe_bk7^3hwhD}xq+h9UD)(Va^^ z4@wilOS=y?+*e#>xbVi*f@`=)s_df|n{t5vo->_`N4WDRp1N^GCEvA|>S+tG^y2$M zeeg;$;|@i;cV8;S`#xkhTp+8MIpFH;#pI#kKNmLVe_r;Wl<)q>uWtI)QHpYW-ADE< zG{feo)d#Ni!z=V!9|ZlgLKO4XE|B$pXiylEs0*G+dZ$PfPh9tSg8&+qBUV4Xs~|%l z=s6JxcC29;{1KQx|NP6 zj3|rCt`<@!T`5Pgn=hmxQ+V)rhXk7=4)sBX zB!*GCvyb&}G`HEaf0+67?y=&f-j4Wa>$+KT1qjHKGYVtP;%Tp?CL(1$ni2m>3!q` zN<{`iIR#-|*~T`~42D7sd(Ejym+nROspLxs)$#CU;dstuf2!JNVA^J>bs)rs~-`}U9XMO*(l9ur6a)0f<&jyA2NW5>|HjE^Q5>?^0*ETKY z40k$jqH`|tb0qPqk6wJb!r`$GO*=eD14$fr`?cVn-ACH&BtrPS@}{@gmpd9Sul0q- zFO$F4ZVX6cXqIagIdhK8I;VkiWGF}Jc_ae`?^(j8LAvadbizaE; zPSnZkT6{8ZuDtx^Yw>XK2DjLL9DZ8-8swuza)z!`+JWaHE{NSyexNfOOiHMGm`0Om z+JM$+!Ih=Duuzh&nsJ=fXOHkA@1fW;pL%eJ=k2b9WOotqyt#=-T_|*X2lt`&8ENAz z-5V}RawsK-3b{n3w|T^gdaA4)NQJtQd~I^>y5~h*mN&ff<>b%n-jb8Q(1RqtaEOmj z8*C+3k-WKDbDQgBfuq_5O8p+1-Guv@aIPyoN8OMt<2;YL;4-v!B${URv)OqsM54^@X6vX6(M2I|VMCoR5F<~@}9h^mX{x|DJrPiE-)$jix_ z)(4(UPjFT6G&9*5__>eYqGnJYUo~@Dc_Bn&)aU5UbgK)Wp;=8QinE*UVWzPZ`N{eb zp{^=I$lo@3DT1=XKD;K9C<)q8W;s}l6!@V@aFP9ru4$6-f;m%qg+CPND zn|)H^WESO&!u5R}5=U<6r0|N6F)SZGGkN=7Od+A4b$ssQdjtzU;b%;x6|?j}0+%^R za814g)#|X)Wx?akz&zX6Zd>$-uDXaZOyA}1!{l_jb=H7vQ=q@Nw)#m8B2QVGcUWY1-SnofGm$Rg+S9FZ z`uI@uEOpRVUaFc)S{bDUI{9Zic0Xd1?lyKJk~BMD<1p!`GIHFEri5j8VvA~-KjQL< zn;Io3`{3q%8BzV?WozAMu>DW;wve*fMecaMZDvgQz*UPE6Y862_94z zf;dd6Yc6n(laVSt4?TRr?|?T~@_lL|Btd#%V)5|P*mi=&8B@c1J!gmaaz-o(Me7_p z!4wxo63#cgr?{M#w69KN*r1QMrEKknoaEaOXBT6)IA5;*b9mL^AI2kyzn#tKec;a5 zeSbOaai%%(D~)B#ghx@#Nv`H&Ne?QliCHTIG!84pgEwfS0^SM|o2v~VqlNRMMq4xr z-UTE~77Dy?oin;LO1E5TO1U$lC$axTS7JLAp5$v?^QWtx*4MQ7t0s1cEzhPH9PK&${RQ=;Z(EO> zu{xKVN$CsS6t6%ZwHE60i@CQO9;%C0R~4kzIuc|wv4u01_`G@E&+Z-L>V~79Av_&! z<2&2OPaPtZs+e2-;)!ILSoy4G@oHgv1azpn7 z-aD%Co1L|kXWqRN8&J+Kdmc+(-19+C>4u1#hRC(z_pb+AIX-s990}3?^5(FmQhsIU z8&NVfwX3^?hm#okJWsv%IBX@b{z)5$qEfeYJb*Dod2&IIYb0XwR7}wBL^m6!u?y?k zb!w=3^@g^|Q;|XOS3c-?21OMa#a3S9GPWEvD!-t}bmXG_s8(Et<=c#y@0a;Le_LQ) zy>7DHE$hF(RKAU>eE1y0LA}|J)oTU))3bRJ7G>`uLQ-P~B)=WHAg(bTProjF`RMHG zjr&HPDn~o0J2@Ae$vXp9-d_$${qSi?Z^*^!S(N{?CIRqR@2$U; zlSHuvEq5(W4%wrQGDVFEw*^bSwHwO6o1xz=^1Sv!%~yxwdlafV@p*Dm$xVySD^0p{ z>j$;{Lcge_o*8l`E55DWk*MFyZL8}N%0WAFuR-!{K2Gd_k(nl?+1<*n10&i;j(ydf zRn)0J*FW9;NXpQpETD6UJ6z{V{Iu|E<0uo{u42kldqZ6e+NJt$3)l%f9Lc~hy-{XN9t39Z9?J z!k28UKfv;62ARSav2y}BgwJ;?PJ!n*+-(K25>?u3KgRW<(6iy;EKGaEGqBNfTku_v;=DYo>!fanx z9>aST+2tWE_o{f0EpK@1{NXXb*V?Z%8G2T(7}szoX^Ni{(o=byYUPt+6_sN=PCiF^ z)-csdxQ3&**xmip$9Q|I)_9T~wSC(0jr|NE-fQH(f}>>eeynk$c!G_zJ{#yL0}lSxQ{X4j@j45 zzB_wq55f=UBe~h3S`Va5?5&R;WettN zrKsqi8==%?)wiHn8z*R;Yor}WD$zj3G#VP`_U2f_g>zrc3zh3rMk;4sSm{ zV0bLv*|DN^q&#PZ{`z#od26Y?GVhey3LZ)LI%gvv>1yK!l6IL~bE=rDr`xHdOc@-z zYh0vgW*7cV!7@7wq+3I-#8<+nO_kn4!O@i8_~k~I4|kuMv7f-ld1Nozc0;C4Tqs`K zk)c1jPxU5`gQzs0lq-p5na7>H8;MR+*(^(9$4O+?rq91>Bt(@M)Ei_ym1FhFTJEMP z>vf0cS@ruz!lVwI9Je!fz8Vfb;Ao0HeC zr0g@YJ(K#1ENhA+Ky!BP?oGMfN4eKl_LvI=FDeANsu2zvyPwC~amh^#XI4F2Vy&0G z?psl-C&jFKi3Gh4kB0*%TdQkyj8z|-i(s(m(2PgzzRUN83HFA}=p-w~+YG;$-y>cW znA4FI%cd%wYxT7#P#y1a*z3bYf$gFDM6w?9%?MKlKDX;0w|&EHD4p*9F`lqMxuF!d zQC{~`U;wXVq?B_U>eXuQx1g`LmLj-3uDvY&s<`{&j(6$4mmLmeh~KeQ6Z>$HkX|}* zSIG#OIJ0{+D(uvW2hx}Ez<;fLLjK#ngCimr zU(N{xq_XvxR%^OlV|uI3l9ovD)+HctRQBum9b?grn#GfZ69;QurzPYXw3rWHUDLj5 z>fGl>AMImeDTqI(XkvMk;`GYui|g@~EBM1)ao40e$xfBA&E-{#1*DsI^CALJG*3Dw zI8TGWP|TuH>9!~ur((gvJoJuJtCH;%B8TxVOK_A_Jli_2vi5akbZray`*&vh4mXqe z?RfpZ9e4S{Qj?okQ%8BTi(llL0PV24_wCo8&-6|I0V~ z2#48BUT?5f`e>0&GR(QhAGt5wdeba|>Ah-`^d!j+FS#S@hA(P8eSE9znBJIGU%7R+ zxt^#h`A$EuT|&c~PXUKTeH-zr6wr3xmSCA7^3)%J-aKNs^E?z-70+s-@ww7_N^ zRT{@t&NgZB=_6x)rEMs+%l>5-sp`bjpNHzEeNK9Qefn&qx#)x5k+o2zOsTWMkyOjp z(^SdaE2|G*i+y;Q7o^?J;q5?bBEsC8QQwY`Te5YuW5=REn> zTEoZ0b=EmsZ75Mhl@HHs&_d_E_&)l5B!mdW;avy>8~nUi59R5Abe3~=v-)p$#lTcg z&xp?a+Vu%`;TT*Todew!bQEf{H#4)dLurLZjydob2VUCCa%ke=WlDaT^5BHHUWdoN zXT(j1WQ5ZSL|ztDI?kJ>z5PBjdcgb?SLD@v8_&hG;MInE{eEvUzHj&wT1pDM%cq=C z>1eUlJ8RQkQ8J~$B_@@8{QNcFmrqD+iz3N1EMFCQw8Ws!&w0=3TlK3;NJ!y!IEuQo z#OC!5oL6}C{0+7J3F)I(%I?^DGz)5%iY+FK2){LLlJpbSxte;=C9LMD?B{ghyS>lO zhtg4hoKv&weTIYM=kIy@l-x1KJG!~d0*45?LTz!3s!l!F5tf9QEi*j9h-(}X%FNhZ zV%@vz#pCHuaYty6?HJkfk-oyEovV1b?il;tQrj0t7cbo;Kbysx{w$Sim)Fi-8Q!xy z4?QZ*xtJo9k2|V%E>-06vt8bgz|Ehd+}b!_56BvK=USXuWZpRBFm=5=vE$tfo;`fb z^hYTLKNN@Ueubm3=cX0!qRv$v@w?|clarH!W019Reg(b6saI08DB4yMClV6Vh%I;c zE?#n>SbVhqQuyU(4?`M$Ky{lxe@T1>M~fE?9ezmHIGU6YCQStI+!a z$`lKc%E~n;6NjA!E~Ew&YaaNIz4vm=-r?KHmU8IR>Wh!BkxvhXlHcAlASt45adXC- z!Eo|*s|g8*Qq?gHCZ&sr9+T#DE}7n|l=h^_#$`rIcdEnnhGzymlrn=+%HcBXPIty) znDc}M3QEDdu*YfTPrV?!(f+~nt=6$l`Z(*C!(P)&&735=O6JUrTnWylz-t(x@0#~o5wwYmtRE6Ez>NIom%((%~`0SCq)Tk_)v|xe!)I>`$k$m&=VM zc3)itUQo${y$722HtjjHQ|(E&9-hb}YSsJvH%7R3e9CWOYDHeWQ9GgiWTV*Q6VvkK z-l5*N7Ije-aRhJUOzdaojD@ZQGd6rKZj^fPCN6KJK2=?Lnv}mUw_4(Ag4n$45$|)J z&mXhT**4X`XvJT$Y7D19st|H}4MmLek!KYVjqzm{tH0SPm(-;>rcc^+GX3H-o|gO9 zI)#Nj)_%9_cN=z*U2(WOU08qC_vM{XXBsQ({D*_Ii-IA$>**=|R_1%NI|HRz8v||_ zSvqo}sz+`E(HON7!#1;6v24v`Di9uVMPa!|9M;@C$t zt7zcH|9vpKIM?yY`LG*BQPzB~bY^|8uZ+)duDR@07e233@3EIXg0&%|X|4R3H3jn+ z?omR4{VktpF2ybjSFqA2idHPhs_wd0*?-VclGERhzdCy6)FnUZ=hpbl-97YP;xf6D z2C3%xPn%HbNharcI&7-(Psy8Fam-h0 zS?Ar@v66cEE2HMjHEPnDg5k`hnu}}9&|q>{u14<}2qwqC-zwZPn8jqf*>j@ThDa&V+&lS`Z$@k(C~$XLGf{cGoF zBmT$z?`Q1EXSfBXlVS;YrwvCIoR9k1RvTX}H-F4`A&G~8DYa>Xpb*Y^`t5bm|uWK29(BCb2B1D#@A*fzEEu+#rEU8-O zv)P@tT^req|jiPX8j>H%ukak`pJ44DZx!qY^R( ze_|zy=&s`NkCek)Zh;N;&Y3&3cCo7uh`OCfOuUAhvg98eDZk2T8t@c3TBTQEQ7S;| z?01}yvrFP+S$ofURqB)J8y6PIgziiYchN#(Vb?kC6$^`iDu(-{yXqt|)L>yr}zl zz(|wd0)=3;7&kNVRsG<+E7xwO)3^eT&digMXUh8X#-pM^PrHRrn_A=NHAJ1myLN%BW71$ct=HTX$Tz?c3 z%F3j_OHxLn$9gsQs-3&$<3X->@`E{-rV~R00(jgkNoY+{A2OLAGAE|en*QW*}hunE0}l*Rp=xShwTr?M?iB+Um@i?G?)U*rq+;IQs2E|oLZj;V&^I(bH+8oo~3C7jF&sdG^IXd ztm3(yY@8?E=Hcru;pDzJnOMk7M`T;}yg49qKad&$T- zc@0xCmf3rW7?b2q`N;X*eC(UpVD{PXrtQIcPI}i{_3nK>_G8^u9}L@)-nIoi#JgiT zZGKJbscK*9^xh+dFVnB9DszyeXB{R73($k+{Oj&Tp6TUJWZNuOQ3Q+6aY{lW==hu5 zZ7GWfPg`kmu1&tQQz>RkSN$mdNj!(b?}PlQs~6iOEfkK*+lf8!u~Tr7k{ghZ8fp9f zZHZ`avs@3Pv7=-1*^O4GwKSX>2|SajkdvSFSSn=Ja>O&;?=1a14DOh zk00Z?c(`kIT6ra;leU{OmE^9t!|2y;^jbmUwaJ12cj2+Wwc(|~ho5)&2xpX@ z9VD_2*=39OQq?mt@9Mh;Mdvw=T_02^+vi`67P@b*C4<+A@u4CU0BM3Q=+J z@zyn!C5{gpIDwKhGLy?yoE;HmGQ<-3rQ`ZGLJe`Qs{3y;x8;s8x^Kk1koYp%-!fmN z(xO*()a9G$Mo_pkxA`fj`KpRI-;?8ER~mBbhKB-&v*!En(~00{7Rz6?e5+9=l528a zgs;w|PQBP7tCjZftl*eN(x9mA!-qb`Bt=9!&K+1d9W`xKR>WS_WYYBd!QPRv4HQQk z|CN*un!u_@jHkGp5#Nftw9YFAlDm1`?490OW!91ybW&cESc?Bv&b73HNH7(VJw^Df zzY^!PcujqwgW1T(7~^7xoc9Sxar%?GB(;_F#KTRJr0Jbr9Ma0nZ(b!eMhaYVtSX{d zs%KcSCt@Z&cQ}?=FE%FpbSX|q+^rV__rG5mkkSA6SpRzvWu4ip?C@CLkOzrHW6KM& zm%q~752&HqJ!>|S^)^{_{BHM6mg0M6Np$T2P5(Egq+Dt=HP~3OI+8-8nxAMaH;T!5eOm#LOP633rwgsznuvD|9;RLBgpdM7_m9h5Bg8=HBMd@|27IvP-p4*algmPbo)8MeG5(ibV<<>4%#;OXFt z^8AhaTNiknIJ7u30B7U!NBw^n0Q&szLl@WyP6L7hjy6`-TbA1Ws}KTRO7M?THa|rC zKMI@uQP|eO85}j-vM@-1{*d?VABElB+?=;83KhEikD{J#eqJc=KME85FfyIwp8J;I z|CRylYyYom!M{0SZ~iF!UyEg}yu3Y;*52BlZvFv3=kvci>L>5TAeC7V$l?gWb~qD+ z7Y0sX{?hw@JO!7Zy$z2#N#aXM4tCV*ixCJO2(klsiuq!&s9<&8xD4vz(Q>>_i3J78q~`DfUEk#oXmPQznS&d31yZG_e7-zbv= zT~S4jsZ~TQEOqmE?AFp|;;58Zf-{MwwwTE#S49Q2Tr=8yulj2NnRes_3z;d0V$XXRqfOYT%kCfo}Kx>Z%_` z;4ur&b@NlTftOL4rnos6K(3o%w)w$4ENdGb_KYv$sDaUqzkomhW7_<3!cLRmY!1r* zJd+$SOr;-A3SKxv^`7>N*AWO3W;_JTCJJrsI|ToN{RGL|BOS1eoprHOt~S7hX990? z2tw;``-`UTX7>}wzWJNJy(bE3qvGal^XvRJrbluUAe!1cfk4Q6;A3>m`3~$gz_&d{ zzBy_YNa+nE=7LB&QHG1DNFbvw%G%A-hF#Og*%|vdFLY|?VgW0tXK67lXa8O7PDa>51=;caU4N8D3z0WN;iyU8Q1?NRW|%;qCLQXro30OtwZ^5&Nl zW-YlTQ_&X$4Rr@EZpMnr1xJgD^|GT4Gvb90zM%#d%n~$e87QrlGV~bB`U~9Ww=LMn(_!LU<08`8R~+7vo2wEsGt3v<5Q`FE?oNj49IZ*ozkFGZdSU7N`DI`bWcN;fi~CLGyXc)hck^7;{o{+ziZ5STFZI{eQM4O`zHX6^O=0Hw<@bo@(E zzom0{tyzw zXh;ZB zC+?g7r%rf!T1?0NRNz#RK=EL8V)M%hlOM-Q-wc%Kxt#2OQ4^j|L8@(b0BGzn&=_?7 z#sBFa_ztc}&w#CK;GNLesR05?9wTN8@>c$#2C&(;HGlY|$Hioj|eCaitPCZ zzMG4?GkBOOz1e6ZB?<5ExS&FeXt* zQ%-RFLmen<2N$IC*4oNto9(*@Mw$pnYoYdfm**d5vmvG;@z;D7v!Z+&!oKJe??7?Uge zGjPlofp`a8hX|xE4k&ET*RuyX-PR4T-E%}V8dxd=J*Kr9tNcR+-kuKIsp#{3*^&?F z*r{(+8O6G6IeRl(8ZE?;>d{)xXTSdaLJ3eZu_({rl|ghcK4n}Q0u z{?bpvxqqu*D?0h1i|p-yt^^_uRQpH3>e=Qt5Ok5i>gSAH!37;Ox1tv^-s+SF+TRD# z3MgIg+V*sGEZd4c+H#QpB^XjRAc{c)@<`S{(7`IDK8Tbc-FI={s+y0-O$Nt6BR+se zK!?`u{C}wCM^QZRfeDdd|*9K{u;iGJO6=?&P}!wztZE2 zOF3};kN#2*1^GYVe`F?G@lm22*)2dlUx9j{=0n50Js<5rx1xI{>|ZMa^i@EIPBgeV z{*At^duEm`owWw~HwQKZX0DrGPS^qPmeghh`?U@%>*H+?pNQO1UCF`$sW>%AWPdNt zp&jMJgIGYSuC{LQFt4~u$1X5~lhyqB0MIY7a>8&$VPIgr#0zF09KBSMHOa_&9IRW` z@?q{el4!$_6yy{=J+Uo;$hVL541gx0w}tlq5()g96Ltv&#{ywr%T@{HZH+}`S859A z=7HZC2i^gyvI2WJ-hVS}c$G1ZA)VR4X1}}y=B`Sw=N2$+A5WNFI<)3AC%?>`1`9qr z!OVdZ(pvEZIG~HXgS$P-Q_st-$bNjt25wLvJF|8Dm0ssKjit%fVHCS=1Mf8gh9FdzQ)^hk24DkaJ8&7t z2o-vRx*5#cpx{wL!oU3%dM{NE%rc!(+ktI{rn15)LBu$Kxj9|C3me$a0qYp=kD!hn z0;awkfH{gzF~i_~cpn9nt%Iur{6Q@A$b45=bIb*dAR^Fy&|#ZjPFSic4D*KvQb3|y zfR}{t{^Lt>(@%gSsst$()Um0k!vMkP(L-7LfGu8_y>qmHL3W%^9s?)I3L-Ddrg&&a zsci_u)76vx={w-1yP6i5whWF4#nxahp&OXP(NG>fU|M1U4oJC!7Gp6?-x@S5cXyP{mMZDXs3M*M!hZ(BLl1VM zUi^VY?@G(NIs3S{V(&rJg5;AOAclMfjuo0(oq72e%EbjcD4i~==`=_XBK0s`(xW$j zfm~hXogKjW9$9a1EK5fHImJ?6f#w*2=0G&s{Bpt~ad!N6o#-YipcmY+r)hSY#|Y7V zKn+OH1f~k)^qa}!50VSGA`6a5V24rPz50j&SmCrTrdZd37BOMI)*$J#g&#U7iul3_ zVA4Wh&JI=lN5Q{HHdb;ra7RzPpQr*o8yXliJ z%QqGPPAA~7Y*NrVLx5F1h*A7#Bl>+??6e^YTYLpDxOV{})bgjzF=?JCMSqkv3XA?Z z7tW}40*V5#d@hJwC<_eIPrLtN7wqbCAS!`lyYT%G?*72EW)}kSoDS1Kz~9eB|)Hgy>yNnD2@X(8EPSR5im4Gf6xuC zNN0KUOaRLiPbVfF^$_3$K+i%o*c%Ch{9`5UM@PfAI>Ef%un|BR0dzm4TZW@ypg)&C zw&2Bw9(n#5@G1chI%J1pVR*ooppOH<%QoN~U6s2BfiMA{5*k@Z;^0|ec^Dp*!Ia2w z3be1s4&(=rX3nR>qW-!vfVHPl=O-KA0lRSmo){|S-TUxptWw&tP44Ig%2NT#gQls2 z{a8`h4i2K-u~Pn7&o=;O3dRUD+{_Qc1AjzlcpZj>@z0=lSm%I$hi+}`dW{A4>v{xG z9y{9CqX!}OFze>5)j=NMD(wu=Qt~e_H@zLQVC|9e8AbtF^tSo2x@0O5f8T-loh?zf#Br_VfwJ3ia*f z<*-n6Ac9AoNs*S#2dTn0;H024w#4?|sGkSAWo?{Mo+1a~L0_WB=aNA~od68l7rvdC zAh68l4ED&;q1@I1%eaw_9SpYPK_HqmFa^`-#=?UaEM<>3;thygI&7HAnqR{Nffxv0 zlCpNfl8hISN7a&ol{yht%&@g{H}R$u`>8B|k_YS2SV%}^icvGrbqZja2C8;#78sJg zCkjjd2^#6nPl2rL99XP^f+nS5KxnnYt0{oXq)Q+49XPv!04c-fmlKA6920c>1UTOh zkGZG&nB_b0^V&cjXx<*Lj)~C&qsPG-d)qGDWXG!mJ-5R7^P2ZBnR3EfjWAi*DWvKg z#H5_~W3mI^zOfG6l|n!ZCBf}RutJpj$o z2qXR?0(&q3$JgOy6T!86MhxUaPyV5%<#gdU34QTH6oU zRTKxs0LsrDA$WOix^ooY1hT||84a{$PM7`yZMkM?^1}Qf`nduxKsx}jt>cM*k$z6F z&^Jp#6R}+=MTeoMoKBhUpph4W#X=kUDG3V|zM-qo&BUstk{Bpvsr~L)O zs`vC+({Jdbp6InkmQ4nF0PWBFO>zHu`v$$;gI>PT!nP6FaM@ja2GG=j@}UmmSm9qZ zb+FLIjy-q@t_5v7&$M$17oo78;g5h!BxlIW* z!R$c-^d>ZSOr87#hkgnI=1MHqROcAc+DRtR^gLiE(DPt|pSQ$9ZpC87$J4e=>;-s! z(AfteW_WF7OFZ;qELOhJT5Bx&psO8-rqJHou@1vW&+mU;wf)aSUA2SEs=NSj|y80#(=BIXTQtlUpi{>fgT_X>du z*fy!?k#0x+2ltoR3jVms@9g6R4j;f<+8cGR1Wqv6Pl0}eHl2|U8}R4-aCTvMVvNrv zoNhqe2bvE?!{(P07QwJJ5uP{B8PAxFj!i&9`Ar(S$yb=*cnWf|&fxtRd3%(#(|?lw z{XcEx+;<)gPbvp9ta^|~Li;Y6`QIvpZwPZqu~{)_$ZpUO=-%Ju1F+ox95pyf;E7&N z1CQ+dY|oYfFSnvQ(Gpk)F$hgypM>Ux4iABkN9;tOmFkM8YN@Wkxj z;_Lc>9M#pvDggb?0@y?QsUdjw+U|YMw?HtO0-XyLnd{hZ_OE-Qe_S!g{+tLp86bH* z(I5qo>%jj*XXNyX|Aa-4et3yx!m7f-jy56;XoC}?jZ6)A^v^vMa3fv~h2;Q3TgWv# z^yb1C=saldlB@rl2@c13|L`8@DFD21k7h1c(tyQ{VIUbRM7SG97$EeD@XzOSu;Vhb zP>JA(20|C85!%)j;P8L-{13UHxGlEYhS+IUbwFFeJqV0cCdw6qiB5^YQ6225OxpuZ zGkP$5N5BsUq`4Fx7%cRq{LO@cUEf4o5nh82D;yQo0FDQU$O1vSdTxb;r>O}oiBSXJ z!UB{CU2|CWf}x>%0-Fy`%*Dy4Y4DsatirBt;V=rQ9lfdoJw7s9h6w{oQu-M`+0oMd_xn#T zvsDn0%|1<)1xA=6$l{<$^o_gN>ws6Ha#PFB8sKK>fFhwC8(Is)+G<_)OQSgf4;b|f zAOJzd@@a)Z{TFuN#ZqT`9V-Cxn|*)>jgv!@7_48Hy3r~$bU=X@J3M`zv7M3moX_aa z2&T1upuf2wqeABs7Ov4j9|vVm2OD%|1}`k0>a@27=<_q6&ksSMW#6ztQC4Uq-Bc4#dtM-U=@ zJd^~lj;fiG{rX@IA8LqcZ(WT428ZXWPUcs}0+T2PKTtb%R@jaU7JR@#GI(zMs9S&w zPzL(xKR$@b$teCC7hBSPi?-*DxbAo@`2%XBD_L*`= z0cLjSxiQT`N8p5eeg5(z zcmJsdM{okuGl1X60b31z^o57Rz^E?+YaKEWvvKMEKWhLQ^Rf0upJKp5QdEJJJp%;O=UN*?|5Y3n2%9wgErT-t9pC zpLh!!2QT!ynl^Ct>@eAI0dLpgz^e@!zU(vvY&fj;9qq)j=RTKlp)yOHch1*cIjP4F_)B z-}@B-plcw;K<$$!LlZI9qTXAK@PI{<%9)+-IZ-d0NOs?JyE_6Za!Xc;=f8z z_5TV)Uj{$W<&ZV7YW9D?w`@Yai$td-7{o3hszGxowW#fD0e8#&VMi{KE@HL=U>Cpy z396aJm~FA({ULCW8(cY10bWPI@vkVh>&4rg3jj_Al53FfY<@XmBp3b#u7Pxip}wI- z*1iR-c3?8lbYwK)U#P$4XMf9^cXN7*4qO0cAjQFO!W5VOg}fQuVWfU_^L!lVIv6?N zXxG6_O`z9f!T7VH$DD9DUHTU&mI#VYSkLP0${7a+O#>Dypp&ke zl;7wd?{>-kd=LQ{z^>?w9Tnvo4rsFB8@*B#C?j7J&fX863jd{Bpuvnqf$P>l{6oh&VVVGYZ5(A7g{M z_>89*5M6Lu2FzX2YGVKI_bn#MXn64D`D-O$eWMcuZm8bc+OWgn1zm^~)zJWgk^;T$ z-)t+|Mcy05V1X@7^b!mTL;Jcdd8F`0%pHnd<1eZH~i$#m~uJ(Bkc1BHfw+vDJ6GJQ8$J$Y5t zEsu?$;f|_E4)o~pkJU31o~iwbkZCyB@7R}CWNNJoOVLD-m*TFRT7Mrogl10 z#k+)$4Y$>VpKq>dmH~KC7LdYlK*S>@hvNZ*{OJSX-2+<0OkUnjG_a3^<&|o5 zUohi%wWH@V`9N{dV=KIe{)RcgTogi6mOD~wi-Cd20%d^(kIgS9EKckP1^s~KkN2tI z734Sf`U`l<3~>oG3mPz-6n>XSFNC76J$X8S`!+DU;Alxu53h#51IyS)PhxJX536jF z&_?iMSry%JKtI_2!~hy7uNB9OcI<7kn44@wCIbpI9upko+c+jnVEZ3>Q@OclCFz}zvFDI<}5(ETd#1C%zKE*a9HbD&$m457)H z?33SsA4_rD*&#KG{Ob*{gu-hB4s&fvf)3@k+BZ??ElRWr!b_8!=MZEI)`brN#X=1x z>>CuZ&6t=H?@~VvRQg#Ovso-uRKH(}1q0!@f}Fglm;g9a;EsOl8}(NXAX3?`@Co<{ zcd-1+25HvI18^vdKzj>$%`JfRV%P1y0ifq!Kr%UOKz*YvfF@xFPMiRvLk4ILx`d@} zO0^kIJ%9cIoj*Mpgr}c~+ zSRAXsTrS}B-j)t~(g-axi-d?G5%$uIbm8|utcyg2>Fx7m@BWC lf5`GT@Ou9iz~6t?Qd^AxBoPRNCXOb~PG{g-UZI`a{{TD99O?i7 literal 0 HcmV?d00001 diff --git a/jars/readme.md b/jars/readme.md index 0752be7ed2..e6a930809e 100644 --- a/jars/readme.md +++ b/jars/readme.md @@ -8,12 +8,18 @@ oracle:[下载](ojdbc8-12.2.0.1.jar) gbase:[下载](gbase-8.3.81.53.jar) +达梦:[下载](Dm7JdbcDriver18.jar) + 然后上传到本地仓库: ``` mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=db2jcc-3.72.44.jar mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=ojdbc8-12.2.0.1.jar + +mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=gbase-8.3.81.53.jar + +mvn install:install-file -DgroupId=com.dm -DartifactId=Dm7JdbcDriver18 -Dversion=7.6.0.197 -Dpackaging=jar -Dfile=Dm7JdbcDriver18.jar ``` -说明:这两个驱动包在我们自己搭建的仓库里有,并且这两个版本的驱动包在已经在生产环境中使用,所以不能很快修改版本,需要做相关测试,我们会在后期的版本中修改这两个驱动包的版本,可以先暂时下载安装驱动来解决。 +说明:这几个驱动包在我们自己搭建的仓库里有,并且这几个版本的驱动包在已经在生产环境中使用,所以不能很快修改版本,需要做相关测试,我们会在后期的版本中修改这两个驱动包的版本,可以先暂时下载安装驱动来解决。 From 6340057c91f5b8c3699e4f43a981aecb811b65eb Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 20:09:45 +0800 Subject: [PATCH 034/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0Restart=20Strategies?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README_OLD.md | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/README_OLD.md b/README_OLD.md index decf8559ca..144b82a9bc 100644 --- a/README_OLD.md +++ b/README_OLD.md @@ -192,7 +192,8 @@ bin/flinkx -mode yarnPer \ "setting": { "speed": {...}, "errorLimit": {...}, - "dirty": {...} + "dirty": {...}, + "restart": {...} } ``` @@ -281,6 +282,19 @@ restore配置请参考[断点续传](docs/restore.md) 注意:该日志记录功能只会记录`com.dtstack`包下的输出日志, 如需变更,可修改类参数`DtLogger.LOGGER_NAME`。 +#### 4.1.6 restart + +``` +"restart": { + "strategy": "fixedDelay", + "restartAttempts": 5, + "delayInterval": 10, + "failureRate":2, + "failureInterval":60 + } +``` +* strategy:重启策略,可选:NoRestart、fixedDelay、failureRate,可参考[Flink文档](https://ci.apache.org/projects/flink/flink-docs-stable/dev/task_failure_recovery.html) + ### 4.2 content ``` @@ -331,6 +345,7 @@ reader和writer包括name和parameter,分别表示插件名称和插件参数 * [Cassandra读取插件](docs/cassandrareader.md) * [Emqx读取插件](docs/emqxreader.md) * [MongoDB实时采集插件](docs/mongodb_oplog.md) +* [PostgreSQL WAL实时采集插件](docs/pgwalreader.md) ### 5.2 写入插件 From 548a245df4f8cb28f699efa9be786de9681d3fcf Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 1 Apr 2020 20:29:53 +0800 Subject: [PATCH 035/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=89=93=E5=8C=85?= =?UTF-8?q?=E5=90=8D=E7=A7=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5564c9ae45..73ddb257f6 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ 2.7.3 4.5.3 ${basedir}/dev - release_1.8.5 + release_1.8.5 sh From cdde2a6b3a8cd09a7491f600c1a33bc4156296d6 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 8 Apr 2020 22:09:38 +0800 Subject: [PATCH 036/136] fix npe and load local keytab file --- .../java/com/dtstack/flinkx/authenticate/KerberosUtil.java | 5 +++++ .../com/dtstack/flinkx/inputformat/BaseRichInputFormat.java | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java index 15a0cbcc83..c89f0b2274 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java @@ -226,6 +226,11 @@ public static String getPrincipalFileName(Map config) { throw new RuntimeException("[principalFile]必须指定"); } + boolean useLocalFile = MapUtils.getBooleanValue(config, KEY_USE_LOCAL_FILE); + if (useLocalFile) { + return fileName; + } + if (fileName.contains(SP)) { fileName = fileName.substring(fileName.lastIndexOf(SP) + 1); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 2b913843fb..64bb73f2c3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -71,7 +71,7 @@ public abstract class BaseRichInputFormat extends org.apache.flink.api.common.io protected FormatState formatState; - protected TestConfig testConfig; + protected TestConfig testConfig = TestConfig.defaultConfig(); protected transient BaseMetric inputMetric; From 25a32b1da381b2537229698bb209efa03bdc2d76 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Tue, 14 Apr 2020 20:55:33 +0800 Subject: [PATCH 037/136] fix --- .../main/java/com/dtstack/flinkx/Main.java | 11 +- .../com/dtstack/flinkx/options/Options.java | 2 +- flinkx-dm/flinkx-dm-reader/pom.xml | 2 +- flinkx-dm/flinkx-dm-writer/pom.xml | 2 +- .../flinkx/launcher/ClassLoaderType.java | 21 + .../flinkx/launcher/ClusterClientFactory.java | 2 - .../com/dtstack/flinkx/launcher/Launcher.java | 48 +- .../perjob/PerJobClusterClientBuilder.java | 58 +- .../launcher/perjob/PerJobSubmitter.java | 44 +- .../deployment/ClusterSpecification.java | 259 +++ .../flink/client/program/JobWithJars.java | 161 ++ .../flink/client/program/PackagedProgram.java | 773 ++++++++ .../yarn/AbstractYarnClusterDescriptor.java | 1752 +++++++++++++++++ .../flinkx-mongodb-oplog-reader/pom.xml | 2 +- flinkx-pgwal/flinkx-pgwal-reader/pom.xml | 2 +- 15 files changed, 3079 insertions(+), 60 deletions(-) create mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 7e2181d908..47e20d2b28 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -39,6 +39,8 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; @@ -46,8 +48,6 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import com.dtstack.flinkx.streaming.runtime.partitioner.CustomPartitioner; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,9 +98,14 @@ public static void main(String[] args) throws Exception { config.setPluginRoot(pluginRoot); } + Configuration flinkConf = new Configuration(); + if (StringUtils.isNotEmpty(options.getFlinkconf())) { + flinkConf = GlobalConfiguration.loadConfiguration(options.getFlinkconf()); + } + StreamExecutionEnvironment env = (StringUtils.isNotBlank(monitor)) ? StreamExecutionEnvironment.getExecutionEnvironment() : - new MyLocalStreamEnvironment(); + new MyLocalStreamEnvironment(flinkConf); env = openCheckpointConf(env, confProperties); configRestartStrategy(env, config); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index ee582f08b0..804f983bdb 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -43,7 +43,7 @@ public class Options { @OptionRequired(description = "Flink configuration directory") private String flinkconf; - @OptionRequired(required = true, description = "env properties") + @OptionRequired(description = "env properties") private String pluginRoot; @OptionRequired(description = "Yarn and Hadoop configuration directory") diff --git a/flinkx-dm/flinkx-dm-reader/pom.xml b/flinkx-dm/flinkx-dm-reader/pom.xml index e7e4d41da8..15ce92929b 100644 --- a/flinkx-dm/flinkx-dm-reader/pom.xml +++ b/flinkx-dm/flinkx-dm-reader/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/dmreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-dm/flinkx-dm-writer/pom.xml b/flinkx-dm/flinkx-dm-writer/pom.xml index 8722439eae..abe284fc04 100644 --- a/flinkx-dm/flinkx-dm-writer/pom.xml +++ b/flinkx-dm/flinkx-dm-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../plugins/dmwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java new file mode 100644 index 0000000000..540db38056 --- /dev/null +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java @@ -0,0 +1,21 @@ +package com.dtstack.flinkx.launcher; + +/** + * company: www.dtstack.com + * author: toutian + * create: 2019/10/17 + */ +public enum ClassLoaderType { + NONE, CHILD_FIRST, PARENT_FIRST, CHILD_FIRST_CACHE, PARENT_FIRST_CACHE; + + public static ClassLoaderType getByClassMode(String classMode) { + ClassLoaderType classLoaderType; + if ("classpath".equalsIgnoreCase(classMode)) { + classLoaderType = ClassLoaderType.CHILD_FIRST; + } else { + classLoaderType = ClassLoaderType.PARENT_FIRST; + } + + return classLoaderType; + } +} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index c7dbeae523..fbbf7bff38 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -141,6 +141,4 @@ public static ClusterClient createYarnClient(Options launcherOptions) { throw new UnsupportedOperationException("Haven't been developed yet!"); } - - } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index be2a11ba36..19dc85366f 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -29,10 +29,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.client.program.PackagedProgramUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.Preconditions; import org.slf4j.LoggerFactory; @@ -55,6 +51,10 @@ */ public class Launcher { + public static final String KEY_FLINKX_HOME = "FLINKX_HOME"; + + public static final String PLUGINS_DIR_NAME = "plugins"; + public static final String CORE_JAR_NAME_PREFIX = "flinkx"; private static List analyzeUserClasspath(String content, String pluginRoot) { @@ -88,6 +88,8 @@ public static void main(String[] args) throws Exception { setLogLevel(Level.INFO.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); + findDefaultPluginRoot(launcherOptions); + String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); if(mode.equals(ClusterMode.local.name())) { @@ -106,7 +108,9 @@ public static void main(String[] args) throws Exception { argList.add(monitor); String[] remoteArgs = argList.toArray(new String[0]); - PackagedProgram program = new PackagedProgram(jarFile, urlList, remoteArgs); + + ClassLoaderType classLoaderType = ClassLoaderType.getByClassMode(launcherOptions.getPluginLoadMode()); + PackagedProgram program = new PackagedProgram(jarFile, urlList, classLoaderType, "com.dtstack.flinkx.Main", remoteArgs); if (StringUtils.isNotEmpty(launcherOptions.getS())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getS())); @@ -126,19 +130,35 @@ public static void main(String[] args) throws Exception { } argList.add("-monitor"); - argList.add("application_default"); + argList.add(""); //jdk内在优化,使用空数组效率更高 String[] remoteArgs = argList.toArray(new String[0]); - PackagedProgram program = new PackagedProgram(jarFile, urlList, remoteArgs); - if (StringUtils.isNotEmpty(launcherOptions.getS())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getS())); - } - String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration conf = GlobalConfiguration.loadConfiguration(flinkConfDir); - JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, conf, Integer.parseInt(launcherOptions.getParallelism())); - PerJobSubmitter.submit(launcherOptions, jobGraph); + PerJobSubmitter.submit(launcherOptions, jarFile, remoteArgs); + } + } + } + + private static void findDefaultPluginRoot(Options launcherOptions) { + String pluginRoot = launcherOptions.getPluginRoot(); + if (StringUtils.isNotEmpty(pluginRoot)) { + return; + } + + String flinkxHome = System.getenv(KEY_FLINKX_HOME); + if (StringUtils.isEmpty(flinkxHome)) { + flinkxHome = System.getProperty(KEY_FLINKX_HOME); + } + + if (StringUtils.isNotEmpty(flinkxHome)) { + flinkxHome = flinkxHome.trim(); + if (flinkxHome.endsWith(File.separator)) { + pluginRoot = flinkxHome + PLUGINS_DIR_NAME; + } else { + pluginRoot = flinkxHome + File.separator + PLUGINS_DIR_NAME; } + + launcherOptions.setPluginRoot(pluginRoot); } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java index 0f6bb795ca..1e16bae775 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java @@ -17,13 +17,11 @@ */ package com.dtstack.flinkx.launcher.perjob; -import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; -import com.google.common.base.Strings; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.common.collect.Lists; import org.apache.commons.lang.StringUtils; -import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; @@ -37,10 +35,7 @@ import java.io.File; import java.net.MalformedURLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; /** * Date: 2019/09/11 @@ -48,30 +43,21 @@ * @author tudou */ public class PerJobClusterClientBuilder { - private static final Logger LOG = LoggerFactory.getLogger(PerJobClusterClientBuilder.class); - private static final String DEFAULT_CONF_DIR = "./"; + private static final Logger LOG = LoggerFactory.getLogger(PerJobClusterClientBuilder.class); private YarnClient yarnClient; private YarnConfiguration yarnConf; - private Configuration flinkConfig; - /** * init yarnClient - * @param yarnConfDir the path of yarnconf */ - public void init(String yarnConfDir, Configuration flinkConfig, Properties userConf) throws Exception { - - if(Strings.isNullOrEmpty(yarnConfDir)) { - throw new RuntimeException("parameters of yarn is required"); - } + public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properties userConf) throws Exception { userConf.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); - this.flinkConfig = flinkConfig; SecurityUtils.install(new SecurityConfiguration(flinkConfig)); - yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); + this.yarnConf = yarnConf; yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); yarnClient.start(); @@ -83,11 +69,10 @@ public void init(String yarnConfDir, Configuration flinkConfig, Properties userC * create a yarn cluster descriptor which is used to start the application master * @param confProp taskParams * @param options LauncherOptions - * @param jobGraph JobGraph * @return * @throws MalformedURLException */ - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options options, JobGraph jobGraph) throws MalformedURLException { + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options options) throws MalformedURLException { String flinkJarPath = options.getFlinkLibJar(); if (StringUtils.isNotBlank(flinkJarPath)) { if (!new File(flinkJarPath).exists()) { @@ -112,17 +97,16 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } } } + if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { - Map jobCacheFileConfig = jobGraph.getUserArtifacts(); - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(tmp.getKey().startsWith("class_path")){ - shipFiles.add(new File(tmp.getValue().filePath)); - } - } + List files = fillAllPluginPathForYarnSession(options.getPluginRoot()); + shipFiles.addAll(files); } + if (StringUtils.isNotBlank(options.getQueue())) { descriptor.setQueue(options.getQueue()); } + File log4j = new File(options.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); if(log4j.exists()){ shipFiles.add(log4j); @@ -135,4 +119,22 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co descriptor.addShipFiles(shipFiles); return descriptor; } + + private List fillAllPluginPathForYarnSession(String flinkPluginRoot) { + List pluginPaths = Lists.newArrayList(); + //预加载同步插件jar包 + if (org.apache.commons.lang3.StringUtils.isNotBlank(flinkPluginRoot)) { + try { + File[] jars = new File(flinkPluginRoot).listFiles(); + if (jars != null) { + pluginPaths.addAll(Arrays.asList(jars)); + } else { + LOG.warn("jars in flinkPluginRoot is null, flinkPluginRoot = {}", flinkPluginRoot); + } + } catch (Exception e) { + LOG.error("error to load jars in flinkPluginRoot, flinkPluginRoot = {}, e = {}", flinkPluginRoot, ExceptionUtil.getErrorMessage(e)); + } + } + return pluginPaths; + } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index 0be4ec5279..3119df54ca 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.launcher.perjob; +import com.dtstack.flinkx.launcher.ClassLoaderType; +import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; import com.dtstack.flinkx.util.MapUtil; import org.apache.commons.lang.StringUtils; @@ -26,11 +28,15 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.util.ArrayList; import java.util.Properties; /** @@ -44,23 +50,45 @@ public class PerJobSubmitter { /** * submit per-job task * @param options LauncherOptions - * @param jobGraph JobGraph * @return * @throws Exception */ - public static String submit(Options options, JobGraph jobGraph) throws Exception{ + public static String submit(Options options, File jarFile, String[] programArgs) throws Exception{ LOG.info("start to submit per-job task, LauncherOptions = {}", options.toString()); + + if (StringUtils.isEmpty(options.getYarnconf())) { + throw new RuntimeException("parameters of yarn is required"); + } + + YarnConfiguration yarnConf = StringUtils.isEmpty(options.getYarnconf()) ? new YarnConfiguration() : YarnConfLoader.getYarnConf(options.getYarnconf()); + Configuration flinkConfig = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); + Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); + clusterSpecification.setCreateProgramDelay(true); + clusterSpecification.setConfiguration(flinkConfig); + clusterSpecification.setClasspaths(new ArrayList<>()); + clusterSpecification.setEntryPointClass("com.dtstack.flinkx.Main"); + clusterSpecification.setJarFile(jarFile); + + if (StringUtils.isNotEmpty(options.getS())) { + clusterSpecification.setSpSetting(SavepointRestoreSettings.forPath(options.getS())); + } + + clusterSpecification.setProgramArgs(programArgs); + clusterSpecification.setCreateProgramDelay(true); + clusterSpecification.setYarnConfiguration(yarnConf); + + ClassLoaderType classLoaderType = ClassLoaderType.getByClassMode(options.getPluginLoadMode()); + clusterSpecification.setClassLoaderType(classLoaderType); + PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); - Configuration config = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); - perJobClusterClientBuilder.init(options.getYarnconf(), config, conProp); + perJobClusterClientBuilder.init(yarnConf, flinkConfig, conProp); - AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options, jobGraph); - ClusterClient clusterClient = descriptor.deployJobCluster(clusterSpecification, jobGraph, true); + AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options); + ClusterClient clusterClient = descriptor.deployJobCluster(clusterSpecification, new JobGraph(), true); String applicationId = clusterClient.getClusterId().toString(); - String flinkJobId = jobGraph.getJobID().toString(); - LOG.info("deploy per_job with appId: {}}, jobId: {}", applicationId, flinkJobId); + LOG.info("deploy per_job with appId: {}", applicationId); return applicationId; } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java new file mode 100644 index 0000000000..2e12164d90 --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.deployment; + +import com.dtstack.flinkx.launcher.ClassLoaderType; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.net.URL; +import java.util.List; + +/** + * Description of the cluster to start by the {@link ClusterDescriptor}. + */ +public final class ClusterSpecification { + private final int masterMemoryMB; + private final int taskManagerMemoryMB; + private final int numberTaskManagers; + private final int slotsPerTaskManager; + private final int priority; + + private int parallelism; + private Configuration configuration; + private YarnConfiguration yarnConfiguration; + private JobGraph jobGraph; + private SavepointRestoreSettings spSetting; + private List classpaths; + private String entryPointClass; + private String[] programArgs; + private File jarFile; + private boolean createProgramDelay = false; + private PackagedProgram program; + private ClassLoaderType classLoaderType = ClassLoaderType.PARENT_FIRST; + + private ClusterSpecification(int masterMemoryMB, int taskManagerMemoryMB, int numberTaskManagers, int slotsPerTaskManager, int parallelism, int priority) { + this.masterMemoryMB = masterMemoryMB; + this.taskManagerMemoryMB = taskManagerMemoryMB; + this.numberTaskManagers = numberTaskManagers; + this.slotsPerTaskManager = slotsPerTaskManager; + this.parallelism = parallelism; + this.priority = priority; + } + + public PackagedProgram getProgram() { + return program; + } + + public void setProgram(PackagedProgram program) { + this.program = program; + } + + public YarnConfiguration getYarnConfiguration() { + return yarnConfiguration; + } + + public void setYarnConfiguration(YarnConfiguration yarnConfiguration) { + this.yarnConfiguration = yarnConfiguration; + } + + public JobGraph getJobGraph() { + return jobGraph; + } + + public void setJobGraph(JobGraph jobGraph) { + this.jobGraph = jobGraph; + } + + public int getParallelism() { + return parallelism; + } + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + public Configuration getConfiguration() { + return configuration; + } + + public void setConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + public int getMasterMemoryMB() { + return masterMemoryMB; + } + + public int getTaskManagerMemoryMB() { + return taskManagerMemoryMB; + } + + public int getNumberTaskManagers() { + return numberTaskManagers; + } + + public int getSlotsPerTaskManager() { + return slotsPerTaskManager; + } + + public int getPriority(){ + return priority; + } + + public SavepointRestoreSettings getSpSetting() { + return spSetting; + } + + public void setSpSetting(SavepointRestoreSettings spSetting) { + this.spSetting = spSetting; + } + + public List getClasspaths() { + return classpaths; + } + + public void setClasspaths(List classpaths) { + this.classpaths = classpaths; + } + + public String getEntryPointClass() { + return entryPointClass; + } + + public void setEntryPointClass(String entryPointClass) { + this.entryPointClass = entryPointClass; + } + + public String[] getProgramArgs() { + return programArgs; + } + + public void setProgramArgs(String[] programArgs) { + this.programArgs = programArgs; + } + + public File getJarFile() { + return jarFile; + } + + public void setJarFile(File jarFile) { + this.jarFile = jarFile; + } + + public boolean isCreateProgramDelay() { + return createProgramDelay; + } + + public void setCreateProgramDelay(boolean createProgramDelay) { + this.createProgramDelay = createProgramDelay; + } + + public ClassLoaderType getClassLoaderType() { + return classLoaderType; + } + + public void setClassLoaderType(ClassLoaderType classLoaderType) { + this.classLoaderType = classLoaderType; + } + + @Override + public String toString() { + return "ClusterSpecification{" + + "masterMemoryMB=" + masterMemoryMB + + ", taskManagerMemoryMB=" + taskManagerMemoryMB + + ", numberTaskManagers=" + numberTaskManagers + + ", slotsPerTaskManager=" + slotsPerTaskManager + + ", priority=" + priority + + '}'; + } + + public static ClusterSpecification fromConfiguration(Configuration configuration) { + int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + + int jobManagerMemoryMb = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); + int taskManagerMemoryMb = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); + + return new ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setNumberTaskManagers(1) + .setSlotsPerTaskManager(slots) + .createClusterSpecification(); + } + + /** + * Builder for the {@link ClusterSpecification} instance. + */ + public static class ClusterSpecificationBuilder { + private int masterMemoryMB = 768; + private int taskManagerMemoryMB = 768; + private int numberTaskManagers = 1; + private int slotsPerTaskManager = 1; + private int parallelism = 1; + private int priority = 0; + + public ClusterSpecificationBuilder setMasterMemoryMB(int masterMemoryMB) { + this.masterMemoryMB = masterMemoryMB; + return this; + } + + public ClusterSpecificationBuilder setTaskManagerMemoryMB(int taskManagerMemoryMB) { + this.taskManagerMemoryMB = taskManagerMemoryMB; + return this; + } + + public ClusterSpecificationBuilder setNumberTaskManagers(int numberTaskManagers) { + this.numberTaskManagers = numberTaskManagers; + return this; + } + + public ClusterSpecificationBuilder setSlotsPerTaskManager(int slotsPerTaskManager) { + this.slotsPerTaskManager = slotsPerTaskManager; + return this; + } + + public ClusterSpecificationBuilder setPriority(int priority){ + this.priority = priority; + return this; + } + + public ClusterSpecificationBuilder setParallelism(int parallelism) { + this.parallelism = parallelism; + return this; + } + + public ClusterSpecification createClusterSpecification() { + return new ClusterSpecification( + masterMemoryMB, + taskManagerMemoryMB, + numberTaskManagers, + slotsPerTaskManager, + parallelism, + priority); + } + } +} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java b/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java new file mode 100644 index 0000000000..41961ee3e0 --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.program; + +import com.dtstack.flinkx.launcher.ClassLoaderType; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.Plan; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; + +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A JobWithJars is a Flink dataflow plan, together with a bunch of JAR files that contain + * the classes of the functions and libraries necessary for the execution. + */ +public class JobWithJars { + + private Plan plan; + + private List jarFiles; + + /** + * classpaths that are needed during user code execution. + */ + private List classpaths; + + private ClassLoader userCodeClassLoader; + + private static Map cacheClassLoader = new ConcurrentHashMap<>(); + + + public JobWithJars(Plan plan, List jarFiles, List classpaths) throws IOException { + this.plan = plan; + this.jarFiles = new ArrayList(jarFiles.size()); + this.classpaths = new ArrayList(classpaths.size()); + + for (URL jarFile: jarFiles) { + checkJarFile(jarFile); + this.jarFiles.add(jarFile); + } + + for (URL path: classpaths) { + this.classpaths.add(path); + } + } + + public JobWithJars(Plan plan, URL jarFile) throws IOException { + this.plan = plan; + + checkJarFile(jarFile); + this.jarFiles = Collections.singletonList(jarFile); + this.classpaths = Collections.emptyList(); + } + + JobWithJars(Plan plan, List jarFiles, List classpaths, ClassLoader userCodeClassLoader) { + this.plan = plan; + this.jarFiles = jarFiles; + this.classpaths = classpaths; + this.userCodeClassLoader = userCodeClassLoader; + } + + /** + * Returns the plan. + */ + public Plan getPlan() { + return this.plan; + } + + /** + * Returns list of jar files that need to be submitted with the plan. + */ + public List getJarFiles() { + return this.jarFiles; + } + + /** + * Returns list of classpaths that need to be submitted with the plan. + */ + public List getClasspaths() { + return classpaths; + } + + /** + * Gets the {@link ClassLoader} that must be used to load user code classes. + * + * @return The user code ClassLoader. + */ + public ClassLoader getUserCodeClassLoader() { + if (this.userCodeClassLoader == null) { + this.userCodeClassLoader = buildUserCodeClassLoader(jarFiles, classpaths, getClass().getClassLoader(), ClassLoaderType.PARENT_FIRST); + } + return this.userCodeClassLoader; + } + + public static void checkJarFile(URL jar) throws IOException { + File jarFile; + try { + jarFile = new File(jar.toURI()); + } catch (URISyntaxException e) { + throw new IOException("JAR file path is invalid '" + jar + "'"); + } + if (!jarFile.exists()) { + throw new IOException("JAR file does not exist '" + jarFile.getAbsolutePath() + "'"); + } + if (!jarFile.canRead()) { + throw new IOException("JAR file can't be read '" + jarFile.getAbsolutePath() + "'"); + } + // TODO: Check if proper JAR file + } + + public static ClassLoader buildUserCodeClassLoader(List jars, List classpaths, ClassLoader parent, ClassLoaderType classLoaderType) { + if (ClassLoaderType.NONE == classLoaderType) { + return parent; + } + URL[] urls = new URL[jars.size() + classpaths.size()]; + for (int i = 0; i < jars.size(); i++) { + urls[i] = jars.get(i); + } + for (int i = 0; i < classpaths.size(); i++) { + urls[i + jars.size()] = classpaths.get(i); + } + switch (classLoaderType) { + case CHILD_FIRST_CACHE: + Arrays.sort(urls, Comparator.comparing(URL::toString)); + String jarsKeyChild = StringUtils.join(urls, "_"); + return cacheClassLoader.computeIfAbsent(jarsKeyChild, k -> FlinkUserCodeClassLoaders.childFirst(urls, parent, new String[]{})); + case PARENT_FIRST_CACHE: + Arrays.sort(urls, Comparator.comparing(URL::toString)); + String jarsKeyParent = StringUtils.join(urls, "_"); + return cacheClassLoader.computeIfAbsent(jarsKeyParent, k -> FlinkUserCodeClassLoaders.parentFirst(urls, parent)); + case CHILD_FIRST: + return FlinkUserCodeClassLoaders.childFirst(urls, parent, new String[]{}); + case PARENT_FIRST: + return FlinkUserCodeClassLoaders.parentFirst(urls, parent); + default: + return FlinkUserCodeClassLoaders.parentFirst(urls, parent); + } + } +} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java new file mode 100644 index 0000000000..b56000f4c6 --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java @@ -0,0 +1,773 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.program; + +import com.dtstack.flinkx.launcher.ClassLoaderType; +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.Program; +import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.optimizer.Optimizer; +import org.apache.flink.optimizer.dag.DataSinkNode; +import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.InstantiationUtil; + +import javax.annotation.Nullable; +import java.io.*; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.net.MalformedURLException; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.*; +import java.util.jar.Attributes; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.jar.Manifest; + +/** + * This class encapsulates represents a program, packaged in a jar file. It supplies + * functionality to extract nested libraries, search for the program entry point, and extract + * a program plan. + */ +public class PackagedProgram { + + /** + * Property name of the entry in JAR manifest file that describes the Flink specific entry point. + */ + public static final String MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS = "program-class"; + + /** + * Property name of the entry in JAR manifest file that describes the class with the main method. + */ + public static final String MANIFEST_ATTRIBUTE_MAIN_CLASS = "Main-Class"; + + // -------------------------------------------------------------------------------------------- + + private final URL jarFile; + + private final String[] args; + + private final Program program; + + private final Class mainClass; + + private final List extractedTempLibraries; + + private final List classpaths; + + private ClassLoader userCodeClassLoader; + + private Plan plan; + + private SavepointRestoreSettings savepointSettings = SavepointRestoreSettings.none(); + + /** + * Creates an instance that wraps the plan defined in the jar file using the given + * argument. + * + * @param jarFile + * The jar file which contains the plan and a Manifest which defines + * the program-class + * @param args + * Optional. The arguments used to create the pact plan, depend on + * implementation of the pact plan. See getDescription(). + * @throws ProgramInvocationException + * This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. + */ + public PackagedProgram(File jarFile, String... args) throws ProgramInvocationException { + this(jarFile, Collections.emptyList(), null, args); + } + + /** + * Creates an instance that wraps the plan defined in the jar file using the given + * argument. + * + * @param jarFile + * The jar file which contains the plan and a Manifest which defines + * the program-class + * @param classpaths + * Additional classpath URLs needed by the Program. + * @param args + * Optional. The arguments used to create the pact plan, depend on + * implementation of the pact plan. See getDescription(). + * @throws ProgramInvocationException + * This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. + */ + public PackagedProgram(File jarFile, List classpaths, ClassLoaderType classLoaderType, String... args) throws ProgramInvocationException { + this(jarFile, classpaths, classLoaderType, null, args); + } + + /** + * Creates an instance that wraps the plan defined in the jar file using the given + * arguments. For generating the plan the class defined in the className parameter + * is used. + * + * @param jarFile + * The jar file which contains the plan. + * @param entryPointClassName + * Name of the class which generates the plan. Overrides the class defined + * in the jar file manifest + * @param args + * Optional. The arguments used to create the pact plan, depend on + * implementation of the pact plan. See getDescription(). + * @throws ProgramInvocationException + * This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. + */ + public PackagedProgram(File jarFile, @Nullable String entryPointClassName, ClassLoaderType classLoaderType, String... args) throws ProgramInvocationException { + this(jarFile, Collections.emptyList(), classLoaderType, entryPointClassName, args); + } + + /** + * Creates an instance that wraps the plan defined in the jar file using the given + * arguments. For generating the plan the class defined in the className parameter + * is used. + * + * @param jarFile + * The jar file which contains the plan. + * @param classpaths + * Additional classpath URLs needed by the Program. + * @param entryPointClassName + * Name of the class which generates the plan. Overrides the class defined + * in the jar file manifest + * @param args + * Optional. The arguments used to create the pact plan, depend on + * implementation of the pact plan. See getDescription(). + * @throws ProgramInvocationException + * This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. + */ + public PackagedProgram(File jarFile, List classpaths, ClassLoaderType classLoaderType, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { + if (jarFile == null) { + throw new IllegalArgumentException("The jar file must not be null."); + } + + URL jarFileUrl; + try { + jarFileUrl = jarFile.getAbsoluteFile().toURI().toURL(); + } catch (MalformedURLException e1) { + throw new IllegalArgumentException("The jar file path is invalid."); + } + + checkJarFile(jarFileUrl); + + this.jarFile = jarFileUrl; + this.args = args == null ? new String[0] : args; + + // if no entryPointClassName name was given, we try and look one up through the manifest + if (entryPointClassName == null) { + entryPointClassName = getEntryPointClassNameFromJar(jarFileUrl); + } + + // now that we have an entry point, we can extract the nested jar files (if any) + this.extractedTempLibraries = extractContainedLibraries(jarFileUrl); + this.classpaths = classpaths; + this.userCodeClassLoader = JobWithJars.buildUserCodeClassLoader(getAllLibraries(), classpaths, getClass().getClassLoader(), classLoaderType); + + // load the entry point class + this.mainClass = loadMainClass(entryPointClassName, userCodeClassLoader); + + // if the entry point is a program, instantiate the class and get the plan + if (Program.class.isAssignableFrom(this.mainClass)) { + Program prg = null; + try { + prg = InstantiationUtil.instantiate(this.mainClass.asSubclass(Program.class), Program.class); + } catch (Exception e) { + // validate that the class has a main method at least. + // the main method possibly instantiates the program properly + if (!hasMainMethod(mainClass)) { + throw new ProgramInvocationException("The given program class implements the " + + Program.class.getName() + " interface, but cannot be instantiated. " + + "It also declares no main(String[]) method as alternative entry point", e); + } + } catch (Throwable t) { + throw new ProgramInvocationException("Error while trying to instantiate program class.", t); + } + this.program = prg; + } else if (hasMainMethod(mainClass)) { + this.program = null; + } else { + throw new ProgramInvocationException("The given program class neither has a main(String[]) method, nor does it implement the " + + Program.class.getName() + " interface."); + } + } + + public PackagedProgram(Class entryPointClass, String... args) throws ProgramInvocationException { + this.jarFile = null; + this.args = args == null ? new String[0] : args; + + this.extractedTempLibraries = Collections.emptyList(); + this.classpaths = Collections.emptyList(); + this.userCodeClassLoader = entryPointClass.getClassLoader(); + + // load the entry point class + this.mainClass = entryPointClass; + + // if the entry point is a program, instantiate the class and get the plan + if (Program.class.isAssignableFrom(this.mainClass)) { + Program prg = null; + try { + prg = InstantiationUtil.instantiate(this.mainClass.asSubclass(Program.class), Program.class); + } catch (Exception e) { + // validate that the class has a main method at least. + // the main method possibly instantiates the program properly + if (!hasMainMethod(mainClass)) { + throw new ProgramInvocationException("The given program class implements the " + + Program.class.getName() + " interface, but cannot be instantiated. " + + "It also declares no main(String[]) method as alternative entry point", e); + } + } catch (Throwable t) { + throw new ProgramInvocationException("Error while trying to instantiate program class.", t); + } + this.program = prg; + } else if (hasMainMethod(mainClass)) { + this.program = null; + } else { + throw new ProgramInvocationException("The given program class neither has a main(String[]) method, nor does it implement the " + + Program.class.getName() + " interface."); + } + } + + public void setSavepointRestoreSettings(SavepointRestoreSettings savepointSettings) { + this.savepointSettings = savepointSettings; + } + + public SavepointRestoreSettings getSavepointSettings() { + return savepointSettings; + } + + public String[] getArguments() { + return this.args; + } + + public String getMainClassName() { + return this.mainClass.getName(); + } + + public boolean isUsingInteractiveMode() { + return this.program == null; + } + + public boolean isUsingProgramEntryPoint() { + return this.program != null; + } + + /** + * Returns the plan without the required jars when the files are already provided by the cluster. + * + * @return The plan without attached jar files. + * @throws ProgramInvocationException + */ + public JobWithJars getPlanWithoutJars() throws ProgramInvocationException { + if (isUsingProgramEntryPoint()) { + return new JobWithJars(getPlan(), Collections.emptyList(), classpaths, userCodeClassLoader); + } else { + throw new ProgramInvocationException("Cannot create a " + JobWithJars.class.getSimpleName() + + " for a program that is using the interactive mode.", getPlan().getJobId()); + } + } + + /** + * Returns the plan with all required jars. + * + * @return The plan with attached jar files. + * @throws ProgramInvocationException + */ + public JobWithJars getPlanWithJars() throws ProgramInvocationException { + if (isUsingProgramEntryPoint()) { + return new JobWithJars(getPlan(), getAllLibraries(), classpaths, userCodeClassLoader); + } else { + throw new ProgramInvocationException("Cannot create a " + JobWithJars.class.getSimpleName() + + " for a program that is using the interactive mode.", getPlan().getJobId()); + } + } + + /** + * Returns the analyzed plan without any optimizations. + * + * @return + * the analyzed plan without any optimizations. + * @throws ProgramInvocationException Thrown if an error occurred in the + * user-provided pact assembler. This may indicate + * missing parameters for generation. + */ + public String getPreviewPlan() throws ProgramInvocationException { + Thread.currentThread().setContextClassLoader(this.getUserCodeClassLoader()); + List previewPlan; + + if (isUsingProgramEntryPoint()) { + previewPlan = Optimizer.createPreOptimizedPlan(getPlan()); + } + else if (isUsingInteractiveMode()) { + // temporary hack to support the web client + PreviewPlanEnvironment env = new PreviewPlanEnvironment(); + env.setAsContext(); + try { + invokeInteractiveModeForExecution(); + } + catch (ProgramInvocationException e) { + throw e; + } + catch (Throwable t) { + // the invocation gets aborted with the preview plan + if (env.previewPlan == null) { + if (env.preview != null) { + return env.preview; + } else { + throw new ProgramInvocationException("The program caused an error: ", getPlan().getJobId(), t); + } + } + } + finally { + env.unsetAsContext(); + } + + if (env.previewPlan != null) { + previewPlan = env.previewPlan; + } else { + throw new ProgramInvocationException( + "The program plan could not be fetched. The program silently swallowed the control flow exceptions.", + getPlan().getJobId()); + } + } + else { + throw new RuntimeException(); + } + + PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator(); + StringWriter string = new StringWriter(1024); + try (PrintWriter pw = new PrintWriter(string)) { + jsonGen.dumpPactPlanAsJSON(previewPlan, pw); + } + return string.toString(); + + } + + /** + * Returns the description provided by the Program class. This + * may contain a description of the plan itself and its arguments. + * + * @return The description of the PactProgram's input parameters. + * @throws ProgramInvocationException + * This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. + */ + @Nullable + public String getDescription() throws ProgramInvocationException { + if (ProgramDescription.class.isAssignableFrom(this.mainClass)) { + + ProgramDescription descr; + if (this.program != null) { + descr = (ProgramDescription) this.program; + } else { + try { + descr = InstantiationUtil.instantiate( + this.mainClass.asSubclass(ProgramDescription.class), ProgramDescription.class); + } catch (Throwable t) { + return null; + } + } + + try { + return descr.getDescription(); + } + catch (Throwable t) { + throw new ProgramInvocationException("Error while getting the program description" + + (t.getMessage() == null ? "." : ": " + t.getMessage()), t); + } + + } else { + return null; + } + } + + /** + * This method assumes that the context environment is prepared, or the execution + * will be a local execution by default. + */ + public void invokeInteractiveModeForExecution() throws ProgramInvocationException{ + if (isUsingInteractiveMode()) { + callMainMethod(mainClass, args); + } else { + throw new ProgramInvocationException("Cannot invoke a plan-based program directly."); + } + } + + /** + * Returns the classpaths that are required by the program. + * + * @return List of {@link URL}s. + */ + public List getClasspaths() { + return this.classpaths; + } + + /** + * Gets the {@link ClassLoader} that must be used to load user code classes. + * + * @return The user code ClassLoader. + */ + public ClassLoader getUserCodeClassLoader() { + return this.userCodeClassLoader; + } + + /** + * Returns all provided libraries needed to run the program. + */ + public List getAllLibraries() { + List libs = new ArrayList(this.extractedTempLibraries.size() + 1); + + if (jarFile != null) { + libs.add(jarFile); + } + for (File tmpLib : this.extractedTempLibraries) { + try { + libs.add(tmpLib.getAbsoluteFile().toURI().toURL()); + } + catch (MalformedURLException e) { + throw new RuntimeException("URL is invalid. This should not happen.", e); + } + } + + return libs; + } + + /** + * Deletes all temporary files created for contained packaged libraries. + */ + public void deleteExtractedLibraries() { + deleteExtractedLibraries(this.extractedTempLibraries); + this.extractedTempLibraries.clear(); + } + + /** + * Returns the plan as generated from the Pact Assembler. + * + * @return The program's plan. + * @throws ProgramInvocationException Thrown, if an error occurred in the program while + * creating the program's {@link Plan}. + */ + private Plan getPlan() throws ProgramInvocationException { + if (this.plan == null) { + Thread.currentThread().setContextClassLoader(this.userCodeClassLoader); + this.plan = createPlanFromProgram(this.program, this.args); + } + + return this.plan; + } + + private static boolean hasMainMethod(Class entryClass) { + Method mainMethod; + try { + mainMethod = entryClass.getMethod("main", String[].class); + } catch (NoSuchMethodException e) { + return false; + } + catch (Throwable t) { + throw new RuntimeException("Could not look up the main(String[]) method from the class " + + entryClass.getName() + ": " + t.getMessage(), t); + } + + return Modifier.isStatic(mainMethod.getModifiers()) && Modifier.isPublic(mainMethod.getModifiers()); + } + + private static void callMainMethod(Class entryClass, String[] args) throws ProgramInvocationException { + Method mainMethod; + if (!Modifier.isPublic(entryClass.getModifiers())) { + throw new ProgramInvocationException("The class " + entryClass.getName() + " must be public."); + } + + try { + mainMethod = entryClass.getMethod("main", String[].class); + } catch (NoSuchMethodException e) { + throw new ProgramInvocationException("The class " + entryClass.getName() + " has no main(String[]) method."); + } + catch (Throwable t) { + throw new ProgramInvocationException("Could not look up the main(String[]) method from the class " + + entryClass.getName() + ": " + t.getMessage(), t); + } + + if (!Modifier.isStatic(mainMethod.getModifiers())) { + throw new ProgramInvocationException("The class " + entryClass.getName() + " declares a non-static main method."); + } + if (!Modifier.isPublic(mainMethod.getModifiers())) { + throw new ProgramInvocationException("The class " + entryClass.getName() + " declares a non-public main method."); + } + + try { + mainMethod.invoke(null, (Object) args); + } + catch (IllegalArgumentException e) { + throw new ProgramInvocationException("Could not invoke the main method, arguments are not matching.", e); + } + catch (IllegalAccessException e) { + throw new ProgramInvocationException("Access to the main method was denied: " + e.getMessage(), e); + } + catch (InvocationTargetException e) { + Throwable exceptionInMethod = e.getTargetException(); + if (exceptionInMethod instanceof Error) { + throw (Error) exceptionInMethod; + } else if (exceptionInMethod instanceof ProgramParametrizationException) { + throw (ProgramParametrizationException) exceptionInMethod; + } else if (exceptionInMethod instanceof ProgramInvocationException) { + throw (ProgramInvocationException) exceptionInMethod; + } else { + throw new ProgramInvocationException("The main method caused an error: " + exceptionInMethod.getMessage(), exceptionInMethod); + } + } + catch (Throwable t) { + throw new ProgramInvocationException("An error occurred while invoking the program's main method: " + t.getMessage(), t); + } + } + + private static String getEntryPointClassNameFromJar(URL jarFile) throws ProgramInvocationException { + JarFile jar; + Manifest manifest; + String className; + + // Open jar file + try { + jar = new JarFile(new File(jarFile.toURI())); + } catch (URISyntaxException use) { + throw new ProgramInvocationException("Invalid file path '" + jarFile.getPath() + "'", use); + } catch (IOException ioex) { + throw new ProgramInvocationException("Error while opening jar file '" + jarFile.getPath() + "'. " + + ioex.getMessage(), ioex); + } + + // jar file must be closed at the end + try { + // Read from jar manifest + try { + manifest = jar.getManifest(); + } catch (IOException ioex) { + throw new ProgramInvocationException("The Manifest in the jar file could not be accessed '" + + jarFile.getPath() + "'. " + ioex.getMessage(), ioex); + } + + if (manifest == null) { + throw new ProgramInvocationException("No manifest found in jar file '" + jarFile.getPath() + "'. The manifest is need to point to the program's main class."); + } + + Attributes attributes = manifest.getMainAttributes(); + + // check for a "program-class" entry first + className = attributes.getValue(PackagedProgram.MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS); + if (className != null) { + return className; + } + + // check for a main class + className = attributes.getValue(PackagedProgram.MANIFEST_ATTRIBUTE_MAIN_CLASS); + if (className != null) { + return className; + } else { + throw new ProgramInvocationException("Neither a '" + MANIFEST_ATTRIBUTE_MAIN_CLASS + "', nor a '" + + MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS + "' entry was found in the jar file."); + } + } + finally { + try { + jar.close(); + } catch (Throwable t) { + throw new ProgramInvocationException("Could not close the JAR file: " + t.getMessage(), t); + } + } + } + + private static Class loadMainClass(String className, ClassLoader cl) throws ProgramInvocationException { + ClassLoader contextCl = null; + try { + contextCl = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(cl); + return Class.forName(className, false, cl); + } + catch (ClassNotFoundException e) { + throw new ProgramInvocationException("The program's entry point class '" + className + + "' was not found in the jar file.", e); + } + catch (ExceptionInInitializerError e) { + throw new ProgramInvocationException("The program's entry point class '" + className + + "' threw an error during initialization.", e); + } + catch (LinkageError e) { + throw new ProgramInvocationException("The program's entry point class '" + className + + "' could not be loaded due to a linkage failure.", e); + } + catch (Throwable t) { + throw new ProgramInvocationException("The program's entry point class '" + className + + "' caused an exception during initialization: " + t.getMessage(), t); + } finally { + if (contextCl != null) { + Thread.currentThread().setContextClassLoader(contextCl); + } + } + } + + /** + * Takes the jar described by the given file and invokes its pact assembler class to + * assemble a plan. The assembler class name is either passed through a parameter, + * or it is read from the manifest of the jar. The assembler is handed the given options + * for its assembly. + * + * @param program The program to create the plan for. + * @param options + * The options for the assembler. + * @return The plan created by the program. + * @throws ProgramInvocationException + * Thrown, if an error occurred in the user-provided pact assembler. + */ + private static Plan createPlanFromProgram(Program program, String[] options) throws ProgramInvocationException { + try { + return program.getPlan(options); + } catch (Throwable t) { + throw new ProgramInvocationException("Error while calling the program: " + t.getMessage(), t); + } + } + + /** + * Takes all JAR files that are contained in this program's JAR file and extracts them + * to the system's temp directory. + * + * @return The file names of the extracted temporary files. + * @throws ProgramInvocationException Thrown, if the extraction process failed. + */ + public static List extractContainedLibraries(URL jarFile) throws ProgramInvocationException { + + Random rnd = new Random(); + + JarFile jar = null; + try { + jar = new JarFile(new File(jarFile.toURI())); + final List containedJarFileEntries = new ArrayList(); + + Enumeration entries = jar.entries(); + while (entries.hasMoreElements()) { + JarEntry entry = entries.nextElement(); + String name = entry.getName(); + + if (name.length() > 8 && name.startsWith("lib/") && name.endsWith(".jar")) { + containedJarFileEntries.add(entry); + } + } + + if (containedJarFileEntries.isEmpty()) { + return Collections.emptyList(); + } + else { + // go over all contained jar files + final List extractedTempLibraries = new ArrayList(containedJarFileEntries.size()); + final byte[] buffer = new byte[4096]; + + boolean incomplete = true; + + try { + for (int i = 0; i < containedJarFileEntries.size(); i++) { + final JarEntry entry = containedJarFileEntries.get(i); + String name = entry.getName(); + // '/' as in case of zip, jar + // java.util.zip.ZipEntry#isDirectory always looks only for '/' not for File.separator + name = name.replace('/', '_'); + + File tempFile; + try { + tempFile = File.createTempFile(rnd.nextInt(Integer.MAX_VALUE) + "_", name); + tempFile.deleteOnExit(); + } + catch (IOException e) { + throw new ProgramInvocationException( + "An I/O error occurred while creating temporary file to extract nested library '" + + entry.getName() + "'.", e); + } + + extractedTempLibraries.add(tempFile); + + // copy the temp file contents to a temporary File + OutputStream out = null; + InputStream in = null; + try { + + out = new FileOutputStream(tempFile); + in = new BufferedInputStream(jar.getInputStream(entry)); + + int numRead = 0; + while ((numRead = in.read(buffer)) != -1) { + out.write(buffer, 0, numRead); + } + } + catch (IOException e) { + throw new ProgramInvocationException("An I/O error occurred while extracting nested library '" + + entry.getName() + "' to temporary file '" + tempFile.getAbsolutePath() + "'."); + } + finally { + if (out != null) { + out.close(); + } + if (in != null) { + in.close(); + } + } + } + + incomplete = false; + } + finally { + if (incomplete) { + deleteExtractedLibraries(extractedTempLibraries); + } + } + + return extractedTempLibraries; + } + } + catch (Throwable t) { + throw new ProgramInvocationException("Unknown I/O error while extracting contained jar files.", t); + } + finally { + if (jar != null) { + try { + jar.close(); + } catch (Throwable t) {} + } + } + } + + public static void deleteExtractedLibraries(List tempLibraries) { + for (File f : tempLibraries) { + f.delete(); + } + } + + private static void checkJarFile(URL jarfile) throws ProgramInvocationException { + try { + JobWithJars.checkJarFile(jarfile); + } + catch (IOException e) { + throw new ProgramInvocationException(e.getMessage()); + } + catch (Throwable t) { + throw new ProgramInvocationException("Cannot access jar file" + (t.getMessage() == null ? "." : ": " + t.getMessage()), t); + } + } + +} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java new file mode 100644 index 0000000000..5f7e39293a --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -0,0 +1,1752 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.yarn; + +import avro.shaded.com.google.common.collect.Sets; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.common.base.Strings; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.*; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.taskexecutor.TaskManagerServices; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.*; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.MalformedURLException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.*; + +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.*; + +/** + * The descriptor with deployment information for deploying a Flink cluster on Yarn. + */ +public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor { + private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class); + + private final YarnConfiguration yarnConfiguration; + + private final YarnClient yarnClient; + + /** True if the descriptor must not shut down the YarnClient. */ + private final boolean sharedYarnClient; + + private String yarnQueue; + + private String configurationDirectory; + + private Path flinkJarPath; + + private String dynamicPropertiesEncoded; + + /** Lazily initialized list of files to ship. */ + protected List shipFiles = new LinkedList<>(); + + private final Configuration flinkConfiguration; + + private boolean detached; + + private String customName; + + private String zookeeperNamespace; + + private String nodeLabel; + + /** Optional Jar file to include in the system class loader of all application nodes + * (for per-job submission). */ + private final Set userJarFiles = new HashSet<>(); + + private YarnConfigOptions.UserJarInclusion userJarInclusion; + + public AbstractYarnClusterDescriptor( + Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, + String configurationDirectory, + YarnClient yarnClient, + boolean sharedYarnClient) { + + this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); + + // for unit tests only + if (System.getenv("IN_TESTS") != null) { + try { + yarnConfiguration.addResource(new File(System.getenv("YARN_CONF_DIR"), "yarn-site.xml").toURI().toURL()); + } catch (Throwable t) { + throw new RuntimeException("Error", t); + } + } + + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.sharedYarnClient = sharedYarnClient; + + this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); + userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + + this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); + } + + public YarnClient getYarnClient() { + return yarnClient; + } + + /** + * The class to start the application master with. This class runs the main + * method in case of session cluster. + */ + protected abstract String getYarnSessionClusterEntrypoint(); + + /** + * The class to start the application master with. This class runs the main + * method in case of the job cluster. + */ + protected abstract String getYarnJobClusterEntrypoint(); + + public Configuration getFlinkConfiguration() { + return flinkConfiguration; + } + + public void setQueue(String queue) { + this.yarnQueue = queue; + } + + public void setLocalJarPath(Path localJarPath) { + if (!localJarPath.toString().endsWith("jar")) { + throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); + } + this.flinkJarPath = localJarPath; + } + + /** + * Adds the given files to the list of files to ship. + * + *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, StringBuilder)} + * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. + * + * @param shipFiles files to ship + */ + public void addShipFiles(List shipFiles) { + this.shipFiles.addAll(shipFiles); + } + + public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { + this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; + } + + public String getDynamicPropertiesEncoded() { + return this.dynamicPropertiesEncoded; + } + + private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws YarnDeploymentException { + + if (clusterSpecification.getNumberTaskManagers() <= 0) { + throw new YarnDeploymentException("Taskmanager count must be positive"); + } + if (this.flinkJarPath == null) { + throw new YarnDeploymentException("The Flink jar path is null"); + } + if (this.configurationDirectory == null) { + throw new YarnDeploymentException("Configuration directory not set"); + } + if (this.flinkConfiguration == null) { + throw new YarnDeploymentException("Flink configuration object has not been set"); + } + + // Check if we don't exceed YARN's maximum virtual cores. + // Fetch numYarnMaxVcores from all the RUNNING nodes via yarnClient + final int numYarnMaxVcores; + try { + numYarnMaxVcores = yarnClient.getNodeReports(NodeState.RUNNING) + .stream() + .mapToInt(report -> report.getCapability().getVirtualCores()) + .max() + .orElse(0); + } catch (Exception e) { + throw new YarnDeploymentException("Couldn't get cluster description, please check on the YarnConfiguration", e); + } + + int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); + // don't configure more than the maximum configured number of vcores + if (configuredVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException( + String.format("The number of requested virtual cores per node %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + + " Please note that the number of virtual cores is set to the number of task slots by default" + + " unless configured in the Flink config with '%s.'", + configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); + } + + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + } + + private static boolean allocateResource(int[] nodeManagers, int toAllocate) { + for (int i = 0; i < nodeManagers.length; i++) { + if (nodeManagers[i] >= toAllocate) { + nodeManagers[i] -= toAllocate; + return true; + } + } + return false; + } + + /** + * @deprecated The cluster descriptor should not know about this option. + */ + @Deprecated + public void setDetachedMode(boolean detachedMode) { + this.detached = detachedMode; + } + + /** + * @deprecated The cluster descriptor should not know about this option. + */ + @Deprecated + public boolean isDetachedMode() { + return detached; + } + + public String getZookeeperNamespace() { + return zookeeperNamespace; + } + + public void setZookeeperNamespace(String zookeeperNamespace) { + this.zookeeperNamespace = zookeeperNamespace; + } + + public String getNodeLabel() { + return nodeLabel; + } + + public void setNodeLabel(String nodeLabel) { + this.nodeLabel = nodeLabel; + } + + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + + @Override + public void close() { + if (!sharedYarnClient) { + yarnClient.stop(); + } + } + + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + + @Override + public ClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException { + + try { + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + + final ApplicationReport appReport = yarnClient.getApplicationReport(applicationId); + + if (appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { + // Flink cluster is not running anymore + LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}", + applicationId, appReport.getFinalApplicationStatus()); + throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); + } + + final String host = appReport.getHost(); + final int rpcPort = appReport.getRpcPort(); + + LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'", + host, rpcPort, applicationId); + + flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); + flinkConfiguration.setInteger(JobManagerOptions.PORT, rpcPort); + + flinkConfiguration.setString(RestOptions.ADDRESS, host); + flinkConfiguration.setInteger(RestOptions.PORT, rpcPort); + + return createYarnClusterClient( + this, + -1, // we don't know the number of task managers of a started Flink cluster + -1, // we don't know how many slots each task manager has for a started Flink cluster + appReport, + flinkConfiguration, + false); + } catch (Exception e) { + throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); + } + } + + @Override + public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException { + try { + return deployInternal( + clusterSpecification, + "Flink session cluster", + getYarnSessionClusterEntrypoint(), + null, + false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); + } + } + + @Override + public void killCluster(ApplicationId applicationId) throws FlinkException { + try { + yarnClient.killApplication(applicationId); + Utils.deleteApplicationFiles(Collections.singletonMap( + YarnConfigKeys.FLINK_YARN_FILES, + getYarnFilesDir(applicationId).toUri().toString())); + } catch (YarnException | IOException e) { + throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); + } + } + + /** + * Method to validate cluster specification before deploy it, it will throw + * an {@link FlinkException} if the {@link ClusterSpecification} is invalid. + * + * @param clusterSpecification cluster specification to check against the configuration of the + * AbstractYarnClusterDescriptor + * @throws FlinkException if the cluster cannot be started with the provided {@link ClusterSpecification} + */ + private void validateClusterSpecification(ClusterSpecification clusterSpecification) throws FlinkException { + try { + final long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); + // We do the validation by calling the calculation methods here + // Internally these methods will check whether the cluster can be started with the provided + // ClusterSpecification and the configured memory requirements + final long cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); + TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); + } catch (IllegalArgumentException iae) { + throw new FlinkException("Cannot fulfill the minimum memory requirements with the provided " + + "cluster specification. Please increase the memory of the cluster.", iae); + } + } + + /** + * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. + * + * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed + * @param applicationName name of the Yarn application to start + * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. + * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none + * @param detached True if the cluster should be started in detached mode + */ + protected ClusterClient deployInternal( + ClusterSpecification clusterSpecification, + String applicationName, + String yarnClusterEntrypoint, + @Nullable JobGraph jobGraph, + boolean detached) throws Exception { + + // ------------------ Check if configuration is valid -------------------- + validateClusterSpecification(clusterSpecification); + + if (UserGroupInformation.isSecurityEnabled()) { + // note: UGI::hasKerberosCredentials inaccurately reports false + // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), + // so we check only in ticket cache scenario. + boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); + + UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); + if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS + && useTicketCache && !loginUser.hasKerberosCredentials()) { + LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); + throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + + "does not have Kerberos credentials"); + } + } + + isReadyForDeployment(clusterSpecification); + + // ------------------ Check if the specified queue exists -------------------- + + checkYarnQueues(yarnClient); + + // ------------------ Add dynamic properties to local flinkConfiguraton ------ + Map dynProperties = getDynamicProperties(dynamicPropertiesEncoded); + for (Map.Entry dynProperty : dynProperties.entrySet()) { + flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue()); + } + + // ------------------ Check if the YARN ClusterClient has the requested resources -------------- + + // Create application via yarnClient + final YarnClientApplication yarnApplication = yarnClient.createApplication(); + final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); + + if(clusterSpecification.isCreateProgramDelay()){ + jobGraph = getJobGraph(appResponse.getApplicationId().toString(),clusterSpecification); + } + + Resource maxRes = appResponse.getMaximumResourceCapability(); + + final ClusterResourceDescription freeClusterMem; + try { + freeClusterMem = getCurrentFreeClusterResources(yarnClient); + } catch (YarnException | IOException e) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); + } + + final int yarnMinAllocationMB = yarnConfiguration.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); + + final ClusterSpecification validClusterSpecification; + try { + validClusterSpecification = validateClusterResources( + clusterSpecification, + yarnMinAllocationMB, + maxRes, + freeClusterMem); + } catch (YarnDeploymentException yde) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw yde; + } + + LOG.info("Cluster specification: {}", validClusterSpecification); + + final ClusterEntrypoint.ExecutionMode executionMode = detached ? + ClusterEntrypoint.ExecutionMode.DETACHED + : ClusterEntrypoint.ExecutionMode.NORMAL; + + flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); + + ApplicationReport report = startAppMaster( + flinkConfiguration, + applicationName, + yarnClusterEntrypoint, + jobGraph, + yarnClient, + yarnApplication, + validClusterSpecification); + + String host = report.getHost(); + int port = report.getRpcPort(); + + // Correctly initialize the Flink config + flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); + flinkConfiguration.setInteger(JobManagerOptions.PORT, port); + + flinkConfiguration.setString(RestOptions.ADDRESS, host); + flinkConfiguration.setInteger(RestOptions.PORT, port); + + // the Flink cluster is deployed in YARN. Represent cluster + return createYarnClusterClient( + this, + validClusterSpecification.getNumberTaskManagers(), + validClusterSpecification.getSlotsPerTaskManager(), + report, + flinkConfiguration, + true); + } + + private JobGraph getJobGraph(String appId,ClusterSpecification clusterSpecification) throws Exception{ + String url = getUrlFormat(clusterSpecification.getYarnConfiguration()) + "/" + appId; + PackagedProgram program = buildProgram(url,clusterSpecification); + clusterSpecification.setProgram(program); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, clusterSpecification.getConfiguration(), clusterSpecification.getParallelism()); + jobGraph.setAllowQueuedScheduling(true); + dealPluginByLoadMode(jobGraph); + clusterSpecification.setJobGraph(jobGraph); + return jobGraph; + } + + private PackagedProgram buildProgram(String monitorUrl,ClusterSpecification clusterSpecification) throws Exception{ + String[] args = clusterSpecification.getProgramArgs(); + for (int i = 0; i < args.length; i++) { + if("-monitor".equals(args[i])){ + args[i + 1] = monitorUrl; + break; + } + } + + PackagedProgram program = new PackagedProgram(clusterSpecification.getJarFile(), clusterSpecification.getClasspaths(), clusterSpecification.getClassLoaderType(), clusterSpecification.getEntryPointClass(), args); + + if (null != clusterSpecification.getSpSetting()) { + program.setSavepointRestoreSettings(clusterSpecification.getSpSetting()); + } else { + program.setSavepointRestoreSettings(SavepointRestoreSettings.none()); + } + + return program; + } + + private void dealPluginByLoadMode(JobGraph jobGraph) throws Exception { + String pluginLoadMode = flinkConfiguration.getString("pluginLoadMode", "shipfile"); + if (StringUtils.equalsIgnoreCase(pluginLoadMode, "classpath")) { + fillJobGraphClassPath(jobGraph); + fillStreamJobGraphClassPath(jobGraph); + } else { + fillPluginPathToShipFiles(jobGraph); + } + } + + private void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); + Set classPathKeySet = Sets.newHashSet(); + fillClassPathKeySet(jobCacheFileConfig, classPathKeySet); + for(String key : classPathKeySet){ + String pathStr = jobCacheFileConfig.get(key); + jobGraph.getClasspaths().add(new URL("file:" + pathStr)); + } + } + + private void fillClassPathKeySet(Map jobCacheFileConfig, Set classPathKeySet) { + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(Strings.isNullOrEmpty(tmp.getValue())){ + continue; + } + + if(tmp.getValue().startsWith("class_path")){ + //DISTRIBUTED_CACHE_FILE_NAME_1 + //DISTRIBUTED_CACHE_FILE_PATH_1 + String key = tmp.getKey(); + String[] array = key.split("_"); + if(array.length < 5){ + continue; + } + + array[3] = "PATH"; + classPathKeySet.add(StringUtils.join(array, "_")); + } + } + } + + private JobGraph fillStreamJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); + } + } + return jobGraph; + } + + private void fillPluginPathToShipFiles(JobGraph jobGraph) { + List shipFiles = new ArrayList<>(); + // flinksql get classpath + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + shipFiles.add(new File(tmp.getValue().filePath)); + } + } + // flinkx get classpath + jobGraph.getClasspaths().forEach(jarFile -> { + try { + shipFiles.add(new File(jarFile.toURI())); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("Couldn't add local user jar: " + jarFile + + " Currently only file:/// URLs are supported."); + } + }); + jobGraph.getClasspaths().clear(); + addShipFiles(shipFiles); + } + + private String getUrlFormat(YarnConfiguration yarnConf){ + String url = ""; + try{ + Field rmClientField = yarnClient.getClass().getDeclaredField("rmClient"); + rmClientField.setAccessible(true); + Object rmClient = rmClientField.get(yarnClient); + + Field hField = rmClient.getClass().getSuperclass().getDeclaredField("h"); + hField.setAccessible(true); + //获取指定对象中此字段的值 + Object h = hField.get(rmClient); + Object currentProxy = null; + + try { + Field currentProxyField = h.getClass().getDeclaredField("currentProxy"); + currentProxyField.setAccessible(true); + currentProxy = currentProxyField.get(h); + }catch (Exception e){ + //兼容Hadoop 2.7.3.2.6.4.91-3 + LOG.warn("get currentProxy error:{}", ExceptionUtil.getErrorMessage(e)); + Field proxyDescriptorField = h.getClass().getDeclaredField("proxyDescriptor"); + proxyDescriptorField.setAccessible(true); + Object proxyDescriptor = proxyDescriptorField.get(h); + Field currentProxyField = proxyDescriptor.getClass().getDeclaredField("proxyInfo"); + currentProxyField.setAccessible(true); + currentProxy = currentProxyField.get(proxyDescriptor); + } + + Field proxyInfoField = currentProxy.getClass().getDeclaredField("proxyInfo"); + proxyInfoField.setAccessible(true); + String proxyInfoKey = (String) proxyInfoField.get(currentProxy); + + String key = "yarn.resourcemanager.webapp.address." + proxyInfoKey; + String addr = yarnConf.get(key); + + if(addr == null) { + addr = yarnConf.get("yarn.resourcemanager.webapp.address"); + } + + return String.format("http://%s/proxy",addr); + }catch (Exception e){ + LOG.warn("get monitor error:{}", ExceptionUtil.getErrorMessage(e)); + } + + return url; + } + + protected ClusterSpecification validateClusterResources( + ClusterSpecification clusterSpecification, + int yarnMinAllocationMB, + Resource maximumResourceCapability, + ClusterResourceDescription freeClusterResources) throws YarnDeploymentException { + + int taskManagerCount = clusterSpecification.getNumberTaskManagers(); + int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); + int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); + + if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { + LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " + + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + + "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + + "you requested will start."); + } + + // set the memory to minAllocationMB to do the next checks correctly + if (jobManagerMemoryMb < yarnMinAllocationMB) { + jobManagerMemoryMb = yarnMinAllocationMB; + } + if (taskManagerMemoryMb < yarnMinAllocationMB) { + taskManagerMemoryMb = yarnMinAllocationMB; + } + + final String note = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; + if (jobManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + note); + } + + if (taskManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + note); + } + + final String noteRsc = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + + "connecting from the beginning because the resources are currently not available in the cluster. " + + "The allocation might take more time than usual because the Flink YARN client needs to wait until " + + "the resources become available."; + int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount; + + if (freeClusterResources.totalFreeMemory < totalMemoryRequired) { + LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. " + + "There are currently only " + freeClusterResources.totalFreeMemory + "MB available." + noteRsc); + + } + if (taskManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + if (jobManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + + // ----------------- check if the requested containers fit into the cluster. + + int[] nmFree = Arrays.copyOf(freeClusterResources.nodeManagersFree, freeClusterResources.nodeManagersFree.length); + // first, allocate the jobManager somewhere. + if (!allocateResource(nmFree, jobManagerMemoryMb)) { + LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " + + "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " + + Arrays.toString(freeClusterResources.nodeManagersFree) + noteRsc); + } + // allocate TaskManagers + for (int i = 0; i < taskManagerCount; i++) { + if (!allocateResource(nmFree, taskManagerMemoryMb)) { + LOG.warn("There is not enough memory available in the YARN cluster. " + + "The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " + + "NodeManagers available: " + Arrays.toString(freeClusterResources.nodeManagersFree) + "\n" + + "After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " + + "the following NodeManagers are available: " + Arrays.toString(nmFree) + noteRsc); + } + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setNumberTaskManagers(clusterSpecification.getNumberTaskManagers()) + .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) + .createClusterSpecification(); + + } + + private void checkYarnQueues(YarnClient yarnClient) { + try { + List queues = yarnClient.getAllQueues(); + if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. + boolean queueFound = false; + for (QueueInfo queue : queues) { + if (queue.getQueueName().equals(this.yarnQueue)) { + queueFound = true; + break; + } + } + if (!queueFound) { + String queueNames = ""; + for (QueueInfo queue : queues) { + queueNames += queue.getQueueName() + ", "; + } + LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + + "Available queues: " + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + } catch (Throwable e) { + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Error details", e); + } + } + } + + public ApplicationReport startAppMaster( + Configuration configuration, + String applicationName, + String yarnClusterEntrypoint, + JobGraph jobGraph, + YarnClient yarnClient, + YarnClientApplication yarnApplication, + ClusterSpecification clusterSpecification) throws Exception { + + // ------------------ Initialize the file systems ------------------------- + + try { + org.apache.flink.core.fs.FileSystem.initialize(configuration); + } catch (IOException e) { + throw new IOException("Error while setting the default " + + "filesystem scheme from configuration.", e); + } + + // initialize file system + // Copy the application master jar to the filesystem + // Create a local resource to point to the destination jar path + final FileSystem fs = FileSystem.get(yarnConfiguration); + final Path homeDir = fs.getHomeDirectory(); + + // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. + if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && + fs.getScheme().startsWith("file")) { + LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the " + + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." + + "The Flink YARN client needs to store its files in a distributed file system"); + } + + ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + Set systemShipFiles = new HashSet<>(shipFiles.size()); + for (File file : shipFiles) { + systemShipFiles.add(file.getAbsoluteFile()); + } + + //check if there is a logback or log4j file + File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); + final boolean hasLogback = logbackFile.exists(); + if (hasLogback) { + systemShipFiles.add(logbackFile); + } + + File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); + final boolean hasLog4j = log4jFile.exists(); + if (hasLog4j) { + systemShipFiles.add(log4jFile); + if (hasLogback) { + // this means there is already a logback configuration file --> fail + LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + + "Logback configuration files. Please delete or rename one of them."); + } + } + + addLibFolderToShipFiles(systemShipFiles); + + // Set-up ApplicationSubmissionContext for the application + + final ApplicationId appId = appContext.getApplicationId(); + + // ------------------ Add Zookeeper namespace to local flinkConfiguraton ------ + String zkNamespace = getZookeeperNamespace(); + // no user specified cli argument for namespace? + if (zkNamespace == null || zkNamespace.isEmpty()) { + // namespace defined in config? else use applicationId as default. + zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); + setZookeeperNamespace(zkNamespace); + } + + configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); + + if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { + // activate re-execution of failed applications + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); + + activateHighAvailabilitySupport(appContext); + } else { + // set number of application retries to 1 in the default case + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + 1)); + } + + if (jobGraph != null) { + // add the user code jars from the provided JobGraph + for (org.apache.flink.core.fs.Path path : jobGraph.getUserJars()) { + userJarFiles.add(new File(path.toUri())); + } + } + + // local resource map for Yarn + final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); + // list of remote paths (after upload) + final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); + // ship list that enables reuse of resources for task manager containers + StringBuilder envShipFileList = new StringBuilder(); + + // upload and register ship files + List systemClassPaths = uploadAndRegisterFiles( + systemShipFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); + + final List userClassPaths = uploadAndRegisterFiles( + userJarFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { + systemClassPaths.addAll(userClassPaths); + } + + // normalize classpath by sorting + Collections.sort(systemClassPaths); + Collections.sort(userClassPaths); + + // classpath assembler + StringBuilder classPathBuilder = new StringBuilder(); + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + for (String classPath : systemClassPaths) { + classPathBuilder.append(classPath).append(File.pathSeparator); + } + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + + // Setup jar for ApplicationMaster + Path remotePathJar = setupSingleLocalResource( + "flink.jar", + fs, + appId, + flinkJarPath, + localResources, + homeDir, + ""); + + // set the right configuration values for the TaskManager + configuration.setInteger( + TaskManagerOptions.NUM_TASK_SLOTS, + clusterSpecification.getSlotsPerTaskManager()); + + configuration.setString( + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, + clusterSpecification.getTaskManagerMemoryMB() + "m"); + + // Upload the flink configuration + // write out configuration file + File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + tmpConfigurationFile.deleteOnExit(); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); + + Path remotePathConf = setupSingleLocalResource( + "flink-conf.yaml", + fs, + appId, + new Path(tmpConfigurationFile.getAbsolutePath()), + localResources, + homeDir, + ""); + + paths.add(remotePathJar); + classPathBuilder.append("flink.jar").append(File.pathSeparator); + paths.add(remotePathConf); + classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); + + // write job graph to tmp file and add it to local resource + // TODO: server use user main method to generate job graph + if (jobGraph != null) { + try { + File fp = File.createTempFile(appId.toString(), null); + fp.deleteOnExit(); + try (FileOutputStream output = new FileOutputStream(fp); + ObjectOutputStream obOutput = new ObjectOutputStream(output);){ + obOutput.writeObject(jobGraph); + } + + final String jobGraphFilename = "job.graph"; + flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); + + Path pathFromYarnURL = setupSingleLocalResource( + jobGraphFilename, + fs, + appId, + new Path(fp.toURI()), + localResources, + homeDir, + ""); + paths.add(pathFromYarnURL); + classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); + } catch (Exception e) { + LOG.warn("Add job graph to local resource fail"); + throw e; + } + } + + final Path yarnFilesDir = getYarnFilesDir(appId); + FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); + fs.setPermission(yarnFilesDir, permission); // set permission for path. + + //To support Yarn Secure Integration Test Scenario + //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML + //and KRB5 configuration files. We are adding these files as container local resources for the container + //applications (JM/TMs) to have proper secure cluster setup + Path remoteKrb5Path = null; + Path remoteYarnSiteXmlPath = null; + boolean hasKrb5 = false; + if (System.getenv("IN_TESTS") != null) { + String krb5Config = System.getProperty("java.security.krb5.conf"); + if (krb5Config != null && krb5Config.length() != 0) { + File krb5 = new File(krb5Config); + LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); + Path krb5ConfPath = new Path(krb5.getAbsolutePath()); + remoteKrb5Path = setupSingleLocalResource( + Utils.KRB5_FILE_NAME, + fs, + appId, + krb5ConfPath, + localResources, + homeDir, + ""); + + File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); + LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); + Path yarnSitePath = new Path(f.getAbsolutePath()); + remoteYarnSiteXmlPath = setupSingleLocalResource( + Utils.YARN_SITE_FILE_NAME, + fs, + appId, + yarnSitePath, + localResources, + homeDir, + ""); + hasKrb5 = true; + } + } + + // setup security tokens + Path remotePathKeytab = null; + String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + if (keytab != null) { + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = setupSingleLocalResource( + Utils.KEYTAB_FILE_NAME, + fs, + appId, + new Path(keytab), + localResources, + homeDir, + ""); + } + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( + yarnClusterEntrypoint, + hasLogback, + hasLog4j, + hasKrb5, + clusterSpecification.getMasterMemoryMB()); + + if (UserGroupInformation.isSecurityEnabled()) { + // set HDFS delegation tokens when security is enabled + LOG.info("Adding delegation token to the AM container.."); + Utils.setTokensFor(amContainer, paths, yarnConfiguration); + } + + amContainer.setLocalResources(localResources); + fs.close(); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = new HashMap<>(); + // set user specified app master environment variables + appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); + // set Flink app class path + appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); + + // set Flink on YARN internal configuration values + appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(clusterSpecification.getNumberTaskManagers())); + appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(clusterSpecification.getTaskManagerMemoryMB())); + appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(clusterSpecification.getSlotsPerTaskManager())); + appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); + appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); + appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); + + // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); + + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); + String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + } + + //To support Yarn Secure Integration Test Scenario + if (remoteYarnSiteXmlPath != null && remoteKrb5Path != null) { + appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); + } + + if (dynamicPropertiesEncoded != null) { + appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); + } + + // set classpath from YARN configuration + Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(clusterSpecification.getMasterMemoryMB()); + capability.setVirtualCores(1); + + final String customApplicationName = customName != null ? customName : applicationName; + + appContext.setApplicationName(customApplicationName); + appContext.setApplicationType("Apache Flink"); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + + if (yarnQueue != null) { + appContext.setQueue(yarnQueue); + } + + setApplicationNodeLabel(appContext); + + setApplicationTags(appContext); + + // add a hook to clean up in case deployment fails + Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication, yarnFilesDir); + Runtime.getRuntime().addShutdownHook(deploymentFailureHook); + LOG.info("Submitting application master " + appId); + yarnClient.submitApplication(appContext); + + LOG.info("Waiting for the cluster to be allocated"); + final long startTime = System.currentTimeMillis(); + ApplicationReport report; + YarnApplicationState lastAppState = YarnApplicationState.NEW; + loop: while (true) { + try { + report = yarnClient.getApplicationReport(appId); + } catch (IOException e) { + throw new YarnDeploymentException("Failed to deploy the cluster.", e); + } + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + switch(appState) { + case FAILED: + case FINISHED: + case KILLED: + throw new YarnDeploymentException("The YARN application unexpectedly switched to state " + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + appId); + //break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break loop; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + if (System.currentTimeMillis() - startTime > 60000) { + LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); + } + + } + lastAppState = appState; + Thread.sleep(250); + } + // print the application id for user to cancel themselves. + if (isDetachedMode()) { + LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + + "Flink on YARN, use the following command or a YARN web interface to stop " + + "it:\nyarn application -kill " + appId + "\nPlease also note that the " + + "temporary files of the YARN session in the home directory will not be removed."); + } + // since deployment was successful, remove the hook + ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); + return report; + } + + /** + * Returns the Path where the YARN application files should be uploaded to. + * + * @param appId YARN application id + */ + private Path getYarnFilesDir(final ApplicationId appId) throws IOException { + final FileSystem fileSystem = FileSystem.get(yarnConfiguration); + final Path homeDir = fileSystem.getHomeDirectory(); + return new Path(homeDir, ".flink/" + appId + '/'); + } + + /** + * Uploads and registers a single resource and adds it to localResources. + * + * @param key + * the key to add the resource under + * @param fs + * the remote file system to upload to + * @param appId + * application ID + * @param localSrcPath + * local path to the file + * @param localResources + * map of resources + * + * @return the remote path to the uploaded resource + */ + private static Path setupSingleLocalResource( + String key, + FileSystem fs, + ApplicationId appId, + Path localSrcPath, + Map localResources, + Path targetHomeDir, + String relativeTargetPath) throws IOException, URISyntaxException { + + Tuple2 resource = Utils.setupLocalResource( + fs, + appId.toString(), + localSrcPath, + targetHomeDir, + relativeTargetPath); + + localResources.put(key, resource.f1); + + return resource.f0; + } + + /** + * Recursively uploads (and registers) any (user and system) files in shipFiles except + * for files matching "flink-dist*.jar" which should be uploaded separately. + * + * @param shipFiles + * files to upload + * @param fs + * file system to upload to + * @param targetHomeDir + * remote home directory to upload to + * @param appId + * application ID + * @param remotePaths + * paths of the remote resources (uploaded resources will be added) + * @param localResources + * map of resources (uploaded resources will be added) + * @param envShipFileList + * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} + * + * @return list of class paths with the the proper resource keys from the registration + */ + static List uploadAndRegisterFiles( + Collection shipFiles, + FileSystem fs, + Path targetHomeDir, + ApplicationId appId, + List remotePaths, + Map localResources, + StringBuilder envShipFileList) throws IOException, URISyntaxException { + + final List classPaths = new ArrayList<>(2 + shipFiles.size()); + for (File shipFile : shipFiles) { + if (shipFile.isDirectory()) { + // add directories to the classpath + java.nio.file.Path shipPath = shipFile.toPath(); + final java.nio.file.Path parentPath = shipPath.getParent(); + + Files.walkFileTree(shipPath, new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) + throws IOException { + String fileName = file.getFileName().toString(); + if (!(fileName.startsWith("flink-dist") && + fileName.endsWith("jar"))) { + + java.nio.file.Path relativePath = parentPath.relativize(file); + + String key = relativePath.toString(); + try { + Path remotePath = setupSingleLocalResource( + key, + fs, + appId, + new Path(file.toUri()), + localResources, + targetHomeDir, + relativePath.getParent().toString()); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=") + .append(remotePath).append(","); + + // add files to the classpath + classPaths.add(key); + } catch (URISyntaxException e) { + throw new IOException(e); + } + } + + return FileVisitResult.CONTINUE; + } + }); + } else { + if (!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) { + Path shipLocalPath = new Path(shipFile.toURI()); + String key = shipFile.getName(); + Path remotePath = setupSingleLocalResource( + key, fs, appId, shipLocalPath, localResources, targetHomeDir, ""); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + + // add files to the classpath + classPaths.add(key); + } + } + + } + return classPaths; + } + + /** + * Kills YARN application and stops YARN client. + * + *

Use this method to kill the App before it has been properly deployed + */ + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { + LOG.info("Killing YARN application"); + + try { + yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); + } catch (Exception e) { + // we only log a debug message here because the "killApplication" call is a best-effort + // call (we don't know if the application has been deployed when the error occured). + LOG.debug("Error while killing YARN application", e); + } + yarnClient.stop(); + } + + private static class ClusterResourceDescription { + public final int totalFreeMemory; + public final int containerLimit; + public final int[] nodeManagersFree; + + public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { + this.totalFreeMemory = totalFreeMemory; + this.containerLimit = containerLimit; + this.nodeManagersFree = nodeManagersFree; + } + } + + private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + + int totalFreeMemory = 0; + int containerLimit = 0; + int[] nodeManagersFree = new int[nodes.size()]; + + for (int i = 0; i < nodes.size(); i++) { + NodeReport rep = nodes.get(i); + int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0); + nodeManagersFree[i] = free; + totalFreeMemory += free; + if (free > containerLimit) { + containerLimit = free; + } + } + return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); + } + + @Override + public String getClusterDescription() { + + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + + YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); + + ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + final String format = "|%-16s |%-16s %n"; + ps.printf("|Property |Value %n"); + ps.println("+---------------------------------------+"); + int totalMemory = 0; + int totalCores = 0; + for (NodeReport rep : nodes) { + final Resource res = rep.getCapability(); + totalMemory += res.getMemory(); + totalCores += res.getVirtualCores(); + ps.format(format, "NodeID", rep.getNodeId()); + ps.format(format, "Memory", res.getMemory() + " MB"); + ps.format(format, "vCores", res.getVirtualCores()); + ps.format(format, "HealthReport", rep.getHealthReport()); + ps.format(format, "Containers", rep.getNumContainers()); + ps.println("+---------------------------------------+"); + } + ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); + List qInfo = yarnClient.getAllQueues(); + for (QueueInfo q : qInfo) { + ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + + q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); + } + return baos.toString(); + } catch (Exception e) { + throw new RuntimeException("Couldn't get cluster description", e); + } + } + + public void setName(String name) { + if (name == null) { + throw new IllegalArgumentException("The passed name is null"); + } + customName = name; + } + + private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws + InvocationTargetException, IllegalAccessException { + + ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + + reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); + reflector.setAttemptFailuresValidityInterval(appContext, AkkaUtils.getTimeout(flinkConfiguration).toMillis()); + } + + private void setApplicationTags(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + final String tagsString = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TAGS); + + final Set applicationTags = new HashSet<>(); + + // Trim whitespace and cull empty tags + for (final String tag : tagsString.split(",")) { + final String trimmedTag = tag.trim(); + if (!trimmedTag.isEmpty()) { + applicationTags.add(trimmedTag); + } + } + + reflector.setApplicationTags(appContext, applicationTags); + } + + private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + if (nodeLabel != null) { + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + reflector.setApplicationNodeLabel(appContext, nodeLabel); + } + } + + /** + * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext} + * supports various methods which, depending on the Hadoop version, may or may not be supported. + * + *

If an unsupported method is invoked, nothing happens. + * + *

Currently three methods are proxied: + * - setApplicationTags (>= 2.4.0) + * - setAttemptFailuresValidityInterval (>= 2.6.0) + * - setKeepContainersAcrossApplicationAttempts (>= 2.4.0) + * - setNodeLabelExpression (>= 2.6.0) + */ + private static class ApplicationSubmissionContextReflector { + private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); + + private static final ApplicationSubmissionContextReflector instance = + new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); + + public static ApplicationSubmissionContextReflector getInstance() { + return instance; + } + + private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; + private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; + private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; + private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; + + private final Method applicationTagsMethod; + private final Method attemptFailuresValidityIntervalMethod; + private final Method keepContainersMethod; + @Nullable + private final Method nodeLabelExpressionMethod; + + private ApplicationSubmissionContextReflector(Class clazz) { + Method applicationTagsMethod; + Method attemptFailuresValidityIntervalMethod; + Method keepContainersMethod; + Method nodeLabelExpressionMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + applicationTagsMethod = null; + } + + this.applicationTagsMethod = applicationTagsMethod; + + try { + // this method is only supported by Hadoop 2.6.0 onwards + attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + attemptFailuresValidityIntervalMethod = null; + } + + this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + keepContainersMethod = null; + } + + this.keepContainersMethod = keepContainersMethod; + + try { + nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + nodeLabelExpressionMethod = null; + } + + this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; + } + + public void setApplicationTags( + ApplicationSubmissionContext appContext, + Set applicationTags) throws InvocationTargetException, IllegalAccessException { + if (applicationTagsMethod != null) { + LOG.debug("Calling method {} of {}.", + applicationTagsMethod.getName(), + appContext.getClass().getCanonicalName()); + applicationTagsMethod.invoke(appContext, applicationTags); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + APPLICATION_TAGS_METHOD_NAME); + } + } + + public void setApplicationNodeLabel( + ApplicationSubmissionContext appContext, + String nodeLabel) throws InvocationTargetException, IllegalAccessException { + if (nodeLabelExpressionMethod != null) { + LOG.debug("Calling method {} of {}.", + nodeLabelExpressionMethod.getName(), + appContext.getClass().getCanonicalName()); + nodeLabelExpressionMethod.invoke(appContext, nodeLabel); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + NODE_LABEL_EXPRESSION_NAME); + } + } + + public void setAttemptFailuresValidityInterval( + ApplicationSubmissionContext appContext, + long validityInterval) throws InvocationTargetException, IllegalAccessException { + if (attemptFailuresValidityIntervalMethod != null) { + LOG.debug("Calling method {} of {}.", + attemptFailuresValidityIntervalMethod.getName(), + appContext.getClass().getCanonicalName()); + attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + ATTEMPT_FAILURES_METHOD_NAME); + } + } + + public void setKeepContainersAcrossApplicationAttempts( + ApplicationSubmissionContext appContext, + boolean keepContainers) throws InvocationTargetException, IllegalAccessException { + + if (keepContainersMethod != null) { + LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(), + appContext.getClass().getCanonicalName()); + keepContainersMethod.invoke(appContext, keepContainers); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } + } + } + + private static class YarnDeploymentException extends RuntimeException { + private static final long serialVersionUID = -812040641215388943L; + + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); + } + } + + private class DeploymentFailureHook extends Thread { + + private final YarnClient yarnClient; + private final YarnClientApplication yarnApplication; + private final Path yarnFilesDir; + + DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication, Path yarnFilesDir) { + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.yarnApplication = Preconditions.checkNotNull(yarnApplication); + this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); + } + + @Override + public void run() { + LOG.info("Cancelling deployment from Deployment Failure Hook"); + failSessionDuringDeployment(yarnClient, yarnApplication); + LOG.info("Deleting files in {}.", yarnFilesDir); + try { + FileSystem fs = FileSystem.get(yarnConfiguration); + + if (!fs.delete(yarnFilesDir, true)) { + throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); + } + + fs.close(); + } catch (IOException e) { + LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); + } + } + } + + protected void addLibFolderToShipFiles(Collection effectiveShipFiles) { + // Add lib folder to the ship files if the environment variable is set. + // This is for convenience when running from the command-line. + // (for other files users explicitly set the ship files) + String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); + if (libDir != null) { + File libDirFile = new File(libDir); + if (libDirFile.isDirectory()) { + effectiveShipFiles.add(libDirFile); + } else { + throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + + "' is set to '" + libDir + "' but the directory doesn't exist."); + } + } else if (this.shipFiles.isEmpty()) { + LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + + "Not shipping any library files.", ENV_FLINK_LIB_DIR); + } + } + + protected ContainerLaunchContext setupApplicationMasterContainer( + String yarnClusterEntrypoint, + boolean hasLogback, + boolean hasLog4j, + boolean hasKrb5, + int jobManagerMemoryMb) { + // ------------------ Prepare Application Master Container ------------------------------ + + // respect custom JVM options in the YAML file + String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS); + if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) { + javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS); + } + //applicable only for YarnMiniCluster secure test run + //krb5.conf file will be available as local resource in JM/TM container + if (hasKrb5) { + javaOpts += " -Djava.security.krb5.conf=krb5.conf"; + } + + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); + + final Map startCommandValues = new HashMap<>(); + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + + int heapSize = Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); + String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); + startCommandValues.put("jvmmem", jvmHeapMem); + + startCommandValues.put("jvmopts", javaOpts); + String logging = ""; + + if (hasLogback || hasLog4j) { + logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; + + if (hasLogback) { + logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; + } + + if (hasLog4j) { + logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; + } + } + + startCommandValues.put("logging", logging); + startCommandValues.put("class", yarnClusterEntrypoint); + startCommandValues.put("redirects", + "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + + "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"); + startCommandValues.put("args", ""); + + final String commandTemplate = flinkConfiguration + .getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, + ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE); + final String amCommand = + BootstrapTools.getStartCommand(commandTemplate, startCommandValues); + + amContainer.setCommands(Collections.singletonList(amCommand)); + + LOG.debug("Application Master start command: " + amCommand); + + return amContainer; + } + + private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(Configuration config) { + throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(config); + + return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + } + + private static void throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(final Configuration config) { + final String userJarInclusion = config.getString(YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + if ("DISABLED".equalsIgnoreCase(userJarInclusion)) { + throw new IllegalArgumentException(String.format("Config option %s cannot be set to DISABLED anymore (see FLINK-11781)", + YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key())); + } + } + + /** + * Creates a YarnClusterClient; may be overridden in tests. + */ + protected abstract ClusterClient createYarnClusterClient( + AbstractYarnClusterDescriptor descriptor, + int numberTaskManagers, + int slotsPerTaskManager, + ApplicationReport report, + Configuration flinkConfiguration, + boolean perJobCluster) throws Exception; +} + diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml index f38c4665c6..c2f9c1ac13 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml @@ -40,7 +40,7 @@ + tofile="${basedir}/../../plugins/mongodboplogreader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml index d2630bba2c..182ea07067 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -57,7 +57,7 @@ + tofile="${basedir}/../../plugins/pgwalreader/${project.name}-${package.name}.jar" /> From f52d81f2846541198f704472953f0af669b03673 Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 15 Apr 2020 17:13:33 +0800 Subject: [PATCH 038/136] =?UTF-8?q?fix=E4=BB=BB=E5=8A=A1=E6=8F=90=E4=BA=A4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README_OLD.md | 7 +- flinkx-core/pom.xml | 20 +-- .../com/dtstack/flinkx/options/Options.java | 11 ++ .../flinkx/launcher/ClusterClientFactory.java | 131 ++++++++++++------ .../com/dtstack/flinkx/launcher/Launcher.java | 63 ++++++++- .../dtstack/flinkx/launcher/PluginUtil.java | 54 ++++++++ .../perjob/PerJobClusterClientBuilder.java | 33 ++--- .../launcher/perjob/PerJobSubmitter.java | 4 +- .../yarn/AbstractYarnClusterDescriptor.java | 14 ++ pom.xml | 1 - 10 files changed, 237 insertions(+), 101 deletions(-) create mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java diff --git a/README_OLD.md b/README_OLD.md index 144b82a9bc..6ed77f1821 100644 --- a/README_OLD.md +++ b/README_OLD.md @@ -45,13 +45,8 @@ 进入项目根目录,使用maven打包: -windows平台 ``` -mvn clean package -DskipTests -DscriptType=bat -``` -unix平台 -``` -mvn clean package -DskipTests -DscriptType=sh +mvn clean package -DskipTests ``` 打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 08d35429c9..223aae3824 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -150,7 +150,7 @@ com.fasterxml.jackson.core jackson-databind - 2.9.10.1 + 2.9.10.3 io.prometheus @@ -166,24 +166,6 @@ - - exec-maven-plugin - org.codehaus.mojo - 1.6.0 - - - Version Calculation - generate-sources - - exec - - - ${basedir}/../bin/install_jars.${scriptType} - - - - - org.apache.maven.plugins maven-compiler-plugin diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 804f983bdb..9b57f5583b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -73,6 +73,17 @@ public class Options { @OptionRequired(description = "plugin load mode, by classpath or shipfile") private String pluginLoadMode = "shipfile"; + @OptionRequired(description = "applicationId on yarn cluster") + private String appId; + + public String getAppId() { + return appId; + } + + public void setAppId(String appId) { + this.appId = appId; + } + public String getS() { return s; } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index fbbf7bff38..dc952713f6 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -31,18 +31,22 @@ import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.File; import java.net.InetSocketAddress; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.net.URL; +import java.util.*; /** * The Factory of ClusterClient @@ -79,60 +83,37 @@ public static ClusterClient createStandaloneClient(Options launcherOptions) thro } public static ClusterClient createYarnClient(Options launcherOptions) { - String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(launcherOptions.getFlinkconf()); String yarnConfDir = launcherOptions.getYarnconf(); if(StringUtils.isNotBlank(yarnConfDir)) { - try { - config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); - FileSystem.initialize(config); + flinkConfig.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + FileSystem.initialize(flinkConfig); YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); YarnClient yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); yarnClient.start(); - ApplicationId applicationId = null; - - Set set = new HashSet<>(); - set.add("Apache Flink"); - EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); - enumSet.add(YarnApplicationState.RUNNING); - List reportList = yarnClient.getApplications(set, enumSet); - - int maxMemory = -1; - int maxCores = -1; - for(ApplicationReport report : reportList) { - if(!report.getName().startsWith("Flink session")){ - continue; - } + ApplicationId applicationId; - if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { - continue; + if (StringUtils.isEmpty(launcherOptions.getAppId())) { + applicationId = getAppIdFromYarn(yarnClient); + if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); } - - int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); - int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); - - boolean isOverMaxResource = thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores; - if(isOverMaxResource) { - maxMemory = thisMemory; - maxCores = thisCores; - applicationId = report.getApplicationId(); - } - } - - if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + } else { + applicationId = ConverterUtils.toApplicationId(launcherOptions.getAppId()); } - HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(config); + HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(flinkConfig); if(highAvailabilityMode.equals(HighAvailabilityMode.ZOOKEEPER) && applicationId!=null){ - config.setString(HighAvailabilityOptions.HA_CLUSTER_ID,applicationId.toString()); + flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID,applicationId.toString()); } - YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor(config, yarnConf, "", yarnClient, false); + + AbstractYarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(launcherOptions, yarnClient, yarnConf, flinkConfig); ClusterClient clusterClient = yarnClusterDescriptor.retrieve(applicationId); clusterClient.setDetached(true); + return clusterClient; } catch(Exception e) { throw new RuntimeException(e); @@ -141,4 +122,70 @@ public static ClusterClient createYarnClient(Options launcherOptions) { throw new UnsupportedOperationException("Haven't been developed yet!"); } + + private static AbstractYarnClusterDescriptor getClusterDescriptor(Options launcherOptions, + YarnClient yarnClient, + YarnConfiguration yarnConf, + Configuration flinkConfig) throws Exception{ + AbstractYarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor(flinkConfig, yarnConf, "", yarnClient, true); + + // plugin dependent on shipfile + if (StringUtils.isNotEmpty(launcherOptions.getPluginLoadMode()) && "shipfile".equalsIgnoreCase(launcherOptions.getPluginLoadMode())) { + List pluginPaths = PluginUtil.getAllPluginPath(launcherOptions.getPluginRoot()); + if (!pluginPaths.isEmpty()) { + yarnClusterDescriptor.addShipFiles(pluginPaths); + } + } + + String flinkJarPath = launcherOptions.getFlinkLibJar(); + if (StringUtils.isNotEmpty(flinkJarPath)) { + List classpaths = new ArrayList<>(); + File[] jars = new File(flinkJarPath).listFiles(); + for (File file : jars) { + if (file.toURI().toURL().toString().contains("flink-dist")) { + yarnClusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); + } else { + classpaths.add(file.toURI().toURL()); + } + } + + yarnClusterDescriptor.setProvidedUserJarFiles(classpaths); + } + + yarnClusterDescriptor.setName(launcherOptions.getJobid()); + return yarnClusterDescriptor; + } + + private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exception{ + Set set = new HashSet<>(); + set.add("Apache Flink"); + EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); + enumSet.add(YarnApplicationState.RUNNING); + List reportList = yarnClient.getApplications(set, enumSet); + + ApplicationId applicationId = null; + int maxMemory = -1; + int maxCores = -1; + for(ApplicationReport report : reportList) { + if(!report.getName().startsWith("Flink session")){ + continue; + } + + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; + } + + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); + int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); + + boolean isOverMaxResource = thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores; + if(isOverMaxResource) { + maxMemory = thisMemory; + maxCores = thisCores; + applicationId = report.getApplicationId(); + } + } + + return applicationId; + } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 19dc85366f..91a4bd1409 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -52,6 +52,8 @@ public class Launcher { public static final String KEY_FLINKX_HOME = "FLINKX_HOME"; + public static final String KEY_FLINK_HOME = "FLINK_HOME"; + public static final String KEY_HADOOP_HOME = "HADOOP_HOME"; public static final String PLUGINS_DIR_NAME = "plugins"; @@ -88,7 +90,7 @@ public static void main(String[] args) throws Exception { setLogLevel(Level.INFO.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); - findDefaultPluginRoot(launcherOptions); + findDefaultConfigDir(launcherOptions); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); @@ -139,17 +141,57 @@ public static void main(String[] args) throws Exception { } } + private static void findDefaultConfigDir(Options launcherOptions) { + findDefaultPluginRoot(launcherOptions); + + if (ClusterMode.local.name().equalsIgnoreCase(launcherOptions.getMode())) { + return; + } + + findDefaultFlinkConf(launcherOptions); + findDefaultHadoopConf(launcherOptions); + } + + private static void findDefaultHadoopConf(Options launcherOptions) { + if (StringUtils.isNotEmpty(launcherOptions.getYarnconf())) { + return; + } + + String hadoopHome = getSystemProperty(KEY_HADOOP_HOME); + if (StringUtils.isNotEmpty(hadoopHome)) { + hadoopHome = hadoopHome.trim(); + if (hadoopHome.endsWith(File.separator)) { + hadoopHome = hadoopHome.substring(0, hadoopHome.lastIndexOf(File.separator)); + } + + launcherOptions.setYarnconf(hadoopHome + "/etc/hadoop"); + } + } + + private static void findDefaultFlinkConf(Options launcherOptions) { + if (StringUtils.isNotEmpty(launcherOptions.getFlinkconf()) && StringUtils.isNotEmpty(launcherOptions.getFlinkLibJar())) { + return; + } + + String flinkHome = getSystemProperty(KEY_FLINK_HOME); + if (StringUtils.isNotEmpty(flinkHome)) { + flinkHome = flinkHome.trim(); + if (flinkHome.endsWith(File.separator)){ + flinkHome = flinkHome.substring(0, flinkHome.lastIndexOf(File.separator)); + } + + launcherOptions.setFlinkconf(flinkHome + "/conf"); + launcherOptions.setFlinkLibJar(flinkHome + "/lib"); + } + } + private static void findDefaultPluginRoot(Options launcherOptions) { String pluginRoot = launcherOptions.getPluginRoot(); if (StringUtils.isNotEmpty(pluginRoot)) { return; } - String flinkxHome = System.getenv(KEY_FLINKX_HOME); - if (StringUtils.isEmpty(flinkxHome)) { - flinkxHome = System.getProperty(KEY_FLINKX_HOME); - } - + String flinkxHome = getSystemProperty(KEY_FLINKX_HOME); if (StringUtils.isNotEmpty(flinkxHome)) { flinkxHome = flinkxHome.trim(); if (flinkxHome.endsWith(File.separator)) { @@ -162,6 +204,15 @@ private static void findDefaultPluginRoot(Options launcherOptions) { } } + private static String getSystemProperty(String name) { + String property = System.getenv(name); + if (StringUtils.isEmpty(property)) { + property = System.getProperty(name); + } + + return property; + } + private static String getCoreJarFileName (String pluginRoot) throws FileNotFoundException{ String coreJarFileName = null; File pluginDir = new File(pluginRoot); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java new file mode 100644 index 0000000000..cbba6cd27e --- /dev/null +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.launcher; + +import com.google.common.collect.Lists; +import org.apache.commons.lang3.StringUtils; + +import java.io.File; +import java.util.Arrays; +import java.util.List; + +/** + * @author jiangbo + * @date 2020/4/15 + */ +public class PluginUtil { + + public static List getAllPluginPath(String flinkPluginRoot) { + List pluginPaths = Lists.newArrayList(); + if (StringUtils.isNotBlank(flinkPluginRoot)) { + try { + File[] jars = new File(flinkPluginRoot).listFiles(); + if (jars != null) { + pluginPaths.addAll(Arrays.asList(jars)); + } + } catch (Exception e) { + throw new RuntimeException("Get jars from pluginRoot error", e); + } + } + + if (pluginPaths.size() == 0) { + throw new RuntimeException("jars in pluginRoot is null, flinkPluginRoot = " + flinkPluginRoot); + } + + return pluginPaths; + } +} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java index 1e16bae775..998ee73516 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java @@ -17,9 +17,8 @@ */ package com.dtstack.flinkx.launcher.perjob; +import com.dtstack.flinkx.launcher.PluginUtil; import com.dtstack.flinkx.options.Options; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.common.collect.Lists; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.security.SecurityConfiguration; @@ -62,7 +61,7 @@ public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properti yarnClient.init(yarnConf); yarnClient.start(); - System.out.println("----init yarn success ----"); + LOG.info("----init yarn success ----"); } /** @@ -86,6 +85,8 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co confProp.forEach((key, value) -> conf.setString(key.toString(), value.toString())); AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(conf, yarnConf, options.getFlinkconf(), yarnClient, false); + descriptor.setName(options.getJobid()); + List shipFiles = new ArrayList<>(); File[] jars = new File(flinkJarPath).listFiles(); if (jars != null) { @@ -98,10 +99,10 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } } - if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { - List files = fillAllPluginPathForYarnSession(options.getPluginRoot()); - shipFiles.addAll(files); - } +// if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { +// List files = PluginUtil.getAllPluginPath(options.getPluginRoot()); +// shipFiles.addAll(files); +// } if (StringUtils.isNotBlank(options.getQueue())) { descriptor.setQueue(options.getQueue()); @@ -119,22 +120,4 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co descriptor.addShipFiles(shipFiles); return descriptor; } - - private List fillAllPluginPathForYarnSession(String flinkPluginRoot) { - List pluginPaths = Lists.newArrayList(); - //预加载同步插件jar包 - if (org.apache.commons.lang3.StringUtils.isNotBlank(flinkPluginRoot)) { - try { - File[] jars = new File(flinkPluginRoot).listFiles(); - if (jars != null) { - pluginPaths.addAll(Arrays.asList(jars)); - } else { - LOG.warn("jars in flinkPluginRoot is null, flinkPluginRoot = {}", flinkPluginRoot); - } - } catch (Exception e) { - LOG.error("error to load jars in flinkPluginRoot, flinkPluginRoot = {}, e = {}", flinkPluginRoot, ExceptionUtil.getErrorMessage(e)); - } - } - return pluginPaths; - } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index 3119df54ca..d42f744a05 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -62,6 +62,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) YarnConfiguration yarnConf = StringUtils.isEmpty(options.getYarnconf()) ? new YarnConfiguration() : YarnConfLoader.getYarnConf(options.getYarnconf()); Configuration flinkConfig = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); + flinkConfig.setString("classloader.resolve-order", "child-first"); Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); @@ -79,8 +80,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) clusterSpecification.setCreateProgramDelay(true); clusterSpecification.setYarnConfiguration(yarnConf); - ClassLoaderType classLoaderType = ClassLoaderType.getByClassMode(options.getPluginLoadMode()); - clusterSpecification.setClassLoaderType(classLoaderType); + clusterSpecification.setClassLoaderType(ClassLoaderType.PARENT_FIRST_CACHE); PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); perJobClusterClientBuilder.init(yarnConf, flinkConfig, conProp); diff --git a/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 5f7e39293a..d8a02e134b 100644 --- a/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -189,6 +189,20 @@ public void addShipFiles(List shipFiles) { this.shipFiles.addAll(shipFiles); } + /** + * Sets the user jar which is included in the system classloader of all nodes. + */ + public void setProvidedUserJarFiles(List userJarFiles) { + for (URL jarFile : userJarFiles) { + try { + this.userJarFiles.add(new File(jarFile.toURI())); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("Couldn't add local user jar: " + jarFile + + " Currently only file:/// URLs are supported."); + } + } + } + public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; } diff --git a/pom.xml b/pom.xml index 73ddb257f6..78e34c817b 100644 --- a/pom.xml +++ b/pom.xml @@ -66,7 +66,6 @@ 4.5.3 ${basedir}/dev release_1.8.5 - sh From c5246cb2649c30acebd456826f9272c795ff7e8f Mon Sep 17 00:00:00 2001 From: jiangbo Date: Wed, 15 Apr 2020 17:41:45 +0800 Subject: [PATCH 039/136] add images --- docs/images/quick_1.png | Bin 0 -> 159395 bytes docs/images/quick_2.png | Bin 0 -> 106263 bytes docs/images/quick_3.png | Bin 0 -> 156375 bytes docs/images/quick_4.png | Bin 0 -> 240317 bytes docs/images/quick_5.png | Bin 0 -> 112928 bytes docs/images/quick_6.png | Bin 0 -> 160393 bytes docs/images/quick_7.png | Bin 0 -> 265959 bytes docs/images/quick_8.png | Bin 0 -> 190881 bytes 8 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/images/quick_1.png create mode 100644 docs/images/quick_2.png create mode 100644 docs/images/quick_3.png create mode 100644 docs/images/quick_4.png create mode 100644 docs/images/quick_5.png create mode 100644 docs/images/quick_6.png create mode 100644 docs/images/quick_7.png create mode 100644 docs/images/quick_8.png diff --git a/docs/images/quick_1.png b/docs/images/quick_1.png new file mode 100644 index 0000000000000000000000000000000000000000..6b1f4450c722e781519428f410bd4602e02a8556 GIT binary patch literal 159395 zcmZU)by(X?(>9ER65N8jTksSw?ykitE-l5q6bb-uJ&vj@_Nz+5OGzoO4dJmWCn@CIu!E5)zKGlAI0_654kpBvc&W3&cOkT7T(~ zkm!(<<)rm|k$?B12UFWyRvusHWC>I(9Cy}wf0OSdC2ies?Us{=49uHXHq*pzeRS`_i+9SBanShbPf_oxx zCL6hv4C#r8C-o4`1E5GdEw;F(+a{|q!)+T(31M>p zMBj)xUv?Bk*F|4548-rnua^BowFUo~s+Rbl!Fxn9>)-QyJ7}C1GpXgZPg`CA=`zOX z%eISwi0*u{TU7sw9x7u({;$wd06k)wi1>wI0Q>(eszU)JKe%;;n29YW6o;~*!jZzf z7EgbE#AenoJsh!EZrE!*y|{l?{m&Zy-N?U<_=UV;w#Bb*j)-x~$3r{r|IDl^r3*s) zf4crN*mM_)5Bx{`5Od*w_3^*w`&WXn>i;3(f64540svQqnYtPL7^lCwiKOUQWwW4_ zNJ(V$LvX;V3}plV|H}RycT=DY3@HX8Hdk~c?8O>WT*!v`&lb)Uc3u75Ze!2ntAT0F z>wig04r%n%6iF4|4&8yySsdleDB7zn`qvCTLS)n&W%wU!peMg!RBHbx{GN+A!4=l(MDHyyMY~C z(qpR$q-vv#oQv(QjBzzWo#oHmUsHU38iq5ow)~1>Ry9y@GSw(ewc#hPU&1bTwEDla z(!qyN`rJX^)>$#ze}%cCt2i2;lb|r;T+;p;@m&jq%`t-99ak%%TBi}CzziYnxh}#C zp@AZtXZZJNWNua~_-q;>Wd_z*J>sYlf=W~W@AGg0%K&pVSZNT~LHpl(GnOS}sywX! zhczX4bG@84C7T9KRlb?Qx3ZU+Q>zRw1XXtc9?Q;;4I>CMWC$DBbZ3D|&8_G`xU42; zn_b<9JF#h3;%rG;s2mnsOqDPWJhRnU;{VqvBhF`tsf;OSf`R!aKmsJcp+h7?2VKWz zXVy_M=OM&Efr1`1>3q{u>7*E1K+x22{9G0hbxbt)1qQv;IT%BUI{^tm2q#Ag0RRY- z7Jd7Z2%+mBb`cjDU}-9__ub?UkKIYyAM1d1I>n|3Dgl?x@@YI6)(O1u($(hSK`D)4u3 z9j4Z#fm!Ag475RfB_;tt=t7Q^4kLIJ6s5A?48kPD4_(3!Uaqg?#RN2vC*?8(cl1wK z7;$2_5YP!{;9HcHkiji^J;r0SraNpMo;d83?0`vx0J1uX@k_E=A$q_bOvh>q<_Z*Q zJCs$4#bSjUyMY}~jon%v8f*PXmpow~DZ9|o(t_vJ<-l@vxd?4wz;vPbO6<}|gix&{ zveH4MlugHatIV-H{7|KNs+q$3!TnpBbbDR@E53{_~Tnlb}ptu6_cwSX#VrykmO5Q`XEG)ZlFyB|!BFBmXmQ$G3O9TuHes%u0@glCC^()N4Af zu!WRVLV7?gDLu1{RO_6LHk)SWu$(ju0`eZ9rcx0eF~n5DkU2G;HHB%#hj7|h*qW(b zIoJ@gWW*S7md!3G_l|^U`8pyz#8fg#Ixm^h3EjIvg3t>R>=$mrRBCE!rz%jb%C{ys zGF#K>&k7^}f>tR03a~2lYZ4*i%0Lkp4I5d>&Jd{AQI8vm&0G4A8=8suOv`NCPl4!y zPr)age|EFrf$q^5Ppoq-P=u=U@~9D)waNKo#GA4s74PjUnGqBGcE-%#7lv)KTHj!j7uc}a z$Ke+ksSC4O1a|%oLY*ZY9L*KcBFE-dtRMDs;;h~2#r1wHXlwtLMpAei%*d*!Y%juP zC~a?z#;uP8yWF?Ly{zyR!PSHoCt;y|#bh5Ew$99@clCdOhDzws^LaRyS6FR>^S3S) zqi)QAJ>vsWLpo$6Q!^a|Dq-9ja8|NxrBfZMC!jnx{Bj#)LzvlHuQkvIgvIx}uoA}M=7;zUWc>58x{TX#Sv!)RcpV1wJsGr9zQgB(eesGoRJfwZUYuZ;@e^6UUvb zA7R-V*mP8zB*c|uw#?hsOZ5!FVuW<`^3rfl$}vQyiYv}_LSsXP0ZI*0R&R4{@^v8< z8LmuGvaadLc3_HrUgjz7f4Vy}_S*2rGW#X$;_6a)mmtk52?z?t!;P}XRM|jsq$CG7 z!gK5?l~>1bxzUxZ(iYo87ICkLi{v{vQsRC<02vbL~f=3eRuex&UxiY-u$w!{PJZ7 zd#v7fzbNwdPr)PA9K^5l_A&6JWy?F?cT@TA@0GFedT^|#X#84j5gTU1{QNw8SmMs| zr&#;QCl1UcE<+HuGT6nk61F+yzB<1&xa{1X_9yU~=A`pFQ0D-W5h@46GP-<|)=y4s zBm1J9gWQZE^KFBxI>u#-Orc`$+Mk|e0#mTQK08Ou`jx+5!nOeuUFDR<#qxnfj3;`) zB~R4DZyE{jO`x?Ak0uEf9W$`^Rwua7(sk|zm{;crmI5LZg6V-niU;DC!pmZ-(4tRl zN=%q2NizM~-vIXhxtHJRzk;RQp8zs5_%%9ezcOc0*pq@&gOTi)Q0hM>T9!pfsW8#~ zYiq{pTlD^MAv0cihld=$xJiLYI{d=Ia^m?2OH{^^q}Qsxs-0zJ=9ZA8%YdJIMuYKz zh+>WQwBA;Q)!t26nLUdh+M{b+7!dd(gG7M~RMd6>5*i|3s^t%ymCH7Tg!Dj|${p#; z!q*J|70OA$Qo-w>UPql|Ab?b54noxyX0XLm!>F?Y4^fw7C{Q76A|bMpP?95`PBNC`&29Fb|$uuMT@T$f<8R*gP*sEWRZs+bt@1H+^R#sQrJU!f+`mB8WKEJ>xA~G0v-ErSMV#9xV zjmJwFVVR^{k*wn`0a3S}IGhjoZB4t|w7jaKtc)R1ka&C1-g;0@SyfYQ6%dfSF9$;S z)e?^b%ygbyP1%psS6*J<7v{C!7!3Du*6cmf-XOi(Bl${|_Mbl$Kbmnd*MJ^TWtlv3 z%Yc_V$Dcc#sq+qzfBc~=wm9RUF!C82zU?7QVMH&6fO)KL)&(^ZaylsV^$&jUTX=fL z&D8Ok>zAhM|9J!lJyx0YhPocROI(U|kHu5FPIcxO`CtDz|JHp&EtB#*Zr$#yfAuA( zxBGGZF_m_)Usv%S?$Pzo=(!4~eGa0vSFbCmF8kzn-Co0xx;!*@ucP(7hxpGEuzTJ} z)A8f0g_S$pldkgU-V{xDt4va?A7A|kIgHCqnQ`#yBu+Bnj=QX(0(F|-t767xiGnSj+#zqr zCbbonyb1c9sX||JRV^LxWHHRcWt6Y1oZD<^mO`Y52+8fn^!28G9@UhEAmidN`zgn$ ze60@`@v@a;uw+Q)O;*Z0Pv?Hk^;VW#Qo#i>JmwB{aQahNS0Cldr1oA>h|4nXW}7!! zyuw%Ir%{79fqt1pZkx?p*HEgdy@G~NlUj3>{&upc3oBfq_%WAVb!t)8`v^DvrWqZ5 zH6}t9LJXp4XqDrU)$MHSw&saFuO)9uV_D8g4fkvCd*;a;MJUuS zbU@4lZ9ZLTqReoqXEHyjs|bh8_FUp)dEr;mDx70yaKP4_W9jx_HHq6^ysGxr@IlT9 zDvD-d3>qh?)*ItcN&M1c;n4A+sIqe<^AY6>*f-OLrs$dX=+VfNv;MEm{D{90gw!Yx z6}y=-d$d|jpo}t@drsK+`y%``A!gA}KhqCe-#%BSIIT%WF*E-+?x!^HGZI-a^Wnmc z3=@Fj8>Ta5DQ#ZdM#(YAF%U2$1Cx~i&BL;SIfM0Jb1ZfB_|InT5w5}mKkS?M5Kl0O zy8Il(uB4?hisu1NtG%!|u6fTdZYP@ghoA)-YNAV)7K(n)bp2b&ZGRf$CxkzZZLs87|Ls;2!JSiOk`vpa;_BHL0$ycbGwu zxapScdGFGh_U?z-#LETKiI+z;o7wMbcC)<<>YP{RKc5+b@O~?{9FF%S**OkQ zZ*7JyJrQ9fsjN20RD)F@JDb9&-WU1hOMSi9ueJ1zqQ`cuoB~1vBogsMRjWpA&&AZ0 zYq%$ii5HScMQ=R``w2l{Ugp#T*sWz46ge^lmz!B$TT7pK;+@uak@y8CdZ*9DiwnJ@ zlR_cT#*#==6nACrY!jTVITgM#WjAGDh9YauBqJ1s(A1gLmY4uPX>3~?iUNe<)6r>@ z^(GlX*E{^g5kLfUmeTn-U_Q^lnhfm@&sk1Lm2E@is~!jqABA2Dh>rrr50+AiPPH&1 zl-I_OkutVWj7`$FU)d>8JWfK}?nkW%xjE z#Z7Hcdx806<<`&Uz|6$1cIf@n>6D+Z_|$R1Es?}cmru$~r^^|3#_OG2XNj9#⋘N ziHxoXHgEvtm`i8=TMe<()9c&u|gp%zA8F4xP@;c1E>!AqW2LVJYY59?dZApHL-*#YzlBli7#AbKtoUsK2lr zy2#y}9%XYUk1Lyv+T?h64U@|yJrnZOaZ+<_3Cfh_9A-qNm`mwMph``Bt#cA{egh}t z8RL)2UxY^x<@PJy6BCTa$^I56-y(*i9PIPuuZn5VxEgTnHc{@*kRMq(7^IT#P-9hk z#h;E0u^z-zn2N*qL%zhh<0>Yd%$Gq^J}il4qS<4tV2QCeDJWBNn~L&mdWl8F)7UMi zn2-ti(M;^vp489Zz!|$Jt}fMo=lq z)MVrrL~Gti6zDmHGH&^acfDv^Rv`~O*A}FvD5ZNjn$vFdN^+naCy`GW5*`u@bH^hQ z$o=laUHY1KFC)>Vloc!gki$G*-f2_VM2Kq7RG!*E#~|B&A)jUz zmGjVk>BD}}@8~W0h&{~WUYSov@G3DimsI|$!SzZkQ-cION$t z#Z65SKYtp(K#vs8C}#82H*nvdw#22Dh)ad3!VkV@i8WIv4y)_x67AU9#>FML7lFr$ zwRoq!qVYqqzS$gDQBWSX?j_sbG{CJwSKfCpvt@2=f+`r;X7$1I6AMiw!mZDiT{jz) zEgKP(hTTsXrKzUab7FngAz2;C0{LJ^0e1a=^Oi=FSTGQ5E)9ZgDLMTDdhs?V|U@bn*DC5`PQ{WFSGUF9Krd$JmW_23w{$zs|Z@FFXh>H2YsUJu97 zNQs>f92@-A*M*u&^(~dsp(%$(MS`x6xH&MN4iE(Eg}XgS)=L ztZhgx^^XJhcYt5X-NNu@U+?Xu+1wRaj72Dle922cs$pvXLwSjJ#&B|XUFYh^7iRL5 z)3wTI*d=;M+7bP*@qwI%QZ7Q&(d4gPZvS(}%7-BlT14D^wvY4#dr_{K7@9gBkxDpL zpNSA_Yh~*%^Rb>OgS;q z#3J$yI_z9rI|*WI%DiNo9tDr$jEszkz=(V}f!ccNr1Rl;?PJ>Aoj(+RJUtaABQbUR z*7@z`N&5*?3j8#j=YjlHXJ=zm<5&NA4w|4%<&)OE4^$!nsgh4yQy#-&E_N z>Dm?zBHqCFbB~D7eVLV;%aAt_w#$T%OEk?TL)={7rdofi+3e;ds%T7O_c+PgvXkxO zKBi97v??~*^`#`2uih`NC>gwyZ*0fbhE=OulBAk4abL-Cp*`|9S&d>m^EY(_^TbUU zP1F!u4TtoQKYZI=Ntc(JnqkXaF8QQ%?K=iZxTU)g`xpvizo92)NlCOjZ5+9@N@(B+ zHFNrgkMUVNu@WnsmKE`Q9Xya`Z8~PL4!tI*&V71E;s4RGKBF%)(GSQH$-Z zQ+dP8;8OuDqxrf>b$|Tuu@sUHg$Hj%lvs<$orsZ)YfTS`Sq*TA>oH27AAnRn4~FVHN4PaKxo# zv7OlQY*O^4kCYu$$suFmvD)RQTD@bcPhxGU%x50NR@sm2`HQYmhC@~gq9POj zOuy-U0PNuhRIh`IDqDU@$2@RDl?wNf17bp^%>XFeUwe-Ku8-L(*jEGbw)28gt?jI3}EXQn9 zaAbc7lAd}A8=R)`)%Q_X=~6^IP!-X0(B~ zScZLlL27iRbU&Od*n>fIOmw~pfd2Q|r%K{7yp%G)4s%+0d3i+UvyrJ1lS(q~FM2_+ z9*Q{4Vmdbd2(I#R&6eC0rB&SW#RU3fzt)37YwbUj; zldWU!U})A%38Nehk%q0WLdSJRX7@{%Sdrs)y$BTnQH6fU+_7{eM|Qb;aRoX(4kTcG6cv{nE!swb zZBteoprc|31Yj97GK8Rw8Q=2!u_h}IujUBe{1mhmZ@%!>?yt9YGICwrZ#SsEU9Z%x4%(n(fJ>6|IQzL_Xv77bP zHK^sS&S02sL(M_4i!|XGx{VM0o_gOy8q(YJyC*urCxk~Bci;3r=*0y%$AyFt8XJ3^ zw_+ch$^Q0VHz}=G22%mXoXg7V2|hI?I~R}^?eBBb7GLpHnp(R51c^QGOp)`QcYqp_ z7zKa$P7&z;4;MhyZNe0?5|*sWN13IT{^8`729dP3v9L(0s&eFJj*l%o5*^Fn+A5)a z1`a3E?iF-j@m&7>dw#@JW`jWBIzL>`EG_jGhhr@^_|VQo|MQ>%(*vlfJsjWV^h-WG zE+7y?85MXdB3gF|8eTs2??gPN*#1dk@5$=xr-O~utgTTC?tkrx1a1NZM#shso1EC& zuYZ?ycNe^0ZaV9yeTr&7X-kPH5adNbh2c6JV!msCf+BuKg$eO^UiA1Yi;rOqh&1^K zM1{$fXxn}F>@Mhj*c3gvzstCpMExMLuyLHTv`%f6O}NYLu$7=08258#_-3=fUP0*LFy!JJJG-3&2+mjDBkD9pWQj${* z+cj$&oV0Bricf*$L6`4aF7UegSbw~nX@4gmE5lq+WgTdrU+~z!cf0$MRXF2ty7gp* zcJG3=EiYyHkL@2<^5(T)@aw1DhKK&^=FO}NmZ$yNz(iu%quYuzwA7s!e&V7ED(B!7 zYDVn{0nvEX+RzVwwm#!e7WO?3Sv%N@Uo*3-bztZVbM_pmdP38D6$zU$ zZ%q|a%jFGp%?%P>Cfg2HTlqH~@1EX3NH{`oJ% z6vT|R=dEQ{lA@j^tcOR|>9i6MRc9EX3nI1smiL`Rj296RYHzG)+DwD-|TTv&Z2bRPW4oi}TQae@iM9-zM=;$IqrHuht7Gm%I=lT0Bd z^vzxwTU*X(lef*Y;ERfsTn2UbCG7R}G7;Bmi|iLHJS=7M+$kG7mClkc-+MT6V{PDL zuDSwkf8%!>cIU9q^>J*&K~g>!R=sblQAO)gI!l=<$ih+PX2hw}=&3Y_H>P_bN$G>J z%HuNmp2(&e<}$F}5M|@NbS(mJf&zZFx0P~3h;vpWc$+yDD)O>LPsr>2itp$U`9Y#K zIruB*PWiWQX}aSO)K~r6;`GLCySi_F_wQ&On16#scdh(ZMX8R3wQ)1db7af*)Ba7N z6zvwA2UAL$tHJ*H@M@BZW4eOBgZaGLD^u!JLY?pNakJ1e!A61UcHt_r$REFvSwxaFn>;C3Ou zc+FEjS&}?%$2)6u8HJaIQq3x*dyPV&`Y4>o-DdEm)ZLSi07Hk|d%ZN>PqicN`owB0 zpWs(ANsYPb=oz(%aynOO3-*7gIa@*lH#fqX&Wk<7?-pi%2u9A#XrqK6adC0|UTLN+ zm{R4>V#jNYE0Vv2)^&juO>zBX=Z$2^)4h{}% zx#?7as`Rq0hZAO&7a|?AI|#@45#ETBHKCwq%iC?IT3)@VC&@tJhUy^bshy2m;;T8Nz0p+y_G z6p6moLPtf40SOT{k`xx!1mTx`BZshl|6bx00AfsIem>zTl(_wv8q)teDe&n}w(AU+{59f5L+8N1LzDpRy zNAWZp-0>)Ehh&Q}HhPk1>mBwTdrf-?M~szhxF59Wz8qK#AJ}A8Mw>8Ft~9hPe#Q?E z2q7KasaL|-z9W;UPadiU3!wY8}YMnw^8Utg$tj-Y^u2s|#JsOK-WSnE!D&cWfK&3yv3X{Mla;C19o!ej7o zwR0I(%TvPj7S-Z#fA?aaus!zKK3?}^$%D!S%E`mcl}+Mj`?-92bBDd4pq#uX9g?*> zdI@i9g5}RB#C>D)LTm7Op$nLMgv$0%dutAU&}|8=2cH7X4aJ)Y+_N1XuR@o5{(^^b z48t2i=MQ6C9fT{#c22*^pMHOmw15aCYCfG7v~HbTpPv5hW`P@Z2|bb&1pIkV<$Ec4 zQ__7W$X-IW@kr9V;>~saSyFi{=Z{6259RaCe7Eg2YY1W%kHcrP(Pc1X^bvVxQLE;4`hx}lgo;)`0`vhc`txj~mj%&K(h0eV5WLCo< z;hy;BFBY5d$VJg1qXZ-_ue6qnN2;v#!Cc#Mlh$m(M<%>i^(kj9AtzOKw_fjznsr?N z+`Y=(1nL<0<)YL2=k})3nJ&R!3W4pE-LqM7clHgv(Iz(OB>h~`+35_;!bgpEJoYx# zE(!BuERkcZc>S$}J}6eORXS&rn!$-ZmPOn>_5Kn-gFYCb`9=b9E1x zZ7UzeZ$<>-)JV1F>6n=@E9A0Ui`y|nu(}!HPxgWyg_5W?i?`p|mRwOkQFS``Jt)7> z!L4qqwXh9~ebjt4`Zomw5(M>oSFMF0vDssoasHS+5r&81x!6Nq_ZY1XT`l;?M3B0@ zIC9_~bvr+0I}qQ4$4^xevc0o7NR!59nnV4t62ddb%=Zx~LjWU zKD`9D>~e4nVWqwnUTQsKzhgNkBQjaSPwObFPh%7lY3;#gJ#{^; z!okk89p3=EIC72E8owA8JR8|9VRF-2ZA{I{OWK9JF#@_sm6cWHcflcj)WyylGDAMk zq4A2FAR(}gnL$*)39bE65r%7IR{JIg6hD`wl&Nsb;H`u6B}|2BnsEYbl(c?uWg;w+ zTx@$_Y#gR5{unzfEST0yRBgTU?W$4*nNei$NuAR5#>L7~Ij({>07@_AAi5$nUpFE; zT0Y#wpN%3eKW;V~|IQOn3{TOK{lApX8ytQ?iG;O1`=aRGsOWfZ>pjx{;BV?e zT^I#Zih?=J1ygnJu4(uWurords}V_jx5ujmZ8p$ivx}r+B8|0$YjDwqVF5{gLNLctCOo|HlBHMIEj!En( znYb8AOmz6BKze|ZMjaR{$-*c~3aMPNunEl{H-wj$TC~12yHCA9N}yi+%4N-Z?MH!! zGQtyQO;}OAig_s>mjoN1=H=UQ3pM!6km55V;A}-lmY7AAodl$pLV_732P27#Udo|4 zSCoEHnbo&E)70$gY8*-@dQlgRs>Z~xmqA>@!C}UA1w?Qi5niuG3>Y4YfDGb3o&c2JG52<%piki!{%RdseTS42I5XAt#d=>OZFdsL zSyQC|+G*q50hC^KJO zvS%xC-9Vp;o7Q;>MMjFYv5oQ|lc@Ch4B+>g_d3F4_tY*5ExBX z`iGgv%e1sKvWVU<&0XOJ7pJm`r+I%KC`nc4AQY682rzzco-Ia(jjXt>E&JerEZx?P z56W=rD;}4S08?T1qwnWMutCv3T%T;2!fisA>cQ|^SJ6B~0zV>Bkrz8m;O)`d)~CB8 zVb2?0R8-V)C6*L5O-)^G?a6Tb;;d({1w}uPv!IJx7d-dlKPxJvuIJN%5~pWV_5Mo> zx0}sB1bE-hw*Gyw>UD1tEi4TT+2kp&0jtq>rB6K_plA>ja&RbN&A}{o7CUa0yB^AL zE_Ysizt77pe%rWmi{4S;guZq+`AAi;o_c@o=HGy;q8Ixn!|6AZrldRVqjm5RVCr|l z`s1f%w>t(Vbq#Hj&Ul_0vzVqo^iNmM51b__7<6N@iI1cPHBU%y9x73v+xP>Yj)OKL zhf5lf6f7y`lun-)!|-lIr~0O!yZ1h>33z%YG}&;C)pL!>(wx&|O%8;xsI%c0n$Avt z{O~~$z4^F1d{)xc#ia0YTE?N!mN;8VDf@uQJ9pp%9$XgoZBJx&X6fSN+FolqwY{*0 zl@pq)Z}(dB_vOjmVMVv2m%joQ%uIdmA04))PJQr}w)^|3@bEkm3Y`yhC}N2phw)|v zFqSrRj9MI!qX(Q{SZRs7Fk`7O5C7EUr}$Q7(vm(nwmV1pYCvvuVq8Skkz>+zdq-ET z5(&$gq~6t*vs1P`YRh)DSRRsY?D^+iiI7<^n>l=QP<2GXTOjrI`nitjLX^Z$Wfvwm zsK^mmovCk$f2>rPCWy~l@2LQR6eX$f4eA=J=;2y5!EAX8T^ms8M~7U_Zx#^V6m!cy!1^Si5-?y7le!WBPp%D-BE7(q|{jdmOc8^Z3zLu`0 z$9rV6DH<#ATmHc&>$rF~4UUnj6t^DP{ysDamvMA1)rO-4$d<=+PpL&gvLG|K#g zDr&O=5RG^0sJa0!1srSGEq(9lL-|tJZ{f&o3b60#?#R8Lf4s&_VT?;qw{ok}`Mvf~ z&`3$4dwG}k%d4Nu*R);4(;b-7Y+qr8UG@@+^yO=+QNx~&v6hmv9g;#{2V>b+cRy~G zC1{)ZZXF*UitBx>QPY#Z^C$OZlyQ3b*ze;&BBzY+-pTgZ%sl2 zcaZO`-6;jnyjT498<>+LEgc+`qUFL$lr@-3$_DP)n&jvI)nBA5o%&i%gVTDVIxUxC zRnu(F)HFr$;ILr(vl=vE6@Ot8!!>*k*>3cOt7$|W93PbZW1H|}2Yu}>8Wx)--Z!tPI67kBy}IO0 zc1UGka!MyPUmS|KIx<2a)7IA30nckeN^joKHod>lNYx46bVtO&G&e(G0lT$Np^}D@ zDIdkI8$M2H2PWMkMGQw~#iURk^vwIDN|*o_dZr&562H3E89cMSC($m z3D)Zf(aB{7%xc+;GLPc3fo9QwQf@{d%rcOQa;i?0DXvGqI$3*xykrwmJ^A?cN3wRS z7d8THfa*eE6{Q-bI%>a^i$l-zfo7O3-{9gm^sc{1ZH<{`wx?B;*ixU#r6*mtyI`RR&(X*+kauKE&u`{V~U46 zyqs`7#h88-9RK3RGg_($-;|pkK=-vQ{JpBu(MX+wjICnSwkCn=0<5l|>D;>XUR5aq zBE_2rsMVrl2EUOk=&I2cT}BsftO)xi)Tp(8ZM6%As26H=m|&t14=|Rf&7`8!1Cr0c zstfulaXSvnl!zqNv}cH=L(4A;?q-EQRlxRqY9_5i`m~X`MbkbT-Y|SRh_sPDGI08g zVcM7Vq?*|Dx~_Y$iHmz1^LyWE%DQ8r|Hg1;b0L-NwbnjQxy7Vt%#o>$?DFEb+j|=U z+QPTt1<8z(MLwGp#z9-$aZU*ypuSUGk!QOY@Z{{TK+|2`7@zd1az1Q48KD+a1YI*; z=>7KJWtWnQibzI9Tq5j=e=Ueit4P$arymedzmTa~oF+{3;Nsb+%su&8s0# zUiFl@igt|1eqfOspNy2FBYS*&JR&lkU6Tw>g>jZ6ctB%2GIO5Xh=Lg!O$M-?# zIjPnY$K8Ce*el@n&r@g}`cwMz#eg@9Ur9caz@n5IO^ zrg&OLAthoz>7R1NBN<0Mmf)^{QLzHYr2!4-Tp*o{<=f;4%`yq4<0r4 zeZV@sp$IXbbU)b7?%`(@;S?XOI%S)ltg{Z=U%SDiy}yy1y}G15bqF_b`GwRcK45;S<4rjZRbTkQ-}sJmI1$eRCHZsVL;CV!V*s>5Bcs7iALrH7 ztKq5;IzfCjjKE`qnw_C?Y>gOciZGJ{Wx} zV%fPd)_*j}4PA7oWHAX4!2UXk%ebcV!?&4YF4Wvfd{F~Y4uP*emW-RX=ii^?3_hWv zA5fDqf+kQX=HI>6%1W#&`GBO0B2(hN%J1BECR*Ld)(eNU*;!%Xn4MBszlhxf@h*C2 zCoAW8{Q8a{w{4HPTQE(@?+^mQFvVgt5yH26og?rSyvLv75kRnMN|s74k*EHFM;mzlNk_+z*^KH{3QmtR9S-(t>j1 zIZ63N`FeYN)0@jTxUwd^noWm8MR}5wG2GiIO*kO^>(onx?0~LWhW$j31gU+BQ5YLD zyrKj;n@f~S#~Raem5yUNCp5A-DIFx6qf^KInD+ zx(p^-tFp#lO)Czm9dfZ(GS%Ng?nJz(a1~nE98(0;od!i_`|7zMLF{ExG3+mJVkwoc zXe~VCo~CzZm7A6s=wn6=_j^X$MFY7c#lMFht2;n=`gLp*YQ7Iak9n8+P#*X7%rlOx-^1Z3OT5lOLjl;LN{s``pjD`}j2nC+C@O$9|gv zgko#IP7o37A!0nW3YC&d zuqFLZ?SKjp8OUv*sK|;ZaZ~%SLHmUKd@3e%-m598G!>nJ|BWjtVH1mbFX-a~$w_M< z#&VPM&WE*o3AhcW`dka43`WccoZ<<=7y%HSuxybDg^n&8suSse{jwSX30pj8Ig$xS z5@GUGuyfKKZ9&O;`aHsmvD6ZUflL5X4xA>XA$Ds`QC(VeV=0Q+xd}1h+hb(t$!~s;llV{Svb=&03t8y zxcT=jV!r8l8U>)13QCL*43FAsit!CSNxWMT%$C;D+=$Q}^|JfY;Idm!d8ha_$;#=G zT%!2+;y#;hjQw~?Ca#9Kckm)W>o@b-m9@s?9sW}f_9&&s%wO7|!9fQ!eULw^k}b1~ z{G2BUGj-E_DSgmJSdEv=$_cE*ontLZZpy;WjQ*Fl=h*vVx%)9^JWn8!w8^=oy`8cQ z^ry*&UztB^+X{-GYLF`6b7ERN!^WzSOGlF^KwX-Oh~nzqwiFPF%V>p?jVNGE{*$U;S9&Hn5tSxzBI96CxtA8ov#;koEH#{+gAMmq$Ys18%{_-%JeG z+%p}K7Ho1p-~7P_5KZ@>`TN~TN+|m5J4@WHJ;5Z&1G0G}Pss?{!>@yGtRemizT<*^ z>aP}p9=dh-yyIM&3oKv@pZ3x})RZ+%uY=VqyaUMx2fsMA4cp1*8W;kW^EZ|l`2sDo z+RMNmTb0hWn-G(SgDTZP9!wRM|)Eqgnehb5k_! zlm_M?3*Xl%YUwNnfje`H9aj7@kb;Wp&mo7688R+0P1IjL`e z47Hcyhlks=F9_c5TUmfN-I+D$Csn>>2Qlw+{z&lMj8R!1_ZGNdK+tVX#ev=LGWs)Y(jVlz# zRq5KUdRxs!E;EO_+3Bv#p65B2#LCYGMA;K z%0yYa?btn=FG&vK`fOE9 zovGEgUeb_q(+hJFF{i&gYyZ{a%eV~D&VrPeZ_S$R2Ty=dn=K15xU<|V_JwEBY8dP$ zxKTQT>x0oZxb1#?_E*q@{U1hu$Gg4tb~h(W%!hjmk*yG%L{LCU)aRzX;;XC4q5Ub_ z^+u~W>9%z)YZ3@-P(xcA$wWm(RYj>>2S|^ANA~iUWjOM7D~d31=m8n3D)5t6AF>2| zaIaVI37#%X7S1x(0urL5QR386-x!%sJ$BdUhdh}-9WSpj6Gk&Y5Q7Y=TQn;w4OnL! z5T&+gGQd$$?9!sBm#R)?pb@9MFxPrmZ;kMG)L&s5j&uyqL?_^TpIsv>>h#n;Q%nPMz-DA*pUa*2xTmm1*7Fq z%ohI9QI#H|#@J(by!YseFmV`xKB~`UGCHCInoF79t^xJy$d#N>G{ntWQi)Lg05-v~;tWtY)PF zYhO&KfG(Nxc$6|>S$c%N?SvLK&)UfMF+xL=i!h8|LuP-?N+~WQ{eQ&0RahNCwgwsq z?iM__ySoM7xP{>E?k>UI-5o+85Zs-OI|&ZK-QBsJIWu$bnKSRVpV%8ds=K;st*Z4) z<$G|Osw%g?%9Jjcph03qmZq}VwO1tA`<%WDl@-%MSWe!)!qA;pQX#bsnLIO+{DDf@ zGe%z=tT_&@KSKpjHA=#WA|;)O{pqlU$s=2ngO9}GIeriw-x692CH3mQ;>l~GJYVIp z5-pDmcL-&g`-XQv%U9~wMob+9Nz-~3d3{f-XSptlbF35|)pqnmh*u|~wPpH3m96_LWsg{I&r+{a~|aGe7Q~QrZCtzct0W{97WH-#*4@=+&lXA=+FD}kXC)>ckj{lNFyZV{c19<1)qFhZJ(Js z&!>gAy4?5f--y>h@lJD9Prhq^Hg`w$U%iEHsq;31@-zn5x(Fy{Z^=cM-pq=+veUj& zq*VUUlWri|+}LXgr`jD)5BP9Kz6vG~FQUEV&6znA4KGbMvhbseB;iY>6vidn$PD?T z9~p&3wN7`ff=fe!hh#Lss50O~w*q~g(7hWygAu92q>^ALT8>xjEM%r;nZ3WoK>GdL zbymN3qDdu!Iu{Q6;{1fguT4%Ov$^DW#p^`p56XJm^;1 zpqX6(I&*aqi@-V#?tn!6_F-00b(9!(oe(wX6$?lkXslmKUrgc&@eBnf@swjv)AwO;Bc2_IU; zN=X!%IMyAl`I!aVDDrfsypmDOJbfN2U0WZeg~gb)DjE&SEZG1YHe_Dc&Zs-6i`9*9 zy|>=URrs}RLCP?IpoelOh7GGuLy0wsY#`!2kvn>WY2@@8mS+^A1r)+FmKQq`jO`kT zA(blaH_Z>G;0PX7jvw(OMxQGA#h4JaNUO;~W&`kLX;Vrk8&kOX`T2!#sJv96WM9Lg zoN$y>)(6GX3C*EoXqi#s5gXWtc|&5HX;U;?sl+teGX}rNl;b37ia2jDR~7YVu8SB2 z=YG^}*Si=qi?p^Law9ai&)b$gPwsmJL|Qro7Cyd#%~XjMkFA*K zxVYIhzifazsH}`G3YV;+Lnl8EI3%#`dubTX|i->v{hS|bQHze+Z`Afib20! ztsPyoVCg@m+0V?*29+Mbk*1?m6;F)e2nOS1_2b))K}Jv)C{LgE=1kEX{RDkpma9+S ze!(Dx&V;9_3Qy)znPm;-Q>IgVEJY21>D=pz`~5TC=gLZ)DfX;HiWGYQCHbq77R|(z zYw#p&V~j&fI@!AjQ4bwlyO;s8pr;WeEXV+(FT}?_5Lju7ppZlaN=m>>11d!Hor+Va zNPhT|X{BAdEDsmR{Ea1Qm&<6GOU4nLtRjbpi<@6llaQK6bdU;cJRd2a2Z=Q8OX)WDhOaK%1<&OrIrj< zwiFl&=Y6nwu+SN5v6}L~IVLK4P8eC*x)1*7w)y0jBlb$N`a&U%}SToh}ruN%Kq}0;Lvqs6;A~_-}rNn+@`II z55~K7(#zX2|Dzy0g9)NHS>^&FwFupCnUJfn|E>93)}-HVH6AhX-d;gvd-TqzHXug^ zphRWZ$Jbrfa+!c0%$I zzaR-O2r5J(h5$@R*lM;F3iTJhv)Mt4rZGR)3)@{rlcLeBv1#|}@ue7u#biTyZ)m`DX@mnqa zyynv?Y{MJrj^3G9N+L42TYop~5z831>!>;lxhq~94Tf$Qmo2}Gt|_w(lxFBHXw;gk z)RWeqA4nC~&y86~!<_6f%?@PiH|Vpv$aHCRHoH!CBOZ0xJI8h5QVQr6_>rBU>i z+39gSE&e#C4;O?r5;l(CjJ4e80dJX1NQZ z2*t%kp88pZ?O&L+l|Z<+=$4#2O(=Wxy&Mld$@O=wj|66jR_>Pwqwk|W#X(ZIjUGWO zN(9~tza3s8mlyMG_wEg_)2v}`e$!GNdtpBY9iK(#e;o;_u)N&mZEzOGRWYQ;5p+{4{dbt!k|ml>FafRPvsd?oqYK< zZ~NSi>^T_{@uAz+HG8Qx&rzg8AMDyd8%xB~BmHaBHp6pMOl;xETveYnbfhh#$prql z9}X##L@K8C9qZn_daadfW>HSjm_{AUnKUe{cNnWH4&(eE1;=~b!6&0;#3+qMC%oNq z!FqWL59*eUtz*H@qnwQMH6})|@A1BAcrm&zlHT6jCHyQ%pI0pw>&xbu>v`Nc(p%{;>$=4ws1zt)vCJ9E)X=a~f-POLo$76DA1E>W#v_nS|mC%gx($- zoabs==(R_icoS=#$Ydf8@OU(bY4Axc2MF0|w|w<1VpRy6QTxSgv{tF&g5C?HpTE$G z1N>$LPmKnW?mQ4w=i!jV23e3+>`)a2Z#XK{tGZ8|F6`2kKZpD|GoLP$V9;$126B(B zEi(NV;CF7bzc|u-pKc8oxzc3jkvZhIWPmkiI@}dAs$Y`-eOR@_*5%^!egaOj->X3C zSosD&5F`LdUkVP`uZY5x1}Uv0TmL#eSL(o#tL|rkmXwVD))da6olNYm%u+DzYmPFG zfy;{=u8pQD*WWqu{bev-U7=%YuhfK=(}ozl#1 zBpiecX>qVd^!1-N!I9c{F)@Qo8j*J|8KSuhMcqS1$>L5EN-KZWxJr;pI(*PH$!yAy zQ^-((b;*!WG!cmi!M!sh78V+!x(10sA6Gf%egiBCRtOnd4ta7ylse}<1ubpu@~SE` z84mm%AdN!>ed3T8qXlF2q$D8RN(vn3aJ1BrMG4M;cD1`;hgsl|QzD8|#>MkGAg}JZ zoD^RL)%kk*`oI(n$Y^q~%Z;>*v|9sU?po75XOk;&!s4;ot)e$YweRSJ^KdS1DBH)s* z7PBaw4V@h#K5{l!SQh;o4h(ge*@9RI$v8nR^`cAp?3Fu{JIk;(U%M5YxW= z^zfvN(8wKDg3<3L87D!=YARKtgYA6}w-#wVVP#c8eqG8tIuVM;cP=JbsIzXC+TN9k zm@y3@Dh3-ar1_QUD{t4t`23jJK!R*(a9{WG3)fGI$PoaO#|Z?QF3RBqSsG|!^EaM}(>A}o{#k!je1UnpdwXUSMjcIM0@up{jvW!= zWbFiO84SX&7m8D^L2YN9QNJtHozAC7srr(B{hq6>cg4H8xv}xykRW>85Ce>)s)9Gz zrZ@n4=UIQFEs^gvE1JXPs{2)w|C0nDPKf|-s(qQvud0fjEmORD@qZ)rc02n-2QC4K z2|7aDpBqLsYKxOUyZdFQNHT}zdVUS?`GB~$GG6w!VDu-I5m?5BPl#u3E=i^`q=)^X z^hmLrT2Kn^s*G^uUXk<-o>!Jba(B${h@=02igJ0mOU>Gvn9a%Cz4jyP03pXj=Gzf& zRvvo&jf47p!Ky&YuY=rm!et5<69PpIF3bc9c+aonioyH1MAX8(BUQ|=wEA!Tz8ru2 z`0t7ZuRHE-JKutQqX#kT8(!Z3@x4ys(657RImvq7yX$(lDW>xl@sutfYy5Vf=3~F? zSKEEM*D0&KD0Sa&uNNfy&Mdv!_fU`MV>r8ntRya;8zyLpgwe}g0Oa=&!(gd9DAzp^ zbN-C!jXUBnH}$%=fUVmb7($R9WNY4_ze{9# z`B$vLCSftB-;&r>`cjzr+h~fywQ2B0$($8tMwDLrcnR&w$zs}h>)A+cO)=*%AxRb$%$RuKkmUN|tgp4Wg6D*Bu-cj}7Q9F10c^8`%a2GO$u&jDlkm%)?n%9#uEz9R zBE96l@{TzR^GXiY_YiIjDe=x%X+}}Kb56v0y}~W8^-ASTWnXDTRi&FqGE1G%o!qd} z?FrmD`dR1LdMBj#tusJ+ZIgk!UlIlc6C}+Jk>)NE87KH>SC;e0jTEXMK%>mg9rCdWfF(7+DBwC zA`mj?s*57nlTG&6< zjC`D6(JXd4m#GjFo&msG3hnRMO=jn3ms(@_L^8K*e0^}<<4v&D8G7+O!BN-Ihzv6b zhUrt~>h_H^SX;hR@#SCivBHvd3rxm=m0 z#*H!M9gK-YnZ^4^N!KEhTJuWexA%yKMtm@})oh)KQH$)NdrLShwmgd8HyjW9ETR@9PykkB4$t`YGy_@`_9N91 z#VA4KV!aE>l-XdWi6jCUf0JQvM zBN?GIiCIphK7&V^WEFwSRh$||f^w|5HwXTLX*jz$M9L3DO;LT`d>L@ya0M#muOb(A ztXGys1$y7j7z?PM1@KtA9qd`fI&v0LxB2yjAQ6J8L;_S02#ZMi24*mFE2l;~vwny=3U{@q1fasJWR%9Iur z=W{Pc$Uiqw)5t~mEcaoq!C{&4Nc!Z3buuT?@nGfGkIsxc#>?}|IN~CD@KLj-!z`?Z zHu`Qvn7&b>dxV)%(>1o8K7+@_V%k2Q;@014$lvAYdUwz7Q5K}rr)~d&0z;}v;jR=5 zh5;vveh}c+#rKM$M8OIVA-M!KnZj~=zMjLVTjVv7Cv}H=~U;I)Tb$gc@tRLpO zUI_pz2H6d#7C5NLDc78X^}DBsO&p=YV_xS)?{%8-tKH2jTpvJ-zP(=12S@6`O^c;k zyOTH(XGY6guDW8)!uyA!R$QgwOPN^{0eQ6}Z!p@gIkhV(|7T&#N`}(?#cI-Gd7GCG zi$y4jC!!vPALM=~)v6B9&^~2%iB>wXz9$_%FV?8%$5-0GqDMtdlggh%V5o4Ir7@XC zgeC+MD&Ff)z)%E`FK}lkIX(ip{!|!a=CW zByWChttv`3id2aV2>66T#3xFZvKdJW*69L=S+Q5rkE`yc!s#V(+w#p2 zsko}12x+iEBpQGyo70v6AO#Q?gVAV8e#%9Z2As~MZ<8b$G0UX4GtMK??N#Pi{)$@P zOdi9q@(-2HWY7GjttDG{WV(sRG$k%C3q8H%J>(E-rZiKJPQElDZh=)~fZo7(b%7U; zm{;6%gq2q4wx+e)xF++2h?*!L?c4$gblulnsn5Ls*w!~RT#dtaNzkPjwTAz;=AtQ0 zD76HffGsnhfvyfsPR#otKMMy131HEMmiQ_%K&b>kGH5Ah0cLlCG)271^c z^b=Dab&iao>~Ot=F}!n`O&v^_mQKDlai;1D2u6c?w9co{$KVCr0tL>IoGQp;xsX-pGyL=ItVmqS-a;Xp2F69KVY88ln+`vo%lJ`m#(=dIV$hUnAUMM*Xrn}nCj;X}hs zpuS*;x+3s0pJ&uYVbNk36B&l8l zT1r67o`J2hl>9|aS!3UvFS%t`3C99l?*vSYc%TLQ!envysMmfWHaD#QJ^J6EWag*9rK|LQw7`l?G<@!rCrwGk zGt>2bGZ9Bh1VltyCR(Z#IrJz13o^RNL`H!966C@s!IuM25QAg^WR`$?OEA!?KhLv;c z`{xqEP( z>Cly6&4L_4Rdk{;AklQPH&?0A0=boCF7(KiXu!OI>`-T1C0O=Sa6Qb(q~G*C-ZvoH zZx5x~C%{3|CNd`1AbcdIQIej(1a1)$0$APPCkj|T!bf#%H=upt@6+vG*=bT;{2AW3 zM@*6zdl{urU6J*mNvMMyjE2kfzNEX+=KuTN{!Z&Vs<>q_!vf+ymU~k&G&yQgC}G7w z_FzD+phzm>5V}5&B}@QGdQbTe$1${L1~|Zx05J4~1B8Wt>n_1)&$S>0l&#xGOC2xz ziy^ccHCSGBD9Wh|!*IZ$jD$xP6Cd(p2nDsMTx3UoJ$X7A0x%lwnl)U;!)U54H?P;E zq@|=<@c$7h33cR?)l6#f*fKa&Wd6^M!jMAg*+Ybcgus#hMLdEka!T92*=fldA5lFn z=^HgWqnc4g*}c5=*4i9V+Dnx*Dk?vd5Spw#(+#~`#N(-g<$aCUI7~=TMS^7}0DLfv0$!j&qs8wa5dOYTkTy}?zqkPZ1GE5N-h;__ z#2^Ci_(-9c(h%e3qvp53f8Y9=YH+u{`egP8;_lB%qTjk=7CeN9hj1TsF!=K%WU_#g z-_9p@&vWclB{pM}&&aYgRo~37ul2l8g+prPtEm1t4e9?88MLTMGo+S)^_vXse|Ckw z+~1A|l8GU1^Qs+EUr5!_KUO>5DQ3;(K#y3SKnjGE;Mx(-aMfJm727)AVB*@LhK@!h zW3Z=@U>{QnOer`yp;6jaas9JN{t_%eNLzsQKg*_n4dCt`W*#7( z7%?;1ziXP0n{ru;vSF8E`UoDQQkHRQU~@Rv0|v&tE6_duz*!0g*{&P+Z+Gl=`c7#& zOx_&)@<}d~Imx1$&NpD{q0)~{Zl1XRDNJ2#KMyzIvZD*p* zsMEHco;-7f3uYinyP)kBb`>)r6l%&Xiy$pzd z3-bkk{jXPdS9xYBDGABBK491zOHN7(0Ra*4rV?rh*qYZekdvI}nc~8P8j_5%L53Au z;zt&e{QElpGt7eIqAR701w|zu12%$#gL_VEdpxYj;IhRBp5d(oAWQ-Oh${;n?S2x_ zpK36Kf{i|{#|QS1U`Ih~rvAUraer2H#JAyqn>8NV9Xtg5qyKgtlav3u?f><|&$#IS z`fOjjh1mYHFaP=ZY3RST0DSlVcO#L&lXMR<_216F_9VS&jK4eEm@t`sNQ?h-i^Tk$ z`XU9h&O-MF%YQkyKdnjto*-=9dKcB-EAg1Uvv=5&i;AhYG3~Yc3R!~T>5vSAHyJ|j z^^K_1_;rV&LgxPrkM%c-4}afxT+V=lIbxPKhXSu9zWQe~W^FP{H#~i*_YI|V-6U+g zMeb9P6NbUQ9KUQOc@?&XEbf<@RCKpU`Zt32zm*r%(EOjl)vX8YCIF&5@TxOQ9qZXL z)-#gChg6Ui!{oD z%+Ciu?6{qrs+x#B=`G=7Yx8X_pv(kCf|iSdiivGT-q-hihZy=;1Z{F^O2VevVxI`& z!$M6 zEQ^A!CtmeR-dtH>ZwN5F2t)6LzuNvpZw1 zEPyO4o!CDulSpyw#t+!!tgYz*Q_id|166DXGd?wS;9)OM5unk3|87*bC@m{%d@xf2 zIMJUE3cn_BTF(Jnyoi5HTf!0~-9ceK$rE*AW_QW`S&uMKLOGp-qf-H5obIOt9v((O z{y-8w(q@tb9W_Gg>VC-3x#6^H)*6m%v;9T*UeKOPAM}auY)VY5((_a~*AldSIH@mV z`5id|eg?40TS;nPNdu4Tx5e9($yk_B`?p`=2s9ZXEXuCt%Qo9kGPFE0TyA15aS4vU zC<-XPZu?;G4*ut4Klhhs!KI+2ga#$v+Z!}!DPuv)&))x36Ege$-D%0P%yC)*Cm}IW zlmov6s8O5edWQD&e5z{RM5yX~v;utnnLF>v0N4f??nUv~2z8BS-2UY#Q}{c+jdw5} zRbF4&%e;A=Yonl&`k!okvI)A*A#3}IY9ne*-7f&~D`6y8OJGY2ZNDa#12)SrLtVM? zyBSZZK3V_u-@Bv}Z%gBHlky^)aM8zRpT%I=6+4rMY3&Ah2|r}FvZQ%)RhK?(&n7UM z(WNq!3NQvZwwMm8UdQFt+Ir(SUvAa>uW$x@IQNe-#HESs^+yCH;+o^a5T}j|%FS}* z`rc|LCMGr?ey{ksXam&Jif9lf1+I{~UQX-*-2)jbD|+1`z;6`?=9dK~e65!%cUKh( zi`99+voZQ>V=IBa#$F^|L^BqOT75hLvoLy?rWtBRGZJbTWd)&(s`u9!j*#)JL!JVT z45*6U-~)VPEy4pP`lmV`+h8N1UeYs?d`OPlh^MaRM{{u1mBbM(~2zd2HOEC!B6C@c&d&*FA279c`3&7jlL z7nG8@NL8K1HnOk?C+9RCKvGBfAxPA0p*fMBDJOKTHn93-A05Sf7^id6GsV)z`vRWG>yXpy-hgQ`!{_QP zN3k{I4tFyp>orUm_37c;%q5lc_?OOM3TN0jzbys5kBGHh&z?~aE9Mh}f=k`{oll5s z+p~#a+CAUgt2C1u>z$yLhjp1)TCdPC`yL|np=N8c1G?y#8BlZM;2v2ds~dkUi2 z1PIOi(fx6=cnK=-kovlDkRTnSqXv4TLw$J&v9aKR`H@V7T-#O>khtu2s62s z$7Aw8T-VP zSUvZzpB3|pSQ-7-MluMqKSu~Z_inJzt5>c|x2)RPsDAfooSjU_%sMWvvrhkGZ@1T{ zp%XtA`GKBC1&GNjD>?v=wI_j&j;?UUQf&E^J2rXPrnJ#5_++`sxi~U1(xdal)5t5D zyg@u;WJp!fp}0~4+^on4gt>nw03NGypD_wh?}h`gUd=2;CnlZ*3Z5VZh0@B(Xhi1r z__92|OeGp|Nl6&Lt~?+g2nivuU(q)oUxT;7Jr0dM1DumJzGC5uxuQNstaWkP(gR-u z7-W6apjFLl92Bt6aKyE(%|AD%FCt5U&arw^ql}o8q0U$?4TX?jjW@3a!>IC{StI z2g-&&NuJ8ZYV%|Suk8?8OFluiYY0nf_Zt+F70}Q(p|CQ7Wg=uZl2X&R4QG-ud72tS z{|G+1m#$s%)}F9=m!nxN_S9uDjL~XDGtMZ8y|Bd69`p~Eu{KczdF%Uz*s3y9jFw{% z*>2=jpOKia@p_!(OBHc+-txszmL=G-FVt*MHp0v_Q!syEhlG@(j=4SLUZ`It;iN~b z=9M$IW~>Me8mnLcgW;CBstlLIL$Z(6>U+2vD>bEDP6;CgsNA^i4ONYq9S@R*TyU@5 zPPb#`VR&DEZJmGr)M&3GuiGT4rOvmj#LUm#sSvE1k!mZoD;SNd+fbqLr#vAu1*G#W zY?tu}uHQ^0AuSMOgeoX{w9EO+bzcrN-nWj16jReda^> zhEe51H6#)hjCZ&)l6B;^KEy~J=ZH}|9Jp#oJwW8eF(hJ7%-Z%oDk5m65W;J@Nx8$6 zAfQ-=Zm};MrPgyN%Vj{v#N5;%2IXgw9cSmd8eN%OZIY)~Xl0{3)j;!$23p8s-qXUhZ)|JcY{9{C3Z zvqWeQ|Mk0Z>$QA8B`S7)#JJO<62p&08wYAB=Ef=i9=B*{_i+AOLHc=4PH!g`5%k>h{3- zBd`q@U~hA9a6rn-o585vc<#uyteURp32*6WH~?w3sUn-9$CsDJ-9Ex%?_9#xQ5z}H z6p@h2B{$r{gB2d_qWm(GRe#B)Noo2RF*fO3bk0RS(0%L0WoK!W?#P5NYNpJ!7=y6i z12Jd6AfoLa{6aaVI~Hg{ur<(n_Z|b_hsCSi>nE9 zTNfR_5&Dy%FoYI?-tM&UuOifwcV%g{Bbvz`CrGZ(i0x6*^1PcDs!Cetu;9cpq*n_N?@WckC-<-@{TRE9L zg~AbtYiZ);rGcSLwm8cWy<`ocs#flD%acMDLHHzj2*I)LDy_^=n|{p!p6rCxWsRnq z1Z8s9;#8QDkV_OQ#%fZRl>Ce0m?|V_x`@2q#@>j>3S9Z~OLKL|$M;GC z)(|DMTmvO7Rtq1v!Uwu5<9@bP8g88IrY&F0iLMR4nUT$f$@s~u1o`OUQHf|ep!9o~pFQ!JR|n0rhlT_i*wa$e^c>u-|y#=lrNznP_lOa>qD6A)#lb#Th&Eqs(2=i#BFDy1E}*tH?*$TC1)c9jM25f7LnH#Bg2JHJNm_^TciK;#|(B z>6Ns?O4DvsGW(^C(hEf29``0^hlvFYIZ)egzXazA-w4=x;y51J>hBc`a3k2W8zFI2 zZ3djJ4wK)s4GA2LuDrIC$T7PXL3%lSZK!@tsa|_T^9J;`C$VpXpaX@07`IA^mx&UA zHY*HsJx_=~&r7Z*;J?wyj_SZ%crR|UYB&Fy6+iC<(Ou|SVD)%X;Wy}*I=7bh%jFAk zwp``zq(IjduR_P|vX7w~U*Ac!k%wZhF*GCJ*kjpN^A&(_c4t%i?7{xon$bxyCwlo% z-O-Q`blG%K(dI8FuL`?`V5ZZz+SL+~m^?f-l-WBn&eiF<_aoca2p@ak;NJcB`L?_L z*4oWYw%gW^6^EYkV%`q}2L+^a4F2) zc3%U{Fff~mm6;D}`T6fLdw7AVj3S18cH7*YHVWx3H^;tCXt;l*&-D(re|^@(xTL$^ zbU#~v<^^kfj&NlCgTm(Lgm`CHQ*#}9cA0TWG4SYBQ(SCxVY}PvijcPIg;zHDgJQx2 zZ9%X*ToK$~GZ>C{eG`1Z&SNA=A^8+8sRUbQLux)jtD|i+XqW0K%$~HXKW4teAVTRp zkl`b=9Pj}#X*YMoRW)1VbB5r`*uya!f|CH6{T^`q5IN~+FAt|HU0%m7UuFFgU2?@5g`-gR0PLPx5Y)iapKyG*r>}QA8p4)uwISd-}i?6eRmju?rRBt=DdFbO?GqFmfeS?}TFD=$1S zR*o%>C@mDo>pb{0;q}xsjj{{rXcB`p?33`0&hkr#rPL3wo|m~Ccsi?Azu~E-8=@-xSr~;t z%1Jmi(+Z_PxMVM{J~b&M3t)v2V}ssF(eUD$9_Wz>eBt1BHyCFi=kMul9-6XiY;al6 zR+F$RgxdJzhB}NTmlwO%mek1NJ$x!xDi647t9zzdu^HmRv>KvCUH^2I#i;^$notgs zTN^&Q*%k?Ab|(wK{$3_)r)OJ`A3E$cbtk*xvL?(8hH|UFMI*uC&qHCvH=M$u_+i4g zW`M~Vbo>DswZ?T;%ZLiLb}{gK3@b)m!YL;^8-_$m(Y+*@M^r_+FmKA;?Eas{?kZNC zmKM9lGkn81+?Nh9afV?E%N(0E77-2Aa}zA3)Fa8?QWi_K5d*m-7Z+|uVPSuNQ~`69geeBM#keun4~hr-Erz%I613ZMNCF}~u zpp+!W8+2^-)VkerbIn!$;E9c7gN$cSxjE(fa79e1-;d$g7mis6rN{l4T$^Mj)|j}n6G~&Ce`_!if&?Wh z{lk@z&3;V)IP9b!I3yT2=c^C^5rOK9m9@3!am}EGz5QOC|1&wT33|BNb3QCDQ&Car ztu-AT8X8*P!th_pz@i+v=6=ihaQBju`c-u47ba+ASi*=HluEiXAjr*VcJU*Q%>=X& zY&y^8Dub7n?ak%98Rofvs;%jD`XI2nKB%e!PK-hT$qOeD`y8>4F+|kf?lL?7-s_UK zB?-`^&K>=N-L^2A8hEfHtJ!`}AaEv&rQJ5D(Q2>)0vP-Wu{p`(<7lJ(Y08v;> z(~{I%dWe$x@<+8_45HIX`{TCnirb_<9zJ}Ha755@=K*u2#i3YZ8;vVP?RI8KHk+8) z74hlrQpw&$$-LXdu4Lx)gl>~hh-{V4K&TMr4&vnccX3qja;EE4 zDPHFb&ga{5uR_N~9j}35IYZQ>zCioCBtVVHD(qi_V*kp}d3{qs%b-B{63YK@nQz95!J zy3iWCjKgep0j*zfcQvZfYCt8kT;X$iIeLHFVLg4R^NrBkWz~QqAAL)V1xeKKqx_yN z9SC}d;rrNNrF459P|xK;L=gBY+FNXyt)UqXpHke5fIU9eS1f&OSZ2^8x@*GUvuir{ zEP>@EcU+LD-uZRQ{KL8$Pkiy_@Q?Lkuq@BkFU)W0+n-VwMvrBT2cDC7jEC{EIjRFG z+Iopl5U?4FIjB<-zhm;ejLnv3X9wth0;*o614)9kS>k08j$Rzn`cjB~Sz&5*(1KV! zCUpk2L&t`_MqK2uVHa7I#U`dn$Ms|op+tp6Kavu{bLx>0l^Fxg+!9G-+htuf5{gJg z1JKSe5rT#T$E#|?qQ5V(fCtVGSdCsGG6~%C%~Im`8ps_zEKQv=RF$Qj+L1%vOZ4gf z;<{eXQIktuj_mK(T-7U8A2*fp!&lebTu1WZKRV!A*)}gRR;L)S3_x$?Tf&9IZSTwd z^t)X%SwW`9E_E^QcM4#5g70=a#YsFa@5_UM=`c`ku|(6h%#U{sI~_1xfQ`Q-hK04W zRYP~Ih$^XUi#>*&At9p zq1GeTNQD`@`6)xB>W}M2H?-{zB*->kzvvviV!7k-k?tw@99A?~YH2}jR&+{F&X+2?8Nob%(jLq#<(a>is#MAw#DW3Gs+H|9R)JbSpu*JjJ@77sdWxDi-7%Q%L8 zFJF8{`Y{a^X)aIJ12zQV(PDksMmzF~KyIkObu(pephN$>aXwRKC?g|&)R|>*M|D0i zm|VqzZndaZx-BZU1w&+i+?ynv)2@=ympEQ*GDh`hWm33RZx7V<$x(~f(h(zUM;`1q z;v(1M<+bL4=uh)uZ_~yE_FwqWVDY&Hv9?q$LjGS#9-*kP&5u&dA6A+*PM{-96$ zWtROn>tBzZ zII5t@TxEr9tl|p3*kMqE3$i%PMCOP)bG3o%4HM+CVSjJ?!?S(!TZq{@wx9P5{PScW zA=B{W=74r*-`saOhOu>P_vc*b5<8N>#R3D(+OG%0*rA7SL#x_)23H37hjsqO6V{67 z6-|7@!nuNyRv!0*@*H7DrD>!DOKJ*S6(FE!AMv;jMQ2KYe(0?6~HUX`hg5`GIzpDC}Bq zAv<4-!X6%6_omu^q!^O;Yr&jN@p9A^C z47?AIr;jU2yM{VbCg2GoFNXdlSN~1s@$u-VKbj%w$izo}S3Is2^K3r_rROe8eOw(; z2JzvQup&@p6DTR*0R%!ZiRgBAsX1Y36@fAZOu;vPZ4Hx1K@K`5hX{?G5b|OWF?asx z%I{k&pCg;AYt&R_JcPi8;SFl=zTTOpb~H2-BW$h?k`YVd2sPhi z0>YIX?cY#_1||wBsg%ujDFvDAv2iRleUkgNt!FMWZ`#%R&FoiFz1LUyl^YlKyN@{e z=HpkvO!uUyLLToKtJfyo^*n+kk|v8EtK%-4nd6pEU-%}bC38JI8Tni5E(BjN54VIz z1yTKAXNS~@?qB0P=k8(lwyz?T9^wK2Xut!`#MBfW4r>48#1yd1sj6ZEEYUxH{76AT z0nj2bfN(>@$OztX0?x++AO5ZY$;(p;3loD9hbI5Q=zcnB!bAysm@3sDyMVS6DX~~Y zes;n0HXZ8>Up$m;A0P@v?-Q;KaQ|`f0Kx2N8j|gL0@is;f$oR)_I!N6S=bkf8$n75 z!av?@;O`u@j;^bnPmlW@+7nQzA+MkY?PSKZQxKL~JHNm}6_pe;6I5TR)6?}m3(pwU zUkdY;1e5Dfd%VC>ya;1%+em@W9^2Z#D^`z+F0LMme;*%0P;w`cp!uOJErB7f_U2(n zV9h}wEv1x2VRmiPT=<1HU>4hHr1xRv>B#dQpX=%{iuwF_YjO(Q6*7p+f+Q5aOKe&n z>yA5s@8j3Kl|RD~M^Oe}NP0*{3Pcag$XJZo?#yJV;Sg@8`5PP+Y}lKM*WS?z>>2|F zt|#_un%9jOLK5sAoG7#{TVHfh{`dVnw?mQQDut0n62{Tw6r&V8CS!lO33;jC{NrACDR7>BqIQHuC<1Yxnd{p7T8#5eY8*OhXY$ zRDy(CE#l`*a(K6f8G9ZYdHJ3ZDT95e(4}w7wJCb!0eI}C2?r{?O37=bT1!%#GhwY3 z-ww!j&&aXDYb(l$ibzn~$cn%m95MdvQ^@bJ*v(>%Fr+c0kEMW+5j53%U4N4AOe5M_ zk&>C|7APhdEC$rlS0!0igXrjK)KSYpRJZXL#Z>w8(D3v`BXlDRi#1uPOE=MyrUD0x zSRb$xa<@fxXU$n@)73IwG>;!d1jDw}C`=;WFR0P83Ym^{_ z?~4-~-x9?Oh7tI@X=^`dayj!vg1{!Mb;85=sI`IrWS?}`$S9j&No)prnCnc8YbEdg zR=m=(B^EOl!^QCSpwr}FgjNE9CVSCO&0TDgQ%Pf51}7PA@IV*x4~rQc?nE*IyatI7vc)utS1a zDdqQ8oYnM}yZI?VU_5( zvR)P+Job?|geL8^Dp!75pgT|UrCCcP24J2se!QC5ocX#*TuEC*$LkR93g1ZK8@AJNUpZA<`@4x%~`50Y; z>K;`~s^(m4&bj7#9;Xp3E%={-w3Gd(umkUSWVE)g4B_pW%*>fsq5J^a%1#x?Ati+S zD5`}MWMAez?04_lc+{QT-!++=xoCeSYO|Kvw=>{e5EA?Vru)0R*`hq+7MSNa%X;xddPIQD%QGuwkRP$#< zyb&cT3koI;DcSpyrOOKR7k>U2lyHe(4TxFqoVgVE%Geh+6?~Nuur}zjN3(3{ng3E) z`2|>KXG-%B#Lxh3)OEH9X&NbRvOy?Xks!qN_23E>)=F{|3TfAh(A(qRNc@c%1_laY zs}ucl=&T*f=rfWb5(Uz3sFL)=k(ivQDSRX{9toCw@H}{kIEZkO%RBOW+BBwabcv~E z2+Rs{(b$p0Gibqbd`e19Ms2zYu?!z!5|$)66L8^lcy^~4jC7shUuW5as6>^sd4JT; zsrX2+Un1vsI)0)do?ABLs}GL6uz~OFe;3q%iDSaSZedL&nsw4&72>&L*i}KaUGKV1 zi5a*3CEv1yK$vH2TgrquF=OO)pkI@X3mqIG-Osj^k5ywv?G8I|GR?+ik_Z)!!-<+z zOjY*br1Xf&Rd`_Aa+Yw`5fg4wm@TN-mw2X=q|Gs>8vCnTI zb(gCK^+9^^eMv|vTVUWjNd7`LkKIlfiG81W(=5bJ4?`pQb$(v{?HgJ4q>QE}en&?K zsmGcVBu&G#ZIwM!Ddzgor`q6K76eqZ*=nw$w)SAKek6aQGP&G13hjE&x6OIg!O%1w zFx7jY*>g9nprG2D7*bP3L|=&?r{^rZ$hAHk6~go&@fo=x&`zkQI8r+C!&o(=5o3S0 zGj+L=Je2E*v>|ZxfJw#Fe#)l@$dhj9DGBJ>vc)f9rK9q}y0_oLbpN=xCN@0tiGzcl zou)FI0J1N(i8WtUV$vg#R)}F_Brb@iQ-DUem)%4)lf@hyFnFv3(ikj}d** z-gLB@byU|_Y$oTVv=+Wd8H74l}P^77U%2iYZ_#R|oB>sh-b2zqJI0rZey5 zu z-Mk?*yVN2s68Lm+(i}&OEaqlUR#vgtJQ_;sy~MQiEi$kH%Cl5sR6YS|jK96e z&GqIM)8V!t-gR!7Geoc$G1WB2gw9HxF^}|)*tT+*fY9XXr?U_pobr{4U5$mFFTvCc zEUmEE^_{^9();g&u`e#4>6e=hlXqhVZCiF4zr(tc4Q^G;u#=W}Pf$`VN4Cw0r_CGH zZ-Aw23Fq1gGx@S5GE277T#044=!%x~hU%)iX~83)n@ioZ6ATWlz>1ND9;js`6(w_o zxgF*hv=~hCXlFkCVq*arF6ZQ3IWE2LErAYTwlI4~zS)ly`0F8iN@B6?3Hxjf{o-^c zCB6wuKS^`$NOyoLKfklRzM7-UF1o?Xz$YXtMy6|-^6f^3wPD7%I`Ta^CTLGHL>B{pAPghhN6Z$6R-YM-K0Vb_&p!BB~rI!h63OQKQ4>2jT`fs@Iy`#HFuBuzo}DC_3Z zaTk9X;E;qg2%fUg%f%qj>!=ihKs6FeBCo1y0kO($mmA^QRwtJ&@+VUmH3-Sc=~!6e zAj{yc`tME{7rc{eJgdo0r>5l>#p4O&_|Y!j6}EcKBcXUTXr~?B0r8 zT#1tv2%Pte@JEQeVlw0pI`df$SMeiIJsNlVpO@6%gaUgB)NW&kfu6f$K+kXSc6Ild z0*rXF5(3*>#cQu~jB7o%j7?`xh6k6e-w}^Bd;0Pz$io;(gH$G?>u@F8!0G<2 zIjk^+f#zraCwO0ZFB#d{xL{~V4+>=|Oh;oJ)IBeqCO~d@Z%gb0&vMA_UBSN4w%%7n zXK!Fb&ni!n?ZelBoy<*>kyPHxnbdbHhDS!UEr?Y#2$G%QvUxd!sTR54?L?oMIUMFB zU90^w!~JeNpi%gBzhzJ~~ua4PC1cU__uKFA{OIoDq-OoK9N}nx&?Z5c^-o8_d zSyXq$S@_*EpS@~eNT0qH_3S{8sdU8?sFIe}Iz((3k($er*5aaD-`Ab(p6lfA-gIz! zZY9bh#>cSn2uV!*KF}`Vv&&IW>|-$$t;0A{l8ORP@$VziRzU*gKEx(4t4k|HL-i7p zx7Ng-TBOp~OpLB{8VUJww+oY~E&Q{=gTJN#=6hX8ad-7qX1rK~GUEsH%?jO;Nnd^_ z5^(q?r4I}MTYoxi-ICLnRpcjRi4E@oa~e?_C*H;IMbRDB`yKOWVz?Sb#bT;=r%cI# z?GqE1Wz)vPizfjpP#4N<0Qf?KkM@oxKpDBfIo6gQBTgwoaT`Qc5~&hM8%UBM<=Nb z_A87|R+N491E(o@5nK=2nA>=AtcQfw5}s{!!bFICUQj zIA~X`BPyq~d*QByGno_1cuN<-SNO(U)WuClt*IrsoV4Z9J-f7#-uyCldK$+!>*`&1rM-~y0y_uFrM#ZcuRBCYJ2&an* zl^ttOf72{JKsO-`J+gy<`JkBz|LVIU#&!7=HBdbx?T$=I@5NI^ZXc)cib@B74`d;L z}QZl;+z-LsfJ0zX(VnJZBK-jmY@;@`3^=2(^m3I(Bh! zF=T`ZEi0qbtObsS&$k}A`8lof(E9W0@i+#A&DS{|GJh+{;C7d?%i*R#WuAB<=o#dU zMRClg68JHkmcCJfxAZu}8gWzCR|?Heq3i(r-5@2-s+nxlG9#|sZxkbIK}}>ic3`!r z7=nD7xA2Z4uZ;fj5`)JC!wo0;C`PaMrEaa$6fJN{W~H+H>D5a>FzdVPSPTkA>%~bG z315cA`5O|NY88Y7oO8lk+S2NQ#KYI0>`sj!j9p?qOSXO2yRHD^G~m_!)pJw7S=g?; zIL=>h+~K?%F8;*A#g6`<&~UDeBd%ve_bKp#B;Ikn4MC3Q`G}ZnDSxvqsLSDq-e6le zh)eXILr=nZ#Lnl*uZN77c5-SpsJV9~@Hj+brx@l2b;z>&P?LD$i5|}iopz%v{MTA@ zVywPz1`YQ8#;YCnPh%d0!5ACAYdAlyW;A$@hAOv^tktmku<#?BnC%*cfe>+3MKlLn z%N?@dZ)^9(=Z#1E3eLH7eMZ&gaoi+Xky3^Bf6gDaJ$RDxEknOEwD?MvfH*F;iaWXz zYYqQI=(b5HGJB)l0S*b;xGqN~GWnD-iHOVq6# zJ^5yr$2HtB;SYaB74Q|2=PXts%hr*aI7y>QUfW#NNWQ2DMRE%7&S+Wf!$UK$WyOa9 zji~69^4L?boJIrLdJ3t+iGdAl-|2FYMoZd1qyK)hz!2#Jd^;c5@P9jh^Hr_mbeX#8 z@og3q?E0ZlWvxROFv&fVfen&~L`qO2Pa1zehK5PMmVk1Py-9dN|Lz?;ID|L~g(V5l zQCZXMYvKMh!9O?7Z=oW3$gI_6W7Q&)RC=q++&eAHi0#Yk1W2OOs@^V&q|<1o9Uugb z-@p;}SmAPF4R4WLzZWFp$EK>~X8o|#-`{{aOPJ1BBkzup%yf;`f=;m$fJM`X@{oMO z55hH!&aXREba(kBhv^@^qwBZwe7Ef&17Q;=amgr7WC6z85WX@P_m2ELjUm6rv5hu$ zWH#*1pAF_2#Y|^pl3P6c2tA61oJyx~PCc7Ojdgb|gLSqHwycUr)Y5*XMHrDmYL;OK zrLihaZ?I!)LQ6X7RJ%kpFu;O5w?APNhpS7W1~Wn5oQX;>bPTApL{;S4ywY;A6b5<3|%5GF~g#I&Ei8Z|`?TJZzn-nq}%u zZ(k|IXi&WQb}yh8$h(~4w%S}PaFNUjJw&RqbqmG~lajcUYwxAj>oEBxijo;xP-2IT zikh%^WT6>E3eD5Mg}GUf8m4lI*X0#zTZt7U zd;8yb^9$haCM9~jQJzQU8#MrFM0{A!d!J8%6CrG?McwDQH8)Z)z@1~#eQ-~g%%^Ah z^%wLWu;sRrc`lvRnfJ6Y`VIYT?&h?pyhwkjEmjPJNB4EU)A$B%=l12gXkr0dS>(MDL5g%wt<`yx0Z) zMMJ!A%wRZ%i=-z@_OUcMe@nkbqw-a2_^xNbfSS=Gmuwi0&vZY27&gA{Wo!q!GI*bO ztdEO3ieS5gFE04_-CtjjFN6?Q-QR`sa6|S!?sTBHqLfvB1n&RW3H+*iM;HfNiwq-+ z3mJ))m6D`Wyv z9BIXncj(C%7kli4HptcA(9bp-wj=EcDa0g$4u0_ph_DJb{u5F_YcTjdvRxKolnp2@ zHaKtre$35MfF+#~M#Cm$2TlkG?jIJjfthY_0%D{DUIB&rW{B+w1HzB_+AWo8K*HSF z%0knFhu2i?PD0_6k1i9bMWlp8)V7ol;iTaHRyzpLLQwkW8chfZHiR{)>N5cUaSK88 zdsV6&ZTtNnQLqDKd&0axUR-^1<1h?8Coq2cbvOI53F8lZUE1{FuB_jlENCkb0n~DL zvOI`EhM`XJreRRi7f5knYPewxp_tSHJUMU3`e@{EaSz*KvzUbhGdl>y^ZQ}H19Mo6&G-8zsBy^tU{Wa2zFcFrY^g`c%d zfdELTe@Ojxd~E9Ih&jgvfmJ$LX@P93PKRu}2ShI{XhK9(8wmX3>Y4&cyoPL1!9=WY zX_1DAY^cP$K(Py%0@Rh2-R|>MQy2uSaK+ip-Iunw(L9n^>BM*2fYBr6X!S=`n-Z!rxa6fjc&knX{*Sf#$no#kWYwy z&BuD1>bU&96B)STqit0Wmy5FXp%d&~dWhcJZZ8%D%e$YG6fzybES7Z(^=q;0@q-rG z1%gF#bQ4F*d&Yos16m$Pw-`>KO(<*c#Bx=Gf+A>FjFYlf*TC4Y(9-OX6?6{w0u+Cj zv}s>2QYO5?)fGC|IS zHT~Q@Mb$5auT$PccY57weoi@c6F(nXyJRYBRGgfh6U!ReQpBIn^p+OM`nfr+Aa9cIXPOLH+J16Ycr-C*wyVcpHk8(b6~X(;5;oDv#)`N@&U61{w_ zlhx1Q3-d>6k|VhsTxn8NWhTTrU=}OX)TD@eyrt>9B!wvT)Ny(m4BSac^W3T8DeQk& zM5Ro{ITd>HA-f8Wx&}DAX8w)TS-01>KviM;>73xU~8!513j;od`@&x%{OzZN!CyNz|A>1=@FNR zd<8A2l$-U%%{21VI_EG&(>!A(@}kvaU7Vr!`>yI_)5>W%AG>nxQTaCuLMH#fnhTaQ z0g)I(yM}hPvgVeNffKIJh7oyrxcf4hD|bTWs=R$MGrViE9E7IRvt3It?UNsp%X`N< zf0xsf?osnR-oj0#H-dT)*9Qk_eSr(r_?ddz1!}urW%4?f-@E_b(rjx$GG26^m}0>p z>COxeuHkYfTBzc-Wew6Yv~V{GC8qoO2?1{Y6qTO9CT7D6;}0`B$uhufaX-o0K6ZHj zGjW`vCpZk0Jl`Qtw`d)rCm6YsJ+Ukz(A9Mm1!fmcui5A4uo0{4mHlr) zf(_D|-V%U}ujHP4SjeH!4&JM)AP8H=(%RY{gkpdNf*gAiE zJtra}GPbpiIXi=($F<@^sL$WtQM_LG?`|jmB}<`RjNpgN>y#MG$~3;SCutSs8g;uX zZs`Mh3pQ^VB8AXEAp+Grs`9y#Iq|D&i}rT2A0BheF*hwH9~`sL!xkPSJ|-E9zN*E1 zX&%{|qt~&Lk_Q_L8V=kBm663|9vkgIBWBt9$*?G_8zysvL?Tq4nZKRcgaX-&cH|n;5^a&G zt6eUi>GNK0bB6qfaWIqZ)v(=(T2zQt5sWc5dvZ%EbZa_&TkAk2tLV^Ryia2E_~+WC z!X+gRJ~lCGqYKhOHZ}KP_^lO|R)jKi>{!_ggyLq}V?a%3iPsb2b6>MA(aO9m2RR)P zn#mIlEe;j{#(hDv;fK{ri(4MQQy{X|yvsG<5C-++H;pCHOm!mbe1nPAz&@NxUM}?} zlu){adz!$<9jfHf7?B)Y#ksB1>jc~f^9)O=hQMMepM_)%os-QI@F_<7kQhbI^g^OgQxHUS=q5H=x@LxxIB0dFaQus_K2J ziiwTAga9Y~WgNjHAk6Aso1~~))?{X7nfUqTS+|VIiavdWOoWen^rDdvn-Aq8ywDP; z`FAb=PRxcMLEd@+unN-(t*K$I&)qXk*3&aG)Fh;VM&3uv$~bV@B&I9g1N$>sb9wkh zN+eHELiQn3d3ohmw%}l^mqN;#S*U2upfAGHN6eGk9_T-;#7Y_l3DOJFrjyq>mRxM; za~q)i+6JC-CSBGB062=|k{)O{&Fui+IYO)Ap->{_Os3yx7>F)gQ+mIu?VW$`$>(ho zwxNw2QN#s9R$FknYwNKG*Zl6G+?&L|kyDb*?S1_$^u!0Ejn&sSGxp155&}+B;$E5( zB46O3!(NW@7<7cy)uj8OOb`>mqaGWp1%s+El->L zVw3E0PevcEXqH0KAx+Slrg}AVM9M>c`g~?mLND=wd}u=2$}Tc-F6!s2>B%{& zWcTP3!BkTMg_(bl#qxy81onxDjKdm2ZFD0w>dm?Fy%__7a2d$?GBOydFO?5wS16+DSG@TlVHXYoAz(G$A6z{Tr|OoWWdW%?Dl{8!B!3xY~jy~u+6_%l+vD~Jd_A-X?Y zWvm~3SVli31*{r9t^vMUV338POsqm?uKu=9wAkCP&*z8p>I#L*!{Koyo-NJLb@W5M zLMxNEh*??B=e_yI5R9YyO$gO$8U&sJUVD9Ws%qad`TWh#Q9oU4yb|qJAC+Rg z_ZgwvI!wHH_TTg3)+EG0QUl;aEU#{&pT};%#bGwo+VoLgD?7Vb!^c#^?e%j%UT{1` zxq77<#6Sy`=q8-L*4EMKZv@!){&9y``25a$h)hgOxX5sj*6Ib=EG`@9ncGnfb{ZEc z2rd<}u(%xC+S?T;kqs$NzMr>715;3c1*Hwl&`m3%JAzlA7^_t;ZaSKz;>r7`@#d|` zz{r^Fxam?f?^4vcvXqI#q(9k+Vck48D8)08wrcTKgbMtM`;b?R8eDg%is1zIl>{D~ zn^Z6jYBe#jWr#7pzFTB&5K1@pG$mPOuf)rK(1TRQ2Et;AJ9Q3?-W33`40VFryTX}r z$lY658xuB*dYL5pwvM@UviS!qTM+MkpKE^Q;^len0QURX;X!fX`fZWJ2<`jd*}H43 zdR}glzesqneMfmjlID)G2n~XD4R$7ygGxnZ67+*dFINjdAY2(jhW&_)N;Yi4WvSV! ztqr8)@CiYG`+x(SwOsj`D*f|k0QJ_raVC$EM$KSfEzQ~jwz>C}Qy#x)V4P+j#d})@ zF|26#!tYJMv169DwR?2+PZ>T>?0I<+Z1^=Mu+$l8&~_%$5*cO&L6S~pS)N_NG%#3c zDl`yFXVY1M(MN=0k*NT0alD*+as2Sw>DymFOtN*5myNnz+lVeI!a@JrBzmc%#EyiR z#}Ig2dQx=LEIJ3Nyu(>bE(n$mVOXG<4v;mO#K_w_8siWSop8Vwun)5~$Db zaYd=37f}pLH!-qSt^p1^3C*~TwZI2jMoeaM?!m01-Zh(C0ctKY>EKz*R5~>KHyfj& zrla0lDOw(AGNCqKbBjyNVlvlQy#$b*(S9wr{P_V`CI4*%?-m|yk(PX`QNlgdt0eb~ zaN0$mplf*#d%DV`i_YCIbvc!^@Ie0R7JGG{EQb?En~Ys)y6|@=ZD84{Q5VYxTv}q3 zqzGyDl<>MzG|sNrI1yf---scg zupO|nC~bX*OOQ@{qB=jCBRbw%mp(Lw%vpsM)Wd_EDSslDrA7;N3^Z@O|3Ri>VlSZD z{cYg!k(;zG-$*YhZVAAd$-%t-o+DG*o%j@}S)&H?vZuMP!m4t9GK%r zNEHpmNFVT^dc`-JUYbbIn#z}1SbXQ+(Oy*q*m=29#EbKT@#Mo+(aiBJtk7-?TFbt7 zIdxLAGh9NWlelQ$c7`!1S$s$6WR563QVQ1=&eF#q+1~<;|6- z+hE%%V%_Qy_VZO5B_CZz18UC_2v^1-m?*QLtc0fiy+t|i9XmnXxK~oSx z79#-_-s-qsTfif5IaPpJpZ%qfr}OEVNP&mnieqXO06;?w;I$_wA+(@q=a9b`joUa+j0)h%w3v58$EuHn|T z`TysKzeKXKu#gG^5@tEL`*#2jGAV78Kn9+?!|m%wh!3t8w+rD>^6RDOAOEUZ1;M;T z9sEbiLDx?D>0aMdAdndsQG|cH?f(uD{nx+tv41Z9yK8^hNT1>Vhpznlj#`fYxaJ>S za!mc_^uM(WHmC~qKkNDXMxfb$T=P#ITOa;MT;cBA! z{vVSFCCFRZg1^q8>)3Z9e_ZW7s1to7qPsGNmyNrwz4xIw|Mm=(7!bMBGP~5?=kVtl z!_@bA`U+*Ug+k-9Olx(*+1@AU^eYMoqI%(hh3(WoC8ECtXA>XH$cetQM{WCPcCe>` zD|qi*0G?i!&F^3A&Hp2r`v7ubn#$X0sN@zZRrASp7fYu*+>04^I-cite1WGqk2EHldWI3CurA`VV{1!H(USDDJ(M#MAu6ba`$dNrFB|h z%|wNf?0pxOh1eH$c+-_lJfdI~z@*xTk*VLFDu&HOf*cam#Q*sy>jZHiQq!I&lpYFx z%Q1ZGr7)^FR`Gi=d?kQ4MRzGDKiKVkaTGIbT&2a^yK=tmqX> zyZ_MYAYR!2>S*BF6SV*9+aImpZ7-x0PoeiS1lNTDvovkahc)4kq502H{%k%0`2son z{(e*w7=I>KQGy4wCcLuq|Tn*TJxc(;CUfbJGnX*Ydc+54*4op^t_p@+csDKk3%O;IKE z`~bn8V(<1#Xfo=FinERrOnH4A4#}4@EZf^4&DHuG*ppvx*6f&IcV<+>VBr(P|{FsG{68STn3(Zba)M>uCuSdo}%Fmx$h ze?oSJM`8wNb^NP{nS2N;$15^+9JObk)4<4Y4w6vfydNRh zR$eQ57n$gxHSZUY>5!VHMpWRYFA{(KDx|k>{MI@lQz$d94Iz|~)#4CiAnkW>%3K6A zCF#>?)$TGXQ?4lZ8xlv;=_wNN9E^&*0a14<4LW~(t0CMGA;uRL5%4zP4#vc{jDtdMf|ex4v7|H%!}qvo3@ThP=T)glIk$v3wI zPqo5|)b-z8LGzQ1d`)N2g#0Ys*pwbuZ|-ZKb_>MQ3a`{834wN^Ho&O?rOBFFB%h24 zq|#|8YVM>kU>DX24~_izU|=1v(H=j3HLhgKWqD3Q`N_cC_?rUdj{3x61GezVZ^T5y z@y5J5BkzLdc+l76xCMqP6i(aE$9&N)O3N&NMxnOrgL!bbW~)!?I~x@WS6~OJ7Z$`V z^hn1m8#;1~!BGWOf+=5>g-(l0o8oJwp#EbXSYxAuOlXcUpXtf>RLg{4C+mFZec36Z z4hfsjy5cGPz~Bc~4F9OLTB@Fl=ib5E^UmyZVvc^d48J{`mw5s`Lf`V?VbwRfw>ytK zhV&oE%;umbcWs&*S;KnfLwy`TPcl zmk4hu!#6c2$7{+|-a(>o7YN$Ei&^09Fe&AcFsltfxy{s!uWv!aX*78=uDx?lD?22oR$`mN05xZ*fBD zi#2)OO}v~*^de=cI;@?U{iVhV3mqN_4h=b*vwAO!RJb`jW7%l?WlpJzj7|o5Ym$--N3L!gAfG#PXWaDD4k%n&{qdEX{ID(cnNOVQ*`_P#u$Wslgh^s8M z$J@pl4lAHkq39-!p&*1`ak!cJeBj|>p};i!YOlo5P&nikcJ$N8M7?oGr&5KV=YtgJ zYOJ=2I;(+$Z8Ie62QQ(@w5uB$a>2?aZhVx z{p?d-ThJUNK-yVQoX8XGzbTUh5DixC=>WA&Y~HZYJsubUlTK6)yG;^9H29^!xZzal z9zpaiKAt!fhHS55nWpy19R&!0li$m{9o=-kvB+VPT!>{gfOE4#%qL??Dk(LT|Kfgz zVdHf6*W=80^l43_N{*C8t zw~({j^a3&M+hBVB3$oJ35o;dmQZ`yNmD==X4YFU-_YUBKoXvHo0(00;4G2Zt#?)l3j z##N6b8DV?(@hrymF2)PidCxNe-_k5NQlsh9Ig!!+3H4W}fYBrV1+ws&0Uh!t_{T%= ziYI5|03-S)k3uvYimGqk96ukIdraV`fS0j3A$PA}>z4F`AD)`Igfyl!B?ItWV-xZD zCe9|YQ-itPv=}d!=;-A9a(Yl#$K{?oN1qi^?E&6dqul9%*(1N~Pzt-@ZCrMN8986- z4+-TJuVrScZJnP`^0%sYh-Z6t*L*8t1J|EN^Myu7HBel+$|^>}_<7bswZ_?bbNhc2 z?ceU+Ii%;~_03By)aw|slQ>TlY_H^mom5s4 zx#Q9xAn3SWApgN+Q=J)1x;7WPlCh1Pewqs4jP~@3Tbq=VMRZ*5et=b-&@U;qi%~?S z>Dy}P`V~(MQVC=Ac@kP(&A$2Brp?6kBop5xcq;3io#SJYrE7-}PaCzEEL)scULrvON>>tpexH$C zRCw6gIA*q=s12ysD{FXjnp|Z%Lnb4)1by!UD3yh$BDg+F-)*tjdNoYs36|aH~|x-51h9D3jAQCq0sUuNN!l$<-Mb3y{M30XW|E9f}aW zsAm?vDj?~tcyk-Lw3T$Qz*QCR;OKt*ws#?ik}v8McAC4IVW?MpEiBq+XE?=?SvTXp z4JdRK^e(2_@WBmqOv2BXcqdYG2{#ZzEuI3+i3zS7-0x99(>QzqLBNw zNqkcIIw?h*jr2V>mm0{DGw8@p-I>&o!Z7bCow7|scwLN}jz-o4Gi|5NbYB0tvc4&I z@RF1mk3b@79Jiod#;G_W^@N^0+K8bqDnX%@BYUzQ_%2kIGL6nMivfe~pYkH+)fAMm(LR5q7d{Kr9{m(mC28twsSodS}J=oD|z@66hmuX&t_s!-YMV zsgNDyfsiGYu2_BJo&%n<$5vPxQ?g(59cjo7_YcC&FZZfW)*WImnVeETBx9PT3kI&O z_n}K$(c?(Fa(fsIT}V8>jZaUKeZ;1rNfEik-PP04Y-8oPB0Jyjn_zXh`ji&eBv>MN zMvLDF(oin|y&Fb>pjm5pP!ie|miLlTuo}T$WWp~NFSxP=e_Zi?ihKmU3$7^jEM}Xa zJlfDlmGVTwZ%~7h>{wbE;(#FxIR4x)`+yeP-_k9E^TNK;^%RNFA;X89N^Qv$5V(UZ zJ;ULYOXr@Oz7=?Ak|ENpF2rh7D2%b_1d7{{4w+kXePCm%8V6XEYgc$e)^JeIHxv!M zQG*u~)qgpn_I8cM9nYXM)LLu0Yv*KxZAb=O`|43qqYpE~0t+LX@ z%nMU?kVQr2LIf5|yLl#uL){coiDS!RkHrVw8?w3qi??hq#DjY_xMe(SDbedtC=jIm ze?kC|wPVJAIWX9==ey|l4v#}L!Zm&-cj9**9xgi# z)0}UWP2GPi{A2AYxN!7Cg7r|>pUC|#j43EJruD&&o04grJ9Tl?zUP%x0_{6s>{a)Mh z$%Z}R56XLA818x6mfcqU2Jjt>-vywPMV=C6*ashDA!~y?lP!ya zH#8x;(3qNGGW2t7uD=CT!!BUG%cr&(BXvdxg* z9|nw_QB5xmDi+8KQ(hgu@%#i;{;~f~h~?TAIaR=%Ql0@;+8`#-IHNrl`!o1`!~u4> zT4AL!z&{{;m!GA_k|a2i>rYn}Uo5o+xp&ZGDJ*jhy9r$#jdWi=HC6^qSF@zQB5toH za9e$hm4mle;cIC;Pkx|=^-Q%F=EnfVn&m!s=7EpHw`yU5`@9u+OiOVnW$z3!oxk;e zM!)?!jz|8efES;xzB7;+eIn(>&*qiTZm7Xj?Q-^0rw40}qn-&Z%MC*U*|Bmydnqs8 z5tsNsB1v1^Mt`}hya`02Q@sz5&+QkR{?YL-%fU)4we>x%6F-;hE@XROqkW2;WA5LY z{H0{Y3MJ*^D1Otu5m4-Sq;E{!ug}K4d%##5k3Um?>4%}%zZ{j;Oh5FzQ1#FCznnfl zMB(HYG6*@_IXmx&W_=lz1pzkq&P=rxMCLt~)ul_d$57rj3JVDeje>Zh&vZvBG8E`5{HaL;P`!8gxFYvnNE1IzXZ?HfbPap{ zWaBRE(XZoDX}k;_ppjdbm`K~yO(wnPuf%fgI&a_(z82ARb<+4DWEGgB?PGCERXRS5 z+SX4+TqbVP__|B1c#i}&WMGDYIg=q}Tg%8WiXx^;lF|r>KQ%R4=;?x>qduox*Fnvp z(}*ymEK;-k3_6kUo3GXHOqZN|JrmA!i}C~uU$eZ+Ym}LlK`|kT#x@c|&)}0C=+#i@ z)ng+3My90(l@;Hp=Cm!O+1<> zO93k~s8@Q!LW{qQ73(lHbgX-K?f;z%&@rSkxzDfA!KGlPkP6TG^T%qe4)A^QXaW=N zl7XI)y(b#$hIAJsiQu0>!+_+!oIe_(tcLqn>bLMm@XIuU=t!M^t;%G>Ojx|*v-s<} z{|nL?d7NYBwuR9NX6T5l<)f_ZGRY2dQ~r7Hn_CJdzz}Sgw{2mbQT5fX5>*#2*WqyV%ySwsD_N)tQ zXC3dL+&6mMqvJpre%B^Uyp(Op3jNQ?b(JScLiHh;U?&=1a1H4PYW2Ld4VNf?xCHb_ zA#!qvo&Tn5BCz-+7yF~v8zc2nXDV-{Po1SUj;-5)w}wg-!lB3}!Z}@eY6h^)l8M+5 zezwA3Nb^kFFK%@V#EB+ebVn)=TbkR9Dv}W*HuEzNo7?4i*8Kx4L~LtrvT%q*N#4!T zzBlGmCgmL2NE9xiR@*5J)euH4ahb^Tk3#FEn653R`F3)COj+Xjhv*$7AKFQ6l7rtR z?Uo;`gbHijN~uY3G2dU zbSydM>*tbIFls>jn7P%>!MjJIJb{LgQm#vFsHQ1JOS1yjevo@od0SJ)*x=10QaIP7 zHI7|qsTx6+Lej1ac z#O!_*n%)hC9Q=8QmlA&+`L8$S{O%`&4z#uUG>~MUAHR_W(UfWPin`jOKXLV;c*bIL z&S|ud8sE8&fZ`5UPO6z3>zh?7?SHJSE30}48jlw=@=x!1C9ijR;Ix7Q;g?Y?7-hni z*VFr8*JLt7mmzEa6n`q&mbhCN8v*7#6_rpmxxzOdEjzquEKcLcIa%5zZk8#pC2*fI zJe^6)E$-i@$$ey(KNEuT-Zq}k6mKQ=3%;sM?zH zceI}C;yzvbwse}gyUcH4u5#{GE-ai&+HVvDMWrJ}72-Vgxpa)bhxF`Qw2@R`B9gK( z(u1!Em!|BcV{C&9?GC$h9)q-T^!?Hhn6AEtGO<2<4@A1EA!fWZ{eJm@5NVYf7ipKX zf2urkm;*Ot@S&RyVh9T5QJ?b}fiB>X5#jz2)=GZhQQ4rNCS{9^S2BW?3<<)IY(N4! zBsekzAT2%OHbkPn%4HNR4vbpJ(_w$PLQeJxh9hcRCEyiC4 zp&K$N1#U(MO41S0CpI!Ne4sO0DM?03IwEC(C-+Z~6Nieo+FIdHiiADNnS5WyD@RK@ z#KGyR&-XnMA7hc(26_%^P?C+buM7=+oUQ0~ zYL!2Th_4`iB)l^w7Gs$N2}PC{ zgw#`rW5WLHV{AekAj*G#44Lr%b#CZU z7GE@Wl!BVGJyQr1;tQYYiqE90sG*PhPfNCKz!HD8DrA$F_5J529ej=pPhrGGsARQS zt^-y^Bd)X|_x_uQ2wXgzTay2%LhxTzy!3c7Af|uBlxYtmBoCU!hJ60rMM#l#!Z`ob z^Y;l|=6@IR?*_AtO8xy@{=OwIzSocA7A5{5-bTh!`bpfz$mJl>lL+9_ydx{p+M> z08o}9-)!|>&mK5@vtpHkzh~({QCRIKkT=98oLh6U{3s+VxXb6$EoMxUeb$w0hE`ev zej-xL#7)=Qnnri|b!!2-tcl)x{KF8z<2aX_&Hq+KCGh`IQ-oEhCGcCOx4nVNXZe(X z#ZvF{>BmP!f=rL+`*wdb=B8+cOilg4n_ShVYcUeW)%~by2_BIbW=(vI`rS(kmy;bt zeguF&k|OW=cukvdOztb|i9qTIMZKKr(D%;n@f%pAaG??n#zuYw2R5l03uEG~y{3{R zd1rLOS4^zl$lU<&RlsgQHhrd2Vv{~ggU>tUAOj8>`iG-<0eK9J#9cCo2C9kmSr`+t zxU)!pNbPt37E=7qxsvwCmVEF=4Kr%1A$dqchjej>fPvCzKQp5Ie?QP#`8+j-8yFGR zImO$gSRv4}LReh)Sm6}v{9F|yYM=Ix@wMUTiktU*H^0&2;zP8%+}WobNlsy$fFd8d zFL-)dYO{(3{rcl1*S|RPLzu{xZ(vU?o+^MXaUG%XM?rMtk5j%6QzvEb51i$fUmUIu zH}xdDuSpw%h}PWSAUor_^X6?9oc~-cgLZvyVwH?)*T*n=yy>G4SWHf1Nwhb(!b$9P zze_FZc*$^x1$dI2?Hh`8rfH+BJ<4KW61B_OY!XTCoy?KS;W0N|1Hb}6yY-`~T`8;v!u%BW9LpwxxMWZ9iY(*oP0n z1}us|p|h5)5Sg!zUgJ({%YCPy3pL| z0&(R1J>HDmvl_HEwZ$BZV^e{~w$>dJ6*WcQ4FiO2s6ySkS=srbaK5uGc;)=e!uyG< zss2N4XgFDRG=17^B*~<667E4}ik*!;iW1lb4-fy4Ec*X2_ZC2HZd?CwTMiV8yE{cn zu;5<2v`B##cXtRHyc7u#oZ{A^rMMJ#_monsXb4h@6I}l}=iYPYw)g#J{@**_H}5-> zXC{OvPxfAW?Y-7-=>>6)?@x|U4lP5&(bCHH;kW46zwmH{51H0`;LqZ-HR|h0p3~|z z{M&cp^mePZs8CixBTW*gE;OCE$@RMC*PguAA~K6uGc zD&gaqA@5uBVN*Y%q_;P76p7827~DD+E1ChY>FT|xIpejt>ik$+ja0^yyvYS`yoy~W zYXou8r{ot`X2w5#GGD6K+t=uhHo99l};(QPuZ53#~BPe=>Y82W>ZcTOeM!4Vj5;-`EjOoh+T#*^_! z_Qj-$)DfV^A#7)E8~u*~##gxGClau^B?|6+Ub)fsJu+?W?l4-&kBocs6q{>9gnPzS z;o%Xf?&|brI0toS{278JCROo2(&bb{2%hAOjWzI?t?den^eb6Cyt3WTGM)U2D-Y$* zTAUfS#mG}}GuAo-N_wEY6Owr3NH`6 zVRysK(m)5h@w0i?*yfEU`NU!$F+s&x&OvYM3Fjo|b~% zF0HxJW$7<$o(zr_QHCnwgBTJ*MtN`A$Pc-epSJI^@8;vl8HKR4jc`QJ7FV|5Ka@$c z?=p>l$=>@4GfY-MK;Un+>7P22$v@;*Wr6BgV&N+Y7fy&bWib0Bld+=<){E)I(S;0N z)_qF#Ia8_zX~7qg^%VnM}kY(b0@52Vwp)lPT)6E{z@*s?bRO8ld; zw%F!sQrpNct7(_a_&wkh*<1G}Y$Kip$3<2kuC1@|)XrG>9!0TRWFg7*DbxzD;?VL>8tH7V6JC>lVM+&>=HpOI4*jOZR}x zBX?;4UIALDmgr)Fc3}^Qdga9lyycy$cV2w1s*3x!GWCyBOyz!lYwNsa$Jd9W-l~3A0<=aArLObpdBQFBnkQdsMHJT2z&)U$v;T2V`2EV}*9o z$Bz~VBq;o;xV@y}cK_q3?Ec(m9@QAD%I(1X`zdPT{9h=1z~6O?Kg<#%e_buqe3Ddt z9SrKUO0)chga7=oZkGQ~bZ7F<4NaLd{>C1Lsza8EsAf2*Hg8i?Q-9uq ziU{1@-5t#mO)5p5L%b3aqo}ZQPfyRk{`VhW=c?ES0)et-W{ib}g{aWxDOBTFRAWOu zG99Ru-X9?{&CSge^z_AzjcKJ_zhjZc3{-n})WA^T?QiTd-le1vLHio@0Yz^}*$c{8 zq9L$uRw0Q&`F_P;Cko#lt#@iQ=IOpdWz9Aj#+LS(KLWJg!Om~L+xaK%$kPW{=yl39 z7DON31dZI?lqUIycBF-W*Yu}L!gr(1^l(gD3I-ijjF9*7szfZF#!zhZ!l_V%J?KGt2+qW{kjd#pM#9Z;Th(9q$*O$@^^N4H1z0?Yz^xZ zBf654c@Wjo&Dl@=-@tWqynO)O3G6vwHu2nx$1Yj4>riuWKw0CyNtL-HThQIw$T7>g z>?;AD)da|X(dJ>tDm`B8K?{0%+ocfQEWzsCmC0QOG#2^`?_$0^2&+zefau9*sVM|s zfP?K`G*y4QkJB26&jsnfFj_E7ti7tQzWG9sN)Zlus?6W85P(hXVm#JVJHNFFg>Sgi z1n$w>jo<==jnWm6$Vq|yIx75&sweZ=iuUf7C1=toR z%-8DM&$oVLy!W@3B+P}tkuKG{7dp0a)PPwJY3P74f_z?o_PDbBg=u55rHo{Z6*|S7 z#?sOSP9or)x!de^*F;)@d;g$>f=-rnY@zXjx-3g0 z+-#QRa=d0YJv(b_?2vnM~`;Ow4-NYVCvr}}-^Wa}#H^kQAY_>zfR z7y0{|`dGswkt^e+SflCD@7ood{o`sm;1+Zj26>rIs2v!SB!#c#%f^r307vN`K!+_w zA%BF}xaSGqn!%uxfishUYnG9V$IIEqh~yU=8e-MCux=B9Y!}C-T&upG)|1Rl$}fexDO((};&weg*cr8i`*2;MtfVx?Ij` z_zH(&4RJ%`kI*xg6bdJF>M@Xp2zr`bE1mNSEA2Oh>@3_q382Ies5Ymfcgka;=o1c- z{Cq){;}4=%5+l|v_T_y*>Ytwh>&$C)<3?`dEcvdAqM%F+bf_c{Kd9zJECp8jKE}Ji z7ksFr?ru4Yfgt&I?JzxRpmy?78vaJXEiV^JuJ$FUDy%EGmZHZ8r}I}{2(7F$JiC9J zo4aGmgs`~9^>&?aZX(geyz?j1VxY}rpo`-jdaOG&n2}kGaDB|${w6M`)=nyRTV&;} zu>RQFrm5&0|7;V96V_sJMc2rqA;Yhcp#G!Qk(gqTgt1Co>WiZ}fygxqynRYG4zton2ixyDl9S7Vypo@zn9|T2`kD%$XHgVJUpr^w69DI3k&~R?W z(NL|X*+ZUweEZ1_|E3MC0O+%XFx>=0uqobjUE!dixf*7UsIdCrpv~70u_?_WNA5-s zbv9~Nn;#2IsaY)EEZdw$7he>!o$*@#s_b;@WLO}YxmQ*ig({#27rH9J{h4$delt_9 zR(L~)W*89jS;GxReT|{suDkxAz)PCk!Oti^Dp5zec6Z8&x2v3=IE?PastBH9W12qk zcbWtBOc|?Sh)`gVub3hhS9y`SerUxhg>_?mjos@%Mxp*`kl{;D|0}qd7s|hS;B_S; z<-%yp-*^5p_ElZnI~QE{C(MRFC89-Lv5PRrC;qs>945=}JkX${TUI)u=xvrRtraJ7-lt&!0Pb$SeG)w8?Ws@|$O2iEW0Q!1Qps zd{kF2rCJ@A%?a(91sSdB=o71@BT|y+TzKae-|!6D(i3N&bH-t{6T&Q^Tv9=dEBa-n zhWWt_+yqZXsZq1cgpGMR;cv}Jj2V0 zy_{|JAf1`V@{M(}IqSuxh{Ra|N=e8I<-mdB^fqAI&{wQ+nKU-8BC1aY{Ae=2o%6~$ zQ_X{vV{!R_2`u|-6!dhx#OiR@$QM$Xrk1v$mfY%PSAVEW6PhB6Jj>eKGkQDk8F4NC z@xoDR)T#`@!XwcVf@-8^d1?s#_Cqi5WSh3GPjf9BrQ4sl6A3S)`pOHNCfbx)2ulYB z3e$o@o}ogZ+$)i%&jcNGV7+y0;O!y$TI4K)sN%&{t`GQl9Jfq=IT|i@OuN`$%d=+z zi2LQ>#xxORCSP7HUu?0Z2VFQ>ERxU81O%T^L{FK(LbfvwGcZg-;nTviwO3ZY{YFwb zUGfN3iB>NRy11^-<-h{VJkL)-2N_iCw<`w$2xUH+gRnMgRJwFJaOW$p0t~rs* zoO}E~;F%&>3A%xLr$^Y>zv$QPt|-&|-hwD9`)RCLu`c<#MvphBwE0Ped0B|}5 znJJOSB7TE4yi-v&%l-6sKKvG25~&5S-Ie7VL}@{d5dZYTurSI!`4d5F{KSTuT zf48OlX-@f1{rVhO`G}w{r&wEl$2ebM(yJ$Oc)XpQlQC!>NkplA^%P&)cep< z4x&iY3|3T)I2?R@i+Ld(C_*Gl)IGekix&ql!bE|isihS%vmNX=2lzV{VBP*tbotAx zg7vRDIr2<|p$>0AVk}Bx;H%`B*?YqvzDK z;5%mj?QvXzcfJ=ax6dT0_x$6P8l<}0yqAW{8(iS<@@05}%TnLl=h7YO@YSY;yFs+k z5}x&>&|&*o4;UHQYch+2xn!P>7#C?*;>n9Y__eD~#Fxzwc|=mm7sKM`JuZt~WlX;h zNZp$6b?ak5yEAd7w$Q3@Bd}*=^mXPeP3}r$>M4H~+?(BWYVoWVR5Y?=r0!bhnO3&V z_)0X@V81a^d_Ri+w1MZ#mg^W+ICh%0UU&S!(_eTKWeCx%Nh882JQ*u9mWqp*Sz}(1 zh-@drtFzXvGry12t}NVIQ&7~$09~-(Wl=Z!1QHepBJo>~wXuCt&v1^B@(jHpFQVAF zA$y0i*5z}fZNh-O&IfVXYqJjOht~$15rvxvyB+>-q9&4}$;Urbutf6*1{;av6Bt$H ztg9x27qL$F!5Pat9i7)ocjV5GVt`qXG$&t1rBiSI*oGl98(@cw;W8iX$*7%|Q1wTF zDrl>VY_m=7Y&MrKnywO*?AW>9U-|_--yI3+;aPfKdl0VbXgm?U3>;*;L?KZxWAX3L z`wuiE+#lmFHbh=V$e6FtKoY|!3eFt?Os|oRcLv^*x;q2Bxc%bzhohBVUw|~5MEtt_ zKU?l1#y(G-<7Q%C;R=scj7Tk9?PCGUQC}ZDvw#`79x@`Vs z_7zlESaJT4?1($b5b_}ac9Crn*;JG)YdZ1T^RcC}iKSob+dQ4Dy4J2}0SOj!dm*YHeRO)l;_fKTBYDQ! zr#hnRjwAzC2U)t%?<2vSO}6H86W3i>>S;id|_iAcx;{eQX0WN9%gY$hhxg?03kZsK-20*_>pc zshlIVv)SH~LLlA@oq@#?ebO83pK7Se^L}FlX&L-g`G0$GV!=lWfIbEB$q=*Zo50!B zTNNz3xE9(#q94vFbAnd&@akgto%*-{Gy#V+V$(=rdj^ev@_~L7`~DU#hEu<1F+V;-2@7%Qic#Tg9yF-jTa~X=4Qfug zz}H5cJiMO@-q$l;tj^AtRBkPo`{Q}XNL1em4FzxGs6ktr)g+hL(w-?x`T%HPYj5uZ zXwjccXLs#nTqv;#{kS0~p9f(xmPfgExm4uGg65Oj73-|HIchpL>~riT^pZTX`X-g< z2n8^7H;2g85%pA3X!MhtsNzfZ!y|Y_Q4sIr)0F@ZYRU-7`sj?_BSvzIZ!fek3PH zGk@X$tcITJ$GYc9tL}a>@cB`J`$`GG@Pd15Ma)np(NZh~8fkYQRl*4I_y;nc2Y@Of z1N10MZXEH;;!`{6ZEO;iWf>MX{5nK|@L5Xsrl~0*Nq<6bbxMU3OM}X2sNC}f zI4+WB1*oqJJMMBTlP53Fr(8RH8TSVPLlw68 zSbY9(>7D2cj2gEH6xYhtuqTEi{E(%MeF_Sym@1vQ}F>jPUm&&}rzgU9-oG<2s2 z>*ma2vm+30=`yxmwAbrxwQ~(F_!Zbi18X4?MZt*1Mifprl9KGX7ji~{Q8dsyR(L=4 ztciPWLa6>`$7MD^Xv_uG{n$J)B$Ix#v+^#o_$-$hade-MtTKqA=h&30l?O7COhTfg zsYPhLVu+34zRQLpEI~dHGjl$a0hdAqRsIm0A{>j=6v^Flw6eC^cmeXd-o>9YnH)_l z#J2rZy3zlzV_-Cj?R>q0x!3JC1<-#tO(ha=b5m@$ozdC#zQ7FKfUY^)=QXK7WpkLH zvw^zmw^d2J>TxnPT%5$s;cysDp)j>sSCr=~a^I#=>g@jJ_1c3?#UE00s{Q~uJ)~ij z$`XRZ=J1#u)qt6`|2!Iyx4ur2^00|TSO1x30AmmNcUKz7NEVc~_rbbKrCmX1kGRUC zV!>*7N@lta*;(mJQb#4!*@ z>XYU4qxOr%0bOIW`xuWuFt7>@p~4MV`=4DER~K-1`l)os)Upo`P(;PVqhG^;!`s~y zZ&12LM^%*L-95EAfO8Y*@x+>)vCkgpc8@-2cu6WLBSZGlgWoB{rKgh6Yf8=2(Ucn= zMg8&7)yIs_Q4epc?z8F3`1Gy<#YMZ}H-5I%Dl13g(K(2|Y$dSch+@5zP>fcwKBc#; zz^K7s>*a&fyX)dCJ9^6>v&t!pwK?T+B(8be ze9~%r2QwIY`;vY>)-%4r#$B)Cj@NJAWu=IEzneOzzt1v$4ob#9In%XFk1+tMmQ)xu zX^&oNICKAbfWG0+P9oOlN&eK{0*;EB}L&W7)9j`IpEb$SMY*`OJvqW(qX`5z-6CyR=y#Uwua z4SyYcUPIM4k~V$P=wmL)Wf*ahu4kaeK)|Aja@+62iDrfpD7FwWPSWruNheDhsqhh9 zn4Vfa4sp3%L6aSZrywafDSO9v6=W>AsDziP>w;}_Nt?0B_1;bqH*A`sQu0L0i*BEJ zU$G?jwSw2m12+AKO$7yk+=X*=YPk*;5)K47sagi>o7Y>lp_w{b7WiMpH-Hw}#9eC& z#=fU|c=a)+7cV|m#(f3t0RlqgJaVMv7KCH;YBN5Ta`(_l^znBrjW-3OBKz(KYg!A8 zy$?0^y0*rb#Ny%4w3*a3Q4J+Ll|YKLjN>J!$GG15v)v9`Z;YkOqBa>`HJx<5`M zI>4T^6KOntYWdN{Iqy>)0W0^>DzUm7m?0*Ka(#3H`Nz)34c@vb* zk7k=aIRym;Rl*#Fpxe!VJh=ah&I$i+GcSZHRP*-jgQ6%6W8*zwAx=~gkLA>n&nr^$ z6=YhYS57ZjnjCL;y|YiFLcyW^49)q39#JeE5QRiwcE4U}=B8h3=1y&@7CM;`8gk)}lzn+K@GH_BX3E=3Cd8 z+Vi|dyg833fC*lcN@77~pVu$82Goq&mjIzJ@NU;2G{1YH2>$!H=D#0s_dn=Xg#TH* z`raAn)AVv zXIQjy>JO9|3eIZgfb)LYb8~k4N`qmzU!xU&-ue6+(@@y_wwF;ZVv*nRw+uo5e?h(R z|5%(*(0_WbqxpYd-9OB-{u4_h{&Qjf{~ucIMJ6xkLUU+jsB^m}A}r+C%M&O<)%B|p z07a~&F~?h0&GgUbkGPvmpGe_Rn_U@syKQEa;~Z`*l+3*&BSqX;05yBjZ@dH-z`eZv1)bfy7Fh58+!P1bNSTjAj#tR2?u zwe+oSk{jMrTf6oBh`pov3Zx&tO0zzmIg&gRn3e?(SU5&5*Df#rrbNCyXg5c4^DopT zJq(-)9eGg0DSDNK9CT=cm`KkJjWEgdck|uW)0kIda)gmu78FhZzYSq{XUZy{e?s zb)srC#G@6Taa#^4wQB&1oDl~p1sd>co81?slrlUt^O6X}_b!uslMvrkX0=&m4wv^& z2^*=$^xRvkH(6F#d`jW%#y7 Qs(xjghLr?@a1rPWau{vOA35b4~l3>omHJRx%u; zCvSU2pDxgq3%89~rES&S5IAIR+o_=`!J-MOMX+gz1J&JA@0w)yhI zrJ#L9MXgi-dJMTPgs)r zp1JU9`qj#uYlJO}n%k88BjWJc2D1pgz7iU3;3hz=6_;y|omu zFhy{mIpK(dvdr()PTO0lB%>wm!lumhrkT}z3aZ;~`T!y%F}|0=nCM>Ydc@%_>Nm&g z>{|y-AAwnbN_yPmDhin2b*`ba@h>}kb$pYJ>c+q$9rwqbFP(2u%zJTYI=1?%+@@la z#d?zIYym6lm#Q1P9DVXmM8FPI>Mk{h-LWnzJS)TbdL1_f`=- zr-H@R2g@6)s?Q$}ZT1089X~?*QCY7}dj*e|RShs(-@w(Zzjos%WRfF>AbU25u0Wcl zrk(}Ynr>NZjP9L6*%mYQ~a9?mxVPw&@?V zk4cb!*6Ew0-;#5y0pS)8`4knVG8&G1Yb0?FG5SV-OdQ>7s?IZ)^rS=$#w z;n);5Fipb})zwbMGnFfe#mk;9n#^R9Mtf`A-B{HV;!%ggZ}nZi#$5=$Lh!@c6dArb zyF{38yN5%xz&g>%aB_v``o2ru2*er?!uTKLsuq>;-%3~EfUR{xx3_-cAJvR*k*vLP zb4B`|9wTcCc4Wz>UnNz$lV|tbk9}d87Q2C=2}Gu~STe5np|6p@j<;3<`4E-(!F_%f z8TYQfsI<}$Pzl<;KP`p)qlfV!&&r{drl=q(|5sT@R~(2iaejV9`ZPT3+o-C|NG#5i zVa)^SmQ;;LBKGeK6TrS=kNMS#JY=z~)`{72cU4{|gP5^Kx&l@}I2 zhirh6%s4%)RNIU_>sAC1T^3mZ8bnolzo4U2dvrRdRCl!20EK-e;$m}pf+qGCB?q-V zTcgIXR)zqDzUGu$kbkX#$c|_xFpQ2jDgT~wD z#eO^K35?C$ACT}|66lnD?6k@tF){3|SQGc*+Oc|Kgs z&$o1aMdePeW_`Ruwby*5tp8R+bJ!ZBHw8!pV`FiUsFyQZfU!Hy7wPSOQ1?8~?N271 z9K08PsrZ8$$dPFfyZMEz|4nF=^4jC4)G`^(z8|calGutsw)^++;`u;)&|%Y&jq>Y~ zoeZtC>~B3APV>628)ACDrW<;#D}7`?=@j~Xi-CmLp1SD+okv!AN8FOqvVu8xdJr|d z_eOU6H{B8vvO-5{L`J*iH0^-{%tWz7kyIpOy2D?eVOeWqX<3gl1T*kr%n^$^&2hXV zd8fH7sHo_JJCq;bQP)7g{b@ad@4Hu-HKGg!=6s)AWdxy8voKvOk5`1mvaMdJ$JT5t z;l2GJLPUlSyl`iGLy1vDI3C2DYss2K-mt{}JMP=Td7Kb&(pROI> zJ;;c8_O04UT~a5@LrO>NLLdg5ZpfcW(sSRu&UsF@XEhzrd~Q!vzxRPBM{Vp{ukmJk zwwiG?_qi*1&G~}Wt7M(3TD$n@8mcFWS9%@Np}*~?gC|04Z+l8b)M#AKwLk;##?X-C z`ittTZW=P@*}-h@mL}5+!y?#-rDG$MeE)(JLY5s)39=2^(GH}(;$>5>H1~>BX8AIN z*02QYzhtaQY&mR*2GU@#Np$jiOUq9PzWB&qp4V<6+=Ew1@zcJrZ*hQvCsN>WwBHC^ z<2Spw@O&188FTt&L1Nezoc6lmI%*b@q9y7{_84w<>Z04XRPdaY&4wuMWO8O>V3C}G zY||>C^PbaK;3bCd+|ltOID7q@N``{5z*FR&w42T%i?vPPvzm|{@UdFfK~t7kTZY~^ zrYg*t^+Zmv7oo8#_ps^aq4Y)_Hq88}fa-Y+1+4-7`d0Duq!7`-;$*n=QjhJs)X$D> zx$`krfYNfyJ&DX`?=uIS{SwJfy?WLev_94Hi2@WSPjLl8p=r_$ zB$f9?WPau@KlMAZBva$;U8@q=70+3zF5kthOn#?&Qt-r3(qL|aWR2GwPU*izzj1WE zC~ZD9Efac1I{*?%L5R?-i{K5o(LgN9^*={_|E{kVnvg(fUQJO_Qu2$M(%rI*id|%8 zX127pcFUVZ)u7Cz+uY0M#=(OUGHOU=SNhezZJzVYYSH^y;LwQ)%t|6CQIvwuXZ8;h z?A->~mp-VzZ>~WfzZWp8NH9Ql!iuey-w4)aw1A&cA$@kz%d?yV}|Isi*&Di zrs=0_P3a-TsDz?b`nH^Yh1ZthK-Z3XJaUZR`Qvxo_&cWJ+KqAC)AM)S>_4 zC(n^0c(X+^!LYCa0m7xE)mS-q1{IM#hHE{V&R&KAha^h_OqbF$#1|?Y#L8bY?4BR| zJB%?@+T_#7YHsnGX3m-nSe+`e|9k<;{4bh-AEy1nv40Yb_y09yJ%8`^ zPr(;DWA?ADcpzi-_li~a{BJ|S2d8%PhZMp~VCFwBX=5FjI2S65yTy1xm6vF9ky&>0 z)X8yvwpW)1-o_>NrB}}T`W>}PYXbf4CElqgB-mJJu{oojpmM4OV&FVKes6&7+c8ap z%l=CC#K81?PLUNj!YIzkI%u2sA3TWvFFGH19U>2O+SS=DfJtn{RaXus8@Fxumy~Z# z;*d#fIZOnRhkJw{Zu-c<@Hx5YKEkCQa~UtGOZIJ{xR;uuge^XT-q%OmVh@~w&ub6$ zTeW9|ZZ|h#jzRwVCasZ~!u1E^u1UrjA;RB=hLjDS3Q@q20NoEw=={;dmi6-Vty}#N^O1 z%#uMm@pmkMck}pWaqjqLmwTY>ZJcyZ&TH=~hcgU}jXUhSN=y3X#zPyL9~u3dd#&lU z?gr-w^7by(Osb(=cl6x5pKNLU)`4EReFGKQkhs)drMc!KC4Xe**%xcb#ksc*QIH$1 zfwND-L-OocjWg_N$!UcxNX2GNma6~$h<6pgaer~(DOyE(_F%-c$>l-gm%TJR0_dG~ALi>`+KqK%!7O-_@nF|-jE%d_i8McE!g{NOj#?F~`7 zQ%vo*`zEFekUmuGRFDZ#(af9;I0f1W$1+I4p!W>p&0kH6=aquZ74 zf2(9-s%#t&>A1jZzj#KSZ->07bh)FC%G}>@P@hitRvmkD`qTu)anh^PtHAN0 zv5lbiE76=KJnYH~Mw1SaB$w-r``zWGTLyu|($d8=e@U99bw{-~!|SQ;eJmjwIJH&r2WMolnFSi#V<$G-jQhUehD zH_=GSnDv|wIf3tn-$DwQa$ARI{j~BnTuFO(oy8)p%aj5_@!{62yIQ1c(TD|hK_r}` z(@EQLj{Dg}cZlaX_4)VyMUv>H5#nlriq=zRM;{QscUi`WPK9pg$;?sIxo&)$s4 zt6;8?W+!4%M#}e+#`-(T4lb79M~|Brbh~-JQz+%!zEW%%aJ<|UxP#t)ve+WJyQ!1V zu*2_PX)A=k*pZTh>$jEb__f8EzgV(8s!+Nyc3&`Oyr*HgQ+?k+ZnM)eT(O>iKZD+_ zaFdYidReNX<#zJer4=$yuQztB9u$!@en)~nsb*TZR5)Kj^aj2O-4EGLWQ-LJdiGhneToTkC(xzXLi`H`!h++_Q;|GsM{ys%IIhbTld# z164@6qa)@PPJqV)V)78CT`_f#=7UJfJ#km-h=;1@s?)g?M%Qw;h4%4B`Md~MSUTo! zJj!rr++enO>1aU1$#Om>t zs~&52i}7%1;lhDQ19Cg}E^T;%9Q@<3#g;DmD5}QtYXEoaP)PwuUP4L{V82qMVjL%K z^JFGl0{>^u%0NTMvsPi1P>WXu@j0eMbZ=~xp5%AQd>tkSSve7ES0&9m-(pbXp(0hu z7{|@FEDOe&1VzJI)1Apw*%|r9LPpR^9;2_3NssiWSX#d-thtz@Re7eK&m^8MpQ;LS zW2$76Xp93ekFUH>^+B1{3XLBQGH~^U{H3kz6SA^B^>@Va_9f(Za40D+FRZUmLFxGR z|NO&?3Vn-PHuzpJA0)8^`(t*R0?X<_Dz6Ime0{YO^?W@qt}&{`31Ggz-pO&KmlrGt zccox!%F1|@2@;~*nq+@o8!@m-W0l65;)II0t9P<6V8wt?rlo{ArKhFDIL*)2$5>Id zcJo#{q8|3=6Ki;JjLp)>PXb;c17J^86&LnM4eIU@ z0lt5V^`-Kk^y`VGF6K&=3h)|!(fxP!M7TV5qRM5T#5;9^qz&dnaEglK(~VVZ{m28x za2LoiT>|>?IpsY!zd^GZ08gaUCk8FZvc+Q_2`&(Hp6VpdqEF-gs#s z`ItN?xYAfMMh<_5&zyoFj?kOJ#u{Nv!B~DOQDLOfH3>T=^z`+q@A_al1knR>LGYMk zdve)0+6YWa$I9?cdN`5^A4q0L<9dIY``%gO&SUAUhDk$?zCL*<>^(8DJTShYc#2k% zZpHGjR`et)j(4OjBuC~UT#}jW$LhFCKuG4^qB6sW2+~v@3w;p5B(D#DiReh9M_48; zkA(b;D+wDNGnu!@lW$FNPHK-fP44IoHSAWFzsvOUg3Jj~i?BMTT;TGY3JU9Zn*`;W zcA;gWb6icBelRX_=kFGkLO#m?@${Yje66%aapFRF& zB+*_S+ojQ2#gQHeyc4-%s`1B?zrs7{mBT+;{Xo~vXWX-8SJ;&-JJmYJvSXx<$-785 zOr}G$V{gMCncQR84zSL3#tG|ut8<`bf&DRS&5HT01YLL(H`bDq6h$d@?i zB@6$X?P7*u8_U`Yozg+)DtIHevki^g-Ht1z{-q}vv zos1zxgO23C5=)S%XFX+^hj8*iUT`<3(fe=ipMQ}`;<^3Fp~fkR<##o6{>_R$-~W*9 zqI+rzd?|DUeaKaDc~0a1W=qw=T!w1-s(;FH%j2-mL`p5@5P<#t79x0(s-bH*WZO)h z+7hpRw=I3JNL>L}DLPtT_Rq{K)W;4?xM2-o3St0wz@v zGm(&MC+;Zcf%j;DZKkAIu;EGpx4!jFnK8YE^Ckt$qFBv$$%8W$SC8mhPM>_r=4}^E zJu>m@=v}_)k=7F05*X)DW44*_^l#g9Hw-#daP7ANd&tmnp{=V_Azk$__no!P@oWMc z!}t9+a~oHb;*E`aqysOgUln<42o%4qTk|M-rNS^}ffySseBa%jO=7XvN@6{luJB{% z;jHh+%%~r9KSS2Dyxb_c89sar7tL(%v$+ctj3mWAzrHi*4c*`@Y24DTA8kqDAbniW zXhbq5of|Z_wK<>LNKw2)bc-lLFL?a__)_O=h)?s4tM1_3Qt_4P#;O- zqmd`<<1hN7!fd4)&Rp+jIyen9|2VqEu84Y1uzCth2sYuU-+xds0-)PrJi=dy@XOM?Y zXl3@tZa+iW&u-6CcPRew9)4AEHQ@fk{!mNGj`b=C=~;iCa+rW^FH#4m^b*8Rd}lQF zbsZV$e1*H*;T>OF(VnPPd^LM>jBs?IWo$Ahw1FwZx0dhXJ@b)`w}@h3>jlX8<90|d zfGpROd+0>{aK0tCQ_GK3r;^?D?s96!9;#{V{s?NlIdSpLX`!YlPD{!Qx9|g6Yn2^- z$e|@GEC=@Po0s!lU%oCWIr30ldlI>$CDBxx-W4o#?Sv+Rxi1roa0Q_3P{Saa*gx%1 z`k(DdKY#w5g^98=^_ugcLbm^MCBrF7#ltV+%AG8I9ot14;y=CnBlN5Q{w!L(+#%0G zp!@S{J^um;;>oAr0KbQF*W3NpfS#4*qV*a99!j{=lnBwrh=KkVp2B%=FMykK-P_``fqd`Xc_fI z*BLJ0xaropp!#ke(gbL%m@>T}%aYMolbUey^EvP?KNh4dVCx-`B~Eqq$w>Xfh0)!9 zgi`DrP@3bL(_SSfryjBMH7@Z%Bh(tmyqpKJz$cu9&P4NlS%zo`CTDcp%nrR!{O)sk zp}KQ$_Ae91KkXaWQz(e@G6)~u?N~HpeRPSV;_xmug{+g&VwGmYe4GR4=3&gXv9*) z5D2OO=2gKTbMg4qoTvP)Q(wO{OdARP>S$=>Y^Y3W3C3-n|M>%7oJI)vRn=$)v-W;% z!5a@3;B@rGn)(K24+~xN>d3vfPX$I_0{-Ra85ofPe>QMwOSZ`kNU0qcgN9*~6M-K8 z19%%|P#Zy-3QuP^5xI zFXD}vX$E&}jS6-uB^ymN;y|%`XZE|wZl2~N>*kNcQPs$}s|ah7BjBXm#Boy7@wNVH zHkSu}+!s9)2waurAupI8qofeoc$sgL1pFfJZW5m`?(5aepXNkOk#e76mKz^q-67-scw_er{K?EiShP;D-qX~d!^dt=0_zqgsP?e zZcYCXY3B?m3o9!?EEW0*M}Pid#G&tk61*omZbIAp4V`#u@5a^ajKpARn&_xO z;Qi#DIOK9eh0SQv^O_m!GQWrqOEMVY-S1R0)V_y%;`Da8k@MBE2j5Ee`~m5zA|4#) zVq9D=mbOpOkVuw2BY@$cM~Jt6=={KSob4v$p0bumeesKyE@cJ`Q+{Ym z2YPlys*#ZShblp-izWC%bNJE7T%fIRSDLb-vf$_17j^~>u^NBdO3$I?Mztv$E%>G@ zu1x3tQmFLw{Q_M;e`n{tppt;AQ^H93h(2@2`TCd;X2QnS2XP}|e^IsjVdPk(X}9YD zqp`mi{(WP0G5*w|#LT3(w0K{$x|kbNW1IRINvoQ3h~ad1cJ5(0Rqf7% zoGB*H%(W?RUoyScDvl?g>w)PEfc`c+bejOWwjWDM;)W3S)BsQ&TC-6pJzHB_*M&qx zL?Y9=BGC{3p&sJXWfGefl*UYy$&zIo4^mSX!8>^&Rw@}V+4mNH@iSCrlW5*6QMn^o zi8!r?QbLj*()eXtEEsi_9gy7E(`x@E7FP9|2%NpmaX)%18v}yD?hfyVxY;qfO;fL| z=LDR0Nklf3s_=xa>nW@P+>CU}!e78zkYYi*PNnj67 zoztrK_Gyi2Y$m4i@s3}p$qn~lR9I4)z|e*i_;;BAh>}TRB)z*>zLX;|S(;8*?g5r3JSUu0K0= z=fac}6wpc8viteDg%&4c3UjIfF(_V-Y>Q^m6WWeED0O6qEG1X zfJ8!|fe?8KN)|tnz(Y*tU_J?BdhYqZOxK}je9YqF^D^K~a7l)6>bIxFEvM&{=$c2x z?tPx*{Ktd)dQDsREHLQ_{@dbtiI(P4tf6MBPfF8GtBbPhdCpMTPaMZ3L(7lQHqTu! zBE+L$%tpBo!c2o9PA&3{kU>%~>|Guc&XL-pZDhD zoO@TbbN4QY)($G49eLT(U?JDJ$~_=V{^sUnsmcFv1a#+8P*Yo2T855}s;H4r!?Q`5 ztIZk7b;c~Zkp{Mu;ZYe1B#QcT3zNze|Ktm>J4TvwP1KWgZLQ}5Rk^g8c-1_usr9FX z3miTw>1lG_lZ?&biK1qo(91 zne~&@Y6Wg^Kw?G%=)JVsm*JuX`!4z+OM3+}+5n8&A~JV*e7P=5|BArSvv;JN+EtcA zL{U)aAy{IE7UvLUR)4Ln;ORpcTrVeZ>aEp~AG~pTf3!fNNPVQ({f;rbpDE2*IQUkk ztk<#Vl{Fu?{77)fPUU`ol2P9`Sz|IfAw!!=E6bO-C={Tsybh7L^oW_%_t2D9hHDopowp7zsQq}@64J@iiAK0$Leht zKZjMw@QN~#*cUj7B$w0z?dF=%??#Cj`kyWH!ru{?nPAlxZLD~<*lyGJX7VT3ok|{) zj9PWu(k|X7pL#7CJ@US6Sk>{4q1*4dh6dq29^zY_F&_Zc{%yq=%d;r8!ZSUvzL~ChEV%Itd`%!=8CFaM(Klvh+@`6bD zm1?d^^FYB(Cjkd>(f?uZEu-pMmbTFZ2n2`V9)i2O27V)EsN1Q6JXX>5?ke?ivl` z1cfOq)2_cX(t9o-j!+@(YxgQLYry}84XYWLd-1WFZ6WD!h|47t*0tN z?z85Y!rm>OBp=Q&^L@T5wY%FKq`bxC1lErtU1YY8l^=(Z*%5J%C@>Y^3yHi-taqsy z!#h+KAfJe~y+@Rde{_GI+hp|9M>{R=g%QT;i%0^$lpDdRM=s1A1nI~gO0{u#^roU~ z5U4Y!CcV3jFZQN{1QErIild!X;R+lGqlCUh0caTrRs7jal$Y7 zUqa|m&2?X6k1y_pf??(^0kK$zn9$SBuPV4+_9)|=UOy4PPF!$Gajpvo=jz@xnC~Y; z;BB-IG$IiSTJ<<8u6n_zstO*Mto1)hIKVWX?Oo1t97Oy$SZT>9Y-E`_T_1jmcm|#m z4gQhay9Iad!C!X^POCxv<%@c+o(y&;N-MhBi zmxR_c3&z2ln_mMBHZKV`E?fmoR@{RBIyLsdTk~i61w`Uz8{jKbTg(w55}MGjY1?FG zifC!V$dPpKk%nLn@HMd1qa^{b9_U=7=%;<>s<=EQ1M7EVa~J*(JPK}G7aJrif3b+7=! zFdjsl+<|pT(OdhiH`gO@v}kbjNl#{;mh_v$*50m;W0xGD6Ws6%$kucTdZ-G_{CQuh zLhlSl?a&3Lazn(BTZo9E4I7#AuLmti7eGep4^qs5D)om}W&w;1341z&Id{PxI`c+| zL<>-G%iJW@9zUFsJZr{qJyquw+)xSVU{|D|$q5HXrJ#}1KfT=lo(=2o?vAjwOv&-Fa+GyMKv$GKq&0zs4Tol+h7Hgo z_GRkQ7A9568R)>)ExBbN64DAd?fXP{VH#)>y>WY4VR)g0X*j;bBYUq5Z}1S7I0RP9M zR1RU%%6!&rpHq${wjpS;>HX-*_K@2ayGugCiFuAtit(^fu`km@-vG1|PTB+%Zui>~ zqq<{5Mzn36BW^)Hr;?p}%FOL?jPBvrzg&`O*kFO4Iew!$)STRJ{#YzDmxlCXqa!E! zntQibHTo=-y2`*1Sbr4thY$Y@w{nu?CKvb8Zq zNRvUD`(eDi(OvD4QsZ^EF-qWoVO}Bh^xeerDnD(sQV5tTI*jv01h}xUM9#f5m*2Fr z2pm(`mtW^^OyQ%Xh0c7$5vLYl!X!uQLq5{%D!h9;0cjp{-{@QaK0`1t{;z`zK)+kF z9fiyvi*xI>9!l&L=-93&EcOpD7YQ}?#ZlTy1X8QdSen*C!O>EVRZ##FXcao(-2xCK zosQ`NKKt+c9Q*Chcdv&g)s$ijF$N+H2Kbc=l>Lwh;!?W8BqpT7>}m?^Tnq{-1+JzA zt_4QTMx{q7XGh6Ugwurn_FwqEMo=ULR$-X8LW|X)E-&2|-rqDlpS9d)*tK8A>Q=yX zE&M=ddwcgZOImnkYH#-P4UndFxWIZj9B_(ps5CfUkO`$|up5`MEcCiGiF|#_P2+5g zAtE?#u)D5raREVJv}aNBxTJa9j^YtyTijWCJ~OcYSa8SFt(CScB_u?1Y}M>{nyM)M z{WHEl0$(d12H&5=XMD~2F&)dMd!rB&!35%!>ujw;Vc8h9^Q_=#;2kuPk)1(aM$klP zg!lv!-|{fx5tMCK)gC(og;Rcb-Y571UlDbsn2(u^=9{WbhJRgh6_(Q#KPbvuxqDmR z8Kuit9dds5=orKi00Y5E$000T@2j*N451VerofR$_(MV^U*KN}zF;LPkwM^ALPjKLViXFp)WRI>hg;uF0>&z!{*%f1L#YHuLW${!runJN0*-#mC=F_)lKlr~mr*F+O&%GH+p(hL5k@Dg+6@20fcM%3)UXK;xTf%P8CVi4Ts{AeXF zr@u-jn0DXnrA$oqhyJ!svFSUwOSYN+&8zXy(C7D;ntFd@Ay~<>U7^Ov6%l_;88s|s z-w+Lvhk2Iq4`M2_Vg4eh7k(}s{7j9({{h0Fwbs{Jh0OZ===&A@H_x&du(=yWL|}=% zW`z8IG56AMm2<9VRFCWPWq*xlR_rZk`kx1_lVJbh?(ftj>VwR1!gYh&`Uq@k^WP2n zTR@t4^qzzu>7YuNk@17TSPt@qd^syNlq6NoOKQ%KPG1xIS0rD`oKznWi*pcL} zkqvg9A>$>%1Hpw&<-})I|AP)->LIeOuy9|?HpBt8Eu`hWG;{A%tuluK)*d#p*0Arx zbzzjXd&wBQ;-W_mj8hvpd6n&Z0ALjVWF z;}pN!!?Q|Xkmo*2Q@dM&hg;NC$E}9@#+ZY+yBprLy5ZA+%gFdUINqY-W^^HZ%>wzO zr)%ZPAcW_k(l5I85xpQogi1HOmkkINpuIJbuXd5JU)acqtH%ZbMY!wb#()%vA= zn~0DRo?fn>d`;&kvQ^NYaG1M`9Q!`Q2Klky8-TT5C5cWR4@$p|kGNw@;YSq9bdJlh zl2_irqTJnlnHs&kwcZC?FiKDNvj-f%R5qvIuiP5;b5OuUQt1^{rq)8-Rb3yjnHs@g zKln+}GSznW@g3;6uL?G3JrkRb^K|h`;e1th;o!L*JH|LDuCaW6e%Rn?j=#B~ZhzY7 z;pE^Me#y|ljhPy(gF`7w(fm3TAMP!P6 zzI_+(vi5vY)Md-0{cd!CRWtm$J<2_aWB$(WJITCxM@FBY)YZiH_=Kt_R>#ezQ*A5K zKc@s|{*hnUqbFN$a%(d$l}vO;BULn+`#(d2M5|yWQKJmd_(eo^2F! z1tnSj7^6DTcYjSR9ggRs)$qH#Q{=OdtPZwii}pu znPTd`ZPXlDiR{>uyXWcM(cP&wl0@XbiCTP1faSccXP?;FA-KOL$@K7z@h>C+eSa}( zs5(8Ov3;*#`{yXV6+ymTBX8?I&+>2*%@qy@Ke5Wv8%V`cduqW_WigHUcA%HJ9bc5` znm_7e%`Q9G7Ai?MRHP^8{3OX7=`qs04)xMw35X8Sr+OPMgxQY3XHe zd?I>u_sd(7NGRVZx#SM)mS}bAGyEnd+x+AjGK3kkSjgHu(Nj>r%ITyF$`#>p!n4v7 zPf=-Len4B=ztPD;5)!RM2;D<@ZfUde09G4DB&I4?$OgQ_-b|@NFL;$Vf}llGwuaEw zavT`7#R8^W8LnmTpR-IbOG&ucS4c80KY|+txv?-J;AxUE3EMq!N!e6wIz@CWElJcb zV-%_465^Aj1T^te46Nc-`F%|xkdXlK50KAJ=NnxTPM2YL&hSer;g3HA@7=OOhi8JzqP9b&0B4_twDds0VExn*}l1_nQ;f60dG<9$s-4zZU-wx5R zZ$nOAcr?A$KS7LOrH_%wo8J=E!gI5JF;~9|rYTKnt3R*D*>I28eq}Byvt!?WSpHI* zx!lWZH0&};;C|)xBa}4Qu%=TnPCUkV(#wb6LVV%3rXa5)wRBql{D!P5alAM{mt*c@L zfBtg8W7O9^emMkzYe+qMP_C$)l$tZkKGLfTVU`#*p#&?=BwEu&*B=OBEY8`jl3-uE zwWQN+M?xxsvDliDT{q+-hxE_MycbKQpXg@+SuQ`QJcYV+@kK%EnSNitfT{qbh@g@Z znhFTMM6iMXnI?El=Y2`_`Q!kr-L}U|FU<>51-o_J(4)Xqg-%7YD`@Vm$IAQ4HwHZ; zw)y6W<+JwKy$ndf%C_iw0I`n~@rkF?veB`N&S|%#)8=?Q#Zr_* zT+KcQmWpjadoEyBt@VqGU4}*!S5L(KO1IlOnzQXX*sGG|-kgOQ>)m72b#FWBfMmuN zHjB+|C#M#-UDDnnz9fzQN_$S26*u#cd3qx0m9HL&;bOz7VGl{#y!F_v@&8xf<-Es-r!p>or#%Behra_FH1sOR5Y` z>A%&tztFTcNKYL8*<%7xNS0GtsDagA=$S**7tVe1TSWp871vx#|)i%8;@eGEI{vJV{1?$4GBQr^iQo&(|Ncv1?# zZ%t%bzd@a<>53kBhlc00`RSTFv?HR!lLx2~-DWyluM_IbqDMTL zWH?{JU|Dtqu^2(tZ*x5KYI#P;!T#4s3QL87g%$R~<#2OzefR7cB9ZH(RK_`y=y^Ww z+`Pv7`O+7i04{8UR|@|tg+F$Ke`;}`@&5-bHUEpm|38ojv{Bha6N)6Ir{^n{{zJ>{-L0e)Yi?<2 zilg^O*!srB#dULY^YuTIZ7-J6Kc>px^PR}-5u5|ZSM48z45gPZ$%fMQIQ<=(nlX}%&bQ9)ON=nL~ z`)q#<0$WV6|KiBjEHKb&HYpT42Zu-u&;?T_0r({XpEj@|j1ogN;Y1^eG$bSRQ;hkl zR8@7x2ID2wNh0RX^+G2>qyxia~0L!&=JrC^(8;iR!N@+UD=RMg<# z3lKU%(|XKV)A$-n71`3+KUo4Y(xtPX@v8{O?e_Nfi{;5FfgGAKWpZ_M3!$4bX9f~v zWNiGGqH~!d0t`j_jV~Clhyd+xBlJLix@1h7))&c>>#GQs&VC!WZYfstC!W*#-b64@Bx<}G@{dS6&2K)aRwMF370vd z`F#1Ze;e%!{w;vN@NEpV{05%`*;20>;o2sdvEPgS+x+@B__wBj2%sD}I6TZQEe%5e zv9q&#sRGjXyFT(i-KmB$IW?7F+%7OE2onFFeI`F~pAGmrvcPvOEj&6wYBZo;_)EBS zHir}ncumEBzZ)f(U~qGL3zWZ0h8-Iu(E2NI#YrL|H^nIMZ&H8^FQ*()27)><)*FFO z9cZWwNBkF^K#Dc7=md4c)g}L*?SuYL3BOZ0RR16n*h=*|$qddRIVmoI(%G&0pZ_TE z_rnIsOzu(Oc8Fv+aNJmEbzuOqGZ*>4rweK^63H}eU=Qy_SFzn0M07k|>WfnS?*Zf2 ziMXU>jnO#na7~_UG4N(spls9F9YT#VTmtd(|BcqadSVR&J$?78)B0!>|9_W)@c+5z z@Y~|g1Lcw)3vqw(5NX6pH=E~WPc*+)mtJlrJzK@OwU6w^hyt57UqV_ze9 zeqb8d?4n|aG?1KRCUc6XYp(Ad>3<$<6D=nsDJ|8k+-@FsHU>pgXgrscF>;7^kQ?(! zvqam{)Yq=wf46ktaO0L#mV>2hM}2Gph2ju8O{oqwnbpPjQ!~&GjWNq;tw}dBnLBL{ zA3;Yy(YH=DRVKkOeegRbz{u# z1WJj8kwsu_0;D5_wTEwjz5@MpXR6rlt`}@H5??X$?y^RAY-NT}V4l@)Reeo&>}3XH zY>ZNNDj^S}&2Kv#PG00;6NRCWjTsFCLqpPvgr=0vC2gAlR`Z+4Ao*%IC0-%PUmdsg zQXdHL{JE6f;_=sy@ACuiDTk3d;2;1gDSEl47@$AL8w!uxF}bYf+O^g*|J$kGa?@FZ z#Ep`3NR4MCjptSrs|)Z1w$wRRP8gmfw_RMv!-3GD`}_N#gcT+}VJr8J!x%OX;If!K zTDY`YIQR37qf%2=&@IIP20&Ib{t)S8$s>)4rNXc%HPPX2LNkM%p8x*pcBQpGD*CK5 zZ0p4)!_yYxdX~i##C5GPz|KcUJQ(xUW*<>h=VDZ&D7=kiBp;vIZTkz?IOzi(b*q*dTW#B&!N9zjgZ{aXD3R3ac1yNTw6-Yqm#m@ww?s!a+*v^4^eeC^# zEx%S~BsmAhrco@hu?q{kk7bWfzB`_<@-B9Ssx_>4I(x;>`{h)3PPUCtG&@rC0Oi@< zG55ka&QqIcKN11sDE|;dGL^zTwDvk~x22wLDg}>Imz1uS#qRtGtFk4qIwMYQ{A@>g z9Q+9a{NP|?${1kxvZ#@E){(4`n@*@sZil_tn7Zh5tjxAFFjA7KGk*4Zqs1_$&63!d z)7P~9{BY;vQlg>22htm3SSv8l301M|&ZPw7bkRdThP`v>M>KDx_D9g3=6D=%fXT&K z5!5dUy*UE4pDEQElc}(jrqll#Y&5ZeBVihcQ=}3wD7ha+WowKW84wKA)r+wlk3BrE z6oF@tIB)atD*#?0KD$e7_}2liuJ<<{uQUm-7n)z5S$#L|VQX)=HU8JCVbGMQWL&EX zK>K2OALeufxJbUJk`o$^iMGud#xy%5^fWA?0i5K@MS<8cUZmHxDi{ymBdV5oeV3TK3sW>!&7($lF9 z1M*l#B18ZVGS0xCu*!WE6Fx(ZMM&-8@p;Q2H7|NHUd_7{Zr@ zTWYKvvv)GXaY@9WAQA7qWVyN$F8g>`$@H-%Qw@~hwA7>kQyS2o)i?o3nYbU+!hC{6 zVx-sXcGpapDq)HgB%}0vU&yY~a>6M^_(T)(33(}PRSgJ5z9vKy<{^mj#Yp$ZpyU6p zO?rpT04#<#A>W_S2c0Yw_=Wn>zl z*TC>|L;`_mxwvTPFF*khPL$C|s})Vpnokakk*j=7_~fFApc3i_)teMWAO_KL z$VAc5Ky()P_@ng-%k2E&vuq9JZVQS_*f+iu!#p5T11b=iH9L}b`4tj+m1!SOVoz9K zut@|HYaP);lMqPB2>i4Iw@#Tyn&hzLVh}Z6FI{-pwZSm>vhnbfUS_?wQ zSRBqN@4%>kF3?)<0KBVxE$6sZ2q`&K6h(`U#P!#5ExqMX|NZti6Yjf#Oud2Y!Xb!6 zo1S!@0jsy8wz8aZbE>NAN}l14NBAh)`|~$5?pCV(H!08Xj;Ets?Bqd=c2kP;w~fk+ z>_)2s3bBW3Q4pM2N$SDFK@dVR;5v!YWR7^@`9U&LiUm?6Z*dqy*@bG}N_9Y6C`@kr zRCnL5kc?jxAO=2Xt~>HXahO|f4zM$x-#%ee(%FV(v7S|?@ku;7ILR+`i_Vo}sz5&^ zfaX1>gpeAa=HM1RHXn;oMg^p~XLyX`?ycy4L@eo{0Nzr-<#%i3Vt5XAtGH1uWT?_& zf^_YMYgCZ}G!si70{v*1lgkvGN2Jz7wQ9$bd)yXpS8P)9dXpwPwlFQ1+))772J;oK zX{Cw0g!WN2HrJ?%QTA8c2{(~+D{WCjO=PI(1wm=kg?k4o)iD|Ugh+*v(Ur@*{hWmb zdxs}l{9d+^uN;nc&N<*et`xr;JcwGhZ%P307RM(Um{Nu;aSbz$8_ct69b=3#)7Zsk(hhZ$Njw@>)Ke}PUB&^A9MZqRH@xw`p|>v7~Vl^yqZm{wsh zUlkdHR&Q$Lauhv~eA?=Z)u1jhaDtz);P6urMjBNl3jxbX zo7!leKh5EoseF$AkZMZ{v?k8@JFE49&5{}&V>(_sF695>BdL~$Z$?JeETbp5{K$RqRY z(0~~o0p%ki%6CMY@83Uv4*_o^rKS_*w%TGka@t0{#hgUw%OV94{h$J-uMA;$oP3tN z2K@4~5i2eIuxLAYprnElWP_!C z7S-UxAi|LQr`yS(;s#4ywU)4eOP#<%r3sXxLrkANOYmkjYlD4f=Y7vmNr?D*kdnrb z?G}DBDwf2-LZ_!n<4frnLnts$?as4R*}+o~f%Z*wZ5j1!d>z$6!yNcL>pyEaU^)N64+@Sl1*F6m zWai2Q?reF~yySBUU%h%CqDgK?N9s?0f?C?K!E8ub0nk3$st4Vhn_Gk((zdGM2?F@4 zAO2ku9zWWcku$!}Rw-L7oHA$VSHjmgIcCV(#20%Iz>vVas78v93SJjj)RmaPMMOp` zS9!+-`riKjMt>OsT(QOm2_f}}lMjNN7U z!DIf)3h2NEJk>#q)=J-hqp`kG@SNcMx~sL_3%>Ap?X?ZT^NLZqur$=4m9D4clOMtl z*e^3jUkP4@d7L!~gp;JjgWA3>qvfpjt+e==kQHz0n2{aL4|oK*DZW%~G5L z?#ym=1_#7~ry83L(KZ+x=xIT4bA-$4mLC{dXQ33%Y8x2(#$!j7#)L#MFgAo&dJw>B zLd>3k5CjQg+%HrY5)Ufp+}!`5o1II-z|qrp4++$!Ag5s>ap2~-EG3X!ZEA|kYnrsS`ke6J+FFE8xi(LW-h}n>>jH>bjns?aA?66)} zVOhl>?@lms4vzI{_queEMaClSw2fho6FxF#l_*(1xE>zX zV`CZ|bYkLQxf(kuCSze4Ty4NuWHTG~_lua}Gh3Dn6vzvYO3-rCb@Qnq$-s@sO|)Pv zeCalHl1bIvsJe2EnJjM08>M5ucR;EsyBh2wtGf_nW?_{k#}K@WqNl^^Tt?~vRA-6W zdItFF6qeT`Dt+@au*odtKP4zWv9t_QM$&V0NtECR$L|YO zCM4tF8O$Q<_GoiSmJeT@fW^jcm(>XwTOMmO@kEptCZ7Ea+W~xs1Lzl@9}#R-ORH;W z+uQT>IKnT#a8`&`pJaJmgPSX&JvOaUB5T-Nn)Vn?0ryC?UEM>yt$tSVZxSg`$f=9T zRD&B^CqHT!f=p6_$V`+PB+w4-lmFmgUqGmoQ>Mh`yxF? z&qFsh5sY$whe|(9aFRJRxx~TSX_E7cb9AtmwsuZ4qppE=*e8T_>vuCY(JY!vZi<%iKi_mz*3aIwx4Gn1~|FeNT~C(xIeTZ_h%VZdJ% ze`p~aR9T!Hojsr)jK~Od^;U#g8sC|G{(F4Wq(uS_bQj-hoiW+YJ-nnwD&JGJUar_HUN+IlNV(Z1SyQ9DUaS_s z)L##<9bmgdqosMFdT_(Gy=jPZUdTAEbObfRTlSsX%0l)U!?f@MUC7<{;u#wlH*1O} zPBu>7n|(4#WDl4)l4r5|zCGkv^roy2?W<|R`FxPy!d6Ub4WseeFJyeb?YLsuH}5l< zELOPEM-nh4W`Rwon$dD30u^IBT1AbtFyt%8V#{6(T|egCw84)ko)jvI3p+<K({2$EQ~Z7pqFx z(uq_eAWI=4+JKJ5p=y&PKk@1eQcfxuAMhhyv^d<<1_=`E(v=U{)DSrmb}b1lVUA3n z;$YUGCN!P6Ba_$S({(TMP&~ONfl!5hcK&48`oY$(V2q9dpAvEe?+3m#-E8mC!%o9l z+GH-UR!Lc;*t{$7O~D9LHYpU^ma@1)MdcCwOHks}pyoYPUCnq|9b9DS?0#a^6g_2V z$GJU{K~k!^Mn+wh!-G56!7gkF(c*IH+Q3X?lbkltQMMQ!Ps7+h&qU3aAYyv2ZFBB#hb<*cE(tg~W z4H#PtVfGN1lZr@`?%_j5BA7`HQG*+gS6K|YqJ$`?+rrpK109tak1SBR# zg^z85**^!WQdCCwAl;~b9l5d60g-8=qnNc-SHs@H+peCSqO9L%J-NKZZ%H19K9JAF-oz3}>V+c3T-Ts7oG71!$65EB#?b&t39 z(Zs@vu*kRlS7JV=KY^m@E{F>Zr6NRqz5%M3z>H}?iA+xwE97&}1E;1f69qF;90kg7 zJbCD-1R-|!F{HFSj&dA`S6rWnnI33$Fwjexa!4N6j4@PZC>BD#5H`QA*Y=x0Elv0z zWeUV+;}Zk^Z>@@1phs@L&&2BeL#6A;GihQ?FFaTOc*P@XV%zQ4%-1s|XHc*3h^)rc zpHkbjv!~e{d;9%O0;WuUxyWS|U)N>`pS|oDfZI%f6PIAMbC2+9;V(ufx2^jcmHT#D z&qkVVw+Yx#CEeAd{0O}O> z7855Ix({uNM6v38G64P3tSfb7_3wC|`(W2{$8`;J!o<*EO9V;E+ak&0k)rv4s~1L- z_;va!4nUlp_HHsVV{B;7MawYC@FIwz6M9&Qt-om>Ruu?S_?+WZP{G1=XF6vfhE-nl zS%qRGN(xC}Dp-?x%A!4HEX9wB!W=|=?R0%Awvielu_4r~p$grSrRuo(q|7wpjNzZ7 z`pP4yHqTeysky`YVJ_seP{mCm^qfPTdsTATqA1eHuo7}Ep*gyAX2wuiq@Y@kv^b`M1Z9Og=hO-uMh6xg~k zb6f0^xwhpv-N#9pW<@pRAkuLxBVpM~i&4HDf>bE7G|eh>hW1$EV`S<&QVCj934LOa z%mfV}Ny5?ejRc9E{zS_GxuB4dwQ=#u=7SmJ8|kQicbScU33*YxinJ;KZ(l>Y+!FGj z;?U3%^qFDy*?P)++xUkSMv>nFTc+A!K&g4ej6`KA=|&7)jO5V{GLPw*+qPSvB5$xw zCMOqCxfDn@=Ip|D)Kh{9Jwz5=>_oZLQFQjzB|d4Wrjg5$-?>ksa+J@|o|1>@;hlE@ zhqlnb>Aer9yBg$zIV3-`1_kcHIU`465KA$TSe|zou-N!(>hJipB%`PIo-h9BH7xwq z-oN!gd1#-qV;ZR5;LA}NTVg0v26sYD0Tm_3cw))pn3d)&iZ4{4nq#Oubcjjrm6T(; zZ~}*|2rE5N=1^y4nbo!bLm?Eqpa|12o?woijl7=zFki)XgtO1QQF?@5Z5*8bxsdf? zVig91*xg|i7?*mhPtZwu26(QX~P1sS?4P;qHFMF#neRiLMglyrtvagHpGd?sN( zS-6;UXvaDW6GhY;<*%g^NEfs|A8-w&DIWOU{ejz&I*ILYmt><+jW!vy&Vnl)-pbtI zt<|o{Zk(`k5iBseP<<0J@O$;IF&S{CHeXFp#w^ubY3gl<78&G#LUg7*+&8lKV`!eQ zSjM@vPTrA;>0>InNbd8rojbZ0C-9fsa_d8S9OqlvTI=Jd?@uW}YDN9C4Z0mbL9 zTIw#{tV*7{7vRJQZKN1(T4de0Cn3!M<@Vm);;FR_NObp{uf>plj?tC$FvRzs@Bk zO$pVc^3(YMC5>PI>&MUu!I$HmUSY0~MtB&kKo34oa7Y^%Snc-!_bNp1{mutvBgXg(w%;zD4lvtbZnVBO-U!*?Chz7V^OvToiK0P>^3Ik zU3a6c^a=RY$RuqE1_qlSTA&OXEY7-&Hi-=6#q?#xb{rU7-IIinUJ`s{09CEO#3w4U zvvQSSc{i&>$pko~bCv0MD`n_D1Q3W)ToaNzmML@=Wh;F|5|TG%_X_W>rK7dS;olEV zv6P;6$S77EqOU&2FSs@Y;f80n3QijBf9a<05c()UH!%W^?%ll&Q_>t+QesT(8FdtH z^Fjfluee31NMBZf1arC;(e6M|u0sMvgAW|PPX*FNqA3-vmnAs+ifZUtx+|g?u_PCU z?PL`wGl(NOED1al`==EcAn(R_+jIOrl5$c@xm$jUCsB4!H@B>$x7Utr3iy1M@KtJJ zvJ_jPLI$)9l|pneOLfwvSyVzuMnOK_DEq=a3B)4pW*EJ=_uJ47W^z2y-3fV>$ucVD zi((`!RA@#Wf8@Qt@0X^WwVQxnfDD)YFq zuAVv5Hp~d)!()XFg$oTA+X_1`2jCGbF*bG`EC`WoVh$CiM)XdG6I>SONVTOI?Cp8` zcv;8~*vI+4%P=C3em|Z=I)rTo&d7hv9j!(3z{G}t!B5PN)XKMQv?iVLZwh{e@`{Ec zg;X<-X-SMoKEt1(hvEKp)|Bf}Y9z&q3&^nubT6n*EBV{;5lgtW_Zu{RpOP`~?8I@3Zpmh$Q*WG4hICz|dHP zUNig_Oakor+hNTUVgfN@kTbkkeyI&rE>)Qo!DyfS){VONt#xI9(OC59ocT-)VpiP( zMf5{o*g^vs#fIoNBXOB67Q3~*Xmp}*CS0gf`XXD}<6%$f$cIFT(G-jIx3Cu)gYsj7 z0vjx!AXug`YGeWXWXLW<3ZkFBh+&xD;+QYh07EbWgEan*??O^GLNtV9mH!<6Ih5ku3wL7*;rJq(fMbLQQv6`qQqdi{*5Y8S07ORXBB|8|0){( z7mIwNGCB@AffD6iNP{V840||+LCOMBbin374xG89W|C@P z@uo*OHS*`V#>r_GVJ#)}+_*w{QrnY~p8#a(QAI|%jKu&H^K9k@^VRM>iPQsQTYias zTBC(9)xnzfhCn#<_%D~29xzh*MKhiH)!T3l6VtQ))$9Yk>M=E4Vcm##I^_Ezm1WX& z&S{d=T;mgGuarA6;b+S|@-@L7?GX{6ulDr;Syhtlb-g~T&noJ2MP_GtUfMq+1zF2e(3@EK-R@{n#=rgDgCA;>fGJa!PBDcRr9-kALvKPtI)|$JWI&NG{_hDjI z9Y8s*G#IZ5BW}8c9_Luv0S|Z7f>uwE9OaV|V0R{*C$1rsgyfRz5JSDvlm^^^V6{WR|e&R(Vta=WqJ!)d9MTO5}A0n>DhK}xclE-+k!<4YMyAORK7;vYmhV|5;{1nmQBreQ=M>g~#eD{+ZH94jV=FHH2yCQ;csMQz(vLhn&eVCRoFpT0Ug4}Ff z2W5O7zcTtKz9I3$%S!&RP9347p~`hrGj33Murp49O>N@SwGEHv`@)rp-xQN0585-~ zVoA&i+r~IfO~*hdS;ZpCUEgcZIbu5#rJ!X*C)2l%-g&%5q$Dg+rS1p}JtjoD!M|G+ zx%YuX5Q(re{?O%%{E zyweLZ{fwJ~Toj0lVwUyT+XMfAD(>Ys2f4m8`ST7(8W2K6@MFVAbSjDG0?avyY)g%kzg7a zGRZboOymw1U*G$!nId@o^!s*xf%dK7;gNW)nZ7T?nFc=cmCh_Q;TK4#(8RSjW=_-y zRxj_I&07|ka}}MF>L6vSvH{tlpTc=EJy?ehzvu*xlh=9rK;h#)yCEwg*$g*)mVjm6 zi^0xTSz%;qtZsGVfe_3!x0?BozVd=_KR@LRFo?M{D0A?r6L^u2H%w3zbg>o+Veg1= zcTJX;%+2pW$s`-NHmm`vv|F*ym&&;ZeT;f?Vr9g~7s{|Pm#m6fT$YuodID69gKHXW z;wc~p@=$E_D(Tj0{@w#3{f4?W27#c@6s$8KArJJ&DApL@DU}hZ>zr~Bj?q#9bSSZu z6}E$hi$JLplI8eIg@ag^eeC-FI zz!z12xl}TX@e1qA**U!PPR!-~xVa0tz{w8U)f7_gdNFM+al0b(70P2fwP`Cc(rkTOm75}hUfkZwWmb5E_Lq^@OC_r0eD?0c$*Pe zC+n}fJ7pt0!_VUj!A*+#HgpnoR#>1^o}AUhOD8sGt6F>YXzl58>^%S;S7;Vs(00?2 zL-(W55!Uv;uxm2kIibnNfZKeSEw6^PhF1Uz5Qe@({I%=`f#;-3M3ex(jH#9 zDP2R%nZ>fpwm&n|Ev);Dl-Zg~Pe@Oe8c=qjF7;R<2M{2akRw13GOlKXCdU$-?Z>3O- zVuy^(u_^pH2ok1^k6Wn=32NJJN%|JEPxm9XPR@6Z5|CM0pBkNl?EH|b0m=HGBx{hAb3&}VD+heUP?);*7bx8SebYL2mWi&s8c7;p< zCzi8p$VszcM9eGJ>fyga9^jtgUK~m_G2CAikuoYac%Id)*?9u@1?m^w z$JrNlf!ajj!JttX*LuEc^**#ngd6`$cI2x??7mRGU)k}&M@DNs-~*9$Z7K3&9F9DK|w9UJy)a`s1axY%|#*d{1x{i+_Pp3Qc3_Dju$YF_h?Gp9^>7??5Mk7h9_(!dVw)s%ZUg1?j z+rhRz;-liM*j>Ahx|7g5&@QHic;)G*tfnfqk3d!KQ);8n+M6aHTzq1hjx-qxse((I zj}AIIxUnG#Y`TZ}Nw|&z6&OEm2@q|3KDG)3Q!4^?ytxI=MbMu<59BD{avmSEk%y)i zoJH6(g3-p?m0>Y|+oY8&4P8AiNKwf*zujKlU;O_43 z9w0z)cXwR~Zb27!4bB3KyX$}Ry?*Pyd0zag9;&*ghMkt_={|kV=hz!!Q@{pIl4gK5 zlYFjr4~C_IkvzX|Xyhk~Z<=Fq!`rl~DBeIcE>~`Ft$CA(u*@u=S(|j_FdtY?lC1Fo z8_DpiGRiPO^y!@ZSh7wfF4lS)$b6SfXKK#rf(4Abb z*#|!R8zo*B|0=}{^n&CkEXi#qX8QhJU-Jg0Yr0_Cf*1`UQ7YJ&Z4hd3?}rrHBkboV z#FZY27ZLt4T6hdr z!1(9Gf60k|-PtGn@6P^+kpHUu_lm@Y|6o!7&Z+)&S*Q62RP-M;BuoECckg+m#Q%Os z{rja$k?wzV_crS5|Cb^EnV&B9ipoldSrs*mXb?u5IUgS%&&%V5L$mgOjRopzi3B42 zOKXPWb#uIZp!dQzJw5$amm?@B=ue!{O0CBKdRHD1MeHc?KXbfJ&<_GxSX)yNqmjE$ z@?^?E&FsVa4F9`}ua&C6r6XeLzl}@Vc~aba9hU6BC-pWvS+1COuH?UJ|26&phm-aH z>+k!Qu|@#Yqaj%2wa&Z;7u4ybZm~wQ^3h_bR}!Srcxz$(-H`nZuQWwgX|B!vkj@WN zv^ZweAjM|E#uQr@KR?6B;6oTKe&Lu6l-n8hP>nzmY$Q?Hh@POWq@>j(cdz7I>_w5r zVYWf_<;W7P6!WbAM!hRsZk(E?@eAYV{pQT=#q0o8QVR3pt|dwyUEBLv@9^e3xLV4r z#8I+*PTXGwbO1lT=51bFDY@T7LFRx=Ia`mU)NR1^CoEY-JVJRj!@UWL^0dtReky?K ze{K)vkE46e&*ac$#JmKDr~OG(6qY%}vH3^UfqJECmNe8i$kW7piz;HUe)!GOGA6h>eZV%lWhXS^NeMBkYpJdD`7@{KnXcce2KdDPIrn zGxqEurrU})>=!54Q4DS*W*JIfe>S%lkSC#m;wDVkE5_87O%5O6We?&Z*}Jy#~C>e|P61v>k=u3N)K12b}(bl~L{{ zo?i+XtvG#5T)3%YK9z4w_IN7wQTJ%U^zyrh6TFvSGcd1@I$}QPs&r5`CCm1LM%C#e zpLVtu2v+U2Sl1gOSKK=(4xFs|A3wigqte2`(?Xl*%2sKX&S+u4~9zOFX*yi=U7=D_` zz9t=GvxTIEk{R43u5fsb5-T#++>DW(1?eqIw_)#Wn{~oLX3%NTpGoktri4vQ0+*`W z)3nG^KUJ>{9mv@l%m;X^K1NBuCBA3waFqjexH~C*YNQ^A_VT;>R1BS+%*3NVVeV$E!9s(Nm9(;VScHhuj z;Sjs%WOxmlWXvUj@0tA)my(NEUi`8Rui8(*nZ*t9gdQeCRXwcbHk z)CE~ONt3%iZXr5?Oq^uyRD}u?jQ%&C^nkvG5Z_Td57t! z4~`)b$MIuZI*^QTK40>eGcZ4X)iMaDuFiWmn>oLoWynUTGj@q=QL?3Avy5n6=cGL$ zet+umv{mh*e2A-B>aXPAfO9lOCNpHN*PJra% zP~hpmYo8N>GI~wygGB2 zo=^!7Z!j$$jRkiUqq z^NmCxHam^I$>=s{2vM8ZUK#Izts`Fh`8pj&IcdRR9)&v78|eal)LyJ|;8Z{eOx<{_ zPuC$ai@VdyzS_T}+nE_7FzokyA|JOem_BgnvkY`YQyV^O1M5JXMBLuitLl5+=sa%j zEO}Ue49?}-^W9$5c0Mk*xYQ`w!8~kxf@u3(1mTb2>Rz-kjy!Ifa%rEPWGt2`X=-g{ z9aQZ=ZqSw3=2`9cd9^S5u@D!^5AP?1E{jE2syvz*SE}$Z0{b=;RQIaJ*RZdpVx`J?%5;iM$)&TF03p6|@@NN8Z4kRI>*w*E z$^}!>?a5RQd78S@^p|yMp=D8s)!MM+>V>EKEuZgWI?h~#N=eb(tXs!`?6lzOB{$i2 zwJy$9l~vV7#~|mBN{}V%U{CZiNXng`>@hup=Y#6T7@2o``LV{vh_uik=w*j-hWRCY z+-F;2vyhHnwDGjC`Wbt8uJIxkUHg;txnLDE*5a%+Vt&II`TQf65GnFmk+e|1nc$hP zE`i1tg5DihD?5f<^A;cqy_e@A?&dt?Xw(>FyG{QB;iD5UyKu|CM_By{9Enu(iBBiX zEVex5aMg7kE6k{A&SYnPeh0CRe7EL@u-1C*%8)(|9+LM9?-Zf9vg!M`B#9{ewznAbWXkRb zCfV!x$ys6N3(>dz%r}^aKD^cZPduKDow4YTM~fVU$k6Cle#u&p746$x`nwr~mvEaM z*vA7rzmXrb{Y0zoR3OQqj5}_!jf`nNg}U;J3LI9;1Hfav-T5>t{Yld!+3kW}cBIA0 z&D{_)`lcm+Y~9#`M8V5!+l!B%xF5c%(icY@*!+~OvGUG=mv*Osc;?%;WUYtVA{X4} z+hl?R%ef@p;52@F8-ZxQ+ul&#az>Df-oxvBE1ay^o<@fU#{^`-TgBB{*^QR*7pxb7 z(uvij?-t`Lhf+|D`asRayxS(pvP^pXX6q-9clG_C_O0c=`0Q>y=xRPhs$l_NC^&4Y2_b zfv2iEx;LP`;DRp{XD4Ja_GG;`;1!bm07*-%-`yPaM7XyVJlm6Z7%XL6ewjY82g52U zPYlYTJFSiKm$-HYMEl%+HeIS`jH3GiPzfF0;bC!gOPvm_+5JNf3e;$do58_2T+&Z=XW~u ztSCW{!j`ds6)&1Ovhk#p$ehmf$k_Czi(JH2j(5VNqmCD{HZubV!d^u%RuJ2jqb!RJ=#>OQ2YP2;fx`Fxpgq6-!t@7<=c zT=Um=F6Xnl$NcW#Zb$$Gp!OTZeP7rUza^{FG#oCsdv($^kT9v)?i+;F5rlDX%B8eJ z;<iJlIpa-&C8L&jcH*Vjqea~Q??o} zVfMsG%gY1Cu5tI5S9-nCTq_?~_}hCwl2&BBvSGY(-*f`dl93sJI$Y#hZ|mB&ze%7S zSfeWo3`8~(Z+ed)u(wo03V`8mXu7! zf@8-Cx6m0hH))iP>@ALY36n`)UG&-B8;|0*Gp`t1g?@>mAAZbBzhyVdco}-~Uvb=ThMlM#hp1%eY(JK2yFawpm*{Ksbc!x1lwIRBALbw0MT zbfsPd&Dib!b= zVVHkajSMPRjY)ebY1Kpz$4Lu1e`D|EXYQaUpNz%M{q zNg_RQL|-ppag=BPSEk!Gh`YdC%!<3lDp(w9)fzme>nTsNy(ry7p;~yB#Odf|#hU%} z=m)t3Z&xhWL?9e{2^|dis4Ho+c6BiOCKZarka^^!ebyCvxav5V@{}#1FGfdS!J5=tgy15)T#v!YJR55V40_bb3+C``u^X zMe6y6hqrJpH$z@uA69gYrUSo>$zrk(Y1DljaOv2WT~BAf z)xlE&f!E1XNIok)H`f<&8fMuxkP@SfPpt%>l$)t`i!uF^3qAiZqvIKG3j%zT^Eu6U zR)^SH682&t0NFD|fPR=oxoU#e!rH{<{oAcn(hRCaw~X;eU$T(0ni1CNkp)VWF)9t% zl0uBf|6GcOe=Wt8U%=FT{;uF-#2ty}Z`t0;8)-0$i}-qezEHakt|*%eOQfi7m83*i zz;cd%o_}|`y}jy6U-9#BwZqcTyXvjp(Az@~B4SFs zIEj9j9JKjc@VBJ})j=i%wpAVSXVz_8DC&i4U5l2lZiMef* z?FignmQS2Oo#{Kjq}R6ApPwGj8asxH^q%w{`{Ao5N;*#BA2)~CA7`;%dP|G+j6g%L zkg9W9N>=WSygip-Q8EByt9V)4VJ}Z#p3lz6AWkpf54wYz8C%WF&p0pk?$~}jQg{1M z6=uDC%?nCVpX{#kiTgKsyQg;o)Vf>2PY8<-sV|{^%QuS8n~g&K>A=@Z8?%nA_tdzh zB^Fyo+4n+^r^;l^f)_jp@{h=!SB1KOtnQPA@a#YMD;66Ov z^<5*ses(<#sD1dj9th4|dg(S;9x$-S>=RZ^Yzg5OoCq00)pUky)%1D1Dvj{2sY(cw z3CtX&w`bJdB1~5v$MW8H6OtTJkuotsEGL|=YjF7X!v@Two$udm#oG&ycCje$7lMq; zH#y`*(2RNgX5#!3%{P%HLXO$VJ;;h_^=s+)ZDtS(m#>7ARnQu z@l#VvtiCIAGW=A9En?0Bl@qxS%J><)a$w(ff)(==D@RXM%eyofSI2M3QOoh`*gL($p zrpygMmT1o5Ce$~emF$*f{OQgQY&Uijx1{hop{}q!HYv%{mGquI%lrC`jnJK7AXDN& zg>~Gj6CRCnmBG=q*WLD!8@un8llOV_7<-D(cOqg#a)`%{bsE|Itk70$B|UW63p^4$ zZ_uWvJJhoBjb(z$R(nG-$@E8~5I$!;e-wJ61`&Ms`JvK=k!b#fvTw&pn@}29$%n$z zJLVU(cBdUE|2X8c6_2F#d;V1P<(b?4l-$SrnfsXt#7FI3z&T|{6O;-`cOnk$%9m%| zT^#g#s>?Q!4I?shNMJ_K?0)0jWxl+IMaV(Ov^2QQM0vAirB;;Xo?AE9mps)5*&vtcnLTqK+IR) z+d|eEfja8quB^NBa{6$4enPr+d-}Be5}@_PV>iFcrxCs?^LpschBJ7P=;wD*?VQ1@8>1WTU9wy8-1mQxmT z=VV>I`1#<0toLC_Z?jHM;ckG$>WdEFul#Yhsrysvg)yrVK1;s7!iW{?3Tw%E98<}A z-uX${{_L$w4&w&=QDv)@{L7RB_;>Eh^ z^C@!l`P8wiZwcj)OL)A|?u;NQ5$_PX-`4AWfiH3$?xgh|$o00ttkFX=_SBg{Hp;pA zY(0HWkOgch)@i>uANkHtOyis_0k54i1wmE2NY>AZaYmQ7Sl&ik9Ar-m52b8z89vt( z5=UJ7`!?AHZ2Z6Ud<>W& z_d?z6T9%(L+j|G$&|A}oJ;?XB$-Q?X5&=KklfvSoFDe5PA0Nkmt{Z8@%#B*z7I!X4 zPQnUxm3@ys$p|bXC>eVL8PCX9tTe2J@W>1JyzPtf&x;}D?Oi+c3A~f^O{m8qBsYTw zZ26hS$!V~=K`8KfDGR~lC14Hgq$dCsGJ<8>zUJ5p@UKLlMXzOSjYU6GQIdXk$_!u1 zkHzO5+>4l7@?gbpv7*&TMr*7rSqEzguKPTHQ=MTvixe z^8+J$i~6Og-e8XlUgKi{QF4?f8{&0lAMQZ%erbo%zA=17yyv}YJln0w^^LUu61oCD z*&j?j`o5e%7q_hq*~=}cw5n?7C}tc2&E?y;G&;J_FnW2(aps;OLnX?WNtGxSD^>Wp z@GBHQg@#`m+;*(cZ71wp8y_9XeCG3#l)j54dw9<&oT%m~Uj9YLJBgq@H4_~$&_99A z?0u0AaC6Mq=RYoDTMe)p)v>nG?kbzBFUg16B*lC2a^o4!k5NtKdo-*xXX@``LQIzr ze*+^8(P=`m6B%ukOD_%|cGNtc)v5QUNFF!+9$7VqVQz<2>nsw}jiUV`+e2F%su#Q9 z>$CT!mONgL(ibPzAtN2!tRBz*umz~T*m{otVfu0TVj{Fe&ecHK#`I_i8kC%c`DkhQ zryC`I!^&9E4rYkF;4qfFfVA7la36-<{QZcO+ImW;6`T zmF{~S#qLC$ChD>COOMdXMozF&lhkDUOqFIB*V^bRhzuN@PonPPb{>{}_l~~D*IO_s zwdx7rbo;UV}7Sf&Y;PPAyKYzl`ZM=&zqqd1Z<}WA`Ga+8|4VCr)sM4_(TPc7f%)C z;R^FDv6fybc0NbtheF(4;jMn*dYY_A_Lsol+$I77

+wr*md|FP}Fm8+0m-yL|QmA+18)RY^$B!Q3Who6asM z>sFR#qO#a}4mjl^xRTj-RPtm1>F%WgQ6Wd}=7@YT?#}4Ubneo)R zZ`5!INr4pN8Y;CcENnm2ER0UeA8+e}-V`1jZ+6s^78PqR1I_5mRk@WcBTsLQ*0#1j zg)nL@;S;9mKRuA;*5`e%Pv_2w$~HQ|THE}!D15ntem%h-}QEhD1~6eu|XfE?T@K@Lr04#c{H z&|7bwyI?A7O$vJeN41O^C>^I^!okZUrJy^bLD;~KK1nK#nv3U6&b1$|}MX%G*ryHB= z0+U(E2zalb2BeARGT|J^v=e{ z25NA#%ESjfa!>7YEC|KL#oq)8+d`G^C_U>o8(aA98xh_D%~{#*?%au0*pRFz!epHT zj}k5zQ#|>{tCwzfXaO0eUc=KZ zrNiv1owdd!a`$nT_;}}=8>hp;_63|9U=j2^Oypbxz-#vaGwt-dTLd@E0Xh41VDV)7 zDro=JzjFg^p?gqz+SM6bbpZWZH<7p|kmwhHlHZHp-67c#Ik-tLcEJ&qy^Or+p{!{q*2 zp&d6+RheLRv`FqRs7->Akx?5C^sb?S3qXo)hxea)%)d_z_y5#fHwphwH9zYARP)>Z zZ_9+B|7n>(^glJ%%Q^q0t^e!I*5UuphCK1_mI>>lyl4&JQ%z`24jQDjtRS#|9?j!> z2Vx@71}Y6rSztH+IM7Bx89P`jBglxKr!6EehYo2C5#;a+PJBaWtjmFz7z)5*D~oSr zLiCq%N|2DClWT7^{U3eEQZgNbt%{2#T%ek4I!Px*&Bi2>bj>hN60L5F39k z>RulUb&X)|(lK9PSL*vbn5wFMn^0Lz$kZxm%!j2TG%sV4*w$fS@7lmao9xsFJ6UO) zrNaIA%z41wFs(o!q2eCO(_7r)vA4jZ<$99W()&lVvOM&MasKDJY@#Js5K(es7Z?21 zFj#21d%zD0)2BSq(;FG>E$UB|<#~MXh*NE7_CN;+8DgW)Hgv8DT6Ho*3~`-SC;cRR z%0+9nEM-(W#73QMU|5h`2Vqil22n}D^4U>kkFxM`$tmtx+Tv{dcD2<~>SaR+)wT=| zSS6LqJS@5ls1(!r3!%8J`m=lgIVO9h7HTmS9jtncDqy&JTBd=uL6&ygCp9&)s@Zf6 ztz&Q+Hi_hT{0z(%6-Yy>doqaH!a_sd2H35B!+TGKnX>zBNVV2LM}Rb#9!bVIv(St! zV2{TA<7|c2nn-US;};h=8d~}u$k*ngA316!2Agk?Q^grz%y%%sLCL#BTE9fdwVT6} zR5;%()9GVo!g5v-3>}k^G*%)884_0nd%p7(Du!O|1VGr5NUM|?J&)FVt;ri>XlU47Lj#eC z9ke|71=l))hER^HzpzTKe}|nWB>6ni%Dj?MSm`x2$XJhCAlZzThj7)LlZcB0b4G=x z03RC3(vL2rvgvJ7r7`%oCZKFk*$=PX0T1VzgPb;Vzy832&Xsl1E9rTVqk&(>Br!XA z5pDHqW^M{qC1k0fOU~4T<&`}E> z&NnbK-P=p;-66DcO<6sIYwJ@H6ZLgY&JGX%3R8t5AejX*KSA+q$~dnXJN!kf3s@GP z`Wfa%27$5$mO0$m5VB(Wn%tt$gWPz(_aVj{A?vG70BXw&EN!AH5lozUWg@QrjwAA3}zg0 zxIY!XH$_^OLQ2IhRSs9C6ZiH_i|yhK#KCcIC-oLg2L>46ZfYd)reY_DB52G6ib)Tj zwxD#2n-Gq-~g195`b@fR`70Eg=48F`20g(G!)hIC+!J2jb+4r>~&j~Jk39`#*{(W14g zmc>o5AFPX7qEDAMblDn44Xu+VtCTJaQ1U3nzX{miUDgrr zW{4nSV@)buW*mI9a8U7Vh!6#}s!xd|B;&=VA8rl|MvN3Q{(-zzbIAYg5bKiN-ZaWs z5DF@Si3&>Pc1Yx`VXeq@&r?oLX0x~OPwlp(`vJh}0rPYbs#en((PHH?Zz!xaVH#D$ zX8|eb8T+FWaE|l` zEiG}k7i{DRR*9DW?_4K_W?CeqsDx?>_UpfBnY499)6?l;Di(p6*XQrBO?l6{d8*DM zmtD-*b>SS9AnuoUzE$C)n8ODg9M$=SeBV|zgHDvmLgM?nK5fjgc$zXR;WhQMv0l9! zC5kxPUr*pv=w_t?JE$fk#D5uMPoL3P#LewKA_usq=Kc1xj6i9*(fg=2GtkhWa*J9* zi~cKYTKh1QT04^>$!T(7PR7WFV1Edy#Zv-!jx`=;`zpc5%>A<^0hnp1A(u9RCH@B+#o%9Q@{zg7pV03{c^BIETfXwcVa9 zi+wR6cs@q{A!AxfqoeQ6Ctl#o{{b$kY3yT?RP5k($|2uSTVBlwkbTQBM$W`sQ}H#( zlh>U__6yDDf{Hj7tMW6V<+uv#4B#+VSv++!i)7w?DuyGK)1;S*qF5J`w+-JQ~}`^0q#VJYl@#;+C)ot^1NB)JFh)nkC@)i37=|GekWZe1OsT6`RL{j z^Xf%RQjK;}Gw}^~H!U7F_{|y76m=~p_zB4Mw8R{**`!y$4tWdKoNJqDNRKcu)E1yP z-Y@VV4a?wSX)w4D<~PT&`9&d{$+7V3im)h)sr2v-bu)Ej*OppHNMbGoi|?53)WE86 zG#(Y?RuT8G4l<0?{Y)sl7;*4auBv52JU5LRR_ZIb5KNwk!fcS`(9 zG#{0p`wK!z?;ptSU}kd#?m6XnXcZ2H%J7n0*g)R{u8%E4!`P_RLjL14o1}g?SfW0c(bsGo9KQzC$kpB@{zxRAU>MDuZ#$@3rATL%mPtx@DvxC$LoR3yNBaC+ zQF*y#gh($^(-X=#AT|$@;cj4;ed+RgpN)NFYXYO&VXo~yn~zV>WfU4a2iwr^NG*%f zh&)HDQ@nWvt$YU^@rAz4!`C5HN#>O_xg`}aPl=tOQOxvq{s#*U?bprTJKO-m0Sdo3tzY(G#c&-Pbcvk@;*@UL{S zOETA`FzYv=)FAP|Vae~&{hpr(yPdI#tEg)?jd1WWsce`;1s$m`M4_f3t0=ms=H|mR zn4Iwyg$0FIm22ncl-fFHmbX!&E<4Dl>`f{%p`x*}afq$ylkmGXD3P4~;jaH{x3!5w z@%&dvH2#D+g{5)NPC6OPO)S!cB^TyxX;Ew7K9k7UsUVI7ca5DbrICIoe{sIl{xj_9 zhW50EqnK2_J|pHIY@GGs=l5?L3B{)!ya4Vxbh;;+$BPyh3r-MThu@<%>FE>Qiq1~N zTC+5CVR@Sh159zBO<4g3;ZIb^d*bJ~yJ?~Z4cQR%OzN5jRBq{lDOwp&bd`UiCPn5T zxH=3*sV#gybiNWZT~2@2ec}~AzqmC9-D@Y>FLcEUEuQ3@vP8g(2;#K|RCo}zx*`c> zg_D&dF);#mFun885IyY??_l9@djVb$N21!M%7OXZ~Y@iFo zKr$R=}DrA(A&7vTDa14c?0Vd^tq!N;|(HOk=wf$2pHJdEvkS?Y;p z(l%f86F3QRFlhi1WeBhCk%gJvaiZV zc5>nL6dY5Shby|GI$qV4)c{63Bd;U?k+SOsCJKj}K3bVq(03D(?QHhh&g|ecAzAc$ z-m}2sRM1=id4|h_%QbXng#Au50{bnyrP4aaUVl}k9X9MSt_p_6) z(z;b*r!YnK-xfz@c3ZBE+}kX%MrUqqO)5H%_Rr;4thJ=>=drjhN$f%+&1|g1MfQqteQ%#S9@EQ5&7gu zbJdg-U@<-(1BLMi#75`PDX1>n(a*E+i-Ku>5;A2i;N@u?NysA>)#3B%V25ZEc{P( zj6d@<8|*A8FrBB{9O!pa*)sraQQ<+V%Alxg38`+v18$qG?nSgHQbLh>o1X(n4Yqpn zspui@j6$;MjkV6#40ur_5{e|w@^7-oR4B{m>iB%qI@l%gCpfH8%F?)|)N#fBCdVX9 zO?^dx0%K!P-Ikgoap(atXe04YxjtF~L7l+e`Jp}}GSd{;NdL60@;BNUOy#OB%CY1N z$;~~jK7{F!ymkzLKVF!V>MgWSG*TSEt09J4vCp;A-?>O>=wTC|WkG>e*+4?5ivj&4 zuKKF1&4gFh&Dh^Xhs{EQlboARv%Joe5)3HHUw3R)a9|mQDKe>(QFK8O6;;7bT1@)V z#o<5suUn3hIXtD7syvzWfuPjeriwbU9!i~T@yxXNgB}bvg{+0N&f;Eu} z_XH>O03A%Hit52ZO0&&5phIeWwe(SsMxVbeJkJv9rdUL$4c6Bc>@flrtNX15aRg&m zIn<26K$}xUU1maEVxvu=ih`v8!n;^V~_qaF{fd0szHK7N*ZMoXSXLJRO96;vM|)-`umSO z>ZY%!zuGsWVpsj`F;tq8v~VMS{E&dMK$EnTqTn8!@NirF-LgN>Tbn8aZV%)_kp?>@ zJ>AIK`bT3ohjF{ZFyDlOv^Vh_Jg0h>LS;@4{E-{#9#UVFCp&t&3>N8CI4XdX@xg z#r_AiSqmnPhsERd+GET5JvyT%J_!}VWoskA8NANn9{Jy78)r8p{+vO9c9Q>aQ2p!n zBrq9gH2r&(cJwe__lY93mQGt2pX;(#V5Ju#T-#q*&FRs9U^b^Z3JwpVO;6AR?Q7Z2 z2kostJLOH_Nyh&fRJi+xyxzpvFtKuS{Mv5DE;`B~6dY2WQBuO_DuhvuY6j9C)*=nH z*{;%ohS&rIqZHfH427!MmfVt7Qz=(t@-`!IjTkkc;u;mE{Tb}=5F6*z@Psi^`utRw zYPIpzkmGV2j)y4-o!GVxIG#FwDb2+@ytim!BR7w=B(KRl)_ttz`qyWYN4?v z77JQ$@ixs`9aHS4Cn{k=%=^R61!{d3TRMrAVaU@zA@=!z)wLk@y>v`-D>( zyy84oR$*&XG|;!y`+F#~cE03Be&a}Ejj2RzP5Q(Fl+;kTZ zo3?3q!ZKLVM{i*(gu_{`u81q5=B{PZ&pcVbo@b$hDh(?jlS28ETg~2y7Jlr;FsZVT zHzBXp?rnW-=8j{KThNF%86MN9{<(y*$hHdyhw{cYBTJnOsw-=0h5k_)n>Jx1OZ@G^ zyC-nnmJ9-Rq`8Lx{lU=Lp1)CiR_4dtdm84j94DvXONL~;oQ9S+I!f=n$4meEg5mMS zFAEXGJd1<7m&9krfnghRQDn>GgB=7i21G29)=_F1G&aA@j6bUn$}c zF$doO2Ms(Q5K_Y_KT}{)C?Oh#5rl+NYYe46eyY3ZEtcFR%A!d69P^p#&(D7y@N)OBZ{sKR3=<Iwx7pYZkwzv%O#E%xE(_ZStNQ=QP1fAV&W zIgU~yKFa9cfM?G(G}ezZx08B}zZj+jvGw6Mw7R9LttxDBb~nKIB> z`3@>3*xfU;_RJDv6aJWLNUQa0gR%C!w;usNMMj~fDL7TeEqk+%ym+@0Aj*)c@9X)U z9fWSbCBq!n7Nj%4Hsz4iwfKf=O)hD0(w6Vu3}}tebrbxD{&-i9+}J-mrlnm!EKGj= z2G0}B;bVs8@F;>{W$9)xJ|3g4*_)bapzEec%r;OIPcbE}&XrTr1gjVxWJ_b55?!bu z0Dsqq(okf;#1NyaX$XZG#A?3BEPdKYD{J5{u>_YH-$(-%Qw_im$7U}6^u*`Q6hIZ+ z*chVn7{d=qAn@FPDH}F1ocw#4h4MIc_4ihR$Jbbq@T|&k#1YnX!q%)2;F4WZp~=s$ zL{*wpnx)egoo+Ot@b-ZQ#m)}U8NDUZfYmHu=Ocd9`pqtlzsf+&59$GIYm{nG# zmBOMYTgxUYhYi{+;xpjgHGgTOeXyu4>lc6x#uSdDv_l*LT()hj=7?cA-?M#!e-d>= z%zHcfCU1lAkAliciwymEpP9BD6j<7X!}DHqtnw?n#q3<)038amtr!f+@p#?00HI@Q z_Txp4NMZOEOQISxxfDiXE8c5ai9Opy)vpD^mR|V zWX9O>r0aSC)9kPFOs)>w+QapK^{_3C#LM-6F_TqL20XKD-*#hyRabySS7sPD&VQS~ zFR6s^4_g3}Z`!z}Vj0RsugXF|WFIk4^XRlqOr{wqzG5>Vh&y3?7ej2}{6*E7LXo81 z?TF2hocx*No0gJ8;tcR9%5Yg&J$yXQ)f+qX$(&AYJMMzwoNqFYB_xrYG@>q{@zd-n zJh|ADZI#0Sl~M1shV|@wd`^ekXU7VdQgvc!wL!BBp0sp(ig3w6HX$-yGz{wyiR}=@ zUq_Y`hi*M`6!FXO>;`M61-r)u!i67yB&O|#eAU=lfcpx>kKn~}=8hU=?ETQ0qW_`! z+mU}>E23onElQ$ERH}*Z2Z>eZPt%Cs!rywh@P% zI~F+^*L?X1G-mdh>Z>slZca|Kj~KbIHlgtp;#Xd{p0PMqoX3r~!pXAMAB4w#Y7{lM ziRCFS6LN2-5bQv}TZ{-CLHayW6Zsc(T}Z#(R)aU)fFonfm=VPTsa)Y@U}W*PciKF` z3}Nm_jO!n=W#St@-Mpbi@*1GP{cI*Xu_6``ac1tV_PA}ceDR5Bc*I*M+HV^`Kffm7 zCCWNnArdmP%n{ZW92B^7juH08;ec*|YEL=v@+;bs;)KBKyp31z)u!pZacB4%l%F(! zWYpOoVn4?*F3^-kapO~MIpx!}wH8n{&j@|ftdA#KY{$$rL34VNbbnSxkEb6}nBhS# z<)A%*nRXJNNqj?NjQ52?X-gJCfyG-m?}LylSO2nioQ$TKr)gGP+`C|9%q23){n=)^ zWMeAD_X^rSD#MDA<7lp7^hY7Mi^9mO#BOn*zsQkeH6m#j@Cbt?mGE8hy`2_C zyoMAsj8>%$jxXo9ImlrQeI(ZtU>*{+V+>ST{TRQvfrjS7;5Z)Y7EjdrYx3gDN8iwj z!kA&E4JJMDO8UXuN6g?6Ob<8OrQ@dfmYTk9b22k#=lGujO#`tGFh8xA#gudq;ZIZu zCx26EN`&yT&y8zqeKQ6*FzxK$#kOjx@995FS(F<)#-3p7P^o5NwL%~Exj zBV`D9#FqXD1o`(V6Oy-t+q816rxKW03-)04TwaTeN(D!N`Yk$7d}a`osb2i9*WU<^ zx3Zi*p1f$ET&&*YYSl7HCs8h)CGxN$Vw=KXYa%fgPSDml^Bwu}$+)F{9da&Wzt}5g zTv|LjZpdWBrXdhvisN|gxCSKQKeN#&rd(_(RD(Q7ojd`?`&}_1Aqm8C=P$nF8CwhM z6iKEcywTg?wLmleOinr_a!Fn`%@3AIGz&<;BL=U^Cx)kKDeFv zA#A6E@A~9qx9ezmmSJ~i?3Y>ihT^O&6>J_GCew2yKj(uLVq;=r-0jj&RzMwy$a)iz zmWYInx}qH11Cs%WL*dAUUw^ppLAoUAR~Q_9 zh#?c5wR1ax5^i9aeE!xhj&aFNk!)_TaNOZ=0E23t15gwKwLsfCZKypbxJMW4PAJSR zv26>$&peXdGQHjVmE{I#Q~LR7Rwu$kIcH)7D0m!0vQO$~&?f z9=YVNz4~!=p{MGI$bLH+pM-BTjQyf%nQB{vl>K*al8si$dG&2HqdY0wl1+Pov6=r!UF)M!&xr27$I;5pooTM8~?=rSW%%^OH#W?ys=|*c>IHnm+eb~G6lEb zi26k28bku8&5oc5xq4oKhsW#QLM~fN*rB-w*{WiZYE(cCSUx*^xM&1e$?htr z*S|NnUh;SIi8BhS+qGp*s5O>m`d!@=SC#apx^Ik;1>l~hl#iTZi?=QET0Sq4QLE+i zeG>)yXsnNXuInSxJf{lJyju!5zkHW8NJ=mox55lEyY$x7l=prM8bU>oE|o1xC2#%K z!Xm1Uja|G$`$L`WblVkSLjyI7yVb%r09=ed$=~pUMVqJ3r8Y@e=_c{eGO~utFT^bB zGQJ+wc?g>UFLI@IQMK|M24rY)NP*x6|1n#*)scz?5AmkEn1|$>M%!5wdfh!W6XW84^co za4}Y10xV9=(xKF&V__+nESzO=y?wVGh3NsMUz~!1=2kD&!B)q{hVfITDsi2HrbE1{oQCB{L)Bc13yrk(O{w2G6S+sXsW;-Ki zlUVAkQ#ARrg^6Pt<`1X z^s)}?XIL2odPP;XnMxpe(H^;~QgkB)Lz7Jk`Rv_3vSn$X&l>{L?w`ed5_Z^(>mHSp zjKFA(z&e2tL-#K?0s|E!xn^p34$<%V*@q^RLN8lbXf*clDR%_@stKwYO>- z*!l-oX#;HpoVlnJr07wydB#ffpJwKq9dvo)yXNac>GK3DW--u+iMK|a(Zfv9?U2a3 z`^RYL<%x^zD8zHhWAWby*0;@CPD=G%qt^hn%gGv%F$^?`G2fIize@mi%{h84wl&E)A%o35_z6tzV z*`_Pvm7$?g_f}CE^4fDfZu*WUCqqc7Z=JL&v#1t_nmhhv}{Gg+0GxmP$t%r<*(ErH-m6 z3NJhLmFgEoqPWcfykC+>?2!%I8$-X@HTu^iSQJV%T|IwNOAsrr6W`94$RdfySlhJ# z?X%B)h19ul94cL9iX%EW29e1z5BA)AHgwc+lZr##q)#Lf9*WX{gbHZ~+!1k#2QW-L zwX)+>>0>bP-BB{3ncB}yZ}gB)F}1I@9wGFV$O8g^dn&|cBe! zF*x}y>K$~XOhSZU^GHTM+$#r9%f0(5g6b14+hbYKHt+vOk0<4wZwsA zoj-9-L<7U}cj!HTHvx>TI>qljbmoa}C^#_B1qGB`&we{Zx1%RR6h@L-xc_1DxQ9nF zR*G7!|E!$F+-NcVTT+N1xsRa~m1O^qOBzTUx0a^4q360)>(sYtJ_E^=Ir{*#6DV!n zzH+#u=c9dx?C+w&$YKMucu8P$hv6~tq5e%0eu9}W>vxERtOTzP;~Zt~6tOdljp1lu zSDPSim)@%(gD1i{8V|$kR%oMp$6Z%h30>PTWqYLt6FEJb{dL!;wS1oFJezExCE1a% zvfgbTDc7i^#F3EM1D#-llkgr8!#sbT8YBuEdCzOnJ`;Oqm2u7tg4NY{He9AK6{H}V0}9%cW2NapH#wz%7EEKxu-a0g!hlxIh5*QSUZY&Y6!oepbvta8c;9DMF6 zH@(o(FY!BgW^3)ub6_DVm)6;^=$#vor(gNimcpcUFar^oibkF()Tu4<1HX<>*au^uhj1qk<&K2_70qi5q(|9v~t z?9>5>7>!-^N|B_2Pp}Tz!B`GcvM#9w#X4hn+=jqM#wC73K=8G|J1jJ*YH*YHONnMr zmY~dNuBC(mCVK6sZh7k)qDqC=4I<}|nzpYLpE2E-zn32ut?7RcH~L1lb7qmBX2G6K zll}X9GETZs`-G3@V98HCYcsR-HdU-%ztmyJT|OTsL@eXn3Vf=snedTNN6$^K5cD!) z=qTF7A#hJO=_suph`-kctmw*`qL$AJ8YZOT`~U9a;X891WM+Ja(d<#t8Hd1xjpS>d z3D2~7t~{wx*kRX%)5a7Km^huo;PUfsY$+sM*%BVa?Cgw{40X7^ftZwanPd%w$fx6L zk}5t&yN;Z-M!<3;bh(zcFR!hp-v3P(7DDF>OgY2?ek1$^72IWp(2Bsgs)y5nQ+_W+ zBMsI`Jy)HW;1q0@_DDj7hlh`PgL5uvfDwWub*(^ciCPNjCts^*|40s=9z?DLtDn2BfEqE`O$|G%@pYpC94f_NQcC zZat8R7@Zf<4Y5iWokxeBDLaG0qI^H~kUvw-&3F|s0v0#(U6b+!WLHcss}dr*8oMYE zzF7==zu-%Lt%#BeZIO>bnx58B52>0`Ox&S}M2K-qVp5h)(;ZW5_ao$H55I#js4Xlp zZ)9^%EB1&y-4(*LxXaRES=~>aXJW11J@g?cAJRAM?^IO8k>j$>B2MqawksK%aB=s9 zH>kGo`I=>wbFcye#~4%^tM>I8g#S3K$x>8(`mE2h$mut?b6c8s0;6vhn}0$raJu`H zSp%}DhAGkATi76y+M(s-9b6;AUD0KlRI2-FeuC17w7Ez^QF_Wxx2ht=CU1|?V z3yVW>dP)PsOxv19_uxO>J72Y*>CIhHQV=7AX;Uf}-T_6Sbl%y8ojN{XysIZreu$Ge z%VxUA>w)vzSD?~@?pvwAP*fL2F@ouJ3L7M{;p2^dSOI7S$7)e!2KNh7tp*0f;>?JW z5@+X+yjF%UcT4tm%gFAu(>@QUXbCD-*IehOGG3HUTB@o@NK5Hi>C`yLXz38KG(?$d1fjWa6>+PRj7rr9(|^JvbCJ(eR+xC~oi1S?q1Zsj2sSD>;-0Hj%Se|s{z)9zR$kJ%|g z?N8{>U^p{e4mBFwzW}6vSFDq-ODi-k9RW)!q1Mh(X<3=LOAbf~JD=XLhz#qqa=19V z%AhA*wao|}0xtdn&l37I-O9HyFdyiefOwfL!OE+In;WMfCe#Ml@E<0^N< zs!(r7{~16Y8>tzwU$s>h#Ez_~MxF}EJFTv%*()Xf2?Cr7p#dM~ZGr08m;m_u^r@Fv zF6W4^u8VJUW0ygJ5zPEi)zFkeZ6h@z$b(CVAP&cZkw<4M?h=evCjJR$&^1SWFu$cM zLPN7fx?~WLR_FIWEuFwjo|2Xd#GW)bfKAD^?AwNUU&6s#mBMwP0C$B!98+ zBY3HPwrH$zDFzg*Wez@}X$d0WaM>v-U zF8-lHi~;iu>x^1!>y2B}M-ki#$w6|g4;L98E`s@X zVx;QEEgSK3XvKdoK#L!v)Q%z19~_`$oEs z*>zP)8ukV6To7u=*mL21swKB6NwMAm~ z=Xj5Zb>)d-@t24D+!~sU%zw{5Qs_8EIeG>%65?}cgoG3pqcpkE7npHyQ8*fpXYh6t zZv`+RW%xLDX6Dpo9NB_*^lh$Z?1)%!A*7CN+j0+?1#RPAfITzV4EEFv)ICr{u;UdV zz78cUE-eYQj$apD8PXbi0fTxLo@T_Ur4g#1Qa+Wc7a$#H=@e-mF7l%sDU*Qqvy_cA zh_&JXr;KM=Cs)NiD)|hW^4L|;kZ@CsT1*ZygI2dqa~)#(g1j7!S5U(*%sDpIO1FPB zBFiNS%2l+{D=VnGxs<){jsNL}gIr*l zCmVr2*d+?+Z2Y;*NEe+jCG1zO?Gt{$Z#+;`!#*~0O=J^^tj#3aZtt&ZD5-_*|9kU% zlVI-vfwGZLIYs^RH6YkAxsGUy*wAdP%}Z^$WuR}4!(Kt-PmV}zAvigf>z6Fz^0}iJ zdWT{L`j%BJ*`$$RnWHXEXN#hzPsV#N<;q{JFfL@~g#j%c=Ye8)db+B!@G- zDvs;5A1)ntlMlVMqUF9y@J5|lAL1U4RsEGqB`KFT`EQXihN*_qguP3;Gm(m*< zVjcA{vWqY{xH!7T#-NQ6G8F@-ueDAR=!P^79Bz9^3AM8ejll2u!3DeUG{@(2bhRP_ ztAj$?)d>nj+SA$lpi!>bHfankPSh|}b# zmUeNJ4@;64)HKnavS+G?13uz^e>m=4{}s}L&q{`#yBh~P(9CK2H~tVoM>a>va}o8~ zTk>)K3bR^WX_7MQN`1NXi{*4X>#*TNX>tT7+x@3v6WJ%=c~G3L1tYqW)V${TZ>h+8j>BW<+P{b>U4J@1cjB?Zs%5(BxL*;3BO{rR1ryK^2*ZU zeMeV4#K$ukY;bTGrx5kY@+0xinhi(zkEei9yf?dQC@I^@ACTT)Wnn;V4GSiG3V2D^ zP=ZhsO&^6{Qm}GZFJa?V-kw3j7FK351w2d;6{|yoH6=J~d~SnZ_=MW}#_e|WE8cJ@ zLpv=cW7q0P^9i5!0KWzsv&O3|R(YDVEN=%eH$GMX+KQyT--iCo@IjCFH8G{g4C#Ug zW#NVh1rTlN&nHYD#PO#3Du_^65)iNyRu44--Ve)qI=j#o+_i(Jwxpp+F;D$~jZ+Sr zh}3!}!u2NIOo5>Q)bqU!@pwciRu@`c5VH~|pHMu3Gr1MSJ6cp;!tbch)(GI9)}XHi zyzcm|1bro9#2L1zW%uw*Nwl&^9-m0aqO^B=1~2#?iTAr`l?&S>78acIes|*J9{y<5 z>;tk*O_1%ia-@K-CG-@L|Bpyilsfet8<6Qb4>&h2-Yr@YRkZ&tA$MTzO>>!dcF9<# zhgz3af{j3GB7rT3*>sgBMGJJkBjasq!m5BBi13+jB|dJCPqrhE%_}~Zq_8-E)6ub- zizWojuN*yDN!OT_W7A4x?xsR}QC;m&mY^|Feb)x=q8+s46ET`woc~k==3#rfa2lyz ze@APjY69hUMB*i#;Y&eV@L>}&RVdBiJvS2z>R_u|RT+B5I`90}lDq{&m_EH~Q!j(% z$@&KxY{a1&B!GbT*Q{Y?k5zS}Hh^#X8U`-Resuw84zT=oQ_N(q5GwR{Ca^@?!CcGZ z*Dr2hbQ`^;+qeD$39>PM>!J9&iE{X*acrjXz6OS>TGqyvuY0J+bO>-@Mx~)9hm=y9 zdPw>;0YN)w#}b}>u*bUmngWa@=HsgfDG;zs<2#?-=Jj?%8H8U_RM% z!NTr;%qFdX2$T$~&oA{c4I?>t!iggF*qg ztCe7~pD*Iits>}$G{*rQiYl2WIRj7ukCxGOdY0tb)Ef&D6o&_$eJs}aLbq&~oe)Hk z>f|!LOHoT#yBSL8nt3H%DZ!&j=k1T^1;#PS+~MC6bf~Y@j1KljM&KW;Y_6k;V$zwQ zWd^H$irt&Gb?(C`yRIZtVju8_sG3t|W@__!g4nGw*d?8SgIRfI>)b_}eEsW+vvT+6 zMfTP?7I%hu1L~4#d5sp$vZ()RjvaBRK?|wS6 zl3dz2DYNF;f842cA3f{b|L}Qja>_AOH6$AM+$_3z7{OW7`oIs$j7Q+n=BAO&ZOZr9 zeEVjp#CYMxnB{v+LQ`2e>xx*kOswB@kM;-_X?C(aNl$v+r=6D+!p7mK1le{JVxh({ zUlgUr-|icEp{q#MbK0divSv0fWBdUlpW1t7u))geN}|p1Mdm^^+y_3{3O$=ISkI(W^)3 zYKKF&t=Iv7$(#Zhi9G;F*zhjY6*tHzE>Gpd>2Rl2)l5J}EObpe!H!sEIt*HKqJz;F z(n6X7ppjm_X|N0UxxcGLZ=9jOF2_o`H`;zhkrQoRTcQVeb=WdiP#wROM=(}_>pr&F z4n<8=9CTU2(VL+Eb>-A=89yG3UQ08xJPr&^V8`asGcmU8)RhGFExC)U@j}v~UiY-( zp{EY+?lMTfS?l{FPzuQ$QxUe=W(3Ak%D_aKEK15|9fB}XF_omzTdi>1X6if9$)QqF zgS$)8bEdT7{}6+wK7$iZrGCd$9Wqn=CS2Y8(KEx)62Ral(J7>XLqxbU^Uh5{ zqHCm2ty8Bqfnde8QG)9S_*R;*>h9jGW?<02P!74FBNU@&Txgp*8-}=ZgMrk>@AgOR zGFuTt6R~I>-~8UdHkSORQKQrNDwhJhjJtezo({^%$ROB4DprSIZvs<&zixzyb+kq3 z_QXt+_8g(2#nak-U6)j`M`lqHV@KKmJM1b0( z0PM18!AJxqu>pHBJ{1s(q^zU_6a~9^{-+9vwi8O@dHLx>>(`9F_V`a>(I+?Hq+T#0 z4GQ8jMDVBJFW*e*q`A&zbhJWdp@4a?;W3(7j>F_SmMkVo#wD?rmJ0vmTUnBaUd65Wq@7@hC77CDQyuku3 z#d$4L0M%i<8%0MlFF}_m>WA_a*gH@4( zWJnY2$%$AF9R_=cjfgZf_*L#D%ax(n-LaK8;Up_oTUs;!mPLK)sAn)2AtuK%42B|H zcvsM`Z z*8|{!n1!m(8@&s=MpGHOX@z|1WT>dI@a~fY-z&X>vZhZV1C->NN_iFF_F)rm3NOHL zVWs9;QpB$>sA40luqj)}pk{k~hJqkMHsMpIINuTKZ48qWO76LW2R)vYWkl?tV8HvT zVQ=9QZ7OjADR$?*xg%)hx%S1Du{8UY3c9l6k$*!4I6*`~IYF^Fr&$VpuYfarN1Odd zDurnqk!kCQm??O6b8K8NOGY9gb{`=(5ONH+(MuOq2Rleyz<@tEgUz+rLnuGjP#(GT z6F2yidq_Q$SuI!xl`n?^2aMU1x^8t310#zIxjY@LEs{}#zsxF$gMZRI zhflKmo7vrqXLnHx42a9;i~0e(dC+%uZ(+V8x2UF*+;gU`78Ws1T3lWj({##K0lKB& z3b|pxFcUR4`P99wLPk^^*e~5Cs1N7o5mvB!`@)8et>vY$z9!%2oL526HZJ3#0PE;M zA3}oKD~Z&7((CvYPOG1^B3}T1=!jh{ESgna!N!j=(D|D!g_IVB{EQ7n*caXos;}$E zCtv%(5nG-A(CgPbW*V}{&?N1|&3I%>QUsUSb?8Y7k$C-E%Kvll#DKRuaN5D(zwNU9 zO2qSWV>Hvu(J@ZG7)($8Mf`u;Zgq1p_l_dAiVf;VMIMWQT z!{9%-f>+L#O{f3)?Z47iCf5J|&j0LTQTuo3|HHt3<^O#b{8xDw1Nwiq=D$l(3jgn( zWcFVxykl0i)6Mf`2b<6<;>WEDDnnw37>X&v&GBWx0SrtD9dyI}73}D6ii2q6=;(1S zcCPTF)dPxWad<-L$%8&e+YTPsO7#L=Ix~KVO{-lFE-2O`RTiAGJTT}7N;KR;Z6Z}o z2{(hn8@gSb#YED}Zx)+H10`7x&woRnc8Z?3$xVDqmF` z(8{J6o;${pG%eWD0fSI$nTm+T0OC;6>^+zU?_6)`V8PllY|<)GtuSp-@f!lqAUMnkBuHqC&E1_}4|R{V6q!p-RO%dkiuIUwE;!TmSp zQknn#6o<|LCECf9fJHN0$EAt`tkl#ni5OOpcS2L5AaNbIYnrOCA4BaRlh>-KXe0qg zwFMC`Rj-<-S&t4DsdCS+M5r~@899gnLxb8NR;U}!pot<=XW2Swz&ul{00;R&h+ZzCsZnSsrEF8%^J2SK}ua$5Um$zGO1SRl5rW9fRiTRSQ@H12E~EV7kQ^F zimAaAJq)-bM+z$)KhX%Ba$%KRqW+~m(P=Rm%jXf&l$28vWtx+#l$(bZn6-g*lK%BY zkt&nfY99W7{_X9?j}z{XKk1%N4-u8CCz5-V+i#Vz&m74skDdq7+Av6XFo(5~F}B|8 zNfbYA{d8Cl|98%Qg|7n_8Yzy|efjz%o&kJRT$gaDpK zA9^AlMyP7dWSpzx3C^9#!^&VO6QzvipxEiaLkTXKjg@fhzYlL|27$kF%NHSR#dv~1 znEsbKB45S|)eTAZfTV%B>!?T?y;K|Zk=xJJCRP@Gla=%}v3DaEqM#f|2}?)W-zLL0 zB9C%S3$SXfMTyzFCEc%Xe-pb0bOMk12at@DEK<`&-_Sm;Oo)$6-dC>x9*#d!UF3@r zwnvvgUU5Wj;sl?(F;hv0%{?ko{hZsgX<=wauzL;WN5MSrXNKLsam>W++K(7g}tGE>d>@&JP@`Vm8;!SBchk>rFLIa*R&UW&V@PIp>2JBX$Cl$ zLmR+1-tTo!9f>p~wdfpRXK}<(%Czhqu6uvAh2IDisC}mvc`UatdpParDX*J(Bn!I1 zeVluP4VrK^2JYUk6eCe4Z1;TJzmb`IY~hyeSe+BWI=`}RxCl(7c0b6tOSV+47BgAE zIvsX@2*1uMbt?%wlMFL(JROVPrg|yewFf=Ho>w4#MtOcV3v@ku^f@2EN{^Pxv!?BS zb{3g_`1=XrD)x3wh?b-~(%@=)dbM1NNsCtZce=^nbDL*SJdQuXRbVl6PRDaQ-|{>8 z^5R`i)0Q7?_lr&A>#{+&BIduLLS6Z@_z}3?a-RdKXd|Ci&Li@0!g`(0eBZ*|OO1kP zL$XKsancQ7z#XzE%c6Bp(j5Nne`6Aq;^^@iYYUYKLBYmnx5l7l?+Yok0_|*l8dvAm)%LLYR*F8uqZD4o_3wz z1dOyuBz?k5uikTCLHYev`@PufBem~KxUD6e;R-v{M1m?GdtR=?DmkVO<+4(4jVEI& zH|Z*u`RuzT4IU_tV;-0j6mHlk52SzLCz& za`PLs!JADrL)h|F%)are!5COclU!w*Ta=(ec&ya8b6{PUvC18(x`dg$9-Ce zY(nRTPF@5A-i8K&awIIU7#ZG&s{K!w1LGKZi`lTZu;R7MFaLzTZhs8ZgKy>Sy|Ic< z7^02N{(FR2F;ZJ)+wZ4A`=YsuchcN{kutP0Yej^52=9xl?_XCv9Qoq=4IKS{_v8wR zOaRlhw$7w=ZjYn;N(I$BaQ_RE7fyq^-K&JE)cD97m|L0AubA7CN_f?Z9z-=%>ia6> zvpgx3R^ky-fz+}>9(uph}19;x2`OTw$^ z2x458;=3s%Y;hc!#QgxNlwsJBv|p^hZaxw|)qcF|izC#s{dtPp2#~zK33|~I@UIl| zKbzd}%uHRl@=qkl9fDR4&X4RKPe7fVdxAI4&9(qVwYTgHt*?sejz{|NN8YE;Z}<-f zOjscr)>Ij0$T)Y0m!_OOzovG}R@V>B!Upv`cKL8#1l?m-=)~W3I=?|VUe(_ICPuD$ z5SZvBJhk}7 zVe&4{duk}k`wkM^IElK;B(>|s>X7vDNc3}lC+|Px>3Su7B(u&rOd-4iJyZw;e&Fi+ z7_)Ou6?$#TU$sc^)(+>_`Nu;(Zccr4oEUR-P2b7$Aa*`pXOoil+x;a=ZCAIBmO$N@ zLgsx7E1w(oX*?)V0W2$_vgvOPAV%BewA@_z*#3?Sx<6o5x2kgaliSt59r*J3{nb9V zWNrV#1{;sx1$XEoa@IIL4=xa6+as@j#zHOCH*$n)xSLm_!=Pkx<1 zzlPWL$j4*n*Z0>-QreokEgj>deTn9qy}+l1t5^2PLx==0zq|k0A>YRvY7jaqhI7Hj zz`XyRv(xZ>u*m_V-`Y)0oZIwdGa$4wfOmsjQYceR$k5*K(U8n2VhPJOF4Qe92Wz}*22|0}_d z&~}QTA!p-s%i9^RRy$Gu!^f_%_@7fde%<}mAKr|!weKjK@VKJ9f-?V}K>etfpdJ@U zQVH?h$+|SwS;5?cH!_VX7YCmH|DPdi<%*Qe7wmQB?DDonBF?ZNB{H;9u7MP%lw`-o zC(tq!KOK~{X|!18iJhy6dJ_jZtlQNUcav}Yt0de1PiVXBGgeug+PPos$I$@(lApQ| zfUCbrB{0~&jrS9?roG1bM`Qc3g2}j+oH5NLXTpQpJy3L~e|?=jEbb+V^EH>3#M=Em zRP&_|&S6d_Xrej6x8<@wJND+U^}`2o(DNiEm4UDAgl>nSRX}hgm>kgc4M!jL5zsnT zt<1ruVZBT%IyM#W%k^37@LuTVX?sS{WPpHg=v#Br$HT0B87kfkn$`0H=IeeGxd_I+ z%kcu+dd&ySveV1QYA9+?#aQU){wuGgPF}w&F;6(ZFky-jjGyEZSb(b~we|~>%EhK7 zHJRT_;RSSX99CEKev#PXWbk3YPPZ%k?$2Uqk`7YkIY&{_pJvf5R$5Pw&c_9%RqB(h zpaB~XBfoz|Yt|?3wmkIci>!9U?>dbRc)Q`mG*J4j7@rqhbwb*v6NJek3}$yHr;4o0 zUo6sl&yJX`cgR_e<5QJlyRny)#$#i?_q1pBjhxSg{BEb}s%SEWbXHtbc(HHb`<2PK zwHDRAcfa`l%V{~W!I8nF!tBuCBaf$=w95~t(RLX=!@V^J2&u2yWX*6jwH?dY1xYD` zL_;|T+Y9X-&;C&epLVi2LPx*%tKyZJY2xuHLdGf2ElV`c{PLZfUPkd1%ug_NtFmSD z9wEG7xAW^FK$y~xyPq8JeEgeeWHT;nEgQWb2JMh~woJmBam?+~?5@r5FRy)Ms-0?` ze4aYqn|F@O3?dO9JCgF|4W)KlSkPh5*eVk%ocRhUe!n(nXw97T zRo=59e_}L`@QRFa%X~frU(ui;{wsI%QfGvVMlO;jd0x|ZsU%u9n*b4gO@>aGggliy z7pU#?fg0%8+?F4q*fiu(#H|+(zybUhpO=fMDt`Yk;-J99wakdwBCt+aa*ZVxi|Hhg z&dL7?N8fk3&S#X%nrTxk+ly=S1$FOq0?S{sM`N(dLm{ox7WzfO-Jf#qSoDu?8#nSW z!|2W)>N5IrJAU|`$Zn%}@}Ely{CVDFVZ?2lwJ{>wxww}rBIPO_NWa65(fx=NOB`0S zn_yFzZNU^4^g6deq!Q*0$#f#cqrm=?W$MsR$TRk%>kV0}cc~4Jy2LdzmcAx}u~pjj zd{lY)VM>T4>^?Uv9hnc7s`8LWWh|CMFINo?NAQOAO872sbZwm_?X6hnk$bl)F6&db|Unz~k= zYL2lE&Mmux7~e%INCCziKSs(TgmT_(31AxomF7Rq*hlLL*|6udEESy_4dQR1|8m6a zwWH3Y<*ea}NB2T1lKYm#7$YY4?)2cj+v26`oDtdbP1lJmAvwrEJWz@^R;kt7i==O+ zW)gt#{k}c=*=K>E7FSsF-sO%F{13j=Ly4sQL#`dHyC;;K zkG!8wyzd#6oZ|G`e3_Km3M6>xQW*+glF-&BGRNNg_VB_qd@V)aC!HxP-P}T;`D#!5 z3OwxT7-|cx(n3G9XDf7`)-%_?%iBpAhKI*pNqT=}^mTWL?A|q-e)wIr_OriOJ^^9B z4Gp#-pR+PABs-aeC0qJ&AL>%R`Dt+%lBjb2IUf&)3y($=y2NVMH0@PaZl%twf5>WY z_w>bhxAxJU{L3Mgmk0mwxG-ObAy*dP7pYKEcwp~{zp^Vvc3LcF?RTMsB$UFyVGYzc zstuU};Hs*;SRgYCsmWIbRKPEUxjjfeDNyoUYzgx4s*diE4+Cob$9FPHejKH1{-|%) zP~yT)xAEHMdNkaTOPl~-6yeCYBB>(Y?SY>Kt=lB6{E2 z(au#{)So4r4rXa{tu=>ru|Dg-z~!p}p!hYF)geZ0lf-YIvG?AeZ6UZrli{fAw}-6 zG*c6(+@wx9+l-J_$^?H@(^mQ|Heh+C1Ri^XD0&wGE-hx_T<-SAGrZFUFq#R<=1y~; z{VHkj$*V2?9Ur~S@cGrE@o6zh#my_%nh!gQlL<9E6X$l8kQ7+SZKJ#WK$P{uk?{7n zS#hnF!(BGWAK~a`8Q?}=$@#=EZZw!;x2Ja@_+u6_69WW7bc#mVSKr6{-VgwjAnMD= z8`qjGs^;%MWlE-x}Z9 zE7n}*F{SI>xcw6;B+`Uze2m5Ng4^w5O(X5?$;!f1 z^!M7@*#9rRH4(h!$RzJ9XYb@fx6qj2nF9N_(`r({T^eDU*Y~@HtIQ$N4<7vAgWR#L zW@z=;h|hZ#SBbRYv+X*#iJ=fD><)a{8R4o%KPLMCdn5N(iXVZS-+}UJ57WawHdfTc zHSJNo7vg$%mz*6!0m9su0-jSr38_QXWO_sv}f+GGoAAYBLha@qUJwHUAF48BGSec6^xXOhZ5|XdpW8z%>V35w`8ge6{ zYek0`-B9MMu8?Do)5jhXa*qb#73j*BM~#GmxF7t4{GFn_+k9CiW8}s`r@2)ljDuC8 zQytAFpzs8*Bz{{y;#XxhuPImg3Xh(GhZ%yLz*jl#C0dU2!&&zzCYx8x?)`qpUlu?NGTV-(Kc|Z^bNOI_62xK|C|jF+;uOHTC@z(#8s4kCRzg+OuvQ^8ymG2>jt0-7yvR%cWpxI~VIryB(t&MrNX`eJ4;I7b87 zVwyQR?G~y%5bUomM%^y*lzfR|F1w<&`sd+ipA4a#w^p*PLKE#s(R(W`+8VuOIiR3# zhJi+-t=*iczKmRLojQz7+rZIHfWI;||3ZSCMSG*GJM3ZXWl!6>;PndA}c+pYHxm#STw_# zHu^qS!P+?|zS0!xjJ4^(HT&hQmuSjLExrAx+<44rn;(+gXMV%cupg@2aY;-`r-iGE z+4ULU9zyL%t*_HB$=P~f<#4hNU7vPuvcQ`0;`acTjM9c5_;&h1|BVHp#@s5U%cr$4 zkTk1yp`?x2q2!ZLH~gu+tg#6CoaH~-_lFB~uopg`@0y`;FRklGU&&6#YSjO9;@5iG zWHL9={q4jst4P3h-5Dv40g*q_YN63%we(y)skK;!;C2i7mDQFTlQw#&DJWTkH2?>a?q;pRAA}mc?eD(a){N5oLOs(*o z7~ju{ z)}Vn;1#gdnCkXa zp-Ht31+%@G;B=vJotSSG+m|C6ATCy}tUxMqL|(itu_5>iISpU4daDQa*Z#*Pk@R;6 zd~?>v?)HQ|k4;-Fyv4}%AruPz&nmO()&Wajwj-U3YzlC7;h*FeI|Kr#TXmYrbz}0b zE*|GP?cH9nY>BUw-gj>HbIS_l$Mzbt5dLm|Fh;mX%QWd@uwd8QHGh`o7Sv`k>-M5a zK)#CWmIxVu>Nq1gdMh74-mIdGdtKieS;_CbMh8!E_Fl}?okwhYKl43>Q0=H&F^pHT zGb5s96ac6A774u*zNWksP)J;>dAZHw_GBv(F)o zat+ZmZqWy3AAeh!MtM&B{b=41&I7&+k$uyN0;X&lbL9yIuCIea_P!aM3n`&A=vmSJP+HzBx{9UNKPdal z^{eD4jKM9ky!hE{sF8aL6^i+-Qq7p+(G6MXeKLtax#4Wsr&0;cjj zE*l4JCJj;Mf&8%+EHo~RvW8d%cHn2JM3-`g>d45&mjKJ4_c5wyVd zh1Ak!c%tYq4d9RPFB+Cv4WGzWt?)@pV?KA9r#JGUN~O4zV^n-A`fh97$&<78_R`{I zhrZf#tZc`H3=VR%e>{LSF28l1annsL#vC z&549Ka!W1&%DLjACN~V_6B8bmRlq2Kkmng_rSSFSc4M5tnn@)jyM|>5Q z3Efe*dS?)|#`5C))zrJ>vpvzCOwSGl83z_AwhJPa6a^O-nsaKX^j6>&pAq)A!PSar zPd2Y-r!|z8MVJQyQ-#wLW=Ei5yY)tHM4HZtq;`z& zaVmutsZ2070|fvdyU+E8!hcEhZ4Q7*Zr?2ihxvYV{(x5C1za$DvGe_96Rdc{UljOU z3Zi1|mdzMosA3gT3?=E~g}=VQmSe5XLfn+L3UTW-#88$?r;h zi%f8FA9gbKw}wpw6}9b(GXylb{A!?#+yAcW4+VbVK5gk4w&h_(73$238&^d8ku_-4ym&J;!0Tge_6f)yRm9_{Jhyg{Ht6?k zI>liQ`w?3M-=#VTot_-2+v3X$FHA~4n#!14?B=k{G1SS9?Ny1svCfK3TS_n4bljWW z?W{1HKka?B7<5-2gva}++(XWntEF;Tk>K+d%1wpRm0Gzzkf%OCUA{J4uqeevc+WN) z7!t$bfpV!U0rbM4RHys_edGd7%Xz#sJi@@=_veQ7X6xstKpz>e2mdD&!Ya>=J6+Pm zSJUY>{)rs^;E_7m<4qJqb`cyY=1DD7?2G%a@P zg^r)lE@2mL;&`q^1}LF1>hbN|qxXXp!o{@8<1sF!>{*m#Rmr+X;LMbXkJPwnJLe12 zG*m~W;KgEAT>cK??(J}~{0 z2oVgtHxQtP0sY9=_n1}P{-UJ>8OCS30ICri&NVfGuVW*qSP?kmHqHt9*~R;jMV_6R zHwvMr229XaH$FN>8#Nv4B|`8MjxMUFh4gIGI4xa)`I-Z3X?KAIsBzWhVqEI1Jvz22vEHw2% ztbC(?7SQ(3`lcIWy0>l-GQXbM?c1_Pn|QjZYiP(Ab;7=C5-he>)=8obMO{>7wx@({ z`K|nkg777lk}9@^j^tYbU~TT*K0ob&!LDjT%iBSu%a8KXd*Y`n4WdxH41 z!KEc>dsYH|LB^AgLB2oV>4RUV(aEULvZP!hKoCqxA&%ou?vuV94Q^>Q@_6^jSjpOr zK_3#o+V>KW;9Wto(S>q9hU`Rk)Sgbq%d3}V30-J|9D&0r+~c}8t3<}eTyaIg-5_3G zj{%j#hC0RUzilv7lTxw{0_624x)I86-^8l^g5*U0y2Ch1gK2BRC6f4*;NbPpk4xgM zf}wR&2I;|B0OkO49OT~uP8t$Gs8T&&VQ%jG1>cPU58{F8nyaYVA^e6@GP-XubA9VK zjkG!BHp?{HEGA>ZfYdWVwSEr&7wkCe7H8Va0v5!zb-Mi0zWiR zwDg~bhi0gHqAKvYj&Vxi(6SU{v&f(HNxUF?*^TgCu5}YT`D5h2=%Psp9^OtFkTy0i z^sx$rJedY@MdtNdm=e$=mgkLNl=5eF$-1(q!Q8h+7F9}LU;lQTwVYDr!WDa`f&J%u zLH#mRoYoREZc9c2o6D*ART$>Iw@X&sg=8|a?1^ely&o+rG2_^B@AhaV$1QB?W|Z!= zWwaqg+eE@ciI^r>_S=D{h7-bENI?BfHU~9s{YC8#@1&pVx@m#$b3P8UX*P{oR$~hC z>U!DK8!@8^Y^;9tZ9>DP7pkkv)8vK_6=u+25tTFS>Tn3&5Qf4{EEF!K!g?X~@M6=`ZBYN}JM= zf+>NpoO?4ZeEk}OQ}^rN z<$}Jgn9`Q&;$8ZlBH=ry;(jJKh68jNE@Wn(pV;=KZ3wJ*xi;ck2!)e`kyxji zixBV~ryVtAo|YZ2u57hANjP5BcOYyR9-fw`CIUtT7b9B7g@RtK&L-$HGV>e&F6p+= z2WuQfAcETUUW0XJn~Q(cz0u>+J4M^ZW&J+j`;CCOt|1YCOC28dUi$EiPVyIV-(Jvo zpM|UZq|?^ZS$omu9U7#57>L=;id{Hsr87+Dog2Gw5czfIK?JHitH2w5o+^4b{P`!( z@sj}8jKb(vmV(R68GH)!X}pughFfhS;a0--7U%bUG}=-rDvyE6K<1a2jR8qJ-4f-U zt^mB9N%&h-GA^2k76IP4U)P^P3jBe>m2^zyRKiSnW|~RO_M+<2g8Doc^0)sFZ(kJ_ z*RnJk2m}r8?(XjHF2RBh?(Q1g-Cct_1b250!F6zV3xPYyK61~$&;M{=?!$a*)|y$} z)z#Hi)m7D1Hx}P-f+K!)JpIri`LHn(=6dzXSOXOwGc-~H`SGVYbXSZ-B%m9Z>%*Vr z%IC$Ch>^l_dCzN(3G7!4iIX~XMr9=XxZOA=YUOS=g;9;;3YVQB4y213>UPEUid~31 zJ$Xl09ucFo2Tii1HJ!gElEsV--_}YVIFyE`bxnfU@kaf#=16K4i6OFW&Zpyb@nEKX zW4Lp%sb4V}sXkzQIJP#9=zAbVu_6-8XXN&W)EROTEJ*WG+Jgb*Fk8ZKQS|ReE@&F+ zfNjW+cX+%GK`bQPxiaQDT*BuzXrI`SPA=X#BQSdA2v&yeXCy%%*V8(D2t~y_1VI&f zpP)*Ej5Q^W^$7!i*Px29sv%^C!3z5O+1_aTTTg^d=*VfG_>=e`(Spf55hOC|L12)y zyaO0iMETYI71KrHk<{7w-d-jTum;BRfpL2#X2BcNnRojh^W88c`&HQ^~#{Athc7UlN6)sr zwLO(A?ATRg7{vtndkl2Qb}r3)6fBD0qM0xVF}$pfZnHgh=UH8h7t#fNc!b{+d6@@I zd_O@Gw2Hb9?4lU_*_STCWg4qi%0 zLNcnp(asRhhnF3g4^0sw7+GYauVe!s8Z6(4Y1YditdxBCa(|(}Hb&m;CDFK6E&J2n zNMt}sd!u$zgr4#devTdMQQU{)@rcyhzWVyg_x7D%jO#H=T-Sq~V9@_m$w5jq2gZ#xFcdNw&%xwtgUf} zS(eDjLt=tdpuIYqg{TW8Ny^wD>xBTYVlWYu$9KxJ`D^PW3A>l?csyF4_5yU}mNcdm z!vp@<894}Cr@SJ|PH2y~X_BA&bSrr$q+gXS$RIiL<;yvpOyx;`EaYxwWXowMNm@47 z1hOr73YhiqkQKbO!UZ`ai5lb-qlkwHE*R~0Q-+#r3C@bMi z^~(7cMZoEVz)l;u;3QybJHV9jQ?1_qSS9I@PNiQo>=xz#zAy)f`~FI~P)aJCL@aV= z19Qd60zwZgpA80jFKUb?01tsAgA5tJPZ|>Up#B?8OwM}dcFAR6Ipes|kD!N9j=9#V zGEQ#o#aSt$u+c*yxf_n63HRKqc}&(0RTPtU3>F`=cY%^6J8Eu8ij*DwYan;_ZEQZ7 z9WJ@Ymt8VGitv_4xwc_B)8_%jOZ*vxy(ICtGWN%fbLl|9((qG&<#dJK&n7NWqm4S>>I|VzWcn5#rLTmN40v&MVz&3c-)! zxLofDd*QJjRHY9F6Pr!G&s!1CYMo z0%u!vuy+zKG2jRpJB)0R$T_>ir!@%@%Fwa;%lGrf8|~ht@I-rrNs>M~!>5wOWkZ+U z_*LHRy0uR!g!x3>Uw{iwm7m!X1I71>ZZ@6BoVUwME#z{!z&5u#Eeh0XO*f2xs1@07 z{t9%+Bq*qrQ0b;ac1Kl21WJvphd?6LDm??ejt?76*X4IypOd`*c&pMnd{3FK9SN z;-Or}@FW7?V;RfDj2jC_hcB+qlh~?H@Sr3gm@Il3R6Jj^KuL*j&l5%j1${HAJc#J~ z*j=mACSqU(Ka>m?NH4X_VI^_mZQ1Km_Mx&j?r=ORPi{P0;yhA(Xtm`uX$gZ%DZ6*r zX5%}prSr|AyGF#{^1ow=)!R%WMo4C;ACN2-?;0ixjHIoh-AU91ywoRSB8=}^v%A=u zUI@x;zB-W6*ng=3r0(0K4BScIu)4$gp}Vuab{k@A_*g9E^+to@maBW>tOpb4 ztws2ErKx=vIBxsKr0da|1_g{P43~v|z6T|Ys%rdkDj#sv*DErDZE?*CftS-_g-2N( zCCh2%D5|HhbUVI&G*yx&aY2dbwhVi_^o!MP`p9&*MmQT@&VI#hC}U+udpN7bU6sm| z%b^=wq=3XAGN;8GRcx%q2;TED(>1R|cTYY1azA)}+xFCojF!^1?x?{`H)U5bz>c63 zS3hNjaPzMKZ7)jZ%j)*Bj7w(5UnUeOI1ugfzTx2Rc1-L|F6qr*xOTkNEycol5a)qp25p}!~T<|?mIxmMFP5@rKouLAL z(S3+%jo)dexS5+zv=AbttD}uw;}4_4OpV^XBz2nMAL(jHZ!BNITU;m zu0Gnp(-+U*tM$UR@fmq>%P~<&hI$*23t8unQji8SuV;KO5xcbx)9`cxbPBgxNKe6E ziPxTnEE67$YB_N&md_faRh;Z4g^b$(a-6r&q}$U+c%5G{`Od+=UWXn@Au#T&BJP`! zt{~?L-Ob|5W%HnKD*WL}p9*+%rkDEdSk5-5uK|JKm(c~ARfuzfq|nmJqo-Jd6?QN9 zt)}o-ABdbnNQuI6&%2^WbLIB#y6MHCDQD^1nf9Mw2ZG(_AKFf^9=7FPuOFR7b~0zH zZ76vL3HIC_AP27X;~SwvHrxb%B#lOS`J$&B5fx}r^h@eXsz@>EiQ4UMJ!11c4CsIt zs+zBM(Pyj1sD9G%-$@kOoaHo<%Ngm;^#Na|3@BEXzoc;nXUU)Zc{$^QVdujZUb2rX zr9e2>7QLU-OMjUtTlahCI3QSt%NflzHK4e;6JBQPy^{e_EpW}Tk0Q6g#~jnHrfBJA zjB#KFg^gu22;7a)&Pa4$$f!0NPgx@X-xAXvhxAL?*$T8x9WF`<+yOvdiYH3u{nu5G zpH(n?osAgFf*Bi}d_8eZ$RpmY{koDC((gAc{6`|TB20cHH2A*fb&6H>*g@4>qI39c zFXD9mb~fL5;=}szUEbijeVS#Q9*CPW76(O8AlaaVwk3A&Xlgf}C;z~8uaK)~>p)Gp z-!Xg-nT5Q1gYkM%-7D6N-&2bEO)le#Mb?rITy=%j^44YGCogwdeUR?mH?m;JE47>i zoqYxEum&r;-)u+?pT1}cIk3BCV%dAA&0%`_#N*-D)r(0E?jUbijL!w|xy$Bq5$$tq zODeMJ25^4ytxv33M9I;qy7f+?jSPy!2nhSh>4iy~^V~pbU#Ob1le@!<-s&c1XFC%; z`BMYQ%-}ca{30!sNVxH(BaSELifaC;q-ed{!w^Ka&YR< z(5hFTQX6ovdZ_PgIdf0POI!?L-}^uuuq-LtM`^Y8t|gvN4ko2&u==OK08T>FLGlzO_t5fn{~kaJ;9`YxFf z*bI{S5f&G|HveFu1oTf@JBD=gLW?ePvP877^>iR}qQxOrN>G6mbeF%$&aAw#xCBbT zP5PlNe!k^MKqAexTZ1c5;4?g)Tv8xmk_nDG9}mWXDjQV&J32Vs2ZJ;WkpyKlY-#Af z8jY+$8i|M?f$IUepW*ulP*5TYh-3q5%mBwcZGeEdICMxYl&s-;PzhCH+L44U+;kpW z-8@T{fE%r1e_KtehybK7M}W72)8@pX7ySMnoRmg;bP{g-KN( z42bfnSio!L-mljxeS!q#iKIhaXNog{l!jJd{fO9RYY5{{M@v;dK#dVnZG=UE0WO`Q zjGlfGXp#XcAOgr>LrDlKpaPj%GE8s|+x#uGr0N4m#-V$qLB^#+o)8P-QP_HjVwie5 zqjUmzSrKJpkew;5&EFPFQ$$1^4NclU!KB~>^gn;35zvdWS;jamF7LG#K$g}zIecAC zCVC;M(sGbtVw8=~-5E|R9y!I|KcHAD3!sG+5(7xngfPlo_npgz*q9f)J1JWt}+RaL8kmHwm+9(+$H{@{rGtADYaQR7Q7!q4OD<(ie=UXzrRrk)T?+rgaN>v4=W*nLiV?F z%**s2AS6^0jk6?d@uip)c=4rSm?2BmM-_}f`+laI?lxiXlN%f(}W{ zRkp85P?1tyaugRYHcsF?zf%!ZnMd6`q==_ zWpPcRSqUndPITBPpqvJ}s}Dp%h@c6GQ$qVqQKmQ`@-omO8ikFZBZb(cA0GfR$E!g~ z8vy6{uLe-r*x2kjh^0poqL46(0fuQ5`YwxhuRXw3#Y?pWB&vW!=kdNqE=g3vXtL#9 z&d3LQ;(EG#So0Zqx+xd4Os=nKPU!gWwY@u)g2uV%PUr-e*!e_`!uYmzc6HoEf~1RgvA>OtxbB{-*|u`me=7V|gSORm z-vv)C3;=@Hn|)ab04291}{8?gSw*SrT2VsfQs^jYj?ZQni_cZI=*n;cDUXC!;m5@8b>_PnD7G&PA}h3H*%V7aCu4|@Q&J1Bq0BeOF{|LK55=K|@rx$RiZf;loe+odFR-Kn z8A@5oDV!XB+uqR`;)PT`u?0t>f{UxQ8Ycu4Uws3$C6fjCfv&1#i2Qwkkho>=jQ5uAF#*q>4 z;NJ|Y(w=~r3{6dRn=Tm5X?^G-CpEPET#V1*vf79d~lP^HmDB zJcxBq1pmRTPTHTTF(D`WB;PmW*2g06JijXl_>5s2`IZJq!_rxXl`U`ayXNWoH2LVV ziq##%m~<8U)#N4cW^$u7ahdl?gkF{7)OhSem(qZZhj>=;0fxPqEm+A}mezzt&D8ot zMV8@b6Tarsr~|Iag0|lNnv&}N4}YRfKrVC)q|F2cYzKEZYn}9S{a_FS&J<={y;arGd%Kj z%sC1n&m@iUe{BW;Y)9D!8o19xTfZnK{x|sl>e*ZRcP#C1JpOxR|Ags^{4X@-u%T50 zR!EY;noV~QKke55`c`ka$27#5Zx5PhdBlJLQJ#*AHQPuI&gW@SvO8#Q)VY({Q{~yA zrODtH+3aCp*$J}{m>O-_4un1u!?O=@$OF>qqL5xc=nY|Z= zPnJ3zA%{CN=30&LGL+@-=awh5S|cr|D4=7G#xCo8!W``ZcamE7qdWo6cO@oafj638 z>2ThV)y^bX8$FOP4ZkPX{%Rnn`)SRomOzND`*W_@h{qQpJP{{ixy4u@l4IY%eR!sB z`dzgpK{VaJM}38=lGERgYUWV6G1!B9k?)Z&IVDsT}gzP82MJSmqbRnb3G{ zs!*W@xY{;af~iKEdSkTqk&h7hiR_xx{H!8(HYuLKfcs@X@vRnlk9i^nz>RPMC97@;r zh1IBGM|qt;-0w}RnEo}2;YbwbF5#)pA@$7cD5=@qs~1^LZEzQ^p7SUhDq$mC^_J(J z-ZS>h!{>2OG%!qwOR;|z|K;a`#&Szd5({bCO`2_tjqZqK#aK9O$|H~3aJ^Q2S(dQd zYfiomX*@kAJ?X|y>;+?tCUb^!b255UNJ*vO z?Lq7{`Yj-qHNZVr3a_uIr77+u1BD?OB{PDQaA{qnr0?Y#g zY#t8l7dNusBmiWYJ1HMoPL&<{(*87fsNjvST4T63G5RUkEl;Du91YA<94xZ9b9k)8dIy3a86&Y z44ASD@v{KfXx5p-Pz@Rm9V$Ad#OFxs$$c@OtMdCigR+ZReE=1)^e4xg!=Un(^cdrg=pAz(igDj9TzR+J{N) z)T_DGlOsGf`++^yo1mj2@y|=cc-BV171*e4NV0U{l@*-ju6^=zcK|t9MfQ&f$d^6N z=7zfdSWtzk-`L-xrDidKI6_$Zyc)F$;jL?3c1kWN7z-`F%`c3oc5}X!f@`}K^5 za}Zl&^gaDRB?ARi?~DufDn_h$$ZYhfuekf()wJfBpBAo=O4YvKU=YagZ*I>aSKREUmFM<;XkwMFeYbqTbg|r^ z#Ew6|I5{^=P~^Lt{9VTR_Q!1SU0k?Oqq zb4pcN$*omtB8JT#(PB%5`*|6y68n9F7+FG3J0ITE?X2cKG~5H4vMz$Z8$ z<17EP9z;gDLup=n^w;UlX<7Dgotw_YuMXI;>tXFtE9AkteJo-VUb_|<** z;HJ8`Iym-J1I%ELv95?gwjR2G$~whl`50vnr7pi$QC?^X<2A9JrJm$34+twLIHAcU zn`w#^pbTw4c9YhxN+5Xz2Y1I8sxVDl)&m zSGtUnkMhmT9Za!NrS_l1{cGi9h9$4&IzIEHi_rW*CkQV!IP6iC$icTsc4l^Vg2?Tb zvkPQ%M!L?9&Q;SriVLO3x0Wr=B{3av>ZczBPJ&WT8R8sn8P(B2N zGR}BcIUk*GpNUssAr3E&O^7(+-P?m}C>YssX&s_lva;N1-#)VL=4}Wr5tRxu3K7hx*% zJh5QFpJJrHIc#zfN_M;=zpR4ajV}Z=^vLOr}5wc(^%IfX>;N=h` z8K=V+DGO(2a81Z>RjRK&JrT(O7-GVouTY7*#M{GdQchq4&nyWZi`EWT%$3(ki>@hB zLS;wSW2v5`lmRlC_K4Q3DyGq*R18VhQ$+7trNR%hJ;nOwSZEDbQ)4>yR?5+}iS+P( z;mt!icQ$C#cE79R^{~oI%dBaQ3lPHvyC4E%JH2Oego|V%&=q~EF3VCvik;P-oU0d@ zE5<;H{1=Cmi5@Y&c8BuQy@Eyc?nKj157gcEkrM$UB$y_mxL}cgv012GUr>;<_@Dri zQhwB7!+rD2-7!v(R#I{3!1pd@=+sEsJj8b+n_Gx*3seGy7gw%%330?X)maf00nk(|5GN%b?T~b>{(X#&CR^ylT>l)naPcZ(V2pE z@GRebtPDbg-1tFiq`b>?44;}fUTMg?T-rdw~uM4n31vz2csqcmATf0232)_rE<2uGEO_nP!P&5)P80U zd7Nbj9d8-5ob7?Gz52TPIR7l>>vtkAcIp{;uvt2oP;zq`*j(s1W8ZaGA&bpn%3)@+ zK6vx?45ux*3$(2)X7=V0`B2eY4}(E&{hf_P!rCP}{t**CJ4%s=6R`o21=A7>zC zXHT9E*t{5Kg16?t&1DyF*ovAa5~ReSeFI=YRBju`Qjzel^yZ_f4IW33Q+9_nnnQ`A zu~h^P+{&?_=$J!XA!LYsHBL;=kn;BCySlm>7#>c{%8;;iy-%!UcR(^^#K=Knq zDU5^*Z*ssAUvL;ORbngnVvZ|{hHmA~*80>RR%hiM7hnrRYN82tq|6~CyihezysMIy z9dPbeNRB{ir-=je2O3mDLPc+H3Q`FZl=Co>5_>Z4aDcn_hfz`rhu5c)ICLAyfAI&k zf>KRg2Q+v_GGaM#xLA_eWFe3z6^(*N8En#!l-Y|+t1N!WnRgaulx%4Dce^5vID?6b z+kKJKc}%n61)w<Ne;<7tQc(Ns+R$Oat?i}l(amaT##C$09C~C; zGBf9o9X3!|QS@_?1w8u~o=j}a5W7I8fh>3EQBCh<1rjvCxq7JC>LV60!q>swLdp~p zE>n+AJvt1y5C9Vwmz2Ifp=6(79VlJWm+BL5^`u8oQWy}A8)TJF6iqK46)v`Lq?%{r z{vH4cm&ODD&kgqX2Mg#WLPq|l4F0DmPQp!D`$BoRE%l4`!q5C^v zflzWR!-?WvdC@R~BEU$+&5aWP0Dw{y^6>DOo11S*1qBB)JMK!Iy7K0tqyK}b|KlS1 z=`Vr*kIw#afmi+?K>T^mB=&#Tq5ey-#mv|w+V!)^$0W_8Nbh%_?@3X&#q2w7$x(s6 zd14h?kRW+`^87;x{`+DdWO!Eh(oA;+^U{pA4noV5>VNN1{4f3QA588as}kwAZtzDh z{MUs`Bsw}et#~wEMqXZA84V)9DDyvxZU5;SFCU7~8!2~sIR5#}&M&h11R7$P>*;TF|$A}cnp)rYCJr;e7UOl8$# zR&GJakN#WkMZb$k{~Hs(#~~?BK9r)6!YU=3OsIcG@tg$hc*<}J0*>_Wz;V&wVDO`c z67*T&V43oQ^~)HCps$Ar+2g7F4S@lL#g|~P;Jpqqf5&2sOqh^8VoqL&ys~n#J6(D1 zMyGFp07L*V?(ZeR@z4Vq^!dHkc%*~Xty>GjnSL%Goef6{+4KFkK6h=(3cZ(k-9xj7 zo=^>!I-c>>7uZlmh`0#;qa^vC`Rkp9gA(vCI5F}`M>B$@^Cj%QG)34`tRcbO%#*Z-_ZwR&q)6rfWOlMh(`an z_4NPJ@b8grL8li>%gTb3pnnnmZ({0|jEsor+qSpon*$L+0BI2w?C9v||D-Z~g?}oe z|M7YVNO1kfgFkmze`=lo&*uhVf2R&mhs6J%8~&}_{wD_fw;EWs5bK*4$G-@Mjix46 z(-cWANjtAR*8joCbRu>^w+kjZE{F2sUN3U)h+0t40Dn(X2+MtY--!0 z*urhc&GoLo)Oo-$>LnG5jKy#OIH9Zor&XH01gwcLNz8 zEhnJt6CaJIxI(j}V&sV#CwVy7vsvw-lx;X~KS0Er}z%*dm35T59;3 z^-b+=OU7__$ziudET0x{2+JIBSAHgj5nDnvI!ejxX{p)7GSV^+=Wr$LDU z4vt{PxbT|x@Q5+_)3sH9fCEylKo$%zGOVdg=!8zQ`)C>p* zUwzjt)op6}t*&#{EDo5-R|kcQyJL^9aT=re992n`xq7gyP2`syU0ohw6vvE?8&!5r zY2GO12;0Xt^E-+`^UXJiTC9?^vlIAos`*M&;o8bAHA^Q}g27Z=6qiKB3p>m;ujsQ! z^TWD|)F9SmU}-eTSd5g5X5Rq!MB{rrj|Up{wmL-aN(4GBbg&3snKEigZGlf5DLP;_ zOkF~H^$nT8sGDSUu`|JP>ZP10Z1r*v-z_2;=R)c z?^=^S%#YY9!Mpv4Gp^Orh@)W()^C{o$m@E!M(TF65~4stLdT|7Ww1SWRuq#h6royZ z_EY5i@u2YZHaOpPvEIFR^^uiAIcayR|MpaO%BNXy?1#X|$mW6!&O5C5wI~%ySo@{`BEPJ&9+2Dwu34xX7{~=P zjAFNq4vsz6e)WtuS}<7WV5D?+ev$trJ%N=Rf@5BCI{MS&vC7igHGBPJ!_eVtsKb0D zj0V79*znc+!07uuFZAO2r<6_!kIv+d9X=AeT9;n@7vSLVg~jT@XA$q(ljrkG_GK^Uz?_-1*8P~u%Ln=6c5u{aj07;!fJI5aH}PS8)JfJ7 z>-KKG-i?+vh3;#)GP&O^GzmR{I{($1`!_ELr=eoNwhC3-#oZYa9w9+XN4stq3|caRcbbd z+2oTbGw1aZAC}K!(4t z+PA8>F-&^Fwgs!Y^vxC~lH;Dmb;Z1o9l;Etes7_@71-(+0AM&=?=4syJB*ny5i^C) z;2zoe>DNDB@ws+6-*f@DvSx9FeqtEC(&&{|c6t7gdQv&aH`!i9rCV@wH}AC0q-mzNTT>w7I~TZpJ=!whZJW<SKj+&yG)b?q$bUDht4`<-@XSv8m+*jB5OP|M9~(=D^AQb%hCg(t?Y!`l7g& zE~bf#?*D!xwnCqkhCcijoV_0S(q;|i(n-)Y0Uk2Cp@--IP^Tyob7<_)W5kGr>}SHl z!z-$(i6sTpX@XK6gJP@%Xn${5E?b#P^=`>B7e>9`pfD5g^O=|U!b4v@7;CDiiA_8~ zG2HqR6H&A+`==;buN)hmxB3%4F6gM;tf8G&8^-cZES$b?GO!6*SUEg;t!EtPepLzW z4DyCoH#bL&ZAqJY0_gaHyn5Z)g6nVKK-)KrWsVI>sV7>+*$ht(!N>b>GE0GS4A2S9 znr;Q@-exM9+6Kj$^^pKd0%JF-C^P}RT02TcetXJkx@BQWePW7G*64S4xEuSyE(=t` z$wtXF?J?pv8c3`^Wx8C{VH6hfY=Ac}r}x8}4%ZN{nVZ{B$+iqHPl{wANowbVW}^mS z!ulFeA@q0)oIPJS!xpfmi^hdV)@9wgp%!o?jEx~&BgaBn0@-Z6vlkB*0YkeEakHJ- z2RV9jr_`WBouiP4GGQ?bu{Xow-;{CA_m_=QL&E(%F9lTLSAkk?sx8=nU(5R`oqIr z^QkGiUKu>)p^zhDH(rI|*M*(s*gVbSayA5BLLz8cLhLJq-OLc14#l!CHqeV|s!+Sh z0}johaJC3fO?qm!>oc7naJc$JTA*H*bdYr(J6D-V=ArN0YP}BBkF(}2F3u^r=Yp$) zF57+{o;@I`u=VwkI&hzvM}*turTTw~uzkdW51^$9vCj+vc~xw&tux-pyhzM$`Y-&V zPobl`I&nP?dl$7bawnfqS!U3fUVJKjx538Rs){Hazp!T|aHp!WjT;coUp9C>HN31R zjx~4lW>N{Pg-uwyrl`$uFb9*t{lxb9MyG2;%&Bb>rXaDA{<4*7hJ*-8@9gy;^ZMBA zrB#3D3|kUL~N=h%Pw12rp0uo=}6r7b#YkJcg5Y&mEQE+6EXGeG*nI| znrKOHO5g9=O|OhA_GGx7E2-P|;{$(lv{v(LvJ+gN5=_0-?D|W0>ok+;fFkxr!xzjS zz2x05-196|@N63PZ)dT!=DNt6S?mcXeowS=2$v7vZ%%#O621E%CnlnSzb0O7o7_TK zZ;q{6opoTL$}n($LO|hY-d&5XJ)QEjJ*%V-34f5c+P~WQVeWg`oQb{t9lEJz4^hx7 zY;ibtC)3DC`SB&_Gw0P3YX!F3+2-SOnpBbyCEWQ7w<2G2#+scXxyZJCM)cZp>Qw{n z(DpkyRZ*B7s3e= z8F71K@Xl`??wk00`b~-VuOxC1xnS%2J=ps~s&d{r!fOPCxhazc$obS%Li$8OoW|Ej z)Ld2ShNm-pTa@N$-pU}|@AxO5H#+G+-n{ZT>E_k$rdb)2wFBmr7($BVl)O6MQ}G1JpJU&Jfg>s zq3=#}qKqb;0~is~Gvr2zA^z&Ll9`MdY&e6?X^YeX#SwU^e-5bAAl z_PCm;qen%m}Hf+oExY_Q$l95H=Tz7^nRDo~m=mc|=R8B$;9?J)})_UQZ zW>^uc;S4G}E>h&ml%39zyU>okT4}R=<`f(vb%k4;{N8#|yb2RVKd?C~bGua0^8DSL zW5O?49M^T{BhBFynT#S&Xw5YB5W#6l+o{WIr#U^pE#gM6@CW)Mo!50jCbOWxNyfDy z!h2IEJuyo3Z+DIGpVMEzjgP>7stE}4h8HWaG+ z=Dj6s_0D29PpmQjKuGHA)QJshtEC95;Hwimswt28+Wd5>6Jsyy#q{IBO91znqn_4+ zFstIV64sh?FHF!q1294)SO1YJ<^ZCArTj9JtUf2<0XzY zr2Y5yiJsF%zLQK2OVL&biw7w+XYvvP@ucG3{(cFbZBYKMpd@`&X+l1Apuw5+Ek94& z`VeEo1f;MJRzxiZQR*AQU21yI?&_U>p@1Kis3e^96@Otb`;ySX zrGCCN+0&?FhzIg*)&iq3m)PP4qf25ahl;O#-*@8Q235i@@>1(7M{Z(BPK6;G50szI zgS@zm8VHWP-!@IAuai3{zsoPQ;cE`%$V$s${al)(U@`gro^Deq(#x7|+gtAsL1WYG zUNhHmS>r6Jq_J4B+`M*%gmjgmNn8uSW(uiGOHa=S+NA+HO!dXpCpz z383gZ)a&1!4^xocFClGs^em3WR#dBR z-kio9gpf-_M7M2bAtp^Rk0v&a!}n@i29zfi4EBB*sw0CtoI@iadL?FK)6UV{y z(dD$j)vJ{%GD&R@MQ4(5Fv0fE?Zlk%{qU}{2lLqJNRwc&_q9|cA|Il&ZQ=VReV#Ju zmT^bPQ)7oatx0pqs&`d<`PgHNULDn^D)yO7IZd4OiISWc8Y1YeHMmTJ2X`IL2R%B5 zFKdl}5;%>>yTgwbM`x7m92k+sj$eXpz8{>OH-|zfq!NrcyrPV9 z!8Ys^)$r%5+k+ZZ`%ATd;>Aw-_|JY7m)n15l$Tylj(BM)O0l@=}a5UYidb_fG&sw%Sq+%K?bY6c{ zs2P{ll5J5>IYM=IxoL}fV+FU@WAQa^%{;i6-tJ~b6)|4-mUAe$iRod1y(r6@c@rlw ze0gzX_3f4Q&DRkh)i$$Nz<+XetIg_h>WR6r#yv@!&^K)}xZ( zMsx^s#lu6E{&jAa>LdH7NA%l`sz#Swc@=BM1?`6fwj0=j%E`wl%i9V4>t8pC_7c>5 zys>Utqd5-P2e#T{^_R4*_iN!|mhybBBlFG}dLz-8v1EOR#tSc)>AZHxM~gXd@N+XK z`;D?)gOwf7r*#Y68?7LV6;B@y$#fvb3E2_OeRw#yw4-7uu*HzU=XJsu9*r;No6j>aA zpZ}6&N7yh4IlMR@6xfk7jSbt5+zp%B&Wdz$kKN91G9-2R38g zrr6L_9WQjmot1X5^^2sw*x~E}+Yq7NKnx9xU=zQM{SPQ#eF|T)Qyp_UUt4%%Io9aC+;H7_n;oP%rIYsZR5bB6JWF?md z%s;yN(qVh@K2&RtdVvQz-@zwnqf8^w!d!*lp4sLzzT3tSpcH=g^1S*=iBdz2w<{$z zz}}KHmGe&QGnd@R@cG`fgXAKM^M(loXiGut@=P5i@8Wn0%RI9f%19jpQ86Uk6l84J zVbbXGM7$EQ6*dsv8kH-@NK|I`(S-3r$o@OYP9hd-+n3%2MV4+SI*lBSiGe7bjg5LJ zK|)5-mC+=s?1-u}>)0q@gfS+26@j{!sCR<$lZCh0X9hEgW2!)b7!#Oyz?!Ea@@QxD znm&JzfjgHO=73Rn&0gDC;2K+dSl@<>`ZxEhpZ#1_RI!=SkHOyK=Mlt=a}4 zi>Oci+jDa1*GIOr*&`dv*JrQ~A_nXboLhh+rt%V*KsW?PeOUNy9AZyfPa zfHw{@_8@%Zw;M3ixVGjTgM2@}gKX{?^UOYne&`v^*0f>$GUMw(%tS&{BRphz~v8c*RhHzJM+B~z2^09gQ83XBh(}p z)ez>?;39T&vjgHhO6W{RDp=eA6iDp!f#v72r4DQj861ExCqDjr zP2jxL!|C?mU9=`JvSmk7aQixOC^02T>a#6Oi2`Hq;+nL6*ss12`Te3qnb9%4p(SR> zrAj8#>rr^Z6p4+}^eCyDnG1C+ZVPx_D_VqL>ipEky+Bls&8anXp}qSD z45A6{gtCUAgyeLItob1@SPl&|Wd;0p+K3h^%KeRRbW0WS>G49f&5_lH9a7o#f4Y0?ptzc?Z!{1H8r&J&Cxa8*o!}0^JtQ~;2AJS5 zIKdr)Lqc#37Th5~f=lqAVQ>g?cb@mWkDPPr*8S&Jef3ehre^5gYp-7Gx4PHrUb}ma zHJv*%M-JEhqM)+Mc1c9^tf#wt>7Wzo(l^uF9q}@sC8w)r@rC%aFGl^xOf3VMJlW-B zA~gLJ)XEm0qEZ~cHN9cJJG5

R`f4^2$KW))bdANXU5){Bko>J~`?nRhBf^ytN9$ zFZ4cH5JkA55^~%Kqpo6OP{>+aCxWnIC{wD-gaQT?cx9+P+iy|glDq|X5>CiToYpSV zN>@2N?qrZubuq3VhF}Ljym8?4O&2>)&*Ebu&g?=VU8kI@mvH-n}-u9E__d zE_Hr%zR5kO&$@3z5Ez!SgIhmNC@HmK;##8rU9S{4T%+TAd{d4JJ~Rc}#Kn zP)a-+9}^!P6+HwQD!RmXP;PlUk4fPJ1GT6G-Qm$?5`M@(rdE~1BEP(mLQ#OwyAyR= zI--j+P~cC)684dvQzBm)4e#(~7cO*H9V>b8eB`Ic&|IM`^moCFE7mS$g;8`TtSj!k zS|5AF`KbS4V#OFvPI>wEYVv4#DHeA=YlI<;4`2qfmbaNhU1~MhGC#kmnKcv?8dvmE zPc0{{lMR6h7xwlb5QtIAZ*}AC_*lBV8tLN%YYtjTy3qK!_2l>B8oIbM9Sv2>3}OQ~ zRw)<8(YOf3F~kQSkUa4jpe5pYko*RlHzXofS6}e+yAE%(afu@Vy=&#GiHmkA+oa?i z+s8|kJ(N7qIC5!CqW$Lq4NFh?UgMe{aTqZga+~w20;f>FtB$j$AS;eEIjP-`Z!!Ft zJz(=pB=ZSHnWICAow-2f6Hne^)}ZsdFWf&Yeol|>*Ayk@fI{U~X9<6lyu4Bty`05#8~cT$pY=RMO@PFt{|L^dx@<*8 zN7NfxT%C)5TGI#<#}!3T$Pd20IX;dM)csLypLXka1M1?ddI8Sh{ye5Cvh$dHzaWgo zvfe-=_>ueo3eh?D2wobAXiO$_t@)d*s%X^MlF^UD1sMfrf!$c}dj{gQEYplk5kzfV zaN!7cf@O9oEDi50>gRgI8A-c$jQl|grm!d9#J8#L^U;NHs!V0+rqK1+3Ja`8V0~F# zlgI(*Nk(x^+_(IKSHRS*{d{YHUCKAE^AUpGMn}tcez;;LCi}CW(qAahy|2jcQ{UJi zJdTK~ddN|n2on% zCL3#+?-BSicK#Zxd3$w5YXSODq75ecv!sc>8oXcff`n*_77w#pU0ngqncXO~xJX0d zyA==GoZ{H(c^9G@sQ|igRy%^Ob z`6jj~Y;aotr`267Ydbf`=Y}E7-HW=3Is1b5ZXy+ktDae%&xP&>!{twFuXaL^9l$i> zYxMpq_~T+7$=A#FvIle0U!sAe*OUX$GZwK652vf0ImRQJoJU)?XI_gZ-;w+7ul8h* zQL#>I9lwcELtn6UkF`ZSQRkGm3uk9eP2?EeWIVxUzCGz*^xhVYymJ#T6Q=q8)7f(K zN!-cL#!SI8c*j;3H+KN#2bs=OA9N%NG_qpgEXmTyF7doru&IfT`XK5|;(^AcMr3lU zq5jcDdO(+7#StRQ(^fc&hSTmMS)0sh6WFyyPnY2bmTT!-`*&p|I?3blK~B1w52=HS zBZxz9u^(#de&I{T{o=4O(i%cz#*2i8hSJh-^7*I#I2EVf(RWqt2|*IJA0&^TZZN&b zXzl0X)*+6n0CKk`bUef`TAc8BR8^*eEuOn>8cuw8fJ-COTef3Blt^oCZkdmt`e1*P zaMVbzv$cw=U5F)PJr+3Iv>hxU&KoXX?lska6aFx*dr^v|Bb$PqjwFq4U0FMR)e&SV zsUd3<-Z2jMN+asa>?94-b@fLRi70XC`$p{c^%@5xF3BV3X)4~2&7MEE>UaoZ#u;LH zGSS}_m4ABF@cN()rYD3RM7ytACb+jP(U}Hr$y@2ntUWFuo1T?K7$B%Ug zW%Bf_j__n#3slp|_`~3eqVfJpZ+e5e_ZzRem8|^(tfNb5sL!?iPHZRb^7NE^<3cgr zlRn=2UzB&3J+Q9Uq8mlO^MXW#5ZiR4r_^g|BiVwsQ*zSeX10U~DWa0;N{8qu%!RkdbhE(c>0B+7`D2X^4Z z5iGP|5P`%zlzHC@myo#RL4})j6|tx@zo^xv{gC$c(^*|V?Kl>hOkqiK^rDK}$jge` z=?m{_fi9AECH*4_QtOF@j))vZBDWjKjEXVU!kWlI1T)*%h}~_%oM;tFxL0+94SQy; zf6!uS@An=^MGHcTzeetH>dgXf)9D62bS@d?03Wwo`5Jl<-*G~LcWBT%zeP4Ke#3Yb zJj^=ca*}RNOAa4tbx0Za3)++{_hL|eW#o;BPzblt<^CubEU8Aez(t$=XrXW24gY;k zpO@dn=j%=9svj{-kepw59mbTaB`Ym&haULKwfYlQj^-%zRmqSW=7c&LLV2;s-B0CU zZN5)Ce0GSRUo%&;z=W8x+|S}zP2I2>mk{XMVPn=BZ?pUQa?%}{lwH{Tjx zVEc|>J}&Ta$zVF!!hglKel(AaoSj~7Z`|`FlT=ge0EFofezYpI{J=YTz4?0ke(Vw! zMOBzMVi-V+j+R;6%UJ_{8dQ063T>7AWJlE~M#XH9DdpdhOP5D&d%ONqGUG6Xn6E)Z z^y#-0}QdVy^`R#F`>yIIbU+P;n=`MN{m5@mx4Hx!MwkN(`maDv;RnXaIAq z()k_BBO4qMu2oy=<}=|OlbzMVN*J85j|jmpU??QD_JRM4m3)F#pi=RV-PpD9;qEQ>F)M_4+&XEb!)=HfuT`W^;8yFs3olO3-Bhn)W&Xk8KtG7bpJiM^-f35{|K3p=<(38|Z#-7aoXrJTd->5nPVAEtxCjW2_d zS}A8HQWQ1AGua6a96fxo2FB0n(0^ri7FrcfI}xdfPs?!JZbF^l@f|C7_+1p=Is|hN_XbXXU%w2RnV96y6_myRVpTUvmPpeXm`3J zz;GL@k|tql4G4sgc0x#*sqIKK5^{T^BSCH;+T@L$r;d=r@MFU@TJ)eQ(zEh6@sxKL zt~ZI}YmvOgTA=SHfs1)bc47m^x}zXHUTm|PR9)xt&+kF<>HTtMQ_@Zy2+r~kqM^>F zg27UT)C~yIB;GpO&YNCJK`U9hx5eJe!{sM&8$s_eYd9}4H~T#k*Vykw!}M|Kjkj&5 zgEl=>g_9galom32>lT`^Imo1!1S)JlaXTbOB8J}hf3H5QHugqvC!t(70W;WJ4h>a< zLm%jRez|Ijx(TQ1(a6sqH7tkToVVP|+%lPM^*onAB1KEF=8Vc|&wO=1F~XMWpT5qb zw_59+NmXe1!+esH-A>mm3n4BMGJ_Z@kHOB-RESNsJpoew;RkCed;gmYuwci1eB?cm!TWIr_H>sSqI);yO$y4O!ni8*4KdsLd`+noSIn2=) zTFw8EW^5AOX2IILGbef>|Kv=2IFN6TWTc2!LnhA`cr)>a$a{)o>j)pNVYu=+sfH_c z>6H1vq7=JIKMk5DH@?`Fz{GtwZo{HQ;4urPDlRIc%5|pqKGD(?%Np!f+pPa- z*8g!F)cK9&+gc;ktSqMTs01z5y+?>tbH#MT~EAtx=-kl z9KF%CNUP^9ep@NpsXfdibfO&WGRxbU5l`f`@_A1=gX}d_$mxmP!ZB&{efpWo?ad&1 zkPF_fA%dW>tvzRGm6TWT#S5d3kgH6`;vBvZwxY z+ih;9{c|B5YLnygYM#$NfB?#L*m>l6C3&X2x>oA;YCd$9Gu{yYe_|({R!n4(?=3fiI0&Go+#9 zFoZb|+|W8MXMv&l4>JjFIHg?HXUg!q~%s}+nf&tdF|Dz2XuJL;;1;mM%n%p(ty zpmX>H!FAGd*j;}|93{6CzI9xweaw}qcjBMY5X900uXaGZM3&jJXRSM$4Lhj!YB(}? z7njJw>)Nb&VQ-U~P=<0b=D2Zb^)k)eXci&+>BatjKe=YN`Xi&d zi}UIxh9SF%U|RI9WlJCL>#bMmQ(2uoDX5FL+j0)t-UEYKT*XP0~}R233e z8sFM&pSvz5!wGoav>sh%i&izk!^)W!z4o3p3|6Ff&h>wD8;Kyi+{wM8k=PNB5Y{Ig z7Zr#L!}6ClNukl7CL0Q?4GtqzUn^T#?)ka)fh>7pa7rCwJ!`M8VU(rxr5dUaeipU@ z7a?JMBBJ=2(E2OB988cTc1C?*Z`-5-8^%G}sX;>S*O1}6*WrG#^{C=*HzEl-SNSP} z#BC+}Vv34vcxVpSz59cMjUm7G_%+x{fvRs}^W<_#-M<$B|5odLSKFE)8uF@G-@KlE zZMtJ|eDVbbaTLvVicMT?G7FqrJJw=1d@zfD-OuUr_Gul>Q}UnJ&7uowqjF}s&wx{4 zg4nd@aw1TL+>ELHuh%Pg%w7`}+w+fe4sYcDjPBh&UmHdoS9ma+Hlq2_^UvBF?0&2k9tm&)BJil$qje&*;~Uh?@mg}1rPBs%k@mX z*K~JF-o(rspHn#Nk@si&fIm35hE|#)xUBkO=@$;|Mvs~5irj=wKCo!5!kPBju>3V2 znl+i@thC=g%K30b*FXLGWWn7Zo769}GTS%~j#RNIosZ?;@BeXWWFirscnAyj=cq~d zK?u6N!8^6B-n~-rF68OKw+TCFu6>s7xyVPd0wt8#b+WuJbPXbQ4+CVUb(W-eV8 zh08!TP_omj;0=92(jSx2W0>n3zUOo!l>H&Lcx+)XinBe$ZZR~A;%EX6_(a)KkEhCD z8@P+7a6Y)seCyFy2@KUw0cG9F7}t)=HVM{`TA^a%(uQ=c{6Pqi{&CFO$6!EaUz*S3W2}sdw&cQwZ^^$0dHg6o&f}OQX&%_f!`X zJCxnK`qZmoGtil@we5YS-L_mi^?O&Fl@;qDgWu{n-$ZPaZo6)Bs=fP+`sU>4cO(B7 zTTeRfm_waK2V!vB!dvbbw?y60+}RtBNp)Pmp01aYd-F_TwU3>3Tr(-CS6JR=^}jf! z8PbJgciTOz<{JIzxRD&)x!p8tT4VLJ(b$)_dXSXI-G}`wG0x)j)y%QKKSE(FOH=h^ zZ=BcH>gmeO5cCFyClHJ)63C{0ODb`Ta8BoL2>(`_W9fJ`c=Ij~h156I?hDFTHbSuD z{wi)3>hSO`rt)}P&Ts5?mG2njnG!i6`0J-JsW-WRIRSn~3a903BMjaSzDMwfsU_A# zb}E(My*hs33dd$XD}7C^KCa}l6&G9d8R+0dx{H@dNSq>h?~Y)Y3kvH%KWyqHV&U1( zB>=K}DXM4^gSLK7fUDrmk-YWY`p3b}>T>ev>0|O?&L_{VJe@>%MJ8J(GtCQ!Rl{s1 zb_|+8Scz&=xFJv@J2PB?k=;U1d1@wp5kuCL>nt&riQVEK8|&dr-XXv38=kPOyF}8k23BwKk7#hI@M7oUZ%{R^v@VUbOf<@7zzUwAgMPt+`;en$W0w;k7MNO=4`TDmP1u& zi~_TxtJ+=J*u2n1h*|!q93H|j8`_;_p#RM23Z41uMvd*NXhv7Mim(;4nP5I5I(LSA0o^pf(}yg=fJqADWvIA>&| zumS zz+Dr=@V>L_WgIqmx_#*9t>X)E#E!q?*v|sX>*988OA1dbs|)uGtjb_~GHK2ApjUD3 zQdM=NYW5SDH64;yD;B4rNaZLNN*Tr9HXUq+Ct0)v>BLYe#P672)iCij6qE7a(pQLD zfuu6q`rYD9__M+=7zWP_>ztkPSZuYm{K-;{U76szb{2;AT_#jb&*Zs%6vJN_;i{dn zGV6uRcBPS-9AJR^WAbA_)dzX@29t|fmGm$Kx;erBL9W`y{%>!&S zV@<2XrytGw*6JNm!c;aVhsTRKX4GvFcwxF5HLkq`diAobnNZ}^oTteKW}LdfUIfoP z39)p!U7>6^&Pt~f1m`?Xi_qg+^GUYrUp|8$$YkXF@Z$Q>I7)Lph`NW}5x!!w1pGyh z6PcEo`pM_jvXroOmx35qY((b?jm}% zV%#_H* zWm>UAm_{h#VyM$St(QS>ngcIS^=-LO6PCm>&W!$2mb2fB(0=-oC)to=5$K9dTruN> zbINoLXC<3Hpn9zu?81r7eFSQR92CPCWV`U`tNDb2DFyAeh(Qvc$WnX#2%TOPx|#$@ z7Q>;8zmfZg&6VZ|#tf}l_NjmZE8hs6q*Mg2aT^~MDciH2SRw5wvP6+o6m5k^G-XUr zCZm6^U6<{~r(Z>ncR^Vn&2Om4xfxN}Wq$g0Ze<|fW)EXpbsdW|Tgjw7^J9YMbK&^` zGMqlqkzYrks2 zV3~4ohAELu!8fWo9e%{P_OID`GO={`D|IWo3JbXQM>@?vxB8ky#M>s`gEh?mW%6+S z%e-Y66Lx3koC*zr`CufoUkCqCS`B^i;~J>Nq0eE2>>%fWh4g^M&Zno4aJ!RcI`7j4 zez)5$){o1y3Q3QYJiJb3!vxn~9kYdB9GCQ=E9jH*3`HE9Rq{_vqgkPCYHTp3)#pd} zrj{I`P^ppehaV@=b-W8QQyD92Y0DHzt*>e4?DUP{z;t!c6{peXqv$>G1bTW5c=ze) zX%)+&lR#)_C~z1(OD~v}l?#Z=@5!xZbGXACqq!Z%$-2*}efPO3C@3x&PtVTcRhThC zrO|y=`;UcMFWo2;$L~R*I9*7rRJNUMaW!SOqP%^NJLU0Er0`HIXr4K&b($@YfAMb) zS`|Of@d)X~m5|b&aEWy)?RBIBo}^IGyhUMIP3F6M{PXUuzuD|i4n@{#-hA>_i#n&3 z`Y1@hbG-PQWweE}Z+&y7_MuOV2?lB_$WS-nLrJzmCVkH&bzjmZ5A#Z%s}W^PM4fxe&kYkiG|e))FOhL$C^MnF@q<8jb?U`;BI{1! zCY|*>C1Pr2JSuYyyK;b80Dr6fu_g}NxJ~JkKC{&NhknI6Hrku&y#7nf%Rf+49l6__ z@NB4;Q`Bl(>!VWn&ugEM#)8*pcy}+GghfO!P>>l3Fy`myU%Y&2sSdnfEFTMs7~ft` z{5|PGyZ=gz{lihTV5BJ8>gS|grvi3{;Lf7ZS)k~5sZsSetR0XSnXc*idmU$%18RZI z;AbunK%OCOl`Nq3Jb?@{JiE>?5)y~+RGOxywehKQbH}Qg2JI1Eg@%)@=}e=o1*sig z=f@Re)YL!rEizSyyBto2B5h4AGSO@G6w-PFbmotG63);Ce?Riq{C9>i8?G*-k!0=8 z({bJ3OaO|JfWoP|d%I`%j17xGt}r&>pIt0N4WDg{3Ma#Yvuy&cj6mfMzqS0gD@0v7 zWhRfA<+GjCmjf;c=RoJs-%9_01a7w1vUlN>{ke|j=V1W_qM8z9j?4g?rp>S1`(=?3 zi9WLP(Brqcm1^%d{I|+&l&;TrU zbf-H~sN#Kq^}I_}4Fp|~qcwK)KRA)PEEH^z8`z_Ty`TFZ(f{p}3-I4iEXCdbnSpBZ+HS9QkVIDkk^EIx<{N{=*iHutI%fmIsg4Af47gmbQ2?nv0sl6Oz z2@xoLp6i~5`w85IT1p9JeC8G{AW(mwC8Z4bH?@Z(9Q~cZv6z40RJe-A2>fy%y!OP1icHs{acH;g;q-t-(B^^qd3fVucd7^WZ`94_^Wm=-dA(X zs^yLFLjx*_=Dm(I%`H290+Sd~*ok8~XT}(lojUUrl1kR}Cg;hF#7b{|j7g8S(@-DP za1m*alw%%l)P==b-B@#?^ZgcE`_Z0};3p85z0dUc_8st!O>Pu_!jU!i;Q+7D-d7oN zqGL-y~i6MSSkvv8urPs5;RjXgaylWu*<5+ipkRQZOl2 zG^ln=JO+_GH@m}{xeN>SJ@kGn^h6-Oj+dPO48=Cb3z>on^zsXkQ>B{-! z`2*MfYRqd+dznr*$@ppTz|G#AcFu0%QA|z*_amuG7Z7mU`FpAurX0>$Dop4vy7Lq-~ahb-Y zvB;4!1VQ{{s&ks&sfZn7P^e=?tfV?8QU@ddMQk;gh~tf6R=@ zc*3bH1+!4=@cCPb(v%yiR`&U!M0e^Qi5;DG=)Mb&gXd)Jkzd8tPc*hN&@&4q2@$1J zuAdQb1M3;r%kT0D`_J#U1Ps3=cUEs_4}^SjfPm4%X1Mk`Qfk;dA&@i~uNM%~+}HHF zLe!-ktM3=V{FgEYf3s30`(-+i#0yLz&#X}P@#Bp?CYZt9j$I;e8Of7n`R#68Leu3Q z_$K*TWfSi$o-$p)c)B0Vm+@RJS3D4V!8Y?UtO!zHTgE!tyTxAdph%Zk z&}46xEW(xRSy6Db@YPGd+uJB|BLeGGm4v4!IU9L$_G%LD6hhPuyS}%Tu+_<7XmP~H zXoTCO?tSM)rC{Ks1C0omeFlldaN?7Ekna$;P6%i@S?%b$?Y^`3{#+rYk$IpjXAIZR z@3F#+tpy^*BO&(OlfVyGjecZE_akFXEp$D8?`pb;^tktf-$=nL$jiuo2skj2JE{BC zc*(L)j>0b%n^_#zI<=!xo}nW6m$eQnh-Inab)O{W7Qae)IaH{RCS{pUQ3 zefxH~ELMwr%sLx2F^Srpg%O0Tr?tB0O;dueUOuI(#EPO~64pEZep{)*zv&^bYinVM zUFA?S9L8%eOH6J`IlM^+Q#zvCL2wlT^uaCnJ9^0+f|LV05> zhLHpk7L;f01KT&{&)KKLv!|=~Am6DV*)NY5MDOa~{5=45gw7x?39-tAjw!;n<{j2I z^jxod^=lPWf>L_Nk5eh{)n48h8haSCi!W;l20FiajcinCM2eVMFthnVzG80a{mA2l zI_x7P1nJEvR(eZcZ_>H~;qh~abxJoBvtP&f`+7CWbB#+X`rmf4B$^2-zYXI+NOghL zqz{R6O83|L&3An*JCMxI)L5#Rg~LaLECz2FdB?9n--Jv{*?O1j0`_R2G2;SRvK)Vo zs$JrnT~=|7FIax?+H2V|P1L0>V$(R0zFm8`v3iW#$D=tZtH?5ARbFuPwdq~xQfc40 zwbkSYt~yICs_)pHW5}QIL*hDryAKk3hG$?-fmD~(co~Et%!65OxD|dmS|69-e%Wm7 z-G@))?Hwoh#MCd^@WiaY{;);#;K87HMh1WF#{CUej9F7O;R)3wg820Oq13WvIN&!# zx(}WT!uZKEw(b)v$`R2N&jX^wNDWUYyxQx+nWuUp?WkzwjHY}nu{wrvDLFTGPhy!Q zepvc4XXm@`B;H*6JM~lea+`^Jfj8-B^a^R5X=M#d-#X;NKW5?C*qCa$&qw@;0NQ-O3e#6PGJxlU%sf_=P++ zar;90eQFH5YwPRa0ogxY*V%2Yo~f9SDVwJ1s&a_2UQ$IY>mW+YgerJGNGa8F)!Z;$ z5;tOw4S4t7qZK!G2I_ zsN($ORGoQG+O&GzvHo5H9*?S-!k#uvawr)dNhdRi(=kY;&S^2S%A#eaBQ5WT@pJbj zOaER$`TMX*W5wQAJowcy`Iq4=2bwmUr#h*dK}u6MeYLH1+yixNgoxMdYq1h5 z3SK*!S;i!GcYE&MM&a|^GGxM@^jG7Hy$2V#YgzUM=#@T!EoOCsyOa#*K%Dk3RP4L6 z&1);aP*Ev>^ha^R$b?RhU^S`eEmKN$zQlv4=&e?@O388Ev2}QAy>o5#jMoBd7@Jn4 zdRBHxR{2qi^yoCHzZY8Whz>1Y`wMu70v^EomIyoQ*2h1~2nEZT;TOrfR&)FM-d3tl z;ww53nKO_Ewo>wb-TQ0j=mW#9k6t&AXTDgKaULH@86V;)yt#|=SbU9p|LeoH;wN4* zp-r7JK_4O{_;~TT?Q$Kq)o~59ayG1a(RGF}gxN1=7a}g4m~TIkxOnv(pP@gxzwL*8 zj>%+D+p4Qy$0a$WK`115=~{L7J?>_Wa`&R&eZkF5^Dh@vEHbY?QlU2vbpI;{^e(@N zsg;59tr@|NeTVk6Fwyq9M>(^mt0yOoEZ!+ z8CuslP`hNeCDbUSE}P1Of{+1m{L1AHI{*UiJ|NNt0`8yH{nQ0{d4cNEz?N8p)(IM{ zJuIV)1tIGkH%J2fKe^cQCs+mjU{1gSbE0g4rVH9pGcd1@9RZC_+WdYdXh4=w%{l=k z=#GV?V&!rM$ON@3g5o#>`faKoCNdJluAZa0IxO2xG7@s|CZdn4jA-Emr!0TtJ-O_b*|@Kv8Xg2o z(IP>UE9}7gy?U2`4|5+`$U`kVJC;Dn09kq|BO^*xRn`3nN!rf?@eGJ77)|bZ;?zW4v zSv|j+i#T8=^Qe82vd;7a3jFAzIXbz6h_=%%xQqIMmRAvpJFCHl9HQ+HsA>K}M<89c z8lDo%C98^PWW-5)nzcmG(O?b!pTWn!zE}jr{L8R#?U&LYJa`M{${$~E~ z*ZG&Dtks?=*Yl)d8d@$NY~LXhP7Uu6UWnNGU;XzVbgJO(+~<@qojaO4 z5t`(V7xXltzayHXe>m=cJc7*pFD3npg{}vb|9n1B@zEcX27fjcn6Su$Q~QL=(s$+v zAJk1%IZ{dqR{hQz`}?(@H)H0XZ2;-Z5#CJF{_k~Q$TxY$k(kFvv-Y1vp3i+<4yJ_( z|2rX=G5H*Jt{)FCxVt<$Nt?`Q@D^80M%kAJSjVvXH~x>fH#bPQ8Y+Tv3@Z_nydN-h zzfNd84=0tnFs`G#ML0jUQQI zhw=3ZqJXK4O;J|02mWWq>rZj5T5%UnStQsuB+@sq$q5J2!Z809xc?y?7=+Dou=kWn z%-S0Sx^O|RAfp0OiT_-;)x5zpCKJCnxwJ z!Kwdi2!9nn{4eDHKSK0hQ)z$E3gY_wZ~6h6i1a(W`samR_5Wg{f7SiJleK?0Z1gUq zl9rZC-4A-zPv++4baZvA8~;lk7K)hT%k{WghdG{x?USpfo8#fK_wOy8tN&e;ArI9w zHSu}v#tQTEgAPpmt~OP&1moKV|3fo4im0flH<$eq#jUMbBO@y9SEJ567F{Uk{D8Cd z_wEo_elZ7d)qeKTx!P^>Szj#GaQlBk1g&my(HNM%p4I>V9Z|n+Wx=&&smYiFGA#WM zUeCpI$KmUO!0Fx!n{LgQjKA&Hp?~j;{x4VoH~&)OBD2P7To?`P`VX=Ep9I*D+i|kL zg2g|~2n`(=EY3Ntu>WaNAW1;#f=6UlP+)5JpURt!0{Wy`iD+&Z*?$zlK~~rcwGb?T zB(fDYO&Bm}{ZpI_@C&X!z?by1jQ(dy$nA(c(0>JUhh6Xh75>gmm$)qh;Cw|AX9T@OyGw2$ssc`Geu#2RT?_n9@a<( z8SWzM299mFz&0Q;Dw=x4WzZ1Qy8vmZQq1IgRE6}=ASIMLj6E7@Va?=G)q{ry0Z70d z$k@)nA`9^4h{Wm-!oW!`_msg_RKpeE2l=|c;sZeOcm$NlT0HGE5j3<%xCTAuh8ZTU z?mt@(cgT&Zrb)oa#PDaRLrfY%;J0y+nY5*{8ah87XdcNf?=;FZ$^?-&j8^>b#a7;2P~op1X}R9qHFmGuU5d~mqtMG@CMhEYNdnBf1iX82 z#k!1vj3AON)j+w~TNzfDxP^Dz(a|b$JaX0MK*dBEV3*_oYTqCSLJ6hb9UouNY|=

auA1);{p2P|#kNojBp3uxMtZYbf&&pk%oAfOdUj+EJ);)o+h2J7Gi zs<(g&O85Je*?GI~31}OkEHt_((Z~d@10SV<)-Xu|0$XM!G6EGCC&`!#(8~kuKt(~` zFnOmDAS-Iinv)MDKqZRI)DfKYlSYMmY6{WI#)i9wziYzi(R>GE1{32ujwninMh
>MzB#ZdzqwVH%nP_kmYic03yip7%x&YS)IW+ z#BZNF<-e;z4U}HoDAsZ<-S@NHl9CD;qC2veaE%TgIy~0mWPL9L{h)bj$+QvyNf#`TZ zyCm_;0#73^M>7Z-O%Wa~QG3A?ZL=GsU2BAw+4kV%rb|1Q-qYQ{7M@CI@ z@XAPr4h~_Hb&`NPNmRi0itX~ zfHBCi3j>^^_b@r(GR11JN#d~rc;J;m5OE@b;gY@8z1%o>K)Zn23aD}cLv&hE&yc!U ziHVVr!JNdTa^_ggWdM?Y8x@^8X0LMZh=%4nWY_&ZZU%76@b91RV1P=UkcnA|ge6hX ze-|i-5(5TM`u%}sp4PiBVGL&A(#%7H=AIvLT@TKHa!oDN%coHz)(vswSYZ8w6GLgXrsGR*M3(O z1Yiu%i0sA>T0)gVkpnCj)jPpc6GMXx4xk%M%n3f`cxNe(4%RXCOtk{6Hx?aP0bR$G zUKWfOo+<5$60H>|s?-AFl@a|dv4Eff%|K+jnwAl4Vz7c-FE=mjV?OX?;5h*9LxAj< z-gN<7FxXTl7CAIhrf5xLE+#!MMP`!~=r59#5s#CN_WSaXHS>a{r9z*DO1q}VudzwC z7aN0ZsdaQ%Q<0NpECgaBt!$yBy>i|O1#1B0bihQBQA5=Lw{YO5A!Ps{BULRo1j$AL zIzS8HL+qaGY2-jxwWcAIB!CL854cGH1h!VRmPQVC%9D``jl`5iNcBb&0oo?+A9;Xu zto4VyiHGct0m$cRA)unAMbH8_9af?hjX4wBut=Ha-<5}oF2exoGTsT8=u||QfI?zB nfStW=PT(|`SKGO49QUX)JUQ~mSiEHqfIlTUHQ5Sj^Pv9+=~yV4 literal 0 HcmV?d00001 diff --git a/docs/images/quick_2.png b/docs/images/quick_2.png new file mode 100644 index 0000000000000000000000000000000000000000..75848ca09f7fc7cd9b12135f7d5bb2f0ce6c49cf GIT binary patch literal 106263 zcmZU)WmsELw>64GfdWB;I|PbraCa$Qytov1ceen=-HH~6;>F#icyTWd#qFl&p7VY0 zxi`P|v$OWf&XO_59CNHF6=j+Cs6?nRFfi}sWF^&LU=V-7z`&!SAVKfQyu%ZMfuVqr zlN8hNggxm&4$MXLyT3d?_saG7{j|QGHWsIUyB>ppA}&e+iz@|?1mVp*Hf!3laAuL5dK~?TO{b*@rilo>i$X%lHz=b}shSw(?N~~yX zbg2=B>i!8DtMbodTLvGKAZ}OoG%g1@1^1rKb#_^19Af-Aga3Pw?mZn)v|*`cKtZBd zfd3k3p&s3CZGeeZjwniW5v^Bg!SKlW@214a)ppURhKub z2)W7GXtn6ujvtVSN>(^^-xy1r?NWA2J{a8(Jc|iBcekLB)H&snRuaIZ$J^-+$Lsge z^J-3Ly`EI@?a(vhAl$A4LXM%U;eY*V2X97+91S)tE(SHa2tZ5{ET^fexy~3J99VXt zOhHnL3Kz{rBAFuA84g}TEQ00@920D8qVBzl30L1VR5OK-R2`eai-&JID0K_x_ zTg<+HgQX}%;(RT)yMz8$!{hMm$SEL2@(z5qM<^7gJl}) zWkb+(G$2@Qaxx%*zK9oR86!#=;+H^Uv@$9b2Xa~j1O$?>#^7j2ess-XP{~1%b1+P< zal$*ZEh2o14oS~Mo1)zJf7mrlkN})0s3A!7=0HmBy$6_38ot#eRS>h)MH{I>V%Ff7$V=n5Jqs0#f0CpwKxZ z(U*gK6X&l~z!q@MlPrlheq8hA;5(DBVKP6mM1aeMpja3|c_HXBZf$;|VBcc)h|7UIB)TF-N8csXGIv!GxpcU(to1}!SRZj!OK?~{V#U$R zuf)3EbB3zPVM7es>;D4ZOT%W@Bd=IOq0A)xMf1MFINO!RjOGw?oou&#lP3E$>7hE) z4pqobf+Ag{WIe!mxW1uC-*f9pr+(K~I=-6vYf?;6@D2myd+2*M!K|hx9v=IZalgCu zsUUdd3k^by=t4SIrN%mHvQtjG-?bbn)+#MS)W1F;;2`PbAO*Gb=oR;0+-tB65Pqcu zpDX?rT1{UeIPj6=JIv@%EtH4s)HXJRG(35OOJI~oOc5NhB7dQin?mJpVUjWQ@T zNICrQhi(aJ9tFcCx+4>C$|$HA(EE%Myit1<&?Qt5Z*W7Qt1nz4o@6_2baIjoi27BL z?m<0Rr!^MX9J#;t{m4#4R$E4;pAdnI01p`uB33aVTJ?*;{7pv8ZTht(SwujwEU4 z0dIL;0^@nS@zrPcx2J0ncB>s3(3e}N%wXy=<4Z|NiA1T8|MZCf^Y7?gz%|VKp?jgZZzq}Yjyu1(XyG@uKCKcB zUwDT)-iJJIeEdA|m&x3{hb8w-nHZiSJP&1m9P?@_z>AMtH6lcBHBt*gF?WR$@DSh3 zv2#Uy|BO7f`5;iecW`&zR`KUG+#w)le)GA#$7ot6x-;xxB%F;%%it^yC}8 zfsFLBfzj;h=|X-?syG}%%(l8+kbOh*gh&-0J3!1}TsH^BdPmB<-tAnPnjsaEv@&7C zfg=5ME#$|9Uq@zexuui4RQUTT;i*L2^eqsM*#l^M0OC&MotItCKA}bk6R- z*J+7s^(k=z7+-gDOs{6g%H>XW!rluUj{ZlG99k{=_AT@}jfGZ3BtA(a#g$*e@5S)z z2u*VLr$99A)#B@7nJ5~b)?iA2e3+28TqgilMXuLp0w`@y&w zTsEC>wdA=(w??q@TmecmXv?lhOnKf!7AbsiL@QO$RK;ndUuk5SRB2p8z$Ky0Id^gm zirab@j$Ghg?CTfVA%S1mdo(-=ai0828%rpF^h-jr`x^KS6fJi2%A8&IY7FO=lt|na z=rk3;>2_+$^vB6cK0QA=+OYxw#fN2QhyCiSoLV?675WtxoO^a#2?0vKx2%*Bx~2v? z(mSL5WE>EY!ed|%XvnKWH5aw|cx#1>DnF1BgaraheXB)6!pI-};F+typVMWoMV2NQrHt{$=Wb?cDb#K?5~XiMw6%q4guY^DH~KvYh)ame4Cq3}PV^k9T%(K}Sh?hXqhh+u?7iR-)RX zpb8C!#e%4l3{fXF)xqKSYwc-IXYF}sXZF%sab}Yz%f821pAQNm$o9MJU%3qo46>tB zcwd53crTH~$J_38JFkw=mOY<-_`WR|8ZwAWog!GoqEmN_@3(t84_dbTMMGP@y|%@+ zHvD=fFjz21-OtA$FEts{C7=c-zMv+^*QGAHhJ{}-P^@jjw^ddeM$CE^V`-~ zc4A8iBKafi_#;-=^Fa;U!pbPcRGnP0+7e4CO|s_1HwK);(XqkJUx>ZjzRxaigP)m1 zMn(J%krr?jcWpLaD7zkDnVt@XT(ze5lEwzFmI#FV;0^q;o#qc7&NeDH54;YX`$WjM zQWTx(a>npns4D%hgx#Bf$&yQ~WUl>JpWmrnJbhzWaH8WNDv)drf$Hvc^~wtlw;ZGwsKHD9WdZDM@|b zQDMZ5RvJGDekL(sP)0#;2CFFK6QYLjzvZ$nz_*G2VCA9T7rUoG;LoRkpxlLMnn9-Q3;P!9`+O zj(K8bL!wyWR00B7#3DBb`pzR9!9f)Ck)nb~Lo1Y6k_uy6?~sKq(655x#pU?wT?t9n zlwKiy2{56Q>T{Cv3O)6rc<^k2YScZ4lYYMr(x@sx!ElABC9oz}RC{r%xGIR6%50J- z$hboxl1MaalLMb#@y_$MO|G(Nik0<0Q+UvQkJK2on3u?6Q-YsTA5^N|9=|SC@2ZNLk9VNQ#mGm6!WBgagJfB@ zBYwnPH8tLOL3fr!D5KyHRm@*7$!TTY6mY2F@8G}{Wx>|L#wZn=V6m$>!f(a!S%!v5 z02+O+@D8g$kpZQ~xn-whg1tm_)$qmkc+amQCKmA}wIIpDhA_zv@*jm}1d#4i5pj-` zKqKMS0c!T7r=ue|7sj^L+azH6N8>#NXgWBg5++Xh#m1(*yb3lbJKJ{q65Kan0@O%O zb9uR0ROM}h=sEn1qtEk%BF?Md6CgBmg-9c1t=-*pm8XvZJIZmScnr$*n9f>Wng#z( z_x-)OahiR(Tbs(T%o(8<@e;Dk@q?zt=>=+U^*tsI9&E<~9OqVU8vk~_UiVBVE{t0L*A zQPD;FOkFkfijC>vUwhOHq<}rEq=~YOALv~6`Nc&9Lja8P<;j*iB}B5eN*_3>kpRKe-u%VD;P$cmzw-U7z}YE z7u1aCmdK~DswW0c9g02-kVkTOU!lzA$XN0H9ut_zEj=^G{0}PMq3)S0N3;zf@dZOAm?r;bJiDR3W}j?}Zs z9{f0wUQgw6LoE}|sympp+&4`!N~Tec%?_TvqHM$ol?q)Af`f(00l*X}4D}8zU_%et z-ZmMGCy%!g_(np2AWO$?|M+fj;eA3@V|a>@kUP1jL7BAY4WCffqsuB{Y5MtA{h=;Hel@2@Mwe^XE$j zw+)@o?Jt}-X-5Ohv3lTc8%v3`nwFL{4R(*4L0dZ%f9A6BHUAXykXI^hXHW(!RMWN` z%5!9qOY%*+^jxzIjt}96j0eRjjy64rdVST4m#j-M24I8%*ryYeDAbkh8*x9FnFwG^ zIsEQ_URDQ<6Ulwt(n?@b2@itl9Y@?bJde5l1y5WDhm>o_`F1kGBjQ8YS%ywZ)h!hAD(p{3iG*<^V&5Z(qo~F5Lt@7g3}C z;Hzs1hh=&-40EkZ_|SVA#-3pmvW>%p&{2?=;1p`YmV zF>@LTc~kFs*SqiE?X>PdHm)ut;xF8tNg~3-k+6W8dNDiTlht<2)h0p~&3R2GgSO<- zJl%mcj|~aBq^&(@SUfsE?yF}C?x83ht0PsOhq54@j(c>i>W(yl^R_G~WqO+Ai}+A| zkJWWQz`ERKN6fqNypf!eQg6FFAoz6hB@=^mI}W>VbNgdJ_+6gC;NvO=<(~^(eqycb zq`dV3in+`I%PwD*cCt6pkCh_*W6WcD<8BWfyB)_}{X>(L<_8se+D^wfB)d|2pM-`n zZPOI0U${2XdBYuEJ!wzZ-YI_?DE_?pJr5kC+DbS0+_je`BHa5E+0M{@_NivK4b7tN zCWzJPILWc+N5Fs-&Xx7LM`$!(!!UP>_A#CR({Yw(L6nG^kTL(_nj>|}g7NK-G`8xN zw9jkL#0w50D72<6yMIJ>3RT$`?1=k74JiQ|3xlHJJKj2rPJu;}596(7Ja3mZ#GYYH z@N?L+ml#M=6{WRS;(Rj4i?v0|ed+_~<^UuU63g^R#ygBOCdzzK0l7%6o43r8VH1K&z*Qjw^k8*B|j!bgL7l-RSiljAy2RZe%K z+>hOTT=nGAaS;zfLwi0iX_f;f)&i#&i`&M#O(%V^_c#g-d~Ht1IfTk6X6BsMJ=hfu ze9DC2h2hkLyJ?wF!UHC%*|p&&JLE{t;lfILsyRCR+T$7SMt+ts(-KZgeLPAWS(3en zI03$eG%9WFk=P^kcTH#g)fr;t?4oHnG-GE**PN?|#t8{}`hBsjZX~O<0FPe6SXnZ& z)!UzcmjYA9(i!FmKc_?l4T;*o>ouQKCvrx)?BhEb9cU3j&;v81=36opnaPdjSA=M0 zF?JVgtrieY_PHw^BL72e9wH`)A8I|w#e?dBbc*SsOgw?+I=J}w(`$2aJKzre>#e?B zvjYmMY!-mA4vn*aOt2pwAv>}VM9~op>$w?@OJ5lYGfG@i0Sq?+B2|pOD+#;&os_=1AVCw2 zn?~XBO7yfU(yQZ7x?uVi_WOp^n-k;iZ^Tl1E$~~1SVcAkTmjZXhgW(^iYF=QCKv^a z16_Jwx{QHMu6;U9MNktxXdxXYm_yd9D4AfHg^Bc>qeQYbpk2-#WtIoUv4vIE#fMi% zj1mjo1Q>ka<(V6E+!n}zTOrQ|G#gLnr~8;I31zs@tNMB>#SD&&(NTHHPzVIW{p|Nh z@(7N~Ug%(m(#g8PM~)0hNzXowcR;r0?vh?v>~xp+JyPU-#E1QB?MUyW6MkcNyqY4g z+%x@k+vIW>a+=N7%-$cFWQ$OzqsYe4L`-(&S6aeS?CV8E=wU<2q^mxtwZ3-WjsjO7 zv;7ua>~l*NtB?ngnEky+<*BN&Jo)xTo)Pu%%XuGaddIS}f=rU!fb4+x@GMcVnkZu; z9brR8U2QCcn!O}XRe5>X_OWgD@69*u>qBGrs&fh~o)OkblhF>fFGG8rg>+db9aw&A z)~L}9hqg+yA}Vs@OYz$m6(_K5O>5!=HC4NnBo?W#nUqE58k4!wcPs6YlO5?Qcc?1@ z0#Swf26LFvF6IQUMAIJF-8ndBV??eB8NRJ>SgGoPuT> z?~p<|GND66mt+i1FNrb+b`WV6Uy4$vTC08HMV;eX*6T`h<)1jJXNn0!!-tB-UMH7B zP#L%*JCZG+$M;{V`EAY`szQd43>kwo@TRW8cse>dih07MQ`g#cSHf!LxFIX6t2$l& zP}mnj=U-SJ8BMvnigWOdV^F7?b3MdzPz>FU5t(=@4Yv~fTF}HLE2qOZqMe;pBSJX-jcybI$m6dPn3D^*b{>gYIaW$QxyeGHSyM{prcBtE=Ga2Y092|x{k2!R8H-8X(o!WRoir5$< z;tQYhyQ9y}@rz`A-N0+4yS{%Se@%8UXilai`U@(OeVyb=aXdSQ6-uIpI1vhAJ3I;Aw>Psn&!F$myVi1N z4E<6~`D^bVE}f-Lv->GoRXt}SabAhWQMUQST*-&J2>-L>)ihG>vT^-xy*U4(@bIG+ zra(^54tT`mxnK7x*g){}Mnnu&Z0vR~<^*+}2l4mM;7~hV#n0Y+1fP?KIKT5^bIPZ4 z8d}>35BHE^zS~pN;mu%goe;Ftj?LyulXCg6e0RusqNJ^u?uV4tA<}z(=7%BhA?xao zs>VEC?dyn-SN$5bO|Oo=unU#YlgI4bpI~EE70#TfV#3V+f{yysiXg*3N?36m_LmRi zV>UZSZc|uE`uj|L@wBj>79QjUiD-n2ZpjM^uqRd+MJ0+Bic-M~cF0Sa?jy_$rWyu> z?ag%mQO_{7%FMz{QnqZ_j>b%qFFrav%x`VYXlqktN*9S=S1_`+P_2oUy_Pgqit0H~xBj?^Nwjj|2D9QLU{~fv~ z=%Li`nIZQeh*9G5`I^=7X8x1&#MOYUS7V5#jIMUlu%)>%zPVV)AOts#)Cq%&gr@$9 zQrNHWL6P{=gkAFnUx*Eb(92P)D z`9cG5qw5#U?VFozrUzigh^6sOM2|;$+n&Wahmlya>_p*2tN*BUK-;Zz8ZuLba>#eswlHx_FfZGB!EB zUauqr{P?hTp3Gn5@9Rz@u@dkhuJ748-D#jz3y=tb1Oiq#Bld{N$rAhS{=U&F4Txsw&`kp~}?A2xhJkC}Pi4Rb>YS3yl(Ca~Me zXck(rwVJRdrl3amlTuE%Kr>4$&G3i|YycJ@J67VfKT)#2LHlOoJ5W`#w~?>N>$&s2 zHY^S<);brTR7$^3eBmF!0;&pSwq2d20)5sZiIk-0){$2A+m&Lj&$o;mVh8|*=vVBN zTlZ4+yjp)g-Lh-sdS^#l zaml;6;oaQ$L=Bl{<3>q?SXrTB2dAv2wT*zhjSVplz4B%dGWqteJ$CQyq5H9z-rr8w zU8>JB=Z*l!b(v~&Rdb$?&u3}Bn}1|zW6M`mWC$9Q=+Lne{RsTN&YNM{e%?1(wby&M zcqLPh#z4%ASajpxS>KTPQMohtqasp&zT>uUiR8+1=l8^Q8v3PE%!@cXuWX;0r!C%Q zNB-ZEB7QGUB$%3>rV9X)u+M3e$OV?d6y042MKzA!E2*2@nEhY^?prfRm7=$4F2n-KWhh ztmrUX$-EmV^ zB)r%NSazBaMeh~G2%-qoZh;qSBs(thJh8=>PmnE%3t{g;&{qxu6xOyto?X$1S9Z{y zNwdD;P>2i_o*M~78Ic-Oi>ccLO9rQ<^9{s;SgGb)a{E|}@Yt^Xt{G&xuTRi?4$Tv4 zT~3KcS%j0yH76=J79(A@K+!SLe81+z*yzROtnnv>Eq#6?54$&#I>C+l!JKk=u!g zfa>j$oI|#|cL`BKjZ#R5z1M+arZ2QfK%mk?AV1Sj*reis!X2mC4V&c zuNq6GZ=RV2z;lo+`K7xMD3zLR;E-4;a}^N6TZCd3ZMw(=B ze<#_0S^>fbxs(K_B=B@~As4@0ri=x67T=cLn6%mQpqx)J8w?n#^K!p&0XOwITJbrD zt~$IilfT_-#**laEL5#}?Iy|`9-W%2K(&pVjYmDGm@rjuPM~UNI63#RW;??&7h24^ z_SO)>H3wmk2@VIm9w_Dz{R*jQJ3~^{F}i=hLak5J=_u<*)wR- zb7W0}TW9k9fTxz*0uIoa)%q@?WHuu5LDKgbXtr1pZI~e37|g=^F`O`cb$oXPog-;cef@w zNa2*gk|6VURQb7ux!e3?wA6*kpAM#%Qz@|B0Pu9nsH`HGTk z368`~%sq|R{z8Ir{a9@RrDOjPJcbgBTj$5)yl?G32t_5NW^3pE;x=PcslK<0ndi?U zT<(zk?ZO&%#!t&dWb!C|wbmPTUrpJSQSc!pUw(n0YS)dK8hfP?N*@$gp(1j;|6gj~ zi!01}(;&g|QW17jIY8|)Wa=jm|mDP|KD6-p_{?Pfy zc0I?)RR&zN3w8)Z#9tRa**|(`62`S%lsb9~N8N%M~&9KnH8WpNO4A9^snHlO0#(oHt$(SAVZv!9kwR|LUL?cM{P zV0!?5z;##`)x7%DT|j+i-%vgbO6cf2gqT1U!}*n``==jwuzayum&To!KWiF%y#s6f z&sf|qI9gN24SX@oU$Q)(>quQ`^L?KQJI=EkMhssNh=s7`x>1pX&}(SS_O#d>Ml#IE z*G6`1+}ZLeN9iUG3d0lF@47kiw;m!iefz+F1gS%h+HspMU0xh9^uJ~?@&nW;LmGG>ef z!QZkHcb;G5ub0*o6IJ^JOI3 zOjS=y!PuBGLoxiU+8!MMbyG;H&ruG>k&Zd?t@tOep0|YTjis_Xtdq30@q7I#FLlyS z6QiZ2&GuM-0x>Xbowa|eHSR-FgW5vEpsIc}dy$OG<<*|=Sz-C}P5il8m*h-GqHH~; z-TLVA&e+2<_t+uWIbF}bHLj^hhR-0}2FsfBsOCl#FBIxU(p0IXo)(^xhSeP4KZg{Y)qm+mfTV^TVfRzN)d4CS0fDOHDZ)dW@w%YZ-?Y=$&x`6iQ<5C zldPhZQ0_A1_}7tux+3Q2?8NfwL^V12U~YXS&O8(3KsJZEy|R{_Pe;|;d1~5!*xF7#=ETt{b$TVBz|+JP5s=rN*f z!8@n{!}Uy#Fzjq_0Js7aEf88CTWzpT`o1tx?uwGl7{G}0Dc71BAHJSIC2M1(XHs7@ zQ0XLqN2)&-8^tMjFn+etpBNq|`iN?EA)7I#R#=()E5<0C)Z8^G}t@$#>I_=mks9*#&<&{)by$5W~rY zL<|l}>+0&NK+9X=P&_j;r{&}0gBBhQ6)3Rs%gPcuJB7cF;7XWBZdP1*V@Dy#KpLHI zmo1@oFhXGDrLMeuGy`jm(LnSQn6^aD`e^7Ot1DyV5!rsbdI%!!g+X7OMjcGTZA>N*?jeU#4Nf+>7foeR^w6ET%#}}&D$wB#l zpOU7vHD51j*9-Ha@ADUYChhJD%_)l9q5S%Kjz53?v>aEr>$KaGRv$Z8AizU&NBnq~ z1A{vi1N8|Rf{*4esZqb1?5WlE#5*D`b>up|PY06skREz!>`li~tIE71f!Wo;(n*tq zJoYe-Vp2VU!|XT$g-Z|lgqVxlQjGc8WQZE7Q(>;CHrL*Mk6pLi9U;g7%?@o6X|1hjXd2w7^nGrL(TpU%QZy1*q8MEb z7hpw><$qHtB?tNH6xL*Tfvb_yh{T6IR^ssD&M2=?MywjWvy`7LyhWRPU<^b`ltox{ z0|211K#BD>)J8l?0rj{783D3w({^ZZpdkh$loc;5HK*F(&2Nr+$L#Xb#u~8$)t_VH zrq6+8jFFD~P)xZNL_#bQZ}Vi0Sdot+1ySzM%F;7E#t%f9S)znVpwnwY9dF%kLSF`x z)rAWao`Y>GWYPdA>AU+MB0H=_ z90Z|68XEzFfx*_Osn!We1mBuD2ZD0efF(w{NNzZ2UWd0YlFo4bN$`PJ-oKXs81{aV zo$JD{2|sL@0b_`zX}N<#ht~R|y&svWd5I{$+f#Z`&vhdznkUGW1n_2gR3aT)y9>7E zj@|`C%*v%X-o^@2rS_3?0-NtmPd-0L-TpB5WY(exJ>d7u9a9+wdiW6LhIZRUJZoQ4wzHOaf zhdjk!L5iKU0$u=r|G2ci3cN+V{9wI4KH!(dF1*kF?z!PL=lQx7k>0lJJNdlJZ+KrL zBUiN~`G!1cZ1xQ$CIlR+qt70kxCJC$yt^DyWa@*_{I!KAsHq`_zC{Cg2!I|vFv@ew zxxc)-^YRz);r=eSQX-dCqT~UxV=y1V1uP#hI3BL$cEWb9%&2d@JTAW|5PF?%HDSg1j!d=zSeeN&>Awm`o*Wv$`J7Xu!vAr`Xl9iRG z;9f2%q6_9B0J1+ zBPF@?biiP<*!bAcn>zE`D^Iuv*}8V@>zVy_ChtY@RjPR3%}I=6F_<&HvDcQL*zlY6 z@YaNP8xlK*f%YYATibSrPD@Sv0>;E{Ii5BlfT-!zvTW5NorBRrsi#T6%gYfMCrW{Z zf&dpJ>evW1P3Pwa1_#5fGttt9y*%9|ef`=ST1fTh?!>s#x{8pDtY^=8Zhl@)+=Pnr zgT?gvj2I13PZM)^cm&k;-xXm@1S$sW>e@hq?)}5m`5tkzZ%WPEu^RuXfagQEa=l>{$H2luDrh8t4qbPX zU5Ro|v@x);u@UORWl4)ME#^koK@tf6J+2UCylu3rG!U?7*Z(~gr|py}_OEZYk;qx` z;!Y+Q`4qgA<6~`iS|WTX&M-|vT#T#ryVtc*O0u5)Jzqg0Tk77$Paol)zkEqcEp7Ox;J|&)ckkU2HMz1> zox)W>q74^*h#(0-4F5fjK1jlt$S4!FPsQpzhEW^(S2ZEUJQ2uB1(|pOxal+>*He%M zsZYX+1QNzQJ5XDb5c_N<&7T#iYkmKW{mU*?ylL~uciL=pK)MFpZ1Kddi!_C$_oGMUgr0I%64@Vg!cn8 z^VT1m>O!c>4P|$bu(3TV`vEWdCcD+il^@3~C-q}MGBV=_G*-1AKck^uz}r(BI;frO zLrqIIpUeKmv;^kd%uJ5}d$}8NSLYKR*=N#ihglvU^41(y!QMRPgH>MUh}l;CPEVEt zWj)Dm5mBuXlixouUGB)qEqNWZ`rt7Er6XjBTF&>+whidiJM%k~m%(E#`2JKvl9_jz7H34xb*Z&J3MD|onH>jbr*Yxe9L|m#Hp8$y<*#M5R|Gh+ z7|0p8CQU;-IG`-<#UQ8uzCo6^{TP$2pg#F37+&N2e1Jz0Rbfmnrm>;Z4@Ra>_9Lbp zDLdU%dQ_G7@~ZG~nd3c7T$M|Pu9o6XC(CVw$)>XTY$z`OZPVt(f=K~emD1XH9#>8* zSB%~FHm?sMP%fwE#GfP|ZGBc_NwdAFKrCb5y)P=mdO3{(|MREkD1EOZwcV+4B&m^u zLoDsi(A<1um=!qs|3H|lNgYVnW*uU;dh)w9=4YJJcrJI+zl9Rvq@zPaM_2fxugm}b zGl(Tuk_jK`u!CqbDI1Hy&d+m0SDb)(CSAC(BU9tT$x>Ysa=pBsI%s6i%Z5%#?--CL*G?;&b=ZT4IJC%5n2|A_Bb4a2`nd;qS;QRyGB?U(=`Fx&+JGve zP$Xt6#tkpHKonHWQI(v_bw_#(zeMf`?Q?3X+32f8EdEwZKfB1;f5|o7l(pSTuyaWd zc<;V_2wvVvV#vw6iXNb-Vex`+{^KH2~uEnoevU3`~0DJ{T0jrI!VE6S^aXqqwTyjg5$35cvllWO3 z!A)~zugE9=3*1vu1HqenKPJ8%I{QZx*7%K!uIH}vm+08lnmu9>hT_G8p3m*k0{#~+ zf3m?k@N}G$k4y&!e+KF(1X#mTBm9R0s(hn$DfoP_uNmpb{P`1&vo8LZo(wtbkh;Fl@XcCED@pu=a$7<@R- z3pLI9c28w%&>k&h2YBUY#VJwhcffSYw~pkEo^G*wKS3>l6;lsQj(YvXAj4*ljjrLV zw=)7(PLkQTYgt|c8R7%h=S5a#2r{sfD8yQJ1#WDy`}vOKaaE+r%pls83l2Lmj5;dn z6W={^!I=H}!w$4}I{1gcQy-7H3b9L`O17NSCzOu{ite@1;bQA*zO_*)-_^^qReKN# zQn90KPE=Tx+~&cVAylsK|KfPy?VvZ!;qByuo>n8A2%dl^q;$Aa|4pzc!16Ef|LpC~ z7L(aQ8|Md_CDvbab0Lez#ZcOn!dgQG<_Iq09+J?~!jFo27fTGSH@l!=iaev}?9_vw zDmgD6o^c5XOf%^qI~U3sB~YyetfgziW4AKCxk(|%Lfr5(*L6`tu*76-FvH5HFTL)X#1JLMBmf2(V6eY?64D$JCj(_hw`oZ|NoFP{}MR{2^_FyvoLw)wFZ| z8=gVO)4h7|#BLxd6|&rx z8A|4>BbCCK{NV|?ku50Mm`$D`%v=Wwz9Fx#{`CzFs`4|ds|rx>XR3KYvPrTUk~{+> zO6>%#)pIv6k5#E@`sGye^CMsQ?B*355ee~kYhtYz-SO~wTjIYLwx#5H@bc_h($dma zt6Q7LdBe45hy!BLs*qDC<1m#gfYOVftwC1c@Qrix0CoBsfm>4`Qc;I6Z`R_;VBTim zsawGe2zN-qF`x7U88x#H_L@?A6yJ+vpIoJ{3Aqp{CC|^_(Z-+!!HwYUTG=IWn?rjv zY}CO^M15Po;+)@?QR9Z$3O6elU)4IS@MtUbX22akQJicrSM&poTf77alj>)B_@#SE zOwtW~36oUhI^_6SigLQX1%0`#EZa|*4WQ>OPR?O{4hFpPqzx&2nUoCo$*MCkhG zZ5GC`?4Gzx7CJdld#pcf{Jp(p*V2zogc2VA$D};t0cJ*v@y!wX+}*W=)YQrKpFY-5 zEP^suOz6QgmqnJ&yS3HUv&VSn`G3{?F8#O#BB*X>m$mOV@%tv^&Dg<|TGs{=X5HqP zL6>TNsmyh{I`pZ_9WBkkoxw7tFU>y^ z_*q`K>m+7-5v z;ndz;kQ3eK=QAqX2U4&ItluXf5ri4kNGEbE^5!@uj zWKlqfXIa*qOOUr7F{S~+aJDUEM~mPk;-BtL!sL+<90hUmrOoy-J>D^uGW2HkA0Aun z)v^iY7(-M%>6MG8@yvZrmBa1ULBjp(D^-nL3r36>PE0Y{zs0W^BXV_# zsm2M}H~$6Uji6O771ZfWZKm)l?uws52W0k00;9y%;n2GCUy1j?6{=o<-kT@6<_Z-V za+RGoanq&AHHLnPNph2&$t4)FPZ9Adl;vf%;anV__oD_SHQRI53aN`2>VD4qUN~{`BgybAaIK6;bjT_+5jU7x0ac6*IM(Xs3kCQIA z_hBgRhFz_%>wtCj1%@-(#*^nHgxXv6+|ru1{#+6kX)6T?aFY{&F#(bim~wR_mORNXcs8)P2!8tfxQ7?23*)=+a9v{Z{Z?4pSzFqNG(hXgZDq9-;yr zc;@`SL~dQgV~H%vsxk-D&I;WEdHG_ZW9W^pEp>5-|A*~rH_an>T&Mj7))bi@|@Y(-E-CIXR)xK@RDvgwM44`z!Py-SI0@9_l zboT%flG2Uz00W4WAV^9`=g_HyG($>v!?(TbcR$bbertVyeD7NC%No`im_2)6ab9sA z*LfW0+?(fg>fH(u5FI(r$FFO9@kvRYE6b({2?-mUn>iIl5ic0pVLd$-qxq?tqX!&k zVzpt;aG~DS0Yz@h(PNJGof*$x5rsutE1b#(&UmRDtX7kW5-S&a-O)2U0OyFS@Y}c0 zU+I*ncWUzWL}}t*-_JYP714$t4SBJ^#1kV}9~Do2=Lz`R33&{$EX=Hd1s))6?Ckzn z+}yyV23%wonuUZoA=b9GQg(w|XxDz0!)b#RCdmaX#{m|f@I(3%`1jPHNl?F^TTAl3R%-CgBGrJ})fux%P0FD_d8wbMmm zqoW_c`A%Ycsy-?=znW>V;=)-~C`#m~e2mO!zytmdYB)@m2P0-1tp=(I39MSy06a>##1p z@TLfDtApw$DFNMf@Zqa0D_Ud_aoH$ksel5j{Y*iT4K0S=46p!I7=KJlAYLySU>Q%~ z05{@sp-82CWHhh*HrfAE96u2F2oMgDE;o0aTPn;VGpBxpnENUap!oKX!;o0MeTO!z(yS8)xv2WYRKqe@{;< z@?>OwV#p{Frv7X?Ax@)Vm-lVP+4~*1G+mc z#g<9Sc|uq4xE$wC`ZxD}RzOG-naTLFnIuP`ay%1bi9oEgAqueKsoQ!%-|D5hocccaYkpR2ANg#9;m^V3r zjbcJVq)mEseo~S-8=K|(jKq%p9!$V*1wlsoSn`SQSlXb1DEUxR)`yD~bz#ll=l<-SpaIKHK+Pz*F${7*URGi__zjn;Vwn`tUMi5-ZbEu? zq6We~WWyzAK~Heif%dgjdXGII{&26fjR&+nKNp;C*iC7R#&0(K_{ke4t4wwI-scjb zHXoAO)vztd?WfA)e|$D-tF*7400CVgp8Y%Mo4I-ds1mSQP_l`c4CaVa4n?$CJR8w) zdyb2aWJ*VNKtc9TV<$i#1u%rCYJw&xN6vlxl*YSsUhup2?5lQGnz0avu|9K!0rm({ zLcxv+(h{6o66=INRyc^YFgIU3cx&xsw_e6;K*KatkLj<#HDg#u#+NVv@kJNUMCn_4 zrRN&15vi_cm-L`ik`>K*1iSQuVpC6fu?%}g;%hAv;i{p!6o=sj<#HzpX0SkuOOC8F zxSS&@Cr8qLMs`1zCI5x8G9w6&DB}5_Na-K@`8N3u9%6a5JZGSm4t@frE_p~<0p`vqn9JT*j%rdc9rFH%XviD<&tdcaY(@c{l z{=hcX`&ysKa&J-ae;(0+e@~)g{2VU4?gzj88aGngf<6N8`8@$sw)N7H#_@th10D4L zz=!|xV)o|&O5(d`bmG~mXCg7^Xx3^h-3hpa*4xiVn-Bm)Ej~A{KSV5 z0FBgC$N^Bx!3>+yM{xe?H39%&o#*|HIwURExq1=7T-*Fd#`T7?IERfxIKXht7kT8G z%=fU-tod_^Bh@KK*i>Lzmp@3`i%DXFG~m@qU~(Z7bWPQZ#||fUHggvM+^gbd>$fTu zvuI8j*M$`l*tyL68`A#U@$z(n)*Ap5>oT-Kk`?%1(VX-}uZhcQzD|vDM~W52FYy4? z^+gXr;=*TEgd$&zMF~XG#q>OyCa^f|y=}MF6BTzue=xG26x{4~e+PKxOcUi(npN`; zYua4jF;xU>9EkA3B7&u@Q4!CBwEojtZCG;9_vplTI7tuU5Q>lA2lTH>0cDz}B5v+< zb1CF7oei$9-kXy;icNkl{lq{dAc8i8D&(CsP8hl16MK^T{LFkEZi#$QQpO`L^Oqgn zaun8OhzcZ}86GUHlM;f21jyR`yUvg{)-Fk)TewmmW)H_9kdmw+KzlS! zvUGH&!X$x-?mb+$=b6Vtu`4?)Tz5j6O+?K{_%-qTGwkMPVEHt(6_Q~YyRei@_RXoP z6i-iBVobVGy&64adb&oLgQlaxp?bo)^wEFn`fjl^FG-3u#Du(V)^V0g2x$KO;b#;~ zSVdlWtE!Cj{%nqM*=HZ)->G~%b;xaT5Q-&PHP8Q}0smd>zeX5j;GfR>_n3Qx{;yvB z=Yv@1u8$`Fi<=N!r?FGw(*JYae=b=ZHmK0S0nR)U8URfQ7_S+6V4?~@hTA&%m?Y(_ z0dTcEMFGG3AJ6^wE!w=p9|GSvJi598kVHDjCNMWoZguhCpfWSg)I=bV=RV^9dBX3u z4kQ0p^ZZjENc%r;|5dWK|Cg%q{Og*34B9#PpSORTIB;1YJ=BzyK@;K!-p#lnVS)s{gbM+|vMr0caB)9=i1P z_ZO9vFwoNjOJ?ik$rgaYt*oXR73Alm^;bIozUqHePzA)*4YNr%a2u8JS)r$W0({q) z=Y^RW{Y;GmJtt>O6n8UhY)pg3_gEbek5S>L1bP}?ZXf#etm4f$Q?a`cFNp8O&_M1- z7py`p;w5#s@mtT+D||i!cTzG#9oYx}t-x)g&b-|qL{33LXi5sHf`UTNzyJgAnAur_ z-G)7JfXN4-bU7s@(Lh8aU`SXB`j3iJ7lyfMz0&ZQ%yb{TP2;V4WfBV1-wtUmjLw8;&gi&Fh9C@OkZE$fozHF zh3hpTr7Y33=x7`bO--y2sp$Ie90e)L^b*E|zZ^}p-=G9t)05|!E1A5Np=s9PW}keu z1W7kj7qNPRX5)98psd=#lpZ9LG~Clgwq??lE>d||$YY^E`u0<@w&3JJtrAQ;ZW)So zEF`2>TboSaG>pt>g`;&bz#8R~ZaxdC(MGwJ5NeFUw0OhJU4!TBYMdWC9=v50mPtw` zMaG<|w>UoCY1AiVWyz8~2xbVKv7OeK>GqDL&O7MxuHV^xqja?>cM*rrna?s8w)~j7 z&~9D9?|k9x@aB$kkbe4TRFmhR}|8{ZC3 zFj)Se4x;t5f?{)vX<3CyOgNwjatkL9x(R~3;~%@`nonl}&&u7Yq3YWi%%hHtX&$6U zwsc3}jq;f%!m~n$S5OxT9>(xM_@%zPsP(n3sxpWPVAbY$PEJ8XG2UXOv6|UgldpD4AD`h8 z#?G*eOS{;L$or8Qk1mYLkBl{@ZK^V|Pv^CGU^*#K%KPDC|J$x(L!*$-SY-gEo1mbe z=;-K?A3tP(a5~quf!~p1#l>udgoHhPeWt)fy|iS~t>M>IAK1FZ*^bqr2I_6|nT(5~}&*mtk*o$(tWc07y}49~C? zQv5Q76`fXQYT+@jZuChFp%4B0(P$gexZy?v-P!ZK+IyFr%8gAc$4B)iyWASf>&miI zw?{_%xmNxI#cZ+iL!CuFJ72tEOP`VYQDrNWLG+FDd1!mcnOdKCXp>#tj_>4=_dF+3 zcq_qazl*AG>Y^c|{*XOx-;ijwCCf7HIU z?Hzx%4J}*}*G4Gp;{f+?S1t$`)0MtQTJE*@NR>X* zUjtqP>F}`0d~$*EE8(oNLmoRnnwSisQOZ*uKPkZ`PLnmbE2U-?kvgbg2l7;VewfCL zNs&y&645bCm|h*gsK~Trl_dSCHslwl{?w1zutzU)>rBOyEzmq)k1W(hq)u$-Y+rw7 z8Fu6Ci#;jdt&kXr7tW|69bz1Gx<|V6gS)Djsi$30{F%vvJ#mN8nQw_PgMI+it>{>p zI%%Ep?sTrC?3dYPg61E#D?*b~Z!5NAL&8xEd*0f0u&RSo{iDiDpD2Q)u<8R#TCFKQ z$hd!v?JXTsgkjf<=X`#BN_|YLkDL!u+{^>IGxb01;PRYRYxW);?T!gbXU2TrotIM` z>E1gtmXivuPHl|PEMj4A?QdWZlnYmgRWOS5o8srF z(Nm3_NiQPM{UuquQ#0GI0T%tn#YuIRk(9qHZi~_MbNOjvwxdMPAJ68n{Z6k1zrdSe znj>y|ULjYzx5D~wua!6k87Hb@*R%aXWoU2l9eoyF%Hr<4?(M*@sj__yrqO1LC{8WE z!l9(OqVJeaWD)kp=IVN)RUK03e@fg|#q4}(BW-VRN-k!#qET;7=*N%3nQ^frCSmcz zFe9*+2#V`q-sL)6xtlP&AA&#FNn^VfV&$i}o_EALJYf8?#7U^-t}Ez!3cjX@1EtL^ zYWZc6->oi9Xr)NPl$<$%b`eFK+kWw-j$hzgaii< zZ){|wb!3DuXm)hq)hokr9EBhkYP`H>aU`xdyIP{6eXU|%Tzkwcb;F16f^d*9>rlh=-_;QJ z>PS7`4`W(HD`|eadpnAM{)4t=pVEVn?Di0?`KeHtdghE@|Bvhoyq%`|F4#vG^JO%v z-$jLTMQX)5<(i#^ICp#-89&FR?ls2f(|V?;#5&sxRiz$^6b(E@y80kLTbb^7!t#YA zI@X(gF=#&zslWclU1_Guo;!3^ySINr#hq_HfsrQQK>v#KHQe_C|9q`>?R`5-RIG$p+iRhP z-lH37CE=r>S)YEPw9FloB!Q8hVL;!Z<@`p((Ci%M+wXL79m44OZsS(F?wsCe@l|MT z0KVTPzRzkmn6O&;B=YLwu+~pS|Lk5YLa_~vcI{#I1+mpC<2}LGGqh`05|@msPS%LT z_w8ue-$L#h*F-fM(R1mp3|G>rnw3i%;QEeNg4#*>qZM|{y(E0{u|iNpg6f$6blCZ9 zp}2};YmSv73N?lgd81s&mCe*b8)ouA+V<)Wtd~NGn~m`##Nqp?;c(z|p+@8NWXa4A zc?&vyzPpS*M{A#12M;AnvT0cz2WE=`tE$P$(ZMfBa7zI*xGN2cPCO7_|gM5S=Etw zeQ zAK#Uv*B_6&5{>wl+B-NQTF${t--IPu62`&3gNGyKg+XaZO(;}{V^+pBm8r!Pbe|-r zz9ya8`)9<{4?e6(DZ@DsZnWCbz9|1G92*;*Ht`qm&XUbue(k-FW>XLHUkqh+MpJ0M z@aUSZ_2t~L;nRWm(iz$Xq|7yZa@b{BvrZC<=^gK5rN~yPZi#h5eRC7W(JJ}BFs9Hitng6 zcK2pnJo%o2rKC@^(&;w`Sl(29&L9d<-ipn{@vJI>i)S`6beUi=Fb*>j&vvR=2eYu& z(Mf;TcF4;y12%6;Mobp2>l#UqTIXugBjY7EhR$PF&x2Q|^AaeMWYRG}|Kz^ZAWwW=-tv#C(VooIRMgs`#!=!h4rU1Rx*^`}4W>X#$g z>y|>PU=se~R#{%a`Q9iH5tS}rM@~s;y4sg)KV6AdDv*|zW?d@0nxy$cMa8m&L8U&B z;zGQg)2{BO*=nQanm9{1^LuX*=p&9&Er!wKcOPxWZFHNWEw)YkG~PRJ0A>-seb^E~c%g5Gkb%`Oja! z;Sv^znlv9=t%@isx1VM^1%)R(U6=N`H7elwvZB;{fn0F@Rrjb^i_>g+LM(6STO_R? zQjf`ee z*6%#_Qrvgks{YgCsL6Ata!>bg2`=xfUrj;hOdLx?!+rrAc#1vTc`FYx1o~4dzP<_e zTK7xJp1s~9wqi|e+JMWDx+3a6-Vsfx82rmv6%BRTKjBt&3{oGW0mzb{nz`r zF-7L(RWUZw0zN_p1DJez?~JD7gQ5se->VH8=N>i=i@r{ViLwQmU^@R; zm?cjgQdjFV+U!7hT|GT9z$P7bt3(R-=1NTLDDkNXB?zl0L{UnY2;zu~u=%#17n^IU zNAV5V8g@YOMkQ-n8s9~iunw>dbuPw8aoTx0hJ~S7W%fr0zBDL^*p#=tq#y%xcMTSh zY;7pAnEOGoD>7#w1j|5Jh5KZJW@WgIml!|whO}KqRt?iPk!84e6+stPXT}Q^E%f5D zv{TG7tQvtwhxv)gY0nn22JLO4)zkztG}+cQ6L!pu1}!NYf*{t*@jNDojuqV_7H%53 zy)V=_-&{YzjCvanBRO8^DeK<|x^LqrycS99>zh_mdSm_Rr>v$E z`3NM(FdYLo(sICJew#31+r9Xg2sz1)qvcKCdXa6+f!1tM{mTLU7ZzTR?rR=QMzgaq zwK-ZKYN)^O%y)gErp_Z)eoW8Un!%cy3hg4{Fci(kDiT-0xv4(ue4)_)girney-(F< zm2fS275O!YU%RxhW>Wg#xc=o(hPf(ahVXt=AAF)mgfD|B*8$OjPeLTq?QF3MiFuy*gB-g|w8D2y zWE?j+)&|q1OzV<4U&TD%hBRGmP5%H(8E$fr!Zmsa66{MRC~Z&WMn)xn)jH;}Zdm>H z*|PaEp~X?9GwGNj|4eKEj+-47Wwz*255O%e*GGW9FvoA~rN6&}Ul~x7?N8+{ECyuH zi`ix!g{(5)kP6MY5drdGfD&*7;8~1~j~{r7HeMh015x%BBC9B_JuYEIM$NJDslTTD zkYNy^Gk&hu1_Kk{I6EOqPocJkx41?(r`%I}8L~gQX;(f*R0+FY^WW|l8XYB5x8ysf zc9gi1K_mO`@jY`d%3c|lX#L?@6#+V@zdJ+Dc-lS=qADC%-!5e*-v9;L<3-gRZg4Iv zAq092&SIVdgtdohFP#}__0osn32s0ewOCFt${2kcHt0*`9dV=Z6L*m zx$F<82xykJ}=6eJ83<-mCM2g2BfU9Vl$k zcFxQmvj>=}&2^DVM!^$VCNGwU(a*;0kOlKOQb^jQA73K9NrFtBkTgnhHN#7y?y>Pa!g-qjqr{1qH?gBcjOAn`y zy~obF7Qzv1L#N+%bl8TR?KnRhIW7vQFn@Ug&zXDUCbMqxz%VYVefy=M83T{k&zF^= zEtanxB#y||TNe!tr%6m^H?4SByA}uEQr3*`xRY->8qXE0#FEq522Ro}GkIxKc|bz_A|#vmCS&g*AbrshtQ zD2v&@h!)G&9p90=GQnZx=Zj`Y6j|xR0EqH^HG_?@wPLCnN<>}ui7L^@b#CxHy+RMy z2w^S^M(b~^bD4!l3bh1M#-_uTYoP&dbSj?anCxpjkVniy3#~j}SmQhSpFT#K5u(7U zO&`&*V#G~`g^er5oQ(V=D`+$ksQMfTdnW0QzJ$om@oC3x$ zCUF~S`|ET6eu0z{-W#YVI0RYOwy*K>RewpEg{Ns+^jY;IOQs{sAB!H-x#|f#~Q44|Bhr)0&mmW7=xC&TYQu~S^r>~0`RTQpZT9P5w@Y9*(ho&xw z?!9i{nHrJuj0fMVtSDaL=QB&{_IzQ9jI!UXsUtNT%d-q5#tkIdw$^w&EFU>`Fl}^t zzD1b4Ewt2tYY%<2_)1J-zEp`5i9L&Rh6{caK|n=iWy~NyeHb#pg#}eIaSy)@j|h2; z8TjXGpSip1vpG3BE-vm%ZEfV))#lb#E)XXO=v6>KYZ1&@ivYCT%>&578T87Kz3$we zuHsb&WDl7Dl@gr_Q#4?ogYUN=R5#~H^LFEj{mj~{qn1RE(&E;3>A|Ps9r%LXm?U1U znFvOw#)L9If1x1wVji-3ry*!aJ@zM}Gz@)8$EY{()y1rz9qZmjtx)2gr}Kq@dD!f; zNV9=TgFvqE*xFG-D#o#=y}14tTFpYj-OimRCXbk-YoJ&6G05o%soLBhZ*Ucgzf@|q zBhYh3Tew*e2*|!pc--vCo94wAfM?jp^eBRdBF4WtO!rv^LBKh zaU3Bp881kzBpvl*c8_~x@0XJg7b}%wq0Zzs+>6pim-@xNTt1HYyKtqfR3(W4fAJgf zTi;iV?04H`7t?(nV>g{mv*$(Qhn~_29Ba|0@_{Eg<*W& z`<)(heM=Q_YAMNxw|f=?tiyrUv@KIr*=aW(Wj>ko9n^f)b1si=w-}=%a}z|i#|IVM`?t*bFDMPWj`3PV0$svfJSYRNzXijunh|ThW*j<}RYyg>o>5$uZ+!2;Onvxe-Ab|% z{)fJT4Uhb#}q2V0SSYA?0oX^RJJ#CXCHJChZ3dOY=eYw99Cl zdD1*7fZr5ov+Pn|0qH~LG}a`vHHCSeR&>-gQY5|n!CN&JWzv~O%9L=ZM%mYH=K?3$ zC&IaeQBsGo(dsxpFnrjt$U>BAVVb9zcdtd_xx7sHWGB*z#vhQ)p9SvKd=77X1)>3{t4rL^*@N!E-&`@?PiMH<0G{i#;vIHZcp=38=(+T%{dT zv|b&p?_EIj&-~IjI~wWGzU~>n_PyVU5GFr5%<{jxw`GfMJ#)tHcvDz*G~nIR7vyBO zSa%GG^RuUS(ueX+HX;X;R?u1Nm+5B@V?A7k7h1yZc2`jmOoN*g1tBD;rXZ4biwExM z<$RKKI{_wy{7M|H-JSfLIIVajNab*2|nM^WnXkss;oiqHrTfi^591r}YZUEca_*Eqw@U#xOg$t=wM%tY z?~3zW>E$%gg?E+V;elO|wtcf8i%0tzD=&xEh$e3^?ASR}3WGHHu1QLo%2SMHFW67r zI2jtfBqEk^OG;dzY`f1e{WtB!_v7xmyX({7^B^s)!s&_XINy1d_JN#ck?h$67clMA z(EM()8))0IEnQE_58sNqL;B4e&Z+3M%H1+g9IVvcmYxe4J2i`t0mubY?~HtXKGJZ#?7mLDRWqk;+D{Sa~_-aNUa% z&kp-*ro}u8(PNyjSwD`{472SXaujJKYX30(@NyRj1Nh;&Hv@GBJ*;K#+K;!a5qz_R zZMl5(_F#RD09!S|*&e5(we`uMyeaZHnz3hy{qdsg-{VN~eB9)aO*aKNdd>$+KM+@T8)O>sRk~9JbxFkf&$43 zfG~U>Rf!vZ#T1UEAD>ru-khAjs9c{mI!rF}2`TegWd~jht4oDf@(KO3c0=o4p{o`{ z37^x9u)mkB-?(Bw5$~&dNKGRpkd3q$;P@hr3V4n2FQrVI*&5N<| zjBoO2@a|l6uLT^IIw`kj&6miV(c3)dv#vQteCmS234_s?S^*gr(N0V@856SEn!}0G;}DO{HFmFI7@j<({C1r z&%F|R(MT>thbt%cM}e!T+1cxETD~B2@Al(-Jme&7R3EX=*Tb{{A;KUF?JT+bJa-9a znyI*&(%w^GFWNI|3UniJDsCMWla%c1L(!D7PhC!on-0F{+BLLfHnF0YB#sN2@~7K$ zK1|$pM~jd;aGL+p1!`p_OR0W$1jMaB41OpT1aSgLP)l6NoK4Us+Vy#E@Yvm!55eOS(~?|AYf&K zprU%P+P<<83UJXP3WT|YDNKA!Bco>x428sL@Ap3ji%SI(X>77G6@VHI$qIUD@6+bL zLSqR1X1zzjxVdj9s3|Q65d|^RCjnT_y%yjIXPw+e+mgJ7aFS zPUPZBm=TdBy*>4Yz~jPn$nFz;|JJhCsVB33jkp}bI>BuFn7C<&{@k_l;A##;lo<`C z5MX8NUtuKF#0e{WF&&U->};MCo+(+w-2KdoS>;wq>x(Ypo4{us4b8wAi+iLL?{FSy z1vf82nDZ#?oJ{*qZ=$FF-n2f(^w%?JR4A}(OPtb;+7b-WC3|ufRTt9B1Qh22@FCWceB)`#fK z7^VNSmCGQarMGMEkhet?lR?&?bkhvni4D1PtQA?6N0bko@aZ7I*Hb$()w9bq#D#m zH`LaQh_otRM0p;~l&zVo&<&WyT}Ly!iz<9NV;dlOSa+VT%6GI(%txBC|DYuWcqo}Cg%IA!%?zUk4Q`5)no-rxXyFsD4EeBEyfghgAh7(mn67s`i z9vW{qpPp;2x)?NT*s_6ng+0&|QV*K#=3AMY-I#^Yl*P}p`aks^nK>sV<~f}O`5d+Q zDcE<1Twi=cJ2yaQh7of%y|1Kd>G8#Pi!W}5U0l#)rA|7du{UL}9LVij>E8Sp;KVkD zK!0X6rP9tqX;1v2Vr1lv9DM#ywk*5qk5bP}VHek{?Sv}KIt}k0TFthlAa<=zmNin6 zLU`cBzE5dZ)*5cZw%Kp-?IxWK)L)}y(asJ9#Y*1@?9m71Mp~7#@Fh0v-7&uFBl(_0 zwzy`WENNlElyOG=WS7(2SDp#sbqIT<*(yDAR@nvrSjp4s>GYJk!^V=ksB=YV3Y}=~ z%KFy^5WgLW;~hv(O@Ds4zF5nVWd);nIm~9cHP4h)Rmjp`u^}<5pvn=&n5?^~bm+3& zk-0rT8Cd2{{ZL`jaP4bhF08SvqedDA})cjB|5?q6J6g&_f^w%Ny4V9 zo%n{kihIwPHy!QhxBCmf@Zg_4XKwLFFEEME0)1%uoRJRDHmCX848LbEMlCOqA<=M>EigIvqXwb3(e@mQ_YP>Qg_1$ak5UN8C@$BC}52 zFWY&+hi!?8a`KYz*Iw&Si@#Ftp()%^!%s{8%(_qa&v?JKfgZ2{nHT`vZDViG3K-VF z`Kb%ArvBDD<^^I^uE!;Ah!lb3KQwXUv$IM7>$ao>r)5EvEtw7o$}$6k!Q>!NUT$uX zc45F+BO4tGhL__RUd^0%XEh7{_DpxA)p{k*hMK`#E3dmtf&;!!(!0Q6wM^j`$Qyj; zt%Vz^xM{&*pX1Nw{_D~M`+<`-Yg!UfF4D+_-zeee7|dmN@g@{jNYFe zzDvb7@J6W_R6MSxHm;pS#4qtCTr*|#JZHW-+qSgc#*9rg_c_PBN_S6m+zEXid|Pv~ zeSkm8+ggS@{&O~X^_PXYs6r!>py903dWY%>9;oqx_8fW>Gk~8t{B3~QhI(Vot7zph zC%`a!dx6_=eHfIz+>mQFRnxI^=P8h!)1Z@p?-^EqRC=ely8h*Zs7R{$<$ZDJj||Uk z`s@`k#YjMoiEd}6?xIQvI^Lm=H8Hq8(mV&5cMl!2iN(S{8Jn%fC@hU?u8`C_-7#_g znCL4J170s%Gxnfe36r>fa(7vp9bfZq5Z`sx$e;sp`^(JWC1Iz$tmoR0UHxr0t?P$j zlRNI_7dAf-b@rBcUwXNkp$H^LiN1=MjmuVt8?9{t>rRe$I9avCh3@Soy(*A^#RmXt zqCr2qZ%gI-$h1Z$1lomr_tk4|>3s&XJ3O44!tRdl<~pJ;r-?7ek3DKepE_!Oy|xuU zddFTg=x)L@m^{}SYQZ!qLAdVVWF(a`*>yBKeRR=Nc$e56Ai$Vd)Z<{auK z9q-aJ#>dgU6K+5EmirH)D2i% z37o^nF^Wh@^j68Gl2a+x#U9Z2s_E#jnl(4KSn}jFT%5{tpqhc+6Y4setY9jSTI2ybrr>OUvR|D{Xw=9xDxkjP<{Zr@`r0 z);e*S_@0XN$Ow>`{&-S?Yj5R`cqBVU(DT`1=emC75;q*ZcTffS1`0Q+>mBHQviQc# zIXZ&-y-W(StC~YfcpA0I5hoVRd7U9rvW@3{wWpxF}fB%;UmkGB`DQP zZUae|1;cN6wLFWbnlEQs$FA|-YG4%@Jh9BIjcsift9ApK-ow{DvD4xGO#W|YJ6c^+ z6S>a@Bz-*E^=QdJ!NB8jnipkbW9#YdHSzI*1JOr(K;#iu)3FA?;Q@SZXNZ!O74yx_ z4TpYBw5O;F5BYCGHy}*uyfO0Rw?_f!PFyq<8x2eNZz?u=8wNDA{A{&qj(QA7actNT z1St4`lo=7#mc``h4*yq>3D_n_J7P7|=@2Q`(N&Ytsqt z(k`n5J9<%fq-w-~LB9;ygXT50i9c5R&ic~;wL^8tZ%a9Zd3(UX4` z1k85f&QC=}P)`y-$>?H*S)#Ws2%~}{>a6*!Nl97RM3?mHbjR#qxeLfD;8F=Fu;1QX z0k|a~`e#4GL>R#%BGNZCMV@`$$qWcrubmb9iQQf2b06$_-QT^nMajP43UtP++o_q+ zskVO9PMc}X9|^=@^D75SC;$pa=HQ6ob?JXRKZ0P_@ZQhFcwxY<({l8qOeYe0?I!^w z4NOW(0ss^#Sy}X!-b#+cqaz9s$lT2>=_Q*Cus{KD3E(BF8yOAny=EH%Ht#4wLBWOp zx$B<*P4L3v@~g<_bfV{rK|t;epkgsGF&swC@Mn*o0jLZh1f$8y!eR{E)nTS)bZiXJ zHC}6f$i0H=HJtNPm@|L&9jG~b`-Z+j7}5uKBhQ}nQh-oWo-HBW<; zl8Oo_B?CMA!`hj_B3OaEg99g^R)L`Q7@3_N09w&&IfB^}5KjCb1(o1PoP3MK9TaXnlAj62dPeRAn>HgpG~8|F&smW##<~25TFef1s$p-`tiP zG|B>|&z$X(05~$gP59fJEkJ~T#;+xN96+0X7uDR#%JpieHZd(t9$;=eI5_-|y3wXY z@i7oalmkx$xa*yJznZUh|J7-KHt!G~(_cXNKZy8aCB8?0fn$G{2-PG#;y<7N^`vp7 z|9+Ju%)J4^(a zi|!2<(mDpyc13v0pe;$pHXL0G)wvuuHwbh7Plx_B|2;@R^SQCL<3{EEI$R zFy(-L1pdVfr0a&@S^fO}o%p~L1W+Yx(*LqbL`-ZfA$G|Ag-_!=U=mBtp;nSZ6 z6--va0fgC}uaD2#| zOlL8J8R$$31UL9xj%ODDpyxOs-M0300SG1C+4<92u2~~J9FU=V^OBkfR0PC5@+GB# zBN;mLO4+62?P`$SzAL3ldc~>3w(b@ zeld=lDUZ6!d4yAM$dd%rA2)0b;2W+{>;2Tj@kB5nZkTSg3aAYIG4MG{4r;-f ziQ!)9yDlyg##A9pO_qG5UG**J{V4sFm%Fz_e4ZrF=kRplhjkl)99cex+ST}lo?@N3 zLEB!A%u6{~_WyMLZCzbupH9uUXd*&>Wr2eYEgg@|{Fi@?N_b!tUuUX*U&7HOPluS~ z-aZ3Kh-CE3;CHu@E*i#*1T@=--ps+DuV`p|ZK;~fpen3RO#P%OyrVT~%(Wn6aMoI| ziir0|`|BeT{(;@LUuPQH*@$;?G{_f$$Ok!I@D`D5>8Ca0;sp6_yIve6Lx}aj9yN@H z+PysPWk7y!sO#$!rb2rcY^glvT*xUVkZ$>d6+bTdiuRz%!K|r>AZ@{iI!v?ju&j8V zQ0AcG>8LdQBCF2TE_#-JL&}k$dn&g}&PArI{HTZxD%KLU}o; z+qr$sOY`;{#SD}HbxA;hfzW5i&ZuMW%vA))5j`}d@@FFjkGqaVdk7wk+_dB!j1bJX zPaC*hkidlsiP=SZ=sj*Rh6PU+-TX7ls_j?iXy4MWif_!_V4jH{bL{x$=FO%JNZe^~ z&6Fo3;3V=r5pWN_+N;kfz48wCiq!G#dRn*LXzPQ{U5=FA_S^`Yz1!z92zg^abent?E~r`i5D#KWypUlfdLEmVWNmWfu0Rgh=6+=Vmy%_aplidhk3e3?$b|7kbbh zMuo>6kBLF#wdTAGn*Fe*Z`I-3Y`>x&Vp1+X)Z%qr!vUmj54=TTy3~FXJ75&3VPVJ^J_7v@MyJ=c?9; z(drhBpTpo;i>^ZKaER5Kk3wzjs9@G&muN8f8nTJGBpH_KLQa|9`iiiVnxeoABI`TN zA?01eyKBx@E84TTM2(}~pZ~)P=c`zMR*H$eu_u39@tdko931B4FH~`>Jl4mzA)s!O zp6(*UXK$c*gL0Shp#U zsj~QSu!Qk)QW8=T@xt4c#l5~F=t*P{RPX6W2p-^8u>ns395cYz05%s^8317tNNJd( zhWW>GJp|p{xQ9P`jMjYPnt|_!ygOWDv>?YUOr%*GrJa|rUb4MC?Vw_PNLg-A+I8!6 zNGs8hsyN}3ApGNG#p0sEYQR+GHQJPkNHjKA!y0)0%@T(J=&R>_VRw8XdnXfy#w@U5 zP$!S-?FtQ^McImx_=QRF1JKvVNCUELAitpfYmsb4%d2<%X(wd+9<<)vZ@k2HUdqY- z>i4;0&Zslp%y$#`3WnE)eikx(7CUL)JN7oRiuQ?PSOu2xpA)9DE!WUJZ5vKBaSQ}h zQ)l7XlktSa4kg*cPk89P{SF^60U{~F=!HX(%nYQZtrLu6m}2HE1(OcL(5IRnew8GT*AZ}a9Z{- zZ;gdzval$xm@6{vv#dzT5?65-Gn;|~^}@`QYEwUOKU2S7?QvpU!ofg}xyX&j`*RZI zmVgn>uzQvSH6iF&jvj*9UJqoXiNgodp#NsG0-ZA?2<#()phhs?pYG6-K}xzgg4=27 zG|Ypy_d)Q2qo;(bN1b(@3tGNp5E7W>^mwuZuNs_p zL0SCVsaTYQXc*Y+90jiRLM3a^phkNxP=zEkX8O`~LL6mn=n-nmAZ}Ub)Jjtx#eT^Q9bT$xCxXKz7 zt+?R{!4;UYX?=j;<#~<1SsfG9p(yP3mDC0dCf%16!OJBC^cXcGspyl>gFS!B@bsG=E)vMM9-3?Dz?H!i%)L%7;W&8X zPp$dv{7CAB4MjXo!8Fi)FRyoeqO$F;114ALMJJ^Dc=u7ISJo{|JKLj!nK!9xiZV6h z5PV&+%|S|+mOTCwN|wS*b2AueN{=zu%YhV6Jw2E}{^Y6{fGp0;j8Qui^M@_1!`$4W zgBog~cXJYpZ4Nt~jeepG@{bz{n{g`b!y;awhT-6-ykN@B8>hK83|cT(1+837`X{c7 zZe4`=dtnU{>F7Kl;$#NYcep&AzkTIG2}(+y=~#q1>C-s<{dI81i8ko|~iFL$-x>O6a9 zZ{?{F@qy~yp;q7QpPK-9LKL5|55L01tXBzlRt#cmiJVzvpjnz)Nl9htmuOGVHk(2a zy7215x@u=quCk~JX(W8C=Sj^@iTO@~ObK6L^_ydG8V{x=nL3}h;BAFgjbJUI`#pr^ z|3lkXM^)8*?W#z3cSv`~K}rOqL8PS{Bo81RN_TgcAWC;h9XbW1^N`ZrbvMTQi|_m6 zj&a8w;~*aP+O=Y?wdOPD^Q_v@tp~uara}ZKp7(JSE>KmW4VjD-jWdVu71%P$i(Xop zs=Hgb-Ef>aAO$NmT*GUY)qMn{qyn$5rJo4rf4V}qH@8CL+F+rDUJ!=3(-$X8ct#KA zUz3{}g9bXjBYf-}XK%=aMspi!8jRk(L&^~%At0vpr%3w>(7=maPY+KudkO$3ZKb@a zDWI=h@&M;v+SoMI(;%G(}5@5t!3CJdW~z62&G|PqmWA*Vs`H@MLfE( zH^RSe6IyyDxqCHdy4v2b^E^Wr|F&A^DL7r1Eat#PKgeXJ1V%fbKd0L3UzGaf9>juv z=VJyhs8K?o9E{@vVu2*~7Qr6Z?>C;?jaehE`?p(p9jMnOG=*vsvL38f-u3ZSTsG_< zhudCW9*5#Bk=rcS2#`}cjLd5AHa-evpcsno$~@lPZ*=YVKRvOzjdF zgA_P8z%>(*-sSXY3|Jyq352iMVZroswt=N)dAkYW5-gUdcx!+_s(cH>6u53sM7Z@V z2#pTds$2NP0?@~Yk0!}WpsAX0U%+euK9T3Zj~s|3|LfOKcrup{xJ+txw0;l+z}o^$ z@r$ZJiu0t7Gbw;E+>`*Z6TL%2>*vqnWdKbSMM1IxAn#w%`)W)@Mdep>w%1e6bh7Qu zi;Z7?P0}H1uaH|_0n!RGJ<|#_0J#6q+x0jGSk(yhR|JVu0$dSKK0YeNpRL*4+NTB_}j{{Pin1Jnll%Nho~SIp&J#@itr*G6^Ve^N24gaUVg;; zO_AoGGtRwEN((C^4r2G``(oYU!MH<9K77sT$H$Fz$KFmA%UkE8Q9=XvH%r}ddN;zX z8G2#8q5QqlXx5%VQNR`RgBy4_aZ0G|GLro&f|y0)t>btj*Yr(E1fOoDP0H)_G1F#o)s%h^U` z15up8Cxj3j)G{0h@Q5_u?%FE9dGp89(e-jZ)nzHAw9;q9W2NoB_jDzM{bCGo9Rtmb zh=>TZ_Ylz77Ut#@0R75u{~OdZXM`SL?F9V706DaX#tCwu3$t7nUl;{Kdp7`Q1P2Fu z0IU*NzySE`AOB_GFU*;N-*6vZ(~AR!CG#l&<_nTZ{_*$y-XsYV0nnP?9P*O6S{b%60!W6ky5 zMUV^_ZGijif4f2~pydq6dl>NA%=Nrjb!to&a_6m`vq$dj?fuW<$-gce3uwOA*v!kg zy9>aK14)n%5;OmMQSC>7TM?ki2XHY%0=SKw($dJ&?MXmw;a@2M598be^iZ|PABKTo z;(j)>t}sGi2VQS(W0PM~lK@2d{;Hdcumfs!`+W%dF>DMy<8asS)hpm>^an214%^C( z0<Vf-SyUIJd4CqT-n%UZnp1%&*rwZ6g{*gyvTwoLxac%dWzyvTtt;NU)W+f-De z$60Ru2!r5rzyJ^~#qvi$dP%yW#(+%(ATtzdp#K5gRv0m21#nLTa?T#w01IARoF71C z_1mz&1VEDB!NK2VUeqO1iu$FV9{FuAa4`TDq(I07un{1F8`RVfc))?MUKk|{3xy-(8zQ1AUY86ifEKira$Ki}Diue7MpWNXfnAhD05?M$pl zvnjpdjBI(&xGx~w;Kz4E54;XBFElHy;TlMB$K0lhTDU>SlD1$PrTh66bp&kP{7@UT9Uw@|^Zirn&>wScVdNqq zO~#=^5G~EIEdvjyr0~@95nRPA$5nA!v|-39se)g_?)Fqg%VY2Fv>dq~IAUp)Qud&q z&9nTAxB1Fi3KPRdqfv&Dloz1}t4WM+1ku_ep4ZsSbwJN37%m}&g@muXQWAmEs9Mg^ zajUsD-YkgzMoAj6va=lG)2ZlcU{?bVs1UVjpI#OCulf7&jNiW7?PCaNv7mG%M`4+$0%v3@&v zB8{_|mkE1~vEI$@Sk~@~%e{%tWgHBVaWPg;s*|?@nbR{jJDlSyHfZ(4Z&k3EH0aVE_DM57$@*5>5Z<>iB~D?b;oh z)QTG%CtI($HR1xhMQim~!a}1%TQcL8f&dP&PFl|-o-%=Kf%Io zKg{rGtOQbgr8&$NYz}O_E4pej?ejvsMGBk@}sf z4x~w1nCx~^58UDOXZ93Uc0;!*n9%9OZ*d}qj4|g3$u}L0kyp5O8$-$H1`8M2@zOpm~_{1AOZ?7HelxD$Tqb)-f z?|orYRB2yx7R8(0`(^*|_i{r=6_#Vq`bw^l_fmAjX3I8(4;b&a^%5&fHyJB#TQ(*w zI*cbUCu?cOR zWaZ4PniIWs9FwY1F-amSNdx2E1$w0q?Ye)w^dQ5B@s|)mIQI|gT1oOKi;IySP>6iO z8Z#DSA&jvc2G}sHd2VanzAd475|cVXV?5N2oM3hPz>!W!i>BlU#UP_GP4B3g7j)5} zOoGH&fJlr{xV9Ib%g)_1FOL=q%vldpd?Fw2rX1wyIp47> zE`uDS`R#%*BU~=q@HL0(@t%pA8&<;D4Pa zC|$gixCs`8v3Bamlz5LoaDVfakm)qX%a3F;jg98-kj1#=%*sMNwI)`Gz@*Adqf(Es zDwvX-dU4*JA4aIv6wQfhnG)FwE(U};cfkDMA@+2Af~1wLadhFmVWlv?uD{RC%@?HEQI z;^Z#%cTHC2AsE%JiUUp>%>}*|@q}TFGNxH&KjEt_b=L~lQMA)SLA%$)ASOUTe4drp z%6wIb*cnrHaJgbsDXuKs?)2%G3O9{1W3KK`T&_bZhLXG$Hgs|+LTGyD&wL8ka6ZxL z!vmRb1}yGG?&XZM4ic`E#k-;vG9d@`R?pG)^&t~Nj@=}bZI3>_#+j8i>%&#%Xh;pZ zLL!Vye$iCw3orwem1c*6Xc_oFg%R?lG~u9UN5H+kma|uW*;L^o1fGTFa$<85zSSA* z5#_H-qyJV6uldbXAl+a=ne&XZE6!vo1pf1UO~C2(O!;H3>M&WeFe<7M*h6;t#>sLw zcNp-m{sqL9z?Ul|-0mSxknYi1NtUDH?^X{^5qfr5@nLUxr)!+OtFiJ-S7mdCG8HW! zxgOVBX+QUKN1T%WfwFIMn-j0i?kGa-)PElMMisS$~;!dMxDN?q8ZiJ^Kyt^oNgLZ72o6-cY*p}=n zOuhZy2R0CDCiz(qpgaW(zu8ZXd`0?IoCx1tP+GV_(hnlZscDUFNs|ttIywAq*IshY z6b4_dU%iEf%YNCIavgJGw~CM-OhkpQF0o=@+cB?&fk;@6U}JX?!*Zy%2bA zZe`WpOVZu}#-$F(To0cgMER23=U4J**7+$R&wD)FJ>3~CEsIk~7NRnN?$Yvz*<#lG zkjd41J>$0wKgi5!Eq|H(6JYR z!ZU@+KN?~6B?+;G*L1xaC{hY5{6f81cyG8F?Xf<0z@Lv^0dlOvWO4=>NuqYH2oA)M zJ_9;XQtidT-&g?Ps~@uNSZd_a&ut-X=V6b4wJWJI{oCVW<~4eFJsXF=p*LpOJX=4{WLBv9`+#p>7v#&KXO|<3^WTC!%#A0WME!Rp0BWa%(D~?bKwK zH3W}WQO=jCP6u;iY|ARcW_N2Q)O42!``*c(@MuTb4QGcxO389dcHnGMlq1-lJ&vqQ zjTjt$r0Y?LSKh$e@WFAxK;XowcI*}75>0NaY?&3M`aFn;yqmu!+=$?W*Lu}4V}LVX zk3R?ze>nSFWxP;-=F((~XFP-QJ-k?x$2viNXxxC{(NV0cqSxb2f4Hxix01Xp^Sw9a zV^WZ|2mylpV1olmv>G5u(Gc)u@J1NuFX_*sNfHJGTM#w%KV;Mq0B$3QQh$kcfQ-MTfML(dtfErE84y0 z`jDNIa1EMCj^e4|{uVEv+OwMID1o?bn_Dd97!W+I>nE{#M>m>4-AlsAQo*3zI{M@r z{=H^Xyu^q(X|8>{m(1=bdYT^WNT$y+8}FIq(!3A_7PpMVUE#FwZ-Vgo-eZK}+pn=W zNXam@b?Pqv*{a}IVZ3)g!yx1ih2qlSxLBp2TN_>v6%$YuK*hH&HqV_A&m8_(sAZ>sK&3F#PJxbV?wgn>Qer%G;7D0~tpMkS&EoSxg?s3zQa6jv<4p_O$ws6@~joi>;FiFXhrXKXwh4Vk952 z++7}Hm;d(lfb`C)PrxN|V@pxG`W=n^2&e{L8%5NL1JFuPrd4i1~2lm8JLxpoLv z$tz7Aif}V*qOuvo3|f}0WYy zGfrAXA&lU=Ah9qi{q)^ZG);t4ijAnPlPaO?Ao$);QIyOkF??2`eZxAJPCvGH+SMVD z)7_0@_~z;v8VUpLv|X8w%_|d$U^pL*6*F-6TI1zBwuqs+0&lZRl#@EEdMzP^n>22~+#XMS0EW93e{!T9}c_cUlk*r`;f)slFIU5Fv9;RMb!PV$hFe1V_rx328+HsB(S+ZI}b8ZjnPkD>N7kFcjlVo3+foa zDQQ|sMGP_&*A%wnPR)4}ovPY^~s<_^#6dz>ls!fvkVzBxiy?#Ppuo=bF>&^!0 z>-iX_{lTZTH)<)8`_c0qTwLs7#;grcU3PxS&X?bRO7Z)dE2if|OfyQpBS-)pwz zJp2A^#+|ZGxq^X(iHeWqqs!+s09y!*%VCyVfXWyG;ZYvNjyQqDvq8{jLbe#VpInnX zcqQBfG!HWGTh|>!ojV~y+|CloF5BvL2HKBVTi0_f6v%s>ZIfjs8sOL|FzAQ$Mf2J)5~xq1qKQ@Ic@*wWi{j@f7g+F-?yHZ`uXB5g%p zvhh5vY3liMMY30ak7k;1&7PK>w}`jYn(h6|1?i_KnlmdwL9!wDoQgMzmLdNmav2J~ zF80kg9S|r@I%oE&?_)>~IZjw(8ZjN79j_C!3^fr1ssoqMEe6?=IE03#6^2-7WCT~)M7LcS^!(JU=|_!H&p%X6OV84DFHd|RiHt|>)dpP?yQm|q3BZYQ~3eRAYEO>Uije=<7 z=iE-UFAQNXM(Lf1e*}=S3!x0kQjwU`;T-CN(yiVe3T`jD!7@FI?YC_S45f`QTa57C zLH^@0($*+C!Rm_KcnXpClHdWt?5KUip4`zVFZXfasM`~v`+lW8ARD8!{4Q1DB9chw z-}i-RVNFKLpyy|BQ@2!}od%CTNn)I;+N@?$!quR^okT0YB z{_#?6OPDw)uO1~%b_W3@(?=oHX?6m#neJ)Dhe~r^_tx5KTm+f3OWMZhtaOVT%b?hk zDo4r@H#gr8=yr5bs@gJYF`>CAwW)F9<$Py)4e$P#IQ^s76485tz=UmI<|;}T}3k2MPjW=At0bxV2_%?^G(?YqS^)k~8} z!8UVHCg_HS!I{wL8?wLFIPuLJAhEZ%2T{?kBPWZ-wW2}qD;m5_6 zjMwo}%fWHxT|x}}T?hwka9a3oM|nDku3uihjazQP=mqnF0;mscgKcYyg_!n5h@cHj zGDOWX4#0Iy{jJYT;SnWCb2x|rkGpGR|884))Ng1Z)928{qDWw81K6XJm>Ohs?o?pi zo#VF#cTd01iVx%s2%1S7sS_+OPD+7b_$V{oAVx<`e_ zWd{T$5v6tI=UTv9PN1*!ZK{K>xnbz`%(_Dor!f(*4kjy3s%Zw&1;Vv#ik}&}l~EQK zd93l&aK9WR$ZCwZ=&`(5b~&JKu$Vc63!U7_VVLMz=98`W8Y+%^>Tahke6Vi<9j&!M z`V;YsfFBd4H0N-ySK-=E_VyODiHi1(GFL*oBj3xG<24og!3`Vz{+64VOp)@Cea~C_ z3g(F5S>9-uuoV&?&-s(n%gYm&o*ro$8@?6v7fM*&;y#a`+@by2A6(#svB}f0jmuVw z3~ahR!6ehOkJ_avG_Grl=$uyCD3~B$k*kOmlB1V}aHGH>g?g2pS4&>prIHP)PA#?* zRu`>p+?V@CwS+5oG)z~oItx}!OXk&vE|gG|3ST-**78VyQ7yH zGHaqQDdyRV#Bf8~Wvb@=~n|IYj0222Va&){1HY_t5N4{~%^%dEo88kDP;&6Me zwYlZ+L5;WN&^?9MY{}^mEzmA2^la##4MV#>Eq|>~C|l`^i&a9Kqc3L1ov!^Cdk!tD zorS7%9QBVNi6OP6kqupVlvFY0hL6#9r`oe;5L7+hGQ;U{hn_6nB&ZTcyat0YSq1vq zSeT}&s{KWkfBEd79!z~w`s)XB8D5QuvtUqqi$F)k9tL?AyMAAkW5DVaffTUx{rokV zO6Md~3OyWEC{4PlH1?`lVPSl7{FNR)9pX{RSw00_s6?ZA(MIG<;WqTCz$;rVs&6D4 zwuRlpVC(SHv?UD0ON%e(4k~RldeE=RRtqktxK2OdrJ0#?KMTgLlU|pRrYiG`U)^w@ zUh-qpk+m%@__U;tfq&R`gjWt>UKk-ZwBV`rV{R+5Mn2iu9i8af-a7 zrZfupcisq8P}VMWfIL<r$35Ub_e*k6i@?U_ zl90?&SC{#e<6hR;m3d5~kaNWih$9n+-i%_ZtQHs$0U?n}SoV^f?_4m26Q$eALO-+0 zP|j`BtuvVnsa6K@{D|S*!lG3*HX{0Ef9-^6Ty^41=tc>H?CXSj>e)Xanpn(n<@${l ziQfTfLupdfKEoRjc9bjM=-Aj53J#*92{z-=j`Ovb6t>!BCW_6lwperKMT*EgbNI zp~uGWn4a}mtw;3bQ*(V3Q3c{)fCSpV(Fh3MSab@CT0%nB|CSt?nbeWy@jUT9ihNIK zz(iRxiU2fl#JAASwRfzo5*_(F<}@<}zyfO(SdI5STk%eumF>Oed!PR7r4n)A3Kny$ zk0aZPta9?}gjWh7@U{K=SrvZ(kp~ylzqB6{Q1Lk?dKLx zH+6rT%Bp%DxJwJ&5|Z7uzL9C|pAWGLe?Cy3C>4#)8#YxA@;~dtg#VeHNcWHj^`9R* zpa08Z59qxGkSB{5`e#l^YyT$)YxwCyV89rVc?P7E)mZ<2BtIh{o(cRXblvYy5o#Km z=%gg6e}XnZKp21^89ET`^5?O<&mMN(3P89EplbBzZ~Nuesep*x?Inqm0R8UH`0qUELxJKLV4$D!2aIpO29jaD|7U9G{|gfT@VpmKI}AC1oTBL#Z;={n z2i^dKsLagaG6M*ZBl{oi*IqFE5G6nk1Td87G+vCCdUdW3rUrK!dJv3p_xAU%Z0p@W z=i%XT`S!aD{jCe=*hBE{_edflB9{wjpyoF>9w1E<$OYpjq9F>$3V#J;)B?@qdHq^|?_K2s4wWOw}%WGD)XH-Dn<^}SjRuYJ)&l){7yQrJ-zS*Oac! zf^b-&UdC>*42?gDnmr)KqjRiskR_d=?@$%Rw62*AudYx_v{CGuOA1zJP8zCiPNObv zxD9GUgJto?>BEWb8|WZQ)p;ks-smywa!DiG`rt;U;cf(+a1dQ_6k(+qhDN2|xMT_# z#MfEbGLR?<>w2}AsgF;te-eKK36Gtsy=ZZf((VMz)Z0F>l7L;HGo!aC=?3a)N%xL8U4 z_ucl0>cgwH0+(zXCa6o8^K`^u{)D`-8&|!>Px%XGasW#824B)Q{(NWC;>1Zv7CHvF z&DW$2mYcMKry`I0wqTwbrunD#F-+R8>=-!iOt%8~=FedF9+t0$?M1_}(fY?7{i&Pw zAU42TGrQZxjVdhn)`>Rv6`JXGmx%{SK!5vg4f_Q~0FZd^+s}Q$9?&1&jS3_CaO{e-jy6HP%O=0|k#xq;5_6_9ii?w^#4-l&6Ud6qPo&xso z7=WVS;N4O)IAbW&O+@Whf^zNf17}HOLhO^+O^CG4YsG!_W+s5#lmDUl7h_~Z2Yg0z$lE!4F{D>@eDnx-F@+-~qO&0^QR&Vb)MV#Gx z>+V^*mI+;eQvaF*+s%e* z&aA88wD@Dc>dLBI{>}R6sNq65Dta;!CBgTb{;Mhx8%I#Yi&>&+^;wyj z?}Z-O6(7tpTq0%=_Bo2g-<33{U*?XP^LqqExFF@4dPVdtVkrU?cZj9qz1fdx5fp`mMN1A=N1GOV9k{+PyHv8BZ zJ@JNain;pCNnKq(p5=F@YKco|eC6-jDw`VY(M#GHE3oD5KE2US2(pA1H%jZ0wQ*u} zstQej!BoIUzp_Aof=HwN+!g8aaPa15+u|4&d=Zw--Gq7)$bhr)e9~y?`+H-U7SB~0 zqIUL`X81bVf>_vLk=`!IGnsCJIrm%rm^Y)3U>eT>W!vkgU(c&!lSIO{ZCgU?W8n|` zRhzm~g(K>*?WY@eXcBCMUk-3g^LTJhYJ(At)U8ZbX7~zA1`NAHI*YdLz-x)D?e6A; ziY?{g3!iwcpbFwz+xfe*_r43wx-&EDA ztFG4M%Yi=A_3FeG4s0;YpKa(n}4}mMVg2E(UZn?lNo3|IQiJjR}bvZecXUXrshPMvYtqyhg&cx&q;%y>ODSv+p!- zet1_}_tVZaTuU(%LgbwR=FKmHt=XlnBBjFDAmmAMY9fv@&ZGLBtHwI}E%woGa8A^% zCAGS%=fi9!w$5oyeZ2$p&Ip~qR<}@(uiM2dR@wwhpboH|-3Jg_)K7*-a)7`7^rIUI z3E)Dy(_x}f-f)4aGstGJUQ@Ali8b}NgQ zHUhyT;usr8;}jdsmk38xZq7A?mwYXus6MNn4GvxNDMB&%MV8XD)y>-ldCsUCdac64R22HhNRhxFz7WxW!45v6;@@H;1mirm`fv#~iMX1K<1J zw?|nCY@nBH_%Gf)g`A=8)n$`^+$z3VuPu!cgxWL<=RT=~vZtyQC$@X$kWA+ zT0$wOTtIHMNo@inug&0%B5M{0&M(1~;6a2pI>8Is`pk=)8Xn4w%obWh)c>OA?oiR)k1<-)(Is25ko1~ZB(?9X7vwz^>Tv!a}8COA#)?8;5% zd}l_Ot3;(FD>{*F8>B$FR+-+ee7oq%Iv3jO~H+CND&ey~fOX7}{nc?nA8@Vjiq!OkX1bXq)>(FVmod4->!ltZ) z5>n`g1|5#%`d}W`)X-+}6~GN%RYj6l@-60>CSq8f1*SqlAimzbG_yAWa^k{ZkRWV{$(E&C@C-|mRzr7ylujcD;dJF!8!BC>M^IxklOg94uA(!&Ri ziVr-&!KX>qeR=mtRD?VR)ma8>&zSgxU@Ar^Z=D8OIvZ}nsBKv7gfVXa9isF_aWYE_ z3C;OQVIiWpFWec<0^X$7X2d&ct;btm6<5Ysa2f+oPH=umC1q&f=kq0!MEauX;i%A= zB=^$G;P3yi-C>#IDvdc^ynfvHl2&05Eg8)x?DKWyz3OtlEEuGSA5XA~DFVfW zDW&zD3!;)t0w+LUAkuxUC`Eg*1S!O|BZG(5{ZqAI{-HRM^|^aKIfD5}$i_SK!+!*; zPc)G-;_?Ely6(sJgi} zK3@9i#&q)r0W2RX-E?wm6tSs+#q`fkD#DC?C>9gmQb%*T3ZIic_`TY^E-x8L=bRE( zGPyH&RSwx;OjPaOK#n+Gb}++)WMT4!!8;zRf`tM3-1iGVLt>(o+NuoU-=`{$%|HV4 zVkT_ajcECDGOVaPm*@l5yZGyN1Oqe&Xs)JjNjAwKJ>|+j5%+UAmh&7hK(y$GW$TkI zP*h~Oad`#N$A|^*K=w_^utv0;&5|2d4pALmRLPzzuea2M2bjVOKBKB}T6Te^lwlZP zlHY26z}j3+Z}1Q`lzEI?SVxekY zJkHHqWw7>vuKI)0m<@$X7Ml9RqkNPzd#5oApWfVynT;;{ciT)mUb)0H9GTRTVS9V* z<^P!PHo6qi^GhAU!;b!{WT%iWMg8RuL|TAV#YO|O=#f1AyH2sTSAW*1Qs+~byS(w{W(f`<&bNb=EpS_Wt7a|q zNKcL1TSPf8lv-HJY^W{XZNOJ3P#5tYp7@}%(Zzb4 zc6~UtFXk~=j!D5;8$Q(eFkPg5#+qi1f5mB!%q*+xc1Ncrj7lD3oq#EkrotTRxW_og zVa0kfq+H}-LONcIS@F@1`UBUe5D8h@6-SF7yITD-p5u;zZ#5fR{48Q^3alJ)tXQ9% zx@%PL;xr@tou1q168u{Z+1YN^l_tn+J7ammV39Yt=x4*1s78AiHuTn1&QRyx@2e-_ zBYs!${D4n>Z>zfq4$@Sm(N2wxn|rE_I6ZJZD#FeWX_GKbQ=Zi`l{LdcjCfjsgnf9* zjfo(Zk{a6Hrm$Sj{`-!?rnnZk#;z6SsMSXbI>lc#*5wVoMpSH=lsUp#oqiPSYbtXE zbT;NSjuy(5imR$zA%$;C?OPak%eLfvGli?nkEEp=iI7(+V-9u63a3g{E3-c*d;$gG zP2wGoS32w>lT7cmWf#(au;@U9ed4zQhyPOub;skBA;Q)vw-70z5c}vFp%Z0^d-(P| z^7V=+EC0#OFsQiW;Mz4-y1%~oW|GYb^8}v2Un1}HX=(Ef5vl=k0wxs!0#-2b=W-aq zt7B~OCy&`t-$vMOB&R@J(uk1sx0IfIq~MtM8*-btIV-RN&v4{=dE60{-e!N?wV~sM zl~bm}Lcs`o`y#3#H`OJ_RD?AUDL~=J>oTV2j~AYe_Kuu-XW0xRapN*mDSY3jvpUR2 zK&7iW+U|nXaT6oq9hzgbh=e`nwz&~noXy6F^o5v6mZP+biev{xNW|?jdG$w!vs#59 zC3U=f`~*r*`>&>rF)W`G&$AvFy0fT*@0Tx$zOj%DdW0R*)RW=CuClNeOp!b@p(C`E*Rya=G-fmejILXRi0{ zcQ)-_3wX$vn<$s{j9dy?1+kCzhr|4TZXtkG6BM!hUm`JybI%X(eT(=x*0Vp$7y zPDdEp+=n9Oxu`5eQh7ho^D4{Ozqz_Uur=!|9B{H*@6pZiH9PRX#FlcV&kNJ&80AO3 zISVES+ni%)-w_R|^JNLV`7NM%z~L}OA~%>Rqg5A9DlhO|K7GNF>ZKW|z4~*AjH(kZ z*Y#jWas8&Om2T;|_mNVW-0}!Qwq1}NcYwi20d82zEAL`|DoKU!gc1}Neh-d9z`d@G zUao-xWE#%ydT10E7eC0+0bDzJG3n`ptG%((GBPiJ8QZpkBtmq1`~FbDoBqY}HEljZ zUz^pBY$82#9Jh8;I3@6^ssKD=o)R!`v$0~Ri3eaL9UbC##v!PZZ_vb;`g&!`z^Wl3 z>k9dK_xe|6bUB}idNn2D64zF$3bpe-gRmJ(Ym>q~YQkdl?ls7OBD`Woap3K>Bg% z{&9cB&%p|2R)AS{fw@1YcKoY4m(z|$2|SRE4Di#g93B{u3c1DIUqs$#efs3zY&=zF z;BmXHSCEzU==ZuM^DUfY!hP8N{bBWwT6igv`VY2Vr(Oa01uH7`HdegEZTc7h_hR=7 zbOoTg9UL57U0eIJ7XOa_Ub1m|rY`>U{k8>i?V;DGwuKy1QrBuzuW7Sb(|GAL_m~++nLf*vQjcT=<F<7Pax>85(m9F|{E!)J0`u7qqC@!9k8tsxS)F%e@N+@{?40@FmgFYC% z{YTkxGXFpB@ns(JvtEzZ6OLS4mqwbE`0v}cRa*e3w7mxusLH0TPIy9cfOW_3gTf|O zXO3NLO{s_fx@c+$uzA#`1Eg&MzTS~U^AsS~ok3b!`kxYiu1g;FyC84~$odV8i^I7J z0n{sKA4b)KcFj9X55SQMp^mNn*|`Z|0J8pX*I>J zqWVA2;r?}w{qyt(cn+Yam?+WdU-;(!fElI%P%MxJ4ulOQ#>4=`e`5cA!=>LRJ3LQE zA?6J;fCT(oUHy6kkE753{Ynpiuao=tOL^)5OvmNrn133jtuZM-xPNWgZS-#^1TG<} zw_yhYLUUE=(=j`XkNiG~0B!~A!pJe<*H@~@t}RLP{# z5;?wIJ2nMTH9>7j_K$^wmN9tPf!{e7>uZ^_huScAw^eHF_HfCGg|P83QM22A7clw_}?x558B{f)&#SHnT*+NNhBJ z1b?xM8O!NrFBo{s8Q_aw`!kPhP(o*5-k7}=m*x35#>Q>+j#rGW*EN38f@i?ci=Zo` z?Tb=!uf%S(JqURvJ4AGML8PSwM5Mcv8X6>|yQNDSL>lR?p<8n37#fCdzHR>A?>%R&^9QqLF}t4q+)vzb zU01ymdUejr)U#}%+aoD~Rialy@)K$OH@RMIYZbWv-uU_doKDoA?C^n;)sT9UDnDO# zpq6*J@g4OIU+yt#CGoLD_EaZKm!LzvJWWt~&j^0(q%TEt&k^#vli9FueOqWljV$ft zwm_CM!=RZNic5V2aoKrycRWLWlx8GOpvfHq!t-SqU@)~DnmE~Mj^d76oP%_;r zpGs(Avn_HZ7`?O0#JgmiL@1x_EOfkZGa(!$XE41h`t-i{t9bE^z3W4ahETsw>c)M7 z;30xjx(H=afYNDK+f?M_V8Dq+|L?S6n?@Kf$!2ZWhi{5hrR|QxX5_#1t@^aWir?Uyewl0*OY!2EA;VP9Hn+AziZFq-mNQkO zxif#?ov95Iz1eOVnZIve@oXFnk}e}8AzX>P!{_=U(a}?(9op)eC^mVeQ9zZX=V2j zj}0a2-BHHH#{Gra-C3v{LBf1TXrxG6`V|F!4n|u4;{B>P8mA&`AiyZGQERv^T~(~6=4H*^ zqbu2KwCm2w0eBOT3CUebZAubwT*uzu0&eZ4VEI><}z6+eh0%~sr_Xrn0H zn)iKpRP|juylN<@5JmrqrA=;LKZPJC;bZ$?y7DN9A4>#jk%6Hs3fW-0^BLdA24-4} zo`;$bPyucv#~&3Nmudw@HU^U;V)DDZpH$4?waBeA=ROGu{xMA-QKi?suDR7I;R>FH zd6G6Lz^((uhMgHTw7qF;+yjXMfEjca`Y@&ruOS@w`n622SGXzYY1-&-KZEM`3?veELwF90>t*=negU@pu#%i7d|1jv z7N3a-r* zN{hJS<(tQS)@F;t*@U*6(H$FkN4F>-5)D^~0*yFmXSZgFMa&y5OQzlM34fGaIN$fbEPP_?O`iYK*^Yb&kH;p+5Ab;`A=Bo<2a(hn_E z4z1tNSu02=(pTeBc3RE9`DJz_GPhc##VH_sq$V?DG*Ao6`EI2@=*_}{!iy6MjB0tD zvCvQ%+2gSZtfiQVz9(zui2u3s`Qh=2gBOJ3GJOh5KAkZzcNit_jx)&@LE{k{h~YlL zipl+KM~swi)K9PQE}o{K=z3OT$l;yhRw9qE!6?4q@a?uEW2*) zZe#FF)1_m?6jJ%-AT{pxGegE44&iWKB6xzQHhsp%l4PTKv@6KkIp)?^=xIHE%U#q0 zmim4Lg7qoINQyY19FC0*07FDnhJTjL*6&MZ3GJCg-{oKQ_fKm)dQKZbC|?hTlS)kw zp!)W%DdCd|97`V^q~{{!S?b%Us}6Aa6=vS`CT%r&p|rgjhf}u#TCYO8X;-0eb9k|! z(lT$ROW(_GlN5<8WMh*C{w8~VGs1x!ig%qFD&k`um7np6_cq?v5eY$;vE5vgze}%9=F%qO_gn>qees zpSyK~0exmbcB%!20nB>28e_YN>T_hTcqN1OmpC*7+koCAc!kYIntVgJbC&d%@kBX= znU5ID$;Yi*e3MLE%`xy@Z>8Bkj#?s`iOHp8tgR?RF6&%#=C~@5l=NYw#|#Q176mr6 z#&Sj%Ck?gsZXyMWuks+>IyLgtqer7p9M~L&URo3J``rvk;F>+D@mQ7{_$E&U8GWJJ zBlcsL2IAWPPGt{8_K|NQ7YKXZl}DhV0puEJ2VU7)<9o@){eAU0QNK>8nQG$le@>4( zBEjCA>+ZcJh4YoTJFH~u4K6e`CSFYJewpcFOg+gs)%Y@0%VNB)XCnKee%AcGXGUY^kfoc0`ERAHj5%Hkw|b4(M+h;}M0m?m|; z6q!vX28iEszIRB!;jN2z>5=|mPlbMpUgh9`!_DNzc}jmb^p=R2$YKWro0s|b{>xqB ztR|3+DYS@isX9}Ww%8!Hz+e4o&;@d{u#m*D(}lUJ5SPC|BC zh>s7STc*u-U=vq$)q+g5&m5m#WYdy|Fd}n~^i|{azx&=0jY@Lud*g+v7L%O*N?DBM z%vM-%vSii#113qK(OnS200K@K?%Cg3fVfag#VfTzLqij<_tnB3_9jW4l09nb; zjVa*QYmk16`R-OYnpzLK8SR7I&o#B!@Fbod{DMyV*)$0Ijj>Gi=OQQ}+6Bc1FDTzs z0;=iSN*_nQ)LM|8&+B#_BGBcdnG00}pqK7_p~8j#$Pn6M$7=E@mv3A(iL2hHrw~u* zYK`StfpNsU0Z}hcf-h0rPC^|O28$JOyC259eYqatSK4;IK8hrP`pC>#Og3F{UB6$& zRdk68cB3ji&>kMb*>gE3HEXZsfCT&uD*PVc7ZaQX^Gy-F1IEp6MtHqWas9ZK>32-E z_dI7^s>+UKxa8OBLg1n~%uyn)5hF^Uyo5eN=c9xg1#0mf!~WGpGp%niaOpe|>HJDs zW|*%g>G!Y4;N|x-LQo`Tm5ygaSu0^9gzW*|7X5~6ss0ddBkDrMO}?58wf?siw+>WK zp9gS;j&kB3)y+`BW?hg%Uv#kRx}7mpr@j~GR%3F;#-Qm1*54sCUGhn>EgN;gs#CD6V!OP3pwNS zV^T~d_F`;ARC$83JwYn>-KOcHMsy6qfqh3p9Gut6pSQcoPGeG@QZdhDxpfgs)VAjA z9U2g2cRawPnIWtdmPN||;jcG#?aQ>KtUY6-^k-o*T*iuik&5~dUE2KZ2B*iuJT7K& zesN*EkruD=6y84rz;fTd^bxj{!b}=Jk67Q|{$(Gkq z+BDzZU73Buz<{=~ine}26Z|gCdjzR$c>v~pXJ^E4H=g3MGBdcQa4&SO$PBg==ngPp z9bb6z+h1stZpG)VOTPj60?B=W*Kddw^WpPH2Lnh`3az%gUy3m&zH;@V%Xez-u9m_< z^*?F9%NZgIoZo8A3t*QQ;p-Y~CEl>z@e^}NRpgSS8?*As%9`bJH>-i|NN*Ex849fi zgy$H(!J`U5*ZCRcNBBQKkQkUzd&tFRPr-I^`84^N=UB=(P+3QTMX0VTL6gfUgvb{6da`u90 zu$e#~>Qi!~4B@nS<{#`tk>JYvw!%n_RENDMB-QXoJ>si_OB7Vvgw@xS2YLLd0;(P5 z+fTc{=4*IOwtZ`9NMq=*d+ojZ&EJ7!q3f6tlg+Ht*lmaJyT9Z0Gssy6z+)!Bk z;N^wm+mBI>6m*p`bKkD&pvGj_J6OYcY|C7lc5HM+7UtvxxN0lkKm+^xoj+VtIh!9a zSlmPhZiQ|{?JRawwrKQjl{_x-L3!$R2X_S%(YE-kKA8EuC}~mn>an!eL;4bqryg6n z0T1B35WIS1M}u%JiyGJ;Fc*h0mA)i~n{{(5BjKmsD&DmTF@!6ba0dkU2;%*_;H1Cm|N?aq>#*~>ZA=r6giiTz$_3I&?Z9h2<&!@N=gWx$^?0L0QmZ%$IjBvb!$i;l{GZw)NA9Km@HM#)xG%c z+S@aTpn3U)D(8apT(G4WfQtcCp8M6FNEM4eOP zz+fSN#9#N36Z{+iBkU6MU!J@gawcv}*f#=i);gm37VVN~Hu&03Q~1lJ@>!uwYa6d6 zd|7iPV6)`%<(QXvU&o`?wozuFfnss#kM_zrieDcW&$ah8Qf9oSV!D;-*`}-f*0j8R zr`YIpxp4u?7yGLoWKA%J;XvaIf>Tn}ci~ooEXQWZ_wwh|#v6tW)}Eg5j+|`HHLzB& zNriF2WsDLY!VVvvCs4;n7+LE+DzwJySbFxyg(VNY>s1`O$krb8ek)q3qS8BfY8N`+ z_Js*aQF!P|5y!WPQNa$#^AHkcP-NDxTF!ER>QaT48{0sr0UPl zpbLZHs0XbOh2oN<6FSJ#+GU=)}5nN?jgm@e%241;H? z^Aqx&ISJN)I~51H#7U|##u_ybJL`5MdOu4qKP0IY;;2%Ooez-LF+TH%q;zj#^@MBia`KeJY!0k&qQgMS!4ApH7lN>N1iyVEa_><>UBK}TPI`JUF|)jS zsY#5Aij$=CMW09@Wv9V4mpQ@$#^R)w`F!5@EBy%`(UjQ0l!944>Z3BYt=;`+MEiL~ zm7(ME%;%UAv)Gr6hUh@hM*3>X|QZ@Ew?X4BLA9xjagh>Z$Y8&hv zcnz>PWNBj=&6PIy>!!$ZVRyA>ZLC$gLmQrt${?OBV1~cn2!kNUN&hGNDl^G7)u@EnT3Bt~Pv?tBot%+me(WH*-jFe#~i4{=vu(@nVNpv2w@| z9N7N;!B&w}Uh@(?ti~$=eaDH7&EuJdwf_)K>9HQ`H_FqYE$Y#fa34w>g7_AHv-G)_ zTRKKE_Jt>02di6p&7VB>MZfTqqNVW!LcMC!zRyWln7?Gu1gv&yFE5k(cSgr2@`X2e zXcG-naLxyCii3!01-AwGN2bFSmKB~VcRVxIY>drnEqlY88?12y`yrZAna^ov=NinkJwmH@@_x*=HWF)E;2Y3pks0%y=4U8N{?w)pXgrAN6yq_My=ptk}8-L9M2@yZkArIF#Cd@zZTikMYM7C zqw_owcEap=sE1jf{8HA7*Rj~Mwc}a^yY{lVWd*{T4&@as%4M6A9Z%By3}^pNz=WcZ z!4?c3>bkO(aJd}mq#X0jc4NTxvp1@%xhdgukXt464a}d_GlE zk$98Q6cY_?pyHxQBVN*`NWIi_ce-2~6VN~fXs3aig}-6+M=66+Yy>}56=5zHwE##J z&_i`~b-k$zGyfE@jgu<8_`LKxf#q01RSz#R}9%zGvefXBMOTs$lS-!}$GrBN9HR zfRZoXj8qv{R{8 zHLpH4J+UoteezBG=Nvn6#b+35%Sw3RQn2e=Kn^*Z9%afmCMn5+ax)Y+C3dX}w-F;8 zHSAr`Py~x1t&bAj|Er4Qug6^LGiXXhfCeP`4*^v}ZTaz>y1JysMxj53d6isi0V?1e z;QWzRCAE_H^gvFHqN5O`1$w^9*~It8UsM{X2?5|)>FE(!^xBw2KFxg`Sq?SdN5cPe z$^OLNP86Kf3>*Ri?_?Dxxi9#}@bZMe;*2RcjJH3;eW8j6;fdqQ4-a|6%ezh!y*0L3 z^fMX%zb|r8nZswv#3F0*%K9PnhJBVw;7Ld4L)=yqRawVbdRg0;o$~RipQ)}kLBE%j z0KWInC43lb=dJ%+p3lyqg1p4mp z|2r+i?Pm)85aXu;1?~SRh(LGd=cURhd)rR1zCn{d_YeGbYma|w9@N#< z{nvz_5Y)v8fGXf6FM!!M=?AoN?%V+dK7dajxG7*2W~+P{fG%@YRaKc`7YYCmt2-VN zlvPyxqrdv6MRz+bL3CB?+x=(<)xv$M12A3k(& zDcrq@H(5S)``;Q57cM{B!tx;A1f&9h)6W+^vi|(@dR{Z9@Ka?1I%EPo#$N#M@6G@| zp+9l`%MYXC;v`?aLIG;PZt8$W3p(vC9WHlFOib9G{!otv#^o=te1={5Yn(u*J^h2_ zY_94+b|y@V8Gl?%LaF<7F*Nu;M#-o2(<1Oc0i-0&XWvW9U0{X)!vLT*BC1A@n^j&O z4X8`#y2B*hPPe;&T0lTn_t%LcxdKHzhyB-Nh96zI>5?@!QNXPZ%Sx|kDy5%NK^MC4 zSy?H2foJ`7@R09MnutfOt1vdBZnQl4B%f8j8-9QmfVJra8&mX#&C${En{HFGa<)`- zax$Qk06b9!k7C&DUg*W0l4pOl(H^zgsp<3t0~+hVvq|l?h&@4m+lvRRH%Foc#q}v( zjYhOHaQMI9(#slqLw3V(bcekK$?JeqKgPb`n(xcg09nG)EBW@=1nCvYTm-4G&Q}T@ zhc30Re|Dob1zJbzp3^b($VEs#g(v(c%KqcI_4dxr*5)P!H#gqzCYu-*gWp{ph3@jt zN|d<}>0Mu`*BQS+jBnF2ERJLPcmucp@x+mb)hjqI8`1uBdZ8Bmck})Bacrz|txlh7 z#y3mVR$yDEjPrjYMR`&zDh58Yak;0^{vj<}%{~ zVh5QGj~>DSHC+6?e<&?3-c%HpPWMkLAZ^*H)Lm_`vbp|zjD~M(LV1fWNOQd3jF!FO zEoWUn&14@vcKiDn)=xfIelch>sw&p;o&V7#U19Z14Q%dW2bbxK)YG`9_-#O*h3o13 zm;2qhTI+Yuqs`F*IoZo0)(~s_03)33?RK7dE=!LvBwc}?x+Iqy5vCQthW1W=d{i$V zBtardl3)Glq*!{U7>}x3{$-~9Tvr`ZQ_p|%#q$C1VW)R zLn*X@R#FRt`TBJdSY*}&bR*luZU(<)K<13&{*I2Xrw$F8ZZnu|{o~Y(Ssmp~ zbLL9>()SPwTCe2zY%Tf^d$O=7%f_GY%I7e6xzn(!SVLis_yZ_QS<&p`A@yKg^hsU9 zvZ8~Sx-c`V@XK4&4?)gSE`d0aR13np?vfPf|d+~MDzB{+@Irt6y+Gl`$V$qu8T zUXsX`fQ{K)P}>z7zihZn3GFhx;Jjw%=wJK(YGLCURW9bEBu|otf1)8Q5&@F?BZTgt zO|+pd>*=<79UiP4^++@!>)7F(S@fI5gF9Tk$d?tdOd^|OVajXywU_kKN4YoGn3IP} z_aO2F^svgC%!5kj&635ouguIaQi_bA6v zk}>FD*N^J>ymj!QQe*N`ZbpYSeYU48Kxopi`#NYcj!~wz0N2J74j(1d+SYc<=;iYF5vvjq3^^uwZ?}iDzeKA?hQB2vCy)1$=P4>#Ix#&n|w&+ z4CdulIXGw6mw+o(BU_^GsOTXPcwAH&tO!Y6p>%%D5DUi4miED4d6LXnbG+N`Sk=i% zz-cc4-(>Q+4vFoXI!gR}TZ!6mrKL0}&(bhyS6rp^IMryNMW+RSb%IG~g7WBl-z{3_ z9$ZshNXD1pFIPQI@59_HcBaa0Zz?&rLFHcjsKy!p?Hk-!EkU~sq#B}1grA{s=$+ugALApQW>De=04C>9Q)!+ zXVn8}$DA3D=;c12+vyn`>2Dh%oc#WgZn^K(5H+aX0n&_>nXf*6GWPbRBUR7UOHpiT zTICj!AecyW@~Xo=#Jk2B(D%lF7;62R>z3AVeMhoI+T_$HIE%2_6*OkA zT2ry^l{dVLEmfJpUKM|aXIb)UISVfLRv5@9ZJi!bA6sxHQZSyy5}qR69#7PUL(NQlm0?diJQ%XV)5YZ`*a8 zb;QPkn=BVS9;mpf3b@fuC4$I2OdplsCD=g~j5hfiedm#~&t${FoC zI&;I#yp*8Zd3GHR!~&F*-fFo#5sj`^QDoS-(AVasz*J zYp@?4V`1y5;tDM4^b5aT!H6?qv*tirn>s)JWWYY_ihK>Lay4t-G zMi5-{Ky(=EQN{hKi6ul`ZU8ij?UX2Nf=0zzZR^MDA91{G&stf3gYbJp0Bo@t$gEew z@jW{Z$n;*S6-BaVG(_*1CPZ0MruxpycIB-L-9k|MR(bYc`%PBULM~V9j!Wih>3LsV zEWO;njO4x<_kw^;;9beV5q3+b`K~KJxZl*IT%-~xvPBlEFZ2w=xVy8Y#D^3!X&=&; zCP>Q2%Ti{an;*@+{5|%NaYZ!SF75exWn_@hhN!hQkRyIg%$-&-A2IY-!h}{*gC$?G zJrTsFT&ZOs@nlabG0ysU;WFl%LtV$;48kAeu?d#`I<1Shk-Te+jniP8MfpA=VH|5+ zfu}eesSqVH*GBUbH{vt~xgq)7Q1qF~5$bhp)d#vRD%8!d+TduM;Lmm{iK`|%u7&O5 zY@C5l#E*LVSiaD6K5^E0CZ-r%GYP5+RZSK^PGNy8Aj~p4!TLspsdLY>TqJBNDmk|@ zcEIz?-Fw5CW+Re=8y;`I3#LxYx&(P}@1Y@Td!Y2l$h3<(AwJ3`QRHZX^YNBIL1Buh zdYFmvHJknS76?I%k3n;I7VI@xnyxuee)d4zY_VOqMpsd6WUZ2AY;>Tg<(d^k=mV&J zC5iXA4hEkIqnFF@im3m_@4jbmBE&V-=<52aGb$A0Vm=a~>s(I3_~^v$3J2m{SjJ0r|ST{#LeX zJch`>wE%Dct(g&DDz>wwAq+_7RoWBH)Yhb@pa9ZqHH7KVe9KXu*>L1k!#3e>r`dOlEMGkTO%wpM<}Rwh&>fPAtuvk^rz)Wav3a! zIF8Gss4^j-bx=noOPR(ejMH2+5`2nN>EXUVSzN-5S>q1sTDf>=M9+MM$5=%47#?n@ zdwHo(y1*fPgIHZLm%i(fF`MILuI2QRYP@x^=~9#%kByZfg!%lnCWN^E(>F4aK6R1) ztX~FB;Xala??0HtkC9K1YM7X6Tr>3m!_4sAQ+(`D>k*soK?oaH^kg-QC9r}-eU>n6 z^o`sJ=AvVmZ#+G>2ULj+03Qj&%- zKloi^C29aDgx5c7d7Lm+d+=sa7>E-)p?N)-a)&)dBz8w&^<^dN4bpZw_3|1@?t8lu z5D3z{!BD4$*KwS1o<8Yge)dC4%xIdjE#Z}z6!sf$9UZx8-p>d&&zQrdgJu!EKa-to zC%yX6_0P~LFIm}y7*GgmD_5Du3$^v zNZtu6p^f1!#x0VoMiu&R*Te!c!{4c?$rt-t0jIY~6=a6Q=FTE?`KT5G5aK&7b@Jmd zG7_WtqWVknvT4&={kV4Vn1>VKkEQShhnr1&v4o6bNmrL_e6k_Pr6r$Sc%h@pNEMpT z_EGb>&fP;qW7^j>_m34$zgsS-4k;LI*v_m`AQ5MuYzoK3Oej`=6MAW{PFw4KLdubz z#7v8)0~ZjH5!CRWu3?8b$li&R6p4RYC>(ZcZw@(O!{aeNVzA>olonE0-B$G<-2aS!_lbdyTk^U?b4o9lnm1q}G=^E->TD^s+4q8Lh{` z{$bE(8v`v5hHRE}DL)D|O3_=X&m7NdLg~DYsndR2#?3L6CV z#j{7&$vml_P?vA~w=pXZm#-Vk5DsCfAoN{NGu8VpKPx_`pru*IfyzUubMp*gwCbPWjPhd4Xve-A;gPR!Re)fOYx7RR zs!(n3<<5e+1|R>T2rQ?SU9C%@(3>eXqbs$~iTYj18rLg0OpA_TSV?@NSPFZjxC+fN zqj0R5t8*GBM|NLRUZ%=bHN|FMzfXF-wlb3Ue1UGF&pVuPdTvK0PY*;oY97=b(zGkA z**}tFtkRS!9^~;sy`pYOfL7O$G&l2c<_3pTsi4f(2nCtmvuobw&Z49YqsdN2!rjhI z^XK{1B0k6x;%ZDE?=&Eltm`p#Q`lgNNtL!E|IlfzMM=*;qJ=((gi{)I(R%j>HX7BH z0s0v#)=k%p+M1d#TKx|}odv>W-ZZ7v?0Z^^OJ||i%hmE|M2Yt~a7m3_OO`Rc`%qet z-V@H!mhfhg1w&rajp%$`fanEz%{Xd9SWH12sT!Wem)B9*>siF_K*5V(qwcYrzGta2 ziI^c)8Uw*}KE~P}cx_uWBO21j$C2YzB}615orf+mOr*ZccA(6%O~g5}TMYrtCg)4 zWJxmDwhX@u=)hd_FKRxFu(BVEplLCMa@el$(EYH{Ymand>H>~36UMwXhR-v2fB^_h z>K{@sO=Da`@1xW~F2houH#RA)pfb|H{CjAN>m(Z=;N6oTrF+4VfT^dcg2frh3_9%h zI=x?*QmQW|p>i8!Me|u9@g7evGc&QFDcX@BfvZ8E#OKIY<^)^I|FCN!ELV0@YPV#7 zy>5x0+I*wY?u{M(sXgf{=FOw5)RVviBf*%Q*_oG(%_>{ARI$SzKw0Qv%dy?#Mfklqbffi#go;8#Td6WZXZX-@*im+4N4b?P&;wkuOr+s8Hcx)8 z3V@^xs6N4Hu9Ke=Tn-Rxx3*Q24kfHk^I;s=kr2}jeI!8w$cDSx%1KvJhuW5{s(m}Q ze)bG$gcT0-mL!n(lF)5tPhM7}JBDVb5LrwYhHx~m@Xg$sV*8L53G5VfY!G3z%J*j? zXg7-{T^;G}V$$*YR0c21`Ej&%l{zjv&i8=Fd2S3(mUf*;{Fr4n&XQbJ(Zfe zs857c#yR#gpP%WLmI9TEl>U`+^z~t-y6QtWcrDO{V*Z@z!8a4mcB6YW9Cd7X<7OO= z3(OODAd!}f<))LKv#}X$noJ}vfVmL}chE&JQbsR@CHY~HKhxIj4&=iCUXe7+7>bq3 zj>^a#hZ^jESo5PTV!d<>`jvm6xjD*wqO$_+-pb*$&E^|8hZ8j1!+q*EAc0n`S8aQ;m) zNA-LR3@v_IEiqjMY1bBE^)mf?(h)lx5#QHwe#Nf+lt=(q+~T-oxs%*e`deXeWIq~$ zOo}u29UOe^$h*g0ADHp9G!Ey)FN6W6$<~MffaQ^z1PCfGE3NVUo;2M1x>my{$a!Vu zl%S0?kJuD^hjTi$X%%|bA!v;Z{AlC}9kz-%B%OZRiZDD5M|aBnT6zKz@9`vSQQ=Uj zk;&F~TT zpRisFWkZt;Jq5v|PMA!|tgY7Y*@BiXw^=lzu zr17sRDy4KCqi)B;>m(vB6Q{H#VMCKI#aDm-d|~0Twz8sb{z7)UceuOMxsh5J;4DhZBRjr?kc0+ywsTy?t!&ent5;JUvC67hLH$O$WXJ z9J4M$Lwd1udwJ54WNT39>ha~@?eDK!`bC0&#>H7FC@H<{eu$dO} zLA;`O3&&xXSE8IXHrUXj^(8yc%WFEm*GP&?5*{9?`}zV5o}saBO83}{^&lb7nzl>S z^4s78SW)-P&5fREbZea>2~cyVB;0j1k5}c?RmLpZgcP<~!N_Xlpp0oQ)ZTF}qAa(=pL=EZKc~BBBYLqr;d~&!L=ROuS-wg!9cN^Ye%G(eQ>;Bi zg*j(@+pn-@e`lzmW;v}pR3&F6Vg;l~cTQKF8UT^0ztq+j-#F3Wf2oFdu>H>G{pyfZ zh0n>p4SdGAN0sGO43l*>t`H9} zZW}_ss%z9bFoS0)rtR4h^t?z{L&t(@rpi4!-WuLYML{>XF!sRb6}w&d@0xGvl^MO8 zvsqWW`@ykk0}P5m9gO6wcU?QkxtPm+#HG-ctT$*Ptr)7`t&m$hET1@zN?gSct;846 zjdj?71e{e6{bdPLaZ6BQlcOZwdU_#Z?)Sxi{Yxyt{#VUXYM*Z+4RIbqc_%+uRM@fS_NmA-k*P|Bw<7FO< z+bMRZ`74EeLX(QSb*PK!52u%J5vlTqSL+!%WKNRZv6J$G{|#|3(7z0BYSR2@&*Z6= z>=v)zokzDlXwi+QWIOD!@N7#0=F z{WHC$V@<@)^Fv>oyAW{$BteLji7QRG4!h!%Moog~&9KL}J*K5&y{qRn14Ml=n)Blf zCF^A!V!fqIZsvi;brzPb`GJFG82VE+KbMSmj5b0b(Z%aqw*Xz(qh|MGCZF|T*x$Jt)OpK5U3K^QRZu;Ds zPIs}-8b3w=wZe$5eja~w2NlwbZ-QMwz!gJ0vI3e#ildYLs36o~>6?Ro# z+zsD{&cKiHrIA`0-Bw~At}d43K-Y-tuDc6sokY7^?#|s6qq-A=)yG)#6Z@^3eHlab zFR5CV&r{}-KK85EdJJJj2pz+FHm_m*^&)Iper6_r+qQ<5IH6#q1fiX(R}9YW2|0U6 zmz=Ze54QBMRO67)kL8v>JhuhGF`o%c?}t370gWt(+lFJ?s-yS{LdhYf&LkB5aS zZRui5Yh0w&_dMTP3JeAH+nBkxFnys*SbQMee9R_yrqfGG<(nGm()sKoWZm6y4EHME z<&!t=JL$`7(fDx)9+jq!H=PYaM<8x&!gfAAr?-CtdMbfUa6t0A>-OwM-K1V;)TD9CFQq*pKYt=BS|zy=1iqXYQ?D^?}n03*FQrp&b0+|MRjqm z)@jC1(vURVjmdaV2dMa8OX-SH)*$GoD*G%3j}(*sT0-h2iXa*$hW_7a+sx=M zX?=SV>0+G^iXJNq<3PKe>*fy|K+K5T@YY*1VEu%$)Qi2BwW%{bS@DQNsqEM`EP6cj z@FIqQ14PuB#if=2+TDCoZKJKAGftU2$6xfqO*C&{Tdt}oJUg(DL9i_8ht}Ds>1kG| zY6apc74OnAbIF^4&nN54P>eAw<_+AG$!^W<513`g!S@E|sMdYP#YM##p@M zgK06~&x}W94FUzic0nN_RF;}`zUEBf5m71LxvISySJHRa_w!=b)^JqaN`_YxZ8Yb0yhSWNRb*Aw=;)z+O8Ci=XSmEZw_n9#pP z9;yE=gJxJZ2Dc$XWSxWEC35}Dh`<^x)_KN>M9_o2X5o_N)yGG!k=}GseqER44e#H1%Uui)+%bp#Nj1NaLcxk;(NEyZPu|dsy~PA8txd*oT>w z)nIEOT3atBwC%d9qnN8&AjcS?cMpfU26Gw)l22pvLGn5i6>2c$=)%!y>;RFV$uo_w z=>!D(iw!urkSAnh$ysk)Q{`8DJk^XZ(F1pgdZzoYHn)3um~eRht2RLWT|NM8pG276 zG>5I@@oS&Pc^SW5lu!&-ZMl{7JPVw9r|U|i{xpxnvYmp=!<@9JBMpHwa7<7*dMH4c zNw+b&QNwHad;_fw>QgO@s&RgWGSsug{6kRBbZZ&G`fj79ww4Wh>0w`=`S9%^oLKa zuQ-r-p^zz|;{8AmW0q;RdXdwd+>!P~YAqjLp^=KrI(y{>zo{anQE6i^NL_n)G_GZ3 zu0^*$npNJk4D-)+o9}qYU!}HcF}Y=hZYH#Vs@dmoJzXD_94l9 za}55}el~N|uwZJ4g3y(=h7n z8b-B)4?Edmx4zzIyR!8pX=P8L7j+0ip#q4}=H6A93)41z;&YsPYPUEkVO@8of})Dv zS7S|0GC;-3>Fr#n%FyJ1L9=tP8m&%yM&lob%c?mgvI@m7`qdPip6isfYzf-o?awewlizolPPf5Bx>-MbckxIhD=To9erWsGd$UidR zVAR+)Hb1hH2PA9q3_ckwRcI#$h?PFEuiaP?NWWiIdbcZ=?J_}~H%ubFw?a!~sGmq% zhaYbA0&MDenf<5}2PMf3nvgU}_hcLW=Oy3*Sbck^JP9Mc;FNAQu9?^x5h_FL_7ALc zspr5ytHJu$+qnSBx{akm2v22zsL+|}+Xmk?bfS(m#**Aq%n#=8of8GFc}m`2qAT5A zAyqkxsg$-s4o@2h{+ZgH@q!zZxzbg9pWY<-Ldn474`4!E{7grYw- z+c8KoJPi=dfMsJ}tde_)9joa<&pwpot#6rK4o0eDb;%8T;~%fH4p4XeD;{d=85nQ? zygQp=5Q!ob2oTwl1daoR09(4Q>v&R;WD3qer4<$6q&!CW;}kA{*X>3}N47Zk zh0pzc(hW_W4{jup?7`0e1r))1_4~@Mk`!{tWYka1DpNWF4T2A&^ZX9!wKBJMM{29XzP zM!<&Gy>q9GAQg}?(cxsLaGu}`sl)-;IN2IDNj+p7wS}GEQT-H0owd-7Pw)^&a`=QPgwWQy{J zVQ3ZQ1=ZZqV)beAv>{1tb>a8==+K_C_(^vTYQ&cb8{84!rmJjD@z7`MPRe0<3=DWf z-(a|83Jfo}3cW}DGbQ=&+Y(q|AC{Ue3;kq=qjps{@yc)j(311B7PhPkC+H&1Dhe&7&X*o{P_C95$#8OJ4DvoXBHYk}i16tcih$KUq2Y?fVf>2)stDYe z?Ea)xHmZ#J*Pc_KK0Usoi&n6B9Lgl`qH=HV(F5+knGlL~s$`R0ZC{Nhde^_+=9ihD zxseOL(xI8vd_B2ZIE9@!UekrAAv9F4+!k3?wN>)2qA3FoIV5 zhD@l}v)G`gM$wzg{m%7n(IcqvyTZlZ@@Ot~pMYO;j7r8SHj3uy68ON>5;2Y=$^WXJ zn1Pf#63*O6070%^tcM)$fLaO5{c$#R;60?9I=1bVOllYw|2^^PjYWL(>hr@di}l$b zrPvB9<=}@5^<-GdEz;G{TdKn8@f`A|mCVly+#Sz91w5b8;(mYO;x*-9(!sndFurQ^ z`C|2?q}*tRR>afgmQ0cwApsPp6!r46f2{QPJcR2YoLo`RvS$l2x#!1xr?69Fcy)$G zTcQEE)RUp2x33rN*LQg%GaMRI=DA;7?ISF2jJY1Y^t)PNT1(rAX^UyHR5doqVPz2| z`L{DI%u+D87Km#JiG-gb-Rt|1Cp@8{z6E&yBA>poc==jTRpb!?nqT5f-y|eZV9MWQ zW2eAfy{-1Lt8Sf#OiVm2QL;N9E1j6=y*C<_Zd@uL7Qa-9d~a5bex0_2cqFqeyeh z+#`vKoVA8o1fmwFg94`r5SPCc7Kn_>=YBOMDYK^#b3p1qyZzvXZYQjinP~gfO>0cp zhC}4UyK|N6H!b>$3y7AmJTTe93=imiG2^crQ)`pGJATs9vPRvh;WU_>pwjI_I{8H3 zu}nma2-z+O=BKgYOm48om;wb69X&Eea(bTx(ncbCh^tstt{up)cZ0QC~c+^-XAjzO5!9gg(zIjaTv%^ycAQ``6! zi;EhED5w`?e|CsWh1fZIzEV)>#5tsyLe}hKaxxT7;zYKZBKOJ4x)lO(7(b!zyHdRnreo-7QDh&fFf!xD zhWTsFNofWmsE>3$tL?H|Qig0Y$Bi_z&`>(k8AZtxcqd|kt@MP)x$U7Qwk8;5%O6O5y`q9=atj*OAnTl4cwj;r-t{6HIK zYz&{+WJAvmj|WsH<+k{ey!)2QQ2Ze$4>^&*uk#aYc6%jb{!uNN$>k%M%tw5{m*67n z@nL8x3zW6*i{G@>g82kFfl4MGTeEcZ%R*}=X`iQ?+5E)yj}{w1+s034cR$aF^{a$L z@ZdnTs;Hb=uC%1dFTHF${CkvLyGd2c$ev2hb*dCr0|k6MED-eb)veEs#(yEN%^CA| z)pN$yJDttLV5fs^(S)YA+(Mix`oPQ_^NV4i&*AZt!=^(O(W+Ggy`n&?cS%sTqdSb8 zl`Gq)?6ZBVI5wVF9kC}pKRgHL<<+-w&qdjp!OG)T?bJRUd`Mw>UO5kB>=!Dl74w4` zy!^s}z}X)@E8H)gAvaIydo`|_MIns9)%NwhSpV=Q$RlIjX5{luzSA!DldBGi@_xNc z3#R=^0%cBqL5vLnv1C(kp1z)W#SI^W4pw{s(2=}E>;aS9(~J~>t{6qmb8U_wL@$2X z((jejM$ppbD%jDohaT6F08pJuAFC<>M2j=EEsTlv+dL((z-;0eh;4hMfz$afWM+9f z8kXH8$MCg)+K|{)(IdAz&k2!UHBjf-0K-p{cyWflb^j9wPd?oh3nk`P!XO%0MrfiB zhiYtQ^6cit4WdSv(Y?jRF~-EN87&yFsg&YDA=t-eXlf;|#!IR2c#eN?MV+2*Fp&kX z^U#?l;nC}`IO=>@C=`q~2|Y7G#}9hl9AoxHnn{!9tznp&9#uI@Es2w2_LZGn{(l%Z8$^ zZz$DWS}9O3-VyYwIzBy4QLsh4b>q}J^d-&PEE+Ooqbs(aJ?yA?;TS{$y4ClAJJPs| zJV%+>irYNb2|-w>iKWH8L&?ZJ)N^zKJuJXt)N&|wGp!Ec8~0|K@&Z z;C8q?QWiI0#*nP=pUQ#3TCl|3dTesA>ZsViFI(;gf5l<=_^E};GrUu#vOWctY=8`h z*RMrkp8K1(qWmYmIlF)LpWotr`O9_RKEy{AL%)4DrPl*9dNMLk#7zxuEYG7HSO1{ZQ_?FroGW3U+(L0 zg#yXk*X^$Qe*pum0wyflMpHB7zqKYnfW@hZQ-}+b^_>d;)tY6fv(66o_Mb|bhE{yA z0V+%g5CX)XEY|%+1Pv?*5dIvxvnlW+%!&*)`u&eT{~dUA@V_xmas782Rto69pfOTl zUVZy3IIKaUQLpz@ zSZVU#N3r|>%O9=H;>`T()Rx*$g@wVeD3HIIBIYuo52Vjk9|rS(bn~xg|8tJ#SI}5P zh!6WvBh=6&MvEIsI?<(EPaen>IAHfi!Ku(8?)Bo^xBS5kA}6RM0v{Kv6EvB3k&nED zxip~LIra13jp3jEzBl}1iK}I%zs?QC^FRP{L;e7OE2*or(&wfuKURoRQc}XyyCxI8 z1j)@U6Lzie$zzdjdmg%$Oq>)yL{;8DaU6x8EgTy_r1An1y%f1ML; zStfu=)aZR!>B)L*xrH3HCMFCMSU#gJk1o)FWuT&socl^)x>}3qZ=^E($>?>_8BB^x*#&+w{}&Q0as7h)`j2vqsr1;3eXN0UBx%dXOJdWlWl7;V?&grH$HCbNEB zQlG7O@M)`%AQG{;%Hvgru$N<8CV9KNbJ7SN+07Z$0}{j5`7fEiv&ENj9182_d7@rB>o2zT>0-DRm1wcp0i<3%5h|@C4t`E8uY$2SDGbfypGP z=Y&oU(PTF|Lbz7ND$e6Z+q!Wd4cRZ!_BYkFCxk`k2J5nQ^or!HM#b6|3^F=B6p&Y4 zg4Di~`2<7mwp0=%mNC1w@)|of1*r)-*l-zfUwFK6lF_eD-+3Qwqa;a*a|b`+X+LOd zmkY3V&U%UZngv&f3D2n-zX3D)G`2+!H8;El z_lZ_UV2X`EM^27}ak?r$AYDGrK8j>DsgCmtclIAWfjlf+BA4FN7~Z$2R@aIC83+0o z^K?}F<*jp~&L}!I$=(mi9+FWcM9v6H62%6iW#d(!rs1u)^|4#i&Ckgmh7!@4r%asz#60 zY#nnI@eEBVy_`Om&&(x1#($i3cYM1H?tAFKzs`a@O0}M4kar7vN1xUZ{u2a(b$ekv ze3h^AtKjv<;loqXME2vdHt*>1=paJWMBCCe$ww<$`bm}o3xVsq)6Dd&PHvB(U?TU^ zqZ#bJOVOU)obj&KyDJ0&=mNyKJ4#(9-8|lbkMo2tF#_nv)++I{4$GT z?0)he?3>KqlNVkR#~zbE+;i;nK4n@ZdV9&^xUj)$3U9h}9|1AiAcjC~{FjF!ZhG|X3pEcJX&n?3vduU*M9&n zLyOTjK)Oto$FqK^VKbU=Y+Agf9ea;#{(#I6cfvR!&vPE`@aL9@o|!Tm-VJBn?Q{DN z<(%^-euHIJ)v5E_o3yo(|CiLc#3d^w6$BIS97G_Ojq!)KdeA==2IBo6O{qVc+=0wy zT!ab**yHb$_`5E-&P2V5YHV0bymu-+3naZVftCnW3w%{hG9rb#W^O*=x#NH(@xak! z5ovB}y>D+TYnbJY>vATgDxJXZzBUwQ=HwoB6V&d`j2qz4@y-mrDvxw6`5v*Lro3QgJ4tB=|Ac+&hLiaV9y zJBP{)^4OBj$cq0B5#;MVga3U2BFB|g@*dCqJ=`#>XK=!Z?NVi^bpMKAg}RZ~o-lsj zCL`9T^4i4g7`!|QoFnaYl05er6T6b0{xOEApA^daB|1NJ!t5NAbw@b?T>U-8JTJw> zedRDj_LZfB{w4n(zJ&?MFg-_KpET6ik(mGu4GpGoHJ|@Ks&dO^gN+ic$MpWnBXVv1 zuykbPsdPp#nIMU@@GKyUkJFWl?@Zo10AWkU-x9I`bKP`6>YZ(D2~@? zd-jFRS3A+3SN+?&u|oJ)HigycXXEc?_8M|{h5jbxur8N<|EK6Oe<&WXEDw!(|Jp6g z<8qmVjQhohj`}6m!`94{JmJ5Vh zpYVUbBKI$B|Fy`@|1mn||B0#n0-60y?}qXy= zZh#etV5$AmNgr|cw2`r^ySuNi8(o_K_Pd|!g6QSY(@9jN#a=$NWdib9rz13!MJ}C9 z*n@27I*=P71v%TnXr2#NYJSPORj$YB*f+-LnZ3K>_3IZMHb+jRh>*%k*2LL$Z}ga5(|3F+R*uxO#Vf>kDdr|H60OyGxGKn| z%~xl2^v(y|={omO8tN3s z^uzi?Z;6K{ftu#}BDj5(*prWa$x9isi0f*6eKDquahMoYNCg#)&JpM4hKV<-c)0r& zp`>oMx^luMPFcFcjyLTFD#|ZA5W$QZ$I5|EiMyWU)$*j9<257r?N1(3!gk7_ruk}a z@@(`aGJ=u;qx5X{CGhd&||BZ<-I&yG*Q#jl;vBqfMZ z*-9+TbwTkite2v96#AM-on>_{F^4`-OhO2i5wYH?w`hPijt!rKiK!|v*5b;{iFW1= zv^yM3x;Tr?qIBHZdY?4Y8XGplZ%l;LN$j}YZz}B0*!_dtD=nL#f^uk;{2?ynyIu1k zOwaMKI}HLKbwLZQet2SB+8Idnb2SWJ0xq^79J5~ZhmAlHr{}BtOk!dk0rWY+${43( zy?87^IUVMkor=QV9_c^GJ;|A|u?%zrmhZ*#DwcY}L>q#%-O~EVMf#Nn=#4V;eqL*j zd%d+^>MqnI?GB{tTzt#+V@N~HBSMDl)R(D%E0naoPPsj=5J>HPupsLj&i|?ZJwzgV zx90jP>@qGK0dpBEvY1gNM__nNIz1Ax)#3Gz>PS{y{z*;puDt$@b|-|vk_HDLH4M_2 zhS85Z=UIfv=Oms?)(eQ~jhET?k?03C?*Fn0wi?pIXJj&s5fGHP#l)Y~ke4?S3 z)<0Gg$)77J$UCqni3*0k95xCb{n!+K-mua*j6rJoX%1G!fjKLaP{2N|TqY#`@?KZ+9wqKV`4)g}u zQI;ye+}dQsij{BcI2?9O0so$r<=t?9U%#MAjGS*_x%&fbh0y2oStE=L5NS(R4~ShW}!>Uzjqq^dfhjJG0(;d+VKy@5|I=LNWKq#_Jc%PHBx} zeITnSG+x5d%$y=yt?@=jFv>l&RL_~5i?ckHuUzpHMO2vCh_LgfW$yRq9)D)lV>&0$!&*(h?sbZubKSZP@`LxVN$<6I^#Wma`#wXdGa`lEHS5WUw z6&pINXB;3ZRQ#Qw2elKwxH?T;?fSgz4^IsMeOv8}2z?U*vL9|PcuRFB<%K^`EcJhr zpQ^34HbBG%z3d<2%e$2IB^R(`QI7^pk{k%rQeDJY~;$T9NZs{@uIsg>P9UnZg=S zL;)c#nE~F28P`hBNv54VI+XRG%vNm+JE@T6(@9*B43AT5H-}w1GYX_s07zaXXU~*L zqnn@FrF60Prku#O-06hzCT7pnf=!iSzV}9%_50cqW-xyD1MjUw9~j8jglI;d4VBK~`4p-w3%klD&F%sYEf|N7fsu`L) z>09jKs(u{R;G^a#Tk5aNmQ@OxwM=_6{jJj4TxhnDc>&oQ?ELkR2gl~!dko5Ek@*f9 zQszXx)uf_E$L1VuUlb8$A*3hvlq1&~h=d@6&uf9C%GF+zcyFh)j2qy4!)~^k1eE7_ z!>8(Mtk#K$Oq$hcyFywgQHSB@UBRT?Lp5hPB176=!yUrj9~gi`4=f~80*+=FhqSTIITkkRb7V^_RF4yx7Bmq&bV_no5w7Wc`B*|V5^~9KpsvL^nCRG?lOaGQF76i-<5PtQzg{rB{%EQMyF%d_h zt*Izyi8z4i@Ig*g1iaz$0x{Nvu4khi3;n|Ki@iPN>M-yQ)FSM8gsM^+zZu-I5&;DO zk~&LqC%%DXQFci{AdT|eB*Aq+Ian9vGt&1ZE+^%*0PEwdnk=0hSk}`5#p6^J0rGTl%E|3f?xXjq#$go7~NW$-}~llPI|Q zv3k}iD3-C#&P(~Z1rbITNtWb~Pj6NAsq`QUwT=A|)M1SrpdKJf=KLc;7y@1dzx$XP zxkcU~`~|s?T-_cuQpE`RY_+E*v5H^lHUNXat^aI;MJDAXo<*9nxg!Io7-w&L<5Q+- zWgZ@WOal^Vk1rwtbIA7AULhsAh9$Y2b}V`Q{cFupr8<`oHXxmUx4ro8Fg-r zVM|31m4HiP)wG$1zFvArPbyq``>{}OiOh)#jiCigEl>*01Dq#LSiJ#R6r7zia z))~i>>_>C?-hiCgv6;-z0-h;Fh6J6b3ej1<0hZ|Jt2FPl$KS^0kgfE^iuJh+$YJ_% zw(4%&fr9qL_`-AE=hYOXcV032opl>;Dza%M(*Zo)B43*dp^d-j3d>$;HzJDV?|Y$H zQ(lzyL8x!zTi8*|MrsWzsecT07vtJUmP_r-%4lrmVKKgZBQ0H+s`J){%$^xD^E(?x zmCVfu=jgNt)Qa3TZ94;i<9{`O_;QFdll^69r zysKdH*5hE0D&9lnV=ZT3y!H3*3twWqK3f*1f!{QY0)fLyBMGd4;j9HP@c#4-@3+b6 zcZ!>9^61TTioH^pY=77{>`>QgOXgc^UqM#XF0#+Rd@U|2sLD=XKv-DNMG9W~Ry&1- zJUliwE~UXWbo~lv(ap7{wFwVr>pe>4WnUNW3;0*h-X^`Y;j8i05PGT*ot?36!qyvk z!1mS7kJIlZG354%4G!KiLA026=+41qjim-Fo(>ldgt{ipVs(uNiqAT;1Vm(BLl~2! zatscV${#*=QP(7w#Pk!5%4a{mnqoAdoNCv{*vsW^Hy0hRD_d%_B#r+d@BKVmm}gL% zGL{oCm^a;XOWu%pQ~7aGbBJ4&gaXXYMqp)hdw)5O72vFy(&pzBGm7RtHd|Os1O7=kk#rOhfO{rXIO{=VoEQ>Pja_NH2q~xT}DbzPM#=@$5 z04;Ydd#;p|okF8}C^QOE%+TpEH%Q{WTmsh>GF*{ff#@4~0J7%$@17wYTATNdo^wNI zfsb$)V??Xf%2OnFWOD2=9J7sNsEKAn?+zNF=sR7N)n)0>Y~jHzTMXq03)>(qPNe!$ zXv_o&J8GuMC(eQIDNOcn@*>fM+-kgB>D6%F^31M^Hs-vk3Qm1;_e6QO&gnA6zO%=3 zslaMfz~-u7L^lGS1lv9!sEhk^&AFbin93I*v91=samJWB{@OKEWlShD*GlS{gvY5u zqh~0B#ruUbvJ2by^zUy7tRASrCol*!bn66ah&UhqHLr1H}fEed| z4zP+vlt$u|Iq=v>hXUmeunYh&CdBkT!@3eJXOk%dhO_3m>b?CeE3&#eE!JqPyp->V zcL>s1G`JdiD>!&PGHbdBz1$9sHN4U6En?0Rr1Z7$OJ-egih>pvo=&ue9L;d;&NTy3 zdJ^xOucj#M8DteTP|(g@r}%j_a>6l7?D8nFsPGRyHI>BlY||2H9hltrl6?)t^B5tx zE+Q~|g|+-Gp?BeVhf?_HfgASlNlh{+%aAH<&+NxS?bMqPZ8cpP2f<+I6N%@4+%A6{ zHhP)0>!vfZuAbXYUdqGc{tWb@pSAO6vd=SolqF5Vwe?BNs(-g zw51GFMS_#K^6b{-cLw*;8FZkk)A2uPl7#EfA99(HMFSu6lFyR-JP z#(u8E#vRT#6&L+(SQRWKf*Gx*YwnzkMMLuKNCAFO{K=|5tEyl|-&cKAU~mq{mwoF_ zVr7kC{Eo_qS)SK8n_nFn|Y>)^H_i1TX(41Xi%YeuliF)CQ~ z+|*G9R3J&{;Ftaa`597YhNcsAzytZa^;hI{Jqyg@irhk`_OoGO9yl1!OL$t&)%lH| z-qYA8I|OxV_Gr_CjI2{Ajr6O@^o+iTN_@{eG9KP$8d=7tl@vx?sc*dBu!sbQa)TIm z5~inkzEyjf;&XNx#|$}ZkceppFKF`2%wqAhql8&mN1jeJt_$Z%{&T*|@-Q0dOWsGw0R~B0tjEvI7syto7(68$fyN$Pl9-%Tk^RKk3u^_>9a3rduY(qwxbE9^Rj)XHc-<$SosF+D%XuDfBBp5HCOS1yb}K$HOUMeh+BdEa>l zbQ2Zkee&UHwqXbFdg;RV#A#)Dr4*eGB)(tJ4(TcO(60MvMWkm6OK`HeN=z4Ob;1Qp z=8GmgyUH}w*}L*xynXXU5PwD7z^VskL=Wb2ybnG2{UMeLDp<`)I9fcea`hQojA^{> zV!ISZJr^IB_~RJTIzIcDdwEcpXmer-$@}l7#34YbOme~^KU2h2qn9Oa!r~fG9ylpq zJ(6@IGmp3qSNGEgyZtnTPTekdQl|1)QkAi#_)mo7^%Gs`2UrFIF42i_C0k3_CLmQm z`=s?gRgT2sl^cYP_XH3`2+*+sK>o{bs1N!g$ODQldX~#Z#|AspW2*I3S%SuI>H6-= z62K{OkJ&#My|NGZ=U$?J-|P2qmk09-rro_9tbwZ>@h}G}_MpRK1<)x8Kt%F0^yzX_ zesU%9JV@j2{UKmkH1Lg<4Cf$<*+(Y9uW)*510&o`ZzypisF3OdNGaaDq!mjo-0%oy z!KwJxey5I1)L>`Ai~L1NqxktNni7s)dT`q`zi+W6 zc1OaFj+y|Okwt}DWsN5j?v9NasoV}CH}kc_9pb2bCeeA!)cG_jaQE~@GKr~F5y8Op z=?bFTBsPY)c?`9fUpkh3Z#z$JoPK8k9ts0^IB}h;P034C6`|1nq@Mz3~+go^jce=e=2B9&TE zo8S@biuI&T4!2zHQDJ|>49-z?a!TmPeNkc2m%jaF%@RfWy{#QJk6^cA3*}qB-u~#E zbx9_<(tyQYdwpfOE#a<~KBQ|Anh`4euv>x=nntKEZ}&62z$2XOu8BK=ASs=4M`oFc z#C+6|&KvI;HHQG4SS+rn=nHgAhu8!|0dPUI{Fl%SXMOx7F2yLIk~cze6;^1UhCr@G zdpn2H(%rMk8WbK`4;_I8NlPGm5Ofri@2v(712x_MFN6xWs@Ic->uTjK~<+$GofYZCtr!-4eK=lRV{%D!y^EQ11V~6$H%w2M{|>XZ@UlbdYn=Y zw$;Fy+ZO_Ci14yaq;G^jNI%C-_&`9NJg9z~@P?`QbHY3ITd%{iCcnc|B~}_kTDrL4 zXG}Mb_s=Y2vW<9LCMK~ynnr#XM7^j%Q~a zBR?jsC@5MT1dB$EcP0@%?sCsjN79un6>&aejgkQRb93qNz*~VF3gPjq+;RXvE{<#k zNwS|)5jbOFz1vdYlcjA5Np}1m0tPv-^VJ;TSDbfS2k+PBsl7dRu3wOPhPO1u#6?;3 zo02S_69^%E*YUgOb>XpyAs19GXbef2p$q=-LK5k?1^XG!CYvSe$llyJjqc?kCBmyW zTiK1_i*H#z7ZJYFNSmq6qIWN+XVV{C3#plFxaTDE%8gJch#%mtFn{-$h#K=zZKzo? zIVNVnr8<5jw#>5mI!tG3$X$Ogb2PoyX6|gxg0Iv+1yg}Y(6Y+s9NTN*E7I`|8kt|D zT+HEWa=K;P$L%2@Pdj*hjZNVC2`AZnbuxZ7)0et$<1s+T4o0$bwxNyy+RV`+-WTEMtma>mB&TN_&EJ1OZt2deDVRHwE00`w_c?8JCxXJ zFSEB9M7p=`9Dwg?SRV+WC*K@E+nt+yoon8SOQ(`PxKbrCs-9Z{#3uFR&LGnTchZ)4 zs*TSL-kF!kFV@wVc1Zidw8I*Z6f#^7v7y>}L19^xAFqV$*q&;BSOV89e=<08Z!f6R(iax0_3q=~d3$ zv9P)wMN9#>8K9aoIy@D7nQ(`Sg^8|C+EWkPA$vYj7UXHZO*WmbVdSzmUmbr`PbV$b z9d{R=$8lA;V$VXp1%tBS=~bWE`6{ttR5{&~Sp&wny$MS;Rr3aS<8X@5_txUO;!(Ij z){8WoWep&i*$c|$>h#6J8jlBwy1n_)D&Lo1m9vy~ck^5P#h{@~Ea~ndMWgoS(B3eH zXB!Gav*iu(^Q7_k9%`See6dl=GX`=$3VH5yH|e-^x!Nv2;Knu|p>kTL(~Yk6zMP%F z#Ds`dnZ972U6x(1`?xpX2znPIURVWWiqa}GTmDQJ{*vzE0=r(ggL)A`yWu^ShN`Xk zE_L7g_g^NKg|a}2f*yVzaX?xZ)okh817%wuGn~@z@$-sW&PM^Rwgzo6)~4JVp6p#;p`5{|a8UI^M*=TDr>`Wnr1;kU{s!b6i9vgHx>d%j#2W^jj zGo?CR$s8OU;tCO?qbjX7&V@>}fYSs_lt=;(H71+SsFZl!ag0OVv;hRvfK<#Ek!g%J z5tt|@sCr4<=1c>GCTf%j2AV3g=IFRGG>-BLw2V?}xN!=D&t(cI91T>a2Ble;y4z)n zOXH;^u;Ak4&E*I6a3yK*;y++ksM00VGM?^8e#VAlOx}3?8rBY=#*9<-1rsg~#~jq3 z2`hDmn)@j{U^YQR9t4+6b&{8ZY7h7w4@c{7gywlU(cE{8VmIEXTQHbNAnDPmmC#B? zcM~83G*W0miYC!6G(Bt{&#&q<9G-GNjH2Fgr|*Z~Syl(5Cd zdxl^#n9u}&@IVL>e?LYoT5`)HC9%eXYoJOE_Za63joAIwP`inm+y?*oJ8T6;L z4qZ6B%m=RIjbRx|S(E7J-I^2y2}xY+e{N)6CQh-E*|VzfI0bds>Zt>i24CaCQ(6vI z)8K^5&>*~m9dRapxC}*+QaC)PI1Gk{X(?ZK2V!D6h~($zqoSgYO-@Q71u#>dU0hfM zV?z7^mTalOO_@JGoG&5%Vm33Ss|o}J1QZ3*mX`FANR+{tf4=G6%gV_S5)gbUDhk7V zrX?x;_oH9mT(SPs@{b2oGQWTLXFD}~tbcTdL@BNR|9J;m5=fNgpVs?gxkW`|{#wN0CT{ST!!-^m& zkB#bI8xa0I{2xQYe!KJkckqA99;6~u#sTS|(f>F=tu`3hqjaQ&LHQb&j2efis7l#e zslP!WyaX~fh6!`!(vyGrs|oBI%<)PkOHf8j3m^8!-z|}x&Woq5&n_<$frZ7z5irzJ zfTs*@P>PPfxNr*%4HfXc;i-Lx_8YEXkW(%yqc%1*B@L@*0`nYWE)K+$)T0X)SHt~Q zS2wY=r1W0^SWjOPshfZ?G7#3aI;?At5X@&XYM7?~z`yVkEu?_M&|i`pktEVT=5`EL z{YSH35B@K}^Z)lL%^$m#$xTg74Wps3v;HUb(8_`N?!ijs?CTo(AyIalJCCBG zqpO!p!xAR^3npf-t^_Q>*#-IeK?KxV3b3tZNzP0d_fk}|pf>?ZNQz^4Nmz!D~F$p4?G%D=$BiE>r|XoQa; zdd8qC<{vmU*Tc)d)UWSeY9vCQ9VKGfKI-;#I<;Y&p@2y+g<2a^?;Zt{FjC-MNw zdTa(%I2k^=h~BmS?8dxuyAZuX^f_O_UA}K6H-9SxnOS-0yP9aZZMZI4?pwN${hRD| zXJBK&NH+^w&PEm3KlAaF-!FDV?itG4Qj778c51^rg0^Er(Vl}zUM`3#FS^JvM@BY4 zeqdM4`F$VcBm(RMqK5W*p7sJB&d|x0{5~Xk>Z=@FBhJ{(^=@Msq>fMC&NbMh(98Dz zgS9!l`-Qy;&L8#tAE7=cDBg|h`2HWso1Ey9`#ld=MBFQWK0X-wC93+lQITJRnzn6X znl+}-S`bm^A?eW z6=V49q(bhIAjfYHZ3hwQYm}Q$rza0@+fhEb2bZnefNPx>^YH4#zK7h?%Wh}Xr|UYT zs|q$mofFqN+ctwdnIXM9o|9HzX-3I>0{uPiX!u!nOu?Za_YN`6Mp^?F?R6Biq&B8M zA7fpJ9WxNS9TfPNc19Dwt+5#Jq2rN3FqrF|JzQ)bhr9PVuF2VK7g`4YIBlYcJsIvr zoWI(5ilEL7#49)3-+}*zr>m#t8SbQ?{7a2?ntMw1 zTB8|BHUDw&GBP$cyA?26XLH+*DBAwbYO(>lx3`yJD8agRFhkATSfuT|r%30x^%6Lg zsq!opIbgK-RJ+;&sv<y#ZnJX+o2sw8ui>0$}#{lW& z5u)L8|7st<6OHm`*E6fuqk=Z*d>5kUNkq{XUUw(!OIVm|yWb5So0)|w4!zqWod3nP z_uR=AodLt`mc8)6_uID4l&&lmUCyu+fzcWaNFVn|$#uZb$uH{FKTq364u6~?QI{od zm7rE0w6a%uJ3c$D$dGs4BLMF$^klEE59?Qcn6H6y)|iINJ+3|eG`%dPb_EwkQI|Gd zie{6g_pixJT17gp6<3^&LvA!KqHDQpg(`uQkgD5891^?MP=BXe9}^@Wp|HB8uKYf_ zle)!2GiPQJ?>ht0o2H!A?Qz*cSNpwawr5(K?Nid}1p3yK+q(xtyL{s{7rM6yI_?nYoM+x(^EJxWd(^)nyLtL^ z(ksUCi3v%VB%1q+c}d*gW?PgNfOK%0BAV4d6}QCB@9XjSm|z%QO%;79yTWX{S+D2H zWsJLZ_^Hu$ZFs$vb0^YvD`r8iney8g05Sq^1zG=gw{J-@NYmDI|D^4V zxvf(O|BS=UmeK#r`w?R|N1%VPJgZ%XJ~m&h*2{i(lXE;fgm*e= zv)HgS$L-%m|F8qFzjc2EGCMGlRR zyDiW~Zgd{5T8pm8AJz|#vOUY5a|IoG#2Mb6i~Wq^E;`n*p+CCxe|Y3)53^hKv-SCN?A466zAI>BmHXg4i*#MH} z<843r{j7Omla6qA*r0B^b80;rQ}=s7H2NVTwC(x$*c%4n&Qa)J@>{RjeNSmRXimwcCtE4z)_W5SDsA>- zlEI3oinr2$a+{tgm1oT;Vo{HjWx2T2ad7d;j_OTRWs-aKo%7(u6QHD*w}iN>iuE0! z2>~>mK}2Bt2ZSKBSxyI8YD<;ujg5%Ou`xwMh(Jb;u($%IA^|PXK&>ym=feP$K)5BUFWY+UdqxUpV*y2PV7V1g$9d%0f(NiS+dB=d zOJvEV?;Tw|Za0C|7HD9f8H&y4u(7Syx=nQO-IemA3{7v~&S%Y~+RtW>G5VQA4*8bq z8`oin&?v)O1fgsZ>092z6@345!z#0Pi2vZhP6of|cwoA>SMijalb0F6`%PU!Az)^} zCY_*eI&w3^1i+XJAg25XNJ9wb03q+|6)I>*v42FHf;X_gFe|L%dp$cE9KjokILakF zYK|aqw83E+YqtvagK{YQXaG9ngTerl&Y?-%?A%=a9X0gZVwGXB51R%iNP2M^AMcS75_cW1-(Qp$qA0vNw>nG!_3FCtv@e z8ujWZYQI??>SzfgcsHS}xWWa(0Sk8<`uYi-`d{^dG~O_(=ejmOfqoF5%OMs&Kz|j$hxh^@H%!+v= zg5X!>Gd559wljx=yj<97fhI2RKPdE>dG+#5WOJ|A60r|}lDI)y?iQD`R}MK#?WB*p zkE1)t@rutVaf6@aGEhk9&`~(UE)YPO)SO4k5nLLn++AJNJp3+Wm^R;(3`Y?k@6{iA z#t38f{9zufSMU#O`L_cf9|OpntoV{Y{6~!H7W0P5M-71{Lz2s`?ik(=t0lJnH|GZlUjh!+^+OH;4q7uqdeyo zMmOQv?kSnmtx{*u+d|gNXO@>vg33mNQ^iVIf+6LOf$aCkeT~p~C?j{O*{jUixxr{g z-=j{y&;>7$pevI`k&3yAl$if@#A7Eu|AD2;#Iv{GqK~iERh`^~ff8kpG`F$jYa|ur zjkzl;xj%#(9pZ*Pt-`r2nZDJteQqLMfU=ldm*tQ)WgCCq$P;#pEXo$_beyHybU9JJ z^}GYsn6b%2vn_YaCFoR}Vq$r+Rl?rSM&at#`Yku1k)n08tii5 zH?esB=9l`*&>dKCc*R(^f1@FK8kV!N)pW2CYC{*VN^Uk(=<4Hpuj988BsS8?Nct^l z_7u*B{zS{LHEem%TZgkG8NbdytIY#eB~A>8&z-));4ov*dI!z1aK1CP;K{T&h4=+z zk(OLf!}9cY6S&52mn?&KydT*`FD-LI-64Z}DZtXU%gH^#>CrYl;UFs6uhTD*Yi{Rj zp%fbFn_P!8i#qrHc`Zl09fSyh7L*v^OVL!KV+jqnZjq72lrx9(9VPJQ} zgEYt}HwI(~V(*_4Puu*pci^d{;siSdIo zr*7HFK6824vh~c$f@gUou$2icdiDhTS-#oZn*Sd__{W90lMK9|W$8*#+ti>bhEGU^ z-vh?=TDt(?Z5jsAPFQD^9#i%>Yg|{HD$04va_aOW$i0pG{K?PQjG- zS*{{t7pI5N%|;v~X}VE^!I9b0brcvoBDoqjT*ZepW#(M}G~67@xyQ}9%W5?)}Mq*#yt9J`SfjG{veJYWOG|CUy+DUXOi% z1GoVvG47v{-*sH`AdcEt?5p~OSMOI`FNlz@mvlw+M$9@+#@?wm-`2rTv2JG7s~UGf z{c^x0mFB0~fu0RoAL$RLBrER%pDHZltV14LwoKtzTxCJ;s;17S)a zK)yrY*M0B0@BQ(ux8C~RUF*A>wZchqs&>`hyLMIWUrue{F!)fGMOQ?^IV~6AVDVjN z4R@(0`zXA-(ItNQSo1Yu+gAtgKfQBxSMp@SwTn@4Uj%`eop{7que%Kni)q;EWb9I$ zSh=6mDH-A8W{qE7c&l6o!h(LLw;~b_Y6)H54}=?grz~Eb6IU?U54iN}?WySk$M*n1 zy*cl$w+XE>m-p{JCSVX%la%nQcl)Gx_}K$j#=NzZ%-Sc@e>#4^`RCngZ4u}0J4FH9 zfqNy-dRM0(IH~0QTwL+&)yseP-85C#z`%51hZqnL00gm4Y!6q~D}ZS|PU`YoWk28J zW|B2G!sDKH2s)kSwkQ~wEqDd-Q%U`hsK1nZ%O-=4WpV_ZyQmg6#);txFF5E=aYG1{;Zb<+(erfAUnB*p$=s ziyCJ)8ee`hG;j;%AjY`)+b)`lhF(CV^lHZ4D9eKn+|$0iy?)SD#ZTbZJtP1G9sCmI zT5EAig5Ce}puqc>M>L{VbH|NvDQj)?EQ9$rd#r{yFy`9TZ=EvGcXiCBW8#XOKD7N= z-A{lJbxeUJWnI|AhD}|(kAMzG{@v~KCX=taORw5cA14)BchH%8qvV`Ani-GP(dv-% zOMCh>eYqz!SgLo>Ju4Pz`|J}Q=F0hdO2g!j3Z9#|BC!9Ribr9m616y>(PnW)cWeh*#mQPNc> z_j}>lrPqwm;Iq5mgvcBe@|(7h#*IKjEB9Wub!`Y6Sor3=-!jeBdpuB{ByW55Esc>h2+52UCZ>#3}jB$_lw=;a;N(?XDVM@HXwDk8X%7x|~Y@}bP}07_}#x#J6}8PKY(zx{&LW1l{VsE(N)FHhIixr{C}Ox+uKcpI`xH3lH9M)4;%tWF)@?cMd|@Jv^R z`hoKBuvqoYF)$%i4_8iv1s&vpujK(>~xT*S?j<5P`E_xrcKlHi+Wq0FdktNKUQoZQ0NF)%KRY3Ez2~lF}v9;WyL%qsI2l_ul(p#H|Ok*Dq*$Vbf=IThgr6_%&+Xcy4SE8Uh}3_ z#?0!K@YQoCU5){mU@NdFkP!wr0NCKk6K3tU;!Zy|U%MEdeDX+&20{d)Haqn+{oyhw z;B`wS<6c9TC({}lub^a>8j<+)%ZcjK*LOQfT)Y3%)z*j`Z}KHy9z1yh05ioE0F19i zMozcki-}T&^>rV#dIth4^r1Tbu%>$aHXQom^4st@@nZl;3I_1P`)a@vL{j2--AR{r zi&tQ2U_x)aX!lFvg@-c2!3u|-eR0e8nj1VyPdOVVJ5zbU&sQrk;p9@zHq;DA{g=za zlL@UCh2stjXeGYg|Lmf~zRP1d=?CurYAd3U&O_0QY5>dw~fkkEgC$Hf3tt#DsiWgFQ1volY)TPm?7KD_v- zPj)TCRE%n&!OyW`UJE}8-Kmu!CwqQ5Q?7l3KZtoQ+wvjtMBlGFwc^^ZPH(S*B-&1% z^;U-y z1&2s)BWgD~T_ED0KVNHbk%=vQHg3MZR{_iGi~kr%7W4LyO@4OG*+(`y-7==fOjS71 z(As9$6){yKv8wwyI9mPF0tHkX2aeTm|LkID)e9wNHiSea#9d!X_OY8kiDc+~`SR^d zbUb3LzruLd;Pg*@1{_>|RPdfpgL*lr7aZYwB03q4m^cdTnwzajh z>oM#ta_(n=ndbaZ__P{6wPIi1%a-SwH7;N=1l6eU`?Z!{*YhHMZ!(hi35wh-Jb&KH z{*sS#sCdDrii+-$JL6Byfh1@Jq}x?f%!>UUTiU~2pTM}O^Sl_bv*|~wJO_XM{2X|4 zf@-?Q=;1QauC4P_QEvJ1bB`XY>mZqvV{H<`-+G~GCr%}uDaVH@9BTwjcxd1T8m*%~ zH*0%~sox1%Wsa>0f>Q$@D{?$^+ivQwdBPvHEr|t1v@}$VAZ^rb>K2YES+76PMMe2E zSug`RDu$VZU8|!!88v|^?qLJ+joZamH%YCcmG01sScYKTYDN4=$$CJH$NCfHTNEe48DMbI^yd=DN($1a1l?Q>^eHylMcbWHY# zn;e-MpeO;qQF2{x0e{#7C7*Wgd*wFkI#olgo* z>5)P6{ZUcoKQcSZ{6}jRCOu>J?>9a58Q7a!ClP0zmPf8<4p%2KNLiCP&1#3#Y`VT` z#pf$DqJCB~T#qe54tQFOiY$FYu+T*jT?Ge>qhJPMYTW>v>%EB7oX>O9#V7Yd}XED3p^rH_hs5h zqlAzSZ82Wjh`dGcO^GlmKjmao!%6 zQXdE1FA@u+?fV7E)VZnpal$QR>5dOCa1Y7@KHOs@qi8NG`~IAxD2~sk*ITAh!W_rQ zOIz>7p49IE=kLDwwDyIhfvLj%4mXz!Tldwr`1#EKY0@;mc7{;Dg4Y`8@yc<^XTVw(Iyg3mlff91Wyz)Xt< zvxY&~nBnLyhGA*h@8;`x(FFE*g=%hhO!rquhmCg25$uEHlM+1&3K(-kMf~kpoim#P zh`MzsM6`GaSLWnPwdcvN{jR^7$`FA;{R!okR3B2gdH{X)6Rd6}xr;8h*dz@qIbX{l zw&*W6e}Iho%sz=PpnpW1!O{1T^gKMzBPV(wHAr}Oau#|d&Va8C>C(hbYB_8T*{5aI zO${T0$m`>1=z5d1s%I95fv96JII-5i5usCvTBrHe__{@6(B)+TRKF^u$( z*r2wMutRqIWxHakJ(%TySX%dYYsp;cGn%%;ve0##i@2k;tevphg$u)$l$G{*gig@% z%TXiNdOgY>=}ZdA#Qq3h%dpW4L=%W~-D{xALKuqd#wK#2NbR)f{V&?cm(XSO5{0UmAhx=2m?0L2Jgb zgZjAfr&$A(Dh2}-NJis00qY&GVMzH8Iv(M_VneM!O~Hsm?xx>a%A z#&b~3QVM@6Rf>(EFnrzzE=`#u^|?{+W7K4*pXr?-4E0pQr{qi>qTg`Fimwq1+iWg0 zZOdH{VPROjReC`8rVvtQ%_wixZy{pBkjjrCV;71x9WWylEB=>vC(=!zh=+gOD=1#TCZU=qGoQ6Ey78i?I0|hgtYt2sDV8Z-CTdgSyR0*!rz%)%R3P=e)k| zRQ-g*g8A^gbnSWrNK;>LtV3PT*_D@PN%0$AZNFk4xDYJ%+H9#^Flv#B^&sjogZDu)GSJ`m!9W#0({{ym=uWm$ z-6VG|r(zz2Ug7C4*5W|R?_s)Q<7j=aCE`bQ?39d9vPL=wHt0RQVi{QF`)x@bdY9ar z*HcWORgfu_W&9Vbr0^Rr!nx5hRs-nb8Hd-al;QX30V@JF<~~SuwW0EWk#dK?p}K0V z4&xW_ygY`zeT9o-Pn=SnlP_{`ZV{NQPZzCmb0Hp{o=12@Vh;M&2y4qZOhh9va0%KY zEzNKkbiphXgDa>LB)#dzjRf|iHZi@b2P<3Z{@gl2MWLX@q1X|7!o=4fFuOZx>~HO! zxeO7TX;qfPay@#+7Qgj!a|wTSW67QJc``S)m?s3cEX>q|vl2*AbRBkhi~MKO$Z~6D zw6WgCGIXY>P&PwGS>IG*~l_8`QO4I??YyvCF<_R*K`TbqYmghW+82d#Vr+2Ze~K%GdL{yoVkE2}{l zPeyu{HNmI9Wu`JCaO-+(rtaN`jlodct#X7vFLypOy{We>EDJ;2mGMT^x44;w;b}4WBjL6_ib&Y0X$}Rk|=~Z7lsx)N?*I~0e|ZRSaG#eOfATp!st}h z(F32>4B&^z`;S#sR~Sg^2Xs_?FoLcvT;JED)0|9NpX?POBdCi%-dmoQ^B*bt;312$ zx?^8i;?vIq*)M*p3sJ!M%FfROK8APHlAzYN1Z02E>CA zby}@Z^U)*1yTn|LJkF5_YF@V|M}JAGL5SrylL4vW^$*Z!YIqJ4Ma_lzFSOeGyg<`4 zvoyd2_xt=;ool?Bg)ok_ot*~w!lMV>6|uK?WBoVHm%fA-ZrY4y*3TQwc?U}iPmwZ-LmJ+b#wJzpqq=5$xo#=wKa zGSPDO5rifwgQ5!d#5v%g5GI!5_1?Ps5vZ>K>wKcLV9Nr1zjR1`D z@`%JP8OQUlFMvD!h%3fqclQzGdIm^apF85Gt3kX8WGO9&HL9PjEBACJJ~)47)i`Q@ z4>)hLv3JDN_yuG{n;(o)v-4)tt)$iMywD_-iBfF)YL}{PgJ%aGm3m}tgh&}hl5T2Y z8ApW9UT`DRTVXd+UmuBLc@7@pPkU6ASGvNfj>bJ(lvY}HkRighth%)R%$R`9*on_xB!|_gRbu!PT`!A zDJ~v994pdr45Kd6kT7a&>`AGijf$yN`LSyN9amR&F?5#XwN~>F&Jb9Ahz)5f`2N~h zt$fgigwRH_16?%G8Agq2Vu#Z71Eh;2dUx!QzjgiUC5wg*?Ai=-qnp5?|DJU=C4=e#OhU$4 zHS@&k7e%&pV_%!*;bPRvX$PnN5E%4FEn%aMwK0fY?PP!2YymAVPsFAkiQp1Hg@;oU z*aPo+Sk^Y0V+(U-sUKXyjj0Np7Nj1?mlG1BU(he_ zh~vc$2QGc2Cd4_w?;RF_qd-eW{9%yJnQ*S`dC;Y0&r&MO%O|h{gQZ~kwAZf>mHrq43?Vvs1a+@C>qvCo zjhoR?dyq`e!TK?!W)x0lPs5o-=ZOptox>&NU?hh*K&Rh5&s7ou@4KbDCFr8QnZepv zxKeH!5AjR4~ z)0I;pVW(BQxDoZs#Maik=P73?gKT{|ePB=~?RDmAb%w`qGv=Y2+wg-F4e-3>C&pZ# zY-LONY7zYqA(>g^u+$9U59vVi3Qx1n;z~%R4T|$_(lW89izxGM{H^>`Gq-pP%QBYg zL34;L`p0hUhIt5ySjeS;bSKt1gsI*SsG}5K%DX`=BhXr8gHqhCbSr;@0OV(-x9I8l)F&FV)(Qx9 za|I530VNJ861%hqaFNFWuhqe-q03U&v%%#4sLftG2%-q^KAl}%Tp`auS_YVW)SotA zx3!|bT#;Nyi|ezp8ODSWHA@h&-P0%DlPGL+9EsnGbMS9?mzJi05rD&0nRaDJ=EpP? zYrT{qJClyDoQAlv7k>XyF|TGu~kT>5FYpdPQk;|ePCdqvgFJRXu#U%M~~RZ;O3j+ znq@!|LhAloak=oYxGcn(3m3|eORsP0d(AKV13jC`6SGG*&A&(3K(}71*Dbs%)&uFf5%k{Wv$(sx;VRj|xRMNCjzo$KlF?5p$!!$+oxoW_pm_<54 zbG|~LyYPU?ulQm=kHCT6!X*lydk{rDA}h;l+0$bVt}M=4T}7jzi?df&ySV&;)=Zwf zH|s-e%?b+3>f?-Fm^D!TYbOQn_h8Tcv{nv0Pery}s+i`tAvi z)fyMQHK$&w`-o&u)(fKkn4e!gx*`j%<)4j}*PU8n1vRX%@Cx^%0=eB`8moG9lMk#L zNduI}TMgvEiDvv^JRURL*I_v-$68v^4;UztnCO!-E`MLMSxBh#aW9^i)_crme)QLO zn|wcwQmx`qv=D@q*EZ<)!2V4j%eKz72JP;C@8QGdQ8y5pH+q62Z@XZ=gtvrp$9}i8 z-^@dd1bnGa^?k3;{Z_FSZ?An8v8l*LjOJVs;>ci^EBW&TJKTBSmL?y_Y5jGnQ@K_= zP*7ug3w61Zz}=?=`#4m+@d4H4uC+QDbQ=j7UL`HDtwf}=MFh?)0WV3dg- zvi&NRTwKi=i|WV<9P23n+xu~~BnTsP`c5HE5qe^Z;QJ#Ns-c$U>&(t~K{3io+;Bo5 zz1b-;VM-PUdi21vL5YQ(9U7^%*DSruGta7KFbJD&^I{_nE3lx2A4kPD#?6!7Z+K65 zB*O^fvhyHrDWh|^$o})(bk{3&KHGV9KaJCKrs6K9im{2p+(Ryvt+-`zxqv}3V-A;A z_>X=d>pl8G3t|p}s}|bOo;Q#BQtRv2MfA@e`6!3|)&?U5OeB)!c=1LfTKpuOp4QA{ z+r%`am^+qRki^=I6jWr}l>5=VV1dgYSXdq@gV$}}fkw)BP`y^Z|Axwo z)4uxt5;LhY`evGJng&Dza6m&N1E0WzxygD19Afl*Eh}Gmtb-@T0XeDYZ*csLIjLUJXS8Mul`c@7aN%)v764lW`G+Qu3JY zr5?yO|ga`vFTLN$er>TZmxVS=jDgOqdXSW(;t}` z?WSg54TrP5%7<~CM>VHs>!7P3O?UprK$FjmEue35cS|6n05+(vKVVH@bTua^D#Z+3 z0EZrRBNq?b-vJW$VtQzNj^Ux(qB>_#Kc?!SCS@^bh$$SWNHb*-QB8 zJYZ{-QW5aDEu&*TNfhLJZ{-a-rfBH1f78?{%aJ>2-ivKBBlxUN$0q10A5RB|a!xB? zPy3!{;616IU2AyZ6GF(eBc&C{9tPeghKeEvhZr3lfldv*TT*kVy0(FUNrPWNJbK+vdPAbGku4^XDf zaL7AFG47&emJhXEO_7au(yg0sPPAE-(|+jdpIluHz0Fbo?87K~VOcbosWaK81p0Kw zmrSSdpx%om?^0Eh)$Jdx4XNgiw$v=plH~@W_njF@6$}H8y#50u1{Z{fk5+qit z1GwSc83k&#HT3DBfw>S8cP?Iucr&&U(fQzES3Mf^_%5=8QE_2@UbeDQsG~)mP1{G( zt~uXv#tj2+Z!UaMG$Nz5HES+>hVg9 z9a^l4H6A}rF8b`Z7HOliPdlbRH8UgkVFpzI~w`$#&Jqp5Dcu)N8o+!%!kL zPS@&cWoPHj7cX8I=EgaMbq5OwH7n3?2wi!e)dYJ07Tw~k@3#Ghze3oubl6%<-qaqc z@$c#CiUD%-M27zdaYu2brQ_3`X}53Rjxtmvh=%<64e&qZ@YTOlW8*P}ztbj19f5f9XHo9v1yuza`2i^t`ez9R3_;|tYlZ)A z8~-1Q`MimyS&d<*`Pn7L>g(5KZe+Ekr6r)I4egsil-LNkB9cfz z<=}t3l?lY9080cK{CWE>t>=z^nLiuR^S?Ek{{JwazYOAk-W~m0fPW6i|E53xw*dbw zz-@o}zqg|Jw*~&oSzsfA>Zg0?ujL^@5r`7Q(2ri83iCaB@zWeei!r9Lr32e_8OIok`|iDe>@JT7HTQg$5Cflr!6Dn(dwM zo_*OX>VV1irJNK3TqX;98lmX1TUlIv@vWKeNxB-i!8BMw9p8~n+mG+Ckp(Ya*J}bP ziXW?xQ&Ev}P-f)h91v?tYiH&x=9!svWIy$8?KlLSGzzr($p+p5vlg0#-AWDj@+#EhoIJwck9YSFTH_(xos$~Req3W60% zn#dNrY436r#+;w;2#zfNAnYz7CK&kyVCTI%PJXreb8EE2#qe6`wD+Ggl7Z5wDP? zA}W=D+<8g-TXxcp;2mpk0iDyg+KEDJlQ(j^@C5JKH)X1@OHENVtsseG;%_eR+-cU4 zEwI<%NsUE*K2X1_bfUUsWMqSkvWaN0s!DVAp~StC!Y{8JyBxafEp5U%Uo%o^>e4a$ zx0l1LO)LN-P&UcjW%yJ6%Vgo;U74YYtHC0LcU=;U)Kz9Y^%mA_V0sJhZdFPQmv&7~ zJubgDaYyjzl!@0ph<(moL*bV&z+e}ncLeV;Ql`7u#i)c zV9@p}7zyreA{$01TI`xUanW7{{@hQQvdcm=I5L;^XwP2BNX3g6Psm3Caxej;-rd4dD%4@SnACB!loa@ipl~D0^n<1fkf1GJS)OLHW*_=1 zFebCTE%N0A!DhgivH+~gBEROh8sV#s1r07vh3AW3k{CD?wC>+HIP)y$R59Ih7WscfPV1 zAc-6klkC5ljIyLdeqbTZBv{$R{|`~-YpTS^oHX$-0J;_~9ZGxm=sBPlXZV|N#mD(i zOm=FiNbZ$HI>Yl#v{bfP{2_XP#fBjTNOWcKhx7rhY5|v`p|tDo?9#LUs2%tqCK3|7 zBQbeE1Q-o_Ei6=kTO_x(a0E7i&lTct%A$YK$DK3TJ?hY-5LUAqb?(XjHw57O9Dem4P#odd$-MsJj|M$*( zGj}qRIdde*+Gm};)+0NSYAUi=05Sj^92}OsoRkI}9P%$XI7DcLOgR95tN0 zl=wRj_~UNWfKTqe>yM|eInIxrvyYZvWr`tL>ZeKMO|?5}#xYh#mvT(#eF-czrUWBw zG;|4}<4N>%_LAsEoQ6@cj)|s(qo30%S1n3NV|jdKxZVHeE=>sUJ>|(rq}hzhjMH2Q z=Er|*vCLfW3B@30(`#|q`0ov%e|^+lygVed%*wJYY<{cv=(;FH0`l(}9f?+9#PNI)qU9MuBjysVr5;asF7Y`1 zQ$(X(yh?P_jt!&ZqIz^fKq$qL4*_yQ+Ktktn%90-c8SOih=lTMr?!bd6Gd8zmZ60ck9PVx|Fbw9u^4D$< zEGzv_U>JBbxhk-*96+f5b(=P0r6JPPxYW3qh5z}M<*|-HwEER6BcAyDVqO1<1u@1Z zGh?}s0Ht1bGZK99jcyIjy3aYlXVV68r10$A85e{^gs)j2)Mh(^db58_X3nCv=#aFu zWR@h!;}0YKS5owoCr2bGF*t%~K-k48n=+aTg(;f|u-klh3r1s7E4qspi!6Xsr)IK4 zq7T}{2S;c`%-{mkH2_j#Gyx92zye&h|7@>sVo8QLry$@}WoQ8K#3B0bt70Kd?IHuF zE(yWeej&NT>&Y5`(%6Jw>+1Xg!A#*002V!2ld1eDHNj_gc~`P*Ak&69U7WI$aY)=A zm6>r%cpBm9##JEL3&$P_@1>aA!Lgo0Iob40IsQxcTWP-H1E z9%k=PM^I8E-PSbIuqi&?#x)pR5KM{(l01`sn}P-Cw~xjzd`AFy8zI0!XIR*x1vul! zmTpppCc}q707GOVVHls~P@LfVqk&9Fj1;{paRJFYSxljnOnqoXv`lRHW!lNX&;!Sp zS1FnZhs4c!;HyXo1RQ|S5A2WKoc8f(pax@qP)0Ay{n@>HZG0ER!{UW{(1(P|CmfTE z=Ts6ap%bHmh6@^Yf^?(Uh~X9B=L3@qqB$FbV`7Sce5`mz72Y^t?C+ygrcOQqmappn z#?M`28*R!J7_q(?bh&EB_Vks#- zeQcRxm!MQe0wOa$UG=qD zYKjdE6qiCO1@a|r5#pQi1}oNS8_~QLs0>rGXCVXnW6M%{9Fa;}SP+d8?ao!Im34TB zTjmGZS=q^Lf-$4MF$8@GX&Q#w$Ei;7h{Qz%7fj(5@$U%DJ>)OUdNIcBO2Z?d6&RUd zgXpHup*J3k)OuLrh29d|&B9_epZVhudpyC^(7re!`OSq%F_zV#6f`ELD`>+`h7typ zDWv4PBa@v2_@LVb4o@v=KKbwa?RVODPAoYM`H>>INXi>XlMufI)RPk@0fs9K4{X011uY zkcQymA(3MRNASQNI`I<-i}CW9RLDjiqKA)oR$5p1it3ZFxl_*T5T+gVr`d|CF9S#C zPL;*MYhgV-5;F%p|4F~y+eaoAmZFXhQ2wMMZ?>6(LqcMry2clvCPE+_Wqi0c%RnP8 zp2io<#mfFv4Fx>x@Gq4WhI5r#7nhf$BqZzWg}s$c^Z2bzJO(C>-3}Z9Xm`WFWD(e0)52;>fzgTOp~7 z%Lt@Srk}$q90I)ZbDKi`Au~^CVP$YMg8UBi+_ zf(^=Skl0jws(ezTCut}>W)wpCIl3%En>5(y0%6Nr5&MD_%@2

NSiCi$H}Fl&JeVwY%u>>fV$7muV$v=5S( zC}{UCb(uh`*@zYn*AT)C++@7 zF_<{fq6HBGAGA!TG&yw4jmQ)9cX4eU&f+0H1Z0V`0)<8ttlwJnv&zs=f;>9(+3?xk zo0Q&V#Atj+Kg1DL`JE;Ev?;%`g#-jbWULZU$%FrdEDY>1W z?k0;ov#2;l zjN;+h0ZW14Afx@1l$7<$`wTD`69*?*p%iw2Cp6*IRr`I`=4ukAnIEbo?D0U5CwG(X zd@2zgN^A+FUseP6z^?uwlwHR!kI)VGu?cPGEhNv5%R$Yeavo))&1#?NJisJagGmfY ztwWAeng(0|l_kK%0+M|gJ-Xh!_B6_w2aYipSD0fD-U=zd;gyLS0~_oX_rW0pkfWTX zkYI&46wLx%ArSV4fb%vX#RM4yyUO*#_0^!*II7HFFrET^4+&) z@mkA2Z;W5=&9Z0NtzX;W`)d0H#(A~Dp7g!o_SuMJ?Q7K#FZ=zBo&Rv!CHv`X{InX| zb*S&t(d!HO%XK6tU31PNj$cOjM-2Hl*->Gn+gL)w7C#>1j}O;VCFsX;yJ8N^h-zEF z?DOW6WqSvQqQU5Q2=oo?xxEc~qULi9Bv&)-7Q*dqMHx*p9MuVFJIodRbcK&-OC})U zmS{v_N4B%=_ZkMe%8E;vGn1m-niLiu^rEvLmIp67@aCQlbso44k!Z)dTuM9cee_XPf;@GX1bj2=N$Shp{!tp4~UlS{CaiaOA_5PxVecq|S z9&_W5$K}`7ohsePoR7VeO}Cd{W8hLxFftAD6z7JEkewqkf{)j_@+F1UC(gHLo3402 zmZNWN6@yk=k|`C7TN-kL{_xs;XB5R?%#8z$T1IV`yM@jfzA4Es0Rp`Ft@W+>3P^VuI zy;CDpI(up>UtCNr>G%;jN193kc-In=6G^PyTpXQfnKe36y*0tNrciK;?pODPDh6%# zt3|Do8S?Z7Qg;zpBfP0@M~s+2hfT7)#f{f}Wu>oHmU6E`bEHGs6{#hJv8TaGS}A8? zZ1x+fW!Iw}SxoZNak*}!Gig@%@`#q!*f(i+C7zXFe*T|%wvVJrC3yA`hn|WVy8DbL z3nUvp;7Gf?HSY+5XUCrfEp3T6yPt&sXoSi63xjPZ6qvE)o?ocPo7X0DuWMY#xsL@z zB4Yp3p*wQFVP-XXxbh0>rcVEy#h*+tsLGKkcjlC-6lTd2Wv0_X`St5pOUsNnqkR`Q z%Ds=K^*TzkeS4RVtWM5OcbAhsS()RkHlUH2MzT;;&D)#husU$4{UOs%PG4ucG59awa5H zSW6bLffia8xz;rN23esH4)sjO>a!OPE{N2y#sftGBn7WfD0TMW%ppNv08{`{>+y%e zVN>r%1Qa+RNr}PdbqJV`{Rjnrk59+#b_1#83XK2!2^Y8JAe^G%Aw&rpEUT(Y^K{-@ zzx3o+b=;I9ZofUI&s2gzF_|}40z&UZhfhKw#}1rnncYe%0W%q!?;RA4!pb80o7*HS zsji)AA*EA1SmXBUQ)K)3H$2_MtMubG(fQ2<`-W=o1u7$2iLb zt5}29xPV9ba-|u0f>OwYdQK@^|50g)AFLD0;eO8b$dv$?qK}*hFys_NRmH3TNKzq? zI79^q4B?3*fd&~&e)$9&60BjV$B1gTJJ%RZiTsg_ivEu5iOy{E=qUpe(7zI~qX z1)T~_ScpD)!U{l(I;I`mr}<1icQ&f0DXJt+?x$_W__HOat9{OoAG57;e~$z{qh@4e ztY51c7)(<6o#9keS8ok7<`oqeCwmCVFagV2TO*fFTXNg}e0+F#)M>JTgyT@{e$;6S zwO!Kt5{j{Vc^)qsvUa+)5$@mjf`}^5khHzC$Laqfq_yIWLr6&Ib4k@DZL#NA)z_B) z65%SkV|LNHLqS#E_H+By=UGvx)-@^pjoJOzTW5bw8iDkoX}K?;-Ok&UC#!4L45BWE znEZv_*?8Nce2h2RZhPT(eS?P!XQ^E~Usic={2sb4rcjN2t}jJ*srj_zFA+q}^SsYo zUfe}qB4v$M5qXwdldN*wjYlBs%U?1=yMOy!sJ;@13vu7w?RAah z`)tN9UjItxFYrdcwcGQ5In%+WpZT?Ni}$jDfpZ?YCiv6;;mNrpCH?E7V&*4Gi<3j; zoTxX4#B04#sh4uqt3$&T(w>j_S<(15e&$$=CcM{|YkhT|82jX_V_#Cj|J>{}JcN5b zd{`n)f@4)YH9U_(zPh$C#Ka+tW&M>J?$E%&ZzO7^FRk*n{*5!K$QcVQ>BKo%==LM4 z48HukX&a|Vg7`~7Vb*X7YEq}MpisL?k}II1-#ebDORt!}xvn#aZ5 zoIE9?kV`8AM`FbfC?r%LmHo*OexI<|Vb<9{vwQqtj@%e7^;H*y7gTN$!YQ3N!{eCvW^My;ptX@Q2E+-PmrVquW}X=nMq34a2#q_{>(FybUf`Dg5KLL^B{ z-&fFm?SX;l&(UH-bvUc4@ObHybmOwulCcWa;v|}d=cQFnbCA}Wd{fn;%vwg|NUlum z2gkhpOpLS{ZJvP}WouG0#dQqAB!U@ry)U3FqVdu`R8F=F5&=qD<-uKx#bwd%MnJl% z#osF@m(&de+VVWY=uf(6WVOl)TXoCR+;oXS-C4_kR#{G z?Y`QY!vBR%Hy$%-rxC4|dvM|I7bjb)W>>_clg-Vkh)fK~kHEer;$HF3)I*-d$;6hN z1KH?bz2TWfi@Tj0HkN9fNhzwO8#m4j-t%oB_i0mmt666F@L;|PQ}NUlkN3Rjcm=xD zq33+NorRHgmTR`4oTIwo9~Pp?AyjYBf9R~uSlWL(x#3}SOQ(Kb|Z>b2l;6vi|N+Q81Y^{OA=F{&YvyfEcg4~em7ofI)TPH2!p6FC&#&T zw$@{rd30&WEYQIK78tJ+M_d$3bf1gzqK*~1`598m#KZ)frMwbSnbV4o1gHzhD}KyY zbh3*S-$8n>BnWd(a)8mj0JupGd_sw$^D#r<`N1TTxYGu55V<;d09`WIp({msgx{M3xKmGBlg`W(ey#I?LL`kva zmeE-e__*2{<0~k5f8JHehSvoS%UBue25AHY zP?JznD2p==Ds_ONY_ur!s~YHKA58$V?-)msI~$B$W=-0r-Bd28}-Qa5ErCMWkOaF$gidtF`*qy3d-Wl_tloVtJbJ}$r@ ziotVxmznvjlFO&PkCA9P+@q;qD=XJKVL)ZWA%flSToDBIh`6ZZ%F4=KsQ1QccR$(f zJ12Nr%=ZoY7r)MlK6mwPIaWz~`V!uDwFc(`VJ zY9A8A9i>`+Md3UV|79lD_pl*&4ge6>Uis#4&)%lhU;6vzp4zR-V&2WAiwvTQ-17Nc zI8**O@1&~RYv_$F;EC-4iAoz$4M$4ux5Y8_m!e?*T`v!J2V?IsDNAbpL&x@e25mtj z&HxG!9%V!%yikpGC^i^@iAPxyQdn}OVyAO#cKU(O*#j)SEHY9r$XollU-vFI)qgb5 z!h7N39S*UJsGyjtf6_$fI4pdTns7v46a#}%nRP^43a>6RnY$d~YeBnUlxAX7T$ zKxky$10V~0E|240ZAlr}#C`4+K&J197Ncg=*w!!{kdt+1@#k6@kz_YSV&`q{tC2A` zXGI+}>gA zn56aE_@QuU>1s^Mg&E)oG;l`{jAaH=N6?}XLrT>YIHmiN$Ia0B(IBLQJwX+}! z5ZLe6ft}ldWFpn@@b&JVtOkxx*@tVJhEd$2oZeG<&W+Zb) zJ-_|+;T*rP$M}!=wxiFIN?SNr4mZElEnWJC$-^=Ky9&QN5h1CpdYQHbK}!>7{f@5a zygP)?K{OqJ8ND7fHGXF{`U5+ye2|fXzS-M)YnK}#$CMux!@$h_9l9>9KYhmsG}zUi zVS%+Qni`Sn8tD2k@dU=L<09edzx%}#JqIYPW>*f0cKIYZAL=6G;R+~)05y>ukv8zV zCCGF?p7qPfJ@o1&tW7CJa*xyxKFXQHTPF}OqO_R2xBzZll9ftG_}~%B_JF42ggVV1 zqDh(@Qo6(K^UnYao>{HZ`n_-psrnqVnYG{J$oNOQ?=IW`KJY?fS(hoWdnbmZuD3m2 zTvkPPdS`@v+npaCuVsqc;-Dud-|$!?{qqrH=(b0V zVbt&gF?-al!)Fck1B?wsji-+VNyr#vjt`F!=_vn8TKT1!UG`wKYj^n}CMPGSqyR6Q znmbc1c}9LbKjPuBsWF+4e=el%3a~q&YqE9jOwrK8jg-pZ_U3JOKP~+7C9*%7=we$_ z)j2T2p%kVPIf7sl#OL30^NDhFI=4NXUYA6h!dTXqq4DpSHPCt;<;qz)lE7P7H4pdL zt3Nsto6?9ddxNdA^x~yzLyOV*2Z33|FrebdBL-xG2kxO!Ct#s12H=W0vZiQkZ%Z*T zGxyfo_Bo2p8deZ*)#nEk!lNN2>@=HnU5X3QBiEe22FVuQh>lv?M4TL;^#J?dYXYWu z-WkzsRQqCec7UWZiUJbKF8%zm%h=HndVoeFH&nV z>FMDbQSaV_R$2d=pHEaOtFg^qT@~3pht7X{%Q#BWPnx5K7ufv5@70Siqv}I}hgylb z3H7jh_7~&kXckh69Nf(%lz;ib9NvJ%)PH^9C1hGp)Hc{-osM@VJB3N1Yy~>m@0%(k zn8KI}VuZ)x3!+wnWNZDD;)#T@bP=>;x-)V{@@&x`n$Dt6?O|$!@+{oO^>#E64~^KN z?39F(F}>z1Bd$iJmp%82?BKysoVq$eHxGr&r$Bra%51|DQE?Mp_)s%(H+xEI+^Z=L z=KQEEwAN5R>{GIh)^Zj61Sh4pLF*1WZc%r`$}TwM9a$axr^QBI%#S!Xn)O$bJa2y| zYQ77o-`7flbl1ur@Y|`0a^r{(DPcNYwT4H^$l_u4NPnxyZAI&J<%HpZ6vl7|eNr!T zn(JE3z1aG_B?Hqbu2#_K7%n-#w4W74+kCujSqssZcUcBco9ft9A`G!+YZiM>*L>cv``7ltN~ZN?mCvLkTNS4-KLbA8(xvyR`#== zsuiV7P7@TIAD8f9;bDuN7C6>(m8ErcNvM=9`MEo@NU z`d|$*=4DSdt|($F2araAOf9?j5 zU10u-s=hdLf4o00BuW2xp2&Xo|5&|Y;n?<-gqlXao+5=A?98IT>_B-8X*EtIhXyn)`Ts%Mv zulIP;G&srV;W<(Grd%>Yc@l)m9GzC|2nwZmQo#IZ{ zI(E<#Zr^#y6>DasM=Qi}2o39C4=+IFsrk}Bm`N#Ae5<1Nv?0hP4Xz8*8MC)9D5yyY zps+aaJZ0#D;xJOK+n!n$IA6Nm;KXbc;8kR+e2VL?-P1Sh2|OgU-}#FY%PRVaEW25N zfZV_{QP|jtCND^$z{u;Bz`CT0tC*Ej>^aQ*d4qLS@hnQ5gYV};o^s{dymAkMfgMe~ z)~dAT&mJUW#^!oQ3wF{8jGg%@wUphT1Ah3VXAg>&nM)H_R-Tqw;zQPsrnlz$P_B>@ zv^X?kE0V0Ro&k;4F}<3E#rm=hbkecbEkCjem-Fp`=a~37&eDu@bxJ^#xtif^oq^B0 zcUlE-f)pmELV+h}>0Am&mbvN^>zAVD3)@0V1H0f-Ow3O;6sUOmVB9apFJsiXx?a%a z8gUFwb**6kqU4IMfDwg$AT$5Oln3B!YU*8Z;$ivxSb5zH96a~i?gbA7@+lW8GiBQ` z`%G~;v)b=K;lZ1lTT9Wh?`NqzbslcQw3F3kUAwj{+X$g$E-sEWxNM{3Wkr6R?rf~! zw^mz|Skte|vo^vaQX?p(n4igaXR}0m+V%*ReccI&w~3_GZaCcLHbBE%nIOiUZ)I|P zP}}m%m^>o|n)EH2W=jEPJ8Sw1;9bwQnSkf^@QD-wuM9L_NAUv}?Igy2h_g#vrDx8k z@~v&&c|18;Qhs4rW0EcPIV9t=g}GzEL?X24z9OSJXg7zcy%|()w?m;9NA#xn%NM-w z*5tDIyy*l(*)p|xrlvMIJOw1h15*kVk?#WEE$4`tf_R?3=#3!9z6Q;o)tN~xnV)*yuy4@`8t<&#YKg< z6u|+B#;0wrAhImFY>F)WYT2{qtxZo;xBKn#WTGm-BbI%|5!&_?qlQPE=CPRCX(o*Ghv-OB4Y6N-rnrO z!cSN^hQYv+St}SPSjWP^U<{M>tJ2p_%VDVKPu53F$?}Sdh>i}S4t=Ug+tr>vshB1E zuGpj`eAskZtDQ4EEU$8tV`^#&9CY+xZL*~W>qXC&s}NwKXxBMl$5UN#h*+M}Vy5jP zV&{NrNn^^Oo(#%Or)aS|G>6u24i5Mu&2;pb#cP7F%m4Dq6aJwN*c8H0%PDPvkzrCb zzCna8xX21EpZ`^G;?T0timdikUogK{;C}sD_?5_ZmaMY5GkHw*M^Np}lBI0kj>DT# zQ5y}tfxR4Ar9uojhMqi~t!!+5 zHral*{t97cHkrubPf1HFN-?&viTTv-hKGif0n@t;45*fxP7RI018?zYEYTD(l$_`z z-8WFDMl2FmUNK9ONn#b%Lw`d}I``{WLuS2kmFI`d>>1xoA^%43~o)e5H_NK{iW0 z4wzDk>zAl!9+)mzLX^3}-nX1YRMgpSE$YCiOCUfkz+`EgDCCP3lgWvRV9Uv>=dba~ zg3BqZ4iOxM^R^Lodt)vEDc)o{*+Wh=ZqB^TFI28GT-6ypNz`w-FST8C=|BBjw7CTm zj9sWB|NcIEaQ#D+&Q=xZ+X*XxK9= zmBlZ?$E$-LMzh#Kzs)>SHZ_uB@hA>xt<5}LCAfGcJbNWDRpFMD99cnw8{5N_DtX5n z*M<(UE}vjSz38(YFA_7jWi&eX9Y*VZ)YbKT+uPe46BFYAYa@|@#eGDmsu`@!52#2A z3XKHn&Br%?z91?oDLHh)ya<1n`LAwnoQ}0o!9Aryl>f+KW*>j^#DVyGkZZMLtZ4#h zu|I%;fN7`+%JVZKNXR7vLnU8ioO`m;#KSZMLON?Y7)Me0(G|zmwtk`6U-UzHny%j| z4FDzIRC5>G98U4nQ}AKkvB_3+2ugf)w%`03C41jE!?1ce{5Yj@YCM^Ur*C(3+chy~ z2+!K=AE4T0&CvOSr@Y-ObJdT0cFD6ekn-V8dHq>1Rr#k!5&vd8;rZg*L4$2}nZJAX zue*vK%|1z}ug=6{-T{1TzQoo2#KZKti-Q)T5BdCr20aEM56Um=l2oRn(_M_+(i^&K z6O5D(xWSX6J>{>5RGC9WhQ0@a9j}3@LT#-8 zwc>lpfMCHU$Inp2Yw?b_U|Vy~#LWINQ%4?ZVe$DuTVQ2JC}H%j^BeLaE`zOwIb}?z zQWt>2NCgX#N(TqFHYVlTD8b+@*JTxI1!!^Tp_k^buOa`|k^jhzI2#vutg@PJ{c4)O zkJ%H$@}rL+21c0nV;}oI`RFFje4W#8r@#7#8^e3pj&Y3nyJ_$+eU7tjD=Mk2b=h%H zfry@A(Wj@x^?s$nOXX~zRxkoSboK}xj&y?A%8n7bIGAY(L=6?)r=4q$A8d{_Boiki z{t>Bb-aO^&6tfXBT>9i`)-d-0e6E60L?GoZ(p%umewvxvo98NOc1m5XJb#`FB)GV@ zw7$iUH{%3LD~;AJc9{@7wi&}*66)&e;#G$hss?EvG&R;_eEUj{#-LW&=}b131H@FO zh7Oq7^-q35$oEJ=klhJp2Ex&G){25Zq7!8n`Nh^uO7eEN1{DDVt@Y>T$n4V2nGa?AC<#rC^&+&miMVK7-*83;Q+DHUhE9jlbVLeojf0HHiuc@1>R5T) z==2aPQz=-qciOjDZ)A(LthO@?F2O)w>GVe~==7KtRuu37+)EOK?P~i4g zAuE`2GbdFcQHgH);^GXRbYgTI3jtELJwIWwlk`S_ugd(x9&TJ$gtUEIc0f1tzP_kg zz^7$TxA(!Lwv?cu}H9AwfJv48DL$H~0>jpV?Y~J~2vSi?) z#Sun%t?kB6ko5b8`P3w`wQ+|eXP3@^Oy?ss1kY0G7|`PQIc+%yzoON)vUSmyk-)a9nVN1FPF#pFo%sq{CO58+FX+N4*()?m56O;e%4B!L)Ezi3x|yd>(}o`BHB= z%@q-uU9_ALkLuWsJAmj@{i=ZYT$T-UDJ~)w#Od#Wz zWNA)0i|e3Rx-?0wWxhk<`Rg;BIa|1H(cIxoyjc0k0Q16M-T) zf~o=~3<4xsavHiiK~+a%m6>72Q4TP$e&?ePz`&U1!J6fPym?~`v&e)<&|p&tghZ%m zYW614D=u_+{zHS4gsF&&!@~wsnV;11bsP5e0d@$CD9rVxe* zYM>q+1>@z=UhR!Yqy`yTR1Hnay)Aa1P(ce@RvAs4p&;*pBMw1wa32^I!n35-7It7t zssJ(tFDV*R2SAreq6`Qo)T1h`G}p8Pp*7yRsLn)hxh$qJ1+epsqtFUGR*;;dy72wb}TCaNJ;4sJ`(lv24hoH5p89Y;XbOr_Y>>DO^ zNQfSkNDEluMVFl&A_ok7!${D5eH=5esd=_|O^)10s_rkw;c(jKmv}!OB8hi($ZzR| zXr6}?|4G7g7PTUaFLq(t71Pz8j_^xSYiU5YGJ$DJaO&v~s&W%V0>}iX(+Fhbw*IoV znp22p%$jo+9Wx$BneSiuc40wI(HaBs8S!|b7ICSmRUNZRT}6e>{>R?cZwq~s<=)0`8L zxOqpa-;`XjS)_lZ2;gH9dBk3#2x;s1`lcA$OtmwL=m&W?J`*5wdRxZHMVM>KA|IrgCL6Naivje2yqCFCy#kOQ*NF z&fe;>pQyzm;$p;se*c5XrXc1^D9eAA?HC9fB{?7W3I@9qVX z7+WvG_Z0=JRVCi8kDcW{(LbTgO~?CdEu$(iJUvVTW+RkvuSXi&U5-pjD}>C}|WnKABHFL8`DL z-)FoX*wK~nXU6Hxx5r2qndwizE4BI4^-e|NZU(0@9M^vaBBEli(nm!{%a?0N6|2J> zF#E^H+Y9GgmD;sIMMV-#rwiRV{LXUa+%UWw1&Gc7n3$LdUF;WYrwzaV@LdLyJxJvr zw9to?3^NytdY@UqxVxCJFr63}j!w>Bojl+A4zug&)}|r=iA`Ys=4$_~*S%T)XX2;s zuU5x?p2XhnRNLMI-j3cK+ewZZ!6#E~!ClXtOWKVUd5l!vIp1{+VCx%jgQ83Kkp5Os ze*!EZAc>X(%hBN_3V1?!kkLX<5ReL5m;f+wqSVF)8Js_;?aBH_QME7@Tg(aJ@-Vx$ z?7aj-EGv&`3zF_Fhgk{ICs~zzm9uJ5tgvo6-$#{S%|C+8$$zP0oPDu&O-TTqOkPrD z6Dl|krvyp*CRy)lmt2xXHvDAMieW&`3izuX;FWKv^`4Z`3-#RFmnDNAv9!fhVKL|Z zhYzwwMl|hiCz}hkCQ65=r+Z=Kc80rSnXtwL4gg!BBH>UQf{-2(P$ee!&KM|239l@h z@>{ZiL@h#$rp`qm1EuA;z=nsqU=fW*O^=KQDb3bMhgVBskffrB{fWz@bt^n8j{6gn(}NEikCpRrSLDw(L6UI_Wb@WQD5}YnnaBPwT@3dg&Ii& zcnnq}(YOe-7WEGIJT{7sruInH29nP5%amC0s)IX8m~QmY2Ima11!=B4i-{uqfiP|* zd<``nVhOaVP~yP#?=1>MGd_a`MkFZJI}qES6rM!O?^Ay1OIms2wT;Fv?Z||VLVg|L zEmGL5o!R1_XQRKZEZpm_p5|{$xu?44fM=f6k!U+7)G~SGSH2`J9;GAh87iI9qbw}Q zMSWj=fYlZ{&-(dqc;TEf`4Zd(x3F0+)tbgzq=MS(9qgCGZ?Ap!zE4yKOIKFB+a9S} zwE=Gx_mTylIN1ME^*0zfjO~Wqem9YSGaxl4)8Zl|E{a*suZ$Lg%cQ8T#v8o*zMjO_ zlbdz$afo(5ucqa3fVs8pOLQ#ifO-eNK|v6*bMVy(r7769C|p6i!x#%|Co~1vBB$?4 z%)LSb@QvCz^%jjq99;Nu%V9!_^PaH2;(XAfmV4v~!i2sx=M;lu;gLF$EEk*fuJJCE zZUo|%<}o4#(NCn$MYZRU)21l1x6FA6WxiWS9a(Jk##ssGOd=(~1+rTeNzgC;_Drh8 zLK8PS0afYOhXLHf-pvh=U(q!3uyvDWWs5C>|8_K@`hLpuH6nj0z z3p7hgB`t4s&p;lv;QDqUJ%GZ``i>*}pqtYz4+OS&Gf|2>qSImX_5&|b;;~m`cymM2 zsP0ovf#n>F!VZ!kj(g%BCJmBCvT2y=6la=wQYL6yO*Q3qJX8L)?qO^WpytZ(iZ}yvi`02OnNm({} zPG{sk3kZ;{UeGW5>Jm`Wst(kgKL{II$tm|X+3L0B8^XysX5k{x{H${&t7?sGT}2bT zbq^Mm<5-tXM|gsl5Eor^A?rm+cPGPo%O-Gh#hF+KO*b1ju!`rFce^dN>xkb`bdnl$ zSNo_Qk$6Z)IO&|0I+if9>%cv1csIoRS9NK*{zCX2O9QS}O7C}CJ)I<)Z>j2s?#Txf z#><<833!s?)|=SMM~%CEm#mv8cH zg-gj*7XXumOlpp3;>wk$gn+;GhHd%A2b1^BdkNYPqD-NqP03eHjG1(8j~sKI`Kij$ zktz*p?$;m7bb4(Bc|&$F@G-rnL;8+zZ~+eyjNA)QhkuJ>da>2)NQPi@BGt>jERc|h zlUHQm&s%Gl4%&a>22&V0Fq2_4BYtFz#yjo4ck*=ju-a%66Cdxq70Y)Ph_gHj^EUer zlcOMH2z&FouX+l?v^0C4t!yb7*;?Cd44M!)B;s#7h8SzFFY$LLs?S8P59i-nPiNPR zw-H1rcjB(s8RXn`>xH$zk4-gSwkyVejK(52upQ|X;>q>Gw3Rs zfE0=Tt!?+eQp!ryDLr=lAln~A&)XG5Kx$6Y-t%Gub}{2%ysXR zr^Xagb)=mZii?o4X$<+PWw<~MM}ve0pN3A*8z8X=pb=Nb492B~N*Cg8+XimVEH=d& zX)EDNsOjlTl(uI!<$aIw!Z9>8ESe1;Y41j%$5cW`2rC&F zh{CFwiUI^CBurbT%+rPx6chx|r$)n)4J6_eGCj>%7G%2w>oRJ*u0Ja%C`1fybtJ-8 zdudhc_jd=gUR_^1)|RAb!=?e)6%CEe=Y=?{vSwz;=N>up#a3jo&UJx#%sIAnl z&aGyP_RjOlCO-GkYel8Dpu;l9v^v|(X)2@OKh%i*H+rIn`$}zs$$Hni8VK8p~{E9=0nkmCe8c*kbj9g z2Si&AV!Jh#dkk`I<}8yJ+xW50FE0(nQ5_ev4_KK)BX{JJ6)hYIfm``nXu`A^?O@aN z`^2pDdWAO+2R+n{6~@}pv) zCyL|8567fB6lHhEwB5^foEAB^f%7z$a;g3W;`qc505uXs$(UWGviHh^{)Ra~`zp&_ z3BYVHi0qj0TRNPL zPBTNqSKb35D1Vk)qKn_WK4^JclNxJQ3hmgDMyT5Pm6lTf_Xa2NU~$}pyosQ8R9nqJ z<+B47W0(ztRTVXcR&{9(i7;yO0fLHvSZK+15x{e6sbhiE(Rv7u4Bc}g``*&g@`Foplj#s z6oE~-IXEJ;gyIRm!O=iioHh!>`bRsEZbc@$;e$Uwviqi>(NIl&Jz6Nf?Yw8( zwM=_XgzsxC3&VfV#18C?^EgF)L#rN@ypIXhEA?DxYfrVbMMYsNaH_KJ=%q$Gr@`@%K=T`i6{;ZpPG=oPV zPizcr-RT?{r8;@Y@6qf#_|NZ zkdj>qL?s6Fl3!DFd(Nrc%_EZA9_)z?TP(w@E{>hE zE53i*h!Z^BW*2D*^S@g)XDA6vi#uO+cV*W4UoU_s{nLT(JRUj4=8?fGgYPq<93&GE zN+Gc2tgMmB%LKz&G|2tm1=7gzxovICg1@T)kPwWV z1(8M=JcZh7-)p;~qvUu=T%TGZEhucu!YGdcp)0FoH~- zbW*;<_0!`~hronJ7z)iQQ-@(OaH1GQT3Hzjwwe`Y2(rq8ML`#`ZHx{tuQ-*S$buH6 zLERzV&?dbw%sG}88zF{giqFIyZ+-um@cVK|W*y^D#=yN(v`H$I zJhnRd`IohyXJy-&Au^KWM?kD7uHguiT9hVGBEW2M%%&0yyqQjYoAI`p4E84g%@fQ0 zvj;X5a6tQVn>S(X+I@7uw2T2xq`nr{y)rpgF$KYNt* z`=9!4+dfYpmL-DmMZ3?qvR5^DsvbF5!|5Gg&fqmQc7~!lr42QoiTuhk)ecK1qk@k6 z;lR)pOlggOg&RbHaxQi7BKlnu18WL519>AO%YCz{;;8V@b?}B~M8QL8rPg2lV~r{3 zP#o1!qLJG@d0^bc~E5>IVjL$YxBGUVXfY}$KIi8=WU-ZFGeg0F{5Et!NjR~Fm7zcDtwJaQ`Q z<5@GtdQKexR5>5rrj0KHWb2uH{NS#(G^wV^!p)ezo^2%{o@@C z$dYYh>zs!L)jTjv2O$(pl?`)lOu<0X*4H%DR4A^_(L1#J=qJw1{lyn?0==h5g&w zQf$%)B`pf;j;`4r`;dIV%O=abm=bKyfEXJd5oWrU+!A{?Cj#Hmu4ID`4`{OWrH|_4i$M5;6gwAlyWc?tK=$S7a64D2r`{}2g(U?ck z#bt?zNMIHN#>Z9o;x3SnpS}3AJ<9(1T_>Qt@VbxdowKv;(LAA_sPA`EnN2PLI2r>= z=&I|wVpCGE+uPeg<=OxX6G$ACw`yv?N2YEwP-X_eI!v)MtOSoe*LANr;TeB0rL$7QB>UuZ} zJRMoL*b3@E2`;rMFHO|YUOm*!`DtTtHa_UB= zpV7XyIn?`O6d#i0gAx+cCUmi6%y(XQN>Iu})flwxd1QmjQeFWTBx>mQ;8)<-ct8?V zrCdmsx!x%v$5!vd!^!qz zDHnC{>ly3I{@HY6Ig;@W0+{cnEDrE=mKKgG+_SlIFKyQ9aLn{a^CqL>zq@lY z!r93r*G9^xOgK1Z0T$^}E6h;ui!r3|0YV8F{!!phw=G)vEC53%zdJtRdPYwe+>v#Icv=a47N=j#pHuh)~C zHA9g9t`hMs4MC4Yox)n&xg;$;=K7+b zR1btKy#q#J5u{*FQ+Bl**MdS^yvc%40&-hly4ZmuUYak*BEst94{yf4z-Y_qbh+Zn zSv{4i#cEM-Ooa{Nlp(}kvj~|2(Z==7y z`nZ5qk_E$_<_iM$vYFGJ0V}PVTPpJJkMF**8qN_Z04D$zu3qa~(Ov_evRJ}m>G4#6KmFWY z;94du{o?eYbHIp;DEDJ(bL{83>ExNAn)1P8`AIX9l*b&bg}cL8I5EvsZz4C4E(^IdVyfezB~Aa7noP&a@8h- z{7qi%>fT&fBtOH&HLA9v8hrC8Qb zF+)}Ia10UjU-I1Jo8L+So4;ew=g5uGa4*=3wd5*Azb5XaaM60LQk$m{n%27FRnyvD zU|>Z0ofa3JJUHg15BtTD19B@}K?x5DAlg7KJk~&jhli)NrRH|2#JFAY&!<8!gZ#it z-~pZU_{SsGGwI<^2)=Lsc@Ss#uhiUc_0PJfP(A=b|10eEx6URt0t>*z1zeWGR2JMi zj62CklvPlW;DB0y3y8)O0~+B3fXUSEuLb<>*@kIKS|MpG2NSo~Qb+uC@`(Ul8>97m z3{w+VXsiM%j;t|ncf{_+d%s_h%Ek68`iNv?W^5TQ{D`n0$z^+pm`#-D+(+1yyoG1j zM6>l?xsJtZTDfsmd1{zl zpg5$t*;uzE4<=xP=jV<%!!H^)OyHkVzJ`U!0)C9djzw=W&}&wOJje}YL2zA#YW3q7 z(n}zqR$>Vt)^mVt@+Osr-?$(>XSO?tkL$gDX} zca=uOR-Z+yKw>P|(#tC(;yH_;jUHJ>l48+Ulm+2f^}aTUK7o!LqYH0J4{=NgnTk5Z zC#)oeUJB^WBY@~H@*ERVCP}RaUZCa*Rtm|UDSRb%$X=>P79X4${p z3i-L>GGaP!?eqvVW^8?PryV+Q3X14@Bt2MsMklwDV6o-nX?-RZNfcAMS&Jr8G3dmm zMkwBT?+#(Il^88IXtd_;zWLKV^jgq32`lV#884!H8(EV0z0jd&R8r8f;O+PIXOpK# zkHYNpx_uzjcU*x4;|Z|DjsqJi4ZCV%mS_rT0FI;mRem zgw7HK7gM5V4L2X%+Wj`JlKXo*P92AD`d&%#SdySIBo9%ol|?9;hFd@>U=lf#g`9ME z_({A~q%kjpY*%hH*b+WSHHYZfa=q)(^R?Sgb34|62NGGgiF$V6KHE-8r24X`%)hr= zcRnCXN<9)C(Fu8s)_vYAY&@uYVhqW=+&J-z*9h75LmDh~T!&4HH9u$=Tp8V*6^2|2 zcRl`oMx%W;TDgwobSESh>ALH_8{6i@LiBd*Ec}&SxOHK6M{W!^jCm2BRI}f45~PZ{ zNKj#KzJwZ&t|;&nvGePSZ{3}6{6jYRe%qhI9qSzEy@um9Vos~&)wi)e+nRkWQf~^{ zi83v3m%yCMU%Q<*&C4ly;Nwj@N#DeGKevo+Ir>EFntMO8=ss&{B+?5;$ zo(E4T!FqLq@`iZh(i5bkFpe#S>@VIbFJ%#!+Z+aNYHjiAyUtbR8X_#-;V}wG_7)eQ zQ$hB((DTKV9Re~cu38#Yag*cfH-~bGl=ebjVhm+?9G&1nyQI} zulWp@_t=(|+-@T)-M$!)w;S17R#mN zGpi(bww%4MV_1{omsBeAYyGc%Cgqm|zL18D^iZpl9<5IFVPd-Op4+U51u*a6k~u65 zK=}4`VrLhrWcwjTdA7?O?{oAt*Zb{1KhZv1+9?uGy2i=_1jO5?AMU5de(?Np!e0W0 zN{>l)2@OF}ucANFU#Rko3&~nZ&#wg$a!_%5rZE1Wqnr?e+!Sz+-Hf-;%`oc)G$1-<;j9c zAF3S>m`SIOpAdKDvS_3s(TJ81f zZP}~uZYVqgs_sRvpSvojGi!T%0$G=;PkZ!KUjn)(IBq!4XPC%ROp!0}fB+$X39E3~ zx>T1y&5(&&dg=DTG-66f7N(+wx~v3#zy2{LmO26fAr=IONA;+rk4V)(kDidj)>e{7 z{04f~Y63*A)?xZ6)CZ@hw$|1W0D!_kkAnilusIbl4*)F6ltZQwhGdF_Kb))}c(EkR zwPBVaP-DsTQ5@0#*;p?{O0?=%yqz{rX19{4Nc+(B6ijiyW?ObioAxpc3KUPC3rAqR zYag7|Vk*B~x@k5iA&@~X2zC{H6vtk1CLGc8?R0K-5tgy`4b!CwheL2FG`0vC}wvJwXu@dB?%>KB|zW}cAHAY_}{(_{)N(W7JrzEc_X zJ%mueNeeQ#&6K=~93WKztDjPFo|b-fMp^46s;dliV~2z)DjDQe+*NRI|5~r;-QXWN zI}q;#ppsH&RF+YEp2{4eLOm0brIFrz0rcls@KuM$;s}) zbm`>{CfA$(^S!xPX29zGZhI1mjf=g5C580X_4UdF(l0de=t)IsTPJuJ(Lf&X$Y5wle08<5d9%o5*HWO z)7$&J7md^Xa*>p;#Mwk$2sn?u#GP}8U(9T|^KSd>e@Q-`I3WRz!3T_3yuhDnfjVs4~Cn1zux};d|DV?O;_-4#ONZESx z2*jc#yHnI;56X8edv4pWtA~GwWL07+wz+at@PFBErk6GizNJna5{`}&K61A4@=Ie3 zZj97gV~{C*zxi!tc4vFQP5>$a?)}TG`bRE5G<-C3baxB4`_ApXgM;4|gb@ggt-uWQ z2?-v6WM(;VG{@yVZF_BZ;I6)V_s$%8F$rB;3$-y+Wnp6~YHQ0bSINB{(eF|aOWBY) zXjf;#Q&{k-kYU2xS%2ydDef!*+X;5dLBaBAnGtuDA@};;Cu`wYVJ)dL+{`-fq26X0 zV1k4&_51^}@ZofU@;~%*oPJYH;5h2?vZ}_gXPDh{KC!$6YQX~>T$aHjwY6>7S#2~I zDr=U(4G8W}VXu!n=i$%Do%IW&z!Qgc^FH25_l7cURbKR`n!4P(X*bn!Ddc*mG;L%A zVV@7)Y$Uh%`6QYTl@sPR`T>I`IE_T5qlM$(;9CGrH z_9o1sYPOcial2RkZTkfsZLkVbA^%Gvv%7Qsn4xV|{Dw}yZm#G`ZvF zM@2^>88(2NlfE>RPk9ueMzP31`BX<~L?~<=9nqnD*(`0w8GtgY)A9q4n6se#fPCvJ z1K*;89N%?6XJhp@7^u;N@!q+5nj@LfVbnqZ1 z-!q=a=6_7Aw(v+*R(2}h*-?Y}y%5QUidC~)#eXI|n1sU6YWe!0jo z-L0|qYizQ#+t#m1|2?6AnZ@Dq`gPOisv4SXRkw^ZI4GaJg9~84cdN^5H39XS-) z2xOTD+JJ)tGD~tpL!b)nPJC*FP^lM%laspBdh`uNpI=v1R{xX)-B!dGR~9pp{Fx#%>TYm17~ZE!&!#e=uN+a$>Cr zlhfL2&Zgusz!CU_nbk@kze^vz&2g>PdN}@W$FWCubic`(*FH^qv00`V5cCaP>*qsf zTQ2+h73jEMm&tp)Ok`GWZJ4Iq8Vn$R2t}I8t{l`a5S5n}XUZ5G;^OgT|DRbA&ncFG zcqAchH+q5Id|JBrz9N}bnmHo56B`kk0g&sWMnH{}0^5AFXiFipLhiBo{VPC1&2xh& zKY$Y6#)BCkqB3#TnTv@`oK11*fY@A2zqp>MSEkXVD}y1{uC6_vCU%eaM(@PeMqu-L zWX8>W-!qPWn}UjGA$co39swOnq^SR8l#m%y0Zc7rQx#@Fkn5oeg<4Y?rnllOxas=x zuUk@KAJIqMyNQ+UnemQIL84F*xu*`it5-J4c5V?FAcOVsosujUKJfo)YAT;Y0#-(d|63rN4?{sLb@;=;e-SmhYJ( z@rqQY5Rrc%p_8i8`-(E#@Pe2pD)CT(WJrWLi?8)d>F?-9*&jA~nek#Hgd^V}b)4KXmi(Z|;Fc-cc&?wgJUcr&RHz=a^2Qgea0 zsVL|f6{K7ZQ*+{$RMu7QF6ECy?j`Hv`2ftq@8W%A_-LqhtQ?IxM`qvuwQ2t_ZoWPq zZN@-Z37^{js&X+F@`N@MLfB`Thz%L$@+{>rowThhCh8YGM>?I42r_4AY1*C3$Uwe5 z>cvYY1Q*6~JQoGzo)BUgp3*~G<)fU%n`6OY1zfyEeNDO?tPw<1EL4E!3p}#W7kE^3 z<|!5*xEm5`$Tr+7+r(50yEA^?>3g{Lg@9EOF4QaE=r25xV{p8ZLH1k)ZslS|!hg)~ zKVO`k?s&JMAcP&X$NBJvZWzP$=;Fwn9ntvW%Df0OqYTA*Af%Q;#)8B8uyMiwz!9N; zlNmAq6%rZn@y^WRXt;mM*0+ZJ8l3AN3dkD%YWax>hx|2V&KVT~m5>^8jGU}RD&cm2 ze{hLArynx18BWzQRKj_z*Dal2L&!TzF zlm-_`qmdSWY$5X%E4>3W>dcmc)MKN94Eod>%Ub)45NYF;xA>&P5cN}P$r$6WvxfRA zvougbLnOTyz!tQHT9r`K{y_x?92RD;c-?<<0sh4gcSERTk~(u{jF5uz0SJXe9p>K? z_tnP^b2SuCs6HKNbo5W^Rt@}`34q+$la|?jX^nn@Q+$NlKAYzV$oTr&!VbjvymjWS zR+x;<&j`yrt$81J$E)#ycqGFnb#k9e4Ze?0vpGS#DDA)aP5&HYb~rLr2*}jap5!f~ zk)ykS>+{#E+E`Wg@cV4!DK4L$fIJ>)laqDFcjt%$^!$Kg>@3Mqhxs{R%8L*JlZXYEB{u>Bg2dKf5~W69pcv5Iv%IH?q81(k!qde+UP$}`O5pj_tPK^U z{@pgVC;8M=p#ai@p#HJSo&@+5rCZC2I~rhnfg?Q-q9DKt<*Wbg0uX9ZuS`u6QG)`b zJa;_m!@t{)p+XBHfU!-6@d}Pn?*I1v?_>Pq=LQ$?%m1~7_E`g`{l7JgRBWvOwF#;I ze|iwzzn8Jb7%uue(N|3p^X?%|RoyIDAY1m0t{<9V<5#j$Xa0~?N2{gSwz6fRsF!-H z7kKxbXolK*o7S_n_v}WrO$Lp=EHMOYDHP--<$#W(DLK!Yv9TY8Ck8R<@c6*|2jw76p|ZNOK#V%tD|>9olPghT zk~z0>o7CKtc$4)1ehY40fH?H=@zLPRUv2w_CC5Ob>_e}VLpZ~TqD#A z#+tPbO898v!>%m4M_`3-=#YQ0Gx z8&jnNZS^UG>9)4FQ*5%kx+H-4L=YG0BfA@voR-!P6nrahoYB#BRjhiPXEW2 zqrC`=ermk5`<}Y4JD3IE&-t1c?%4Q>?dA)`9i2R~^&T_3YBBt6o}Q;7bLD^!yy#sZ|k zmB9Uqp{vfog2boRXW4KDJewiiH#T?YyT(CRb9D6d*fDaSYHRlzhA3ngylOkol>r%{ zN!X%QQUgGb1^W2AHpwWkDX2LKV!EJAVexz2!QF#2xxwL;e3RT3h{*{27XK zNYpdyrVs=3<1k20dfeq>mkDo3Q9-5bHAv!TDg` z5WT)SqubX+qmX;7oQjLzLO1PoT$M~QEwS1cE6=b-9|=M;0sN%yD3=Fc`fa;{$-DhT z+25bGqPvYCFt*mqraXS?gp6;Ku>LBRdp|pr0_ff4cpRwZoKwN~DxiBOdz$LGc6P;* z<7wxcwNbm@uV|kOf2Q7)^NKO60b#pMR6f|)m$AK!YB;d+Q0u+@sBfFG>iNT`JiBeHaf+;1#LG}CcS7DEtsvb(_HJogr|nsl+% z3CR;39GH$;dYY9yB=!)98}b*D>M0SaXF`2XhJFV}^kK|2N}7*=hMEI0jwZCCRNji{ z1G-!vbqTiUsW1M^&?L0fDJj-(Ii4LI70kk%OJoXUB#DP_Yt=GBX8j)9-zXP_!5W;oC%xOZKc{b24(Jp3 zSj{hLU@zSTsAqtU)vG(>Lmsv%^J?xc_AdN#aDB08LyA|{}+&`B1`SD~8S3*>%OxQOJSd^1oe2n5>vPS0KJ3AbSEO`lF^ zHb$SxUT)dh8xkesK8QYHs0G>;_RzA930X*kW@;H4f^2N))6>&+_4Ks={<4d@m#9X? z@4&DhSOOmL*rNdn@SIn;4T8(r#B9VEO9HQHSUB605pm-{8;g!^yKF#*|E-70)~o4t zkG?wr-RzJ!R^bHseSPh=K<;uqv}YFK?)WCo){khJW97-1;OLncyM~fRa3pW+Z7T{! zgl}+WVMaY91~uBMvoj!~`9n`g#Cv)U_PB?`(N^EY$PlqJbhY5{kU9MeoXM`QaZj!h zN}p1%F%a$CX)sh!w{6R(zn$XeM3ug4S8$ygP0V4KLA`MO$;LG`R6y=^iOBc$fayxK zW!grqoNYYx!;hg*$U8gox;IYc%avA5%m|(Aj)7kAPQkcb2dgIXZ^kL$q9U-*8a<## z$G?!xEpd)lChimXeZ!Fis(Uq;oduk$DwgX)KSRx)k>1R)-+JNGPW2^@=e|22lG{JY zLO)p=kNP4?9IK3V(NO6RPfCi!EgbflcXqx%F~7t9;$D~mDwl)-t#6AxQ6rB>;D1Ul zmbYJpA&HWRo<){s%FaRgct z9L0-(qfb>o)fbe|bJcnl-YCuuzV{Epj{lfc@r$K-)~k3_AT29t?}T(wbAMpREE~OL zuF>Mc8;O?cF!=horlhJCGyHozwzt#Dj|l=!EyEsk-zC^!_3<&X`T0xU*I|7pj_bEb z-%i5iyi(y+(M|f6JJYMiy)N&=h>!!P=>ECz|NWjgFdKsqjsW*pm2op`A|&lIE{goL z+L{i_@0TF3<{Ny&43Mg-^iiYfa*oPZ)rOI=Zi?Ks*2tL@uhR zh`Is4A1$)%O~>=)>$Sl5nXqPflQ;6qgcH}C(++2#yxe-w2pQw1#Gj{Ovh41LkYBe1ydzkhe53q92Uu12#-c-%MjYxhG|7NX;%|bVOpa zSln@CdQf^7szJ-IZd*9DL9w>DG$_hrg)SgCGqt1`5ngianB>Ywxop}N?cymJD*O`> zLlCjt3Uakbbg~?ESo^~UQ5J1RuOp^edaYMaB-=3C`Eu&TvJXEONxXYC>gsF zjsdu$S-!8^m`!Gu>fAPWh6eaCL6_6c0*(%HK6KhVw&=v_>1V^p8U1o>5%vLIxU!(b zcxLv2;v6}o6*?HV&9ARFuN#`yrS|Ev7}UO(I{tP`F*}yqla$rQ%?g^H>pY(;ylB~1 zzB@*Qg6fsXHY|P_qWL`#c5fT08TLna9JVCx#ztWboEg!~wX`U}1svw+amge&VZyeU0sEWZcNg&n@-D;ldtzo0A9 zuK`V*-&8*e~uz(_a86Q z&vg~zE-(Esu6a-}y)_qRrTf#ggphRXzSc#4G4kF_ATjTU5h$dEiW_~tQ8v(3mq}ZT zBJXTyiup)QJ>ZcgnyI5cx=TenJ9*?_O5GJmGms4$CIeME`G_n0GT2k)9C8Xha23oo zTbEL^N+ul?_v6v>moid};b$jfHF8MVy8^G1?^O zSiTY;Gxs44msU@XbJQ0A>EW{rNDaLs49>?*i?Y%tXha{>WZg`pR?Ww?{AL4AE&LEG zUDTaE(L=1IctKP(D9Sx{5dTUmMd}rO2r28cXTi!o?91uqo=2;ByGXTt1y`{_i#N= zdEsj%XkQtUV^r4>-^Q!m@WQ_+`VH&zuwcl!P&6H$>!?PstW9xlP*m=M>!^OM&3H50 zUcL{(dg}>k8UxEQ@60*sde^p{*mrMH7Zo5}Oj(D0k(7|dRE)J6Bqr2tD>XDxEy}LDS56I>^sSl$YW1Y_;^kJFxl#BYV60k z{(hzJ9oL97gUdNTtb6?)b?%D%nAiRZ*s-Xxu$;*Q4)=oL%>7&cez)JN%sM8j#2|Eq z_kw3<3%@tI@)O3MFZEjwQwg;;^p+Gc{SB{LI@(TZB|SdUM-Hq%p_tVUOCegqe_J`< z<(z@MoG-)SxL-GOgO)ftR!A?`65j2Sckc`cdY5*j3jL{AZvFKjAAhs>0^aQCNbD|C z^87JiVQYlAkKgQjJ}aRvmR-ss<9v7Oa(4h}5+A+W?w9rb85Bsor}x(IXy4{~kJ@N3 zq`%I1gpq)=--;jBa>+GYS8@)sde89vyO#fJ^&IT!!!!E`bk~vNeGZd;L_)fhAImDF zl1aS0O%3q#y*dV6V=}S$j)po0ZqgsoS^t2*GClFUhSmhwmA*o|C1_Jmf_<%Ez3ita9Ag@An^@|i}*m0O;AGztX3`i7Mp5SxT3Fy8(&vjmXL}JDsjpR4PVn)S#Zw9it@vk zI{*oZsL^8Avf+0x);~%A62g4NRJK!L^`*|rmrB_*5nl&$%2;1;@(o<^#Iv>C5R@4%dM`oYC>uS2MT`+|Dv6#S?EW$;!5Gnj62jCt~-7B@juE%c$S! zRuIr%Y!-pjyioVq>wdHhX;?Jm=cKb5E-$Ko*apdrccCj9a=&#!<`2#qZkEKl3J~r> zXJ8jf8jvSwSJ2f~B=!9TdBdk@MK;;L1sB{clpZzD5tVPh^QzL0H!21~m)^)YL9Nd-9=`UVv6HW?fy zBe$^aX|H@h%+#{MpnmxTaKV7u41*@y*Fix+4~H5dG$aW;4)bh4P6#L^Cg!z0Tk)8j zUS1vzpmshLPAmU}z92T58p0)PH9-i;7I>w<*2Fc{BMoNTu8;J+{G)$SRSE8JHW+Kn zm6*Igb>7RnVb)6<5xd$j5xbiakbqs8xlW|r)wJFk&aayc>Ua6ESMHM@KTMy9i4pC# z*R4;0<7$iT_T$8$eJ@cE@#@+VtGkWHn`|d6@4)h&?MkRMf- z%E6v6nl9fy_s%cNpWknONkd>Tdom2}+Q5|)GXHel5_q#9Dx`(MoD?nT!#|ByyL%6i zTmKw8WZ%DqkH#C@X(#2}4}(E;zn{HWzq>0u!$1_mo;#wjP)xqmwsv!NqI7)zYJ!3TR{0CM_XU)FfBskvJJCtOfbqg=T zy@m*-!V;K5m$H@gfe;&O_KGT#OJ?%=>GU{qux)&(VK(-ah=!m>@=lZ`!>b#*y^J1a zL4-~ge`o<)vm!z#ip)t|WNA3sJ_yI5W%W(t#l;y`7WM^ij|l%w4Unsl5hL1=F)y}7 z7jRkAR}?G$;nrnzCXF6OQQsNH(lROk>dB^vdMPkdDasV5BK6wa{O$4LHXB)yd4o{H z`xA6hl58^eio^c;&1P0I9LngqDHOy&Db&XV(RY{uS;i@S!_D4_pym`kp_G10QGvE_ z+r)~y&e`KNSz{O;Dh@wVt`G{v;CNol_-;|1H&&FdQE?Z>`{igzwpc1(Wv@hzfYRfH z2#+Jh447w_gM5^_Vq#bm^LsGs(ss?_>gTjJ;eO^B&weu8yV7vJg_C|7!jxm~ zk2wM}WGvA_+w~SWWLMJa4EQ>OXZL#xh6{&siHI>Gu~}Lfk@=X({Milo&;}7F^yb8= z5epPzL{$}LI`!$rRYR3{L3?WdpGId(XxZZ2uX4%CgKqRla+aA{l?NfwALf+r$C!7r zzO6j}IvTz+D|(pa@xK0sOSHMT<)P_ONrgyBHAAK#v5RJKaq)4^m)k4xOKZf^cH_a? zeMslLU^f1!Eq6JfeF1E}m*n#)ms4U+ZS+IZl#u9=?Vg+SCe|CHAhJ}pR6Z{fD<0X0 z?deREpXaUb&gFPWNr-=KF4jQO_SI<>op%%G=TtpH655vJYlsi&$hH^VM_dafu}l5v z#W^v+{(V=igpc^^h45thJqWgH8byb@gnq(${c~ux+!iY$KYcumW9(t79@%1Af-(W` zQ3Wdpb$E!%7Lxu2=llrye(;Vm*C;2u{xT#h=g?Zx0hO2I9(}$7OFiaNd7jlp%rR44Xm6nsAA&-Xd@kFDzNXp?eVSyA%y&1$Y8 zObJK9lV>6L#Cw9xEtHIw12ho)B(t|COk8`1FX?+oE%4f|WZW;tubOhR;Ejp(#NLZ% z^(WsWe*iV0b(j#^(~!&sUwX89J1Yvc+>4K*=^8G(Zo{dSE*`MG8Z#IF&V5Z2F+lz$)Ur2Z!}rD z-UTeaM2yZaLYR=i@$%rM+sfZ(G2@1x{3%8y|L{)~n!FfZgzDnu=5rwld9>b#Ed}o7 z-kj~mO%!!6#7W{Y1ZWHVA+yq-N@2yoeMNZ{V` zeuyGI3Ld-#H&`?5yDeqgZB%4-KW|YKnBCz7w;|-(p|k`#i`j9Q{P_MH@JiqK@)gc8 zj*Z}B9&4qY)f=h<*UilrIsjXG?$zDN{nfz_jV@7&Xg-;_ItuGmok`xbr38n!*o9`v zeM2b`kPdnkDrz9ronlEs(sioP?C3(x;lqQAxA8m*+HRFPVJvdjtwv4LCL7pb(6 z@}K#wFU5EEnHu8IH#Eg*_8$mPL8QDKg7jqkCjqGeAG&8BxLP^g_s%Xh@>FL21 zI~U5hCM$GFcz=l%)bj!b1>a!y3l{0FLD{ifn%t|mS1gu&yYFbZkFECMlibCw!>q~n znEJZj-oqV6sF4<|!0H1ChqliRw7ycozfC*+5CwLmUbBD?LHHNA82ap;GDW4(uVAmv z65DU#cBon~9K0q-vB|tlyyCa7Yo1K#{5@g0FsrZ54pRpJszfI47<|?YL8B^|XW@(s zJlk=0O#!>I4_(v+b?g4G)YqM#+V6$Ey~?osPAfYt*|z6J>w319$7R>M;V*d2EZhWs zqowXo0Cb{vlMATdu=Mmy11doQkTaE0qXIx>{$hnd2pSRsc*k%0 zhlX^FjpIw0jQw`g%gV~mE()LXfL&fX1ol`aNHXC*#s66mY<>11({}$!ullufOaX%{ zfvE4%VcPRLwz1tMf>Q-Vr@^|zcLC>oH(}^YBU=+>13w=AIjwT!nD#SjBSHl>t?e(O zAaXIk3okK zjX$rbFFbWc=i)*qPyJ4Z5(Ce-js;I@MI^=7m=jEyocH7NJn!(;A{EaM(ozWpRoqLg zrwzcF=*>1&o)9(+{9RF@ZSBAkS`wLHMr4lCuC`It&K>YK+y0ASLgN$5JIEq;WeN6RTXgFMwVl(}@p!#Lmy z>|HJ>Uy3y`C<92iMIP9%4O~Z=t_Y zKR{WiNkz1vJ6L@PbRQEn^%urIms5;Jmh1HuecO;7xl=xH@jLcj;Q-)5HoOyqHTGNZRLSot zRS;MfH5ITg8@cVM*=ie7sK@ocWY98`1ajCjJ04ul_jKnt`NxcqPgP?Qjcl~JBG}Lk ztiB&b|8B991^S@NWxe$Yp-a_34H;T16$W%2r`!R z(sm1V%rq8B$J3d(h3_fdr&EMB4J=0+VR-QZ5YzE;Va$mYFp6EVR{ z$Fo|WP2$`r?Go+HLyUufZp=Z2efui=FHBGUtOU=Gi95UA;~!}CdMErEPZ%5B3o1V6 z5YAV(oyY^ZxpC*`=R2;nrrh580^G!g<&&{I;nZ@KOfVQ6Ho0Hg-k!6!XA7_Zh(*2G z>hN@p!; zF_0|>szkK#KYrqY5*ChLrZE|XoU;bsZlU4j!Nf-fZCs735Hl^ME$ZmdnvF$KDP?0- z{j0)_RU+<*7zci7Lfj1NK>%eakWcQweq}u4Cx#dzZZ;o7wC@{R%INVhO9Ubm>Dchc zF>J~9xx_h&XnO_w^=~oc`O-6S4BW+RucCWj!Z3Cgxi-!5jP;}#Nzkygwp@9~nqTJ< z?YxMtPA2wnuUFPm2oq@){voMspr_Ov@PC+l%jh_kWoyu~WXTpmu3tuA$SWo2b#L}X;d-gd-A@o4i$r>CEs zUI?Pom;+q(Ae3+GDvSJQN_u!>veNg8TH;W{n}{;e5||PQxC7X@1JJ%8Eik_Xg2T4{ z9F~kc+%%-e0vWJpbT>1TA)SxU_d2mD*ELk-Ce4mS(T$L8^kzzZ6I`FS+LUNZ8Lh|M z9Ttlb-LIrB3y_XsM4BcD2Y&(U6(i=v34-dy_(iOM1KN?YCt`y|jO_mq9E(&=m3KQP}BrSj8UlmquRz+;>S zqCm1Z3=%^M-WPsxvq0ND<(#Up*dOqT8Do~_?3E{FXYZdj{Ou97i^<~baf_i2D#8?y zQap4kh{p)2awKLt*%fjMDog{JWRPkw4~Lu0!yc4^4AMwj&HpDn;aA4_IRos;3rx(_ zUM-B~-3?`kVli&xnizq%1E*_Bdb{GAk@`+tJ;V+TqbMVDbXXZ;s-ED%=N@{+FU^lemDQ%GS(F2eGK$-_&V_WtTB*nzV^MM+OO3fxlM#gV|HBBJ? z%d7&qD?oA?wN`UneLeGu3s(X$10!QEuyZO#kMSL_zJE_V@)QngTAWyAy#1CTpNXlr z<`8fPj>9X>=Cd+~LE1JXOF_8j;fUyTQ?Cd|-B;RUr0T6!ileknmmCWWy&T$nN_(f9 zKS2hEjHXeUf%Pn_<}E&$`z?;mjF!}|K`k#49y|r#w!Zd;KX5j`t{Pr4I?A zwa+_#e90j1j*9y%xi}E*DGb%L+(|Tj9*K?L9eH-ONdp^>%`kLCy`7O^UQOyX-Oz&rR_I8>a^*SQU@sx^fZ$AHmz}M2|VuG_=N}j_F7o zO{s+4$2fd-Xwi5>h|^D$(2?gv&CjN%*%=au6>O9dO1~Y5JY&-6M&V%^zE$~Gipq>-{6lK zme|jLy-5OP8qYUjm&5*;?dp(1Uf}zgs)gJ|s!a z1cn*C(TH2{Gcu_j0u=8?y=|irVt~?OL_BAdSBWr}gi?gR5n>=u5!-}#7yA`1gaelcO(52+L*pJ8nGbJZlYf^6>Aa@AV7>H zsIn{W)H2dIe9J^?@xbUh5vl~h#pA36PWJd^>__tcy1 z2s9h52+$(!w}a6cy`P+bd@q?|MaGkGaK2wyIt9@$q#cQe|reERDUfS zq%&^($NBo9xAOif*fwfb=ah(ZAdrVQeR@mx@ktxG`MQjW<2gr^J+z}9zhOaB{Z{Pr zpaU#J*;^}o2YsexHD~HMPkr(^n7l4j?7IsToaTcchM7KGM^|nx--GMH`xfMZHcM)F zwY7n@WL%Tn$==?9qgv3NGo8abGD4k&C`7?BEvaM6Vo{i;#1=d;yjPq=@I_= zgyd?9{eA(*%UeEE+|pRotnNLQ@8cv>oKkoAv>_vjqs_sktTz;DV*Vr!_g!16d1++9 z`lI_>+ymESx;w5cZx@LE3qs??ES~K_qpmBx{fs4Z3|HuvRKA{+QG6Pr!YCnU6i?8k z(1yxf_WjkjyW(^{eF9mYt~fP%F?OUu{G3>#$kmaSMDF@$$mQQm?1&?{P-9dHBMpCB zc8K3^J{ee(%F_raM}E?){F*IOcj*tl5f2$mka`<%xp%B55Af<P$az|(IQ9V3AagAcv>=}%N1|H@GZX4X|moqsu9kmj+5p$rK^^)>QnH=J{ofGp`$U>c%x5s%tuSYG3~|1 zvz(zgSRw79%5(lUyl*``5|imix}C#Qp~$9BaZ2qM4L5S!l5m<)>hMo(A91U|>I?J% znH!@NUFGYA7~WKr4|I7U2^*J2ISQNUEJ*m`4Y937{?ZBk@f(`@h=^PSyEX-Pn!U=* znW~KxmZ9`Sfz)%i=AK2NgJbYgz1c#ZbnXPndjm}62#dr$ydN%BfeW;s1`EC!V$teQ z%+%)eF{Pco)rEP>dWOB~;sZ@~8wMs*4wa(ENIlfDd{ZcY|5o!Ktk3OhQnOi~jG~j&1v*s*TLwP z-g$%B;{Af{?DhGxqXRPj+T`(+?e_R_*Pu~TZ_16-$L#ESW^pTbcjW4m3nc7gqVd&4 zkWG>F#CAB#vHLP_m?ED3Zjlcqt;XOimC40WvTr{tceGG|)p~nKFB9+7tk(uQ<#|Hi zHa?KNJ3p`F;OKaCb~dwqHhNP^(rVS39*9)k8c?{IVB7X2wPfO0Q~xd zv@|gw5xA-<25|8KV0<@MYFOs^t6`q5Z@4zbuO2Z!t>?;Lo>Jp1BWn%W{pY|GVH9s~ zq$`d%a+#rmpfuFpi%m;DS@>Qu5x=7;Q1jQv(#SrMx0PTV>#_O!i5!CSZDlrOvKxy~ zqTNNn{`9=39|cRV5+`hkl0e zbhY2q)!8|%@P0vn?XdV^tcg?jQjU>5G3B^Pw<`9zQ7@yA*~&Mf&9?=)uMIqe%l)+a z9?_l0e~@xRX+rnxmF0L3u?)L?DQc*>oHu@QVgVLxd!cK0NL=6Hr0+Kv_Se0UcvT_ zBNDcbFsahq4LcY*9v6fT3TPHhj_=ji_MPty8m!c6enTcmSfF0Evc#lRiOsWYua{xs zL$b?}5}B<90+(QR(%98W;kwzh91{T`0iCE<4(I!R>dx?Jeg86?&>F@N>DZf)e?}G?6zb=+L&JgUO1c zu;qy^=bt&D)s2r(jN9C)U`E_Y#E2aTQApwwu8_FDpl!skObTzODQL-HBRwyPOUf2i zYHlx~XqmS}IhFw4uRu}7*IZ?ZbQaXM;KGs=qfycr5?&76<*)>K&T!(5(RjVSGGhf^ ziu5K2J@EMl;)o#`A(_SxNMFI#LB*W5~&VRxf=VMF~<}SFFw=K{#%(qgH#hgA2k6^l_iZ_FE z(RZIX7=>cym@apN#=051Cnx#+z6hqp5pW~Bee7qCsb-HM)NIoD^W521US4+0+x67B zfT-?L6G)S#*7$yYe6Xg7Jf4+F+-F%UV0}83@AGv*t~4rIQIcl&LWBBZzZkD?W7A(I znoLGV|EJP6UQu05{fI58YxCeSg2S+;XnAtFgzDpgRf49e+legT>$+^CIYRI5zJ6MG@fE<3C39`EwZSp#Lit8UA_$YOVf3s)`}w?kYc&>(bmNa#|L56yV?9g}9Y&3LR@*maJ zQ`@ZdO-4Ef8I%Cvqb9|Cx$eVj0yl7uy%aio zPTYg|Hjl6EW#97srVe*2Cx24^x#gryjaU#cxSA22OwRbyP(5EHW zl_-kmowRtoe(uATTRG@vMLT(XYl_F&f1UdXF<=nCcue*_AbVwb=o7tI-nZF9mx^oh zjNlWyNOjYBe{ID+3MAjUcksS!BU~S85M3q=9N)GLab0k*{54d-h@B}lS}T?+UAS?t z#Y{{{udJ@xx2?A;S5#JRyR-pBpKI{ht4ftB0DJ)~0)k@s!cv_nLPUWc2Bbf~-|O>z zVHxFONm&Gtu80;XU%sHGt-Zh169|;H09d^MS%nhi0D9-er=QEzAN{vv@_swr2|Zp; z((|R!&^Kh6a%w`M6cDgMLH07YGCN5NY&X;$qpe==O3_v-R$96dTA%R!j4i3EvELS- zv{`{Ld~WuwM=s7Wx*BI+m?Q!nLkDOVHhYB^lP08j;H<6>&dlh~S6*jnPTEInS1XXu zn`5^`7JBP9nocp*Y6&e^HfJI@?(nBYmr{*a7T1EcsT9do%=qkefX;uVbvfk(3xQch z^NZ3!-JSMSC#hw0_Md#i5?!BJl=tz~cxDRl!MihbW7Z^msxgu=}7^RV~LgI0;wuHKJDc25t;ckLlem*sIS-Z6=lk06T$ z!XFZTGxuwfqAUXs{isN&s1sIuzj+FU);Z~tbGar|7kSq1s|YAz470SCz?cJSZ9+os zzcMiE?9DHyd-`8GB@y93n^sZlFT?98jvg`;yn;^d7c@+{Z(=4l!aX&v3o z16wP1!yB0#+`=F!sb#Yb9exCrE!wf+9rXLVXeEhH=Irhn;P$DQbK3a{u2=qE+hRw{ zgegv6gLS<5&D)C$l(dp4kR9FW@I(wbZXS+wmxkXyEb8Bk= zPYJ-c1CX_2Q$&1Y2bmZf2dAfF0tvI&*w}y+0@SF@05NuYdiu5|WnIrB8E-7Fje`#~ zpl1*OQa4-T;drq)qIQ`9VwlXfd21;8aF+)Cy5_7IM1g!z|d@dP32H-4oBGW68 z;#0g~&#ejCScYbwD96gxHMp9qezuLmIJUvA-+l(4K+BATIYGm@gI^ze4sEt^eMB-n zyt=6uF13wi=rniO>v=Z6{lRg+2P0+8+x;6L5s3NVlKgNYJXu`=d*dcXD_xUtZ+Ft; z8OBbMlFCM&T27w+>40s!`xRmC&!T#IHgRtG*C-G1%87Vt4nJ{4SuTCS!`5E-UDXr{ zb8U9``BUi9h*ZJ*by3r})LO<&>cMNR-3X`Yh2PsnD`%v0OqNOk`XD`1pQvO5@F6@9 zLB(OS!yO3bKpVDxr9zgrrm?fzgttF#*DgER)R9M~Qv*uLE84m~q@zS7Jo^FdsNhqH z8&NbffLNE)Ib7w7hrlx(U1y(D-AOOL=MR)E)Ko%KQ88J4r@Z0Ha~Fi&eEr<#XPunh z0NZ^!6r{tszzKO=Iac`E{6 zw_M@q)dR*HjOH54l1xVuNoR-jhy!^s)%-EM%&rcu`#JuiS__;^E~u}gI90?s(!RX+ z?uuJnvj?_udIRNqjx>E&yWrvWGsb80@hq;sQ4b~{!r0%KS%|>HAIUb+LZ++dQRfe4 z8lgA_PloHmx4j0^<=`4oab&{5O}4DKK~q#bP68JILD+KXw1(ah{L47~ZKJ?S)`~0| zMwzC#1xXXPhvSL3pHJ$D6UD2-2Mhp`CaN#N-(cb4kz$7)Z*||DdEXzWm;ivW*Za+> z?(Pqp4y#OF+Xz01vA!>xSC5IGAp&gEzXbpj|JN03nTD~adqUl=Wm@M~PXtgN%bvi3 z62QP~P#bVsLJnOL5m7+?y%EU`Tf@xaqQF5qIs1WCS(lDS9?otK;1TU$Kxq1|o~A=K zTbFxKgUvHN^7}WvG;bC3uInNNs>7PnM6vqP=uCsFUzjohqAb$cZ{j`1X>Nh1kPx%G zw(#nTZS%7Nian79Cyx;LuaO=Uwv&vkb>TnC<`bx*qUq!hGPos?9nIDm*4Fg6+P=?m zpwVXU%$F)kRid}h)(tK#QFAB@H_VP}caD3^Y=^)`DpFzKRHm)TjW2}N*CSmXnPsGg zna7V3m)4G{NEDVfWXEalbC!3noM0eZ3M-*Hgrm~xq!BC9#;Y}t_e&&TQSSF*rP4~p zj}EAZr-m~(?SgSvX1k4Y5i2NHmG+}Ak7O_hNp;E_yCye$mzmN-*GZKbn07++OHcHC-$>b3L5RHMSXJ#Zi9a7}L!-*-MB(VJv%X5bT*uK+VoPy(S?VB`O zrQ#TMDMy=Pzp**d@jmBdt@!9{qzP~7`oWhIYMC7i%i@`1LI4ONBk@Ojr+Ba>wM9xa zo7~p;oNPk5W^=~9gOM(sK+{L|Jv5WBV}`TSFNSu?-=kfs9IPtFKU%uec|#Rw9nOo} z(#Sb9Fsva#aEUIc%p^)?Y2g-Xv4x*5kICs$?OWh*b&V~X4vx&D5aK|wH!cp0&%;)o zg)X-Zfoq?llIrhH&*!hC2M~s4_vOY_hazj_Q}xH zw9D^tPDm?Gk;n5QE7nr{W^o~PDJhtp?#MON<_sy4lQ=%4g0K(8rs1luN$YG!_(m!P(Nvuv&)jL>)Alt-4Tu2%Z< z{L$St%en zs4u6_+RYzH6%#HBgj+OGzEjFG6}*pnm-~BnOOESsNS9s)y7oW00D7fGh}NfuzM2r# zV*vhIOmwok9?yC4IX`$#oJg^CEz={*a+%$=1@ID*CdE^Yms-pBolnzhv&c zH}Aaxyu#Cx7Qx^5@2u6S1uaXlu{k-oI&`r^eQ|}?lCI7d9u1G$C(q%z-S!}z>Nu|c zE{;1}w+<&|(WZ|2w*%X2k5XPdgfhzeHAjSMRs648=peW;d$$hmnBc$sJHQEDO*hW~ z{9iy~Z9g?=zzx|0WrnLiu4A{@;p^(^UUul-2>BiY`G&)h$pB9|Y0S7?hpxKc9t1ed zVx);6E|(~Pfb0@j5fdX3BLM~MJ*y$o4NCY8ZvN>kFh5d?{(NyzB*`>72|g@!1?819 zMQJp-QS>qA_p+5F>Wtc-1R~5GprmnoMjVoLw4d8i%{x+8|M={(CMdRxwo2H=G?Xq` zE_9!h$M*d%zwKv3Jn4K3RF|YblH3F$8)wV=+iuAK`{861!|vsl8c9UR;3e`o+4o|O z`+dcz1P=Bdu^&LAj9B7A+_ zUT0~cnPROEfu-Zn+fgnZc?uBWmC=2x{oy*b0PQqPmok^<$Q^SFCv)vs1tr z&}SltcCgah!c~K`F?`d)`p=BTIbC9v3*Rjs%;=T*Nw6*pZK&L^+Ax+?4ZLQCv-0mu7y}Y9GcfnyK_JK)Q0b|21cITE)oH zAh@Z0KXc@m#&geoc!+X_bv76+>|+|kl={mI#3^7aw#%tpZ2q9b%bvA6TBo~x;+3>Z zV_wvR8X~G;pTGr(hX6Usjtht1PNY58n&0Z_fs0#b2RiQ zv$pg@CaRh+aEdOm%?G2~J%pN>Wr#O7Ws)j`vxqT+3|zRt>J`R_mnkxC2EE#cYey<& z?d&;3zn_ysd460oN@ZQ9Xj`89DgbUJ+;bChWD&c=s*JKG^nx|w6-_GZS9E{@hnsgs zgvOfIM9FTEZLzQ|ER!wel{}FROJb7OD$T2E>TZpl$|tH2K0Svhd`PL?H@)6`jJ18% z$L&Nv8hOH4o_Hv3d|Z;4sO0eFDxDg}vJ!M;o9Foch4Vg8n})&tfY^C1x`(R7L_G~e zR{ja2^$;whQn}N^9DKFxC#z%lt%BbfIGLI2KD+lBv|fQFDs&5FkhGo4h^m&GrMStW z>kKw}a}ss@@cxYH;WCF)7%BRm3LAyHW>|2`r2QhWI94CyWHRvd|fc)<`z<Hz)~fozItrVp#;t_+{6-#YP%{N9gQmZ}n z%}T_zIpIi+gV8JK{a-q3ZIbQu$@#~6c!fF8@N(m*V#Jm~0ldtX;pyoR8+J))d_8Lp z@fUAIH|l7*IT;2W@7ndAR25pCVx08j{EJ^h3zR{BzBj-gMTCk?3)&cFEhnY=cXVKA z&Q&CFgTg}O-9Ig0B^zWVg9ohUm{^2Xrt0cb)AgnwWa;^Zu!)jo z=6iZu{*=M$qVH?cafwe z6hQdjf&#yN>j8jBjQy;!&g4oiGEXZCFDrX7{QZ{=Yw{}wFq*SbOmp3lR$*t5&5Mnj zx<}6#j|@;P^|Ol6$>cuhuJXd4M{i$#Vt@@sr{Ki({V<^f_DS^`|D2b7Vouj5V$2y^ z_L%{6KSKIFo8XT6}O zd1D#xN6C2Ip;v0LWEgdr514ptp{))CuJ~MCU5UAgS1}J zL)Pe(((Kb3J~+9)W|y@tjz+8@EnPL`*gnZR%INd%T5dE5)+0BI}S zRc>J-uR&|LT0NvIM5|HWr!B6_Fa0D3dNDaBo&Wj6gL7-_Ioib8j+ z&u$&$Yk63-0jKNp#kmKU4mh%_J~4O9+e}y-_0^-vFjEZLikB?TwZ5|GM@NU`8p;y8 zWXowEPy$mnyid;ghA(%_C%NC)BMK35ID5A5K4g5%`JlSooTNRg%oeRr2mQai^h97^ zQy-%pnH;F_fCoJ8%G0^rH|9`yoNxGF+XGRcKLH7efE?-m9gw?JF;z#q^}tCT`?r$> zxMO#wtE|ZqpJImsw^-zXu){yE{`ZkxWW>Kbs(+cme}5mK2?YB7*MI&#S}P#@$BOem z-*2Iz{^#RAFV$4?e`pTBSy=erI`p4^dT1#B56ykH`G2@&9okf&FShoL)0UFxQlt=lby$JmH9N(u4&wp!T`~KOT z3jiznPY<@T`|Lpg!~}qpQXOuSK?*!{c)+Km+h>Po1Nhn{|9Syfi&e+}_s&(1)5Ahk zr~jkz|2gCT=TGzh>suBD2Bh;-84xGodrI{>u;ra{8x?)e4}Exx^-4p)jkr8jlAOE} zeLv4F;Jc82JMw&*LD{&XER`lTH4#s>)`@5PCfMPJb86yNgM)4h+oodjI7)Ig;SD>e zC6(P8uskE{tI7mvYyFt~Ws>epj|<*6wmKRr2EL#SdQ*orINMmoks#y=OV2eMW-ju< zcJN_de6Tp`J*Mn`(2eHz=z`YM404WVus&i}4 zeRzjWZ`HYn`6+O3eX`zRxeuqJ%j9Za&+PM+oQ=C%7k7CWq@Da>a^yn8mIkOyd(`NF zr}5X(IT#2K;b?$6bU-RYt0e-CJtlhhqEU`+U(7VJWaI086mU zj_X|qK7Z2MV)yH7N>rw-;p7r+SL->z!d`W;1N}oW*qKfPQz?KgPFwxZvFn-Z1ooZy zeP8$W!z&^>wB3Ch?w&Wo3klzKs!%iV=yXd<`!;DoE!9y=VRDu)rl(b_p=&r46o(i4D4g<6IVnpj zUNqrl1G_R&ko-hkwOUD0Y*7mN?Fsyd)fZ7`L~|=bkX_-99M!(q2{DDzF?ohK`HB$n z@9ll541?3(0(6whjjC*=$2vD2{#wi5X0(H_Gs7RW$^Epc5+sO+QOejOV%X;S6$QTv z?#JY(`?Q82%pq@`*Os}nE;iW*(A2X$%w$ovajgO#)G3d3b2iqpQi4$t|GND2I#`Bg zfg2p$5M^EFl0KVmkG#^FKRL||euxTLZcO6%F~kjJ+aqHMr2imV#w*scQP#Rwa763C zY^#hSKEGrI_}q|lj51+uhhU@9fV`5FZkn1UI49cC1M5+lxEftNrBbc@rRZNFy?ihg zhJnF5+J)A@g5MCCD`v2*a&FfPR2*oB2jb;4ATIkjlj=zpnN8e`;wcAv^Jq6urgb=< zKgl0Ug#xrnLkfSp^~q-tg25Jxuy%^8T<0xDph&QQ7Vwvrz7>j+G&D*OYJZ#QGmG%A zW(XOx-Bv*9&Q}qu>qsoH^52HrE)+tfvzA9MbweUNt$%Q?8au6N)+AcN>eF7)XHxFj zZ=l$-`0a1finVT(xO#Fhss>M9=HHbcKNuI=e?XZofof@)jjBw77B!1oXP=I>sJ0Ip z5lR3?L*BW4TO^~`$Ei(%vr~BEqCoiyWelwZvZqKT<0I`q#QY*;=5H~VbHAhDWm4Jo zeGf*cCENM7O&Wb5xH%BZ^Md}ejU0Q$Q%m~x^8UyjegBGz_j|Uxl<9f=eXGl7WUPAQ z^}~gXJMS!y7@qsVSfLbt_$uq}bQ7L$`aV*xB{HL6xzrmz_B! zkNe9GB585}Vdbvt3fp(FC)`!^AyfAC%i#Nbk2VkOpwFy!ucg%5iF5Y+^Zn4T0 z(bLzd^`6?v8;}Ns1DCeQpE>)?pk28KYKN_8YMPS<9H&0#T(K$|rc$t-7WP%euVfi} zIV89LA<_Faa%fp$D$;C<-}|cX+PxJ0 zT5SwF3d9j5Pp(ikUt14qOXo%;8(P~#4S0SwyW3OG%r>&1@IOE(ScpI=gHG=V#W=f- zSIabXD3=(n%unM?)(UB58}Og$>wm#48ltVusNB8xtZ`7c&@>mh!^@+oxIAzmM|BA! zqsb6$HNW}2_MMzlx`PM=gKqzB55OKoj*><0wSi3jL`00U0e*mtqf|ouP87ec7i! z{yJFgh$<#UdNtu_U6am7&ZI?GQ9=en3CB^o>VgQ(_U1M(r-+0L#zvB!1a~eSYdm4DXA-SKwo*cs!UO7bkF_`M~HjA-<0ba%xpGAdDYc8x~bbaluuAvPvq<|-G!;|wQ|hM zbk{!A+1Wt-m=w=j?&DJKtLw%)-n;4@YXgJ5ar*45c7L%-=^-bEt{S0o(`A+PB};iC z9_O!7&3KCkOiq_L5p?R@XGY{Bq&}j;8)i>&+nmF3qpWYW*-v-GmcrUC?;&sbQl0U8t|$<@4*^`m1CTxUBu+n;$npTxDX4o8iW8b?tuU7}5 zOX_i(Lv!Lnv^BR|ih)3=t9kiJO-|zxw;O^d7F^N_tEp}pS?1U*R!rW7i9?CXEXu{~ zxobXUd(`&T+}fYRjI|XErcCA5d%U71mN0{T^(>H~jYUe1=iNv66y&|(8F1z&sR(Sx zEF4_}Cca)fTf7mc6r8%$nb!%c$>_R4WtrscQ+G6EdawI0oF8qL|uGxGK>R!X3w zfqmg4r$RH%F?3pe)_Z%qnoPg5xaexTqaE(dea1fTzV8W#C3o;OU3KvHE@^JFmUQeujouN+n&jBf+C12rt~>&(5dWkpUzr#;GS*B;iz3{!x;Ck2n@|VnF9t$boHG)vX$oT(F;po|NFs=KQp#xt zx3CxiFkyPpcYqv;m zNVl(7pc2Rpo$=fZp0dunAa(36RjNV#gcEE_T9^LqOXKwXeAp_)_-AWrnhG20S_M#d zU?A7L#M0pqbI*;Z-4WW4ihzSTIn+-maDt2Ss5T|a6ldS%up!N1An2t()ine*CZnR2 zx099~5Kd$CGq+T|FA1L1(v^uH&B`ROk)$MfddP%j;3tfy%E)l24&fneV4E9dWcfk~E zXHWxn%Sc!Jfa$Ra{_u81G1yoKL8sF{H#;Ggh$qork|bGs_!5LoZFY+RJrDSspXGz zr-k_h5j~46%yaqfGt9}pVZEEGWg2E7p>o6#n_D3v{ET0pqy^hr;W`_Z+FaHR5=^hJ)R`ht{UE^HO>k?(_~6xn=VI#(%9PH>J%n zIE0yUT;zL1cWUmrV$plD#(g2<8w!PV>YUj_eaqp0N90|4diFZnBkoW|sy})D5G{Z# z(LTJ3Ftb`8itsiqdu^ASYOCZLvJ5~!jf|-dXGLF4%i{JoH}%uy@{WNoFlP^iTw+4o z>e9kW3tU5(A7kSeTr%ED#IJ6%GEeg`4?US7ShgFhW2UGyksl+zEI?3UYcfBq%A?Ye zYN8yYvF3gJk!@TxK6L5fq%`pzU?!mcbFe>JVJ)~$xw~Z-D44Gx$n@IG9k-iIo>#Ke zfW$I17&z!044qPzk4?InB|`j5u%l|m^-DmB(na(+nr z83Q!V7O0__SIN-u)kXa``&9k6CXi2|Us{|jg0+i=$bg-ajBgzV^3`BisvJs{oX(m< zyHGIyiP``I=7s%Nyr~C!*vum>_Vv2W{p{W+Th{xM`Z!#h@dyQb4Eg2O_Wh%8dvmDo zQypHxT)Ex@w)+-8diwhv-oyH3=tKOyQ$F}v;Q6*F7xc?^?i<1&n~PX+qx+k!ZzA6F zh3Mn)B|B?%_wVfvyq3(aw;SJ`mUR%qMi*Y!=omAOPnck*``Zp3o}mrZz}Y0$l=q)q zuc7-_Ms!A&`d_>oj`Raez7Ip(cphP$jUS%R9(B<8uJKxGP7fdSV70fc$AxV@lN1~x zZ}+XQf(cvZAq{ms1L$%x3M`h_;C)@mHWU&TZjD{8KS8$;*QLr_Ev3rTYL`0}+X39i z1d^%&>xh9uaFx+=bbrs2=s@l3tDeIzPbZDbSQ}rdorkT`^|KFo0CpS#`JNxGBRUJzrrIWe>U z9$ZtN6W21}QSL*+B4y#ck%Jx06k_zKunE zis${@!ltZG?hxbQe{umF!mmx6>`R@?8U+?N1Ts2ovUu90Xr&xUYeOQT_ge+7bG-Yx zYkHJ3L`)6{i&LfkEKrJ+B))o2kSAR{dnu)uv@jbpeErod(6hPJ%kK+B$|SkF_66!F zmF((F&EkIHQ(yggnKPU_znYK*}e|Ac=$JBjdYU>)G9&E3GO`Bzr-im9a*NoRyl(6N3!OVO+yO?2mx1-+J zZUZWoS>J)Q4O0`v4lkvU$=2y}veME4*WOkZI*NKsQ_tLyvOvXw!G@drp`T2nv>wQ& zSHsHJmu!$+C8D!ujb&707`41Sc2dz&LuG75)W6~Fd7{SCG;n$naT% z9;_|v;3`;r{!U+{wO5X+riFdrVGr7UK-Tm9E79wzn{=|}pCjWOFQ2SCK?J7RKafAO z-dC&j;noM@{mD69G0MN0un8}R%_6b3N!3ku-lewjJiGnA=AaKZ| zQ;0??G2#!u2=#lXddP{gSA4eI-{Zkw!h4Ny~?+QcB%YEn@Fl#3I^Re zCQFReJzrKak-fXqK6nY9JrCk%`5Z(9!2`KC4~?0S&fR!Gdg3fkZZs3fK&S)c+Ay}6 zL+@k76c~5}2w}gN&5xWH4KF$sOd!Hml_o4n6+C?l3e-6<2r=S}HRfWEh+v!MXWtT0 zZZ>4UWo#>D{!tdZ>sNsJAP5K10SV1EZ%Oj=2zI~%2L-N)Pi*O2xUZll6JOM?C=5aR z6XhqQHq3c`Y`8PZqjhaci?|d>wjWvON7TyLPU!u`q+!QA?a%!m zT9pAR9L2JglGaE$exlrJ1xW| z7yNWQ-hKV0grj45U2FwKMG*l?0bV9y8#%lob?%>$P?DA9DnMEv%dAn5D2&kVc)D=RA)1pSZbLk#DrLkx%g_T!!dz>5dK=&8`@ zz#`~JAN(vW+IabEhFG;ySHPY#n7yk&UL)r0Ku(llEOoNrJ3aB{=J?&|bKbTAt-4w? zaM{1E9af!0{P!naXZ?uTvM(sCxO)wYTqV@dm{=IZGO|WTOD^=UZvNd@;LY&cf4=#5_euX>PZ%-9SC{JzdJ>;kLQ8}eEbt2H z$&t~};KCR^cx+Y>-k)ca`ccF!P3Y%kl9uyJG3yToe#B!kh1ep!op6c$f0K(~|;gg*Jc@F_*4}X;;*eIx3DF?xDfqbMv*j>xvsnBP8t6EZBIi<>d*k zB7-eiuRha%Chlhh^iGaQWBU~Z-6^|tj&5|qsQa4Jq6qx^HQ&WUBazX+pUuCzcS(|B zw1zTWhrWFexUnoZKK1HnK3e##?s-Bj_aGIdfyng-}hbJ z`qv6mX>02BDo&%p2%0t^6& z`8vd454w)Q$KTQx_{tA*Q~?e3zeM8Wf6nW2M7V!b23QSojNDbfLO=*~tJ{N+hX7o8 z0Q{LW#$EJl5Wf^?C<_$=%yhs$>YqIboBAOnEATCQOk7~ z+%{c9b+!19$EO|p9%3goWN)ciB9P3bIJ~m?WVhLN`IlFz9OUiiMk zBa3RVuPlj+uUmCKi@wKe9>CY^@)Hr+;~d+(N_YASGgRB~37`=5v$p`b>gHx<{QnSb z2KaA-0%au`$*I}{6|(~HXUhX;9iqP{__z0zU!HiH6nUKd4klgT&o%~f09-@L+_8pZ zy6+YEJ#^4%0Fdvt85!AqHsQ|UUTp2Y7~R6{Ev$2HQA?L!I#p5PRz=*ArAeYC=Vba_ zT%>Jh;)1eu`^7Ndbu6E|pEX&c*4=HU`n()a)@5&>;eW@&eIa{){ZL0YjUr=Wa*6`| z@pA+jz59xk1KGxo9P1PS6XQ|Ux^HrT!hAg*$hTj7&okK5^F_<^6|xA0#i532Q`Oi7-;2?tXRCF1_K7FBGuW2*?u|N?}BqF|zT2Q8QG{gz6#>7tzWhGjuO$H|>IyGeLv&+bT zHTnJL#IH4&x>DADM~(|s@eetLq&%}Jd|RTC8o!ipb!*U-3g43m|0*J zf9k`Ze-7|7VinYPk1&Fj=M|)78drbJ2a`En9x}#Gl~-*|m0Fh|9f)q9U6JOs*~-%U zV^kI~Dn{Fu_VlTp=reLqA>xl@4v&~KvNi`q=Y19>>FR>ZZr-bfy0n&>sDpmOh^LsD zl}+iWueqw)+S)Idc<&6cyieo?Yf4H=7Z-g9OrCdE|5cg&hdK%N^esg}DcV!l_k8Yw zFrD|d&HCazUzt|sMyq%{M<$#7BT#H3%{Q)L-*-2_weC)JO>LyXr~96x9ST~&Y11V8 zb|hR*PY8B+0vJ=qD+;PD$vU#VjUaV)r!ORmq_qWR`j4U-&x|)qx$V7$c-e*z@ZR~j z9q5OqBz0>u?!!gRxxIrD3nN?HY*2?%#|^fYuAN-&1wR>Hxij*Rx$g||?JR)zy|e># zqh_FvG~-+JxTLJ)%sz!3p6^#8FDYo443r3tkkMWIZr1Jb0aSD#vf+9N!z=Ty_?eH z$}{2>KszyFCsXAyBgNAPL^aGBnp(JvjD6}U*KW;p`93n!zv-gks@n7c$mC4mW@lN* z7KO3N9AlD8Z!GxZ=g+eqvZpvGgh_7`QcwUKGA@SX-wq1YXm2>P4FUG}e~O-MGc{RF z`CtvfT1ZkY6I416?@kE$mTS=tO-SR z=d1Q#eW`;W{;O!hVr8iWh0G&*fS1gK9ezXb=>8g`qCKMJhW`5KV7*-LK#`Y3Yec`y zemDC?%b;>aqw{EA~LFn@Isd%IuPN-)^l#$*l< z0V%7B`}kl_oY>Qny>tvsd0Yf2uuK>7epeO=^qGB`?R84w#*K#Y4ywY>*u&6{iz;&b zF}c2H=Q*ho=!Z4bnzK5nUw6AR0j=nBxG9Tjj@n%`Ymf=XG;iuYMaJ=QE^ zTv1*E;e%dT4a)=S?6IpzPH2Bcmt?CdPE0n=wNykHh2Y!&v8Wss74>&z*DtE9t4jo7 zfP<49)puqoGf%=kca*pw_5MU>D56M&fL-qEamsbm+f@ zCj?`YdumFwK;>Wv1%U{{6RN;cN{Uf4hx; znVe{(->~BV<>pe3r$Rr5hlfAd*yJ}hrVbCwjfee{Uv}k*2cTV0 zl?@COF<=q9+4G?d&Y&F*p|`h54tk@ets)krL^<(ZiBG&4|94KjVI0|o;Nap?`V|%z zhhJT}{qc+ULlvC^?lU0DwLCLvuRW5{mlXR`ocspQx=#;ZU><_rAz_!w%fi1r!In0a zhTl6TqKv#fgndb){uO5%VSk^D8t*JkrXZ?>o{eg8R`f(B8yzo!m8GZ+@VT`jgZXb# zLAfQIEg2Al*-0K_^5>1JXvy&F53+GNJg^w}l_Arcul$lPW|lb(q%|1ryj@34iguai zO4=+Z2mR6bq4Iz|R>*-&H}?8W8%gb>p4UX@c5b%C&GWH$_Kv79b|S7p@AB#$|Jd`r z2*iD{<;J0px%q*&qoy+C`N{u*hA#IXl)%VIqu%`7*_yh32%F$jzrj$}RLF}Sf`j|U z!B|OzjCLH&@${3lr<(_N@ApO>7162F4CV(6AC5M+&N0c9-j*6OT359sMC~d-SAVQb zdwX8Qr&=Cv>x&Lf5p_%)AzRBz!4Ugga%6jK!se;G^*V=pSm3KP5_3eHq%Ud^?h|J_ zG6BT?CiY_CO0+`&u=kPs&GO|m0X7G8{kpWf2vAri;T-9!OUKpftv?z6=l~`xk{qK2 z*Vl%~hwqkmGVBFy=@C4@{`}(73M#d9c*;^*Mf0IiMzrIl{**l6j=l?!sA^S4^I8e0 zquQ4V1>9f~?rk{I9aI8O^rhOEifiIsw|hoU{fhVzN|WD>qn>C{;AU$wa)1fZ zDBrvznw?y_CAS@dTXNa0_<>5iF~&ky0R)Dt+k#KxViHQXpu0%Hkka>f9>IZBgLWlIv1YbZz zBefm&y{wR5->{2vjz!ZPGpxdu=vww`m?ZvbmXmD7CQJPNs z_*lX7aGD*Fq@fP;vbl?Q#p6jh+qVpR_*(oCT^m@D)UE%J$;^Uausa!+{lyc|HFSm5 z;eHLWiX4$0?+i-o;yCu-6K=>kq`RC!a&q^}U^=*g#rmOO@NG&PuFdgTzDq zXZt3rRin=9j8^ds+W(vMdBA^btbsTLy2bTbm?^b9x1%UAasxK2^{UQ$*N7hh{rgHs z^k*U27rnnL-i{x&7FP}@!By;WvK;PX$>oN?43}4DRjH=LrlDa_g!$+Q`W*gP`Ld$; z-QW2Aq#7m9NDOyQ5vWxV>P%Z-y>XH%=XEXH+X6v)TS+@S)IB z_e97`S@HpGBrJ*(>;Vjjk#V?*@TYFmAOJR|^g|RF5EE|Vr}(zCI&=6u4w~5GoEJQa zj5x=DW>+vVAzUh;QSV+%Oq285aQetW^si^2=K9MGu*JSiD_@sMZS}o!Pcd6+sO1~V8 zb?&f{^7n>Cnlle?#Os!zhenF_u5GfJ3Jl|FHJ`-(&YL+-TR?5kOogN z-}!`>wAr1AdZg(J;WK{+L37r(xeeRNXk?1&Pz-ga2OZh-Deklbx*3RJwm`Am&$ZK{5o)3 zSLvId*j53SB(mGw`N<_4>O(7)VX$leybrg4Q0r4~oTBHBz-P$)iGA&lzzYBP#Kfa1 z0UcLur8r{&@l^;F5+L?Ej2;6BeR+})dhE?aJ@^B zY%>J8Pa9-=yJ7L^`(mb!$#5<#4o;&B4^2K9ajIZX2tT~IJ>Ye?b>(}B3B%%jbEcHD ztHs7VVUBM(-zALea5<|HA09UMc+oC+sev5qh$rG%9euLo0-R+X#O{5WC2hFVF|hiX zDe5;7-xw*ppBoXpoD*@*<=WW&N;W^FChc}!2|F;7dj2(;OD6PX&c|^ow$mGv>**!z zvq`MXa)Y5yOodpquY^zjOsbz%scQ)YThip!*Vl8h6Jp~kZv0y87&A+*I?M69c-;&Q zM97r>FT$Aq9%vhYTc2!baQNTM+-t!aV}eG)-}`V2xoui#czEM~QS{tsjtts#DPe-<#fQs3ILx=Nqw`Nd`TLOIiKS1Hm5+r;Glc#4LuRpmSE z5wAWmE7{x9lKpcqR&Aa{*V*x{%%@p_o5^5Ze2QGZV>q9q{>5UJ4NXdIp(v)xjJ{L_ ztTRhfctyyR?{lI8It?N;dW1#@gWLbOn3xX*4Q7raC}|>oX!kcx5DSM;#IsToz+&L2 zywd_vO8D?YL#W?8VxRZ|i^|>DCvl#h+gF9GdWzbw&-VYUXZ+V`XOg%nN}K6b3gXcj z{|o}e{=bhZchxx4sAEieCwd)0_@S9>6u;d&BE@)-DW|NxJ=+F}Bk}a)q^PVc(yn~? z@6Q(wp+S7b$jG?2zyGDEC=3vD%Lok=U0hu`xVa^uIF!5cSifne5R6?lPfwu5<<-^H z{Jbob^L(H#z#pxQAQXtNup(gL;NX7D&Z;RZLxZJ%1a!T(IH44Oz+%382Q4NTnu%gK zXkfr*D*cPvq_CoIF<`?Xzh_JSC&iQhD`Hw<`3Nz_vN8raXn_6OMEHnO6IeuSFz&E+ zgfLZUk7Pz{=ZlfY;2#3U%alY$c9s$L35GEPBMeUeUJC#jp?MuyTwIV5b{ty=J7CS_`Mxo^!OG!`p_L2g7oA`BjX(DXL{HKSDtOXQx-k!eV*^1H6B zU#e2~3wwtX=PG|3VG2Etaf7e>-MI_7`49o^zb#^vnO|C1ZN5pe>y<=ot{jyk+jTTdCkoqJL1D*Rh2o@)HQ9l?hbgX&2K4j+4Q^mtTj1WJ3>~b>Gp_+ zA!VxCjR_}@GqVJiLUL-lB4h=o-Q2t@UR2v83fxyHr?Lti=cIo33u)Z83z7(AMw9SQ zkJt7^Rvv7vN*53n!pbMKPXA5Hg=@P4PmHr|mkFV3XotaX9)qTwP!8wM`pr8JpXl@v zR-#S(($oY#OE7KAXVh+Uqnb5r32^y_@H*(#j=qx+Q5byl^L*pp_$ftX(`n0MlbtMP ztATH{zVsq~L_(61LGMo@*7XBE6vvZQ5ozb%T}4Mv1~nq9eOcIl5=u;tll2v2(Wvp* zaNj$ekcS`u<@9v#v6?n6YxYpW&(vz)RR%9b7feW0LfuTQk)+Uo;UiLPdwpuW-l#x& z`krj6<18Gs8i>r2B?)Rjm9uN!5F18@!9UupNF`lMJ5q?=*>M^;S&>kmHMlFo*ic~E6`)a|7zDlswC=&|!r6Yx`i;BvJs(UQmPhwpu3vHzL<&Hj$xPGxh@!3=YL z6FK?xY;T#8FcgVFXDI%&$GNIqyC+OtsTxYdv59vf4*pIe5<@w7NMZGXA7Uks7#u#1 zt?Rkfg=a}R>Y$;l)pu^~m*on@Xo!)X>)dwC_CCUBC-|V%5tvcg9FXpURSR>tdX6(; z2b3|S{Y@wkDLNv>`o)z8dn2OKbaZ9jO4#dS3-w^0gj2!mTtfy$9)zx+E}z$KH+yh? zD4Iy>qm~4qW3aogGeSp{>|?gZ(s;^N1?2Fa^?fg;{MiL#3i~JMD(B z)g8Qx;c0w;=c};#9Dq~s6C2#~P^bEvik zwRSkNBu)t(F0^EgKMA9?bwpH1OKCav*)BQu5ig*(^>eY=>?r&5@cCXZ>T#FExhzw5 zIL_LRD!896M6uOWl|@EbC{Ux^NT{kiHy`nJ^i0*Vy4iTeq^>L}8TWg2_+CpDCVk9Q z*AouBYk_Sw^p4<*b*no}w|83ZbEaq1?bTmx3F?knZo62jvk?&PjkN9-_Wb1QR7U0vo_?!Ku! zc%aRSU6^rMg|+}5F4asgTQWMG)}a9*V!?N!J=-JN2KSU1F(l>=N3RsgmgF~;DGWQ` zqot`;)to?gn*#y0Cm387dR?VFkPr6hebK{4*^e0r9+z6^`Bz+e{DG*KKCc)GnGJ>? zTA0}$`;@V?n?Xez@FM*7y;)zENYW?}5{EW9c8ckb0&w zvB$x~AserUXAA~1)j2HShF-}Z@xv2I@ht;2y#vMSo?D-n>i z1#JbZ>UePysy3>{zV+#4J+%osWf@m(;C{S@8*l7+Wq231^qxGkxtKO;QDA%^+Je#* zCd}<5N(I7rT$>kVNtea^qz5on@0Nuy#eLGbFy2d5J)zenbgl87jAVVdWplwId7VWDP$hjd45p<2(Dvrhmqbo!B4!Cm>D zmTqMDMxeJ|L#Pk;i@4A%4v|kBxUr7|8djKu-+F8SXk$DyM+G+M5qcj$7;*dj!{Q=O zm>K96p3sh%VD_>Y=T~AuK76{QO{vliq~=N2;quyUw~YZ{Fh`F{HoB^oj;w+iX1f9l zijyLUcpFH!({5}`PD=~FfxiLfaD;k`SiAh5du-4423aj}>ej5LT^G$u=j~?9K07%# zyLj##apJ$5i4G^R{yuu!qkF*Ql9I&RqouD)ZUe5PWhLhQaGDMWW3%k-jD4+?@JeJR z_TC6UVXehLq5AD%Etg3e5VO3KVr_lQSEd~`VswX{_t?R{1hk`R@}7gOSU;I9>CHa3 z#y*h>f1x(RwxZ8a9@O6d<@Bpa^SkdMKn2H9@rOrxS1uB$GDr3$rV1<#4-|^?%*tH7 z(6cJuhN$8ZZZun4n*R;7^ycXGDoUf$?6w^1E_6JOA$zzjo%=*ay(a%lD719mc*EK8 z?A_T{iiAY|Ikz~De4OjYs@&{bvb=q-6eY$r{g*V;yUJ-~pYu91P~uoCzL;Bayo>{- zvYW{Y&0q264qNKgI$^>s3wUl6`HvVr{d_i6lTX;6E~97or|M5gh)#Us8t}A)xS9Vj zkh689P{TFsKI)quoCO)lsYg2Ju9LdiEq9+a_!$dJ4ESYxIf}UJd%tlnpsU{fCVQ7> zT*#s|eD1k@=y&ROMPmCU@Aw6ioz?_NBTjXdCscjOYjH8WpD<|!aJ}oN!&Vk{MIy4p zFZA+f`ijhIZ0U(GA>Oy^)#9PSBS@F8l3AR^onwL(uz zmcnJfuS(e1+1HUZpyjcjyhNB>eQhej-n? zZjGBGjOYY`pTT{a;+<4f)7H2^3BM3%`LF=Td=cR36Ia+vidL!9-(tTaYPDztk4#(Z zGsY=J*yyw&fpCUHUk)B!IIyGndJnrF3Rj+;VzIl4uuquQg>KiGQrVt3?{|tKy3LIH z&cago`^b0p-${FxoCIA=c8lFFo>b7d4N#bu+fk}2AWtrAD@;GnrG_rT$sSc^)6IE8 zLhtkm540H#i`wJ&4cV-KLM`ieCwzpYaaj|p8}Vh3dIngyXTJvwPtJNJBtA+q8+u|Q z?~dhup&THJeF`aR$T6laHGYEQiV#3MoB6_(sm|w5gX+k1e??O0&HXt)H?X;G7*Ovn z(pRU+#j;nxra2l!6IG1F?|BQ=|4zl9*!2e<;9N8Lt7CVrcolO5CG~hR%;a%7EN0%l z4pTh+(W6wo>q6FZ3;Xe>ns$>1DfY&+;Nwz;i^@SJMCowft%pv3#Aj`vBCG~ z14<2QJom3Om@@{TNSbsyVGN_L)z@RbE*!M8tNlaSaUVxgxvYOt%Q;zi90%Te);R_& zIq*rN@v79lry0RxF}(L7oGaKp@G+25ANMWTexc~vx%x1W{HGM<^BJLv-5hUObyd@81VNondID`5)tbiizQ5 zXMd}pfVxRD)D*SP|3r(PJ*Q~IuOP}vJ#0ZhLO_A{yFG&zN5*Mx%X>Yg!Vm}~e^+fG zH->6XGYBF_406bffPVh-HIVl2Z=k33iA1blBpl0k z9l2mc!4mA9P~m^YgcZg3{T1TQ7v&%M&YZO^D2fd%^Gf<=-tlC4sojkzxJ}r?m=QQn z;K7nOi$j{<(VVEYv{T_ww9U!bms@ni9&BWwX?6UeHi1Rzu4R9ZfZP~@H4P%K0_22v z1f(=5zWVa-d+NTA>=XI;5}Y}>H73i9KcT(LQpn!-DlD+)-)dt0%_i~YaQ66seN(icKWb;o zHtQAN7S)+!;#A5wRG-)T4>CbXwo=6W!~kud>^eEGCLvlH_1+(E`RRqqiWWEfTkGi; z(s1o4p~m@W4a$c8`0S@Qk-+PPSWX5uxUinyYU#PlEur!`k8n)s*IW}VnOvn#)gE0k zcx7a58K|zlnS;H-td^Dr`V?@bI|y4o{zxAoX9b|#Tv-?2vt+i#5K;~~U% zXr$d)BPtrkXw$f?M(atGZ7C5w;ZwB`f)t^eKvJfxK1~C0X!leNBZ|CQEOyFA8~+Uo z()pzqie%V3tj3dVj`>Z@#I!fM?hdnAu}&U<$nBw>BxlyoEu3W1%x@RRh1fe1W?Zl9 zZ1jan@)!Y|4|N1N$pJ6xXUdKGB2nwnKH^`;PdM9^1o{t(OI3yml^y2`Tap${5R2#b zkmK&H-4>Bq9L2x~sSy2>(O}nUXrFQXu56mZa+cZ>QYvXl?^koqcUkU{PSKsrIG=-} z(&B-Gg*+pkF57B0_FUk)8=t~dkD)AI)?EdK0=H!~!3H0mR`_Xv$$oRRJQuJ(IcX@m zjxD!^eY{So(o`Bsv?|O0x$;6kS2hb@NZb8aIT=a{|z)O#f!BL%!15hBa#uKy{$5 zjMSuWVGK-Dh6r#Ch7e2GoaXF!Oh1~Al=ohAAU1nuMA8H>bxDlw zKtuRjOZ#%~Em8L)Hz}Gjw;AUfBCYsI}6dWGs5j-Rhy=s&%%mo1LHTh?`;v*7gmlh*je`JKj`(3euecbE4BQ-6;Ub*uqShvH_0>2skN-sh?IwK+QO=Z#bDY>J8 zQgnkjOFjG!yO|T#qPvBvPNv*FB$f-rJK#bxC)&vuzcghDA`nWSI)^6hX}OjT*CZW7 z@8Ki*jfpWMB_jHn1&1)b4$!}^9?)=g23c+wY~iJ-s(_`+IU?GUFc@Sp2^$-*))6zh z`Uw|aIR}`&=9)UAE#!La!Q)87OGD!$t;-;`=MXcQd2(Q2%I%Fnk`c zdrdcPx|3gngEPLhIUBoq*6J@!f`N`&ZM*3Vh}>};hnp^Gx8;FRc`DSJeR;+0QqyGh z5*q;lKkNI1iK}?rfFVKBXlqc}c(zP=RaD?QCOizx#hvphzlU#OqKZ1%##F&iYS=q` zM3l`FqR&up%06z#bofG}_u$NL-`G8dWUfi=I`?^hnY79MG&L|UHevI>xkblU7mm>8 zVq{>*?0%Dz)qN*mzrC)1^e>Vivef{PUhf)?evKM&OoQ*!#H~3vYn>D1j6j~qkxHMY zKB!fSJKgo3%V`c^Qmp!B$xqKftT11j;|xg*^Q>!=d;=mAiM;#Txs9&$6EmFPhLP=y zl^br{x4nouhKs8jGlUY=7?yIelwEiJm8w`3*m#}SZFJogT zleq>7SH|t_{gZ-)L`Rd5^qxvMRRg|s7~R3ML>A#~SnbiFG)46%Z1Q|C0jC@rL^Bk; zODc3IcEcc&VaXo<@oKKso2@EI*bfaG`+lz79UJ|p4P}YP>r?|*;2V>rYUF?GbU;+% zx76bvTOgpDbBX%C)uBs4TwZHA zN;ujJny^^+qarAo2;5C#mfE$=|3U{^*ejuqzL@?wF9nG0d_7g~i>g}~ZE;*u zq%!Tr=FLy}z|w=@_T6&S;t)3D61@=fVEY&^j`6G1LRtX^Z-PHIiW10uD-@l@i=1Qhf+M}b zzQ}QfdWW?gWnjklqE9tV&>go#e?qO(nyxA-NvDfo7l*@d6mn=sNL}IaS`z8erVN?# zUn~Tbs-gD~&IVf;-YU~YUVB+?=wb^#-zR$M5W2mKti>X!<$ESvu}+~HSP zC?MmAQscax~ZLksdeJ_$i#w}ir zsVXi!TeS(djc@3}_YXT0EPd85&R)14%uH=$NL+5cLW!xiT)v}6w-<~)wL-E?!j0|! zRki-sI*uw67HTA` zF$c7v;6HBU^ce5;N9Z9_Q&WdptxVWY6GGi&4bAKTN<|0iw@c+k1#UuUFx4g@Vic5a z$fyEKU}8)<9AZ%qTXTMO$ix@MFF{4Kv+Slf^l8FGWmRdRp?<$R+|a*6{0X6A6Z4UP z6b-_8#!fZK&W1s`%-pw#Y~8njacrcQn+?8u^*c#) zZHsMqu`N|w9tfs%stqJ_`KK&(FMbpIkN+LQ=Qc>T(TLE3xuRLgGVxxl6tCy4hpqk> z`c9EKadJXw59wwScrvDuKaTuKh!uF;zbCW}=(yN#grDC^qE#4pnr*rW9TDrRZ3rKY z-D1F}3A1%ad_?=QCRCv>^p1;Ce~cK?82{h5Y*0GVHA}axns6xPp+p$dd)vR$Qy(jV zu~IKF(&f6HHae~-;i2n_koRyO!`l+{*CcrmY5%PbPG@fP&^VWIyS)<@qtCF2*2m4ru%!PS&OPA2=-rKw9QYq!G~Mh^(zxQq z^vB!X)^;Zg_sIHvU!_jaB;bTp#mnjn>Nhg21G>F+U8<9d$XYXW?c0+DPW+?qRuI2W z9Eb5v5^VguLqGr4!-H}@6Qa|2>Y%`P68-W@s4eiyNCaW#XkB^WFSoD_9sFN2;%2#C zWB(lY_=ubGb1ZT7ku`P}-h=LvM6WBB%s2CkL$rG=``AJDdK#S-Vn60ek1^Gu_XbCb zNKFnqe8q&wH?rpw%$8U0)8XN(zSvT&-gnixKxXQNuXC0zY+z1MV0ySYLKmA)X0`>L ztI_{rM+9cYSN1xjKnA;&(zUk&M~|Za(Ff}=xmDRDsCmE>1o^bI-L*EjWqeP&jY(3C z=R5eP)n$E z$Am0fwc96{=f0p(Hw|CN{lga9SDdk1{MEZ-X#(u|-Z6Ckn>kl5*)=$~aNpRKVTIMs zM6aSQJbzCzm=TS-OZSO7n-$}w3inzgPc9(Y6E_)4$@+j{`$tg%3;T(zJ4-fv;`wtwek1Oq?+IMgrmjw~bhIiI(>gFoK zCxScPDQ6`oA|RhBO23D(D&;JT)4?N=HEH+Co?1|c&!P@1{Uv0S#c;cSJ8VcXr`u9} zyEh0%&|yULW+>+xOW-K6{H{^n(sv7UrrbZ9ZjlyJ=UrvQ-QB8g8X|PxQZrnt?KSx> z0T{dfS+=w&EZJX6UeYp&%vMkSm#?+<&e%qJ)oQPWs;6jX*PX>}e=Molokhps)}Y<5 zS7yuxT9Me*n^}U;Z7h&F^$gi`_TVc2bwazgU^m0R*8((~2poy?on7R!K?bugw^$d& z#ycOuk(Z6Iziip&Ap4{rPGgToey)!=*-!gc3FB#9-*ZF;etqF0sB>Vp zbU*6F8-2fg{=D&4lmidv4J3oiKhP}0R`IdB!pB_}>)G>U6ZSy!yXqsd9k3PWFQ(gO zRt$2q@XutTYuY;`rR=wM+!z_99NX=Rmfo`x7_FxI#nVNvwD4>t~fSVyKZbeA3~KaLuwf61KPIU>^X$ zgeP2-jg;8;?I1N-n7vtbe%*AC@@(>edVM}JiEz`Om?2wFYLuMTv-@PmK{Q*j8MCdv z^jjT7%~Kbh%M=RDDLxX5zy8D3dN zRz6eghGef3iL|8xJ3B%~Z*#U&&5*U*4;ct^7DSceWtY$2_;~g?nM3i+mg;?ZNBrD{ zo*~}|b-bcJh_s~)lL_|xTuC0iT_HE@Jkz}>zZdfNap6g<63WoF@DIwcF#NW&-MO)| zYYix{5E8Kj_Y3*AMYwi8yvYd2M(y1878p2xwiC>N@xS90(Qk~4y*T;YPuf#u-x;YF z;?AYSFJGBC{$%(NqGTvM$Ut_>8YRLVSf9z>j+zA<)KkxTP;{5-?(lRa0y%y3{{mq| zbg|#r@uqs`zcZo-;K4)4D_(1I7zWxgtd%cqwxkR5N&DUg@?>{*<{ z<##Fau0eQ^(iSlh_?4k@Y2(m;pyt7H_#eHv6ItSeKZOvDk9yvZ4q<3dG+p|`4@kSOy@aghtn_i zXGk7Tl0tJ_M3lFd*Ogv&%ZV57wofRLeRS={fr#mLll@f9&I{n|L#A#@(qCrYJD^V% zb6|CP_-Pq6`cx+YUo9DmsMZw?2h^+kBxO&7ZJWWDGQ0$-e9w5_1kt=!9XVhy+2@hp z_t_ym)@zQMH3JOhxC!S7`ZH@>oBN%U`F{pNu|IpuDaYG35=Vz)AzN(EBBX*Cf7*5^ zw>~*?v2{jlr{0$|KU4VmKP$~=3&uEm2Qw225w;HR#7DX*rK)(2$`M~ z^)tY&OqhIVGYv--(H?qhDUT5sYo> zEklqT@a81X+nGNYn-|-S?(~enbM6cIL{&>zFpCg3*@tc!f&oms*)*IT&wUp8wWT0q zy%oiTDZsE9a8Oa!w5cdjrY-R?!#%))^P?mj{uf@}3cr9)M`*7Q1dKo-F*`6gpVCE3 zq0q5yue=c67gwcybI26R=u!4Vg6p7{HoeDk0Oz$U`yt;cO$&zykNY^RFvmW5tjMKz0}*obUBZ4ZK=flQ zN|kIepWO2bhPN|sqUjsw@j33p@8q#7o?MOs-GNu6j6llZ(fX47g8T%L9tAr~B-t@i zz2G&bVJZO&AQ2j?@$rSJM#ZYQqZH$I8p97jmg#Lmppk)PzBQAS|C@|WyYtF4$xuuf zPFbbD^0(dIAw#n7rA?@}lQ_#-(trK3DlIE3sjMu7Zr}OypCYx)thdb2w!e+^S%Nug z{kvGPXE9L{lxsviTa#!p&T;Yl+1)O5XCB1=aiajTgc@fx2ljb0IlWEDRU5yY=`f=J zGP2jzrC(l3y*k`S`JR&?p)*V$bbp8BQhz!p6MzjQwxg^e{CZq@3Hd+*jNFO5wsOd^?mXw zeSKfONn1t7a?+z$KiD;+FtU%->rtS5O@UUB>b%_f6gQ34I`_3Q4Wr6*ZCr+$y&@?} z81jZSM|ZHRkiP%B-|^DX`ckZ4ekdEN)k^-l*#S6gfHPN!a~ zSQd$zKBR_iXPePgB5y}8{`6V9y`h;T<@+uD2d})SMn>+raa}X7X|Ug&Dy|Y+p9ukY z-H%t$k*%zehl>~<1py9w$IK_hH2+6UJw6QaQQ4Mg;fc z_wQ1#*BBq_dXrsOt&Ay;%CG;B)bpA_QO$%eG{f9qm^i;DGu53YcXTIIQ!*S*tCM#W zu<_Epo-~S~g?!Ij>aapL)u>*(=o@7o`34)R<96O#N;kg@5R=`e(R0!u^yOglL_$L= zH|(NPvNOT>i@(;Fi}&6M>)Zl`z4~pz@TPu;w8tWR2vPQsicli3`vS=*08;lb;)yLw zvS)Jc-qG(Vk~xqD^rUXiG>sXi-EiIn4X;z}W=|;jxiA#XeAD!AjZtGN#D2PSZ*m!0 z#+hhOvG*+^)|`O%gM1E-)#pou{dPgE27V=SxAW1)+WquWcUF1ZD;??nIZ+)UB_WL` zGK3m0r7R%Uf1lCsrWulE0a_#KyeDW*<6xCXI?ZF#lO68Eb6)u&Y`lX%Jc6XmYCvxnENzHzU+3{1Wonk6z!gExb_6n)r9h^<8vr#d;`De& zW^hY9D)Oc)w$gO{T474QtZBWUOeK4!2V&+)?UhJBJVHCU|1CLFPAdE3c7&?%%-D~# zX#xXJ5h7X_cHWkZmbU~^cqtpOOU;|~{#UEb$E&k(%7Pj8K9*=E>8S~4b?H|iJnsVS0eWI`7&iL!-36rB?5|frV zH&`8LpcO75xJI_vwyoE+guFuxlF{iT-_rF(BlpRvI*DUvUp5WVk5F;+4Ln4mJ>Bn* z37*#Yn@uRpLe@Rui~MhKJJj~f#Z`Fskre-QkUV19Yinz5o96=DA6ZDY~s4 z6V|$;p2y@ZEzIZcKU7Fq24v|*JUFo zxk65NIYuJV8=o%7oqJz0a#pI8oZ^4z|OZkK<=A@aWM(H|*3K|9Bq9-5P!g`5#Q zmRl*BiasG;Ki;_?lD^VB2UOm7-(ED37JAsw+@IXIH+bMV^hujf)oyFM@mwZma%JEC zoW`GbBI@*bAI??PkuY7FP^{k=dc-20ACuN&nM|s&e9JV0V1C|wfQ!BGHB`ZM8DADS zDqw=mcqp)kJ{>ws86S)f23?`Z8J=47I!3|nVj5Y zCQVN38D>TJ&Q&F5+#V<_>mR{gH_}W!`dOB3g=rIuU3&b)91eQ7_mnbqKiw?flrYwM zOHG=dN_wCrd0T@ojA%zo2|<|A>DOUm+$~zFwhF=rqbFNyWc@D0*%S*WOIiLu69{ck zD7fyE>%{o@t|4Pb&N;`2cH6^M?>O=*nvapaD79SJ9Get6=!dy8YD4t^dj&6~I@;v# zt=FWap2m1DiHKX#*BNdpHwTjIJXbN?m}Qo)cm%J0Kv4QXItU7t&A)IMr*iUY?%6pE zhIUS4bRNv==vgOBuV?Jq-l)lvSenm%v6G73hT{#cK=617)UmQgTSu4dniRFNL`s}G zci`;BoLnI!)qA4WH+o-{n5Kr8m?H4njcj8G;iSO%SrH00y3Lk7)W?A2fqf33oEReio&Z^2S4&G2p}HJQXlIT@l2EqJnlN z1D=5|Lx(s*jouSF=5_&i{ey+NXDY@(8R~LR)OgrVs8$*$w1)zp@=);dXtTNt@J0uO zen?jNgV#G)Ti`i|*ik@F!|)lJoHC4=LqBITrQM2yV*)XQdLDnc-=&!n;wvK1!WUW^ zz2Jo+`=U*+(J#{e@Q#x{zGIpA8R`M#+gAIprvg|?NlDE$Ia291*cff}Mu7BO&}u?d ziBtZ{esv>6^vSZDuS@Fb8BI>|yiJH)qA^|~tu>4hEdYN8)*n`P#K z{jc66p_Q*$_|M7AC4&+MD3`bfl~2fFp6DgFH)ZFmBf0MW5!!H_=BN@ zBx#V98_eStKuRZH`QCfFU!=@k8gg{W6Wt3^g^Yj&YXV}%bi@BHjCHTtZuTpn2u}ud z4MBm7{y3-7sZ0sWN2nd6SaC7)|ZR!}HX_Pv94 z;5xS?mNj15Vw5Wve{h4~g8q}f61{1aq{M`>5jy}#8IPD^cBTl`)5`R{t4zdbCX2k% zy_=3u0ASL;605N5!TlT5`1>cpL7a+0jY+fS@6$Q>(a3hf9~h8_dOkaQhild#%-^gn zdyE#0G829xKv#uqoiv}6Xyi61Omkg3SUVueo5z|c*2uaPCnMN-hg5um=^e_gtBJX4 z@FH%;2FIIDwp+hZs?<@+5E&ks4+(AgaDWe8;eidg2i9BCeHxm#(OvDsoQ$X7UtGY) zKQ!50KseR5U@vgO{aEv&;(q4*bls`YOuSa0`@KD#ZOzLV{YWo_P|v zL4qJ!OoX&JEERkT3FR*kZP$>lg){+@ltass7TRC=<&}Zx@^WUP^{mY=9;!49R27F+ z)-j1jauP8d|Dw0te`xLm&2{Ja;m%9oZO^Aqc4#+9E2!hFxw7I}K@sYu^cm#*0hWbP+ zH!qG#>TZ?OGVtb6T!=T`-?K9yA>GJD4%HI3sq4E6%+2zH9DfQ%s(Q}iNA})q-AOG- zQ|j^YCNjx!v7?6IE&JUcL0j#me?-RPbU)TJp8x3M?2Uldvu2$PWFA!5$zhWk=+Zwd zwjRg=%Lu{~eB1;pr+@4t+Os$9>3U&yjCQ3nT}+zV&CzwG)8Nj}AWKLA7S5h5Y2!J= z7q9U4sB{A}s1?R#?Q9=?Uqgr3>+0^kmTGDcOq7?O`qNmfzY+U_;n$fH9gU)hi@cs? zmZ1}tszw%Z^^YLiKo!@QcssYl zC27N6`Lhc-ES+oRRWiE0!Ow$}Qu5TFYVxM{fAJNU3t&y-1-6cfK^SK&%N?Df-uDFIcC z7VRq2r6rrlzJ%WHBTO*3@GlcY_5!zCfT{O*+dba(Npck6$}!E3bj^GILwsZ&y0H82 z$lgQz4BEZ$dTOo-Cd`b(YssfpRZ30>zql=lyHRxU ziIq6*O_OfJbh2s$+k#x8Nb32NSZ3RxgVxrY)M;$1KIpzYXp!n#2+BgZht}qOBqA1Y z|EYGJx^oEOD1armlZ^G=asvjtd@s3O(KvC!*@dx}WcR7*wCjKJsrom?J{@BY4|jD% z8FvXUEb{N8yO|J>V!dkX&C&?nJe6v&FYhDd`XB7QWmKI@(>9n8oFKt9xD(tx1l_p1 zySrO};IMIbcMtCF?!n!i;4(WWC+9iO%&hgTS!-s^_v8Ho_r9gNtGc?oy1M$R6g_F8 z1CIM1H_r38f(^3+UHaXQBS+g*|LMU;bs7n4V>rC$&%nx-!EN9W(SlRrDSu*4DJm>1 z%09=;7&P6b5ip;f+Gk$|D)0xC*Z>6q-->u{yMK4L80zk0rHibALFCOGdROp$*r23d zZhJ~yI<#vq$3i#HH3HY*n3R6*6+zkahe{9B1S5UK+ljd;@kES7DY=@<0Ws;G?be9;kvw%&79VArlMDRh;)HL{$H-0-*1YIlQ4}Ep>C6w_kh9H$cu?f zlk@PvX%o=}M>rtc=EbC^ep@oUO73c%VR5+hLH?+>rk!qZa)Yg=ma_Q##OcKPQSa(9 z(8bA`>*VC*yYtU2e9i%R#0mptPp%Vqb(Qtlp15dpp`(Ne6-(VJd1{jXd zeArIJ^$bkV z2smoL*;Tv?#b}hT!D3tOwV?35TcYW- zj{T}FhSnDM#V5ZOREfl)z&_(D&z+0|EKCvvR7ur=BK`M3==WCLC&NZlsq$E73tc{I z_?gayfqi{%NNdO$HI&7dbkIy&1ehhH?Wuzew?xf31_)nqqg_51`t8l15j%p3)E;Y% z+e{LTX{rdew5OdGh7Rb8Vk?fN-d~B{UVb!)W;J6DR#ACU6rfd9w0hdWL?{T{CLwR6 zWEG>dT1KJ5g6*?$9kSulT)=c0<<7^}(vZdXdm*SZjaru^Skm1Hc@$;g?h&~Z3-KkV z#EZb{bl|F@Hr63mNi^IsXiV!VMWU5jm&ht@!L6SU|7$UJX8)hP?s|^2VD>az{XaWgo?gg?jwn*Ls`%ug`9MSEaU-kfpzcY)U_rx16 zB_7dH{Vxrd`8C&J0yn&f>L2g%4_6`;oM4S5!;HA}N8Obw(E6(V{yQRns z(5D>Ev654`&M)jXWWGM^O3C$&k?OBw)twx!qmti|jS@7TaIhZZ`*&|-m2^imGh$)T ziH_!d1p%IYxm|T!UTng%He;}O4ugjV<~J>R7Q&F2+9yQC`v?! z0{?x2Pe;G5K6>9_(lHb06Z|!)|C)pLqsk42M$EXr6DW8;Ee%|hc&;hqOBQOh&j`#F z$I*yuuKVA6Vym40iIv*GkJ46@oE&psY@>!ZeveSrnBX(|wLXZW1~1Hk?dn+nwsIPe zjGxuCyVXFVc(*jE*WpC9`$yA9FyXMad`WH(X}tLQpkPyYLAbrjYv08?u%(8Ro0IQ| z+ctu83mhZZ>fKHRC3b|aB}z7HJWIij2yzYE645>{e(J8*okq*$f&lVSJu@NJzjdW6 z#o*6!2NWP}n68jd$C1%69XfAXy?<#Qlt>#kC=V^9k89W@CRNw4{lFmT8ntu1$*jMA zxoM}e9iB1nnrFKVJn-<6OU#jA3oZ{1W925{=nyU6o-zk8XiDkTiAB%%rbc)qQ`V*A zf+H5Bc{9Aoyss3M67LJkwN=!O#H6FgbwNh9{S)s1@sAoqcc~k(pOcW;hmljFA&QVh z_6#>NazA|nbtMCG5knVOEX00xF{z)nM9#j5Y;<-cd7Dz3?9N0*r-HvNr*G{k|AbWh zi35TDzm1Ro|L3)**W*sO*cE5p_1m*E=d+PzuCs#Gcp5_0XqIRTViFwvuufKRxelLi zu@F)w$)kk1&BoKmtV`*mbt@<7E9omcvDK2|)i5;QPzPc#%|4QGKgSymyQ!T;F}sBs z2{YNBjif!^y_zi!CUWW(9QS z#3bau4xS)oQ~r%swb@%VgR_IGBZXA@LkQWt&{ELK3(azr>el0e$ zG#7!=z{&W~poBbOzz7>-PpI5DvbU>6bVZmHv6kIliQ}*1Di_GZf8#{BY#5orrzfU< z2~v)46z*#)g55OE|Ag`D7=MpA#h)68n1}kX(VcvL(l{_5CHBf)LU)!ON8(c_?AW|f z`b}`~Z-RG|RvHt7;c%QE_?*EuT}#g>E6a^PY=$8HCQ0@WNjT^W0NSsc-&}TrpO|+r z&VO*7>2zoREgxvu40NbJu>gO|bNEYoMhL(7BeK>l(D@2ZU0FtrP4u+l)M!2!tGWjc zSpZmfwktzbBc5%9 zg)gUP5K*u46DVYqFOMYkDvUJGYHfv{W!$-k{I3Q|kJ~HAoM`o(e^qw2eXN^x*v`~H zk!*Qf;jr^|Wa9#+?}-fC&z^r;zK@@;F{;E9?TN))d3~eZev89x>k&e8!HsTv#I@Fs zlYqN^`|_-P8Kc_e{`$B^^t!Ninbh7C5x>E9IVSmO`LU>d2U6@SyxKk#lI z|CMoXn8hn`tYq#Y{kA=!&ieK3>j4h&e`$=1Qu;8>cbe^=XD%4l%he%3+_e~eouB3n zM+sn88TZ`oQpzq`?~d+!oma)i@1)wU%lt-Wuu&i4Yg~uXPun{t{z}@`5|Q zTu&^ks~hnJer=Uk9P-<;79eZD-D<7l@Ts1o<`Qz%L!M9j?x=#^$qRjdgT?w$s671f zZhI|#*Ei96?`Qkd(Q;)=0zFrs^XZHIM_yjIlDZ2YHh6oh+s^^4+T%Lz>m#eym$M@I z3l2*s@99o-POq&_pMS8i+tdvVnFkKmN<=}j+4kofUZVxx&3Y$p$!p)i5*HLS6qYsk z-+HX8jp6Z0zE(TZ9FNFiuZ$|6J+q4R?TTi_2_4nh^nX%UnTRufd4f4@zdbs}{Zw;c zd1n0OUib864uR}Y)57IM-F3RJhv#+2+SSeoI5w-^cyyzC<8dgxpe5BpP?Z*!VzpV9 z5^!KXF3aJa`q=j1=mZS2m)&3Kc=S3-P*M3>gu>PM7`nFW&=~NS{{Q`_C0dBKW-ySi z|Li3P@oHo3wcU07oFup)R_d~m?_ur!-1=3Y_u-d>iNkRB6iCNQ0;~Vnwg3tvWA8 zqd7d&v)e{`l$K<0e<5kEJJWMB0*d!8bcbwwYT00!kKt<0NQ3I%U$TK}f7m_RH?=mL zjjz92Yq^zrpcZl6o=i&(rgoXPe&VdxSP^&S98W%)n_ki+Sz81XHQX7JwH>y^Q9HmzQAP_D}ZN4Cn`gQ(}Gk}aAVuh#Y82t;GEw#`uyb@qc}H~fyXNq8$)3b+tDb$ASp>Q zQhY=@z7P@22ctkq*P=*{^zL-Oe-DYE2{?V2Q1HhOF3LK=^mxYN zPcoDViRoh>0Mj#a!1B|Nb`0m=6$-dD^W%}n(M-+HG9%4p(lK{BdMLgPoc81IaN$^Q zc%HeSST~#=Ei3O(`V0s~VS*(9!a0LOk!#f;DL?m3%K@Hbg7MWip&---<&Dmq+XfO& zm%1%BRVhp4imz?Va(i$BlqXqSSi38YV#m%$FR+j$2?ayaG`}6< z+FvO$rbo#^f)fR+#YbK6P!bO_3WRMp5ah__n6^pri09J$*QtB0!AlLuGQ5@L`%dp6 z%aR{3+<6TBQT-p+eFEG22J~I4-Qt!NE4G|hYm!z4fCY*och|2EP!@E`Am15l`%k|X zeb4ardvD&}n=0h+SlKd#L`ua%L-+xLys*-RLB)ISejklJq%Wz(P9FIM_a2TFmm!`^@PIa96}JMSTuu0E)nx7kz2 zK4!N8mX`!Hx-MaltvpgX;bU#}9862}9jPvAmYBYV2L>J{_(5;B85aU*P@} z_KPVH*YNIUwaW3k?_w<1h9+<06>{CB;c~{^Dpzg?a+O5!6dcmDI>Du&lYe|UjxnHl zqo(4umC~3q(<#^*8eY+w?y}3^gq!^N-+ct6Mo-_j)eer$c2rG6PBOf|buO_nx!uvj zRcH+$GC5xB@;)Yoajm*REv+{Aa0m0J4H8FOC}%u!K9cQ<ctK6sV+=rbi`mRV6$;EHxa}T5e;bcs!2tb$L7iHUW z;ad^jpg2Oe*hYYV>snLFSbY6?{`?D3id~cfmi{BgYpV4zmY*TqSJRu*;>wdx&XZ9Q z3Q{MVHQx8|Qe(eO(7#8|3KLxu6N(EIv!neD;H_7Xq+b*UO}Op22w!hb4ca#(X)#LM zqF7{%yx0Z>K(P3V{Wy=sN>ki$mIBqJ2{Q|ad~M7{R(#GNM>?Bny6Hi0{#^XZsb z`~JHBPRcwTqf}PctlanCRsyuG;h;T7uApPNpd-S6Rch0a_P@+DfuQ{V!aRc}O#WXk zIsS)0Hs$}w;D2QB4Yd6JA3gXV8T^kP{IV7PU(f@P+s7lLsEBfChw)!t6^;99nj94x zI}VTsD@feI$q59n`EO9_3XKo{HN^=q|8aFeU+9GafpR6YpqvodG8FWjoB%*@Ru&;B z=K;IzCO8OY7W9J_6Av%1tSlUKX;zDEJeHcJLc<6G_-xeJU2e1o701ZVJ}@(b8WR(9 z4TAPuS{m-@0iT|p2IaDunw|Ck?VJ)5{Ig_g!Q8P+k}DEy)gsRW(a6`JoirDR}Y z`qHRDHq2N}qi{DnTP#zwTR7|X`MgA(P z#o!++Kns+1QWcPg{L?KB4%9Fp?u2tb1A;*;{>@L00u=)nmpCBUtyQW>LC?wx2=|*C zQ0-|8l}cg(q`P~2jhS*qGE{Mevu1yI`Qz>E36-UaOH4}2QYwj(%LWkxiRzd1dakdT zK^70Fw46yDkVNbnCE^WdjvVvz^Apq3Nb>}L3#ze^gElcE!`RF$bW5xlOrW*;mx-t$qXfw2)B6!BUc{sjCMwF!J$@LzI=3eM2f# z0Ojd3v9u)p#-Arg*e7akPEGlXmBAf55VCQkA=a!}oCd83=r%|;YGuN_83lQH{-BF8 zCE`BOEScYl%34S+|EdY-j|&4Rb!-9^Dy&8^NS}-Jss@Az!9#k9Fm~)X{`z<2`m5z? zzki>Yn)+?fxrhn<(X8K8CSyS@)FurG2DJ*PIY1@H{^j8P+x*ku5P$|L0H_>LhN&LR z##B(@J*2;t4pcL5Z$1X$e-UiBG*A@=HJY7W^y;b>NLLcXLCr!dTQY0HjD3B5jimpl zl|UZ_QOk!#ot|3hh5gb~4p7B5e`y6s1hfhNT3rQEYiG%1s+HOOeuENZ!2b~dsFD|# zQF5Dh{}(s*|9j@D6NNTT+XP&<+=EG1ES!(%)ow0KIPtd^0%0fZ5&dHbHd~VB z2SZmHlsOJJG&>qmot+=;o?NOMVVt;ILYFVYE!vtQI)(-Hfx-4MvAai4&uaUcOyg!kITSVzK2JUu^7Jcz@Amj zA9ieNY981F8=uW|@bWEM?3jrDtmf*PoWj{JSR{3}DdZrSK36!(#NZUrr~n!W-m2M0 z6&Nyw0j1=Ru`(-4I@ai9Q_}RB)O0fIJK~I9o>a#ai9G%MzVErJ4q=myg-J!iw!Xh{ zo=)B}BrH+p@U}{wS&{a3$MM;Vnx&K+k92gHT#4S>Q&jrI#A}-lDEkaRQir|1zL{1X z`7)G<#no%kmI4CfRU=9e@l3zk+DkOYGO+c=Ns#srX4;xjb5mVd>%0QM1NO)VVf`F1 z8msdlYvlFAq?N^({S$L?j3e91S;6?)EOv2o@&vVCk?<_JD-O3!rN$Lpfk*>8aM$qg zlj(ntPnX-E$zZsQwHxL334`}?kj}12HkXHV7cf2Itlb^>((m1=sg91~>IKzyWBW2% zuicHmS{Yx?Jszb<)_6Oeq31BGeplPfD!1J)&*dF9?s1)i3W`s7U$IY&@8ViGYUj31 zKu)$RN@8|}R;&sH>QRFrCzB!GIqccBP{feCcx37s;k?4)fOf^L9A32rwZMe!y7H?H zIRGHlkE5mgCR?U^A`2G_V&$vXY7b3!ghp6A zErDC%aV_P}49dwUmQ>0g>}qHtD6Y^*W(ypd~*m~h!$f>8E`zN3bImnl#vmpRrRd-z#` zX)?~xbjBd%P-=M8N2{jR$W6tByLQWN+*Ds(*dE@o!#Fz0!2a3rQ;)&RI*-pFNASSm{K0^7gM;?_ww#)cCYDNz={u+Zit$_m=IW z-cNBU`*ec4!}=aq5w8%h8`_@9ju}q{7W<|$Wu%v%T=mvXRdb|>AujK6$3+dov1jk! z>j6D@Mc(^7ETcl*ob6fkmnj^=huq^*4x2C^GFO9x>+wqSV!&{d{+esBReg~L_pY|} z)*tg?Wh-W38qqHs`G>GUkE+25cL(;adOF6n)afd)SGz8SA!D%?+f&}{aW>xXn!5_; z){=_E8*)>F>p^pWBCU>Xp!g;V{q!}KQO*|$j1WAX8c9F>aD+Y2E~!nA+{aiVjeFFB zR(e}5IF=N74F!w=T-0%wifLr5Him^!4v-sbVy1gh9Dc0@k>(){Frd6BMG7K(^Xp{{ z2)*}D#}G{cZj>L{M@Jg3Cgz7BYO9!L)Iw3QuTpuJbb0~ltSqeWEYlorog?Lh0nGmV z?|ovaMq-B))Ilc62R(u}l*6JLs6h-ip8;5uw!v>mz#RCO3C=w~&}J1`%?O)Nrif+%EZI6@==kZ&;-j1UqG zgG4S~lx1!vi`b38q zL3I=cQx0He8|?ISXJ=H@*_uniq}U#eN=)PJZcB4tXsifN2}rP&1z@T6P5&b5B1~i$ z+;jU4BoE^}!FXg0e3=nZJFb4j3WI}K`05>s`9>F;esGJ@lkOSI}dF=_(qg9$b+noARPmkw&q^JmbPUqM`*ztgMAaO}oWSy&pHmh-cRaysvs z^n3rF#$ZQ>gfv-z1X5I0cr%+n$ftwppTa&94I_{LB8UN+)uDZmz%X)SMJYe%k>rNa zO#TwK+>mInu|FovU-NNLh78D@Y#7-!CdU-lwD24?nxoMcpgoTjYT5hp`VDm8wpBt( zd1FMoW~cS`>Y~Ey8_uIoX&>9m`t7|m6x)UO%hD_MK;p-h!HkzB?~xm89s$cvVXuwn z_QA(ByLU^k+nyfR6^OF0L!KV>PaoPYvT$71Po=Mt=sf%aZ^H%lb$%XsCwaec;TueM zbxc0Cb@cOHxSp*=H@AY4PdOwlXKh=2mTQ6hYU1mIg z$Y|a>W4CD<%XVb5pND!SY_%K`(u5Op3N}6j<}R0Ya74 z@`;o+H@NvbbsV+AUFRJu9Fz-rU9mjS$1=R1Nm&w)wfGF(Jhr3KZpx%g zcTCi8UyI(fPcdd|k=+yhB6azpFk@@r0ILZdK@P{-=JOD?LWuyj z?lF%L5~5b|xywWNVclJDY3{0w3p>A9n`uF3 zlDzzPIxPVa$LI@r^1_Oi0B%w#$q|`-tTa802SXRR4$xJfxBTnt?D1?GzY zZ1M=X<5N8=rvSTJsWx+%BPA!>!-)@M!;09Av`UsuKywUdMmE=I9(M#AqkLG(ob(-) zyW((-dMnEC=|C5ge!|OV*L?C~)PTA3;zV0nPpj1ak1j!j@uU{34)Fe$&`ZuA^~&E; zlKy~Fu5#VNepmw8RM>eaxEYDY=f&m+MYu+q*5!8MSE`E z&RH!FT>7vAKI%xJ{fPz8?_cU6w$Il_DD{g&bB;`b7DYA#zD+{R4UUFW_3-StO6kmk9W;>Te!^<0i-?rPOd^1?VjBp zmcAx2I9YjzC6&}wTM|lA>N`DwyWgG~TjUE1XGdMqsMPwvQ$Mgb95IZI%mpEOB^=Jq zR~b4Y79gG(-bU5TpcrgpsWqmDmn2N`+NVN&TpaaKhPsL&)f=~~D_0)~evV*ZR3-l= zXm+4C+IOVVSQ+ZEZd}@Vw?ILxuc&0v8nxqjqM<-RtABFNEh22LAP-4$m?!JROD|wr zCqNwfzI3_c=R8f`CuWN9@JjbJ!an08aIjvV9Se>2fX-7T>?=|!ydT{tB7Fk_1e0CS zwJDe%hb~Hm(}GbR7pttl0S!|ywW9TJi6f>}W`>UkZFg(JR4@&m`*fAwJ4Tz;r1Dfp zq(O-kp$rT#HF)sg%+uuLSVliD)ibTwf3j}iyqvgM8M#G_?(0u1nJtpl8HH!7&Ts7* zBL~v+u!-MSwdcsKb^P!dXcPIkb8G==unqgJbg3oS@EzWE1LL;^fV(?+S}$&7;a)T4 z_PfXif|QiaK}{}pR#tKJOxf{$jqf4hICB%)BMhDo6puIf>&q5?k(p)QSo2mY;`ow_ zPt48uO%XAR1UPKgI%3vQ9k;=*u)R*4TyPpQwzFV2Vd*+nR?*!NC;DfY*Zo*z<`Csg zR&W(&X`3A{5PFLJH)Xm-md+dSD!CUL7YDY~a`5pi8 zjJ%o!r<6Tz4*A(_t~$Xo?A-qJhR0lMG=CQED=1Fu(Z%4FM7?f>zq~F13O_lsmQR&I`LzeB_#l~e-#K?YJe=9P4dXU?3+`FqefRreC>y=h)BG) zaGdg`o?^vemC9|Umr?xHq3N3|;t>WrJH9;s_Tl!2yFd>2*S-4%e6QGapVx5bmMh|? zAHWDZtGm`G=dH?1JB-_{k->|0=!S~FU;#{Hx}4l>3CnAMH;v2 z`TE*5U=g2t`|oPTJ>o5KUa^iILp2(yQhC2eY%zT*U2TXzRien?>q~zZ$T7*+Q_Hir z;HeXqj@C_sPa#8{W-uP}h0p)wr3iU8_A2)N7Rou@?p z!QH9jpLDt%LkHOMu{d0Uc%Y|ctFr4qD4Y7=snnuu#S z5Bywe4k_bfm-lUSN!&T`-K>`3noGLMq=?A;n~%i30~7R1&g`Zqr&+ymW$9X&urm6AU9|HRchQdgT}k%>-p7$aRRjvdcuediVTlsM zolC8*8`N=bCST-3L&F@nyjY#r^bY~#?_AuDDs5;Aky5lstT|21&HL~q^^($bxf^}k zzCzBT5_8ovs$Wj=OneimVC1Wg*yxm9Mnin8JIgyZH_n`l&>CfIv5w4=rAt%NN|Gg2 zA3?S_(ydk2HyT%hyxlIeG&8(EM2N2W5*8Nzp#)K9YnFkElq~S(gS@Xpg+)hR97qKE z7A`>#bz7#UX9P(Z9dk?4(&2#%Jz7pcvcjW%^39qk=@EM7mb4XCHuGoh3ISSc{re%> zV<*JMq!WqlREKhl{t6tLuv1+J8}N2(Qr*C=qhd9p44`avfXR)D3j* z%QdN)!veaxzR!Nm7pe~~pWU1idAe)Z58YH2)dSlr2M^iwqvJE1%Cb!TJgEe;7EnEyh3C~P3+e;ACIR<<7+5Pmrl++~= z=r3XV$m^Ii5h^_t@qc@12{*6YH!80@Ly!zdSS!*_FE!W7KV>g%F!mnS`5{Z>J?!r4_f25hxUp00|l# z%x9ft`U5n$Z_db}^=_K?6BagpL#X>{d+<^_pLj2Ju6cK(hU09YimLfuz5PV(p#ZPV z;u040p+d@X7@s{$1W#33C|^r*X1`=!e$<>AVt^ZW##1Q#cW1}V+M6DR631~22nqKXMQ-xIw^%wgMnB!DF!3-_rHe-x#vcxhYQr~*1Py{*&G}J ztcqnRG_B6TBn6nTw>X99Geif|yzSULms|TGm7|@Pm;u=fkd&uBDT7ymgPE$)&y7`T zT2i;&R}}3H1{zYBAz*VVBiFgYG+9u<&O{no*~SXQYfr({I!;fmuYX-&mKkDXYKd{^ z96e-to`E=~@E4*&d)G~G;{pb*@nN}~K(PcTPG694(1Ye}x)Js=w%TmOJn!IB5BG`> zoOOcrUzqH3M2ssW!FVzLFdh^us2l4^{3LO~)o7!NWAw4fF!}*>XoW z`m^~}P`)mo6mQ1pGH0)+mKFmvmFbD!QU!zOFvXB8dm)_9-cJA67J8dsFoYQZ)N}j* z?GEFrDR>F{F>PLZHZ%E?FH2B)VGARKabeOJ5a9^{jZP@r-d$B9&ei}F3;c<>*#a{T znL^uF5z{5cK9Uo8rfzW04l+d%Y6Nj>G#_a9Lnk z0qsA5Tfe@*b(44)J zMnYeSNh17&EKX6SR`+W_CR|*C@pi&O{T5bV85I;}99WScw)GViHki_GK7T#b!M+)3 zt}jDv=B>Ehep;{Ia>WU#ogj%?eSmd#hc9f%dt_1(cZtKvEuvS zjM>h{ujQyK8}g)t&Sp6($vpQk@JXVYF1}YwJ5VY~4y{xRJ%n+lPyXh66m_;LV?E!hz4Y~h2rEMh z?ibwkjY{ML0ia#p$mSgg`&mb?YLPODl8LW)$4xTGGmo z+)Dw0u7^>C5-U=g_E2djZxw(LWP^F6=$jnNCvO2s?+c}t5pYnhawLooq4VuxtJYx~ zWH@>@5s+;$FwVRpjjqTiP}ghGV&eJ(D}JYaPg?Rs0R_eD{{3nQd> z^Rj8sNL~USSc$3qt5USscQs@xcW60!GBk{+*Z1CN)@Rv> zY8qo?Ezk<5yZe4tSnhP=6#+#u*Z%qH9P9U+1;*d?bppt8yd zDJsS_>RwwH-a;~82$g3T?AMoJw() zn2qn@w)}b$HfhKq~S&i|+pUq8S-KtE45M&hKLx(;e2*%sh4or<;9&mUK3f30JRzqs8X!baT z*X&^+)~MGa7lJb7ltC!gPR_23%xz%RakKhoFBX}a<+aWR7Ng#mDZrN$XY0t&6b!}< zDeRIhv}f7{!6@Mb$Yw$>{L|%y|LXG0No1jZ(p8qLT}|-j=gqC`%US|aMh|Q>#7>h! z%a65Y zUvAdg2cL18T_2m&G-{tc?n2pYCwq85dai4+@;spM-Uo}^Ewi-^2@MP2))c!M-B=s; z3qooSL`c@7l-A3RNW3G<-^|qi2{TiGk(3r3Em4rPaR0vOiAu?pU-r{X?e!L|Bx92? zz9mclhf3-vvJf7A%Z_fKzpT--Rsy@ddo)GGs(WnJRxfx*D}M}XhJ@W&SyWWT z-cqHG?SvtcT?`5}k8e2c97T1(jYU}Ko3_zavaJ16sWyj@zMK_Yl0Q*wyFvD=0e2`@ z_4|yjE3g?Ti({!VMty3XpU!$rLxLjCm6q%Q(`WsLqWSj58*G{kj%DdzeoP?XUMfW=*2zasSwYh>59ZRa5?M-f?}3xohQ! zJ&&^UX@sWAARzAok2{63(n^tzm%4(wMvb76S-h@TKV@}n7t8-kvBH&DHWM7U1r;or za4$G20!0Ec4XqJn?0foR1quF*OgRr=Z#AAc8a;d%Y^J?TquIF}EM5M&orFYp4>NnA zS)u~GUqW$wO_=39L=Gid#(4P3E-C5^WgI$)1&OypZ*%nQ62*GyWzo7-2&>ciBa zsp$oswOm`cBhNM$z&KPVMqdn>__gcml4hh&R?_dCgEf>nXYuhe)8Mv5Vp}{xAZ8*O zLWu&y8^!wUH?tv~QB;a3LW;5qS^-@px@8C3+w#~!qvJ^7q@uKvm&-L+N@eO_a@K|z zKFdxz!l8qPGjzDJ4{SLosv-U~p-k>BMic?mNIizrFyF9c0Q#6@Dhlw^%=9H>nQtH) zN05WVLfXwT)p));*UGD5>KV)WJl)phAo}4W=cx<$~GM0e1HNiLD#p z=tLjMoAS3Rj`f|Ydfo7R@iRAUo544<dsQ7hi8xY9wrq8SsG^~EdIYEE|q`P{Yf`Q;_O(tv9_U4Ve3MH;d|)UB=K zh1!f>g{U^ChBGYeYNFE{CPYQ(6DC?1{8 zh}148XBNkcS&sILs-nh?2GLwr;^17fYbY%o(@HgJfVD?+)lMsFbLHw^%31|12mynq zJ^2cvy?P29Bpi*QC+F;)+AE`RGG4wLfdTp55k3)r7xbA7L6&B<$x*!ygBn+kh#{qG{Y}h_MQW;P=QL8^Zgr%3BB8lmP?Ql>diSBt zrp?6Gw$h^d1`(vt_*rA+5B~YS@o*AN>WQ4gg?eV5s z^8?G<9BfFahAyocfp`h3@VAL^VD?{@7CBn@i(s2(gV~m)w4G6eJ)$>?NQcNo{A>CG zwT7aDKFPAWri;-s45#C{Cyn-EF1CgSy9CR~lT#~IWLFTsZ$xeE;(l~GHRJ`kh3AHg zXG;!DFXW^OiellJi?;znA!LcORQGqtLVgKZo8Oyk)Bz)7Bn%uqBSMQc_82H|s=s?N zGQYh#7Dme+>`qI^GIK2yk008NgW!mm9bqmOpexCV|N%)>eLu7-XHsM z*D(Z*?-4d_eK>6>NcmYyn%?q^e!9#ek%!|@(>3wDwoxbIbk*jZ9`tbqbS`b&Zcb!w zvtu1X$(njk>r(0TJ~XA_A{fOr#V{J*bEnQYj@jpzG+B@yKak`nans)MTA;@<1}=7|F*mR>wOpxIUvQ=h2Q)TPplQ>kQN*Ic z6W4bPEU?4t;c>CKawuWM96=trf-Hm zg*3y%#X*HHVf&U8Dko!JbW7dBe>J~h5bIx@09lWwSf*EHwyhuY9zoGI(N*utzQ-P| zB{9*Yuo%G0+BfiBN>lRP&rZ`m=NDf5Z;b7OE()Q7I;57AdBVPkp0vrtu7rui5{E=SKUnk}ax!3^0# zTzKKNf@WZaxO20|dlwrNRfn)@-3U(M7F&%1JX$>c+k&vt@_g``R?&kH!u7mDRlGs* zp$t)Srl%EotVqTtF3A^~<8no|k($ZGhAK9f&C}*z_EDyf7jr7NWkFM3bWt(53xklY zj$){wabFcSWtc`-qe++t1pQ}O5He^_*E$6gp5+V+D^rbqY(WN<3`TX7YN|f3?o?A2 zUOni(2cYNQUn{4O)*K1yh0kUhZQ1K1sgR%E6~)FyYJytAapLOFWcjk;Ny4q#$Wh z2DUG&$a+G4en~9kP>OX?4mb%k7AX#Uwxc#oTLCQsTp$ZT*Hl8$pFfD?9MRIu#reQY9BQ(1iMSjjPUJ_}QLd3jLH9J>CuYBn6 z(XNW@=xB|($lK6-FvvWOfILBe9reVo^J2@zj5rxauUAXd72a7m+Yfv=Da-mB`7vQev@{A~z8d(kBi z?=&I!@Sg{sOo`LwRaHghfKFoQVnVW}gwF$<=k1nL)_QC;z4~tWM!OODdOZgFVG=w7 zykCZ$C$=Z};c6~swnTZ{5$mmg_!ys{X}C(UDR$9l&r6YV`Y3t7wLVne1fHz|%0VpN zYkS3pTWTqU#{Iw!5AOFGYK3$3*w0C*?qcfKtY00|k3j>rpTm~3*8~`F4;<{J58|+u z^QU-*q*%9v7ZiL-wcjzmi@8$LDKfyqk3iOH^O-#%e3C5tuG-hIm&CoNX-ip9`-tUs z-T!pEA{|}M7vBCbEN$cA{ssi-6DPnbqG~(6j06@M8ae)%GJ`BhBC6t0kW+a=XHXV| zfW`8C5_w{vXDfGF^%BkGSWQUV9aEe`qy^f?=ABnT2or85wmLzD%Eg1HilSFuq?k;d zOVA8t6}1ztaJ(-mzqmf}$7VoYDq+fW@XXx0lY3F{OlCq2Ni?OM5Q?-XTgijEjVg_b z0*)|)Bx9VgKD{s9CpJ+o z+Lkkr3#hY$yX*H0~&2Uxc`KREwypso%`I8%ohV-r^kef(KN@uL7a-3{n+1 zC0VsQ*FcA24J{j(wVXp^wiLMGzq7Kef2Y6|v*(~JkXKmIt|f(OO0HXIhBfdwo6i3LDhtZLF!DTdmUXNs&gDN=%=u>;m+8|3ACAK@c z!ZLGDlWPxQ@ghvb8}1i^6%XLt5XJjg0VmEOP8E}1`nbP;+NA}El*kh^zS>WBi?6^X z*qm}%0p+Oj2=FKZ)T(7^(8|*)(<|b@Yf@wGoA#8e6%}2`4UvcyFd2j*S)FFF6v}jo zYqASkdg9I5g{W|gbyiL;Gx0my!>ch!W}6Y?2CvtdH~3nGgfhUg(tVCVk=#(GY6Lgc zm=P9Wx5REA%ZwbAlTDkMo7dTK4UwvBds`4i4C>T_3AW$VRN6`dGlxUy1pjlS;*&`S!&G z;3FYLtbx;KTAx{{1QrFY)jA|cB*m&OX!}j2%?%Cy<1gzA!(W z{?@IF0vb;rm?UF}lkx}QB&N4T^G)q?A5WVj!g1K_m<<@QKylYv6&Sz277=6M?VmGF z&ztcAvp$CXy>Ii2E-+iKN|ZJMgz;$mz3|U~{m1nCwT@$#5c%irr!zSP_~vPUE7?Db zJZNz5HxB>2-uX&fLN`*3A}#cvPkdzl^X^~k!oLc!N&K^U^cQn~Kk)5V_=nc-djItR zH=pQ_59nVtT&G9-*E|0_2rT;N@;}9axcE<%{`0(3_tz5jpSu2efFS?>q{_e8Und2n z+~vFLL<|znfd(dna_rJJ?GtIxV*EFG-yC6CR3PiG7Ig{6<8nHWc)g1Vi-;g*H&ZH8 z>m7=t{*_ht|NM;r$g3l?Ve;1=Tf#!KGhLPpbi%OzdSTU~#qmOw-;W<^|0k)5gqtR_ z<01gbNO2KTK&msJVVaCP$SkR)lT0SU2vV69=QU4@<%Xyn!k9ssB879e)<3-eq14~@ zztoHW&m{W4wGse9`qCmC2k_Uc3ev@F!4;CNR4W6yb-maO^-yBAX|#x<{twpPDlU$1 z+ZGOzK+r&NO>if;LxAA!?hcK6Xdpm<;K7~7CAb9F;O_43+CbCj?d*H^Is1Rl_wYU3 zeuCfXs;;V4Ys@iajY$L`^8Xp$gkv!pwOn{5z=n>}BzVAm!ksxk#I&!Y{UZ-*i`-P! zswvs@xfz)(nN(x5;S;^Zb^?>=ie}Kt6iEawj{+7eLuDsjtZbqabk#bMgu_+kBnJ;> z^wvS54mPpYgR*kv_e%yU}yXH5<&Xo4YE|$zU z?=8@{6Rw19{o9qvnX6)ZmE2~FnVPOYBr>ds zAWbB`*%FEM$LZGuUko0 zqO=s#X{Gk@)pR5_qB++!n^V9d*B}(XExB_d2HIChVX*c(i!9vv2zTDaq@(XS^nB-D zfQ-WMX43c3TmOD8-Wr9sv$(49T&4ZyiM*P1&v-VS#nb}IomJHa67kuWWeTQCcX%d) z2>%8JtG`wf7r{4p7FBeRcOKSi9~$59s{)?ss*>L$pYxYTTo2T)T*i}wBXT5K?P;%) zBG&%A_5H2Q72M6f@&p2HTPcm0t=?M6`(L91w-P>W>ydAbwfK+KQaZayeM_`59C;WM zAnQPn+a7lw9&NZ<$nrkn%FMW&^K>4&B()~#Dr4{1hHLlkrun+&j_Y67p4BbSa7akj za*XHT@5N{{JzNvdx)BFf%H{W>U5AN6EU5Wq*cBGt?d%K7TX{z!oi_&g@HkqO6mL7~ zkNOB-`*9^Uj*Xt^QMCz=AIQF^x}~%;xGMLXNy3 zh=CXT&|Nq1p@WBwuh4bWXv}3%!m8@<1E!wT#bkOb@7+Q#|>m+CBf7tt0KMuI*!L3(iI?t z5^bEOnE#mXFvHVjGu7@wkK=Iz^b%rz4xNb<{_~_)bm2}i-ooOIWVeh^EE#)f5aTj$ z0J)smpwMn*61XYX_vjj&`O2gYW~&!+qg`(SvoV}`jPZGr#(S=c&lk1#rR$*$H4!MS zTnYI-9w?$Dy_w7K{&+6rNlIoHX3z6NIbWpZEVhM3MpU|M;D2wTXb<@d5C1mzUDf}h zO@^@A3A#jCoCpcnwzXcvQf0!U$BTe%_=BNKWweaxImaVltWT$?%mkX| zz90$OTMR=CO-S)S@c};TDJlC8J>1$MeLuMN$>qjd zM$ancU)byJtVWMT#PpE1#2Eh_((nD`8}|YpSPFY=rG#0{myXag40gR_+()G?rOh7s%Cdo-}{CuW0q_`)aZrh%+Dbt zd=QHS(71sB&|Vx{O!muYD8t}rIEH&Ze(+R2ZTV(b*^~y!nCq!0ZXZ!llAvck*?L%S z&NAtLQ*l2s`#34dBl2{lwaA&ZBMojL-x7H7MH*^uaO(P{YnUImb-C=YgQRz*?*yII zcV9f*A%DPlaoOhexI;J3#kfrxek13uY0gts{Of3HV<&cJN9f7Jp%tOr2zy%*uj7QEpV z@g-t;JdxsaOht->12oYKdx~_Jt4_CNxQ+?O7)Hx!p0W4t3??tM4T!EpC=Y<4`s!_`{X5kdf zJ+;0nv#_cZG(N^*!_&O1WR{7U&icj5=lvR#(C{Z{eV9@&KEn@c1g_xWbp7n9_9OI7 zIh9(FId0Tg77PD_-k~FksUvw;z|gR>M`o`Es5w%z2!m0tIMOfU;cT(XhFp96ejxJ^ zI1EplvH01-k2uBw)8&eq^q`eL#H8qE2erv%)cJar_ZwOB>3hKaK3h_a1{3NuU4ZUa zA6Ucq#)DGG#<3?=thh#%`jiy zo>s)nXC$n2dPiRKiJ+}0TFAC$PD4{Pv~8YEAUWIo&F9OilYG?1gTVKf$GC#MgdWdc z8NC295amD#zJEqLmt9yR*Fnk0YO9|}miPnx;exj#yo_re)aQbE4?`RWSIZw-vqZi=z=NLlGEaCNU!lG`;Eng}crriY{4di2^EdAha{n5e z{vTq%5N5ql8$C}vxQJ*=d&{RGezL}Ws}p~S+4=514U7=4tlse&wXuG>-7v*7Gk|#tbIi){m}x54B0MXe)p|TfB#mE> z2-;fM5O{<4Wmot=@@ngF(7I5UON|@F63EPe5 z$08dXTI)&%cI1wNH}ZN6Abb-W&8L+THH2_$E0?vB z!w*9EeKyAMClv4CSe3W+Z}|QdjfXxDhi?NCu}@W}~L-hh56$jSOkyBc@@sRMp5V^E+R^fHrvKIwNXy+fG8G$FBa*vK=$LH0i z7m(21EAT@*s^H7>tpCeP5rk$m(lURtm}l5U$+Spx>*op0jxj#Vh0_YzO10o{H|Yw? zxJ$!Ib(LTzvy`4JGNw)R>4I~EMQKB<D`SjM`*9*spP8J%1$A8RieC?^`1l{C}XbyUbTVIF1y;p)|+$+g8m;JBL?{t@w zp>0Ml@FJ1>^!jdYQN&lyg?Q`<8unXqFK3w0oskc$A`mJmj2GsP``JoxrT;m^yN>yf z-eGg!s9X0J9^gTU#PU;&@@-rHA=%N+RcY;s&^4XKn-_Bj+ob)MqnNv+Fv7rCxhco8K1pZ&sL$QLkA384R?X+UtR0W+{m+}8-e>6o(A^iX{}-f3iSozNO2LufO8@wN z#gnadAb^9PhoIM;#2O5?vhuleGvk|1b-!(xH7W9G?4#tmCxLY<_3wl4RaZ}%n5jwJ z>oY;gicrqelc$zJ&dq@XtrrWtD=i`USr)9}QatO$wZCRgyp)L4e&Ddm~3 zxZQFKaMcMaKM5vpWX-vtp||#M_A6ql)LL+$7qBQn(Br?FbEv2VouIAydN06_>Eru@ zwOPp@K>o>~C8OYyuA>dJ*6S!gKjp|Idz0v!+nVc=QBFjzgm{yh%7|e}(w3#;YV@dv zJQN$2_IsveIr5!`=WhH(W%EsH$-9V%eigbYMm0Cg$|+bM4JF{_`bF3{%hlug3yr@= zJxE^)SGo7umlWXWX!;T+vNgD`H2JIs!LD|zaCLnv>2vSIplj1|t_Em2-yR`(lJnbr z;qR8deZP{eO?tj{7(Xod7|}7Bd^;cHq9goV^w^CW?wRXiho*nhTe{YFi$rGe@B}h> z8ZiQMyg-q=8I6o79i@tj%fjRSM4*jPE$R^?;2f{6Z_U2|gN^QXmBeJt9_B`YXN-E# z6Z}x9W5-o`D@@xz&pzF|G&R;)6NPNH5DrQ`=8mb_dZ>nTn5}WT*FM*=T~L%oy*UGB zt=GC#+OOtx%b*>%gyMJH`hyRnQIuICDxcx}pV+f&f{N_d8h@Y=xDLv2MO${xV!q-stZXYqmPOvw}gyDE6rf zLd|S0Ca_$f0LqTp@DH)-WV5Xop80o31Q!o0MFu%|;j{xf*j zbvpb7MrWT1ITgg|gO8hG<*1lybT}|->mQj;vUiY*mqcp zE4hd?rEYEN6RAl%w839q{e`|>isQQVk+hvlcgXQXaG<^Ad@D@EFZ>0fXCl0YWp+KF zCMZxPd4uuON8Fv&o!L-|SnKX?9V1YRD4!958M>6$KP21M=N%Ic%p}SXFPsEzti+OC&z;UJ64m-;LQCM@(A2Axpm8P5FSPRp(F;7M?}oC3TojiqJ6EW6MN=; zdj+yxr4G&EBFG$v|80vP+e)X>%M6~FK9VOQ4^kpsQbrFk zkGl=y7sn#CnduVT0JDx8M1RN~36jXmaKin^UU#A-o1ZDig)Nn_l0L!j$A!BX5eT{6 zuVSXqqwXHj+MWzEc|Te||qx`dU%=36q59Y}l`5?k~ot_RGJ+uG* z{d)xk3M3l`UqFGa@8{i9VVXp~zJJwm*i)%e<>TaOWN6~4;$<|ov}B5=|0Xt^ct-?u z1RvbV$Qia=jqcGDR5H^SduPisywip)1ww_P&fE_)-e=1(myabfLy%SR*{>qcw@3>u zC0S9Y>2#4Kdjd+ry^F(s9_KbRe5Y0C?N49kSX^x9w$55Gl<{7zVoWZ~DcYDMoeSTh z8w$>iy13DbKoVJ}bly|zR|kxJ<$XM@1Yei7RXO96_7OzvLM{e!3;_0Zyq7!k-baaQ zk-b4HPXqlW`lnsLR=elWZ&DUT4ZV(R@PZUb7CbSR)5Oq7eL`qEASC$v=orBJj_t_@ z?)TN`4wh*&y1)>8kYBiKd->yzl*i_1*5)8BXzQ*M8#fc;WaOJvlE_Z>d?y5`s3sJE z3a)#d=qO?^)n4X_42}!8W*VCWgAK&LH=I~V`M!nK@mz+`?BkCUAi?ot|=`db7{vkG>pv4YGEHNX?J zXA(C`*el*5dUkSeeK>$%J*8@S`ht;9TlE(MLd(_;4xbe~E;G-8>HE=c?^)Z@jqZb` zY*=owP(rki{SzxY=9G$?(+aBLenEiY_P={M>pWGH2sZ@Q?`T*vGxHzMST7?$BA*r9 zg`y@otqlFb``aI|SCct0o_5AB7u%v1Li*5Ovp_s*Rcn(C^f%67Xd4tJEBqqzNCIg{ zGPTyXNWl=Q%qZG@9(RT$*~&+e56M+dta5$xpLX!z#(U3Hk}LdkZ?=ehdvSiO#+>_N zpy>RgyI)|OFZpK0Drth%gV#|QEp!bmMJO|S!SOQ8T2sl^IkZ8~l7H;Aqbv~c`K|p; zcz?$w$+JQWozB^wSaX&X#%~in71LnXR?d@s*O#k)8?@scO({NKzt~%G%xcJIq)YJn z_TYKjz~I{O%=r_657E`ge#ByJdF*9;WoMty;ERT{1 z6sOX|h+@v;=YyU?A?vc>UG)=SVM2^asK>MYiBu7T`T%Cm!@_jxeNQ6X)RU{cAMjAj zioCAE&YkW?-Wg7JKTc^0D~QWm<(w0=1#6PEml6{HXC8Eox#C!r4hQ*7&ie=kaB)AeAg5Te&?J-NLVet_*V3((w!`c}ZR0942*g_DWrv z^N@Y~=0fmwQwds%FZ4WH`7D?&%ltg3o>fpnpHzpvbR;*ynf+;a{(0VictgCS+uy5^ ztc%pb`;PkgqS9!$3-r{m((*BKgMlY`W2%SK!{Ys{1RN($YQXsw_~qfP-{H7l`l`x5 zyyI5`yDs_DHnMVZO&apBkquDSCp1}ESy(=ouHeo@e!8cI$>D8c&Wn%#*!JS;1&yhy;m7}bl18(<_XuxKm;1|q z%F{TLgj#bBA;n0!RD?P{?;2emO~CUpF%WxRE`lB@9(WQ>;?3?Z%z&rW_yl^R8#)qh zYY*GNGqunKWuhEw3)924` zDVH$(Hnse+5YSEB5a+$U_ufQbMmY-0%x}O)pB6bBfArg$BsYVEnEvCj8~?{+|Nnn0 ziTnJGS)r;#%_hbK{6yw=$PiXJO=kNAavm&;e=?ZcV`3p_5ZII7C40Su} zCls2cy&(#0P(>McbvD2(usLWBFgdXPFPj5sm;U}2Nc^{O z;C~FPWdCPQ;NRmLZ~jje$Ul$ZIrzWN`ggqXU%}1)`{Z5de|OLPkLz&wo7wvx15W=h zE*Vy!^4~w_P=3e(yeS~lmq*JiO|Vo}pB}0p&78q`n7G@_*rC90rY&BZlTlb04^=e% z2@`DTZLYaU=hvbW)W`gzQBJZjEhEvh_oQare9kG=?~ z?2M$W9qXd2mqveuFht!VJbp1juh?(vmCgJ|Kqo89{3F>$eJ)P>(CA7_TiWqe33Xp6 z(gD9~Y;Md+5kCOzr}9k|1@>UQI#U00-kgSQLSn-2J(fzF`s9rH>@qTRx&{kV2##n} zX2hh-Ue}Kuv+ExnS}>6J=@OX>rS5E%yw7hS$!=RR^|kz-HOY6C70s&In-pys4y4(@ z29a%UEiifGM;gB^Pj65r{oR;plcq?gsi>im!lRy?*Q>=RATZ2duKvotxLLVr0MG2#vj!E7{`H{kNu`dF~dr3 z8ShsFj+!riinjcwsjB+#0A_plJ5EC_Ry}*7;BxYZfXlI9`zZ!##hnM=tajx zbWct+%VWa5Qa8(k?Y$30O3{_hl~12q!doI{tU^au4=5^i3=%%E_ZDSf8UTcemBQk6v>}Qh1hG^ zf?Ht9>zI)^7P>Oyw+iV?CCs-uA7S`#&hn2X?;x}rJ6-@|xB1ibnyf~9SEgw-&9?G# zQm*p^gsop__FaO*S=l-HqluZQvEQ2Ma8Kr9Q}`t>25^uFFN#kgtC3XyP%_%-O&+V- zqEqDvA`zNZ5PSWjH_&`9l+Wc3jgicgAb8($a1XXqj)3keBB=H53X7$ei$X%>dd6a` z^&VTORwseDX#Y1ppO9dvT`2Mq;hS;t{gqC5=!k_PnP1{ss{4sGPDicecu9B0QD$B? ziRM1d;qjp*w;eWeNYyH@FRjt$^#HpjRY;5A*uVjG4T`e2W&fD#(H3t7E+yVf3OR4@ zCh9aVSpyP!g5HN@nRvw_nIi0wnDMH(>5_`lo?9F9c}qeP!!OQJ4ZLHW*C>`{8PXG> zHc0gJEPose2?g&dXhZNmD1RBzGOmFjN{yk#yWp1k_VY~%MC?brLnR(T&)7Df|3u&W zpy-VO6?>yanDQ;%aBpj{JA=@$6Q7wTzkpE4Cq>JLW!syt5NYgUDGXfJB>YD-b2kjN z6$nfLeC!?QRU;AN`Mw?Tu9}nuGh+uje?QA`Tfw{|=K z`jrXgVCP~)S(CnoG)&FJ&VXr`DLg#SoE6h$OC8_!m<0yC7Y-c%iNPC5u;9wqXu-a3 zV;Y8CqA_z-wG81VQ2KH}r-*PL;8&)0=lIp5#axCd23PV%i~jI{esrHgdhx{^FCTmG zY+d>j^X>r=pe)Me?l;+e-;xbOSxrT_ms7uoY#yAtIH6UD)}Up;hX&>X{jG@Pl&XJ0 zRnH@05bA4Nfn_@P`0JR+>H`gMHG*C>SGG4ow-m?<=#^5F(S4yGUjHD{KgJNjn0Bwo ze%MDyo?&4+x>KP9(-nJHsWy}8m$q%zJu`2Q#7R+regc!KVMmAq8=#G+1BARVNzo;& zo1(6>1VAY|?D^j34&9-k40^v1wOovO zr>%Ejbx_lnB;;dFhyF?-0Xptto1HG_E|5Z++2hcZ9_q<$eDw1dd9Dxp;K(0~5GlNp z)u#^+>BAFT5)U)pSt4^v>$_3vP7>cwF<0bVJHI`mv6AX!m+gLloL2M~b%k_kdEElT z8C!$-wj?7nNd>G+kheIsT&g|EMu*`^cni$O@nOB z<8CVYU&(PLs1v@wNzIU{;oGaKYW$(OCR>sdzr!!VpKi(=OCvNpT@!md}D& z**}L|jt+?Cceiw|Y*S14G6OZm3<`SP6ZLM@WCGi_#f!BOZeUuud0ygNO9vz{A%Kah z$B%(;w3>#8c{fdEBTNZe$MAC->mob(jhDVKGB~${@VQfnO2{N=>u|L+B%2TtHJ}=m zr7@>3)4hE)M47B(W@`LndtZFTvnuFn2?f|KuAerpV%R!x0E~;frJ*3AY8yiwkzY_7pZhZkMu1wE|N0aE#e^DnIlFW7Yr+a`+sE@5HfmhT(l1}f9QBzp z$Wo5TE7i^^k4~vZr`b~#N?2uktJ&zyJB(d?3TizP>+9@ z693W-71XCkmryjpOwP|^ofN$1O*g)cghS%S@y*0q-^@6byZJ28Qf00#6Uj_NJ|=RK zXGX!q`e)7&hOo;x9(>5yY3GGh$uIj{njx;Cf)$<3!>+e1m7WutbDu4$K)X%eNvOkR zmVqa>;TM)H(Z?H$)NW^M3O~}V-{eRq{QMme8u)u85|e|UeQ;m^IbI=svCWRLKzL0p z;7Wl+aN^Yvbpvl_1TD?LjsdK!A00oD)fj)as;M?A_Z5$|@<4^t7)LcaX%vsz5W;HD z1p1nB8bMmfHox(uWpq%H8$fTLUjfxPUEf0TnD4IhX+%khjgz}?;Q*O(8~tRhB|fC) z9noLZJ!AU^>I!A&SWp*#nR$m_cvVJ}NJ1B#>&k(?jDp2}MRoH57dP>M zXv0!RPbo)_9|{Nw#50{9Ygw!VJFMIxj|=;R#7@2@YS40enEG`3s>}Et^3;kqPw{*lu#hR38x$}}xEd(rIEc07$KbBM zVF@kRWWfNPS@2Hw)l+0H$Gu5=BTHW~l?Sk4-iK}qkI{CP+7?ar>!o~0fMc?X1ym(O zVZ}4H0^?@C2bs{R4x)LepbKDfRg2Sasr7pwEg*v3(G%vRyjfxV()iN=Zr6k%*@%VL z?X^xbQ24fC-dNJC_>e@CPE5l49lFHFOsXvB%TUuVr%ZB#Oj+>aYCBfV(6dXnUjk`t zC=opzI=>=(t-sYKeK=*Hm?YS-@``sN=s2rsPUvAo&qD|vfe&NS;ya{zO(|x!50}<( zO54Ice2H&lVqFpNEAE}}zbdEa|N3VCwW6Gc=^Hwa{!qnxSGZ(yCil}yTjQQoOf>zW zt1Muj8J_55!d1xpF3zoWkQTCMp`&Y9kk2Wr;GTl&+lHnS9#v>26%W;kcJfIjZK>f?gInAgSUh39Hn=%WR&pG}wNRw5ujZYh zW$?2r{>9O)vz8ogK>*vEG4w4vmadV?DUNoTg~SxFAiL`imQ2UEPnr#)El~Tq~VE1$$8FJ{lMN`oY`Fs z^)FI{f{O}OZAsawR?e-V&jqg@&dxsgJ<|mR+uj1~Da`4UG*wglM4rEmJ}|=;Jaetw zH5!_-nvSHq#(RyYqQm3!ZYSc)7MVg=gXk=V{T!qQWfiP^E8`S4@}3bAwV-(3XCmDr z+fOV}pC&|p7}-%2vyWh4V)29@mW6d9I^vb&7O)Ql=Ww9CUZShMlGi6Oo_*^q;c^c@ z4xzGMtdF`k6-h0Ss7V9GeZw>mhxvuN5+n0yKWF?OD1w0 z60sY-g^s3ZX>GIp_SFm_-+*mb{zjIZkn)ptbt#((lk+n{E0&DygOuo@mhZMawJo*5 zDdW54C!b$uws5`f_UW|$n$wu>5IQwG|KCf?0giuR;t?P@h zB8Yc5EYD0kB*H4O${phID?Ea3xjv|Oo0iY{1iQXs;8=9Y^DM7$VfH-E+s1zMRTI5D1PbF$c3gTH(-5uX>r;O#2c8ySoisR)vmKdSx(Sxuo_yH z$XLF~R3fQj>M&D8S+I*-N!rXmBKN5Mxj6 zaVCM4mj8S5yF}vCX%m^V6SmOPLQJ&pfOyS?&QTPrFDOcp1SDTC$d1H_wCH1iEK*%J zpO{4a^!f+)2XRsU#(mxhcVMc`zY5kHh#$pQM&VrTyoFYyMJ*?3eTZkyUWNn0y67dO z)58AN69(45bqD~k#2nLeD)iU+-Z2H)30^HSy`C4(F_-1*`W&C~6w4(!r5l3DYIU}R0bz4Nv9!nxwRiVjuHCZ?$$@0;OrKr$2- z=RbGhP?%<8BJQwv$z(7m5PD{M(iDK4+fbU?=QLmag4 ze7Rx71Lv=SInKec4hdU7@1O8Gr-R*YMBXBzFOB0?6RZ^hiN!m#`%YK-9aq?K6B7!= zAp0De)6s~s=~S#~-%+_Kp8tKH2V@D2$Q?7R!G%Wf`oLiKie9V~Jnwf}||L~Z1B z`Kp_<3tR;tFU;A-bJ@AV$Wpa#4zPHrk|#xn!EEElu;GSHu1GG7Sjd?biatzY2{V z4OvKPauRuHV%Ko4WvAZ-(eTv%Jv=((6w?$eJmm)Pf0y2nLwpA>3Iu{2j30s)T1nBT zuS|4>{k|2N-9M%XuZeY60TG$~zS%sm*{>y;K#_HY|MKoO%YX20pAi##zZ5mvdzoNj z-WlEC;LjIb2^L?gB2e>^-*TPh=F}kn%Q*)k|HBkrka5ueTIw;toXvw$;7GY424ent z<$l^ix3;l`iH~(__t$Q1h*8baq1wv9nPGgjhVqk>@6NHF)y@-EVNmuB5>hvO}3sZ*(FnJfK6bk`}%7@g9TDp8cg*aus9b<-X$O{Z;en8X^z(Ue5jQW=dtlX~1XADlNpzJaOennPBkX!tHOV88b&5AHxErG9raDl?{ z0GGtgSeZ0Wa&t#l!n1pdr&XaZm}I#t=i5@def_*XCTR*t+{)NH3N9rYc*)rMn-+6A zGTz@KA})!u5BdFQhp*hS!tcMDekAVVatH~QqeL5_mo?>f#Ou`3=-ie{wDR`IP@(Ot z-&0-7)}Okxz)N=dMgh{M-dyO(HIYZSkQ&%|KVoH82e9jl*&xkljhPZip}>pS;NZH@ zpDLjWj#a^%$E>NW#RK(~C+eAAY`vaB=i*GTE@RadFAO`NV~ z-Bjw-qZ({(4v9?Q_uWidGZTjRTix{ikJqnOr`Z@7WIw^Ccpb&piuI>GQ_kNx{wOTv z`0}pHn#}~%hnp2on@D77?vfZP;*BrC-!QXOU_wJ7d_)945``XXSDalyGbEtGFt)Hk z7iBRzbAXD5p_Hv@*_?S3J4&L8nG_kjm@)ef%VJwT2K}d6 zd7ainlPkSygj$Axds>)AjFZZ{wx1tk$t-Ox1B4(X`7AI1bQn{*Z?qF<7G2Zf7V3h! za@3tL7V!co?xg<4db?UIY`#^5xgU87js!3l5PV~xA{2fgqHSTRDY^Zyc7%Ogg2TjG zZM%#U%Is1?73TJ49zkKcT8u4RoExX6tXIuiI3CC!!H$-(Yk^&YX2&Q;mCN?0KDdwT z_wreh2eDS~114DAF~k5Ns6LmrbN1-qg44oBi=ljSfk{CaUBQL^o_Z zyB(5GCm$rL-xXE_Tn!Xpmlf60xycJiGZFR7aGl|FrI7;f11#6Fx|gnqnoomZ$|han zJig!au;7#6=vIIsUKA2*8VhikT9*iJ0t1Cl^K*m8q5vKV{_|Y7cxBj;;~Sc6b?}r1wDe)5x%8HKmzpR% zppUWAv95v>oM-~lDxBX-NaDhHqj5hE7YyJD!Z;!m-k;$fnS8KUnhdX|{J>n3nD=XJ zyJK?{7V9?wPLa|RVt+}pa0U$UW)mVTe0FMBG^{ zEEqMy4VI7d`t`Z7jsWB$O}(98^BRZWfMFO}fZ0>RN zyr+k;B!`9XxmfV^VT3G}$E%oI_jx~ctk_T|t#Lpl&cX2t)mbB04D@|5a{P%OQU~)U zsu4cs=DZ52fJ328Hx-KQQz3d*Cew0WR(1$52WHB5N#vLz{rS=jhlZ|9*5nrG#Sl@% zDewQ&n)kbfHE%H0Wn<9z+@NJGFImd-yK3ooF2_ujW$}tOOw8rVnmUr58wg^X=F>g# z^e7c#986-dbfTcJzT*|aORo(1wb!ei-Fi z4yKQOJ?Si6-alw+hBLL3Ad}Ax!0|LD$qP7DZpkZI~ z?i8YUO02gjF}bmRxFp6aRN;+pFa%p_e=(InWFqBBH)q!ABo-j9#lKLRFlG7+cPl9@!`Uu*We#r@sI|HY4jMq#6clXUDf2FJUKU88R3N2=9HOd` zpyq=4S(Si7irnY9oTQBJa(K6-wnta=(|&)!?l$_u1lY9v*K3d^#sLtUr2DwMQNL|| zIh>{b?^*yIf{p?8oRFBOJ3|IaZY{4cwk+>^Dg*?s1OSVLw?&jX&B)ahQLHwio~KOD zwmB+`?7wc1;_iloQI4@k&1OzBDNKs-5`T*-5oAsEqXbSd%Zmc8iz2sPk|mxhc4+G> z%IXzX^cQ69H{>*b9mWfmDNgk68fBD;&$VJRb#67Q(c6uDEK1QeNeZzi=iRq|0LC-E7Fh-?V8R=S5J7Tp+XIp=ay126VR{?NZmziQb#GT zOGQBPP{Z}mAvoR!o+Hx(Qw;I?wM6&F>^U`XO|CxXEUJ5+Z zuOMPEYVJ{`z>LQytV>50sJ)IP242yg(PrqFm35?_UA~rIrU4CO(b=2MXBntV%^iw( zr#Kg-U^9!(T}o5cCfF}^Kie~Zv3CfIL(jZpfpAQSNJ1+VVAQ^6l!oA#Phnl1q}xhm zfNJnah5we!RL?o-IlhvuT&kLbkc3sK*?4-T9?QxkE-_M^J+&a$p8UZB@IgMUB*5@1 zQsID}aGe-Nd2vAIvOqB-dhXx!8M$yt7+BELBg?u8lc!W=&F^RVruW!QbWI$bo-52b z<_4_w&K0t|(T)}bu#Clw!dbqa4^MC;1N6u`^=Ikvt16q|Ll7d&12g#pM*7!{|0w4hSg)0BmkG>xL? zIru)i+81AsC&A2{un?)`fGDlE_ECAj6wlrm@~k*?jYzq=PuQH%!>ND*g^x|s^nhi& z%Y#Uf-x-vt#BMTVHJQxl6v_@b{ggNMRyiM`bfi|tlO3blL zLQA$?lVcX|-J%>{T0I`kI0Mp+#N<;v`Cn;HKsZRX?izHpF~9tVB1Rvy-YD%dAtlXg)?6)-4W%ODdnxlpv?UA}U*nR*wcmH|CN|I>|ZW9r;atjLp7af2Y z-(A?5vwJh>deqqYaNM7~Vg;p8%@NSF#1+7nFYfHLx=EFncNl<0A;h%)d|Am(0Vl{) z>f`S}$`HbkJ$CJS8J5fsj)^spiI;vdVCSJP-kYo!p_i-MwIVHr`QqWEm9B5}eee3e zp$Et{RSja_axYSWSg7zbTmjPS9uL1~kXL8H+C<$?Ny11~(GlOgfeNjEn>FO5{GpUK81(9sw zlE5tqG#7`#Ct`bV=J?Kj#Xe)Fq=ig+eod*kCop~H^+k}Iajz`;P1AX`^9~79(fIXU zQz8dyW97=ouOFeo0lp{ve#qtC2*lj96IbE@-#r}h_l(kCtQP$*sLpPOb@WdAtHAOZ z3sUQw@pn(h(qFk)aK#X1B;0B6TyanyapH0GN_VMrUS5iy1ZHaxm6NkH5^@&QLS8%m)S0;#;=?IBq^-?(WK9obnHJY}x*P%D48IRuSn~|LT>1)6kL37fRZC6@LNQKywD08}{`;CM9vjAQjYJ+jUs`OIw_AAf#02EZ4!C5T_iW3GW+jBC-+sthDCrj>r(jBDR8utx? zcxDB4=FFYW$+xz1hN7-LuXhE5kh1obJCZv?SOPI{U>_Elq~dqQOY)xONZmSx=Z9tq zxu*bKtsp%{4z7vu>ULq%!Qo2_MW>A(5{*ndoAM}AD@i`54c;el_c}aG7l!S8*C*Rw zk2Jm4gcu}8`xj1J$-PiEt-7$r`k;~#cz0(WOSa*a6Oy&1*~Tli6G6@&r4{j9Q8H)( z;|G`BBtlLJX}`Fb(8+ff@{M=0&$u$`o#?#BPVo;RQ>8VbNk3tSn`r;prjcw8X$yk~3H>HS6c22l5d!|hZ zzsTsD8P|e)btwaGKdU97U~71AxC(YhNoIq526KKTUmfjs5%&_fS%{9n(Ym0aGG(?8 zWIjRksYTl5q?3B(fMyycx;7rWQCjJc?k|X^xXyLp005;dKT&w*ck$sh_?BCWB-;WX z_UDq3)Z{^Ly2$vnTof*GO%cQXk4B)()`cNTtda)y-Sw&~3Yl@1E&`YYH0&k8)$}cSLQ|42%|ZDGF1YqQVgM!5 z^6Qd4RRKPze$NI@X1H7q?&`D?mXTQz-NjXWuI}EoV!&~{jpJS@;F}qALt{ISG8uXQ zN^RKP6bJp(Y|AZdEZU^%)?`$Xp~=m@*gfi@wP90WED;;BL^w#8Mz_fyyY5HQ`Z32B zk_)S<(imL-J}D_rta1lR;Fpk%HGG5->>{j1O`1Z;f|kJ5Dp0on4olBNsQu~(2_;W8 zn@Ipqd7Zr#a5h-kUH^E;qM*$ibY)&FqE0&H6wuoPG33Ijw$Dwo6kx-v`>^F z_5RsbLqpBAtOiNNob3MROy%%(ck*YuuG1h90Zo=uV`mH2lkY^%&30tox3ep!!As1$ zJa>8t8vSms0W22p%)mk}6_uF6OwI4RPGIax)wO@wRpYZti9&NX967(+;eLDqLN_if zYLFtU{c?LGrTuEgwHSer>6{P=lWt&OYU7eo-Uj7woO?2hX{4E;^Q0?1jBf+Ez@<#V zL#j0I1#1YcbVW5K*I5z!{F|QDL`}W~$iXdFVjxZ~09^;Cmr4A+71o4&QYnVL7p(GJ zsk@;Xy0+j_0u5~Ru2|K^h_lYd^~fnQU}opx72`*LbIpuM;I_iWWCklI>FQ9OpgNCX zYiDX3R#Lo`+FU^0US#-_B!TrmctAZ z!ksGhp^`*0ORoxO4p4B5KaWxq!+IFSzJsN`S2UCp`38#WhNc8Jef(`Wp=??N9Mfb> z94EZy4c@OC90@M(>Gru>TE5>wPgHhcJ9RpA5$!C z2XvM@EiWapnWf3WQ3737INC5H6BGZ)wgZ^Od<3SN?vGPD{V77zCr?#^tYlk z#(mRDT!Kl|L{J7|pCAm2iyE}>J@4AZ@+;bg;;w6^br^OYIGCYT6oGrHw~U!xP)LQF zh(dL3%EwlF*IiZ`US>>_8%u#uI=ms<0$C@M%+$3!v!PLywDPycF%-h$?ut;qx+S7j z4zk8lJS!VnTCgWlWqiKVh(z}K&uc6O*zVN3D&^K0)5^AdJd%X}dL0Xw zD%(Q(x3!Y=?1vM>ffy~U0d|Zpq4fv}2$?AR6U%rt4la@NNrZn?6VDcax}u*VRdvnn zzviYUPutC+LQjoz@+Z(PB=;KkWTG#e0u6)apb)0y9DIl)%HZ~$iP6?0;4(GTxI4tn z`bWB5VJD#6H=+0w+Edn5X*iTf*XnC0Vbt=XxSlK+^$-TWg+ko+6;y-Ja=(lo#866! z>jcMjpUR;H1=30!7im8=17=|%p&_GOD=&Ttb{&P<{4`=BZUeGDDH&@Rh(dq=Ob`r7 z(6T+|PLe2HW~1rW#83O_M0fUOhgZpcAaLnuI3BIKb!Px!AduF+DpC%JE0)Dzx&qa+ z6j7bgDr963HhhYSmNPzwqv2+Qm5-z4U{`KC@9dpzAdm8oUa%?NKChs4%UF@A9jImM zXd5y=@)uOQr#ne3EDBv?o_0G4DW#m7XJ$z`aAO)aX&PwF0agepxlNsMcUO|Q!XllT z0m?{N6Cq@|(x`Nsn_13J;evS+vEur^|eh)jv%2f4u@K>yR7%sh;$Imq}26-(-&75xMoJiQgy{s$8^RK+xuTyGWK1>U z^en=&fhMB@4pMeu)^U(qNLV~Y@%~g%8&%cunThQ^=*uN}3DdZD^U&N2%@;S3?6zROO*+xhF;s9vB=eDiPO6mUm$2I)&0;E`M?_FiP=B#JLf{Mx z+qfmeGG1sgtfl;m4lt#^)P~vO2H3@_7``nfKCt6nm}03jzLu;@D`ZCv96_Niy9Uv_ z{P%If%a8ny>%yY>(WETJ~x~lZ@ zf^zunlt6`a$4=!WDaeVCW4ELRn}5@9K(a_e~9Q&@eFiT+$y0o5T=Q zG8V;6YB=wefPp|6KX=$qgD_lqaxv}d>h+C|T=<)NXqkh7n6@2HO0fkaZc^wUW(A@n zYcbo3E+F&hwM2U$D^Laj211@3c>-nACZ;kp;%}mSLXsMb0C6(+kDtmVEiEITp1ds& z@pKF(6Q`JWT=yAVVrRIyr!fn}zq)2al3g;~Ggl_DqE=`oiI67_n+{y}CvX{`n7JBW z(z*0vmX4$g2kj6STIRACoNf0pR7ecP+xx4mPfes0glMe@t;2@n5Ccnyt+NYlpqMws zp{h7DG8hi$Zxz8Mx_Wwmtg+7kR~#WG>&xHy?xx>@*T~Gq<{~ut=GHc(V)~-ylDIizu!YHm_(3bCVAm>fF*1dMoW zmNJ$EGy)@~>ppFIE-Cvq0-wVI%+yZVG?RI%K zhoBo6^Yh%jx@_QbrcIN89@08#7$;G3KfA9e$f!U&`pbG^{B)!=m!G=@a|zqcV(fOh z#?HP8oe?%$J62O_kq+;SyWE7D83182**PGl*Dph5yY;w!CWKxSTTOWjX5^U|IOUQOUzJO5-R=w^#vPYtor9ESKzK>hQ z2~7=`q7J(TrRZPQJJ|aHTME9!5zFq(e7?eRJxH6rcm$+EO%Q?Qk$w1Q5Gx@86chp1 z=@^`S(tGuMq8k8wt7Yab_e(xK$AcUkZJKNhcsr`5|B|O1 zNRpmQZ)jIj1iIKhIP0hKqx)gd7|;6yx>;(dKEZ&bvmA|AhFB*(%Yrw@()JPHgC-lq zLKHS2VfJA8X{qB;4td@nTM+^Qep>E0$_^5e;1wDwPx4tQZL{Wtl+6`NGn`eJ`FSf( zh}ZhZIxI?40=|-{t25Hj{Od70_--oR))tUkQP?iW>HSE-)!7`nGa&fTPhWTVX*YtX zV38jg8lK?LgMhZZ8r!xj+D@SJ1+Yf6&vWuoPur=a**i9)4TQ&~K_bwZRlid_(zga? z>&w6mv3ttecDotHu9=+hRRA5}%SZuw|k@C$Z9FaQ5Z)z>g8wnC&R&NL1R0~Q6BXeE6 zPebC+D4y}nJld*DnU=O+Q$sTlmG8Pfbkh?PzZfub|10aTh+J(V%^;<=`IAfR6(H57 zbxkOe#YxpeJX9q6$<>7>kJA~MjZ%a_CnPo$EDryJJ)qa>?}0xA%->lKyak{ST&tLAhn4d>UTCo1^Y zpQISTQ}Kr5l*L7fIeYE_kZDsPyZt%$O)N!=zDja84l!RC*xxo8 zu2)u%fMF#i8MF1B;ED!=)tYJ4^(N6t48!U62?qS~u!nRbb<|A9=O_6M4LPZ0S?WWFO}GT)WZ|E&yDGKk z7FXoVltN5}M^2HXVcI&k>TpnyxrqYfSw^ZGM4OlQV@X3HqhF6y62hlAtT@Z+tM z;Zqk=+|)5GZqq5^Ooj?$S>YYKeHZ`syJMX5`*|kH`XJF*0>-Hh8IN3<`l3DSgg6E&?Ebb9neWOZO}{ZnhW}!^!E( zorp^jc@L`5n{^z8ISDXsmt-Sz=@abyMx&uWs3A^d>hs4L?)7Ux?0l)$TN?=jIrAjA zA)AdN2FYTK_y?xbWr4>H2ujt?*H>N?p;Svz8}}lOw5_fYIj(h!aaITqif{ z^B7XYVLOl=%!I&)ddGm4BP`1LONw(x;zKut3D;>Vc1INvj^@ zHS#)?o7R+~XH~41jhr&X6hFArbl95G5J@j==2!IhPqe2Qt*G4V<*cDFSXeAR7HEna z`b=zB`GU|(YHhAHLf`QqG*C#*BiTeldz5;$P*eEpXvk4dZs6=z@NRielg8KlkXBcx z`MVqDjM?D20?G0d3^g>v%RWKgl=@m7W<-7;vH6?y?(+b9F&iK2l4cK-LKbM%=*LtNn(s%pg5y z-k3Ul>MV0qs>Jqp?#Zu18!P4@GJ{h%^C7!oNF0THo5`5BH+|}R%ovgHw zc?liE)9*GxqT%ubZyxykZ|Je4+dJ@dvQk3-49cBOcTGKPD@%rqlrH5eAuEr{3|@CjZD?rN`R|ENOW?r6izHWm1Tlr4I-+vuB~GX6Mlu(wiZ`TdYaX`vMu=NQ7#MeKx)(qW%nl zN3Zq!izSx_NE%b0%R$KBiI$?b7wJtcr?l;NpqykHfn^JUDHayOYGZ4Hjl?vOZ6I8n;)7T#NLqlR72BN2VDBy26pn9Qh zX4oT79ncY6?cF|IJwZz*EjE6sE@|Njn+8=oCw|w5rgZkgZhUZ0x&k4{6t<##ykH;m zf@wi)pO>P?>T^H>5al}a7z#NILgvwjG5h&L&ss#u3V||upeRP3h%6^z-_Ael=KPzt zdEr{R;#j9C>{ohJLlMhW*dAbjY&v1NIoOL)taoqwh>9ka9bvHI`xB~>DS_jQKOd`c zNJRy3=X17mZ)lh#*9IVq|JYt}*Tn~6s1O($%a=`o=}rYBOPt-!J|>9}++fex8bOEs zXh4?8BPSRZivcx{UDVc<7Fz;?h6)P{8>*1h*h>h=_`bvOA%7?86DE4wY`k??xTGI( zSB?^l0UU%@f(gAxA@wv@GN(Fi6-h-~G4t2}vbbFiSYD!TYaUW2ZFTbNACw9G6c`Fv zcOZ;!7anFvbakt6wgqI|qQ_|Bb%2C#*pxKwOrlzD_;#yy&O}6EYbsP^eU$b^7gU=w zl752nfLy?DzNgS6-qsgDqDv>72;fQ@68YM&7B4t)-1Tty^Q8&nh5&+XIV*|PqoG>Y zi(g)BrG;Pq1HM#1RuFT*49T9mbBA$NySj+uP=Z42_^}YedUWF7E?i7((Wn+t7lu2n zzs)E?)#3ctVCH2i%9D6VJz5n}H_oXs+TeoGRh z&BsjSQ(`{$GrTCXet??aqr*cqA)+V_6Cfz{J0QxWa&!+*^Ae2RGNW>E{@TCLR8!Ex z5JW4g2jxj2uQdqu`P!7k=6H%Nwl9VVAcwK1xGOvq2ymODK{7Hj2ZpXzo+XCpP$>Z5jAr!#?N!P{t zu7!C^cx)u%UmD!df2XuC9DJM!1F4_~AnmfVK>@WsA{e}}mz$&XfTPy-`i{`CYAOnS z7Kt*G$fh}8Ka$9DZ-~{lXB-ycD_L7-28zt~j_0CLp^+L3%FkdA9^rlK-BD1*j>ANQ zoG6l>aMnh%VoFq^>?Cx!G1yU$>>-j!qydewxtq#~AMt~blPq+TCoBZX(4Fuw<3?y8 z`h$}Q0}UHp@jk?&FjAHun-IERHv0!gh;IObVgkYX7p3_2k3Ik2T7PJb4s+tf0R$qX zQ};GPHFRQu#`^SHO`TvL%U{MH9?Xb`~M9CKiSwLLhW(08Uv_KhC zZN_9KJaub=u)u0N1b@j@2>&zd2hFf6VoP9mO6uXtA0S}`I;N*X9Po8Nky7Mg^^x2M z2HCk`YAm(nb&k`+Vz*1PuNdbFG%J1xWFH4bL4s(SWtz+Ys3~@KRMNgU z*(}$#=zrgwNt?MnVqdA)W=cr|2^!cCAoPLM)ytF1b$G?+IZ&Vtv0v5zpV|~x>M_L! z*p=Ls{_VDYh7`@cPle_jDJ8Pk`53`ysyew`_TPU?$>`xSHuti7z_8KPjaw8snpy{W z^4sVqPvF`!d<=9XfK+bZ&Jj1&=e6d~+$L4hvlveeJDI_RaEgIJky5dGiK$#jiAg%7 zf4lhPC@JQ#2?TD#5pI-_?KuEVihM*QjhJ;h8hpFx&QHp7qytx2$O@q<4v1oR^4e&y zrq$U;U=W4b6xao&Eq}^zT@k06afhB?z%;N1d@2?%Hl@V04hyt_3|Vd!cf7fm+a2a; z<=^L^GEGd0_=A=pWN~c^Vxjrp)6b@^W5ONBMP}ee8R(tinNH9-;Ix=HJaC_nC`b0+ zDUvD&aOA*2h2|wBC(0N16Vrr<9>d}4-$LSwLt%5nQlKe{rlYf{j0gUlL6n|!&7S{NB~tg_VPv8bE_vMzm24y&|;%Q zWc(+2S7eBOVKQjo*WZin+@CV^i0BcUr;k5Xwk9K6qn6aiV{7j^Jg>L8WX6_N*Itc; z-^Pe_b+JW{1nq31eY*ePlf-Tf20Weld`Aif3&jKH5$3}d($qzb5W~9 zw)x_BMDF#tZ2NZm?j^!uWLgLG4iV^nNqcm9Lh7miH6oB!1PlnXBJav`a$1=>6GvNf zpKZT?UH85Ht2>|BTkbN;G)ZbTk!sjz@J`rn+F2Om^py8e z7Os0d1e0Xwz!(QT_@9Y5@8O=qkOZYM9gRG#ZFPsjkMzR`(n3YBXNP3*T}`xXjH6cC z&9*KqE0TC8c3iZ^N$ly`d#0a?+><|zF6O|HLWS~ad3z@wk#~ms!*7*x2;X)Y8?X(}7OjpIB)QhK{ydhYyM{B2(<$!l9hrk35Eb5hZf08-jahZ@ zo7icuq_P7ML@DrVWZ+?QA_-oaLBP-5p*up~sXI072R0;x8C#`OO*I2&aLOdV8 z?65mbX9i_8OR|qR#_DI>V|3BUCK zF!efd^mRv1W9CXX-Ba4-S;4tsPk&8y4UNE+;Di79>5E-O#8wyWWFLG_--e8$@NGwc zG|#-?9R!K6#q-zoQC?5xkKxK}-$Uh#Ev=o`yjO|#7i9ak2cwIp)bPjmHxpZ5ezrtd zWx+et?uQp(4Ef>RinsEa*LQ)77qqqKN^d?9xq3nGWSEg)H!r51`zP=WPF3w?M91SVkyoZ_krxtT@uc+!a7&V|H*vt#DXBTc%iGf9DG=Cu z={i45q}a!VW2NGJxzsbEc-!$LA~HX5Q~q<^<%rL#u`MI6CfYUD-20h6&i_&juGFxT z)g~!RXsRJ@b7ZOK3DAAPYcyqPehIzab$}2!b=q1c8+CA!Wbd~x@G?QVR<_!hnKtU0 z%O#V05a@HUpC63udx*EWB5(6$<3X#tS=)Jjsz*#>Jlr7L{PGuz@7*Q-0Wn-x z#!m+^3@A98lGx$od)4G5RES9dVY~3xzyo%wk>=Ha}SrR9f7t|bpP{*w`}PLDuJE{t+$;#YVLp?Yyc`N!a=AQ zS?eq7vw_{yI=Nr0s&|kT`3qdY1NdY2)kurlY1lX<1}1og+U@nX&t$-8%^L^bhaIBRHzif5*ENMp!m~uA?-K zNUXp~=+1w0&^fc;KOiCX09DYzZsp7T+#{jm|D^>b9SsJ)wl1xY!`J95=!A!jh7$_k zNX4Q@C;jxSeR!lIcKXYnrTy#21W~%sl(BG6Tqv;~A;^W`rwcW?|0S)uFzN5zrm1#tv$UFUQVlyxMEy!Fz)5H<2i z5pKSFH+QiQme@twiwI6O<-I~_*IZM5J639O{`0uS0ZFSFIs0UDJl&Cr_h4+=k!ruzk4jL529*6V7R(o>*bm* z)rqZc-j4Xb470)6TpArXU5$GWQ}b{j<(x z!s>ik)}1GQ8f*|sK}XJyv|dj8tOU@8_q3(OZ9F6Lo5G32RN)+Nxrm?(x_rwh5Ch|q z?+jc*?!5km3V4b`uY2D|*$cS$d^sfdO<(!xnW$V<;A|~0zp3=%1R-0*3%fq~3!2EX zNMK7z@Tt=aLYDU646JJUPADDO-Yn@~*)&qg(im>W-&*gDum#yFRD7yHONWZNKQa9w$;YtK6TSZ#;D3r_SVC!x8HEJ=b|6VoEmA zM`Rs&GG98IVS*YLV~9QfuzIjZOKj?c?7x`umuQf(Y$-M47X zIXWfRk?1Z>wO(z-34ZIIm(V0+Ocg!2BL z-G%%kx%4N=o<90LO+4-y`{R=e2X5#LEhLB>gbIzcR`ZE1Z>K2oEwXGtAW37#bQ6^y zxYxQDMkLXGXB(&=KIfTl^)tR*?JdopW6$V;?YD$;R~JX_?9>MpaW(}f=96@IPQ4zq zh^>(j`hUTI&mYnDPH4Z;(j8^&bwkG*$Tk*}ct?9a-v5+vUFJw?2- zOtf=e|9355e9OsgwyQoXLl`@hPiimB)5XJ6=qqTT0erE&kYVHhX{iygyp5^KdJ8Va z*P&l~IeYPSl^{*?#(#SGh3RjZ%j;;}n(NpSb=L3Sj3PLN;nZcYva6m@sbf&~uX|vV zlePNq)O+0fTb4G)oVOY%&w^Ed*j<2O8ZDmn(WWXgA6e8*ZGJt-%{r(rG3u`8k6*^q(d#+!a!bSZ90OM17AiKGfdD*w%U8o1!ru*}m`=(#E zr`AjkemPAAL5e91qc|%*m$(#^(IJx)LJ#XgU?x=zgX2=Y{I?$C_xvVBFDH1?n)cX3 zn*;;J(b%Kayw?cht<_-en(0vHdsd&gn2{ucg-fJel$42JMa9<(!TWvcKeiYnHp=P0 z=cqL+IJ~mu?=OU;zq~$6j%DZ5%(SX8%J5y3_OB)935tYavIxm$W0vvvkIAD-EHSJ! z!7#LQ5MC3NjLMN#t$)XZ3r$Iv}y z?-tiwouxNH-aXRvy4K)%X#Ms!EIz)j`{#5u=ueU2SHs1euP*#j6PE0b{o)J5h9}GQ z!e38pjUM;N(+zQ^zU6Draz4X230)H;@#XcRky%_!6nb9|K;!)B3as>L4s)3II`hP4 zKoe_iPxh!Ry_!31aOOKzepsh4%X_4Wd-EXaxTF1Zx|tlamwObGkyYj69}LCJ%@h;P}~QD=VU|%(@d7 zCo+&2>;ZgS;eD6b%xzCT{Gna}BE=yfwFHNRK$QyX# zQ5{dj+_GY}2CCTqPE`XYWZ4zY&%coUv=EH6{BBriM$$ zY^QdR@BBn>Q8fd-&`AOCZhD<^x2yf9oGUq@-gygce7@eDg}AvxP)srme3VqweI2`F zN29%}6;xM@wIo8<2EfE$U9Rs2ONXzo?pVBjM{2egOW!n&MYt<=2Z|xWaou0iaorMC zkJ(B@4?g@xUoz~w?-012RSKPaR-;6|JRTfm;vChYAtP81Y~tfR9Oo9MoagKUFYkB=tMGl*w7&miH&6oM1 zDyJJ0OoSgePWYuZ>>Rn2qmVoD)X_1FC|d=%qR^p{e(*^! zyO@zb%v8;Yw6}`Eqz~SWqKB@Fh_2p-7(Zq6iFPiqH}KV9H2=BRbTdO+{78VDitP(J zRvNnQtaAqsgex@3^d^=WDo0QFBM)6D(ZX1;_Bws#srnD{tE)+RiW3Pi$8pi0x+OW>bW^xKldBZr6 zIpnaE=6Jn)$Mt^Z2XtrdC={9XoB|JK;8-!xvdvGjGe{Jf_BZ?8?4Lnk^8({CCA@pYL}IQunAquX zU;pxEX^XA@`xFJ8K-Sl`JL|bl+!Z-a%dxK|Z2aRX=z?P&=ZR`3ihI%Yxn4HH#0p5G zJEr31_uUy;@a{4R10=Z*f(kD&-e(cvHfIOeLbA$l@FTDAIzTJ_6x5o-9m?|=fc0t_ zM;|_!Y2S-S5Wf!d&_=084-Jv&})3 zueuB$xv@}5p0qG`b1K31I~-u8VE#&mAeS6*=c|3*J#wstyV?HTc|wd( zCp1}1ZU>?C)9^olPiPHpZxi+vlcs})+8QFt;nE;U`1f(+EyKy$Puf&Ne1|d*Eu{)c zI5>!_MUb+&Yq&VtVQS4RJB8yg{x5&S>1Ai~^|>tHuJIK?XgVVqB0GLX_k{#JWgM9> z-gmADFI!H!o#sp|PTvPOOh+$;v-7nN&14Y}5PSpdp*|9WKxF9eRC33xq?Nx^C3xgsZJ}i> zr-bz;=Wx4;vUj%A{1k32(8$JM6b$C{EyroD1~X~UUX8>!+Nx+KX#zuNBQPn(iVj)| zomVD0pI_z&$9YKV5?h}FX(eS1IL)P~`+vG*Vwdc0+iqrdX2Jh-yf`eFFuS^Vn59ezz!1lhJ{>2ZZoe{eSTWEIcp;3te2v79b+_b zgpOO=W|oK&%@3+)Od7>LfQch1t@uIi`BD#GOnk`oRBTlq^8)SZt|+x5Qn=XL0W5A8 zm)_R@Jj2c#YTlAZb2|ZCaTif83~QW2iB%oZ);K0D-%F=N!^?Q&iTdTjss}yM+6v#! zU7R(s`^^S>YN@ni-JF)r?3b@-Mq8GWr4#?~$Sw$p4%*jgyWjY(1^IQO* zv)UUeCvyDym{C|lIt`?hfZNH6+d`Eui)56h?D_?-i)%bxgK}qxvL742Du^$ZDBBF{GqivSZ!-BvYxQK1ruuF*$!njdB%x2<*dcrs@kD z*WHw`CfpE!A{F$?h6W7;g61yXV z(}8B}Rc3}mBV7HBMTU3Ms_#t}Zflh=Pd`6|98NCLX66OSvilVeOr|HT94y^<;tcDj zSB#zSrW+;eFcyP;IA?MB<(>@Kf4wK8BYcZ$$+|htQdnK+GUPA6q02p`Bz}OCQWR~l z8)I`uf1NZiR6O`sbQhZk$3ZB1;|JSa*GrUmR-kA9n1kT0GH4`LB4X6(gT zQ9(8K%Fl3TW4mru>8Q=tBc**6m7jD>OyRsa9kVqmXliiT`&mo zY*27gr_jUJ()iYmEr<8Hd@vl!!BPl;3UP=%Uz?5<8X>stS6gQ+eNs8g?}U+0&N=}e z6CuEYyv;?1hTvD)U7>M#;LEB|ovV%sY%RHno;8&<_H5La;~BUD$FY9lXsfGp=UTN8 zW$!MV#Nv`t^L`~?v?Q@6s(vIuLHzhJP$7752`@o5ywb_aI$Tkc0;jCN>sBV3^ns{O zH{i352AhZ=7SubY{$W>NfAFk<&-;0!;w-S$kx~Os(bDlN(!gpFT+^_NllFi;fo|f0c>-KM@p3Dt&%AbQvnc{?3gT0C~4JS&tOeTqoN?(GS%ZbwP z!Xu$8z5X7!;Lts6Z`$c$_esGhr2l!h^H_yDN;2jY>kftVUy_}&Ru$po>b&6fi#633E@8*jPRxz4hJP%J~ zrgjG1V3`#C0EyX>CaFHsQ%Fj(n;o!N~@#nQU?;T+OAgQDDpt= zp=-pE`T7}273Y?ieOiHW2rp3mzFEz!NdtA|Cg)5N^j8V$r%DN#|; zZGi)fYT?Pwx|-|zOo|r47i9i}pD zio%qJ^os?V#O#l9;t7dlrF_Ky%f}|!SJGaUoV24Q*lNj#{@Tvq_Vz%->lUdz@QeEC z`rVy6mCaPGafdq9K_@N%T%&uP3A3Xok49#4KrNDQy&SXdrKRu7n8r(|_`?epEOd>O zZ>jBa6!)BTI!{qnz{offDZH$l~-T5#KVnNZ8cZmvEVE(;pn~3`pm5$b{Mr#Z#pbcrHB6a z%TLKP``?U^+F;r%ZF}tKZwK0qF&a-b0J(+BuS`8-zbC_D6m)@3L{U~#1yA1+);%Tg zcQ-iGsOd&|ptCnxOZL2&X28P{&4*u|HAGXht-cOG$lVo{UFj z1fp#_$L;}vz6lsjxF#fd#3?GF5@P*<9~O5oSm5LK51S*h!QZ24XQ#B~t$EXB$eqSh zra#K*W4@{MaUo zeqF=be22bWBDJ}!yre*LuCa>Vz~*6_J$+zq;qy$D5mqUn)Y{(Y53!|_i1ONkCM%&b z!Fyk^;KTkYkfr*tdgpcYzT80ZaTHB>`OtZayt>39HWp^^PfzB?>3jdLa~bQT^MP$& zw4I9JN$*pwVW-5lyZrZ^NH`7xB-zZU&pC1qh8Qy-8tX;OF9i`X>fbqktHuBLVTV-8 zkYLU?Hl*kBy`Eq8p28WV9#4ZSw=0OG7d`j{klVclaLeubCIa$LeOc@tNJ9rD9e1-5 z5@yUZZW-#)b>}{cY|+Ujy4WC*Nzk_I!sscH-Ui_*wIT?N1`>q>jfs|yv?<*3>*Yma z^^eA1n0d8ieHplVf7OZ?X1hDq#!+4)N>OnNgn6x*yLzW%V6iGmCwzeKZ&so>a|ZpF zFh_$`xj5QbGWvWmt4QYSivw$;mVKaHCev7#{wgz0KdC5H2aR*Nra(a*8jS#3SrQUA z($k++N$U8tnDnRY@}R1$RWY#(Z7quNSc?tM`Ra1BA5AEB`s5nM^XYq3+{ivQhZ+F~ zXhFqex%&vC?6NmL-SYyYJ=B(=n3FYcbrc+ z?yd%C;3`3%E#X@KLDj5h$xeRr_8jqs%_7U|*$MgBcggS1^sSm;09)Plbt`ZvN5Sje80jtrUdk zliEG7ZYUhK&Ylcs6#A~G9A6k7S+{r7defeYLhwbP;|LNt+A@|`+k)ENP&4qik9Ei$ z#%5#9UwRcWQ}t(z1W;DO&0a?Uo(kxj#7DcmOkeDPntBq2$Fk=372~n$8d>ra0KOZ! zh?`9L3MK3r1npFpA0DPLl=65FX}{R@{wpjyeOk5a=h8VL3*~VxF@->F&gnh zY_ySkhhk2GV8AUd^EeuL(XS+_xXRlTk2{Ct>bz0NFf2~=GnOKhhI@T%6pJqn5DriptT4{3X`a&&wv zANn=ZE=tShhK<`(Z09@H^vzjK?ciIH3Wm@?=~QmF|!>|#ulpdE})%VG%@tb|{_CEjs) zABO4{-FM`5gf;e+-rUh6F`7(%MwkH&p@rAl!>0i#T__vu`I3zMB0>-D9#{6LZF0|c zm{m-Ls1|u}NLflLga}uv_pWk-qF3XA)jEfyhn41ZAa6RXe$m zN6oUs5AQ7^3C7%VfFYzpjD6%4F*0MHM7q_Y*%mpNM3it6=YBLah*6=23D@FjtN1qr zL2rbxYPmTa(+%DAe;+H;ANCUkkfz6Ez zo>HcSW*ye-`g)}CC9{9(59x4l!- z|3%zeM#cFwi-W-(g1Zk8+})i)gN5L3!QI{6-QC?axI4k!A;I0*$?v`Ieedr5f7x^P zoUJe2)AP)9KV98b)m7D1VZY`_CS}NV6!zYKVqV`#i46ea(Gy0vRoMsH5tV)~Q1ScC zBiCe%7pEigxLArMj!_Kc-H(`1b~0y+c1LSz>zX_av1jI+L7yO{EB@5hWD!Um5;QfQ zZ&9Qh=9t*snKpAIX*@*g-E3X`Ha!#dj<$dhtnn-h^p& zXDfDpLBl$4KCv}aHc(=DT-k!>(%mi%k(m2IDgXAlOAX=GL(pLCM_RVvI29>Rbl|b; z6gB#2sbwboep8(LQTK59bI9J2)$Yq4yvha7f%mJ*hHmbjCFz@$=_Yaygo-V>cxcl5Ep z(S4QX$YDK8C%mo@RYNYT`o+)DRu_EcS})4H{DTRNi;@LG!Zxy+NY$Fo;bj8sjf|pQ z!TNwOn<%Q2Fg+Q_Sl}!R>y@n&D)8<-a1{o3L03x;ms|I8pe7c$O}CJOe-ERSTiiE@H&v_1tLGo3WRgN>J=j z06F$hV>-8wV$QXH^!i0z0z{P1(6$;m)Upxie1?Srpxz6 zx~(d;HKX8{P1r*?=?!*f&4Qb%LV8hTwD!)~Vq7DEdq>QT7972z<7r)kg_CEKVT-7b z9D{T1hUnj>Gx}~JGn`ihkgGEK`^b6tpH0Yuek`|}Fn64~-_lM{w5SV={f6q~q)qNV zyJnJEyg5G&VEfIPnpxhX6Bv2@pdaBJ5{r55o=!GWd!T z_s6AxWNCaTt>$`jRl^<81|-+>T7+{?rX))`%QDd0vth~SNTgR_cb89gwrqcv>$oH1 z=#%um;gI!l0^2h+wUg^miMCN62`s|vdM}~3gVky-r>>?h9;=qL*TG;1y5X2`d-P#p zV;gyChn^Ef4d7&PwMcxy(Zk-VO3xk54yVy@2c6)!Fj_Ijq~B77yInA;6fSSqMP1hp zZakw!s*7VkS)v5DwX$q#l+5evu>g56a<;)E776oRbO^HeXub<~|JpIok&)j2J~1%bVa1 z0;Q8ufN9AeU+6E(-u;sSl(O z=-^h{bfn_fDB9xyRChXI^gfg@A1V;*2dl(q==6>#LX6zD$3=k$->vm(vg+=Haohc3 zw3O_4l|}yHNvh^9dEVZVe3d|XIT6CX{DV%r+9MW>o+Y%o=EyI7G&1WMO@?wzPfUqx z+dA0ELMRZCV(o}kUp{ob`7gyG;(YS5*$>~VVLhC#sO{rGHK%XAe$Hc)xgt^Hq~J7X zG5okzE7XnJ7l99p>aahO^B(Gc5Ky4Y_OJFB`N5kS7XS(yS#Bg1Q`O1+ChmPBj1jgX zXEtAqjnp!A=qiph@`?{Is7Jtcv!J->gaw4LB>cVx>js;#s9o41jlaU#k z&s{-RpUq>CD-};)u6^Cg8X2GF4=#|{mb2=Ud(e`s03^xyqwmiovsbP;rtW2AmA@!d71pls&JTj-}0Uu0>f7H!p`}X-;V6V_1Psb5*Du+f8dTkJ%0qRjQ+*; z;gV-F^Hsr{b)1xiC+nX4e7oG_3RY_|`fPn<6*}se#SAA+AEhE?($Lh2PNWbllMG2*4TtFl zQ8jR?#1JE=cE#fqvf-Z6!9kxt*m`AzHxJf&B$T{AmV4+xQd&yN_OJEnojbb!iC}n# zZrF}1XOAn>=KgdA0}Jcgs=(f9S#*h4>A;7s<5Sz%0^ESwI??WuaSN_mcGzDqNBF>?v03W zLTrH^4h(THPUl; z4}j}G6HWr7io}K03nDZU@3+PMg2uH&;^art+#dvh+F+Vd#T~AzlSQY+fEMp(a#T^@ zmEq?2HZ#!JCsiMAjbj!*aKxqm`!^k13WDlk{%LHO`Dp7p9$F4|vn-va$dZ#094yT8 z?Kxz03$D;~rYnO}v#~)irWONc*v$&$8yM62arl6FbQC5VTyP&(bYT7oPJ1w&-AaR~ z@PuAytrY}XcyZnzp6~(OXwfei0G!6!xLHuF5#>05d_zB$Fssx^+>9gPARKBk8=@+i z8FVcsL`t!3v>`g1T;*3BBh3B-ktFWPI?+_HgxE8ZQTX;95oLgUUDYSQ&o~U&;RB{z zf&P@jw1H>7mA^qw?_^*g^~|Dvu#gL`UK&@qUmkVl!Lq);FgmRfIS@WQub=|HWLY0! z9ucisSdraPn-T+-4A+`XiCpl?BkZ5c~RAbmn^)iw);h#ya1xSxzN z*SKBZ_g)fV+DO1?X}e!FwUjSmGilo*bq3IbeLigC}m$HDY;whg%*qQkBZ3J3;kO>il9R~vZ zhN&212O>E21tv!%@#IHgdrNm3pmOrS*Y5E`7+FsV1?tXCxIo!PR*qXk5tO zn0$l6HJ4%mwx>+O^8@f}QuY3SkvoBl+>BYsV79{r!h_*4Ui|rjT9H89S0fvz8Nf zvMi_1n07bTW5r5(kE<8L#g4^rRF_Nw5Db&yG{ZXN&1y@cS4rMjjrlwo%0C8p#_1yc zkntFEb3V~YPLydJGn+u?_fjER?cIoYYn-;~iCJ^~Y6UZle6<8i@6 znS6H8#*YaoIl6v150jE?vD=bP@bGBN4Qk7I!6G~e-PyZma9hW~+;4kuQC=!0{V7|I z!)=OkC5Lw{o7~TM2y!yG`rL)jzQ5y4)NHdn5`YhWTt0Cb9fcEjNE#TJHwsBRj>F@C z@*X>BI@r$Mys;7FlVYC!_~yO4yZhKI1qTO*{MSoD@W03n$clU9KoD{>na!Wk*C*1A z1iJ4UC_>t)!w9~W)=#@>stQE*sxkKiov`Oj)y@ou9+7_iEJu-xC;MwV|1Wq%Tq{2) zFi&94&=KE~n3aNk?IGlMPXwXlMy{x^FxKZx>bwl(dKuO-{g0d zXUQ>eV^8IRj%Z4an8US1P|EJiOPD9pP$vt^jt+zmhIvr*jkkfp!7HnObX}hx6>dz} z@@nlkh~WIj19uBnd}*bkZb16Vf5WA?QBo&X)}@I!If#{#*gh%$Gf>d)iVt?gw^_(M zc1N!M{V=G!G&UL}sq$v{J1PEwuvzHuB!ZAv)>$%1;lBSHdRnBGobcGpdN?t$u_Y_> z=IDkdIg@@bnh_8KTPnG_%LZVxB$%U0;lEwUnb!}soJ}p_@^x7f4_gET5{7(8gahKX zXWJ@B5@eDRV`I1f_W$;G`aw)UJ$lbp0ECsT)0GBlI=cO}ugc0GTGsdHj_iL7_n+z| zF*fW99UYCKnmT>*+2xj00>!lfzvFatf6>@$2Pk#+#REDv_Hc{%Un9f+f^DT8CFAEK z${9gUkO&+V`M=-tZ%a>!5#;|F1d9F+=#m(XPG|HN6pfb+O%H+!u|_Yi;h|mRw6y*u zKj$o;{1G@vIi{Jy?A0qRZDAab-)yRva}SXGk%`4OI>tS(-u*`6%^iH0^)g$+gsr`M zLy(aGv&iCcZ*Z^1^UG#FWK%l>y}(YRwK`Ky=CIaw!<)_p+^YS)#L%0^I*1r!m~XWG z$->TFbEaJ%yxu9kiFm@Ow-_QdQ-QonQ4Jc&TA_0aitfX1aUyG~o=_6jjM?#&J&-n? zqCmzMJ-^coJpzl)#MGQ#@nrI3rGpnKfjo|)J5klcpMv%q+I%ZMy)u8MzOrg#wt53r zU5hkXl^sm==}NFzOz?kCS5FFWy*|xHYF(N;A+&>B%3&+}RNb2~T^4ItOtXZ;8-!Zi z*QNw@Wz*Uke}~3W$aIYRe3ut3Ei}+lGYuLW|B}O$Z6h5OGrUwlz1S=6sr*C4VUgI* z;c{EJy9|^|JRvzb7=#9uzgXxt1S!m(FEw=BoY~g;L{IFM!E^*j$4j$e$+-IMzzH@x z{U)g(w5YkYdN@{Yu6SSH&Wr(A_qpo!zIO;)+i~=gu7`1dLK|Y3j}8Qxy(=VAbb<<1 zEe|$6UI<}mh>ofSR?N__OKG*Ia)^B}D}Yq1M!}DdXGbfV?l3M}GudX~bUC+F@6Dy@ z_uB2TM9Wi;vp&U+zr3_O#2MNLP<%zXa6kdWnO_fVc#j{Xkm>O$Ng$&0_O8WpZG#Q7 z^bSs=GH;voWBYi!dlAS ztV-bSw5N-)qCa#I#FzG`{7V4-1($;~;T%kcLoluK1G8mvqAXs}rm8v1{r=R3X-ysU z8#WgTMo+@uDz#$LHHF%zp_YFoT5nIz%BQ4)RA^h7283epEz-4euX++i&nKTa67ZK zQAC1YWds+Q%7yV@uwY@`^W}^!hM7yZ^qT4B$eJYYqU+UCO2!cvS|@J2ALSzn#RV~6 z_mRA~GOqRQWNU_c&H)w>N<{JaL{I;t})87u>3;tbP7AMZp1fXpPz+&N3x9?2^e_cLr&nC%m8B8eC3(}_Nv1C{w2 zDhhZija^D@*u?z$SjL-t=3?; zhJ5|JJ~eH9+LKZz%1)Oh5LZ6Har+}Knnt*^J~e25xZc%;I~q%TdRSN765AZSitV=IamJ5VvM=1K17JYfY&w zCWQH0m%_guY&FihDI&92vP2j3JRESnH<;2XsWT~oVq#BBbP={(Vp`t1!AJ~%&22VS z*2`ARiQFK=g*t}?I!gLgu_ki}uzRazX>}vXbB{{~c;zP*rk+`ikW2~gk<76>xfQt;;k?)E7oplX3{$ffeEEY!2YTtI-Td zzvruM7^m!H@2j6f&rFcL{&<8&TTgDw3Sjc|?zYj+*2Bc(A`z_jwgNBmInNckOVY=9 zLe>+xrsmjC1mq4z!e}hnLj=?|dNeO&_ft@!CCMVGn&|s$EF}{eu0%hl2!?i~i+#2r zHohR}j@=XN9lkl*nFe{p2HRyG2-ap^U0jft?2)k4ud;rLaImb%R3)w&O!9`P+k{2q zMJ_}&XB?~l*mESR|lwD>N?Gxxz5(kU1Y zpcIUquP>6R+uyhc;T9Az=QBgj|A>Ji>Vh_d9jCRh5d>pXj9gK*Itx(EiPHXgSmilD zD>8W&W{_J%?+kID@}c0!k-X7{o|K_ldrc&=fBZZ50~cxwa-lnp?X(J`T(MojDEiS!lZD*GX%Pl7D!Q z_csetSV_ek^kloD*c411^in_u2t(LIU!5<%m$ri?VSy1iY9G+S!Kp*j5|CkVjb3vr zuNW53eym!%DpVUj!&E%ES|MC3X`mO4zP7M>h5^97ttfV<_J@gPd~YL{T2edvCmP% ztuqFq+)iR?E0=s(^_~dosNb$&^N#{D-DgWnT#Zcf;1s?<<9cAkK8)N=B*A z-NyPaYVMkHy|RXpeHYs9el|s4MWP9d3i8GSUV5G4A&qikVrt@k6(nZ zq1^Zz>z8XV$7$F>`;DZW`g96itReh___H+B@x*+?huhDCnYZi(Zt`UCIq?`sDSaiG zOi68)nuvr>vHoNqZ^m!pm_>JqcOh=%245 zbOFp~0fQKXi)I3klm;#DX(=uEBOrIh{{k%)F|Spp(4DNJ44h?>sKJ|<*}G;>twSP) z&spLRV4^3P=;P|5AVZCsPiF7jR1hL4l>cAFJwj_N8)N+GCuBVc>%?y?r#obwBjI3N zzDUa?9C1oW^c7y^|R zOml=n+H!{#LpAbnv(S#R^QRL3n%)@PT|VSIJcK_0aByr=au5T)^s2Ams(pfuLrRaY zb|B>y%laAYYrGd&uKv=i*~wHDXmQ>V^A44UBma4x4Q>V5dBJ+q?Y)LBWz}Jte-tcYR$oJ?(y}}B`riACqe;P!sIF}JNCNi2 zp(~zZa{~=h%i*2n7^tv%vAv#gXDCl7iFtH=Hm0So*no zV5y1$6J7pA^s^%3NPL$tNqs-8Q7mNr(!t?PC%CwfVr|X0+Mm-@ut=a0Z0&G$AA)0W zf2!&Ywj2}s_WMfZv9*@}aU)4egEpQL3rGyr{UZ^Jp%1w{z1N6IZ-zpj{kJU#+uXL8 z&^CDE-zD*ynTdq4xHwyG#l~`EPrNZSukcFP&pz#-;7;Lob6t8}RBLdE@l7TZV1DP= zaW1fnGD&6g28Ni1yy@eHQUWDWVmrAU)>y6HlPW1_v}d8J|GQy2OJ zo0RobWEAs=d1e5?RHnva{Rmy$^+5N+X@pk(t^VaJxmr|r0AvRg*sr}o3=3ID6ekk9 z>T>pkwu3AY?~hX%$QqB7HeTtl(bdtH1Q&Y^J!!QelO>bHnLu^2YWzu&1hpn$-b657Jbh9^9zEyC%ndzREHrrpRe(IWQ87;p zC(}~kYtLWlQOvOJ`#Xq7^cL{A3w`Y4M$I7PXh_ z^uO9`N6g|A;%jk@*phAQS?6JqW^&m>66QIe8g~1N;0pib&LVb`K~!J?)MZrcJ(S4l zS~D5ggzIkGo;BQCB8CeOe8K@hvs|1wcLl0GBl@;ps`%i$zFf22M6IDld9bStR}^S9EJfYmFB@;oDzX`(Lo#0j!T? zXY*?uz?pDI0&CXmOJF%(6$lb4zyb{iGB_bVTP#;NWY&^&ged%Exh*8s(s^Frs@ADya5nR56>!>iPAx`Wu61$CbWD8(dlP z{s93`bt8Yb^Zx-d|Dn+TIVJsf%{KTya>{>eEB;n`a5es)WBU8qTdckG%dU=R8nN`; z$qb}MZJ^POD3AvsVAuHh_M{)hxJd)`Ggk`MzlG5MQTh?b6)xmQMD7YwoJ_n@RYKQl zRDo#EN5cQm4*q9De*VlQ`X8kK8~Oi~+laXT<8S{*)sNNa|1@S$+f?lT`%dWplJ^g= zlKHUz+r<1cISP^mvwBQ8kKAT||MwBawM*xJ_w1eaFX{>d4gvAMyMHu}|KE-Oy#N2- zFaH1K>_+^rx%<1eiuz9OZgw`|o7H~p-BZ7!;Wo0Iky&bG5QT}C>?jrfbEKmxbj`x> zt_zG2)b887rle!$M(l7#kjd0sVDiwm9i=_G!lQ+Bxm5Vhql~B4 zPrc#rYOwdmZEx5uF?)2cb^@~n1%?9o%eRT0sF-g`F#(O69@&Km3t|L{ZKC@^d@wc5l6LC3ua;oMPy~-4K*C$0k=tvGr z;U6j4V%O}H`AqDFOX?Qc&@w>4XSXXYdOnlpR8E%k5(_Dv>rqCYOJ;7smd6y&F}it- ztyv1L`qNjFbFSM%5blD{e)3j>&=klR%Pi7WS}FXzQ7yP<(;EDX2^|kul;_BaU^1=u zG=+8>ch=^<{@vYFEO$xDGopOEoTtY>ZT|bEINZ<(8+0_$u2 zpW}yN$zxpo?Uzz-lmpGf%lEmj)-TF}{@f|QEe<1MbWMQ9WG%8da)m+T;lCejUGHQM zt)dKfZQS*b*f47vGN(7)!HTAHaN~@3QYJ2}_j`&dWOG^)$&K&6n7|NvL>WBBS`Vy7 zeW_+Ce&18S?IPZOrbU~%-hfZ(8rY|YKJWUY27tOQ_{tx3x)~pXktX48@P#D>KWUX5 z-}SA_*b&zSn;)HyVErCSUXx|TJkIRz!sqH$Sn!*i$mGbflKefxJBP!3_08Q>LT^So zQ)W=zEtJBcxB3yvWQL z+EMAoaAAs${zf;Cwfif5a1kz-XwlAi%;TNI4Xj+oh|T*~A(HU%)sd zm6OQ~d;dw+Jbc2pL({zHGVWI>hvMnoi2fwkTm4&N4;az*;?HKW+>;leT#8hTEaN=c zR4_yKeRY${ZkIvgnR&Z6?w9f7=r)2v#gii|8pt)fWht9QxqU$s)I6e$*`5!eNCSN- z^EvxzPcMpb+`tN|mqV&2(HyhO4(jz=IAv4%1KWh;#Cy>+F8xo;U_1Z@;_@x=ljx+0 z?1bTJ7VKHVXBLwICYFNA5j0apu5K6Hnr^Nj{w5x_`T;l|i!WC0Q3kJ<=YeTWQd-ZR z^s1I0mRJ`I>rxSFn!tfEpNqgq zk0ZK;pl1NebrWLR<0pM2%_rNb47O4i`=uV)-wzB&ZA*uGlWoDeF~CCi2YZvQTgawh z)7Gkjs4fJZy_2Bf*5~IJsYK}~%f$-5Gx+VRurp>EPnn(e2RFA>{r2dtB#VIZ(F}ns zPP~nE&N^a!iL?Y2#-fbks~S*>R@Qv<@G%!<{=~DDp3{dd{mn6wxeL5W&Nd6=>)u}{ zmi%8uB#Bd?e3Wr>x|!3&G4-{4mWbhQFzQ^-9p9hBb4-Z{qnq4Wr$0{UJ^3Z=r%Rc_ z7bg!6es&GLDSV_y_v$!Lc&8sN{lVqECt?Zcs?2_1NXV|*t=`&{Z*EFfXMB&!iuPN7 z?P>aDrP(K0BiHA0M2={Q?jP*n_6LQUCw4iLODK>3dPUK}k7bnwfkIm%_ z?Al~fub(4{NT53;ykiH)gM&`*dVghv+GX!3LnZnGlyF7)sNl>Q!8>U@)W4(KKkJWx z@m!$-?C;x8-tvlXMCCHhb&iW`oek}QSNtm8vz~IEPqPtxUA2i*YPTk*z9fn1{gs-` zNhQw2yM9};xlO7H-(~rDGQh$vj;zW`>W6&Qb|mMOG$-aT8$$7~J?7d7fM$r_@(m4Y zahqJzY1FTtR3_BkiC6v#BCneB0y^(E^K6ay!!<Azosv@M zs>Rl9{HT-p1e49@5$SEdKlbtBhTy{==;MW3bR%U=92c$CoyJ)r(0lbP*=A^lWul%} zcFlw@=#ewZs`mM+!Y`U_6|BBN)6BufTTQsOB+;HQ7>N$MWL|U3TW_K^;s8 zx_Nx7mI`L&?%k4L&#D&zN>qI`yE5%=$(a%UV* zNocX;cTe1reA)_NIFM2}Z1^)w< zQ?91)FX|eeX{<%IVJ=_Zel0Fui{eH}Xn6gi4mDI z4&uhA=37jv<-*TZ^3g-OV5}f1kfA~H{F$Cb%yNaU~sXj z<2Y_JLWt1x%E8k|U@g55*XQWFy{0-TD_Ya7%Rc?quxeP=EJXR%;Nn4!Vm>rF#^KJ( zV>+$+%?_)qZq4U70&$A1MWyDbG>4n8(`7K?6l?`i+;LNZaTraufkN@Q0o(U~ zB#rOxa)4_vdy+4YN#E_VT&@ixU$8#4TX8&_i);OyRuv%t@KJyZsxHG}Z<$%q8f%7# z3dFbg{&Nc`(bDZW319SM8alaZO}GnBs3KAe`c`WAi(Pz!0Ws~8W#O?agg{kPWVG5` zl@&N;R0d&=j~GyjD!<&hwk8A<`jtd{6WiYiT6Tkf9x=D;aDfe#ULj zOrSoa?FD7e(p=PQ5VbIrD6+%k(`YXb8ok{9$D853Dvvq3^^T`Kk4sO37n0;}-gsPq zgMUeR?Bwv-zEOJFHll?m@AKV7RFHCdgF&gsEu5_G{Ltm zOAf;G@k>Pfl!!k1oV`e*0qufkbgSS;k}n=t%FRU8WE5O@VK%U#xuOr7T5;Rk7(sxh zRN&J~ne#)B$zl9qIbYB3Go)LB2mD~i2q^XlDIe~b!>ZguUHC9)$f@O~OYY(c!RQ6l z;BH_2h7doMDqzWwWC5Tol@4*FK?D{%DE))x@m~3i z7Mm!oJvmRj-mm#}S6_PLwbJd9rR}S$pZT+UdrB1lC@K?X;~geN`PjMC2{ zR~0i#+9P$8v|-Hp#{eNiN)C3o$v75iFP=H~t)90QL<%~gaUj|kol9C8xdjRgHJ}k? zkZpJ%e*yhyH5qae`xX79_$hy7Y&=$sIy3+&`WVCCOFo?#oc3$bRzR! zEh%ktT%hRrUP}uSIrFapPSrEb!S)S!HKb$FLg4fP8A&-|i=ZoKmohd;PCcvWmzVlbXhVVm;x~N5Ro+-Gaq+7}%(KeVEB;J#N zKnqL?N4%Yj;DB70@)MRC6s_PZqT2J(AfxGc);s6FYK>UdG! z&-I@RgZx|K=*O{ZyGo&h+&N@3#`dXg8h+;{d}<3jxcE8nl4D&w)z6K0KY7&TlCtw_ z4Q^)q7gp6#+ZO-un7=f})PkcYBkQJTT2c`GGI->v?iIsy7iUhvoxm*?FLVI1j2;tZ zfUNcvgFyBx0V5-~?@F=god@^p(;DB^96|ImM6KfFSlRUDG&ir=H@|jmS+{F9fi>lr zL84HVE2sn=*SI##=-pw_-trY$H)Hr4&SZmMZt5IoZrLBhf^H78CNt*TttQZnQ(7WH zYrywTH%o@{y}-kQwbYw?wq$FDo)ah+ZQP1v`Xk${jB>AW*MlZC_0}qI2zdG^N%byI z?jTipM@krp0OSUGoCw8`n!LoP3%=n*H{RGP4nB&EwCjl*x=l5tY-nfe$3!22H+WvW z#_vru<$sUH+bLhTqA|)>5`X4U60;F+2n(xzQoPc~;ZCWnLq#>%pNZa3VSlz=NKv#c zP2WD2LWHIV%R~Lb9i618=d|ELExd`CL466J|4>Fi=)^|Gw^J!lM0fCl*-v|!Tj znX$SPc4v^A-T&_ z9dJ7hf7tQm+~Clw)qf`X&5Rg3aDvlSo1n&$WPM-%-Qr_-<3V8JWMZnX3HeAV8XFV{I)oM zdIhG@UHo{}ninM6Z25+*wGAyPP`ons zF}V|AH4(hi^nkzXAt#xi$kRkmHc%FcegP|&7`{gizbnw z=ff8^G!WhYIUSQGgbewDJdM+SH`SB_-m1%tI83 z2t=u2*y{~svkgOGnT+NoFfhbLzpsC3Zmcv`{|O?d>h|i9_=1XaYGM`NXc+vfw179}k7(4uRms445l#EU#>qS$khV)=zYhbO-~eVeU=3nwc1d0 z#6qWz@KeSm!*2V}UJW8&G;SCwkfH6yy;|aktRam*Ssn3B%dn=*AI)mzLdl%l615t_ znq&n6fPcj-G^j%6ooFyw?;Cwe*{0lPwm9xX1;i7KpkxPgM=W=gpJ9DVTZh?U^RXDj zq`0DOE<%#vjvBJKUN~;Xq{`V{8EM{dDGY(V=;!3L>wPY>nuf=z82#mw9;H_3!5eDR z+5AajLwBUDo&u)4EFZvX%_lyV+;VS?!cXqeZ6Wt|!cwmUMiaeth0qS+zYD6(=o@+2+V>-6Qp%Y4 zn;q)eQCP>%IB;Li?I$ox2J7u&0BeS=6Gs?pgrNZ4c}Da;T`w|`8n|Dt+A>J3xTe<^ zp(7?J`MX~(XJUY)`)>@Uf?#z zG$=64jnRY9je}+SZbodCH&;5i&pgktDjz5+qhp88%cn8d7a+xc%w*OD2!VTNDLm1=2X9t1amA#o2g+|+@|gz(;Q zv0M=Ya5Sc+;W^|Pf(K_POX%|tYKW7ybDavpaYD(UZ-F#h#a|wu!UvWBQ(W~{s1`9$ z%$}Z{{<19C;_){*sLE3gJv(A6WfatrQ7*Z z@dHmPXWG-d{P1c~XOdFB(b5bG@c=t-wD^G*sdzChCH(8d>20|qZw)Gn*mQwZI7L#N zMyK2 znp<*!BjqU$9_E99Ldw#?kuS^Ztn=%VjAq_0|MO^tj~UYt zR=)ZpS{1{twO1^n3aSNV1{YRZ3NS!NT875cm3(=rClX*Oh4vl%>^p=Pi8)YijO3>t zBRf5HA_i6huO1UUW)p#peLlY{>`(46+mJLK#P1NhaP{V~fFuVg5->DfMNAsx)8gF9 zxF^;v{i5guycnW9>zrUR1qP}C3CMzl5vjj!wLewlLQ-xPJuIYJpHa`xaRP2;I1arz z0%8cj)|Y=Gfw;<{1yA34ONk;jzwg#&wv3PiSj3Un)TzQZ+lwf#QuPni29Cvf(>|Lv z7H;2w46L{oJ*+hJSPzsEmF2VIS{lOgChhN#6Jddrp)T z0QdFPdhFhE|K@LqdEkszED2ViiZ30D9I?pnqVxFEKzN>L}UR9Hlx(yj7CjK(QJr};_}hm+$2u8yv3)MjGUD4{PM>) z_$hvRu6g>~Unix9WT};W3;c;M-`K9KKn8)b$hA5M&aG)9l!L2g;ZDa$p9C{P)QM6y zeqHP3ezqFopYRda2n8{Dc`NLqQL_w z$IR`U4n-9Nj)n=gz_|_4oJpAp(vz5%W@7@tB!moG*^oiP4CDzMgj4?ZGFT) z8REXdGTBNU87($byY^`c1Iv$+dqr8O@094$*xI;MlUdxgorMX+Q&lAtK^ z);*WR+MAF?z;E4=x(l2dKVfy}mzRdagbW6*w?X3W;r6I>Bd5rcsu98S^Wzma^n>0F)Gp10+9v1Cr zzHi9wqSiSh`}dG^vKa>681h%8l9GEHYwNeSGb1STNAY;0IS`f7_S0N$cc4WmVLT-F z!r-s&_OpXnLoH2d%^Wp>v2sT&kaNMT>4@9i$Vla|qRgp`-aQAa;tB!y0dBN^WFR@V zl|x82SL@B5SKwrvZHrhcTH*S9oLO!qPY1K-)SGQe1{!G(o^i$i;w}od?c6S*#o`kZ zL<_~I&8-|l&=PiqelX~7-kMj^mP*9;;9y956`&6u^<5kRfDkOEai79K?gd70v4B41 zc>O=#V=?D@J$_%RJ6rg9O(u^ar2Q<}-8r`AUn4ULhIcF~q2{V!04Qg@C)<|7-8ssv2<{@8a7}79#>&<(d zqqVV^(w?1JCg0Vn$^FB0vE#VJDdB~Y*v~0Qa|$$rvqlX5-gTrP#4-52P5}Lm9d4@T zVM9$NR)wjc)8s5bY^8KZE4kFM-T6;+kn*o}P}-;I>)&HlY$&6BG{9Fn3>5JULE|wG zTDW7TWZvOBz{H5SQrTupwp9}|-!eNi{ZUC%;EYobdnUjxo`NGtHef7zmuCp&WGEFw zXcuEtL}E=go%1YrYy~f>ICE7(u%ztK?$7OU0G(NzmKl?o=P!1S9f(FxaYef)jLL85 zN5;sc4ceydR9swDh7wFMA19*C#CTE~W{7wU+IUsaRSj?@OT$Z^-s|;&3kD#!qLM=K zf?0BzOz{Y2Y5y)Q-$3DHt_;994n2rKXe9bT_VwYUj-uqp8AaA7H=MBTjKss)IsjuS zK=CMLtx-w3eD^KU1mVg*-7I_UF{cWjWD3GK~ zhh|*b3=V;GwNm(MvB6{3kcdIzNijC!tG^<)f{C0+BngZWmnbT$&css}bvDDPZ)UD2WevdfIpJCyMFD*XfRs6*53Me{%w7ATDVn$ZyUKd|W32>6CI4S{uu} z(<3;q#|m4hW7^VRYzT$4a#7X6Z^6VQp_xkxF=|$cvLL@iVGvSNd}+hvpm)WCPkVZ8 zu$*@R*$S#Ds5oHNcMC(8?^{za?JE^Y2WSf1wG-76(LcvU0DqYHcnd&fqG7s2c{TKx zPSzA|9#Hc2K>k1&ghA0a;}2a-Ghe^@iXs1ARqdPKZ_UziPeQdZbtN8jIx$@mhrYhg z(XI#5w4FlNc+L@edalOYUYG1OZpvc3 z!qV<6PeDcIJ06o`2#WvUc1Q%h;l{gRa3&QWIAumo+C^_=fL&YGOjI)$PE5u@p0NY+ z125Q{e*Lqd-uTpiRtRGe_h=oyrJz?q08QzVJ6&7%i(q96-)2J3&KvuDY{z3HwGs<) z60dL7xkT6VGZi(_??0KxRXolo!2}~ZXIhPjpv?8nLas*N@Rp)D%9voGHaw8W z`UjlA`lPjy1ql{Xi4h;>-x<|Z*0w!5qs~ZE5dmp-iWKP}Z4?lt z4njaWh;#{31B5b)fQW#FB7}~#(4_!`Y6p7YSK0i@43kHB~aj z%;(-*_hgu76`zx2nx#=|vgZEuPOj(MUP$rH>92fV;@P|Y5NSE#b8Pb!B{`GA?_A9- z4ww8qtCd?Ux_HZPF*V1< z;yO-=j1->ZglhwY|K7Re*;Zls?%nUhR~B*|D%v8cZreArPw)aiBJ3a)wj^gITJ4(P z$!A0#+&{N^?O71VqRGYmhA9CDRdtcKXn#xb+er`wl8a%(2YUTaV@AwQ%MAnPpv`0es-xlH1krL!dM zP5)U<@B6z?)2e@x-*un9hPbjbj4jJxr*Cx3ZQu2sd;yE~g-U#MI2r)IC7Z`3N>tUbxcr@o*5MiO`c@%)23J1P4c zm%gRD+zqyrGhC1{+X58rHeX`Ds=6v)ZNZzX@|WCdf6@8+ey&!HB5W?R7b_QBcRD-l zP$rrC_)ObhXdX&F^(W;{Zd}fp{2cr~N3~}$U!c<8D4V*e zU`BGk68c1;)%$@eYF@X0SsbH(P$>J&YWIXA644pz$C*{?H_R|6#KgY7Bq{c;0>qOF zGuLvS06#(VTq1bW)W2yDzZN5rmpZOE}NHbu;Sm?EVZwJq)$8=^pLa8l=~&J<7{#N_l)qquWCg%hb})~Jd8w2x$GT-kF)f^)kQktdrtp?|WSZ zZcVpQuS>x7@7Se}?Keq?r2X{thQR%>YYEZoOQ2E2FTRqgn_ns4uStF%nPQ0%z`*{f zPH_ajyB_3z){*JM?c*Q;hKjd3t67d2P?-j$GY^k79M~^s-+J#Ete&8rt;cZi)@7JFP3`P=WRND5} zPy+D4bU(}fYORjWn*rR`gJ_X*{ebMzl*%VKcKu?(-XebAi%p$ z7PqW9?U5yZ{9vrqrY>{2{@VK&nw9y5Rc?$c&Kmc^5cCMK09wa++}JA2PP zJQs2DOu3u=j0;(WeWSaNkT+OjEGKE9HCLcTD~MhYpXn}bdOZE6B~?j>)g3jiN|{2J zJ`mLQfF-mg37I_Yaekt!bS{M@^-aIpBRNOu5(V^-#GQe;u~_BOm)<72BIi~X+;W?0 zUfr#Y=R6>uJZ-jY>>BU#%v!_}_z>JT(`j^Q?`vwwgDT3UFE-sPS*6!%ddfcia{1Ec z2RQB#`E1&%ZeU`PGxXv~FM((qb;;g7+msZ+0EwInYOVLw!YW{zr!|U$6QVJ3eVIR^B6TT!TdQ>vgun=rrd9L!F-VR+D|aXbg;o{kPeEcotONImvmW|WE0oAOybR5Mc{c?uD-#)s3{e%Sm~nL z{T`yKcdC=~>99}3(2w6_m~16iJsp=3TlX)`B%idB3z2a7{`0Kzr3gXI@|>Lr)wz%` zSs6*zvjV&$D2a<+YIdcJH=$_!v;Oc8D5+#+iBf!-YJipEZJSVu$Po?uZC_~Rx3Jk` zzv8qlk*;%-Y)s#ctK}EIHG!`*zZaJy+V$$- z*$0b8$qOY$+qGY=l{c?B(-03__9ruYAE{;l@4oyoDV+JVUS*>aqF`t>p7Q~F1xuSGVPQqc|$$- zXNiwx?76;{Cadrcf-&_D>+?@HW;U1aAV!tsR$sj_>ck(USI)9)@its4yZqx)* zq*?g47Z6+fn5j2dDb6uZ)~hNyTMRj^8nEUtENCOXB^8?Imp)J>;k%C>%6I(<1Tr2v zmpYv=bE;M|8F)En;&T*h7b&K7iV>LG7vJI{2-FVDJUqer0oB4UN%ZJ$=g9ZWIz zxUXwlM(<$NX(2&(`Wsg}jn(ujWn4{MI)7b14@429Xk}pBI_$ z!$LDV$toy?3=L~auX5Ir_F9>2K}43 zA6`5Y74Rl^f_XA||IH2?%Qb6Z-}Ih@^A|6|0b$rAn}Cw4f%nDT;BX%W5J-6@S37of zwn0`Ss-WNouvNDLEbh?j^@qjK=wdRJTJYstpk?IbTaZxP7+0b|MSG$2xpQZ=?`QK` zY&~n*xrAO14f)VI^!huG%Ej}R>tdp!D)Fb(>IOwDBOkrH)^7xIbt}p(Z8}$I*C}|> zAh3un<7){}l^25#LZmt{_a8g}w%+~qX|M!#(QDat!ML}oc#P4Zn8*XsOw=COQsuvHOUB=7TFOy?8fD`GD$eMlyKhnU!SCvK-X%iZ8RYi#^W zs{j3v2*kT~?b=C4$6tqgtY0B1BKf;F9xn;3?%&{Y@l+n`vuwNCKIrkgx8moZepYv1 z9cdBemwGmO{-U_P3y&QMuF5dj;e=rNpU~j9yR9u*d+Dj|<1{ z4EDST^T&FWSy{!_)s5F-$JWhjbH~h#zdWkCKa`vOPX74mb9EozR$lyUbVIgF-`TF8 z9zxD?@Nqu=QS0a5wvi+WA>`C(L3OE(s#C5DN23egvu>^#og*l2QQIntH%rs7X{nE+ z5N5$WHZz1N$lxQBW(y_5bH7G2Pb-PU+M{AW7bI0jrH^jWooQq&U6_i$;6P zhOBld#;2#Oz$}B9h$HjvA<3OE3^FjUT2WlyelJ7FccrD)P{px<0Xc{>!ql46YsAPG zBoP6O;F~GEiiuK9>h?1Rh)ExkZQch7%c}8Z13rn5+irOoi16V$YT<3@?s0V7DF~7= z?&&<9`Is>rv7RO{KHq3Y;X5A<5>=hRWC0E#bwKpa$v$fpsovsGjtD7ZkR(^rAWfUwc{rxUL z;bi^eDzSxX7$!6Y)e!18-GJ}o7W||_Y8vUWrO50dTyd5j$r2BjF3%vm5Kz)F;puws zyja_98(ZoYf#}t@NHsIenJJ|3`?x+8rmY+FG_5!ypcpo*kf>B%m(oYBK|(%#b)iI( zVhiQ5xhK_PX3q=qzWDmMl%>nm==@+#Mjf%C$5t3RW!}>JI_Rcqx+HBEGF`WqJ+Pe8 zvU^i)#_Vu=0K8e?8Bx)^{}8obny((xF5g0jx3_G?BX(o~G&EQbQU?+ebhc+C}__Tb3dMMT%oCMvZ0CC5-Bx zo)KgOlO*Q$w^5bU>jRK3(VmxQ?*4i?q-q*6Q)|0hZl$(YlC@9@VtIL{ZKWpzGPmX= z1x<~1U+RpL4DNyi;m`YPBJQI?;b6RIjpJ!gL*;@#QDKiP3_Mb{{P1}poqqe-{GZLxJQaa|4+py;BTZP@g>dfjk?D3#(>cYsO#ijhuw*IVd?EHB zU7zrTQ0*b?;XCc(?mL|x@R^c2&5d#c^0y3*r$Za$+qmBqbHP-@B9DB3Ddpod6*VHx z2^2O4uQiFzP-yQf{pMrwENbmea8#l zLh%ezj-`^CQ^x_9CA28&qUx+|2=g9_)&kp5Q7xI#xDOIf&)i-l1G}BVYdt3F&a_>v zKxPyn1tu_tqZmU!T#|X?!FG=te*kHYu({9<2=tAAwH?w*P?N*UPhb}qkPSckfE|vZ zgW^_BMe#iYZg5SYbN7Fuqu?KY{bhP!%_BKk98wT91yKvD-h9RTuX>M;9QrT)XM!(K zyp8)?j;GKT(WAhcR_ZI>*@6UbQWQhKs1dztrNS4gGa8;wKcH1Y!HsiJb-xJY9*3Ey zI`)H<+CX>YNc`=l9-X;8^Kul`@RWP8tW-;HidsP4+IDJi4Zc4CiXEGn{+wd4V!Ixb zpfd7aRHsjH64!~JTS+r|>K(}RSy3m-aA#JHH?^iDs@Kd2tM5yPc3TKm2SqMyf`w4D zijY;WI;B;uk@0kjl2KSUHQ7j|YlA7EPJ52>VMxpfq}#5)5LO@4b}5y!S+zhCn}*~k ztnuT>J&SsZxKZHZlBaF$;JhykgBqz-4EZ`dB1qa8W8oj6%u#eiI6ecI5Gl1|(l_fR!8x!J!RLm$+ldOsV;C`S=1Ft`y~B|$N)4NX*^u^6C=xugYT zseh+iDoQ7d( z0gB-RD9U2bIB)vKd_2B+8^uiHTk#6$OlzN`E<^Ue#fE=DsIDB3rDfyic21+V(Tb14 z$4{6EFtbbMn8&0JGD(#5SR-kh7CArqDBYHn629eb#LN*27+>a!@V*p+s!=5plxch${iA=-2Ctb_HaI|P-9tN7OJrdM}&SMqgJ5J^S{CPXK4=R z2g4QgucP-8rfS-f_nI*?bBCa?RV1oa*@K%_O3U`Z?)YfXmkG=*4K?&s33oQ3X$xb! z3#-_94xK=LpRHA=Fd;!JyHeYnSO%+}?%!;bDxHD`Oa^q6sG5=olt$V{GMB#iS8V^$ zr$Rqb-r$hR5xS}=2#~AVZ2YbL`F&J(uasgyt!|!=9VX zySzr^-YpyEMAqzbkj9pm>dFQ#YAc&>Zqqkx3lfH15~OSPTP^=mtsacyDo+h-%nDhH zO|SbCHpg&BQBGC(_fa*7pZ!%1IzS|UO7&qw_;y>Y#58Iwt7Sc<3Fm1J-By;WabK7h zYMg3@1RnC);Z`d4S!M#j&r*$Gi0%)zI|7-T2NpVU(X$k8^Zf)BL$UyN>#2uJ`uBM> zb{*t}!SRKyyQtDPwAVKUNnyJKK?J^Xzpg2)1KlcYqIUP125rD@qtwZSGE14TWn#m7 z_~xK7LbNoNK32m)3T9Vvj<#;M_Zng6LI2D$yE2F=Z`n2qDau+rWTd7%LOkir@toKn zJ3R1U4qUfeR*|%U|E(l}=ZiXe^d-`b1};JfGJ>k#TjA7_vC-d!WQeE~K{XzEeT$kW zF{wFks!w5%9u;QzT9dPZsc95BL}!AN8&v;O?$&mfmf-0VU&GJxoBzjk*; z10U3KAYCG!t{55Ae|v?$JY!|7ySX&lM&SR8Zo|1R{u5R{-&IZ?m!bYMTNP{pI~3R&8#?;gzDfV$sr(>WCZK7`?C{p@Rn)1hf6WTDo3 z%Zfp0;bbaWW7`p2zivRR4oC1I2;`4a$kC`Iy*}1ntY2}V8rZL6recdNMSV=>?~zQ< zt;DEmTP1vSTXyebhK<)D($-~0sJT6GzVcMfjx~!^c(Fsh9vfMpo-!fO#K;qTlf-R9 zgtgGRI;+Z71#B76@cCpF5?31*3=AGwIwmaU!PANizQb*(-S=58mJrEP;Lrq(zyY%q zHM+8)g|y@1AjK$<;KejQ+|#t`weyCrgs&h1*BvJ=@(-QDP;&NYo6p($)%;^>SJXx{ z7?c%D6|ZjNYL%wL4AMxUYHO!cJ=tA@9*bJUC<@qP4QB!p`qiA`mX`bM+;P3CRtTb; zVZ}CR;J}?;9#(cpA){8gZ7F)!$4@J70zNie=2O1cRVStvwg3TQc>=}SThs1Sp3W# zit-Ak@h-z$_hz@eg<5Je8r|ZG+mX$O1GYGG+dav>4IybPuTf}=5Vcr{Jj*h!(luIh zUG^#FbDvUOo--+cF3h3ihYz6tJwykd|9H4Bh1gkSqcF#NFys`76>V(bb0GB`N9Rte zl(d>ZGD7IUj-6m_&veXk?2aJU7pnDqE}MXo@pEqU9p?e)cBI*_3(xTL$B3lSy3)eEEc5*-F8jS@Sl(n+A5ZS zTKkP_Cle;u8~KP;{%Bi_$l<)i+$PV;{2)a@J;laKW1)f>4AVm`bd5`<2h4U$(aliL zXvYdmy+3PY6$us9U38WU`GwEyEw3sAmvsEKCH1U&nr_8@ znMTd(8g>P37gBrzryfmg?f_LF+dQyA6&3WZEK2Q)r2ptak|4BSSEsN4Q8xc=rC3qU zIKfoGf{kWeA9I2)Ag~KX=g15n(jAkcfV_h854VvO-b2b78;iBEp7Fe6aR+V&6o zo)y+Co5FU(tW*PYnxpO3fW^Q;fCgg04;9pbZ(eWr2&J;vEnerdC5DH&TK+SLudLiZ zq72!pryVfR+WZ3423>aAMvXLHEZwpE^m1pZX`sPV0hBOgI=rq>SnE!Uvl z;Mee2;O=?zRa!0Xs*vAEZfxkli5^B7J8|b7Dx?{^-@F&xOAi00mkCY9v2=Scl^ap> z%$sMeG-^jRXjMYHZ(_%(rOQ%Pes`$flL_gl{;vy~&<)jRlWAi;agY{ld{Utbel)53 ztUR`Hq`z`|P6H8oc==8OJ#=R}!7_YqPsItR8V+MXLPv$Hw`S%L{twf zP<4?Vzx9*aK)NQaj`D?lRb4-IWio}Fq%9gWy z-&J3r&2}=>nJMAlQHT9>8;YmK_Ziz~4Ye#ip%g5!55>@EW+of7kfTupvQp|DO|349 zULW)_D~E*-*DYt91#0mW{A`m)&WMqX;}*?7nONUZ2uA9tnC3ajl9Tg z#t3WN#_y2(Yo1Q81guJW?e`rnsHqI>lpSu+LvRfx>8>kSGYT;Sx- z^!lxNHFx)%&$b2?3!Cgq&4MozCL1bxVVTm7A0lNl@Xf|hhnX5Q=O^<}&d~H|XK$)WmW}yZp?r?FXdG&?1;Ngq#CqqUy3*2@Av4!dC zjq7ow%7bF;+OTYj)$G7#TVYd|M>t_+mbOzk{cPXa3rYv}oSNIa4IO*jTF-Ds2-<#+ z!A}66W1%C%`cQ#=ePR1prVGBjxs8id-&&lKu^@%rM9!JSTQ*U>GwD0AO>_`yJrp73 zJ2j4PB0|8=X5CW4fsf+bHy}{U+y@I7qfJ@tq5rr<>b5dlYzAb?d~&}WOYuc1>~EWJ z1nyt7Y#1;hCXv4cDAzOcmdjcO zM?1Lr%*tiw$inPFw*rFcaaIiAMTN=z^hgD9-q6Jg4V>9!)wf4MKOqy|9+bK5Av3(jmcKj5FXKiphry5uY;x)Bc@8I0x%OOy?Hz>2BvG5>Z z*U$<)vsRzfelA)^{Lzv(x2-+qy(nj_5wKwslS8j*AB1%J=zpi_t}KhXFU&Z{H7 zcR*qo88)oP%^fgHFje+(70kJm;Azf*I={}8L(kL|a!pE_tJVrpkoY$(@Gp^=y%B)!x&xB!3|yHwx{Jty`3gJ8P38xpHk4@KlSOHxBF^iA?u^J=XxZQozCX9*`WJf|s0sET{ zJ;@Tl+QQ$X62yCR8HPDn(pb((3-3%-V9xt)&<789e#~5K!`o$qXoJzJrr3i%>_%>^ zHlj2uvqv1 zX%rU1;QySm14yvrhj+Jo0!t(5_U`Vw5MWaNH#`<%H9@JFnUrT~7w-Nl{4q6ET3-J8 zKd%CdbnGT3CKtpX%`eDFygXxgOJ3gTF}5X?Lc)6S>tKMSf8tIj0WQNDk($cO!YwH$ zE4%cM0)K#_9B#?VIzA2vF#9(EC8)RG`k0xi0940wlA{oi%x4-v3edc)?A3p-6an9U z1_MmX(gywZZ&vG#{4ZSpPiu((OIiSszW-06aP(B*UkJVYQYYpPFhY%C{z(wPvV{x{7KV!fYiJQfioGA=Uu**wvR$Vc zKuX)yPsqO~wMQid9gdpNgdkD>ttvj?S`dcNy80hh4pgrJ`2x?c|3|9M?`7e=j>y-7 zR(WvKB;vgsP?GmkkZDp(fR;)wD2b?G3~DU)8C6oSWQ*|8PZe=YxcKu$P(ISYPd1S* zcRKEF9{l|DSxkJ17~vHdNCzcVS%F0T5Hav!jHxo*YRop-$kCnlThL`Rq<)w+~oS)W$z*fQSx0K35EfqoC zT5I4Mq8aE0ynC_?zN#Q?>gI>|$GH|n17@lKh#{Sl7|BiK6F4yU>%Tje4Yc3 zz8_FpdPym*NKBr!)7#HF!IP&uU8Yb?De~T$pn+);U?|ZfA8iaOK5_vK_|FeYT7v5; zV)A0(x!qdI&Km_*ptU3C4CK09ZXX_yB`iK`d1$f$w6kzUff>8W>Y14T`wOh4IWaE)t8ZvO-)<6N`McL|=>P_#+&3 zL2KTh_>IqN0v&b{S9DLg0Dcs98I|~gs5^N;N5lY|u-t&_j@k3;p#2k~95#Rd@mwgq z6lq|}1!}ZP9RUmg97D^)brbU7K3QUo{y+sjKQlxy8Lw0fQjB#1g!=qeBF5_N_sl~&)wH%9v zSH1o8i+!?=kSK!5bJXPmIc*da=Nc~q%>Zeg-s89Q*9vOkGR8b;3r9d3)0F#z^i2&vrVPg4y34n2s5;0B;t0vI6kG@AF_5%8s#+g4Nh~sK-F_2R(5^yW#A35WE^Z{cqz!*7%F?^U09NuW0(PH}<00u$pD62of zH?FvxosA;6zFd@8*{;%UxUMMLUr}51wIEE4GxFj|0L3D5-RS^g0CVE; z-g3qufER?1<9)Xh!SG(U^BB`15wzTCOc~%9@bD7oRO`I+mVmtgh%-u$j8DG{fLET? z&qUj#L#YnvJ|IyS0y9~z0csa1GhmA#DhvTmn+_4`!R{hzbz(kL1 zJ>}dV;RAq2G#<-_16+#?c%P#Sa|Pw|Y3IO>aO%M0r;;l$Q3N7PYQ!2kMq$C$%fCzP9RT!R>{S`N&*l$GFZj<2q~a^s20Fj9`|Rt0a^%n z?jVrqhk_l|bcDfNUi7=%^B|ztY9A(EJ)-1r(c_}ljU(y8u)GH(6wf|5EE@g9w3xMJn$L?<8D9+ejpFf1Bi!d zl8-*bB+?8>jdB<~;$s=ofVF*|*=Y5{UO;z2`NqpNX1X5veu&~VQQ*3OIY7-~AVPn! zt_lzYD1bX45gC>M4_6)P@G{KcqY!mE1?s2^zfIE-nu zO3FJb1$yHMv;>6g<4B_b2=@vQ#3PMJ1-vbL4aj#=j5j~f98m0tvN?cUc6+zq0u<22 z1QhR<_+w}Spz@z^WBBQ^nFmMm z06ya7k89b+K%OH_0z!xcQEPmxk9y>&|BH)7F?ry@ zQijjC6p5j0fI%}}H*S3on2_77ZLsBd_6nA$hQYh{YE$;4?06|(DS_%~R;uc(5yimLpm*R5r zyyJVvH_kK8`IoViy|VXOb4|JDwc@okmGE(?aZylE@Ku!Mbx}|-f1seC<6>hWuOt}^ z!BJ2cP*mh)^!-o|dvU_v*#`gFSPz`My>h<_)?(vd%0{Rvk4$!f6SQ@ti3Z3&e_z;n z_qDod?B~~}@5?4D?Z21b)Zr$C5s@@=!ZQyMEIEk7`*GomqrLp=(3}C~Mot9F=jjV^ zr?U*(h!=z}#r;ly_eTErhYcY5|KC@={bPXt@0RCBELoxA;!$R)|K8C)$oak6#{%XOLEekOrAECs9 z_Bd>&|9;lfdlhiBE^fcT>FvW!aESWcfPlvT3|!4g1ZJrC9FxCE&4ejyL`g$^SL~r}Kp{CL(O2 z_-zYH-2Y6YA4eQepU?G{E6Ph%r2dE<;6umwk+Qd`AFe-nMPipVTvaFgnZ_%Eb^8&ODyhK|6_8o!sx_t!%8X@bN_94 z?yjhzVGf!EF_s=h;4;bzm{oop1Jgq@HX1Gr6haRg!r8(Yj4qV-V=V?mnOA5dU)>`F zxD}NRMm{iupPw4ZMim&7sSxPisw~6Wun-)U!pt*WA1JX*?qx$Q%9mUSv#KPDz>3lX z;zpBGyd8}D+Mo2rt{xCUvi1X3TpNVTpY=Vl?rBS)`?P=|cis2@Ja44_YYisJn7IGm zCY?=SV6S$ZgTQEXAxu{O^!`!b&_GAuRGxvT$CK8oj`G(pi2*BK`6%HGp=kZ|A$;ug zm!xxVkgr3e4GPHvL?Kl5TxAPlte1PpiOS&Em{DfguZPPaIT$n8xacD{%#HdJj9G87 zK+=@#AOOb4Km939;dyqlu<%U1WT{XRrm#&`TzP33pFbeC&ct=0GN3&PvA8LbB(v=F zxPmicHeq+rSU*yi&!My$CX_RbA>IuT*$EuEKNiV+2c>BeEJ zhGX;S6bexN97f4graJ1}2P?c&p~K_uNX+_`I=9l9v+hey0k@I5?3|EV-wnsY4jA(lLW`M; zrwERokI?NbChGg}CQIs6Z2MlFBxQ`*7#iAuo)jOZn@>tCYtmq419!ISSEr$pNmN9^ zqnza0i@iABUK$iGN=8u)FC-!caxbl+4_rT^RM@|WC=>~d=mTu)cK#7!63qVXG~K7d z#NePvDBidD=^L|K%sN}YcG#vswIEX;&7d@Pj%@IvrAkPa3Nb}=ng_sVD5z89uRVkF zZwG1c8U#0@E%)Jdp2a7{ipZq=+Bm{?hip71rz~Bvk81M>q)nnmY}!WXhJ_WPY_JeF zTFyx}e+2SQD#*y7hIALKt-X7fXnFVfV?&C*_|OcURKEur5P|Cjk0C+DWJ2px`OjMN zg9p`sGT@5*uYcyHSgQl1aokOq2JW<>2lD8V^Vi%vOLpC0nBPBi*Ly-@cK=azIj)`) zhECCIkbk=0EI5yUUe#FM`2a4%j+MRPq_?lO!R5%MM&M!Bb&0%TiJ+0{ZCD}Dn}y_+ zgUK#w{WHFEb=Ekn$#TE@G}UuVv9ur*$YK7t!0B>#%ana<*-Xqg_V-ltnp7*k(r6SKS%N&QoP}i7f zP}?H)-^`M*?EH>l(5-#)SH#5bPgPpMc6L@|D2e`xr zu2{Fj!5Bk;$!uF=eM7_af&xM&SE@AB{i@&uPR(1=`yhddh`45h((#HDlF!1SK#4(Sl4NjYbi;SmRNcT=kERRtJtN^pdVdD~E zs40Ll1us9dkBp$Gp6#;sDVuY!6Nsw?&l%AAUA=Af$&KgJKyFw&i0`rK>({Rr>uouj zUA>Fo1_pwmy_iJU7)C8_T)w{^LtKXK4(2It%;5a?LS&g_EKn7acntT_+l#!`t(0re ze}aa3Iy#yH)V6Sh;f{&;1znEfW3PABo;Sl)!5R)Pn@!qrp`u8JQ*`p-}<+b z{z*|en!r~sL4+u8OHDI!qmSO?$3@HQ!Y*Eb#&C4y&DF& zwf>WIbE34{yCV2&D3sx_*0=F8d%-%Ft-*exxTPg^txNKqp(C?l@M-A9Uduc6iy5KP z#`x=XUwY?*hM<#*w?=+*-yhbWA5mq$n_Kt0y)$&}XTKR0UekE`WG248s+rZ0Ikn2_ zduCtIm8rsv{`oFv)$1N#y}tSEa#BaLId?Y`O9dy39oN#app>XP6rA%WVcC5YHo z)Oc1i?hBqB^eYtbZ@4yGdCexP`kk~&&w${=j|r#ZQR?zQ55U_zp#F~rn|~w;eBeW6 z;^bs}*iDS+Oty__ZGCzoPkUT$4+v~FQiZ99)2f2?LeW!HkK23x-RP!M_!wr|@WU8)$rpVGiFp;(iw=7QN7L4brVBg= z=Q+*}JHed{MoZ_Q4K(Tst%~-6;)W)Q*lgIuPD~S?XW+Y1Zx}yDPWyW@3N3u^_n|a7 zn~tWzx36|Jst4%yivuC?yNwX!gG_en=2=fjEW+xJ%Q=k2tI0XmwCv3RhBuji6TjIR zd@ML7bsZ=R`+3rtrRp1XbrrIzP0(OyKDMl&2NuR!ntMGk7Njr8xjSoXODi@n^JL@Hiu-{@uf`s0`>T2PbXf}(@dG~i~b5P^*^2+1#5AS3) zh|zd=wr$9+N)tL~m-cx&?hEDK9j2I;GKKSy6Wst|x*K}GjylXX$J7hqb$r|N%QW!l zXVtr=jGtfM)Cc@iee=x(@z;RE=(`iETCnlC*p+qH{mZ0GLup4GyFJEs>k^l-8*N;S z!6jYkl&#tThqMeAtPLP0c7%DyqE&r^9aV5TuFDI)Wrl-SE-zp_*k_vcAKywxd6#H+ zF>EaF7|pzrmre|?#y53E`j&k>-iGDj(dd8UpZK9m4feeDSDbbexrlLNeoA2xk*GL` z4pW3>5C2trqfdTAv#a>-@xXKBG)M z&vAyOI$~O23Kj+E9LM{DjyC*ziKMh=5oJuct+CMzXLN&={LvDy9+r{sSY_gePaa|= z0M(EVNV*{B!;3R1Yu(f}cq_|7%_N!IryV9OKwVr@L|X=o_oMEIpdc$sVA7I=NQNSK z(nN%Is8M~>+VkDVVxME%QmE%^0(ZtgS$dot#0xzTlrn8x7+RS$cu<6R+JuM;`Tu-C zlgRl9AHfig(bmSlim%GGoFyr(3Gm~v>nulwLN!9ZJs2RpLs z-F7d6?!2Wers`O_-AGPudMeuf7OMh?Ls*H5MR-#CTAsDKIxPT(-Wj^n*P8)dS7r1b zFg=*l_EF2Pv5&$Sx0AsI7LV}$3HnDhw~hlz#p7qOPd_6(HhyS+?5)n`VQjd5`O?n< zkFz!1c^ywjbLp`pbSGsmRJtl#SUAprk(BP z69~Djs2Y+~>Y=_7<|^DAz#h(!bh76vcRFMsJXuy#f1AW<(*Jjm!qeu2`8{+vN%R8l zLbumD(V=1!-1t|`f8oJD>7UQE_`q0#Gu(u>>>&CVVEN(Aw(z9g(|hPZ{P)WN(NI>X z0|i`}fgTa=sQXLZ0`YZW#3# zaG^jfuHd)()Xn||CLX3z(7nlBqXQH!W|c_=wyg}Qe@wgdlA|?l8$Y97Rn&V8%I2HV zEY8kP8I9WJ*FS7;%X{b=|6j=^H4Y*mATR#1;`8UU%1YIQ#6*>{*?a^9h(|Hb&Q5K4 z()AdJO`!{+w7EJozIl}V`}gnXnws*Gl1OjXVozi(Cj%aAGx3_SdNn$ByR0RAwopO_ z6x_KuMjNamVh8I0jj(#9!%mJM`yw|Yq0hq$Ubj2h7mLV!HC@!q!W5pg@?iDwN)JZ&yseVz zNkX)mu=xntKWfy6l0~8lsC}Zu(jqF(pefSwY{v|t&PBJhY0HEIo{c+OSq7kVJOh(M z_1SHZ`2Feyt<;+At4vvWICdG3b#PEXaAKpUw;u)rS{sw6Y(X@^iJO~DOz9A$Kt$a8 z!E7=Q+h20RRZyioTDO&qALsiP^iLfs-+Cv#EoIz%To}2AvcSm~6;F?GE5mU@5O#zu6cNf zP4U~ukpJXkYoXWoOLf*S%%6t8A?wO#FOU z$(EXU%3^soNSpl|5F|l((WUi}=BY|H4Jk>Q`Q#H2OOn68tQC|a<}`}qt-5rW*mgV_ z*jCjF{r0?3aqDR)we_@5dOiD8H|2eAD5df5x@#y@a$Vu|fk&6D!JHAKA$&8A(7Y6` zaIrUSoH3jOvqUBW<-IC9&~n}M17^`ICL`^zDHL`=`M1t^ayU*@eyiYT+=|M9{- zC@DrbiWwMKNT|?6{crGzg!})DK;n~zYJ;}P$SqlvMsB?!<}uZ+J3yfPy^V(VX0d%1$pF{kr>Q&sfLt~HlF|G>s?+P7LB4a7g( z?k)MgSmTi^Vyqb3@P8FA1}NfmfK;nKc$gqaQtE$Fh5(8smzd!Dt4*hg9KIZZbY);a z1S!H@kl{vat8QH1gSolwj#t|r9`+85Yb_+RYu=7&h1_8`yKadPAXW4gA^Uz$m&SQ` zIil?q7|fVCS=;>~tj>eWn(Oqj_SBvqh{M%L*gpc-V9FWjwSBJBOliL6Yw02wt4HQw z>M+W_-}*GwQ6o5*pcuQ?4w2a)Z8VUkZvJ+ZfR7RB8#dA+7_yK`&&x}6ka5P(LcRCd zx}Uq35m02oeOu7`IACriz%{gZ4>+F+O_~eN4~_?Du@z%)0r%uFVBwTl+Z4o)4|}{{ zE7R1mSkcgkF@^6QK;49fJwDPzz}hbgi%mH4e({BH?C5GsyRtF5F;h(dRa7J-U;pqx zpLU8NbBu&@P9DrQ4gnxz9$eU+0Lre#L3T>8sff2>$0NQk!wBt7O6fBelTMj)vm3Aj45C`>T8e|`G#WkwJp+gB^08ddp2OW z-8p;E@ku?>$_S5D|5Py{kyLb%22aLf8BOcc}54y7Q zKP_*g58OTujacM1?^5dRkz2xXk8qoh&Ns8(nGg`nK^oHPbFtIuBym;c1jacF8(+(U z4!3T4Enhy(TlQ(vIX1lL-66BZ+Bbdne7$f{srn`aerwB0^tsIKHrBW?h&tdisrNkd zc=Cq~b)Pwj#gR)e_t}gkiNwm{<-czB2@xsG%#&g!_JTuJUtVS&!a*LgkbI8aGqwK1 zkl)|_c)GlcKUf^NGXZ}qNaZ0K(j3l-(|Br>TspIK(xW0AGrcoM`F_E?mQ;Wli8M( zo98Zhru5&!28M>-!yiwWN#Z@$?(XhI4TrW_JFh>9#os-4x~jZPm3&d+7}8I~;}!t3{17!) z&e;(?6&ge(^>l83J8vB*I#*YH;4kt+OUl5sF@RV2*9ZP0IjPOL=h|>|v&j7Y3iBt9 z6tjP^4`*VMifgcB@yDOf+bDbNe71~gyHhMv?iO!2x9Xb&#lG0&(dZ;G$h9XDHvOW7Ko>4d7>$5wo3#Kxdlo+n$ z$CZ9}YuZxv$vThTW~)mlh5o3-(hbvq)l%`E?p4K43&Ja22s~_)3P$O;Jd9N*%9=Nx z5r=1B&?a&(wtw`u*wagbY=+!KSEh1QEj`@2ao8V0#Pt#VhwwaK+aDsO7r{S-vaASw zW(CuB8>K^dGj|k{UYcqVZgHyq{hQNdA$eQ!5nv|+OKc0obDaFmV>Ma-VH}5UyMTVR z*KzKR`Bc>*>bz82{L7LAg+x0&SRNy}_jHQJ?;D-*Vy8-6S;~;2#`c%|iz+7;5!y0f z?<}4o7SW+vXYDun{n{&TpVrn?HEkuF7pi-l#3l>#7^r9%a+K5R8Zhw2KEE2X|MQzX zz+a^m{$-8^oCr@!%+g{Cb&)y{J}#C*C)4=a=HwJtleLxPKwnIzkeOMU;=a*l#y-7) z#@&AR5td3@&R?Q3+;DR~_9Z##>|t)oqmeziyESG>1MdqI6|26m3h1&Vqm>I)4~12J z$xNbKS6p{MGw2E3NW{P~$_-Bkkfy(++SQ)Y;WiV`Lnyr!{Q=!E7z|umC$9+l#hL4r zav^&UvgaA>HFv4F%r#Ii&_VqEM4-!Uuf4sStz&5Gb+{MVTJygmDY8RK_V4QIY%?*i z_3tQ0j<`SH<72Qq6EQUnji|?O229M`q@)357LrnaD%xSB?tVKuHD&+wz*hOS5(0?q z>=b88$Iekl294{jc5Xwn517%>(JUmM83^E)_qlQ}2tEItTG9@0G&vYDGpP?COG51E z2-CU%oP$ut_keQ0bBizzHGDMH89|#lsoZi`hLpe{u`xRXUi-f+1c$$pw-(i542z5! z>rAol!|~8S(kK%=U`>8`M%8Ia53~r7cAqFxGK})9!N#JNw{-gzXVORs!!h{qnTuwc zxx~ipiE?+FiqcGJQQYR@-`cx&S7`=oRZi>uqwCGzi4oKpBd|-GJ2M_h((DPNa$rXQVQ{f`3)6y zen#onE(DH3PzL4Wu#2?QkWNSE?OP2tXi;e^906tLN9vQsGswoL?iV1k+-4;@MNU?^ zV@7ttaJj}?N@{95tEgJmal{AgKb&@h`tPkM?eFC%Z!_T>ZVF^S3u9mRo|mj#7s{7l zvMHGUWfl^%4w)<{Y*Hi|Xs%6`iC~mMEuioBz_|%-Y^1e@tc8EnHU;4hB86IjToKB` zPhi%ru4&ekp46{NvJ<1x?R|s8Up@e42V@KL^J>O?NY%jqvH&V9F@e)Ha@wQ3r_eb2 z+cRMTU4B+Vv66kqRPTMU1JT-&f|?S~S)5ppl!E*|l-{(|zu-3Y9Ajl2<1~*ST4P1T z^;9p$1aKBJLFyRUhP&+gooV&B3fJj$Hhfg?49b(&wx%F_->55g6l9-3XU!i0&Gc+< z_;dW$bzYkg6orGcSJmw-Xk4o$fEw;BKOgB2*WO1|Chn2&lG%Ha{%WLK)10mI=gN@> zghF1-om=yJB5V@o=Se$>IoU?)Ip0)uMyL`OhrpuiWJO@8|*ekgwP++^LfVV;evtW?3#@;v$GL#9_7*u($?1KNV~TG z*oh=^WHn{ z<7xJE%Wd5W_RUy(L^+^wvnKY~q7P1TI1qaIFr(NmTAj)z_3Ybxs`hWpf9lr(}vV0sg|-&w`d|zY7gceXWb7Z_jwcPs3z?-oNlE zPc@IXM=WzozB8|Pl{z4;X2H@6lN;`;J{&Rbj~M%AqdLyYp1TuW;WHy)C1*)A_MuLX zDrC^p!{r4pidb*FL|)u>jrCIdS9sZDRKugoN+;BEh}H2|LuPeRld0FBfjYe)IAzGs z>ybNP^C3e)$CE4*lj=%bc(heog$pmcnql?|sMR?=3P*pXtDf#L`D%~T$}YLzJGFXm zBrTCgUwVMry405)@$mX@a4vJlr#)#Y%3hEjIakU2@$Y>Nu|E^kunChgp;b@1EVc*N{V0K7!c zDM!lv6??!{`sUB%542YH2R2#U`5Mmab^-kPf^_coQ=dY(bA*zMNzvZ}9N*ayjeaec z-aKsNclf`!MQ1{9uME=+Hox8Y?`VpU-cR2HK_JlY*$)S;$L-G87-%9Xi3hL$lPS&2 z@N#1eH=o@=F{acGBvXHVKHZ0ga5pWO<2V^SKEK-7rOCbBHicgCf~o9QDN zuon!sx)@OI2N5OZHK1+DOQ9!(;FKObAcS^p(W56!d&c`bQVzbB3 zw_-cUkdgJzvs>PYi`3&CVz`M!Y-ev&b>XXt9R~gG5O5EP!;{5;p%JrWDh_Xfi0j#x zsaD!!iw|nha56?61h78#bMyFsVY-DVma-Vgk~hka999IHoHNz0*r$o?;-$|^UApw` z9`WGQ@b_elv8?s5V@~kwlL!5W!9nU=I;urYhRR^GFSqs#_wzK9V>bG zJOga8v_XC4+J6_qHiP1@%N?1RT5IH-riVQ|dCZC`z~0%UwP=6dXG{(VZTirk)nFH6 zYUc}Ms-E$c)iw|$&wZ_H;0|o2vqp>>bDF-+{JxUPtBIlAX(=np%A$BrwP%RwV94(s zf|CjLT59i#^y5?;+aR{%23bzw1s|@j#qRLKUsuGje*KEVNZE|76b$%RMvR|wlmXpi z&m7roIY*5&++)}6N^w-@GCnvSdWmC)Mw*6U2M(Zd!+D|BvL_d^m2 z)~OPczFol)7q&TUADTYs>%%TgV7l;f6#?p^;?De{UdG1xdvs-XDziopto2v6n())*pqtXuVG~R*#t$OsYI7kM-Qc?(pSfMIILqi~L z?%hKw3|TzCsb9WzJ&%S@3!yB2!|bAtbJkM3!Sr{|PrJn=PiJB@qV?Yj?lKL%4kk_t zAt-Mcl;5d> z*)7Nxn7-cj_5bbOJ)!*yukmbP$i3d_U+nE3QksL)?5>1fgQOf^+Oie?5-_ocW7{_t z%+G{q@;wT0z8ysNuzpRsZ7&iDn0n!>@w8!Iv?bGG%45uMe;dFCLV|l$cgflOuFk42 z)d=`0IIZ@VGXamM;JXh|qH5Xv9Ri4}e^b*L*t{1RWL2h3Nv;P4yGG-vp>f6Sh(Gg> zjRdLT=X&o9)NDP%*nNEYc0Dp!uqaUyM1`Lmu)8x^b4i&*a+y9-+eI0YPvnf|zfRoc zGAn(w^mzgE|soh`gE@(a$*~%Y)bP)Q~ z+1TRS-p1CVg;W(Xu~S;c0hiKIV?-xPU50E*yCYMYB~3N)W?EhFZSh>I(oz>UzCWBQ z#ZuPgK!inmc!a(y|7_CW{Pc6M9NqX`Ch6W{xqXh;R_1u#+?cmms&})zsY%LVX{i<0 z*xvjpbZ_}ausnX2AjLtG<5O3hZ37!AjuD0&_luCOa929?|Ldin?kg)RnR1k8S69{b z=1gbzfv&Eu+q=6E$n;bR(%LWg=l1^|+MKKf|M>aSlW~9)N62LrxrKJy4{FMaEOE6Y zn!;<2-Q=TOgvLHe1R5h-oyd+Ltz;1VB;;g$|L90T-cN!_-3VFS7Ch}2++GyRK6&f| zt`3Pp9#-WZ7(%#S!D2NBJle1H1cLIXLjDr4Wp1DL2*3PI(S3utH#QE62u_gyg2weg zO|0pS23h@6{1}1lK>t6kR7RetnwS_inP%pRITY@C<0BPI$(BQNJix^ntxnEmaT6KG z*J=SkS0`;~&mDQZjy#CKp5Ns5Zct%yZP@WVxwNNbBQ_EeK+XxWcfM3>iC$e@ z4JxCgr2Kt%@U5b&>(hTWvtzYPRJ?B37X}M7RT-h^%a(}!{?BlNCrBKYIZ=rd)?p9^ zCuT9Pc9eRuED?~j42l13Ejw_f3Gv=n@S4mHY41XGLaD&MaCxt&AMs2i!B~Z@X~1rl zPWd{&9d(o34hONdv6P8eBY#^`|6AgWc!6?925fhjwkC01ujUXHi8NSYrB0`^Vq1rw zHT!Yp{_GY2D}ty;TfPdCjO_1t;dkC^y-oHaeweSS%Z+cpG?EWEsvT7HO1Qd&u>d^jFo0b zH@EU{-zds}-{ipx$R29bho;MG!s+Dzv}(aUjh+u?vPDFa1%vJ+#C$oNTY&zalWXf0 zp_&6eofuS;i}v~@PG)9aCuiqcPUsxx91|tLGMBAFq0iFG+k5}yWX}at7O(YRO-Ih# zy5#QLK1-4fC(Q#l$K5UZ2guiN-@^$XZYlmek9MC<>{bU|t_}C%LhhGWZsx?YF8;(y z^>)8J+HTCa%|0LO?n*6LzsEJdSt7T3Xh#h}oRg}FA>q)vB2+A_8$sTX*v7;I8&M5Y zR9ln;=(zYBvq0hi55j`G+3Y3gX0|DT#3Mi|bl2%=`)&j&(j-D%%IM+R z_(+-#6>m&n#I9YGqW7;Trz2;{McYYzg#@BCa!NwuwqtFmY6z>n!)2iURH;uo0;*2h zFW@mpaBGV{o=3bhJtaIRDG)J+z2Xz~r7Z+=jNG8^%Qu)YzU_O7Gl{EzWnma%^X^4| z9~XFo-@N$smQcoxtKlz@YwDUr3-5+$n*v7M+(A_}j2OIT>?fE4euCt3K&4H_}g~%8<;sDz|f&2&WcN1?HYbit(3f4?tz#H@L z87ghl>uZS5P?nu29aYa}ix2UG3MiZ+A@X&4j6b&u8JX%q8m%XS5PUH!nKb(cA=FNKy9%8-fF!`fp0~hI~ zTArFLNjBvRB&Ped;?Jd7UT_rny3J`#o$bR_#>U~kpVTmcAiw{F!nlOEGEW7Lk4qR# zM`N*8I}8Oqw)p}ow-TTAvu-z6@J565f06Il=i~E>i?;0yTbIRFVmT^!d2joN1Ec~w zP|wLZrW6_~HKCN6j<9P>b6MHF-4o`0{d!bv{g%M3ZHMq?lj1qN%C!V$JYeEleAxtJ0KucyS)R4)`WpK0X z!>QH-O2lOj2fqF;-;yJX0|ywlj6j3_#KfHFK8@5O)Vm9~){cH29SWRfkm*vEzQiK} zMNwkd zi9m}o_u~8W$!>h847*T%`s0gUdRYa$^cfj}gi>b)&}24}lP4dd%3?A(%J@n#8*r!) z6D(l^O<0VlRWE~RXeZ#Q`o*Iyd;oJ#Ran}QMF#U8KaKnYX7A`|!~08zaIcQUV=O6= z+5>P{)ACj}iPUsx(AxOr^OFv>9mVDaO&rR9cn1{BQEkfdzCHf9)aCarsp%@OZyvD9 zGKsHT5Bh5XjUvwl5;rL0UpomU9s;LC*Js!2V_3)JGFtGBgx*>3PpDHah0n50XygWx z2CQ5*JHC-yX4A-R6u^m!ips>m4qOaFjdbmFHLe$QF|N;QD$}~V650En6)xL@F-(rp z@w|sBaKg6MomWlxn5Od#a?jKW7S*foBRVL2Ec-V1q)SyMnE1uyfe|fvig#M7rxnpK ziO%%i-9PHdl0hOV^u(R=YkPQ7KCY@AebK>A$NuIKU&5Y+lF%mMQYUHu`y#sJ`CKSj z>hQjU``&cX8xw+;EI?e|bWY8TEs-Bb)ms!&-B6yVlWwc|Dr1+mOI&<>e55D-7Ts^z zw*eV0ffak`GPbp~0SNKuYP-XcjD>uyYhKNyCi3TUnws4G{mh-vzCKyWpov(&jav#w zC#OvtP>(hnfy$Z&h;rLvOy*6E>15*!%q_MH_kDo_dQJhADdpvg- zdfv@v^iofB5|J{XEw1*()9vcUf7QGF!a?EPI9=n{ZjW5(XSb%_k4><$E0UkN!sk+C?i5}CE;%p*+MJ{n#1>mb^i5^D>1biK+aFF?wN2U7 zeNem|!B_JrMPGh9oT?ofWoWRS_V$xPZ42T04^>=p%~%O87U4@`v-D_7e}A*3Qoy-x zVV5gmXwrBUaMg__)EvIR`jJQu6*PLj=|;_DquIOTgU&c)EGhiOkvx2oq2|)Klnoug zTD;gfBsFTpBqQ^N_*RBTQY-2SHFm*KqQ_!H!Px1QBpzR-{RZ7ZS%-TVhSHKsMjfbm zMJYj7UQ`hADd1BeBjQH_cs!>CuC}Xgz30tS-_exXGP^+(ZxeHIhY}qRjm|68u`iz*^8Z#!nmW_ln=54$SAuPbY|O1GJX88p=2luG+dJ`$jQCS0 z`|~JwKRe_{p=D2@5BNpWq>jlB%gKv&r7R!OetQ3spF<7XSUO^Bj)3`v@0@;TZd*fn z{ephJNM87rolgs_{<9sy`*Ugy5uqM!FK&L^Ig#lt_Af}d9SvzvM4ca3j;wCacho<= zPEJoJMs~SIrlyd-!jzlB-ydmsgz4&R{yZ$MdCT8FaGEv4fabW(uF!();XjL%Df32I zh;og*IC-WAUVv>OY|}=U_6;343Ek423yvdmDLu;l$4 zb9Vx+ZjzAgAfDnwE6q>rvSRoCJ#bFoOH~yMEdjCvij-V>w$6TE->a}eku6Y>=*?0v zHX9k8CDebvQ1J6*kSQLI+;HDOK^qx!0}8WT^v1(sxP;A(`mex3iK%t01 z^HE_Y6RWVyz=GDVaOP=ftgC&Rx=7UUL+f_}FxXL@65qr-35`~Xzq%45d(7%O3thgC zx1Hdczq&oDQWWa=*%e*jq|9g}Rk$S0Q^*pAcI5DCZ>RoYmM^6KJAL*_kEY0gwNPDL zvM%nC1FaG?!%0W?lcrN;q5~-2@>Rm<>{843f+SAD$o>$4Ts_|TOW=pjC6ZT$rW!6M z{YlY1(A+^9a^1F7h5!p5^l(Fmr94^NpZKMk5~==k%IVq`D+ko=}9V>)?C4 zkd{kbx2I7lPq)uG!y)&Oef!Y#RDD+eE2)sa$zz?6bri980V!D7y}ad#e^(p6aEtV0 zWzTy$q%An~p)*P@W?Z_oB^D3_$35STPpq!eZqi(fE3x0^ci$wMN=i%$cm3m%ysxG| zuhUwnE!iR}+70~Ukd|Zlm zP=^#<+%my#_Lr>WiN)Ua%7#SKVY%|2DX8CROQr!$$M&rq$yx21+~3`ZZwHOx)xj8w z%we$$gR-udLw0((P?@<6)IOnMP}#JJcMudV_)~7ZODF(cXqb{){3U(-h9Y(DvhwRf z+;fDFanc!9HhXf_0}wj~H8QEu<-DHc{ULV5o3O;gK6O*>$j?V%=e*S{c@BPYU7s*i z!R>^H72m#R4l#$!bLI`hf(Y;L8sL&`X(Q=JjadFlK7LWb$<_Mr8q>U|9C!9VDorzg zyd2!sGkT|Br$$$jYpwt0<+-?~l7wQ|jSV$A{H$48G0V-sH;>n-Qb;ucRt;0M#DuugX>G9p>@|9Dlu$dx0z zQJ7b`I(Tma&~GtUKH-2*VOpCoOdpzYEvp!SMYT_bRA0R(MfW^5qQ6oh>7wB)x|_!m z8+-c_n5^?eJfAvx8IWDgSwnz&XJ+_iMSFV!$E4Y4*KYsnojlh?1)2ASlA)CdZ^$`x z<;vO{{cH5Ifcj4nhu(wm^yxM8E4IpvfePqVqIN^ z9Qn3ipfYWfEl|(V(0dw~U9M??YY{4@i_=`yLZPVt@{EI03Ec}=pQrQ|3a*kjOty&X zgO`z{D_i8#%fD|48fpVxH@lVyl9qq}{_*_W%Wg7X#m$WyN!%9{XhA4To0`a~1;5QX zXntaHkvsaF_!E;?e6h8jeWS81>!v|!s3sV973?4ueqHjMB)EsR8ThGBHO3Lwh!)ql z5Ool&qe`qbprWD0>YiI}@0a8k%B#qS5Sioi@Hbsl$@4UEhUwt&)-*4Y;Rt{2bfg|1d|HGl|>Lk6)WJ6Su#rZppND%j}0`vYr3Qf$*UU3)d}d zLYI-cFSuol{O z*6tmUx#mVJeed3d=dY#(k2wutJ_uyw*Y)Zc@?;kK=3Yw>w7|PqU}H@vH_(5AAvPDQ z=u3Kcpc~d0`T77w*9Ce|dGYN`sI4orNT)&ARpi8j@L001h$^RTSaW2uz@638u!DPX}(DMz+b*e~bUeeOuS$dhC=;df+XY04DY^Mfe(R(!V=$1LU=m&p} z5o!m!iwz%HpCk$A7SD}b(R@;aMtmi3fjQ#z_=uL4lC{cYvAsGhGjVUW#B5;mUHQ~U zthzTo$WWPDU>3+P8i*JiPj#^;0!ao^*{#Ur64Lg;yCPIxjZUJw4$!^ZnqHZfcGZ04 zyVE3>1)+~Usst23ToR?eH$Rza=RV+YXO5;gKQU#-S=^R#ld!I0O#D*lQn zEjOBfQA5f8g-*}UxrslUn6Pp9;OD=aBd6H^RtW1iSEclR=s=P%J3TzD0Al2YnQRePE7syuu599pod82l&+ z=$8i%M$PH}Mss&A) zT`(_wGun52DN{6TN?-6pk}4{2n2DMAo`ZCKB@!#184#|iB;2<&NU#D1!d#XSB?z#I z`fb=1;X|9>Ec9M{QUu)rr|kUe9$h09=CBM$^n7l39G+abZrGc!-2ul&ACE&wN^>tn zn_8RJ76thl4%4boNc~6wbW_Ice^~$l=#IDA;=D&#CD;}OtWXEj4);O;NSbmHsk~n7 zO;xnE!aaZe`gOS6l!z=40R7U=6C#6OW~iH!{rrlgdNZL+i*)zc>vz1$1KL?HGCY&= zb(3a|2U=FO9+uwX47ncfkF(g&6vBy`kJ8e!5VKz>PG{#S_Ei9po1j6Feh3a0nqM%u z)s&kpLY9{sm_(91Ffy*n+QN`n&2z~7QY|fGchKvti@&FoVbFW3=)tA;bjL5s{JsHv zfAZV^)UY8}XIxy7Tcs*#Y=m_k6HPQlnm!>^P2Hf8|Kv*T1B^V5Sptw7Lh!y0N>B&a z;IzB=Jw*+FtJgDY0Hvaf&xjL)zsZLjIk>FlB+sL2V}fi?H9P0_(xA&~;yt`RjRnmb zE#3oDZ>#JkbJ5mD3Gn%ZE<5U-ijBsasJ1`Pw!#6?%xdskAQ-QAM7`iT+ibLed9U& zI9L&fD=h`upn*WfbmdyNBmP!S3PV$WOz++CV#`ow&qp1Y8Rwiid+)Wbb*;6o zy+t_n!IJdBbo&6ce?VJQMs}+$vH^AQt+jr#&gz>Cp*((?8%A8zHmMih1Uu%}sq*Sy zoe;>>pVrCKcl_cuUaVzY?lHTTr#c%qW;Lz9Lnip*Ix>f^aWpIte8JGhMp<^WI)4yN z9nNqKP_*~%jT~w?p*v|j+INqwne((iguEq9$wT&nTYERe8lt-6iZIpTz}d5!S#Q?y zaSZ;>{>Iy}$L~~xY;Fujn8#Z7{9+Lw8uPvCVaC-O&NDfb+gmh??gw22n}5P6sA72E z*_=kVQ5eWtX`u)6ZRz9f=bMZpO|;dGS??`V zl2ellkY2x!Y~*Ytd1e$NBVtNVQepmF4t*nl`uRe}sC!J4i$Psg`Zq=1m%(j)(37lb z-fdG=IrFS%B)#0(yLPpg+Gm(dvf`>+)7~0Xe6$7Z-wD_%?2eSm? zx3p^R?y~ZhkmVsODP7{OK;a+Zo=Ip5nWwdV56ha`vT>H&zB*W%L21M&7TjxV{Rjj+ zmLHerLnta|B~`R>i?H|4VQf;S*A4^^ymWN*^A)~dUp?acV`ml<*6(nCG+Q;bB!E^V zZ&vCxW9aw3d&TI4Q#>T=qZW(ina{VdMM;K9iG^wvKyd!>NzAD}Je1TQk4a%RvO8<3c$(0so<%tc5dN2Uq4M z1_-JPjufx{rYvm0<-2SVHwGrAGXzA{pvXa~V8WIR=aDrRN2>W$J&)A$v_O%3MQ{Eg zWDm(jO6`0hJLMM;vVO@292RdxuwC;Xx|Wj`dGb7AM6*O~^3LKe9qHts2ug}5$e_T1 zCt!U>(AZ=z!k{LW`p3c>qQA;NGjiuJLQ!M;xA>(f+mw$Ga9cNe#qx7!3sg+_5K@U` zFb0`IhJk*&a%Zm-t28LnT@~l3%!o%=6*R8t*j9gz3)YHqF16zk5a^$-HdInmvoc~N zp`(tsq`&|vHqaOUL!eBSwOin-R3ucoMn>%ko&_C*j0zo^U7&8!asE2FBP&5Mv9)DV zh{%Qav$ywauj8&pDCz{HsL|SpovM!upMBV3$+lry)aMk#9zmD!-fJZnACC}Y-_zJu zVfmHe_WV@8SzcbjjV53IM4R6Bv+RW2`#3{uTt7Ne-%pJt^n7>w7;7=R-}&drtlnFNRa!}+Ee;P zT0|0mSwz7nLeeGOuoU^RFD2>bbA^z2%GDhG_U2@cbhXW51BzM`l5C22&fuybmzD>n z{k5doRSgX~W5O@!>TSm985nZPa=oK=cNtX48q2LKtb-`Ry0V*H2L{Kid&1Jrw4r%g zasIvI-@Z`*)d3ZgnlvF->w5ch9?*0S&;=0YAmF2UcKK&#`#p6ej~EW2B08 zDO8;@Q$8oFeET+?{{=H8WybO8>D0tGMHv}XF|kkCb#>3huQhJFtuSw6$FCRe^%u6u zv8MT436Gh=rIqPrd)mqEHkBrR?c+Q~g;;vEjih@JsBpHtajjRQZryre4;X%LZ}bX| zVA4Wm_@e95wngVIjoaJ2|2guVO(O?g)p0KY>;*J%1 zT-SJQIE6^`Cm+3AEv$oaiXEb12DQa5qLnokK*-k4HK+8x@z?SY%?5((&`B7`9)lS&!zb z$++TLl`z2uDL(!@PYGJ4S^1il-b}(xqkZb4q?Jbj%xmza2uv6U5OWOK}GaLbj=## z<4wqVu76;Xm93f|PnRjphQ*%=+7HW4G1In^$eJ3$ob1gBm^Ek2P%a22NLs)pS0<*= zJm!((@NIAOslY-cbF|V&KXa*(&N8SH7mUSCU44_X(GOoZIsXQT_uCUw2_}-r{KmET z)mN<<{BJWezNVy?_Q6Uzo%tNCDKMVC0QqdY7b2=JUp`z^4ACy-FA>jr8%deeaYdTYWE2FlRe%-_x^3L#f-UBqa1vJ{1bc?0RzWk0G!-;k%r2L% zD-!xEE;djpTUT3;U*si00!gmvRC91(ApKrOhOiywzPnO<{N%%B3{On z=h&vK5Z2WVvR5)P{EA&VYam7Vdp-z+Dx|y-~HnqLYtOER&Gle^(Kv(9JQa8Savb`7Lsc3y!L$K7wt35rDVh|f=XHGu#? zU?;_gmq);+qo&>A`EMUUfDb@y2-jmDJ3G&9qM`z{r$_hiv)Wahi88D6$U`aU9bAn# z9C7Sesh7HlrJb2$hF6!0Xqt0dTD&(mw}{(n@P&{0IM28!&z?UNjMqV#ZiO4ycch&&qW{I1lXS5TOFCB|{M$zGM- zRl%i}l=3L;s6xXp=y(J#83iKA3dadWeOi=)Of`}DP~?@jYA>`Jti8-) zhU45c?_n1q_id2{e@{C9#dO>GDEb|8H5VT&pu(Qy8XxUcLbA$+kSca*#cp&K+9Rht z)@S5$pv)mAfm<^ZT0j<7;MDxf-t&NI_iEJ`SW93S@IKC}IXDn-q~QWKT)Y}N6tpGd z6B0+tW2g4w<3KBf#u_lIsN?$0EVumgQEg%=Iez{C9j zP}zbd3HF0dM1(MJ93eaKN*A6jkS5C$rWN&3I{<^MW->tf1?Ee}-+0^f7V(8-EbM}U z|h_^-tyoB=vZ!8@H5Ho+QXYK))X{=YHReXB)>fJAel}<2LLHaYN1v^IUtid*Ec@dmfsJ{WlKE?5 z>{}}b&TKI}ew=d(+S<>2#Fq|W-~`UftaBPq?UptOB6VUP*)CvHs1~zom%hnM3!bJb z!IKQ0&X=@AqAKB6nk}d}M@j(3q$D8uS>j+VROo=hHtutPH1SdKq$XeV=NM_ed_F+S zeol}gDa!f%EcrBQCave#uB3?_)ygSZ8t4^ej>V~Cgogp*_Qw=e@{;Ph7OUq7Uc(3E zzQ`$C+UJ+fiu(vf6^))c_A3UzIkhL6_fppVl1r=Oc_yYMOHCBxOdf7Sm6RhSB;tl(M&;R>n zXdk!oK|9v@Yhb@DKYv9-Tc6v__RLHZQi9x< zF$Txr_y|ly`0S26800aiazdiPz{>Zxg9CX{{`olxb22~)j}$jPAtKaio6!a(Z8m)M zPeNjbhRKGiOY~bOueM{zH)*{djE1BYx59s|KE1f2+8st zspz@B3_5|v`%jN$`!7EKSad7;>vUsiPk|-+Uw73T#0~1KpVJR! zx3#!?aD0zq+4Day<3F!?#x`2?;ruJz`@dgrldm>zeKMXUNcewzpsca+!_?WG(xxVg z|Law4ZBmT8>;3x`&YAt=DRAlq{Qp1R?2O?8^1#lZBgs=#ohPGr>jX4CJC-!|e|QPZ z<6i?FH@nlnb+Y8r4BY)cukI?|!a0oc!Gofjns`ZC&=KCyz{kVO3lRL>EJ;#jzIzQM zCxDt+Sa>)#S`eU5Kqbb%2y)Pos$yWk%4@BYn3@`9mkBNT`*#Pkh;noZuduNc5Q>it zKXAcdikh0l(f1$IKYD}<{2dHZ3eufxYt~c`vc9$Zid0v>l%+R6Jn}vBQ8MCC)D`yPQ32(*)8n+4nO$Vf5c)Sju z0Y<;oW6v4{BW;I6k3c{YZpAt@d?a*xRY7*oD}$RMXWKr!fiGB*QOd(3AT^cvsiBD1 zLf|Y=$vU}HMf&;{l$Trc|6_2u3i^MXodjd*AJ*!ss=2#*Q(3Lo_^H^RsIfnC<)ypA zY&x2Gt$$Uqm62n8zDxOOo0yDUogaZ(@51WpSWq)o@9;~Q0AkB5A(8a;?YQ|^ zcEw26dP$p6LS|-fUE!SxoTfqm-KO5(840&h02%)X2`h8E) z|Kr}5p|Z#?yLud|zM!M&IH1DOJ~P8UJvgjytvH)-DJ^X|_w8vrtC3h4te{!Nq{-;P z)JqTMtYQ$|oBHeWgB0{TUNQ8|7!7aZ1)oOuQG(zNjI+G6GdIwf+n)QBb-CM`7W=@I zSwtlEMa`#ltKsMRP4A;(VjS{`v5Ejl0A5SkPO_je60yu6GOMDiEAUp2{9wNa)AM4Z zta&vR^{3~uTGi9WOwdjTXw<(i6Q2b;`Wq8(@5dwrWf_ns*By@aI9->$OL&QjjG$

o&2A?+rE8ur>#A7HJfBEpe+ra`4JZ2qByPLCK2}ScQ4CJzW_hd{t(yK1y3iaR`8k-j#`;&k~l>hE+UOi>n%D^1?HzM|B$fx)d z_iI%{zkB4PkNe;Ox3nFvUxnaGXvy5?YP;pmrOEC2`lxi#I{6o&xXe%}O+lt!EcINM zBh6e77rgdLczWc~g1Z{$&wgArg5|ZEbG~7=Ng-Vfn>WCWXlZK;x&0>cm^)VyaGdKc zNqe`jmL<`1zMiM@G?P#fi91;th2W`3K(H3o1Q?r!S4f!E{0sSw%$%IDk7}~V$U9&u zF);+YEn5V;&FdxvG~zMS|HEPa%hWh!Mx8xqT5j!F%GZUW?GW$X_|jLh?|)gD^pWBDyDnVTD{#U)Kb?PxoO)JwT9b7CHd zHRS>|h^@A7c^cpftXgtud_5jsL+0Vb)-6>ofal0a6&xzMy`iz+iQsWKE&8~xm^9e1 z`Mk1JgXw2oAcJPy#@qpshu8)CC-XSY>EVl6{{b0H{;yxZ>gZ|MXho&zF>U-fXZ+}Y zx@Au>1QOqwiP<`2HA;W#HgXNW`-K#ReZPoqZMrUtpSq7B{j#dngW(sp$2R){*5>1WOPM-Bm&EE=W@P`WN-nT)j=Kt`LG#+8+=(yf~ zxw|~blD86s`@HF(e+#gGa$9ImU6UiVnDgO7QZM2rZL2kvQK_o=^FRNPV#T-ZlwwtT z^$I7=Vmm%Ch_L;I|9*%P`#YpSLYRl+6ym)fNLW zud<=8o1s3(b@#|x$h>pAhW!sn9$M;YzUi9PF+Di-`7WUhP3&YHEPB2!_9Ek%zVu* z)5rr+ewyX5VZEhBvJMsx@wa1CMK=>zZQw>^wXOHy_=X%JIJXe)N8x5CWUK`Gz&lKwYiU#nbF(FByNu>$IchSS@~dg z_sfJ3&R7`1e0R^}A{051!uX76jLHNNUTGsIZdnmiih52wUOi$uHJIMzwXvMQh` zgNF?^J#$IU4G6buZf?4#05lunkQ7xet}lJ>TCQcKFKa!&7FW~{HuCk{P*-g~pZw&Y z9<|Z8tQDhO8XDRE8KRIKTvN4LYx8mPWmZ!TW3^aj%2QN3_Kub3o75Ng@;)75@z%7& z5k5BP7E-ZEunGhTl1kUHJ&9B6vHKe znzk2K0+B?eD&c$j0zAA&#;ksrT*r+UJTd}FWj_FlO#|(|GNMPS4<&BB1Rp;9<+DUe zQ&d++0@#b3=}b`WS|Ae+hI=HP`>q9@eOUVMKJzNFq%18F#wSmpr&jDj$KhF5eS9B1WSXT|+agJ)|?uO6(YHJ|sW`iK*XbF#cVEPTXz)4J~I zw#v@YWQLKHL(KC*w7jwRSv-Fz8DUxP0$)jE>Bo<0 znADyDhK`?7YHOt&n#|EnuQ%mQl2?9l$wXf-sqiFg9D1U zSBf+%=7jCe0!%4!kmA4-@*ep?b&_ok8gL?9YQf3LZa6qD02h&nA0|-xtUkKOZn$Ri zt7YD4#@J&DG&)ab&Y+`6?Y8cTxBqFZIf%bKb8o*nsX6#v6c0pd)a+=D zquY2yQahO7pq~8-!p_BkxW=kIJG#LG*>7GmdS~5f zehQp5E~{!Rj}Oo=!x2!9A-WoSjy?lFYKcPVu*RaX)%dbynwRHNIN=mEuqfXKuL-AJ zclqD4*1?$Wmto&ogG3N^&?n@B-~kkJWrx%Fg;cR~`roX-HTWcq`SEo`V4!(sRyZXj zhD(^L#~E^X_;Vy9Nyg7lm;=x=zo4r~0#8L?acchl0X^Xg6c6eR>KF;rhJj0w2Y#jD z)x-~=cLVM0+&ba*aG(>MqhO#A9r@8n_$DiBR88<2q_Ko#i!`adR?MkA7t|-GW)0qV znjzxkg+*qaTE&dqKFa}~=?G=+XOn!5LvDvKa6_ZeFqF?bqbl?m0julMVWFy@y# z=Iwq7%K%4)tkvLZa*GkPC*Nt&-RCk($Y&ifzsl)I$m=j9muJ;HE3gh~2Oyq+}-T%?6K`NK(~ceY&GH&sdg73c9X=ep`Ra;IY@eXZme927@*ow+8M0 z!)fO#ou8IC&o&Y3!~=&&cFDsBC@v!~q=d1Z$bZj+%V_-j@Vk-^XKtIy5Ti={ z23yu(_j@s%aW1k=$&{|8aB8wcQ8tzlc?%Ah`I`TYy14&v>)&&}(bvEPFnORw&rTYQrr-s1ENqX_ybCt9G@JHAf zg!`SSH3yxjvl~0Hyv`OO8NAVUw$Q?Yg6vw$;YfL0Cu4G5B27t6{)Wbr!!6Uj?xVVF zKW}{PvQ%O$SO5wJF*;r)hAt}93@Nx0xegD{?cjJyq-mp+V{)U}$p5}5 zSV2F+gtW5(g&@g*>&s=)gAgv?3#+RmiCbKEcX$1N9&Lnn%!6H@vhrVvyKCC|pUn82 z=HeS&g!-F*{)S|X`81LRmr|r4#(s}u_iS$T{eJz`1`v(Ky6^9q>vO0iI#=d1Uj{-+ z{uk5l&3iCDIQ&x9(<3)FHa`CIha?`6A>0_L;eWxy(cYBM-EE!g<0AO+*B_D>GatCT zT#_HvK*lp{o@nXnDmgo2$1NkHK=T7ol!bFvKPzN`+Tf&kpW=j#DU}0JOkH$5`hR_Y zRA{BR84))uPD9{ra!T_^?qkujg%?-bHWD@`oBX2J$GnV5BF(!kTouV$n#60}xMCn? zeby7!2-J($_G^YbS)h{(eL(FmU)~v}%cAH#-p+dP;c7sVtcXzP@I2}EKS5zem z4%BSm*`)oiAEV*oI{j4!t4;5~h~v>mRmO9R=SKuoO7$xAB8l(p1Xme-9~??Cwx6F< zGc`?o$FZ8AKwVf!-yMQ2u9AlFduKFJ5e>E+L&C;3Ig&9o{Gz7u5R_akU5Y0tqFLhY z=|IT}L8HAsdsv_0=Eg=rSy?z}Y;9d^;Q++;(2%k$8t&hYXbBwkrjY3Nrfk~~nlj_Saq}f2JaQuJ7X2%!BTpKjoeLzKkFJi!?_>zx+l)I*ZqRLR zy%k!|W&&bRp>=g^pav2k{wEXPV5u6u&nZ>X#!9Q~Y1-$^#Y)&)}mm?rCkrtBx3j8Xee^YW;81#Ep&+!HweS)~d zr;{htzQ>xP9*t^?o#jn_uq;(HygojpVWl)*7EY5tgP^<0Y7-JbP$6U`x< z_O`}ET37c`+?1SXZgF*WRwh+=B&j81(!@7?^}E}JN)a_Y|>ZD}Dp#yDRmgVK?fm>{sr;aRS zBpj_(+4bWbiwj-sq|8k*G&Ch21f;osqj2czN-3+DFZciA6BOiCMdBnYZ7R$R>b3w| zWAHqWI6dL{_DkKsfO64^BKl*pa>=(gFJ29a>pPQa9=8nFeP?Yrl++`P2>9vK_{=}?aehK2{!2L?de{`!hLtftN!VjG=v_#oIB>(75n8Y2g_BD zn~X&L7`v&`MK9|6|2$~Ss96HIG3HMJI&*vVO6$p!J8n%(OsvZAV;s4OXCTnjaRIa^ zTt_E_JM8Ah?+&p#wC=Fem-^*X~;e{Vf7>x)8wkX;0}o{jEv zKAczD@;$=KZ!+wh**-6LhS;H0z_&?e*zl zFDYn^r2v~qXuu%Sx}ZnK{A;D|)e4IQ9feNn<4X#aG!vkL1hz*)!g&V72dWH?=sV8( zdPcGYlg52_lT@?y!v{F>QMsP+*cE=}D}aXt1xXuy41hy}x4Z)qmoLRdUL9*|C0Z}_ zQV>agRnSG2({+Y)oz}a#9RpMR^2-dl%XcdkP50*WHA70lF)mkY?0Oq=O;Z}l5Qd8j zEzzS9y38z6PZJCT@qJDXIttaqvT~P@N)!xTiPNQSj5O7 zMQ8|35?^Dty;&tWE2icC*q$R%_Ba(?yO%FPH*YY6m=57Aro8olwW7S&{UMih91>0h zYC8MlEEC4xl2gS>AwA~o|2)JOMCXb?vK`b(c^&kT+u7R}1McZA9S-8C_b+Fkx3n4t zN_%^^D3(k<<1sMtTtvS!-58j!&o}k{BKp*Hby?$zB&ifFR55|+x9E#XIcpy$Z{KGb z$rg%*m!p2QrVWl*x@5KBW+Gu_CS&V#7lYhQD@|P;Uty7O#S#pqk}x=aQDp=L0A>MQ zr@oy|b6)7G>2N8T9{WGLJ8Oq`*1jqtl5`mQD0-oVLx^7d@(&Xw)uF+A+U|=J@z_{7 zBX`}HXLYFKu7lb-_+|r|x-Ld~<>gLT7@-L$1zh)H)k5esB|RA=-X43LPR`B^C382% z&`3~^AH9ch%%1|maG=QbR^*M^rfnud35Rv~A)#>|m zdbKG47zPn`iwFur>5rl8Sy-TzIO`g1_~5WmbMu_};~b#9QaPrpU#HfO35UzlkHz~P zGkkEI@9((TG6jUTT(5<>Y+Wbqd%|y{3uX`E#Xba98v?<)N{#oP2lTQn0mcz-iD`1~ z5s4)6OG_k1f)Nq4OGr_b$Yfjr=JV~y7Z(>5Zy(9_BKv@q9Th=>g>`Zw_hkl?m30Fc z>AXZ`{C%2R?o_8H3SFs`9x`2~q`@VaxSE=#4n9A>!F`SxkgF*zpi}koLed2eA6j{p zTKF7^iNx8-SUxfYx13rq#SDrArN#1!N>s8D`Z6~-QQ$x|u0L>guzVm&7;UMg3wRMQ z3z;%hc~oVwLZo#x(xv&H1eS9pB7LvYaMU9VC)l+yQey(Llb}cYpL3A?uSp21I&g4t zDJA?w!TWytT|V6Yb;;v@QASF^%Pg&-ksTLD_UV%|2F6A6(U^#x!`vuB!jGHW$Vt4f z&s=s%^1=IiU;TU}`aZnex!Tlw4UhyKxv399ToTe#_%DsP2)Zn_#YMc|GkX)3D( z^9cys&xrySI-;e8Xs}p=mR6RY;|@?pRTfqPQ0>g^naedo3~E+1#jV**Xz|!qpdC!o zJaZ~~!wx9O@jre%1*tXqpgwL&%Ww5Igd!fhj7fuUiYj4CL^ulNe^cTqZQY`bl{v*1%g{bqp z_BQc3kYiSd4A)tyRFp$|oxKCC^C`c=7N%yi7AngAEM10Xd(2s|u_*^qn-S7GcPe~k z$x>TtAy{-mUI3`cDILtU2nt!e&*c-J6G}e{#yzKU0(T3U4ESmgClE!Vr3*Kv7Z#%E zpu=U!bkWIlu`rC;*n&KBwe)`z5rjv|8VQTk_gWC7q)-c0(iWDpRVv3-TEuiwpeN5e z8!K=1!}YZyma(3b5rsvfMg}^xWO~5KVzu=25EBzb?R!VzLr0>(ZM@q$l<%iV9Apd* z4h9h+@XM7$A_=3`UO+b;sCV?A`nt%X$wu@mLn|tKanUvNdXhzv1MLO60v~glU*TC< zSy5QtDXKiv(gmLefFr`$lR=j$YZS<>9{AD_F9vf^;DI(wAUyDo!%sqVj)AzK9N1zh z-9hVmS$Z?oC_U4c3{=%Ci7MjquU|g^%l`4RdOMJyDpJkhp`KAgG7P+LhDFTPZ!_Oc zP3_|Z#wV}4qHK(gjHi=&fws~>lAAfYk_0d8pp6L3At_m8RN$WDmbFc3%xo1mwque8 z@j%NmlH6A$jQv5))d({#Rj%@grhvPth65F%k6OoWuSrh!Y(T_7WFWkucCh#W)Q znP2&rGYPX7P!V<5_w_>@p?n!l*MMrFu?6r%b$1qUwEWql&r+^)T~`P7h=CFN>XBH8 zqRK0AEC57go!RAqQD|&z)X~vdfD!y_Ke1r2Rl)?&^wH2DjFd&O#G~(?O&PWR-s1k~ z@^nu@cTiKeH}J5ZIqK01Vl*a&0b)rxC8p&D`yY}5DPS*`G~KsPh@<-`N-MMRK;TnX z#{&|!;0$EsIKe|<**g(=xUTwr;>GH6HkPDi7$}`mR#AzRFVcB( z7sbkf4bi&)bu7il9!wP{QsObds2AZxGsIyPRFi{rbzz#>k`Yc5GG|EIOiAjAV^s3a zC}grhSpNZNj(G@dF6$8r*$}YxT((fXeys)n+b27#wTPz!)_*0b-|9&W1Nd-0C1Kp( z9f}KOKMug}bfAr9i+~0PD}AuHTc4m?EQ}s2sj5OtP}dQpPsAYOD!=f|4PQqeAU@Ds z27Q+zR{4#Ko1i1RMs4{WE_kd6s$Qz6R!Ee zU2(63pZpBFdKc)atru>tKQT-D1k`sN!BEf-pb%R=9gbN}C{Cd!6%|#}cHJ=8eTV=EM~a?&(Y}MWK&THyiv1LEa5}Z4+J5>7Le)X!oVtm4k)A%7EHBNccRhKW7<2>o;7rFXM ztde_`Ss%wO2nY3H7jAo72$^0)XMO9#QcF8n&T1d~zC28k`uj)kNAV+KzxGf~Yrl)x zpU&3`tT;kGaI4FwO(zoK;~)2ajX_;+vcF2)Qhh}x|98w7{c(61EkeEs&#>+pHW>5K zT(KH${on8HrtT*WTFWj0K{lt2l6Br!_ITf7pYy(vlSM{P)?t&eu?Yed2rplv2TG!R zL&YoDL<$4X@?xxs@eR&SBnvy*GW!wDI}#5--IzXm^v6i)&A_C>zSh$qhQkh4*^hsM5Wyi!@N zRB<_4^1Rj|nnY6J$7Q6*8^4&nqJT=0xck7nK|UwCbSFRqCEt)rBZb-v$hbh&{q-Kq zzS=e$_vlao+YJKl7(_H6Xe_6XlPQME+x^nVQDMUhWk3s}gU^W|qt|Va$wb0@A7aAX zoM<+lk?6_=m_b#ix{t6FaEOTE>QPmBOdxwS@wKZx_0v!H8RNG7ZrrOWiEGmJ+}9j{ zU;1n6Kon9x#J=@>cEWt2-S*f)iuG#>(-(eK%fn=69WqadUS&Lao`1~O8X7aknFqrq zdFRkr;y8xBo0IXQl^-)3%TNTa%Vx=pi?>XAQ60o{6`R_iHMWb-QHy&Ak3-^;3`d3l zk!Wu){d*Hgz3n{+i)(mKjr<2`IOwLIB$xBt@qkm_ZhQAel8(HJzxW>D%(jOv>S*Ftd8LBuo@-TbePH zEu2yknTd>~va%ZpA)9zvJp#-S5za%}aWGYX%wAGb3~-rCb1%GMut1{GMFp54znrR* zD6$C+g(IAR2}F|lnNqy~H~<6&raXc6ovUK&0ZRc=dS~TW^U;EX*eHr1C1CbLN`bKM z(gEM=Q)^19B`m~AL~5$$Q^VLTuY3se3YHg`dDvy_h1Qd*G2Cq;(!!K1LK6y?xsTf} z&Vsi+7Y#0iMTm$i5o${^d6y6R)pPsyLZt<=N-XzYnRxE6+xx9)5;Qh0T#8plEw_-R zr1CCT;bt`Kvs2mR$XC_j5HEfr3Q=r`fWxHY-Z0<|TWh<0+-1t3kk!Q|m#whN9P9PM zwG%$UQ{F;o!*!Iov^pb`m6#(>PbWScxqqlJ?km-ep8$y{ZVccni6ZZ7ir5J!sT+?p zHRA%=cgZF$c`g$4WKt%vWZJpe(ZGi_wJ%S!+uApYNl>q8r8Si()2>YCiyI1Yq+kX- zV4a|Tauaa0oZ^kId$}{!_Q9D!@TU_hX!!vlYrGj(#2wk%$9)F*c?OfXx#6^>TzKh> zLL7y|K$bQQHx@}9Zg&#X<8WYPZ!>5~JJI;rbC~yOKrwLN!rJ6olZpYNkmvG0< z__jGAw~MbwnR%q>Dp@7{$-R{(tsGJB%h;>4_VsU{e=5(zsa8i-p9k#5A-9P9sZX%! z!q*hD79}kkTaMB&MOfkX6T85;-QJ9LG>7lapAi`=S@9LDWRvlhshMdRP((|>?Os)T z2hhxa2znx)*-3u8wP&DD{9mx~>klP%T?kv_6_eLFlb$TtG?#ecOxU5=&Zx;EkVv)Y@ z)83x3UUP#p<8RIyTT9!0snWJ8)P7>>!F}4#$<4gvRCg@sxGzzm_xfP;SLXYau%>YQ zanYVv@|%_hlTM;b-pNk{C%!zRu(C(b{V#ys3qfEr(`P&PQ{oZypN~3HuQsm}0KSIK zXWsJW>N^>ES%84-UjXvT-IGNF*|YvW(zT9h$URo zQ{hX2gApkhZ|w$qzffO8lM?T%XE;$�cc%&ml=NQlWK~RO-!VmfL;@1|+_plAnC7 zI~#goaA;#nOTvZ{!T>(y5Dk3Liy_5s#UZ$;`b@jAtw2u|F2{N|=gmWq)>~Faig60* zc>twtBS(QQD7`t{d^*ki2n1ni#NQ-Eik|25s z3?MDF-u47Qqm&_Hi`RVLM znpj+!V9d(%k7(87FqFxEo*TFHA%ZDpTm@p^W zE*1$P0$sqTpkmx5M(zj}K$L#Xe-7~P{xy4hUe}RNCurx@)Dap%8sY*`5V59946-E| zF_Q*mX(gVvuB4_FnQopey%fOPCtKQXZWJ!P&R7b#6&bL53ZgnvFlEY8Cw3(A~h_=E>uaeu&tlHNmr4V|#eC0UW6~JP71tEc%6zL}kD(rW#!`ZDy4&2qVJ@wBtw)lSQrg=rjh2(X7(xv6SNyivnn&3s^c| zo@h?-g?Z6qic$dzQSJ{Qq-tjtHUF(#zMo3vjh^7 zXu%jnl3_trp@Wv2?J1hI+vbJ~B2(X{`UyjvKdoBtKtp^!LMLDwHLMMhYnmi_dh*&v}C$kU` ztS;;*oY9N%yjCIekKkTLw(f zU-n@y=0UOFSTlR;rW^fA1yx-IY@I!`U+{Kq-e9&UCiz=q*vwOH)!0zaTW&Xd!@ge< zrUMN%CQ6pI!-G8EuE=hWdcre~M&DN8VeQ4p*pKjWG(D$M=p`hYuaCO-)`Rycx{ZnR zz4f%CTs*#-p+_ejmZS}b@h6vh#N*?y_&8e6(A9&Wgy+f9H zGw4WI^AaF43}gh$&r<_;pG-^Kyx2897j2zg4#9bz?z~2lYT`|IgwP{74iVIzlyROE zXTUPyl7;>@=fM_!)zQsBIEP?(EAZLIXlRqAZvW?S(?t|-ITULVXD=5}QD}B~nr&w3 z@Cg~10Kb9Ofys|k`^C`pJe$?9YPZHl?qqaMcy9Xcph~R~#FM9McQ4o;2q3aA5Cl&LE7kSd*^dQF}o=xH5A5XMx+{^Xe6s($W zNsHTM6#ASwsG7x{(`S=AT1dGJWs6x5>aHyxOjKqbq>{%DDGD)S%7NscoHYKtj?O#y z@hAAWly;gcJ1wDCj^#K{+eG008GK0HiDKATgOG9x5s;2BtP2AGF{N zU6N7}$jQ?w=-#1bxX(P00+FKjF3%ZELRHP7Vz}?>Y#c6%o@`#JoLe|V4D!xg{ci^G zKF`l1hB;52_dY58**yUmhm)w#M-`Px*D8V`0;h;IDkw{iLTNzz5J2U7IuGHPi)T7r z<6xi}Yk$bo*Gk7+ob|8E5+fs=ot%)6LoBrh7=5)O0mc*LezoZqvCbZyCGo;0I9efxlNCJzUNn=3Ug^W%i7Il+$ z1uH**E0K*t_ewQ0vGS?)__jd1YfdvNjx?D;O+ zM(tO(*CFT8etqqeTB3^~w!Y_dxqf1Oik}aX(la;LR#AUZtYS&z=G^EczGZJxBFnzLRzDK<@)se4BU9oNjJ&71bm zqfug}5%_r~ov{z_6m;v3hi|)0O}P_ncCR@7JWr<@%3#m8n5V@KkUW0f(6wJ^_#;+v zS%~nl_T*b?QnGIT7^O<|LCJk~tZ&v`X$m`+@$qDgUmF~q`Ccug8hSZ9qV#0F{C-99 zXt(*Hv$Sf<@2596H8T0IalP)tZOSF4T08Rh-)g;DU817cTp%Cr=%4XF6D=~j zaI`=*ispDddF;Fz5Jnw=RxGW)9@G9^C5{VRDYK> zv=BjFnVc&BZ9hNn*_Wct2egzu+-+mg<6XZb+K}7m`#GEA_O35&GM-TO{hOCJ-aiv_ zhDb5u79-Pu=s+g!F1vwFhQKC@PPt=Y|-zA4Y zYNJxw)ivnmhOe)W2v1sI3l+oZjxz=MOSoht!uQX1WRf}yD|G+&hLWZpIS|{lh*`LV z6yQ`s3MwO@^vby?VXDl73uZJm9dpAw9|`MRZ7}B7v54saA^jZ|Y1V`!7qzOobZ}X(d~`N{oTWN*Su_>X`%@p!m*=<~EOakOyBv^k zkAh7QOCBEv_&!aq8=I_6QSuD0D_BGcOq}3Xf98vi9Mba&30E9s9%}Rn13W+GWYlHo zd(61O_5iYQJT*KNFE6Wwnvm9!+lw?pJNXW*;1bJJNo^^D_&_G6Ftn~<69twm25v48 zTC>M@az%b%^nZj-gL&psbG>FEkJY2DE$+cuPEt_si2046SX0*kodq<1^C53|31del?Sqtr^0e}3?tnOV6RX&2aQSmWRj^Lbjs@;DQh zC@EUG$BOiM&Z=3Mr^m@2WsUpZg410vwDYVd zJop_t)nbCYF#Ti5)Vg-b$;NOtd%fv?6A$poSC2eS-8+tF0%8D(W$+m_q!*dG*$3Gu26Yc6vmWUm;x7<$f-nhr& z+Lc>a`Sms>e&`hj>wbN}y@4;z7pwWKokw6GSCMc$O5T54C(RfXZH$}FWDBE@^x7Ha z{$*53>e`^P6fVvM?Kwtw$i#KpNapr(MeCq?I5oja14USDO%NX=du!EzIS0V9_wnSs zxa5lqS_l$23m4&{AmqD#Xir6k0|kDkNeXK@lg05mo!Di+M3M5gSw|pEkrJAw;VvG; zz2xad>nlvq)ItY`GU$O;PZQqW28mqTP@XSjFHT!%(%}#6m*8`RYw-KBWRW0f6H0t1 zpxU;uoU2MgZ9Tq!e~Kj+PPJOuk=~Vf4B5tE*u4;k_o}XS=)jmgzdp$i&9+c#X@9ZebKG3ke3i;vy@Q05!ry zK{Z^c!by$UFpH7*0flQzGm2P-c`8djBxjXLg7bSGtJvUFDX+vMGbDP8N<07DR?#v> z6?zR_|L`D|w!&33SuE5Hx0=0qVaxgYfccytPc@$GZ}Gb=w$3F)B$<9ZU%B>4Hg@6Kn!CHEsV0ZVExYV*c&+B2U))az z?YB2>9WSmdY-BdMd@HXNS*6z#$nv*s4^Daop$#T_{`|Q@O?r|1_{p&TPeEG%de`~M zmphI12JfX5)ecH0&-j%4V#J&2neDd*hI2k6`yT9r)pWeCqdy+>-Fo+e0kN~&`Q~@8 zRea7T*o4m|C71BU({lW&KqEY+r=LDy$kHHklHk=-vuJLPmM_;H2V(azV@uT04(de& z|6aXPSYMq?v_0RoHCUbt*N7og6HhHb-_%O~8^GXQD+7Q&OZKTEM?iWiq5@^ipqrWN z{p6m)RWUO-E2Dm?!O=$7E+y8K63Zvo?e^?@TZJFz{UM`;22-<3&5Og1BBi5RZEoMw?Hawa2*nc2!`fGRiVb(YL^nI1&c}z53hg$!ywA&>;M<&>y_U-aRl5P%ow-3yNUE(BaNYYO!}3uQI$5=0#?z1V?G$TBzmd$K2)DQXU99D`s4U90 zJpFP3z5_NMf_4!e=x&;k;bE(<;es!?i5=zGH^?$W`W{b16lqEZO@NxcUM2GGx6XGV zp|W-U#ZzPnp~ywJ-pkr|%ZK`H!;5#%*bIL?S9jpbH-5D94G$in=3M=b_)vVC& z0%_EW2oU)s!Nd|s};hTnE?4nrFk2wei5z(vzSXuO29|ROPw+}Mj68+B7#{Kt*+ge?wSye z#a-ioCId#HEm`)iLU*>z>+8an^Jr9r)(Am_@39IL>>kZBk&W2>7r2~%d@#1NRk&g< zQoYtLKQo;jyP8}o3Z0j2bA1`UITRYBfjCV3WO!iRFbG${psCmYD zxNDIQj~;hUl=JgtjY^(8F5Vbjv6Z;V^tkMkx1Ak4r?jE5hFZL~xjUS^IGoY`<19BFt{hK2*zr+%b?6^(K{l<>cYAW4OI&Zd<4rhU7FB``*)}h$Gu(X^_ zSEH#DzTK4U0p=v%*d#`^?rMzk^t!*Q0QM z&AP$a82-BL;ED}>fXrj^bn>|B+~~OrUHC4YrRUwdyUkAvriyN`lrBEy*=KG?-(yl7 zyFTzRFf%6hSM3Ju+t2zmL0yt)yA|KaKA!a3!!W4vQ zOH+U&c7Iflw;0Na?UmgUw126v%lyQBM~gf@LCgEN~dAq zKS|7mo!_ONC*|N6D*x89;ojdwv?lf@}J}HE6qHqCS}tbz8~5ykFE6tSOXWV zbITQ3hzqUuokW`NjIyT3RetSDJZ$a%_NLg3umm9R3z6(zk^i}gRK`IQ>VVFa6U_@k zgt88(6wfBl(=Gfs3t+5pZ}A2Fub?s>74F{mgJ#p;ti$^#qN-iCxpsHan&)8?69YmU z?OPL~EG#zC%@?-_Y>Ll|6KrPqhI(U=KkI3Bf7X+@@x@6-_H9$sUNd!v1HnbIQZF^} zV3mhca*7c?4PJ&`jSV`wh?b5QGe<3mS>oj6iw$Mvu)4Y=`L<{?zL=fu$nvBSyY9~` zbEHUV15MXpQGq8zNc?M?IIJ>%E^QuBX}dv+$jC@g=2on~IT~bUW0RJKnno|(UGG}} z>34yKe4>MFlJ3`(lfPWA0~$drl;i)&%JXcO3XlAC^h1I5H8lmfVz5+j5R#S$(GnE` zVWI<#f1qPH6?_`=9mnHNlTNMJguMNB3K3t*!U-aPOj0Yt3;6YyOISceVZU|mv#uvP za3YYPg@KM;S!9_b(B+K({w)h&+}gdm$;uK1nIE=TnngLwlxPvlpE~c&qi<<&dtn>d z;ew4ZK#8H_E=BP8DF3{=(ieg9W)T&@dJzCI6|GExC21LG$<1%CP9NM8NZLIPu-9W0 z8iyuYL>-8U4(K#g*tWt((hkfS{N%F)prE)$74XY4(?{R<%?1HZmCYu;=tTz(kKdiQ zwZ)ZRT8=ePRjYrz)fB)GAM4 zn5m6h{mWGIiRMA|cz)gg@8Me+cBpZF=M~s=*F?mLTG}CxIbOYD<$L-gD!N{2Dc5Gt z!+Uwl3GLc;UKGi00$iS^OMN-*?` zJ0^bS`8XNXRvq|Ab2W7OatLE*>q+lHZyC8x)q0`_k_b%D?ShEH`}4=le3;2)>D%VS z-j|%uxh;%NR1_%NqNo{|u8n7g6xpA#m3{77p8Azlo!(Y|Ss|1kO4}*3aa7uNIzD+O zzxRiH0}^w*>ZHGUd*DHM{kXYRW2E6s^Un< zL?0~gYQk?_Q6J%gG2bDaGTcdAghoQFepYT>-_R*{bS!v$amOZ?N`ZaSO-izQ5_iLT z?n~G6Tln45dfOdE1=ob+yVl(l1A*~T{dh$gkE;N$=?rcqqAWjAe6Q;NG|COw`Uf*=WA~2Zx2JsMq84}?dr2IMs%O+eGilGo4TDEy%=b_ zOI5bzPDE4SsY=U|D5+R_KbEU<<*Co5JG1C;1VTKS3939T9n>_T@zIuZ z*Xmai;V#={kdT{t4pe3uSYm#@X)r3q-<72>FH1X05D#_rkj%2_F6QTT=H-Y(kZh!F z5Cd?H>>kaYRlbt&Uor`kqugB%0RiYU;UVF9_3xw>ga}5x-7DDobw%>USOjbsWMsiJ zFWq;zDGx7f5D~;;6FRoP*&9;hGQ9$_@l4kl8e&c(X)`lqAS1<@YI5T&8$hbEWicPk z&#A`+np4tzC57m25F$v+$RIz+pFZhSFaW|mb;j@!5cQcvOFU=@lep<8c$t~X#j%)7 z6H7Jv*>1mF_tVLdnno-hUo(ICI2_$e6s$N{1>AZOfn^%wm8n0sgi4mQrGr;3Bi+Ad ze~%UlM#sR#(-L|v)ETJt+W(gGnN<4*7-y=T>9yCOq^2z^C+D$K)yLJ#l{6(#_{}Bp zg|?|4VyM+GriRZ_s^c6tOEf@4R>_9J5BgHN?;~-3G}`jMH2@=-`X_XZm2GX=HA`85 zE$Lw@S;gca5?)8OZ6u2|GgAZ=O%6<^e+|}n-Wn|i3Va%o^u!PQUVl6jYHh2(EgER& zuL4{_UQtnKsaDe1m^$!5Mow;?$LM;J%U9mG#R3Mq699c8DmH8pApT9F%Xp}@fQHAwe+|I}E}fRMT1}d_ zd;Q%^LrX`806VOTE$Tl<;C!Y*;76SM_ia9@3ldCeK4gNoJ;e={dh8TSw!P-PTh{fP zM;jMIaRTzI(o_Y@+;e1bL2|GJotRP-c8jpgwxBP|wXt?~U^F1GyTHj00}3 z33~6vBa7)X2eXLKk{Zsl( zSZNWK$nQ?)7s%F}P01K3FAmcKi86g|H^SMbB+ETfD0)2xiS5@WFyNi=g4gOZ!ZUw& z{uZ?qhvmjgG^U@AB`<1+F3bC7S9f_$T}XRL@U;4SBS{IVA+Rz7#`7Zi6(Vp1Y;Q~xK=o9J<$KYLHJwORS z(tLJ0UU9f~#?K~fi_A(!mMUD6uGVfItxLhOHF>b)`Tv_iU1yj4Ivg87CNVKFt8MiO ztbZrsM?fB|izbiR1*x6}^viFb1;%0!wqU7@lF8bx6=Phuc+sp=;d7vRl>L!y^0x#98>1zDuc|vkQ zhWh(EsH*7ZO4%othUG{{78bpe)%*+61QwfVHu=p`$^QUw^=VyZCDB%ZOv{QBY%z+4{wy$d_ zL{YD54}CowaIJXFS1^LXqh6;(WSrywsvZfmLV|Q_k2*t1v+hVq#1*2AE1vcynpyzY zw4^3UZz2iI*3JYAac}QkdV2cvLXB{UsTR>}EpxU|O5V{+X(Sm0lRyWoe(B`g5doHU z0bCZgb^cPGry@$KgA^bU!60U%f?_)IX%~L_6b0~LU_sJ!f|$T`R%<9-PF;_mN{Jp) z?jYZdkLgyY*<^{_5%^k*2$unp!*tkz;F<2GVW1)D6WBkrv7!t1XmX#>E4Fet)OK#9EQl+DZYoA+5)c zA3HlcN8VU6{!a@4jBcc~G_X8{>CX*>Njfw%Oyg+pRSN&w&4u+B=>fSgXq1FQ{*z5* zy1V+%XzLuD8(%s79;%>PC7Mt6Hp@JZk2~B8%i35PfepZ)?5@x6-)4qTep#132xfub zA8otuTs{-2EX2r%CBN!=axXm{62(0*8;Hc}z7ZxmgpBiUz4f;VLw$rOg~7d|xiLkD z`h2Px?Q2?C4^&AY1Y_z3;my_b(T<|?c|qEMu_FS>N26G-^7_2^@fU&zQ*#xsQs2M1 z_ED|6PuS0+eslaHyiaj>2j%TlyKm8R=W)Tzpfp&@D=R>wczZY|mn?5?!N{ExoRDvN zPJqetaVWKKKwFXNOuw0;sw=a4{RwOsu7^K%E8;o2`{({I!%XOhf^jPzFzHc=xNkUq z7|!J9o;G4`L}vChO@8H%5Yi2?T!KB(AaN4jpGRqIa?wRE;Xe}IXzPUwzMgJ=nUi^W z9C_eLm9*v(GHA|S>?+#IKHJYCNB7crRYz>Lx9glx1Bwpp56lrm=F^uO_B~A;1tu?o z{YQiTd(&6r`woxn@k?2}&iBkF%fYZpDWJSb`c(c>*{YQ4H4tCZz}`G~n1Q^MBZrXK zQ)s{WEh54i@Hf_PWN~EqHpbacMM1h>>!F}%B&IYOczqc@TBCkz{TL}o;^q$5*AJ3C z8rQcX(T#|Nhf%4f-QHR2GnBeiSjF9Ub&4(QMiv^jBAp+4-co_o#~qD^#MW#S)JOYS zZ>6yDGQW6iv3Jk(Llwf(9_rnYk zwRInq8A<@m2^eW+YtsNzDhl}Uz*7hcx!{!CKGk?3oT28x%v51j*%UO^fxs4*a;0tI zbQdNnf4jM%Mi}u*4{-p#^o2JEy-bPGzwo4PuWQ(>Bx^F<_F5uM7Rhm%&*vKFy*xu{ z+2;pY+<%tt+3u{IK!#ah_~Nlg>E{eplLE46D#!p`K1Yz@E?rm{sBAFm>JC{uJn8x_ z9-Uk6N>fAmYwlF%T^-#?LEQbCyMK-Vnhgy#jewFIB@0-}k|pJ)duCY! z1{xvZ@ypE2L3_W%r7RAXfkC}`dul+#&L=em(jt@5tY@o^LZGr`5-o;HrSU8QY&}}= z^zC)%)+R%WlBQ5Go^cK&QOBWLFVySmIVQH6m>Y3SObpm{i?+75@=8n5`W#XRh;!fW zNuty8mn=aL5`);Fq%elx<1q`{56(|ue*Dj202kUCSsWF;XCATpyt-SS^Om3T{!f_4 z+PoO*1g^3Z#P9aHyb+n$kI4vYvSCnSgr@%U~qi$lh5~7yuAkk)E8yHkJ2~| zhLi8Fjh&-G!NFs(?&dF}Omq=!{=JxzZ9%F(mP3ks8rxwqg{%^7A#t09Li48x3xT%h zzs%50*1b5)90IsHG{FW{oZa&$%*ezzoi?*pxWQ>-La6R%O1Onh$qwjg2MXjy(P7N144ZOTg4#D$3C-^mIRABfvQs49lo)d6{ltTH*khiO@hq9P7y!>V~^(AtaveOCh3pW9n>* z%J&{B)FgwEyz1)o zo~Y5~sy?IF?)9M9!nWFcj6T&mM-O!_{Y*&ALFl?ud2mfq4EJ-aLJZT!83#gU> z0SH=2Xf(t!N#6)%Xr^jW2?>Fy2ah@2oOVh z5`Swg;#{hSbp@j>2i*N;;3=j1hE;q}E_TOD{6&b;FXs^^w2QA3ZkEZVg0M6Nj$*S9 zk<>Hun8`8@W<(UTL5~KYOSl0xPv!UTbfB6qj8r^4gb1{V9z+U{!4x#pXb^SM&>*2l zGL@EhQ=$iApo7jnr2bv#ypM2y5rMS%a0dw_2;DT;0UfRq8*eFA8V{}67N^e2n1O^pij+oJc}KmwY7-r3F*P!7sA3=I!lQe<@6!Er!GbNg%YZ zCJnvzlxLJ>uE>x8oKwK_2P^RJ?rvdmF{&vhfpoAqTuEM=^zPLoEhw!&BF*AcD^LBi zSHILWw1XxorEz1El^IF5c6qS82w?;D+WroaE5lTu=If0?zIY}YzOXI9`tNlN+N96b!{Tx7U!#DkNF z*e<3q(;i(`;IrvBX<%WEklE&I^Wl<$JT9b7qb!e; zcdo97XQuj7wIK&Dk!Fc;RhZ)JF`I|{`cA#?YOohB_C}}v@I)`gSP66_U*9Egy5Ssu z^Si{fHYfT3O0@s#v@ld0#sKb6;HLsq00H0APp3m!AI;|)HGD4zO=@jtRZUHwtgZP< z!=EDrg-c94q_zIQcUD#+Nwo+#6prg7?29+Yilk&@y$;nEx5V+`+4k`8N%HP(tXH!} z(rnS{y(KI;py^JP7p=|@>hDSnF)PrgPcb8c6{!|iB3b9Vij?_9ZVu*vXnj~{14*yC zAl&A6Za!`QN?cQP>~Dvam`VHJdKOEE?(Vd`&_&p_pf*co z=^ulS)%pY@`hqk!8XT%1j(v#8Z-GL+2w#!u%a{BDPZUI^NGhfkX>o4_yK&cM3Y?AR z`vkU^P6&76kVIOr5||Qz-uDqg;({`?7^ZBmex+~RL4v8FWfVSzh>pmV@UtI2r!0JU z*$kY5_~;?2EGdKB07uJ4+y(7I(k`s`M4?Zbn4*Casu7w*=dC(gHD(ygPSEq@qh~L~ zkzuQ26VRu~|CpSKP8e2GSC>#$#-fM|%Qxd8%^PC^Bjih1Kq2hImR{nE->)Ka2KW%` z-$m}V=;O}jeQktQjH03s(BXYAT9KYTUs|tDOi9r+G>qw6S+WFEw}BA~(0V~muU+`H zHul5TOSW0bC;#4XgKl%C23fUT)j@{Tr~>t&uIKzV*isIow>~i`383M@^yAXd(9pm8 zz-Ebu*BcHAA(|l6c>!*?8VQnGFB+;|XqHdYcZTJm^0EQ|E}pIZU)AgwDYfoQ;cj)s z0#|yvjYV=N>b{-VmLmc4$>>6V-8lTKiP&t-9c!U;au z4)y$Y74PY5js$c)&&{yQ8`OoK&!pOkgkj7F*UxLk`VL~7rl)%-6T?w)7P)%tBUhi3 zby@dw<2%_K77>WUFA>*1w=S=E+um)8T>rAWTv!n4#PIU8k3MQt3jM9@R&|#)sONOc z^ZyCt$}5M!QWoHE^xoEASpKcwHad{hxgpk;sK}Al|AC%@!s89)@bInwV`#ffx5NZk zhv57JcoN{leg8T-5A@h1wzkT2TfIeDbp$Fc$G|)*b4^`cWiH)7TU0e6&uQw)dL6@ZfgBxN8p*YprEGG{lE&jIn%rj|pD$s4K>H8;li53iFs zjbr;(Ov*q~3GDmwg&O}(k=g1}TV2TlPtkfr&^lHlG;@Y4f3C|&i%PDx+Sr) zMfdFbOheRy8o8%1yhmbyFZqKK=Fb*4a?vMdRj9%m;V zKbr6LvEM4Wn}{5ng?o~CctbooOR!b6)$7@LiNr~B&;*tCf|@^_Mb{L}4_~1%D*J6P zvQaMfQ8Lb+Cuz9py{h*zdf83qZ2Hu%@@D^xasxS+@9V=Xy7kGdB)=TLYV|Sr9v;t5 z&j}6eYkIz-ZEO8R8dhg+`HC`5tR<;nsXoo-Lkp)}UeabkZET|kWuF}Bj@6R5J}<|3 zh^nEukR~&u#%bpGA-7@em!Os=?Du@lqAt*#%^W|v7{scg4fi6}VZKv~!>Twx{~8@m zvyo^8g-+^bBOID$c6aObs-Z#Wcc{v?h6ObPV|7bz($8DPpV46gg>Wt=f@cT83JN?T;cM z;GlyQKvzBt7jZL$l_3hyh}sDI+oD$5QqmBcM)$|7FoIm>K~XA`6a;)QfR&>ScwIr< zLXC~E&@jW_<=;xb3|E>Bn%m5-R}XTmuo_>I3C>?2b2J~UG;ci1UgNx5w)S4a@cWne zfsn5cq$d$)XRaXm1M8c{e{>_i$d_Bx51|Bc01vU(0pvK;8R44hMk8QQG$_85UR_^L z*V$2mPD!bVC}sT2G$_{sO%^F5EH5;7L?zQXd0J6NQ|o09)GXSO&lEH5glY3lv@?$^ z3n}f#=y-+R=k%rp3EV&Xq5t{zQ}AsVHTa2g5BIxR9S~L9Z_d?`2xjEJRiPC%rUusm zM?hdqF{>|*>;6GCp{>m)Ah3MunN|iI`G!rdfUoe+p4?N-M#ImGCdVS-LBtkbv1)nGN4>S=&8r9nx_C)Va_4?q^%92Z6p(eABBy#zhBtQ))`5}3W(`S z6LWLQPENejHpBo_q^@l=0gFzq2Ghl5xhAAgF&VI?|I_JQ3C2}T zP1-JBkxKc}yI++$@Z@Q`-Mf1{oOgTe+)5kl=JTB%T8T4CYmGt_eg_CMaxfzqtlR6_ zgFlL#39p_unLlAQNADZvSRXp~i&Z376iVyXZy#{;^FPnz$7nmKWtis~Y2A|gENqKn zm+mHz;(LU?9-ON38mp5@7tSCciW)pkFY++{Md zm+$&&^Oe>L_jZzoFi_zX?-+X587a+Y=hcf?S3d5;L#Q;-7&SIOIO!Py=_vx=y z=ysE`Vx+xG-N^Qg)Vb1B%E!JHHd^&N-(~D~raw&?S7Xi;;h3AHTsLmJ(eb`7@3VZO zM4L?FNA(=8(etO!!oGika%tV~n%eE|gxtVo-6nQDHu^pFf1Alia1IKUN-dV2`4+F| zbXN^K)ghma3?(E2aAvWcwn}9Z6EUkzq{KEdHI+?FkU+cUA+H433q&T7F9WQS!%J`7{5KZm%TZT~k9bX=s!tfQ!D>=lX#xX%=m!hlGyKT>iP2lI2yVeXu__ z$L%gVlt1zJQ0F#AK;WZDkQK@}#0-k7q5=`ed->SBUi%?w2uTu`#ioM7Cv*RoO8C7{ zmrbkb7<57%?UlnTwO$BF%xqXB{ALjhCe=V534+pETi%KG_D8n1w)N!?9hB{3^_P0e zB_&AOMm;56YN{YLVj^-my|_WnX^wO-486XR?+z_tLDvLiGuzqbtMo?mpCN*aW@d{@bfyQ8PRvE3Pif?Cq5WyPjp40-!O1 zLKZx>3yNo=ltZQDAVYNb3W5vvsAz{dSXax(`UW=bW8!6?#Q9ft!hHPX$(wQJfo*fO ze;1%MF*`d5m|gaUDW%|~Fypn*{I{s_86dCJLa)8#F>&zFQA+JBK?ywJUn10pW=b@= z5*g?7A*FVIvHS}+19Om9_L^`Vn4fHS$rh#Pt#qVbcf&_!p;Kc~Nzi4|YjRI3w!U4+ zA!y` z^af6+Nj6iLQfDXh9>O&&DF+d8#)AZ-ejjULb_zX_))(Dg6ziYdY%CH>mu;>l{2ZOG zIYtbZg)g8_4# z$Bs@oQ3DpsOQ&YMq@kwre+F=C3*RJk@!8$6*ZJOii7fOuhaA_heqJL)e%8z#HWqIp ztBGx9gaZv5>v)j`c=g`b8*x9Jn(E#4gcuVH8>7#STBC=>7E~h9LT^WJ)BUcR z=qT@22RM}%1#k=f3UFr8;^caKa-+P|A-%OLB@f)-YVMGeO zm>0+TH2Ja5Fk?S|axFYy80X;B?!}?s-A)i&nqIW3!OJ?)A=TDnV;kFuJSFX(#`Cry z^r(cP!t1rZhK;m2k@JhR&jzkfEi%7_u3_;qV|xg7%kx5aO_`EcEwN&bArGph|D?nu zIE=bfxB!a{RNvm(x|OIEI!mjF%kqGJp^Qq}ee`;I0A9 z9mQaKE7JIMbZ-PsIIKWvEuHOl_u4*Mv7m^!La?3AOp+rbui{&p4TCZq402HhEF*wx z(_C{D0yzdL6e zONj$BD=RI21gH{#q_XM{`YkU8laXXM6 z_e3G!Ake6CrMR0GoVsYYB}DSpMSKhUd1!Kl`g%1{{x!kND?LHRRbFZ?|=j0YJ7pYQ z_a7p+FV2l`xOw450@|NMyBX)tnf$!21z>Bb5rUmPl=n2(D0j)mBJc@|_*?m}rHwmvWP#zDHpfHVE1VjLK{?eQG~VSXjBDtmpkmvEr#16Q)c?T9#@I8m}wruM2$* z%rZ$$`a-=f!sbRD6(w-v*mFW{o^~Akz|Aj(B_`)bvwjv1(UcbZ|(jq z9Ca{5Ttb}@Nf3kme!v0K23=8+iRDmIHMX>N;oi>MdHhRov%uCHDE9DjT$?+?1{>45 zZUkfdz;VS!UU99{^|&ouR;U}ZW^;c3u$c3YOu#}Ay@IFn`+xwbRhWEoN~7YtWvODB z%8Oh3m1j2ICls|d?!m%-M$+)VeWv;#h*ok84){xzfTTLjQZ7~z@c)%og?*x9T?s3|K%%%ZoM}M8fT523{B7n-}^H5XSO_Jbe6X ze|Pu{+&*Sk^JM5M9e9p|QK=sW*W6j7xdD3=PQ) zEI^qd5rc>hF_jMPelFG|Akw+FkwwMLg9-FkP~gCJXD=IoW-q8pSP6xgeS5`L1Qwq& zPgi?@ISUC}UgcN9DmKu>0DrGcjG{os(qSJZBUy|FpqGYoF#shh0ECH9$Z3PdX$cFF z6IuAL5d>tjrxMQ@$1$BLsB{gK3Tg>wX%`*eb4fhXPWkm67i`NQ#BV)HssIJIfN-!` zZ|?7JRP|BNf_c55Mou5diCb{^goHQWgnr90_0d4Efq?u;J4GjbA)!vx9PY?}=A^>-_pT%2mUW=?-QPy9mU?0$@gyb9a1 z#$M05^xg0h8QmD=isJQW%28Y2I9;|DCcoU<9j(s)Mp78}VRJXvN(aYI`VW^G!N%b< zxmeHFGVgh8BO~Nvj<)m-BfU}DPcsKXqM4RLB)&Jtf6j}yJsjn5>{R4k>Z(jvWCf07 zJ*}py<8LEZ*b6+Y=@PqyC|k2gQhb1otainzAh|1p0sYhEqwTkJPOn1 zJjF{gPs^oYcs&h7jv36aiF+m^r7sxNgRYo*lT}LS(TtGmKH!n5e`f8wkkiayxe_0u zc+p*0%v-=0)pe+emhSp4H$re(jr-I<^sdypgq%#mofjcupDuE=9(1AGfjEb` z%kl4L!7GYPpVNAZ+YUT<7VepG*k@&)s-3B4kF3u*3pnXYwgtvJ)vu-udwIiIlH zCTh81%r@#nm85YIwC95vl~Bq5rv*^sc}rHTnmKe#p2fI#OeQuV(^bo1q$xzWW{D=G zJ$`wzVMiB#DY)n3iqTrF`aWX(s}pP3itI7eXFcq)mxQyJi!D>FtiY?uYICIluE*NcGU0OJoX)k7$*E=*-(T4EN@j6XiCPKpC!&^EJ4=#(? zz9Qvke|*kWBBN@c0&J*wXAgNfU|B*SfqYGFZpS+{yF2r9vM-X~e|&rnVF?F){>|0E(bQ zAan>GdG@4xR+yQTKUSBGL|qMgEhHb4oHsV`r^(md`QDFi3t-~4~T%1w+QU80v zxXqvD5PjcER$QY56^p-HTQmerxz7plsPPAj96S4!M9&{5zKBI|fl{4}J;I-pPp`A1 zUqzoV?R}q+@bK_JV=>Uy><1cjoO+IQ8a8u@PA-}m#^(m|jPHdBWbX)4Mk$nk1=;4Cdz+aBg?C{HzEIj{a&4T+S#7bF-dH^2Jx zmf~XH6A|TtmE_R*_L~yZ@Ung0v=C;5tfj3j^%odRqUzkvrT11%_C{voYgnvwc;`>a zpN6~6^dFij>T=Sf#2lXhK=-4G5NZ_&8u#lNnVVa&B#py`y5q1A)v5;Oh$9rb?n;LQ zs9@eB68jnqp^dV({09Cmb{vl47QwD4IxAP~kNgN8>V!sQU;XI!)liXN4JRx_T%;fcN(8Tln_p^Mj+oPoRJ)nDFt5 z#Ym=s2xq|C%mfw3R6aBckAnEiV?_{s;G=thHS>T>gM5GU;cH!=$5yFPGN%qrJnnl2 z2oE^m@FNIYak3aON%2=Qov0v$4~Sc>B;Uxlq3+EY<>eGFDpZbsDq5|N#;Fa{d)Q` zlh2FeneCUag74&}6@zK>6u=vd6)S?$4IRXUm@yczH2X z3dgI_kq{!O5#nBz1d_yZ`=05Ye63TvztN;?6rPrYi(^800w<-b9f2}tsGw%l>|zS7 zUIn7HS?urcU#~EDHvHy;{6L>_Cze@|Bjoh)4Pjz*2~BYc60S5{obY>LHVfcRE6ByH ztvZcZ3^5nuVfTp(N#kBP*M9vuRv)DcANo6%-#;?a`|w=!DSvim)A7A|`l9X1g<^iH z)8GI1mrJHzhx+cey+B==f&DYA^TRlPck;|Th5-!AJJ94_HFoRz@E7JS7%x!NNc zC@^LX-rrs^nRECCg3(BUNvxBVg6wE>9je&l(xhWN`R-QAKFxl?j#fT`Toojapdw{K-|V z&6U2QxjB(&yRO>f*46XIUo^Mbv5eI^6>f^IS*%p>?vJ}H7AN)c!y|+ewDjeD2nNX( zxECJ6r7R%*|Km8w{>OQ|jUdi$BZ$(u;h+=1`Z*5W+aKf5xi2Ct^@(MD&lDmG?xL1P z1mT1l^2DPB)gM196BcgF3C`zqM!-x~N!wlc%a<>8n_SubZdOcf7Fv^-61blo*#BQx{l$8hKd=sTia>F2B>#qOUO9jr% zU|jsJYXx&0==)gWpC1HK-uP$VofPx&@#TQ>Ko&{SRU)&&?H5WlJz*?w6E26GFCM7p z_kU*}n2mR5SE@%slu5!f{()Y{RyOxx~AD@2aG z*1a#rGq19a51Mfinb0Aq8umPIZZLgiC8iYNY}y7o2b~;w6kEbOjQ&4Wkuq^bj-n`Q zt|H_-_8IT89LCL;B7?%gRt6JC)ba+MTZwifbVyj?qUU0Wf%LOqZt(Yrfe<*}+8w2D zMvNkf)BMI=8aF{PE#x+_hOgSg?Q!JGhf0w$XC&QD+CzbB2}A?&f%Ie%rDv zQAZc^=dFF9$K(J*eFzzvOQ{lQl16Zu9!uLx;n8GunOKRlq|p&Dh1rXMW2GEYp?b(m z>MceEK}6I(!|TB7aCl}g+2aySi}yx@Zk$xhBL+?p{~Rjk{#bz_jcyGU*6sjYOzyE< zy6@%3G6w7)P12HKFNNFdmc9AOtU;RnSV;P{y1^2EKsG~?5`q1yw4}5tbufYigA7X& z^((G8%NNiv!91{?{`B?d5r>7T!6P^9gA?0v4sJ1haY2?{odV{>Jk*oQ0g9+oAMe6S zqm(be@sS{mepv_>(LVZKp$Gzh?{4ziBu&j}foqmXP9@DMn)*`(ddPR z!_WvA?Drb>O+n-UG`9#76e?mZ?{U0yf<2BSp-=fjHzb%=sXFg9w5TCJyH^aIoxdhL zKdH7<|KM%_RPYcXA&RmGGt#0qRj8r+X+XmPJ02QW*G7`7(6H zKaT1|Q=AYyAnd{dLGa*C6KNzuBoi?SaMDANh9Oii0_3GvlDJ58o*%yM&B!w)OEKXC z1C2QRJ=IP=Vrvu%M-X!mbKnoTOJnprn1VQRx?lz1PT~)bbU7FXKSsMv6b-b5sEQb< z3gOcPUUd3)J~9!b|JYyYG|XD?0j;wN5(}YemWAf*6k=mzQ};>9{QG~BlqTTeR`(=m z;j__IhNmmL8ezvm%BW88u9$8O`a0s7V)y*HQOU` z4rMt0!5#aIYnQYV65(kQo3$vD=iaOhc;E1H#qgCu2Pv=;=g|RQ*W3ff>R`}`mr|Dm zDJSVYg};p@r%5MTR*t}pe1o?<1)eWQ14-)G^l(L{W&CA(gU)!iPS_I%14)w-Q%Dm*KHjQX;VE?(XhpkrD!eAPthz-6-9i(jXuq zNGdJeARyA+(%tZ%xc5HiclPsqeZRfyi>{k>&l+RQ7;}v4;;Lr1ejSmXj!3bS59}VK zmnz%Z;$~!g2It$(zC@9_x+fqsDn345^6gv35{=JB7YFZ#y$bxas=|7Ek(TQHia7v) zBbn^T*qGd_R|vwwNP2EL$+-Fo0Ih=E*)rvY>?A7zwxuOt4l@PE+~JN;OmL#i!-jhd z?zOJ7Z)PNk0K7m%YJtd^ioBP zFCq8S6_?fYz%T56q?`f%6_g-U;;=twkw3nNhkZk<-Hkhu8DPH}$P&%csxqwrG;ZY11;38{`XMW`!ZJ-rNLNlE zLxq(e4f9~LOS=VdJ5v?0QNoec`?C_Bj<6<_-OK!jFThN|(Pz-+149=~zJvXv-dK>> z^TVc%ZGMvl@0JrXFUAuS)+K|s_X&^N(m*W2;{c!crsmf3L!7QRv<*CEz?YFS^h@}{ z=R*vx^cE$Ng4^{I@t~xFU8d$+Q-KdIYC6vR6+7gg91sn`Zs3s=Iy_kUWgq|fVxL{( zP76>nL_YA8Z+wjJVh7h6AWQj=<@`S~ixQ1qQGkX(DWDV(@aIpZrEdWBPKOcs`Vetj z3^_S~*FHl&xTVU-@B~5j6d6%fHo9@5H{$%4tEnKEY#cBztndLwjzooYzSzRd;*Cz+ z2L7#&Ixlsv;KV;bbJG-Pq`>_6aI_>LAoK(|e^{=wvAGo^8j80srt_zV4e*gC zmD_%>in4ycE>Bb;{9md^H>KU_> zP86h}jv2O^UHtBcGCtY6%B)f4E!q<`@aq$`1X#oif**5J9m|5g{r%DN%33NSzUQvK zGJs9|uW9{%EfF{aeMx9zibqFB+raFE3A{W>ZxCrZggYi-I@#!i4 zYi&oq^VuR0?z#-<*Caq{D5u#F0NlYn{;4)Nke(`~u8vwPFEkH~m2k@m15uU+w$@6{ zeppye&zn1q-ME0w#VUmR?FX$23MGLIm;?a&gAod)XJs{;Z*)v$GZcziv10<45D&yq z4Gn~lzmiO6M-9J?u!Ta2yw)C}R)6KH@|=@*s4`MW6G$vAWrFVql!SmR1a zqF(v*dFVQJ?4}(0M{l^CtX#2ao(w@WLM#U+M^99#wD z0g34;zj7s1oT-i5*`h&&(ev3k56EMuX*Ltdr|gDAnXZI9>RSTK@WWF4xb4vs2!~G? z@!MCUV9S-77cVkWJa7&;XYe)b!V zA0v>(dX=DOjI>0kloTgmpA86R0*llN0f&d{;Sb4%ZzrP?M(hvdk}nzF7b^>q09xe_ z4f{$J*4mFb&6)vp)3*xUYTjMbjp@XM^)(ev7bcL)-wGnVCjkd5IAZP; z7N5u!###BSQG0kUn|E!9$undatvr9ewY!(AEaOPYDq<%nI8mDIF-;VdpaHhuEa@3wce7P3=9>23Z)4P~Ysp=bEJln*)5ZU!4a69^y z^`kmJGd8{;HX~V?Dk+k$ZH5{VX0)k^YGsbgMr*j9`l-cab<}(&$xB#ynen*9R{*Rx z`1h8T*{fTE`k9#*3C7qLBB6UMF4mfZNo&t1t3GGOTXxxPQR6x;1>&l{ehr%Yw>TT; z7}kAK^&@|aB~Cz(k&9SXnR}K-%V+x{SWLuHmLYk)$N$XZjht@j2JJ!5jHjzDz!o~n z*2fW{ym3@cSCI(-iRZY!zlHk*b@p0pmr7Km&|x zysupzU@=H1obj!;D?4}GERZM~%fzr*v5eX_wGt8%z=^=0o144B=x+?((oZB|m6F(>9!8eF9lo3SyaiNaG zM#_$y+rK^oWE_758vo+aio#G_pL^{jVK>4uTneH{^5p{$=p%?2`h0q>IeN@*#h5+C znBEIG0+7FB52Y*rOet6%gdPv0hisntr&_4T3QI`J({s=@^J{ zW4!ED{s58>;T;@dfB%;FRtD~s!+@QROuz1<6{&c)UDgmaI;si{5GmVsXpF??3T8jo7yxn7)>t9f_WFmgr4*|IlLaKN{vkhPpqqKcL`aY&gW|p{{}4Mi^))C2 zf?7~a#9FqE5#HI;-=7CKXu-Q`fKuy$>hGwR*8Oz+)f6!JqL;|vR+B&B(L+6SSQG@2 z6cj}C-S!U~urwZPig*PIik*!*mVC-Kif9`bFO=x<_ zuNt)i5X0%|=(39~sm2NvR|YtTACeLNI&=Qd;R8!`97!2K1&GUL`IAm(V$Hza-F+a<6}7>2f4LQ5e=WIX zcJODBMVp~!Wk+ESdDii;XmmJBeaP}MU`54TTa8yPXlzUeI3pDx&FuUWX!i=0!h{2Du{-bHVts& z6)0uM(_t;2f`Grt3gZWum&?mbCY=j9G||RH5o+r8x|^dZ9hWJS2i%l;7Ft!1bPAmI zjo5(KiI0d$P%!QHZ%dSYO$t9ix4W}Z&jMiEfDqKw)dPzixIlvGk~zrc1p|_y#F@g! z(y7k9iiOf@4|j?WMJG$$DKKnjsLVB-=X7PMG8?AB2(@E^!U2%mznn?`m!RDQXqU6oC#WWhVzbA@jD!l#RGl%e8NV@e<^K8F|WQc z0SSCETm+rRLjN9JSCQxzv0{LB4bt)VX6ptr1abcmZ9#z}GCTe(4TU)z^bPkkM*SEML^WG`RI(qIHEeCgeaP6w&*kC_B8<(^IRN z+Gh*rQ_4iab{aTR?SSMcFPlgV96`sbone8G3rOS1lPCYRuy){#1Fl+7$j*FITC2`P zUbYpURJ**rBntuXQ3Bn{PPS&uq$O|w)&?fhzs}JAdlI-gJpgOUK5)@c#Aa7i#Dbha zct@e=#!C5{?>Tuj*v}GLJFq?K1t;q~_@7L%( zdX;m_>4OM`MAr9ni)qziW8V81bEc>2Yus&m5)4!QnZ(+7;0EfYx;n^Y{vXr+t3~bl zXlP(~_wtG#=v_hB7MNKP{T@LKTbw*OwKMetDBD`Gs1Xx(G$X>okU#@|MfGzkbg`2&lc@5Q5DS6Kd}I)|LXCdH{k!-0sNop z@D~47{-4?m-~IPW{{IF4J+eWM|9>_VB(^Fv@#L|xF}-hIsG|g7Dnspvf@P^Bzy$Ii zqMApJ{+bPvAV3!W50FN{M+68a|Ge@7Bpd+-!@j=0|EoRVTafXZeFSjd&wRBl90n70 z@8Dpk!NoD)0k5v9sc=6wb#QR#>Fr$_F81UWy*(4gYThScNfQo35Be&(k9v0ye!sky zTL++GxN`rGCV`^6{oC2Yx>o?27vMz(OU6UFVNePbVu1v(xPcAb;Nizqstjvw_f-T^ zvpp@x2TDE=kF?~sFC?oi!}{1W|AX6Eg;Zl>{JowKR^N-r6|_P(7Z zUS7SP^ziZx*XOk{OBY*nbwSsp4Oue#I!cWH()sqoWxEKlSb=43c2?&u?u4s!y%$%{ zP&qn!NmmB?>C>l&fm@;O>Cn0p6?re2s@JmTd-6s4f`52>#AQPb~ z2hPyyBhy^$m2U=n?&;o#y;%Pi9}z1_x2*c1cNQi?yutZ8w$Sm3Di61dI9Ie`UKhLL zFN+9kV#^KW7ki^(%1C~zb-N9hmZKgiem%O zggOJxbrD^e*3F+Z5evD`xPGfSZkoMGe2dTCvGPnx#;$eaF81laH7SZV+BM{5w-QO+ zDZt8e?qBOM)hXmG|rU_0LsOx&EDs=X2lYjQ=Q!rIn#rm>c!-&5%g z4$bXp+DkQ8>Q_Uh>t}*Ej@4XaNmSOfjR6D>w^Za2`|m+)R3miZRUhg@3(&ikqb+34)Kgxy_EIeCr~eneDHr z-OdW49nU;P+F_N2J5G~_jI4GOnJm|y@vSNggqL(V9ab#}$$cGu%QeR%;SSog*0COfv)_Kgy^G3IrSH5TJj`MJX`P zhAXD?p03--YWY5>U76zC34=@-V;Nm#GgYNfp4x{0v@F+koFl36M`!i(A%lw_vFSRg z(UNh6KIh!chvBdAxv!CN8hK%Qd6Go#V4EkiZ?0%xsXh9{P?V{W>3$@3= zbvIV+IC);kvD)u?+)?YU!^WVH74^B~VucZj#@7L(;@~yvBJE`5g_6j9jc+&qE(}hB z%;wn9@BMFQK{(C!W03<)qN$854Jm8GLVZ^TTxQByVx2uBFIOMgQ}>*FvFVjKqHMqT z&K>`WBn^!#=RF2opO5aT@{uy{4XNHxdfO}`ZVWDb3uot$^{LZ%#2T9D4TY%Jua(m0 zAN zT47((hU4Okp6hkBSlnfUCGz|Tk~R@Vl!9t2kN5hwR!C1RY1xYD(r#;6n^Ut`eOS5w z2%Zr47D50196_teo^baNC1*#1K{|6Va^Z@|eR!_l`su458wLBU1;d}*vlc(r%dNh7 zebLU^($7ljm3d`7NJR}*>BKYD^?_Zu5;cQ32iCsY-zZ#I==kaEYt%m$uGLiZ-tRXW zyXhWEm?Yjjy~oyUB9$zzYRaJf&w=7j=7{D+d6DD%XifoyQC;imj`&p%0FBnK|7Nnh zWw;)<)?s{Mu1Bl=Xwqy$qS>*VLt6KmyV?B|KVc(~e7tvgrll*{=YsBSL%FFb>NS<0 zE|oO%;&$?IAzL4WUf|Nm6xBP1*5@9>(I!Pa6W&U@E<|xjsH}`2EKR51poy8V?tVx+ zB;A8v|88_6m5)my>z8xhcU8ZuL)R0>i<=G0OWk3I=~1)eEmocW5OR`huE&R1 zHpS^^vlqPSLP7|J<$E-!)xq7qUYLxd$CtAcqTQGA-Eo*OErl7sxB_azBg@P39gYew zXIOob*q@;KD04P=2Bf3^dx14{OBah;c}QshT=7Gr`OiTZIDj?jiX`lAbUy}ef1={x z&x9Z7da|Sx$I$~;3~AABHkAbAS(mR}btIhiLYy{Ol2~3VQms1nCF-v%)HnCLCM6%X z``)l6UP2npQJ+W({hB?+Y(D?NGViCSwWwy7x;ByGOC%nQ{mKh_J_0p1) z&{c6`y8PC!K1_!q#|pWUy)hoz!}xT?&2w@LTMVp)>|w{{SzXEd7tfv!d4Je$KE`Ij zdnNBP@`3_2c)a$G|M%B@!*gc+rK5X?sp0+Ik2Qjw1-fjGOxI9jyRL{T`Ur>BHtclw zA-S6s(tShg_Ll`XyGzl&ZtsyCTa>KjLRL}XRq9{6R{D6uGHUNHUEz{ST0emHvD~GQMCJw z46hSEu*7`z>F;#D#KK;M*#FzOP}^6S(IHV9z&HO&-koR>ng06-|^}F4b;|^LCK`bo9FJ?IA%~I$NMfMS8E-dWrt>fDuY}a)_N*2OhT# z-11x}=J1}u*^|U?L`^UIy3t_18)iAh1;C>mxDA{6^bEh;aUk%;Y>yH+Edbyh1 z((PAe8S36xHk8Ed7K^7|6OSj-0epXkZan2k?(ukfzwMofiZ^s6cYVDRX*0D4Lt=*y zmTL{YoTlN7P^tJ|FwD!na>S98nNZbchYM~+E+W(o=uSfn?k%RH)#fXm2fms2!yQK$ z5^dxik@o8~3(HA4=MFRN&)xQ9NY@t5uc)mKPuisuPp&aJI=I1*5l@H>Z~v}EuXDQSyyI=f-EtE=hjr3V#myc0KCgh?*o&N< zMkM2FP}nF7>NU<$G$GEB{jGNT#YGH$6@FyD_R*pRmR3^b`fhTHj++8$PV>@C)5m;Y5TUqjS zYKnA!uGCn!I|K27Q zu32pk3CCeJ0*NUPOO(tWQB-6V^vAaoXuxxt+86}V=G2hE@+0MiTO?MiA}J(N{=3%r zn;p?#OMoiW^r@WpQ@dTQuF(t`6&R<4!v@jD?>T2SL&avjqGcZgIdiLmILXSM#ABee z@u#QT1zyBhFPcjWY`d>*(Q0Z>`z=TO(j7ir!Tn{&o z|MYc%`mEjS^Od9OinG9pPiGFV6|yAY&v-_#aDcP$Umuo%o(*suxhPej!L+xx*JQ%E zi%XFEYM+sxIdOMEsIkK*9j$NwA>h#UtD{@HU}KFXChd(^#QUBOtT@WFqB3jBPmi#3 zOFqBP8+L|6hJr%Lo7-jv3FHWAO8Bps$-EshrTi!) zwgpW^VRmn5{DZI_pd$}sD6Y<*Q_^t2U~aHw+i&5cz_Sn&-WK=?_T*j5MR32VJWtRC;4p+JXcuO>2|p%RC9C$ zr`mjjO-+&+-Uz|64|1lgj0V)s2B=e0-4|2ov8z#;{kE@?F{_gg&|2St;BFF`>D*;iS!LI%-}$tmR`jhT zXD>Y6SP5M-$#O}-b`50?;bJ&TB-q%Fe5+SQ69w69P!H9Giql#Q-t;-#VUucTxsN|> zrh=)H21-S$e=0{oKrioHYo)JNPr2{N;Li#vDrr2VOHcFSdF8RiU^gQl2;L)DJT?O$ z%2ZCxo-_xv7Gk!zSMwgxK`yMMeGSVjHUz0%77*`L_w8pLsGEVQYA942%B$A^r*8B5 zjwM}0c1tvUs!ICB`tMx*av`GhxS_e|TgH)c>)gnACYLLHXIfqqrBxKpEy|69@8{U= zGM0$Y0@ZZkzVv`iH{w8B(iY87 z#PBtZG+eG(STbAZLj6@ADmO^MjQnNYaYk{0OjI!FSQCRTlONYphec!P%jEU(N^+** zWHGf>iJgPI!`=&1eeML!;N`9-(dZ-R!UX)i1d23P3Xem zRbS1{XOZ``Z*pu$;L4h}aBNb!gHALFGm|)ygl<8cV$Eswg(8;TIeN?8;oKV;clwzl z)bM%Df$7hl7Iu@Di%Y=Aci58P@H z{Fje!NzK~{Bh4z)NQDf6#1?gJZNlzoQnQ1_Rxd-Uh=uK(hHgo+kKfJPOdZj&*S=fw zILP;Gw{1a;jEyT&pQTeQJrdTnsrMpRNJ7lb&0~KeV0UfZK@Bx#7wg#vm(2+JWC0kk zNr(t4BP!68$n+dUw|aYLcSIyj;*kz%DI*@gPw35=HXe;4euiuxn4y!Dp@U#Bo z)3w$b@*fZXJm}Jr{!h*M^JIwQ7(aE;6rd+)fnaC?0(}+$1o0me3V7h9VJeabE<79@ zIQ$XNq$OAS+y8X*{eP*k%ghA&f7j@LzO68E0sBf|URMM{H3i6A*31RVvw;`L{9 zFGFYmppPH7lv=?Aub5Wh=Z%DPx&P0@+@tH;Nh_Y_A-`5Kns}tVk)8>4J|bx-TiT$x z7mH5|Y1&}dM8^M=_W6mwIH*86FSfiM6gy(x@rEPkunydPLE);VBD%~lP~dxedcH5& z9cZNhfnRL%5up;HA`svPj%`t~u%P~_9Y7a{L`Tc4V1gKfFRwp?`ngy_!3fOk(ujzV zM6xOXU+k_6CMjuj7p;?v3kEiJ;FOj0iV(QN`#PiOlG-8F0bLz#<;ONRy5rGdC)yziAR$a&W&!A=_t z>e>KUI4bfGeEiTZvt^ftOQ8rasZd0}{}cvz(zVdnv&OKKS683z?RoAnOS~*(jd0Pu zKd^3Q`^wJZ@{_PrK)sUaf3Z}cN%+z%YOQA+ICnG3mw*)gAge!WX=6Xb$Qu;s4(oCuOk8L3PK1YPAr>?ud1~J{QsC zBb`1Yl*x3&eJ3g| z0Re7g+r+qBa7nnB?2n6*5aiLht&deR;TT*)IetzcFmUq(U3$fq2Ciw&$;D|AiGfQb zkmARfRPb2n*jC4u)6%n`%La_rg_X9s6>27(^|p&ymULnkq4#WA@>tCzyh?>n^<^&v zkPw=_WhkZViUd4GytES#v~#^id`e(O=ZmIfhM+6#lW;&E5VPQ4(k@p>8`34y_86^9 ziMAO%iy~rpw{09C!#!^k_n6J#17#@DX6%vp87?_cFOdn`(CW^UwRIzj+oENZk+ik( z!}&%(E&S8Asi{z2I4zPqEucY5!Iv z0W$R0$f<60i?Wl!joOVN3qR83mV+}GCKgviUtW;kB@q^_;x?DzbnS=e`;2^Xt6*$R z+}`37Xp|XkbIh{D%N`lCLPGr)hf?t|xPMpV^w2g}Po$^5+}H;{wj7qw94`^C?VVkg zLAlgp8Lsn}zLix~m)h<^B)|}5<>Z#a*DXZ5(pVd17A(H9nJ7T!1obw)@TRG>JA6rd zG>ilSoZ5?Bc}L`!yhlgnT)s_xxv*|bRihsY73c527>_13(9(qipjaQf@c1fisAoRx zMBq(j$@)Is*6INL8Luk=Ogysg?y)BB!6 z-!5C3dw34`q%3T03zf4(bQA!RDVS_Jjh3EbS{im5x0o&=tU(LWs3_TCVD%qxPQmF0 z1s)bxZQ@Xv>E@KO^b-f0xq+yW6%7k7Ni;%mduu0-`SdWSvhdJoP2_3>p>%FwzY-bx z_ct7d;P}`U*f4p0R&{e8ZaV^bHIRKm7vXgpf(RzRhPj~y3Zl*2vmPZEo7vW(CIo&} zJ^8I1g4c7>svL2lYy{uvZde!?e#dOO5t+jJh6dd3Xx<>&k|4)rCCelvvQk4@_s@;L z&eVP3`E`57PNLs{*;2#mj*pb`ES+|CZlIz3F0)&IQhw(L?E14< z20}-jg-$osN}0|95V}}GXHU0#U(#W8i$|2P5@VgeLkNfC6ea{r{w@s}!+mud*msw_ z;YD`zl@3>sx10_nYk3YU7?~F<619QHDI8>}<*?D3*&URq3e12V_ z$&{i%!$S2#rGIpT)I?p4BJb+F)r#vNHYRP3QO|T~S|coYW0?)Vb$j(ktg4HxUi;Z? zenN7Rs-Am7)R)AM0dN#d(#;AIIq35Ar5cP`!G%7wPqVIIg9jz?GV$BpVFnwo2eLFZ z)(kpgxR}y^jtxZVdQLTYV>X~AscO8ImvUw%QRAXNtVmIW|3}1TIF=@-E73E{NbD+_iqg&U)BUdw)696#Gt|XAyx_8=rYe0mrqJqp z@CDv!KK13DGuc1b%$ldFG+@|b*NIu!?>4G1;7GkF^?KHDITnAfs6EOD|8eqLO51xD z%aY%SQ$``xA1;Kzf(+K&s;PlMqJ=vS<CA{LAoq)Rw{WOfB)* zcXA)494!JpKb3s?Zv32L+H{fg?R+mQsb^HC9wlCk(TW66xY*d(heNN%`1p8A9ST1q zuif{w9#?pi#V?b1JY^xS6FVR4orRFRcf8CuofkGACAAbBb^bQl%H88 zjP(wmnq_~te?K!^XKHRcK+fBcB-GU+(&R^6GnNwHbI(MPF@-iixXycS)Vz^(*%{#k zcK0m4YWyZjf=DVpmTw91V=)1KbJzWVmUoJaBt`Xv+T0R3b5~fGocM=r%PlqmS(FL8 z%};5Kxp-W6maRiHeS|_(R8nJ_9&j5WFXNca)0gv{c2#*2ujxPV`xo(BOe%8DfeWo^Z(p|wYe=n(AZGb6%3vGe1T z%~y0X2Gy&pgF?a_4r;Jcp$nOmjoJdCm+h*2%GnvZf`nURg7A^bc9Wms(!2R}<|MLm zMN&|_sD6w~OKEeuI8upo4ME~FWm-1fqV^c_-}cDn@|QF;`}MIHcEl)HrLPv7c8=!% zluBUR=u7LB4{bxd5cpkWPoH3?RoO`uGsT;{>&ak|rY7-WY9JV9&&879!=@Tv6kZji zad+o+OZquMZVI~v3+Hhh8?2!(TI93w04#m2*dqKlv>z*-Cm=eQV0RLJtdZ$-8`TuE7eBr%L(0(2Ojsx!)oNjm1LIrX&p8Da zL|>J9H%h1&YGO*t`=}U4hRhNL-q3JfQDfMOuN-gNh~{)U)3r{PfF0HN%@CZRNJit%Wwko}{f z+IGH%oEl5kks@P!;c7IV_{Z_gNo|ICyyDXsJoY7(uboKI#vb($2b1)#3i1Pv zk^aP(fxb{6`qKCn&PcH>N7a@qCo1p3dP84a(qRTuyjhN8(HVX<6fB2_EWNow| zbl?a1=u1qHhXy}%Q4$@D2|0xhgUW~nG%)=ocP-{R5yi`$Ia$6s(Oz~OFgRc+=mQbW z0Dn*jBE_FUviNHxmjD#b!w4iGZv2UTO#lof6n==Llp%<1o#--m*>ILT#0440&KmoM zW->6Hb?jb#e+y#+0|FKMQ?fXp6uJs%6GTMr;a||0Hut|}fDTDOY`X+smQ*>Vw0k3J zRCM)Z$f(lu%}b_~9hI^0L*?);@gRS?1R0{12hq0=?SWgu0$rmI#8`@lQvvOjN1{lV zc?it~4FyAYU=K!wg;_=bv-{21*td)iE<;N>wT+o9&qbuXt>2}=?l*VO*DBbga3=ZC ztKjidQix>ul{RZC!i!M-={Gq=`tyh4KwT7G(7%WF@^78_X3oTZLi$pyc!@2T04AOK z+w|{GwxNawF32>-Azh=yrBh2LM<+!ItJH?;DDlj35$O_f6pCK5f4v@RNAgE9^noD< z3zbryS6$X8uP z%P4Q1kn5(VpsyqUDHzI^4B`?WGz92LNcO|Z{kL7*#^J_s;{DbkhEApuo?ex?dq$B|{LH8fD?ytr?*d>%t@uBN~83fti+EMtIDu*0E zJSA}QWN=~$-mXbL*a)?s9gFyb+CVQ}7`4Pk;KfN+r&d89boikN>JYlWwJPwy{?DkX z{!zyVt%pD_9R4;TxFzF1uK+>PNPx}%8a5P39!Z}j_aD6+4wL@9Nvtt7Kb^bqR~5j*0oM;gj`dxSA(lH zJ)JBV#%wj`jm7FCv}-Zoz3Bm8&|v^S9liHv_e$}lb3nxu1-|R)F4lQ?*T7o0e#LwB zK@FN&j?~#cS9g-&g5**(gz%`J2I8f_PX_2q>nN(*)YOfJGbxF1p5vy0xv{vQw3J!D>1F{ zE*uoB>!sSh!LKh)L_@>+EW{VS#lMERX7o(t|CeV9Z&qPfZk8{y=DwWNbrc>)S zq*fx^gs|o2l%cXaf?}t%U~oz*nLKW}cf#)UhX~O^Rr)K9-H&x8Y2(8;QMX#KAcO&a z3c^xrt=FCe`z#&f#F>5hDmXlCh7_~siiVxrHf9%2X|c@L|i3OQ}HQO?|WtHvy_-Ee>3C*^3H7! zZ~f5Qk!sBTvSqkSl~=?y1~4kv%|(n*LGj+R=?Z3_s*rFM){6aZDoHMV7x(cKS-8Y@?wxF=*vnaX6+gomr`&WHzHK&AiJZ}=cRXG#kfOx#>Qz3Ja)Uyd?_tFLaeZ)A@Q))k1A z^-P6mUc2qTimswILYMNMB`R^|aM(LfNw+TeY=ib3$F`F?ESpyL_m5B=$2D^dN(HzE zZTf|8eqYhP8p4hu7do7t{IybG|r8ASaVb@SX4mM|+^KY;fy~^(} zK5$mi@_6JqLIr;sy5R>M8TSf^B(=(~9ty;351|Z=V}HfrVn^UomF(ZIx3RV>UZ5Ca zdY&U)DKqfqZ1TOV4FB8t+jmFsyfBlPRo{&Etd8P|WWNLlHIaVtlEEiwL0>2o9W&LD z)5v`Ws}ILXFGc;P*)v44{vUQQ_nJEU99G z(?yI{osh@=V@}3S2UkRTP8*4(i;eMZlwvg;(d14=fQm>7vO#p*%`RFF&u{3pb>)ALH7oX9f;+<(0Z! zIuoG8rrg+|JmIW+yDp1COw-IqLvb9$$@mtQ@zs9rVi^RAS+F~4_^Z^*-%=l?;04<1 z;QC$(d_10ocwi%u=(kAMuY0*Q-S2uKG9w*Y3iy)$0}=q$S1{SRa8g3scWu+BtiqCpVk?*P_OlkjmH?i=if3k`qtUYjP2ep{tLRpL(kGCJ+nz##@{WhW*Z#5EQlPW9AI6DKmBHYlNF1{}VkLh_?U#?4X5Rj!#ITf< zhJewj1w1C2Ri00~>o?dEI6Y={FR&+33}+*QwQ8E<=ASrjVn3Tt$Cxd*els?-r6Zcl zniLc6FZfk;F17XNiMc}3quLOXvn0}XOaf@NTM?_ESZa5Fox;+iIoj7-NM`635|S&q zh$E`{;Z=-LThv-qQe82xz65&bD-IB)Aj97=DLi-NSHC0twl*^XQRKb;)!bIBz8@3E z6@6nq^l4IQD|_E=!>l!^bsyqH|B&0s$ZfyMw)21m6l6rVCJ-x}} z_F|Il(1fa5owx%(t39H0{-jos?EZG+Q;}^hAL9o^O4383DY^dahKpZ*X(2Cm8}B=g zne;0|-H!`mj25X_*JRFOZzBaY?G*L>h@Bhn&m0>Mm53J7d67JiS@e8eh@ZIL2H2X& zylYAA)973q8uRv>wj|9fAD|T}LzASTsw&M1QgVfi*qe%AA#zomo1?kTUr=%^GyU9u z-IjZ5`-OSkh2|`Jqk3h{ia)*I)#t<*14YD=_w)MNL=r}<9WAD+#ExEVjCD?ln_a72 z?z^)vd9QbgtVMe`eII5g5olHxB?2;Q<>$Q*b<_Fd-Oa_>A@52oiCLQ)(Ko8EQ}_#b z;gxGggA$7#Wg4cnKJh(6ZYlpZSk!8rlZ1W>^z-)^sTk1RAw|! z%OchiD#u%lr|P*+o;O78$cjjzaj$jbiQH>5ZQ_O=AH2?Vv!G8qMqp#)4a;0g9AC;6tQ9@pn1?1%L;G`18>>s>4w! znX9zSGSl-6e|{dXH*yT#LH_HElp0-ZT1(xdw0%6=y9rXavm#@Kx-}9UoeXKSOX<6l zWrklHICgW{a6*H@nuntfZ+e7CQnH>?*)4x((DS+0Y8=})m|nzE5*+!?-p;q@yNF{& zBdY!ur^>G1rFfQlwbL#x>w~0B=sxfJ^m!E>qNX0!us6H0l1?XCvbl&&l|J->1Zv;bXLx_GB1c)h9?;$|%YHQ~RtuNOXjwr7>0|6&#USq=%tg3B>WId(qR2Q$7{%?c6&kH6m9H_)!!EPss48OHS zsN1MzlwFR<{0*<#;z;W_mW${#F&woj1G}Z@&2%5&3 zAl%seCa6z!yS5&Pd{(ucVdj3D*iBFD@bsKjQ_M;k)>gPB1~)nXxMdGt^39Ll=J9Z? zk<6})^V^V8xTnoSV6FF{FNgCQ$P$L(EOb(_|K59xCM4@pjpW;JL%@JrW#-GMMH$Zy zJ4Y*Tk-%iS{?VAz!9vQElS{Ya-u%fBWQ$50oA~oHt_Dhc{%kaOtf3@n9by;ccqW|~ z1JUHV&(qFNlZFjg6u^DX+P5X6m5@>-a~zk#u`Fkb;id7Uc8G9Z%a}z6la*> z*)1q&NV%ay3>GY+drNC}p%9lla52XcHBm2%M;zWc@Z`x1gS+FQ>cA7nZ|8(k&@@;g z(Tm5drJ4~79-LLkd3>Ln(xtC&Z|uZ*z^ZGzH23w@NDN1ktuSANxxiH81-}OCf_z)! zq3oc`yry=0fcCyA)KsXsmZv_F1dB((OPpv6Wzhah{S7)T%?TwlO#PhV&#ZSjoSYR} zDBlE?e#{$@(T)rSA`KbOxbHUUzKN)d4GtS)FR@|ZEnlshSonUD;YqjGY0>Y%dX&W zg2)Wk_h}Wj4_{B~uq%)zDN5+zy2J81WZ&>{hJ)CB0>1njEAH^*Uz|6;_U$|kk!f0z zUCaa_b&hqkf*@Z0+(ml94(-)5g-2d5kHEyz#Bn4yTOIkEhj%Kp7wF_*r%(}oK(k(0=zJ0@}H z>}sd2hXHSk(6Hc{`W1(n_IDX!Z`e|`-CKTUi0GpB(=u^x>!p^v6>O679>cwxOCh`G zM3I6E%SDO%_GFZ)DpRrP>%~iU+i=Za4Ia~&(U%KfQuumViei@Elbo~20=iIyXrbl^ z?k0>n7c2M;<}} zI72J8hI7lj{5y2(XQZ>T^3{-6?LPA!XT*BRsb2pdZGYX?=JWKA!)>9sySuv;g1ftw zQrvi=W_M;^ZTF<$EXDN9sUk57$hsV|@DRoK$Ju1dGkEHk?b}~!H+r6bWJvVOjPCKfq z|5~qz?0V`qNBup7`&n%}HJ{xJKt=3Zv-LL%dJ8b?na5W9r&&X${%&`vc71pi3#?lB znt>h7kQ+RVgr)LTx64zsq2XhGWV;D@mh~q%*)Z{Ww{im;p=2Ve-{ozlOD$RN<&b00 zpe1wZ4M#WmhwUB`9=o51m^p<*jz<)qS_;O1jOT9Ovatd~myf@}YL7oy{rp6!GBLuc zzhdX>&G{Ip=%IG~1L>!+UOVU=i7-!LN<+bbI4<6iiHuTRgg*3xU0u+TB$+QJuI5PJ zZ^J1{Ua6sy>&=eKzw~4;EVM;Xlf3z!!GWWijniIhW+dG`35lRKcSS3L!^}m$)8&DS zBH`rM@_|T_R3=>zBNCqVxZ~yis+H7M!#bPMX#tF#C74K@9v^A|p#8{j#oq?~x5*BF z)d|Lthr-F6XFNOP8n5N-Glam)u*K=t$##v3ru`dUd_$72MNov@@vz76ub~@fpR==? zZ?;KYVU$YtHW!GCk(UfXVR+ejsqom!!s``jm7e+4og$jxoPX$!|L+wNbb@o|nX?^e za{K)^>2tyTaaZ#F(Y+Mn;zMO!-61?dlrct7wxM`dkbjRY`FWKV^n881c`xmkVjZ)sPj%D&hJ<^ zNht7ErRRsKfEf#B|0N8|23gy^Ly3FvB62Zk>!yi3Q|=h z)O73DCTctpX>|riOS0;ro^zv|xr6QD<`8Sm5#U|B^}<-2-u?zwdS(Fn9P$2QbeyDqS$JuJ%1m zV?dT)P7)geQXNYLnvhUzUJV&#SvrWTeRRg{XUE^d%IzKOmx5TwPFbGi5G)OU)tx~% zR`vt&mD1nnI<5P@a z-X_b?diNLyWn|K_S3+4wHVq<4sy6q8edJA#FhQ2tzSNHY>NmA&nm`8^|v)V{qRp|Oz{Tz7+V#@^z{ z1qX|b1xFA38a!%4{i>%O3FAWg`XIE*?3t4EX);qlC>thgl>3)2KYqC~^#j+4oULi6 zHyQZd9RYftaHC-^Oq-qZC+^gD%c9(v)hYIen*lpuGfuz~>nP3UaXKb)J1| z^#gDV@Z^^C$;W&N<1QgJMhYM-c$33_%N`or-imcMNa^_D`~~7<;%LiKnT;1GAs!3A2$2NFoZ)ViC%CL2PmHVcow_Qv+Zt!OFPDss}@{> z-k2^)7a#M7%_I6>kyi>Sh8gk2m4wM63f~t;CKJCfVN)w<=`mHlv%VJqUGN3ew6!$! z!J`QE0KZ5`PHQ8sMX#r1_ z=4-313ps01_GXcY`yopQr&7!lYU%O&z#29h@l3>jvPy+PAfHQ+F!lYF{_d5B5)l$C zfkAS5ko~K91lh{e*bm7-2Tb?(Hn&zK9+;ylack4@pH@>$#k77-FC8h$1v{zvZmLhR z@-pM-AV0j63sYfWOfXdetgVmKYpSU64~fP?fU!2;%lK_Olbb$D?d&p;i5trTEAKepu#dusqY0cYnURq zDgTn~Xf~y{2L!MPvt0aiv-I_)GMdVbTdgh;{kTQ4Blyl}__WUoB_L^rzr;r05P1y? zS>@zjwVuOlUsLG8JJy;-ZC4;lL|~1m1YPd;fx^r;RW0>) zurekunI>^n71iO|Qy%1}NH0ACrE1ZiV7r&==6NQlBQ+4sngwmM%eU{-;T_!q)Re44 zw*0kpSy`Qjt=OljeB-imsj`3HHdG?ue=zNV>&Qe#Vl#0cg=oZNhApKpe=Rni)mtk$ zQuw@X^jA^S=(9hz-gOpF>6l@MV}5#BGc)yb46)m|Zran~t#m+7&lTY&rd(5DF9ED5 zIvv@@oo)PxR5N}5QgrOa$AQv!(9vwTVapllMS!W;P<7LdwR(=QBogYU*KxTUw02eY zp=kE$K#^gb5&;;Pzo&S51;f68}~VEAd@fl_GngI+Z{0}ad36td?Zmd z$JU8a zB;R;pQFy(XCHzg~l81r@ee%-Va523)Xwt|tRy7YtEuK?VgELvWl~+p+IzFkIc74Lb zWt}Hv=Zv#yJjYv(=ygZh7ZytOzBD9a@yCnP^6=qT1ri44cj;0|d7QJP{`t|H@cnmC zOhvaa&KuwJ?m;G9{+4i|B5E((TyjSytT#U-EZ0;gOW0$}+Yc&}!HwMcInmMlzJoNTq@z`h z{TG`DuC(X%h1yzBq)0ivbs->2-oa;SB|pE=kK!WuahyOOT>`?TEwhhr{J$#s7>V#{ z@CIhNS@V4qBW6~edoi?AVf$BuAVEo(#$mUh^(Y_{=FaYEuhnmZ)Fl!YUc!_#OH<}) z83X?I3+o0W*Ca2N;e~x@E-`a^ff#8%jJWLiog^wat>Y2wK?RveH5*zM^t_e7G{38~ zTTh;{m(@af3^ZzLW7Zhka=u-W4vwAomm-omB|W=5eirX|zQrsIw2@|gY3;e!o-c-|}$eHs`x?rKX4d!W*Z zVTi+Z{L~VbN3B_AV2DR^Q0hF`Pl>iB@U{iyM%p3x3Jt~vpr(=Fp{$%c&=OkScxOUO z)J{f9NcVPEXlMsUO#}R7|H4KMM_*6LZxdI#ta?w0=AD78DTFq=-d(Y&U+8FFaR3xg zE+V?O|MwO0UdJi)>lbjuFfc7;(<&Uix+^KyKGn5{^@6HK`Na;6vrk6wARBSbPh=6BfNt#9^_2IHFs$YukfD+W$BwprWeQ%3d`h~e z$kqjrk{Sx1%7UvSDF(1DWKp>e4s*;SY;ffds7t5cB^tvZoa?@t4eI$w(YDeU^U(T2 zsuE@g-=?46`NVBa_cH37um8g9tIg}PSkN3mo|Tz1(fyuo9gWb7Iv*YeBMYl>ta_i* zNKbBqzM>@Xvm?6lY5Q78bjc5b5%KL~WC2J)gYY{bJ85xGMi+Tg2s>Jb3|rZ*)rw5}kcvOT8-(dP`O176@gN>lt^6d* zaFQZz!=&Wdwu6#VkCzK!W>hmA?ZqSUV^N!`Y)q?$6O-SfM=o#YJHDlk`+LTE)_Qq> zf2@qG{3fIdd?G#YQ?yF3L)??vE8`5;w_eghCKOr*mvFR8Xc@4ubz)O8Ch*@S{a}zB zSBn{-2TY`9C^dJ+_E|;f|9pq*!-6)U_1={oMA_!i*ivN6Kb8u^Z>~8zSs>=YCbrlU z!g#*mT$%eJ=j1~P-$%*0?X_KU;!aVrLl;CuJyNuUjY&}cZV~OP6=Q@R-wdh^)7+KqF#DEkNq+lU2;;vq0Vv^ngY9t3cHA5^xrW_--Q`t*7y zH5+J?P;nm<+s3DX1b>CR%RkrzrdgUCMYcB7n)z17;-3-?=J!hVz3(Lr1H^AVi!Bz=zEM3Or3Tq_b5+%2BwwB-zf zVR;kW$ZGcKM3wxoy|eYRTm`IA@us@qq&JbVoT{*WJ-`IMvQJGSA}1CW3tA; zc5tpMzI7Mu`B+DyiC_?)Lbff_dXHziML=btGmrM+Ty|&a&fe6Jtal$&f*~akv8;Kp z;uBd`ZI{FN=oQ5+ar$y;he!D6@RPfiK*=HAuVs9UUHc6^|5c$7j!e7TgT>C|`h9ix zHtJ3HkWk~9v>!@=j!yeqH|wPjTNXaE6gs=*F}GTPAOr%X>6enKfa@~_MQ~>3ulbr;IC%)Bi3rUMJj9ex{GEAO$6ca@+6Av z%~4SZFUOnNLrVTSh}pMvfEEo9gNW*tVdBu9B}8TzxC=Glj_7BnH#fWT^bz6NX^@sg zY7jwbGaD)r$?c7wI@`j1iefD4jEJ(!y|I?~6W-!QD;{a5PA}Rw=@=IuSw?2M?xPe+ z#xfWHfmAHItHhEEr^tW!NxNUNcZ2!hu*#Xx!c_c2SHg}0{6pO&UU6gTg;w-h#@r2& zfj2EJyG>-$5f93%+%kZH79j?xntaYgDV8j4yoBSfXk{?Ddv@qE?y8)k6M=aVQWPBG zG;riz!k(aSNNvcE9?e`S3T^l9WAB_uVwWdl`97~Q5$h@e51t77#bLwc$R}2!`-yGj zhwE3~kU}G2;ZH*Wc&YSYC9Dhg$m^jB8{Y+8az-6*Y(;GRq<5c&bxrdoDG`_BL#7xw z#(`g{XLCJ#X>g<@mMJaoHpFvt+2=2|c;{w+%Km~lI=n)|2GxBoC{@NeAH;P+m`7W` z9I_CSxFLY%Oib)$iGb`6Au1f!&(xBUa*SSIu2XcVdR_Gan(8c^zpQuH&{B(iC;~Jg z=`*p0qMFyYs#jv70%RKxUBd+jyt!uZ8%kGLJ& zloBUX7YZY^86i=MQQIf#w%)!kP_Gov%+g}E;!k&4fX%;k zyS~7L^$#x`DnHj8bh7!W#@M75jZ(bJ4zR8Vs)W3)-tNM7XF zts(-@{DqmJs5IsPA72!>f^Bg7t^4_Bdf|f=`NNs#u}MS!C68GoQ~k7r3in*{?cU!X z9DHaQm4#0#88uQ$0T0wJtG?m4{-;nvM`oUtCIyN5aZT1Cej{SMFRw1um6qL0s(o3|3y6 zifAigvVKHF$zTSFj8%W>whPIrBf`svD~s85z%>F+$&(+Qhevx7&`h{=u%fTN``|+G zW|A4XriMY2(9sy0iO;WNC{i;P$iOuT^lHAU-USfl5A?sD6=mM!2n-kKMMO_=L|rkT3b1qct2?ZLY;9pDn)`TDr^)eD^LZVa*zEa7(ka_Y}F- zd{ITsO}yN5caxN_-vsTonQjbwPo~?d$%2wvj1H{BdAp z?Ku=`*4`3*t?oe4nV~YE@Cw6CK(jw|xdIf75jHLtcmB1}^BA3l)Z5))u#u1PWum%@ zrp*EtFRwwv&gEulWZDitqyDxFMos;v`Lp)phHizzNznuy>ErC0Jvs(56gMlYJQlr{ zV$7E$yW3;=6XQ7)NmPz>UHdPhH(6Ww;PGqI?n4!+`Mh%hRZ-OetkE%>cTg`#IOwRI z>7iT8=jhlfpd|ksUrv68*=lC6`9)v8Mna5C)1r=OHr^{k&s6bqC7O1B5`M8=73dC& z)UVp6?sd8bsVm91L=&5sxWQnkA0=r*jFp}pOMcryIjE<9GRsQOtx!Y83Ig;@O*ydq zXd@^;a}H`Jnk+Bbf29uQXZm!;5{uu|>ngwdAT%Ls8bEXXaXt793gPSy&v(Oavbv^m zDta>VsUO??-9@|hS9R0@BRgRtSH-G2nm93f58#)vfQTex*{rkadC zUa!D^V&T$e7FL)EMIR2fQCr@sqXLF@e>g4~%kL)G5)ct%qfw&J#<-{GzLT*$#BfT& zPFbD~&rQ{=pM6;oUSH+Mx;of)R20Q_CJN?4>t`qF=^=GVb|Pg>7&v|$E!cQz`C z7Cnm{%C!{VNE~+H+|}@Kb+Bl*w&Vp}3&y4&r)HatF}+55TO%KKZ0r0Fe_R9X{2jXI zJk*>1eiwsD5*io98D{z!h4-PB5G#eo#v2v%*O~|<7?cV-5ID+F0WS~9P*;VmCl@uE zNcc#i7gPF{^ev0UkfcR>R2s5KcAis9DFq7H@fC-?0wQ$4AV2Aq3eO9(fjxA->-wIp ziZrs7`VYtOx_7HG8dp?O)TSz5KNtO)k{Er+Vrxx3S& zDGu#j;S+9JKYc4IKey*?4p1lYH8TrWM-ws}GHH%!SPdc8GFv~qOZMPlN2f%osVG5v zZK%(uNeJKI#$o`=oc^t<0b}>+{3pG%w3v*!L*Yqn#kOju?aA$8UTzlEZ7!fg3S(ZJ zBZbN+d4()le7xo+hJb{GhOaLI@xW}BA1i{bnu7gl`K<(or0Bo@m-qeWhIVEfS5KP0%wV11~oz8$U$5(v!wt3_HwZ8bul28yc{dBG!iY3&ty@vOCR-#uvUf_p+x@hlYcG1S3+5KL}7b0oG z@zTQ!Bq`WCtX7d9+lj2~zvKJa@JhNGirf{#D_-Egr_BGZKw->|1M@p?SYJ&!Ij9&t%hp7k*Q)u5^Ev$4%SaXibHAKm3`6S3A-$bdA8t^slCWNMe|y z+9)XpI7pK)I0Ih^n^~C}(MmGX_l~xA+fe;!K^xZ?p2&HDNpnX<%&lBye9ZxOGSXlD zmV->1&5j#&UYUL~lV~ujwF|CR{R2$|GD@)%4<`|NVMVK4{na2V?qR?O zYwOZ^Q!t|x=x|7!vEunz~K92q}t>;T7|=# zYuo{CauV6U6LY`e9Mw8?kDDkOsylmCxX_WtxE-S9%C-C5i?-U>RoDk&>0nF1lilCG zVng*edTE@<1?$wO#+8+yJCXbIAy%S7%n$ELV)@%cU;Vko{-ar=W*0b#(|b2o-W3=k zxX36eWV_bIbWwfZA@n|OQ&h1jkPTw0cv;(3?O46Fz#zTrkz$;p_57D*p|hWfb%9J| zTHPj)7hNf0QWPMXf zrt2B{#SP2a|BQX?0u;BSZncm>;NxaG6xLBrGz7w-H?`dwZmMm`?3RrnD}uF$3I7$7 zx1E^m)rxE*l%wb8OzxKaaDfx4;x#9l9lf#d%Iv#><@6Rh8TZ*_?$^)%7KB7y3Lk2Q ziW8@!ju3uTA1%r{|E!Rk0dt%-GkG~{j3IswWgKd|Dghc1dwnX`rHAW-*!c{W{DbO~>efULMD%n)$q2|Rq!C2DpbeH2$IjWgb~je;Oy)tR#ZgEAH7=>9js2nPZ^>thFN=s2lVrmtLw3Za(Z6ScfOh;UkPq`OsOZZ(hzTzGcHPBMLi9EoV?%riU*MC|;k`U9U ztmd7XLq!KPL7I?9366%5A$EIiPJPrNZ={mJC4o5-&#rOs{Q0l;n4Cqc@*kG5FbF+# zAg)WMVv}WD*7w4m7k;E2px7#Cb!Bdad#JI$>K~aHOggFfS06L3vV@h?t=-+frY3Dh zHmkK?uk4RrRgrxhxWKuxnG@yBUyiDAg1TKXjj5;I{L1=Lcb=T0*PQaM3l_C=Vkc;0 z&N9{A1YQWo0+w~CYHEZga(j`p>~;3vMq)>NEeuvUfgyMx+g+MSv_Q;15N+2a$3naWHL@64-BYt!!`zbSAgB z4by|k822RK58Nd#{-McA@S?Su>CF67Zlqgyu+uupEz#v7Ew9IbihFv7xfsiy56^&* z4X(Y+GNM&hqa0<{A65SzbGox}4cbm&8Q%BR;nZW-h?ggZum~l#_6t-=_V#T`zDq!# zf!eXV9ULpDsro*mWPK3r&DeKkflnZl7gqz{P&&OwS64L}AbER8X1f5XMmEJz&Uc#k zY)54`+cMN-o+ngyl9TA1v|~Z6JIg{>3v5Mc;7OAz8jrJ<6RcxPv*j0+7b5jGdz8v# zsRX6Iyu3W(8cIW0XKBl5FaUAWs>uN;7HhRptJd+z5Q>8!cOMQm4I&&=(d>Z|4 z7F@2NHhPM*Z>&2zaU10*>K-Z|E#71xGS6(sbo;Oc9Arc$H4IXH&0bnswyVj&I5{C| zfXlpXH%ReCqN^^6o;_7>+S8Ab_i%43qA680kY+o85?;8d)|R2d?znPA&=rkjpvev^ zKT9%rlf^#M;yC{WNqfb%d5MBFLXtnT9~rguBl>twesYB8Jk(f&H`xl_>HYJ|m6clT zf%DGx#Fq;k${@T>!PFY(F-MaxNs`K4eC4)VgHqTBrkO3uFhKGk8b^c%M zu?mg;Qjbwl>*f(H&+oKF@bPek$>Gd5ne%%XCg%Q0cRaBfY3nPc2k%;9C;4b&3IVy4 zKs%kT9e*EcY9%$*5M^pNC}dB;a^w=x1Wo+_^)h zr?xc$`psh|@B8#+e9F^PO3pV;-&8he3dOumD&b$P58}@wI~DXXUM=3f)CJ#4=RP_T zWt45U?51SafQtP$&e(X}n?pj780Tl$E?JnO} z$E+oLy|M2H?EMwhk*u1C{|sE!XE-g7EVZ3yp>KR;ouAlq!@VyH=({hAa;dPh!k1pm-Eevb9>7vpwQWqUkoWddmI0xtOrz|H zu8cNjyn{rOOx5}8i+0;E-bZUp>k)oO{?y4{XsXnUz=vo!uu)Mf%eWA6*@#*$Fg3_h z_{YPE4z6?2BYi__`|Wr^5yJpn73T~%MfcTSn$x1XSX-^k(f#4XNBG!r#4Ntq z`Ax0xFMhJrk#(LVA*vFaxq+YgJB&WA6#+(qrFsNMAM76QF6Y$hOSmj|O9{|vZQ%Q; z~q_Z)GhZ;-^pn&TnP`zNZgdmaIt!~SGm`zW~ zLA*UZF;HV?oO7rcZKWW8+POkduPZmvgc4!N$jL2Nlc*1z|GvzRSQ^cAi2mCAxfn@b z?T(_Yr8z)kz-qMIRuy4G*b(-ktUOJiJgz2^k29nv%;(<>HoM1gPFkqS(^aj*(ag6f zx6&zrl?i@k!fcz>)U&1%`_zSmY%5)GD4C!(I<%wCQBg8D)M>Q(sl%6yow>QTa84?kr8z z1%80Eq;QgLbU|muTZFZO8i_f2)-!x>j);(Vjq}w1LChoZOG!&tbZ=72e90i08M2j~ zkyb{#-ccS%_K%lR?q3ou-wN~@;Hw0Vp&?MlU=8Z|kJa;tC6 z7ZxHB<-6vuhK)q=Me_aA33Wlb0q5R+S8i4V74lfBs(7S7{sV%IrG1b^WaoY0D{x!`I zA2Z_zZ!VLrh$sB?cumGQwvzUsR%&O4+E1Z18(s!0F!Nr+c%+{>AFyyTUrVT%<5v~n-kx1 zc_sD2;yI%;(pCz!I5>Aw@Q*iLIKsw9HS*tTYfsS$SRz3UOnoJJd14N}S$F}kslFL6YN-Mw@vAlk|9vYlaJGvobFC+^J7p{+iA0&5 zJHkheM1r2aD=4bF?Gio*6#!~D6_;)!mm1^12FKgON{W9y*s9{9MO~OkO-W{|CdBf= z1~QyU@%6hBn4JS(;BL1)yqV1^FMbIcdj0eHDw)gi*h=GQDe&fD+|;067i z#4vJl(V<$S$U*^r#3I(&hdO80F`idA3bx=`PY&g^V!4*~K-$BR0h4D77IamDZ-4bj z$F*{e6i&4D2oV;^zn;#a>veH zyFbZGnKai|DEIxHCfxiVEcs_GLa2?TFLo~;|BwfFY+=(CnMj4;7H%t_hzj9hbVqVc zPXN4#iQrmt?{Qpv5ZiZIP)bUnBGV)~8{n*AyK8k~y}U?b*hZW@Tj*Ho$>lAT%>h5V zZFz+Hrlzof-ZGcj6$7utD)yjW@TGl6fD%bKrs{A}p{i1E#2B*Ck}b*VOrN_Ry;N5} z+0Z)k8^MBiT^MLo^Pab~&lWtqJpO4#M|P&;7vg?fc`t0d+z94#eU`)siHP0k61li> z%!m{84+bmkN`~jiOS{fH&X2nXu1IOOjnsb&!1jwqHGbm16s=#YGS@UIaM=ofV zG{&MWE>l~w&ly5AvRkoFO1M8Aqd>(}0o=S27oFH)1`-Wb4q4MO1>v&=8 ze9Tf#`+Zhv*8_Z{V`b*M&*Bwq0iY+Q6QRbE&!hDew9RZZv#A^Y5!%&Hvr~3IbzD(s z7Nqj?f;rZ|$m$%M#G%cGvU3^!6DeVa5m!|O&{?mSL(T{oBm)gR!AJ^$whQ&L+ z6ICW}#N`@%y=8}jID{uF{%FOKNc68xI2wvk`#?yuy0=|3K+LH;w84Q&Dz`fv|kDO1zQvCMGX@Zuu1Z-WyV3zW$O`2Zm;-B|T0>%n71yKpDZw2Z=}RXZeP9E;F#7gQcIyYO)F6s;WDygJJEcNr}rBg_Y6D@#e|1A0el%g#rcx%Pmjqu(rcE7IP< z+L4TH^G@f8WxV;13mekgx4pBTcZRW$k|McCq|We|RefMpKDgE7qJENM_H(L}6H4*k zpI)4%yY4uzj#v&&OP{1ChH`_OJ4BlJ;K{GbG>u2Tf!8u;Lxyp)WUZ};F5oIR<{ zDM<^guhS5h$Wn>58*N_`M#?$eepqT^E28Ya`p|6j5V^`TKT2lExnArR60g9*L}b@; zamxJgQb(x3x#!^nM;h7g6$0}AH%3;&BT}*(_hxs)vJ3-hvURe}q#17N*FD3mr(qi` z$^>}Nj(N9!W-J3)WfhMeBFD(}JidmG3yq$D2Pn{6W{6saMSLcT2QWdCuIAI(Tke@Q zR+cY4HLL&obm9EDFF-3!b2o2DB$dV)^;p8SD>x%Fb!z*qR?MS7U!6_K?cs9f4AJ7Fp+E-oqg$nZ+)SCE*#FTvrhPY zfzs_`!(e0J8=A(h&(03FzW0<~AF?`Ow)cFF@J*$Jx6dsZ2i?M}p%*{(vRo*wH6DIa z-sT6YeJAQ8b31d9dS14D%X^O`ZPu@J`rrtrT8#B| zHr3d;oV92p;dbk;bo?{`3@#K*FL8%u;=%?6Kdbmu?HWLYbD}IId`GBSiqMRX7GM zDF$4fUL+T>u0OeX$k#{s+T^hyxW9P}mYV0f%lmqMLF*%UnIxW*dZ>J|{txI94f%#? ze}8{q+oO12tjGY7B#{gpHoQq6hg;s#k}gBn%UbaNqk`~rLjx`X>u&NUALCqW-D(=+^rzDI*u84%tYQ!x9R0^ks#=uCh zWFTJk&{p{k-v8A>R5Yzu5^V-yBh8;zkGm5N5zB6>&73gIhCHp3#qw_QcHg!I`6#i} ztd-#lKR>wLfMzu%o(l3$PGloRn|xXK59bjFO&w(!WZ=GSId9n7`5gXqFCQS=%F5IK z6q6Ec<=)Z6h44u05JjmEWvpG?@gH(jSXsV@S&;=1al!lfuz1)BsaD9W zAsmQ0QWdT4eh=-36w^X{?H6SkITf*1WTrksBK%DxH9~w?lgyftM-0DN@v{CsPtrhm zE4i=+=;G9s5PLs@0lu3#4Br;JA!nC*H|FOB)b_7JFBK`s(tPcVNx)VoAmJcuRBh>w zI2|R?qHb><8Y@dODCdmOe}vwY{av}{nEG5ZKq3;6e_Y3;^o0*A0vG~O}t76shbg=hm&6yb)F!eeKq1;a97X#7+dsaCix59M#h^Sx+urLY@gur7ZH zlJzqnW+4hYbA_Wv$B4xP@fo>|bsj`L(dv#n1m@K%q{$U7Cw26A6s%dhRB+#QY$$Pw;B8 zU-tNY1d!{oVs*QdsL$1v&07tli%X`bN7gmfkwZfOALE6yt)wE}csTw&6OcoLq0z@| zQ2pO0G!jn%)Q653K7?{~k%-ZR7KSJ*C4?wT7b$}#OH}0WU&)(xLODJq#CO^_yjSpM zJUP4}Ww}i?CIbTaj=RZY?Zbm~_Ib52E+H#mbRz-nzUY26=rUS;?+-Z>*XyJ5hL+D) z@P|u_6}3)Gv3;X+M1B!g^Z8#e%Xikkf!_xh-{oqgAh`toExIY{d8g@YPiv*WX4;vy z0PER2e)4$v@=ZTw&mGS5{wTHAB@TWQ(NY6c_ECOyW53TiW+rxHQxQSg|E!LwAA)N? z<^RWeNz4WVz8u0PktQMdD#`iBqa30zY2-gP1u%@*th9H)fkzCFbLkH|m%IO5r|QM72-fZ|F?ZR)L!PzQf>eLEGmAosoVGchF6X@u7Tc0PI|onA z?qsi9C#afSwmi%Tb{U^cSTtGf2LHV}+?s!WBo}-8oc87Zt|{S>EAWhOru&Ms=K8?? zx$scpzSPabyBIf<4MC|H`u{ZZKlcnVTM24sCMEy1ga$+x_1?Y><-N%UP{aIelc( zc%jZ}5-4Q6!r_><3Rqv))FqxoO3c9Hq!)9 z0Pk9|aok>dVr+EGMdBqOX0JIGbuGfRDy*f-UZ8GV6Q7*%LwPyr{T(5zAmie2IRTA} z{c}AEG^%`yDF6>rg!k$nK|C3S;14e1KL^Nh^gLK~Ge~82r=32VsdE}Dku!P8#0w^z)M{=n$PZ<+Y`)b}`qli74WV56@D6QK93I)aXP^<$rSt7mkJP z;Wx#S7XZMej3pn^;Lz%aMUj#_SUGD_I;M|}#CalO4-h+n$hF7DaXfDCMZkBladN4% z%B-}4HQ5so-p0+!H!7M$8znno;T36mWLl;6kJ2WTTxyk>c(vzU;=R#W-7kEqf5c07T*;mIk_nVpp7lWi4mg-F3P*ZUl-QFNJuZ>-7T%RkU;s zt&v1UiMVPAVi8g}p1`}@Z<`S!*dUtNW9+F*OL&)YR;)iod4Ks#q?N0hmXFJ||FFK(Qww|VE5JoWU8ZeIM&<8BZ)CqlZm3~&$sJt1Y-K(x9hv2;GEa-w&_H# z~4Er)MhZlF-}ig?12iUy5k95^jisfA?p&~wm?&om?H_- zCt2>Hg*&wuX~WZFavP>gxobibA`OCT?CVc|C2Suwa>;2U^OFVCZBJV)ik)VQ8t+dA zxZ=;U)((&r5b+or+^%iOgG9J9E$NYRFDo8TiR{Yphs9&pG@{Vo6zzyV-wH`H*htmN z(U)WmGz8si$E*e>!0o?!y3eSme2KKUJ##zf#5E*vdm!j}H;$zW&Hpqr+MRtZpl_;V z2ogSWPgyS{d7rjvtOEhxfPfC7pcp}85SK+-a&{N9+Un&16}g!TX#17kM|@buAnpmidZ?hW_JFvz%x4!ciTv@UalGetpQR1Uq{?~%jR;@&aNiq#ST>3rM>M2FWx z%fsu8${jzBt+u~qGPy6F{QB^7p`o$l`|y;8dj_ZS^-Kpe<_)pq!#gW!Zc^>l@)<8hpF`6R#P{wxo+D3t5sjf8>s4YRLaP1HC=}7107PC?=h0CtjgpyLB ze^PXfCy1OjZ&JNWl-M7Vc1z6Uf75}Vs1%zzxGeBHD63RcF@p<&I18Z_82y42ofF&~ zZl^f9WELFYG}YWt8-zPT%4j^b8I+*RoBit+~c1mFAGA-Ka)B2EXT_UR&~d zlU#2ur)L_|2i`vj{t~7f{OF%$8*uyQeCFex(7p0blePD2sbG(^8@h*YUk;=R=tJfS z1fUNoj_;GGH>S@Rj(D}Vk1+UudPO}}esbBC(A!)J!UKYtjv^(@rv=VqS!CRS8JMV7 z_@4uE3^C77*66wCqnQSi2|wL$;3rLGoDQMxo(7L&{cL&G5n1Wyz?cbIVi#moA~>F6 zn#H9mYr2u5>1l`rST?Q$cwH$sZ!2eYbE&74%>Y~9gHB(}rqkG)`}3dDtca*WL+I}s z3M}uNaMa#Tq!iqvpAq^m`f=Xl6w+6py*x|runBvjM0g1l^?q1m=xyFO#{W+kRcO(b z9RZDC#(`0+u5Q@w?As0eHpPA{;et&EH+(x{L#DtEL0Rd;SuGDsC;s?aHI*2`oE| zKeQsm-|h@V0f78?Kijze_E-1c>3$^vWmj-J|k5yY~KIrHJ&r zC=w~m*g^+)iREH!r)&wlYMIfveN=!hZP1s}rQRs_3_6G07{<+PfJS2?>v~S^=RgXqxvHrD29ad)3h+ zr|oRo>J1{#~vOO6JFoztrY|ZufQ=~en{qzdj1>2M&?vsIYa8Z zw8n;|lF5C?$7A@h1Xt4kH7YnT&X>kkZ$5t3Zx91EPN5#Q2)d@*r6fJK^(K(U2BMIG zt$?KWDN;RV99^AVlokRP)Mh>ypbnE*(uc4no|`7s1G_+2r1su=!WSpgr$NBFQ1bliaC zWTmlpuV;)esBkTFK&V)PHc8|_@ag1u0s;%km0u7#N^Zbys%0mp(4Wa9o1L_Pa&DQSJbhXT5i$n94s5&LKz0 zJ_oT{Au2(o!MX7n5Vb5W3k3;0HZ2?LO{K}7M!FEgM5CaxTyo%&^$0?zB7&Qbf>{j_^DkGIEd&L(Phj3$JkrIMe(%(qasL4cgND*UDBPRNJ%3|w{&;slF}du zN`rKFOG_;wA+Ri+clf{ke((M6J-;9BIx{Y zuxurSyzIBz>DD(;aRLo?F5o2~$bc!1fFI{Q!`dAwa@M39vX2O3ux^=-Yotmaw3Y}? zu9XoR!b?w2PT%SAoIz(J+$Tbn55S9Jksjrur56;FaT!2=fp1|>7twVgj993*)fYQ6 z%_9XuEx%6HA+X#ea6q1RyGIq9!8T2TDMwQN2|jd=f@wQKrHCcE@#WQYQMFa2M-i#^ zF(zkRV;76KmC(3Ph_G_=E1FZWjj2bJrjb=IwAZFpKRNMrwF@JbbZ$enTHXukK}W&F zd!N4xC@9*C+L4PPNMkFEn<-jgqX5Nx^8&|p`*EW5-eoIU*TmnXQA|hgVE^vuTE}aG z_@s;)%jt?bpr~U`ElhXyenb{k+_JbLqNHRfwV`W9X=UJnw8gj?Zk@$4V)}pJyC#Mb zIRNFwy!2BRq2llC-MPou>?wtcf2LU6M4*htO-)V5iB#jA^G)%P!)cYjGm_GBjF~IZ z?T24iW-a*S1Sd{Kf-<$NGvfM+X|YO*f*&eA+ne0mDG-nAvwpks0slBa(-M8h&8wy3Nc-cU_k$NM9& zPvqLk8hH4Tv-26G4NNS7-%r-;Q)EG(PfDvIR|t|N`=qbGSNl;{*Bn~1F*kB1?_AC} zF+iGAw5LS68e~9Ic#p0&j&lyuHZW2%%EW}fGyBC`ADf_O)a$BG+vArY0{&uwrs%?B z)moa^WlgcYecc<%VEyF}RXb<+qynXMGKQu)Hy!@=R>=+LDz7x`S~!dr+q6bW8O03~ z$M~I;rseZ~Br8iF&lvwfiA8l-tb55bhfl=OyW8tK4d6r!lB}B(cnSyM)ac#B&7GGf z{ocaX2PD1mS+kYALA^wjD1f|zP)2j|El6n}JcH|e64u+#f&E@G-8`-a9j}zvi1HUP z#`Av2$Qcn^6caNF%E5{e`6VF_~uqK*AE?`ucsp#QPP?Rwgo{;yq_tNC4edt*Igxg*}?u zr4!^662|*9$)8blMc$s$DSL_b@3KB+bSWr|)AZU|_9Iz5iCLmYN#4=7#-v|!dJy0| zd{F7|VS`Gp*r)L`eqYlr@)e)w5{&1bCmHIdS!H*j{bIVbAYMC@mKn#TM9 zwCxa)1Gjy2v^otN@u&XM2$^SGRxQ(-W^SVq&HI*_`HpU1|G4_zpTOZ}o(epJX2=#~42>HSt%tzldsC_SL z33wrda>og=!+yleu~R*ZM|ws$q!*f!zP*9#kae`{V~CV!ji-fHhAev({yh1#JA;sB*2I6<)^IUk50UXhxXiAE4E>O)UEj?8E_K{J07vuY_hM2LJGZ&P7)$)D6 z{#_57!3T=X2G8Mib{dh@KK*VCF>*H7mui6`ag{n8$L3AvG-JWkz*}heEBc9jSwZ8R zWA$Sy&#H=O(%6_gd%Ssgj+o)mPmjDar{hTC@|xwYc`6j+aJl*=@GLCr(&zfwOPc#| z)s6Nq&Sr9-kJL5CW``7!a&Hz^Pb_sbidG;&KHVQOBIv+l*0$083K^m_HFY8I2HER<9>E%BZ8)F$-DCY?uTM zD1VZX@bc%kYXib0Wb|3hTBxkiua+!QF6xMVG9k~*d_C&msSB5nSF32PHGg=0 zac2b0#Igmy#lBJJqeTgoYx?HFqR)Ot6)E}XX0&!fQ*F1b>^p%F2bpxdL;5J;nr0%e zK*P&7xY))~`>2IBQ;VSuZ%4M=7WlNnbJ~Bv7DW;85&i~dU?l~|{d6?feR#X2khH%* z4;-VJ^^a8*xp1nr!WnQP`qR<1n(pCL@0$`K^S&~=aJhX1jpO+92XjQLFL`LUjj-T1 zew__%;d^9Gj@d>S9Zg^}!XGEfGSV3Jw@;V0_r}%6j*bhfT=3QS56EH6W7494owBG= zm?R-n9D7p273^8GBG9@ zD2j|E=02u4>YwN%*$Gi&UH<%5^K)z^e!7BY#` z4=%5Im(0Zy>L-S^;1OH}3GO`_g!IICKY_ z=KpQ-oIIxgVwrzRgTG@A@Saru;vsYh0S~C_Bg1XOpMok|fMqg>?2GFq0zd`Q0t;_z zqYn(9=&lET?A^A8FE{L3fy z{2Mr*81}!uT4n^A{56qY&lwb(3$g;=g)S#D#CH?Wo3vow3w1cnSAG)tjtDf=WCQA) zcwM|hvG?;tu4H5bU3<;lEY-&aPK*&Cb3~$llX(Al_TQ2DNYr3 zY`i)U1Vn-F2P8gk4o)(hq?7m+^V|L$1M5Sxey~N;kI0HQncz);F8&)?0I2~*!v4q6 zzYxmnayc?vEG^#`{t@s2DEx$NXyiaR9`}WsuIvwH01z|2kZ54#kZv1PJ`` z9DeptuwPPA!Swe&fBPj+oBr|arCbR){eZV%ix=812_mvGATyVSAHG&QHq1z&^0>m# z`AFj?4+I61#j;mxa}4Y60oY5sc`wwMVX{eM03g=(j~cPl4gl!vKVSxg2>+8F%`@QG zDT_yRog6lik<@wD(-T6_IXQ8YBEsmKzSV{$r!RvY$pA~a*>%gUA@4&cN+2kRcIjLkmU z5LkW6XgCrtx!3%!Y4wc$7w7!Ptp7L7{qv{^Vn3VHVx`}y6nECRW2=3_w-9rp_lTCS zf`))Hoerx5V_jSTB^KNwdNrEF|5Ld97nuM30@HtE zqMjHb0^`x1`0CDP$EXTJTp2nL;8Ott(j?42*W|X}>wAyU^uWI6sp>3^VT^P?mY0BU1~3Ba?LMkcWFd)_lOfTC%| z<`?fz0ksi_otp6cY&O?)1Py5~7vMfcVUE0=caN-=?*V(`gXqvs+rS=Crm3ZK2-+>b zE{LGY+4;=8gu>hZ(u4mJ+!23STmQuRe^3Zd&okIIY@LCk z)9qT2FjGEKVGtet3C`GZ`W&^kJ=DVVrA}nw&I{^X|w8w8w>@~Xb+^+G5!I2HJ)nZH)L|5l4wp7;tj}ln4Y1vCR;PfGY#wx)g!!g^<86<||Wr zsBP7?Zu6Q^XtjC??=!kGM1uP+MP>BDvJzRZ4hTIa6Hdt1u10&dJPUQaZ>}rMO7-f@ z;fSw|MmwMJaP*BFTV1~uAsi50x$O*2vFgW77FzDzxS=MfG@4j?Wgj{1eU2x;-y?c; zh+E03NG-b9b(Ah}y5BDRaJb0Ko-5iJdVYd|DC@>3ItZ+fc#6;1^# zasoP>I4q2ClgiLozJd2c+M+s%2iYgtCa|SbF(*WHL-tX@ix$}`;O8m%1uf|3p1;9<;gU;F0G&agr)cc) z%!w8^WyRB|_A~1C+H+A=WrAewG==%pw!>qk&BY17ApAZi8R=SHii6VjdQ6_47!QlT z!qOsH1wY)fiz*G0aDkAwrfW&+S5kK_gLCveZ&R*FCz?ZaXJ;GQ>SGc4sdS0~E5s`l z5dODMe}uJGsFe(Y@BUmU%j90!iMRQ2iF*WGRrybJMH9Yl{g&d>2uB@83uS!^<*Z=g zjzF|z6u&d%KU|S|z3re$2M<5W8C+hXeJJ>AY`_oF_|`BfL_{ZM?RhLV?c?uH z!KHBm>VQFuL0)|tiV??;_i*gnyQdoUS;_hroLRRB6FxWMHEry7g={eb^I8%XPf08@ z0LLE5V6K7AQBM1441AA4pBDq)fMi^tQoW>1(Ov-1e2SsY*wt-MSEB(fh4kO-Nw4p9K9 z2p&$lUx2I}K+pxsv<*I567tQhonPn#;zN%Yvsy{5rr!j6(H;+;rNdxvv*t&t%)X}h z-&BaUw>%5&xgtl3@B00-jm#c*!soh{bv%}VS66Wz=`s4=%4Nvcl+F@Sn;rGbo0CD` z?Iv2#KoM8)6&5H3z&k_ODF3YqyAxuE(eL`j3U9Qbb%3N4$3Vh zDRx_>o@~$iXZf29Ui;dYNyH3yw5sR=2KjW)0TM~(;Som$Oju&eJFK3jJ|I>0`3Q@F z9TPTs>>ugPZHSd?(tX@Vw7YraYY@%4nU&|AFzZS9;H`B{>EE7=+kEYF+V0A3zb5Cg zMUnfsA{>RYiL8F6?A@MJ~i!ghS*0%Hb619d*)D za1|c4ShL|~B#}6DEM6PbS8-f;->BA{;AfA<>F|dc$65OXtB-i7^xi?hx!w(* zPA@crYv6D9h1_$E7RVH0KlM}j3_atodI9rvH{vo%$0pm`TFBRyN%zmTn&SYmMEo+`Av?6Lhe2+)mFV!mWb0ypm21)MWTv9+amP7u1G|1>>=hM*K-l%Ox%JB5 z!Q9yXg|UHgAhC?7+(W^wUL%OqG8}o1q9DI87`#naFJhgsXpBJ zX~<{5cq5ESC|54#lyNUze0Q+slay_aiBQ_iYc$d))dL#+e13*ebp!LdSd7Fd7;dj97Hw285OmJyY~?tayeaPe(LE|iy}4-sWLd+XvT#6R4*`+Lvo;+D4Yyr% zydWB3mXQCDWMUa$Vu?^1E>!P=`-?apx_E9oc-kCsRoc{W>EP97iP>lTC|X?i{l^Yx zGR!~JAG&3lv>C#)&^tM%9g}H7{|fReThxc_v3tPN{T^o%?GNXd6<>_jySIf22&(AD{!&ddVgd_4 zV9$);}VSiNyR*?tU%N17hGzZ2#zIasKYjV)O{9Oqn&$injsle=Cr+1Ln3Km}`Q%^kfp{Ekxg&mjW##rl zUo2Ub9axo71>{N94@7M8pyp#D{egn41=n`VFVY9?~D< zYMty&nJMo?{cBpt8sFdDe28;JYI-yi|Fo19;Y4r1DA>I|iI9&&MV*s${d0@ni<62@ zEGs1hGA)xK&9F6(O@+RTXLd{f)7@T|_6Q_z(W6t@xEJOCSZ0|?_==4rZ70pQ%Ts%^t7{${mZSNE%bHT3O6*l^_%WF8i9OFM}h^a74#_fD$;O=OMDIHtTu3WAUldE9r-LZrXPAWDd(1 zv|=L41YWr+?M^TVEC$0d?Ja^BR3=4!hckDg4?=RizbKwLMn@NIn&Mb35tC(w(NIc^ zHqlY%Oall8c||CyPS6Ya#jNH(!$WNLss&H&r&f-tz3TN;sAgrzQ@(5EvFBM0gc8UW zsEoN!IoBp1%6gfD=#qX1Mv__y&F+1(Q56;YquP7LVA%AE7X78#`o!g za4Qh7YoneM-6}5q8kw=EC~z9j$;&(R(T8+be$(@6&xRYa((B1{$~d98>)ZjYH6_Xj zgy!PCV3?KH_eGM8b{PF^ACzq&R-M3lKU!CtD$eJ`xheOfH*X}DtQSsV3MH%<9Zb5@LK zTZ{pZDbRd2Tvo)|?l-(;`B$$ETm*-33W?YfW%=wSTs=S)7w{Q*<-~;>jvZFjYvrmyI1$_I322s`y0c z4cw7>gkx=S==G(jl5SGCP_E3Eo!6VqE5q52XyKRj5dFrWbxG1#_|8#uxr~w{>bEJ< zyuy9&)k)ddyhba=si{}TB*??Co1^I(U7aIZOZ%c5jv zm~D!Qgx#nFEIMZ$G8e{W`n(E(ULI+WGK-px#G-#q`e01$lyEySHB+)C3p0<{LCb)o zBF`^v=lD*nni5g69sG~I02~lB%8?ya#25kNo&JbC^-2R3Q!YU*v6L@djV|JkUK2vskV{cZi>IZX`GuaNK`#2LFWWu+wMKhDJ)~u~t)i#uFA)7) z{>V|>oc&q3U*B!|ab&jA;)eAC4_!kKi;;?%`Aykil4Kxp2@Ib%C?Fz=TOCVBhoCIL zIv{HJIX*s^p{$HmgP@!dcB-NpqJppHyd?=1*yF|l}U04_@6Z{%o$Fovy2;+yaVrSCBmm{wwk^_P?Pf}jr78R!^aPl)T0Tb z=IMLB*upA>^_ewF#(66{>8sId! z5@ZpdzG)JavtkA6l`uex!fHUKG|JB{0;5jzicgnn>fA~_oo#|l^+<49OEwQd_05R; z&IPTkO58d0iAjzz8>PgbAP!}+sGwUj;gjx(Ib2zU{ z4zl3%9R8YBSLP2R(3I44s#40&C>7-Co9aPQ?0=qC)a*3jj#%R&s>dWPFGjo_^;T3_ zn1MNvrdrh^aQG+9B=5H*g%HrqAJN{!MIW7s=dhW3yS|mmWPua)0U+9+t z*f~PODy&L|^r6E^2#a9|QyBO|OE*3`omM2VLd` zh*L7(XM@Q2_KXsildRbPrn<1gmK31-cEYP@#m~`= z?_ochj&G5OTP@fkBk*9_A*mnKD1;=1x>#-+k4%fIBV~LUe(=}BitqDwAOjfRJ$jte zs@KLHxhG`myGIuxKnI;bmY_NO!y{sPT+9rMKmg;`Mh#$mws_bXPwC=9_GTv!$8?($ z<=8NIaU>*_$w<&Sz%=^}y8ZIdTL;?2JqB>>z+ka<8LMJ~Z47QRGV|$9c`G@9@ofhz ze5}+t3lY6#@01^pr}2UjA#1D)642_Xhk4Ne?lh2$XE1UEJ7 zgoTD44sEJ3j{0~39J51eRMhZtag8LMhPhPa-cSOSHGLLNGmRR3Iw-9Sn~|AYT~kj3 z=!L!fc=xz7RL-ya)j)A%U9O*=5?)po%)kuXVfcBffLsiSsP%&O?jl@wy?o%R*LZat zJ|6v4Swvn@zfdQG&NySe_j`{@Q2}e@H{St_{&%>%7X05VwyE2%$|bgb|9&@DWiZyz zW~-PMof3^}Ab!n?q-51hBi0_Q+9p<%q6j<*RR-q_qx${H4`awWm^jRoPR(F&=0|&$ zCxTL+9_qC+nZ&0Iv>{|=)5E3a}mgU?a$ z0#{=jC+~X+J+G#0fA1&1U?#8vf0nDHme=x<<(4%Q?FzvOqZ@uvK0=4`Jdtjyj~iE*bKmU8NB}00 zwh|r}iHsUP&;$gH`BM0zkDW+f>oK)E`k}0B-@8~Xr!*gSX?6l>fU^z7UK+fjlWibtZmZj_(Ole+6CvfZoo_g8%KN>hEGxE#b__D$%c5{~{x@N+w z%qDKQbSvLU%d-#vEU1y(w$Z@W)Kuhmt2KdO`9mjrTAS|LSg`}0{W2F-GF52QJ{w@q z{+tQ1mh`*9ktzazluMz5BDf9lMrH~M$bT19cDkUMpb+W>dxp>{RxP*X9OLn(VQaL2+IH-R_b{Z@f!@(P$Fn`e^DeEf;_6lo zVe?7Dyzfs~Yimu(IV!Z+378nT%?q%RU+94fMscz5ED~8Y6VsWfh6`T#q1y26*M87R zi-3aMz6+L@IIX%IP-wNj%OJ!8ml8!Lsg)>}>@S|LVooN41I`FilY0; zscwQul2JN1QQ~|)_&k+rY52l;X=G?(t*bGQXoK=2DLpfwWDKaOK0mKfkVybBUspmFdCYp;}3c#2QHi*(!?E3OemK^3Bd*Ej<;eS*Tt~ zXo(6czJ8k1^1uTMDW+(b;iYd@huDjVySNMEh*UIh+Kijw}kL(5IHu zl0<)V2A+Pn$!B=m{unf8ivhhCIocWL5F1N`S@@m%vRKFTegK?CZwGRQF7@))QW}7q zDUcDVcvA;04_1dwJCY(PwXJ+Bc#*2pkZVoT|K(R>O_Czr8nmQYJ{fR@Y63^3 zq7i{s1{-ck<{%N(=TTgIMj%vl>D=nb)^Qr|VD6^QL5#NId&4zO%lw>M{cram(gT$h zwamMWxP@PJi(xuCHszeReMl;2-IGBgy=)JEog-xQHS0j)#I`PuYy%0?C#TwfkJyM& z_Bvc&3zVcN{N~Tb+#lao0xxg@t5Sn2_e$bspp{w;kA8uUow=K_$dp2?IkPnL)KD6! zgNc`ISRU^{bTd)n?-!P+rkS!XF0Z4mi4~qloraaSI@L(-Fg=IfNL?Hmb)|P)`o@(W zlz*30vG^b}t}>>ji_=#y#J|GvZP5CSMz5A<#PsQx{YwNW>D^^>o+ zd4Ss%EmHs{Yc>U&epgXN@mxGaPlq3}&?@TwgdWGu!!q~X9I)~dN|CK4X>w@E9n9`? zv-H;VdqEGe!`v&DwCL&4MZ z6=;ssJ5N0j6?haV>|(-SS>P<|W|^r)qsC0kpgk``q^7|b1yIhgN~*HXi$X5$NIa;k zh_}Rvavgm^>k<}a^Pk4IPV2<&=VK11sSKE$;9=-SPB7?dQKyQ;BC?G2_q*o!m+Fsn zYyMj({mKDBfKd6&03M5WYpql&e_N0}niMIkSX5JF$)12&QO3%|DW)eaNvA!pA}Y|+ zf8b!@hE`2IbQFIlEWI4(xaNO{V=Q{Z$f1IC7u`)PoxwGo%SPr!@RxK+E%Vkro#G zZSX10B;!>S6lhh1rR;Fm@e5!@%MS53O!0~gfk(VFY*JxlVvdPg*fKfFg-&dTmZgvuXi2ej0!m^7p}t zrTH{I5E*o7sQI1rVnYkqOLOn`;l0Fm8BCLejP(Wh*9tEf>X=ATN5KGXfFmkE`3aG8(#QUJI&wnSF z|NX(L$xAg~8}kQpPj+>eLzcm8LZj^MOa8sKR8YSvUyF@m8_ZA<5HN@XIE1OPQBkbe znwr>f*qYc_*qQ{9fEzX_+GkE{*5g}*DKXZyh{`<2tU~6Q89a*nL<;{u#{R(Ev=bCB zUd$zEnygHN&FQ9mpgG}3+Mfy%O%$}+q919sdpG*myN^Boef2{$gzWg`q(oMA)Wfv# zlqE!S8u|S4<(#U542g42bMu6r59?0sZ)_QL#yfhGIdUyq=>LJM|L@Ny51Lo$xW;YP zLJihCQ(8V6IlMAT7bpHmV>c33uHMI$Tfmh}r&p!%8n0VPp{Ok!nrMr^nZsCG?6bRoxy7>94VuGPQ#?C`kbvE4t%N}5OP^?Bi4_0g zC(OrfiAP`iJ$1(&gFx2-DziT??|wt!w^;aebcU}Lhh0o(kh5CMvicvrrzley4`ro5QLAH-dZjFzW*iLXH zIG(Ij7>Tj}lU`$a?qW|otXn2D^Oap(?F8j^w+g9k+Rx;VAI+Mo!=)$4-ZwmFu2;&1 zj~9D|qQS-A+w$5aTf`@UhliSxTb|3fNb75tLFyNjn*m*?`|1aA|C6a-Mf40zDHZ<$ zoI(EomL&i4C=FFX0gfVHt;M`94ad=LevbtK6S$wNYOi_J1$eRYi;)94_nTFYr_+ zS0!Zp?_T7JyXSr>&|;ZzJ{hvIJ}^>#MILC5M+*iUDt2+KwTmA<|!(O~5a*Xc>H_9cYaRzn0H2mFU_-+(f* zv*{}%{|`DF#{YLv{p~yIbKrAIg<<|zUCvNxN59#?NWlA${&CGP(s9S%gG@?%8s)E1 z$$aiKNksUk)+ma-t`V1PXGGnuyt*@yRdT%ocZ~(5@VOMGg*YYCflg)6Csk}~QDQYb zn_|Nh|I+v4f)j2BY{D-OGsaxd!#(YPKr2jU~|KwiGnH6MBA?S%Ut zCVarnfDz2-d@}@@-3+*U1a&W5WD$Gb9jrW_XrHw@kVEwC*Y_kIF>X7E%_@b4w$pPh zgfUNb2PZ%dDp>hIi1MT0l`KV*gG74RQW-uw^lr!{1*M6kVA0)u^$z>$!cY8u4+wNj zj^L**&HyiUbnjPj`RxV&xEj5`y>`Cly=QZZ;$aYSuZksI^I&^fj_kEHmh=KULvYxm z{hBHraepvRs6SBq^~J-|nCpwXlz?$Jfp32n3`Nh%C%Bq@iHQNTQ&AwU#@<3?$~vg3 z3$FdXhY~vJVZ&-%o0#9lcev^lh}3cQBHK`%7HxP`_ja+_ETHOK;5&4ku<7!mn%l)r zfNRHdZ)j%L^zK*U8zOU}U*95aF&^kHSI#+qQLgk{jgzzbL#wYw(=!*R^9mgI^w^8P zuqJq&a(mfsF${`&;a_t>&?{Gm8*q)+dGgw$7y0ZovbTF{t%hnO{c?**t2e|bg@*_! z{1R?q99ns?n$4a`rN*tj{g?OLzclqYz0Q7!x_{3#KxJ%Kuyf_h(!ER^rEiArsQNr~ zRYMmYaYvRnIp35JbWZO7axKbQbQ;lpILlhw&Gok2vx`a26CLX<#+l+hraGMos)}7{ zIMzGzJd5@@KSgY*O_1=9!0obGt#x|=&X7#1N)&(Baf#<4u`*m!Mctz7Q1A#|ZiXFU zr17PK{q{iVE_@_Sul#Zc)w!of!+e)Lw{VR&vM>#L!ab_DO0on_uaS<|Yd7T58v9JD zRM-OYfByK86x3hgNJ66`Nay-!sa@Q1akLFfSP$g*Ny!{YgxP%HE4~|f0Mi$ zuW2V!-YTVgH~-z~%GL)8jc(nY{}h2()49TUd+~KUdhJ%YF@1t_i6q=6@*V5&;1}{b zy5$8w7~;cWVE96Ze^k{CQ7tY97oR5m^x#^K0_E|oyvWto{hHaM;qLE`YB*{+yMrzh zJT=Gn=)KPgR60&*{mu%CX0RD^ENa{$amGG`37`cDMg-(DH6;M6Yjmc z6-V4ZSnz=v`CV;OIu4M(?ib-nff0H_T?2`Z(WQ>n9yW4&R&EE=n_-(aOVcg};=UPZ z=lWNj{Za*RFz&P1%jDa;!j=0r^bzmy-^)!oI}<*sKQ63@Y#5#QdYvq5CVMCH-Wsxy z^4sr}tKTQw=21ZIZRHmlylF3eAQ(qc4*B30`1ELh3>rO%x3!V3rl7<)dk8x0PU16N z7i04Ys&(J|2Kd=mTsZu1<`o-W7QNMA$u6+?6;5os<)*Ma{I#<6c*f_l9QB996?F}_ z`hH9Lce8mWpwK|Y$~Jwbp2mjr_hhm8*rda^T}U3}n=F@pVGbz`lO*BYiww;g(aGky zcVE$*4G&xFN&U|sSJmvQBI)1?1y=iScH8?sIv!rD#k=CN?Ff0@3HIFDTgqw^6za`= z{MGMbe21RbToL-y4lad3&`}(5FbTQBzBSb>&o{$i)r*kUmrVxP-0?98dJb{zMI-xk z=B9WHE-O=^* z<>F%}Yuh5;Hmp?8M|{NFZyRw5o=RX?Rp9G}%i0aEu@j}35ToIzG(s40)nMJrZ*CL1 zM-MlKE2*tA`k-gb2ALmBj${Fw2Y%D{km&tdkIFyScxAXpa z>b?-68e1@fn=ecA#*O09{my(7%Yp)rhNxFcVPxuxD@=~$d0BlWB#mutCkxXb>}^?H z%tgU{JzZC>aquToO}(ZsFej4o%PzM%;!Od_1(;o#UFm^NH{|E2O;7LqtOI zsy)3HodNQ_jg7Fz{k!<9M*l0TmD?*9b790i8wDD)VY1sDhU-J^8XG+%&DE$8pIgjF zkB7#>Sf9fd&DErCucKRFpW3B)RDJMQpltO0@0mv)%15IVyC1rml-q2Aea3f3{u9B5 z^G+jtv3CVbsY(W|o{3de>{TVN@+Irv(NV<1mI`>!V{zJivSBmSCPYcrzb$ribI$mC z-(rDkT;1KKu<{BuP$T)yUH&Y+K0C+co?Mn<#=)B4<*f7LVfj?3qBg35<{Wdp)KfpQ z9KkgaVBQc}wJIf?#f|V>Dn&(_R2u%i&b)w?6Q{JQJh*)ogBu#~mRs5O63Z9ZutS?P z$UiwZ^=unej@0^~l7XP2u(2X=&>)mEv$&!@@ncow;M!*0KlTCyPHA#Wfqn8-h?GWt zqo$qn;&M^_npHIRz%}7gZ^lVFFdS=e8~L#J+w5CovTMwI>clP}?8;jCBV{17jc(Ht z3$pwa+x1(*={!rX<(=HqIMKP%Ow7;umv}y7bGxyf#hy)ND^mesrjq=e)poddHCzg_ zJi-ggqO2O_I>=^!J~rFa2CLfrb)70!B)zrup6R#h;E&u&qN#;#RuM-Hh^BC`=ceVt z!Z)|S|CBM{j7^GgOE&0!)uCTHtuav`I@!L}^Vke2|rsLzgvndh^#0Zf}3@ zmC<%5$$Uv{ctE82PWCHjjF-QDYH|75Hg~E`Ox09srIFC^dw&Tr)ekYCGTS}N&mDO; z2+`=KTO>=rDfjcKaT!RIr)THJ-aC-)d4{}@VpRmXcER5p78PRSEYg3vz1_1{GU1gn zR2B%n_noDCYYvjD`kwlbPeH=%gtI62=|&{-wKCoa_0$c!?uEIa>lruX$HipwhFhu< zUBMLNlyyoTU7DSVxeuKfL6Zg}P8dKi+>U?5)9U;65^RXySC%H8&f*nwf|6{^dR7f% z>PxXXap-0y<<$Kp#ufp3czQNw@O|6XCTmo$HgEm;^zxKStM29kCS}O?tVMB+30La) zA%eNXro{UDm%tA~%&WldfqpF{I9OlJ9DPzsJaZRcDG-I3FAeC4tjfiBNrkkOU@UQK zRQ*3>eFan;&C+%tKyVN4?(XjH?!nz%7I$}dcPB`&;O-tA0t><2;omFod+&e0IcLwD z+3oG=p028{>3X^zwXv4ld6o2H8`G4nG91+f@_we0tW&d*Wn$FH=p|;Zd3h2#6-3dh zC3B8mk1qrGAX>5e`GiQ)dOLgjIF(8z0NQ!wj4@3|eN`_^;NUALHe3@e&#>(=f zu&^VyZyE}kHpEEE-H?@6!+l|nQ^dpP^A(XySx@MDqCl1l388e(Gln$5M5%JBGwCO^^%-saJD&FUp8jOdaDtGX;A8Iu(vlUNvY zb91Ut;_selHmw(C14?LtbTVyi%7Aw=RU;{>tnWVe%v*Ve!mTdcCYyKY78WoA$VJi; z%cljTpQ$>Q4WM+BQliQGJD}}IZPwbYL-KkIo;z_O9o^7jAyFeC5~V__rO6X3L5&Vr z4nqeM)L8LF_*`pMR$c}(kRWEl$m?-Y&_+C*6d>n;A0EL~H9yLkeR66_nGOS_pk3HBk#hX_-J-5-izogl=P0 zn!kV48MUxV5(~;=g=LJgG>ww^vLH2c`3XOk8XQAxl>PhcMk*hwFYgg2_3!f()MGi{Bf(UY2zfJF{V z1VzG3&tO2?Fhi|enBG{@1}4~^ngMJR=5Ha4us~*Ev)aM1g}XmUibT|91^I1riVWP3 zE~}`LT3YtOStIT~Ij)f`sl?YXqi~ukoe~bDR#+Ft>uD(SfkG)|@5;Yah)bP8T7LV@ zhxJ5~_vjSAq?%ijZ3iLqUj}nnnk}=m)*)c44$AmdtEeY+x=i7Rqy+Z)1qE6#kRbkE^6o>JU4w4+s&4=GPR$$>VN6r7Bw4DrcH^D zWy10>8;T#@0<(~&FRkbbU2%DBq9+;A!u-^z<>$~!?5KkTouXWMpE?UKT~zI)6FmjR z?i3l2?VK0r$)=}$IXXHouCEQWWD}OXKql>`E@K2%h7cRo?cx7x}p8?LilW8Orn>)hX(yyt@CS@L+1&CqG zQ+)Fh#{MBVuYRVGNn{^~EqO%>!$7E(TZVFOcF9I`m1r_7{m$U8q}{i7cMD4yK~hf(8|rEfXoHUXWR8fMLenRttwu}=*b7L#MHwc8>7GS*aT%O)ktt+QDY@Ktb?dB zpE;nuJX=jkc!?(zbbL=ut22<0lsK}QLI#_!6^(IUQO(LKLJt#TJ+tkyrzEwrGBrpj z_}&>__e{n5L}HX->bOs_4DT_=%_QZa8%H(E(-5^)1`!GfWP*=e@@RU^N?oKR${a_j z)cd`smdI@aU$Z^++{j_9$Db+_s#7aLlDK2d6ag}%Wfl$jhk8i?gw+=~Mb(wby9Xw& z?7#>JScLvmctA3J4CW{$-Vo8?hXg zBhe@?dtT0~&o(>LIZygS;c|G~qL!*PK_1=_aLs$Ymsmq^G7RZM^?j-So1#gOmsii* zg@3re&#s2RORrx9p2LuM&pYEgBQk?&A&BXOKc0Y z!!q1|GQX=O-2bNDKemU=q0i;DGy8*@@pYQ;Mv@TcH|EshBD!Q8p3y)Uf}VUjgMN`3 zP2!lTC#L4m(+$wd-kt&){C%r(3T0ZD4Yi?U_;&2#y8Pn`g=71E-Ij2gxc=gFt@$6W z>4UnReH=F#>S&No?PxRTrRn2VD{lKGIDQsq*G<0%kSRtY%#!&VsQjI4h*RLb-m~XH zv*YZuu4Tkk99kH0I^(ypj*d(Z;aUx|I*LRhLuI?Tv0#r&f8Qrafmhs*5O#d!x-jeS zbBVy}wF=brBD?Ch1K6I5c&HqDc7 znz0Jx$(E`V*44#%T(#!#`=s69yZgy~)!vNtg>*mHDDD)4)fE~uQW(%am#6+!H z4TY7-Ox8Ko&)WS(N4`#+hg0uSkU+LFn{FuusTk>X@2>6pGEAScdw z6e@?qmQ2S${92vTp<{6-m&lEBj|r=;d&=G|?J5b{m{jaQ#q|or#wo+E{?;VtoUKn$ zTXJH0u$5OvM_=P+U%6!38jV*ts@}Z&_f@1K5k@Lhxrla30VC$H9w~c>M=+5xB&_T> z-Tg3LG_tE*9TC^ooK@5zt05r7qb-&i1bCm|)OQo?e1_@*D;c&ba4t?(yI(pga^7gCX*0VDru7?k_ybSsH_b(O}i1Qn^+Lz>v zG5+b9a4SlO^NF(!_?nw+L4NqGFe==>btDx0FGTwmbfzBP377C>x|j7ktXHRUw#lII zU%b|O+!5;qxr4m~r;AtZC|KLRaK^P~5^yt9fqgmg%b3h|=p9&uv{^l2O$%L0g`# z)XbRuwvbRdp{8kEo3g56u(%pm=UN>>AM*k(1M?zJw<>nc8mY!(Rvc%P;Wlez7o%7hQ@%9 zGtcaqBgS(Nqi9kf&HsVmLO?KYcJ-y5h%yH4)M8s3QIhEA}ravC-^hC{j&cD%7tQ% zaP+CHG*e@Ic++O*?03SS-9RLf>eYTuLcl01n#J7J*n9iR)Zr55b^+aqhy+5{*J_oa z!+Zhb{s}ZowGRR3e(vFOazbV!4Q@;XpQsus{JQRS?FRENL!H$`V^KRP>KYn*j$FvE zqu3QCI z9(;?ch)E5xPpM|)YAQFDSz#+Dr6na0eZpa$*VlX8{le0b!Tw2QR-WAkGBPsAcq98) zJ2y{C=#YKPZ1MyDtU73i@Zipynwq+hwO^b(sReZ3fRd z62;z*ElKRnnY!ph5ilN320<|DbC$}2fmNOMW?MyWt`)cw52_q7Z=*1`f5!qO(dQ#75HzSu+v+iebkJToc|+F%o8b7rF> z3Z!@^6~mUiX|bofhDcDPZ;ff2Kzwz5cMJ~)K13I2n*{6Wl~;|6+X%l01MZYv`(1w= zSy;C6tdz7`hW{q*YlAr^w;gvHA;9~qt{h8dW}#MXhwG8`pdlLxmzmGQ`H&$~LQYQ1 zzjXZHst!7F4be&^My0EgI0(wv#O|y zQYaVwFAe;sF4T~RZkI1ZbR$yzmLQsJ_4knPFJkC#G zEA4*k%`Z{>MbVAjPiW+#Qp=X-mOHnjT#=WKCv;%ah;1}dbmrK(=^PkZgJ?aqP%N2{ ziySsoQT_`2aK5^+;pP_fk@47D4o5ayP_|-2*=(Urf3OXns&LLAPx00^5`r0F8gxD$ zWp@3gBD9-E!cmhoVzHk$fX$Zmk&_$+_MZXyhY9}26`pu$nCe|$OGkm={nR$I*OfT{ z=Zo0V7E47M-xp`UbO36wI$qr;8cNJdFUJnOc0C3?226%@xcCv- zpO`|`hy_T9BIRCd-B!f;e(c#MPZb-gljoa0jy(9MI=tDDgbD*I?c|xl94db-kjE9O zPVbK0g}*)DpUGa_{7Odu-G8F-*gduIZExXeok?P)P*p@jK%&6F(GzIoN=+b^t7P@hLYLfG@44vbb+OR7310G9e8}Y6o3vk>1 zN`VoJGqZDJ&|lq#lAkB0)Pb4$rbgnzX#Bf0+v{j)7w{+qvt%#GBZ~#o0d(GvNPA5* zbJ&nc=H!aX+)zNexvT}wI`1~?E%z`JEQy!`SYpwp@Fph8APJpJ^4qBBJqnsb5>_jC zJvO(#Hdj$r+6mUtZ;ZJxA|0IhcW=Q_ac6L!Uf%RcN~izsQ4 z#qta{OeET-fy$H)Km8`<8RkiW9_+2z8daq`^^U%>^h!!Fx|T5njyL?2Np$a3R?;XQjU zj1aTNT(cBcRSaL}u&qqknv2pDwphKq2YDsU1Y4>`HGa_huHC{E`p)8AB$Xk?yW`9X zOr zj=}!b0nD2?{?+Znx!&eLNl6+00a?E?f%y&4w^*r)@=Ds&+}yDjMGK9D1hfUge*1p7 zos3RR9ptMiDFswjm6Vk^i|}GPUaWeoYM<|Labq$)kDrTClLMaLvjMCQXp&@3W23r9 zDfW_%Iph8L8JG9jbX%}}SDEl#-dwfGQ_zT)+!LA4V{nhSdcrg2Fh>;!;~LkevS`{J zR-7YamNRyu3}t>Xp#!f3X^Q@_z(Y$t|2H!>MOblRN1PmTo2T3A%Ee=*h~?v5D(Lqw zdZoBefO#_1n=i@fD3Rm&PK}Rm<(@{v7>HJGPgHeQm@|H09~cIm_ULe2Bt615?=A;R zi{DjAEd@B?1n>u{x4$;!)AJ+>xABSHSR&HuUO_S{yhin_5LIGpB4Ep9qvQHlxpJEr zWJtMff^E22xzN2DuzwS9! zVLk1+#zB|c&Gf;aE)sPoy>NZcFZl@?SzhlI-LVIst z66v$n=EU;(L}$DXivpTaZ_9|hH`ub^{^+9Qa7h+vKrVbvT_&qVQF++x$=&UrrRbY!UZg%jk@^*r?>Z>2tVLOlD{tB$2Q9o^mh9Y|PuxzU(Q(a_HJwQw z&cLR}hJ?KgGVJ%1!;!}Flvu$Si&G!LX6x+8zJ~|OPav5QX>K=vlmfhe3WwnH$zVJ5 z^J5=E#r~{)>HEm)ij{YPe>?5`Q4L@VoJ%>8CCbPROVz_PY3xP?`(a%db+yW%E zCTC@^xi*1*dQ3K<3p(KX$^|fw@|DjG__ZVyn|pyjTD?*3GsS-)R*L;CddQK#h zNBoDmHY@n)v)(34ANJ4o*s>{wN#6u~B9%!cSgkksTO!(=ShDI4%0+;8=lWz+&7Q>HSdSg1YQ#}{PnTqm9CJAG1vpaTVK3Pl~gBbMx4@W zEX*p$yOH8Z1ee6TcKHmvZ^7~q_N! zW>Gk5Tykf8+#|$%TMc~~xhL~p$Fs*{N3Z`NuRF;IP!@NDeO0H^ZouNys`|#s8#dqSAU9FJA6n0N{D}N`K*I8JNDiL6<yM5$>POIJjV&R4i=TNR(QJtyC?ILYESX$I}s?y z)R|O^^e)qky6on|!E|)| zkQsEyrnLVTOp^Z?`|=D~dB_ zCRXHDMkqPQHOHHtwumAX*9_Iop1Ll1_`A@ceHN?bZR| zXir?qk|%2IE41y}b+Lq{sQxRwAr~)`SC*vt0d7^VC`{I?WOB9+ceYp&QDU}yl+WfY zj<+|H0v3FmNeDR+MV!c{&1g8*#4UaMnQzreZtpEu>*J0v#^t3vh$^Kg3!;oSx#w{m+d@X5sQ%p3<<&7boj!r9B6 zoMP~Gd6M?@*)ji7vxZZ=LZ$F)%E+-HC|9ntoraKVu8GH*$&6&uYlJj+pLvi`4;g5TjdT*zIuA;-7A<)Da25r30Z8uJ2-)- zS6ydT24ZS0aYvb|6~OT7%qD#9YW|Nxp(F8_qerlK1wi;Rf&(d~NcfI}@;}~zgZ{Y5 zdHW+?=}(s{gn;!~(HNaC#*%TINHxWNo-CXzp_FX)T%zde`Sd-FAp;7iFVQ<4+jxZX z3T8p9YVd?*^DtqF=kObvF;vj^r!MV4ho6+93UUsjD1;r9-Uws|4D`DP{BFx14+LMG za>uIWDM0AKG@iFxm108N0`gD&EJ*O-ll z`qbrvz|u+WSR7jkb(GU$EGetC@OR&7;*FRcWXR67kWFK;n-?m*XIdRoQWX;s<<$`^ z@$N8Xtz`z&+~DSbWcHgh6@tx3jw{%3G7fvm0$2W%HcgVpX!LtocmxS`bZaESb+fmX+4#_Jic zuE_J(AQ$YuxP5%xO*r0w98I-U=R1S`983eet>4R zOv(kyS7kVxjEz5R6cuH#ROsFP$PN)Ci0o*;8{Jm6a{<85HF3k+tK68xbm);)*We6m zs~yOL6eLB?3B9*B^8FsfO&&;uc-={}+GPYe^!2oul#L}fW|B(nzH)1zCPJ!4r4^WVUhiUswRmO77Pb2ZO2XVh2O*Ju|+zJ$J!8%NzW2U@ARX_wQ3@nhYF@Lt*|bUEXh(hJa! zz9kZahOF4!a^)ixRuEkR;5n;#vC*FIfrVWr%2%DYB`XfnauHEwV6fnQeqe;v7BN5& zhu0?oormtQJ`kZY`d2oDd_`hDD!l5*0atG{Dtmx{bx#1H|4&5)0NKba_UspMh*v*0 zi1I39G>6`M9G5h7qxjhJ1@_NYDW;Nz7;{8B4~aO~kyEC>nR3B`p|VyaKMC9IJnV`V zI|0bTsbQ-W;{V+2B3b6JgZlxpB=$q3;>!NOAH52^27U@sMT6dfgBX>L*t3k94%oV& z+<7P_e#}<0rn)$1@48#`-EduCfAzTb-guqZ;C;ORAlN-@{H8j!t;q#cKgZ<^><1;1 zwaXe9@Ywhn66%ApXz1ro_ne0vNCgF?bZRLGG1q&T6{Aqd86AXiU^)*2V8{akKc@BK zceo7v1LmzMM;h0dXhF|r41!9(E#qPW{Fa|GnRH|F?dWS+eRTQ75I%JCA7tohl|w_@ zsQ$Uva~W988H@MVC&1-xE#KX|-u-2EffmoV8>V_i&`sL5e=i$k0_1A07p*~5CH zz6;9j9d|nXHu)wPrtgKl#w>xzl8`1CLHR9fbTB=CufN1D#_N??;MELw1Xjs_EB>mu zm!xRsE@zu4V*gZoM3+%e*6D5Hc!3-j&Z37U94>xlhS`p|! zm2F6EmT)Q!-p7i?nlg8VR!HqeT#3P&8jM84@_q5A%714=61+#@?eeqEW(mh_+Xmg5 z=6T2P7TP*-Pv*-AOq22Q3D-KhCMad_P|#X|BEj}OF_x_n3IUeKZj2GG1pe71H!?S!0Q+V zuk}oFxa<8Z`42S%g71MVbB-xr8=Zg#2Hr>Re&_5liNU(Y!bBBUpaeRU5E;F;Xg)sW z#$RIYA*Q<@HKrlWxq8iwVJOQuReR+zz5V#3#cACg#j{PPJ;9JB%Dcz;6}uEI-;wjW z&5>(WMY)JYnYe)lLLUx(hspVH?L3T`0~dTCzqzYdsv@+74HzX!5E`H`XthMWF6Un=W|;9=)l{7SrH=7*YAI zo1vBfwRkYA1^kjR>AIg-C}E&?s5XBB>-i>Lw&~3~X*0rVf4J;Vt<1A5jfOGi1J60j z8r)6{UZH63ei@9;;?@_6>+{Ctk2pOQCs#UJ>G61=@|Ymo)#FTL7#J7;?wO~(3|X)f z0}joAS@~Z3q3MtAyF59EWZnoJnZ&-ok}NrcZ5Wf^d6SNN^mvBtfw>-(jhlqRo$sO@ z`%w&o5XUg>wAvpx>3nyu)pWEq!#PUvd2iv4ZXeTPS;((9IlP>#4qoi?4AXrzG_Soi}8`M-pLV^Apl^? z4xnKH9Up%xQUUup?0Jq2F`M_@I&V8+EF#e(DcNv8rVuMqN`^#2Fg;gY0msNp@Zyix zt0lCtr2XkT zL~5MzE_A3$W|f5(!+BQLRsB3$S)I|oB`v<_3190JwOX!Y1+vNRq`qqO7lbs7siOLo!PRWHCx3kvZwD9pIS?l5AtSW2Kt77^C%$Mz^BGopa+`A0@)WezC~^%~Aj$twXY#>e0yYAOZ7KKG56 zHqU`CpI0!JpS~h$@ExA$O|!`jjfU-}Xn-#`L6rYnBsDLpQ60+L3eztn%e(KIEY|hUUoVbI`YaTqFDN6oB zf%Vs*_tSIqw+cf-3(&jQ+t;hu57BD$RMp*Dn!}Mb)ypu81bj0wqScv+)>2mLik*h3 zXhDlz4aQiOq(-beQ#RiC41U~RPwoMR)v2S{U1|4Z2KBu;D{9in_x%zT}J&5hx3-wKjzwY(@ndizb}ruD4? ztrDaQw~IohHoX?1yg?@?lqPeyR%t5{4rOosW%_3kQVq^h&Cy}kznAnQ5IOehe+}b4 zZ2pWFd=nY-vJz{G=TCHD>`g|&WT)&eYVsI(;B)%GGN)%K(sak61R9+LRmvG)496A| zmU=9()%^IBTB^IMVLucR>AhKJ;`t1I`0*pmjyHdu03`E#VabOtC|o>nQ0)5J45`f&skt z_;M#Lj(~#+FQ-8%KP`>H2*kwGUq3==*n>|9Jfq5paRfBU#Y1i8B#9p}Ed^*i>_nqw z$?0j(LyI9-JO20>0PK05!=y8u_AjZ}dJU<9Z%~o(C`&hmo@@Q{X7aJeS|Xs);{!27 z>nH_jVZR3Mm~NP5SGEo&<>5qw1s1fjKvK?HJlc`TJtb)7`R3HyNg!60D0v91+&aX> zZgY9CUNonBS~_u6Z3hpWl&lSTI62vagWNvYwDwhP?rcnnTHzUG7G7tYZ9vN(`UdAOW9$*o)q0|V6vq;cQ#i4!m-bikbysdkM!aVtaBlp zvkaeeb0btrq0V?hhT`?Xs4V>GQlmq8YD~{}+i<>C0wO3FaIR9;<;ggdinlYX<_+JO zd0(5v{3#LJOOLZ5VUAF6<`BD&3mqTEj?qjhoNXGTY*5c0$B9`{(4CPK(-c|lSUBC~ ztLB4)0(6jS>C@WxLmz>cQ&pPHox3m4apOj-Z|O<4i^|zY0}Dq!2cHZG`STJV-a|Hz z2XK=|VmUoXQHAMIH+BGZ-Ik<;-AqXxFjrVr;A9V-p?P*H%+BtO=I;Ap6R;VN*HRe1 zxe=B9l8d2j3&4#Z87MhWE;6`bTY8iJ`90x2{x#9x`(rB)G=H9GLnda8MQfTr=Vc; zkni!_CBwvcj`UGpOhZwEe?7BdmFPV~MkQK+??#8Se~(lNIR_*3F~=ar>kUt2zgP+7 zHD1iZV}FeE1|~9E_#nO_q?E1Tgxy1>-jb3bK3)_aMY=>%hg;}dcxTUJf=nC}R46ho zZn(H20`fFue2au8vNJO|(!mEmU*Tv5 z#ao2LI5G|%w4BE^7V~(?o-)c6vq%Azq^ma?kbyU@k7w90l8A#FH0}_?f#Z%hnXJuQ z>F(8sE@x{?=iVIG93J>c(cKO__D{45Ghg-RuLMdK^2 zWS=Ffx|3fiN62tJRiPVO7uv62akGz~VWSkl@{xL>yhA^;O~n&JimNMefr*RnBBvHi z8pQb2v%~qn<~>FG8TA1bua~PMHy)n3evoIQybhpG-dm!clAwV0QH6bLX!-MjQLYnAAEjH`~4r}hiX63HXFJ!4gQ<$TtWXd9(Gp1%cS7|El zA_L!Q+0;~ol0$;#5i#{aeDg%n>fr%nVQXl7Rx{6U)phWI4{y0d}1x5LDBBspu zuO}GG%_h=v&SYd|JI-3yxiZ_^x$3&EWty6rkaLypwBBxv^5{I+zy84GS1PQYFzXOKj31`OGEN263>buh#8kivbGxKnUWOo&tS5Y^U zh((MJCop+$Bdf~4%wLqCEYGFq$H^_z;Bp6(kSU#x7K24`uR z!TJ4%hnBzMuA0P@IFqYnVI4yKUOpIxN z2n@7!w{S2Nv*j$2bub7^G2iC!4Z>g7z<|KZ%M1El-P=0p4?Td>m;4yGJ(fFg&Eyo~ zv)_ik@l2PDc~+#Yium|6rL6a+kHteCPz}HpS9j~!4@z5g+cC>;k_X&v=+^)QvBYm0 zEp)-^7W3d8@V-29ag$ovbl2eT3%g2=lyu25QsyNxbSH5 znn3(>-;sQ1WkjWRJ)&p(c#bwaqZLR^1Ca}!)frQ(!}8Wl@?Y@4e@F5AU@8}hzVdQ3 zjpfU2pT{yg(WzQ9r2n>P-d|Z#b5wG-$1QU8Syk2x<1YSUV&ES|;1;a>6w99zlOq@8 zdvHqUqSCgal!wip@;s*@G=xJSE)#W4;1nd{+Px#aw!##G#$Z|{ab+~ii;F{^EZ359a;E#M2n&N%RV8I*f$~TG z|E=H}0F{qZ%q9S^M+Ct18W~Rfp*hI?r#9 zzgnRbZ1u>`pttoY{3ynJ5(@u|m8U);pC6*C8oop_On23PHMX}LXp26V@MXlepO6`f zbNAR`OZeYED-}a-sH5BK^((3hm@JR0yyAQ@b?Zri`_)FqOAe3kgYAFEEk`9sR;&Qw zO&_3WQJ~$6qPJN|;NFb*H?D+;D~sT-ivo%HSA*qAKz^Hr14;7>LKhFo=iG5HpXF)u z!#LO`mHyer{MbYM@at95O=u)fZT>g2bI&u#CRM^BpI}S$|A1C@pg}25|JBF>3FnKf zkwFhXqUsR@c>DZ7g@cCK8khD8H&`eIA$H{atpHnpVw<)L@rC~kP>@N*-?GA0~ zFDfjh_@hmBu)=w~B_Lj39P%`>N^yVD5gqcc8-ZYXVgKpL;T8JjevI`b<}$g6`lE|| ztKDlRqhQs~J+}tOKawR6o&Hrco`tHxO1un6L*Y+xWPXHL=+^azj zB4tlN*uU%Cxm_iRWW}28@?%4s*X)2&)1^;|+MlP~3Ky2)sbqD;eIV$f&RG~2TBiNwzrDLs zA{?TS3idxUlq1{MzAN<8EBPb!?f&00y`_^Uz+0?}l^XUs_KCIAodEq4Or>%kAGs}& zd$p&Z`eQ$0OU1~S$ybXc|E+$GzeG+v_Q$uTGVfL$4f0x1zXfcc8c*pVQI>j9XFPn( zXIwg3A^-CKX|v=4DHm~fth;{a^}_9!46b@k&NfZ1yy!jo6Kzeq9pqy2jH7nX8f^Np zd;YParLSz(D&vaD;I7@>ywzp$W&?FBNtLD(t=37rLna)5%dbLq6;Z<-lf_uRQPMX*nqgb5O0U!C{tPec`9pL2td6$Az3^GfDOTR=Fck{4@w> zC8n@DoB8yujk|XZIZm_xG@eokgjq`fp1$xB$M8I-uMU6HPx&RxCS#88>gUt>#ByUSLyzaL$MpF(X@F}p-e zO#}U(s{ZOnj%$h0--)j@b;XgJfb?MXwPMD$<2GwO`T_CE$h>>v9m6Gv zrvxouCwu!#G2_%gF?cIBQVDqx4+I%5ZEwf=-*!2MRoB%WUd)m{vI?WParZ8CyuR+k zwR7t*Z+Bhs;o9;bHjb3fb@(_W*&>@plk@ZBJlvBdVcYEnjPkh3=*}D+Nb9-Fas}a# z($T@EPl#)4WB*5v|0Rq&@D`w@E1lX`b#I7X%*}h}^Lku4tX}1*MLU1$_}&dv<3COL z+~n~-k|V0Sm(FZ1Qd}IQUn-WRe&|0jVNWBgoSGvxB@4b152t=f%rq2JG6@L{MZv)M z%Lx5#-zp$)p%FotbP(-nG-xYG_(v&d4jqog4gb5u-;Z+|+8ZML=*~L~iE#RIci{i} zL&CMCAW3;2i2|}*Us*gln_+{1U8++-jCuRV4`t2O<1^H7?bI1B*ou5_0yZ9+s@YuQ z9M`+7#ZQmnckST+Gi=tJph?H^54gb>$Y=_ehaCpj? zTBw#adUgi;M;|^YU7o;~O??wD<^4&$065)h2Q6AnSy+^2`%AJ%Z~Mnhbh_@&6W2En z)q{AR=hgXKw%Kow+`v>}pES?t9~0vZ@<9x;b;HNv|F#GJJ}Gq`qrB$10Cf)QLC|y+ zC~T{_47K~K2Ae4znc$=`g{J-bUt>*6xU0Hn2bLH<5Sf{kDfPOsf{o1B$`toEJ`WAg z=?D-@wId(RUkC$~m%n@t_rEyXFw?(RfF`S98jxA_0R?IXgLE{jYxk?&mNabeYg^-Z z9BDM?L``)tRPkrY%Y+2~$=@MXg6f|&B&nD5Wql9N{A z+L`zTYZAL@IZag6vn*YqSZ=%%v1|9$wyOATd4Chpbn@b(4Os3~^2&`h5@C1PMX*?L zsc-J;?0;Ge;bmbKiY z)gSpr=$#b5dR*u}(HqTJ&OglhEyX3M-uttE_W@E@+wCX6EGo4N zgVAr_{PK(IJ_(LGnA}ZjdU`)&f)WLSG6@b2LpoeO+7LQ5aEz`5xz%Joa}xOchRMrp zjFr&gimL%0SbQ@aFruU>SL&wod|4dz{R<1Q&nTy>8nv!Ge2?Jbb&jNXc|6Xq#txC7 z{}sS;U`61$W00rs6@j*>guOrr$YGZRu)#g8cKRX4_udm!bJ`dGv+;J5`B$in-R~4W z_%AyfA_D5ei`I-_?-ODY@9JiLKzrf|8oT9bK-+a!=kk-GjW0vD^=|@cG4tRNtnOoO^SWXPLJ5%Sy_^rr0oxFdb zm?J7_+EcQWB8-`nZZnP+ekZ(|fn3_S`b~J8kE4(ru#YU;M)1=PNhBwa;(gp0!>-Z! zm>qxhYDh!|FVwYF#8wBhiB5K{wLS;T&V&iHb@GRMW4Bh>#czAl7Iu~~9El+w&R|Ld zaZm>DpzrKLu$ZyZ76&z6=t$3kzjPndo<%1-fm5ZXdcK@12)3f_!f+vu1e+uGclk~~ zVa{rD9{$+K2$M2j7EO#rF3gDpvsd#5Hz9L**xiIT-yPrC(s$3*}lc~$OA_0e8;dB9D-`I zBeD;zZm_6L`Q*BggJ-RyKJL-|ERyNuk=Ve3ej2$%XDY|A!JO55)RCu+c|q<@waEr; zYRyz8qZW`1hMti|(u-J<@G?Nxd1Hb^#Oq znVt@a@<~nzE!Hz}>m&xV{@K^)%%NRPnMgra3FQFGj zJ9X4%yR;q&>FJW77x^dOXI3{+eiq8mF2sA^cTOnG!{qezkx^X&qbV@T@@w9>%j4(= zCN{?St}Z@rB>V|DxQG2)2yWt|y2HyR;L&0w8fY!xAL}O*(XtKPzB$M(gnI|u@ds5S z0{zG8T*B)3d>(e+?0sDkM_vKt3FUE9H~9xiU3{PoR7V@7v`ACGq}98TQukjK4!b>Z zy*NKo?{tCAHlQU2G(uDH$mk2SWcCDReYG??9mU#jyCt87WHt8WJ5#1JjihoEV7#gI zAHdr5*&{&ByS|3)*bQ7+^PUVXdR}D*<&2)doq`KTY<4*#g`b)rb?8@Bdj#Ma+#n{0 z8w{UJv|c{H?T=1$W|U)|4jJ?$ei=am#xL}g5R1VLNy;A^8T8WT|Fhcbqs<2}j^!_( zAU))%2?hKR{?+BV^6k&I6?S6(M?`k`%0+i#$VlzrB<6w5A0~h1Nnf9wX}2{-e#{6I z5bKP<>^gz$DR01Fih?VQPkQYOKDwAjvy~a@XM&5qVWdOnieJJuu^`H9#@c6cpBv@5Dhq_ODH znK`eBo*-DuM58QrDjMWD>7+qJV5{bqLek(vG6^sU&oXTPa3L*2s zom69>O$rBJhXH=qi?rSMR)fIj@T1)ZSQ>-JmLXeDLnKkXtClxT-*vaz-E~)*=w~vF zV)02ltwOIW1_^G#-Q8V-1b250 zuEAYHu;A|Q?hsspJ3QRo-6h!Gd^7W_t#4-ko7&nvRS(|fUf%1|-KV=xpO*Tf>dno) zpuR$hY4Wc+y&>Ux2v6n)6^QkZMFHhSmmyTxNa%wDEy>@lWwk3>C)i-Q$%XHN*IfGD(EPl7ne6sMz2P)bnR+-KTz?!}nH0G@7F^z8%fmX~QNG zU-DnE`Oadv@;LTun3tW8eqID*;x|T9_M=LPs+(WW9kqA;Va|7K-KN$biPM8!eb*#4 z?@iR67+o3LEQ;=#_)|2bP=CLPk) zu}H9?T=gMXo5_#WgZ6j_LA#&tsGV9qQoI6b(G?lL+vw13eIKPL5o{$AkX|NG*S)s& zi>lUtGvv2!S1$GqI%yamJ3uIWena_|$HO?ZzSr`yw?$RDHwdSs)sVtver7mZ{oYN` zg)P|*rrJMRhvnC6uuyr>{g_K$IJNzi;twCuR`kNd?{2NB6pOV&t`?%ty$9)|hw)j^ zkc(Z;jL>y|9PNr=!?^W!eAlzDKS@vOc-&Ue9mZh$7r65@KYlxZ8-=c5c;;9*Kt5=C z`UbLD9=}d-SbbicBL|5N$j3`_K0;mI?>xj}NY($jR+gt&NM^eTNUexCAKf5q58vXS z20`S;#OLp6+uNFsf1z^dxY$;R*874$3{8+gfME1*WhL~FENVNFJQfUPskhtBr&u{G zyGCTW#pTZmS3&en_m}7YQ)hAm=4!JVLT3KKA=tukC3W#@d4crO7daCu*7&gu0=7zM zB#QEym{=^iZh{ZhK75mFYytf(ugwmUoE*>#TeoRa;VfHacI2N0@T_+{=g+Eh-8{(y zO<8R$aGf>`#E*9Akt$aujFXjaBep9J$6yigc>@zK7|j=A95Mb+xB~b)&Zw4Q@GoQd`}AvL{2<+UI%Pf>s*VSR&=IsAF{Wi!ye=35~601N+RObDxRXHHS5RW>tHN?Gec0gNO-kW@{6b=Mf?9D{TqmeTBgH z@}iU@g}0<71&}Zz=&&N6^9(yG2f4H%F#nmgW5^Y@LcK5b3AvTEUx+`ZAQU#eesP}3>H`Qi&9t~RwV7pE6NNYuOt75v7eTy$r09LBJl zVkuv12m@b+uHz^Xc5HT36sGn!cCFO!tD&Jj^-rV4lTLda*3D1%1}t?dyf|kShbaFZ z+>=KrkNUwTjBTFi4;{})U>Y!u1m+AaI$)Jaeq&hW&JnHc1h^5vQGoQ4)3erwClM$W zLb68krH0tWMV}rMo>B!H)^dsbgTFf&fs;XDRZplaS^9ppiMc8Ede^!P{Kw>uz_>ZS zr}3%zLr@%_J^g%Bw%ObCx+z(%w*qCM3@|6_Z!lFp zAA0hN;xLO*#t@`_kp8S)*%u@}jT0&-O~rYnuFilvu@e3A@Rb)x6-$v5&{ zs;>#;o$=}coMS(e(8wD+VWmXEzvH#7gqMYqkny3h6^{hp&Zjiex@r*ET7Rag#gwP8 zbjFSlzjR20@g?01WQWoMt=K@^KgN)4UM*(;!T+~$izu@%~;qoD!vi@B+m z-Gk-ue~)dvWFDng8>)_D;g?(tjPn~!U4A2!Kop2f`dNUWO%do&rrt8eJKv2wN({B&*nHf5NbbGrRwlx#<bwkw!ra(Q%HFeSSE!(-bt*moOcM!~w5)*vN9buiL|e7kQfT z&Y~k-hJAhFb(B?-I}D0sTRfNfYEvv5!GV|3!862^?GbX)Dn3Qp2ij9<-V0({y1-G8 zV2eqZ?5_lG|1&UJP}F4VZAO1UP8#X3Gf;D7$41>F%cSj$?}PWvt$k!h#mIeE11lDX zKpf4$7@{oCo`g-bZ|avG80hBSp=Ls&D?*cWhgGxXU!ntgcCmFn5#U`CAdo2Hwbay7W#qE6VHcZ-(pHtcfq}&z3~~ z-~TnbpJW2Mky*q|S-lUxqE^`c$S4a;K5zAxil=0;oF=D~OY6HJe1&d1V~2;2reGk+ zT25fEGwQLxjFr#6SDRg0o@tCstVfey0OS8zr8D8vl{Ho%7611T5qVOh<)||RdtM+Z z%%$Ez95E+H50c7K>WCx7@K``!#~}d|beqkLdiKjF1+6W{h{Nf}w{gLigSW05v0?SB zj(3Hzb@kyUKyI@p4L@nyWSGcL9Nc?d7}eO4MXx$Q0B06P(*;timu4T`1Yf8>O6j7M zb$Ula8>A|bQSpqp)qI`T#y_eYie$@;lqY7Q!1I{XdAZv9I6FmpSZdlkSZ;6si}Y}+ zn4iQ0y83m;BkZgpM$hKS9n&f4yua8&O43@y)Asw4UsrU0AuWTA_wAcZ{LX~Q^sm! zpcLzzY3m(;fteu=7T%DoU$w{b6|Ltv~7)SR@yUDzg4Do4;Zgp7)jKNap zu2V2GI$UtI5Spqytz|X8?VwSL+xu7a6&@L0I9rc2Z>=pzsr~0Yh;-7S&S@r@!A@OyRA>V9LV+Cb3#lJ3^elYD7dcE*=&he#g5A5|75< z5wb)7k)O)KnKy5z)&Ay;Yl@!&&Y2eLb)QRhL7?|m?1A|*-YfP)Bxh2t#!IMR)B(Mv zB(68wL-fYP#)14^U9^D+8JISIEQ^P(mmRbaX}g+&ZHhO;uLJ$PD3FH*MK`*fyeF7X zNRv5}0n30Aw-ICjr_b<;TclwbxnV?v>84!Q&M_n3(x6*xjVsQ0X1wd=BOmn~6ht!)NCPZ)iQZ_;-6IXuk*4$yE2BRPs8m`BR3HzSE$PCF7&CM(B?8&EB-5f&wN(dK0y>+U4fJ5`Kg?g z@uNh(kyp|&e^&QvW}20eXAW2+I-gKVMco)b5}AH4Z)!$qGgwLS@a4cw699z zW~>2U;Az+Re%S(ZrHV;Ik2OOyKM4~PYWQ!P&BkdSX~Ky`-uiw%WnLjHdnHGi+UHAa(} zBm({=#{cuOTMGBTvA}=)L4KO#@*CWJ?}{e3`NzbDpOV&-L!Zco8jdelkgxiCBFXRZ zq!tk9P#M7Q*XZ>dvL~>jsmQ?Z>3y?4wwcI>70GmMAslA<`x9fVEYw;onbi%f4{c^x z$Xk>wkN4ofU%3*F>_N2UxFptigNo5txF;mVSCB-XX_r*2MD`KJz9TDg=M!Qn^msad zl5thNk?UY=OME*bab?sQE6Zbj$(Vhu)>G>fH2M>Rgh5q%o#0m2rhO(xnBmf-F zCpa&42E+1kN&Snz575xl7u3|m$d-edV6|zCnYt7P_4NF6V+`(UHrscy9`O1DZLa>? z*z741*>pyvPrG)bH3M?lFJ#PRj>LQXs$tdfYWU-aNx8s<)vuw(lC{3Y%x>HS4m1Q`K@i5ger=1 ziQQpa?}`WN8>i17^qg4k$Q5Pr<|E=WPjp2T8Mc0Fa|lnPndRwxH?Uey`Gb5d6KyXz zI5>-<`|V8q6OWf)U6*mrf=!V}KcxOAw(R~W70`Wj8;AqZT#mebDcuk=b-aDFOoJG% zHQFWq`>Z5a{#OjD@s~IY%KitV(`I_W&lOGvbB9-4uj<9pvC>a^B`7+L8owG@n{2B5 zvPWr!_usW0z~qdofWkA+n@Jh1S$*O3bG{(^jgYiR&e-)t$UEo|wR)<}l^AlN_2bh= z4O7^OX8&8O_(0L`Xsn(FVz=-_^^R_z*Leo9**w6H()9y9eT5l+G5MxTNMjk!UN(|& zy~eU=Wu(e0c;<=(JQ@^{W{sA;i|Ll$~$vT z77d&>5V3L(Y*H=PED^X~e+h!{u}TcrLfD!go)p|KqVsuv$E`*xd6a8?cHuQmn`lqK zlDnexHNtfCLOdw4e*RQ>U>ys)^*#Jz2Np6GQ(mtlKeJBa1kx)kWp~RsL+XMl#8q zAQR>piY@}83E!Zscdk(Kj$k8SEA255Y=Nm6YWab)z=SL;=(hq$Pp&5}L6B12))n(p+mSG*(EdL^{PpfNq) zgyk9_EZ`9=Ry37DemX9LD<++AlDJWi6L)y>oizJ~*`VUH@bnOwc$tCc;j!q{U%FB#v| zKuN&J-RGxuC{Kew_)A+EscjhwSoC6NEGg~wh|t_pcv+HNJ)&tJUtcC2u&%ZP{EErH z(IX=b_ZMuL4_mMsai;$REKQg&GdB6>&-i~S=(vw*IMyvV&AdB|@`GD46`4LciU_S+|z6 z)=)^(kE%h2ZUU3zudJYHQeKMCu)S)>QH<&tVG6gQxz7(kIN2-|^B{R_5^8`vnFFqr zjHowmDlQ#wWMG>OnO(;Vm-=I43DY}~-@0I^!5VvpD8!&8kRFp?Ndg7z@Pf65Cx*#| z2c_bWtR2Zs!;9@tjx~7Z9tGP68VPRu3QVS(Up%TbZ@?K;2iEz5+OBt|EIIzY@%wwN zJ5&G!Czgr?BW&r;&CdgsU%nhnWVQh9(|Z2go$z=+GJ|i>v9ZO!y*}S`4edGB`n~&l z-X8w$iU+bDcCNnuA9VhhIU=W|g!;duQ&d#+FDqO+9oTaN!F;0SbEN-V6CF6VNJ>g} z+i*HnB{3O9A^EGM%2JY&k^=q$Lqo&=5zG>2n2a2d1rRq)B`8Sr_gnrw{IeMd{xX*! z#**0Zy7DSX5aoWK`YyTHwMe^52u)?8q;;lPbCh2?6i~FgopO3+n)&4E3s`a_+1_1W z({R64%ww&I9sghy;qRk81cW}y;`j`dC`@2Ihmr1-&=yuK%c1#ba$9OuhO%FFm}Cb` z$MKR*&-{#LC%L-F*)Ff%w`fU!*a=hYo= zH5s;2&!oaR?c;=vzfnk7V@&H5e+31XR8kL8N=&oRA%(t!xA==dN9W27k+Ka0e*fO> z_p@whF`A%YwF>n=VB5gV>ue7syQ96b)G83gd0L=x&z!@ryp5lwO_Yp}y7_UMU$#xn z3HXjYv;wUu;@i@~XA#^ZRws)cDFnlK3E|`i|2fOA#a!&5=HyDe!JakoL zc#rnW&!=udk8X-4f)m<*|K(bwpFZu5aFh$nY0RwGtUR|96dm@xJE+L}#2b@dZ}QC% zk5h-j6#79?*Besd=f$4G=|R2Qe$8DCM37%H%6+d+>dn0_5_d;Nrgl>2E92-d4OGY% z7qNE19phgFE;sS8>@`kvRI=G!QPbDb0GQBC^at_Lg1b2Z*&#V0WwZbgh%0Y$?OevR zC-=)WOE!x(oVeg{04r_`_ufwyU++hx(r+{)M~^iPRl%uG_#|r>_(W#Tu>;9k2WzPp z^^+4?ur1b_Tzvzyzvj#MUStrT zl4@eM?vUK@JckYc?xyuR>wX7ce4*DFz&tO|rC*4|=md<8XD!>CT#0DSQA!M7sV-Pd zkDn_TN)HBMm3+RE5shi~8HNB2F}u}%k0@+>9Z6_fQX<$?t!oS8$W}1l`sFB03wau% z#eP7be(H3Yy;;fMm*%2VaxIr4Ph*Qm{4!LETE_Rekd_qb^H*UFZJnY^_r*cYqqLULT(MP55vJKq{i zzT(lWTk`1WP_wge6?f}1`On@I=JeH{S&>$@m)bd91TZqB=&D+G)LY#J_K@u=4$`ly z+v5!)PlpDOvo`N2B)DnPVcph7#5&lfNQf`8?)CE=W_HDLjd2v*QxKP)_rG*AZu=> z)a^FMSGgroO7Uxz`hs8Pmo;Fb*c^~+1ez>6E|?WO-Q&r-U$?=rQh!ew;;@&m4(;=Y z94T_g`5*mV${2G(1?$2~v3d!7P8N9P`{b#|)31=;;s}Oj@--&Nn*(^)p^-Zkhoq=D z+;5maiXd-sC*b`DprQy%DH(gEfo!(fs0os&cf{YGc?H=S%8osrAh-19(#d zeSp@#B*G#UVEj=gm%b=yeks8IU$Fo=Y~Ok*n+iERTcYJrA=2&xoCBCB4Xgh~6QI-Ir`tqWW`AT*^Jvd7t* zVbP*^d5f~(0~9=Lt-7oyA{FFXqv|q0J1Ar@&oQnH(qXYoiGf8kzh}x`cSu1`kD!hv zxQ`mmJP&ehk@jqe9fX!17br#r>N~0E#$h&^FKu2UY)}d%7sKJPt_vx{g>Ee&HyYR0 zE_qyaJ;tTjHp++Pj^2vR3uS*Sb@EhHoeCCn$9iuQ@o8JWF`DZAWe_7tTVTPiubHKelJtV2M7>st!Ub9 zMo!rm{KUj&)3j9MxKQ{7ryr+o#?_5G@c!!N_nv^%bg4uP@-{bHn2>^D4g4v;-R zBf|%Bww&Xj0zlv8McArmg4ryHr%N^~9#E8jV-w2RIFs`t8o;y9esyxUEQ;>7y535X z>l@Hyv3UCO=hE4+zE`U&KOtDgkr^a>&!MPs&`ks`Wlqe?=?{}1Y?k{0$%_DSoj7hn z$k_(`lQ`n|EHPtS<{xoE+8mA`G%Vx(lK2l8U8C}uG8BvqNDvIW;(Wh?QCMngQ(RE1 zmL0v9VG!z~q5NQ+WWj4O`AcZ4mh-tg*x3fj(XgtDL(4|EY3b}o|MU7lcevBDi~x+( z*a(Kp%93P8Q7>Ev?TdE;l(`;(vOD;I5QA%zESirh>9CS`XVFMAfh)c(5iP2?3_1%- z>>&jp0Nu(&b#*D#^$2v3=cYIr$IysSiq(?B(9_D|`EwFd7T9ok(nU>)NEON+tKTOz0Nvz?#Jn9L zbN}#K>FJ}Zg;UUCS|`9FZj4*5^mj_b>%rGA)c`FNnr?T(UbxvaI)p?JIYW{$J-3zP zS*&@)V4{2d^M8xLz~)n=EvCRYqYiaK9UtRAihM8VMJ14t4g5qgB_yWx(FIZe$35*x zRO-)~0b>rUDLe!Vc2ug8+z)s6xVq~o#D*~84?1qFb#Q{!YFV?#WI9l-tM+f@g?>XJKB^HN zL`5?cP2`_dUj&Kve~o0%PZYXuDXHgU(Nw9iXuwABY(UKkwy0>@fx-UC6;e{yW@+>@ zf{I2tdQ>UAp5=Q12o)z;G(if!=v6S zDh-G7!8I7_CkGl$M6fdJvqx|eb||*mT9&{g4sR@u-Qo%hyu0~dw{n#x{b?UhN4kRK z)e#SR^xV+B@}<$F(Bzf15j^E{Axo>vd-o7?^7>H;!a1e`p+5SF@Vi7onMBRtghqu@ zOc(U(>4>*fB2^DYpK0>!3*px~fF!l~p*oNv`!rO+39(!n?;v^P^T|jl!OFEH(A4e? ztII_=1`TgDwl1gWNf|N9wZgk6!+^b}ie(9{X*&Op-(Bq)hgjL>cUd~aM}I?}UO!}y zX^YRNc(oMzvGO908@1~7%2Zlkdb-SFb@Yy@soQ$Ja<&%YSelKSU9Uwe+AE6Dm9odO zYXj9;Jv)-v{^MOv@nj#KWY!fmU6}8eXMx)9Kje$W;@A(JJ%PuH)pwfs5FOh zVOe%o1@8TxzfM;IjO6-$iYnU*tnoF9tC1X5njI{gVs{9E2L3@E2d4d-5_*=38Pb06|Enq!kuP}pQr z23u*-pPC?n-YXHOE$?tz06y}Rs%(^5T(r-TGAu98F{3Px8V+udBZ=+(%+lR!bx5w4 z6$DmxYCIW+tfd$`;G%(iuqmNX+un($w<3miIW$B_l}L!F2OVW>F$kFkNg$#5&`rFiY?ej7=K!8b~F*3&cCCs3mA98s@+Kms0qaGxy8Ko&u{*bQwAs9Jv=XVaZ{Vm z@7Li0<5?*hA24cQDz8OFx$!wGg85_Oj_OA-rAyUsTXPzm%FI{E5f((fZ+mtuPED~@ z1kY?%{qMf-GN}y=h@9Jy|LYbm&`&3k`e+8pPnZ1brZm z8cOK2yPZbyQn)Qy^-vpa9xrL;J^7|tXpok2$msk~I_hr|Y?ps8N1ng{ee|IEOm;!_ z5)yfL`cXD7aeH|8UnPHvY$$&5aL1~`pUaN0;NOr^Qc_dI_$Zc}T5iP0R%DyEg3 z3W}6^Ov-Q)Nm1>i7}$fqY5OgRk+E}6;Z{DlZqDv{IoY%Ipy|uO&CU(SW{pU+%-~GE z8hzOfwv3>A7hVj#<9 zD~t0)MDE*^m422>pi)AL{Kn)vS5it53ms~N=u;v7q$rgy+l6w=NZ_^Cl#)>mz=<%iR#_f0}1bc~(DRjl%) z9aVHSpn0zn22h($O-)fvRZJ|_1i#R0so9fk@Yam}dsh*hh`zxew-aHj1f*G%#=;fN zb|WqlXY3w?l*-iIhhy113d!MprX>x_L9%jo-tsF^*S#iil2EsFR^16bAHl%FLLpPDHJg7|2zj>GSyV% zw<4OsNM&n)qA>SmHUktvG!?{M-xtedek1RI&L!U-F<%SSt#I^bGNlvJ*5lZ7x*dbw z%MLfakq2QYGG`qm1ywC<;hh@e=am`)Lbl! z`v?A&149mf+*`*6cf?W7#C)#QsS{bz!iM4;2|U?c?fZ)LFQucQ2C=1$3tug{BIy3O zu;b;?($KA2EhGW1L*bYQ(Mzef?)_czvGF` zr2QuI1X>70>AD3zU3AE$7E)#ItTw-jN$UP!iHNM;eQOxa6nezGvFI*`si7`=brF(x0cHhD*RfO@hNyfSRBO$xw;IUqOIb z84_7 z%~+CIWJ+s*N-R4US5>$!f<8o1RWSht0LCc;Usz2oK@mnPDOLoImX<_IJc%D)yD>p5 zq__F43{)U?a-~{nek^mNX&P$eo-ArY5jc22@|iE2Br^_VEtF(BA&N9iT_#OUPBzMl zIv7NuvPhF?$)|2#^4w@5Av~obAtglsB%GsI#=(@b&P8eeBZS7fc*>u)#)=ubHA zXKK#l5#W9V=?4Vj_jG|KI9f@tG_+;7!bzs&YI#zUV5&he^kkzXs!=M!Lv^x}>OdE8 zVb(f&G-dRRNCD?zJ)AWLv|*KrZDez1zct&|qC4T8QltLuN$NOWmZOoazI2j9YSVcg zy5BjZAtolNGs7h4Y9BcQ=|_jOgO0a3Dpqj{ZV+2iOmJ|x+uu|;@uLiPnW_rpB5($O z-+4bieV*4QJ4)X&TxyR@`4fGR_hbIhRM0j$Ifj1cb6u8}%&+*BcSop$LZg@;vuE#d zmQDx$b0cB|WHA^YRTBZoXN1*Ng-FOphw7vy5x1=nVyB%xJZS;JY0v(NW0`IK>dE?( zIgGH_VAX5l%;YL=CJZ+M8xjX6RYh*%?Ai((L<)YXL3E_e-X>gmw;5X0qVbB7;z^cf zb-(g_u035biE?zxEl3e@R|}!3OPiRRCE#!kSjx(j^|gCrwlt=BNDw~HVGDSNOeW)` z*n@=f3iIh-Z5SMjKSeMN^-DsmV2&B}ug;On;>w4KMR#}6qr(AtcYoKA0Wlglx*92E zJ;qgda_yUabUpJ=(MuA?q2;H1r1eOqAEq=^7435 zoj#w}Cbk`dJ{dDOxzP71L{!vPiJ7$L)0WBeqvwu?(OU#?ScG4{B{U}=I2IFEfzdU7o$J%`}7~ZZFLGM9M=k30EE~jrbW#A|90opXuDEK-- zLo4V?SlbaWOEhXj6lj+MJseBtA1on*6t@}z?Are)Uaqan5ia%9YI=Rn|cso<`4 z{Ks{_74c(pq|0t|!tI{4Z{Cl;tSzdUn0Z`dwR5F`pd5(uW|vXYv@rXv%lcKH2VesL7+g!5st+xwamy#GP&9Lrx=9boc3 z%tZL5y7;!a=6w}QwZyzSS~mgpMtX2BFc>=xzRuI0@GB_1<<|vWd!03vIVXKiIb@=A zqPARx8h{K{IyEclO7&dj)UDfds>c^|eUpAB-xMA7J4Nt`Grx#Q-#_qU6Acb4ckrpa zw+Ij3XJVKeF9Pee9yBifvq8jAJ!tdKZQsUJ!~~-g5t-#qo;TpzALlQ!*U$MygGr!ed#!BX_UfAJu`;p7k%-r|* z+AOrZ$--OHG~WmCN;__e+oI&aSlK*cqb7pr&#lN_#3*^3oTDOj)IojGZkzt^A<;g! z2Q4R6ZO#ArdL%kunM5vU@v&f1$<(va15Fy(M)^#>(7fN~yE!cA{h!wrlfW}1n$wHq zH@!986!%azV*m&hrT?`7np$j(;}g7(eu`{jG#f-izLt33M{a)a7a3FSJI2xX^`>IO zv)hU+Z>6RIm&@d5#}F;Z^`J@TtjuenFNo^Q6_d^6A6@KYM>BW3x|P{Ms51DXVj5N( zKEv)tpJRE>zJi>6&=cx&y3RMS%&;&sLrq(FPho?t5A!o@#1zOr3mZejNc13`8GN{U zlnK#tI)hq5 zt*|U(<%S-5bZyo9Ogl~bi-?L2ogAX6sLi?Qc%$_ZP{MoU_V!^$`SVDu3Py&&Z?+B3 z&?B{ln%0i)zC~=~L?|dPTpm|mC(Q6V7c z2>7L{h@vXGUs?il%ZP8%dx4{fA9|7I+1oygyo7ALcRen=AHQ$LD`f19cio@Z;Ms`wU+H~*87I57SrSz!?Ri|p z=5z+Bb#}jjW@IO2S$!SjQbS<}ip-=+qvn6P9xEGfI7Kt7=})vKLL|T&Njq2S_Udj!ey&g zhti_J(Tt5F)Nrv+zis!$W@cmpd5wR^#}-i~B^pjw=5|k-9X?kF8*f}|ES{PZW<)x> zuP)Y~KKe)6`NE%g;VRbFhMZLE4By5EWZPpKfPv|+;#hV3=KfZyfn6~KY$wFmu3povYy?b-|Q7Y{I zz9{4uU>DsYs;G)CT?kgS^jTKr3w>j~O-Xdt8%2eFYm9PEOJ(}U*SiRXF=ds%JlHU@ zq>;5_k*3vQ#X?c&bLWD9FE}>3^1!BrmPjWmCbl2iXc=|p8Kg1sQ0s9+j2I?eY8-<% zk6dT>Fp4iI*xTPRSv$jNyIkl0s}&qBvXX(FYhH<(eG0TDZ)y|)xKFdZ9tmBev$h_W z;8g*4o9QKe1*cJpKqkysv9gFqInW=$F%CsWav+Qpta z(^px&#oi;9|26si|Ah}?{3Pw8+vNcZE$(XX+Bz`S=b+ES`g^K%u3;g>gOrwrb87uR z@dy9;VTk_}hrc(p9xJ6B;}BbWbe_WHv+Kr=_j5Y3eXcbbmz}OO4XFloNluLsfeuzv zzVb#}%I{!I-ekSevY>HM%VT|UgByC488%fQdJABqJyt+H$l-C-=av&D?*9hdLU!g$ zWplTUNx6y=ZttYHnMSS3fB8tP|M;VL)|#Bkf_O00@szl_L0epamx0+sEPTmN7(lk` z7IAy|uQ8o%3SCJ5Vm!$!DQRF>9$ThRb3Mn|>M@BVTgf*3<%76TZ6Mt8LA#)%Bk0c` z@WNpAvl1ZIn^+uy!qm1tEt5Y5J+vBkvZJXn3h^fnkhHKV(GC(mNHzj@>R zdcv_ukL-6%g6VtPwKJcK>es?L+T#K965Lq}SGc}@VrkWQP}YDEqz)98P8j`&$_N`| z6gz9Cl+xy&redzZ{C23p{UV#H!Qp~j+nhf%+s7Thb9U>>5fiB4LVp<2O9CqelMES$ z!YBn36G%>OsE=AiHR0s24Jpxir#8{0xbNku!7{rc&O;)Ut5~E;8*^zKgu)0*PNJ)H zs@)qI-Ze7H=jP;Vj74Rz{>5}N65&1p7Dc#4Riadte5+2mR5q8GYBJNjj(4E1Pb%wa z8EZ+8@w%+_u3T$&PLza1x8FbswQ9Rqy#kPXbBNx~bBOp}`}N+}oz>UFld?A;PQR)miOR; zW>&Kg_fyI+8VMG*mbtY!alq^Xf!COfFsiU5c%X-rr0{ogauV1=qgdryN$>(-dcflMzh1=~Z143N!=QhX`L9@j3hIfvgFbc-26fyxO#JPB{F@h!>ai+D2$eV{MhGTII8?b*2tZ%0Vt3k@ z3^*i=6bx9{Y+^XTO1Y|eG2kwRsIW1L^q4L5B!7|K)6UEUU>%m6;&?<1m5~lpCL!!& zSU)K&`NueBCRo^NE~kh=q9v)y)|65eKKCQ^+5IBRW1jKz055LA+8;&p_F@2lJ?$~t zL093%vd*_PIaQ*l`g5ym5<1^9(hQv^T}R{=Kp=I>Ef=%N(*q~Jd)Q8wg1)fdzPQs8 zIyYMAaYDj96eO_86oE3Rto#F4M?A-WJS`alH89e(4{dF?Eegd^C37?-S~Q7aZ^HUf zl|@8))LzyAz5y1bAX;3Ey%e#Sr$%+*?HF3SqdkAGInj=yLkt(w7nlqG)* z2VlEelYVBYy`>uq+Ox!LzU_%^AQkbTEU+?YT* z00YcSTfj;RKaf{+0pr)BKeB$C4V_-QiQ z?w|)cX?jAT%5w|y`QgHRecIsZoA1sqm%uLm;wYjVucb_Sn;qJ!Z9BylW0sZmEB%R# zN8|f5-l4HNO6_n8c{%x;cYgJ+p(ELe-8=KeO3n8!ez^+YUZ~ESO<#G!!_6ap4D~Ma6inZSrButZIXH$$xrBpb z>cqC@rvu;AN}qAvezlkP(e8cZOaD^Kx82a447~KB+|tyRaI~>;u)CMLgi-cn>4vI~ z$mrKqj+dAgk%ofq(omZH@2odgW8)LbsXG^v{*C}uugO(rIDA<@E@M`0-LFJlg**rj zIpHNa?esudSQJ!thEl8u^o3OGbR)-GUp!JAL;t$DZt5B)dIP~%_|AAe1@Uo$7U>g@ z$(=BA`rO7@IFy3+#g89h69E85W#uO>MWjvBMqL^z)&~a9Tcf&Q<$Odc2#$ASlOI^L z1$#Y~n>}*Eg&$+fRN_@fD2_)daMPA8PIS8Y{H~I*ZU17r>uXSFqwdNbHWUBVhrOMd z3eQ0(-lq^#BWEJ70XhUb!ms))0cqxk7q+{4Vh`vJ1P7 zw7{p2njTAw!O1bG(qg4%Lg2b*<}_c}FO1&r>W8vYF5HW_(aKO;GYAR?kj!A@wz8-;BaN3?VgsOh7#d+;GvdSU#y^1snauEl#kEn9XMQAZb5s^W|9`b_rRi|-P5?H zCZF|r*p!o{xhm)t8i~!LS9NIO~q+;jzf`ter8tLw2D=%5In&> z($e+w^#^m@>VwRSl>xc(WM(%bKERh&eWRhaF@=!v)e6^Qs|}#^m{O^AK`u&(BLKS| zMb-A5{AD*0zxWFD8!VB}C}~wm-7mqHt__bY;~lpe*I}#RXTg4~t|vIRs*6Hh1PxUl zr+pYKrhA81KHII+i}msRHoF~uI=rXOna>aHHrg87)px3tWw*EVXXPYj;_SaEJ=`X1$z;kCl7*RJ1#0SlWPI0`OhVPi1z z^%(+R&?9(Bjh`LUBN8y@6`p_axDs*?{xZ$fW%=!|7xFzaMv=N zUvRYVk%yR^#M@xzhyEYF^rhMd?u(7ZaRa~rKtc7bAV+~Af!%suhMJ!;XiL8EJ^@3T zuiVa*(UE$r;+crd+}{=rTK~h!JU9#@la-@4n^Q!CXQVHv+*vtY6&%3(qthIg{d1(9 zT(^ItW@>6$T4DzQ(Zy6-y~5D-fJQze{cBBQ(1J9v5NaICeh{EyRH#Owc|G3xeKIB} zJs-hbAS(3rce`^tGE()1h4>XSw{C%ziUUtxpeiBbL$-IbBl{r});<`-k5K}R3g}cx z;S?-jI=Zoefil9)q%_lW^)bIDsFpl7!MPZ$cEVtz2tGr|(MbvSoKOI!kd)-{(TY1i zSJx$dwD*%zlK1Ty{i$xxvMV>R&&A?$n;Aid8;6H#c}o(j(CY9vjb86vh{E4}xy1}t zN4!e3Bif&B;qml3X|_3K=TnHTKP@6TR*_J#=I@_oVU!PIcUUUT_xkP&sI6vlFkx_Z z*Sd2DR9ErHd3S7M!Fp~??; z2+kaFLOrHl;tFwi&+c=z9o19wjX~!V;>#fmzvG$MJgqV?&b|Q%4*QGc02Jata&$O? ziY3;enQX5x_-}YizY2oLg;J6O1214n`~ydYCBJ{z0zg6l^b`127@(ZsHONW8P>BKi zsM(?DH@f%`K7eHoK| zkZh)V(3P><&B)e`jXtZ;---AK%76!ndqV#?% zXf(ffyuB4mYHGaJ-Z_}8&3RY1U29E>0@wZQ5am=nKwSxjI^u#jJIBQW0kEUJ7OL{X zWSQDsyYp8pDs%PvX8%#j$?W~vC2+d+RViR5kbsK&OZoE0=h#318Iyq@z`)6KS67Fx z6GuJn>7pH*>teE>`+Y}XH{xRguWrnb2p0!_RazfWclV}w%a5}1552qlM<6oyHrKmklp4GI!n;M;^yk97dY2R&ONt<+cBA85sc zsZ|ZJfslZW0~k#1YN*1GS%Kv36a6!%+>U8UjdnW|P5LsS7miWA@Vg3K?%zSfmdnzB z;SN|PTBy@gBXRzOVaNoVqGyyz32Vw>$dB1EFi8ah})i}=)M@Qjroi5_TW6oar zIO$@S*IBHouQwagAgZ=7`vyA|_e=*Ch>AFoxgfFuX30$tg+B@}l*W`6w&6i8o6XVD ze% aa4A9XaDW|LTn2&6BuBQp%6o&0#;IlWRF6ApIZC;DP8nsafsRDxZj(AEjB`R znih7J@9mEFQz4p7vk$}jqnmH|``N=AKhZ1J!DIG1-&lJnfx*_hw^7>06V*Gv4A(nm z*ZawP#>VT$I}X+QyOWLHI@Hc!^F^l@cm2Lbi`kjgzFao{$Pi%ewAT)m<3g*~|>ceq}^So=-UUP74ttJiw(dGyAhxuDX{xLZYji@Ndk zswMBdo|5;wKD4p>scF<-fU)o@ceDs z)%IoFAio%0A7ghLFbxku0#@n4C0R@V7j<75Raeuj2|&uG!=m^99 z9L<6I_7z!q%Q+$4v3XACoy$4jCG~mR#c0BY13$cGxHcEN@WHgOCOQmAdcfg=eu2SK zH`X2%a$aTT>Ft~gx~DW*;}?QA+gUIUrgL1)H|Ugi?db4bg=}o*e#X8WKzvgM67R3X zboG*VkdySQ&g*uiCcX%anszde@yrwtm3xlJ8-00e#QGZO8Ea2q8{8^+Y#85RV5^c*K7s69zh|Pxj-3j=+bhPK zOIF&aP0(g*l-FiIh|w#)ivNq2>VpDkVL2GK4-V$6%|9Za;PZ%ENA#YBRmE3tvS8;Y zysNi*0<-0P{ml0~$*DgIy&R6cbn~i|LC~D(TVE29KlIzu#^lD)cK}=P4P}p4b<)m; z@!=xHkA*z<7fK}+hR7!jtwpz$6)wA1U4b{-uI|PM+d_yG9&V|fXHJK4e4BmVJiWb{ zk0X+GpA|T~pEXLJxYnzGRu-JE$S`}2L57l|QBPrd%N2gtps2+2)28)_8Euez5acH% z-t&W-4&z1y4%V6%;Z6?+)TXWrdOCCJ?`6K|wmLV2MBa}+X{NS0RSkc}QaduJZflI+ zc<`jP`{n;GW}sQj5A&Claird(RR+bDymJ+t)%gW}kW_(S0Lh$ROpHOlgi^5Met}W{ zC1v1W1_z4#HKq-8N&LV-45tR&0+^kwcjH%jrh)tjI30WRY-1UlOKOd*Cg{0BXdo{I zK(e_U1thevzl44UVu-l52w$ z9L7(uTm`DBHUnDQHQ%Jqf=^IsK1Zspei3b*!||-5t52Pr(<(>}dn;}aNupj`v=851 zN23s&5^#BWg|$9NgE`v$QmfjB948Q~3!9>KVtlgJu=RCBNOBjafeOCF50O>}mm|>2 zJf4SNv{B=D<~tvfH9i;x`$R$E=TWxm!hK`pWE(z#t(HwS&gRI&7=SMzAp{8^NU4M9 z_V$nIY$^f=1@r4yAjYu5UnEx&+J$U~gP)OmOIZ$d7(6=%YJ+=Fs!q2D7v={|b&4ryvRPNaRocIZj{iO^;a{ zJP^18yJ$8fjOxxUI^VLoz;|1SqBW=M-i#6)q#qOn@;`V-xo)(_&Og%(%B$h(attk7 zCfi+9Qo5Pvr>?iM^4&xYavQLMwn>Ml7ex!nZU#nzvVHIxlWW3gpp|vjn~>P!KtSN> z%=0nW)<8v%f2YXh@=r9Ht@}rm6Gvi+Ot2jQ4g;+cv}$7fU1J~=U+47R=01!;>D;f* zvT|}P0s?Y+dc+@Jad}&kQ?H}i->l0(dHzz?nYWk+Tq4L?YWE7~ngeGS&56&p>k|Ph zdL~9jfqm$sPnjEUwo9{0{(*joO?D^SXk;<`y{u$pk*B;9rPjn-oa=3=!TumajoJQ~ ziud8v2=rQPEULJ;IM+;rAALl6YGy`(y%`}V662;siONB7(z1G+4?TR^iJOYEOGZ4% zl(CwCApT4Fybzk$u;tx`S*79iy6el^WDaNpRKkC$*W9yYU&oK zS}L7(^RF&|l9JNc__!iNG62Lo5Z6Jh{Og_hZ@L_bqSIaMS?ok@yjUKLol*p1B{>|t zYycH3!^rb}z%8qEgn_3&Cf*gJ2?d2$(l~2M%OdA+txiqd`q4C{NMPspb2=loj3dmf zt@L}gG0U}KLR>EPHe(gV8E0#r3S|-MXX6tN)mg@b`o!+fmnIGmsrE3N>{piMq|PF6 zh_oe@T7*{oeOl8@arvE6@`?tJa66;fM?H=-nlp^~#}zX3R4ojTN->>gG0o1k`U{QW z7hgt@#3dwf2?>p_@CC-w&OZT)$7Pj{ag3IxeefdwlUk4^%AyaOEYHjM)PIRc9hf>3 z!tGWWqII$O&D1*#w8~97Zs=P~aoFHc)wxdCMC%pTdY!a2JI|nv1pP`l#M7K8&HB`) zyEF_Mv_y<--4(E@rkMG^mmmM;^Nop#iJ6rZ`F|(x|J~RB6tCP$%TUaMJihB$7Z8QH zJ+}GaH*S5_TisFj0!3Q<`f2BDnG8?Y>te|Ail`pw4BU#d9ZHqU5a~3TzsYMs0CH3f z81CqFJCoLvNX^fUXBzsHAEK*Gof4*|6lvg_t{d;xKvREp@G?$Zj+n1|Yf6!#G+@>1 zfT9&|8ArP#A^I6@uMs@SIho7;5k*2=BK4>}!lH0gduJz)uOs8N&Azc!9h z&SZ%$9TIYAUXsVbDO+O5JerI!sg49YyI-2H{G#6QNr1EG4t%oX__QkU zd~N)FN>O}bf~DRC{scOWIyrr4RXEGtfKnocC~wsj-PZdAdO0Fj-NwjlPkcmaaH+wQ zIB!sHs`?Rc{U+(e+>+{7V8V(PuT~^gou;hkn#|L6&JmF4Irnka(-xl*8ccwi$jci;S{=&7 z_xMlMP%~2ROh>3Ire^rhy@VFGUFzs2>%D0F2+#9Nd#sse%zS69l{69N#*UTEozz@XRfKOkfs z-SVAxG(>H33dO>42165@wcYDy;Sb3i)d~fxBjY&*!Kgij5|M*$IA~AJFqKksef=dk z53OU#PjlQR(q{F{NEbiddI~{N<3VsDHud)@~_tO!B)F?MOi&|oICkd#NzwBq?So<{bkwKu3{-0!Lk4QPrv_u&A-=?|At1AOs0%G`#K^!jBF^CUH;HbV2qF z101USZF%{@9jyIt`H}u>KiV!UTCdynSgB>PuAe4UEzqIj}wgl92wfb** zJZ&OG=HZe*A-E`sL4|~AXvK@;>lt1IL3@D(x5h%*swwRLFpnJqv%;us)_4)Z6(Zld z4+hGRDcQf`M(CoTeuG36{`UX!k2V9gM+Z@&R8Z&_(oh&u|Njqves1jaU`z#4)Sw>D z`VUxwuiF}D0o#{(k1V5LW>l8c&H!1l{<9w=vaC5ghXN324iU-_NFj?}eaQaU?&Cx*r1a2E6!40%9NM6~VW(NYQL{Xqbf?}vQh+sTOe`2tjDDCb1^q~5 z=ugN?uAt(2N`r3QZW5^@!?B}6Fs6&7>V;!JeEu)TbjSq*+@twV+8(4 zHOt9vP4y`^*Dr4Wo_x}Q^ao9sC^AVh$ic=Ojr?%Ywg)>Z!ECsuurWUfb97Vjul|s) zZV)uRJJXUdoFt^7ppSzHWCu`C)8I7x|0zp?RwnwR{-cbs5nqWb}6iqF$>kX9^MjqM%?!1buL*sHmK-G{wke@?2@r(9-5L zHe%&Xg?@ti(CmEvt}2@@X(Q=c+1S*Vk1D5mzvXR&|Jm_P#XD>^cT=zL;z3D61E8*s z1BdZSJ6oqeMoS=tGs2$S*k3NvQ>9#!93znS(|Ym^!Q6bCyvUGNFB)H7A^WA7ffWww z<6_l@)20c(ZYlB;6zKgbP>kFcE-st~&8WZssUCgRbaZ5ul`$yd!{8g-u70plaRDk+ zS?xB3K(JwtdZui6uyAl9J@S7A5j0#||2d0(i-zAqJ*j_nr=Pzs zsw;Im$m!(Ukp%1ujwi}HaEhh22Fxrm6+@w7yjuRFwEm6-V4HVfM{hYeVf4e9pE9uF zV1ld-m7MC)`MB&_{dydYpysUEP?9oIb4wXdiE4O!hB*))71Tl(`afD4G!(e^g|NQ@ z@Xxs5-Z#QvJ`gnO%@lL=x_q>YZB8e-Qkyd@e6Xh!PI~wc&H{$q$Sn z89t!r=RbsR)(3&&Wlx026^Z_<3+_xgP@3akBLBmd1DCe2n3zE$7W-T*M%_qg(A^9z zSy|BZA2<2u4GgM(5eoG5zb-zuK=@Su0`2obM1+%bbsHel=J4Q#mHo|B+FkK|>UP@G z?QjI4cif}YG5LB`)X?1f0$zKA9u_mo!lCH;knE=U`L4z(!tW@i;b7}1Z1u4_FS5On zl_F7S+(oy=8e=O4xI5p-gEWrUAH`VlrDQ4F?Ko3QlB2l$Zz@8zNlg>u!$xyWDe1ps z0qmKf$~rfr76$6}?f9=!Q?;*(LRu;u39BC3{3e{JU$2Pe?M*i*%pi{)u^cu6Mw|D~ zUXIWW!C#fR9WeUhlWo&8A%w*wPO>*N*-h7a;VkOC|7E0YPv) zC)Me2(hbWlG0$`#R^ixq$&yFrq%`$`o>$1Uqk8lgscEw}7bd z&>M~SZA-&y)c-H&^_PM@^2G{v43*2@!$)aC;gQ9Pw)~67hY`AR$s{Oc_1$Y?>QEc9 zIQUTIec{I6qEn_|c}C8&ULz3D&kwr=Q%_P;&vQiMnS|7%~wc$AfW=#7fq_p+?n+p~+k2DC zxL$2Pi0^cL^rJ)f;%IFP&5Pqt&54;GLQ{3`g`F&k!qR5kXcues8KfKcr(POA4_9C1 zwTfT?)5nAPq1eJnPQrzq%t23GKz%yZy8o&)zk8Dh)4IqAs=3amsfXyPI%fKw1}6H! zkwP9H9d8kS&+@VI719-7WP_u|EYR`VvrPSZuC3hF&Y!!SK6!&B6cVC2rgCeJG5MN| z5R_2MYNu~nuGy_IFe0|0NU#*&1Ih|3e$@SW?7udq)tNr;eM(*&&2*A?L;#6^+t<9X z{x7XrrBIX2*qJKr13;w24bWmfMHU6HT%#MW6tSF1!Yv?M#bm{=5@@z1a7%M20`2$J zpS4*+D`m+Pu2$^#Y_}uKN2y92roI&3pj4i$z-4Ma#%xg=zm;DUTe7E=?;HG`nq9G7 zS*y)}Qb~dWLYZy{fJf=ypYNM{s->53u8T=t@}6M4!CM z?o-Jc+*ZMt9BWQdWyT7;fO^F^*X^x8I-!OTA@guNJL8A>yXLTi6dfRly^yTW=AF#9 z{Q&Tuq%yEO(MjGEhwcOoFWfVg0aVkKVKBv>_X2yH9#LrB}%s-kB;b zsr5lk#w~tewKl7=fRcT+*`|Xv0b%6^FVS9j=AqWM2LoCkLvzV@243P zmhRr7KRe(4!5zP+u<|x1SSfwtEHnK0z2sHr#%Y5Gg#?ATOJnQRCa%*LJTC`o!SIT z%?VNcB)3OXQGp7G%KJ>l({3j8+M}GTFM4 zt^dHY+ceD-c~ZoRM?2Pe_8U?7awfXhC4JYNz~%cIlJa&|6!~;W=8|+3upDQl+rEV? zv#KpVxc(PN-CMIaaN&ICOibc2Hd_?mq}oZInZ9kQX60pup)xp(k#9VK9x7)#&-tLn zZQhY?QE3$UlYoP2^ubQzi{goFaiAlO<^##Y`<@HON!ve|?PvA!SJjW17k16vO3guG zH>!qK%lK_;a;hed4kqgpJBFr;-%lW1q0*Bk^tbNw9zVT)v$jXNIRG9A0SI{<*>Laa z(D>R?TAsfAT{bTD@}5yx3w@%xTK(Czx4Dp3x-%G9edCI6!njPgQ>(G+b<}>CbrR8Z>iGv+9$@D+kbDC7B(K

f+uX^r#kEu#j{tdKA`6xiDcnlouQV(vh#`5;8H(CkFd)i$&o1H5DvEjAu=1viu zv{vnE+n>9aef`RbfB9stTo(}|-X-NDgM50g`wc_Z=$33l@y|p1U3vDMfL;Y{zPCIt zE>r(^OpuCAgE(Va%*?%J5 zV>_x+8^bB_k(oAD$Ip0wTq-!1+sdy4ytTlft!Xheb5?-kawjE6DDy;AV5LF!Gni>N! z%jOM_r&4jS1#_pwW3ys~SxmR4e#DdC?^PMC^{gYST8+u=cRGA`*%t<1o%Q)N@lwkb>n*Z{>lD7^Q9}>zL=rV6 z(5~o-T2#^Eth;~B_I<=m?zHE$_qbF!xZE?n0RZu2({~k=Uq5vvx3lYA-r`FiJxtp?+1`WwgJW zh0QK;Sq*9;HR;y-;V)zvtdKPGH|G(oL(hX+ff(z9>Vkp>(Kh3D2jfdKic0V}qBoMC#=(;g}<+Aaj*YxNKlNj|<{Ym=Bjq!!cam;9eKs z{k<2p%gvp#M_4^(B|V>5!&Ldp%TOSWU$bIJ%S&v{*y&{835%gZu29bh93mW0j5w*= zcwu7%7(#g?EXnlnen%Rt;|SkC%&^%p5gzs(HsQ?3y(UWQV>aCmC@w4V(y2Xf2|EEQ zJp418ButzASPEE1oq8-y@On> zbEn_u3%>~pzQLO^j~Tjz9WSm?c(XZ@h(HAhI3Lz0+JWl)R>leBvv3v445@FA!cRn< zDK%Kxfm0cP9Q_}4y{Rp+tPLi%SeOE*TODm#rh64{I!gIK7<{7c4d~TV#zOL% z{UKl2t}$#3S5IX#CduPv^nS*F65wo0lF#kV05nBYr_BCrip!)`tC^V$FKgx4vU=6U zZc~jk6(yon=S4O#^MFL;!+c3?GZU;C-=SXZ3eW`PPIOl;I)TK0K6Ww}drxbjj=d@{ z;a(9Y%Hn3B#y|px-&1{PzJ_%b^Peu>&D?Q=|x2;+NhD1@s?LU)buidZifu zvgh|Dc1pBT%o2vr@y*i9c3vbKq1Fa!V#bFt0Qj zyiPPCrWvpdD^z+As1IYlf=~k=En*~Tng>8=twi7~GNYo(*;!?c3`~Qn+|1Q@UVyg) z%u_79>o2CFjLNda*-zhXNVsEhZ*8&k=fc^|(h$FtwP3=4`P;dmEe9!9xqydr(Paqd zA&TeiNj!h)Md&d*;r=XW{JBn}rZR49)Dzu$a`MY&mGRE$QuB|tnXh|H#9j?i;K)9b zFK!TmB>tCVdUxRaGP>f-QE7L$!Z8W@c#vL1E{xe@H91b@1tE*NTjUKZ{kLXlkxewo z+)7DGA_0=mkb$@yR(Ph2Iih@k~=Gu`N+FNF~Jz z;B>K+&IL+p$!_t3My>F$w=Yt5ve*jznQhpX%<8%(j0dT)>^OXhqz=~A6u1|!s((C* zcek~6YL~EuES=*SLzIsTpK>$)J)FA|fL(d92HFgBk!%*-Zi*){tN<`U1=1#&Pgpd? zj?6YI$4vW18qBm@x;s2DUjpdVAvVLU2UT>^?Y?_IfveIjN_$NZ{h6~hypPswy>AUQ zDvM|(Wz2%3)Ww6r>p*QP!;4vvch(?kdw4!ttb!8ekqebMll(D6o@B+adA;c{Z?rJ^ zG$NN`PEKNdl&0CgEZas!xSCMvyrGD<79Pn)ONm2YFJmdE--8rA%M-?t&Q05u-hM_q z#DH9+x6l}fqnx3TI+`G+Wxo9Oy}Yf6)y9@8CXz5z_7N6!OF5A{BDLclMGkqef6~#8 zF-;uO-hZ@bTjS(Q1$(Q1#V%G#%i*zf7$+EL#xi-*-+?rf^aV+WBqHZjah$W>#E;|| zM1vA-2Sp0zp5B@Ye2s=Bz{H6MdPU6EbgcX*pKH~sU8nO`tE4lwIA<*$aXhHOM#{BY z+|T!fxt)#(!hSqRRwG|gk|m|IVL-m0E;Z0@p5gb*9N*)D-yi(Ptfb~h@YR=Tom(rS zAiU~)6Z@A9+uz<+Y@k}8S`nR*&np{;nO2+mz&@hAU|njnrz~c)7j`oDv#6%Si$Y5rT-P-+rl1;*I~Q$M1&7vn`Z0;xg&L3*DXmOw_$THUwz> z1quD}WUT9Zf}|OQYo1?p^iz=OI4kzn3RLv8&Y-S;vi3#j{B;yFKj?GrCg=CwQrxys zxeH;|^yqAr#)zh*T{JABFp!ffdTehTThQ2+o^tkpZu^K*;wFZ(i?j&%lYX_gMmFBv zZcG^|fg7B7u+i2?$p$vb4!@C!y#;>Y>QT)S_OhcFG1BKAs*vat$mSd`G<+B#=q*ex_TgJ(+RCGy3WC+M<=}szlPwetfoL&blbX0jL1dWv`gh5`}>#gxE~?BsTFg@G*Jz;~5>%E_7GV)) zgR}L85S`Yc%jEWJ_YWA$s14hFS2MJ=*wVJ3MXGhKOTRxFufb8t5EM)_4pPRUBM&)I z1bKxnGFV*&$Ua;cAG%ng387RVxqwO|t+s*(L6k&0@fCNV3be1>8%)o+p14oDo>mN( zXf}u>wnq}}uTL`9v>j}A*yr0bP1(j*sDd2nn5OGLbUk0CkvzSG=y{8L$EEWmZldgxz|&Bpam_FJBJtd0EM;Lyx1>+2Jp_>54xE(@ zVk<6V_;bSi1987eQ3=>tibh(F1?<;rje#ZZ8uHK3l)Fx{H9W=7u0FM+7sKz``B@2@ zN{BXfeUtGy`DWX3qI$*j?-YQi*()>6+Pf)eaRcjG=x?Q#zgq08Vi9bDwshm_@nFX= zz1fantSuai95?C9X3a|0%Np|Ik>g^zFkUBa!p;QAvq_%Y8JraO!j^NS`-5x}VflgA zIfe4y` ztzN9Q0ykLFF4$Y2GmKlto=*Wp?}NOj@d{S+`Xi0wn{UY`3PUk)wMEP($9&;B79))l zyLZR~HFcASi-*3}6i@g`wR@6?x@y#AYSt7L60T|A4oj$4`#M{hdPzklFFUHumT~;K z;ygpQ{UHdb?9uTpwjN(2u$$1#E`xyP|g> z+~=rcS=#>++g?AihuX(J;X9?WKfrS^|y)^b{Q{kkB|X^%K?#;e(qkPHj~c)Y2mu4t~M;qBbPY%y1` zBX>#^AFg(9Zm2SBFvy3sqEwxRDA_Ho!b*H{;gKE0Yg6cM0)yCI z`sMrb{ob)bChQbU$8=t?$GN+J_GH>YrG>oFjLWU%GHRZ%M7NT(zbWNI=NsY6$7-_4 z_dk&fgSB%@@ij4PLhXtWkM{GeQ%e2FPFR}Gr(qVJRC*Wkxr6-yX4nYqs7=4Pj}(&_ zmc&<1cnutq>kN00dNps0bP~d{r%`5{xnkBkQWY<}dn?qcHd_Thf>1TALlLH@y7gMq z7WC3Xo*ECDz3rqC@On<1cWz2@kuEI&k>>-6YKSDjoPpR)(-*;kI)jO?_4VS_KJ(b+ z47L*;d6l^u`AG6j-)Jtp{3Qk-cFQcwWu7h#lOT0^Zrei8%hmjvkj~!S`5Kb+hUJ%9 zZQZOjp?p2waBJgair?O*4mN}-t(OCGMLp3=Q8B2vf;i&FbgyJQ z3~1VHPsLjmgD$#1t?OtGH89rqR~s`FmC{FFKQcDU4FFTubZoIk(qD6BA5MZ$Rk2j`^Ro4`=Af)l?ol@^tN7K*dt(dF({t^%- z`j=Npt<15#zSDYk^GlOr4VF#C=zgc2Dp5vCdYrU$IJey>u~UM@c$~ao_L(kyZyoIO zt{a-~<(hmtJ8n<+lx|ic!~$K+Y0!A`ko=!@#Jkk_{oVoP&(~7-5*%BNxoVRPQetgp zl)OZ#eT5`%lFY?V_Ys__T8YE7dBO1*g-~4~y2E-bhp`6$bDkvBGddwKo#w>KWF276 ze6E6H;7Ehxqs^;UM;E8R!tj#iEyZ0!V3|{v}HdodbF|n_tX8hn0UZ1^gB*6T( z%L|5ZM|P9e1jw3QtLe#%eoiD_tsB&2OYp+z9V)L`zdhB{+l-8*#B+;SNt6%`eK|Y- z`&0g!N@O|{45$>^fv92FUT{~(pI;eYBfXx?p(i4LOd`56my7bn&g(0KTz0lQ58riE z0C(p{tCzD)hiOim@|-b+o;Pk|8NG)28dlH&A-9wnn&m25?uUCiPVFA#eT+8IZp&M% zPK`uf0hjW`{HX#oc zQ|-JqNaMuKM68GzkWJ@6OfqQHQtVZG_Oa#@eu|1#zcx>G)b>+#zL*r%he;ml(8tcz`oN!buJ~=yc z@jE~Anf&Nu%aF2V+qs|LymBwTDQ04e_%jCZmn##_bN+q+cUh2*Y#an`Kp%W z!Z&!xaE+HWfykreU%0B{Ax}YnC@6Av4ifjv_upI#4HvyAhWr;l0XFmsWWVP>^XJsshP@*TZ z`YcmGw+n;|ACYXjDxIi;Td>%)W``xez%<}+VS|j7PqE)ZE+G=7qU5Uj_bLZ3J|n~~ zemsMvvbT7-t<0>}IGlswaeh zB84`k@vpdkSKrD9)&v3cWGNWCAVNkV$W?3&U4Ww^h z(Pmq(NM3CsEn39XGRuoyl(nK*DcJK&;h8(`S<0NXMKKb225>X351&3Q+$?T<2qk|b zd$Po?9bNHDX0!*G;#;B zj#ISplofQ;B$hfwF?dGQn?7K$*Ve=IJHpY~DAF1&gfEwW_+aXhNK??03f4oN>x`y! zMWA1oK_p-$f7d-D_1oG0Ejer^Tr3VUZ=Gni+mH%;xAjsQPjTp>1FOxqT+Mh^2MT1_ zcdfP%lB!AC*`G+_p4`!vqtk6<_U4b2S_(%A0}m!lpE%&v)>aVfJV^uNAskz2m{B@PHv^ zPG&0pQpy?9-d6^N-jAdw^mvn6L(NC-l^5yZ(K{J`U6l`ilxnkKv)e}&_f+azZt({A zJgAv_G=bud_Q=a=Y;2R?iefP^{Oi26(CF|2!IFx(dVe==-Mp00^7hvDmnk+$nhdsz zofZglH{3ZVoD0g(Pq9rloJbX^^M48z$V#}itcpG3u+i!~X}Erx_?j9uy^-0-On=iduB%q=k< zzcys++PBcO;0FF=@9hRm@k_8&9!SdBai=`YN31&DIPwW>QA7WMw(qpK;bc8IL3T4g zHVBUK0P`GG2a3^^EHaw+jmewT-rU>NBbJ&@JX@t9dc}buwiZ8G#w|=p z01!`a^txF>y0$zvlM8_4F|Pkpt~4c-JfB&Hw!X9CPTNA^2onpm!m^Dt=}z|!Qve@O z_#MSoD$v^|zYgr#jcsRpSJ}Q}AM-g8&+;G*c_He^mh3*e*|~b+&PaA!?L9K^1G5^WZxDDaca|Uc53(9@@dD|I;WFBtpn9E zG)w77wEWLCZOd`7)51>5)Pclv#e{qm<(!m4~j48ek?xD=GgQ|@oLrUx`!Lhh0%_VfC3 zQpdB|61pBCO?tl3!eLAX+XMu2L~_Q}f2J$seSecKJ8FF-WiWO$z}kNYh$%u5`DiG} z;Bg9G(p&!82Y^e=h`l&oh0cvnPedNKBM_ixdcmAV+EyX{f<(Db zfA?Bj4~kkU!W9yt&GczZ^7J-7@IBDIR_s*80sPT0Q={Kyuj(x2z&% z#!+tl;(^$fyf}R~ww4i*B4WS5T~h37&*S`!izFDO-h=q-oPf{B&ta4RMcXfIII9vg z#RpC#-yqAL#6abyJe_;$oasF+y)pJqjoPR)8K_N|f__*{j$K_Pl#_2iAG-|8m0%NLy^m)}^Ki(>LyWGA@I z>FQ7UGrHD0!@7X0@?oA-sIi)#FceiG045Bf;+6w=m7;T<#0zWF=n-kz#iRU@B*jVF z4wOX>N!X4ldc*n@GPb=K>%o@qcId(xC513ha6o|cNRFi>{teiLkFsoRAvI@x=x(;j zS#M%Txy<{kVG{P!kVic$Nd`m8txd%hWA;GdD?Sd&ATE-PW-x;n3_uaDd$8m-$td{4 z7*b2uD;u_~&F-V${8gxegpEAzvYO|&zjFqok{v1i%<*EY<^9_>DpC6Bt?{UJ_=eIip1E``naw?_oP zw$X*^ad*fV?jYc>Vn(t8ehDc-c|||G*^}{|m}lzadY_l3JL-kt3wCQmq{uiF%vtx> zv(+)3U*iH36!(o?p_Jcm*+Q*m0?Zap=vKcAE%;1j0K^o2g?(}47E>XT+jCA*(!~oL zwh|vtMG!{C32<6$MczZkF0$hoXKW2{?K~QrhRtTZKapQbQu}sDYdgiBrZ-p<*Cr8T ztS9!fGQf80EM055P=f%Z2p?ThDq^v`ZMPHa1u(3##;jx3`Xfucn>}|4jo4$p^;Z>; zvB|`&qexZJeh9h3n_hAuI~q6z1|2C{hEtXJv%zoNeAm*HweAkrCye8(_vR3xhm2CCQW~QXM$2q@Wuhz#qFf2{k)6w>20Zz7rl( z-C7xbPLIwMGG_o+lH1(dA7inYo*qNbAI6j!{b#mtmYAWSmFRaQ!9o6}K5D^~EP(0w z$=Kq@H0$AJW`7(#(()Nyj`0ET&u4t0yaMAFGi_Ts1Fg9zBTcUOpWJMptev4S?-+z8 z0PF?*hfd=#%#Nxc@Wo#W5c`!7ApAnDTN)tKWl~7!g%fw3pm(sm!1_=yP0DL;LkhRA z!4$Qb(+yPxST9#@XJ?=&<{kN05Hf*dxR*flkxwBZ?4=ydmzI`{Oikl6J7OGM!Me)A+uwMX8sEU? zbbU5U7{l*-i&{bU zAr5h=G?`GuOnxwrveWLC5pi&w-l20liO^_sfbIXP!p20|w71PsGWKW#?rzM0=Uzfp z@@FvhM_k+X&D^Hb6Bht|F2T=QtJdxQgiZ@bd_q8{+KdaHm_0}gBYA%Nz_Zc=3G2K+ z>6e^LbvRZ>fTa-bv-Vh}RIP~S ze$EeGCbsnbqGYbtxMKYckBT{)UgmVE2%SUyaZ3?^lclUfeTWY}vP|f^9Gh_?myr5= zy&DB!K56TR&gu+jXcdu13_Vn$8$`xt2{ox{4}N>0!dc%w3S3w9wXK>Mpu?Do*~95tz0bveER?eYuR ztwh&-h4~>@O}w(DS{vRdXu%00;TIyp;?!rni}H23U;O@a=_zkpq(54b$g2{G{e&)g ze;@;Lsvm~=p2uk0U1lVc+Fy~Q1@ENRbz7nOx9o%e#BF>H5xdziyqov9XDVK=k0l%r z@7qF>&B@W?h!F6OX;{HqqF0|N#I0gwq=^>}0;D{mQV3>7pTryml&n5@z%sk{{|P!g z{lRvwKOX+IoYoRdbFs!Ae#MooHhrpl9@*rwlGop7?Z*C+HKC_fp7Lh@-e%ReFVSf< zymt(lZJ~BQy=qB&rnhJ5i0c-zbU^@!E}y`ZBW>Ew)0S%Z`FbIOr*609#O>%tmW>Vn zbWXi94N(Q0G=3#^V}4-Fb-k!PmS3~<>A0h*cx))o+kyP^j?~a^H5bcu_Y)nC1<0i` zlNG7bw4eI)>9lNd@NJIwXECg2v8?6{Rli|j=4}j@7NnQHqwgl0mC$T$b!AFrU%7tY zcp_bQKhm(lSxfGX9MafP8ylgGh9Cny;*IA<@>pDYIzV2a`a%)q>tq@9_ zq|C{rzki(rcY){NROmFukAe>A`6d;ToBzib6cNSvo1)vs@A|I&=9 zV5cf-Na()MX=5Qy0{E}I7>;u6S2nVL0L;0;1NbAs_P%PBl-g48R3T?l8K1XT6(i!E zqAgw_Di!*O-2Xx!y1wD>cWsXy-gPk>e?>N_&1LB(5_Yj#6>^4WTToWador78NwFmM z3G)48#kS*;Y0 zQR57@O?fX|jtolEw{-kUuGDDj?^nk#kiK7d{E3&Jd-@`~459d+vCvGmv|^p`SyBRz zmtJ35kTu$CV^@IzxXBs*9k1}`t@jLAYU*5Xl|?C&1A`2hmV%o-TKf*YX9R~A^_!81Dp9PR3o)5~oj!a{Ga*7@_?y;QJyup4@V ze-}1FuVG4E4b@QtLxt?Vsb|z=IP~weU+uEWw41Vu0k%@uTpYGa8)ocZuP~5K2iub6 z_c|#p!&q`kEcmC!PX}|4OWu#IvzBVW+Dd|)r4P{fcR$Hl;fM5nmuPw?Y_A7b_zb15$0H2x@qG*#T2V8Y z=!dXbwTAfj@DID~c+A&##R933kSc?&*=aSJxAAEbMW}NeXyGk+na_LI1M8H$m=|KQ zCm?Ls20tq3zOavl53h&!{zlpjxt=K};z8odjKaUz%9S84l>QE*Kjqr=tFM&J;kG!8E^hIixAj{y3XD(7sjg{UmjkMI`0{4$Wggy1^5t^8JlY76syq&% z2O=q%KSW@WQ=xZ=w!@3l*8%IDP3_x`O zOR7+PPNj&Lp-af{ZPMx+M>=v>f!G@gwwr4R*P0S*XpS8Bo1!t5OdMaY&m#0Nkp3H# zh^F^;y522<(pVJ(nETt+ORr9ZZqQnsYop%Cf>HECsU}e+Q+Zt1uvGqg;seIySiv%gj1-q3H0osf75T8BS5`hOFmG8XwvGVp8 z=FDFAw}$OHmXy5o2%lCoI@|q8@)%9+q9gQvd+lf*AE!SWJ&%^VToT4prwQaiqt@Ly z+O#~|Kd;|{<<)u-$hZN8JX${b@+@akQgDHt)6m*u?y~6a=SQCxGuGj%R(ct1ETOV` z?f;k-v6!4{q=2BV7dB>ugdr!ZgA=vla6IR6^Pfl-|M*$e+!8sn6ijps%{AfQ%GB0L zs}~Dgww1s`_m zE;<}Bj4j_XDZh^!WQ%xyt|wXQTRZa&>DGQ}@jv#>^SMJi)y)=<_!g=(+DYzmZPi*L zRoKS$7>hstxRke-ILJLIk5cFUeY1~Yf>MqL!oM|MAbaa9bI9=QYmshYq^8Z>)Q(EJ zTF$8%&PQ<;Ws=&2LEy}UlW3P3819;t076UTzRI1BxGEj| znS+PvoArms^Nmp>31WKuao?UK$Ggj7EDPT(l-N2LRSVpAvS09dyP4S5&SvettMCh+FE%Utm)YZ|$%R4&qva;}jdgu2=3$-TP z^*k0%HewiBDu>E>Wk;6EHovsC0XH_YPf#k?SYbTjKD}Qq_Kqc0XMO zyPl*4e++Lu8GYF9RcC*~oZ%5{DvCGA&-Wqr4JvTdF`K3sNsUE6ore5cprA@6pg4L5>gAzCh8 zqw$uDVE(j|iB~qG3Nz*Ep7zKqVml#!^*+hGKAXyA4$b7{p4h6zDMOpGhpqhiY|P2U zrDkBDY-`IXqi$|al~uZN_w?X|KVIVr--?a4Gd)%5{wGz?Lr_1=jbhch8zWcN-5b#& z8moyV4jJNcrp*t`tHrv{yOOgid-S{E@j3{BnPPS}rhkW~-ZUT#+k+Zp4PGBR|NDSa z#;bObFr?pd`uiFxzZ-~#feN_^xm6?(IkZLc$jGV9sjZ0icon)@dyrryYdMH8$<#Lk zY>Y@DyTPj6o}@s{sZYj}Et~VYR7o+jF>iB}W|C@pp!?t+V-wZ`=9>-ohHHzHOfma@ z@kT?uN8LwHd9;Ubwezk-O-mBCdekjmiCQ3K6OB6BAq9#>Q)8U^emPflY`%p|NpB)v ziOnGvE19qQgP%M$-NO?Xeh_G*DRUwbq8)7S6`S(tI<>7xnz@_af37mK*h|hP44(Nk zfQK$&A;wN-Pou0tYYnw>FjGw+^`K2D)!J=FQK88X^?=s^UmJW%@zaltu9n{-nxJV( z$@P;@YeTFAjN%8D%9(oWItceUuQtf zRp*k6dzMLwzwpbXAQb1@)yyo+6VY)3t1=x!`Vyz8GgZ@pC&w=}5@iT+v>ks1O zO1g^?N3t2Xb{(Ei(HBA3IZsENFHX)6Z;}gh#kXHmK6AUzoD^)k(ou5%iqdY{a&LpP zTS?ne#+4hSB*9;s=3L%@Guy#TK*xz21KGFoWF5!27WO>%_4RMn)g;4Z)8sTP+p^DW zI_iLq)h1x7g_Wdkaiy`P3S3yRduHnB4(FknCPM(4PTVnwvn84Wz3xFq&LK%zi89$S zA3W`-k>x#-=Sventg+b17yx5reVL1tiIojfV$Xg6u#OL?T*cgVjuR5R`z&-=+RT=d z%$@*WAhRN*k5VlmUrN0=T&O~lx+K{!K`Le~5DqDNMdJH5;&d|l`8*@91csHyBy#6i z$k;~D2{J{S4+?wFiB-mhrH79duPLSRv#YzTP{g{%iJT=~d+&vL? z#j4ijqI;Jqyon31(~~JJzds~|3;E=HFW&DgP&>H#mPBpl{6to0bv=`hps9DmfQ1Q9 z@|cpIJaMR3X5T$wGr9P=7ZLBW{c;EGWpp@eg%31$^1!RBAY0@%mRU09NiRqle_jx+ zn6eBFYZKrPz$Iud$pMrb|bx{A38S1tr$7u=0FHcrP<+R^9904)6xHI3L1Q*N-TiBqX ztN+|{{Jtm0;+9=Ir>|r?DL+*?WNE}8t6xmq*M@ur8?Xn!6kzG-(DsiD4(!pV>#Vbr z6`M!FGirQ#Wk<)TIhc^k`Vli6!>AK7bLG@iP%!w>zJ)f(MGu<%2B;5_rk?Kb5$FL9 z6yY|};25&WRxX~43k&PV6IWM8*3PWFn0Ixr^){6o9i#Z6NsHZ;JsH<{0GEN;ykbwT zyTf)LB32x#z0hywhMt_Gr0fozXh49!W>cXvAw%`!TEv)lfz6)tw#Kt?XvkW|Uk98x zJ96k3a`8axBqsxLb$(3TAX|fjfcOcuXfGbVio3lab~02rOBk{c%oGC)adf0mP9o;6 zxyc;jvOP&PbdMe2;S4{W)dUg7oy|bU zOKpA8OEjbUQCj1IGL0c#lX0KN6~n#4pM<$1%Ax&dz$Hv*k2NK|V{pdIs07)fgRUKe z!tlyQl|1EXl<|MZL#Ai0C@Pz%1NusdYSGjj1*0lLauFMQlbwN38OPp(x8spO&M;`=Y znjmXxYvW~e+mVZia8jL68y=dAMG<$;Sgp8O+{T*b~0&)hNI!T}fVRrv(tGk5y3&2WLDHJwDD zmMdw4ul5sb6K4{#TD;2918G)R;bmC&ucByeFuuXl!^ zu4QBTg3_BSPDHX<5+AM#9WhhVG{4yoeq%WYGjQfC1Z>tV+r^SjtXWhs4VDFG{aki#VtsOC7tRy7?A-+>196Vx)i{#DJt(!kE|eIUdJP@$_KMsIiS^WT)lV-;B_CM7A7x*67)!Hn9UdJ;#>VbA9>B)Y)`!~%OS5|o7Ze6v)`^duH+gsR19>TT_`_*^J-0Vu?gZg8P z<$mb0J@Dx)T*$U#=sJlX@+4(A<1{`cmUfmrmpu+^$TKO*) z|Gkf*=R$V7pv>~-<_rnV4<80fm9mE>CiddR>H6b{1ipTo7uO7i%U_GFs?FbRWvBlX zd~Do}LmUptU*GDZ*};#$e(gQH*=Wd`SKyjcQ>#1=Z()OALc&V<$Wt$Z;hv5Yza%{qSNWdj zQYaa!X)b!Sv_5*~pl|Q0z#6~PV|I47La!z1_GIagWc(+8!$1A5(B6l=g@+DS%;is= z5P8da;6(WRLxE|dF}on%JDii+C1?c4W2NUl2FrbP1rN^jF7qt3b{TUIDb#j zW|f8Kp+!6U<`wJJtv4=EiNKANCBJudU!)LmJoAS(Yx>NIWEGIbWT@$o>jja6CoDDw zr~O;f$NX&h@eCK*h#%q_RE8k`usLC!QKobxnr>4G$M>`)z;Kmd{->+}>Kqpb5n4^& zsTt6D)8`?j(Zuf*|9ccrMfF?U$=_ISD$Sk*zEcbl(x2IEbl~@= zxDcZ8KHXclaz!EcLmc4ufVscpcHo85e`06tg`<~^-&9uY+)-w3>4*$>5Qv5)`fp(Y zSj5L4B!PzbtIleJ-d?1R0muWC) z{i~|{|6ulW_W>-jS&iKajG(s-86ku23fGc`q#&-Y6XG=Uuk^fNao9B9ClL! zIJ-C%>AiE#+3NL{HhZ(Q*fU8>H>061fxP!pgJV;1dQPIuWX3HYXqHZ@l{8J zM=9zl>FNI=xa%k*JI))9yr{5o?P+P)<>lq`>FQB$VZi!`pLdSvjf5|s6r2lcmMC<} zG=3^Xq2s#Qe#WHHO4?r}OS2Tw?)u#^+_%8X*vgf~3=y7mwjqEVz$?rEdG93F`=zi0 zty`@q1<*e6YGHw!x>;dzGxHM0xbS0U;#>PJEqe1O?s0DNbR#1p_eyl3V1P6GcmBqa z$s5!58)OZK3^+*8K}=3^#!8siFPuO4!a~XzXlP)hrlu0g@drUz^De6u?=_Mwjos-q zvb#LOK-Nu|J{3+ZAajOFYdl?qGa8B{%)6rChmTi0$&On~WTPUZuFG)#R;%{e&mfFk z|EdOz$7Xm;DL!}AW5h>lzpjC_cWshCPKM_zAOaWZTUm@@4M`Liz4 zYJ_wtiT~7ZowbXxihjH9zolcg9+&vhjYCEKsBF^H)#uQ|W_#4kX+uqDpycUNHWO$| z0OL;R-~BcJq{V4ROvNYBe0DhFSR4L{acd05-srIq>le1AsmJ9J{O`K^T+5cD*|C_1 z(QlOesDwak#9kwjK%F!uT}2l?%f$g7K=$>0@dh^_5PQ2+qgGM^jU9i{mCS$C7}Lpn zd|J~wJ{FB1|Mn{tv{%63K;t<>b*+q0;`@MTA7IWuOqUz^!_NE=f=9#V4W>8RWa_~U zFRVKe+xBZ&MQM~YW*9pwqJKzebNgvd(x82q@!xf4w*y0s7aa-n729RZZO8$17+C9y zjE*jekr6JTraa!4dAS0B7(jVQAoi}y0-8196p@U*C%V7yuAKW)TZ-Z(+;yP;j?;&h z=;?a9_X(%YSAmA!pewPiO38FJ|37*5P5F(-FMP<9yw% zYO~Qr+GzG(S-otFH=u{Nhl%;RRh_u`6KDJpz7cBrT5z=e8;u>tGHe`w3FZ?UW*D?# zdWB{0Es^JDyVbEu0c%S4Z!8kvjLc;1Iys0H#p;c2*U)G{Y-&6}=vVb)YJb4OCj@=ALGpyzVfGg$p#;AZD8 zueUw+=PVEl+&RJv>*|t#g+E38c<1iv;^~DjMm>r|eeA|o`x%-?=m`&5AMLAx5y;Jy zF5=46f!`?n`uiN<6l&L}h96uN|`i!Zm1_8*<|I1G$cQb?K`tndZaj53a;) zfoQ|#g8nn6wx2i>8i?}i)(Lnl^|+vCp3^-|ym&w63#8!2D=U~XF<1)rC6m|(8znq8 z$LaYlqOupQe98J&`51=d;2E`J0W}2`BXUK_PM-63!^Y7J66||zCY53ObSW)#(<}%D z3WAIA%@JS2B0yRSVxAnG&%O_rpXQ0pM(nVW*?cHr=S5AHE$Gvf zH(qJ>l7$*jmP$MNvp^X34^xm z1q6BPPZcz!FRJ5&tMRlSF$|Snc~m-9yP)FpWXkm|4dv+|gAC)H#(Hg&-Zq0*KQt;f0r8=IFZ^DTBs`m(`v#T{=d6}D9ziLFKtSsOI z4zoD0N5!lx0(`6Ae&LxnKc~XElE}uHFupQ5_Y`cfo(W4xK!xlC#15wBfp&j7dpuuc z0LkNceR=9-Xl#hA$8J{u9z6EOx4{T2K=|mxfV9@b@k4ax^&vt-oN#;&Yp}TsHJjH5 z3W$j(1R|XWYdXVzhKc3gp9DwE&&R|K%2A&^I*i;j0_u*Tm(u*ld&=d}MIJN8^f4mp zxispWg*s};LcCr5s1jI%s z$22W>rE-uCxGHLr2?OQMX=9HG%WxR?N8k3QYd&FP&bqkS0GeEvfJ(3nEi7;50QJ1 z#!BHxCm|uYIV!0m@?1P(7lT|0tJ7q&;3gUZ~LQi zaP=CU7aV!5XI`Z=JM2|HHZi-<WYyB7u~NHO75bl~wnV&a)|VS^276-e#cK8UI->1Xzw!Z z)x|p90_LHUe{}v^V3gzn_~V zmvn}ZR{5QV;%De=^jz{{2~Xa1NyWUDf}#nM&4*t}&N}o4^?}Cc(RTW5@ed`;k2y(! zq4edzS+!n-84JCqF9SFyqhaz+OvvsBcaA^wRx2~`;CoQ!kW!l=&KB@3MO^@HC!;TkXdl= zD^2Z!)8p9C<~c*vk9T8~N`81Dr_O?b;A)Hc!9j1XP!QxqjPsas=`;LwCEj4VjcHRR z4`HE7?#j_{5*qORcYu+X(_>}O9b9xauzUs|=4u^k0Aw~U*!sJ!^lwDO$6KDScMD!m zN&s7dtJ*K+0VhsGny&v2k-o{k*zD-jNM8O>&_}mh*qX}P%=S`fofL=?Yb^;0$1E&) zuq?P}r82fjQP!`lDdGj9bciK@MFu@mY#fQ)U5Xi%3)ot!v?){K{iUePX~MXY5bN76~$_3&KhrW$i~K z6~YJXCrdmfTm2WZyZn2U57%vhPb9dqoT`T3P1y(_vx8|xA>7~lOZ;p>{4TfU7(*fg zq*D85CaVZ%<_Y;+Y5>iBpylaE6OD27r1Mi$k&SG{=(<#K!vev8KlLwwm4=HO1_qx3 z7<&-Gxrm|QDaJ;)&~13oVd`Hn!;HVA1ckWg_+4l%4z%#8(o{WYa5V|P5WAEug9503 z?1BRHh7ua#GJufWzr5DUQgR8{10uT8fBLr0F}L6y#4CC=c~zx1^)%w}acjVa-~& ze;{~1nGUBta0j~+W=*iVt(otMsBfb+G?c?=gZ#SM3=z=gik^#&ECayfIQp(=WqIE8 z{+;=T;~^}R;D%aEYqno^%ji0a=HrKL2W)HWvNv_3AZYvV6g|l|BjIf=B&3kSHop)a zytKTM1Z)*L^;+fE1oM=#6#t1J{!6#{E4>ygvNQ54e~+UR%yiWtDmiLGU57~#Te(F) zd?w=CyL6$AsSz_3TEaiF+SHY=%~n4)M^+&ld#_+U`~mz}g|44DDi_m-)}QSU)6T-< z*2D2Tyut;_pm2RwQPGc&Kh)n!Pz&8T?EB*N7R_-S1kP!VDmS7=DH9GJ{8ZPv+f`E3 zHU#nQd2~;_eo$_U{8rMR<6?mGeC*Iq5U_Q6V1CS!QdO2SEJp-v6Pwm-?}#`HwQxL~ z=h#ofj)Ut~D@`#0&edkrYsaM!j1Kxumht(JE1%T^%N0K(qzV%ZGfE#*zBYVQ09ytz z*W2=Hq+!7~l}KjrgMZfON4A~Zc)3F-Zajlf_0J-qoHA7Pb7^etx`N!%%96hL9egei zR;jrh$-cBD_k1@y2wv;eDuvLmk-FmqvYcx(x&HjcyN*kv$q(hciF$ra8l}E+2Gur^ zj#|8g#PQ)*6Cxz(qx}_$m~*|gG8;a= zVawFF{Mw$l!F2qg(~E9G!j`%~EhpDO zBl7vtkd>Wo^a_)e^wDz*z21kSrH#VSX{Z^m!?U9M`-T9gjV@n3z&QE`4Qv8K^3i)m z!xJtqQL&s5VJw*#EX^4`m?-jeles`Pw=oZ0iV2TW69)R}pfOwEu)KR`JuV4%Y-k6}c%S{0MenF3mtbOl@HPv!74d=NW6bI2-U=sPkOc=C z8rm~D;NTbe9Jo&RH$?el`kXe0rGt(!N8$Ejse8*+M?x4lQ&7{fQUHF3&0s{W*~v&0 zFdDm_>Vpc?S9eL8n7?cRiMVtNK{T9*<~&6}mr9JC9%1iVSgnoR``OlW6e98-5$|4K zs(}R4*|gfCxZZNiw0qc*JkLCvIR`1UVz((H>g&$!IfQZ$AkFS$(_+!q!GlS)=-!`B3YF{W!r!9_BojuT0V1OH3eQbp62MG7|h|;+T`Fzq{rm=;6ne_C!n|hxzohohPQK=goa5 z%pXL1dw0<%wp?#Qcqb^BLwh@?*OW`cLejt6ligw>PP2SGRdYzh{BlIJZEy{~=N8`; zv#$H^ki83KrFMPT^t3uqth|%w;NYNHk2%gQ%(%vCiu~Uz9iX3|#rpgDP>G0QMIEiO@>=k!`?yHoee}A{ zh00m=BPD~4LkBF`=TCrf@NcCLm=`1|e;JQ}ks*^5w{G-)hTB5Oi$voi)%p4ug&b1T z`H{8BLKzCHaUX)#JVQ{&7W4LXRU%d=ttC+t-9)pumbke$N+Bd8qk8Hj;0RcK#e zYh>$UFmkb(PN5chmu^ixq|Of=^2YfP6Dwmr*;Fl!t4whsDM^st!`f+ycvvwk?sifQcuoyv1Fsg7reKdh3n`7X8#{8 z0W8o|UrGuJpzTl8lofYdb_VU46Gva#aHteGob+|I46fzbc5z_g>taLffT++;HjmnA zEZHwp2uvj;EL%ppf=Gu0cX4qEFewcQJ^}@KVEJs#O_)^0gjTxQ`Rn>m*JZMTW-j+n zrKO(;2mts4TyqVyxe(APddi2w!PpOgod3}u$735?+v3_y z6K5`3zza{VdJpU0ZRURrTM&g}HlIr=Ju~yJDiC3=wVt8Ayt;ZQf(5{+O!m-zpp2o< zG~k?g-yTO+9l?IBsYK>c{&E%ghaGc^ST1bI5(wztusZqjZe)bF!V z>nKG91zPDx1#0I-0) z5kaQPE-oBdr3FX);N5}v1O&AUwpLkT>nH%=U{tHcs|TR->*=*iC%m};a@l>R2vnl~ z@Et3AY~V~yLIMIHp=oLCt**3`8laLm9_Hc=ILXB~HQadrLZ+|ar z37yf9)!Ui~cuyVO;37hE z`dUR5qQ(ybL|Zi%#Kgs!aN+}3teQ6fSWr@}55UaCdfVutfP)33k|;T!r*Mggc8XA^ zzK8`wBjIyeP5$>6J+i|8*CnfFamoKjE2~Mz(1`#2K;RV{@Ob}xSD1(wz4039%F0Uw zT&5Xy#=||`{iFB(#i*HzymVan!P%k3`#lTiy43h=^vmg)m2l=Mi6qVVzdu^o^pj&k zsf(W;nMor}a@ww_MGU@aLudGEO=8Y9B&FJVuR3^e&KYrEVqW;@P#XZJ9^7WjUQwE2 z_PRE?qI;Ak9CA}!4kN_7IvGQ%YiQK$OWD6C{}(R;4mXX*iIxUTMP51u8*AfO^F!W^ z`EXx;r}5a*)Qq^UefweQX=}+rk}Lldx$^4=N&zd`AA1Xunuyc+ca35edOYQg&Ophf_^Vjv9HT201~p@on%m zdeNZjs3)+@)A@wCRXpav^r)J6E4Ftp@_cnODn9HGg7XgQf%Z|^emC8Q$H|>_Bvw)S z|AvQkV6Oj`NFWjZV|{biFcy4p<>0)NZHU)*>?HeH7Z=#+ItLyf<%Z1+4-p#+YNsEn zdrrU4tjvFB`DEDcs^pf>fZW*Lntc9Cx7k#Oe5>$P4B5diZ!0$jO>SJLt9+@-qX=ZS$ zES#!BJVmlVztb@Z`*f(}}51$3-W6qLFy&own?Oq)bwl;XV?9e@=wn zjIa4umf4df%zHZY1&Ib(**=-I?>BbQOpEJtHIR^SW!8#$pTtU=U11efpI>sUZOKH! zpwCf$%*+!OTm9yYBdRiA>UL4oNeb=bO|_aFv=Z&#oy5Q;r0U_tYZCw~>XWKnhXq`b zjBCheVUsBMI=Af9_1L-tGJ*Hy>7v9-TQKyK0F;Mr0NDC(gjA#&iOu^mBy z2a?JYng!#vjYTm!2S;KY@(tIk*plVwfJR`>NN3-v8H&Jgzh(_CVmr-6(W~2`2dMwT z{W)l`7rC;h?K|<2RvGdaZ>5akmSPXikcU{`T<`9W3DcaLcM4@?WLFzs6UmV6eG@+% ziJuy(hpgrdEt`9=Ghl~{k}Ule8Eh#|^>vaX<42>+1{LG|8tjk4Y`%z00t+7%57>j#Pl1akeY$U99B%FjwR9A%?@rns+HoJN42w z!^OoYJvpb&TXl@fWQbB*^tm`NLCia`m}Ww$YbPgZL(~t?+b9zVqs!*PXnzTj5NrLZ zR%cmhGM1x<$UX<}QU8*;=EUp7^>)6lNIGD2M0b0f!u4GXC*0_PUKUL1Z2cuaiaOp0GwB<4zlxMb+ zbP(OyMU^-SHO>t=l#91l6!9Q)uy19`YsriQhYnqY2Vn{+;+-2(!DCXZHKovQhmP_x zA+lvbBa1Gb4E24skBgEZy7T_+yqk~K=bRzsLJe*Rn&Sh6DP;^lp4xBITAkWUZVh|} zl1BwjD^DJ=6xOUttx7$Gk==r~2R~svwndMh-P%j89c{%Kkks2s-?x*rXl3UfETHu? zR3E;yTJi77$i@}@S?PJABbu^j_N0dy=zGnlAi%27Zk!rR0wo5=QQ>9a;$6|56}LK` z?5TR`A=n}eI-zxxrIKkyv+Un+=4 zlk7$}DT|RDZcN7M=h)wn>zFIjg@W>VAillewxQ&9;oxt867?aoubOudqUEAur@dVn zj#M5Uv%BU9x-@ySmmI3Q$lxVq_`fc+C5~B=E|mm_C?RR#+_4Mx9?O<*hcQ1jUXBL+z{X@RaEme(M5fx5?y_n2ftiJ7s(34z z9F`b+<|zHwBfSWCD* zkRtuJFTP>fX?6r(Ph#HFLMm^Mz5D|{K9K-~S{8T1-qIXZ z(Z|V-jZ^PthhQo2Vz#Z-3|dlr>JlV=nEr(B%Jw*M3|I#SrkyuxX(|d9IUb%V!u?~t zHXo9{e6;$$(4*1z+lBIk05q{96#ArGf01gbX5pJDlZLsTK*sGH=%30)FAwV9WbuPz zFnzXrKE;WzFLH-QVagj@Ss1=1env_gnH8*|#1UG*aSBB*u5{;_AlbLP5kg^QSVD%h zUb}-#v-6TMGb01J%<^Pum8P!YFmT69LslBpnjULGgRlj0I;)=F z6CD|mwdl0lL+}o`2axiC9L*Ug>b(}LYHK4(CIlN@h*%6tC|;wx^pB8w_=#_+LjH!=4V88c-yH73-~DqIePw&r@%*CiDhStI7P zq+^43F88h|q=^{@1JC^rTKBg7MPi3{c4oW`)Y2&XQd9Nx*P9jpByO&=DH4r)T1+J z)Z>O?NblRX<^=_|a)Yz6mSNQxxzj~r!yyH@Zq^Fl{`7`*Ina~8S`P6(Tu@t4P&39? zLb^eG>PblG-E1*07C=xbSm)U>}~rsi)Q+JgE1f)AuK7QgV|JRP;G9;3mGSy&i|59pl`xtKQ9|aj&Ce z$eu{W{kBw}cw@)i@i>G}-FCBnWOR}&@JFpBX2;1p?h|H{Jo$87JovP1-3m0nB|2Q( zi3G$+j6u-SpHcL+NVI8xTp1s%+}oCJJhmqrf}W48FMnq^fx2OY_+BJ-$zbn6KNytJnNXg|=SrkM5e+ke22)s+`|EvY|D&4& z>8_@lrrKwHmMb}hp-gG*Mo^>B;CUEk)!>(w<9K^N zL3QPpBaC8p{pX-QwML;Q1pnu^SA@#nneR1IlQC^gP9}J!OW38UR<*}i^8J#nelr5w z_%bMeU2bw@^O2~wB=!j$^5L^4>nB7jfh+8~Qh#5`DHj>Dieb>3(Qb8xhT|U?qTFRM z%d$+kogsqETvxnN)zpp}r3^MB{ceR%HSmsR3EI;EtrcIznO~=GL<)-6j5&#jSe+?` zEs>20!jDp%a&zZ8(67Pu0r%5U&%{&oL-WrZZy)LuyY0qauFzRLv zp%kn=)9J_|YymtvZOL&r_Cnfa))ZL36&%&1`eFABFP_NlYs9{D?ZYKs)T?QA-La`# zG-kC_+P%YDjey2*S;AlEu(P^`#u^GKc&P{(nkkZ3<2VayYjQcfudnW(kaLPzFn8F` zpl(ToM?w>Ex5!_SSA7qq7EQI^f6!DrzP;GBA>duu*wWZiEb>)HClbitcpy@`H-V@B zs5LGBm6m0oq&`vq{rEFfNwXX3X_cjiZpie=!+Xs!vcbu*0UGX^66eIu?|J%5Jat=+ zINC1dTD+AJ(GPE@{K_h8=4X<#GIk(P<_E3=sRx-;d>nwY=fR zi~P)1OJy@|RmuI*dJ3*4iqufQ!J{&rrzfqciKR)!9y?Z~S6D7|?ERwT`k)Dj6Gan| zMQ@n?b%>YYMNRzgM+v|KM~#-~TatK5WD;AJNMVI})VvqE`> z2v2yuLpx|UR|ygqXUF$M*oW8WM;cRQc?nj6TJ-)p`xjZ928G0mYq-112 z=6liDc-79%tlcU#Oj1GTxZdo!t9u(6xuJYHO({6_K`{T?SJrMT!gHrjE=0A*CV)_X z&B_y-+nCexXwUcHP-k_)?>-G~f8IQ5KpKenPM?0T49d_SFQ&v`RsnO_9V?<71Mg*O zcl1peX}S7VUS3FPoN{7gSU6l};;K?uQ5R#jASplYy8BTx!Fmn@|75qXK!-mpSPy%@ z;Bm-!99i;R>NrH_R>=xlDIhnRnxO~BaF72`fpNBVi9Dvvz)6ja28~gID{9>KLbAY4 zrndb`MP-FPQd-HN;GA+HF&ht&*uBRc>nPIp4;HzIwRviSM{jZ1Rm?Z>Om&I`ZWNA)4ah7HFoRnuNVpDQ?S zlH>Ij!&g3S3!;C-VtQ#P*eCm^l4V%21%qffFy!9FhqKWKkNr3kcB~voVp%TiTzI1j@zJXM_vf@`$inXCr7@;^q?S>yVoMko=k4zFHp8=eSS0yp2`vuH-NS#C`C7a5}<+M`K1_PwXDcRZaR%|hx=Ry;8* z63X}VQ`S*s4BqfmPKikAm@vxZB!46HnZnbbvEU=KvSX6BOh!_|>5 zh^0xo*7ZpReWsO%WrLF189<9>XJ?Nw%nxLQ*Nfj*Bvn#}liBn|6=HIzEmJnz=~bXp z z7XY@lj&Qv4s_=SOYrS0WN~Te^W0EXp>6Dz6^;ztfu}4|LhMbs%;wQN8>Y1nr1q?$L zE@SS>4XYA5RUqEn1?AzMe#J31HfH*V8hrETTS@lmnMnm-UUBr)5_8OL^TKF8T-$GZ z*Y+eHZwrF(=&tM9;tw6rhsrtT-6;pMxjtG=gM;nW{(%db)T$cB4|x9d<|Gdf){08? zmvRVq`IbkEhAu;-?96o4DOjV}TiBjF-2)(9Yz~G1c5gIAtjZkUTP9_sWpQ)zxqTZ! z&{Qdy{m(Mg_-r;}K|L~3wf1AOU2JV!O9O*}__?{MRedq}#!!(Q(Z?LB_uJm@P#AXS zoPGTms7e^2#rfHcoQ?@A zT)W3efmBEcB0G<*MX)&p?t-;c6>gx3+oi!eBZwV`t`rkg7_u%)-J2X0$ZNbRv(DJ& zkx!=eEF~{UOxd+{@mDA~pHl4ihNAM|R_!|{+ltLh8WiU`S9*nd$|}{!dx80@V;9?s zgRa!~MQ>QjaVpITxBgVxHiLVtwDuNXb1y38;31eqAwRk>zlp2- zfIAe%W=kp;c=(|aImwAODT&}~_nXP-rog;bJJ@!mnuJtavTdTSqx4~wT1PGo6L$N@ zmZ{-oGjEj6zSqQq=bg?Mq9&{o(*&r^Db(PVNlopsZ zHRVaJ0vNw&{g`Ct1NI?v4N`NasTiv=V${~jxv-0iYQuuLkglP~ei(6UrcyPJv8RVT zWzDy`v7At_5n4a=J@x8g+gi>rgr76EWm|vq9tee5nr|`{DV^2?jk!4rMRkTlEO|JV zyP(cyh#wT1zu#_dwpyoDu@@R!4vjv~Ur=znLd7%xuC1LGhWuC5`8~k0m;MD9npsyix53B8K!PQowOmlMW<3eNaBrZ+2Qrphv>V#btpX-G~G2+?FiyfL>y55;QFUB7C1V zW?rYOa)KaY5}uACnPTSZ9XRC(skuW>7G#XJ@Kt7$nmlU%2Ku@^+!2?A9#0yAq%U*} z1}}zs%GH%8cMuQ5^jwc8(U`^`3$Z<3mWGQ%t-*jO$aYl8E_urOYy_O#WFcM=97>qa>S z>5AS-k@5OLxgn4OfcePJ2#pa2iF5EF5sv!UJ`zrCoalQ23+1$0Q?!y>OeDwRj2wt% zD_&9!EKntljV@c469*!1_hB0NonbN z51;Sv|9V|sTnoF;KIfd7duHyLXNC)1R2zyvsM6>hVP0<*p4meki$r|qj90^(X<*vG zV9e(p_c5~*E(xjnmB}?Mb2m!x<6LIYuGB<-NuggRY?Pav%H#)yTu&JOhXHJi5Rlg~ z*WIW!e))ZUl=D7tTfyK(#Gxa$hXJ;SdvqeVF7DR9p6IJrRY4uKu{-Zi&^+8+ zA4ufD6(P=tjGpfb?M7cP}Wh+wp)BLEFjD^O_#I$B!n5Oyw=A7l!{?3TKW z1V12F9wLL{w>N@#0v@YTkveZm*)NX8566}`ObxciHE#}ku^v~Dkm8koBBVCna=MSl zv|^y`FW_^$B@(~eE{BQFLKlmI@n$ zTTpLLr9ni7S_X1VC68>23U|mz$n8z0+&&~)vsjIqV;+3^m$O_Qa4R@wFDfxpMg7Xk z>NrFAa`A(I_nzkPxMya-{Y*}^-!;Dv#gi$a5{=UZ)8wSWy#LY*e7=hGN<1eUBbyhs z6kdBDU&l^QtF>E`J&H<|(*vfZ;D^4WPFD9DzB&G%a>9zTNQ_fQ7{E>7qUOj)>;Pq!6X}UFx zZR=Mw5skW43=)~UQyC7QiWuGU+cK;_Lqh1eRZbfx_(7h74INbgsv9;Hk92w)Et}so zf`32=+vzF)rJ%rfD2WI$#N80l;1z+tHCR7VHocM(E=}l!96ZJE4GSFn7(B4Hfv>89 zjG--L+16RCIJw4>J|wDpsXR2>JI3$WwUOStbF8)aZ1~d5soH;!*r(8R-`9(yT-34Q z%hF&BlVax=nn4u}QL4dYX6Sw9-9TBF==zSxW06Hx_11s7 z?F?h{2Yd1t-O-xoHadpCq>?^Jq!w=1$BzZ*Wz@HJ zcJd1fLb|&jdJE?>gPaAz8LkX6>J!#U!NDk?+DV`t?tcJTG>KQP9e%DA8t93}T^ept zUMA0rRPdq>9krMAsqnY&t|s=jpG2!NQFsp+!|6#lbB8B-cv}sPl~!|H=J*5-#TnY?ecohE<%P?f2qm@pj}B*G|2tYd!^Cftf0dcsXEQ)#IpPy5qiLrY0O zS#>?3GC2_~bvQ<4gE6soS>5t}U+!kB{D}si9mgZ9(cCKL6#B+QX-^c!tWN6_G5@0s zl0#y#;YZ@FAG8v?4=>3Za^k{I8sd=(mu{`@T<+-EC;0F7TE&eA?0&Md>Bm%fir=59 zW;ichFL|wG3(}LB^6mMiQ;WA{_b7eWiQlkVSnB}sIZwcOYsK(bbsidq_u5j&;Co4b z@sCTKzsIJRL659eI5PFj_Eaavf6P+$thAFGb`SbXih_@5jVAYcz=8P;hPFq zo`4=#9w45ETFQ-U^O`q?QnXiIcEiCoX~Kjv@)@+)5JO!~%7jS#CPPukZu(^IRBrar3v#|Bf*AQ2 zF8Mf7h$Saq>(RmbCapGQQY&6~f(hyV78ti4p> z(KIl4$;GH9767f-6*##WHQ#4`iSZtOHR_rTaN{!x5JxG}wj1|Ex(!^*|1WR21k zv{@>~+osG(a+8H&3>_gukETYr8i-Y58*QCBWH3kuiBlM3p`BrHeHIE;mnI}kf~WM+ znQ+uL4RU?Y{Ft?_ob~?I@_QOxe3sOX;8~|zraK8NB1l{ueI-`xq*qJOcy#N6*}VIDf0hn0mK(7Fug86k^`Uo4@b*~x$jJ7 zNyyJ*1KhD*ISrUj(jFrBK8?opo)Df-H5cQK%;)0bk^q$`At50bzG6kHIV!TU7zy+) zC+ehsMP6t|eZ0O~-PN)z&P`d(+?x;j^&QigrmZUavi3=>1HA+vUN8+!Bz}sFoK28ij^j{M3Z7$IuLzjQlDI6D2; zWUXKHui>>3een(F#-|f9Y*))oimy$j#Lv6ls}Rx50=kD1xYV1 z5bsn^j-$r%lKrwchewvPu43DU&A#MpHc;Q7YdZK!eYL;*k~5JQApV4X2vk6F{9U8Y z76q-A?-&YMAR=uM_qF@%o_=>jTIT5&O`$y|pu9)$?GE7@T|CxQ(M#~~xJ!XLFh*Ts zKA*)*yVT1X9lEuu(O<63=|G z=yA$18O$0|5Rmf9C38CVtx2QVQeGeHh`MeYJMMlY&P0AmUS3puY$$}+uA+Tx&-2VU z9{nGS4dJ?#G1BUL!kSEIi%M3JXlZI4y)c7?n~mW;;}>dqOA8kiNMS(>@l>!rv#U9` zlM42;jGVQRuLr!mWKtw$v+i_ADvn+}0%Y7aQ~~EX_ut>Yb#ngPoYC082Wk!WF5Ib- z1NMb|twwNLM)}5xng4_>wd8cWW3FXZ68XLrGQ%LXGB_3Wcb-vZ95$bwwD(_}3=_xQ z8NB_0gSX%KUeMcFfaH?ieL~h&uWfk7Q^T_;OuyqSWuE7OZplN(xp_ zPdFuIZkfT$gY$x^?YMtFi~0zSHi>~EZ5vgb_yay#q=~!}cWP^M6-)H}waXee%#`->G_IEs2j= zRj^rJJM^n%U$?k05$vNI{gY;HdV{k-O3Liab8O#4?;DR}Qo>OYDJX?EK7(`J6 zVkSBIcUIQY2-iweRO6SjO;Gk)aqPGkN7$r2W&*iIt|!vZt=>H@xwO>;_q`o{FL&&MjyaM~OxDMfsrc#7ywOd|3vQ>_CU zL2vv<|2y@6i%cc_X>rZmIvO}>J0%;v)tX7>+6m@Kv?9u8M$up01!|ZNOEpmH_5)%_ zRNw2IRNRjyp>YTqwZ=Q%n==a1kngO;EZyPQ>=(=n3Ou4b;QXyUL~wtK@yF=ai2d*| z0#rJ${3z8#NYR3x1=e5UlZpuyShQgw_~+_r)vkRI-9Ch41Oz z?D}*<3jMFSqJB2Y3Tn0HjHC{x@n^g{kk21kG0$W!u36V#Vz8K*g3Z>a5d`}E#@Sau zIJ0MwS=7qc2hz-@`plt5$2zVKU;2z%3AuP#a%n1dG4l3HIeG)mNEkm(&#-ITN}x6k z+l~5;k#`CN|Lzg&Up*OG&$29MHX!9xdeXLorc^KGjn}dZ?8S>|Q~tb9I5I*P0^d`77~6c1$F43( ztyh2WL%Jj!9XY(cy|IEj8C>5!FABIM372h7S5SwbAxjc^i+d}^g|%WvP|rrsT1cVg zxBFYh_PextnsUpRR70fT@4L)ph}X=$!kRg05`29)dDQpGVaDzQc%N-J=p_TgpSY=Y zuX35-yld_VAUzrE=rn~~&m4Nf@5AvYBI z7B!aD^NH8A8Vm03vzXYWA}J|$-PqChh|^Pz>m8Y!8_aigK6L(_645(5fg|@hZhIf( z0+tAt!YLOQH_e~W9f1Q2J9?bRoNzOoMh^B~tmxy}DSeH!fOFRd`_C)&R_WXn7FBSC zbdiv+?qYsHhOyI)2_kDs$_mRh!)fKs^p&{dp2!-P%TlOqxUA}8HgDd%VUCn?apBh0 z)lF4q9L*H$X?ENE_U)Slp^PI39iA$-Tk(&ocWX+SZe$Y~_TL;SO*vJ(EkwC+K9AgZRSvP7VT6wTTIfYu_+S{ zv*!6v5gQ(6TP;2tV)1yVQ&kmg)X^SdM$=%;V^5TZ+ zBC1`Ja?1Bf(38IfE&kKI;Czm7SUK5LQSC<8_Kn!K#rdH%nReQ6h-T9YApTaXnQ`c9 z=xM*>6Kmt6kD}RkS@Qp#@0XBgnD{I=^HXsyF6Ty?HyFv*fR-3jcf&6Y#$ryg^!@zq zYL2j(9t~OT6)2?tm=*+|53yI^}3$vz%ZRLek>?HF?=b+m6Pm+^is1dBLUiu{@#$t!ON3v^3^9pDJhW z4>LL7;P%_rCxBLOR#r>}1qIQG7HNZtOo`dqu{2l)FG*o!isL_GYa8wtXcsA3(Y#jVFh*|JQY#eQR9$E}6304AA<#Arxerj2D!qp?V zP*$vJPH^KIyA`MOL<%v@HUp^ca7jp}LVh!-WsXX|Xv9p|ZOwPA4o}NP;qEWp!W6a^ zT99fReK}jlu3Qt%{BXfQW%O-jHDvTEWPS;BQ{^foZbluXM-a@_zzl3DBkI*`(JeVb zV&7YiB0M~(29kQespm=^IpiXVG>b50m`0E#F%I7TR*xC{&{ko#j?!f7HUmncE!{FQCCF=S_L05 zuyn0m9_7?znVX0(VBD?+Pmi|8tUKqS!UP{(?l-bkH-A>tb^g+of@~$psyBBjw({2yxQLyw30OyOkFeg9%3g}Fw?dZ95s=I1jsGuB9ku%G9j zO?}E@i$e5g>rQDxUKsL=->Lo`OY3^k_^f~JM*(hqjpG{ssC#svt=Ud65qnxnlrG@~ z7tnsSS41IQf%FPqB2)w|-U z(PsUn+q3pBJIO6SD~OOyF+-4QQ~M&g`(s5F`p@7*Ry!l>46nWz{&{8DugUo_UAaH; zE8J|rtb4VJlw$5P1}a8f-kZ7~8s+tkKdu+g=TF2{wWX4ggl+ep9o_p@rs2e`gv&oW zzh9Y|Kqu>DoEq<%bca4(OBYd(lvWp14>Rk{YqAhD;;^xpJjG^M?agEokExpSx3qyNqdf$ZNs}oHy-! z4>otdwU8Uh>AD{H8Hg97>0^8Q*ruL3-5^M&_hlmTnWtohYIAOb`g|{^@yx9;$L`k$ zwvzS39dF2X7la%;Hu?323ZnmgUJFezG@r@s=43XX9XWUQb0cTSZ8PD&`-^TkH{avV zJkq`EwD~?3$KJrFvArjX^FA^}lfSTDzw6nF%Z)qW) zYfx9@F28`SozNz>F8#qAYf~9{<;m4@T>Q6c&U<|&gT>>?(XkR|m7^SW`xh2_gJH_X zMP*&(_~JGgDn-twFsFMyT1i{_mqkA!q2wNIByvRhc9ADxU()|*{oL!;&nxaj?Ie-!&Ue9^rTc4aLl$pD zACU_oRmgSAmKe;k@8IXSbhfecgjaUhZVTzKt)Obi!P+0!rpmYz5%Am{C-^NE6WSvl zLgo9&TfY}|$q04QVJYxXEOVhwy)9AYu7)H$h6a;ngU--??q9gk!_E2P7 zo$}Lq%u$w*Vifc9@k#TftpQk8{ptqrsjtwIaTyO%dzn_vXt zh1f{hyQ?nB%*nKbYiMJAr>Wc(&-LuU=j(n!BPU@iD=k#iL@I<(4EIrV>;!gP6fwb> zKg>k*A6ZZg3`kI_>B%HoImodJClTm?5TQB!p3$NAo5c_iYK?UUs6_|)~>;ZZ3Y3%&Ux@1qUEA3dtYbOos@IQXzc-Jk^+!99S&|8t)j+_<2Yu@>eruV$u&!2}*lS8{)VBmZc6BEICqoIUj6|Z_z z{kKWcg4{S$fd{KCz6d(QBu387mRu?GNm_6s=>?y;N|i|uo!>mkWKPRxWYS$G%!>cR~KG0OD$`zjv@50{Z^RC==RSq;)z#ag{g~~tZ zPRsP@JzZe`+vmT~2}wu}?SA+Cq_X;cA_8uTt~Qx+_N|n^$6p{ldILOHEdb zDW<1SW>0I&FJAn0PpT`CMs4YdeWZiizq*FAkoCnc0WrU;dVaK@d;J^iu-xqaa@aca z6M8)0eZJkXS-9dub+sx`qN@K__Z0X3@ZUQ$;!)b&mTx~uZ^O=ND^KKZ76tB}F0rqc zv{yEIT}6ESv2X-gN;Q(YFG{(%`#{yxaxGa|(x7`%E9dQ5^D(T$gOlZ5=9_CwGFT?P z=khE6!@rJ|0b5Tl4J=ZFf^PpEz-aUcHr>6l2iDi=>4F{R!m$7V4Gt>61|(TR!<|{P zH&?F3z^?fJ8Nav?k{f-i4&xOC494yCmbck+qYE&AL>Svk(|(~oYw}f*NNbOGTk!!PoCeQ=!e1Tkk3B*g0J6p z?cJY64J$Okc6N%-J8Aoi-pnuaJ@spI_wZ1MT*ICOoOlGk(7$Z$@J}lK8+I}L*bU1P zQ};5F`uO)9zS#N4FY1X9pGEHoCZnZJ^AlK2WmJ#cpTon6-%mGNnlkA(g8&i%y)jT- zyB5n%>3RERw9uY^2&_6t0yJ>(29N?Cr;TT#8%9j08*qe8lZOnwpH8neaZ_+%&e0D$ z581pt7`r2ALoHCYqg59Ri)ENOlry@9)k86bC%Z`6kKbN&nce)M4v3Hxkk|EN^Sr%# zi3YEiu;=JFFE(1CzCU8QySl%TJHu;6Oj*>q&-YV@Qfht9Bs%wt&AwgQ5WgQx84QiI zXIY2XkZC_CsF$@HVM;r= zwyv1ab6e3(YT($+dmTHecamZL+i`P+KX%+YrmOVfo73IK@_mmSZWdSP;3{?_m#f!e z7h(@r*f0Q~TJaoXGjlp!1pFk)oF-p72bbGm|4y>hyYSRxarPhH(ydX3&X0z0JIA*7 z$5nKyc8@PE^uX6Y{j{@@K&Q(4T&&C7gV5u{q^Jwq2^~&_U#Fa1%rvatZI2CBC){ME zE{c4Fo%CaTl5}-eK%mxClEDP3qOGk>;^*i0zv^%qCnwH@g$2MtCbR1Xufa1T53bj3 zD!5Z$`7RBd`_)ZnlEoZcU#z{m_{qkJ@5w$N6auTkX@1-18c7+GJ$>;y^?fo!U6RbW z9dgyaayUcagEgO`s_K*L zv%P|%qWACLW3UgcYWXfbY*PA((i)Hn@`~r_W_8nn~`B@YDeb$nR<9AuR zn6s{dfpZ`zzSQ|dlm?!D1SSO1m)aKyRe@l{0jk*V8SBS-*MT1%--OZD=?>bH(SPGB_RURTd3a-jS zBSwEZw3fat-m!G2@z@Tn!uDrns><-8QOB{umnYo{JNPEYT3=)5I{wMk8Po@(hCDU* z%K+Z|^Pe9-p-B9Ux#^JoN? zH~1Ag6Ww|QmKBriP)ws5cSmUDOSa4Ha!yz0rWE0WJN6hB2pU8h4JsLEf3MZtjU7*`2U+W80;5!FcL2KD0EoVOjvX@>)62m+0@`k`}FA(VsEYyKzf6N+txU3 zS$R1~2uK4VP?3@TMgYadjje8$Hm}vC3b=8}ezz}(8|w-SyWVg8fJyBO33((X&6RqJ zSVMzIT*UZ(=vvHzd6enCPLjsPs7(n6jW*PqUex@*S^zi&U2srPm-n(a6XLV;=gh&Uo%cRs-*aH=3r07HnVIuM-7B(opCK6IyTz3N z(PlhE0pbZ3_2GnJT-k=*#qh$TaBL>#uqrI8{Hm(B%acv;7`iOK{Vxf|M3cLlOF|+d z(9Uo<0muqvka6VI25}hQl_bl^AH5!;^Bc<)CJEC zX{35yc^-8pXdnoZC~4Mp*xQaA@m}6#RxEHH8{PkV_B(m0>kZp)maKP=JQuk}qu3gc z)&uUxzUnonzN5P0i`k}$OsjRsC@W(H+#{QI<@)XUQgK;XB)CU{7}+!jX|xs6l>$WK z%g1}Gp#k3SM+Y@3{SN&rOFxXCUa5NfkYI^rXD}2$OTwal3VZeD^>Fs>yKU+WsqsKM zRk$qTDME9S+z71@k*29be389TP`8Q9uXtn!gdx)o1U{fYsx;zd20w7S0Ood|fdwU300K#MlFX z5it}MZX6hH%(bhmstR137px!Y5(jgWR^7v~Qc$6^s_=7Zw2%`Z5lto12seeKmnD`Q zk5<2M+sx_&zD7o&jm4)C79Gv=LuGH;1Fg9irPcT}g~HDGXx0UTXz04J)md!|?5U`F zPhUvm5N-yC$0;-FHaXLRbzEKfOm6cjvREpDi+{$ZxnV;jVxe#IM7q6!ky8znN1M0| zUo1K@cJ~n$zxY(blpE03{vvTWCT#Fzo_#9O46@e9#$3R$Uu|WK#i^xgaKs%y*(`GL zr#bqsS0Z)$A>P3kJq4wA&y?&m*xPQS*xP(a#Qpy~=KtCn|B7u>z1frls`9}?7YI_d z`?dxim54lOk%Eqno{0xt>Bd)K^p&`%|2yt@w!^xAF-g)?#(}ZOEV!5O2kTGVH^M~n zrSDzUEVfg$DvVg(T}5`6rc?SNXZ;$`4~wr7lE~z=r9MyV&^5% zr3|TSYlp4DNezxg#?}s3dX$xw)xd~2X2Mme8g4GGR5BfDU2d;N&$n z9(+RJF+@ny=_H4Sb7|5AKm0yFgWPl>-Sbz=ySSxKRe2$7B*RaMmuvW+KbP_J6vz;8 zV$4$E4Y>2gdGrWO3J^-tFgTW$K!_9)4gjC;;2PXD5LDnxLs#5c3b`&wm(Gv;a*mE% zfD$!0Jf#~>0@2*VLTpa_a_sq|c3TiVCcrZRg7^R|y! zFc$${?7PI2F_JL!2P>?Hxe!kIzik<2mWC^W{+)dT4~C1m@LN<0-?*67<8njY7j{!P ztS|7U+Vs7T7F0Gku3wblN2;0AQucKipS(M$$vtL_l%f*NSTuj(Nj*vXuu!CjwVF*1 zceG?b=BzMr(|Jxu95=nmd+yt zVLe^sx@f;_{}KJ@uk4Y}1~Yv+MKE>#wBN)V+V+-z#te&hlydS|adADWpKHHp3q)$K zDbe>8X2YOfD@@7Bqq&lCv{#{}!=lMoQZZ(ciWnXA8` zO*2lWJg6|a4f4NXlmv|dU7kOxWV#zx^?LW_nI}iai(jJ?9cItIw7Ug12p{D0efmbQ z!u;PILvS%l%gcMtAG|!e?rnT15G@@+yta3yy4Bv8& z*o@9Edu02$QM^=gDyrw}EtIeu>Fh+>o%v$r>Y4zex=ca0Du;|TeCX)=7+}4I9~_VL z8dNkCv1E;K<|GT~xBWkT1NT7!u&A#FxzQQgzUdj|yERh4Vc8P2lWmJ2Q2ATX_zEpD0#Q#@2~2R%1RDJB_%Q{DoapCUDwc{p{0e; zhy?e{@m3Kb^5Afcp8NIp_y4LgK@ANJ{V_8`fQ9y-5*r_K>)E{z3)5}&c;>b-ocH-N z?tn`w%foqd!5u1hYwTt?uCA_91)Qh`&G8EXyn?li>M#$^m-T%87ZQ+bWgjUbpfz$E za42Z2NEaSR89C+u6F>WUu-5lo5BdJp?O5&K-*&%lYmE-~I=r)-%MjCL{EFvi#w?Q3 zG(Wo|Ei8QRn>)U z#DnUPW@-Wh^ssW^9qQ@n$ulMbZ}Godi#DFc3kv*zFDZRNTJ4O23{#?|TNjL%s%+Xd z(dh7vWV+i7Kz+7lAKB;W7sRH|iuDNc&3U2i3GZov+~q)r--kd4bv{q1+b-)Cp=5+q zQa3hQwLKs&3{EGno0b9+X98|07a0T5@ax#y;wi|x;mJyl#w6l56Jun=Icg#!r|04l zX6gtQ@!p+*yU5~xN%oSqQxmcJT$mh!mli0!Gb{}J1Pa`JNzZK06mG_r1c3j04WOic zCDVY)jZn9y{oh$%FV+G~*UBX|(!BZV8^=Ar$wok;wOtpiI^-wHm)5QWuC)<_YRz$S zVsFBfb>Klr?G?^+HpB})PYN)zm+Qei+3i-b0n;Hy2v)8%ME#daV4y>uG$a<%<%)ua z!mo&}@(Hnf@HJ*J=j(URI}f4;*4zdMyb1=g23E-k8efC2KFCiNm~Jk9rd-Z721Mgi z&#eM5EkM6I-98X%^W0cCIZyVe1kpl1hq!#xxf<>#Xe)ue(#BO8Mm@PG(Ux$@)Vs}Jo%@0aU+ zFGmXr;l6Z#=bIBN^cxdYiqnKWIY8km@ci*{LUrn_p+{bsXMXlCCD6xc{>pzh&0QU}8B+gPz8_j^d`(sQU;CJGk3B|G^x; z&9{};^8&gP_!bQmFqrvTe+;nt;{gBQFw40H2jGTsIW3CV+S&r}H$Ym>zr#5ZVG94Cf-C>n;Q|>)HOo z5D=Vj*#bU({u;9ZyaxMOKvIG}@%c%sTG9yJm+&r1)047O+!S}`w=cj5TiJJ>RsKU* zF-m&ze&xT!;`5q&E!UDMm7B>mej2pphMND{X!?7|vFLC2Xl=bu$Ni)}38Q%nB%LcN zD!@^gR+EC#E;Z)ffB*Qvsi~5$kV03Q9>mvvwi}41gM$O^DgHgnBiDA&2G>Xx6tZ2_4B8UA#gU6EnHYZ&~1~Eoo%@}n)LvEVDLu5AbqRHo+;S8 z;T-WCEmq)y5P)~-g1Buot+bTIM~<7)at>jXYk zH*tYhh|ixt2h2odR1_4tF_JL=Ec1h~u1Hn=|K2RH$@UDup9OA~3b0MU;LA4~!~&%P zD*S(DYrqANJ*%BTsD#ACrntz?|AweT=V1Ko)K1c zL2={Rn&zV&tk&DF&BqpZD^&vlO;)sY5=+E)m)j&RtSPl6jNN(l;v1#KBKrZ$jNe}v zX!Z5=foyqNg9Dl(1YLmx9vKxC6-3^k<73H}FIKj;kpN~UXJ+KUtY?s< zN_$uPhY?E^JjW+$PRP-5Ag{Air6)0Rz16i>tXN$&$|!oU8z14mCxSLl=)gECSH^&r=W%f4s<- zP_nG)xj6PU0Z9=%Mfkt>xR_u|Wz)S{_+E2W@%1LEp50nc(&ZZl6!28Q-Bhl+Y7Vfx zwZ?YP{-K#G>N9A zeO0lpeP_xOVcS6}odMOHYr!)=&v8BS{Kk`K{#UO#A6J|($8`7hBI7;oE0xD=g5*yy z17%Gt)L_vleeiPN6b~rF3kVu4@Uu1v5CXtkbLIiJ-4qAv?1z${JsqoImqxU-2mzFW zK-q^(86WsA=_W$-5XoeYH;UO7y5U&$_4R-#rUl_q-!=qlNdyHZSQ?uyFl%E+Kjq|R z={Dh2)}C(}gCvLW=xD`tiCsr_KB6a3Vw7d0jO~l5x)L{W`cB>9-huci8ztqu#?LX( zRV1Q)mEtmyh!UQhoAK2BcH7gtq9z&B(Vpg8U#;J@2S7Tdw{M?=ehA7cDm6eN7#SJ2 zwzg=}lzB7(7OF@~JlyBE@-JVC8ybio<~ab)3Yc4l_5(9r;{TCr)RPkvK?m0lQJos- z5Uikp{eOQ1dhtO2MT4D6<}`{8H`XQg2IK-5M*ja63FPR1P8tF5H4WbF)rVsxTf_@^ z?i>6!AuuoL|28VsuUqRadtqH6<=7^(I$l?9%;6v%9jR~h9|W_V?r7U%r6WGM2nzyd zSkhs{gUdvcOcPn4Mbovq`^9Xx;KRx9tHSvJ3zmjJX9N}>43ve?+R4Q^ST{%N0FKUtD+U_@(|YZmGRffG=2s_RAH_l-n)9q1HZZp#W>6X<^)*oXoop-9x`dfP}adU zcpFHpY`c>Kw;N!)wKdjI4f6v72PXn&lalt);m;Qf-Cq}WPGg2z1lH=IxWHfb^8fhY?}G^?BF|SjYM1& zQ@s56!omm`$#v+L(-Zb&YHsd9P(ZU+R1h7IQcygl`k<+%7C4=rmq%MvTs#Qk!4OH@ z(O=`^kjmWzd3q4euDc3?M8LN{t^l=wv|wz?@Ml#gU4`jJ0@7~fwA$eWspXh)hk(DE8@<%@`%vPI?U+f? zoaW!X{rtKs$H{7lT!c)^=Y*1aGHDvmK+mPWjM|31i>C)wDNkKBVdN)d;|1#?FX8y> z8b64>J5y6v?*akwuZ{q5PzVZ&W&X39&aD5(d!$war{lpn2lj)Kk`nyA z!D(p}m{c1Z8z2z5|7kujFjmTTFS9jpLflM2jD5Zo07x9AckhIODR!7^Z~-(l&?~V0 zz@$Hj4lpHLY!MFz3Rmp^OuP;bttzBYO`_prSSnvo`Q$bC9X0$B`OtJyv-w ze#a;7WBNqc({uc{D}(l%H>!w71-ZF_AV+izau6=CFdcSTaGwCy{)^u%-QI(hJYz(4Gsu9ZZWaZ{uqk1 z)YRPK;xKa_1v9g!$rgYW=>GgPlg3wmcHW|QroC&QvV2RI_ggn+a$KFDA{t0=`AU2{ z8Iaq4>y~7(ion4~NJ-UraL;fk^Umoc%YaGW4Gy)oKS5TD)!NaK5kCTCaaO}=e099N z1wqL1_ji5NSD@a2bN~+mc!|Vz>6TVhSlHV~gFHY%z2(7T8%SmXqy_=$FaqRzegi)i zSTXzGbrJP;Ga$|pbY4aRei;zm|3Nuw$Sf1U4Q=fKuo~S)N6KoQXb`xNlarT}lt{4j z{gO{2I&bcD{Re@jEuUN zhO!SEErPN8FF7Rxpta>LdqQBx&tHPJ!J2>JxQshGz7E1rF^*sE3`Uo-ylokV%Lm!s zVU%xff2A$`N#CGj9~Fnv5=U_Zy944Mkm>`j&;JO=ykD7lpndkib#zP&G^&q_tLXw{ z{l@cNfEaWX^8u^?Rk{bD=s))Jfg@*+0_mIoQTXP-@`C^Zm`6!=VzZ+EGx@s7GT?a) zAizJ%wI8w%VDS$YH(nl4qCm;G%EM_;!fCMRZuHnS2Qq6Kbxvz@c&T(p-@K${SNYZG zDWLDMUu-Lh&MMG#O)<@7`P22_D=s?nOZ~UpoRvN)6r1ykE@y88jh z&HkgUNHYmQPy@z;fnYt$bJ4`vISv$pPcAOTI?Mt75F{P(fm-+rKQQD#OQ4c)M#K5; zkB=7DLSNaD!bXCy@*`6N-aK^@Z%BVDEZkJX9kF)(@65h%(>V@(s2oeKfN$7gGe&@a z%FPQ}T1L9K4zbg2>p^6^fb_Cj9lb65?0x!XaY=tHxQsOUfLll;+KLGBK`8pkV|?Y0 zlfFG9mmNF17h(6+_(~ zIbuP)k4Gug3&QSa&z=FSCmqJ1lj2D;-<~XdP#=(10l~$^Cnyv@0!=BI9YEVd0LMcZlDQDjS)@0zAnyTyCrJYcC3w6EfS~iVGy*)&4ri|uwjxA)W5o4U`4<`-{S584iyF&ng?$A0~IXXb{eWXA?ayBb_3ylb+ZQTpt2=Hl@)X*s+*GKs6PK2#@2=Agmzm#ah&YV#4Q zmQeue32P!V2RoD&hDU%G)=X+Y8s{*lrLFC@+=&d}G%Y>7yK{N9+93m=Du6zaSpYFd z$ZU5FJ8%%HbpR2mrg)_ADTxkjFK|ZJ;LL?h>}aO!O-GVDuhOeVD5r zCBcuW!uR6U6;66GZUV=XL@Oe#qtda(fxCrwWN(oz5}C{L^GlFIRbO2{m%pJ&3?xdG z(|!4LkR}JtZv}P`z^fc^)qoQYYH}W;wf~$D(qVo@77!-qxc~Pfh3HQlNBPr{f$bwP ze*1m;1+S-t)evec=(Ey1#!;s~y+hV^Y(2{f3}7q>GVbUj=zsK(I&ktWbV*4KvO%QR z+1a_=tuBqjma6&9$vBa@oKd|$j9FMh0gUOs@#vBk{QszX%c!`TC|x)qBtUR?cXwz! zL4&)yhd^+5celm~AwX~o?(VL^p^@M;jWqss-nnzfzCU-ZyY8&1A8`6~ojO(3C3`>n zsl8pW_Hrr&jD|;Y9Fw#5X~^l}3&VJR;asn()SmfUx{NJgkXkecOM!AH6&m&g#u(u1RTdKn5C=Zd;-}M)knw!N(n5vA06Iy#Y z$Td55?a%V$5|K*HR$@kx88tPIH<^TlMC|R2GmJU}W2eEG4oNVEPUjR%Xvw$eJcrrBVW6PF`;+AN zS%6;Rf+mm9gs4%H!nC~qAO)P0Rw_yGqwUoK$3(B+kM?zAlR2(UF0ES(f`oH!VQ-G ziqb<)u*DYEBL*-gFCsiFiot*r32YNSP!lHmUr=X^?jaAXyL5CuwU$ML{!!5XDjbtB zL9Y?vKb6e=$jCCXdKiq0`ybM{D2yCr{_^|;10rE&=6BXol89mAscC7d zs;WV-83da{Rmxed1DzB?E%kcfaapad4RWUTIQBLVn!R=g1!Zq&%d5(+6m z^KU-PLI`$5s|U%h0H$O%TEhPS)KU{IL0>2b3|anP?0}r;ArLI2gfmR|gsry!T4V!9 z!h?!{*Y#T-FURN9HclM2T@^$^ycVtkcStxkklc*gy&VlN`tF-dR_nEfSOy-X^^Y15OVukiOicd^>aJOr_7@&o~(f+8;9nCez z+j2)V4s(z7D`Us)3+Bk7IUdqC=+B#yx!m8j_4NK=E1Y8nw)|>7`?E>s*9+q7fVWjg zn+as3M;J%2%~rjTH9oB3navtyMw z6JUrs4g=-a2jRhIhy)rSFr6QAmKYB&0EoLjw;p^mbrDu5aGTJc<|2~Arqa7~E<6qq zVgfI(H@TRmA^JzbiK?Y{jKrv?JMo}@KW;QN>FVUEz=+-^css0omz;#&+2>CWFLwm# zhPE|Ru)!e=>DDE8SzdeR%DNv+78k%`9d_R+$AFM4>X>66xnmJ)s4{q-k!jabbF`0)7?MtFeDVg!GL!DJ6Dw&(M)2^r}k zFzcgC2A4m4Tlr?M#FAm?&_FoFbbR%T`+Oa z=NbWp;EkSi`(ni9qi+Aqk801oLu-NA6aPH&=^&1#5V06Iqf0b;NKYGgExwV=i@fa? z&tv5E>2BCrg3IoA;+BImHVu^h+p+brP$;KwGayNSXngnno{~U`5f;fdJ64ZqbUCRB zbC2UOabmrfI3c#jZ%qEm0dK^q5fI_!Al%PP%*^ErRngOq+h_6L*zj`g5F$H9q#Cw) zMZfpOU~ntK;k&&C2ilvBH|1X21DW$@0q7z}z-^xayKIQ4c^A^o2!f9)XIu&=!12*l%C!f@z9MtIFDlIIg0GYVi_zQPQ77D%fhGHiLN&HjmN58e1khepuz zoTtpuENX?Sfp;}X^XT^5b60QUj`N-A2C+89?!80M`;Ud ztu4s009%oo9-i<=L?lSjcOyXEN3KqJ#R_Yv&O7)9<)^Z5aTk2ITrPqYcO) zrC7EEumPCT?L)#{Ec<3d-?eatV_zJLL?yckTe zDJdxdsPqem-4~Sn>ZP8zQ(~OJ{o8eUATm>8W^;zqRQ&#GO}W?YKu;>V(`9Zu4&7ka zr=yl8@C9$2Qx;1n$rD_)e*>X-2uF^_O^(tU_jmG!`p(^S1Y_EI#NeE8kocI!_n~f? z<_aT}f*yO60JQw6h`#L)Sk~Z@nnf%`cz}yhp$ev$M0G=YL`r$pZ%uT`wOZD|Q_+ z1a00u*rDK#Nt#yx4dum5$f_bST>ONMuD_~gNaSeW)5lSLH zbB#(YXwPx%DVFM+`*UBVG|Dg}ehrztd20u?LMoWqgY}Zl?=p1 zs7OW`1xZAuNd*d+k&DbTI8lGo>W`GAKo6l@X0r~JM9w4bC0YgO*kQ5mWbV$l!-b#U z75zdYC>d_{_M!RDB0b?sEt-9w4cuEHFWcccI|tqjq?gY~ZDI#O^!D_}ljsD~TtxJV zO%9|EonXG_Vt($kDy>kw6LWRB_cAd%k*KJN`I&=Bqo-b`l450&Z=JF}P_`WmK>Ei! z*0|l@SB_ap{u4|t2HOP=p8owp{SBm}`nUXaKergNyNu+O8jz#lZtSv@ zQ+m{!D^Ye&?}8D|+Vi#BsR>q!Fe%?LjF>)EM72LF3O*TSiN`o6c6kYosTX{u?X-^h zeRG)I-tmPoo{1N%=TF1$#5L9RD{xm@TV2KMPHJ%|4>yG%B~g5eey4?Ia%kM+d4$`1 z6VyU~@1pc0WNmkfkn`~k8_6@?)PhItX^OR1-3!549+aYTEOcjVt+zfwSTT&O(DUK& z-l96E`-%E&21@ZxNSyxsf?# zTp6EhcSTym8G?^F&>|x%0P_-+E*8qtB7=spUcp=#u`WHp(TUGs;W3GMQxv{+t*;On zGn@gClLb@11hU@6>za(Z+pJUlseXMaOiBU-MdRkned<$ldT59g_Qa(F1 z#Kf{wXDMz`vrr%=UpraeyVG*|kH_IE4V=DOcMhkB*L}W>*Hyv8XAI&{x%aYSo{`N| zVkOuE0y4h?7~qXRd6_R4-M24V-mUNCk^ftO)_BT+{ z{f4ACz)SSj{mZAwm;i86tytJEVRCw1F0>O`k=DRhLpt&sNl!nD&rDB3OI zR0)+_JY~pAebAHDcR_$I{sf5!wB)>NCWnc${*a{wiqHV0)I=xft3xNLPk~5@`FV0$ z16974`(IOcHc2L_uRr4%<~^nsR8k4BvHKyb8#RYKP^~x*1wQjI;rHf$#7a)=*L`H9 z0n5~QA>3Jwyt^SE-MuJ@X*ug9>u-hj-CfRpl12K?(9NmTJ2t4=sDT*HZH}C9IfJZR z(`d(@_a}f+)t=)_Wwbw?o@%*=d`Q{;==E$Ao!jD&-4~uv$Y}FjLHm_q#mg6@2sBN) zw&kfza2?yTmbArYZ)$)mRZf>tPtSF%kE!rpYyAFG`IP16pKVPc$=8YL5*d z=0yZ;GLVLb5=r)xIIO;*|N9cAv%%QZH8ATv%prBnBGtdk5Co8h<*o52Z(C%uV%PjL z4Z?eePS{Htg5Yd^kPd%p+|zq~2v6he&SrUMB!;?r`{U)C5CNA3;*+~RxBqd+TK89}fo2E_9RG1pXWpVMr{%j>k&{87l-zxL(>h<4s`O@VWCOul?k3=xqjg?N6R2N|q>;|U1sjTM118Ta%lUzI2gAU@|#WhP`j2Ycn1ivkL8O(an8us*Znh{C@c&pKI` zWWeoKME=}`8MKA4OAm`wwYZ;J=#t$Eh)uafWh*w9=X}bo9^T1FcE#&>hI?iv+iuK@ zLqZ7d9i9F$7bfYX))$XH+l)dVI+&+?{LUWjR7^Z5-<*Xo%J_2SlIni_7Qi~{B<|Tq z_N%fVJ#y{MaYh4I%%~>Sovgr!f)X^Nh7KQNA^?wJ+~N*D2GPNvFpf$3<7Zt#*U!q( zv#E>@UhGF*Uv68nL$@|3wvqv*QXvN=V?`qiOUg)_puBi^Ce&GL^KD=~ll2~R?*1+z^fV=TDUWtRuXmCl!vEEl zI(?NkzML`g!k&|a`$rIJb9S^As)GMl`=cUeFacdoXxKY5YX0s?bwCFn)ZmaPZht71 z_q@*ZX=$btqwe0Wgm?ug;vD22ZzX6vB_EgObB4)sFQtGE#NO|oSsz}WrhZusgHWTE zSbWv*wf9hD6Jd{cBWKxTum%$Yf`xybdE;BoHr9i>G5gO|p3K;)^Lut>^AnRn2SE6+ z@R=$p(niJn-mE8qM=jp0rUHo>{PwFeZ|6I&w+Pzco5)hhb?~dccMU&qNo%UeHq z$cK+Elp6%WbM)M#^Ji#wb_o}`=31IZI!M@$s(4>zSn=ESl~~DY%cI0>pO8wEH;*-2 zNbV*cP9Vm1~L(I`6vvj&e6&YxVATqgj}2P*+fzvJz7d3sPFxovRiY%^yAp8eKV z`nm;Q0)^X<}C{M zbG@E5sqn#ua@AUxILo9B0_0B(={}yqYVfRo*)+AAhi~mN>XtCzBE`!;M5-qmq@9jC z1G$9Uoq!USbJNiec(!jk`Deb%0aUt+57je&ac5C>>|{KTx1GEaPEsTgcII&FgJK4# z^5u7NCqJdxf4kMh$2i={Akl)!n*r$q16q}htDhCR@K$g zFQ%GEJ4+a%`Lwgx(uv3y2esxCMBmGJ3zclp#fH2gIs(>c5ulan4lN>3`h~-#u9!Iy zcZ}eLC;!+_baY5Q&MsZOQ$+SV?Rm=C2_)0Ka&@8iTK4jmCcL#)GDrt*4j(vPu+-9| z8+;Wu)B05%)W5Iwc*;NS;(G>}Sg`sVBiWZ=n?u!c?Vcm)i|w{%-7A*?w-RWYhXasX zH!2hWbdMpN-fqZ2pcvC5j_gB9Gh!$9uyxM`5y%;9yI?e#K% zsbJicO}%RL-C+lA#W;Xjq`E>L@rCwI-^;R}#$f%-(s2x6ccb@ww_7sbx&MAvYus#o z()H(bV75K+hX}nQXf0MUUWd%%5HsVdIn7_lV^i@y{q49g(P*N6Rd>Ugu9iQPm7n+Y z6A%5~*>iFe9m{w3{2)M=Z;p*EpkRHv_UbEMpX_V2D&y`|cE1l?Lz3x*Ppo~-EwZ{w zj2rc-2gdoMRlnc>C4?dO*F~0={F}eKBK0Ezf*XRFfS2=cI!cKMdl^UMH?~Tnm;r25 zW!MQ_Ycb9_gaKzCtrWe(Qb$DyJT27?bcHf_z^uL#2tX}8A zzBPcUj@Y$B_s;eFgDc02o0O2)!3cOxcI-yHl23TL3m<(07 z%kw)|Uo_OnZd{;(dZowut?J|w%H3{UbA$)Tf?}G|wh?G>0aXxvNaF+|H!V9;P^5X0-IKZNvI5 z%DtHQEvEMr37_U|E83k&v^CK8WFA}W?svG|2Y9Xf!!+-F5D#{$nlOL8xb3t5mVU}p zMDnuYNSgbvspUg8;kp$$Zc2JxoJX=LY z@|flG45nM>0ZZ%=Veb8UWx;-sQCEV+Zn>iEf8uZoTh$4{GXitHX1BY*|6ldB{YwnZRKz%IY!8+cgRUuovg{I$_7SK z;9G;mhI$BLKB@k*P`?8(M=>=u?H?G}c5e9(B0xXqUwJ;ru5MO>0T)m469!Bdc>kWV z7l4T411=4$=rb~ickY)Yf|V$c6WSc&gzp2EgGIl_NLF!_sygx$!PnwEd3L8HSMZIq zmKGjqE|rJ5JL-XD`R@}u{=F{qx3OSjr_XPv=rt|Znw?^a`Qs6UBEb87Q#FEp&IU8v zG**f}38v@OC7{6VSmGxd?ZFePWeC_#ksbcgqKKx}{peGkrx!h`Oq}^zv#Kk@1MAcV ze>|#8N8}u%N^cOU?Q&*vl=9Zt$^?w6*{_aZ=Jk1=0LCAg9B@){u6G#`XpZTh$GAr$ zA8H0lb6<$#pT`CO;}DP;mVP4lk)6qOFeMrAdD4K$h{dyyl2np=_8W+Q1NZit3)}Fx zFP~L$mQE)AIDrVboe`7THg`9yHx5)f$uCkebB4=g3%#z@(L)AdRpV|l-;dt`H)$t2 z2)z0JlL5%mL~LwC)Rvt5X$2^!jr8}aH-DJ3%PZF*r z8(7cO1m~q+d@;NwdUd!S)A`2!p#Eznlx2&VQsjZ*jxPVn$0H>*RpDqx_;CkbS%*+u z*c1o%;VzF>#R80fU-MWOxH%TItJvH}ysf&5) zROzJ_0YHI~TpH+c6k=ZNHkk;YriLk)m_pB6eri;{6S!L)o(#EwJUDyQi;KRPaBbQt znk`sk>yWQ!u>o@XO5U?mJn$I1>7?aX`v+Zd_ zFdOu3M`K_33FB5NVQU0vOe-pyx6KbF`};@q&aY6pZm+4`ao;3_{tfrp#p zBQCdSpu&a+Bxe1DRWh>4hPZOI;y#ldQ2zHN%LAMj1~*bd4bb1=TC%F!R!geGHGvwd3JCYOZHuej_ED;cm~_A*}k|7 z-5ROob>(AeO=W((4hOn`!%mx@s$u}iJh|75{xEBrqWS7Qy64GR+i?4Z*A58e<=-te z?zykEanF>7g}kw5u{iGH0nd)@;PmM&Ro)2E;}t##tV}!yZ%8}?-m06+*bHdR6+v% zbNV}y@$o@xff(Z+?d~&D;^lU+?DMyu);rTHg%VWYhVkkyuD4^2Snm@^hQ2v|2Z|Qv-^iY-?PZ%o#0=u_ zL=IKBqejT54n8OC9sK+exDaENNVFbnwXg#yvUmV02oY^X`*+xQXnF`PqL^Q<#X$iv z#tD)}X1y=1;qOVJJWew+S) zEvn*e#jDQ^`8E!|{b%j3UaDw>avfkNd44IOewSE#Gz{%u+{r4>s=qG0)y>T5alln; z5p7s|%d+A5YE1UEF;b}ymaKvT4W|>m&n@|Y?Oi;-ScG32_P^Z2M@ZP+!55ptRcs^n z#%$k_5H(DXRDP`Z(1YntOP8FH@yX1L>Y*YGIN=|9=ca%jIJ*8uL_7bC0_chUcnz=7rf>-#F8ZnNQOcsxf4$Rl?yPlw)i7)W+}nKs}YFZkB;Nw z2=eATo1p6E3=;2*?*cxQoENT|vtyEr<@xicTA#NTeqf3jES?q-&4WTcjXFz6V~ofK zBaZ7;?zDu3U|uZ0U`WMONf10;0$LX#7!1O_>8P8@$)^aCs*BQtH|=3}d8>rjbR?gXL@E?L!`kR*Ak!}9Q2{qbUr3>Wc~j#Bq349!@Tlw+%9Goc zcCusJG-3z`?L*e_wl=E(Ec#964G8YsTsfpyzWSdb?S8>+Ka6b0{MwH5n0~Fdkb4wt z@y~j|BX2C>-XeYFcVXk5ub(*zT$@~mmbBEKwuEk3f=a3{=b@5<8`mh+aZmKDA&cMy z{7qA3Eu_Wlp|eU`ue;g@Cs$&b$2kud*~&Tql+^jA?&)*j#q#H1D9{AmNp;WZTPT)_dmp8{AmM(HScY;^vexz|_{aaC`{R`_pPP#ogndPqs4>1haneU7cdcwP6x2wx2Pcyo!cL0}BsX9Mr6 z=J0Q*-Q`?E4;sO2FBulo)WAebaQ;_P{X9Uelb-1&CN)r&bSM~~NL9oY+?Iim~( zU?jguMa#RB5rsVA%%V(Oo0)bbHe)h4xM!^BnB)vGTe_lZzj#LV;*0%1AEBazBeupx zI|!FIegkdiY=v6YtDlV)LbhE8WEwe7`=mnMn$zXK^k0_C_Irvm5btjb4VrM6gw=df z?N4`xNDOX&s6$L-kC}e|HR|T8#fD6Z?l0aLZ(icOj@lkgY_FH6{8%+}I&(xy03|?F9P% zMk(^d#KaFUH}dLYj~Tlp{<+ha2L5giE4)?Wo*KGYd=82r3F76PtUmO;Gu_FIsCvf+ z!b$9v{_h>b~*Y449j~{5_?0sX2vj%p+o{?Yu zU^3Fl7KD&{S^HH_QbY58?O5p+cnjQoalal)B4;(UA>kRrr7H|E4_>)0wmudNjqtii zbr>(YF2g44mo3tFeEC+Tfr#FxUdk!LUKR^Ewxmh>(6bR1>tl}GLFSpow;=2(Y-tl9 z@6fJq1s=$f(DBz>iRbkd<_!uBg~!6Os$n|hcnvPsIJ==}YK-jzfmQ1L9FDx6SXOI@ z#WtA*+cR(T2`8WN{5=08ZOq5InnKSnt%^7THpE9g+QO<1w0l6{t0p-C{rcQ3dO7Q z1b=aU@?~dJf!}B`3b{47-dC!nLLbTr3-)4uG%xGhNtrO&QC*;P=jhL`O*2u| zn#bJd!Q}xYh2g7{_f7#6G)bM2{2SRP%sz2;Z8q5jDBowpJ12=~WVjDc?W*K*A z9Q^r=1Csv^I3%Q$j0Tf8grc1(ZWK{oiwzZ39|ATJq zDd3Xbr(61rx%j8FO1>5N>2K;IesgXz5&s4{fZjjtz6CP){f&n#AwrNe!r3q_Wc*>r zma9DnM>Ct4>`*@Cz2$}H&S&l|yjGr-N%^_PZ4g1<6qBs(Fme9Aqh(iHGj#2{z@-M{$a9keQr}YNa!Yl9IEGv^t}3GW$)@n<9aN zSt%ZtgDo6w6_zxngCt*ukngX$J};F`Icc4X$Gx8+P{L;b^lG~l*LXrOE2Iqtg-%6%+T0>bXj)w4+Ls{^%(>C4&m-kTvDtbcgiI? z`oJy;j6;|%FXUceMlclW{PBS}vxy~WLDq&RvRry@%)a>0l7u?CE*bjgcT)THJi@%! zIa*n$_Wnxtd4J~}K_9vDUd|`6vkaG?E8|kKc7TtML`LtUk-=|Y(Qi5d%2b%fWj==E z2dGk4wROBzCDG1`u|FT5FPPJuny|k@ZPMb=3Q$@F? ztzi0$dGxC*C8k`)*+An|>8g46P2Pj+>yZZinu}}gVKE1W5cab0>;R9{slkqs#;q8F z#nQJ)J&qa1>`l#~Cp%^oq2yegGoguZc9YW14_rTn!lzNwERXH$f@CxFqYKRQziaL> zwA&nexm}gFXPxhj=(E!oyP~$P-Ov~Uq(#vjGKMaa)VcdTHB|0z2f>#_Gu=aAV$RyqO?yssp3JQ5nbV& z;W769u`t37I9tj)@~2+BO1ujO7FO_3in3M^IpR(zE5y?~Hv6QN|I%QJuium8@yS!x z;DWJD!q&0x?qKU3$;`El;S|?#HZI8*;bL35eg@S1Dji7+`|b7;|= zG=~H8P-@L-xqzLJS_EIq0FKp}xuqEfzN=_L28O@h0>F$~BxjedE|nNGOjR;H6~ubH zobD9*-9x5kh9Dk-TjbJxJfN@h>6PP!`GUUD$eTXh#{o%96zFSZSC)=5pDY;XCS`QNW^_v?uNX$S zQ6FbVjQ-~JP_FxY2Qa!j!7}RLO3vRYb)iiV^CK>T#*v^IsH(z)50d_&&Nx{`FLNBI zJ2UfqGpz*Vd{Eef*ci|SRcn1M@PUk`zIS(2!(HKt1>ux1M{?mZJAV!!k{Woe^iwC3 zNON=t_U$K@g|hIi^$Z2R0|-QJ`FcVv9&j|*>15+Sy7-&=laAIaD4@n4Y4ha9@pQ)5 zH!$Vp8%6GQ+gzm0LW)i&{tlz!f?!cbep2|-kgVpY8lNBu!ca?8q$|jgjJ6E}ao=Y< zt))zn+p6>Btx)w>h2?uo77q9bx)^{5|9*>=iRzb;j9{MjyAsaOh31U-ICn0eJq4rHnpaJnP`UR=l1s(5 z+2NtBBm}k`B-Stm89AZO=@5S$7Q|%F(J==#6(h1zp?qZl(S-;FMzc4M{fHI7h+7K= zfn#&Q0^4qFoPYBCuZUB7<)4Nw#zLhm@V)MIu@>JkfFBTb{8jxIF%8(zmFUIl{7i+Z zjqYNt@9!70XHOnvyR&qLvv>>we)w6K3VQCuC8c2GMi?&Cbp=xPh7Mo)DkTsJ7f6^ z&RsMZ;T8Q*BkHXSq=AwfGc0IfU(F5*6w0L1 z_746sFTys-O*Zk_!58u_QJojwWPg@YI@g7gFCNQDtYhkjM_qQ_zwhmFuw(?kLxeAz z>v(zfrIo~Yh)~isQ^(-(x;G@}0|s%l86)xNW_TAz%-YKknZYX^&3$z8muIC7A;W$Y z&R;x;pMar;3`_xUJ~Fzwof;Tw8=9{#zz(3}x@cqipsHwaV9-c1+I(!KC!jkMxD`II zw4!;-V$cndIw6mF!(YQ_ALqqi^1b_j;CX@@rfqLp1j7evF`Bxg=`fd zfUobYB}|>z+EGP^k%arg<5$a;QeMV%#EbfgEbhf`Xu_v+e*sb#YK==>UI)Q?txEH! z@UDx16!Kzm6>+0Ot$oF(WK@x3RqEiuTiqE8y=&pz=daiTjbAnGyLK6kY_tz&{ceJ}Z7;kO$J zD+t$OH26oKLro(OhLBRtHpMkCzP}&4_|0Y^2KOG)_(3Nwa>7XbeHWB}N$ldCr9f`; zRUI>dpasGC8h4-->{dexbuQ1A>KQL>%D{b%($=Av$f_a`DHKZiqmmKU|x9r55RXewXPV8bw)_b;ZrG7#NMn5^%UCH1yj< z?})l&@wyTtFVtb@srB@T#B!zN+)0TB43b$thMtD*60sfzIN`aK1WYcC@fD2IeBt{l z4|I?FQQoRv07y_XMK8)z zR<%KKWIXt)JM#Qo*q@McqVs_;jQ9Tztv7y1l`c z+27<@nV`0rTu;PAB>Xu!@I7wunncvESQ?RW{nrpydvHR|k>Jce=KSb~M?vG7`|e%& zPI9o)>}wBSmd3<4wVbpVV3JlP^}-yMW7gs~iygSmb&L{sx0v_-lw}G2Yh(Xb4tTY(N7)R%2lgZv3@_%ST0gHRg#Ol^CtB zlItcU$4mB9i(JUs)6gh|6v?sg%{+rj{+MsWC}~tE+no3-pg518(5{c905yq~%=ny^ zGjx-_TVD3URj-AhVdb8ejq({c1HMY=19){3zYNlW4fBF^6x?Bov^pO+br(GhqELa6LN~7NPE)mT612 zQl(^>Fh7fBXZYfQ>+onpp_#(sp7FAn-+X^JrQ5pi>3l>_{rLp=(aK)=ouOc+kc^n( zn0M1`y(&>sY}xLQ18#>6rkx>;q3>!sTD`5P*?=-rk7Q~-mBHI(*()MPrgZ}Ou*G_RT3xm-90toiAa&VLdRYuQnJvj5QVgWVXlPMfnuE zS`jhf!>Bw|*V@^aJHj~EG-b{OL)(uxxJ&GwsYSF)N2M;FU#2V45MtoX+c*QD3Nsy^ z!mqI(9P*X=dKU16dv8&%zTK|fa$jtnQS&pfIJZ!P>4!-ww+ZWZ>%#}g)+yfGPr^QyY0w!#U{xj&oTgNl61x%!2iad)9#O{rS0kId>ybSM!Lk`y8FvMdJep^x9l2YUV3>c1|AT){rE_<#Y=Cq4FVb)Sd2;O+k*jExn1qbM z=tTj(voo;xM+@FR-X=1X;Ns#1@{rlc28)1b=G~?1rP+i7f8E*Bhcpi5k9;$dfq96M z9R9hUqdwJ=XiCins>AF9yj*R}%hhB4AJ^=^|AFnqn&eVFk*FVO#KYD?4%F&-0ckei z!1b?s4XGYtIsNI$uQauW=Mx5NISX*v<^Gv(VZ%hL z_!u$w6BN+ra;|8p&+-axk=Se zhcmT!&JfK0raIjeY%y_JcF3(&6tk2l|fD1Lo0J`DyA# z2h0ocDMAZz! zH}uV&FH28lw!r2S*+Ws(OHB=OpX`FBEfUd_4)qAf%9#6wYQk@7G$ILJ1 zODjH|Dgq6%*J(8>CFrEd}} zstfL=YGgLQU!A)A?aKs+x;}K#dz0RNFPMVoKxfgm)4*a-pHv4BW2l#9Z{?2z-vk(K zWs5UGE$_y&g>xs!B4X9+Nr5lVYI>|*!9IEZDVAQM5qi!G#aBnVVVUfmld+mo8Vri} z3g`dI1%T1PdGFvUiNwUs+ywc{rFwwXTt3{ZcNIC0MC87P<7kW%%HLf^E z;uM^QWy1zKLmLCZlo1y^Oyp_k>u|*sH2PU_R+HUkLg+`t(guNYW&ZsR2uZbH?hqPt zao0fefcR40scan4NkaB~;@o|c#ho>Ijk7!hfvxq>9^Uj?d=m-rExnbYPs(qg?$D}n_{5XLFK^D+;1=P|{{#x?1pLG^y!ex05 zEfLcn;Ph)vNja($3r|zZS64PUZg*la)Ey2=-DTZhcpw9%JQ{>QZX49y8Ar`-$Ht5p z-3q2ySAu{HD;K=gYY%`Fd1CIzeSeLEB}yuoE};f<3i;Q=2tyIfLpRJ}dD#Du8$0-n zMiWBh6W!%LM5lGXCBCd{{Tg@A;MsS%{Jl-Q@i>Ld!u+H6aw^~S@(9P6P^AQ#v`BloK%Lf3h^9vWx;DV~2b5vB9_kx%zg0k)%Jgj0r_ek*YDgn7Nu#d}eA zr!zz_8jkHjhqTFWP{Z8foz55jF`2aX&@8u7vpiNf>vZP{qvfm^&q8Uwa$d2Zf2Cil zQjOX|Yu<8_Jmkn&sIuZX_w|iUX38+StiyC}c{PgX4mrEyp3D@56_a9X?qs)1>-7Gv zsRGd*yM6nK`m8#(%|bEm^_PKV#um|ooY|!9sf`|5m-y|Mcc=y0Og=w1AzW+dkKvMev9m9na zg;lMG^5o z_}qKFcHjR7#Y^A^4{gLo2|6sAXt^q-0Pr>Nh43LN>Cqo5Pm^h>@s%@BEN3$!NAg1% zmhQ7dxJVl!c8x*;U&(rR_vH0RBvIf;_crY2>$1Rrrzdm;R|&JEvK)>bu1UM}o@&_~ z$(1~3QT;`+-ah`gZba;1yXp8I?W+>4c{rA%YtnAg=l_qkuYiiO>;5&65JVV|ZU&H$ zmTnOkQbGhN>24&X6se)R26^V3 zefHV=w|{4!XU-O{Ih~jvl7D#r!$^pADQU;RIPNmp5oO2M7Gn2{lpc4k;b#7TbxL%{ zKEZUkmjUrbO-sq*;qN}W65(Dqb9(s7!2Mx#IThAR*a?F2yBRyJH+tj zs^fYMvGY-vQm_-+Cpz9WM=}0VgU&nsQ|+02=(=tS_S}L$DtHD|&bPinV&i82dwGFzZ6pjJg|uFFY?cyZUqQLLP)N$FKjPeE#Q3)Kf{vWND{!>iOw*|&%t?QT3N zrb^$iUFzrwe1R~wVJ|{@@awf|_k9zL(Y7N(r2DXEsx-a5#l_M#(!02|reHaKCG>5~ zbgLfAYi@WhA@2~Lv`JfRB}&Gxw6F$qCV8c6O$R3;!=QmKe_lxAnRJ?tJwC{cu&vH9 zX1t@)80`5zVs+d)kgM|fjb+8B;fo~{lCufJT;t+gde09yWu6pJwWmGaB=>=cq3PHGU zaL^}-cITzdYkgkl4#r1R+LXy??4Rjv ze&Y3=Ga-#+eYrQqJHYztPza}XTY2L8-Zwh^I(xLiLALJXXiVE$+@<}-2L|RA&TG?xsr;GV5y{s5IateNU#mMb zFS%T3$m!SUS?v`?May=zlY(4w~e^0lZmU)Ii_I&LP3wY2H& zssTw~eCrGieRJ0HbRV3ipVI3;NSntqGkd~Y(bX8p@?rfvYv1rg3ZT&%5o;sSdy(SE z_*R7D$YG>YNUI&&r^szYv2WLWGNSawF2P#TYwhei^d*h{2`>k^f)AXl1=9tOH<2GK zUgFfO;m$6aIiA;;9}L^7IIjJ3{88Qd+kDaluZsLXe#vqj9aSVwd*)~?IlL)-4EzQv zsW_G8mR=n%)+1_ziqJXch9n;emX4~X`3x(?MR;X+1{vId^CKnp^4SaFp_1s<6|7^2 z(~ZgMi^iZ%7Hwr1jM*qjAR^pWd#E(Co$2mOYo?jTV(uzCQzHYaT!x<^fuo}96UV#a zgcOr9udAQQSO|Z=s0*sn$YhAbfEQvrgUJlqy6%aCuG(y-wYcTAT{+#;+gZr55zVsuzU)Oog@b zK8ChLdZ5wLpo63NFwgKH?o^Itj<#<_{crals2XA#842^{eZ;tTyt@$M6u7mJ)Pid` zM{sbH1*>wy=(eZoi7D0fNjT_^q(v-15{JWXp_-`WIDjQ$HE1j%oc5gN-pPrN1SUyd zxNRzC9oAx8`+^218(l>Ew$!~jPq7duKTF}UyeI4$*Be)}3xMFZlO*7I9ju`O)tjEZ zBp)X25xyseLtRYeZ%wtq@uZjjZ&Z4pjg^ElZ_@4>yE{I<#pS3+{>WcXYN0WR`dJ)7 zYnvklj$z`H`F10;cc9)zrK!V{fa_&Rk%o0+c^~tm+}UJaj8#`+y=w{ecu^;iCqaT2 zdP-6mYI}PbNOUG+Bu&)@O` zy1nl;OV4X#(<3bvu=+~|$y~f9HW>>+fdomClQyq_t=gm;h9qYUfSTyM`7QL zOd`hnly0~nz?X*^6mL2Bq7zTx!lnAm4)@l*$oX50CDyc?>``w4m@QYx7H6|LFj<#vo!0)KvJhDEEdM*WoJ~ z-Zh-VKe&k_>}Px%ZvkQCABeI?TE|)XFKuNH{HOhC9h-2(KRZy8KN@4ua4#c~>{Jf4 zTofQ`X$H4<2eY!k=c++7O!rRQeS#N-NLuWq;&v9>Zz}Tp*<$)%H|2`tG$F>fL5VmG z^pH$!uHv^WFqZcw1hE@8H-;PdG3d!HcYTUM$B#I)8rL(35iS&LkH*q(FPvmVjk`v9z z*;)HwT#nt-=FYR(0}QLemnS|HIM}0QT>iIFDr$D5#KD3ijL-tZty48_V|m@>D7kM= z!WUspD>0Wi-~OmOxia*fNZi<1CH&gE+q>S85^t8JKR(7k@WMf~^ZCou`g{mOnKEvP zF`b6mn@HljWt8OV{!XaIAK8qy$3?mK(z6rcBTRy5w%@h2@ZDhn_(l8+>aGP)9|RZS zwSB2nildgLAfCi)`I6rosxGOhJKBubNU#u}fKQWD`PP_JLT>KtWbbv<77hWTlIzZ! zC-a_E@74g&1sX=Brz<%*af?4ISU};XBn~#Bj0-J%HV;m~WVFYVoCkaLO6c_T6x7yG zQ zpAu)#aJf6a{&>dI!ouPlv!~3Q^8%^ZJds}v6PCF%_hWLwsd)Y^U`f?j@g@$Nqi-idnIz6x{BHW5d*< zYy^z-2s##xDI9D`z%>qQYYUeVObcGagJ(Iu2waAgjbSu@rkA#LEn{PIrT1H|);hn2 zZv*z|f(;j>vtAo9wL;0l>AW(N`0xi;5D+dPA+?bZz9Q}ou8)+7-jJJ<;V;V5pGlP^ zOYW%GstYr;Y~%FMT9r}znbv7pY=#=*!?iY*6v; zj!lEgbmNkWp^iKCJ)peJfWQ(sLwK{iKMxkmc&=-tRgI4wer2z9egm92A0Ll8SXmMe ztX01UW0q_{;L;}s)}tkqTN4$F5((ZsDU|4U$l8wYeX0XT_JT8B_Z9 z%D{#!&IN!6GeF>|k}#m~FaZo^$`UTAs-Zv=#!7$mM+#@mCNx2%w~OKpmsoHbCg** zW7YLo3`Ntn%6t_tphxtAN+BwhE`s0f)~brz!-CQzW(p8v33b?aO(f&GA0IM{JO%a zA9*a58Xy>9*{ZHAptGRxsPn^~;Kxn{2}F;tl|nNQtw+^2%ZcAwI1!BtCRd|-V%F1* z_>`yE#t~#CKP=RHi1l_SG||@dBIRL}W@uU%6vvPwWI?G&yCIk4o*E1XQZ)L`HAQJJ z@9BpLH7Sx$#6e+cWEx80G^k1b_o9#|0zLn_#AR$Kjj3w`WP2|ao4^;8fk1B-hIvjc zCT=j+U*Iq^iFQ6%{oL}n%(fN6Sbxgbb9+n)-=L8PJg<2kjZq>R&4OFEck-$wHtci< zz4twzq1Ko6;qkQNy6Y~2wN!A%I{)jTVy|MzuTDrRf<(Gz*~JnH+V#6n_D^+c<}aeC zw$_cx(=7+*;uLBM^jNG{6_=J(66aYP2|m4?Hp)ZtkY0|SRTn6*=K;n4z8o=82J!1U zCs8I!#|Wd*a~k;iRbWUX?AO6fkB&POV?%xT-7W`EdQkt_6|Jn~6;o=37Og z;E2r*z0u<@W-50)@2(upY!ae#iFvM2`K>E`pBBPU|BpQgw6XnmuLuXpu=8SuIG$;= zm~3`BjoU^`_7C30D5+V|Rnlsx#Sz_8BQh6GT>|G~1M(FY-Bc0hl$R>!G$a{z{~=iw z<0D;H;z--no7(CNQw0-*jy0vp|A6 z*Q zjJKA2tsTF;Y%>3{*C+Q+So|QIvVRj-W??PG;#z2+jy@delBrZFgyI}@$d3PR0 zh`kBr5J{gHA=9r}B%FS7|TcCVs( z5s5w)$7;4Vusby?+mc>_R*bXY z)Wy>&=N_f{kLZoaCrAFjCXDW-&)AF}{d#9R|RgI-s)<1j@Z|A!$ zA@mE1e*F`Kee*X^(mxG*s4)8E6B(TWe^p zyRX%;RM^8j@iFGHDMOARZu~ck0Xppa#)k_J<4SJw%ipRpGhSyL-z; zQfF577Ja_UH+&x1h# zW4|u}FRkAmMp39{IZ(%ZN6b5npw6e=vekYiE2GasgFgLSX+x{bj4JbDnM{ntQk@x^ zp&f6mW& za(^5PuL^&;F{<{Y=wU?`kuY4+b-F_%RXuDcNaG_V1fr|>&d;=lBu}>YE|Bk}A|Ql8 zeb&%=Fnetx3pekh0gHb3LS*LRB_hMnww~#@-i3dQ#}?L&5nSEPLLB9%YaeuxTz4U}1<)75|E541nl1{a&gy0fZvlD;j?@nAe-kh%RJ!cM*` z5&Qladd^3^FDdQL+7-oO`45ZzK-r*wua5cl-(y~sTlsd* z%PifU8TI&4Y+M!_FO@Fyu#1d8==4-faS&MlpxgnE*q-9Fc1RM}C68IZfn?dvlX7MHPvy@C z8@2DHS@j7NtLTh0c+qD!b<=ZjMAp|=cnPdquGqf)#<1w&Jt4IpVq`M%Ua3Sq{ykG= zHsfPO-(#K?HeDD@M5<5dPGQDN-mO&AGN#blhM!z>_uuB*dqkXicq}b1m;Go!kd}WU zGf(%V)5Zlp9`yWjVcgx))^eC8UNnApjlQ+|yIZ!F2G0D?8m~{|%l>wha=A(M^&6vZH^tlQ62+3ZKUHY(0!KN3Qz^d~MXD^Y!_GulZs2_R#W& zRQr6DSdCqyA)ARnI~l&^UOBzXodG7x^KTXB7kR1o&4u+mzOSj8sCOwBNNw0{b@s_i z7nm?U%U24r8&ax5aVbpc7|WS#0=Qry$z~y8sIjnHmU2pS22RZTA-XCk!RTwX{#@vR)QPUx}E^Xn2Iy-8G z+&taA@8lHf^l-`>{$ZO!F7~QH1>t`TKd{44hEXvI?`VA(Mci*<`dynnM#&SFoGoiS z?IV2IdN*5HESZw62UNj8#Z;ergL%zSv<^Sl9FC`dTNUc%99=O@0RML|`Axq6zJ%-j zS3E|w^7nG(`j46Zk9YqEZvTJZN!jnFPi*%``pkNTGrui_u?p0pRxDQrtk1j1?IX>* z*1hDq-<|C-eQWe#M6Y8@7Kq_*y#FqkeR1v2vjDoGvFr3cCEc(*?M6^Tpg~0 zjL9IM)k?->z1u`h{j59vbWLdA$UiR*HHsEw*kt`-(eE5hxc?&sA9A&VIMZ~7$Vad# zxL=>N-T3MvBufCuBOCWEy#BC-KFq-KYne5%(=hj?bF)Aso4*9K=| zWX+sz3X|gE_G%vFnpGchn3ttUpO4#v{7QS3Gj;aG5h~L#pShQsww1jHY3Z6AB5#3%?-WOb=F?Lu$&GC3#wy||gxvu)x<5!`pVhf+jU}+a z8RM3@ye9~WN;+GWilypGybJ7$mk#-7Oh?`u?I(4zW{-X=QSqG^a$v4<>ge&R|0m2z zv$Z5`w2XQA$y>fAEKXr8?eS4Z0oW}i;PuX-u=4hepe301PW(jToM49B1QAd%;ev5!6 zaffOd9b?l^!ct{@czT+Q5sC5Pe!vn2UsqR`w9QSX7&&<>7JQv+k&hRzx@M`8r9>Wt z^YUljrSg1@$hY}Ns*AtgeKxb^aDFwZLR(Hrsf7s0gxeV97tiBy5a9TMCAhqfbE>07 zo3%@^i$Sp44bo{Br3m#UWvzOn-f-`tcA5LE$$n1>3q0-AU*Gh4+dNZRiJ0#}T8q&8 zSoMc3xL7c4lc%hv?SUd8BOE64BO2d|@(mBtKfmVlJd8}bp%W-17Zb08}pq-s^=qFG&IhM64ODBMAu`))R4q)oYbug)Luh+h@f z2@98dS6C%fu-G{@>)Ey3vEf7y!ezuoi}}JZ&+b;!gRQ1tqk!v#1(mpg+sF5S1c&4% z`c$|hqS65o!)7};GS27ilR5a|30tV1wq#{g+RniXP7-~MEY5*`V{zZGvKwKd2P7BS z7gL@HrLanW-p{ViORBQ5xNlMriABMy&QMr~$e_;7Qnv3ojFIES!0X~F4aR+iWAO8! zY0mBHm^1U_>+%rn-us6%FZh`k_DBQ@hD7e|jK?Z4VsVk+pucSf{oy{o{`-w;JnUx% zr7_>I%+0e$L%GjK90u;m9{f1IiV*)9dw0+G>J)pb(lW&3sX{Md9`g#**w0NORPkd_ zfbzRrgV)@&{bFYc3HsZ422EyVuFtmEI+ZqNlXE0I0nF%CwuYe6+K;>8Pd>b%mvsKD zXn4kJdNlXkQVIESZ-v^439)N19=M|;6`-|7^wV^3aOZ%Je(rwfU z1_I6A2hc5^(s|yVn2z;A z-L>#3b}te6cRe>o-(3~{{r0;VU7KxChPwk=F_n+4@5hC>b2#)-)LhDGD$lmwIsTz; z#`H9ggd(K_oa};on;lbNp|>1diq&J(JXB3@?~ zI7rQRnhSfLKGMF2P|H?4mYZ}XYe5LBtGxM`RiA_ z5$#l}qbj6XmE*#G$%nP;bqv}7^Q)&ak zU<7BE8Q;rKpH%cT_Fr{o6TDPo_t@V#sVWldDlZ~dCz+AiEO6UL4kliw4NBi9n>pLY z61zHn#_iorM}ugnvRSQ~J+Djq#Er$aQtbPaPt0q1^`vu*{(}Yc!B~O}y9a_?Yk-JB z3Y&c9gptp8iKXPc7mrmfqVJ+ycNfvrml6nk=3lk6zGHSeT~54eHw>Z-Y<@9@4RhZC zTEZxMu_)uGW7piY{?uZa#TCo%H4Ucwsr*;xb2vo}+im$6MaHfYk#P0Ihj?v+Y}zkz z5FhkmjJVR}AH>Wa7KdYz#lqASl;^e*GukH}%9gu&5Gxfje(TFvt7l5Kn%iCEn0rb- z@|EP8WZY54y$_r61fdVVmbox&^;srRJB!Ft5aq8gf@R&b_!p(6R=&2oe6z%XI!;{}&ep^Q=8 z>Wq@S`_6RXY{F)gtl9VE%LKYi5}p$4cc*e_J@QMBv1Jul*=45_P`jh8Ts;?_)=(S& zT1b|=F1SLLoi>Di;w#e8yguX(f`f_bwzRS)pSLIFDL5+fXPH+>eYTx~{TaNWJ^+>eO`Ps_!YN6rFh5c?R)1Z=hZtX%=}| zVZO7_I6;0gYF)ZK0sfZGI*Mdo?s}eV+NKs_dNCum41UUrRpll&U9~(rT{pi>VCeS= zzdGO$H~ZrrKpyu0mZyb~N%urZ-xoRG_C`*fe^r0iDuTo_B;1&xAk6vraMswOZ=-st z6P*T`f<4;-dvJI6uvDy<4Ub(g^=ePn{A-J{g0csJy_Tlwpt??SaZB$U!8!Q;E>ghv z<`z84;*5p8VV5nVVZMBd-|r7@?Q#?U#|Ch@koa%}riU``j5LXY+B2%@4)4fBfuo^p z_<_RS@&_F~({o}KZe28p#ubMK1$_K+5hTY#jR#z`cyV<94;0p8`zZ-E5g*W&^pQ~c!0U6ejXD2Kzu$U zGtx;*g}5u&++}jm21`9>fy)*C*{z(NY_ajdqrWY3*#C|&z_jP_^61>ICK0D6P1EfY zRSgLhO0ae5K;+8({?(JGPtmD<(q^95HNCSRK23A`DZj88XBQS5COe+q`bXH4ej1cB z{p%(aS+{jgI>dbfRjziG6AXTu*IzwkiV7AA8B*fmQ=dQ0f-Z>WKsGhf7tCj`Sg!U9 z`}YopcNQ{zu9SXMK5l+inA(-d9A6~S$&M6EsNgmH({6ozz+dqLABEq)YrACZd#u{% zJxb)WYCOC2$@sY(6WtI4`J|_eoArgl=uymEG8@h@}@7c;(LU|zH>XLyE}~uxyb@o z$Xh0_Gzn5Ckd*^4X}$1wvfGR1^Fl}=q42+kwNLa{ktDbeme23swVP^zH+c&ZQ;av4 zTIjY}BZX7n2sd{3^%K6nooat2bi8K5F6zv{B`@OI6aKxIe-GKC{EnfUilUe`Vlhqs z+k#jLg?MJYk!dIPd6x)_M^xy z#|O*!ll7ofr(N%8gwkGMVf_62YW?Koh@}D9|Zqs_-0ja-)U>tm|FquAv}sR6{J9zT@QW@)hg{Z(MVuhWj(n%g`h zopDzw0tI8Dfa>Z=ptNurOaTq|9Y^GR+Tx43v}|_F8)?oI_kjisXTgVl^+ziKi9xk@ zQxamt1p7VKO&BBw)93F?dXoBMIXVd3p#W%Anhqbd z$Ocsfzr*%>?M(~6KuLL&4h4cr%5Op?Krdh{SW&+R-I@Uf&ozSPPE-^iF%C$>lL~=V z@Hh+L4Nym#->Lv&1ip<&gSbc5gZUxBMf11|k&|orS8IkU;A6D#xA9P@VG=!D2nI%4 z77E3^1&ZD@cLNgFm$eB`Rrk%a9pDB zgO+4&9r<6)DMNy8zyww~d!ySzffl3uog?0+LqS`yZH-ZFmzcmK&XAt{^&vW^$R?k+ z;jtd;MCq?c6En+GzDQbbOnc1-o(UdS&Gd8GQsDJZ?K9hvizSTez9QS=(s!z1vR~FA zC(!lkV*dPC@U^_i&$rg@IUA4}CmZs!BB3LkQQu>9Dsq;x{T5dD!CK9gr|ApPsg3;- zJ7U_G+V#hOjG#*}2)Y7h zorIK>HnYFH4%BM~PLrILmZQNkFg^|gMn(u#WrCk7s;KaAbGK}#q^5!zv}lrFLX{wc zgM+=jyBq7VD~ObWxUI^<(3 zQ(q19!0P!Fl`^h|H*WeqEDw@-W5)D~1=^2n@9xK|==RW0VhAi0TMIl>aIANcS5K{8 zcQ(jfZJAvhw*JIcz};He*AGc$riWjBunG?!A|nU(5I9Tn$CpK6Qji0HFhF&9;WBI6 z+h&P0m<6oy@`Ry}*P(mV5Bd0H-aYxJs3?>ZXPg|?o3FtFuvNR>oeM7Qr&icDH%C)m zUjF094?MK%z=$0owH#n&e$x}|>2p!*J}5ori! zs1_8FCJEeBV~J17$jB=$z60BgbgJnE?ShKmE<$}$098&wvw8XXmKGK?$%^|%z#yQB zJ!POix*yF8+L+y$#k9qM|KJ)-M8zW{^I{1RR+-T{sq$QskMv36^ls3P8!=H5paK+b za5l%8PDGxvTs>Ob<_VFVPfq3?`!kviwWj{jAW3~WHj7E)s;E?v2ze#B^9iR^s=1j< z&6&$L$Ma%8xB_p*j8UH^Z;4#EzcDQ}fIX89B%y&%E7{eiQ`@Z1iTWP-G@f@wa?{ds z5bnk>hl@0K$mUykUi@2~}fD8csfQr@4&6(=t7R03> zK&=`@Pf??n`&W}#s}#|4{+u{itx>>!!Pdn6#GbZAM|~ThC>uV7bSN$_Y*Q&$&Ctlm zucU-M%^5&k{V{M3FOK79EU14^Iy`eMZ!-qx`O?IE2<7AT&O z&E?t8!o|05AUACD>`wMlYb>4PoRI?I-HW46S0`4;SIcFer4co2q2f^2^D^ie_2PVS z_K@0Ez0t)uMV*Mk7mnqOg_NBsSfkc0BF7cd;;Dm9SLJaB{G!S1&BAxZi3yk4rmGH;`w+;R- z`>mzkQ9Wb|TYs$ffz?a}I!m^SDw97R1iMIaclSZn-j@W!_fWyju|dHL|9d%fQPe*L zt7a~tp~|8G$!B(RWvZ^<=gzAPTJ|-<-vrsMcvejzJD0Ax`L0^Y_&7lm8&(zN#hW50 zGv7O|4_p_oK2)uZ6bUGgv|Cx(nUY125<9GaVq76aoUEl+l{pUHk1|^3GI#jN+!L(+ z%%n#TlpgKpkB;PbZew0`SYOAb`)i0GwD3znJY(91h6n*)0v1+(tf;7n2bYeL7R$`f z%?&W&?gDrj8yh>6J7}p^NN8+q3~-npetGEwD%$t;^#LpXg|yR;>F|FAXbqOI$jDgv z+|Jbl_Z>@1OIoT(_4|ZLNlE``7U74>p#Fy_luJxZS&uQ67W2kBfDX_+Fi%kQx@+|S zyaBlUa@Z<?k@1mUnU;x-4bF}z z*ibX!CIv)JoP{j>6L|2V{&~txYIyTLd^n$$UGn5UCm&xRG$kbk{9jmjcr@rmgXPEZ@!;5)+^_M1pp^JL7$krA z0h2grBhwzQ3kkF?S zXQpg+go~*=%NqvO0lnj;OCO9=3vtBz?u`0`*B~>?=3A4p0B6U)M!i+2@Fykj&Pu zWZTr7H0>o2Opdc+p#gnJ@m`{3*8F50dd0pnWt{Sc%dTeM+3S2D)P2=cnKoTCOkG76 z@wGLzt3)j0kclH-m;t)NfK~QZ0%CMMm%`0whsADcqYMV}FOx{|&;&(c`1;_~UaOAQpeK4hn{8X7dG)GJ=8Ej1LXE?PwtuXO z<6y-zb>(xuRjTRkgiL+J;jyN`;U-UK9!uE0z;UsRTw(HG?D{vSCXW9d1^_0zt~Qvz zdX*yePQu!nb;i@Fh9W@fO+*Clr*XCci#TAAjEsz(X63+OE1u^*Fa9<()CL^0;(0jm z0q$gJloQVY%zhbDz?#?t2s3yj{&_LCm>9CN6H`=F6vT`cPH1L74=1ODP }G7lG5 z^OrBzLWv$^sTBfekjqsA_dvM>1!3%o*}(Hb2nSBK=8_@@l>%g-sVNi08@lL!K&!mG z(uEpX6&3Ndy6m#t$vI_Z(SSb;TP-;WW)0j24!d;VQE5M`3=EzqRJBm!e!M&sf(4w0 z7*%I6OBr$is1=lp8oV#5sYwRD2R!Nf=4OsEkUvVgOe+WUmY2r}#$KZv@#f9-SFc_L z%VY!KAwavnwY3FkN?=e>mkBqJ_}dyUP!u4I;0;)`!N8&dDF`iNf(-$ZQ&*(H0!rP3 z?OF*uv|c0y8Kh&Wa|z0Z1tkm%thmd#}>J7kh8L`q^!`Jn@hI_psf0*nn>ZY^x8b^nJ+5h z$7m|JVS|kYZEoUcmS&NsqBHfE_i`TnQhYUs8D`k!Y3(rSa3dtO?1;1U4Z?2z%&paW)|U&xZg=fff$d})lbEMy{zdB!o;)UE0(%xw zq^u>Qw_~4(WBemj!GSkD~ieY!@aShKkA0a3)dP^zbWV@AvVaoKwCu!+&v9_2r4G88Fm z4V8z)fG`0CQ3<)Z$yUeq{By7d%j0w?Sc6J|^nu^Vh^8Nn1O&tZU@ngy-2|aW4~h@M z6bKQVIMHoVC}Rf9%goFSz&$We;J(04!Ogj9?@9^_;VUa<>-EPtxVT`AVPR_vO4H(@ z*-Y2Q1OEZdEiNtsM2d*Xs!H$*&wk!hy7=lD9qobpv zZi|;k6-3R_EaDOnkOTe;2>36WS*%mV&BY}N7EiXewqX$wK-kV@C*Zok5y6n%MIxLd z+(ncD2%ucJ&FdF$G;%W}0o&|SVEM_#$Jg=W2d^OsXywgu25?G08k%@{RpLO}0Sj>U zAJ8NKlYf0{6B}^;e;2*;4Xe){I`o?E7!9u>_LuFaTSN`cWT`w(q-Q)%mp^o)%h8cx zX-y7>9B0(ZJqqj?7Xco>Fl3go+3z%DT?@NKwUa#PG*)IpZ?H7}Mue$YzXku!;?&uk z=7_me##D?+YXC8~E8?jQDalH;iFkilzJR$UKmJZMv78(gha?@$)e$2cKk^u_BPE!r zCro9TSt)V+o4CZz05Lu`<&5Wg%$nIcH1as3AsP*$Pqm{RMJ$ZT5qTU%?HgziR_VO{ z4R-;#DC#_c>w8P-a`rnpi-VU!$SoKqtw;TFZ@z$FOG2|@gg`~m7oe0{xmOI<&-1%P z>(X1g^yH*o10!=MT4C^y649qxO&(3aurtb;>RWW1mNV$YeJE3Pq)ujI{LS-|{y%mJ z9Cj_9{b?OIl*25K2Fu-NUnfTSFYV3Tl)6z%T>e?m?>IvcLbFS?qad<3I7!l3YP6`* z*uyjQl;meSXS{BR-#_mJ+gHM?_NTSdLOF!r@czvZHotI(3#>+=rW6ifUPa|@-N6j7 zAQXtgrEjcK09%AY49&~}0VA52m_X%Hz^_n?2>|O2(b8ZA0-_gKKY;J+*43qIumphO zCU6sOICMRvosNfx2P{cImCC}RVJkqKz~&7wNOuvGw}A*FpX&z&@cpZ%l`4B8b7Bu* za_dl4;?Q;nq$*21U{1i4QNRA>kdk z3mmJpwH5d>Cdv^&0w6h=G|ky&wlTf-d9lnd(5FG&M-qsGgY#r`aPScx8mgP!8Nlx3 zj8MrK*rqZB$PUOK;KfQxT*AT%Lg{%R!3D-dPfrh|G%6zjAq!QF415Mm&fn~d3?xQN zuHKq5(ZEqahx#Y&)ozXbL4}U;N|Qk?5!?U*afZ6a>;_nxWvjNh<1o?sFlXdU-l+de7Ayp03#J6Ux<*NC(!7s{ zlMJ-Q-Bjv3PeS<`PC=UgUNos zOqZLLO^c6#RTdjO@bwx2&@l?^g9Bhj#IWGgWpf!g!!z(81B}r`*?g6OqNXt(7sC`T z2;g$D9o#5q6yXn*`1S9K954tLF3~$>(ma&u>AWA+%~y>^9=d#sM#hM2+?hHn9o;?T zuw8%~Q1q}W*G3&61=G;}+y0_aNeL7$poOFI6R254b&v?sqkx%{l90Rr1_sCu;OqZt z96;{BCW{~1ZeefFl&ttE6eQSwK^xK&xGN_P;Om=U^AyGyizX>yY)lW>gnTYwGhYN{ zdC4CCvM8{`0ITq8uuU+l|9%#%kqcr$LJeRx$Ul&hkr_ET#eg3sCMV(2bbw!gbuo|t zfc_tx6<7t#?8o6Dm#Ap?G1V_*oo`^mc?6834(zK!;Ak%aMgoH2>nje-U5hOcB_~Ku zu}S|+R(3=|mu=f%oNF5!*}Ci?aq(KIP?pIsbv;EKXl&>5GeQK8pXgH+IP>gj; zl)N4*O!7=H&p2Y!YluUm;gU7d9~g2{7O8pjz)h^PeDN{b(7y4M^XOJiVn)AeD8P_MT@xjq5j9rZOvTI6T(xAA$|u?E$)aflBLN~Fh(|bz@r6a;>rtze zRRd-F!ymSsHseni%Az=@k&d#I?UmujyFVP>q~Er=SbKnjZY}6KOef`vcg1DIRQ69s zwsJPtG4HWxZL*Wn&^-iyxvdQZPT5w9Ckz*iUU}>yEgzSMJybvd;7TO&n(C^SPm2dy zjpylje`cXGaa}G8F)js*}so zV3|(?x#Le>r%qrBkb>)go>e@*vm*);Yru940D^(_1i&6JeR1*(fCU12h{|gKgOH|c z29pgU6Nr<}5g=C$32g_AQBhf0tx%)jU?#R_8Kf-$|4MTP6AO%O*@+Ye)>iwcA}?cr zsmXySXy3Hr3WtGk>l|?bkZ%ZPW5DF`wUAh)Q?jzAe6AYzP6Ow94}peg_}QRuG} zAxKVuEfPIFJ&?KJ0;CVbLv{!pWPU(KWAiR5V%CAJ78*)4vL^Sw0u?5&+n8 zPhzYb6oh8taG7ZT=giJ-R`+&n@6CkzGSPZ~HvH=mH)B9}b5AsBhp#$0&N7`&3}-HG+oV~sDCyX6 zSx|syYsSgpl;B2BqBvb2h_yQPd@YA=SKgq0SCO(V5?1PiF}ySNg}>=@e1}EaghssF zDwgiiO01k`q&WvDb@B!5pZK9`reCtm;9o|Oi(PqaTSZ>?2+3tfo)u$%w+)|bKq&M^ zM*;fA2#^TH-k!bcg4gez+j=cB%`S9@TQ0oIuBm{>Go>l0jL?_8jRLw7gcxCfy|)2r z;*AJp=QFp)wv&!$yE@O({KjA)=&i{0M(@rHe%$1paYA0>pWb^~;`K4ygoGhoyEi(Y zLZOnYd)*8=wo=ThR_a&N%&-2bD9S>|n;z)zaG9*xfw|APHFC zBLzVoaLt8ml>#XUSleW!Z8lipSnvXRE`+)tDF_H62|2m(#z?Vrwkoi75d6SK4iwVt zslu0<`{e%D(NS48BGg79biN^9i;Y`Q5Dusvcy#vb*cj;k7yANs0;W(q7^Fy0Fb$ye zm}->*6-d8*YXLi?SimslP)P{DNPqtP`3r9WQRL<42X2!DwvWoofssmq4ogcT!9BIL zwcvzi4VLz?F%ppL1hfK#=U>@xfYxC13AHbw2gVZ~9*){lnw?c42^3gGnZNrEGvRGi z))UMduznP`1sXsKf?h$E51?wAGioa{s!gG9+GS8_;-leBu=mM<+BB}TowTsBT6k6< zWp~E8SpK*TcL2S%cEU3Falt`5#YJnZNp0c)p{a5OZ{O}$-f+sToLJTM_orNeGEQ;t zH}Udw%hMH&DG<*4ix)U6kA#FLEW>Wi^@N4v#1`dg%?wKUr|%o@cw3ltzDI61i==XT&?J+Bk)%*m>!F)+#TO|o+4 zpI*D~`Lu_p`ekrE@yo&ys#+c_*S+Bz*>AJ`lW{*OIZ|M&SG0|Fx>^;w3Z=$wJ}vIt z-NhU{<$l@a!b(|8+H%dR9Tq5?2Ycxg&pnRPs%t|h$ep1&qwf&*)>W&K zcGRt&TR~RZ-k?l$>)CQq`9UPKqZ%o7TgBuEx*c&>OgFQg5bGqdL(1l^e)UG$gz73~ znIe%$?l;!j*`ILl@gyhbeDnPJJfH9PeV+IGeV+5atta0-_wIp29pZQ;wtIB?`fKN! z!>?x0_4|FixISkV7j|R*5NAt;ppX6A;Hl(%!_BQo_LnxmDmh&MQ?+>-g{@(@PfYmi zu6auRE-WV8!x!SxuYxB6dt~6l`@3YsZQLWuT28hlv>k`LO7LbHdRD=cfuI^V-AQf7 z!D;y^%-j$x>{QKkB2o?3c4psTAVjj??}`og7y$o$wIRLpd-IG8n?BLIs$)wahvZgj z?$qC~NBx#dY?| z{WRJS`vM+4Vod+*aYNHK*|)E5|4ZYaE`R-J*PG}1N54LIy2pA!EYxhT{d%9~MXu$m zA4Od~Y+pvD-Ca#K1?H!0mlU7&dSU56mbAQ^qF6I5jr7)HJqw>!C3U3eKzeILb)zyO zXSh=sVKU_6B8=J$+_b20erY5!6g5k8?j{_XL0dmDvZRHF0;t;~pTP42Qu}YY~kb0E(Wvsg%9OInz;hhBTM9!StC)yJ&1C&)uvc zHd{Y2%Qh~sH9HgrU*p|A-8+h1PS29&>cg%w?U7ZfLO;-_tdG=JnG6M{exU?s+`~2{ z8H*R0@N&oBlG68wvrbMGe4T{-h;M9ujx~FVBs@!w5~to?$Zr@>T-)=!ytgqwd_)EG zQInz!Wz_Md@P@fgq4BjIm*>4ZvEQPO-|MD;9(k(yM+^_)>!L;Yogc^~tK0hm$kKB8 za$cvfVGeZDx3?oY>RA$}^5te#n3>2!(G+#Nc2q|16tdhms&q>>gn#z_-cw{9HOWvU zUsio7G=4QndQev8`5{@xeKv@)GVQ4}5IyfYIYs^lhzH;dk4>1GkQJ~M6hOx0MTs3& zpHSHMLPeSv_-x}hbByPpqQv7rEQLqt)oA2uZsc4G>v4-g0NU_exEdFs8R|0(vGm>S zd!Fk*dNP(DSIaS*?Wewknt6)Q&Ain2^l0CVsxorrMt*>|F`f0mm34c>AREy>{dZh% zp?++EvkAH5Qk_OK;HlT~O$>!r8fth4m}O`4nZI6wGeVoFq71K4e;1;}d0WC32T@yg z0GGj*9c*!6o(;F`V2gt-J1{e_Wd~avY?#6S@tj!i^J9R8#ZJE?Q3r`z2EU>0tp)IZ zY2php8Bz^H>y!NT>+;MRtT{5EC##ogv1mqb_x$UqiZRAr-F}v=zNM!JqcS&7dw7uw zh&iQMt`|a35hZvcvy_BhyxpF6$A0J}1W4|13q8<&u6#Ki{O;h0WK<#lGnWD}arh^+VaYMNd)bSedw&epmRFbfPC4Zqzn|8Ug7FDl}a_8g+ zI#POX38{6O>7ekD{x|xeM7Zdd@S!_S>TuGYTKHZm;GsYy2uH0uO?&bNN}iBtNAFLD zpwhJ7EH-@!?Kd6*#^uc%j|_@=|X;|8(Y*h?4CMF!im=JBpD}h>(@w& zZFQZH8!?ntniXshJTX6)X-5O4JIdj2!=>yt5*Y|sds^vfdO$PPxC*CFrJnHQLGRViCPSd&g$e=oyL(JiEoEAP6O10m&-G%}C{#mV+k`bzL70}fJqRP?1V!VS&f zUxY#Oup?jiOKYGZ0Fol((neHw!>A|D+V|g=FVVAssJRoVlZCuoe<_~=${E)l6rRl{ z2aV+iM@*2=3{Y8zKMbI)Ob{s4qa#@ujYZ|2_29N%$aIlTtUP6z#TjYZ7!I)@fA9-7G> z=YKU;Znlh%vE`cdn;{UK`EB4uro_mM-)|iYQOkeF;Qp$o+i2GDSAIYYc zKzisoagddptv~7VD*{u3yP)hS#$V*tGewl z)tpL;s*^;Kt=J{Y-HTxBOx=@y&92&RhLU6LO+=y)kIE+R24ys3SD8f>{O=5fkChDW zt*tt5H|uk#(V`d+SuY^cre+sRb)uzJw(^5(iA}q_D3-q-4@P+$zTdCMvylqa(Tl*d zu&hpu54Yi1#tH?zqImDA*=Q%70PlG4%r?d@%d81f1-^(G-%5|4$|RTh5p)xzVL#+l zkFzD3(8%2BXy;%If*@fjRdB%;HT#Eqk)Joe%k8rPQjIK^KdK}34YL2s~~zubz(8TM||(?OD`cd zwpKQSMuPspM|6wiWNZ;yX(uSPR^N15ORLbwX(?;L(v3=i-2wzeQ)# z!VrvKOoKD0l2g~zjJHv$rFBl#^4S~dWIqBcu_>X*i%@*1Xz(*ul*qH1HC?eQ;S8t_MoTUZc0iGe*pgUDd>96OJGx}BmC)*V ziJ%ks5mFE$@tC9%83wQ=v6^N=V-7+twX?mF*px$jKkNfu-Gdk>Ys3@NatMww3yAJ9 zwp3SZC+A|09cJdpRJ0HN;6?eYpR=O%`O~)xA0q z&|Rxhh9Z>Cmt$<25A7&94<=Piudh<>#j`%`QA<^t#^ytey@W73VKxlv7axx6?2%)V zRswKqCn+DCBvlN?`vFS$)>g^2z6ixKA=});DPS85CHEsSx_11hE?3wIz$BB%t7}pc zW%d}b6h}!MLjDAA)7+?(**ki9Vt`FatJd5YFiH>i_$h(el7l#N6asZ1WW{^g62dTp zM>Q{l+3t>uwfX2pI~#Zz;asK3P;WBoeA(;Et~Lr$p^E**6&FNJpZNMYSJF GJO5vmF!2Kb literal 0 HcmV?d00001 diff --git a/docs/images/quick_5.png b/docs/images/quick_5.png new file mode 100644 index 0000000000000000000000000000000000000000..bf7746440ca7d9ed6729f61cd16e775664a71f88 GIT binary patch literal 112928 zcmZU)Wk4HI*DZ_$2p%95FD?NJ!M#9mcXxNESSb?R-L1Hk;zbG+2~ynJ0>vGQTY(nJ zm*+n3{c+##Uox3FGuda(K5MPLcATcV0zM8U4hjkizLKJ>HVO*HHxv|fAP^IIWT40) z0|kW+MM+j#Hvsiy0IQH@&+72>S&XmxZ28NF(_S*A?horAIXWv`g<_z35_}aJ$DBrp4rLY(c@^B_N2shVEnAhxT4Fw{r&#Nxth4W9)Dg^g6IZu|mCJ zm%jA?90T_Sr(u)j|NbDXh8R3YTi~C&WvhauX`>PUuhX_7l<5F$@$fq5n!5P^jPdAS z_R}nYnAfXwN9$A_{5^UTp~k>-0}Nu46^6!y27wl4AD|0O7gklrqFbJ^H4Qf8qVt(y zvdaJK?ootLdH@Wlt0%YMDojnluOq-8h=j-jnw zk=bLwL<|5z?&(0Bshp`mE)06+z9cYCx)3o?$~%8|8~8%eoE22af*KdKp``6S9f<;w z2CkEcBs4O&y3(vEnA>)izvOe?OkeJ_ay+`*2-K5U@0A0O3ZxGBg zIy0Rw5~K*}qOQo~{Lgn;22jripjrv}L|hmHCQ2`WA!1{d^!KMs&uc(PT8}y~6}Zl! zt&DtTR8%+y{8>HSCtI`Y(wwsR`Q-v*5nFTy*aw|ww_Fj64y4HB05oUBgC#;=QDVo2 zFk!+jO7|0$-n-s@4A|BYc`Wc?REr=K{hu(4LYS|=1P+#hUQFkK;F-I&kKricQYin= z(zrKcjqE@2^^VRYmY|dq!dvGSEWkHgOw;!M!apSsgNeh~OgYsSVpCFBB_xurT6pId z7CeQE4nI%deB~#0K8h&QR2(qh!p3%*Bl;60hj3lCUdEp%4S$ckw9N z&@R{lgcy@mbE(_~FApagWD3+%=4B+($9&qInqp8R0kMT~v*h)D>X<9}L=Mpf(4k_p z4G!i0VhFI7uC`~F5>q@wlL&MClx#lx-247-%sKHUYV-qXNmjC`0}rW#mMp6Z zLwF2;1Jwn7upB*c+{H8i`)F9#{qbG$SwQO8fS{>uhO>E`3MWj65eR^Jrf=dwm|a!*r^Ffl4OM~xN;H_6Y*BL$b8JXZES30 z8Hm;;amJinYL!afUpk8M6*9^(zuUn#?FD}JJFT?x#fk8*cA_j0vKVc)85Qw+6#jn_ zW;41`=B ziY}_FdrqzLQ-nyeR6~Hl(lj#uP_s5{vo{PV5P&uu#1b@47hOtn_6`-I1~^b>&^&#) zt=x#n`hh?JrWbILd6%1;G%m)EPuxA4DK z&PPb0(96soEiOsgR9pR+E^P4ZQe>*5A@-ITvQu{T!*`OhgxS<-(4X*ifeV$Xm6C%Q zf`D>toT=%H*qV`2%5j;6atv}{4_g4x`}OwDPP5oaB3{acy6ZG~3%d~S?B|@LErLSc zB(UHn9fgj(RwO)=7$Ty?fV^~U8C)91pH@YuR6j@;Yrs#>a}VXa0gn1Vk$v!Z;LyHu zAwO?8uwFj(6MsoKU>Rl!9jdJG!X6$T@xEg{hoNmeg>LLTY3i6DCU5wAd*2~f-k!dZ z(Lj}pP+Cz@Qe9nc)xiBdV5PlxC=H>Is;zBckg#!H-P;?1YekN^%EWqGM$qgVvYh)UFx8`b8 z6j2q{_^kBrl$D!Dc$#Wg|C{Fwt4G=C-;IO0FGTC=!G1I!va4*+2V38rtgXOO`E<3t zRg6rH!c3wrPM~TEDbQa#0%MhLIJuP-bowQ~O%7y-JQA7jOj``->D6*V9Yh1%Ey()R zT*tiix#pcjeIuKpw1)Ba<-w=&rihRHZ}5rsuf~nq)2vs(qf`aCZKR%4pT*@xlV|6J zCMs<;#!CIXn1dF%C^2Pv3|13fwDuUzhTM?bgqV}6Jj7Rmx#DFm*qxx?S_&B7p}uhB zBFy4wq#H$_|8^vCkM=R6CygUP?*oUD*9x7m@;ez^5+C|lqoH)>@4HMQfo8eWYhp(i zl$^7!dG);w(S|P~kCu8f)Efs1D)k%680_Q!M3_Yis{|8#a2Z$ny(Cpq{dDLRB=5iz zXr>y>!|@}B7+tpK>11t9HEe-c`A16we>le6tS~+|jdi?`_nvM`1216=3lOivyQVAW z0q9|wU?KZGz0!85~Sq^I{{@Fo9ymdW4Bn-lo7= zCg%%oGv#=a$yv@*HT;{zs6aLA*Hq3baXwBU>YdFk`w<6NVY*>|^iSe#Q%VO}bRGV4 z$$rgLA@2jP;a}n-E5}Y$IN_R#<7TSK@6vxy(-YSjG_%bPG#68>*E>!LRQw4N4(TD2 zq@`BMaxu&XS3Ni~BCh`j=H)19YZG^@2l0)KjTO>Y%C;^ao}JNyaW2hOl$5O2K7Z<5 zAfRzl(?cmMZJlw`ktuefiKl-vuqcgjXrO34g-W1f%vm#Q?8n|u?ql$tjn zaTlBpe-!`sB>83H@Q|ELMHw4s@`XIbPjco`HE9NVcI>MB!cKNLB~gu5xC`I~P5TbT z94I|u+grq_%Nir_A&`z4*q>R*k_Jni+ZfTC>TYt_XMDcLtx0)~%!=`Aw&xlzr^ix^ z1H*loQF5F#Pp{Jt?{*Bp6(BlkoN(ge>h})Zzs5B{-pr+nB(*ij(KL5jnkutN&Q_vR z3vIp60Jy3;(kZe$z9ivNXvA@{mKtpR6eUSVr(0913rv(r-VkhhMa9_ax;nIu34FAQ zOkY1gKH&f~1hkV{kR_&*l~t=SA_Mxc9VOLR8OY^ATbS-K5>@M z{gDjt^|!%+z|Vw|8s>YXlgYjy>`CuZS3#k zLRu?l=lH@xYG-F>!!N-iDdJabDX)JEhxC>BG@n0M_}DHc0H>?8LiRpBE*K|>to!0_u-?W9~=60#%pSQm&Gp`=FL!!?2W4d-y z?m)AJOhcpBU474#R2&>*Gr-H)#*hII)+Vxso%T;hZPY)NuCb#&9}70`8Iy5)c-W4o zR_xYq{`|muv}!Q3RST^=7WDZ}Id=UvnU7}v>qp~(&!k8ElC2RhUxnLo{P4bJVC@Sk zrJ;$c_;u@Ql&-$W(N}865Jy_D=y_a4ZvN8?+&FOj%~$`MUOKN2d1xdd|KBoE{j92a zalc4lj)kfG>j#3UGn+6<8E0dhek5ktBbK(hmP>1Vd+KkkyY z-&%O>63x1kQQe(8T{HAOBnp4qa`+;D z(N+5jd~m+@RGUUGDpaev)a96>y>#I-g=YEUn|@ueiZ7J7CA`(e>cV6fq5UPqW_@hNeN zwm1JYrT42QUA!l2dO^;_*Y%iM+I=tte~^A%mcGLtd)Np*Cu5~f#E!ij$b_{IjjL{H zN&VVvzpkoeI{B-PQ=Z@xXg6Z`Wx-_GF3K9^f5GUW-LCg%&n{C=C>}b5IlfF&&nW!v z^YKMcZ%;TV5-x3QY?4<{u%k@^)6`7W@bhDF??|8#^D8?RJ(z&vG0Xw;tGwvz*f~1S zzr3ZOzPtJ^gSb!NdoU_x8sjpAR@dBX2KMM&ow5>KNjCa=A)@i|@bRyEpC3`RmAT?^ z7VNa58^6U3AxUB?3;E!XFYFs?DR1~ua{AvM_!iDI0{DASR+>2+(u=dVE}&?G;2`c??ZYFV3qwFE7u|-8~t} zi6cR#oET$&Dxf9Yzjt({O@jFD9cE4Z%WC|Xh*O7v5yuO#p=G=;KVB*<>2~UZgk%V7t_)WWmR4A3K|R%vjJj#?)Uyk9IWcct~e$~5(a|bE^;vw&pE9wCfDJp z>ZBL-#pZu!zVXBJdDJBI>yAc(cP3OI7cTu*HnUN~4IcP)#?UyswJm+L)q6HHY#@C= zk*u;hPbGi^d15#c2BSi>^n|djAw*TPc?WLTq*Gb3h^h7^KH9!$q~j^iY(5L(%3L_I z^?D=l=FKN2R@TdJ85%@*m|eR3T;aRBwrAbv=K2lBV;LH+hs#sM$NyFy1&~xZY3u4n zg5cBBTzF;&+TOy{s{soN4@;x&HjC3{;*S^HdHMPKiJ6(;*H>xBT|3h$^vs=WT*y!Z zAxeLe>!dBTgaMQgd=KuHynI@@p#b>dm8@reHl)AzJ0~j(FVsk3wRITs_q*?6uGI= zmh3iP$^ZM;8ah*CWO9#fx9WLU1h9CZw0L{w@_5{5d-Z#}#w1@gVT`5=!@x}b? zUh!p~$f@eB0-LFxwD0@0hINw3UpQt8De0kaNHLV2Ca0Ibva!n@zspNrc7pxkpL%Ds z+xW43_J?{*p0b`6L3@|{uea~QQgR6Qulog?FA}>>wZ%( z=QJXh+P2Aqc(q;&Q+WJOC-(A}Ic=hjot8%^Ds*ju%aEZY#S! zRK>LZb#KJ3b*F@U^x*o^z8&ryFAhh%r6Ru!pKmV69=_AgEmAm_B0+T^R&mZ@2)b)J zT>@H>1e=ZK?($bJL?fJ_Vp|C0rI^l~dT+9r3f?_BTM(kYcbV~vQLOD3o)os4&pBq_uXL0Ifej*OUEPt{UwF^%x}HMhKH-c|H9SH#Yx+ zfnUy2-F8}DY>-5~A4O!oSAMxfu6L}hr0VBANIR5aE~}=8Nh8+2!23D|yG>~N!v1#! zXB|h2W*HHa2ZQdm4-RBn9j1)r`04sOO8LAq(-EV2IB?;%f`92^Y;5dR9%V`Qb$b=} zG0Q)e2igc9Up0C%`19#enBmQ+rncAQ1}_3Nd{wuvqa1%7()W2XzF}66PL{t6pssO_ z&EuhcP3xhOG*-sRn)mR<%{PDRc6LaZn3;82eIYTFl)@I;$~QI1M^G(r)U{qmJ;4J1!@xp!;Jm(4 z3tz&!ICu(^iZ6gqdbW%u&pqRbH!t-5vUQg;-sUO#_Hi>t61IrXCC@fs+c4YDLvR)w z?}VfJMVk3dclJ#M2>5>XC4$x= zo2Q5xfu;j@QkI$*ExmF~!A{Qr(23B(ywfei&7$xtF{8#ra%t(o9n-yCsD=6c?SIs%3tp?Vxe*2(b1|{xvM`k6$+5{Qh;}lF zj3d+MODaDbx=!#a7i(ReRcBR`+=5c#*yxcTDCrlqxOe0&hl`@yjKk|w@(eTeR8urG zkm)-FZ;ww1!Kss6>oT-V>xV^m$x$sS8s4B#&+f z%v{{tq+Ex@%a^xxbhN&2YC=Ik4^NNe7T<7XA-O@j>+)b}M9lpbVW=un@^sC}$Y^VC zXNc_I^Y(5o3(EBQ@v6=0Ow-TJUvLHhFTf_?$&u1H5+~JG$f0oc!y8@ZNv~OHYA=49I_367S>3!df0?EH*k9*D44+kY@ zmO)%Y!^&M(I>CQ-4ASZTl_1K_L~k6Hre|5YPF}m@OMrh!sD0KshWQ)*9PLS z9{?fU3(y6swp~2(Y0UFanp7O*cwe8=+N6ofKl?HSVqc42erx4_Lm+KUK;_Yycbg~u z6Dvo{m3Bwr)@CwB*eM;|#x0S8$|z6;L-L`cT>K`fT#A~i;owyv#qhUXR=)54k<&!s z3{uTv&DZP;0D2HGk|U=K5JBR`IC)%X%X?5@Bf43}FPV%`h+uMb*N<(6)F$PWnw znu9mwO(j>f10$!hsyw#(?DFcgE+Bm(Tc_yEKHLd#=p;O(2`V(QOJYjM|{hUJMAU+9kRuz}C(;J#@_w4k4p@ViAs9QM-^{YsVgH~5$H@@ec z8b;H(TUeXz_XK*{HP;Dd9BmP)YKR(*xdXIVf<7NnQnSL7GACimSEs(TCG;H)spKzY z4)OoD0Q9LesoQkk8|UR1v5#p_@rj4PQ}fFB<0q48+cn{ zEc0L#DiDx~O$gHb)GM6F7vznQ{^ynGHaEKL;+)v|yr}WAAxcv+~x-$g#cR%bdaA>X}y% z5UUNgSNJpm;cfAIyJ9`Q?Y>Z&EwH~gW@Y6A;3we%4-J}zs0s&;lBbJQZdyCR=TUnQ zPaHVQU^DNGK38A0j&kG!E;WPH%0Cu+ktvD{<%fCLquHtI@V?GV+S1nEC~V31!b&sT z<-)8lNtHb9Bd33G*|)jsFWN!>JN3Y%iW3}m-ZE;BUn5kpFje*}J2~NfW06&fnEw_- zFXWMEleIVitxrG5^!L&KwK z`RU5~`Ye6_ZhbgS^hf{T9|NOc_rb5x22chpV;>#@r=zS66`jyIs)8l28jG1{JK;hJdVfH8MKLlQVtRn$bU`MuoR%gjhLbWVtcv0HiTt}QNRybRD$V6isKRrMhqdWjtX~s^ zh?&0lne&~%z9e*1DqN-mCFJR#O>e7}7K~RMxIXrnzC{@MWtx;E36#}DJpE+Zxh!lj zVkidPf2xDiF|yvq-@@K7i@M-2i>>I16U3E;pS<*f;dOoLJL zV8dc^=9L$+Cf>NoA0#6vu1aaw;+sT3JlMHkjHrbk2`^?W#eWmZ-a;2EqL=yhEQRaQ z;0g4V0Mr3XphLP834v|_J{TiLjhPtRr)i4~d2}riO`VHbMrgL21-Bw|=~OVDqea+w zL&@DrbSz;zujqHF6MK%yNf}zAV{HE_Ks_%)h6!1-jn(i?H35$vhNb+EgNvvnt* zdnYe^F%NQ!VU1PuOf5c;W`m!=gOhyGAhP0-}m1D^8JW)ZewgkI`{+x;Jx<*@it=QCr*s~m*ju_zme=Q zZP7ul*$dYN$@=e^X>y!9piN&5A&lH8r*#lEws0)GQsmrd^0?=XQZ-&QwDTWArw)afQ z+HjJAo#(|se^5^|NIAuViTEpRK9j+vkw* zED=$Q$KS1mQ(JL68PC@R!9le}&rcc~?-!{?*Pd@kmCe>3m-|%H%`d(F{o;0Ke4Glk zJ^w5gw@^(UdmrchbLmig0aYp37~zy&bq52E zsnZ2Vk%=y^C(Me~smo#t>Prz$b1m4OEuq+5$9B>6_Cir+z46368ri8BRW1tghs9CzUSt;5P`&U*e(*Hu#fY0s5w0>GyLszB41D=z53e%!brJFm*Eup)W^;bCI7Z*^=SHR{PnTR}NJrSC`lq!I)vL`o<`lOG;4 za&;%(UjW{%6{)KKog?ODYcPiKdq(c30aJ&r)IN;)JUG>RumZs!yOXDyy-q%C%%p0h z8kwskRVR*32mSu!Pn4sjJT>y+;v^th;>pJpT8&<<#&~jjy%LE<=~TeY(ovJ%P>PtW zTGX;C*$J=^dX3(afYXGmpuaMd{~qmnf^#2}zq~j*=?kLv!|z_RU1bf#Mf5}Qq&(X_ z6%`hAb(8!n>Y8n<90Ab`vgJMO{fUyXN3o1Qilly(`6gbPfjvGKjmt)1w61YqKbC5( zGgO7$esWSJ_^}{Xi~chHdgr3ZQ~UEiqiNQeG{}Bakuhck3)|~w*)Phv7jiYBRarx% zwVT`c+GLFt{jaAuMn%UaCh4>Nx8hiNek6rhCj%RCRi_!}3muh~E4nOMb{U603=+Yv zuWOu|fxgi(-4quet?b^gqj!&v%}g+=(d;g}OX8YxVZ{><93Px^5q}*Y8x>iUzDo=* zo#b0xOcMH(pV%gyPuRX$(*-{aU`;wjCm#?g!(?@`-M7ogCNJGu)?(mPw7T6F;Ixir zOfriMYe`!4F$?B=drSoso;T9ZND8$MJ1PZPvI>+Zer8TsS_}c3z$iZ*H&n6tCYrtC ztJuwIKKY3s%W6@vNTK4sY^}V;?91Qt^>T~t_oK0(UJl8rTbWg&j`ciQYY%%Ix(rQc ztFmDsjb%$85)3dIxY_D-Df?V|u{=_KKNe~kvA^=pC`qg{*1qHsT~(g7XLT6k%_fV+ zCP(Cq;KzSEAkYc6ys4NaZ=V4@lZ&n-U|O4T9V>sj%aY>qb|s`)8LObJE9!RxAbKh z%oS4HxDYOTkaO>|g9wmfxV}3R@m<@5DKSN_*}mRj0y~5uD`+SKRElx-HRN@1-};+W zm~ffU1|C4LuwAf@a%83@PFu1~Y}N-l^2kK$%;?@+9UXF$mr|LHFq-q7qoP~!rC{<# z`c86q1|B*Z>dl#Z@bi62E;>4!lp*lZrGe#`BUI&}ZG6TYn?wK*p9)YFg8`9N8xajS zMiqeG%BaCsqk4R{e)F?)Mw@WPK>=wHE46RZ2d1uS^5RRK?3}nNC&zqzIiQp$eMKfx z4ICd=QBqfjExtjz0BS#etVHS^3rMe3dP#|fwKYQw0SyB+L3te|S9Qi^9QMd4YbOXT zP9Z~g=+_cB^$VoTt$U8DM4H1qs^yQmthV6(B-Gx55y{^pi3T!LxA-Bka})%%be)b> zRt!;bFXw?ga1QwaJAe8(OC5n|g+F;Xzln-l!*}OFJ@P;hFtJ8j-<;hGn@9XvR-$90 zK-TFE*WY$pp!_+?|U@kjM zLEq?Jt;htr;hY&#{p(`GhNz#Ylj`Z@l4C=8g*Ak0b~5vTU?z`IHQ3fDq*34L=RfG{ zl9Z0h8IXX2^lhrHY0Q2cLa&Lqqc_c729aT|?yibS%x2`z{jWI2aVa!&av0-&#r%D> zmd$XrP=>8eK#^35`~Iv-U%v}^Plo4g!W2V?onqp>N9tBgnirNlZmnqlT=f8KI zmsM1kLA^*sb`cYLW3;aH6dvSvzK$5y*2_9MaZOJr4rE2s)XN%=$H6nJ?&u(dFq6rA z*Nu-)I6OJD4hYB<@)0ensT*I=keFH)6&3B&tl}3T^J;EJO1wUg0VFy)3qA5-AhZFe zDyR9iwW&A~)|!tWt^Ov{6p9HY@Otbd+`Y0pPU%bZyOz8p+6?D@=2IC0NIq{71_WBh zYXYn@#F<9An?4wEDow&$T<*)o4CXd2=tf7a#?ps-4WVX%4o@UxW3{V74x;kcs@F$_%dotxtev&dO;Fct}}$7;h_ zPokhch>HmwNC{cSJA3&hBmnoh`O;52WbP|oW4ucBZf_lhoTL#}vm*4M~ zYGv-<1;F3J8LCB^dJEn=!K+}ptm5Yf0#$MR%xqzqlfOTC2Wz2`8m!C4&04nS$P{Ie zb3%FK3znAizD5fOX|`z(yAfO$yhG%&RAXb+*2-1HYqb;p5iSUANig``4ni5a*Y{&t z8_uuHQdK`=?1zKQ6pSejDVX3e`CG~#KlVKz(=0rL*$3g5W7ToJx3BfN@@$+9(D9K> zfU$$Ks>U%fL<2u?PTJhr)zVimy1laG{rigev+xMjd`j$PJK=*rM+Jq$fXpR+oR)=H z0_qBmLCBY6bl2le{%)*~qZ3J$7`9acd{p~P8D0PUk_Cl$NW%weQH)mw>Q{Oq_hSgW z^KEaIR8G$G9R?0}=-r=A6pT#y28O=)trsh=|8ZeU-Eo!dEcqg$PO+z_U||n85ZOMY z{$^BEBB$+7D3d`lsi1n3niJQoRTZ(Ry!j@+U|hh(P~UOq==>>~NWCC1gbLd~B=TpM zlRbfdGo9$d)H=egFgFfvK+jwqcEy}`a#XSd&$F_x9vQ3eDNZq9@3bybrClDK-YE(| zgt^bYaU^RXby@aR;i=lTu5jX5TVP??jP**(G=PWxVQzojMaUVhGBt4R5Mlq7l@NZ%=JGwF-HF=j6ieR=vODyKWh^(@@G_UHw?^|3xiCm*>2O zSTCih7*irx<>A@Y+x2uj7JRuOAZPv0yP^A^v-RXh*W>)5BE6|U%p!Pa?+OV@V)P2t z7zMkNKd|pOt~}wHwKng16M!+~fM^S?J@U#G?v~!^b@Zwk?>&ka6-k`P3t4y$Vnlp< z3O*n;y2}(?>U&4=c^K5Tr5`xWCy+IMwQ_FRwN1B9YI*m=%$HDuXsfOBp6RNk=;lmP zdI`#N3*sZ8zqv8Ksg({=c&CezmIP$q{~1XzE(-)`p>`DbBgxSzLDm)OdCz zv?<(w7b3Q}{w40`k&|DC5MS3HXWoU@L&`Ij6W@%?f(RbZ!qQZwkSWY@*Iw~wZ>pVZ zQN#kvW$j{;ZhM~GXwcJMHP%6moby}J}3yXv1bwJV! zoq=1I^rQ7pS%}kxuxO`a>xoYsY{Hk~Y7!y(tSq1deW)zETrH)!WM&TQtS}{=1SBg! zWSj#*|6gZtR0mE>c}<a3q+^$SA* zckmy1QRDpk2bsQpezpJ9&cdZ|u}@wDhtaE0P=To0@Ju=YTGLp54XrFFl?HwY=YJa$-IQo)$+6n^F7Rg%nROKs69m zkjmMZ^6c;VXkm$hVw`wz=4P^lV;aaPu6b4aSc)CqUdon2!$V{$t}_^^k(y&YFIY7% z$-HFh)Y7XKHoLTJ1r?o*)x`K)u%e}xH0AOODIJ&sk?LYuSuHy*9^ySx2DrNN4to2x zwy|-PU#+gLuBNtj;P}@2WQj+rtd8BUfQBBRtvsER2`X2frxC58lwX-3cjJx#J5IRE zntaA^%C8am9+~Vmd?FUVeM5&*&7jnhM+YENS{d|CC!09YAV8=-55-Y14`)= z!!#wy9`>0Senct!Oe#Mrnq6gBxW#`?#l7SXlBK6i13ZbMT7c!GT;WA+_2iKZY<0>^ zuO<+?v9twh81%LEDpm(N?;Pu}!UtD$W><}v>{-+DO`;|acBY6)(lv)T*|=`fFbSTv z$=jEF;~Ve6tueIr!Qv%XYCF`yCt^3Z4V&YV%l-X`P~{n0hPcJxqF;Df`}Q;${HtDE z(H4IM5k7_6svpX1FL_-Bll)1rA7-h!boF*%NV{Xmd4q=ML6N`b{RPp#qvL0+zKxm6 zGOuv{g6Jxz!_Ud1JEcucAjE8-ShUsbDm9{#F0y~CiG+$O0R-bxKr zaMZFnHjfuLQI3v|yH9kHQd(d<1k!m)(IwJl`u3FN7val_?&=tf`OMrZSv|e!qKNt) zlZOmRGmVXg#_I%3p_0#eIxqkHkc4_(TX-WJBxSq~y!@UHS^>O1M%`Agn*CxmX6(NI zqJ(w(d3&-Xe`xrejxn8UC}`S2QF%o7R`l8{1DL*ol10i<6M5A#%e0d4TQ|w&?rMkgjVVSH*?EM=A@4zwL-Z!GW|9 zi{|Gc$*UypcWG?zf)76YJ$xC>9A>WqTi6a4;ffcF`phAD_1ua@>*-g772ffDE^_d@ zYvHZlV$Fne8uLW@s|iRnSn(i1MwK!ZWT^bwGTe^7jadaPoD%>;V_Mc7^@{soZPc6i z>t@~IEOX@;a^9V<<+_}ma6-ZC7jhS9c;@SU@kJ+*^}W?~$nUi`Vbd}xUN1rU(Pw_B zFJ_o8z`CtF$?tTl-v;i!b%=$ys_5vm%q(!w3p6>QHYZ{4^PP!=CxRsu^>q>1L4&p4r<~<2s}XBc1Zlm))Hn_v z7l+yOM;8@nwalSAnQfrcI)k$E1c;9;Zceto znTiod;B}uy-vF&KZOL0XUmE>roF7pQ1L(USOs{=!qYciDK-c-Ysr|cePmP+byMIT) z1RFEha8Aqkt%8z~R6z84gE?h~N5$NR%w2#aR_N@E_?pJrp8}C47eZs+5uK;j8$a`k zCQH_iwXTF8;cQ>VopI{%+D|u^dc@U6D~}JuhJCtBG0jVGUD}Iuim>-9Fpay~+Vfti zxzxJj^5ln?+7R*%6*35(txpx4d(T}hN&jSp1A5XG{b}Ek=q&Z$*h==IPb~G=K^$Nq z?d~4YW4MLXlW_fhJ<5Tr%GID_?=qTKroS}MrOmMs$Efo?bo5YDck@`NIOketPLT=* z8NutXQDfAVpInyST5{hO(_T7GKwHgTt!{iD6!*veu-=sO&f{my@jy%$xpMd{n6sgx!MOb35Sp9` zVk_g+0ubGDEy&x1K9P|F*~aQedwaIZ$DG6CV~6kKS)AEv zb435HCHqEJHvZ}VK3P^nRvo|9>IiG|R(ICK_P)KFozOK;g*3vJ7 zH|g~`Ek?dPVcmcI_x!1f;_h%?@Xpg(ZR#> zm7;vX$(n4HsoLAZ#E9~R70Ru#zgQ5iJEFX4&&{FK~hj)*^hV0aqzQQ>qW>293~jES`W{4udG!x-eJO@?^GJh-QB zMWU;HOy`CJOt1(3^zwhdPV~IfsaS9_Pqry=%ufbUHfAGt%&IEPlu;QsH+-arR+oL) znkOJDJ&tLk@9s4_2gh>D9t)Rw=U1B)$z&U`9uS14s@lO2uicFg3jK=gA%FqRW2>-{ zz9}Urxh;}DZ*5s+YIEkIP4L#TakGLhGE`Xtc|JFg;JrvycYvy87^nclwc~R##_WMn zRZHz|9PS+jIohV$s&;l5l8;*m)5gK{j?3DOFG*tw8@n5QlGu9_0mdv82V&2^ji&33 z5n`_|t^RFl=wI%|e3{7UyC>XyWqZIKn(y{O?Eawf5jTZE{j%+s&@*MXJHD>)wbi(^ewjI`eii5`EMv$vsb1rQwtg>@*#E|CA;=}GhNCP31kC} zE1uC7T>Q24OhG+mGezs-pA{L{Q zzIH6p!`XpF%PqI*P0@*GUf2-{>yV!xx4aClbY1R_q2Y)JBKxQWJ&O^^)7i!iDWx`@ zcqgvW4m#RtQ^uRytY7V&o3f@g`Ao9;b=Dz9EmcGU3*(jH5Kb-?0-vL8CjV1&&IaUD^{q(dQfz0-aJkszx7ucI$5XdrlW3%sgwcMz;N%gV zDXt2+HL3mBP~p#@&f>s0Ptju3@YQ~-V&M?qj&pfd%14HWXA$SDvlX@SE!#(6S+{}b zVuA7n7EUcEH$%Gd)W2?hl$Y!YPXW03T{}r3Tt&TK`+cd0uXA6Btdt6GqBBgApuO6Z z+1(ayCTH2a3J8d{vJB{5T85Yw6K6Lz(EkXDEk5G+8QfWGOR3+@7NDweqqT=3^z0+g zvI*KhuBjoeoJD2cEd`-Y6inp3JY=C1l0Gu*;*XQ%nUD|}S&301zYQp4GmG#d?&8mW zCA|QtGm%pYO3XfD(?;b&&ndNGY?TOo`p91)KblA%bMO&ysk)fNna;qM*$E$v&Az+nJpl+p|N zH?Dm{1i(z@W{LDbzF>1}m4ey)GH`G1N{SSK{A-0bj_LfzYB6@K6cY zS;M-o%d-Lys2hmQdtBzZz968i$Wj{h1LX%KsW6XKz8hlo2|yZ1QwoMFz-(Yx*pA|_{P9{h&gwStudL69MfZ>H zI0@*SW=4n`TktUTK$5k6M&~=}eMMD}rAhj~WZ>KN)vvpX#FbKxbo}^gBtSLGPX5)G zMQ|8d+7aFp4H3ZdkW!5!UN~R_#k^1Yhbn4(j?L&F$9Go~LuEY0Gdf@5ts60Z@ayFK zOQ==pUV8k}vCV55fIIFPnIknOR$2H~RnKM^{cXoDy|M`T@ zoIjp{X4_dAz2F{#;^{Y-O54dXN619@w)tBdy+{~ncQfYx z^syRvKy$L( zT7dO3066N^Nw8gKBTjDt!? zlr3bkrKv*J|KZ{g^Rs=%1f;ksQ)&7Kn*bian!{eZeeRJR19*6=iyyH`fhNu@Z5xEf z;Q&uLXz{ky!REAp*nPHK-NvE6$ONRtK~4&z_Ft8SEIR}7i6IIInG^m_WNv5^WS!|JMspZ>~MHoA>vAD-Yr= zg6kpkN~p1MlRdoZ$T4h}cWdAr_kFdEEGgJ0Zx+hmFs}^36|P{Jw^cPaWl&w!x_@K_ z46#9m^#oL$8m{I854)+|lTuq<>v~+4O(#fkDs+2lOp{ebBf8tb=uKTPA-wnm-}F-I z_9=~2x#{dMsZKo8&&j-Msv)WnU9On<1So&etSrE$oG@hrYYvZYR)D6Icyb85*u?3o zP&PZ;HK-JMs1%u7-L>32H}h?ioG2!TsdCJBsY2v%c;`ih8ip}ySa%fT3uM5{G1gh#W?jYsbJ~cyzaQjk9fpk&l!#z7FA?zWR7$4iZP0 zji`11)-)BLEJeJUB!PbzUw&$Tr_1tJBxIK%&Gp4FuHr~+<-9Igb{StXI*DzTXnV`3IKFLrG-$98q;dDg-QC^Yy^-MVu1$d84#6Qof&_PWO#%e>5D4!2D(9Ya z-|yb{ALEVp;eB9qS2tC)t9sR5d#yF+TvMa7m>N78{M7xHCAs8>lSsxH_j0u?;SKvnOJyK&=6m)cP z3U$9@5eZLAElLo&_Q_XF4&oF2rm|qjpEO<@L9N3wMuYIG>#9e$mc&ks@C(>D=Ccwi z7gOy?CrOsncPD8W zb;dLA`)8d7k5Md#<)J@b+itBG5*TQR00E+~r6mT4mN>AnnE=)|c1ns~x+E>OOhI)XU4ubF!B+(}4T=?*8DH9uwM^y`0>)o>-~+ zB6l?;9d~=&Lg%Ob0f&a`Lyb&-H5_^+z@)?sk5Y0u>F>1-|+%rO@Ni`Dg7|ieFX->w~{T6K@SNGC^Cjo zS!VI8N^*vGE|h;jIPe$}W`aCT+pnMq>J#q7(Q4pCi(JLsm?)o}Thpj8Ti>BYyBIo(@UoWv+1%z8nh zv}z$5(7kv?7pCaxr_p9P+BrGdv*?$H@TyKbdYeRVI4BW)PG(_+AGfHwP2ur@cFOwe7%2?PlL zfCwqSwx%Zi?OGu3fl_wtnJaD|8=AxN-Y&S5&-0$ET($XhHZdSmi(~Q(n~pDdld>-O z7WV1sDK$0q^g`4d&wHh=!2je)go>)zC{eJJJt5$s`F7y}yCc^@XyPpRmO0^~@%7#6 zTOp*Jo3q&VwV@M)z=)xr;PqRc^oW8D)b(55sePY()PU>xf(?SG9Omw;;Mndm*UZ#dP(?0@=RWC$*i!6e~xMRHXAtI;L(G8q~C0%PU* zsM1EVV=6mI@j?j;90F5@mVE;QCJf-a7!6B`2Xh9u_I_dcx zO^KR~Bz$XEu@W8WKGB;av%3%_dXGw{VU`(k*+RwE1Lr1Y1%?vQ}5u+ zow^0R^klZ9Lyd~-Q61grx3#g*s~o%IfX&_kd8o=q#D3sRZ*T<;~(!z8`skoR63MGQ*1#-eB_%ONJbHS zXQ9u-pf;NA$?b!0mf%5swnQz{il@8)6TLqoO+Y{h@to)2S@3WRq6~YJ`lALCD=L5U z3uKo=G^DAg5NXOGf+MYvXLiSs{n<#V>57A&C@d72khjWA+fI~Z6HVf+MspXkO;L2E zUYJ#V`0N{JGx7CPXg@K)pFYlvu#u=(pz_j;XQ<w@#6q3LJ8j}2L2WAs=Q%FztiJY^h>;gDD3-GFVJ6noNTljnSZCW+^F z=?Sz!mkeD^-GkERYCO)Iz8!s`ASJ`15_yiaEdv-PSqjG#Er$5PI5l%We;K{HcR24D zqwES2EK^x5J!41J_VL*{RcsdUA9R_`g02xveZ4eyd;DaX1kiS=LC|R1jxpRs^KZvtLfEX>Ke-?$~P<6b$Wd(X&;h?A7 z$tbhD$i)BJifL@oHZ_lt_v3@vPNg{q))!{PHd+0zUdZv>Pg2oXU=I|=G1qcP>_wjg zqinEuj3Lnvb&cPpH_ze@gWN^J#uHA0OoZ-dr^ner?fGT;hM-Cg3}b#_O2@&y?MBcy%6-uS&vhqpD*uU-x9fzU=Th8(e>ouWZW5-Z|^T;7#5;2%U*B0U@Y z_;x~DP;UJ?==~Xb*!rE&&uI1gxKc=aV3rgtex4|W(jHd~3G!E^Ammj=1{w@f z@cBEV!c%;vObrwaFc*lNOHHA0=2fHyIeZCtyJs(Npg3G;$t_mU+ew-}7D!}ipeWDd z6A~6xRlV2#tmC@T1Cy%pt4+n!l&rI}6Ny4i0S09d7Af=qtRl6tqLWoTuq&vq-sDEkOxXNT*%3MCghYK0( zlsY2CtDLhIvw58~sZx|G>HD?7bD6>CHmCEkPx#&GEr9ulUwRsn(Q#&=BUW%fn#GKu z9VuZxen@0tQcPAbY|I}w-=|)G))3rHL&!8hitLvl7#AIU?j&S3ebA)U_~@%p*?y1* z+V#mte7xtH#%sTIWNDP+dD>bAb8sQC6qj*~c&sBFO;Bf$-dj>`60_2%8AWC#SQ_f% z7PYiw{`^S@RN4Sv#Y7t}jgkN6qBQtUH)yw?Z0o7}QUv%)Rm3-@AZm9!WA@v(k;Pm5 zJQ8C$dVL2?P0f{uPXD{x+MxHQPhS%+PT!K1ef=75OZ}IqKlyF}r~|&PsHxnGd$|n8 zCn4GXjwP|-);hZ9_7TKtV{e~b8~Eq&@rQFDbp3Hswf@ft?o?j47-iB%x^VkalP9B~ z_wmlosOICw^J}?M4WmTo;3p!c#XCBFJ@sCH=Li04l2fb{{yQ z)7{{}%JQ4}#aX_dK5tzUDV5+1kA&{NeUcV|_{Tz@u5+pDPpu07{!0f=x-(Q6JN| zBMcK}1+e<0X8n)wtsk1cY+nZsXwc?| z9m@z#)wRSjCgP2>5u?bD>Ct|4qUMv$rY&}jkmmYasc}q$*TbjGhSNk|TpaUPR6{lz zRyL}fjn|G2Yh9tEk~UfLnM$hh>`6|LQsOd4a6b+WS1mIQJHHg4{n`6lzI~QAZf1uf z*9a1Nq;CzT26ps`i3pJ90xn2(^l%ZRciHC03NNbgCxpkX|9o$SfnRZ zbsjX!-Vd3-+Ql+P8}yVen_NLDIl#<#wg|#i(^Hq4KEnotNL*)H)WbZ-mjX56=CL{X zG|EegKG^Of7_g!z!$lbw@;N)0uaZ1epq0)vN-!9O3wc(>$;}PTz2^67j4GFLSh zR`4$Tj3JwFVmtl1q@VZi7G)~wG=?mn^O9#&iKXI=ITv7%euf(SNMK}%+Z-&n`SQ~8o zVlIsKzLDE-@Xzr@4EytQf`(i{x}bVLV{V8~$c$3LXQrVx7B`71$ZFRzeO_iSBFZ@G zflyHL^C&oZ{0Lr>ips&DDZbBv)e>8OPd+EBCMB0)W%S;$onAj#1|~br>Las6F5ae| zluUbUO>6wFeaP!F99A%QPZIW@l}0zX2j|Z;>0TIPlp?dTLVeb_vsU<;rpjzQy|z)~ zZfNhN*?*)bAI2w*B>yWslqF>)2gE;)7Ll`abT`j@e6wrb0l1Z|Gap+A`<>II;t~yd zpq2wTyWFs_S2#GkN~)?CBgKj8Iyz%(o&GNE!c3-J#Ku4Z$*mIr3IM?#ed+CNO0>^S z`?|ER*s$<-RYx;sk>=h+bP+&OMrNKX0)vTG83h(w00A2@p>T#8471(tPEADpHQHF~ zYfDU(*Nj7G(hm%%t~7n528!(bOtj(I+a~hINGrp)#jTk{2orE1~fX8K?;?69tq0%u_cb;`I8=n6<`P$p6?c!^g~ zUAe-N$TWOeMrg>K7!v3s=%W~g8a@(;7Lh=|uR7-mO)1Ky1dwK-i7-pR0HhH4y^o1! zZBu{$w%?2BH}~`N1Dp{OMJ81QaKT|n;?84(f!ih`U0jj5q?8X&-D2R_(_3-+h~?<_ zD(3n3q(=7?tMYjdbb*Ul7-54SUoAH#721cza$9|4LSn(5NK!D@r`1s4Qnq9V9Q3`w z@JNX}Jc^K%4VFjq8!C7tc);7G7q5bF@Gy{kVx*IiC|cB$8~VYe;7S7-AD>s6GBZ*i zQeMj<$CrW$Vu}yLo)f?lAs8QG0EoLKS$&En}Yo%D}yQah zPrZKojjr_~-urwfCHmIR*1^Hb%_4a!fyJ$rm^W;judy+A7d7}LF<{4%Y!AEO(UUR> zT$TmU$kny9q=ZQK3%X8Kb43CP0p=p0@7DG=NM9(f?#!DG*kf&dIDScldwsmow0sRD zLQQJ%=xNsPro8XgI&U~QImhFHbaR-)+8z7x-Km28r?G>_A7p3iT|&wRew9`FeS#-< zj>mpBtT)etqNl4#7>Zt`8~Ha+uwP;FY>>o1mL=KWEZ_EbuPOiBVb}$Ps!T*4(q#23 z3OQBJ5B%5SYVX0d=_Y*gRQtLaO8*kb@XoA zA($|cMa+O2i(P|&=#*nxWybwCJb^NM`Dd!}2`x|#<9(vgNGb$t8n%@s=(H2FJ{Iim z@Gd=WIDuWy=)Ejls-yrWjh%ru+KBuqmpn)O`H0jajoyi{|i z{z>Sr2aO@kJRhgDdz(0E2~&vh!s;D#X$c3om%dZ76z*EuYMfJ?IJ=o?!w92WY@7VM~HRg3p{|nmvnzZ@kw9 z6DAR6j3V!SirlW6?faYN(}DMI;~7)aXt5N{6b6Yog(j9R^tU?$>hOT6MaJ7BYm9-n)EZA6{| z?j6J-!P{1no)em!DmxH#k60>_SZNdE=f|qUx6^cc!zi&FD@*XLq2mji4XHm;a|&($ zxcr>wF?hZ>65r^@cNr_UbYk5_NUq{=fkg{AhC<9h-y#XjKWYut;u=_zj0Uru7NfGb z%gGi|yYYM|IlROidUS%<_1M``lAQ;yFc})CCS`Bgs)p?=D#y>T4|bdmu$kwVcEqNY ziVP*1QahX(K{AgOZ8Ec&`it48TQjMN<+oh4{W?-)$p|IA=OUPcl#=9?=KU_oEu|4! zxM08&R;jjQQU@uyewQXjtzeBaCg{#3Ml@Ml24_jE!r*+_v~X{yW3tbCU+h2SF^Ra? zmxL2ZI_W-43P1KZLNf{gbeV`pdg{qdUr zzBHJ#b}2(h_T~tUo->j%l17_w%aYQ>ia-P#ks?UA0C>h7QscX|UP==HA z(jzg#_}gkM^>Op^@-DBhIfk1Y0hNV$u2c;qiZ71QMnHNP8p)QnQTQPdt9dK%%8rYe z30c8anjy;bpuzc*CqCGvk!io? zRX=Rr`@ju?(z$2W&*4%%e7*{i@?O2U`u0uBav2u^0mh;ZKTg_Vy-u+%1|VVfhQXHE zKyK{-tVq-3YbWIq+KHkwV<&_jUcn~sE_ciH7hb;h%B{U+rFAA3?@bP!4)DL~`yZihuC(*G=< zd5}qR&RgmJL8LkJB<^FW9CTQTA&3F4!CxHNl0(l2a44isRHO^`N(xt zT(Q$!>P9pe)Uehi zmS4bV9_Wr?yZ{q9y}$jfk~3hFtXqyYMVx++vGkDeD>pY6Z|%^wD`3a)64(=(AwXgV zj37Zy5}m&$g7`YR&M_wvOQG=Hm)4n`>hAWb2C#Afgrcnr- z@goKjmoa&%1byV3!;pr}C?9*oMq651+U9jj7Jk7-DM*K@(06*Zk@OicDaqXayYa29 z7=0`TteO#pPqqdM1w4`(R)kspjF>_sh+jfTMdOkZzSjTH`HAKcVX%?(Q8%?wFXrlS zq3+!ci-F-XpVuXRoc!O%C29Cw-ZmdKmMhaZW#ZwUTdU7>`}*KuTtjmwiA$+mIeww77VPcmecu$6}6wKlH^=27hQoRzZyye#$(934m4v%Eq?Q$Xz*%P#^FM+Th3@&v z?CLR`O7&7yT$e9FjPvw+NMZmkoig&PP{+Qdk$o3VAs0MKsn|UAl!svDn(PfBi9JsQ+qStNYlcV)TQ=rPK*jfY3Wi zclRrxRAYfe=&5zDVaSK12aBCbq-soHhS`8=Fkm5k5?X?Z0}@ERhx)GHHJZBb7Fs<{6$fbj zeVDPAy^l0@&iu^uX#xEtdKzCNJmvX{+FB~yIPAE=k!^0t8$5MQ{RCmb5v#sUx?XJ~ zX-i)iF9ro$`}C^*zE-IIoSGB^v4uePu7LoMG0{prh=`Ibaf~h?kSI1kbabdXKMlw_ zb0-$!6(_9~K6c}A&{vhrBI|7!Aue8ZY`jvoNcndF?7MUJlOp2d$B(aS7lz{jd`gu0 z1|NFD_QzCUi)_>oIa?5#o$1?$Ux92SDBfWHCXS68*Zq8 z;z(4meHxIkNo$}cK83xp1pn~}#6cu0pF=e0atDadNiD!5?I8onQOvkO`SSV4BQ+#3 z`Wg)KSCJbs8VY9`z%z>b1}|z4rc})32?Q?uVvQTQljhRCYMU*?le4?)gUWb@rXBMi zEP!6~O-2^s@yCa6;qc$E#&(^A3}`g6L&T8&n(BnQ0tGx6K%jDg(A8Nmek5$00Z-+- z&@G0hjx)>m-a>;u3z%xj3mkQU!z|LIwRlTL=~^7IsmcEWHx#zznka<|k07|DnYLIwI%zz*69K^GCPN?V*HkbuQS z@0)$rPZk$BqS7ysbofq!?;(g4YW}D5A!>*yqo~+OF#M3%7+j4ayw}(iKC4&~YI>C;;YFu3PWVGaOZC|8e0bqbh zvk|E%&YMHMx+~@g*=2&ZQ;epp%`370n%fTJt@W)M2;7#KphCD(FtDKiOhkX}?n(4| z1MW*rnA7d)Np8FBap#hMEu{RQh}n_8)Bytr1_l8WVH1ZK45kc2D%!$TGZhol4=YBH zf)ti|lrY-O^E52^ukY+7xKAypDU1g^;m@VNT##cB8-T7kUPUItNy9V@LgzOJ?7vcC zAi>KbBIqhH5R?vExDU^N{}Os!rYr?(&g4_KANfGgy-1cG$lnlwB(3^$%{_5s4f&;z^l3}cv|PuC8R%%!{!(k z?muJx_t5|K@=)=Qi}~*__{60DdCq_D*bB2K#|Ma`|MU9)wH3CI2?rzk-}mGn|6yW@ z9^H=wAJ)I}hId6QC`cr%|F2NI{nsKeN?3n#%^{#oU!}wNKXy$ZhxD@n=ZJ0v7`h6I zikQS+u%WMjo&wt1n8fK^%2dSYmd(qU{~Wgqy!!vx{`U}FLZJT`%YS~fFgUIMy#42S zi9h}|nSXxrcWdJn&j0DQz5f4n+Zg`$QU8t#UG4w3A#;*Z0vb7>$S1UZvutzx|L0YS zx!7`CP<%0{28hvWMn)6#<+&0P60n~>o!@RJ&Mz-3dU|q8N=gEnJpbptf3aP_Xlkd? zD5kprlB3DV$$bFAAuurTeSEy@<*ckGaLBPVM*TVqgsZEo2nhR@B$Y3&?b6ZmlN2ahR@4%Sk}4z$*ITKrrcT?xeVf2Nz3&0yW#z%m$d^nFv(nMaEvoGI0;e6AlNKR1ceR zs(p@}7dnWV#hve?g$D)%*bg*4Jl<}*+@0PLV_eU#^7g&ZFwnx(&MN^6r_DNK$_~yH zdT0wZD{dt3?5Xz+N6|rtDdcl}W{XdiLT1u&&|}+FZ`BmID3Dai@55|94DD@&o{PW5 zsUK4WZPWo-{zi);g~2h~!vKKG17N~5D+nCbK0+sDTCf|u4y zrE|+_Ffqzp%Oz{DR5QQp8M_(2lmtvz6hZVP^wTL~8!n*yU|zpem#wwN+fNQc`&g-S zq{E5wf)FJG#_wZW{*tA=7K@uti9a{<2JX96FQnOA!$&s?*Mh;y?CUZcJuxg=qg&^pJy-mF1i9Y;hUN`;S4TCZ;FOP2TolB~xBF6yT~K~9gpLh`~dQN2W!w5UAo zZ&F|lY@ud)lDuf3@9!s_hO5pQBoi@mQ3?+PDrlK>h;$lIe!qKs7 zA*Fne`tBg{k6$(HW@sW>Y@RDSCx-wZ|6_M|L1iVqf`US(pf?Ku`~3Ff2Z!Yl8ldD& zKt@J~fpjVk%?f1VeYw?1FJJs+^gzDezV*<-H4z*dpWLc@@mWw*^7jSrr_;{RVFH!L#VgdQwuG1Q{hID@ z=ls<1D8b-9@8IWG%pDEC1}<7})^(FkHk4)buJVXl9G_5r8~B=X#C>l0lNek!(G^FO zAd>SS^F&;=bL-x6)%IBqKOie7p*u$Qu42b+FC%D3I|HX~yLNr^=pE^%9QIN5-qiXt zPU%)kz}4q{jSWrTym{%-I=oXynu)}q&M?_H$mx|P^uYP2Qcn8FM4y9^PTEsn%F~hF zLJU=R%!&sY#G_2+VR3#%Y2g$^+aCR`inu^2uJ&Ua8ib#u|j5wmQmPH1Z)k93D%+iVw? zNB>%>%l>Ub7MD;%Nio(3BS2*==xeP>cNdpfMm{w8P#R5J8GJ|3Ry?k+mmfYq5j{SW zNo;FAU$&|T#dXyxi*7Dfz0_>C#`Z@H%T&f-Z>0z4$U7;Z@nAcTBdN}a5EI9veMF*CjzJj@5>zQeEJr`_Y-~;`EGXUoXZ~)g7^+g~jS81FX*p zMbuR2f0r2lZQdPt5D_!nxX8;(kFBjOKpY#8Ujww~0Uiy}#s0L-+16lVW8=*Hyrh#8 z3s9nfhlLU0b5hs_WaIYD43wPMmm21M53;-x0=AE;f-(!9Y?+68oSyvyn7Ynnh#5Vd zI{vtNU6J!m5NL8$`DQt;nya*Jp`ipYI2VGgbXcs#mr$#GAi@R0w0UJ2WZT!jt!mmg zH6XODOU^CN`c6-O6mxICejb&-;FxQT4_rJ8UK@YFoyYBG1xE6*R zXHHd)TMuM@Lq#VOyRWPjut+|}*YA?uxbF*)iFUl>aV3UT!tIDX;4{oQxPC1cQ~#v^ z6Pa^mK9)33qNu<3wK%(>JPUZ(GO_>D`Q}{zW%F^7lQ*_;xOu&gO+WKjuDAN&Onvcs z&C=TH^m`J)GpR}1Nt^9>w6JZsMxLXhno8{cl+i+RtY4*NeyEareD#W-V{U%VmeJxn zI9u(fPgbNyf@oWB<@9o?_tQ5y9@st;hCjnZ$vEIYk|}v%)hlF@kPVm>Y?u*!y%MJ0 z-`U_)odJGo=AgH`4Hu_N=2f*xj>}CJ4XotLD^ox6NPHT`D&#H-mEvdBbY=4frzQxH zceEyG*)2{PByFnN^No!Kd=Kg>#=!r>rh*|NuReVODxqvSlRLY+t`qFTfPQRjeEcA! zL0wO8Wc}sYyREGaADB%6Jjd*;I>0&vlEmBLARMyc^9EJ&DFG?(+MJRWqXb4iC|=-{)n-@4Q>CqZovg=70Ix(Re?e zi`b)uN33YDJYC;pWto{9Se0anK0bR3wbx|9fi$M>VBQvF#^E4&bxgGCt_>Zg;_tC` zXK}JH_F3D0>o~P>dR!Hq^?*}jOJmt!%m|FM9%T_Lc3zfze4Og8YnPQ(2YnK`qv`Vd z5$ofKGGAs;up77V7&HJw z6DGl^5HmZcJu8#JQl~J-+u~|SJy$rq{KvOBwr7151)G%1iIi{01m=+BoF@aGXCh(d zjI9ok(oaC>VzXR`*Qkfzv^CN*5>lrQ&*x>D$HB~5~HrMP&p82S-Fw+c85&hak))74?BK>^jvWM3-8rCP3 zS}D#W9E2C#x^FX3`V8HkzmUHcSTb5xvUS4IA)jeWVGfF{EK~UzHAJuC@$vnQ`tnku zNmnlM>u4mLMAm$GQ*6iTel2SMMOG}*($Bnd5<$f1Ib*g9D)wGY?3g@Xe75Kr=&1(Z z4YK*^zx1&vnY<0%H1W`--Q7jLJYErh*r1NGr(qE;C|cnIseAPeb0Pgov=yPHD%hn#Ird@_MlW6Kd>p>UGwxC~gSndHkhuA<@NBl^>WQJmSA z)Q9gKl`~051M**;Kvu|ljwdOps0b2adA?>uHJOBkV%$0qE9GQ=EQL^g?fj6nQ|z8B z*(at+VoZUzo2G3w3@o0mp%g4X;$H?c4*n33Gmb`1&itxbpQve~2qWYRpaZ+@Tqi~LH^Yo50dJ1tjIUmO=bYRRCJNVO`l?%4?u4pDuy zHFbAzrn5LoI6NhDRX$Y*H^~;6eQrS|V^X?mRzX?$^|mI;I}_q!k25TeT*P@m8TAc_ z)5%oGiR?k?jpl+t0(}r7&eGW;Y~CgML+qpw^hk>rW#dB6*4b!qaSi*yzphH5(VcYpk=&!DCPT^7sV7+2W!9M+C89 zDZLx00g5b66v+lO7oc&hU718qEuq%q>8?NYZ3;gz0AY^9dKU?x95ED6q~GdBLq$cE zISC-M{YEAy2M;{X#m^uC3DRlozetE$A8Ud+t9-!1otY(+uWRf5Ye%ZdzOVDCwkXorK#4Z;Jbr2 zbed)jAr?huy;&1Le5&m$}Xt#jG?KINhm+WInNR>3iD^ z2*`d|dI85*D=+hkv94#=Ahk&d&;SRBk8>uYuznZ0^Gy)xpAJFPjSN)Oof!0tsrM#6 zR(0ggPjzM}_+!W#p`=}?Y#53@a+gneoC(;VEt$5*r)A`c8+-%Z2tUG4=Jj_}k5I>7 zwF7D;02?@}!01jf-;Q{YG@~DR^e%PRcrq~ffq2)IG}R1)rZi;xWuL(bSQpeAjT*V{ z_Zzt=VWQ_ClSDq=*_<90JcZCK#db{TkNDzE_aS|fqT5sR?8|7`zvpyqP}TB69(Oj1 zp~D|Y$)3J5?F{V>I#Jwdb)cR$or3zGY>&}IR@heP;Gv56GY0!_ctPh^QdwWHL_4l? zCdSw?(Im8#aJ9>W!lQ%yU`YKzmA=wiT9Z31^D5L7_|%KvC-$Uy-5ySY$Ir*OK#~-8 zfSzn;3DRrXiXJ}yN2=QfPWk$aDJ7nOSrMbMKaE!eCHuN+xy7fKyyD+5mu2s&_myd<5nN7~Ipsu~hWYt)&N`tb$+;lemWI${K;+Pao2_K%LoP=C_8G%d--ys)ASCj*D2Dr@qD=Sw)}twj1<(jx@$Jg^3BHY#oW{ z7+Au;`7nO!JJfmy5S|Ohw#Jkk{9>l6KkIe6W9VAo-nCX%6f;)^9^eQJQQ~hRw+R~9 z+8^|1q$@*Z%M0%j@&kQ&&7BcjM@eM}o9e2o3qIgNs+!|}u98S~9PH4_vd^_-*%X*a zH)AS0si6GU<9x@%Rd|32yG`|8C4ya+COQ3g|6T$euV|v4f5;CK*0W!hpMSlT_K}sz z%)ds=Mx02Um>6<`U8!W^+eE{ibSeI#>Qi5E_JJU7Fv=pND4VVDOF>ajNpP+U+sNU2 z3}Zvy`FASRqI9Uy@|)U8>;?u8hCZ(Ds1(9|8~G#-i*ra_9#E7ExpOh^ zNy1Od68K5%UbVro%F1v)nrr%$?@Sis>G@+>gWEHAD!?3{Rt6e_qs9|PsX0!uyaR6xo;h~5 z484U%lADCB+9e&fxi?`*AD-`4&LyxiR(u$BbB*P_t`Ty*?72`Cv41UhhJCJiq|FH^ z*stwwhtBC=@H_2U3e}e-HnaIww4EwAb)PC+AL<9w6tq41K5H`93XLqVKJh0->Yhcf z8!0Jx;N}lFn3U?to1D`-xeRY-+ioW0IPaRWem^hxT-N0eLa{u}b}ZTBzPMRmS6Ti^ zg0>i-6~G-|-(rP@+v-xKCufpO(iiC9cOB%^?uzlkKWFeQk7$>TFhRjBruT{;It?8QmEA5UwXqvBr%=k# z^FyG~b@2-gzV^U((YEmU1d*uP0{zixlgy0+*<(oKvq=7%{NCnip_CoaTV#2J)cFA} z$walAm4e&r=Hn7%zWr&|6pJ@=*Naa^*Zu7j4@yC&B*X0{OA}x>eyK&;j(uXodX;8ztV3d(;UN2o9QHFH9=%b-AAIn3xedR--^9PJr%jqG?-0pu0(PsZFL{yGEx83Pr z&>XtzNW~;)$tq}9nx+LAHrJq44-aa(y27BuX%LL63M0_#q+Ltk1oSqp2@khkETB-D zpFb45FI0vE-wZtxRt}2f>O}HtR;zHJK(lZOw`MTTIWgJ42;?sM+Wgde-M%a<$1IYr zk1(4Vaw)|MJON7HC$D0?)T&wPp%BgNO)A^=7%_vDJmcYvv) z-oxQ+;3F|~$=pzT+kO?WH^J0rJ^SVL@KTFS=Bv>on|lA)SWiomrlj2Dh~*OO1jX_% zGd*@1zH~|SqQ=a|d~;*+UO#;WF<)L?EfdaI%*iRfANmv1|LxmAO&i25rP$0qHGL1J z51JS*EwjAz9h=;BNC<0%_@jVaf%@Q4pH-DTd;!jnA3U?za>IwGAB+W}SE{J1AdwSV zyrHw6x3a^?{KLPF3&>ly^!BX$veF5+ORWc_fQv@{N8%wMAk&x3&E0);YKoSHC84WJ zgoA?vm}}0?Ty<(qCWUWS`hkQn4Gj&GL1T<`94;maiL$09HZwD`mbSKe!xG=kiaXFL z$>Q)wHIO9r_h#B>8W{*vFdE!%IWB{#H)h~VzMZgBY}q~$jQNsSh_KXp$LermQLDYu z*^nrGKtEA%upPSmTK~3xn_o1TfTjT|7yA{4qO4N7@|KBh$#Bvj7>_ED<+$Zq;dyqp zr>TZr0k>gV4(@H2!z)L$2f^51oM%h!SvYwy%QZzb>8@wo-Y)E}B`wu0a>F^PuP@Kc zZ7YL67$04^;XQ>S(c_}4IhO2O0d9_WRBv5pEwRn!TFrCI0=GakqoW?HK6&*BZwa!gVC<@DvNgyDO8@Fx#Z1Wv;n$bQf?mwwMO-{6l_MtJLt{#_f3EkR5E|jsh&B#jXsS`}SPnYWzPAF<>MRo4|=v8Y=OyE;kn5g?Fc*J&} z^EK16u&nS10wVlgB0-@2IeJgYC%!*A=xVW(%%xmT2su1I+@Fv5!juh{aM8z zC`V)w%04&wJNm?`L3{Vro0ko?rm7&?axIj&Jq+Hj;d#u$OrD-~m7)^6{MIE=Kh7Sf zbfVyB8XLJdv$d7VhNlXLj+806{pcpE9hCsyCgGnlOJb~xIad`gb}S@z5%uYzLc}tl z;jjy3OrVe$-`1<|5tH`Au=u#BvGnX7Jm2b^T2d;&tg#3BR&;*s?^1}_`fO)p>RAb+ zV|DdcSUN~wlZ5wwfFLx)gUboD%~`Q=8whBysF9?6x9GvK`%gsWz7JlAX;u*Ck$#)d z&1YPowVrs~g0wmGEp{~D_lPi9{EnOt!O3EaGBO5SwAbuQ+cEA3mv?l?`eI-8T=-Ti zj!G-Q5LA|EoCXxAeV;|c$6p~ZnM^IjIui_FF!)p021(5(?mh!iQ?lcZ^oSn#u^Z^+rT#mkhy+h23R$CSU!97j%a^maX8>gH@9+J9*3-Zn z7ryty+nd?%YAC3v;DBJG<`6)IoA~bHcpa~do(+?!`uPcDm;HqYmFqp*7C*l(1hc?tR_5o)jEtqNez6jSSDlLmq9rnyBTT zXy5g6A!vHvM#hy*`)jC0nA(TUGFq;A__kQMK8CJG|NmeC#M{_?sBpBo3&CcI(DIhW zp~ewg*1VKJB_%r`h(Tg~xwm<|O|m(#ArlrFI?aqld*&=O<2{iziY&)4OIsr)u=;GsGyBff@Ei#;Rfj zF0=%-DSQQvR{7r3c;#tFyrliPUA@rf z=pS8HpPUsz!nWn!2f`YKrCCCqHz~^ORU3#ont&7O$)6 z;_AxKusO{${wiWOEj$EYyQ80rD${1@Q#C9S*H-l!B;bAl#1A@6sacf_uQio%nf9E} z%QCA%8zsZXnA`DT48uSYq#DAy_dt{+*(wW<~|cZ z06K*9v$6U3qtvSVFh;(>hIa{*fY+=nEQKqGm2503;vB!{;B%F-IhzBxenAwoUnpIT zl<8J!h4>#U*nWJC%qJ!Kv^2qG=@xPPZ0;bBL%jrarxDi%fv(kY5Tw=Baq?zsRvl{W ztTZ&$lyHw&uxYgiXQ%*>QbJg4xGe>37$HwPT*E)h&mtmPYg9RtGJwW60Cwc!<_4M# z*6A^&rl%(Z3-?v0^7i%|fN7B*{Oo(&x=;FiJ1hz`;vPQV9Uq;Z9zO7};K=eh8^i+? z!>x~3T06XB0}AUgn&peD%^4YWssJb!PL}b1TX6&wW5eCZnFbPe~Ljbea7^LsCArT z>20HskeJ_Hq?N3sZAH>P9ocqig8G+X7kfL3GZjX4_E``vWP?rm*GUZpgD_09UGkOa zGY79QuGobD8)L`ks5<#CQtE0t5&Z|$0<|GUk;sdD5|!#8mRv~_i4Yr7Q)SbhjcPPM z!cY3`tyx7Wc*)!K?8>fI?nkj%mySW9-w&JT_92h=}YCNvZnU)f^`D0)*5<1x)lCvOM3Aln}dvDj{n+5;{1y<-f?@VB?Vd zXD_K`d^-m-)qM-hPNXaNix9@|X@_++F~U-@15h!?AXmXnD1d1!?fGHpEa!K$L- zaT$G3gj|<%w@>F5_=iVkltr#$AQktRvpV%Kf!aA*xLZ0{zRualh0MuJZ6t*9nMGON zzKA|V+7xJ!3%*NII_acUupmh%wk={UJ5s#o%s1{Z3s=nDujbsZ@sIMwY{-~}rcS9_ zR?(Sg(bbDR-bS4bQExOb5`Vh3+@oy76=}}-EGm}kOT<6xVziEgyj4C{qFu%S9MZXSGBvyzBg~Wr)Q?8`_tXb zE#~^6!FNgU0wueWzEMC6?$)KQle!w0%#9O3&f1qKz+Z5q;K)gKf>V(1GUuJ}&1`V3 z8Ca?K=hG9ye{*o0Lb(q_fQoBnW`k6(RF}fR1Hk6TB_g6|X~_nF6$UgSy3%T#{A_PODeC`{Em$y838O517|p zrS?g`QZ)F$kf^}R-y-x)rd5yI_p0c z!_@{-AYi4FIc|JqEG@h#^NGR=IhFPwXJcxl<29{0O*?P;fjaPmxZtwuu_Uf z-iW888K#&tw&vxA0bbu^B)j2lrvH5cCGX(J~}BUwu|Q^{*v$8 zai9AmMXiv)!P(TBR?T&jsJ&izh@(=nRM}Wyo35mPYaZ5=_gMB}4Y7R>;=v|Sfaj|b zGft-S>)G9J&6A``u;*@N5mzy+ig`n)c$|fAeJ0#8XqbKm8v3vOJm&OrpESzxJV*7& zrU`L1k_DN`ybjJ!e|yj5YX=kCD3mF^p@Tj6nkiM+BfaC{$_v7sXWYDj>w1gFdQ*OD z>NsIYd2fDdj= zD*QOy14ES9=sEb(H|hDc$oR|QadN)B?fU9S!t=}z5FSaxtCk;grDSM0>F(#pi@$U- z#am0;Fdwgq?umAa$xidX5_hV63^tT8w9$P)r5+5+F|GP;-Ii;zQ2ugM#k?#)HsxQnRddg!}4|7>P%4Q`^jA(SarxJ^diRuO-u+3B& znbC+!qoI1Dstu)`ie@~KUgefus#hwPeuMgO@ zsxX7#Vn#-EnJ>_E#zG`$<;y3mAZdy~n(p0;tA;UVko8eARpPyO!r;7 zg7uF}1(^%U$*=md&K}B-(#2MMn-*_Vo;6BGW&FvvA0%hHxHJHAxthcKOm#^uYsQgR zVT{ApR3G`AfAA5b5fRCk)7dXgQAhi>_SgAQE11VAtBZ-%*S=EM({lSlY0~u1=(9>Q ze&gbJ74q}vS&AAP0#moo*$$$F6It~u_B26t%dSo?`$Q>ucp(x)n3R6iJeZUm_>ZzI z-uV<+0(d&w6cm!x6iIq~u`#!ZNTLv2<;OsX>eyodQkY_+e}{ahKH#T`cP6xl6y|i& zJHjn)t%yk=d+qPhb42_`dmy7_lE-E9h;akbs==Y~^qH zwcDxc_O0>ij@PA328PAmXvLT3#FvYisw4s6VW10mSZSV^;y@m^cX`?5PhQivX=tHF zC~0bVoYmG~MkowU^?n!C5?kR~RT6F??l&zuUss1L9V#J_d1~z}byZ)*5c4bHJ*-X5 zC(`cdIm@?mk!Ql$h6YGqQ#csKN^^pN)Ore|Ls+_Fd#C+V_@iIr_0D*JYROL)D0Q5y zEa@ACFciiFXg^VfDqisDVa@b3MqPzk{#fL)ouPOM!+Szjtx{j^BQvJmbQ*z?oVp~w zbjR@-!vVOf)-|kkgi{sQ5?r3akQo{_pw(GS?eUIW%eB;M5A(1|lCP;Oy{O&j?B`wp zgm5N}thX;4<7~9}$6T5>-$NGNnUIdeu4H15nAuBv27aBfz7dA_@vfiamtWa#sCD#*X}9$!XJFj`e29qTuhlT?>VECxvfR} z5{Ql|(lw76P`@9b%3}{rO+_O5A94J6{<=py@1yC-u|mb2&)^i_ugS(;wJ##j=yduw zmHTpXbLJ`>aPOvfd~BrEnjs0A#4X zc=3xG;H<8^Es@oq_RV_Y-L?>R*9-pselm-Wi5g|MFlAA(@BY*Ci~0oct3%AF>;S@_ zE@LB|mhy@YUpFOIm|hBeiL=&nE&C*e`dv)1gW^GKV^VS+9_9slsYFGG}CRTa=F49v}gA6xzah-v{! zy2=0mSYS~C50aA;Gv%oMaxFiI4Is4CMl&dM;OI(w}2BWJH;Ctb>P+9RaD}W2mvj8j$fYIUK^fp|yNE%s1#mH9o_b^yryt-~rkxo5eats62 z{bF6-V?I6}%OC77Ra68tAQYwm%OsvZkB*Oj4*UTmBKGmQ3*?ThIbZ*>wzf7eu4;!l z+q^_Hz-GU{s>D`v3_|v$Yq>g^RWdPQ03gc$^RBS&47ax<6A%ypd^fC~+=B`&uB^lY zC7b}82~gaKzg&Zzo{@26bQG}65~_W97}W)zo1*~&3UhLEfoiq?wq5h9mYJ|5J)h9pEWGqbpf6`m@ZtPRgIl7Chn4h{~O z091K?>-u6RzpMK!^eE{}=f1Q}4}IRQx5FDJLocKwf;|2D5X>FV=Jl~8?DcC8>_1oI5REEQ)KS> zM@;ZR-|fw)q?fU4mQ_VM4GbsmTud<1pz&p+y-B?}K=Y;f^J=3Z5W_|O&lgJazrO!- zAy>D=e}9#~7~?;!{QU*9Oz_DQT;%zCiSmqiAZd{d93U(Jq>*QVIC_isucyq@8eLp@ zA&Wt9YKnR-Ym0i>w8xezf4vcVkYRJe|LjiSy9gRI;-;x7?FSeb|JDj{l>u7P|8iG( z{sRp3OK#TSplnxg#`=H4>Mrk zwSjj9ph>&C3yQPE|B5;*VaMn#`y+F@@$!soJ3CBFOsKtOP|@ev$`oBde+)@bZoA$_ zTC{=x`2Znw<{ut?0~Wnf|{J9bo#j;Q}CIAhUA%LG;)LP)0X0 zD~t9&1bcq!*VqP111`UtU7VYiXgVnfb#h)$gWEyPLZl$z19|!%3&6mDBrsuuf&Jr& zpkPRcA*>iEwHGXzO-M-ipL58}j1JA$?ryntYLw@u5wPPrf`CkzKq?;C@v+No7es<% z(oo$l*WxKQf*n8t&T_R&pZ887)eb2+Y0R#rLQ=+LHe3@=@{lMF88JU^S$D^O_CcZdwf7uh2GK8@MUfjD3z_rhsN~Bz0@7aR}X(v zGv(w9PgNTddi%i9OnqMbH|Gsj$1zEfY)z?D3CtYNDwI#H6D0<@tF|^iSX~k_s*Z74 zbXpzTzV1)B>i5q=*Bc?g)_6$pCEAjG|8T$o6|W!4{u)mDa*y1p_pkzxFx`$XU;a>m6n zKk2|^T;%Y?_kdXWzr^cOMx-hM6+Uh zLQX0a+~D(|GrVVcvDC;Z5Pwni+e=kgM3+C*o2iSOY%QW*qk}H_1wKu>T~FUYO9+GA zG*?7dCfgrnY^1kUEL_T-zGmK@vFFa)GO?iKnq2wcF7ff3|0!$+2?MJi3qZ5=~Ci2Dv4cv{Jz401ni6*YZ}%parQ%vecV@wf6jB?j$Bnr{jt^RoTopwCq z9uC?de%Ma8Uanic#cA$V^4`1S>9#&pk$oQgDI zW%m+|IFAu^Xl>7#`S}xp%-zp2ae>67O+b2xk&%}?JUqYW%)H?O`%Q)6cI5%~yEMXG=!% z=QC!>ytXWr8?Mpmt`Em_HN#{$$*ZW_C~st%9D9X$tO@w>OyHj0gPiwot^o#A2_V!Q_|bMi7=|p`+rFF|QArXEMC@syxHwjR|gkvCqZM z`CZXQZ=}(t$3cFa4eh3`D56bQrVf{kpH`KrE?3+k&_m?WAXS6g#n8@$PrOBoFFo`_ z$cE_)@QUmklDKe}k=eCuD?{uK$Ko6!Kqa8d58cZp+LDR|ihjNuLwO0oHd~Uw&%wf2 zMd%yxXLqW-Af4(p=zNi!mSaC=Z>>* z_|x0dUrI>kQ7b(MU2R$Wd){SrY>>EmxCoiH3X`zI^A8&?F@OWLK$=?f6jFnMP2LuO zBqlw!Bsx1g`@Q7>2pmIRYrR&T+2*i_ei@He4_Sgrhrq47{81oljaamr!O82lJom8B8_S`4HfFvm|z$E!8 z$WioaHhH72JsYS%f3dn|kVpX$uv}x0@Hj8!)G^QaAa(;Ha zof{piVeMS<)IKyrUF66ZWsio%S))H(bc@l(nnvrTC|RfpTx;MI_InzKmvBcArg3yZ zW(L1RQVBfGDGxJVW7cJQm|b2Oy}Mfa%GhK?Mc1#mIpElwWNcYgvLz@ve8hgAgh{+% zef4f|#q;3T{r$M!Niz@IqK@pr5ZSL))0njF}Az4lbf{AQO4!X)ST0p0#% ztm&w}qzyJ>F;{mVYs<<;_Lo7QdowiCW8M2Ez2javv37d?6|*?eAcW7GnZj>R{hNHy z7|v=N61MsuB?&%d#;4n|3(Y8aid)=IowT@^V3;&%zJp$}8e@6->eQa9ZUsbDDMzBL zBXd-vsqK0d|H>@&_7DLJ>&M-7mp>Ow`8ezNPQs1j``O)=%wh{_OIHfGs12{KAGl42 zX}rE3J(TU!{;3;X&@w?<&iXoYapB_7v1tr&ij=R9S=`+2|p$tBzFdV!l0p*i0CYa3SQEnPu2aD^e6K_4@%$8gkR!$L@g4>+K zfyz9(OOi%8Q@m+~3Y!8WVHhNVAr_=HYrM$f6-Qce>OTtxe2zibSlP zI3`Pv`i`Q^W3Hck;32=4k42()Qe0eFttUESRo{Fyd%joSd|vZt^da_sdEV@F=r-FcXfM!sic3UEH&_5_s9O z8FG*uKhu_(r-+F38Os=4T`T{T6Q*$CG7^0+oStc2i-g~sa7ir*%Q98^?x)TL-hj>* z!Zx7CI`ymab&Cn#xNU77cKr|;vPt^l)68Y3LioR>962rfpg@ z;!jYa0l(htYCl;ZYb_fdFkNJYi z@Noy6Vg~^_w|!=|4IGt=Qg1|O0G&X;!eQx&1QO*o1UU8fYoPeME0ws-au>nWr2=Ok z5kFsgDWBp@`q*c1KNhLqAO8jJ?GbLiz$)-*!Ke7{ za9Lk5oMJoSyRrvmkQtGuF1p-zC+}m*Pl}rkA+6ySDsT^gdRnn8OH@Wl1Pw&!*0V6_ zv4Fe|Aurfc$E*$X9DTvsLDxknbp+Yyd9NY(vW}8v@o8FBv0yVYjKl@e%6KLP~P)QJ9Lr+_M|bOlGN#5ly_gLQ5S@oG_V!;dAiKsEMsra z{af?XUY+9A;gB-7BDurJQ(yGHoRw;D>UHc@eG0^36S>swF;obxc)oD?D|y~-an#fx z{5fNyA~VxSMY4&~<|DHoN8!B(&-%pkmJDu&;T6sgV+hy{p}ItZ)kS6(Bsc+TqCr(f z^Ao~bkYu_LMwPjFn7kh2W1}~2h|-5bjjY{C`uvoTPiig7ch1_ny{%_29tgTq65KsNjIR?u%C+;Qmef;1)%S!c{*s!w;lu64LblM#) zWOp+vBvke8es-9&GsRNJm2zm;j~8>#Z?z{FFpdo?TtPpXxEuQh*)a8}7R*3dANeFD zb@2p5{Km-41zVzDggVM}J{d3HyPtx+6id0Qe^2p=OigvzwjE}o7*b=u6?7Z@_(|ho z{zyY#ZnZtmmo~~plk*aJKPOsQC3U8&QeIQITKZbq>h`iTVhdHXv4H?gl*QIcigcyW zP}7j%Qv`acm~!l=r8zMyZIefgB;5sfaz5gE?*u8lIQ3E!@xl#c`Vg|qNV+zLXp3X2E<5XnfZmu%g8S@UE!OYAitu)P=G`nFMXwH|;hOr(*^o&t_gz`2a9KgT{; zaQ5W_# zqrttar9wy(MV_`N2DJ_^{^%~3;dRj28=n5@lbc{-3WDB@o^Ffky#UMnXrn)CtD>Z; zYP^=JYDc|c5)j?f-E(ZP?*eH*7x$;N2hK7OwlQY`V&Gi};Ix*PP1D_$Nlp7xI$o>- zW-zL#*y!k1!AbxQ({wStGxFm{r=atw0>3>Ri+@*lb#--aKyYUfDDx(tE|k3Eu_6dK zJbv4q+U^f1V3nK`dk@TBx}f&m&?;R`_Zz#(514M)HVyD!8<8(hO`dM)d8LF>aNdvt zr=buDz_D&@Y`p7bPZ5-mK(v2wK>A><{SK0ScQ>{^Pqz$HNEd>B6g$vj;oxF-oCt-} z@g+KnAvXn_5x2XAZ=@XsMLLIqi^D@ilnSiCp3t^%Vnhxq)xhi}Z?Q~mh?aYjtWQJK zWshZ5jMF?AX2I1h1K9qgfuf=C_6>}=gV!W5L|r>7EhLXPjBrol0Xuo~Y*ZJITR^HJ z4;L=oQjPxWQ;)kIRJZ0f5N(ffj4_W4u%ZVVa1mMs`flJx9S%_eXRUcr20_78ARFck z24IcegZ@2#JY%I=z|^}lIS0Gc6o3IaliOju)J_TfPZJXPxj1lq82#*`4^ZyGxN(`1U2YXlTz~%G_Ax5x>3oo8~TY+Gd_iS)=dpi%n9-gFFqyQSm?c70!^9sP7-N3^$R0G{uF$TYEyuV&;spUXp zB(y-nxL@)BG=J8;u*YSOkSHtv^$oxqT743f>jDbQ7Oaz4yVk*p=WRN$gxIu@!M}#+ z+g@@>00#K8%N5Z<=P{kGv+go>pxAu(Z%d#Uu78-1MYM{B;@u&)nl$XLN5MVhxog{R z<}22HDLkqIes{%}ra^!lxbzWVxdi}~K!&nE#-nsF$Adtu{3l~Mv8 z`A}{k^y9g+%J4JB{< zF3vhq{De-6XH8bZMw1^`!J3xStY)gB@~XT--y}GUD#)=6^4`h?u79;Dz&b98m>&8N z3EvZ66HUU052Mr4`UT@9Tq##}1M|l# zp9C13p)C_DgRdSm9c~qKHMR%+9VKBtfJ4OZ0M^`4O1IzNk)1_Hnp3e?<7qY!Yo zJzjv+NL4)$<&+IQqf={Y&04B&wV zDU3V*CYt|fTiu#X?2_^+UBNO3e`kej7hqnVzXv!c05?03EeObWn+7awG|BS+%%$Ib zwT=2E4);G47f0#coR9tTssqs}g3+?Mt7~h&O)SXs&z2$BNJvP3rh_fumEcy0%`|3; zl>xGw0#9j9+Nf(t*r&6L*6t}hI1geBIkpC`RcuNm48Jueu zP$6#rvIVA2d$z>+EA$S&SU0)*lF929klVqyk(e z0%=PFi`3r+d&|&J{QfqG8`TB)I?R}VF%2`iuTS2l`O4+j`(+F}V^tkY{!zpN#R&Zx&L(~efT-o;&6m1?6X(`Ej%GzB$Xx3{N>Bt*mi+6ntW|m z?hD_FYZKkme@g0orPa`o)ikB1>>2jhj+|t+S1P&g>C&t`;F{0Dm=Wsi*ZC?{loZNT z5k)7Ec+1SyK`R1r#v@?Ix0ePwdHjnXt&MnhA&Tq+!Q zI@Pj5c#GrHw#?c}*d46wMl7(ajvWj5J1inMc-~9@x?lv`!$T5XthxWJ-4yLrpLU)4|jr)#-EI4RShIYScP9I7#GdFGf^R-sm4cFfNdxGSU_i z^4PlXYM(esC0@M3nHuMMUqsCs);At%-%x>2enxjg?p;@dEjZPkeInAAV!iOO_H!tB z3eulSwZtE_X?D(JYqs`{)q~MYRYNe0^nzOds#Qc1DAr*?c4EV*SslEdm%oaF?^#&j zc76Y)YS`&l#HUl0X)DkDl_}3zvZDH|KrlAN2iN_?nyquZp~kS;>w{vb-O7=NS!8P! z6Lrp`6IKz|Uc&t?60B5egjtkmTf2oy(pJHOI*yg$sdL7{L2MX0^=!AT*SRBNLDKOx z{5;6*4XXB84<|U2H@3)cqvmEUUsLsEC*r|}#vte!RK$CdeIJHqN!1ut)g~}yKWXJr zYN|GMg^^$lY5Vq>MdQx~u=DJv{e)dbqEuBSoLrn=0P z*Po?+8u>dOym%!Ls9DZ4aXkL!iO%h^>yJZZyJpq2*GRhfaAl$6QQG5ZbfXM2BfVhb zuWiy!qhm9Y{-44NQo1?x3K6w`vx&4@u=5tE^sXfsUjfo_WU5R6%_Oa$0W+WD*US4Q`rP&KU2NeG4w*)Gz zWn5S@Oct+QMc_(aWnsDA&Vr6XOTRa_w)=TZDT`~K&&uQ&Aa*D%+FZv*95689YD1E978c6Lo1cyDP%Y?P886ptHSfaOc*eNaEtflP*2O_tb3qcR=Gr zRb6d!aZk|`^}K~SPDI*HZq~6;8Uo~#xHEF)K>56)u{dN*Kj9&fx}ck>S%>KwcHIQV z5tsK5qlCZSoSus`on7se8bj~+o%s2z?Gg0#cA6#sUNG$L{4NTfZs3ysX)aAN+TGsVx2b;%!`0-SU9bsvcNM8Ys zWS19}`=Z!!75yI-3_3w*`y79)e&vVSSo*W+Fj#;Ind9Z)_#gWop7AHgH(U4 z!hhnhkE#23Xu73r_A*Ym6kk$3URlV2RXuSWcIruP%ks1HtNV2XlhPvC%}GF@BU=N~ zlvE^`fy!2{g#)>b%Aw!KpD;#p0A8<~_OJ05n!6Jg z#`|7eET8IqZLhB$-U^G)&naAl_P!@0gyozfk@vmB%XUiU?ys#M$nDOIw8$@NyxwqE z8Q6)Gklusm>2_8g1aULVxL<*)HlI98D)0H=vP{uw%v-Xik7FBTf%@@d>fqr|9>e}g za})=tW1xW{xl9$PKGE+{&{s|Bl|UJlk5i36KA}Og(TSS)cyb)XvU=$$9ks!E&4Xjt zZwF5jf}Ykz5hsXtWr98?Um6sp3k3KUqH>oJ+jkW3U>ADEB)ag&stQNvj%i1T+Q(H7 z(rCQwU{D9q*SkGz)~!seiEd2_pf=olC?{ZlDrzUtkt#Pkld(ILuG>x0;mprDA0vPT z8=p6tpBm;BI+05kBx&s<>6Nn8>^rQR{{|GO0Zxy<&Kq`OVUE!02X&u|mnQXeWkbg3 zC%k>#)%nFx-oIIz&k#{;HOW|v;Rb&sdF&%=D=PYWOp-VKo-Im1E}TTfnrKXEwiS*e z+8b9q(;iF_>&RWG-5K#^xkq)dtUe!>YQv6P2ThidYcA6!C{?GaVw%RN#8%1-EG;RxrLOo9S0!-qdAI<%kIJXkVf0@s#g zP~LdK1_jmTS2_Br3~ZYam$`@zjGBCS|qopWFTI?4^B zF2|Go(=gOv(^rmL&S7ga7wp2SV-6cx!y328hz33EN-4K|Au0^afpt4~!m*_W128fs z65u?kEO^T37VsPyLe812ivj0?5W~rhkAbS@UiM`0eZZY)kFWeTs(6VXrzjss)V4=q zbfzZO88tH2;Zf}pq^GtLxy1i zHP-fk%20opkkykEIbW%BtSQP`t7_=mmVN~j3y5552|9BZC*UlWGf%q?h+Ppw`&c{`GnMa+>77-cm)k*t zXeU83e-wG`-G;ay;j~T5cwQHi%miUNF1DW0X;KOg4k)$3Q3Z=s+h^6@h@nf(xSXGw z)tTvaayR~n4^iq96Ma{!3RYL!-n--Lr5@MfD&1vR1D#wPpBd%pk6Y3z2Ab0BYjy3; zko@`@UVgsB5;`C9q$iiee>2q+IeigAk0G54RY`}@zUiG`)4QPnkw;B<1v^i!bAu^i zadI;-rQP{>?_TfA1yKk6ZH*XIUH!8GlG;&$j^4Npo8?^ZOO~6%D>Wh|a+%2)tj$%> z+%hMzAsL=-F9F(|ctcaX9lPHc!N6qpv7- z-9Dvj%i!8Aq-{<%rG`f3xdjkl*qaZE;FmAVL7XVQnDzgN5K-2w@CATY|yy@TYL7m#IsGN$ZB;NCa#`-^(SWKr|Vxx$<3ES z1aFAlwcL|WYXfRP`nUSrMJ^j`G0lN;;FDlXyR%NqoQFOb`RB3ia>pwXTYW7Jm_kYw zUA=|qcu#H3r@N7x(8!~sN1J`Lc>}3R_Ew^UBDJB*O^~^jE$RG`9BtiY$h*5Fu;nKf zBWr}kb~;>`1FTy$5D=#$r8cHKO_B-T7COP2Go)VuXNY=@A$pk-!0m~)U!Zj?G_0*~0bJ zrfYeU_vl!Sai@9NCYq6%?z1vlRE%!W`OSnMP75;*d>h_;(s-g+H@a34z=)65K0|KR zLISq3S_oKx(o-pH=4dMXT(;cLRTvHLREVw6TPWcBL{N+yjykUIM~Z@()EGqXf2_-t zw(YcVg^5g#q#F`$a&aS+U9%SHX7$mMi6nHc5(YErU;w`vaeB$qNJ5gc7&ZS@%Z3JG zPC=EAl!ZrKm18UTJH7?&l4^Q+Xtqg|6st|#4&CA>y@vom(CiGrP zHskTzyAC{2Pw&Df(Mdg}*OchVfSV{g287M)&*ch*M!$=P~sI_0l zg*{DjU4gpu%wEi|B*#5Om8YX~3iqCDN=y-Co@Z1K8i@L+#I={qG@8!P3e|C1JU#9!*_75EQ8)L**sqbe_-FBk z?OgVH|8`Tgp|&`XG{kQQV+D%A?lS6=F(%x-E>g5g4J)HSAvq)^YHX=80#M&GUy}N@ z!U8Nli*%+pR_c?)Rr+f?DaEo*)_!;ap(kFYYR>C@8o21w2}2%!f+y)czFIqGO?OM}nAmO(G)Gh%cH8d;`o&#FYkQAL53DGH^oKd_H-rer+pABt z)Ue^w5MhmKsLEPE0Heo1y28dsB@(zZ|8&oHN08Ce&pqzZT!xt9FWCp|V$u?opmGF0 zG`Khrp~b~WdZzvk7So2Hz8v0r*C4~H{tm)OQW>=dztM2|vIh!n?vv<}Z$Hk~#@h^% zbmGW$%{>l${-dQ7lN2Tf{%vd3cDV8Y{k&%YGPM*BcDtHdT+xDl#gcr;GzYeRGx?;o zsy3-lf*1wGGvGOAHA&v!JY{0*jQ~2D)QH>?Xos(eXXj1-YA4mBPlJ`a}s! zm{xIe>*I3ly)WNY5B=fqlUmGlm>;}Le`M5ms4UU@)$P+HEekhwRCerg0xo4pmpS5X9!^hA#f%bRSR1;ZbTutS zvC!T7MvroXS&)PqUQzkBKk8#w%w1cwc{VXxdor*W$A0xtE3(pVB%IEEoDd`@-Tue@ zN`ppCi@vAK2l5QwujHW!E*7`^)Rv{z$WB9F5vak3Lyy2Vwg9=^ce z?B(Cc^yJjP3z0V7plKufAi=^R!Qv_8rJQRpSes~@o!Z6^jLC(W`o#NPlC>juGK&c2 z=p1{H0YT*H+IGm$Fi{I{%x?R^lf+5Cu0;92tfzBmpeWox7)I_CF0Q_7CB*4pKy{m- zXQ&IVI6K?ArfTB`z0s&59{n&_vU6~*(J`-6*a=S(#bIa)%J#2B-OP)(!~+F z(QCDuG1!*O?_hE^IPP_b_C1@ZY|%0!KZ|rE#?Z;tUgj|+G*Cf&$8?5i$K=&>%E&m6 zG8OG{uK|M0PkVPzSv$E-(ovJvqwrRoM_lZ%1O@BOErdJIv4S+5mhw+iGnvfGUv|s5 zs+7Aw-oObURlb946n?g-PV#dqkck81%NKvMA5HQWgHI)q5t_TT#bBT!gSiKzfQ(pG zbBW)+;vYA`ym-4N^zrPqzW1$Yaj*F-rEiyq|M6n7sSUpryxRIGyUPuh^}h;*Pd1tZm&-!6_gMR-9SnwVsr6K*x9N8)1KQV(XTfwmRrHEsV$56+vY^e0VPWVrTR;J(9DqCp(-|I z@k-Vc#ge^T5`~9eGxV=|6Dq%ykU3a@8;~Py&CXQBByqaiD*QItMBj6g2Y zKGE*4gni!<{d!)h@QTj~cW-C;IXtT@9{l#+tL@i=Z|g&!ay^?ku0*cy-8-o$x4e4Y zS10QW!_UsHjZ?6!|75jAzq*51GNaCRUVJfv&;ATm_9GP}5?xaVR$Tun=4kNJCt?2i zDT4CQiO=D%MGTnMA0n<&)$z?)jO-Lyu~{J%xiDdO+0{-lg&wLOF54e+*y1xab;B1> z1?=qvgUAh^q66`uQm?yk=(uNmNWYl6et+ab4Ur-F>*OivA>j!dyu4!fKgl2+d~Y#k zceQDKd5&|_pTm9A-5@4n*I|V0$6HlN01qSG$ZX2;|M^ofa9=DWTYnQ7rete$Q_6=D zF5Ov@{VhYNWp6?C6bMTL)}gjsKAP$N!z^Q6p>KU1-+ojbWU&2o(2BT&{ggjmP(Y?;FzC1w z?^jQ^ohN$tjmoKWUxvwb9}$7GK15V{L;bCc?lwg>T50d`6z1%T$&$Hn9E4`9%x7Hr z&ta*nQ0T$;XNCHF&e2AH>tXXT2c4Si6!ER4=XaP=>(3lbm^$qHS217Wv!CN*%yZ&> z*`W6v*4}2PXbbu2f`1Sgvtwbnm8~j~{mW^4`||WzQ)Cq7jLLz=0xK#}--2VtbboWf zc-OHwxW>j1TdwJv6H4V7m+!Sllis#A_4b5VMz=A5zWRP3lIc$h1h=!A10((>u6~!&~fR5&o4>r2z(hwg3o$ zor&GX>`n0IQnOhHfv8bo_4_YG-b8d>q#-lFx$U)ux`X zj;xQ|e!R3p5u z?gzw-G&4hF<=}W8KlVddDc6*BN{pkg>|LhJE2M16kbQO!JO|LC7@S{LkgjuB3ZRpM~ zilMXPJ_MKBv((vkty|iH*0|f-vbt~IvP3k^dl)`typ5&w(3f0%d$!*2%thU$1H)E(`)A#ABkexy+`5O6upibmVt7dbyetbSqtv!(*=XAU|p>9o#@4D}EwaV$t# zK0y-{!ptZOZfz0l`V)*-( zhStB>jemOU&;J)A@sGs%?xOrFWbSW!Gry3icIm*#0g-uC)_)GiFjEo~n`&|(vh=n= zk^NtM7>I?zfLwQv@I&+`|4f8m5sgkyEUc|>@p59P-Q~b75G8v2f4{2DB3c^xUyTvT z|7y%=lJB4AHOR0N{Tato4X*!!=6~_)nI3V3AANq-dC1?-%{s+2FhX7%nF}+2C z{hG3e&VYQjw@cj8=^xc?CxK>mwMGpK;<|Ej)}nhJzsR0B91-Qq-G zqWwY8+;#x?6oLLCs-4E$(>OK@+u(Omc~!+`>YvzcTUrd8c!~yk&{I@EAeesS8523uomADVW zL{5Z+S)s2fJr2PLf`%7;J`QU(GgBG^al!!nQg60k{=ulSdj=$ zKUc1*_bI zE=SAkqT0AB{mvUa_M$|S4whbZ-clDU`45A7M%rj|$_?)rtjMj9r(_wk9+#yo1ZF$u zBnUZD=nzpHEmgaMQK47#Utt7WuXm$^()lHTD%Pq6M?#=)$3H9a7#$bILB8?{TYJVM z=yvjDJfA-lc^8|*s(3lZ+8-_Asx%HM&$B^eQmlW=jIr*XAKamtZikN#ic$R2~sz$Dd;jO7c~uxtvrzkbF+O=p5u*F(q>K7!d9eReIv?^PO(mmrqP7bCYOHl|#e zQNF9Z2(ndXwA&n5ohB`$G2dBkknARG^&43r(z-zWSsy;`-YJ{fT^3*bn-JZkonOnW zZRdDF_Q1)8d|_dsNV_5V@!^imYLWl1Jh@ZM`0;^`yM31oJ04wmYiol$feL|(R!NdR zqP`bP*My9u%WdG}6Q!S!yT_xaVeAkvf#!06*EJ8<7RPC!j-cp4v+PN6irDF3t@Gxc z*HGqH+0*8&+?e*cApe=Wd9TtR$P8%zO7`5@~a7!2WuGNNS+6stM$x6?8*SZ|De20$S zHJB~`Gu`*Zw(fcZX+|=Ftn4|e8eL?h60_XLlepL_{~8)ulBva-MEWO?=Ti!HEPFsH z6x-{Jnkvueq^YI;9?YiAx6Q8x6NI&u32frMi4jILbtFXG(3UBkbU6n<5=K>HVXQHb zEgXzbaIQ)Akde=zM7;t|SU;|PDK@a9^!&Dic_hv|^aFj_xyDCF(bwGr+TPoWaVGVU z5CzB#FG+tAVaxm$cn|eL{NkpYfH}x#5Ve<3#5tO+yGOdk@ZHzEn1Eg0SsyTnMKgGP z(`e0(4<2roSqt@%-iUwZ;Y{4NjyZVbz$y!13P}nZGIe8^dx$~<_dfima1Qn#j_5yI zHU?gC@r66*Aw-d=rq9FUoj!;?Y*?U^kWYUA=1@}lG81O74pNQE;u$@%Zwj!w5P z)uYSd*c>KBsgMU-bv!;3^Ick3TL?To7Slvod4$h&NT|9?`}5_SS{U|EWax=`r<|d5 z2fY|;O-L0R-dkN@5f=wh&)`P%@RfdgetEBR6F*2Y{90~esW*tfAc zONG0;n^^Ru*;5r3Hw2cu#u#Z2t&(+Tw*#4wJ#4i|CEucus@H#1Od9-+srwR_>*FY1 z>t>j$3T2A6YkK;ldm((){tqJks7lm}Hm#u}!>0WSCn|?dnVK41I^NmRDjfB&CS+Z%R zw*BbF=L>va(HXOXi1}mfI5BO1>4(oxH&;AfM}Aroroo;u1lZn_y3s_Bi@2~s>roO` z@#DmC(%IJXJkDW@TxPvnB~3nKcF3S|>&UZkM_$;pdg_3LmYB^_=N5`p--B~*4~q8P znQB`~jmtQ@kiLY0AB9$iSztYvXn2*T-U_-Yn;)IJoeottGs_e{X;7NMJ?DxR#y7nQ zH2FvGUJ4Qbxr;y+Fpwn)eJ925~{T2_k);?a4-a z@H6DAs<`h_LVmnwz@kWY*Hp8y43O38&N|U%ED@@DoS1UC;@7-s7P6uIXY+BVKQN!| z(nyMp4K{R=o5OnbGOVmal=M5rP3J<=G+gjS?DxD9z(aR?Jp2Hl`fg}T9l9-V$1h~ zKaJnK&a{5c#Z>Q;l*2bYt=5lS(Yd-*6Sgr|z#J1=<9gzboN`5xX~KSnE1d@bpzt>q z#|O3bMY>2Ox=l#$5)|${qi@UIoTk#A*M^YuB)D)bwqs+`<3BTFvgMSTA3iHSOZ2h2EniKu$yJdD6UGKGqP)n)o}Wk!cHrJm)%`juu_lMgkCn_s{q*NH#Oi~QLeq9(rdW)FpEwVHE@OGWz_ zPIoG&rJsrXOy3boH#5C9oL4XAZMvK>4rt=;v<+@5_GpXuD9Uu{?Y?Wbhpnh_-(d1d zk7ZuaYn{+zYcYo8F|JRV=SD>>uNUp6(Yd&H@Paryx4Fy+q%3QqTJli=uQhvonN3Gb z(TtAN<<|~PE}B1z>2u6uOg(mlsrQ1&Fa)EGN{kDg(A}nuSrti?N5{YE`CMho7g^>V zl-%HrvsKnF7tQjXCry1W=O>0l-z>89hC=U;+J@xNPgyVG-4gklU$n<09=6+CqX$#C zE$9Zq!#SJdt6`xW%5HgY&B)6OYk4qie3&Y`RmFwr@a^e&woh}h;pV)ditWIGooca{ zo4rEOg-ViDSrQP_1zGJ1-?4^bN$x`}HNQhG*frxRKYDg2YtLpd{t`$&rZ?~A4)|Il zF5aWpdfu1Bqt6P%Lt7mU$LbjQKCcNdDBr2L(aDW<1+}Q*L6F75Qd)}ojG4>6kb+d6 zi---q;;KlL&i27(BF(KWf%ImlOk0;GCl0tRXPPv>vRyS-tI#8j`*|_099M4FK?gkw z_c2lDGmJe)hVD?nf#T$(oVKlB@f|@oA#0*2n^@IY*B5k!?TpRW!}fkl_v>B!RfVa4 zs@{EutKORyad)pgT!_*dJf_MyOAqQtu{)yL*%JGi^-g>J@X_^)C{|^EXE%TVAobER zFy#FB5epP$@6JANw|F(*5v+g5qqu-N4V_FkUG?a7J#{4X&SIP2vfhz3YN2UsZ|R=0 zi}%wj$%7~XB8E?7$m~b=C|*|)UawTSV`S%R{8qPLd)_ajIXlWSuncn77;-$_7IGNK zG(p~v+49ui-?<;)62hM)o8Y=V9TPVmFOwo{>I@cF(Z8xOVpw_z^a6XBQnP$1ynH6~ z^!T)L@&dM^q%NvufzRbWN@?8w_mOA?!yM)3sdiE>CUbRSB71R;r2pcpV z$!|h62Ag4Uw!%PvP|_}^AmiqaGlsL+JfQ%qmjwIrOLk_UYHX-UUgku^C)c1WsDt_> zrie0Mc~eN|igl-n`Tna;H5LLsA8tF%>Q!zAoN)9qo;FxMBmFLm=4WKOXVh}Fb9_68 zr92&|=PylmT4#ia+qPHf$tMB{3ERNBJZY(VP8@d5PHZUNFSp8x9*%2X8zrlPsA2P< zX@p9VygpI)K%j;fe1tGo*!jdfK5!j{zepGh&aU$<_%@(6qKmXpiTRxkmHOq*4%sly z+7Ij4j}CAnWe9i+cP^JR$hi@33|fdYI*grcES?;-7fZv}et!CURP+<1!^!zs5?ufy zj1FXZHP-Sz=>d;>6oop4TTbge4Zk;Z-})NO{6uHL(+zHA9dWApeX-6O(o`5VB$VcY zZM{xtHN8hIm6emcx#6ic?sT?}hNM3w_DBQn0mpi5;EXoJi5cAf@l>U_-LW%hIu+*> z1;7M!k8&YrWjW|Ea8x-P8am8jajawy0YT=KW7g{mcCNw?^$Ipb?sQ(lF+I0d=_UVKVr$_|>EDFd~xkPtXpu19E~j(oC1Wy)rx>$G+W214T- z;ZD!d2NUGj-dUOd_*W(sT86czJ5n&i4}9+LJ1hNKLanJ2=&S{x}leks5aC{otA$?KOwJ zAa4U(+n~yi*?OabIGtW1erWl0YO{V$!E#s9w3eJnG8h+`9x=AQiXq0_$emQNdbbgU z0~uZ30$FHjl%bPFb5>jW8euCw&l?pR8{2)eYEG!phgWQ$>O316WS=<=OV zkTrM00ukjDbo7mf!Bh8vtkr)6el22uQz&V2;v`pYPTE+{N7H?xcTA8Mk*OaLlRg>r zj{%yXTFaLPNk?VuBt`KBudvV);U!Y&;t%7`M#zMPN`%U`HGlbDx{WQj`zFHfSd~Fd z-|qM;HcNy(y~N*KfTwuE*OejZazKuzBsxi6i>aeuE%_a{y26Lp&^_LIf8T)^Jhq27 zy=ECJ)-s>*VAJ=({TZCGCVif$P4WX@!jwD1s>7fP&A%abF|UPIyGKUEkHV^_xJ1y8 zo}TtFisxO5!b#N3V4irxALXRF~_s05#Tw5u1Ia;J4klO-yIoKAJh-G_5ZlaUDozBy#3z+$c8 zCLE(#bBCQ=~8yX6R0x;1rne9)~Bld=NI9sjFwT!4Y=0jB}`?n5d(g zBcLzYu}crj8uug2+-IBm)qS~Ye}bvyL?f;Ct(Ql56{eoRg-N4RwN$WYG$kYr76 z0U?7V1f0dxO!}{{JP-O5C!C_<(_k37)Gm7I#@~lJfA3!>!(pfiLKQB$z96dLYO;62`Y_wU&B@=_2qcnd zx>II6wd#V#?W2-o!s$MCmM`Rt+gDZC?7hX&e0rvnswqSNI($8r!eryQ17#Orm?ife z-cYT*c+G#pNC5d{Ff>9VW1%ktLJcjs!_Z0Ae6OMjdm|y|_+6(bGukm4&imk;6+D;S&-lVyjjCMR;`7q9DG{WmfFtvDNNHxj+nJa=`mSJxy{Y&v*@8b5JMamemZ`B$| za`ne0sFqyL&uNI2#iau8<xt`KN9mfhHEx86Q9b*MoGIAgnq0J{fK96t4={gf%7|a zu9cFrY)O<;*^1&Nf_398F5m<8%ebH5vrArc$CIwWJ$zOR=A=e$ zX^UP|P9TQAVIKgcQayp@<0!(DMm5(SEs%8k)DShUA_ z3x}Ap>r9CN=$d(h5TbRF`HDro(~E(L_*>>D-A@2ht0XZhD}~R zEaayW_G&Nrdg3wRcr@ju(Ejxk9w|rq%0BshsKq&H_^u<>6l-Cm`O`1A+~dBiB2cw| z^MI#kTSq)#4V&|@M1Q>Bo&H5u3vc?(&eimY70Z+L(@*4d<`6^Kqh*s0`de&Duoy2b zkF8()dY)9}m&?;e7~cqrf?{`(}z8=Z(r}BsvQp5*@ z?P8HQN{tm>Y!xgvc$*EDpKv9do_5NE@Y~Jb702$+w4XRGQcSCL#G@#sVx!#B&ebKA zH)ztV=eZvl^9ydXT51dlFRFdZ^b0qato~-g2Ul6L^$gdHX{MzX zW01QY=H4N2M$zftc;+@DOf5IVYXB)z)R~lmWwY1xev>fbG&FD}Ip24OUaik=?s-^! z4c_;;^#gng?15EGl~1c~P4`enYYhEwLX4MYyVPu2mpcimA`e+kWEz8Q_MYN&9#)%n zYHUz#CQ?*u?a42^fA{37Zs(5=RCB4|9)F{3c7IXg`@PGMPGN{-=-05pDc9xoi zdMsnA)`Q3e=pw8`2$_S`z)yV&SPvTw$?AMqve zSK3oP{k(~2zakvu(0Tk&lKN5H2frg`JYv3IF_r4Ooh#e6j_!)hXY_>LbGOE!h|s5Q z@*)1-CZRE3r)HDJ^9EVApENLi&x|K&4xes?PFC3>%Fm~SC>h<@RXk2DF(xyb&GG*+ zVXP5=Mh%hcujg6LQAv1C;g!;q>Q#0Q?GN!B;j%}?zywqt!BJ6BU~=U8eRVPQ35$iw z&;U#5l@~xT=T~SCpv0zT-h`YI}>EYAv-rkOs zSOm$yRMVpa(5UDi$L>lt#bn!|v1cpSqX$fyOQ_3wl@8g`#_~HB<)>}2#yfi?5nQjN zqot-q_K2R0wiWB)iHY~_?)*R(ODUj;R!&9b1JI!M_d@d+IjLOZSGvW@Imog1uXCoh z?30m`M@K@!W##o?|Nn&nX$@1-bCCZrX8{HZ@=L?fKG^nz`jY?b$~Os(%GgS@wmSCjM_MkW@Zc%{P(P z6hkATp?!1dF}3W*s|~xuv|73uGCE@)7Kio(>HQ`#GF>K(mmK)VG&vSzLBWjJXwi4M zF8vi<`Te>swWVb99MsG`njf-*t2%2EdO#~ENp1geNoBnAiuve+{GD9%_Mny+P zIOKQ47zm*n{}*W^RflbSjH$WvZpzx>6Vlsjx!_lC3>_EcT6s(O>G4%rwfFo(&5+%N zpXTb{5Y>uS>3<_&X45o4&gc?)u%M{pDqiW3naelX?@Ur|F6yaJEnQ}Xy^`wuxBA_w z2dFR4OaPDuq{sdoK%?2D`SV@+s}IM1eg84jTXi~tk24b}C@G_&qu=QOfi7BF+V-xl zKatBnmmNWXlKpo~60)+!vp-A|b0u_E&TjC8JozOgB>r6TTi{yj^np0uuoQjYY5<|7 z?j@iIySmPV?dlf#Q@A2#Dva0cdH+B0NI$2pUn9l6aTh$SI}!vI!8Cw2Z)j);^yX&F zsjlXRMJ4LqNYEL~dLJqd0{yF8{^84_03b?!+VKeqT?bW5gYz|ZrlzK^aYAiq)#Bev zN+Qn8%v5VK{^jBC)zwf2ygQRtFIWPt0ch#z{~kksUnQvdE9Ut7<=4na5I#CG;{DGV z!N=Mg==N&$pGdqELvWTlcy}q`vJh0Ur7Pl~s55o4%^!y1l}mV_~q5udb-1vqXg-;19s=DcSP>H!X##D&0e0L1QF^5yf!RJyjCSH<0mo0 zrbTTc)tDhSQ#xISZFbD+eJ=NTecMiBY4hAHq%l=E2jBE2j;bP^`Z%Qf&;N#%y-5CN zk{EZg8!;L9qE67R1d!vqC$0CLt#dVw2C#YD9%}7sutzDXY$={k8k|ny>w|SxueoMq zrghp%9GFVaqp;T>7POF;7eaqbmEV1&%+JL8V|N^H{mZ|VwLVF4O0TzX^8EPIC|v5l zZlQj0wnlwPKCjuTfc6kJ5ZJpN#NC6(>@U0EMA%^p3WA=l+LcRpHsuNzusJ7XiK(;H z5%nV7Dh-A$P(3-zs9?HzBj|Q#!Mf%w;v~=|rIBd8rDr7D+YUZXZdt#x==Hi_rA~Q{ zEWSd-IdX`D+EgCDQOG5~u=7=SUTNO_K>D`p=Sjc*Qwsz`sO#~D;!f~E=8zN{ycV*- zU$^n<^#??9BdxwqNV$=_zeM=g3+SovN5X)l1N5>%%=?v=r!G1H|9Qu!rBi=dEh(`9 z0+Rb3yYcr~<4yPGp2zN(2}}3*jkigJJtf*5Xr3n+4Cs&TRMW~WyMTU7P`^9tp{(c$ zTN~i#3Byks$V8!2d)gI*s7$V)oky~7BR&EExT^)Lii6mDClwd@I%mV90?2CCTsqZm z==l>vX9r$xey*d5o*gpCJ;Xf@8IQshp6il_xUJB`Gg#4*w zc~wvRFWdb>gF5Ds6YaAh3JxP1S#NWpgsxh~8$Ttnz}IK}ahyM~O@ofC#_$ z?LJ1OU|@-Bb1sGYqu5tGhF>DvwJwR+F~4(S4?w|tTi-^(F({nL1+j>TX&n61U=@fp zkPS_9}$LKTF&rY2@NjpcsD8tEGw#UEUj3$K_*R~-Zv zWb9#Mk`yC@0guvk(IfIj>uovJVgEA9p?$72N$cj4;2;-Tqp0HweC|-K58%IG`J;th z)ih*dIVOnrtqyW&v0lFnMh~8u{tTs>>pcgKCfL$72uY!{b2n($I9I|s!fk7}bSc*y`Rg%?%JCW!8?XL4OK$)k*%0SoG zZmA}>NZgF1PTWQt+&l^c$g)BeNO!Ir*V%Fj?=V^Yb4!Z<3z>0_8G}vZ$fP4mHd3)L z{GpX?J9CD(Ztir$p87NB{aU)5w z=Zs@tzLz=cv_~Zgg^iHg+GtRfiQ~rIUD;F@1T#Nn2wXplEGWekdi1bbJfGp6yL*mQ z7Yh>)c@>1!5fCD$?MieV3;lVr(lB*69QD(eK8?okfx2%L0=I^dC#A$l*VV>ecE(2|v3{+|d3VCL(f&wvRY^O6+a8rG;c#8{>4F>tA`r3hgw553>$*Bv z)nyKp4Ig}_&OMaG@0YJX=5penCoy`c>o?YRIKgDcZb|fHEJ_wKFWPq4xoAuy!(G`*yit0%%Ut#CKG%T7hk zr_|@Sugi3aNS_kW*cb;SYbLzMjKrYm52NOvq6lvmGqeqk+Mip}OD^+w#0?x|JfNBu z>gWy(a_sAnwtq}RPU4R7s(mu*+~^XPB~-3M3C+z?$4so;)>mCqDc?`hIWAukT}+&D z-^g^q>P-1)AyN%CLzHE_VR#%)xV~<ePB>59y{}Fe(x$El591>ve&zni3wS)#em+9nT2q#yjE`c=Hqi z1D<{g4GvhOa3_Cn@HQUomuCUqmUwve0`1>R9U@t+32bz0geX{ZO22yw`fEM3J z_jTB`?W+$7(%6E$tcDf*3;*kL_wsoVJGLH}lp#-cjBwnA4f8Pd`6v zz8||oX-%PF4j0CqD!F(^2ckAR6*c?i;Dy%Z6U=3%VJN+D)ACc-epcLRj zD_#oh!4B-p%>3YE&%rgii30Uz@GCp(SWKxY6|PZRU=Gid?%Pn051Dn{p`u8~&Z@@R1l3dKPP%?CKWL$jyWuUl{OOU7PnsMi_g{73k z@rVW5k{~{%-ZAgOXYvIovwCoJ5dosFZ)Y9Z+qV3)qFH1vQ1@5x60GACC+6yrA(d6X zG|qw}kf-B$>p(|Kd-8)+txxK^<(CeYkU3l?Jymx+*|)E0MN$P1RA=RQ%pc-pJ(l#+KC=yh;+u{V%`EZuH zBb|?7vf67F@Y1@}PWoAUML;EWclua?cR}i}a!n{GXLoqZ1D-rB`Ep}l_p3Sc8M@%q zFVYOQ8xXlt(`9)2FequSB6XH+52oNpe;FfXAHFB~5rz!;?iRAe#m>5U3)7#*iD6!y z5g8vOAU_nZqEeBU71cU(b(|@W>v5pnYV>WcGGKb!B}|fI<*LiNlW&{9Hr-hHAh%A) z=f8cd35m1(+$;SH8JJP$>E0u(*E;LjdrHb!9OL5yuRCJdVw#mqHz}Jpf5fJ~_#sWO zwYKFgRcF!&a$3A4u~+w(dXDyeENTQtBpP96=a?W%>o0v>&Q-)QSpukOcH

i23%av`Lsr45s;uibJ6{YHml#$Mn4i3Cp=}wW1%Jw((a*i7V z!(92zx#+GMoJJ&!b@G%Hq0h5FI3oJox!i9IuwOv4R4W#J4~5PT6l+{KH3W&`f zzy4Z~w*MWTxuU%lA4VaUZG5zMma>p%f|$Ibiua3}_gs>ZenZ$94{W~u$!#BtA$eX^>4%9) z;EJQoMR$L;qBd$)uT7d{PN9L#5N)yo-&LL~+lN#McE=yj-)trZw(;rc>ZE0?gSpDM zYXW8ZROCmpV%Sc3aR%x2uLt$_caMbs#oMkQx=<`K8jxVBdYoH`HQ<)Jrt1(r4XT+J zJTRQBQW?Xf?=Xe)+M?xbxMkHq&FCbAy^3VTS#(6g49v;uxSg)r#|L$4b)EYWq`TO> z$P?id10Z_4aAevuZ6PF?6|SUx@g>V2J<#$g?P}~sG{p>SK03>ce>m_Ss6XyiH*-_6 z?d&%?NwdR?Ulj%GqTI!w^`C~1J8T=ogSHmY=4Ir{gYA&-iSwrG&)-iHeR}P7sx`0u z_4ih}tR=sJV8er|1SLXkg17zIa=RT_ymb2G@pD+*!b?A9s&RAy=qhgVIwZF#_{#<96lt{trl)gPEtUkv@3isgXCYiyMyC-$^hloy~PkjZR~(m+NVR zCY_==JUFLvQq)Fvjn0Sp@2R;Iu;~q{wurnoR8kd%In+v3i|ySSe%#r!M$6D3eh(hj zbP0;#xNv^=h3f~i-LCcNsZV{m6+#5Dyc`xv%0kG>r@TJGSG0^t=N4>jXPEiw4{jiK zd%b6$q$;f+s^5Q3ijD?XA0VcOu^N4~iEpzcV=<=5r_F~RIdR}na4HX&WPCN^o6hZB zrr$Zb!H{=zP32iO6gY${`kx#e-F%yM35w*6x1kJ9tPaxCUrg}qAd)sA5Ea+}a22ga zRGl$HaRlD_EOzFq=&ZUu$qh9LoifLaw%T(jmKh#davH>!Z(pJ@m)$(`a=GPGw0BSF z5)^>R%Tj3Au_3wHab(?`Zxy)8u{)pGy1wdyoWwDCc?T!)YGmBf$`+Y|_dhuuM``*K zo5!EVc?tBAM+o*$D=yN{4z+A~^qmTqG~nY{7WWr zNbdUDtbX<@scbz{q=28XXY@8|VI*u}sr{OxWAfJ2l#z+stz+M8)6pkdmWhBq*Kg&T z_a%*{gY-`A?7I6vTgc<8uuT(F>%lvG>bHk;~fE=VVBj8V!Csl-(%%?gs*ve+tzToiZf3_5W!~M%49rnI`!T7Mg!JSGkRoHzs zj3X$i1;X>$j6ODgLJ3C(w~k|AXjzJZQiP{~9tM@DhMVUj21W*~ZjYZRES;0menKtM zPH1%(X&f09{e%?Pp()t$OUI(md+4wj!z54F-}9j68pWigOG{p=ZZ0{~fo0J)-U78l zT73y^+Zu_Dx~PG&_rV3v5^znH#Xab<17*ST3QU#_Q4VRI7FAaoe{%tTSUgG^?%&s` zarb|5Kk`JKAfmL8(rdjq>t6n30tyJzX=>m1;YJRg2 zF9!V=!?5KpwEzn7D2UP=gVaTrOs*+mb4-hUN=42_m(4){mJ;^6T{uzOv1qtmzk$Kl z%V5W4wC9c4`@28v*1pO|Go)^a<9DW^dE!I}_&(lVqz1Js7=@0?>-6!kjE)r-&3S}f zMkE{u;1zRFo9^1!Xh|mCCaVvI_yo$J%z$NiY)#C(wL%^l`9XH72)eiLU-R^Y7vDYOGpBROTOF;S>9?1wFcaXr zbjqYNVl@JhLD(>oocFpCcP%$=jJa$J>XJHfsHe1af)+NLq1&^BZwVsozR@M2lWH}G zpLLF$_JVI6e7)a)6Z|OSyc;f^F(NFxf#d4&TArIJE*jnRT)EhAt5U$e9cwgD_VuFa z76a|)0pZ@^cSQ$v(~ZWspR|e1FQIei7+(4{PFg09i|APk_G?XGmAA+Ts&o=O#v2IBD@UH_I1?6bnKkqF0ab zFACssBd)AtFed2d@EBSRBO_qgUuPpa7Q#^Og2ytvIx7h2j>S;0RCpupE!@0WF_x({L31g+F%}^>W(2NDj(vF&G^VXVQdBKj%b`i7$+xEb3t`0;oEaWK z>Rbu44So_H2ORBp{`c1Xii0C36BhaxgBXJD3;agooMnymzVI(XISz8lt)MCtr1#Zl z!kF|TTV2vpi%j%at_iD!eg-uLrC~YoqY8-juR`Z!K;ge-erxQ&(UP)tyaE?6jL#C2 zxZJF##A0gH|3E!cX-HTTx_LaD5Ndj1_O)hp1Y28w*hv>IC2)h)xFzp;^_uGT+}Ul>|LvQX^8xemgfb;%0T4H_3lwo}ypIMSXSc3@c`zuKkf#Ni2l43q$13QS( z@||oyixRmBUm}gFx`cJ+l7wXO9*LVFC`9`z#*$_`-6LZRygXq)=_U0784fn^v)A~A zJsR_9&UE|DlNG%Ytp%N)Gm&hpmkymx9|zaBNSk>=6oP57&O0!E1=LJ%VQu&q zpqkC2Vkj}6<8`?`XNT28rWms{GRQ*lZvmG2x0!nk*$K+bQvz}^a|akLNr9mY(iXGl zdcx`j<~#Z7AytCdn8w@olc}2wE_~dTcH~yf$3IgI%=;u3zvjoC5kws2o^U4|kCPbb zGkg!dA)KzP^!6Sno&^zGnF!STD=!Yu+4#w%To*ortExOTnjqRr#pL6%eeGqAGN)nA zofjJ7p^?A*?qd9u;OgV%K$3s)C1w2gnLhl%D5o+Sj4w@9acp{-mc34vS*cP#N#}(5 zteEv)&zhl*DgO_nsFK*Kfe6&{xU6Se_R=TamIJ$_bW+*2;DaKy3UA3d6!)zcT#;sM zB0Ch%nGOwh`U-m}j|pEqh`&zTBC$>({dle%~GWF!a-$yRFM2|S=Sozu;9aToj;9V|6K z#)lJ8aLj9`N#O5-2nwJ2hBfp=7GY?)l9Qmll0aDh6>DgtZ>c(|ywyNDq-7xEj#H$w zc&lbZ_H2#B_dKh|{i*5DU^UYDRhR&6j88v>;@dI`n+0mG=AjTkkVcKKd+5Nc0by%& zLf14gffRc3&8aK_Lmq2xMH-jvQl3{x7D@VQ*HLg1)2g9T3V(b=aYBjs4kDzqy__*SW7TvGBqlwH882+X<-vS^_h3N+=SHsvWFYB3r z{1Ja3MUB6WxjGVzr)e@#p!BPqeIbAYih<~F%}w;zWwzLEzzW3EOJ^*&iBKQCMjX+emnoEhse@{!*RNwycO_^T=wT|1V1ps3=#`6_Y`CTxyrfP`Z%XRnm>WzPs zqoAhNXHEKh8_joP?9tKD|Ncrwgo+6dU7gGh%zydKs6$$V+tuma(o=d!2qGZQY1VZM z?wf2G;Lb4oU)dFZ9&VLIgJ@KN|CfD~06@2y0uZ0^Df|Y2w=O+iRsr?Y&)b-t^J;5r zo$pV&0Q>@^6>Gdd|M2&1ne>AQFu{^v_0;8teZc>U$zcJY4b?ZBDP?m9njipa{pxVx z7q~c3WIy>`M;3tO<>BG^OOn6CdBXcm1bch?Qm-dZ02&+sOW;2r76IU1ztA8i=;OgE z%ni)joj!^IUWdHDN z`C)K4*iXc-6s5kVpaG$iX{KfKx88h<>nnEwHhsY4wb$9`*9jW5TUrdDBO z*A{m2l~?*-q929*5=-RgK%u?+<5>m(YaZR$C(0t=0Oy+}`8wRVESK#meL%T>IQOF3$Q@U@k*D}67UIB0{ zM&{@Hfs#`|QvxYbzbR9HglC^N-AkJ8>5UkXc2XtS{7?m=qHYlzpK2NT^!>OCm#fUC zhg94;?<`nD%9`fy5K0n)~vsZ5cLREyILER2wsL|E^a`G48OVWhmbC zd-AAD&S3{eFdkkMe3`gkVwcFb63F8MpAFC$sv4h7jvfb=Fud@U{y8$$7WFX**r4zQ ze)7$@{&)e%ZuKm?Oq&JgAGfQ=z?Z6;V?q$nS1R~jV6Q)x0?`@{qtW`98}~=0 zfv5u5&JEPAKhpB{)%6{I6mS-p+rB@`2)evDh2XtCtlCkeJoN20-3j_WrXHIXFn!wv4Vn_<#K`duu5^@I}OGZ81PoG$F=ttw%kB!~P$ zgacf~ZKFQ$xpl50N<*_(wLkXzj0~L>Wm42y*T`Y9!9Nqo!pKmQp7}~vBXCeO;e9!TB6!;1c)3m3&*HOBTtR@90&WEr;`o`NUGRY?LAy?sKruoL8u z!Ay&S7)V|zm{N`Xq>!clsYbSlKNP4pZm-vNRcVy`Ru&a%$|EA-wY|qZIbuu;bn$QL@(2Vp zd0>nsIeAQjRizPKDc))*5T`BDY^kLoU_tPL`}D}x55Csa#66a_yls7WXs1^TW0G3# zt^^IrbI6}k#aJMqqjqtB=tzncQw|_Z$@R&^ymXM$m=c>VjfzhZiQa9=rdoAW`xr(e zz~I2{0K@!jAnBd3C4plF0&~!{tErhqD|%*> zZT)NeN5|K)ga>JYdcPHkvn&EvYOdFiC9q(Pw60~jZM8(J9#cE1uP~lQGGg2putsr8 zzCKQ+;jt`!*}Fz1rf}ZHDly=qLG<+o{GX>6BMYCbbNCyXReF;-lIL(4zH)bjb&*oC zvfBJ|ik zz$ZsDqY4bInM0x!c_TEwlS)O?iVv5ru-|ak9$+i)n^6=ViwWB6K!KYw-s%WVz;TUC zPKrSP-B_2^Uxo(o^jWmDZ_H?(q)sf!GUwVAkJ39zQZea*lR}%~a&%E6+&xgY{C+2f zSflJMRUnZQ8NC|J+@Wz5TJaS;Kg4^KiKCEiKvfWXP16OHhJ+sl|8RG?oXU;RVW7u6 zT#Y#T0hBww&^qWb0gWfl-w~7hk(VEvVRR^pIOSVa*cS^?Y>nPI=dN-4-2U{wZ|xxM z$1%y>2VR?e!}+I7!ojzxuj=9yXN9YcN@ABFY%yd9qf{ zW?Wp+HxnwXbBzB?llowzpSH?0?-ZxdHMOe)$y1b%?>iTMI0cqcZ7?6?qLrFV!tW#c znJb?R2oZ0j%eqJ9yJ(W_?&&SnQqs@a`w0qopCBEsLuN3=9loU+`k72OLGt2@awU3< z76s5E|Cs5^4KyFsdNva23AVxZ0MVaE2H9A+bs`)KyVFi!3pk-2ZDPD2^yf$;O_43?!g1W-K}wVx5nLFr<0SM@18re z=GXkVvsS}uimvLaS0CGZ;|4<5CHyn@cW6`&{a_^NECLf9PJPkO_99U7bPL7=Uyl#|lfy-NH=~=X4@njJnZyx? zjaoGXBLt)&cI9qHCT)1XBOmZMeg9-VIh-ZG*rSs#blI-#w3okbE?fcWR&=lctDv6WLJ zfmi~aed&?1?0iI}-$v7nh?;u64i#borjZFWNV^O&UwGsW*0o8!1`t1LS$WYxXz)LZ6U+lnU=5Q+uAl+Gif~AbF=aRwmh9h z4{!A9TxE#1@@{+Ssyd|rBWu=dMRmU3i-czt6M+t0F^Q=n-`o8|T7&UKZt0+r-a#QT zcmW|v+vo&wF=yB#5gaSYzC_-^#4ntt$zJ)TX=UxInG(Jz7M`x}B!fX>p^4~1TmZV+ ziH@jdhN`;`?M-^3|Ew%Ii887BJ56!#w3rOLdce#{b)Q8a+=}kcBLF+hR+*>Qu1O7{!Z9 zX}k6~;oFy!*?jWW=#Ajsd)`~qd*sxx%9pHnteO0g& zlw=Ixwv#y6P(p-Sxq^X~cs$kjCW#IvjH(|^H@a`V}T_CI}1XRaNs(#L~LI+vcZQ2& zgo1YRY8q15LL*-$fCjw!4X}9@xR<~>-QQoZ$8_Kox=_p{aC58v9lP_s#^+#6PaNra zr_=uGJAR=gG!_;IaTkp92o!{mRwOLMka?^QAvz#VF?C|(G=>4PjHIQacOmsj3WwYR z$_?T2hd?y6652ygRLc}!DJv~0a^L$L2-Zh=k|$g^(mmaoJFCxsH9lts9+)Crm1I%s zXv;!)Z(kay)!KnCJ&mm*wN!+`nFeYd24`Yg@+Hgt=WfJ08tJ9_OtRepb?(lL;lp1J zj#&doHVIRRu1DXf@LpHgteC6u@k}4E=JA@*XfFH5G}-0C7IG<5HQhhL{Hh zFryV`RrHOdW|fEb^N5NnhfY>0N>!wa{CMO1fLnZitRpifQ7A|9jjJ$<%^06+h%xJ- z-q7fuC8W+ENrobSJ2La#d7pt=HmsxpELvW3E}lZ4yktiqWV{z7Crzf3`J+0pppoxo zJI@R}T$9Xm@&7+rol$q*tc|{60;vcL#Uyi{OsjtS~|zCAmJc%G*6IYD|u~QSY>(xz>!9 zvW!at&M22_K7prPV3ln^U$0bds{xZbO58+tMEB?$77zMscWd0`#2EV8;V2I20;8ku z)w74kCBjm8%auX){+UDa!I#2NJo2BwT7VEZVw!K{IZ?RBZKj_T9jG_HS;D@Q2LcxD_$K=_wH=oTTX;AJHV* z;6`Tz3}unxddh7F4LzJ~k934RcQO&zd5Pq|{&afY(ORO9Xwf5TP#)G%g?gUxY=xsr zHOwkZTxh)ah6Po620QiYuVrm4XaxYd6!X{LUOay$w5K6rXr2w(3kAIWFZ<`_ zMu&6BCXZAHY@$#QmgKPbh2?cA;od%^qJLlRVjFSB<)FX}P|e)iL3uk6F*-}|B6+q( z2hUjx&RCdJ@^c)t8As~(@9mhILr7`Eg@;8~u)7z&EZ)FEhoY^`BLIH5BR6P9v1@Az zy+NISayLnMq zIX&65JU#Z+3RYPN^347P`hq`ZjBQ6b>LOpU zA4y{R+g~UxPYCUdPLTgz;~&!1u(0dAW(d=hkU6LH%H@P)PX7B4=pFss+^l$lch*lU z>7|*tL)jLMa>UU(B#4u$L(0Q{DL(fZQhZrm;UG=7t#lF(ag3|)XO#x_c?crP>Z4{} z&hO1meeU62QX-y}qr@r954V|*ILh%ytS&RrMckeuT9zfUn4ax5Sxd6IcGI&Z(Hruw zcJ$0ir47x%$t>r3M*&;z8IXMTWir+#=%|L=)cj}Q>DR7MCzgb1oy1cedI@Lzeenh- z{cZ1ROl@4$SEvg#yZiXFANe!jPhVt+1!Klu!he4Sn-i7%Ny7Qa8Z!H6xk(bGa&lAS z=p4q^Lwg91$kAYN60)P&4s{3BZuN4}9YEf<6OI=V z@+Z^#Jqgu`{f5HS)u_@f0q}jz#jVnZ4DHOy%$`(t~@eEl1d~O?->I z{#kHfeA@#Q92f^?Bi<-i-;O-4vFZ*?5$KAr3BFlbVFJ%A&|}HB`jiTMHMFsLVes3? z$fuzStJ|GHHwMVA+*RX(nU$qjk}PG@#5z*zX$;tWf})d?jY)6B6*=HUF%6knxu8M{ zF?RTdqCjv3$4EM)b^Z{!(*Jv2L!%&DJmReR*ewFW+X>;X>UodKFCr}b;`tSXT2q|F zEtWyOMnO%jpR45*8A56M_YgU8e{HINsg?fVanKIu8Xo}MK3zF)8fnxrgJ++E3m*~V;-y*$RTlfWVrj@YxD#l?b!5>5EU z;px$}+wd%`mo6{GhgUVvD@>)OD}61=Jo63N0D3`g(cIXG#KEvyt)563YLB-8ARz|_ zF5(EdjFFz(Xc;P_uV24?`}Wu0SSv)?rs}wO!}P56!`Bpzjoiw4BjVbNjujS97@t3L zdIUp4@rH)fs;a8E{z4XIXiwwOr=(LRn%U0C5Z=#t5i%jNs9$J7-w{N4;!5*ds@H7B z1>DY|;o5`&o0M#=fzeNXIZnUfhrWV{bmTro1m>Mz6onU}l>?`zb*M32=_fir%_dyT z*}RfT4vkB1_v`%y^SpU(4hk+TL6{!v`cu*#c| zHh*s4xFQMt4QkZnenFcfml5P~0O)9R7^^pD7@JqaNF$o-!CF{YcyjCduZwlm|HQ)WQ2GNEK745Reu?O~lNs>U*f=XY+ov^l zaZ%%_>0HhqnZVyD`TumDs9b&wU1?J^i*9$B?XDuH>FhfS46y6j{bMK*Z+yMYc63aP z4-ogio%$0)xTBwEq^O}W_!D;u`T1K7{x`~y)U_e;U#I*$KKc*P@(*qb&6oIp$Aez{ z?{vMH0e|ad|GbK2CfgrsS8&rG3>V~rjJEOmQ8HA&L0#?;rBju7QHwut;GZARgYeX! zIzpO~>K{eoRP2zhSAN2f%!YH!A7YkQ85BZG4ckrzpODu}>O%^3s67g2gN)PNfSCFT zL-`#GgFl$e^0epK#ja)7*F_7yG>76u;bDg!2oyyk0FF&!5{cRz>};sU5QJAyf&yc< z<~P5`cG@qA@rA?gO=}=7Y;)HiFrIIOVZgp>r5@V?ijoaIvtN{h3~ASfd^A$EmG-rO zVt$Y0c>j?sujFSX)&>iF5+*qlGmc}^fwem)2u#VQMuJYA%e040pHtVG+t42*?rcEP zAs1-JQRaRq_u~CqJk`h!NvH?{A3-JWE$9@LiSdAx?Xb=vYDtC02jZtwFXWAQl&q?6 zsF~}$ju+cMR2=$Vw78qS_L6{esXd^mHAPyNs${-#9P?k|JDpvSd9p_rbw|yhgfK#1 z-v@Q+P+-9S;Z^{j53|eC4|MF>>LgP&|3rHKy+TFAy|F)?@R4q=h6YRZ^A=z4?X#H^sPzh%_ z#Q?=>-8|Z-x|YcqK0U3lvM4%k8=d@d(9^x%G0l}!$PnaijGrRu$hs@|Ep$-{rxhfJ zVanZ~?iK(~m6mf!tD`MROdQaj^g=W{$z$THTrlC1$aa)D!q!r!cmIoJ%;T-zcA`nH zYV>M`hg~P6w@h!ePZ?9f@1PHOty8r-gw*V{VB=n|Sy|TTC1y`G0?fBq9N$4{!ts-F z(5u!ZLn%L!uRM03HNk3&b5~iv5p=#mwwEL5gJCk1?%pxw>29Cr;n77Izr*_Yc@d5a zXnbX!S#!#4L@*Nr{KP`PuzC~qp0Ln@3h3Osr!K52L=|szVg$(uLfYCA@VhmKAyD|{ zq5rfk`mw?DFppgi5P(fzGMOFmY&ppCa zuBC(yL%j5^=9Ek6ScE~0d;7`oNcG;Wt^106VU3PA2_Xd!$ImfN1*zDmoRxdI0ZwM> zh)}H=mg@XH-sT32H?ro5u|qE5MeAd5sNjo+uNG$Q{X{*=tya)(+#kI$r|6uvZuJR+ z8YHbczaC03@2u_D8%;Ae!ERH{wU;QTZqi!sVvWK5Z{-$*@Q8w`2Wbyt7*N(#w7oI|QKw@1gtX#fd8VsQ)ILmUw z*1V-8P$L7;)Opql|GH!=$xr$_m+KAu#H5je{TPl4bFn8moc+Y)Sldx?N*^t;SU(t< zufmZ-QXbzRCv7V+1ERo$pMW&HG11SA^n1Jt^8N$-6;P|n4kA)V{lIF^$*t;!$ITa$ z?{M(VXeeTew`h7N2{eK8E+yOzPU27I4S0)o#Eq&x>b17EvcY-i@#Caxl=^i zH=|>9zNb|ay^;E~{d{CHZ7WU(Eymk5td{G24=@QQ5^%5+UZxmVwAGs|PgAi|vK%uN zd!MI0Saf7Ok|pisj9hjEuhikZ=)?AeJgf7=HLk}PUwpAgKuAerQi%0BpTi3FU}nNu z9Cx;5cX;Jm?RA;yx{v|VaOZCV15#JQ|Frjf(0h~F#pD4Fa_Lj2CM05BN>lksx*(_rHT-jm&_S~&84&%LSI z{*SpYHi_>HhzOR=*WC^~!)MA&?>|tz#?n2zufGr7ND(APne8M43S0RUc?;|Oh!`ib zk4}$)EoIyclXb)W7%e?LS$dL@kiS|(#{O5DIG4@w(iHCN+Vd%H-B&8EFw|WWt%DvY zY2AC915PHpY)y|BJ`{^{GWBa>B`tN{SkAW92l@->TrsTzeo(h~xNW^jm5Fy}&s(`+ zm!Wvzt_lrDH-V;QK)sPyLl<5www}UeoX1uL78AAQ8!&)7%X$YJUBv7;0BQRP#j)PL9**H zJ;aJ8fQ?nVXvwE|FL> z|D_WH4~CqIoN9S640)u&;8wS(x(rhbjFvZ47n}4^iksH7~0s zG?0u7e(%_MwE8a?t6&;jx5xI4(eF9aRVLKqHE1pR zl#wL&5mBYyl^f1HRmSOCWFd>{gDD6Fg@QmbXX8VBid<1;)TTUt=ez>^#~x&$!e<`5 zOvq}^JxFaNaKahE|G;CUTZqeJil(|toIE35ury?p21b)y*ovK=<{>P2JXLXRYqRFD zQoEIDzCvMS7}<)La?>M^SiZ*UL!ZDEta|0)3`>jb-f}(YzPvtQc!lgSv@zH7omWx( zaCwUF`6N3B%Ok~>RJU_;@iGn}{&A0sufdj%yoDVP&d_A3(buoEtNzR+;p6IGNN`@M zWQ7J=-Qb~xxla?Y`iyccvT!VHQp$i}WZWh?>^wSja=|IYstHh=7QmZnkuwnQ@^X^6 zSbt=%i|?w94vFYgKREUG80+Zy$2J6mq6rr+-oh0D7p=FotoFUt>+RZ$7aLYNpZ2D2 zFW73z9L>ljHDbe40v+!0)4zO@-|{Tq$W`;@A((ZmN@PEry#4Du6-`j{` zpE1oHj_VO++h#Qzhmc#}7@2>~=It<|<}@?dFaJz7M$_J1+|*R$RbfQl$8zH-X`o5_ zSi!4ucg5KtjGi_2YU$GCW_QIs1INDB%6u&qpdo8lg_b)RXeB`(d=>phR#v+rdA*5CnGprCAle$}vg@w9a_#=O`<^eul2~N_QvvojcyPMi+ z!*7Va#$lWY3H{(`W40aoz{r2C1G1Ay(`8ar`Vbz_d29zw1)OYL=&N+WP=(RA?-=Lv{<9vw_-8#FHh@Dt@0M2Y%W%cf4Qa01 z1rzfH-Ga2X+Ho}3dpBNQ&=qN`wcgiVYiqUGy{8qx)G4?{qO&?*t3*Xx|f9V`ABmOT|Q^SY!%NWEbO-Hc13YC-?vxe22lc1jS(N_!S;FOJ)E^HBg|K-x=JXXI0nB`69M{he zn%&5~S(Wwa-S*T*K`>Fi5pT$)oSvSvPO_o1Y2^H}1MFkm$Kqs&6WgQgnno-mdF@eZ z!@rDJkZ4HkI+zRp47fRnAg5=M%kkxw~>?=H;S*#gb;zI6S%I3%WM!1JV5Hq^dc~jP&poleZ2#h%s zVvKARDvxQhTP>-f|CVrp=s{3h-aI}XlfUZ7Utp+R6b1s{G{3Ou_I;`$3dVdbl(cAZ3jOblmY2wPio&#B|n!#0RsvB1&|h+C~}17CgA;7RmWf*Z^6 zNv$Z?;Iv?TE$Sx8v?sq}i<%xw-wzufx#_EROd*+J;)BT(z&m>J9&weSM2T^K5`eoK@o4XN_>3MJGCro-#D z*Ju82jjff)6-8g$*TW)%nIP!%1TD1_v(^z8-?Ivp>lj8ON#{2nysN=)f&bEr9zp8< zqY|qxFn10R4IiBs5j~b*Ikfp{qYhanAfTuN3W51B{Wk$lx7xZhP?9$73lpHwh8GZ9urt>B`T_7~c z>ZS4suGfl@jL4Ef1u8>RQwbpoA5UixwlWzATUjtfo2akGY`99V4+Bycwu_vWmL@`m?kYg?w{P2_ zN{9jnQEPzsjQ=@i7~hu>0;Zs}Kyq-sx2Ut$fM62-jvvv(6oFUibz}+M1DY7%O-H?n5Mc+b*+&2#B4CzpcH2iVQwQ9{Zc40=L_v zS<$?!r=YAS5K~N>b4tN&6}RsliPV`J`_Yd~AH>iF`4u7Zo#U^1kHczY2We<&w^+x9SE_7F|O$aukMC#@#$u}3y5+|_}-kt!!;T?*_9$M;0783`;l zrTTK4f4Pdli>^C9>{9R;t6cA$)6Z#$^Yu|n=$_O`C#sf3d`dw*$?wxX(^u9pQym0D z_ec;3rVL0os#KNwqcAe3E5XXuhdQO(9n30=Q^aqq*RdfcvR@-CTFjIme*Kqf%FlUV zgxx$C#gn*y+~ftU7ohh+O}wd)N_1K>6Y+lT+HX`VzlC|EW_HoR>zNT8$>RC}Pia;Z z>sq+w?qy_SSmwShoQy%GSu06{OBPiHg_$rD5`(++MnHS@^rfXL)hv8RJL#A&ay&t6 zk0dL057-)r9~tNeFCGKnC)Pm0xt~T=XeL9;gF|KO01oY3b|O0gFT^heKS87+|DLjVaRcMhBiCe9o$YW9f~KBmu~J{hTMr&JMkrTO2liAgt%GgS}0zJIEI#vMwpdmUi<|(F!** zR2<%qjKUN$D1RGSfBUv0Y$2N1JIb%Z?q}zcr9N0`u|CQL1_0C?eDi8ZpB7!48r9&> z;G3W{I=`CFN=v)&vJmX-=u8xHokTE^P1r}XT4UF0BoCAxyM<;`^LQaKHawxcB_g<; zxi94IlVVugD)mUo!one@>#orsDc?vttcsQ0wlGf>zHY&9>gu|A6wtX$aWau+gYFK^ zWoGl_wU5;w5gb_HrQhxYdQMY+H>l>M8IRS!DbV?1VJVG&NbTOm6jG7tWJ4=kb3ZsJob?H zbn%piTVqV9+a?4bDCr{L`Uv;~m_|qjqe&hJ_8Vg!++kZnpOl7@24deh&DMB{MlStW zUXA18!d$0_nLOiOfVG7ZlZh1s@uO{lnZ%;69 zOoz|K!v$k6JFRnnHWe7;E?62K?q)&CaE1;)ct1&E@vUZywCAd5J!onFFC6Hb$IuH6 z3=6@Lk9KJ{2;DZfUFW-B`P-$@nTd!MCpwx=J7R1T^#@yxU@Ki+R4dOYKi5FGWR62* zaM@zN1YB85Erm}N5dvc8RFRvm78UW;8g7{<5~S3u1uYm*|TnMXob_P^Jye+>M3eXBPJ#0Gsa5u*y+agytKEYr|SX=YU%#A|6Y%i8B)^I_4A z9Rq9zEol?NS%*+B%$E0ao^SLHVR)my8Jvd{EzQeCGg4`BBxw(sPZ!Pg!!P)8Yx6+g zHCQ%qGc4Mt<_$AjS@2#45WG5V3{~Ofttnnl8*&N6Gje1TXSTIQH-BdIS4Q#;KNNu< zrRM!>jCqNL3G^1WLT8t8UL%3I7xUW`4aLgqHOCYm0sULtfR2rep>O&l`!EolIq=yA ztFayw@_xGhW?^wvml+B%d{mlN=gH~orb}aTw5`$p8P)nsd;*_j6%>#~Qg5Kp+`-bQ z@4p+NV>_yAXm}ekKYHO6!jd?vqJn4^L^CCfOG}%1l~ensiN^1{{af zT7P2^F;aU=+rrGr3D)M7l#|k4kjxsR83U|(iZF{s3ks77Ik1Zj!kVV&v3r3nb&}j@ zdoLo_nDnazBPfp9Fb=D_HAd$<+o?h_vnS^49RbV3#irb;{W~q>c(G2N(SxrRw2VPD z+|(%KaqM&*%klO+>zve--V9YW4r_0RDMQ3aCDxx%Oyw%zKXHvG+pdoE!zTZ0Wq}d8 zX{y+HnYWqs%@ymBbrE8g<86xAB+h?Px2l3EYmAuC%V&Yqvbf5{Tz4+aRa@aOl*1UF zE2s#>X1(KTEov$XN7%nRmmp#_%aq*@R zwX(DkGldu>KcoU(*NS*$q@@8<#7T>cxV=V~!fMqHQ0579%$ll#g2;ru* zJ=Tx+>RgV#b1qsBsc@sPpPjX%Re!exHj7gm(Dy0_Y$Ix9S|h1ug^{iloQ8c5rED(X zG*^Hto4PebB2#cIivrs|h&SBJ7lK)h%-(Zv+0unMbRbA<*x#nZI&=Ijo5LHOMtgkn z?U~PllR=ljUib_;o}h1m9IBlO{&zvN98lum>RYZM-uUq);x(DP2~ZiHvZC{iK0XfD z*sg8V;WZlHm>BzXZxp1^CHFSRY^mq;h~lW}1;RP&Zi$v0FZJy%SK(hhCD{~~W35@C zwrqppAnN~9d*m5moH&&eHjRKu8=n;lgZm}tJ%Ovt$Yg221J3{z_TWuhaM!AX=v?Ii zvOP*moDr9YPjEaYB=2ATpt@m@8wIF{&V_4$-=6$1!x@xRt06w0PE{c#MI&Ps1!y<& zFD2n}-7EzttO&`eavL2^(W_VbXfMk=skE28@(OO3X)TH#j#~gMqi<THj6ly=sV3GBs*l%cHnpOKr=BTdP^GnYf7BarDmIeO^<3h;c z(TJoGXzRRsL7JObR$to*-i$YWJq`3Pc23AvAuvMd-uit15R^(x0-_%N)+Ub%YGWsY zAe9L7f`quHq-&#kWpnekNNak!#^Uw?SYfBYJ{3fln!`LI9Jv9Ap+F&AOt^SKHPTd( zjJaf$bqLX_t6rDvBV(JA99aHj$}=gPQ=GmOpe8AaipF0NZ?;r$F@%6-G~e@MgkHzc z2CIqo;uL?9WLE48-LZ9aQi_<%k`T%i2Ra+lq8Ghu5bPC^X-7L z&fXpwM$qb|^Y1?X8RBl&M#~rpnA!g$I5`Me3r{5ii!GXa83X#brKtkD*C1&puX6ej zJ=Whw{RuC`j|dWZ5diw_NuMMcjM14!pW;RVyh&Z>>otC=Uz(Yj{g<4DP#e1}Bu5G1 zXI1$l0suJ+!u)9oJHaj~Ee$%fhd{Ujd)EKnZ(W!1AYzBAj?J2j=3IG3e;z#VXR^>W z5Cm@9zg)9#EI0dYCPIOjnPo92*#G!7=Reb^0^Yw8LI2vqT%Z1gp#Jt}e*XS{lbQST zHT*LF*jInv>;V5^>HPC4VbuTbk_e$W|9^ICklYCPeRLlM1l#mesrD4`kN)Uyn+XR6 zqAds@gmfYVZxpXTmh!V#h9hODb|C{XNd{7A5d5b-e)R@KyXF)CaX54G^42T?I}AUw zqlSTE*^puMM-uRlHD(E{q(_`!hllmKpn|ldFI(rj6-E#Y1jhI06TjYMpp1RRTBzCm z`%-0phRbi?;>H#t3%RWKKe%&#Ns~sOf)+hyQ0F=WxF!0Sw#Q9M^9%Xl^4o7GDnc4P z9&uH1o*m5b6Z&qgLIaK(QY5DA79HJQLb+!Es(++!c?}UiUG1Msvi#1v{5>drTX7+= zFJDc)g*V{`fMAcKp+(C}i3bR1m2Y^f7W*{h_z@Mw^)jPQUkuMY=DfSc46>L!1S!;3 zCR-Z8!4Hc@YwxpMP2WMvng2x7el`3_!cV>W(f!mMLmalRz1oTZeP$nV6mqMBQq^h$ z4x9+4Opn4orzq~&#+G}gfA!VhseFJu5?fr>6J8aG`5P=bv|2T%{W)q0q?YL2^%A)G9tiF(@`>iYrd)F?P zJ9dJWqGV0LH6>{BP^Bwc-^A$EEgrWURqj|?lT%hAlZO=s@ss7L0$t-YiN8M(VR_aU|fG>|L$NS zAlNkHUy+hglb_4SFdVNW{bWZwC~U)EGiI++Ld=J7z9t@8oL9n9%J-u5sL-LFFiR>k@4KqjU&yNrASGCk$6b$0_WsYFChnw=(IjG6KJMMDRf{&O_ z*Xt&dCnc7M>Rd&6caFFcL;W#L+RuH$ODrxIe9w66@~9gF0_Ol)?;6*3ZwIE9rBdwa)-5lp)~pfW!IYvvV-{ z;RiVr#}KWD2Pf~j*D^HMWoxGyry#9XA>B!TJ-U&H9TG=p^}zP^d_~nQgH33`^%?o% z91fj-@s3~M{~Dx_+A@0rx)r4LP3Tk704{p)&?-zxEAVYSl#d?4c4vv8@ECF{IJ}Th zM%$4j<#@R=#3w8bN#gW{P`LObeccnM0Wt*7Mg2j9SJ26ht>>sHi>Y-0Fxp*~@j$YL zie#x+kRL?CxFq5>(4_|(el~wTK%01P)?#HwSr9+e3G8}!5Es<9OBL)7GLt{Qcw4u1 z+|srAx?Y8%fG~MP-s6EZq9mo)!SMB2`ZQWh%elOUAYWKacLKu(ub|f8Uf*ktIR_?_ zYwquNN3t3$RB&wEerHd)MF3Nbh_3cO-1dKO_B*eCa&R{@N__;m*O(r3Bn2wAT41^m zEQO|UBv?ESX^6!d0HHn%hk-swb)68(EN>6flJZu>X#;}85@M|Ha3%z~pko~H1gsI7 z0ojKzx~>#-cI?^xFH)nUm1Rh+BNf=Cs;qbk)GO(o@8i3eJ1 zgPi&N*znjWa6)2a#_v%LmE5sM%uA?Pe?(qy$mbbbzRic}r()$ic>#AlgN~ml?p2z! zDGy1^dZC9Ba3n)zZ&@!m{@L|@K9TS8FLmYphuDag*aeFp6x!4~#d7)B3d|Fn25w#%cLhx#6TZy9uy~$t?!M50g!5!iXUkwSY$V}Q^K}( zbjvn+?G%!568AiCCHc(d;Y)K^4Al6HugS%)L9Cd@xU)mm4LAPrRbr|^vb&bd)p?VwG$!6@d$UnO)UtHmG_13jI^zP$WQGkEAlc84G83K5NW<|IM(1N1 zD1@mD!1v>@6A&9v1}&_Idng?3C^%Z?1f56a`UJK9oFpSFjV%#2Z=s{pYsLg!+BdLs zCn^wAlB(^nV*?tD%i>HCBgE6_2J?b85&SvY2&8h-taUax5plMDyx6WVB1HUftyOua z!1inJ{QU&Nr9X>;0H`l0KP@_aqC7mN--vU#!j$?e4?18VuedQpCDmchcW-}oX;~_@ zmC&%AS*%WNuCmr&+xyyAezKyVAzdLfVwId6y5rL2yOF%LV#Ogq)}4zsx8Cs-vVg`O ztrN_-eNTg-p0k}fHtad23JypBuR~X>&sno~(jIc`&n=}3eR$?0>{$j=KX2z3@lE<2 z%4B+UVYOHexx%Ww5iW<=i2rGC;nJU(KmZ96sy>^!{#M0vHyslb@M2W>t@XnN9fLvF zeu!gk+tA4*=jWD7PULcC|30lyI)#pl-CVwt#rJ$2aT-cG<)@dTMYHPirZ6P!Szjk% z?gUQhh+DSN_rzcTb>?_>yU=wy5^t~-n78n$nC_wE`7Ztna;n_f<{1u7z^+)}2(?hY zT4!G;Mf7lIy5gwpm7Y9#h+*pZ{v(l(J;2lE1^EO{#m-oO)fetCVP#&6){D38Yv79! zsozHP@43tIYwq?-8p~lbwj%mea=*CMu-YG*e0*?P*6=_FF_Ls`+-gx7^AE34KZU$D z5|wGbU@aL5Jhg7?lSFaw43!~(RWeWw|a&9M6A=Xw1Wp)ch4nD@ib^$rHMF^UmTK8ZRUroTT;4`(a_A zP15Fex(E-c|LR{B)dfB|HFb{|nr|!LLJfBeB*6fY(faA1ura7s`1bcpIlqV0O+pmy zAw`$lpIDcNTZTAb`e79#cR_zex8q|!?MJA7a))#9Fe3b~1E=F9Id%woNEd|Cpo1C7 z+2>~gn`NFzl}|0nGag)s0jtL_#o3}zp>p%^zN~vmOS#~`&NfW{bpq8u%*f(`#fG0K z)A>G0tK1}K?+#b|5A*xCY_R(OdJNaCpT5T{LsprP7IK5rOZ0z~1b$hQ0tr7i?tj1j z$JqV!koyv#z(PO|6Z@BY?~AC#n$)>|I{M2p7ob3U6&W4mU~XvL3{) zJ9(35q@((eOvJBQYzda8z_{T2|KFYY?Qj355C1RanYYge?S^H!*T9wQ<7e7!?#IRD z<7F3z9^kyWA-bklv>|!`i0PxSr()|93D5q}9ZjT0 zrjJ2EL^ad)NQm4UEv&|A@i-Ih@)4r$FtW zk>?BhPg47zWB5M@z`tMrQ!e&XW=cR8Slv5#K@rl0{{bmJC9kLg!XM`@)}7GC?9x$4O4&Ai@u7LIY#*-7yiLN}&yNp;-)!gJe9 zm9!VdhTkZ=s6QGW9T9!T_=0r3;iRqtG;j@_i+js*Of6*7u5b}^fs7LAfbJ_^T+>!f z5Hp#T2pb}j7j(2Nhaj2IiXdbcS0CqDGi^}xt#e}*036_fRhz*5S@e_E)a5vq=kQ4S zG65CL=5)?Er3AO{tZsQ@8Q0~qnEriqaYbJFBW#F&0Rc{dUgPe*Ii(jVU&&q^8RoTE zrdgz2F&;e#E|7*{gjL{6p35W88GHb$vZ4D0BO);U@_ky}6GpAMtU{qUJMN;ASIIJC z@;E$FhwfnMu=VyGJI8Xw8`ZD*rL`edkf3H+19RVvch`vmD*TM_jd0+4;MClQJnY~b zA*A^CCAlH#!v=f{f1yuJIbrW}H>iKa70@`qNlN}PKD{6ZX){p(WEQ*nd!HdFi)(JK zEs4so`e6nr6$BHLLP6Y^V~{Psg+^_*dQ8((!~ ze@+807JO8~#H*kUfD>*N&NYuZ#^37Gg+%F8`PA&Ts7tN4!h6_y5GdZ#Ew%uJ&1~5A z)y8psNL^n@XD0yIwxSM*a#qjY7%w*3#&=Eg+g{hKo`9unX1Fi!aChq;UC#@M^fXce z))LD@@)M7_Wn3KS;9&dHP? zpqvAEtW3@|h`EP^Q^_*y9Z3t7X=g}h>eE5uS7R~qq2O{^-QE#&BUvy6-P28e4pm$o5|BGRzM}wIby?xXOiqFCqK?Rs&jMy+kszS!t%OWg@ z)p}f1<-7sGCnleC7N}5ObkM-+u2b@BEUb9}E((;~vTzSX-VvTB)W7t<=Ia|R!ToNz zy+)8)7qj+5YeD`k_%GN!N`Kc7jkl2JSnJ`0$5cQhW5(8G4SGFQaBG(}yGjx52u}d{ zdXc44#UJ8)+Kt4T9bK%iI(X-Z-P9v2P1UJUJFPnBC+)&B03#-bYBzUd$P^v;acUxD zF&$u8p=uc-I3}a@gdZaD(7?1;DM~y+K`)G}@qwN1<6}s7>(M@|sH)~E^?!I9C%G{{ zcRx52sQ8i|x)WV!DSr#D)3f|bwC0NiR^LnfDd*7b&dG;J+%m!lrSUG(hiKL&b;6gE zg&f{Aut?zvoi=7j7QyR*YPlk~-M9no($0W*@FOl}x$`w&zB8<&wKOK@F*VhR$>biC z)u01i7E2vVxhyNf0i5fZjN$90?6`+O4pfhei9>-!(|y zC$Ik7jjfq|XqGZ!py<}7=2L_Zi0HE3!R6^yzPdPhM<^bHL~4Xa zEMKq)n|`8o-3`kWm1p^Tpkrpc4xYAXdQ2a9*LBy2%p`t6EPqvAxn#P~ zm~cs?O6Wt{)P-xh&~?tY98Eai{p4rl?!1qX-GyvBlD-*0jRQDaSjoxY3Tr+k#2I3i z7hD=nB5CS1tq7}XmFYSHOKYvjp6gJ7&f4sM{{S>bZ4A& zcwfWpcBXHflCk+iec?NOxRuey3x{~)SrK13FECN#9RFhX;!ODvA z#o0!s{6s<}x;t%G(|aZ0*}T)RMuYUrlwQ={OAFoSBU#(6dL{$2E6yIRI;mU?v`8J$ zkp&1@&JSFct#kWphGI}7cF32-+J@jkBL-nyWp{|xA8BlE96T&c*Rjn$#F(b z)=)Jm$0lW}o5XUrFb!R1%QJ&^)g4snmA`l4)e!RaKV=%crK?oSZ_X$#LqMRobbjY|l(*TT+$?7My6_+8A!=%Q4+{0!1ox13>{- z6G_##fZwfru+qDni>Fy<8Mi>kJ3H-r;>g-f1)k&6CcqiMHSK})#3GHh(;003@E2eK zzAjmY@lvM4K2-Q29;sT|+Cu@`qH57vVq)8HIdBH*s&r%w?ZcP!)eRHFM7ukutAZup z7#wb=Cv20bmq3XMyGXYH`-zluGyy>&B0*o<%&^pi&g#06Y>%&WUGqd)&TWgGq9yVr zYwG!q)h+!k)cz_hj=>=g34shgT0eyI5_Crh-!wa?&^hGM#4%{zCk>^$`N%Zer&fEF zo?k8v@vSPGLjoErI_DEwKmgl_Nomut(o0iJjCqCw9$VkAW+y_Jc8Ku`X4CDhK^fgm z+ZylPr?;(|m>-ifG@UjIF)X4z9keo zX68j9CMOy?n;%H1bE?g#c|ypS*X+b!94KF=TKpO|_G255dp}yBA)zJ~V!@_Y7yKX; z6zFRNIQiG2PIN$j+S=AdIpP;FnbtX*=VsKX>%2-!>s(g`xami=ECtMsrW@F;!5a4b zaC}(X>w1}m8~|vb6FS{8(&YAtZvhV-@K>a1?^02Vn=>!3$d>dNZcM&>s3|6#n+U>* zt9Lj@z!}lZJ1l6f%J~8V2G!X*)Szj#4(ut0nG0wTcKK$4xlu`y)JG76ELJe98&0yJ`& zOGm7CGxw$2wy11&K)hy52OAW$rIWjSJ=OV~Z((_NK+9UZWNKE4?kmsqJ+9vUiapz% z*9C>-8|4SXyXn`A>D;0pduIBqGPH7b4mL+)n(Lkjm4{QeC%aHuob}KmJ7ePyzMr(;XycMGS?PkA5U5RE(ou)D;zzs3$PKJLKEueW?cNDzbDnOhEmRG?7s z;vQM7b%w<>sqN$QZjnqKAzxF~?v;Cd`xCUPLp(lo%fB}j%< zTl<;qqF(hZ38s@p(jifuiwS%$z;aDFesO_ZR`LTg1pkr}bwTpO*SD*x4?2sSH3y6D zN)6G)CsLbOWba1>{GBmz=XZuRwTnLkja^gy3PG0LS!!yz2#(wlYsGkb^Y~;#`A*W%#xeYjxnvn1Z~ve* z{J`Oxm{ZF|ELmBnCaI)aCAyfbOTHAlA`jl*A;buziPw*|-;}*qml7@3cN5 zNU%9PVsl{|ejgP=04Jyg8IO_H|EUEyb-`3(^Ep^)NiQ05BU0J?pxMOJH*QRa8xVgK zb}(BN9*X=D6}#J`e|C}8$4pftq_uO{zT9wyd8E}f?uYK-0m;!uni>}|*aSpeuK}Bq zn2O2bW*Dl-Pr4>RCwje#Mlfv`^PM7bc#ggyWsZGPzCQ@wpu5gRH!QB2gDNjx!rlS+ zfI(+Rn$!^zyhxCyZ=%lSeX=`c{4sR66NzZC!_tGd#w%W?^Wea$WaW#V2Z}^ZrS7%? z#07pwceMt?wAj*Tsvr4;l@S~#x2Ed_8P5peeJx68O zB6Yq>(k7FZU(GG@BTaP5@M9{FP`mUQTiv~P>(Zd*kF&Y9(Tn08pH+{u5`e^d&!4tMcZ1t(TmQ?iy5Lb zl>f!vdxtgIY>%QU2q>W=MXGd>9(t7`h=3>{y;td-&=Ki|A|NGH6#)?lz4sP62uSZ; zr1t=UoEN{}-urW(d+%TOK4+hEe|eI?lXuFR)n;bRTC?6dWd;GRw^m)+_!~L2m2|&M zEzaCRVI^@Jt+Ea#d7e+QZY-u#i;tW7JHOqha_#HqE-vO7bm_=q{Yv7T$vcr4 z)nRmVnt65aJ+Ep2q!3CDm{{m|zMzEKK}-q80k#ypi#{i%`Z&3DO@Ys3bQ zT5lbWlyPttz2GXBp+1!U{8etYC$87L7(_66#xWV(fAs6_mmAX$fOU_#i4NA=_c)75 z8LHW9%YyFjE4=hd3-9EMD$N!JTEzCObx(+|Fvsz)+c}VL$`KjwOM+A+>O_jIGnTCe zhhPD5kY`N#n&f0P?z^J6xUR~BoW*KK8|)$NM8-Ny)KtBDEM~HtsGd`>=TuC7jFXtcF=7HKKu_eiN$uh*79m4H3xMtQur0M|Nd~Y%VB7JkM zTc05HC%eTjQ%^>ktfkr?7 z`Lohoxe(P>i{*@l$OgXFN%E#nTLHEsd*kuw?Ue(ZR=?}S+?lb$(aAOLFqhiY#vM&! zVCPkKOtr41VfL3> zLoX`|`mhM0! z>EeU!rm*BH%mNO$i5>Z%(jM2N_?e7{!)4nKGlYGUgRoEdK7{FZnYCwUF~{T$f<+CN zIBnI*Jy(%9Wsd}ah4TBoF8+I8m7~hk2t4VvVWy@vtyT|br=!&kEj=uwMG52z`es%t zl2rK&zcJv@vE=Yc^>oyMne@Vb4hGrl5hk&5?dP0RFCPj>b`AgJeAdn7LF4Ji0-M`(BH+%ASVvoD-)HGqR=pm$t&*g;0mX|Q;VFHGo(376t z?{TUmTytineS=T>=3%You1xBR;TZ3FT{%WQziOnJnvQ-deBWA$Bn&nG1l`sfZ<}F8 zXd-Kv9p(&~=9_}DRH+xg2L=`F_7a{EWoiuf4#nY7&=@d*%xJ$UgI!P-1M_;o@>{o| zwmkvNI0G|yz{K*EJn#0MQfusS>s;d^zKUWQ&ZWlWgBb=TTgiSJdvT2nE4S)skdMEmjGWpbnS z_G2$K`^||GD=F+Myx6jL90dsR4>q5xveOs0sc?YRnY~Lw&lFQ`Or~)G#R#}O!4Lbi zq@;*4!1`t;#p0oQ+R_Y+M88^6mUC&{)Mk}0!RE1O`kKL8c4++R*VQl)r#+l7d|Kmi zW2(^PQ>(>^&|km!Bpr2W(sSDVhVG85(pjyHv2#kuZcVtpxvU#EVYle9eT8}Rw^@S` zQTlV%D>-7>(Fvsx(DM`y(;t4J^GNL*rklMa!Z;PLg6X-&>Q2W9zy0$##ad-U?+}t~ zuC!qeGcYa^!Sc?`bPiCY;-6RR^sCd%HwS3Sy-^9hAP1A>ZACaOn&?s1<;wOe#7=)a z&CClBWwp%?)~qu!k3H$X0n5~tD|)48`htNrCPozE*)Hzs@(oOy_`lT(rwpbY2uqUnZWmjk%2?MlTo*^ zEu>d5)cbBdGtAcFa7^;Kp>G0tBGzlyN6OJEFCW^OoQlR&h~YGC@f{s`I5}_KmqN}_ z>*V(gyTITEZRPjLD;=_|mV}EW(9{hL_HxkpNUmwSiH5++w0;BAcV$&xm)XPyx1mEm zW=yqJ;2)#!`W1M%`enr?g~WQH^+ikk*}S*W@d>p zj2c@pJ{ehBcttgDE<{*W#R=jVDM!b%X0kbN59_>L6|~7$S03b0i=Y=I17bkM4(}e0 z_#Lfqtg$FOWW}c3Db?0nRY82Na7>PhSC32+m)#xrkloP+19&D%1eU1)}P@+7j0kN=^mDXdOJ;ytoY zb!&>`)ml!j4x6TPeK^HlJ?799CL^$iGxMXnY$()k;%|i(alAB6h^P^WgJZL(AU=q= zpWi0S#kN*bP>z~s9u<-E@u8KzAr{|_k_1jJO8nNv=08B~Uitw6{Vvu#)frF+AoIUk~qHoS{P^xEZ4HFEPt+CDDv(rL z4VE-gScctJ62V7d#)AMBj1)nYFhm2~Fv}@C zqs6TrspbZO#2?Z}GukAu*PL+ax=>DeW4-p+EVwuR6*F`!uyAX>CN}3w6g^5`BTE}! zk%>Bzm%_}6m~skR&=vWeOB8&EmM7wt#J2S}+b7RP-`kyPp5R&H#P_>ZTI7-Y@!8M1 z3FLY;+y2U#JbYM~@Ku9R7pt<=qBzyuZs^Cn%7fU}Xm(O1i#Qd(F9o6-C$zP9by9Ri zU-ya@zzbfSaHfpgFYg6%k`j}dG{{Rl|2{hXWU3^PqDjA{2f+1YaL^vw2dk(i{^Y|a z-9ui7MEK$?_0$m5wz=Bt{9@{pnt8T1gVTZCB;><(5N9>ZXWbL)o!Q4$pZjEna%&*C zJ`l$&0gHXZPewWNi90(U!*`(Akprv@~_8fJIfVnb_}==O{cW*5uV@s#iF7bAXI2?P);7Fq4em(Hz4(g z;R&_m(|~~|gpw(%p8Qo6i%$el-x)?KqYdgH-$&vQeE1<_&!=4RI}+|a6L|e=zULE$ zp7rx7;7H(DPD~8Dl!LR7kmau3jB7B>tWM;j1cS}$U6v7fRV|X|8o3(yPWn`4AOuC4 z+`Gyv6>8z))VUl|tVLXgXjyjVOc^Vo!ZZE|xzNN!YLkj~-I}-Yo0gr?Q7huI-*}os zukcgULe-oTDr{(qyxw4hK}KRL^DYE##b-C=;*Be*cQM^TS-d?W)C==@ih1u{2XRGL zw(GJ3zQe`?6RP0BnAPE_s*LFJ2k*>ii)Tr6g|~%>an>-sZ0OO=eqQ!{@*! zQ8?Nw_xiJ*;S((zhQH573VrgK?ehz^zxfmatEreK-9q^i?JpG-_sl>sZ~u9_(LbFV z@c#5-xUS8w{oA48?+tx72D~2r{sVCL)o)LeKY#mQ(bxh2!XD0T={J9X;_DKr3s5GE9LoMD=%HHIURf4T`_m001j8nA9bR}{(Et0PgJnU(+XYY zglCIhvOZ`WabgI}fSqn^xA{|+_=(E?td(m~ zqph@$f0n%`9wQw6rw>(U3@_Go`TegVfUMO6b^kpMqRIC^*Z<}}7qV*|G^arNMDC>= zb?2X2Y`K#n$U;Xuu_J5kH)*5^UvMSmjoM)Tn>G~px0_~(efs`+t|Ak$__J%<%PwK) zgH;!paCEmh-@i5b2qX)f1OoRX2;;}R17?g?F=V4EMQAnrPc0}uXEzPRL-efj#l-zC zg(>}*BJ|I;$YrTJfy&gK;_Uz1%`@UQ&T{VVp}5>vwpaU7s0$Q!(?uO>7QJ_hyhNX& z7g5N&{_X1peYohYQzUi(zf3|*ZlhpFzLZx&w`qu~V^lnP9No2xB9NkwG?k2wMPKY5 z&|Q+F!*OwaIiN{b9|HF1&{b(VY=2CxzBbp?iP2E@I$@u`l)8E|z7|h?_G^AY3q2sJ z-Pks0=o^jRpfg;rLYFmlCRLR$-(@Vn(2hUfe+fGR;)()QDeK8tm0*P zw|kxD2~&j}+s}8eY#T2c)+c`)R#9?XdrST6-MZX)eKbk*?wb%2x;vkR7D7v6hw2rB z9Lq`@d^b{!ERW9kI@F@77YN6dA~P-@&R~GR!TB+Z+UTMwP; z18Vez)FoF{qVLsyfI!P)0J>>qmu8#kNXv6u6@43hf_z1m#UU1ha=$v6_YFPy5Os6O zJ7U_(zZHT;G0Q)C*my+M^hBNF5E{P8yc3D~uq$KOXJEbkjK}zTFnR3FRDV z8NYL=6f%g;^?Z)qJ~d-u7d7Bm`l-9+#?=(a5w)j^T+G+f`8r`Ij3k{uUo*1OgOHj1 zoi>Ec1Lk~9G4wb_cJpTgke}Pzt1tKbGI>DG21%+}9>-o+-GeM%YvWgB*gnJcP~LLT z)?%;GoPrgXv_7}X(mV=$l@#C=eZ_i()pQos#$@??FDw8_y|{3*^4MIvVV~p*d6rb; zUe}eA8xU2r)qOA$a2#-8B_%ef%-KKWh`LLa=7Tz;YkJC4oWAYnrMZ*77&eb4G4%FJ zQ8BuNoQ;)8vdkgDS7$LSlOAq#G~Ms@Is>};e@Hi$R*_c;b~d3{(INqE4_ij&uk?I3 z_vnR;P$d4IF42MZdK=b`EX$-dB_9^6{7)gsgEh&x3O&@2)VU+Ftd26{@_2r2(`TyY z@*wkIc3BCuJNZnnXKCsX{Rn-DKEd)cI!{H@`Rv^^KQ)fS%q(F@h9oOtyJ?wih3R2R>dRKRcPclSo{5R-+Kc zgBK(OhOBXmH2!D}qxFldd>Z6xgd#_^WTt0t|I9^BveF`&Jl_VO$>%?&QJI?Sz3W-B z$nd(djy8g2Z#NDesK*efRU=6QPC}q=G$JxqUwVYlqv)>8xu<)OtDSXE;7%cy%nR2> zZ_|iIKkP2~KPD>4*ZSif?gl{KwNMi$?g>C5XJl6^EP(4ZU{1+G>bkW(>Z(Q-<5p;F z|HTMDIlCYUvD!O{{;5*vRv}Nf`N2bCvx=@OqYDMp(Uq{%o6M%8SN^*)!3qH=SL*Gk z?^}2HM9v3c$p+aQ?yd?C`x=3?NB$clZOP3~9YZ_j&t)6;XILqa8dODzO-D~MYBLts zuaeNws`$)2dq1tAdkO)2x#E5&4?O&R!{Ko1Pp_jg8VyC&c+OX<&JcDfM(Ft~UV`$A zK%!9ZY2<9cd6DNi66461qovG9^!Yj~ny6_no*=H+M8W6n@Q+=mSjg+c`vGU;R+Go- zXp*KQ62^ABiP`z+qWue07wY zP#JS2`l!=u<-6fqvhpfsul*zTmvs@?6#vp>Z|TEDa(+ynO+<^`IA7-^DcEaWbPOT}7ekOE`2I_IY$)kv*?KJ>XyKs$MpYofOs zIo;(^mYouiOW|KjN@+reGGIwp9kw)!A!kbtGnj8?3~sR%0DQEB9wE){q&*7+!o0e6 zt)S98q#cLPG^l1w?x&Ztc-^C#jy9RBFR27^O4I4x!fd0Ba!If4&9~zVD3tMEWkO#O zAoln9lH?nJaPGy${reSs;1IlLLXLnUa-t*&O7ip+KEB~smoI$Ahnw?+pQ3f4jwAKs zcH7#qfbEm9@ker{Ix=MQoez_&by-^8#Bh>beT=fu$P@9^djEcY;a;dT(@Iutjr?rH z{zMTP{MqP26?I&ty5Ausf-vIeJdeKOXhPM*49DXe|83HGOD_n_VW;m$8#A$q(Sdo7 z%nIo27`KHcmiQoxy{LV(Pe=@^N6J`LSn;c%YKdA0V|a^_IFKYKG?t!pp$?=RRAuF# zP@x;-FEjHh`$h4c?6_b%LuQ%HH<@<(tjKLCb87xN~(tEyyZ00{GL&dlcS%$ z-RfdQ_0C^WG9-2@>F4&WtdeNN*C~b;4Ncm`aVI)O7`voyzJT24_woxF$~{Y;4|u0~ zxzn^(q8VSuVfQF{5X*@F-3I@|5BZPZ*HM34#ZnQH=^BK*dN?j&xF}KpCyNM~z9Xij zpO*abqSHTQ{-V2S5HZXd9dHaqbtE-5k3&EQ&=`V6p}&(J8EXYZc{mZkuJCESOod$M z&xi*N(>nJ(L-b(FdPb9kx&y9 zqh>!*{lga^>*xq#qDs5or=O7Y%If)n=WV{@ z!kvQy%PvV}zKP`U8n@jE=^5Q6+kgYdv;5_eA9zyTmD17vTV|kj>R-?Q2;*492g2)v zeY%S&{I?E$T@L0CGSR1Km@VBIrhRzs;v;ddu|ZXId`#SNi}>#Q3Pct9$cXUaCADnJ?To7ft?peUC{uJe zH0G`zXqYh|X>jL~^9qGV`Uz$rh1*J08c#COX?aIPR~MJlB_OCV+f%1Y;(4zt>XI=0 z;fp<1&!v^ySGxgMgUMI>sw113%c({v;?eDY~7` zQqc*H`i399lE*Eob{Eydbdgz77c|vg6t=&pI~^%}SLo(c$FPxDXEW=xYXs=!%!7cj zHDni@3@(LcZgiPjDr>sJaW+m!`XI6&BDFIA%0E2_#;pZY zk-n0155+#1j>lrxIP&!LKgtz3K|O?XYy1>JSzj$WqWth3peB^Y$^KK=r)5Q~J^i?Y zhkYBHp7vqL5F@k&EzMJt`!D9xOe`f@J}I0MuEGy`NNqJ)8GU@hwcqHOSv%9#*(b+l zfI4a(iA$v}X$2hLNsN)|cGVi+@ZPgJ>ufqQGY?`}DCGr93zWsk260YkhNmmgT4}jO zfw{$dyOqEzJ7|uk^HcxOPx?4K;W;i}E}5UxO0_(D{XTZ~GE)42Vmf)XV?AT_$MF+g z0>4lrbVa5Gy@`qO`=1jnEe$k%2BL^ER)vTt-w&!=8y(7Shd-|zl-m{r`UoYk>Aqx3 z_MYKtbxqIuo{alLGeu@XyOoqnFBz#RB6)olDF!oYm9w_}ujYzS3^BV79hhmud}Y1q zzB_cjajki|$N^aD7kuB)c~r&VcS^yQ;@9(+gg)_J0}XrPovH>MRfo?0$4OUIClCeR zKQzLEvjO?I`<<{hAkRUAv3Z#e0k9AC$yndF0X}?#hn@n!r-C)pHx9Vt-ie#^?rp%E zdC@7EActjywXu2E-{h^)S^vDTkuB-pEfI^374liqSwr;k-NqDHr*HoV`~J-lddk3@ zbd^{V#Nf4(P7{|H~Zp6Xl4{t)lq zH7b+;z6kjJe{skE918XO0)@YCecsaB{q6es|LWQV`X|uW&FR?z4;#bQy*x9*{$Tui zv{y!FgC$^k*^>Z_Fwx6@JjK}!fNv55X^4}4*H;jy1wsV>y+nm>l+35*hX45nhyOJr z{SPL>|7M_FGXNtC* z)y>v=m^FhWwsw2>?&fwC4s9xPCpmdZJeVrQvQ5AVYOCwQ1m?a%zRo-_$%j-fZZI(O zh%4+id+-51gr|mCJge82u?Qvd2h2W5xhItVN5eI{MOz)VGNp9tm2&ZPXu+OD(15XFcb4BUbQI{Uw^OW#W)1T&?x5VzfnzHT*Uf zfL$$HEihqH5#iWh)~NGYgZ<^V50ly%>tlU<_;2@i@{5H92HLmSv(+Nh^5lTyQlJ2P zLY|PpoMhN2Owoap%nNTY`(@D!_5`*BK^>iBQ#aGZe$&|Bg|o^9I}2e?0Q26S(5J?h z_6dp+=;nuK%+InJsPbKBs*wyvXasKGJ?fnXQKd4i8c)AK?~yip4eb<|sYzMr`U4zC zb>|H-GZL_!peG3)nPU{Vp*XNDHGj>DvcV;AIT!x&tDsj1cCU~lPeZ*lhP0=Y>o;ag zgdM!zT;sZ%((}T{QhkRgQn)uNd{U+2QSp!^&%B@1ww!A>#1V*eiEwZ zo_x}+nbQe{$w}JIh01gG+{S%=Auhulu?Fbc!QIqV_SjXgv89J`CUmRE_XHo|=@Wo@ z6*yv9$W8*aEWJ!fM%LT%UcJ6g+uOVOLFob68aZ#3J7Gml8%T~q93>nyGiJwgxfHNi zW6Di;{#{y^VP;KjpeVLTH`pnU?4UpkFYXF!Gvq$pnnW=tLPub1TrjVOF#O9?);8@o z>w;_rMQUG#Jwp}QQkY;kz`Ze!3N9BnGd#bBjW3Jf3676D1xcqQ5RcYEt+B&%)#NY) zxve#E3*e4w5WKg%bSg2s>rikpnhoQkrmzorw(jN6(Y_ob^_b20Fizeiq_R|zZ{kaN zqkIbN``l}NO4zywu(V63mH5zv}SD>$`$J=-~ zTT7VsU*ohDmR6QVNd3&!e5h{wR*IVO9_|Z6qbzL*&lV4mBlx5JEw1BX7TYt1Xj|j6 zwlBkX-U2I-8H~RGEs+_yEk^hN_Ku|7>q%BQ!nY+W;t5*@0}Za}9vKB$&6TV#>IFG> zK3rvg$#u{fHNjk(DX!cn=NWLf<6QV$j*i>;7u0vTs8P}F^uZ3@4g%Y=Uo-?o?3y47 z#}1{T&rl}9omvoh=;%=VvS7g*OOsxiJ%Rr9p{_%ajDLFnH$^4*?Myx9rjqA1W0)fh z6e{l{LrOrwr!V|cT*ucHk0iRx2#$#OZ&U#)l0@8RjkpSH>Zd(dS?SBqTSlfP51%U8 z5@)Fy`(-_C*m!HQfXf0P*QqC!qX@2^ht$^BMm+0yZRhYEWZBj3Ti|l z33_MSYo~d?IwO!K^8Itx{ZyP*``^OF&;~BH!)*h5g-EpxTwdHE#n4W`W62@U`@znk ztk0M*g|P`u6Lt?NC1QCks58$I_iJ+S65AtouYVskZul|wDalDqwIxnjy>x||`B7+$ zu^#rz)T*xoTR_-b8s6bXW^HCvH&oomK4>So(;EZu2MDy|Hi*3iee~-7sLHn%E?Un$ zoM|x2;rO8I&N=5VOPpy=^+StES7WJ5num6ni-xZ?5A-EYILS~%k)dH5BBamoH?A&_ z6S4j?OYA~i87r-Yw-&W=z!a?<8V?4hHIs$=g*xq{{ZS8k$YWf+CqILKeEeBW`c6Q% zl7P?_USe3F&bh){Cg$~XPtxTjbame~twvC#<8blxmqtGY#>K7~MC<;w@H_~5;ULC% z8SlG2g;{nc;RZKchQ;TL@404vNv$K}5+7tXd!sRgpZ%!I$>%+bII8JO^9-H(Zibo|0G)O44rJE!sI@Ve zi+=bexE*xi3Vq9oS8-TmKjNg0=q(|}8vz%q$0mjj1I?G@#_;8eS3hQKl zp^8d&=Ap zq!9m7lU3FA*RNZK^x-dyzJt~#fLx^K@g~8i(N@NkSQ*U{H7_I*YeY4n_RlHvo(2~t~%u_cWzb&tK z(^81e6X4qNEWr-kd0MNVv7H+gxn7GQuB4HKH_N2+)~xoLFSj-ZmlO!E*ZFe-EG3DO zwWgglPFvPFd?Hh#VjB*Q%#*zBh8RvX9=kIGEBaCP7tzI8g@{qd*^h}6-9xdoGKsyv zW^@dNA@QRb+e|JCqO`GgLSZL%tj7F0jb5P#g>2Xa+n<=*33AJC`bhP(Z9hnZL47Z` z@Ybq!m-oWFajKQXPgfKj29Ac-$V6PvSh^Rpwsyiz>t3zF(-#gpUdHXqmdmSPE*4H! zdxaLQ6&>7fB=w2e%IaUJ6g_)JMdxqK-Cf+$S^-Y0caF+toaOOJR}#wK^4?>U*dWZV z*`(I9TUZWT*JN+U7jF$LSFt`4HfrFLpWnk#IH_8;i#$ZR3c-vfpNqa&+&WVPQdf%1 zWOl5-f>GABk3zLJ`lxf50R;IIrY71E!wX(ceBHAYA8b2doxE7!xicM8;T%+QcE7=g z&io_iv$FE8Y;bN3Z%zB2tRD!U<7G;O>-;APxo&WqLDaKYs`Y^~TFRAA5`OJlEWOJ> zq~94o$@El&tDaXpVu6u*Tgi-{Oyn{>UO!dalzt>|kJ0gR?bX1%cFsjt7RVyVvw|Th z15HL=^hA-RGH88C*Tnmj)>N;@(teH4irc`lmbv4F&#m}flhB_tBABa=uE(vfbZJH9 zxcY4SVhl_Z__3pR7^!%=2Ip6P2(`zU@1luxjE{a*G0OHnvKP0A}xiG-Gb&hL#~ zbqjJbUcW}LyJe2P3a)QF_iOD_m`}vT$5Qd0L#qMjhODGZd$PBC0|MWe9n!=)b&F%L z@d@7V3FyG8`?~Y3C^j8f+w7z6mQydqexgn+8%Qu?6^1y!one#>_GIN4-oUIB>0i^O z=oZW=SRASh= z$bnTeUqAZ~Zh{J`r-Hg*P7r?84ccf4xp~yxk-!G(`_@@JzzqS+P`^{U98RJyF9ExW zYi^BXO;59*1judP+#NC4(&nRxeOLKV{+qH<-7`t?W(EAyWua6Kea}SPWuc9oo;W=E za3Bv_YaJWm=1L&lh45j#P_?LFf7D@_zJx(1PL(tA6ZT}{9iqGECCbppY%%*`4aXL$bCq9mSU3O08Rn%Q)>l{jRN z3b6_FwGH-c4cV)FX)2z#4LO}-CN=eskZ!jfzCy0_grs>+mhzzN+#6{)q-PYXi$G!^?MZsc8K{9v(!{!A9I;3CUJK*P3qRqbHqOyU zR9s^E3g&D82?`dox~L9`aL)1uRCr4!WE{DqF9PYzQl*|YhtbdcHYPdnfj9yp;h!#k z-WLtmD=IGK=|EQWQv2)(kD0|3M`?k-k|!po=$SA!wIC2I*)FlZ{h!C&oND@SwDBOR zfV>7+5K16vxi$-gGMS}nlugUF+VjV=+xmuNRu?C1pFUono3-i@ZBBGB5*+V&r^yiU z@FS|>wz1pKqCuW7$YmuG9{7gUi{yl z+{K5$Zi@QUM%)*CySJ7m6U%NqDfxPD4TKSNy}%!xe5g5gP_S}Hdf@dgy9s0QL&jxfqZPv$c?sU zIuH?0b(__+TpL;MC`(2s)!l3?ib;viLm)n7GH!KrW(un+hkSBk!q^o&h_H+>{q(GI zs@?S_1|TZNP!kvhRuyG(&P^=g{#&D>p2zGRpL=`xroF>ieTCl;NPFgr7d;;V0d_jx z>Rucy93;p61Y!s>sGTFe4ivY~lcD#~5aV+_W4{{g+os8dFAkluVZ4ifY)!|jvGK8q ziJ33j9BN;^G#=eNuM3;hYDD~4q0A81YRP<^of{DDSEpNJ<{A<;29kGfeSY} zMYi~1J2}p*9PAezn$CXCE{kbNucNesGSugSMidyv2V?c6JXN0ALY1Qp7JE9(JQsx( z>b&r(3{9;4qem+Q62}idC_TA*cB`@SuD~2V)V_^ztH97?G&^pag(k;~Z%RQZP< z#V^M!q3Mto(ImsPh08XCmEOb^Q=0-^_N)xy9BycYwcd|>e0V2BL>Q0!ZN2R+e>xAg zmW8f1kfk!_;;5GomWFfeJW5)A#Z5#yjvaiGuIC3OVoqd4T5!Gz~3NBz+}Gdikw2A638U zHgG76wz}Zqq!N$G8EL+0_i@rKZ_EHCjoY?>N12#e8EE27p1Ir#dsMzxdOsdmjBmv~ zaSR|xhfz5RY|2)7-3>LlZNd-=85bsw)m2kA3W^YhxTUfXU9iOG@i(D`aRjQJ#RX(Ls*pccw zKLboNlJXF5z`I`Mn`=1Z!7xs&l3I3w#S~Tb_0g%Q_2aAvKQ7R zH`*BDHjGufj9s@W$;jS)r&+w`ke;1N?zPy;M-z1EX7AWSzL(tOlqfQ%o%M3v!Z7<; z`tIE-*PpUF@2K(%9PCPY1A=g^28hGOd!ok(TVuqpZytt3EP8o{cmWgs;L42XacI!8 zF#mmvL`%(PIwop-r=+k=!F@rpWSvJNf}Y{CVptoyv(I}t)!aN~v8ued;G0YP4#n1k z*(E9Y%8^CO{Hgq@`1vHA$ewW{V~2)!^!4r;)9udG)SWH=^_2qeBp}5h?ERA0uTrWH z|86Ik$2Dte)6iG`4a8%#Fj4kjF1;MM;A(IoYeAZ^TypJ)0^JoBM3^y4JseqD3Oa>i!clmFf}7fb zI3@v+cB0W$f-T5U*|Nzm8$V!ml0gYi3GuO8?Bk4JR(U^xZYQDcY7@w`J8-p zCPULzLFuwVfW@>!kotafm=-s$^Dpa0FU$Mf$)DveTz9+L5#BCC zF<(zep1`wIb@9U=BlVp3oTZZ5;9V2?*UI=< zH#t>12`@0Ih!1uYzUty0SY)}FkSLE2yaKjIWC`9uT8HAmG1a~}ZqwkG(h`}x$uE;` z+OXuTKKp)mE{O}$`I_R#Y(p}qCz9tEuG+_<$KDvyH?GffAyDg~610iOs~G3C)PR2L zfDf;mKkBkCPw;v~`i5sj-l?E4$wK_xSjXA!j#E)c7QcM;4Oe8VN7VKn_Q?8>lYxHB zt_cy29Is%fu-BBu;3D=j`PsuJ`A$7H0ATF=RNm;q1I>*;s znEKhFn@@%@!mRPIh=R4O$C7KkBe`dfZqhz)kg35MnjxgCX(GtH*;wgNtCv%KvK3%a z$Sj>*9dQr;P+o02eL{UX%@eMy)W_0PDP6N>I*E5*8x!MYUli~2G!M$H%@vEl*Y9R8 zAT9?-Q1J@>=RNR}dR>O^?(bu{QY(0a>D*0feJXiHAsO80QEKV3=cD^2%Uy(&ZNak_ z6xv>aWkzUjWlQzyO{D@V>l0dP1}Xs<&&R~fAQs0~o(K;- zSM(mADsAQ>oGG2=gM8p>*CS~8jT;n7igMB#$9M61+a0PsDe7No@;{m1 zb9`^f!<8*3g97fr+oJlD-kP2;4HR> z^U{xp8M+xv+xcbny7b?j-^4mdr3&55Y#Hv(^|Ijz(bCQS9_c*4>=Qv*FEVaoy@hF% za=0I7nI)6=3ph1cj}7x!024|kqpqm8BYO44T_Y=Cq)`W=9cl?Vi;D}4iHa{pozsdq zBjchd3X0&1*&b#2ZN>tF2BC_jg<9%SlD6cERyUEs5K?J6UH;czRJ!d3jF$*w1EASQ}WV;_49LY#qx5KjE3f_ zOata+$Ksc4YL5KQsrb>@v9Br0_*-zf4cD6CG)Kn}u=6htRS5yrzJm8fRPCrY2_HGB zmZv6n_p)mQhabeOuHI8?@cG%*L4;Y%5ZmPlWTL~KUYA09iHeEUkpDUs%~&S_VV$KK zV}D)&a!E6rwtwbb+GpY{Ob3BtigQ_`I_y1yJtxAptM0JWQEXYWu2k($`?ux^dhE_B zShXq$kIL{7xAV!G>$$eKN;WeQh7;a7E9OXj{VhuS&8`)NF$E@xj>`sxBvcHfEEB|SK7eLoSjy=zq$=S*2y zOioFQ&W}cCcv4e$NqR2%30vZ4fz)O`JVQ%`k@CtfI67pD=D3A0VKQSUJvK!>LD>lc z3alLK_`bF9FbpQwbn|6tyWUpv=#j}Ps6>!~9E#=b9=m3^o9GjU*lRF)%1iBySs8)7 zJC@x}an_11F_MGIh)2$2n(O4~3r;(qdE84o)EAe+FJ?Clmtt05D`En1wI1}!3TOw9 z6OUL>nBt0V_-#hHR!QY)mP9*7+Wq*x%DCvza27su@Eu?gN**~fSC<;QxE*Xyf)W3i z?Rlku?03GK7#Pr}qDq1^9WqFw1*kws*jEf%EytKo@Ycu7RSLH_rPwh9+hcc^TSkOoP*51318xuIB+|-3VyIZm z!b#A+d8j_ilkb8hmk);RCAKIjS*S`0gN3ZLC6{2-5l-)&_@&|bnTrNN*X zAch)9b`T-=)?8ExGEaDMy8V%PtMf6knTH2Nx1s+i;^~b{Zn;`!m5JT9a=-SnY05c? zr!17rv36Ny6(P!e?*>bYFQtdK{kGVvog1pB$ zD^^Z@%=F|mdKTp9*dmA*eER+g$M_RH(P#W*jI^rnm8~{*<~zEa=psJi<){c#KOikG zLA?~MBKV$h*G!M)%IPgU(8R;T3g^l47qH^e5KhY*y6F=jlU%ol+k4P?v7e+&gr>*# z>=Qw1hS^~mB?at_aDwT|5{w7BwOA8k^+D#%z?cA6m>-+&|K;jQF=;m@OBj5bvF=r$ z0(|-o#SQy`&vrL!=1NyA;|y+y%uOI1go*eMw}#dWqe9KB7i{ml&3QmUJ}v*RIyn5N zoULEmeIzsrH39#XKb%24eE{U`lK=Ml`~K?}dxb9EzdiT%@!iqBAk&*B^5_ML#S^Ljnqs}jnMpO+A4N;sl zL_|i&eny)jZ(Kf~z!_onBs#^#4vuS*|o#P2qt=XU*1 z#iTBe;$8oRL8Hvs|e`4$Z7>4}s5rzM^cDmLw|1Ge;b?tvM!+$fwZ$`NO`hR2w)PaZAKPEpp z)H7#peI^LQBp6_^o$K=~v$gDt+^kIoD(%}8JJcB#&bklmn_RYVff4pw zTw^;al#!G`<8AP1SjmJzovr1JCBh!!Cv2rZ?hPa)6)`KSNN({Kn(K=+aI~z3_lk7d zLtwmf2IH%O;5F)T_p{=0Q7dtT+2t$CiH?Z|pf)hr0DiNg-o9i?02%`DaxE8lxvg)) z;?yNoBvNR3wxSmfE2*k*bpV_W;N?SiFOg0z+b9zX-$t$~n6st+_-fmZ=(={Klw$+c?id{h zc#Z?B&XyO9(B2fB+lIqTLJjE1z-C2I%H=j=gE|K*M4fZ$Vl5XGARs_b+@KCm4)B&~ zf%||-gKphi9|*jq3-TCGGbA+~NM=UsYf4Q=5K*XVL(-vF+FxZp<6x^BG0-rVofdRlE4KQ8@vr|6s zSpI5S$;9RXkYi#3peN8}7|)09|vP$9P}$He4j^Hw$`-qrRpy-R7-kAc1*JpYy2qkmrl(yUM zUSP9Gf@`sJsO$9vBhr8X-D!$VfFZ8O+`zRCpqdXX2~gGpkQiHT$Ha;O5kNw<9m&8P zh<1M?ZA|!8jl$*G$?E^pHXc5YtZ^z+oS9s$$b)z$A&%16DF_k>rX19@cGg1FkUu zrve*}E%@|KZ9fQWvu!6u#Dg2?paGDHN{U@FD}*T|3IgQ?8qu}|1zA8i2O?)Hd|)EK zrO>)?x|iB^77!J_<^KQTSAfmt07wBcI9x*$bewHq$pcV-OE=Ov)#RGHI+DR#wH~ur zKo1iJ%cGGk6}G6B$ZI*Q0*FCTMOpM45Z8%{-bg86v$LIM_7_K|xK0Rw8!G`MQeXl2 z2V~4!2oU-0wB^ccowYg21-=IBFNI{AeyzYPA=~h2k46fO+u@a~X5Eh=4 zL=Lc?Uoxc;z|3Zl>tXz*_#EnN*Ng!vxn^=R2(LDv9sqg)-~v5fbHOwh7{HiFCtM3? z#v9yrjVm9K^^P4=UZ5(Vp(bo8PjstXqX{-EIV=H8{G|^p_3JD{4FLJ9gZJ87QWS+= z^HM!f5?~pzbzmU&p@3!rV?kR?A?eq;3APW2B+v{%oI(p2rK2Tk2+ z{MAWCfe-*?p#UJDzE%D8Et0{TE{ebnke3iFiyJ z7^P6s0cz{IBgrlJ@29obhyx^*909cbd!)YBK!?}$eXq6lx$AEl6>s(fiWoq~v_T!f zo4}u{S@%@RfC12Wg(V`h4A2&U@Xxx7{id4x!1X8qjSVP}-x$WSP5SHEZ`w2ffJ0`l zmBs(qM{o)=1(v0!PZ!3%_BL+b3XJW4M}UlP^f>d%5mkafT?np+GBle_>|KK!N}O literal 0 HcmV?d00001 diff --git a/docs/images/quick_6.png b/docs/images/quick_6.png new file mode 100644 index 0000000000000000000000000000000000000000..56854b4219557f16fe0390c4d0c1ec8bd7a1ae90 GIT binary patch literal 160393 zcmd3N^;aC-wrvuEySq2;?(Xiv-Q68RaJR{bRH7Yc7h6nuG)&?F(r_&$5i-w86y-E_D!m(2~jdhj(#RfXbB@f&MZJUM%?ft`tc!XpvtjlOmOZ%vfdki9mLoi>*D&~A+6 zg{x=8Ru>=U#=}CB^9DOFxaP$C7ad!>oe!mc*SDs|jbVzT$-YUJM_upx;U%+~u9%Dt zN=YWgiIF3%W%a&}w3)ts7KEjnGHgy(?~^5Yr~mNwf4lwiLPi4Y=qW_y|4Gx<|zdc6QyNrv+c1dYd0O+*|_OEVamvLb>9;s53pI-mWPfh zj%)P?C-5zto)6fZoDkkQ4YWIg7S(Dkg7E+6Tm_1`>05ev;q-1PktJ^LNdp1$l<10O z=uNAWLZbimB;-JWJGFJUK@&U4$Ts+<&_M(YrgosC0*f()OnzH*U zjv~>jBK$=%4C3ti?{I;#{xA??6W0>jo zG%X3;R9MiD|JV9@sT3A)aBze*G~nRjSu*7P2u?Z`Nl@~8d-J^b<2D?mfB-?5l=ukx ziS`YL%F4?6wQ~R(tjGCp2@;gPqniS2yf0uwDRZ6*UjNCEx3%Fx`tpr`+_cLS!@L6F zIqc;0d{Etu*0AHs#f3*S9-;m`RLJF{lv?xSDAuePF9C;eYHMJeDsMlf^Rr!)I5Qqu z-5&x*-Ve9yKr{Z4kt5<_URH5-Ug>$&{*DPIGevJaxPNZ=pZ4)RsGp>fJ}D_F7$D=P zCj%6hm4%BcoLycHU$q-CV!jk9_xq~Hh;5ncO^HsW2oMk;3?CHv_6;j1C+GI=?rI2K zNm*ZC0f0uDAx{Q|fQAxuf5M`=x0$rjQyu2MACQ#3AG1@){b9-It=ZKc8*DW1O=}{`1F%$oGr|3|mojwIl)p zIa0*_$(n}#DNfumKYsLq5^|;M0|L0>?cnQ~icuAhtq90IXJx(dH%&}7TH{VKHJhja zHY-jJx}F9O!xI5WKn(~v$9JJrrElD+D=QWBxjJzZ)j^hUd-O_;Jst4}Cj^W4BT06( z4rV(?KL-ZJ?Es2DdXwDO9$3E5O-_aD0(hOlBtI_g0#|{OPV^VfNa&IqY4o zy5>@vbWTcAR(B+8g*qFS;A#+CZt$I+4K!^#?s0*b-*|>4(A_0Lls^|_iFA11T zz-tM%1bL|_{DoVkvMZ=F^}9&}4pn7gbMo=@c{cT`8KtVW?3ObYU}IHPQI4J$67`L|hj!hZox zAb9cbZbgC@#Ik$00h43*6;w191K4wB+7?4V3c1H&rtc!2o?PDE-rzU+^}7&pXAs=9 zzy*J4X$DUD+GtaJO>t}hR7!VaiJsdD20{PZ`h68h8UTDHpZ@twH%aSq?cn@J^DXzT zr>y=AIL7_u&hI0@&}hBcmD_Smt2wS_-FwTBC-({4KW>4JzqvXf^hndu{Ktc(=j2f( zQ%)SHj}M{gFfS-Y9^mC5$E27QSR#Ri5hp!GIP(e# z-+)W8$v7!wvU$Me9B9hd>+1rkTNZy5)8Md4$ugOXST}c)Ib1j{kCe3Tp|0Ie77p{( zgn)r4&Y0EAl=VmLOXQH394oIpztxZpCwLAooiUNUt?g~hnF*lBgmn~IytR=K&>ZJs z5bwdzE5TfAtT>flPm-qIG!S8CmLB&D%O{+iq@vjCntECRR4yGO{q===b~a3?(zu>` zjEMm-WwQM?Bx<08uZsgmrA&`>9W5)}R@JRm{P5`x0uFL`n0tyq3e8Z7j_@=7UX;LS z<4}b1k)RD<7>_9BaFU#g3YLW=o&3!v6hg+qerGHJ2K@ZyyzsJeGPyeIh6*s1bJ0NBLOs8xH#0>%GSCVpnx3rJUNCidX4_ZV3(B1UZKWo&^ZC2_y8j* z?Vyk3vrf>?JCY9H3oOW8Jvw+j$dA9m0-%_yRh;(5!4SeN+Te|&maSA@lgViqoqD8Y>S<2w?-x;AWH5Z>-`sl7L^r?mI)3_l)b6>}lU?4i z>N;jUZoD@Vn_Yf0Bb1l-OUynm552|ZNj>-bv$bOKu!-^0^#T9e?Lu~WR1{n_{Pf=H zH)5(k^JrYFSeW|D{|;#$+lsGIl{GNIgvw1YV$;%SyUYltuna+eUb$3y60)!nDdBzPU0Iz|avH}NIVeMYaBp;7;Cu!^uU4Sd__obqOEcS=uwZQ{ znNC^Q@D_PBvZ|23o!QzJ3Gs(}n+l*Dms+eLWkS$qIM8jDZZzLPX|>nGbq#eL%LsLG z{Z!N)l9$Yk4J^g;ff|(0(UA#LIAj+11&XupItIE*PcV#FAHJR71qNDp*sqQKuqQi^ zATmQbvE>t1^F=Eg3>t#L9Lo(HIX)gEvG?_;f9oSD`DNm=At?zM_{@ukcRWA25!$C6cW0VY_%&Zj6-WJ5A_Evl|2?q>mtnx!K7 z!ob?ept_@1&V2-|k|gg}zW?yS->sQLu0N4F8)m!+w_F#cHJ0hCO|)b6_&nOXo8Rua zT1=!BhRa_@w%r$(S&KFv0rsh{NkcMSSp+e_H|BzeFbk2nw$Lwh$* ziYj9^K?5=pIet7`7zn_S5J@%vhvmwYsJOV&Vqy>okf1Y58!V>qR8>_+$HxcESm$PE zg|)SD@9ysW($!6|l0b%H4RYjoDNct<1ulPRx>eiam>nCrY3La#2TM`z5%Y_En8y+j zkmpp@l!P3Cn5yt})DK#TxCN-k4rLWBSg7sf`xvTBw5!NjtQB@52y6)Ytf8Kr-tC?bd@QQTSN`9`zapW5MrQvyX01+UL zd>_OU+P$mVs1_5_?K`VBV#@U+7hz#lSqy}7VM0%&lmd#w!Zo{iw+PzdzCc4()?F56 zCNLMb*ulB3QaqU8+LTdR=6YP4W^PSpN!H|MVHTz0omEg1!yo)HmnnLc-oP!-N`C8= zaCk`h3?)V|mCToVVv1=d4DY+D3Oqq@iA2pq{Zn`v&}_}H#L+b-nQa4f*Gsz*f)-5rzQRp4;tJbOOgSg$>#~?TwnhaIMdM3fEiLg0^IbE2Ditvhsqd$H`W{${LS_guJ~>?5de7K2XKuhDwnr)RuM8}MgnG+j@R z>P&XB5fFq~XUDd3JM%6!?+_dtu$>n41Gb}7smP&_y9X57^kz;uRwwmc@V1`cA8<~; z;jk=T;%&XY>v!KiJ8Y`Cq!S{)5^wozLw53Hrr=sSVPLb(bcbF9_@YMs5n_2bopBv& z3z){XJecjeKppK3vZDhOy;XnT?(NvyT=vR>b$xpE3>XT~;MKYr@b}xhe}M_x>yCC5 z+IoIgP+x9iGZ`<;>e!=NR!bL1goRA0uK4x18?ZIPk(Od&E)FaZ!1vnzcuNraYrmUp z>u`8Dqc0MvW-iXUG8G;Fk$UlVO|xkty*K&s1ZkL)8vHrpVfi{G%b_q$ET&eK@%isX zyR{6Fg?>@Tv-#iGekScMQ{pjoY|HN_x_V7#c9b1cKf>y)36XAMg%3S7JGhD-(+&y$=bL&xX@S#Uxj~hd5NjX@-Dhf1@`NsC_^hG~@ehDAI0ScX> zWh0{)j-B|KxB1RZp)Xp>GkmY_VLMy4Oh$~g5MW0|ev)^AC=wWY$9L3d) z#5*e`pG0l=G}r)OMZI4>CoLT-_Hh}zD$aKz@B!}p^zduXU>=lMD}{=ijQ~MM%0TVD z5iZ^Xlw&wq8=IV06{Lr7&rXUhrAy2C)13O^g=~!VT0_O?_YX>&OuN!Zg^F=wg+ua% z#OKY)c99FlXeEnKBjz?%s`N$4x|KDlqYACxP3)Lv)r3!}QlD zL{sd098*q3ncirSY>w0951MZv^oN?l*gd%xbC*+Z)ASuelbq$D=4_ZPosB+x0n#|o z;k4OK2ez^UVg7<%$b3EUNb`IMd%u1C$&}l1^+~J99$=5#Z0{3mU0B>C#QjUz>Wh>KsqjC?eeZTozj5q{tvgyJ{RJ8c|33U7xV@caXdoVz z7@$r2L86_i-Dn1zl9B?(!D;S21a*yJn-(;<(Brc+v#Csu+xz?L71MZBn6K$Pu9R5O zp-$Xju?3S(v$nXcEe$!?{clH%0^9alTTt1>61V-Y*!{_L1)F~O?el$R8I^i%bpETG z)74|G5yEAL-EQ6YB*Zs?bRl=b2@vk%Y2NoK!)0~Xeg;pc``b`GzY{pl*YEUP0jJUQ zv~#Pj%Sv8u)FUm1Z=6Sry>Yn$wpw!&q!kqMUT+h60aCCNwLL(#(wPHb3H*u2zRSXL zuVlgEsHx{$%u=rT>*c`0FE{?0zAZtC>Jr~Bx+j{rA!Pl9b_*pg-Gj@;eYv#INhNDNN zUFEV0M`v_t*Q;_(*52==MBjJZZ>xIyk&mW#f|wNdhSwU~U;oh2u((~F@?Dg52#p+l zyg<`;+{$JW8&0^s=JW?k`6e?N_Qp%3`Gwsm;)Bu{d9t9Fhj5KucvTpfE*DM%gs(V0 zrc#<_wtv+@9={$c;l3_vK22LrgLvCxFE*d4%q&TlH=W2eKP&|W5H(dUUdga?SyBQC zxAqQw_rPR9ft|ybfnWQ|-WAXshzcYlh1Th~cqUdO#La(e)P3CXZc=$^y*@YO4-`1! zukA7>f)ez0Q?TSt08B&Y+kPd-gGE%XNuK77R^5yb>^?b40Fi2=V>kMdK_4jS_Z<*0 zq|jg+Okzm3Yx(pa5EvJGy@z7l5CO<}QsCPQT|Qq70}TcWZ{)nNrEswg^hP4ChD}jX z9+wk<8Y3a}o4|;tW2-yk3EDoR81yJQttl*d0y2CJdCH8<0+Q6IFC$%E(O5)65tf*?)%)nP*kj|P}LE1iD# ziW)W;vyI+0z28AX#tj`#oMs#!^ti)twsFa8?~#(erS!YqI@#>^4-&$=mRl`3=ObSy zixgJU=r(}KU~}73y*Y*vv-T)4S76(*OjwxD@eTeMuH*JTEc0>ncKqEug_KIt_vd?# zIW7m*r16`EidX%J)J}hKyC;l;GSg%dJi_7z_aC>rk;w-gG6TsF zn}dI##!vcMZd)WZ%Id|0)50k1#~onlsyr4-xGZ4}UXMUabj#K~89Mn?liTd3{z`;O0ax+vjEu z4X(C|Xt2u88rJ6IM27sgNXkumB}}Y~hnMa&otBU(E81p;0ryC`zKAN%BrSWvX9@_GN7}j$dbU92ZA>XkC_wnEvHSbDBOe zPcIVuRrkoIRJRdt)7o&H>F9Kb15b>Z67XZgfr^PKJ`^5%_s*xHvU1?W4O{{O8GAoi7GQny|S6@7|2<+h6Fh;G+sJC)sAUA6}k7#ZGhHYEHBw z_qF%H`QG5pX(nZL_M|!k_5OPS{SY2;rID4(zH41UHdi17>^<@OI~#9v-+bxP;; zlLiX#-=1rFnZxieOk{AmvzTiixtMFRH5QjDZ3#_as)@cp)VE zjWY6cVz^nOjjh&?KO>i)QNJ6fY;gj8=FX zNMjG<1C*o3-33$ig|rHSEBrVTC+nBcP@RFc0z&&6K6=CWpxx6HMEC0=T$s!6dIg%@ z=o{B3v-zidGhtD1l?*T+DHXQTWewjmLQtBFVWjqE)fq!pHA+B9N}4E7$*n`iFE4hk zi08*iIm0kq4w~hFl}OIxcW=lT?P=Q~4Si!=&^xHH<`VFN@@*c^r z{;LYCbq9Eq-JC*a!f1h<>ysO5*%T4E*=BDF_m}%U7qAe+c=Zm5;e?jl3`X+N8_{4+ zt%Nn_W`Jwx_h_}-=#T4J&E}!V+8tLLC1x^CF3?meoPbi7*=1<_+8I|m9O<=YWrb|y zW}dXex}y+=YiD1`v=+Q%Z+d+qHwS=(kXitbt$eLH!$6EvqXcqAJ+kc>_O(ahY<=}d zi507PE`}l=LyUuqs-Di+(7c;E5TW_ZOzqBqcb+h2Oj_a#(Eps^7%_>bBHOKL5=MqK zAFZ^k4q2%aqKz%wy6n)HITM^oVih+N0Q_MD8e`3>KM~S3`a-!m?s!#GoKWxhBS`uj zpWsO{&J1Mc!sieYM4NFHVU!ej=TQ*B=DR(rj8l=WaShCpfK2L3cUg6>?&=EIQu~Dg z(_bG5@-y~W4$F+y3Kx;jOF>zp%mv%DxiBqVJh@KmGgXM)GNg_!wBED`a%hxsVywn3 z2PR9ec2^^bcLpXStdw}?>@4CyQ{`-KH9ZsuQ%l2in&3g^h6owpBqw{E3@6Y$`4~^W zo+;T#N3U$rtTgD8@4xj|?$3#rpCZcx$GZ{KBgySnC(}nndUS~nY3#AuHhCd08?OT# z_M`$b#+WR;R&TD~*csj(9eVBvG7Wsw4f(smPPlRvHsIqgyL8Au+!vM`p6;&~tC*B^ zzjH)tzNak@z(0-Gq#0da95l4IWsm)<7a;Eg$Duuxt?^O1+)p^Fu->Tu3mhC=#IOkh z0s=}rsG_Qh90#G%_rZaekB^3qZVxEOXnZ)CQCw2O!pz)vIF-ecHIyzqA|h)3n$|l=Zg|LmS!N&^YAmqEBfh+ga9a(?+*Tlc|HU>G zAUkX}O>H9iy71adoD<8&ITXg#IwzQ|iH;Tc+<6#~`!B%|wOR$pgQ+mGgWQb{`ewvDfulIn7*THcUbJrB|QxT4Jt zza`!4@)qFrVx$3vhcCSW5xO(k{mJ7>H)%)HMFFRHL;;U1->O%oLwey~tdPZj0Av{A zkfebjlccbDX`6_~Kw<7nE+79kwZzo%FMHY;T~jD`jmB09qzcdsL$jBk8j8hbzwu;P zT3+>J34yBFQ&I)$RJP8HOeCPB7MiIye=ZGi4;9j)AkVX%QPo-le)}W*LO{>Uo1luV zmM-p6pJZSm%O=qtu})&41~*=0WB35Km7#q-)EBDC*u4K>TxUzVZTz6O1|-3<3tNQ5 z)t*vAL4*&#LM&cv$_^%kUM*0n?KhH2>`FH(k`*}rbAoh~xL0|2YCj4dImVgww*-o2 z@fbk9Q$%fGweZK!9QG#)*@*eod3m!wCHl&Rh0UM_68PceNTv}Hph;g!V))?XOXK0U zdS*)kAiz07)ItM|(Ju5oElgf4&KRA#@8D|WN!X&V=BHhrA+pyZTzXcQ&DBhoy zm+C5+UN2Vc^Tl>kpjzbLd|LD1G?}h5Qv94mV`Lh&KN^sb$1Lw8E#~f635zSLg^p1 z!NouN2d&hLwPxKB2@%~0wK&)Hjtz zVoo2gn!o0}a5hz~N&y6aIV>& z#)lSu82sJPi}l{5x8br6GQ*g{vV1x8Im`t~i5@^SuG&pzazq1xXq{bMtBvOHoLRAj zg=F9g=W!i63^wk&1`9(U`V0v!BpXinV@dg$P92%!|T0;m~m4G<+dNz|p|3L{JGHucp9eUPv-pW(pW4 zWa<^(v~F{@^@akK;7O|>P?n$JJ z%YAYBFR+=e<(1jOwF{##uxy-SKw)-qjQUSH^os16L#f7)ZUbWaXB#a^tzLL)7BN&* z0+69x6|9xo6yW;$vReWLi8M*}kQE22L!If-G-qKyCClu)-PuiO7m`-x zFkumPWRK(EgNF~-mhr?pj**O{_(blA{@2m$W!fjG)UJ|Jfn>&dpIHF4psOK_(A~d% zf|9MEIs${7Dy=RjXUc#-6n)HnJZAO|(8*Xdjs`6ej*CMulFSjx9!hQo@%Qy=a+2D_ z$>~SG2vZINqTQA)nd>*)N_yu;iak`}BZf|>F_NM=qDYpdQC)V1z|;t5ejQbEqFCcY z7y+y=3}aWfwy#)1Aj}2n zdx<$&Zn6xLVp{@*W%o^UZjtpoVNVxvGD(=6J-9#O963c49I?#*)|siKHRwZtBF}!L zqtZJOFnoqSUnVAJ66S8W-d{}+#UP*>DUZ-&DdwKKCe>AA;H2az(ZrX&wql^OO2N9% zE&_*GDTMn*MV{2Km3b>Zk>B}Ys0)&SyWPae(x)t19X7@r%_qQBtip@`<4hYz}dGdNXNP~q6R@cx_(bNP%la8@z>v0kl z5t$wA1#Ev1eNPd))@ppUsO|7W*b{W88UNs97X2nKFLX{jt#e0c;2IEiTFE*Gq5tas z!F-d%T4}d0ynq7nQ(}@{?x24Ex6W@z?zG0&4!ukRQ7QKE1I>UqS9r$46=#MA$32|i zR)4I`nYnShB*@wyoZl-M_bj;##3jA>HtuGWCu9;379Rib=KegQVEy|Jf25c`!?SSI zSmg|1kmKnZcW?v_(VYMz5y>(9*935z{<;+oR;IagF~vVN=M|;4v>eLmx0#omXacT~%9<`Vfv$ zZ<7V=me`U+Rt-oPi{MIRju`h(pWBr$Q|EDq>a_(v(#X*$Iwb(;w9W{EgD?mSTMF>S zJc#xLBxg;ViH}E)srzzmp!CHSM1W?1NR=eF*!H@d-$aOQ@DKN`ebD?UoXe7}|x8cpQBLZNxzZm)=>OR~iYO_Q}Qt+f>A}T=0022X< zNNPcwVNhYtJBm|-@<8DLcnI$FOcRMj5xjpa0{gc`owZ2jmDm^iudEVruHHYn z{1yj@kptI){;mZR;qQBe6_JMRs<^}^B`%T1KtTk~Qc4c;p^#MKGiygBoKqseAt3!? z#iQYb=b|T=$jFfF-XPTL?Tk$&0i{Kt-yorBu;NseH<#1kpo!cAOaMIKtZtJ61Heuo zKHjWGiO-BjKe|I|Gs{zJ71aOLtb+9k7^9Gg8%MDc{clAUmX)uu-05hxy((AuO8uoFk9aIa0N#pR8jn1iX>g!@LXw%rSkhlf`8#c&sKGtY$CC>H3BmS{OF}Sg<%m zC*G8riZWEuo!DA3siZM(_^^nwq~D8<{yqakn3LQs`~7~=z>XFmYS1e1_DR=<2+&QL zRz>CguRg z#P0Ei_jdk^pzo^o>)BP$D?-Fem)AY=i}mkmnLy1nJPQG8(M_y7|K$)w^-4?9rLC(r?*X1-?+I1*ceR+n4Tum9=Ja`yX9)ZU z!@d;wneBe6X^~w89|Ffci`?gX_-$G<_S=xetLUF&@XH@Kn1qH8M7ggUI+=e8ha6wN zAoE-pk2ObYHCXWEqZ8VgZVVdk;P+aYUKes%5%h+PEx)G->nC=A9xQn=nqRcqGdt`#Z8>u5d6?^b2z8-&WbmLMmc*&tcrQ2^t4hG3uOM074jHG zblo}5-*GrQR~mjs561j8u~1Llq0fo1+~efz{!30SVUiz!Iz45ywM+)Qn%}@3**&e6~`h^-D;fNXLV?Q8w3}Hzg+J-Xv3@j z?aQX%ezVJnrKJ%IEdhf=LLvoh@K}h*XR%6rx+01lM%tog%ZN98gVLF#urp#K7-;m$ z5uihA1c$@+ssKyrjJ|Y3mP(R3g!Ufo$!2$w`)SyiGcL~ALm>~yMb;V)rP%n0vXzTA z+r9rw_B4WQ)XhqVCm~(e0?TkX4nkE-mH<&&e}wxP4pi$>#UXQ1Rr(3~`q}z7CFQ2sHLZ!?hdM~EY176Q z8No>ddPDop=iMs3u`R(2#iOJ?Ju=rLN|b}<5N-pBFa_Fv6=CeEB?5L^?Wv_P z?8&1j5T9e`#~`U2Cqbewl)DmaDho=(vW#CnfmSfAc>S?mR!q55F}tvn^v?6TZMlD+ zxj_enWSurtS5>lbPjpOwzuaFx}b_|8h$ zWGn@9@6j-ZMd6K1=As=%;=!kn5K18i+k=+zozNwR8lf< z{`lXPO|rUg@?5d7;yW|=Dneu-VoV#Nt95hcqxkk(BkwM2=U#&Tm$!yVDqy5ZKqB zP`X~iOtz>$=t%TE^5Oc~H6G=+MSjEKr_Y--ikdezM9==n75z|d0@9}E(XUs3->f#e z`j*EIEs4Xz*7>Rh(o=Df|1uP>26q9yx+D_pK)o`Y)`QC4=vup4%-?u2v_Z z^DNW-+?;UjWvkVmR4Rkt<01ltejiy}jn-GI>Yei7Qu>p7Za_z-6#}Q@J7(LKijqbv zp}_9JT`Lh>)5K`v^ChLO*2`k>BP)hYrMH~pOVR7LtuN1MHJfvW+W>@tUm(Cb6!&)V z>>x>sw1+y@mP{rHKfFBJNj^@UZNkEWnS26FcYQLbpzU!>Q3`wvTG z!EJV-vjN3`kGpgC!<btC*HBc4~xDV_QUpA*s``ri@dtzI)B8f%C@9wdLK{89@{hbvZ1|EY}Erc1lr2sqL{NBf)X*O z=j#-=K_x~J1HwJkqmB+qM4+7hGKtF1Tp^=Ni^@}h>5YC!PBjU;ffzTAg>|i9sRLd% zl+ryTWZ1f~iIYtf==f|vedqM|u@OY}g_UJ-Ie!F=9fn=FlAD02)z!EGj9;%?$_IBc zR5Mp4GnywC(fJ1caiJ44jpL_@C0>U&fGGw)dw2EsL#5GJgQ4BKmF<(K-T|W!JcWa? z45b^-VM2pE38I%Osj32;j%m?r>Q`%2ngr2Da^Dx`HU}HE1Tm8 zhELaNCw#Lpo~KoQ#xB`47t;MkOY%CWzT?$f z-#T#BW&9E-Oo9UQHAojt(242erq^zpE#M8KOewA1Vmmw_6)1k%KAHGvEjEH{zp;Ikx0Tr^hbh{n}vI z+t!an;@g141#CsOx{T)Dmp#7+!YM0>=$Cg=fV=};NmJ9p0Tw$K*UjZ1GMk{FqP{-i zh5v;wP1*zo93&V6d0h5ksFZTWH8kwjN>wWoz!lBhTnjU^uY=6W7@dMM? zQr;_I)h^YsD)`~XG2ndpu|Py9;3^O4DmoxU_!8;zL@6M1FdW%cmj??z+M1}RdmZ@v{u%xYB}mh9jWgAJ z=gP~As@^`!rX_0m^x)e{S^%`=8}26cM~)CE;C<4ySe0h8Jm5;`?W?cQhikWrBw7D2 zrhw!vqBr}^@-F;~QtN`qqg>xOu0O%VuOz+9h?<^_-74|^kJJJ0x*w5T<3~4FJIDd) zLhpM*hp~SyEFYev> zcz0>(>kpsa?LNG8VDh$=>(r>KsSWlALpRuGfhnYR&YiG#Yi4#9rLxi`w$%F3Pf$lk zr=C7KB?U7qJbaEl7JSmc4s(TiloXM7h#hCNFo_qPvO5w%`3^{@=+%gFW?wb6A_kFq zQQRNFQ>y&t#0wEKv~CAWJF|UgNEFVoFbf@oyt|iDeK(Qa*&arvC+De_kP^xP=YhGb zJ2AV>Jk?@=@sp*iv46FU5f$g1C=B0A!^1edqH2$xN-p1)-Gj*VQW!)3K!fK6p5N6c z6F7mmgY@%sNQ7NuX9?q;v)ylpfM!kH{S9F~0Ekr=I8fy*Jsr!!+a|lTq&!=3LUcGA z$bGV;!WieU8L0E&OZR(c|F-Pu+8&wK?vYqhg`G~*a=CrwVGwS7E( zv2+Q_!sYk*F_{uQdA8p%O`B9@LxA%$x=7!-mWEQZgTF^o_kuZ=L$G>)(=@HNF{9$g zNlDEMbx(~kdy1xIN=EM``(76kltwp$r^~YN7Z*Ccc4mc6JJS%f+uPV#wB<2}8K<2A zb_$N^{vtwv)J&6Q7IkYpn5PPtY>ny+cIDDIFNg7Pi3o}XZJZQroMYG~IJev{U@|L7 zyri5L_^jLo7NNn-Cja+U{}Q;3INu?0;+8o#r}DSsi0J2aN}Y0rx-Xuy=)N@6hE=3 zAnW7w7F5MwlH6mb+nr{b*xCkDC^j}lU0v`!>87TZS%r{cO!Cy9CYtC_>`Ll}2gi56 z zO^<{Jn|WsZx|kzb&;Vjdj(O@~>#R-I@##5vIkozAmR^oNsV>a6Q|$2gGm*V#cRV>F z99O%#DqRfInKG9bGC3lp?!-;;jrN})W0x@r3#?Qv-L;~FPh1^Ht2O3WslUcRnO-U! zS!VuR(dX1&MI4I~Vi;|j>H1>}{;%J42J!Hl7WU^k38gMdl5A;V zh|i08Z|KWr6m|$5j7&W+8P@xS9qF!&GGY%nqEbb9QVO*0b~|AFkt$1ReBY2;r^ zte;`FLn1{slq%ON2&{&KQN{Z(!?OM3{RNy9gF&k3l(&u&Epm2YVQ^^ZYwrzTuNXDP z?jZ3y^Tx(T@q+cwb55$Kb;mbf!Ej-c$5v+oe2}}7J${v{h5b7%K4y*)IW| zc6)nmY|W$9UtH2}N6>Dsno|TX8tO8FY zehkt38>Ljn6qQw+=1RIgxvqeyoIdj4PrM+?l_l;{UZs*`V|SatRYEI=2<6R8OA{W@ zSQixaKEwD4?sT^HSEZPrxgM{X!)GhztU5omX5}oLgGx3&D3N}<|D&Bm6f>YQ}70U2`y_v*Q;C5P5&T_ zAfr*#=8)bpRV2QdbzNJ|#xtp~F0lXiCo9@J=;sRPY}t%k3$&(&xXc`OCQA+0YP++- z@IoBgWsQfnjBegw!kT_Z%yh%GQWq~Adw*fR_>9QoIgu^hQyHA;l<`GWn$^IRZ%kx?ra-f86ff`J~uMw5G%9|2?@tu zoDXPE{qxRAmzp;Hi#xLt6)t2JegQCHU?S;$sw<0 zYo!}OSMYtiVo3?A+)W})T|-^->}KM&gagOKy&}-45Pfj3q{K0Vb0NBd?#xG3Pj6DC zvF{XPWqlIG(V@7pagd|r9I~Zew za~Qf7bc>?4HSwZ8p=JgpHH^~((}K$>a2T?Xe&}u<3^7}u2+3DbO@xl|IJnT*PbRV8TgBBRP22>ruHnlS3m_7l*nq-A9k0pQ zn%0_A?Af`7Q!I(dHB`&3Pt3jYA?pf1QMCTjq=rPERXIK4H6#xnDf z{?=?x#q%~~O8pLV{zfkTM(eRALPluNO+;8WW_}f^!P>!E5*4|Xdq`K-TG7diI0(Y6 zy+gVGrlmJAkHa!%aW7qR9U|)9{Oeqqae3hvhQtj~39EZbU3RAA&64Y`H1U!N2PedAsN>=e!SX8?;DIJ|depvx|74Z`x+sv8))Cch~**FM5V4C}{wd<5)jDry^;U`G!2)$&3x$Lf@k)m62 z!{9^^9vKNM_I0qb2fU*<~BA_cX#e_L;|vuNjpQ4=x2Zb zHum(yf@7=j1~wYnFyvlCsn7g@%tS_cnl4q^ha=OhhUI(HKcU<($CuD_Z}#M=9vbAl ztA(kInz~qJHn&Ecdd4}qFs>n^e13#fe4^bHj!5G+XSLPzzPE|X9RyX};A61idT{+} zZ|7E8ZKdH6ArZg0f<8V7OW!hih*T+^7jUzp@UW<3ZAlxZIj9dd&YxVmIVx|>+3)W< z9aG3Vd}=Z!=q{|5e!H+C^)=5qt)>mN%iRCXIdjgKPpx-U!_`OIU}?MVeXC4~Epe!ynn-_#cWKC0lx0RvuWIPeBP8eXZtlZoXimoV*^^v4C{EQ;U8bg{zP(kp zzm#*kKwc^p>ZRq+(q0_O?qmfdL`5;+#7VE{(I!x5lN!6c=FYWhvDL#N z6sZ|Xr-OnBCRB7iN|C9p`sdGFT@6FSnHsoL-hU-qjxyW?7M`Bib<5GqrxF?(o0aqv zWI2kivDTUEv^i@=zP?#Xe!J*?JMf18d@rAx`hTc<>!7%|=6x_gAh^3j@ZjzmAcH%> z-Q8V6aCaEo-Q5Z9Zo%DMg3C_s`@Q$g-QB8RZPjkoZ~yqf)XbSPN4uYXx=$ZWshOR6 zB^nIPh24~qBnHKD6*Vm_BM%R}Zy>`a*HsF=)!*uR3AS}qMn^Ayx;sYXLolRcNdQ?~ z61@U^ox?#!4|Z{>W-|EjvXqonHv2;|?nt#NtwF^Y%CFX<22mVFSD<;!X?|&VInNC~ zx0uU!tU(r!O*Iek;fU61Eix<&X>U zZa~bDqDSeK@7r2pu?DEpR5LTl2I9{5_XBKFrYI126fbSd=js)Gr&+& z;beO;xyt(a*7Z@+LHFQ5-8XT~xvh;@T|E7b=BX9(z-ni3NUVaIGKtg6C?dGPVTK-< z1MsqffI_xBCu8);xUmKzbC{+C?k&Y@C0UdtMr=3zSMh;N?t3Vy88UptT95$H!!9Zr z)=02g2BB{x6MD1^OglN{T*6W^f#J2f%@BQWq+gpHfnIMhpqrYeb|hE z3}m0mmWk&bs<1hgNTMT>tzGVAYI$4FIv1B1MhVT7uh~U}@93n~r*gkmz!h1V;TjW5 zNyjDK_han2Sc>6VHSAHS)gB$(R4_M~Px0IOkNXGVUF}HFhjf9do7aw}Yvk^{Gf!Bp z$59X!xI>z+YHjH10A|h5vYb609-Wa^>^5`OiviA|%%MGK3~w8XDUJrqX%!eher`jQ z#E2O0(#8$-&{SK_(I>YZfmA6prV8-831V|*Z@8Fk&`ZW;m?kAyvCA7oz+oMuK)y1F z)pt9hNic9+t$xElJ=30CKyO3Az@n2SY7q_@|c%>pt zBKuBhLJFAi@GJ~R>i<3a6yV?91?uUlVMcCWs2HVC6qx0h;jY6HfL2D+Dmp^cL{LF7 z_8PAHhZdaHN7|^!Vps6`8^UstO4@BzR#>1{!|hf4NNt+8dB*YRd&;x3{Mtp`=h1B` zbDHWs64F*`OYz)};t;&HlTj=Ec?xK8t+Te;UY5eljaj1)%Ni3yB0}>6?1TI01q{4H zW#FblErC$jhn4$7*M(nI*|t_AW(%CDu)N?`oq{DN!tFpj$9D1N;GmGXh;5c$`sF^6 ziRMBuJ0h~g>9!IElg4!9Ez*S6U=_0 zbRhyDGos-imDpdd%xIST3QYCKysGe~bR*!d2_lQ`Lqcu366jSF=j_Fu>>q#m4BnLO z9eB0h{5-p@>C_c>uxaZP9DGOIzDr%f`_ve(nf`U=-N#jgn1yF`aK8b_NDEfgNA+)jt7Ly!-2k{(dL~fH zj%9Jugv7KS+CYD!96tKVMMzM~DEtc^;ckONOQHaa2#Q7KOrzT0raL`IgArPUb~Nj95opiASyH6=CzFFg)>Vw5EOFkXzVm7hyLT z(jP0-b>{cN?XudfaoQbuJ|=n+di~|o-aGi=6{qw5l4$#@ZKTdyh|j33*A`PlDEm5f zg+a{vMD(ZSrZyI;l)!=gp8e}h7IcZH);viX{!FCIiJ%D!UFARTI`UVFvdX51t?{7i5_H{>}m%tc@SM}<|ozu4~MNo#Wpxdb}CblKl0{xnNW-1{jdl9Qe2tD37&ytQ$a zZisPRHa=RFz_9%;(g1{l?X>>cFQBQM&-zXLG{z_{Zq&Xl_~<_T=y`r+cxL3YLxR0H zyW{V@>)7wSjC*@dy6@?iz1$X8;Psjsaq&8!VoJbjGUTYVKf5A)p_El1@*QgqOXK6{ ziagCdp9CNx)S=Ci@cG1;6ntWZy2a6vo>`d>PyH@(M@We>*<4WLmC)(N+^2PZynPl# zD9k7KeHg9`{F5RUmxFU?nY`p%wE@U^le2^6diJ2Ubk-DAflHKdLQ$CMje7UvWo2RA zwWQCgq4kdr0F}^g8uX=fm=T*GP_Bp38nD4q2ujI(Evcyeum3 zo0}A=IP8e@;pl0YE9>|;uP%OInGvLQO)m`IF)bYjEh#+2f?|^>*3+CBY{bU z+hLowWQx369xH7fqug$)5+hs!jcQ8;*TPdWfUeh-c&}fD8`&V zA)AS}#OlE!5sU%nKQbVk9^za1^3h2s{n#FmID=XJ2-Vg{mU;+VFp(Wyn)3mzuD9J# zaXHVy$nwLaqu&(nmY#rzhJ~R7M%gKXic@;O9Kop-BBA0oZfTl<4Torl<+q&8s98BQ zm@FO)<{_)A7NqjnLb)127U;QIMLgC&bX}KK?q7>s-t5-y^+W-f`Pg%w%X22QDT<3A zE-)Cyf1&nzG5fJi0A)9>^AS2`+o$VEoz2$DxZ~>W%A=6?dHJJ@Jm($2`{gb}%*K** zr~Kjolve6Si`=Mi+^I^ZK3k&EC_K5n`J0rCXN3W`dd?e5E49vToL2jEYaop zYfVfiH_i@&&)XeS$N9Q#-ds;U!wX^OOY?}oJ_5E%BE#!1tfsetS8^g8sjFsz7isQa z^xlu(WTr<7A69+t;>gzDD0#0WiGI*lTs+7=gITO!m;PS)8X6(HLrhoIggNzkiG}?? ztDf(ju%zpU_AGNkaH+6tl>>yizZZ5S^?cWQUk5*vL|>a&fAP@%Gk_Lk^smKu<9fj| z*st=|s#a_qa(`yUHMV)P{ipx?lvNDf9jG%3q2vVR^aO7pZ zBh=60@+-MWEc}UaJ71tylE>w?K-@9Sj+U!RHG4BPsUNeHwtLizs^&B6qKge8pV2PwcKAWk7qxjHnXUjrKAlu^GEka==5D4?gejFUu) zUAaO~$mIXH+95Y)zjF2SUQx(k_EvvHn$pRQNh31hE!8#W=8uxl6M)NBi6l6tXw1x( zv#XkeUj`8LR8+O7k^a<-l9yo-@D6A`MSacs(0U}u7DT`+2s*C2wZn;DmQ7#epkK&bn;4NQ6LT{F zy_6cRRc=p;+GQ*G+jZbv8_aRgL%mmA1hxc)I78(0GXu}0AWOG|TU(JNb1Ykoo6~Iq zndA>KZ<~7XmrAPx8J^*9+E6^XMQyP;?gxbU#9r=!N<9-78C~nU46+kff))%V%3CLW z`$e61s%A)1*E+8g0@=Ib-f>ZUKLf-|nXgw&?jt%u$-ma8MA3{&>;%@YKN9T|!S|6g z4!^GR^)$mtdfNDaw>S0A=wxhs3*r|J>yl)~GkK_;;2YYwdxI5lROKj8)P0uB+|YSB z^x18Fz3#y;v(mjfg3W(g`XW=Vn}3X>@s-Sq=U%1Itzo)5=-hPU^vl%q4L(7^JcJ}x zhQ2+A?D{0*sgpPJF%}m`y=cdze=T2^{o2mP1qyBtw+wa9^A7e53jh_OzKvo*=<@Rc zK28>WR`gQaXx}Bm6+r|xEmOUER<%p=?IUBTqM?CzEbvfm2@er%nr#iXq>ryZ(<>Y& zK^2;Py(^>xn?P&zBqEZ99hg>W1PwGChIFPBE6p}3EenofaBw8{Me^@Z{<*o-s5sN=DhcW4zo zBA-Cz)>1u4c3RCq&?&CAaR1Bcxgt?W_Jy&1vC=g9#SFn!7EPPqCd||%`c}%yBle61 zxjJaEEMk8<;)g>s7?mBLv|AARk+1&&r`aI5B@5u~l?b=}$#UldC=y}Fnp<+3_eXXY zH~W6S+AgGZz|K5A+%h8HK;#VWr2>H7VRM))apX9+rB^+QX?9Cw4k3{~Jg0w|Yh_vJ zOVGK)cP0C5TO=-}1ro^uEi z5@<-gu0#{Bebk9-KGE|lX5*Q+*X=|E-7GKr&r_chf+~3XDU%4dQ|!$nn>$mto_i6U z=vmn)4vsA;vEM(_@!FF1cM7-!z7TPSx7dbIhPHd|bkL(Gx=pS9G|TYu4ik|syx&O} zXewx_@9kGrX{x%LNy9GLJlTcIyNzP(YyxZxcqYeOTS)DN&W2|WSXuAbT17V!LHgun z>;)r1Lc76B`6Qc4<5zwtiBWGm;^s6`X;k>B%#b>QM%LV4@U^JM#!Ajvzl}olBX)!$ z+TC_nA5pjgXP99bP5&MRSE+=(LC6Xs$e}7)0;D}Q3ewivE*G5R`+S5Og5BoFaU=v9 zieGg2yj3F)Lk?0$^6pWN@_6i2#?C*-aO!Smkzw5BzcqB^*8LK4Kf?f-bQTkKsxEwrbWjQ5+naSqjQE%^{HuW9?IuuVr zh>J<@s*sRfx9L{1Hs7@;y2DHQT1d%!T!MDO`oYc-<4a zJN)L0a*JV$xC`{MXjbIPqu19&i@Cl?vidD3U9dM!w>1n42P}xrwS7jFZ_}esf!$im|^=)u~cF39m=0#)8jv4b9T}(O~UEhOB)1Af~u{S0$%CY>~~C+LE~uW z`M;)Q?VJwvW)H=aP2WB|3wuPbEE3pynWlst+ot0j?m!@AZhSUDcS=)L5V=`(AX}dCvmOs?fcZoF>CnQl+qJ1N@v${EI2Gf?gcX}2RgM{mWQW*4+^|>u>UrzDHyu+ zZmRsfdq8M_u`6^S>3eu8iOm24cal-8gF=u2EFb5as`NE$*S5Q&0ey8^E4 z=$R5Z{HZw~Hu8)FIUM!IR)kZyjW9>M>Q;!u%QUnb0oGe_pqfTWvJDDL^TXUl?k5_C z%hZF4?*f2m#R}j=GoJuYpT{Rzmmt z{RiTj_>EiPd=_Ei`0nWr*i9W{N}4MW|MdfTXMV zf*{i5qp@K>vb` zc&iiGjcrJJ<>h{7);2Kfh@31lIcg=)B+2=r=9lzIBhH16gxT-H(pTmWj_xwC&*8Rk zdq8`zm3amNFJU@xlohK+*qt#ZUB5}r)l?G@*u1m+W0PGFWc?OB z-o*K@6GklanGNq|EGx$@cGq9sUTs?fh+ckvFuVMOovnU`XZi7^u(QhsbhMwyy^4U~ zBBCuQiuLSTJoAB9aoOXfFfL8t7v6I(l23dm;d=k3axww$J%sqfO6Ci_Ptwx<>K!8L zv}-_bwhPNi`ZmMco{icHAZ~Ga=V9Xc-1FkeCq@!Tqb@?Z&Q>=t<8wisao3p&5upbq z(~;RM@b*5**>h!a#=qhMaW6ZUTFlNSEFysbwvY7T22)kmCqP_QR=yhFTC}PiM~7r+ zsUH?79HYryktxx;CLw>j?2CozgaU0=ToMvJx~2@AMPJL9%P z)zc26aIIKoeQtmYV=kgRuj2CM6pLz4v8;@~7T6G6--z6B`}ONOaXNzn?MW@GccNlp zhH?FYeMD_%*NtJVF=iBF;eZYoZr8uocH!S9{k1q;+-k(dn(DJN@*{!Z8Zf zg6!=>H$UMMY(EmjAYUeO$G=#yFK`TEh3&8`msyu7-Y!gS$Twqz4v18dd~0p|aitw9 zTrKEq0E|!~T!3^$&~h1~Y#hQ0MX%8qwJPY^Wy||H!u?s_X?CFJy%)vYFnx2)SQxry znLW_HZ)FjQmYbiiMinPV{t+3OM=1O}wsE z+7%Q-iOf+OI|J0&o*6n{bzI+U(F3rG__@#V^^cMG!OmZZuwg!Fn%rKAyd4by-#XuX zeV!Ny^GUL5ffCbCYnfYT5HO&SJq%c@fj5*O@3SX0i)W+9r5w+4+ZTE7o6YlWf!E9P zD;s(rV6wKJ$HGgf1C!Tiw9aRuqM5E4hxo35_1^Y3lY_GBD;cbl zgI&Q#W=XXDMRi0$@~C{#Lws|w1tseM%Gs_i2(yC33VUMFvMIs{zqpyD~!HJUPo zz{M~fBl%5>_lI0-V)ot`xz$k>OT_N^sqB1csautX-?i96C zj2&wvWIu011<2)m%cy`ybHH-5UX3zNpyDFr_CT6OmjAj_4~Z5rTS>0gbU~ZyO=Qg$ zmCrG`tRO45KUVl)c6N}fKT&1-0Dr74z`vRwJ&_xfigR>Iip%=LA$n~de5O|QOT|N$ zLRLxO!O=NIQOSw@CwOzz4+mwP-6sLHWgpr(RWM4A?6ru64J1Df729R-)%tDzOx7mQ z+GLxZTsbf?Hu7WfD8*K)!iVgcts2HGIqM;sR!ax!m&s+b9W9b4{v8W2E(spOkH{Xu z%;E+-#XyoN4lWNm#5o}G1v&2jVX`|u#(yT=1PP%v&x+$Kd(cC=;N->ydYT!$FT;KZ z%{8L7#u|A!hnBL;kcB~2KSIF6|FR=S+1Xv$qMyk0Mr=KNe|JOS8e^gQGX-yVdJggy z7OETq0{)i^+O@f1BK_6e11jZtOITA<61DxW5OKgf{{GIFkAy#{&+INoF=iW35S@DE zO1M<^`?1^G1Z7~RmNOzsueip$_@l&LW0i4HtQg5O!XDI(EY%e&CbG0by{FO-UgY^kCG@)Z(fvyPqRP*7XMe(hN(J{+K50)kjAsXO^@G z=$Sqx zlOitbjDQyZVTptDkYbCX{32peR`$=bvJ5?o(4c_TU1cTJ_~=lF8y6QwY$c}R0)NK$ zVjnn{=Op#!l*}bigrQ0Nk%gg|nEW0eAK#PoJ|vOB*u;~;R03lCIV3=j`$2Zl*Lrgd z;oztUc^RU$;Sup^O^4lpctcp^eO2CmU~Vhdiqn2CZyJSm~2(Q@MIXX(;rM|ryc zshfmfdHSzk@mvakj8~bD=C|`jU~raG#%Jbb{G#;p$Fv&Te_BD}e?JTO2x?C-l19q# z(jhIh2<$Xkms&*si4;RH9xuPMJs|OnWXm5(T1XyY#)N}$);+XgjYR+V zVE@wwPrrPi97(<+mfcti9FWfhQ(BpNY2sFqt2olr<&!uff$nvn374yLvcHR}5_?2F zhxSMbjl!eI4s0b0)18qR%-MK5yn`6EVV#N!j{a4r=`Jbnb4+# zL*FjUzCg^s=v9%FTI^|G~dL23ZcNVJ9UzDeHD1q z;hjTqkXc<}xbo1k*@+=~2~H;<*SrNTtF=}o`VkW^c!g7*btch?^XQ$XKX0}aJ32+m zO??%u$W8K#B|Lk&j9PSegr?H%!Z?1oYBA1uPAxBcyq&}EcnW&fOtUa{xjGPxx<)ocq~EjYZJv^XeaIuSEXU+P zpCT6OAY4!m6tUkUBOm}P6}VHTw>a5-$Xp-L5@^oUGLdY4yv{U{SA-Nw&S}dDug-vY zKFN4;+bK5T)L={iA}CV95>r)}4A(8%x>J5|VA7)zMShSwCz2Z3H#F~`0Wov`Nf_GFTAyQyZ5?4B-#5R`S#wa4MMLt$rV#;Id&k6W*B|=}u zpfiJo;w4twYa{5{eOy?v$7@`3g#3l09fN0Wa2}X+Q1H!SVST)uLq|r0BG0*tCUG9~ zO2ueaDN&epR1V9gv@{Oc^9U^(k-SuD9UcbvG=+8SG=+C@Nr2p7h6wPxp+v9 z|3V&!#O~f>B`Bx6F(Da5nsRPHz{RYstM<+Pz*B5lQJ9zGyT!%w1?d-aT7`kLJPI2| z98jHkaAY9p3+lf&H)}rd9S{<(7)}i5U_xPr~qT5V*7ClL^Zi_hu>$*fKw3{lgggxhc(Ayqsys)D!@aT{&jlU_X$Q_Dh zFUgLgK6>k0ZW_S>xlnLaNk#_Y>jlzVG$<_`GLs(Y%9#5hS?}&qAj==Zl*rAD;zxW7 zv$_lKxWMZHIsN_ox!rXvKv1}$Go8=lHa7hgrv&ysz<+)f>H38Tf1}36^E=nCF+n05 z_TkHU`X*g=MNHKy3ljk*8|3(ES7QVSEuMqpBNInkhz|oQj8>1Z%m^$)4W~xum%MHWfsKUThf)fZ-lNC`6Dj2<8Dp_!`t`}?s835`il z7Hfk*j?0wofvArZQlKN~NW2&C)?aTM|Ia>Rk||_|FsiRrYi|hNOF_-KC0QE>u4-_(_bDWnHwSu^N7=PG1pDhihXOmE%~d!8 z@I8#N%G;4Y@SQBJ513$<8@U_76cR3qQAS%I<_W0&s1v=W=4ndmdLR0!p!TQ+^5%T0 zUvJI4t~>&*FtEvd7njDu0I|a}JOhQQbjRtk8tulULT@R4F&=-d#}-{-jx`7aYc$JX zjW(3@>VX$6yPzQO-21LI zJTj7$LJB)@G*ckclRxUed;ES-k_D_q#zoQn!G4~Nj#w*tFU>VCOk_Zog_BiSb_B&; zoFSJTO<}Gr8INL&G9l9K5_Exw%DEn4$2nKWZGGweUV7YOt&4z_2uZ}uUaf8yt$mQs z_yh;O9wS(6aHN;Vm-!Vb1N(T7yIWhC4yO-zNaedDrPkY}khNU1K!d%lFrd~{7sL|> z$|bvfIzA_eAWE#rsc>PKAh01HNZnm(tiAl1py@KUPgR!OeYuIc$Dq+adc*YU?g0K# z%Q3?$+qKU;R#A3l8V9ufQ>#Y@aR=%qm+zc4+WqAh0zMb(ZBryQd7(GeuP#3e}CqaB+X8 zlEe9pYy{Z8%o&$5S_Yf}uJMLs-62Pq%X#%1xk^lT(tuu9H|o!r%C;fj5A3md|0S3p!9Cm98mzJ1TS5`~)&$;33=s{q%85 zKbfxI;WOZ$QXI(?rD{vRO(iY1J&^T4GfkZ}Z6g*)0sa0v>&W`x??JZ|z;qU5RVBMw z*yxXq!V0g2C3o~NT>R+&Xm|}H9IOM%M)n(2E|3Ab@yJ0B-}6OJnR-3!AEVg&{tF9O z>GGLa3*zjUoXHWul>#B#9R z|BeNcKmc})IBA55m-lbT`nz95L;(Q-=vY`R@*o7=ix(ad5i@2CN@ykY-%bZdb!gHL zTwE~r_4P@`vkF6BN`h`mp{An~{a2alY9^=;AAn!}cj&y?za}Kj2E?9y`vxW^4${BD z*W?KjD55C)lHBP9b#?KeZ1#S{8=ij|1^@JC%M+J?U>BrxK&O<<&CNM=wm1vw>l60& zjK_nBp|SDsKnLDZUthgbQd7Bf{$qOl{awyXP$Do;?!erNib!H;PMsVoQg{&wNl7jp zP`0q5va+4oVntADXa!~Ez6a!gEu1Y+kVO;|gF-_?1MT-t2s?0gc9s#8cJLwA7~CI{ zTuC_|`O3Q&{?{+YSlK^zg>bNJEmbHL4UKSHMN?C9WF+Ev5-XbS<>UG$UMVQ^!e;{r z`0&qyU*rKWrwp$I>#c9^Y+~pOT59~2kT1jlsBuJi#=55QoSWa#-lMB3DWXS!jz$W{ zUwr$5WlZ(y6AA8-kuL3CdH{Mtyg~}h!TA~GACee(3ymCr+N%NDZoiwECn-TiN4Kkb z$bCsGYjX@Gk=(e)Y|d=9HlJ7joGQ=+tg$5W>Tw%~=|%@i1zHYxpLOnQH)I~{e@Dec zw|lU8p3t0q!uENZp)@1df|#mv#n5>hc2_@tM0#GGw_a0&k*s_sZPbw;e!J^@;~Loh zQ#gXwax$pjRgIZ3ngnKRT0^sQnyQ%}gqiz9@z+@fc>wl3_^y}U89x`txP+1m>bwaT zG+iJ$bbLbNd>+uXJQ-uDCSFav#qJKMA8Y=e&A-O^-Vwk8Cd2zA`@A>~4|XozZ99Rp zCXYI`?iIK_FmchcO{X-PG9q1T%p5W%)%++%pE*5t_j(D?p8K)i;q|GxJ?eV0-~ZQo zOGfB8AN)M6k&FoSOe?CW@?d znnO9?cqwJ}nqJ2>V0a>#RrrG-<0OWV(u`&lqiTNrP_xe~O6>@@sAvs0n-hi4Pm#tk;h+!4=~#^pI zk_h?U|H=ct3lI%xQ_DEpep;;BeFxE&j*6-2jDZjk&=yCm+*>DV(cEDO(%woazhskPrj%3Q%2obuj8`AkS!xk2j~ zmmS@yRa;`jJ+&0mNKZ8wd|Cs!E3QJK&ZsK~OHN#K^?5Z*J7KX1U z+%30OTi;sQudZHadX+9p$>Hx@rV@5B?9LhGTY{BZf*r1MMd0C`Q7+-^MmN5GmIs6~ z?u^9-e|rUk5=SK_8AghQ3oC+GM3*TytlC9BHF*U1RG!=!!uBhE+ z98zwkC67bU-R!6+vbfPue#|;2Yvm@@Msca#HbhdTFUdD{WND4MwY_H3$v5v(YiV8j z+35wxKwwJ#7a#ODJru&>u8;QiV)GW-vFTsGD3QpJ@pqpi`5#1H9UVx{7fgNd&dG)% zY;CW z@T$qxc;cJN?ey(mmDm0|k&-XzejqWCc;n)IQ;amlVl z5hmWx`zNU_;d4pagK3A7&1Rdhn3iopt%OVcx_@GTtvK2cdolp{PRJIoo!?(8x3=$Ag``0y&&n+=hJ+SY{Nh@ygz7ME0_quurjh;?_xV5-8i-M zh{fy9D2YGt-bh&rX2TF1vVO*tO4=?KX#zO!J&eV!z)|d&DJ zANgD>1x7A<2y`vt;4&y3L?ix2bHoSiO}2d6mMwE!I5@Y;oG$tXfji#o!u8?&%;rE5 zt2_IK)+A437}au(k+Zevy4?qN@Ub;n?KLOx5X<&5M+`Pl=F->= zN?c|pPP4bg>9|OMt9u7>gqp~Fb8-WkpvPm;!yPh7+vlVFrSqerGOfB3ijl~`&+5*w zP)T1YmIbd@F`8Q^x!ERv$%6bvk9GDXW|pRF2M)B+)ZT&~tDUf;WJN{6^J*5yh(7_u zeI5=^Z3NNk6TH=n+BnF1)nmnS7{A0K+VuDyL@jd-*GJoA%*RogCd9o1!8Bh3^l7@| zYJqZSYemH@f8yoP;Y?uM_{cE!9vhF4B5CCE7PX|1ek-K}_!$&$0uY5XiHf{xh)wq7 zm&Ko3W$)YFcaxj97rYsGeqZgdE7s`{MExlspzJ|HL@KY`Zw zo9+Gm%``IP{h{5DTmC*V$l`8S04eZZ`3z6=I5HS{esSbJvNjrkKF9>Li=Ff_=RtaD zOVO}y^4Bn?Gs$aB&AvtKmq1&i&ECHu%TkPlK91DyoBJmE#eADaSQslLJ zo7gBgZgPIH5*(deY=6E+BHXrRppE)?b=u0MSr550#|aIr#SQVlpHSBishM3k(FuVlPt20Is>epCBUdWaUy7CKM2?2I1D>VTu2u z+TIWJ08~Um-ofXGYq!cJj#tRHSAn&@%d=$Fo!ap5U4U;@R&VO(;*TwOc{B~1bkM0lFDZ!t)4?4d3 zJAbB*k)>rg=zPDqg$0nxthX}Pe7^pbd=#Dme)G!z)5-hBVIHayde zjJ3z#Zq7$=#<7?U4g=z`i2tu;0uos6M;~#5VbK2E!XUrI|L4ZzmZwDF3>m=BRV$Xv zUvKaM^$zVVUe17NX=>X->0p(ve zwY1Eus*1iQkNEdEQzuBsD=Ui{8Igmwg&Ed+FgcKW9*mDoG_}{6j-{T)t%N43#$m$>juRWb^V+hRcm&x_b5iX z7N#T=SQ82>5jy=|5Oo*(=-fz`OuRdYGJzgMp~YvvY6prtw3RYSo2gL`4I=k6^PZ~; zxgoFtlh-1(`(3es_PC`IICuv+E>r`p)Krz(4V+n(tzj{||Jn#^S|xRLH&MyNhb7dS zo~u|7dsay-^)2z;eypI}K}s#YTcPfIbUH$cEkTL%weaD6ZK>U{fU>qnZtIRima=wJ z94y+-;QFL8eHD?wjCnN2sL!5XOK|D8IDGaU!s`ik9iQHvgU&0^aCpDgDmmBNmbgUU zSq!tSZ@Ac`^&Mh4PY<w>+X|ySHc0_88M7pp}Sy&zFY&T*`_h_8t_P;l5*^cB% zW;TLFBS{^LC4&JU*VR)lO4U840K5E|`gYM5^=J(IbBp>@^%E2K(?^~`o zqy#j+5SGHV9uU$Q?5%)Kf$EO1L{-_{nXS=gW!W=hX?gQmU`ZD+1|M);GDq?K;w^ll zyaXl!aC9^s|Lge}cz8<{5H866kN(Civ`ch>LaI{A+=! zu)(*B#bYsp0aDD>M;Lav!0tL}o~1K8936#Vfc^T%M8@vPmkbBLZkj~x7iIcX{`4Hz z`1Vx7tzs-Bmgw(*u*%KaV--2o*uax_9vpn#aSQ~;j7h}*ReS!(nPMbpB!iQjX4Esd zEbn`7Tr#q4SobghqjJmM-nU3T1M6>%0{fRAd(hd(b&U)NAi-=t>aJ~Z#rN@@thdC) zXpSCl(NbpKgT39oJtbLMFytI=r9RxACdH&J_Bc#f8Tx1p8a2MLA2V3zL<`d#Q&yYy zkt;b7>2DVhpiAlWq<6mH(N^N=Uc;qwrb_Mir!ttB8w73*18Cb_$sKTOEmy0)<%(dG zwVr;`Y^aK1+;A%H#;tf9wy}TN(vP(C`L@q~PGYV;MxfL3Z3w6h^1uP^bOk;%TI)Ye z$tP+$@#wT$qr8Z7T;E(X*zxiJHGGa`=*?-#6gz_%yk5d+OHNEi;P5zkXMf~7P@+J! zF+eKUHlOmWZ*`{#-L~D(GMsP<54Ab1xD7h*m`1vwAutvlKHWQ{^~^gl zwvWSpCSQ_@kh?oqpgjrUiKj@*lPe4l_FMGQ9;w2zFK=g_e@duA>@zXMXbULnr zyE&Hq1~FKce8FzpCm}+Q4ef?>+$`I5v+l~(A@Y&J>g!_P1$0(gld*8{NSl?uY{m4? zfDayUBS6nxNKLsrc)Un_UE48Une*kop=?s?W?TA2G`yc~MF}yyH<6f3R!4K1ZN`M@ zbhSWjzF{<76qMnV_GR9Q12ay+3WhnQele_PTv9i4U5Ca)J(#>T0Hbyg*zN2Y$Otw5RnZW$W zS5_Ek-{7@GOzv^`=ywKu5OwKCdUMY>{(7G(VN9g*E9zov*fy2hrb%x7pRbp&qQl>-U+v6}9>q}E|Hsemt@pmzr=@Arp4&e34M;_rsv{4X?V*k3D_7d3Y zvJxL;rVo+)+IA~R1`v+DXvy~MU8bfj&T1hQd0TWAbCnj3{oFmF`{+7pQbaM6b_%7?+Ds9 zfeQJK9*3npe5%v-ZE-F^O2e64=e`?Zw1kdyBh7y^(ct9l07SM~EO(|x7WwsI=?IIc zr7Yx09;-QQf)Dq|b0ak}&SvI%-{KELXeX2yPsQ6qc|2jCYlX6c~r}pB+SD7;quZ8z86OR>v&pU7+13wrH2c?!6vzC zH9w8<`H+ygiO!GwPZYJkh+f0RW$`y`7wjvvd{}^ELD|my7FcX3H}#hz+@Po*>`$Aq z469;&hw`}lv;|b4Q%2h&X)>pK8zn}Xmwg%lE51=VJqWBaHL=?pb{YJ0QXn`G1k&t# z+!c7D;(#^2Y%{3N0-nAZEAZaOlE()tbG`Err|%P3fEwxBqu=<&#Qp@+!Ls>EW@d*( zOekCUUM4b=DYG@Y?tB4|qDu2~@@i=yB`alZ>3bUWhIYqSoB7SMnKh z^G8h|Tr!6Vrc(ffrTd09;j(l5Ca1X1GC(FGzQ%se9Frlrr>@1n9BH+$M?;nof3C`Q z7vFCj!ji`}(Sd?aeT;Ry#$-asy&rtIBn2o?7z@#QE63}BF{1zhPl-XR-jE#V$FF_= z@M6h1DMK?em_r`N*=&PEcWPCaW=^8a5w$yl$zn6^C{bJJdh(iCd#%KfYnKum2n4xM zeOq?Wqxh=>Ivfc=X?09CpBGtG_ZA<1w)3`Zvwr-}hcB=y`qGwA)}~7`W3j0uo+vdt zl#z5h+1K1xPZaPx(}Jogqc>WJq_nqz=`*f32)5d(HeImVR134^ zOKc>*_U{!_(3jPA0|YLNM#Jb7!k~=?KxGEP|A8hwZ$#xkv!~|g<@tx8&MquaN)`Do zsg*VzUF}L$&=e5Ef7uF->0rOLcJdSNyn-l6d*5R}ha(~} zhyJPWpfBxW#nP^)*I$`5z6AeE>b_HTVdx6UGpM70Fm;^e&?;G@r3oFZqF6Dr{pP;ZaQY%;}@H z3|xV@mP2Ae&oWVfFuTkkIn0*o(U4fUC`^E!FDFC)RloC3wI zEmolvD=x)?1ZkmI(GUnO!JU)J`~JT7{mvif-gWL-_pZBES|(Y~Gka#wo^7*d@9iS- z?m)dfyoR=SsfH^GG5Ak4MMk9A0w@NQej;;8{^cKsHFEym-EwDk8&d^zWR=j1_=Zgb z{H0(l%!XH`S)6R?OftH?7&TI(qa#m#`$Op7{2w-)yKhElgKpUx0#E5~WJV-wdUIQe zmE|y*Z+GgJ((kl>_W7~d;25ay`nmPSpJvj3Zd}jE@xxL%8Yv;^-AB_*VUf2OUOYSw zN%LI^E^Id^q#hrI=e3V05!{8{C4NY7_n#jPKEi+f^1tUX*xXplndWTJ!O+itL5}05 z^NtRb-zS1l# zp6<={-d+q1sY@}{7+MPhot~BI%~Q>_9M-G|;o?;u!N}i<3&qf5hSOux-C@1acj6&R z)mBzjWn$r{EyMjY)6+i5KH?j9szy&9cr~jjfi3o^k>DBBvtb;4+54(dR${Jqxs zLKy1s!{4=AtM=_3&C2+d=si`T9^^R28j->EG)QYkR$uV zKgcfNA?T(Qn1G3c-kUQ}$ju=8 zJ)KW5#LX!X(iFPdE>W>O|H=B`YaU%%@JRcUgFe@HbM+C6ExBit-<^Y2L$4n=j@*?w z0M9hsFNm(!8}?OCFL;9sly>Z2S)B%!^IUPYhq#cI%;2Tm?A|1oue!kN${aCE z23L1Ok>115gM(9F*CnuKJ^%9C#SFR|@_R$R2dZ4crtb58;qnndnpn<+X(*3Nd&>Le6-4s{Z9(%S8=dI! zBL#{j!4j+L32@DkDUQZt$Dku@a#`_<gDg)A8C;Q>XQumN8Pb;m z9Y0vnKd4X5blhtX%BTrFHVhFKiY?paMTjQ1KG=OCYylF)k!?=+;GE3AxggPaIRK@Y zJrW8vs7i`5Ys-tu49t{97m6sAe62tHT&Mv3(BTd7`y%~dxpykkW zk+!A=f?up#=GIorNJ8!RnkAUatPh&hJ+5aSBKFRjZa3#HNmzJ9fT*)}GCc6(J`QGa zjslV0TfHTyI^a?KoqxDv=fWGOe)3Z!k*qV7DDsrJeW}h_wIPJP?{HAft&u~`Lr`ji ziKxpchKWZ48;WfydZB#4ywo?i#F^dYbZK#?VCsd|+oEU|^E~(+zSnGEDUr`Cf(doQMf%6R%S* z0>M@y>$K#-;x5C3{oSw)#`{TDTOw)2K+(4Okn%hy6%TCs)dRJWPu)QAzOIvbjX-VT zcLfIK32ha&qdFgAJsbOviruXV6kA)n6L@WnHALVtXC_1gmal12EoZ0HiDALKB7v-h z*Jas}tskX=CwKkr;av%ZUbjE>Co;@C_tFq6NqwrGwsM!?4tmW$AIJ1LkJmVWA$O7s zej)3C&=CzT@X|;nR}F_U3t6*C2!yp3RykLhJ5Vt)&70-tNm1b&H8rG@ACpuC7hH@e^=na-A)uYI?; z)zPR0m#lMS)jG`uFHY+3pbt+;4WFILK1oS`K4&Xu9htLAyOhlh#+tIsZaSpQ(6wFU z`$D;Gc~pd7*=YYFZ91=iuz5egmh}iZNDQ6!_XEbKDeU2$@Cjr*^wtwN`iPcb_?$c0 zkaP9jn;(aPx?hwhLx_@2&wtVh-q=4g{6jiz*aR0tT{GijV$R&r0tRL1b#H(;2I zkIaRb=C)l&buYgKf{|<_TR9w>WN+X%<5TT4t@0z-YRZ^nP-fQ>p+4L}OAG58o-?pX zTlYKDD){mBQP8;`nVFXi@t74~rQL_Xu=Ek9QY&WVb(X@_mOFb%k5Y07dkR)}O86aE zK|?^z@;P&-;xul@0$9%vvP%n`-`Wjq&fV=E`OSgeHEgD^Sg1KB?mbqd6|r(Zba6Vf z-u*~Ml(+8Ojcf144lVem3jLUSXSKtE_offaM+xAgL@4P{F9_>%xjvd#YxByEZ#7}4 z;N`VC8ROl1Bb?M<@)&^kFsOwJE#?&bcBVOY@4Mrf9?W5<6_s~JTQb@;=9_sW9AL!y z4F?QN;JwF`ulK-n0X8qjoOssl#&(Z3ms+}I$Z+Qpy)o;2YxiG$I z;Q_uZRDr!|(9~KNaf0~wg<6N?4eWV3=R+NyS{jh4 z3uJS6Amw_|Yq- z=-5fy{Uv?Zk}F3ul&bxZAQB&Fq^35=o#*qg>6wFnl|@3_D7)SbLaE$tjiH+(PSr|V zfd%U-clwF!-6Rq+<6tz;8pWw<7>fg5Enxzu_3%r^Xd@rOJU0%y!pxqw%*mS16;V~+ z1%}@a7S~jeh3b)x9n4byXfFIN_{q{fHvWi{&gS4cgv<1k+lG_}B*0z<_{L2xCUFLz zIL_daW>(MWduvKj?=+F&p$g~E-5L!q9m#D*Ti`5pTR(a$vmc!eIuIpqS?%(v<$aFK zE^()xuFvQG46ZCousS}JIxhdf5;7W8(O`~#b(HIGZ$x8K{D@5j7l$7}BKt^i+tyXmVwFP}Tl}l3+^^J^(>-6wE6VtM8!9$zUo(@P?x2Eaf z8!V1PnjgcZC_8}h9+#m&ScNpzbZt$F+mhdKR;s*eS4dOq?MVCZR254NusL6h zjbX!_%P*Tnte?U41-V3YY+Wwba9l6E+H)W^F5BD;IuhcK|3UBL(wE}gQ^d3aGs$4_ zo9)MK$PdEBLt@FA#{9U+z5QR~nhivpRxi8dR5H~%c4!#!r!NC%mQ*4g;W5~|D9SV>B&4l>)9cht$u&O6GoUecR#p9RHt!ocMCM6U*tQ{u zol3OsINvpQvh^ycKdQmRo-0c>{(Z{l%eDDc`sSPx;y8}{<`=OO4k7+C^54lq6I{)V zJEqKet=#=zs&Rd>fxo=RM5b8_SfYvN8z&xGF17=p!9E_{g&2&BMA`!fAIF!_PxP2U zy&x|#F(mb`a+>K3dXhv8{LWQj{OsODUZbH>|y% zo)Y=Is6NuOHLq@xn?IX(SM3XJp`zM>=jG+`#aa>+@uR&t?u{K=;AHEg;pOS)`2+jf z`#2E6iuNdsotAj%FzjoX*)SU?chd4I9lVZ4T`eo4zZ*&1mnaPMUKOxXxgS=cx1=({ z*tKQZ_w-Bc(j7VJh;BC0C&E=R7Bi-)wuQaRBy3kS&c&#WX_2ZBI)-T&f4`slHU0Y#&=AR z2W@x^D1zP!>$UydL z`d)W($Z&-P+tXDuXk(+Z)MuIW51-T*lk0h{=u2PRQIf^QO$6Nil5N;&L)V;=@MFQV zmyB*TWS>)Q%cO&M*GzN4E;28B71<93Rj-2=Y?_N=QjOovO@DDS2Wfw5fPxyssP2esbWfNH!_lP`AK z=xl?NHa#T$7k8Sc-K`VyC<+m~JPDvN9jIZ=Hi4osE3t~e+H4It!-B5!BHYF>Z?Kvx zsfH^)%`l^C}jHbnu7v`1~8F}~5mdBy#nyrRo(q`uG>Vb5Kb`?(_P~t^3 z+ILw<1To0yq;`5M(5j(mc^v=zYy&~&6x}hDHkw^X2;QrZSjOE?+;;Z5kPmt<7WDxq z?UyM6!$oB&HvL4N+}pKoxhfQ^q5*PP@%ypq`%Q8Up4AOxP4E5|Se!obFez3;qwIZk zTm%OGS3!XR4y?nf&|cLkD;@TVte%Dzr)r#m*cuA4S$?6Nx32>cD|rW1xmn_T%DxAt z6BQ=%7Y`z-zfhzNblY=Gp<%2^2|b9c7gP+?p`vNaR(ls?J6WI`hn>1Mb-^nvtHKIHhnSDu;fls*0ZEqiG0t;6daL|7$tt6fYOR@Pwt5+V7NXciq@VKR$A9#*qJd%i7lJ1aGBvfpO#3H(Rx-kzyEeI>!lAqcDyd`4 zN9VaQ!D|nXONZg-HLQz#&Lk_O+SQl6v)4gXp&#E~VyWe34Rw5OM+u~DkdPCBEK=p# zDgc{6!NpKmeCM{0lUWzq%p^g+NM>k@KgasIp%5B_nJidDO4I0}&>1djy`M-R64lTdr5N z>`C$%(N)hQTkhYldEMs1>-thxBENd=_6C*6e%X0qH>sZc(b1%UMN95V^WUyVF>%Z>-} zrdMi4*7Gr!RrcI<){ZpHTI6J0B$ELlnx+NI_*g1mS~W-hv&TXV^m1x2U7mt`xrY;q zct_}u2R|_WkbR)3kMUaVdT<1m&s*90K7k^htg8OXVxvj!l4U-cvME83g9n_MSueN| z|DxO7qSGR6>bulH+0!)ISS!nM5>~%F0PXU&Az}F*uBXixOv zt4(_0KzvXCqbOraw$LW*|BNVsF?)Xr!QfOSPhNwk`A8?G9-T2(~__=ow7o zO0vqans-pcF;kT-&E3UJrkhGE+*c8C`wx}DiZ}@{QH8!yQW~=tbz2E`6!7ldyVvjD z#U{KvibR-2blJ-Pa`M88x`)@zkpUU^3|dl9+%?rL<{K(&xMHu^K&YK$vg|f2u16<3 z)Q6)Z#dKprIw|y6?hQa}{}N4Vy)9a>bkJ6c6=qK7NY|Em6nLSmvM<1AM}!%%vS~X6 z9}&Dx13l5wmQy{UjpjCm(@|LS2@L>0VNx%zH0f!NO4!>;%6+K@;-GHQm=j^+BQLM; z3B9LLH{g89Ql4i*&hwqNklI@+7>=kR-Huj@c*K>2g>ePI1@LWbm-$@j6!+#06;H=! ziHWH6w$Vtnk1IWW6LEKjpt<0T*yW;xD^DUbbq~|I`FJe*X`GD$ssy+Djq5IK7CLOR zk*5GOIHaBLljofLDgZ3DwvGvd5y}xJiC4oP(l(aND$PS2<4O|SvXxMGLKvn4IhCotHh2w{# zCJBjG5QxX?gCjBWe)=@~=x2~)k)|tD_K3XB@u>da6GRVta0^wjqZbbIs`8|y{;>!2 zb@sKmLq>R~h$(Uit#;efLhu%6!on___2<#rvBZ8Y!aHqRk@dC*d^Q~{Y98Y?=RN4r zLwB)DhrY~D4#2w5{s@astRi{vbaOMVthvPO@=G-qwlH6k=LrC<(L*65cc_@!Zc_cNgiYHEPS#zrpV zh8M;A?=Ig!AFy}cbsc!>{z`W zcr;9Hfan=ENX3*zc4-soM+)fc%7dj}t>6t`*62;yqPsm0mnkVN&8wo`huWs-{ohc989oW{ zRaW{*ckxwn;dJqdu~IMh?ckP6Z!MSm(^?kM{H6#7TP43x?9s{E?1d{k!9VhU&j*lH z%@wcAEXQp5@KxbCH-_E%o@F+_T8{1mRe2hwlt-gucWOT;675my76_)nAwj`Tp+3ai zAu%e2ke7E%2^YT(Q_hz3*S0rcv@>OoHjYm@6xy7;pbBNq2!DG!>8>Sv;S12YmN_2A7rDV96u579n4o&GLoT+1uJ*ov3iMSG~xcXHTeZo3dh*Kc&)!^ z7`v!OSz1aM%J+~K`_zV-91HhJy)zZYGq5|GwU!C3;)!f&fc6^f#^!`>wtW?(`MUr2f2*s z9SeSykufL$xOM_qN>LQEu@Kr!@6A|olxM{i*<)O%MyXNFLJ6?cJ`OHM3JC~A12^DZ zj)O`w9TJq1iedv4g>vrpGF`YwLimXu34-adD_FH#Vqf?^#f^M;^`1WEfhLnAo>$rn z9SuVb>g%L{jq|i@-HfC({d>_((+f{V>?|n+!J)RtzdFH+yT8(a<5;x=$_4O+gn z7sREx?N+KK>`bi}Du^36vv+_!rmpx93+_Qwg?vZ^nVC^n{R-|PRdTVQ)v9vg0s*8^ zg|XxwZk2-fqsI6ppRKP^mA1E%wkJ>CMF+3rL@E#U_X3?ts>H)3zYS?wWjyrC1LrM^ z5fyPk6qndPR(x?XMpb{zR@34Y{{{G`XCyC@WlgRqH9pT4l& zr}WjPE$Nx=Rywy6n*|TSzR-}OJxW`wvS>2S3c4GmD6ZQaUkc;Himd|B&8s`>5PS1} zDiME~&9D#MfDb3*<>a`bWwG&R(T|bVTrmB-I%u!Xscz2vw`rNG! z>n8R`Yd#n@Hf0cY0%~_`<_46eN4HQJYYMhMgOPPx=Ht;_S+w{m#~;go1iWZt^8UWH z3!zx;NXt_$p9$;0NKE_w21|D&umA9UtThmgdTXDEiZrncn;1c=LGA}$TchbFq)8^l za~D8E%N2~vYVaHF)qj#l*yBFjz#JDBr>tJ3u|0I;kJJ2DvyHw5^P(PM5 zkq_;XH-H#C7B#mlKd9lPjlI6UE(W z2M%pWJ4X-@XAUyI52X7jO*r3&eQ!<5>+Z;Rhkd!k?9Esasx_^%PSkT4DgS=CRl@>Jk@E+~B-!J`_f#)X;vPUMz3@5q#l$9T{21DwYDD z_?HB-ByOZi_ia}Wc#HGW{Keis*MDjHg;^uD({W(`y}$Yor=)DdA^t`i>ZgO7)3a3x zg~mPL8FsLb^BZ`N2xQdO!^V*8GjB)R(8%H^(U^h!&ROmFFAvyeCS~P|IF&uL?%Dcc zY%?)R&1IWB&};xEG??);4j)RA?rfy!!Lzx9ku z!81W+vjt40bnd2TPY4CIutaUd+n2_rCT_Sk1tEIZJIPMCBv$dNow&@b-J=U*@w^$6 zSpG&pY}UotBvyX5C`2hMI@)~FFA6sUseikGQVH;h_LMT%iMQL{5|G{X3 zeV*?Qb!yQc39XS&^V>2*{SR}s0J4@^ybuWyDVf1D{kQ!%&W~EDt)MZrp+o9$m=`T6c#htaY)4jSrb{GX%0gLK3IBo z3%VoZP_A7_*st@D@IC$N-}>gyuX)w%_?|q?N$}w&q$+id8^5d2jx1;|(^|v$i#M@2 zBI91muh8*ay+ngE8ssUdKeasi2}r^lI@eftb583kG+a?SHLfXLBDa2f}XDN~a`ef+qteowVUMyy`cXZi6u)ej)|R2zr1aM){{~Bk<)aag*(E)+HSdi{=gN|VOxKA?_Oyt;655^aguQ+I z(ui!AD^`%_3aImTwV+V8FJxYWUq9$vyJ9UJ8@BH1o#Obga@)z`qJGwIZUVF&n=i7JJqgTMx(4DjmXct7UOYc z@0@8YxjJ(n1xVPr_Xt>=TkFwGsbKO^Yj}Xt2{yJZ*%_jXoQCfXta4c(DQvnWx2BxY znYM*zeMda-a7(IBH#pyE*p+@0DTrOKYH+gp{wX*{X>tON+uUDa(EW7vY1*O?|F*~- z6I`jQKCADw0kI84@3KPqSBLL+I@hl`e<&O4V_Vt7zTOL>(cAY!@JQrR^l)Su06tGt zZZwX67rx-g^Agw3OH(!z%h@uJybzJs4>ZNo zcU!Ode&BQFA=;1QtE&L@eOrtIXk@x?9eIHtG%SFe| z!8#Ylx_4glqzXc_gsr6f8z~ZaRXeBQ-4xplEEBH+M`W6=$dqvL>#EwJsx0ORP#SgG z^FW9piTY=;fl6_=nfziG zvMI!ZmF#wo-SbGXh&wu~zX;)K*b3q@65>FYNYTyR@+^Q^kAMeCm8Ur6gmpW1CGVSJ zw)!<#ZF07PUq?ToWTunX5HHn$A!n zU3+9g6>SqZ=u|1%ZilvHR){apM-2cNi@t5p(s>Kz1Av4-*bSjII+hgK!QTtuiTm*H zRny>=EbBN`(Dz)qi(xV+AL-ADN2EQr$K1eQPGB;memJA;Mjez@D68GODOR2B`^}pK zuY8g1#EEq+feB*6^i{O`$qk?UJYOTKiqlMO6by5Udi^D+5jeGL<8>DR#?f@ZLmulz zcd|atX0dK=aRO4ZSR%<2g|>=AcMNrhRoT9UX649Ryj^^{Bjy49@R(|@C*N1P|JsGl zHi%*}$X2Q!efNv{`b!m$JQJzo+?nYKX_f8-%Wve@X&$1BuJFDmSy+uM#Y%Lqg1ZDjAJX;GIidf+`eI;ZKqqV3X9ajTs(K6ag7X#0IxXk})m z&(3O%>hQ`2bj)rI@ItyRu3rnS+;K`Zl%8CTTp=B&fs1J5^*7*skOtG#DI;!v>!VYs zqGFEyn!CG?y~Orr!7IT#9G8aOj6~R$nmpzkwd=)av&$JzwNvsRyvp_^uyx`}SU-Fs z6DV?Q8lTWf^d86Rb-un=jlGGq&Zqnyo6MZ=b*B_&!nvu;l1jHiFMR512Q7Bq;8gY6 zj4s8$uC0!9ujra>v|{^@!2GSu*n?Ju_V95np>!ji2RJ{ss}EQ~Qwby5CPmVQVR~w9 zo?=tG*EnPy+bMi*o5;LM+ z?z&UECH2kmFjQvFNg2iJO{e=G5Kn-#D}D2shTo=#Uh1a#>0|Cb;57kwh9}Pt^PnrW zY?u5ET*m2-kL!)lU;2K3sGl3^qR69 zoE$nhxyeYmrO59dYzMPC))$Wgu)tl+ z%*=nOjlUORH}t=FWtmzgN8b-4Ql)r#TN~2r2qWIai7zWk7=D{k_&2fs)L8vo?@e7D zqFEaU+#Acx>jj_!yuJ?NXiD`|AZ|1MTNm6Fkn9W`n)Hof3prsb_~j1$FQo5B_qP+~ zZ}IkHy1vnD4FnJUe?7j08dMdl3>_zCz3s5^z2vu&$K z`_qcq<{nF|7f?v#7YqyJ7kII*m$9X>;h?TcM6wdhm{SA%KB|{I(=8coGc$Rt?`t+J zepB`Stu{qVnNw!hN!#ChwG*A{Z!G=Q5SjY-GKW`o{97?0h{P57yO+NjPL+RoI{)hH zu0=%RKWP*7ES1xLA(G$nV#@r#Dg6KQp|#}X>CJnM%x$?O|F?c!0eGLVaoCz-C3l(& z>HNlSHRgXbFgq1kMeLtOXLeWLxyxiBcbf`Y%?IiIsQ31}aRN+XDw2^_t%_%HQds> zKG5fF!75IIUIe<%C`#S(Zr)k!(sXAJKV_u6fX2wb?*tE-1p@fkosKBxS& z*FQwX1ao^tTb|?1pLfu~c7_QeM z;1i11JE~?(`5bp16nrq6e?(rWFEg!%8b?YRB*OjTElHK85DSdSO{b*)6yeySBq^e{4|S zpEfMGJZAz7kx!rOBv*d$d)Xf9^@5bQV@I?#zJ6O!>ad?zFqKoQeg$*4Jy3L2=4zFi z7idZ(sd&?7zP#kUBnmyPXX`s)ECm^si#g4E zV@L){+B@1alX+1DoC;pLwqFP4vFpXM=B)2-(o+N63|~Ya(*}qNdni<1u?HU!ezUU_XN7gSM4SC@aAtOjO=hw z$3_ckAXRBc&$I*{1UrZvNGqk;_5|_F=T%r3no`4sIv5KCp(uD(wyw%3ZGtSAqd zJv5a$PVxP=GvdB6$oBy@G<%aq2Tz(0h>OCM#NR1xlTmraqp=W%w)MkP5G=uHK3)|( zBjZzlrk3evzeOri-AEl~9YxFf?kq+i)ty!gE0A(HCZ6k+H^I zkX;_-l4@b-^J#~eo^$7*%GM;iyblw$!-A;;9x;X*(!tSB+cH;3#jd8%E{D3`AX3#(lx$$Keo8hp-O)0D7| zC(HGjuo|Ke@pfSpJ-5k7gq2__=8Czkh78E#qlh{rz9f}FNUR?mde}94jQE&$t%v)z z%Ii?|5(z5k?p>M+N-PY%x$QE2?%n25@%*1$t@DHvOqDv)jr_Df9Xsab;=r2XWW6~Y z(Xx%<0#m<8M!rK`y~%tdBsypp3sors@J=+ezDq~Aky=SjjOu(h(C_gz@o;opjzcHr zd#S(QF!a0!s>7K!fMEBN>gpZJsABZK8?o&)|MFFVe9OtT$!WRgTgL<|L$s_}dcl(B zsTnGelRE}-0F!Bqze&RP>#UGEsJZ8Kp@cs=lNo4NT2ld8(~bL_I?}W;ddKtK(>iab zqOh6=Vf`KUK@$`i!|G}N3lDReB^GBD{>2X4%*2iHGZff~n6{>jEdjz9_e&U~mFS~G-Y$)MS%>CRgnt1>eG00F5BkO(tlwIEu$Rh~3B_b02eiwc~ z1L-;m=9+<6Q2Wh`` z{C6pQDwMQLp7;hAQ$q{$ zt>;n5F!NMaC?asB!R_+y2aFwf{>>J6t%Yw$<+U9BXy2D4>DorCAXIE z;&-q;sVnsjU}lNzClwOu-&>>tww z;5&w%ix=dnSoyg_J_sC7>7S}Bo6H7owr+`SS!82dDbX$M7SMuB+&#jgo@O4Oq6#`0 zzUu2aqXt{uE6rr(peX41lp%ICgt6w>mOQiDnEc#5o8}zdOj71yl`&$rC{&5@)Mck_ zGnl%TTg@hl7(b{WRIK@~$;SstFi)QPn=e=0cSO>AYJ!e&lKFJx{)rRVMb7w-9}cHV z*ptG0!^ziN7e?HsF-Y7gG9Vi3P{hs3h}bi(gIO-Fna-tW=nJC87KUUf%PhOY+`?in zIKpPyAOqEO&qif57^&oPf_4JwtQxxOocYiffD?upi8d5L3+6K-zzA1>xy20GW&)Bw z+RI;(%)b;Gs_O1RC1GlSA=Q9&A+Cdyl!6Lw*g}qron`=V`zYN4J{pSlKL*_vd77?iL zd*KPYR(>uJU;=*-zSUl$MLs|J-=@Ov2JAj~vjh0PgnFFlA1XFVHtnH<{uix;fAK}k z{}&lB|34lYHA)P(slI;l#oOc`0vQHb^`{Mc@O-l$MFKF&DuV>n#f^XHA}#KB@7%6n z!$6&+IZoB7i&6i>>>y|h75@)fd3y&2unSgG)L605Bg!i(EE8V0wG}*||FXX?fC6GI z(BG7vP8!}p-)~Jca?ocj5Gluy+RvP>5`{|3$4tbKqWpx5tbg^O8}7b@6gvwlu&h!R z$Y!QY{Nw#-Vf{Ie-=D}#D|`D`l@d92l1_JOR1JZ)UEt99 z^7kR)LqBz~|1viv9UbC=RaDniT=A&%>IwX)!8^^AqF#PL{~Rt>?&#{uhPr%!c^4JP zS6g3C{OdtGA(g;DNfcVWk$!o{$38@{cQb5Kl$k4?a}BtneE8%LWt*R$H|>P+p%8&0l)HDjE%~{iZ>p+LXpv(k`m?@% zL;t|Nygb^ZgPq;tGj(-!;}4CE*_H9`pAFbJ=})S`898W3`RNHV{;76C|J7A+hE3J* zNw5p^PbUyc3ZNkW7jE4mlE6P}_rK)j5v^4z=^Py;`b7HQ98nFgX(WAqW>U?_%=G8} zIh22VI&t4U`?!A!PfpV);~W1pME3buFUi;nqr#e0j-- zf3*!dTPCNo^?R?Zx{Eh|{5l!wNf*M&9ZNvQyR|NRZFt>9EQ&M>x#^{dL2^O3-H#oU z`O;s=qb?P--!*!}jpZaBVdhsvvLBNv$m zfjW7st2)N{Hr7p@6WeIrOt%X@c7C6JILtdW8*pNg+CoT?p{OAsmTfgd<66o)z_MZ- zm!>JzpK8ByEV}T>lewFvM62yM5lg~{*3vhWm~5oS#@itaDMux`?EqJ-@z`-s;<+~+ zcwiv`Z($hUw+h;%0wHtiJx;&)P`(jyz57`%+Ot<<(Pd`jI?0y0pwO(J-#rB2YtA`C z?0(H5;F7+zF9ZU!^*uFCaINh0K6IMzM^vze`z6R6VR|B+XKMqWKYFpr)IK<^)BAXQ zZqVph)WW@$i~scF!~K4m*{XIh`7(k++LNqiovg0T)9PSgupl(2qe{5axdQvH{q6#^ zfIBxvjO>%H_ZGWZ;o)@P9$6EW|3gPmTYnK8C5RGxd=1Dnio<9y@W0Ig8UEPMdz45i z{GX*RxuA?B*zVFlWs@;ze2_i#+UO>_ozfM7Mt62Eq)EK{a*E#U?0fslYUwr2&C%p; z5%(a@`={8}2R2e&CwpuAd8`%s)3 zE;2@^h0zWhhMc@xyt%vDAIJ9qyaT=_4xh0dRXIsKNK+FnD&BIbrSOdJb@H_FxA0{; z`D8?`0CN*zfz#coiM9SBc&@r+>PK2{xnCH*|2j!!Ya%S$^=!KLKu`@Nox^0TZ8-P@ z!jSs5cj?SJJGT(TJvErv=f^Xi6E#AWKJsIk+J6KxFj$?nT`ZH+y@dM(*zyc_^2DG2 zb5K%G6v?-Oq06-84CTZEMEBZ#-26EwXBDDS$iOI<4k%ftd<*1i##CTbga`n3d0X!} z2pf;Ao@0-FomCVggDa3CvhzkN-z8EN@}E}3lPiB+m{IJj>NwTKwl7uEu#~K?4pk+- zZOSlGqIZA{GJWap*bupnO87?MX#kqN&xtdZC*vOX6;+)yX66s`_!K)n{?yji)=%8B z3K)2@vl8HeV#*D})$ih^#C2d8q1TgVjrFcQReq4b29RrH^7TGB@Dk~ZOzPtZf!|QU zl|o?hfbQFH{>#Lh4pHs7OHEn7NY(e^(R61*p=W(*ckAZ+1Lsd%j(x`6_jf}hMuHEF zUPkNp3ttP|99_$oYXFc(7<9`iI%AQ<(z9N>!9k0|`Q7p&pVRqi&jvF6PW!dWE2LZC z(kJed1sVlIFY;-ZJqv@fqMKzqF_MM&1JC)duRGdTzn4Q1OuNg{y8(hmxkUyo!+F4M z>0En|Nz%6g0o8G4NuRAnfy~20F?p!JLtC#%-QJ{`A-jg97K8dOa(??Zz7;>6L z$Hu9<5YM77FM;Hl_iKGl#=ensA*QL>6#J}|NpWh0qp>P0x@Uu;A22?AHSpAwrMGhs zpgJu8!A`Z?eg)DYE>{=i&wj1l=ed)hFp0Q5`<=v8NLfbbxm%v!Ehm(fRQ*h(=k!qO zEDpRKX+QN{Gt`J$2P!BktS(^$>1EYk7=GYNMc9itBC~T~B&05@tcz3?#e0!!)MM}R zrb>@dupQl5>y@H|Fqqi+Rn3a zy^G6O4f`i68IKMDhhIr>EgkNW*@7P1VEf5TX+&legfvkV=}u;3Wtp<-x~4oPcPF9| zngCEa3?}BWZ9~@9w_nfq3yMgnS1eg_fvr_VgH6=$a=TE^-zSduPaVpAd!Vg0TweK+ zJJ!ZM7HKj%Ry6pD%sn_>xjK7i-X)a*r#-_ez$tayl5_$4h0{)HeKUou^w5-c`tY2Q z-D36O8iiix$3q{s$N6+cL~2CG0_DpercxA}0)OlY9t@Uvo`$hI&q>#cLO3+>1=lCW zCdVGm-X0LQEiL5`Jsu-pF}zd>>uI#lm{4gzzk`}jg$&ApUCP#Lz?s=!DBG(%saJO- zr2Wjq+(Sg6YQqYnsMhObYz`RKiEWj3kk!QAlP-LzEI5N9qGe2N*5X>=EUba;qz z0af{5QDA2)D6(ol3Ow<2)5Ir?SCx=>7E>TT+_xnaHO!pZa1X&2O~L)L@(y86bHJL@ zdnWTRvwi_dX?Y3#+cU2#8so!~6!V_n1FAqh6NY%S+;M%_2TJFMoT~jZ9C(J7zjL{? zq+Qn$;G}upeU?VDn#tZoP~rHxy*r%&)QBV!Gc6UJ;B~X0WNgOHj8B z%3CSt&;`AvQTV9LY?7OUhSFcNXPA|3$*MnqTU6(VzfSoTu_MF6;gO?x1b6-8H-Y7d zMxY?Oj=bGDQ5Q^&+OjZ8Pyt%s*}h3?ZW@>7RTwn0D#t78bj6TBi@Q8!z^IWGx59UF zuI#SK2hI)FryJx^F}Lasj4}P1Q-VdFK|V(qP4Efvjwm(8Hd5ftuQeyi72azdviP`S_I$q9#0r?zQ4R$2coe0Cq){!*YI(B>Wa)B={+%?(C zRZT zhp4o7!oq5%kC_9-6JiAvQ33$bImhKQ6hFsC@qGfkrKJ@3p&u@z=wIiwb@&Bkg4 zhRRrV+AQ$H3Ko*lYF~rDp*z*ER@iFkg#J3_2xnIJa2Hfvf{w#*2M094e}~6dFG>9zx^cF=}8d^}5%aVb=QT)I-u|vZ)om#z-O+ z$FPBc=IQ#S2Jm?KVTao>YosP&Rkx?BMDj)S7aN_YL!neoC+V# z=G%I?V72Sn`dng;6$GYe6CZx*c%yg7%?I>PGj5a2&Zwi0M0(ZMdT$z=nxO>Ny4MX%$K&ds~!Lc>Q;@)|SrBIzfnKy;DUcVjG z66-B__J~J?gDgK+>0mF{G>Hn$Zb7umsw3HZv=S$vGV#;Iih{$9Y>Zm`rI?iD*0$yG z`V+jK5CZuqrYf>8T0>1hT@wx9~@#AnF|gqtHX-l`MczFdSSEq5!o?`1RN81 z*4XsC4;_iYz|1FQv|BLS#r;!H6S3O;HfQMP`dT^z#S9W2~U}ei$#W5a##Fsk$Vq)3*%-E2^1h(HdQUIarLx z%-?xD<$E(^Gpdn8ey|4?&}Ekv#~a>Z#6;fInW!Kpkqpk6*GbGdu~?vqBkzF$(zaMJzd-r=4;t}j6bt0HTz zj)M*Qq6Te6i5j&}#LPjnY_!uj*O*nZ3W*PS{c}k~<5CX2}*C~ehbhT24@+iN( zt$M-z_6?&papY&G6Uz4cNIy*2#$U&ihLBp(;O(6If&Qz$;m1LSxDmwmwT~EIi{ZX! z2zxrFEbP$P9zU5G|CAmIhnc92X z14#JO>BfSTgKbvh3TE#0=15xw+(|67((ibOpen;BC4^G3$?xWa^9Ro_X|{fA2H&qv z_r~i0bW50ri9xx^znIQ~pr&{OWZHIWJOZu#O0DTiE5)85&eJ0Y26L&PW6UHh)yBL~ z(I2E_LVYo-?TiNPf%kN`Z}gaE-GY3QGrmTNw6v1e@W?v0zO~xSr}(8MKMo-g^Gj(m zDv?g%5rX3eT@K>8bmVWGi4MkbepfB9@o#!N_5)GRA`+U)E^xg$yuF_57P5BfFw3cL zKuK!j*r~KR<7%ZIfC|nAJPAyt(=iTV@Dpv{V_#Eks5=c?LVkyf_Y&;PCFlPB5c3)4 z(Eo-YWz-Oo0Ug9QKi4DwUJZ!;y3YBDW9azOFEKxlE@9~H4IDI1wgi@C>WxbW@R*L> zhuL?%HR>HtNL7Uh+yIcVa0Mi);j#yf6hCr^@JGiFFZuj{nUd9$Xnfc%JA*?WhWif`2- zCRcg%Y#D6v{{^(~N!d-A6y^7Xl$zC8We zu{d+m%(mD0A^0u*$zv-Tm88hXuov~i!|uQR-mJgq;FYcXir>xH?a-62=P6`N#NRd= zx_L`s_N~cb01^_}g0ww{P9?v~Mib)| z755gj_G4-vk$wH6o1x$Q?w7CTn1DV3u|DfE+jM+^jeD5#ni5*OwL&^(LSQ@Gd>2o~ z2MEU!dU>mO=sl$XvNIQ2Thrlzo}02tH%1!Fd%0{9_%T*hbH3mIR^pWlX6O&w!J*W& zIHaaK%phUKCSH|c0X&@P&xVtq-Ida+th9|j5d}zo=vD57(O5g*RyC-MTs77((o7;} zE=d=}o+MLA4*U|%B&m-#YGq0D{Ql|{vx@M6TRJ5}cJ!98`4UH7h$_Q^^iOm+!EJNI zRW7{fAC3F{7J>D0w9W5|4N4mFMy^cO&^NbnRX=rR7)|p?bjFgO?kSid5P_lv=H63V zSf8>m$9-Vfbten(SzOkM8cHZ`YMZ=pEHD^zIrfv&##i;WI5-#`zoevT%PF;nZ)t1W zbu%on<8Hs~n9d6#!&g}NLA1k{Tkx%Y0l`{66DXA-%Rt0T)o4KM1&>&YRO+i>zZ7wu*KB*l!U^a)N1hOCukmI z@>{dz8W1PIkT^qaBmCzrflBp`7C|Nj2BVd4-PyqLohv&S;lTVA5%)AxjshM!k|D8& zG$Li8$82d^mr}W&#l;}0ErRh*CdPL*L1ieRV<#WvT5ZsvwmHRwLUaM?PC#Z+L#=Sv zxJvHfno@w#%E0(_NqJv5&k08gWx~6t!wGHF4K0dN*w8u!S(iHU_U}@;`+Ok`y|Y z_n+P(MMfgPa;rtCu42!fb*EvFE=0gPWfM*o46r>AD3kWshrRoVAP91!^KuqbK+&Mg z9$>f2%A3_}K3-|>%%=f?bok=)B}d9WydrADuc+&cT7O!#kCUcq2-n1;U;~8UCqxDrsfzZ*beTiR;clNVUSvjp#NpF(GN33^YfIq4hk#<~eO*zFP$uui% z`RfU2(xEUk5N&Zw_$Nthx_GBn!h6S0rR;*4OJBY_5_qt84F7v8#gz81S)?DK+F}zT zc22&PcmquHoQls6hz`D~4@&A22Zop(Z_6U0B3mL-a(RX^lxWuG4UiO+cX)+@o(F;t zFp~1eT-R0X0^*H}M~xd5cvgm@g3gv#F7|fCFRxw(%pNO2kf_i{B!3JP$Otue2*ZdJ49&8cU@r_KpHo{no>4lAx>7D3 zb7$w!&M z+OB1u0;3@-*M>K5sNSh~;+nN~+8wgPP;*CTvurL#;_Z=2x57A6U4D{%@TEV zN}`1#A_Tb573A32I%dQz+T{HfSU2)%_-KxA5w~oZ(CVDiDoyAbl;v8V07n(!9Kol| z!9n=Du8YKK_Y+6KSGgr`T3lLZNUWjp)ZpL_UtH#luVDcIH13sSSrz7lNd&uV4;k(b zn^ivv2D5W04CzM9ha0_tNc75^9Qq{wyyBPz@rd>Ie?nX6s7f*sYoDjU0e7JzqB4~l z9Kq{u(yN7FMXp^?l%`I#IB!fd5^YYA#Gl|rw=eW{<^r>7U{nX*NKN9r&sapDs`ps) zn}pWx?gdMAdZJ_{#zZtEtsQKqTLxR5y77&ukrZ`=9`LulN#{@rj;Vu@6@i=cZ8zuS zh-?@NTzU7h5{@I>jV?OB+vhnead;h2xvZjU%a^)wq>78mNgzd_qZ8=ry|iRad#xwg0S7^8ha83(y6 z|6Cxt{7x4cB#-3a76vrmr0N@m3?^5FIy)XSrBVwCE6jU67SqtKqm%(FTmBRsS;ST| zY2)tkpg$XA*>5->ybJh!>}(gTEmlXxP~Vb4n#!yJ@TLmgFclU1wX(PpYx$0)w%Sg~ zJLG0|#GU`AQbjL$hrz*Q8UqoTW&`?a3iB^BHMdXn7`I8>R8a*gcpTXiviek05oWhcGTj-F zR@bMTG1JbFD!VjDa{93q=xiMmh@TaF%1{Cz%yZF`Y)%h1W=D|wg!YENDoX%sS*gM1 zitW8v!1xq`?v1uq06~NveJ)v!{cjE%B}KUOKIsGvz0WdEp5!x6nkw{3VdikYdF`L1 zgVtzKBRyEA`8B`|{0`Ic%5U`*z*I+oOSh2epEPYoA?mvQlon%MSn(r{S$g@(0`(~? z5$PR1k!S0xKR13!kCm}i4+j1gm?>geR`*nAzP68CW5u5nQLvM?Nc-_|{qi?@&qM>QKvmTK>#E-xuF2UXwtW@8N?PY{78@Or=g(cCWQgt>dD&4o<)Vn>wRgJXQ z=Z>`624i8JHi!ZWzJiTDV}Ol>*R#cz*O+-3ssaKZO5dE@Pa<%&m89zH$-Db;)}B`lLslf2rPE#6kLaaC*+Qp-S9)1W+r?)OJWXJswIA?l5a2^El?AOiE2 zU~=;I+&yGC_7S&HdRvwd`OP+6XLVv;Ud)1ACJF3!a3D*D^ib4b$&sPipTZd^K}h_m zw8@RYN|mm{$ovNlrITziUwGQWLHk2fLD$VGv8w8@x*@OzS>^OlCF_ej);CnM_gJTnM@)ckcnRSgT0=ypTr43HS^bwf7tJ1X_uUHd@!OvhMr*{Qw%?Jh zqgvhi1&imlU^Vvep~p^TABe_N{2c765R61eA7@L1B$s`;6+i+(!9DIdWInvV5BRJ$ z0zG8Nd_1!LaQD5#iyo5!eq=f-L$aXk5yT3MpBP+OD2vC%da!YOsdH@bc^my#;$ zT@X6>+8#OCfJj8-)kd7gc+w6d_{fle2uTBOOwB4s5Flw#Azj;$*R!BhAxb3i(AEHB z{oYUQ?A>8{C^Kv60A4RE&G$MXh`qC9@AMdlHYQn8JXpLzY|_j^>CMn!Vo z*jNu>0_-787Lg*8{Ywdi1&83k)WIzcD_i;EVe-c@RpWp^Uyc1|8+8aH+~oosOxr(Q zee}Hj@m(waG{}}z%r9u<{V~8$&hiB+-e*CbeAGibu~ES&{;VPa!{XkK#U(-BDQEFZ zzQONPms#SwylR;j(HH=4p%SISxL;N>CZ_=or0QoYQ@4mo^KJUKcS|=KUXwP)k1?$F zF;u_g37K^EcXnwA3qkTd)`ErKZViIm&cD@)R79pkZeC2;jdp_B%?^Gy;eIq(w}$s4 zfRX0P3L*+nS8lP(ddym-UmmU@tmQH42Ego0fBiDM-}bxynL>#-eaFDX==3a5Wacib?IUmycpCp73jSFD@0E`HD?DmCzI(z zs0#^+D0P=tNz$xl(oR>y9E zvQF9Aw}c{Kp`fP8YJS!9@n%Fzmmi<>nZM4xZ1sN4JvR~GwwS7e2EJLOd(~6->fXP5 zeu3>NAB9A%bJ$M|q@3ceYsvPrih=$I3#Eh!13#BYkJ2kycLnT(&usCj`dKm$z28pH z!O7IT3Q0aq_1v<0-e~Sidw&+ZP6M+KAS_qf)zS9^quQhdg^6eU8n+Eq!yf3Y>}Mra z?&PYDoOj7I0FY%zU;Z~r)(dv0iR+Viu>Kda?LjJeCdJZHWq7;Jl)9U0rHCD*d#$3dCSqpe`^RP+7gz$smuSXf1NO+7rQr zo~>u3^cG4o_0t}JP*@Dr+%`nJfACizVcXjNrFiX~h#yP*8av?fJ$6oU>3deZQljv0 zoR|!hVIg`N(m&3=dbRzbEh8$)k!fkgw%p4sxs%f*AeOz-=5yeq%;#e4gHLIsw)6{k;$I`;uNnbv2Eq;oP?TuvuTujLH&fTVm+|Uej$q=<^h$FELF{m-a61Xamd~i6Can|4fTE86FZq;t1t7B7U8$cZ2D-ATYi}ucpN{&5%ky| zcoT|QM@3`mjs9||J+@;#dOmqh-!uZWXkcS- z^@cT99D3Clcd@M^`Qz7!VJ3B&FYeaQ@Ud3NABO-QR=k*mS`?P=l&b2gY_@n-cMosLoXyfN&hQ^+ zzB;ciB&h2`qDtR|2dPXk$i6m&kUZS9G_=mc=#Cj(RjDhz-?C?Nf6eI<%cE)Pnkff~ zzio=U;l!OS?-5Xg_A&|jlCe}L*?*@gdKfPo8yM@>8jSd^Hf<%d`05w>NmVshMWX|G zZkLx1Xpr9{+t^G^yg$n2%SY-8$*2htcdzszmc7i(-E)`X0OQiOx`OFCCLiZBd0mdQ zx35#3rY2?uzqD%$2)=3RR7Y54G7ygW)das2DvTfZ;NZA2J?cX#o1~RTZXMlc_Qjgc z=9}2ca!b0}x;zaD10Gfp*r!WVVgHj)&T0EN^XJvjTwgS|`b%hJR{FRz?bAaL1soGj<4(ik&R0GDVxL@f=%Y{X9 z=1RFk;yD~Ezo$@xcU8X>1J=juV435f4nN!E^&kw#Y;mZ3{Q*SNv|^}~0YasYS84j{ zmOwjtBO&iyZ3zyLx%3}_i(M%ITi!byv38syx6e_BZ+YtD?|<>_gu)T(GGYp}G&xvt zM5!{s_e;SRO4jQ8Uq{lG7)5<@LxC==^lG`fFv=WV|I7 zmfY-x#Ei;&(b3*{V{CTj<1GQPJeKBr{)in7uErcTAHnvfkJ>nZr4e0el`MgnHG3kZ z8Jh0vGaG3CD*@2Jl9ftr4IM?amZ{N*h84M(&{e4u;Bf$c&?SdnQ^uX`_*mh5Z6rgL z9CBhVMizm8QpFPeZyj{<{o%c|Q^EA@ng0pUhGNls3R*h!0;=y2@3qMu77v8Koj*I8Geb1kYuud}kKP;p+CxrrK3+Buez4x&b=sNXa%1GUP$W?Ssl9gH<<0nZ4+MIb2~iSaVxM(3UOcD{^!s=Eja+ zS?aDe4=U1=JsU7F!Xd{X5hXU(g@i>7nVfn+1CEbLksqNt=`GKvI+3gx2GX%z8r?-S z#PC}kw*|!%B1CCgFGBHUx@?~LN2$k1oDpw@TtyDXx57oGEarA9TBMm845)fJY1so; zFn1GEW@y%c#Xmaa;ImTseAsJvKtwnNztn;r>L=;nY#nMj)^y=Ss5>{EWq1x0pj{f( z8+^=LO+tWI_8y6YI*ir4rR_()+^WE+W&R13lq@5g;e!Y;qyhyM8B3=h>O5_YJ_Uh8 zlE9_2x2M>%)+$rMAlCR8^jNKmoNwMQFPhx1`-B*d%(j&Un`G+0lYw6MA(?OS_W*7r zy58up$|{iMcNca+_#p=FrIcv)AQT1lH(6aelZ~w#?5SeO+-w_=&JBIS952((B1Ks` zWG1-SHFVIjC0j-yz@o=xD5}{qV9;hFc`Y3|cp|<~g-ALzfW(`aO`s7%% zh41>o9t8kCl%}g9BnOI})NeVuQ+BzlPjs_RA*lDYaR0|D%1QqY{goR=!3k|PL8pdQ^CU{O! z9#u(h(;ZBvVXKG2JQz0FUp^uVOJ0wma9tH!o!AcivZMd_0hVeVpgQ2`tim6$52Pr& ze?&wG5`anEs7iSQ1&-_z_(-(w13oHpzjCSyY%o$@g_9@7d=)X-jUoz~Z{P=TyDI*r zd%*5|0s`+~Ed3^q`-cYySJ%Vy*XR*hf*;6WAptnCDTRgSw795%z=A_}VGtCUdgw-G-D!b*(g6&}Ua$iCAa}{>#yVX*M1i85tO( z8pb8>>nnsY)1Rq|HKH9BD_)}_-WGlN=hgo*ss8mPJRJ28sOE1Qw}s{MO;x$ga6%+f z|JItgfta`ddBsaJObv(&QU4R(fn5XUEvEa=zyGr%xPxTUqV(dWJs8>b>#X-D(`$`8Q*>+e=GWHZ zKde5QECh6(@gG7W&P_pWrnyO#0_4=)nS|YVkKc?vqcbxl1|ln5j5qpJmi2-lf6hG~ z;!uDu#*sP87g3c$snZ>gs@JGS8EeDM4}Q9_SH3~H+>fWMqkg`6AR?koC`V~ewVlM~ z-6pZ;@&PY^WVa`9?2YkdDT<;Y*ZJ~XUF98QnYht@o^Dp`hC@wkpFm9v>O5=I5-l4? z1mSKMpU{}xKeuXffZ%jK_EFi!`|_={+@j=p-yc0rVX>3A0mYiL3(Y(P8`=eDbEkitNRMZW20y!@o&B)j^(Xo}|Oh*7qJ&5nfaQZ`mphn}j+Z9l4j16fh+6^f4sE1$#)Xf%UfSI)3RUZSoR z2IuXQUvCl2?zQYrt}K0@)nqP^YS~39_<5#vd~O5nr9I%hr>k5-JlbfBMaitQ7*F7L z*r>|$+|jWNN(k3l7xVL)*cG@TR}rzGy_$tO>&7JV)?32{E6XMnT~~mYa}P? z%zVA#oAXI$==9-py7J*6e}^juOuTKKPI&xTlB$>s1`MH2;0J<*aL!uew)o6kf3!Va z&p~F8b&x8+^^n)c93$!3k29jubdXg0!Y_9k70W5J*=yYgZ#X;f(t$&~t>F8Ht~lhB z{gfxSV?x<;WtqD~i_(0mny}WOFZ{I;bGMi1p(&6EKLJo+aDm9}jn%N*cA^0iO(mOQ zNN)oJ(+ktrjzj(7?W>3_zpTm10 zu0zQF$&oLe&a8M0hI*lliav>0Lqa)-!%kj4rMBrP&u5fXg*~Czjlto)Y2YV2Gv|I- zdRoJ{Sxd*)0$&4jDp8ogAYzWU%LLeDPW}Y}W?CKKg;drHC>g!!bc^2zMcUDX1@t0; z8{y7@KaVnMRO)`-8fO-gM=g{y1NnduW@f4LLXQ@m-1&|e#Qq!pv%cTeg# z4%lawhx|EE@`P`(<1(z1g5Zm2BwFax>DuB3(8^%?!5VAs0ql4;;IASFQj3FN5Ygue zYnU0iD8Tz)KUUk^DWYyBo!3XTXB^bTZC*9osxmKv(Z#yu{qmKo?-YaXo4bGRrt`K= zf^t?}_kKhzH`I6%EH|&*g1wDjw&dW`7qhp{_9_Y=V;}U+izF@he{@C!oZVAS8F5rB zXQWbkt-y-Udn|FP@6_asttqM>@n~3O66Ne!M)A za(f|W@xIuHt+RKW{W9`%zrp2-T4c!)j@t?2sIHu`(~^oF_{M#4V&rP|WG^^I{U*uM zMiR#~fH29KdYIHfALe!*k4o|u2qhS%(+WxkN-T4( zOxUk7JwLQ2Ef~1D374PUs)| zj0#3svqtCnndZ7L)Sge4Ye*o#@F=H*#;2oz+nHTem2Amyj_~E%A^a?{^4RM?3Go#Y z*i8X94VJ-Z%v8FevnP48#JBa-PZ_qApE~71hU zicPJUehKb8mfhIGtExL31R2lH+XRi`^bBIvyw3_a(iaX^Lt}IEGNF>}xDV1R;7y;q z7;``^9Oe#k6Y-0T_0+#f*u%y@Zq8R2*ZYh_q?+VD=iUsxy9o}CZKDleOv;gw zz2#*~5)F0OM(bK8Jho!+Mr*|cG-jZuoy&d%`Ds=}?)h{_b|wrnj}43HJ;V1&0yO_jOt zIYFYB3sI;JO3d6lu7@9_d0MGsQkCDo1RnjO^s)wwtT|li$jRtX1;A@FmjnG6>zk2R zzIjteEOH%$MU1`1`ZCRi0om(kaopgr{ho>Uhizwg_u%zOP=ruEByXnGr;Q5(&265C2;`nji$0^;-QowPzC-LW} za97WB{O;SkyD?t&t~|T8c%G|UR*@5uBef2n}%Qx=AKBANjsNOpOOuynTz&C&5DL)8ar z3)(5--%bf*XP@n0nTWG*moPo`*9LrZu;I86#PlJRSX6~~&&ye<#BhlgQr=nT&$hfq zhrv5ngPCsD>^nbnFKT}mBMb#%lQ~WG9fiYFXXW*I3!G z;lWq^Hf8k*v){TMKAzsR)PkX8PSMZXADtb3%3Jux=5+E;TvSn%?W-S_iO^E*=srs) z_MMW$z^v_FAOEhdpWpo0zh#Vk&OZ2q>|oqpc$PMjFXSw@tLqZE@uYg0!>?#VQ4*Qd zR|FTUHZdJxEI2fx6hGJ1o;oYBy0XguZgpIEAP>EQkVV}6CU~*YfzJm2B+K)jQH3Mk z(ic)KW$L%LeCe|ic#JZpo$&!o{K#9O#%gtkfa9V4!d1#=qF_v@Lb}?Me(}4_+SLiHvZq*bbr)$~&_rd{=WZO^FAoC38o<7Lc`m#ftbh9br4-)j&xO^a>{7}d9IkOtV#}rQa z@ca0z$YJOe$0|eeLRFz)(t-`Ftocdz7f$XSIVo*mQjoGVY) z&mO|?=1Hds&-S@F!QuEk8m#WIEFD0UTT!;qKBzY>IOoVR5o=+7d1m8aM&MHT-2zhv z0^-nK8X^80V7eR)8PhO{;uKT+H32|-*Md>r!Szpt)eSZo&gDEA2|aH{#aM#d;~pMn zfLkRZ;#pg=Ho?V?6QMw8`3MV;0R@}qXM4DvZ%{J#ft`XVujOlMSFJRfZIXBd2Tx4h z8G4c}EcD*Dh{5sn;D!a#h3)rh5-0G-7-2m}C7|kIOHpMrY+P&<4T23^8a(}h9bK+( z zms7V_BO_8;8ADkP+i|Mq+lPI!f9mA)dcX|oYn^HG3FutU&%tgkT=isD!q;d3F&i8w zQ(&U{bVuGXM?(9&vd$T4J^$f3eLwXJ4jQ94qdF&nnA^#3Z4*@GuPDr47!4cxG9E`{mNzp93(93KdEbRNa)2_pHB=wOMkm~xh7r?>71FVeR1Z~qL!yAtcF^;FS($W08c#4vEwGE#@ZaaRYl3kQ9W*9=8@GE zQ%)iP%}8c)>Asc|Y%NE72fs&@Va3K3v^ofdLn0s~c-Hd&lFm%LIm@5BI)UiP$`QL` z?L4Z2vFtO!5;Pb@86kuiM(~Af%aVzswxrY8ZJ_GiBbjU0PgG4Es)K&^e`~eB$_>oB zF9Q=Q0KE8E31p79V7(BQGz<~Rlp$L_SNdNJL1H7h7wrM}u*ggh^y zfEy`mrs0dpIlg+gB6=B{>{kU-3hLzRCKRF4!t8N*=E#igdD+d(MewG!Y1fHTbcUj` z5ceN{J}~e8!23?n@UZf2EDYVP`-kVy434(n-5X;+50S{~&-MY4=XC*TeFVj|v!Dvz z^aY1zmN`N<&d7&uQvERd>Or$zLo`7^Yu1}^!oZW9mm(%1xNHP*aH-&XNW-*yB!3VS zr8nDRk{A}~)OH2yWn^9b+n_@sJL04WG|W@DuPf=S#`Q3HfAQ`RYKQ-wU}JBER8CIo z-PEDmz>8=kw7$jOCJbS-;ssALG`OJ0L!=-Kb+z9J|d5Xis6X@=1D`WvyE zJ#7#YJTzqm9LrL8|BCoo+T8&SfGtM4oE*10l;__!mSiU(Fpt(PMc~hdTG^w4B>=n} z6sA2@u#?SPX!E7}le+2qgvo$?-nm=xcFBf>(%x%&u59i2lK$IBF~j)|q6Sh*im)}Z z+52fsqM-Z2@yoh{hZip~(7AYy;amFyzsGZSHLGxcdZCRc)UoHiLKaMqXzpCnu^*l{ z=YoyyTCcL_9UZ4UG=2eP!j}3TNS%6`I}b)ZvvGekzyh;`I=K6pvtD0so>*q6({zR`7^uWEe(U`I>RF%BR(?=SV z&Z<|~mu1oSAOD*eLdyTbpXcEvEWm=Oldc}}F?S8!^u7_^Rl^$(vBe~j)ZL_lk<0AG zY=Eceq8%0JE|Y5H=b4>fx!7#>nEL+y7!h=>!uHpVP}A}JgW>9=6a3Pco29#ClH?e!mpf8 zWW<|J39s9I7j7?_AMfv-I)lCW9maM)5MYg{%&+`Vd@(@gPakzRj*LP%`m;8rtX#O? ziI87LnDp+D=&S?s{w%*c{53?ikK>qpH;Ll&V|EDZ$?xG z7jtSOFyVLz#p_hE9}FZ*TJ}2vn0A~a2l+i%2~hkltE@?v;9S4;V4uJUtG?q;4JNxCXIncm!yap~*-H_EPa-Ng@A|?&WW{4$E^c;awt~1U)~{)wi9q3A zd@=hh@rjrV%j=Vo`yhzQV$djIgqsUvGxI3W??}66ZBBaiO!e`uvGO+=8N#DVmSEu# zgox^KH`RG%Rc&=6pQGC0u~)Re2Wvev z(|sd7)AL*+$!!so`FfJIh^ISj!6I>-n0-558IW|bL_}#=6Vrtj-&ex z@w~m5Be{8eM$SO7B+%96a}QIaHYWh>@e_mg+i}*$6YV@ErdTa^$_y#=!5me|?#}2{ zW*YrF4?!_?f%#L3eIe|#iWi5Ku69s-GJ6ZAS)*|n?ThiZws0PdXW7e9_(0W_;Bvki zxLd}*;$r^~p(DM`=k@2nJnj`Ig^RVwQejL5tEVGX>yWc}@f z@A`K+hj=9h9}#l|nRBMT{PAicG~LD6_c$djEW0YMyuI?S(a;3j>3TpiK~`bgV_#~R z?;oGm{5^3pgRhikC0sCq18)U=xTfg38^V4m%4>f+LPD0qhZS@XkyqV|+%$^|zNM?C zFfILrF0y^v?9g|k=yEsG479&3NjHY4=e1@ZF9{8B z#3xREf3rBKePY#m{->WN5Si>?po#!i=&S>TZ>mcB)1t6SQ8CnzTY6)Gvn{2PQ zu{X9DV4giptLwg9pOHi%77vb3?Yrm`DQ25I^X_5P5KrYc_Q%}(_MrAd6tBMA3>Q^5_HqAa_L10kW6ov;WrSf(swyh`@KvG65 z!wrOA8hG|#Rw|fd$$|vir1}lpE;U}5aF-h$ZImFS$EBshDXpQ(wnkrUYn&!Nuna4;-m3JbVSy3Mp5)*hbRCXPZlNAWoRe5qCshRl5tJZ;D zX`dXzEAV4SW00`i_~S8?N+Yb^AprVm-|$Um{b3do3LCXBj^bFOGiOF2Rd}nXh8=`u z%sv2i%hB)=d^Ka^3=DB!l+(nbYh=kuWw?R`G1UJQT@)Vx_1!%5rg~bd(J^Y_F|rY+ zvw3-E6=biE3|?>i>^A5pK4E_t% zRg%RI+7{WHI0`UPVyx;G)oJfi)_N(d+-4vEYmAr86`)GhBOnP>BQ@in6!9Ptai-xxW`Ri&{RhadX3MU!8%K z)rP%aeaKTy_6`oTL~+Hf2ZykVcpd95L8Mdov`5=Dtl{Q8q{pj|TftA89qX5%JKvX6 z0;%Td$v@Q1I=>G#y@oYbG@mnHjwC%U`tGRjHq0?~e_CJuK$HWu#~>LSP!XfyaBCxT znSCzf9#W|-iCJ{S2{liRczmtZOb)V*a;oZGf893X+*Zy zZ}YAFpD|}8K-jw+7;-ehAyvAmt}tkAe((9ltF;34WddDCF3}vGT;o^{_wYj4h3nY+ zFnHUz3#GZc-)HFtX6u383n#pF9XZuGUs^<(T% zIzPmoD2JEE_(=0SqX?}rV|Jzjz=3%RSxRcwe0ELQI2;MqS$6Kd6m5DsMYa$56mGKW zI`J|(#10}7mzld!GdsId!M$R@&29JuehLX+oR;%*VW#9Ak)l~K`n_#DIi_&xA;eV* zFL{a2=f%J8I;nr$4L&dYeo< z2=eyf#qssZXSC^&`~5n8 z!yLWOBgret=}F*&>hG5jpa&ZFK0<@-?hE2`C*L!v4*(L4{o9<@vy$W*=QG z7L#C-Qx-~F&>fv9>@wc#6dh5{aA{f=+bS_<|NojFHt0s^)`z64 zJ*oh0sO4y+sev-xjj+Zk5-Wl8d4Lva^mWLGXRVRT4>nwaZOH#-HF@tH^N*?cw>iai z;@^AlZ#zr31MJ^l|2>Z^mVe(B|9cZ&D)4|fVLnHfL6lHbfygr~&{(|Z!ei%@+%8^$GP3D%wZ%Gg#7a7hr)gh|3f6WXbH_);{T4n=UB)N(wsL#)i$K$S7K$!`#0`^h(n+j6H#U?t7RPwR7zOHnE{yf=aM&)qiq!Su8=8wdP)#`i6l`-GJS3 zFLTiQrod@@Ou5BfB&kmsiRNrH)N%8S8syfPfBN?EpMAS9Q-V2c9Pa9G2k}8q(b2WM z^W|rf2Ivax`t~o`v9(hle%mQ!#ZR>7KuF>YX(fpK3D#Y>MY~HkMBQ_(b`QM!$rsR6 zJvz6xQjW=xCxFu;9sDF^)^XE*B`h7#6jV!?s*zvgX6R!Ucr=@_SosF3Ow`yrx?%lF zi+ehV&I;hX&35!I!L(1xo3A$@%An2Xf+D(ytjR_?j@47REnM0>IOxW^5`|=N?2Os# z4#kLtpMscd@f^)zHw~XiFMorI*txnkx8w`H@Pj|t$Qg3VyJDmt_u(c0)8W%fX6H3A zk3s@_v^IHbMy?*m#k-f1M>2=)I>PTOq0Vlu1=SpbeJ=>w3<{F08Sd@Ej9>HgaMD^| zXzymfcy_=4mZ@*82Ts_Zv@d5EnHZC?W+2{z52`VTE7)HEJxp*RISSX@!>Xf51KIE^ zSP%QrD&=haxI%xHSM%{ry8m=z%sSx5na69j3yI1LFzpli)5LM_4yTK^qNk)yHnI-7 z_o1#>X>;_5M=xz27oZ*I)bCu9{nr>0f$*Euzqk$c4HHk|L)4}W-FkC$EI#ZbIwz;K zHl?mrphvweAu34yy0ZCv-s&iZ{72z&GWsKp{k?E^R|&5;?Nz-{Co>SwI>Trw$hJGY z?OgFQKkBV>%x>4p68m&7xh{9jqgX8WY9<&+TpZ1n?_D^sn9p03J6%k`=J?)E<-DE)Y@Dk5-3q*&yS-n)Ocy;Vb$HA-0X>{`2^RarK;$2qNC7q>qQZ~%tTOL|yD%5WQ=R7X56 z(mZUuQJu{G_)7)TIl0;Ch6N+mSI-{n-=x73QpB;;PHQYr6B-1JZPGjA!sBztKxyR^lx$kMiz1V zr{LFld@@6&5`Jpz%%8vyEG*+8ykQl&Ja%Ke0qQAibPpjn(#d!}#RS7=$ggK^gk+yx zsktYFXu3sfz4B5gFh+`|6Yvp}f`_NDqH-al!G9s)pF1UbAyS^zOMQr2xdLN#U0wgq z0d9Afkp{F~x<3oO4e?aAV}fi&!WD3K)4>2!8mW+pAwsmhZ~l~&^^B?xLme|lnI`em z4m%qjC)&p@dh!I8YnP_2JIdy3u)*6^Dw}T_&Ow^lCwg=llFV{7#vd$>wvNv~h-o}m zdt{I6M@^mLV>}=e*L-pt*+sHwb@N~0AS0p<8*B$bnOJ=OX?`NIV(~oQA#xSu91n$q z@SdI8EZQSgXDPS@hL9JXl#j6(CSRT`VHoWfuh`7?7MjU|!ty|}>G`{_p`ofZu4jxY zHgKE1F-jz!F|m_DTjLuDO;am)y?W45oGrb;j5>Bam-fMjSL5%k2HdGX4ezRBtQQav zX%lYORs55~axq7km}JHu2Z(=YfKNj17WdI)XjRElRtnWlDY4now9@oezs(TdV(`5{ zd3k?M?mm-ds2hv2XiXg{(i&6xmeIG;%E9g#eo95+?vY#}7&SFPx834ITr=u6Qs|}P zf_U$8+4o%*Eym3bdZLk+*YYS|;p?Ui3yQ|$viLoi=Htw=u+rH0>Ij=q0752j{9o-& z`dpJej&X`db!c}HTK7a_l|Rb8hlpmk!rjL14>9vCo^6kX`0!613ICb>Ri6?6T@dqT z9r-xT(kBlmx3t5y*L?ecd)FjYPjZ7HC90;iLiLk`ti3_5xezX8G*4kes!_XOx%&KF zk8qwhQdX#_rtLfZdwuU<#0ZYV?+3#k-RLCS3NdV?Ym}MoG-T;@ip+Ek^f|O?vTCL2LxR9% z&?q{JHdCgit2z<;+|AnI&|lBl93G$FJ(pQnPDBB|G{UnxG1)$2ifIJkw!N;#$;xIP z_a5!QxAyhLyrw@LSzJ4_VpyMav%AGeKgQZRCB<_|DmnP3KxkL+A-dJHDJHd?h(WzN z?=vw@jYEop6oF;sq^(zNAoX%joS#6n3#DD!v#i>t>_j%n$Afvy*-z4A)Ze zw5ExYi^&1}AlbP|C55dDuq%0be|Xraz3%1guSGRA6rAv4dW=y${S-Rij1=InMW?70 zgUd{~CPX^V?v2H@uJBnr2 z&@$HE;V@0Jef6I8L?i{nJG4>rIf;U;E4ZLF>B(s*$hL8G<&v7_ChkEN`DtUvrMr4H zkQ@ynB&h)-<-R;jgW*rzytD|9yX-xfkf>|siD9|CxbquCSH9e?BBNMf{ld>gpgCsd zWfq}K>&ua?&=AGozX;L*YcxdnZo}ZW_0=*lh%kQLZ$aM;7rh7-sBQ_#6($ga3r`$% zGq%<;a3Ub zixJdh*@-&|LX7lj`Zc?4(K+1xT--g&C)NzSZ{BdZ1(H{ym!X=dtxPHmkyLYfg#3Qq z_v!6(4vU@urH!-B&$D1&a?GowsA-LEBR0&=ep^&i*EKqe72J#+@5h?IqpNpezhdzu z)kxpk!8xl*g;;n(Dm!4ZU8dX+v~tpO=%hy{Pt$=GoN!R1$LQFqo<_9)`bhfy z@cmje*{2AHaua*ZOStl})jdeKW_IbbFQzszwVfRUkD|eXudzRpM3Uzb?|V-i*)8T% zNNQ_c#7TXI270mPmA-I(Prt|OV5*Q|Mj?s*uSR;fXC?^-h=p6+-FIidVBP(zktc&k zKz`c4BqQU0&N<-~WR*caW0&+n%T`GuR0bHKYq5521Pr7Vu8o+Hnz+zc0WncOb^To|Y=4~HmFDKZ_V7)Lzg)2XQBw#gmGG_x*MHCr__ zm}SB-S-I2)0lA2Tt7(SUk9?ahw3iR~-4mp;4Nn`f*;_^JF1~)NxJ`TyfxZg`T6Hzj z#8*@+9P=%seNjG!pEkt4AS#y|%Mu+yLc)mBSjspsJiQ4J!&jzrf*v0PIn-z5^y5fF z`>FWvapQf(0qga%#Hq#h%yBn~g0+Z#ZCly|r#%on8}fu2?Ux=f4NekMet19a0~}uL z(e|#z>XS+GT-!P*)p_QH`$-4~E}p3Fc*1jVxHAg_8=tSfaiJoy-_CA?2NCQ@v0ZeK zKIZWzp3%35b>ylq}&JwY(SG4s1i| zDvF&OkPlM1p5&H>5R}xX~#BqoyFwioSx1^i{R$ti;<0bNFqZj-I*~Yyq z11q}iAHoL6T)|X1rP!!TKlyUBnwYhr^lRSh6hnO%;PBz+%*dagN@e7mR(o4DTc9%> zw>-KT(GTyQ$*uI+XlZFwu?`ER3bjmS;Ce7DWOWf@%<_0rKEh`@58mn{(~;J3aE{V) zPjSp0kV1-(FAuAp^xJ|5uk1~jWY?^ChWR)b5f!?QF`6tbk1Mix0jcvrC9x8Pw95FV zMjPi zmeJeHtPhsq(v^`FSjj7LK>UFtAe2m@>(J{fRMPhW$EUDCGFL*_1K3@ygJ6-n>PkOBvbuB7dZs2o<4?Lh$0+t^*;(fG~{Q+&FchR4t5dY<#qC%1{-dM2C-pT)Nsbt8j~(PdBNb zZxgo|kKlbgAP)`+al^WN;J$GOL+)e#Z7n*#S#*vvFYH@hWG#{eKX*6DZ7M;eCgco_ z7C~o|IIg>Sko&H7G}ZGTMQVc}W!UiCy?xG{6~bt8badoNq*Ijwp|e=EX0HnWhg@Cd zOO@i?1(u^;EN$huS!eb{@Y`sx@0v4NXvrQyw!8m-_BugGH+V7 zj_W^S+=JUFnp#J!D+g2+5sC*xw0}@!Y`bbc zhgL2O9a}3p51NMGGh%xrGajvmh^!sy0cv8O^osiz z6h~?sJ$ye;i_DE`4i%+J)TE3nG(j0{Aoqp#`-71AE$AMEBu50g&iZ$7ER-(|5k4S4 zF(CzQG)?@~ITpVxCsvIty`!|WH>;7%e*h-<30_Pl9r9R1!E(%8fS`SF1d=GUr?#_q zQe5k(BltJUEAHpv%5ggzjzMzg2L4LRZTPpgbX~x6(R@N{Rku4XRWk?m=k-rk_IHm~ z_Q5z+N*A?E`zsmn>9Akrf`>C)-h-cbYkad!%pWij>k25bPisx9hL79z2%{bVf}`AM0Ifsi5tx+>o63x_2b(ypP?gQblO4H z3~?u|%?)PZqD%|n$@X%*)MoO0xB&~@%j7}#oe-2wy zCX#%*W?1Z&iYnyIO;sfx_hU&xCDX)A&ee?|scEl;!>+4^&ysv&BdHWq6|beecf;ds z6Vk7*w6$#P1O8r_XWFKlCo1|5dYNgS>^`n72~85#W{KI-h8K@*Aenx0=XSw<=NX4z zWc#wt8wzazXO6+%n$|fvojqx#fG{rd`fQKv*va9^;$agK`#b9}7s%B#TL=G15>B_f z7b;KL4!0;WO{pvHMee?VA`M2x^0Yu`YfOaqc64~u$MyFbwPmXv95M?(81r=20c>y< zSCNsE)!?#iTaCpAvgNd#yB#yx+s3VFtob5Qagis`?9Z^_&x%GiR>2+_0=#YZNGZ7= zp6AseD^|D7=q7>zOBsl3@}k|3iZHl`@#IR^J~6C@!##eVSxN-O-fc`2M$-bRa7rV; zVNXg@4`Lj#)XYy5v=}G9u}^;KrL#gNs6bBKk{YM3I&aTsi`gMr-=apP%XGi<;%w(G zLMl-K`o$7VEjEhM2F<4rCickduB20A2IsnpS`bvYN$TN-vx5QJj#;iLb!$30##?q8 zIjoKIH19O75__01Cru%-uVQEmALuGg;c(VvrmFo`*<8-nNf#W$P~`&M#_Febd2Ju; z@}?U_KLyR;kCP)9(M+*vT%{_)N?rucFKS#x{`oc!?*!g%6_OyHWqY{B-zMK)%{)IB zONriW#B7|$4en;qSTU_&LYji)0x3BZfRy;>)l5M=ghJK^{b1=z(R{rCG{gJWpF(cX zAXmY^>Yj@fw1pOBWs#_8THz)LX^HAUWt>iyNn=G)gxD5tcvV3|)V!wzEllB?ikgQ1 z0f4A^2oxjoG=DaaJ6}kN`Ix52G(owT8W#2Q+`00>f)|mZEBNC|5a9a_Gz=`pStRjj z-UY8rcI}?c99?e%fQ=OjBmA^kQ}~(|iF!G2Dy-`F;0u``9rEXC1IhdcW78GUt4s(6 zW|o=|ED#DIm{vf9pGW`_rraNN8Uc#<5Y%WPW9xg5z-?^2_Fy9`1dDZIm7Fw|v&?*k ztsOKgcJWE={l`^`i#?UcZK^*7rX(uT6uyC=&JUF?mZ;}D55XP+tz!7se+{TVono|3 zj`QGMCh)P#;Lj^ckTC(#yVx3G3;8XGl}7k;YZ~^L|DeRa_1@0$005@8%IUkjet*aI z;(32E3n4ybvUSs5Y=jinBOgu;0U1&ROyRvc1k@-9lz#J&5k_f*OSAr=kY~@>SX^_> zb05{iE&~>5e$3t6zcy^PHT{vuTZSuXa;^g ztWgf67osXyN0{V&|K$2-(EC^Q|FueMhI5237qk>6m<~nd`s(@`G|&|;VLtT>HHul+ z+NtRsZIi@tT@b52X0CR!_gp{nnwI>qgkBT!^th*Z?G1VE<%`lE?8R)nRe&Oi(vSA6 z99i>u4!sOX0sSEC-+&KUiRm53u%;d{-MD~*e|!)OXm40~z8B4M>2yfS`98_#-6lzK zO5WNjm#acO=bbrwf?_;iGL&eJ+rbw6vt8U}PT){C2F}1h`ixLNhY6-4lgB4LtLP)F z4O5}5`}5lXZ*P@=_9ThQD9f1yp!({Vd<83$Ah_-6m2%l1#k}8(IC@RrkRxFCa+))? zK;ObUy*BgyS(bP<1SA7F8pl9_b=!)y;CoE9UwOAqaO>~{_gk24PAI2&K&Mi=H^K1W zT0!#2f!@ZX)Erf8LTg9*D+`LYp@m-h#S-P^m#N@{MuTFIS6f8R?3R^2XZza#fh_po zHhG?%IxZG^R44RF-i_HB;F*_gknkdMVzbW+M$tF1`>F}JmWL(eTOKG$F1%1j`(1EC z$r_%u9HY`aLaRH;rFK4PN$21MGW7d(=%O)tV>TfiqbYY zy(LMW6~V5JzU={_m}al?mC|#sSVSNO6bg%3coII{+{Af5qT^sYjkiN3f2uG|S1*@~ zsg!5PL{37s%BN#y?<)uFJ33#PLZ}KD96a71GP+MFKJH!#MbWq|CsvY4_?r{6NI)~} zJsk1e+MYLf2 zV}mx;oJSgJ^mnyXA`@#T5={$fpZJ90$=L<`9!0jL8@GW0lSNvfyd;`A(Z+MjcAu70 zBTf-+wl>+@HIJD|UT>j>$! zyUEm|AT2%J(2fgEV>`Uj;ny7A_l)Nf_ptFNgH^Cju77~M1-e)bimZ3>v3XpnocECZ zT99^`z9y-xcC&POE?`! zz1W7N#c^>jGUQIB`l&GLa6sawi`hA4#!#bx?z)*m5*F0#vbH;2s|fnZu|)YEEu6_s z+|E3?f^Ik}W5yI}kS%)!7P`1H56`;xl@n3fa9^G|N8y~L#bPJ6NiB7JzAG5mn8mjy zpbmWz_>`cw@DV5YF%*RLaE-9NolsslO7m{L4{aioxPZJ!Ev=<{EhOC8Cab|Q%Y1xb z);a!6`xm*FPvk=blF_yU?>FM@9>{GEZO{CAZR&Qw_|QFfS~zTBXnpohPnv-+4^`nc z9R@~1`7xG&AN73y;cW3iIM8cH^%dD_LzuHL>M`My%Nc8|?bpt9AOB1g+3 z1{6=)7jSe%ghw>pKZVGyd;J#TTveyC_Ud_d#wr9hlVx_ermgRNYzd2bc!ERL&_JFc zz@uYLX!Il1AbJvKfNB2Q7hvpjdEBRk^QNqg8Lj||HtbF|qpizFh}Al5aF;(#*5B_V z%4m+x^8RbZWU3Ai|0kJ373z71tkM0au;yO{DO3;Fr(WN14>l>q(v8;D)XK8RnH{lXSk=_1u)g9rs- zR{59{RqQqF5DbHVNKoJs@tktzRLABLbMbVf{7b%S!LBdb*6p4?>loKSDP!lD==D^L zcH%2&;PHULe6Vx8DLT;2(jbtpvt_X5%qRQy5th8^tTw$OuRaRT(&YY<%TZ+Yy|)qe z9|3d){UnFwKDOad6SsvTPCiwQUbN%|m!zIVXa(65hI(=I^9ODZzwW&^D~RM{r>4=_ zt&%fkpjCBUOXJMaTVBS82f-lA_0jVr4A)HA>rJo9s6r0+TQaZdkFcE`?z~c*=(~ME zpVV4dsZ6LIJgdT#ia#@(8mG0e_aCe>o|wFhMXwvJMcZv&e&GFL>|z!jr+Ih&yZZ^) z;^wf#s#sPRi34uy!`L^p!HB`SRbwRCAKMim{6SQ7HV$C>DYnwWbW~@sx>Aov*daOg z#l*raTB}_k{%36Cu#qA#8#Jnrz1Mx`HG(7Tt--x^#BrhJ7Ivp_(i)~#`!gd{49-`l06D68 zcp|RJprSWXTfBAo24g(p=zO!iE!$kHsa1B*kYMPqu$tV3i1d-`8+yEbVTi?{97pB6 znMUr0NKJR>5{URgFf(y9)DzmIQ{v2AJfnR9R?Fl(AnAUavCG*d7wEI&{m3YW58v(m zV*_v!)wA11^-24t3Jn6mY^5ja17~tI@#0MHl|Q*56dKFwG)Jnd8SR|XxS)OqE%S}p z0mfvdnOm1dm2#rS(^|)l)338ygzMyLI$vX#Jn~)rNoGh~{sazzu;ZZ8e7n+WL+>SU ztmA&gyF=82a@w+u7%F8fPP4@ur=@ASEV5~WJnY_gDj2_Pa&H?XyyXRj zkl*-Dr!few2@zV{B`m3Jd9#I}?GeaRDxKRsj;U)v!#cZ#5!_)zMCCn4hg5g(n2_1+ zRr_Gq-_oh$r5Tadrm~r>!mGs~H82F+z*q<0wF^y4&SegtLM%SY+xfi~rx#*X-ezgv zVs1T+czDR(G`zSbjJ#o+7Udt%m>{F9Lj7Q=A5+;KQ6I>ojN+V{-7(G+60*sQsf(t*htkPll%B@>gWA+K@#!iK?s)J?uv2KQ}md^P6Dvz-eiY)S1 z%B|quYnW46>FQX^t!K2UDA*tP*I%3ce;yWh07s51_lD9&>)%S(5}>BF@QuDqurUTc zn13HyUCSt69M2>srZD`msP%pFdJ)ud_p`fKb(DTp&-v$MX~bX{7lGwO3^Nb25Xcja zTAq$r{s&BEr>FncJn)XnCc*4XZ{rz@umzcl`mC;cO|qUktn{9tq}IVFzLa5n06JF}tXVwgV(OBmT#^|mZ`j=>&$t?S|O<32?FXZYjwMrAR z=Rrew+~km^@GmwH1MSjKJZot5Itj`Qd_OVhq;EqZy_13OU!Fk;@g7~J-0N@6R_zD2r8;}x{tO85ctN+d!sVz#s;3^BDCF|%A#au)oXq~n(rm{A&0 z<3S?dDvk98*rPOvLOx3m)pgIzV5h`TGE4WbjO{vZ*zymSdzCNO~8k&sWqrN_62jGaB=zBPKQE=Py`eXs#& zeqUW?|NSB5ZK&PF4qkL0kuG&8#WKyek1P2LCEwG--mL-6od`RaNY7xUq%_-R*ZWz5 zH;dbU^xj*id)jb8wf3CKW*OQL*0pu(e-lvhL29ZsGlhd(V7aMqQn`Z*wP933bg?t9 z&Td{P;Mk%_SvN|t@t3G>$XGvh_BW1$Gf*Nt_~<3@`Mb)XV~}`Y=2$ro#oF}vuL<6` ztTWs1e11^%)MSCPMxaFl7Hv{@I{yqIQT@SNBr)Ht<1v zO24|LdE!+E+a-o1X4J12gA5zxoD;&1E!iO1%XAU6EgY}VQ^RI{`mtdQ_(|QABy?jA z*>Y95M__(0MST?c!@GK}fvMqbN8z?9{o2;X?)i1z%2yiR7>p1i3T))d#?pP4W|t%C zRTq5Ufy$KO0+i63uZ3K{^7K$HbwX11gx|aaHZ`lIKQJdL!jI!1MVLx`epv&n@lSij zm&xep|K{^Jhr%`S!hQbz#65XZ$OQ>hK5I6r(P+6%dgcd{s#kRKhEyJ!l(Nul=Hm6q z=C{?)`tg+lvl)KDctJr$Xy7rjaX?hmoldRM@bb1J^3P#*v_=4R;98)*oucw_ z&rzcidIAlTc)^>5H5n@Ld+}-hHcBdbjhcK0aH8i@U z8!n_!lu#@@A;Cih@um;->RHYD~E}xCjN<&ZMw7QVj#U$YcN^3QKTcKwcj7*f7Nzqa0&4b&66A zh$pdG(zbOqa-V+az2|$_w~)ju+Sv0^=3W1s%y|AD0O`)jwz(pp;j@UE`zye+EzGcM z@A@O<77~QOnF(Fb5+rO&CZa&sO%O@NVv~C{Egz{C1#2i9OF?d&0Ux#?0gD*`1)t=| zCcqs>Htp>zo-$_lJ=hQsCrAC}Q?^Qcx8Thji&<&I6?VM;^bmYFT88@G+jjwE0qN99 z!|{@zFq1grIJyrQ`;6ir0wmDGm%`PzK*D1196^ibKGZ) zSs$r0Y;juTLEZX|O1cY&_>=i5Z^F?cxsx4ju?OIZ24#neuOfi*Z&IsaA1w`)I3jt(( z#6AgazR!N980nCDllrsPv%(jH+`!Uyo(uwQbsxx9Vacn!Wiy2Y;*#p zJd#?Zs3Cl;5fbb>M_y4_%L!<43S5wTXs&JG^bk`(7-6?%dS-EWDeH?RS8bBXG8?%j z_nb_cWc$J!yD8Z+YLsKtmDk%6>C_sq6> zZww+BI$n0_QZrZ-D-#Ecx2m}W5lfN9ZGn#5$Omno!eT=E;7BC^wLj!WYekgZh&<#|3=)? zo&pAIRuk9VjpqrqVSB`!u$by_z0PkOY}U38O)rYEK4K4m`#v@8#*SW`_c0@LZc?LX zjFS)Wr@h$@wf_3GvrK1##ww|{Lv>`Ou+^m&4vQ5v6(q3m0E~@f`&RUsQjJSgPL3TW zSI_;n+~i}U?}I2_U9tZ|1digVf@>L2JKvY$!i<#GcqAQ1NsC9;$A;ZYs27RQHLCjp z8=I3t`dwi?y^dtmLYJn|Az#maa`$OIgi)8S-FRI7zOiDD-i-k!;#n#jbq zDrNKY!zLtnOgVwxe4YrVJ^mB`iFkQrAOH=)Ge`|dbai*vkSaPSKfo~8{TAmvoAANO zsgWsgWctJc4=;PtA3fF(rqIcOW;&+U<_^kSs_otqU@t%^HGzyybe!Aj6MC@tkw6FD zVIXavX~|~qx+ozXSI-p~(8TS#W;kF9FsPBq8`{=e2*x=$j+vG>+@t_6@_#yNA{MQ4 zvt6cN^y)t_PxjW{p99XdO%PnVVG$9<+9t8raWjQ=XQ4a_kw*rhT-uu^8tpR{_W5_g zk&}P?`0<=M?53gCZX0`9$WD1ZCvKu>XfH~wye%j_l6h(+rqmK40utVOIPXFF^PT^g znec!)V;`*;2PdJ>EQc)7nZexz(RJkBLmNPiVUY;XtKGZ9L5hX1OOby$T=RhjfI$@Y zA!--Im~EKj^?60(X$lCQthLk+gi!ARq5&gdQ3?rx35u}2E5dxr0lWyxX;2cO5LjD! zh_arZCxKEbxmx!Ylioe}Zr6ExsxA@)23UFq%4P+!E%t8+ zr*ChOE_?K1{$SV`V@auXo%Sqxce10*3sh(9v6y7SBqojf!67OYJ<|v=VZ7M=B}2Gy z>#Qn9580APL~R2u&@^053&;rvRTM+9BhVq$f#Sg^cKEI0OenE+l^Pl?8aiN4m?8(5 z>PWGw@G424$e>lB*2#{jS$F1yWw{RzXqNtn^I&Ia5GdAT=%iDeKuDx}cDr^>dAW#4 zMA1XuhoK%lEE98bg_k$YZIdDxrSdZS(D|zpfBmp$V5z5-pp?pV$*An z#ET%NYAHp=ai5h=#eB-9Kqn@vxZniOxrV-QIZv}DZ_NRa2-kZ%^7BD3tkO2>UH%+V zY5?Y~3lL|q-wq8=djPJaN%ddZ>vSEPqkbE(I}}H4bvY$Bzf@S^P6&!fW*TN-W@Z*u zga`{4iL!Z>mC^dNeG>s*1r3-mCr?Xj*xgDRy?6|WYPC6NN{s#YYMHayvhMZS+8ONC zt>n_O`*-#az+ogcdW85siSHApCIk|iu}g@__IvR`W7g#Z)^DPeVks#kD2Zk4GP8_m z0MSK<2Zwf?sK;AV)uLwOm>PSZjW>5b8j9jYk)C(s{Utf6(O|N(VLkVp2-jSirqX@);-ig}t(&V4JErY+pOMu*8ojlsibqPlu+5 zPtUFg4{Auntwr_?O7!gAD(Rf+pJm;YN5~cRr5oq8Mr^jl)D_WqrXQL|ci)07BXM21 zeix)+o}W@!ps^+iKv*ZYl91sgwdvZ$R>Jll`VNNf3DT=OYZJzgzWn+YF&yPI`-7mD z&w*4NJ)5$oTZPT~*H*Ju`lfhbfqAi;ERS`9U-`I%AM3C z*JWoE1nL%|q(kKrEpw`Cvx4PAPpn9zS@4+&kdd6v3_3YNY4M6$n^Hg|S^AFhKh#8j zSFV+h_LY8O#yCAGpf)eqEgyalNuH-_(7=Ue*PC+`Hp9F`4@}tBW%zx>G}ThwD>ivE zk2D@l_PA6`VS%QK~J@jB77Aq340^VC_geI#BCyU_-fIURH(Wa*pCxZQ=UjsM9!# z39?s^qpUprTYs$kR_E%6BqyYW(fPRH z_@y?LJ;}d}&MH;a+1cQ;d4x*J63|K$yD%|qsM9=^-5lARV&vJ%@XGRhj|;xnM&`f3 zx!x42mNMSOlN=T2+eFqLCaJA@r+@yiCuMGjAosf=y(YoR69BmAdxVsS0=y}G2U7bCuG z#z+e4wV8ZAf$3FVc*|VaQ-9YFF;p6SwpUV}>3#(hNXX4?viBXF9viO$!Zg(N{qO!n zsxHlSj=o-k&;u-D@{%CIN>h>7bif=t6jmFH9-JKMX~Ci9Tsc!&8Xw>tZ^?gZ4xNTX z9spHtZLLYn1JcXrEjgN#kV_T(C%9k0;7IwNcnO4Jp>(z`@Z!UGcL$K>DfK z-^L!EQ790Df<6nOXu;*`*l&#xp1zQVLr~M=Q;jK1KVwi_x3h8?Vs@wRY__z38D)bX z@$ISFptuLrJ4R|zRv8*}(tOu64Enwj-|krI)fJJDUcW(es#$h^P0de`OP0BnvX5kT zj;IGiG_`)KrtD(h5liSkcbzy-J2`&Rthh;4B6Wuy;CJzdLN^FF<^v763b7JXiumlm zffd7oLWL&D)Nfk(jRytgC)ZR~)se2tG$clvF`j3JZ(NtgmuTrIucH(uBNZ=BB~hff zLxDlbI;#e1MJrYps7aL{!XQFPP;3LrRQAQ<8_;0tk$qC)bc6k(0&eRLTd*J{meRXZ zRoYSni2}Ye8IuKy4a$RgCACy2!d&DNwuK$2pmzl_MWaWJw)^5mB?J0Xkrp2RUy_M73uZ{ zEp2G?TWQ~CJlgk(+YWdfl&rn3^nTsi0_t&T7|K*+2Dpe)G#477V*07vg>T!Pqlcx1 z#qoF&lU-}*Vke72DS_cIl&<07@X5`?(Woq#(Xn$C$}(1)k-zkj0NKb7z-hI3Pevim zq(2p#umHV$8M96kvg!(>m%z1T^t2`@t3nM(mpjT6&(sZirxyAi<+C?Ll|n~qdySi3AuRG6eume6yEbNuJ$Lnj++IC zz~T{0ZO~X5T8{$5i9~^{0>E#Zh_E!(8E=x5$z&y665O9%^eCsJ1SBQ;$7@%zTORS*Nv6O)(Fo<~N9Nz>!DJ zO!%tUX3bSw85b@W%ZiOg@2qryxZ#3_7lVX`k0(}#$zmXj_(g{c{V#37N`q4rSh!tQ z(r2~h1gMOs^PHzA-~6ySjSH*F!J=WTpkvytTdSmq-xV7@#Rrl`piVHE$gLR*=vNS~ z?vrum9+L+ND%L5oRs^wpi(t-)PYor*O8h`$uOCYhDU0#oiV)D0Jp3hy#+QPC7D0If zQ1m`TFfzKh=AMeVsSvNKH+1^6sJ_{E)iv~09<#J0Rj~;%gNwiA=LEIL0x*6%|DaMI zdL23vVvf)eb~7VQQ%jYn5hNquw#XdZDy77wz-#S{&mk8xwe>w$LocfeW5JQ@Hml#L zD(TA|z$Hdu24o33tJdmH6F#fpv9ihn4RV^>kW$bnBzjC!#EUF}K;^B*jM!KvtVDS% z+YiX7=qNF9V`(K>Eul*dK{WCHW%c+~+yfi@p>YM$Ii>hb)&r^<{ulL&!lvB6S3|sUN|Sabrs*ba#5dA^zw^ zaio@!|lEnqyR17k^YU9|Ret=_p z|4sSO=`f^5esV-(xuPz*1eyL&ld;4U90N8w{;Wkcq_xxh+P;W2inwr*_i_bx*h!m! ztF+jNm=8a|iAmVo)iiAZ(*j`&B2;WWwjgQxVtFpX8lAZc~gnThM>M7 zHFN_#RtAPyc+Avc6N5f3c?!s1nS58Qp;_g)i39x6CD+&t!(x0z>4?cIxJnrB$TA4;+ZpHj zXmU51Y9gi0ln@gvC2Eb?J1!-FiHmpy#AN7rdfcT7>)hFCQIx6nc4QaaL4`kDPXKJ4 zBs~Grrc@);>=;Z%c)0i!0DMXddi@DVykczJdtfXkVt>*XR~a|`i?_qd@ya^+ADO~h z9E=rIOa)XhBe8>ADaMB0&P`_dX_8PABCw!KDakJhd*ytn zJg16HYLt(JBqH%53;(w-fVdQKkxYeen}1bavcxX0^b!qBfKX&4HIw@-@e$ zIw8NQJZ;){c4xqZdtV~Xn=L2C&%Y5Px8Src4f*68aoQ9?KDWnp zlK_(mLSBRm*Gx~|n9*H9{{KuH+w5#?!Q;l{DWhm%!Uso3+r8CGmJp%|gBrE_lA%AM z(*KY$DgNkz|3k8kkM#W?Qv^o8$tOW<_5%p*N{@c7w~bDLzrV+nwF1|@ zSp_9o`;+bg8;H&fgpb`CR+xgnffVuu(8GNmT{2}DqNc4={`nLx*YDHx<%P4u*5Cih zBqZ^BP1*5kZ%BDTP)JyqE3xIguIq7Wlft#Bny8}G^6Pkt;M0y1<{wPl{MKamgGF_d z_XR>|`jfs_vnuBVS6@r>oh z|9fFQ}S4t8IE+Li;QiXk{Hip);wQJYGC7k ze`K1ImFie|oC9=qfys*r@`}c7b3He}lLcy#i{zln^22XUj%K}oN3&911Jd_LI(+x? z@vfLUlNz(-lOPL)mw6vkyer^6udA*cZ_02ehOx%dwwNqza+S*G;GlPZJB$yKQ@~%b zC~dOEz%%yXR-afK#VjD|$S!i8sS>;$bSI7{tTe96ok(Y;IpY5O`3gfeT)+MOu(i(e zYl=r&fYISu?<0nBwkjm1KkW0s64coq)i}*x;8zgg4H+!F&q2?->3WY7r#JP4mcg`d zBLh1FW;vGOl*<-4YPleDT(YX%^w3ewG~f48iBx}cT9)6=sN8LQF<`egV@Iozc>SIe zUAQ}c!!=LzhNGtY`s26(9F>O|^t10vzx1*nRa~)|)O9?gdqj3nv-%e68AY+z=*lf}Z+{VFndw{Qyfr>l z>d^a+M(T6N#;0gmeXbdFJ-^AO!xkm-`bW_~aDJdslEcS;zL4sf~{AO>kz&uMw zLal*d%8|UtZd4*M)h~Csb^VLo!`>AsljF2Cmm$o7(~BK#^lEF)VvxUIet#bN^{Kxx zCAQ*^Ytma{;Q4CXX-k3Y`;C9}zGe_BPhY>GEMbPh(jbrsE0&VI z#c-$ZRZEVqW!*RVn%-!8ZH2kF6V^5ACvtF1yW)F5b1^&G1jh&M z&q2e#dVM1bC2;dLRi$m$7b3ie+pG1uV>GL@>rdulbyW_wJW7Wa`ZWdID7<;an zonCgn&~R@I_ilE_gSw<%j*wrD4y0$ZwXW}oYJRY{f4>ZTXZcI3Asl-R!Jd5$d!K?{ zrOoWxGhpeq;cG543VJ5n+?~ENJPLrbkY2vNRX-CV+-_9TZj6hN&Xn1QST9~fOA5i~ zM(3MbldrM0V1^6@z|CV~@5ns-h>EX3e006)q^5hj%^ic*FQc&t>lxtR=a28#uboGV zd*HR%n_H*g?A41P+zEmVw_QzR(bXFSk=_UD;W>tnWCX=iBh{?cW{W49{~0DDtA$cp zEU^*8i&nz#8~P_Sr>%s7x3AJI_sP@txiPqxTRrcOX!iW^0o&su`^wW!CJ(P7TN$h# zNprkHM})^m9|ZEBxi()xDISvQHo2kwQoy*O{|eCVPEL#Ir}TtP8!deOF#&%;Z~Wy( zW6Dozv{3kn^>-OCY?LYo)$lh`lX=$hl_Qk$iz@3ZU~(8(=&J}xHk&8&C38uk!5HYQ6`Ur ze3LY}B2#pQtNvq{0~{KNSKbJ(7be=S=PZ5VXH`zf37^>g|N7a2R+igE2F~Pr``eKj z*zKBi)XdlF9r3Vk!aQ0pXw*3hh*(52)XJewLh<2xrq*-N8yh+@?{eU5^jw9iGPpkg zDqBV#pN)mbwr_H>jQUmwm%-!QUB+$EixcIhDH<`URPT3#-2A{sItKTT{Pi`=f>&~^ z%>a8@&_+>I{=vOK{^x2wdb$P>F0lKS@3if{3up5Bnm)D!X!ii_K!JMKzWzYOnBRyBM+UR<6B zzPQW>6&)(>j9vt)_(N$MH=xvNmi=X(zX_vVX7>?3)Liwc!ghpErdwQ7ztpnk$4iIM zFOq)QY)^CiWvwbi`}Jc9kVNoqfF&Ts{^)F9i2AAZX4Ls|d(jAwI*oLLg0PU;(faKg zS=Y#X=e8W)qxn>;%n9lF#}|tZmnG z)Y{uMT8*BTB2-c$YJ=mA75~$2W=v=!hz;$c|{o$3ASp0LLfdoT;g1BQJ#_~CiVyFTjf?ak$rT=orGXnd% z)Z&6}uBveekO~y!Rxft%u&HD}gFhVlVcu%0@UuzjLmw`8Gb$T@L*$5Mp02vv8R@JI z8%pE3Ow3h!~1RzX`0zM7Q3>^gSaxJ zvlykBX-|&>c{=vH#+yEJ++gUN1EmoHfx%d!`y@7Smaw;QihLebBwBsFd#;^)@;SV8 zMP|>}74^`)FTWf%nqvE5u+w?!P2NzDrie45@9W$rl=!2H_biAs6&thon&HYZW^dx| zi7ifBUL~47mMPLp?n-!NqrZdlju8fT-S(M>DW-ocN=DzUrv4f-*}Z&}gxio~SO3c!9Ktz^1M+l%&@M4|E zV?Bh)^LbagSlO{S)-*N?szj9e4H{QpyIa1A6BtKW_R+2{7VveP1vYQKA|HTJ`aI-_ zWrMtC2osvt2Ae3LgB^~4d*ugjWeB!!N=SAmdRsQB11w3o)OnSZ(QXC;1+Hd{rwVcb zh~l^8_C_d;x7oGs;5BlN*-y7>UG4Dn@PWeLsFmdT?|st`d;ko|oqSDwuRdd@m9V0|xG4nQ{wxF`_$6tLYq5hV)5W%vDos z;AMXX3=ca#%am{ge6k7~h%=)VE0*BS*&Yj_B1?#6w8Bm%BHR65yc=NBA~nt8G+1gA zCwbq%{8utpIeo-*yQqwuT1sio&0&*glpA~6k!~mh(eVz_>KH`C&+A;m_P!XYo}c1k((Y z7}EEsoSH`}?k+NSJ3w14!RZ{SE-!41{Gr3g&YG~&Pq1rF0&`^7ohUzUHhdvuM({YM+K{F%N$`B+9N}Z zI%_7>^*AVX*2rS6s*Z7K-PM|tTxz_nzf|+%ul4SfnfQ7f%++3jc+Zm$* z>fa<@sgH(VCaU&0Wc@my=~;!e2C!P@^Xd*-mACD_jA)KPB5NW;u3=mmAX9tx@|a02QYB z?Yq{3J;msXk@ej(H}k7OLm}UW%hN2}x5h1_X;UirY4O)5+xF~V&NB|pp_-0NNxRNz z*BUSAewNHZfYBf@{UX--GO(CU-B+Be9avqT3kb|^)LG}a(!-8SFS>5f-VVp&<7iV# zZZHeQPuq8CTR{UMpVYVqhNUmJ63SaJsai5I%^8jD9OZ%iVuL-Q!5jS46v z2jKViv;x7lye6iUnFCqxo&o08m*go!S&QVezze0I`%~*sXQ7xFJUY9@ruEqL6M#bDviM^M{I#mVBw}w_<1l7-3vL;i-*g`=ciA z1_v+ZhQU)c6f}#hAW#9QSyLX7-&gxs1A9?d%J=&KQEmu%+}#{Oj646RLR<=q&0ZBx zV4iz+cL2Fxeh+jl=;d?t%}kzr$jACOEq3B(P zBa!)PaW}u9A#r4b=h_(0kr;OI`ixl0c@1K7bfjFS)!aHj&rmLxm60^?T3>26b7lTW z7JndV;G%^b5tP;!XIU+1%}=Ib>Wh()k^P$)DRvyWy<~6z{(yClb=rQ%7;n+VwmBBP zCfRboice%#M$Oq|Xpd3H?%^|N?0T6wz}Jf_1KtBIyvRZJn3G5QW+qnN^)FL(uI^NP zilYv;k+qePC*NkSp&YbSSsENT7Tg@*Zbd($0T*?wXq`;#gU84?^}i%dj^=QG|Q>9m#I)$1EDD(JHTG5~IHbe7=MWR`Y4v5z= zm)x|moqb=5AUi2j&nPSh%Y#eW=2DMTAayU7sN8haf#Q<2@01hd&&_jSke@ILV{*vc zfT)4}QIciDgkz;{T=$E)?7}VnYC&Lbekg$fpf<9FtE=|9(P_tTOti-t8Ob@l_~A&5 z29w?oGo#@p6v?YLFjT*1b&PBCh2NdQVKmpt;;hFmJ0SN*)gWBU&0v)&p|+8Dy=!PL z3&t(Z6q+3JG@D&BV%ucWzRwUl_M8$)MRzxge-;U!LanK9FQ zXx)t;n24U=l^o6WPpPzWdqW|#g6=cDugJjQo8Y#Rm5703sev2&4k`Y>(_!c} zv}~7tgk4g(YFtj&9?rg@0?xsy;=58>FXtj^yUC_DWHZg#;2jZznE?&~wKpcxw_5)r zWa+FkW{I8o5lo$d43@Q-?$#HQ8Sq3Fz#c9BZZS0-TV4BbxMXMvBY}XbtfD>H!6GVv?*JmgLI#GF5)-P4jy*clFCX`~GTiK7u;>`xv;kw+L9jGQpm7XOBi zdhvFjSQn~(xo_ny-;jTiVc!J**3+ZJn;Sjd5l#<&CDc-2ks3$68toN{oO2V})G>;s z5WioGSbGJM-K^cy2h>-mh@EQGbb}ZmJa&F8l2f&BS=tZYF4J8|dP{5a6(2WL*`q zT@I;gL<`&_oO+|U8l@l?XO{(zklv3I&R;W>Lu>CdX=zSMq0F$M(Ml{@V@|R zETl@|A<1e2O1`G7&6lo%ENkOQ$-8yIyGLbk#+yC3^AvX%q`@`@jU~P0{_OmgbAFC^ z!un=BS?fgs(rOq}^bE`4u_*3#FnA{P+YLBb_u^OQz}2SbZX86U$z`FODv{Hbo&vT*4tWqaUSkxw*H;C3J${jfn9U z+^yz4vI~~SgEkT;-xOxXjAx#( z((+$Bwj{!3;l2>wFd*s76CR|LVA=uSQ8mT3sNvDU8lWg3=6(q=($xuc@>u3Z_OS|?NRhs9y|jU<5L4rwr-u zSt=E(=#+nzrg`48oY!BeWV^CpjyjYsdop|+`^J5>nFh^Mz~mM~qwKyLN<$b1qiM#D z2g#+LI#j&of2l5>Am=|#cZ14c4!1OT18oT+S6e`{>ksDn{?vHPHNf1zQlv>1(L6G_ zKW?Y}iYiJ3Z>NI8>WOUP9%!ojMIzFJPc( zx!o7MaT!H@S7sbDvAh^}Rug$g=xn_-F@846-vA31+kHmXHkcmQ_4@lnp^)u~3)M*= z2`32xcyDWv2ABX|u^`$ic16P06$1{I18mCgccT$lk5Rk+^%Bd@B95p7tc-E!^YRb80e1_3{zj~23gq4vi+>E9=v!uy*C`W{X172V(jAx4a1gk zJ~JQ{dBB!DaQEHbm(6s&y}g^0BUDsKpfL2#STEf;h%Ae3tk!}SyyC^`H>6(RbC%~Z z%@)34I`uvqgUP*|X(DRatl_+suHp46SAQ=ON+{u!x9Fw!6~;iCnvhN@am zhS)2rz>0rc{OGv0&E^kwy1$0<+KK}He|NBu_T!>7c+$+r{QIgjW!b={%xv%Jclzp7<{Cj1)8GBA!^Oo;7V9S@hF}*mM}&=6Ax~ z(99KbyrjLuj@?J$q?fs_DiwGrJxP!QrC?%HlTz?z_W6{RY$6U5&%{)@c4xx)*`K9m zWgNiz$}`h(w&ps7u*OeDx3Su{b9h+)v->O2MXo}Oeld+xAsqu5r!4NSbfXSj94v8vG|}WtU-rqK6Rl-+C=O-%nIDkvKl_pN(nt%uagyR!A{)f6#&OA5thvxurxfmGNelK9aUWk&2X1d-NH;52X z{I&mwk6@^6hl~pyvE@+3W?d3|fv)698&Ugy46@4!gQ=qQDcUSqG^oZ0MpIpe^P@w+ zF}>EgE%`V`kD8TXavWhRK0-K@K$rrf(&U1y+iip0#YgT)4uq(;SncR}qt2*E8KK_O zlidTpA%~zHO*?{;6m^a?Rzsd#7^M-hw{ZKh)(;WI9{xzwmcqErJec3Uj*MBS5EPnD zU>*o}W15dT^mxq=S@3?X2yxQ8MgQMT3D#`4tg2x5#mwtArk5WIDT+TST)bRkBdHM& z`f9%`2jmYbq$5UNa;P_@bS5pdScpItIij2LmCu``yg`=uq%(<$tR}jN?n>#kg60U zE4q@E>Id;)%)hjGPN%*`2}jSE#qa5 zBF3Prs$}P*+qfG$1qoB270W651E#hZfcUn_8*2suvz7M?l!j7a#n%B3o<`@QYKJ)NvsHwV&pFaW=S^~SdYu)j(BmZUbiVP#+ zbT}fg-aU>l2>(9#6)(MdW29-^?kpD`8x;M)K>4zQ;9 zxD%GRznqGWRV+3 zr%_x{^>72zTL*T7`!#r}@>qfxP3n>2#mZ1f|LZCk0rY8NN!NgbB@I-AkEW!#>G$nz z0n>@#Cx$D->;!xa%+q5nDV_+^jD_}c$QNQv&Rj$WEzWW7A9S>7`iyh3JlNkNeC?#M z?De5@l|B9eMX6gvcKROfi(EgB8(D3*;>}Cpm@e^-A509EIs7Q?Bx78K6pl@|ScO)O zS+sLiKYoDMnmi0SY-M{&zjSx>#6X1DL>6U!(#4W~vQWVU4*k>0#dCu^vvS)huRk>_ zlHVvBiVQgS8RoiO(ONuA(yWV3Acq>1T8q@Ok`?|m8`IK~9?}7%6iecJOrU_$*gxP` zSK6&m^$9G@Wh{6B_(~tkF{&-!YDr5h=e*nP-B=h*hABO~(YHY0y5j(DT~|ULmaU~= z1L>999PoA1?-E|k1h?{(EEq1u^USpN3embv<#JngK`y8?d}e`+n~@Qt&-~Z?7wvCm z0W5i99oyt9Im1rDwA=X0o^1XvXXuf16S8HUwgg0fs{3a!@&(l*wZ5lZn7IZb)7Vyg zCAchZFGe=@vdEO2M(uc7}cD{ak)CQXW{B%H&PVgls9cs@^ok*nFW=P{NuDhey6aAeARrt^gO z$?k4wNYZrN#7?jPLK@Q8?XjO`!*{?ej>dwG4KyR)M-iP42Ra;BEZMooT*L;@@%vWu zs6vnTs!lQ|tRz52fz@K|kzKI$F|jGiLr^LyGQnvF*9J*i2gSoZ`hEAXw0|^Ts9y*lVY_^%iu7tAc3u`=KZdMKU6p0 zvZF27sI{Bkd7zSiGFT?za|*`Qx)~!tJGby#a#Bu8o1B>RM{jSeL|=MR=*)^ZI8#=EPC*oE zQYTzxaPT4=!k8KA-Y4Y-;rrDlvW*uDna031M2!y@gr)N36h>RzrdhG1To4h>y$N%o zN$-cTZ7O_UG(n|h+snVj zg2|Cspdq~K=y&4VHkg#3DSoCOP(imk8wrwA#fO*S1;#aAR0i53%G+~7c5k~b}*wVc7;X{F)wT2o6y zp0X+V$sI}(!6q{EBN?j_+oPDFu@}QUK~K880%avCG0lp@u%TU5qfM8u_BHs!u!u+? z6*7;Jql>u0z16Ir19rjaHy_31Sn-F0(Pb?Hy9=6sK~2;0{{yM1Wan%0BSmKx94%Sx zmD7wA^LK|1JKNAm#+;}eZ(Oc(L$&OQl|fl-2-r8`h%%H+clurD%Kk8;FZOok>~8x# z?Bd?418V)4Qi!<#w9jyp(K8l7FfXI=w0t{fW+n-!WEi77rxmgOqs$v&|AqS|y=S`Y zB@3JXV^?DUSK%PiIKZ8*g03V+BsNO+XhTtb27x7?KN%bPZ6;J6yqpV(Ij;JEhwyZA z0x89lJ+r-g`&XmZ6tcNfC7}HXAAS=n6;-g2bdFnpl?|T447yv*xSD*V$w-{0SpNM5 z+Jsl=F<`~O#}x{_yC)M>Qd{*Tp}`i+;g@Hr0jUnR+ZfwL7gCa5;aX#_PYs#Ih2w`> zSCq=m8Q+fRx*L|`zMMAmNuPexWUNq?a@kMPU>JV=DiUxEpIZ%Zr9~$5OS3c&h?85i zU4J87eE}P2Ys%T?r>H66FJBx+kcbT%dVL17JYJ~JTp0?}P==fdk~zBLfu2SKSQum| zW?Ld6|XPGFpRs`T_6z_WvUFHLqVs_^dK{ufKIG68C_U0 z5M)AMuPeldbIL%KR8_%A>zP)1f*Z)pm75_MuO8<-5PJAnYTTYWAQ9G*-Rg6(vWU{V z-)XVH^=Tqk#!HWav{P1nN~`63BVNrDu^=^HcRa3;Pq4`aigK=|XwbU6)lxuNOzI>3&|Vqsj_VicT_bLX)yQN3jEwn$B(;OV6NCLttvQ60xn{Ap(brmlZ%IrI zq!YQic`&u4o#cAwlid8lU#ojm{ZzB(eZCfg7(0y^5f>ASfAT`j%Dge;xp+LQ58PRN zrt7&SWk;Q{gKo{-j9g3CZ1i)QWFsltnDMM{58RD6E{wXC#WFW#$=Dlx^0Bl@{M6`AXxHU< zOR74rLUQVw!Z$?__KSc|^V$loKXZ!zo-UI$x!c-u6Vg)Q##xxHoOWHWono&GMq}$I zImQ~5wcDVy?w6%tE0(;r(5WPqe z=OrT*ORHGrV#msVa?dI|(n*ET=A2zB}BhT55@<~qY6)M zv>1^JECaj61CNL?POV%O85@z^+MF~#>hs*G6a67Dub|5<_>bc?k7;i@^kUg`zoPRz zN4C=N&0Ev(mH8r5ZTis`P*L#CpSa$Om!{t!1#1RiiyX*8&iEY4-n6V|m&miF$5g!pH{nQ`HQ%yu= zN#|U=ow+DkzKrUG2ka1_E}2_?gh++RnT`nAwBw4tp3=k{2YoJ)r){Q&^IU%rDR<8k z&9lI>jPCQt2gfZ#U-L4z)4N8xd>^4ju9@o{@rT`}5rjcN+gW7g-ZKhXcf(hl=S~o; z@nK>1ei!4j+1qwh;~1GIeadP6_C76{RRlC#krRz!u+R`992r&niCmAz_HNhP(QCEW z@$EyEfe-K;!ldTrMX;C9lS@R@smuES#Z5(vi4F&fZ;#BO!6}3ee8#;gNQ42e8ozxt zMa=cr$`sG}Q}5~wD4fI*yqe%5KAd9MiLBjJT*TpVu{@mby4rg6S>qlkq_flxOk*70 zhwv{MaCqUxlD!KNVdd&N?Q92o~?Rsu(n*xwti} z>2&8FLKFBS3219KhpH6>!*96Z!$6B+%GYcSGe+)wv0R?4s`RC}1Ld>7jc@7*mo{`qB_7suI(Iz`&S62y43+%Xx8iHhNlzJD+gWjc>@9gFO1 zITkw!2#Xs#)Uac~a_WxHOB9+|gZD~0Yru*Lr_b~u!9%6*yJ$}?+GO3@e)dZGe%4>N z;$nqjO8$w45CE3YATzJ+{w|~}JkbRoBp)*(NwuIXbr)~I-+}LZ;SuTVn=bNl|7nHZ zh%<+T(jMksC0vG6YHTm3tqnwPl=Uo--sva-4HZYl8kQ?7Y2doBKzn zb;tDjj8#4RMFqo9$spy8WF559awQb?P8pc4!&=;X9NUtADn!vv;9e zI1a7d-?8p3LMQy3DPQqMbv2=Q*XEhVEzmRQ2p8I?J>U;#gCRGJ^~#6yDCogP4%wYa z81!Fz+P!|!Iq#Q(R(pr8`%_C-qm4Ih&Qb7O3yF;;{H+FjaWD}xJt0!K)18Tc?k;y6h?^xB0NMAY( zPq?@bW6eRwvAGP#53kjaL5-ZliNV)cZhBfb*Q<&W^Vs-4@lJz?pOZ#FcmQVga*=@a2j`a-uc;^M*cCnSY7R}*h3N4Anzl}$!_DP3~_Zz_}kK(W=xy8rl~nJlXt$5G@8qS z=TE4=e1h)UqdGS?#{`K#wOi}r(l#>1?lMj6SfpFJHY^$W$Df?oO-H`H1)0_3LqO~2 zqos;@iP0{4o}DhbAB>bHh_|j1pgo3ny0q1l5F?JGE3{ILf&q6^d4mx29o_bl=d}tD z8?&ds?L`8}=sxue19I|jHbk#xmx`V=1TRd08@i*b{baSl{@RI#dv0yYa#;aR0vEb*}{%zp_KXM9z4dFB|KnCh!^)p)+} ztX^4{y3=eNZE?TQHT{c5T2^WOAw|7C1u!}@)nAzf+C)PN2jCrpBTAOFnm5UQ_eEhmE(Z5*vGOYmSeB^r)<* zCuBnu#)DQDJXAvcR~7scBmL2%rFCv&QJ&7OXm!2B$|B`^?$xPu_I85szi8V$DWfy8{bkuO{Q8xrY-6mG1C|w^&WjI{ z3QArCF^dcrn$eQ7GGa$++zl!^BRYn);0AD8V90!7eSOXubX+^K?QLx?X=?bdN-a~& z(C7XV&;nSATA{Yz&Zp^{Rb*mTw_bN3a4KTv&bhnTn!xW#3R+$? z(JasrH_@_ftzl2YR!L0dFl=R*;?MSn@ZovA@OMz=LGft~h zz?M?0amY@$u{606NeA8sQ`vMy4Snqmc{)G~EfX`7LW>}ftw53<0(NsQF3tbO6LY6d z&d?T3mN%ajo#)1_T^&fGH!jq$HF@sm-k6d9J^Lj9jRZj0x49syZv6pxPANc}L)98G zfMEq$zF5tWRA=7t2~^ZW)| zY41Rd8SmNv(G)L0n#P9G#FXQoU$jC=YRf()4lU2)?RQ`NRM@4hb=Wu2{`%m`Pmr=h zPK?md2pIHd&I9BcJM=(cf9B{EWq>t$Fwj$+FrCpZp)(h|8Cb!xv)B5|P%FVA?@M&U`lg8GSGWUrpe4aloqf*^= zF=vf0lS`OU43Qq$T$goU6N{LU^!X8cZiw5K7YE{-9WcGWA0CtS+D?dCnZ+rk8d8kJ z-2HVB)c1pFU9n@T2js9)r0JDv zM@NTCyMaWp>PwIaDk^F&V#b7|rX~(rR9K^=th{_91nBcH-%>YsvVkc9qG3~X(}%{4 z{Li_5R#ov4Jv~2151B!@Nt9>_<9A$m+nz5v+aEUkF>>qEf6421P5>==;(=8Zk7pd6 zt=D^DcM7j#ixzf}l)?YvXe)pR8Exo3Vy@$4033B_3d-`IP`lb2vRs#@C|0AlV(m}= z{9@8;`UcQ6G>i{kw$N(1buIh9{Uv~Ak7<{ndNAp9>i;1109SQOLXjMe!DEQxM*(gX zdw6$oA|xc_#bU{b{rQstA_?o*gnz^!Z+|_?GsY_^uYMv3ukHlW?6(?x zDU1C7QK$G~#BbqqY`Z^8Eu?~j$3uK5LA;k^sJ@vl5@nEfV6$_xVi!y<94 zTM>ch3m)+7%sok%DDkXmWo*=}(f`v6KjktHMWbD`v!VANX2<+jO8+-j;!v!AyNv(m z7h9Z@AwiP~Xw#_&prQ_@Y8f(XInUDh4H9zPSl6<-`FT}r*pRK1-~Z&=o2;Xan&WM+ zp-4A4Mjre1{gZynSs&R}c(thkdl|mCI&Yd}-76bW>qVJ>9px_@gkrhIx>E+}p|Ayq z&Ay&U76pBZA)pk1YGeYQ-5mPN_Zbzmtk$3 zomrfoqiRtA(UP|+yCofILUcsuO0ySrls|8OhV)$p1BVb?dBeBcmU`<$r4#-yK8+I< zKba_2O5HV>L~jba&v<8L6dsa|*;B7`u~8JaZ<#{Rqn2lD*n2q0#5M6Pve@;|9V{RE zt>9eLb@)PKLa|X`|F?MheQ8K{uJWfGs%*<$CQIW+?_3&uj2UxU9EQk74|WK|qZiLD z=BFVvUA`IXIa01eY%zdt8zjhVYP%{^ycNDjux4;Q@#w-E{t zj|kQVpV^s?jDo%Ga?Q7g)IuL#ngw-Wk%*vDpcKIvOMGzyWom|Y@-aKFZb^Ao>5R~1 z1CRlnrKG1_W3t<}vgW(!lN?9xul*kFd$UQAt|<{BF#K2Z6#sc&Ad`zCZywQmv0-lo zK1;v_!(IM{%>l_|u5BCk?KB!kr;-}GjIP?aF`d2f&}ARCeHr_|v-R-1jmWCj)FID| zV^M$--22_4s|~UBoU03ovdD|S-A8S8uHxLj8p{_@G^z#mzPUCw`Lt+xfy+3ZQe<3m&>s(#zaWCTbEJ z48|7y_l8UXFYHFX{$y`kFiN;iwX~Vs$pSs^&W7&jwOB=rM z#Ww~`=LXY2<*sP~N`}nT<>4IprDQ>{*MS0Af_|(@=Lux9dacQV?+3rae`z}BZi9S7 z+EakjseeqQKGhk@)pb3nwuW`NBLHBOomI5k%$iY;Jy6f}$D`zM{I8mt7Ko6P^r&5% zEYu%SwcE|q5rOyS2|MVsa{1!LwfPPrPS%JJ+s>!Se_yZFX>{VGL}DGNg6K}+xRdK2 zM@TxZlOZ3{Z9(;P3^Ct7Ael(j7m005?1WWjC#*Y3Ym$@3e~N)HlM9nDK$OxIeax+Q z-OfJk(Rl9odz(84ZDSFJ-YZ6v1=;%4^GfsK2j*^7VlQY(D8K>`f^t%6i~i}@*4kf@ zEDbO2^ZH@`v69(xcr>;YcRg4I8ym+EwB>TLOsquA4bMVEr-~Ip`SF*-a#z|x4{ung zqGwYZ16xu2*?M!V_>>S`B(%6X+b+A_svd~O_sH?@%!LqF(?ibk8Mj6RYZFQA#crMn zf%lp#GoLMs(x1%*3F|NHFefXG_B_pwuN9FbhU#@Y7Oww=73HzgGPznSal zy1h3xSu2}Q$7g)M+z+3o;YO!o1MeWq+;zCN3GQhtlBW@Rk=T9yuNB-9OclQV!s6k) z^@~JEhZyU`$x_*x496VVb(K%7F=Y$1bLR*0>!yNyXnbcmQ6UQMYMUcg=6$o5>hkbi z=I#C2x(iS1%C(X}-No?Qxs4UfhnTSzb^rp^<9o%>C>+*Y2BxiEJ_#Fl+Y*i$(L$;V z#HArdMOZSM@_TFzm_5T<+r;Tnh&ER0L7a9?{Uli@W?5wjRh%}0Ka907_|E&cOIJQwd_?5P< zljpeLErBn52UpYXfW6^EW{$~fx;A=U|jq8sRD_f%*ctf?(#4qH@s}P zt?gw56Khxzl!)OooQo#>H|T#>;GdBWy;1xEwYW>@3p&r6Bozvgfc48ppXEs!|CUUC z$_F9G$q%^2AV!jocwV)gi*$JnM_PmYbPEWna{MaLuY7oUSTyx0T*)&@-sXgkA)IX# zL}X{e)!!XepZLa*Foy`k(;GU5h;(^)^@Oh*2z4+G%TzJ8Tg8KVxHAsTWYnAZ>0|#& z*+18(fMV)^s?`HO>?N0h^t4nYyDzCerTsp+khhq`LFX3xra))#Dn@`_hWW7N@)gU@ zjRy4QRis3u6qPVgP|1&kWY~|O!``t&`|n9i`9= z6wHwwdWt63cg7wM_7$ayRS=xAuuuUPpRWSrX%SDV*dlcXz~2l*sx7`5Ip1^}S-yg) zE8;~$z*xh!)Epn%=0FJU=p-&hz)Vov&>e%|o)@0GYhXTDPgE(0B}1Er{W!9K%|L$G zJ{IdsfBKQ=;S4q;*nf;9aaEGaCOE%UDorNYdoGWiiCVqs}bFAW|CAE3WuP;y0> zW6}{A!bu!&2?eAXu>b8RBW*|D&1R%iJ~P{omABOh-21On#p`T~9Y-oZ0IaGw#h9MROyBEW=U1j&8UYxzhH8}c8fgNhXF5Yh7v za1dB56*@wyY-&#S7~gMa6h^&1-~zNnCTD`**;d|f&814p*qS&Xhg=jTr%@IbnA{T-|AS-V$K6#|k8g_*@>$y62Go@+<~b&YkN8!R~a@;jIv8dXsp zpVRD&+CjH7?fjXjt%M_fq03DTijlOx<_i<{$R00pUEL|6-WN3O;9Y487TrGN8Np!z zaB-H8aV%dH*Gu( zkT6hf>9Drimn0G4c)E%YjxTIOW8T4D*#-9}_C_j096d?s>R?c-;Pnop2KL5Sk_Rnh z-9TFwbqbmc*CUt?iJQm|w0-i4N>mB8Q-`FdXW`Q$c9bMQP=*0?ic>?DfM{JB8t6gA zu)!|^EmNYbANzIxvrZ0WDvBfH1^DR*xgyUb!o+^o6jrbo;g-*$(6)%?Ce(aLhouw! zk%Pu4C2951@a`9i1-hwE2Z^R8L8`C^%!<2l-zG_<2JK~xSDoV>?!q`?+^8FejH-qP z$&Oa;w_sN1%X71ZWl#yNOKK4TU1W@{TTo0>qwS*CnkE*0^0rA*(a(x($w>~rw9tvG z*`*y?eE2B1aA=PR=v18YPsm>N*4iYtDNeiZhPzTwMW|Ii!s&hcwrJBbO*w%?tx5b~ zNcOIoKkO-+r_t*FBJM4q;%b&g(I5c^XK=ScLtt=s9o!|jLy%y>gS)$1a0?nN!DR*~ zK(LUIL4qeRI02G7`OZ1_{zv}1-dk_Idu!I3*?aeNcXjpd>gwvME(Rr&dUCw~DgCN; z1yB~0!V-$_oUX1~%u5F^%&2!6HLKWb>tm`pXsUX~pFD-N2MU{q6T*=(F@wvn3?*hc zrr8a}>E*Ge-ii!G7?TVKgA%bkGc%1xf2vQjBj*hnMdY}dO+S^vQ7a88%6?B3|16Ah z>vY51XBD~YD}8$gLkTXMgM@}(c0on?h0#F{&>Fnao`8ObI+%Mr0LbG2TtSCoy1YZj1=edGNVVc$D~S8N;`$E-R~2 zuD0;9nFrpH{Bl%%5Vyr|Td5#ZSv$MVQz?rfC`V%G+~p|FQ~f+e@U3w`mK|FK(iHM9 ziB-hXS;*^&8rHSD)weBsJI&rS+{je2)P*&D<>p2Ch%ZQXU$iW1^FV7QGOwc8Q6<-1 zxNj{Lqmf4>(}5PvUV!&xHIfw0Jht_oGVV~s053V1CNQi23Y^aHhlqbB-&rE9pW3WO zHK!kYmE-&@l%82htL2`+k=9*`;V_1Uwn%0A01Delt(%`wTu@D#p+nd%g8EG%POxNS zh^?S=?yKTW3%obafs2WqXQw=&Q{Qqe!+1RM#+nB-4oIaP{NG{VP2CJNTRJXxA%oU@ z%Qy3VAj$bEo%_{vCTXq9P{JVx9fZHMj<%(Fu6^0-Vdumi^g^^NRaN4*Ppn6PXa3gGZFm(KpYnK^EH z{%9Z+-5~{Hzmg#AJV!U@#!|Om!cMOL?R0+_mPbDj5jE?^^{OGf&O1b<8pY=rPJS&Z zxA1mgyO5awMmrE=n}>F)v?L%dV18;Mo)%tZe@F4^A(bM-UH%`T8xxAb^h?# z{kjjysOP%YUknr%717MbsDL7z3s%4TYQh9GLay3Nc|-2~tJU*|VWdabi*z$qANOf! z!pg-bnrURi8)Dy33oR0!fp32{gOAGjiV7nBC}>;rqlV}89ReZaT(;@~7_q|5_C|V9 zZBy=>F8s<{9}P%Mf%4E477pVPX!3gP_m{js3dFnZRVjT*v0k>cH|3sflBX3q!6)l& zeBzU1Xbac8T9UWog}v7Ca?`)->IjM}jzD#--nQD0uojeaO!pr6I_T%ZKak8~B6ZTc z?&8eQ+)alxIh$WtF?$U(pE~2ecbMZA28s*M6sWw@8{@0+5ujz-;jl6G#NVj&iNWW9 zxo(X8q`d?g%-GH5)@0<`VU8&|Ob|{k+Vu84r=3YBss7@HX_PdnXcX>5k9lNzN}Bwd zZ*gK`_@UURt}f~6oT3q_3?pMG$w(%etL1O}yZ5x}1DEU5RJpMle|XBDQRh*@YkT4I zvaC8w*B~!`#2!YLL^wMT#*_SHSevB$;f+t!BBfvx$(ydsCQWI#H^n}_J_~E}kdMW% zY11D&@1|n51(EB`q0+I#=YSV$(-Vykg>fud+mk%<-#=@fSA;UrC3x^DdjhU#GX3Wm zH632MV_X|wLjWq%WYh=nW$RN+naU2HYB9$#jZ-DZd08_vpD5?m?Hg?u!vq$tKWH{<3DL4Zqz+QXpj*ql@AUNhrf9dB`YEB9s~pg zJXp=R$qkUE>NNPXWTgM1H~LFtWvl(KHU7OL7(`_M(I@|S@~4M}0RIql|1R(T=`*U% zNmby$CGBihdOn0}4OL6jk@J{yt8EGhh#e{7&{#yuN}~Tkb%yu!Decpz#RE|jL?ukK z`ZVku2u8vp@jna6|61o?cbeKrQR&|jJR<(HhD`t8|BB?+Wq-o;{%aX<^#4qh{Xea> z9B<=NU;Gm;`oHG$m!4|7>0S6snWim3Wp8Em=wC2~Kj4lGI8Xv#z)gy@KEMy%Fap;pQ_eDg$C1kj*=@m`Sf~xO}u3v5rN8^xlZ=`$t?ZT}4 z-!6KY|J$T)f91cj|A$=N9*C4EWXb*Gf@ZTj3kBI~Sn_R=uiGHB}T;LxxPE`IK z`F}k5-!`xU{wKRM<2seMXMycCyRx^H7v!9+$M=XVznefYY>lJcZ^7}`4>87H>M4-B zi@0I-v6*iaZE*1l-|RQdrP{QQQX<{fMdai%&{4ISd4nUR3=feZ-wKDK6GLNdb;U!V znZ=x)AV&1B%tWfq9w(b(vcG7^k{8`N+kn^fpBaubZw+7V>GezW!0}+vHrglf?+U;DeQwtba2z*;_6e zzMX$!QIZ#YL6C|CiMY<${;>M+bU^%&0Roge&DpXKJizEOte&?E`B4y0Rc1GD&Wq96Rt%KI_Tsapro!tncec&=l4Xb&4(0@1oIc z-HMS?CI?-bbjlTR@zsLRBxtZ>4Qw>itpmSJe1+O2z3HpE*IWb54Be2ke27~=m)cNU zGXI5`Aj)B81Z-rG3?C>8E$7DH8iXVXKKJ(VQF679t~A=NrxwPcDb~mv=#fin7=4`` zG)ZnsN~5R*j8IFljcnu({Z8l7t|MS_x=7h}eM|n9S@?EeDWSB$?P`MFem>*KZi{Y% z0KfPUU#ow*oJ#_}T}Y{)|3buP5>}Mb$HnFN;b~-ha8B!sgj*ky{NG|DPyi-rvZ~>_2%q!7M1Q7fr(^_FaC!;IxzLnn8CIc?z0`= zPSLuTd8-T@bh_S0jZYo14*1kDvIxA@`Odkb(e6bOH3LeI8D{!AD7Zs>N&dXGTQysG zZ`JN3);PD~t4s>6n5TLB5;+S?L!tX4mTe^q2V|)nox^rF(pLDjC1iDVL>`ZZ{7AmQ zE{3sX(OkV1o`%Xvr`br6$`Uc$Nf^%vDQUx^6MsL#5o!t3xzXR5pn0fMHvvdIXw z!-_9WNOn%*RWEE96qpm0YHBAj-|pL7eS2n)(HUtnyYQ@- z>6>2n&MJP#XU3%uFLzNEDfA{w%9>ejnhiyz%4V@wZVp7Rq7bWtZL}fd)_io<>RdFx zP#^eWqsfl=ySpP;X?+KN77dGxt2>~SR$EwELu*1tHNx|(`G7wM@0? zS6Iv)9GlCC_`@OZ^&0_SprT41;RGjN8Md-}c9IH%ZwtTbNWMvv@z6a0AEvIZOYQD1 z(1h5li{{p>3AfOOA0}qxm$K~YQ5d9%^mu4TK?bCibZHNhQ6vxIzW4D?QRATdtIeZL zb6f6Z2p(H4sfr*>3|AcI?JcOa=?D%uf#C9txkWDn5C;|XD)=jmC>CGlO34PsKK((zD$n1W4Vgn0j>Sz=VJJd<;awiYN+NOLgz$NNw zkcvZ2Ui@9!qp5w*oH?Bn%JB^>lw#@0Tr+rJM%N^ouS=Rm_pEBs3JnIi!5K5L2@p5P z{;Z0ZXh6MUZB&Laqr_1N^>&0Fr7IV(sD(Gx+Mge9F0A)QizNdRqPqd*gh7r!(L4^r@wI`&|IFJV;^MMo+uVCIA-a7<{O%2_lESx2 zZN_f@;{32-7TAyUFEjk}jy3$ehEB%_a%8?LAoE&Z+J^apSV<-6Tmo=jg>oMnY01&fCUUUgPV}f4rb0 z4Q$rn77^v%8eKNlv`xNqT!A0d>Aia6!)j`a7maiS7?Y3nt0056S|Wr)M4G zZUmy-J~Wl+*1zR-Jw*pj&(r_{KtJDLM<1rONNn}G01FE2rD$!UQ-_0&AIrhGD4|@nSExGBOU`~v@j1ypL6DuFA2`>&ElqoW zX|gDbTAe0*H?_4pi?M7nOk=>1Dd>}xO?V?_zPTCbhu#g^a@wL&djI!lxt;9vb%;xae?Sc6{GXFaVpO$4BWw;=)YD4aN70MpR?JyNMgjy{76Y% zpYpzW8Qq|u+dB&6s2=A0s#>${GZ#8CrkIE@!)msjughgG66EFjAP?E)<41o9@Su1} zot04I17poNbmZh$<2+RM;=*`?J+?@NHlnmVqCRJ@f-af2E!1-?TEMz^aGCzVL6+m7 z9aKZJ$B9cD?IAWh>U)zizq?;6yUTJUE+04F z8kcWSGusi&ABgfH7!m~Gy~brH$xMFF1@4GRUSy4Z!}VoWe!Vc3^|G4WC?6m4cJlG0 zJOO`Ts?D2J<|Fl|jbPdw=Nz1@0{Nes>BX-Ij&S(Vee$nmy9~?xH>?CawS%p9cgf>_ z+Hso~(P`IS<%^WvGEMV51r`XbN=w-a{LonqjqYq!VFk(zj~UF1?ltbE^<2gUB}#!T z+S!U|-OgMG)Llf9)5WVrDLzQ9+K2i?y0nLgE%7Jpy@Xarrs?0;1uQRGP}Mx$`U;Bs`%o_DFUTv88+YeZ&)ufn>zOs%C7Qn=iaG2@tlAPu(Lyk&WKKTyL<3qGU zW#Qb`LY7WH)0+y2F3Mw507I0F3b2if`FxK;hi9)29o4Kf*SR=%wb=o28ytno`Nk{0 zJrGZtSYqgrQpL2hOV|aRk^dLjoOGz_;sv2v6u2;;#OToVbN`gCSX@+MDL*F7I<(Vm zg9wX}aJcf7p;ebP5#kOj#z$=L@oa(o4kke!KH#jdwAH*L9<;^&g{!JH{OQc;o+rh1 zA&KVfsRQ4*L*(aKK=q0mh$?bObgZ4ze#T}io~TT|)7XanwydqH1M_P z2Zb``sVY*eBxGURW2A=6B-ioIw4=se5(kIyq?gSq_9;|J%H6_=oav1sx8Wi}op>T- zea|vC&~1=Qu#e?0ODZ_$z~0PImiOjcW!l!h_IM^hEXLR4d>IrvtM>)p&rx3c1DluZ ze%nVUZo0WQYw%CHpSdkFAz;HbC`b9dv`6p>+q~9=j-;j1}+B%#c3BE2`-)QqW3>73RHayzxR2693_#R zXB)6AU-$$kC*Z#@C2eCPb|9-rq6*L~CPbvsr{ZoqL?7#f=UX;;diqXJ<;LRndg_MA zK1C`z8^?G;UZtkJ92#0?z59kT?~@>Vn|9b8$$aZSASyBpZIykLBD&@Jo011xQVxM` z@qPqh82|i4irzwPzS(~5J}|cg$U>?kBE44qndZEUo@-J3i>-G<7_laBi8TvfHS774 z`@<1Ir9OYTK&PFJNp@!(%u6v3W08&Y>XbNR-E+ky?ViEKPh36CafOMeCA zcs0a&pD^aPj9%kWv-sk3V?l`(UgAwr>3n`QWaz8{N|)g;q{&g!ze`2x9mgtPO)gJq zOrj_S zG5L>(iAc_3n_%yvx{Q!viWk!8vsG3DMaW5g)mXUQA0mRzW#84nBtmaTusR#s7yd9@=Y~7#KJ(O)UWqoR-rN{RKd8|)D^QfspMF~=ilJhr%&5)2s(btHbAW%TOwA<18ql%D0#7YV zF?G#-uI@ixt|gZ)iv@t>$exCzak=6|!IRysGw1xC)P_^&G|M~xc0qC}0{1st9Syzr zXB-BLYJ8M7{(&lW%iHrKrcuXybN!#}%c4~jBm}(Iv+AGHNpwSuFWBkb7jxv)HIxxr zT+_bcZ*omPCOtanjvcnh_6$wpL0P56x9EqLS*NRxdIatrmlWGd-3K*Q+4aBh_Z>aY zg^3KI6Qz!sbgH-So!%g@1E0Whd9L$a+RTOr^|*d`f-ats{U30(g4xT-E8sU|%C{2Vzp)dcv~e-2F+@diJsEF{ z0s%k}xg7gx*}}*-HFkx-Iv@1+uO=JLIEsTpN503&ZeK)IBr^NF$g2HX?e(nEkcSb< z`l}^%nZIz!btwDOhGkB@3LLb9*+MAu0p{rO@f8r!@L zM?9o#Y=y!SpFsXTxwOUBMdRfJHTxshe~(Dev0Dv8=QU#XE4p&Yqa4)`0jVuG@AtYzymB%QOoW9 z>S^+{rAms&@0Cz_x$vuvn)`Fv(lOY;F7)lJUD~(O-`7Os#W-18LZtlD52-(ZMkp@X z3k`Fkli3iZW}-7JvwK&KXC@(c1Ls&n;u(|E9R#$BYgCX{a2~AYyZku2f30^3&p~ilYjd=0Qz!Yv=y!UcX7|*|_rs#^}UH^uNB# z>spW=qrkMJY+pF$`q;#)$;F?-jp&Jt!iIQ%-)LceJd#{EM`?2ZCt~Yx35*A785@Q`4p$M)b8~&`EBm~ z7%2oraDuDfK1B2x-ErMGa?tHr{owum{R+1&?AHOKX|&L{0y)Hf(P=W~=K=upqxO*T zn!@)qON&tfY74JM9k+J8Nn;wEB{CpYhK~ECwh(I1w*jM|m&K{e4j=5SJZ2Z0EzUUK zLu0z+ez^`FDu+%bXOTDJ_)7mSI!?Pzko=+3yrOMC#4@#?@nr)V$CClT;QblktNC;K zYww(@Slr-d`P;dbAn~l4SGWY%lu+TD%RbWkdMjg|wpYfRjbCGf*E45d6415N zTr)I2wvdu^;QO&nsBkLZCOv!kK6FG*zxwOyo?kStTGvT?9xg+RGpX3ORf*QE8xJ>`6QvvY4*Dq>H(UQ3p$CsE26& zuo=`hhPevXb>i`LE&4oHrZHl1mkd@H+ehJgemA%F>5H{ct0qfs#ooZ|@0%SePB8lm zM4@lWb+O=K+rz={nI@#(uuXfI^2h90xkN3O#};L;`&N@}d{8|WBl{2^N;&<#01 zjt_UZsEe8lrJlvDy}zIFLWMkf0G7V$fmwu6JP}j9xLTI!kz}AVg2-n z?Q1tI%{%@$zcIwt@T@)VD-$^e7Z}sf*z?M*+q(tCfX*6HmmWrj`@eQ{|jVVtkzjZxGasJfK5wetHktcA(CQ}gD#qiOHFv!}%g zulKpp=Mq5q%*6OL1NL9baE-)YN|_x@~LC{L?YYgoU3-q1LQ;k4JkRr!5; zHqsO%&xw2sUk%MJ(>0WGp$T>Rg=>YpBrdf{3qnNAd`=7cy*{%Z`F(Srq;!Nq zC$kDXC-|Lm*^R21+x+{$a>5}|qJLmm-LsUDzp>fj!|j2|gIHnd15YQqw3UfaJeJk6 zf5*$Htlk+OdY&@0%ppwQI^C- zbwX;<{-!NutHc1hO5Df!d{27bH|6n>@qMu^!&r}JP~>eLp>&y+=kE%VtI@I*=Y=ER zx(=9e^Js_7cfP-GfD1uZwz7h)D0yi#61qn)} zEkQ+AKFR2iQA^Epq7}(;R4ztFK(YtZICVzKTm_SQ;gE;S3ouk{ecQx*^9?Vvt=vg@ zyr6L8^b!js;y1hE(uZCPZb(7um^eLrm25kt(04$qD)n6N(oCdl zu#px#UB&F`ChX*w+YNeLK1#ZA7BKHx7|)A=YWDaUcx-Zk1_jT<6*_{W>5VDzbtxd& zG2KoqMW0L0Su+m(h~&_gseuwzCiR7h08b!AjO^v9%d=kle95HO(^evk>Db=g19OWr zQi!xt57+1*9BT=_P!$z_AEa#2LW{9PW-IUU_Bx|?lv=4dQB;8Aa@!}CbJ1Bk1CIB5 znu>mA_3fA?K$wo^|607Z}WPIy>Mt!DD5XyO4`$klnWZBXou*76UNLfH! zUP-AW_kkZh7gvm#xI?F!`ozj#cREIYBW+~58r2S`*JwHcF0^g(K~;#NSQyHvwM1$!zQRU>3|8Ip%KNwNAa7$v?0mRZqXXTht*kPP)n1?Fybx1Vc!KE9)Gxl+db%KM;SGkNJ$JP46!OEmKO{=m&WjUP zyA2-n8Z6&%f-CIBsR3VYP$pxVT|g;ji7dO$dj%c9LOP$^?iYz7u7oUKJ!BjhV}&wI zL;DmX=t5L3i4$WuihAKa#f(%J#7WIw(nLyc6E1!-L(2!x9czLc_<2t+#Bc@DKm{N@@&OSnE)k z@L;J0GXum-x12*4C7ZSRb{I5Fhu>9dEnI{3^FhJrBn|hDFt}7zfKK*_!|j=2rFC%8K6Dm!t@D5x6$1_@3Ax(X9#JOg~YBZbK8O0&j5E4zY!OQ;Rhwb^ZF6AJ* z^lqudSKS1DnLVkG(q;K!HbZG*=?Na0Hq znRU`g< zLvLj=aJft$uWw41!efy;M7{Plc|y-bMkaM5{BmRwz>lP6_l=Me2L5)OztsBU)DhCp zZJw!Q#FQiO!ns`7GsY~{u?T}x;RI{OSLkXtd6Uzb7eP163mGy53WR2E#T!_PI%%Z8 zB#?Wf_faFqlzp36BVK9Ik=7=8)gz@lA|fe~^%Gb_egXziPgnKCuBcIi>Ki8YIqvk+ z^Bh}Xm-u~?S>=YMom2^9^*5*G0<)w;Gxgrjljt{$IE#F;c)VS2qjLQWqvgO<3~9YH zwkM+5Uh$cITsc$>k^|67m&7RjUm#dGl4=B1`Q~{;HK)leQE5b)spS=A5a7UEC!RXX zpqWFsip;jD8rDTxs%QR*KQj@Kap=~$u;+krCLC%rjH5|ROwd|~oA3W9{smWf`B74# z290PK7y8!)L4p%%L5Zn{yTsaznQnufBcsPO`lzxzKsW_4NKz%xH$N}+r14RPgO}9= zI*2m<>ozicOcK{ruyT4j3jii?iB+=&)q&__N3^TZcb*n21AL% z!%vKJh;#(KHO2S_c+q=Gp*Ps^@rOQbA=i&_QE4|QFiP7kMxks*Vgd$4GK=;lc(@)x zJEqJ@wCB;F?p9R2;#4-83t;DqoIn>$v+`w()cPhamw`~hcT*d#PzzRGI316;E>07) zSybabhBn^opzZ<6zL~~u zw+@lP+uu<`HPK1GeI+CW%s^WF#DEw%_)?rv@ujWSMRJfY5yMRkW-2ElhRosHvOMaN zp6V@A5F4|8Xx`XmkeO*5!}(KLaNlsGvNdNF>R1I6vocGG1sp1ezTS4qwreu!AK(aB z+h(h!>SaVf86fI-Ry;I5C>98<39x8K`f#gLkrou2@L6X|kD)+gqUj23{?Kxh_BA2V zhi`&I1OUQb<{P@D$XGuKy2tARt?ErQuMa^B-dq|7y^DKHK?;Q{uGBChC%X6-02u?8 z#@0yTo=MEq2>n^LtdvAtT7vj|7WdD_Y$=HinphWB8DmyJnnFa2a(nRb@(>-Y4wpKl zA7;HOwv8OchA+m)-=^E;Q3$)A0Ye0CTKbDdJa@S6C77rK4XQS0cxmTnnhwmFD>*rhfv0AI>mr!#pZgybS7%ZPx@i0ibpd3{P)Ym1u< zero~X^liHRfUQh$Iwiq?@=!xS+A+%W;DRJ>&j}$|5-z4jW*!A_{(`A4|FIBPp)W$z zl4b{@QH90w>YhU`Y^BE9yjEe$IRk~#1Dwno!{GtNmkLmKlo&!mw_3u#U3mjU+&W8q zB-*6lKLDe8Gy*CR=q7p;=aSE(d^qJ83oB~#j9&}r4A2MgjXqgl4NR-crxZXDVXa6< zX;bg2V5T&WsZO3_j0NZ`ktIfmO{6;1LN z;9>PosgGP2t?Z1kU<`2HgJf79{y6HfZ#a(y&6atdd@BrBnc1~hB}|frzaY;ah+;KJ z;6x`t8AvXFT*m-}t8?$C4icNaq>HJL-r;GX4i`V;PSi&O;d`mh{hfdxoVa?Z_rcnsk!HAfG;Zmy`q zmmgMl+)^=Xqr_7xbIK#{U5U~@{dtd$%6aqfS&ydQzip&_FGMQ1Md`zlJSi&VoVPfh z2_?mikB$;cgBcih?=(1$k%~UsGPW^A?z2T{XKj5El&6WM7{;Furc2U!&|^TCE@Nk> zi$UX|>+)YiYJ*1{Y{Gdmc9pye z1Q%_*JVytDEy`$4%j!9+pJtX!FyJ6%xfppah@t^*2zR4q;khcBXwT@Rdfbb#3MtL2 zIgXV=kUYmP5=L~30nGPRgeg^!$M%Cm0#FLpeVg&pDay-(b$G7-O!+HbrR%B>ec&z0 ziu&3%`D>df;%lZy>z(^}T)ili>z=1$j@>1*ExuDWAH=p2o6m~a^c0x7yX#qtzDw*a z^$bU9{UFk_k4ihek*mz$MP&=XV8*-;hEZN~(7Wp$v)#fsU&*~o0=r23z7R$GMG`sx zAw2Do>$^S__~W1m4xehWCSLx52>bk^{(Irh^T)b+919_4W|G){OtcWhIwaER;N$Nn zuZ`t_veIoWyh(IDa{0y<2P|2lDuk2JBbVjJg zgG(y|u`mRYk=+E9&9WxwX}3)CmFPMq=!$uDb8ed%v7&}SwtQ>FwhL!`Av4Hb#Lc5V zRyBgqq06_a4TG(?+|OQ0;6Z-1|ka86+W>K->eyT=!FDtqn7PJp1a0CngtG1*<&BA!0Xo{ND?l#J_b@7N< zq|6o>FTI(^rHN9hVMW51JwZ(G1r|%EO`JD`c&7z}!s4+74+U-C91%Le_%0A*3mcl) z7hZ&2hDX!%{FH6)6NrGb9RI+Gh5X*H5&Ht3Qt$d%Prp|f=AIOJWC`Wv?0nLy836m; z`6R6K(SelsglnE)j#wreDai3PS*0*h5g{GtRc15XMU9nO-8g#bDl2j1wZh2Q`v+a1B_}HhExoS_) zd->yEOZO}Dk?t6@s;o5=UyFbAQhdhDlQebZucG|tF)na&@)DUz(l}P2?bpocovLaAkWp}EN;kwZY^h&9F*efcs zL#VXhjOI;Fpu`GB6a2bj+_b|E5?0>)s4TW+Z7livBVBv1Sh7<3n@r+_cd8PV;Q!UC zmmxCyp1YYLQd0v=G%D>>*h~`<+RtND+)Mjh>71>x*ADfhR;xu+&*ouApiCmwo>vLH z(P_IMBqkOF)FXPo(yUjmH0i$dAL&%m$EmqEW?$I`=6j-T6f$?a}nj=qZxHV60k?TMk3Ch^j3V^PjjoV}W@ zotRy8d>ef9*&@CIkckv$b$xmfSb7s!rekhx@zkJRPG}3X!4Wg6DReMlIu!`agNA83 zsxXu>t3^CfQ`w(jaepIt^JSJO_Lib6x7s9$#zR*G4r);ye&S?=O=CxG_d~+L!69L! zPY;`ocNa+aE?c51XXO4&VB=P=bFPg>Sztjn-=agI$33l|Oc{t4B6E(6%r3enwY9I$ zMypL(M6dHB1yE(Rz>QckdhfsTqqX3a*~FGrh`k_%6OqDS-g}qyy?SMD=nt2mws*fkoma)3b#OT7Nj=OKR{Z&3`h{+U+FlbRlRc z)eHTkANIlo`sjTCRHs$-1#1e3CP}tb^M$%PDj?B%K=}m?tB81NSFoH=XoNtaLW$1= z{jsM*_3ch1ZYcNrph@?uqHj@{?V(U>+6>_u0OtoHFy+<31+3PBplx)CETKIOxAo&E3C%dvhQktM)e=vOOA`ALq zaEc|IFk%88@dKQsb-iVPp6pdp%@!f)?x9b~)C<3u3%I&u)!ijp;jTkL0gmo z%zE0s#3*I;K(sqta=H$3{)_L*XF;k@K}Gu7rs-N4R~ zt3@1%HW?fZqI01eTLH=8`{1PSpXuf}#o9odIJN#SlMx<|5>lN+R2T=was-r(H>t7H zKk`vG5dYD=lo+(K5?`#+bT!RwR>AHDol0BV*JQ|g9c-V$lV?1RpnTTR!unbnb~5F> zxV*&6H-_wFiW79LJ$0Urw%N&rD{FxrbBhl61_f3?o6B7>gh$k12WdfJki@HJQ0BEH z+U#4GE=ZKby@)KsyK(5xurJ5lTAD9GQ;#eosD*Q}{3WKFUVuMuY>%B&)wh6`z(>L> zkQ<-3N5Xl>$LGfo+UDoHLXvjNLV3v?9~+)FG%`)l(!o3rUhSl66A(vU+8Y4n@60M}}v(=|6 z_v|x;JD;F@>vy94A%Q8-sCb~2^EMSEHKy3YzG7fzXqHvcz^Z^R{+lV6#%)V5^+gh` zuTW|E5ns$EVMz}(kQF@{L%K4}Oud5cN5${o$3vJ!*_Yw_CnxK(LC8;jK7ZdFxcGI0 zfDVbyaUUD&@+gI0JL~?;k1t;5zwNFaPQJKy*1XdpF(qjoO+bxcH#h|I4}znu1En8& zzt73AU*(N9Y@GzC@&PEYx9=$x)f*ERTDszsU(we30!;~YOf3~vu8FvsSxP=l%xX|0 zm3b-3O3J&5vyica)xLE1Xn8Xlm@Fc?HcuiQwRGCr3!_RjG-hf?zsSzoYIH25|;S3$zw?nwyH0O;qZpndoY{=}>ubLf<>vkr`^! zYqbU4iPN02i?k(Fk0+g1?&;`>PG3o$wA3vn9_NV!l3iYYtK^_zyB+6guyS%`GZi9u z?ZO9HWEyb5uIgb-(tg^3_;S3~8qaMGgH)aF8u3<15R^(xE9>kKGBPVFpBt*b6yl#; zNWZ!FY$S>Ng4wc|$gXnk$)tFmre~~hH0vJ7OuA#|nC9wk8_U6a{n1PAfX=r|2q-M}C6WC0WFw+c!`8-{H6dznFEU*`oFnsT!S%7@UG*!Kgoq=t+F$CdAGNAQG~_jJ zRkT)C7vCmUFT`i%|8m?m}z^w9aPF{kN4T7If>3sZkak}#h>*ZzkN5kB< zR%>x;Ogw!PHh#8ZfAmTj?vAA@SGU#aYHptT!ePU<+G4(NTprX?EnQ8`ltg5(bjP`u z=9)Y2xRraFrsB<3zA5kw-1D_N%xq~f0^^bA3OhSH2U9ExAdevU>^&!W566^-r*7L6 zLbed-)!)7H8sYr+{({k-O_@x<6q_DSr9qj_Ero4XZiWWB-gl~;B?{+v^NkcldQkAa z6L#!c9s0aK4JIiw#VGT+RFiFHoKlNEeMmE=$rfRuzeqxzKZKtII8OmFVwQy47ZoP~ zA@6QbQ6%`fn-m3TVf`DFzhw(xFrK7-P0ljmN*b?nX7L1f9{!eL)49^p?IXH_0C&NQRp*ZC(I z505?uDUO`pA=rVXn7&NY#-F#$L6jQTo7VwpH?%8^M0lBCS~H03^{vMYdVzac4?u~S zJ4QN8d6STQT^*Rn(n3QHF+>CkHrfG=&WSHLj3-3q{edI|1=tRgENOHyC}vL-1;|E_ zEw=L+N!`>zDO3c>v<>6*~|vHd~R5dllAK8VG+kr_*Abt2H3rT8f*lT9BFqh0ESUj7C znT(+X1qzx520syVjt`&`6X~*^luXoNcmL~A!SG3U-h@c2KM?j9*;uNO-hyfCCPfJ< z7Hct;wghEQURu)7g7C?`qNB7VWrzrsa|lnysAr0X1R0}sLuD1^c#}LI9dmk{M|zqd z?ZBpDHp}3cDr?*r@oH04};jfZX-rzBQ7QG}qSGNExh@M&_DjC#su81%-_H?wX(<0zjO zX(Jl_fXuF!lnmAB#fUMI-@;lEiig>PG%(Y&edtk2Q@A|yiW7+9Sv`^WcoT26du4IW zcX1|?_<<0(7ZYHt!uF=Owrx~rV)?SSZSLF0o^}rGahly{ARFE-4$K*92Dl!Wn7153 z!p$&2B{e{Ap8bd%N%8`f&39qQ@rdd#n1Aj0)Fzz*GUZUzc!-XQkuYlQ%clfF;)((p z+9QB3*%j$0d3+wGR+rBe7!`KydRmFYadnuS>*6BzSVMh~QE z2ZhE}f|A0*o5m}FI$IE8*khCVQ!0#iocyz$8KbtI>g$TS5)?7v%1Bhivl*zgY{a=U z$~XCOeGH(j8s+IC^>c$qI_VEDEu<(f6wk5l#Z(&(Y_rNameTsgvIdps|rM zQwe(-(7!GtNA}H_O(hp52w;F5v4`VL8cbx2EY#?Dcz|4e3DwV8Sk<@bqNAF8WJs2G zbn#rqm=c!^^||F*Bip5eV927}2Uiqr8BHj*EQL+P9v0Ld(f4MpBz5X!ZyR zVp~agLgbY(8tsytB;vOt{=7cYbc!}+d_qbR&fBONja%oK9}EB<@v+JfEc;YpViQG&D3HU*9tb(vfs$A`{8)@9R`VMMb%h0nbV$S6+&yAHAdb_Kus)%RHC3 z@BVCRxZEsRpRfVutUmQKU^qP~rK8>Wme}saB@XS07J}NeeW!Mk%yNP0f z#DZY~J7h$9^k2r@hMcloRMjtGWKg~uen(XlDr7a5GLTPxt~ScKvk^EBz`w$Bi8QYy zRyzk5ZEjj4-6{n%FfcIokB@gU!MKqVnQYsafwy`(M94_ze}fxB0RE3PP47L1jfnUi z$H(b^l5X@`D8GN5OL4o4^Y;=($_;pba6>})90BRyuKwH*yv{VWh~)V?q%sH6$o*5^ z&7w9|R6<=Tk=&Vm8f_M`tefG5@u$s>4*f~wf3jr8CWQS124l4WQC?Vo#Z#o~yiV>OMq5Bs;e?#@3zC|>Z{FMY{{ktQ z|BD{~|5c**7iF_RK!EG|zu^wAtgPpJ*W}Ho`?r7PRmVFQAc^tMQ1sTve9x|u2}zk# z4gf~vKd1h=3r39p`@_GU(6If9^7|jaqbTVgJ@U`LgPwmP_x&e;P5fW2q~;F+{uMlc zYM=0V-r^#T%q4vkGm07$BE`BiX7`2@)O{3{;Hl+*QO(3lJ?6cs^Tx`o3hKf+8@RQ) z2)A_7+Au3+7o&gxx1#!+j$~_2c^W}HGf@7!>(`##&Aen`{n3xkmv%qxpgk#yC;hYn zo!8dRl=|u=s`i~jv2DZ`9Vb<-RXB^8vH=>j-);|TfhSw6u+_y6sE-XklAx#GIJv(~ zbX&LiINdNjN95od2K5v21W$YB+fU~|lOL}+3-WY}5jQo$hZ=|U?!_h)`9%) z!|^Hu-g{p!5AdvKz_f)(vN}PMuRu%3ErVyG?fp`FzGZfN*dW`Em;$Ew+IvU(zD<%1 zZK9Vi1$U|sRKi@K`|d9)>s~%mp0kqst}yAfsLg^d6(=9uUASBA5IBOLKW6aJAP4x- zeq)A3XmzNm_&sHkchVM~u0C)2rv)y3KsXQk3hU>^$&_lc@|4twNwMc@UZ?CpRMb!^ z_r_Q^Q&??@k$jlEE}Cn|Hto-+g?{J!)}MTy<17v)Ei5glc*7WOylb1@RIsnh>`G2! z1yT7eczHw}n(XOhS9vkHTrd>+38ZH{$rF8n>VEAjB;b<{7F8l};D22olhNy=kL1ym zM~+7F;wDRa>@y#3*(X}vpS@S&U~gvp^2A%;raQA3PXkxU{9d$5z0gdfK-r(FggD>iL=`Ik^NjW3f)3+QQMSQ*g4uF3)LAViJmRz02%{gT$4!QU3>fZy6L< z6ZQLsBtUS7!QI{6ox$DR-GWQ-;O-jSJ-GXz0fM``yWe@9SL!|Ip3k@HR-LNeUxs34 zckf^=th*Kc^ITBnowEDah+~o^Qd`>jatQ_s%wB7S2ez5aE>9`o%ed8iYVZ ze2JnvoDXIrMBBZQMJdZf6Yu7$q^DIl8xBKxy`iNmrHAxMers;9_Qpz-&jwb6CO+WRmPXo|JK zR9F738{8H4ncio+iQpY@nDp!1c!og;MA{aC+nH^?KU)Ca$S=QRbl6$`j#U1R+-CRT zbT%7QVgyHgsAc#1);hScB{)#e1YhSS4b15@EQUIV?!9S64f_*4bs z%kTQWHkj@kzL0|g$&bBNd>yya+GtwnA?VLM21ai&;y!0KScs1Oj2C)u3i5fpS zS$ZOxn_YJ3o0$rP4t%r7@O2D!ruQ6-i!e5&?`yZCVXIs3wB>0*Lfh`TndhO&s~?Ot zO1>`-rISb`(mqtY^J8rmgvLja>91!UG_QCl=SzdE-P$4s^4wPX>HRWt#}%|NPpW=| z;p=*@`&dY5WvziLz?;hvJdTbRPru);oaBQ`rBR6^j0h}N^4J3F6Zoqj|BnMyuSVT^ zUl{dj&EYH?hn<=9kW%}=JB2sgDf`GcJ>|~@;KCQ4G@PrB$d9J^U*!1iHezczF~|gG4QpT2 z)a~?o>=x+r*(QqYW`TUKeVC1vK1rHLHJx?Y72SUHCCMRi$R4qehPaIU&qTP;eCWlr zuAT2V-w*tfZ6@JMKLz}83FW|!$(KJ>3$j(DLu z>-*uue!!9p@f39PK^q&*%Q*+<>0Mp$n#04bGWTwZaW7y`;N4yy0RoX3dg=L|M_Yv= ztd-$v%U+uYzWQUux!gtVzREkLi1A`zgt`Y?XK?h6wdKerRWbGCKo2#j<7tMJ@7kL5 zrc0+EiJ?qvpDJ@bH2L*ekdPR=Ejq&!tu|AC_dS<0PDLZ47bUz-__Xmz*Zt=Ac-f@H zmZ@2T!=19<>xw}2M}v$EtE)hiNp!8-5~F7kWh(D^DcKq=E5rKli(a* zQCiy^&o9jzjyP7qOKq&36R9?P{i8nSCvNy8Nq-y|*Ud-1&Qe$VX0#!&rR(a0$f=JLhX z|4DeArY#ectVQJ${OG z?P&R4Cxvb$DpM~w<VXE z^nIQE66(1>ZT3sCJV{}|i6&c>UjlhWC^faTq?v9Uc7RB^Ov5y~Kr_!!f~%ym54Nkg zifd@%x7TUae-=~%q;5XJ2(Km)q=&Oa%t96hX&|iiUV6BHs-8iEok5|l_PXX!j0b)5swa0cHjWAYMeT0y zUge@d{0Sv*rH3cG7ArV@3XZZslGo$C7uv(f4s8~=p9eC`q@W)AM`n=gFDREnHIT3F z51_B9wPMP&ad^V{&i$upODm=09#-zoEK5CU=*4uwrQfcZnOv}kixR_V?#o9er~cZ= z_%jgnKy(R-Mrgb zlouV>ACsK!Vk%r?4w>#s4A!P{Y%-@<)Zmm!uUgVy%61~F>> zHlvvxZGf@N^3e?wThUGtScp18xZ2V+UX$Z{Sgi5fpOEOd!*Yt^A5X#3FO(KY_P!8E zb;URK*0?G#V6=ExVofXHxEeTFNprdpbR7K4M&^=M;&;8!QSXe&&M8OP(>@5zQ+KVf zmC}A*NHg%yy!Zq2V7nWVid)*Y%fXPX7qhZ4Fau3sJUCDNH^*S?P>3)-OXX=^Zb2z+ zf9qRx9?6Z_PI0J}?i<-hn<*qT^mTe*&#;YuoL%F%ygJg{v)rTKW!u7@(>xes#G6eO60m{fZ@AWKl!x&x|>G9Mz~*=G1GnRfnC?-o75?5i;! z+|od~UldNc0S+jhJ$K^6j?z6?Zxm>^cWia5MMk}KM^B(W19jJF{CU|9$`d-VkW>PM)*)>Ch% ziH!dNFQefmgg~Ka!+rvjdRk|-cs#Xw{q(rEo#7?m4j9<(S zNw_8gym3e-DmP`e+48Z5on?j(>ez;n9Z3NIU`<-7ovpFz4ew3sg9P{l@N3Gqgg-A3 z&#aAH8@#1;KfU{|x^H}JhJCaJPgIGvFc`>G+thonaIo0yD`^n)5q?wR^kpa-W#V0y zcoK>T6Vm4S6@?6L>A&c%U0CAKu|eXSk(4|$gXi~Lv+|h|e(G^TUJwmaS+P!rxb0B? zxFS2>{oEzwvh5Td21>@+Vdex0CjKBriJyuRIG_}a%B9@NWDUkbvQpIZ)uCQXh6eM$ zi5Qn?G*u?&jmkK<8R094goPgGTC%cLrYGy)7wKt~(2o?0CDfHd4;|49XHny_w<*$;t~* zShy;sp+Vgle`KO>$2r0dH{oR{R~qs>$dNNs`UaM*G#-e6uqHB*ZObURd+o50)0;77lO{^x@3N(fbDlN!aMBUg{(5@Ew(1AFQ5x!EvC!~V{lw?0 zkIhY>xa&N>>Przj&XDiZ_f`L6DE;*WI|R(KC2h_ErQjAj%PE?y##UzzKcAd%apIDJ zHb;+&_`Kif2)bQqko35-lMT{`nr%7(G@o+`{t?##rhXDo?a$P#jxyUvRCh)<7F~0* zevR)WHHD(R^i+*e1Tj$cdDTS<+!B}4 z=`aN~-tX}-gm6#(dg)E zE&oW0V&?zD@t0IqXIQ|d#1skU+|;kPWN`vvU!91-lfB=e=MeU|$YagyS0p|p)myT{0vqxo^xIQ2l7iqYx*>Sqa1Ykh~ zDD(}l(sobi#yDf2*_kPDCQVEbrSS*QcJp9=Yj{gH>u4{4Pd9zH~XpzhwQ!&PU8 zRWOhLn9A*qa8|r8-`y$B=+v7H1=ceFi;#c}Z-io^#wl2yit~qCZKB^W@z3#>LJd~m zQFRXcwXwc7?8mwz8wXK!xYF@&wHZ4J;S%7gu)%CGQZaYac5VqxO_rOoh zQBv?a!mFl+Oz5oi_QJ@_%s5P@bXxnXY)UJ=p(to~7F6k1kU8B&0A&o`qUsLQM^gL( zp&C#G)IY7>XVaG5hH2INl71!})8Pqo@Ux5a|9Fy^?7HC{uR*###q}v2Y0_IgEa4TE`_=nE(RY#-Mya&!@6#}aEnO}1aGb7!y58v+9-mzU2>>=W zhDWi41?QO$PIpIpvfYR(2Qp*3)xI&upuQfU|gJ9$e1L5y4akqK|DoS7v_!et?i?M=EBOo|$WC0W^njXH-dGLj)vx+x>i=!})ld!Jio6gG{MI4Cy1cuyr0Dotsf z78{%Nv`~rHg+ODRMiLnF=W?{h<8h0U`t6;wbdo>$!^we|+e{z}1#SgIytxCAPMDTW zlwu1B67|SSTsI=*7DpyLEo3z^)Mg-xS7Q!y@bxF*F2Zh6PpWfFI6Xef@?H8I;X`qj zslTl;CD^xuMw^a~-+yby`ql`U#ao8#(wMzK_)H$lch7;bp(hj?{G*3(B*yLLI#B_Y zo-c?M3|*AMGotv}4x3?$a=tRvRjst2Y-&72I3 z{I#4T%ipn0*CTyr>Z*#Ia)1 zZ9dZ1kp>nQz@w1N2#(g4J9mCKV-+M11*xb;NRFZ$<(a6$|;OvSF zK2hFW;Fd`sQxzw%qHw|!f=Zw^W}vE=``EvAKJ}@6tV*@216h> z7#%Y`vpRX0e>74@Wl|h_ZN=K^x%b5huKPYrPMcyzbB8$vEf=g=%aU zG8r;wIF{acQIn4jCzu{bI=57n=2EN!`z{oAB0$7BXFH^z@}a;{pW}>BZid8@Vl%^g zq|3f_rDrZOtLw-8w{Jej{~^uTxFx*5ojlET^9S(U@Yr`dBSO#zD z9w60K?ckJ`m@=C=V}^;FnsUdOIYmWAl7dFc2QEV;J$)RA{qIxVon&&LIh|k}`0rFw z7!8Dv@HG+$$BXP`7{Xd{uvV|!=@L}g=y@b&jIYjZR(v`5Wmxh&lr zqxxO{nH}fdOT1)im)sMA`!>wOALbc3E*y*6 zmrwj50@th^uYP1?S|atdTqhk*s_3RH3_IJAX-%gewbeyo`JIf{hw9wLrrwLhAquu8U{MiCuL;iBT{(eD$FJ*C3aq6&`rrV52@p1a$gc)+sKU45s6qQ zA@Q)#wS>3bUt7nuJy2Mu^m|t4Vy|+i_=o>KlbEB2mS*8?ch*}^Z4YHUrLE~afj-`- zk!mvgec?`??l<0%rSI~nQR8}7BN=ypYxoU4zXkoZFb_CEJzpMlQQ#ZO+@d*unf$I= z40P>&V-wVYGk8-c-dacO(+f%~WA@@fRMyRq^ExAcz3JACD)v?+BQl{m7+;?3uCz8K z;z)(BiDhI6|FMzBpc69VN)yCjc0n9Yl=Wie+H%8Oddt;=iZK z@U3M`FPz=uU>10Z<+Pn{^C4S($B%-%msj|xm;@=U&Dl?3@mu;yx7zjaJ#y2tmzYTW z&ur#*gFYAKkm!%>Lr3qjTz~q7VC(tt%tN8Cz9Kjm$`icievAO2!?-HXwcp<|pU4uJ zB}kk`SU#mnnf^@T%#5>$y8qeNbHCu}_;O-6$o zujHw7WLkbGp927Eh>9JZNW(}5jDKX3@DlibBz(_7m`6D;mFbR|ith2#;Xseflub@_ z$Gwqm^n^yr---9lDX|=$kTO!t9qH=S7019kYiOZ|qrcG|Z&&D%n67J(6I1=_CH@jJ z5Fq?TE*boO#4r^imUi&w1eL3#r|bytA&dXA9W|XRX<3Utjj0iWmkw};VxS$6 z18>zn+;0_8G&bI_3hn%p_7tkkkKL=m)I1{-{;E?#HfmR1&4$*XXCTG)9pI}kY!z7B zN$uwvmS@6LK!r8S8$_)4b0?gewTRnG7P?Q=P?t9^Pk*{KdAK}B>jQ{1{tGI{?=e7ZOG<-BOg!G3(XZJgVEEtl47=9*oy?`8I5&kqz+0n1OnQTr?i^`5Lz2(b|k+Zg6RQmL%&XGy4=rhUl^^?qAEUiudcD!CpS2lubC+Fd)8X@y69Z> z{`bPq;%!GqA|xML9`*;_I9!mPf2yA7>Fvb(Y06Ya7Cb=6Ft;%<8%t*-cLI>)y0z6& zaL7vtC%5VVYEw7Ryqzw$rKXJ;O@HC#0>CIGq{`{b!%Uw`cuC+PHpoJGw<)>`wv{`Q zP+}X6(Z*ix;JET*Tv!o;WSX}GyQE!A13rmSo1|eY>-@P^@O$t+0uxDU#ub~ z__&^yM-`fAQwNy_?QtYQCdU2Vp7*`Oi+@;xCy3<}&c9>_|3>DP3AUrnv6!faU@0$l zUP5qp9R3ambL-~Fv~_JpCJmLGOb-A1+=%P=Ga~8Qi&f-f85QF7OT_oz)1;m-j|iq; zy9{5gJd+AVfi{fA2rU$P6TF94^UBGQKYB_=O>1vHG5e#;EkRMjV#<>(;1_m!f5Ua$ z5k0jNkTQ3|7S`JDm-50%{@!}~-OhM66PG&81$K#sa7vzOac)?ONGwBg&lI&(f8?LX z(BZ)xiN&}-b@2+Ta!lcPu5rG-%x(7iP8O&oo@AP7yX5sLlQh3du8|s|{Kf?|0ur*a z`9(U)9E5Kdbu0{DGul4r@d7fq8qb^!F=zPh*!p8omhWK5Zgbbr^&LK$ zUJv%(#GV2gJy)(^`;P2==PTeun{*RovV3cM?_h5@vR^yUqpuB(5dn1hlW<1&G8k03 z#2Y+DEFaNZ{RRH99>m6%HNtLks`>O)p`$C=amGy7IWKen5`cEi${e=ram0yqB_k>b zogOOMbk167& z`|<~&?GF3n)K8lD4mXiuIfZqMATCN2GY1v8u0R;mXVQ5pK+_<5gfAX5+sIJw*AQ&K z{b0j`uAtYwV}8EX>;aKYmEwORldz><|H-^COb+S1+}e1_@s+<{M-xFs711;z{l?`O z%TumsaZ+BT*WCQzwz}f&5LPv(ee(I?leoCzeNrc9Ffz*s&k{E!_rY$|!y6!3D%V*$3+N=g&2^S~SUcS+GA&D?I2JqTKu(UxWJM2jrTDEtxFJgUBl8OTT&LRW;K24k4kOpk{>4XX=(l!aW*Pm!Y*g{W%| z05s^zgR^cynac>!K;xOwdjbM{>v=bI47d{JG}Rcf7)GI-)^1wq1=jQV^hd44Ly00miYZ^jPb1Bkyq~W z7OLua{gOy#!#0deKqn>4jfu8F;1#;w$Wv=wFlCJ4pA5?Mr)CRBJ)XcEQ5&t>oK&Dc zTR0c#=ic-`VC~C}Vg;83FkKTrchS}Rk`Os%qUwcW23g_p5|aLv0Z&qF0Roe8W9rYC z11aY!_%K0czoQ|!ocbV@`rQD%?$a-XFi;(&~jVsGi$~&W~1_< z8Z}8op&^@%Gy@S zuhkysA~E^toYu249>~pFCf}0ATbyFZd|4Nz#4VenIY4;82P#-rOCoFayjTla9Nrj8 zi(;x|PQ>oqui<+wp=`PAAUVZ{_H?--hOK!dZe@n*2;0A@;Kmqld4Yf$DrA1z7Jfa7 zKJXCmASP$lPnqzC2KK7KgXD;co1i#!=9pt0=2>7VNtqw*(d{x@IG%xY#j4fV1|)s zXo=$j85U>dLv78kRN6$BG-K|8Vp-lYBUotKwnaqU5$Wn2hIhx(c%!oP+k@OEt@?_1 zrzNG;EZf~gw!hq-M}~N!p+hbSsamu;E!X>|_S8}ZZaQddDCs_e!xAIlxZ{&=T})76 z{Z}?%9L&Os9Os&twKYvHboWMVv6Utrra(-w6*zjM+iZ6vKFfXG^Y!DpiO~E&ow7}< zprHucA`$K1mbfdt1Com>sBKL1Fkq)$urJ`H%RRAg5C2uR^|%~cUsDh3njK?KIZ5$v zQ^#uh>jTTOElo{Dy0SRlgrHM;J-~7{-`?87g_w+_0*^U!1C@X~-JkOPr0gu(?D&q8 zwUzBl;^+~ZAU(aFSm8r~fieI#SN5h+X~D?ia>S2|OP;)I1NnN#d#T%uv^~~M#d*uM zI#*qyQcWD`xpcifv-%P~!Wosh%m*I9-J7Uc#ZXMM|7HB*<%V}e;<7ro>bJ*b zd?o=O^ECnp=j58)V7(^KL$9Ulv~nhUOc+!SX8D8*@n>q_5gT%DS~>JC?GF4$%ixIH z&XRq`z(|z+f;(@Yzl=8YSW)aUn8sIW4rkT+RLH}3y3~=A9flK5moa*_kgE{5Ho%z} z$WM`--{;DMXlLHPPN6KJKPi{0K*T*hA_(WfV`WuZ^s5fzKOQ4ucTl|*=S}trMVzG? zvdzFwvur=il!dtsr_9c#sC6>D0rXE>NSNHUvEOZ&8!L2981M~3t$bs^DU zPuIlBqRNVBxAcP3uvx*++EO2tqh9@GZC$a0TT)U1c#K}kKVH06*3%>O*kU4p#=i?& zBLt&AJm9hsep`kI0g~`TnxClFP$1#QB8R_O{?gaZ&<_7616E2d{t-q!v8yoG0$(6IA;PqRlkh=}*f0}tQ zVpI&?H5?lA4Me9No)t5bho*oM1(T);6@hldbZaiyYIYDA_d<_2nQ0iIg8H=VG=NAp z$8C>PC>*R;9Hba`^py@U&Up;M)~v`D8dvSH*SqSB(q&4r)Uw+%UummDd5UXniGIf4 zTEabgYq}I>lUDgfsiQse*>Ww1sx+h2Sf7^K80l1mnGt^}8bWSA;a%LFA8D*e;{0CQ z$||Ox7EqibufEXAtyzIWXp{H5e{i{tfkqFXj%QWgO`j(flZ=IWR#rihvRlhLSWcdi z(9Bo@eeeo#5=x=~C8)Qj1YEK`O_qX9u7Brh)47!kjUq8TIQVOn1Zgf7xqyH^n1bkY z2hIQp0$;vu)!;(a;42c7-gw|AiVs0sdpkHKKuk>R(&=-+C-sVCLU88c>+?dl#O2e> zn`*Vb+;8&aIRVL&v!7Vw!n1*o!~!%foetG zez}Ps>x%uHgZ|N=?a;ullThjv7>S^PlB2`J9{XmrZ(l9#?24+ZV_1!vSEy-d`mBFn zd;Y^72a|QhZdxo6zW$Vx=`H+kn3#`}kKSK59Vt1ffx_z$*rwAGVp}q;|G;`(vK#$N z53@jy*&`X4tN#U8qaHYuruPzq~61v~~+1C^L@z`(G;n&9#u`9G>%CRH#L zF}{BA{EH~dQbUIf>iq%se*Z}$^ZA7i5A~(I4@-gFkd5k}`_mu?8Bt7{Z)Xxsnv?Vz zNnA3j{J-#Gfi8=G1{|xWz#tRzZQT?@PA5^}&qXjs-`>q6{wL+^-(T6p!?a^})^S4B zxQ`{xJIbmUFBgfL`?+Q+bjis2-y`s!_XDQ0bA^F^^qUG1J=*?vH^GDc-*+c>iAy1@k|1|ASCjMWs%=s$~s~8U0@- zbs!Qe03Pi4x?-aKF-vTgblJc2^Z&cOWM%&UZ!Z7SpZ}$+U_#~QDMwS_V97EtE9L>tk)GP1n&TcVUiUW!n**Dlj zvZ52188zTgbISV-Ew{jpC9WAY4oDj6>}L&c1aM2;iIoQCpi8_r^#9dB|pgDeUN9SK)nUl|b5 z$Bd)L?(OL?CeQ{$YPYEsviXT4P9APEBDjeDj+&MVQC{C#4nV=9oedD@U&&ka__Tsi1ZGy4i{Y@r8(pdm&53sGr$OY%g6pzPgT8DnlLUjy{3Rj z&U?PLiZZc`)P?y;5086BT-%Sw9BdbyvOiO<;I<#7+CtllLiKpvlor;3uIj|1k#IlZSg(oS8QknvM%GEI;CVi@BNE4b6l*&p zFix7bSkYOIw)yy=IWE`FwS9!;odztFHTiz7>O(MT|5q%)?l*X!-^(;jDrfR}mQpyF z@6vghUwAod-xOIt=y?3N(2DoBKIz`QEH?TQ(yt*WD~+_&lk`PrAs@QcTa01SWa|Zw zGbcsi0rj1^phA#lgIw9w>s$^9aC9MzStGI4S$+W81CU^Exwvb)6Ivk53dI0i!bmep z4YA)>?q}ryTH>NqXQ_dMd6&sEL9TPj4OTDu%dhGzu+SjYI?yJn!96&KIlv_JmLv_sT5J!%$e`NU0PM6FdLJ zVuJsugLL!k5_@$&kBNtv{M>n>S-+25CK{S}%JfI8D_D}iH${|ywNb6s^-3lX+YFsz z>5)p0{lw$?m&MY`FT+dK2HR;bA7<`bh1lLwN0}NITBMI_6vV@Hc*2CpwVvUbS9>=G z&j4_6Zb&7ynW=;(cUgF<-!<)gqm73iJy&Knm$$333oIz&x3fBIBhyD6bb7-tg2c7J zc570HDsCxLa}+HnbuXOe>%Tz?^$lvnyEP_Dyf%k1BgBlfm_Zv8&zyui>F)Q)jf14&qY!YTu;+Y9UBhbC)xO@8b!uoS3?{ zdTnW)RK=?ns;XWq7QBOt=k2v91`tB-p zG?pX_Dd0|mMS>{MrXk8gXtxkQah`Xs@9k_T=hC@75Xd4K~u4c zzqTIPVH^CTPl;h@=88|6ddcHH+e$cSL##Z!2~)t6dn`}a*2w7sD8sFj($u#b!o9LJ z6kR^QPix)>*O%?zqX<-jA;Bwc1hm4Z9tw5)Wof$tG>ZmsOI)MfFqUix`c)vAu&XG% za9F-RGTzNU-e(A-fh50O0G>$OdxHD}uly5>rxfMxm8L+uxc4`gZ_NNMIR!L_&Rali zH&bp(BaAK+6G6xWWZB9ql)u-bQxm?t*^C+_8@*?S;M7u}!OisY>A=!zPY%ME=J&jy zzpH%81(j4j>|WU`k2f`hMSZCgwejhiPm}@nh_(j(%lKiA9K258Hji&Q*A6VLgEtv{ zqzg=@^|;(Rpt@i)p4VP#^g!9mVS~aTH-Tva<h$UE{Pby-Hdl$fy7WaYUxoX<&mSZi@}lbcA#vV6vwcuBe5 zMPqa+9saTI6}JcEM3$lb#l#&|6C?4m^_}l#x8c-sj;XC&v-Ol;m-!2oe5{1h;}4>_ z6Z4zYgVbq!*gx~z;Es;;R#j-#737qTLzoV(C~@f=xvD#C4N~SBi;8}#XpUhYQ=i@+ zDWal91PiJKPb>E~tGR?mYM$_9IOi|@J zx`gCXnFXS1Z8Rpanz1goP@Ryxt4Z?kzuMU#WK421=2;bn>ZE7k4|k6q+!(Q>|6xOj z9S$1X+m;2de^60TnQ)?g^#5H2VM*-#sZU)jirsvNt|sT4`B`Fwq#*c=4#$z~c(7=x z&Y&y3dzjgJcdd-)M>&@38`TQO!Sb4OQ?}F*jBp3z^YE@SqJT)cTV}rg@k*%B)ezRb z9ybuNfH{-ykKf(uNk{RD=XF3#;B$-+D&WeyWd1Jw)9%KgnAr+Pu*I82dPx7CAp>-C z(E*fR$495oG}_wv7z%`>r5@>sDu-lL#^6sTT=R#Z-bdaW zvtYhi^MoJm4x`&J$7FCY+{PE&&+i452Dou_UAyOGG$+7>Xec(=qqKK~tUkQqj>Ir4 zckdziJ>24+;^|0fbtKPP{jK&(FQSKhUHOnN>DKGj`@dwp2negP(eB_)hl%Y5|i=Bu{{#qYA*7Z_W7G)W{N1NGF#Urr33Ua}D2KbBNk zBn_tgu|%Szi8q~Q(G=^_*>uaDZ$zOy57 z4S3DdEwNCoZ+G{9G+pA#%`NRtdeE_B=I5k)tf6ypbM47_C_Wbu5VkJA<1) zNhrnx!OP%fmPF=jSU;v)RB&(so7c(0Rr|-W;mBU4cc z*fa^FyN~)FLgnemUl+(l`JEHra6b2AX!*OT*lR% zR(Z^s)=nGP{{9dKAI=%no#{^a!3~F5nNPF>n_!!jALe80X~8jJrP5;YjIMmj^PLxq z#b%4cngJ_nb&jqQjb*y%)vSd^*fATUnK*wO!( zpcAg~By-q>$ib*52HZ9!NOK}~!eo5G5*Q4s8Mm1E=`GD#;Ul#S@VkxV{i%czbMl@2 zdX(@Z^7qLDYvqSWepFVnt3PMEdKXY`k*dpoCApC26;Jz$&)?5Qz$DC!nP?_(lg;Z7 z@GJ(B?{0kMly6mJ+3;&Fl8q*rdckKeYl?N23~ z!NnC~nsULJt~{#{=#lAEk%0>fz;=USV-6G(=07VZ1v;FlChQ_qVu_TW06)K{F&z=} zc(M1{Qpjrmq%inNl?f9D%izvvPc&{#(O0^8f47Yom@ib>AECDQRd2z`)7uI930W`f zN3lya%Rf6!gEaldpLi7dY&okQyqGlNKv~Pd&*mXma(Q2PxHmFU0DppNf+7b;lqOpX;Aruq!iWWN)(=J2n{-h_&it`bNG@T_EHt@kguaATSF|ekk6R7SSn6sET)goUQHsjmiE1^Wp2+C$ z{qkFufo=%%xX8MnWW8E?XN7a%deg=yf4&pw>3fDU*?k0kd#(k7hFs31dN^R3SRXi_ zkb4}T-o-?ZGt%JZwYoL__3>C1MJv@~&iuvP`TCmqu5w`S4h>CxTzms(@&^^J*B?m> zrXO0rP1wuwycZiD$@R56zPJmCB4LAao$)zkae;w)TPIQ{-=|F);}TM2J<{-#P4K>? zKUbBx+A1Fs3*9_xJk6mdOTIhuPDJMHgs|i9?rBmB>PiE{V$%Xto~C^l5@iVa|HwU! zYIq#}Uek4bU~KJROW5!9=EjREuyur@5TFv1-1ExuKe9t)Z~w!_M2^9{6Qd(#?-|<~ z`7ooZF=&u`xL1sws`zx`Z=rV2l)#HPZArTW{h-j_@Ezqw!>-_01J)Xze78|ztMxKh zef~tFhOO~==0$x?*O%NWQ3a|%Vyul3{8WVAP6y@>UT^#$)>-b3k56)I!l+@oeS72( z^Nbs^X1-s#4trN$-W+DzHkDGWcb@L>v5?IVP2Vavt~PZpU)(>0;xRff9Bv%^x%%Aw zSwh@Do>{!q;qfIie-iS=d|t5DKS>{Ts0rV#64!B3&+yot{GyY(@-=ZB&M6NwIMoKoJVL68MqUMOq+YrxHZ&sr#q}A z2*17Cv9$(?6~$~m__uvd+&-}Uj$XikD=)1LP>*4jOvp|jFMpeO{QM(YMPyy{baBsY zmp|x=aB0$8PuO@y;`+uwH@v?vusgYGEVZ_iG;zh>M}RkHqwl3{<%DZ7Pl<;)_mGK} zMm;nNTXHe*>C`SDJlu!8Q9?U(Gz=iSc!mfz z+#Ojz)}~7)Sxi}=ia>ZS30qz!(fz@aw!5*&uv>5!t z0%+~YzofNZJ!MeY!)VjHUa4~B$kNLeoE6*z&bp9Mak?y`di;iiwc)1(N!z*JQI6M8X8KR)*nsyC5L^E*AwzWFnp>T9}&b z*C(6TPx30cG9@L7>da?hb57*g`9g0 zRazfNsz5sf11EQCY`2;x1nluR<8pU1p2qs*UaN+N7FVVG1eyqY=NU47Z0|a9Tr7`W zJhc2j<-KK8oXyfU8eD_R;O@>4Jh&4af;)i_+#$HTySoO01$PG50Ks)|g1ZFyhOO_j z_qWb^f1a}r^n!b)@2;+{uCA`W+CHCMM4P-$6Lhk3hZs20iF@QKN}Iy{`SPw#s$X1* zC89^s%M3tQE+Ii@bjUlTQk^b~{hcaTsDovva&1;@BFXwH$I%N9IIM!F7_Laku(20b z5A#4{PpCwsFGK~iRmdmX^Cgy&vPfn9x9H5*^5w>)aOyJd@z$DErs>eS!nq4rZMQU2D9(@l#-hjx!ymq zw_vD7N6YmGLD(Kwhi!&{CqZ0B_aF6HIWIFD1VWv2Sc-)oiN)X0ft$U$LmaFK?Oq9I zm9;AuPYh;KL1`UsG^_1Or5!DJN=k5tA1ld+4>$Lx3WGM~TaMQR=p!`v{s6*`c~X}D zOwnzd{Q3PBHX$^T5La;*ZazT?f;L4nf3+eI7_k~^e_)kUW%=0{Z((s-bl2#aw!(&d zIyEkMeid2C1n9$+jHmZ{(wZJgOKVJMBJ&L!F%AT%aE4W0nPDLqf}V`gNc%yZJe4_KvziiV7QRHLTFEZMio$ZMkkB`0{^X2Rl^g9It z+0UW;xw*%@lLzi1TV7IBtveK$y-(1`4EW+-+P?~FD(Fx6AH%J9?#)t5e=URkeJJL2 zxRx7-N#I`yIdpJxS$sGdm}U?c)nqJixvtauc&25WIUUO~y1t7TJl5=sC0k*}m)O8DcDKZCb-d}r^_cg*sDX|rMbl2nOv^4J ztM41#yyU=R)}JwsH3LdhZ=ylEO+DS^W}!|WXp&b0@0!Tg<)ZWpH^&=(7Gfp74;Wz%QO1E1myTuIzg9U|tI$5L1{g7%{6rUxv4?0&i7Y69zA;Dw%J?%R}h z@(rCa*@2gkf_YJmx0PB$+U2opT6pRLZx5~>^qiNi`=l0Cc!zOFlrr+e3!{Z(eq3lE zKC~UEj&Bcym~K<6nV}NQ;e8=$kHee@<+25bhdxJXXz+e-YD+dBHL@i&lu3)P|46@} zU?D?8MS+D)M?h5|Q}3cW{}Uz{6H(C=S!wgp?j;~B$>#^lTJVNbJkT2YsZzhv-3Ie! zN)veBj`{1Pq99&LGX+mFt0|Pu>Pb(KJmgUA`0*(&@}rP;+Ia7$!pjyqwh&a5U-t0C zPw82lX?ce<2110x*?}bvomMbyU%pM<*B$&KsRD;G>Fje!U!Eo+IO=LM_FsOvb7rJ@ zs&C1wgJB~}!5{*~qL|#xeiVxDF)|)$c3#u}*n^O4UV65CIa9XS5&E+7Q}Hc(EqOF* ze^`GvPfP@4e$Y!+?ETv@&l7@H$Ub)md%Q0-_~T~X)|_WTfnUT_zMen02+?nRF&h5H zR=3KZbNXQ@$SBUJ87_3Vo1p?pVaDMPw3keGtf5BD(uZ~>4OGA{3?bjRlifgOA3Joo zM79F@;Ry@ObiS}7UM8~L;=)B~)dSe?oe#?^KBwsSQyrRMgZG)ihfL=OS=Y|KbVL@y zeV*T;Svw%B`(V$QkhwF%Zy6$>0xQ6m2vaVKQw`*vshvXtxM<(oKXZYlvJDC`tYXsn0gw+HJ(2iJ;9>QiHFV-Dc6|;nz`Xj+#nWh76PlO>OglVzA-Vl9(QE61L1Nr*hC(OA zikvCC(_B`E0#Hj6cl|;qStEMdGkIa7*lhlAI8l9w2`Hr zFmnk2e#K#s3!7WRpp)3;G+y$+F)vLtOjF7#OszCd1v(mW6L?VZ*-3DbRIy??K5Tt< zbzH#BEA8-qID@5q%TwFZi_}T$div$l&oh5sUe;jn9^@J!bI$N{jx-HLcAX;$=_Ph; zkW5(Yz{z*>=Up9D_-*W%&=@rI#sx1EXxyVkcL2K4TPJ01ALy1v?IH5}xa;XCQ3MNePXsVAGn9!0ppU=?V``M3bR=#_9wHSEQfWe zf@t5`PU5XRxf&Q)qhg|Qa1m_ZCdhUF&Sacc_0IWYWcYcl`L23{+VVFi+5{5gqQwu+ z&FE8r%02sT3Aw2BYQ5NyCS8ybMTZPK(2g z?iHp7e;A6U(zvcOk`{QE_2WW>Wb)Xqv?9}vnCf-Khvdvx%WrEVk+FgbF4H?+FDFbx z;mVdA=^PLPIEvHuJQ^1dYQAPwO-W?J_SlP4=KPtc0MEcXF~ZV) z3LMEE6U3jYPH81EwqnGl;nYmX%&m&uWYDO@rHA|(-%U8~w6Dk37nTGP4t8B{j%_w& z#F_M0-yJRMk8cDZ>M(tAaLSU?G9yf1m6iY?V^Fgu)ZNMU7-My#v2ICQb9gIH%XVf$~LUw-5N9e}F6%2qg&yWE-R=R$Y>A z#lNGl?Y#9V!J)(a%^x8-fA&Z0^JC!m!a`FX|F#`^>BHqSr>H48u_Us{3am4Pdvb*t zjtau$%a?)5j&LtKF7=b;;8S~;HCy4(8+8mIJCvczTN{e#Ez}U+2mDgwovpOtY%hG^ zBm$0N1pQ8gyxcvQh>3SeTgr^fqRL1K+iLx5&f;ih`eUJ?(-#HQj}`I z@NFtUVjAxjJeu$`=ZnK_CTtZFD#82s$Wnb7LNbL09S+B=%vP!@E)wm26yNm_x!kf3 zR2?E|vl1@Xq^@K(1?A`LZMR28rz+UkdW3WfRT5fniA(8`={FFY^6j%QL-nC}c&8tL zj1M&2Cts*%eUe~3G53_hca*H$)F693q;B8XGDg7~zBlUm!hL4hPBQ+(t?jdy8{Ru6 z)T?2Wns*}Ph!q7!9Ix}&dpOZ0BQ{56O~B(;uS%)l7cvGK&9mID2-ArbIKe;DAEgxA zGKh#7W9JE#@7!yjq%d*06ZMm$&9;9i*RBu#{PSSE*lTGGQ*t(AxFW+#XW8kRGVvg6 z-q{*P@l=wCXS{n?Y z7h^3GtbM}O2^eN~TfE3`oA+QlUop!pf8UZasoqU&fRHh&N4Sazj`(1d#c)OHqUNc& zj3Y0ZGzWl#5lSqdy!10a#cM~cp@K^LA%r|%8G{6Z#Lgof@S7*b$0iA7Ao%#8*Fr{z zS9z(?^}drJr!^Dht~Lh6Mkj7nbt+oZIA!)QGvll10fh#ua&?1+DQOwfopmvX~gzaw}PD*Q(S9J_rh zD0+Nh3#*xGP%cdryan6FuPp^wXHYaMHQx@>LODmmb5;^?^1?80N+7cAuYk@Nh*v~s zStDPkeqAsHS%Q9{)?|fLZ=fZYxBlsIdyYxA)+wNeRJQvxe|8oo9PB66^ex)J3UwHh zabI+asp(>*eo~F#T@KlbD~%)r8cw&_+rgE^z?q#r98D&D+GKEi2ea>QCBaTJI?@=~ z-0qjh3svR;3~5~f3Z9fLFCqNZ*7&hBawtR2^Jh7{&W}i6!{(MizYvzv$!f42oT!1P zk8d;iCd7uDbI-bbTcg~Cr1GLkM(3v!dE#sNFtbeFi19xP-)z9tj9mPDxx4jPzm3XD z?0|vXJ1Ww<7J$t9i%~<@;g?FooW7N=!0GWzW7C$7RqE=DhIGHqt%ziWi-ttVQ3pj( zP~0{(AK#YuGwc_b?*X{^4}ZK+Lo06-oSL6X+O&4jH!7-gM~B2x^?S7?poOf6GV)8} zZ1uwdHVoqHk0125X4YyF4t!gqOS#SMcT&HbPj~APyE?*{Enr+PyT)Bn?7zvbvR3FL zP)`|7AZWT+%@zFcUSLPv@nkkt1@QJ_sYHrA)^^UdwxcRJtAhUg1{xmUdLzB-G2OPD z#Om{E%I5H&9NI1;u1SMSNz%`;Gb{=3l67;R8F2}uv3dWw7N9-a8R6Sr`>;34J2l%2?UT>)*POc<7QyzP;6hD9&^EYZk!E&aTAOW3l|KjcU@VqV|1vfNIk0z87xHur6rN_63uu$v2uwiA=5!Ec_wCrCP9GHN@d+!BNbQC`y%M z6IO^wq4w-$wJ+d-soeMlHTMBd2iP7vbYgf2up~Fu413 zfj#sIkJWhSOqDddD-7(tcxD}P{+usPfn3Ya10<`(n)*SR2>P?lb6f^ z*|KW4@C6l$-k5U?P;9q4N7{HLFWWs}j}i=7bo#JTq3EwgFwlP@hYth^Yol=cZSl3g z7iUirE$YZFnM3osu=i6xL~FcTjA96PtZ_GhpOVPhJ+>)g=!KcC{m4zj$ydOf=xyBo zrlaD^P)=>!?gTFeSb4jd!0cXDr?DZfD2Y5Srhj2NSyVwAFW}5aCPD7cQNL2cXEU-p zYOk9FMePJ6w6dbp#JZICG&7k{qdjrS$)NIv;w(<;$X|`uSvS*Ch-S5HyMkRQ`ve-q zD%RnxN5u|*2%qP&#x}QUD9?Z-hm~skM_(^;W zqGCcRJJB3jgtP2?+%`GU>kLH4Mi<2hf+=%ZMR`0a8AOWC94S_#Q^f=4Qr^G28um!T z{no@@%O9!S8a<1mX~YOWEwGKX{KHT9>G|W_%{{##?+fh1HoS*^Q5pKXe%W1?mA*j) zTqvQ;z0R?RD2lB4E_A*<1KHDImZC+5!Z^{VQA;wwMzP1JB>~W%m`-vlLz0WH<%;RI z(&}DUx5HeWd~2lk?Ni7>6{zJRV2dI0htt7Z9((vzVn^gJL%SqtI79|_DCjJR9Q!C_ z6zhTC8{duTuSoSL5!Q(lXP7Xx9M1F9(jR{Q-nwrFpJaBfl;4R;l+|;^C;ypLI9_mm zy4{vScV_Ve<9NTFoyo3#*G~2Fx0a2FdcJnX)cF=Lee#nh&~6-gi9ZB;+}YBgJ`0Wo zE{i`eD5Dj4UtJgOT&qSd=HY^?pAB@ko)c{Jstn;oR~?tvYKDi;<%$D+k>=n`@2%4v z?6vd)$2zb+u6Up_g!J0^rh;aKR`sg!``cl*W_}!b*m_a~wo(}88U6}ZhhYOX?;%5+G>Epx(&*TKnw{60K+u>dgKZWUE zHNj>TJok%7P0uh7KqE8U!4ARDu9X*5Z1`+h(QfeZUDct-CYTN>`hB7)LUf_ss#Z*S zo#5>j4c8BGwl&$bj8^}ZhXs_%irntBI`$$H=8Sv#$%Y&7Rwx2w@3jqT*AbB(S8ak{ zqS^8KNw;g?qOWm9C_2sgnRO23@K0GfYzF3H!-2Pg*Jc5&i35oQR|{nRn~ovdbROUq z$t><|`vUk!Gi;?5S75oZg_8+I0J#^&y-$XVm1e^Hsl-LS-dfedz~bZm6f1H?wXsaT z_~`D=n0B_&kT$M+4EQ)r8Sh=7=lK;A+eL4Mq&l%D3>;~nyG7%-M3zi$jL8G}kKJ2w z__~wR*UADeZ&X0S07p*^we%O>$*y+)Ld9wjoDSZkvG(kdSQYXT7eF6dANdFnMT& z5pV3X_l3K|&X=VaLJ?7Vg{R-amGwJW(@bIH?PT0DON#-+Nb#bqU=dboQ!cwtWX^T( z;re65J!AObp6ql@9fyOVL@&6g#`!nxJ%8Q<*jG^Z|c8%04olZZa$HoL<5fF8gy!Sn;Z2G2A+2xNwYe_Gh}t$A3B9O z6IYQdq&Yk`VWpJ#lBKJ=f{Bj#rBWM#TT`hAPSeGxC)u)~N`ZTj=_mt9^5nOfmp(m^ zZA&1Z#cxd19DC}?=ijv7|9B83A~cU>+za|6S#WkTCsrDPnI&yXTe8bzxL`S zSJbC)Hc#zI*lW41hT%0?;Fi#a~xBCyS@7cs`~{#N%!y4)e+>8hH~dqP#s9E zaogE&z(}O32Z7r$z{GiB=vl3k;zmpEQ^bp<{#b1wiZ;)w?5^$UWHDVb#=aOv5IW@{ zWCFuHH^6~Lg5)M$!R5S3w{s-0wpP#czyk+eD8fHhe{$TnIk@nUGyU>v>xX9cz&7TS z{ad?9*aHD0`o-IqILkwX^Cvc3@R|Af?Br#e;X7>eremwUizUWXQ&yqWf-&;Yq?Fmm zn^fU^m+e`v`yzkm%ck!Z;Vj!j7uGj8^n@eYWMrVGlXm}8o~#f01~5v(=g@%@542mJ zrS3LMEP`AXDDM)vFRu)rhd#J6RTA_&NGp<`|F*lEy8`5P-N^si!R7GyEqyn2`qHhW zGtXX96xj9to^y84JENzVuN|AGlqZokX2YWCIfASqcj(O=@N%*L1Mgnf0rqyD1Lb{G zCU1_Wj~kKj(Nf<~C2fv}IcuQs10N$$Z*4k-Z{;e}5bIrtt&J`o&`kxQaI!fnqJTb> zkuIq}jEeSrnyvIaaGl#;LRMgMEK9P>wA#qX03hlv?0mM_>JoJ*9M`G#fPJDXav7q{ zY-CvEUPjZi=LCIK6*bL+7@4tIxzYW5VXgaji4^T_j;I1rQl?FNYi)KQ!`jps)nwI1 z+Hiy3*XcJtZl}lwx(frp9Fq0o=7*oA=%2oNT34=*V0Wh=}8m~uZ{4OP&PzlXK<4dH0h{fE1S(zLIS>~-%! zYs}d;EDJ|Ms1=M9+h;M>oQh7F`CsndL~ug;Q7CJT8iSA*I=@gv&T&t99$D%+PBC1K zI`7Z5I=~$+1)!2>OE(Voq}m$K1}OA_Fx%DXj%xw6e=7oeQk13D%AhCmCQQ|LhNhta5iNfINZ9HK;42W;COsWf znwcY{YO{Nr{AsZn(0=4NK+FyBy`nhd{!+YO8%J-5c z3ac;kqY29ig#cKIJcC(Cr8QJWkt`8_7DqexGfMYe7DtCTSA(mn4ln|$A2R@MM_pP# ztA$o{rv}+t9Oih@n}DYh9qML7q71;DZ8=HYSN2;> zkS8;$o13y@z^QM+-wYm=VvtiA+6OwVlv9;|xRHFAW35XC6 zPg$!g`g*jj3?Lly`;pyl9LvsMge#uZ7J63vT(&+arCQI9jiOT%#Qw-ahCo|5Fgg&D8FArgohflBb-Vj;wA?(As@5?>JM- zZEfoQhoS|w)3uHR`jf#{t$_Fa%bU~z1gMC7Tn&BA9{K_)9nlO?7uyk4T!V@2vg9Hj zZ$4VD_8f&MuI&eaY1*}3a6D@jn2%RPeWVCxr?^Jv1P8E}=P^p9LsENWap}%amwwIl z=i=GyR>2CUw%hx;=w`YbTj3LBOj}3lTK}XGs7-IfAN<@XDuqsOrm6zl<-j!mY(ur2 z7)OG(v84Fi`Bi{P7pn*>POd+_-k3;U0NSjj#%q+@;LE;=0jFf+wf1sNU@GAztaylx z0*j#9x~%a+aba;Z-CRRWF^9smh_0uA?|Mx(ZY@BGW%+}YB}1sj4zSiPEQQ2oMaYfh zw(<+O3YBSiH|RO|v2F95*I(q#`YmKPXI?`?@)HDarDtUwxD;GtDk&=)g7gVJ7FC~= z=5!4H{_lp%`@wl2nrP6I0xvRNt{8HDg%qz*V|Cj2C7X@uV+)7-t}@#-^)3AUE3$kX@`7>hc+UIGKPPRm zPw#A3cnJ}dHtnu$j;fiyMBYsoUTUwd^gL&r59dD`oun=rdgKfGeEp?VJ9lxK*3c5H zRyw{5+I-CKxCd|RJnN5-k4sQuCJa+YM@L()G{-+-hD!iA%@D!%X}1x9Rg@or@CN0d;655+5^q8N~E#_%6YC(MN z^4F~Q%~5LbIG3V3e`oez8JuxDIynuGkMFGhP9@l0P5Z@@>t>}sMUR$JvuFR+hJo&e zjq>1+mokBA7`$F@4hSJR^j~TJ{%~e+PAsmtAbuCS`yVmnh+^ok@XT%GwQO?pOI&?M{{#@dSgfg77XzX+$prRwGn%}249DW-Wzl}+T2^_K7VA$o)0zE z81i)^-m)od{Sj_YZy2Wk4zk^1+x>2(L;>8cD>bubs#eS$sEY0UV8tjil^3q(exfdQ zJqne;Rqd6m>+^zOZu1#*=~({$x2c}j9MaW@qer3HppK8v0bTl9mNc)?La~WpZrcNp zSQGoxI0^*Od|B%$k-?caPW4G-+wm3l=7hscMd!?nsQ+{lsCV0;py#>XaAnbSGcbR5 z@Gf^}agJCl$2c{^^v-u)Z=|_19U_Q6t(#5Iipx6cN4wa>MNjI|9Bp$9137!i0Xz+Hf%@GhG*eq;K`HzN=Sbb|D7mfaKev-7w(o+qYcY+;v^k z2V4PXri0Uu-=km*>z8I7@-7?#E+(CYRvGodnW0M z{CF{|^edrfZMs&0%k_Hi=-YQAnc7iSu--$e6nVC}^bY|)6JV6Cl90bI3zAm z_^6W>2Zm%D`2r`!Kdu}@Bs1NQVuUYl8=v(a`;}A(R(ep--*?-cj%@4Z3g=jZGO3q; zdz|zvH$QdKX#vkvpBY-B(|w!c32G^_+{TMhhYmOULr->N-Dq1brpvF!|6)emc?4gL zR-U>iHqiD3&y9`Q_*ans*t%G(dVozdiG&rp#pB&M1mo28u_mhDvo3lZD)-v1p@7w= zM}G%ABtmCz@?jioQDC+~cG~V&1L)$zRq#5T96y@hAe-ct(FArBH(KX)8u9Ka2xj&0 z9re=+d|s#ve9T}V_USV=6Y%=X>~Zwf)+;Z4WJJfYDRfb--3KTjC(Zsm1S%iiJifP_ z8}|)v+$@AR8Pd~iGsT86pYE@_c6WB7uTIcD*l!cHa?C z7GbUBg?Bw)Nj*1L1V{M0a7iB2dh;o&2|Fr_E^81`z{ z{~ql9%yOOHra#q@H|B&0oX|B;4Ro@DDFYhtx6IvtJ`<8uzaQ%yIaLFJvC#_2VTn>E z&38xTF@s=0ug3jN-CVIz_qo?K(aSjB7So~{BW7u*F%iTuIBT=>PEj&&=UabX6|emK zc@SDZbJre}I$+#kG3<<%Ae=tn z&1|YJs<)x&ddi)&?+MmCd(zD@Q&1ybzwqCcsA##fnm=!u8H^{o**7jXYznxMKu}t_ z64tvIR=Nm2wB6mARiO4h6KK5r8LfFSyDl`=h?aZ(&48-9IpDEOzR5_GArM8ZQm+K1?bA08$JQk%DDw38t}H&ND`PId#H6+bZR$*F`S1hBjyYRtd)Yp)uKVKGvSL#i)pa%?D@s#x?`q2{o_bMmtqlkqcM zi#MNQb!?&vkH0r~8;|2|I~GXa8nDM*O5nSKM9lXC9WB6~{MCXGi};Uup$Xr6goxQ3n1$o~E4_FH6-)7qG#6XWHtDfNtSBA-}P zpITtwwzWV7*v2I<>x7%(s~Kme-`^Q(x7(szAEum-p|GivSq(PfiM9NCOCqVElC~+cdK-x{-Oc>M~5^Mh<_c_aF3Li4;_IZ0+g6ZIW*7ly;HPpeRBBTaj}(zAG;elA7$Re`k>SvRZYSZSL;Fpi<*Ql zZP{`nJqFq4r9yYR6*T%Bxzajy#WvBs@Vq@U;pX!~Ioe1j=|G){P8Qljrw7ukM*!B9 zg!%a!c#tl4-Flj{=uC!Ph(-xqBM(6NFix zQ;6KHVdoDbQ%n~*Ekn2}Ag_z&Gt8a1cu1r_B`cBkn-<+-aDjFY(eIst{+b->UT#Tb z2@J>&B?-YtHCh;k9XebM)jQQ_JdzRNBNO5lxvv2oY{8$Qwn@}SGB8|gPyPC^gh1>H znMaJfIHt&fgcr|_VRYta)V>JgBNTaOhT2w@h z)cd+0r5tzO>)8+QrPWe~Oe%8AZtc4K$k1a%7W1NaVcrdioP0IgaX*f*tjaCq3#DHS zB;8qG%%a5CpFsDE;Kq!iqM|RZA<34Yy9c?{6EoR60`XtH5;-x(b#hb;h0 zK&G=-NGmwE#z>DGC90>f$t3eWb^2L~9}6wV!G5UEDWPx9HU6_C2HG10hz}h|U2hEl zjqtnw_76?(aAslJ6CUh9Z{=>9u*YH@yi$K?*7cJmh&E4;3;4mTkw$9)x4W+c%|&XT7cC_El& za@C%q!f)CS8~Pu7JvfNwShvCvPRu1+BWgkQ#w~+y-e6kGN{Fg64VM#7VtUvJy1xXD zX?MopUgqp{hVCT@pCH0-OWr9*eB?Md<23NJCj=VkkA%!ddR%pO)$kR@)Vg#ysF5&x zfJJlC8j@mu$ghK`!hZ>mQUMK2u7A@AJ`GVZD``4t*+Au2dOf7Jj%5gC&!<2(l^>P1 zY&|a~k$Ivvq>KD9&-uDNDcDsJ?aNS@1QwTbKaclx_c&+#$l{*kd6F)AQdS6968QVb z?bRZJ-zEB#P1T{^YhAU%xN=Sz`i6Y>{B*ngj)BKN7eHC2%8}i`Z}`Z_`9%^%Jn%aV+`kZ(kO4?@ki1>$G0u(NO77%N#*#t{Sf>d zns(&ZD684hkI?&m7oa`WA}TRQdZeO2z){U;s2aR!i9t_EAIZwg=;^rykBsro=ZVfU z=ihP#3lma!LVqlB!o8wY#H|=A(udMZtxcB-n=Y~u5`!YY@SSIsqq2t)%Q52|7y zcO6|8K#F-Hu(lEoW+C$|-WR7bEWkeTU6(oH<@t zae|-#R`?p92zLuQSSmAEz1I73*z9bF>A4L!kXaBk0AxE?%Csh`$@zAG1_K!z?JK}l zj_&iJ3`T?mQfr9qfmzmsd%%xi5%v$fIDH&P=H6KP*?rgy?4__K5^JXUtQ%xJC@bPZ((M_NYcI;qC&m*wj>zq((Ec5YT1#NB_B)Sx{hej4_Wq z#KQ#U^bw{i^L}98FoD?OskNza?1%(R`voPk8F!LbxD;XD;>BZ#kO{}pYavXb( ze|o#*h^O&$=}I$anT8iqv{cxGz>QK_H`f5WMYV}->><)j%Gwjl8xvy41?;BXw6(*$ zb~pr(IrguU-(Ck_bc5H3@PWMW)8(-G;W&0z$<5}&6f}%l z!?T5fD2*88fxYE!CQ-kTxCIw(4(B01K#>$LOCQMN6lnGdZZVMvF_8fAG9^l!otOr& zD?Z#T%31FY6NrzRNPv1tJAg4ZL@k591wRx9KeSN=Xg7k_?m=DaI}bBAKmWKQk~a3n zZZnv32>H{$bc(#a+S($D>onlHb{KFPlBZY3|K?`>If)6*x@`{RRe?~6_VZrNr|CNZ6N(zgBBf*VH*_-fm zfWQJ(90QFMVy^$0&p#f%{*STnKgK9OU&k7M4MC)6|A_o2=KC5GdHgjCLRQLyn0)IEq5sFyo-Z=YTzrz_}B3NBY8ul zQiEWMIso{K2*6KZUbF$gT3K;%@a#9u%a0H!u)MNzTjU%fss#v+_7_?9{u#U!gvdA= zV!dWcl`IaXi;ozH(VT{9QkAu}hy6Dpa<2&$As1l>Nc=_AYh+-1z~rTSVL}2HYs)qk zq*0CLygm%Xkc|3o|BbgnhnhL8$o~EPeez^E<0{F&5)c>wK#nwwmkElBLWOWpg@cQG zK5pleXyn>1Fulx)?XC}A%-KwTSZw}L{t>&6#w z$m{=RV*W}7qL%;b_V?dP-fF}?)V9;gdP?&OK>sE1?n4S1iv0iXri~_K#q!sNNlXcm z8E7;;8*Jt=c)(vidie@LFW>n+wRFjheYSw*O3ZCUjBRa5M~nSVjcD zh7|Ar)S#fip{=*CiR|P4iz+!}um}EWRJ9ZR-&9|tHtPQ`8QS}Kyy_M&toDC0NzP6) z3sVIFts~*0H~^+KbI|ZL0Mj$?NDiZ@{LD{uql$Q@sB=241g153QO5Y|O@955%?+!l ztEipVwSF*K-GIP`6x+D~(>pp8KsO|@9|t^)mTO(gJS4@D8Os2mvH?P`svi&tRoUPS z95AYaGhOJ{iE`LCuYv=uRes`ufsf+QV%Camz%wQ8HmQO`9pHdWLgKOeeWJ5EFpq*6 zF!CldMQtPKu8Vw)7yz)tAO0~yZPQ{8$woCgFs~}kY``0IaL?W$tAsOUcLr)%LA??~ zU5(bi0mRrc`0AL=q2p&BiY*F#j(W-3V z9q2Ksox_TCt8MAbv7M_++=-6jaISyN1v-SMe(vH0C9sfK8l&h<0j8xQ4AyNIGMQ(5 zJU>in$4SL+f<8}RU^m`+IDMb4q+~aULL`3^7O@P`8dC`_!f&XtEXywxtEP=Oxv*Wm zJzTDhTq%Hk#MTR++>Q}fdS{+BmH?VjJ*NA_QQdhVB{eT49sGh#Zg(!}Q^Ctfk9$!p zCRx0RP^_=+qejmdGZN_t+nA?Zu3QcW3%%3y#veM^#GF2dGG-`I?3@;p@wmWGysaz? zF&>)(trRb=i5KdEf)N+fW=uweh>@{$vy_*yRE$O>Mj|XLA|0&$NIs-1h>Hsagz*nFnel_J4CdSGNM=rL5c=Wuv zpI67iCfXkb8}?`jz0D$$CjtizU6fJ?Bhwe(gp8XUL6+pIa<7nI#ycV64`ICwP%Ksq zeb2)Ojmd~ZjvG6ujHz^9Ac!R4Clf3O7a_yKSSsSdkQN_+=n;uQFJiDQo4=4}LQJ_N zDY>Jbr|ebg$f*9B&l;n7h}wA|r5$eRCNG5femBNa${b`|XsL$=UNH=IIf=G*dP+oC zTtrGln>=DDiv|4{5kD`LaQ8&l^jCokE$x(*S)b#<0af$7kT8&w$e>{vv-~#;H_Nj&ap+-@b=9N9Sr~D<7`kv7ap`em2^baIhap18%$p7g7&$3` zk${j#DvLX%7NZssDK6M7Q~y;EI8rW>e`lM-o3H9euAJAdK8AX(?gd*$>>p9zpH4t8 zLayj1$WR(UDY6sxt3cI{T~4w;bVRyeg%6jEk+E%5rCfyXqDa{bk`5P&62l@A7%_5k zp~6lDH?po?)|V*)ha8NCL8hCfq9};Vc&=Wy@kS+2IZrwNLSK}{!u(E6vVTn30XfN+ zTGe0q++DD&>`wV!Em*E!rCgaIM{MVnE-pkPAeADvQ&=G4QLYThdR-$cJ>MOfgfT`g zc%(dyfU)zNG9P1u-w%{tUN3y5U}R{~{&L7S5J3!L2CTw~!b;{*iouGSxv3y2140cU zx*ma!dyl2eK?RG;NY*x*PJpq*d#=t46FegA2jRItQp^EbDx8<{HA)e}45BhbTW}EL zS3jADVesMN{)_e3+LjNJ36kmG5++qw_bSVR4vr{%RYFtqvK`2$WVQTu9N)qVyfz1Q zaz9bdN$Vs4?Ieks+DmLccs2z6`%S9vVJg&l}>fy`ZpL$TRvtwMoB${K!fw KN_-bH3iv;Dt$|qp literal 0 HcmV?d00001 diff --git a/docs/images/quick_7.png b/docs/images/quick_7.png new file mode 100644 index 0000000000000000000000000000000000000000..24e6c72ffd0518a802f834e032cf390d664461bc GIT binary patch literal 265959 zcmZ^~WmH>H*S3on2_77ZLsBd_6nA$hQYh{YE$;4?06|(DS_%~R;uc(5yimLpm*R5r zyyJVvH_kK8`IoViy|VXOb4|JDwc@okmGE(?aZylE@Ku!Mbx}|-f1seC<6>hWuOt}^ z!BJ2cP*mh)^!-o|dvU_v*#`gFSPz`My>h<_)?(vd%0{Rvk4$!f6SQ@ti3Z3&e_z;n z_qDod?B~~}@5?4D?Z21b)Zr$C5s@@=!ZQyMEIEk7`*GomqrLp=(3}C~Mot9F=jjV^ zr?U*(h!=z}#r;ly_eTErhYcY5|KC@={bPXt@0RCBELoxA;!$R)|K8C)$oak6#{%XOLEekOrAECs9 z_Bd>&|9;lfdlhiBE^fcT>FvW!aESWcfPlvT3|!4g1ZJrC9FxCE&4ejyL`g$^SL~r}Kp{CL(O2 z_-zYH-2Y6YA4eQepU?G{E6Ph%r2dE<;6umwk+Qd`AFe-nMPipVTvaFgnZ_%Eb^8&ODyhK|6_8o!sx_t!%8X@bN_94 z?yjhzVGf!EF_s=h;4;bzm{oop1Jgq@HX1Gr6haRg!r8(Yj4qV-V=V?mnOA5dU)>`F zxD}NRMm{iupPw4ZMim&7sSxPisw~6Wun-)U!pt*WA1JX*?qx$Q%9mUSv#KPDz>3lX z;zpBGyd8}D+Mo2rt{xCUvi1X3TpNVTpY=Vl?rBS)`?P=|cis2@Ja44_YYisJn7IGm zCY?=SV6S$ZgTQEXAxu{O^!`!b&_GAuRGxvT$CK8oj`G(pi2*BK`6%HGp=kZ|A$;ug zm!xxVkgr3e4GPHvL?Kl5TxAPlte1PpiOS&Em{DfguZPPaIT$n8xacD{%#HdJj9G87 zK+=@#AOOb4Km939;dyqlu<%U1WT{XRrm#&`TzP33pFbeC&ct=0GN3&PvA8LbB(v=F zxPmicHeq+rSU*yi&!My$CX_RbA>IuT*$EuEKNiV+2c>BeEJ zhGX;S6bexN97f4graJ1}2P?c&p~K_uNX+_`I=9l9v+hey0k@I5?3|EV-wnsY4jA(lLW`M; zrwERokI?NbChGg}CQIs6Z2MlFBxQ`*7#iAuo)jOZn@>tCYtmq419!ISSEr$pNmN9^ zqnza0i@iABUK$iGN=8u)FC-!caxbl+4_rT^RM@|WC=>~d=mTu)cK#7!63qVXG~K7d z#NePvDBidD=^L|K%sN}YcG#vswIEX;&7d@Pj%@IvrAkPa3Nb}=ng_sVD5z89uRVkF zZwG1c8U#0@E%)Jdp2a7{ipZq=+Bm{?hip71rz~Bvk81M>q)nnmY}!WXhJ_WPY_JeF zTFyx}e+2SQD#*y7hIALKt-X7fXnFVfV?&C*_|OcURKEur5P|Cjk0C+DWJ2px`OjMN zg9p`sGT@5*uYcyHSgQl1aokOq2JW<>2lD8V^Vi%vOLpC0nBPBi*Ly-@cK=azIj)`) zhECCIkbk=0EI5yUUe#FM`2a4%j+MRPq_?lO!R5%MM&M!Bb&0%TiJ+0{ZCD}Dn}y_+ zgUK#w{WHFEb=Ekn$#TE@G}UuVv9ur*$YK7t!0B>#%ana<*-Xqg_V-ltnp7*k(r6SKS%N&QoP}i7f zP}?H)-^`M*?EH>l(5-#)SH#5bPgPpMc6L@|D2e`xr zu2{Fj!5Bk;$!uF=eM7_af&xM&SE@AB{i@&uPR(1=`yhddh`45h((#HDlF!1SK#4(Sl4NjYbi;SmRNcT=kERRtJtN^pdVdD~E zs40Ll1us9dkBp$Gp6#;sDVuY!6Nsw?&l%AAUA=Af$&KgJKyFw&i0`rK>({Rr>uouj zUA>Fo1_pwmy_iJU7)C8_T)w{^LtKXK4(2It%;5a?LS&g_EKn7acntT_+l#!`t(0re ze}aa3Iy#yH)V6Sh;f{&;1znEfW3PABo;Sl)!5R)Pn@!qrp`u8JQ*`p-}<+b z{z*|en!r~sL4+u8OHDI!qmSO?$3@HQ!Y*Eb#&C4y&DF& zwf>WIbE34{yCV2&D3sx_*0=F8d%-%Ft-*exxTPg^txNKqp(C?l@M-A9Uduc6iy5KP z#`x=XUwY?*hM<#*w?=+*-yhbWA5mq$n_Kt0y)$&}XTKR0UekE`WG248s+rZ0Ikn2_ zduCtIm8rsv{`oFv)$1N#y}tSEa#BaLId?Y`O9dy39oN#app>XP6rA%WVcC5YHo z)Oc1i?hBqB^eYtbZ@4yGdCexP`kk~&&w${=j|r#ZQR?zQ55U_zp#F~rn|~w;eBeW6 z;^bs}*iDS+Oty__ZGCzoPkUT$4+v~FQiZ99)2f2?LeW!HkK23x-RP!M_!wr|@WU8)$rpVGiFp;(iw=7QN7L4brVBg= z=Q+*}JHed{MoZ_Q4K(Tst%~-6;)W)Q*lgIuPD~S?XW+Y1Zx}yDPWyW@3N3u^_n|a7 zn~tWzx36|Jst4%yivuC?yNwX!gG_en=2=fjEW+xJ%Q=k2tI0XmwCv3RhBuji6TjIR zd@ML7bsZ=R`+3rtrRp1XbrrIzP0(OyKDMl&2NuR!ntMGk7Njr8xjSoXODi@n^JL@Hiu-{@uf`s0`>T2PbXf}(@dG~i~b5P^*^2+1#5AS3) zh|zd=wr$9+N)tL~m-cx&?hEDK9j2I;GKKSy6Wst|x*K}GjylXX$J7hqb$r|N%QW!l zXVtr=jGtfM)Cc@iee=x(@z;RE=(`iETCnlC*p+qH{mZ0GLup4GyFJEs>k^l-8*N;S z!6jYkl&#tThqMeAtPLP0c7%DyqE&r^9aV5TuFDI)Wrl-SE-zp_*k_vcAKywxd6#H+ zF>EaF7|pzrmre|?#y53E`j&k>-iGDj(dd8UpZK9m4feeDSDbbexrlLNeoA2xk*GL` z4pW3>5C2trqfdTAv#a>-@xXKBG)M z&vAyOI$~O23Kj+E9LM{DjyC*ziKMh=5oJuct+CMzXLN&={LvDy9+r{sSY_gePaa|= z0M(EVNV*{B!;3R1Yu(f}cq_|7%_N!IryV9OKwVr@L|X=o_oMEIpdc$sVA7I=NQNSK z(nN%Is8M~>+VkDVVxME%QmE%^0(ZtgS$dot#0xzTlrn8x7+RS$cu<6R+JuM;`Tu-C zlgRl9AHfig(bmSlim%GGoFyr(3Gm~v>nulwLN!9ZJs2RpLs z-F7d6?!2Wers`O_-AGPudMeuf7OMh?Ls*H5MR-#CTAsDKIxPT(-Wj^n*P8)dS7r1b zFg=*l_EF2Pv5&$Sx0AsI7LV}$3HnDhw~hlz#p7qOPd_6(HhyS+?5)n`VQjd5`O?n< zkFz!1c^ywjbLp`pbSGsmRJtl#SUAprk(BP z69~Djs2Y+~>Y=_7<|^DAz#h(!bh76vcRFMsJXuy#f1AW<(*Jjm!qeu2`8{+vN%R8l zLbumD(V=1!-1t|`f8oJD>7UQE_`q0#Gu(u>>>&CVVEN(Aw(z9g(|hPZ{P)WN(NI>X z0|i`}fgTa=sQXLZ0`YZW#3# zaG^jfuHd)()Xn||CLX3z(7nlBqXQH!W|c_=wyg}Qe@wgdlA|?l8$Y97Rn&V8%I2HV zEY8kP8I9WJ*FS7;%X{b=|6j=^H4Y*mATR#1;`8UU%1YIQ#6*>{*?a^9h(|Hb&Q5K4 z()AdJO`!{+w7EJozIl}V`}gnXnws*Gl1OjXVozi(Cj%aAGx3_SdNn$ByR0RAwopO_ z6x_KuMjNamVh8I0jj(#9!%mJM`yw|Yq0hq$Ubj2h7mLV!HC@!q!W5pg@?iDwN)JZ&yseVz zNkX)mu=xntKWfy6l0~8lsC}Zu(jqF(pefSwY{v|t&PBJhY0HEIo{c+OSq7kVJOh(M z_1SHZ`2Feyt<;+At4vvWICdG3b#PEXaAKpUw;u)rS{sw6Y(X@^iJO~DOz9A$Kt$a8 z!E7=Q+h20RRZyioTDO&qALsiP^iLfs-+Cv#EoIz%To}2AvcSm~6;F?GE5mU@5O#zu6cNf zP4U~ukpJXkYoXWoOLf*S%%6t8A?wO#FOU z$(EXU%3^soNSpl|5F|l((WUi}=BY|H4Jk>Q`Q#H2OOn68tQC|a<}`}qt-5rW*mgV_ z*jCjF{r0?3aqDR)we_@5dOiD8H|2eAD5df5x@#y@a$Vu|fk&6D!JHAKA$&8A(7Y6` zaIrUSoH3jOvqUBW<-IC9&~n}M17^`ICL`^zDHL`=`M1t^ayU*@eyiYT+=|M9{- zC@DrbiWwMKNT|?6{crGzg!})DK;n~zYJ;}P$SqlvMsB?!<}uZ+J3yfPy^V(VX0d%1$pF{kr>Q&sfLt~HlF|G>s?+P7LB4a7g( z?k)MgSmTi^Vyqb3@P8FA1}NfmfK;nKc$gqaQtE$Fh5(8smzd!Dt4*hg9KIZZbY);a z1S!H@kl{vat8QH1gSolwj#t|r9`+85Yb_+RYu=7&h1_8`yKadPAXW4gA^Uz$m&SQ` zIil?q7|fVCS=;>~tj>eWn(Oqj_SBvqh{M%L*gpc-V9FWjwSBJBOliL6Yw02wt4HQw z>M+W_-}*GwQ6o5*pcuQ?4w2a)Z8VUkZvJ+ZfR7RB8#dA+7_yK`&&x}6ka5P(LcRCd zx}Uq35m02oeOu7`IACriz%{gZ4>+F+O_~eN4~_?Du@z%)0r%uFVBwTl+Z4o)4|}{{ zE7R1mSkcgkF@^6QK;49fJwDPzz}hbgi%mH4e({BH?C5GsyRtF5F;h(dRa7J-U;pqx zpLU8NbBu&@P9DrQ4gnxz9$eU+0Lre#L3T>8sff2>$0NQk!wBt7O6fBelTMj)vm3Aj45C`>T8e|`G#WkwJp+gB^08ddp2OW z-8p;E@ku?>$_S5D|5Py{kyLb%22aLf8BOcc}54y7Q zKP_*g58OTujacM1?^5dRkz2xXk8qoh&Ns8(nGg`nK^oHPbFtIuBym;c1jacF8(+(U z4!3T4Enhy(TlQ(vIX1lL-66BZ+Bbdne7$f{srn`aerwB0^tsIKHrBW?h&tdisrNkd zc=Cq~b)Pwj#gR)e_t}gkiNwm{<-czB2@xsG%#&g!_JTuJUtVS&!a*LgkbI8aGqwK1 zkl)|_c)GlcKUf^NGXZ}qNaZ0K(j3l-(|Br>TspIK(xW0AGrcoM`F_E?mQ;Wli8M( zo98Zhru5&!28M>-!yiwWN#Z@$?(XhI4TrW_JFh>9#os-4x~jZPm3&d+7}8I~;}!t3{17!) z&e;(?6&ge(^>l83J8vB*I#*YH;4kt+OUl5sF@RV2*9ZP0IjPOL=h|>|v&j7Y3iBt9 z6tjP^4`*VMifgcB@yDOf+bDbNe71~gyHhMv?iO!2x9Xb&#lG0&(dZ;G$h9XDHvOW7Ko>4d7>$5wo3#Kxdlo+n$ z$CZ9}YuZxv$vThTW~)mlh5o3-(hbvq)l%`E?p4K43&Ja22s~_)3P$O;Jd9N*%9=Nx z5r=1B&?a&(wtw`u*wagbY=+!KSEh1QEj`@2ao8V0#Pt#VhwwaK+aDsO7r{S-vaASw zW(CuB8>K^dGj|k{UYcqVZgHyq{hQNdA$eQ!5nv|+OKc0obDaFmV>Ma-VH}5UyMTVR z*KzKR`Bc>*>bz82{L7LAg+x0&SRNy}_jHQJ?;D-*Vy8-6S;~;2#`c%|iz+7;5!y0f z?<}4o7SW+vXYDun{n{&TpVrn?HEkuF7pi-l#3l>#7^r9%a+K5R8Zhw2KEE2X|MQzX zz+a^m{$-8^oCr@!%+g{Cb&)y{J}#C*C)4=a=HwJtleLxPKwnIzkeOMU;=a*l#y-7) z#@&AR5td3@&R?Q3+;DR~_9Z##>|t)oqmeziyESG>1MdqI6|26m3h1&Vqm>I)4~12J z$xNbKS6p{MGw2E3NW{P~$_-Bkkfy(++SQ)Y;WiV`Lnyr!{Q=!E7z|umC$9+l#hL4r zav^&UvgaA>HFv4F%r#Ii&_VqEM4-!Uuf4sStz&5Gb+{MVTJygmDY8RK_V4QIY%?*i z_3tQ0j<`SH<72Qq6EQUnji|?O229M`q@)357LrnaD%xSB?tVKuHD&+wz*hOS5(0?q z>=b88$Iekl294{jc5Xwn517%>(JUmM83^E)_qlQ}2tEItTG9@0G&vYDGpP?COG51E z2-CU%oP$ut_keQ0bBizzHGDMH89|#lsoZi`hLpe{u`xRXUi-f+1c$$pw-(i542z5! z>rAol!|~8S(kK%=U`>8`M%8Ia53~r7cAqFxGK})9!N#JNw{-gzXVORs!!h{qnTuwc zxx~ipiE?+FiqcGJQQYR@-`cx&S7`=oRZi>uqwCGzi4oKpBd|-GJ2M_h((DPNa$rXQVQ{f`3)6y zen#onE(DH3PzL4Wu#2?QkWNSE?OP2tXi;e^906tLN9vQsGswoL?iV1k+-4;@MNU?^ zV@7ttaJj}?N@{95tEgJmal{AgKb&@h`tPkM?eFC%Z!_T>ZVF^S3u9mRo|mj#7s{7l zvMHGUWfl^%4w)<{Y*Hi|Xs%6`iC~mMEuioBz_|%-Y^1e@tc8EnHU;4hB86IjToKB` zPhi%ru4&ekp46{NvJ<1x?R|s8Up@e42V@KL^J>O?NY%jqvH&V9F@e)Ha@wQ3r_eb2 z+cRMTU4B+Vv66kqRPTMU1JT-&f|?S~S)5ppl!E*|l-{(|zu-3Y9Ajl2<1~*ST4P1T z^;9p$1aKBJLFyRUhP&+gooV&B3fJj$Hhfg?49b(&wx%F_->55g6l9-3XU!i0&Gc+< z_;dW$bzYkg6orGcSJmw-Xk4o$fEw;BKOgB2*WO1|Chn2&lG%Ha{%WLK)10mI=gN@> zghF1-om=yJB5V@o=Se$>IoU?)Ip0)uMyL`OhrpuiWJO@8|*ekgwP++^LfVV;evtW?3#@;v$GL#9_7*u($?1KNV~TG z*oh=^WHn{ z<7xJE%Wd5W_RUy(L^+^wvnKY~q7P1TI1qaIFr(NmTAj)z_3Ybxs`hWpf9lr(}vV0sg|-&w`d|zY7gceXWb7Z_jwcPs3z?-oNlE zPc@IXM=WzozB8|Pl{z4;X2H@6lN;`;J{&Rbj~M%AqdLyYp1TuW;WHy)C1*)A_MuLX zDrC^p!{r4pidb*FL|)u>jrCIdS9sZDRKugoN+;BEh}H2|LuPeRld0FBfjYe)IAzGs z>ybNP^C3e)$CE4*lj=%bc(heog$pmcnql?|sMR?=3P*pXtDf#L`D%~T$}YLzJGFXm zBrTCgUwVMry405)@$mX@a4vJlr#)#Y%3hEjIakU2@$Y>Nu|E^kunChgp;b@1EVc*N{V0K7!c zDM!lv6??!{`sUB%542YH2R2#U`5Mmab^-kPf^_coQ=dY(bA*zMNzvZ}9N*ayjeaec z-aKsNclf`!MQ1{9uME=+Hox8Y?`VpU-cR2HK_JlY*$)S;$L-G87-%9Xi3hL$lPS&2 z@N#1eH=o@=F{acGBvXHVKHZ0ga5pWO<2V^SKEK-7rOCbBHicgCf~o9QDN zuon!sx)@OI2N5OZHK1+DOQ9!(;FKObAcS^p(W56!d&c`bQVzbB3 zw_-cUkdgJzvs>PYi`3&CVz`M!Y-ev&b>XXt9R~gG5O5EP!;{5;p%JrWDh_Xfi0j#x zsaD!!iw|nha56?61h78#bMyFsVY-DVma-Vgk~hka999IHoHNz0*r$o?;-$|^UApw` z9`WGQ@b_elv8?s5V@~kwlL!5W!9nU=I;urYhRR^GFSqs#_wzK9V>bG zJOga8v_XC4+J6_qHiP1@%N?1RT5IH-riVQ|dCZC`z~0%UwP=6dXG{(VZTirk)nFH6 zYUc}Ms-E$c)iw|$&wZ_H;0|o2vqp>>bDF-+{JxUPtBIlAX(=np%A$BrwP%RwV94(s zf|CjLT59i#^y5?;+aR{%23bzw1s|@j#qRLKUsuGje*KEVNZE|76b$%RMvR|wlmXpi z&m7roIY*5&++)}6N^w-@GCnvSdWmC)Mw*6U2M(Zd!+D|BvL_d^m2 z)~OPczFol)7q&TUADTYs>%%TgV7l;f6#?p^;?De{UdG1xdvs-XDziopto2v6n())*pqtXuVG~R*#t$OsYI7kM-Qc?(pSfMIILqi~L z?%hKw3|TzCsb9WzJ&%S@3!yB2!|bAtbJkM3!Sr{|PrJn=PiJB@qV?Yj?lKL%4kk_t zAt-Mcl;5d> z*)7Nxn7-cj_5bbOJ)!*yukmbP$i3d_U+nE3QksL)?5>1fgQOf^+Oie?5-_ocW7{_t z%+G{q@;wT0z8ysNuzpRsZ7&iDn0n!>@w8!Iv?bGG%45uMe;dFCLV|l$cgflOuFk42 z)d=`0IIZ@VGXamM;JXh|qH5Xv9Ri4}e^b*L*t{1RWL2h3Nv;P4yGG-vp>f6Sh(Gg> zjRdLT=X&o9)NDP%*nNEYc0Dp!uqaUyM1`Lmu)8x^b4i&*a+y9-+eI0YPvnf|zfRoc zGAn(w^mzgE|soh`gE@(a$*~%Y)bP)Q~ z+1TRS-p1CVg;W(Xu~S;c0hiKIV?-xPU50E*yCYMYB~3N)W?EhFZSh>I(oz>UzCWBQ z#ZuPgK!inmc!a(y|7_CW{Pc6M9NqX`Ch6W{xqXh;R_1u#+?cmms&})zsY%LVX{i<0 z*xvjpbZ_}ausnX2AjLtG<5O3hZ37!AjuD0&_luCOa929?|Ldin?kg)RnR1k8S69{b z=1gbzfv&Eu+q=6E$n;bR(%LWg=l1^|+MKKf|M>aSlW~9)N62LrxrKJy4{FMaEOE6Y zn!;<2-Q=TOgvLHe1R5h-oyd+Ltz;1VB;;g$|L90T-cN!_-3VFS7Ch}2++GyRK6&f| zt`3Pp9#-WZ7(%#S!D2NBJle1H1cLIXLjDr4Wp1DL2*3PI(S3utH#QE62u_gyg2weg zO|0pS23h@6{1}1lK>t6kR7RetnwS_inP%pRITY@C<0BPI$(BQNJix^ntxnEmaT6KG z*J=SkS0`;~&mDQZjy#CKp5Ns5Zct%yZP@WVxwNNbBQ_EeK+XxWcfM3>iC$e@ z4JxCgr2Kt%@U5b&>(hTWvtzYPRJ?B37X}M7RT-h^%a(}!{?BlNCrBKYIZ=rd)?p9^ zCuT9Pc9eRuED?~j42l13Ejw_f3Gv=n@S4mHY41XGLaD&MaCxt&AMs2i!B~Z@X~1rl zPWd{&9d(o34hONdv6P8eBY#^`|6AgWc!6?925fhjwkC01ujUXHi8NSYrB0`^Vq1rw zHT!Yp{_GY2D}ty;TfPdCjO_1t;dkC^y-oHaeweSS%Z+cpG?EWEsvT7HO1Qd&u>d^jFo0b zH@EU{-zds}-{ipx$R29bho;MG!s+Dzv}(aUjh+u?vPDFa1%vJ+#C$oNTY&zalWXf0 zp_&6eofuS;i}v~@PG)9aCuiqcPUsxx91|tLGMBAFq0iFG+k5}yWX}at7O(YRO-Ih# zy5#QLK1-4fC(Q#l$K5UZ2guiN-@^$XZYlmek9MC<>{bU|t_}C%LhhGWZsx?YF8;(y z^>)8J+HTCa%|0LO?n*6LzsEJdSt7T3Xh#h}oRg}FA>q)vB2+A_8$sTX*v7;I8&M5Y zR9ln;=(zYBvq0hi55j`G+3Y3gX0|DT#3Mi|bl2%=`)&j&(j-D%%IM+R z_(+-#6>m&n#I9YGqW7;Trz2;{McYYzg#@BCa!NwuwqtFmY6z>n!)2iURH;uo0;*2h zFW@mpaBGV{o=3bhJtaIRDG)J+z2Xz~r7Z+=jNG8^%Qu)YzU_O7Gl{EzWnma%^X^4| z9~XFo-@N$smQcoxtKlz@YwDUr3-5+$n*v7M+(A_}j2OIT>?fE4euCt3K&4H_}g~%8<;sDz|f&2&WcN1?HYbit(3f4?tz#H@L z87ghl>uZS5P?nu29aYa}ix2UG3MiZ+A@X&4j6b&u8JX%q8m%XS5PUH!nKb(cA=FNKy9%8-fF!`fp0~hI~ zTArFLNjBvRB&Ped;?Jd7UT_rny3J`#o$bR_#>U~kpVTmcAiw{F!nlOEGEW7Lk4qR# zM`N*8I}8Oqw)p}ow-TTAvu-z6@J565f06Il=i~E>i?;0yTbIRFVmT^!d2joN1Ec~w zP|wLZrW6_~HKCN6j<9P>b6MHF-4o`0{d!bv{g%M3ZHMq?lj1qN%C!V$JYeEleAxtJ0KucyS)R4)`WpK0X z!>QH-O2lOj2fqF;-;yJX0|ywlj6j3_#KfHFK8@5O)Vm9~){cH29SWRfkm*vEzQiK} zMNwkd zi9m}o_u~8W$!>h847*T%`s0gUdRYa$^cfj}gi>b)&}24}lP4dd%3?A(%J@n#8*r!) z6D(l^O<0VlRWE~RXeZ#Q`o*Iyd;oJ#Ran}QMF#U8KaKnYX7A`|!~08zaIcQUV=O6= z+5>P{)ACj}iPUsx(AxOr^OFv>9mVDaO&rR9cn1{BQEkfdzCHf9)aCarsp%@OZyvD9 zGKsHT5Bh5XjUvwl5;rL0UpomU9s;LC*Js!2V_3)JGFtGBgx*>3PpDHah0n50XygWx z2CQ5*JHC-yX4A-R6u^m!ips>m4qOaFjdbmFHLe$QF|N;QD$}~V650En6)xL@F-(rp z@w|sBaKg6MomWlxn5Od#a?jKW7S*foBRVL2Ec-V1q)SyMnE1uyfe|fvig#M7rxnpK ziO%%i-9PHdl0hOV^u(R=YkPQ7KCY@AebK>A$NuIKU&5Y+lF%mMQYUHu`y#sJ`CKSj z>hQjU``&cX8xw+;EI?e|bWY8TEs-Bb)ms!&-B6yVlWwc|Dr1+mOI&<>e55D-7Ts^z zw*eV0ffak`GPbp~0SNKuYP-XcjD>uyYhKNyCi3TUnws4G{mh-vzCKyWpov(&jav#w zC#OvtP>(hnfy$Z&h;rLvOy*6E>15*!%q_MH_kDo_dQJhADdpvg- zdfv@v^iofB5|J{XEw1*()9vcUf7QGF!a?EPI9=n{ZjW5(XSb%_k4><$E0UkN!sk+C?i5}CE;%p*+MJ{n#1>mb^i5^D>1biK+aFF?wN2U7 zeNem|!B_JrMPGh9oT?ofWoWRS_V$xPZ42T04^>=p%~%O87U4@`v-D_7e}A*3Qoy-x zVV5gmXwrBUaMg__)EvIR`jJQu6*PLj=|;_DquIOTgU&c)EGhiOkvx2oq2|)Klnoug zTD;gfBsFTpBqQ^N_*RBTQY-2SHFm*KqQ_!H!Px1QBpzR-{RZ7ZS%-TVhSHKsMjfbm zMJYj7UQ`hADd1BeBjQH_cs!>CuC}Xgz30tS-_exXGP^+(ZxeHIhY}qRjm|68u`iz*^8Z#!nmW_ln=54$SAuPbY|O1GJX88p=2luG+dJ`$jQCS0 z`|~JwKRe_{p=D2@5BNpWq>jlB%gKv&r7R!OetQ3spF<7XSUO^Bj)3`v@0@;TZd*fn z{ephJNM87rolgs_{<9sy`*Ugy5uqM!FK&L^Ig#lt_Af}d9SvzvM4ca3j;wCacho<= zPEJoJMs~SIrlyd-!jzlB-ydmsgz4&R{yZ$MdCT8FaGEv4fabW(uF!();XjL%Df32I zh;og*IC-WAUVv>OY|}=U_6;343Ek423yvdmDLu;l$4 zb9Vx+ZjzAgAfDnwE6q>rvSRoCJ#bFoOH~yMEdjCvij-V>w$6TE->a}eku6Y>=*?0v zHX9k8CDebvQ1J6*kSQLI+;HDOK^qx!0}8WT^v1(sxP;A(`mex3iK%t01 z^HE_Y6RWVyz=GDVaOP=ftgC&Rx=7UUL+f_}FxXL@65qr-35`~Xzq%45d(7%O3thgC zx1Hdczq&oDQWWa=*%e*jq|9g}Rk$S0Q^*pAcI5DCZ>RoYmM^6KJAL*_kEY0gwNPDL zvM%nC1FaG?!%0W?lcrN;q5~-2@>Rm<>{843f+SAD$o>$4Ts_|TOW=pjC6ZT$rW!6M z{YlY1(A+^9a^1F7h5!p5^l(Fmr94^NpZKMk5~==k%IVq`D+ko=}9V>)?C4 zkd{kbx2I7lPq)uG!y)&Oef!Y#RDD+eE2)sa$zz?6bri980V!D7y}ad#e^(p6aEtV0 zWzTy$q%An~p)*P@W?Z_oB^D3_$35STPpq!eZqi(fE3x0^ci$wMN=i%$cm3m%ysxG| zuhUwnE!iR}+70~Ukd|Zlm zP=^#<+%my#_Lr>WiN)Ua%7#SKVY%|2DX8CROQr!$$M&rq$yx21+~3`ZZwHOx)xj8w z%we$$gR-udLw0((P?@<6)IOnMP}#JJcMudV_)~7ZODF(cXqb{){3U(-h9Y(DvhwRf z+;fDFanc!9HhXf_0}wj~H8QEu<-DHc{ULV5o3O;gK6O*>$j?V%=e*S{c@BPYU7s*i z!R>^H72m#R4l#$!bLI`hf(Y;L8sL&`X(Q=JjadFlK7LWb$<_Mr8q>U|9C!9VDorzg zyd2!sGkT|Br$$$jYpwt0<+-?~l7wQ|jSV$A{H$48G0V-sH;>n-Qb;ucRt;0M#DuugX>G9p>@|9Dlu$dx0z zQJ7b`I(Tma&~GtUKH-2*VOpCoOdpzYEvp!SMYT_bRA0R(MfW^5qQ6oh>7wB)x|_!m z8+-c_n5^?eJfAvx8IWDgSwnz&XJ+_iMSFV!$E4Y4*KYsnojlh?1)2ASlA)CdZ^$`x z<;vO{{cH5Ifcj4nhu(wm^yxM8E4IpvfePqVqIN^ z9Qn3ipfYWfEl|(V(0dw~U9M??YY{4@i_=`yLZPVt@{EI03Ec}=pQrQ|3a*kjOty&X zgO`z{D_i8#%fD|48fpVxH@lVyl9qq}{_*_W%Wg7X#m$WyN!%9{XhA4To0`a~1;5QX zXntaHkvsaF_!E;?e6h8jeWS81>!v|!s3sV973?4ueqHjMB)EsR8ThGBHO3Lwh!)ql z5Ool&qe`qbprWD0>YiI}@0a8k%B#qS5Sioi@Hbsl$@4UEhUwt&)-*4Y;Rt{2bfg|1d|HGl|>Lk6)WJ6Su#rZppND%j}0`vYr3Qf$*UU3)d}d zLYI-cFSuol{O z*6tmUx#mVJeed3d=dY#(k2wutJ_uyw*Y)Zc@?;kK=3Yw>w7|PqU}H@vH_(5AAvPDQ z=u3Kcpc~d0`T77w*9Ce|dGYN`sI4orNT)&ARpi8j@L001h$^RTSaW2uz@638u!DPX}(DMz+b*e~bUeeOuS$dhC=;df+XY04DY^Mfe(R(!V=$1LU=m&p} z5o!m!iwz%HpCk$A7SD}b(R@;aMtmi3fjQ#z_=uL4lC{cYvAsGhGjVUW#B5;mUHQ~U zthzTo$WWPDU>3+P8i*JiPj#^;0!ao^*{#Ur64Lg;yCPIxjZUJw4$!^ZnqHZfcGZ04 zyVE3>1)+~Usst23ToR?eH$Rza=RV+YXO5;gKQU#-S=^R#ld!I0O#D*lQn zEjOBfQA5f8g-*}UxrslUn6Pp9;OD=aBd6H^RtW1iSEclR=s=P%J3TzD0Al2YnQRePE7syuu599pod82l&+ z=$8i%M$PH}Mss&A) zT`(_wGun52DN{6TN?-6pk}4{2n2DMAo`ZCKB@!#184#|iB;2<&NU#D1!d#XSB?z#I z`fb=1;X|9>Ec9M{QUu)rr|kUe9$h09=CBM$^n7l39G+abZrGc!-2ul&ACE&wN^>tn zn_8RJ76thl4%4boNc~6wbW_Ice^~$l=#IDA;=D&#CD;}OtWXEj4);O;NSbmHsk~n7 zO;xnE!aaZe`gOS6l!z=40R7U=6C#6OW~iH!{rrlgdNZL+i*)zc>vz1$1KL?HGCY&= zb(3a|2U=FO9+uwX47ncfkF(g&6vBy`kJ8e!5VKz>PG{#S_Ei9po1j6Feh3a0nqM%u z)s&kpLY9{sm_(91Ffy*n+QN`n&2z~7QY|fGchKvti@&FoVbFW3=)tA;bjL5s{JsHv zfAZV^)UY8}XIxy7Tcs*#Y=m_k6HPQlnm!>^P2Hf8|Kv*T1B^V5Sptw7Lh!y0N>B&a z;IzB=Jw*+FtJgDY0Hvaf&xjL)zsZLjIk>FlB+sL2V}fi?H9P0_(xA&~;yt`RjRnmb zE#3oDZ>#JkbJ5mD3Gn%ZE<5U-ijBsasJ1`Pw!#6?%xdskAQ-QAM7`iT+ibLed9U& zI9L&fD=h`upn*WfbmdyNBmP!S3PV$WOz++CV#`ow&qp1Y8Rwiid+)Wbb*;6o zy+t_n!IJdBbo&6ce?VJQMs}+$vH^AQt+jr#&gz>Cp*((?8%A8zHmMih1Uu%}sq*Sy zoe;>>pVrCKcl_cuUaVzY?lHTTr#c%qW;Lz9Lnip*Ix>f^aWpIte8JGhMp<^WI)4yN z9nNqKP_*~%jT~w?p*v|j+INqwne((iguEq9$wT&nTYERe8lt-6iZIpTz}d5!S#Q?y zaSZ;>{>Iy}$L~~xY;Fujn8#Z7{9+Lw8uPvCVaC-O&NDfb+gmh??gw22n}5P6sA72E z*_=kVQ5eWtX`u)6ZRz9f=bMZpO|;dGS??`V zl2ellkY2x!Y~*Ytd1e$NBVtNVQepmF4t*nl`uRe}sC!J4i$Psg`Zq=1m%(j)(37lb z-fdG=IrFS%B)#0(yLPpg+Gm(dvf`>+)7~0Xe6$7Z-wD_%?2eSm? zx3p^R?y~ZhkmVsODP7{OK;a+Zo=Ip5nWwdV56ha`vT>H&zB*W%L21M&7TjxV{Rjj+ zmLHerLnta|B~`R>i?H|4VQf;S*A4^^ymWN*^A)~dUp?acV`ml<*6(nCG+Q;bB!E^V zZ&vCxW9aw3d&TI4Q#>T=qZW(ina{VdMM;K9iG^wvKyd!>NzAD}Je1TQk4a%RvO8<3c$(0so<%tc5dN2Uq4M z1_-JPjufx{rYvm0<-2SVHwGrAGXzA{pvXa~V8WIR=aDrRN2>W$J&)A$v_O%3MQ{Eg zWDm(jO6`0hJLMM;vVO@292RdxuwC;Xx|Wj`dGb7AM6*O~^3LKe9qHts2ug}5$e_T1 zCt!U>(AZ=z!k{LW`p3c>qQA;NGjiuJLQ!M;xA>(f+mw$Ga9cNe#qx7!3sg+_5K@U` zFb0`IhJk*&a%Zm-t28LnT@~l3%!o%=6*R8t*j9gz3)YHqF16zk5a^$-HdInmvoc~N zp`(tsq`&|vHqaOUL!eBSwOin-R3ucoMn>%ko&_C*j0zo^U7&8!asE2FBP&5Mv9)DV zh{%Qav$ywauj8&pDCz{HsL|SpovM!upMBV3$+lry)aMk#9zmD!-fJZnACC}Y-_zJu zVfmHe_WV@8SzcbjjV53IM4R6Bv+RW2`#3{uTt7Ne-%pJt^n7>w7;7=R-}&drtlnFNRa!}+Ee;P zT0|0mSwz7nLeeGOuoU^RFD2>bbA^z2%GDhG_U2@cbhXW51BzM`l5C22&fuybmzD>n z{k5doRSgX~W5O@!>TSm985nZPa=oK=cNtX48q2LKtb-`Ry0V*H2L{Kid&1Jrw4r%g zasIvI-@Z`*)d3ZgnlvF->w5ch9?*0S&;=0YAmF2UcKK&#`#p6ej~EW2B08 zDO8;@Q$8oFeET+?{{=H8WybO8>D0tGMHv}XF|kkCb#>3huQhJFtuSw6$FCRe^%u6u zv8MT436Gh=rIqPrd)mqEHkBrR?c+Q~g;;vEjih@JsBpHtajjRQZryre4;X%LZ}bX| zVA4Wm_@e95wngVIjoaJ2|2guVO(O?g)p0KY>;*J%1 zT-SJQIE6^`Cm+3AEv$oaiXEb12DQa5qLnokK*-k4HK+8x@z?SY%?5((&`B7`9)lS&!zb z$++TLl`z2uDL(!@PYGJ4S^1il-b}(xqkZb4q?Jbj%xmza2uv6U5OWOK}GaLbj=## z<4wqVu76;Xm93f|PnRjphQ*%=+7HW4G1In^$eJ3$ob1gBm^Ek2P%a22NLs)pS0<*= zJm!((@NIAOslY-cbF|V&KXa*(&N8SH7mUSCU44_X(GOoZIsXQT_uCUw2_}-r{KmET z)mN<<{BJWezNVy?_Q6Uzo%tNCDKMVC0QqdY7b2=JUp`z^4ACy-FA>jr8%deeaYdTYWE2FlRe%-_x^3L#f-UBqa1vJ{1bc?0RzWk0G!-;k%r2L% zD-!xEE;djpTUT3;U*si00!gmvRC91(ApKrOhOiywzPnO<{N%%B3{On z=h&vK5Z2WVvR5)P{EA&VYam7Vdp-z+Dx|y-~HnqLYtOER&Gle^(Kv(9JQa8Savb`7Lsc3y!L$K7wt35rDVh|f=XHGu#? zU?;_gmq);+qo&>A`EMUUfDb@y2-jmDJ3G&9qM`z{r$_hiv)Wahi88D6$U`aU9bAn# z9C7Sesh7HlrJb2$hF6!0Xqt0dTD&(mw}{(n@P&{0IM28!&z?UNjMqV#ZiO4ycch&&qW{I1lXS5TOFCB|{M$zGM- zRl%i}l=3L;s6xXp=y(J#83iKA3dadWeOi=)Of`}DP~?@jYA>`Jti8-) zhU45c?_n1q_id2{e@{C9#dO>GDEb|8H5VT&pu(Qy8XxUcLbA$+kSca*#cp&K+9Rht z)@S5$pv)mAfm<^ZT0j<7;MDxf-t&NI_iEJ`SW93S@IKC}IXDn-q~QWKT)Y}N6tpGd z6B0+tW2g4w<3KBf#u_lIsN?$0EVumgQEg%=Iez{C9j zP}zbd3HF0dM1(MJ93eaKN*A6jkS5C$rWN&3I{<^MW->tf1?Ee}-+0^f7V(8-EbM}U z|h_^-tyoB=vZ!8@H5Ho+QXYK))X{=YHReXB)>fJAel}<2LLHaYN1v^IUtid*Ec@dmfsJ{WlKE?5 z>{}}b&TKI}ew=d(+S<>2#Fq|W-~`UftaBPq?UptOB6VUP*)CvHs1~zom%hnM3!bJb z!IKQ0&X=@AqAKB6nk}d}M@j(3q$D8uS>j+VROo=hHtutPH1SdKq$XeV=NM_ed_F+S zeol}gDa!f%EcrBQCave#uB3?_)ygSZ8t4^ej>V~Cgogp*_Qw=e@{;Ph7OUq7Uc(3E zzQ`$C+UJ+fiu(vf6^))c_A3UzIkhL6_fppVl1r=Oc_yYMOHCBxOdf7Sm6RhSB;tl(M&;R>n zXdk!oK|9v@Yhb@DKYv9-Tc6v__RLHZQi9x< zF$Txr_y|ly`0S26800aiazdiPz{>Zxg9CX{{`olxb22~)j}$jPAtKaio6!a(Z8m)M zPeNjbhRKGiOY~bOueM{zH)*{djE1BYx59s|KE1f2+8st zspz@B3_5|v`%jN$`!7EKSad7;>vUsiPk|-+Uw73T#0~1KpVJR! zx3#!?aD0zq+4Day<3F!?#x`2?;ruJz`@dgrldm>zeKMXUNcewzpsca+!_?WG(xxVg z|Law4ZBmT8>;3x`&YAt=DRAlq{Qp1R?2O?8^1#lZBgs=#ohPGr>jX4CJC-!|e|QPZ z<6i?FH@nlnb+Y8r4BY)cukI?|!a0oc!Gofjns`ZC&=KCyz{kVO3lRL>EJ;#jzIzQM zCxDt+Sa>)#S`eU5Kqbb%2y)Pos$yWk%4@BYn3@`9mkBNT`*#Pkh;noZuduNc5Q>it zKXAcdikh0l(f1$IKYD}<{2dHZ3eufxYt~c`vc9$Zid0v>l%+R6Jn}vBQ8MCC)D`yPQ32(*)8n+4nO$Vf5c)Sju z0Y<;oW6v4{BW;I6k3c{YZpAt@d?a*xRY7*oD}$RMXWKr!fiGB*QOd(3AT^cvsiBD1 zLf|Y=$vU}HMf&;{l$Trc|6_2u3i^MXodjd*AJ*!ss=2#*Q(3Lo_^H^RsIfnC<)ypA zY&x2Gt$$Uqm62n8zDxOOo0yDUogaZ(@51WpSWq)o@9;~Q0AkB5A(8a;?YQ|^ zcEw26dP$p6LS|-fUE!SxoTfqm-KO5(840&h02%)X2`h8E) z|Kr}5p|Z#?yLud|zM!M&IH1DOJ~P8UJvgjytvH)-DJ^X|_w8vrtC3h4te{!Nq{-;P z)JqTMtYQ$|oBHeWgB0{TUNQ8|7!7aZ1)oOuQG(zNjI+G6GdIwf+n)QBb-CM`7W=@I zSwtlEMa`#ltKsMRP4A;(VjS{`v5Ejl0A5SkPO_je60yu6GOMDiEAUp2{9wNa)AM4Z zta&vR^{3~uTGi9WOwdjTXw<(i6Q2b;`Wq8(@5dwrWf_ns*By@aI9->$OL&QjjG$

o&2A?+rE8ur>#A7HJfBEpe+ra`4JZ2qByPLCK2}ScQ4CJzW_hd{t(yK1y3iaR`8k-j#`;&k~l>hE+UOi>n%D^1?HzM|B$fx)d z_iI%{zkB4PkNe;Ox3nFvUxnaGXvy5?YP;pmrOEC2`lxi#I{6o&xXe%}O+lt!EcINM zBh6e77rgdLczWc~g1Z{$&wgArg5|ZEbG~7=Ng-Vfn>WCWXlZK;x&0>cm^)VyaGdKc zNqe`jmL<`1zMiM@G?P#fi91;th2W`3K(H3o1Q?r!S4f!E{0sSw%$%IDk7}~V$U9&u zF);+YEn5V;&FdxvG~zMS|HEPa%hWh!Mx8xqT5j!F%GZUW?GW$X_|jLh?|)gD^pWBDyDnVTD{#U)Kb?PxoO)JwT9b7CHd zHRS>|h^@A7c^cpftXgtud_5jsL+0Vb)-6>ofal0a6&xzMy`iz+iQsWKE&8~xm^9e1 z`Mk1JgXw2oAcJPy#@qpshu8)CC-XSY>EVl6{{b0H{;yxZ>gZ|MXho&zF>U-fXZ+}Y zx@Au>1QOqwiP<`2HA;W#HgXNW`-K#ReZPoqZMrUtpSq7B{j#dngW(sp$2R){*5>1WOPM-Bm&EE=W@P`WN-nT)j=Kt`LG#+8+=(yf~ zxw|~blD86s`@HF(e+#gGa$9ImU6UiVnDgO7QZM2rZL2kvQK_o=^FRNPV#T-ZlwwtT z^$I7=Vmm%Ch_L;I|9*%P`#YpSLYRl+6ym)fNLW zud<=8o1s3(b@#|x$h>pAhW!sn9$M;YzUi9PF+Di-`7WUhP3&YHEPB2!_9Ek%zVu* z)5rr+ewyX5VZEhBvJMsx@wa1CMK=>zZQw>^wXOHy_=X%JIJXe)N8x5CWUK`Gz&lKwYiU#nbF(FByNu>$IchSS@~dg z_sfJ3&R7`1e0R^}A{051!uX76jLHNNUTGsIZdnmiih52wUOi$uHJIMzwXvMQh` zgNF?^J#$IU4G6buZf?4#05lunkQ7xet}lJ>TCQcKFKa!&7FW~{HuCk{P*-g~pZw&Y z9<|Z8tQDhO8XDRE8KRIKTvN4LYx8mPWmZ!TW3^aj%2QN3_Kub3o75Ng@;)75@z%7& z5k5BP7E-ZEunGhTl1kUHJ&9B6vHKe znzk2K0+B?eD&c$j0zAA&#;ksrT*r+UJTd}FWj_FlO#|(|GNMPS4<&BB1Rp;9<+DUe zQ&d++0@#b3=}b`WS|Ae+hI=HP`>q9@eOUVMKJzNFq%18F#wSmpr&jDj$KhF5eS9B1WSXT|+agJ)|?uO6(YHJ|sW`iK*XbF#cVEPTXz)4J~I zw#v@YWQLKHL(KC*w7jwRSv-Fz8DUxP0$)jE>Bo<0 znADyDhK`?7YHOt&n#|EnuQ%mQl2?9l$wXf-sqiFg9D1U zSBf+%=7jCe0!%4!kmA4-@*ep?b&_ok8gL?9YQf3LZa6qD02h&nA0|-xtUkKOZn$Ri zt7YD4#@J&DG&)ab&Y+`6?Y8cTxBqFZIf%bKb8o*nsX6#v6c0pd)a+=D zquY2yQahO7pq~8-!p_BkxW=kIJG#LG*>7GmdS~5f zehQp5E~{!Rj}Oo=!x2!9A-WoSjy?lFYKcPVu*RaX)%dbynwRHNIN=mEuqfXKuL-AJ zclqD4*1?$Wmto&ogG3N^&?n@B-~kkJWrx%Fg;cR~`roX-HTWcq`SEo`V4!(sRyZXj zhD(^L#~E^X_;Vy9Nyg7lm;=x=zo4r~0#8L?acchl0X^Xg6c6eR>KF;rhJj0w2Y#jD z)x-~=cLVM0+&ba*aG(>MqhO#A9r@8n_$DiBR88<2q_Ko#i!`adR?MkA7t|-GW)0qV znjzxkg+*qaTE&dqKFa}~=?G=+XOn!5LvDvKa6_ZeFqF?bqbl?m0julMVWFy@y# z=Iwq7%K%4)tkvLZa*GkPC*Nt&-RCk($Y&ifzsl)I$m=j9muJ;HE3gh~2Oyq+}-T%?6K`NK(~ceY&GH&sdg73c9X=ep`Ra;IY@eXZme927@*ow+8M0 z!)fO#ou8IC&o&Y3!~=&&cFDsBC@v!~q=d1Z$bZj+%V_-j@Vk-^XKtIy5Ti={ z23yu(_j@s%aW1k=$&{|8aB8wcQ8tzlc?%Ah`I`TYy14&v>)&&}(bvEPFnORw&rTYQrr-s1ENqX_ybCt9G@JHAf zg!`SSH3yxjvl~0Hyv`OO8NAVUw$Q?Yg6vw$;YfL0Cu4G5B27t6{)Wbr!!6Uj?xVVF zKW}{PvQ%O$SO5wJF*;r)hAt}93@Nx0xegD{?cjJyq-mp+V{)U}$p5}5 zSV2F+gtW5(g&@g*>&s=)gAgv?3#+RmiCbKEcX$1N9&Lnn%!6H@vhrVvyKCC|pUn82 z=HeS&g!-F*{)S|X`81LRmr|r4#(s}u_iS$T{eJz`1`v(Ky6^9q>vO0iI#=d1Uj{-+ z{uk5l&3iCDIQ&x9(<3)FHa`CIha?`6A>0_L;eWxy(cYBM-EE!g<0AO+*B_D>GatCT zT#_HvK*lp{o@nXnDmgo2$1NkHK=T7ol!bFvKPzN`+Tf&kpW=j#DU}0JOkH$5`hR_Y zRA{BR84))uPD9{ra!T_^?qkujg%?-bHWD@`oBX2J$GnV5BF(!kTouV$n#60}xMCn? zeby7!2-J($_G^YbS)h{(eL(FmU)~v}%cAH#-p+dP;c7sVtcXzP@I2}EKS5zem z4%BSm*`)oiAEV*oI{j4!t4;5~h~v>mRmO9R=SKuoO7$xAB8l(p1Xme-9~??Cwx6F< zGc`?o$FZ8AKwVf!-yMQ2u9AlFduKFJ5e>E+L&C;3Ig&9o{Gz7u5R_akU5Y0tqFLhY z=|IT}L8HAsdsv_0=Eg=rSy?z}Y;9d^;Q++;(2%k$8t&hYXbBwkrjY3Nrfk~~nlj_Saq}f2JaQuJ7X2%!BTpKjoeLzKkFJi!?_>zx+l)I*ZqRLR zy%k!|W&&bRp>=g^pav2k{wEXPV5u6u&nZ>X#!9Q~Y1-$^#Y)&)}mm?rCkrtBx3j8Xee^YW;81#Ep&+!HweS)~d zr;{htzQ>xP9*t^?o#jn_uq;(HygojpVWl)*7EY5tgP^<0Y7-JbP$6U`x< z_O`}ET37c`+?1SXZgF*WRwh+=B&j81(!@7?^}E}JN)a_Y|>ZD}Dp#yDRmgVK?fm>{sr;aRS zBpj_(+4bWbiwj-sq|8k*G&Ch21f;osqj2czN-3+DFZciA6BOiCMdBnYZ7R$R>b3w| zWAHqWI6dL{_DkKsfO64^BKl*pa>=(gFJ29a>pPQa9=8nFeP?Yrl++`P2>9vK_{=}?aehK2{!2L?de{`!hLtftN!VjG=v_#oIB>(75n8Y2g_BD zn~X&L7`v&`MK9|6|2$~Ss96HIG3HMJI&*vVO6$p!J8n%(OsvZAV;s4OXCTnjaRIa^ zTt_E_JM8Ah?+&p#wC=Fem-^*X~;e{Vf7>x)8wkX;0}o{jEv zKAczD@;$=KZ!+wh**-6LhS;H0z_&?e*zl zFDYn^r2v~qXuu%Sx}ZnK{A;D|)e4IQ9feNn<4X#aG!vkL1hz*)!g&V72dWH?=sV8( zdPcGYlg52_lT@?y!v{F>QMsP+*cE=}D}aXt1xXuy41hy}x4Z)qmoLRdUL9*|C0Z}_ zQV>agRnSG2({+Y)oz}a#9RpMR^2-dl%XcdkP50*WHA70lF)mkY?0Oq=O;Z}l5Qd8j zEzzS9y38z6PZJCT@qJDXIttaqvT~P@N)!xTiPNQSj5O7 zMQ8|35?^Dty;&tWE2icC*q$R%_Ba(?yO%FPH*YY6m=57Aro8olwW7S&{UMih91>0h zYC8MlEEC4xl2gS>AwA~o|2)JOMCXb?vK`b(c^&kT+u7R}1McZA9S-8C_b+Fkx3n4t zN_%^^D3(k<<1sMtTtvS!-58j!&o}k{BKp*Hby?$zB&ifFR55|+x9E#XIcpy$Z{KGb z$rg%*m!p2QrVWl*x@5KBW+Gu_CS&V#7lYhQD@|P;Uty7O#S#pqk}x=aQDp=L0A>MQ zr@oy|b6)7G>2N8T9{WGLJ8Oq`*1jqtl5`mQD0-oVLx^7d@(&Xw)uF+A+U|=J@z_{7 zBX`}HXLYFKu7lb-_+|r|x-Ld~<>gLT7@-L$1zh)H)k5esB|RA=-X43LPR`B^C382% z&`3~^AH9ch%%1|maG=QbR^*M^rfnud35Rv~A)#>|m zdbKG47zPn`iwFur>5rl8Sy-TzIO`g1_~5WmbMu_};~b#9QaPrpU#HfO35UzlkHz~P zGkkEI@9((TG6jUTT(5<>Y+Wbqd%|y{3uX`E#Xba98v?<)N{#oP2lTQn0mcz-iD`1~ z5s4)6OG_k1f)Nq4OGr_b$Yfjr=JV~y7Z(>5Zy(9_BKv@q9Th=>g>`Zw_hkl?m30Fc z>AXZ`{C%2R?o_8H3SFs`9x`2~q`@VaxSE=#4n9A>!F`SxkgF*zpi}koLed2eA6j{p zTKF7^iNx8-SUxfYx13rq#SDrArN#1!N>s8D`Z6~-QQ$x|u0L>guzVm&7;UMg3wRMQ z3z;%hc~oVwLZo#x(xv&H1eS9pB7LvYaMU9VC)l+yQey(Llb}cYpL3A?uSp21I&g4t zDJA?w!TWytT|V6Yb;;v@QASF^%Pg&-ksTLD_UV%|2F6A6(U^#x!`vuB!jGHW$Vt4f z&s=s%^1=IiU;TU}`aZnex!Tlw4UhyKxv399ToTe#_%DsP2)Zn_#YMc|GkX)3D( z^9cys&xrySI-;e8Xs}p=mR6RY;|@?pRTfqPQ0>g^naedo3~E+1#jV**Xz|!qpdC!o zJaZ~~!wx9O@jre%1*tXqpgwL&%Ww5Igd!fhj7fuUiYj4CL^ulNe^cTqZQY`bl{v*1%g{bqp z_BQc3kYiSd4A)tyRFp$|oxKCC^C`c=7N%yi7AngAEM10Xd(2s|u_*^qn-S7GcPe~k z$x>TtAy{-mUI3`cDILtU2nt!e&*c-J6G}e{#yzKU0(T3U4ESmgClE!Vr3*Kv7Z#%E zpu=U!bkWIlu`rC;*n&KBwe)`z5rjv|8VQTk_gWC7q)-c0(iWDpRVv3-TEuiwpeN5e z8!K=1!}YZyma(3b5rsvfMg}^xWO~5KVzu=25EBzb?R!VzLr0>(ZM@q$l<%iV9Apd* z4h9h+@XM7$A_=3`UO+b;sCV?A`nt%X$wu@mLn|tKanUvNdXhzv1MLO60v~glU*TC< zSy5QtDXKiv(gmLefFr`$lR=j$YZS<>9{AD_F9vf^;DI(wAUyDo!%sqVj)AzK9N1zh z-9hVmS$Z?oC_U4c3{=%Ci7MjquU|g^%l`4RdOMJyDpJkhp`KAgG7P+LhDFTPZ!_Oc zP3_|Z#wV}4qHK(gjHi=&fws~>lAAfYk_0d8pp6L3At_m8RN$WDmbFc3%xo1mwque8 z@j%NmlH6A$jQv5))d({#Rj%@grhvPth65F%k6OoWuSrh!Y(T_7WFWkucCh#W)Q znP2&rGYPX7P!V<5_w_>@p?n!l*MMrFu?6r%b$1qUwEWql&r+^)T~`P7h=CFN>XBH8 zqRK0AEC57go!RAqQD|&z)X~vdfD!y_Ke1r2Rl)?&^wH2DjFd&O#G~(?O&PWR-s1k~ z@^nu@cTiKeH}J5ZIqK01Vl*a&0b)rxC8p&D`yY}5DPS*`G~KsPh@<-`N-MMRK;TnX z#{&|!;0$EsIKe|<**g(=xUTwr;>GH6HkPDi7$}`mR#AzRFVcB( z7sbkf4bi&)bu7il9!wP{QsObds2AZxGsIyPRFi{rbzz#>k`Yc5GG|EIOiAjAV^s3a zC}grhSpNZNj(G@dF6$8r*$}YxT((fXeys)n+b27#wTPz!)_*0b-|9&W1Nd-0C1Kp( z9f}KOKMug}bfAr9i+~0PD}AuHTc4m?EQ}s2sj5OtP}dQpPsAYOD!=f|4PQqeAU@Ds z27Q+zR{4#Ko1i1RMs4{WE_kd6s$Qz6R!Ee zU2(63pZpBFdKc)atru>tKQT-D1k`sN!BEf-pb%R=9gbN}C{Cd!6%|#}cHJ=8eTV=EM~a?&(Y}MWK&THyiv1LEa5}Z4+J5>7Le)X!oVtm4k)A%7EHBNccRhKW7<2>o;7rFXM ztde_`Ss%wO2nY3H7jAo72$^0)XMO9#QcF8n&T1d~zC28k`uj)kNAV+KzxGf~Yrl)x zpU&3`tT;kGaI4FwO(zoK;~)2ajX_;+vcF2)Qhh}x|98w7{c(61EkeEs&#>+pHW>5K zT(KH${on8HrtT*WTFWj0K{lt2l6Br!_ITf7pYy(vlSM{P)?t&eu?Yed2rplv2TG!R zL&YoDL<$4X@?xxs@eR&SBnvy*GW!wDI}#5--IzXm^v6i)&A_C>zSh$qhQkh4*^hsM5Wyi!@N zRB<_4^1Rj|nnY6J$7Q6*8^4&nqJT=0xck7nK|UwCbSFRqCEt)rBZb-v$hbh&{q-Kq zzS=e$_vlao+YJKl7(_H6Xe_6XlPQME+x^nVQDMUhWk3s}gU^W|qt|Va$wb0@A7aAX zoM<+lk?6_=m_b#ix{t6FaEOTE>QPmBOdxwS@wKZx_0v!H8RNG7ZrrOWiEGmJ+}9j{ zU;1n6Kon9x#J=@>cEWt2-S*f)iuG#>(-(eK%fn=69WqadUS&Lao`1~O8X7aknFqrq zdFRkr;y8xBo0IXQl^-)3%TNTa%Vx=pi?>XAQ60o{6`R_iHMWb-QHy&Ak3-^;3`d3l zk!Wu){d*Hgz3n{+i)(mKjr<2`IOwLIB$xBt@qkm_ZhQAel8(HJzxW>D%(jOv>S*Ftd8LBuo@-TbePH zEu2yknTd>~va%ZpA)9zvJp#-S5za%}aWGYX%wAGb3~-rCb1%GMut1{GMFp54znrR* zD6$C+g(IAR2}F|lnNqy~H~<6&raXc6ovUK&0ZRc=dS~TW^U;EX*eHr1C1CbLN`bKM z(gEM=Q)^19B`m~AL~5$$Q^VLTuY3se3YHg`dDvy_h1Qd*G2Cq;(!!K1LK6y?xsTf} z&Vsi+7Y#0iMTm$i5o${^d6y6R)pPsyLZt<=N-XzYnRxE6+xx9)5;Qh0T#8plEw_-R zr1CCT;bt`Kvs2mR$XC_j5HEfr3Q=r`fWxHY-Z0<|TWh<0+-1t3kk!Q|m#whN9P9PM zwG%$UQ{F;o!*!Iov^pb`m6#(>PbWScxqqlJ?km-ep8$y{ZVccni6ZZ7ir5J!sT+?p zHRA%=cgZF$c`g$4WKt%vWZJpe(ZGi_wJ%S!+uApYNl>q8r8Si()2>YCiyI1Yq+kX- zV4a|Tauaa0oZ^kId$}{!_Q9D!@TU_hX!!vlYrGj(#2wk%$9)F*c?OfXx#6^>TzKh> zLL7y|K$bQQHx@}9Zg&#X<8WYPZ!>5~JJI;rbC~yOKrwLN!rJ6olZpYNkmvG0< z__jGAw~MbwnR%q>Dp@7{$-R{(tsGJB%h;>4_VsU{e=5(zsa8i-p9k#5A-9P9sZX%! z!q*hD79}kkTaMB&MOfkX6T85;-QJ9LG>7lapAi`=S@9LDWRvlhshMdRP((|>?Os)T z2hhxa2znx)*-3u8wP&DD{9mx~>klP%T?kv_6_eLFlb$TtG?#ecOxU5=&Zx;EkVv)Y@ z)83x3UUP#p<8RIyTT9!0snWJ8)P7>>!F}4#$<4gvRCg@sxGzzm_xfP;SLXYau%>YQ zanYVv@|%_hlTM;b-pNk{C%!zRu(C(b{V#ys3qfEr(`P&PQ{oZypN~3HuQsm}0KSIK zXWsJW>N^>ES%84-UjXvT-IGNF*|YvW(zT9h$URo zQ{hX2gApkhZ|w$qzffO8lM?T%XE;$�cc%&ml=NQlWK~RO-!VmfL;@1|+_plAnC7 zI~#goaA;#nOTvZ{!T>(y5Dk3Liy_5s#UZ$;`b@jAtw2u|F2{N|=gmWq)>~Faig60* zc>twtBS(QQD7`t{d^*ki2n1ni#NQ-Eik|25s z3?MDF-u47Qqm&_Hi`RVLM znpj+!V9d(%k7(87FqFxEo*TFHA%ZDpTm@p^W zE*1$P0$sqTpkmx5M(zj}K$L#Xe-7~P{xy4hUe}RNCurx@)Dap%8sY*`5V59946-E| zF_Q*mX(gVvuB4_FnQopey%fOPCtKQXZWJ!P&R7b#6&bL53ZgnvFlEY8Cw3(A~h_=E>uaeu&tlHNmr4V|#eC0UW6~JP71tEc%6zL}kD(rW#!`ZDy4&2qVJ@wBtw)lSQrg=rjh2(X7(xv6SNyivnn&3s^c| zo@h?-g?Z6qic$dzQSJ{Qq-tjtHUF(#zMo3vjh^7 zXu%jnl3_trp@Wv2?J1hI+vbJ~B2(X{`UyjvKdoBtKtp^!LMLDwHLMMhYnmi_dh*&v}C$kU` ztS;;*oY9N%yjCIekKkTLw(f zU-n@y=0UOFSTlR;rW^fA1yx-IY@I!`U+{Kq-e9&UCiz=q*vwOH)!0zaTW&Xd!@ge< zrUMN%CQ6pI!-G8EuE=hWdcre~M&DN8VeQ4p*pKjWG(D$M=p`hYuaCO-)`Rycx{ZnR zz4f%CTs*#-p+_ejmZS}b@h6vh#N*?y_&8e6(A9&Wgy+f9H zGw4WI^AaF43}gh$&r<_;pG-^Kyx2897j2zg4#9bz?z~2lYT`|IgwP{74iVIzlyROE zXTUPyl7;>@=fM_!)zQsBIEP?(EAZLIXlRqAZvW?S(?t|-ITULVXD=5}QD}B~nr&w3 z@Cg~10Kb9Ofys|k`^C`pJe$?9YPZHl?qqaMcy9Xcph~R~#FM9McQ4o;2q3aA5Cl&LE7kSd*^dQF}o=xH5A5XMx+{^Xe6s($W zNsHTM6#ASwsG7x{(`S=AT1dGJWs6x5>aHyxOjKqbq>{%DDGD)S%7NscoHYKtj?O#y z@hAAWly;gcJ1wDCj^#K{+eG008GK0HiDKATgOG9x5s;2BtP2AGF{N zU6N7}$jQ?w=-#1bxX(P00+FKjF3%ZELRHP7Vz}?>Y#c6%o@`#JoLe|V4D!xg{ci^G zKF`l1hB;52_dY58**yUmhm)w#M-`Px*D8V`0;h;IDkw{iLTNzz5J2U7IuGHPi)T7r z<6xi}Yk$bo*Gk7+ob|8E5+fs=ot%)6LoBrh7=5)O0mc*LezoZqvCbZyCGo;0I9efxlNCJzUNn=3Ug^W%i7Il+$ z1uH**E0K*t_ewQ0vGS?)__jd1YfdvNjx?D;O+ zM(tO(*CFT8etqqeTB3^~w!Y_dxqf1Oik}aX(la;LR#AUZtYS&z=G^EczGZJxBFnzLRzDK<@)se4BU9oNjJ&71bm zqfug}5%_r~ov{z_6m;v3hi|)0O}P_ncCR@7JWr<@%3#m8n5V@KkUW0f(6wJ^_#;+v zS%~nl_T*b?QnGIT7^O<|LCJk~tZ&v`X$m`+@$qDgUmF~q`Ccug8hSZ9qV#0F{C-99 zXt(*Hv$Sf<@2596H8T0IalP)tZOSF4T08Rh-)g;DU817cTp%Cr=%4XF6D=~j zaI`=*ispDddF;Fz5Jnw=RxGW)9@G9^C5{VRDYK> zv=BjFnVc&BZ9hNn*_Wct2egzu+-+mg<6XZb+K}7m`#GEA_O35&GM-TO{hOCJ-aiv_ zhDb5u79-Pu=s+g!F1vwFhQKC@PPt=Y|-zA4Y zYNJxw)ivnmhOe)W2v1sI3l+oZjxz=MOSoht!uQX1WRf}yD|G+&hLWZpIS|{lh*`LV z6yQ`s3MwO@^vby?VXDl73uZJm9dpAw9|`MRZ7}B7v54saA^jZ|Y1V`!7qzOobZ}X(d~`N{oTWN*Su_>X`%@p!m*=<~EOakOyBv^k zkAh7QOCBEv_&!aq8=I_6QSuD0D_BGcOq}3Xf98vi9Mba&30E9s9%}Rn13W+GWYlHo zd(61O_5iYQJT*KNFE6Wwnvm9!+lw?pJNXW*;1bJJNo^^D_&_G6Ftn~<69twm25v48 zTC>M@az%b%^nZj-gL&psbG>FEkJY2DE$+cuPEt_si2046SX0*kodq<1^C53|31del?Sqtr^0e}3?tnOV6RX&2aQSmWRj^Lbjs@;DQh zC@EUG$BOiM&Z=3Mr^m@2WsUpZg410vwDYVd zJop_t)nbCYF#Ti5)Vg-b$;NOtd%fv?6A$poSC2eS-8+tF0%8D(W$+m_q!*dG*$3Gu26Yc6vmWUm;x7<$f-nhr& z+Lc>a`Sms>e&`hj>wbN}y@4;z7pwWKokw6GSCMc$O5T54C(RfXZH$}FWDBE@^x7Ha z{$*53>e`^P6fVvM?Kwtw$i#KpNapr(MeCq?I5oja14USDO%NX=du!EzIS0V9_wnSs zxa5lqS_l$23m4&{AmqD#Xir6k0|kDkNeXK@lg05mo!Di+M3M5gSw|pEkrJAw;VvG; zz2xad>nlvq)ItY`GU$O;PZQqW28mqTP@XSjFHT!%(%}#6m*8`RYw-KBWRW0f6H0t1 zpxU;uoU2MgZ9Tq!e~Kj+PPJOuk=~Vf4B5tE*u4;k_o}XS=)jmgzdp$i&9+c#X@9ZebKG3ke3i;vy@Q05!ry zK{Z^c!by$UFpH7*0flQzGm2P-c`8djBxjXLg7bSGtJvUFDX+vMGbDP8N<07DR?#v> z6?zR_|L`D|w!&33SuE5Hx0=0qVaxgYfccytPc@$GZ}Gb=w$3F)B$<9ZU%B>4Hg@6Kn!CHEsV0ZVExYV*c&+B2U))az z?YB2>9WSmdY-BdMd@HXNS*6z#$nv*s4^Daop$#T_{`|Q@O?r|1_{p&TPeEG%de`~M zmphI12JfX5)ecH0&-j%4V#J&2neDd*hI2k6`yT9r)pWeCqdy+>-Fo+e0kN~&`Q~@8 zRea7T*o4m|C71BU({lW&KqEY+r=LDy$kHHklHk=-vuJLPmM_;H2V(azV@uT04(de& z|6aXPSYMq?v_0RoHCUbt*N7og6HhHb-_%O~8^GXQD+7Q&OZKTEM?iWiq5@^ipqrWN z{p6m)RWUO-E2Dm?!O=$7E+y8K63Zvo?e^?@TZJFz{UM`;22-<3&5Og1BBi5RZEoMw?Hawa2*nc2!`fGRiVb(YL^nI1&c}z53hg$!ywA&>;M<&>y_U-aRl5P%ow-3yNUE(BaNYYO!}3uQI$5=0#?z1V?G$TBzmd$K2)DQXU99D`s4U90 zJpFP3z5_NMf_4!e=x&;k;bE(<;es!?i5=zGH^?$W`W{b16lqEZO@NxcUM2GGx6XGV zp|W-U#ZzPnp~ywJ-pkr|%ZK`H!;5#%*bIL?S9jpbH-5D94G$in=3M=b_)vVC& z0%_EW2oU)s!Nd|s};hTnE?4nrFk2wei5z(vzSXuO29|ROPw+}Mj68+B7#{Kt*+ge?wSye z#a-ioCId#HEm`)iLU*>z>+8an^Jr9r)(Am_@39IL>>kZBk&W2>7r2~%d@#1NRk&g< zQoYtLKQo;jyP8}o3Z0j2bA1`UITRYBfjCV3WO!iRFbG${psCmYD zxNDIQj~;hUl=JgtjY^(8F5Vbjv6Z;V^tkMkx1Ak4r?jE5hFZL~xjUS^IGoY`<19BFt{hK2*zr+%b?6^(K{l<>cYAW4OI&Zd<4rhU7FB``*)}h$Gu(X^_ zSEH#DzTK4U0p=v%*d#`^?rMzk^t!*Q0QM z&AP$a82-BL;ED}>fXrj^bn>|B+~~OrUHC4YrRUwdyUkAvriyN`lrBEy*=KG?-(yl7 zyFTzRFf%6hSM3Ju+t2zmL0yt)yA|KaKA!a3!!W4vQ zOH+U&c7Iflw;0Na?UmgUw126v%lyQBM~gf@LCgEN~dAq zKS|7mo!_ONC*|N6D*x89;ojdwv?lf@}J}HE6qHqCS}tbz8~5ykFE6tSOXWV zbITQ3hzqUuokW`NjIyT3RetSDJZ$a%_NLg3umm9R3z6(zk^i}gRK`IQ>VVFa6U_@k zgt88(6wfBl(=Gfs3t+5pZ}A2Fub?s>74F{mgJ#p;ti$^#qN-iCxpsHan&)8?69YmU z?OPL~EG#zC%@?-_Y>Ll|6KrPqhI(U=KkI3Bf7X+@@x@6-_H9$sUNd!v1HnbIQZF^} zV3mhca*7c?4PJ&`jSV`wh?b5QGe<3mS>oj6iw$Mvu)4Y=`L<{?zL=fu$nvBSyY9~` zbEHUV15MXpQGq8zNc?M?IIJ>%E^QuBX}dv+$jC@g=2on~IT~bUW0RJKnno|(UGG}} z>34yKe4>MFlJ3`(lfPWA0~$drl;i)&%JXcO3XlAC^h1I5H8lmfVz5+j5R#S$(GnE` zVWI<#f1qPH6?_`=9mnHNlTNMJguMNB3K3t*!U-aPOj0Yt3;6YyOISceVZU|mv#uvP za3YYPg@KM;S!9_b(B+K({w)h&+}gdm$;uK1nIE=TnngLwlxPvlpE~c&qi<<&dtn>d z;ew4ZK#8H_E=BP8DF3{=(ieg9W)T&@dJzCI6|GExC21LG$<1%CP9NM8NZLIPu-9W0 z8iyuYL>-8U4(K#g*tWt((hkfS{N%F)prE)$74XY4(?{R<%?1HZmCYu;=tTz(kKdiQ zwZ)ZRT8=ePRjYrz)fB)GAM4 zn5m6h{mWGIiRMA|cz)gg@8Me+cBpZF=M~s=*F?mLTG}CxIbOYD<$L-gD!N{2Dc5Gt z!+Uwl3GLc;UKGi00$iS^OMN-*?` zJ0^bS`8XNXRvq|Ab2W7OatLE*>q+lHZyC8x)q0`_k_b%D?ShEH`}4=le3;2)>D%VS z-j|%uxh;%NR1_%NqNo{|u8n7g6xpA#m3{77p8Azlo!(Y|Ss|1kO4}*3aa7uNIzD+O zzxRiH0}^w*>ZHGUd*DHM{kXYRW2E6s^Un< zL?0~gYQk?_Q6J%gG2bDaGTcdAghoQFepYT>-_R*{bS!v$amOZ?N`ZaSO-izQ5_iLT z?n~G6Tln45dfOdE1=ob+yVl(l1A*~T{dh$gkE;N$=?rcqqAWjAe6Q;NG|COw`Uf*=WA~2Zx2JsMq84}?dr2IMs%O+eGilGo4TDEy%=b_ zOI5bzPDE4SsY=U|D5+R_KbEU<<*Co5JG1C;1VTKS3939T9n>_T@zIuZ z*Xmai;V#={kdT{t4pe3uSYm#@X)r3q-<72>FH1X05D#_rkj%2_F6QTT=H-Y(kZh!F z5Cd?H>>kaYRlbt&Uor`kqugB%0RiYU;UVF9_3xw>ga}5x-7DDobw%>USOjbsWMsiJ zFWq;zDGx7f5D~;;6FRoP*&9;hGQ9$_@l4kl8e&c(X)`lqAS1<@YI5T&8$hbEWicPk z&#A`+np4tzC57m25F$v+$RIz+pFZhSFaW|mb;j@!5cQcvOFU=@lep<8c$t~X#j%)7 z6H7Jv*>1mF_tVLdnno-hUo(ICI2_$e6s$N{1>AZOfn^%wm8n0sgi4mQrGr;3Bi+Ad ze~%UlM#sR#(-L|v)ETJt+W(gGnN<4*7-y=T>9yCOq^2z^C+D$K)yLJ#l{6(#_{}Bp zg|?|4VyM+GriRZ_s^c6tOEf@4R>_9J5BgHN?;~-3G}`jMH2@=-`X_XZm2GX=HA`85 zE$Lw@S;gca5?)8OZ6u2|GgAZ=O%6<^e+|}n-Wn|i3Va%o^u!PQUVl6jYHh2(EgER& zuL4{_UQtnKsaDe1m^$!5Mow;?$LM;J%U9mG#R3Mq699c8DmH8pApT9F%Xp}@fQHAwe+|I}E}fRMT1}d_ zd;Q%^LrX`806VOTE$Tl<;C!Y*;76SM_ia9@3ldCeK4gNoJ;e={dh8TSw!P-PTh{fP zM;jMIaRTzI(o_Y@+;e1bL2|GJotRP-c8jpgwxBP|wXt?~U^F1GyTHj00}3 z33~6vBa7)X2eXLKk{Zsl( zSZNWK$nQ?)7s%F}P01K3FAmcKi86g|H^SMbB+ETfD0)2xiS5@WFyNi=g4gOZ!ZUw& z{uZ?qhvmjgG^U@AB`<1+F3bC7S9f_$T}XRL@U;4SBS{IVA+Rz7#`7Zi6(Vp1Y;Q~xK=o9J<$KYLHJwORS z(tLJ0UU9f~#?K~fi_A(!mMUD6uGVfItxLhOHF>b)`Tv_iU1yj4Ivg87CNVKFt8MiO ztbZrsM?fB|izbiR1*x6}^viFb1;%0!wqU7@lF8bx6=Phuc+sp=;d7vRl>L!y^0x#98>1zDuc|vkQ zhWh(EsH*7ZO4%othUG{{78bpe)%*+61QwfVHu=p`$^QUw^=VyZCDB%ZOv{QBY%z+4{wy$d_ zL{YD54}CowaIJXFS1^LXqh6;(WSrywsvZfmLV|Q_k2*t1v+hVq#1*2AE1vcynpyzY zw4^3UZz2iI*3JYAac}QkdV2cvLXB{UsTR>}EpxU|O5V{+X(Sm0lRyWoe(B`g5doHU z0bCZgb^cPGry@$KgA^bU!60U%f?_)IX%~L_6b0~LU_sJ!f|$T`R%<9-PF;_mN{Jp) z?jYZdkLgyY*<^{_5%^k*2$unp!*tkz;F<2GVW1)D6WBkrv7!t1XmX#>E4Fet)OK#9EQl+DZYoA+5)c zA3HlcN8VU6{!a@4jBcc~G_X8{>CX*>Njfw%Oyg+pRSN&w&4u+B=>fSgXq1FQ{*z5* zy1V+%XzLuD8(%s79;%>PC7Mt6Hp@JZk2~B8%i35PfepZ)?5@x6-)4qTep#132xfub zA8otuTs{-2EX2r%CBN!=axXm{62(0*8;Hc}z7ZxmgpBiUz4f;VLw$rOg~7d|xiLkD z`h2Px?Q2?C4^&AY1Y_z3;my_b(T<|?c|qEMu_FS>N26G-^7_2^@fU&zQ*#xsQs2M1 z_ED|6PuS0+eslaHyiaj>2j%TlyKm8R=W)Tzpfp&@D=R>wczZY|mn?5?!N{ExoRDvN zPJqetaVWKKKwFXNOuw0;sw=a4{RwOsu7^K%E8;o2`{({I!%XOhf^jPzFzHc=xNkUq z7|!J9o;G4`L}vChO@8H%5Yi2?T!KB(AaN4jpGRqIa?wRE;Xe}IXzPUwzMgJ=nUi^W z9C_eLm9*v(GHA|S>?+#IKHJYCNB7crRYz>Lx9glx1Bwpp56lrm=F^uO_B~A;1tu?o z{YQiTd(&6r`woxn@k?2}&iBkF%fYZpDWJSb`c(c>*{YQ4H4tCZz}`G~n1Q^MBZrXK zQ)s{WEh54i@Hf_PWN~EqHpbacMM1h>>!F}%B&IYOczqc@TBCkz{TL}o;^q$5*AJ3C z8rQcX(T#|Nhf%4f-QHR2GnBeiSjF9Ub&4(QMiv^jBAp+4-co_o#~qD^#MW#S)JOYS zZ>6yDGQW6iv3Jk(Llwf(9_rnYk zwRInq8A<@m2^eW+YtsNzDhl}Uz*7hcx!{!CKGk?3oT28x%v51j*%UO^fxs4*a;0tI zbQdNnf4jM%Mi}u*4{-p#^o2JEy-bPGzwo4PuWQ(>Bx^F<_F5uM7Rhm%&*vKFy*xu{ z+2;pY+<%tt+3u{IK!#ah_~Nlg>E{eplLE46D#!p`K1Yz@E?rm{sBAFm>JC{uJn8x_ z9-Uk6N>fAmYwlF%T^-#?LEQbCyMK-Vnhgy#jewFIB@0-}k|pJ)duCY! z1{xvZ@ypE2L3_W%r7RAXfkC}`dul+#&L=em(jt@5tY@o^LZGr`5-o;HrSU8QY&}}= z^zC)%)+R%WlBQ5Go^cK&QOBWLFVySmIVQH6m>Y3SObpm{i?+75@=8n5`W#XRh;!fW zNuty8mn=aL5`);Fq%elx<1q`{56(|ue*Dj202kUCSsWF;XCATpyt-SS^Om3T{!f_4 z+PoO*1g^3Z#P9aHyb+n$kI4vYvSCnSgr@%U~qi$lh5~7yuAkk)E8yHkJ2~| zhLi8Fjh&-G!NFs(?&dF}Omq=!{=JxzZ9%F(mP3ks8rxwqg{%^7A#t09Li48x3xT%h zzs%50*1b5)90IsHG{FW{oZa&$%*ezzoi?*pxWQ>-La6R%O1Onh$qwjg2MXjy(P7N144ZOTg4#D$3C-^mIRABfvQs49lo)d6{ltTH*khiO@hq9P7y!>V~^(AtaveOCh3pW9n>* z%J&{B)FgwEyz1)o zo~Y5~sy?IF?)9M9!nWFcj6T&mM-O!_{Y*&ALFl?ud2mfq4EJ-aLJZT!83#gU> z0SH=2Xf(t!N#6)%Xr^jW2?>Fy2ah@2oOVh z5`Swg;#{hSbp@j>2i*N;;3=j1hE;q}E_TOD{6&b;FXs^^w2QA3ZkEZVg0M6Nj$*S9 zk<>Hun8`8@W<(UTL5~KYOSl0xPv!UTbfB6qj8r^4gb1{V9z+U{!4x#pXb^SM&>*2l zGL@EhQ=$iApo7jnr2bv#ypM2y5rMS%a0dw_2;DT;0UfRq8*eFA8V{}67N^e2n1O^pij+oJc}KmwY7-r3F*P!7sA3=I!lQe<@6!Er!GbNg%YZ zCJnvzlxLJ>uE>x8oKwK_2P^RJ?rvdmF{&vhfpoAqTuEM=^zPLoEhw!&BF*AcD^LBi zSHILWw1XxorEz1El^IF5c6qS82w?;D+WroaE5lTu=If0?zIY}YzOXI9`tNlN+N96b!{Tx7U!#DkNF z*e<3q(;i(`;IrvBX<%WEklE&I^Wl<$JT9b7qb!e; zcdo97XQuj7wIK&Dk!Fc;RhZ)JF`I|{`cA#?YOohB_C}}v@I)`gSP66_U*9Egy5Ssu z^Si{fHYfT3O0@s#v@ld0#sKb6;HLsq00H0APp3m!AI;|)HGD4zO=@jtRZUHwtgZP< z!=EDrg-c94q_zIQcUD#+Nwo+#6prg7?29+Yilk&@y$;nEx5V+`+4k`8N%HP(tXH!} z(rnS{y(KI;py^JP7p=|@>hDSnF)PrgPcb8c6{!|iB3b9Vij?_9ZVu*vXnj~{14*yC zAl&A6Za!`QN?cQP>~Dvam`VHJdKOEE?(Vd`&_&p_pf*co z=^ulS)%pY@`hqk!8XT%1j(v#8Z-GL+2w#!u%a{BDPZUI^NGhfkX>o4_yK&cM3Y?AR z`vkU^P6&76kVIOr5||Qz-uDqg;({`?7^ZBmex+~RL4v8FWfVSzh>pmV@UtI2r!0JU z*$kY5_~;?2EGdKB07uJ4+y(7I(k`s`M4?Zbn4*Casu7w*=dC(gHD(ygPSEq@qh~L~ zkzuQ26VRu~|CpSKP8e2GSC>#$#-fM|%Qxd8%^PC^Bjih1Kq2hImR{nE->)Ka2KW%` z-$m}V=;O}jeQktQjH03s(BXYAT9KYTUs|tDOi9r+G>qw6S+WFEw}BA~(0V~muU+`H zHul5TOSW0bC;#4XgKl%C23fUT)j@{Tr~>t&uIKzV*isIow>~i`383M@^yAXd(9pm8 zz-Ebu*BcHAA(|l6c>!*?8VQnGFB+;|XqHdYcZTJm^0EQ|E}pIZU)AgwDYfoQ;cj)s z0#|yvjYV=N>b{-VmLmc4$>>6V-8lTKiP&t-9c!U;au z4)y$Y74PY5js$c)&&{yQ8`OoK&!pOkgkj7F*UxLk`VL~7rl)%-6T?w)7P)%tBUhi3 zby@dw<2%_K77>WUFA>*1w=S=E+um)8T>rAWTv!n4#PIU8k3MQt3jM9@R&|#)sONOc z^ZyCt$}5M!QWoHE^xoEASpKcwHad{hxgpk;sK}Al|AC%@!s89)@bInwV`#ffx5NZk zhv57JcoN{leg8T-5A@h1wzkT2TfIeDbp$Fc$G|)*b4^`cWiH)7TU0e6&uQw)dL6@ZfgBxN8p*YprEGG{lE&jIn%rj|pD$s4K>H8;li53iFs zjbr;(Ov*q~3GDmwg&O}(k=g1}TV2TlPtkfr&^lHlG;@Y4f3C|&i%PDx+Sr) zMfdFbOheRy8o8%1yhmbyFZqKK=Fb*4a?vMdRj9%m;V zKbr6LvEM4Wn}{5ng?o~CctbooOR!b6)$7@LiNr~B&;*tCf|@^_Mb{L}4_~1%D*J6P zvQaMfQ8Lb+Cuz9py{h*zdf83qZ2Hu%@@D^xasxS+@9V=Xy7kGdB)=TLYV|Sr9v;t5 z&j}6eYkIz-ZEO8R8dhg+`HC`5tR<;nsXoo-Lkp)}UeabkZET|kWuF}Bj@6R5J}<|3 zh^nEukR~&u#%bpGA-7@em!Os=?Du@lqAt*#%^W|v7{scg4fi6}VZKv~!>Twx{~8@m zvyo^8g-+^bBOID$c6aObs-Z#Wcc{v?h6ObPV|7bz($8DPpV46gg>Wt=f@cT83JN?T;cM z;GlyQKvzBt7jZL$l_3hyh}sDI+oD$5QqmBcM)$|7FoIm>K~XA`6a;)QfR&>ScwIr< zLXC~E&@jW_<=;xb3|E>Bn%m5-R}XTmuo_>I3C>?2b2J~UG;ci1UgNx5w)S4a@cWne zfsn5cq$d$)XRaXm1M8c{e{>_i$d_Bx51|Bc01vU(0pvK;8R44hMk8QQG$_85UR_^L z*V$2mPD!bVC}sT2G$_{sO%^F5EH5;7L?zQXd0J6NQ|o09)GXSO&lEH5glY3lv@?$^ z3n}f#=y-+R=k%rp3EV&Xq5t{zQ}AsVHTa2g5BIxR9S~L9Z_d?`2xjEJRiPC%rUusm zM?hdqF{>|*>;6GCp{>m)Ah3MunN|iI`G!rdfUoe+p4?N-M#ImGCdVS-LBtkbv1)nGN4>S=&8r9nx_C)Va_4?q^%92Z6p(eABBy#zhBtQ))`5}3W(`S z6LWLQPENejHpBo_q^@l=0gFzq2Ghl5xhAAgF&VI?|I_JQ3C2}T zP1-JBkxKc}yI++$@Z@Q`-Mf1{oOgTe+)5kl=JTB%T8T4CYmGt_eg_CMaxfzqtlR6_ zgFlL#39p_unLlAQNADZvSRXp~i&Z376iVyXZy#{;^FPnz$7nmKWtis~Y2A|gENqKn zm+mHz;(LU?9-ON38mp5@7tSCciW)pkFY++{Md zm+$&&^Oe>L_jZzoFi_zX?-+X587a+Y=hcf?S3d5;L#Q;-7&SIOIO!Py=_vx=y z=ysE`Vx+xG-N^Qg)Vb1B%E!JHHd^&N-(~D~raw&?S7Xi;;h3AHTsLmJ(eb`7@3VZO zM4L?FNA(=8(etO!!oGika%tV~n%eE|gxtVo-6nQDHu^pFf1Alia1IKUN-dV2`4+F| zbXN^K)ghma3?(E2aAvWcwn}9Z6EUkzq{KEdHI+?FkU+cUA+H433q&T7F9WQS!%J`7{5KZm%TZT~k9bX=s!tfQ!D>=lX#xX%=m!hlGyKT>iP2lI2yVeXu__ z$L%gVlt1zJQ0F#AK;WZDkQK@}#0-k7q5=`ed->SBUi%?w2uTu`#ioM7Cv*RoO8C7{ zmrbkb7<57%?UlnTwO$BF%xqXB{ALjhCe=V534+pETi%KG_D8n1w)N!?9hB{3^_P0e zB_&AOMm;56YN{YLVj^-my|_WnX^wO-486XR?+z_tLDvLiGuzqbtMo?mpCN*aW@d{@bfyQ8PRvE3Pif?Cq5WyPjp40-!O1 zLKZx>3yNo=ltZQDAVYNb3W5vvsAz{dSXax(`UW=bW8!6?#Q9ft!hHPX$(wQJfo*fO ze;1%MF*`d5m|gaUDW%|~Fypn*{I{s_86dCJLa)8#F>&zFQA+JBK?ywJUn10pW=b@= z5*g?7A*FVIvHS}+19Om9_L^`Vn4fHS$rh#Pt#qVbcf&_!p;Kc~Nzi4|YjRI3w!U4+ zA!y` z^af6+Nj6iLQfDXh9>O&&DF+d8#)AZ-ejjULb_zX_))(Dg6ziYdY%CH>mu;>l{2ZOG zIYtbZg)g8_4# z$Bs@oQ3DpsOQ&YMq@kwre+F=C3*RJk@!8$6*ZJOii7fOuhaA_heqJL)e%8z#HWqIp ztBGx9gaZv5>v)j`c=g`b8*x9Jn(E#4gcuVH8>7#STBC=>7E~h9LT^WJ)BUcR z=qT@22RM}%1#k=f3UFr8;^caKa-+P|A-%OLB@f)-YVMGeO zm>0+TH2Ja5Fk?S|axFYy80X;B?!}?s-A)i&nqIW3!OJ?)A=TDnV;kFuJSFX(#`Cry z^r(cP!t1rZhK;m2k@JhR&jzkfEi%7_u3_;qV|xg7%kx5aO_`EcEwN&bArGph|D?nu zIE=bfxB!a{RNvm(x|OIEI!mjF%kqGJp^Qq}ee`;I0A9 z9mQaKE7JIMbZ-PsIIKWvEuHOl_u4*Mv7m^!La?3AOp+rbui{&p4TCZq402HhEF*wx z(_C{D0yzdL6e zONj$BD=RI21gH{#q_XM{`YkU8laXXM6 z_e3G!Ake6CrMR0GoVsYYB}DSpMSKhUd1!Kl`g%1{{x!kND?LHRRbFZ?|=j0YJ7pYQ z_a7p+FV2l`xOw450@|NMyBX)tnf$!21z>Bb5rUmPl=n2(D0j)mBJc@|_*?m}rHwmvWP#zDHpfHVE1VjLK{?eQG~VSXjBDtmpkmvEr#16Q)c?T9#@I8m}wruM2$* z%rZ$$`a-=f!sbRD6(w-v*mFW{o^~Akz|Aj(B_`)bvwjv1(UcbZ|(jq z9Ca{5Ttb}@Nf3kme!v0K23=8+iRDmIHMX>N;oi>MdHhRov%uCHDE9DjT$?+?1{>45 zZUkfdz;VS!UU99{^|&ouR;U}ZW^;c3u$c3YOu#}Ay@IFn`+xwbRhWEoN~7YtWvODB z%8Oh3m1j2ICls|d?!m%-M$+)VeWv;#h*ok84){xzfTTLjQZ7~z@c)%og?*x9T?s3|K%%%ZoM}M8fT523{B7n-}^H5XSO_Jbe6X ze|Pu{+&*Sk^JM5M9e9p|QK=sW*W6j7xdD3=PQ) zEI^qd5rc>hF_jMPelFG|Akw+FkwwMLg9-FkP~gCJXD=IoW-q8pSP6xgeS5`L1Qwq& zPgi?@ISUC}UgcN9DmKu>0DrGcjG{os(qSJZBUy|FpqGYoF#shh0ECH9$Z3PdX$cFF z6IuAL5d>tjrxMQ@$1$BLsB{gK3Tg>wX%`*eb4fhXPWkm67i`NQ#BV)HssIJIfN-!` zZ|?7JRP|BNf_c55Mou5diCb{^goHQWgnr90_0d4Efq?u;J4GjbA)!vx9PY?}=A^>-_pT%2mUW=?-QPy9mU?0$@gyb9a1 z#$M05^xg0h8QmD=isJQW%28Y2I9;|DCcoU<9j(s)Mp78}VRJXvN(aYI`VW^G!N%b< zxmeHFGVgh8BO~Nvj<)m-BfU}DPcsKXqM4RLB)&Jtf6j}yJsjn5>{R4k>Z(jvWCf07 zJ*}py<8LEZ*b6+Y=@PqyC|k2gQhb1otainzAh|1p0sYhEqwTkJPOn1 zJjF{gPs^oYcs&h7jv36aiF+m^r7sxNgRYo*lT}LS(TtGmKH!n5e`f8wkkiayxe_0u zc+p*0%v-=0)pe+emhSp4H$re(jr-I<^sdypgq%#mofjcupDuE=9(1AGfjEb` z%kl4L!7GYPpVNAZ+YUT<7VepG*k@&)s-3B4kF3u*3pnXYwgtvJ)vu-udwIiIlH zCTh81%r@#nm85YIwC95vl~Bq5rv*^sc}rHTnmKe#p2fI#OeQuV(^bo1q$xzWW{D=G zJ$`wzVMiB#DY)n3iqTrF`aWX(s}pP3itI7eXFcq)mxQyJi!D>FtiY?uYICIluE*NcGU0OJoX)k7$*E=*-(T4EN@j6XiCPKpC!&^EJ4=#(? zz9Qvke|*kWBBN@c0&J*wXAgNfU|B*SfqYGFZpS+{yF2r9vM-X~e|&rnVF?F){>|0E(bQ zAan>GdG@4xR+yQTKUSBGL|qMgEhHb4oHsV`r^(md`QDFi3t-~4~T%1w+QU80v zxXqvD5PjcER$QY56^p-HTQmerxz7plsPPAj96S4!M9&{5zKBI|fl{4}J;I-pPp`A1 zUqzoV?R}q+@bK_JV=>Uy><1cjoO+IQ8a8u@PA-}m#^(m|jPHdBWbX)4Mk$nk1=;4Cdz+aBg?C{HzEIj{a&4T+S#7bF-dH^2Jx zmf~XH6A|TtmE_R*_L~yZ@Ung0v=C;5tfj3j^%odRqUzkvrT11%_C{voYgnvwc;`>a zpN6~6^dFij>T=Sf#2lXhK=-4G5NZ_&8u#lNnVVa&B#py`y5q1A)v5;Oh$9rb?n;LQ zs9@eB68jnqp^dV({09Cmb{vl47QwD4IxAP~kNgN8>V!sQU;XI!)liXN4JRx_T%;fcN(8Tln_p^Mj+oPoRJ)nDFt5 z#Ym=s2xq|C%mfw3R6aBckAnEiV?_{s;G=thHS>T>gM5GU;cH!=$5yFPGN%qrJnnl2 z2oE^m@FNIYak3aON%2=Qov0v$4~Sc>B;Uxlq3+EY<>eGFDpZbsDq5|N#;Fa{d)Q` zlh2FeneCUag74&}6@zK>6u=vd6)S?$4IRXUm@yczH2X z3dgI_kq{!O5#nBz1d_yZ`=05Ye63TvztN;?6rPrYi(^800w<-b9f2}tsGw%l>|zS7 zUIn7HS?urcU#~EDHvHy;{6L>_Cze@|Bjoh)4Pjz*2~BYc60S5{obY>LHVfcRE6ByH ztvZcZ3^5nuVfTp(N#kBP*M9vuRv)DcANo6%-#;?a`|w=!DSvim)A7A|`l9X1g<^iH z)8GI1mrJHzhx+cey+B==f&DYA^TRlPck;|Th5-!AJJ94_HFoRz@E7JS7%x!NNc zC@^LX-rrs^nRECCg3(BUNvxBVg6wE>9je&l(xhWN`R-QAKFxl?j#fT`Toojapdw{K-|V z&6U2QxjB(&yRO>f*46XIUo^Mbv5eI^6>f^IS*%p>?vJ}H7AN)c!y|+ewDjeD2nNX( zxECJ6r7R%*|Km8w{>OQ|jUdi$BZ$(u;h+=1`Z*5W+aKf5xi2Ct^@(MD&lDmG?xL1P z1mT1l^2DPB)gM196BcgF3C`zqM!-x~N!wlc%a<>8n_SubZdOcf7Fv^-61blo*#BQx{l$8hKd=sTia>F2B>#qOUO9jr% zU|jsJYXx&0==)gWpC1HK-uP$VofPx&@#TQ>Ko&{SRU)&&?H5WlJz*?w6E26GFCM7p z_kU*}n2mR5SE@%slu5!f{()Y{RyOxx~AD@2aG z*1a#rGq19a51Mfinb0Aq8umPIZZLgiC8iYNY}y7o2b~;w6kEbOjQ&4Wkuq^bj-n`Q zt|H_-_8IT89LCL;B7?%gRt6JC)ba+MTZwifbVyj?qUU0Wf%LOqZt(Yrfe<*}+8w2D zMvNkf)BMI=8aF{PE#x+_hOgSg?Q!JGhf0w$XC&QD+CzbB2}A?&f%Ie%rDv zQAZc^=dFF9$K(J*eFzzvOQ{lQl16Zu9!uLx;n8GunOKRlq|p&Dh1rXMW2GEYp?b(m z>MceEK}6I(!|TB7aCl}g+2aySi}yx@Zk$xhBL+?p{~Rjk{#bz_jcyGU*6sjYOzyE< zy6@%3G6w7)P12HKFNNFdmc9AOtU;RnSV;P{y1^2EKsG~?5`q1yw4}5tbufYigA7X& z^((G8%NNiv!91{?{`B?d5r>7T!6P^9gA?0v4sJ1haY2?{odV{>Jk*oQ0g9+oAMe6S zqm(be@sS{mepv_>(LVZKp$Gzh?{4ziBu&j}foqmXP9@DMn)*`(ddPR z!_WvA?Drb>O+n-UG`9#76e?mZ?{U0yf<2BSp-=fjHzb%=sXFg9w5TCJyH^aIoxdhL zKdH7<|KM%_RPYcXA&RmGGt#0qRj8r+X+XmPJ02QW*G7`7(6H zKaT1|Q=AYyAnd{dLGa*C6KNzuBoi?SaMDANh9Oii0_3GvlDJ58o*%yM&B!w)OEKXC z1C2QRJ=IP=Vrvu%M-X!mbKnoTOJnprn1VQRx?lz1PT~)bbU7FXKSsMv6b-b5sEQb< z3gOcPUUd3)J~9!b|JYyYG|XD?0j;wN5(}YemWAf*6k=mzQ};>9{QG~BlqTTeR`(=m z;j__IhNmmL8ezvm%BW88u9$8O`a0s7V)y*HQOU` z4rMt0!5#aIYnQYV65(kQo3$vD=iaOhc;E1H#qgCu2Pv=;=g|RQ*W3ff>R`}`mr|Dm zDJSVYg};p@r%5MTR*t}pe1o?<1)eWQ14-)G^l(L{W&CA(gU)!iPS_I%14)w-Q%Dm*KHjQX;VE?(XhpkrD!eAPthz-6-9i(jXuq zNGdJeARyA+(%tZ%xc5HiclPsqeZRfyi>{k>&l+RQ7;}v4;;Lr1ejSmXj!3bS59}VK zmnz%Z;$~!g2It$(zC@9_x+fqsDn345^6gv35{=JB7YFZ#y$bxas=|7Ek(TQHia7v) zBbn^T*qGd_R|vwwNP2EL$+-Fo0Ih=E*)rvY>?A7zwxuOt4l@PE+~JN;OmL#i!-jhd z?zOJ7Z)PNk0K7m%YJtd^ioBP zFCq8S6_?fYz%T56q?`f%6_g-U;;=twkw3nNhkZk<-Hkhu8DPH}$P&%csxqwrG;ZY11;38{`XMW`!ZJ-rNLNlE zLxq(e4f9~LOS=VdJ5v?0QNoec`?C_Bj<6<_-OK!jFThN|(Pz-+149=~zJvXv-dK>> z^TVc%ZGMvl@0JrXFUAuS)+K|s_X&^N(m*W2;{c!crsmf3L!7QRv<*CEz?YFS^h@}{ z=R*vx^cE$Ng4^{I@t~xFU8d$+Q-KdIYC6vR6+7gg91sn`Zs3s=Iy_kUWgq|fVxL{( zP76>nL_YA8Z+wjJVh7h6AWQj=<@`S~ixQ1qQGkX(DWDV(@aIpZrEdWBPKOcs`Vetj z3^_S~*FHl&xTVU-@B~5j6d6%fHo9@5H{$%4tEnKEY#cBztndLwjzooYzSzRd;*Cz+ z2L7#&Ixlsv;KV;bbJG-Pq`>_6aI_>LAoK(|e^{=wvAGo^8j80srt_zV4e*gC zmD_%>in4ycE>Bb;{9md^H>KU_> zP86h}jv2O^UHtBcGCtY6%B)f4E!q<`@aq$`1X#oif**5J9m|5g{r%DN%33NSzUQvK zGJs9|uW9{%EfF{aeMx9zibqFB+raFE3A{W>ZxCrZggYi-I@#!i4 zYi&oq^VuR0?z#-<*Caq{D5u#F0NlYn{;4)Nke(`~u8vwPFEkH~m2k@m15uU+w$@6{ zeppye&zn1q-ME0w#VUmR?FX$23MGLIm;?a&gAod)XJs{;Z*)v$GZcziv10<45D&yq z4Gn~lzmiO6M-9J?u!Ta2yw)C}R)6KH@|=@*s4`MW6G$vAWrFVql!SmR1a zqF(v*dFVQJ?4}(0M{l^CtX#2ao(w@WLM#U+M^99#wD z0g34;zj7s1oT-i5*`h&&(ev3k56EMuX*Ltdr|gDAnXZI9>RSTK@WWF4xb4vs2!~G? z@!MCUV9S-77cVkWJa7&;XYe)b!V zA0v>(dX=DOjI>0kloTgmpA86R0*llN0f&d{;Sb4%ZzrP?M(hvdk}nzF7b^>q09xe_ z4f{$J*4mFb&6)vp)3*xUYTjMbjp@XM^)(ev7bcL)-wGnVCjkd5IAZP; z7N5u!###BSQG0kUn|E!9$undatvr9ewY!(AEaOPYDq<%nI8mDIF-;VdpaHhuEa@3wce7P3=9>23Z)4P~Ysp=bEJln*)5ZU!4a69^y z^`kmJGd8{;HX~V?Dk+k$ZH5{VX0)k^YGsbgMr*j9`l-cab<}(&$xB#ynen*9R{*Rx z`1h8T*{fTE`k9#*3C7qLBB6UMF4mfZNo&t1t3GGOTXxxPQR6x;1>&l{ehr%Yw>TT; z7}kAK^&@|aB~Cz(k&9SXnR}K-%V+x{SWLuHmLYk)$N$XZjht@j2JJ!5jHjzDz!o~n z*2fW{ym3@cSCI(-iRZY!zlHk*b@p0pmr7Km&|x zysupzU@=H1obj!;D?4}GERZM~%fzr*v5eX_wGt8%z=^=0o144B=x+?((oZB|m6F(>9!8eF9lo3SyaiNaG zM#_$y+rK^oWE_758vo+aio#G_pL^{jVK>4uTneH{^5p{$=p%?2`h0q>IeN@*#h5+C znBEIG0+7FB52Y*rOet6%gdPv0hisntr&_4T3QI`J({s=@^J{ zW4!ED{s58>;T;@dfB%;FRtD~s!+@QROuz1<6{&c)UDgmaI;si{5GmVsXpF??3T8jo7yxn7)>t9f_WFmgr4*|IlLaKN{vkhPpqqKcL`aY&gW|p{{}4Mi^))C2 zf?7~a#9FqE5#HI;-=7CKXu-Q`fKuy$>hGwR*8Oz+)f6!JqL;|vR+B&B(L+6SSQG@2 z6cj}C-S!U~urwZPig*PIik*!*mVC-Kif9`bFO=x<_ zuNt)i5X0%|=(39~sm2NvR|YtTACeLNI&=Qd;R8!`97!2K1&GUL`IAm(V$Hza-F+a<6}7>2f4LQ5e=WIX zcJODBMVp~!Wk+ESdDii;XmmJBeaP}MU`54TTa8yPXlzUeI3pDx&FuUWX!i=0!h{2Du{-bHVts& z6)0uM(_t;2f`Grt3gZWum&?mbCY=j9G||RH5o+r8x|^dZ9hWJS2i%l;7Ft!1bPAmI zjo5(KiI0d$P%!QHZ%dSYO$t9ix4W}Z&jMiEfDqKw)dPzixIlvGk~zrc1p|_y#F@g! z(y7k9iiOf@4|j?WMJG$$DKKnjsLVB-=X7PMG8?AB2(@E^!U2%mznn?`m!RDQXqU6oC#WWhVzbA@jD!l#RGl%e8NV@e<^K8F|WQc z0SSCETm+rRLjN9JSCQxzv0{LB4bt)VX6ptr1abcmZ9#z}GCTe(4TU)z^bPkkM*SEML^WG`RI(qIHEeCgeaP6w&*kC_B8<(^IRN z+Gh*rQ_4iab{aTR?SSMcFPlgV96`sbone8G3rOS1lPCYRuy){#1Fl+7$j*FITC2`P zUbYpURJ**rBntuXQ3Bn{PPS&uq$O|w)&?fhzs}JAdlI-gJpgOUK5)@c#Aa7i#Dbha zct@e=#!C5{?>Tuj*v}GLJFq?K1t;q~_@7L%( zdX;m_>4OM`MAr9ni)qziW8V81bEc>2Yus&m5)4!QnZ(+7;0EfYx;n^Y{vXr+t3~bl zXlP(~_wtG#=v_hB7MNKP{T@LKTbw*OwKMetDBD`Gs1Xx(G$X>okU#@|MfGzkbg`2&lc@5Q5DS6Kd}I)|LXCdH{k!-0sNop z@D~47{-4?m-~IPW{{IF4J+eWM|9>_VB(^Fv@#L|xF}-hIsG|g7Dnspvf@P^Bzy$Ii zqMApJ{+bPvAV3!W50FN{M+68a|Ge@7Bpd+-!@j=0|EoRVTafXZeFSjd&wRBl90n70 z@8Dpk!NoD)0k5v9sc=6wb#QR#>Fr$_F81UWy*(4gYThScNfQo35Be&(k9v0ye!sky zTL++GxN`rGCV`^6{oC2Yx>o?27vMz(OU6UFVNePbVu1v(xPcAb;Nizqstjvw_f-T^ zvpp@x2TDE=kF?~sFC?oi!}{1W|AX6Eg;Zl>{JowKR^N-r6|_P(7Z zUS7SP^ziZx*XOk{OBY*nbwSsp4Oue#I!cWH()sqoWxEKlSb=43c2?&u?u4s!y%$%{ zP&qn!NmmB?>C>l&fm@;O>Cn0p6?re2s@JmTd-6s4f`52>#AQPb~ z2hPyyBhy^$m2U=n?&;o#y;%Pi9}z1_x2*c1cNQi?yutZ8w$Sm3Di61dI9Ie`UKhLL zFN+9kV#^KW7ki^(%1C~zb-N9hmZKgiem%O zggOJxbrD^e*3F+Z5evD`xPGfSZkoMGe2dTCvGPnx#;$eaF81laH7SZV+BM{5w-QO+ zDZt8e?qBOM)hXmG|rU_0LsOx&EDs=X2lYjQ=Q!rIn#rm>c!-&5%g z4$bXp+DkQ8>Q_Uh>t}*Ej@4XaNmSOfjR6D>w^Za2`|m+)R3miZRUhg@3(&ikqb+34)Kgxy_EIeCr~eneDHr z-OdW49nU;P+F_N2J5G~_jI4GOnJm|y@vSNggqL(V9ab#}$$cGu%QeR%;SSog*0COfv)_Kgy^G3IrSH5TJj`MJX`P zhAXD?p03--YWY5>U76zC34=@-V;Nm#GgYNfp4x{0v@F+koFl36M`!i(A%lw_vFSRg z(UNh6KIh!chvBdAxv!CN8hK%Qd6Go#V4EkiZ?0%xsXh9{P?V{W>3$@3= zbvIV+IC);kvD)u?+)?YU!^WVH74^B~VucZj#@7L(;@~yvBJE`5g_6j9jc+&qE(}hB z%;wn9@BMFQK{(C!W03<)qN$854Jm8GLVZ^TTxQByVx2uBFIOMgQ}>*FvFVjKqHMqT z&K>`WBn^!#=RF2opO5aT@{uy{4XNHxdfO}`ZVWDb3uot$^{LZ%#2T9D4TY%Jua(m0 zAN zT47((hU4Okp6hkBSlnfUCGz|Tk~R@Vl!9t2kN5hwR!C1RY1xYD(r#;6n^Ut`eOS5w z2%Zr47D50196_teo^baNC1*#1K{|6Va^Z@|eR!_l`su458wLBU1;d}*vlc(r%dNh7 zebLU^($7ljm3d`7NJR}*>BKYD^?_Zu5;cQ32iCsY-zZ#I==kaEYt%m$uGLiZ-tRXW zyXhWEm?Yjjy~oyUB9$zzYRaJf&w=7j=7{D+d6DD%XifoyQC;imj`&p%0FBnK|7Nnh zWw;)<)?s{Mu1Bl=Xwqy$qS>*VLt6KmyV?B|KVc(~e7tvgrll*{=YsBSL%FFb>NS<0 zE|oO%;&$?IAzL4WUf|Nm6xBP1*5@9>(I!Pa6W&U@E<|xjsH}`2EKR51poy8V?tVx+ zB;A8v|88_6m5)my>z8xhcU8ZuL)R0>i<=G0OWk3I=~1)eEmocW5OR`huE&R1 zHpS^^vlqPSLP7|J<$E-!)xq7qUYLxd$CtAcqTQGA-Eo*OErl7sxB_azBg@P39gYew zXIOob*q@;KD04P=2Bf3^dx14{OBah;c}QshT=7Gr`OiTZIDj?jiX`lAbUy}ef1={x z&x9Z7da|Sx$I$~;3~AABHkAbAS(mR}btIhiLYy{Ol2~3VQms1nCF-v%)HnCLCM6%X z``)l6UP2npQJ+W({hB?+Y(D?NGViCSwWwy7x;ByGOC%nQ{mKh_J_0p1) z&{c6`y8PC!K1_!q#|pWUy)hoz!}xT?&2w@LTMVp)>|w{{SzXEd7tfv!d4Je$KE`Ij zdnNBP@`3_2c)a$G|M%B@!*gc+rK5X?sp0+Ik2Qjw1-fjGOxI9jyRL{T`Ur>BHtclw zA-S6s(tShg_Ll`XyGzl&ZtsyCTa>KjLRL}XRq9{6R{D6uGHUNHUEz{ST0emHvD~GQMCJw z46hSEu*7`z>F;#D#KK;M*#FzOP}^6S(IHV9z&HO&-koR>ng06-|^}F4b;|^LCK`bo9FJ?IA%~I$NMfMS8E-dWrt>fDuY}a)_N*2OhT# z-11x}=J1}u*^|U?L`^UIy3t_18)iAh1;C>mxDA{6^bEh;aUk%;Y>yH+Edbyh1 z((PAe8S36xHk8Ed7K^7|6OSj-0epXkZan2k?(ukfzwMofiZ^s6cYVDRX*0D4Lt=*y zmTL{YoTlN7P^tJ|FwD!na>S98nNZbchYM~+E+W(o=uSfn?k%RH)#fXm2fms2!yQK$ z5^dxik@o8~3(HA4=MFRN&)xQ9NY@t5uc)mKPuisuPp&aJI=I1*5l@H>Z~v}EuXDQSyyI=f-EtE=hjr3V#myc0KCgh?*o&N< zMkM2FP}nF7>NU<$G$GEB{jGNT#YGH$6@FyD_R*pRmR3^b`fhTHj++8$PV>@C)5m;Y5TUqjS zYKnA!uGCn!I|K27Q zu32pk3CCeJ0*NUPOO(tWQB-6V^vAaoXuxxt+86}V=G2hE@+0MiTO?MiA}J(N{=3%r zn;p?#OMoiW^r@WpQ@dTQuF(t`6&R<4!v@jD?>T2SL&avjqGcZgIdiLmILXSM#ABee z@u#QT1zyBhFPcjWY`d>*(Q0Z>`z=TO(j7ir!Tn{&o z|MYc%`mEjS^Od9OinG9pPiGFV6|yAY&v-_#aDcP$Umuo%o(*suxhPej!L+xx*JQ%E zi%XFEYM+sxIdOMEsIkK*9j$NwA>h#UtD{@HU}KFXChd(^#QUBOtT@WFqB3jBPmi#3 zOFqBP8+L|6hJr%Lo7-jv3FHWAO8Bps$-EshrTi!) zwgpW^VRmn5{DZI_pd$}sD6Y<*Q_^t2U~aHw+i&5cz_Sn&-WK=?_T*j5MR32VJWtRC;4p+JXcuO>2|p%RC9C$ zr`mjjO-+&+-Uz|64|1lgj0V)s2B=e0-4|2ov8z#;{kE@?F{_gg&|2St;BFF`>D*;iS!LI%-}$tmR`jhT zXD>Y6SP5M-$#O}-b`50?;bJ&TB-q%Fe5+SQ69w69P!H9Giql#Q-t;-#VUucTxsN|> zrh=)H21-S$e=0{oKrioHYo)JNPr2{N;Li#vDrr2VOHcFSdF8RiU^gQl2;L)DJT?O$ z%2ZCxo-_xv7Gk!zSMwgxK`yMMeGSVjHUz0%77*`L_w8pLsGEVQYA942%B$A^r*8B5 zjwM}0c1tvUs!ICB`tMx*av`GhxS_e|TgH)c>)gnACYLLHXIfqqrBxKpEy|69@8{U= zGM0$Y0@ZZkzVv`iH{w8B(iY87 z#PBtZG+eG(STbAZLj6@ADmO^MjQnNYaYk{0OjI!FSQCRTlONYphec!P%jEU(N^+** zWHGf>iJgPI!`=&1eeML!;N`9-(dZ-R!UX)i1d23P3Xem zRbS1{XOZ``Z*pu$;L4h}aBNb!gHALFGm|)ygl<8cV$Eswg(8;TIeN?8;oKV;clwzl z)bM%Df$7hl7Iu@Di%Y=Aci58P@H z{Fje!NzK~{Bh4z)NQDf6#1?gJZNlzoQnQ1_Rxd-Uh=uK(hHgo+kKfJPOdZj&*S=fw zILP;Gw{1a;jEyT&pQTeQJrdTnsrMpRNJ7lb&0~KeV0UfZK@Bx#7wg#vm(2+JWC0kk zNr(t4BP!68$n+dUw|aYLcSIyj;*kz%DI*@gPw35=HXe;4euiuxn4y!Dp@U#Bo z)3w$b@*fZXJm}Jr{!h*M^JIwQ7(aE;6rd+)fnaC?0(}+$1o0me3V7h9VJeabE<79@ zIQ$XNq$OAS+y8X*{eP*k%ghA&f7j@LzO68E0sBf|URMM{H3i6A*31RVvw;`L{9 zFGFYmppPH7lv=?Aub5Wh=Z%DPx&P0@+@tH;Nh_Y_A-`5Kns}tVk)8>4J|bx-TiT$x z7mH5|Y1&}dM8^M=_W6mwIH*86FSfiM6gy(x@rEPkunydPLE);VBD%~lP~dxedcH5& z9cZNhfnRL%5up;HA`svPj%`t~u%P~_9Y7a{L`Tc4V1gKfFRwp?`ngy_!3fOk(ujzV zM6xOXU+k_6CMjuj7p;?v3kEiJ;FOj0iV(QN`#PiOlG-8F0bLz#<;ONRy5rGdC)yziAR$a&W&!A=_t z>e>KUI4bfGeEiTZvt^ftOQ8rasZd0}{}cvz(zVdnv&OKKS683z?RoAnOS~*(jd0Pu zKd^3Q`^wJZ@{_PrK)sUaf3Z}cN%+z%YOQA+ICnG3mw*)gAge!WX=6Xb$Qu;s4(oCuOk8L3PK1YPAr>?ud1~J{QsC zBb`1Yl*x3&eJ3g| z0Re7g+r+qBa7nnB?2n6*5aiLht&deR;TT*)IetzcFmUq(U3$fq2Ciw&$;D|AiGfQb zkmARfRPb2n*jC4u)6%n`%La_rg_X9s6>27(^|p&ymULnkq4#WA@>tCzyh?>n^<^&v zkPw=_WhkZViUd4GytES#v~#^id`e(O=ZmIfhM+6#lW;&E5VPQ4(k@p>8`34y_86^9 ziMAO%iy~rpw{09C!#!^k_n6J#17#@DX6%vp87?_cFOdn`(CW^UwRIzj+oENZk+ik( z!}&%(E&S8Asi{z2I4zPqEucY5!Iv z0W$R0$f<60i?Wl!joOVN3qR83mV+}GCKgviUtW;kB@q^_;x?DzbnS=e`;2^Xt6*$R z+}`37Xp|XkbIh{D%N`lCLPGr)hf?t|xPMpV^w2g}Po$^5+}H;{wj7qw94`^C?VVkg zLAlgp8Lsn}zLix~m)h<^B)|}5<>Z#a*DXZ5(pVd17A(H9nJ7T!1obw)@TRG>JA6rd zG>ilSoZ5?Bc}L`!yhlgnT)s_xxv*|bRihsY73c527>_13(9(qipjaQf@c1fisAoRx zMBq(j$@)Is*6INL8Luk=Ogysg?y)BB!6 z-!5C3dw34`q%3T03zf4(bQA!RDVS_Jjh3EbS{im5x0o&=tU(LWs3_TCVD%qxPQmF0 z1s)bxZQ@Xv>E@KO^b-f0xq+yW6%7k7Ni;%mduu0-`SdWSvhdJoP2_3>p>%FwzY-bx z_ct7d;P}`U*f4p0R&{e8ZaV^bHIRKm7vXgpf(RzRhPj~y3Zl*2vmPZEo7vW(CIo&} zJ^8I1g4c7>svL2lYy{uvZde!?e#dOO5t+jJh6dd3Xx<>&k|4)rCCelvvQk4@_s@;L z&eVP3`E`57PNLs{*;2#mj*pb`ES+|CZlIz3F0)&IQhw(L?E14< z20}-jg-$osN}0|95V}}GXHU0#U(#W8i$|2P5@VgeLkNfC6ea{r{w@s}!+mud*msw_ z;YD`zl@3>sx10_nYk3YU7?~F<619QHDI8>}<*?D3*&URq3e12V_ z$&{i%!$S2#rGIpT)I?p4BJb+F)r#vNHYRP3QO|T~S|coYW0?)Vb$j(ktg4HxUi;Z? zenN7Rs-Am7)R)AM0dN#d(#;AIIq35Ar5cP`!G%7wPqVIIg9jz?GV$BpVFnwo2eLFZ z)(kpgxR}y^jtxZVdQLTYV>X~AscO8ImvUw%QRAXNtVmIW|3}1TIF=@-E73E{NbD+_iqg&U)BUdw)696#Gt|XAyx_8=rYe0mrqJqp z@CDv!KK13DGuc1b%$ldFG+@|b*NIu!?>4G1;7GkF^?KHDITnAfs6EOD|8eqLO51xD z%aY%SQ$``xA1;Kzf(+K&s;PlMqJ=vS<CA{LAoq)Rw{WOfB)* zcXA)494!JpKb3s?Zv32L+H{fg?R+mQsb^HC9wlCk(TW66xY*d(heNN%`1p8A9ST1q zuif{w9#?pi#V?b1JY^xS6FVR4orRFRcf8CuofkGACAAbBb^bQl%H88 zjP(wmnq_~te?K!^XKHRcK+fBcB-GU+(&R^6GnNwHbI(MPF@-iixXycS)Vz^(*%{#k zcK0m4YWyZjf=DVpmTw91V=)1KbJzWVmUoJaBt`Xv+T0R3b5~fGocM=r%PlqmS(FL8 z%};5Kxp-W6maRiHeS|_(R8nJ_9&j5WFXNca)0gv{c2#*2ujxPV`xo(BOe%8DfeWo^Z(p|wYe=n(AZGb6%3vGe1T z%~y0X2Gy&pgF?a_4r;Jcp$nOmjoJdCm+h*2%GnvZf`nURg7A^bc9Wms(!2R}<|MLm zMN&|_sD6w~OKEeuI8upo4ME~FWm-1fqV^c_-}cDn@|QF;`}MIHcEl)HrLPv7c8=!% zluBUR=u7LB4{bxd5cpkWPoH3?RoO`uGsT;{>&ak|rY7-WY9JV9&&879!=@Tv6kZji zad+o+OZquMZVI~v3+Hhh8?2!(TI93w04#m2*dqKlv>z*-Cm=eQV0RLJtdZ$-8`TuE7eBr%L(0(2Ojsx!)oNjm1LIrX&p8Da zL|>J9H%h1&YGO*t`=}U4hRhNL-q3JfQDfMOuN-gNh~{)U)3r{PfF0HN%@CZRNJit%Wwko}{f z+IGH%oEl5kks@P!;c7IV_{Z_gNo|ICyyDXsJoY7(uboKI#vb($2b1)#3i1Pv zk^aP(fxb{6`qKCn&PcH>N7a@qCo1p3dP84a(qRTuyjhN8(HVX<6fB2_EWNow| zbl?a1=u1qHhXy}%Q4$@D2|0xhgUW~nG%)=ocP-{R5yi`$Ia$6s(Oz~OFgRc+=mQbW z0Dn*jBE_FUviNHxmjD#b!w4iGZv2UTO#lof6n==Llp%<1o#--m*>ILT#0440&KmoM zW->6Hb?jb#e+y#+0|FKMQ?fXp6uJs%6GTMr;a||0Hut|}fDTDOY`X+smQ*>Vw0k3J zRCM)Z$f(lu%}b_~9hI^0L*?);@gRS?1R0{12hq0=?SWgu0$rmI#8`@lQvvOjN1{lV zc?it~4FyAYU=K!wg;_=bv-{21*td)iE<;N>wT+o9&qbuXt>2}=?l*VO*DBbga3=ZC ztKjidQix>ul{RZC!i!M-={Gq=`tyh4KwT7G(7%WF@^78_X3oTZLi$pyc!@2T04AOK z+w|{GwxNawF32>-Azh=yrBh2LM<+!ItJH?;DDlj35$O_f6pCK5f4v@RNAgE9^noD< z3zbryS6$X8uP z%P4Q1kn5(VpsyqUDHzI^4B`?WGz92LNcO|Z{kL7*#^J_s;{DbkhEApuo?ex?dq$B|{LH8fD?ytr?*d>%t@uBN~83fti+EMtIDu*0E zJSA}QWN=~$-mXbL*a)?s9gFyb+CVQ}7`4Pk;KfN+r&d89boikN>JYlWwJPwy{?DkX z{!zyVt%pD_9R4;TxFzF1uK+>PNPx}%8a5P39!Z}j_aD6+4wL@9Nvtt7Kb^bqR~5j*0oM;gj`dxSA(lH zJ)JBV#%wj`jm7FCv}-Zoz3Bm8&|v^S9liHv_e$}lb3nxu1-|R)F4lQ?*T7o0e#LwB zK@FN&j?~#cS9g-&g5**(gz%`J2I8f_PX_2q>nN(*)YOfJGbxF1p5vy0xv{vQw3J!D>1F{ zE*uoB>!sSh!LKh)L_@>+EW{VS#lMERX7o(t|CeV9Z&qPfZk8{y=DwWNbrc>)S zq*fx^gs|o2l%cXaf?}t%U~oz*nLKW}cf#)UhX~O^Rr)K9-H&x8Y2(8;QMX#KAcO&a z3c^xrt=FCe`z#&f#F>5hDmXlCh7_~siiVxrHf9%2X|c@L|i3OQ}HQO?|WtHvy_-Ee>3C*^3H7! zZ~f5Qk!sBTvSqkSl~=?y1~4kv%|(n*LGj+R=?Z3_s*rFM){6aZDoHMV7x(cKS-8Y@?wxF=*vnaX6+gomr`&WHzHK&AiJZ}=cRXG#kfOx#>Qz3Ja)Uyd?_tFLaeZ)A@Q))k1A z^-P6mUc2qTimswILYMNMB`R^|aM(LfNw+TeY=ib3$F`F?ESpyL_m5B=$2D^dN(HzE zZTf|8eqYhP8p4hu7do7t{IybG|r8ASaVb@SX4mM|+^KY;fy~^(} zK5$mi@_6JqLIr;sy5R>M8TSf^B(=(~9ty;351|Z=V}HfrVn^UomF(ZIx3RV>UZ5Ca zdY&U)DKqfqZ1TOV4FB8t+jmFsyfBlPRo{&Etd8P|WWNLlHIaVtlEEiwL0>2o9W&LD z)5v`Ws}ILXFGc;P*)v44{vUQQ_nJEU99G z(?yI{osh@=V@}3S2UkRTP8*4(i;eMZlwvg;(d14=fQm>7vO#p*%`RFF&u{3pb>)ALH7oX9f;+<(0Z! zIuoG8rrg+|JmIW+yDp1COw-IqLvb9$$@mtQ@zs9rVi^RAS+F~4_^Z^*-%=l?;04<1 z;QC$(d_10ocwi%u=(kAMuY0*Q-S2uKG9w*Y3iy)$0}=q$S1{SRa8g3scWu+BtiqCpVk?*P_OlkjmH?i=if3k`qtUYjP2ep{tLRpL(kGCJ+nz##@{WhW*Z#5EQlPW9AI6DKmBHYlNF1{}VkLh_?U#?4X5Rj!#ITf< zhJewj1w1C2Ri00~>o?dEI6Y={FR&+33}+*QwQ8E<=ASrjVn3Tt$Cxd*els?-r6Zcl zniLc6FZfk;F17XNiMc}3quLOXvn0}XOaf@NTM?_ESZa5Fox;+iIoj7-NM`635|S&q zh$E`{;Z=-LThv-qQe82xz65&bD-IB)Aj97=DLi-NSHC0twl*^XQRKb;)!bIBz8@3E z6@6nq^l4IQD|_E=!>l!^bsyqH|B&0s$ZfyMw)21m6l6rVCJ-x}} z_F|Il(1fa5owx%(t39H0{-jos?EZG+Q;}^hAL9o^O4383DY^dahKpZ*X(2Cm8}B=g zne;0|-H!`mj25X_*JRFOZzBaY?G*L>h@Bhn&m0>Mm53J7d67JiS@e8eh@ZIL2H2X& zylYAA)973q8uRv>wj|9fAD|T}LzASTsw&M1QgVfi*qe%AA#zomo1?kTUr=%^GyU9u z-IjZ5`-OSkh2|`Jqk3h{ia)*I)#t<*14YD=_w)MNL=r}<9WAD+#ExEVjCD?ln_a72 z?z^)vd9QbgtVMe`eII5g5olHxB?2;Q<>$Q*b<_Fd-Oa_>A@52oiCLQ)(Ko8EQ}_#b z;gxGggA$7#Wg4cnKJh(6ZYlpZSk!8rlZ1W>^z-)^sTk1RAw|! z%OchiD#u%lr|P*+o;O78$cjjzaj$jbiQH>5ZQ_O=AH2?Vv!G8qMqp#)4a;0g9AC;6tQ9@pn1?1%L;G`18>>s>4w! znX9zSGSl-6e|{dXH*yT#LH_HElp0-ZT1(xdw0%6=y9rXavm#@Kx-}9UoeXKSOX<6l zWrklHICgW{a6*H@nuntfZ+e7CQnH>?*)4x((DS+0Y8=})m|nzE5*+!?-p;q@yNF{& zBdY!ur^>G1rFfQlwbL#x>w~0B=sxfJ^m!E>qNX0!us6H0l1?XCvbl&&l|J->1Zv;bXLx_GB1c)h9?;$|%YHQ~RtuNOXjwr7>0|6&#USq=%tg3B>WId(qR2Q$7{%?c6&kH6m9H_)!!EPss48OHS zsN1MzlwFR<{0*<#;z;W_mW${#F&woj1G}Z@&2%5&3 zAl%seCa6z!yS5&Pd{(ucVdj3D*iBFD@bsKjQ_M;k)>gPB1~)nXxMdGt^39Ll=J9Z? zk<6})^V^V8xTnoSV6FF{FNgCQ$P$L(EOb(_|K59xCM4@pjpW;JL%@JrW#-GMMH$Zy zJ4Y*Tk-%iS{?VAz!9vQElS{Ya-u%fBWQ$50oA~oHt_Dhc{%kaOtf3@n9by;ccqW|~ z1JUHV&(qFNlZFjg6u^DX+P5X6m5@>-a~zk#u`Fkb;id7Uc8G9Z%a}z6la*> z*)1q&NV%ay3>GY+drNC}p%9lla52XcHBm2%M;zWc@Z`x1gS+FQ>cA7nZ|8(k&@@;g z(Tm5drJ4~79-LLkd3>Ln(xtC&Z|uZ*z^ZGzH23w@NDN1ktuSANxxiH81-}OCf_z)! zq3oc`yry=0fcCyA)KsXsmZv_F1dB((OPpv6Wzhah{S7)T%?TwlO#PhV&#ZSjoSYR} zDBlE?e#{$@(T)rSA`KbOxbHUUzKN)d4GtS)FR@|ZEnlshSonUD;YqjGY0>Y%dX&W zg2)Wk_h}Wj4_{B~uq%)zDN5+zy2J81WZ&>{hJ)CB0>1njEAH^*Uz|6;_U$|kk!f0z zUCaa_b&hqkf*@Z0+(ml94(-)5g-2d5kHEyz#Bn4yTOIkEhj%Kp7wF_*r%(}oK(k(0=zJ0@}H z>}sd2hXHSk(6Hc{`W1(n_IDX!Z`e|`-CKTUi0GpB(=u^x>!p^v6>O679>cwxOCh`G zM3I6E%SDO%_GFZ)DpRrP>%~iU+i=Za4Ia~&(U%KfQuumViei@Elbo~20=iIyXrbl^ z?k0>n7c2M;<}} zI72J8hI7lj{5y2(XQZ>T^3{-6?LPA!XT*BRsb2pdZGYX?=JWKA!)>9sySuv;g1ftw zQrvi=W_M;^ZTF<$EXDN9sUk57$hsV|@DRoK$Ju1dGkEHk?b}~!H+r6bWJvVOjPCKfq z|5~qz?0V`qNBup7`&n%}HJ{xJKt=3Zv-LL%dJ8b?na5W9r&&X${%&`vc71pi3#?lB znt>h7kQ+RVgr)LTx64zsq2XhGWV;D@mh~q%*)Z{Ww{im;p=2Ve-{ozlOD$RN<&b00 zpe1wZ4M#WmhwUB`9=o51m^p<*jz<)qS_;O1jOT9Ovatd~myf@}YL7oy{rp6!GBLuc zzhdX>&G{Ip=%IG~1L>!+UOVU=i7-!LN<+bbI4<6iiHuTRgg*3xU0u+TB$+QJuI5PJ zZ^J1{Ua6sy>&=eKzw~4;EVM;Xlf3z!!GWWijniIhW+dG`35lRKcSS3L!^}m$)8&DS zBH`rM@_|T_R3=>zBNCqVxZ~yis+H7M!#bPMX#tF#C74K@9v^A|p#8{j#oq?~x5*BF z)d|Lthr-F6XFNOP8n5N-Glam)u*K=t$##v3ru`dUd_$72MNov@@vz76ub~@fpR==? zZ?;KYVU$YtHW!GCk(UfXVR+ejsqom!!s``jm7e+4og$jxoPX$!|L+wNbb@o|nX?^e za{K)^>2tyTaaZ#F(Y+Mn;zMO!-61?dlrct7wxM`dkbjRY`FWKV^n881c`xmkVjZ)sPj%D&hJ<^ zNht7ErRRsKfEf#B|0N8|23gy^Ly3FvB62Zk>!yi3Q|=h z)O73DCTctpX>|riOS0;ro^zv|xr6QD<`8Sm5#U|B^}<-2-u?zwdS(Fn9P$2QbeyDqS$JuJ%1m zV?dT)P7)geQXNYLnvhUzUJV&#SvrWTeRRg{XUE^d%IzKOmx5TwPFbGi5G)OU)tx~% zR`vt&mD1nnI<5P@a z-X_b?diNLyWn|K_S3+4wHVq<4sy6q8edJA#FhQ2tzSNHY>NmA&nm`8^|v)V{qRp|Oz{Tz7+V#@^z{ z1qX|b1xFA38a!%4{i>%O3FAWg`XIE*?3t4EX);qlC>thgl>3)2KYqC~^#j+4oULi6 zHyQZd9RYftaHC-^Oq-qZC+^gD%c9(v)hYIen*lpuGfuz~>nP3UaXKb)J1| z^#gDV@Z^^C$;W&N<1QgJMhYM-c$33_%N`or-imcMNa^_D`~~7<;%LiKnT;1GAs!3A2$2NFoZ)ViC%CL2PmHVcow_Qv+Zt!OFPDss}@{> z-k2^)7a#M7%_I6>kyi>Sh8gk2m4wM63f~t;CKJCfVN)w<=`mHlv%VJqUGN3ew6!$! z!J`QE0KZ5`PHQ8sMX#r1_ z=4-313ps01_GXcY`yopQr&7!lYU%O&z#29h@l3>jvPy+PAfHQ+F!lYF{_d5B5)l$C zfkAS5ko~K91lh{e*bm7-2Tb?(Hn&zK9+;ylack4@pH@>$#k77-FC8h$1v{zvZmLhR z@-pM-AV0j63sYfWOfXdetgVmKYpSU64~fP?fU!2;%lK_Olbb$D?d&p;i5trTEAKepu#dusqY0cYnURq zDgTn~Xf~y{2L!MPvt0aiv-I_)GMdVbTdgh;{kTQ4Blyl}__WUoB_L^rzr;r05P1y? zS>@zjwVuOlUsLG8JJy;-ZC4;lL|~1m1YPd;fx^r;RW0>) zurekunI>^n71iO|Qy%1}NH0ACrE1ZiV7r&==6NQlBQ+4sngwmM%eU{-;T_!q)Re44 zw*0kpSy`Qjt=OljeB-imsj`3HHdG?ue=zNV>&Qe#Vl#0cg=oZNhApKpe=Rni)mtk$ zQuw@X^jA^S=(9hz-gOpF>6l@MV}5#BGc)yb46)m|Zran~t#m+7&lTY&rd(5DF9ED5 zIvv@@oo)PxR5N}5QgrOa$AQv!(9vwTVapllMS!W;P<7LdwR(=QBogYU*KxTUw02eY zp=kE$K#^gb5&;;Pzo&S51;f68}~VEAd@fl_GngI+Z{0}ad36td?Zmd z$JU8a zB;R;pQFy(XCHzg~l81r@ee%-Va523)Xwt|tRy7YtEuK?VgELvWl~+p+IzFkIc74Lb zWt}Hv=Zv#yJjYv(=ygZh7ZytOzBD9a@yCnP^6=qT1ri44cj;0|d7QJP{`t|H@cnmC zOhvaa&KuwJ?m;G9{+4i|B5E((TyjSytT#U-EZ0;gOW0$}+Yc&}!HwMcInmMlzJoNTq@z`h z{TG`DuC(X%h1yzBq)0ivbs->2-oa;SB|pE=kK!WuahyOOT>`?TEwhhr{J$#s7>V#{ z@CIhNS@V4qBW6~edoi?AVf$BuAVEo(#$mUh^(Y_{=FaYEuhnmZ)Fl!YUc!_#OH<}) z83X?I3+o0W*Ca2N;e~x@E-`a^ff#8%jJWLiog^wat>Y2wK?RveH5*zM^t_e7G{38~ zTTh;{m(@af3^ZzLW7Zhka=u-W4vwAomm-omB|W=5eirX|zQrsIw2@|gY3;e!o-c-|}$eHs`x?rKX4d!W*Z zVTi+Z{L~VbN3B_AV2DR^Q0hF`Pl>iB@U{iyM%p3x3Jt~vpr(=Fp{$%c&=OkScxOUO z)J{f9NcVPEXlMsUO#}R7|H4KMM_*6LZxdI#ta?w0=AD78DTFq=-d(Y&U+8FFaR3xg zE+V?O|MwO0UdJi)>lbjuFfc7;(<&Uix+^KyKGn5{^@6HK`Na;6vrk6wARBSbPh=6BfNt#9^_2IHFs$YukfD+W$BwprWeQ%3d`h~e z$kqjrk{Sx1%7UvSDF(1DWKp>e4s*;SY;ffds7t5cB^tvZoa?@t4eI$w(YDeU^U(T2 zsuE@g-=?46`NVBa_cH37um8g9tIg}PSkN3mo|Tz1(fyuo9gWb7Iv*YeBMYl>ta_i* zNKbBqzM>@Xvm?6lY5Q78bjc5b5%KL~WC2J)gYY{bJ85xGMi+Tg2s>Jb3|rZ*)rw5}kcvOT8-(dP`O176@gN>lt^6d* zaFQZz!=&Wdwu6#VkCzK!W>hmA?ZqSUV^N!`Y)q?$6O-SfM=o#YJHDlk`+LTE)_Qq> zf2@qG{3fIdd?G#YQ?yF3L)??vE8`5;w_eghCKOr*mvFR8Xc@4ubz)O8Ch*@S{a}zB zSBn{-2TY`9C^dJ+_E|;f|9pq*!-6)U_1={oMA_!i*ivN6Kb8u^Z>~8zSs>=YCbrlU z!g#*mT$%eJ=j1~P-$%*0?X_KU;!aVrLl;CuJyNuUjY&}cZV~OP6=Q@R-wdh^)7+KqF#DEkNq+lU2;;vq0Vv^ngY9t3cHA5^xrW_--Q`t*7y zH5+J?P;nm<+s3DX1b>CR%RkrzrdgUCMYcB7n)z17;-3-?=J!hVz3(Lr1H^AVi!Bz=zEM3Or3Tq_b5+%2BwwB-zf zVR;kW$ZGcKM3wxoy|eYRTm`IA@us@qq&JbVoT{*WJ-`IMvQJGSA}1CW3tA; zc5tpMzI7Mu`B+DyiC_?)Lbff_dXHziML=btGmrM+Ty|&a&fe6Jtal$&f*~akv8;Kp z;uBd`ZI{FN=oQ5+ar$y;he!D6@RPfiK*=HAuVs9UUHc6^|5c$7j!e7TgT>C|`h9ix zHtJ3HkWk~9v>!@=j!yeqH|wPjTNXaE6gs=*F}GTPAOr%X>6enKfa@~_MQ~>3ulbr;IC%)Bi3rUMJj9ex{GEAO$6ca@+6Av z%~4SZFUOnNLrVTSh}pMvfEEo9gNW*tVdBu9B}8TzxC=Glj_7BnH#fWT^bz6NX^@sg zY7jwbGaD)r$?c7wI@`j1iefD4jEJ(!y|I?~6W-!QD;{a5PA}Rw=@=IuSw?2M?xPe+ z#xfWHfmAHItHhEEr^tW!NxNUNcZ2!hu*#Xx!c_c2SHg}0{6pO&UU6gTg;w-h#@r2& zfj2EJyG>-$5f93%+%kZH79j?xntaYgDV8j4yoBSfXk{?Ddv@qE?y8)k6M=aVQWPBG zG;riz!k(aSNNvcE9?e`S3T^l9WAB_uVwWdl`97~Q5$h@e51t77#bLwc$R}2!`-yGj zhwE3~kU}G2;ZH*Wc&YSYC9Dhg$m^jB8{Y+8az-6*Y(;GRq<5c&bxrdoDG`_BL#7xw z#(`g{XLCJ#X>g<@mMJaoHpFvt+2=2|c;{w+%Km~lI=n)|2GxBoC{@NeAH;P+m`7W` z9I_CSxFLY%Oib)$iGb`6Au1f!&(xBUa*SSIu2XcVdR_Gan(8c^zpQuH&{B(iC;~Jg z=`*p0qMFyYs#jv70%RKxUBd+jyt!uZ8%kGLJ& zloBUX7YZY^86i=MQQIf#w%)!kP_Gov%+g}E;!k&4fX%;k zyS~7L^$#x`DnHj8bh7!W#@M75jZ(bJ4zR8Vs)W3)-tNM7XF zts(-@{DqmJs5IsPA72!>f^Bg7t^4_Bdf|f=`NNs#u}MS!C68GoQ~k7r3in*{?cU!X z9DHaQm4#0#88uQ$0T0wJtG?m4{-;nvM`oUtCIyN5aZT1Cej{SMFRw1um6qL0s(o3|3y6 zifAigvVKHF$zTSFj8%W>whPIrBf`svD~s85z%>F+$&(+Qhevx7&`h{=u%fTN``|+G zW|A4XriMY2(9sy0iO;WNC{i;P$iOuT^lHAU-USfl5A?sD6=mM!2n-kKMMO_=L|rkT3b1qct2?ZLY;9pDn)`TDr^)eD^LZVa*zEa7(ka_Y}F- zd{ITsO}yN5caxN_-vsTonQjbwPo~?d$%2wvj1H{BdAp z?Ku=`*4`3*t?oe4nV~YE@Cw6CK(jw|xdIf75jHLtcmB1}^BA3l)Z5))u#u1PWum%@ zrp*EtFRwwv&gEulWZDitqyDxFMos;v`Lp)phHizzNznuy>ErC0Jvs(56gMlYJQlr{ zV$7E$yW3;=6XQ7)NmPz>UHdPhH(6Ww;PGqI?n4!+`Mh%hRZ-OetkE%>cTg`#IOwRI z>7iT8=jhlfpd|ksUrv68*=lC6`9)v8Mna5C)1r=OHr^{k&s6bqC7O1B5`M8=73dC& z)UVp6?sd8bsVm91L=&5sxWQnkA0=r*jFp}pOMcryIjE<9GRsQOtx!Y83Ig;@O*ydq zXd@^;a}H`Jnk+Bbf29uQXZm!;5{uu|>ngwdAT%Ls8bEXXaXt793gPSy&v(Oavbv^m zDta>VsUO??-9@|hS9R0@BRgRtSH-G2nm93f58#)vfQTex*{rkadC zUa!D^V&T$e7FL)EMIR2fQCr@sqXLF@e>g4~%kL)G5)ct%qfw&J#<-{GzLT*$#BfT& zPFbD~&rQ{=pM6;oUSH+Mx;of)R20Q_CJN?4>t`qF=^=GVb|Pg>7&v|$E!cQz`C z7Cnm{%C!{VNE~+H+|}@Kb+Bl*w&Vp}3&y4&r)HatF}+55TO%KKZ0r0Fe_R9X{2jXI zJk*>1eiwsD5*io98D{z!h4-PB5G#eo#v2v%*O~|<7?cV-5ID+F0WS~9P*;VmCl@uE zNcc#i7gPF{^ev0UkfcR>R2s5KcAis9DFq7H@fC-?0wQ$4AV2Aq3eO9(fjxA->-wIp ziZrs7`VYtOx_7HG8dp?O)TSz5KNtO)k{Er+Vrxx3S& zDGu#j;S+9JKYc4IKey*?4p1lYH8TrWM-ws}GHH%!SPdc8GFv~qOZMPlN2f%osVG5v zZK%(uNeJKI#$o`=oc^t<0b}>+{3pG%w3v*!L*Yqn#kOju?aA$8UTzlEZ7!fg3S(ZJ zBZbN+d4()le7xo+hJb{GhOaLI@xW}BA1i{bnu7gl`K<(or0Bo@m-qeWhIVEfS5KP0%wV11~oz8$U$5(v!wt3_HwZ8bul28yc{dBG!iY3&ty@vOCR-#uvUf_p+x@hlYcG1S3+5KL}7b0oG z@zTQ!Bq`WCtX7d9+lj2~zvKJa@JhNGirf{#D_-Egr_BGZKw->|1M@p?SYJ&!Ij9&t%hp7k*Q)u5^Ev$4%SaXibHAKm3`6S3A-$bdA8t^slCWNMe|y z+9)XpI7pK)I0Ih^n^~C}(MmGX_l~xA+fe;!K^xZ?p2&HDNpnX<%&lBye9ZxOGSXlD zmV->1&5j#&UYUL~lV~ujwF|CR{R2$|GD@)%4<`|NVMVK4{na2V?qR?O zYwOZ^Q!t|x=x|7!vEunz~K92q}t>;T7|=# zYuo{CauV6U6LY`e9Mw8?kDDkOsylmCxX_WtxE-S9%C-C5i?-U>RoDk&>0nF1lilCG zVng*edTE@<1?$wO#+8+yJCXbIAy%S7%n$ELV)@%cU;Vko{-ar=W*0b#(|b2o-W3=k zxX36eWV_bIbWwfZA@n|OQ&h1jkPTw0cv;(3?O46Fz#zTrkz$;p_57D*p|hWfb%9J| zTHPj)7hNf0QWPMXf zrt2B{#SP2a|BQX?0u;BSZncm>;NxaG6xLBrGz7w-H?`dwZmMm`?3RrnD}uF$3I7$7 zx1E^m)rxE*l%wb8OzxKaaDfx4;x#9l9lf#d%Iv#><@6Rh8TZ*_?$^)%7KB7y3Lk2Q ziW8@!ju3uTA1%r{|E!Rk0dt%-GkG~{j3IswWgKd|Dghc1dwnX`rHAW-*!c{W{DbO~>efULMD%n)$q2|Rq!C2DpbeH2$IjWgb~je;Oy)tR#ZgEAH7=>9js2nPZ^>thFN=s2lVrmtLw3Za(Z6ScfOh;UkPq`OsOZZ(hzTzGcHPBMLi9EoV?%riU*MC|;k`U9U ztmd7XLq!KPL7I?9366%5A$EIiPJPrNZ={mJC4o5-&#rOs{Q0l;n4Cqc@*kG5FbF+# zAg)WMVv}WD*7w4m7k;E2px7#Cb!Bdad#JI$>K~aHOggFfS06L3vV@h?t=-+frY3Dh zHmkK?uk4RrRgrxhxWKuxnG@yBUyiDAg1TKXjj5;I{L1=Lcb=T0*PQaM3l_C=Vkc;0 z&N9{A1YQWo0+w~CYHEZga(j`p>~;3vMq)>NEeuvUfgyMx+g+MSv_Q;15N+2a$3naWHL@64-BYt!!`zbSAgB z4by|k822RK58Nd#{-McA@S?Su>CF67Zlqgyu+uupEz#v7Ew9IbihFv7xfsiy56^&* z4X(Y+GNM&hqa0<{A65SzbGox}4cbm&8Q%BR;nZW-h?ggZum~l#_6t-=_V#T`zDq!# zf!eXV9ULpDsro*mWPK3r&DeKkflnZl7gqz{P&&OwS64L}AbER8X1f5XMmEJz&Uc#k zY)54`+cMN-o+ngyl9TA1v|~Z6JIg{>3v5Mc;7OAz8jrJ<6RcxPv*j0+7b5jGdz8v# zsRX6Iyu3W(8cIW0XKBl5FaUAWs>uN;7HhRptJd+z5Q>8!cOMQm4I&&=(d>Z|4 z7F@2NHhPM*Z>&2zaU10*>K-Z|E#71xGS6(sbo;Oc9Arc$H4IXH&0bnswyVj&I5{C| zfXlpXH%ReCqN^^6o;_7>+S8Ab_i%43qA680kY+o85?;8d)|R2d?znPA&=rkjpvev^ zKT9%rlf^#M;yC{WNqfb%d5MBFLXtnT9~rguBl>twesYB8Jk(f&H`xl_>HYJ|m6clT zf%DGx#Fq;k${@T>!PFY(F-MaxNs`K4eC4)VgHqTBrkO3uFhKGk8b^c%M zu?mg;Qjbwl>*f(H&+oKF@bPek$>Gd5ne%%XCg%Q0cRaBfY3nPc2k%;9C;4b&3IVy4 zKs%kT9e*EcY9%$*5M^pNC}dB;a^w=x1Wo+_^)h zr?xc$`psh|@B8#+e9F^PO3pV;-&8he3dOumD&b$P58}@wI~DXXUM=3f)CJ#4=RP_T zWt45U?51SafQtP$&e(X}n?pj780Tl$E?JnO} z$E+oLy|M2H?EMwhk*u1C{|sE!XE-g7EVZ3yp>KR;ouAlq!@VyH=({hAa;dPh!k1pm-Eevb9>7vpwQWqUkoWddmI0xtOrz|H zu8cNjyn{rOOx5}8i+0;E-bZUp>k)oO{?y4{XsXnUz=vo!uu)Mf%eWA6*@#*$Fg3_h z_{YPE4z6?2BYi__`|Wr^5yJpn73T~%MfcTSn$x1XSX-^k(f#4XNBG!r#4Ntq z`Ax0xFMhJrk#(LVA*vFaxq+YgJB&WA6#+(qrFsNMAM76QF6Y$hOSmj|O9{|vZQ%Q; z~q_Z)GhZ;-^pn&TnP`zNZgdmaIt!~SGm`zW~ zLA*UZF;HV?oO7rcZKWW8+POkduPZmvgc4!N$jL2Nlc*1z|GvzRSQ^cAi2mCAxfn@b z?T(_Yr8z)kz-qMIRuy4G*b(-ktUOJiJgz2^k29nv%;(<>HoM1gPFkqS(^aj*(ag6f zx6&zrl?i@k!fcz>)U&1%`_zSmY%5)GD4C!(I<%wCQBg8D)M>Q(sl%6yow>QTa84?kr8z z1%80Eq;QgLbU|muTZFZO8i_f2)-!x>j);(Vjq}w1LChoZOG!&tbZ=72e90i08M2j~ zkyb{#-ccS%_K%lR?q3ou-wN~@;Hw0Vp&?MlU=8Z|kJa;tC6 z7ZxHB<-6vuhK)q=Me_aA33Wlb0q5R+S8i4V74lfBs(7S7{sV%IrG1b^WaoY0D{x!`I zA2Z_zZ!VLrh$sB?cumGQwvzUsR%&O4+E1Z18(s!0F!Nr+c%+{>AFyyTUrVT%<5v~n-kx1 zc_sD2;yI%;(pCz!I5>Aw@Q*iLIKsw9HS*tTYfsS$SRz3UOnoJJd14N}S$F}kslFL6YN-Mw@vAlk|9vYlaJGvobFC+^J7p{+iA0&5 zJHkheM1r2aD=4bF?Gio*6#!~D6_;)!mm1^12FKgON{W9y*s9{9MO~OkO-W{|CdBf= z1~QyU@%6hBn4JS(;BL1)yqV1^FMbIcdj0eHDw)gi*h=GQDe&fD+|;067i z#4vJl(V<$S$U*^r#3I(&hdO80F`idA3bx=`PY&g^V!4*~K-$BR0h4D77IamDZ-4bj z$F*{e6i&4D2oV;^zn;#a>veH zyFbZGnKai|DEIxHCfxiVEcs_GLa2?TFLo~;|BwfFY+=(CnMj4;7H%t_hzj9hbVqVc zPXN4#iQrmt?{Qpv5ZiZIP)bUnBGV)~8{n*AyK8k~y}U?b*hZW@Tj*Ho$>lAT%>h5V zZFz+Hrlzof-ZGcj6$7utD)yjW@TGl6fD%bKrs{A}p{i1E#2B*Ck}b*VOrN_Ry;N5} z+0Z)k8^MBiT^MLo^Pab~&lWtqJpO4#M|P&;7vg?fc`t0d+z94#eU`)siHP0k61li> z%!m{84+bmkN`~jiOS{fH&X2nXu1IOOjnsb&!1jwqHGbm16s=#YGS@UIaM=ofV zG{&MWE>l~w&ly5AvRkoFO1M8Aqd>(}0o=S27oFH)1`-Wb4q4MO1>v&=8 ze9Tf#`+Zhv*8_Z{V`b*M&*Bwq0iY+Q6QRbE&!hDew9RZZv#A^Y5!%&Hvr~3IbzD(s z7Nqj?f;rZ|$m$%M#G%cGvU3^!6DeVa5m!|O&{?mSL(T{oBm)gR!AJ^$whQ&L+ z6ICW}#N`@%y=8}jID{uF{%FOKNc68xI2wvk`#?yuy0=|3K+LH;w84Q&Dz`fv|kDO1zQvCMGX@Zuu1Z-WyV3zW$O`2Zm;-B|T0>%n71yKpDZw2Z=}RXZeP9E;F#7gQcIyYO)F6s;WDygJJEcNr}rBg_Y6D@#e|1A0el%g#rcx%Pmjqu(rcE7IP< z+L4TH^G@f8WxV;13mekgx4pBTcZRW$k|McCq|We|RefMpKDgE7qJENM_H(L}6H4*k zpI)4%yY4uzj#v&&OP{1ChH`_OJ4BlJ;K{GbG>u2Tf!8u;Lxyp)WUZ};F5oIR<{ zDM<^guhS5h$Wn>58*N_`M#?$eepqT^E28Ya`p|6j5V^`TKT2lExnArR60g9*L}b@; zamxJgQb(x3x#!^nM;h7g6$0}AH%3;&BT}*(_hxs)vJ3-hvURe}q#17N*FD3mr(qi` z$^>}Nj(N9!W-J3)WfhMeBFD(}JidmG3yq$D2Pn{6W{6saMSLcT2QWdCuIAI(Tke@Q zR+cY4HLL&obm9EDFF-3!b2o2DB$dV)^;p8SD>x%Fb!z*qR?MS7U!6_K?cs9f4AJ7Fp+E-oqg$nZ+)SCE*#FTvrhPY zfzs_`!(e0J8=A(h&(03FzW0<~AF?`Ow)cFF@J*$Jx6dsZ2i?M}p%*{(vRo*wH6DIa z-sT6YeJAQ8b31d9dS14D%X^O`ZPu@J`rrtrT8#B| zHr3d;oV92p;dbk;bo?{`3@#K*FL8%u;=%?6Kdbmu?HWLYbD}IId`GBSiqMRX7GM zDF$4fUL+T>u0OeX$k#{s+T^hyxW9P}mYV0f%lmqMLF*%UnIxW*dZ>J|{txI94f%#? ze}8{q+oO12tjGY7B#{gpHoQq6hg;s#k}gBn%UbaNqk`~rLjx`X>u&NUALCqW-D(=+^rzDI*u84%tYQ!x9R0^ks#=uCh zWFTJk&{p{k-v8A>R5Yzu5^V-yBh8;zkGm5N5zB6>&73gIhCHp3#qw_QcHg!I`6#i} ztd-#lKR>wLfMzu%o(l3$PGloRn|xXK59bjFO&w(!WZ=GSId9n7`5gXqFCQS=%F5IK z6q6Ec<=)Z6h44u05JjmEWvpG?@gH(jSXsV@S&;=1al!lfuz1)BsaD9W zAsmQ0QWdT4eh=-36w^X{?H6SkITf*1WTrksBK%DxH9~w?lgyftM-0DN@v{CsPtrhm zE4i=+=;G9s5PLs@0lu3#4Br;JA!nC*H|FOB)b_7JFBK`s(tPcVNx)VoAmJcuRBh>w zI2|R?qHb><8Y@dODCdmOe}vwY{av}{nEG5ZKq3;6e_Y3;^o0*A0vG~O}t76shbg=hm&6yb)F!eeKq1;a97X#7+dsaCix59M#h^Sx+urLY@gur7ZH zlJzqnW+4hYbA_Wv$B4xP@fo>|bsj`L(dv#n1m@K%q{$U7Cw26A6s%dhRB+#QY$$Pw;B8 zU-tNY1d!{oVs*QdsL$1v&07tli%X`bN7gmfkwZfOALE6yt)wE}csTw&6OcoLq0z@| zQ2pO0G!jn%)Q653K7?{~k%-ZR7KSJ*C4?wT7b$}#OH}0WU&)(xLODJq#CO^_yjSpM zJUP4}Ww}i?CIbTaj=RZY?Zbm~_Ib52E+H#mbRz-nzUY26=rUS;?+-Z>*XyJ5hL+D) z@P|u_6}3)Gv3;X+M1B!g^Z8#e%Xikkf!_xh-{oqgAh`toExIY{d8g@YPiv*WX4;vy z0PER2e)4$v@=ZTw&mGS5{wTHAB@TWQ(NY6c_ECOyW53TiW+rxHQxQSg|E!LwAA)N? z<^RWeNz4WVz8u0PktQMdD#`iBqa30zY2-gP1u%@*th9H)fkzCFbLkH|m%IO5r|QM72-fZ|F?ZR)L!PzQf>eLEGmAosoVGchF6X@u7Tc0PI|onA z?qsi9C#afSwmi%Tb{U^cSTtGf2LHV}+?s!WBo}-8oc87Zt|{S>EAWhOru&Ms=K8?? zx$scpzSPabyBIf<4MC|H`u{ZZKlcnVTM24sCMEy1ga$+x_1?Y><-N%UP{aIelc( zc%jZ}5-4Q6!r_><3Rqv))FqxoO3c9Hq!)9 z0Pk9|aok>dVr+EGMdBqOX0JIGbuGfRDy*f-UZ8GV6Q7*%LwPyr{T(5zAmie2IRTA} z{c}AEG^%`yDF6>rg!k$nK|C3S;14e1KL^Nh^gLK~Ge~82r=32VsdE}Dku!P8#0w^z)M{=n$PZ<+Y`)b}`qli74WV56@D6QK93I)aXP^<$rSt7mkJP z;Wx#S7XZMej3pn^;Lz%aMUj#_SUGD_I;M|}#CalO4-h+n$hF7DaXfDCMZkBladN4% z%B-}4HQ5so-p0+!H!7M$8znno;T36mWLl;6kJ2WTTxyk>c(vzU;=R#W-7kEqf5c07T*;mIk_nVpp7lWi4mg-F3P*ZUl-QFNJuZ>-7T%RkU;s zt&v1UiMVPAVi8g}p1`}@Z<`S!*dUtNW9+F*OL&)YR;)iod4Ks#q?N0hmXFJ||FFK(Qww|VE5JoWU8ZeIM&<8BZ)CqlZm3~&$sJt1Y-K(x9hv2;GEa-w&_H# z~4Er)MhZlF-}ig?12iUy5k95^jisfA?p&~wm?&om?H_- zCt2>Hg*&wuX~WZFavP>gxobibA`OCT?CVc|C2Suwa>;2U^OFVCZBJV)ik)VQ8t+dA zxZ=;U)((&r5b+or+^%iOgG9J9E$NYRFDo8TiR{Yphs9&pG@{Vo6zzyV-wH`H*htmN z(U)WmGz8si$E*e>!0o?!y3eSme2KKUJ##zf#5E*vdm!j}H;$zW&Hpqr+MRtZpl_;V z2ogSWPgyS{d7rjvtOEhxfPfC7pcp}85SK+-a&{N9+Un&16}g!TX#17kM|@buAnpmidZ?hW_JFvz%x4!ciTv@UalGetpQR1Uq{?~%jR;@&aNiq#ST>3rM>M2FWx z%fsu8${jzBt+u~qGPy6F{QB^7p`o$l`|y;8dj_ZS^-Kpe<_)pq!#gW!Zc^>l@)<8hpF`6R#P{wxo+D3t5sjf8>s4YRLaP1HC=}7107PC?=h0CtjgpyLB ze^PXfCy1OjZ&JNWl-M7Vc1z6Uf75}Vs1%zzxGeBHD63RcF@p<&I18Z_82y42ofF&~ zZl^f9WELFYG}YWt8-zPT%4j^b8I+*RoBit+~c1mFAGA-Ka)B2EXT_UR&~d zlU#2ur)L_|2i`vj{t~7f{OF%$8*uyQeCFex(7p0blePD2sbG(^8@h*YUk;=R=tJfS z1fUNoj_;GGH>S@Rj(D}Vk1+UudPO}}esbBC(A!)J!UKYtjv^(@rv=VqS!CRS8JMV7 z_@4uE3^C77*66wCqnQSi2|wL$;3rLGoDQMxo(7L&{cL&G5n1Wyz?cbIVi#moA~>F6 zn#H9mYr2u5>1l`rST?Q$cwH$sZ!2eYbE&74%>Y~9gHB(}rqkG)`}3dDtca*WL+I}s z3M}uNaMa#Tq!iqvpAq^m`f=Xl6w+6py*x|runBvjM0g1l^?q1m=xyFO#{W+kRcO(b z9RZDC#(`0+u5Q@w?As0eHpPA{;et&EH+(x{L#DtEL0Rd;SuGDsC;s?aHI*2`oE| zKeQsm-|h@V0f78?Kijze_E-1c>3$^vWmj-J|k5yY~KIrHJ&r zC=w~m*g^+)iREH!r)&wlYMIfveN=!hZP1s}rQRs_3_6G07{<+PfJS2?>v~S^=RgXqxvHrD29ad)3h+ zr|oRo>J1{#~vOO6JFoztrY|ZufQ=~en{qzdj1>2M&?vsIYa8Z zw8n;|lF5C?$7A@h1Xt4kH7YnT&X>kkZ$5t3Zx91EPN5#Q2)d@*r6fJK^(K(U2BMIG zt$?KWDN;RV99^AVlokRP)Mh>ypbnE*(uc4no|`7s1G_+2r1su=!WSpgr$NBFQ1bliaC zWTmlpuV;)esBkTFK&V)PHc8|_@ag1u0s;%km0u7#N^Zbys%0mp(4Wa9o1L_Pa&DQSJbhXT5i$n94s5&LKz0 zJ_oT{Au2(o!MX7n5Vb5W3k3;0HZ2?LO{K}7M!FEgM5CaxTyo%&^$0?zB7&Qbf>{j_^DkGIEd&L(Phj3$JkrIMe(%(qasL4cgND*UDBPRNJ%3|w{&;slF}du zN`rKFOG_;wA+Ri+clf{ke((M6J-;9BIx{Y zuxurSyzIBz>DD(;aRLo?F5o2~$bc!1fFI{Q!`dAwa@M39vX2O3ux^=-Yotmaw3Y}? zu9XoR!b?w2PT%SAoIz(J+$Tbn55S9Jksjrur56;FaT!2=fp1|>7twVgj993*)fYQ6 z%_9XuEx%6HA+X#ea6q1RyGIq9!8T2TDMwQN2|jd=f@wQKrHCcE@#WQYQMFa2M-i#^ zF(zkRV;76KmC(3Ph_G_=E1FZWjj2bJrjb=IwAZFpKRNMrwF@JbbZ$enTHXukK}W&F zd!N4xC@9*C+L4PPNMkFEn<-jgqX5Nx^8&|p`*EW5-eoIU*TmnXQA|hgVE^vuTE}aG z_@s;)%jt?bpr~U`ElhXyenb{k+_JbLqNHRfwV`W9X=UJnw8gj?Zk@$4V)}pJyC#Mb zIRNFwy!2BRq2llC-MPou>?wtcf2LU6M4*htO-)V5iB#jA^G)%P!)cYjGm_GBjF~IZ z?T24iW-a*S1Sd{Kf-<$NGvfM+X|YO*f*&eA+ne0mDG-nAvwpks0slBa(-M8h&8wy3Nc-cU_k$NM9& zPvqLk8hH4Tv-26G4NNS7-%r-;Q)EG(PfDvIR|t|N`=qbGSNl;{*Bn~1F*kB1?_AC} zF+iGAw5LS68e~9Ic#p0&j&lyuHZW2%%EW}fGyBC`ADf_O)a$BG+vArY0{&uwrs%?B z)moa^WlgcYecc<%VEyF}RXb<+qynXMGKQu)Hy!@=R>=+LDz7x`S~!dr+q6bW8O03~ z$M~I;rseZ~Br8iF&lvwfiA8l-tb55bhfl=OyW8tK4d6r!lB}B(cnSyM)ac#B&7GGf z{ocaX2PD1mS+kYALA^wjD1f|zP)2j|El6n}JcH|e64u+#f&E@G-8`-a9j}zvi1HUP z#`Av2$Qcn^6caNF%E5{e`6VF_~uqK*AE?`ucsp#QPP?Rwgo{;yq_tNC4edt*Igxg*}?u zr4!^662|*9$)8blMc$s$DSL_b@3KB+bSWr|)AZU|_9Iz5iCLmYN#4=7#-v|!dJy0| zd{F7|VS`Gp*r)L`eqYlr@)e)w5{&1bCmHIdS!H*j{bIVbAYMC@mKn#TM9 zwCxa)1Gjy2v^otN@u&XM2$^SGRxQ(-W^SVq&HI*_`HpU1|G4_zpTOZ}o(epJX2=#~42>HSt%tzldsC_SL z33wrda>og=!+yleu~R*ZM|ws$q!*f!zP*9#kae`{V~CV!ji-fHhAev({yh1#JA;sB*2I6<)^IUk50UXhxXiAE4E>O)UEj?8E_K{J07vuY_hM2LJGZ&P7)$)D6 z{#_57!3T=X2G8Mib{dh@KK*VCF>*H7mui6`ag{n8$L3AvG-JWkz*}heEBc9jSwZ8R zWA$Sy&#H=O(%6_gd%Ssgj+o)mPmjDar{hTC@|xwYc`6j+aJl*=@GLCr(&zfwOPc#| z)s6Nq&Sr9-kJL5CW``7!a&Hz^Pb_sbidG;&KHVQOBIv+l*0$083K^m_HFY8I2HER<9>E%BZ8)F$-DCY?uTM zD1VZX@bc%kYXib0Wb|3hTBxkiua+!QF6xMVG9k~*d_C&msSB5nSF32PHGg=0 zac2b0#Igmy#lBJJqeTgoYx?HFqR)Ot6)E}XX0&!fQ*F1b>^p%F2bpxdL;5J;nr0%e zK*P&7xY))~`>2IBQ;VSuZ%4M=7WlNnbJ~Bv7DW;85&i~dU?l~|{d6?feR#X2khH%* z4;-VJ^^a8*xp1nr!WnQP`qR<1n(pCL@0$`K^S&~=aJhX1jpO+92XjQLFL`LUjj-T1 zew__%;d^9Gj@d>S9Zg^}!XGEfGSV3Jw@;V0_r}%6j*bhfT=3QS56EH6W7494owBG= zm?R-n9D7p273^8GBG9@ zD2j|E=02u4>YwN%*$Gi&UH<%5^K)z^e!7BY#` z4=%5Im(0Zy>L-S^;1OH}3GO`_g!IICKY_ z=KpQ-oIIxgVwrzRgTG@A@Saru;vsYh0S~C_Bg1XOpMok|fMqg>?2GFq0zd`Q0t;_z zqYn(9=&lET?A^A8FE{L3fy z{2Mr*81}!uT4n^A{56qY&lwb(3$g;=g)S#D#CH?Wo3vow3w1cnSAG)tjtDf=WCQA) zcwM|hvG?;tu4H5bU3<;lEY-&aPK*&Cb3~$llX(Al_TQ2DNYr3 zY`i)U1Vn-F2P8gk4o)(hq?7m+^V|L$1M5Sxey~N;kI0HQncz);F8&)?0I2~*!v4q6 zzYxmnayc?vEG^#`{t@s2DEx$NXyiaR9`}WsuIvwH01z|2kZ54#kZv1PJ`` z9DeptuwPPA!Swe&fBPj+oBr|arCbR){eZV%ix=812_mvGATyVSAHG&QHq1z&^0>m# z`AFj?4+I61#j;mxa}4Y60oY5sc`wwMVX{eM03g=(j~cPl4gl!vKVSxg2>+8F%`@QG zDT_yRog6lik<@wD(-T6_IXQ8YBEsmKzSV{$r!RvY$pA~a*>%gUA@4&cN+2kRcIjLkmU z5LkW6XgCrtx!3%!Y4wc$7w7!Ptp7L7{qv{^Vn3VHVx`}y6nECRW2=3_w-9rp_lTCS zf`))Hoerx5V_jSTB^KNwdNrEF|5Ld97nuM30@HtE zqMjHb0^`x1`0CDP$EXTJTp2nL;8Ott(j?42*W|X}>wAyU^uWI6sp>3^VT^P?mY0BU1~3Ba?LMkcWFd)_lOfTC%| z<`?fz0ksi_otp6cY&O?)1Py5~7vMfcVUE0=caN-=?*V(`gXqvs+rS=Crm3ZK2-+>b zE{LGY+4;=8gu>hZ(u4mJ+!23STmQuRe^3Zd&okIIY@LCk z)9qT2FjGEKVGtet3C`GZ`W&^kJ=DVVrA}nw&I{^X|w8w8w>@~Xb+^+G5!I2HJ)nZH)L|5l4wp7;tj}ln4Y1vCR;PfGY#wx)g!!g^<86<||Wr zsBP7?Zu6Q^XtjC??=!kGM1uP+MP>BDvJzRZ4hTIa6Hdt1u10&dJPUQaZ>}rMO7-f@ z;fSw|MmwMJaP*BFTV1~uAsi50x$O*2vFgW77FzDzxS=MfG@4j?Wgj{1eU2x;-y?c; zh+E03NG-b9b(Ah}y5BDRaJb0Ko-5iJdVYd|DC@>3ItZ+fc#6;1^# zasoP>I4q2ClgiLozJd2c+M+s%2iYgtCa|SbF(*WHL-tX@ix$}`;O8m%1uf|3p1;9<;gU;F0G&agr)cc) z%!w8^WyRB|_A~1C+H+A=WrAewG==%pw!>qk&BY17ApAZi8R=SHii6VjdQ6_47!QlT z!qOsH1wY)fiz*G0aDkAwrfW&+S5kK_gLCveZ&R*FCz?ZaXJ;GQ>SGc4sdS0~E5s`l z5dODMe}uJGsFe(Y@BUmU%j90!iMRQ2iF*WGRrybJMH9Yl{g&d>2uB@83uS!^<*Z=g zjzF|z6u&d%KU|S|z3re$2M<5W8C+hXeJJ>AY`_oF_|`BfL_{ZM?RhLV?c?uH z!KHBm>VQFuL0)|tiV??;_i*gnyQdoUS;_hroLRRB6FxWMHEry7g={eb^I8%XPf08@ z0LLE5V6K7AQBM1441AA4pBDq)fMi^tQoW>1(Ov-1e2SsY*wt-MSEB(fh4kO-Nw4p9K9 z2p&$lUx2I}K+pxsv<*I567tQhonPn#;zN%Yvsy{5rr!j6(H;+;rNdxvv*t&t%)X}h z-&BaUw>%5&xgtl3@B00-jm#c*!soh{bv%}VS66Wz=`s4=%4Nvcl+F@Sn;rGbo0CD` z?Iv2#KoM8)6&5H3z&k_ODF3YqyAxuE(eL`j3U9Qbb%3N4$3Vh zDRx_>o@~$iXZf29Ui;dYNyH3yw5sR=2KjW)0TM~(;Som$Oju&eJFK3jJ|I>0`3Q@F z9TPTs>>ugPZHSd?(tX@Vw7YraYY@%4nU&|AFzZS9;H`B{>EE7=+kEYF+V0A3zb5Cg zMUnfsA{>RYiL8F6?A@MJ~i!ghS*0%Hb619d*)D za1|c4ShL|~B#}6DEM6PbS8-f;->BA{;AfA<>F|dc$65OXtB-i7^xi?hx!w(* zPA@crYv6D9h1_$E7RVH0KlM}j3_atodI9rvH{vo%$0pm`TFBRyN%zmTn&SYmMEo+`Av?6Lhe2+)mFV!mWb0ypm21)MWTv9+amP7u1G|1>>=hM*K-l%Ox%JB5 z!Q9yXg|UHgAhC?7+(W^wUL%OqG8}o1q9DI87`#naFJhgsXpBJ zX~<{5cq5ESC|54#lyNUze0Q+slay_aiBQ_iYc$d))dL#+e13*ebp!LdSd7Fd7;dj97Hw285OmJyY~?tayeaPe(LE|iy}4-sWLd+XvT#6R4*`+Lvo;+D4Yyr% zydWB3mXQCDWMUa$Vu?^1E>!P=`-?apx_E9oc-kCsRoc{W>EP97iP>lTC|X?i{l^Yx zGR!~JAG&3lv>C#)&^tM%9g}H7{|fReThxc_v3tPN{T^o%?GNXd6<>_jySIf22&(AD{!&ddVgd_4 zV9$);}VSiNyR*?tU%N17hGzZ2#zIasKYjV)O{9Oqn&$injsle=Cr+1Ln3Km}`Q%^kfp{Ekxg&mjW##rl zUo2Ub9axo71>{N94@7M8pyp#D{egn41=n`VFVY9?~D< zYMty&nJMo?{cBpt8sFdDe28;JYI-yi|Fo19;Y4r1DA>I|iI9&&MV*s${d0@ni<62@ zEGs1hGA)xK&9F6(O@+RTXLd{f)7@T|_6Q_z(W6t@xEJOCSZ0|?_==4rZ70pQ%Ts%^t7{${mZSNE%bHT3O6*l^_%WF8i9OFM}h^a74#_fD$;O=OMDIHtTu3WAUldE9r-LZrXPAWDd(1 zv|=L41YWr+?M^TVEC$0d?Ja^BR3=4!hckDg4?=RizbKwLMn@NIn&Mb35tC(w(NIc^ zHqlY%Oall8c||CyPS6Ya#jNH(!$WNLss&H&r&f-tz3TN;sAgrzQ@(5EvFBM0gc8UW zsEoN!IoBp1%6gfD=#qX1Mv__y&F+1(Q56;YquP7LVA%AE7X78#`o!g za4Qh7YoneM-6}5q8kw=EC~z9j$;&(R(T8+be$(@6&xRYa((B1{$~d98>)ZjYH6_Xj zgy!PCV3?KH_eGM8b{PF^ACzq&R-M3lKU!CtD$eJ`xheOfH*X}DtQSsV3MH%<9Zb5@LK zTZ{pZDbRd2Tvo)|?l-(;`B$$ETm*-33W?YfW%=wSTs=S)7w{Q*<-~;>jvZFjYvrmyI1$_I322s`y0c z4cw7>gkx=S==G(jl5SGCP_E3Eo!6VqE5q52XyKRj5dFrWbxG1#_|8#uxr~w{>bEJ< zyuy9&)k)ddyhba=si{}TB*??Co1^I(U7aIZOZ%c5jv zm~D!Qgx#nFEIMZ$G8e{W`n(E(ULI+WGK-px#G-#q`e01$lyEySHB+)C3p0<{LCb)o zBF`^v=lD*nni5g69sG~I02~lB%8?ya#25kNo&JbC^-2R3Q!YU*v6L@djV|JkUK2vskV{cZi>IZX`GuaNK`#2LFWWu+wMKhDJ)~u~t)i#uFA)7) z{>V|>oc&q3U*B!|ab&jA;)eAC4_!kKi;;?%`Aykil4Kxp2@Ib%C?Fz=TOCVBhoCIL zIv{HJIX*s^p{$HmgP@!dcB-NpqJppHyd?=1*yF|l}U04_@6Z{%o$Fovy2;+yaVrSCBmm{wwk^_P?Pf}jr78R!^aPl)T0Tb z=IMLB*upA>^_ewF#(66{>8sId! z5@ZpdzG)JavtkA6l`uex!fHUKG|JB{0;5jzicgnn>fA~_oo#|l^+<49OEwQd_05R; z&IPTkO58d0iAjzz8>PgbAP!}+sGwUj;gjx(Ib2zU{ z4zl3%9R8YBSLP2R(3I44s#40&C>7-Co9aPQ?0=qC)a*3jj#%R&s>dWPFGjo_^;T3_ zn1MNvrdrh^aQG+9B=5H*g%HrqAJN{!MIW7s=dhW3yS|mmWPua)0U+9+t z*f~PODy&L|^r6E^2#a9|QyBO|OE*3`omM2VLd` zh*L7(XM@Q2_KXsildRbPrn<1gmK31-cEYP@#m~`= z?_ochj&G5OTP@fkBk*9_A*mnKD1;=1x>#-+k4%fIBV~LUe(=}BitqDwAOjfRJ$jte zs@KLHxhG`myGIuxKnI;bmY_NO!y{sPT+9rMKmg;`Mh#$mws_bXPwC=9_GTv!$8?($ z<=8NIaU>*_$w<&Sz%=^}y8ZIdTL;?2JqB>>z+ka<8LMJ~Z47QRGV|$9c`G@9@ofhz ze5}+t3lY6#@01^pr}2UjA#1D)642_Xhk4Ne?lh2$XE1UEJ7 zgoTD44sEJ3j{0~39J51eRMhZtag8LMhPhPa-cSOSHGLLNGmRR3Iw-9Sn~|AYT~kj3 z=!L!fc=xz7RL-ya)j)A%U9O*=5?)po%)kuXVfcBffLsiSsP%&O?jl@wy?o%R*LZat zJ|6v4Swvn@zfdQG&NySe_j`{@Q2}e@H{St_{&%>%7X05VwyE2%$|bgb|9&@DWiZyz zW~-PMof3^}Ab!n?q-51hBi0_Q+9p<%q6j<*RR-q_qx${H4`awWm^jRoPR(F&=0|&$ zCxTL+9_qC+nZ&0Iv>{|=)5E3a}mgU?a$ z0#{=jC+~X+J+G#0fA1&1U?#8vf0nDHme=x<<(4%Q?FzvOqZ@uvK0=4`Jdtjyj~iE*bKmU8NB}00 zwh|r}iHsUP&;$gH`BM0zkDW+f>oK)E`k}0B-@8~Xr!*gSX?6l>fU^z7UK+fjlWibtZmZj_(Ole+6CvfZoo_g8%KN>hEGxE#b__D$%c5{~{x@N+w z%qDKQbSvLU%d-#vEU1y(w$Z@W)Kuhmt2KdO`9mjrTAS|LSg`}0{W2F-GF52QJ{w@q z{+tQ1mh`*9ktzazluMz5BDf9lMrH~M$bT19cDkUMpb+W>dxp>{RxP*X9OLn(VQaL2+IH-R_b{Z@f!@(P$Fn`e^DeEf;_6lo zVe?7Dyzfs~Yimu(IV!Z+378nT%?q%RU+94fMscz5ED~8Y6VsWfh6`T#q1y26*M87R zi-3aMz6+L@IIX%IP-wNj%OJ!8ml8!Lsg)>}>@S|LVooN41I`FilY0; zscwQul2JN1QQ~|)_&k+rY52l;X=G?(t*bGQXoK=2DLpfwWDKaOK0mKfkVybBUspmFdCYp;}3c#2QHi*(!?E3OemK^3Bd*Ej<;eS*Tt~ zXo(6czJ8k1^1uTMDW+(b;iYd@huDjVySNMEh*UIh+Kijw}kL(5IHu zl0<)V2A+Pn$!B=m{unf8ivhhCIocWL5F1N`S@@m%vRKFTegK?CZwGRQF7@))QW}7q zDUcDVcvA;04_1dwJCY(PwXJ+Bc#*2pkZVoT|K(R>O_Czr8nmQYJ{fR@Y63^3 zq7i{s1{-ck<{%N(=TTgIMj%vl>D=nb)^Qr|VD6^QL5#NId&4zO%lw>M{cram(gT$h zwamMWxP@PJi(xuCHszeReMl;2-IGBgy=)JEog-xQHS0j)#I`PuYy%0?C#TwfkJyM& z_Bvc&3zVcN{N~Tb+#lao0xxg@t5Sn2_e$bspp{w;kA8uUow=K_$dp2?IkPnL)KD6! zgNc`ISRU^{bTd)n?-!P+rkS!XF0Z4mi4~qloraaSI@L(-Fg=IfNL?Hmb)|P)`o@(W zlz*30vG^b}t}>>ji_=#y#J|GvZP5CSMz5A<#PsQx{YwNW>D^^>o+ zd4Ss%EmHs{Yc>U&epgXN@mxGaPlq3}&?@TwgdWGu!!q~X9I)~dN|CK4X>w@E9n9`? zv-H;VdqEGe!`v&DwCL&4MZ z6=;ssJ5N0j6?haV>|(-SS>P<|W|^r)qsC0kpgk``q^7|b1yIhgN~*HXi$X5$NIa;k zh_}Rvavgm^>k<}a^Pk4IPV2<&=VK11sSKE$;9=-SPB7?dQKyQ;BC?G2_q*o!m+Fsn zYyMj({mKDBfKd6&03M5WYpql&e_N0}niMIkSX5JF$)12&QO3%|DW)eaNvA!pA}Y|+ zf8b!@hE`2IbQFIlEWI4(xaNO{V=Q{Z$f1IC7u`)PoxwGo%SPr!@RxK+E%Vkro#G zZSX10B;!>S6lhh1rR;Fm@e5!@%MS53O!0~gfk(VFY*JxlVvdPg*fKfFg-&dTmZgvuXi2ej0!m^7p}t zrTH{I5E*o7sQI1rVnYkqOLOn`;l0Fm8BCLejP(Wh*9tEf>X=ATN5KGXfFmkE`3aG8(#QUJI&wnSF z|NX(L$xAg~8}kQpPj+>eLzcm8LZj^MOa8sKR8YSvUyF@m8_ZA<5HN@XIE1OPQBkbe znwr>f*qYc_*qQ{9fEzX_+GkE{*5g}*DKXZyh{`<2tU~6Q89a*nL<;{u#{R(Ev=bCB zUd$zEnygHN&FQ9mpgG}3+Mfy%O%$}+q919sdpG*myN^Boef2{$gzWg`q(oMA)Wfv# zlqE!S8u|S4<(#U542g42bMu6r59?0sZ)_QL#yfhGIdUyq=>LJM|L@Ny51Lo$xW;YP zLJihCQ(8V6IlMAT7bpHmV>c33uHMI$Tfmh}r&p!%8n0VPp{Ok!nrMr^nZsCG?6bRoxy7>94VuGPQ#?C`kbvE4t%N}5OP^?Bi4_0g zC(OrfiAP`iJ$1(&gFx2-DziT??|wt!w^;aebcU}Lhh0o(kh5CMvicvrrzley4`ro5QLAH-dZjFzW*iLXH zIG(Ij7>Tj}lU`$a?qW|otXn2D^Oap(?F8j^w+g9k+Rx;VAI+Mo!=)$4-ZwmFu2;&1 zj~9D|qQS-A+w$5aTf`@UhliSxTb|3fNb75tLFyNjn*m*?`|1aA|C6a-Mf40zDHZ<$ zoI(EomL&i4C=FFX0gfVHt;M`94ad=LevbtK6S$wNYOi_J1$eRYi;)94_nTFYr_+ zS0!Zp?_T7JyXSr>&|;ZzJ{hvIJ}^>#MILC5M+*iUDt2+KwTmA<|!(O~5a*Xc>H_9cYaRzn0H2mFU_-+(f* zv*{}%{|`DF#{YLv{p~yIbKrAIg<<|zUCvNxN59#?NWlA${&CGP(s9S%gG@?%8s)E1 z$$aiKNksUk)+ma-t`V1PXGGnuyt*@yRdT%ocZ~(5@VOMGg*YYCflg)6Csk}~QDQYb zn_|Nh|I+v4f)j2BY{D-OGsaxd!#(YPKr2jU~|KwiGnH6MBA?S%Ut zCVarnfDz2-d@}@@-3+*U1a&W5WD$Gb9jrW_XrHw@kVEwC*Y_kIF>X7E%_@b4w$pPh zgfUNb2PZ%dDp>hIi1MT0l`KV*gG74RQW-uw^lr!{1*M6kVA0)u^$z>$!cY8u4+wNj zj^L**&HyiUbnjPj`RxV&xEj5`y>`Cly=QZZ;$aYSuZksI^I&^fj_kEHmh=KULvYxm z{hBHraepvRs6SBq^~J-|nCpwXlz?$Jfp32n3`Nh%C%Bq@iHQNTQ&AwU#@<3?$~vg3 z3$FdXhY~vJVZ&-%o0#9lcev^lh}3cQBHK`%7HxP`_ja+_ETHOK;5&4ku<7!mn%l)r zfNRHdZ)j%L^zK*U8zOU}U*95aF&^kHSI#+qQLgk{jgzzbL#wYw(=!*R^9mgI^w^8P zuqJq&a(mfsF${`&;a_t>&?{Gm8*q)+dGgw$7y0ZovbTF{t%hnO{c?**t2e|bg@*_! z{1R?q99ns?n$4a`rN*tj{g?OLzclqYz0Q7!x_{3#KxJ%Kuyf_h(!ER^rEiArsQNr~ zRYMmYaYvRnIp35JbWZO7axKbQbQ;lpILlhw&Gok2vx`a26CLX<#+l+hraGMos)}7{ zIMzGzJd5@@KSgY*O_1=9!0obGt#x|=&X7#1N)&(Baf#<4u`*m!Mctz7Q1A#|ZiXFU zr17PK{q{iVE_@_Sul#Zc)w!of!+e)Lw{VR&vM>#L!ab_DO0on_uaS<|Yd7T58v9JD zRM-OYfByK86x3hgNJ66`Nay-!sa@Q1akLFfSP$g*Ny!{YgxP%HE4~|f0Mi$ zuW2V!-YTVgH~-z~%GL)8jc(nY{}h2()49TUd+~KUdhJ%YF@1t_i6q=6@*V5&;1}{b zy5$8w7~;cWVE96Ze^k{CQ7tY97oR5m^x#^K0_E|oyvWto{hHaM;qLE`YB*{+yMrzh zJT=Gn=)KPgR60&*{mu%CX0RD^ENa{$amGG`37`cDMg-(DH6;M6Yjmc z6-V4ZSnz=v`CV;OIu4M(?ib-nff0H_T?2`Z(WQ>n9yW4&R&EE=n_-(aOVcg};=UPZ z=lWNj{Za*RFz&P1%jDa;!j=0r^bzmy-^)!oI}<*sKQ63@Y#5#QdYvq5CVMCH-Wsxy z^4sr}tKTQw=21ZIZRHmlylF3eAQ(qc4*B30`1ELh3>rO%x3!V3rl7<)dk8x0PU16N z7i04Ys&(J|2Kd=mTsZu1<`o-W7QNMA$u6+?6;5os<)*Ma{I#<6c*f_l9QB996?F}_ z`hH9Lce8mWpwK|Y$~Jwbp2mjr_hhm8*rda^T}U3}n=F@pVGbz`lO*BYiww;g(aGky zcVE$*4G&xFN&U|sSJmvQBI)1?1y=iScH8?sIv!rD#k=CN?Ff0@3HIFDTgqw^6za`= z{MGMbe21RbToL-y4lad3&`}(5FbTQBzBSb>&o{$i)r*kUmrVxP-0?98dJb{zMI-xk z=B9WHE-O=^* z<>F%}Yuh5;Hmp?8M|{NFZyRw5o=RX?Rp9G}%i0aEu@j}35ToIzG(s40)nMJrZ*CL1 zM-MlKE2*tA`k-gb2ALmBj${Fw2Y%D{km&tdkIFyScxAXpa z>b?-68e1@fn=ecA#*O09{my(7%Yp)rhNxFcVPxuxD@=~$d0BlWB#mutCkxXb>}^?H z%tgU{JzZC>aquToO}(ZsFej4o%PzM%;!Od_1(;o#UFm^NH{|E2O;7LqtOI zsy)3HodNQ_jg7Fz{k!<9M*l0TmD?*9b790i8wDD)VY1sDhU-J^8XG+%&DE$8pIgjF zkB7#>Sf9fd&DErCucKRFpW3B)RDJMQpltO0@0mv)%15IVyC1rml-q2Aea3f3{u9B5 z^G+jtv3CVbsY(W|o{3de>{TVN@+Irv(NV<1mI`>!V{zJivSBmSCPYcrzb$ribI$mC z-(rDkT;1KKu<{BuP$T)yUH&Y+K0C+co?Mn<#=)B4<*f7LVfj?3qBg35<{Wdp)KfpQ z9KkgaVBQc}wJIf?#f|V>Dn&(_R2u%i&b)w?6Q{JQJh*)ogBu#~mRs5O63Z9ZutS?P z$UiwZ^=unej@0^~l7XP2u(2X=&>)mEv$&!@@ncow;M!*0KlTCyPHA#Wfqn8-h?GWt zqo$qn;&M^_npHIRz%}7gZ^lVFFdS=e8~L#J+w5CovTMwI>clP}?8;jCBV{17jc(Ht z3$pwa+x1(*={!rX<(=HqIMKP%Ow7;umv}y7bGxyf#hy)ND^mesrjq=e)poddHCzg_ zJi-ggqO2O_I>=^!J~rFa2CLfrb)70!B)zrup6R#h;E&u&qN#;#RuM-Hh^BC`=ceVt z!Z)|S|CBM{j7^GgOE&0!)uCTHtuav`I@!L}^Vke2|rsLzgvndh^#0Zf}3@ zmC<%5$$Uv{ctE82PWCHjjF-QDYH|75Hg~E`Ox09srIFC^dw&Tr)ekYCGTS}N&mDO; z2+`=KTO>=rDfjcKaT!RIr)THJ-aC-)d4{}@VpRmXcER5p78PRSEYg3vz1_1{GU1gn zR2B%n_noDCYYvjD`kwlbPeH=%gtI62=|&{-wKCoa_0$c!?uEIa>lruX$HipwhFhu< zUBMLNlyyoTU7DSVxeuKfL6Zg}P8dKi+>U?5)9U;65^RXySC%H8&f*nwf|6{^dR7f% z>PxXXap-0y<<$Kp#ufp3czQNw@O|6XCTmo$HgEm;^zxKStM29kCS}O?tVMB+30La) zA%eNXro{UDm%tA~%&WldfqpF{I9OlJ9DPzsJaZRcDG-I3FAeC4tjfiBNrkkOU@UQK zRQ*3>ePvi&%hGic2=4Cg?(XjH?(XjH?gSXz0tEM9!F_NG?hxD^zPWPlJ>PwQ%(I7? zrgux#s+!uXtJFrDZf8|8i*3!)Hp*~R=g9k*O0rMQhZl)aCSn(uxn|`_=u{9yD;F#{ zdORWf@j+$9?q`#tNo(yL91>J2#CUQKY@#V<**pOJY z!k*elJ2?!jfl>ArR<3DzQ63pOkeGOeowxYx2#3k}!@iOFvzenpfu6fqTb0*324&d6 zq1?qG+@ZkUsL>D;)Bjjs8Lo&-oNEXm9p`NvS=Fpspu&hPnX|6RCXz8-A~KDKwy?0E z3MX#%LbYwVFz;7F4W^T6ZB_PrCsQ?+lFDxOy=UIYGZJob$37P=>8-zQ$}9Ti-K0-p_CvwS6usQ>#?UKvzFxhUgU_?#xxl5zB-BF zKm2yu8#2BZNN1DSMafw1^7wG@ZmRh)F6G3j*Yl!-+DOQ58<~fJA#}@xo$w%z>nVYGGF7ZbF+OT_%i7yX6joHB5g}dpkF1j~(TN zSmvarEs5|MP27DUQY50TOGt0)lVo24=(39{ zsHJ5eTr?8yQxodRQcC=dz7+zgGAUs}YK3)SvYLTB8!VJo_OAR}g}BrOq~*7UzN{yb zyho?_B~{##Y+LYI{{qY*X|}A=YR90B8VHkDt)lL9V41>CNeS%p3kuXwBAwx_F<})s z)>Om_6icIhg_a-ZfF8v;Q0oFQw63*)xdP&*PL?ak36E68M- zJP~d(w!usGrRk+?VjRV!IEgmDZItjoc<%b9cI!w<6FmjRt~41C zcFqcP=g`x>937n(*H#Bxu?fpwAdz-amoXkzgb^G6$d%U}8W9IWfxiN!tcq;24x?rm2_#>~4S7WhqhN@pTZVFOcFB4)l~^(?{fY1kGlQgp=8njk zXDZew660@XPP-I~aGo>VOj4e@2~^WObuk-d;NgD3OmNW)o(-?r>GPCCS!2i*dV4!+ z$=s%JKQp4vH_|s*=b*d#ulDBM_qCh}eW?7eipqCOvSapF@R8^6>y>IHqei#L= z{D=1i8()FN+c`sqJWP#|*I{kk$Ulr0$G4`CU$s~bred)qaMEFs{-n6PH9s$dPC~^E zE`EBGLRD3~kBLmqt6tQnj%=b^#5fMCD!U#M(SkL9=+jY@gh z{c>J)w%(D+dD0gS3*d2&S*X$kd3Z;MD?Y0|#2SJV5r`kE?+dktil!mn-ra8(fsui} z+ZqBdJpoa8j)UUe?~LzXKO}cV{Z0%w`*dBNus#k!XQQr^P8@Gm0$UQb#5O=DEF%La z@;jR%18-^r8UU5Id*zuKXB5az+1_r?C z>FGOTHQU{eQQGwqwCIL99-%=l1~<2~tSs!sP9=};J>A*a*(*s0sKtt1h8G`@u@dCT zmZ}ui)FgObwE*~iGw$y_0%p{!)F~(^tgNlK^yDmsBdWL(@p(Ozwa)wP zRH>*u3$wU{Z&cbmA$cDTb}y;5`gf(}6MDa2KRoK=jjW8GcZ%Q<2*zy4Y006C))OD0 zD@^;P9bQ;WzOSb}-G&X3C8OcTD3?2hZHjJh;G11c4(~6a;w1~+?MdCjSne^5nhq!6 z!HnyDFEc9IMTvrGG*Lo^0{`GR+)(%HsQmUUo%|hUYq@tK3s>51E>%Lkno7+iJKrBS zv(6+n@v({N_K|wIJz+B=bFS1;k;DxO95X3YH_+l8L2zNWNQrCjmrl2KL+11PiwdVs zq7V@h6!@3me_R-Bd}v4(q*Ou|zi~|L&Rxk`-AQqNc_bBVAE#h_zVp zy)%t9uVE#h)@p@f}#=dZPsu@&>FoZKe>8g z7b=v@$Ni2;81?rB^zosg@|S|i^mOb^Q{L5ba;N-B=2}_g-udxcOXE0)`8M1)ipf{| zt>4n``C039Cec}2f=YI%r_UGm+S<2YbeafiZ>z2QYEuZWS7uew>mT->k1Eil^(#Z z$!aDA^>uH5-=01vbaGcj1rwrbl^p>p4Z_HX*xNI< zdzF-#DQSLix|{>{@g8+_(?_!i;SKf6?3vHq*Qa4D#rQ^)%E%CSMdKl@MYqop8%VD|FIhg8bE{o2+M;R7jhe?#pbi+AnfAL%+(Mm(@opwf? zVFrd(TCr7POCF0pG_-Fyd(d4+dei&TR)! zmzv0dHX_2L93tj-e_FK~1R{r=sgJk?vrPsF&_^oCZ@zW8VXwN}r{?$uUGmhQ(gt&( zMuI18G?8+y`_Zdc^IZCV$4MGbI>c-oT|r+e61QXL+})u4jY4Q6Axuhbu$`&9u{{j` zGHL~^@Y`Cb(ke!&Fj!X*5~8r@!DYsZhXStjhC(u9!I4o=ko-r@|I&6R+ohVpZ6_oo zq+gqb$56)QuzaJrG) zmaj2a8mFJYy209ff)gt%nf>Xm`McRctO>oheTEHQU47;lgmC`?u)i)LFhX-|?+pCt z0PcGBOwRY1i!%Ar>X||2h)=jwy32;(3#WM7cRWb@kc{n$mD(dX{Z;6A=?D$>!wuF7LE ze&fc$s-Xvn11A%DaCWp&4Hg{?$(>{vhxt!K^Z!lyH8>AOk(i~KI+G)(Kh>1NSo#pk z|COB2ZCK^pd!L)(&58;kUq;H!o#(DA)xh2c$>@6a?3q`K!;cP(r#oKKk-SCbOK>=W z8iFeHRA-9u(&<)(=f%!AB{LquWuCDh22KcU& zgn~XnF0By)za;J=?6I!3INzlX*GHErYxj>|L%H~ous~t{7ENdSIX1fgmCG-|TZp%6 zY=y^qtF??39tuq1v!~a#SSHea$w?8({cdU}&WyF<3kSoSElC%Ps^rD!_Sxo0HU~)j zqw>#}N3JsyFYc%au5ekCxF(|pgDR%%2})loR$qwUGk?eBXZ)}XjD}U!Qpahvld0ZQ z0*;`Gq~3QCm2sD|AQ?78T(R3+j?@|10H4xT^r6^UbmvC8Bd=s{qG9)+TqiVFATKwH?JN(4swDrmN{*;)M zH1q?JATbgem4eB12oC%s=JwXjp=}#NL{zl>8yY5NqxBpG3oGjrr~PURNZ+Lu6b|C< z>9=QQ^6P5r0CQ$~Ls7!|(NHW-^8^B&ch(p_{z=o>!{^7M4U=(OY~(Aj*bi5cbems= z>lcna=k(d+*;H5B@97TL5e3Z$?@173ZI?2IH9g2CwJj-a_D+aBax~)s@C-Pp}FZf$t=az>QI6`5HptmPS3^!Nb{Jmu^I37^;aa`~FX1 zPUT67@B(jrx+f2`i}$2SarXK z@kZC;jhZ5ypX?Q9+kHM^l0Nj}Q3PJ1ZPtG6qE2$~jN59+{tN}oIk;hZ=vsX|)l9Gw zIO2w2PZV()s8p|tijODf;{*KLxkvKp$|K+Zdshgd~C7TWAMv}9E@0D3eA-1&|aw)Ywd10FUr-0;Z&q!msG2{Z`R9> zJNVJ}&Qoi+Zd)va{Cv~ba}tVpAHb%Za@gu5;0K)}axJ`ne7kbybX5;*Z@cq8i|K)e z%|lPD19C9hkLQ>}iF@;@H&v|PJSe?^{ayLmf_B7rm8-{S_N7jfUItkH@Z1|$d}{&; ztk*9in@stfn=jvH<8ZP0c|uJ+=%3@1QGYuvp*K&z$ZB=i;U+%_^z&yfh+I&aJ-;O0 zjKB&Q_+3J%h{Wj~jV;OVSFtyHo89JyeYWlPPw_n`=+j5!lc;@G=DL3!YT!4&?XqM; zjz9@`JGYNO#P7XaA}Cb;k={7d`84r-DLNF$l9gopdwVyr^EB%jCozyKXV0Vl_TFjx zs$KizO)BqcWSI0Ko`N_Ko-qz*Fg1_TyDZRuf~MSg&)&kmd#qRyq1?P7y6<$b^X(M1 zb0;w5#dmSIgyhwMqvkcfVOfEX*@q|(EC-A84+8;hKwNUG+pvSmcR0Q$*{o)Uybj64 zAt*Y(4#vc_o_J$L0qK4!{O$&QIj>!E3>+S&*Kp#!+PtwUQo~>9x8RyfiMvHx^oN;= zj;5!x3VZ@Td6PRZon9_^uO2rJGFqIm)WHvbX^zE^SIt=7?qzi+bGc+(1HTjV8C;_< z>TjYHCyezUTJ+yyV!sDRN^TI!i@fVdBoGXyI>1gDEFm?<6u3+SzWTB3ICzjF?pMsf zQ@`wk5qO;wzaqkwt5zeq!C)eKLT`3@(b9%q>_yMWD{BOf+E2mtK5WJ6X)wa&hHT+^ zk3C+k_xd#)!$icNi+O=6_4+((bmq0T6}rQLfJU&mJVXiYYwTdipouV1+ouYNkkEL5 z-CYRw=^RU2y(|BhyDf%gu|>*V>7E^Ahl{kh%1`$Sa3W3+IhI|MCfQ6yvZh5pj82v_ zpKl2F8{4Z!u%_Ws6UIi-WHL``McH$vU8B0NLu+h?6A=|Hm1o?W5R0!>?!dJ}xwX%) z9iQxV`7}3IGo(`coX^2t`5f?ac2c13NVpKotauEOCz%5hs#WvvI3#-IASD&Jc5Fj~!{vxjS=~ zIt~AVl#Yeu>|-u_eAykN@mPc17=YlEf;2stxD%QaZm&&Zy>>S=@}R-n5gEQ%WVy?S zfR*E6agvt znsk7TIL9QCxK(=`@_nh2bolE+skIMjo`#5 zSh(rx$ke*d>n6`e^c@Wbv!}OQ*OujZ=6+zccXbD+=9@d1p<w8XfMq zFKvoffF35w*_NXpJeY;tR?gMjs9aZe7!i6eH}&cze(`We8*3Hf!fDW(Ohn6Dh-ITd zq4n9z2-F<$qof-6m)^hvJJeJGLm|6z*$NzsPp_w&O)=G_^=wtCQI$lGtlo6l_34ig z_R>u*VJigE=>t+3wW|rkC*`xV$4uGsN8G(AXwe%+!cH`xqDh-i*+Df+Dx8ZW&;$*b z0lkv8qV4WRo+(JLqA*EVCgPVLlG{-ONVJxX)3M07s< z=)ovxLE)%wKB&RHGl2LaDk>Q!;$R-&(8VIG&wWXG-75JzK0g7e%)N?dwy{ zN@qEE?`DaURwqA{y7G5E&^>?VonE75YbNe|i^7?Vy1lR%9r6QD7bA|Pc$cx-&6|hX zhXPr_v_I@48!hpDDx8>!p|33N(7|^Mb7$g2T!uK}v}KKTe5n}Kl}0;AwV(>{0mik| zaiM|<1&;X}x99HUo?RTtZ>0v{dAUtcvu&uc#-b_%-;vmSDZ(rHr^H;;?whN4= ze7axUIGQ^QWX`;3(Up&T_3c(wgi0o3golv>@*&ITgsnFa%H;79XPfZlcSX5KC#Qhi zgslAK5Sn@`c}lT)s@^O2CvHQGi2J{T^`lhK8KeR@y*pYPr_e&yP4?QN$Y%X;UOdl+ z2D7XK1RXL_CB3n6F%*nuGn}419VLQ<#X*6s;a!hyC$16g57NyyC8_fB$$H463b|JU zPJ~FXNK;tD1b~cGWFDI|wAhXBO1m+23_Dt^^gb27va|XO?qW)=?BF2-iSMw0OmR^i zEzB@haaD>_-M7w8zfLG%`oZuSsQ0u6_fb2Fx3bxCNeYQvkEt+4%8Bvc=AX+bH{a( z6vmU2yF%97I3cdvG)B4}1i)`rb^xP<8X385;cFO}E8@q$>eV6zi)j`||KS3V z`jvuGL~-X7l=zf{;UKGNbQFZJ11zQ}x)?8WeoY|T=u(J3gP;f6~d{vGm4ySUiW_u*f< zKT)ua#4a~^>QXQb-lOQbzlxPEZvWQ(iHd4=5QlJX=9(44O`RZ=aLpaNQg065K8RP; zPhb!}{|EUyo=Qt%JE7pz{Y9wKt3e#`Op{E!$j!5Gt%~y_)mnWXqU738<)f{4p*j2n zOr+Ox{gLdDA5SR118s%8?`*BI{SOJ?um0flRP4769UuM|j;v(Jm5~Y=kD9I2AGM>M zc4+joeOf%Bu&HI+58l!Uv3)iyE&zo@<+)2xa%S$vqUv$4w`ks{fubGPKNTvC%*wqq z*$`Av2-K_nt=bnXcZc@i5fQT%mv{5B@RFq{SH9YUzI2R!4d9ovO-n`)C6p#gYSV0; z2I14cl)lK}mCxk}Id9*oTs`?ez6!9I^mB&;gElUPFK!Bnxikk?pB}p&wCYG8b=~OB zARmeqLg2$?#K3x{WL>PSZze8XP?9o)q*C*M`Q#(^Ohra)5vo)a@Ii81&cr4< z595bEZGy^(KPI-{v%JMyw)x(&jXfSVV0Mn1*2a@|eA@-AVqV^YwK6*#3xlJ7OY9my zU|l8b4a$JV163=grX8a!@t~}+H^j~TQnn0NbG(tl&njr{O&p@??H6!8jt3aB8T>(| z<^#kl>w3UZ?B*u$Z!Cm6R)+1pc-?c<0lt0_>|&?@LNj3a-yt+|49=cZa)f~&A-(b^ zXYsNAfp2IKIOyHT`tfLnMLPUCAq@Ue8U?|}y~EznFm!dvKB@ikwA*<(xSeSjupOIuzCYNCsXTD;rTCcX(RyQQjO1h&vAtE&z1#Z@dWdmTG<=)Z zd^a)G>v((h>Gx_5acB)W|JaB!go@6+<=(2cS-f{LEs6*0pQppe>!o$Tn4qG&GEi zJtz5?yL#_eSUwpK3h?0_^lp1>T{**uM^jJ*#Y#9xbC)yYFlSF5ApvN0)gch^g*6tr zB-R4)!y65Djt{#{YKJAwRh7>B_HUeFa$^Ljp!h>%$XZJPbKnq0izjW%dl;FnII&+F zg%d-z_&|R+%8Xwz1^?34ZmIn{oSjhdk@Q z4Sv1JMA>S#>{@G5Z6m7W>Nbe~VEwyqap{B;Ok%Q^4Js^a?86n_MhoscOV6fi3xs!2m{sR}IA zAb>Z`QgtAexac-mt)Lql=4#BwB_v0`n z50>`H)8WnShiTxfFSg@U+-yvxy}*~=eBRbWQp3;qg#_4F&@&(`S5m)SB!YWoFXj(M z(D!9Fey3Z+Xp@F!A>s-rCksEJ$3&&NBExX9u5{A zKdVgrorf-%xNQutty4guu=41KFcy*;#Al8^H;6@$fMov?0y%q_@Qvw8q0wtQEDESd zK`TZ=P7ZMxc4b1>G2>kEqhT|465^ML#}z?rmdv!6^dkAQaROb{bUlKxUuk_+tG(f9 zBGG>AuWa{I*YF4VgtI=H{6TSPCVSp^`NZjN9X0u5he3a|xGSHu&Cs#0F>ou^nwFWI zv+Tp(bGAy|{mUId=Fn_?iSW8?--W`0>{GzlO)JxTuIA%wBMgmQk$&jW!!}E_2cWXM%+qBBEp0 zHC`qkbJo0*&`q6V|CHb5!K$3$Pv}VQU;?lU3!|xnfz^pEzL65pZ>$W?)Rr0F$Gth} z)2r~A%BfsFCsE3xFL#gnVV|d+cyo{F;-#X5oKQshj_=&)R?y@z2dzwO`czOTsoij{v zZbE2Y`F^v3!pQzf@57b;LNK4;4^$8I=Z$00IWC8pb>Q10n3 z%Fp7bwC+lgpAc?-7)~fg2%S$z?X6KA-ATCdz~?K9I!XA1e$GYc6o?;_QFgHso6nFtCKc&OAe*aN5j2K)ZgD&oIzKu=##4=crn>&)-5fof zAmRFjiji^uC2`Z}z=Pj9es#JsBbu5KVK-Ns*9i>AV63&)_)odvrEYgRg}sjILR)C7 zMCZubWHSmTd-IwqpN)150c_YWD-TcLeq3!6fHL%6C96d(al0($pJgNf-8T$W{4WCu27oHL%bGnJPBMobd71 z+2iDwb`nyA(3mZ?QpiT_#=^M00W3Nng=uVPgcjq*mAVzXWjl@^!4|ibc|uj`M6sX0 z_U6x0t@otE>$dD%x`dg2;7};!i2CJHkm(}7VzP}P*je@;Os4sA_jw!Pr}pH_t5r*s zXmNvuSL!<+Hhg#K_P>M2uxH#0L%H^N;-<~du^-1|Ayq6jIMZVABhXNI!}E-NUvroB z+zGt}h}_hB4)ZLY()3Lc2XMCc(3|}+V9aGn{VW*HtHI)Wt>=Pg#0F{d*EsDsXfe7F za;bxiY>_?(3xh+h{%ibp9Js+2pa5fXff24n=vcTYd4mmDwz%&%j&02t?t$dthq*E& z-3Oj*vI+pKqBVoyWKIkvTly^&=@-9q6OLLvvAj^y&ICV<$$SxOC6wl3urcYiO=hsS zjc+#8;5Ct6O?_wzUgK}HcVTuHv!k_cC2*R)_dl=(n!mhjgbjXmk_)B1Fx!*Qje=v@ zpnv>9siMS_5e((ei$m=Hz-V(|>EPMYcDuh**(9v7439b7qYzKiwliqv-}Zu>xk z8!xYf@Q!@17#(B0^wfTkd`@Kb*@(jCd2K6aQ?4+3$v$3HDR*8D%4K-ki_29aLzPI3 zck?-2e9A5h+o9K+)D?}q+O5QkXn_V*D9=r+tXR7qh@D4VyyTG`8Qq8y00o0UG4+N( zP8~TGYGe;Ig)*d+C-I3Po|HowCGXae5{oFoJhal#UtLnM)^j2kL8x3m$ ztl{TX4XzbUmPF1-N1-X{%GUi21NxaUmn;E)l>CFt0viW5am7@Fhv`;xk}Yz}V=)Q- zo)ll)7BT^$eG?O?YLQypm#g<1Q|$Yr#AQ4KKIQ7R-E)O7SC2v7C6@C*MOIp^$nyDc z8-=~HVF)GiUr}2~f?roNnF40f5O{h|mY7yj+pV<>ZIrC45iX^8({B=c(+$+)B3i<+HQKJ-g&!L45ynSV-$_6 zKNJsg1BiS75qL2Dh^RK5JbpNWEeQO z+=;g7J@fP=#uFw-Q9Ds_=ysts?Bb3Scdo~hc_X{;jy#`CVRmtb{=N~uTC*;(_|2Fu z2P3g!S#|*ZVXjf1eF6#?%3Xo&xsf~mVc-)c8CWSHR1d4V%iC6HsUL#$3JZEhnw^90 z+B1x{nz-lW#I+u#X)82fY1m*gZt!PVLwr~nTD=n0oZaQy29D%-#PMJ;&(~08|g8nwod`4>z(Zfk5 zOZL;!c%0hx+V~09lHEb{E@y|a<#rCjW)DLhveg;as9u>#3VW%iM|h7zv@Q8~h=~DB z>2e=GtjaG#NmMC6h~;QK!7M5{?WJ=sHg;xkpOcijDB^LNO6^}H$aIijA<@9pbq$AM z;n7J*x0YgYg!jZV=#UAGYcm_;5eNoYf!#`9i z!ZoZXRWCHl0dv|#r%+I`C%{Wh9wZd}k$hEYtNGN-%@YwRJ1m%xo)RRqbRc0SmS7PQmU1o!KSWBlX4o(y)|cpn=spW(Cl4B8>E^VW@Gzy<(EXMl-6P!(3JxFJ{-yw%LM>!FA=Z=5$z?N@$)2 z6%bgzu0$s!^!?aHil0Yly;jhN+(!0Mz!ZK+uAS+_lK5n~xRssUdBA~cy8ESsv{MJx zA&@wjKb+xL5I;|ZR8Cf7w0!{VX2MB7v`wO zcnFvo%g=Crepnr_`az$4)!>jh1VJIa)X}iYN)R{Bp%u4wNm1fKSZrxiJ{s5U*~lc- z6l!|Rx#JEiUV;IFAQ3WgSXSrdeJy=4!i4j?4N<-N4s;1#rflS>naE{krvalwYvko` z{kFqnH5dS%2?Y=N=bRe6a@9dImO>E4sYhP%>+Y&xKI8C(k@Ki3C2&NkG0neTZ!DE2#=rB$s_)AR0w!|5(@|wS>GxkYG3rZi zU2vS#thKQT2(i=%AwB6W$4(~)=6e9&iGF?d>S`Oq_j<$O>B+&D#bk2S#h&aSzdD={ zD-lC^T<7c7ozDuGoP(Nup&nbGXz3*C8;8B}DYp8}k{WjT>2)7+rOt$$HyAFU{CgT6 zU0Gcl96cRUxF8H!y8?JOW4GnO&oVce#9w>#CQq+fcCBJFqyljKfrFzp?7CDe5vr~J zj4Zr&ea-$5ts87HgSZIO?Y zheg~FF^h`-&40d?009_~Te#=9zP(d2==?zFPCyX3aIG^vG^o7uONyW-rb@d!1lT=o z;|>80La9lxa=&C(1?lUM_|Nu%*tP%VU;UF0+lyVHMl|T#+OVD}Ra@iE>`bHTLZR6z zXL;dxLVfW4cEUpJC}ufFH;B6#w|FBk*=_!{!Z9eV~xHs&I=2n4DBCogiWA1z$ z+$Xg!BBW9=4TVnM@(M<THhmcKXT($JN^7x*W9E+$)ID5cD6%tLL$QX#RJEXUnRMk)k zd~NBL*rQA67$fy0KXpZP_eJUQqM{doiAy?p!6_F8AE5=5_v4_;nrMA8DwfW#I$Ea( zGgL<4enBjXwVzCNOe|9^!v4El{A*1B$t=G2cGdbF9)MafNQ64eDFPN2mdB5+nJ%l}*Q~6gf zZveSDw$}hb$~m_9)Tg%q(A)Ir%?rsnw*TGJU?C?Qe)KSf@?wx&&sS+Cf=DzFsVgBx znjj-q=D6AYn%v>`_23gM=!r{ChW%gUB{d@)fmYP=;VEXGV(doYVPzZwOKu0kVd_c* zP-N0T{jYXV;30yuuFNo1lc}}-{rSug3%jKkg2Ml%S1T1h>6e;0>i70lY%+V=3kbxM z&hqH>n^)te=AnfogvNCpqYRl3r|Pi7bsY>>u}&8}C^^^|g5_vO{+D13koyl^TZ!0O z01@}V4>)|&qS|$0-^t5{o;!vtj(f;#>Bwv?O?<>PA=BUDdA-J}4|V6OC1?wW*!e(l zJgPX;tU(6zUvYN|0cE89*Bq3H+wU1EPErY_PdsqJbjpiVxq*2e(3%|beZR23Lzq;} zldNY>4)H_I{B3Qw&}jbZO!(`b@ros5BYhjMzV4u@8nPq*4tgx#O4ICPl}BmkbTJ#Wuii@D>>~Jj$_hrARBZxsTw8X z@1rOnc_i&wlD4+SM##jBH)ZQofK+Hr>}TtN1U>jGEiFyU$WYPIk+HU>Hs43^3m~ky zcF{8cwVRA={DIfHG{u}f6%@1Y~>ISl;>o zgsrMTW{RCkj>BsK|D(AF8;8z!oW5r7xU2@0d_&QH?*ZtdP)71t45&~{%~}c1kql7q zsLw3E&bk^vBAC<}HQqldXR5PAe1`81B^HffG8;jPfWz*atBo_?-LMnruHsBoSQ%vA ze+k^X&*<{<-qF#+D@*RURdwioM-G^z|4Sa70_Lr3IAJLQpXpjUxI|`hpb}-t+1b+{ znMf_mjS{M5PHaZ))*EpDXJnF2Jp9mC?uBmjg0)lR70!H>eb{Q-i!+XMq`;91J}%sq zZ=F`~gPLS0aA|hBg9wchw~O>>9eKJiQN}^|GcPm(`9@o4=3Ywwr4RxTjZB279KYxqaq~6K6Zvd1pB*6L=p3e|+_`yYGg@aqhOB zTQXEeq1B|BmSRdQ3!}i?uXt>Gbya%Bs?Az)f1&L=g}UXhl(m#YLx<57?e^zzXubHq z=_@XONF%FC*W9N+ayVlH`Kv1Ff-(i0p;ObKeLtp~|85Zf^@O8=nmFgQ$*m7eq(XmT zJPgo}Se?z6yb zQm7+A-(n9UKgQ$ENUBlJ*8ELP{(^T*H2x65^6S9UUPz;~@VCpWi?eb4MRU{!D6$`Q zp3A=9qj2lD7BzH-doVnk7iVlvzJs-rP{l-nfoC-&cYQJbDg`xazOK)(dw@&q98hs@Ovwd@oXifT3fG4}(E0AE`T%>u&`Op8)< zne1){OC*BWNunphumqe+lX8(HirX^a?CTTDJxW|A^_ii7*c>_V|2jtTU!?fg&0Z5@ zV`BPD>Mt*5Ngunz)@;4o)VVmiOWEgaeu3xtHB*wc6a{m4F(L}2$8`w&ik?Vj7Q?bG zX&zwy6Va2jA5F$TCYMU}ynaRX;i5C@;2`D%{8y;&|IDQ}h@h9b*CI z;|BxCQEvMvSXL(=_fvyznp`quea5~0tPp=fx} zNg3-gRE-&^>iBED$wGM4+X&aNG*Dq&V=_LG_hw?>(atwV3tjM6)_IfhYRZEv7aIcm z4M=`_kWhF}sEUvE%=Y^FdSrau^zD_?d*@DZ*${h+H<(64 z{*5BJ?6^Ta9#kLj{2S8pyx?*Lswm@x-qKy! z5&Ea>qum(mTHuFSo^C*p>AVv19D!GedSoXXjQ2{H2Zv(k`)ArEN6>Mm#LiCoiJW&9 z#-1yTj$8ny9uL3E&w(P}%QbH2-J&k&mD73sy4h|nXW19818!MJiy1Fv+~4qo zSFsBj;;wGD){whb^A;V?;d+lp=XFh-=jHHKL17AKDI$l!H{wLHiB#@yUgW=3BylV9 z)~w`#4CotW&YLjpqeQ_2U!F8FVjDOdJ{b?76B|z0?(DH5Mb~U0BXBz1kTWg{P7BFx z`_qABaiyw$O2pERK1oLrpc4dECAe*PGv2;i5qZ4H2Y?7T{EXqaAz4lO$r%FYQRYUw zZggl+v(`bqd%;g5)pEh?(LaoUdx z^UE;W<6LBkf`%_r2O-QAbotzwufiXay0n5y{zlLj-Ob;cs?HC72UkOJ{s}^=>Gxg! zc@KNvUJxJeYTzzTY{d@Oxy913bIUekGJZ5x00(Ya>)yI;7hpl36U#u_Y>!6if*#P{ zje-gm*=lhZ!4BYOWcT=*@*mgo-{J>M!RW2{hWeX1z=_3B-Z$s%{F9}mWJds|@G-}| zCQVR&-x)3z-#g3?2;OuDMw{H&GvCT6$GHrs*u1%`HX7&fg7(QiW(i%2;z+o1Q(D+~ zHNvmF98M+fx)F%&Ne9f4TFqTiNiYXt7;qQsG5t9$D!cHrVufyZdiTpUMT#xagyobq z!wY(uEg6R!I$^XU50Bb+8?tI-`V)_Z)zF-Otcpl^n(@IHdm@>!0hlIp9ms|3R|^fz z%T<4rADLmA(Zl`j{^0NJ?IoqAineaNW&H$Ys2C3CQ@Hng@`O0__+bIA`gK=#Rozv@ zjDago@N}Lz@VH!mtC5p~ zZ|?XF**-SL_;vWJ-t(+Rjn5mh4qm9S%KV?(#k>85VKe7};W+4d^*Ue}bDgArZo%bt zJ;+aArL24?a7vmSTuZehK0N5IbGyZ{oA^U$-UmuU?QpAX#Pp4+ByqF=+`9aZ66cWI zf7*$M5`C7D7Z#3O83Ye{(PzL`9i9l->#QuDn(j|HHhjmzGCj9(zd&hcvD=-%U>p5+ z3Snob5zJ2(QaySI_0F-P{M4JF1lbP9O%5j~y1n%0c1M)j!$9e<+bi!)1>}N=o#KKk z0{gZ3n5--uCnu+)`A5G6_1CYVE0IEcemhpcpGN-t$#?I5)l}CE|95eNQ!4Htg@s)G z8eQ~={y98182&V7%H~%V$*S)oBKteXxNkROw&C&%QX=;h<(V(rrRxlNeHPqUBF2@7 z`+`49IOx8dmQ^;6qnE`rJz`?)EW=cXFT_gc3)rOmu|%#3E!+w$hIHzI8VeQ{B_0HK zGfG0DCnl6*xr=PnLDarAqwz(Y< zmwH?c(pzwES=HO(?3-_P413g#2v@>b7F2+}>2(AWP?0M^dGJ2H392M3SJImw-<5i9 z#hklfGRMk^DX7wG>vyH|qx1LZ`*{;eazTRoc<6XPy*0jz#9$A%%+ZmEVmwi{{i0~Q zvzZvT99#0&q;o8fXSrHczrGe?7yg(>*Pf8+vj2>v( zS17#uUH^Xot3>>&u%Mx3`=Oyrb5HC6UwtI>T4~9tA_C>~1efzpbocfn4Gl}jT z9lv`LroudcW5_xD2<9}Bv<5~sSL5*Ov>QCPSfWh~2~i>O`kbXt%MC$cJ=gBI{&)c{ z)Y_v5k$GPV?~i=ig9eL1EU)5HXQE&^X`7-@v5LMxR%Va;TU!c<(*oLd1{ z;fov@4Id`^@sEIEe64|unBe2hOD4MZB2cwh=$56sL0CRL!`+8c{GAy?-3ZnIB-kA; zrx1oN)l<<*6Q+*~>*EY8*{RN41pM~gLw?=fnhfkCbksb9XO|8A5{Fx2dEnxVU2wPM zjC$3Zn3&$?gxwr{5?yKnnhVg3Xnnosc^(k{9eGgpr9IW`c2PX1AR(7p#BxJuBcnA+ zvD^D6w;%IQ58|@Oa8mmBobGDPu`cPbgd~~7_hO6-pQP~G)0lYXJl5HIW%_!4uM~QA zv}tZuh^q$oYn%tuJ5#4N%o`5Pbv?eU8z`wbZO^lsG0RkpK(Xsz{@dJZ?~8F7x_8_) zr)*jLF4i$_{chU7M1n^i%^#JYw_nem0XbCJ;babZz16M|8{*2z6qpr92 zz;=ucUR2QU(5)OgRiKMzYUl2Vi>GuSZ%_LJUwQ))3*Nu}kMrBV#Fe8L7{8pn(q}|; zx+`%m&5DqD@?}K+;gNL9uhw{crkQ+ieKgy(;ih|U2QxSV0EVm^HuLFOaT4Wf*^e^; zuY^!$f<0k!57 zR&VN~WGpDdV_58rzAS`P1rEw!(6L8_MN1Uk0uEv?X%NtGc*Eyc>19R}oZ(|gOCg|( zO&(H&SRrX^zk}au6#@$F))0Aj-Z1MwcnJbvh;K@E7oMk$m2L3GK_zaQoJLNdJmc*E z#=H%jQ0lyH-HhnFVxWlI4CbdLDi@Ixa{%r(jxH@bLTyE2h2YWglrBdrDMy3tWw+&d zZOe^L_!p5rCA{|#X1ubnWa#lh(D_q4~( zHyJ(nLsSJ#mmk5(pMC7iE-Xr_Z})PeeQip|HG-1xg^D(4!LR1Vd*kT|4VXXEdv5h> zdKlTlk0NP1?p67O?XzU+5l=YMwc4^hU)MY$Hx6QB@biizb}9!i%659Tt&R-#OXp`g zF9%ITZirPX;4h74+0fC~ZwQFniVI33WR=77Jcm%Fa7+pm!E(2Pt|bEu05*d+U+Qe&U7gz z*?-N-f4vr)rG&^gYjD zJ2^8Q03x`nhrXjFK}=z9c1*g+jVv4;eq7p}P|@c~MiPHljY0DZ5~{PetUQ8P-L4ifG(08%2*yF)RC@0+EG;h{M!mi#YlTK3*C z<|PAHNA~mtyj;=WHrlXMM7Mp+qJQMOX7Y(luJ?aX(ZA0aMMO&*KgascQMZa%HNI)f zuqkDTBUv&#zs}mvu!ku_BFR|DRhfVF43!gWmfXVVUIYZ%?*9}d&en@#9G6m5gklXO zs5`Py(I4mq6`+@swI`&b41|;hQ(t?;*Gq-Q`r$B$f{qHN{8&D-H=_>)*0?e+jB z?rJ-!+-0mauIWGa^151JrN)s`=1)_-BuiFM4h0~3U8i1Ta#bAYrg(w3n@x|67ou=buuR<{Q$=lv-K9=8(r9i+QVt?8gpP9GFcq)r<+vU z7vgw@@<@lF_4ay)f$E1qU;}n+$8%e;%&lPRcgK$kkVhl|BrQ9(czz~)s4P%(>h%}1 zs2rTih`&$QQxqJqYd>$nZ*WK+b_~+fOZfK}Eu{ ze^Y^RG#?}46jo3HHTH8&9a-U*b2}rG!;;9XY^kq$Y2aMM&(F_!e|7@9`NIpGkYdze z9(-=bIHZbCiyxV#r+4VNCHydBlxp6X>Geu_x0a;4f|-j-Cut|AXv-X;#jJaZR5_G> z#;x3_<#?)yUm5T{XG!#rbn{3_s0;h~@xjeK!FaeWNc{E!RSkFB&5OX|{js^eo=b*x zCr17{W_UEQ7{oU4@#YgNiE+zeA2Vk#g%+PX*$nG~eUHrr50`&K8v~#CC!qcA%YxYi zBn-^1P6^lv#m9-HSJxAH?DWaK$gu0q!D?-(wG+ybD&5ojC1tHoiY3g2;g)&g^cO8% z8e#fRW(55{%$Wq1C$fun6Y9@2=45GpUtrd>Da;?yf3`FJoPUAaruA}oFS7o{|B2-u z4G=8-IzDYu{E&Zn)pT~CUBclc9&IocX76!W!zeRv!5})sbXCIdq*{7@wmod|%N1)P z8@c&x9#>;`} z_4A2oa>El*{u?U`gH~b0r7teKlMIcm7|U+8y1s$VW9syt&R9XYyF9@eqZOzTzy+mR>@4DHJp{V$M%Eky930CE1zpw~ zS$b`oY4@*^(CYgNhc-gKG%RD0UTsGM9gl5$^!A-OW*>7ql7Y{DGH#SL+3 zU-b-8D~=(I)QPkxx?)w7&6%G|&r~_S^ih+1uSx4}Sc6T+W69`nnT)dH*r3qlOn#Hin2!%ytdbZ`fndQQR*PT7skdJ(8`u7#{9Yue zajLk+O9L*kJ@4vUA^31Ftq-d= zW-&{Igyq->BC$mC@iiy>)xG+uFhR1c9a_SALR{y5c8 z^gl^;*8fFje^*gL=kDSI=rVW@U`-Mn<}nvRpR^Pls0 zX}jFNpN=#3%sJeG<{hM*T7MAbXWs93T^^r)0-{DNfnxJbyR(r; zLm|?Ebf)i#X|j)ezCXVD92?raypT5GIx!Z=f&rBxxWRtI);pm}f8=eS{j%`jWLb{xu3avE2F(}jZWbz>R^?Gyka7-1iU||T`p6H(ju}h` z&2xY%boyaMxuVZh%F?;fpxu74+YDpFivBXw8F?b4>>I>f0OqL=}pA@`gWG{iuKrCpVn#oIxA@2WAN+Bw_-9l z6WOv(WFRpkM-CQlCIVY~r|51 zS!6_XNe_=7zzi&Cc+w^vIXhD970?zID%w72qrYg3h&W;-3TUumXQye*3D|k##ctWs zEm-+J&^>h_p1qz46U1W4=yZl_Wo2dI&ec(JeepUzgd9>;8czyJ4ZkbXOGDCez2OK1N&O9Xv#i7GyE0w;gU=si7u-<V9`?Wz9Wz8{*IM5U%PfUkia~UioRdm`dbHn$43@O*;yigW)@ra zZ`dsg5_tEhaz4-ZMNZx{j1nXyV+CvH9gt#Te>|bgG_u`k!8MAWdcG!iH;k7G`|f`} zjcr7{J(O&dU~x)`izq=?$1qwv5$dMJj90l3-Qko@8(X7EPWMr+wcW7Fx%>&Pb(ZA@ zW&CqIN9!9nrrs%moLrY!>Y(1tamV$aa>#*imPWH)1eeJK(!8?19>@?RF1NFhXCXzD zg&N&4U_>^=w<2JSqnF3a$xou@)rpNx@8gY5?oUbh@ZgshFCemjtmP@CfTl~0N=w7| zI}IHceE9JFx^sX0_=nMpfMXXVOw0(m(tix%pRRVznIR!yz&p*Yk7tUH5VJ;X*pLws z1GmUSLqq@Nobo@-lZVv2LB_weXxH0Z3$>f={?_?FZTdAa$PU^KT?t#s z!a&kSP)&)Xbhn&K!mv0jDFrj+^f|7)?y_{in2ePaJSuSZru`g;s)npfm5r@1ESLJI z{9t5TxP>PxD1OPToJAT^JY2v*PcevN44RUvwm6WGCMzjAMFi_vQm{6$FNlgrN%=<{ zOW`)Z;GniHV0%YvN4pr=cE;HYFY*Rn(1?oidb=!#>H*gYDKSAzC?sBAWJF%n5|LEZ z6~~-}uMqr?qM08mPx9V5Jz-H7TEgmg_Q!{bztayC+u zEQp6G=TuXiT9gPIO_|hme~cy=o%Kqrm=EM{e+C=CK|J6Krv4Q4iU#`8UFjE43F3%9 zD4!`rOj-y!C8gs-E9&a> z;`6%{erOYZBc~9=uaV+*YtP;ReWpo!{wmFW;5=YAY*(At1_+N7aOk4k;E8*VTC8tg zie1-}n*Pw;JH#*G8<4MXzu1p*H*a{(Lk|5+`W_pcnf?yId1&RyN|!?GzU5=13XJ=g zFMxDY3ep!JRZky|cjYFWntk-$TEo}K$uu^5xX{63p!39oz1FCCuHU#Hc{hBZ*XZaO z@V7H>_N(5z@~UOqVY_DI=GF3RY9Ouns*?-!r8f(Am^8ZG;V%UI-(6CF`2b!!9#&3m z=t~uSx3xBMuL-?zvf>qN!L1KujW-U6f}jt>3~<}K(}%H@ORlz8gyarNN$gRXH}oSr z>AncEbE}VUPt-OQmQi)LDA0mus@FXSBwl(%roDc^qFuTH*WZt*xq#FT(_9AE6K|hi zUIdY8Qthy(>&= zzML_&O(@7py_Xy#LCcr*HgSoc-{kkds#PzGB(sk&qOS7ozNhOe?*QX-NCVY+C04J) zJdX!9ApIYso5M{z;jvdku8$3K=y`$DNWJ&|p4ozwK4ZfhqPbT?tIiZ9l9~Ct&*vdg zk}M0qJTFu>UMyJQN;MIIL=_quV3aolN_jMTW(^t zk5xW75xQDaE>GPCv)|lIychiMNbG621{~9)yHC~Vj*`Rh0=LXz7V6FWofhyS4o;}W zrg)^}L2GXrHW>UGqYpx<4qmPJ@Fg^azf{jA~n?!zdWyRrBk<-bDsj%fP$H^AO`NRg9cVny36R-_G zaGCf$J(AFC%B~kjM+X0Q9-|ZS(FN7RVJ~s#5m)!KW{tQSP_I@qn(Xc!&L%FE&?nm6 zAY)r^ujMJ|dZz8HFldR*)ftv%cTLPZZ*G;^T(WiP?UZP?KMZ8tvq+q=pYxq|ykvIrH{pqgz5sNcJ&%h|pRH!!dECwocB;9+ zf^vKHY|GO)<>0`}Yp8n9_bK2I9sexRaAlN%@z{;((D^_gMVlGz zAVJRFW(#BoJDJlGf-`7I{%AC0v*09C(3UJO%=s2}rJ)k5ocu-3o3n!To3;vuRxI>$ zSw#hrMI<5P=y*9rpw?M2d=Lx6_spyi)!O{PVb!29`@RD8^6F}F^kfO8qzGoTs?w^y z2tKL^ji^{Mstqj&RadMXqr&U4lWd&ws{Dx1 zqTd`q+sdcyMg6;q=dOemUZ|opap$ts?{wslGG`8l1uumugVj4Wfx}v5NCw05eKf!7TLyEW~tcJ&c0-w6RhU8{d!a$ljob%LT2n;j;*tDIMc2 zSXge_q#R%Rf5b!$mF=5g(TXJ^cBY8orQ)jhNEAu-s_M#1fGc~j)J?}9HTn$eD+jJ0 z=f@L*WG+kFPRZCr!xhHPGKG{TWMcwWY(wXCl0wtT1nq%w66Jgdr;cJohRMDg&dgBk z4Ih^k3?T~?vLoSU3WP{X6SD@aZvS1~p^wJ52L_iIj#&dDC(rXntC3QX!sE$$xgZ`g zzBAUqQLCY5GT2V_!Rn?*onc-QBQ>>tx9oVK*VSYcrZLwfpc^`Ub<7Y;ZqPiZE=Z(T4*0xRSIBbjv85s@e!{a zHeo54of=WMEgdx`=&LAhY6}Y}=(UI-VIzYMMw_yj#*WULogvt(1O~Y^{?U}rL0tyt zqyn^b)E)UWa#ZL%p)h3F4v>oELHyHMK452|fSvt*r=omZMoz>CRjDVOR}%@X{u442 zHEuSL9n4D@<$8h$QdM2?_#v;1t4gtMsbJjV*^%xQ{vk>0I4ko{TjQx_TRDEJdzA24 z`t72MfxDN55kZY7GYyyA_kxY%0ZnCjNH{WUmO4hnBi8SC3ks_H1Lz&iu$qK5wgj*DaM?cabfjXW9MQAql;-Xsa3-lP6x<=mJyy* z;Bvf+?a&gA;xw`_hz%v8SoWM(4X7T97tSMMrivtKLX>%|C#!h6{tHcux}p8mgmk26 zg*(HVilK`bl0_x4RL$s9-w{)zCyq)xW&#)Sg{*&n$Ho>;6C8_E6y}S(X;QGYmfbuk zrqO22#a*|{50)KUF01B>E|e<{6vLKNl7jsvmj_l>Q_**RU0BeMg&RdX6O9J0FUzV5 zzzV`CW=F(DmM+qiw>3o^e4ygF=Q|URtb5u8PA{=SW&@TZpv@*FvU@{T)s*#X?955T z56K?0=~)3dFGm&Ce61aZ=TV1+Sqh>W;OP&(w>A>57omJE3AKI@v3Cobj`*b*U8D6> zt)44PAZp9WX8QGE=(C zW8vL$ZNY3A%anyH2JAQ3Z@8re(Xq_7Y zy#`w%Sk>^Iy#c*Fk|=O7EL^<6@B_hLk+^jGjO9(8E^{xo_kWmNWG}@sfNcU5;E@Fc z9K6->M6(;Q6eucRUM@U12Y#;L{U){RYU>fkJ?BP zXfX$^RI??j@fXh!#pH}Y$~&7fL!+AzH)b(nz=Vpk6@1j=3@N92TE|(ckJqXcB2bV& z7KA{|{>J1dgI=Wtr!X1lYOUGihfortamOmofixXTE%rdnx)OloQtt==X~K0xO8G9F z)*IRDkKz7BMOE;+eW_zhn9sFo-<}z;B#g%O?NI(l3ia&D=lbLu_CB|VlLHo zr%`=z*1Qic+5)?Oqu8F=xHJNAt~3n>;$e-`0^;QrgCjqQrpQv(d}+47LqGCZJWGS) z;|KTkQ0?r1P6o>#ygWWjMo~c(RrT*+if&UN07ePgmLy~%gFwK1QEk$ilh#!m)B3AD zD$kD^!Wh|_apS`XNIfexQq+SASPj@hYK`s}kU%h|tbJQLsi0`R zn5y)xt6si@Zu<0k!=Tvt9e62ODkZb$9;PlZ819g8k)CKgUe8uN{4VuX!VxUYiM9(q zN;b@M=x0-_)eq*c;WQMG(Sqk~YbIaW+Mar`*J!gt9}3}sWhF%kci)Lre3y)3yDCIM zRRvd7R6xOMm2P74J^9wA^MU=XSzcgFal@#~>r$6$Rj+?O+xppL1ww`Vfq3;`a91(! z0Mav~Nd}AiN7WO90xAv!JA?l|T@V(t*SvT(dRuH;8l5GZ3l=a03UFXlfpGi&=ARc- z5yTp3XRB&fJ=VX@o9vJ=mwdkei$7?s%EZkUR8>_0rEI-9B$a=;H?;XJU2h`KVqd%C zBPOtBT`T|b<;u(2fh+HIFmK+sriR+wxbL_WrcPU@g9J$ga^>8sgZ7PCRa$==(&u3MU_0&jl2L#1z{wkdl^i?PZsLJ`8 znA3HaAxwW5HQ50Q*UrgZ46rL%;kDdP#Z^=gnM=k8xK7mUj%H!6>g?wiZTKvP?V z!|q%I<6t>MzOR%RGJ0_SGQ-2Cp)7XJwGry_N(zYT6y*xj^@QTp8Y~dM`V6e7D65Dp zpnwEGMOFksR}6_A<|B@$HAMH}sm~Hc7phc9LsLGGPL-lUIYJfk!v0$)HRXI= zfSvq>z?i|LHu3f|C`mLe@m=B=>==!ayiX!^8 zBgtlS@MOaJI&{$Ri&d;jJe$WC-jxFb=3uNV2M&K!X*LvNhD^qz2@%rfV);k4G~whk zt=1T!W|&;0q8Y`&#t8htlR!b31XXMcLgL*K3(JXQJw= zt=g5z9aZe|3P{7~sond<0t;;>4-xm7oqi}aT%!$$O9hj2!QmDyD_^n`DR)E(+=0Qu zZ;2+N4i^fl$+^ODTM%>KAA)E-svS+tAA9GXdClmxCPdH0knB|`=$CriW+-YdrlGS_ zkS!u2Wk{%I27~nZBBzJ4V)pRgjP!6+zL=%**DL0*hI9?wm+%R3u& zGfDdnT;EWFdI^GNXEQ8knNxF~@oxvmk8!BZ#JbnIibSzFhFaYj_33Ey##9%7u9Bez zS127;2|$5Eg{goDvVXOf7DKQOR#l&T(X6d^dH2#PjVVem&pD^=GAdXP5kn|n+oCJY5 zxG)f~NYIEtrEM6Bii%J#L9YQXgc^-mZ!belyAuoL^tY3-1*ZG`)~a7(j2+5Ef}|py z%{f=&cCg1&u)%0Rno5yi!4*}K6(}SaFhu$l4JSs4T0lS3Lsz<<}cYm2g7dfd4D6+jVBMd!p#`qX7@|4Fh)f3b&)Z0>eFrJiH5 zQ=$Dd9#d&S*MHP{E08Qz9GQ5CS%#M%5v+2#0Y zxA4AWzQocTytCx^v4`LnWzRP~N^Yk={Jhu&C|m`mEk5MyZ}U87@Qh5oSQR&|7i(0? zS1MPij!-LCp&DSQ)&i|ktX3&gsTdJ$qyna{vMLUfsMh-DFY_GST!Q8PxcEB!anZOt z`SkvQueqLyuFh!Q1OdX{f!sf=K5&)px}Y>6x7UTyI+h_>H+ z1cAeU0>k&)BpOYY1jNNCSF3Qee`;QQELhXL_ms9+-rJlo4VGt5UPC-?KxAk8c&Jh* zYr!R16$^!qQLzSM%}a$!iB`xl@@SbZu(JPh^n(E3x~my(ecU+IysFbT8C%gi3A~;a zPp?qpX#BLicpZYTc?r!myA^;KtlM3?4gZXVz_;F4cCRY!97j;EW~$bzRw#Fn6g~t| zyW<~Q=+&*#pAyv84ZMYLt2@+TX4mD7>Kh=S3Wv|yI#}hKQ0M1n z-Nrs15ybm=nN?X>1CVbtCwYl^+y}OMo^6mHzpvSjThTd#^EX*FN>!^BDwR8c`$JHv z1ZL!0Z-98=y|7JQ`0}&Ev49}k_#beJX>E$jOMi{mY7{C4Nw$Iu0TK=TpcNAVLyXd{ z?({koI;YFgTj>7D7sva`73H-lFWo35`%A5uNi|=oI{%Q z6ruj@Aa~oGzb%Cfa8>(* zt7gkSR|=T#OM$`xoFCCD8CrEvE9Ryu`R?#>x$T_c{nVRMZD_gm9+eC$b$VGttVj(A zC6zMOqGEqacDpQDWkf*dJGB1GO`^?ea~gKa zeV^#XL%;#(7+3j%h_wneDpfLwLcUgI;NR(84xFO%#KbQCFa2v=g@Fy+Cr^vruKe;k zuFU+e#|Vs5N`|Y%?zi{DxyJVedT1_eSN&Pbw^ZR$%uiDVp1&tD`rEo7Jgg7l<*IW8 z)uE`8sTwmhDwTl5sa4D+X;ljqGyVCaJ9$&8zcIR#`wq)abDj^Dwfsn;{|vNXF)R;F zs81Tbou{@Q=2wrmei-{W^z)oVsseIwOErLiOFq(V%h|teqtM=-I^Ep6AlEm!ylJiK zI3Ql*@WU4e$ky-o;~N@!gDPy>$o>{<)XI1IVAyOs)ceNGcH25>VE~D^ft_{tY58!44=a@T+08#Jbp>V>{pihdDR|w^AyDDb5a3cvqdget+rcR(1YsJ zt!0|fthXXLmL)rWVP!P`%BJYOi|B@j31Q-N#)F&PkTyFn_GCE3;p!2|ik&($kZb$d z|4E&Y+jtE25H!7AI%<Vs%M11K{e4A$f??g*|J3>Vi0$QmLrY98ntmUNN%lzX}CGS(ybc z=wemr@!i*^3r;!NbdI{r~1&ua3jG0OVuC!G;jVUj6x@Vkv`{RM^ z9@`U83wAnk>vIhK{l+M-5El5HeZ8RMt$F{d`+xmiMj1%h|JE}7EoI#Yz5C@kZst@G z@1?JMfAe!A5eKhZK$WxM6%{_Iz143WIzviOSXrA|7S2?}=EC&34fT#CnZeY{3{X_; zktR$|#o-v8n(G-HQLJ89)!B1&#n7Mp?VA@Cr;`J@IRM}IboZ5lcD3iG)fOe$NS7Nl z3Vmi8mWqmqfa-4{qN0GLBAQiE0rN74`{gyQBLz;u*34X485i*$j#)F$;cY_B{TSQu zl2LhVhmx!`{G}4@4y533Au4b%TP=JW-hA_aTKv4xzYaB@yb9ERBW;3ZDcBj2!}EUm z&WpV`qg=tj#1jpt>ykWKXMaS3oM;@4gKzBQlC*sdQ=}qpt(M11O&6tw1$~hi=u!n_-v7>-LvEw86&9I>h}%ggW}-lY&}`W)$R7dF3=noGCAf< zM^M?W^U8Gj1{fPB{;#9z|8flk1Sst(&i!pyD=_wqO35e7x*c8iL9UBc`Naf46sQO& zsF3`2r!%}CeP%%^R#+6-%+dqoa11^Tpiuoq%+#L@YqeQDBpBK8O!laQ6g9PK9Z_dV zB?~h^fgKr{czOmBA{dZi`$U3@iWJE&tycX+8PG=M+2FUYN9mlYrnhE_&@9ca%*JNC z#D(Z(lz&;HwST8w>%=iZqsZD z5C|?%I>Fe=Pz60(zb6x)lymZ%tUUV_bIMjzQe6`uWNZyH5d%yj4d>$0tc;0k=HZd1 zj^7@xD_qLq^@5XJK|z5Y0bFPUA{&uAu8-Wv4)rD~z|T=op-B&D^o0F)J@uCSFi=s= zXm&H;oGlb+ntmDd$_!1OAME}5zs#)vqs>vELlyJYY%rw02d{GSDjGZ8QP|^RnSP9# z{$32X1cnu3o7z~HGj&PZ-gfSIdE!08;~K1+Jxr@WJxc2ins7e#xalhO_&5&dTx<+G zT^uh&i628zQ2?fbm~dWEEWga|7Z8TYhFPd{QDaujXhWY`^a;Fb+sInD1Qn0}?nwIt3ovIPDutfR7DQ8@(x@PSrBAGZ59e@@$EBq_qw!ys2|M z^kHA8(~52%#3KIT0qx`zCJ9T+@{*2`k7xWC8N%Ztz7%7{JrbGNcrIy71lVS)i0NLB znjWV|3LG}u)Vzj3@fufR^67z-;VK(^v|K?@z0tYZfoqC_Wm;nZXGmgrEK@$PE8;2s z?Emrw_&>_K|5o{%YH~}^CA+I10{sOipkBW<@&N=TwIjDx_A^2K+oSVDk5Y+lnHq3w z`=&WmO7-S>u=yhWcKf;CaL0n~S&J-_$#aFZ`Jx8LF=4Flk))aXir?z!7s0Qe9*WIB z*8Rl($vc+e8MpFe>GxZKf<=J_xb>xgl57d;&LfJk#ijB3l)vq+<{Owr@Q;D$e#(^b zd&s>}c8Pzn(@M5?eVww<3VLJW69kh7ThlA=E*U5Ne?y6hKP&8Dc4!hQT}bJZ+5Qbm z+d1xjPiOiu&Nd456*`ouSSc`sUm0?{Kme)Fbg*XMCgWCiW34W#!Mcs(uo62=Hts_?OOl zBVA-)>Dp~);59O)zYX)~n@I2D88qp8VixAsDXBXKVPQ=he}zZ*lc?AzUPvgwC|^RU zPz@#>WA|u;VpyJkSp9Ni=+ULYmW56DFLB@hAy7gH zQZj&n7D^KGX9x*Nz6y;J6`EuK3Iz&u5mm|P(+DJTVfLNde0+#IN+c;k$j>WK5SN;+ z2>++@)2lhRu4~}BIdid3qrD{EV)2CXDlSIZef5Pnn6#_X4)4}$#lkARH)Fze?X~Cf zYOU$?xtM!#mf4!`{X8*?ODn&;IDDZ5WScvZ_U>+B_YV~r$#*&_$V((hLa6UjgGhr$ zfnXSX{TIrm7U{FXzKk8idlI-QVW7UAV4qNubrh=9nA^O89S+m~+=bw9R>JFkl=sLV zai7rOh>eF+Ig z@)ZOMDnx-Ii6R-AL;xBE2E6LeZxVq3;Lzqt=(b8Am7q$ILQ^WJQ^kLQ8Z?TRMiNR( z2_cbznH2&yA)r)HqeHTtU{ohku92rflOUi(05zNN^GYvK+~edLcdo)2gTF4DT~5H( zDySqU@r>(NWN~L(cq;?3cpxb^Th}uNl3I%0GkeRP!u8ylxFxU zqv>OE&vuTUR|qeP1AsgC{UlYtYOt^9G;%p>oQ34<5F_^T4l`747rR z-gqSfo+I(=yz6sA8`>mW!w;NduzrS&0U5_YyIt@zlbXu&DxmOvD0UIdE4DJ@`GtWT&>py#_Pkm@k&ScW93_LraIEc zK^8|_?gh9zQ)LV_kS${Sv79!~TjHFqXOGM6w_g%rGAw9QI2hS_;|divMU-b{YPxo3 z*M4AUHqbUB; z&`GDK6v@LFxn_qD`hxLEPVL?~ zPhOj^4{qgexw~Jv#x_0`{SFoyS=gkDxrS80DXwvqqvO=_TjRO>WcHaRfH?q-6W;=q z(hr<)X~&wbp4%607xT;Q`KS-vZ=N!}F)MQSXl!*hH$@5GB<8OfsrHfX_mSXaZGYcS zYzgzfI@4I~Lf33{8-;eER-Jk!u;=)u-AqHJ_&KmSkSkNQ_-wUP3~Rq;+SW+ z0JO(e+rv4W+(I;~;oc-dL0nz$=VD|T6cgPmqvI55Q_HDQDOnm7%CippP^NmkaW-BE z>~l_y$A<0qQyQZzbGUkg-9H(L-@ubkEhXa%{~4~XLZ-o`IiSTFPNx@Dr+4$`LG0I* zJN{nn+U}7$WC%=+&&Ol?Z3rCC_skB@nmAhg_GIk^x)RS54up+&QZWzl_=iZD7#hocUGET>?PgH6ZXD?9;r}G{2vUZShWYj_^W-cV+TyGHrW0*gZ9_x zs+&`=IL1C87TerbGO}`t<<>>{-^-l9)vs_#YqmOLudgooBv;8l7<1b#*-}+C8JXXD zrE48c_SDqIb%zSl_4iKYy9cu?y(rQ9O;zQxxdxtZeRbY@u{qvP-ERkKGM`5h#JUbi zYP-m+)l;6LEZR=s{hm|px9ta~kl%*bADnLamxbKZsh_qJfIEns6f!PgT3;@0ON;E@ z`t<6v)vi?LX8MAtN2wMX!tDYFNdl;t^IHrH|Fptkc}GJ_gc#8cQWuxWBHtkt?uT5% zu^>4*Y|~Fbb0F(a1&ZHOD|~^S};BT4XG2&wP!_dUJ2*}0yc*MTAY9PPgFt(N3dEF}Oockj(2bmN7IVemR% zI64kscnn+@vob}shi}i8*vA{0wSi(OYVYtEm%*0t+Mr%;LvxFiEbkGz+DhVA+`1u9udkJa&+>F@i0Ld=$|3lnc2gTKW>B2y8cXti$ zuE7Z&!QC2nr-9(^!7V_r;1Jy1>EP}bf(Q3|cz-i?tk>5&py5PT4&i_ z`*{dT=y20FF3*hzAa4R_gB=wldwgiAps^(pse*$&h(MMQs?fUl+~#c!i+O@3rGdEi z{-K0cufq*(zOiG5*JwbZMcR;M6A%^Yk0x>hLvZ&) z%f<|v({Czr|s{q6wfRuw{1NvR@;TO-Z8TqbVT@>5` zapdBR*Gat(4~IUbr9c7i3nC^A60lg3w3}7=eRyrlA&unwacem{sV|FzO%{}Grp_9k z=Mn3B&-}49_bGUf1%?lJ&ye+74Dd;A?xz3KZ|b5Ld;c#l4~s~C#}qHBW6NjEk9+J- z+GN|^$_Y2Ezm8T3I}K6o9RNv%AE#W_8s$nZez zs~l}wPV3m~PWJ!M6(6hXys#tpHG@CZ6-^ph3=(;n(T$!|#QtDw-akvCReQq>zFUsE z-55ciIQCDeprKKtKt@ssczwjH!Ncy6K~i{)>H{ROM|>5V%rB5T?iWNwCZCY-dAbkC zFfOJdhzswiLV)}N=~Y_TC8x%t2oBqm>m9j+1_nLFmHI4o!2}VqAVn-S87?r z=7Z;0d+a*BD@EIFk9|(wnm6hUT6RXeQ|6bhGt;%_I4gc8b9qTBDm{(2!LV-|s`vJm7yT~{ zIxYbTrNlXWdqXHLCcXXR#%Z>G{$lDE|J{_o-&bLK1En_{U=s|f7@KBOa{}=XRC++3 z#3o}=-kzhp{)|R0%9EtJ+Lj2|01bIcj*h2y&p#W-1_CT(Jw$u>;~;w}ajShRDb)4% zH!jP6Q$4q85Ue^J(R?fD=v9w5wesR($~305wvhe6|4MOAB`Ec96XS6YbUpDdS{u|? z!DueJ#PVMxfgBkY8OTBq|1nKS@i4C|iHH;*FZuS>ssRZfHliBv83wo%+;B3h*ZM3& z?!TwtJxXGgCNjY6U}w5OFA3=D>+p^WR1!cgMHv8v#26fWnjzp9uL)#RO!sm8M=D*a zx$@%GY^^}gV`3*nD+AqhXg}Sqf4ysBp>-?2{SB%6V+c-{JN5mlJkM5S>>S;_Pf={Y ziq8i*q#|MfR2)VDFuXwe2Bab+fVISXl$Otff(8;?Nzpn2*21bMydR48ASsUsjtU7x z;0uAg^IYdZsmrTAcURQy2>e%dkr%ZLQ6*<@mx0_^*N&>Q$zII2-<|LHLA2xPchsft zqfVj(C9p=0+Ej3fP_LC4%y034rwx%zNLQ@d&X|9}i4(noxzc1e`g=>ce1py@Bm8Q& zX@TsOiV8{LEiI6w!@g$qr=^aTH+=3CB{k=JUSDxKnBB&*8p9X$Ar}>(o4UtP4Qxs( zB%ltXE{Uwn(t1KK=C}jo&;4O{8x_n}uxoysG>hMGn zt&-6Kb9ujdcy}3g{WoO)baQ#FZ>qyvR>gLI@tDs3{+3=SbL7(iP>q5^MoJA)#6|`j z%rlAlV~F8x$ct}xAl##QOiTI1SuH6PrSK8p@V=G*|h#hCV3qW0N!r6{QYPv5U*AwIpvl;;%hL~Jq_g2{Eq z9MRSrP97d?>e?!HtGbeA{)6j>`V&qOBfI zmwL$|nc`zHHv1pgg1<}8KYnI8W~{!bAVWn%bGjXZwK-|aUX1zQ6UKiQavGnYpn|C> z&40f3zpoB%`WM=ILSx(Pwx+=H_6|;P9X?U)H=xxqGT@G&jcK<9Y7<$^pR|(_3%zRw zu|@4czGpGbTED3yfD{*31kkFLaU^^1vFN>XP)y*tsdZna{^0|^C<X>$906}9G zvP$EOgS)Vp()x;G`m4^9%B3ZfQEM|$c}$llHz!5N`XkO)p}lHHd-0gvcnP~7?@Qa` z3Zotvi>3G5`Y-VzwV+ez(9qBlbX#N7X}@dLtGbgP=09g&3yMh4OtVbN+hdsPcVBhMBiyYyUcUs4U%yEl8~ z-rndB+BH@bwG&3}%I>jiOL;xM74|qb|2>36yrq1a3A5LeEiKLLzd*2bVENF@@osdC4)xMQT@!E@`Ur~iRr!9N2(8!XuW!t1{-M_D zvP6cDcB3C>y=c%8`9JE9@|4-+;)d~mRYW7bI@T*HWonl2AG2XTLE#-xV3*nKHZ4gU zMTInbF}T>`Mp9@0D43GvA7u9sD!{2e7(;Q#atNAD3YM&iZC0G0+$3^8T^s}ih7(h# zj3GBPq?zuja1Nu1(;{?Y|VKy|Lit=h}|Z=2Bn zyZryXl@6#{i(;nArXNnwqC1LJlkCBkAGVYP2ftBxOZbh=h=#XaaFQzU3qciK;AOND zH5v6rMK5lOlB_q3N?knV#*MMNm*pBy4rbZe9TgK*(#!$}?&WrBQ1DMB5jK*sXVu~Yvy+9Jl5a6Z+8b!EF?{oXiLYrB5E_c)+M^g zmA&hDdRQc}p&iIl!p){W;yCGq;PxG2U=z)MozWtA)t@%E&oeVZmb0azNC{N`*q+7E zY5Q*aa0LjZo}UtxJy!*9Ups&OFq__@}t1G5HF2ZM~I#9#Sr^j8YW+;pohi* zb1ZCFu-{l7Cyz}^c(1J__ASZTV!^PyD$NSpzx#y$nezVs)cODBwyUcv;0&@7{cCdf z-~9XkeLHpARC%^oiS=2YHI^3RP z{KpQchHy_EzKDJkKB|B!u`kSmF_(RgW7wEor~Fv+-x+H*(QPLjw@gH52u?w#0pLGU zBCeM6DZfj!xOC`oBr&TrgBgkvi1sD1--b7;^qE`4Z@8NzBHsJLRv7arC=Qs{Q6edX zL-Bsb^X#1-wqAx!9S?k6=BU-!Rog+c2+9g89#~2y#-tfS0_raW+)kzA3ei7lYy^mo zm$3~RLX?Fu;sMZaDO-mL(E{cK4dBjR8Se|e1#6TApoO`ujcH~0fJY&sFFQV8oF^) z<)hWw%|wY)xB><)N?;9MIS6*UZj6#G70KkD3q#Rj5@dPkC(?+oWF z^@S(IU%&ut$DPARZSCRf<7EI-0;Ti&)YOL`Wn};pR`-_xe^!4xq$u&b_?2)2y|iiQ zVi{8Im-@~c`W+>n?KoU46(B1OQ(~HO8}gV|AGIA!4ZwNbVi1 zeC?v?d>H6;@f>GWudeg;Yd{q^*QJ7mOf&+c%vzB82aP}BWRjZk#(rtVO7BYCUSjS- zfK8YiRUcC<$VoPFE}u2_Hzuv|#!L3AHvF?Xh7A9V_0Q6ECiRI+)qx=B=%fPiNuTxf zlmH03mAu_`YmUQOsuShKY!vz^#=m?`RxKH5ddQ-K#Km)+`1Gl z9-aVf7k~{GSbmPP&dSP4^w!nbf6ct>r8P7ZJ~1iDNhmw3T+n6DsQU}_8}XB~&0hH) z0I{YE`1E?)fBqxj#ZlPfAZ-5fwa6(r?OV55{^G&B%i*22 zbnDTl-HGjPUBkR_Gz=Fj2z{EsAIB3|Ae9s%5W7Ma#WC8-CTq4cVM<*GBhav&L{8HR zv;_=SbtL>2ra#ZT=DSwuNEoHimrvU3++W+Tq8>MfaUgOrXpIx>fL?JuE+y4@WK$Iv z%z)g?Ca$y#?>|2CRi5bdetg5eZKo^L`7qf2=p^xxC2%Rb?sl2=qWO1o=I>lgbl8$H zi22%J)y3a(y(|mL-wZQJV+WcmL-7*|!ON_Kx9VcK7=Xr3AN2h<^M`)(0tXWSC7z1( zW*rN6xME@#pzGzdUlCYqKD8z;YU^vga95+)2$$$m5 zWO`VU$-x3hFPJ-u-B1l0qVncM7x)xiqw3uf@e-8sYJ56be}&2YZ{y7R(DrIHb9Vkz zk`t%fxeI~l>zszc*P7mf)P%Ia{ix1;n@_nHBda$x6Yj`Wz8tYm+XI~TM+hrmCyW%O zjInDey#Fe?{}5(?1Bda(U~qq9q3=f$xDY(gH(9Uhr;bO@Egz1$6K9j8?f=&#TLV8O z@70>k@f0a%4t&R8P@lfi3&pSv%1$z;Zn<$}QxWSrG)+<8-4D)UtPm_yW#Sgg7|2_K zVxU&(+M%`iwfM6teI)Y&`TV^8+j`vmKK(4th&ph%vclqe%t|SfFUBfrPW{vBf2$|2 z_6{BezJocuu_G|B?TuPDH9@DlK zmavYj;NN_Y2ajTUPn*m;KuLT3ld?=qYC+(PCVrB?5e1yj9JgIv#4<%kCN$9`;cW!tFrU?saskMh5rb4{Z?e1(B?+!hr!76Dh?YTw3x9{-@DO5 zU&KwhnUlSOMi5q_FFELEb*ei*;;8D(w8csQRvLMrI@Cm2LH)e>qO{9)e$!c86~$XaiC)J!uiWY`w>?6#-Wi zebJsSt~NQOihWP{kd^0ZgB6v|4i{vb`$E1g4%xQhbR8AU_R}8LdJ0=?afIg-Bh}n@Y4wvuw%1fs@C4U*J;qU4of^h=TNigZl_Pq_oA48~LGJLv|re83-nj?P~Y%UYyEi;7@}D;4?~a)8qCtH>XnvFJ(3&zHItT5fJ} zC4L(FA7fpm))C|(o*`>+ZXAn*WHY|lNv4v7*^%ek3oEvmJ5eYlWU`+If6bOXIMZDA zb{nNq`aYNMEBUwy$N6X}M7O^+cdWRnHb)joKg?b9(H7SXNyzj}lx7Iij;v6tr#&d8!@mz(o@V3XC!N2SUGH z@7**c2&6J9;YKybh**QUzv zu|>`2*mZq>4F7R9^Ofb(1l?{dJ~@Fr<8yFkXL-1u3d@gvBY2AxF8?{}k@l>3-Xc*= zgRJ4iMuzGq4uGl zIZN%){u}N&Yxu7m7yw6xnViFRcRIqYx|`P+p>XV7E7Kia@-Ew6(6Ec3+U7RHIewk< zUjZ=#&2Yd0einVuoG>+4hkf$7p{1LG!4oDDcjrk0v6>HLwBO$HQ6TWM8(-p7UECJ~ zSQUTaC=lky0eE4UrM-X1JAfxX`M5Fg1+(*1n{NsX`0@oDB<-tz81?Xsy5ZX0ZuGpn znqT}hqWL3g{%pv+bGLW4(Dfws^_M>BcBe7C47k+YT+~DPXo5%DDrA2CNtaTYpcJAd z16W~CO&^z}jA~8o7J9RWF}Pf8cny*67n?1499WhD7Ek(U{XT8h&NppB=Bt<1qW+B~ zBkja|;vzS}5MwQ{9xULCL^Xb&0I-9&-{2o*x#D=VbER)ti}QS~v+gb)TXKZ*k9R0x zTN;kG;$HZy7c`Eis-eQixHhh&(@yd)%~!8aRb>E6e3N-yS%9&Gg-)_aT?g@`MUt`_6@0_T#S`YU z6DH3ekFq>*QuVInWIKd{vSnsTtMVKw=)5_NvS`1xjQNdObj{wJfZt`}kyct16H z)P4OMs4zJL2sfnAaddrxlCwu@dh9(u`&yUwY9^*V{!D6}_pZhkH>6kwX5APm!uZUq znDb?`k#R*=8lJ&~<~VTOW#mEg{NyK;?*ws|Dt_Vw9)bBqT(Mk&;ar$3y{3>9`|r{G z1l4mD5C+J@R<QHpLKAP4wYa#HXl7B|?zp~vj{hH#sh(rf+4RbHKy+PYXbFZVTdMCd!H!YV zAQ?V^(zQ@Bad;3<#Io8WG<4%cAz&X|Y-L8e0tfXI%Pp6)ObI|&;l*WCi#lM9y5DS% z%{h_cpw86zb=x9-$IMjg*ZhNiwp79IQax%X;%wu;f-Ps}Rs_<(Ve8!V5Yc{t_a{n> zPA){iFTjk^#wkd|F{5okQ5Uh_KkRq?+UN<0FNI}iIfKqy&# zXpg6;InUp~h+Tu9*=|Hd;{qDR#_6(gFb7?WAV%@uh^qzkzSsYe*T_`sQjM$&)O8?% z^n7e_a>o{r;O$ecD_8ki`zSykykedEX4<<;iT; z+=Pi3`4u`HKzZlzrY@rT-4D??8wcO;xCh@MFXhAZhk@y_NWzhlvinUKA&^e<^^vgG z$u|CquRFIj2HQ?QzdOh8Jdi+p5{dQ4x659_kao^=^cy{ew$UEQZk1qHnop&+PGk)t~XYCpu&Nw^3|@J@(y|w2R1h|L?C@GT1xv_q%~45YQZ2+{XceKr!UTDzmxo{kWfZ zxSfs|K3z}b`7o^~tQ|^*j;Hr>=YDmHE;UEVKYp*LN2ID<)jnj;N=Pl^@K5dwLvT}HfI710XRp8}GV^t{5^w8<9g z?2&w;tSLouD%&5X`PPQj#n3BrAuDbVB{$GmR;a`a*9-H2K z*EvTahLAH}_Rnx(rJ|Pokn`Za`9K47d+mvQ^JT&4h_qmGRFMj}JaB14Fa%U(e%SHB zQF^M#M!c4*fb*hoL=?=m?K1M?4;02K{7s&lGwjw|hi-3KW$k_}<<{v{ik-A?odM%0 z11>|e**V$MP8raKaGCh57D)xX1eca05wPGZTLXa%JRgCc)hngY#+bZ>Mliy(#VKWt z3>Uz6^`XM`9^)Wj5D4NeNEoy4KQ!129$w=nt#P6|eR5ml(y)!rS3PySF5mht&F~XA z`9?9VvEyuUC=h$DK~QHe#a1KQvN^Fl1}q>O(dbuS<)(*C+m$T=Nyay~5Aipi8`BY} zkY{cpsw@e_1V)weR^$>GYN-saokTyS+d)ry7STBI>$qaG1L3iX)1ivu8``-6jKA}^ zoTqgMX)XFar4?J4gE-jhy-LBtp+iGD$V}jWl6SwW$JwK$gfJSu^gc_5Tc(9JOHL9y z*57y?ap}z0ze;4T?=B;&%qR2vj|9|0d$Z-^I*Q}>8qShOR+ZAO1@UIDBuGV74y8te z9i>@Y@!3q2eMv<#W~yB$@tFdC+xHUA^^6U`MQPPFA{u&%=zCuk>jx%P%}|(h_ciZm z%ufzYKjVhFs>p1GKhEN4X(vif{~~U%T@I;Ipr5c7Cj-KWal8GJUfVlcd|}f%!WogF zt&m((E3vdAe0xenL?@HTqk$Mz*Eh(E3?02DXuT%8HrHzTq{dn{cUaKxJ*td1Ymr-W zXX8A>LS(xoc$tfp^2BNgR;O;x|2!NLfk$mzmE6zJu(xcUf9NZb@#j*`y>t=Q-lDwwed-fSKd&~p4O@^ zO_S9RHBb?b3keNWRxHBS2~lH0loSjSy}0dTy! zW6{-BEk3bY_+m6}P^4s_s29ffX|K}M;T@fdDvtNnfn1^o^Qn@~yMnYiU%`4sc)mIT zYQl|;6gVZ_hD_PKO)|mb6_wyO(1x5S18jYZrD%J;x(jKB?iw;ZLl{I_Iv@&Q^j(2CTg=MMHQzM0rER+IHP;EooVghNqqu8n0VPuPgwfl@L(92C z*A-oQE^A%d#!fESxLbGy(khdE(jt~s#q3me+F$!)9XwG;yW#iXJtYJ9bh~OY9k~-K zf?_>Sp<(oIUqfJ5GJJos>>gs>SZqa$0k=y->Lq>0ErLw1Bf{+seBKJXfXY!R_-6 zE5R=$FZY`Fg@OV_hl?JL+Uu_6U`E=MtSH&DooNUc$E;*QS~+vmU9GL;tNo3_7%6IV=b$MIi+vh5($8*SjJP}!BfT28Rl=!L=czKQIX5rSLFw@K3o=ttS z##4m+r#~@7-$3}aRmNyv(NUN zo0H@s>K4wR*saQ&RO}r^IA+4Tu3n`D6!JKPxWR!G8Isa%4w8YmtwN--Qf_U>kXEPG z@-$YJd=rr-eRS(f;x4FN1v30>*WLgk^*K_;I*;p1=5Z+wBAt zW&N?*%6=xfjs;vyxnYa+XBf-8p6jFy~^_Vk1vHiF%jjxE-bl zuSu82zn_=Mlekg|j`!Sg&hEBqH?{=C+Ua7Fn`{-MsHy8nG}C1Lv=n|Vt|XU@W_O2o zu5~HL#hGm9y$=L;>-N;ft#59&p6-mh)dSLo5f(Suso>|>$|2D99g_Pg94LBWuwUar zpP;!}pf1&IvF3dOy?2DMao$wA%pJyX>K*56Ag(rE6hU$Jdnlnzj2$C&FsRIv@n-ef zL-z3sY45azo=Ye&aCW()u&Ow=zBFp<3?zWAp zsrS~Ll+Dz~oRjm!(NF1{nXu`aXK=c3nc#qCLyl&SU((&f4@SK^qE=hwgEdAx=JhK3 z8yIr0GrxvgQDD*{d30_5FK^no*$LP&&t~gvAKO>FmT|_0M5Bd;GB{d^BInMAXyVSi zN>SnlrHuA1$$gHb+-Dv8Di7k$r%LP=fFZ~+WE2TLz|5TL4veyk^ggcEKjH=~aiv82 zr1(D`d)wz$zsAW@v&kF}1t21)rx+12KRh>5ypp96-J9s01_E&thwL{VB@uY0r#nR? z&m(B;6N9_wn46=fV^KG+$yaQt7Jn>4j(?a*6!0ST-=C|VEtSlEg7z~M+dd^3wu|c= zO+ns7ud6X%Qe1hwIcz@Piz~hHJkuIYI~y<+5fDu^eXr?1^aj$Ym@xZl1N0g)Kqvkw z&AP})P;Q0IN`4RI_78&>wwko$9)yL5;=6@%;)P$T$aF4{*j3Y1)Z5di(2_j=%&OZm zSwe}RwloQQG4Iv5zZiqB=vcJdGRD^0Es0I{9lJFe=B2UlZ~{WuW%{QlutuI&+HuL2 z`YX5Kv2l9ubP$!fOc^Vd6lC03b?0pR@uo;U2n}m-vI%8^U!Z(ONpAh1-#4&tkM9rG z7M^ho;#QK6E3*5(H*h05)U^G^LlLotflcTa*(q< zRNP?Dp!(Q;W@Lc3J9|zK%8h?thYq`V5swG2k9Dtt0^%lNrfA}=8IBiSSji1c$9ujp z(*Ab-=NKRIjCY)`7GImgF&eVJ-A`a>_SzB_pK7?Dr&5*hE^HvG6Aw?MRa`#QJQwm(Mof$Rh0K0s_W`aJH(kE;W+2R{PK(z5}u1sNb5A z?=DLfzg4zC3nzp7@l_5dd^zmGcY??cF2&s5lJ#sjj(v8e@IkoP zt1Nyh)OY0jTH~}Uo3DeOFO`+#P#gbiTkoXFo|nJlmcFyukt+3Z6Oxs>8oQuI5(7i0 z==o>(psCU?-e$r!q?a1p0srX7Jp(1iTzW1l_!4MLD>$}O47+d^H(9>mO+SW2XSmRHx z!B90C-4(SWG3n{Fp=Tds-q;FT-qsS1aVTSX_=>e((KDfbAmy-^v+hb4Y&zCgf2<;I zZ#vV)+{`Ep4>yu_w;86j^b}+ge1Hzz%+p$nWvax9 z_N0B4ao{vQ%sH>kXJSxxwvF>FLWddA}sGHW>3gh{FixMm7)g5hR^45xg3r%!3{&h4hDAP7~EK^ZEr+BvP9?BrzsVzh6$42@_RlXurcFn zk0M6`cgx8qDZyv+Mo$c@oDpzdJvVBZh$eH+XDg?z20KV#3#)V{^XoS3#d7zTXzr_V zJzBAmF5#nWR8$VYmh{3!6LWu!59jbhR`jG!eY#Q)JQ=rwnX#R>(}b5)#LuYE@4evrVgjPpTdvB(W7*pU z0gw<i&x>oN9>HxhwI&Rc|d~U3$rnDk`oq zg>vTLDlOaKv@uv4>ynGPjhhzv=vqe+pYSO7b>lWac3uOV*u9!20ojFee3m&O>pwwY z_N45~hHUwl4LE!|=5KP=00xR*3H|^IXn#A_|ArGmXZ)H@xj2Led14p;?~w&=?eqhMWA||nF47IC7~i1 z){rxOY~e`b4~aDrER-CJpnr4vNtlb*@t=n#>q!BX_Dkdm;u&~VC0utLzz@9F0{lR~ ziE&>zRy5&}7={18_X2Rjyn$CqT#QtvH{+x6`ywcRRWw`vwt{xSF>3>CD%ru*V80;!&Wnuq&W7mWS>Srd&BRbUp8oS; zdaa9i@v|M`Fcp3**CwJHRPrMs$V{l+yUEMv>3!tHQE>4+6!T5mZ;yBUJ*4y<%5#`Y z*)Oy>@a6lf!nqcIX#CAaq7|DjPCE;Atr5KxK{bo>5rZmrF}*!l_)%z4a3#<^`h6f! z!cH|nA2=mtV#euJ7*@)5QERjZ4m?P$b=xI%w)&*Wf+brqAmf;y7<=d15$A>Re2>U6NJu;5X_ znGrkWy+mK{L1O9t@hL$xTD*tcEZLQNmlpW$mAk?{1)=9hO}Yx(4~(^MH1%@29q3gy z^6lrbv-yMI4ttw7LUWpkEza@k;L3)BZ~fjGzOjf(#{_r@|ML4yjK-y!+~9rC!W^iF zh)k3o=L;A}-b6LI9__Y0-oxuW?GnKc>fSP%)fn?4B`H0oE#>=y|1Qz6U^IRa%_?Um z-7XfEp1_va zV+Eu}%D~`3IxM_Ka(_w*q363Z3-qc`e#ByjOL5QUrJaEhSce%=? z7`kNj^gtQKGA7uVdwD*C=-|hDlZF(u!p6c32m$H$ei?$Pkr4vYCR0*UoOKXv!F}`w zkS7<3KOoK<-cfqQG^ch)@E`#GN#yH^W+w+z`o-|hEVSRbXW3r!OpBa zP-##+BS%5AsDja7h-)C0Ux~)^S z(0hK7=bBzlz&(`i+FapP((VkyUli5i0SU>?s-6D$Zae+%2jqRP@zuv_XLy%o-Gj9v zqRqTKQe&6R^QPNjC49Yvnx+=LuBOReZSe;(12R$I0C3!X&VxNwu^kLPAdeUPOMXYA z-N9(^0S{_ee8_@f6*CjB; zTM3lSCJYEtZ~3`F0~P>V@#jr;*8HX>Qh#pB6=jRCxxTOrEWVZDb7`-ZNnvIr6hjWV z_(>kv=|K3$+r@f!wlL&OXZh12^eDPAeIGGz9~!SP8_RbExGiS&J?7k#Ln(_< zC?5=VwhyA3N**{&4(B*P<|0A4L=w?Rh!M16!396AFM34htA*`g<6`{+%^r-4M0^Bq z*4mRgSDkQbB!a}h%u{SE@3@-eCB(rS#yixS)HhY91prMD6$Ikzdz5|j9l7D}?G(de zC&9IrqBQXQy?WkXltZ@Kumqb}cNfMjB**AG4C4b?qqIQEdt>eg0!sM_i&&~J?|i*> zHY04sJvkNFx_0}LvSkv-r3*%ameU&@$tK8;wdyVD>4}ro3_a4evLDVZgb>qi>TKuq zbl_U1D}7<|&||Axsn41OkkEK;p-2Xuj2c6YxS+Xkuute|SH`oa2WmjWJhE|qN#f!k zd($yi2Sc7RupX!uRM9=PjouzxRa86u zZCkTuV$PBlBGXbLWgzt+7VlEgnP(9SIX0av`e2RXUQ@&*Up&G1_|}KI*uVveb^P$w z%$?1^4_Q1!cmHBUivRCrBBp)w4c4=(yFc~uBdoj59x4NCb&BBO=r>!Oq!byhZw|~y z3kvp$f#OZcL51KrC|=(BnVDdmV<1q75)C|)?$GqLBXX;k`GijppPQ=+baLI@?fw+q zSIzSGR+P*>Kz&$E=eUBi?>NUlX+C{dWLLVAOTUTel0>*afL~ttJwKKbaK^f@kDoMC zJ|5R&9!D^Y=qkVIgk?`oM)M@GemnLmwO27Swbf)@r856cCyI$k+AYk_hwfx2Zj9R2 zLX1G}CmO~G&w~`UbJ%4;Ql!vutJ20Aes;L3xSZMbGaGEzt!4}CZ=uMey-8yI+5uXZ zFd23(s+f?OkSDhO^R}vDe|d~^Qd$7$Lw&;4yt|`EP?3Y7az{uo{#D(XrPIM0n8v{^ z{+1^pt7G-7Qg@`mT0BIUzovUJ#r}ZJQ8pfnTxxdh&CFlXK(&wBk&s==72|EsJav0N zc_}Hh&&dW%ipc-cK42c+I{NOsnt;fYCUa53(2drZJ$|3zgAb*3E>BJ@jI~rGsIFGMkxHWr)c=Nh}$ zO8j11)Qw*8pAtg;FtW!l_d`8aA6nn(?^I0Tv zRgTy=yRT8o70#AbNk#CHCz9DU@Wa^i`AQy2&!!UD|EGt{u6M?XGF5Lj%#nLtEvXel&XQNg+>NxV zvOL`PQ3;(u-JN0Lx4YiuGBe_CD>{KmS65tZqSz%hmtjXbfk9RNuc_!!oq@lq0E+Kp zhy6hN1w0D!liB)YGGYg7bc5(Jrq&r!*Ou}LY_0w>@OBq zAJ^8GMz`8@Fr(CA(tt^XrRWN9co_5fF@dz0D!6R48og+e7t^Je@4Rf$u_b#KwY<}e zI1s}NU}ipRsA-b~j2g@Rvn3xW$_SKMJ=qs+#P@d_*Y}o63x4}`v#vO+%aJ~U`D_T) z?S&^Mf|`$}PJ%P;QcP4*X7OdsKJgZY*_^|5S!^kq;~v)=$3^yyL+(93#0vuL7UpAKu_%-`&<-hVSnW8}_-;7#jO{ z#gl^!`du^|^7Suh%K_~=rL;k0MbPWsPJa^o=}oG+e#an|{VVrSR!`AAN=xZckB_?G zXn`!@QuL+q5_w;^&Fw)zkS{;~D2+7)S5wkL!0@lS{s$SiUooTB0p2Qa*$zdxq#TTo zf85z-c0?YzyhtR*oC3hxF|$)&f6J4|MPPzpHxuefnS`U;?U*7GxJye*slJfl7`|PC zBxbz#u7BT+_xZyAs2joP>(y%4`KimzO2>-+fH^{~2MiL;5mwO<_6l-7VueP9#-ZnN z^@MeO(~WdRCKGie3&C4tSxhq=`7|n@DxYU|DO^|(R_T+km_Ym0QKkK+%~knoQ_R^> zEUa#YtNi=C((i#S^x=-xa@@jEqb1==R1?PAt?3aXCz1vaEckv6*(zo3iIWv1HSgSi zv6{KYe&3x4DKE*E2ng%7+PDreE-fo+DTS+clJM=fFQ5C=W*<9b;j(?T=yXHa-N)Iy zphRlX_uVG^SSV#tb!BKfj&(Ol$MfzKw6Pb4600QLIFXl6jA)TtROH<7iGQ*th$1XK z$aZ@(!hdJ>gUmdR>edtjeNs+xw45B}Z88VdZBfb(Yyh$CEfZ6INl7?>`xqHHS6xt> z(2`vSp!fajn%k#(rsxzMzAD51NC?0B{DkIG1{33=nxuYIz@&kylR6fFs)mb)S7Twr z3w@2kqgmbXZ$m|DHG8x2vL2cKoUJ*Cy81fX=u|E7#5PyC_zvZPH??ZN4)R|ftx@YG zhjxt2)Bu)M|MyYVe>$@Y+hnLC#u9TM6hBj#21ee^>#JV$pLpXU{8VeOXLf_WRzOmv z6%<8Jol{!OzZfCq7vB#epi3LojiQ?0{mL1xaWw;+;|G1xBm#12qh$k&d z>gk{-XNrSBpHTqRPQwSmC_cNQ1R4&RUoNgfRAuWH*ME_h7OLv=5^Gn{2oYh$zn(A} z&3wbhpD>Vc&Yh5kU%Z0|{`NL;AQlIw&#s{&PlHJ(q<1z@nq9v|fvTEwA?|a(Di?|? zI-=2BPIpdogPpp0aU$v+1;&-+-5n-QQijXL*CrB|pv0vWuQW6qRwSiJ!fHGVhsh+z7AGPkb30HnvSH9q6qk?>8Sm!m%`aSkcrbO_&IOgjE8gB*l?4p!Q#`DV=!L2bW|?ngqIC zb7j~p{B)H`h-7;My)gwuJ5Xr@Un<5~?`uV?iZTzM0NAqk<1mw0NM~x&b}Ov?gIpzFQU>B)1NH= zVjlqz-S&z!*0Y;>hZ$RRri@@~R4z(!IGhz-W>0=x{{Xvfhad=WTkPhu8s|gaHcZ7KR=N$^b$fd z1WV?fEB|2|0eowPQ~_;S@fO&~|7?c#=bc%6_FO13pZ$uGvWu{s$+d^cN`(3tc*|=~ zgExMf_f4wI8`s1^gQ3Dz9{Zo0m%_%l7V1^7VkH625yJv zHwF-Q9w8Bh8Z1CEYC@ z(!zIr<9nZ)Kd=k;&Y3f3PR)r&mIdqWkOR^K$fSa}4p;*`qtmu zUPq4CIr5C*oBp91HI4%tgl%58W>jye?es>IpPb?1#`?ygY}p=BhRDv$*7dO6oyC|v zah-cY3#+@(*4~M3feuvC_oeLBwX&zPT?h6$UhdW8+pXYX5slJ0Ztb@!T4sHIMC_uI z^{u2$?&c01N3i6Vcp@j*5cd@jmj8KX%&pK4U7MTO*N6%y{>?Y&8ry4RnLjyYyLvql z#jn{woYs~QIYYQSu6bMMW;i!jjwAHI;MeIu`=>$7pL|=DZ?-!w&F-rBmUi5MXHxJ6 zfe#s(U>#^{Nh-cR{8HTMOOo|{U3iiTUh1dfK&cB(m7Y?HDjlr)Y2nu{_#|dRX?(AK zT*tL!>eah_foC5*(vbp9`CGJIJZDO-;Bzo);ZtqHV)H2zsr$W%Cp8STEAioOt}|BG zrUhKKD^lL;HBEK*vX0=duG4!^^Oz^*e2g>qL+GnfQf3U}#` zJWY&GXp3!o?r9J&WC2tLVFM}{J3|w9b@7W%40!T3I|EN=hF%98*1{{XaoWY6EPMU7!CF9{ z-PatB?2H|chk=4h-7d^eU!(HI_KWT=8kU;=*q(V@7%`+6-`TG+#`g;4K|)($lC>1u2q= z%vG1HdjBp$>ADnDoV;+PGSY?+Y`$=z63PcB$ptJ`ge=fZoOS-G0Y=S0j3F%z@ zDI0i2i=CA*)-Q8l?0IXT>-GuT!#Cd&``DF+8n6Q<^&Rx zBSOO1v^??(TEFjy>Ss9)%q_O(GKKJ;KYVInbmvZzyzMm)`W!1iTV;B86m`%>geY)+ zN2TS0ws^VjlYKYr`r`N`7Cnal!zC&8Vr&K7LCFEDx74+{<{|qA5&JL+EB5RszT=~N z^Xel5x;4Q!kL*=O4PM@lvbz^Mywm@t zsIUZA5h!K9bfjl*j}cSD^9P00hjU(Ob;#y!i7vbeT2YrKhp8-AUVH#4UYDcBaKC)zb1v@RW(qM9!MI9tDLqS7h~z8S;c`P6@-V;0rx@qz8J#q7ki_&^}B$$|(&&zX_dc%D-1iC(> zDbZxBalU^Zr)v!GwkZP<&f=94+fx&B3bOrv+YO^MP3bpfv|axI0RdZU)KH#xBYj0; zkh728H@={wAf%&1bV_#SfGR852vVMusIW4V=`Y%k_noU;yM>Vxg#4x<}kKqaK8MF&mk7``#Qc z0C0K6W0Hl=!+hK47jybC@VwDiHPrpqWc(c?r^N>fms6n-T;clnFVi& z__l)=VnjH_&+}gEnJJy->*Y#D9DTtQ{VmQ|`5`_wkvARU zfoqD{A{52Dzs;uN1#x!4@fc@tNikDs<-2>O(-ezn-_z!*AE+db(;017ZMhOg^p56v z;7C}s1)Sx_i`BsV6pAFhuU~OFnSJA!)v+L6)J8KLwQ$p{)L9+T;-cB$TgQ-r*^gW~ zjlr?pjD&`L`w`>|GwF_g(!|WeK?)F!L*t#Cz+Lq7|{>Unxq?;qMx3hfZCwXMjp#(L5n% zi258|#g?s*?cw1clpq?EtPCG#C8`X|a`E8D@W_Y(Gb42D=6n{_dUhG5h+a$91_R3T z&S$EW3~Zhf^0zsZbr`9MG9q`17FCRYDaV$}Xj=P*AU9|Mg+>Zt#FEEZ_Sk<2ydCb$ zb(wP|gFbJ;#pBSv1~UY9dMw`=iKxA7a&5*NHxV&m!}cuu+nHJ?-8xuG3hSg9fPX~R zUj3~{8v#`?d|h73MXMWv+!IuN&ns zPS3vm_N|19{q_jTprWA?hZk$oB(P}p;} zb72RX?u%guDLh&WQ43e#YM)B)OB{#SR$D>j+%8)Gm!sY>g6obe z5dHQydv;ZXqMuq)rRj@cP3mKYw<@zYC%cq!bL)khJY|_16zs_KqBAypIs@8v!_!o! z7+v}}_v&Bll3Ynn@loR>!QIy4TwU`TeX;2UXcaBiWEfO7TND)4$&nPX`PG!oiEoyI zA3@KAEKDe++M|uT@eM#REh`G(yvt&^EaAHuR&kYe!%f$A#7W{YP|-EQ496H68RC)0 zt~m|%FAgXFk!>m(P}XX+`m-%Pyag`2LmdgCn+~u)$XbCivR6e7d8O?UawEk~n(vL6 zkPZ9J5UHv8l5UdmvvJuC`v|b4Ji)w(nELXE_es}B$yiYXmB88&5#!3NHkg7x3(s1M z)0QR7@}$_uk)WbRv8M-DGO*3?VVTY@jsA+Dq2>$QMWfJqqoj++TQ+E<`WMjic*rm- zr8HtwecH$`st>G5m17)z0-PLOKXp4p&}DxS@Phd>RBSnJ8(<{jQ|CiglL6wO=}T%) zAU+kHt%ULLE_M+Lm=0>kFy?|HoQT(AYcQ!xXW;HoaXuCbgyN~wl0P{>p%tPfX$Nn* zdL1d-zc0I0>?^E`1J6WA57SWeMGY83$kHaJ62xQ>8$|LmQqCug7`3yxzhaBv#!xzj zBqdBGJm>zAjzJzY-nROw%=`ODQ8XBO-1IL$L5qFwCk$1tLdubpQhSebr3yj72WK;J z8OaE6Tdrbk>w<4tc_L(gkTdgT43pD8W!OLL9A)RljaJ%o>$R>KnW7&mMNS5e;|1~= zPFc@5c>LMD;S1<{jyG$ z?YKS-gZMLycreo+5=SK6jfuB<6**u6cb2O4+7XbD`i`8hOy1$)6^-W!8N9>Ay)sEj zOKb3Xa&>fadT0W9oGLxv4zSqz9jdaed5ihSob!)BGpn;a*rg?6+Ll<(^67f~UO{OC zbL0u!D6JN~^o3B?muvs|Fl(#dpPpr#Z!X2BUUP6uL|W)`fm2PK8!Gc{Gv?dNFvqb> z$7PGkqWno2$};q+1r7c9yQq7}(cR46K3~RXj9#cFInwRQ?vIXlC!^ADehN5!=zhIJ zAoVcAsv|xN5xOyUI7EFuT^6}tRBF<=*xx9-XgpOVZL24<^(FaYWu@I>OQP5Ax@WNr z6osHkinzab_f?!fU;Po`XVB+@K4~Zo?@J&_DVNtBbBr5uxhsH&2^J;CAg%~?v+WKz z_|krrh76BMOc**_cqufxINW&o3HvqffCscR#yoCeSLk`iB@$_Z{=cyR`g=D*0;By|dFIyj6U?IX*ISfsEL>AbqmyKjqk+H^#7-5R5g|%? zkR%=UwwHQPzZXZWl%&%~W+6x9l8B^McX2pPRKRVrVLcG}PFzX~8y9yluF0_XtusuR z+MI^AZH!_ovPF9oaRuBltU*}l#a|7A3O_CO)ncskX zW|V;UF5WHGjO`vhYD(pU1ylol2k}&+5#xB(G;w9uBl3ENKo$mCEoMrhf~G2rRH3%$ z(Na6I+geLz{jG1S3rrz+w2a|%ed0h~d;0T=8X`N8v5jUrCK- zeeyi{)Z**m|o*g8mdkH;)ll2 zFz!VD?n1qi_icc%eRNa(&hP9Wn0f8?}f#3*s6a!md_)(b>YfF98Ck(tqu z&lgFFY|=kB2%nXKjNS|8bNM$0lYhu5l!kO@Ki4&f&T{xH_ujw*zSpXGXCriN{F?u^1e_2duI|Z!K_%SU1OrKLZ*IvwINa_Ar~m+lztAgzqvB|DWmCN` zn1H>x=pL6>*F~}ZlMJf^EIiPpw5~1z&@FvGko2mUp%qSstEIEv#qmF`k63ua7iCm< z29c*hw{r!~bs*eUT&Q0T_8py6R}NfUG$3*$Uk&O=6w7ByO0fr?r>3V%8ymj|=J>C9 z|M$nHpFd)+%E9E8n>i|YFB%WD&&Od0q;5`L%-ytETW!e!?V`BDzBqj#DGFk}ytXK3 z0I%Pyq_42~%OUOjUBLgo=zo^;$r(myFqF$~dyqD*w!AT=-fHWvAUWuu#BmP91 zwjz!QMaY8C!8D*(k@KFqP3goZju3h<>U&E)Mz0+Yf;k<8ooonx zL-7H@)=@uY$Im`|UQ}{>L_n-Bz|8OJNDTGwg-Xp=7R~1M0u`lX{(!3_Wmfza0j;hB zozjG4@Rf*tYF%AM!iFe}&B)CI!Ewc+d`ZHF-2e#P_u_B0=Jn#qQS8uX&Mdv$sBUgf zbvgby-DLIYUr`&dIrO>K5Q;Yt8mI-qZa!fOR9<Dd1 z^Ydq@D$+mi3vlYQSu*6c7;u4!IJ072gJU<}=q+S$JG{rHRTL-Z*45S~pr)p7Y-%#9 z(_%q^{*RA0HaBlO^G3wQ4FNWe+7UpX=L>Ul{i%#PwM*8)p`nGxfBDNl(--&b^0KUk z#yjBUK&=@DTx6k;SYh&eU!c=bV@nIbc0-q}B|(2#*FO;x_r!n0+U|N<2vm2;n;cRq zpu8_#Mm@jY=Ck5aN62~lP0c(X*H)5gM%F1c?TKeXgW6w{wK!^F2cGAftTknSY7Jt? z=dR%DP03?F4st}2M57v<022194F@}v%I$0q3=1wZZjNV>-F67qvoE3cycroNqPjZ0 zu)avypZvu{!xDfqx{`l}4EQtz`ZgaqV)`Bf;R3)ySGniB4*zdi)PrX*Rh?@WPVj=6 zgqVuP=l+ICfu5@Ys6hA|Bk%f{PLAo^n;|7PzK6kn1Yz4E16gJD34w-wysf6#*3D3?yj`%^iyN`6N z>%o<&vpNLcP5`eP*wBB)Q7kD*ZycsYmAS_>xI-I{m5n}Hdtt)#0We<>fAY|9uutT} zD0ZEwgO$FAv{_B=k1-$znuvlz{+)0y=*s|3EwZ$K=YHSJE6R9cR3hd|Q6%XKQTYRF zkjy!IeS#ypqqp12Y<#wj#ha5JKA|)_)0%32)*1lI*@cxc9iu`Sy0fVRROyCi|JIT! zril}1GRT7RvI5jLa@IR=FG~h`vi&pqx>cFdki#IrvAJn#X;IVD8}Ch4eWnGJgINS5 zIg_67X)5kTNHiZeRvLjTvtOR7nje$AV<2_F`dXpKW0BbjVYYO1OrCg)*%WkZGw_c54{hqTuxh2%bU~$#gt*6Q*w(P?>at>eqhBIo`Z`e73 zJo5A#g>3%N?`tBJ^H?fC$dTMpH>q8Kr4X; z`f3kw>t5Fy`Z~?fVFZ;-Tw;pt>FWeWz+J4Gq!ne&pB>%QCqIZp5)xv=RngzG=u0XK zN>(M72Mzd%N?nM}S7(j3=RHZsJ_dSRcNsDlNkEJ%zz^RUZxkm6ZMX3)Od-b&4=kne z5#>xUL(!^8B{*R$pBHbBh)Mm9kslSi5=T|NPc)hyQJcB&?@0({p^%bQ=>{AM%8gO4 z*FAGSu0X}**+Pa`^NYg%EH5=V3bNx>=YL<41K(|-{Mvxx6Jq|0q}XO*h~mDG7wy2M zxx+}hHv>3Uz5yd)IR=vyk16V3__43i@0Ik2IUAfOTBo(DwM(6Xrn<=Ok0l(9Va84N z@&vL4i(&x)r1vVxs=(zVSqTojn(SrX6y`IF3qxL0dSa!@s?+a`)bMl6 z7G4LpqPlq=;%|%C-hdS52Dz@qN_qPnbAm7nd%KxNf|wxZ@C=w zItGQy?>Eur7SFhGE@X2iDtCsqE?apx4ibktXG({!jm4<|HO_2o7-d0d7Z2V5TZ*o2#G=g}nK?Lg!S*lYgK=F!|1r8Oj@r&g8e&&nLSe85&Z z2`+Nwx!bts&O2i$vzZms{cQ>FnaiJ&ggxqUGbpvVZpjV$NY-H8nblB}8E!d(%->8C z{mqX(^DBAI?Y`&FaUcJo4dx(W|H1S#UB`X88OpW%<*zmRlh+GEy@U;c!~CV$PFR+{ zwEssDz+RI;^?W%o+`}tz{t(`0^YL0C;3x0T4xy+8TwhT$?8WEHZ2K#5Pnt&-D=0$O zD&xoNng$1}camQ02@kX^MyvLPtKbixItM%Bm?@lY$bW#lMoMo)tGRtxe0BJ!$+{<^ zt_6+VItlE;tOs))KM)k;vsl>D?Ie9VGu^6J{hUb@@ryUpX%L;FZ@%w<{bgXMhV!yL zjG26?+KB~_s=bp^pn?Bk{_w!Tx!N0DTF(A)FH#GBIj-EY0(B|{rlxefJj$Y(l}x6; ztnDk|AX=k}8G3M~#-wq*F)NF1xrBhWYH;;1In&kk8e_TAqM>_bcZQ$5O|ELw*LDd|O&nAS<)sko{5Lfwkq824VSFYk5vs zp)<+9faPF`#K(jr_}SQJz2`uBlYXG-TtjEI!DUY z(NE3`%70K+dR2^-PJWE6r8*w`@hh<2k%^{1k@2*~2V+0cy)zu|Gk3q#%SmGT9;>*; zJ9;44%RVU(Wqw8FD5j%bM--~qnq92S7qcR`72%a@<|@jW^yJ!aVO7`+hCT1zNTIj= zp2N*YR}_ z%|>`L{PZAff{%?{=WFa)abY*chN`@X>q^?4Z3&HYuAJ42qK-8IFe%l>r_G*)+sBKI z&AG+wNtfxUC_!Y_3(ErxAQ#rdMh*sbwG~s!_rn1he@+yIeliZIHxWE*0d~njEZL! zq+fGM>ImUz{CKq;m}?76iDIe{xlu#j0>T;6V~AeTV@=(4Y>2e2iysOF$ZQSAbRRnpWH^Yhm@4wuIv>E2K&DA}0I+?kDJ{9IXMC6b3J(^QIHPEdce)*jf1 zu-W6Wvy)1_!3n*+MxCE(bH)c_UzIsZl)V)|0|}t$=B8DlEyXdhYD4uJv_qG9@1*g4 zu?4+noVvQYXkwm3cvAN_VUXUGPx=uH$9pSuCHI*k^VK%VC(F_g4YOLe^%K5T$sb!1 zW@p6P9rHO*t)y8L8$?0E>_&A*94(xf2y`eV%5=&|VxZ)=$pLJZTJMAEKq1EHCCW%R zNM?;{Z9bGb3ykI~K`j@}^J9WlF6uvz#=wEj$W1W{|E0Y00IDJl3E=2~02FA1?@ctmPx;aBO?gDZ>ceEa$M11! z=e%WPGdksbKoJ7eCjbM<0R_Tp`6$p3mZ^Ky9$_eOQm6>`TBrn@`A>5OCFh0}6KaaN zFv<_PA@mFN&0jv2el;qV3?HX3Y7iXP*5O-WRM=P3j(RYxg0ouRoM_#LplJezdnE>Jpj9a zxb8)P2!n({D3BfiaA6RvEWkcDBmjTI#WlbdgFq$sO6dT3NM$f*G8Nd4G12noFDik$ z3YF=EK_O>H!X4j2uTI-7kS6k{W2a4?xBAS{!&;c({zi7*w4Vob(wkBVf}XsrE2-?Z}O!bOI9 z-Eyl;T3UK@JBoQfnth@NaH~KvF5sRnLw`2-S}|?_fmH((@?dswREiIdfhw^is0;jl zm`qOm7_C|Jrx9K3Z~|}g@ynw*Q!amOwZq2_!S-;Pf6%wd?|B}uW@H);xvO|fYsFO+ zP?K$!Wau~YQ3zerrZ$f!9)Qx%T^n%YG07jfruxzVhET%YL!2reS)MzSsMqL72hcX? zR{5mt9#DaR*cAb1n2otcU$mCVFJ?2aMd1w%N_v-1@_zdUQ`hLxajShdPx=Mv0@l4a z(K%z<(20xBgE1Y=CGpqHCn0fSRaq{S>~hFJRsU_1DB#QKRN@Ou`Yv_V zd-KiRg~tF{W+z(4e3q6P1v}6Lu=6OEe}O*Ny`^Lb>WJ-4_g_{ZtFwTYibrVz9N;uY>wb}YIBFqQ^3Lz}MmU-PWtA?Yad-2}azpp43(!O@9%99VNgC%Q$6 z!ofAu@0&7jYos1t=Qv$D<0l*8e-Ng7jWiE*hW;zP!tkp9VMCM9-!`#zW6)j{!G z{niS+*XPFioFfzIoWHbvwDd9f=zO9~M-#tonzV5+r=znpaH1TTWwLU zKplI4jA64%qAI%08fRLQ{cUeuL9*!o*O)ivG&RP;@5R3ZPBu6uHjZeTD zg=f#*``)x-0gYn!-o(!h@j*s}u0+@ki;ysKtynqIhht6AJ{>)50EUJ>Lu|rK_K-o} z{TCR_kx8*xc5ql8eUclg0K)Nl7(SYh82BcJfPQBTN4(BbC`PHZ21+@>09k1H>Jk665%G8oZcHySA2(XvAJ|DvSzVE}aUOEoa(f}PA3xs^{`RU)pF$1O-YH|r z8V3p``yv2Z|CGQD_YoFvf5))iYilUp*a_x%IxScA^2?;VzC?I;r96p(`u-~>+E zNQo6N6x4jY*^?B7puVYU8UINZx@B0fT?%Jl8%=}A$J|O zgygj-5g#P?1!F*k0XqQqebH+$vY^}PliiqH(fC&deXn1d`ZgbTVn&T`WOnMlY+l@7 zwwnrN(lNuQrJ!>K3G6+ju zRDE`>HGJd{bco%pkSW8!>vw=8@W(eib&WxsQXMomwqPi^B>@3<(ZKpI9{X-pSrANM z9^iiY>j|V(VKifKel%_ZeQ`k;K_l)AzjJj0w!HTxi9 zUv>0lC9q-mFrJ_P^89}k(SL6Sfg1=&Q-kGc)|mCNbpBlb>RwU!VV-RK#ALsu1rP)_^Bc@TNDjU#clqv29M&`)3 zzls=dR~uSLa?|(%MGN<@#@=H?Y=1yv{?UmFtjfOQTV-W&T^(k?<{>i7TVfmD!1MjW zh2W?~AZK@B?GIINTAtJaKGFQjsD-UfvGI`W2Y!`cn58!oa%$8ova%Krez%*@f}>9? zH-F9v&T~DRN`Fqnld#!`tKqUfQIxXyuQq~_jCMEjtS={?CF)m-9_DCu+@Z;d85FfoPENy9q z`PA82Nzr*#0x1O=Ik>QQKfLpbhHvjTGYWm=Ynxk*es?I;5@ep|7a+6o2>Y$R3gO>I zv|oTQaG~z+-*YM=vDt}I*T#^IJy^mf*<4n8Rd>yF7ESD+Alx|1|PgEP*c3kA)IZ_dYIe zZh!BN92{8v7&sJ6PM4dq`8^U>MmjvM(_f71yZ$#8K*&$9qT+qe<|es|i_6~v*-ENn zV$kE`<3L4vn13IGZR_j$va+(_QBga0O(sKV>M+%bW*G$RU>1jMaa&pk2M4;yz#ena z1($z+7-o=1Z{b1gD}pYy43>TkUd=+3e@_f>t5u*J9vk}~r{hT)3GOxA+)>5Ez__;u z!X*%~Y4R&R0IP4wP^Jqmq@#=MJEg;l$Uo+V3kqOnQ<$CNKrNaM?;%44y4?&Ii){Kl z{P6{*c$>$qw}1#2ls*oI5p*h0MvzcZ#(@opQ&?~TD)56F2m&0a@llLf@Z-&ZUulWa zC}pq!#E{!J02@AHDw;?I^I{P!N`V3gO_ml3kbsm4XI9NYn&|qy0%r04k8Q@^auuf*S<#*;!>1yLueFi%^@awK10~C z9yw+71|evT#2Iefw8UN6G$Kxp=xEQ_i@7?P*m!KUe`8kssW!ya?k$){`sqQl@Ju|> zxN6w@F%a5Dm(ahsSln7N<5E^q_`ky4^-lFdwL;3qOxKOx%XvydJ{^2+bA5r2#?4FqvXa(vkqCwl@eALsx-vGw5fnqu~ zK2B1){J9ksd~c{SK9oTT|MmRpI_!|QDZ-$7X*gt^Whdi*(As0x#o2tm7?IeW4eMcB zJ$-1yJrTPDJ=SGH3UZ}vffqWsI|WaU!09%Sna}%My#E0!r8sYSYB?J z#+KMUce@8sN&$oB8apiVx!vZoAm@oq!E-@ahT)#T+w7ID)T$^<+` zJa-Q2DjAT2qWyc!f|^#eIy+;DT4&R{?J;35gRFf$X`8j9GW3$ee~{5l^`$PjVP!kh z-pR+BsJpkOL39ju#p%;VsEQ}|V&5g%|0TYQA%&(kP;N^$>=I{TfiGG$a3%R?ez&+E z+iuedYpAcrC&j}0gQTEVad$2%k*nH;Fe%R^?#p8N|B!?(F6sVu2rr6^Tx)`&VSdu% zqhcw;OskJLzJ<;9Nd|j5JeOMfB@4C*inMM4s+&{wK7Sh~jA3g{M5d<@>@1Dbv;Dgj zG|r?p7LpJZUZd`~UQCj((P8Oxatg_EZWDp@cZ1^3&?qoHV4#oV31#6?ex^MyaXKm+ zSAzP7&im%7id@-IFHDW8Atp8A58@D;xyx7^}!f3NAtlr4IRDXq3*oh67VJ@>$TP zia3b9f}C=tMq&cuYl0N4^!xcwDRAG>7F}H)76_u*x@W*W@ssLm`A-uLRSw*-F=B-z zFf~_s1?=CO@i()O1))Ut2FoJ<$r0E@FLLsVL}^ypc(?n)EF@`Jk-?f6IAR@m!2Wvpq71p%J+tYVRR>Cn)>~9$j+V$x$wA7_s;m4%<~a;LmhOY zaXhQgQK`#G(w-1U-q$t% zD%)^JL8=uWg9rr$6>zDfDc@>r?c9QffnsU63H}_v{zCEobFzwL7INBnaK8p`K!G6K zA52M*CRd#+C07L8(J=wl7nUt&ULfU&T@Y=}SbRHD+9J5Xm&{0*l_%UGBLW3|W^H<+ zM;hP2k1HqV!+mtRd9<=69`?cs)A9=fURGit$Aic8{Q>ni(Hdhj2 zp^$Trg6YZJnB;K)VaUjhQVVGoo4iaS`ID@eg(S8H1MRyU)@Lf;9;CtEz*@daw>WWo z;5)R{y7xQ_Q42umL^WS(up+`MV&zt6@j%w%Z_kFY@^Ah{7)w%Bm^CWN%TtDXu@-jL zKkpYR0FORPdRdgq)04U$HTK<{M}GsP|GY*-wR-2HiAyVH@BbaR8M5Y!J5vpKb1H~F ziZ@PK!$QD5)p%!_%a3pT`4XnDi{cp~hGK=pe&!uafNiWyHHbRCF<8yz%9dQ`f9~zU zi@R>r(%`&@ZLM|9P;)M`t>cDgxQT3f|9<5zo2uYhDLujRpkf;~&QXKTJ_6ZXayXZ8 z4;_<$w78@s;iXbLIsr!>Oi#m<6v|47qh3UiH6E+7MUugP)grX%N(({)N1?W_^`YXx zBa=RL-yiBG3qGUDK8VSDrJ;AlP6MT*eyIP_4{k9zAW1b%6}bBJSB59P8`I=-<>OLi zbKkXl%~p~MF4ePh=v5iJ!^Pd+ip(C_(|w0{Z`rP;q%Q-p{rnYlOAT$&fd>gzs3AMkz;(GyVm&2 z_T+;W8%-Q!u6N4NS!OuvRBr)E`&S!$ z(dKVjSl1csUP!XD4z6Up5zH(-ZAI-D3kUWGMabcCC^Pof^v8-1;Z6QEPQmHVKyy@9 zk~9WPQi}q|b2!eNplEz(_nGkXV>n}ibM-F!&~o1$)aDOr_oH4kmoi^gAK06l#g8#gWXWFEhQNo534P8e^_zL(M8uL(DUJVLVD(nn8L zwM=cRj>iD!UlY4*qQF?^tK!U9w54EuYQu|j{VLSNoUS32F?E9lubG-LPq&SLMh*pe zz*RdqiNs{DB4J|Idc!r4LRix{!)t@YeLR}oCtF~BB)AW0W+zcbPkH#~$Ah*hUimMn zF>a{yvvcfOL*329aTKC=Z36sHb3`XCLcNJ2h|z?<$cp8n_P%{f-D3U1p=rCdq<74n z`Ao8JdL0m`*JSN}>d^=?IJgY)XjKnK(Jj`|KSYH1mULB4jm*%M&0*R5#|()# zI8X$x&*&{Blx<13FFMhNprTi8T7nM|ex+Q|_6y%-CdHKMfR5b)MzlB-p9C^48}&f% z*B7VuB^@F&QAHGm6mxuO#i4cg(9BdL$xzn@@I61zEG{ei=m`Au`<#wHk4Dxd4e6OA za`~N^r$gUCKa`3~A32~Q6qO+C3@)D!UxqN(c^7-W-fv4V;jkAmY1o)fW(Z2|jd#Xb9&!GaSb~gO1^ThJJ}E!N zJ{0t#@dj%1E9b_IzX@(js4iNX)vl{RX_ujxatOrpeS&*BHox4Q;|?S-cR&99@#}2Y z$j&<+ZVd!<-NH`?yo8;oDIE`ktZZpO1R<=NJ&~dR-GYNLG@sI8<@3Y>U8DwLjPnec zstlcePX!Eg6W*7MMv=4B{9^At`@9!t zZS{077n#98uomV0XRYfoMe@{3#r@`@opUSjtlI0aTi4Z_yUNxuc(PZf*ZT%MZuO0z z(rb#h9AwMm%ajV_R>j|+Nn7;qgGFJs}jJ~*4?mSdrH0+NR z6~0F|kRlWDg8{y6?i`6~_sn23Cw-^ZV8PcMn}*Jb4ciw~rB3RR`AIv z^rLJ%>0`zWN{WqFEIZCrYs%Jj`;t(+yo3boyLYn@$2Gf6c<3|V*h`OBS2}(VenxBQ z+0Aeoi6SyxxfVK*dcAv<3-bB!?wwix)3Mfd18U=uZYj2nKp%77hCH_qArN+IDyz}) zyh_6qdl*eSojOZQRF{~L79$3w^O+SJXe*anZN}2~k#&IKGY4#3z^MAR;{of^ikzAa zvTwCLtw%W>GsR$Gpz-@oLpF(hOvR747rmm5^Vx-*x;^y|0K*&n%ETi3B3$x#wW*1n?1^b$C95D`J$kX$_h>R3LC}}BKAgbcx_wda2->2adC?-CUY-!ED`2-k2 z`@jBVcTI`VA$i*SQ|cI)>^!ij{WxH&BF}xyVQ$!QrdKIMapO|)*u>EbU^yrJ-v0s_ zYURK%QnzRg8K3NuSEI_gibO&(@v-(b5h(lY?G0WXNj4jtt-t2cUsVZXE-Y}0VQdRO zG7IU)CpN`NW;M4(tb;>n&I7xdgeP9{sH=}rjm}MkmT^s&vV&VsIog};42QNjXUsKO z>>Qz}_o&@X6#Buqtma@Ij|g1BXwsvDo0p=w7SsZ$fESur>(8ThEei?S963iJ`n*7~ zAf=bnJdaz$c*}S1vm~yeFQXatW6d<@j`d;A}!mLGm`JDtqykM(GvIT-vAl0jw~%?32Awy*{2l zVJ5OXnY%0fo4U_maHjv!KuAnY6a)~7I!^iA9J*slwOj4+_&B~J{nxK_VmAC^kH6aw z5v~dfMaWRg&{fvv#Hf%83eXhE*7~BlwdFXv10%2a8&m&0ZyWgbpL2e~)KTxv*4EZ_ zKi1$ptuzT%F=Em@-QD}$)Q_iFX8cIiSn9*yT@@)TF12}EaxC6Q0r*=Al+&ff#o%ua z-;~{v6?EBPpdz4gIJwXA`aTV^^0%p#r|SpLIQr++@I?y|!+K}0sVh(w#YaXWeTY=E zs*QgU`;XZKK}-LVrI{>I!bV`3NlwfdS#j6n^YoIqelF_<+pw$uQUZ5Da;(C1~9 zey9ZYPhEMJ!1O8e)7R^xbQ5+EBQv!~Q+&F{Sa#%5F;5H(+zOA!*T_^~p>x8c&EaW? z?ja%+Zxy`n5PiFte8ty%Ab@r>B*~fLucdmV!ieTrItB#Q2Es<2lMmTPqw1i~d?enY zk>tZp++mR+r0`=l|E4%IwXUGx#D!73H-<<^2Oe1^&i-J6!o(cGx&Cq0AvcF;gPhwE zbCjNDhx4MqTaPh%Q!2_Icb@e#wzNG_Ys!>887=xJ>@=>N6L-o2Wf--@G*U@>lz!8d5zJn4$pHXYlupGs2rw2pWSEP>~kVobZ8Ful1ubp*17dQ8{Lnb!YGi@ zwqC1?Lq46oq@s_mFELeC)s%D%#OyZXbfJw?CTIu`DZ#EKFf+z$bUZvlozs2BLPShk zYy{JWZo_MT1zt;LE3x8yVQxuQV zc+*{)%MTFFan5a4EcxbUKQ+VVQ8!?5_y40XcDq0Ckl#mF= zkcwe^^$^y?UvDbDDWuL)IpYRg$ zjA6GCT67J6XXeP$d&Z;V72A$lWxL4l9lMQD_abC^QB?gFG9T?_s6PV@*(Tr5ftX5# z?&-SL(-zzRq&?T{Bx$!a)8~!wy=S zC}UjoNvN{&9iT`W`+A>^mCB?qT-iTP1+R9l+tav;h;LRaZ?`#jZRgZ{p!j2_RpisS zQSAJ0dmu@^qfU2!VDuvvQb-h;k!C=Q%1`Q&XA8ZL3a7I&W3`u0Nzy>Um!l>9p3xZ| zjclz2VU-a!Sz6DjBAK!5hNN*+^CQ4Ng=1kn8B1x(=$TEB)Pesd>*qv`X7@zE%zNC@ z49r%>hwsdFM!)|bTVEYkbr*CiAzg31q2l7?hfg0 zk?w}OdB5*I_n+(YnCJM59W!gz%w97xW4FSYvavwIY@i*Rr5C^cvLanpS6mQ@W^m`V z%HlxJ*Or#-J|v;%9>ZH`IsUZ$nwJz8FN8bB$FFN^bE7TRUU(k{h!`%or$1iZCJp-S z*%xgRt+-{0Rf2^QJWy)v>?tQN-(|8MnDSJO-X>0N2Sf38eGOOO^&89keLsUm}fJP$I0vY9eE z^Ib!1TqQbK_PidR^J>=MB;^DcM118<~s~}PT}%JL1};EPqy!U*pastsJ`k> zbY7s(h@&a+gcSL(o#Z_psSFy_MQ3<^U!ViF#H})b7m*h&ha77V&|+K;XAZjGD^fHSJ?&^AmO3#gi0- zy%Tatf88@Yy8k9ZamdV2T7Xfxe6+(gN3{PzlEM1LcpsB)Z{Q9dyJGYht?%3SOaeMy zp{%0N&7n{Ejo~`q`nB511#9dV6(__H-#D|59Z(&f@KqlFlJq269@&7>76dlM?9EqF zD)zk??00v1HRp(oTywZ$Yr|GAkdSyCm61!(k)Ep|T>UFm_jHTP+n!DK->n>r!En{M zImUmO%v%OOiKKXFxJ1LDdVsLVd|h zBsbXCGp#9!dFTk*6;2O^l*3i6>o2^;dHfMbJryk;g7X^(i|f1e4_%lym2}s4Qq|CV zL^!VLiN7M%VM?-eTZ;{eC*!)Xa31yZaqL-+PFb2)Oh^Tt#_Yl6E;-L0K1ILh&37bA zu!xqylMxYP568~Ytd59V8;=M&3tcY}s>zzd(znlK(-d1UL0=t>xPsc?kizm&yqQ5#2PB<9 zvPery1pDHO`{4ctlpT#={>0%s<&QY#=U`XpGUYO1zu@Bk9)u98_f5cgi{{~njL`&1 zY@U4^F<&^Pqf>RXF7~mN?siG3VEgB3Z5LrHI3q)mIN^#YJ+jX`M^3{R82PeDC1v09 zn??Qm_~1K_Dv5171u-?X4lQj={jHNC!>bulqHR_k(;fMv!SS$E3q-{7F}EC$E-T*! zIVz~U>0MH#&})9Pe2my&xDbyWlea3ACVt;!cJe>!cpsW3DmCrhj~G=)4d4@0zZW~W z#6yHFz)}f{hFXG(>XY__4SNa*u0g1XZFZD;8Y2r&UM@ILS;nSXSM8s&?3=DTTiKZQ zJ)J|B9F|!fsddU=npzuBv4c}e+C~}f{F-IOD)?AslNS#Ki|Q2`g^Dn}mT9srJ6@W0 z?;3acC_?u9F~;Kw2%YpNBH5VD;^a*arXM5xZYSVno@2cpFv=refpyT`AP7c;ApgC+ z_1jvtKJFc3=h4gCkQMdH!5b>W2!ZO$ z(MPS-qGasSUu|ifTNW8uXy1e>b8<3`e$0(wyG!NQBjIAG^<*o%g{s`&j(FPd!p+@; zjbSR&+b;E8`dw9gPk$Fuj(6A7@|X-7e2?k6q=y>jL>>}U{op1+_%AV=2kA;}-J5mc zKfe30 zE4aqsl0&WRu+(ak6=#;m9!94)a2+|_F&E?Jt-q@JOlcxQUZHE3!&&QQJ;{mTi>ak> z_~@i(z{Laq!R^Cqm{W;<;Wu64mTQCM5Mr%3!-Yt~kqdsS-p>gL9Lt#a_^0P*$<+f>#Z74m`) z*4Cp`xgIl)o>cv1BxP8jsDOX=k1~H^oV*%wAvWLmi~}v-YY5(N$fW=LX28TEMreEI zTY{y5<>{{YEhxuur!3?Q+oWnMKKWmlopJPZmi)0WgM6eEB_+*9PgpP=J*0v!^;@H^ zSGuYC4^lc2jYxFJa*a+(#U`GeY;&lLXA84!>);`#?@Co(u4P9IPW zSegru-ud3iymFE^Y@|RiYc#B)AEBpS3~oJSA&ueW=I%UR?|C}sd3=2QwX}bqX<@$i zBBrAHUW(ybvAKUiz$0PLJIMgk`x%L>I_v2Hj6sMcYC6|1!45eGTK=`Q@1yiz{-(TN z4vo^)xx@%_je!(bdC9LBUX6)0Gquf=&!ms#}%yH4S)&KolsWV`!q&q-fcrB2MEoPsd z@qcRpX#3EtyQ1X!7(bhDF)n#KbMd$>!bW1HYXqEJV*2*z1s1zkr`+M&^am|7c6DDz zK)ZAVJ~qFIT(2$Nzhjh1Z+&w5eVMn2T)vl*zE?wGjOn~lUK+*mM)6AC-?M6UunB`Y zjqgQ#A9mA-`yZLu?#~pVy4*Kx+Ew>oB$7QaqE%0t{MPff%}vqyPSy|a>4cW}+ttUCP&k*5B-}l2F z@>;H`Nhr`|sxi^QEn6M`>o6LdmN@-nB69W=yEw#>DO(*=ATCTJ?#t0-dtaQf)nizFS! zlyCvJlph;6@GrXSee9=NmvrITE>HXy|5pC?O`L5G(kz!dzV3RV;#;3qTK18@1&SOo z@KgChxFsaLVuUu=H@1A%Kes-FgAuS=kWD!L-`Q2IEEs4jP(>=mTys|5S&x(<2=|Lq_7XGh)aA3Y;j9+}_ z92a8ILeKI7UlwsODiS^x34usBRKB-r&}u-!I})Kzo0hjJTq14os`xY^BO7LTq4Z$! zmdwGyDdgL^EfbDNEnw5VzISw*vUgpmd8;*QP9 zV-|6c>60u8#hxUeV5KgGoHyba4uX#)u|cJdrU*}F(b&Xt|eK33#;1BIM@*%Dld^rutY%d z<-rv*ZTw*6kW$Zo(NboN)p_pg_<$B?B$`@zVmuv*?$l_$;2Y_7>QT3HRmY;?m5+~t z3Fs*wXMUJqH%#9J26sO84dRP1_zv%TN2T#;fBr~LY>r9;_a{Q}dTv~OK^c$J2nLYD z)RZZP0HK?!z)jp05i*kvb+brDU-51-ro0^nq))?m(pR)}3zav4rU>DIJbvW42 z@$#~btyn6W@8Cb|{PuFyW;&d7yVJ`3a7qKpC;dzkgH-x(^n8ys1f3~2K6eSE*A5r@ z+)bQ@)!jz49pXZK(C`}{HbCpoglnetO$ zK*#X|=C0ovIgcLF%Aa(4de7fy?8S(bLAW@e!1;=B^iA^4_5Bhr@U( zdk{C-j3%m?DW_LmNxJ`6#spW7@S5sh++9S8QqSx7||hyRS!H;jq2W!qa#H z!xITgqceKoda#|upBv8k?DQqL<>4S`qqvrMGgh~sFRJc)nA(<5^N7(BU&^bZDo-^e zQK987N1QW_+mb|m;dj{1|5og~ukbNeNtwt()6HxPZ}gEpe0D2pUETxorFIbp*wW$)2C zCO{)pq8U@yO1#MZ+}Oetnv$(LKf!mfxRHOS{N(snfHf-Monv_J=Vw?R!gyQ4tsa`_ zBlds0zmv+rMi?~a2LtdhVRG5g^6)bCUk`ZL`*tmbe{^qfnf-Herw~=jTOKAZKn;7} z8WwG;o^bb?e9#mg&JMdPYu9XPX#(ZY) zG-uh0$l!9+s~z-bpiSePjD(=#lUZMP1@BYH^u4DcSbm^DZOC06EmzV+m~L4>9cK@2RyX?m9<*G#4(V)~Dha@q(z}VA#P;KOS*d?pAnLVE>h| z;-FRHE($|jUg67vY|m=6HsAO4G@!C4zaKY`(Jnaj-r?w^7Z^y*-+V2avAd};F*epVTWy7k2%nsqIu0}# z-Q0M_-X))%xti;4UC@}YXL8iKwg#(>9lPaj9#yi^%puwney*rYg{X*>D2J#){P?!@ z;Gr8uCVT<;Bu-tOT$~#W`O5Z|l)!fz)J07UaXe9OjhR0{%N;H%O$+PyQr=1B$Fche&yOd2U`_b#U5QULF-l zG{`4j36qq8qT~I?VT0&@+bNXyWDg`pTOgO5ewF+e68Hc)lS|g)sME;MG43uoVH`)j zy+`aI6+DE*u&-g`L~zPXOGIBG1oPymWG){TH#g^oh9Yrta;j>j01slWOAk7eWDSKv z<)e`kX;a1{cFuBVhl;rLEvf2`Y=qSlf>H*!X^(gCGPc4yK7HjsDXP3HXAdjAs78II zn`Cv=`udo=ejz$>5=!WM`4M{Xc4*8XiyxJrfz85#9Fdk0#+W0Mhzad#`_e#Z>gY(j z&6>Mqnn{-Cfu@lnMb4Bq+spcjI{BXzTw;Hb#_b(FuT$6eiU(AUB+O`a45z`5#v~0F zlT|hv{z4QPD(YkKxksKeKemapGI?5M9vYv++)Aj#)n}O~QlK^3_NI6%!+AYv z5ju6uySY;4<9WPrv3)#njUwM&g?+HD)9HBN%iAh366yLLNx4djis#ye>%eiOmVv=r zjuwkCy6Bpm_qeXM^oHa+2XYCryl#f zi|$=3)%?;-*Qo&mhk`#>?%WYcn#=tWV|hx(Pyw6$mRn1(R*Ec?Nph6AD=isKC|)Nj4l~ zXg~WmcSi>0p)3sC6%=ei|^TkZqZ+0xkCs^Bw^oS=}qakuu1ae3+Bn5 zL=LzFG0OAeQdFYFkq%b8H~hUu_1O_44~abA!j9@SHM2ry@JH*~GXzs^iQJF0HwkmU zRhwx-_w4y*HMSGcbnqCeYV#8qewVM;rAlE`Juk*ZcXiQvKij0~y1z9uH<=ybR`60L zJHjhj$?Z5}%b6djEB= z^GD-aDhlH%MnSz^oW0H|u`q^r@RPUZ}3{E8%C~ zM^g85DH6N`>EY&}5R2Ive6gDNjHEad4Z4cvarxoF!epg9@u7(Q#w!VIti^CzZ8}}G zx^9W3pESx23+;-ML>WHYxM*fggE@RTMoo1=+dfttjP!ev1iCzqA+dlmV%s|qpEfav z4h|N+>Wwh*cCx7QwkEOi)$0lw?t&JQUFQP>?ds}k%$vo?%GzyN{rM#t&~bW_T5{xn zs_!P_ueNmdBQ!!mKtOPHcTddDhKY;ARWZ{JaK3*18f+lb|KQ58SY%!&9$0fejN0_c zET-^pbbFbNq$~xd89$}lKy^RA8^gnQqSY9O9vJXM*qEl@r^u!c3E>{BK9n@lhR$M_ ztr>(u311&0{)YB-T8(V9?PchXWa|%>hDCXaG8Vq_%1_>ft*k~we`NZd@eUdt8eRIb zw=|H2K3DiRa#&05xzO)1ywQkiQjVWIqceOk`)mSv_}8V7h``d5&&=ggoM?w@w|=L^ z_?<81a6j$7%ann=WpegIE|o9%Wkt+_R)tYdNS(jasK|jE@$L)*^Rt=kxZ6*;;CIP- z>M?|asjk|0T}DS`ba))SxCLkL@Vjhj=?MRV$|8`ii<6wL%e6gH75Zkn2W91FpdVQht)O$ssKwNp;{6~c zrp+?z&EZU&VWv!Y`I7z>2M5QDiq>Ze0SP$B_)lM?fAynvgMSpp3(>y{(M5|>E_2mZ zHlDv4q>qS`Kn;tKDFEE0uP7vJm>RA@M!mtd<|>GpA}5%7?%y#+M)NQn=@4=1nq+82;K#?4dE#S4LY#{F z>_hL5B4bj)w1IyNoyeB67``xfL8vX4u33so_~EL&HHRBQ7wWY`yE*-huW6y`PTrAR ztoj}-e#f%B0;r;tR#&w84}~qQHO}G0E2)F40f&6JFnXL=l+>Yh?8l$E#)*=qgN?i2 z{`{q)ZtX3zI9NlJw9fi2#w^WA3VkJT|KVX{gb*+4=(F5Tq43oTj3$D(=c=S>OXLBH zN7cz;`7_?8`RcXptJsle-#n2nADo_R3wNol*4tc7IIuxe*HF|nS!>fAhJVQgKeJF% zcwj+Te(BVpzh+-Kq+`3Ln8xv(8L>GN@Xw;bO_u(^= zw~ocOG8R&KeTk;$fgM0r=r!i1@1~1MO$X`@CW_bVS;1+P4)&KUEOcya*kwFL)|>?8 zyC=GIZ|&C~OP%GQSBMrVPKQP!;Sd#LxZ_A^@9p{1@GU6;Cc}>c%U4VK?TqLh3Y@n$ zw~UnJbOmihkq{1|%6ILuk&+TA3+Fw{;$a&8@})&7UVXyvWq5S+@sAr#X$WP*rF}aj zn$<=-Yxi-(d%|Th>-OinFqFPHRff|S=yP~bD?|I#2wVN7B)8NaUR%WYZOI*Y6(I1N8+zPM0I$JaLOWAPZF{~VkI zK?_Oeu_2_9U?+xtD=&{dIdO(ffWv%VU7ILram35V6fM3tN=$^zCZ5+Wwp`%c<*Vs< zhr{iZG$Zq&8bkdzNqRzIbkjzXYMKd!IT-hLBOom@uLiGa*!|nVz<*>DfpsOn$NjY5 zY^3V=lj9;$i?qqe>D^Y!YyJBn2X#sk^GRBj9}8uh-?+U0FuCWjf)jw>lh zZmFYTQz*yuJ-3Ly^u0jHB_3QIC*hLs@=iyizZLS@j40k2(G_#C4{qP8`2dmDM~1!qE4rY~&54u5 zR{JyTkX2yR{;m^XH?OIsH8wXFxqQfw z)W=Q`yS~DYRb$7cwQ3G$5^H5P?=5#{qV8oPvSF!O(X0U4WMJjMz_`gD@AUHvh+sJx zNChWtB0FEX`cietrW~mWkgp@-dTaWtFs&EdiDBmDU^^NR(Y4nB^#|u(wl}A@`0yUi zy{Wvyu!xjCV=RpL<~!$Zy|<%9eI+QJ=qR1ns^V=!o%gMxIn@r<}4| zKT|W*#}U501=%LHn|~Jzc_mjx)oU`>ErZgP6b!y~hCn($wGx$}03RE(?-;GK0X^mK zud$0olfgv{fOT&XOEa^{Yj@X(KbNNq^6#$U(UoDBU_!B{4;q3rzr&?mnc>BfMa*Y} zSBrJBwGCsOcw6wRyT(pAl7xn>j~c6H|H!G$lR{j~$`(gdpw73I&TO8%=nksx-fFTv)7c=X3 z<_pBQieBJ|67Q`}ky)x~#VfYrSDBtqpNd(Nh(2a9Fg6|!EO^V45c0m(n(%WyozUZ^=PE%$50Y zNJ{aT#^5>E7}T((=?Efx`q8(vG&-r$^hqY(-sELGB6--5RVnfwahn*s`E1Tt3ih~; zcD7k@YMm#E<`%YOn*N%c&)(VUxuip&325+O`1}c|Q;1g6srsmDYhq$(GGPf8^Srv+ z(6~U9p2|wQSTkJ34cv8^gd#IE8sA#fDv!bW`73s~)<^#DzT8l9Ep^AE6`+xsnkQEr6Y#R>p zZT4JFt1E#9*4%nMUNd;P<>bw`oKmHTM@1|io{l>Bc3$W%Z(x^yrCEtRSz3SW9gv({ zxNP+(Q6}BfKSq2!ySlxyRHC(;@9jkKxxM^RbkErQu;@2(dCLF&w;5xcXnl4m-R1Ej z$@8-^zYL?-=j`2OmyYsveWlx!YfZ=W%_l4S&_T8O3k-I4c4f;$o4cF77tGCvao6hB z@edEL&+HBv)FJEU{IMNZSNt@A)|ufbJFc0>$4<-xKr=Jkkp}GNde#~{7fXxL+FC)$ z?iw|64gvRgpO5Yn_frbZH=BnpHdrgLA0;*nB?2Bt_jI$G9d3$O3mQet$5 zacm4dgVW{_7EW^xHD17fm_B`YkSE1N-Vfa6kB-|j>zYuh&PTAnNkUF1{5`uoCOfip z9}SzEd637txENhp66e36Yd*QpI>NSJqx3z)iOaZj>PeY6Dn{0MnD<+cdHvjbCktC0 zLR6pDH0ZFT-{X1DJzf*j)nno4I5tOZHa|nou3iOl@A-jXxFT=UPxxB#!%bMmABjd! z(lQsK<0j-5DU05Aj9({{4!+2~_w0{5BhB6Z`^8`DBlI5rzW2^fXA{h+c%0rcsyiNL8*ADwzDpB*M9L5oQA*Pnw}^>bB-d(J!uAc= z6#bCiayKQ4TVcC3zQq)I!h2kznXLPJsV8V1`!p)=p5?KzI*bF{u_h(!@P^O{9eX=> zxBgw(v7_MxYi)Vd6lodH+XnE5Z#-{pg3W$@b6SL8#VW7ylOkeo1AUPYP$*ne1CqH? zfWrfbsEkKrv!3;>&CjW9dDc&-rzH1DqSORLzFmLyelvAo>g2MbMd9|W zLS3{jl8-l*V>kC(35@t15!&uHm&2UpR6Ur1d;@)=$St3Mtz;P+EE3E8SBjNn6_)ls zaG{}^*0kr91kCfJN*ek8i;FL=O1Di=-Vqcva>J&!%Iq+ZPEf8{euiden>ss-*$nj& zzj$HM5sDcd7dHWjS19i1hK8)g1`iE2HF1y-3N>-0c^YLIo1u(CCj&==m?_dB$Xgq+ zuX;a%`_pA?#JvAZjnF9C3|+}l(k6|LHKp~rvaH|E>DJk$PtpC>AQ^p!2_>_BPJe0B z+U3S+>$T0Oz4uf1s*8>DwF$Ux!^+y)(;JneCnRc@>*&j>+AKKolucrNyEs@`@ni{T zW@-Fwk`7iV%;`_qlq#V7w;pE9RTq|?vefTi^=cD>#|%AMzJ&=l*++WVNP z%!!hgmb{@XzV*}VuN4)tjMB2Q$U#9tf2OB3)YSul8G3rD?ha`%VWZ=%uoWwSdJnhuF(^^{JxwyYI{I zPegta6n-Q7RQ!Ky0cxu<|IXy>k+RT*(UO*5^FIE%V{$)N!hsQ}>Ef@8c@RBRsY_u* z>N#vxxb&1nBKN6RGTGnEUB1(Cr2UADmFTXt=GCQ{oR)Y5XYuCP!*4>)fP!N91*EOR zO9fF%3{i?7`~nf@m%#~UPl(5+3q+w9RuRdI1C`k1J#B}V1<7mX;V8N^?HOB0K{G5Y zGimzmJDdUn5mTh7NY}!O#S`Y7B$_XS!=k!pF7|6jM9|Bl!l`IHuMoj4tQly`Bftd( zA=HXeSGM`w`fe<>TC%qdKSuaSzFzR&ZTdqmh9iapkwSzU_nG&)A6g!(^MkiC zO$T5VzGaFAE`GP_XQPu`jIx;rJ zm!_r{WjsP22U3jEAkSN5O+y1q313-lqjSH)ESwo%|F zTJzE*RT1@vW?Cse^qSdohJHp|4VSv+Zr|}Bp%`MHsg6OV6_<9xhhTAGq7M*JDLz}p z3CP#PqG>l-|2RA2TpPp`tOD_Y-ZN z{Fd+;L;^qUq!1^#1**F=Er2mxZ?0IqSc%2Y12rp6?}JBwsEFETT1fnoRM`divM}g`eQ39H*mri?s_F) z8oWdH`xA5c>tJS0Mo3w0mQ`k>RPbQB z*~-#8+@-}~K)!-LjkSv%g9ZU1xziN?a5--IaC?102jk;~tt(TP8jVD1_)1#`gq|oI zI8WL*;r0m?(U`Cf=7F_gp~H>bj|XiXy3{nD{D-Krs8s$(CHi3jx%=cl`ZMy^c?A%fh3TwxFOA%MWhUkJnZeCrfkt$p*SZ zJ$35{jh8nLUM_Z&C0-s!`!>ZKOHI*__gOqt*3XWIERrJ(VrH+su8yB5QSRZ5b*Yw;?dI`Es0;V4$AmJdfL2UWZhzDWfOgHR*l z7d_K8FIxIc%9FZ@e_L}X!9bv@EU5Is1$oTEH*p8mnSrN`pu&_iGS1PqoS00K%ojG@ z?Q6sH*`HDpRd@A__;-vDB5zrndL{{N0W8TvD)j~e!mpw@Y>mxws=IENm1vjkT1n$` zb?>4z6s3XQm**!G3BT`AD7QHsT#5CjaL}%CVx9J#4v0Na%Jia2T7UJ_SqV!z4xxD$ zIo8t|5bg8x$4aJ}5Cc8J!Yh6f${t46trcuvf8|2{+^78xVjDMuWOeXzNqx`Tw@;lO z?=M}z&-Jz-Wb8+)_8!#ssR@Wuvoad8of zi>h7%RJfEgP$)#irH&kB!htzsgsi&aTHt<7A0C_`izcH-`jmogsRH64X61;UGgY1{xOVbZljMYFN7!EQ6feCI0)Qb^Dw z`DFF~ELA?e|B)XRxhx7pRPTuE55IT##oUo?kb(w567;R4JdF5HqM!SXDjO5VfuQZ$263o3sAoIf8ZujM_5ng0KrrPRaax+5L7wexXQzRO zQ=_Y3Ot`Twq16=eK<)>mh6@EkZUSTH&W+}C;oEMef8}2OBwpmaH7zkjOLI!61aCN5 z*q#j&>uEO+H6VB`d>q0Uzn%Bm&9T!oaN<#V4vDqG-Z$MjTXB?tAfD4MJ`*5miyO^mRMch79Fd`Rp zUN&!_y=B59Cg=3>FJx&GfZ_5eJ{) zlH81cpTR3Hb6sq7LZhCa!jx7L-A=@R4|~QkHJL&zGQCM3xkikC{i1%mzS_9WkA;jy zZft8Cy|!iw`c_yaMF3Y*BqOD&u)5kv-)=$^s2+$b{tP0{81hUq=kR#0;S~vUeJB}g zmbkLm>S{hcO*o1Iu>U}3cX#&>X7s#~uyDt!WOd+{r`?)R1&vT>)~K1>eZDyT%CL`{s5 z{~sCw!yTznfV0Z324n~GsDPvRluYwVTUB!4r`C5*Hv%O$K6TO`>3g%Ciif|rd>$w@ z2;U$|RKFYY%fUQs=@}gV)`!mTTH%-FmeVBnjS5T-{L}oz%5kn&y`O4ojKmxw7|crz z%OJ7P>#@J9qr(q2xY++kuRSZJyf0x-P62v!hDL-1Vz`%?^Shy@M7&R3U_Pr@R)P{R ze8P`RVxYmt#|Ok6P%#IZjQkKdNcO+?xxp!Mno^aZi^j?szX) zdZm;u`FTJ97Ow@<*D1Z7ndy*R4}#O};69%n6T>CpL7l}MeXTrW&q;Q>*FsGnGVH_c5wFedvq@?Z_MqF z#IXw8tw*#Yn|2!$p|pmG#QgH&`uXc=l>_7aDS2(L|A*hH!jj!CjZR(stL>E4b$$@Y zujzd&(RUO&msehF)z0<)i0*b;{<(y`HAKut$`F-`xDXL^ zJZuVKSk9BdFM?5uk?9z-tp#d=fwF~(ZLYbSJd&rkY2yfbrgoMOovEFTCYfIh0AZBD+Qi`7hxE>V{{egOC&~_<< z*Nzkj?CGRFJ(Vu_D9Li3*TBF4WJ{k4VU(4zfMW^;*UPRF!1nQi#~alfYlj*VwHCc*V`l9iNmG6z<6RG}M1kLakR%>t2H4R>46S*zVxO2gr_~ ze=||9Sz^s2?)dNH&Eco#Im+zYsTI!FlBnWv-i+n7_pe~W=QxjdNteeNNL~|GTur6H z?vUh{QjEZvc4&s9$fk@mBsDDj_BzeTJ7yK>Bq}yp($d$(>*TcMyf*-?$Jp4RtN}WS zT^@@{c81}oe69Uqe!XS|R7jeJj|Amey3us);Ng?i!SEi^J_QFf?Z@DO#ijU@5JRl5 zD`6qH;`|aM^nkOOz>(r-&K8P!#kxGKIC7U*GYx3bm_NU`M;n%#Xaz*fjV}JkDz?neFD|OAMkwCCeS5UBdgpYsB3@%V z+rCieB9L=)drR`-1yFNa1DG3qu(uZlj3vP28TdEHamS)JFKI;|I4^eXqb*fKaIb0e7yJGAu9#)vOy?xP_FNhuN>5+YX z^0xEzxDk$q8N@3?#lFLWkDK1>TVBB|B1AO~%fm%VS#tDAuso5RfB-lUX@Nq8!Q9tb zI?Uag7`C$#UCR$FwX9E1BFL8(6|!4@c79%IzbN?O97;(=Mf5l(x6gFWAEL)l0?%$jXZ2o-`Lo%4@B)&;u8s<-d{%C&E_RE9|(3otovAyuFTT9oW%ep#QC zz22;TjX!DOXqM)3xQO0+N=9DeKp=YcNU@%X{J`}_%kiK4L+s;GHDBGD;_`NPD9U!j z(}2#+5tES>)YWAev-g4)T7M4@i$R?mU=NbjvwM$@*q=Xt4%GUTr)(;kLZ48RuqRk2 zX9MP(doz_mEP6G-R6G@AKMfY3nm|$Z$;k;0q)0-lR#hOj1GJ!=sj@&%kP8NtbHJ>2 zcgK){F>h~|0GAIBcQnL{twf&vVag);yi8v^sRP2=@p@jT;h5@|OIG~is~N%6{N~m# zcI&+KwwMvltXZ>S%A*@!&s2#HSr>a>>Z(~9xd?>#4%g{0%V=oe0U-5`TLFp-dKR(+ zhGS>Y4~D=Y_Cmd<+>MD5sNfap3}^=IY@=OY>)P6128Ea_s-`;lm8*kN4IFbSCeL zeIW!k3e~rwA|%kR1_CG$`K;`0bD#!kJ6{t6@{Zug-+%w|L9qA)+2o-Az3S0X24G2) zvjC)favg8p5T&G~fVru!uYU)40Wi3p!m;6wCS;Za6e>2OIU@C5a zTC|o=<^Sx-KdgO`ogJ$Sag;~Dh9K;z32L)ig=F=-=8CNQ8t#~mLdigr?IV%;6&hu4 zySC5prn06W^;;I28ad!G+QSwG5e>+e%q9-h#84QR-sMco#rAO&eDwO*g@hGbNNw2O z_+_YPw!p{s?JY$E65J~}-F3Rbm#8kmLjh%3IxTKQ>!Er1bGkqOKsoq@zyb&Q1E6aC zHlF)upC&~fx88W0EP>}5eINDq0jv9{yrT!))V6h zJY1^97$^W;YWfJ=Gaw6;rKEb*6`ZK6?6QD+1`rk#6BF!8=7Iczf*_yU3pE{`aP>$$ z$bZ;%n*jKvfzc@@BdfqiTp^Gu^`Gxoit{ffOXN91QHYE7@+|hj>4;d^+sq{PN623_9Fz_%^a`Ja!5*5w6 zyYr6a#0UZhu#_YI2Su_H-rwJUnXvx%@88$*a!XUwK&33+vH5vLAgBxUHUINDo>$=r z$XHY~+&nyLZ{I3#lUfX$8?B_geog#Zpw8umdf&BS#fB55tD|{B-I1mZ<4GzwAk6yx+qmfoC`stOgyy`~KkN zT`l7aTfCBAv@u!!)J|M#2;I*0g*zRl<0e4*Yi-o;qWJNv_4_aE@MdNoXGMt%pgZff z;ZihGG?3UdIVj+&voRp6)$GTBLHYx{A7FqeSjC=@olFEsyo95GnYcdPPO>0Iq5-Ew z!BQm%M+r4y2O|VtNk1GcUl0n@4m?F?U{3n``+leYa8xT~6eE#}1 zJTMR*1cOg61GdH!UeidGgX+{eQqzDA3+9u6`F`^zEZmqFi9r%@9Q{(k%unQPLRQwm zW|~p#(+9c}#VVk#nBwVnz&Jml=TnvVswppg-k&SA`nO>PJ(CC zpLO;vlr9AUehXW45N&H%FZ4dgw$B$fUD4d)`gvO1c&3a>$2a_Wf5?g?c3EL!FJt2d z#$<*2W%P0=m2EX zr$C>SUepI*ucV}8dXYR+3UFC7b>)g6QYqg}gwX?>1fU5LGBW7Uu&|*2-_V0Wivy;* zT`f>;so(Yy-PL}_+wbA4LT^CJ0(F({m z7iWn~f6cC|JortsQC;9~uo98(vW1R4vxkS3dF>a%ck9Xd9sP52Df$Nnj&JX?(ctu1 zDQIbFqobp7qPkGRI^LC$kdvD`J0}8uV;LmW7|F=VJ8SILj`;xz4WsV>wS^h5P+-qM zmx&IN1m$89Vq#!I0R5%AzP=6u8Bj$N-m@N#!axJ&m_h^(K;95C zLcCCP+o_e=^mxw-6f24Z?P547ypd`d~l&Itz`klbpEQ;`8o>ctL$C96Hf79v{YV5jeo~5RC z1zf=4RagqB9ipuKcxmr`umDaXpiUGinT_PW{-0ryD|`xwB0v%sc;)tvWF;8vhs6LP znAmrAc69h?m2SJ*7yI+FK%4s5le`cgjNlW|{r@~rTwJx+spa?%Hwz03B0Sua?RvPs zwSNDehA;~Bij;Z^u>jlTA|cg_3x?P``@j2aXZd_E~^ zwnMg??w-+QlCPSI0VlC~X!OM?=qpyH@1 z1XL(LxeuRnZ)@RVBfo1rv(y$j;a7QGpR_OZ*7|*I^7Aw;ytO^BF+k!kX>w^P)0)7YFu~k-hJRSM!-zimB{)C)nIe^$SD;ZNNTnzgQ(+rHw2aVW_!b zakteIzbu8bTzs8Ip41`L>eklvJeb+THJw68QjJN5iktxl=l@XmR$*~{O}=mv65QQ_ zySqyuxVyVcaBXNHxVw9TySoN=NN{)P;1&Xe2F}j^{oXTk&dkksGtYCb_U`U2tJkVp zRrM=#Wq>$8R8c{QfG9!+hJG$ZARI~zSi9TXTR^-1iy(qOgha~%ej`9RaQE;?6c~+rVCa={#HTgAUL*tBiZ|7Am!q z6=+sEy;uQ}2)MMrrkF7R@`7Yqv|cBsqJja?_jMYrP=E{^V5_mSv!kP@7yAb_`Rh~k zC&d6}2t5v6L{@?i?=3puFJT>{<~34Y`dC|U=JuZ8!&~@}vD-%%-$w!$atdP8Qhq)G zIX;5E>xam}V0RRNdtW6Ftbcv|yE97w1Ab_!p9bgSX;+z<@WD_-jGGVRGplwemr7>TV7^hiQ~ek#re`)9p4RF!zKNLodwW*T!x*ZoXUYs?5r?R7GAg z^|pZ@fyav0i9to(R>oEfIDmnQE@(pMUJT+qaR4yo6d`;72sBAjDvgw&_!pE7IP%r8 zkzoPb|I=1aE%6%F4)6<^H-o_x^0WVVjzLU zVzUbb2!0h!%~0E{e^eKKQ`qwC&Wt!k(b7iWoI+Q9$NiG&CGVG^@a4|>n9nC14Ylb} zl};8`k}Ps!pgHlHb0{^d=pD9Tlv#!dmN3CiS@dNW%y_?XJ_IZU(f}I()Bv3PHymsTlBgC16rBOcGR+c@8wK0Z95)rdWm>X9 zG^CXHC4s8K*%Nk{V1uf%vx6pAp((igt{P2dEW8`4;{D891}1&90DLLw!CrpzvLt2Zm)T zJ(ze$Ii>#RqLWHXYrZ>dl}uE_KK1<6doo>OstNdXmyK?(2I~=)FozGBgEv6-08*hd zy}fS8;BR(R{2r+&2D`DVn|AYSg15n96%`eLB{1S)2r!w~Uylkv9DlbhaA8XS#oZ04 z^zXq@i5$2iHtgVWVaw!X&7=*}R1~t;Eia0!^5HhIy>k|(T=L9zW z$0u(-T0qrfL;g!OnI->VGdaRhDvk(qt`FEWR|T%SlHd>yKWq&i^=}uDcsx#Kqsgmd z^Y#}}U*|U_w{4d$(8nBp>oiVzPKUV~KjZH-%ZijFF|6Tg5w^Pr3G~Yq*!60>cVb^>lJ2UI50YCDum;C#A zP~modR6rP5SX=M8wgcn%d;fMV1~M#=-v@-nVBz6~vxi}g;eYh^7Xg5NU?AJQG0kZJ z5D>!-O&rI=#e9#Pl%4!h){2HUii%1V*oI%MlM?>6)pUCe{b$;JkUjaN@4; z{YuB1Oz}*b4A}T&^?-nSv$(sB5Ae_AqyjON9I)>)Gc)<_=XC7>ow5%Pk=BU6yYwzoErjK=`3$VibMmrIM9kYr0gvMk2jl;HRfyBq4d9U| znONf5U{Qb>qSYvS_4Vs~nJyL~7x1)v$gUUS+c$u~1>S^83>C&V2gXobRtEofz5l%w z|8a8Py$)4MN(f;Odm(rxPGi;iCZH@2>+TQ}{31tAP7a_=|JPl?z4^kvzZ3HElz?eZ za&|U|VGjupS8uk{tkABF>-)RpO2oVY_77G5kIYugD<3dKSzDw&{xR~xnqqJTSH(Zs z3NB-MV*c0B(8{slTL2CZU?#ENzgKK`J#cpJ|BK)N;XKE1!6E;UC2d%CFE^S?^RZh4 zD=}&K+tHPt$a!MkTYVn##R1dJYt0Chf&Y{2{*sXsJtJfE)8n0bxf-8r^9NfEknDHxq&{%!^u{NxGj|C;gD7gjK|Ra`eB=4a<4ur} ztG+Da{&ki(o3SbL@b6!xn8=_*x|2SxZ%L`($aDaI=*D<)XLOfLB5BW8* zk;^u;$TtGIy3ck~4J6AQ&N3rlW4 zY&{U%2%dW+>Z(%IKoQ7pH{qi?!_}0uNgcH@Y^^(W-)&ZZcPFp0)C_j%lkPTJ!QgY2 z32#CdVmdrH@+wJpdZrg1;-UOtb`OqZFW%kIPF8ZB{;4hF&xTuF1NWuI&0m9K=)nc6!Q@jTceUS`!4hs%TRPvpmxw{-Ik=AJ4Asju))*|4Oj$ocL-$c4&vvu&0iX&DUS#{yN2MHejcwBs2#hCIP z9VSGjc(Wb3`_3+FFl>ioTV8*(bya8QONR_F7U!Gy-+OJUJ+<4H5uzY3Gok^z5N@(k_|)oMwDf`kAARnuDy$HFfgI*c~r{x>#%4xDl___v;`dzaDE@ z`ZqrwDI+0%6KJA?B2 zY*7J|znmvR9&K=P$4>Q0c5iTD@ZHRPDfmeGFK#f{9Q!|R;D6z|btCb2<(LZZ8q};d z<702Q|M?ui!B3rE6D;jA!O!hN5chTVlL*)kIoS%2&3WDCPfOY0Rh{KfiVoVQGg?^h zXQq{GJdT4Qh)%s=$R*;!Pr*|RAFg=H!O-9rMaSni0LTeoSjmZrVt~V;T;=N-s1b!QB}VZ45bUO>Y3f-X7(4S4Yd4 z)c2ubbTrvDM8oiZl1COT#0(FWP+{-kVBaB!5+KfMT@5WUJ{)() zKbXlQY8w&mhX+|oTAcrO8@@&3?9w+E3NwEj-NI(MT=+@AkB;@F5T*#+Z-}-bb3Wz7 zQTCCT{6pWgX4iec{M%K7fkAyG+5#Ke6VEfV09m0_${_4Riv-O^q&;ADwLZ~}z~d%! zum*;2ZuL1}mCV)Kp&AQa;orq5So{*LDRoqU58PE)^skR#?aiIBCUv6N@2}B`!|XY= zz@!zD_L(@z6gbr2Sn3)nq#RMltoo5`(=ggt^YN2Sd5k2YEku}Pl zi+I;k`z5kD21kW)F9vheDIkxv^8T+-py;OGcl`wK>mqkeF2 z_4aFAMdCib8k6S;7AQGX@^qy;dF~=_#1As)TE%re>zS-o4DXV_7=x&}>*BisWk6KR zNbZY^@ebPU4FZEc8FZu+Iq>gywSG2DZz>847mcKw+M1~mE*Ny;C$|`zk)y0}a2k$l z3p`E)v+5|w8Q}`f4FDzfXx0@K9=sVOq)qNUf+TfMm%GC$L{Xm~b9`=r9yVD6Jl&r3 zlp@DdgS-Hy9Poz7Vre`S+{hEYp_`vL`@*D`h6WjI+%KkXw0hz&^7lif9}6{D?xNkL zeT%5W^FVI2kuyn;a($^E7tc&7$QcWG7q4cKBAXr|ua1X9oI@^gEb8r@s9rzk<~ty5 z?QEPG4RggDmLF!@{j5v zHh3oI_2eS3&U_guzf-4-x=6t}Z17d)DfXYhu(Y6IJ}if`Qhk7)fGih3rtBrC0m3Uh zG(T*vQQw|o9V#M5`pD_2o^-LN({G$3{by4_4x09H9IKNG%Xet*PHtd`Ioc(muZ8~4+# z&G!2TJuL)CJQ;$lOn)D#I3{fn^YXw_^irL#6YArpzQoqtAC;9U1PJRW-~dvQ3nIoA z*C_U8NJScP27kKp-z2$1XJowH`x8L{o}N65djI)% zbhy8l@@!kLjZ2<>!8Lp|82N5G&}RG7q~`cX;P(xA-8zBfFA|$+thx=9AEH^(*_}rf zk%@R>UO%xGsOrEX^{ju8C5xbnLr+rKG+Hc$V9Yf!@@{hF>OT$efBN*rxQ){6NW`6q_qR}}4ev-XlVEa65X{^Ayvp^lbruZD;V?EcZ$R#fd4FjWI)t?SX zcrNrr^>r)J3Sh6h(7M=FwrKo*BPKdT4bq^)Q_K(y3BVg&=@%sZv_Id|I#++QU!_ZF+7#e5v-4$JQOy<^ z^OGmUEHjzF{=|Vd7kgn7dKm9Qq_m1W?&cy6B5*g(OEe`>qszdE@3VcEI|ZW)Ucu=$ zB#Z8~Q&S_i?TM{w@6?sy`tg;*ijrN=X|Y@DvT)lET_#_vWv1PuCRQs`$GtG=b3bl$@dnWXtmGjnY!xx||jyqK8FH z#;w!4YC2j*W>qOa0_CWSJ-IcNF3+QSo|Bh@Y0k&MYs*fb_b12CRj6_kn)JkXk7DrE zNZ@x*XcR&Aa!y+4{rls*G0;>)CeN**=fS&s>aQ4y?v7g-MY=$r8^QRk3ZqUXcjkwb zQe%BZ5z$(k8Qv}~JJ@>OjiCtg_jDtwJ6y+})aiG?=(9ytkvejh5|oYna~FK0ADFui zh}cZ_8{MNfM%|Jji36m?EO@_<`YY&@?``x+Xw}@oZm^da^*Che@N|^nQ)zWz zon~r{_QIb6l zDL49X%Niw#`yAQkNPPZ+(;xVlyH=o5tQhMeuO@#DZFGSMr8y=pB(ZsF1H37t_(F*6AyXypUk#dAsE-LDg9_~rqk|0fc$oRb9%5f zc0uW3h-tbYsM;+~6%JyUEt6#;H?Fbv#gMS5It*U$z~?LA`3fcMzi1uwrbO_7348-& zqKJLUZQ-ifK7i;Lzn(apwloIQvN$C44GuxG#kFSg#vZ8+L6>F|8Lhvs^05y}8$27s z4#*NbWGQ; zO-J?KJaY(L+H#p+%N~$Dqg%d-%|B_C zYwnbBl-KGa=HnAcw(dhF;v&8YnT&i@HT(^oVL zS?`S3gD9i!_cOzFi98X4!`#0R!@WbjJnK}?6!3cY5TaH)MU6~6pKE6dWqOH|?8@3N zeN=?Z9S2u;rQg4M>{$595w63*!?Itz#Pj{2#BGJc6EiwWK9KNNJH(U4PA~Ce6wjp9 ze$#>b@oeNR>9mP3B#{PM2d%%`{qED42ni-lqR{ieiV#!{#eJAZkP3_eZA<{yg2Yw&Ghijx0|!p42^gIMmTqcZ z`rSb^xznYrnl+2lF#vV}ScP$*8V`%{Xi@@$!cR{Udt;)Z>vu%$b+T{;d}c3M1Pb0B zM8b|I%H!FE5AImr6y`^MH+#Qqb`KJMX*zMbO#8E(EoC|7`KBI#hCX1yLiePmEA#_N z*Jhy7!#9eB6K$PaVx!JK zvS6f~5uS8WBXpoT2Yf`f*!937s?p?++baA-ZuuH%zGw@^Yqr~zSekkY6hylzyPMz^ z9)GZVe5Bxegka^cDmy%FP0+Qc-eKTD`P9ip-hW{In9d3EAAZ9T^+kM+v;FFiy{+0x zu&6H)r;%@7zm~Sj!wFEtcrL>p&TwYsbMzKNFg58p)$@8bAAQn*kgCgkwPx;$zNeZR zE|tK1Z14*|P%YxWDE;D?;rQkeF)>skA|$PAvH!>KsDEt(6cRFC&K%c7myqxvDypyF zoeN^4P0G=FqeeF%17{WD5BR~rP1^7~d2B+(B4ErU^fw_?*jl+rkH*us&B*jyEZbAs z`Fr)MXq+y?W}XMBrquu0u=u}hkpEfL1bExM{Hfu1vc5eh(ASR_0t>N$4WAZk4JM`y zNW)LJS|WaU#eHq^4;ThUeg9)1LKc*`M(P{f9r~K=gOVM`mcsjq#tRSuhYJN3X(z_b z4n7IkmOeCXk4#SPdhFKw63?Q};Y6ziXl78$HLA)!)j@J-|N7st0Gy`{Enps+z}gl^ z%d>TfO*#Hl(_ldinEhJge%^-#x@ET9$1tNloTo>ZvoAysbo6k&@1T|zB+JTQ=`HT! zuv;7zsr>?8zjn#sZJ^lMl#!Ez0ncDE)xV8;((%`2+oojpc%L_lFXRZs)bCdE4!14V z`x+C_pJ*lF5-Cmhr?)*tF}CeS_55nzR!zH@qWy-^*Ff?6e0>~}z7{rCVoHkIFk zw8E26@=oxhDs`0cPBO*oYD~4>$sTTMH{|EFz-myY=}p_4uxS zDpc$b^Ylxu_G@M$AU8r|r}c4O((cLQwzywHcNSg4D#faBEEj&5PCfkTw?xVJyL)I4 z;|nRK>0A)+cdlO7W9$41#!ad6=ZYrlbu}SrC7gFZeB1oAxX?szm=F*UV!AlK-S-KD zp0b7B5bq0qJ1q@kcL~iGGspb7P>IJQ7+ad>6WXeMPSc>K?%3unKT zAHqWwBl?-l8la@)WLYI8D|~ixbn!VDlu+`1cGi%L*i_(|_H*59QAy5uT&)Tu;0J))M+U&OG zm2~QXbwbl0MUu8#TvsLEq>zxedBpCpjijA<^!cUgR(xGkbABdX<-vus>X!?i#@0?dSma00#7>p3-nHU9%-Mt%>2T^GR+(q^=KL+q zx~qthzu>#*2hYvQDij{eZOQi$t}gux_5lQjCr*~Df~1YTgV$~vT7%c2jk59f@@oQD z#F{NfoL1*(uFs#J-Fq(8Iu_zBR>SSD_O`W?m@U^(b`{Xq~Ql433^_P zSU<$ovc>1pk5SRmCg*g;3hO?+9h`9+`SCFBAHwVxeb*y5%<8XIM;Gld_OEMlCa2fc zo|7Yz^;E-!M7G-CCP+X22UhhzEhk1QtEjSZkkY?jo!;22mT~f;mJcizy(_YM$3LFl zn7c56)&BTwO}Aw(1Q5{rCU+d3)YkZ;eFFkwRHIj&-KvFSw=Jr}IfQS*ivmTkYTtVxz{av^`8gf$(n3}5bIL*gEcU-0Ap<;kC zsi-a|9ddQ=C*e2sa-%EqruPa-_WKX(n=XW7>Sq*+ywq9ycA0pp9?}fq-h8GTGn@#S zsM#~Wh>cPHhbxgPx~7?f&&`hF)R+p`>;)OIXk8{I%GYlO-(r&V=EB-KxvrSI(eL>V zzgLW&%Svp$VC1pbcaiuk>}Cw1T%yjIV+cd<(e8KxifAobuFuURC0ugxdZDnnEb}Fe zztSH+POsg2UeWaCBM$^vZ`G=1`$KA)f$v@6FDISkEbEQYkPzwiYe4M|XEF#VoF` zKpK^|5Xpac73x2CZo8(IiZ7_SUf-!3F#NsjOHl%`y^Aymfm2nZj4)2LE!?ji_wSqF z@97zshw%=E?12x+uxx+d8`>!@%+hm+>)3l{1#95T56LsiU)PL&aFs+X=J;<0_FqQ& z|Kt||krkykGBaA<-9UFGQV}mdg=s=9C&6#3iJC;5bpghDiwyCtU=)^)CQ=A4^I`4* z0hqRh1?Idb?GUHiyBmk!YZU(YRDZCbwnc52b*=juOJs|9%y8+~7pWoqu1)-`7AM); zDE5@f&4_8?=|U@h)*aKGHU)(9T%zR!|5w@(;g^>x;s&tFI&U{G+16PG>JfE*ohBD8 zoJiK5SJPeGVyZWj`~CSv;SO|*%1;?kB~KOL*zZ9q+qvDXUe6Wc zZ2wX}PWXKWwxa5p6wSw<${;XK&@>n}Ee^e}X#T2o*3Fc`6YGU{QDk=_aDN@eS5(9n zT~+0z(?J=uvsZF6Jg?nWpo`(8;1gKnN4N&w#4jDH&ud-Fh|?iINbIs8v6^?j5OJip~&iNP=G@I z^sl2CH^b=nF%KgeEeG%Tinv5`sYw;gSDS3Xnd-T}&-C_wn2Ase46*0P>E1Muld}2FutCim~kB9cIRO;oJnAi2aJ%+VYb}5*Ih?QmF zhu?=S5s8)2z8#39m6+}4l~qunVrK_B_I7`?Bk5I=(46ip6V26MpkQdhbXfH$TG-^S zQl8lhziwqUg;u&bdl8O(OmC%KKihSpb)vrMK`FHz@zjWRR&96CKE^Jtw|JnD3xa+? zX0G-5%($jCd~&PSz8AahUlOfsHDhIs5i-Q}WeEnNI(0I1;<1P14ZoE=e)!(^Ds{i^ z=aU!w*?Rtj01pkrY;>y~pAFNJ9f_AmbL020F zTjV6QOY%=FPqxQMmkU|gZ@{${TY$m-iK=tWxKH@*w$wj{XuxFs9>*Xuj#k=WFiRJ6nr0$uK6jrY8c+L z_T=Gec6$#9;rn;SG&g4Un2=f<`Lo|LhF(iwrc_S9OE;FaxHiLe8sCT9M{(*rDH4`8 zIZA{1(WF!Ao*RdjUC;OnLHq_kP>xocNgfZoM>;-4}$n zOlNz^cDQFAxqL(%ow!nEZxJGAiMzCJ^m&E@N`oY4a$0i)STb0OT|p2Fe`E^(TwqCW zwPqsE-x|e_X};&+px=Qv-g({Q&6_wlD1nJp%3=LX*zg3V<#k}Z*_8WBhl7KyI$f@C zLt_4_Ljv#HyNnfkG9NL@;lNO0z}+zUFJc95KBPIgBqaz&fW5LjFH8{)YsR%_Uev2< z*0HZ&G^-xci~fJ;i?lt>t~>pnRn=5=JzjF2dC85r4)h*hPDY+m40KVIJirq%`fiF5 zh>5aYci{80KQrAdHBLC&i&&O20Btol)o20j0P~cBYYMF=6orI25k%n#{IynU-gR9X zk>DY>g>n)N@7zkMJr|HG$z%(P3B!!w+QX)E++@Y`I=(e(UU|0`}+VHy; zvRtl?JT6%tqNYURFti`BmExxNr>M(`wQ~A@*%H~VzkI6bIK@`2KhZ$`R2%I1^VN0hK1i z`BHRz%J>`pM{jRF6hf{qTGe_Xp`lm4ndq#|MA|zisV6%1s5?T2kOZ%_! zG>S?TNh)?cRl&L)DL4~as@*qZ8rNZUUK4s+G-v&5MS?iu;$)-j;|;BiKGx*43)S)m zuBsa4KVCV>kTBvpP^%n{?0)OJYtZG}u@sq@h6`XJO0O}gEFT>U9r^%5r8g!|6X4!M z*nwA8Am%PGyLY#yQ3)Kd%MpuaV2N`L*D8^kdZTt{rH;9e=kR!lC4T(7$f%hQc?1#H z^6Fvzh)FL$^cJtt@&L?pRy{%>`Px5q7hB5-mE6(9oFV)=_op37nek?WU^V*ckV{0l zZZ@G&57v94_*$l9HPVqjbi-zXCr8I&Q!&!Dw63wy5ImVGy|IgDV?8YQjmu6nxDSnv;(Vi3j?eHYCR3JP@Og z*>3zYDf^BGXCu;_2Z_A4&_9avQ59p57kKLPTBa7lA1qy} zYUss?!iQYGPp`y1pm@Tze1uPFD!Pb0i#$j`%yBVCvqH`viYnUV`l*sT8in$c6KeYB z6}7L{^*rZHy}Ld+8d^sLT>$-1s4myJE2s5h>`ub6)W(t9Vz$uLCRmNA*}Cs0nNA4A z_i?5yI6R;=Kc1#XMs1lO=I3T!2HWyTZMw1EaNw*OSq>@3!5%y<8XZ|H+qZgg1hi^# z?~M3cH=5*oQ~oez?~pnB;oS+HsYdhsgB5X7G~=br$zsk-K3_^aG*CufsRpI93DJcT z$ygC>AwKUB1A_~gVnArV7IVau=iBByRYT=siBho+vUKR&{`}#7t&CYf9suy$N5t^u zhBf!!$F0fV5VBKQofgRSPvj5a4-hN~eFNfjuCs@{BWnyp;Za>MNJ!$A^kp1XU%?;L zq1<@8cZI_!*6c=32&itJ>|TsWUqkO?BFjcnOLIctoQJpz&DAGzYSSB9<<*ae!<;2> z1e(kG(1w;A6{b75P_ggpO)bIvZrC?Gt||nw`7_k?mF&6K^7PXyV-5Q%N@p${2Llhb z&3om<2>l+AN@<*RwoT6!e18NlQ}oi}|hH1KWr^&Rh7wrenW zRfLl+0Q;L;Zy^pMxT$0I&h`Ox&rjjx#84}Ig;rpfKh4}~D3uHUjC*}oV|tKxZQ!kl zRUpK7Z*Bn=&h}_02I`y%$bc^?xN;*!uR0&y?< zwrrVSSFEkSs)4dohZt+C$4k;09YC+Vkw%W~*Pi@Ttb%YE2C`9U7V6bL`pug8>vTe+ z)0tyKPm?-CR1Q9sD1Yjp6#Q+DwM()R;p091*)1)!L6jB_DI;kO-bHtofuEKc-Y_v zM5D&%@EvZ81s-+Ng}9nY-!d)3{$^tMG8hbzU{(WNy$lF@g#E z_LY-?<)4i!I=uNqT20&au&;N*auZR+g(=49go>Rkc=h|qkqUo)R=t>`DZOgWx*3V@ z6}9%u>%7o^?6Fb*Q6M8}5zAy=iZig>+GoNP{D8tq$Z3QBI2}TfLQPMd5}}paqQBAX zdvTL56&;{SX$Inix2chZROn=qrIA2Ib-g zUyr|((H(yn(R#@EpjGI1lyd`r0cK0zyK|E0u&Is>PW zmoY=uE+x56@|YeQ`~fM}z4M&_bBH)Lxh1!YCmpgX!#DZ+w^6yPDjm2Ja{XKnaUiGX z=qWbD5=W8fT`a)}%gx^4+tY|>G2t}D3R6{ccu~f=V#%dVWIAxHwaA|+ z`9}#5WZ)>{EdKD4hksty_S=p$+IA+=$klmJGC|T7?vjzRqaJ29yrU~a`84b}&mcm% za)eb&(bg9~0i6{P(tZ;mdWAzEenVsO5uIVmZ6d5xJ@&FBEEdjTG zjKr&qPuQ4mUz;5y5l3$G$PFtpQ!r{SPs&Y@bK;hERfHutNJa~|o(AT&?*tgD8Gz_? z9T9CaW5<-0`#6w!WGPC&hQ9oe6Un>gQ7^?&zaH{~?sV7BM^)>O9?vcq@IviTMfuW~ ztiVde-9bdG8J}O(Uu#w|{cb;rO6+nS_gLp{f>|@S#7-c-*BJ3@QHhc$bDmX~mG_iU zQndWYWKx+xhEB|j#oP1Qh1E}PHqq;Qqbe*c`jpee=RAk|o13}&<59oNm+1+KtQ`4x z&(MrFR#r%;@A*ELI)amqCY1KdhEGLIBOpWQBmF!m$BmS#5(*M;?Ch|^BwUz}F>)+` z22MM?MY=TS5P6viPFba9gx5Pu)a!K_k$aH3uSHp%tkCx_JfGQ$Tn*_flOSvXng~-I zt}7VD*!~n$iSRmqhBpj6?z}mC^9BzFl=dh0(Do7qM^xm8WSfkN#%BiV1mBz zkD!st%1@Xe&5?7`qDAK$N14At&7a>-Bs$QhR5g^7Yz}qMhbQ!v=_<+j1&u@Ix5su+ zFQb)n4F1%^T%|kI)GAWYWrp+8eu|yPRgT4|KSpTDVcmz=fz4y}D4v3dpn=S4!b@j; zh*ZnA*T^vGv2^oeZl2yyYr4ak2`@(=JF(ShK-HmfnYoJVx8UBLyYLy!c|KRd7JW}~8tLqWOw zxfk-sPHH#jR-qcw-=ukGn>jy%S9cJ}-V~^iZ((AF0kt^lglXCz^M^@)XOA|%_~dGp zGI}!jkxJzVC-@rq4kTW?)e?C0amk34@@2~-ulffDU zgtTe%sC<*}&kps=jT8?Xd^b5gIleQTUA-^5rzGT&)NUYSL3nIt_cBHg6V6(7(U=H4RN$mSmfJ{ElXi@l z@7v*=^sBAB?=RXT*;z>?UTUp=l+3s^$u)mTsqU965q&1R6S|}@O;&F|f01IZa7Fbp z_DAzL6@oPeag!ngY;gBNnw-~_4=h;VyCt^o_lgbokrE-K;|)we2L1gG(=1~4tiXAa9RkHC zsv@4BgWug;ZFtNH^O=pJIn-^Idx+rNofJFF)kc&`mk=OHOFoF4B8GC7#2iE|#nKl~ zHc2>Q%cC&~Yi0Znr$9lLA{s+JTN_06(tyc|Bsol-BD|LEZ^5WVrX8WfwMh}^VdSEf z%M2oUf2hx|TCe9N;nYeiLI_QXxbu`P6^Wh852}raAeP*xymhbjHPX~~!Y#7JnR|Ws zqqVG5mTNy-Waru~cW_#ALKFYi+;MES(^jVcynrF}m zrfjCP=te2yvXltfNQKG1ciLE*tPU}n;tWj4SI2+rEf$nwfrkvU)TF{qA~uRU9*va| zWS(i!^7BL19g8tDqZIep2-#q?raH5vJG0CHc8}!F2#f|-Bvmpq1p$g}npm#H*drJU zl8+Oq#OR+UXi*eve6twPkr7q#*06tl1Mgd8mP(XT#d1e zp-_M`7us*Y)I*IdOC=;e2Q(d1QYx+!bE9F^m2A!@Td4nxe=ziV45fqZvWb%KsOhHc zB6`Gsgu2!pGGoqKY>pU1=z*e7kK5zIizlozITX;7^_qEZB~eCCU5_VHwW}h-JKew_ z@P4*kK@V^5=Vo<;1&(@#;Z)a<{my5p`77<~=NQQ*A$M#^*`O2n3L2$XI-jr^pdn~2 z!Il}zaq<*9SY_zwWP^7?{cKV8^}d<7?_%{ZGwX6vEs1Q>(}!pf(x!|Rr2Lf;Ig*@L zdB6Bv3-#zFe{l(ve2-%dB=4>}R*#~;o*U*?4Cyv-Wo79OfcG*IO3OYEI$;%x2`Zba z4L5u0((US8heVAS@9p!b@>Cj$y z&gi+$yok)7J+W6l3|+fU;(MItEH<^s>2`vt7laXIIPPB)M-k#B<@~`o(4x%Z^?KlD zYs_Xl-Iag1+^OV0V`Ygq^bm;UVDh*9(Y45hL{sN;ptqar>ijO2bd*cBg{i`B_tTEH znih#TrU&BM?;*LacvAWT&<(sGo?88>{N{@*^T@^nzBb8f(iHUiAg0?9pA>x0Kk~~) zA~(y9p;8Fjq(RjTCEqWPtl5+ZF=+*5_Q?0>tP4cHhVP|_mKTQxSdd4o0`!?v3zVxcpSU@v#?ei*Ld*&Bi9Boy3*9X%xya3saz z|A`Pu8@ZrNk|AgqmB51cXhavHbWlP|Qlx$(I$P@jn^+|j#UY<5=*1uTgjlys6{Fc` zH>J1QB#G5d`o9C$o!%U6J2eWA$aBx1WgqBJ>QP~nn5*6a|)M&rX{-{wb*K| zm@J(a!c>&CNmU<-S7~v}Aero_7%2cXky28g7AgPCdC?KgN=VDTJMP(5vaPBt$v)SU zq4?*h^=1nRZZ|{R>;UskuoFI&@C0mNC7)?sw^nOvldoGgr#O0ri?EeO+ zSoF85@&AuQdP#TuPLn!np{d3{clJ%$e7N`+&SCbeJ%-RlK$`e_9`e#Z<*?q)$rjxD z2lJifB@%P5H)f5@_wW(Qt4I99OsWsk2ifJowPBD`*_p!0lWDW~f%P!6vTRVfMQ zXDI@nf69*ityA_-;f?<{4p53JjcY_T8QqpJHJ6ZI$LguGW)hHbr)qG>kc@k_nxI(8owSKo{u^UFLbh3n+yDE zC?>!DGn+;VQ&wM5D6TZ=YAh95Tu>H|%Vutw2@wz?^#8 zcD-zsHw%_gzWiKGE}e`Ud6Mjf5nN)&|Dx`#qUu_jXyM=lcXuba1U4Q#xDyEO?(V_e z-5ml+a1U-9cLD_W;1=Be#W^SEd}G}E`aj%$*z{npy}G)(YIawzUNtv-Rk~2?t<#QO=tY$-RW%7ZdwB; z>r7VB3BU>!$e)fp&Dt+*{^yde3*izQsc**LsF$O&*W*ts|Gu% zR*K|QL+u;&gDO&%+)0lS^KHEaO5k>)wC0#yuv{@ae3{zSHgrx-YffP36|mBc@I*0t zUP4VfJNf<@1h7UaYArdOu3@Xq&F#(*>!ca>*bIkD|DMBpu**LhNvb~4_&mue z>qKv<{4ScYxHTc#aK!z#qxG7{C2I=@iNA#!F~`fF{$dWjMrl^MT|-HhvFm!sb}koc z5$th03D#7VCl*p4S}Nnv*gcDMAaLD%5XzF-g}HV*2;9`r-nfvF8MRV)_H7>VJclu5 zt}K$@RuB(I;}KnG!m44BMiS6&Eu=Dx6^*kI_{_5Z7O5d z+8cB$lIwe)Nb+_box2!Y67{Gz;d(`X;SP__sAOx(Pf<@=$RiVVnBg`*;U~)0NIMS8 z(}`Tf0;9;m6WCtfZ@t}b1uVmnx>*|@0oKrJiFxBoK^saEdfjuo=0TEE105?jwA0ZO zc*11X;SZ;rbX-Fjl04X1h*D^H42{FgU}jb>vvaYe_cX{?EQjZNB!zYE6h5^%??k74 z38?~P_E%79-r>k;JJK!rQD-tUD@vz7js^rUV)VhvK)!cr`Xpt#VnylvB$59|{-Gop zL|VLWrR!luqgz|Jr#rwkmFZHVlHZe5TB0;$8i{xPYmPr;APJz&jg^8MX zfKHjtz9U${f1<-aow&DD&oN5aw?I)3Lrb!dsPx?CZ{T#b>1Kvp>c_iiK6%1F#^M*U z`*Z=P+sci7M#*V87dV+pB(+JEDIimAyS}(|aZNwLO004VS_sYpoxg#tABR99xZ=$q z)NZ;iiS*#<`;?ckqZgWSy6|A~gT2HFjr+eTu$e`_e{sgc=;rM2gPtAN{ zuT;rmd})pErO?g2S|M9ERgISG?jd5x6rCjm&L}ERI^lz#(`9j?d6KPS&T6_03;tkx zFiQPLV)g{|?bwTM>fJNrmOwiM7=f{k1Wd1Zl4_~NXK3aMgeJGc%T|mJL2QWuwt-F7 z^ZM)y%Rx!qFq#1@8ldL=IbOp*(LWW{kOHQd=B${<3hX#+VbTY&MNqr`FwZ2A@$O zV1M>4k@u}l)>0#kQr< zVWX&Kb2>Q=9bbBNWt32)?kJj*q>xB5IafO1EvU8r_-IGL9Wv>#6 zF{g_!$R($DA5qgdSo+5nuv~?rlyW<|`f{1_@r6`jea3iG{d1AFc~A@^fnW->iBMza z(>eUipIfR1OOwY>UaN^Jk>!C#A9K2?iRH|bV)gW~SJa4k?GUbp6+%>P@@z)P$ud`6 z49Qg%DdP$h(WPQ~I{3BmvjX)-t25Z#R7hWC*q)J>)E=i0^+>F?0BL`*$`!_lStziy zz&hdQr>@c%Q}Yv=LzGDx0UnzH{T-cqs#W#dGt4Zn{zVPz1oereTITQJ$L|VC5_X^2 zGw|_o>POxORrrjpJ&5v7)saODP-BEEYHEgWg3J%mUoxbL-LMU%jBEw+8up$-oUd&mhEWwL|AxHUAOdQ*hCIcTM?z zMzxmrX97B|-k7i)g?HshgSPuEGXK>LrZ(tYye5J1JQKxWrZ05bdCgot%@yNA41eH~ zd4TI<%;u*Y#X%YjDYK4tfpqtiHf@k4BpBFTljNN-QffLudvcdVM_u-UFk8(Hf1QTh z!hW}5;B1)^E*wDG!?ZstMf=;-Vi(x)+B+q-MG2JW^^dK*2RI`y30yrzJrC#7Q0S#* z*b)cd_=d0Vwo2xSSQZ2$eZO;TF|Cf5TQnLkS%7MEoLsm7?SDnjn(9CXYdd4VH6qz; zQ0gID5Jp&JU7J^NU@~wUK5)b`L)Gl_JK@d6J#9#JBqKl?MI# zeGs|6J|E7|b(gF8QeK!#qV4trE|>kI-Z4d~=j++ow_yewAg%_J$pUedlZPX9+pP5Q zY1CW=L!Z|`<}m$vF`~p#)$Sft7;pnR*?J#5GTb8jJQdZt2bJo3rfJ72CMbBR8C33>oz$Tp*Y$p9`4=S=Y z*szxM^c1qo)f&lF7;nu(Fod{^eVzgd1K4zplj~)5AR^2c492}sv^La0i7`>kmA<(N z|AfZM2nYU4hCV5}c5)5i_6S<^4^5J2kcI%2a!3_1x&LlF8sQw*yu{~Iep2iZGMWY> zBZNi8*QhM{pH1eOW9Y4MUZ)xp^K)2*8FTVt4A31I_Yu`To!rt{zAib^&+NuA__TZb zIW8J{x~7^bu&9WN0LUw5fM*?Rq6wsf@KCu;x$k=W9hidjbn#$s za^Ok=jncif@NV3FA*aKYx$dPZT6fqyA8#SX=4lf@eT0?o?ZawEPdGKafpB322M58W z1lkbHAV86hE98lEvN~83PNS)IgAqF&tda8N@!WfT?6u)UKV#lFv<4N#q%~Cpl3Po@ z@4kiyE=4rZdb7R`6R!HsJO6j5@_hpnf=K-Y%VyjX+^bhOQ9F(MXL2OY%x z_PEa2O==O#^oD43DtMr$vO&?^aX}~Ra1kF3(?BtRI1ETDh3raoYDiUOPlUKm z3y$SGl7mYRP90@M&~0l$a<_=T@I zMm3Yz#AiwBFpt-a;6C=C(e6(01!fp=7h5oIes^Tw>v^%>ZU-BrbwYCglTC&$7n`6v z5y;gZnNOHNDo2rIvSaG1W%!b+Xx7uPdkDZm*kIt#HXTHI`WxhefS#i3Z_ML}2pGKY zj$IGMsW38v*9!j2cow=@ZHT}k^TZ+#?omO8nCL{@0rb}Skfh_N4liRyD|RcWOVQ~} zIUn*44=vIwh+HdNpN~0(dN)9SHY?HpLni$D|5Kvw#bjhYZzQ|`t zeir{dkVc4NE&|3)PAuS2Ca(aeTZf6b6GO6|>!TvWxrP^Q!02+-W)z)I1hRRjbK8;p z&6Z5?T@Ei~#!~I0^qom+7rJEUx5k<;(myIqZqF;(4&fW<0sC}^&KLgf!&`OtO$*=Q z2&~P1EOmAEqpGC|ueaBQ7|EYTE~9FrjVF%UF4&3C$ z15!4EUn2o1umBuq7XZvN#b6#(5&X)Zv_~drhTR5N)PoLSw8n%G!Ct#3@rN7^*N>z) zzQv1)w1LkIx;evp@laxIu-Lp944|l9S8G{x%NIjLt=7R+5y5bX4bpf4$AF+v48%pw z&zP|8x{y7tRY(w6zDtz(B4|*HTuQ!>4%*hDfFGu%=Kbv+-juyx!qLgn8Ul^*-@RYE z2LLTD8VM8Jz=rIvxb8)gzqh6V8)BfSq49oy^%+HhAHoPMaixxXb2K{ z@W%jy1=;L7L77x$liH*tEao<6P$qs!PmRg2`%(Se#Kgor3~+jVeJfD`;G#J-3bVju zTVS4c4gYxFvPl4WmhRJQHoQp>jV^x!G z!Ix%DT_V?8cl#Ed&0&d-kD?^n>ZkQ^XSl?IV3%Wo#*C4~X9Zcpxw>1fo|UqH)$1n}=o=#T^YjW5aLIbS;;_#~UG%oZ+CNbG1^CUe?Hew1T zPS>$2Z3>)4@-)|=&!g%o!QuiG8ZnW5*Jxd?g@<8yH1bpkbPcSsK$_fcw{V~@Hm=EG z;hP!Wq|@R0HH2aQM@`nLc=NW^?((-bpi8YoJU#Au9*eIiC)NRjV* zHx_&A9ON=>!u&C#Ym5S3`WLJJlUXT~skbUjjIluykY7q5;7DR#y2=_i@(Asvlh&bl zpm;j382#@`)~+M0HWW0A}^d%I_q_Y}8UIIxGBSw4c(oToIsH&>wokaq?V3_hC zf7b_hLyabM#OP-`jR$BT-#@@&15hFhqvhqF4WpxL&jIVWQ>TNKHa>ta*~|=w#WP@Z zjQ4=C{Y%!J=kH&_u_OY3$^2bzE*+=#-rf*-<^!eL3gEv1h%A7F-5d9U1mA;g7K=(s z5>irNWm3b7~FiC!KrX9A=s7dD6;`jHG3O_0dQ(QQe63dWi^Wo2dN=H`cD zg`&)x{y7m@)~;m9kK^r8p?+rZ1_&MEZxL74LcDTJZ(A()hb@;A*C{2ZiwA&H8{e$g zt(JTrDH*r5GLSba{rH0ms0fjx^*TnbBprvL;4nr9g+)b;Vp+{6T;Sb=yzwkN7T^h9 z$|@kA#K(^W1YX}Batb}x%J97^&;BQ;;7@^F<#K~@2tpqQI7 zN8cPpH1Gy3K05$U@xMi-ggPu@h5{OJ<5*TjASfvSs&EUFoDl@YFN(>}KU*}WHkr(0 z<1?W(nPh$a(&KFZkpFAxSBL#eu>bQz>qU&e%3*n4$gIo<44J`F2VDf%Raz?b_L$@Y z9|VH!MB_~0l0#SW;xD+Eu~_lB=;E&`TasoOKB#P8Fu_jF5Ftd?I3PEyt5q0G&QV*N zBqXFO2>bFjR0Zo*X9P1ymnCP+g%y0oLR#kwD6eOH@Dz{^o*AOEls9M@#xZ!OHb|h3 zow`XWmWXIR0fHg~NNoNt(Rxsbf2ulaZ)5%R?CEoff73pBi3=s%3X&kw2=uF=dXKtX z>q(AF;J*kdp{6&!@eOr{ZErjDJ>dg)1y3~xa|Ey(n|Fe7tej%~#K7U-`!izTdwVBD3fSsV7@@4TuEH^`@A(5H$GwU@>zE#*y^i(+ZoeLuBzD zvYzWi+Nf6UT!xYwZInAIL-;LwOGn%}oS(?&`wcr=s(pJ3qCN3S@GwD%r^g8vz&{)z}LlL6`vPWHZ9+5l0UFVCB?jXjo&IFTwOogjpx;@z58a1l zZ+`LkH#AR=mi;T&km7#4p+1go~AHB^9H@JrZph#q&mMYX=;c761zr6o#5Nmf$*GqvxYTJ zLf8it-H*~hCR!8hWOw0##$agG

Fq|Hr}tMi&jR_cJ4WE^%ViodHaR-TA`%>9}l_ zWWDp=)m-Z5Er2bZ6^ssgbs-|JOjZsAr-yQ)H{_ z=wOvBJXATis-pUl5^Wm<9O^IG7#ro`G21Tkwia})_N*G{AZpT!{-Szkd{7)LJxxfm z$S%aAl{$f!kKrkM7-8K}F7OO{0+vY+VlI=#2eV(YncYw~^vO}Li>W6SVf~v7{rdT0x zZ=PY?;w`yNbPU>N$H{Uejm@2k_=v4Hx)!{vw-(4lO_hZvoA1+s{$OiQi!c?$OQ_Ca zH3Yrl`KXgJ4?!*S2DBDn#j_j6Y^?DzT=uZ>Gu9NJ5Y(%jUZ===MW4wKI`qgfRhOyt zZSl#kS303?*`ewPm#T)%sT!s;Hw?PDs~y~)7S3*6;|=^AFF6xeBYIzGvg^t@BrhQ? zxckGZ!ccpRPOxp;oE19o4#WC$m;nL0HR2CBzr<$et(gW*_&tgVeO@8ywknNM`+i!d?bkkv41!yF&)(@XJ!2*# zN;>evg4Cb9!+@9|t2$~LCE}d?{I?s}t-)nECJ787fuxg%a zKQ!;fcr|@sUW*mwQ!?9jUddql^!@4aRWzmGuo?eYZp8z`_|8*strvH%$P0GAF8{&q zbcexyMDmbN)VTwC$FJUfUMaMF zP(CQew}p}6km7Sn%~Eu|7QrW9=%6eZO5{Z&m+-S%VEaOEv-*Qjk;@4OeMDzQLM5xY zsB{pMHFt0k)=f)60;?E6lk<1k#5K+P+r+9WaLsVHrJ5-h5g3iXWQ8`3aU;XrMejmd!(#D=-#cOc;ak z^@5F;gIFqmN!vNe!N%)-&cmJs%Ge@@plEA?BR`Bl-fvqy+=8VTz`P+qXS}?D>TwZ> zfU>`wL$#=n8H(ehu$8pwnXs!^x@h6%w3s`#5eJ2mCrOyN`|J*7l$$Pxjn+F%m)V#h z%^m4qin7^y!MwIF1E<3KteKOpi!{lP!uud5K{OL#L$B_4N8q;Nj#vo?$FcugzNjkP zqA1}I@b7=h8H33`wd(8P!O>o0Og5wfDSAdt9T5*Xz=_(rv#!k?ZmZ_(gLn<%$S8No zkho-MnEDKA^Ne}tq8}J87O1}K({ab}q(kn1F+B_D5Ed@~a~5D$LjpGgH^bV1X1dBm za4ZY?Mp}uMJ~T@;^jkrZiD%L`Jz)!EV~I&%N~RR)3)rhvob=VZbGDA4KU7qoKK^Yz zL{rY-x>q0Nxv%u|a51RDHcXjmZ=nzuxrWHH*zIsk2|2-iYZeu6)(*bRTc$OtZ1Z`P zxUv2HD0!j#T|%Nox`EPgP2o+KbOb}51HmO{J}Vz4+dn7*{J9DKGXp>L2SsFPybZ~W zvB?liKs0wdEnD(jw$GrsAhx<;=J-O}D0of8XBMAijHWCe4F~cTUAH-v(ijAVRQZso z5kfCUZKMI7e@LJI#$-ne(cI&gdBjwZj;RJ#-QoMpsm8L;75!s$c?MoDkvx6pD=}7a zGM?9&c4so_7Y_p3-q+43LMN6IS?-L$iiD!l0arw@gcr6*iB?xdDwy3Ofu!%pP;XbUX28<(N%0-{Wd`A;l3J`XQ$wx`SiDuMOrsk z!MBzP_Vy@E#5omO*(%R7$*;-YS2iiJ^e@xd-{qnE-$~7tyq2Ub*r8TkQCYeB(82Aj zV&c&PkmR?u)uj4o)==(oO-jjgagBOvf=$S@JE`;xPBNhe@{udHuDyM_oWLcrUsB}dmhO@r zGU=?QRj#t<-!vU7E!wgw;TOtLK#W;`5DZqlqxSo53my2(_4-t^*&JQ3xPE5>DuW{V zuNx^Vi%4n0yloGt^Wm{GcB2uxLZunkkvrQDZsqjlwSgm+3=J9lqaI)yIcNNP(&xN# zq{!lQgtqEm9#H#2o{=&->(?Ag0y86J^)vF|>5lA~cbY#;^wF>BX2$wG7lniyn7rVYDqi#E+;~^bo!FnrEeLjn%^ugZ9hSHgSA*> zohliG1iH=?&RHkZTy1ogDUr)h^BBuqsa^6^$Kpq{*oRHeD(x$tZIq{pQnde>%HYyD zEtKaw{QLmTxKhAFdanZPPev?2r~&Z8fR{ZP_|M^^h!K76PMsE5@9E{+Zct)6t|`aN zhjSWD5emnfV|WiQmMGoMX4 zg!h{1e(h#I8^{44>FKn{v^&KQ1P=d{XuY!2%enY-M^4UmhOD=U!lFhWUe*#X<*=>b z8|t>*HSti+564q%er~)!zXF!CwMkg>W~9dJB56FB$k8VlT>BFc%E|zu^88u7|9deI zr07kT*!Li#MFQ~*W(8t20r1xe=l_gMygG`g%$Es(Uq&x?{f9jNLeNXah~b|GqE~6` zX03BPw{>_)ydds(UKWqf^+P7DH|JtF-Z}{M;zXcd_E&cC(Ov3@G_mV_s9&B*z3d@0 zOdh$tZ=4Kif5;)>Svxn9WgDU)ASf457@~2(BQ#Kue@o&ORz_E)l5`8LieJ=c{o&zy z{?vvf*89Y86LxalLZ`f05!zJi$>v+c^M0u#=i~=51+G9J<)WkA_0JmzxvR5TMqQsz zjD*n1l?q^2Cc0y@cVuV)A+7&AvZl9t0A(srJ6+w96&S+35L$l^>zB(A%M@wGe);8b zs~PyZF{xWMWjwRDoJt@%gNt05Z}hj@jy)B1ECUhI(+2U%BWwB)lh#yC1UK7eNbQ=w z;0KdzjlQUKhH6!7*;<-eo`xG@L0#uK$ch1hws;Gp)HP-`AV1uj#8w~X6Xu^G%isQy z1`Xa;&hsz1(BPjw6_Q2zd=XloxwT$G@PB82pqRDrJ?oYb*}3BxB4c$W2+q^3wB=fU z<-r+_loUP~MLazA-!*E;${hnpUJYOjw=EiZ)jR>Um4&Wkudf>lYbS<+zMPw)vxsgWL58+uep?3SaRx;qMDJtn7_CWW^bckquu8592PgPq zO~RxUDK#s5x0!O6vD8!cT7Zun4V;=L9Kz*eEA8N!-2`2^LE|=NW+I?#fe-kRF)~O& z;(GS#tecaaAJD0Nf9X{*|0&^pChu6vjT zjdXi$DIHY9uX}TQYj8GYRCBXNiuS2b1`ZUicS*iUW79*7_dbs&@~z6M+0TYZZBxF! z2e(zj-d+WsxC%}ubm_||8saFIQtkyZ*yT-=`&B;ikOG&uNgBq250CpX&B<@X^-wWdkw zN)tIWLqKDr_C}IIhp3G1OP&vk9scbqfF(3iQ2wn6TeXO#9tL-}EB23VHCcj}P;7g` zPCG_(fv)~SQc6@pkFPGwj*1=-E2l3j%H1t&vuwE~5`$fBJ~~)#OKOxa^%J`_QVht% zyIH@*6cT)wxGOQ)Cj2@oBe48#@^|h7tjn-mNj!|b7dpObEb}rirU5%(h))IxxLkoS zxl_Vv%|oC14By#R88|@-`*WV05#X!9vK3tuFy?4e@FAJJ%S_OCC27DVA!;J>coW~H z-#z@b=h!}v$NJ{hQ)Of>lN4W00~xkz!5#wVA9sTe(27R?+uj@L6i7lFY{=OP-~#eYYq|#I3`7j41ipcwtgGC+UQDpR-xy{ zVW@u5h4Q0Zo1Dn9#hRNo=(C&}konntKQ2qX8FKCvwk0(WOqeYPN8`bof6})k~{)#|1cBJ{PzZt}P?VZ$uZd#$7k<6}SGXEh> zCLa@0Ry#qmd)?Z4_CRrHb4|>XtHIFZH5WmXG_G<&m787X1tRba0ArLtyS1O^u^}5r?w}-85>s^aLHqWb*za+W-%+tG-|m(Te!2O* zu!Cj1y*AM~Y)oBeY=S`S$|>j1{K-=8i;|!n?dq{?{~-2I@^EkxKPLd#9Rc!{rIr^d zX=66)6N5;c#dNX{IhC<`TNUz+MsH!1wi{ZKgc=f&2N$&0sa%$&udo=vHuXcBI>j*8 zunZ}E#}woF`J8ZPd~<8?U*!USVcfupJ9%B&quBoJ35Mi$Cn8#G4l#BUJ+{FM`gg#r z{tVa;?3F4w(K|Z9%p1sXIRLp~Mc==AOXxx#aUjm=6IAgTIu=6=IB4KS8xG~)Jk?B+I`R|f5{6{UsF%U*j#LMh^GDpFrnBf4j5 z)5@=l?GoR2zHgQw_GtCP0XBG~bPE&Saee=xx3x|JWv;q{kW{c+rQ6Eugkr>wf{uN- zWgftieeX>DW(EGj@}Kh7I?EoaQavwxQotj_fTt3VJ#kk4A!due{vULWBB z=Lg3j9(;90T4+^6&^=HlRD1F|@+KJ=wYFeO8XC*Pc5)`+&H+`}E z@?+Wa<4>4ndDEC53wfd`ADj9`chW@`w}5V<1UF&e8Er;{~hQmTvxF;)&$Qe$;}q813p*Oc>kY8E){Fq|;nus$WiFs7X` z7h~d{Uxiu?{@dv`;#TpNncXDcp)EsPF~ zy0%T`6fpC)eMq3C|Lx(8Ck%b^_lf2VRPi@%zr=gSNzKjz0y#ZP*aVBU&ZiVPOe^S? zKlAB)u%O=vPd<7ibH0{sz)1h%lOQ2#*i@6_NH;&+*1s@LRX{ z-%$f#Y){{#{s|=@j$07ByI)}F3%g0@IBM5w0gn@?$_vWk3hhmZmr|T#%uJnX`Hfa{ z|GTnZLt9iN6dbrinoaT_yVYGuq)Ml3`l<;yr^a*0x-^jNYQw)E8Bl-THZgP)LEvQm z*QZWbP5$&$@jKT{MKqv-hs~sH;z$~Hhe@vT-=3LoeAN%>^Wcj#KH<(ypUK#ck%pPv z!JXJwY>4tN7x9}`oL;{*dh2zj!u!X=0-j0n{_PQTT#Ck|gWSANCF@aFhFHUKm-~3=Z@&UsbO>HY8SeG^)SMz)2TXhaX|F>n z8t;)jT|L7!T1!{M5fxNL<|fW;SX45nc{u*3H4%F_rQMi&_Yz5?k^e@}r3E1QE(4OO z0*LDy3f=^W`N)BgRal}By>RkhjQR_JyK>P92_s4VXy=cczI-?L>p$WD19bbB)%Phl zN!gwMCD=urI>43&ZvL;y_^&;4^Ah`||8*4t;aDbKN_#o9D0v zBpOV=^#6PF|6}NXHUEIgto?ptqtE#8N(7oasL!xiqZ&Q}mnB-hRP?0@tpD)-uWkfn z4sJRm#@S~D1Xu?ILRD#Wqc5RJ(>pp%^Ej1!8W=#pLwd3FH3`;#2kdY0n$p=hrh$Wf zH2nNnZMw0_2wljVCnpNFwj}uYWkI2-FK%rM{eQGZPfLsWWmXnlZW?XLUZT+d@DQoH zN9N_;+yQ_#><>Uyzr=BojAQKuY}?%+1HvrUC-g zjt6)b8R%Ce2({xtmg-4s`5*oaVLOj^)78GzUnMFONv!c5zUFj<$(;>01q@F#EE+AT z6hgyh2Vz?7UQCuURNvm~PaM!GH;lduaV`&4suXQT0DU)uu=8#pj_eXO=(LG)Z)Y)q zYMXQj-f@k~O&v|YG)l|$A}zdo+ckum1n2dW97jGEyz-k-J4P|Xq8}V=12o0ZyFpsAzq;;F1f<5zx5iGP>S1^M#y>9YoxuMQayl!B7$F zbkTCC$J@SAI}ltsal=)ooI3-Joz8xCn=i8KU5k%JI~sLK0{551hO!;WQjH6-eS$51 zxDm+s9v8$^o7QGX+RSwt^Hm|1#HUPW2sIl;k-0|xo5|FDq!jm_JxjGr9`Z-m%{#i{;JHTd;a znRUhiiVq%``&R2E?-#n`KJ>>WS@n*8-DOWCb@|Q@#=H)yGCW1>{ox*PO=1<`c1)s6 z%8pVgXoR=Y!s$1V+*LB(&v9w3Cg@qs=dVur?8^*LLXO>plNuaxrn`Cm5FHm1bFz1y z#ahvV8y9+pejDL+;bw6~q&`M8ROdnueRHcn;UK=8BYU_S<8)BUVOpDxK8~8#x<`?# z;FGSh4SPBT#dOAX9?IS*uRpPrh=ebY9MXrj^bv&_ud@j|9m`>HYOq1;HNsWbE9j7A zaF~Mteg9~|d%pGzDZ6$F%g6O7LU-AJA(-72l0+2LOud$vS-#G2;Ao}-Q$tr6xLM(+j zJFuN-b|0s`?g1+>iKHiq^2HRzr28BaNJ>l-sP$os0JoIeXRN8R*8MIRLiP3a4g$Bw z?s$#KC3qxKAR>oCc*g3C4Ar?KdqWV%nfEE<(7i;~6V9t}oqx3*W6mAqr%#WW;;k@V z8lqep#9hiYw)0rVhBZUb-J9BiX}y(Ew;bhtA@D|sJ#?l1Vg13A#Lt7#!j%IU0&&?V z^172-NE@pps);$mwwU4btFfXJ!i%%>*ZqcT5mgaO*QZX-WhCAC!Q-sf$LNrdkf&2l z;(ZORzcuuv=Ns5}k}I2z77U!rwIW7tW@SBh-@1O}WAj=9^0-`SRh;7x`zRyA~Tp6GafUAcO~ zD^dneV(#F@qi*w=!Xy!U74wXc#cwx@s^pUG=>usz!(~Fh8DQ(-CBe~dwxh$XEuFn_ zs1ho}Cy;t5p8m*hbGMekw&2VTVGFFC-Hi#&)9dQIIpOua`c^Je$XVF2QUx17+bC7_ zR{j8s(hKv8UyDeVguzN+7gYm-RqyjRiK#fJ^G(%DdX!lmJvPH+i$`){9oWRr{Kfp* z{13>vy(2sKs+cq#`n>$-yxCT`J;oE{HwrJ&uPw2>7T1EuZPVL{y*BvRuOT=jVM^fD zjX^<)5iL7|Bf(%_WD)uT4p=m`W{kc^*#}I=@nii(|HNLKh0c^FcV3t28~WH`3Hgwm zk~WD`#mERe7j1B&Ujj^J^zI=*Ltv|+1Nt6*-H%fGPAIV*S(Npahm==3pC6m6!&B2P|&L`)%cmnmxSZ90}jquEw%8~vW$25%0e{I~4EBDp>O%R5HPP}5qsF)G0u zeL>M2A0U+&<#E6T(NarzPpNCI(~&Sg)3Jm@a(52?_(tq*f>t9c>5@udR8z7nF<&H$ zs9v5z8#@*~s3SO?l&RFYR?_2n?~OaQse_>%5sZ?{DU>qiwQ|QQqbICLSF>Cz-@c2X zGo^cz4>F@?XMA7#$xiiecZlWHH1TM(s_dH>%Jd1cEUP+^jSr*S!$zU3N#*`BXNVqP zTp(AX{1gM+x-)}s>JEj&HRR6Mk&P8{%req_8rMM$Q=bPO78IYL_kK`3rE{zn=h6fn zpyaDbSozBA-uVxun;s|l1fgD=L%buQn58Nu7Y9*og^@s-jVe;PsW3^@Tx4dtaZ1*7 zIkV3em`0-Q`fyPZr9B;7O?sUV{rqaQr#Ubalv4Dbd1+{G<>)iM^?LtBT(Wyu8yKZ# zBU7!HRI1SBk%Z1Sm@hQDcGcLzk-gs0#vRKq)-f?RGrVVAf6^>>;`8CVAvnBdk@Ytt z)Rc>X{o~D``aP>3rdYe*B7C(?#$(K`uSCspIToDh(*MFzE;sVqq3%K!pYFi=#|dq& zAEUo|>ZG_rkbGkH*Kv}-Qu69JT&bZ4^xDB5#>U@I>0r!3h>oEb8nb`S0u0)N=rll) zZyVeCvatgvoPu{65|h(|dkX0aw7F{7S>m=d!E>1eXbbtw>XbsKak=sGG0Q#J1TdNw zTRVIcO=9$i9{5;TRObs1<9phEobgW&14ndZ41B1mk`BRv7jPfgOu7k%>JNs0r_G%e zh6PS%wwtb2xc{vC9NgEbSJ*r_JlLuGYiD;!o=|gv2P6nCv#4_uay$9D+-QUMWo_M9HgNu zLzpLw`9Ao*T^7ULwB-<#Kz1`ra&6j2G4h4IbSz#0tg|M>!Lb2w{5?IJp4GjtI(#Z$fp0?}+D~y!Y+y!q|~%f0RP_A-liXPl?3VCn()q z9)et2z1RZk&Qiwg5nXs*+D>vBPS-o{ ze72N&$8uZ{rI_wt34Hv~14cx8LRT%*NYqDMb1-5Vq3oQ4OYG_#Z?2WEaeRDTzIusQ zFWO8M=h=3rDCmWJa7DWbn*@$oB-5V)f^^?Q7m@7dqSjhsYmUpA=Gxds=TD6qh9&V~a|V5iau4qE`(v}qAtNkE z7+Nl;D8=$0WuIW+BWxvl-VsX1(i+ZnZaQu-)~%gv2FioK^~85A^b%z;KM}VwOz!rf z|Ne^E@26&9iy(=jlc!$0qcP=Hb*RYu+`)Q-Q|eQR&Og(Fg98QGC0Ar|OSIyUrd*$) zB7qC;))`zPLik|7d}YYwadhj?8Mejk-sEl!gBTm6(h=cLSSCB2GcyE`8*FQSdVNQgC7> z(H!l^Tv+n%;`H(aDXgj?bNIy$TToKo(g~ToQA!1ew2OZ?C$;6Y>u|(FHUphL@+@he z++4vF!PJ!)8k70y;h3S|6mUL`Q(33?IrzwAER#gz)9KvX62R!{_x*UB*GA~_OLz%A zDe2BPQZLSvDw}2Ld4-tP&g{p5GsPuA%>&dn5I;izS`M1!9$U*WU7{6H!!! zcwuu`JAys9`nnS@V>L(gB63-nBa`2;LRqnYNLWqN|~zJLqUI1YIZV+b(b^=vi~VwACt^> z2Bl*0MzR?c@0=kkDFDf|0?KASKR-gV74}$a%5oji)&1r2@E#nU>Ss$Q{Lq;_+-|pfx`(SZ0Qo>^HMv{8 zYoXCE{nQsYLM#-peV;G@I%oDl^%Y+vf$~_Zu++E_O7IH{FEle~wF4q0VF(b`3Q zeBZ}REVcXu4*S#_TWuP0s!uUm~UfnL19>^(|x?yIGcg?1=8^1N;jEq*CQ`@)~rgd(^e7DVhppDFzSsUJCw3#XOr)2mQf||j}F{hI*CJk5p<6a+#%tA661hu>P;SuVKL*lUd-Myq-j}! z`t8?>iH<|u(dXj%mJR&&*keQ9(#I>S632SWFw{IkNsMxPK-$#X@e`NAQeSxWW*8G4 zwDj~*@w#(4j(mJT&jmh@8ywTmhV_9)wk)RE$g}a)NK_#cs#ZtDV~BGMj9aFPC>6$M zM-K9Do+?bXs0=zwpgzlqT@*|sp{EEax$-IYq1@Gyc#kXid#A@=5Bh3=J=^EhJ2kAS z9SZ)em)$uH!fT*LDNrdw>B=qY88+`fbXNRjUwFe1G1*{8%N&zAw&TM@H`~Im`@ow$ zDf`V8Re#tw)NYQn#9_e{&-uY3#ZYZ;?wX6Hl`Z?oO?@}U+|IWvyJvkqIV;+HWT3_5 z#_crUAoR&SJG;9@PV%$wv&{{`%>`qAUCXX77jjO&TPWZ3_YfBDfQ?edr2mh)w~URm zS=t6;h-1c>DTbIaW{#PeIc8>NW@ct)W{PplG-hUqnc3dSIp;a=x6=D(rIl9uXXf^; zQM(Jes=B)RDvL+xd2KYcHb;o45a+zsG+cm2+2omm$=f_p@Lh11GI}$MaMJs#F zfE_+e8tv3GpP$UcY6DGbxk^4Mu@GS2G1HEPTuGKy;@z=wmEY>A#ki}r4jAknwaccl zxZocjg;l;%2*uGN!7W`o^a^gF44~skHW88CSJEPzOt87RO~!U~ zH-^H8K)UV~Z77|yKz&fTD+oQHGCULYO6`ak6OiEkEqSpaYnO$X7&3sL!)WBh+jT)T zahnELyR2RMN88oo_gf+&vqKUKLwqs1-c9woAxk9b`08(TljR=HFwS)-+h13hS_Q*= z%sF|4o;T1lfG!K&aG`16j+$kiT!z@> zq=>vjAE)dy>I9gJOL(Ksw(&9Y*m@KcPbVsrjTzYc)sgr@Dr+c1FsE3<4(X}v5L6UF zGlUMmzs%S`VbN4)bZ=W6ojF`Uk5s7A8kD0zfQfy$BYr>HU-di5$-j-z`)8$#CRr4= zM67b0>)zMqPHtNrQcCJnQPC;lrL8U3Q>h`(Ys&9BSVCtS+HQKT3^g%sIi;dQGd=ti z!i|wys504o^qazQw`kO9jCR=a>)8gUxz|`X@lVe{Hdm)LGZFYd!s5C}Zj>frF9*~I z<#F6hZW#QZMX^6l#Wwh-BD~VKN)G82zG582Tg{=x+XWkl+W+1W6%sP@uM`P?R5G9P ztrksVG@=^JIrW`7ruB5_CW6qh4n~T_Yzgpk+y2VM9^+zSb0|>#C6e>}^n=xk3bTLD zIX*pwznVNE0nkG}I2P(-DlUNmH&6;uVZ*LMNC*q7t+m^iR+Fc`X8Ot-vpO>9AbLgR z!lomFuX+SSs{?Z@$&s@?Kz;rR^KJVzox>BmBS9*2uS99A@#uF}j_}$@1bg4?lks!IDYv38(*w8O1fjujfHriUZCFTy$6>>H zApLqsBfw-eMN8C8jJMXG{$c;}G4~zv`^4dtt#RgdK0pA15YF9u3s=A_f}SfrUTk6} zAuUT7Mwe+3i$aAeea#CnOz9gx?{T9PfN!bwOj*ntWe4!$*K(yMuu(^gK!iPA-aFJ9 z$V?ijyU04~^rRq=4@cHz;(RBNegE8SqahhB%Usi47ij;!)%g8tzO~LXO*H|@IGT8= zl-%i1NCf`;=Wg%jbcq?X&OI}RH$!Mt`$8wP5lmdSjEq(0NU2^x+hH1tqm5hyxrp>{ zPJOHhNgnZW2dmaJ1zEy9-yC{jcoDQBs=05&<9BOMr1&xY)T_Nl=aw>7&(R2JI*~X$ z;k|Jfth88zp)CNk=Y@zlgk7A8!8BS%sq~K2OBqE_M)>Oouk*d<3wn|{Vv*T;GBbW> zkRMZu?YBCfDiUc>s`iH%dtKxpeU9FvheH_lsbKH+6!xRDY-Y?vq3)O%1^Yyt^gDdC zx&X4R`MO6Z^9_D}z-F$QIyE`ll>Tc7ZmZW3mQGg{eEX?ubBS^61y!$+M~}Xe8tKge z4e7TBtcxCwNb;c+kQvBg^nhi5-^YzyD`m7?L;VwsUTgfi8;P}NxYY4hdUP^aGa}0e znCbUuGu=jI$}1XE_#rnXOW6`ISICY^%V7%3Yyvg{19&u__!ca!CNZASU1;fPfYO z_|RJ_diiB7NK1@!5?hgA(N#>ATg0CQjxD#*Q$-ec(q!4m)vC{PG!>}+y0IIXae5F` z!l~G|I!w>8)U(6nPkfPQP?7vKhJBE&0HcAj8uLMwRdGN)cIddNLEYsR&t@W9ya&}i zx+ZNva|&UQ=~$3c^%jtj(^pMfXQ+{nb+>|=5B?~b$d!{tZ4BxWIhKov@bu2krnS7H zZKitXhC%n1ZZ3QOZG5ywVbb%yQd7!Y&cQqd=8y_7!$LDL58NFjV2mo#=4AS9+=M)M!SIwfirm?hB<7C^0~pzPc{F;XPD*jLjXPWa_Z_-$wVD zj?q3nm??ZISTNA^(q*WxqP~3Ga#!#b6(=8?dmVM7V;VhW$Py>o}!U5*nGxsw7h~!D$tyL2n&x~ z?Fj~!MU`S(7lu+_aFg#~Fh0}bl8AbXNF^o>G+kWHruwyhi7HTB*I#m=xAlAiL@wnF zKBgYS!(P{3G=z1&OjFldg3}6CX-$bgm$}M(KG@ktb)l;RhA=!yxruaf#jJdnE-cR4 zTFY}3Fy~zUL7lY0DCvr5oAvB}PLE_HKP^R2x|$AiAw zzvA)xOz^Ydn-?39kWlbEfFun?@e5q7!z4r}yDb`oj(g1NSQ_QH44L<_fX<{>cnrCZ zNb4JO0sy{bYE?uf`q(ds+7U{(oV9e)B}B#HQ9x5eG~|;^eDZ{GR1w#1`IEgVBVRqa z^+85vx>U~7HytSQtFcWGr8e^S5m!n#^>rqb{hcnka6o+#G$_(4ZE@>Fu#K3*6If;^R(T60eufXv0I$`4gW1DAyaB%e`>_Y zVqB`8%CXQY;W`y8Ko|1t{513;O}R2`YL7~=j_ekH$^RSORhdM@>YF%MT~5U3R|L*V zzc^+yIed>Yi@Md1@BAb?s3qsMpWen1QzftytO7j7Tqa>gGLz3P+5M(U$Ei1 z`q)SwDn27=T6A`H+tUgQ=hv5qX;%%8`aiCz*VsXgAY>{HT|{`Yv)B{D(8y<7^q`o? z>9-uCGl;AE;CCLZew{2JWI84jw@YE;JuRF19@y8jc^8FxsR~ znykj#LAZt0(df9xj%vFrG_{81G?WnPEw`+~Sk8p4p!B}dgVPxMS7LZ;e(LqsdBvE> z7RYNt?A5_!Z*+hVRn$s^p$jYchWXj`DJj?cM8wr-sLtxfygQXE3Y)2$h;nf<>K6>9 zaD#`t;v}J*b^Jrs&RD;U`~8Vnth{X83iM=XYwc-?;O}@B@r_*_ad& z+`0wP&3jVpC>D@IXhPOH&HtE0_g$_5#}95M4kbpMWrAfWCUbbUghdpE(p;$gR%=Cg zUabvT8!`WUU1T5+TgTp&E0Ul>~yPnog zzm;}J0asV>L#in7`WVlIV3?R)+1z?;xaCUCoWvT#?QjAzK~AK?;%xk&K2R9j(hp1cA7;d-$4i){Siw&)Y~$mw$ZU$q(-vfjEJ z`HF|LNa0PykWcjBL_M`VziCIdI%#891Zd!JW~r;I2#eC5I4{(C{~Gf}RfH4K8c`ZJ z&5ZE>f%PpR0H!l%k>A2Pw!z}WqR-PI8He#&=%e7XC|rlRkqzwT+34zV;{cl>*I*F`!Z)x!XF}uCkS*_BaY?Txel*x)-4Gp9{ADuwWvPEm-@7O393kklN3Kv#Q!rPP%*kp>3 zHqAcYl7yk%pjRG7OkvJ!ufdt6p~6ICaXOs~kKfCt)fI*I|A2}*XmkwF9W_5H6b>SJ z%+coum7L+gKK@BdNmZ5u&->wAsV-WLA_-@M9ug!P9G8h4AwwHM?(eFK#pHnWiE%E} zMlt{^svq709tOtflv_E13L7xa@`WKl`E-LhC+o8l zVFIN~Z(9iY9e8d?H<-)V> zLb|-|7T^`E7bQBYhM95Lw;YEe&!4&jEU=jlK5s0Y9Nxo0p<>+1@6(c20Ph;bujFWH zy2wo&J!4FYWlTih7RKFgSOl!B_2n|Y(s-`&7k52!a`m^{X~!(MR>n@ns2uEMoISDpUY#O6jyZ;8@f-hy~6RVa8E0{{e<5!S7y4L(f(Yd>iSp>Z5@cG&>&l11fNjxw^-kf)eo^dGtED zftWu0U0uDE7TO#4CB=Ju#h>T$RC~P6HZ-io6tnG1e%WQnWA!I43@IaFaZ*)BQ{O1T z%)6sm-fTdM=SVXDnOTBqo`|6rPT_#w%##Rd8G;kl;YDsP?9Qi<2?2L4Qa;uBdu;BP zYxW<(-2yctFj%iD@cMsT4l5TDT>s3sj99aWqmAgYgrWB`S6i+_rtl+tyf=B)x#EW5 z)06f060!bq`4e?ApOV^PNlq(M6NBChV;US3+G-e|pm+RYQ%`b-A<}5!57@%26Jh(c z6x*!zehVjMvF?AKDaF2gs4KH;QFHhvyXr784DB6$xo7$8B6;w{1^^b#S1qxR>N{=2 zN){#>9I+DbB#1TtsZ8h0Zu96nPGA4jA-vl&nk2N@Yx3hFna$OR-^6-st7OJ8vefkS zC$g>3z>lrj4tAa>wD5`XgU*VJd^31J*$!zXfrb0v|_XCF;{hteFLh5bRatu7(}R)wRpoL{a63}Cg`@QktX5>GSbyJMq*C_Z6(e+#f;#GtrLpu_nr*3|9UvSpaIwu#hfyjc#Zj|aEeRR#al0$^I=&B8u_P@J2K9duVoDOP&z z%AFaXlGq;2Uw9LHQbEzxtp1boX zW7+a_pQV1*Q&Dsv$royJMSCW>g7MI*1Kx?C81T0f^AzNdL6JEW+#@fWnD2lZ$3DCe{4S`E(3Lh>1Mh0jI zDFRk&G8NNk4&4(AOLQiMCKhLMn8SBb3o13RXL<4*z&RI^?{4im_DdLQk@5DV`~un6 z0a7gJTy=x1CA=jC?j@eOKn@l};JUmkI2e_ew^X!AjK$B3W?D5yBizsE-uY*Gfucod z$IiJYoK1K_AnT)wXSiGc-Ev7uSnphNjq3uvTp8W^@Yjep0d7}FHvVo}3rUOw+4a$S z!VJSz()r;}e&3W2J2YT$6o8IM_bA-Cu6hJcVfi2GD2PD3`s-om0&?CBtikPs4PIC%sOAddrV4ejR7 zr_g5CN0v0m05IkVzq}K?vD%esX54m+K0<~WW?YOc15-6isP+l}$s92iD)H48`Pv3} zV7?wWjt`CTFW$LKN7brT4-9 z*R!Ue!n;=A3_8w?*54{aywA|{O1Y8Oun_CDxmSvy3BW3{9PF%qNI*mkp|pQaA;+Ws z;2~@3)0QD7-ceIF1PCD4UWp=Kr^s)!Fo57zdTmIL9#E6!qTy`X(}wi$`bE@~*f)(; z&h_3;jg%m-v0Wa-hAifwmG%_zzt4*30%k} zH}XX6tYJPMP)By#f5+T>@VT7|KTy?kHRZTg;hbm>N$EKqX$#-|e3QoGgmJYl3Z$Em zK>$uwH^Hej)Wp`8eeBe`shFpFQwg9~YPP1nX`jUH%prS*=sir=O53=%+{OZ#gMD;f zm@i~&#^&ENwm`S(wFQ9!fdXUzMro;{S}aZ+a((3aKLXZgKeAeMo&((vEiw~Jh*ES| z0}pv$eU-@%EGT$mYxn0Lm&Wd_j>q;fBRER$3T2wTwQiTsziuwsRd(jllJNzH`K(G9KS@Ki zSOoUkjA5*X!_l3LRmt}BS1zRJU0X0**{F{$s$nk|6G8PL0GS!3(RvBvsk9~!fz1l# z;cj~e4h+n{;pKJIM&I*4@99iuKzBr37zBUqjqOt+BQJ)2l9saZMJ6vOrJx35LaFF& zN&QOa8P}2X1}oZ6pk+5mQIE#V`*hfLx+SVWRIUWwG0|7YEZxs=derV$e1(Qh+IS>E zeZj;1F@x8>Pg`J?ax?6x$r!;dk#u?2n&Rrz=h%m%)pyjBBd)b~gT&<6L$q}J9C;l_2R31(l|JUvx?ZCT^2vPbIv8ZtE#7`)A`*&X`Zk|7J z&A$qJEGsAoJ3nyZ$!my(ZIX`W$gsjvE)__=hq*1i(I z;sDsXIeUeCv>^*OZS7wMk8r1;t3DVzntFP7xp$fYa@-#hQfMC>aAO z^`0^An=|U0Gt7XAg`uVLr8iu{zZk|;2~H&$r4g(3RHZK-7i~LdqI&JEbWX@o>U9H_ zs!TBA(bh4_21Y>{<=YWFz+fM`f+-w}oCW;zjapX$kx6Ds;Lyl%aMI%(Wcb2)?l>;# zkg=Z}V?|0;-}(+bPT=xo(R?p{bNob`cm0o3CAcI^1#E17Pfr?*ZsD{?Pjm{9$*@q< zB0xoDN^NkU-}s3fGY{@cxq;c2lqsE(=Y_>^iyGZ@*FW~SdaJVbK1_krw*3mn9y}7g zJZ1mk@ulsWzGa4kB=f?+p(Hl@l-y?%%v0K+&SGm!8}OR{t)!_ef+yd%kb)4`g&@$G zat?a0&T(Jnhr70TrzT)`R(=p2@)K7`DpC2Xjx9&sAI?pbTSbD~T^J<5mTIx4nAk|) z(-Z4GM#>BW9aNM#n6uS}fRMfkDv6dFZqasDTQdGjKp~lWDGG(&OY&Q*!U}S=*{`~7 z2AZSHol~pUojM|Jmy0p(=gy#xYFx4Su_3|`VjrMA##s<ydwzaG5Jqhs(8&t@I9b9F9z;)V z9+wBC+p|wFmUH6Lz5D8Y8>3>J^9?s1v!z9%#SA;6n(7Z5S!=h6Std3eneFl_-Zqo1 zG=F}Cd3hjhSzv@{3tpmOV{l_o?{82=Lw{9NEBq}U69#Qf zAzhz4;h9`RoIOpue12GWATpEjRu1kWBC#PHnvmt`yL}D|v~^J1Z%1lfY{D5#<)Z(#V6|NvrOQ3=(bbiD=J6#q6FWM1%fl=X&~kV>rf~_nBun1W z+jTn}K17i8(_V?vV&Rl@niYn+NlN##$HO5o*V~~lQzbY2f&p(Y(}U2#`u)pp32Q#p zfcTMNBUgq@1gNz}&g8dI!kv=ZK(>QZ3sKuK-RwgkuR7j#c%I-1D62n&hjGRk<&9wQ zG~Z@mn*4Ptu;_%6uY!Z1JI@nusfm!dKGcPagJ?p+{KDd>V~c7dM~y!$hwy9p+Snl) z-|tc;f9pHi*8^raMvg;U=@yR8%ReuqU!Dd9_S>Y=FTt|M;~dACAFEP}!?xHA(#iOA zL%YUI8g!=iYHGR#7hR(%RxgQM)<0ky*wAMh3nDaxza~Ett%x-F_GEURd5BA5`+{Ig z5(OowN=r)xz&Jyc4(j!#zhAxKpun8&zOav&#!3vY4KH+o{fMu^+cyG^24PXAT+J~p z#zL3=;`Jkro@Rll04<;W=ptc#Zr!R4A&#soxiJu+d-q#-BP%w0)vKX~A?_qEQxbKK zSFlxCoYqVW7%PM)UIBN`eg`Wu&kk62S344`Aj#3;0+F7}vlPyZkMK|EuG7_t%cq2^9c8Jgti{tlGXo6k3@XGEV$ zM?DXw-ocrgFwcN%ufDzSZ4PP9oNashG^h@4}arDq|`qa1?H`rHA=$-_@hVWbfEJ!l_ORg{~1Rx zG&D&-Hipn7wOVoh{<tHt1fG!9q{!(}XG0_?TF#B6Y# zv248EXq0sF=L*}<=4x=6fy6j12M0gZxn#76a=taoh@Ov=nCi{CocKt5>KH9Opj{hU zRAh_aC}$BzaSnH>FDtAI=wB?ukL8@w@tHqfxfbp2ozIQ~C!`{bh#c{|KZ(-8D;~7sNCjpn!Y(gotb#ZB=U?)xH?T?!lx&DJ{$r>=}rRQx|V;gSY z5u*JL^^}g7@^5?;_5AehYzT}`RjNx`Qc=DwzRC^<_R~`rL&;0=5jEX$3vD)+kEvId zw9uCxHv=riqAi9holM!n9TKJT?MX>(p97^J`9RFPmj+VkzO#-Y*@x!fI{=s*6u*ZA zvEckHDLs>V(*NOB#1BHesxf|A`JVWNU8gqwj{2~E)Kccfy$)~ePjsPP1M6uPV~o{j zqdAJD-v9$%lNfqvrP`|-7%APXao?L&wBE8Y#U3`njGzYB1Mu+jM-h$+6I%WXzS#4- zOgxo(8~%5tmMbquNm8C?gx?)@u&WLc9BHHePMsf@2{hNN2eX|Rxv%PIeV)d%R`1*HF~eJwY` z;gfxmDIV7e@7egJG31b0duypd%AUsOO%R0FoHaGnHJXfSZu#*nd_Wjl>`^i)v{IJT z5s$=|Hpy@E$zQlK^Oce5jz%PV0g{UxGsO#x=Lg%(mB;R~0`e!R@Cb+H<_SrCd)U#X zfp-DKt2(>q2q~Qr6pW07qXENvOuXNRpoT!S(V5;|BP)zab`33CB@4yNHT4TA{3)3< zcu;(j_8q+g=+;BYYkos!b*7#X{elu|G#i~=O6g=ud)LbY;*}J1CBZiI4Jp>-g_VUa zSmWO@9Z6S!nR{4>VWeGYSwGJauPaKoLnn$ongeRcWQ{?_$2{@k09bTFQ3iW`Vn2J` zh|;3Ay+-TJv|wyFP~9g^wmba!V%*H``(){S6_)qLzGKw%v=hE`frxpBWLw`03}MU% zTu%3a09^a9Vhj0Gf)ly2cYovLVb8C#TYg8E;oZ; z3&siC?byn(4D<8RK+EY=)*^Q2^?>eUA7UU?n6(p_OV)A}DS1dB@NVa%?D@zlyKR^1 z)tTCE;>F?kdx<>2;JcjKc>Rbh6W4>67Hd@Q_soN=$%FQ8d^>EMTr_M=P%Eg-UQlOQ zC08=9mz+WDv)S5f?eyw`6CBawz>=t@)z*v(u2*bWW) z0G{s^*St=?usiXMaHQC;PEQVSg7HKJzOuC%&1pFP)@M0pKecvm5+hryhTyE5aCbOy z!ZFBhi;*|@3MPvaP~D!)n}(7ZTU=lP0*Mv`GCJZ*?-nvgBp7#1+K_oLuk9Y4{@`3U5#wFr$Ou|rQr?gPI)EtN+-{k zpAgJo3W;}nW72Y&v7cmrz?tMNphWbR#%a$hFw67}E94 zr8W{^?jf~j7s_3pHo!TH>FMZso7?z{Zd~v0jvPa=6UWn^K);rm?d=F`uX*mZ1!Qd~ z-Jh!%1~beE>pS`Dp&qvE+1vY~^t)%d5b>G}*ZCHvsgUW#c_%oa899RH2mBZU`M^~H8 z{c8k>KBgD@eS|NMFB7u%9&WzmB2lCyqmvhdwgb~|`8}uv@ANvaC&HbTm9aC~)O2m& z_`NNERo?-RBoafg`mj%W(e47FKdhM==N?N;#(=G@U4&&_JB9kW#nt<{F`5$fh?`$zmhySeT=UB=b;$s>_8cE@Nmlnil|T{Ruu*5I+{#-Z)zZ|o94 z26%jFU}@2|#JKTN?0u!{T~4IMX7bnyAb#Ue|2>X5i_t}QG*OIe{au%jKh-Qy3iX!ZeWwEIP;+^0V=84}gWyHs=3AQ5Ct zPMILD8Zx+WCr>VwSZ~#<_D_RV2+v&2`Z%a$D4lY$FiVRrI!%M<5Vz*>^`=d!b-q(# zMg;!+{CHa@skX;hYHZhjH2pwUeU>b&FNGy)ij(x&jhoG#eb;ghR&qBIPK2@3K>I(V zfFZ0;&oG;tnu3TcHp3f}FF_y)Ml{&n+z?s>4eFRRif+($pFe^%7cMrs7r#o77ge0+ ziSBt@6OnepT55bJG~IS_r?trvF+hRTgC5c^ZfA$hngL4d2`nObej9hIG z#c*ZUDhgkh<1K8krC&Im%}qH$OIBVIq}y2g7kw>(h>_*Q$1Mfl=}GN!bqjA{b&ASR)@+OCpO=47LPl0&CQ=beBu*1*5MViy@f`tO?RKZOAq z0c$pA=5eC$vqO3y=aA<|o4i3HQh(rQ()K@yj%rCOR`vC=KDQ`SX*| z4Vtl(7XQ`#D~nL8m7ECC)BQWlnuGqsbs-@`roCXUBRM`;{N9{IR^B7+I9_s)rhCj#aeVYg$HGTsuaDpdH_tngp5tWxB0Rh(DJ9lft#Uf959rtB zn&J5CMk&Ao=fow)`pyt{OE_IGKee@Wuwz%uOEefSpJu%wiSV>$%3pNJ?rwSCpDs_- z)z{FAEiS@zpG$1>640)vSL_LR!^*NpKMEYbMDp~^j!t0X*B|W|1Wk-WXRqHwdF$oC@ zv^O3c=-;Q9%@%Ti+K(K9Anx`<`2!|*IwUw$Kzv}fNdAji^A{6iD`pHsrhikcca$RFApJy;I}FtTg!_SmM;WG-2uV)ay}(K4Cmbb{AywF(v8dOdmq zl2!?Y)${GKag9zFXh?E-6%}EAehxm>VO5#5sev3`v#JQHlF15+OKQ0p><0_lqgOa$+9 zIH)r9a;qr})o;f{8Te#A2MmyphVM%SSLKgUI^UMZXn^e$T&U35|8}AWn?L+mxcbP!l86bo^SN}TS$$~`RWT0Gc2yUa^o(&;5 z$i%};{~tstuD{2He?@y~ptgJDxj{QO6aUvbuqFm+g1Z+kp$v_{P`g==x@d) zgeddkp2*ICL8XTFg0Lvtf)iF^G+=2+hQZU=4TsuJQF2ylp(`FI3rN}ZyeaQ(k$US< zsRD=?GKGXix9hP%HX}!}C2qkOnnVMK{gf;9={LI!nNoea-N%#f?G0hBgFl>{YXp)U zjx91eT9`CaiT6pgDagZgzTK+uQh(`0HTAbYwKe2_D7qCXgiN?BokjMcMul zYWMxtU60J3srIP1ze@H)?-}fQ!-3OD>mTib^xw<@9en)dK}q$mD;W$a?5PvU>(CZK ztpFK#9t&tE5=+PkmmfZ(aGeY_;kTE_PnzXNwlhtcOtt9NRLiU`I0k$ffx2)Ie`a4^ z9gf$TvwvM7cE4M48>C!(-+;PO*JgKE`ZZhS=6V14apljLiT#iR139wUnMzOQIh;TH z&H7vH#rNHDIINX22++n=;(q*~29sAn?ZdAS5;0a)T}p>P)VM@@psAm8n-ydgdZ+UB zoLpxTWF{`)IoYo%!4h3d^W~LpZfAp&A1a2A+8Z(ye zn`|!HTrDkbuW>(HCP%0B%XG_k#(i+RlIIL+O}u)15uA=;`${}(b{L{u``A^NXt*u} z5ZN~Uw{C#g;{*90o@JJITE1Uf5u~Y2qJaQYc)mO$S?=Zv2mC}C)7+8Ov$>3jBvX& z0RZZus>brMcWNA3#WIH3MHkAp+XI(%i zRqyWFHH4OO`Om45&r?0nn!I-0m~q{bt&sm#h~6$2+|*l~9iB9&{f=Y>7&p5$}r zFI}*ocDl8|nQD2mKV))q`>Or15dB{k`GdlX|L77G;TPwN0`nF^AS$V0`qB=QxU(kZ znD)@#@#+z`DJ1s!CqG|tM_{yJuyeiTlQR|ULp|_t&^kR(05-uEhdC7LZ_XY`x0*2W z8xe3ISuVyfC1_W<-f08xfMDEkI2XgUbYI3ww8t5nMvFb6bLSIFKJb{clsrvdhC!>) zgyp-4=2$Q{2x54P&BWezmDre$y2RmSh`oxlsj{#cd)ODKfo8Q&p@=e?F}D7&Hnyhl zMG%B5J7a`g&OUs5HPt}bhELMLy3K{hxaWvJe3{ukXKVc9HM_)ZYe*8C+m#(Z;Trp) zroz=ln34=YWjyjFr;f6j)`S`4cK)#^yS4v*+)_dpZ`i0ZZM?J#eo&X%jCk)`Ue}x< zCdrQPgV;OM1=N464Q4X3MvGNRR$!55k2k}cXO568KKzoOx@kK)Winl_N2!~nNYB{_ ze~!p8Tsn?m-_2jeT4+LFc0#MOh4?q>Iw$4g27%KGZ4_de6 zTg*)(;HDFNLuUSKX;~EOoMy;fGxnzQ>AzB#yftFF>5qFTwN7pIGClx_BL)2{ zpKtjIa`3nEB{O1sjIdwAUFvH}!*_Tx)S5VhpQAL8&$cMm`L*1=b{<6%M5n{)4z5bF z3y%{XX1M)vdb@BRJ^NXoGhIkvp#E{by@a9vH8iQohjupzoFhz_^;=Q6ZLdIU?%Nih znMvb*>zc94)$F~#OeLAUwN+6E25;hu6e<1NCPJp8bf2pfi+0pi(8+mQ?>DCdVvI=a z!9Cgbo2f5^MY?37*ea*24K?g?2RxGxY)?)YuWL3PG|A)MYf5`iRo3>83lid8ayjDjU)5#vqkr95h z`aSFME8iIF`Tk$pgko&Qq}kYv7MtIt+}?6LIgJlO!*L^90c79zjw!!BSxi94#78J8n-EmV^dUrFc_!&87zBww@VvfZSnkPb6N zj@jA6!tXp8?%^^^)(3e=JmT@JcPYM9eTm9v5e1z~9JU2WE-!mxrRXl9h4dm_)_dJJ zb5|ph3dxV@P7WbdP)M5#;KzXQRsW;CF~a{wsq8H)kK6|u*O@+XB7SwZYuyctu@XHz zfKl7c`0;pKo#@4xzSy%N42op!9a?@^jaO9Cf4Nwoe>^%d+B@e8J-0{ALlwhbOEVl* zd)}AAsL`78%cnX$XW!Z7oGnx51~kMjyjhAEhulnhO*JsPV1lB2HJ)P_GW0oV1zh`knQGID} z4H3R@^T>1PFtn>HH_>q?Y3;FX#cI7O1Da^E*CVtI9sPm7W({#ae^hRbe^-a3>0Bwy zwl($tBG_^Uji%`RAT|Kg)lbUU(cie|xsocR^BVB&0VRLf_6T>g4VmT@usHvK8l-)a zb*W$A8@tIl%hSl3RK0fX!GmXBX^h%lX508q-vckUJDzcotJuW={U|ndE_%D*`|1n0dJm)`B{70MrDs}XKV$iLrk(8Fe7BU$UkEN`}u)nKF zmU?ya3!$t&@ErWIaJt|l&CwJ&WKuq(_&ov%LsgE}W2*_?G;RQ86cGJg@zHVRhs2Ed zDAw=V8++)((j9~Z*Tb9dUzG)vK(W)=Vn`taXwUZVWz<_H{-0HpqgeZkNs+u~Dp}u~ z!-ukA()8;E^M`YL-@SFPk}x~KqOqJC2}_2HA4p5gFLRP)rNO4kow0-sgU z6urYAm6F=L-uJ8<8XErMRj+pmu4h%tbH}b^f?po=Hs^eP9Aph9rs3%;bUsOD`y4R! zY2F&C^_;sdb^{77CJKN2`Y(uLX8J!O@%HVhPjzzGB&o^GME)yE{d$8ZAu>8AM$b#+ z`V3>>!?>QIXysm5OAi@zX=hRA{RoYQmIK5(4r`uXizN0s@Yk#ZVLUFlznkaf4mH*~ zy}_?36=t)QI0w4p-dEY7`dzqA!qlbl7?y!WCji@iW`NE7ks3~APhcqqowR<8)#O>s)4*eOWw0Dcr zi}VQTj?;ys%GW(P%;=IUF|a(6>&|{bdQIX>mH+h}7SyXf$@|o>rb}&1Fa!Z4QiIJ! zy4&Skp^>g^EPbeES#fC5R?X=Ae39 z=77?RVmB=P4Ly&QkLHb(7XLL!nxUPjJ^g8Ls;af^- zeR<-Zdz4s=b6k%FZIA|}6Vo5m-DYHr3<2Y};DL7d-+%z0 zA89QqILcg)w?P26$H6*FgdzDJZyf8td^lmEoX{L>6bCM=V(h6EQyJW(^9Z>s*m6Av zXAHn2-PfO;(tWWIpq~7b-uovEZAk%UkAC;g4ENai{#(3NY>yl7moO6El}SB17aB%8 zk9=mRR{53fM@S%zYtuFg73$ zhB6)&{Tc8Z%sc+t-^i2Qni9e0gR9Gfb;y{?KM%QGugFWrV?n0Az{n6-Cy7K%8y?T1 z_&C*}dE^0rfH%bpuy!T^j_;3RUKB{LV;&!YCuTu8wKJM-9S>lGOvYBvIH_3DgdZ0R z4p+$W(kWE?K@rub)-^+<))=O1o9&GX6Fr2axS1D>;5{LO&*0q(H125n{Tu>|Rce+v zygfOwJ>j-35s)z%B=FSE>%Qew0{^HsWC`(qY=Vq@p27l+&Omo!14ls^T7X8n9smmR z(affe#iqc1d(ER|giO=#bNx=EqP-N?-p{UU z1I-p9nRz7dIXvuOwH?(LT_5jnA0G{q_LFI}2~rr)q+t3>4_U*};Nc3Y8WjpHyjC^q z))o~HW;IqhD=V5bP3BE%If{EO##6_uc}1RnyGzL+cV$6v7}U`I}~P z6>1u&Q>=;lelcLdLk0ByTEhJNp9j%-GVCR05z2Y-^c1=*u22Vhu4rkOS2&-$GQM3O zUAyls)gw#=?n(M?;SvQRZ3@L&bc%E-W{!kYACJ>t*c;jWdVjeFjVzWak0y>8QO)BZ z=$2?uf__q_QsL_7Kv{oqMBIsSr!+?6&l}lDPlG48KO_Hv%VVQeIvWU zJ+6VmV`~X;wC45F7A}VSjN^y`j-vzUQgoAHHw61Dh`;3YcT@iU_XF>!lkJkasO{G8 zGtY+&lQi>MWiNLo5+mn0{oeS`q<5q-+*tsK^mCK_(Tt^PoeX_q!r7Kfv5hxXHrv4v z**ulX=}-A;>D3&P6v@SA%#hVt0nxfceuFQqSCb}`4N>I=m75GQ5FmP2kRkCLcd!dt zH+uzT5oTje+}EkRAIGr{vs}2Z)S?5TxzdF#&W-k;8Jo+dKXxw~%1uxJbX>7}`HF3& zeD>Q8m({_)aw;bcp$rhFe2VD`llzYkGD@ z6Q{Z3MvN-ucCVP!+kubpp9_SbLR0YvBC-+RulYRO89p_zp#8En;d(89!1qX_#ESI@FD1~#~AU(L}E`oHb|*8qel zb9Si6J1$9k%m?4Z{GRG3g%Fp%IsX@m+i}B)2qJ3_G*?YIbVv$)^LHd@WtnDK^b+r88VI-^~5_ zs_w2XKe6UIkAA~3HUF}Ah0TITB7h4%LPdJ1%kcKtW*^^EyIx)AtGKu0!OYu~Z5y?d zsif@Hd4k7sO6dD6bGPu)xZcuHX>^_>AZ1s(mSuI`V)dn#lxFL|B42;! z_gRO*551qhW>r;ZQVCmn=QG163*{)6WJS;}Ht&C4mCI?%V1<%*p^{;w- z!NKQ<|IkY=;kUAl`8YCMYQF5bJx*Wb*!rsddXmS-PLVfCHPN&y|AyMHHY=HUH2NhK z9^P4t{`7+2;5by%J(nF2zv)$F^!(e!utPOc_%IjYp%%h3Z=B#+mdifj_^vaFX!`W! zFB&SfvdjZZ(kbUwZ45u(<^Q?9Kc#fTFDvJT+lL+J6*LPtuFm0$N%HynT9)@3w(o6v zEZb0Q|CCFONy}W>A+94EgJkTTZ!4b?NvTE$L|eFwwQ$Ygua*51`!@fq+JslInoxl2 ztufS3)7OmL_zjYd;*&>B<>v}+a7`6l>)xFi*GuZ?*fy5U=)CPqa+Pmmzo~`>UEas~ zszYxyO=L9j!qMrFo$@u04`U>FO^FAjgU_VNu-NhZ3d=5@JAUJG@pw$hVp-MDRIhhn zMy$c>yEd4j#|6k*Myis6n?HeX%6yl7WE`g(Iech)Gr zQu6$kJ(?Q)l46x`^y*<<>r)|Pn}#}a1%D|R;P(duPNqygUYsAbSV>8hKUp21gclsz zA1A=}*NKcSTx&w3#FeK7KAHMCk!pmfkh?ddyB7wPhH`Ta>#VAlSLt7ZwbBLmKaati z5r@RHBrarAH<+B5*Yms2K3#Z7{3pH)dVdqKrm{|VvU?@FWvb)zUrW^%Ieq;;dTJ^g zyXp#hNFz_Df=@0lsXHkUdW#=y){+V9#Bg~JGE3SHQX8hjb(2w{C0@PGy7;xjJgY0?7R#7b8l4Nq3*wXoxKAQlEp+|&o zEXtALOJCeeWXBF4(Ri2 zi9cRhC|K%Stn#ySv>*Lz;#2w_K2@Hd2JM?6XJ5-d-4rXO2

yR*yiwNQuIcwkShWM|ZyLnPoC|Zc=j6ulS-IMd$QZ{b)58wYyji3I;{=?0f^f1}`VPVWa zZvHwWTdE&6K5qMa(RCiwSa&bXjL@l5CONqSc|E2-BM|;_ zS7nd-`QmTT;9=M=`mJ5=ixFhRsGVldJxFy=!pjUDzjcstLcy48(jmfkHsv@|r>@I{ zJ-IK5R`?RFdYP3$X2_*b1+-BC!+y=XHtFawj2e8C=&R!cg^vw#IT$u7vl?|gZM9_e z=SQadMimS~rh7l)^y+vmy208HRX*!7!FcaZ1%FVi0PJK`;eVfzLHQIb1J_DdlrJg? z&tuY`+AO4I7~PLi`%e`Z+hJaz15gH8D1OG8t5KFwwN1+GT^JL?1B{sR ztFEvW`e_vX(*Ll7otP7t{z7$jG8$#=oRQ1qKWluC8~PXl`Gk8(K{ z?c1k|YDERb`k+w!L7sl71QcBl1uL>*%6F|>%H&*z#aw-reoafmW6K1>S7PZ>Qid<{ zn-*&IP?aZ3X5~=0%=1Luy03|=kvF!4_eT+u>YvHvmgE~Q#f|W4I5c)?J(MF0=SV4# zvMJsVQlH_1?_6I@t?v?(3q~^+@wM4OMI=Q#IKJX3Mt$w%i8k%g9DUtT0QFIeW zhvW<^?`67)_0KAKvBx`Z!i-+*dlJ)YKE<|&9z^@Rkz;B4jW^AEW#uMywvh#|V^J&o zP(D9F%xfi!RP}APbU1?9IZNBvK|m{CPnG=-9=bF~_@Z$A6`9vWqSR2sBO|7|q33+u zDm0@}fG#d>?uR-@#VyD@X@^a;P~y=y3)L$OEU4wKk1icK8nZo7%ErXaEx~o_GaJK3 zu|t>>dw1R}+)H%_$VU6(nK2|4_O`aRkdTn$Xf)==jeSua`vK)9LR9zrbpp}R(ZDcy z-b0+h`Pf1|b2qoW6ADo-N1lE-x=-5nQ9(hphEUBos9zFbp>g3tYkRvt%0QGxqlJWr z7Z_Co+sz25zKRBP!;Z+>^K1}zzlQ-^cs|Ip^t#LWby_%(AL7-7HYa>eofQAPIV$4R z9Tdu7cYyvJhWQ~zt$dS;)$`QFc1;JWyAoOZ7+>E4KR;to&&QGl+czU|{Odwpk)>>O9`Lf?L-kl4G<50dSr zGDQ>5le8u~Y=leFoexyu$r4GM-|U;KE);If-{VkSjZTnKnml8x$3aViOm-|X*J1Yc zC9>iKmn==NdpcK|RcV~--9FyF6^kuaFG&5qgBEzw#$<-2cO7DL{WHZ9*ryw(WTrxU zy6307G|{u!BFVcgbiai8B@sTKPVnClbj-JQYInf3(*A~S*F{#^nOSzK}65%@fF-b|ho!EeRWRgT_86$Pg;G``qy*Eg<_OE_m~Z@A9L ziG$32&%uF)DC6@1?9*n=f0 zP6=rCR3&Gt*K|9NguC9JVSW21D*XYz{>To@JDvV7UAE?fK3VaaIP~=2c(jd`82_J6 zAs*B3r)HuX6&Dxxk6!k>wl<&Hl-5@5SI1S@>r5iaWX*S9PWYX^8x+KtAQ8`W>{$P$ z^E-aN7vLBZ3yUNg9fXp))p1HH@nCEd-9*jA!k9}S=v=yT_El3eGhav93;EY--L-Ue z_XMfjGs%D5(ZMz2mV8a*-1;k#k!%UFva;Lm-FXZb=jLR3dU|w@?tYw^`33wvXK<@d z)(0*wu76rB-~Qb+_}WYYoQkDopo_ewRAL<(z8%8o)~%4^OkakF|9JxbOG4swdh0*g z*`fNPs;mr4Wod`gii`J5D2Qc(AOev_CMNeZgwBX?niAAA*wfR~Ax(fFg2iG{?2ME1 z@|Q1Pc64@*(h#Ckqg#ex!bfdZ{{H^|>Kt7URu|TVj#qnyZqPvG%f+bs!3Mw)lHvRK zM8?L(fMrKHgF!EXL9mXyyPTk)Aows~*0m}x1sfY1M@L6ihK)qQAYDBq0hEi1sKwWI z$qYxBJ1#dA&yR@J&)05^jlS?WaFf^BdpDoFWT|rB9IHQ#6Ga~(FH@EoDf%OptJ#F- z|6E|ou1vPht%P@PAj>9%b?geKPAl-15ftn^C(i26 zJur3sPTg1fIoR+JeYHHc23LGZ-&+YP?2~`TRwfd7C3n=f-=3l54o3DdY)!h&A240p z!I2+|LZygB_62fK)jS47+m%Xur43xfw>lG8FIlub-R-e#oh36oW(S`#arJ2;-(Z>b z%7a1+ucRi;aY4uOJ6qD1Tb{eh))?DhB3R`(W~#4vj6WT*y1u|HiWb=s^1R=0#i#0q zlhDNCu$AL?k-;Urx<+=V#(3Odl6U`#WAXN&p6JX4lBxXOy3*ra_fXCm?+y!BE%J3s zRYnjhz8quV(7N45s3e(v;}=!%Qj7`ODep2XUGk+`wPf0pZ!k*Vuc2!8Y|c{1Pz_(z z-=)D?pOW0}Hc+}-9%f=+#s}ltG?PD!qzq*C&M}5-lm|D7mHOVYe&9&k(@khl;Bg%8 zOnPp(uTs8xjDl^%emGMyd?A+gf_ybgrPs-r+C`bw*Ssl}VMT9f`YL>XTcr_$L zQ$sC>BGp*WvH=5_n3y7(ahS|D0B&1vZ%Yyy|BWiCsHl1V{EkL5cFX{L>&oNDkg`E9 z99YtbD}eQKPEhV2+DNV}`$cQD%qy47dkWP-gwKCN^+iG}iRy&HE1@ z^2^G}VNmQqct|lC&7!av6FZ1oNJ6avsWp;SL|MKuli0QWFaLmmcm;Q<3vkUB1vv$* zHsFBP)6?VPwRf)o)0r|N@kneFu&}MQRrzR4T6(&Vdd6vpnH3eu zhwn$!=AinL&ae}sRH9L+e0WFfxrY}IL^nbOf$W}{nf)x=7srIgMhsAYcr9^!GbjO0 z3mBbo$r!SrloVd@8j!x{3uDy!KOmj3wR}XMZE-(=x2m-C&FuJ+DBofQpFQ<#Z2nTM z&u~+Q{hjjL`6u9e-uGng|9JLt$e8%&w=dFV8R4R&Q zDK#4|Enev^7!^Z*jW8!+R_+{H%pxV;pam`JwmJ+On>E-CHLlRmwwiZU`SyRhGxgwI zjPHd_8>1p-UuU3)|87H%4V?KBsH~|vRp6f6zmlnX{lT%RHa^c(!xu^9%oc@^TN1vN zieAM^98vEb^Pa7;!KHWoO5424)Ch~pzEFdK7WGt3BCUSzw@`cjkc95vMFk* z^%gCPab~+VcrMjX#LiD2%I$khWch?W)NFoLYtNIz=WY}wb{1H0sH%~k8JM&s5a+8_ zJ&qMTSduysXsiWIjZu++v@4BhPUf4#uD$mCF$=w3 zdb;&Jq0gam<;lM5GbC3L{g2hpazyeEE4%B8d3vHRGV%u=`*d%IX5Y(7siFkd3#Zjg zj6M{X&d`}f=Y6CLUdS=eND1jGk*cV-A=?^5Knn>oa8H@_Ku~_Zc-mos7Q+R6kt?36MRJ zfNcIm}*dZAWKSn6I77$c(azx9@%D8xWVQEQmab<~k)Ec4kuQ3hrvR{)1uH-qn{R$Q#Z98$&YMeCgi>^@Ku8 z?B|Fby^R|ij?r!?mG}hM&PnrLcRE}(f42$!NV4{P?W;R=1#;i0ZMz;zzStG~xB9_1 z^M%iNU+3TQ5_|HJrf_4BQzTt-3B3=#H)iFND^C)6s$groy}D>Ut>rPAcCUcDbopre zO!#dbhs|1{nKIT?kxdhk4D3-2=OZ*CliMzwAExuBT@wz{38kM7a(MV=;X)$%Tb^S7 z!y0$3%W2ukLXA*}YLzpv8}Lqa9uq5Kh2IZP6g|jnD3h_Wm9`(Er)0}=@?zQz&#xLW z_QvDR4Qh{95dM-jnAyA(H1RpvB1KL06-}WvzS(g;M9v@-uxUYRHH!ABm_;0w> zr(9Qc#fwE#qo+1Co2(>VEuXy=LwYt(dhQ=2=Jx=d#FMXXo_yAL zibA5!Q9l%ve~02V1cDwzMf3jsE;4#yanb7LiM_xcKoCI8OrjBj5Tsb|Z=OW0Rgef7 z;SabBwf+fA+7Un#!$#YdRVs15x2F&*bXYQej#YHFh?r*84{E2qn!J;Ez^zwS1Be=PfvvfBsvgRt`sDehIP zyDJ@QwuF@pESJxOa17f(sh;z-aQlbtFHrq=B0AJ-F`331n;7|vK zuvZj(=au^DrxGUzEo~-D>F+XL(9fS|p-t-?s)*;w&a4VuOf9>=uX5(E6pyc>VG3WG zlf1v1T>7J%*k>;EBgKe`e(HIjs=yD(xwB;n!(@?3)CgX^ZCkR9y*tTgsImx`5r7gk##y-03DN2 z5_AxecAgMM4Q3#tKX!EiCLsTaf~3k!VtRTS@-bpmEj3~-Pmx<&TLCDPgf}a=E@j)T z_VD6M_wY4$9-(qHMj%b#noniJ@u*SG>mw~*aKPT6_u0njrEk0>Uh7HQGs`a@SbZx)rW zs(rsRAcfq;TorA1;R5ZTFY5tDunqqyc4ZuM!Sb$XK~o)%uoBa(kKfLvT)ijg_D(Ht z;BQH~(z2pbr8#$e&gNko8$T)!LD7Rn$@MFv8+k$SbG>iv)lH!5RxtFbZ5!EZ*|QxT zAG?}a#_4&A)8o>g=p~p412cmFmh8mcDvV$l<-f;Fua0R>nsVrP`p3a~pWIRy+mgED zp`ADAy6`<~o-#31I=q>WR?e{}YCHHPjC}IJ*x+k&GU_!GTC90j(q&(LHc9C2<`EBi z?#p^w?eysBlPTBdT`OG{w=jECQm48aTKKo~s0GG5%^th!5lUTt@cfbBr&1M0*fZvq zV1jQ1gR%<4HhxCdO}@Y+0OmP0tIs4mLDaEK?KQ0S_o*?;e{Q-bVaq&?dd>g&VzA#v!-hyd`xK%|!>cOJzKwXuYh)VZTE zfFJE$Y-ib^OWxjIuL0>XQhdNwX#PQhlGxmzWGaXBrNM=w@+C3gDxog9R&pE3gY>fN zp$X7+ke!iP=F=)Dv_gu6~?SeBR6Q#oaNGcc9vdT9UFq?&ivZw)RyuxCh@ ze~wFX4c-%>fg0|#W2D!&E#1139Q}Y?ogZ2&wJFGk!DL^M4eHeco~?Y2tb*aajhAZ< zCVb`CH%HeRXYlPfzbW4$xqu~2Z=dPYYnex4I#bib)aU~?N)|LTzsGLIlEpnWKcgFj(-x{x>hvl?L;h&lVk-oE{VA3K63Po~=;1 zQ!8bHYuh*f#dF`V%*?XBmkFQtCl~&L*pyR6-*(L^mien=PsI4^PLbZqupgkyZ_4*m zx!j?cn3AV)#PE@3^rn*LtFpW{}&qE4U64HVqApi`iE(8JA>+I<7!bnUUe=p{P}; zm+_?4@;H=@;0t5aA(x9$*Is*Dlp4}V1_!$yHzCwR;Ws}u1i2_<4V+|LgETW5Ww0G_ zkgC?!!rHqcTZTGC!Q+1Xs1Dh-zNjET7SaP~U9Y!)*1-A@YCAgJ9$Lx>H2wJt^+NZY zfioFE?v8sKhIlSS#U&)b%6oxV`l7UoOb`>g*^wGTj5QIDfe^F26AHPz*AD_-xwqu# zo8H^%?XtC(ule%Uqg3&U_tNyoXNb@?t-U(w(t;fN_y$Muc#-WcRnZr_zon)M@(R)G z?i9_d(!UduP7!Ba5*$Qi4%f_Hr8JV?u<`3j_Bg(`?(Qk_-7M!j+pmj4g~I_3{{;OG zgx_zXjVldF%{QdH!>p?u4i*>JS(&-X94Jt-+y-e=CkJ#h%NyLCg$sKbc|I;s^k@Bq zU6uy$u1$8aU`S4!NoKK&{JX5;xY!~mg8#51 znn`m{oX59wE3Pzk=RnEXe>l*zY3Ya{m$0Hz)v2%@Qur}!@8|nlFV&tIPHJm@48V$na%YANC&X`N}5=18EJDvpE{J0Efb1z6rDYHqEwJZ~5{w+OS zngSth=#f)lvNMO>bS%u8H+MTN@6-tRr1SZ8uvq^;*N?q+|9!F#{y_+;a9BzRF~@95 zGMF35=@O9Fh|o|3e;EZ;z*?+6%??#PG%}GYN;Fn5wkcnZ>W)KNB8a(>01ukZArce& z`izl|=B~3)iU0tC8OY%zlF7mms+FDHq5A^mvnPw_nw}aRb6Rltl`WNQeiS32UKt ztus+R3(Z7G4dHMXQbs5z88L{5m6s>^@7x!d7Iv*ayEFpeXAin7xJB!s#9U<<- zxU_boU@n_dk?bd2Vf&Bzv7Kw;FTLfjBeKEJR3g>lDO+Z7R`MO2qtRz}eYJXCdR0Kz z!ow)!9|cu+S^v%N|CK{)Owp%sUxDj)UcQRD?StcP~}%(Y2rFTRo^cBlG={ zJi1m#@PbsA`as8KafQlw561p%v~1TUnXRD?DXl?PUsI0to5>cuKndUmghO_QYJWG; zrML4Wj~(HW{I>ht)`ox8P8{qa4qJv6B-`o4^E7c3nOR+ExGf+1b+?d^(BUilubo<2=&TBq*5kB1Xc7IJ1&dB5le-+?&$$ zlFUp@5@e5*4Vkq%-VI87?Nr3EgXhM)gka5I{qf8nqxmn~)QL}uGb+2ZbrbusF|)2V zU(NUGmo-B6{JYUuMDuZauvy?A#g~B>IKbdKD4HSIlhJUVr#s`ee?o!o!h?anU=kkM zb<>w->6f5pTN9Fu($4x|c^{F$5-Pnvmxb||S`%?#%a1tp z;|63L9IcURA-8rOTo)qwthUOovoIP$%>{Umz-sYURpFZPFUGOCi>sc(G!eNrtFoI? zf2*bY{oX#__)SvmbOWucR{O~ZhC1zXx#oa=v3wnp*XB3MkrYh&# zC9NOZf3#x1DqA)35i3cfHH7DkOb*5xmgO{&=0*!zlJFjrG``giCoCDQ=(S{tTY3>2 zyGLn-9zK?nIuxL^^l-v^b+K};f5+-muWvqlalH%Ky&rp*ntOM#ttxp95xg2EmZ=l- zWI`paS6*yoZgu6M_evk$aA~ZnkI8!>U^R1f(0gfOIah48aCN+Dez8(mVEy66&!^-y z-D#0J|M(*o4j6^0qd9QuKGSpyH=1LY@P`<)-m9~otBbv}b$blwbZ30{mR_o|(VJ|S zhf`N7y{EmGT2?ZZRyZh=Eh~+ygM+>EElawq-zFAw0)IXXrTEU)iV?#r=U7$+SKDVR z$8=Xyl_sChX|3M%o>iKgUCOo#{(1A`f8A5XLmU|#tQ(gdniB%; zkKfPLrlpbLJ8bN;YtZfXNuw9 zaQjMG9E^UdVVew zr8ousx@wmvR<-nwH21B~XZ@Dkm9M!A-;1lpv{nYR^eG2_m`y;S@-9fUG_0yLiP&Rl zs%1G1_L({{)uJ@JuZq6UYnI?m;ahIyLl1Ree@RE*DGs6&surmVHdFL~EI*?;}sf5ET@1mwqm!SG)&{QnPz zr9G>IO5IiS(p46OPV;(n!}!9RkP1;rQ`|FMqGCKZr4U!X_gQ>?)A*p2B#CQc3ni!L zq@**trZBiVf!pvplXJW=g=Cy(t$_14&xQmWS8os-Z(KiK)*RAvvYvQS(v~>#y?3Rj zsexYLFW&A}z%|iMq+grZGSb52gT>`>r7jJpF6XnPde%$K@p&!c#V89_lBVJvSfX_- zev7GpkJF71UCAv(Yf4sK934k!4(Spd3h8pfa+;EkQ^#_oC2{rRgFPk5czw$ArVxU4 zNL`XKC6Sxb6wJb>2-A>UPBKxP@=3SJP-tj5Q<7*>mA2v?GdTa5n-zM+@%CNj{yl|F zq&VX}wB}DAPfFGs^|WW1<3yR2!ol%>^v8J zR+lKNHpN&b&YJNAZoBqGZfDPCrjZXO9z6?X)oul#5F5hsw^dJi)iTX{ylCXZf?0(e zXvG=_lh?Tv6Uk^$(V(gw>sWEw=RLyfu0^<%r@#U*pBJ)gUyhpvIh zCgnH6=}tszd))E%vQPr7PYUUoX*9u&I>mDvf)xu#oN?vx_WSfHdLyI`tfcAeD%xlfm8wo@r(8`^Uiq8cL8fB~-+N7rEBvVaMLMcy4%4;Oui5{D1B`q^F7G!qg6X{%jO)9N$;gu4hpX1|&Ak zNz$aQS;H>Gohbn<4dZ&bVAI?!*5d1zL&RZUv?9c;vG{sXqIG>@;T9s7h@`>J_QKH| z$%gTd6{HXV6_%;p=Ko-5YaHx*q=;yXXAZ9Z)6ODmhA6I!&u=$~m6oiZGb3=^mirV!~L#&$BDm5}81aT+U2#&?+e*X2>5g#a{#3e!}+9c&JO=&J!F!74`n*wj%d zo5dt+5Pv``;+o*Js5c>m5`sZ7q-(&*(T`ZL5n;>#2a4~)ZW(EIn>DU411}J4oQJGB zl4MHB0*Eoy7!i7Mu*H&FgR9Gq)n{=R_n1>maNYGCpen2#87lIRPW!ht>m#-pABN2Z4T4j_{um+3xsDR> zBew3@UFZfay(S4s#ZD|Dk1gm8@g<3pA`~a_5aW6y5GdZbe6KqUH}Q{0HV;%Y$AvVQ z5RxTDU}X;nY%I)TL+ELLPU3zLPjA#S9ToKf-cd!!U^v9_!gWxg6l~YTU)+}1WCRY} z3F#*why{O@M9Un@0sbK-M*Pp35QugH!hh_(WK49++5|Q@eTwW*M@2URbi}-p)X@d7 zN>5piQzG}Xo|AY|UK2tX>pW_*q;rK{NU9XA*(rP7=K?DbJ@|w3>~WGCa(RZegE#`u zVICSH9}_?fapK^#q-uagIFIK*sY62{BYLX>u!_mL1&ZV8wpa?l71RtnhbilIouT)$ zr@rbn^n^M9qdBnY@v=2@7Qz%Hqp^hq*kXq{F3%a1jc4YhMfVhfmqljqT(iKs@&W2w ziLqG2cmyFeIjP_+5oUoOy~er5@bb&OUqZv(|c|)K%p$P)Si?U|=v5?bDuxGuy$6 zNPEN4h}ewUTzC5Zd*BuY`F|bS*+s?pe_XZGH%NfQwmAVtz@fH17_s46Zi$Nw-2?pB z9D`)Zny4!CGoBa;Cxd%e1u0U@|1*C_syzgG9AFjxKAG(#t zYDpdZh8>t0s>Z@e7uGkRsp0Mb!SA>Fp@9&=FI;-^AU(LyQd8XJOwQO65K-~9xI>>9 zYMBCkGF=kNe+5NMD+3O7rt+?Ul!*z2BC`u`DZU~z8y*fg%#=`4;w$bT*;iai*dht_ zbyP)!!H;0~}&0YXV?;aaoPKiXM0TJw7%o( zzvnX{;MA&%s9lXyPvFAhGube^5NO?4IKYB|CfxK^e-SYG>WKQu3ffOQOYm^wo&AUz zY4??c=^Jr#si*{!_8dYmmV^7$Fie=9t7p4KP9}erwVm4sI%6ah1ECkJKz@?sX^y4;p^7p$c>O9zgM`5j~7;V>f^z zd(0?_Bq5#Bkb-Xwi3Gnc6F`Bpy2rUm5yKH=gM@=DSx6gpQ|0Zs^FG)c?zL-2fln0- zL&~?3Xi4`oC`5D~9#9uvFf&Od^A`*EyK&53uxk_mfH|8i&gk}JDhb*gO93!3agwjz zG)-IyGliAZxQW=f^4TYShQU#fUP?aK*w;1mzG2PzkudU2mBHMj3L(y&1>Sl841_v3 zj>o29!~iIAM`w)x)?&#t_!%u+iWt&6F@atOW{#^o{OBD@5A;1;6ClZ!(pM}5$qa{@ zc|aEfO&_RK0*vj%gDy1}LQO5(tfB|@;l{%mdXBg7g`Y5e#zqdj<}lknBmPem2ZN6pS>KclnffKF5%#n+ItrztZ(d3)Xv6k8So%R_)f3XbHV}E1 z(0)gATox8O{L2CuMj~@}A=Rl}Qi{6#q9Xsf`FU`oU?9%ME#4&=Q%{2j9iBaqO^dh; zIe$8|tfzr>t^z-;9}T^ZvU^y9+0~sS=>!yi2{(yHHC@iFbOvewL(+?VUJ=mc8MXBK z$kH_*;3QL0sdGoEJGzvEC`_yqRZAOEB036?KP1!SHn*pynD5>{epAlTR(-IZMW+m8 zx6-r+MI70C!v~5$AiF+r?aCqAzs=L;csjmPQDA_nvZE6l+R|)r0S_EJJq_|@b9ybo zHrqFkm&DLDtV|<0L}re=H-CWh=FQgT3hn>snH;s5Af7!W-DLT2&hxGM|vxul!k>{(T@Ewh{n@DaCBpCM-?y4kW1%?!agI4 z@M_|T;R;Hs;7Mcckho7Gx-D?~YMCrwQZM6;GAYerW?`LV<=zL{*c&be&1C|AvIp;2 z$=f=cOyG_5yEg6vG3u7+(xFCsf6O|k#^x9;owEXEla=pWJzbLWLH&JN_zZ_2D%5nQ zoCF>^#Kkb>2$s{GaCf)gtpKu*UnXAgk66!_a1-_sK-lGbiaEy>%jKZ0bZp8Su$WLf<_^kpjD!$RWVVQb_xkQ5V#u!vW zS+Pp`ad26QPD$Mzr5U_ThsO+(gLV^fN=u&ukG#YbEG^;F1>^gZ)kG)DR1&CZUK$Z! z0ym#~>Kgv8*q)pS_B$R3@Qqp}sWVZQ;`3K_b;TzHy%d8CyWl=?M9%TfrG0A(vr=V} z`kpDAzv!rQ<9Ol2|Nb)l-uK&iZN#c^9W}%ZF|6=2$u0iC2UaKeIY(gMBxaX=IDZii z=?qHzsh!~9d>q^CrVt1 zl+p@xLI%4f!l3%6j>}^?4JyQ6xr4(Dq3uYd8L`m8!GU+DiA@w>Z*Q+e(;j+2g&N-n zcwQy%_Q?mLm>*jB6%P2d1mn4%qDvgOZ8gz@>%xp7^>b3x*585bdScF)HYBEV6sCP(kDpAuEw8`UPW8jn4r4H})?)~dQuVYfzQ?P%< zodWvDYI|n+>w^g7Tt*ENndwagHeueyFDwU^8LU;)jF~Ji4a<)I5 z@Ro6MLL&Cii{*MeL;dCN>SSC!Ym5^g24G2g=`$hw`L=ZT8+LB4Qskfo)OS*aA{{2| zY9d#3;txCBb4yDJdjduYwZa$X{kw?A(T@ANzZ<_dho3DsiFmP$6JRDb89rWQo7@!fNu>6-6IZiH}~eUUKo~S#Jz4HGMutf1Wa2cP~U;6~FxD z|96XhZ~tp3dc3cm*O70m2bRayaJS-8L|{T);4Vep!;@my1FOkL>G5>$2h;S+v!kw_ z{1+!=&RX4I!MKa<`}foCVou%xtNP%wB??+=RiC728rp@UkhoENbgnZj3Y)a&m?0% zHJtGkBYaSCadT*yjd(>yxHRP4cukOUO0qr&QAsEx`4bF-41vkd{LP`L9k(PtA|^OQ zdLEEruYW!beSz(;K*t3W{49uFT?n!H?|Qo%uZJe`L-u=TwqqtfAM25^7pRwFb&^(G zSRzChx1i9OT1hBtB%Jnm@T2^_mXoNE1Rcgt607q$)l~RoyRpeCFce-)OQ7e(H73^xwuxYc!0Gc0LO%a)YBC5gpQR zwugLGMP6y|{*_8{e>dawe*FLw>Kp)sZ?G!Po}TA9*tM3*PQwQ)O^mu6JT9 z=YJ(%d-nPGI7r^{9B?D}IG}YV&K*4&H`^ftYJXJ~dMSSxmc!Zu)Jc>VbR?w_RDJir zrQ_t#v|a8vz<2XR9Xr)q-&CF{4z2r!>YQPlx1sj#9pd=-U9T}QLPyul{obbKOjrq` z&k%k<05V#Xv1d~t#ITvVI`h~VU`-yHT5IcUif-7%X*5a#ijMsb<<_1ymRB-H5z@`* zMRgc=#uUHd@l%caw$})A!`G4RXNYkp_R8a@hGFRz&eg#KYL64vnPO~w4!R0GD)Z*G zF{gLr-;`w{p+??MP3LL~`tUrB#hqN=&ipgKaSV0r=*Rpb`Y(!ilyU|ilz!8R#ENtJtN#~pk%Fzv1T@x%cSqw@kqVl-wQy*)2UaXH@ z?Jx=MFd*6)%}njy`MU2g`D)-q^-nimOi(4FAKQF8v{uSAHIC0!wKzP$6YowYmxIXL z0Ba)%E3Hf{StrrfIO4?z)e) zlXdlEWi%k<;!m8&a=0D1v~?L-GLG#5s_|E05lOhoc}I7y+Y3~JMs423mDS&^&es{> zzQX8cZ>^54aN9;P;&$jUBvS)1E%gcOB34%ova*jXh=oK&ZQF3#jpCj7VwO%1WbyZp zZjenML~5MTmbuc3*^w!k;1bGqc+F>8d|qNFPi{EVo`!r}WeDzx&wt@7l-%k7%;?inn0_3^Mt%Pl6;)<+ zR78Q`B}FZ9@06YaK(Y0W`Ysi!Ix2n0aC!m`2)R_f2xjMr8Nn4sQVszjoR$g>l>?0| zv(+2EbQOCEyY~z5=^2@rgrwIE=mOD1o#$}_W)BS^rF~)RJ^kkzwR^VW0Tl3#wBica zj5e$o*tR_=@#Gp>aK)l153|q@cUPVXi@{F^FsXbEj^r43lVA?lq@sxo-dje9 zV?eb^B5B6z-JMhhMr;qo&e6t%H(jY>5;)&uKXECpiznG+Cq{O5YHj`P4IRb7WWY#N$EzTaDIv~>_Td^B^ ze=&9}@fYKGpr|ZYVoCk?Z`R{!?pvMr>c+;C{BL*Jx3@Vvz9O&v7uUB~K8ZAZ9rxS| z1h$X)rq5jBYa~Yo1l64v{URLQ2)R}JjwRY+N z?KgY=cUo~yZ)2v7_8gPX;_vwnw{k@FsS1FyFrz`g@ML~MlfDpQB&dPt0;EAK@bjPKhrg1N{X4;4Fu<8$9&nX0U z*`rxr!TfW)u-N-)E%I+!>vp(}raNAS7crJ{V+;)8|Di68oO*{K93eM)sgy=)wseQT zGfT`bO+rDMzA>e_M=ZX<)$fVw&Rq;#X@d6yjh{{?>yC8DD2=q@Avi=|%Evl-i+H1l zW_qlx{AkZlXL7J`)@8&)=DWI`bvsLv-F%F1ZJ>MQ5l%o);`G;Ut}&BA%vo*D+$*-9 zc;RB{6tu}Ki?8|BC9c@{p6bgF)qIi zkjVB6#uf=`pG3!aCM$25@L|ZpV#T#~X8+;`F+L3>d#deoz(0Zv+Ovw*0n=iiBcrO- z^)#Z*?Tu9UJ130W!YaSPGMn@PQs)}#Ged%VdFrNgMe`Rw9L=`=tEfG9 z{Z}Aw`;KHDSqpRiS}%m2`pf&va88yhle2Muz^RV8T+moz1G+9k=8{e*>b?6Hx=Bze zv*HJ-668HACL7g;(S=bC_QN`3a3}v%;>|0%E#8@G0vC5bdm@dt=1`6zmguht>>UEc zvBR0>mSpV<4GQsP6=3r9kRp!_GkSu`bRKK$fCu(>o4r@BCUff?xghsoVqNcXQvIgy zNUm*CI4l*G|NcCJBNDEcyC}FGzE`sLNx~vg%!6g7ub0c5t=e=ht^#f;j!0D;DSPrs z3jJI|fVvNrYkG^C=wKhgE(r`YBK~AsiLg^VzyY_r9hvY{{5M3!tTc~{q1FblUv#b< zWW?1HJ&0nGe4IIr&PIjPq|8TOvAm=pHji4|?Kr+B>MG0h>@H?Lgc5c}d~w}fN8c`X4{q4d2>ClXiGQzLB%plsa!z=yP+T*Nx-14=QJz@1UWTf9*@H zu9>cXykb1T@~dz%?#=!1OCHQ9$d@w|9>d?I2a;t;We^unfIxg#S67M2$qyV9?Nd@y zU3d2trt(B1B3(cDC@24ZWZgeFn4O!GvbSeWPEPL7iQhBR7ZjnuQe-x%BKVNjWF|?L z;3PUgnLvPjX1DW~S|&L?CIlkezSS=#^>T9P;YsNi!XCKKcHaFi2Hi@0IhoX|A-$Lc=4l`6q#k*+&C#;dY<*WU08qC!o&Ui zN=hn0?u&UXCGbI~_)nzc^x|~*k zPelcQR1`A<4*fXL<1$C_X|PAQQ6S1dxeTCE;*mW=;Z&k|tm()^`Hxpg(yEY-q={p( z__M7^dXUG4i6^rv;yrpbRV=!JEo z{MX}05)u+g=D`p|XV8h!#HQ%8e5vNTqKu48$9*rdolD+$e&EUU@%cGJ_N4pDPJhHo zvm<}dzoj6B(-$32<0IK9K)X}?=|?TC$-4I^>HP!5$GP;biCN3xippxMz%_qN3W1TY zugC&^hoJ6zzx;!x%^R1l$M?6O0SitS@JB6Htd7pii=byRkuJtjC4$+mKQ1~1e|MC& z{Iitsvudv%uqcLbb{tN{t|{BQ6}``AozdZ0>QWXdpMd$@te+KxZ8`36ddWzE zA3xH#7CoY8Gx6h@AkWhr^Nm?)ja2=6LW2j^0?Bew%Y1rnkRgV8gCj0Rtg;%K$`~}) zB+=NxcB;|sOwJ#VpHtEvjuT4T2c%y+grZ;FT5>a}BBYqvuq6spEYc!F0n=JcgS_Cd zm=r0kXe9(UgdMn2^$A>X8<-hi7e}&9iWw0=BT^R|AAT^v7|+dIM`UN&X0qJ?e!;hY%hT zda(8n%y?EiLDG|TjitL>Jia*Opcl}D-pcYCC@KOnLhat%30v{G>x0l3hvz27WEhFJ zaT5xQ$)tp6JHlEc%gPKE_F+{qhBFArga#{4N(HfxuImLBC7YNAOd@?G;Y3QTmJ|fjU?dHATwpD}9nJ?SYDQ*bQ2Hn~2u_#+LOIJUH1o;l?|3#b>GwG^R4 zBf1dUzUtkdiZ7~ZmjgV6+p+&Roa)L-+R)jx7Fu`{vqdI*v=OCovAyVQG@?)1Y1TLE9?w39nUe#1Ze!!)-uJ(@WCVbRmx;8IfOKa37(&*9w(l7?2JFx+ zTocqNLey}*MsH4JOrX`<%VYJKLGLub=#Vu{-@ir>+OXTq=iH^r!iq#N&FK7eu1&ow zVjiq_qx=InmnNKpBGhxNkCvH>LOkK(AKkL3he#gR+P z0;E)^DKOD72-qW!;W?(4#ceoHLhnXeTSsSinQvF1z@CFI@@jCpZyE6LIi~doo3e2jior)TVBCw8=+Anh8w__4&to_WWvo7`Q%= z`R{YFht)vr9XQFWRi+)da8`8+T(OqTzJ*?%ZDC>bcuZ#B{y>`|^UASTJJLu%JS`N! zM@2)O;fc$}0X*1eE-CvxOTEB~2X<4c%rk*NU}hPXRH-Fi=+0PyL2J2Xa(f%QK`%DD znL~VdZH!BxSM%lK3I6u*pd{#y7YwZDirh2XybB7o`L=PzaZon-T$%hb+3r8bSAfqy zZke_3PWaP)r{TlZvy>eqzKlop^Szm!$JObe-B**IeI4vD{ZxU+gxXX_yT)Us*b#lLd_SSL1m2GAy3p1EtQAR=;y%BxSMhtVj;@XpJHIX z(}Gjflo>yvz5&;ev^xt$7I@&YSctgUT*vpwKTp_*^kS-K`P(Zhoe!na)maj*-uhpn z^)C3P63AX?!j;6pH|ya*K|bZzKMCAHfwYgMg965_iOMMPq6%9ip=H0N5UBA;4i=iQ z@G#YAi1$-B<)j^np88Tj|NJy;#C*b2k248)WVnsE`g?;OaEo=)(k-pI!HiGLm3H9! zhx#UF7cfiep`5sH7#^dl`-|`;W76vi+4Pa-UI_W|c@JY^^zQ0kFLek;(%je65ZD)7%)>;$lkxepYC=q8`_;@|bcy~4LTpr2b zfzQ(!hqYmsh^NqX5jcCw25St7x;|-QG1t!eDIDylOpWo z24_xi#8DsY<^u7skBF{x;6c=38SIK2TN zn9%@j-J{XQygBO7b-l9;(nag1kj`!+ z!?^@V=rpL*nD_hdk#WthKp;gzD=wck7@kYlkv!$^*O4ZWZrU#DVRI?0c zD{+bv93cM8UtJYaEJ>%#Ro-Q;Fn^3bSV+Uy$f;xc(~*scNWbr>pwn>*dw4H4+#=mc z=YhJFBa}6Ea~$A{HP5HbzBMAM^kL<_Gib*Z;qq{Py*|UbsuIdTWnF)Zp|Bi}r=V%T zC$dL+9Axe23?P8*AE3NW7I+1!or@~)FnUYLOc=iM38CSWP>H&KEB}&1Pj6)PgqFFb zDSwd|syu1MF=1gq#9mg~fFN!E5FqqA+|sl+LSLf{zXtb~+B zlHpaP=Xrr^yC3ZlLB?|s62Z|z4`v{nhHP-!6lT_|rPC&yE8SlF}xbZ;GJOCdouY1os-krp#c(vVL3h&#%30{n(oQfx)3wR z<6q{wyOQtmUHmIOX_uMUWYzB(6Wl&rGDg9Pw@&1fYq6K_BkD z^clU)H(Sf2RW5M?LS-q1#qjaY9#IKUL`Q#B5Q{b$=~M7>=A@IL0JyRsQUnMRj^I#| zr<)(ZMM||s%7Kca9)DDK=Eh60>z5{f0q?9CgbYA01_;Ix?b=u+@Q6h7FEb!g6iNxP)esx0s$UtZEb$m?dVIuL92}*gyATvAI6<<_6?0CJ;)l1u z&?k#tcdGr+!i;eCVqUs#>GMHd~ofH899(aL$H239)?mk3&%i z2nd);GxG9m-rrsyUIT5IOF4OX25L9_qQFJ#*f?;I`kw+yU+J!@9{>i6BJOf|A)dI! z*1VM^n6|#y0kg6>y_R-AWua0zeT{gZUf{3=L*FD04$!UM8LgiaoAv9`(J&8&Cx)`t z(RN|K^04ekDY*+l_%3cZ9RBt)AL5|T0>AF{ez9Ns>t-oWi_W}E=HpFN z#PEo9(!LTNk`LHM)9J|t3oR{8LzAMQtNB~)nxlZ11ElbVlR-{XIi7A-l$0W0^?3}{ zU`>gs_Hqx4I7ghktdcFEqXQ;N4_@Q1KuG_%J21x+X)T$aPcYV6zazr|b?@P;vbc2l z-Aw)WsL}kOqz@p63{p#ZUQz&(Uga0zDbwy>S`%K5mgnZSKc z%}{jX9hE^*Lr-jSf3c9_5|aFQgNi<%kgMWG3mh(NZB5wR6f?m>;+J3IO#|sqFb1CT zbE+hh3wv2)F+j!L`$Qy^l(rTYA=#xVG}wUcf4+}0?km84T3R0dY3tb{8K9_piacH4 zBo}(<)D%5BCZ_w7CqGT!qB2y?)s!sQ>>=QDO_wfHtl<&{6AzW9m~qIDcle-VCAgkF zpsD&un%@_RGazuJY9HYz1WP&gM6<-tZm5R$cgD zJsp#L`71Vrp8oG@MEr$0;j$z8QRKG$c^2u;<8mK?z2!#O)Y7&f1JTPrY&xhQRq|8T zSC7z%ziqkgZA3{r*?U?_b}RS>7%aKV6bhz7q?pzG9%%j{N#7>VG#+Y3n(S_3)Da z#+qov`8)kR5lg%+Pp&1u{4o#9&GwP`?TY_BCpZ6%D3Eb2A~ohRB$MYPHhi>dT=mXl z-!-IS6!dN9wyRV7sIK(J*^!BSRAcSzXnE84Ho>-lIe6HuzDdyJ+hpS8zAFzE`Of;R zF@D`v&(B)x+oO;LeEXcu_Bah28G$d`Ffyk)P3L#1Ih6dFYt>D*7cNtimm;~l*NP5f zVNKJMuFZ`@k2(H-6uw}BXU7eEmezC|3b`i_14NDXKNQfcO%|&GX|h@_(_bj>wuL*U zo;l85@9(>8*?~D8_4jnJuEA*QG|f!BKK!9s8{e?`*Rk(G9q z1XXHq&zY(KNl+*koE|*d|L3<5s8nAzw3@?*g~@nxdXHPbNAdYDR^zWv+)X>WnLDV- zajSpa<2!Fj>)7esbUYi5@-WU;GxjSixvT4S8h-|ar04mEgm;AhTwH5&yvViskuJbX zvKQ)|<=5`vi0tuNSO4z*tz(yJ`$NAC(z-Qxs;lQ}5YEhUyu2HH|KxIrnX|n}B<<93 zq49fv>T0*Q33Sl2d3?mma?o7WwwL!AoX|Ca6>-P}he393brIlY9`M(CN`(=Yz^gQW zl;31GH3sKw(RkhG9`noX#P!3Kpi9PL^rBVYj-%zxA%g_1(3i|8y*z?mG+>wdRN7tc zS)*zao8zG9+Vm-H@_z8@Pxmv-xT(dB0W{o26WP+GzJVxRLC(Yh2R?I7!%Loj_3e!? z29Szbg}xIJD-WwiL2X+>9R_CCkNTPLk=N#7uDg6>R-Z9K9ivOo=sl$oBgl32Np!Tx zv-wd_E8u=hz<(YSsp|}kt#D1W?)u&Q;|XN;Ps0+;?QDUH8>QA&^VDOTi+{$tjai=D z)`x(Nej@61I8*tk*F^O`-UQSuIfV()Hy&Nnhre|?mB|;I9Px~eGkkE>LE~#eVNE|` zECH+8GP{S*$6?6h$Gn8;6s3$8HdPPouFJ$@q%{?PqAj9PktxmJUqE>(R|@XEErnJ6 zB54o*m{rRj>l&C{Uuo$67&7g;ftsz0D*=7`vc^ILouA<2XDl(ZQ(3t!JrCT|) zG(Ji)^9(pfuz%C6(F5_-YuEeSFr*!XSfWo3&SGp1ndmD8gvYH|T3WTsZrA=TsahFy zeTTIel-(RX|75iUaj=={8k#tMVIvkymrXZ|bILx~Tk+2gQ`iv%HG@m|p<_w^2IZ+7Z4?N5zDyXkNU z8B+tXn#%9z#j|>JW=A)T{|-c8-TXQ60e;oFJ6ju^+N7JNBrhKMtn5O%R8wR$%FTyo znb~R&|EuLU*X1>uQAU)7{nOX7! zq_#QmYUJjd_48c^e0S%Yk`5>Ogx0*-7t5jS{?WE3WOVj5W;f6ZP_g#=rtd9~#_~u& z|HNhzBD`Y47xVp7v1>(sjO9&A&S^cILBoie9lhcsP)L`F0dA6P`Pb(3NCVMv{>YA_ z!L+g0T?0MA!M1!D)?tzJDMgD;xzE3FTVgEh`sFdegE7-SHUnjquc0fp=*0iNfPl^X^`^cxWqn*|2uIWdH;SPY%GHrmfK{153ED)Rmk17aQI zKo7;D!GLF`=ieQON{)?X_ zhewg`muN7QJnTIQ-Ns+mMwj?qBm4Xk1ZFMqIrx>}cD<0$Mqh%!oEU{L_*!Q+F-2WM zM--m~s_Od}`U!7Z&)c#jj195j$COoHEDeHGaLPJ13ZTa_;-(2TFx_g2C6}*z|C5Ps zJBRJzK#3$9r!q%kX?rd1$B+7p+6iWv`?3J)dIMHyvxzTLs0YFYAS^dpqp6lB_x4Jb zr10?aHaM^GVN!}s?C#PqBu7sl^`CeqDVKfIyly~otw@sn6>q_(CG<%5v<~yMFIav5 z%6B;V4tr)j;ve0&7ZmyB&L>P=9lK3A$sv0xyo3aSUq%DJgnAYu*MCBprc-`SG`PEs zu#g2m4!3s#CTb3pi6}g`{TsVOp-~E;D3kkp%Af?cJOOilZAsyiDRMvjJSZ6#+us71 zhuS#GF%IfWC?w`XK%Z<91yH;BMp7~71Zi3 zdjs?Daq1ic$jpb_8527Aivw|*_~BDisz3dH>Svf1bazu4FSUPEUZ+YZkM%q6j5s{B zi5{f6-|(XR&zlmeW;EUL+jxvSeH4c^|K5j$>_dojB7wZ^iO4pi&c)YO!sNMBn~Q)7L9Zj5fy zF>-bW8pj=_n?1(;_wTd62*&!=Ox@MNZ`zlwt*()~t+r6i0zIb7gP?!UXDhc`kMkGg z^u(cy0e%k?q4&GZuBokgo3*~Tu4!^X2d&;I_XYBhVY$citrx+g8*3+Y0lmTuyMW`R zhuENR!%&U=39;@37w-$ucAn2bS?FCGHA=zcdA2Eb0`^YQM3i#pX6?7R2l5E9(P995 zx5t_boL1e{#rs2DhKU}WyU5K;^4UY^G_=>%U<#_wgnxf{zs$Xo$@;3v8Kz^fE4xoL zd}0bb${M?~5vi2*|B~hnmlKXu>G($vfT)j9;!;2m-NCr1AY;u0`YBc?GhHR_P|KJJ zW1&H*wT)`yX2{R6FSjcmIk^B0&_EmU#uQ}L25ySv@P?)9P^hs<8oaBYra$(ojhGXL z6pwvW`SQX+wNeW9G>-7jN9>Wk$;mapAv?!ki({M2C^I1S_gUmtj4)PmkMC(U{AS5YAPF zT)sA;3Bj_to0)5lX0gm8f*dlrFZLa9E7!aUu4bPS>&wF-jXt_T7>ev5bOZ@FCM@+W zU2yT!Zv9Cfx0Jpp%JtaLG&XBX3-Q@S1T(jQSr#J zc7Ck6wOxaWWCg#t;Y^6}t$*P^ZN1|X3CQqth47mM68u8pXty_BRPvJ}kH`}l@n;@d zu!ua56LmVz+8HSYOr@E$wD_7q0hk$P2U4L68?GunCx)CZMVr7&{C*`w*N6fJgSQ0%966Oua_1f^1V_0Ees zOWYO-eLpk;U#_rG#=M33tN3mk#@YwI>dHl>zxV@7o$~H!OG~stmz(%w1V+>skrV0Ya(NU*T9lVgs<8<4 zty`p7x8I5s+v{XZP#9#r=yism#^i?W3^;h~@`Vey@6zpb6jVENV{ObTUEE^V#aY9| zDRaRxq~?suc5caO8PYewQ};{{wTOWo&!2IsoeGV5nZ*44y*^JEsA&mm}U=TfSO;qJ{a$s27j*+TBf zaX9Vn#t^^bwC~HHE4UGmeMJxPK6eMO7i~U4|Ee%RuuRSP49)c2m`Bhlr#^1T zZ>4=n3PmO{VNyRoGLE%?cpJe8<{W(1*5TLU=N|_Hb1;SZBBb2yqxq*M3#wdxpDzy| zvXYUQo((U}$9>@JJxvb*jmt`b!seX*ob`Bl#>#A3xy}9zjbqI+HC$$P~t5P3ap5~Y;!^&Y+5X-4;+w`;C)#TE9+IP%tfYI&69oI z`>bzLJG(}S?-#*fMVG7;S92Y6wQ|*#?@42*==pR>gx}%Hc2y0|LRwZ)A-06B%V)Pg z_-N_*RTM)VNy*5rd^>ON?>$cD=+h=AC*7c3!2If}A`?Cw9mJd}Y+Duu#Rf5l0Yyr( zjD~oW<2wUb22CI!7%b4q^p#;o(abHFM;*4@fhtVG-6IV97!qGgc1(g-`e2@;&a-6ePkgkZq~!QF!ecL;95;dRcr=gN8a&wKTKGtetxa4Z7aE=@6&C0h94O5Vo1d2CcXeU@_O$2ZFgTB*|Wf0HhLh`1>Vj<(9415 zlnuv7dn!Ng*}cIr5(Z+34lyGDGqz{|LA5`F5&z&GYV!&kj_d|e#)xb{fT&kno;`~- zs&-|XGTn%L>1WSn*B9sWod@j41D!Wq`kyR?b@Bp_2Xck*b~o4qUN*W5qf0FC5~7R?#8eP z59TQ2CFgXEem4)988Egm4Q>(~5GR!$Nm662l?oZPGp(b*!A1LVw3q+YpNQHk|K%{_ z1v^rjuzJ{acVOc3=w#(F=IbN(=0N_&`=E2KA(4-Sx_P9!$8kS+n6=<#kR9J}JYEeF*>5wNlWMcbz2wZ4Tzhsc9tvjRJ9b7zk_x(VkLX`@=^ikKqz% zn@$x{5jH+?ywqF0%DAfg*=(lYb6ZuUeGOlr_(iV8{>*?y?e&{0>6#a;kZBcovQhGF^F!u6SDl`ce~Spwox! zV`4idwex8UXAYB}iDkZMH`2=;rIDpRFmK-u%7u+_|2@NMqldE#^ zXIX};XGW*_xU6}3t?~73eT}bkdG^d7Jl@_I^VaB8mEYGGe-pm%xyQJD2SSK=*Kby9 zj-beb%^*cl@;X{V1E=m1pD+PeuZ-4)FmuGkS%rGGLlUBtsGm#K$CajD6@w_&77XDU zuZ@9u%ZQn8V#=A@^U5trc6V>n!Q@-?+yRl-pSN7Dn*!%W=m`VR*(xe6W->LwSyubS z#;%D$)UV0C!2?^pMNFTpiJcN+Abj!~GrstYNmxfaWE3MPl%QxBNJw>Pk*Ry@2OQ=w z2?iKObgj0WamTh#6ZuP+Iy8fwt!Ypmsbhaoz<}*TqvwU+I-Uy<< zSL5nP>^VLC*q5<2TFcQ&=}WJcg&L`15lGo%fL6Ci;razTvZ$_`A!#^$5`9&R!r*6( zuBgu}EnX<45dW7QrG$F{_X`Hv$vUia`{ex75NX3-ZOHYINmh zp1(BGu&gOTpBNM)4hnrYee9C{It>5|O>#ADlSz70ty77K7iX+KyABDI? zlu|IK4pH2%zTaPJk31XIM5EO+b4}Bj-=_C@=1rE3!+l3+aIzx(1?jT?2TAvC_i;c~i{Dofz6kc4R>RJvM_=(oxASBAfh~UMe($C$}vWs|RPKGgE zT<|d}iO7mLV!CfNaaEO|8!d3aO$&jYwcHE~wq|{*(+kOx-#lglnK>P$UYYjnjP`-q zZk$b&?PJ)xujX6hMpgFb>R_uRZF=n#3TM9HbhG`1grT9fMb`lvy)U6;#MU&ar9wGW zBNJ!bk35xCRi@X!-#$LwbjY_f5~wKw0(m$H2DJR? za&P=H@6 zu+%t3B*IKIC>BY}Dsd5{Sr08k5D@-=$2^Xcot%e>B% zupi^{rQjr*IvlWY?jwLhbLe#`X~9rsIz}?_cx4(8#5}k2gJ8ky*Gi0`TU$gCpxAI8 z6k7PMxMY?Th`ILqkHqND-Hv$(wCKg%kp!XqKYF#6b_<-X)BXtSG5hMhgBqtQJ?AkaQp|<6y(ML&51JkH>2I0kjNB zu0&h;IJ#G4W5JBdjY)RfjWPV>1oPIINdGece@ZhjQQVe*D?-(E>5^CsG8Haw`Pnwn z7dR?VB3^?Nv4H6Hb&CU3mnTw&OFXUY3%wejX7>rt#<_lGjxtrL3|I_{Rcy3Hr$d1$ zX*;j|d&0CvdI%({uRci0<40UuVd(7p$6ogKAKl9&a*UB$M$N-&E+^0}%k)>y=yPYU zBbrewXTy7zCJs6~KZLCNO*3fxRMu4vDWdZ=k$MeB(V_uSY=A%=_>^kZe@vn~(PHz1 zXLvA31ft$?ou_?$gb^!9pbxsGsjqhjGv3iCFhcvxfm?#r7*I%E9oO7^6(GFA$squy z!P?OV&{TDX-gr`BKyZjAMPW5z{?KY5{y10IJAZLe&ndNtVZU`eQlS(Q8ykPNpV0vZ zXaj&W#7)s>rb=yZh>=m>qWj)0m`KNcZP#RD=0@f&=4(LACAHF|?hx-Z2e@DLBCk_q z=0H&glb*ig|ix0?S{yn9m_`FBcgVQHJY5c|u)ao?!!~ z1^*?XN3S&8XlT0J8#WVR1Lq}qqg?)j-v0P0@$8LqZN{PA3IW^|^rO2+rucvxo_bep z+kBo@UM0$$0mI!Mjf3z5kbY3+qcvjmyDg%iZ#iz|-TOaZi^WT%y=j^;bz=#!uA$06 z(4HYm?Ehkmt>6k^5%gDlk?ukh1V-a5f@Pp5ht}fk~czq)G~rvvMRxk3j#* z+%qYIq2d-^l38HN_boxL^G${?jM5=d`yxCW%V<)DKZO||<&_9TaaERxpfz4=EE8@X z6DcM7r$>p|2M{<#szSZjCDS!Sk22Rv&+nY2r$XKq`Bu-ZadSLUKp0;ZhAuMlSED{8 zBLPc!PuHx{>Spy?07Y`VfY8*`1*O=UJ|XaP_D%UmCZ_Jm$yeN`eX)HsoM^B;>rV$l zkPk(f>aX$+1@xQUo#_?dWpfXVyW6>8I?q3e)FdVJb8vXT&RTBz!Ej_r%%GKQ2Vq0C zb8VO)B}02%$EfPZ?9wq9Tsl8>3vt0JV}&In%K{=$KH!6Q4vyS?ZFbtMS~R~cG?=aKh{wYcT|a~u&p_D ziiFanYd;adBpL!hGprrBq)T|;k{g@FvT3VviIZEN2icesgz^z~`UeMESRwTz9u>FQ z(d-w8N+ihKjb9#2!ivZcbaQlmkrt@4}90nOoZsk2%w<>hLUR$-(bW zR&O0iIwz!V{S%AEQNB0#%6tOb`@V7}EV1M^dW(QPP_8!&rkEK0A!!{!)NV^#Gxt&~MfDqnTCo{>UDs(Ik5RL?Fe zPt9n!Ca1+a!#?f$wEDdaFtH;N4lsk#ROnE%{q~f?(k!&Tefz3`Ereu4N|%gTO?xF* zYtDFUm$aoAMr=ykPYu`CQwW)(I}2}xuk-9{e800m;kY$Eoi0lR&@=Gy2S!F$*ohkG z(WGWn9am8D0(B+97KZr~GrkkW4xTP)^&wxp*W{b%$JT@Q1UWmvUF%Hlyuzo8A6U63 zBwiH8<02WIm%9SIeRSzOk)uP382m}BH{^y!F?*FJ${bnh)8Cr%XW1Pc_hb_7b+0o( zv7D1rQ84?ZWg7+Z7Fa=0I!v*Dnq-dz)s&*?KGAtCce~5BNG;5%`Q=>wWqzA&6Pw%B zEVdyLDr8foHJFVS72jmzz`|@*Gcal9puGj-ffchQpfmj5eJPOEsI1KxiI?-D6@Q-8~jN?K?_j~qdGIR^(eP;@WMT4eN zfRI9Ss7m2_fGb$W2X85O05c0mN<)1MHEIMtbr1&()mV}7n(rZSVxLZ278ye6`CyJ2 z;1GL7{)BnIGlB$s?5B#sq=teeIvbm9Dq1iTji@J+4PA?5h!s|%ghr&zc(%_*hz!33 zzrnHmPCRd3Qf#|tA|)_FnIZKK;BShBLgrW*<~A7AYw$ zE7CcIlmWIfJ(4+Q9r#=jkY#eeW3ovcbgcr!YIo&YyQ~&t*<@AP%r+DG)Fdd>rQ6y- z1xmfmxZA_=`^6S_L`_XiK%?$`%VD*!@BRiNGD7I+__!N&80b@UboHTsCZwYG?h@Z; z{rTbct{^D&>+?`SGMEmaD+00y(LN&Mb^Q?tVo!tAu?Y#>e0=@#u4{W@6MH!icfaN_ z+%7h$$hVidUs7H~M0=`#0U6PdWM;0r3-8bE#h-rQ;bVYXxV9s}X)szBY56NO3R) zS0NJ2wV0a{{*cYZ`u1@Hphp*0^(|j6vfEU{CQBVI_?~MziPMb`J*P76dc_ltoz$Y% zj#@NeWwhNt*%ni=1*WTlRw#Q(&-p8zU~9o04s-4fYl9EBp$IA6SJ$+GxPf<8&!dv} zh}C%wdp+!`{48%A3S$ujADoI5B+1G|KSGKWVvcOaJ0@X4B}`+2xN z%T^3Yv}je$4s0yp6qP_5SEwnK zin}bVaak0s)Ph}AT^&yzOg_Ut}e<;P_ag%UWt$h31r^$7VNu;t7Rt*F;IlSZtDtSD=05#{*?k30zCnsn`9dmTVKHR67zG>aP)@YSg77e?VgYszVKi+`v%`WTs0C+ZW%Sn2t3(+)!95L>%HL!$Hkv$G zmekHD)|V8jX*ltAb;O2fj?3#r;7R(@wmt7lSzg;o2kOHPb-Npvg8QkeL$A-%I%Zi6 z?T^qkN0>ZjmY<%^g(EqeKb5|6a$uMJ@Fc-at?;EACoc@*L5N0n_Z8&xNKfnLE@54{ z%_UQ|oVvBG8*;_?&M|(?G=H?N+15rN_K5PN5$au!eU`~6rf`}SHuGiKC^YkwF`$)SAw*iwsTlE!(ez8Bl9ZcZ*cmuU}`$T?*wkrC(22WCN+ zW&&4g{i3pEu5byUVvJLBy-uNo$6aGzPJh$*E=B8>^So?zb%rtJ#^a(OBO z>hdP6Pt1|v*f(2fK0tZ(RdPRyJz*N|Y^Fs|Sjjd$4t)Ju0$RM=_77L7h~A?HGjCmW zNISiO3VmE7TzP-`KbSm$0Fb`AdVTe$de6_REslTHClpd4V`4%ciSd1$zRS9P<#s*#XscpDt-KBIs3bPrvhEd^!0mx0WUV4dQ&uii&Pul5gA=;c^TGmej z&Rx0M-~>BMCQtKYBHGAc4md^R#ATeG$thgik?mJhd}m!9l1J2_RG~bXxUN_TPp(J7$96!afvhy4=?Q38KtY zEa0{z^^;W5p&_LPJQ$V+CYYxBLC;`CLL|$~jZ*o&1!+VzB`v>vNGcX1Eg;tOmx<%F zA~ZZa47j_92n2vH0w_fgL9!hsAolkl*Jl!55 z*$CEU_T5jO7FDmti3Pl#E#b=7CLGCZ#tj3dM$AJ|57bqZpH)U4(sJvO|-#l&b&5ziWfc6AHsDG z@9(+t;Umo^5+jG6R^zJ1#QqN~-Z&=*FKnwoVqSDri0hjOHoaU0>epXvIXnebPyf6_ zn26BJR5J6POH>&PN)oq7F9_NVyzT`owuDaj z@Y%wfMs^5teM1)@&^P9uK9h>R9^ZS%qGYIKs4&HlQTP?+J&o_rl^%O;UV`)UodPeL z0%6-$#>H_nM*3Pk?OlcPa>av!T^M3hlMQ4QhtAHB=+$z!lhnjVkDE>bf>IcW1zYMP z14`3cN5(|p$(}&aiHNjxMhhWbeOAKn?(o;feg{^K&P)ByFMph;7Z!d1g59_GCXck5 znlo2d9+&SsZ=5SWA)WQm7tj%)oonPibv50FzFff3v$I_kme9EE;^zCEfKZdrT1G+a z&u7SF50~P5F9Dm3Y!jXrUtjh+&);``i@e)=$$UBAq%d2x%b(AGpFBVo6!MQp{w*b< zoCOU&;=;=^w#c8`e?3x22*4LJ&S5s4&ykOdGK6oQPl7hR=f0M8l;*EJ;um!LZM2Ep z^_1U|zXas2B9iSWIer;UAm4acOQ<_-3JQ;W$7HM96oGiMCbgaR^`bYdE7G;mar5uM z?DEksE=rE8(%TuX$){*aYTz)o<=~U!2J$>_tnAZR`V=aE5=;Ok7p2kMaEM!dX zkL|6PkwqVWI-09By}{N|SfQv-?frde@oh;XY~AsShdQ(vF6Qmgui0=-4RxIGxi4)M zgJYareDabP&~rX-_lI~@2RvNa+NtTbBvo;=0)YVtTmPaa^?p9y+_hE_0}7rJY)+FF zh^&CFJ(U)hWP7?7sp=E=x`2l8Z{N< z7J;>HYpiCHgD9(^o_l??(1v~3jf{X_=TA_Pkx48l-`f&>T*=H?;YF0QWaj|~C?X+< z_AHcY?7F{hbgq!sz*#;%+`l-8%_XF$POCqUUphGd4CG9UnswhV@hwC3%E|y1_~l*# z$c<-aMk{p8vaq!*&nqaIdT;M;dzKE8Mt84j+Bgy61p$H% za{KX@g&9n|-R@K$=hA&R`pCUA?~{*Bec?q6x{fMWfIudP$nYj49ydB~Zj9K@U04GL zpVWSCTHk1MxsbAM4u#J_H*U#}GDF`o&=p#QY2jqtb@~QWQ`#cX+G{X93b^zK|J3(5 zRq?<(5BqGa!_%SMd>%=VC%-vu0Sz4fJfmYRLW_2>)L#AB?$wp=%_Mj@vXT}1mJ<nyaM3*=BE4ohc&%TW2<(X(K@6hC$e^7Q&a%V4;V9b zoM7+-He>eLy*;DmbI_l4Sh31Wj;ohb!M%2+_rVA-P)g*)SDd|E1h5DWu#KGY(?YIW z*2_Exl%`M?y~+&Og-9SB^b_#Y@03Te}We64w-_}>FT=*uWTu$ zzy!o^dEywAv^DlIrhn;1&W8+|lBPLwwAmuqDX5AtR*W*`%bivTA@uuSebeS9(HYE^ z_1$lcD6_L1Gw7;2a9vlRqGpXVwZ~Oi2?P}SW_bQS!BO#nce0$4-FIthEQy>{q}}_I z_`k+>FVlcRmf@Kh2Pc6q4XX;T%FU6o-_q-sq6>5uANxXSU8UQ7CBUCuR!F;D`)-U1 zi{s#UzsRQ7^x4tM@*Poj*#@|@roGA)S^F8~*AoMS z;Q&A%EwckM4T{jeQ3yn_ADI%MP9H6Gl1G}Ydk8e+(OFq0g2*SX4rcCp=>rRrlUp1X zT!eRaZnoT7GV==(%Ipl0dnPfZTzE@T{}$_ql0PCfQ7$x8MOm7#+wYcX^4T!8<*)Q9 zv6IRmp%>aAQD!1ZN%?=3J9&*?pMS>TULHv};S+T_Rg0eE60PzsG&Gl5brN0*u4J}+{=`~(e;azfXLvX)KB|m?te8=T$^2!=#BRg9-WS?e z4LgzH&P4(WhEYA}kw|&aXTasQ+KfuOqubyyy^swgicBPkB9DrNsp*GkKtwE19H?^~ z&Ek%99f&6_E-yz^nMnkY{Yon(0lwcw<2E1dVcI&=1}%cZ-(JG#a+WmVZz#yI8$xFT z($8jIqLAV)JiekjE5pKJck!@J&rG)j*a@C@Q8l>0Pyq36n8dt?R2PLcOTP@JVaXv; zL}5xM&cNtm%g+KQ>h(8;#6nxDq;?d}+IT7Tn$2@^LKBIZ>C%P}1k9;O+MaDywQg)@ zbq&(p%4D>8l6|9bI&HB-8o*U+U&BBGBp6|3l#V&x*if0< z4dA0zi^~Bw+S*@@WgU+R<>cfB$MU*?IXpN%-Z5Ca_cr!%eUonGD%dLA=5vacJX4mE zfj_}Y{zT2E26butgRdo12LM%e9my-B+zFyR=k8-3kyO2OIB?8L| zi-7*v=HR9$O-CUa(b7YUe%F-bJ2mZ1TNL-;phWk#^n}MJmbp#kX@Ky>^sf-KW z%u7g9S5NGhY2|M?x)1eyA`b;M z1i=Ory_F8AGqGliSmu=Pg1*7N43DvVwd_rXX9}H=2L0$X5S(jS_h_@W4iP!H6L{-TX1%q(8jihr$0%I`>!ja}&n)xzh?`FV za@um+=!ZcDd6Q+8fHr&=cgasNJvKCzjm}#kF8FF9eqo%9g!m-CizfsXdtsEtMOOlR zo;dsC{%D`|ZmqBwx8|zcuzb+{;La;AD9Y7+Vnn~#Z2dZ?iT#5*|9h_jQ{zGf8tYKp z;`$8^`jm{?cTYxWd>!3|ClsOIMKfQcCWax}FW#*kS1r^fwKYyy8QOjdHWAa11%ae2 zsycI?HN-h{Vk~28Hv(IG#MpFvN(#vA!)k^qOBy)2c5AQmeGNf`mZ$Lp`&Z!v_`j>y z&rM?84oNE)#qbmP0^|#JNPP~NQ_nNacp>t4^ReB+rAG$$$R6cQG<_e{mbWi_| zNYlikKi>GjW4@~TN5ng-dV^q#O4?K8dd6(du`~R5CLP8?OYEE`E|>$f*}Ez<=iC|D zl5J}EX-`0Wab=YDIbSY+&9Tno+`JL*cjj{-#?xPmVLrfP~AKwLG z(>ebPdM7R}+6QmL{KqYQ-c{A&Zdn11w^Jr#ewVc*+z!Soth#ujuDCMx8#A9q4O|Is z@pW3B79Tu8`M~4VxBSvCeG{<;{OU(@ltQD=FTUC7Qy|dh5tz@O-+^wNl0s;-%S1#W z2Mn-^9b_gHTBR_JEnL0UQkr$XSnQNJE=+M8zikuT$r;e)L1_3QB3*LFD`b7Yn}maj*2 zs%_F?Lqzu$t_^uZC0$x+%WP2W`L|s)5+m@*id)|!#VBn~^|#s{lG50o$Ft0+nRts^ zbTK41)fH1l-21uQsYq-(3UxgTykW+7vVJf>`N!AIKE>NS~KgsBZW3^MzH z^meA-q$O?Qavc56IH>Vc^D8mz{l1v`s$}%8$Bqu{*DjLHIRinDR2C37$%cSqa}2rUdE zqfq?7q5Vy()YXYDBmu>Nwo9*?83!IHn z*Oh#=0hr*WGP{=v!C#>DkbI#(-%aZ+hqxNYBiGf1-+SpyV(*5G|H|+k`ZTG!2Uyf(Bjx>IG?p1v4E`TKBdTM3a6y-_pChk2=K zVylRQ-s?G*yBIHnpquLGu(Y!$A+=pD+k;ofjlmVUuhO}^H-2(zT56}_)le4_sJPN9 zb*EChhIKYBDo!I{-*?&XwMdO3#)vltKFBDhL#o+A3t)9 z+;n0!Aa=$Bosx$R;-$)`pfCgn6No`7)Vmshn3P*NFxQC z2>{_TbaHj7YkwaJgIi+xOc3knF4H=;-ev;Og_z<&r-Z{+00)z$6PTgjMD;h@=E{!D zA;o{BxF(_T_-)G>%uV6q3omt%4`xhij0^u^D-a>P8(wiIFis{(ZacuB|AFsUWM^A& zGxmfTi+)`BGi`U%asq4CE36Wk`~I_l>q;aEF!U7-jLM8H2K~jgC?6zZsx%yj2u!S& zOHAp(mTZX0SZfgo6W2+FM<9!Y4g&WhCw{<-+#Q7BaX|M&RKa5zr4hH`Vk-yA|8!K& zm?+^K9UTFFc1m(`w?M%}fjtZkZeEpUyH9vQ()t-jzw!?@4yi{I z-Pz^ooVlE1$HTWRumq9>3>J^HDFM-*>YFKuc(fVPFXHc4-uGE;wpfS-j`U((&>;fJ zsrcSa{!8pDa!RJa$XGj67eH^$i)&Ab*b^}AwY&ogDu01fW=5(PHz#bY;O-6FwptK(w3GiSVZGp zWg}7@$6pR)kHc&3Iz8g&2a#_$@pe_F^_u1OLFGZ$)EIoO!OUV% zgVQ?>`}Fbtfa-M8iN{2Y8<@GT#;Vx{dv)NP_4@Vm$m(c$hs?Aw{Jxomw4hw+kVday2K#<%l!&eKAfV<;mJ_DQiAW@1*DH#-d!iUClN zIsj&We{bjD&|SyUUrIw??{NUZ1FHF#)}Ape#K<#xIMA@LQ-F9InwXjSL;N`?NX=TC zkNrp(d=TPmSw?@}6!h?hr1FytQr9<{q*zp|1)fLM%qyn6ZR^maZUWdCM=sTO%7Xjd zNcQdc*=0#7d3^-|&uT+z*NKluofNN5hY=^GjFaO`!Tgub{2_w9vEbd0?sHReVU~lo2z$MSs^6^zsJS z$PJ=cow{^f*5E+~UvEtn_6n$^wbBC*RigDsC=SUcS64d$?6vrbQyZ#O8o9fbWeU_} za~cAyE?O*Z>nTjni{%fIHh!XTEOAgAq}xPD(V-AT5hyHStYtwRX&|`tg~r6#OqVE! zfegQB$&00|xI?qd%sEjh;aUsL$t5z$(l%6of?yw-vSy!`LFB2yKw0n&JrtnJ$^i zvGN$13h>DF*&*~%mPB2|8@6nZevcbfa79|pZ;^@zozMVy=p0T zH&RC^a4+&&VqcYO*Rh8{Kv7MHIsTSb*`zTF4z@dBK+yeOAcNZeUJ7NnQN8UnGOS&+ zUB{^@=izj1ykCV_z`ZQq`Q;&q14x|9aAr!B*gG|Q3n@EjHgq!Q#9nG_At|$S+n+Rx*CC_lWyLL@ zk!2vTwDiE4;mHEQ<{iE4u4(Q$(-7=9$`BCl7pnN!eO^)PSwuj9ZI`6s=toWPFNlWa zW*E-t1%vK?sa+1X%rGpUu>`#tFjG z<<%Wzf4eG$1n^NK%s4bcx|EOh3hYNA7)`;7# zCknk(kyCU~j#SbAB57R;D_V7Kya8_efPlaC6h@`g-X-7|ZXq--K5 zGTV;iwCx_uKFPL@j-{<->6K=%uch~7+)z?rUSCL2((r}Egn`idO1YBTUiDxj+S+0{ zAZJ^fq4Qc`{QS_6w1BiqZAay}+MWIcc2C3|v8j0)=b4^a+0T|h3SK&vQ$z-@Rsxk= zS=`q?-^FrO5g{S#B3%YxMoc7@rLwlR90`seGJ;#~XK`gRyLA1!i8z2LH&ipU@RQtF zW+=8XJt<{g6lUPT8?!Wd%SG#2)Vs38mE?^1P*SiWE(vU^f5Hz z6%8b$44K4TatEau;xn-ZLI|D z-XCqZQftYmuRIL;)Q#pyJx@WZ^+WQiYY3^-C7 zb%YUJyJOz2jn6FCHWNiID`(1CV~))D*h1$74>?rrqPC=l{tp+pc8{snue!q%TG1!=n!@TO>>Gt5!slT+p)P(Vb1 zk<~~-)e8^u>jMHzA{N{pav~OxU67UCaRoej62y&{hfyJ}FZ1vwi4-|B6^KP(Bstj9 zeuo$wDbaOhhOw$hxZKfT zK_`y-8hU`tt|4OuqX4Pu{lkMVy%Z9c4XefP_PE@~*&8Gb-=V~&L>e2|5Ld_r5ZV0& zHD5zQK#$o!Xe63^lB4z!yU3iJjn=qrs8{ z+D4P2?KWF`TJjLvD$7szXxQ=xU|Ixb7>zkHmahchzFNrdzY~!*au=sV`HwdLCbCD; z+?=Y_$)!FS3>TtAgB5IA^!J1zigd~AB8^6iLZ3a7YqV z&FLkn?bwuXN?>UyXuxa$-L4ewHCV#-gKU&j zL@#BxK0m+E$_AD5$rt6kw&S0O$nEwq&Osll}k_SERXUUa3I^iu#x^=+8XMax~&`Lb9E0c^i~?)`>y>6 ztqL_Zi3~Pd>5BwZ7&S>?HY_9_2RkfW>RTDoz$tL71}K?0t4x@IVF^%6=;JBy$=(|GhPf zAyWb!pPWR8E$<2L*#L?bae(He-P$wO;MeR{GD^dFEgc05{Kg07;U+ z)*dQ>gdQpU+S?o{t$iUFv>98aR+>wRk1UoJvQG%rY>*QQ6lORBHV}=KZEygVS?={NbJQ4JK znVOzn^!I<@U_`nKknFd{hjc$MV?jS23IQS;+);AHNR&WG^Iyif?C-EdhXq|f_fcDnO{xnw4*jCLI;Q`s&jdZe3e%m_h{@Q5yl>tl`h-N>MkBu4fC#c+N~Vepsgzax>=v zxiP7=X0B`q8O~LQJnS7nE(!ou{Cc1b+!#bUHk+k*~wf>SYp+;y>|C5IW>1FMs&9_|2GhL+q54GA(0sAxC%Bua7 zEi7IhsXd*KloJIf15q*5N5rxPb+K=P_En?li6z3uv%ib{{vyL~;RDNAR^6PDu<}-} zPD%dX>~15p7Z_XugvG^0z$Y3bQ-Z-@i5VUsxB$dO0ax_56BMkhal$9fp}v>9Nx8Yo zz!U)Q2EZ9+Czi>7aRGp_7%WZ@8?MkS2lAD==|F_u?Wu%?9`Q+hZ%0BcRWb&Zq_w~M zKw(*kI!UIy{~L(!A&PFTN4{x-ew<{ph5V72om{MEvnbq;K^hWdwWM^jw(3(Df z{tRrZhY0LP4v3InEKUTblL~vCZN{(ze7L_;hp|{o;CU_jcqQ{8B2i@&A!nY%`R2S= zfqyAo+4?xvXVbH5iICU_`qlq}hAqxS$#kPE;R2dawUOtuX~gn~~5Pq9p+fsMSFNB)p)q-h#CJIr6;DPg_{uoE0p>ET6chGa5X%y;ElkIg-w@l;jk(b_~ftAcD!Rc7=jqpmE#n zQ3d&P)@9}W0-Aa~3tzYuc$;D9f)nHH<%E!d8=zo%!Pk+3GJ|!m@S!I%h6wmk&2V(@ zH$R=-{nHiuRN7=CAj_~;|1^TH&_dUCDK(6U>7yzzRYBJoPD<;OkfM5edNQnNZx;dV z76~ELutLy%S6xSEAPtZXZEv~!M1HsUEGMkwsI3w{SRGT4c%>bd59T_zr{x~$e*2+XExtCJpIJi6{L=9iiR{jtoc?1 zp_2!iVls9Jb=;~6GaCE7xpBygne|N#x*bHiRSpM(wG`TaYCB%x?y7b@&e}u>uOu`Kb6y z)YfC8rRzw)8(;NUS=)tc$Ab?oKQ3W$t?P#ZEqKDTcIJ^fpT@d^R5(O~@`1ujISg++ zRPE|Ao+Rj&vg!Ce;_cD9m{R})F(EM-rq8e{K9@5p&);-$e(k-=8R^63CghPD0b4F@ zj)55U$~Cqg8cx+4D#*=)J)1F6BO=}GX5E*zr=7FQXBhZ_|UU-rAvPkt|~ z*kEq9JUw^or0sS7#4z~>Tf5_4;zk!bYYx#Mriydb)xmDzNmci+V@KU0B5a}Fc!f>p zB*qhH(Ph1t+L-#E?V+XHKjIQcGx7xg{9vRO?rFJO1M1xJ?Q9#Fx#OGNN6|HzgFYe z%?pLG2ZstIz21>zzZ!iddhF0Tk<4ArMQwwnDt)$O86kII>sT1X5`u$JvFN+K zXSqHNZ@PJWT+&jU5CuQptDs!KnWmKxdN?Ic;#4|=zoD3>!n{k5F212?<-?`>X>DNe zO~*i*qW<>??mKP)|62~7->dIWclM`>udc3KO3PI9hG*2o$I>o*M3xOW3JMzHT}$!G z1~gbyP4k}Mb%r%A-jj%lfR&U`b(r}1@tc<7{-1LvM=K*m;ZGyGXSUy2OUf=~~^^yzTU@Z&6*>RRG*HpvVn~S!n#mCzKi%=HVVvZroUJOT1fgAXRy-rpXnn>XvSPM-Km-f+6otWQiB z=<$VStGY1`-ZY85?S}$l)gwM9ap|`~Q;$;KNl9sn)8|e8@07(}d!KK=b442?9r}H? z`fEe%?Q0qt8uNDHcEu=lhHtp)J2`D=df$&!^pFFUE=FQ=_~-lys>iJ$Fn)SP%^TrJrcDjZHo z5-thmhu5{IuG(!rME=n?qGzg`d=)=px^v&A=HEv|sG!J7LQRB%gNA?{5dz-{D55pb zi4#a(+CaN+ZG`+Hk-|dYr6ofOI3XGF@StzH31n5p&WvFL&fCs4MY}${aF$%(SUKm1 zuW0bWm=SJm09MWLvw)Fe2S1WcQZo<3X93@|f~ekFxW*i@VTk7+g#f>*+QfztO!xLW z%5}E_S$RRfCq@QC#Jb`kJ8zm){E-uqcncyvxGzg&BJUHD=>BiZverC>SOrc6DlDKV z4J*N*v!+;<7HmNt?wYseTOMIVWn z)>?Tv;iSRjH6@8R;M|l z3~w@1K1<#*hy2&vs$0Uhf>$+1)7c|zOc?KaQEqRPtVL_wp?PgVsj_24n%wg%D!+ z@n4&NH7&QIWlQc4ffb3M`V>Eep9C!x6EE{i9?d3B9DbDX)OdF! zum9(I>vC}6aqOcTdGdx(`aWxWXh`l$zj$C9WGou{?PG4$%jkzvT6+OK@lnyT=e(Ea zmB9V`91;I741$DXN5P3TprIldcm3R@r~97nfk|*~#Hk$4^2SU=2pCwm(&KFP8hjmw z>bKU%*`H8^k$>75#8z9$E28a)s=lT7SM*$K?8GXgCUzDZQf8>MW6v7j-(i2-5sFu} zHo5_3*t1zCH#5C4XfLz*#LvrJVO{62sX32c+ZliU%~0LYmha|T3#&7sk?5qNMDMV6 zH%*hCjs*U9mf(lRh~EX}DXNP5usjle-xb?KFwht-Mib)R82ZKy2T?}!GY{}?(@*95N=(3 z^~3l=@ZvJAV)jr>y@h5jR-RyL`>8JM948+Jm>q(aS2>iOG&5duqj+bx8f>ISCMxs2 z4VxR4_w733SVjx91R;~Ce|Y`v)C7b2f|w(^;5kk`>Fx?B1so1e-*R^cz*5%kU?Y z!LHZaU&$luLnlCJKN~H=3W9kf1Q*dXJp{?}e7GAtqCAoMyQ#nWMf}Gja8DKc@Mu>N z5MD+puHr|FrW^FLE&F~n@}k5H8hbZ%Z2tTsG12{{8T7@VSIVm%9tUnwTU@ri& zqFkPbD)aO6z^bVitP0p2BDDFnwXwgZMp>DL3`5JyVeEfW!myG7xL zp1>RH=5(u1o_TR|e!H^IdlN5%mhqx0-QRKjAG9!Q+nsPQdQpB}_fJ#al+=h1SK{0} zJgxMoKK!&Aj_14ByD`0|eN;PeJKH3E*fkA+glOlwe@hE}T3UI|EW67;bT$xEdpQ?= z8cGE6dsbqLoJ4Dq`=MTelX~6$!Zu4ZRZrsTo?9=ELV>lck$uDYG@j#QF>oy0gUaHM zI4_rf2muw2mg~%(-z0%x{$Nqzm*gHhryHJh1^H+5RAH-g41EmNF1s80t3TpggXEwzU(i`n-HV627ZdGr! zbNZCj_NHh4(DiuFe8($meU)9;JRtILLwV!<(w1MmAtm-4*fSpLs2uHyG(3dpcz?R} zL?`a3wAaV+Lf&HtO=zh^ooTf9@n7e3*S0;Ry|X2SMq1;ibEqI+VC%CIysA- zxAh!4kCHvgP#lf$iAObBE7|aN4Ps?*Wf2TS&R+ST@IHvkUhKfzHYxj6D{5Qdy6B|B zA*di(t22#Fkf~U}IHjs9F=s6EIA~|w{EUHe-z^eO*e2BQ2*>ufh}fTS@m)7-ayavN z^dS~~zCa-gm{x{{gN2vGr}3gi?}ViKdx{?hZ-?V1Wx&G2lh%e0rV|XwSdK-d?iq&7 zHI_qALD~+lzm^ZSg&}#3j)EIU6lr<^EeGRu7CsGL!ZtYuL^mU(6jdS#nkfmbOzrE& z5-cyD%rNXvu?kh*a3P$>Fq4%dCIMU`9JHI;ux!-ntm?cbJ?ThHt-kS%bTl7neC9pe zM+Auo^PwsZ^=hWBuS#iQxgy>=+WiE^?j%WN`3!H1hxX&!woIL@$cuA9P^Si~fXc^@ zuctQzRlbY$6Q#3AnYv=u-5V*P?449y96Bo|&*j;Wd??!IU;<5TcM^p zOi2wzdV}&S;99-QQDbPwr=tgxsqxEA&bSH4g)D%G&@3D%cD;g$i3-XO@!~_VH(Y>7 zS`33F{X`l8j%1phi~(&2`w12safM7R~}QLMfoYli~gyU{a+#i-uUfzXvinN zi#dO&g&)_EFMs6FO^iJ)N{`NYMlBjib#zZ}|8QX+sjBDQ@S1S?0Eg|dXSQOwRn{Eu zec(1y=kv!VO_j$Enmme`w{vCteqnB=Luqu>=6jl7=E-T%0uQS98LG&yT}8P+ot2aO z0u-zY1NgCKLb>m5F)kHs@szKtWp2OwDE*L%c$^hUj@AxLw)4il7JCPB_ThTg)od0+@-Zz+awr61D^W*PTgy6e%iVPsFtO3rm4EZSR#Ue5b3W41-Q(p*+v%`( z{l*I;Cc9@$SH$m;Z&TkXE02cnIzv~yT#q6u*=_p_hF`zz&Xms%KTPN5AMWgC-%lE{ z9|gXQq2Jql5D*xufu6tnb}c66H%19LOqsn*e^@1a6wtCMh;iL=7tTD}8uF{&9P_OK z$Zh!~dj`ClosXd{ZhsIlUk6^HKVLO$4H3q8(to(~j~tP6EiPD>E!&(}3iL;B=_-#J zd%QUO@p#w!@|!Wgtu~wgd5k{b=?dqY$As#l0;>Oy=^Cz+iOc82jj^-mV`r5>SAOyR zbUcpg#TMU{@6tvO%ih-OusIEi*E!x(>+a@4re@E z=2mrIx!!lQn1l-YWMBsFW=H)X`F7LSv+wM!T`M9qm~^Aw`n<~yEB~CA7{EJ#u+kmc z(p{eGz4e6nL98v&D}=W9E4=B$vHXc*&2Hf1%T9jHiQHSkQPY=+yV~^4-p6MrLX;Y@ ztP1S-qcIN*aH@|IT^TG$ohQZXs`TukiHz5WPQBfFMy2%KEiHzpwi|&jM%T04lxa2}6 zL##ty;F}yL+QE8`sJ;f|&iqekU>oONgCyFrhfyhO zw-H6P@vdc~Jwi6$Fu!(A%?$4;}Blz$~C?wp%#$1N9KtQE%K7fphH zPOnc)X>0!_cj|k*z#!Rkx#TcD1PbDiZTf8))#jB4SZyiTa_-3dMHR;o+?m{YmK`;!X^;558`GKVF5E zYihxa^gX+I$BRgjJ<-$7EM~}#MpZ3ZFJ#D9Jn)C1IVYC<`RR6V+S~UkbSI{{kG@mR&Y@Y(|sw30He%NP@7qS6Ev8c0a7UZWV7qyzK zsm*os9jr$R49~SMdh?r0eZ4A_WtPqFX4g#A?Jiy}2W7}fzip~nPGNa=6Px5gcI#cT zD$nEm?;?Nt#(=@GaaN9SH}R6$q3YVzgnC|rVa z*Y{%ks@`T*T}e;7@V+K_eP&kchw*mO;pPzfr1bW&qUN$uPk7o;{BZ1`2cb&IZ z76A!rs`CNA^6T3c#~08{nm6xF&~O#2a2V(WeP(huJUM%rvhOPcdvfPzIky6wvTW`^ zTaWITZ)Q{}{e=nn@IhERqb=Q2cbMqV;w03qd);b3W8Bi8B;9wZQ$McU6dDlfrS641j z$PI@n(u>Z&5j18cm1g~;<3bEzvi1-Yx_-1IWapaR&J*~y>Moc59QO6p`RV(V^9Hz9 z0;<*MAHfj#Wc~aruWLUn)c@nU&9j9c4gbpG^$O)9mg}<`j6-R%_n+@0ZH0X>76i({ zaVPD{ZMEa$UuOyLfBx_jsMl3;IdXX~dQIMadkHEza#w21cMY)O@U!$|TwfcOH8fS1 zzG~WNpHu!=o?0?6^6>l(zrNpmn7upcJNjm0ledRkMWiV;Bi>@mtnpSqPa682xwe31 z*W-`F8^60k@+zB^1h#iXRgCnc>d8({ZKn>+NT9K`F8>Qku@MW#iA+=+RU1hx4S0z3 z)*t!jV>@Dx?ikWu_N_^uth z+&NX3;`uKmn43M>pI3wCyzm2kM-Z!9f8+f4U8a6{OZtM^q4?=xMr`7X;RKgL5~r}6 zdrX?Lr`cA^{q?u5{oueqtB8|Q8KI{41bzWOu?ptt$P7UNs#SwSTtV208Q?x zPe~pQC|?~%IdHmQlpPI!gAJUdEix?9r^6c&d75-It~u|J@+hgt>iE_nI&D?y6ze>7 zED0SQa4Mn3m^}!}2)b5=MG6VZnt{Z#^N4_h+X`YMU`Y#z%IFw%jnveXuB3b|P`l6& zRT`Zghb*B6yG{xj&KS!>3r}yOFKKV6;eM^ha#u|+9g*5A5~V_l(pn!PYC3{eO%i!_?N6P=1uz?n&AAj#Mp zgitTm^9ddWAIH6i2(T;a{7avOPMEh(hXz*AM? z5pBg0D6d&(X1Ig}rM+~p<<^3r=v-VqA?4k$%I-Aj+D^SzDLy4p_sgpTiimK5IyOVJ zc2OatCkipxWt*z>zo?##(DXzimX;DE1<_7;O%(PJ&`KB)C>k$m>kg2nm$0PtSl&~O z+Y_-|zC}puH$Q}}l)(`^HCkGn9RYKwH!p8%YLXc$98vt%%Z5C!&4l+Yo;#IEQzb zkKAUJPNB}TXu>;L64TXM+4d4}AH|yf*WMh@US%b#RTq}VgLUyh$RZ<} zDz>lHZ^)l9#JNp|bG=;fpKbm_X6OHCEQzjOC`>khp+#`U& zY8pbTc>c8s;4?ALpMZlUg0z{30Cp81(7dx_4%nRB+}w9wUUk+J1V9uO2_BYA#0M*T zN)CWKLuuX5zO?KDGH~a%;>z^NJ2||Y!)ISRgH+bZ14lN!FRLO@T}q{WL!@&G%MF%c z6O$)AH6!Go%%7zgdmBp9P#e!4CIbD*tKUV9yQ{@96Iy9uLxxktyFbt_P zfac(2X3=FH#)ZQsChZl+#Z+8@V4<{06ZS;0B!Vwr>GL{mgRpEX$Pzc<9r9AL%E}+f z{r4*`N>yTbnl4_cPZ7H7IK!NIJjmYzsKnR}{PmM>KRiP2Xp!5V9x%fC#x4)Fl8SiVn)iOM-Hh0!| z0~j5gWR+=aRWyNid&@(~JKN|*>ZOJa1r<^yjak9a%*=w5Xd(MwLBR^J@mg`ynpTo@ zkeTGycxv;1OMX*i{CPqr!@v$Gx|wW^DJn=2H@J#qrP+bECk}5#g(ws-J z6QE!)d?*_;tWZF-$v~3egwWNsPcBwv{^sV0VZF}QK#rUF)LwbnP8KC1Es~q_Gf|Rwfwm z4(m&6DjVsk8Uh~5*-EUXp_y$67$Ra}4z*gECeq4=xCr#Y zdT0%q3va9#3eC0~nWq#?;0D29wfR@c=9;h2!P;P%xNt-?m)^v`6Oz9(lM%;+ zhzLmt7bAC%+FY~uRWzI6;p@Ep<3SP_!<2U>2llsLjOL8vmrg%b7f>ufUO{}q=1RcA z9Bt4T3qn8u=i62*?&Vd}KxovFGXu?v@PBdvEMEpNrNyMjQ;P161vSEdS3qwM)>NpJ7hhiDnl&)ZvWLBX-FUrE^5*o@qV@PV&g zKZpRjQ%>JHGXVw*;_KIfdPhCw(V*l(Wv&#by~7D>40QJxD)YJ~xIa~A9{NG7S9KUJ z?O#ZheZn4lUM~C(tED$?hfHVR9`8TI9g@(lA?QYbS=pMlz8!g(xj?(T<}UU`zGh}c z6YVGb;TvXaz4!B1+$D5TfCls1MDCVQt~4tZm){t=iLE*6G* zuQAkNvR&Iky+1neOv&H!KWdh3Wf=F zl1-yxi*@R{MtAyUBk){fx}8t?2Z1;JBlNHtM@?DRO#uam0iThGMOC2_MG-U0{N}dBceU-0VF}<-DlCi03`HDuasEZ*wXI3t_JmpkW&0xQbR9yjTcSd9)C{ zy04p66S4#X0ab*7)3WQ!7!@LXc{ZFD+E!9?SZ0>FrPfijI+G?m(L>l6nCwYfzHc=~ zbhJrjVaxH>L^5cpa;a0N&6+zP`vODZ#83#;Sd^Sl9##GX4jN*TH7+EYCJuq8O3Ntq zT%hJvw&E_GHE&gTNz@uRI2C_+sDU}thv>y2R~0m3y?ZB5*Da*}awX<(fxX_+q_xjS18dJK4`SPW|2D}w z*b6!pv1vt>5*4heAS?ZP=zteqZzU>HkBIc7Z0qf4^&zYYFqBi zsH7T8K{@MT!{TL*kyv!;fdHE+32zQu^pO?xb58IO-bv^1&4CB;d`=!;K4{{`ju|*! z!ouk1{g3w7ULp+nGzH}%`K9(xF^zKcUkWmP5-M=^R0ij?Qwc{)fw_hK+Ypt6BaN}@ zgHhSJ{|xays!(elWVQrxoJfn;Gm0MP3|j-8d%1WV;=L^a8`9w2;`e@Rg^ZU~AYne) z{f9-PgvOi!3x3eGNVKmF0hp{XiV?OjdKuMJUeyp?F5!JPW66`Rld6Y^=i#-rsr{N;(@L?yv%IRq3#272y;`x&R|5$q(9BooI% z-5;&m+0&3^J34OI`e=JmEW$M){<2Q!d#n#@{y4Sq#!z)&Id-Q!U^;AQeK3`ObG+`w zC+JZ8BPJ;sN#%eak>UD)-35I5a-;jE^NRb+Hr`uFww29V7|(#z9cfIEOBWgps?v$Z zFxryQJ4KrJ$RST3wn#INPJe%B?mGICay#=z&kJtldxgx7jsg}F2cWt@X6Yd&pTI+d z1dCq6_f>f^>YpyI#N656J7=QnjDEhVpSS=@o;i30vu4%*y@?ZiL}RQG^vMqXFg@Ca zZ84Q(7Ki00xp<$&SLwUycpp{7M*oByDynF9rLOljM>YmZX_{6+=clw2Ot@ZZgsI`{ zSIgFO;5>R#wn5vjI2FX`h-Cd7?m3AhGol#7b3ES;#?K1^bn0#{Q3FNDUxkI-0lBw{ z>O#7jI<+LcNgKGHw63s(YAJ9YPpfS>uexY9pSC5G4$Q00UPC>B6GG*-3<{8{fC#0( z4hQsGk32qSMOBTCt|2vONutq}G!l?VDD&_%hqQenb~NWLsWvZ7FEc3j*yTl*SvkBc zQCa*tsSqM3mM$$TlST_-DI)uxIkY_6H~brgwO)z6EQE@>LU8w)*q@kCR+k5HcXE;g zq8(Fy9re_0wI_*~NH$!j64Z%GTBQwrbkN$HBngrDFQXxW z&5_T&PLA2F1l5TW- zVU!{?$wLssjXt5EbXNh2F6kh{**eGl=QER<^3n&S681p{?Kk_n=Mf@vZaFe2|4srT z6IKA8c*=WM1RD9*29@6uCa8ka7(}0uOrPC@FpX;EwnF_zwYxICjoYzniZ35M$grsN>vFrU05&}iOi7W#bJnc>($7L_GOpDUBBI;!!{YGBQY9t-L6VJ0vy z+oc`3a!DyLP<&}=>96%56Mi4(#5a-s?I%5;DXGMrIq1-2(exImSJlyh&>{m#m252= zQbM_h)g3=}H!@ZO1`v!|n{u42GJ+#7EG$Ayv&?4RpsvVde_ne+{O0+mzkcNwJTD_kUd7ieEZtDZg0Vba-C{CvFVAer6vwat)iLC#27g+&{&lj}*6thQXBQ{N#7vl#alAXNn(ZbPuEIK7UHR z8_wV;ZC75;-zPpO?i&-S3{m-~ZZcHwEKhh}=ieDc^eVoM=pEI1x)T>eLg3Ab_SY>o zKZ62E4vM~HL><&d*h zRF$0jveT!E$zV{h#V$fhJwZhphWllCmyaWC#K;Zn`-D2;b8Q!r+tJSSK3mYRZHc$O9=mdC5|xg4^M#1! zN{YR;g4ER+*NY{RSp1$bpoX+x(jo#!h~&(jR8?R2?G-uE;dUMtMVZPhW$CBtpx(kThJT&Lolsm+S^=j2D~VZy|B!E75VZM7sm@dqZeT9eJjaFAcqO zh)2GFZeaT!e#a_rft2Cf4N>#*g!@Jvaox1UQ}3OIAa3YSjQ1Y)o@}|Uif^o<-F=P~ z>vK$)y`l5$VU) zb@h(Ps@h!hK?-_FxK#IkY8De>7J>PSoZ67a*pqWt)e{z!tt~~kL?+Nlfz*h`9fe09 zD;pB=|5;Q3?#>$3kiMZqV?(0nnwJsd7hD_L5^_z?^T1;wz0M=olXpsFEbMgMwY`gB zy>$!;msa_SPRe0;9IBkcYxQj*lw!^43@VlSH|7}R$g^I1yheX+o&9f~T`i2(L$cvV z62;`cTVEWVI{a&1hR=R!>gy|5Thl2kD+37Q;o)HbT?x3c#P0hrHTCr5nN$0ZZm?sA zfJ9HgA&T?Q#^9bifSduSmiya%p*f`C+pC;g4_^tfR%})~(g_9Jttd^OFh7HBP7E77 zxjr3e0&<1z(m@;k{dl06 zDx!f%R>99}50$rhYXxOK+jF=gQ!S(7FZr16$MX}b+kCLhkW9Fz^SXd}0|AfYm+NwM z^f?WSaP6o+sA9~qXN>r;RmG*NA1_w)uawZpj|`r}AMY2pFg&-R%5KEG&`N4hod|+X z`2Fig-(T2Q+Ju)QnIcyosS%-_tmzAb@$0dxZ;L-kM<$4aVTn*GPL5}NxFp@Fsm)j` z3LPsUZ3jBs6%vZ^=@r*fRFP{2Dq0=rsf5rt-ANQPcFQwg6dq5OVvjfa%L>-cbbQh2 zR<*u%o1NrGg^BudLxx`!e0{kHLI+W!^^NJk{5D*~v}!Xd_8)l=ba{Hz+>Ke(Vo@`_ zK)RK$)nSl2MMB49KWdS?WJ8W7)|FOg-^PuM$1>vS^8ZDe2`IjAFn8onSd%S05eex? zYQ83k$wgcyMj;V!jf@XZ#VXH` z9>MyZ6NFJ813GSxgu#Jejh`Y!Yr?d6CHNx;S6}b@`!`<>kr=clZ4&+ImRNy3u;l{5FV&v-Q{YckCpUm7p)BWNP0o;#=kDBfBhJlT^|@O{@mdF zcw7@he0`+9;&t}A%_=n8E8Dd@b@fjow}1c=UAFH`zVps~2Kv;&Zj3Q~%ooIErtm4F zKzlmfF_|5N6_1ZDn2*DeW7}ryOM10m`eKc)MmC%RGidfg?`6VEl`pohtGSD{^OUz( zwVd`_ls;EhE(p8^(pxtI9=`*jY@iyc^|S&g$Hk$=K^`6%S?l!W+1%WWDvYYAu;L~T z>7yDL7yw`mW?&Knet!NDKtr2_g=NEQ#TY0y2q?>9KP<|B2DdoXx^4BmyS_7(k4(@D z_WRKmSzXN@P}xZ4$zx7&ihOyjnUDcw5kY&0Nf=gKf7aHmhJgx!R=yRf%?!{Jfi-=? zR+frSW*$V=A&dXbF}QG#u4KWURuPEmn4oiKqJ>ZaoYkY<5LZfbv8+R<@7>OD-V~3d z(`hT`2l>UM`oGV6+?&Y=O$Ph2gl;dkX4cxDf1e5ZTv7q{HRsFULJ*FRC(1c&4QrFX zfZ+d=@)ozPWAaG~wAtgIBd-wp+?CvdR=xw~`C)XG&HEEtzJG&o+-HS@hoFl#4 zuYwz+tbW#8%QH|EO)fNwK-$c2F~8LPn!mXmB^e8s%sV!Pm6YZ6v2)#02-h1H=TX?2 zB{R^FlPIVy2q&Q%o!gPsj!*U&q)|LS`0li|76LcB$U1>5`jaDjweAZ(;QUBojg`OS zuF|P>cEo*el9i>%z0#52ZAL&>W>J1(=4Xh27~GJS$hxp49#K%5d&tcXnn9kVEVVT% zwLGIoVM9I~ek7(qFN`MMm`ZObUyhMNeAy1TRW0LW5!7}RC3JP|tGa|KsAmx!HZ7+1XW}Acz5wi106m&@-U5{BE|^`)>e`(H{+mO9*~8W;fBVtWeYLg zWQ>)5{7FFu1g(om&cR@B4O1Iwy!)o>HUSL%#JZ!t;?vQFF)E=pYs2Y_8VNMxF7ISXG*-ii^K|li%GU3&R zuF6uu z8nN5h+7gqK54p57w6=~uJ&6Ep>+kC=JVHX_7ur4$OU$iA5i?+X2W*7^xgxZ-l^ozN zuTC0_M`Y&yO4j9-=r_>}){Ef+i-opX@DPIWu?%y~xpJs-lZnpGq`$0AT2Ga*lK&R` zBOQBwT+#o^YFCoJtjs`pbv8(4a3s18U&(OB*0L$Mw;%j0L&8>xlUVxMZcEbA8U{4O zxEE7_u}mkucfKVq>-!NMLrpO(w%a|B_<+xp4@m0+|(g=@TRECj>l873fhm3CeRV37|Fs!H+ z3X-RIdE!>ag+r@RYSeS=l-K~ISkM$p5YZK0j}T={&exrJFsIN_taarZ+zq9aMhLbu z);q7HrU+Z8V5)4#N;Z+!V^exQBl+s|p0gLX_Nti`rVkkr3$r^8<>ViYK1&{o9%&eU zPC}dHqEu>_Hqn-krt3ehGv5tYHEJtQ3v*Fe%{=nYxDQC3*6+@{l z<5i;t{sO#i8dBM{LQHu7wZ}P1amY>FG!VrTOEl7o9A;{>3-Wy-0lbbmCNW*?{xl={ zDW|Q+F>;G1WvhW!cq`KwSuMsAJPZfPtY~Bfu1l)UB?y|>1iG-a+`baS8k#D>iq?&R zuf90(r1PaiZqOGNc*5+pUQ;B8CFmhzm$eX-5&o~{_?6Bh0WhD5qPIkVdf8MqKY&-F z0Id2MKnMpw!s&nni@k#bFxR{)SN|CEc+r!emPWwdc_{^`%|7HB5+C%{Y>df^|91be z_16*x*m66n{G_x6eFnua7lgYf+{IzXJ?g6pm$Q4)MB}Krl7IOnn}G*`rm??ej^%un zg$PUaDl2P5oL^eb&4M>ZBEiirJi|!IxekGIo6WVb=_|5B-aGK%#vs+}yNjd2SyuNv z22$(yRv6`l;q4ZO=AL2>CW8hBjy zv~){-V8Yfg(fKB>{F!S%i(4fJM5=tex+n*ch5<{w!eO2KV#a7+veBo==fL+)^LvDo zPk7%mmxV7r7pb+4T~wPsilI9XsbOU;sU+SeQ3*Ix8;4C`D7w!8r!Xo5nR8V*N<4=? zRp)a#lU44J%TNYNJ>Cq~tvV-oho&gTYBwMvk{+4ix|l$oC_^8_7K1cK_xo`e$y(wS zfCPMbFvG=hdhsBQY)S@E6U@#$G419;eFWY`&{2>-a(9^e+F@ zE6kTmOknu}jJWE4e!_r@B&u*!85%S*P~88;*UExg(1QPDe>hG*-bA5nqfk;QKTi&$w{-^QAn;*rE1tw@UqA8=s+9BxR(Z9}6z5}LPPi6vEoHu#-_&62nzclHoJuiEf( z?{L~%6)8G=d_lY->_<653qC0y``{zHLBgX9!dUwoIl=BE%iMFu(te1Q%RN$2axCM4 zA(Qt$v2rYtG0XYKV@71Q#1aou;kV>ftW#Kg1PeL_(Tbx*`yYr%@3dOlP=p!meJ%Rj zm)*2XsJOoI#JsIfE>*&XzKtUh?6vBRm?-7qIV@|RkXWR&V55*uD9JF@?>}`eX&#SH ztK(W09-sUQ`%ou{#(Sh?>LN4wA&OGCzaJX;d-_I!e0@e5Gx)r!u%U!ZCmxy}-)aR1 zV#tikpo_9S0+9Qf?SQr=MtB?`mwrhbY{795PyUX?V^WR)*;nJUW03)7H{NoSO^|5ji+sb*T{ru)96Z~Y1oW2Nr9F#d+LdL)TRLlUeYS&uw-rpmD&Tema78H zk5MgM-<@;3zxe`_W6g3rg4pnM0G__JhZLsF|eM=mUP4odVl0GE#leTiuwoGzL5) zsdu^-yrlZ+s;nLsBJ%6gB!g`*YI9MKga&Fv_D;8jsz@Tz3-yDr6O&UMSZY$BQjLBN zduGQ5nxFA(Z&sJ%R~VBLY@@1Gc@|sZm~D_5=*EMa6kxT&(Hlk;5m|NN)RU>w4}8J( zT$Fpt_NUp7Njx;2NtxqmU$mQhFL5zFEUGAI1VOQa(;e4J8%*MncR%)${9$D4I4ff% zF(Qw!5b|u;xi1_2vzmq^G0^17FV0H`Nv!#D`YFZ6gAvrQRq+frz$^NScX9QuTVVQh z{GzdF+6o!z0N4VuKHp3&6>TD%$?D;#vP7)RL%boiCal7jZN|1Na-%h~A^!Iv((;dO zX3D&FB+Ml0|1i2^!;s`c2Gz(c)OPKmOR*YwSm-Dq!MtRK%!H6SqEOmUB)QmdTLaXS z)c~$9nXwC-sseyz(aazMRAd>*wn~43M~qo9Z}bU z#S+f52hEs2)%HeF{S&$1tsXbuHNwx171Z;vXFBRf(f^`59X@BbZ{#aNF|8A=c(}2O zjVFB6zw@_k{JZL70w#JuRU)u#u;Qjx#HrCmG0 zFe8b3O3E7xZ>ktGPap)#3Yoai%t+X&t|;q;g?@QO$FJt*5jD2Ns&gZXCIdH!i(EQF z;FgUpz!JmAl_h~#HX$9rtsrD?Vz0U_sN54s7^9L)gBCPS5xc10ljYevjVM$!FAB!#O!sL?x8-i+!#{P z=M#JjjrdGoN<_ynor0j#7+9;`Jg@OX-zpg+%Jju9C4q<&VO-|x=s;Py zD3RpYqYMLa=(ovSt*fm?q=xU}{?Jrb@d$Q0ZMY+!;W#dkiRSg1f&UXZ1#b60&8YJO zF&ZMUYz*k3G)Tjih3J8Rlnk9zFN{by4Lm`h5z%F;L|k3D1B-b+Ebe=Hq+Dt#)> zXM2ZYFr~Fw07PaudTO^XjnsyVY}lF;*U%8;>PD@oi=#YZpp_$V+7(t-Hkx6K2VQ)m zoe8Rud^=rBrs?7&^!-FTBbbg@^eddTY6guKd1?d_v+o`06uysbojhqaLa7`$cB4!tJ3-c+h>V^x8j3`V7 zd1|r6xWWg1lQayD_i9_Bs%XQC)v?IUnu3fYrGWW0EtEL{q%$VPsm-oambvW7p?v97$eZ027*Gb~e*dK(2bzhBBe&uFUQ*OWU@0_u z#ej)D8EsZmICX40w%?b9%F|Pe2o39H3Xd1IZ-hT4C8LI>x^VgmVMt_({J&087TUdHC9ANNdH4l$M0&GIL z;49Ocn=!^k<)GA*a9#4q#WJ*^;q3`G5i|9tR2&$W{>{~qsu zjtV@7S=0Z!@BjHCD-ZU6KmT{bJTvOW}L3K z+s+QKfr(a|M{M$`4IfA{?F{{uEm@_?J;v-JP%DgOWX>i@s|H9p`Ti6fN!q@5A*p)_rM zsr*SNRkij^|LyVl?rQn_m(H|t)T8^!mpj4pkCcZ+EnoV$eKuL>OOM6<60G`mtP%DW zbsGe{jGd012S&!-#u4Py$Qc$^rClo%gW<|9n`_KfZD}Y>wz!MR1i8VHd+nVvpTj?P zeBe?IT}hLE?-Ew`F7B6TQ>{SX9FKHZyK8YvG=H~4L|-t^B&!vZAVwPbC;O$h?duL# zL{DKx+&lw06l64dbx5lO*04@%9jO`ims#%PtSWI3l%+WXQhHJEwC6tV?B*&Ol56V! zwk9L@za4;O-1_?X&y(w2cly@D2#-;*?4@OsAh87C8~EXam(q6iHWWFEEks7lvsD+RuGI#lvDlF>O7A!r|{id zNBi83j0peR>vm4fXpll`LvnWEO`jLU&0PjY?rUCl;Hsh@IfEQsTf_M;Iug~NI$lNt z|1av^GAOQXYZu-~a1HK(Ai<@P#zPX^f(8h|-JNbUcyM=1La;z^4Frb<8VP~mE=};p z9lo`X>~nVB`{Vw&x4ybnO;r~)m(8AQ%rTxZ#`Emx^}@WlkYB8C=P&OhC$c19EJDy= zG>e8~e% z)igU=X`Joir~r4@ch1Ud9`acC6>ajSc1=u9KpI>AHrt+~$@t>8U7$sNMutAn?d`t3 zv!$c|TTQzpKx_8Ne%9Pfdo$U1KgbQKfq~l6Wd54wRg#D%Y*y9NFewjcT9CCW__jN( z0XtS2ii5IWcj>KX(B&7dN3O7QC$j7EePg7~+Ab+V=qYTtsxlm2Vc0PRs2C1j3Ld-T zS}5z_!IjG^f*^t&(NJ3($L=@`es+G4uD2A65~fxeSUvJU-^K zRBj4a<#aIZ!hZXBo^LhBx(mS*U=pS^qzAMv>->`W*X$a+I4G3>jTgE>Dbb+n^jLs{<%ajj5|3im&x)$6p- za__^`Eo?c}K8Yl zxmdlw*4cKxkzf2_8}xc(-u%8J@ZKunerMKyz5ST#E?-?*#zsd(s1tSRwy)cvmbyecL-}Oqgw~^j2fkyi1wV%xn6QTcBrQXI;N9FwNbGH-QySamV zYIkq{IDeX@ND+~d+x$D)aqll1agL6L3pg>kdu%ku%8uqKhwjCh9GF-TJPF-Y?XM~x zHRJ^;uqy7FXvNl8;O%G)Pe`>`p2jW`Am~iUS8|HPEjTTo`A?{hVQm;~NQwrmL zk?2xj)UP6XYBZ|8(NhqgL_ArXS*p8*GKTU_L+uCgo|_BkRk3o<4SD!fQHIe+YLzO> zIRb-ja8u7RN>0brX7q`aGArw6(Zo)FR#*3{}t|(`i8uXDe@2 zg24U=twH|Tz!dj!=uP4yC&uTU*x%bsX=rvF$-^;!DF=-n5#i}sq-!><$a)>s8o);SpqGjhpu@sdJo-Du?r7+(l`MoQtyl=R_yF5LdZTf~U@ZT14d} zk{tT}D$zwHKhxkgt+i=lX(_mR6SrEcVAog>Hb>qg8D0zWWblfv*Y2zeWAdk$t}Z{= z$;U-M?h6)Q!(XhrN~q1n`7yaO? z?pe%kcqsdqN2AC1=Vs1nQ5XA7 zxpJkvVnT-Z8)?$-Ij2V%?#7m?ugvoiYmF6M_PUgQSJ5|zOMz+DmHNt^*HuDo5 zoNk?XJ^C$qFQC`B|*V_Pt&aqT(9w9dC8TXdY*%1(kjX7|9Wre+5_jNFQA$ zBih~pJWVH6G6i;{aknSUYB@j-)^jMOb~-htNoz8&G*a<%FQXUl@n%Hn#Fx#hQ#$lr zj@oW(u!v|UL^-@OktJ)=!A!>;pWMpN9mXO~I|K{L?Z{9>^1u!}GT-HuZIITUL`UHi0 zmPhG>H#F$)&!FVwp?Qc38IHrRnWttee1-e<4%;PcAjx6eALGTM z(re#qJt^Mcn?5N9FL?*!-fsSCcEVqN>9w}oW`*n8uIlCY)>>ViFLPe04&yJk#?*2e zu=nNfJw6h4Pe)rGX@<)@(p7#hAd_h2G!ytZKJYXrwt&brRY)7eZ$a6>A;U1klzL_So4YeHe*KyPeg7wM@)}6$^LG z-J0U{c|IqNrYfcvvxDnfnYx)tqe&_c*Lr&y$~)Kal0T;$s*EOlNy*5iWD@FT*)WV# z;uy9VVBD^5qR#Mo`!$Wg%*d#R;@p%Hqq9%gr-cGM5tve&2j21a(4wdw;2lov_pv>Q zCBgRL2U-jF>dv3JExZ0G#Oa&Bnz_{A`HUFX#(@@5yeH@D;x2cNTs@v$b}T5kLxmwW z6~6MTba(J^JItBsCwe&L<>tb4f*z)MV~dgy_hZuc^z>C0drYyQ6PeUO!98d`7&GZWKSSFu%bL9Q}-Nl<+WE13!M=E1tw`Ws}LX9{^qP zDK!Mpb)}-SEa@pTGmv6B)o3;1(XYK$E%)L`!Fk}8$#m&QHSW^CNjvm5)|Qg8A=WX| zkViDNBL=9wcojX03)!)8w6XB>iK)J!#xuM*)U^wiX%&Wa%Sh;E`r(xPwd7>-{3HEE*W5)1o0FxpGdDufXb7wJF5EwK|5}*MO7lfL-Q({Qy@+tW^~)eXvCfoH zSnL>OBGD+WF}(fuJV1Ct&Ss`Hv8Mn=sh}dz>=BOdeT%UZaqk?LpYiGPf{7d7nUxxe z{A%&G(-M!z9NCH*h6A}}w6otqBK_-Q>t4%n!PoQ6k(!s958xz^$SWW3u0ZQO1sk(_ z@T1GZHn4xx_&hFSYDML(ALFC%E?hq(#zN;ufemZ*v6{?ev}U~KTA2;_Snb0U&{i4O zC;SfMeGcjzO@|;}o;A_QEcoZc4b;kx2UDGOA=BZVgy^l)713H>kfS^6J<;p)%VqxaMvTs4#TF;ZvC`Yuc_TA(lW8GZ5)IpG7eLIeM#_@0(wK zkfXj{(euK7mT=P}-W4&Pp8}8qBfRltXn-F*jna+Y(Q$!cM#kghfh1mX_S7;`xK8;) z>7=zLJrrDc_c;~Bfn?mHgtQYaBm6%A22#uN9|4hF!%|PtJeuDQ^3ha7MK4^SIBT;% zD5R$)AaYVnI&oTevTpKlw;F9o(j~z@;4ZLufiU$Y_hMByxAeXJ{I^L&#A*I23zNOq z(IldMJzM>3+}|#*DAxId)OMJJG!Yi;my_Nwpv}W_c8-2&s^g88U9(hpzs{aU@bj=J z-m%x+KX1tgL3FL8DCXt-Xq_LjcC=EBSN=35Ew8?Z+vtyn_-r?g1uW(9JwnF@`lZhJy-R+1(!Vi)VmQdB+8w zHs=FF!@SKT?zHMwQ$G}iV>diLl^nOcnjNB!)g*_+C|gtC)*Bo0!%3YNGzPy$D4vl0 zIAIX&TLw5Y^fIc3$t;$SCx{R6`1!MoTX>=@kNkO>Pr#d_cpA0e5;5TK4azmSQBd;M zG%IvV#mMifI^%ltUd-z4z2sSc>XJ+ zU@;Oab!X0oPMVDP{RXik+;Nqdp@ad10voI7;Y5Lio5@?v-W3?}{`R8`V-p9n&|Z}^ z2cIB!SpNEoGrdWX;fCGHS2CO#(o*dxZ4OzERHjs=L?ughQOhFvaLOF^TB}W-h6-n^ zt*$=`K+2GBv4Sr+W8z1EaFZol-^9ZqrCD&|%QmyFDj%3%hAEdNTT#H{$QGd5ZHd`@ z!{er@g{Q*OZ&AxNoU_)Yk_K_d#3wg-tHqPRo8}4$iqBF25qk%RdCWmnZVz_|F)qlJ_`fAp}Rwl;5Qng6P9Q^B%k-AzC3VKQEh zgnVgH?0y!XXYR0GDHSV>O<~jR6GbsqZq9p3yN|v0$ThxJDJJe%%1FlWO$s2r&7~T{ zm)XDk^)0OLjM4&8q-b4hkyZqju1DvoG)c7RW1%i9kfQF|6zbo!8X*1L+QDbYafupc z76$+6CHUMeYhZv~pqX~7S5c0&xb%gA&C!1_##Uhm0Mye>Fr78L4q957%2hwy7`?@o zr+;TcEnMi36*DvBeB0@aMBr=i@ZWkI%nr>P2jUz!GFY2uY(giIclStTLx7Ot*!TWU zgBq2dyvA0zeS>6vdc4CynzzjVd~QbQh`vZXFy51nKS1x3DGkFN^7^8n{gx)+d|l)n z^fQ;0iqrLXw&{n}L13D5#}5YwQr({CH`w^(kSPaq_;kKW>!p>h&;_Ib*YmW-XA8Z+ zI!-{GJXDhaMpEqUF5X~yw1$)u+uBofv+{WRytV+6^H6q5d1sb;;x+lJXeGwp%K}VF z{!2#FNyG8ZoA0D;lcy&W7}ZRY9xBrC%OJ)qK(BB!OUU9XLe$L$BVWI89rDX?O4L z{oYPqzr_^a6mF5n5D_sO?&OmN$`SW^xAmT3>Wk(VWUEQc^wpg@Mr|~(a=IWb9*IdB^0nD} z0}Ef&>G2~kJxrDWX<7gUm=hOGDnUVP43CMI2KHlBAt=QO>+53yLk;*2`{hAyX{8@f zuCC+7a7LY z3E?(v$80owl&X@%t@8(oI1wv-^m0931;&O)@QgOW)P^MlYwEI}mmF)dClbS-QhT5< ziOX{E1x+?qCQ!@3qM3H{ks<6>sPQ*ylj1nIeR9PaYe8x@w=XPVQ6}4nM13n`XIUI5^rt) z=*6pzW^Ww>9NYiwtDN07Ha0HA=qrQ3(uOxzz;XYX7IABip!azv>3jAFt7}It=%15c zdymLPzoif5gtH3>L6T~ZWf}mU?{!dV5u2#nx?tVj#ua}3f6gtf^{2fBJ^_=N^j(KO ze&*I-M1vUIzIuwD^&vl`B+G?D*xW|{x9=^I3G z$60uZv(twP)^{*L*Uv090}qOdHV0D6S`C2Ji)bL&{usc8_PJ1EU`s=869nd|q^fL> z%UdKDFz;63v2ytfkNMY^#SRDE6vZZc+(zTZ=)63hEIT|a*7Oov+b%^lkRT+jnBkrP zO-s};K+yP8boPF0uDy>qUsSY36li4~;EIyhLxhkS*OK?){~xqCPpVLYUt7 zS3usc?2kj7hqarz(xcpxWO-Kz3Q@O!G_#)_G!E-ojjbVbi+4t!FGrEzKu{ha8DeB& zLiu)Q6*b)SEg(o=u><5k(kU@8MnQOJhUw9UQ@tD4cxc^kPrBIKkX!sXxW_5bJX1;_ zuP6JDvB1$1uwKvR9zK)iojI+46|yaM6RyREU})KX9y_pcmj1`RuJbT@jrA7vXa z-xXshZXS{1Kx?+3BY7DgZ~OcmVlu(isyC~!K>F*6HA8^nK)Ze4{{~UHSIh^^3mxlm zJh**w-geI8)Z!_9?e!(B=Lu+l>#QvvdCkz0o%_0CC9ZL4AojJl%e7;+eio&PT{RwL z26EhT`7?e&ZL1ZY+kQgNRNHNsq4sclzAN|O^3i$gHP4@IAfDL;NRX7!IR zdMK=%9V8`|Q?b`u*w&)%+vsf9BemM7#ydiO$VG3hY#I|c(>M^Dy36*7sM}b5>m`Ej zMJqY4<@fQ0V0}ZA^^+$A(w}^SLN?P&9GzGj!JQVf%I70ITs#6>U%iA<4;ifxDrarS z{=1Q*N7Ga7L-#k=l`fDTM4xwSa;zgi9$v7FbY`=b-v+~7Vno|1<(d`e0{$KZpD*~d z&dW=SN~&+hEtO(=(dumJR&5f(20bY=UR6Gq?9X(NbZ$HLnng3u(XZOFD|w7S33QND z)d*2Irv6AT-F8LkSjMDshQA8t`O5? z$$WfiupX!|c?x-WUx0ltP=9`g<4QQrO^?|nXXG*-_#m@b9^xKl+ZMH`s5?_yI|w}$(nBGyvAt@D zjFVSr^Pqv(6#6?~pLc0BTrA|+FQ})hI$>+cP^0n}rP>%OTr?%fhj}(wbL^{P`f-vA zqp$b%RnNDH+=CqJXp|*<5VVPt_%Ch(wuKF0d-hF$2G_0|w(&&BUa%Io2Kz>HNfw;X zXIoHYu{yQSg`u$TOQk7|>vU;^E6R>X9X?(2aZ%ymYs>gb2cHk^w z3JA&nZQFqq{i%cQQ~NjXN=cShjsx%U<^m7Xo9*2}%bF@`?yK4i8#>;^$4J`r0j>C> zDPzjzK0^P5dQZovNfev{5~752Q$HF?!%1MtV8gz5Rn6jR9IiQlaB|@g$_T=HY^FN- zsTM`=mXun&u4^{k=Tt6r|3kvdt4ti}p+GpjQ4hC(uq(h~$a+>03#x057z#V6s>%qb zgn!mi8X%^>TeXImJsKs%Z6E;Y&ci1Wdlxjc;d#*T`B}L$$N3i#Gd=TR-Jdkc&`m9Y zs*Rc%F3t}C?&QSlc_p znz?vJ`B6O%T2m#tS)a$#Sum^m^?27S@xtupElBX$U?6ANtgidXJbR5! zGQ^j+#ruTs`b65*kAHYwFVDVmE__LE_4_mt?%@MJn#Srf&DD!eMNLa|)3D|WAhBH@ z+Sey1bvdA}$wf}A8)HgYZrn}}bN$#QGP`Py)ul)RZwSXTIz ziLG6zMdybR=ZE%l9eb+ZGQ7e<2-zU|$`p_bhWkju`PzI!|4*@Z<7g_@s#z_QV=WCK zHyt4o7ZVR^c$RfDccjDI7v^j|Fq;oNlA<}rVNkbaF{!G}h4~$0_~oU*0*&_=RmS=b z+mSYuY6FT94x4WG;#nsd zQjI5>ZlIpE`T-6!MHEj8V3 zV90p6C(=D7c-W(S`(r^fiZe8*d+CPB_-ay7fF*sidA$r8^OA1=d$M0~VA5x_CYGWr zLXZqvE}dhY(3F-c9x}EvAhy7qDke+!YY4CjZdnTtpEp||I40^|DC9`?bS7)1pu~&F zx!Y%Cx?59b=bB$QFyK0I14VXqf}sPk3~QXmm#h>B_orboD}_ANW{8tyT1tZ+eV0%^g{UI*XKf$bR}5&Q4w=Ir>>@w~*QTKvPl(jPQc9%Xi92j+SM?s%+D zsPVZ8m>AlBR_3&J0Pet_!I$rPqWpBN^}p+!{&l|cKeEY|(~fciw;AOKXDC~#c~Uha z-G2V=IS^Iw2y^z7rm?TX|8^h1J$3B>dbYp+^Z#&R)BmEh=Kp%l72q0&!c&x3+?r|s zd**|YS1ZSt06kzi>Ay#Cq~dg%5dFB{DkY;3Mb*&w~eEmAwl+qA8ZjDb$O3m7u?UhogwwbGIB7oBY4Mu>l zBFmq7slR<*{pjjK1)AHgu6*#imz97kW)uOccFRiG=%_$b?Vr>N1_mFCio!WNf1k`I zO~a|emrnpLZ+<*96yR>Zd;$7=-3zD7{}?Cv#7f?^YWz`2xLKiM^|#79WNo>@m(}A- zq5we6g%4=xO*N}U6>i^obrEL&n(tz4FpODw2FGNTP(CB@%K75B5+86Ch)(NG9lDog z4I|tbyUjQv){e%_DMMHB(Ld`aZE=xBAVE|a()LtNoukY)sC$)lOFf`+PdUIuoQN9_sDS60w zIEDjTcP1PC`igZpyBNmZH)*Qh#N>9t! zU`5k%=0&0`6>WNY zLSDA#*)JYe9%kpSw%uVY-gkPB)j?v4<5kQ>oYuqV_B+{c*zb`O62HYJtt+BGIpXY1 z^jX7$kjcOV(dVaAlvRPf;;adB8HeH6HFx3Jwn!f+OZcAkPlW$Gm8yH9?PbZ%Zeb%2 zEcu+l0@Y?y9%L9XgA2GVoJ$rT@NUOF{f-t{yABOH-XGo9Vluq+A`Aak8R(nue>If3 zC@A|Q#%e+iv`rTJhY*qM+s+S^4y)5@%>yk^#l;qVFC!7 zcHwt9A%(?FfcU(!+S;gNq;S!GgHTpL++*%Ju)GBs=KUhKfm6fA_o(@}E7GH@0`KGt z05%$)8thcvYNnPL(u3Zf{5&~zt(hMUz0rPV|B%vl zO)oIYd-^%8EsfdFT?emNr!p$(Uy~#ceeIu3`R{mn?N`=IDIP z8-ELWye14EFqct%AVtv?nAdCh{i1K}W>|_a-Oj8o;(U!6R_g0ZqQ1A^$G4DT?8|#z znc1CFYYo;|>K)C*AWvAQrQk_6UIs5C187}nrEn@Yx+3Sm!uKPct`4*j_Z=O3_=!{k zR8;gVc?zz1A-Sk%XtXpm5&5-rw2Qsf)y+>*YXEWc2?*~T!~JVD(oxd^q2o+l&EBhP zyUhAcs(_=efUe*6P`@QcUY`AzrqG%2_%Q%%F>T#-q0z==Clz>4sdUOchi`>NAPawASlp&pkTBxy4?@IKRVOp91{j^jmVw`DU$t z!ml#1${IeY4M- zWPC2GfAC6p{ZZ~Q8E|?Ia7cB2S_a4oe=nAol8A

>aOowLsOqg7Zc|yVL&RPbknc z>W}1bB%P$nC#D}ocRTI_Y`i!y?|KO!6?VH(h`|85R^r zTVFQ^EMNr7XC10mSRR(6&L`>yXhbR0#me|H+E0Edc+ai0azg#gd~YH?c3e6jD%tT^ zzwORycbQS{P3Dk6-MhMeeS%zSW_QtLcB{Qyntq9yh{~!g%R+MCtx<2Z#xY*3LMf|x z`|Wgr*&nLC@s&zyPVpZ7@0+OKWRKX{@u>PYnRA60_lN$)k*p3QwthP-cUDMdspx!c zKKtR^`8Ehg+5j1`{SsXI=0x?130q3S*VBCzlnxMS-ah; zYE?IPgh_KNOwBww7*G@72Qrnm?un{R$4_`2csFT8hD`>7Ir5fnF)GQds`s-_RpR2u zecfmxB&ST-G~Yh=GehM}Tt$TeM-o3ytzo2MqXsF?=4^tQ*GN>r?6jCkhuCI^%njxw z`R~lwlE69EbBbmpB+QNxV`ts17tb&n%U}0>eQiZT_shDjEMmwb+jyOCA|QCGk<@i3 z7Pr|dO4i|1?5q5Rp&6HyriGET&3wlC0Q{+ncCxBS54ue@^){I`tO?dt6%V>qW>C{% zk!(|rYuvXh%1YXOvmvVWNb7OvuSt7-@yB5+lz2G{BU-%QhD}ia=4um3*@sg%9uF1; z7C<`?(%bgnE>3)T@Tjbo4S@)EMzAHdAvPMm^U;r@;w{>SOA{W08VVQqjiyR^CtGe} zdiv(08*Aus(M$4}v7bLxnGq&F(M^XVn9Z!m+;=cSuLKoQtgZ;4C*%MDCZGVvb+?ED ze~tj@Z*Me(yC~p$0xg?BE&1PEn^!q&;6D}o{I5Ku$`>V<*5cc4wX!g@S|3xH`!PkL zWjOIl#fO<9LF7W`{?X3TeslrwYC9-KeK2b#-ugC`dQoRdy|| zDEj-wymTkEbt`fQAFO=4y;+`-R58)g6HU){YEG7rX29ia$w?q|ycxt>QrOCyiFItmIXnfzZ zzdfE(ZFCn3ik#P{gU&1O)o~^=KpFt zA7uJIx=Ot>{i=vtxDq4(u7|{sLtTy@8ju~dkHUKK#~mK4Wic_$p5>#QgfYMJDlGBUy= z$L1jQjtv|sD^k9OIJMW)6px;9N7Q=EFmd37#CQmT`0`fQk}})hR{HG;vwj`@u}Fgm zL3fAidBeCvujXuN;#^8u-`xLUY%HF7_C|hd6K%+#n?b^lLVt{e)}G*4Cd%3G-BPn9 zxj)<4xM(Xe&mK zoM`JFm6776 zj+g70Kmbf^F<2+@j=;s_+RBU0c7uWnQmdrrw_uPq%+=#Kjn_zvs2B16E|xMneD0T6 zljCv3a6=vTnqgyL!IUGt-0U_f6I}x1aLmigDInD!e}UPRI72W(+{ySkv?05CKbbbB z7)668Hd4f#qjAxlMh7aR(rxt~D$Og=k~&fi-lSaQvU?R6nQCcyk%O(a2aTLs`fScj z&|7ccfi3tD=gIc(WQX1jlSZQT#_I6l(YRp|J85XC_ zRyb!jx;ShH`1IOFX;d{Npq4|Uqn(aOdOQy6G!g{@4R0hEn9%%M5)TLe;P$(0dC`fT z0*f(>wi;J#lHmx*aqJ$gqiT)CTBZBWM>B~gMfgae-3Th`0^4-8x%9{wE#sM{O(5hmiokJ4s#=!jJ#6|0fbXSLomS$(R8*5 z9b>rrzcR0Me%pn6DwbxJUs;7m7V;&<3ZwNBV=_e7u~wjyjw`<0=5m^DBYa)FUp@6 zo#JO3Ma!^Ahq%(AC*5G8)I68T(GGRtJ>dQFuG!scb$fAyME%=H34W7W0@uYGJL}Ox zqzy^9`M9qa?`UgNSSLDIM|mdU=TO{~5yD13xWNS_)thYf3O?VOK1|*;DAAAt@d4b zf1{y!{9R*P?*PJxHR-!cOa@;>&3X6zE(erid z!T2$W!}Wf0>li;uM$XnDQrvs=z1jAhAWHR$uvZ&WaBd{d>?)2}{r-*aa8_Sj1?kp4 zI98H7M@kpg|3S;cQ=4Qb+U#%U23Wb57OtX{CK)DwIK|5to6{7F<+D=lbxgUuYoxgz zC~b>t5veeBE22>_VRN8eRE~MU6d8hov5Z{&frbu}IH42oE*XZFqQuolGom%-l5n(Sug<}YBZeCb z8~>&=7bNPk?6?@+Jqjiu*c2rI`+?)umZx(zkpwU`R03&h${x(oqj~j!j{Pfiz$+JS z-ncY0F){lTK7Bl7-dfe3M|rUNNyB}w5`}<(LUs?pEwa-T)r#^H&v^QpIy$Sh4408k z1O&^=%YVe7#dXWZh2$zNGh;&b6Q#|qw^YcvGhs0UsSY9R44ZOMY(;nn>8Wrl!qQDrd)TQI}45iL2#kyT#Tz$*S9g*|n4-Avb;1q3&q+dI|BV6W}!TON!IOI|CeaGXa zYsQwZT&(INOuyv+zF)4m53CAcp94MP=Q2g>Umo!;7h0UT?c<~*4zhn>h7UjR^2m66 zeg1udrfu@ z6SnMxrBzy(A~6Gl>3x|x^5#s82|ME48Z>~k%9cSw`RrJ$!t4R zQB>%ls8TNV;{6_WRI+GxkQ}8dMuvuDG_CpDJ>G8}d0&lGjHSk{*T;RXm~;KVdDPXd zTIjsgl8m<9D1^YWp&OU!qKo`Wf*W1gZw7~Ebk@b_TCc98Fa|#PsNsV+iZwIHH@|a^ zBDbpvSo^|l?Y175neiOke||EcGB!`wo4vc))0T|qrXet|46hm(nl%_yj>cm3h1_Nq z&rFdsQqg@}Q4tpfk-rqTXC7~!`;`0IE9(pq9EA$rI%N>p@oGcDj=0_F$E0{!<7 z5~|(?B&#Io*@&=Qnkk`BC_#Fv%-KX(xuMZ>jUt9dc#mS?ySOtKyV303&kW1SHd~eV z>^tA&p}W<^r3^Ae6{s**hzQKJsl%0aaTxe>JVp$28MS7>86QR_C>8ei_y64imAV8v z-V+r4D!baj(>>^lzd@Zqyc=5Eq?ton{fmBQ5Fm9ADAgPB82v1y(g&WWigJzM|08+#_hWy!f@$Is5yIF8JfUSDMY0}s zuQ6>Pn?Y$RuEMmB$Jc<%6W;-J;p{4CAOAIbQfz=T>0&Xo8{U|Cl|b=wFd=7py=4=w`NN^ef+%GpBqvYRSh2 zkB12o@hCFel6bWIV=b+?_x*3q-Tw>2Le-hFD+)%D#s~T%M`WBAi;JdGQUsCHeCEUz zQIQ|$&)UvN=`=#i9+Mp)eB0}DHsI;U^#^T3dKMWW*G8LrXV?K_Pv+XsQTP17YFr}_ ziyZV<2I9N{ftV|LIo18(e}%0)xqq#Vdg=Hd9(PV`>q46EalY9+TubSvvw%5p_O-+5F zT$7IxlP?I8LEIye@m{V+3TP{U5WE_G^{w+AP50D2l|9@XukI{a*b--LA@(GWwIr)C z$Im{6qss^zz9oP_H4cG-(W;A%&E1k6n(5ApR*tLbdkn4*AT!REa4*LpBPhzlt(0^ zX$<1vM4pB!a>BcDrz2erb_l-aI)F zAXmi5_?Lv-;Z~t>+l9-AybQ`N=Tnmj5+1>7&KP8=<4+W#$9-C{#WS7{OgwDm`KKic zi2SeLv52i>-S($M%c<+#QhFEHw`=bwF@#l)Ax&~_OSM<;d-}jOnbYFz8Feu=h5|ej z{^}k=Re5tLEnq0#H?v}>^o?DNW)_N64@^!M`PO4%%x#P#=5RMffem9D`*C18e1qcE ze#lhp6JAvu5bInKSG868%7%V`sgWx;t0LDSU{}=u)#-@h%Uat{TZv2G_@v{G`guVp zwTwttYNDt)UFUEma?knvWh(^D8`N2G>zt4w5RIt3RX;OIe=fJ^(027*?%^0p-2M#; zcXgOVl|XLi6@k*=5fU{q2qbZSF8#i@HX-&wFF4m4(gF=8Yf(d<7h^V?cH;QKmJlZA z=t0|3Ij<{vJv}h98>aYwjcn%e`bYK`1onM+$ots3rUZFyLrJwhiz!^yXqm#xdv@66 zO```j#*WWXUE@t-g{;1=&k%Bvkd}<=+YC)ew^C;f%E&5MU#T z4x!@Ms4zAjp8RsnZrqeN)EJoGfPzfpSwE-OlbeGF4jj*AJsEIZ?(bx(@IMm!xF(E` z|HZ-*Z_I;3>9ViV#W*pVuCi6>pgJAew!)0U)|-7x5RGtpRN%+bDCL0(#d?+C`@<~Y zz(+(pKJ;DQ9gnlc{SriU^xLx6-3{-nYJ0d0Yu`sa56Ql0^q zu4m`Wr6GVu)xXizanFGMY*QoenCsa2=D9Gs$l(`Q<9CT1)w=qyIx5^tHw^++)#17a zD_43kg5Hcf2e;hR`Ae!ZNjrMux0KFbkDf^X7@&HE**RzZ-PHwj%;D5&2`$m{Boz6= zy}Z=GP~2qpfUqSQ-Jq@em>WAvTyC0nWVLUwS;fGGlSa7WU)Xl&xz)eX+b%YBv-0e{ z(c1@VKMz2S4WZ$;@>ng;J`v`9SaFbiyzq_r*g6@Z{A0`|ZGYb=3dcIxoEp6qMF6*D zM$1^;zdU*}NbA$6-tbgfS~3lm2DmU-P<-g6phDWxdd~M0n#&)7`7&z*-|F!3G*a9s zsp1k4=!&>ahw8boKFqRAf#P>BODK=O6rAW9!$rt>3h(qV8}Q|f=&9%xlnE2|PnXX( zUDd*;LN;pp^K{|<#`;}JRF@Y5IJTkzDdqb1m$$Ox(9=KA-5s)gW^J;nC@F(ikG9HN zcI47@EmsCO%zQLLze+F|g>shfhw;|(-m2?Mi9~-ehn2)rtsfs8`9YeWRPIXAXNZQ4 zXA|&k1W`+v^S5T&>b+(AlHhrM&sr~8JgvPnJP2j)A}LT|cK9J6c`+WX*MNY}8;L~^ zl@ldDcf2{Sc+`G6*WR1)|bW zEkeC-W3MB3t8-PNTY;aPuJe`NFa`bBMEk)vkYhG2((R5xwUvL?X|3G;g_|P>)(8*+ z1vheD{+ro;$Le>X3t`5Vy(dcah77BR?bqUGo>tW!P;O;E@e}-ww=3Rqje3|zJhxVq8DN!MPf9)D8F z-MOPyP+pSZ&>nq~WDwOG5*Y4M@O^u!pn+2Fjbx&8p>wuAKhsqMpmEf^4QRnZ0A0-njs+BIp8?^ zX7ucVLq@8ij474uR;Knu+Op7XidxnSFCXe;3NEXkbjp;Yv_%E&@mhfI-vwQ}Er#sR zEv+O5iVa8grdn7Qj@jmaf=k98aZ%i{<7Pf#L$KiP?(P!Y-8b&s-JS05bf0to+#hee z_r|zC));%QRclpEtywi^%{lJ}kNLodF{g&&cn?%!2^(SK>}-M0-!WnOGKznZ#As}H zib2y}6L(?B4dw5VptikEnsKqHbV1>95W|U5*p%k~>m~CKILuwsi49GdP1OW2e_Q2e z;{(IPPGF1u&s1jm-wPb&ukr@+)+_2P^36zHyx zVdSbFNbPL=MH#fHq1BH6%#!Y;@c1P|HX^d^Nt?J($ z|6$bz($e5YIUdh5#|@I#+5WZf`&6Kzg9@b6av^fFPnZAdM2dg^i_+a_iA3Vw;M9QP z-97gmef%v3QC?+fa>9uNb&F<@9K0r?b1f_G9;7Lz0tMp=DbI}Jcn2(8NEYkp8*$}u zxcMz-=W7;1+5fCu&ZQ@(rn-n;K}6u%kd5HNAk0Z6f5liem|&_0LZq~$4Thl80S8Cx zOY_)F{D$S3uzM0=S#jk-(QN@wJGc78xU!Da7nsXE_B!!{`DeZ%y1HK9J@5#(5iXq$ zR@=iQeVc-!Lr33bKE3nWar2G(F<|*}0~6zv@ABsFrINtArluzMhY^OU6KoEEEP;?JfrC2!aFxVd&`&mc9)ipGFfC&p@MkW4;T>3v+xgZ7t>}>n|5e?2} zX;%i!CbFif6wzA_V?8}krupxr%k{DxDZ}~dnMg;UcPq~rn1HmiYkO2##STTr*2q&&t4!J3a5^A~()sBe; zik6_3Zg@;4-&s$!QTlFsU_&H1fn@zz>yz-rx&$2Iz0=)K`q}d7_xi*?Kk8_vXpbIPz_` z-1#-UG>`|~(-RK*tbsc6K|w*wKHd1?Pj&*s3h7*((FD8$BO@jX@<^*6I4gR0;T@$Y zjC!^>u#xkZV#aMRgpa|mni}RRH7wg9j#Wxi*h(#|UmJi7xpUC~Xc~9Rf0@qU^*=3X zR~Wyd*g!5%u;*l3^GF|^u#cqgXdWOBzn6+U}I14F~kabU*x>g1Dd&Fp1F+GZx|Z7xnjf?5SabP#g5TtZz@*>WHqrw=rT)tEv~6XRa=ayM zN4B7WYGAPLq{Y<;FKqt=I=!N^D9W+FFnLQBU}9#<#9Tmg^;9SG4BvS8K%gugN|%7r(pnZxu0E_$2FOaEF}VR- zCPYDEAGfhVesDmML-u*Z0J`jk;$dj${Khq2_uH^Lmx7i8ZU8@I!$Y5e!n*^qfJklu z$7((?=&l}lNcEvUIFq;ymx{`B&y{%)9FGD)HfyVYuks}m(r@u+K!_Ob0mtmMh|5xq zUmBjD`S+THIBHhBL|{aL&hUTmyR_84QhB4M$r_Pc^3)`@WnANP^Y(D}r@kF|EWZSW zunZu+V^(H{=1Jw6ofwTxBIKcX+?V&mV_FubNnVvPEKJKt%QD*=^s!l$^h!y(0SxWV zvG7+!iRSne39zSg&NXN4W#Z?zwS~oUx$}g;CGkmp_QT6zhnq*7J2RLr z=EaFzbmp>@YqX|jtwOqS)U@n>L{6Q()^^b%f|!8>%tTfS?A{?(NhBjQ?`#6K6_aj+&{@(1%cEs zjW-;k1H$@G`gT};Yp;fMItFxKSAp8ucuQ~!&X-uKvjJz;Rz4v&CrBi+7P1uGfu}>< z!^5Nfh)5@CBT+U&!LOYB$-v99>=qleUS%^hN53V!Tu_H$6Xt4f>Z5N!-UgGmt@(W0 zwabHqEd@lT0U_(|u0byzjtx=oZb0t7)!}z1FKQ?2X)Z6mhC8NC_9LA#Hyie3)VISkjl`-3{paP;?5Z^xx(r1o64LOF( z4g>X4#fv^h{rD95Ny|SeAcLH%->_zfO?OMy&soN)3ItQFS#gRmtJI>g6vn=z-EvOaN~-OvJLnnTdydMsWEu=J~3#JtFQ$}YBI~wC4>Ds z^L3EHewIy4NCF-@gjJ5LR5UXErZ`u19L%Y5-JvlkoKAydqhxbVUk%lv30cKydObi? zIm{8x?L@=UMq4Bw6xZF~Zf^%#SRUoJGX*lLh*9y&Xh|>xlty9e3Ho%rp#$e*)xu2* z_)`eFOicke4Xp4$B{i`DYZT;|iMdbX*d&hT30w0Vl~RJGF5ZE2^L?h0ug}Yj9?)@@vw?rc&W5B*NLn<^flhg)z z$Y~}fxz*kdrn^RqS2)7wVxq~E&E%Px3xqCy%KPNAr;qV8dll}6?8>r|s;%sQ#{%p* zuGx>7{)&0c+>3Vk%b4{FZajw(xqAWziugzcf$qGp!(hpz86nz<>ic^x-Z3#r^%yq< z_`FRDs{fwSl8w#v*@iXiTYm&+LNs~|x;**0sac%eyD%8uMd;OJtmvB6c_J-M+f=X|H8aUhX5kFQx1FEIVUxp^yhr7z03qSK&* zC8FC(trV7CN~9=Gp-O88t?2b=H?DOviCLj5Z`BL+WvV&@m)E%E1DY^LA zB&bUaMt6@I+@*odZTFBxzr3Qyb=X3@1@9?ZY1_v>vU=#=W{2EwkcsMeZn=z-2?vsjuFT5fnY>?e}n1=QaM|)bJ3T*z%zG9;BomT@|!3aw<590 zIlQ7SRisjiA(>KDT~b~}x{5-y>8;XW=+L1|6@S4%5Kbzt6`UWFt~9%JAc_bq359YG zc}nBG?9&T7f^-nL)cF@o#^)4geJKdpPHYBOP1#mRlxB_5@u>wmDj!f_np!G zg%+zL?GKmMmbO3%7^bA7oM)i#riS?kC%dMEVzuL))e~I|55ewZpL?obt5uP*YBwlu zCmSkufu5^uK}gAT=2z?SPg@!9r(+3n%*L$Ro@y_Dt|t%7uzb1aNeyv3FZFH666N>k zqTT~(`?h?g?)bFEz;>C&6!8hUdirU`3q*mUctu4Khf1AXl}W$syTRn@=&#$v374 zh<#KxXJ@swLMXjX#E&TgGS^3oRSJpoj1Ejyf%GCj!Y!){SZZs7lA7Sh1f;c6&B0^)$hyn0 zt9$rss)tvF!Hni?Vyf#XseFfiwk1CaFnfu4t^k|^(uu*61fm!@ip%*g25U`y_6L0a zmY-h-1c$fF(FngGpRz29r%pY}{}xW#_pu968Q03s*h?eN2&lyU-J?^B0}-6iO3r*R zRaNEFn{X4e(#`6$R3QHeck~s;&#czqAsxY$f+i}tEvyUYWB16?;$+8-9rcRxO9edv zZg5RRA5+Q#|4}*~#@&N2&Dev9#%mVcYCOKwvlC}vm|(!E7!d6;5YEo`!5R`+$ql-+CZIE@xQ+$%9tGZkEVzl#c% z2r!+!zA^!`+en0eIeDn*>qxFmvejOXvT?G1iiPSzM53n`TOfpAK(0GoQdD;nvsaPc zdAgJW1aAcJfu2yIrwa~7f<|nA<``Phc6LZ3Ut?;bLJ0}!MC0Hcm{S!6G0EE+v(iUJ zsr-q5kx)g69+OayG_GjB6u0oLl$cx%gP&(?-V3B#5-~PAnUB`@o2q-&2jTY^!HU(7 zlSC};ojbXHW0GzCF-rPomlxL~$RaX1{Cu8!x9*<$MAx+eS64!1uz(k~n3YS|OCNjm zVJx~ITQkFmy)7R(=Vk3_hOx_W_(!H039{REFdn1bSaACzKLqa|_=y0%Nw(ecD{<=g z4n;8J@ulvp5077}?7hWQtwBkK8MwKg$RfzYzigV?hxO*?H>qwt_SzJ{r7bhud63mm z!Oyo|tF}2fGM5+irw$`=wY`TpIKR|s^CEMS?Z#iKj6qjUMoS(}@Vvcjwf9N0Zg=3P z<#+#f>&ARt#xGN<+7xV}7@b?KZnV z1u+oij8g8?%DhB(S5!%PNP!-|yfsVzFmLA8I;~5IPIGvXO47=>fN0~7u&7}gu9=3| z^0kIUGtAcSP>T*z zH7Aq?x7=O{8ZD52`Q!y}g*YCs+|a?$;AvDhrXpamlm0mt0asxsC#R8e%duVU23K)b zi1IL5{Dm7R7zy+0`z{X^C?Su)07d>W(BcQ3Dl2G+EhC=#mGRbb0?`(2Q~t}{5sJgL ze%cI=zK`1Fn`DbAJpWRJy{Rc)b12gZC({=kTN@B*OH?(Hx8Si{u?AZNB5d4DKIr0g}M=IzOh3PR-8` zn1rQ3os2K{pDZauMbgHPM}0!p-?{TEkcJY{F>-cJsJ!Enbj^puQ8#1D5^brk@Jmb+ z93C3#eVU$r#D=#5etD7j(UybHJr(oEr1KXAMM7ibd&om&-vx&tw8%FKm1(S>3`HWw zkoYv(z{EUXQO@{mXqB}BxJ}=}KbJe+65KjgFiQN_>#!s}{S+eU8GZZqYeTP3#ijpz z6bLQ`{ICBgMk1mA>>NDz1^z$Z;J1*&k_g}!@1%%uOL+Yl0z|s&X5*6yR4H-)3C!O2 zzlPD%Z?8QL>-%(t>i-5bNYl@x7R~+Z`DbeuOboKn*E@?%YzbNBLb0!ZKru%0m#zwp zQ6z;;QOywXI>enq-TUE8YNF->53-gGf%pT{Tp|A%fq*h?+*8(Y0jYq?ftZjkO=lzd zn>kKZmUfD$yX$uoDbas?w}+ebF9Lx;h0?)v<>f-5ijvx<;dL^9Qmsc4{WsJ9hlX$@ zf4%S*SyNE{f7t*v{`(If+*0K!FO_>g^z{94vcDnxlRD}Aq!gWTcs?;}0*t+d)1nFP zjM7$VsfFdEC`tYMr#yWQKQb@cc<&V5T1Q}Hr+LgVl?SFgaw7}{r0AWn>RA?-#LaI} zh^Sx-t{6{+e=yxwLRQfZ;()8fV zisS%DuL#&)6}PyEXh=O?>AhrGt}#}{wVPbL)rDtojP&LHbZvPMLSW#n-OEZE_jS+h z{b^G2osh8qfwlc+jGdO4^Mm+HR(nfEP7tYmkyU1|OZ_t{UVgI2_Up6T@}&&K|L7V8 zOID?R%vki9&S+Iuh$S(fjYlRd)T5Y#l`J;fjW(;PvXc47;(bYK#`?Y=slC*MMtuSh zNqT4p|6e>YMvWT3-a+PeuAeCj`W%Ivxt;@0d&vDdl-9WWY zhBr`fX6bII`ug`C`Ew=%S?<6PLicYO1kaI)9?2I63X96@beiqqA;^5-lI`)9X1Wcb zT?g@qb84&%GQTRlE^BpKEt%OrizUDgW-RxT?p{ucdKh^Q{_~Q z&bHh@-B~AQKUF$M*wq0?*GDOFh_E@Km8Z{qcims+SDB@_b;na_?RT4^E~O}qJb7;& zS<>aFHy-apLiB_4dv!c_*h};$q+k&V)_O(X3wE8};5#Mz0vYx774-}KEfeANJRT^Q zZwFkV{}R>TVh*AWPl1Xkd&aG22tDAM>&8C&D1SMkDQ8_oN*6UX`$F^lnYB>8ecZRr z`aO}>r_nCiwvJJ}wE629QG71l?=Gjf3{T6p3H@!|-AiAo#V>$Hpa#mzNqbSCd>r9F7>ApuVZOz2vl;0abh`CVO!$wfk9ctP^a&{q?GRp@H zV%ZtOo$xj8SUzmwCY9-q@7Xtnzu5DOp-8AGW{Dr%_cz+B!+3j3;{1zwddf+RSb9vW zT6K@toI?4|OyMJnaV;&u*ov9DACko%pfk4x#^f6^;Hiz|xsBg8o@FP4Q-D@JG65TX z`=&l3cmU2&0Mz511TmnF97ZM;_yc^z`;~G#BS1H6O_qQ}m|Hk^-Q}BwcX%n)6=5q& z3d)u4NG^>zEzWmr)mWg^l+H2ryMq-Nu8K3Jg-KF!1tm^<2HL}dSd&md@2dCmnS%x$?%~heF$mN2Zu%C(44Gvme%hzfAW^_GFc53V){efenUO{yV4BwF~T(`xN!2 z;KDSJwhF|-L9r$)f9J*?&;IZS>V8Ok!PQ=W1obM?I5-g)`xN#o0SA|FEKDaDM;@mO@>94+a)UYVmoJ>m=BK2%Zjlw9 zvvdzJHX9QL2w2Qz1484dH<2?FEMMMXF{H{aU2%_OElJ&5u)DNmr{irq`*9*-VIzqA zpptLfgME~Eut=>-AHBq6a_UprWwaxo%^=mB6*E@tCqe_dp(pPZ=P*9gBEGsb-(?v?WM4LelnX9UAZy~!8tw3|MESokC=EH%P7>N;Oimq z=zj5RRcd%&S^;@`;Qj68VIduD*H>a7F*z|J*;`_8EYZ#S<9bL7Yi zS4+&Yf151ZVDZ0XYz|_ikLFt77mB@&`%#t`l;0%N<>-elsh7uSMe1@F@-({g#XZdg zFU`Io$^%kxLvZZ{KS%{>78oirPAu9QfM^miqScTe7;M zVQ|fN4+1%J40VnHybne9gLysqrvd2jgPdaKcINAio6QknHols}4yRf*DbjY#B)5jq z;E1?fDUZ;egPh9_+itCjC`>|(Gn3a2g<0*WX@BwKK%&TDKBK{G6;8SrfcE$L`R_SZ zt}1j8rOV)#Po|I8`|g%xPc>an?ooN=Q21{AmF7fmAiGY&_KG7V*W$`>lWNw6wSI=A z0Bh8xRuNZ?_j~UEe!kY{H?#sy1nt*(YBw49rhWdibi-GmyBF^(Of%sWv(=*Edr}vC zoHCQyKK4jLuefJQPpZq8s&0Z~-i+PFkEQs&$9f8h*Vz3WFK+oKnq+mbw?q^*XTHu|Z0b(Lu$G zCZD}~0K{vxQG2``fWbu0*i@E`8y$98wNev1=M?+ahD4NqO|J2DMrhj)j|P2j8D5w9 zR<(t&lej#NvA6Qe+afKY(ifSBntKBH+{Ojt;OxzwOBMom_#O%Gk=^(grJ17-+&3Se zbYHZ}RvH2kb^|d&cp4wi&WyDm-xSE-!+FOC~ChVMn#$|G)<`?qa_6Cbk-pEoaon0<&YwtGu4d}AA{ zpUv2Ar(BEzyENebjVMwJJG*GqtV~2FH9B*dviN5hUYvJU`I-IdFL%R;f_MCzG{^L(;`){@lRn!Q)bB?Nd?Bum*JHVXN z^+B?RqrR%U+z0P`?M?a&{hs3gg5WMtJXBG;?TkwB{ze~!{`W_I+mr7^{`DW0ndEOn z{jdAz|F^-P6#IV#58GQ9wGNZ)>eV(@(EsAcO-|9Ceo|u1g}&W1yNxEK>OlreQo#80 zu>gk{_pT_MKh(jH40tn*_-liXS;;ptK^_V2-DIh6WhLYUm^dT5ch}@UQAJS>p{FCWq=NN!Gj4xX&Po9enC$D!rlhH&@{rrNIjZg3b?JR>i^{*-2>84g%4f}Pz zgGj)=`}eQU1y}E5Kt4t03>$N6jZs(euZgc08ZiHmDOi$KH0D8n6WZkEEAcW7&X^*o z_HW+-ot}0tA|3k#Zk2mtneS&B62^UAaIL<$g4WStaByj33$uEQT#g#XmAMsQij#P&=Q1}N28?En0f{>PPb3*n{i6E>reY?%M z8JGKCpFnVZL7zV+dmX&j;4O|e$W1FaeT87eC-C#YyI)Kk=6=2qPE{|>E@WBXUhVg| z>xEF$^Z77bpdQN$Vu=yBz=ly!Wx8$IA&-OXk*96++(A5~O}!r?qHK6>3)$yl?ns;Z zaMW;Zy$5*Mhr>0a%dX`SuDdiP^tqBocH0XLz)(&u**c5>*8)NZ7mbQECw#_f<_XdC z=sv5kxqEl&YsA~yi<_kD%&%S+x^R&}bUE}X`tTiI!+p`tapSXZg$oy$_h^C57a|Pp z4&2v?a2L;#_xIg`#_E#q?cooTK(pn(eshS3_)ERd@A(G9HkkjZeKvJ4Sq2~U8sdGQ z+r0zqpY&g&%=J&mcG(xUmhVen)9jxPvOt`I%Q8v&KeniJ??dm08BG_KfpXTCxiqRK zw>fKDYtOyDCPz|!axRdp<~4CL6z-y8woZ7{+EyOQjt}=XRx}m9zw8u}7Vo;Uo~V;Y z%LBuC%tp1ldev=BN}L40EMC$zUsyk#>9+^nhcURGSQePwt9rd$MAhU^*Nml)&IIE= z@E}*VTYrsx**acrO6$zVH;3=mGGo;5H$D3GjjJ|G7pA#T-<4hee#FX(R(V9*jjY{w ziJfgb=6thGHyzRZvRCi{k|9&M|JfqArWH?!g)hh+7{_LZ6ZRpy|15u^N4aVv>Lr`zpz!jmvueL1}hNjf= z*UoFUf`6K-p-x&oloY$7M{AwW?>PXy?~3oHh-~uNmPH?9Me$F#hWz*+xfa#8yQ!{D zcg9`;ydf7tyKj6iuzUyNKF}_BzS(pH28=(GxjokeFz+%u&FUP+9B;o6Jtr6kVXR!s z-WfT40Me|o%zJzlT@g(ua;^lYPJQsG|DLduXI|%Uo5PEPe7H}`4j3L<$hAhUaG%CS z>#x7OieQ{hi{YMQm9Twg*Kz2SYkOW6s7m`arS@6We!k;4^;>7`*SX$Ws0`*(+fqb= zmaJ8--EJvUTn)gaqlBy7hVRNPMu8Lac#Cu7)YK-vbL~DgZ_g2udS}VBCt^#m`ht+C>7koz@RK8;=?t(Zs1SMJD`#N z-Asl)jA`8k5WCT7*teNdk07nC7Au{AeR{GjWKRi2!g#jar%cPRTg<6DXj<&sH;&Lk zLA)(bVu<39-=IRiE6;3fY#M4zb=kxf?V-=yUX|j8Ace|2vRXj6-kEh~NWM!CXcXem zmsOWc8crI|$+tw#D@%5-%t^vgr`z-omPy@;=UZhJ8wn(aE&N$VLbW{*g-LZ8%*6l+ zLBedj!(!?~!l2BM4A=DyYHKK=FNH}fS0g%X*z~_?3T8MZRpo}=+n4m=Jyo>xRHf){ z-pri^EW&JKNwG!^VrD1S*TXs08B?^DvWZT&*JoeTBZtiTVw2hH_I`QU{)XGM`s!L< zS!xF5fjyj|T^!ov?}z}Me4WpuBUGDI-+tQVH(*UY9bR{3TW~Al(n@m}MDn?HF6yg1gv6I1GG%-?PS!03& zjP`Fvpx|L(n^$0^5u%}(sp-c-CSp0L9#djXQE!wBJIx#W6^XMVy!#PbeX|Ez$pbvX zk0n3NPk)%!U?77gI{6^wTM-j*=iNaP5;tzs4em%zX+?0*9 zdZ9Q?EU)0=YSk-DYq0vj?TY7}_p$hwJ;RkO?xll9U1Y7)@{!QsNSATSd6tGp-#c-R zyR;?CkBssBnRiIX`+3%;#RvzL8AAL4`D{It*SD{2_4DSjU8cJyLst70OIuAW`txJt+M>=_J8i;1)eC)D+RiMn-)Z z%dH{V2xS`=jCzW%^=F$4(bGM0pq-1)3IvMVp?Ii$c1k6 z{mV)WfG^~q^Gf=(|L*Q?ioSAFQBtw}V{(ZN{QbnHUER~@i4EQ$S)jW-*h@qmZM+Iz z3z`417P4Ka3PQr;V8Q|Cjm}dFTU^xohdgsVMny-*jhb27*qBs_$@x#l3KKq@zV<5@ zhQ2RNFegry{5QFp#pHsy^1lr$4g8b)QAD1C5&pHuNzxPe@Aa;T9PIyX@CVNRKY<6Z zA1u^uV8^is5cv<@GH*AkltO`$%Lmh(OOzs>1X1)Kta3LBK8YLpBXkf042C)Q2UYk# z-rZdO@Ha?r*435A5CKC`3`9 zOjvMIQ&WEy6&dM>i@yW6RZ&$%g@RC3Rb}Jkj2|@l-A`7^f5eSQ977yRSO_~fGU7)l zS2UdeS7O*dkq+*aD_WW&UhH#E0}~?n2e{Y1AVVn@AbG4569b|s_^bgqB|ZRrnXpJ> z2mkrSP(``FeV?H*n^y0AJRaMEJ@erGo~r^6X)_@K97?4eUCOsuSo&d+1_8pVR~ z>gnl8l_(P#9X&NWE3U6k1i)uyVfiCu|M~ej5|5BDzqmM}vs3st^1!Hb?Oo%`ZTiid#IM@KLJMnwa;5ORYcwpoa!~QHO$t^973J-_( zC;r~llnQQBR8#~89<0E?jSrl0|D*^g3@$Y-%{UMxtL?XkyalZL{@4dHU#v1kz~itH z`V)RHQt%jcP@p0swo(AXC`=5|#(nREI~us6aj>8uw*JRMf8up{Ft7};Z~v5l9?CzQ zqN}2n8o8gSKGFPhPul1Le8^zxfb9>I8yg!x6?HNFK?hB+VYo>E-ekhT#Rcb_{D+y? z)1RZm5I?;0v#{`wGYsGi531m_O=w|I@gKo`jD4cR1n*0!sHw#%6u|kv`J3&)k0|u< z`?npr!D_s=wl-Z}1l$24n76@v#*;zu*Fb;E>zzB8bHHr272Zo~9uHQ;T%wISe}MlV z?h=6of#ChW0>Ct{mj0VFem?~jhHa%(%m4f4|8SL?*MemV%n?}$|JL)HC3V)n7+k^8 zH-9|*Z{)%0HPqAEEjC7q-v@N8S@odLkKNIkq%~n4RK1KXu z=FHe6<97K{P}8Ih zBcr`<>@)x!kMrgAxDzfV<5qY6@$A)m?Xt~W-;B4K5qC6_jVy|87+^$+tSR-~v{@*- z#*p^*ZJqo!L@&cC&NKG+gOzg{kn>!6EEVMQ%yj;way{-8TS9Hn9 zNYSI2qy)RPantkLz;T9kSHg`M5Lr(^csZ0pYtPh9F_&aM@Kieca>%QB>}Cl1W*t`Y zhv7@|b|ae%&hG9nD)vS9wir|yCTElI4QIp^Cyn-VsD%k^O3IKh=myrqcbMh68%#XI zc+r%fm#IT^e0OnY_dgS+pYEwPBGn~f0-6<=7kgVrZDVyC2Wq2O+LV?zbDSft<`MgY zd70RHtxfH9HbJ&Ag}~`6N)OeYpaKuiYDB7pP0T#lKqa}cu_-+EPvDIBenQ|p=`WNZ zZ=&(bJ0+^ThG|6b&2{8zS?i|_#wY!!F^^usyItRrU%qL*V^7=ycL=Y%iu$jrFKceT z?1gX2i}dDAjm^K-_3r_!V7$&_UQ?b~1$(>;JVw0=23K!5Ivvro%88rATXJB2SwY&u zKYrQ9#Y89LUp7LUW@|HXjj*bf8Qwss!ecL*Whq^OEb@L9i>Da3JS(l%{<#Siz*W{? zTuqHRw=fcjgAcm*g72<@4Ps+!Q=*PT4NyDz*4@ehDN4avV}*y0amdyJ$x*~mt@ah~ z_!y!$NmifAZ!L6j26=a_v|h!K!_uTJuOeUWW#xMvV@wA~YhBZ@ z&-Pt5I`Ji|waSb-G*ml}7=(VeJb$NJuc@_`!`az7ep`{|a#+_X&6S+mxGUX1#;(ecYBzcjobx}Q!KWG%YQ-G`S5 z4Os{bPp8{QCCB+?ZEwAw7AIoPFZEgFJxUyy)Y2IFo;qTZP3T? zbvUquZZaT<;b5nke8r%XqXk{WVSAFF(4h2=W_Py{6cP?FA)u?uPKX0iPxNu6Q9GHRiivgs4t?HCbU*E_{? zg>;3=ZgW%5!BtWVnuP6b!p7e3QkmeIH!0pX)I`26ib^~dDOQIKrcbSfcY!IfpnT>x z3ck5KkIe9^3dh4r?M0bs7?v&Vry-J+$9-N8T%LQ{K!_A6E-Q< zoTyCdxu`e+{7gC2o{v*vknrP$u`7$&GtTFeq4}N%vXmfroVp7Re#ww9Cs{)$;Ex4i zU;DuBT6!C@@B=lfIole8^wm{`mLY-1d)!RS-0J$6GONFr2Le3Z(7+XXno`WD+7EMS#*sI@mWhtX#{^T(d^eDMTd8lg@n8^W1cns7p6yg1O9BuRR7`X~Rv~Z_&?lUkkKpGJ-swju z5)6GepD4<~3R8nml$Vz434I{{&hJgE1{#@#NKVhqDTEK7pEh5v;TMCzeLS3x2Z;-d zgOe3Y_%;j`me-S6@So=w*DL1wEKJh(THi?=u{#(pEDua2dCi=k*NBQHfX9 zBw37pxHFsf;s`iK`i&9S-_39^oDUX&OvzPICftV1ZIjc74a<0>BY2RJGneJ6YhIGMFU}GPCG_v@6 zHfp1T25UTr5F9`3Y@x48m62!n-)V_E&eMFqu6fxfy&xmim4n*WyyG6*pZ`>5!w(CR z_tiU^t50+BLJ>jLzGL8`wvc>G3&X20Yu#&nK0*3-3q*aNQ0Z7h2@ugIU1*lD+|SCN z(6Cym39-Yf1#b;~?imGoR?AOB(+-fmPs&DcNj+1I&e zD)X;*w_g@N)$gGC+z>z@s}pYh@;Rxe$~OZ_J$OB|KfR67dlv9{mTXTftLSVUE@)}( z={>S;Q1efDwZl=iEVe1M=E>KQXZy=n?*d$L_hq{EFO=qd9bhclJ1lfuHV?=gyslu314BQ zUu7hF_TrpBL?TDJS<~FtC0b4!Rlyl8qOOpG)+7S@AV(B?hrdj}bx3WHBs0|TmyC_g zS$Du~&e^z=sU;C@W|zZZ!Y@uLjmT7^=!-gvho-QoYaNw$ zq=Y_TX6vAA_XL_JSD{3rB5+~MaZs}|J;yRe$gYK|wSR9nJ|v0hM#pG}Y7& z1EZ1A_maBiSkfkQoNsVV<}ep*am7S2h0!m9TwLposm6P2-p)k4tGhs2by&l2b#6tE_L6G5s0Q##Z0R(RgjGqr=}5*#~FEktY&XD+m15Zim)yxA`DYH2K}I zneDP!r`a>AHCUPk^}8BPbz@@Qa}y`ial*V7uwWe~RLm$4%@7x8IyPP^OeHKxSQ-Fx*6={`!sTYQNz=fXxRMwfQ+01dN)L&(Ps)r+WJrbnr*Oui+IyMvb2 z3{zu9WRP1eVdf2V>@t~|n%C1MFGuv32>oWLOf@se`QbLI-*OST?aZ(quf{g!VizB) z>pc zqwE7&B0M@L={83-9lgyYGDRq!txO)$4FbDDRwnTel)uzqi|cPK*c$>as&n>W`SzsC}MmTmDSl`y_Vd%abBLi2r!<-D|S zJ7~TuipM+7@W2v0pM$&Dd*QvA;Js@+dmuEM!qoGKy?ws)?JO8)ygY%l6=-T3#A`dl zx_>E8?I~fnd|7)Eb>%o@8kED!=gCA$uBWe+1}pqHSMf`J{SXVhi4N3-%sB z#E(?_4n7JriRUlm)7m(k6~%({LkPsWTY(_!G$y&qKb3N|O&w1^2h zC2)8-Fg?6x5y;wlWV_VX;Y-P}LAvaNi9wyhV3Y(JqUlQ0$fl{J>+v{8J^4hZCK(k; z?k!SqvR@+5nGr~sof`k7^}oPAjw-|>0P)>%#1tM$pt=n%*8<1XaD?Q z{e{iKg7OOAd6TDk;)4-^FMed{>Bj{bnnsr1S*PeROtYqM1Zp#A({;5ASZBo%Cq}4@ z`w2HK7IRld&S%)O);plgqhnC4uz-TE>h&E zWV+a?il&#iin>c1-=Vsf*U|7i`QOX;ewK-CuY=E&1z2b1jUpM5nRBtu229SAR_o_) z4Wu>L(?w=4&0w~@_3H(7nf|6{nV2DSzg1md$FVl>p#D?a@M3SlKypNni(bZiKMSn>v#dPIl7Y-e z?D1_cv+?C|DYsA&v&>7Z+re51>GNKIZgRI$eCD8Le(Qu1^> zn@7j9xEyLHH&*HA%$|pev!BxRXX_SjgfvTOK{MtX25oAPoTqbe{`~a*HiKK(jZX+* z4!lgBTPTW|Uehq=K4Jn~uss(tt$)*O8QGk|_ipW{wk@qPce|XcA4i*GTi&D?A3Je$ zpWlRDv##{HXNl+~W-U-;nfcJoF!KgRn`rYSW&}k=pm$c5ua#}wN)eyGc|?J|4{z<| zjkN9tuLpzA4PCLdb*_@M-bin#+{HwIu?D=%r6On{Ll+r*aF!>|jCK0^?Eu{tKhkd2 zvZx}#hD$*V3JY7f>>jXu)`E=$dpmO-L*4xt9nV6>?`*4yW1}q|A15^D>#||2!7XaO z0{%x!Dy><3tW2(9y;2E9gS4PhiU7(n2b%`;QOk;Iuf!9bca-Y^XG+5i7c-|co#5jF zK%6eHWDVCA{-Q~U#9ZH+d9T^j7DRYJF^iK2fK4Sd+su~4uEAiNyX(;k8yi%XL#CK; zpgyGsm;!)0_HkWf0&1Rd3kS?bslzW9ALoYU(_O=&2lZvR zmOI^kUz9p1-`RP^VEg)Pb4_61kdwxQ6$hDPcY$$EgEj${B0nWe>#5hiJ>+IeRE{WX zjxRYoFKxq{mSjqAT7~S*u!gXuV>TZ};M+OH zul(Gc;j5z0%2oO*GL6yT6|KO9+Q;O~mgl#b>7SJTy&v(rCA^4ty3fz|$GCWx z%`==>4Q05_6l7vk%6uu|*-rMbxxAH0brv&LD%Kw92I*onQuB&vbFNg!$u-Te_0EDK%p~x8p0PW0el|K{UKln-DkiZ>(4;B>KZfz4UkFGy#IsbaaUu7! z?dBtcLVPWU%@ZuXZ$A0-eI z&uy@4q0niC2o=9{$lE3!X>t0SI(4e|8RY4VpRd`1Jz*-HdyxnvPxs^{@$1Snd#AUq zkOL<^WaTjX?4tJhUC3mr=V4@9q`S`u{V(d?Dz1)ZX&+63I|O%kf(LhkyGw9)cMTrg z-Ccvb`@%I?a9uboAh?{Bckg$9@BYp?7w6)CaW2-6nPFzSx~IFk>v^iHAOcH-NwP@r zV)Cuw>Ej03Z$ZAfOeuEI+^_AN+ie_^ULihR^5x0^K)%|Yk~9RAw55k_rgXWKO_zLn zbiQIIPqYJ>7!85QJh(|}5mU8Siq$4&U2K{)u$SlCMt^A3;^V<|eEPM|6W@*g#RZ~R z=mcjU$(|s@Du$kShNp8l*(>vA53);_g+0f3j$c4lbEz%gI4|+?N?<=5QN2l=#y2YF z+mxIcwnTeBpQB=dw$zpKwyGehE-=)D;`@neT_(IK9Y!8IQgNV5iBVNmmD5p%C=V{> zug`rDqmU_m{Gtf1%p6(6@D^Jgjd-8yc@cE9m|N|=lDdsiW==X9@m;0&p_857qY#%K z&5i8UiLxY=Gk$3Qz~6s4ae@o|O9ppMJz(KCt8MaLWek#$WrW7g=j#c5zfiFuAPxHf z1;K@x)hU-RnkJMBn+R)B;rq!`($7o6rs{1;yHqXfgLdqri>o(oUzRttfbI21a>L(m z2!OI=p_Z*Kl!z(KsjWK;NAh37W2T(+`9y^1ub`j;U9_>@%PH9`&@8OBKBzIObF>zV zb*vXCPi->?|Al1!!7e|ug7r#Z4>sQR&GvE1!{5FD82Z=Mv>Oi!HdYS|OF1v7J(m~$ zA@_q6Z{|$;-vTVZut2PRgpGp@1JNUQPe-y}TR$%lKQsDYPmo1xW|$0`+xHA={|HUT z+vfG#qdxS#QQ>>M&jr;9dJ1|OrTE`o3y!~Wi~#qv%BQYkdxp10QT)>umflDb389Y? z-{Od0mPiF2%BQbDf{#eAUkA)Dqc=yVd!BMx(_x)1uF8SUnj0g^)eDwQHa8%*{eQ5(jj`5+`BVH(NgPiTK4sjKKb;5RW zwhCuGNP_z_|Ceta3Y^}Okm}{^$h9c+kj8~B&<|NeJpI>ubiAipn4OZ;*%=aDt<%Gg zSRvBUUaNh8!gUuoUJlef4w%&@P!mZPL{M;>t;ZnAB3fU~RXs*ts~qjnCwRrRy%mgl ze4EofnkB2{7?>Qtrp#$crIq101QWw zcg!U=^=BG5UW1MyDya02GM12#3UxiF6QxE)$-i>7nTB$)yDrDNU0I`@Vc{E;iAKT)St+$&7ERR8+i*>yZ7*=XK3}l$IU@=1BdU2#vOIVz_}DclIsS6UfnbU54cB6x*d}uqe%-zCklBH0rXGOG;djWDDyFp+$s9QeKFP zv6|f0PuTt_2xOTPMVF1eV_|j95HgFkaf>{`my2>p=&qxs3WM+g@#)lr>}I;$-pmm; zdrWP8Bg2u&|9c+y!TpOokiXKcejcj1+?YOoUY_pEJUV|TjevTFw=<#P%MYFnjlFxm zuU1~JGZa^#5W;V|bMq~~Giw_vSs9zRc`%@T7SEQ=|03ZIudBj0-O%=!|2FtUJbtY! zc*~ZBdqQP`^^o~u%J?n^CjW+g^M0M$1K83XIroZ3>^D7qP5gSA*m(^o{YDD5eEh!? z`?iksJb~^m17ZaR75ttoWldS#4y?H!dwK6;`bpERiMLC^u3tb=`v<&@m*)59T0cb9 zQ?Zt9U7e19z6^fAixAE6yn*MuCkj@!jBzW%CA8mdtkm&+#9eGgcX0_c)4WmOlhm+| ziIll!Ktg7oB@_a|-{3%GOK~hUYkeBDIHWD<7cK*Vzj?0))#%jwP##3POC6Xc2>z|Q zY3$J%mOv`Eu3+HZJNRchglWB^kKFwevvyu#REI}|L(TXm8ek_HeJVDdL5H*s`K6$n z5=3<;+=yZ-uWw?!t{@8s|iUM^3oE^ z8XW`EZJu@An!@QT7BX6KDjpny?7`-*HhP{JzkloU@U|a?J85|SpbA-MI5VJz7)5=r z>nOS;l(9HQ1mmss*zt*df2M*bj=42oTW9O42@h=&rc`VQ6pjynNjo7p*(8oVKy|N& z-+>(bezWjbJ?WVB*6uIH90<)BCxVRT0hyJv56&8?7Oq5wk3SK&zlon2B+@Ubd(sxu zx~!L~IbQk-z3-O!VQCE6uSu%c4;U!&zJy(e7?Ayiy#E5lsH0Clzaq?+C9=ynCo3|` z7;MMv^wkG;r=n`fd(9Qu~|l29gk2R)L3ngHY~UKknhNTnO#4)Q|g7~#dSVxQ;@5a&Ncwsnt7(x zfhV}qcr<#UhViF|RO`5Vk!baZiV+T-vD*UZQ1>~JyIvNdF&AHs2($CdEf71_jqG>n zoKLKao<0^0(1s4HAVv`suUFJ{xP%WaAccB`Ds@Y&y4D7~BZ^T0xOw8@wPOt`3ecsS z6*2Y6bw{3gShNSk{Gkf}Ot9m5q~HVhgt~)>9uAFxD=Vw}&(yCOZx>$Q z=25bT5f3dgMC)o5sx_$_+M?0ZO)P3-JtXV=ANYt@HA@S8t}0_e^MY4Q57We6DG@&m zA?0crc4krWI3d3Wi@L=2g!|vdo&!OG&-^)hzkpZz&#Zzs=lTaKkzx0tA;mDvfh=Sx z!~8#1N@gvL#XI1R;HNEBHhvZFIjQ)X$0+#isOU_qTW}Lq*UY#e=`Jb3E5ffM&U121 zNlKglepGWpO4TU&j#2{G0mYH(1lhq02Zf|Z!^-!i(RF0;mJVi!b-$k|GBsy!H#QqM zQa;bUU{#-hXEV4jJHa3Dx>rlG_>0yW+r;t} zxqjd+go>J*V}lekqv`4Q;3>z7Kddh0?ilBKg=QNiGKHNjmNjA2dEkhuw4UMdhKXr6 zk5|sIgL9YF^xH;@mpsHH`FOo`*tXKV1DQ{*EMt^8Du z(d`~X1GRZN|HHdP%b=YNB1(RF4aK?^Zh&9im$oGHr$-)F)7hm>g*evCYp{-DEe3~O ziss9(M3d37@Jn2ik5O(L+&*`8qwl2B9vrO193eiSNxeEXYZ+P6#b7${q_ncl{z4ID zO0(t`GSL6tb#<2t3O-i7IP?c%`yA~tFTKqLkVZp#k2SLDd@0=Zcl?ZJIJ7mkzyNh_ zr%3f;|8E^1IngB-7rEW@c}dq>)->dL=c5nwD+EJMA=2f&=Lf8jCzQ$q5{HeldhsRb zNBAU*0tSZ8VLs&|<*m~@57SrS(>hx-u!*O#a}$iU&YF2Tj0svg<~4H^IML2Qc4=P{ z`J-a*lvi~OK56WxaCnbfp8{o)BElG$&gbA=;soYoB;2XbU->GZJjeDwKD?M7+#L_{ zwtttn-=T%-mZ(%OS>fAzREY&0Im;yB+wv^|IKk42&byprcCQwxP9#nPM}a7#XZCD8 zx>Oi4s1dC{k8LLuqgsoDtW{`j<7?X13~qj-yN{W;Esms)Vcgy=>rHNH877q!CdCka zXJz%xLhczEyMj+{24ou`A zn*oP)D0Nzqwp`P4}hx3*z{GPXl&L?}ALj|fM|Mev_`z!YX7hKu{frqE8C!rjX zS%ZVjH`>j|8C0{p;Xzp#|KGri?aeg7;pxt^6K*e1&sC(LZ$M7IU%RLw^sw?E9p#4? z`+a;Ozd)4d6>sh*!4LP{vFQ8*J&&)PXeNn3Tr_#V^%l^9!VCSzwM6d0+k@c7=mc92 z;Nm4Lx0jW{2Ait;Hun;#l&7s!EaB$vw}b0%)@fBTobE8Js6@oa<(SBY&%Kfw0Solg z4ycWn*;I6Vy`XLO_(!^F1Pa_-0@J5*%0&*ZaJd__pPrt5>|J&5+7jf=$zsZdgH|UF zQB@?yCu(XOUJWw8nH!%NhL%ztvdfOCe;C)=|FwWM0QMy^ea|yzESrSSpzD_sh1*-l zZ}hS&8*?`UpG?E}31nl3F6wOA@%|AIn~3ZbI|Hkoo%V|dvO^LEV|uoVeRSq?1)Y2< z+8{Ik9E9)bryma6KB3<}Og6%1yeB9XOGE`}CN(XXGEQ>K%0;ud+GJHOVRLD$_5RfP zRXuvShqxY&8qH9n#r$(r$UZo^n47KxYfM8^Sge=1cGZ>HLyeB7ckd7jTb*F3$3Y=VAGTeh|`m^deSQqAaY1A2y_t7^as4_r~l}f_xk)KK#2$ec~ z?m|LMbxp`rW}%0N9O?De3}8_Gv^%6e^RQgWEOf3JE*MN^AM!g732gKE7T6|>uA z`<49WOxOJ3(bJ6qLYEU(i~)nm3~~aS?W`PZ7#Q{^m(e zYrb@81~yL6rVMpH+XN@V2yCC8{l|b1cHmI&fyIM2MatYiX||vRWWwV{>q+M-HrCx5LPu^6UUPR&CZ~5&xvNFW28uD8Qmw#{@{f=w)Z2_C%`L!gv`^mR*}E<94XiH?cu z=^kuz?$B&kXh7#E*o7MQ^U{L|GQr)vd+cEiYs@xZ?2(&eF{G_!;CjN2ipqWw(-nW~3$47Gh!B|8mThBxUV0~If4U4lL|$;vZDGkZiQ(ZUqZ8Nc}yDgj~V69rKb=$RMIWG}qLthT8Qxhw;ocvD4E1(MD*KqJC{ zOPJlLpvj6rJrlAp`wN2rpd^WYCj)J#4(y%e8SNC%ZjLH5%pBlGsW7?-&LIp5fhXS~ zQhxeTZm{+PPtJUa`j44LT_224Oe!om^clf?K^tg+_n|_A8kKLYp@PF z`)n5KP0TBjCe*=(BIm3ZY5$n3=bv$@QpB0Go9QQuj}mH-E}!dNk~0E9rkpF%rdaSW zcGt{IlGJ%%sHiMAkrk(qWk{m_?7P-Zx>Ad>h_aaZ;)(p*DZB)W61QOnqwxfj4k`AF zIYEzBp|b6khd_qiM|hZ8e59mg5-YBx%Fj`nFf1y&&>h-EKh+06!Z8r)bcok2lb}YU z!QHD$tJy3KNT9)re+Tp*Ms!r`jVawR?0|4erXpMABc1A^;D^G)J~UQPT780g_F6vO*bh)=LOWT??CP{jZX zkEpf{-gPYWwj@vtM2I(uG+7pAqJ0cU^=puCQwKQuCJ~WfM58yk)Mh5pslmF%!CJ5F zC^2D|x9IR~O0db2)p6y28}1skUZPby=E@1Xpl|#p#g0dgGt+xlmU_kh8x~{%D;lt5 zr(CT%OGdjROgL?+RH~yA9pK{-QI$+|H@jetN>5$bSMx+-Z!}{~ontHTyam?-m%_Ip%BzhBKo?NkF9}8Jg zpkjq8>9R0cm7_{Szwjg}o(vi*|8X%{Q%WRS3>pfIMN%ZY4fZC?WbOB%UCq#)>TCXt zrjN<((bKllmTB8GVBdWt@T^&^47K`O*itheHdLc@J4|x8-=bUY2N-VV{U*=+k^Cb^ zz8=|TumyuK^SkQ#BAu!{5yu@v1UV32sdnDX+;@Pn=KJVr;A~k^`6^y`3>S3EeU50C za><6Nzi!sML<#u_1&xt+c z-LB|MM{F^9w5( z+_1U)_7K^>S!FOVTJ!D?+4ApdT%_8C9v%~a`PV7D)L_t6{Wg{#>2+Niyjs>Hrg5sir_Qzh5y?VE&;*+G{s+XUGf!U+~GlhCHDxY z&hZFbU)R>B(dn%KW$z$-H6>sB2u%y^mxHUp4af zL}J<)H_-nWy);Z)b-ms5%pM>1N_xSyIAp!RXyfFlvHSDqyOewnweHE+=iO7+?&L&% z+Z5uLnrFEqjLo;nJM=v@cBibK+G$W&DO_^OLlDX{<$(V6K&LCiU~0|i92nH-;XB-w z=s9~cN0EQ`Om_$o>%Zy!_LTEB;eY;=GhsrNC`AP)MG}lE1j|Vl@!h@h)@gO!_ROyM z$LeBXWfS}dWnnaW4+mIbWKz<*&r(#QCamdC-9Z1`3)_#ukit;X^p5_y)6xtX{i|q1 zYeTVqk2ve-o%wUqW@h7VERw z?o-cn_VhXV5bY>x7j+CP(bVOXB+I;W@Q*O)(L-qaR3oPy6bvO*hf3U_TC^9okWiv4m#}%(6LB!GO0S=TKN`j?$JFHB$&# zRmV<|^CL11uGBRI?_;y?s1!t@+|2T_3hOM@dS8`Um-W?%ar@oCt{byWoc)sTQkI(q z?=My)L!mD{l%j(1I>g1JE?*vDtSG2v5b`n%o+77C!?~?PRlx}@}AqIRk+C?Ov ze6}D4cuRc8GxK@aXQ-eXtPplG3`FPek_}c~=J!-p*N8i*E4UP&n?NE0lhb(XXn7^w zDLUA)cez_%eu_R7kg5Wc=qlS92Gs%%YLU612 zibvz}^nB-M%3UY#giELp7lO!e21M*}O{I$w`_4rNP~ftXWDcI<_z zfg9!P2kg=#wrsx3-`-ppb%`hS27H-f42_Wq|0N5L^?uIg%%c z$MUaxC~v`8g@XI`=SL-NZHO&mh1{h?t21_f_aoP`T5DXj%-On6$n~Dg1?{i)^ zdO#K$-+g+pZwS-KnON1{xV!H``wRs4EN_on@)-K>ym>}?dtoW9?e`0pm`ZxCxpqOY z+vJ2d9BPU?NT5^qb8~QM2+HTOOGA;c)>%Dc`@Aqp4hoKte*~2BSv6h4 z<2fH@SDo)%I2_^H z1V4&BrR|-e6ch7D)fI;)b1h(cUa5V;q04s&nAmvx=qvlpd)7$7Yh8g~R>n75^FDWd z0NjNu5ZFzA;!wxuDc4cdHY1n+#P-4pTK6G!*uE5o`}T&M@ABC( zYej%Hp7!xcq;c&M;B)#olqPX^b=KiGn07p9EYBuT-&-J;yEAY>yRRl55l%jRHCZDl z{o3R&F6zrzy2AIHf$_ZLZbYg-(ixQ23yWM^JTTUdP_c5i%rXU=vlURYJ`^66X_MN2 zdP(BEp)PO>`l_X5SNpIr?cCd);uR~xCji(QL7dd^4P4!5AdqqSJ}KB5b`}IJP#V3p z_ijF&&+d6ZaQ{l2c%E8s$9vN^37r-j%I!X664j0d=WkyCgPURqoyP_0wtUu!aBN~= zAN5@#=&_bURCYL9$8)uGW!VHc*l>5%0z*(W=AKZex>a3OG-LZ`HvD^C8|W^G4ql92 za^Qpt(=q_sG*Fi!8ZrWU&uOpvFV^ZJz;iEe4Dhx_vnR zoLgg3y9fbdxwY}bGF>UXFC4F^t&|eKeTie@zeJ>7tTFv9r%?}_6;0Z>dB50lkLwT8 zU-n_`4ST)k!E+)Q9FD!Z42r$li?I^$Gi7s&)~~NSy~1DNHHEVI2_@`RD+glytL>`LC<+PdZNC~YUXj|y=in^%~<{Xg%?}DHg!gopSaVN zJZG5(#g;qJe=>T_4MZlJP6IK$IkYM8NEWo@n-a;W1?UF2E8f1M)aAG+BPh|0?nM0o#$RI_wo%mUxBJEQn6FyTAeIsv9KLHDBE0b37UnX4bw&60 z8-QN}qk87uqx=fpUy|((+LVI6+C5QBiZKT2PCEg7dAS9g;VQ3wbWYU@@8q9hwDnD!8L7f+AeZ?`Zi)*8EBaT9SDZvrzu-(3g$9*wh}kDS66Rq9p%{0^L1Si(^nj>CA74* zskoh^F8p8Q^_B%khUXxs`ER*=Uz?s+KAns#VlDqZY-y!OJ7_$d@#}sXF~k$NWMR}* z{mxBT;6EPZbH*!Jd~n_3kl{Hu8uIos&v3iq&`$WDu>&4A4t~skY@E0Ep@s(FffHkh z)2ZbJHm_EZxAdGaf7{W6g9ut%>8!mhV{LRCzjy$>JdNI9Q=j|vXLa^=+3$)u9UrcF zr69JAd6zM3ww*Pq3uXYcb{yS6X*d3var!HSYs<3o-jjJ;!PbzeZeb?~>Xq9GNcd6; ztQjizI#YMfjUWJR&mo7pI+l?#sr7FvI}Z>X*yHjmTITDOT{WwVayZv)SjbS=lA=TKxhUYfjm6;kj7NIlf*rt!!_b)v zyiG>6x+}X!A3~f*iUNz&_&UM9q2At&s9{|qMPNF`yjjtv=UeBg_Uj>W&-W!m($5?p zC~zK!bon~$)9_y(D|LQl1&!4(YmJU8<1XJKxFtPWR!o?#*uF)M`yk6&VPq}@t{4q$ zHyXDrUT*VtzYCRT#iGoZa|B`;74g7{f38BN$5IV=jj`%*0)=={ssPMyBtaFtED;+YkcPkCMEU0Sxli{+omhoCf(o?)p+* z&*Ko?v?ccVuRdsR&rQ~j+H#%of#cO%V=3@9RG2>4&O?-&Kz;w_uyp^{xI6Y=ZEq_< zI%*ABnt(_?%&%kmEe}c_&J1nQKF5)7knQrUM5Gx%K?j(LF-RX{GH4J@3|~NOe9c{P zm(Z4QT>%wC#rwG@?5Ot2rPM(zI5~qXUYADm$kQent=5VRNvvNE;GT^`~ zZT?O5Y8D<4FOL>2F`AaW$CE{nmThXtBLYmD8Y0$~GZp5bThZ~`SL3_&!A3Q|obi+Y zf<+_X`3*Z1*;$(hgPNl@djLX1=^dB$s-|{M5W+u!fGKJ!@IrryDALS|GqH{<-qM%1W zAaA4{7rV~Ob?U!x^ZGs85~a(b(o_Ni={Lqp)RH?{vNgFxCXV{d*_=}X3%=EKW@l~^ z?*PBvUY#+$r3F{Ug&s7mJ-^b^x?4Y2*t8%(g=p+SR!9O0DO`y^oqQejxu*0IR0K@q z1l_N4D%k=j=h34GR|jq(_+{l4(B!G$q6w(e)nNf@@TonitJGDUt!IX>``g^Io@zRc zcIc`0HhYb#o9_?sDrRABXZhPRI_ZdkPLL6nSH;{eE@|WNoi3B&BR#c{l^i3LMu|3_ z2cBNy2jOBt6Jgj~i~Q1Ju2E6QG^M8@=9cJ95;?#SYLtdoaW+QVi1ap!q*d-oW}}H|=g@^?l0;le3~0VA zqqwNh?^D;%7!w=_&1+5^ofm~RWs~CD4CL%1FVn#|87Rg$;7z=a2WGAg5PfZ-FVAylnx9p&=s9MGV!=Myq1V4wX6gyTro;UG z1j8<#xqG7dTLLQWYYfls!Hn0y&G`fF=F61mPsT@rl}oW6PLauIEIFW z;s~@wUTpdkPgT6h(Qeta22Q`W$Sk`}_gt-0yVo7FrGo1_o+9e(OeDDB6ue#X4iuN2 zkwEa%(vNm1-27s@JJ+=JrefjeBVlW{n^VH3LE7f9fyvwM+M6T{k~Me;lYV0w&*Sat zMa{EPp8(My%tG?yIZ3k*I#9{s(s8ul%E*&*`X8f|J7|0S!0Gs(^(t5!`4s=*>FpB@ zad1NW?2T}s?L%|m>)gkYBQ!P!!vsM}c5fqRYp?OEZ7<}x+vpe8dH#1O+!&MQx=oS# zHhuy521#G|`dyYHZMX+<&)ZFZ;sZP!zjF3z25MS_l6yBhQw-uOg0L5rv-zhGR*~88 zV-mkhr1L&vZ>|m?NXIR4@Z8Rt6q?3R=q72^ba#k?68}Ch+yxTF=bsf2}f9MIX z7DzMfiV`T@Q$o#1QS214t}y88PyRg5t-ILc1APe`Q6&~^yGHJQzF38&M$(gcuS<=l z5GOf!bior)?ty#4*p{XygHOv*%uNKBAY@)*j_`J)@HQ6H?fF5hsj;!fK&dTEk}Gh^ z68Y+bfON*9qiJSlsoxDnVIL$=8cM?RM!#pIC6_1ae6-LH#H!9%-A{mmz96hbk}Vx4 zNtvE~SZLihHun2Ov%!mUazP9d5St)X$|b6#`0WcADk{8&=U<*C*Y64H$UPcPP?KEW z;aFrS7Y>IBFMcN$qMD$|DkskAkuIX{he>&k+kAdZ33l^{h33AGlvr6{0UY*8+&^m& z<+&rn(__yz7erg-4vWzA;ioURCB7J65AHaM4Dfmyjq^2Oe~vGFcZ)GZoFjS@06nIeKV-|)IWcaPSck047e0senI%+||(-(nWy=6y1 z0(M9_tHkq)hM{`@7@wnmiukfnRG2ku{EOwaQ$#4QhsU)NR>Uvbt4uEI81iX?0nPS+ zN=B{mh-S*Yj()`4w7nJ<#GrWNuyFHFhU=lfb!t1)ojSxX`WXx{_+FJOd}y1`c=}zK z#rg1xoTt^Hu@2thn?^^{L|vF(pBswQL56Dv*XeHVHu(RZCcshnC+6MvF#a!Sf+{h5 zKCq7jIV^}XVbtWicp(NB*465l$6Qkx{r49STk!BYSQt|kU;W94Q2f(vpGqHYU<5}} znAQidyB*TRpv=T!Bt|^AL_rMuFD)(Czr%X^1zj1)0PKD7)sCkXxtvDRKeor@@Y^l| zyzif9^p16C9~EK@>^tIbBWwsZtTmV1vET09^^1zfruzqJR0bVCGqSJ%`i5{`^&U~x7?%fzy?!>yU{oi8NoA$o|ywX+aJvWD; z39v$(({1doWTBZ}?}c6VAsDRfc0e#3&cJNZB=d6FSb5aZq8TH{Q|<*--rOB;_g8(l z1y!R7lzF!Ffls-11LA@;inj07bv&VX-?lS-fJbhxJ0)1B@aAdRdKFeqp0Be{M;BF8 zn*^q{?n%S=0+K=N6GDzeeZbuP4nf%lPyv-TeD0paoloNIu=2-zTB<;}wRUOtc*x4! zIU_-ZHGj^PCH*KhAmRdHUR`rshu=L?_ncPbCPrb8ciMiYfVj-Nod7GRP-0*v%7LS8 zWUNM-lT|CI9m(gU|sbcE0POng+zwV%Kkr%JsRE?{2M4fOn4o1sIkP zuH1iQVa4_PB&I~8w-)LNXjprhq^-EJnTU;HA(Aq3l^8Kec33Hq!EL=DeKj;;T9|^| z9pGo6koy)`YWA79wLR0`ABg* zZ)I)mhy|;CPN~p9uCcG04UJf-=}U@d+!!#Ydsu49I`dz=Rk#_E*xli*H=Id_LD`xa z9CLc-0S0ADd*Eo~SFw?NiYAB(z765l{)SB8(W3FDK)=9rnTTlOz1Gr2RA5L zS?#^z+Sb4|aKOLuYsYkqPEXUMRMnZ~Md|SAn0$-9rKSeUc}(`yhZ&DglSb4nd8nmY zeF@-1UW=4JQ-KD`7^f@yRoyqz9Mxr|G#%f@+O3XH@MuQs6J_b|BaYi{GORV#_9fKh zmb+F|RW;&(bw;2)#gyEj3NMB zFrW!&Dv~3HDm521C8}k}jhz&mJ&=~{eo5MrURbN`ejiq0aE+y?d?=>DfRUuGtgQ)-+s@y{NZ-BVm2ncO zm=J__7RKdpFXBWK)c!#W}^~NHMiq4+Kkne@u-kJy}aP z0Zh!0`|B~wGTPz50Ja`**xX9485XIhlFITcZQ3g9lKjqhY`8YB%D*`E0G}%>>uBMr zB7@fEw5=lR|BHa$S_%j3rcIi;=u+cFe?)}#qz6+@ZEbC7vQ!(i(j=(N;1x}ecc`HQ z;{dZqh#^kaI5Xl@ljpVBuhmsg>fixGaL_ZN*s#qnORiw%yw4!cg=Z392?GkuWG$VD zsj0mVC>!&+g0m?7vfu6+cjEfQ(j3zBkh{4X_y)qPMkd&$2UAsZD82~?@8YDKwj|z0 zf+P^`Z*qN;;A_S*Q?v}GcViueY`4iZJ`ESqq4JDryzL=3#%_!f|F|W;?_zC#f?0b= zbhLXtHMf9IL(Pg$JKGH>77TI!{aa@(E;e-P{ubu`R3q#m^Wx>RE7m_%ga-^*0c<;P zfq{82Ys|*SH}MsGW?^O3Z`J^kXZbT-+6PrsfLS;`B9u4N3GSb&MfU(+K?Q$rUdlx% zWSiA6-!lnKl@sWuterOR`$z0yOZ!UEbndY!V@D33d)5ekI{Wa0g0hjZfjhtGp0l2B zVglLcFCDnF8-s*revmz!9}5Yew@PEL`(=C?o-Xy3u$f6ck{6CB0X);b`Y)^ijpm&Q zlsb2Rlu6=y<70+@0z8-Avd%o&xe_F(S(|oUzttlmB0Q)m5~W;p0ai^{=Y`ufdeWjX z{WpV<4N~}%C-ytO8ZC*Y&=}O8Wxih0Bgo3i#k;B3AEkV4D{4b+=>J-@!&qf1<;r-y zD=I6FA)Tr>3}e3{nJ+4O`9w}=Xkn4DgPGat=8MjV)WtE!W22gGI8fUCO{E?x*3B)kFNup(<`#|U0=xJP`1g#`Nv^j+lqt4Ha9 z!CfFLVvw`#MrS%W{Ne^2*xSQ}CmOK(UxX=GuisB|a&m?ZY?CBQv+(mzUS2ZWhYW1z zm6yY1P7d>m{S!g{+r;bBqWk;szaRwVe{wZ{oC*GP4u_P8zYqTjMmMPbv*-NlR&dMz zwc!8XFW+Lo{AW3TKWJY5&(HVw%eiF!kBR)psQr&1m#bnWM??hFIdh)L72B5wifP!r zhDs07HyF-r&Si{DdL7Ac_BgmADnW=*VKbheCXXp#KBR_Xk`>bsw>((-EgzFV z9QNP!2;>w~`b_t$+Q5>9Nd|y(xHb$LTn`b387Xbyv5-S`cgz3GIAr8Vqsshs0V#4Jbg0Owo3#VM!p?T+g4HoJ zA3dFa>Ux^MIIrycw-(d}!#Me&aO!y+=rSqRBnXT!(sKoekMAS&Lr}8p>ELUMk0>Xz?aA?x1ESb4v6wsoFMh~+J z44PN!(mzjOSMOjgGFA54y+>pY)a840;S?=gz)r|u4#j%>hcl9RUU;L5wT)4WW|#IE;NS}H#TwN@LS;Mp9_U_dO2$ovykhv zZ(%?+Oa@F={V~%o%>Jtjdd#yoU{;2(xllr7z(tk>#^?KA%)3z0w;l#knyjix!fL4}Em92Izam#gHS< zR)S69Y;dxqbW_{zEGo0efJ+=5q(99O={y*qn4@JCvTBbNm1xJwiOx4P(DePu-mI*@ z6ZVsj8pYkA${${Le#C_R$~r@9Q8ScQKy0Z`G%F1@pDvEI3S;S$nUpR|d7GY|4~W^U zGQAZ!r{7Ea#5tX>IWlGgYJ!7)>iI)dV@!}r6xS|4%!*$vu`Je&3A}_9IejWa|AitK zF--+rAk=!u5?;@BOx zmQ{M%VlMZv*#H>WVzrrhE~iSZIc0^MrKx^JYHySr1JWP)IJB}k%G#?Lw70y94@?DR z7F0<8sWdsooRkY9*?bql@u`ssV?bz(;dnuFU2^*lgXGk#b8C<>#5_)hxHzYh;`~It zSaBIhs+|XUmJL^24N*BXJK=GNp`4zKWqlx$i4feA&=xR##C|s}?lZf&V23>hrrE0N zKtY+z@z}|;7)7}gn*g@oGi%MD)9UO;TH2Qb>@~n&KG2>153~56vD?Wl9vW}Fx*{-$ z>qR&0!z}75`k65dv#=4-^#bF1o5rsrQq>`+ybXg=W~}_@kBM_$@r@l0D};$Cx3gmJ z4QbI;s6B4Aew8i)F0XI9E1pRupRGsE1Tng3FkUma6kE5X=(S-&9=h~l({YK})nMiHaY|0%CkoxNqjluaEw zuaDZh))I#9jBJ%!UC>+1%H|PR2tamN%M>fd{4ODh)mU3l4V$S&&gbww=9L18eo1jl zF^%}x$X*A~N85+SgNzc_Wr2z0iOd?WbbHDp?Vk4?^RocP#!EpW^>T;z$3%D>+=7P0 z<#B8!wZj2RI?^S-gz{}iWlzeqfVMtTc7N{ApVYYAH}V}0__P@rZ#(lW4SfbIPe z+!3^J4tnEBo$xf2a!*Lmi45<~<-IC0%W0XJ28$yFEaNfc+rtHxT>N@f(iz*^QnFf~Ey`)G5x$K^ zMAJaLpkrVM#v`)j&XnKm#qGOo)5{sSmSxo>uu*$%udv8ge89m0b2RB>rPbLP1q?s8 z9OdOpN2{9?!$URSu{R)5vI@P)Vt}_c#yqQAos3^v!E7*^y*%cp%$NH7LO6T`}a!E>v zFubDz*Pc%jT>GUhoT+|r?WI8$@Z18#o|r5c)^WB$ELWMo%IKK}tG;cP*E2tT-4<8x(PrHGMF7;;*_+$y*_$ z0nglh9jS2tPz10W7-F+!I&PF_l${eJ%xn(df2^X+Dd|jPd}Q*3vg3IimP}esCnD<8 zn~55?83XXJbN(lle5sJ=*h~sGGakR&2smbQygmwhU4m)I(4uVlgoyw>VzgIq%z zef1?;Zp0URKx+5I#WhuS_BqY7{&(P!t}`UFTw9M97Lsa;wfuU&OTg`xby`)tr>c?1 z@11ZoimEjXSAxNGHso2BML(v)EYFi}QdWu(twD3IgQc9N*=Yk8ALM)L@d%)Bp@&lY z{c}lu+v!qY+!?5}su(bzWoB2EnlsC_FwSV?Xo`#l#X8{pWe)D=tiaBE=h&%ZVM+f) zu-Y0Wqs^b1twaFKDHC%@^S_R|dot5!<@4BZ5vcFcd2E(Ea- z?Es85Bps#IDaY~5mz{S|6>^uwe(jZ5%^G2;wK2bg0)Imfch>*HH&+WSJl3>4&MdJG zTxePnu}212nTHKV0Lxj@hHWU7a64JWG-m9H*>eY&3^ktyCr%cctBb3wLi6L54uDig zg00bME8}8K0#gzh3!;XOv@3S#NeA2fjgh)fZU*36>wdzsECs_;i3=9Vev>0?+nG% zQBogYKIEFeMu;NV95~0Rva9JUc6y!d#DJ+noB&8s^aAPf2VJ9%+LWRmXl=M%_q^{h z@u(6b2K4(f##cd=ojENbzXPmFsE2WKGYqX7BeSn@s@G79|f~T}_U9$CAm1bHK=;;n{h|>+c{x9O*GB9p$*B5QtFmoDc zn3-|H%*@Q(Ff%hVGc$AA2{Sj$%-B#vZ~EW&ocs2<`@OGMAIA3B%eJJo>?P@E$uM`P zR`)y>w^~mvM)@B=Ys$Y_OYT|DeF?|~>7nCQpQv-YKdrPvQ^ejQv%idv-WJ?wVKycC zFBv$Sgo+3qML)6TbhS3NnP;Eukl3vGWJmuJcM)c>wME$PU^_Gye$`|2qO{YIDxn!_ z5YCC7CL=2I2Nln#7-cfzF8eBCF23C5!m}}t4ckaJF*$)6d+IOf>>Eh(!hM$T=%+7V_gHSpZDf1`y*KxX13;br;h&e?C-5Rr85ZApP= zr7sUnrjLq&J4_GXH{`ZtLc8azw6wGjrGqIAoSu zY9vDs$Kefjmq#Raa*I9So87U@Cu4DHekCyDwlaSv=3lgrKOW)p-PwSRAXZ+_xSR(qg@&gLikuQ80SPV z14R&!5KJ*l);I@)Im+N1BM=eL^dCg{n_1 z#l%SV+3EXj9*tvu47^{=N26oyF!#a7PQ`0)hRz%AxQVuy&vzF_R2(5X9=m_CvnG>1 zeLLsuj5;zCr?M~p`VE`pprT;)yKt4=P|pRjc55)zWGf>ZKx9FGJo5b0$)acL)#Y}` zH7NvsNJIe&Ce38gVL^?lu_&o5PcNTR+St=3+FpzTtDb9e;0V&=B*Q}r zwo$Qg9veU)LBZZsJzTd z=G(?>lU)?z26QudzL@b<%Ey)ZPH%7!t`p6WI01aQv0QsqPWNYQp?xls1jU|-3?J-W zBgh`rD>Z0*zFwURE{w+*Da|j>wn>u$;1H!UU+=5WuPVvMjz|AGpXZt6991-BuT^>N~2TL!^Um-`38;$3dv{0`0%K* z0d?h(N_pSp^Zf+~;}QV`qZa8Ps5vZ634$$cEuw*%yBmwM(kQqkt4wBSsg*@0b{NOc z!o?KUWKTsdS)dVVM$T8Vgtl%hM%k8>|7By)WgI~>Kg~40|@Y~Llw|p zJGl0-Qg_^h{&KB!jkz$hswCbzAN}h9#I_n+p-jaxQ3t{b3sL-N0z2~lB%S-S-pmY9f8?bOgtjOh67>h zC282laMmm;CxFF&XT0VEdk!EUZ)Yu9j2H}LP&bBIQ^xwbjtn{46*?I*H0U##z%AZ> zM$R-4WULS5>@@|s=2p8sSS~Iuy3s&BUQo>qkaPFH%%^KtlXR}`kRvFY>Mk1$Fc#39 zO2*sF|6&bk$68m~x&aB|=3_Z%`mw!s;OG1gr^}jf*iWz_*)X67p@C9>WHsz}4h!&=WbWC=XE-R7pw3`7EMEv#6 z+_IoAw;b^PCT(9|-(Q0c>j#ySKmqx%b#37Pjr6|A@-T}01rPX#fK|(1|Bbpa$6Q9RXa>6K?)3E8P`?Cm$X3J)i)MCse~%YQ ziTPiRR{pKc*KWu_i^4D@zm1>(U>5bz4Ac(;=r4aM$S-Q(=YOPaY1Ds?_&7S-Xkrezh~kBaOSYY~sE|>Kp4?_AOHhP}!SxGL!pd0n z#i3y=ikp!?3tIyY3fSrh>uunGZ6!B=ka$d`YqFZ_WFc!kSqV}cI?`11uihz(lqCvU zcg#fKN?OoBH<6-bG3W?noM~lfq6DSupgGv3c?l}e8uNjN1yOzRKfG>>ixmMvFj1vp z^$22=U}FXn1KSeS+@(n9;DFY}7`by2mPFwEvRVbr0#ZP2d75NQ2`cw2i-R4i$sgpw zB3ht$T&ZI2XR3tH0C}Qf5m+F{tX2WK1Qp6^|NLhvGNEh88A|zr=EGm$bB*zVcH%UE z5|lY1*i9)ds$?~bNh%_`VsvP}q81^Nc%f^%Ih+zfsQ}<(-w;R*+dM@hmo<#f&e=dQckxyKLu+D+RtM$vZJ z=y{G(Pga&322F8{-956ML0$-!wN#U+d9?DIelqB422b*??BhoJ^A`^HM`&|ec2ms)_V^xn8NAJCGUAm*M5xBc{%1aTFbkZm-gj*I<`A5-w9KfV{-k# zXTilr$C-AqPZ_vsLxv`ty8d`~ni{NnQgfwv&TA_@Zolz!Y)NU0F7b71X>aTJI8aQDJ@_lEt|+FXnHz>(I^H-!%E@N}nreEA^}QkD|xW*AM?xdk}L4X4<`Q+)8!5RVl}FF-tKjFYcG%8x7UzP7A$uP z&7F|B2ij?N8fX1>0UzXGWDVaeojEd`TSI#(4PTtDsrh;2?W(|q7aKh+eHW6T{-qN~ zbo9B2>1oAkuIa&8YB>lJjbR5B7X)@leUdrWkpmV7IS<5>{%`E$mkDm02mJ-?-l^Lz zl_nf30DB7sLX+@Ipp(YvS4_K4bt6N&wBxdpQ*6V?|2&64X>IlWOLGMa6Rd0kfcn#U z7RL63k~IWDOH{*)hximbpt=$eu5J$_<=}QCG}-xeb!j3CyATJ-Tsk>BCqWb{Z?7b) zg^3&y{4103rf5jFX7wb^k9wr5N@B`}b}j)WD)H87oei{L;F6)Azj}|3Ha`6ZLkk1* zWY8hQX;e%Bc%4N7JV6O!ci37mxLOjdDbUbV-H5#1zBab|IXuEPHHfFkSZ*BjlP zC^^$Ph9`8Fb5XdMU(e7m|1mRmv(NJ2s2O{?0p#(DsWRWgtDa)&X_OtEm7o!@ee|ex zrXcK^JRX2`WgePcMbx7V&*CLaqhWTqV&C+_It{$A({r;_nD zh&{CQa8n;a%G2&%n-o61D=w?0vgGF2O3|mUWb%-fp~Y9wg@2glMnxS3(oGtJGiod8 zwqrB%WYgd`9U7sd4z1)v28{yFnr{wrrtB-)`oXKNQx6Nb8K0GL;Z$BFSuM%0WR)sT z2-7tUX_Sc-=Er6XutH+64>^>Os}ClA656^Ll13VNhIM3kJUAh$)o0fB@^RzrYIN%c z{+i6busn3qQ-?!2L!34{T0XNSOkFc`JPCF>dWlef4uH*?->dG3SwDth;o%kM!X#^I zl_i@SDJk$Zx9;rGMQf$ogA88|~qm+D8%%>40Nh>+m*NC-4?R_Y){&a;y(dj|yGfGILrsHH4 z745((O5dj8wVLHN)Vs*6C`;&vFK$ZzZXK#eJZ$GmE9s?@vog*Jq_I5bmY1J1Hnd9Z z$(AlIh;r)D9SA~g9=};+R*pK`r$&zEnB`v<_F?t^``+BXP+QgA9@WXr6dRoKOjQ_% zsC^{o+{`rgw9HC`+cDw!(qk^EqS?KD7^jh$Wk#YZpe=aH^XK@?hR}j4^5om0+2P7G zmj=+P^Ut4Si*S-;aUw4nwMFHXEHje|iK`h#x?hP{w@|n;Q{QKuJ~?8x!~AY9pQ@xSBPYlFV?gk!zu6gSW@SLBwJcyekLV zZO3`tbfY_RFfHmYlP*dp%(jXIFHuPo)%Xgx z=o);PUi2(8nlY0o#%`sQ;;7|CvT_oR7e#{WF&<`W50u=nhESHrr)58S@JBhW$Ifdf znY$XrMI{Rna^SNs3b{!=GN^OnrK#oz1tjHj1j1|=tC-|Z}8Eo_ZR!taD>WbPOA9l`8(?MxI^?SRCY&%ga zF2*bfY+w(VxeiA+@N$%rUQS;G879wr_-}2*1xtPc zFJbrgs-CRREG1ZH<`iNs;1rh@gmfwWx9dtO(C5$3@su0^IuzrJ>)=RbY1hQ}13sc= z_*?t>?+40rjj2Oc1YvMUUo)f%Ht?K^NY-)mDbg(pxlqWv*vGE;a!9Xk_Db1Ogi=98 z06{e$lSbo5O-A-3Eov&zo6hND9ITzucxFX(MH*O~!n~R^XYgN|>S+l|-!x}AP@pnf+2ASOrR3X7kiTlm z8Vk@S?)fn^FW#guFQ8nAv$nFE#J_1?wqI7z5o3~K;ByNjPo72@Y-J%Mb^}paZf8|6 z1O>IWR@^9xPS3NKXMZ;et5+Vo_=oc(#eBW{E_eeCYMR&3%QsS7If`?~6Q?k+XC-Vs zZ(~rk8EG!JBj>K zqslKf#S5DH>tp)8v#@=o@n~=6J}scQvJ;%@CVrf_So*5Wo(Fx#o9#mrgi1Azxp_!U z&lf=MJQ?C6rslYiimHa^tGF0>TZ1)@rBtv49bh*pjKyRe|M|`3GH%_N&U$`ELDtnL zlZsxyj9Myr%>W9Y3te?RldI z+mTsH*az>*KB21(-%$Zz`2L>P&eIh3qctN{B>a4c(yyIMZU}Oinx(PTjhaamNvYfNq=GWdIpe?E_jfRq9_IvRPdyO`~x&O4E-_iW?6HhKF$^^@e zeFCu^A4Q(oE|6C(4&r`ta$o=k?eUVSEe}XjlF|J^flr5zBFjFH+_aqA-A|BLy=x`9 zLpr#+K!RV5ha%68*5pXb`x@NvMX;G&(1p{Ve^EeMn2solAX|Nm$;U5vFqwFEL8?X9 z;1K9j+7@R|LovbkiuP#!2t$@>QGL_I;grYEt5tmLk$fv|Bg4P5`AXkl0_UKgkTRUd z+QQBRu$zJhAQiGgG{<+eb_TwuY0g){cnAwT59;5i3H@#@TuB4;`%w^;dH%i*A1NkCt#%W5*e_MxBz zsPp6?P=!*Zi@JI)zP=HgOaWHGLhfh=9cgJN5=F&b3%@iK-~n6-JP!{PdN#4NF%q_} z1WJuvK?S*h)eaa(-uD8=C(Qq4O{nJRba@Xntzf_ZzES?1poN2zRW40hZYO~X_grpm zLB^d{*jg3U+8(!&^ocI$iUlweO$YH?<*sXM{pgqy4gpXmne7qx7Kd}zf4-qi2v<(|5v$c}Bd%*+-eDci`zZ4P1_<)qHH%*VL&D{yP z1;V!Eyfq`~785}q`xd5r08Ji(6MaIb4!_=cA=2sUE`Y~A2^j{N-&tlt`VM&=;oiW> ziaQ~Ya13j%TETZ1YuAdgS{;1pCH#||CDZ!^PZ(*EGc zw%#QplXA^|mloDJmL(3zWnuEYlb@#CzqR0p<`FsSz}4Z!9j4)kfB*3jZgFz3G0q#j zLEC$KmBB-+Xh~Dg+#^(f55W4P1tVn<8Rq`mJ$m#WFD9tvugTSY5&V0W>6N9nWm#cm zw~d6#s;kzqpbSx3fEefSq{vcDNf&${Z=6Cgl>aGyvy?h`UUvQ#099*d=b61{0H;2M zq=@?o$4Fx@E+b>D=*p8MugP0-6_*qaDr~>=G{W69wlz+oS+D=-g1JSh*yWh}i>(7` z=4pQ1u<>2BxmJYTWPIb=^L4*9>)@+O>{+{)b+EB7bYNvFSS{T9E(tXB z^$K~>P0UE7*Fy9ijPo(!EO0*P@+R(voU-x4JS?y|h5hW{e9KY>`iNsw_vgkZu7svb z;GBP{6IqZS28GQl+vr;DLXcH;{MiwA9t={bEx5gb?po8N*ZrPPj!8=Jy`-xqfFt1S zfQwU+^#N4vY)ej}B}oUA(tvbug1ae;J_n25p_M8!Kc!v&*1|D#5{@nyjLcZa_k-k5 zqN;*q{=0ov&xLXA4{iBE?ui7eLm6=rJKVZ#V=tcwsZ}V8b)(FZ6;tB-FRT=xs$v+r zN8jyFbXYK_5y+oE-d~`flh9bFVX`_~1#5MAr4@bx`%rSQ>gLzx{m)Z4#r~Q0{uD*h)%b36RPH4UJ1AvY2+?@wY$FuU)3J{aW^^ZnruJs`2bHnW~y+*%tTB7FXfn}jWACiR5bHX}YK_@4LnBbT&9YW01i}sGp1|)3#LkASsb|D{(L@+9Aa7y1`chLMz4PV~e4~Vs__3}PV zzNH&*G{tDcV7ljb7fh&cNP?efwv9zO$ z#$C`0u|o*aMTPvRG#-?zPXiCT6mJq0{M|%?Y%5_@tx>&3=;4^;)+C`(Q;6b6 zfcxMYE+AZGc9{x&Q{Ez}2R(Yxl$`@L@=rU-SZN5tsjQ)+sJX=7EKEF7m-adC z)u}_Re4b~8$(UlJ=y&Zb@+;ct-TBlIuqDJV;*oN>aAFm3%h|UtH`?qeN+{vC?8*o5 z*3v9C+5n1c2&09Z4hi!|&BTCDzx9eWxzJ;gKs{v???UMsU?hi)g28Tq+o6$xSc34RGw=U(3{*C}lW2)Zg?2fB`%NOL_XBj&htD;4&9^Rf&K zBk{1>Q?xqM{)%TohKh^_#^;xA&-#;UT<9rKB`oj>+(0|{GngFuRso@``4lJvAZ*4L zr4!@WxQ7ugeBZKYT)W~Lv)C1iHZILNBGj6K8uReOOhzN9=A-(K>*z?d-PIJxSEWyU zI5@b_bn^CRt|6W7$r^? zccF*}KCB>NVm@2qry6@|^AVc~hoezambEwdFpmgvGwT4*+t5TBv9=Xjj5Znapu?LN zct#{}jS5tULB=W3RmqQy>Q5_=gL)1`ehO0bT?i&snitWw4*s!!Bb&TPN!H*K$RNQF z7PTY{-79qs0nFp#93!7y9X=>vf9)3e1EZPPrdyy=s?$&&C8dx(=Ph}6@9dE`OoFsL z!D&%5(~C?*cLRu-^X3HwSd7*ZrzNTCX|DnAGS@xVON{m&;XzqwGBs*DcTX&`W(QOx zd)CRkA#;fW193C3@*Wm?ve3PR;C_d@TB^ipP44f^Xa9@!p4SQADP*IYd6Lm!U(?mMH=U^RZ}C~+|n$kBt(>(-tXs1 zWDvvVz|4X4g7{OUfl8wvI5aRWUNH}25W0iX9XBbhTZKQ zQ~FgOCH#p??@RD@*+?uaqxT#2BpaUqAsw#Y&$#JCfATJ-(Vhm=TMVocvH%I>fFWgk zoGiZ!6SvH;xTBF|-e%+{A734WxnKU$BB2 zHH`F)QD>c1RafWjoVFMSb`w5H2VTV;nplTI8N-=K(@RNzP9V%z920)F=K6SNu-M!( zaPZCem4mRjsK({>u1hZ_Kmw!AF;A;JAyYG}?uVa8R&I59WcS`0agoA>siwZ7qi<$w zn>5a`Sw=_fK5toaoje39VrdgKSI(AuWMAr&)gb)YpA8eIj^ zn~!v1nS6BH(tHS)$qzT<{s8-<_R7fPdt^!r*J3J`#^%@EBBao8DfoBpf4yZRU~%R7 zcHT?#aDTk}r&1FLNq7e!H&~?mliBn33LV91W@c_v&P;E;?c&Y)i(sfd_uF0W+{m(S z5lH^TBAH;<+$pPT#RMzAg$WymIFH{i8^gLrgZ0F_*jJDRy?@M<@zO95y!lxTtw<7s z93il4OW7#}JN6qb9dpHQUGK7KwRv#&tI2B@SHiypP-2aG(%<{LI-1;(B~Q{&ww4ZQ z0B#-L?EGZiS_lg84*B22#b~^nDjE4ycb<;HRdGc>mpBH<=qYTNk&tdxY4{+eO?_iz z<_girxEb_+g)G{0D&O~d-QdXyrYp~;y}kK(Nw<>@=XCfX4g3YZ#}-T%;3`!d7kA@~ z{x=Gnm#vYFoT)TOq&6Q;_tPm7kHA!sNVc0VmL;bs-ih#7*o$Ie>p znMhD|LY^mcml4kcWq{5*R*YM>lp#lO_uTcXuS@f#8vto581 zgo+b1jC+Xa`x3vb;{2S10@es>X&rkX)lqPd3W`2s;I?VvD$B;x(QtZtc|3+BJsX4o z6ikQ^dLwqW7_Rs(Y|tBn;EF+?*ex0-Vl7=eid-=@c%vZ<6+w7!3dYTEBRdhYsImrsG{ zY#g-IZOp5BDUj z^<9`}oe=jIWCB-&0Clt9Z6716huq)HZHi6j*?$xH$T-|haU!MixYw0M8nH_&hMjc1 zg88HYrR8rvUd1w31W6_+3B^@zAEmm5KoWOFJAD zT64$ZDBhVlSP6&dwm24X!~+6x5maPPP2$?BFR@%IqA1Acwn3}%m3@m&P_;Us^p9pL zIeGXAU*#KOMGO~tE1>pcSA8Jx)txGC0>~|`=I=+BYO|L$QfZ}?d8VO_XUttH5sc1r z=Txj42iZ&TIcl*fX9A#O?AZ-l#v72fVdU!yQRb1Z|4XDkc-vNS>b$;LXFvJK-nqSh zNG!JW{chpeJNeD9k9l}*4o6jvt|Q0PHP)$|TTgJu1E;w#o^LxJJYxY71Vfg%#-_TF z@`y!5EQf>ZRoEC>0oAy0DjGP)w|b+;9cMYO(h5B@(%6l^O2No)u#UgXS(CB#T)4?D zyKWZi5>uiDh2ofzV70$M;A?jip{v(K~pG=VYt+2NuM2SI zNh-zgh?@Qmf6tPTmk$Dxnpj2*k^=7UW&O+&RLQydr9m^pq?B3n0jREUNFC;^rf6H( z+V^vk(`Raw?*(k0d83oEr++qytTYV!@e{{72CR&N&yt!UE*Uv^BT70W z63vYbLl11S_21uAdox6&1i6}q1{cRuMk2I( zyC)zY?PSr;2ujQN0%?yp>7^@AjlfLUsB?+i|1YFyM6qU1fg$-ee-3eb{{ zmMB^9fPgN$ljbPW?6Vp@1N(2{$(=Y6p@AA>L@w=vLz4b1Z;Cc>#lJa087U-|3UrVP zs7;!}&UrF_FH&23X?8yA7 znNCr`$jR|pHhE?Aw=^H}ChYx8(DJ6mU5w!-8;l)80HQYUY>K~xge@`$&E1H8gDr8N z#s^*{f2%dzo>N^oX2}J71u|fa)gG)LR5hz>zouK(^-b+khS9u+Ou6Bl+}+%pgdc7i z*0v*I`?y0Hu05fuij?fXpxff^TIn(Iu}tN#!POL$mgLdA)%*qM?6;P6xDj%eurce5 zuM|9jLV)4Zi5e}F@V_swEj+W=&U}S_$&y-Ett4~@$|`RmU%a?scX0?Y8tg@pT3%Y3 znYMvSC|(b0NiNe&L~r)P>F>i4_pOXyDR@65kBYkq&_GjQce_SBWHQRdUgdm~WMp0L zKiuIgBaYGe3h2KWE?QrKGg{+At+{9xFQZZ!8445;*_jg0s(9oz@W} zjOuWaHyI9>)!G&aY;399BD%qrWX``{!rm}<*$49H@)!cE~E_x5j%{Xq>dm(_pbrFQDBc7mjH zL~IEPoR+9MkkUP!vFHHMtd4I(qBv(lD*9efk3f>#^{u`}*0ETpoj+d9zaXgyZZU+O zEHAB0Jo$yCjq%n{{o_QAJT_fek;n-e$$YjgE4vsRJ@P~1w+_z_+^7JZISRAF;4{3$0g+>u;5nWI#Z<(C$nJ<^Y zr!TKSlGyPKt<@etfUelLRUzw#E)j?N3UPkax+kHwdDIC>!FTPrOs zNt!xE2RUgP5wBGRr}S`kk8fy^#My6Nd{mZ}6w{)elBYla>P{>#{8;7TG&Yt2&CIet zUDTBx?J|C2BXWf54$M1rS~><`8}7b;D~8YtJ?XWNnKHo$@$q#p0PlG-m*mafxg4S9 z8!zoF37X0jmKzh5aCUN`9N!ofKc8N)m?l{TjQk34?d#C&C9FE;^a}{N&zhLJa=btH z@RY;9egZkgNkrCGuoUoWAJnkX5AqPYF@AP`;+ zTi(AG3@=Z6aUH&I+;zhAY5gv`Mh#!9?^wY@f~{&hK#f8ISds7z3cy_L)Z4gil2X7iTw7MUmeShkq}Td$tr<9Np)(9-X{^1?O|P~3 zL^}#M=5igs+zM)OYtI{Fsx3h@GuD3(Xg=f$Xr`r=CU%#$2s!JdoKy#0Ra1iMZxE>V ztN@INjubXzz#K+rNmQDEPx=}=22=n}>Mk%IGj*XIA9Ox&oh3ob_*R5kxYQV1Nj-(! zs52@(WF;d_b|wEs{Mw@U))+E~e8t?bs-IU)&q&Jj81nxJL0W5vlU@Q2WHGPyI{;W2 zK=Z=S)1s2N+LU;ivIzw z*p{V)olzH0hmk$8#M}aOn%ryy;d)XzJKZE7CW9mtgPuL7wF8px7aY?52g5M|*DAVs>I`jHx8}v}Mdu$G?Q! zYDyF==7C@PInR~^^uNxqC>&fK-R$2}HUD$3uiMRP?zK^K*{FZT|38|7Qy%PkZa3Nf z?WsrYl!iM^WS|SxhGBkQdmw_agSg=0lrjL_@3ocIPCLEUUADj~W}lN{UtcRk?p`2B zM5Az8G=L>hOF7B$&ov4Xuc-qry7jcQV6|sNbRfBc6oR!hpAG8pt?@$nAWc|;xS@m+ zhp_=M1*#V*Zz%^80P4d;Gr3VkKU-~!Kh^N+h!fd@n*J3y=tAa_dPslu2H~4G`B!CF zB_48wbZ`Ya$@TTNa0LLoCE+`4%a!F+@PnOp{J95X$=vz&*dR>IR=QbGPqXABsrLbm zC~*-`*H}{amV=G}4JeTEb#&AV5}v#<$Np*+0@R6dD;uj3~tWqV`nIVWajkU@DkmPA3x zAV7XIAdNM(PAV=)YxGEkphxT{_7t8$MXM&6EP4c|ADqi|LBaGx-SK0vJ1|(h?ODMxvRMj5uygONa>x39m)xNZ^})DRlTy+;8Yr z-*iM{^JGSA&@c~+c(Z=1HW{C?dL`!&EgDix#0XP73^ve5TWkBiXMgu;sN6&>I#0l` z@r&l}*O=Z_-RjjwC?q-N&^Ti$MdnD_n}lY@4TP8FMq7d$!7p6>6wH;J2w8m<&5SN7 zXIz!$jNet*)k5TpU%1OP)WvUucZ#_|pXAgB9E`Umd2Al+i#n;nA+$DF-h!q8=@*U;ozR!#(cI?Ub=iX_8aG<(1*|q18E==>EWphrZvu z9{cwV*_U`BBUh@Vp_Fab>k^EIM`i7nC|;uUvubk zg#4qxG|hoE*@Hvs>JU@(5)mRILPC4BITAfYTTNM0S&B~Hgxk^ik+?Sd&QD%xJtCsY z;g6%0Fm9eG2St$*_L=cdu(KfD;hBxMN@+o2%teYZwUT0mNs4Vp!#;8}g7UtduJ&Xn z558uV?xm82yH{-V&4U92swR1QqCfJd`NHDjP=RpR zQ@4_`@(sC((h+>wEX46?`!9W~kA zv&bCceO&KE-B|a=VU@7t*)BDSeb2-CA0-}>-Yd3oiGj= zKeE+ZoxpZi7*okLUn~AOWxpOF$SdPR6>VOaI(1uQ`I;iB*9oV(`z7Gv;`JHJuVgBI zupC8G1A{LW^LM?|N)zDsvRtQ)GD9SXZ?7L29?W~dgm`+jd(IU<)mwx>pqJ4R2d%q`0XB@m3g_uB-Ay6 zQ#_rwDpiT?tGt5)tXV`-XzcAR!N){n5NuId+<@seTFa$x6#vEtVnQ=^%AvQ&`f1}WuBl6!Y|ymS_XU(C zf0>F(ZkYJ4q$ng*(&Z|tuus|YGeGLoSQE{65~MfF_6F8-KV zuJ64Q#BY<`cXN*2qtEE2{~B_bunyN3{I_XsNX9PWOe$Ro2U)n1RJfvns)s6ki+~53 zR26YX>^}D}$de13V#j_f_U7~q05?_>@y!=VVVwWM0{qIK->MHf!uQG6eZxeB_p#Aw zNW?Ve9?ZBYXZ~y+Tjt9$-Tj;Vn_!6j&++XU!XRO_Py-3FR7D=7 zM&}dQo}`MJtNMn@HD za=EhmZ#%!l@r`72q->(6c4TQZw9O4(kMU%b(mUDY?YER^uEk4sq1hXaPGA8yBvf(@ zBsaMu25>a`p3!A`7e>2*4kC9bzo932whjx*)OK11jc@VH1{*Kp<7L&1(BWI_e-DPa zTj_p@7`^$eZg;SOfKZIcKs;-fe>KC6?|Ch0Mcgou(Q~fZ6g{a+PaY>ByfXpq3d&r@ z50DX62JB6Z4}wOpin4NlrD_!@j#k{zkT5hPhJ%Z1`Tw~A8k-B^WlfDf;imVc6nRuL4w> zJ<2Wa{%wbP+R)9y4pjNgg2G-!!H}|WOj#(7JOW!Tjun7y!QJZLcDVZSV43I%P@?Z% zIW%ugf7saOn=+21X==<{;=pe=mPboEUpuI%iYlV>#cW*-8!+Zu;^@_-1q6WcAq4>i zyuAUHm5!z5&>-?QWJRp;4!~LSSby%3mS~ZB>KgO-GmTW=Yy>`vR;1D84L91XsI>TJ|8YgX8n_HB~EV>gEv{s(tJN?olOffL$C zhXE@pidpq5+{fKAvDn9-to`(7Tq4)*p74Nfn&(ZrB>Qng@@kLF;nW|}n!d1uFqpA$ z9;ceG=S|M(g%Jly5MGG4V?{`f?p`!+@7~UGCE|e#w)&%Mst8|QLYlI~mSUGQbdmb zsans^Py?`gwSEP(4( zz0ht&bcfE(P#;>ome}`j?35;b&{x+b)BIy_)~8>8mloal5pURd*fL#Iq&ZAqG=-~? z5pR0pw;#E6_t0gXk&_cagdKyj`OFZAv1`~9eTlV(T_ z7CCpfut3_b2MT;352cGF7RrEk<9B*_UWfd&i346Z8mXDikUT+po}6M621oJ^g^XK@ zr#!1@jfmU1VZ8|pniQp2q-9Z7jVZ{({c@RXKDlRQAKr^yJia;QB;()6g+_G4qYn$I zZws5M<$Fbei6RNraAOtAK->4TalK}LmE6M zrC$soT9>yLXCPXMreTkztO}^F%afBw?!qbPV(_OUM|5k(nB0;#H%IDaOs`fP80=yk z&CUmj>hjcVhCoOdh>sCPGZZTb>73)=7FS!#_qh|S3`g)txS4S?0H50nDnbGhkLkM( zB28rV9>HZpX3hy&a{pd$nf7ci#;De8oMh( zryUJE<}&70s!We>Zn}Db3C7Mbv11(VM{!jv8@y2!5#+tFp!r76*=ETkZ;oGwC4R&r zqjD?l?xuRGy@@k>$#=NV<`UtX53b&lNgZd4){@RR?s36|62A#nhqAO+Gu|$`(v`P` z=DsBV@Zh&o#WS!1TG(V4yPWSf9I-Rr9_l;nm0QdN*|_h&boZ18Ch+qiVSRX`_;ILO z>+SEPwax!3V+xEb@LKutuDkefc$<_*OFC#qsM40|_co|(W5u1vnBq^E3qn4#x(j)mS)ogsQ9yzauX>$kl_lt(_sb!W(5ZVF}3Hy>$`KKQMnXmz$P^ z4_!(~)6oTp_}Ze|F`_Iiu1I)>fFoSEmnL?iBo2FZg!6^=*<20LyU|Bnp2gUH4y>u+c}3dljkP>k zGz<|(y-O~9l2-|`)m-NXiMT(;e@^|%CZ7RjE@2u|W==Q%%mU|nX}Qvo5!L?;_k$^D z?4avOz}|ncR#3`3Wt*8Ak7A9Pa!Nt-M_J~31LA2K->`L-^{h#T1fhu0GlUNB6#&&1 z^@vy(t`J)#TMoCflGrJlB_{%Fy1*|k=aa#$OTEva(dM%ES5{~-R@BJShV8Nchq<>5 zi?iq2M(^TIv7&|IUSP1n-QC^Y-L1Im;0#c_Kyiu}cc&D0_acK9`=S~Sit2=-vP@?ChJ(~Q^_M>#_s&Y}zS!Iq7MVrKSo~O( z%Q6JVVIP`o{K^f(mICn34n!XHyt#Gq&f%UvwPGzS3ir^@0zE8ys2#4fr^QER6WSV% zpNO!7V(onHbH#8avyLv7>GItO*goUb1%=J@eAEHaKoBT=}|ESdW_Y|TNwYwo*X10i%qVh%nTYQ${EncWt+Iz4iq7+9;idjK)BT~;j&tC{B=>d%mE=!sSbJq1k z0Xg|pQsgcbwWr2E8!PEaNkb{bre_s0qz3ov&2~>&7yXkI{Eq3{@3~ZY(`zFW3uG2m zBV8UWvZ!@<`q1!uw!`5x^L*@uZlB3WEihY^-zzl1r@8U}x`V#%mhzP&2Kc z&yx=snN$j=>nICt8$Vej0}cUE@d8f2Nkb6>Af+c}=IhSoeR8nyv>ed9+fn1voy)b` z&g?i&9OFz|#*L+A7V@waG~RE!dH(Ly$x2EB9vP==kbg(>-I>HDn)K-Y4|pwX1;iNg zm{HDHo#yY^Qxr7wBVO#M-juvDF_T4+yi43~_u1?8vQp9qYDx#x0*bSVHMy>qT88Cx zy1Cf_Hl#60Zw31+5e-WyfWuj(kW2789JPj>C=DJNR5gvYfk1N_^Fb_})2#_!vf!xX zdR%;JHlm!{8{;T1l|1v$<*jMo42VWtX5?jSXO?Hi3z9-*=v9W-MGULXV84zJC(`h8 z^heyjm9UxV4~y`wqV8U7BtRJ_O7mMOhlx=&Biu?|nqBM>#H6e5kL$%%MFYRZKwMgy z7DbEAi$zNS*-u1qTZU6@t zZ@IO40?DRB-`iJy>Ep3yn7Q7%97`P>xDYHFiB=1_{U!31a?r(wh71YQv(9Qm#6Ft6 zwvY{%uosUwUSwZDlXgUsvNEUb6HpP0q;Lp_ikm+DP9_Dpd3K~H!O`CYsri@XO)0{l zzBlaA4~Fg`S{HN0eY=fJnL_GeyLL_X{>a17)1U*wTiroORk_dC4Rn#o!0K(edrKYu z%N|{fubWhJVRPojPG;OA0FSR!&;;>&r>(Xn!PfBW-BbaN#(xbA*7feqkmzZ!8#i* zebW!j3Kwt^8Y;x!FuCVMZ8=(j94$R_1^K(cNA4$3q?sMJC)H?opQ*Xl5%iMy-jFwW z$7tWjpGZ3kOF6*TbpA$bDkfv=Yl`)n=U{s}F|tyXHDyuKg*m-)sVbbY9GrE0Pb#=g+mu6|#>g+vnsxG~MgaE56eWFqr z-(7tx^kz!8M*Fj;>go0V%`Xm!*h;_u#5^BvQ=9(M^A?B(rr;QUNt?QAr<@rkDGEO% zic#jeGet2)^VblkF#JskxV%&$@JTdvOR~P5{=tP*!iEIK@?{m)^Fn;YY}|6+rSMzU z{%*=cOsn6Qo3T5+J}085fT(mcmqcJzROZ1vmhNP_W#9$i?dgsrG=pUNX^>_&M=Cgd z?;Ul}Xe=*T^$lR-n#7n7GtsYSC2rPL9H%*Svq*QsYy4RMLa@jZMEDo$BqLlvqCUjODf$6-(%B@AUd`NOwmaDX1o7HML%G#NQCsBx8@j!CQBQM854ZHx> za11)z9r;hn=}z7L7IA-ZG0lDVD^pimR0;O(>~7No*iu22Y-gt7xXwMiK#QmPvnvuW zUJwQgl^wOAwdg~C?jwSWrR8M9szm;SeeD^OI{hPAdbbFUCTWICNScyQ=z@nj0keaA_%ss;5diGDD+S1pg)QPG_E&hCRrWUNN-4 zRf3lP=j5{0OSfVGF8&nrkWA($IGTphM{^m;bPt#+aEBfAaFe7|H1yj!&UYSbKK_3BRc z*hdeBRwcH^G2e{yGp?n6$nCM~H4NfH*5DpHoKXR_M}m@2J7pR^;hUiJ}=b9|MnRJ<|IEsSz7#q3GBUoM*c zttGUkCw^P3iXY;z@oP=8lChxXdF3yEvvKm3nUQpBCxWw@qQo3T$vh8%|q2cxYbDb z=lHDmC@hqyZ7CucM5Pl^D+ztg$V(VHkT8oU7RN}Gqd~r?(l^MNV{?dksZnr(C7K;$ z%SwJ3WR4t4g@qLM7HCGX#Y#Rhd6)%h*{(QgIm+4CxvdVM`9;A)j|^gf=!|H~ zkZDK;x`x44X4G}@#gSglG&-RowxOHjrYmRX3fG>oKHBjj+k)SMG=;$id+ew-+6^cK zI^g)#uKajBL;@u1Fa*A(b}yCyU$k(_v0-;8R|BuG*wots;Ur-NzJk+{Rqr*0RsWUu zt6bc%xiuZ#)sDGc8K(6qNh!f=RFuvU#YykqNA91Lm9=KQtiLn&BRN9J4^G1diFXNw zSWV062~$Fk2Iwx!<0?j38wT?+yj1X~@A*}$Jb#Qce20~?L2h=TDo#4AznChRU04`9 zLDciH(L>Zbq1eTzVU3*%Wsi_SH43J3xjdwD zWRAAzn-qAOdeJmo-WaWlZGx~6^YLtNF-rY?LK^$-a@O#B8a|nkxY3}+YP~ZTy=T_P zxlh7&n2kEmjI3b~QEOYKJ;(Wi5wO}lB^q=((c2QzCXV>0| zwPtcCW|HEwO*IjL2_Tn#^JF7}#N@<6HfLwNApZx}60L;Y!)gbT8`6@slfD)^zQ_>@ z#yP0}c3Lb&U|qhQi*IOZr^T@Gu;8SOTh#q2Mkdnu0}LHW*$Gh_jfpxgRkWGA|91uZ zfD@Oz6|24FnvMQ@ef;H5n4ov$1V}}YAa`_yg*rm&-+Fdm%1=IYvsHu_dlMEh;ao&M zuqw@U<>_*|@1dLm3MP@s6+CV=*Sed3q)FthYPsdIvgG{GHb;E}w^jNXa%+>z<^SG~-G zz?f!V_2R|k)kbF&rOsy$E*-0;* zONeRx7MF;V11jS0He^Tdj3PMVUA#E#~X;@0fLZ1cU++=y_vj z_SCXC{Q!!@W$a(R0BG-jw_m`8kgPs&ogpa3(0`aG5B<7rvp%sNqv-3$XJhYJYP;YF z21#Pg10_W?n1^HfU9hoJM&d+NfQp(LR63FiK8YeCY19c2Ki|uH|Lb$@1goxIM3MnC zH~PbrkQ&He8fjoK z&8bjQ*!`(MrRk1;UE1-19K24lvp7%&-2SChbGqAm{+b#KeICT^){FfD(ZXyH zJm}&Pni@Q?jG_oS1zrs$Segeib@^En@(tl9d7IPo`9YvyJTYAqci-tqYKJpfkdH65 zP)mQ*_N3XCu`)H4Uk?mG5c#3MifpXegc-x-577~9Br}ATXVC!;aCGMNXCYf zq=rIo`^Wrv5&Zo%%K!K{{>cIkAq$x7#Vmb#Ye*U?vDiU-;PPm5X2A=3R2{*2jnabvlWo)Ms(aWHYXFA(NYh@ z2RFQ+m4nlZnqy!88luoN$X(4#mpDDku(T;lE=#~#Yq<%3h_ zN$IEC{>eIF_U3~CENS7@>7g$}5xXw1)`D^<-0y_o7_-4{j1&Oq`}+J4)UbGR0E~Uc zez_XFlRhXqgyW3fy*7V({${_fjhaku+b#fyd1{`)9@D?7ZjW+6K!(1;RlA`&>hQ{& zyW_q##5hJAJ=%ekQ}I?Pfc%x`x9G7QhTE6Y2k6uO8UwK8_gaUnu~7O9BC4Pld1Wnf^L?G3wd7A{6oizUe2;yz*_m=}d> zOQEUcewsvdXO5#^&5v9I5YN^YGDy>Io56|6by`u;iPzC$j%Zz2W3I9TFULu^M|QH>{@o;^ zZ<)tM%<{t{LnOD#dK$4{P3aUBlUf^Y7XAn9HTVoj?iekWm6lIFHCK2?%Ll((Znu$I zZQ&w|JFaYI>Rrnc?1l@^t9^9rEdn*%tSlEe)HwUqfXq&no#^yFcPM|LlxH$lj>rpl zXAf@foj8nr5JV*%GqwJr?WkYxY4x^!V`~dYsHe`L;!l!x-#elY;>0jG5D0y|cXMEb z*V^cQSfYW3l`rd|8IlrK-X{=FU-Uy zi|6_|ag0w^Mxi{4&OKGeG}rT}4oZs~hx44%*ttwMZD73qNoMjY%#^V^kgVm8E;Ny= zkYuLL?m|D}9eSeOy9XBXflKm-s9V8_tj5&IImv+V=o*}w0 z-Db$@F>Fgp=h(eFVm;g;IjjyQ?IcyGVD~SFM;97H=zJ*{ zA)i@Ym5owI6Ta&)Ac3KAbpIwvj|xE)>48>XpmapcS3gzx5ydWi=F8&U*+y+6xgHKhv5*jy-g%wwIA9Ew^iv} zj_jAU2P6h?>SR8{w`*VBQ-+OIe<*PjV6FqT9_-7>07k8Q0u48uS?Kyc0bNnXnbuV# ztb9-|Y2x#mY;7x7qTo-<`=m+ygP%DkQmHIWQr&$qs`s*}(Z?SK@&0V~)##}9Qa5iq zM%M?CkdIKTNpH1&^`u;KZlC(o=qfQHE+aap4w>y&_87aT7%*Lq+#XRR30?bZP3pF;j5`K(?1zkl>`-l8iols$mQ7-aGk!Z` z1>2sHWJt7{bSQJj<6tG-KvC<5m2MYAIMen4+&&k+j@oLik6bg5>H**U>TY4%BT?9$dM((*8 zEKldo6?t`i-1Ptt_glYcc0KYRN^4u1Fy-OJ?P@YHw{9;bRm2D zA!r9oU=jUJP?rjOx9a`bIA7PN84P3?(!)g7pJXDkT!1%=+`DFaX56iPsP_g5@&;Z) z?QQ=4XKT~rzaHTf^KZry?ORqOp~m`?ivSvdRbPUZkmS^DKgCFYfJpSNiT;ujzV z+xfi9A}8ia#`p2|NJK)LjY0NA&8yDC4oK^5uc+lX@zmtdOaByK8K|hSzT}Dy#{~8+R=4=tIW2p?FGkkl@-_HFw_rTEKgADCGKk zHqzTjKaj_*RLOD|KBMk<{h;;rr4M)Laj|iWRU%`xx7M8aEGKWbT*HGZ4XE?&5KB|0 zgVqwrtuK&Ph&=mc@s*(Kw-`Q7gZZufO03&&&Qx0Mx-XtF+duAEpF0@o4DYN0AIG4# zu^mWO>(N`EbW8CDOaY&wK~$BM?Mx zw`JS?P_X|NfxE$xo>Fp;q&P|7fa_y*TTa*0HHM+$6!nBdX;)R=pdy)t(c_yvN7`i6 zeHD*umtQaWf|HEVWzWj+k1K2X5T^sI$}g!CgcF5QucPPRbnDS_?S+oXx#JUyGr@6u zuN1d}R}?{4KUGg}8|)y40LxW*mw8que9VL3gY&XxOTdE)DFfr33ElV9Q#zcUglGPz z$PHHDOXSl^hbh%MtyzOafv5d;;@x}W-b~Z$NlN=QTK022O14Ur8I>6lZ3qTq5xL8*l>JowTt`XiTD>zbFJVnh+Ife9G^j9V8wjlRV?-p5Z!x2^ z&Ys*5t1ouD^uVusR!W2zb*)Z>`QuMKax2ruZ=SU3P|<}GXm$1#B_a@H6)QIcAwDK# z*ILZ~SEFfjYs-$luiPD-#hUma3*mF!G&{D{xBui=jh zSoJX1#j}N8kG?NsEl%7yK03KSsd5M`m(VzKAF|e?8fMGcqdd8C`#xU1bf1elj*ybNRVT?{dFNIOz3q>5C-ShcC`*q zT$VsiZcsq>>$3q%Q6!J`l17h-1(A@Ba?DE7K=0Ax{YieHh}wfSHoCy+YFoxNYUjXD zea5nB9)-STp{p?stj?f=TwL)zmk~T_{`=AVi=SyAQN$I`dA8#pIfQv6v8Fj<4=;$` z7oC2Dd~F0(-(t?B+^e4_A6g7N_w$6JBl82J{CpFXcQ2IdzQ>p>toJMuAO|-j<==45 z#Z3|_PNs!trkGk+uJ=0%#v;{R!`++2B|q_J$rnu~V30*Vm_D@Y0jK9iA7)VLFKwyp z6zpv;pg8T>t0a+gd5VBq+vliqza|@hbZu8+SkR9oK@=bvU$@_`6dQbp9|u_fSJ>0p^TZba8pu(9oShknPXtueW+oH(wcT2ss8Mwlg3b4>5qhuziI42XvH+~@9Q;J(* zP_9|*0|j<>tQ8nraR#xScE1+cVZoTBrFJlFpV1_M>__4V@+7}}&(=%~4k_2FzX(zN z3NUt8(@l>AEK2J6Aqt97;2U_s;z0E2un(w~Hj9ma!l9_@)9EG&i}$1%_O3GG1A^?| zRWOn3;n2NT31glX4Nzm*Z=BKpc0nXbo%06SPZrCE4Uk*q0C(my-et;lM3Z$NC%U?i zlou7Bl7g*oDV*XmXr;=}lqReNJMqjg&``adLO%eUK-gxFr62X2URajTSPcP<4Lx-}e6(!dr`Ra}*|N24=l5*-@NgH>M#vjB0cH4QF zkEHpLJ)0DKjD(!qeFPa2ftDmRgcJoI+IZ4U;B6M;P!3HZL>Y9#d0No1C}KbzPxgi5 z3?EDqSUz{qMLP#ERv8!;y3x2!ctA$H2y3bFRSE2h+rMZ1`V<@@li@lyjxC=8iuW_PfqnD>)Q18#%lRhtiz)7OG9e3!`U`;P_$1aoH>W=zqnC$mEwS zeXqU}+yR}W`--I6HdR8+%;5PI@v+az_0U@>L9T2VMRDC(l;tg|t_10OrOhS5zdom7 zHPo2*Tw}io-EP7uPqhhNJm0-Cop*wt!C+>qu@To4FcpxhcN{^~<%cIw_ zWw}TC-`7fW37woya+@?As^;Foi^P10`ox4MDtf!Sd-~Y<{qc9lR1gwY%3oL^ACMmc z1O-=wmrQuM>_7A9nwr`ZRm?ljK5mSa;=RHBoJ&G~W;O9!K^>9*XVsdd9_z$zKb1^F zUggF^;CfRlsh-O`{eJzut*(s|L*F)Go49>NqWT<7Q#Df`j)O7eH-@ZSQQU;p7ac$N zuG>zdLVEMyE7fL4)Z9I48rs2Vyiog&{k4wsY!5gzMGc0e8I^gDn*J%5fom8XIUU}4 zrxIRc)uoPvN#2}?Xk^ohIJu1%((1L&n9t~%pF=oL=GDq^&=kf^)ZLxe+Bf>B&(a7Y zWOR0Hxjea&z@#SRWS<|G@2;xl_-Yjf>_(W)>EVfWW_98bdl)(R_~e3uglK1H`}*Be zQH)Zz^~t=1T5|knw#?Dd(ami&0oe@B^`Zc|(Tg9}45g+vj=94ZqI#7fPqjhmd-}=v zYzm`|eaS}zkN9pK(~)g<4Bu zG2y}a5X7DPr>jQMcAWr@KTA$gX+;RLN^D-8$u0h9OZB&u-6uLD9&O`6Vq+<>Pp14m z0U-D$cXbs5>8nIao%J7veF%(0M(5eaZ18L7dq0;X0)GgN-2IlbPTX$eoN&*ifszUN z2ig(3LB^i!rufxtBJp*N@OGIe_P_}iI+>*-V}WWv-X;fITiaTu z*oByi>gstYa|(Cfd^X6Y$|vFPgl8$>Zf)rHY{iGUSe2e+h#WiGnRmbxD}QQef_q~e zzw-hf&dT1#m|KY-Z>RT8Uo0(rlSwmkyZZH9gvZ zB@LCb9t=r+tXV2+g2)_9;G7H}@6Hd_{m%E~9X2|1!>bv+-DyU$ephiqBai1jqcK|n z)r5X;BD@@|>jBmF)?GWjkMKv#0Z;sSc?y@E?THtvqR=~Ugr3a4ORKmg%;trkRLO_Lc3igML+2|1ZIzqKuN1JDwF zJgH|t?%3qvD_sin54gQUXyCVc9VU77QOpxCb35G%OR4nf2qJp(?hmQby@XxcAx-vM zFlXyd+>EgQZtcI>bw0f9 zcvnaodkYXc2>t9(-^SmalNSD;AOE570!vCsA(Gv9BLdZurNd0SP37o%vnuXdo@yBN zMtDD3Rm|VILfUe8)Y46o$M)`hI3vD4-In^kIBwtuj1fYsS6wu_>g43qU(oFN-^*tu zpdvuKveocongm;lX(VY?QvT?vguYA_Vo5A-sjdpK(;J=X@S*QkMiIL#wfMi8`2HnQ z$U+PZlNdR$jPO5b$NtSf1a5$z?b#ArPd>-h1cOtxn29?)eE~b{gdb zb7*qBrbqh^FgGE(bdxxa!Ri%ZLfb>Tgl*Ccd4-#{iWb|b2$z1$xU@5WLOa=4NA2^_ zyf_ndIXK6i?2Z&S;OfU)JR%|ZXGUxfaD>Z;s8{zx&~ThS$WNO&%d_RvfS8qkpd_*g zcOUd9|68m{_GRjP1<)kkWq2VRKMn0{2?dleY3SzdM@SJw=f@HWb;bc*s@)`Q#gT07%t_j2(TNA~lV&GyAr+@4MKl4}mk^;;HxXJat92$|b{>;1Vu zB7UQmiJIx{;m)e(kN`>6sHv|#(frw_*18L4rEMrdN9&PkM4f;aWFd%s>OAm-z40Q> zoy_Rj$;i|QFyt1~N+ma@61XSy`i88v0!Q!iPt=I^pjWNc%WE|9{P^um-@rXLZnxRr zL@0GKo-cJ%_k-%p=IX3jWw z4<1{e=bbLZr?9nSocrE6`W`6gPa8&8{jmjqbPgACxioPjlku4j;-)x^-u*C{4@VA+FVDN&@S%- zHg;CL1Y^od;t#@zK5mBQ+Z7MHDEcNup8pn9OXE9aYW6AU-^R5uBJR4JIkCCcAt^!= znh5+L1m!Y&!Qqq3x$iFfSTq__clXQb>L$o9EMjV1IyVCZL=$>p{rk%$UXE8?=Sf2&B9De+@D}m}C)txgwZ6?I zQ(vjS1TG!x#k7#y>phC#lO9sDB&1%?>pk9hs>e#H`o7 zkG5zDZf;Kb{UJ8= zP?u-9+H8ISpQAUqUg8<>3k*U|jnVDq7cQ920KY-7OoZ_rk6Jaw^H#1E6m<_g^^D6c z-hGmz9W=CZXy`l1Sne4EvYt)zY<=Hg3--~3UlZ=lk~}(rCaDI`a{A8CkFEY{ zeCZ*0b7V!oLV$CmhW0k;>#8{CZO-h(72d%d4eXGg_5oxW4)#N^+8JLh4!RN=!0y)Z z-v2QHyfhtGqmBbYLUGRbk1auF;A?=C)%MRS2jlX59GJPUC(FX&<-ox+Ym=ktkGE4S z$t)t>(24WJG?V!9CV3u-N-Fc=xJyM}_DYt{h2{TkjR-pq>Lq>!Mnq*hbz^>+@~?=87}Z%SlaO?CaBByUwt!Dk=y#E0eJPs^otQ%HIU#KjL&kq%osI#9Vzs z>cT>UF5d>-Z8+`Yki9CrY&!iipNVz5-<5WJ@jd{rYm zD9V3f@qe5tqCzD?s`p)?Zs9xMULV4k_VdOU^56VxkC>&feDvQudz&5Roc5MO12JUJ zyR1AD4cKn5O#MtvUv0bXoJBpOdpZq${rry3&MeMBn00*bjAzs5!W^I;#+2#H@wuvp z3TIYsb26%Z$&t`^RFKx->XQ$DfeZ?cP;fRevV9BoKsg7CW%|O8fIdDOFE6*d0s{O~ zZ(^sV)sSf63*TC(N`)*WzmKXvx;o*p6mcP;g|YuIT+dn9d*ta*7Bg5Hr->JyFV{k8Tq9E7tFc;S|6HPmvR;B>jS zLh)qO0 z{PbajwwA_{cPpC;RnJ#mZt|s(v(7raSFCqv;`h^JPDSIJI-^%V+H4TLbuV(V9|n9% zQK#2^k-58Wpv0w5QBE5#Q-_;YEvvFg%U(%{kiFIGUrJZE!r8Rv?P)>xL>w^8JV#z9+gJaKCa(Krnr#sBgrH7G}3MCNP~|wLBl# z$C)C0hQ8(g>HmM3wgW~?yJq{H-s(Gf92z=3hcm#tT_fqeo6 zUmwzT2=DRO_s?&jKAyMDo;We_bREyt2H8LbiIhQG_J^d+y|3Lk(c6HbzjO7ZuVkYhuF;@G3xz2@XrcRZUMajm>#={Kfcwvbru_`|2^GV#xs_K zO6Y<149)s)#;zW=<&pY5hjWXxuHZTJJpz<$Vwkv>7o&^NXSI$wcmv!0Y|JMB{`~Jx zo(vN_(IM=eNH+i)0JkVjPQ+PU?kI-}1MTRYI8U%Q<}#uHcru=N=b_*|PD7 zHwLYipBkGvu^RChi3_PDJBRP``9}vbLi8%OLr$DFnep9?>8+8hj-H>my-ly5JM0v2 zxI0h8NVEzq`_TQ{My3WmCcJSQo2!a}u9-nW{K(5j2uwu>k8fT} z#^bNH`0R>A=b2GDlz&#B9v`|z&o?(OEVJxQ@=_Tin+1VHo*fz1r)-LKrhHw`Sw_^h zbiyE=%uS&{Z|;rXhCCi*-;*Pv;-RA@!>nE4>Rjrq)62{hTK1+LmSEGu-E8rb-d}?= zB^0A%67azsQ?nhJHlEmxa^tPkvg1i!_JtmX{BJ&l3SSVh|CwUksWgJMv1^d{j=dk6 z{4p7&Orw=K*5%R0ba@sLw%W7C4IDUqdPhAg%hGB;w|ROhh+q_B^e6Xlm&rj38ws;||PGu#{#SKWaLZOk2e$FVl$m8KG7lAfn6fB7Z?vWHy+$0ieNooa`h z#<(`M!09ZiVs>Wl1NB_uwon`D*<=GvNVzv@e`<<|c@T(QGCAq+>}dv@xAoI=co#Ar z+L!kg0Tq30?5il#h-kT%r^NbboZ63c=Q$QsHQG}gw4Sh;eZlLUIx+v@7OUk6QyRpl zU7wRt2zq>4+o@b?Y$FN9zeuaAFNtrO=UUcX(doKTt^RI{I%$l#Q^x^aN*b9`O`K6I z3VS6ub{WN;%EgtL&~YGD7Y|K4I2ggif`T;?p>7||7JyQaq$ zUbnk0FfnhH-F-CKxJI|I6d)CzMA^s4C4+}KW3FHtHUWh}EfJsoKKiR2|3@XKExW`C z)m>m&5=|EK_MMKlAxkNU62$V^C&H6{*9|u{xCHlrC30ZIWMAN${}$mp5<<${d# z!;ULBr?kc^E@t_gx}?Jsu1qn+I>9ZMX^BERJ^Igd{bMZ>u%0)Y`p|gCFS) zU3tz#2FBdz_W`&HIAg5PTxFc$AjMB6CU1qa-q7{a{o2Dj9AdUrey1FviolB_HRP?6 zz$aU3Y|IE7N|&xe#hG4|DgMFxz4o6i1#x+%xh;i^1N3=^;$A{isYZcTv@a;3%xgFPCy^UlFNc0Qdg*2d-|tdv*Lq6v zXo_iC?xYg258Hy@S28SmyKs92?M1Hs?jx!`Wio3zASXRUcippUQr5s+aIA0HnRmht z7)ooD}u_~ z-rJL)_h*5TPCS=^Vg(xs3H+U@JEYU^{;qj<>f(+ffl&f0)5^o67aN z++%rpyBA+;J2vNk_`CAUtsb~pR^pvmLu%oTw?izbb$|)BJ!m)xj$gmAHMW$Q`+`^| zW98Q?Z;)>W&5y+ikMl>5k4JQ3Ix!9l0e@Bf+MEn}VEpCY@$+kYs*?YG#+SO_JsncN z3sMF#zQsBQ(X;(#NI7OVx8oxpDT_8-rEcplv#yB*&NKoV{ogNBNe{ocThE+c5ee;& z`B=8s$N~|GwZfGD!`PSwAN$V3iScXBPp11~ve@|I2feY&;KcfqaQUX-8ZPSd6J^kZGtM!A-WRp^m%JxdU)NUnZI*?5~YXAGE$!? zAiE8Haw(Cz?V}43ow2K?uAeAi{r!QZ5O-=e8z05~dsf%W568*q~TmddiVd}Q~AHfQ6g+fK?4O3KO)aX;UuTw?e z{O6*rNIx(F%_B;WD^!Au60Z6O)l;xDgVku=n2m!&Nkv5rdr7M3KPb&VUc|qh2^;VE>b?fg z_calH?8lzYr6}E;3yl^0&2-eeloo=1=%<#a)<|+CXCl9#H%lJnys)LzWq9-2W!Y=79amzP`nO5#pa> z{=b*yzqRe}c7)*k|GNACUu*Ke*8EZGd~I`o*KF@nq|t61y0Y?bS%8`zHr{+=-;ZKG zYVQLlQ{_Gi4vCV>2WY$%yX+980DRzNn4GOo$R$r`8uJZY2f~z$B1wr8*MuBE-ai;ga;Bi^dpJs0Bsb_( zUPy`t2P~h(wKWRmX=AKS+y16S(EyxCsVJ!Hr4r=pS@e%isMdUvb@}z9n8xoYR|djg z!pcOJcEe~>jdJDcE;xCwpVwAO#1FQU?YE*O$niQnwC$_Gtw~#5eG4ncRb`ir;T@SpfsEH&e5F3t@Amf zBIM#E6H$EHBEUAZ2(s91H%E#y=jcrhAQC;s!vSIBLHrET4fE1MKhNsMhmlX zW;{BlFFiv`;vho;OJjG!NHs!IArFPKpJ_fS9my#2H<(GtT+l<1t@zS43OPsvL67dJ z7}r6JzHLmw3*q>&CofT9W=#H%-JMt_e0he-u{w|F{vR+tjN%p?Q9UdPARSZQ((=lh z{{8qpDq0@i>{3SyMh%l392EKMThPtIk7fPvfy&}oCV2&u)S6<7Pn)MjYy7A?Me;0V zbm<%sXGMig$#`=r{0N)xq9;BoGgW?GKHsyN-Nj)ax9sCshPG=RSgY25zw6aAz402nnpW;8NkLR0XV;~RMXBgcQy}cEdfIR zFYceUG=hb39uQK7p_nw=af{*-@#sgO$Mo07BPfPo#*WKzz=ggP&K5CLsvSoy4XxT# zPCg!^*6BcrK>J4rcwolxx8y#Z>qpm-{(ZvtIVGPtYf4P*BW9v`T*0vJ$@foily3BB zvx7|z_A3#lF?F5Kr|TPIuFG5T#cL#LW@b#(5OBO~E7k0d8IvW6p&ntW)KI1p5ZnNW zg$lshp82-`{o0wN2N&yQi|mc!G7xr)^n*hIUg7#f=Z{h7WLt;#Fg}xlt@j+1AXMB& zXc%4S`C>ULll*nV!BH8Z9@E7jLp=xQ;7`+thF~@D{&0^;`YiP%p)M$!pxDVds_(nl zE*KlSBrkQsqD>6a!6*t2O(%ni_)>$HA0DyQ!{lqm;%-7sK#k;h>Jcda=j+H0B zd+Q)6Z?ij@+>{5-s)zyyG0*hi#cOkaE$MQ_lyJvhq~-je$E{3Q%QL7D4;fCpW8+ts((wx`(N3|Sw6I}Q zWF;EpB2m4h;wcN8H(nMDDR?Wm(mm2Iz1ySwaljP0#H}Bj&~+aWI~F*~NV!OIfUZ@) z)msTHO;vE-9g))cYB(ifM|_#Z-$#xor_35E94${Ff#0(FV8~e% z*(3luy(cQ>4L;CoqQ;p?TFFrZ^r_uf2+@*HnX+#_hRer@^G3$@Ha?3Ni7=G_^)iEH z=be@$p%!I_A@Re!-V33B+MV{B6FfXnIIna`1CtEze8{IM*!gFB@^D8b085{~XA&hiu)xS= z2H@)?iLD{_H#JL^C6oBQnbX*N4I2{RuLjHXovf9hPmv?xn(HqmNHJ)LAx4^F1WTH_ zuNNDDe!WaRz@~M5gfSwEgdJ%96e1?N6a;w|r$8lTbYljfkIn{i{W4ZlX0P1erp4~k zByY+XL5Hol~979z_cQT>xbBq$-} zSXC=Zcc{^Y5Dj*p(X+XG;}xRV63uex)uM7`jOL_3%aC|b(M&+D!#;M592%p!f?0L& zMwc;BYP4R{)Et>zHgixaA4PY=8+cCzUr}DlRLnS+>yJ>UnSmW-RHSEWV*_$l7%kcM zcQh(G%-Mc)TQA(**)G=$(w4OnM&Lj0z{S`m(?Kdhy3!Dd_` z#+MZ$l)|i^W5eK@G|3ZX<3@O?`}*QpvBJkSgh4^*>LvPTj(qZ+K#s zmvc`_vDWxtzeoQ!up>t)S#Qwq-y^26fb@hv|R8-TTjGsW!)FQL`gLaRPQBSj7 zO3#Qeca0hvRD%d(7PgFWrHrLY2N)z#}$|j2) z-)_-M;mECH^`uO2pxHM6Y&%V!JgVr>a}NU_4Ars>EptV_Trn^U@)IHtu*S`hlhx(6 zF8^H;`Im2KV(N0(i#IhRwXmrUK6?SfK`QQuk3!YZ4JtrHn6C(uy-*~|h*oaKzNY~` zKm@v-10gbgEJoIpCP@o0Cc});d%!YN$QSAg+oz0yAP!hj;KJ*rE(oP!ndOl#4& zX$NJg@k!0t3*)-_p^`x$hhfiR%)+Ck@>QweBn1Q6!-jU%ZCGUZV)N8yspr7`+Sk=*LDYj$r^1Q?AGo6(HfT5ByZD9^>@eNO^ z^b>!Acp1j}gz*>}3N&Dz5%9Gk( z+g>7)q%|{8(=L`r>cl9_s?_}hs|T2nBN;10A;n86Tn?nw6r&3lBA1@dA1;ie9|G+H zYg#f2DA9U3Fc17hIHXuJ^LH%(D>BpsJ*G5KBgIKlCWKmuMQ0QQp_}&g^h^x#llxNi z3~>MnI5%qyzj|@2gs?ta2w#IEx@^mpE*Zsgn1~Jl+LY$sS%H6QQRqV00ZuOLLZw{n{WFkBj;%I*(gEYA&1{W0*M3ldGT`E<L}zWGf7(v5Z6vr70;E%k%@Pwy8+^3 z;nrzO5Ro$6rV{a5IL3=v<|?|*q6e$vfILbh-|NH9($j(C z6E!Kc-)WIpPbbi8la6qy9ro^$Q5cI-aQH@NYQ+K3pNnJq5aws+*CxX3(qjACaOFSK zu(9oG4;Qcn;4Rh~dwoVlHYGlb8^fk<-F$eVEuX$eob`nr zNGqSHI0n|KkkQvXzm#-{-SD*Q zAf(@I8sEp}PB*+#E{o{QWQmO*Qi9LG-VLo0lv|Peg9Euv4`y;xJ^T99Ru8V$7?eDb zCuW64b8KRw!1a~-GZLX5ITA#6VDqZvf<64@{x+BR^#Gbc7qPS(rH2cPg_GU#zB6Q# zpGd@dN1Iw$MIN7k97ndPgWab&#(b8xejx=WCG*QvMz^7YBI9o-s+@WSScig6F*C04 z)s@40+O%m4b>?2^0`e+k0{s2&eWJSn&bitXwNJyt3U6 zG{r?K)SVc<-oPf`s)O$-ybPc;X^)Nt&q5HM>?NU5Vy{#-R&==}f-sJLgN6I_NE&(7b z*Ks`Zwx-z!c5Fu!+aW>Zo0E%e5u=q|oq=ai-)AJ2_#dtea+IGVGuJ9kN_xQYf9C0- zO=ZaH#k`m)^wJZfiDM4h*D-?i*7%m{xKHZRHlbpJg`JLHPw?digAazW=47K0w?czL zpZhY-5M;j6C%if}HN+D0ckf=3FF(t_3RyU(q#8wOnWTBIh+2Cmsa5^r=JvZzP(1p= z!apl|F%0>)X=VVDR!z=y(LJr*TmtFbs26dQ28-s1Y}rpr)RN8JKFqaKj50pIqBVR1 z|3n>!XSkCl7}FOrG*6f&nW`qwzpAm4%MpE?(!pp=JxD?%V4(r^EOzMoVQ== zcfb^=;0t3T-D2JEj@a+^Sx{{cHENfejqJnTZrZR9GX}tg_Jnl&~sNH9~Px zu5fCMBIDd|IiJQPf6n3kuHXtoyO}fWzu}IECFPTV?$Ru{nAWFM!t1IWE}CoU!L(ZW z?K`^)2p=IxJ;m`ige|w+JSu)M?|kLq`N~JfX&;iQd9axTs89063WX_g{;*f~lCc_7X5 z;*m8prTz-soEVdmt@Uw1SW)JiswuJ182f9&G?Z`+LuScQTQyFJj;}ZMZjXhBM_mnc z*gYiA3i|2P{KIe;{g!xXB98Xp;K%6R^n|sQmU=Kg{=w!J(esxxtp+zbqK@%c{?E7s zx&;h}#s$sEd)?Aa)@r^})UlbwMlQO@J#~10X#tZuq-A5)lptIOZXK1GZzWnYBW0MR z3FfR`MhQ+izUPxBek!4RSKdgsh^+NhnTc|ArJ354N*!5035BsFlei~IIXGpf>@o%s z)6Hm+L8A;a%^xI^?sPOO8j~yovPogq>E>{MF_NC$wl|sQ5aq-yIx-L)#&%9OFoz7U z0g>@?O0>3yN{$%O#)_KfhOQQAT9!D(F&SwEht+w;)>LkWzPq4+syjllI=0hO+`*Nm zz`I6Ldd{idL;Yv>VEEvL7#spJbPN5J-x5}P%wDe8`8!H~7Q|@kxqpm^!}K;-pB9X> z+Q;<~5XjZ$8GeoYCcG-*7yFg}_Ex^ez>os{jei_8wqVWhXq{ZR)ki>tQ`FR}w}1F4 z^;)H=@z1s*(#VLw{T*R$`@R7ns@@Bt%mRuaRlBIA4t8ayuDwBav?PaD!=R`**t|Io zQBwu&PQ=DM!z(zNRc%cy{hOri2F`n8y4r?v5Eya$wwo-?F;4SJkC(JKgxgt0(N5vN^b&{=i=X=>{2dj}!dubE?XaTTw-C)Jl>>u5E+6h#B?^K`(rw*JBnS#&jDCdqo_eq?bm zG3>I3q5N#e8KOV;9?OpRTsAr$nh8ycJWSZXj>M&$UtQefa&<=Y@6ng7=pC&}O=hR( zKMtdus503#?%#Cxy{_n`T}M=lV>VueroIUXCMwsWhJ)`k4uo{0NQe< zc-1)%-(|M|T2oG|QxtZUj=rO3h;a1E?;lwE`*4mPZg#Z6xXmr@*Wy(=+J-qxeb;;S z^z4r9Cq1ZfO*v~C2fJTPTaski5c=)$kH#;nj`?VekU)wFbq*?A`)pV7`tV!d95 zfIO5gyqJyZT@GoD)5Z#{k#hbwrjttb!Cbf9pQO;87pm;HM%|JeX~Nn}TU)SdO!t}n z#gmNIBWZhDY(r zNB9Yrc*7QHYcvJJ>P>@=VhQdtzhq>WgNOvXzb{L|{Met&+Bto4dq&3k6{3Miy3(Y$TZ%XyN<=VUUK7@v{lVz0=1Ci$2M+xq7ty$Fc# za$U^hBl|Hv7rED-+_aa<2aY9I#p1XHyon-_S=rqWV5jJw7gH@pM-FNai;0Sg?wpa4 zd%HzkxG7`>w#sN}Y8Y9%WwF^usHW{s`?Az!`5UGI`+9E(0P}sK7cZm%?7vc|F&+ci z^JYoIL~QI{=?6fm${Iq=VTRIxF^a8aU?;ysQFLl2Bw9=K7gvk0P z6o-jd347KkUM3MHx|5GFi>BnSFD(LSSNA`4w~tv22haf)!R-X z=yq@oNiYl{nlxn}bwb9SfI3=qH4D{kg_gH$NCJF~SC%MIBm)SA9p7lNu{>IR=%@5_ zePxQ1)hgv`s9O6H2ed&QO1I=z3-nO>YOYy*lf9H})`y5q>e_$zgKP@SF02$Qo0u7- zJiHR;*C3%<$^9QXg7xPHIo!UQ$4w?uAFy%R4__Bj9M{UVDe9Z+<9~PBN!dS!yxCz) zUm<;aKvbkPiV?E-_O?mVFRUj?JI{#l84t-&skw4vJQ;d!Z zM-|z7bh@1p-@~X+(P(@#$64I;1mC!$C$H5P@K_4vT(LH%uAAMd;M0CYWv`2ycsg?=l{#b|ljL!X z$yi1pUbFfpkA5HT-S&)J28FGSBJpODd%O@>+|Zpmk^a~)rYW%(Ri)8L6jxbvI90aT z9U}U-^`)u&TAE9HI#ehrNs(utq-P>|py%ZnwbTok1f?id$ruz^yXK2BLew_2iZf4* zx%6kMR)FT4u0;05z0WFy^?wW;f5P|tEc+rHKcC2O-5ENwy!}@q@#>xLecF4yd8pUk zm7%Y@PWCu`ap_oLI$oajz3Rg7uX9aVd4;VCYA7J_&X{%;MRe83($PPQ063R#I|bJo zu^H6aFfw2gFT~AN$Gb;#g2xz!aI>d*mA9Wnk+`T_FZr}_i?RE(+v95M- z)(3B+Lz9hmC>0aP96UU&tNGD$oN3%p96~{Q7CqNhBb|5xKwnX|VqmIG$D>H6w?FK- z7;+|nc41|vM^D2aw?(QcrV=$d)o%A09Sn8hYEmRkqA0^J|I;E&Dkfy?=>H5vmIlra z2;V2v!ByeYJJ{%llY|TxTNlJ|RRtSLpF7W-Biv}p)$R1{(!=*bkuwpQy6LC(biCm- z1zQ6FE2LZ8YOX0D18$-$hGjL4JiK%xycK$e(@$YG1an&IdR{)h_^;PoZ-tOar!n&G zz!mVJ&_Ib#P!y-FbWTvrpS~_nb(VUAz-lggtF`@SpXY2w$4Zo}_DT!izwK8H4tUVP z9|Gor&7w7iea`aRac?Gnn9mU8Yw#9dZs;q13{zP34pLg;8>4loU+)CFZ6rh*Y66W( z1ATWWYvkC^S%Zd_-Y!U+Ii?cJztHQh8Ejpv*MLq_m&f{rh{o($3Vrw7U8{oG>{GT| zx)V?&P$dMyZSfm47-UxlSt){!wp%&aT^);jjiosj_LmqV6feQhvkxyU*7#o{+Y&;9 zf`SHDfN-Ii<7}N5q=OH%zt8aS%Vo5F+hrxo5AwNw0IErI8tW(b{61}WI#7XJT<|GCitZXd?N3SStUae`NXnT zBg&~fZS@_Iw>@uqLy|)u1%lG0S|dL&fxBQA9}RqH_9m8iv$nCJVeqj{;F^ZNMu%|6 zJn5ohd#M^~FpJG>PmG@?w0AFQl%*4-O)OWbjO?Ou0}v8JL%6gDc7SyuxE@4l;rwNZ0Y8%t2~~&i_}uCZ3X5;%k{lgPfty(R zDN7Qv3>MSym@4H`-;V9Vg%!X-FwP)GCUm1~ZiJC?`bu`{O5(EpP&_aj6lzU-K+g<@qzfmT26 zh}h&eW?hSCQZe{D9ZK*2btfVkF}AKnt_T-5J8ytbRe^7;G^%&%fmPs}c=RcnljwGi zaO2-C%+T3Sp$*|7WxvF5{8Zsc!Ff|mF(Z)Zb340y!ufUm^MME>zPR@aciYlA%j6+n zBjm#^8VBe=g#vN%z)E=M!M@m4@__yZYZCGZimK#plvQ$B^s&a!CJiA!`~Z|y3~@2x z097ZjXc|FIv6MJy=k?#-VwmnHaoxP4vm1sRe4r?y@7x;kLs!S#0;1Vwf)ZQN`P|C< zw+?U#vkl16XAyr>&*!Q{x&@Bk&c)xew93N;=##moqU5y5+uOn~t7#Yi4WkZf(@I}~ zfs|b?tZVAu2c|lRT$&_+*zp)bw9+E+a(A(KN*qmAzx;5f~X33WdgV;Wp>NfTg4X$chluwKewk&eLh}-IJMY+p-w^+HNXY zF|ag@Upr<@w1`yckndO+MCSQ5!Z|_B%FiojXC8aQBp_Z-M{|qF#=#=3yFMhe!)TP3 zo11&Z0q;-ih`n$BEP+E^5F?`JX8tTW*HZZ{OQUqw%zuo~f{sAYIm5;xqz4o$+nHR` z+TB_n))wq*3|t*YdEn;@4G$XR9Aib?ZV~1D#AgFi1Qw9{ut5hk#LVW}UU-dTd7d+T z<8U@WgCU-R`R7vz$NZ@fvNxof4k1m8{|)Wtj9v5%#@&%0(A>=JwS{0S1x9dn@`o#Q z{Q;gC&Gy&jjPV0oz}pP-B00`DVwhM#HLc&(jjDc7$==Rn@Bbt((kGG2F8xegBzgVw zan+>-Xy}rDW|5iMNSDFug|x^!t~a?~sa6!4iL1BqR==y&lZmH!LmwI=PjFTCjAy#f z$xQI2?-qC2BZeP_7;K^Fvv)X{Bozb(MiRw2&KTv|x@F0HBk)s##GS2z{J-+2tkUT1 zUBi+cqtFb{{nm%WS(O3siYqAKu3+3IUbkF}M`iK_nRkSTb^>}Em{$Qi^$%=XHLXh1 zs|Jf9aN8j`%q>+vwq{pZX^qLYL#P3Yzczm}lSLQ;#vsz#gY-#Ypfll^TcuzENSWA< z(s{(}OD6B{ksALpE;45BXVkR%xTC0E@6&&oyGdDnV1#wRasi#;9qf|}qvRS0IcUcZ zd~sAZ_%c^B95vx!R}e5_`E#M#AzipindIjcdvY><&3qnnTpv4PfWH1C#Xa#~Nk)CU zVl$US#IHqjVQxPN&;}9wU2yc{q@TG|l1)T~S)_burV?+)=uc^z{K*Xip+)RI`=Ve4 zZGBH$pL$=X_v6INJ`_?HR_n|jl#%hu(>_<;LgG}x4IP}8{t;jC@ojcuBz{^mZFC& z!w0b2-Nf^GmW<<9-pJ?vZL_knLFG38Gs!x}yGpJ^lUIp=pQs%cN^d(*CksxQdOMHtYwLVNvU6YW>rhS*piKpf2@BA#{>4o z?ynqA`?o?2X_`)!fo3Lgg`VR2w&80PAp*K>o8W=DsmnjLbbokR@Gh*RbGK8tqC$86 z_eF(zJ#9{@-$r#=ZJ8_rQ*m0~(mYUQd;>+8bt_&9dVK0f$PNzXP=BLamfJSmLDKQ6 z+_oC(5DU0bl7uQ#*(NkK)c3cs?6R^@ekPgj|1>o+(YOp%S9Q9yt`if4M`UbZkip*v z)q*9a*e~aoo&W;p$|f4e4i~~NBrn^zgd~z7S&lPY=(W1l2|Pf9`!LXNFH11gujs!Q z$y0Q;MbR=z(p4O6W|>7N^Me)2-*r_gH$j**eb6NN$@0fG7iE;;Fp>Ou^Hq43E$`J~fOD^fH^V;86+vu}%Q~hr> zd9~W@0;M~`)#>9AoYgC>W}*y~3F_*de2a;ElIEktNg0;ZHgeaZzT5Xt4zeaQRidN%&VkBJAt&iKZ3j?_Fu`^=^fDg}+t|TKTB|aLdv`A;hlE(GxP z9nIXi)zdzf`Ugq;Ey}M=*gz1%lw916C{~p@<*;(Nce{UV!In1@L!?aH_=S6{CLzJoY|#PGsjvU!^b5X(d+rVfuo02`g0 zbRcwNCWEs}8#U%!#g7&T{V0osycES#tE#%SC`H;L?Q+HCl33bZ6woDJRCu=3j7F*{ zCjhKddX`|2-(dW|nm2qOyGmM?>P9kefM|K_rOMZ*$NE%)vZ^@Xvf=e$jA`eFc8APn(WL{Wu5xbEU)9QzF=g> z$4h;1*4Fsj!V*{ zlT@t0Ncu|PCE9n2do~mbIS6_dYE|jtJz;-gB z{1VDdAUka+wNkbl>ywp_U~4i>sWY67BC|oyF-QRE;Q&!V8`l3Qhq9r4uP7 zy~GEbOK3xr{8N(_L~ZpG8kOO(yS5?S;~2Pv#D&iHmY!GD*wl|NX0STn|4g93R>HT@w^4&SiC<&53YZzJEs|_I2?`J z^Q3c&70Xx7lYft+2J|#sX$1$4G1(8|^n|vGb0vJH#9b5U`Jx~oRT@*Iy_QGxSC%k~ zf=-j(@!>?d>S}uqKPB#jjHi+#_ZDZ3Q#5-_PTnHG@2BrrB@xoCdwZ{ptJu6-r!!ek zXDw+_a5MMMYkMMX>R8rvE1RFxkr!X%&w9qk9dA$Btf~`$D=g`W2(3Np2(9F{QaDB) zMXA&s-AAU`dWEb0R@U`%%C{FD0U6okSvZCdf9&6X{}vlGtN&rGnBo~zD}rF_kfLN7 zeZ4oKgJ2_62QbG7Zt>msh&gH=v6)6^Ab-7O!{Ay zz;62MZoQ-geKvu*fz#Ivfv-t=GQ8AkQL%S7$VecV%Oe6itK|D|amdNaX&G9* zR|h#@9Ypr6e|!m+N6P*B5YVb%WPyI0X!im#iWiM2?Z-UX7uPWM--9Yds(` zt#2mgUCISrfU-P83|RdIIeP9#<~;ApF|P$qPtVaL!aTJ9nA;r{N~*KB%A-23G_?8s zxfCdxh2?}Rj^5ff&q}pr_O1cWqOR&Kb1;6J%;1va`qp>Z&(&5ee^uC}lS{?0wDuRG zEBopl?cM~@ZI4~o%_`GiFt4*-d=oPjkH(H_i$aQuL7vbfW7Gguw`uurnGzmPw-Aj2 zL1ewxHbJzHu|J4I*vrG8d1mN63vF_N)pp>D|L>QZ48DDPGn14CQ+rNylek*Z&uTBD zY+77CS>s!8rz@P}^;DDgMlOj4(sdo`!uvyRB$?!?{J@WeZ~~8WxG$~HcN0IiU!h;F z-B$lVIA`%r+$dPbDy*DV*`gH&yDgd#`6KFxwg3Rpip$(1O3?zsEGOSL)(V$Hhasvi zc2*#$qWMNaq`|mczPx;L>WNLS@b z(sOM|PW*CXC40N|gvA@K%Ed^J6nP-EgzA_)nYRsfdXZ_8_X=s6?GZQcfGlAdd}bnh zaoSg)=vls9GSXU1Y)liM)6o_wednm>fFmiydIainNZm0dK#D-#Y=d?Amfzj!AJ1oh zOQ0aoDMm`OyjroDAd!lTL)tpGJ6mhWr;|!`k*?g_gi?s=9?lJ@4*Maby*|U8E{LC6qN&hD!0=q4BfkX2^bpd5H(j!d{BDF$8HJTI|waCD>~@wIQ5EI zUI=NUW$4IjVh6S`(SvIvK+1Sxaqz0i`n{vzgChU{xXeU>hOj5t;R5@;QHI`Ui{6ov zL~a2WAMY0jT^Zd{Ce&Crk0a#pOv0>*x!=!CEliSn{KgPXSVO;(2bssyBC{w<>jBaG zYA$8`Wfdah9O>hp=;6nN+;PE)pHc8|=!;i&^uEkiuU1HlnO}2wqDmp9f_T?$LxE0{i;1 zEUz%_-p-3a=f%I5cxmI|Jyx!uZkYobqj;QEGUgGiF9m$~UEe8BaC6Aze+O^PKp=Pr zc`^nX!*vRXTfHP8xbcN+V&JIwiLeA)0t>(ho5S0GVRO1`fCu45eDU}RXk4ZAfe{L< zswCOc20&(pWUDFWxq5UX>z*}XQtfQnze5jRo`T|4$|kO!L8-B1MVpko+9U_5z`kzj zUEIv9yo3fRQ%Bb(2>CFOK)9tZJHu=%K^6t-Z1-3KPBuY)ihS&)(k@5}tt#9c0kATl zP$)$S?mMm(uT*CYl|DQM?1)w9Oln627N5=cQIgn6rE$s@@!SNtCS{U6_4Z0=PTVR5 zo`0E;O{RIMZ)A{E+5_;@EC4yt(kY|AjRO*E<$4fp$b8ME0ezI2lQX8Rlt+G6M0x=Ngik?Cy6lnTO$k|shd$v_~IK^OEfFQ$#wDojz`vHNL(+As)7xl%BPf zuhWt&pt}Jb<9RS8lc*b0@5#;GK}xnyC=JWIb9{Fk&_RR#9k^KheS8OZH4j4`oOK%C zFZ0ADVt~VOq=mQ0+TuYC;1p%a9^{~?Mv;iCu=TYA8hh3S3P`IXkD&njNEdl~!E)7F z5U5WH{Rz8C}EFrP2==Ag#pU%ywUcZqHH;9JqpeGNElKB_~}_MZhWArJlrW z^=uegzD{fjU(D%c3~;bQonmW`)PoQhA-|83XEMdyS=%V$NW^)vNcLVyxuek_R)|Hl z+d6WBYV+ofW6lYSs16LlAw{&YD?0`#PSu>0DLt7`{Ur={hk{q=pcpt_DO-iCe@3(( z!gt3M2PfU6PB5JGlX(mfo4SA?;8S6@lo2)7EY51q_gMN>CJO#JHRlN5l4g+5t}C(v4IodcmJ4&w_qpQZy?v1$nn41<2GXQGt>Igs8z(4_B^D@dLjWS68GO}UC zntqHd$}nw$a^51*Hqco8_V!jUjdFt~Pnnj6ot=H%Tbky)AYmKxKZU&R>o&IAVQ%oF ze97WTfIg5XbAQ6AOP9mIljrAv(+cgq z-c-7ZZy4yvQL*gcn*XI1%anf}_z$(iLya|EeSwNVQkaQn{pbGUBv|AJuWn9p*ZYi? zMRIa;?W0*Q!C_Z;baXJVWTXqyKgEA2l5=aCw=-G?%qI5EIl$}y>N#E1KW|JS3~Rdo zHy4171|X-1Bl1uG3B~`3CI1g(pZ-q{#Q#7$9RE*G|J&n#%Ry_j|2E_QEF)L`4Qc%o z*80DwS?ZsL{tsjm|6hLmU*#(0KLheVP|L)>IU8-{0MP`{|MilYS^HNh8Ww=R3)#OM zyNFXM7e|5qAD(Khl3`8V&HPXPM_d05pVlJSr};l=hfqp-gg5cusi{k-dD{Q08nac8 z|1UZ$_y6n2rT)|H_DT%JB?`FTVhDh~UT1m$X^Yf=lRuI$rvwgc)0rNgpK3{oOdC4N z00|=c&`)M)CZz!3;2iwC0N*{XrJ&sjnqC_rlmKe*dfDf(1UMbZ0qD$6viK_ro}%`^ zHgrP|jyO(`q7ZLLtv0skB}3?Mh}bx_rqEB@%sF%LAsE%91-@s$)j*dTESVDVRog#$ zne7R%l^~^2oq3_x-~ziF2(8A1F7K$fYhyi!w7v3DJxMsDopV6gb6kF{9k&J&g7HTu4qZ+tX2luDg0cDxQ=I! zp!*U2**nQluzB)5AQLImCS;pqOl~s;Z97J#O%#JphqNH@gd4~21Y=D;og_jChpQ@_ z1_tbPi};a{gkjFs)L4HJMX&*}tmTNJNRR;NS9(|}`cZ}3#5wm7={Z9U5#GxS41Np4 z1%(<*vv)#><2W5Ui6%7Eh(Q+x8;Uf~k)a_)S(CVau}_xZ?YHNb9cdO5*XGIm1@vXN zCKwXAW-WwkBlNQgKtmgIu8|wz%-ar;pB`u+kOeQMksZT2`7ufZWmkGw5Os3%7|-N| z3c8u;35FyPbPDqrGjn7xXadZ)xHy6~vHn{RB)A?~sGcIEUt5YBh7dEj4_w?>~+>l!!AexX;$_$^XdY3D)t=!@)~Ld?zQC*D06nD4ng zB{+5!$RknL?RD{jBKYR}GNKk&(>ds<`%a$ye!|A?2ZzT0Ivwm2MeJ$hmjHja7w@?p zEigD^iAe^M?YbCx@*sZH8SU)BF9|uGHSc^Z1;<&(@nX+duDU-zUPgB%$42y(E6d&; zJH85b-I-qtJ}EpsUEqr%tS20Er1)$pz20BIzK&kJp$EBUwoC*vy&-+PkFC7EcD-(J zEAOUbEhIgS4hakzU+8<^o34v4AgDk8J$PK(_`pytZ>KG>o}C!K7JM)F0iSEpwkq{= z@(N$@M$nP(NB_508eZoVpW81VP{E(AI2ied4g^1v3EJw!bI74yd}5+K58q^y;(_M{ zX0}s~yI->RzeqC8o=b<-@zICw!CHM%}r;Z)T z38#q}CHQPEaI++1_08b5r^EA7lz7X-#J!f>B}9$;%FX+R^W*D{BJ5(qFA+8P`}Pyb z$JA)gU&{_?zJvUdGgVqUrBgm}U9RttzIBJGZk&R5F7F&K&z|5m@Ur()*h8N%#H2 zRI!vDt66OX@`dzO^9-yuyyGZt51INJM=%GI?jg2An_ zr8jotN$!0ob_{N68}s5RaAPyOvpL!9O6q-}>s8?t&Ub)e`P$>DF#4U0Z6sxM&-5w& z<5fz&av`LW9QpNf>S~L)Jv-y8Yln?tX1?w_fv)#GIesIhvvVBr<8-jwN3NpQvhmUF z&g(MOdz|Bmwd|tlaS!rkJn^f}b*9Yg*2mq2t9zhoU?uHJ8wByY;CqU$C)bdV7smWR z^;PYLTf+5cpD<#N2O?Sab&(~lgL*J(;B}zPx6|4SQ{98|%f7BjrjC^ZnXbEXKK@IY z*OTlH;^b!L2k!T)YQfR!_h;LibDd5%{lt8qD;9y9M|j%I6y*!A*Z!$@cI3!|TIkbB zpU5fiYjC=Pad_43(#vJ^75c}-SHJ-uw5A|I5p>x8vs>r& zh{9zd`r`MXq3#~y&5MuQXx5C{5WEwDx8vfc$KJF=Y!K2b3h~qQ3l0-S%xXte z)t|>k)BRQz>l=Azx8ecM#);}KmxRwAN@kv#vj4av>y-Azli@yh@L4TM`%oW;J_fXe z4$s5IPd9?^j$qI`$D)@0;*ZB0t7sp=k8RV3nf7N4pX*}cow59UPowOy0m+XDLHFN1 z3U7V7CUrwbXRG<1rynocVp~3YRFsfekNF#W1vg%=E?w7*XpP(gcT~g&QyHv+A7K0h zzr_%Z6@sU51&2KTNx{cNA7Gh#+LZV9m?y2~cDC20fb58z;`goPu2*!O->chi?Tt2G ziOEDS1jHXVOspG^5bMBN$0Mot%bTgWTB@g_(vat4n9k=^!94{*pRm-f8|G#l464rH z?6->{Gx62g<;^R>S1*+ZuU{P<9M#1O*lLr+yIpUAT|;JJ=>Voay>+ploR8bYuMm!2 zg1v(8doNOr2T!X*f)l316IbEHZpgSYmk9IL_QzLKmjm@5f*zL;rpEe|o-ZE_b5p77 z_-uXS-QCl?)P5ka(}k&7QmXT7d#YE#|JYhR5` zN-+oPPUwLvOIONgSd*_Po+43lC#iiEnmo}5CY8OqD&sKLrbx^_gM4N6L6_YVP{0tP zy~N}s7A54%%&U%>A22=Q1o9VJlkJJ0sfis0p}w#;b@BuUGIfgWbK2yy5A+#>MpHfe zphrP(zvMLEq*TVU`y6j(AK&>^<4e9i#)SqicX}c*5&ubb+UVOop0CK@4KLm=x{0^d z3CH%`oMYCsT>5PQ!Dhmsulpv%crfj1MySBxnu*iePDfN&)RcNFjNa;~rs~Oa(4vGe ztsM9epOiMZ@I$kdm@m#K@uq_LWuvUWX@~k(ubYi7Q;`6lFJRJ=uPHM*seSl0ft?6v z4!wFXv9Ff9-5&L3y~JR8*LIe!&tPs=ecbEHxUS3o%Yc*(){_YZdS#{<4Qw4@zq&~_L@gdDnEv!4eeg7; z&29Qh@3jS!rK2(27eB_;=*$?+#64I-vyo$~&w3ZUz;g1`wJrZn)W_q>eYX$D(p(-c zDMgqd6u&Wol?ukAOy72y9uF9 zgdG{dUU^O(_vlO!oEMuvoB$>v7^ddYub=GODZhq7ac(@WvSNg$FJJ@6(Q)S=tuYVWtn<~HBFJ@B9FjmlAGWGljUlH3&X)UlK(@P%Efjd3@;5HE8I^}L}Jwn zcg`vPC({L0)0s9OZU(KKO}WuLdYW*@3~EUgE0eBrY~*#;2&=A9k8hcLBJ{b~Z?N|y zk~?UXZHqOzRiBYPZ2{WxTGaALF8nESmAKvIv|8g4jSi=3-;Di>m`fz#1?B_F&MMqn zPaJ@k#6KXyzbF+d&2&!=d4$D-^Q|DIQNOii!U}=o#(HslFIm% z8w=UagY0ilHzn;It|SyLhi~OFaOUD68w0g1b@gY;})9xHuw7cmaToKp5AjRDp zh?WM&?^bY@snP-1*xY^li%p8``yLU5Y7v@&75EoLITyHbaa#T9>7|YBx5TU|j03nO z+&1pEYC~y;rr#qcov1H*+&Jz&tK2x`C!=yI6gh_VsLHKX!Mt8wwX#z^6w5r}LKBys zO-p;<9pIi?l;m10?AWbjQX^f|l$5Vd#r~NDkdB@Q`k4GL_TDl$uHe@fj2%PF%*@O* zW@e6=nQ6>y#~d@p%*>9N?UrP4;U#6t z7?I*4$1avcRe6C)fBWrDRC}`!bnTGqZR*DI`K9JM84+knEB=rQr{D06NEN#RM<5zr zluY-zy4f&kCua%+c7z*2&E5Ci6gkb&Epx87Gq_8_7-^6?7?)IaR!jwyA=f?4_DGNW zL3&cExhIhP(Ka^Ei)&sR=QEaJA@*af3s`vcD$e{X6?;>yY$*@RayK&F~@SavwQQA;MQ2Oa8ngW>o> z%p-GjJ!c+{v{$3sLxC2}V=qAKip}ZCw4K5Yq@7Z=q(0seqoba@Df9c*z_CMklo6EQ zl(~x`CGl@5QyX!6r>qHjPFV_)q6yf} z_j!7^Mn?m^5==@eVm?405E>)4LVcH?eYJ1a9bNd8DVJD*Wmu}IE!kQ)HI%OAIE6p5 zyeZI473%z;f|aNHNIN&Y0=&+f+ggREYqh-797a^#iQ*?j_8g*uCs=>lAp;K;(uW;b zt`F_X+I$Y78kR7+I(C&`Yfl3LI7>*v$F<@Y1{23QC=E13MpXb4X-Sf*rWhk}Tq2K% z%oqr+9k;X*MG({MSxVPu+m?JNmy#VgeQC;eu2^LUd~dEvb6DG@N$M(cWYGYx{?bBq zNz$jhyW)q#pg??S*#-7@XDue2M)Kya`g^SPhQDkT%IU&&6^Fsv#gUmU2TeI%IXkZ> z!n}*Sh5KhW3WOX9*%CPLX|7nv#hrJYcCEyPk0OmV*FcO)_iu2$@JUOb&4)*~X}J;nuYHZezTwbA-U=1pSr6(nBz(7}WnCCJ zkU7f-ZlXWjlnjXc;Jucd18DPD-pe*PnEcw8@S z`PJjmv&~rqqH=Ax=jcv*b2$r^29woy*m1hirczKemL@3i6k>WohN*(I@u-fls)Kl; z>E=by8veL~&yj07<3PHce|>cPOd*`k*9){0IFkCg`H3HSFIA5w;c8*^ZduN>eSo*; z%qCZ_Ck!9*p2VBspl(cB>v$hC-je5qFE0Cy4*w_gQXSGJRqeTaHT6`J!p&QU; zk1{dchO3*||M(XRjrCxV3%Yl!KX3W>QJ9%?A$Rah#J?(teyW&Q14sM8GFv98Jzelo z;d3lALvs$!!DKAqwa!5!M8DYWY#h~QLmZh|^(xn{_Yof`|LkA+fchSAG0jocz0ypd zzO+i&SVuz$F@pxp+60;X3~_MU5LxhJ7bBSZ75Ot>=9M~>GNoD%X$hXK4Qe|>8vBvp z(;9exI>R|}m$5qQ^90A%-O9Fc@H;zDqdJ+8T+P&zUwFGB(fDC)E;12>c|JAZasod= z%klyKWD5{VM>i*JIKgYhTbms0dkefpe1KXaN=-GTE7p5Lr^brfZ{Dv8vkhnN+XF3N z7TT;7Q2b=u?};%f7!E~MG2I5BwbdQ7hm6uL%7loGXA{iTu`rGB1$|R*K=*0S6Lano zcIVD%NJZoe(IVGl?ZI^S?I4(Gz7ksf$u!-V72Sb}2`$vNQiAQ4WP5)z*gsR$9s>QR z$<0r99SdbzD7Q&IiD}uB9egOr$obbA2gYbazj9E2?Gj~%E2~vFeYxuaTiD;cz=###=a*pAsL!-tdK3ulC&({v_8!|*y&~jk$^nbqO`k-efV|G?U)KQ>tB^TxH|AyM~H( zzq4X}t5fwu*_J&nCM`M2(Gj-7Qr7H=By#*UN+9MP03X%g+5f}N+o3L6b?02ZQfse@ zwt0e8>XYS`2^2{*@pxgA8MAE90H(*Kl8H~u55J#>P45crkCwNBvsIKqMZ(Vruu8cF zp6tfbyv+`H6U&Z;6N)>#sX^`K-&9#+Hl(VpmgPBqaxXSAn6U2TtfomEGwru1T7Lb8 zpe{LNRvxN1klqrta}=CGr^i zl=|+{)7N=*=(>WR77N^$>$pKN`ne=p{SI2kz_V|$E3-d2!|pIQ!^2lp=Tv|;_>n?9 z1AIpu$A_YBibGcAxb!ELQ8NxeMQ6SRXI)eKmDXXde&Sy+;ZRokm+8;SEjcVFq=K}7k1D~byw&wuPQ)ExZou~$^A^!4=KDqksU<~f_$HOLIF>zBGGiWk9D{s2?hG^h zFAw*aV+plC(3PirQ}^ROBy1MWC-wclIr3~xmFGfB*r7D+4`m2&w0@5${X_Fj$iZRfGt@mR|={8EP;$$tJP29z~0zNpN zrX-tA&9T%qlLYvkE6-%)EKJ~WQ(p`vAl4C&7+j!sw@ACJmDP2Otz+lGU2a}rFy0l; z=_?zL-ALsQgKB5ir}ci zYeZY<#!9qrPasD|_KK!&g1dYghyR1iZLjy|jDQdZRhJclXOFpN!nKC%<5Zy;JMZ?* zRf-%B{4vS5jK9W4a{O3mZgwdiqeGsSEWJieL2z$U9KctHdqsvKSItkfC9E?e+gES7 zSe?>r3Ch3ff9gYY=nJOSYyH^OB%F7QbA9!6U2`rDuf9eDF>i|PU2ouV<$u9PzFrTe zQZQ(2Pi_UuXg)ETi$Tp(<(@al0FHvx!%d#0OM?*;9yJ#$sJZj+8!|PN|np{ zHj>A-8dTavicD7}TCSd%?zpLLcZK6iNI!8h_%YUU!fd+1Zq@OR{grG?f9yxUeZRzx zQsZZL)Rmr&;wCmdy8Ux_-y3Gw4yUc*=NFZyw4!8gYfgRlam#Ngt3Xrslj7Mxj@Jzy zD57fS)(zrzY@;U-4{F+AQhGm=^|jF{EIIPJI>?(PM zz-Wb50^Do3sf1XFZ1yPV=ml>V+IZ^1L%Pz$Skp%jZ_QH|uM5<-hRTx^a(8w_I-xEv z(n2-VMzY`Bvkx5-aRc?63Hpweq`_r`dt~Ait37ZQNPL5TODUIb@kCDR1gLjnO#$Z? z7v(0KKE*b!lTTSo;ggX#xqj!;YJ94Tzm?NqZr$Yg5mYip`)$u-AW=7yP(~dv64#1) z-7f1AcReWl&Qn#14=A(d6vnN>h?TIhi55+YZY-?VR6w7jc+qY(7W5V9^s-xPr1eP2 zJmd4S6Q)^FG{66b4H9lnPtN`^eNE@darAfE%Uc-Dqegc$b?b%qwTk4X5MHw>sIn3p z`Stq40t*Hte19g`Hc`gSh%Z(TF z4b7psWRxvT&xLwkj0h^S=kulmpn_+<7vddi>QAijBNuYvk6c5kD%Bh=Q7wf+I^O=( zpAp7%GutVh>T2+SR<1D|?aEbqUTc*?o5kv}#pK)^v~jeiR>FpHYGl+^$SN-WbTvUl z*_!`7=>iqUoJM->=C1vsNMP)I>t}ORFUCOTl$Zx@>Y$LfWA;Sal`V z8bMSo7u!ScMjz?W_^b8QN_n>XyYYejPq3F3evh0{6X-Wj`q@QoGE-ASRy|4@(c%^H zkKhD}(qFBa-7m`Xya6en-KjR$$sqoqrp?%WW02}oY&&Q zC=jgB+xKID4BKX4PQWWT68xh?Y7h4(X#MFhL+5yrsKgKLWXhc*j&GPY%v%}183(;u zrR+L<|5bkxZ|nRBl8G&1-Yt4O69zLa{HvNg?iXM#UgVyaz*9Ff^PYTR$cVUy5{zS5 zE1j2LB&8QtrtbdYqQgN`9fw-@9|~d7ut{|(E)vT(vkidzF)N=fpVEMkrej6fag%6q zC+Zg=nkabqtC@K*x9$h27{My19nmq8aa|Ot2m(8kDBMk=k_Gi=+X9(fmmLS1i;{k@ zSg+P=+Mb=Qw!Ms0=e)A+OwP?_Wq}Ju>hDUJHI?(plA%Ki6 ze*|w3k*AdJvm?Uaal1vx$i) zPBUjF(i6X#-+%cQYcFL5Pr{6kI=aE>rCj)-2j&w{ZMgIS`?=73K~?|%>N zl}eD0q<1n!qWZiuqnok`*ST{FsR_wAc44a$-}sTyPdpH8GR5yFg+M~@04G|1uj?!7 zS>CM)Xu5VR%7zqw8q>q$j!zMp{=U{q)09kfiG%lu4O}G0#o=E{%TO0v)pQJ$<1Y}^ zJ7X~-kj=2oRa19z;y!M_pl#2&B(<}2g|0bpwjy4HdfRfGuB~OKiHZFR8;5nCGkIEh z4)iAnWUu)6aVbx&j{WrQmD-6LqxJKl@$8o6>_#DSeWUgJ_V>5IZISd?=eg%52LOxc zdziAS&X$zs4rA13gP+U9`(ExTlL|cA)UilbGIG zV<}ymIJ!^)KHcbWym*XO2a(T&rirw)e3ii>*Ld(+4tqBr`ZCW`fg_{Lc(j)3Qw$HN zEaPRf7GGUop;QB@--frwe>tlGnr)~*9<}UN-B;Z2QMD}zpvmDHc&D6wasLTrbM|u{j0dM2 zz_!TOjTsH)?r-)E>x(rRO5i#6Ee7Jxe!@*(ROJVtSo68peVQxn)B0n3l@qbm&^fZp zZc|$Ss~>!mm>&W%ux+h?y!;zRnafo`^&$i%EGBbelJxPnu>y*bNH@BXBHpIv#W~ca z)07j*s)fE7bRH%V=n-n-Xfqr7%tD~%Ktat#E5ECMtGPN@J;lW2*u{r8C2UwC0^i;m z8BcX5)2GQeX~rMYtQwS0hj<)qE%($o=-%c8Q)nz^ODG2?4I-@1H;C#W>p>Xm%hF>L z$^(GzIFffiIG)$t57wp-oT2*2mS%K!Z&tV1XsREvAR}VDL1+gq-;8E!I!SRSE9-bj z^unp);q6_ZMtZJ}=&wSL59G?%tx@IJ2D|sGXEh5RVt;EyV(9FaBKtM(sBDv=VvFxPic_#%erSNq>>e5m`@XtPJEMN>$n-G%!C|LW@bbcbVN6}d`6D&TjP9e zMYJN4;i6B65PaPpQH~|vekn45@dhr%nBJa*ITPqFcg+05StAc-<{n3vs(dwmKMkz# z)A1f|7MM%aN8GJp6ksQuXEA}F4dm{gG|MpI=h06JIbF99_*Zps{>Q{+x|+kWTyxD_ z#i=ps6pv4)rwEU_?)NW?u~OA49ZjeEEpP`q=6tM~?pn(c-#Xd-=J+(3yn@y`i5xRE z6*eNb?1dm``DF3JFwH$pv`+~QGkLQH+5F)+NkBtbuykD@e)AH~{F6Nr_DBy+J0ycU zBu!>RiBdb6W@CPb!42E^Y+u5Zer7F@rS z-ql*Vqre#ouZ0oGAbcaTF&`lyu$HaMcPdjr3{$4<9~)1GLZD7M1bvN#P5*P|F%A=3 zMBW{G^A`2=J6c(77f<@Q_=^*B?f_gMib0^tKOdB;74@%tZlqxa>TlVY?$ie-> z*N#B_nU{gM)^x&-Wg&etGN}t}0!Hk$Dlk|%3*b+fWVM_s?QE4!y)_YBH?;~O7_d~o z9BRG+uK1?*`!Oq7eEbD6kqnBMys9wYX`tzLzt(l!Wsqib$n!+p+|9b)*qnw?!WmUv zamr}8HzVJk>HZ8~zANuSwaIbn)FRN(ZYp4zdF%VO1ynl(iWa)$AGP_4&ra2bgA91g zK9+PBI?d@@mfXqoE~LwYEj(Ik1Q)}z%sMHlkNvIhSeDFVR`dA+XC8GH(>@VZ#3zb| z;o|WwEYYCp;!)v5fj@{&T>=s7^HuG2lo?PZ!v6C{Ce!yNfPtt2TZCxtB{jBYoH57m z8V+XL!}Ksn1hR3$X`&SH&of{RPy%4TqzB%}OW@^V7C5x6&@E#9%` zDD-PW-ZP2Mm@y+0ey~EwFfvC&E)FGV7oj9+-JSBl*I_@bJ^CbtD=el1Ytm3da>uun ztwTPBF+0Kw>I$ypXXvzLZ;^>&7~9st(B)hv}Wj68^m(nLX6 zY|gCfb{?k7pth8gNks2W)zGLXr>RH$Bq>kInKc$c*Kdk4+|L|@osb8TjKrR=Dw^JJ zM;LQPp{f6kDvx7dcv9(Yyi3U8>t5fq?<9kDWPmbH-MbU!}wMK=albG&yZ zFe)t=F3z5ZNmJDHEwljA@Rps8k!kXrkxQ@gG>4S+W%Y|u! z*FcR70Uv)@pTGKuP{_L6?k%y=C}#DYe=*@k)Y?iyvb-4506JTfdQ5*1FY=V|sb8Kx zIXG1F&Q4Avm)xV!FLHk7Lb9)kcD4)76D9MqSwXwx^@<{>`#lwSNb%%%`VX)Gr?|wD zr><^Yy2d5ym}Dnbu2WX{orA~yY_xb|n8Ozo{rY#uPz<4X!MC4CUn^DS{1mBrqa!w) zgB!flA=o~W{dd~z`jO6F+jAgfTdI5q{r9I62HGCZzev3q&Yuz31nB*12?i?K z5z=o~S-$Jx!|@&|w7U{|8^+dAhfUV5u%C8dgBZQDxlAHL`yCb9PJQ?&eY}jUJlM{A z;|`Y9r@tcdKpTWRUoEnwl(eDwJ>x=j7{>4H{py&prS`hA=oNNqvqb-$x(>HS#4LbJ*FOxT2LNVeGBx0RdRIh2JuNJ8$^ zjo5R14M*O15NVdGTqRXG$0K&8pH2v|1wVYTrLmIW>2Z17;r$WNNqv%XoER;PJm5nn z?qsg~xE363%!acwUeDp9MNFCB96O>3I!cOgS^aLeSbbrX-yV$S?my8{y*-1PoDX5V z^oTJoyYp~zy>o?y)hsoDDc=SDaFUWwF;J z@pqFo(Y@Cz>1ZPP+CC$G%Zi@D$8yi5FI{%!_9LM8o5mJDd+h|^5^ra`ODF!uc9D#& zl_&W!Q^q+dsL-5kZ@W|HX+mj!u#^ssFATYF?}nC|FSpomXm0f z4v#H%Lod&2>@Lt)7BT7eh3%*o&P~CiNOipCQNdy^IPH0X1*hmK=|t$&au|trxX2m$ zC$CROl6U9Z{Ym0PD-5U0$TBUg?LmypL=no1H^+=(hta`d1}lHiT(;KE;$Yo_)*KB( zn%_3VI0L^jME| zx9eI`TvEdOZ)1>q0MMUwCS(Q@Mnv<*4Gg6+>WK9Qygf074wxt`P-4R0i0apXN-cumGhft%W6&k=idV#evCjWJAYiJQ-k>6Z;>w&lH1H=A!a@`9lg7dv7n0x zr09aXn+DkXb|ajibY#E{5AWwEgq>h#V94 zc!;RZauOM+@y_%`Wobu+%WI*1I{;t7h$<}t2O3e1wFeKlGfnd8ku3#~GlOdBwQq1+ z3c-|G@CGDCN~*FC0O^*5oSzNqBZrv~2g1OD4Z({)D5qcLwuT8W1*RTor#mA^^2};L zkF>BVVr=~5$sH*S;opom*{fi3DoQV?WBzispKOb)B(SdRry?X+Xj0)MQq*Ys)uOOp z3QcR>5m#HG|KFPTt?&#iKr#LiZm>%Hr#DXka;gQmaw08#M=f~Rf#$j@$6Jr1%DtIIE zW&|lB!?he@V~<^!XfvoHXW?0I_r{AAF(m%0kx~52F$HnFnx5Xo&B4esB^UtCpwnaM z;O1IKb$))H@ZWcQ25iNqZ9@N+cf(o1hV z_v-|gjOYKkIWP>47#>653lvFw)c=P5_K)*m5Bh(c{7?7) zJ?HEH=ODqYtjoZFfB%un|Gbu;^Z$4J-!u6CPsRUxTJc{gOTw7(BQ`cRqW_yhw`JF! z#{%b2jQ%np083yrxw~jXU|IZi) z1=yVP7+^V;rCilG=GtP87J0(zXPsfiDF1HJ`k_X66^E6Uva;z7fuW7=eP=P5j{p+=NKL%N zVxlaW{Sl33oLmyX2*@_~9~0VWXJ?r=lqGu$&siXCwoC#7o7jweTqq2Tn|621X%|)s zCQk0K$Tj$F5lGQqSj9cU5pEj?NAgYVAyqfOJ}5{-oXUP)kLuXrQfhMQwk}chU_&>J zy^z(LFR`()5(=;bM5JtN-)ez)4D%BH(PCmQsN#DAyxa~`CX8FrJ68GE#2(q8a4A9~ zaKnz) zQ-cSO`QRnACkdQjwDhN@NzK_;oRZo{6@s6*puIh#r9ATNHZcLUzlNcUgn^iBb5TNX zQtT1xOBiGuXV}Cl^^_t~=*B)&GRK!T4RF@UM!B7Jf4{O^CP`IE&|lD(mVr*rfe>Nb z!Q*AsC(|LlENcz}C^2I7?K}r5Yg_wbCQ!cbD`hw&Zf3tXcOpm%Uypp%+UqkJIwUdX z_CS=15e`i{w#0D}Kq8ai`-v!eH{UVuV80?KXRGPlG?4-s?=IvB`TV?sSsRix*EKX4 z(TQ0-j3qq;&!(V&PGM8^=5Gkpn;~b|Bk+4Gx9ZMjl*JvhhUWoF__ePa=KC^l`kZEF zW@e-=J#k<{!cKH4`4do+Q^B=LigheHeAg^$$h4`c=~}jZ%h7&7e6ZoUI5=F&Ko4=j z6(?OQ>ngPMEM;m$ACZDJK#NKoip+Lv@51D`8 zO(E%;8-Z?>g8{{BS2;2tF<|Tb{Fm8zV1~ zi$vyQF{g|>9>L|NXyd0d#Tztpt*m$kdj^&Zs=05^z4gM6)IA@6(26^rM5aGxi1j~d zYS6}!eB`=1?rga~4o89YweQ>9mccwJ{i^A^|KEGOt5dbSHw8=;5tsWDC6iw#xG_k} zTh5+Wf`JeSqG3B2a5KkdT*MtcHZ%-uCZ7PRylFsIAO; zT+++n;^~!Jm{}H#NSb~aFsv@dT4I}SoGE!`zc+QN`GIkKYf(8`t}wC0QWK~+>dQX! zYMPC}XG?auv&0%m<^m=>cGFMw0m#}_cWaHd#0WejCByVZ9>^uf3265*chXcYRPuPf zsgss_r6KCN<`Ug3-6K2VtpE}?NMRdVi>nKsXJe;NEU#AVdr|>vlFJZ1K2CHVL|3Nf zJ2FkLG4>^l{;*f?=R|~3fxi}+-#5?9v%-Q8vYrleoNV4#yYFrG$DmUi5H%-1GO>1Y zMHkl(PWBZ$--}N0ui_04<0v-p0gb-A2wILu5|$9|ngq5*{Bt$F>~1PJ!8y>=_J4zM zR-(E-hQ+R3_e8|%;uvZ}WgT~_* zvYefvdku@2oOu*qOj?!ew3O~W_u(#2Ol7OjY2%ogASp6Ye?nE4Jq z^X%_e{K0)W`@rSs*X-t=;E;1rauah{k`F>*U)o@g_~{BYda~vryN1qWz?DUhzIP}@ z9W%&j%OCYAD7G*}-urAxxx6WB@TDTV+g~5dwM^L853mWPclthCgRJ&Ji4{zI z1H(givO+-YDve(!D=B@Rv6O6sctLq~qz(gCUDsoLth`10kOaZQ64v{lq);y9(mNr7 z^nNH9*ih@GO`8pEbEGrnKgOB=x>jw921=u=0Ns}}AzNl%LVt{8-7FILJB#;m@wuI# z%F4>>*M){cMHQ|$a@8fBb5Zrj(r6J=b~lBeHwV~rIuUzCogWpu?{_0#kxj6FM7v@& zcXj;cta!%S9J{SL_qNmGd%(w{DC}y(s8Gol2?{1S*=&uLSyvTpg^QnMpUv$WIDhte z7Fh1^Wn-KkY_UK9*hZS~W8^qj^7hTIF=X^kM_YxDr~$1hTs3n@d#=3r2EA0>g~Fam zrp!u>pk_SWEk7D`sz+T+ZVxbkW$6wG!Uv6YAznunj(}+Wy+aUhwV;Xl?3*&0qgEdX zx>2VKDA@CeGCWhfPu`wtX4h-{Cx|@Q@7ZD5HqJhM=6+3n&6h7>`!z{HW7!16Y#?p& zBSX(%YK=)Dt2v@v^%3aV#3OcUW2yWR+^%ayOq?OJ{otgXSZTU@NE7K&ZnjZar^6nI zC4>_DtNRil>#8aCs#5A<8(d1Cv($Keb7ZRZ&Q-z@_5wD3X=Dc}0U7x~tcYc1iUp&Pn9p!%?fMn z>Y7Ztzsh`+4GFNKrOj!mSA%5H+wr0nxEuL-!f!Tf7onPn`=9Uwn?&~3axMT@w#DWW6dSDOE2G=<&0Pp0<_bSfYIlZ;M75dn#+F|Lw@lVLgY_r%jn`zjB2N3!<}h!$YSf*QkSt&8GIfCv}?$Wno6p zzIz~=H>fxX@JpXxM}2Sv1nafpoS+j#=_Z_R*D*-aW=f8pBFLf9=t>-*PR@ z?UW_5kO)X4$$KeR(ta|X+NC=3%luf6O;-|6pKZlElGeOi>vG~v^G`oE95;Ct9H7g& zVLPqQK~+lvSUXHpC?k(lfe5RTLG^5Z2zB!;?+Mt`TM`YeC4Oq))H~g=r#f!RD}^DI zeX~Tw6Z)Q7P@9syJ|6Qs!!7oP(tDYlU6N%Tg;=_mHfV`M$VyEY@4ZI)`g#`kfhl9m zSq+x@!8Tk;V`^`RBcon*WD(_ZIPD7aaxWI;VcuWC>MQouud;Ry{?uAYoQ)hPcl-@} z(oN-^^uWr>>Nmf)oAZm>bCHi+ z?AvFoW!;9xnFG{?MB+$Qz$vAl2Gg}+Q^wsTTVx9=?DCJIDgM@x+H*rr?5 zd7mOQY_j3*02lcU6FTjB?eWSOCV0|m>H=hCqdo2&yP!FQ*;dpOjmYQAEf5t8gA+c9 zVxmFQ3i;=%E%|iBT!}{y6Mz3UlECiq=D_{p;=3@EmDR7WG1)h8Y`yrGc%2hTl4kv6 zhbJPo?l(w#%^2N)vewwG$XT6}Iq4VlvJ}MO)yDXH|JO4I=a!=MvX%8w3pWWk#FjilQv+ zNmo0LZWkNkoxhXBhkeP%ev11ZqDSh8gj9U2VGLgFv)B5vv5rqU{Al-TeFEZ*V-AC~ zg!kL$9Ijc*+hX)*5|Qozss#P{#BcyxN=FCX{=Y&qlPglYbk2$PHAj9v+zn$`1`gO| zuI8DcPX*xazl~9SLa{eLOgUJz=lt+Kx5)IZYWrnbLH?pI)UXawUl4w=>FVX{Ycl&q zKmXyHYMK&A$&|*hi0SgGfX9XwI}4{zX|rW>c%(2rMp$WY%q%$3VL{*=e=7MRLlO|5 ze^>O9!sGQ=AsWhS)2{!yL>jx%IBPOr^$pHrmK=DFac>gie<^O*U53%)fgeC_))=J3 zLrdvX`8eYqFKW=cr|SH9Yv8>td}VBcyEl ztG2Few}$xSwbnCepwoFo%D`L5YWiq_bfcV!iPT~bQE1&z>Z%@vy)7*Ak5l(xQWIYQ zOTIqPD7^Vzq%ggiMbTn);jZEk#~$k<{hi*3IOSB(nLZhIIJy0h%1~J- z-XF-|(-J?(RLO8Kf?Sf=s9+<-C));-_%HrMk z!}9WhbMlYIh4vE!2~A$v^RJh5pBW70V<;LuqkEx{NHG>-VPa!7Z+^0%xDp zfNOe|?mt3YdvEES)kxlBb$81kQ$lyhm#@fQZ5%~v=?`}cC zIdCi|<8h@TnN}#+I*GttXGV8-=RV~IAfyK0v)$t1>PQPv-RRZ zb9823*sk{TW_reQ1vnPdl9C!?4j1|U#^6g{wHbqbJYZI`{4Lgj9U) zsBo{oqLi$liw=zM{5UZ6F8yuz!|e8nZS!%G|AOUHh+)XDqy87c>4_QS0WD^b^;s(= zQ?_)j7N%l|K3Y4Uy9fdLf$eJ3G1MP+QoS*GVX5w2g}YB$5&@<)qaYPq`ihE*;?mLv zivioXF~g=PBq^GkGK@uLO2mNh`1dzf=S!b}ALh!@eahG4;%}^La)2IR1|vlyg$yTQ zU06sTWnH(JBlBB@)GHZ_oj47@W)S1lUpLCxQOSpQ1Fwfat={=sh#vTnJB2RSr8D9z zcf$tm^zJgfTpBllJsBcHz>EDm_OMJ93l4NsZf-%BD?6Km_O4IWcjuF zroL(!h$z*>#P<^5CtiJDec>k8ca2iZu&k}*%cH1(l~Gir;^O*Ykfqg7wU`^4ng}Rw zx5zk;0vKt=TP5K5L|%Pa^@c{b1dL=%>SF54vZJqg!+&_c;dMlG7VM&!?AYKMP+S07 z0-3BzR4xhD;6ATVb1sHF@NvkyRH^W;^%>(PX&uh0K0uLv((W-krfZ+>^OItj*2^N0TV_W%0M7-X2@`dQFtPI(cvZY~{$} zorNcdUjeSQkC%%TZIl8LV;mEIRaE3WS2jl?taBky9R(?MZ4hQkJ~dj%;XG_F;W`oR z+?f7Nf4IO!l?zV8$%{GvwcJOxyWcNQv7UGXB*=ABaqSZ{Fyjp{f_!|rBSM-Jnr0;4 z$yFQx)5o64_0_$Ri-E9W3go2+YgzuTGI`d-T8}N7RoEBmL6|~%zVEta!}%-MpPS_( zC^f%oI(*g&Ft1`c-d(4vW_mOhJ06xh;+j?=@Iqt-_sgQ%`#4rQ^VfZ^&Up zS9-V=K}N_je?r^=xL|IK0ZsV(&#MTgMmZG&7MJ^~IamSKq?UvJR4XtQ`hS2$3e5<^ zBt`QXeFtNy94c~h-nQ1bah7xg=)rl50a2qXspyXd3fGp;cB7)@&4Nx0yR(Pkm? zF7rro(G8c$XN-ukh3yy_o^Pw?>7W+1CR{FB;3O>)-+d@@jx)RO_SbH)MasK0j2T;l zH8LWBB;U7QVTY$7(kgd}0gb&#Eo^Q;DH4rt1bpwNfyZ4BkDin)OYilE4hsR! zD*SW;XS)$53i`zP8e9^*d8gq8#a&`!4qy(a?fSiRQH!Cv*13OWAr?3t^3&Kz;m|qA z_q;ylWDY*$(+IZzFr9%O@#z+_u;5eseHrx?LXsz8p=UMDL(t9UMj`JMfo~DE`RhGb zwwF7Kw>vhza}Y~+*{1jvK>3iA;Jo!p6zdW5;6;KQJ^E-=SBPIN)UHf~Gb;|*(*a@T zz+A=!lNq`UB<)T&+{TXkJwsMK1<F>-V+UApe6m*VSq^Jf)hw!vpf2?y*mAYDtx4Do^28?kwL?X z#PUxRQnr9zsF@o%y{Z%pRIQY>n}XWI?^S8X85zhXOZEj06<^O^8bA7HmP(wC8&Q<= z!C@S#ux9`#QB08`Bp&LM8lIrI^$}kGpYkc*{-_}aK!#edbhRKiN50wt@jUJfxI6`S zFCmQ`Sy`2!7wP*drkaBn0p^0@*Oo!t0^NR1nCAmLF`%(Y*zwmf4hzB{9U%$uPRsC`>l4^>IA%Pf zO~v$#K+M83R~V59b`qw#^jZgdXZ;+pL|#XqdS#Q`+|Ex%Zjnx3^+V%PuI?Yo4iDij z*D_~hjaB?8#JH#h+_+K}@6rmzz%qw_M??y<&wHaoco;`aXAj0D@g0wbg38#T+F&G9 zc+XZ|IhqSt6>NfMRq-Z-5&`MLeR=J`*uZwq7mDfR=Nc-nzwP~rMQrvrCMOixJ0e_$;S6#ixczE_Bpq zcZ-pvXKG2)MGC>|`n2-B52f01tAPeZb*cNkTxv(->BAP9^zWC#i8r7O#^YtC4 z)iwx)Sz+~6$iV0V&kH|L*B?lJ$X7U}Dm?l0~fz)o}tEV9VY?ZTBvYsZbF(=3Pj z>j9bp&9T2}r0yPH)?J`cP&Se9ndfm?Wo>WitDgI6q4U1#RTS<}^Z_>P+6G0mTi-*S zv21xV-R-9{S%i=*nye-E=7rIh2+pvSo~7lt*0za0i<~Uw`4@NDKCk=%);vHlR;4 zJtw%@Q86buVCL45vXf_>$*9PZPd`Zg^I9TQyoYX97lNL+BKvhdhjR8CD;+eq>jAK~ zAggH6W@}kO_$7-l`g~7tr%SrU&k_-J!vH3z?ZMl<(2oMB;?6$sMpL?z)v*u)Je0)E zqZ;i}oC9rU&}FXUH#c-N6oL`IAWT+Eobmj%d2;=lP(1WW)e>Ekn&s2Lpz$n*J*_m; zK9PDkePz5Dc~OP*o+OPs=|gGfX5RD#;cFae#&+&3GoB7HbDE;-572&p*3#Mp?`Z3Z zZQpSrZm6nJ@=R_l)v+@Ya>pCeKC-r|4@4$xdUzLNFDZh~xiS$h!M#?&gwP33% zQ|ZcMm2WxUexAPLzXwzmh_=j$d~W8(>(=ZyL|MGe8n3c_1}>=H66t4b^`)KfRm|_x z!yR&ss;^h4teR)ylB0$$j5bUvs@aruOh%2%D`TztXL&JmOqs^G?HS%nj|KlvRup60XHiP^_QN>UlWzM3 z$9YuK8{g!hRPFM6YU6Gz`G8B#q6Yp{*nM!GzGw+H<;m8V!k}f~u7pn1o*yH}f%>YgHW7lav7)GART@xt) znd-vFi6`P5Dpy%2=$nGO4aCv1fk4v>K>4+sO*Z~5;~R|B`(;{Dv`oHd>Lh~Rr)$&~$#nO0wfUQq_?^(x(=k+(Vos$%OwF#K@Mka# zo;u{Ligavq&=)N(SnS>b{=C|PZ&#}?9A2-i63tN;!R)^Vp&RT=DqE0!aWaY+dX6sY z|F7o0GpwnmYgbW<6hWj{6$PXt(wiWV2aqNN5D-Lq@4YD~y{Hfn=^!n15JE?~^xg?o zI)oB>0y#VS`h3s(em~B4u5*4IF4FewJ+sz*-)rsJlgS=|`U_Kz%qvyXpSNMmn|WC%hsk!2dYkdZfy8Uq8W1G7EF|%HEr?JFb>hd*M*Nr$RDZg?Ti=MPVJM33vp-7 ze_f*)8mESQx9d=_^Wq=-?C5@YzilY?1M8#sqi)>Mr#$(O1^hL^Wah(8hwGwWMsr>23m={cm4`g$VWCUDK}_7z zc7jv?AzXETI=uDgglo$ShlTk2`uX{V-?=uO_`W!fB{{lnDD3hePY1p{O+GAK)Q#dL zJ+zlek6?0a^;&o`&W4GFTPTMeIj+UrQxcn%+$kM_R{u4+gxC$Km5OSyzG`vO5$U`1 zKw;XJfP?!S`n3MB{?pQ|)FgQd@cO1rUSVEGxB&T`i3!cuw{EyqtU+CGp5o&z-n(Hy zu^!X7%7w~hUNo>$Lo=pSQ-);{`aNX?uRwGipW^;FGFRfpNwq%)hW{TbgQprN_tp7?JE&UAL7{6^9E> zAIozb%GV%z!v!qHlLKw;FbX3C#xK8zKLTCPK zg6;CXg2ZcG0+YJ5?eNyx#oZ5Tp0YUBJnDPPLfN8Sj4JCd!5Rd7DE|^7}hg<7%rKk}*fiW}9aP zo9#X^?j43#Y|i+ftH$3F;UIhY0xx9P+LB`a)7@L#v=aI8xFyfbcT-IcEq5Xc!e=5A zpCXMGE-J$6PY)es87m0_-za5uyn37ov$}P^SbvL$(Te${d`I)i=Q?X<<5;m*ao7E$ z42(K9Zs%&8Jw9MZ3CjM+43U5eMy`Lh(bZWEPZ3fHPd!IH3PRc}UZ|1V5@ZoxsnJ2saZDTSRI|R4i-c{WqD&;ba&^wlx^zJJ3{PLYyay-^f<4lyT2rF zJ%&r=*}YdRm^S{+l!iSUir^{bp6M*MJWEJu1L3AkL>DJ)X>%!5;pZb1bwhVx@=Xaz zw>xv*S8e~x7`Q=zn6R@j+RrXbrj)#-!O$gSmi@3-SFMZKV(eSPDM~V>SiUo zV-J+qWM=#1g@s+${O(RzkoI+D+_U#2d9#dVBR~Ec=TIu(T4<4XcEbg0f4ou|E*)vR zGP*k8<-;g((96>1y>+V){59>br|u5#bXs3DdIyMg6rb=F`w?*Tnj%EM)4t?dMvu*M ziDlt89&_|ttZFdthRBpZ|K#-vnBa7{|FN1?i2i5{@)=Aw`sqiJQkF~eeBYDlDO+pu zpfa5)_8zg;L~mxq!C2;#=Cq8^?ph8$rM#gGa<@Tsc`Le!^wj4>pRZrNaU<$VWu-Pp zL3IWpx4BZ0_9(~Ilsm842;wFXpm?)OUmSacp~Q^T2cc#>F8LRS^Pd$PrlWyA`DD!{ z!*s?~*U=x75eBSH>``WvDmCKGVRS`*nbGQi=hb@(K+w-q(;@HhZYUN9v1_(lqIQ_RqtsRVl0Y*f8uZ zaGcNxzB1Q+-}=XXjzwZWGAk{@w#9DfboJ)%^qhKYvhw?dnDmK#uj1kvu6sB9xHrbV z@1A%W8%=Yxm8Eb~NE< zF!6ZOz|THbMjKFrB0sTRZBDJWmPp)l)+~;CGuKQhjj!Xcr4gK^9aI-^$B@S$uFQqs zha+%fuJU0thy7J;=VP9wh_6AX$8CNZQ~cb!J8$Xtk$8lpi`e zsK};Y^Y3^o@loVMTEVc}lM`8Tfdrck^31i#ptiV{H~~CsD~J;KNPS$emG#Thx&jL`KjYtj+hLgsdYBdB5?Pn#MNnj29uE$EI% ztM-l=5DV^*3E!u|%G3&vk7S^;-Ls^t3!-8oL{GqeKuJ2&G5i{dz zUc1s~_d7t#YlYeOFkJXm4o@OC)e7k;TrLian-39VU)CBw51ilJCnlg4ZFHs-@0E$9 zrHF}6E7Nx5^J1MT+>fM-3oX!UUVPcffBpV~8IFcs`XbsY+LVH9$86b|Jj3pMxpqU- zywBO#a!sTV7o}U~$oZvB<9c0EdPeYU;p5=}x-v>8^Vae)Q!2f)CP%`ER$&zPF&+Bz zD4a4%FDgV$P%%II8QnsJ0(t4dj2whXlr81y##x&u?of#z@c?g92Sckm`BR1;qP5ZQ zyK*-UUYW>lW4j3;+e5?W~-5Avt!(ZZvA)7izJ@uo>*A^Tn$2fkfUmnj4go| zx>_y$_$cTi!j1o^ER588swJ$?Wy*;8*+6vIb$MW|a=AZq0bqeCoVukgiH2o{raNlp z;2gPKe*U}%Z;c*o5)j0EuXnqtn5fXv!2Bkh^F{c`RE-}!9?!!8iSM<7YD8Dp#VO3>dFi#L+`~}?DrRVwMai5e{h3?fJBMYV>-PhiCy=okM`0PPcK9yB zz&I(CyYaaA-I?WIdbRHea<9@Tk(P-(gzPb$+jizuypLU6ewbRA9gMEV~u$a zlJv|;yp>|Ryk@=GTPT&HtcH0a3s&zc`Cz7+Yf&LK8#Tm4?k4!eOt2Hl$yZ0ezKh3f zL3t2UC@VYDHo~LYWAyQN4@YBDV6uFPy}g}ky#MT7)IDiyQN*r<=$+K#pia#uOCFbb zuSBtw+fFIL{Fye7W&Lj8pV+b1<;8W&r&Z{apT6~2%SY!rvAsT5HNEF@dfuv&cWe8G zrC?}&7CB#-0oXF6EV3lIGm{o$943G7vGElXkZXJEoqgp%_Irx=CgjM}^X~(v-#1^M z-z7w|V@`RhR=U4!Oj3X-g^ixxeYo|!i<>jUafLX#comef6SDWi?=3K4q+Rj>QkM_K z7(YhzIDB_tQbTh8krKl!EV;GjfhA|`k6G@6tc6Bj^uUc%p{*|Xn0ce>c``Niq@%;^ zTc2IpH!(7OW0UU>f9`i3`3hnL40&Kn>+^wOnrvhn&_%+`P^Aa=97(Q|J-DGkQ7v{= z1F#K8ns1HSY$uq59M7X!zo!U9_Cbq0^ZUv5A98e$1QS6PZ-^mLoc%BxyG#m83R_2w!P%J<=1bV*u$L5D~DZObmpUipP*=*)j z$HymW_}Q5lrEeY19Wn@{9`LU1Z)|mq{t%OB5!-`>;4c;6+bs9DSxGMxok*Dt#~Fum zP;zK2;OC^fONeBDc~2DU<$}U7jiu(ECVl-^iMhzk6pe)c9ix7u@0|KniK(Elm2Nm6h|5u3gS>)xEDg8*Z%n-AO5YM#m zRC0U-r0>5qBKvTk@)P?Noa4rfHcLS+VoT-Zknjk=D@f=Sg3NahOo*84Yw$lRyY^px zLkPXS+MsVT91aYPu-{e0!I=lXHrPvzikn>87#StP#9w$Rsq>p0TNI`^w2!fUnmdtJ znkQh>%9IRZ{v7UqOe_^(BS^x5@0;YhumQrqwRGQ!Rz@zH{MMEv zBH8ml$ThMp&2lz4QM#3i^z@NB*R+EcMt6uAD1WDm4(ANrzK`Q_e5jJhKp z?>-K%Lshl5<}JlPuqLn)59JqB>@_^(DX`Hx72axT0IoGMA?kZc*EM86$z7*F$2fZw zFFEct+&^Z1XPIyAG<;L3S_oet!s^y$%Ou&L?&j%4jdgd;&GN{04wJGjgNoW^uAXT{ zOG{=15$tvUiykQ2z|=lB(HI+{sQ%0v;}mPhXH;I_2$JHGbIR%qQ4UU2Ij?bh~CG z)1qWK6)K6B6dom5l$jvCXJ=IIbkqL2Xf%KL@ZtRj51bmv$;sKntp~>Im5L|g2!vA& zygnQb(-lvgmkrf>Ej%nvh#e0*5vvd3>Ou1!|BM?6FeKDj-ZAYQJzlhQRzuESIzzb> zf(408bwi_5LXP(lH}DZQ^iR~7N$$NW=33dXTl~El8usyCg--G-`G#Qsva6AV z{H0qz{Iw^%_Z*XX+iCI7Ku}zq1~1|9+jbgPcbQAF9RiuJ3d&2krUxo=2d-xpn~rX} ze*cJ?L5{^OD%P|wYA-#@3STI+$LQ!2KwzAMNl)a zGSW49d3jSaGbc%6ylX079?Rs8?1MBiegSo9yYd2*KmKkxU7@+C=&lW~MLW%x4WJkD z8^vn=O=Xe9tPbAcA5q$xzQOgiS2B07b3Jizg+Z!Tlxp1YPBrL@QVcuMF|xr~3iwuO z;4~X=oJqQWdfB%mekGqxH?{lZz)M6RjoIwyp02`8X^}?gXJ+5B#XClOL64;7^g24x zZMJ0x!v_=;>A!~LDbQ2NPrrd1d00Y>4orRuoQ%xA>@n+BL$36mG4ye* z;&Z5o+)itXAz+y=b9P@Nrz(LfyEjDtsJccw#Yq+P17fS~98+kbaX|H``~DYKhVJh? zWg^g;AI~-Ax@+6-?~1*jT34J&eY!r&MHbX%6j*f1SJBw}6{YF7|5JP{_w(g0Ekl0s znlsNgM*c62+&Jn)=&!|}RXA76oF`~a{TCDmx;#d>dvQxO+|5Z96}uR?%a?ACyNtj)-NwR}^QoiRH1%UPn=;zIDgBYPNzl z)D5lr79iw09PUVIGbw?jl$t8Z)=6XX9Ahw_3UT-#xq7$5BTSCXx${$zxzYf$dGoiJ z7u92FRcc2o_qyI#nBE^1bPRj8xFNYMGWQTC_vD)euM*}N`2YNCHF>~_2FLHA$D_jxQhP3I3NHe2cjXZ3sJ^~8OntiLND za!9?FkSzC)H`R;==*D?GLxcr7`Hv7h)mGR4sWc!yF}=^VvlDqJSK7;n=F=l?dE=9TzrTe$DK)^zlZGaO)Wh%a| z90_Hy9mA7`tt*8TYZlK}Cf6?w^>CSJx(0 zkd)SJPR-^VVpaXw>||F)eRsAjdcp$n)7BWDAw^ll8yb4E)$q0L3EAUS&uzfE zVvO>ZER_fi8JQO|oNmzI5(viacMH#SaoNTs#o+yW>^inuCb4F^jRw{vLq}OAt@{ja zLvBHQT;w33w*6UmcSRtW(U)7V7XYpD<1{B@Dzkn1tD@p z={;R2b<8TXO5HYOH|u?d?oQJyUQxEfE6{n6Wi1w&s)46e-VIz(J3AlQt|s%eKY0As z@TLBOF-4cSOQb6-CsH8QtU00LMP_w4fUAg1*92)cI&u zJVqF!|9x!C#TuTl&Ee2{8~$Y}9WSXS{2D#=ha4-Vv?{03_%-8#G4u=pZy* zbM5$@S7IJV*Q#kzywF|ms{4K|87?HJ3NYhIxW}UH6^=)`rLKY<<{m@h-5;kjZOd0) z(e;-sRlEn~SCZ!F&!@_H(_G>1pJUQW6H~Oe;<|g2Bw0M^ea9TNPtd!6&!5#!uf-Om ztgcsOYRTo&95%Xi8I`8k+{KHMW2OnsuBC&d&<6smO3YTEd4j?e7*T_q>ETaTQqh>> zh3jZ!P%$Zb3wd#YL8hU8m^!N<+MeOOeG_(m9HiZl``-6K zcdleYV0?RTRk&%a4~8IXX>}lzC14vwe~Fbuo=LCmN}>pJi&9V2-2)C39L`<6Is%UT zR9{F6yqIl(gXS@+!f^TKxpbz0kEkUX&Tts&iJObJ^7BY?Q6bUIW?9VR(wpMuk z^L3j1N_J60TPYRB=Jv)}HfgAnBFy3L8Ho2%kry4(^s|-cWVXo-;)YylGEFc%k@nsv znW)~26T~x`&(Oz=Hk}0}8C5qQ5{<$`kA;!jk!)@0kz09MI^kTnm z)^)}{ve9?OVy6+KR*wqu5cAryI`%u~_CKP=knYy)qzmox4_sPJ6rZODpogiwn$;TH zn$Bav#R1JE{cyh9vIKYE0bnm)r!8zc$k&eam^w;ws6Yvziu|TWijMne3PrZ^XHkmy z(g!yqB9aOx&n3+p%iL_nQK`!PWm{^}8!QfIE<$N*iWYX2+5~JZ$z-x3Z>@zac7kKk zkel#jMhEW-l7yzw*s3By-EzlOO8-Tp`k}8Uvvxb<5fZy*YRD|Ld2|1TTAR5oi-6;R zlj##91XMDsofqNOE`5nR(BN6rHRf<;h8e+J=;;`qyELNETlE`m6Qcb~y&F1GBg*}L zg&vnf1sBat{!yBjsD#?3pQ!pBk`kAmZKqVsu*GEQ@<9DY-|h)$w{dW4z$ZM46AzQE zS7{v4Wlw7C1dFLhNmtulR2uq^vAV?fYC9aa@&;5gyX(fd7mH4vT(l`01i%GVjWmnv zj=TI(X_ub@jz9^hGd2e2g{o`)XCDI=m@iV;^gizw?p{htqasT7qixcbrYwvW)0#F< zxTXfeDJFvO4w|9^AYc;AXKf&sS{D1*9!R}iw^lENrxE7u$ zZkUMY3LTODfZK*ginaf0sLGnrb_>{H?@avI@z}eEDjD^S`AAxV(yXhr>vnPdp#D5Y zai*!Xe{0@5$pYiiv>MDQMC&+bl~jL_*3f~n_x|LP>YnQL3sB&R$=!ySQ(L5e*195M zUuOP0o*?H=-gBY?-3Uge!Nzg*t=<9Aj@}k4q`6J0CIM-M48dKA%$v14vCPO7?gX#R zvjWMu29JP_t$^i;ixBCik<1dhlPOqTUkTh4NxM}@cPv&q$>g~H!AYpX+P~lF7`{nr zc+})I(}!XdY;w+uqVJh&@Y%t1->jRkciQZ-iEFi<3yy{9)>*`$NO{it1*e_XxTOLd zgLeI9X_=H_Lh4T{OZEaT7LBT#xHp1sZi~LsZ5TgyW1I3EzKVa$JltsQG}LH*(T&?F z>IgeQ*Uca53;VZ}9dE{zz#9xH`vm`f$xsV7QS4pn{^tvI7Uowte!N&17_XIx6)k0Tws1N$Z`LU zUASj#$!yaMBE{ou8FmRm!KB(nbNgHz6Z9RLS}#x_D(!V%zZUxWJ@>TvvYt7~I^Rp+ zerWsR+GoW`HP}XrU4Jd+_S{K^+GLqcjHLLk>qv<7Ca1nC%&jkQSJ!XwnAgAg?fZ&k zk72z|T55Q}0ApkQvcE|Jm?#U@v^%)R zXUxoQ;{f8NxbL=fQ%|2;e&@^5?-_oGTBOj)wp1IMZfgy3lqy_56yne{fsm}*souFZ zcV1j)a2S^>f(mqbS-HmD(+u)MW;NyNaoIAI6`=q%1XuEIq{lB7G7fUzV94eA^x~+2}CWA;?*%dvSeh zjdZfnC8i1P8N)h&+1N_1AQx`x zi!i#xliruc*kF##ay`c*{AW82&%7~96H=SoQ9=4mo(U0`<48Zu?f!Cik@oq9#ixcR z_?TgCW{FN#ak%%uf~YY1rG`Mj`AZpC(<+Jo(S(E3c&Wr8=On9ursY=t{wtysJn)!Hppkcbee_oQ=sFNGI>PxK6>8mE) z_uEzHONYy>g&Z#81zx%iVLx_HYxcWsnDAB^8=Loat<`RR*^*f6qSoc-wC-12Gu#{| z8u;eE5~yNfwE1G<xG(oLytFVP2?K&RW3 z_!p9J<{9@}tr@K@Rlsy>UN`3mz|I;iq&s7oeR3cI^S6R)Aga^eIYhi7V}T{s)+^Q) z2A*X|Rp~DD!c{?H?IP)^8fwF(SxvQa^9w3!$<_V-*>BV6FMT|I2c*@AFFh&c?o2_I z`he}(2_jZU^@40^TU`US{grAGGqdWwt3~t9Im%UbF6Y6N)?-vLUTBl#M&ss|>AtF$ zPEpL>{hBF_wa>;=-NSmSQp|R`A=(aNaIEbXe!C`i27eNIOREWn8#Z~7%j}#i?%=e5 zc-i4|EVj3{3tLWM9uJ{nv{^Z$)}0j7oAruV-bTj(QMTRCv$TWhdOIV%)4V!R-Hm^_ zbkpZ}d;d)kYUfv|GaN?G_-zkd_F6QR!h>G=9gS%h8$3iul{?H!A(6-qr*9SmOeb)U zG>3LHx?WRNw0(9qIYilRsyzEW8Iu=$M;E@6hj1?h{^=d+haftJ8;hkj6Fea|8O!}I zy#JUkOK{SFWJGua-5qT^&(+HDfPuUCje9rep<_Dry_`lE;~AL4Ns;$ikCFXyM^wu^ zySV#ysrQjJJ}J{TKa@Lc&bz>&eRHhfrT0cQVByB%qu(8eKMk~$1E()+fA7yzJjs8( z$G$7IRsFjR_$=|xP0gpaQmwWd23=hnjigLy{0@Vf6ZxO#mJGui!9+wvvZok_|2XDL zlBSExWvzA9LWzm7u_{4K^8dEgf0%?d!z}3EH*nGa(`Z=&e2af-Hi~rq(q(6P--z`aS*v{2YDYB*f{D|#G{ciSGqn=uM9N?9Y|L-mM{{V#l za})jpo6)Hnm)rqAgiB*83B8aA28~Q%clif+0D_ORA%s)Mdnm&2 zCmK26k@&MN>_3jL7Iz)*{WX%A{fMJ+Es!rH;r)Kn{+P(C6TP)O>o3rDbcX z+$=(;eu=VQ0{Lufya-vI(9lb%W3VWP<+t;5pZUKXl9-kzOdJ&QD5($KJAjT%a%h~o zopvfWe-p5Ikp>8T^NPfOaNj6NMMc%y*LPNc6~}5{l3yM6ylg_;nqIdvUD=Z=>BGgt zvuXbiu}b(tyMbgeR-mI_X$^rMSUv&}i`D=G6${#o>()4DJ2Cv}DgfZ@?a>D}+X@K~ z2mNbTrmxUmbAeWDXY7w5i*5NFF$!1?jI17HAb!0&XhrwfUDwl`}s%& z)j#U;yx!;a0odaC&K33#Yg<@s$z(lU*X>Ei|E=YDbD(9Gk~%+uTK#`AbUg`cH7Mgh z>K@Ifw%_9Lwv7U$TTB058{AO_)&R5al)CE{aKP<=jrq=PB0_p3LFHecbx&q>yiove z*vHxjAG}e>(6H2k$ch)vXn?aB$BUq@Fz6%q0*?0xp?nUw3WTK=Ou`CR*ytrdyl_Fks%!Rj@Y$7y=!>x#g|}rtQ=K zi$bBUBa&aVD8Z~|U6U#>=rA0rw8&8tQY8l6Y zasZ^AP>v1-4Z!*Gulde$!rh+~hE&d=<%MpV0HQmol36G7wG&@wjCY!$-Cyg#5QQ^6 ztdH-@X3CWEGr2eH;YIufGFjmWAu9@~lJWNVPHB!(MqC*PY&P6y%O2&#prWLqC7M~s z{Schp`O#TdgBW1RDvpKDOcBckv|J?Gyfu9Y4ph$uffHR} z%aw}Q7w9m1>@dJO0;mk&OjRS^RlLv**1<>o80;#ZS@|4`h?{s}{_~p!!^?TlXLL^? z3Ncv^vb(R<&U}Tc*MgLlD8XjiIr*6y@qlfBu$HSWY0PV_im)bETRs=e<5(6EgVvEe zwTe4X2N1~`Q~_rUVQC(FGd%s3Yr`I#4JdP2Yr{zk%%m<411d32PvVGe5vdvv<^TY3 z6<#Um{yL}!9*P=_JhIF*{uA#W;H{7MsU}@S@%27=Xblt`)jI z4rc{4;uZtN1EdM3n05;0up5Rd35UxHgQM5&Tlv=Ab;Y4T`T$Y@ZJkQ6aXKI)?2*rE zlkB6Qj~-_GXn|`1`PS-th(nFXySg+exOg#g(fxr-aAi-qUsr-&)fE2(lY|Gd*gOx8( z6%gV8toBd_Ai#cU+r$ml#Yc<{f4pA_^4G4$G_Ek90Qv~XEi6hgEbZz+CY7zVV9NJ7 z0QbL=79b$32r_9qKW0*Z0m3Su$F(W}@id(GI&Jpn zAj+eGQ-LMC$XZ=q7#lJ$fFpk5>>1;zgIB+)9|14}Qhq*1EJsHfpmtLlydFIuAv0(L z86Ci}8(=vT7Hz@{yYRz4taPiTVU9|IW|fB}ee&9E1i8AA1(TW^W)Xi$r5NX9XO|0i=p`!!HWR=20dE&bmPCKxHW5zr;Wo6R>UK kd%l{)DB}18#&RiL-6}9|=4exV1^82xSCuQ4eHHk>09z#yu>b%7 literal 0 HcmV?d00001 From 5ba2e9649bb79a9bd37f8e0883284640f6fd681a Mon Sep 17 00:00:00 2001 From: jiangbo Date: Thu, 16 Apr 2020 19:29:37 +0800 Subject: [PATCH 040/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DRDB=E5=88=86=E7=89=87?= =?UTF-8?q?=E8=AF=BB=E5=8F=96bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flinkx/reader/BaseDataReader.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java index 6b1c67bb71..f00482eb59 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java @@ -89,7 +89,8 @@ public void setSrcCols(List srcCols) { protected BaseDataReader(DataTransferConfig config, StreamExecutionEnvironment env) { this.env = env; - this.numPartitions = config.getJob().getSetting().getSpeed().getChannel(); + this.numPartitions = Math.max(config.getJob().getSetting().getSpeed().getChannel(), + config.getJob().getSetting().getSpeed().getReaderChannel()); this.bytes = config.getJob().getSetting().getSpeed().getBytes(); this.monitorUrls = config.getMonitorUrls(); this.restoreConfig = config.getJob().getSetting().getRestoreConfig(); From 69ac8b3c50f95f22974f66869308d93701ef7c6d Mon Sep 17 00:00:00 2001 From: jiangbo Date: Sat, 18 Apr 2020 16:47:31 +0800 Subject: [PATCH 041/136] =?UTF-8?q?1.ftp=E8=AF=BB=E5=8F=96=E9=97=AE?= =?UTF-8?q?=E9=A2=98=EF=BC=9B2.hdfs=E8=AF=BB=E5=8F=96=E6=96=87=E4=BB=B6?= =?UTF-8?q?=E8=BF=87=E6=BB=A4=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flinkx/ftp/EProtocol.java | 12 +++++++++++- .../main/java/com/dtstack/flinkx/ftp/FtpConfig.java | 5 +++-- .../com/dtstack/flinkx/ftp/FtpHandlerFactory.java | 2 +- .../dtstack/flinkx/ftp/reader/FtpInputFormat.java | 3 ++- .../flinkx/ftp/reader/FtpInputFormatBuilder.java | 5 +++++ .../flinkx/hdfs/reader/BaseHdfsInputFormat.java | 1 + .../dtstack/flinkx/hdfs/reader/HdfsPathFilter.java | 13 +++++++++++++ .../flinkx/hdfs/reader/HdfsTextInputFormat.java | 4 ++-- 8 files changed, 38 insertions(+), 7 deletions(-) diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java index 60d2fd8861..2cad866aba 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java @@ -14,5 +14,15 @@ public enum EProtocol { /** * 安全的FTP协议 */ - SFTP + SFTP; + + public static EProtocol getByName(String name) { + for (EProtocol value : EProtocol.values()) { + if (value.name().equalsIgnoreCase(name)) { + return value; + } + } + + return SFTP; + } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java index 23aea95dc2..fc58ca41e7 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java @@ -20,6 +20,7 @@ package com.dtstack.flinkx.ftp; import org.codehaus.jackson.annotate.JsonIgnoreProperties; +import org.codehaus.jackson.annotate.JsonProperty; import java.io.Serializable; @@ -42,11 +43,11 @@ public class FtpConfig implements Serializable { public String fieldDelimiter = FtpConfigConstants.DEFAULT_FIELD_DELIMITER; - public String path = "/"; + public String path; public String encoding = "UTF-8"; - public String connectPattern = FtpConfigConstants.STANDARD_FTP_PROTOCOL; + public String connectPattern = FtpConfigConstants.DEFAULT_FTP_CONNECT_PATTERN; public String host; diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java index 3a04f98a63..cc83e0343a 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java @@ -28,7 +28,7 @@ public class FtpHandlerFactory { public static IFtpHandler createFtpHandler(String protocolStr){ IFtpHandler ftpHandler; - EProtocol protocol = EProtocol.valueOf(protocolStr); + EProtocol protocol = EProtocol.getByName(protocolStr); if(EProtocol.SFTP.equals(protocol)) { ftpHandler = new SftpHandler(); } else { diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index 242ae70c3a..6f0f4471d3 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import java.io.IOException; @@ -73,7 +74,7 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception String path = ftpConfig.getPath(); if(path != null && path.length() > 0){ path = path.replace("\n","").replace("\r",""); - String[] pathArray = path.split(","); + String[] pathArray = StringUtils.split(path, ","); for (String p : pathArray) { files.addAll(ftpHandler.getFiles(p.trim())); } diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormatBuilder.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormatBuilder.java index db022437a1..c4f8508343 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormatBuilder.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormatBuilder.java @@ -3,6 +3,7 @@ import com.dtstack.flinkx.ftp.FtpConfig; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang3.StringUtils; import java.util.List; @@ -30,5 +31,9 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + if (StringUtils.isEmpty(format.ftpConfig.getPath())) { + throw new IllegalArgumentException("The property [path] cannot be empty or null"); + } } } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java index cb27045589..69213c0468 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java @@ -74,6 +74,7 @@ public void openInputFormat() throws IOException { protected JobConf buildConfig() { JobConf conf = FileSystemUtil.getJobConf(hadoopConfig, defaultFs); conf.set(HdfsPathFilter.KEY_REGEX, filterRegex); + conf.set(HdfsPathFilter.KEY_PATH, inputPath); FileSystemUtil.setHadoopUserName(conf); return conf; } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsPathFilter.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsPathFilter.java index ae208b9a6e..e6434a3b26 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsPathFilter.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsPathFilter.java @@ -37,9 +37,12 @@ public class HdfsPathFilter implements HdfsConfigurablePathFilter { private String regex; + private String parentPath; + private static final String DEFAULT_REGEX = ".*"; public static final String KEY_REGEX = "file.path.regexFilter"; + public static final String KEY_PATH = "file.path"; private static final PathFilter HIDDEN_FILE_FILTER = p -> { String name = p.getName(); @@ -65,12 +68,22 @@ public boolean accept(Path path) { return false; } + if (path.toUri().getPath().equals(parentPath)) { + return true; + } + return PATTERN.matcher(path.getName()).matches(); } @Override public void configure(JobConf jobConf) { this.regex = jobConf.get(KEY_REGEX); + + String path = jobConf.get(KEY_PATH); + if (StringUtils.isNotEmpty(path)) { + this.parentPath = new Path(path).toUri().getPath(); + } + compileRegex(); } } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java index fadd17430d..898fcf0895 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java @@ -65,8 +65,8 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti throw new IOException(e); } - JobConf jobConf = FileSystemUtil.getJobConf(hadoopConfig, defaultFs); - org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(buildConfig(), HdfsPathFilter.class); + JobConf jobConf = buildConfig(); + org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(jobConf, HdfsPathFilter.class); org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, inputPath); TextInputFormat inputFormat = new TextInputFormat(); From 4f8832afb558418576bc025045feff06339fbbac Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 24 Apr 2020 00:17:29 +0800 Subject: [PATCH 042/136] flink 1.10 --- flinkx-core/pom.xml | 1 + .../api/java/MyLocalStreamEnvironment.java | 7 +- .../main/java/com/dtstack/flinkx/Main.java | 20 +- .../com/dtstack/flinkx/enums/ClusterMode.java | 27 +- .../metrics/CustomPrometheusReporter.java | 4 +- .../com/dtstack/flinkx/options/Options.java | 53 +- .../StreamExecutionEnvironment.java | 1822 ----------------- .../flinkx/launcher/ClusterClientFactory.java | 62 +- .../com/dtstack/flinkx/launcher/Launcher.java | 161 +- .../FlinkPerJobResourceUtil.java | 13 +- .../PerJobClusterClientBuilder.java | 58 +- .../launcher/perJob/PerJobSubmitter.java | 62 + .../launcher/perjob/PerJobSubmitter.java | 94 - .../deployment/ClusterSpecification.java | 259 --- .../flink/client/program/JobWithJars.java | 161 -- .../flink/client/program/PackagedProgram.java | 773 ------- .../yarn/AbstractYarnClusterDescriptor.java | 1766 ---------------- pom.xml | 17 +- 18 files changed, 300 insertions(+), 5060 deletions(-) delete mode 100644 flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java rename flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/{perjob => perJob}/FlinkPerJobResourceUtil.java (87%) rename flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/{perjob => perJob}/PerJobClusterClientBuilder.java (66%) create mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java delete mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java delete mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java delete mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java delete mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java delete mode 100644 flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 223aae3824..f859484e6f 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -200,6 +200,7 @@ org.slf4j:* org.apache.httpcomponents:* io.prometheus:* + commons-*:* diff --git a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java index 56fb01de1c..226757fa10 100644 --- a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java +++ b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; @@ -96,6 +95,7 @@ public MyLocalStreamEnvironment(@Nonnull Configuration configuration) { setParallelism(1); } + @Override protected Configuration getConfiguration() { return configuration; } @@ -116,7 +116,6 @@ public JobExecutionResult execute(String jobName) throws Exception { JobGraph jobGraph = streamGraph.getJobGraph(); jobGraph.setClasspaths(classpaths); - jobGraph.setAllowQueuedScheduling(true); if (settings != null){ jobGraph.setSavepointRestoreSettings(settings); @@ -124,8 +123,8 @@ public JobExecutionResult execute(String jobName) throws Exception { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); - configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS.key(), jobGraph.getMaximumParallelism()); + configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), "512M"); + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined configuration.addAll(this.configuration); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 47e20d2b28..95fab3010b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -15,16 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.dtstack.flinkx; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.classloader.ClassLoaderManager; -import com.dtstack.flinkx.config.ContentConfig; -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.SpeedConfig; -import com.dtstack.flinkx.config.RestartConfig; -import com.dtstack.flinkx.config.TestConfig; +import com.dtstack.flinkx.config.*; import com.dtstack.flinkx.constants.ConfigConstant; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.reader.BaseDataReader; @@ -38,7 +33,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; -import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; @@ -46,16 +40,17 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.lang.reflect.Field; import java.net.URL; import java.net.URLDecoder; -import java.util.*; +import java.util.ArrayList; +import java.util.Map; +import java.util.Properties; +import java.util.Set; import java.util.concurrent.TimeUnit; /** @@ -205,7 +200,8 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla if(env instanceof MyLocalStreamEnvironment){ ((MyLocalStreamEnvironment) env).setClasspaths(new ArrayList<>(classPathSet)); - } else if(env instanceof StreamContextEnvironment){ + } + /* else if(env instanceof StreamContextEnvironment){ Field field = env.getClass().getDeclaredField("ctx"); field.setAccessible(true); ContextEnvironment contextEnvironment= (ContextEnvironment) field.get(env); @@ -220,7 +216,7 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla contextEnvironment.getClasspaths().add(url); } } - } + }*/ } private static Properties parseConf(String confStr) throws Exception{ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java index a12446768c..994c415f9a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.enums; +import org.apache.commons.lang3.StringUtils; + /** * This class defines three running mode of FlinkX * @@ -29,27 +31,42 @@ public enum ClusterMode { /** * 本地模式运行 */ - local(0), + local(0 , "local"), /** * flink集群 standalone模式 */ - standalone(1), + standalone(1, "standalone"), /** * 在已经启动在yarn上的flink session里上运行 */ - yarn(2), + yarn(2, "yarn"), /** * 在yarn上单独启动flink session运行 */ - yarnPer(3); + yarnPer(3, "yarnPer"); private int type; - ClusterMode(int type){ + private String name; + + ClusterMode(int type, String name){ this.type = type; + this.name = name; + } + + public static ClusterMode getByName(String name){ + if(StringUtils.isBlank(name)){ + throw new IllegalArgumentException("ClusterMode name cannot be null or empty"); + } + switch (name){ + case "standalone": return standalone; + case "yarn": return yarn; + case "yarnPer": return yarnPer; + default: return local; + } } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java index 3735cc7c64..ae03b6d4af 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; +import org.apache.flink.runtime.metrics.groups.ReporterScopedSettings; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.AbstractID; import org.apache.flink.util.StringUtils; @@ -137,7 +138,8 @@ public void open() { public void registerMetric(Accumulator accumulator, String name) { name = Metrics.METRIC_GROUP_KEY_FLINKX + "_" + name; - FrontMetricGroup front = new FrontMetricGroup>(0, (AbstractMetricGroup)context.getMetricGroup()); + ReporterScopedSettings reporterScopedSettings = new ReporterScopedSettings(0, ',', null); + FrontMetricGroup front = new FrontMetricGroup>(reporterScopedSettings, (AbstractMetricGroup)context.getMetricGroup()); notifyOfAddedMetric(new SimpleAccumulatorGauge<>(accumulator), name, front); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 9b57f5583b..92c8a4db27 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -19,6 +19,12 @@ package com.dtstack.flinkx.options; import com.dtstack.flinkx.enums.ClusterMode; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.yarn.configuration.YarnConfigOptions; /** * This class define commandline options for the Launcher program @@ -64,9 +70,6 @@ public class Options { @OptionRequired(description = "env properties") private String confProp = "{}"; - /** - * savepoint - */ @OptionRequired(description = "savepoint path") private String s; @@ -76,6 +79,29 @@ public class Options { @OptionRequired(description = "applicationId on yarn cluster") private String appId; + private Configuration flinkConfiguration = null; + + public Configuration loadFlinkConfiguration() { + if(flinkConfiguration == null){ + flinkConfiguration = StringUtils.isEmpty(flinkconf) ? new Configuration() : GlobalConfiguration.loadConfiguration(flinkconf); + if (StringUtils.isNotBlank(queue)) { + flinkConfiguration.setString(YarnConfigOptions.APPLICATION_QUEUE, queue); + } + if (StringUtils.isNotBlank(jobid)) { + flinkConfiguration.setString(YarnConfigOptions.APPLICATION_NAME, jobid); + } + if(StringUtils.isNotBlank(yarnconf)){ + flinkConfiguration.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnconf); + } + if("classpath".equalsIgnoreCase(pluginLoadMode)){ + flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "child-first"); + }else{ + flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first"); + } + } + return flinkConfiguration; + } + public String getAppId() { return appId; } @@ -195,4 +221,25 @@ public String getPluginLoadMode() { public void setPluginLoadMode(String pluginLoadMode) { this.pluginLoadMode = pluginLoadMode; } + + @Override + public String toString() { + return "Options{" + + "mode='" + mode + '\'' + + ", job='" + job + '\'' + + ", monitor='" + monitor + '\'' + + ", jobid='" + jobid + '\'' + + ", flinkconf='" + flinkconf + '\'' + + ", pluginRoot='" + pluginRoot + '\'' + + ", yarnconf='" + yarnconf + '\'' + + ", parallelism='" + parallelism + '\'' + + ", priority='" + priority + '\'' + + ", queue='" + queue + '\'' + + ", flinkLibJar='" + flinkLibJar + '\'' + + ", confProp='" + confProp + '\'' + + ", s='" + s + '\'' + + ", pluginLoadMode='" + pluginLoadMode + '\'' + + ", appId='" + appId + '\'' + + '}'; + } } diff --git a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java deleted file mode 100644 index e331b3e672..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ /dev/null @@ -1,1822 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.api.environment; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.Public; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.common.functions.InvalidTypesException; -import org.apache.flink.api.common.functions.StoppableFunction; -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.io.FilePathFilter; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.ClosureCleaner; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.MissingTypeInfo; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.client.program.ContextEnvironment; -import org.apache.flink.client.program.OptimizerPlanEnvironment; -import org.apache.flink.client.program.PreviewPlanEnvironment; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.StateBackend; -import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.source.*; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.api.graph.StreamGraphGenerator; -import org.apache.flink.streaming.api.operators.StoppableStreamSource; -import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.api.transformations.StreamTransformation; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SplittableIterator; -import org.apache.flink.util.StringUtils; - -import com.esotericsoftware.kryo.Serializer; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * The StreamExecutionEnvironment is the context in which a streaming program is executed. A - * {@link LocalStreamEnvironment} will cause execution in the current JVM, a - * {@link RemoteStreamEnvironment} will cause execution on a remote setup. - * - *

The environment provides methods to control the job execution (such as setting the parallelism - * or the fault tolerance/checkpointing parameters) and to interact with the outside world (data access). - * - * @see org.apache.flink.streaming.api.environment.LocalStreamEnvironment - * @see org.apache.flink.streaming.api.environment.RemoteStreamEnvironment - */ -@Public -public abstract class StreamExecutionEnvironment { - - /** The default name to use for a streaming job if no other name has been specified. */ - public static final String DEFAULT_JOB_NAME = "Flink Streaming Job"; - - /** The time characteristic that is used if none other is set. */ - private static final TimeCharacteristic DEFAULT_TIME_CHARACTERISTIC = TimeCharacteristic.ProcessingTime; - - /** The default buffer timeout (max delay of records in the network stack). */ - private static final long DEFAULT_NETWORK_BUFFER_TIMEOUT = 100L; - - /** - * The environment of the context (local by default, cluster if invoked through command line). - */ - private static ThreadLocal contextEnvironmentFactory = new ThreadLocal<>(); - - /** The default parallelism used when creating a local environment. */ - private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors(); - - // ------------------------------------------------------------------------ - - /** The execution configuration for this environment. */ - private final ExecutionConfig config = new ExecutionConfig(); - - /** Settings that control the checkpointing behavior. */ - private final CheckpointConfig checkpointCfg = new CheckpointConfig(); - - protected final List> transformations = new ArrayList<>(); - - private long bufferTimeout = DEFAULT_NETWORK_BUFFER_TIMEOUT; - - protected boolean isChainingEnabled = true; - - /** The state backend used for storing k/v state and state snapshots. */ - private StateBackend defaultStateBackend; - - /** The time characteristic used by the data streams. */ - private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC; - - protected final List> cacheFile = new ArrayList<>(); - - - // -------------------------------------------------------------------------------------------- - // Constructor and Properties - // -------------------------------------------------------------------------------------------- - - /** - * Gets the config object. - */ - public ExecutionConfig getConfig() { - return config; - } - - /** - * Get the list of cached files that were registered for distribution among the task managers. - */ - public List> getCachedFiles() { - return cacheFile; - } - - /** - * Sets the parallelism for operations executed through this environment. - * Setting a parallelism of x here will cause all operators (such as map, - * batchReduce) to run with x parallel instances. This method overrides the - * default parallelism for this environment. The - * {@link LocalStreamEnvironment} uses by default a value equal to the - * number of hardware contexts (CPU cores / threads). When executing the - * program via the command line client from a JAR file, the default degree - * of parallelism is the one configured for that setup. - * - * @param parallelism The parallelism - */ - public StreamExecutionEnvironment setParallelism(int parallelism) { - config.setParallelism(parallelism); - return this; - } - - /** - * Sets the maximum degree of parallelism defined for the program. The upper limit (inclusive) - * is Short.MAX_VALUE. - * - *

The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also - * defines the number of key groups used for partitioned state. - * - * @param maxParallelism Maximum degree of parallelism to be used for the program., - * with 0 < maxParallelism <= 2^15 - 1 - */ - public StreamExecutionEnvironment setMaxParallelism(int maxParallelism) { - Preconditions.checkArgument(maxParallelism > 0 && - maxParallelism <= KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM, - "maxParallelism is out of bounds 0 < maxParallelism <= " + - KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM + ". Found: " + maxParallelism); - - config.setMaxParallelism(maxParallelism); - return this; - } - - /** - * Gets the parallelism with which operation are executed by default. - * Operations can individually override this value to use a specific - * parallelism. - * - * @return The parallelism used by operations, unless they override that - * value. - */ - public int getParallelism() { - return config.getParallelism(); - } - - /** - * Gets the maximum degree of parallelism defined for the program. - * - *

The maximum degree of parallelism specifies the upper limit for dynamic scaling. It also - * defines the number of key groups used for partitioned state. - * - * @return Maximum degree of parallelism - */ - public int getMaxParallelism() { - return config.getMaxParallelism(); - } - - /** - * Sets the maximum time frequency (milliseconds) for the flushing of the - * output buffers. By default the output buffers flush frequently to provide - * low latency and to aid smooth developer experience. Setting the parameter - * can result in three logical modes: - * - *

- * - * @param timeoutMillis - * The maximum time between two output flushes. - */ - public StreamExecutionEnvironment setBufferTimeout(long timeoutMillis) { - if (timeoutMillis < -1) { - throw new IllegalArgumentException("Timeout of buffer must be non-negative or -1"); - } - - this.bufferTimeout = timeoutMillis; - return this; - } - - /** - * Gets the maximum time frequency (milliseconds) for the flushing of the - * output buffers. For clarification on the extremal values see - * {@link #setBufferTimeout(long)}. - * - * @return The timeout of the buffer. - */ - public long getBufferTimeout() { - return this.bufferTimeout; - } - - /** - * Disables operator chaining for streaming operators. Operator chaining - * allows non-shuffle operations to be co-located in the same thread fully - * avoiding serialization and de-serialization. - * - * @return StreamExecutionEnvironment with chaining disabled. - */ - @PublicEvolving - public StreamExecutionEnvironment disableOperatorChaining() { - this.isChainingEnabled = false; - return this; - } - - /** - * Returns whether operator chaining is enabled. - * - * @return {@code true} if chaining is enabled, false otherwise. - */ - @PublicEvolving - public boolean isChainingEnabled() { - return isChainingEnabled; - } - - // ------------------------------------------------------------------------ - // Checkpointing Settings - // ------------------------------------------------------------------------ - - /** - * Gets the checkpoint config, which defines values like checkpoint interval, delay between - * checkpoints, etc. - * - * @return The checkpoint config. - */ - public CheckpointConfig getCheckpointConfig() { - return checkpointCfg; - } - - /** - * Enables checkpointing for the streaming job. The distributed state of the streaming - * dataflow will be periodically snapshotted. In case of a failure, the streaming - * dataflow will be restarted from the latest completed checkpoint. This method selects - * {@link CheckpointingMode#EXACTLY_ONCE} guarantees. - * - *

The job draws checkpoints periodically, in the given interval. The state will be - * stored in the configured state backend. - * - *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at - * the moment. For that reason, iterative jobs will not be started if used - * with enabled checkpointing. To override this mechanism, use the - * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method. - * - * @param interval Time interval between state checkpoints in milliseconds. - */ - public StreamExecutionEnvironment enableCheckpointing(long interval) { - checkpointCfg.setCheckpointInterval(interval); - return this; - } - - /** - * Enables checkpointing for the streaming job. The distributed state of the streaming - * dataflow will be periodically snapshotted. In case of a failure, the streaming - * dataflow will be restarted from the latest completed checkpoint. - * - *

The job draws checkpoints periodically, in the given interval. The system uses the - * given {@link CheckpointingMode} for the checkpointing ("exactly once" vs "at least once"). - * The state will be stored in the configured state backend. - * - *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at - * the moment. For that reason, iterative jobs will not be started if used - * with enabled checkpointing. To override this mechanism, use the - * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method. - * - * @param interval - * Time interval between state checkpoints in milliseconds. - * @param mode - * The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed. - */ - public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode) { - checkpointCfg.setCheckpointingMode(mode); - checkpointCfg.setCheckpointInterval(interval); - return this; - } - - /** - * Enables checkpointing for the streaming job. The distributed state of the streaming - * dataflow will be periodically snapshotted. In case of a failure, the streaming - * dataflow will be restarted from the latest completed checkpoint. - * - *

The job draws checkpoints periodically, in the given interval. The state will be - * stored in the configured state backend. - * - *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at - * the moment. If the "force" parameter is set to true, the system will execute the - * job nonetheless. - * - * @param interval - * Time interval between state checkpoints in millis. - * @param mode - * The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed. - * @param force - * If true checkpointing will be enabled for iterative jobs as well. - * - * @deprecated Use {@link #enableCheckpointing(long, CheckpointingMode)} instead. - * Forcing checkpoints will be removed in the future. - */ - @Deprecated - @SuppressWarnings("deprecation") - @PublicEvolving - public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) { - checkpointCfg.setCheckpointingMode(mode); - checkpointCfg.setCheckpointInterval(interval); - checkpointCfg.setForceCheckpointing(force); - return this; - } - - /** - * Enables checkpointing for the streaming job. The distributed state of the streaming - * dataflow will be periodically snapshotted. In case of a failure, the streaming - * dataflow will be restarted from the latest completed checkpoint. This method selects - * {@link CheckpointingMode#EXACTLY_ONCE} guarantees. - * - *

The job draws checkpoints periodically, in the default interval. The state will be - * stored in the configured state backend. - * - *

NOTE: Checkpointing iterative streaming dataflows in not properly supported at - * the moment. For that reason, iterative jobs will not be started if used - * with enabled checkpointing. To override this mechanism, use the - * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method. - * - * @deprecated Use {@link #enableCheckpointing(long)} instead. - */ - @Deprecated - @PublicEvolving - public StreamExecutionEnvironment enableCheckpointing() { - checkpointCfg.setCheckpointInterval(500); - return this; - } - - /** - * Returns the checkpointing interval or -1 if checkpointing is disabled. - * - *

Shorthand for {@code getCheckpointConfig().getCheckpointInterval()}. - * - * @return The checkpointing interval or -1 - */ - public long getCheckpointInterval() { - return checkpointCfg.getCheckpointInterval(); - } - - /** - * Returns whether checkpointing is force-enabled. - * - * @deprecated Forcing checkpoints will be removed in future version. - */ - @Deprecated - @SuppressWarnings("deprecation") - @PublicEvolving - public boolean isForceCheckpointing() { - return checkpointCfg.isForceCheckpointing(); - } - - /** - * Returns the checkpointing mode (exactly-once vs. at-least-once). - * - *

Shorthand for {@code getCheckpointConfig().getCheckpointingMode()}. - * - * @return The checkpoint mode - */ - public CheckpointingMode getCheckpointingMode() { - return checkpointCfg.getCheckpointingMode(); - } - - /** - * Sets the state backend that describes how to store and checkpoint operator state. It defines - * both which data structures hold state during execution (for example hash tables, RockDB, - * or other data stores) as well as where checkpointed data will be persisted. - * - *

State managed by the state backend includes both keyed state that is accessible on - * {@link org.apache.flink.streaming.api.datastream.KeyedStream keyed streams}, as well as - * state maintained directly by the user code that implements - * {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction CheckpointedFunction}. - * - *

The {@link org.apache.flink.runtime.state.memory.MemoryStateBackend} for example - * maintains the state in heap memory, as objects. It is lightweight without extra dependencies, - * but can checkpoint only small states (some counters). - * - *

In contrast, the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend} - * stores checkpoints of the state (also maintained as heap objects) in files. When using a replicated - * file system (like HDFS, S3, MapR FS, Alluxio, etc) this will guarantee that state is not lost upon - * failures of individual nodes and that streaming program can be executed highly available and strongly - * consistent (assuming that Flink is run in high-availability mode). - * - * @return This StreamExecutionEnvironment itself, to allow chaining of function calls. - * - * @see #getStateBackend() - */ - @PublicEvolving - public StreamExecutionEnvironment setStateBackend(StateBackend backend) { - this.defaultStateBackend = Preconditions.checkNotNull(backend); - return this; - } - - /** - * @deprecated Use {@link #setStateBackend(StateBackend)} instead. - */ - @Deprecated - @PublicEvolving - public StreamExecutionEnvironment setStateBackend(AbstractStateBackend backend) { - this.defaultStateBackend = Preconditions.checkNotNull(backend); - return this; - } - - /** - * Gets the state backend that defines how to store and checkpoint state. - * - * @see #setStateBackend(StateBackend) - */ - @PublicEvolving - public StateBackend getStateBackend() { - return defaultStateBackend; - } - - /** - * Sets the restart strategy configuration. The configuration specifies which restart strategy - * will be used for the execution graph in case of a restart. - * - * @param restartStrategyConfiguration Restart strategy configuration to be set - */ - @PublicEvolving - public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { - config.setRestartStrategy(restartStrategyConfiguration); - } - - /** - * Returns the specified restart strategy configuration. - * - * @return The restart strategy configuration to be used - */ - @PublicEvolving - public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { - return config.getRestartStrategy(); - } - - /** - * Sets the number of times that failed tasks are re-executed. A value of - * zero effectively disables fault tolerance. A value of {@code -1} - * indicates that the system default value (as defined in the configuration) - * should be used. - * - * @param numberOfExecutionRetries - * The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #setRestartStrategy}. The - * {@link RestartStrategies#fixedDelayRestart(int, Time)} contains the number of - * execution retries. - */ - @Deprecated - @PublicEvolving - public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { - config.setNumberOfExecutionRetries(numberOfExecutionRetries); - } - - /** - * Gets the number of times the system will try to re-execute failed tasks. - * A value of {@code -1} indicates that the system default value (as defined - * in the configuration) should be used. - * - * @return The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #getRestartStrategy}. - */ - @Deprecated - @PublicEvolving - public int getNumberOfExecutionRetries() { - return config.getNumberOfExecutionRetries(); - } - - // -------------------------------------------------------------------------------------------- - // Registry for types and serializers - // -------------------------------------------------------------------------------------------- - - /** - * Adds a new Kryo default serializer to the Runtime. - * - *

Note that the serializer instance must be serializable (as defined by - * java.io.Serializable), because it may be distributed to the worker nodes - * by java serialization. - * - * @param type - * The class of the types serialized with the given serializer. - * @param serializer - * The serializer to use. - */ - public & Serializable>void addDefaultKryoSerializer(Class type, T serializer) { - config.addDefaultKryoSerializer(type, serializer); - } - - /** - * Adds a new Kryo default serializer to the Runtime. - * - * @param type - * The class of the types serialized with the given serializer. - * @param serializerClass - * The class of the serializer to use. - */ - public void addDefaultKryoSerializer(Class type, Class> serializerClass) { - config.addDefaultKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with a Kryo Serializer. - * - *

Note that the serializer instance must be serializable (as defined by - * java.io.Serializable), because it may be distributed to the worker nodes - * by java serialization. - * - * @param type - * The class of the types serialized with the given serializer. - * @param serializer - * The serializer to use. - */ - public & Serializable>void registerTypeWithKryoSerializer(Class type, T serializer) { - config.registerTypeWithKryoSerializer(type, serializer); - } - - /** - * Registers the given Serializer via its class as a serializer for the - * given type at the KryoSerializer. - * - * @param type - * The class of the types serialized with the given serializer. - * @param serializerClass - * The class of the serializer to use. - */ - @SuppressWarnings("rawtypes") - public void registerTypeWithKryoSerializer(Class type, Class serializerClass) { - config.registerTypeWithKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with the serialization stack. If the type is - * eventually serialized as a POJO, then the type is registered with the - * POJO serializer. If the type ends up being serialized with Kryo, then it - * will be registered at Kryo to make sure that only tags are written. - * - * @param type - * The class of the type to register. - */ - public void registerType(Class type) { - if (type == null) { - throw new NullPointerException("Cannot register null type class."); - } - - TypeInformation typeInfo = TypeExtractor.createTypeInfo(type); - - if (typeInfo instanceof PojoTypeInfo) { - config.registerPojoType(type); - } else { - config.registerKryoType(type); - } - } - - // -------------------------------------------------------------------------------------------- - // Time characteristic - // -------------------------------------------------------------------------------------------- - - /** - * Sets the time characteristic for all streams create from this environment, e.g., processing - * time, event time, or ingestion time. - * - *

If you set the characteristic to IngestionTime of EventTime this will set a default - * watermark update interval of 200 ms. If this is not applicable for your application - * you should change it using {@link ExecutionConfig#setAutoWatermarkInterval(long)}. - * - * @param characteristic The time characteristic. - */ - @PublicEvolving - public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) { - this.timeCharacteristic = Preconditions.checkNotNull(characteristic); - if (characteristic == TimeCharacteristic.ProcessingTime) { - getConfig().setAutoWatermarkInterval(0); - } else { - getConfig().setAutoWatermarkInterval(200); - } - } - - /** - * Gets the time characteristic. - * - * @see #setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic) - * - * @return The time characteristic. - */ - @PublicEvolving - public TimeCharacteristic getStreamTimeCharacteristic() { - return timeCharacteristic; - } - - // -------------------------------------------------------------------------------------------- - // Data stream creations - // -------------------------------------------------------------------------------------------- - - /** - * Creates a new data stream that contains a sequence of numbers. This is a parallel source, - * if you manually set the parallelism to {@code 1} - * (using {@link org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator#setParallelism(int)}) - * the generated sequence of elements is in order. - * - * @param from - * The number to start at (inclusive) - * @param to - * The number to stop at (inclusive) - * @return A data stream, containing all number in the [from, to] interval - */ - public DataStreamSource generateSequence(long from, long to) { - if (from > to) { - throw new IllegalArgumentException("Start of sequence must not be greater than the end"); - } - return addSource(new StatefulSequenceSource(from, to), "Sequence Source"); - } - - /** - * Creates a new data stream that contains the given elements. The elements must all be of the - * same type, for example, all of the {@link String} or {@link Integer}. - * - *

The framework will try and determine the exact type from the elements. In case of generic - * elements, it may be necessary to manually supply the type information via - * {@link #fromCollection(java.util.Collection, org.apache.flink.api.common.typeinfo.TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data stream source, i.e. a data - * stream source with a degree of parallelism one. - * - * @param data - * The array of elements to create the data stream from. - * @param - * The type of the returned data stream - * @return The data stream representing the given array of elements - */ - @SafeVarargs - public final DataStreamSource fromElements(OUT... data) { - if (data.length == 0) { - throw new IllegalArgumentException("fromElements needs at least one element as argument"); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForObject(data[0]); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + data[0].getClass().getName() - + "; please specify the TypeInformation manually via " - + "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - return fromCollection(Arrays.asList(data), typeInfo); - } - - /** - * Creates a new data set that contains the given elements. The framework will determine the type according to the - * based type user supplied. The elements should be the same or be the subclass to the based type. - * The sequence of elements must not be empty. - * Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with a - * degree of parallelism one. - * - * @param type - * The based class type in the collection. - * @param data - * The array of elements to create the data stream from. - * @param - * The type of the returned data stream - * @return The data stream representing the given array of elements - */ - @SafeVarargs - public final DataStreamSource fromElements(Class type, OUT... data) { - if (data.length == 0) { - throw new IllegalArgumentException("fromElements needs at least one element as argument"); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForClass(type); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + type.getName() - + "; please specify the TypeInformation manually via " - + "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - return fromCollection(Arrays.asList(data), typeInfo); - } - - /** - * Creates a data stream from the given non-empty collection. The type of the data stream is that of the - * elements in the collection. - * - *

The framework will try and determine the exact type from the collection elements. In case of generic - * elements, it may be necessary to manually supply the type information via - * {@link #fromCollection(java.util.Collection, org.apache.flink.api.common.typeinfo.TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with - * parallelism one. - * - * @param data - * The collection of elements to create the data stream from. - * @param - * The generic type of the returned data stream. - * @return - * The data stream representing the given collection - */ - public DataStreamSource fromCollection(Collection data) { - Preconditions.checkNotNull(data, "Collection must not be null"); - if (data.isEmpty()) { - throw new IllegalArgumentException("Collection must not be empty"); - } - - OUT first = data.iterator().next(); - if (first == null) { - throw new IllegalArgumentException("Collection must not contain null elements"); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForObject(first); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + first.getClass() - + "; please specify the TypeInformation manually via " - + "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - return fromCollection(data, typeInfo); - } - - /** - * Creates a data stream from the given non-empty collection. - * - *

Note that this operation will result in a non-parallel data stream source, - * i.e., a data stream source with parallelism one. - * - * @param data - * The collection of elements to create the data stream from - * @param typeInfo - * The TypeInformation for the produced data stream - * @param - * The type of the returned data stream - * @return The data stream representing the given collection - */ - public DataStreamSource fromCollection(Collection data, TypeInformation typeInfo) { - Preconditions.checkNotNull(data, "Collection must not be null"); - - // must not have null elements and mixed elements - FromElementsFunction.checkCollection(data, typeInfo.getTypeClass()); - - SourceFunction function; - try { - function = new FromElementsFunction<>(typeInfo.createSerializer(getConfig()), data); - } - catch (IOException e) { - throw new RuntimeException(e.getMessage(), e); - } - return addSource(function, "Collection Source", typeInfo).setParallelism(1); - } - - /** - * Creates a data stream from the given iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, - * the type of data returned by the iterator must be given explicitly in the form of the type - * class (this is due to the fact that the Java compiler erases the generic type information). - * - *

Note that this operation will result in a non-parallel data stream source, i.e., - * a data stream source with a parallelism of one. - * - * @param data - * The iterator of elements to create the data stream from - * @param type - * The class of the data produced by the iterator. Must not be a generic class. - * @param - * The type of the returned data stream - * @return The data stream representing the elements in the iterator - * @see #fromCollection(java.util.Iterator, org.apache.flink.api.common.typeinfo.TypeInformation) - */ - public DataStreamSource fromCollection(Iterator data, Class type) { - return fromCollection(data, TypeExtractor.getForClass(type)); - } - - /** - * Creates a data stream from the given iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, - * the type of data returned by the iterator must be given explicitly in the form of the type - * information. This method is useful for cases where the type is generic. - * In that case, the type class (as given in - * {@link #fromCollection(java.util.Iterator, Class)} does not supply all type information. - * - *

Note that this operation will result in a non-parallel data stream source, i.e., - * a data stream source with parallelism one. - * - * @param data - * The iterator of elements to create the data stream from - * @param typeInfo - * The TypeInformation for the produced data stream - * @param - * The type of the returned data stream - * @return The data stream representing the elements in the iterator - */ - public DataStreamSource fromCollection(Iterator data, TypeInformation typeInfo) { - Preconditions.checkNotNull(data, "The iterator must not be null"); - - SourceFunction function = new FromIteratorFunction<>(data); - return addSource(function, "Collection Source", typeInfo); - } - - /** - * Creates a new data stream that contains elements in the iterator. The iterator is splittable, - * allowing the framework to create a parallel data stream source that returns the elements in - * the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type - * of data returned by the iterator must be given explicitly in the form of the type class - * (this is due to the fact that the Java compiler erases the generic type information). - * - * @param iterator - * The iterator that produces the elements of the data stream - * @param type - * The class of the data produced by the iterator. Must not be a generic class. - * @param - * The type of the returned data stream - * @return A data stream representing the elements in the iterator - */ - public DataStreamSource fromParallelCollection(SplittableIterator iterator, Class type) { - return fromParallelCollection(iterator, TypeExtractor.getForClass(type)); - } - - /** - * Creates a new data stream that contains elements in the iterator. The iterator is splittable, - * allowing the framework to create a parallel data stream source that returns the elements in - * the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type - * of data returned by the iterator must be given explicitly in the form of the type - * information. This method is useful for cases where the type is generic. In that case, the - * type class (as given in - * {@link #fromParallelCollection(org.apache.flink.util.SplittableIterator, Class)} does not - * supply all type information. - * - * @param iterator - * The iterator that produces the elements of the data stream - * @param typeInfo - * The TypeInformation for the produced data stream. - * @param - * The type of the returned data stream - * @return A data stream representing the elements in the iterator - */ - public DataStreamSource fromParallelCollection(SplittableIterator iterator, TypeInformation - typeInfo) { - return fromParallelCollection(iterator, typeInfo, "Parallel Collection Source"); - } - - // private helper for passing different names - private DataStreamSource fromParallelCollection(SplittableIterator iterator, TypeInformation - typeInfo, String operatorName) { - return addSource(new FromSplittableIteratorFunction<>(iterator), operatorName, typeInfo); - } - - /** - * Reads the given file line-by-line and creates a data stream that contains a string with the - * contents of each such line. The file will be read with the UTF-8 character set. - * - *

NOTES ON CHECKPOINTING: The source monitors the path, creates the - * {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, forwards - * them to the downstream {@link ContinuousFileReaderOperator readers} to read the actual data, - * and exits, without waiting for the readers to finish reading. This implies that no more - * checkpoint barriers are going to be forwarded after the source exits, thus having no - * checkpoints after that point. - * - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @return The data stream that represents the data read from the given file as text lines - */ - public DataStreamSource readTextFile(String filePath) { - return readTextFile(filePath, "UTF-8"); - } - - /** - * Reads the given file line-by-line and creates a data stream that contains a string with the - * contents of each such line. The {@link java.nio.charset.Charset} with the given name will be - * used to read the files. - * - *

NOTES ON CHECKPOINTING: The source monitors the path, creates the - * {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, - * forwards them to the downstream {@link ContinuousFileReaderOperator readers} to read the actual data, - * and exits, without waiting for the readers to finish reading. This implies that no more checkpoint - * barriers are going to be forwarded after the source exits, thus having no checkpoints after that point. - * - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") - * @param charsetName - * The name of the character set used to read the file - * @return The data stream that represents the data read from the given file as text lines - */ - public DataStreamSource readTextFile(String filePath, String charsetName) { - Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(filePath), "The file path must not be null or blank."); - - TextInputFormat format = new TextInputFormat(new Path(filePath)); - format.setFilesFilter(FilePathFilter.createDefaultFilter()); - TypeInformation typeInfo = BasicTypeInfo.STRING_TYPE_INFO; - format.setCharsetName(charsetName); - - return readFile(format, filePath, FileProcessingMode.PROCESS_ONCE, -1, typeInfo); - } - - /** - * Reads the contents of the user-specified {@code filePath} based on the given {@link FileInputFormat}. - * - *

Since all data streams need specific information about their types, this method needs to determine the - * type of the data produced by the input format. It will attempt to determine the data type by reflection, - * unless the input format implements the {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface. - * In the latter case, this method will invoke the - * {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable#getProducedType()} method to determine data - * type produced by the input format. - * - *

NOTES ON CHECKPOINTING: The source monitors the path, creates the - * {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, - * forwards them to the downstream {@link ContinuousFileReaderOperator readers} to read the actual data, - * and exits, without waiting for the readers to finish reading. This implies that no more checkpoint - * barriers are going to be forwarded after the source exits, thus having no checkpoints after that point. - * - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") - * @param inputFormat - * The input format used to create the data stream - * @param - * The type of the returned data stream - * @return The data stream that represents the data read from the given file - */ - public DataStreamSource readFile(FileInputFormat inputFormat, - String filePath) { - return readFile(inputFormat, filePath, FileProcessingMode.PROCESS_ONCE, -1); - } - - /** - * Reads the contents of the user-specified {@code filePath} based on the given {@link FileInputFormat}. Depending - * on the provided {@link FileProcessingMode}. - * - *

See {@link #readFile(FileInputFormat, String, FileProcessingMode, long)} - * - * @param inputFormat - * The input format used to create the data stream - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") - * @param watchType - * The mode in which the source should operate, i.e. monitor path and react to new data, or process once and exit - * @param interval - * In the case of periodic path monitoring, this specifies the interval (in millis) between consecutive path scans - * @param filter - * The files to be excluded from the processing - * @param - * The type of the returned data stream - * @return The data stream that represents the data read from the given file - * - * @deprecated Use {@link FileInputFormat#setFilesFilter(FilePathFilter)} to set a filter and - * {@link StreamExecutionEnvironment#readFile(FileInputFormat, String, FileProcessingMode, long)} - * - */ - @PublicEvolving - @Deprecated - public DataStreamSource readFile(FileInputFormat inputFormat, - String filePath, - FileProcessingMode watchType, - long interval, - FilePathFilter filter) { - inputFormat.setFilesFilter(filter); - - TypeInformation typeInformation; - try { - typeInformation = TypeExtractor.getInputFormatTypes(inputFormat); - } catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be " + - "automatically determined. Please specify the TypeInformation of the produced type " + - "explicitly by using the 'createInput(InputFormat, TypeInformation)' method instead."); - } - return readFile(inputFormat, filePath, watchType, interval, typeInformation); - } - - /** - * Reads the contents of the user-specified {@code filePath} based on the given {@link FileInputFormat}. Depending - * on the provided {@link FileProcessingMode}, the source may periodically monitor (every {@code interval} ms) the path - * for new data ({@link FileProcessingMode#PROCESS_CONTINUOUSLY}), or process once the data currently in the path and - * exit ({@link FileProcessingMode#PROCESS_ONCE}). In addition, if the path contains files not to be processed, the user - * can specify a custom {@link FilePathFilter}. As a default implementation you can use - * {@link FilePathFilter#createDefaultFilter()}. - * - *

Since all data streams need specific information about their types, this method needs to determine the - * type of the data produced by the input format. It will attempt to determine the data type by reflection, - * unless the input format implements the {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface. - * In the latter case, this method will invoke the - * {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable#getProducedType()} method to determine data - * type produced by the input format. - * - *

NOTES ON CHECKPOINTING: If the {@code watchType} is set to {@link FileProcessingMode#PROCESS_ONCE}, - * the source monitors the path once, creates the {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} - * to be processed, forwards them to the downstream {@link ContinuousFileReaderOperator readers} to read the actual data, - * and exits, without waiting for the readers to finish reading. This implies that no more checkpoint barriers - * are going to be forwarded after the source exits, thus having no checkpoints after that point. - * - * @param inputFormat - * The input format used to create the data stream - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") - * @param watchType - * The mode in which the source should operate, i.e. monitor path and react to new data, or process once and exit - * @param interval - * In the case of periodic path monitoring, this specifies the interval (in millis) between consecutive path scans - * @param - * The type of the returned data stream - * @return The data stream that represents the data read from the given file - */ - @PublicEvolving - public DataStreamSource readFile(FileInputFormat inputFormat, - String filePath, - FileProcessingMode watchType, - long interval) { - - TypeInformation typeInformation; - try { - typeInformation = TypeExtractor.getInputFormatTypes(inputFormat); - } catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be " + - "automatically determined. Please specify the TypeInformation of the produced type " + - "explicitly by using the 'createInput(InputFormat, TypeInformation)' method instead."); - } - return readFile(inputFormat, filePath, watchType, interval, typeInformation); - } - - /** - * Creates a data stream that contains the contents of file created while system watches the given path. The file - * will be read with the system's default character set. - * - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path/") - * @param intervalMillis - * The interval of file watching in milliseconds - * @param watchType - * The watch type of file stream. When watchType is {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#ONLY_NEW_FILES}, the system processes - * only - * new files. {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#REPROCESS_WITH_APPENDED} means that the system re-processes all contents of - * appended file. {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#PROCESS_ONLY_APPENDED} means that the system processes only appended - * contents - * of files. - * @return The DataStream containing the given directory. - * - * @deprecated Use {@link #readFile(FileInputFormat, String, FileProcessingMode, long)} instead. - */ - @Deprecated - @SuppressWarnings("deprecation") - public DataStream readFileStream(String filePath, long intervalMillis, FileMonitoringFunction.WatchType watchType) { - DataStream> source = addSource(new FileMonitoringFunction( - filePath, intervalMillis, watchType), "Read File Stream source"); - - return source.flatMap(new FileReadFunction()); - } - - /** - * Reads the contents of the user-specified {@code filePath} based on the given {@link FileInputFormat}. - * Depending on the provided {@link FileProcessingMode}, the source may periodically monitor (every {@code interval} ms) - * the path for new data ({@link FileProcessingMode#PROCESS_CONTINUOUSLY}), or process once the data currently in the - * path and exit ({@link FileProcessingMode#PROCESS_ONCE}). In addition, if the path contains files not to be processed, - * the user can specify a custom {@link FilePathFilter}. As a default implementation you can use - * {@link FilePathFilter#createDefaultFilter()}. - * - *

NOTES ON CHECKPOINTING: If the {@code watchType} is set to {@link FileProcessingMode#PROCESS_ONCE}, - * the source monitors the path once, creates the {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} - * to be processed, forwards them to the downstream {@link ContinuousFileReaderOperator readers} to read the actual data, - * and exits, without waiting for the readers to finish reading. This implies that no more checkpoint barriers - * are going to be forwarded after the source exits, thus having no checkpoints after that point. - * - * @param inputFormat - * The input format used to create the data stream - * @param filePath - * The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path") - * @param watchType - * The mode in which the source should operate, i.e. monitor path and react to new data, or process once and exit - * @param typeInformation - * Information on the type of the elements in the output stream - * @param interval - * In the case of periodic path monitoring, this specifies the interval (in millis) between consecutive path scans - * @param - * The type of the returned data stream - * @return The data stream that represents the data read from the given file - */ - @PublicEvolving - public DataStreamSource readFile(FileInputFormat inputFormat, - String filePath, - FileProcessingMode watchType, - long interval, - TypeInformation typeInformation) { - - Preconditions.checkNotNull(inputFormat, "InputFormat must not be null."); - Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(filePath), "The file path must not be null or blank."); - - inputFormat.setFilePath(filePath); - return createFileInput(inputFormat, typeInformation, "Custom File Source", watchType, interval); - } - - /** - * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are - * decoded by the system's default character set. On the termination of the socket server connection retries can be - * initiated. - * - *

Let us note that the socket itself does not report on abort and as a consequence retries are only initiated when - * the socket was gracefully terminated. - * - * @param hostname - * The host name which a server socket binds - * @param port - * The port number which a server socket binds. A port number of 0 means that the port number is automatically - * allocated. - * @param delimiter - * A character which splits received strings into records - * @param maxRetry - * The maximal retry interval in seconds while the program waits for a socket that is temporarily down. - * Reconnection is initiated every second. A number of 0 means that the reader is immediately terminated, - * while - * a negative value ensures retrying forever. - * @return A data stream containing the strings received from the socket - * - * @deprecated Use {@link #socketTextStream(String, int, String, long)} instead. - */ - @Deprecated - public DataStreamSource socketTextStream(String hostname, int port, char delimiter, long maxRetry) { - return socketTextStream(hostname, port, String.valueOf(delimiter), maxRetry); - } - - /** - * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are - * decoded by the system's default character set. On the termination of the socket server connection retries can be - * initiated. - * - *

Let us note that the socket itself does not report on abort and as a consequence retries are only initiated when - * the socket was gracefully terminated. - * - * @param hostname - * The host name which a server socket binds - * @param port - * The port number which a server socket binds. A port number of 0 means that the port number is automatically - * allocated. - * @param delimiter - * A string which splits received strings into records - * @param maxRetry - * The maximal retry interval in seconds while the program waits for a socket that is temporarily down. - * Reconnection is initiated every second. A number of 0 means that the reader is immediately terminated, - * while - * a negative value ensures retrying forever. - * @return A data stream containing the strings received from the socket - */ - @PublicEvolving - public DataStreamSource socketTextStream(String hostname, int port, String delimiter, long maxRetry) { - return addSource(new SocketTextStreamFunction(hostname, port, delimiter, maxRetry), - "Socket Stream"); - } - - /** - * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are - * decoded by the system's default character set. The reader is terminated immediately when the socket is down. - * - * @param hostname - * The host name which a server socket binds - * @param port - * The port number which a server socket binds. A port number of 0 means that the port number is automatically - * allocated. - * @param delimiter - * A character which splits received strings into records - * @return A data stream containing the strings received from the socket - * - * @deprecated Use {@link #socketTextStream(String, int, String)} instead. - */ - @Deprecated - @SuppressWarnings("deprecation") - public DataStreamSource socketTextStream(String hostname, int port, char delimiter) { - return socketTextStream(hostname, port, delimiter, 0); - } - - /** - * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are - * decoded by the system's default character set. The reader is terminated immediately when the socket is down. - * - * @param hostname - * The host name which a server socket binds - * @param port - * The port number which a server socket binds. A port number of 0 means that the port number is automatically - * allocated. - * @param delimiter - * A string which splits received strings into records - * @return A data stream containing the strings received from the socket - */ - @PublicEvolving - public DataStreamSource socketTextStream(String hostname, int port, String delimiter) { - return socketTextStream(hostname, port, delimiter, 0); - } - - /** - * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are - * decoded by the system's default character set, using"\n" as delimiter. The reader is terminated immediately when - * the socket is down. - * - * @param hostname - * The host name which a server socket binds - * @param port - * The port number which a server socket binds. A port number of 0 means that the port number is automatically - * allocated. - * @return A data stream containing the strings received from the socket - */ - @PublicEvolving - public DataStreamSource socketTextStream(String hostname, int port) { - return socketTextStream(hostname, port, "\n"); - } - - /** - * Generic method to create an input data stream with {@link org.apache.flink.api.common.io.InputFormat}. - * - *

Since all data streams need specific information about their types, this method needs to determine the - * type of the data produced by the input format. It will attempt to determine the data type by reflection, - * unless the input format implements the {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface. - * In the latter case, this method will invoke the - * {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable#getProducedType()} method to determine data - * type produced by the input format. - * - *

NOTES ON CHECKPOINTING: In the case of a {@link FileInputFormat}, the source - * (which executes the {@link ContinuousFileMonitoringFunction}) monitors the path, creates the - * {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, forwards - * them to the downstream {@link ContinuousFileReaderOperator} to read the actual data, and exits, - * without waiting for the readers to finish reading. This implies that no more checkpoint - * barriers are going to be forwarded after the source exits, thus having no checkpoints. - * - * @param inputFormat - * The input format used to create the data stream - * @param - * The type of the returned data stream - * @return The data stream that represents the data created by the input format - */ - @PublicEvolving - public DataStreamSource createInput(InputFormat inputFormat) { - return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); - } - - /** - * Generic method to create an input data stream with {@link org.apache.flink.api.common.io.InputFormat}. - * - *

The data stream is typed to the given TypeInformation. This method is intended for input formats - * where the return type cannot be determined by reflection analysis, and that do not implement the - * {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface. - * - *

NOTES ON CHECKPOINTING: In the case of a {@link FileInputFormat}, the source - * (which executes the {@link ContinuousFileMonitoringFunction}) monitors the path, creates the - * {@link org.apache.flink.core.fs.FileInputSplit FileInputSplits} to be processed, forwards - * them to the downstream {@link ContinuousFileReaderOperator} to read the actual data, and exits, - * without waiting for the readers to finish reading. This implies that no more checkpoint - * barriers are going to be forwarded after the source exits, thus having no checkpoints. - * - * @param inputFormat - * The input format used to create the data stream - * @param typeInfo - * The information about the type of the output type - * @param - * The type of the returned data stream - * @return The data stream that represents the data created by the input format - */ - @PublicEvolving - public DataStreamSource createInput(InputFormat inputFormat, TypeInformation typeInfo) { - DataStreamSource source; - - if (inputFormat instanceof FileInputFormat) { - @SuppressWarnings("unchecked") - FileInputFormat format = (FileInputFormat) inputFormat; - - source = createFileInput(format, typeInfo, "Custom File source", - FileProcessingMode.PROCESS_ONCE, -1); - } else { - source = createInput(inputFormat, typeInfo, "Custom Source"); - } - return source; - } - - private DataStreamSource createInput(InputFormat inputFormat, - TypeInformation typeInfo, - String sourceName) { - - InputFormatSourceFunction function = new InputFormatSourceFunction<>(inputFormat, typeInfo); - return addSource(function, sourceName, typeInfo); - } - - private DataStreamSource createFileInput(FileInputFormat inputFormat, - TypeInformation typeInfo, - String sourceName, - FileProcessingMode monitoringMode, - long interval) { - - Preconditions.checkNotNull(inputFormat, "Unspecified file input format."); - Preconditions.checkNotNull(typeInfo, "Unspecified output type information."); - Preconditions.checkNotNull(sourceName, "Unspecified name for the source."); - Preconditions.checkNotNull(monitoringMode, "Unspecified monitoring mode."); - - Preconditions.checkArgument(monitoringMode.equals(FileProcessingMode.PROCESS_ONCE) || - interval >= ContinuousFileMonitoringFunction.MIN_MONITORING_INTERVAL, - "The path monitoring interval cannot be less than " + - ContinuousFileMonitoringFunction.MIN_MONITORING_INTERVAL + " ms."); - - ContinuousFileMonitoringFunction monitoringFunction = - new ContinuousFileMonitoringFunction<>(inputFormat, monitoringMode, getParallelism(), interval); - - ContinuousFileReaderOperator reader = - new ContinuousFileReaderOperator<>(inputFormat); - - SingleOutputStreamOperator source = addSource(monitoringFunction, sourceName) - .transform("Split Reader: " + sourceName, typeInfo, reader); - - return new DataStreamSource<>(source); - } - - /** - * Adds a Data Source to the streaming topology. - * - *

By default sources have a parallelism of 1. To enable parallel execution, the user defined source should - * implement {@link org.apache.flink.streaming.api.functions.source.ParallelSourceFunction} or extend {@link - * org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction}. In these cases the resulting source - * will have the parallelism of the environment. To change this afterwards call {@link - * org.apache.flink.streaming.api.datastream.DataStreamSource#setParallelism(int)} - * - * @param function - * the user defined function - * @param - * type of the returned stream - * @return the data stream constructed - */ - public DataStreamSource addSource(SourceFunction function) { - return addSource(function, "Custom Source"); - } - - /** - * Ads a data source with a custom type information thus opening a - * {@link DataStream}. Only in very special cases does the user need to - * support type information. Otherwise use - * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)} - * - * @param function - * the user defined function - * @param sourceName - * Name of the data source - * @param - * type of the returned stream - * @return the data stream constructed - */ - public DataStreamSource addSource(SourceFunction function, String sourceName) { - return addSource(function, sourceName, null); - } - - /** - * Ads a data source with a custom type information thus opening a - * {@link DataStream}. Only in very special cases does the user need to - * support type information. Otherwise use - * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)} - * - * @param function - * the user defined function - * @param - * type of the returned stream - * @param typeInfo - * the user defined type information for the stream - * @return the data stream constructed - */ - public DataStreamSource addSource(SourceFunction function, TypeInformation typeInfo) { - return addSource(function, "Custom Source", typeInfo); - } - - /** - * Ads a data source with a custom type information thus opening a - * {@link DataStream}. Only in very special cases does the user need to - * support type information. Otherwise use - * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)} - * - * @param function - * the user defined function - * @param sourceName - * Name of the data source - * @param - * type of the returned stream - * @param typeInfo - * the user defined type information for the stream - * @return the data stream constructed - */ - @SuppressWarnings("unchecked") - public DataStreamSource addSource(SourceFunction function, String sourceName, TypeInformation typeInfo) { - - if (typeInfo == null) { - if (function instanceof ResultTypeQueryable) { - typeInfo = ((ResultTypeQueryable) function).getProducedType(); - } else { - try { - typeInfo = TypeExtractor.createTypeInfo( - SourceFunction.class, - function.getClass(), 0, null, null); - } catch (final InvalidTypesException e) { - typeInfo = (TypeInformation) new MissingTypeInfo(sourceName, e); - } - } - } - - boolean isParallel = function instanceof ParallelSourceFunction; - - clean(function); - StreamSource sourceOperator; - if (function instanceof StoppableFunction) { - sourceOperator = new StoppableStreamSource<>(cast2StoppableSourceFunction(function)); - } else { - sourceOperator = new StreamSource<>(function); - } - - return new DataStreamSource<>(this, typeInfo, sourceOperator, isParallel, sourceName); - } - - /** - * Casts the source function into a SourceFunction implementing the StoppableFunction. - * - *

This method should only be used if the source function was checked to implement the - * {@link StoppableFunction} interface. - * - * @param sourceFunction Source function to cast - * @param Output type of source function - * @param Union type of SourceFunction and StoppableFunction - * @return The casted source function so that it's type implements the StoppableFunction - */ - @SuppressWarnings("unchecked") - private & StoppableFunction> T cast2StoppableSourceFunction(SourceFunction sourceFunction) { - return (T) sourceFunction; - } - - /** - * Triggers the program execution. The environment will execute all parts of - * the program that have resulted in a "sink" operation. Sink operations are - * for example printing results or forwarding them to a message queue. - * - *

The program execution will be logged and displayed with a generated - * default name. - * - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception which occurs during job execution. - */ - public JobExecutionResult execute() throws Exception { - return execute(DEFAULT_JOB_NAME); - } - - /** - * Triggers the program execution. The environment will execute all parts of - * the program that have resulted in a "sink" operation. Sink operations are - * for example printing results or forwarding them to a message queue. - * - *

The program execution will be logged and displayed with the provided name - * - * @param jobName - * Desired name of the job - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception which occurs during job execution. - */ - public abstract JobExecutionResult execute(String jobName) throws Exception; - - /** - * Getter of the {@link org.apache.flink.streaming.api.graph.StreamGraph} of the streaming job. - * - * @return The streamgraph representing the transformations - */ - @Internal - public StreamGraph getStreamGraph() { - if (transformations.size() <= 0) { - throw new IllegalStateException("No operators defined in streaming topology. Cannot execute."); - } - return StreamGraphGenerator.generate(this, transformations); - } - - /** - * Creates the plan with which the system will execute the program, and - * returns it as a String using a JSON representation of the execution data - * flow graph. Note that this needs to be called, before the plan is - * executed. - * - * @return The execution plan of the program, as a JSON String. - */ - public String getExecutionPlan() { - return getStreamGraph().getStreamingPlanAsJSON(); - } - - /** - * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning - * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig} - */ - @Internal - public F clean(F f) { - if (getConfig().isClosureCleanerEnabled()) { - ClosureCleaner.clean(f, getConfig().getClosureCleanerLevel(), true); - } - ClosureCleaner.ensureSerializable(f); - return f; - } - - /** - * Adds an operator to the list of operators that should be executed when calling - * {@link #execute}. - * - *

When calling {@link #execute()} only the operators that where previously added to the list - * are executed. - * - *

This is not meant to be used by users. The API methods that create operators must call - * this method. - */ - @Internal - public void addOperator(StreamTransformation transformation) { - Preconditions.checkNotNull(transformation, "transformation must not be null."); - this.transformations.add(transformation); - } - - // -------------------------------------------------------------------------------------------- - // Factory methods for ExecutionEnvironments - // -------------------------------------------------------------------------------------------- - - /** - * Creates an execution environment that represents the context in which the - * program is currently executed. If the program is invoked standalone, this - * method returns a local execution environment, as returned by - * {@link #createLocalEnvironment()}. - * - * @return The execution environment of the context in which the program is - * executed. - */ - public static StreamExecutionEnvironment getExecutionEnvironment() { - if (contextEnvironmentFactory.get() != null) { - return contextEnvironmentFactory.get().createExecutionEnvironment(); - } - - // because the streaming project depends on "flink-clients" (and not the other way around) - // we currently need to intercept the data set environment and create a dependent stream env. - // this should be fixed once we rework the project dependencies - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - if (env instanceof ContextEnvironment) { - return new StreamContextEnvironment((ContextEnvironment) env); - } else if (env instanceof OptimizerPlanEnvironment || env instanceof PreviewPlanEnvironment) { - return new StreamPlanEnvironment(env); - } else { - return createLocalEnvironment(); - } - } - - /** - * Creates a {@link LocalStreamEnvironment}. The local execution environment - * will run the program in a multi-threaded fashion in the same JVM as the - * environment was created in. The default parallelism of the local - * environment is the number of hardware contexts (CPU cores / threads), - * unless it was specified differently by {@link #setParallelism(int)}. - * - * @return A local execution environment. - */ - public static LocalStreamEnvironment createLocalEnvironment() { - return createLocalEnvironment(defaultLocalParallelism); - } - - /** - * Creates a {@link LocalStreamEnvironment}. The local execution environment - * will run the program in a multi-threaded fashion in the same JVM as the - * environment was created in. It will use the parallelism specified in the - * parameter. - * - * @param parallelism - * The parallelism for the local environment. - * @return A local execution environment with the specified parallelism. - */ - public static LocalStreamEnvironment createLocalEnvironment(int parallelism) { - return createLocalEnvironment(parallelism, new Configuration()); - } - - /** - * Creates a {@link LocalStreamEnvironment}. The local execution environment - * will run the program in a multi-threaded fashion in the same JVM as the - * environment was created in. It will use the parallelism specified in the - * parameter. - * - * @param parallelism - * The parallelism for the local environment. - * @param configuration - * Pass a custom configuration into the cluster - * @return A local execution environment with the specified parallelism. - */ - public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Configuration configuration) { - final LocalStreamEnvironment currentEnvironment; - - currentEnvironment = new LocalStreamEnvironment(configuration); - currentEnvironment.setParallelism(parallelism); - - return currentEnvironment; - } - - /** - * Creates a {@link LocalStreamEnvironment} for local program execution that also starts the - * web monitoring UI. - * - *

The local execution environment will run the program in a multi-threaded fashion in - * the same JVM as the environment was created in. It will use the parallelism specified in the - * parameter. - * - *

If the configuration key 'rest.port' was set in the configuration, that particular - * port will be used for the web UI. Otherwise, the default port (8081) will be used. - */ - @PublicEvolving - public static StreamExecutionEnvironment createLocalEnvironmentWithWebUI(Configuration conf) { - checkNotNull(conf, "conf"); - - conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true); - - if (!conf.contains(RestOptions.PORT)) { - // explicitly set this option so that it's not set to 0 later - conf.setInteger(RestOptions.PORT, RestOptions.PORT.defaultValue()); - } - - return createLocalEnvironment(defaultLocalParallelism, conf); - } - - /** - * Creates a {@link RemoteStreamEnvironment}. The remote environment sends - * (parts of) the program to a cluster for execution. Note that all file - * paths used in the program must be accessible from the cluster. The - * execution will use no parallelism, unless the parallelism is set - * explicitly via {@link #setParallelism}. - * - * @param host - * The host name or address of the master (JobManager), where the - * program should be executed. - * @param port - * The port of the master (JobManager), where the program should - * be executed. - * @param jarFiles - * The JAR files with code that needs to be shipped to the - * cluster. If the program uses user-defined functions, - * user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static StreamExecutionEnvironment createRemoteEnvironment( - String host, int port, String... jarFiles) { - return new RemoteStreamEnvironment(host, port, jarFiles); - } - - /** - * Creates a {@link RemoteStreamEnvironment}. The remote environment sends - * (parts of) the program to a cluster for execution. Note that all file - * paths used in the program must be accessible from the cluster. The - * execution will use the specified parallelism. - * - * @param host - * The host name or address of the master (JobManager), where the - * program should be executed. - * @param port - * The port of the master (JobManager), where the program should - * be executed. - * @param parallelism - * The parallelism to use during the execution. - * @param jarFiles - * The JAR files with code that needs to be shipped to the - * cluster. If the program uses user-defined functions, - * user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static StreamExecutionEnvironment createRemoteEnvironment( - String host, int port, int parallelism, String... jarFiles) { - RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles); - env.setParallelism(parallelism); - return env; - } - - /** - * Creates a {@link RemoteStreamEnvironment}. The remote environment sends - * (parts of) the program to a cluster for execution. Note that all file - * paths used in the program must be accessible from the cluster. The - * execution will use the specified parallelism. - * - * @param host - * The host name or address of the master (JobManager), where the - * program should be executed. - * @param port - * The port of the master (JobManager), where the program should - * be executed. - * @param clientConfig - * The configuration used by the client that connects to the remote cluster. - * @param jarFiles - * The JAR files with code that needs to be shipped to the - * cluster. If the program uses user-defined functions, - * user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static StreamExecutionEnvironment createRemoteEnvironment( - String host, int port, Configuration clientConfig, String... jarFiles) { - return new RemoteStreamEnvironment(host, port, clientConfig, jarFiles); - } - - /** - * Gets the default parallelism that will be used for the local execution environment created by - * {@link #createLocalEnvironment()}. - * - * @return The default local parallelism - */ - @PublicEvolving - public static int getDefaultLocalParallelism() { - return defaultLocalParallelism; - } - - /** - * Sets the default parallelism that will be used for the local execution - * environment created by {@link #createLocalEnvironment()}. - * - * @param parallelism The parallelism to use as the default local parallelism. - */ - @PublicEvolving - public static void setDefaultLocalParallelism(int parallelism) { - defaultLocalParallelism = parallelism; - } - - // -------------------------------------------------------------------------------------------- - // Methods to control the context and local environments for execution from packaged programs - // -------------------------------------------------------------------------------------------- - - protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) { - contextEnvironmentFactory.set(ctx); - } - - protected static void resetContextEnvironment() { - contextEnvironmentFactory.remove(); - } - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - */ - public void registerCachedFile(String filePath, String name) { - registerCachedFile(filePath, name, false); - } - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - * @param executable flag indicating whether the file should be executable - */ - public void registerCachedFile(String filePath, String name, boolean executable) { - this.cacheFile.add(new Tuple2<>(name, new DistributedCache.DistributedCacheEntry(filePath, executable))); - } -} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index dc952713f6..960b7be4d0 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -24,14 +24,11 @@ import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; -import org.apache.flink.configuration.*; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.util.LeaderConnectionInfo; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -40,11 +37,8 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; -import java.net.InetSocketAddress; import java.net.URL; import java.util.*; @@ -68,26 +62,24 @@ public static ClusterClient createClusterClient(Options launcherOptions) throws } public static ClusterClient createStandaloneClient(Options launcherOptions) throws Exception { - String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + Configuration flinkConf = launcherOptions.loadFlinkConfiguration(); - StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(config); - RestClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()); + StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(flinkConf); + ClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()).getClusterClient(); - LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); - InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); - config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); - config.setInteger(JobManagerOptions.PORT, address.getPort()); - clusterClient.setDetached(true); +// LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); +// InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); +// config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); +// config.setInteger(JobManagerOptions.PORT, address.getPort()); +// clusterClient.setDetached(true); return clusterClient; } public static ClusterClient createYarnClient(Options launcherOptions) { - Configuration flinkConfig = GlobalConfiguration.loadConfiguration(launcherOptions.getFlinkconf()); + Configuration flinkConfig = launcherOptions.loadFlinkConfiguration(); String yarnConfDir = launcherOptions.getYarnconf(); if(StringUtils.isNotBlank(yarnConfDir)) { try { - flinkConfig.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); FileSystem.initialize(flinkConfig); YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); @@ -105,16 +97,18 @@ public static ClusterClient createYarnClient(Options launcherOptions) { applicationId = ConverterUtils.toApplicationId(launcherOptions.getAppId()); } - HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(flinkConfig); + HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(flinkConfig); if(highAvailabilityMode.equals(HighAvailabilityMode.ZOOKEEPER) && applicationId!=null){ - flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID,applicationId.toString()); + flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationId.toString()); } - - AbstractYarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(launcherOptions, yarnClient, yarnConf, flinkConfig); - ClusterClient clusterClient = yarnClusterDescriptor.retrieve(applicationId); - clusterClient.setDetached(true); - - return clusterClient; +// YarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(launcherOptions, yarnClient, yarnConf, flinkConfig); + YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( + flinkConfig, + yarnConf, + yarnClient, + YarnClientYarnClusterInformationRetriever.create(yarnClient), + true); + return yarnClusterDescriptor.retrieve(applicationId).getClusterClient(); } catch(Exception e) { throw new RuntimeException(e); } @@ -123,11 +117,16 @@ public static ClusterClient createYarnClient(Options launcherOptions) { throw new UnsupportedOperationException("Haven't been developed yet!"); } - private static AbstractYarnClusterDescriptor getClusterDescriptor(Options launcherOptions, + private static YarnClusterDescriptor getClusterDescriptor(Options launcherOptions, YarnClient yarnClient, YarnConfiguration yarnConf, Configuration flinkConfig) throws Exception{ - AbstractYarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor(flinkConfig, yarnConf, "", yarnClient, true); + YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( + flinkConfig, + yarnConf, + yarnClient, + YarnClientYarnClusterInformationRetriever.create(yarnClient), + true); // plugin dependent on shipfile if (StringUtils.isNotEmpty(launcherOptions.getPluginLoadMode()) && "shipfile".equalsIgnoreCase(launcherOptions.getPluginLoadMode())) { @@ -149,10 +148,9 @@ private static AbstractYarnClusterDescriptor getClusterDescriptor(Options launch } } - yarnClusterDescriptor.setProvidedUserJarFiles(classpaths); +// yarnClusterDescriptor.setProvidedUserJarFiles(classpaths); } - yarnClusterDescriptor.setName(launcherOptions.getJobid()); return yarnClusterDescriptor; } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 91a4bd1409..06f83dbb14 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -22,13 +22,17 @@ import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.enums.ClusterMode; -import com.dtstack.flinkx.launcher.perjob.PerJobSubmitter; +import com.dtstack.flinkx.launcher.perJob.PerJobSubmitter; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.options.Options; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.lang.StringUtils; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.Preconditions; import org.slf4j.LoggerFactory; @@ -45,8 +49,9 @@ /** * FlinkX commandline Launcher - * + *

* Company: www.dtstack.com + * * @author huyifan.zju@163.com */ public class Launcher { @@ -56,11 +61,63 @@ public class Launcher { public static final String KEY_HADOOP_HOME = "HADOOP_HOME"; public static final String PLUGINS_DIR_NAME = "plugins"; - public static final String CORE_JAR_NAME_PREFIX = "flinkx"; - private static List analyzeUserClasspath(String content, String pluginRoot) { + public static void main(String[] args) throws Exception { + setLogLevel(Level.DEBUG.toString()); + OptionParser optionParser = new OptionParser(args); + Options launcherOptions = optionParser.getOptions(); + findDefaultConfigDir(launcherOptions); + List argList = optionParser.getProgramExeArgList(); + switch (ClusterMode.getByName(launcherOptions.getMode())) { + case local: + com.dtstack.flinkx.Main.main(argList.toArray(new String[0])); + break; + case standalone: + case yarn: + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); + argList.add("-monitor"); + argList.add(clusterClient.getWebInterfaceURL()); + ClientUtils.submitJob(clusterClient, buildJobGraph(launcherOptions, argList.toArray(new String[0]))); + break; + case yarnPer: + String confProp = launcherOptions.getConfProp(); + if (StringUtils.isBlank(confProp)) { + throw new IllegalArgumentException("per-job mode must have confProp!"); + } + String libJar = launcherOptions.getFlinkLibJar(); + if (StringUtils.isBlank(libJar)) { + throw new IllegalArgumentException("per-job mode must have flink lib path!"); + } + argList.add("-monitor"); + argList.add("application_default"); + PerJobSubmitter.submit(launcherOptions, buildJobGraph(launcherOptions, argList.toArray(new String[0]))); + } + } + + private static JobGraph buildJobGraph(Options launcherOptions, String[] remoteArgs) throws Exception { + String pluginRoot = launcherOptions.getPluginRoot(); + String content = launcherOptions.getJob(); + String coreJarName = getCoreJarFileName(pluginRoot); + File jarFile = new File(pluginRoot + File.separator + coreJarName); + List urlList = analyzeUserClasspath(content, pluginRoot); + SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.none(); + if (StringUtils.isNotEmpty(launcherOptions.getS())) { + savepointRestoreSettings = SavepointRestoreSettings.forPath(launcherOptions.getS()); + } + PackagedProgram program = PackagedProgram.newBuilder() + .setJarFile(jarFile) + .setUserClassPaths(urlList) + .setEntryPointClassName("com.dtstack.flinkx.Main") + .setConfiguration(launcherOptions.loadFlinkConfiguration()) + .setSavepointRestoreSettings(savepointRestoreSettings) + .setArguments(remoteArgs) + .build(); + return PackagedProgramUtils.createJobGraph(program, launcherOptions.loadFlinkConfiguration(), Integer.parseInt(launcherOptions.getParallelism()), false); + } + + private static List analyzeUserClasspath(String content, String pluginRoot) { List urlList = new ArrayList<>(); String jobJson = readJob(content); @@ -86,61 +143,6 @@ private static List analyzeUserClasspath(String content, String pluginRoot) return urlList; } - public static void main(String[] args) throws Exception { - setLogLevel(Level.INFO.toString()); - OptionParser optionParser = new OptionParser(args); - Options launcherOptions = optionParser.getOptions(); - findDefaultConfigDir(launcherOptions); - - String mode = launcherOptions.getMode(); - List argList = optionParser.getProgramExeArgList(); - if(mode.equals(ClusterMode.local.name())) { - String[] localArgs = argList.toArray(new String[argList.size()]); - com.dtstack.flinkx.Main.main(localArgs); - } else { - String pluginRoot = launcherOptions.getPluginRoot(); - String content = launcherOptions.getJob(); - String coreJarName = getCoreJarFileName(pluginRoot); - File jarFile = new File(pluginRoot + File.separator + coreJarName); - List urlList = analyzeUserClasspath(content, pluginRoot); - if(!ClusterMode.yarnPer.name().equals(mode)){ - ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - String monitor = clusterClient.getWebInterfaceURL(); - argList.add("-monitor"); - argList.add(monitor); - - String[] remoteArgs = argList.toArray(new String[0]); - - ClassLoaderType classLoaderType = ClassLoaderType.getByClassMode(launcherOptions.getPluginLoadMode()); - PackagedProgram program = new PackagedProgram(jarFile, urlList, classLoaderType, "com.dtstack.flinkx.Main", remoteArgs); - - if (StringUtils.isNotEmpty(launcherOptions.getS())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getS())); - } - - clusterClient.run(program, Integer.parseInt(launcherOptions.getParallelism())); - clusterClient.shutdown(); - }else{ - String confProp = launcherOptions.getConfProp(); - if (StringUtils.isBlank(confProp)){ - throw new IllegalArgumentException("per-job mode must have confProp!"); - } - - String libJar = launcherOptions.getFlinkLibJar(); - if (StringUtils.isBlank(libJar)){ - throw new IllegalArgumentException("per-job mode must have flink lib path!"); - } - - argList.add("-monitor"); - argList.add(""); - - //jdk内在优化,使用空数组效率更高 - String[] remoteArgs = argList.toArray(new String[0]); - PerJobSubmitter.submit(launcherOptions, jarFile, remoteArgs); - } - } - } - private static void findDefaultConfigDir(Options launcherOptions) { findDefaultPluginRoot(launcherOptions); @@ -176,7 +178,7 @@ private static void findDefaultFlinkConf(Options launcherOptions) { String flinkHome = getSystemProperty(KEY_FLINK_HOME); if (StringUtils.isNotEmpty(flinkHome)) { flinkHome = flinkHome.trim(); - if (flinkHome.endsWith(File.separator)){ + if (flinkHome.endsWith(File.separator)) { flinkHome = flinkHome.substring(0, flinkHome.lastIndexOf(File.separator)); } @@ -187,21 +189,20 @@ private static void findDefaultFlinkConf(Options launcherOptions) { private static void findDefaultPluginRoot(Options launcherOptions) { String pluginRoot = launcherOptions.getPluginRoot(); - if (StringUtils.isNotEmpty(pluginRoot)) { - return; - } + if (StringUtils.isEmpty(pluginRoot)) { + String flinkxHome = getSystemProperty(KEY_FLINKX_HOME); + if (StringUtils.isNotEmpty(flinkxHome)) { + flinkxHome = flinkxHome.trim(); + if (flinkxHome.endsWith(File.separator)) { + pluginRoot = flinkxHome + PLUGINS_DIR_NAME; + } else { + pluginRoot = flinkxHome + File.separator + PLUGINS_DIR_NAME; + } - String flinkxHome = getSystemProperty(KEY_FLINKX_HOME); - if (StringUtils.isNotEmpty(flinkxHome)) { - flinkxHome = flinkxHome.trim(); - if (flinkxHome.endsWith(File.separator)) { - pluginRoot = flinkxHome + PLUGINS_DIR_NAME; - } else { - pluginRoot = flinkxHome + File.separator + PLUGINS_DIR_NAME; + launcherOptions.setPluginRoot(pluginRoot); } - - launcherOptions.setPluginRoot(pluginRoot); } + System.setProperty(ConfigConstants.ENV_FLINK_PLUGINS_DIR, pluginRoot); } private static String getSystemProperty(String name) { @@ -213,10 +214,10 @@ private static String getSystemProperty(String name) { return property; } - private static String getCoreJarFileName (String pluginRoot) throws FileNotFoundException{ + private static String getCoreJarFileName(String pluginRoot) throws FileNotFoundException { String coreJarFileName = null; File pluginDir = new File(pluginRoot); - if (pluginDir.exists() && pluginDir.isDirectory()){ + if (pluginDir.exists() && pluginDir.isDirectory()) { File[] jarFiles = pluginDir.listFiles(new FilenameFilter() { @Override public boolean accept(File dir, String name) { @@ -224,12 +225,12 @@ public boolean accept(File dir, String name) { } }); - if (jarFiles != null && jarFiles.length > 0){ + if (jarFiles != null && jarFiles.length > 0) { coreJarFileName = jarFiles[0].getName(); } } - if (StringUtils.isEmpty(coreJarFileName)){ + if (StringUtils.isEmpty(coreJarFileName)) { throw new FileNotFoundException("Can not find core jar file in path:" + pluginRoot); } @@ -244,15 +245,15 @@ private static String readJob(String job) { in.read(fileContent); in.close(); return new String(fileContent, StandardCharsets.UTF_8); - } catch (Exception e){ + } catch (Exception e) { throw new RuntimeException(e); } } - private static void setLogLevel(String level){ - LoggerContext loggerContext= (LoggerContext) LoggerFactory.getILoggerFactory(); + private static void setLogLevel(String level) { + LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); //设置全局日志级别 - ch.qos.logback.classic.Logger logger=loggerContext.getLogger("root"); + ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); logger.setLevel(Level.toLevel(level)); } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java similarity index 87% rename from flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java rename to flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java index c0cf771e7b..50401dc3e8 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.launcher.perjob; +package com.dtstack.flinkx.launcher.perJob; import com.dtstack.flinkx.util.ValueUtil; import org.apache.flink.client.deployment.ClusterSpecification; @@ -37,7 +37,7 @@ public class FlinkPerJobResourceUtil { public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; - public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; +// public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; /** @@ -48,7 +48,6 @@ public class FlinkPerJobResourceUtil { public static ClusterSpecification createClusterSpecification(Properties conProp){ int jobmanagerMemoryMb = 768; int taskmanagerMemoryMb = 768; - int numberTaskManagers = 1; int slotsPerTaskManager = 1; if(conProp != null){ @@ -58,11 +57,6 @@ public static ClusterSpecification createClusterSpecification(Properties conProp if(conProp.contains(TASKMANAGER_MEMORY_MB)){ taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); } - - if (conProp.containsKey(NUMBER_TASK_MANAGERS)){ - numberTaskManagers = ValueUtil.getInt(conProp.get(NUMBER_TASK_MANAGERS)); - } - if (conProp.containsKey(SLOTS_PER_TASKMANAGER)){ slotsPerTaskManager = ValueUtil.getInt(conProp.get(SLOTS_PER_TASKMANAGER)); } @@ -71,7 +65,6 @@ public static ClusterSpecification createClusterSpecification(Properties conProp return new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(jobmanagerMemoryMb) .setTaskManagerMemoryMB(taskmanagerMemoryMb) - .setNumberTaskManagers(numberTaskManagers) .setSlotsPerTaskManager(slotsPerTaskManager) .createClusterSpecification(); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java similarity index 66% rename from flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java rename to flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index 998ee73516..23078e97f9 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.launcher.perjob; +package com.dtstack.flinkx.launcher.perJob; -import com.dtstack.flinkx.launcher.PluginUtil; +import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; -import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.hadoop.fs.Path; @@ -34,7 +35,9 @@ import java.io.File; import java.net.MalformedURLException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; /** * Date: 2019/09/11 @@ -42,21 +45,29 @@ * @author tudou */ public class PerJobClusterClientBuilder { - private static final Logger LOG = LoggerFactory.getLogger(PerJobClusterClientBuilder.class); private YarnClient yarnClient; private YarnConfiguration yarnConf; + private Configuration flinkConfig; + /** * init yarnClient + * @param launcherOptions flinkx args + * @param conProp flink args */ - public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properties userConf) throws Exception { - userConf.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); + public void init(Options launcherOptions, Properties conProp) throws Exception { + String yarnConfDir = launcherOptions.getYarnconf(); + if(StringUtils.isBlank(yarnConfDir)) { + throw new RuntimeException("parameters of yarn is required"); + } + flinkConfig = launcherOptions.loadFlinkConfiguration(); + conProp.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); SecurityUtils.install(new SecurityConfiguration(flinkConfig)); - this.yarnConf = yarnConf; + yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); yarnClient.start(); @@ -67,12 +78,13 @@ public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properti /** * create a yarn cluster descriptor which is used to start the application master * @param confProp taskParams - * @param options LauncherOptions + * @param launcherOptions LauncherOptions + * @param jobGraph JobGraph * @return * @throws MalformedURLException */ - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options options) throws MalformedURLException { - String flinkJarPath = options.getFlinkLibJar(); + public YarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options launcherOptions, JobGraph jobGraph) throws MalformedURLException { + String flinkJarPath = launcherOptions.getFlinkLibJar(); if (StringUtils.isNotBlank(flinkJarPath)) { if (!new File(flinkJarPath).exists()) { throw new IllegalArgumentException("The Flink jar path is not exist"); @@ -83,9 +95,12 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co Configuration conf = new Configuration(); confProp.forEach((key, value) -> conf.setString(key.toString(), value.toString())); - - AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(conf, yarnConf, options.getFlinkconf(), yarnClient, false); - descriptor.setName(options.getJobid()); + YarnClusterDescriptor descriptor = new YarnClusterDescriptor( + flinkConfig, + yarnConf, + yarnClient, + YarnClientYarnClusterInformationRetriever.create(yarnClient), + false); List shipFiles = new ArrayList<>(); File[] jars = new File(flinkJarPath).listFiles(); @@ -99,20 +114,11 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } } -// if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { -// List files = PluginUtil.getAllPluginPath(options.getPluginRoot()); -// shipFiles.addAll(files); -// } - - if (StringUtils.isNotBlank(options.getQueue())) { - descriptor.setQueue(options.getQueue()); - } - - File log4j = new File(options.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); if(log4j.exists()){ shipFiles.add(log4j); }else{ - File logback = new File(options.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); if(logback.exists()){ shipFiles.add(logback); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java new file mode 100644 index 0000000000..a0daae8cd6 --- /dev/null +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.launcher.perJob; + +import com.dtstack.flinkx.options.Options; +import com.dtstack.flinkx.util.MapUtil; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * Date: 2019/09/11 + * Company: www.dtstack.com + * @author tudou + */ +public class PerJobSubmitter { + private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); + + /** + * submit per-job task + * @param options LauncherOptions + * @param jobGraph JobGraph + * @return + * @throws Exception + */ + public static String submit(Options options, JobGraph jobGraph) throws Exception{ + LOG.info("start to submit per-job task, options = {}", options.toString()); + Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); + PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); + perJobClusterClientBuilder.init(options, conProp); + + YarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options, jobGraph); + ClusterClientProvider provider = descriptor.deployJobCluster(clusterSpecification, jobGraph, true); + String applicationId = provider.getClusterClient().getClusterId().toString(); + String flinkJobId = jobGraph.getJobID().toString(); + LOG.info("deploy per_job with appId: {}}, jobId: {}", applicationId, flinkJobId); + return applicationId; + } +} \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java deleted file mode 100644 index d42f744a05..0000000000 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.launcher.perjob; - -import com.dtstack.flinkx.launcher.ClassLoaderType; -import com.dtstack.flinkx.launcher.YarnConfLoader; -import com.dtstack.flinkx.options.Options; -import com.dtstack.flinkx.util.MapUtil; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.yarn.AbstractYarnClusterDescriptor; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.util.ArrayList; -import java.util.Properties; - -/** - * Date: 2019/09/11 - * Company: www.dtstack.com - * @author tudou - */ -public class PerJobSubmitter { - private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); - - /** - * submit per-job task - * @param options LauncherOptions - * @return - * @throws Exception - */ - public static String submit(Options options, File jarFile, String[] programArgs) throws Exception{ - LOG.info("start to submit per-job task, LauncherOptions = {}", options.toString()); - - if (StringUtils.isEmpty(options.getYarnconf())) { - throw new RuntimeException("parameters of yarn is required"); - } - - YarnConfiguration yarnConf = StringUtils.isEmpty(options.getYarnconf()) ? new YarnConfiguration() : YarnConfLoader.getYarnConf(options.getYarnconf()); - Configuration flinkConfig = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); - flinkConfig.setString("classloader.resolve-order", "child-first"); - - Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); - ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); - clusterSpecification.setCreateProgramDelay(true); - clusterSpecification.setConfiguration(flinkConfig); - clusterSpecification.setClasspaths(new ArrayList<>()); - clusterSpecification.setEntryPointClass("com.dtstack.flinkx.Main"); - clusterSpecification.setJarFile(jarFile); - - if (StringUtils.isNotEmpty(options.getS())) { - clusterSpecification.setSpSetting(SavepointRestoreSettings.forPath(options.getS())); - } - - clusterSpecification.setProgramArgs(programArgs); - clusterSpecification.setCreateProgramDelay(true); - clusterSpecification.setYarnConfiguration(yarnConf); - - clusterSpecification.setClassLoaderType(ClassLoaderType.PARENT_FIRST_CACHE); - - PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); - perJobClusterClientBuilder.init(yarnConf, flinkConfig, conProp); - - AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options); - ClusterClient clusterClient = descriptor.deployJobCluster(clusterSpecification, new JobGraph(), true); - String applicationId = clusterClient.getClusterId().toString(); - LOG.info("deploy per_job with appId: {}", applicationId); - return applicationId; - } -} \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java deleted file mode 100644 index 2e12164d90..0000000000 --- a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java +++ /dev/null @@ -1,259 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.client.deployment; - -import com.dtstack.flinkx.launcher.ClassLoaderType; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -import java.io.File; -import java.net.URL; -import java.util.List; - -/** - * Description of the cluster to start by the {@link ClusterDescriptor}. - */ -public final class ClusterSpecification { - private final int masterMemoryMB; - private final int taskManagerMemoryMB; - private final int numberTaskManagers; - private final int slotsPerTaskManager; - private final int priority; - - private int parallelism; - private Configuration configuration; - private YarnConfiguration yarnConfiguration; - private JobGraph jobGraph; - private SavepointRestoreSettings spSetting; - private List classpaths; - private String entryPointClass; - private String[] programArgs; - private File jarFile; - private boolean createProgramDelay = false; - private PackagedProgram program; - private ClassLoaderType classLoaderType = ClassLoaderType.PARENT_FIRST; - - private ClusterSpecification(int masterMemoryMB, int taskManagerMemoryMB, int numberTaskManagers, int slotsPerTaskManager, int parallelism, int priority) { - this.masterMemoryMB = masterMemoryMB; - this.taskManagerMemoryMB = taskManagerMemoryMB; - this.numberTaskManagers = numberTaskManagers; - this.slotsPerTaskManager = slotsPerTaskManager; - this.parallelism = parallelism; - this.priority = priority; - } - - public PackagedProgram getProgram() { - return program; - } - - public void setProgram(PackagedProgram program) { - this.program = program; - } - - public YarnConfiguration getYarnConfiguration() { - return yarnConfiguration; - } - - public void setYarnConfiguration(YarnConfiguration yarnConfiguration) { - this.yarnConfiguration = yarnConfiguration; - } - - public JobGraph getJobGraph() { - return jobGraph; - } - - public void setJobGraph(JobGraph jobGraph) { - this.jobGraph = jobGraph; - } - - public int getParallelism() { - return parallelism; - } - - public void setParallelism(int parallelism) { - this.parallelism = parallelism; - } - - public Configuration getConfiguration() { - return configuration; - } - - public void setConfiguration(Configuration configuration) { - this.configuration = configuration; - } - - public int getMasterMemoryMB() { - return masterMemoryMB; - } - - public int getTaskManagerMemoryMB() { - return taskManagerMemoryMB; - } - - public int getNumberTaskManagers() { - return numberTaskManagers; - } - - public int getSlotsPerTaskManager() { - return slotsPerTaskManager; - } - - public int getPriority(){ - return priority; - } - - public SavepointRestoreSettings getSpSetting() { - return spSetting; - } - - public void setSpSetting(SavepointRestoreSettings spSetting) { - this.spSetting = spSetting; - } - - public List getClasspaths() { - return classpaths; - } - - public void setClasspaths(List classpaths) { - this.classpaths = classpaths; - } - - public String getEntryPointClass() { - return entryPointClass; - } - - public void setEntryPointClass(String entryPointClass) { - this.entryPointClass = entryPointClass; - } - - public String[] getProgramArgs() { - return programArgs; - } - - public void setProgramArgs(String[] programArgs) { - this.programArgs = programArgs; - } - - public File getJarFile() { - return jarFile; - } - - public void setJarFile(File jarFile) { - this.jarFile = jarFile; - } - - public boolean isCreateProgramDelay() { - return createProgramDelay; - } - - public void setCreateProgramDelay(boolean createProgramDelay) { - this.createProgramDelay = createProgramDelay; - } - - public ClassLoaderType getClassLoaderType() { - return classLoaderType; - } - - public void setClassLoaderType(ClassLoaderType classLoaderType) { - this.classLoaderType = classLoaderType; - } - - @Override - public String toString() { - return "ClusterSpecification{" + - "masterMemoryMB=" + masterMemoryMB + - ", taskManagerMemoryMB=" + taskManagerMemoryMB + - ", numberTaskManagers=" + numberTaskManagers + - ", slotsPerTaskManager=" + slotsPerTaskManager + - ", priority=" + priority + - '}'; - } - - public static ClusterSpecification fromConfiguration(Configuration configuration) { - int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - - int jobManagerMemoryMb = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); - int taskManagerMemoryMb = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - - return new ClusterSpecificationBuilder() - .setMasterMemoryMB(jobManagerMemoryMb) - .setTaskManagerMemoryMB(taskManagerMemoryMb) - .setNumberTaskManagers(1) - .setSlotsPerTaskManager(slots) - .createClusterSpecification(); - } - - /** - * Builder for the {@link ClusterSpecification} instance. - */ - public static class ClusterSpecificationBuilder { - private int masterMemoryMB = 768; - private int taskManagerMemoryMB = 768; - private int numberTaskManagers = 1; - private int slotsPerTaskManager = 1; - private int parallelism = 1; - private int priority = 0; - - public ClusterSpecificationBuilder setMasterMemoryMB(int masterMemoryMB) { - this.masterMemoryMB = masterMemoryMB; - return this; - } - - public ClusterSpecificationBuilder setTaskManagerMemoryMB(int taskManagerMemoryMB) { - this.taskManagerMemoryMB = taskManagerMemoryMB; - return this; - } - - public ClusterSpecificationBuilder setNumberTaskManagers(int numberTaskManagers) { - this.numberTaskManagers = numberTaskManagers; - return this; - } - - public ClusterSpecificationBuilder setSlotsPerTaskManager(int slotsPerTaskManager) { - this.slotsPerTaskManager = slotsPerTaskManager; - return this; - } - - public ClusterSpecificationBuilder setPriority(int priority){ - this.priority = priority; - return this; - } - - public ClusterSpecificationBuilder setParallelism(int parallelism) { - this.parallelism = parallelism; - return this; - } - - public ClusterSpecification createClusterSpecification() { - return new ClusterSpecification( - masterMemoryMB, - taskManagerMemoryMB, - numberTaskManagers, - slotsPerTaskManager, - parallelism, - priority); - } - } -} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java b/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java deleted file mode 100644 index 41961ee3e0..0000000000 --- a/flinkx-launcher/src/main/java/org/apache/flink/client/program/JobWithJars.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.client.program; - -import com.dtstack.flinkx.launcher.ClassLoaderType; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.Plan; -import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; - -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.*; -import java.util.concurrent.ConcurrentHashMap; - -/** - * A JobWithJars is a Flink dataflow plan, together with a bunch of JAR files that contain - * the classes of the functions and libraries necessary for the execution. - */ -public class JobWithJars { - - private Plan plan; - - private List jarFiles; - - /** - * classpaths that are needed during user code execution. - */ - private List classpaths; - - private ClassLoader userCodeClassLoader; - - private static Map cacheClassLoader = new ConcurrentHashMap<>(); - - - public JobWithJars(Plan plan, List jarFiles, List classpaths) throws IOException { - this.plan = plan; - this.jarFiles = new ArrayList(jarFiles.size()); - this.classpaths = new ArrayList(classpaths.size()); - - for (URL jarFile: jarFiles) { - checkJarFile(jarFile); - this.jarFiles.add(jarFile); - } - - for (URL path: classpaths) { - this.classpaths.add(path); - } - } - - public JobWithJars(Plan plan, URL jarFile) throws IOException { - this.plan = plan; - - checkJarFile(jarFile); - this.jarFiles = Collections.singletonList(jarFile); - this.classpaths = Collections.emptyList(); - } - - JobWithJars(Plan plan, List jarFiles, List classpaths, ClassLoader userCodeClassLoader) { - this.plan = plan; - this.jarFiles = jarFiles; - this.classpaths = classpaths; - this.userCodeClassLoader = userCodeClassLoader; - } - - /** - * Returns the plan. - */ - public Plan getPlan() { - return this.plan; - } - - /** - * Returns list of jar files that need to be submitted with the plan. - */ - public List getJarFiles() { - return this.jarFiles; - } - - /** - * Returns list of classpaths that need to be submitted with the plan. - */ - public List getClasspaths() { - return classpaths; - } - - /** - * Gets the {@link ClassLoader} that must be used to load user code classes. - * - * @return The user code ClassLoader. - */ - public ClassLoader getUserCodeClassLoader() { - if (this.userCodeClassLoader == null) { - this.userCodeClassLoader = buildUserCodeClassLoader(jarFiles, classpaths, getClass().getClassLoader(), ClassLoaderType.PARENT_FIRST); - } - return this.userCodeClassLoader; - } - - public static void checkJarFile(URL jar) throws IOException { - File jarFile; - try { - jarFile = new File(jar.toURI()); - } catch (URISyntaxException e) { - throw new IOException("JAR file path is invalid '" + jar + "'"); - } - if (!jarFile.exists()) { - throw new IOException("JAR file does not exist '" + jarFile.getAbsolutePath() + "'"); - } - if (!jarFile.canRead()) { - throw new IOException("JAR file can't be read '" + jarFile.getAbsolutePath() + "'"); - } - // TODO: Check if proper JAR file - } - - public static ClassLoader buildUserCodeClassLoader(List jars, List classpaths, ClassLoader parent, ClassLoaderType classLoaderType) { - if (ClassLoaderType.NONE == classLoaderType) { - return parent; - } - URL[] urls = new URL[jars.size() + classpaths.size()]; - for (int i = 0; i < jars.size(); i++) { - urls[i] = jars.get(i); - } - for (int i = 0; i < classpaths.size(); i++) { - urls[i + jars.size()] = classpaths.get(i); - } - switch (classLoaderType) { - case CHILD_FIRST_CACHE: - Arrays.sort(urls, Comparator.comparing(URL::toString)); - String jarsKeyChild = StringUtils.join(urls, "_"); - return cacheClassLoader.computeIfAbsent(jarsKeyChild, k -> FlinkUserCodeClassLoaders.childFirst(urls, parent, new String[]{})); - case PARENT_FIRST_CACHE: - Arrays.sort(urls, Comparator.comparing(URL::toString)); - String jarsKeyParent = StringUtils.join(urls, "_"); - return cacheClassLoader.computeIfAbsent(jarsKeyParent, k -> FlinkUserCodeClassLoaders.parentFirst(urls, parent)); - case CHILD_FIRST: - return FlinkUserCodeClassLoaders.childFirst(urls, parent, new String[]{}); - case PARENT_FIRST: - return FlinkUserCodeClassLoaders.parentFirst(urls, parent); - default: - return FlinkUserCodeClassLoaders.parentFirst(urls, parent); - } - } -} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java deleted file mode 100644 index b56000f4c6..0000000000 --- a/flinkx-launcher/src/main/java/org/apache/flink/client/program/PackagedProgram.java +++ /dev/null @@ -1,773 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.client.program; - -import com.dtstack.flinkx.launcher.ClassLoaderType; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.Program; -import org.apache.flink.api.common.ProgramDescription; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.dag.DataSinkNode; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.util.InstantiationUtil; - -import javax.annotation.Nullable; -import java.io.*; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.net.MalformedURLException; -import java.net.URISyntaxException; -import java.net.URL; -import java.util.*; -import java.util.jar.Attributes; -import java.util.jar.JarEntry; -import java.util.jar.JarFile; -import java.util.jar.Manifest; - -/** - * This class encapsulates represents a program, packaged in a jar file. It supplies - * functionality to extract nested libraries, search for the program entry point, and extract - * a program plan. - */ -public class PackagedProgram { - - /** - * Property name of the entry in JAR manifest file that describes the Flink specific entry point. - */ - public static final String MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS = "program-class"; - - /** - * Property name of the entry in JAR manifest file that describes the class with the main method. - */ - public static final String MANIFEST_ATTRIBUTE_MAIN_CLASS = "Main-Class"; - - // -------------------------------------------------------------------------------------------- - - private final URL jarFile; - - private final String[] args; - - private final Program program; - - private final Class mainClass; - - private final List extractedTempLibraries; - - private final List classpaths; - - private ClassLoader userCodeClassLoader; - - private Plan plan; - - private SavepointRestoreSettings savepointSettings = SavepointRestoreSettings.none(); - - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * argument. - * - * @param jarFile - * The jar file which contains the plan and a Manifest which defines - * the program-class - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, String... args) throws ProgramInvocationException { - this(jarFile, Collections.emptyList(), null, args); - } - - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * argument. - * - * @param jarFile - * The jar file which contains the plan and a Manifest which defines - * the program-class - * @param classpaths - * Additional classpath URLs needed by the Program. - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, List classpaths, ClassLoaderType classLoaderType, String... args) throws ProgramInvocationException { - this(jarFile, classpaths, classLoaderType, null, args); - } - - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * arguments. For generating the plan the class defined in the className parameter - * is used. - * - * @param jarFile - * The jar file which contains the plan. - * @param entryPointClassName - * Name of the class which generates the plan. Overrides the class defined - * in the jar file manifest - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, @Nullable String entryPointClassName, ClassLoaderType classLoaderType, String... args) throws ProgramInvocationException { - this(jarFile, Collections.emptyList(), classLoaderType, entryPointClassName, args); - } - - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * arguments. For generating the plan the class defined in the className parameter - * is used. - * - * @param jarFile - * The jar file which contains the plan. - * @param classpaths - * Additional classpath URLs needed by the Program. - * @param entryPointClassName - * Name of the class which generates the plan. Overrides the class defined - * in the jar file manifest - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, List classpaths, ClassLoaderType classLoaderType, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { - if (jarFile == null) { - throw new IllegalArgumentException("The jar file must not be null."); - } - - URL jarFileUrl; - try { - jarFileUrl = jarFile.getAbsoluteFile().toURI().toURL(); - } catch (MalformedURLException e1) { - throw new IllegalArgumentException("The jar file path is invalid."); - } - - checkJarFile(jarFileUrl); - - this.jarFile = jarFileUrl; - this.args = args == null ? new String[0] : args; - - // if no entryPointClassName name was given, we try and look one up through the manifest - if (entryPointClassName == null) { - entryPointClassName = getEntryPointClassNameFromJar(jarFileUrl); - } - - // now that we have an entry point, we can extract the nested jar files (if any) - this.extractedTempLibraries = extractContainedLibraries(jarFileUrl); - this.classpaths = classpaths; - this.userCodeClassLoader = JobWithJars.buildUserCodeClassLoader(getAllLibraries(), classpaths, getClass().getClassLoader(), classLoaderType); - - // load the entry point class - this.mainClass = loadMainClass(entryPointClassName, userCodeClassLoader); - - // if the entry point is a program, instantiate the class and get the plan - if (Program.class.isAssignableFrom(this.mainClass)) { - Program prg = null; - try { - prg = InstantiationUtil.instantiate(this.mainClass.asSubclass(Program.class), Program.class); - } catch (Exception e) { - // validate that the class has a main method at least. - // the main method possibly instantiates the program properly - if (!hasMainMethod(mainClass)) { - throw new ProgramInvocationException("The given program class implements the " + - Program.class.getName() + " interface, but cannot be instantiated. " + - "It also declares no main(String[]) method as alternative entry point", e); - } - } catch (Throwable t) { - throw new ProgramInvocationException("Error while trying to instantiate program class.", t); - } - this.program = prg; - } else if (hasMainMethod(mainClass)) { - this.program = null; - } else { - throw new ProgramInvocationException("The given program class neither has a main(String[]) method, nor does it implement the " + - Program.class.getName() + " interface."); - } - } - - public PackagedProgram(Class entryPointClass, String... args) throws ProgramInvocationException { - this.jarFile = null; - this.args = args == null ? new String[0] : args; - - this.extractedTempLibraries = Collections.emptyList(); - this.classpaths = Collections.emptyList(); - this.userCodeClassLoader = entryPointClass.getClassLoader(); - - // load the entry point class - this.mainClass = entryPointClass; - - // if the entry point is a program, instantiate the class and get the plan - if (Program.class.isAssignableFrom(this.mainClass)) { - Program prg = null; - try { - prg = InstantiationUtil.instantiate(this.mainClass.asSubclass(Program.class), Program.class); - } catch (Exception e) { - // validate that the class has a main method at least. - // the main method possibly instantiates the program properly - if (!hasMainMethod(mainClass)) { - throw new ProgramInvocationException("The given program class implements the " + - Program.class.getName() + " interface, but cannot be instantiated. " + - "It also declares no main(String[]) method as alternative entry point", e); - } - } catch (Throwable t) { - throw new ProgramInvocationException("Error while trying to instantiate program class.", t); - } - this.program = prg; - } else if (hasMainMethod(mainClass)) { - this.program = null; - } else { - throw new ProgramInvocationException("The given program class neither has a main(String[]) method, nor does it implement the " + - Program.class.getName() + " interface."); - } - } - - public void setSavepointRestoreSettings(SavepointRestoreSettings savepointSettings) { - this.savepointSettings = savepointSettings; - } - - public SavepointRestoreSettings getSavepointSettings() { - return savepointSettings; - } - - public String[] getArguments() { - return this.args; - } - - public String getMainClassName() { - return this.mainClass.getName(); - } - - public boolean isUsingInteractiveMode() { - return this.program == null; - } - - public boolean isUsingProgramEntryPoint() { - return this.program != null; - } - - /** - * Returns the plan without the required jars when the files are already provided by the cluster. - * - * @return The plan without attached jar files. - * @throws ProgramInvocationException - */ - public JobWithJars getPlanWithoutJars() throws ProgramInvocationException { - if (isUsingProgramEntryPoint()) { - return new JobWithJars(getPlan(), Collections.emptyList(), classpaths, userCodeClassLoader); - } else { - throw new ProgramInvocationException("Cannot create a " + JobWithJars.class.getSimpleName() + - " for a program that is using the interactive mode.", getPlan().getJobId()); - } - } - - /** - * Returns the plan with all required jars. - * - * @return The plan with attached jar files. - * @throws ProgramInvocationException - */ - public JobWithJars getPlanWithJars() throws ProgramInvocationException { - if (isUsingProgramEntryPoint()) { - return new JobWithJars(getPlan(), getAllLibraries(), classpaths, userCodeClassLoader); - } else { - throw new ProgramInvocationException("Cannot create a " + JobWithJars.class.getSimpleName() + - " for a program that is using the interactive mode.", getPlan().getJobId()); - } - } - - /** - * Returns the analyzed plan without any optimizations. - * - * @return - * the analyzed plan without any optimizations. - * @throws ProgramInvocationException Thrown if an error occurred in the - * user-provided pact assembler. This may indicate - * missing parameters for generation. - */ - public String getPreviewPlan() throws ProgramInvocationException { - Thread.currentThread().setContextClassLoader(this.getUserCodeClassLoader()); - List previewPlan; - - if (isUsingProgramEntryPoint()) { - previewPlan = Optimizer.createPreOptimizedPlan(getPlan()); - } - else if (isUsingInteractiveMode()) { - // temporary hack to support the web client - PreviewPlanEnvironment env = new PreviewPlanEnvironment(); - env.setAsContext(); - try { - invokeInteractiveModeForExecution(); - } - catch (ProgramInvocationException e) { - throw e; - } - catch (Throwable t) { - // the invocation gets aborted with the preview plan - if (env.previewPlan == null) { - if (env.preview != null) { - return env.preview; - } else { - throw new ProgramInvocationException("The program caused an error: ", getPlan().getJobId(), t); - } - } - } - finally { - env.unsetAsContext(); - } - - if (env.previewPlan != null) { - previewPlan = env.previewPlan; - } else { - throw new ProgramInvocationException( - "The program plan could not be fetched. The program silently swallowed the control flow exceptions.", - getPlan().getJobId()); - } - } - else { - throw new RuntimeException(); - } - - PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator(); - StringWriter string = new StringWriter(1024); - try (PrintWriter pw = new PrintWriter(string)) { - jsonGen.dumpPactPlanAsJSON(previewPlan, pw); - } - return string.toString(); - - } - - /** - * Returns the description provided by the Program class. This - * may contain a description of the plan itself and its arguments. - * - * @return The description of the PactProgram's input parameters. - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - @Nullable - public String getDescription() throws ProgramInvocationException { - if (ProgramDescription.class.isAssignableFrom(this.mainClass)) { - - ProgramDescription descr; - if (this.program != null) { - descr = (ProgramDescription) this.program; - } else { - try { - descr = InstantiationUtil.instantiate( - this.mainClass.asSubclass(ProgramDescription.class), ProgramDescription.class); - } catch (Throwable t) { - return null; - } - } - - try { - return descr.getDescription(); - } - catch (Throwable t) { - throw new ProgramInvocationException("Error while getting the program description" + - (t.getMessage() == null ? "." : ": " + t.getMessage()), t); - } - - } else { - return null; - } - } - - /** - * This method assumes that the context environment is prepared, or the execution - * will be a local execution by default. - */ - public void invokeInteractiveModeForExecution() throws ProgramInvocationException{ - if (isUsingInteractiveMode()) { - callMainMethod(mainClass, args); - } else { - throw new ProgramInvocationException("Cannot invoke a plan-based program directly."); - } - } - - /** - * Returns the classpaths that are required by the program. - * - * @return List of {@link URL}s. - */ - public List getClasspaths() { - return this.classpaths; - } - - /** - * Gets the {@link ClassLoader} that must be used to load user code classes. - * - * @return The user code ClassLoader. - */ - public ClassLoader getUserCodeClassLoader() { - return this.userCodeClassLoader; - } - - /** - * Returns all provided libraries needed to run the program. - */ - public List getAllLibraries() { - List libs = new ArrayList(this.extractedTempLibraries.size() + 1); - - if (jarFile != null) { - libs.add(jarFile); - } - for (File tmpLib : this.extractedTempLibraries) { - try { - libs.add(tmpLib.getAbsoluteFile().toURI().toURL()); - } - catch (MalformedURLException e) { - throw new RuntimeException("URL is invalid. This should not happen.", e); - } - } - - return libs; - } - - /** - * Deletes all temporary files created for contained packaged libraries. - */ - public void deleteExtractedLibraries() { - deleteExtractedLibraries(this.extractedTempLibraries); - this.extractedTempLibraries.clear(); - } - - /** - * Returns the plan as generated from the Pact Assembler. - * - * @return The program's plan. - * @throws ProgramInvocationException Thrown, if an error occurred in the program while - * creating the program's {@link Plan}. - */ - private Plan getPlan() throws ProgramInvocationException { - if (this.plan == null) { - Thread.currentThread().setContextClassLoader(this.userCodeClassLoader); - this.plan = createPlanFromProgram(this.program, this.args); - } - - return this.plan; - } - - private static boolean hasMainMethod(Class entryClass) { - Method mainMethod; - try { - mainMethod = entryClass.getMethod("main", String[].class); - } catch (NoSuchMethodException e) { - return false; - } - catch (Throwable t) { - throw new RuntimeException("Could not look up the main(String[]) method from the class " + - entryClass.getName() + ": " + t.getMessage(), t); - } - - return Modifier.isStatic(mainMethod.getModifiers()) && Modifier.isPublic(mainMethod.getModifiers()); - } - - private static void callMainMethod(Class entryClass, String[] args) throws ProgramInvocationException { - Method mainMethod; - if (!Modifier.isPublic(entryClass.getModifiers())) { - throw new ProgramInvocationException("The class " + entryClass.getName() + " must be public."); - } - - try { - mainMethod = entryClass.getMethod("main", String[].class); - } catch (NoSuchMethodException e) { - throw new ProgramInvocationException("The class " + entryClass.getName() + " has no main(String[]) method."); - } - catch (Throwable t) { - throw new ProgramInvocationException("Could not look up the main(String[]) method from the class " + - entryClass.getName() + ": " + t.getMessage(), t); - } - - if (!Modifier.isStatic(mainMethod.getModifiers())) { - throw new ProgramInvocationException("The class " + entryClass.getName() + " declares a non-static main method."); - } - if (!Modifier.isPublic(mainMethod.getModifiers())) { - throw new ProgramInvocationException("The class " + entryClass.getName() + " declares a non-public main method."); - } - - try { - mainMethod.invoke(null, (Object) args); - } - catch (IllegalArgumentException e) { - throw new ProgramInvocationException("Could not invoke the main method, arguments are not matching.", e); - } - catch (IllegalAccessException e) { - throw new ProgramInvocationException("Access to the main method was denied: " + e.getMessage(), e); - } - catch (InvocationTargetException e) { - Throwable exceptionInMethod = e.getTargetException(); - if (exceptionInMethod instanceof Error) { - throw (Error) exceptionInMethod; - } else if (exceptionInMethod instanceof ProgramParametrizationException) { - throw (ProgramParametrizationException) exceptionInMethod; - } else if (exceptionInMethod instanceof ProgramInvocationException) { - throw (ProgramInvocationException) exceptionInMethod; - } else { - throw new ProgramInvocationException("The main method caused an error: " + exceptionInMethod.getMessage(), exceptionInMethod); - } - } - catch (Throwable t) { - throw new ProgramInvocationException("An error occurred while invoking the program's main method: " + t.getMessage(), t); - } - } - - private static String getEntryPointClassNameFromJar(URL jarFile) throws ProgramInvocationException { - JarFile jar; - Manifest manifest; - String className; - - // Open jar file - try { - jar = new JarFile(new File(jarFile.toURI())); - } catch (URISyntaxException use) { - throw new ProgramInvocationException("Invalid file path '" + jarFile.getPath() + "'", use); - } catch (IOException ioex) { - throw new ProgramInvocationException("Error while opening jar file '" + jarFile.getPath() + "'. " - + ioex.getMessage(), ioex); - } - - // jar file must be closed at the end - try { - // Read from jar manifest - try { - manifest = jar.getManifest(); - } catch (IOException ioex) { - throw new ProgramInvocationException("The Manifest in the jar file could not be accessed '" - + jarFile.getPath() + "'. " + ioex.getMessage(), ioex); - } - - if (manifest == null) { - throw new ProgramInvocationException("No manifest found in jar file '" + jarFile.getPath() + "'. The manifest is need to point to the program's main class."); - } - - Attributes attributes = manifest.getMainAttributes(); - - // check for a "program-class" entry first - className = attributes.getValue(PackagedProgram.MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS); - if (className != null) { - return className; - } - - // check for a main class - className = attributes.getValue(PackagedProgram.MANIFEST_ATTRIBUTE_MAIN_CLASS); - if (className != null) { - return className; - } else { - throw new ProgramInvocationException("Neither a '" + MANIFEST_ATTRIBUTE_MAIN_CLASS + "', nor a '" + - MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS + "' entry was found in the jar file."); - } - } - finally { - try { - jar.close(); - } catch (Throwable t) { - throw new ProgramInvocationException("Could not close the JAR file: " + t.getMessage(), t); - } - } - } - - private static Class loadMainClass(String className, ClassLoader cl) throws ProgramInvocationException { - ClassLoader contextCl = null; - try { - contextCl = Thread.currentThread().getContextClassLoader(); - Thread.currentThread().setContextClassLoader(cl); - return Class.forName(className, false, cl); - } - catch (ClassNotFoundException e) { - throw new ProgramInvocationException("The program's entry point class '" + className - + "' was not found in the jar file.", e); - } - catch (ExceptionInInitializerError e) { - throw new ProgramInvocationException("The program's entry point class '" + className - + "' threw an error during initialization.", e); - } - catch (LinkageError e) { - throw new ProgramInvocationException("The program's entry point class '" + className - + "' could not be loaded due to a linkage failure.", e); - } - catch (Throwable t) { - throw new ProgramInvocationException("The program's entry point class '" + className - + "' caused an exception during initialization: " + t.getMessage(), t); - } finally { - if (contextCl != null) { - Thread.currentThread().setContextClassLoader(contextCl); - } - } - } - - /** - * Takes the jar described by the given file and invokes its pact assembler class to - * assemble a plan. The assembler class name is either passed through a parameter, - * or it is read from the manifest of the jar. The assembler is handed the given options - * for its assembly. - * - * @param program The program to create the plan for. - * @param options - * The options for the assembler. - * @return The plan created by the program. - * @throws ProgramInvocationException - * Thrown, if an error occurred in the user-provided pact assembler. - */ - private static Plan createPlanFromProgram(Program program, String[] options) throws ProgramInvocationException { - try { - return program.getPlan(options); - } catch (Throwable t) { - throw new ProgramInvocationException("Error while calling the program: " + t.getMessage(), t); - } - } - - /** - * Takes all JAR files that are contained in this program's JAR file and extracts them - * to the system's temp directory. - * - * @return The file names of the extracted temporary files. - * @throws ProgramInvocationException Thrown, if the extraction process failed. - */ - public static List extractContainedLibraries(URL jarFile) throws ProgramInvocationException { - - Random rnd = new Random(); - - JarFile jar = null; - try { - jar = new JarFile(new File(jarFile.toURI())); - final List containedJarFileEntries = new ArrayList(); - - Enumeration entries = jar.entries(); - while (entries.hasMoreElements()) { - JarEntry entry = entries.nextElement(); - String name = entry.getName(); - - if (name.length() > 8 && name.startsWith("lib/") && name.endsWith(".jar")) { - containedJarFileEntries.add(entry); - } - } - - if (containedJarFileEntries.isEmpty()) { - return Collections.emptyList(); - } - else { - // go over all contained jar files - final List extractedTempLibraries = new ArrayList(containedJarFileEntries.size()); - final byte[] buffer = new byte[4096]; - - boolean incomplete = true; - - try { - for (int i = 0; i < containedJarFileEntries.size(); i++) { - final JarEntry entry = containedJarFileEntries.get(i); - String name = entry.getName(); - // '/' as in case of zip, jar - // java.util.zip.ZipEntry#isDirectory always looks only for '/' not for File.separator - name = name.replace('/', '_'); - - File tempFile; - try { - tempFile = File.createTempFile(rnd.nextInt(Integer.MAX_VALUE) + "_", name); - tempFile.deleteOnExit(); - } - catch (IOException e) { - throw new ProgramInvocationException( - "An I/O error occurred while creating temporary file to extract nested library '" + - entry.getName() + "'.", e); - } - - extractedTempLibraries.add(tempFile); - - // copy the temp file contents to a temporary File - OutputStream out = null; - InputStream in = null; - try { - - out = new FileOutputStream(tempFile); - in = new BufferedInputStream(jar.getInputStream(entry)); - - int numRead = 0; - while ((numRead = in.read(buffer)) != -1) { - out.write(buffer, 0, numRead); - } - } - catch (IOException e) { - throw new ProgramInvocationException("An I/O error occurred while extracting nested library '" - + entry.getName() + "' to temporary file '" + tempFile.getAbsolutePath() + "'."); - } - finally { - if (out != null) { - out.close(); - } - if (in != null) { - in.close(); - } - } - } - - incomplete = false; - } - finally { - if (incomplete) { - deleteExtractedLibraries(extractedTempLibraries); - } - } - - return extractedTempLibraries; - } - } - catch (Throwable t) { - throw new ProgramInvocationException("Unknown I/O error while extracting contained jar files.", t); - } - finally { - if (jar != null) { - try { - jar.close(); - } catch (Throwable t) {} - } - } - } - - public static void deleteExtractedLibraries(List tempLibraries) { - for (File f : tempLibraries) { - f.delete(); - } - } - - private static void checkJarFile(URL jarfile) throws ProgramInvocationException { - try { - JobWithJars.checkJarFile(jarfile); - } - catch (IOException e) { - throw new ProgramInvocationException(e.getMessage()); - } - catch (Throwable t) { - throw new ProgramInvocationException("Cannot access jar file" + (t.getMessage() == null ? "." : ": " + t.getMessage()), t); - } - } - -} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java deleted file mode 100644 index d8a02e134b..0000000000 --- a/flinkx-launcher/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ /dev/null @@ -1,1766 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.yarn; - -import avro.shaded.com.google.common.collect.Sets; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.common.base.Strings; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.client.deployment.ClusterDeploymentException; -import org.apache.flink.client.deployment.ClusterDescriptor; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.client.program.PackagedProgramUtils; -import org.apache.flink.configuration.*; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.clusterframework.BootstrapTools; -import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.taskexecutor.TaskManagerServices; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.records.*; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.client.api.YarnClientApplication; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.util.Records; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; -import java.io.*; -import java.lang.reflect.Field; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.net.MalformedURLException; -import java.net.URISyntaxException; -import java.net.URL; -import java.nio.file.FileVisitResult; -import java.nio.file.Files; -import java.nio.file.SimpleFileVisitor; -import java.nio.file.attribute.BasicFileAttributes; -import java.util.*; - -import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; -import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.*; - -/** - * The descriptor with deployment information for deploying a Flink cluster on Yarn. - */ -public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor { - private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class); - - private final YarnConfiguration yarnConfiguration; - - private final YarnClient yarnClient; - - /** True if the descriptor must not shut down the YarnClient. */ - private final boolean sharedYarnClient; - - private String yarnQueue; - - private String configurationDirectory; - - private Path flinkJarPath; - - private String dynamicPropertiesEncoded; - - /** Lazily initialized list of files to ship. */ - protected List shipFiles = new LinkedList<>(); - - private final Configuration flinkConfiguration; - - private boolean detached; - - private String customName; - - private String zookeeperNamespace; - - private String nodeLabel; - - /** Optional Jar file to include in the system class loader of all application nodes - * (for per-job submission). */ - private final Set userJarFiles = new HashSet<>(); - - private YarnConfigOptions.UserJarInclusion userJarInclusion; - - public AbstractYarnClusterDescriptor( - Configuration flinkConfiguration, - YarnConfiguration yarnConfiguration, - String configurationDirectory, - YarnClient yarnClient, - boolean sharedYarnClient) { - - this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); - - // for unit tests only - if (System.getenv("IN_TESTS") != null) { - try { - yarnConfiguration.addResource(new File(System.getenv("YARN_CONF_DIR"), "yarn-site.xml").toURI().toURL()); - } catch (Throwable t) { - throw new RuntimeException("Error", t); - } - } - - this.yarnClient = Preconditions.checkNotNull(yarnClient); - this.sharedYarnClient = sharedYarnClient; - - this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); - userJarInclusion = getUserJarInclusionMode(flinkConfiguration); - - this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); - } - - public YarnClient getYarnClient() { - return yarnClient; - } - - /** - * The class to start the application master with. This class runs the main - * method in case of session cluster. - */ - protected abstract String getYarnSessionClusterEntrypoint(); - - /** - * The class to start the application master with. This class runs the main - * method in case of the job cluster. - */ - protected abstract String getYarnJobClusterEntrypoint(); - - public Configuration getFlinkConfiguration() { - return flinkConfiguration; - } - - public void setQueue(String queue) { - this.yarnQueue = queue; - } - - public void setLocalJarPath(Path localJarPath) { - if (!localJarPath.toString().endsWith("jar")) { - throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); - } - this.flinkJarPath = localJarPath; - } - - /** - * Adds the given files to the list of files to ship. - * - *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by - * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, StringBuilder)} - * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. - * - * @param shipFiles files to ship - */ - public void addShipFiles(List shipFiles) { - this.shipFiles.addAll(shipFiles); - } - - /** - * Sets the user jar which is included in the system classloader of all nodes. - */ - public void setProvidedUserJarFiles(List userJarFiles) { - for (URL jarFile : userJarFiles) { - try { - this.userJarFiles.add(new File(jarFile.toURI())); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("Couldn't add local user jar: " + jarFile - + " Currently only file:/// URLs are supported."); - } - } - } - - public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { - this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; - } - - public String getDynamicPropertiesEncoded() { - return this.dynamicPropertiesEncoded; - } - - private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws YarnDeploymentException { - - if (clusterSpecification.getNumberTaskManagers() <= 0) { - throw new YarnDeploymentException("Taskmanager count must be positive"); - } - if (this.flinkJarPath == null) { - throw new YarnDeploymentException("The Flink jar path is null"); - } - if (this.configurationDirectory == null) { - throw new YarnDeploymentException("Configuration directory not set"); - } - if (this.flinkConfiguration == null) { - throw new YarnDeploymentException("Flink configuration object has not been set"); - } - - // Check if we don't exceed YARN's maximum virtual cores. - // Fetch numYarnMaxVcores from all the RUNNING nodes via yarnClient - final int numYarnMaxVcores; - try { - numYarnMaxVcores = yarnClient.getNodeReports(NodeState.RUNNING) - .stream() - .mapToInt(report -> report.getCapability().getVirtualCores()) - .max() - .orElse(0); - } catch (Exception e) { - throw new YarnDeploymentException("Couldn't get cluster description, please check on the YarnConfiguration", e); - } - - int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); - // don't configure more than the maximum configured number of vcores - if (configuredVcores > numYarnMaxVcores) { - throw new IllegalConfigurationException( - String.format("The number of requested virtual cores per node %d" + - " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + - " Please note that the number of virtual cores is set to the number of task slots by default" + - " unless configured in the Flink config with '%s.'", - configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); - } - - // check if required Hadoop environment variables are set. If not, warn user - if (System.getenv("HADOOP_CONF_DIR") == null && - System.getenv("YARN_CONF_DIR") == null) { - LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + - "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + - "configuration for accessing YARN."); - } - } - - private static boolean allocateResource(int[] nodeManagers, int toAllocate) { - for (int i = 0; i < nodeManagers.length; i++) { - if (nodeManagers[i] >= toAllocate) { - nodeManagers[i] -= toAllocate; - return true; - } - } - return false; - } - - /** - * @deprecated The cluster descriptor should not know about this option. - */ - @Deprecated - public void setDetachedMode(boolean detachedMode) { - this.detached = detachedMode; - } - - /** - * @deprecated The cluster descriptor should not know about this option. - */ - @Deprecated - public boolean isDetachedMode() { - return detached; - } - - public String getZookeeperNamespace() { - return zookeeperNamespace; - } - - public void setZookeeperNamespace(String zookeeperNamespace) { - this.zookeeperNamespace = zookeeperNamespace; - } - - public String getNodeLabel() { - return nodeLabel; - } - - public void setNodeLabel(String nodeLabel) { - this.nodeLabel = nodeLabel; - } - - // ------------------------------------------------------------- - // Lifecycle management - // ------------------------------------------------------------- - - @Override - public void close() { - if (!sharedYarnClient) { - yarnClient.stop(); - } - } - - // ------------------------------------------------------------- - // ClusterClient overrides - // ------------------------------------------------------------- - - @Override - public ClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException { - - try { - // check if required Hadoop environment variables are set. If not, warn user - if (System.getenv("HADOOP_CONF_DIR") == null && - System.getenv("YARN_CONF_DIR") == null) { - LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + - "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + - "configuration for accessing YARN."); - } - - final ApplicationReport appReport = yarnClient.getApplicationReport(applicationId); - - if (appReport.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { - // Flink cluster is not running anymore - LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}", - applicationId, appReport.getFinalApplicationStatus()); - throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); - } - - final String host = appReport.getHost(); - final int rpcPort = appReport.getRpcPort(); - - LOG.info("Found application JobManager host name '{}' and port '{}' from supplied application id '{}'", - host, rpcPort, applicationId); - - flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); - flinkConfiguration.setInteger(JobManagerOptions.PORT, rpcPort); - - flinkConfiguration.setString(RestOptions.ADDRESS, host); - flinkConfiguration.setInteger(RestOptions.PORT, rpcPort); - - return createYarnClusterClient( - this, - -1, // we don't know the number of task managers of a started Flink cluster - -1, // we don't know how many slots each task manager has for a started Flink cluster - appReport, - flinkConfiguration, - false); - } catch (Exception e) { - throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); - } - } - - @Override - public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException { - try { - return deployInternal( - clusterSpecification, - "Flink session cluster", - getYarnSessionClusterEntrypoint(), - null, - false); - } catch (Exception e) { - throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); - } - } - - @Override - public void killCluster(ApplicationId applicationId) throws FlinkException { - try { - yarnClient.killApplication(applicationId); - Utils.deleteApplicationFiles(Collections.singletonMap( - YarnConfigKeys.FLINK_YARN_FILES, - getYarnFilesDir(applicationId).toUri().toString())); - } catch (YarnException | IOException e) { - throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); - } - } - - /** - * Method to validate cluster specification before deploy it, it will throw - * an {@link FlinkException} if the {@link ClusterSpecification} is invalid. - * - * @param clusterSpecification cluster specification to check against the configuration of the - * AbstractYarnClusterDescriptor - * @throws FlinkException if the cluster cannot be started with the provided {@link ClusterSpecification} - */ - private void validateClusterSpecification(ClusterSpecification clusterSpecification) throws FlinkException { - try { - final long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); - // We do the validation by calling the calculation methods here - // Internally these methods will check whether the cluster can be started with the provided - // ClusterSpecification and the configured memory requirements - final long cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); - TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); - } catch (IllegalArgumentException iae) { - throw new FlinkException("Cannot fulfill the minimum memory requirements with the provided " + - "cluster specification. Please increase the memory of the cluster.", iae); - } - } - - /** - * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. - * - * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed - * @param applicationName name of the Yarn application to start - * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. - * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none - * @param detached True if the cluster should be started in detached mode - */ - protected ClusterClient deployInternal( - ClusterSpecification clusterSpecification, - String applicationName, - String yarnClusterEntrypoint, - @Nullable JobGraph jobGraph, - boolean detached) throws Exception { - - // ------------------ Check if configuration is valid -------------------- - validateClusterSpecification(clusterSpecification); - - if (UserGroupInformation.isSecurityEnabled()) { - // note: UGI::hasKerberosCredentials inaccurately reports false - // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), - // so we check only in ticket cache scenario. - boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); - - UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); - if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS - && useTicketCache && !loginUser.hasKerberosCredentials()) { - LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); - throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + - "does not have Kerberos credentials"); - } - } - - isReadyForDeployment(clusterSpecification); - - // ------------------ Check if the specified queue exists -------------------- - - checkYarnQueues(yarnClient); - - // ------------------ Add dynamic properties to local flinkConfiguraton ------ - Map dynProperties = getDynamicProperties(dynamicPropertiesEncoded); - for (Map.Entry dynProperty : dynProperties.entrySet()) { - flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue()); - } - - // ------------------ Check if the YARN ClusterClient has the requested resources -------------- - - // Create application via yarnClient - final YarnClientApplication yarnApplication = yarnClient.createApplication(); - final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); - - if(clusterSpecification.isCreateProgramDelay()){ - jobGraph = getJobGraph(appResponse.getApplicationId().toString(),clusterSpecification); - } - - Resource maxRes = appResponse.getMaximumResourceCapability(); - - final ClusterResourceDescription freeClusterMem; - try { - freeClusterMem = getCurrentFreeClusterResources(yarnClient); - } catch (YarnException | IOException e) { - failSessionDuringDeployment(yarnClient, yarnApplication); - throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); - } - - final int yarnMinAllocationMB = yarnConfiguration.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); - - final ClusterSpecification validClusterSpecification; - try { - validClusterSpecification = validateClusterResources( - clusterSpecification, - yarnMinAllocationMB, - maxRes, - freeClusterMem); - } catch (YarnDeploymentException yde) { - failSessionDuringDeployment(yarnClient, yarnApplication); - throw yde; - } - - LOG.info("Cluster specification: {}", validClusterSpecification); - - final ClusterEntrypoint.ExecutionMode executionMode = detached ? - ClusterEntrypoint.ExecutionMode.DETACHED - : ClusterEntrypoint.ExecutionMode.NORMAL; - - flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); - - ApplicationReport report = startAppMaster( - flinkConfiguration, - applicationName, - yarnClusterEntrypoint, - jobGraph, - yarnClient, - yarnApplication, - validClusterSpecification); - - String host = report.getHost(); - int port = report.getRpcPort(); - - // Correctly initialize the Flink config - flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); - flinkConfiguration.setInteger(JobManagerOptions.PORT, port); - - flinkConfiguration.setString(RestOptions.ADDRESS, host); - flinkConfiguration.setInteger(RestOptions.PORT, port); - - // the Flink cluster is deployed in YARN. Represent cluster - return createYarnClusterClient( - this, - validClusterSpecification.getNumberTaskManagers(), - validClusterSpecification.getSlotsPerTaskManager(), - report, - flinkConfiguration, - true); - } - - private JobGraph getJobGraph(String appId,ClusterSpecification clusterSpecification) throws Exception{ - String url = getUrlFormat(clusterSpecification.getYarnConfiguration()) + "/" + appId; - PackagedProgram program = buildProgram(url,clusterSpecification); - clusterSpecification.setProgram(program); - JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, clusterSpecification.getConfiguration(), clusterSpecification.getParallelism()); - jobGraph.setAllowQueuedScheduling(true); - dealPluginByLoadMode(jobGraph); - clusterSpecification.setJobGraph(jobGraph); - return jobGraph; - } - - private PackagedProgram buildProgram(String monitorUrl,ClusterSpecification clusterSpecification) throws Exception{ - String[] args = clusterSpecification.getProgramArgs(); - for (int i = 0; i < args.length; i++) { - if("-monitor".equals(args[i])){ - args[i + 1] = monitorUrl; - break; - } - } - - PackagedProgram program = new PackagedProgram(clusterSpecification.getJarFile(), clusterSpecification.getClasspaths(), clusterSpecification.getClassLoaderType(), clusterSpecification.getEntryPointClass(), args); - - if (null != clusterSpecification.getSpSetting()) { - program.setSavepointRestoreSettings(clusterSpecification.getSpSetting()); - } else { - program.setSavepointRestoreSettings(SavepointRestoreSettings.none()); - } - - return program; - } - - private void dealPluginByLoadMode(JobGraph jobGraph) throws Exception { - String pluginLoadMode = flinkConfiguration.getString("pluginLoadMode", "shipfile"); - if (StringUtils.equalsIgnoreCase(pluginLoadMode, "classpath")) { - fillJobGraphClassPath(jobGraph); - fillStreamJobGraphClassPath(jobGraph); - } else { - fillPluginPathToShipFiles(jobGraph); - } - } - - private void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { - Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); - Set classPathKeySet = Sets.newHashSet(); - fillClassPathKeySet(jobCacheFileConfig, classPathKeySet); - for(String key : classPathKeySet){ - String pathStr = jobCacheFileConfig.get(key); - jobGraph.getClasspaths().add(new URL("file:" + pathStr)); - } - } - - private void fillClassPathKeySet(Map jobCacheFileConfig, Set classPathKeySet) { - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(Strings.isNullOrEmpty(tmp.getValue())){ - continue; - } - - if(tmp.getValue().startsWith("class_path")){ - //DISTRIBUTED_CACHE_FILE_NAME_1 - //DISTRIBUTED_CACHE_FILE_PATH_1 - String key = tmp.getKey(); - String[] array = key.split("_"); - if(array.length < 5){ - continue; - } - - array[3] = "PATH"; - classPathKeySet.add(StringUtils.join(array, "_")); - } - } - } - - private JobGraph fillStreamJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { - Map jobCacheFileConfig = jobGraph.getUserArtifacts(); - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(tmp.getKey().startsWith("class_path")){ - jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); - } - } - return jobGraph; - } - - private void fillPluginPathToShipFiles(JobGraph jobGraph) { - List shipFiles = new ArrayList<>(); - // flinksql get classpath - Map jobCacheFileConfig = jobGraph.getUserArtifacts(); - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(tmp.getKey().startsWith("class_path")){ - shipFiles.add(new File(tmp.getValue().filePath)); - } - } - // flinkx get classpath - jobGraph.getClasspaths().forEach(jarFile -> { - try { - shipFiles.add(new File(jarFile.toURI())); - } catch (URISyntaxException e) { - throw new IllegalArgumentException("Couldn't add local user jar: " + jarFile - + " Currently only file:/// URLs are supported."); - } - }); - jobGraph.getClasspaths().clear(); - addShipFiles(shipFiles); - } - - private String getUrlFormat(YarnConfiguration yarnConf){ - String url = ""; - try{ - Field rmClientField = yarnClient.getClass().getDeclaredField("rmClient"); - rmClientField.setAccessible(true); - Object rmClient = rmClientField.get(yarnClient); - - Field hField = rmClient.getClass().getSuperclass().getDeclaredField("h"); - hField.setAccessible(true); - //获取指定对象中此字段的值 - Object h = hField.get(rmClient); - Object currentProxy = null; - - try { - Field currentProxyField = h.getClass().getDeclaredField("currentProxy"); - currentProxyField.setAccessible(true); - currentProxy = currentProxyField.get(h); - }catch (Exception e){ - //兼容Hadoop 2.7.3.2.6.4.91-3 - LOG.warn("get currentProxy error:{}", ExceptionUtil.getErrorMessage(e)); - Field proxyDescriptorField = h.getClass().getDeclaredField("proxyDescriptor"); - proxyDescriptorField.setAccessible(true); - Object proxyDescriptor = proxyDescriptorField.get(h); - Field currentProxyField = proxyDescriptor.getClass().getDeclaredField("proxyInfo"); - currentProxyField.setAccessible(true); - currentProxy = currentProxyField.get(proxyDescriptor); - } - - Field proxyInfoField = currentProxy.getClass().getDeclaredField("proxyInfo"); - proxyInfoField.setAccessible(true); - String proxyInfoKey = (String) proxyInfoField.get(currentProxy); - - String key = "yarn.resourcemanager.webapp.address." + proxyInfoKey; - String addr = yarnConf.get(key); - - if(addr == null) { - addr = yarnConf.get("yarn.resourcemanager.webapp.address"); - } - - return String.format("http://%s/proxy",addr); - }catch (Exception e){ - LOG.warn("get monitor error:{}", ExceptionUtil.getErrorMessage(e)); - } - - return url; - } - - protected ClusterSpecification validateClusterResources( - ClusterSpecification clusterSpecification, - int yarnMinAllocationMB, - Resource maximumResourceCapability, - ClusterResourceDescription freeClusterResources) throws YarnDeploymentException { - - int taskManagerCount = clusterSpecification.getNumberTaskManagers(); - int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); - int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); - - if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { - LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " - + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + - "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + - "you requested will start."); - } - - // set the memory to minAllocationMB to do the next checks correctly - if (jobManagerMemoryMb < yarnMinAllocationMB) { - jobManagerMemoryMb = yarnMinAllocationMB; - } - if (taskManagerMemoryMb < yarnMinAllocationMB) { - taskManagerMemoryMb = yarnMinAllocationMB; - } - - final String note = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; - if (jobManagerMemoryMb > maximumResourceCapability.getMemory()) { - throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" - + "Maximum Memory: " + maximumResourceCapability.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + note); - } - - if (taskManagerMemoryMb > maximumResourceCapability.getMemory()) { - throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" - + "Maximum Memory: " + maximumResourceCapability.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + note); - } - - final String noteRsc = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + - "connecting from the beginning because the resources are currently not available in the cluster. " + - "The allocation might take more time than usual because the Flink YARN client needs to wait until " + - "the resources become available."; - int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount; - - if (freeClusterResources.totalFreeMemory < totalMemoryRequired) { - LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. " - + "There are currently only " + freeClusterResources.totalFreeMemory + "MB available." + noteRsc); - - } - if (taskManagerMemoryMb > freeClusterResources.containerLimit) { - LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " - + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); - } - if (jobManagerMemoryMb > freeClusterResources.containerLimit) { - LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " - + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); - } - - // ----------------- check if the requested containers fit into the cluster. - - int[] nmFree = Arrays.copyOf(freeClusterResources.nodeManagersFree, freeClusterResources.nodeManagersFree.length); - // first, allocate the jobManager somewhere. - if (!allocateResource(nmFree, jobManagerMemoryMb)) { - LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " + - "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " + - Arrays.toString(freeClusterResources.nodeManagersFree) + noteRsc); - } - // allocate TaskManagers - for (int i = 0; i < taskManagerCount; i++) { - if (!allocateResource(nmFree, taskManagerMemoryMb)) { - LOG.warn("There is not enough memory available in the YARN cluster. " + - "The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " + - "NodeManagers available: " + Arrays.toString(freeClusterResources.nodeManagersFree) + "\n" + - "After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " + - "the following NodeManagers are available: " + Arrays.toString(nmFree) + noteRsc); - } - } - - return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobManagerMemoryMb) - .setTaskManagerMemoryMB(taskManagerMemoryMb) - .setNumberTaskManagers(clusterSpecification.getNumberTaskManagers()) - .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) - .createClusterSpecification(); - - } - - private void checkYarnQueues(YarnClient yarnClient) { - try { - List queues = yarnClient.getAllQueues(); - if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. - boolean queueFound = false; - for (QueueInfo queue : queues) { - if (queue.getQueueName().equals(this.yarnQueue)) { - queueFound = true; - break; - } - } - if (!queueFound) { - String queueNames = ""; - for (QueueInfo queue : queues) { - queueNames += queue.getQueueName() + ", "; - } - LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + - "Available queues: " + queueNames); - } - } else { - LOG.debug("The YARN cluster does not have any queues configured"); - } - } catch (Throwable e) { - LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.debug("Error details", e); - } - } - } - - public ApplicationReport startAppMaster( - Configuration configuration, - String applicationName, - String yarnClusterEntrypoint, - JobGraph jobGraph, - YarnClient yarnClient, - YarnClientApplication yarnApplication, - ClusterSpecification clusterSpecification) throws Exception { - - // ------------------ Initialize the file systems ------------------------- - - try { - org.apache.flink.core.fs.FileSystem.initialize(configuration); - } catch (IOException e) { - throw new IOException("Error while setting the default " + - "filesystem scheme from configuration.", e); - } - - // initialize file system - // Copy the application master jar to the filesystem - // Create a local resource to point to the destination jar path - final FileSystem fs = FileSystem.get(yarnConfiguration); - final Path homeDir = fs.getHomeDirectory(); - - // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. - if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && - fs.getScheme().startsWith("file")) { - LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the " - + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." - + "The Flink YARN client needs to store its files in a distributed file system"); - } - - ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); - Set systemShipFiles = new HashSet<>(shipFiles.size()); - for (File file : shipFiles) { - systemShipFiles.add(file.getAbsoluteFile()); - } - - //check if there is a logback or log4j file - File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); - final boolean hasLogback = logbackFile.exists(); - if (hasLogback) { - systemShipFiles.add(logbackFile); - } - - File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); - final boolean hasLog4j = log4jFile.exists(); - if (hasLog4j) { - systemShipFiles.add(log4jFile); - if (hasLogback) { - // this means there is already a logback configuration file --> fail - LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + - "Logback configuration files. Please delete or rename one of them."); - } - } - - addLibFolderToShipFiles(systemShipFiles); - - // Set-up ApplicationSubmissionContext for the application - - final ApplicationId appId = appContext.getApplicationId(); - - // ------------------ Add Zookeeper namespace to local flinkConfiguraton ------ - String zkNamespace = getZookeeperNamespace(); - // no user specified cli argument for namespace? - if (zkNamespace == null || zkNamespace.isEmpty()) { - // namespace defined in config? else use applicationId as default. - zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); - setZookeeperNamespace(zkNamespace); - } - - configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); - - if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { - // activate re-execution of failed applications - appContext.setMaxAppAttempts( - configuration.getInteger( - YarnConfigOptions.APPLICATION_ATTEMPTS.key(), - YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); - - activateHighAvailabilitySupport(appContext); - } else { - // set number of application retries to 1 in the default case - appContext.setMaxAppAttempts( - configuration.getInteger( - YarnConfigOptions.APPLICATION_ATTEMPTS.key(), - 1)); - } - - if (jobGraph != null) { - // add the user code jars from the provided JobGraph - for (org.apache.flink.core.fs.Path path : jobGraph.getUserJars()) { - userJarFiles.add(new File(path.toUri())); - } - } - - // local resource map for Yarn - final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); - // list of remote paths (after upload) - final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); - // ship list that enables reuse of resources for task manager containers - StringBuilder envShipFileList = new StringBuilder(); - - // upload and register ship files - List systemClassPaths = uploadAndRegisterFiles( - systemShipFiles, - fs, - homeDir, - appId, - paths, - localResources, - envShipFileList); - - final List userClassPaths = uploadAndRegisterFiles( - userJarFiles, - fs, - homeDir, - appId, - paths, - localResources, - envShipFileList); - - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { - systemClassPaths.addAll(userClassPaths); - } - - // normalize classpath by sorting - Collections.sort(systemClassPaths); - Collections.sort(userClassPaths); - - // classpath assembler - StringBuilder classPathBuilder = new StringBuilder(); - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { - for (String userClassPath : userClassPaths) { - classPathBuilder.append(userClassPath).append(File.pathSeparator); - } - } - for (String classPath : systemClassPaths) { - classPathBuilder.append(classPath).append(File.pathSeparator); - } - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { - for (String userClassPath : userClassPaths) { - classPathBuilder.append(userClassPath).append(File.pathSeparator); - } - } - - // Setup jar for ApplicationMaster - Path remotePathJar = setupSingleLocalResource( - "flink.jar", - fs, - appId, - flinkJarPath, - localResources, - homeDir, - ""); - - // set the right configuration values for the TaskManager - configuration.setInteger( - TaskManagerOptions.NUM_TASK_SLOTS, - clusterSpecification.getSlotsPerTaskManager()); - - configuration.setString( - TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, - clusterSpecification.getTaskManagerMemoryMB() + "m"); - - // Upload the flink configuration - // write out configuration file - File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); - tmpConfigurationFile.deleteOnExit(); - BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); - - Path remotePathConf = setupSingleLocalResource( - "flink-conf.yaml", - fs, - appId, - new Path(tmpConfigurationFile.getAbsolutePath()), - localResources, - homeDir, - ""); - - paths.add(remotePathJar); - classPathBuilder.append("flink.jar").append(File.pathSeparator); - paths.add(remotePathConf); - classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); - - // write job graph to tmp file and add it to local resource - // TODO: server use user main method to generate job graph - if (jobGraph != null) { - try { - File fp = File.createTempFile(appId.toString(), null); - fp.deleteOnExit(); - try (FileOutputStream output = new FileOutputStream(fp); - ObjectOutputStream obOutput = new ObjectOutputStream(output);){ - obOutput.writeObject(jobGraph); - } - - final String jobGraphFilename = "job.graph"; - flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); - - Path pathFromYarnURL = setupSingleLocalResource( - jobGraphFilename, - fs, - appId, - new Path(fp.toURI()), - localResources, - homeDir, - ""); - paths.add(pathFromYarnURL); - classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); - } catch (Exception e) { - LOG.warn("Add job graph to local resource fail"); - throw e; - } - } - - final Path yarnFilesDir = getYarnFilesDir(appId); - FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); - fs.setPermission(yarnFilesDir, permission); // set permission for path. - - //To support Yarn Secure Integration Test Scenario - //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML - //and KRB5 configuration files. We are adding these files as container local resources for the container - //applications (JM/TMs) to have proper secure cluster setup - Path remoteKrb5Path = null; - Path remoteYarnSiteXmlPath = null; - boolean hasKrb5 = false; - if (System.getenv("IN_TESTS") != null) { - String krb5Config = System.getProperty("java.security.krb5.conf"); - if (krb5Config != null && krb5Config.length() != 0) { - File krb5 = new File(krb5Config); - LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); - Path krb5ConfPath = new Path(krb5.getAbsolutePath()); - remoteKrb5Path = setupSingleLocalResource( - Utils.KRB5_FILE_NAME, - fs, - appId, - krb5ConfPath, - localResources, - homeDir, - ""); - - File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); - LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); - Path yarnSitePath = new Path(f.getAbsolutePath()); - remoteYarnSiteXmlPath = setupSingleLocalResource( - Utils.YARN_SITE_FILE_NAME, - fs, - appId, - yarnSitePath, - localResources, - homeDir, - ""); - hasKrb5 = true; - } - } - - // setup security tokens - Path remotePathKeytab = null; - String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); - if (keytab != null) { - LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); - remotePathKeytab = setupSingleLocalResource( - Utils.KEYTAB_FILE_NAME, - fs, - appId, - new Path(keytab), - localResources, - homeDir, - ""); - } - - final ContainerLaunchContext amContainer = setupApplicationMasterContainer( - yarnClusterEntrypoint, - hasLogback, - hasLog4j, - hasKrb5, - clusterSpecification.getMasterMemoryMB()); - - if (UserGroupInformation.isSecurityEnabled()) { - // set HDFS delegation tokens when security is enabled - LOG.info("Adding delegation token to the AM container.."); - Utils.setTokensFor(amContainer, paths, yarnConfiguration); - } - - amContainer.setLocalResources(localResources); - fs.close(); - - // Setup CLASSPATH and environment variables for ApplicationMaster - final Map appMasterEnv = new HashMap<>(); - // set user specified app master environment variables - appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); - // set Flink app class path - appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); - - // set Flink on YARN internal configuration values - appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(clusterSpecification.getNumberTaskManagers())); - appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(clusterSpecification.getTaskManagerMemoryMB())); - appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(clusterSpecification.getSlotsPerTaskManager())); - appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); - appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); - appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); - - // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name - appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - - if (remotePathKeytab != null) { - appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); - String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); - appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); - } - - //To support Yarn Secure Integration Test Scenario - if (remoteYarnSiteXmlPath != null && remoteKrb5Path != null) { - appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); - } - - if (dynamicPropertiesEncoded != null) { - appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); - } - - // set classpath from YARN configuration - Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); - - amContainer.setEnvironment(appMasterEnv); - - // Set up resource type requirements for ApplicationMaster - Resource capability = Records.newRecord(Resource.class); - capability.setMemory(clusterSpecification.getMasterMemoryMB()); - capability.setVirtualCores(1); - - final String customApplicationName = customName != null ? customName : applicationName; - - appContext.setApplicationName(customApplicationName); - appContext.setApplicationType("Apache Flink"); - appContext.setAMContainerSpec(amContainer); - appContext.setResource(capability); - - if (yarnQueue != null) { - appContext.setQueue(yarnQueue); - } - - setApplicationNodeLabel(appContext); - - setApplicationTags(appContext); - - // add a hook to clean up in case deployment fails - Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication, yarnFilesDir); - Runtime.getRuntime().addShutdownHook(deploymentFailureHook); - LOG.info("Submitting application master " + appId); - yarnClient.submitApplication(appContext); - - LOG.info("Waiting for the cluster to be allocated"); - final long startTime = System.currentTimeMillis(); - ApplicationReport report; - YarnApplicationState lastAppState = YarnApplicationState.NEW; - loop: while (true) { - try { - report = yarnClient.getApplicationReport(appId); - } catch (IOException e) { - throw new YarnDeploymentException("Failed to deploy the cluster.", e); - } - YarnApplicationState appState = report.getYarnApplicationState(); - LOG.debug("Application State: {}", appState); - switch(appState) { - case FAILED: - case FINISHED: - case KILLED: - throw new YarnDeploymentException("The YARN application unexpectedly switched to state " - + appState + " during deployment. \n" + - "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + - "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + - "yarn logs -applicationId " + appId); - //break .. - case RUNNING: - LOG.info("YARN application has been deployed successfully."); - break loop; - default: - if (appState != lastAppState) { - LOG.info("Deploying cluster, current state " + appState); - } - if (System.currentTimeMillis() - startTime > 60000) { - LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); - } - - } - lastAppState = appState; - Thread.sleep(250); - } - // print the application id for user to cancel themselves. - if (isDetachedMode()) { - LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + - "Flink on YARN, use the following command or a YARN web interface to stop " + - "it:\nyarn application -kill " + appId + "\nPlease also note that the " + - "temporary files of the YARN session in the home directory will not be removed."); - } - // since deployment was successful, remove the hook - ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); - return report; - } - - /** - * Returns the Path where the YARN application files should be uploaded to. - * - * @param appId YARN application id - */ - private Path getYarnFilesDir(final ApplicationId appId) throws IOException { - final FileSystem fileSystem = FileSystem.get(yarnConfiguration); - final Path homeDir = fileSystem.getHomeDirectory(); - return new Path(homeDir, ".flink/" + appId + '/'); - } - - /** - * Uploads and registers a single resource and adds it to localResources. - * - * @param key - * the key to add the resource under - * @param fs - * the remote file system to upload to - * @param appId - * application ID - * @param localSrcPath - * local path to the file - * @param localResources - * map of resources - * - * @return the remote path to the uploaded resource - */ - private static Path setupSingleLocalResource( - String key, - FileSystem fs, - ApplicationId appId, - Path localSrcPath, - Map localResources, - Path targetHomeDir, - String relativeTargetPath) throws IOException, URISyntaxException { - - Tuple2 resource = Utils.setupLocalResource( - fs, - appId.toString(), - localSrcPath, - targetHomeDir, - relativeTargetPath); - - localResources.put(key, resource.f1); - - return resource.f0; - } - - /** - * Recursively uploads (and registers) any (user and system) files in shipFiles except - * for files matching "flink-dist*.jar" which should be uploaded separately. - * - * @param shipFiles - * files to upload - * @param fs - * file system to upload to - * @param targetHomeDir - * remote home directory to upload to - * @param appId - * application ID - * @param remotePaths - * paths of the remote resources (uploaded resources will be added) - * @param localResources - * map of resources (uploaded resources will be added) - * @param envShipFileList - * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} - * - * @return list of class paths with the the proper resource keys from the registration - */ - static List uploadAndRegisterFiles( - Collection shipFiles, - FileSystem fs, - Path targetHomeDir, - ApplicationId appId, - List remotePaths, - Map localResources, - StringBuilder envShipFileList) throws IOException, URISyntaxException { - - final List classPaths = new ArrayList<>(2 + shipFiles.size()); - for (File shipFile : shipFiles) { - if (shipFile.isDirectory()) { - // add directories to the classpath - java.nio.file.Path shipPath = shipFile.toPath(); - final java.nio.file.Path parentPath = shipPath.getParent(); - - Files.walkFileTree(shipPath, new SimpleFileVisitor() { - @Override - public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) - throws IOException { - String fileName = file.getFileName().toString(); - if (!(fileName.startsWith("flink-dist") && - fileName.endsWith("jar"))) { - - java.nio.file.Path relativePath = parentPath.relativize(file); - - String key = relativePath.toString(); - try { - Path remotePath = setupSingleLocalResource( - key, - fs, - appId, - new Path(file.toUri()), - localResources, - targetHomeDir, - relativePath.getParent().toString()); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=") - .append(remotePath).append(","); - - // add files to the classpath - classPaths.add(key); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - return FileVisitResult.CONTINUE; - } - }); - } else { - if (!(shipFile.getName().startsWith("flink-dist") && shipFile.getName().endsWith("jar"))) { - Path shipLocalPath = new Path(shipFile.toURI()); - String key = shipFile.getName(); - Path remotePath = setupSingleLocalResource( - key, fs, appId, shipLocalPath, localResources, targetHomeDir, ""); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=").append(remotePath).append(","); - - // add files to the classpath - classPaths.add(key); - } - } - - } - return classPaths; - } - - /** - * Kills YARN application and stops YARN client. - * - *

Use this method to kill the App before it has been properly deployed - */ - private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { - LOG.info("Killing YARN application"); - - try { - yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); - } catch (Exception e) { - // we only log a debug message here because the "killApplication" call is a best-effort - // call (we don't know if the application has been deployed when the error occured). - LOG.debug("Error while killing YARN application", e); - } - yarnClient.stop(); - } - - private static class ClusterResourceDescription { - public final int totalFreeMemory; - public final int containerLimit; - public final int[] nodeManagersFree; - - public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { - this.totalFreeMemory = totalFreeMemory; - this.containerLimit = containerLimit; - this.nodeManagersFree = nodeManagersFree; - } - } - - private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - - int totalFreeMemory = 0; - int containerLimit = 0; - int[] nodeManagersFree = new int[nodes.size()]; - - for (int i = 0; i < nodes.size(); i++) { - NodeReport rep = nodes.get(i); - int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0); - nodeManagersFree[i] = free; - totalFreeMemory += free; - if (free > containerLimit) { - containerLimit = free; - } - } - return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); - } - - @Override - public String getClusterDescription() { - - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PrintStream ps = new PrintStream(baos); - - YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); - - ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - final String format = "|%-16s |%-16s %n"; - ps.printf("|Property |Value %n"); - ps.println("+---------------------------------------+"); - int totalMemory = 0; - int totalCores = 0; - for (NodeReport rep : nodes) { - final Resource res = rep.getCapability(); - totalMemory += res.getMemory(); - totalCores += res.getVirtualCores(); - ps.format(format, "NodeID", rep.getNodeId()); - ps.format(format, "Memory", res.getMemory() + " MB"); - ps.format(format, "vCores", res.getVirtualCores()); - ps.format(format, "HealthReport", rep.getHealthReport()); - ps.format(format, "Containers", rep.getNumContainers()); - ps.println("+---------------------------------------+"); - } - ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); - List qInfo = yarnClient.getAllQueues(); - for (QueueInfo q : qInfo) { - ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + - q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); - } - return baos.toString(); - } catch (Exception e) { - throw new RuntimeException("Couldn't get cluster description", e); - } - } - - public void setName(String name) { - if (name == null) { - throw new IllegalArgumentException("The passed name is null"); - } - customName = name; - } - - private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws - InvocationTargetException, IllegalAccessException { - - ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - - reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); - reflector.setAttemptFailuresValidityInterval(appContext, AkkaUtils.getTimeout(flinkConfiguration).toMillis()); - } - - private void setApplicationTags(final ApplicationSubmissionContext appContext) throws InvocationTargetException, - IllegalAccessException { - - final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - final String tagsString = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TAGS); - - final Set applicationTags = new HashSet<>(); - - // Trim whitespace and cull empty tags - for (final String tag : tagsString.split(",")) { - final String trimmedTag = tag.trim(); - if (!trimmedTag.isEmpty()) { - applicationTags.add(trimmedTag); - } - } - - reflector.setApplicationTags(appContext, applicationTags); - } - - private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) throws InvocationTargetException, - IllegalAccessException { - - if (nodeLabel != null) { - final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - reflector.setApplicationNodeLabel(appContext, nodeLabel); - } - } - - /** - * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext} - * supports various methods which, depending on the Hadoop version, may or may not be supported. - * - *

If an unsupported method is invoked, nothing happens. - * - *

Currently three methods are proxied: - * - setApplicationTags (>= 2.4.0) - * - setAttemptFailuresValidityInterval (>= 2.6.0) - * - setKeepContainersAcrossApplicationAttempts (>= 2.4.0) - * - setNodeLabelExpression (>= 2.6.0) - */ - private static class ApplicationSubmissionContextReflector { - private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); - - private static final ApplicationSubmissionContextReflector instance = - new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); - - public static ApplicationSubmissionContextReflector getInstance() { - return instance; - } - - private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; - private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; - private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; - private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; - - private final Method applicationTagsMethod; - private final Method attemptFailuresValidityIntervalMethod; - private final Method keepContainersMethod; - @Nullable - private final Method nodeLabelExpressionMethod; - - private ApplicationSubmissionContextReflector(Class clazz) { - Method applicationTagsMethod; - Method attemptFailuresValidityIntervalMethod; - Method keepContainersMethod; - Method nodeLabelExpressionMethod; - - try { - // this method is only supported by Hadoop 2.4.0 onwards - applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - applicationTagsMethod = null; - } - - this.applicationTagsMethod = applicationTagsMethod; - - try { - // this method is only supported by Hadoop 2.6.0 onwards - attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - attemptFailuresValidityIntervalMethod = null; - } - - this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; - - try { - // this method is only supported by Hadoop 2.4.0 onwards - keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - keepContainersMethod = null; - } - - this.keepContainersMethod = keepContainersMethod; - - try { - nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); - nodeLabelExpressionMethod = null; - } - - this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; - } - - public void setApplicationTags( - ApplicationSubmissionContext appContext, - Set applicationTags) throws InvocationTargetException, IllegalAccessException { - if (applicationTagsMethod != null) { - LOG.debug("Calling method {} of {}.", - applicationTagsMethod.getName(), - appContext.getClass().getCanonicalName()); - applicationTagsMethod.invoke(appContext, applicationTags); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - APPLICATION_TAGS_METHOD_NAME); - } - } - - public void setApplicationNodeLabel( - ApplicationSubmissionContext appContext, - String nodeLabel) throws InvocationTargetException, IllegalAccessException { - if (nodeLabelExpressionMethod != null) { - LOG.debug("Calling method {} of {}.", - nodeLabelExpressionMethod.getName(), - appContext.getClass().getCanonicalName()); - nodeLabelExpressionMethod.invoke(appContext, nodeLabel); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - NODE_LABEL_EXPRESSION_NAME); - } - } - - public void setAttemptFailuresValidityInterval( - ApplicationSubmissionContext appContext, - long validityInterval) throws InvocationTargetException, IllegalAccessException { - if (attemptFailuresValidityIntervalMethod != null) { - LOG.debug("Calling method {} of {}.", - attemptFailuresValidityIntervalMethod.getName(), - appContext.getClass().getCanonicalName()); - attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - ATTEMPT_FAILURES_METHOD_NAME); - } - } - - public void setKeepContainersAcrossApplicationAttempts( - ApplicationSubmissionContext appContext, - boolean keepContainers) throws InvocationTargetException, IllegalAccessException { - - if (keepContainersMethod != null) { - LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(), - appContext.getClass().getCanonicalName()); - keepContainersMethod.invoke(appContext, keepContainers); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - } - } - } - - private static class YarnDeploymentException extends RuntimeException { - private static final long serialVersionUID = -812040641215388943L; - - public YarnDeploymentException(String message) { - super(message); - } - - public YarnDeploymentException(String message, Throwable cause) { - super(message, cause); - } - } - - private class DeploymentFailureHook extends Thread { - - private final YarnClient yarnClient; - private final YarnClientApplication yarnApplication; - private final Path yarnFilesDir; - - DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication, Path yarnFilesDir) { - this.yarnClient = Preconditions.checkNotNull(yarnClient); - this.yarnApplication = Preconditions.checkNotNull(yarnApplication); - this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); - } - - @Override - public void run() { - LOG.info("Cancelling deployment from Deployment Failure Hook"); - failSessionDuringDeployment(yarnClient, yarnApplication); - LOG.info("Deleting files in {}.", yarnFilesDir); - try { - FileSystem fs = FileSystem.get(yarnConfiguration); - - if (!fs.delete(yarnFilesDir, true)) { - throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); - } - - fs.close(); - } catch (IOException e) { - LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); - } - } - } - - protected void addLibFolderToShipFiles(Collection effectiveShipFiles) { - // Add lib folder to the ship files if the environment variable is set. - // This is for convenience when running from the command-line. - // (for other files users explicitly set the ship files) - String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); - if (libDir != null) { - File libDirFile = new File(libDir); - if (libDirFile.isDirectory()) { - effectiveShipFiles.add(libDirFile); - } else { - throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + - "' is set to '" + libDir + "' but the directory doesn't exist."); - } - } else if (this.shipFiles.isEmpty()) { - LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + - "Not shipping any library files.", ENV_FLINK_LIB_DIR); - } - } - - protected ContainerLaunchContext setupApplicationMasterContainer( - String yarnClusterEntrypoint, - boolean hasLogback, - boolean hasLog4j, - boolean hasKrb5, - int jobManagerMemoryMb) { - // ------------------ Prepare Application Master Container ------------------------------ - - // respect custom JVM options in the YAML file - String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS); - if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) { - javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS); - } - //applicable only for YarnMiniCluster secure test run - //krb5.conf file will be available as local resource in JM/TM container - if (hasKrb5) { - javaOpts += " -Djava.security.krb5.conf=krb5.conf"; - } - - // Set up the container launch context for the application master - ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); - - final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "$JAVA_HOME/bin/java"); - - int heapSize = Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); - String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); - startCommandValues.put("jvmmem", jvmHeapMem); - - startCommandValues.put("jvmopts", javaOpts); - String logging = ""; - - if (hasLogback || hasLog4j) { - logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; - - if (hasLogback) { - logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; - } - - if (hasLog4j) { - logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; - } - } - - startCommandValues.put("logging", logging); - startCommandValues.put("class", yarnClusterEntrypoint); - startCommandValues.put("redirects", - "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + - "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"); - startCommandValues.put("args", ""); - - final String commandTemplate = flinkConfiguration - .getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, - ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE); - final String amCommand = - BootstrapTools.getStartCommand(commandTemplate, startCommandValues); - - amContainer.setCommands(Collections.singletonList(amCommand)); - - LOG.debug("Application Master start command: " + amCommand); - - return amContainer; - } - - private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(Configuration config) { - throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(config); - - return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); - } - - private static void throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(final Configuration config) { - final String userJarInclusion = config.getString(YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); - if ("DISABLED".equalsIgnoreCase(userJarInclusion)) { - throw new IllegalArgumentException(String.format("Config option %s cannot be set to DISABLED anymore (see FLINK-11781)", - YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key())); - } - } - - /** - * Creates a YarnClusterClient; may be overridden in tests. - */ - protected abstract ClusterClient createYarnClusterClient( - AbstractYarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, - ApplicationReport report, - Configuration flinkConfiguration, - boolean perJobCluster) throws Exception; -} - diff --git a/pom.xml b/pom.xml index 78e34c817b..32d4f8c4bc 100644 --- a/pom.xml +++ b/pom.xml @@ -60,12 +60,11 @@ UTF-8 - 2.10 - 1.8.1 + 1.10.0 2.7.3 4.5.3 ${basedir}/dev - release_1.8.5 + release_1.10.0 @@ -95,14 +94,14 @@ org.apache.hadoop hadoop-yarn-common ${hadoop.version} - + provided org.apache.hadoop hadoop-yarn-client ${hadoop.version} - + provided org.apache.hadoop @@ -114,7 +113,7 @@ org.apache.hadoop hadoop-yarn-api ${hadoop.version} - + provided org.apache.avro @@ -122,11 +121,6 @@ 1.8.2 provided - - commons-lang - commons-lang - 2.6 - org.testng @@ -169,7 +163,6 @@ - \ No newline at end of file From 2b229f3eb45044ad46c0fc39c17d07ed1b573d78 Mon Sep 17 00:00:00 2001 From: tudou Date: Sun, 26 Apr 2020 19:23:07 +0800 Subject: [PATCH 043/136] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E5=A2=9E=E9=87=8F?= =?UTF-8?q?=E4=BB=BB=E5=8A=A1Accumulator?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index a01102ea94..8578075390 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -322,7 +322,7 @@ protected void initMetric(InputSplit split) { startLocationAccumulator.add(Long.parseLong(incrementConfig.getStartLocation())); } customPrometheusReporter.registerMetric(startLocationAccumulator, Metrics.START_LOCATION); - + getRuntimeContext().addAccumulator(Metrics.START_LOCATION, startLocationAccumulator); endLocationAccumulator = new LongMaximum(); String endLocation = ((JdbcInputSplit) split).getEndLocation(); if (endLocation != null && incrementConfig.isUseMaxFunc()) { @@ -331,6 +331,7 @@ protected void initMetric(InputSplit split) { endLocationAccumulator.add(Long.parseLong(incrementConfig.getStartLocation())); } customPrometheusReporter.registerMetric(endLocationAccumulator, Metrics.END_LOCATION); + getRuntimeContext().addAccumulator(Metrics.END_LOCATION, endLocationAccumulator); } /** From 1a156ba8528bd19dea2f1ea40b8f97afca9f0083 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 27 Apr 2020 21:42:04 +0800 Subject: [PATCH 044/136] =?UTF-8?q?flinkx1.10=E4=BB=BB=E5=8A=A1=E6=8F=90?= =?UTF-8?q?=E4=BA=A4bug=E4=BF=AE=E5=A4=8D=E5=8F=8A=E4=BB=A3=E7=A0=81?= =?UTF-8?q?=E4=BC=98=E5=8C=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitignore | 3 +- flinkx-binlog/flinkx-binlog-reader/pom.xml | 6 +- .../flinkx-carbondata-reader/pom.xml | 6 +- .../flinkx-carbondata-writer/pom.xml | 6 +- .../flinkx-cassandra-reader/pom.xml | 6 +- .../flinkx-cassandra-writer/pom.xml | 6 +- .../flinkx-clickhouse-reader/pom.xml | 6 +- .../flinkx-clickhouse-writer/pom.xml | 6 +- flinkx-core/pom.xml | 27 +- .../main/java/com/dtstack/flinkx/Main.java | 26 +- .../flinkx/constants/ConfigConstant.java | 18 + .../flinkx/constants/ConstantValue.java | 6 + .../dtstack/flinkx/metrics/BaseMetric.java | 2 +- .../metrics/CustomPrometheusReporter.java | 1 - .../com/dtstack/flinkx/options/Options.java | 12 +- flinkx-db2/flinkx-db2-reader/pom.xml | 6 +- flinkx-db2/flinkx-db2-writer/pom.xml | 6 +- flinkx-dm/flinkx-dm-reader/pom.xml | 6 +- flinkx-dm/flinkx-dm-writer/pom.xml | 6 +- flinkx-emqx/flinkx-emqx-reader/pom.xml | 6 +- flinkx-emqx/flinkx-emqx-writer/pom.xml | 6 +- flinkx-es/flinkx-es-reader/pom.xml | 6 +- flinkx-es/flinkx-es-writer/pom.xml | 6 +- flinkx-ftp/flinkx-ftp-reader/pom.xml | 6 +- flinkx-ftp/flinkx-ftp-writer/pom.xml | 6 +- flinkx-gbase/flinkx-gbase-reader/pom.xml | 6 +- flinkx-gbase/flinkx-gbase-writer/pom.xml | 6 +- flinkx-hbase/flinkx-hbase-reader/pom.xml | 6 +- flinkx-hbase/flinkx-hbase-writer/pom.xml | 6 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 6 +- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 6 +- flinkx-hive/flinkx-hive-writer/pom.xml | 6 +- flinkx-kafka/flinkx-kafka-reader/pom.xml | 6 +- flinkx-kafka/flinkx-kafka-writer/pom.xml | 6 +- flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 6 +- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 6 +- flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 6 +- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 6 +- flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 6 +- flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 6 +- flinkx-kudu/flinkx-kudu-reader/pom.xml | 6 +- flinkx-kudu/flinkx-kudu-writer/pom.xml | 6 +- flinkx-launcher/pom.xml | 36 +- .../flinkx/launcher/ClassLoaderType.java | 21 - .../flinkx/launcher/ClusterClientFactory.java | 60 +- .../com/dtstack/flinkx/launcher/Launcher.java | 13 +- .../dtstack/flinkx/launcher/PluginUtil.java | 54 - .../perJob/FlinkPerJobResourceUtil.java | 71 - .../launcher/perJob/FlinkPerJobUtil.java | 148 ++ .../perJob/PerJobClusterClientBuilder.java | 2 +- .../launcher/perJob/PerJobSubmitter.java | 36 +- .../deployment/ClusterSpecification.java | 249 +++ .../flink/yarn/YarnClusterDescriptor.java | 1614 +++++++++++++++++ .../flinkx-mongodb-oplog-reader/pom.xml | 6 +- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 6 +- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 6 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 6 +- flinkx-mysql/flinkx-mysql-reader/pom.xml | 6 +- flinkx-mysql/flinkx-mysql-writer/pom.xml | 6 +- flinkx-odps/flinkx-odps-reader/pom.xml | 6 +- flinkx-odps/flinkx-odps-writer/pom.xml | 6 +- flinkx-oracle/flinkx-oracle-reader/pom.xml | 6 +- flinkx-oracle/flinkx-oracle-writer/pom.xml | 6 +- flinkx-pgwal/flinkx-pgwal-reader/pom.xml | 6 +- flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 6 +- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 6 +- flinkx-polardb/flinkx-polardb-dreader/pom.xml | 6 +- flinkx-polardb/flinkx-polardb-reader/pom.xml | 6 +- flinkx-polardb/flinkx-polardb-writer/pom.xml | 6 +- .../flinkx-postgresql-reader/pom.xml | 6 +- .../flinkx-postgresql-writer/pom.xml | 6 +- flinkx-pulsar/flinkx-pulsar-writer/pom.xml | 6 +- flinkx-rdb/flinkx-rdb-core/pom.xml | 6 +- flinkx-rdb/flinkx-rdb-reader/pom.xml | 6 +- flinkx-rdb/flinkx-rdb-writer/pom.xml | 6 +- flinkx-redis/flinkx-redis-writer/pom.xml | 6 +- flinkx-saphana/flinkx-saphana-reader/pom.xml | 6 +- flinkx-saphana/flinkx-saphana-writer/pom.xml | 6 +- .../flinkx-sqlserver-reader/pom.xml | 6 +- .../flinkx-sqlserver-writer/pom.xml | 6 +- flinkx-stream/flinkx-stream-reader/pom.xml | 6 +- flinkx-stream/flinkx-stream-writer/pom.xml | 6 +- .../flinkx-teradata-reader/pom.xml | 8 +- .../flinkx-teradata-writer/pom.xml | 8 +- 84 files changed, 2340 insertions(+), 453 deletions(-) delete mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java delete mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java delete mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java create mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java create mode 100644 flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java diff --git a/.gitignore b/.gitignore index d8cb487ce5..61e703a5e4 100644 --- a/.gitignore +++ b/.gitignore @@ -8,4 +8,5 @@ jobs/ nohup.out flinkconf/ hadoopconf/ -/default_task_id_output \ No newline at end of file +/default_task_id_output +/syncplugins/ diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index ad9b9893df..72a35a83da 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -121,13 +121,13 @@ - + - + diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index 111fc621c3..9c786f536b 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -83,13 +83,13 @@ - + - + diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index 9b414eafd3..0e97624135 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index feca24f0f9..788c4b011c 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -78,13 +78,13 @@ - + - + diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index 565ef3ca4f..bcef303f1a 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -78,13 +78,13 @@ - + - + diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 01d798e35d..4422b401f0 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -73,13 +73,13 @@ - + - + diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 94e72635a6..6a187c7a91 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -73,13 +73,13 @@ - + - + diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index f859484e6f..d3751c14a8 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -27,7 +27,7 @@ org.slf4j slf4j-api - 1.7.20 + 1.7.21 @@ -86,12 +86,6 @@ - - commons-cli - commons-cli - 1.2 - - org.apache.flink flink-yarn_2.11 @@ -162,6 +156,11 @@ simpleclient 0.5.0 + + org.apache.avro + avro + 1.8.2 + @@ -200,14 +199,12 @@ org.slf4j:* org.apache.httpcomponents:* io.prometheus:* + org.apache.avro:* + org.codehaus.jackson:* commons-*:* - - com.google.common - shade.core.com.google.common - org.apache.http shade.core.org.apache.http @@ -235,14 +232,14 @@ - - + + - + diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 95fab3010b..952c7c74b9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -111,14 +112,19 @@ public static void main(String[] args) throws Exception { env.setRestartStrategy(RestartStrategies.noRestart()); BaseDataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); - dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); + if(speedConfig.getReaderChannel() > 0){ + dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); + } if (speedConfig.isRebalance()) { dataStream = dataStream.rebalance(); } BaseDataWriter dataWriter = DataWriterFactory.getDataWriter(config); - dataWriter.writeData(dataStream).setParallelism(speedConfig.getWriterChannel()); + DataStreamSink dataStreamSink = dataWriter.writeData(dataStream); + if(speedConfig.getWriterChannel() > 0){ + dataStreamSink.setParallelism(speedConfig.getWriterChannel()); + } if(env instanceof MyLocalStreamEnvironment) { if(StringUtils.isNotEmpty(savepointPath)){ @@ -201,22 +207,6 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla if(env instanceof MyLocalStreamEnvironment){ ((MyLocalStreamEnvironment) env).setClasspaths(new ArrayList<>(classPathSet)); } - /* else if(env instanceof StreamContextEnvironment){ - Field field = env.getClass().getDeclaredField("ctx"); - field.setAccessible(true); - ContextEnvironment contextEnvironment= (ContextEnvironment) field.get(env); - - List originUrlList = new ArrayList<>(); - for (URL url : contextEnvironment.getClasspaths()) { - originUrlList.add(url.toString()); - } - - for (URL url : classPathSet) { - if (!originUrlList.contains(url.toString())){ - contextEnvironment.getClasspaths().add(url); - } - } - }*/ } private static Properties parseConf(String confStr) throws Exception{ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java index 48a3a785e4..baaf65f067 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java @@ -21,6 +21,11 @@ package com.dtstack.flinkx.constants; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import static com.dtstack.flinkx.constants.ConstantValue.CLASS_PATH_PLUGIN_LOAD_MODE; + /** * * @author sishu.yss @@ -31,4 +36,17 @@ public class ConfigConstant { public static final String FLINK_CHECKPOINT_INTERVAL_KEY = "flink.checkpoint.interval"; public static final String FLINK_CHECKPOINT_TIMEOUT_KEY = "flink.checkpoint.timeout"; + + public static final String YARN_RESOURCE_MANAGER_WEBAPP_ADDRESS_KEY = "yarn.resourcemanager.webapp.address"; + + public static final ConfigOption FLINK_PLUGIN_LOAD_MODE_KEY = ConfigOptions + .key("pluginLoadMode") + .stringType() + .defaultValue(CLASS_PATH_PLUGIN_LOAD_MODE) + .withDescription("The config parameter defining YarnPer mode plugin loading method." + + "classpath: The plugin package is not uploaded when the task is submitted. " + + "The plugin package needs to be deployed in the pluginRoot directory of the yarn-node node, but the task starts faster" + + "shipfile: When submitting a task, upload the plugin package under the pluginRoot directory to deploy the plug-in package. " + + "The yarn-node node does not need to deploy the plugin package. " + + "The task startup speed depends on the size of the plugin package and the network environment."); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index 46de3749fb..7cc1e791f1 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -51,4 +51,10 @@ public class ConstantValue { public static final String SYSTEM_PROPERTIES_KEY_FILE_ENCODING = "file.encoding"; public static final String OS_WINDOWS = "windows"; + + public static final String SHIP_FILE_PLUGIN_LOAD_MODE = "shipfile"; + public static final String CLASS_PATH_PLUGIN_LOAD_MODE = "classpath"; + + public static final String CLASSLOADER_CHILD_FIRST = "child-first"; + public static final String CLASSLOADER_PARENT_FIRST = "parent-first"; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java index bf52dbe64a..f7019f3996 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java @@ -54,7 +54,7 @@ public void addMetric(String metricName, LongCounter counter){ public void addMetric(String metricName, LongCounter counter, boolean meterView){ metricCounters.put(metricName, counter); - flinkxOutput.gauge(metricName, new SimpleAccumulatorGauge(counter)); + flinkxOutput.gauge(metricName, new SimpleAccumulatorGauge<>(counter)); if (meterView){ flinkxOutput.meter(metricName + Metrics.SUFFIX_RATE, new SimpleLongCounterMeterView(counter, 60)); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java index ae03b6d4af..d460469bc2 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java @@ -122,7 +122,6 @@ public void open() { if (StringUtils.isNullOrWhitespaceOnly(host) || port < 1) { return; -// throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); } if (randomSuffix) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 92c8a4db27..b2304e3264 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -26,6 +26,9 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.yarn.configuration.YarnConfigOptions; +import static com.dtstack.flinkx.constants.ConfigConstant.FLINK_PLUGIN_LOAD_MODE_KEY; +import static com.dtstack.flinkx.constants.ConstantValue.*; + /** * This class define commandline options for the Launcher program * @@ -74,7 +77,7 @@ public class Options { private String s; @OptionRequired(description = "plugin load mode, by classpath or shipfile") - private String pluginLoadMode = "shipfile"; + private String pluginLoadMode = SHIP_FILE_PLUGIN_LOAD_MODE; @OptionRequired(description = "applicationId on yarn cluster") private String appId; @@ -93,11 +96,12 @@ public Configuration loadFlinkConfiguration() { if(StringUtils.isNotBlank(yarnconf)){ flinkConfiguration.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnconf); } - if("classpath".equalsIgnoreCase(pluginLoadMode)){ - flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "child-first"); + if(CLASS_PATH_PLUGIN_LOAD_MODE.equalsIgnoreCase(pluginLoadMode)){ + flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, CLASSLOADER_CHILD_FIRST); }else{ - flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first"); + flinkConfiguration.setString(CoreOptions.CLASSLOADER_RESOLVE_ORDER, CLASSLOADER_PARENT_FIRST); } + flinkConfiguration.setString(FLINK_PLUGIN_LOAD_MODE_KEY, pluginLoadMode); } return flinkConfiguration; } diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 6a6f40a2c7..2ed4f7d593 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 2ff75d22e0..524d018ed7 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-dm/flinkx-dm-reader/pom.xml b/flinkx-dm/flinkx-dm-reader/pom.xml index 15ce92929b..9d0f47efc8 100644 --- a/flinkx-dm/flinkx-dm-reader/pom.xml +++ b/flinkx-dm/flinkx-dm-reader/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-dm/flinkx-dm-writer/pom.xml b/flinkx-dm/flinkx-dm-writer/pom.xml index abe284fc04..46e845ca00 100644 --- a/flinkx-dm/flinkx-dm-writer/pom.xml +++ b/flinkx-dm/flinkx-dm-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-emqx/flinkx-emqx-reader/pom.xml b/flinkx-emqx/flinkx-emqx-reader/pom.xml index 0e106143fd..d5a0c1798a 100644 --- a/flinkx-emqx/flinkx-emqx-reader/pom.xml +++ b/flinkx-emqx/flinkx-emqx-reader/pom.xml @@ -71,13 +71,13 @@ - + - + diff --git a/flinkx-emqx/flinkx-emqx-writer/pom.xml b/flinkx-emqx/flinkx-emqx-writer/pom.xml index a6117433f5..24546c00f8 100644 --- a/flinkx-emqx/flinkx-emqx-writer/pom.xml +++ b/flinkx-emqx/flinkx-emqx-writer/pom.xml @@ -71,13 +71,13 @@ - + - + diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 227dc25d10..ce677cb829 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -71,13 +71,13 @@ - + - + diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 1cdfab73fb..d14efc2af0 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -72,13 +72,13 @@ - + - + diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 1771055c60..262594556b 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -89,13 +89,13 @@ under the License. - + - + diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 61d8ffaede..08142241aa 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -90,13 +90,13 @@ under the License. - + - + diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 3082049696..020ac16812 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index da6f21b4a2..8fdbdfce97 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index f77a9613ca..567bdd5f84 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -83,13 +83,13 @@ - + - + diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index d406061b54..77e2a7a861 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -72,13 +72,13 @@ - + - + diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index 1a64c0b3ab..aa1260a385 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -113,13 +113,13 @@ under the License. - + - + diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 89f059b99d..e18ead798f 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -116,13 +116,13 @@ under the License. - + - + diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 25357e479f..26495120be 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -110,13 +110,13 @@ under the License. - + - + diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index d6cd16c26f..5aa7766c17 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -51,13 +51,13 @@ - + - + diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index b62d13da11..8c9c1cf14e 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -50,13 +50,13 @@ - + - + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index 980db83475..61bf52fd84 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -50,13 +50,13 @@ - + - + diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index cea09d8368..1daf62a5f5 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -50,13 +50,13 @@ - + - + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 8fbbdb883d..5a621d790a 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -50,13 +50,13 @@ - + - + diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index f28e37fe50..a889295eac 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -56,13 +56,13 @@ - + - + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index 4fa02eb7e5..5007476af9 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -50,13 +50,13 @@ - + - + diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index fe3808dd63..015af46ceb 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -51,13 +51,13 @@ - + - + diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index c8a99fea22..08f20733ff 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -71,13 +71,13 @@ - + - + diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index 742fae63ab..ed004a5682 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -71,13 +71,13 @@ - + - + diff --git a/flinkx-launcher/pom.xml b/flinkx-launcher/pom.xml index fed35a10a9..8ac6150544 100644 --- a/flinkx-launcher/pom.xml +++ b/flinkx-launcher/pom.xml @@ -12,12 +12,6 @@ flinkx-launcher - - commons-cli - commons-cli - 1.2 - - com.dtstack.flinkx flinkx-core @@ -52,24 +46,48 @@ org.apache.hadoop hadoop-hdfs ${hadoop.version} + + + commons-cli + commons-cli + + org.apache.hadoop hadoop-common ${hadoop.version} + + + commons-cli + commons-cli + + org.apache.hadoop hadoop-yarn-common ${hadoop.version} + + + commons-cli + commons-cli + + org.apache.hadoop hadoop-yarn-client ${hadoop.version} + + + commons-cli + commons-cli + + org.apache.hadoop @@ -86,6 +104,12 @@ avro 1.8.2 + + junit + junit + 4.12 + test + diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java deleted file mode 100644 index 540db38056..0000000000 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.dtstack.flinkx.launcher; - -/** - * company: www.dtstack.com - * author: toutian - * create: 2019/10/17 - */ -public enum ClassLoaderType { - NONE, CHILD_FIRST, PARENT_FIRST, CHILD_FIRST_CACHE, PARENT_FIRST_CACHE; - - public static ClassLoaderType getByClassMode(String classMode) { - ClassLoaderType classLoaderType; - if ("classpath".equalsIgnoreCase(classMode)) { - classLoaderType = ClassLoaderType.CHILD_FIRST; - } else { - classLoaderType = ClassLoaderType.PARENT_FIRST; - } - - return classLoaderType; - } -} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index 960b7be4d0..6375a284db 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; @@ -38,9 +37,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.ConverterUtils; -import java.io.File; -import java.net.URL; -import java.util.*; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.List; +import java.util.Set; /** * The Factory of ClusterClient @@ -66,12 +66,6 @@ public static ClusterClient createStandaloneClient(Options launcherOptions) thro StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(flinkConf); ClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()).getClusterClient(); - -// LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); -// InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); -// config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); -// config.setInteger(JobManagerOptions.PORT, address.getPort()); -// clusterClient.setDetached(true); return clusterClient; } @@ -89,7 +83,7 @@ public static ClusterClient createYarnClient(Options launcherOptions) { ApplicationId applicationId; if (StringUtils.isEmpty(launcherOptions.getAppId())) { - applicationId = getAppIdFromYarn(yarnClient); + applicationId = getAppIdFromYarn(yarnClient, launcherOptions); if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } @@ -101,7 +95,6 @@ public static ClusterClient createYarnClient(Options launcherOptions) { if(highAvailabilityMode.equals(HighAvailabilityMode.ZOOKEEPER) && applicationId!=null){ flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationId.toString()); } -// YarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(launcherOptions, yarnClient, yarnConf, flinkConfig); YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( flinkConfig, yarnConf, @@ -117,44 +110,7 @@ public static ClusterClient createYarnClient(Options launcherOptions) { throw new UnsupportedOperationException("Haven't been developed yet!"); } - private static YarnClusterDescriptor getClusterDescriptor(Options launcherOptions, - YarnClient yarnClient, - YarnConfiguration yarnConf, - Configuration flinkConfig) throws Exception{ - YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( - flinkConfig, - yarnConf, - yarnClient, - YarnClientYarnClusterInformationRetriever.create(yarnClient), - true); - - // plugin dependent on shipfile - if (StringUtils.isNotEmpty(launcherOptions.getPluginLoadMode()) && "shipfile".equalsIgnoreCase(launcherOptions.getPluginLoadMode())) { - List pluginPaths = PluginUtil.getAllPluginPath(launcherOptions.getPluginRoot()); - if (!pluginPaths.isEmpty()) { - yarnClusterDescriptor.addShipFiles(pluginPaths); - } - } - - String flinkJarPath = launcherOptions.getFlinkLibJar(); - if (StringUtils.isNotEmpty(flinkJarPath)) { - List classpaths = new ArrayList<>(); - File[] jars = new File(flinkJarPath).listFiles(); - for (File file : jars) { - if (file.toURI().toURL().toString().contains("flink-dist")) { - yarnClusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); - } else { - classpaths.add(file.toURI().toURL()); - } - } - -// yarnClusterDescriptor.setProvidedUserJarFiles(classpaths); - } - - return yarnClusterDescriptor; - } - - private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exception{ + private static ApplicationId getAppIdFromYarn(YarnClient yarnClient, Options launcherOptions) throws Exception{ Set set = new HashSet<>(); set.add("Apache Flink"); EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); @@ -173,6 +129,10 @@ private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exce continue; } + if(!report.getQueue().equals(launcherOptions.getQueue())) { + continue; + } + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 06f83dbb14..70dfdb32a3 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -26,7 +26,7 @@ import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.options.Options; import com.dtstack.flinkx.util.SysUtil; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; @@ -62,6 +62,7 @@ public class Launcher { public static final String PLUGINS_DIR_NAME = "plugins"; public static final String CORE_JAR_NAME_PREFIX = "flinkx"; + public static final String MAIN_CLASS = "com.dtstack.flinkx.Main"; public static void main(String[] args) throws Exception { setLogLevel(Level.DEBUG.toString()); @@ -91,8 +92,8 @@ public static void main(String[] args) throws Exception { throw new IllegalArgumentException("per-job mode must have flink lib path!"); } argList.add("-monitor"); - argList.add("application_default"); - PerJobSubmitter.submit(launcherOptions, buildJobGraph(launcherOptions, argList.toArray(new String[0]))); + argList.add(""); + PerJobSubmitter.submit(launcherOptions, new JobGraph(), argList.toArray(new String[0])); } } @@ -109,7 +110,7 @@ private static JobGraph buildJobGraph(Options launcherOptions, String[] remoteAr PackagedProgram program = PackagedProgram.newBuilder() .setJarFile(jarFile) .setUserClassPaths(urlList) - .setEntryPointClassName("com.dtstack.flinkx.Main") + .setEntryPointClassName(MAIN_CLASS) .setConfiguration(launcherOptions.loadFlinkConfiguration()) .setSavepointRestoreSettings(savepointRestoreSettings) .setArguments(remoteArgs) @@ -117,7 +118,7 @@ private static JobGraph buildJobGraph(Options launcherOptions, String[] remoteAr return PackagedProgramUtils.createJobGraph(program, launcherOptions.loadFlinkConfiguration(), Integer.parseInt(launcherOptions.getParallelism()), false); } - private static List analyzeUserClasspath(String content, String pluginRoot) { + public static List analyzeUserClasspath(String content, String pluginRoot) { List urlList = new ArrayList<>(); String jobJson = readJob(content); @@ -214,7 +215,7 @@ private static String getSystemProperty(String name) { return property; } - private static String getCoreJarFileName(String pluginRoot) throws FileNotFoundException { + public static String getCoreJarFileName(String pluginRoot) throws FileNotFoundException { String coreJarFileName = null; File pluginDir = new File(pluginRoot); if (pluginDir.exists() && pluginDir.isDirectory()) { diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java deleted file mode 100644 index cbba6cd27e..0000000000 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/PluginUtil.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.launcher; - -import com.google.common.collect.Lists; -import org.apache.commons.lang3.StringUtils; - -import java.io.File; -import java.util.Arrays; -import java.util.List; - -/** - * @author jiangbo - * @date 2020/4/15 - */ -public class PluginUtil { - - public static List getAllPluginPath(String flinkPluginRoot) { - List pluginPaths = Lists.newArrayList(); - if (StringUtils.isNotBlank(flinkPluginRoot)) { - try { - File[] jars = new File(flinkPluginRoot).listFiles(); - if (jars != null) { - pluginPaths.addAll(Arrays.asList(jars)); - } - } catch (Exception e) { - throw new RuntimeException("Get jars from pluginRoot error", e); - } - } - - if (pluginPaths.size() == 0) { - throw new RuntimeException("jars in pluginRoot is null, flinkPluginRoot = " + flinkPluginRoot); - } - - return pluginPaths; - } -} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java deleted file mode 100644 index 50401dc3e8..0000000000 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobResourceUtil.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.launcher.perJob; - -import com.dtstack.flinkx.util.ValueUtil; -import org.apache.flink.client.deployment.ClusterSpecification; - -import java.util.Properties; - -/** - * Date: 2019/09/11 - * Company: www.dtstack.com - * @author tudou - */ -public class FlinkPerJobResourceUtil { - /** - * Minimum memory requirements, checked by the Client. - * the minimum memory should be higher than the min heap cutoff - */ - public final static int MIN_JM_MEMORY = 768; - public final static int MIN_TM_MEMORY = 768; - - public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; - public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; -// public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; - public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; - - /** - * the specification of this per-job mode cost - * @param conProp taskParams - * @return - */ - public static ClusterSpecification createClusterSpecification(Properties conProp){ - int jobmanagerMemoryMb = 768; - int taskmanagerMemoryMb = 768; - int slotsPerTaskManager = 1; - - if(conProp != null){ - if(conProp.contains(JOBMANAGER_MEMORY_MB)){ - jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); - } - if(conProp.contains(TASKMANAGER_MEMORY_MB)){ - taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); - } - if (conProp.containsKey(SLOTS_PER_TASKMANAGER)){ - slotsPerTaskManager = ValueUtil.getInt(conProp.get(SLOTS_PER_TASKMANAGER)); - } - } - - return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobmanagerMemoryMb) - .setTaskManagerMemoryMB(taskmanagerMemoryMb) - .setSlotsPerTaskManager(slotsPerTaskManager) - .createClusterSpecification(); - } -} \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java new file mode 100644 index 0000000000..ab7c392b2c --- /dev/null +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.launcher.perJob; + +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.ValueUtil; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Field; +import java.util.Properties; + +import static com.dtstack.flinkx.constants.ConfigConstant.YARN_RESOURCE_MANAGER_WEBAPP_ADDRESS_KEY; + +/** + * Date: 2019/09/11 + * Company: www.dtstack.com + * + * @author tudou + */ +public class FlinkPerJobUtil { + private static final Logger LOG = LoggerFactory.getLogger(FlinkPerJobUtil.class); + /** + * Minimum memory requirements, checked by the Client. + * the minimum memory should be higher than the min heap cutoff + */ + public final static int MIN_JM_MEMORY = 768; + public final static int MIN_TM_MEMORY = 768; + + public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; + public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; + public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; + + /** + * the specification of this per-job mode cost + * + * @param conProp taskParams + * @return + */ + public static ClusterSpecification createClusterSpecification(Properties conProp) { + int jobmanagerMemoryMb = 768; + int taskmanagerMemoryMb = 768; + int slotsPerTaskManager = 1; + + if (conProp != null) { + if (conProp.contains(JOBMANAGER_MEMORY_MB)) { + jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); + } + if (conProp.contains(TASKMANAGER_MEMORY_MB)) { + taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); + } + if (conProp.containsKey(SLOTS_PER_TASKMANAGER)) { + slotsPerTaskManager = ValueUtil.getInt(conProp.get(SLOTS_PER_TASKMANAGER)); + } + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobmanagerMemoryMb) + .setTaskManagerMemoryMB(taskmanagerMemoryMb) + .setSlotsPerTaskManager(slotsPerTaskManager) + .createClusterSpecification(); + } + + public static String getUrlFormat(YarnConfiguration yarnConf, YarnClient yarnClient) { + String url = ""; + try { + Field rmClientField = yarnClient.getClass().getDeclaredField("rmClient"); + rmClientField.setAccessible(true); + Object rmClient = rmClientField.get(yarnClient); + + Field hField = rmClient.getClass().getSuperclass().getDeclaredField("h"); + hField.setAccessible(true); + //获取指定对象中此字段的值 + Object h = hField.get(rmClient); + Object currentProxy = null; + + try { + Field currentProxyField = h.getClass().getDeclaredField("currentProxy"); + currentProxyField.setAccessible(true); + currentProxy = currentProxyField.get(h); + } catch (Exception e) { + //兼容Hadoop 2.7.3.2.6.4.91-3 + LOG.warn("get currentProxy error:{}", ExceptionUtil.getErrorMessage(e)); + Field proxyDescriptorField = h.getClass().getDeclaredField("proxyDescriptor"); + proxyDescriptorField.setAccessible(true); + Object proxyDescriptor = proxyDescriptorField.get(h); + Field currentProxyField = proxyDescriptor.getClass().getDeclaredField("proxyInfo"); + currentProxyField.setAccessible(true); + currentProxy = currentProxyField.get(proxyDescriptor); + } + + Field proxyInfoField = currentProxy.getClass().getDeclaredField("proxyInfo"); + proxyInfoField.setAccessible(true); + String proxyInfoKey = (String) proxyInfoField.get(currentProxy); + + String key = YARN_RESOURCE_MANAGER_WEBAPP_ADDRESS_KEY + "." + proxyInfoKey; + String addr = yarnConf.get(key); + + if (addr == null) { + addr = yarnConf.get(YARN_RESOURCE_MANAGER_WEBAPP_ADDRESS_KEY); + } + + return String.format("http://%s/proxy", addr); + } catch (Exception e) { + LOG.warn("get monitor error:{}", ExceptionUtil.getErrorMessage(e)); + } + + return url; + } + + public static PackagedProgram buildProgram(String monitorUrl, ClusterSpecification clusterSpecification) throws Exception { + String[] args = clusterSpecification.getProgramArgs(); + for (int i = 0; i < args.length; i++) { + if ("-monitor".equals(args[i])) { + args[i + 1] = monitorUrl; + break; + } + } + + return PackagedProgram.newBuilder() + .setJarFile(clusterSpecification.getJarFile()) + .setUserClassPaths(clusterSpecification.getClasspaths()) + .setEntryPointClassName(clusterSpecification.getEntryPointClass()) + .setConfiguration(clusterSpecification.getConfiguration()) + .setSavepointRestoreSettings(clusterSpecification.getSpSetting()) + .setArguments(args) + .build(); + } +} \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index 23078e97f9..c6a8538b70 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -19,7 +19,7 @@ import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.security.SecurityConfiguration; diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java index a0daae8cd6..a401695ea0 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java @@ -18,18 +18,24 @@ package com.dtstack.flinkx.launcher.perJob; +import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; import com.dtstack.flinkx.util.MapUtil; +import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClientProvider; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.util.Properties; +import static com.dtstack.flinkx.launcher.Launcher.*; + /** * Date: 2019/09/11 * Company: www.dtstack.com @@ -40,19 +46,35 @@ public class PerJobSubmitter { /** * submit per-job task - * @param options LauncherOptions + * @param launcherOptions LauncherOptions * @param jobGraph JobGraph + * @param remoteArgs remoteArgs * @return * @throws Exception */ - public static String submit(Options options, JobGraph jobGraph) throws Exception{ - LOG.info("start to submit per-job task, options = {}", options.toString()); - Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); - ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); + public static String submit(Options launcherOptions, JobGraph jobGraph, String[] remoteArgs) throws Exception{ + LOG.info("start to submit per-job task, launcherOptions = {}", launcherOptions.toString()); + Properties conProp = MapUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FlinkPerJobUtil.createClusterSpecification(conProp); + clusterSpecification.setCreateProgramDelay(true); + + String pluginRoot = launcherOptions.getPluginRoot(); + File jarFile = new File(pluginRoot + File.separator + getCoreJarFileName(pluginRoot)); + clusterSpecification.setConfiguration(launcherOptions.loadFlinkConfiguration()); + clusterSpecification.setClasspaths(analyzeUserClasspath(launcherOptions.getJob(), pluginRoot)); + clusterSpecification.setEntryPointClass(MAIN_CLASS); + clusterSpecification.setJarFile(jarFile); + + if (StringUtils.isNotEmpty(launcherOptions.getS())) { + clusterSpecification.setSpSetting(SavepointRestoreSettings.forPath(launcherOptions.getS())); + } + clusterSpecification.setProgramArgs(remoteArgs); + clusterSpecification.setCreateProgramDelay(true); + clusterSpecification.setYarnConfiguration(YarnConfLoader.getYarnConf(launcherOptions.getYarnconf())); PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); - perJobClusterClientBuilder.init(options, conProp); + perJobClusterClientBuilder.init(launcherOptions, conProp); - YarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options, jobGraph); + YarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, launcherOptions, jobGraph); ClusterClientProvider provider = descriptor.deployJobCluster(clusterSpecification, jobGraph, true); String applicationId = provider.getClusterClient().getClusterId().toString(); String flinkJobId = jobGraph.getJobID().toString(); diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java new file mode 100644 index 0000000000..25656e2219 --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.deployment; + +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.net.URL; +import java.util.List; + +/** + * Description of the cluster to start by the {@link ClusterDescriptor}. + */ +public final class ClusterSpecification { + private final int masterMemoryMB; + private final int taskManagerMemoryMB; + private final int numberTaskManagers; + private final int slotsPerTaskManager; + private final int priority; + + private int parallelism; + private Configuration configuration; + private YarnConfiguration yarnConfiguration; + private JobGraph jobGraph; + private SavepointRestoreSettings spSetting = SavepointRestoreSettings.none(); + private List classpaths; + private String entryPointClass; + private String[] programArgs; + private File jarFile; + private boolean createProgramDelay = false; + private PackagedProgram program; + + private ClusterSpecification(int masterMemoryMB, int taskManagerMemoryMB, int numberTaskManagers, int slotsPerTaskManager, int parallelism, int priority) { + this.masterMemoryMB = masterMemoryMB; + this.taskManagerMemoryMB = taskManagerMemoryMB; + this.numberTaskManagers = numberTaskManagers; + this.slotsPerTaskManager = slotsPerTaskManager; + this.parallelism = parallelism; + this.priority = priority; + } + + public PackagedProgram getProgram() { + return program; + } + + public void setProgram(PackagedProgram program) { + this.program = program; + } + + public YarnConfiguration getYarnConfiguration() { + return yarnConfiguration; + } + + public void setYarnConfiguration(YarnConfiguration yarnConfiguration) { + this.yarnConfiguration = yarnConfiguration; + } + + public JobGraph getJobGraph() { + return jobGraph; + } + + public void setJobGraph(JobGraph jobGraph) { + this.jobGraph = jobGraph; + } + + public int getParallelism() { + return parallelism; + } + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + public Configuration getConfiguration() { + return configuration; + } + + public void setConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + public int getMasterMemoryMB() { + return masterMemoryMB; + } + + public int getTaskManagerMemoryMB() { + return taskManagerMemoryMB; + } + + public int getNumberTaskManagers() { + return numberTaskManagers; + } + + public int getSlotsPerTaskManager() { + return slotsPerTaskManager; + } + + public int getPriority(){ + return priority; + } + + public SavepointRestoreSettings getSpSetting() { + return spSetting; + } + + public void setSpSetting(SavepointRestoreSettings spSetting) { + this.spSetting = spSetting; + } + + public List getClasspaths() { + return classpaths; + } + + public void setClasspaths(List classpaths) { + this.classpaths = classpaths; + } + + public String getEntryPointClass() { + return entryPointClass; + } + + public void setEntryPointClass(String entryPointClass) { + this.entryPointClass = entryPointClass; + } + + public String[] getProgramArgs() { + return programArgs; + } + + public void setProgramArgs(String[] programArgs) { + this.programArgs = programArgs; + } + + public File getJarFile() { + return jarFile; + } + + public void setJarFile(File jarFile) { + this.jarFile = jarFile; + } + + public boolean isCreateProgramDelay() { + return createProgramDelay; + } + + public void setCreateProgramDelay(boolean createProgramDelay) { + this.createProgramDelay = createProgramDelay; + } + + @Override + public String toString() { + return "ClusterSpecification{" + + "masterMemoryMB=" + masterMemoryMB + + ", taskManagerMemoryMB=" + taskManagerMemoryMB + + ", numberTaskManagers=" + numberTaskManagers + + ", slotsPerTaskManager=" + slotsPerTaskManager + + ", priority=" + priority + + '}'; + } + + public static ClusterSpecification fromConfiguration(Configuration configuration) { + int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + + int jobManagerMemoryMb = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); + int taskManagerMemoryMb = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); + + return new ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setNumberTaskManagers(1) + .setSlotsPerTaskManager(slots) + .createClusterSpecification(); + } + + /** + * Builder for the {@link ClusterSpecification} instance. + */ + public static class ClusterSpecificationBuilder { + private int masterMemoryMB = 768; + private int taskManagerMemoryMB = 768; + private int numberTaskManagers = 1; + private int slotsPerTaskManager = 1; + private int parallelism = 1; + private int priority = 0; + + public ClusterSpecificationBuilder setMasterMemoryMB(int masterMemoryMB) { + this.masterMemoryMB = masterMemoryMB; + return this; + } + + public ClusterSpecificationBuilder setTaskManagerMemoryMB(int taskManagerMemoryMB) { + this.taskManagerMemoryMB = taskManagerMemoryMB; + return this; + } + + public ClusterSpecificationBuilder setNumberTaskManagers(int numberTaskManagers) { + this.numberTaskManagers = numberTaskManagers; + return this; + } + + public ClusterSpecificationBuilder setSlotsPerTaskManager(int slotsPerTaskManager) { + this.slotsPerTaskManager = slotsPerTaskManager; + return this; + } + + public ClusterSpecificationBuilder setPriority(int priority){ + this.priority = priority; + return this; + } + + public ClusterSpecificationBuilder setParallelism(int parallelism) { + this.parallelism = parallelism; + return this; + } + + public ClusterSpecification createClusterSpecification() { + return new ClusterSpecification( + masterMemoryMB, + taskManagerMemoryMB, + numberTaskManagers, + slotsPerTaskManager, + parallelism, + priority); + } + } +} diff --git a/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java new file mode 100644 index 0000000000..b0403ab908 --- /dev/null +++ b/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -0,0 +1,1614 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.yarn; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.*; +import org.apache.flink.core.plugin.PluginConfig; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; +import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; +import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.*; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.URISyntaxException; +import java.net.URLDecoder; +import java.nio.charset.Charset; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.*; +import java.util.stream.Collectors; + +import static com.dtstack.flinkx.constants.ConfigConstant.FLINK_PLUGIN_LOAD_MODE_KEY; +import static com.dtstack.flinkx.constants.ConstantValue.SHIP_FILE_PLUGIN_LOAD_MODE; +import static com.dtstack.flinkx.launcher.perJob.FlinkPerJobUtil.buildProgram; +import static com.dtstack.flinkx.launcher.perJob.FlinkPerJobUtil.getUrlFormat; +import static org.apache.flink.configuration.ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; + +/** + * The descriptor with deployment information for deploying a Flink cluster on Yarn. + */ +public class YarnClusterDescriptor implements ClusterDescriptor { + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + + private final YarnConfiguration yarnConfiguration; + + private final YarnClient yarnClient; + + private final YarnClusterInformationRetriever yarnClusterInformationRetriever; + + /** True if the descriptor must not shut down the YarnClient. */ + private final boolean sharedYarnClient; + + /** Lazily initialized list of files to ship. */ + private final List shipFiles = new LinkedList<>(); + + private final String yarnQueue; + + private Path flinkJarPath; + + private final Configuration flinkConfiguration; + + private final String customName; + + private final String nodeLabel; + + private final String applicationType; + + private String zookeeperNamespace; + + private YarnConfigOptions.UserJarInclusion userJarInclusion; + + public YarnClusterDescriptor( + Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, + YarnClient yarnClient, + YarnClusterInformationRetriever yarnClusterInformationRetriever, + boolean sharedYarnClient) { + + this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.yarnClusterInformationRetriever = Preconditions.checkNotNull(yarnClusterInformationRetriever); + this.sharedYarnClient = sharedYarnClient; + + this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); + this.userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + + getLocalFlinkDistPath(flinkConfiguration).ifPresent(this::setLocalJarPath); + decodeDirsToShipToCluster(flinkConfiguration).ifPresent(this::addShipFiles); + + this.yarnQueue = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_QUEUE); + this.customName = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_NAME); + this.applicationType = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TYPE); + this.nodeLabel = flinkConfiguration.getString(YarnConfigOptions.NODE_LABEL); + + // we want to ignore the default value at this point. + this.zookeeperNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, null); + } + + private Optional> decodeDirsToShipToCluster(final Configuration configuration) { + checkNotNull(configuration); + + final List files = ConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); + return files.isEmpty() ? Optional.empty() : Optional.of(files); + } + + private Optional getLocalFlinkDistPath(final Configuration configuration) { + final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); + if (localJarPath != null) { + return Optional.of(new Path(localJarPath)); + } + + LOG.info("No path for the flink jar passed. Using the location of " + getClass() + " to locate the jar"); + + // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar + final String decodedPath = getDecodedJarPath(); + return decodedPath.endsWith(".jar") + ? Optional.of(new Path(new File(decodedPath).toURI())) + : Optional.empty(); + } + + private String getDecodedJarPath() { + final String encodedJarPath = getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); + try { + return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + + " You can supply a path manually via the command line."); + } + } + + @VisibleForTesting + List getShipFiles() { + return shipFiles; + } + + public YarnClient getYarnClient() { + return yarnClient; + } + + /** + * The class to start the application master with. This class runs the main + * method in case of session cluster. + */ + protected String getYarnSessionClusterEntrypoint() { + return YarnSessionClusterEntrypoint.class.getName(); + } + + /** + * The class to start the application master with. This class runs the main + * method in case of the job cluster. + */ + protected String getYarnJobClusterEntrypoint() { + return YarnJobClusterEntrypoint.class.getName(); + } + + public Configuration getFlinkConfiguration() { + return flinkConfiguration; + } + + public void setLocalJarPath(Path localJarPath) { + if (!localJarPath.toString().endsWith("jar")) { + throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); + } + this.flinkJarPath = localJarPath; + } + + /** + * Adds the given files to the list of files to ship. + * + *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, String, StringBuilder)} + * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. + * + * @param shipFiles files to ship + */ + public void addShipFiles(List shipFiles) { + checkArgument(userJarInclusion != YarnConfigOptions.UserJarInclusion.DISABLED || isUsrLibDirIncludedInShipFiles(shipFiles), + "This is an illegal ship directory : %s. When setting the %s to %s the name of ship directory can not be %s.", + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR, + YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key(), + YarnConfigOptions.UserJarInclusion.DISABLED, + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); + this.shipFiles.addAll(shipFiles); + } + + private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws Exception { + + if (this.flinkJarPath == null) { + throw new YarnDeploymentException("The Flink jar path is null"); + } + if (this.flinkConfiguration == null) { + throw new YarnDeploymentException("Flink configuration object has not been set"); + } + + // Check if we don't exceed YARN's maximum virtual cores. + final int numYarnMaxVcores = yarnClusterInformationRetriever.getMaxVcores(); + + int configuredAmVcores = flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES); + if (configuredAmVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException( + String.format("The number of requested virtual cores for application master %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster.", + configuredAmVcores, numYarnMaxVcores)); + } + + int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); + // don't configure more than the maximum configured number of vcores + if (configuredVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException( + String.format("The number of requested virtual cores per node %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + + " Please note that the number of virtual cores is set to the number of task slots by default" + + " unless configured in the Flink config with '%s.'", + configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); + } + + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + } + + public String getZookeeperNamespace() { + return zookeeperNamespace; + } + + private void setZookeeperNamespace(String zookeeperNamespace) { + this.zookeeperNamespace = zookeeperNamespace; + } + + public String getNodeLabel() { + return nodeLabel; + } + + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + + @Override + public void close() { + if (!sharedYarnClient) { + yarnClient.stop(); + } + } + + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + + @Override + public ClusterClientProvider retrieve(ApplicationId applicationId) throws ClusterRetrieveException { + + try { + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + + final ApplicationReport report = yarnClient.getApplicationReport(applicationId); + + if (report.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { + // Flink cluster is not running anymore + LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}", + applicationId, report.getFinalApplicationStatus()); + throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); + } + + setClusterEntrypointInfoToConfig(report); + + return () -> { + try { + return new RestClusterClient<>(flinkConfiguration, report.getApplicationId()); + } catch (Exception e) { + throw new RuntimeException("Couldn't retrieve Yarn cluster", e); + } + }; + } catch (Exception e) { + throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); + } + } + + @Override + public ClusterClientProvider deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException { + try { + return deployInternal( + clusterSpecification, + "Flink session cluster", + getYarnSessionClusterEntrypoint(), + null, + false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); + } + } + + @Override + public ClusterClientProvider deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) throws ClusterDeploymentException { + try { + return deployInternal( + clusterSpecification, + "Flink per-job cluster", + getYarnJobClusterEntrypoint(), + jobGraph, + detached); + } catch (Exception e) { + throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); + } + } + + @Override + public void killCluster(ApplicationId applicationId) throws FlinkException { + try { + yarnClient.killApplication(applicationId); + Utils.deleteApplicationFiles(Collections.singletonMap( + YarnConfigKeys.FLINK_YARN_FILES, + getYarnFilesDir(applicationId).toUri().toString())); + } catch (YarnException | IOException e) { + throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); + } + } + + /** + * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. + * + * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed + * @param applicationName name of the Yarn application to start + * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. + * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none + * @param detached True if the cluster should be started in detached mode + */ + private ClusterClientProvider deployInternal( + ClusterSpecification clusterSpecification, + String applicationName, + String yarnClusterEntrypoint, + @Nullable JobGraph jobGraph, + boolean detached) throws Exception { + + if (UserGroupInformation.isSecurityEnabled()) { + // note: UGI::hasKerberosCredentials inaccurately reports false + // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), + // so we check only in ticket cache scenario. + boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); + + UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); + if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS + && useTicketCache && !loginUser.hasKerberosCredentials()) { + LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); + throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + + "does not have Kerberos credentials"); + } + } + + isReadyForDeployment(clusterSpecification); + + // ------------------ Check if the specified queue exists -------------------- + + checkYarnQueues(yarnClient); + + // ------------------ Check if the YARN ClusterClient has the requested resources -------------- + + // Create application via yarnClient + final YarnClientApplication yarnApplication = yarnClient.createApplication(); + final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); + + if(clusterSpecification.isCreateProgramDelay()){ + String url = getUrlFormat(clusterSpecification.getYarnConfiguration(), yarnClient) + "/" + appResponse.getApplicationId().toString(); + PackagedProgram program = buildProgram(url,clusterSpecification); + clusterSpecification.setProgram(program); + jobGraph = PackagedProgramUtils.createJobGraph(program, clusterSpecification.getConfiguration(), clusterSpecification.getParallelism(), false); + String pluginLoadMode = clusterSpecification.getConfiguration().getString(FLINK_PLUGIN_LOAD_MODE_KEY); + if(StringUtils.equalsIgnoreCase(pluginLoadMode, SHIP_FILE_PLUGIN_LOAD_MODE)){ + jobGraph.getClasspaths().forEach(jarFile -> { + try { + shipFiles.add(new File(jarFile.toURI())); + } catch (URISyntaxException e) { + throw new IllegalArgumentException("Couldn't add local user jar: " + jarFile + + " Currently only file:/// URLs are supported."); + } + }); + jobGraph.getClasspaths().clear(); + addShipFiles(shipFiles); + } + clusterSpecification.setJobGraph(jobGraph); + } + + Resource maxRes = appResponse.getMaximumResourceCapability(); + + final ClusterResourceDescription freeClusterMem; + try { + freeClusterMem = getCurrentFreeClusterResources(yarnClient); + } catch (YarnException | IOException e) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); + } + + final int yarnMinAllocationMB = yarnConfiguration.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); + + final ClusterSpecification validClusterSpecification; + try { + validClusterSpecification = validateClusterResources( + clusterSpecification, + yarnMinAllocationMB, + maxRes, + freeClusterMem); + } catch (YarnDeploymentException yde) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw yde; + } + + LOG.info("Cluster specification: {}", validClusterSpecification); + + final ClusterEntrypoint.ExecutionMode executionMode = detached ? + ClusterEntrypoint.ExecutionMode.DETACHED + : ClusterEntrypoint.ExecutionMode.NORMAL; + + flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); + + ApplicationReport report = startAppMaster( + flinkConfiguration, + applicationName, + yarnClusterEntrypoint, + jobGraph, + yarnClient, + yarnApplication, + validClusterSpecification); + + // print the application id for user to cancel themselves. + if (detached) { + final ApplicationId yarnApplicationId = report.getApplicationId(); + logDetachedClusterInformation(yarnApplicationId, LOG); + } + + setClusterEntrypointInfoToConfig(report); + + return () -> { + try { + return new RestClusterClient<>(flinkConfiguration, report.getApplicationId()); + } catch (Exception e) { + throw new RuntimeException("Error while creating RestClusterClient.", e); + } + }; + } + + private ClusterSpecification validateClusterResources( + ClusterSpecification clusterSpecification, + int yarnMinAllocationMB, + Resource maximumResourceCapability, + ClusterResourceDescription freeClusterResources) throws YarnDeploymentException { + + int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); + final int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); + + if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { + LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " + + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + + "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + + "you requested will start."); + } + + // set the memory to minAllocationMB to do the next checks correctly + if (jobManagerMemoryMb < yarnMinAllocationMB) { + jobManagerMemoryMb = yarnMinAllocationMB; + } + + final String note = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; + if (jobManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + note); + } + + if (taskManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + note); + } + + final String noteRsc = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + + "connecting from the beginning because the resources are currently not available in the cluster. " + + "The allocation might take more time than usual because the Flink YARN client needs to wait until " + + "the resources become available."; + + if (taskManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + if (jobManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) + .createClusterSpecification(); + + } + + private void checkYarnQueues(YarnClient yarnClient) { + try { + List queues = yarnClient.getAllQueues(); + if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. + boolean queueFound = false; + for (QueueInfo queue : queues) { + if (queue.getQueueName().equals(this.yarnQueue)) { + queueFound = true; + break; + } + } + if (!queueFound) { + String queueNames = ""; + for (QueueInfo queue : queues) { + queueNames += queue.getQueueName() + ", "; + } + LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + + "Available queues: " + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + } catch (Throwable e) { + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Error details", e); + } + } + } + + private ApplicationReport startAppMaster( + Configuration configuration, + String applicationName, + String yarnClusterEntrypoint, + JobGraph jobGraph, + YarnClient yarnClient, + YarnClientApplication yarnApplication, + ClusterSpecification clusterSpecification) throws Exception { + + // ------------------ Initialize the file systems ------------------------- + + org.apache.flink.core.fs.FileSystem.initialize( + configuration, + PluginUtils.createPluginManagerFromRootFolder(configuration)); + + // initialize file system + // Copy the application master jar to the filesystem + // Create a local resource to point to the destination jar path + final FileSystem fs = FileSystem.get(yarnConfiguration); + final Path homeDir = fs.getHomeDirectory(); + + // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. + if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && + fs.getScheme().startsWith("file")) { + LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the " + + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." + + "The Flink YARN client needs to store its files in a distributed file system"); + } + + ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + // The files need to be shipped and added to classpath. + Set systemShipFiles = new HashSet<>(shipFiles.size()); + // The files only need to be shipped. + Set shipOnlyFiles = new HashSet<>(); + for (File file : shipFiles) { + systemShipFiles.add(file.getAbsoluteFile()); + } + + final String logConfigFilePath = configuration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE); + if (logConfigFilePath != null) { + systemShipFiles.add(new File(logConfigFilePath)); + } + + addLibFoldersToShipFiles(systemShipFiles); + + // Plugin files only need to be shipped and should not be added to classpath. + addPluginsFoldersToShipFiles(shipOnlyFiles); + + // Set-up ApplicationSubmissionContext for the application + + final ApplicationId appId = appContext.getApplicationId(); + + // ------------------ Add Zookeeper namespace to local flinkConfiguraton ------ + String zkNamespace = getZookeeperNamespace(); + // no user specified cli argument for namespace? + if (zkNamespace == null || zkNamespace.isEmpty()) { + // namespace defined in config? else use applicationId as default. + zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); + setZookeeperNamespace(zkNamespace); + } + + configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); + + if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { + // activate re-execution of failed applications + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); + + activateHighAvailabilitySupport(appContext); + } else { + // set number of application retries to 1 in the default case + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + 1)); + } + + final Set userJarFiles = (jobGraph == null) + // not per-job submission + ? Collections.emptySet() + // add user code jars from the provided JobGraph + : jobGraph.getUserJars().stream().map(f -> f.toUri()).map(File::new).collect(Collectors.toSet()); + + // only for per job mode + if (jobGraph != null) { + for (Map.Entry entry : jobGraph.getUserArtifacts().entrySet()) { + org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(entry.getValue().filePath); + // only upload local files + if (!path.getFileSystem().isDistributedFS()) { + Path localPath = new Path(path.getPath()); + Tuple2 remoteFileInfo = + Utils.uploadLocalFileToRemote(fs, appId.toString(), localPath, homeDir, entry.getKey()); + jobGraph.setUserArtifactRemotePath(entry.getKey(), remoteFileInfo.f0.toString()); + } + } + + jobGraph.writeUserArtifactEntriesToConfiguration(); + } + + // local resource map for Yarn + final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); + // list of remote paths (after upload) + final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); + // ship list that enables reuse of resources for task manager containers + StringBuilder envShipFileList = new StringBuilder(); + + // upload and register ship files, these files will be added to classpath. + List systemClassPaths = uploadAndRegisterFiles( + systemShipFiles, + fs, + homeDir, + appId, + paths, + localResources, + Path.CUR_DIR, + envShipFileList); + + // upload and register ship-only files + uploadAndRegisterFiles( + shipOnlyFiles, + fs, + homeDir, + appId, + paths, + localResources, + Path.CUR_DIR, + envShipFileList); + + final List userClassPaths = uploadAndRegisterFiles( + userJarFiles, + fs, + homeDir, + appId, + paths, + localResources, + userJarInclusion == YarnConfigOptions.UserJarInclusion.DISABLED ? + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR : Path.CUR_DIR, + envShipFileList); + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { + systemClassPaths.addAll(userClassPaths); + } + + // normalize classpath by sorting + Collections.sort(systemClassPaths); + Collections.sort(userClassPaths); + + // classpath assembler + StringBuilder classPathBuilder = new StringBuilder(); + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + for (String classPath : systemClassPaths) { + classPathBuilder.append(classPath).append(File.pathSeparator); + } + + // Setup jar for ApplicationMaster + Path remotePathJar = setupSingleLocalResource( + flinkJarPath.getName(), + fs, + appId, + flinkJarPath, + localResources, + homeDir, + ""); + + paths.add(remotePathJar); + classPathBuilder.append(flinkJarPath.getName()).append(File.pathSeparator); + + // Upload the flink configuration + // write out configuration file + File tmpConfigurationFile = null; + try { + tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); + + String flinkConfigKey = "flink-conf.yaml"; + Path remotePathConf = setupSingleLocalResource( + flinkConfigKey, + fs, + appId, + new Path(tmpConfigurationFile.getAbsolutePath()), + localResources, + homeDir, + ""); + envShipFileList.append(flinkConfigKey).append("=").append(remotePathConf).append(","); + paths.add(remotePathConf); + classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); + } finally { + if (tmpConfigurationFile != null && !tmpConfigurationFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); + } + } + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + + // write job graph to tmp file and add it to local resource + // TODO: server use user main method to generate job graph + if (jobGraph != null) { + File tmpJobGraphFile = null; + try { + tmpJobGraphFile = File.createTempFile(appId.toString(), null); + try (FileOutputStream output = new FileOutputStream(tmpJobGraphFile); + ObjectOutputStream obOutput = new ObjectOutputStream(output);){ + obOutput.writeObject(jobGraph); + } + + final String jobGraphFilename = "job.graph"; + flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); + + Path pathFromYarnURL = setupSingleLocalResource( + jobGraphFilename, + fs, + appId, + new Path(tmpJobGraphFile.toURI()), + localResources, + homeDir, + ""); + paths.add(pathFromYarnURL); + classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); + } catch (Exception e) { + LOG.warn("Add job graph to local resource fail"); + throw e; + } finally { + if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); + } + } + } + + final Path yarnFilesDir = getYarnFilesDir(appId); + FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); + fs.setPermission(yarnFilesDir, permission); // set permission for path. + + //To support Yarn Secure Integration Test Scenario + //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML + //and KRB5 configuration files. We are adding these files as container local resources for the container + //applications (JM/TMs) to have proper secure cluster setup + Path remoteKrb5Path = null; + Path remoteYarnSiteXmlPath = null; + boolean hasKrb5 = false; + if (System.getenv("IN_TESTS") != null) { + File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); + LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); + Path yarnSitePath = new Path(f.getAbsolutePath()); + remoteYarnSiteXmlPath = setupSingleLocalResource( + Utils.YARN_SITE_FILE_NAME, + fs, + appId, + yarnSitePath, + localResources, + homeDir, + ""); + + String krb5Config = System.getProperty("java.security.krb5.conf"); + if (krb5Config != null && krb5Config.length() != 0) { + File krb5 = new File(krb5Config); + LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); + Path krb5ConfPath = new Path(krb5.getAbsolutePath()); + remoteKrb5Path = setupSingleLocalResource( + Utils.KRB5_FILE_NAME, + fs, + appId, + krb5ConfPath, + localResources, + homeDir, + ""); + hasKrb5 = true; + } + } + + // setup security tokens + Path remotePathKeytab = null; + String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + if (keytab != null) { + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = setupSingleLocalResource( + Utils.KEYTAB_FILE_NAME, + fs, + appId, + new Path(keytab), + localResources, + homeDir, + ""); + } + + final boolean hasLogback = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME); + final boolean hasLog4j = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOG4J_NAME); + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( + yarnClusterEntrypoint, + hasLogback, + hasLog4j, + hasKrb5, + clusterSpecification.getMasterMemoryMB()); + + if (UserGroupInformation.isSecurityEnabled()) { + // set HDFS delegation tokens when security is enabled + LOG.info("Adding delegation token to the AM container."); + Utils.setTokensFor(amContainer, paths, yarnConfiguration); + } + + amContainer.setLocalResources(localResources); + fs.close(); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = new HashMap<>(); + // set user specified app master environment variables + appMasterEnv.putAll( + BootstrapTools.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); + // set Flink app class path + appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); + + // set Flink on YARN internal configuration values + appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); + appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); + + // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); + + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); + String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + } + + //To support Yarn Secure Integration Test Scenario + if (remoteYarnSiteXmlPath != null) { + appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); + } + if (remoteKrb5Path != null) { + appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); + } + + // set classpath from YARN configuration + Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(clusterSpecification.getMasterMemoryMB()); + capability.setVirtualCores(flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES)); + + final String customApplicationName = customName != null ? customName : applicationName; + + appContext.setApplicationName(customApplicationName); + appContext.setApplicationType(applicationType != null ? applicationType : "Apache Flink"); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + + // Set priority for application + int priorityNum = flinkConfiguration.getInteger(YarnConfigOptions.APPLICATION_PRIORITY); + if (priorityNum >= 0) { + Priority priority = Priority.newInstance(priorityNum); + appContext.setPriority(priority); + } + + if (yarnQueue != null) { + appContext.setQueue(yarnQueue); + } + + setApplicationNodeLabel(appContext); + + setApplicationTags(appContext); + + // add a hook to clean up in case deployment fails + Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, yarnFilesDir); + Runtime.getRuntime().addShutdownHook(deploymentFailureHook); + LOG.info("Submitting application master " + appId); + yarnClient.submitApplication(appContext); + + LOG.info("Waiting for the cluster to be allocated"); + final long startTime = System.currentTimeMillis(); + ApplicationReport report; + YarnApplicationState lastAppState = YarnApplicationState.NEW; + loop: while (true) { + try { + report = yarnClient.getApplicationReport(appId); + } catch (IOException e) { + throw new YarnDeploymentException("Failed to deploy the cluster.", e); + } + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + switch(appState) { + case FAILED: + case KILLED: + throw new YarnDeploymentException("The YARN application unexpectedly switched to state " + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + appId); + //break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break loop; + case FINISHED: + LOG.info("YARN application has been finished successfully."); + break loop; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + if (System.currentTimeMillis() - startTime > 60000) { + LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); + } + + } + lastAppState = appState; + Thread.sleep(250); + } + + // since deployment was successful, remove the hook + ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); + return report; + } + + /** + * Returns the Path where the YARN application files should be uploaded to. + * + * @param appId YARN application id + */ + private Path getYarnFilesDir(final ApplicationId appId) throws IOException { + final FileSystem fileSystem = FileSystem.get(yarnConfiguration); + final Path homeDir = fileSystem.getHomeDirectory(); + return new Path(homeDir, ".flink/" + appId + '/'); + } + + /** + * Uploads and registers a single resource and adds it to localResources. + * + * @param key + * the key to add the resource under + * @param fs + * the remote file system to upload to + * @param appId + * application ID + * @param localSrcPath + * local path to the file + * @param localResources + * map of resources + * + * @return the remote path to the uploaded resource + */ + private static Path setupSingleLocalResource( + String key, + FileSystem fs, + ApplicationId appId, + Path localSrcPath, + Map localResources, + Path targetHomeDir, + String relativeTargetPath) throws IOException { + Tuple2 resource = Utils.setupLocalResource( + fs, + appId.toString(), + localSrcPath, + targetHomeDir, + relativeTargetPath); + + localResources.put(key, resource.f1); + + return resource.f0; + } + + /** + * Match file name for "flink-dist*.jar" pattern. + * + * @param fileName file name to check + * @return true if file is a dist jar + */ + private static boolean isDistJar(String fileName) { + return fileName.startsWith("flink-dist") && fileName.endsWith("jar"); + } + + /** + * Recursively uploads (and registers) any (user and system) files in shipFiles except + * for files matching "flink-dist*.jar" which should be uploaded separately. + * + * @param shipFiles + * files to upload + * @param fs + * file system to upload to + * @param targetHomeDir + * remote home directory to upload to + * @param appId + * application ID + * @param remotePaths + * paths of the remote resources (uploaded resources will be added) + * @param localResources + * map of resources (uploaded resources will be added) + * @param localResourcesDirectory + * the directory the localResources are uploaded to + * @param envShipFileList + * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} + * + * @return list of class paths with the the proper resource keys from the registration + */ + static List uploadAndRegisterFiles( + Collection shipFiles, + FileSystem fs, + Path targetHomeDir, + ApplicationId appId, + List remotePaths, + Map localResources, + String localResourcesDirectory, + StringBuilder envShipFileList) throws IOException { + final List localPaths = new ArrayList<>(); + final List relativePaths = new ArrayList<>(); + for (File shipFile : shipFiles) { + if (shipFile.isDirectory()) { + // add directories to the classpath + final java.nio.file.Path shipPath = shipFile.toPath(); + final java.nio.file.Path parentPath = shipPath.getParent(); + Files.walkFileTree(shipPath, new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) { + localPaths.add(new Path(file.toUri())); + relativePaths.add(new Path(localResourcesDirectory, parentPath.relativize(file).toString())); + return FileVisitResult.CONTINUE; + } + }); + } else { + localPaths.add(new Path(shipFile.toURI())); + relativePaths.add(new Path(localResourcesDirectory, shipFile.getName())); + } + } + + final Set archives = new HashSet<>(); + final Set resources = new HashSet<>(); + for (int i = 0; i < localPaths.size(); i++) { + final Path localPath = localPaths.get(i); + final Path relativePath = relativePaths.get(i); + if (!isDistJar(relativePath.getName())) { + final String key = relativePath.toString(); + final Path remotePath = setupSingleLocalResource( + key, + fs, + appId, + localPath, + localResources, + targetHomeDir, + relativePath.getParent().toString()); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + // add files to the classpath + if (key.endsWith("jar")) { + archives.add(relativePath.toString()); + } else { + resources.add(relativePath.getParent().toString()); + } + } + } + + // construct classpath, we always want resource directories to go first, we also sort + // both resources and archives in order to make classpath deterministic + final ArrayList classPaths = new ArrayList<>(); + resources.stream().sorted().forEach(classPaths::add); + archives.stream().sorted().forEach(classPaths::add); + return classPaths; + } + + /** + * Kills YARN application and stops YARN client. + * + *

Use this method to kill the App before it has been properly deployed + */ + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { + LOG.info("Killing YARN application"); + + try { + yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); + } catch (Exception e) { + // we only log a debug message here because the "killApplication" call is a best-effort + // call (we don't know if the application has been deployed when the error occured). + LOG.debug("Error while killing YARN application", e); + } + } + + private static class ClusterResourceDescription { + public final int totalFreeMemory; + public final int containerLimit; + public final int[] nodeManagersFree; + + public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { + this.totalFreeMemory = totalFreeMemory; + this.containerLimit = containerLimit; + this.nodeManagersFree = nodeManagersFree; + } + } + + private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + + int totalFreeMemory = 0; + int containerLimit = 0; + int[] nodeManagersFree = new int[nodes.size()]; + + for (int i = 0; i < nodes.size(); i++) { + NodeReport rep = nodes.get(i); + int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0); + nodeManagersFree[i] = free; + totalFreeMemory += free; + if (free > containerLimit) { + containerLimit = free; + } + } + return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); + } + + @Override + public String getClusterDescription() { + + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + + YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); + + ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + final String format = "|%-16s |%-16s %n"; + ps.printf("|Property |Value %n"); + ps.println("+---------------------------------------+"); + int totalMemory = 0; + int totalCores = 0; + for (NodeReport rep : nodes) { + final Resource res = rep.getCapability(); + totalMemory += res.getMemory(); + totalCores += res.getVirtualCores(); + ps.format(format, "NodeID", rep.getNodeId()); + ps.format(format, "Memory", res.getMemory() + " MB"); + ps.format(format, "vCores", res.getVirtualCores()); + ps.format(format, "HealthReport", rep.getHealthReport()); + ps.format(format, "Containers", rep.getNumContainers()); + ps.println("+---------------------------------------+"); + } + ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); + List qInfo = yarnClient.getAllQueues(); + for (QueueInfo q : qInfo) { + ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + + q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); + } + return baos.toString(); + } catch (Exception e) { + throw new RuntimeException("Couldn't get cluster description", e); + } + } + + private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws + InvocationTargetException, IllegalAccessException { + + ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + + reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); + + reflector.setAttemptFailuresValidityInterval( + appContext, + flinkConfiguration.getLong(YarnConfigOptions.APPLICATION_ATTEMPT_FAILURE_VALIDITY_INTERVAL)); + } + + private void setApplicationTags(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + final String tagsString = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TAGS); + + final Set applicationTags = new HashSet<>(); + + // Trim whitespace and cull empty tags + for (final String tag : tagsString.split(",")) { + final String trimmedTag = tag.trim(); + if (!trimmedTag.isEmpty()) { + applicationTags.add(trimmedTag); + } + } + + reflector.setApplicationTags(appContext, applicationTags); + } + + private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + if (nodeLabel != null) { + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + reflector.setApplicationNodeLabel(appContext, nodeLabel); + } + } + + /** + * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext} + * supports various methods which, depending on the Hadoop version, may or may not be supported. + * + *

If an unsupported method is invoked, nothing happens. + * + *

Currently three methods are proxied: + * - setApplicationTags (>= 2.4.0) + * - setAttemptFailuresValidityInterval (>= 2.6.0) + * - setKeepContainersAcrossApplicationAttempts (>= 2.4.0) + * - setNodeLabelExpression (>= 2.6.0) + */ + private static class ApplicationSubmissionContextReflector { + private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); + + private static final ApplicationSubmissionContextReflector instance = + new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); + + public static ApplicationSubmissionContextReflector getInstance() { + return instance; + } + + private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; + private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; + private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; + private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; + + private final Method applicationTagsMethod; + private final Method attemptFailuresValidityIntervalMethod; + private final Method keepContainersMethod; + @Nullable + private final Method nodeLabelExpressionMethod; + + private ApplicationSubmissionContextReflector(Class clazz) { + Method applicationTagsMethod; + Method attemptFailuresValidityIntervalMethod; + Method keepContainersMethod; + Method nodeLabelExpressionMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + applicationTagsMethod = null; + } + + this.applicationTagsMethod = applicationTagsMethod; + + try { + // this method is only supported by Hadoop 2.6.0 onwards + attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + attemptFailuresValidityIntervalMethod = null; + } + + this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + keepContainersMethod = null; + } + + this.keepContainersMethod = keepContainersMethod; + + try { + nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + nodeLabelExpressionMethod = null; + } + + this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; + } + + public void setApplicationTags( + ApplicationSubmissionContext appContext, + Set applicationTags) throws InvocationTargetException, IllegalAccessException { + if (applicationTagsMethod != null) { + LOG.debug("Calling method {} of {}.", + applicationTagsMethod.getName(), + appContext.getClass().getCanonicalName()); + applicationTagsMethod.invoke(appContext, applicationTags); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + APPLICATION_TAGS_METHOD_NAME); + } + } + + public void setApplicationNodeLabel( + ApplicationSubmissionContext appContext, + String nodeLabel) throws InvocationTargetException, IllegalAccessException { + if (nodeLabelExpressionMethod != null) { + LOG.debug("Calling method {} of {}.", + nodeLabelExpressionMethod.getName(), + appContext.getClass().getCanonicalName()); + nodeLabelExpressionMethod.invoke(appContext, nodeLabel); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + NODE_LABEL_EXPRESSION_NAME); + } + } + + public void setAttemptFailuresValidityInterval( + ApplicationSubmissionContext appContext, + long validityInterval) throws InvocationTargetException, IllegalAccessException { + if (attemptFailuresValidityIntervalMethod != null) { + LOG.debug("Calling method {} of {}.", + attemptFailuresValidityIntervalMethod.getName(), + appContext.getClass().getCanonicalName()); + attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + ATTEMPT_FAILURES_METHOD_NAME); + } + } + + public void setKeepContainersAcrossApplicationAttempts( + ApplicationSubmissionContext appContext, + boolean keepContainers) throws InvocationTargetException, IllegalAccessException { + + if (keepContainersMethod != null) { + LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(), + appContext.getClass().getCanonicalName()); + keepContainersMethod.invoke(appContext, keepContainers); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } + } + } + + private static class YarnDeploymentException extends RuntimeException { + private static final long serialVersionUID = -812040641215388943L; + + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); + } + } + + private class DeploymentFailureHook extends Thread { + + private final YarnClient yarnClient; + private final YarnClientApplication yarnApplication; + private final Path yarnFilesDir; + + DeploymentFailureHook(YarnClientApplication yarnApplication, Path yarnFilesDir) { + this.yarnApplication = Preconditions.checkNotNull(yarnApplication); + this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); + + // A new yarn client need to be created in shutdown hook in order to avoid + // the yarn client has been closed by YarnClusterDescriptor. + this.yarnClient = YarnClient.createYarnClient(); + this.yarnClient.init(yarnConfiguration); + } + + @Override + public void run() { + LOG.info("Cancelling deployment from Deployment Failure Hook"); + yarnClient.start(); + failSessionDuringDeployment(yarnClient, yarnApplication); + yarnClient.stop(); + LOG.info("Deleting files in {}.", yarnFilesDir); + try { + FileSystem fs = FileSystem.get(yarnConfiguration); + + if (!fs.delete(yarnFilesDir, true)) { + throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); + } + + fs.close(); + } catch (IOException e) { + LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); + } + } + } + + @VisibleForTesting + void addLibFoldersToShipFiles(Collection effectiveShipFiles) { + // Add lib folder to the ship files if the environment variable is set. + // This is for convenience when running from the command-line. + // (for other files users explicitly set the ship files) + String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); + if (libDir != null) { + File directoryFile = new File(libDir); + if (directoryFile.isDirectory()) { + effectiveShipFiles.add(directoryFile); + } else { + throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + + "' is set to '" + libDir + "' but the directory doesn't exist."); + } + } else if (shipFiles.isEmpty()) { + LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + + "Not shipping any library files.", ENV_FLINK_LIB_DIR); + } + } + + @VisibleForTesting + void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { + final Optional pluginsDir = PluginConfig.getPluginsDir(); + pluginsDir.ifPresent(effectiveShipFiles::add); + } + + ContainerLaunchContext setupApplicationMasterContainer( + String yarnClusterEntrypoint, + boolean hasLogback, + boolean hasLog4j, + boolean hasKrb5, + int jobManagerMemoryMb) { + // ------------------ Prepare Application Master Container ------------------------------ + + // respect custom JVM options in the YAML file + String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS); + if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) { + javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS); + } + //applicable only for YarnMiniCluster secure test run + //krb5.conf file will be available as local resource in JM/TM container + if (hasKrb5) { + javaOpts += " -Djava.security.krb5.conf=krb5.conf"; + } + + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); + + final Map startCommandValues = new HashMap<>(); + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + + int heapSize = BootstrapTools.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); + String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); + startCommandValues.put("jvmmem", jvmHeapMem); + + startCommandValues.put("jvmopts", javaOpts); + String logging = ""; + + if (hasLogback || hasLog4j) { + logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; + + if (hasLogback) { + logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; + } + + if (hasLog4j) { + logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; + } + } + + startCommandValues.put("logging", logging); + startCommandValues.put("class", yarnClusterEntrypoint); + startCommandValues.put("redirects", + "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + + "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"); + startCommandValues.put("args", ""); + + final String commandTemplate = flinkConfiguration + .getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, + ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE); + final String amCommand = + BootstrapTools.getStartCommand(commandTemplate, startCommandValues); + + amContainer.setCommands(Collections.singletonList(amCommand)); + + LOG.debug("Application Master start command: " + amCommand); + + return amContainer; + } + + private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(org.apache.flink.configuration.Configuration config) { + return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + } + + private static boolean isUsrLibDirIncludedInShipFiles(List shipFiles) { + return shipFiles.stream() + .filter(File::isDirectory) + .map(File::getName) + .noneMatch(name -> name.equals(DEFAULT_FLINK_USR_LIB_DIR)); + } + + private void setClusterEntrypointInfoToConfig(final ApplicationReport report) { + checkNotNull(report); + + final ApplicationId clusterId = report.getApplicationId(); + final String host = report.getHost(); + final int port = report.getRpcPort(); + + LOG.info("Found Web Interface {}:{} of application '{}'.", host, port, clusterId); + + flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); + flinkConfiguration.setInteger(JobManagerOptions.PORT, port); + + flinkConfiguration.setString(RestOptions.ADDRESS, host); + flinkConfiguration.setInteger(RestOptions.PORT, port); + + flinkConfiguration.set(YarnConfigOptions.APPLICATION_ID, ConverterUtils.toString(clusterId)); + } + + public static void logDetachedClusterInformation(ApplicationId yarnApplicationId, Logger logger) { + logger.info( + "The Flink YARN session cluster has been started in detached mode. In order to " + + "stop Flink gracefully, use the following command:\n" + + "$ echo \"stop\" | ./bin/yarn-session.sh -id {}\n" + + "If this should not be possible, then you can also kill Flink via YARN's web interface or via:\n" + + "$ yarn application -kill {}\n" + + "Note that killing Flink might not clean up all job artifacts and temporary files.", + yarnApplicationId, yarnApplicationId); + } +} + diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml index c2f9c1ac13..259af09b8d 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml @@ -34,13 +34,13 @@ - + - + diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 09814f6c3f..710a998234 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -78,13 +78,13 @@ - + - + diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 1d81ef36ef..383656c888 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -78,13 +78,13 @@ - + - + diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index 047e273350..5c8124f959 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 8856cdf063..7eac7ed4be 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 82e3aa55c4..f96d806524 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 866fd622fe..4c29a6cf54 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -68,13 +68,13 @@ - + - + diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index b268bf43b1..1ee0a81c61 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -73,13 +73,13 @@ - + - + diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 52ff1880be..7afaa78a95 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index c6fc7b4e2b..685f55c47c 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml index 182ea07067..747361a258 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -51,13 +51,13 @@ - + - + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index 180385dc01..6167e63015 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -74,13 +74,13 @@ - + - + diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index b97dab27c8..627fdd665e 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -74,13 +74,13 @@ - + - + diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index d477787cac..7df3b0a7d3 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 093bdb5152..7aa478f6ea 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index d088128085..0df9897338 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index f103f55d85..29126254b7 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 2264b0dddb..3522486db7 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml index 72c8493094..014511243d 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml +++ b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml @@ -55,13 +55,13 @@ - + - + diff --git a/flinkx-rdb/flinkx-rdb-core/pom.xml b/flinkx-rdb/flinkx-rdb-core/pom.xml index fefc94f1e1..189de1ac45 100644 --- a/flinkx-rdb/flinkx-rdb-core/pom.xml +++ b/flinkx-rdb/flinkx-rdb-core/pom.xml @@ -26,13 +26,13 @@ - + - + diff --git a/flinkx-rdb/flinkx-rdb-reader/pom.xml b/flinkx-rdb/flinkx-rdb-reader/pom.xml index 82c83d5493..ecd46d3def 100644 --- a/flinkx-rdb/flinkx-rdb-reader/pom.xml +++ b/flinkx-rdb/flinkx-rdb-reader/pom.xml @@ -35,13 +35,13 @@ - + - + diff --git a/flinkx-rdb/flinkx-rdb-writer/pom.xml b/flinkx-rdb/flinkx-rdb-writer/pom.xml index e286d72d92..d4c61d86f9 100644 --- a/flinkx-rdb/flinkx-rdb-writer/pom.xml +++ b/flinkx-rdb/flinkx-rdb-writer/pom.xml @@ -35,13 +35,13 @@ - + - + diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 4a92436c6b..b08813a704 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -78,13 +78,13 @@ - + - + diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index ba9e5c4375..dfea6e6ccf 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 8c19fad040..95693cc2da 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index 7300b9251f..b43b10f4ba 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -84,13 +84,13 @@ - + - + diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index e4f6ed9b36..b0665fd744 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -85,13 +85,13 @@ - + - + diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index d2dfa853cc..887103a51d 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -67,13 +67,13 @@ - + - + diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index f106065763..eb35697a56 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -63,13 +63,13 @@ - + - + diff --git a/flinkx-teradata/flinkx-teradata-reader/pom.xml b/flinkx-teradata/flinkx-teradata-reader/pom.xml index 44cc030d1f..ef7cea1706 100644 --- a/flinkx-teradata/flinkx-teradata-reader/pom.xml +++ b/flinkx-teradata/flinkx-teradata-reader/pom.xml @@ -84,19 +84,19 @@ - + - + - + diff --git a/flinkx-teradata/flinkx-teradata-writer/pom.xml b/flinkx-teradata/flinkx-teradata-writer/pom.xml index bd44533374..b1b009442b 100644 --- a/flinkx-teradata/flinkx-teradata-writer/pom.xml +++ b/flinkx-teradata/flinkx-teradata-writer/pom.xml @@ -84,19 +84,19 @@ - + - + - + From ab48609b2b7392fe6da2503af94c7c5b942aca0c Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 28 Apr 2020 10:02:48 +0800 Subject: [PATCH 045/136] =?UTF-8?q?=E4=BF=AE=E6=AD=A3no=20session=E5=88=A4?= =?UTF-8?q?=E6=96=AD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/launcher/ClusterClientFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index 6375a284db..2b3c963559 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -84,7 +84,7 @@ public static ClusterClient createYarnClient(Options launcherOptions) { if (StringUtils.isEmpty(launcherOptions.getAppId())) { applicationId = getAppIdFromYarn(yarnClient, launcherOptions); - if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { + if(applicationId == null || StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } } else { From 33b09ef9e01b02a2513d16ed6ae818302f097f1c Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 28 Apr 2020 10:08:43 +0800 Subject: [PATCH 046/136] update ignore --- .gitignore | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 61e703a5e4..bb6fd8924f 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,6 @@ # Created by .ignore support plugin (hsz.mobi) .idea/ -plugins/ +syncplugins/ *.iml target/ lib/ @@ -9,4 +9,3 @@ nohup.out flinkconf/ hadoopconf/ /default_task_id_output -/syncplugins/ From 89d4370303ca9c080d9100739f3b80a6694b692c Mon Sep 17 00:00:00 2001 From: tudou Date: Thu, 30 Apr 2020 10:26:56 +0800 Subject: [PATCH 047/136] =?UTF-8?q?=E8=A7=A3=E5=86=B3=E5=A2=9E=E9=87=8F?= =?UTF-8?q?=E4=BB=BB=E5=8A=A1=E6=99=AE=E7=BD=97=E7=B1=B3=E4=BF=AE=E6=96=AF?= =?UTF-8?q?=E6=8C=87=E6=A0=87=E7=A9=BA=E6=8C=87=E9=92=88bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/metrics/CustomPrometheusReporter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java index d460469bc2..fa43732d1e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java @@ -137,7 +137,7 @@ public void open() { public void registerMetric(Accumulator accumulator, String name) { name = Metrics.METRIC_GROUP_KEY_FLINKX + "_" + name; - ReporterScopedSettings reporterScopedSettings = new ReporterScopedSettings(0, ',', null); + ReporterScopedSettings reporterScopedSettings = new ReporterScopedSettings(0, ',', Collections.emptySet()); FrontMetricGroup front = new FrontMetricGroup>(reporterScopedSettings, (AbstractMetricGroup)context.getMetricGroup()); notifyOfAddedMetric(new SimpleAccumulatorGauge<>(accumulator), name, front); } From f2df1032b4fb13b4e667816aa861e8b6fbe8a290 Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 6 May 2020 14:09:02 +0800 Subject: [PATCH 048/136] =?UTF-8?q?=E5=88=A0=E9=99=A4=E5=A4=B1=E6=95=88?= =?UTF-8?q?=E9=85=8D=E7=BD=AE=E8=AF=B4=E6=98=8E=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README_OLD.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README_OLD.md b/README_OLD.md index 6ed77f1821..93dc01fab3 100644 --- a/README_OLD.md +++ b/README_OLD.md @@ -123,7 +123,7 @@ mvn clean package -DskipTests bin/flinkx -mode local \ -job /Users/softfly/company/flink-data-transfer/jobs/task_to_run.json \ -plugin /Users/softfly/company/flink-data-transfer/plugins \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -confProp "{"flink.checkpoint.interval":60000}" \ -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* ``` @@ -134,7 +134,7 @@ bin/flinkx -mode standalone \ -job /Users/softfly/company/flink-data-transfer/jobs/oracle_to_oracle.json \ -plugin /Users/softfly/company/flink-data-transfer/plugins \ -flinkconf /hadoop/flink-1.4.0/conf \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -confProp "{"flink.checkpoint.interval":60000}" \ -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* ``` @@ -146,7 +146,7 @@ bin/flinkx -mode yarn \ -plugin /opt/dtstack/flinkplugin/syncplugin \ -flinkconf /opt/dtstack/myconf/conf \ -yarnconf /opt/dtstack/myconf/hadoop \ - -confProp "{"flink.checkpoint.interval":60000,"flink.checkpoint.stateBackend":"/flink_checkpoint/"}" \ + -confProp "{"flink.checkpoint.interval":60000}" \ -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* ``` From 218cb6717fcddcb5cce40ef6aee4ec447e8b1573 Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 6 May 2020 15:11:47 +0800 Subject: [PATCH 049/136] =?UTF-8?q?=E5=88=A0=E9=99=A4=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E4=BB=A3=E7=A0=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../launcher/perJob/PerJobClusterClientBuilder.java | 8 +------- .../dtstack/flinkx/launcher/perJob/PerJobSubmitter.java | 2 +- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index c6a8538b70..7dfbe91031 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -21,7 +21,6 @@ import com.dtstack.flinkx.options.Options; import org.apache.commons.lang3.StringUtils; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; @@ -77,13 +76,11 @@ public void init(Options launcherOptions, Properties conProp) throws Exception { /** * create a yarn cluster descriptor which is used to start the application master - * @param confProp taskParams * @param launcherOptions LauncherOptions - * @param jobGraph JobGraph * @return * @throws MalformedURLException */ - public YarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options launcherOptions, JobGraph jobGraph) throws MalformedURLException { + public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptions) throws MalformedURLException { String flinkJarPath = launcherOptions.getFlinkLibJar(); if (StringUtils.isNotBlank(flinkJarPath)) { if (!new File(flinkJarPath).exists()) { @@ -92,9 +89,6 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, } else { throw new IllegalArgumentException("The Flink jar path is null"); } - - Configuration conf = new Configuration(); - confProp.forEach((key, value) -> conf.setString(key.toString(), value.toString())); YarnClusterDescriptor descriptor = new YarnClusterDescriptor( flinkConfig, yarnConf, diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java index a401695ea0..491ace1686 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobSubmitter.java @@ -74,7 +74,7 @@ public static String submit(Options launcherOptions, JobGraph jobGraph, String[] PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); perJobClusterClientBuilder.init(launcherOptions, conProp); - YarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, launcherOptions, jobGraph); + YarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(launcherOptions); ClusterClientProvider provider = descriptor.deployJobCluster(clusterSpecification, jobGraph, true); String applicationId = provider.getClusterClient().getClusterId().toString(); String flinkJobId = jobGraph.getJobID().toString(); From 6492ec5d34fb4c1978c572e9e46b0a938ab93f2d Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 6 May 2020 15:20:46 +0800 Subject: [PATCH 050/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DperJob=E6=A8=A1?= =?UTF-8?q?=E5=BC=8F=E6=9C=89=E6=B2=A1=E8=AF=BB=E5=8F=96flink=E7=9B=B8?= =?UTF-8?q?=E5=85=B3=E9=85=8D=E7=BD=AEbug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../perjob/PerJobClusterClientBuilder.java | 27 ++++++++----------- .../launcher/perjob/PerJobSubmitter.java | 2 +- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java index 998ee73516..5d6d17094b 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobClusterClientBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,7 +17,6 @@ */ package com.dtstack.flinkx.launcher.perjob; -import com.dtstack.flinkx.launcher.PluginUtil; import com.dtstack.flinkx.options.Options; import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; @@ -34,7 +33,9 @@ import java.io.File; import java.net.MalformedURLException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; /** * Date: 2019/09/11 @@ -49,14 +50,17 @@ public class PerJobClusterClientBuilder { private YarnConfiguration yarnConf; + private Configuration flinkConfig; + /** * init yarnClient */ - public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properties userConf) throws Exception { - userConf.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); + public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properties conProp) throws Exception { + conProp.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); SecurityUtils.install(new SecurityConfiguration(flinkConfig)); this.yarnConf = yarnConf; + this.flinkConfig = flinkConfig; yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); yarnClient.start(); @@ -66,12 +70,11 @@ public void init(YarnConfiguration yarnConf, Configuration flinkConfig, Properti /** * create a yarn cluster descriptor which is used to start the application master - * @param confProp taskParams * @param options LauncherOptions * @return * @throws MalformedURLException */ - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, Options options) throws MalformedURLException { + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Options options) throws MalformedURLException { String flinkJarPath = options.getFlinkLibJar(); if (StringUtils.isNotBlank(flinkJarPath)) { if (!new File(flinkJarPath).exists()) { @@ -81,10 +84,7 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co throw new IllegalArgumentException("The Flink jar path is null"); } - Configuration conf = new Configuration(); - confProp.forEach((key, value) -> conf.setString(key.toString(), value.toString())); - - AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(conf, yarnConf, options.getFlinkconf(), yarnClient, false); + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(flinkConfig, yarnConf, options.getFlinkconf(), yarnClient, false); descriptor.setName(options.getJobid()); List shipFiles = new ArrayList<>(); @@ -99,11 +99,6 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } } -// if (StringUtils.equalsIgnoreCase(options.getPluginLoadMode(), "shipfile")) { -// List files = PluginUtil.getAllPluginPath(options.getPluginRoot()); -// shipFiles.addAll(files); -// } - if (StringUtils.isNotBlank(options.getQueue())) { descriptor.setQueue(options.getQueue()); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index d42f744a05..d9025e2ed9 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -85,7 +85,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); perJobClusterClientBuilder.init(yarnConf, flinkConfig, conProp); - AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(conProp, options); + AbstractYarnClusterDescriptor descriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(options); ClusterClient clusterClient = descriptor.deployJobCluster(clusterSpecification, new JobGraph(), true); String applicationId = clusterClient.getClusterId().toString(); LOG.info("deploy per_job with appId: {}", applicationId); From ba8a5869de9f1d58195ff181b227a737ea02a64f Mon Sep 17 00:00:00 2001 From: Dulingqiang Date: Mon, 11 May 2020 19:36:27 +0800 Subject: [PATCH 051/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E4=B8=80=E4=B8=AA?= =?UTF-8?q?=E9=94=99=E5=88=AB=E5=AD=97?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/binlog.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/binlog.md b/docs/binlog.md index 7bd60a6563..e425b01c4c 100644 --- a/docs/binlog.md +++ b/docs/binlog.md @@ -1,6 +1,6 @@ # MySQL binlog读取插件(*reader) -## 1.首先给莫个用户赋权,有读binglog的权限 +## 1.首先给某个用户赋权,有读binglog的权限 GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'xxx'@'%' IDENTIFIED BY 'xxx'; From fcb8c99041d0905fb8963eb7fc2b06622053ef63 Mon Sep 17 00:00:00 2001 From: tudou Date: Sat, 16 May 2020 18:14:55 +0800 Subject: [PATCH 052/136] doc update --- README.md | 85 +-- README_CH.md | 85 +-- README_OLD.md | 376 ------------ docs/binlog.md | 196 ------- docs/carbondatareader.md | 197 ------- docs/carbondatawriter.md | 118 ---- docs/cassandrareader.md | 134 ----- docs/cassandrawriter.md | 0 docs/contribution.md | 514 +++++++++++++++++ docs/emqxreader.md | 114 ---- docs/emqxwriter.md | 107 ---- docs/esreader.md | 116 ---- docs/eswriter.md | 139 ----- docs/ftpreader.md | 148 ----- docs/ftpwriter.md | 111 ---- docs/generalconfig.md | 200 +++++++ docs/hbasereader.md | 108 ---- docs/hbasewriter.md | 176 ------ docs/hdfsreader.md | 144 ----- docs/hdfswriter.md | 132 ----- docs/hivewriter.md | 107 ---- docs/kafkareader.md | 82 --- docs/kafkawriter.md | 94 --- docs/kudureader.md | 183 ------ docs/kuduwriter.md | 187 ------ docs/mongodb_oplog.md | 175 ------ docs/mongodbreader.md | 144 ----- docs/mongodbwriter.md | 126 ----- docs/mysqlreader.md | 0 docs/odpsreader.md | 114 ---- docs/odpswriter.md | 92 --- docs/offline/reader/carbondatareader.md | 144 +++++ docs/offline/reader/cassandrareader.md | 164 ++++++ docs/offline/reader/clickhousereader.md | 412 ++++++++++++++ docs/offline/reader/db2reader.md | 420 ++++++++++++++ docs/offline/reader/dmreader.md | 443 +++++++++++++++ docs/offline/reader/esreader.md | 137 +++++ docs/offline/reader/ftpreader.md | 320 +++++++++++ docs/offline/reader/gbasereader.md | 436 ++++++++++++++ docs/offline/reader/hbasereader.md | 152 +++++ docs/offline/reader/hdfsreader.md | 192 +++++++ docs/offline/reader/kudureader.md | 187 ++++++ docs/offline/reader/mongodbreader.md | 146 +++++ docs/offline/reader/mysqlreader.md | 431 ++++++++++++++ docs/offline/reader/odpsreader.md | 134 +++++ docs/offline/reader/oraclereader.md | 445 +++++++++++++++ docs/offline/reader/phoenixreader.md | 533 ++++++++++++++++++ docs/offline/reader/polardbreader.md | 431 ++++++++++++++ docs/offline/reader/postgresqlreader.md | 418 ++++++++++++++ docs/{ => offline/reader}/rdbdreader.md | 0 docs/offline/reader/saphanareader.md | 224 ++++++++ docs/offline/reader/sqlserverreader.md | 469 +++++++++++++++ docs/offline/reader/streamreader.md | 241 ++++++++ docs/offline/reader/teradatareader.md | 216 +++++++ docs/offline/writer/carbondatawriter.md | 139 +++++ docs/offline/writer/cassandrawriter.md | 172 ++++++ docs/offline/writer/clickhousewriter.md | 159 ++++++ docs/offline/writer/db2writer.md | 269 +++++++++ docs/offline/writer/dmwriter.md | 233 ++++++++ docs/offline/writer/eswriter.md | 180 ++++++ docs/offline/writer/ftpwriter.md | 316 +++++++++++ docs/offline/writer/gbasewriter.md | 211 +++++++ docs/offline/writer/hbasewriter.md | 209 +++++++ docs/offline/writer/hdfswriter.md | 389 +++++++++++++ docs/offline/writer/hivewriter.md | 369 ++++++++++++ docs/offline/writer/kuduwriter.md | 194 +++++++ docs/offline/writer/mongodbwriter.md | 309 ++++++++++ docs/offline/writer/mysqlwriter.md | 305 ++++++++++ docs/offline/writer/odpswriter.md | 118 ++++ docs/offline/writer/oraclewriter.md | 230 ++++++++ docs/offline/writer/phoenixwriter.md | 248 ++++++++ docs/offline/writer/polardbwriter.md | 303 ++++++++++ docs/offline/writer/postgresqlwriter.md | 228 ++++++++ docs/{ => offline/writer}/pulsarwriter.md | 0 docs/offline/writer/rediswriter.md | 167 ++++++ docs/offline/writer/saphanawriter.md | 162 ++++++ docs/offline/writer/sqlserverwriter.md | 229 ++++++++ docs/offline/writer/streamwriter.md | 80 +++ docs/offline/writer/teradatawriter.md | 156 +++++ docs/pgwalreader.md | 184 ------ docs/quickstart.md | 248 ++++++++ docs/rdbreader.md | 276 --------- docs/rdbwriter.md | 151 ----- docs/realTime/reader/binlogreader.md | 347 ++++++++++++ docs/realTime/reader/emqxreader.md | 123 ++++ docs/realTime/reader/kafkareader.md | 396 +++++++++++++ docs/realTime/reader/mongodboplogreader.md | 171 ++++++ docs/realTime/reader/pgwalreader.md | 187 ++++++ docs/realTime/writer/emqxwriter.md | 124 ++++ docs/realTime/writer/kafkawriter.md | 388 +++++++++++++ docs/rediswriter.md | 145 ----- docs/restore.md | 8 +- docs/streamreader.md | 60 -- docs/streamwriter.md | 41 -- .../google/common/collect/MapMakerHelper.java | 2 +- .../com/google/common/collect/MigrateMap.java | 2 +- .../flinkx/binlog/reader/BinlogEventSink.java | 2 +- .../binlog/reader/BinlogInputFormat.java | 2 +- .../flinkx/binlog/reader/BinlogReader.java | 2 +- .../reader/CassandraInputFormat.java | 3 +- .../format/ClickhouseInputFormat.java | 15 + .../main/java/com/dtstack/flinkx/Main.java | 18 +- .../dtstack/flinkx/config/AbstractConfig.java | 2 +- .../dtstack/flinkx/config/ContentConfig.java | 2 +- .../flinkx/config/DataTransferConfig.java | 2 +- .../dtstack/flinkx/config/DirtyConfig.java | 2 +- .../flinkx/config/ErrorLimitConfig.java | 2 +- .../com/dtstack/flinkx/config/JobConfig.java | 2 +- .../dtstack/flinkx/config/ReaderConfig.java | 2 +- .../dtstack/flinkx/config/RestartConfig.java | 2 +- .../dtstack/flinkx/config/RestoreConfig.java | 2 +- .../dtstack/flinkx/config/SettingConfig.java | 2 +- .../com/dtstack/flinkx/config/TestConfig.java | 2 +- .../dtstack/flinkx/config/WriterConfig.java | 2 +- .../com/dtstack/flinkx/constants/Metrics.java | 2 +- .../com/dtstack/flinkx/enums/ClusterMode.java | 2 +- .../com/dtstack/flinkx/enums/ColumnType.java | 2 +- .../exception/WriteRecordException.java | 2 +- .../inputformat/BaseRichInputFormat.java | 2 +- .../BaseRichInputFormatBuilder.java | 2 +- .../com/dtstack/flinkx/latch/BaseLatch.java | 2 +- .../com/dtstack/flinkx/latch/LocalLatch.java | 2 +- .../com/dtstack/flinkx/latch/MetricLatch.java | 2 +- .../com/dtstack/flinkx/options/Options.java | 2 +- .../BaseRichOutputFormatBuilder.java | 2 +- .../dtstack/flinkx/reader/BaseDataReader.java | 2 +- .../flinkx/reader/DataReaderFactory.java | 2 +- .../com/dtstack/flinkx/reader/MetaColumn.java | 2 +- .../dtstack/flinkx/restore/FormatState.java | 2 +- .../com/dtstack/flinkx/util/ClassUtil.java | 2 +- .../com/dtstack/flinkx/util/DateUtil.java | 2 +- .../dtstack/flinkx/util/RangeSplitUtil.java | 2 +- .../java/com/dtstack/flinkx/util/RowUtil.java | 2 +- .../com/dtstack/flinkx/util/StringUtil.java | 2 +- .../java/com/dtstack/flinkx/util/SysUtil.java | 2 +- .../com/dtstack/flinkx/util/TelnetUtil.java | 2 +- .../com/dtstack/flinkx/util/ValueUtil.java | 2 +- .../dtstack/flinkx/writer/BaseDataWriter.java | 2 +- .../com/dtstack/flinkx/es/EsConfigKeys.java | 2 +- .../flinkx/es/writer/EsOutputFormat.java | 2 +- .../es/writer/EsOutputFormatBuilder.java | 2 +- flinkx-examples/examples/binlog_to_hdfs.json | 60 -- flinkx-examples/examples/binlog_to_hive.json | 64 --- .../examples/binlog_to_kafka09.json | 41 -- .../examples/binlog_to_kafka10.json | 40 -- .../examples/binlog_to_kafka11.json | 43 -- .../examples/clickhouse_to_clickhouse.json | 91 --- flinkx-examples/examples/dm_to_dm.json | 96 ---- flinkx-examples/examples/es_to_mysql.json | 59 -- flinkx-examples/examples/ftp_to_mysql.json | 59 -- flinkx-examples/examples/hbase_to_mysql.json | 67 --- flinkx-examples/examples/hdfs_to_carbon.json | 67 --- .../examples/hive_orc_to_mysql.json | 66 --- .../examples/kafka09_to_kafka09.json | 43 -- .../examples/kafka10_to_kafka10.json | 43 -- flinkx-examples/examples/kafka11_to_hive.json | 64 --- .../examples/kafka11_to_kafka11.json | 43 -- flinkx-examples/examples/kafka_stream.json | 36 -- .../examples/mongodb_to_mysql.json | 58 -- flinkx-examples/examples/mysql_kafka.json | 77 --- flinkx-examples/examples/mysql_to_es.json | 65 --- flinkx-examples/examples/mysql_to_ftp.json | 57 -- flinkx-examples/examples/mysql_to_hbase.json | 63 --- .../examples/mysql_to_hdfs_orc.json | 90 --- .../examples/mysql_to_mongodb.json | 46 -- flinkx-examples/examples/mysql_to_mysql.json | 78 --- flinkx-examples/examples/mysql_to_odps.json | 62 -- flinkx-examples/examples/mysql_to_oracle.json | 63 --- .../examples/mysql_to_phoenix.json | 72 --- .../examples/mysql_to_postgresql.json | 46 -- flinkx-examples/examples/mysql_to_redis.json | 46 -- flinkx-examples/examples/mysql_to_sftp.json | 66 --- .../examples/mysql_to_sqlserver.json | 64 --- flinkx-examples/examples/odps_to_mysql.json | 64 --- .../examples/oracle_to_hdfs_text.json | 71 --- .../examples/oracle_to_oracle.json | 61 -- flinkx-examples/examples/pgwal_to_stream.json | 42 -- .../examples/phoenix_to_mysql.json | 72 --- .../examples/polarDB_to_polarDb.json | 77 --- .../examples/postgresql_to_mysql.json | 65 --- flinkx-examples/examples/sftp_to_mysql.json | 64 --- .../examples/sqlserver_to_mysql.json | 78 --- flinkx-examples/examples/stream_to_hbase.json | 77 --- flinkx-examples/pom.xml | 22 - .../flinkx/examples/ExampleGenerator.java | 173 ------ .../flinkx/ftp/FtpConfigConstants.java | 2 +- .../com/dtstack/flinkx/ftp/FtpConfigKeys.java | 2 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 2 +- .../com/dtstack/flinkx/ftp/IFtpHandler.java | 2 +- .../com/dtstack/flinkx/ftp/SftpHandler.java | 2 +- .../flinkx/hbase/HbaseConfigConstants.java | 2 +- .../dtstack/flinkx/hbase/HbaseConfigKeys.java | 2 +- .../com/dtstack/flinkx/hbase/HbaseHelper.java | 2 +- .../flinkx/hbase/reader/HbaseInputFormat.java | 2 +- .../hbase/reader/HbaseInputFormatBuilder.java | 2 +- .../dtstack/flinkx/hdfs/ECompressType.java | 2 +- .../dtstack/flinkx/hdfs/HdfsConfigKeys.java | 2 +- .../com/dtstack/flinkx/hdfs/HdfsUtil.java | 8 +- .../apache/hadoop/hive/shims/ShimLoader.java | 2 +- .../hdfs/reader/BaseHdfsInputFormat.java | 2 +- .../hdfs/reader/HdfsInputFormatBuilder.java | 2 +- .../hdfs/reader/HdfsParquetInputFormat.java | 16 +- .../flinkx/hdfs/reader/HdfsReader.java | 2 +- .../hdfs/reader/HdfsTextInputFormat.java | 2 +- .../hdfs/writer/BaseHdfsOutputFormat.java | 2 +- .../hdfs/writer/HdfsOrcOutputFormat.java | 2 +- .../hdfs/writer/HdfsOutputFormatBuilder.java | 2 +- .../hdfs/writer/HdfsParquetOutputFormat.java | 2 +- .../hdfs/writer/HdfsTextOutputFormat.java | 2 +- .../flinkx/hdfs/writer/HdfsWriter.java | 2 +- .../flinkx/hdfs/writer/StringUtil.java | 2 +- .../dtstack/flinkx/hive/ECompressType.java | 2 +- .../com/dtstack/flinkx/hive/EStoreType.java | 2 +- .../dtstack/flinkx/hive/HiveConfigKeys.java | 17 +- .../com/dtstack/flinkx/hive/TableInfo.java | 2 +- .../flinkx/hive/TimePartitionFormat.java | 2 +- .../dtstack/flinkx/hive/util/AddressUtil.java | 2 +- .../dtstack/flinkx/hive/util/DateUtil.java | 2 +- .../dtstack/flinkx/hive/util/HiveDbUtil.java | 2 +- .../dtstack/flinkx/hive/util/HiveUtil.java | 2 +- .../flinkx/hive/util/PathConverterUtil.java | 2 +- .../dtstack/flinkx/hive/util/RetryUtil.java | 5 +- .../org/apache/hive/jdbc/HiveConnection.java | 2 +- .../hive/writer/HiveOutputFormatBuilder.java | 2 +- .../flinkx/hive/writer/HiveWriter.java | 2 +- .../kafka09/reader/Kafka09InputFormat.java | 2 +- .../flinkx/kafka09/reader/Kafka09Reader.java | 2 +- .../kafka10/reader/Kafka10InputFormat.java | 2 +- .../flinkx/kafka10/reader/Kafka10Reader.java | 2 +- .../kafka11/reader/Kafka11InputFormat.java | 2 +- .../flinkx/kafka11/reader/Kafka11Reader.java | 2 +- .../reader/KafkaBaseInputFormat.java | 23 +- .../kafkabase/reader/KafkaBaseReader.java | 6 +- .../flinkx/launcher/ClusterClientFactory.java | 2 +- .../perjob/FlinkPerJobResourceUtil.java | 2 +- .../flinkx/mysql/MySqlDatabaseMeta.java | 2 +- .../flinkx/mysql/format/MysqlInputFormat.java | 5 + .../flinkx/odps/OdpsConfigConstants.java | 2 +- .../dtstack/flinkx/odps/OdpsConfigKeys.java | 2 +- .../com/dtstack/flinkx/odps/OdpsUtil.java | 2 +- .../flinkx/odps/reader/OdpsInputFormat.java | 2 +- .../odps/reader/OdpsInputFormatBuilder.java | 2 +- .../com/dtstack/flinkx/rdb/util/DbUtil.java | 179 +++--- .../QuerySqlBuilder.java | 7 + .../JdbcInputFormat.java | 4 +- .../rdb/outputformat/JdbcOutputFormat.java | 3 + .../flinkx/stream/reader/MockDataUtil.java | 13 + flinkx-test/pom.xml | 370 ++++++++++++ .../com/dtstack/flinkx/test/LocalTest.java | 402 +++++++++++++ .../flinkx/test/PluginNameConstants.java | 84 +++ .../AbstractPrometheusReporter.java | 315 +++++++++++ .../PrometheusPushGatewayReporter.java | 94 +++ pom.xml | 15 +- 253 files changed, 16442 insertions(+), 7572 deletions(-) delete mode 100644 README_OLD.md delete mode 100644 docs/binlog.md delete mode 100644 docs/carbondatareader.md delete mode 100644 docs/carbondatawriter.md delete mode 100644 docs/cassandrareader.md delete mode 100644 docs/cassandrawriter.md create mode 100644 docs/contribution.md delete mode 100644 docs/emqxreader.md delete mode 100644 docs/emqxwriter.md delete mode 100644 docs/esreader.md delete mode 100644 docs/eswriter.md delete mode 100644 docs/ftpreader.md delete mode 100644 docs/ftpwriter.md create mode 100644 docs/generalconfig.md delete mode 100644 docs/hbasereader.md delete mode 100644 docs/hbasewriter.md delete mode 100644 docs/hdfsreader.md delete mode 100644 docs/hdfswriter.md delete mode 100644 docs/hivewriter.md delete mode 100644 docs/kafkareader.md delete mode 100644 docs/kafkawriter.md delete mode 100644 docs/kudureader.md delete mode 100644 docs/kuduwriter.md delete mode 100644 docs/mongodb_oplog.md delete mode 100644 docs/mongodbreader.md delete mode 100644 docs/mongodbwriter.md delete mode 100644 docs/mysqlreader.md delete mode 100644 docs/odpsreader.md delete mode 100644 docs/odpswriter.md create mode 100644 docs/offline/reader/carbondatareader.md create mode 100644 docs/offline/reader/cassandrareader.md create mode 100644 docs/offline/reader/clickhousereader.md create mode 100644 docs/offline/reader/db2reader.md create mode 100644 docs/offline/reader/dmreader.md create mode 100644 docs/offline/reader/esreader.md create mode 100644 docs/offline/reader/ftpreader.md create mode 100644 docs/offline/reader/gbasereader.md create mode 100644 docs/offline/reader/hbasereader.md create mode 100644 docs/offline/reader/hdfsreader.md create mode 100644 docs/offline/reader/kudureader.md create mode 100644 docs/offline/reader/mongodbreader.md create mode 100644 docs/offline/reader/mysqlreader.md create mode 100644 docs/offline/reader/odpsreader.md create mode 100644 docs/offline/reader/oraclereader.md create mode 100644 docs/offline/reader/phoenixreader.md create mode 100644 docs/offline/reader/polardbreader.md create mode 100644 docs/offline/reader/postgresqlreader.md rename docs/{ => offline/reader}/rdbdreader.md (100%) create mode 100644 docs/offline/reader/saphanareader.md create mode 100644 docs/offline/reader/sqlserverreader.md create mode 100644 docs/offline/reader/streamreader.md create mode 100644 docs/offline/reader/teradatareader.md create mode 100644 docs/offline/writer/carbondatawriter.md create mode 100644 docs/offline/writer/cassandrawriter.md create mode 100644 docs/offline/writer/clickhousewriter.md create mode 100644 docs/offline/writer/db2writer.md create mode 100644 docs/offline/writer/dmwriter.md create mode 100644 docs/offline/writer/eswriter.md create mode 100644 docs/offline/writer/ftpwriter.md create mode 100644 docs/offline/writer/gbasewriter.md create mode 100644 docs/offline/writer/hbasewriter.md create mode 100644 docs/offline/writer/hdfswriter.md create mode 100644 docs/offline/writer/hivewriter.md create mode 100644 docs/offline/writer/kuduwriter.md create mode 100644 docs/offline/writer/mongodbwriter.md create mode 100644 docs/offline/writer/mysqlwriter.md create mode 100644 docs/offline/writer/odpswriter.md create mode 100644 docs/offline/writer/oraclewriter.md create mode 100644 docs/offline/writer/phoenixwriter.md create mode 100644 docs/offline/writer/polardbwriter.md create mode 100644 docs/offline/writer/postgresqlwriter.md rename docs/{ => offline/writer}/pulsarwriter.md (100%) create mode 100644 docs/offline/writer/rediswriter.md create mode 100644 docs/offline/writer/saphanawriter.md create mode 100644 docs/offline/writer/sqlserverwriter.md create mode 100644 docs/offline/writer/streamwriter.md create mode 100644 docs/offline/writer/teradatawriter.md delete mode 100644 docs/pgwalreader.md create mode 100644 docs/quickstart.md delete mode 100644 docs/rdbreader.md delete mode 100644 docs/rdbwriter.md create mode 100644 docs/realTime/reader/binlogreader.md create mode 100644 docs/realTime/reader/emqxreader.md create mode 100644 docs/realTime/reader/kafkareader.md create mode 100644 docs/realTime/reader/mongodboplogreader.md create mode 100644 docs/realTime/reader/pgwalreader.md create mode 100644 docs/realTime/writer/emqxwriter.md create mode 100644 docs/realTime/writer/kafkawriter.md delete mode 100644 docs/rediswriter.md delete mode 100644 docs/streamreader.md delete mode 100644 docs/streamwriter.md delete mode 100644 flinkx-examples/examples/binlog_to_hdfs.json delete mode 100644 flinkx-examples/examples/binlog_to_hive.json delete mode 100644 flinkx-examples/examples/binlog_to_kafka09.json delete mode 100644 flinkx-examples/examples/binlog_to_kafka10.json delete mode 100644 flinkx-examples/examples/binlog_to_kafka11.json delete mode 100644 flinkx-examples/examples/clickhouse_to_clickhouse.json delete mode 100644 flinkx-examples/examples/dm_to_dm.json delete mode 100644 flinkx-examples/examples/es_to_mysql.json delete mode 100644 flinkx-examples/examples/ftp_to_mysql.json delete mode 100644 flinkx-examples/examples/hbase_to_mysql.json delete mode 100644 flinkx-examples/examples/hdfs_to_carbon.json delete mode 100644 flinkx-examples/examples/hive_orc_to_mysql.json delete mode 100644 flinkx-examples/examples/kafka09_to_kafka09.json delete mode 100644 flinkx-examples/examples/kafka10_to_kafka10.json delete mode 100644 flinkx-examples/examples/kafka11_to_hive.json delete mode 100644 flinkx-examples/examples/kafka11_to_kafka11.json delete mode 100644 flinkx-examples/examples/kafka_stream.json delete mode 100644 flinkx-examples/examples/mongodb_to_mysql.json delete mode 100644 flinkx-examples/examples/mysql_kafka.json delete mode 100644 flinkx-examples/examples/mysql_to_es.json delete mode 100644 flinkx-examples/examples/mysql_to_ftp.json delete mode 100644 flinkx-examples/examples/mysql_to_hbase.json delete mode 100644 flinkx-examples/examples/mysql_to_hdfs_orc.json delete mode 100644 flinkx-examples/examples/mysql_to_mongodb.json delete mode 100644 flinkx-examples/examples/mysql_to_mysql.json delete mode 100644 flinkx-examples/examples/mysql_to_odps.json delete mode 100644 flinkx-examples/examples/mysql_to_oracle.json delete mode 100644 flinkx-examples/examples/mysql_to_phoenix.json delete mode 100644 flinkx-examples/examples/mysql_to_postgresql.json delete mode 100644 flinkx-examples/examples/mysql_to_redis.json delete mode 100644 flinkx-examples/examples/mysql_to_sftp.json delete mode 100644 flinkx-examples/examples/mysql_to_sqlserver.json delete mode 100644 flinkx-examples/examples/odps_to_mysql.json delete mode 100644 flinkx-examples/examples/oracle_to_hdfs_text.json delete mode 100644 flinkx-examples/examples/oracle_to_oracle.json delete mode 100644 flinkx-examples/examples/pgwal_to_stream.json delete mode 100644 flinkx-examples/examples/phoenix_to_mysql.json delete mode 100644 flinkx-examples/examples/polarDB_to_polarDb.json delete mode 100644 flinkx-examples/examples/postgresql_to_mysql.json delete mode 100644 flinkx-examples/examples/sftp_to_mysql.json delete mode 100644 flinkx-examples/examples/sqlserver_to_mysql.json delete mode 100644 flinkx-examples/examples/stream_to_hbase.json delete mode 100644 flinkx-examples/pom.xml delete mode 100644 flinkx-examples/src/main/java/com/dtstack/flinkx/examples/ExampleGenerator.java create mode 100644 flinkx-test/pom.xml create mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java create mode 100644 flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java create mode 100644 flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java create mode 100644 flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java diff --git a/README.md b/README.md index 0ab6323025..c2a3a77d5e 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ FlinkX is a data synchronization tool based on Flink. FlinkX can collect static - Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [Failure Recovery](docs/restore.md) -- The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/rdbreader.md) +- The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/offline/reader/mysqlreader.md) - Some databases support opening Kerberos security authentication; [Kerberos](docs/kerberos.md) @@ -39,38 +39,57 @@ The following databases are currently supported: | | Database Type | Reader | Writer | |:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| -| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | -| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | -| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | -| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | -| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | -| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | -| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | -| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | -| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | -| | Redis | | [doc](docs/rediswriter.md) | -| | Hive | | [doc](docs/hivewriter.md) | -| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | -| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | -| | MySQL Binlog | [doc](docs/binlog.md) | | -| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | -| | PostgreSQL WAL | [doc](docs/pgwalreader.md) | | - -# Documentation - -[Documentation](https://github.com/DTStack/flinkx/wiki) | [Old Documentation](README_OLD.md) +| Batch Synchronization | MySQL | [doc](docs/offline/reader/mysqlreader.md) | [doc](docs/offline/writer/mysqlwriter.md) | +| | Oracle | [doc](docs/offline/reader/oraclereader.md) | [doc](docs/offline/writer/oraclewriter.md) | +| | SqlServer | [doc](docs/offline/reader/sqlserverreader.md) | [doc](docs/offline/writer/sqlserverwriter.md) | +| | PostgreSQL | [doc](docs/offline/reader/postgresqlreader.md) | [doc](docs/offline/writer/postgresqlwriter.md) | +| | DB2 | [doc](docs/offline/reader/db2reader.md) | [doc](docs/offline/writer/db2writer.md) | +| | GBase | [doc](docs/offline/reader/gbasereader.md) | [doc](docs/offline/writer/gbasewriter.md) | +| | ClickHouse | [doc](docs/offline/reader/clickhousereader.md) | [doc](docs/offline/writer/clickhousewriter.md) | +| | PolarDB | [doc](docs/offline/reader/polardbreader.md) | [doc](docs/offline/writer/polardbwriter.md) | +| | SAP Hana | [doc](docs/offline/reader/saphanareader.md) | [doc](docs/offline/writer/saphanawriter.md) | +| | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | +| | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | +| | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | +| | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | +| | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | +| | MongoDB | [doc](docs/offline/reader/mongodbreader.md) | [doc](docs/offline/writer/mongodbwriter.md) | +| | Kudu | [doc](docs/offline/reader/kudureader.md) | [doc](docs/offline/writer/kuduwriter.md) | +| | ElasticSearch | [doc](docs/offline/reader/esreader.md) | [doc](docs/offline/writer/eswriter.md) | +| | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | +| | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | +| | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Redis | | [doc](docs/offline/writer/rediswriter.md) | +| | Hive | | [doc](docs/offline/writer/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | +| | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | +| | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | +| | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | +| | Oracle Logminer| Coming Soon| | +| | SqlServer CDC | Coming Soon | | + +# Quick Start + +Please click [Quick Start](docs/quickstart.md) + +# General Configuration + +Please click [General Configuration](docs/generalconfig.md) + +# Statistics Metric + +Please click [Statistics Metric](docs/statistics.md) + +# Kerberos + +Please click [Kerberos](docs/kerberos.md) + +# How to contribute FlinkX + +Please click [Contribution](docs/contribution.md) # License diff --git a/README_CH.md b/README_CH.md index 6a8ccd289b..28ac7c27b2 100644 --- a/README_CH.md +++ b/README_CH.md @@ -23,7 +23,7 @@ FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集 - 部分插件支持失败恢复的功能,可以从失败的位置恢复任务,节约运行时间;[失败恢复](docs/restore.md) -- 关系数据库的Reader插件支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/rdbreader.md) +- 关系数据库的Reader插件支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/offline/reader/mysqlreader.md) - 部分数据库支持开启Kerberos安全认证;[Kerberos](docs/kerberos.md) @@ -39,38 +39,57 @@ FlinkX目前支持下面这些数据库: | | Database Type | Reader | Writer | |:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| -| Batch Synchronization | MySQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Oracle | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SqlServer | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PostgreSQL | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | DB2 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | GBase | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | ClickHouse | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | PolarDB | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | SAP Hana | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Teradata | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Phoenix | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | 达梦 | [doc](docs/rdbreader.md) | [doc](docs/rdbwriter.md) | -| | Cassandra | [doc](docs/cassandrareader.md) | [doc](docs/cassandrawriter.md) | -| | ODPS | [doc](docs/odpsreader.md) | [doc](docs/odpswriter.md) | -| | HBase | [doc](docs/hbasereader.md) | [doc](docs/hbasewriter.md) | -| | MongoDB | [doc](docs/mongodbreader.md) | [doc](docs/mongodbwriter.md) | -| | Kudu | [doc](docs/kudureader.md) | [doc](docs/kuduwriter.md) | -| | ElasticSearch | [doc](docs/esreader.md) | [doc](docs/eswriter.md) | -| | FTP | [doc](docs/ftpreader.md) | [doc](docs/ftpwriter.md) | -| | HDFS | [doc](docs/hdfsreader.md) | [doc](docs/hdfswriter.md) | -| | Carbondata | [doc](docs/carbondatareader.md) | [doc](docs/carbondatawriter.md) | -| | Redis | | [doc](docs/rediswriter.md) | -| | Hive | | [doc](docs/hivewriter.md) | -| Stream Synchronization | Kafka | [doc](docs/kafkareader.md) | [doc](docs/kafkawriter.md) | -| | EMQX | [doc](docs/emqxreader.md) | [doc](docs/emqxwriter.md) | -| | MySQL Binlog | [doc](docs/binlog.md) | | -| | MongoDB Oplog | [doc](docs/mongodb_oplog.md) | | -| | PostgreSQL WAL | [doc](docs/pgwalreader.md) | | - -# 参考文档 - -[参考文档](https://github.com/DTStack/flinkx/wiki) | [旧文档](README_OLD.md) +| Batch Synchronization | MySQL | [doc](docs/offline/reader/mysqlreader.md) | [doc](docs/offline/writer/mysqlwriter.md) | +| | Oracle | [doc](docs/offline/reader/oraclereader.md) | [doc](docs/offline/writer/oraclewriter.md) | +| | SqlServer | [doc](docs/offline/reader/sqlserverreader.md) | [doc](docs/offline/writer/sqlserverwriter.md) | +| | PostgreSQL | [doc](docs/offline/reader/postgresqlreader.md) | [doc](docs/offline/writer/postgresqlwriter.md) | +| | DB2 | [doc](docs/offline/reader/db2reader.md) | [doc](docs/offline/writer/db2writer.md) | +| | GBase | [doc](docs/offline/reader/gbasereader.md) | [doc](docs/offline/writer/gbasewriter.md) | +| | ClickHouse | [doc](docs/offline/reader/clickhousereader.md) | [doc](docs/offline/writer/clickhousewriter.md) | +| | PolarDB | [doc](docs/offline/reader/polardbreader.md) | [doc](docs/offline/writer/polardbwriter.md) | +| | SAP Hana | [doc](docs/offline/reader/saphanareader.md) | [doc](docs/offline/writer/saphanawriter.md) | +| | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | +| | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | +| | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | +| | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | +| | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | +| | MongoDB | [doc](docs/offline/reader/mongodbreader.md) | [doc](docs/offline/writer/mongodbwriter.md) | +| | Kudu | [doc](docs/offline/reader/kudureader.md) | [doc](docs/offline/writer/kuduwriter.md) | +| | ElasticSearch | [doc](docs/offline/reader/esreader.md) | [doc](docs/offline/writer/eswriter.md) | +| | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | +| | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | +| | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Redis | | [doc](docs/offline/writer/rediswriter.md) | +| | Hive | | [doc](docs/offline/writer/hivewriter.md) | +| Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | +| | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | +| | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | +| | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | +| | Oracle Logminer| Coming Soon | | +| | SqlServer CDC | Coming Soon | | + +# 快速开始 + +请点击[快速开始](docs/quickstart.md) + +# 通用配置 + +请点击[插件通用配置](docs/generalconfig.md) + +# 统计指标 + +请点击[统计指标](docs/statistics.md) + +# Kerberos + +请点击[Kerberos](docs/kerberos.md) + +# 如何贡献FlinkX + +请点击[如何贡献FlinkX](docs/contribution.md) # License diff --git a/README_OLD.md b/README_OLD.md deleted file mode 100644 index 93dc01fab3..0000000000 --- a/README_OLD.md +++ /dev/null @@ -1,376 +0,0 @@ -## 技术交流 - * 招聘大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com - - * 可以搜索群号30537511或者扫描下面的二维码进入钉钉群 -

- -
- - -## 1 什么是FlinkX - -* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线数据同步框架,实现了多种异构数据源之间高效的数据迁移。** - -不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。 - -
- -
- -## 2 工作原理 - -在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,工作原理如下图: - -
- -
- -## 3 快速起步 - -### 3.1 运行模式 - -* 单机模式:对应Flink集群的单机模式 -* standalone模式:对应Flink集群的分布式模式 -* yarn模式:对应Flink集群的yarn模式 -* yarnPer模式: 对应Flink集群的Per-job模式 - - -### 3.2 执行环境 - -* Java: JDK8及以上 -* Flink集群: 1.4及以上(单机模式不需要安装Flink集群) -* 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 - -### 3.3 打包 - -进入项目根目录,使用maven打包: - -``` -mvn clean package -DskipTests -``` - -打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包 - -### 3.4 启动 - -#### 3.4.1 命令行参数选项 - -* **model** - - * 描述:执行模式,也就是flink集群的工作模式 - * local: 本地模式 - * standalone: 独立部署模式的flink集群 - * yarn: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster" - * yarnPer: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" - * 必选:否 - * 默认值:local - -* **job** - - * 描述:数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息。 - * 必选:是 - * 默认值:无 - -* **pluginRoot** - - * 描述:插件根目录地址,也就是打包后产生的pluginRoot目录。 - * 必选:是 - * 默认值:无 - -* **flinkconf** - - * 描述:flink配置文件所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/conf - * 必选:否 - * 默认值:无 - -* **yarnconf** - - * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop - * 必选:否 - * 默认值:无 - -* **flinkLibJar** - - * 描述:flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/lib - * 必选:否 - * 默认值:无 - -* **confProp** - - * 描述:flink相关参数,如{\"flink.checkpoint.interval\":200000} - * 必选:否 - * 默认值:无 - -* **queue** - - * 描述:yarn队列,如default - * 必选:否 - * 默认值:无 - -* **pluginLoadMode** - - * 描述:yarnPer模式插件加载方式: - * classpath:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快 - * shipfile:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 - * 必选:否 - * 默认值:classpath - -#### 3.4.2 启动数据同步任务 - -* **以本地模式启动数据同步任务** - -``` -bin/flinkx -mode local \ - -job /Users/softfly/company/flink-data-transfer/jobs/task_to_run.json \ - -plugin /Users/softfly/company/flink-data-transfer/plugins \ - -confProp "{"flink.checkpoint.interval":60000}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以standalone模式启动数据同步任务** - -``` -bin/flinkx -mode standalone \ - -job /Users/softfly/company/flink-data-transfer/jobs/oracle_to_oracle.json \ - -plugin /Users/softfly/company/flink-data-transfer/plugins \ - -flinkconf /hadoop/flink-1.4.0/conf \ - -confProp "{"flink.checkpoint.interval":60000}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以yarn模式启动数据同步任务** - -``` -bin/flinkx -mode yarn \ - -job /Users/softfly/company/flinkx/jobs/mysql_to_mysql.json \ - -plugin /opt/dtstack/flinkplugin/syncplugin \ - -flinkconf /opt/dtstack/myconf/conf \ - -yarnconf /opt/dtstack/myconf/hadoop \ - -confProp "{"flink.checkpoint.interval":60000}" \ - -s /flink_checkpoint/0481473685a8e7d22e7bd079d6e5c08c/chk-* -``` - -* **以perjob模式启动数据同步任务** - -``` -bin/flinkx -mode yarnPer \ - -job /test.json \ - -pluginRoot /opt/dtstack/syncplugin \ - -flinkconf /opt/dtstack/flink-1.8.1/conf \ - -yarnconf /opt/dtstack/hadoop-2.7.3/etc/hadoop \ - -flinkLibJar /opt/dtstack/flink-1.8.1/lib \ - -confProp {\"flink.checkpoint.interval\":200000} \ - -queue c -pluginLoadMode classpath -``` - -## 4 数据同步任务模版 - -从最高空俯视,一个数据同步的构成很简单,如下: - -``` -{ - "job": { - "setting": {...}, - "content": [...] - } -} -``` - -数据同步任务包括一个job元素,而这个元素包括setting和content两部分。 - -* setting: 用于配置限速、错误控制和脏数据管理 -* content: 用于配置具体任务信息,包括从哪里来(Reader插件信息),到哪里去(Writer插件信息) - -### 4.1 setting - -``` - "setting": { - "speed": {...}, - "errorLimit": {...}, - "dirty": {...}, - "restart": {...} - } -``` - -setting包括speed、errorLimit和dirty三部分,分别描述限速、错误控制和脏数据管理的配置信息 - -#### 4.1.1 speed - -``` -"speed": { - "bytes": 1048576, - "channel": 2, - "rebalance": false, - "readerChannel": 1, - "writerChannel": 1 -} -``` - -* channel:任务并发数 -* readerChannel:reader的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为reader的并发数。 -* writerChannel:writer的并发数,配置此参数时会覆盖channel配置的并发数,不配置或配置为-1时将使用channel配置的并发数作为writer的并发数。 -* rebalance:此参数配置为true时将强制对reader的数据做Rebalance,不配置此参数或者配置为false时,程序会根据reader和writer的通道数选择是否Rebalance,reader和writer的通道数一致时不使用Reblance,通道数不一致时使用Reblance。 -* bytes::每秒字节数,默认为 Long.MAX_VALUE - -#### 4.1.2 errorLimit - -``` - "errorLimit": { - "record": 10000, - "percentage": 100 - } -``` - -* record: 出错记录数超过record设置的条数时,任务标记为失败 -* percentage: 当出错记录数超过percentage百分数时,任务标记为失败 - -#### 4.1.3 dirty - -``` - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } -``` - -* path: 脏数据存放路径 -* hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置) - -#### 4.1.4 restore - -``` -"restore": { - - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - } -``` - -restore配置请参考[断点续传](docs/restore.md) - -#### 4.1.5 log - -``` -"log" : { - "isLogger": true, - "level" : "warn", - "path" : "/opt/log/", - "pattern":"" - } -``` -* isLogger: 日志是否保存到磁盘, `true`: 是; `false`(默认): 否; -* level: 日志输出级别, `trace`, `debug`, `info`(默认), `warn`, `error`; -* path: 日志保存路径, 默认为`/tmp/dtstack/flinkx/`, 日志名称为当前flink任务的jobID,如: `97501729f8c44c260d889d099968cc74.log` -* pattern: 日志输出格式 - * log4j默认格式为: `%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n`; - * logback默认格式为: `%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n` - -注意:该日志记录功能只会记录`com.dtstack`包下的输出日志, 如需变更,可修改类参数`DtLogger.LOGGER_NAME`。 - -#### 4.1.6 restart - -``` -"restart": { - "strategy": "fixedDelay", - "restartAttempts": 5, - "delayInterval": 10, - "failureRate":2, - "failureInterval":60 - } -``` -* strategy:重启策略,可选:NoRestart、fixedDelay、failureRate,可参考[Flink文档](https://ci.apache.org/projects/flink/flink-docs-stable/dev/task_failure_recovery.html) - -### 4.2 content - -``` - "content": [ - { - "reader": { - "name": "...", - "parameter": { - ... - } - }, - "writer": { - "name": "...", - "parameter": { - ... - } - } - } - ] -``` - -* reader: 用于读取数据的插件的信息 -* writer: 用于写入数据的插件的信息 - -reader和writer包括name和parameter,分别表示插件名称和插件参数 - -### 4.3 数据同步任务例子 - -详见flinkx-examples子工程 - -## 5. 数据同步插件 - -### 5.1 读取插件 - -* [关系数据库读取插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix,达梦)](docs/rdbreader.md) -* [分库分表读取插件](docs/rdbdreader.md) -* [HDFS读取插件](docs/hdfsreader.md) -* [HBase读取插件](docs/hbasereader.md) -* [Elasticsearch读取插件](docs/esreader.md) -* [Ftp读取插件](docs/ftpreader.md) -* [Odps读取插件](docs/odpsreader.md) -* [MongoDB读取插件](docs/mongodbreader.md) -* [Stream读取插件](docs/streamreader.md) -* [Carbondata读取插件](docs/carbondatareader.md) -* [MySQL binlog读取插件](docs/binlog.md) -* [KafKa读取插件](docs/kafkareader.md) -* [Kudu读取插件](docs/kudureader.md) -* [Cassandra读取插件](docs/cassandrareader.md) -* [Emqx读取插件](docs/emqxreader.md) -* [MongoDB实时采集插件](docs/mongodb_oplog.md) -* [PostgreSQL WAL实时采集插件](docs/pgwalreader.md) - -### 5.2 写入插件 - -* [关系数据库写入插件(Mysql,Oracle,Sqlserver,Postgresql,Db2,Gbase,SAP Hana,Teradata,Phoenix)](docs/rdbwriter.md) -* [HDFS写入插件](docs/hdfswriter.md) -* [HBase写入插件](docs/hbasewriter.md) -* [Elasticsearch写入插件](docs/eswriter.md) -* [Ftp写入插件](docs/ftpwriter.md) -* [Odps写入插件](docs/odpswriter.md) -* [MongoDB写入插件](docs/mongodbwriter.md) -* [Redis写入插件](docs/rediswriter.md) -* [Stream写入插件](docs/streamwriter.md) -* [Carbondata写入插件](docs/carbondatawriter.md) -* [Kafka写入插件](docs/kafkawriter.md) -* [Hive写入插件](docs/hivewriter.md) -* [Kudu写入插件](docs/kuduwriter.md) -* [Cassandra写入插件](docs/cassandrawriter.md) -* [Emqx写入插件](docs/emqxwriter.md) - -[断点续传和实时采集功能介绍](docs/restore.md) - -[数据源开启Kerberos](docs/kerberos.md) - -[统计指标说明](docs/statistics.md) - -## 6.版本说明 - - 1.flinkx的分支版本跟flink的版本对应,比如:flinkx v1.5.0 对应 flink1.5.0,版本说明: - -| 插件版本 | flink版本 | -| ----- | ------- | -| 1.5.x | 1.5.4 | -| 1.8.x | 1.8.1 | diff --git a/docs/binlog.md b/docs/binlog.md deleted file mode 100644 index e425b01c4c..0000000000 --- a/docs/binlog.md +++ /dev/null @@ -1,196 +0,0 @@ -# MySQL binlog读取插件(*reader) - -## 1.首先给某个用户赋权,有读binglog的权限 - - GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT - ON *.* TO 'xxx'@'%' IDENTIFIED BY 'xxx'; - -## 2. 配置样例 - -```json -{ - "job": { - "content": [{ - "reader": { - "parameter": { - "jdbcUrl" : "jdbc:mysql://127.0.0.1:3306/test?charset=utf8", - "username" : "username", - "password" : "password", - "host" : "127.0.0.1", - "port": 3306, - "table" : [ "test_sink" ], - "filter" : "", - "cat" : "insert,update,delete", - "start" : { - "journalName" : "bin.000004", - "timestamp" : 123123 - }, - "pavingData" : false, - "bufferSize" : 1024 - }, - "name": "binlogreader" - }, - "writer": { - - } - }], - "setting": { - - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称。 - - * 必选:是 - - * 默认值:无 - -* **jdbcUrl** - - * 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **host** - - * 描述:启动MySQL slave的机器ip。 - - * 必选:是 - - * 默认值:无 - -* **port** - - * 描述:启动MySQL slave的端口 - - * 必选:否 - - * 默认值:3306 - -* **table** - - * 描述:需要解析的数据表。 - - * 注意:指定此参数后filter参数将无效。 - - * 必选:否 - - * 默认值:无 - -* **filter** - - * 描述:过滤表名的Perl正则表达式。 - - * 例子: - - * 所有表:.* or .*\\..* - - * canal schema下所有表: canal\\..* - - * canal下的以canal打头的表:canal\\.canal.* - - * canal schema下的一张表:canal\\.test1 - - * 必选:否 - - * 默认值:无 - -* **cat** - - * 描述:需要解析的数据更新类型,包括insert、update、delete三种。 - - * 注意:以英文逗号分割的格式填写。 - - * 必选:否 - - * 默认值:null - -* **start** - - * 描述:要读取的binlog文件的开始位置。 - - * 参数: - - * journalName:采集起点按文件开始时的文件名称; - - * timestamp:采集七点按时间开始时的时间戳; - - * 默认值:无 - -* **pavingData** - - * 描述:是否将解析出的json数据拍平 - - * 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: - - ```json - { - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before_id":1, - "after_id":2 - } - ``` - - pavingData为false时: - - ```json - { - "message":{ - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before":{ - "id":1 - }, - "after":{ - "id":2 - } - } - } - ``` - - 其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 - - * 必选:否 - - * 默认值:false - -* **bufferSize** - - * 描述:并发缓存大小 - - * 注意:必须为2的幂 - - * 必选:否 - - * 默认值:1024 diff --git a/docs/carbondatareader.md b/docs/carbondatareader.md deleted file mode 100644 index 9911572b43..0000000000 --- a/docs/carbondatareader.md +++ /dev/null @@ -1,197 +0,0 @@ -# Carbondata读取插件(carbondatareader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - }, - "content": [ - { - "reader": { - "name": "carbondatareader", - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/carbon.store1/sb/tb2000", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", - "dfs.nameservices": "ns1" - }, - "defaultFS": "hdfs://ns1", - "table": "tb2000", - "database": "sb", - "filter": " b = 100", - "column": [ - { - "name": "a", - "type": "string" - }, - { - "name": "b", - "type": "int" - } - ] - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "root", - "password": "111111", - "column": [ - "v", - "id" - ], - "batchSize": 1, - "connection": [ - { - "jdbcUrl": "jdbc:mysql://rdos1:3306/hyf", - "table": [ - "tt2" - ] - } - ] - } - } - } - ] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填carbondatareader,否则Flinkx将无法正常加载该插件包。 - * 必选:是
- - * 默认值:无
- -* **path** - - * 描述:carbondata表的存储路径 - - * 必选:是
- - * 默认值:无
- -* **table** - - * 描述:carbondata表名
- - * 必选:否
- - * 默认值:无
- -* **database** - - * 描述:carbondata库名
- - * 必选:否
- - * 默认值:无
- -* **filter** - - * 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value
- - col为列名;
- - op为关系运算符,包括=,>,>=,<,<=;
- - value为字面值,如1234, "ssss"
- - * 必选:否
- - * 默认值:无
- -* **column** - - * 描述:所配置的表中需要同步的字段集合。 - - 字段包括表字段和常量字段,
- - 表字段的格式: - - ``` - { - "name": "col1", - "type": "string" - } - ``` - - 常量字段的格式: - - { - - "value": "12345", - - "type": "string" - - } - - * 必选:是
- - * 默认值:无
- -* **hadoopConfig** - - * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
- - ``` - { - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", - "dfs.nameservices": "ns1", - "fs.defaultFS": "hdfs://ns1" - } - } - ``` - -* **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。
- - * 必选:是
- - * 默认值:无
- -## 3. 数据类型 - -支持如下数据类型 - -* SMALLINT -* INT/INTEGER -* BIGINT -* DOUBLE -* DECIMAL -* FLOAT -* BYTE -* BOOLEAN -* STRING -* CHAR -* VARCHAR -* DATE -* TIMESTAMP - -不支持如下数据类型 - -* arrays: ARRAY -* structs: STRUCT -* maps: MAP diff --git a/docs/carbondatawriter.md b/docs/carbondatawriter.md deleted file mode 100644 index 4fd4dfa89e..0000000000 --- a/docs/carbondatawriter.md +++ /dev/null @@ -1,118 +0,0 @@ -# Cassandra写入插件(cassandrareader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "name": "cassandrawriter", - "parameter": { - "host": "101.37.175.174", - "keyspace": "tp", - "table": "emp", - "column": ["emp_id", "emp_name", "emp_city", "emp_phone", "emp_sal"] - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - } - } -} -``` - -## 2. 参数说明 - -- **host** - - - 描述:数据库地址 - - - 必选:是 - - - 默认值:无 - -- **port** - - - 描述:端口 - - - 必选:否 - - - 默认值:9042 - -- **username** - - - 描述:用户名 - - - 必选:否 - - - 默认值:无 - -- **password** - - - 描述:密码 - - - 必选:否 - - - 默认值:无 - -- **column** - - - 描述:要读取的字段 - - - 必选:否 - - - 默认值:无 - -- **keyspace** - - - 描述:需要同步的表所在的keyspace - - - 必选:是 - - - 默认值:无 - -- **table** - - - 描述:要查询的表 - - - 必选:是 - - - 默认值:无 - -- **asyncWrite** - - - 描述:是否同步写入数据。 - - - 必选:否 - - - 默认值:false - -- **batchSize** - - - 描述:一次批量提交的记录数大小 - - - 必选:否 - - - 默认值:1 - -- **consistancyLevel** - - - 描述:数据一致性级别。可选ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY|TWO|THREE|LOCAL_ONE - - - 必选:否 - - - 默认值:无 diff --git a/docs/cassandrareader.md b/docs/cassandrareader.md deleted file mode 100644 index f3d24702d5..0000000000 --- a/docs/cassandrareader.md +++ /dev/null @@ -1,134 +0,0 @@ -# Cassandra读取插件(cassandrareader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "host": "127.0.0.1", - "port":9042, - "username":"username", - "password":"password", - "useSSL":false, - - "keyspace": "tp", - "table": "emp", - "column": [ - "emp_id", - "emp_city", - "emp_name", - "emp_phone", - "emp_sal" - ], - "allowFiltering":false, - "where":null, - "connectionsPerHost":8, - "maxPendingPerConnection":128, - "consistancyLevel":null - - }, - "name": "cassandrareader" - }, - "writer": { - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - } - } -} -``` - -## 2. 参数说明 - -- **host** - - - 描述:数据库地址 - - - 必选:是 - - - 默认值:无 - -- **port** - - - 描述:端口 - - - 必选:否 - - - 默认值:9042 - -- **username** - - - 描述:用户名 - - - 必选:否 - - - 默认值:无 - -- **password** - - - 描述:密码 - - - 必选:否 - - - 默认值:无 - -- **column** - - - 描述:要读取的字段 - - - 必选:否 - - - 默认值:无 - -- **keyspace** - - - 描述:需要同步的表所在的keyspace - - - 必选:是 - - - 默认值:无 - -- **table** - - - 描述:要查询的表 - - - 必选:是 - - - 默认值:无 - -- **allowFiltering** - - - 描述:是否在服务端过滤数据。 - - - 必选:否 - - - 默认值:false - -- **where** - - - 描述:过滤条件where之后的表达式 - - - 必选:否 - - - 默认值:无 - -- **consistancyLevel** - - - 描述:数据一致性级别。可选ONE|QUORUM|LOCAL_QUORUM|EACH_QUORUM|ALL|ANY|TWO|THREE|LOCAL_ONE - - - 必选:否 - - - 默认值:无 diff --git a/docs/cassandrawriter.md b/docs/cassandrawriter.md deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/docs/contribution.md b/docs/contribution.md new file mode 100644 index 0000000000..2063b96ddd --- /dev/null +++ b/docs/contribution.md @@ -0,0 +1,514 @@ +# 如何贡献FlinkX + +本文面向FlinkX插件开发人员,尝试通过一个开发者的角度尽可能全面地阐述一个FlinkX插件所经历的过程,同时消除开发者的困惑,快速上手插件开发。 + +从数据流的角度来看FlinkX,可以理解为不同数据源的数据流通过对应的FlinkX插件处理,变成符合FlinkX数据规范的数据流;脏数据的处理可以理解为脏水流通过污水处理厂,变成符合标准,可以使用的水流,而对不能处理的水流收集起来。 + +插件开发不需要关注任务具体如何调度,只需要关注关键问题: + +1. 数据源本身读写数据的正确性; +1. 如何合理且正确地使用框架; +1. 配置文件的规范; + +
+## 开发环境 + +- Flink集群: 1.4及以上(单机模式不需要安装Flink集群) +- Java: JDK8及以上 +- 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 + +开发之前,需要理解以下概念: + + +## 逻辑执行概念 +插件开发者不需要关心太多整个框架的具体运行,只需要关注数据源的读写,以及代码在逻辑上是怎么被执行的,方法什么时候被调用的。以下概念的理解对你快速开发会有帮助: + +- **Job**:** Job**是FlinkX用以描述从一个源头到一个目的端的同步作业,是FlinkX数据同步的最小业务单元。 +- **Internal**: 把**Job**拆分得到的最小执行单元。 +- **InputSplit**:数据切片,是进入Internal的最小数据流单位。里面包含了基本数据信息和统计信息。 +- **InputFormat**:读插件的执行单位。 +- **OutputFormat**:写插件的执行单位。 + + +## 任务执行模式 + +- 单机模式:对应Flink集群的单机模式 +- standalone模式:对应Flink集群的分布式模式 +- yarn模式:对应Flink集群的yarn模式 +- yarnPer模式: 对应Flink集群的Per-job模式 + +在实际开发中,上述几种模式对插件的编写没有过多的影响,一般在本地LocalTest通过,将任务上传到Flink集群测试没有什么大问题。 + + +## 插件入口类 +插件的入口类需继承**DataReader**和**DataWriter**,在内部获取任务json传来的参数,通过相应的**Builder**构建对应**InputFormat**和**OutputFormat**实例 + + +### DataReader + +```java +public class SomeReader extends DataReader { + protected String oneParameter; + public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + } + @Override + public DataStream readData() { + return null; + } +} +``` + +reader类需继承DataReader,同时重写readData方法。在构造函数中获取任务json中构建InputFormat所需要的参数,代码案例如下: + +构造方法 + +```java +protected String oneParameter; +public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + // 首先通过jobconfig获取任务json中reader信息 + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + // 通过getParameter()获取相应的参数信息 + oneParameter = readerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER); +} +``` + +重写readData方法 + +```java +@Override +public DataStream readData() { + // 通过Builder构建InputFormat + SomeInputFormatBuilder builder = new SomeInputFormatBuilder(new SomeInputFormat()); + // 一个setOneParameter()方法只set一个参数 + builder.setOneParameter(OneParameter); + //调用createInput返回一个DataStream实例 + return createInput(builder.finish()); +} +``` + + +### DataWriter + +```java +public class SomeWriter extends DataWriter { + protected String oneParameter; + public SomeWriter(DataTransferConfig config) { + super(config); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + return null; + } +} +``` + +和DataReader类似,writer需继承DataWriter,同时重写writeData方法。通常会创建一个ConfigKeys类,包含reader和writer所有需要的使用的任务json中参数的key。 + +构造方法 + +```java +protected String oneParameter; +public SomeWriter(DataTransferConfig config) { + super(config); + // 首先通过jobconfig获取jobjson中writer信息 + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + oneParameter = writerConfig.getParameter().getStringVal(SomeConfigKeys.KEY_PARAMETER); +} +``` + +重写writeData方法 + +```java +@Override +public DataStreamSink writeData(DataStream dataSet) { + // 通过Builder构建OutputFormat + SomeOutputFormatBuilder builder = new SomeOutputFormatBuilder(new SomeOutputFormat()); + // 一个setOneParameter()方法只set一个参数 + builder.setOneParameter(OneParameter); + //调用createInput返回一个DataSink实例 + return createInput(builder.finish()); +} +``` + + +### InputFormatBuilder的设计 + +需继承**RichInputFormatBuilder** + +```java +public class SomeInputFormatBuilder extends RichInputFormatBuilder { + /** + * 首先实例化一个InputFormat实例,通过构造函数传递,通过set方法设置参数 + */ + protected SomeInputFormat format; + //InputFormat构造函数,需要给实例化父类的format + public SomeInputFormatBuilder(SomeInputFormat format){ + super.format = this.format = format; + } + //set方法示例,建议set方法内只给一个变量赋值 + public void setOneParameter(String oneParameter){ + this.oneParameter = oneParameter; + } + //重写checkFormat,检查一些必要参数设置是否正确 + @Override + protected void checkFormat() {} +} +``` + + +### InputFormat的设计 + +需继承**RichInputFormat**,根据任务逻辑分别实现 + +```java +public class SomeInputFormat extends RichInputFormat { + @override + public void openInputFormat() { + + } + + @override + public void closeInputFormat() { + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + return null; + } + + @Override + protected void closeInternal() throws IOException { + } + + @Override + public void configure(Configuration parameters) { + } + + @Override + public InputSplit[] createInputSplits(int minNumSplits) throws IOException { + return new InputSplit[0]; + } + + @Override + public boolean reachedEnd() throws IOException { + return false; + } +} +``` + + +方法功能如下: + +#### configure + +- 调用位置:configure方法会在JobManager里构建执行计划的时候和在TaskManager里初始化并发实例后各调用一次; +- 作用:用于配置task的实例; +- 注意事项:不要在这个方法里写耗时的逻辑,比如获取连接,运行sql等,否则可能会导致akka超 + +#### createInputSplits + +- 调用位置:在构建执行计划时调用; +- 作用:调用子类的逻辑生成数据分片; +- 注意事项:分片的数量和并发数没有严格对应关系,不要在这个方法里做耗时的操作,否则会导致akka超时异常; + +#### getInputSplitAssigner + +- 调用位置:创建分片后调用; +- 作用:获取分片分配器,同步插件里使用的是DefaultInputSplitAssigner,按顺序返回分配给各个并发实例; +- 注意事项:无; + +#### openInternal + +- 调用位置:开始读取分片时调用; +- 作用:用于打开需要读取的数据源,并做一些初始化; +- 注意事项:这个方法必须是可以重复调用的,因为同一个并发实例可能会处理多个分片; + +#### reachEnd和nextRecordInternal + +- 调用位置:任务运行时,读取每条数据时调用; +- 作用:返回结束标识和下一条记录; +- 注意事项:无 + +#### closeInternal + +- 调用位置:读取完一个分片后调用,至少调用一次; +- 作用:关闭资源; +- 注意事项:可重复调用,关闭资源做非null检查,因为程序遇到异常情况可能直接跳转到closeInternal; + +#### openInputFormat + +- 调用位置:创建分片之后调用; +- 作用:对整个InpurFormat资源做初始化; +- 注意事项:无; + +#### closeInputFormat + +- 调用位置:当所有切片都执行完之后调用; +- 作用:关闭整个InputFormat的资源; +- 注意事项:无; + + +### OutputFormatBuilder +需继承**RichOutputFormatBuilder**,和**InputFormatBuilder**相似 + +```java +public class SomeOutputFormatBuilder extends RichOutputFormatBuilder { + /** + * 首先实例化一个OutputFormat实例,通过构造函数传递,通过设计set方法设置参数 + * 如下演示 + */ + protected SomeOutputFormat format; + + public SomeOutputFormatBuilder(SomeOutputFormat format){ + super.format = this.format = format; + } + + public void setOneParameter(String oneParameter){ + this.oneParameter = oneParameter; + } + + //重写checkFormat,检查参数设置是否正确 + @Override + protected void checkFormat() {} +} +``` + + +### OutputFormat +需继承**RichOutputFormat** + +```java +public class SomeOutputFormat extends RichOutputFormat { + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException {} + + @Override + protected void writeSingleRecordInternal(Record record) { + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + } +} +``` + +各方法的执行逻辑如下: + +openInternal -> writeSingleRecordInternal / writeMultipleRecordsInternal + +对于是执行writeSingleRecordInternal 还是writeMultipleRecordsInternal,关键参数是batchInterval,当batchInterval=1 时,框架调用writeSingleRecordInternal;当batchInterval > 1 且 record != null时,则调用writeMultipleRecordsInternal + +方法功能如下: + +#### openInternal + +- 调用位置:开始写入使用 +- 作用:用于打开需要读取的数据源,并做一些初始化; +- 注意事项:无; + +#### writerSingleRecordInternal + +- 调用位置:openInernal之后调用,开始写入数据 +- 作用:向数据源写入一条数据 +- 注意事项:无; + +#### writerMultipleRecordsInternal + +- 调用位置:openInternal之后调用,开始写入多条数据 +- 作用:向数据源写入多条数据,由batchInterval参数决定写入多少条 +- 注意事项:无; + + +## FlinkX数据结构 +FlinkX延续了Flink原生的数据类型Row + +```java +@PublicEvolving +public class Row implements Serializable{ + + private static final long serialVersionUID = 1L; + + /** The array to store actual values. */ + private final Object[] fields; + + /** + * Create a new Row instance. + * @param arity The number of fields in the Row + */ + public Row(int arity) { + this.fields = new Object[arity]; + } +} +``` + + +## 任务json配置 +配置中尽量减少不必要的参数,有些参数框架中已有默认值,配置文件中的值优先,模板如下 + +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "", + "password": "", + "connection": [{ + "jdbcUrl": [""], + "table": [ + "" + ] + }], + "column": [{ + "name": "id", + "type": "int" + },{ + "name":"name", + "type":"string" + }] + }, + "name": "mysqlreader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "errorLimit": { + }, + "speed": { + } + } + } +} +``` + + +## 如何设计配置参数 +任务配置中`reader`和`writer`下`parameter`部分是插件的配置参数,插件的配置参数应当遵循以下原则: + +- 驼峰命名:所有配置项采用驼峰命名法,首字母小写,单词首字母大写。 +- 正交原则:配置项必须正交,功能没有重复,没有潜规则。 +- 富类型:合理使用json的类型,减少无谓的处理逻辑,减少出错的可能。 + - 使用正确的数据类型。比如,bool类型的值使用`true`/`false`,而非`"yes"`/`"true"`/`0`等。 + - 合理使用集合类型,比如,用数组替代有分隔符的字符串。 +- 类似通用:遵守同一类型的插件的习惯,比如关系型数据库的`connection`参数都是如下结构: +``` +{ + "connection": [ + { + "table": [ + "table_1", + "table_2" + ], + "jdbcUrl": [ + "jdbc:mysql://127.0.0.1:3306/database_1", + "jdbc:mysql://127.0.0.2:3306/database_1_slave" + ] + }, + { + "table": [ + "table_3", + "table_4" + ], + "jdbcUrl": [ + "jdbc:mysql://127.0.0.3:3306/database_2", + "jdbc:mysql://127.0.0.4:3306/database_2_slave" + ] + } + ] +} +``` + + +## 如何处理脏数据 + +### 脏数据定义 + +1. Reader读到不支持的类型、不合法的值。 +1. 不支持的类型转换,比如:`Bytes`转换为`Date`。 +1. 写入目标端失败,比如:写mysql整型长度超长。 + + +### 如何处理脏数据 +框架会将脏数据临时存放起来。由DirtyDataManager实例写入临时存放脏数据文件中。 + +- path: 脏数据存放路径 +- hadoopConfig: 脏数据存放路径对应hdfs的配置信息(hdfs高可用配置) + + +## 加载原理 + +1. 框架扫描`plugin/reader`和`plugin/writer`目录,加载每个插件的`plugin.json`文件。 +1. 以`plugin.json`文件中`name`为key,索引所有的插件配置。如果发现重名的插件或者不存在的插件,框架会异常退出。 +1. 用户在插件中在`reader`/`writer`配置的`name`字段指定插件名字。框架根据插件的类型(`reader`/`writer`)和插件名称去插件的路径下扫描所有的jar,加入`classpath`。 +1. 根据插件配置中定义的入口类,框架通过反射实例化对应的`Job`对象。 + + +## 统一的目录结构 + +#### 项目目录层级 +注意,插件Reader/Writer类需放在符合插件包名命名规则的reader下,如MysqlReader类需放在com.dtstack.flinkx.mysql.reader包下,具体命名规则参照 **项目命名规则** 内容 +```xml +``` +${Flinkx_HOME} +|-- bin +| -- flink +| -- flinkx.sh +| +|-- flinkx-somePlugin + |-- flinkx-somePlugin-core + |-- common 一些插件共用的类 + |-- exception 异常处理类 + |-- pom.xml 插件公用依赖 + |-- flinkx-somePlugin-reader + |-- InputFormat + |-- SomePluginInputFormat + |-- SomePluginInputFormatBuiler + |-- reader + |-- SomePluginReader + |-- flinkx-somePlugin-writer + |-- OutputFormat + |-- SomePluginOutputFormat + |-- SomePluginOutputFormatBuiler + |-- reader + |-- SomePluginWriter +``` +``` + +#### 项目命名规则 + +- 插件命名模板 [flinkx]-[dataSourceName],例如flinkx-mysql +- 插件模块命名模板 [flinkx]-[dataSourceName]-[reader/writer/core],例如flinkx-mysql-reader,flinkx-redis-writer +- 插件包名命名模板 [com.dtstack.flinkx.dataSource.xxxx],例如com.dtstack.flinkx.mysql.reader,com.dtstack.flinkx.redis.inputformat +- 插件Reader/Writer类命名模板 [dataSource][Reader/Writer],例如MysqlReader,RedisWriter,需注意,类似RestAPIWriter,MetaDataHive2Reader这样的命名是错误的,需改为RestapiWriter,Metadatahive2Reader + + +## 插件打包 +进入项目根目录,使用maven打包: + +windows平台 + +``` +mvn clean package -DskipTests -Prelease -DscriptType=bat +``` + +unix平台 + +``` +mvn clean package -DskipTests -Prelease -DscriptType=sh +``` + +打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! diff --git a/docs/emqxreader.md b/docs/emqxreader.md deleted file mode 100644 index 52ae367c6a..0000000000 --- a/docs/emqxreader.md +++ /dev/null @@ -1,114 +0,0 @@ -# Emqx读取插件(emqxreader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - "parameter" : { - "broker" : "tcp://impala2:1883", - "topic" : "mqtt/test", - "username" : "root", - "password" : "abc123", - "isCleanSession": true, - "qos": 2, - "codec": "plain" - }, - "name" : "emqxreader" - }, - "writer": { - - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,eqmxreader。 - - * 必选:是 - - * 默认值:无 - -* **broker** - - * 描述:连接URL信息。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:订阅主题 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:认证用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:认证密码 - - * 必选:否 - - * 默认值:无 - -* **isCleanSession** - - * 描述:是否清除session - - * false:MQTT服务器保存于客户端会话的的主题与确认位置 - - * true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - * 必选:否 - - * 默认值:true - -* **qos** - - * 描述:服务质量 - - * 0:AT_MOST_ONCE,至多一次; - - * 1:AT_LEAST_ONCE,至少一次; - - * 2:EXACTLY_ONCE,精准一次; - - * 必选:否 - - * 默认值:2 - -* **codec** - - * 描述:编码解码器类型,支持 json、plain - - * 必选:否 - - * 默认值:plain diff --git a/docs/emqxwriter.md b/docs/emqxwriter.md deleted file mode 100644 index efc33402a5..0000000000 --- a/docs/emqxwriter.md +++ /dev/null @@ -1,107 +0,0 @@ -# Emqx写入插件(emqxwriter) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "writer" : { - "parameter" : { - "broker" : "tcp://impala2:1883", - "topic" : "mqtt/test", - "username" : "root", - "password" : "abc123", - "isCleanSession": true, - "qos": 2 - }, - "name" : "emqxwriter" - } - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,eqmxwriter。 - - * 必选:是 - - * 默认值:无 - -* **broker** - - * 描述:连接URL信息。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:订阅主题 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:认证用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:认证密码 - - * 必选:否 - - * 默认值:无 - -* **isCleanSession** - - * 描述:是否清除session - - * false:MQTT服务器保存于客户端会话的的主题与确认位置 - - * true:MQTT服务器不保存于客户端会话的的主题与确认位置 - - * 必选:否 - - * 默认值:true - -* **qos** - - * 描述:服务质量 - - * 0:AT_MOST_ONCE,至多一次; - - * 1:AT_LEAST_ONCE,至少一次; - - * 2:EXACTLY_ONCE,精准一次; - - * 必选:否 - - * 默认值:2 diff --git a/docs/esreader.md b/docs/esreader.md deleted file mode 100644 index 32c296e8fb..0000000000 --- a/docs/esreader.md +++ /dev/null @@ -1,116 +0,0 @@ -# Elasticsearch读取插件(esreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": { - "name": "esreader", - "parameter": { - "address": "host1:9200,host2:9200", - "query": { - "match": { - "match_all": {} - } - }, - "username": "dtstack", - "password": "abc123", - "index": "indexTest", - "type": "type1", - "batchSize": 0, - "timeout": 10, - "column": [{ - "name": "xx.yy.zz", - "type": "string", - "value": "value" - }] - } - }, - "writer": {} - }] - } -} -``` - -## 2. 参数说明 - -* **address** - - * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:Elasticsearch认证用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:Elasticsearch认证密码 - - * 必选:否 - - * 默认值:无 - -* **query** - - * 描述:Elasticsearch查询表达式,[查询表达式](https://www.elastic.co/guide/cn/elasticsearch/guide/current/query-dsl-intro.html) - - * 必选:否 - - * 默认值:无,默认为全查询 - -* **batchSize** - - * 描述:每次读取数据条数 - - * 必选:否 - - * 默认值:10 - -* **timeout** - - * 描述:连接超时时间 - - * 必选:否 - - * 默认值:无 - -* **index** - - * 描述:要查询的索引名称 - - * 必选:否 - - * 默认值:无 - -* **type** - - * 描述:要查询的类型 - - * 必选:否 - - * 默认值:无 - -* **column** - - * 描述:读取elasticsearch的查询结果的若干个列,每列形式如下 - - * name:字段名称,可使用多级格式查找 - - * type:字段类型,当name没有指定时,则返回常量列,值为value指定 - - * value:常量列的值 - - * 必选:是 - - * 默认值:无 diff --git a/docs/eswriter.md b/docs/eswriter.md deleted file mode 100644 index f71582dff3..0000000000 --- a/docs/eswriter.md +++ /dev/null @@ -1,139 +0,0 @@ -# Elasticsearch写入插件(eswriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": {}, - "writer": { - "name": "eswriter", - "parameter": { - "address": "host1:9200,host2:9200", - "username": "dtstack", - "password": "abc123", - "index": "indexTest", - "type": "type1", - "bulkAction": 100, - "timeout": 100, - "idColumn": [{ - "index": 0, - "type": "int" - }], - "column": [{ - "name": "col1", - "type": "string" - }] - } - } - }] - } -} -``` - -## 2. 参数说明 - -* **address** - - * 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:Elasticsearch认证用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:Elasticsearch认证密码 - - * 必选:否 - - * 默认值:无 - -* **index** - - * 描述:Elasticsearch 索引值 - - * 必选:是 - - * 默认值:无 - -* **type** - - * 描述:Elasticsearch 索引类型 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:写入elasticsearch的若干个列,每列形式如下 - - ``` - { - "name": "列名", - "type": "列类型" - } - ``` - - * 必选:是 - - * 默认值:无 - -* **idColumns** - - * 描述:用于构造文档id的若干个列,每列形式如下 - - * 普通列 - - ``` - { - "index": 0, // 前面column属性中列的序号,从0开始 - "type": "string" 列的类型,默认为string - } - ``` - - * 常数列 - - ``` - { - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string - } - ``` - - * 必选:否 - - * 注意: - - * 如果不指定idColumns属性,则会随机产生文档id - - * 如果指定的字段值存在重复或者指定了常数,按照es的逻辑,同样值的doc只会保留一份 - - * 默认值:无 - -* **bulkAction** - - * 描述:批量写入的记录条数 - - * 必选:是 - - * 默认值:100 - -* **timeout** - - * 描述:连接超时时间,如果bulkAction指定的数值过大,写入数据可能会超时,这时可以配置超时时间 - - * 必选:否 - - * 默认值:无 diff --git a/docs/ftpreader.md b/docs/ftpreader.md deleted file mode 100644 index ba7b5ee95a..0000000000 --- a/docs/ftpreader.md +++ /dev/null @@ -1,148 +0,0 @@ -# FTP读取插件(ftpreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": { - "name": "ftpreader", - "parameter": { - "protocol": "sftp", - "host": "127.0.0.1", - "port": 22, - "username": "username", - "password": "password", - "column": [{ - "index": 0, - "type": "", - "value": "value" - }], - "path": "/upload", - "encoding": "UTF-8", - "fieldDelimiter": ",", - "isFirstLineHeader": true - } - }, - "writer": {} - }] - } -} -``` - -## 2. 参数说明 - -* **protocol** - - * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。 - - * 必选:是 - - * 默认值:无 - -* **host** - - * 描述:ftp服务器地址。 - - * 必选:是 - - * 默认值:无 - -* **port** - - * 描述:ftp服务器端口。 - - * 必选:否 - - * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 - -* **connectPattern** - - * 描述:连接模式(主动模式或者被动模式)。该参数只在传输协议是标准ftp协议时使用,值只能为:PORT (主动),PASV(被动)。两种模式主要的不同是数据连接建立的不同。对于Port模式,是客户端在本地打开一个端口等服务器去连接建立数据连接,而Pasv模式就是服务器打开一个端口等待客户端去建立一个数据连接。 - - * 必选:否 - - * 默认值:PASV - -* **username** - - * 描述:ftp服务器访问用户名。 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:ftp服务器访问密码。 - - * 必选:是 - - * 默认值:无 - -* **path** - - * 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径。 - - * 必选:是 - - * 默认值:/ - -* **column** - - * 描述:需要读取的字段。 - - * 格式:支持2中格式 - - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: - - ``` - "column":["*"] - ``` - - 2.指定具体信息: - - ``` - "column": [{ - "index": 0, - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - * 属性说明: - - * index:字段索引 - - * type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 - - * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - * value:如果没有指定index,则会把value的值作为常量列返回,如果指定了index,当读取的字段的值为null时,会以此value值作为默认值返回 - - * 必选:是 - - * 默认值:无 - -* **fieldDelimiter** - - * 描述:读取的字段分隔符 - - * 必选:是 - - * 默认值:, - -* **encoding** - - * 描述:读取文件的编码配置。 - * 必选:否 - * 默认值:utf-8 - -* **isFirstLineHeader** - - * 描述:首行是否为标题行,如果是则不读取第一行。 - * 必选:否 - * 默认值:false diff --git a/docs/ftpwriter.md b/docs/ftpwriter.md deleted file mode 100644 index e4da51b536..0000000000 --- a/docs/ftpwriter.md +++ /dev/null @@ -1,111 +0,0 @@ -# FTP写入插件(ftpwriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": {}, - "writer": { - "name": "ftpwriter", - "parameter": { - "protocol": "sftp", - "host": "127.0.0.1", - "port": 22, - "username": "username", - "password": "password", - "writeMode": "overwrite", - "path": "/sftp", - "fieldDelimiter": ",", - "connectPattern": "PASV", - "column": [{ - "type": "string" - }] - } - } - }] - } -} -``` - -## 2. 参数说明 - -* **protocol** - - * 描述:ftp服务器协议,目前支持传输协议有ftp和sftp。 - - * 必选:是 - - * 默认值:无 - -* **host** - - * 描述:ftp服务器地址。 - - * 必选:是 - - * 默认值:无 - -* **port** - - * 描述:ftp服务器端口。 - - * 必选:否 - - * 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 - -* **username** - - * 描述:ftp服务器访问用户名。 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:ftp服务器访问密码。 - - * 必选:是 - - * 默认值:无 - -* **connectPattern** - - * 描述:连接模式(主动模式或者被动模式)。该参数只在传输协议是标准ftp协议时使用,值只能为:PORT (主动),PASV(被动)。两种模式主要的不同是数据连接建立的不同。对于Port模式,是客户端在本地打开一个端口等服务器去连接建立数据连接,而Pasv模式就是服务器打开一个端口等待客户端去建立一个数据连接。 - - * 必选:否 - - * 默认值:PASV - -* **path** - - * 描述:FTP文件系统的路径信息,FtpWriter会写入Path目录下属多个文件。 - - * 必选:是 - - * 默认值:无 - -* **writeMode** - - * 描述:FtpWriter写入前数据清理处理模式: - * overwrite,覆盖 - * append,追加 - * 必选:是 - * 默认值:无 - -* **fieldDelimiter** - - * 描述:写入的字段分隔符 - - * 必选:否 - - * 默认值:, - -* **encoding** - - * 描述:读取文件的编码配置。 - * 必选:否 - * 默认值:utf-8 diff --git a/docs/generalconfig.md b/docs/generalconfig.md new file mode 100644 index 0000000000..5b69242bfc --- /dev/null +++ b/docs/generalconfig.md @@ -0,0 +1,200 @@ +# 插件通用配置 + + +## 配置文件 +一个完整的Flinkx任务脚本配置包含 content, setting两个部分。content用于配置任务的输入源与输出源,其中包含reader,writer。而setting则配置任务整体的环境设定,其中包含restore,speed,errorLimit,dirty,log。具体如下所示: +```json +{ + "job" : { + "content" :[{ + "reader" : { + ...... + }, + "writer" : { + ...... + } + }], + "setting" : { + "restore" : { + ...... + }, + "speed" : { + ...... + }, + "errorLimit" : { + ...... + }, + "dirty" : { + ...... + }, + "log" : { + ...... + } + } + } +} +``` + + +| 名称 | | 说明 | 是否必填 | +| --- | --- | --- | --- | +| content | reader | reader插件详细配置 | 是 | +| | writer | writer插件详细配置 | 是 | +| setting | restore | 任务类型及断点续传配置 | 否 | +| | speed | 速率限制 | 否 | +| | errorLimit | 出错控制 | 否 | +| | dirty | 脏数据保存 | 否 | +| | log | 日志记录配置 | 否 | + + +## content配置 + +### reader +reader用于配置数据的输入源,即数据从何而来。具体配置如下所示: + +```json +"reader" : { + "name" : "xxreader", + "parameter" : { + ...... + } +} +``` + +| 名称 | 说明 | 是否必填 | +| --- | --- | --- | +| name | reader插件名称,具体名称参考各数据源配置文档 | 是 | +| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | + + + +### writer +writer用于配置数据的输出源,即数据写往何处。具体配置如下所示: + +```json +"writer" : { + "name" : "xxwriter", + "parameter" : { + ...... + } +} +``` +| 名称 | 说明 | 是否必填 | +| --- | --- | --- | +| name | writer插件名称,具体名称参考各数据源配置文档 | 是 | +| parameter | 数据源配置参数,具体配置参考各数据源配置文档 | 是 | + + + + +## setting配置 + +### restore +restore用于配置同步任务类型(离线同步、实时采集)和断点续传功能。具体配置如下所示: + +```json +"restore" : { + "isStream" : false, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0, + "maxRowNumForCheckpoint" : 10000 +} +``` +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| --- | --- | --- | --- | --- | +| isStream | 是否为实时采集任务 | 否 | false | Boolean | +| isRestore | 是否开启断点续传 | 否 | false | Boolean | +| restoreColumnName | 断点续传字段名称 | 开启断点续传后必填 | 无 | String | +| restoreColumnIndex | 断点续传字段索引ID | 开启断点续传后必填 | -1 | int | +| maxRowNumForCheckpoint | 触发checkpoint数据条数 | 否 | 10000 | int | + + + +### speed +speed用于配置任务并发数及速率限制。具体配置如下所示: + +```json +"speed" : { + "channel": 1, + "bytes": 0 +} +``` +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| --- | --- | --- | --- | --- | +| channel | 任务并发数 | 否 | 1 | int | +| bytes | bytes >0则表示开启任务限速 | 否 | Long.MAX_VALUE | Long | + + + +### errorLimit +errorLimit用于配置任务运行时数据读取写入的出错控制。具体配置如下所示: + +```json +"errorLimit" : { + "record": 100, + "percentage": 10.0 +} +``` +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| --- | --- | --- | --- | --- | +| record | 错误阈值,当错误记录数超过此阈值时任务失败 | 否 | 0 | int | +| percentage | 错误比例阈值,当错误记录比例超过此阈值时任务失败 | 否 | 0.0 | Double | + + + +### dirty +dirty用于配置脏数据的保存,通常与上文出错控制联合使用。具体配置如下所示: + +```json +"dirty" : { + "path" : "xxx", + "hadoopConfig" : { + ...... + } + } +``` +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| --- | --- | --- | --- | --- | +| path | 脏数据保存路径 | 是 | 无 | Sring | +| hadoopConfig | Hadoop相关配置 | 是 | 无 | K-V键值对 | + + +参考模板如下: + +```json +"dirty" : { + "path" : "/user/hive/warehouse/xx.db/xx", + "hadoopConfig" : { + "fs.default.name": "hdfs://0.0.0.0:9000", + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn1" : "0.0.0.0:9000", + "dfs.namenode.rpc-address.ns1.nn2" : "0.0.0.1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.nameservices" : "ns1" + } + } +``` + + +### log +log用于配置Flinkx中定义的插件日志的保存与记录。具体配置如下所示: + +```json +"log" : { + "isLogger": false, + "level" : "info", + "path" : "/tmp/dtstack/flinkx/", + "pattern":"" +} + +``` +| 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | +| --- | --- | --- | --- | --- | +| isLogger | 是否保存日志记录 | 否 | false | Boolean | +| level | 日志级别 | 否 | info | String | +| path | 服务器上日志保存路径 | 否 | /tmp/dtstack/flinkx/ | String | +| pattern | 日志输出格式 | 否 | log4j:%d{yyyy-MM-dd HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n | String | +| | | | logback : %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n | | + + diff --git a/docs/hbasereader.md b/docs/hbasereader.md deleted file mode 100644 index cdf1c68a04..0000000000 --- a/docs/hbasereader.md +++ /dev/null @@ -1,108 +0,0 @@ -# HBase读取插件(hbasereader) - -## 1. 配置样例 - -```json -{ - "job": { - "setting": {}, - "content": [{ - "reader": { - "name": "hbasereader", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "host1,host2,host3", - "zookeeper.znode.parent": "/hbase", - "hbase.security.authentication":"Kerberos", - "hbase.security.authorization":true, - "hbase.master.kerberos.principal":"hbase/node1@TEST.COM", - "hbase.master.keytab.file":"hbase.keytab", - "hbase.regionserver.keytab.file":"hbase.keytab", - "hbase.regionserver.kerberos.principal":"hbase/node1@TEST.COM" - }, - "table": "tableTest", - "encodig": "utf-8", - "column": [{ - "name": "rowkey", - "type": "string" - }, - { - "name": "cf1:id", - "type": "string" - } - ], - "range": { - "startRowkey": "", - "endRowkey": "", - "isBinaryRowkey": true - } - } - }, - "writer": {} - }] - } -} -``` - -## 2. 参数说明 - -* **hbaseConfig** - - * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),开启kerberos的话参考文档[数据源开启Kerberos](kerberos.md) - - * 必选:是 - - * 默认值:无 - -* **encoding** - - * 描述:字符编码 - - * 必选:无 - - * 默认值:utf-8 - -* **table** - - * 描述:hbase表名 - - * 必选:是 - - * 默认值:无 - -* **range** - - * 描述:指定hbasereader读取的rowkey范围。 - - * startRowkey:指定开始rowkey; - - * endRowkey指定结束rowkey; - - * isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey),配置格式如下: - - ``` - "range": { - "startRowkey": "aaa", - "endRowkey": "ccc", - "isBinaryRowkey":false - } - ``` - - * 必选:否 - - * 默认值:无 - -* **column** - - * 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - - * name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; - - * type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 - - * 必选:是 - - * 默认值:无 diff --git a/docs/hbasewriter.md b/docs/hbasewriter.md deleted file mode 100644 index 52a6948578..0000000000 --- a/docs/hbasewriter.md +++ /dev/null @@ -1,176 +0,0 @@ -# HBase写入插件(hbasewriter) - -## 1. 配置样例 - -```json -{ - "job": { - "setting": { - "speed": {}, - "content": [{ - "reader": {}, - "writer": { - "name": "hbasewriter", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "host1,host2,host3", - "zookeeper.znode.parent": "/hbase", - "hbase.security.authentication":"Kerberos", - "hbase.security.authorization":true, - "hbase.master.kerberos.principal":"hbase/node1@TEST.COM", - "hbase.master.keytab.file":"hbase.keytab", - "hbase.regionserver.keytab.file":"hbase.keytab", - "hbase.regionserver.kerberos.principal":"hbase/node1@TEST.COM" - }, - "table": "tableTest", - "rowkeyColumn": [{ - "index": 0, - "type": "string" - }, - { - "value": "_postfix", - "type": "string" - } - ], - "column": [{ - "name": "cf1:id", - "type": "string" - }, - { - "name": "cf1:vv", - "type": "string" - } - ] - } - } - }] - } - } -} -``` - -## 2. 参数说明 - -* **hbaseConfig** - - * 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),开启kerberos的话参考文档[数据源开启Kerberos](kerberos.md) - - * 必选:是 - - * 默认值:无 - -* **table** - - * 描述:hbase表名 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:写入hbase表的若干个列,hbase表的每一列由列簇和列名组成,用":"连接 - - ``` - { - "name": "cf1:id", // 列簇:列名 - "type": "string" // 列类型 - } - ``` - - * 必选:是 - - * 默认值:无 - -* **rowkeyColumn** - - * 描述:用于构造rowkey的描述信息,支持两种格式,每列形式如下 - - * 字符串格式 - - 字符串格式为:$(cf:col),可以多个字段组合:\$(cf:col1)_$(cf:col2), - - 可以使用md5函数:md5($(cf:col)) - - * 数组格式 - - * 普通列 - - ``` - { - "index": 0, // 该列在column属性中的序号,从0开始 - "type": "string" 列的类型,默认为string - } - ``` - - * 常数列 - - ``` - { - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string - } - ``` - - * 必选:否 - - 如果不指定idColumns属性,则会随机产生文档id - - * 默认值:无 - -* **versionColumn** - - * 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: - - ``` - "versionColumn":{ - "index":1 - } - ``` - - 或者 - - ``` - "versionColumn":{ - "value":"123456789" - } - ``` - -* **encoding** - - * 描述:字符编码 - - * 必选:无 - - * 默认值:utf-8 - -* **nullMode** - - * 描述:读取的null值时,如何处理。支持两种方式: - - * (1)skip:表示不向hbase写这列; - - * (2)empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0] - - * 必选:否 - - * 默认值:skip - -* **writeBufferSize** - - * 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求 - - * 必选:否 - - * 默认值:8M - -* **walFlag** - - * 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能。 - - * 必选:否 - - * 默认值:false diff --git a/docs/hdfsreader.md b/docs/hdfsreader.md deleted file mode 100644 index 3ad7e03d0d..0000000000 --- a/docs/hdfsreader.md +++ /dev/null @@ -1,144 +0,0 @@ -# HDFS读取插件(hdfsreader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [{ - "reader": { - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/wujing_test.db/test", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" - }, - "defaultFS": "hdfs://ns1", - "column": [{ - "name": "col1", - "index": 0, - "type": "string", - "value": "", - "format": "" - }], - "fieldDelimiter": "", - "encoding": "utf-8", - "fileType": "orc" - }, - "name": "hdfsreader" - }, - "writer": {} - }], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **path** - - * 描述:要读取的文件路径,多个路径可以用逗号隔开,开启kerberos的话参考文档[数据源开启Kerberos](kerberos.md)。 - - * 必选:是
- - * 默认值:无
- -* **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。
- - * 必选:是
- - * 默认值:无
- -* **fileType** - - * 描述:文件的类型,目前只支持用户配置为"text"、"orc"、“parquet” - - * text:textfile文件格式 - - * orc:orcfile文件格式 - - * parquet:parquet文件格式 - - * 必选:是
- - * 默认值:无
- -* **column** - - * 描述:需要读取的字段。 - - * 格式:支持3中格式 - - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: - - ``` - "column":[*] - ``` - - 2.只指定字段名称: - - ``` - "column":["id","name"] - ``` - - 3.指定具体信息: - - ``` - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - * 属性说明: - - * name:字段名称 - - * index:字段索引,当读取text格式的文件时指定此属性 - - * type:字段类型,可以和数据文件里的字段类型不一样,程序会做一次类型转换 - - * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - * value:如果数据文件里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - -* **fieldDelimiter** - - * 描述:读取的字段分隔符
- - * 注意:在读取text格式文件时需要指定此参数 - - * 必选:否
- - * 默认值:“\001”
- -* **encoding** - - * 描述:读取文件的编码配置。 - * 必选:否 - * 默认值:utf-8 - -* **hadoopConfig** - - * 描述:hadoopConfig里可以配置与Hadoop相关的一些高级参数,比如HA的配置。
- - ``` - "dfs.nameservices": "testDfs", - "dfs.ha.namenodes.testDfs": "namenode1,namenode2", - "dfs.namenode.rpc-address.aliDfs.namenode1": "", - "dfs.namenode.rpc-address.aliDfs.namenode2": "", - "dfs.client.failover.proxy.provider.testDfs": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - } - ``` - - * 必选:否
- - * 默认值:无 diff --git a/docs/hdfswriter.md b/docs/hdfswriter.md deleted file mode 100644 index 0c2a81aae3..0000000000 --- a/docs/hdfswriter.md +++ /dev/null @@ -1,132 +0,0 @@ -# HDFS写入插件(hdfswriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": {}, - "writer": { - "name": "hdfswriter", - "parameter": { - "hadoopConfig": { - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - }, - "defaultFS": "hdfs://ns1", - "fileType": "text", - "fileName": "hello", - "column": [{ - "name": "col1", - "index": 0, - "type": "STRING" - }], - "rowGroupSize": 134217728, - "compress": "SNAPPY", - "path": "/test", - "writeMode": "append", - "fieldDelimiter": "\\001", - "maxFileSize":1073741824‬ - } - } - }] - } -} -``` - -## 2. 参数说明 - -* **defaultFS** - - * 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 - - * 必选:是
- - * 默认值:无
- -* **fileType** - - * 描述:文件的类型,目前只支持用户配置为"text"、"orc"、“parquet” - - * text:textfile文件格式 - - * orc:orcfile文件格式 - - * parquet:parquet文件格式 - - * 必选:是
- - * 默认值:无
- -* **path** - - * 描述:存储到Hadoop hdfs文件系统的路径信息,HdfsWriter会根据并发配置在Path目录下写入多个文件。 - - * 必选:是
- - * 默认值:无
- -* **rowGroupSize** - - * 描述:写入parquet格式文件时指定,表示一个group的大小,如果字段数量很多,并且任务可使用内存有限,使用默认值可能会导致内存溢出,可以通过降低此参数的值来避免内存溢出,如果值很小,则会生产很多小的group,此时通过hive或者spark处理的话会降低效率,因此这个参数的调整要结合具体使用场景。 - - * 必选:否 - - * 默认值:134217728 - -* **column** - - * 描述:写入数据的字段。 - - * name:指定字段名 - - * type:指定字段类型。 - - * 必选:是
- - * 默认值:无
- -* **writeMode** - - * 描述:hdfswriter写入前数据清理处理模式:
- * append,追加 - - * overwrite,覆盖 - * 注意:overwrite模式时会删除写入路径下的所有文件 - * 必选:否 - * 默认值:overwrite - -* **fieldDelimiter** - - * 描述:hdfswriter写入时的字段分隔符,**需要用户保证与创建的Hive表的字段分隔符一致,否则无法在Hive表中查到数据**
- - * 必选:是
- - * 默认值:\\001
- -* **compress** - - * 描述:hdfs文件压缩类型,默认不填写意味着没有压缩。其中:text类型文件支持压缩类型有gzip、bzip2;orc类型文件支持的压缩类型有NONE、SNAPPY(需要用户安装SnappyCodec)。
- - * 必选:否
- - * 默认值:无压缩
- -* **encoding** - - * 描述:写文件的编码配置。
- * 必选:否 - * 默认值:utf-8 - -* **maxFileSize** - - * 描述:hdfs文件最大大小,单位字节 - - * 必须:否 - - * 默认值:1073741824‬(1G) diff --git a/docs/hivewriter.md b/docs/hivewriter.md deleted file mode 100644 index 9d381ec564..0000000000 --- a/docs/hivewriter.md +++ /dev/null @@ -1,107 +0,0 @@ -# Hive写入插件(hivewriter) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "parameter": { - "hadoopConfig": { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "dfs.namenode.rpc-address.ns1.nn2" : "node002:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "node001:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - - "fieldDelimiter": "\u0001", - "encoding": "utf-8", - "fileType": "orc", - - "partitionType" : "MINUTE", - "partition" : "pt", - - "writeMode" : "append", - - "analyticalRules" : "stream_${schema}_${table}_flinkxtest", - "password" : "", - "tablesColumn" : "{\"date_test\":[{\"type\":\"INT\",\"key\":\"before_id\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_id\"},{\"type\":\"DATETIME\",\"key\":\"before_datetime1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"DATETIME\",\"key\":\"after_datetime1\"},{\"type\":\"TIMESTAMP\",\"key\":\"before_timestamp\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"TIMESTAMP\",\"key\":\"after_timestamp\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", - "jdbcUrl" : "jdbc:hive2://node001:10000/data_map", - - "charsetName" : "utf-8", - "username" : "" - }, - "name": "hivewriter" - } - } - ] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名称hivewriter,hivewriter一般结合mysql binlog插件使用,hive插件底层使用的是hdfswriter插件的共,所以需要填写hdfswriter插件需要的参数。hivewriter插件支持同时写入多张表的多个分区,以及自动创建hive表。开启kerberos的话参考文档[数据源开启Kerberos](kerberos.md)。 - - * 必选:是 - - * 默认值:无 - -* **partitionType** - - * 描述:分区类型,包括 DAY、HOUR、MINUTE三种 - - * DAY:天分区 - - * HOUR:小时分区 - - * MINUTE:分钟分区 - - * 必选:是 - - * 默认值:无 - -* **analyticalRules** - - * 描述:表名映射规则。以“stream_\${schema}_\${table}_flinkxtest”为列,创建表时会将规则中的schema和table替换 - - * 必选:否 - - * 默认值:无 - -* **tablesColumn** - - * 描述:写入hive表的表结构信息,示例: - - ```json - { - "date_test": [ //表名 - - { - "type": "INT", - "key": "before_id", - "comment": "" - }, - { - "comment": "", - "type": "INT", - "key": "after_id" - } - ] - } - ``` - - * 必选:是 - - * 默认值:无 diff --git a/docs/kafkareader.md b/docs/kafkareader.md deleted file mode 100644 index 7ff68ddf48..0000000000 --- a/docs/kafkareader.md +++ /dev/null @@ -1,82 +0,0 @@ -# Kafka读取插件(**reader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "topic" : "yxabc", - "codec" : "plain", - "encoding" : "utf-8", - "consumerSettings" : { - "zookeeper.connect" : "127.0.0.1:2181/kafka", - "group.id" : "default", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "smallest" - } - }, - "name": "kafka09reader" - }, - "writer": { - - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,目前支持版本0.9、0.10、0.11、1.0,名称分别为 kafka09writer、kafka10writer、kafka11writer、kafkawriter。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:要消费的topic。 - - * 必选:是 - - * 默认值:无 - -* **encoding** - - * 描述:编码 - - * 必选:否 - - * 默认值:utf-8 - -* **codec** - - * 描述:编码解码器类型,支持 json、plain - - * 必选:否 - - * 默认值:plain - -* **consumerSettings** - - * 描述:kafka连接配置 - - * 必选:是 - - * 默认值:无 diff --git a/docs/kafkawriter.md b/docs/kafkawriter.md deleted file mode 100644 index f4b7d2661a..0000000000 --- a/docs/kafkawriter.md +++ /dev/null @@ -1,94 +0,0 @@ -# Kafka写入插件(**writer) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "parameter": { - "timezone" : "", - - "encoding" : "utf-8", - - "producerSettings" : { - - "zookeeper.connect" : "127.0.0.1:2181/kafka" - - }, - "topic" : "mufeng_est", - - "brokerList" : "172.16.8.107:9092" - - }, - "name": "kafka09writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,目前支持版本0.9、0.10、0.11、1.0,名称分别为 kafka09writer、kafka10writer、kafka11writer、kafkawriter。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:topic。 - - * 必选:是 - - * 默认值:无 - -* **encoding** - - * 描述:编码 - - * 必选:否 - - * 默认值:utf-8 - -* **brokerList** - - * 描述:kafka broker地址列表 - - * 必选:是 - - * 默认值:无 - -* **timezone** - - * 描述:时区 - - * 必选:是 - - * 默认值:无 - -* **producerSettings** - - * 描述:kafka连接配置 - - * 必选:是 - - * 默认值:无 diff --git a/docs/kudureader.md b/docs/kudureader.md deleted file mode 100644 index 6de48bcd1c..0000000000 --- a/docs/kudureader.md +++ /dev/null @@ -1,183 +0,0 @@ -# Kudu读取插件(kudureader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "name": "kudureader", - "parameter": { - "column": [ - { - "name": "id", - "type": "long" - } - ], - "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", - "table": "kudu", - "readMode": "read_latest", - "authentication": "", - "principal": "", - "keytabFile": "", - "workerCount": 2, - "bossCount": 1, - "operationTimeout": 30000, - "adminOperationTimeout": 30000, - "queryTimeout": 30000, - "where": " id >= 1 ", - "batchSizeBytes": 1048576 - } - }, - "writer": {} - } - ], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,kudureader。 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:需要生成的字段。 - - * 属性说明: - - * name:字段名称; - - * type:字段类型; - - * 必选:是 - - * 默认值:无 - -* **masterAddresses** - - * 描述: master节点地址:端口,多个以,隔开。 - - * 必选:是 - - * 默认值:无 - -* **table** - - * 描述: kudu表名。 - - * 必选:是 - - * 默认值:无 - -* **readMode** - - * 描述: kudu读取模式: - - * 1、read_latest - 默认的读取模式。 - 该模式下,服务器将始终在收到请求时返回已提交的写操作。 - 这种类型的读取不会返回快照时间戳,并且不可重复。 - 用ACID术语表示,它对应于隔离模式:“读已提交”。 - - * 2、read_at_snapshot - 该模式下,服务器将尝试在提供的时间戳上执行读取。 - 如果未提供时间戳,则服务器将当前时间作为快照时间戳。 - 在这种模式下,读取是可重复的,即将来所有在相同时间戳记下的读取将产生相同的数据。 - 执行此操作的代价是等待时间戳小于快照的时间戳的正在进行的正在进行的事务,因此可能会导致延迟损失。用ACID术语,这本身就相当于隔离模式“可重复读取”。 - 如果对已扫描tablet的所有写入均在外部保持一致,则这对应于隔离模式“严格可序列化”。 - 注意:当前存在“空洞”,在罕见的边缘条件下会发生,通过这种空洞有时即使在采取措施使写入如此时,它们在外部也不一致。 - 在这些情况下,隔离可能会退化为“读取已提交”模式。 - - * 必选:是 - - * 默认值:无 - -* **authentication** - - * 描述: 认证方式,如:Kerberos。 - - * 必选:否 - - * 默认值:无 - -* **principal** - - * 描述: 用户名。 - - * 必选:否 - - * 默认值:无 - -* **keytabFile** - - * 描述: keytab文件路径。 - - * 必选:否 - - * 默认值:无 - -* **workerCount** - - * 描述: worker线程数。 - - * 必选:否 - - * 默认值:默认为cpu*2 - -* **bossCount** - - * 描述: boss线程数。 - - * 必选:否 - - * 默认值:1 - -* **operationTimeout** - - * 描述: 普通操作超时时间。 - - * 必选:否 - - * 默认值:30000 - -* **adminOperationTimeout** - - * 描述: 管理员操作(建表,删表)超时时间。 - - * 必选:否 - - * 默认值:30000 - -* **queryTimeout** - - * 描述: 连接scan token的超时时间。 - - * 必选:否 - - * 默认值:与operationTimeout一致 - -* **where** - - * 描述: 过滤条件字符串,多个以and连接。 - - * 必选:否 - - * 默认值:无 - -* **batchSizeBytes** - - * 描述: kudu scan一次性最大读取字节数。 - - * 必选:否 - - * 默认值:1048576 diff --git a/docs/kuduwriter.md b/docs/kuduwriter.md deleted file mode 100644 index 0bbb3b0850..0000000000 --- a/docs/kuduwriter.md +++ /dev/null @@ -1,187 +0,0 @@ -# Kudu写入插件(kuduwriter) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "writer": { - "parameter": { - "column": [ - { - "name": "id", - "type": "long" - } - ], - "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", - "table": "kudu", - "writeMode": "insert", - "flushMode": "manual_flush", - "batchInterval": 10000, - "authentication": "", - "principal": "", - "keytabFile": "", - "workerCount": 2, - "bossCount": 1, - "operationTimeout": 30000, - "adminOperationTimeout": 30000, - "queryTimeout": 30000, - "batchSizeBytes": 1048576 - }, - "reader": {} - } - ], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,kuduwriter。 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:需要生成的字段。 - - * 属性说明: - - * name:字段名称; - - * type:字段类型; - - * 必选:是 - - * 默认值:无 - -* **masterAddresses** - - * 描述: master节点地址:端口,多个以,隔开。 - - * 必选:是 - - * 默认值:无 - -* **table** - - * 描述: kudu表名。 - - * 必选:是 - - * 默认值:无 - -* **writeMode** - - * 描述: kudu数据写入模式: - - * 1、insert - - * 2、update - - * 3、upsert - - * 必选:是 - - * 默认值:无 - -* **flushMode** - - * 描述: kudu session刷新模式: - - * 1、auto_flush_sync - - * 2、auto_flush_background - - * 3、manual_flush - - * 必选:否 - - * 默认值:auto_flush_sync - -* **batchInterval** - - * 描述: 单次批量写入数据条数 - - * 必选:否 - - * 默认值:1 - -* **authentication** - - * 描述: 认证方式,如:Kerberos。 - - * 必选:否 - - * 默认值:无 - -* **principal** - - * 描述: 用户名。 - - * 必选:否 - - * 默认值:无 - -* **keytabFile** - - * 描述: keytab文件路径。 - - * 必选:否 - - * 默认值:无 - -* **workerCount** - - * 描述: worker线程数。 - - * 必选:否 - - * 默认值:默认为cpu*2 - -* **bossCount** - - * 描述: boss线程数。 - - * 必选:否 - - * 默认值:1 - -* **operationTimeout** - - * 描述: 普通操作超时时间。 - - * 必选:否 - - * 默认值:30000 - -* **adminOperationTimeout** - - * 描述: 管理员操作(建表,删表)超时时间。 - - * 必选:否 - - * 默认值:30000 - -* **queryTimeout** - - * 描述: 连接scan token的超时时间。 - - * 必选:否 - - * 默认值:与operationTimeout一致 - -* **batchSizeBytes** - - * 描述: kudu scan一次性最大读取字节数。 - - * 必选:否 - - * 默认值:1048576 diff --git a/docs/mongodb_oplog.md b/docs/mongodb_oplog.md deleted file mode 100644 index 7c8c62a410..0000000000 --- a/docs/mongodb_oplog.md +++ /dev/null @@ -1,175 +0,0 @@ -# Mongodb实时采集插件(mongodboplogreader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "name": "mongodboplogreader", - "parameter": { - "hostPorts": "127.0.0.1:30001,127.0.0.1:30002,127.0.0.1:30003", - "username": "root", - "password": "123456", - "database": "admin", - "clusterMode": "REPLICA_SET", - "authenticationMechanism": "SCRAM-SHA-256", - "monitorDatabases": ["test"], - "monitorCollections":[], - "operateType":["insert","update","delete"], - "pavingData":true, - "excludeDocId": false - } - }, - "writer": { - "name": "streamwriter", - "parameter": { - "print": true - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 1048576 - }, - "errorLimit": { - "record": 100 - }, - "restore" : { - "isRestore" : true, - "isStream" : true - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,oraclelogminerreader。 - - * 必选:是 - - * 默认值:无 - -* **hostPorts** - - * 描述:Mongodb集群地址。 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述: 用户名。 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述: 密码。 - - * 必选:是 - - * 默认值:无 - -* **authenticationMechanism** - - - 描述: 认证机制,可选:GSSAPI、PLAIN、MONGODB-X509、MONGODB-CR、SCRAM-SHA-1、SCRAM-SHA-256 - - - 必选:否 - - - 默认值:无 - -* **clusterMode** - - - 描述: 集群模式,可选:REPLICA_SET、MASTER_SLAVE - - - 必选:是 - - - 默认值:无 - -* **monitorDatabases** - - - 描述: 要监听的库 - - - 必选:否 - - - 默认值:无 - -* **monitorCollections** - - * 描述:要监听的集合 - - * 必选:否 - - * 默认值:无 - -* **operateType** - - * 描述:要监听的操作类型,可选:insert、update、delete - - * 必选:否 - - * 默认值:无 - -* **excludeDocId** - - * 描述:是否排除_id字段 - - * 必选:否 - - * 默认值:false - -* **pavingData** - - * 描述:是否将解析出的json数据拍平 - - * 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: - - ```json - { - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before_id":1, - "after_id":2 - } - ``` - - pavingData为false时: - - ```json - { - "message":{ - "type":"update", - "schema":"test", - "table":"tb1", - "ts":1231232, - "ingestion":123213, - "before_id":{ - "id":1 - }, - "after_id":{ - "id":2 - } - } - } - ``` - - 其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 - - * 必选:否 - - * 默认值:false \ No newline at end of file diff --git a/docs/mongodbreader.md b/docs/mongodbreader.md deleted file mode 100644 index 72f74cac66..0000000000 --- a/docs/mongodbreader.md +++ /dev/null @@ -1,144 +0,0 @@ -# MongoDB读取插件(mongodbreader) - -## 1. 配置样例 - -```json -{ - "job":{ - "content":[{ - "reader":{ - "parameter":{ - "hostPorts":"localhost:27017", - "username": "", - "password": "", - "database":"", - "collectionName": "", - "fetchSize":100, - "column": [ - { - "name":"id", - "type":"int", - "splitter":"," - } - ], - "filter": "" - }, - "name":"mongodbreader" - }, - "writer":{} - }] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填mongodbreader,否则Flinkx将无法正常加载该插件包。 - - * 必选:是 - - * 默认值:无 - -* **hostPorts** - - * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:否 - - * 默认值:无 - -* **database** - - * 描述:数据库名称 - - * 必选:是 - - * 默认值:无 - -* **collectionName** - - * 描述:集合名称 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:需要读取的字段。 - - * 格式:支持3中格式 - - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: - - ``` - "column":[*] - ``` - - 2.只指定字段名称: - - ``` - "column":["id","name"] - ``` - - 3.指定具体信息: - - ``` - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value", - "splitter":"," - }] - ``` - - * 属性说明: - - * name:字段名称 - - * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - * splitter:因为 MongoDB 支持数组类型,所以 MongoDB 读出来的数组类型要通过这个分隔符合并成字符串 - - * 必选:是 - - * 默认值:无 - -* **fetchSize** - - * 描述:每次读取的数据条数,通过调整此参数来优化读取速率 - - * 必选:否 - - * 默认值:100 - -* **filter** - - * 描述:过滤条件,通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) - - * 必选:否 - - * 默认值:无 diff --git a/docs/mongodbwriter.md b/docs/mongodbwriter.md deleted file mode 100644 index 838b34e412..0000000000 --- a/docs/mongodbwriter.md +++ /dev/null @@ -1,126 +0,0 @@ -# MongoDB写入插件(mongodbwriter) - -## 1. 配置样例 - -```json -{ - "job":{ - "content":[{ - "reader":{}, - "writer":{ - "parameter":{ - "hostPorts":"localhost:27017", - "username": "", - "password": "", - "database":"test", - "collectionName": "test", - "writeMode": "insert", - "batchSize":1, - "column": [ - { - "name":"id", - "type":"int", - "splitter":"," - }, - { - "name":"id", - "type":"string", - "splitter":"," - } - ], - "replaceKey":"id" - }, - "name":"mongodbwriter" - } - }] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填 mongodbwriter,否则Flinkx将无法正常加载该插件包。 - - * 必选:是 - - * 默认值:无 - -* **hostPorts** - - * 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔。 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:否 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:否 - - * 默认值:无 - -* **database** - - * 描述:数据库名称 - - * 必选:是 - - * 默认值:无 - -* **collectionName** - - * 描述:集合名称 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列。 - - - name:Column 的名字。 - - type:Column 的类型。 - - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中。 - - * 必选:是 - - * 默认值:无 - -* **replaceKey** - - * 描述:replaceKey 指定了每行记录的业务主键,用来做覆盖时使用(不支持 replaceKey为多个键,一般是指Monogo中的主键)。 - - * 必选:否 - - * 默认值:无 - -* **writeMode** - - * 描述:写入模式,当 batchSize > 1 时不支持 replace 和 update 模式 - - * 必选:是 - - * 所有选项:insert/replace/update - - * 默认值:insert - -* **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与MongoDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。
- - * 必选:否 - - * 默认值:1 diff --git a/docs/mysqlreader.md b/docs/mysqlreader.md deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/docs/odpsreader.md b/docs/odpsreader.md deleted file mode 100644 index 6f0d838a6d..0000000000 --- a/docs/odpsreader.md +++ /dev/null @@ -1,114 +0,0 @@ -# ODPS读取插件(odpsreader) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "writer": {}, - "reader": { - "name": "odpsreader", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tableTest", - "partition": "pt='xxooxx'", - "column": [{ - "name": "col1", - "type": "string", - “value”:"xx", - "format":"yyyy-MM-dd HH:mm:ss" - - }] - } - } - }] - } -} -``` - -## 2. 参数说明 - -* **accessId** - - * 描述:ODPS系统登录ID
- * 必选:是 - * 默认值:无 - -* **accessKey** - - * 描述:ODPS系统登录Key
- * 必选:是 - * 默认值:无 - -* **project** - - * 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无 - -* **table** - - * 描述:读取数据表的表名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
- -* **partition** - - * 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou pt=1,ds=shanghai pt=2,ds=hangzhou pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":["pt=1,ds=shanghai"]`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":["pt=1,ds=* "]`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":["pt=*,ds=*"]`
- - * 必选:如果表为分区表,则必填。如果表为非分区表,则不能填写
- - * 默认值:无
- -* **column** - - * 描述:需要读取的字段。 - - * 格式:支持3中格式 - - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: - - ``` - "column":[*] - ``` - - 2.只指定字段名称: - - ``` - "column":["id","name"] - ``` - - 3.指定具体信息: - - ``` - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - * 属性说明: - - * name:字段名称 - - * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - * 必选:是 - - * 默认值:无 diff --git a/docs/odpswriter.md b/docs/odpswriter.md deleted file mode 100644 index baddb0f64e..0000000000 --- a/docs/odpswriter.md +++ /dev/null @@ -1,92 +0,0 @@ -# ODPS写入插件(odpswriter) - -## 1. 配置样例 - -``` -{ - "job": { - "setting": {}, - "content": [{ - "reader": {}, - "writer": { - "name": "odpswriter", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tableTest", - "partition": "pt='xx'", - "writeMode": "append", - "bufferSize": 64, - "column": [{ - "name": "col1", - "type": "string" - }] - } - } - }] - } -} -``` - -## 2. 参数说明 - -* **accessId** - - * 描述:ODPS系统登录ID
- * 必选:是 - * 默认值:无 - -* **accessKey** - - * 描述:ODPS系统登录Key
- * 必选:是 - * 默认值:无 - -* **project** - - * 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无 - -* **table** - - * 描述:读取数据表的表名称(大小写不敏感)
- - * 必选:是
- - * 默认值:无
- -* **partition** - - * 描述:需要写入数据表的分区信息,必须指定到最后一级分区。把数据写入一个三级分区表,必须配置到最后一级分区,例如pt=20150101/type=1/biz=2。 - - * 必选:**如果是分区表,该选项必填,如果非分区表,该选项不可填写。** - - * 默认值:空
- -* **column** - - * 描述:需要导入的字段列表,当导入全部字段时,可以配置为"column": ["*"], 当需要插入部分odps列填写部分列,例如"column": ["id", "name"]。ODPSWriter支持列筛选、列换序,例如表有a,b,c三个字段,用户只同步c,b两个字段。可以配置成["c","b"], 在导入过程中,字段a自动补空,设置为null。
- * 必选:否
- * 默认值:无
- -* **writeMode** - - * 描述:写入模式,支持append和overwrite - - * 必填:否 - - * 默认值:append - -* **bufferSize** - - * 描述:写入缓存大小,单位兆,odps写入数据时会先缓存,达到一定值后才会写入数据,如果写入数据时出现内存溢出,可以降低此参数的值。 - - * 必填:否 - - * 默认值:64 diff --git a/docs/offline/reader/carbondatareader.md b/docs/offline/reader/carbondatareader.md new file mode 100644 index 0000000000..7c3ab76cc7 --- /dev/null +++ b/docs/offline/reader/carbondatareader.md @@ -0,0 +1,144 @@ +# Carbondata Reader + + +## 一、插件名称 +名称:**carbondatareader**
** + +## 二、支持的数据源版本 +**Carbondata 1.5及以上**
+ + +## 三、参数说明 + +- **path** + - 描述:carbondata表的存储路径 + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:carbondata表名 + - 必选:否 + - 默认值:无 + + + +- **database** + - 描述:carbondata库名 + - 必选:否 + - 默认值:无 + + + +- **filter** + - 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value,col为列名;op为关系运算符,包括=,>,>=,<,<=; +value为字面值,如1234, "ssss" + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:所配置的表中需要同步的字段集合。
+字段包括表字段和常量字段, + + +表字段的格式: +``` +{ + "name": "col1", + "type": "string" +} +``` + + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:是 + - 默认值:无 + + + +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址。 + - 必选:是 + - 默认值:无 + + + + +## 四、使用示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "carbondatareader", + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/carbon.store1/sb/tb2000", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", + "dfs.nameservices": "ns1" + }, + "defaultFS": "hdfs://ns1", + "table": "tb2000", + "database": "sb", + "filter": " b = 100", + "column": [ + { + "name": "a", + "type": "string" + }, + { + "name": "b", + "type": "int" + } + ] + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` + +## diff --git a/docs/offline/reader/cassandrareader.md b/docs/offline/reader/cassandrareader.md new file mode 100644 index 0000000000..4291d6f435 --- /dev/null +++ b/docs/offline/reader/cassandrareader.md @@ -0,0 +1,164 @@ +# Cassandra Reader + + +## 一、插件名称 +**名称:cassandrareader**
+ + +## 二、支持的数据源版本 +**Cassandra 3.0及以上**
+ + +## 三、参数说明 + +- **host** + - 描述:数据库地址 + - 必选:是 + - 默认值:无 + + + +- **port** + - 描述:端口 + - 必选:否 + - 默认值:9042 + + + +- **username** + - 描述:用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:密码 + - 必选:否 + - 默认值:无 + + + +- **useSSL** + - 描述:数字证书 + - 必选:否 + - 默认值:false + + + +- **column** + - 描述:查询结果中被select出来的属性集合,为空则select * + - 必选:否 + - 默认值:无 + + + +- **keyspace** + - 描述:需要同步的表所在的keyspace + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:要查询的表 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:过滤条件where之后的表达式 + - 必选:否 + - 默认值:无 + + + +- **allowFiltering** + - 描述:是否在服务端过滤数据 + - 必选:否 + - 默认值:false + + + +- **connecttionsPerHost** + - 描述:分配给每个host的连接数 + - 必选:否 + - 默认值:8 + + + +- **maxPendingPerConnection** + - 描述:最多能建立的连接数 + - 必选:否 + - 默认值:128 + + + +- **consistancyLevel** + - 描述:数据一致性级别。可选`ONE`、`QUORUM`、`LOCAL_QUORUM`、`EACH_QUORUM`、`ALL`、`ANY`、`TWO`、`THREE`、`LOCAL_ONE` + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "cassandrareader", + "parameter": { + "host": "kudu3", + "port": 9042, + "username":"", + "password":"", + "useSSL":false, + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/reader/clickhousereader.md b/docs/offline/reader/clickhousereader.md new file mode 100644 index 0000000000..67218c00dd --- /dev/null +++ b/docs/offline/reader/clickhousereader.md @@ -0,0 +1,412 @@ +# ClickHouse Reader + + +## 一、插件名称 +名称:**clickhousereader** + +## 二、支持的数据源版本 +**ClickHouse 19.x及以上** + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[clickhouse-jdbc官方文档](https://github.com/ClickHouse/clickhouse-jdbc) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段严格对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会报错。如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select id from tableTest", + "requestAccumulatorInterval": 2 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "requestAccumulatorInterval": 2 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2, + "polling": true, + "pollingInterval": 3000 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + + diff --git a/docs/offline/reader/db2reader.md b/docs/offline/reader/db2reader.md new file mode 100644 index 0000000000..d04953cf60 --- /dev/null +++ b/docs/offline/reader/db2reader.md @@ -0,0 +1,420 @@ +# DB2 Reader + + +## 一、插件名称 +名称:**db2reader** + +## 二、支持的数据源版本 +**DB2 9、10**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[db2官方文档](https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.1.0/com.ibm.db2.luw.apdv.java.doc/src/tpc/imjcc_rjv00004.htmlId=t14:12:14) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:空 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:空 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +** + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + },{ + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "user", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], + "table" : [ "staff" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + },{ + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "user", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], + "table" : [ "staff" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + },{ + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "user", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], + "table" : [ "staff" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql":"select id, name from staff where id > 300", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + },{ + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "user", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], + "table" : [ "staff" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "increColumn": "id", + "startLocation": "340", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + },{ + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "user", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], + "table" : [ "staff" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "polling": true, + "pollingInterval": 3000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` diff --git a/docs/offline/reader/dmreader.md b/docs/offline/reader/dmreader.md new file mode 100644 index 0000000000..79639771f2 --- /dev/null +++ b/docs/offline/reader/dmreader.md @@ -0,0 +1,443 @@ +# DM Reader + + +## 一、插件名称 +名称:**dmreader** + +## 二、支持的数据源版本 +**DM7、DM8**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[达梦官方文档](http://www.dameng.com/down.aspx?TypeId=12&FId=t14:12:14) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:3000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +```json +"column":["ID","NAME"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "COL", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称,注意应该为大写,否则可能会出错 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "increColumn": "", + "startLocation": "", + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": [ + "jdbc:dm://localhost:5236" + ], + "table": [ + "PERSON.STUDENT" + ] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "splitPk": "ID", + "increColumn": "", + "startLocation": "", + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": [ + "jdbc:dm://localhost:5236" + ], + "table": [ + "PERSON.STUDENT" + ] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "increColumn": "", + "startLocation": "", + "customSql": "SELECT * FROM PERSON.STUDENT WHERE ID>30", + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": [ + "jdbc:dm://localhost:5236" + ], + "table": [ + "PERSON.STUDENT" + ] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 4、增量同步startLocation +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "increColumn": "ID", + "startLocation": "20", + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": [ + "jdbc:dm://localhost:5236" + ], + "table": [ + "PERSON.STUDENT" + ] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "increColumn": "", + "startLocation": "", + "username": "SYSDBA", + "password": "SYSDBA", + "polling": true, + "pollingInterval": 3000, + "connection": [ + { + "jdbcUrl": [ + "jdbc:dm://localhost:5236" + ], + "table": [ + "PERSON.STUDENT" + ] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` diff --git a/docs/offline/reader/esreader.md b/docs/offline/reader/esreader.md new file mode 100644 index 0000000000..c9916083d5 --- /dev/null +++ b/docs/offline/reader/esreader.md @@ -0,0 +1,137 @@ +# ElasticSearch Reader + + +## 一、插件名称 +名称:**esreader** + +## 二、支持的数据源版本 +**Elasticsearch 6.X** + +## 三、参数说明
+ +- **address** + - 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:Elasticsearch认证用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:Elasticsearch认证密码 + - 必选:否 + - 默认值:无 + + + +- **query** + - 描述:Elasticsearch查询表达式,[查询表达式](https://www.elastic.co/guide/cn/elasticsearch/guide/current/query-dsl-intro.html) + - 必选:否 + - 默认值:无,默认为全查询 + + + +- **batchSize** + - 描述:每次读取数据条数 + - 必选:否 + - 默认值:10 + + + +- **timeout** + - 描述:连接超时时间 + - 必选:否 + - 默认值:无 + + + +- **index** + - 描述:要查询的索引名称 + - 必选:否 + - 默认值:无 + + + +- **type** + - 描述:要查询的类型 + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:读取elasticsearch的查询结果的若干个列,每列形式如下 + - name:字段名称,可使用多级格式查找 + - type:字段类型,当name没有指定时,则返回常量列,值为value指定 + - value:常量列的值 + - 必选:是 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "esreader", + "parameter": { + "address": "kudu4:9200", + "query": { + "match_all": {} + }, + "index": "tudou", + "type": "doc", + "batchSize": 1000, + "username": "elastic", + "password": "abc123", + "timeout": 10, + "column": [ + { + "name": "id", + "type": "integer" + },{ + "name": "user_id", + "type": "integer" + },{ + "name": "name", + "type": "string" + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 0, + "percentage" : 0 + }, + "speed" : { + "bytes" : 1048576, + "channel" : 1 + } + } + } +} +``` diff --git a/docs/offline/reader/ftpreader.md b/docs/offline/reader/ftpreader.md new file mode 100644 index 0000000000..e687c6e84d --- /dev/null +++ b/docs/offline/reader/ftpreader.md @@ -0,0 +1,320 @@ +# FTP Reader + + +## 一、插件名称 +名称:**ftpreader**
+ + +## 二、数据源版本 +| 协议 | 是否支持 | +| --- | --- | +| FTP | 支持 | +| SFTP | 支持 | + + + + +## 三、数据源配置 +FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
+ + +## 四、参数说明 + +- **protocol** + - 描述:ftp服务器协议,目前支持传输协议有`ftp`、`sftp` + - 必选:是 + - 默认值:无 + + + +- **host** + - 描述:ftp服务器地址 + - 必选:是 + - 默认值:无 + + + +- **port** + - 描述:ftp服务器端口 + - 必选:否 + - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 + + + +- **connectPattern** + - 描述:协议为ftp时的连接模式,可选`pasv`,`port`,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) + - 必选:否 + - 默认值:`PASV` + + + +- **username** + - 描述:ftp服务器访问用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:ftp服务器访问密码 + - 必选:否 + - 默认值:无 + + + +- **path** + - 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径 + - 必选:是 + - 默认值:无 + + + +- **fieldDelimiter** + - 描述:读取的字段分隔符 + - 必选:是 + - 默认值:`,` + + + +- **encoding** + - 描述:读取文件的编码配置 + - 必选:否 + - 默认值:`UTF-8` + + + +- **isFirstLineHeader** + - 描述:首行是否为标题行,如果是则不读取第一行 + - 必选:否 + - 默认值:false + + + +- **timeout** + - 描述:连接超时时间,单位毫秒 + - 必选:否 + - 默认值:5000 + + + +- **column** + - 描述:需要读取的字段 + - 格式:支持2中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +``` +"column":["*"] +``` + +2.指定具体信息: +``` +"column": [{ + "index": 0, + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - index:字段索引 + - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果没有指定index,则会把value的值作为常量列返回,如果指定了index,当读取的字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + + +## 五、使用示例 + +#### 1、读取单个文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "/data/ftp/flinkx/file1.csv", + "protocol": "sftp", + "port": 22, + "isFirstLineHeader": true, + "host": "localhost", + "column": [ + { + "index": 0, + "type": "string" + }, + { + "index": 1, + "type": "string" + }, + { + "index": 2, + "type": "int" + }, + { + "index": 3, + "type": "int" + } + ], + "password": "pass", + "fieldDelimiter": ",", + "encoding": "utf-8", + "username": "user" + }, + "name": "ftpreader" + }, + "writer": { + "parameter": {}, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 2、读取单个目录下的所有文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "/data/ftp/flinkx/dir1", + "protocol": "sftp", + "port": 22, + "isFirstLineHeader": true, + "host": "localhost", + "column": [ + { + "index": 0, + "type": "string" + }, + { + "index": 1, + "type": "string" + }, + { + "index": 2, + "type": "int" + }, + { + "index": 3, + "type": "int" + } + ], + "password": "pass", + "fieldDelimiter": ",", + "encoding": "utf-8", + "username": "user" + }, + "name": "ftpreader" + }, + "writer": { + "parameter": {}, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 3、读取多个路径下的文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "/data/ftp/flinkx/dir1,/data/ftp/flinkx/dir2", + "protocol": "sftp", + "port": 22, + "isFirstLineHeader": true, + "host": "localhost", + "column": [ + { + "index": 0, + "type": "string" + }, + { + "index": 1, + "type": "string" + }, + { + "index": 2, + "type": "int" + }, + { + "index": 3, + "type": "int" + } + ], + "password": "pass", + "fieldDelimiter": ",", + "encoding": "utf-8", + "username": "user" + }, + "name": "ftpreader" + }, + "writer": { + "parameter": {}, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + + diff --git a/docs/offline/reader/gbasereader.md b/docs/offline/reader/gbasereader.md new file mode 100644 index 0000000000..605a762f0e --- /dev/null +++ b/docs/offline/reader/gbasereader.md @@ -0,0 +1,436 @@ +# GBase Reader + + +## 一、插件名称 +名称:**gbasereader** + +## 二、支持的数据源版本 +**Gbase 8A**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串,需要注意gbase有A、S、T三种发行版,jdbcUrl端口和驱动都有区别。
jdbcUrl参考文档:[gbase官方文档](https://help.finereport.com/doc-view-2569.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + + } + }], + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select id, user_id, name from tableTest where id > 20", + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + } + }], + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "polling": true, + "pollingInterval": 3000, + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + } + }], + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + + diff --git a/docs/offline/reader/hbasereader.md b/docs/offline/reader/hbasereader.md new file mode 100644 index 0000000000..5d4688aecc --- /dev/null +++ b/docs/offline/reader/hbasereader.md @@ -0,0 +1,152 @@ +# HBase Reader + + +## 一、插件名称 +名称:**hbasereader** + +## 二、支持的数据源版本 +**HBase 1.3及以上** + +## 三、参数说明 + +- **table** + - 描述:hbase表名 + - 必选:是 + - 默认值:无 + + + +- **hbaseConfig** + - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: + +Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal + + - 必选:是 + - 默认值:无 + + + +- **range** + - 描述:指定hbasereader读取的rowkey范围。 + - startRowkey:指定开始rowkey; + - endRowkey:指定结束rowkey; + + + - isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey),配置格式如下: +``` +"range": { + "startRowkey": "aaa", + "endRowkey": "ccc", + "isBinaryRowkey":false +} +``` + + + - 注意:如果用户配置了 startRowkey 和 endRowkey,需要确保:startRowkey <= endRowkey + - 必选:否 + - 默认值:无 + + + +- **encoding** + - 描述:字符编码 + - 必选:无 + - 默认值:无 + + + +- **scanCacheSize** + - 描述:一次RPC请求批量读取的Results数量 + - 必选:无 + - 默认值:256 + +
+ +- **scanBatchSize** + - 描述:每一个result中的列的数量 + - 必选:无 + - 默认值:100 + +
+ +- **column** + - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 + - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; + - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 + - 必选:是 + - 默认值:无 + + +
+ + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "hbasereader", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase" + }, + "table": "sb5", + "encodig": "utf-8", + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ], + "range": { + "startRowkey": "", + "endRowkey": "", + "isBinaryRowkey": true + } + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` + +# diff --git a/docs/offline/reader/hdfsreader.md b/docs/offline/reader/hdfsreader.md new file mode 100644 index 0000000000..5b4e92ffa6 --- /dev/null +++ b/docs/offline/reader/hdfsreader.md @@ -0,0 +1,192 @@ +# HDFS Reader + + +## 一、插件名称 +名称:**hdfsreader**
+ + +## 二、支持的数据源版本 +| 协议 | 是否支持 | +| --- | --- | +| Hadoop 2.x | 支持 | +| Hadoop 3.x | 支持 | + + + + +## 三、数据源配置 +单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html)
集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html)
+ + +## 四、参数说明 + +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 + - 必选:是 + - 默认值:无 + + + +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:否 + - 默认值:无 + + + +- **path** + - 描述:数据文件的路径 + - 必选:是 + - 默认值:无 + + + +- **filterRegex** + - 描述:文件过滤正则表达式 + - 必选:否 + - 默认值:无 + + + +- **fileType** + - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` + - text:textfile文件格式 + - orc:orcfile文件格式 + - parquet:parquet文件格式 + - 必选:否 + - 默认值:text + + + +- **fieldDelimiter** + - 描述:`fileType`为`text`时字段的分隔符 + - 必选:否 + - 默认值:`\001` + + + + +## 五、使用示例 + +#### 1、读取text文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "hdfs://ns1/flinkx/text", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", + "dfs.nameservices": "ns1" + }, + "column": [ + { + "name": "col1", + "index": 0, + "type": "string" + }, + { + "name": "col2", + "index": 1, + "type": "string" + }, + { + "name": "col3", + "index": 2, + "type": "int" + }, + { + "name": "col4", + "index": 3, + "type": "int" + } + ], + "fieldDelimiter": ",", + "fileType": "text" + }, + "name": "hdfsreader" + }, + "writer": { + "parameter": {}, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 2、过滤文件名称 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "path": "hdfs://ns1/flinkx/text", + "filterRegex" : ".*\\.csv", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", + "dfs.nameservices": "ns1" + }, + "column": [ + { + "name": "col1", + "index": 0, + "type": "string" + }, + { + "name": "col2", + "index": 1, + "type": "string" + }, + { + "name": "col3", + "index": 2, + "type": "int" + }, + { + "name": "col4", + "index": 3, + "type": "int" + } + ], + "fieldDelimiter": ",", + "fileType": "text" + }, + "name": "hdfsreader" + }, + "writer": { + "parameter": {}, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` + + diff --git a/docs/offline/reader/kudureader.md b/docs/offline/reader/kudureader.md new file mode 100644 index 0000000000..bc8271299c --- /dev/null +++ b/docs/offline/reader/kudureader.md @@ -0,0 +1,187 @@ +# Kudu Reader + + +## 一、插件名称 +名称:**kudureader**
** + +## 二、支持的数据源版本 +**kudu 1.10及以上**
+ + +## 三、参数说明 + +- **column** + - 描述:需要生成的字段 + - 属性说明: + - name:字段名称; + - type:字段类型; + - 必选:是 + - 默认值:无 + + + +- **masterAddresses** + - 描述: master节点地址:端口,多个以,隔开 + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述: kudu表名。 + - 必选:是 + - 默认值:无 + + + +- **readMode** + - 描述: kudu读取模式: + - 1、read_latest
+默认的读取模式。
+该模式下,服务器将始终在收到请求时返回已提交的写操作。
+这种类型的读取不会返回快照时间戳,并且不可重复。
+用ACID术语表示,它对应于隔离模式:“读已提交”。 + - 2、read_at_snapshot
+该模式下,服务器将尝试在提供的时间戳上执行读取。
+如果未提供时间戳,则服务器将当前时间作为快照时间戳。
+在这种模式下,读取是可重复的,即将来所有在相同时间戳记下的读取将产生相同的数据。
+执行此操作的代价是等待时间戳小于快照的时间戳的正在进行的正在进行的事务,因此可能会导致延迟损失。用ACID术语,这本身就相当于隔离模式“可重复读取”。
+如果对已扫描tablet的所有写入均在外部保持一致,则这对应于隔离模式“严格可序列化”。
+注意:当前存在“空洞”,在罕见的边缘条件下会发生,通过这种空洞有时即使在采取措施使写入如此时,它们在外部也不一致。
+在这些情况下,隔离可能会退化为“读取已提交”模式。 + - 必选:是 + - 默认值:无 + + + +- **authentication** + - 描述:认证方式,如:Kerberos + - 必选:否 + - 默认值:无 + + + +- **principal** + - 描述: 用户名 + - 必选:否 + - 默认值:无 + + + +- **keytabFile** + - 描述:keytab文件路径 + - 必选:否 + - 默认值:无 + + + +- **workerCount** + - 描述:worker线程数 + - 必选:否 + - 默认值:默认为cpu*2 + + + +- **bossCount** + - 描述:boss线程数 + - 必选:否 + - 默认值:1 + + + +- **operationTimeout** + - 描述:普通操作超时时间 + - 必选:否 + - 默认值:30000 + + + +- **adminOperationTimeout** + - 描述: 管理员操作(建表,删表)超时时间 + - 必选:否 + - 默认值:30000 + + + +- **queryTimeout** + - 描述:连接scan token的超时时间 + - 必选:否 + - 默认值:与operationTimeout一致 + + + +- **where** + - 描述:过滤条件字符串,多个以and连接 + - 必选:否 + - 默认值:无 + + + +- **batchSizeBytes** + - 描述: kudu scan一次性最大读取字节数 + - 必选:否 + - 默认值:1048576 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "name": "kudureader", + "parameter": { + "column": [ + { + "name": "id", + "type": "long" + } + ], + "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", + "table": "kudu", + "readMode": "read_latest", + "authentication": "", + "principal": "", + "keytabFile": "", + "workerCount": 2, + "bossCount": 1, + "operationTimeout": 30000, + "adminOperationTimeout": 30000, + "queryTimeout": 30000, + "where": " id >= 1 ", + "batchSizeBytes": 1048576 + } + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/mongodbreader.md b/docs/offline/reader/mongodbreader.md new file mode 100644 index 0000000000..3c087d46b7 --- /dev/null +++ b/docs/offline/reader/mongodbreader.md @@ -0,0 +1,146 @@ +# MongoDB Reader + + +## 一、插件名称 +名称:**mongodbreader** + +## 二、支持的数据源版本 +**MongoDB 3.4及以上** + +## 三、参数说明
+ +- **url** + - 描述:MongoDB数据库连接的URL字符串,详细请参考[MongoDB官方文档](https://docs.mongodb.com/manual/reference/connection-string/) + - 必选:否 + - 默认值:无 + + + +- **hostPorts** + - 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔 + - 必选:否 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:否 + - 默认值:无 + + + +- **database** + - 描述:数据库名称 + - 必选:否 + - 默认值:无 + + + +- **collectionName** + - 描述:集合名称 + - 必选:是 + - 默认值:无 + -
+- **fetchSize** + - 描述:每次读取的数据条数,通过调整此参数来优化读取速率 + - 必选:否 + - 默认值:100 + + + +- **filter** + - 描述:过滤条件,通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```json +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +``` +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value", + "splitter":"," +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - splitter:因为 MongoDB 支持数组类型,所以 MongoDB 读出来的数组类型要通过这个分隔符合并成字符串 + - 必选:是 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "url": "mongodb://root:admin@kudu4:27017/admin?authSource=admin", + "fetchSize": 100, + "collectionName" : "tudou", + "filter" : "{}", + "column" : ["*"] + }, + "name" : "mongodbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/mysqlreader.md b/docs/offline/reader/mysqlreader.md new file mode 100644 index 0000000000..04ca11f99e --- /dev/null +++ b/docs/offline/reader/mysqlreader.md @@ -0,0 +1,431 @@ +# MySQL Reader + + +## 一、插件名称 +名称:**mysqlreader**
** + +## 二、支持的数据源版本 +**MySQL 5.X**
+ + +## 三、参数说明
+ +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 2, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、指定`customSql` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["id","user_id","name"], + "customSql": "select * from kudu where id > 20", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、增量同步指定`startLocation` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id > 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "polling": true, + "pollingInterval": 3000, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/odpsreader.md b/docs/offline/reader/odpsreader.md new file mode 100644 index 0000000000..57a0625031 --- /dev/null +++ b/docs/offline/reader/odpsreader.md @@ -0,0 +1,134 @@ +# ODPS Reader + + +## 一、插件名称 +名称:**odpsreader**
+ +## 二、参数说明 + +- **accessId** + - 描述:ODPS系统登录ID + - 必选:是 + - 默认值:无 + + + +- **accessKey** + - 描述:ODPS系统登录Key + - 必选:是 + - 默认值:无 + + + +- **project** + - 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感) + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:读取数据表的表名称(大小写不敏感) + - 必选:是 + - 默认值:无 + + +
+ +- **partition** + - 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou   pt=1,ds=shanghai   pt=2,ds=hangzhou   pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":["pt=1,ds=shanghai"]`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":["pt=1,ds=* "]`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":["pt=*,ds=*"]` + - 必选:如果表为分区表,则必填。如果表为非分区表,则不能填写 + - 默认值:无 + + +
+ +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +``` +"column":[*] +``` + + -
2.只指定字段名称: +``` +"column":["id","name"] +``` + + -
3.指定具体信息: +``` +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + + +## 三、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "name": "odpsreader", + "parameter": { + "odpsConfig": { + "accessId": "${odps.accessId}", + "accessKey": "${odps.accessKey}", + "project": "${odps.project}" + }, + "table": "tableTest", + "partition": "pt='xxooxx'", + "column": [{ + "name": "col1", + "type": "string", + "value":"xx", + "format":"yyyy-MM-dd HH:mm:ss" + + }] + } + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/oraclereader.md b/docs/offline/reader/oraclereader.md new file mode 100644 index 0000000000..df2f749531 --- /dev/null +++ b/docs/offline/reader/oraclereader.md @@ -0,0 +1,445 @@ +# Oracle Reader + + +## 一、插件名称 +名称:**oraclereader**
** + +## 二、支持的数据源版本 +**Oracle 9 及以上**
** + +## 三、参数说明
+ +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:3000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定次参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` + + - 2.只指定字段名称: +``` +"column":["id","name"] +``` + + - 3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], + "table": ["TUDOU.KUDU"] + }], + "column": ["*"], + "customSql": "", + "where": "ID < 10000", + "splitPk": "", + "fetchSize": 1024, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], + "table": ["TUDOU.KUDU"] + }], + "column": ["*"], + "customSql": "", + "where": "ID < 10000", + "splitPk": "id", + "fetchSize": 1024, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 2, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、指定`customSql` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], + "table": ["TUDOU.KUDU"] + }], + "column": ["ID","USER_ID","NAME"], + "customSql": "select * from kudu where ID > 20", + "where": "ID < 10000", + "splitPk": "ID", + "fetchSize": 1024, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、增量同步指定`startLocation` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], + "table": ["TUDOU.KUDU"] + }], + "column": [{ + "name": "ID", + "type": "NUMBER" + },{ + "name": "USER_ID", + "type": "NUMBER" + },{ + "name": "NAME", + "type": "VARCHAR" + }], + "customSql": "", + "where": "ID < 1600", + "splitPk": "ID", + "increColumn": "ID", + "startLocation": "20", + "fetchSize": 1024, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], + "table": ["TUDOU.KUDU"] + }], + "column": [{ + "name": "ID", + "type": "NUMBER" + },{ + "name": "USER_ID", + "type": "NUMBER" + },{ + "name": "NAME", + "type": "VARCHAR" + }], + "customSql": "", + "where": "ID > 1600", + "splitPk": "ID", + "increColumn": "ID", + "startLocation": "20", + "fetchSize": 1024, + "queryTimeOut": 1000, + "polling": true, + "pollingInterval": 3000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/phoenixreader.md b/docs/offline/reader/phoenixreader.md new file mode 100644 index 0000000000..5a3de8b392 --- /dev/null +++ b/docs/offline/reader/phoenixreader.md @@ -0,0 +1,533 @@ +# Phoenix Reader + + +## 一、插件名称 +名称:**phoenixreader**
+ +## 二、支持的数据源版本 +phoenix4.12.0-HBase-1.3及之后 + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[phoniex官方文档](https://phoenix.apache.org/#) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select * from tableTest", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、增量同步指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、增量轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "polling": true, + "pollingInterval": 3000, + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/polardbreader.md b/docs/offline/reader/polardbreader.md new file mode 100644 index 0000000000..fd9eed5962 --- /dev/null +++ b/docs/offline/reader/polardbreader.md @@ -0,0 +1,431 @@ +# PolarDB Reader + + +## 一、插件名称 +名称:**polardbreader** + +## 二、支持的数据源版本 +**PolarDB MySQL 8.0**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.只指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 2, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、指定`customSql` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": ["id","user_id","name"], + "customSql": "select * from kudu where id > 20", + "where": "id < 100", + "splitPk": "", + "fetchSize": 0, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、增量同步指定`startLocation` +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], + "table": ["kudu"] + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id > 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "polling": true, + "pollingInterval": 3000, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/postgresqlreader.md b/docs/offline/reader/postgresqlreader.md new file mode 100644 index 0000000000..39c5bfc3aa --- /dev/null +++ b/docs/offline/reader/postgresqlreader.md @@ -0,0 +1,418 @@ +# PostgreSql Reader + + +## 一、插件名称 +名称:**postgresqlreader** + +## 二、支持的数据源版本 +**PostgreSql 9.4及以上** + + +## 三、参数说明
+ +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 +
jdbcUrl参考文档:[postgresql官方文档](https://jdbc.postgresql.org/documentation/head/connect.html#connection-parameters) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +** + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 3、 指定customsql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select * from tabletest", + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "polling": true, + "pollingInterval": 3000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` +
diff --git a/docs/rdbdreader.md b/docs/offline/reader/rdbdreader.md similarity index 100% rename from docs/rdbdreader.md rename to docs/offline/reader/rdbdreader.md diff --git a/docs/offline/reader/saphanareader.md b/docs/offline/reader/saphanareader.md new file mode 100644 index 0000000000..aea38f3a40 --- /dev/null +++ b/docs/offline/reader/saphanareader.md @@ -0,0 +1,224 @@ +# Saphana Reader + + +## 一、插件名称 +名称:**saphanareader**
+ +## 二、支持的数据源版本 +SAP HANA 2.0及以上
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:3000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定次参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` + + - 2.只指定字段名称: +``` +"column":["id","name"] +``` + + - 3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": [ + "jdbc:sap://kudu3:39017" + ], + "table": [ + "SYS.P_DPAPI_KEY_" + ] + } + ], + "column": [ + { + "name": "OID", + "type": "BIGINT" + }, + { + "name": "CALLER", + "type": "NVARCHAR" + }, + { + "name": "RECORD_ID", + "type": "NVARCHAR" + }, + { + "name": "KEY_ID", + "type": "INTEGER" + }, + { + "name": "KEY", + "type": "VARBINARY" + }, + { + "name": "CREATE_USER", + "type": "NVARCHAR" + }, + { + "name": "CREATE_TIME", + "type": "BIGINT" + }, + { + "name": "DATA_ENCRYPTION_ALGORITHM", + "type": "TINYINT" + }, + { + "name": "IS_CURRENT_KEY", + "type": "TINYINT" + } + ] + }, + "name": "saphanareader" + }, + + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": false + }, + "errorLimit": {}, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "trace", + "path": "", + "pattern": "" + } + } + } +} +``` diff --git a/docs/offline/reader/sqlserverreader.md b/docs/offline/reader/sqlserverreader.md new file mode 100644 index 0000000000..a3f2ac056e --- /dev/null +++ b/docs/offline/reader/sqlserverreader.md @@ -0,0 +1,469 @@ +# SqlServer Reader + + +## 一、插件名称 +名称:**sqlserverreader** + +## 二、支持的数据源版本 +**Microsoft SQL Server 2012及以后** + + +## 三、参数说明 + +- **jdbcUrl** + - 描述:使用开源的jtds驱动连接 而非Microsoft的官方驱动
jdbcUrl参考文档:[jtds驱动官方文档](http://jtds.sourceforge.net/faq.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` + + - 2.只指定字段名称: +``` +"column":["id","name"] +``` + + - 3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +** + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } +} +``` + +#### 3、指定customSql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "select * from kudu where id > 20", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "increColumn": "id", + "startLocation": "20", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint", + "key" : "id" + }, { + "name" : "user_id", + "type" : "bigint", + "key" : "user_id" + }, { + "name" : "name", + "type" : "varchar", + "key" : "name" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "polling": true, + "pollingInterval": 3000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } +} +``` diff --git a/docs/offline/reader/streamreader.md b/docs/offline/reader/streamreader.md new file mode 100644 index 0000000000..697571098a --- /dev/null +++ b/docs/offline/reader/streamreader.md @@ -0,0 +1,241 @@ +# Stream Reader + + +## 一、简介 +为了让用户能快速熟悉与使用,FlinkX提供了不需要数据库就能读取数据的Stream reader插件。
该插件利用了模拟数据的JMockData框架,能够根据给定的属性生成相应的随机数据,方便用户修改和调试。
+ + +## 二、插件名称 +名称:**streamreader** + + +### 三、参数说明 + +- **sliceRecordCount** + - 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止 + - 必选:否 + - 默认值:0 + + + +- **column** + - 描述:随机Java数据类型的字段信息 + - 格式:一组或多组描述"name"和"type"的json格式 + - 格式说明: +```json +{ + "name": "id", + "type": "int", + "value":"7" +} +``` + + - "name"属性为用户提供的标识,类似于mysql的列名,必须填写。 + - "tpye" 属性为需要生成的数据类型,可配置以下类型: + - id:从0开始步长为1的int类型自增ID + - int,integer + - byte + - boolean + - char,character + - short + - long + - float + - double + - date + - timestamp + - bigdecimal + - biginteger + - int[] + - byte[] + - boolean[] + - char[],character[] + - short[] + - long[] + - float[] + - double[] + - string[] + - binary + - string:以上均不匹配时默认为string字符串 + - "value"属性为用户设定的输出值,可以不填。 + + + + +## 四、使用实例 + +#### 1、单通道构造100条数据 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、单通道无限构造数据 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["0"] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、多通道构造数据,每个通道100条 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100","100"] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 2, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/reader/teradatareader.md b/docs/offline/reader/teradatareader.md new file mode 100644 index 0000000000..c226ce3603 --- /dev/null +++ b/docs/offline/reader/teradatareader.md @@ -0,0 +1,216 @@ +# Teradata Reader + + +## 一、插件名称 +名称:**teradatareader**
+ +## 二、支持的数据源版本 +**Teradata 15.0及以上**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:3000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定次参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3中格式 +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` + + - 2.只指定字段名称: +``` +"column":["id","name"] +``` + + - 3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": [ + "jdbc:teradata://kudu3/testbase" + ], + "table": [ + "Employee" + ] + } + ], + "column": [ + { + "name": "EmployeeNo", + "type": "INTEGER" + }, + { + "name": "FirstName", + "type": "VARCHAR" + }, + { + "name": "LastName", + "type": "VARCHAR" + }, + { + "name": "DOB", + "type": "DATE" + }, + { + "name": "JoinedDate", + "type": "DATE" + }, + { + "name": "DepartmentNo", + "type": "BYTEINT" + } + ] + }, + "name": "teradatareader" + }, + + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/carbondatawriter.md b/docs/offline/writer/carbondatawriter.md new file mode 100644 index 0000000000..2ad1b8e41e --- /dev/null +++ b/docs/offline/writer/carbondatawriter.md @@ -0,0 +1,139 @@ +# Carbondata Writer + + +## 一、插件名称 +名称:**carbondatawriter**
** + +## 二、支持的数据源版本 +**Carbondata 1.5及以上**
+ + +## 三、参数说明 + +- **path** + - 描述:carbondata表的存储路径 + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:carbondata表名 + - 必选:否 + - 默认值:无 + + + +- **database** + - 描述:carbondata库名 + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:所配置的表中需要同步的字段名列表 + - 必选:是 + - 默认值:无 + + + +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:是 + - 默认值:无 + +
+ +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:写入模式,支持append和overwrite + - 必填:否 + - 默认值:append + + + +- **partition** + - 描述:carbondata分区 + - 必填:否 + - 默认值:append + + + +- **batchSize** + - 描述:批量提交条数 + - 必填:否 + - 默认值:204800 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "id" + }, { + "name" : "data", + "type" : "string" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "name": "carbondatawriter", + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/carbon.store1/sb/sb500", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "rdos2:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "rdos1:9000", + "dfs.nameservices": "ns1" + }, + "defaultFS": "hdfs://ns1", + "table": "sb500", + "database": "sb", + "writeMode": "overwrite", + "column": ["a","b"], + "batchSize": 204800 + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/writer/cassandrawriter.md b/docs/offline/writer/cassandrawriter.md new file mode 100644 index 0000000000..4a93d69050 --- /dev/null +++ b/docs/offline/writer/cassandrawriter.md @@ -0,0 +1,172 @@ +# Cassandra Writer + + +## 一、插件名称 +名称:**cassandrawriter**
+ +## 二、支持的数据源版本 +**Cassandra 3.0以以上**
+ +## 三、参数说明 + +- **host** + - 描述:数据库地址 + - 必选:是 + - 默认值:无 + + + +- **port** + - 描述:端口 + - 必选:否 + - 默认值:9042 + + + +- **username** + - 描述:用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:密码 + - 必选:否 + - 默认值:无 + + + +- **useSSL** + - 描述:数字证书 + - 必选:否 + - 默认值:false + + + +- **column** + - 描述:查询结果中被select出来的属性集合,为空则select * + - 必选:否 + - 默认值:无 + + + +- **keyspace** + - 描述:需要同步的表所在的keyspace + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:要查询的表 + - 必选:是 + - 默认值:无 + + + +- **batchSize** + - 描述:异步写入的批次大小 + - 必选:否 + - 默认值:1 + + + +- **asyncWrite** + - 描述:是否异步写入 + - 必选:否 + - 默认值:false + + + +- **connecttionsPerHost** + - 描述:分配给每个host的连接数 + - 必选:否 + - 默认值:8 + + + +- **maxPendingPerConnection** + - 描述:最多能建立的连接数 + - 必选:否 + - 默认值:128 + + + +- **consistancyLevel** + - 描述:数据一致性级别。可选`ONE`、`QUORUM`、`LOCAL_QUORUM`、`EACH_QUORUM`、`ALL`、`ANY`、`TWO`、`THREE`、`LOCAL_ONE` + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "id", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "cassandrawriterer", + "parameter": { + "host": "kudu3", + "port": 9042, + "username":"", + "password":"", + "useSSL":false, + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/writer/clickhousewriter.md b/docs/offline/writer/clickhousewriter.md new file mode 100644 index 0000000000..b6fc22df1c --- /dev/null +++ b/docs/offline/writer/clickhousewriter.md @@ -0,0 +1,159 @@ +# ClickHouse Writer + + +## 一、插件名称 +名称:**clickhousewriter**
+ +## 二、支持的数据源版本 +**ClickHouse 19.x及以上**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 语句,只支持insert操作 + - 必选:是 + - 所有选项:insert + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + +
** + +## 四、配置示例 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "clickhousewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:clickhouse://0.0.0.1:8123/dtstack", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT", + "key": "id" + }, + { + "name": "user_id", + "type": "BIGINT", + "key": "user_id" + }, + { + "name": "name", + "type": "varchar", + "key": "name" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/writer/db2writer.md b/docs/offline/writer/db2writer.md new file mode 100644 index 0000000000..e7b66dd319 --- /dev/null +++ b/docs/offline/writer/db2writer.md @@ -0,0 +1,269 @@ +# DB2 Writer + + +## 一、插件名称 +名称:**db2writer**
** + +## 二、支持的数据源版本 +**DB2 9、10**
** + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into`  语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 默认值:无 + +** + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "id", + "type": "int", + "value": "400" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "dept", + "type": "int" + }, + { + "name": "job", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "db2writer", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:db2://localhost:50000/sample", + "table": [ + "staff" + ] + }], + "username": "user", + "password": "password", + "column": [ + { + "name": "id", + "type": "SMALLINT", + "key": "id" + }, + { + "name": "name", + "type": "VARCHAR", + "key": "user_id" + }, + { + "name": "dept", + "type": "SMALLINT", + "key": "name" + },{ + "name": "job", + "type": "VARCHAR" + } + ], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "id", + "type": "int", + "value": "400" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "dept", + "type": "int" + }, + { + "name": "job", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "db2writer", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:db2://localhost:50000/sample", + "table": [ + "staff" + ] + }], + "username": "user", + "password": "password", + "column": [ + { + "name": "id", + "type": "SMALLINT", + "key": "id" + }, + { + "name": "name", + "type": "VARCHAR", + "key": "user_id" + }, + { + "name": "dept", + "type": "SMALLINT", + "key": "name" + },{ + "name": "job", + "type": "VARCHAR" + } + ], + "writeMode": "update", + "updateKey": {"key": ["id"]}, + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + + diff --git a/docs/offline/writer/dmwriter.md b/docs/offline/writer/dmwriter.md new file mode 100644 index 0000000000..ec37a9f433 --- /dev/null +++ b/docs/offline/writer/dmwriter.md @@ -0,0 +1,233 @@ +# DM Writer + + +## 一、插件名称 +名称:**dmwriter** + +## 二、支持的数据源版本 +**DM7、DM8**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 默认值:无 + +** + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "dmwriter", + "parameter": { + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": "jdbc:dm://localhost:5236", + "table": [ + "PERSON.STUDENT" + ] + } + ], + "session": [], + "preSql": [], + "postSql": [], + "writeMode": "insert", + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "int", + "type": "int", + "value": "3" + }, + { + "name": "age", + "type": "int", + "value": "3" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "dmwriter", + "parameter": { + "username": "SYSDBA", + "password": "SYSDBA", + "connection": [ + { + "jdbcUrl": "jdbc:dm://localhost:5236", + "table": [ + "PERSON.STUDENT" + ] + } + ], + "session": [], + "preSql": [], + "postSql": [], + "writeMode": "update", + "updateKey": {"key": ["ID"]}, + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ] + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` diff --git a/docs/offline/writer/eswriter.md b/docs/offline/writer/eswriter.md new file mode 100644 index 0000000000..35029a6c80 --- /dev/null +++ b/docs/offline/writer/eswriter.md @@ -0,0 +1,180 @@ +# ElasticSearch Writer + + +## 一、插件名称 +名称:**eswriter**
+ +## 二、支持的数据源版本 +**Elasticsearch 6.X**
+ +## 三、参数说明
+ +- **address** + - 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:Elasticsearch认证用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:Elasticsearch认证密码 + - 必选:否 + - 默认值:无 + + + +- **index** + - 描述:Elasticsearch 索引值 + - 必选:是 + - 默认值:无 + + + +- **type** + - 描述:Elasticsearch 索引类型 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:写入elasticsearch的若干个列,每列形式如下 +``` + { + "name": "列名", + "type": "列类型" + } +``` + + - 必选:是 + - 默认值:无 + + + +- **idColumns** + - 描述:用于构造文档id的若干个列,每列形式如下 + +普通列 +``` +{ + "index": 0, // 前面column属性中列的序号,从0开始 + "type": "string" 列的类型,默认为string +} +``` +常数列 +``` +{ + "value": "ffff", // 常数值 + "type": "string" // 常数列的类型,默认为string +} +``` + + - 必选:否 + - 注意: + - 如果不指定idColumns属性,则会随机产生文档id + - 如果指定的字段值存在重复或者指定了常数,按照es的逻辑,同样值的doc只会保留一份 + - 默认值:无 + + + +- **bulkAction** + - 描述:批量写入的记录条数 + - 必选:是 + - 默认值:100 + + + +- **timeout** + - 描述:连接超时时间,如果bulkAction指定的数值过大,写入数据可能会超时,这时可以配置超时时间 + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + } + }, + "writer": { + "name": "eswriter", + "parameter": { + "address": "172.16.8.193:9200", + "username": "elastic", + "password": "abc123", + "index": "tudou", + "type": "doc", + "bulkAction": 100, + "timeout": 100, + "idColumn": [ + { + "index": 0, + "type": "integer" + } + ], + "column": [ + { + "name": "id", + "type": "integer" + }, + { + "name": "user_id", + "type": "integer" + }, + { + "name": "name", + "type": "string" + } + ] + } + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + }, + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` diff --git a/docs/offline/writer/ftpwriter.md b/docs/offline/writer/ftpwriter.md new file mode 100644 index 0000000000..9b6995ef01 --- /dev/null +++ b/docs/offline/writer/ftpwriter.md @@ -0,0 +1,316 @@ +# FTP Writer + + +## 一、插件名称 +名称:**ftpwriter**
+ + +## 二、数据源版本 +| 协议 | 是否支持 | +| --- | --- | +| FTP | 支持 | +| SFTP | 支持 | + + + + +## 三、数据源配置 +FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
+ + +## 四、参数说明 + +- **protocol** + - 描述:ftp服务器协议,目前支持传输协议有`ftp`、`sftp` + - 必选:是 + - 默认值:无 + + + +- **host** + - 描述:ftp服务器地址 + - 必选:是 + - 默认值:无 + + + +- **port** + - 描述:ftp服务器端口 + - 必选:否 + - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 + + + +- **connectPattern** + - 描述:协议为ftp时的连接模式,可选`pasv`,`port`,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) + - 必选:否 + - 默认值:`PASV` + + + +- **username** + - 描述:ftp服务器访问用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:ftp服务器访问密码 + - 必选:否 + - 默认值:无 + + + +- **path** + - 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径 + - 必选:是 + - 默认值:无 + + + +- **fieldDelimiter** + - 描述:读取的字段分隔符 + - 必选:是 + - 默认值:`,` + + + +- **encoding** + - 描述:读取文件的编码配置 + - 必选:否 + - 默认值:`UTF-8` + + + +- **privateKeyPath** + - 描述:私钥文件路径 + - 必选:否 + - 默认值:无 + +
+ +- **writeMode** + - 描述:ftpwriter写入前数据清理处理模式: + - append:追加 + - overwrite:覆盖 + - 注意:overwrite模式时会删除dtp当前目录下的所有文件 + - 必选:否 + - 默认值:append + + + +- **isFirstLineHeader** + - 描述:首行是否为标题行,如果是则不读取第一行 + - 必选:否 + - 默认值:false + + + +- **timeout** + - 描述:连接超时时间,单位毫秒 + - 必选:否 + - 默认值:5000 + + + +- **maxFileSize** + - 描述:写入hdfs单个文件最大大小,单位字节 + - 必须:否 + - 默认值:1073741824‬(1G) + + + +- **column** + - 描述:需要读取的字段 + - 格式:指定具体信息: +```json +"column": [{ + "name": "col1", + "type": "datetime" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 + - 必选:是 + - 默认值:无 + + + + +## 五、使用示例 + +#### 1、append模式写入 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "/data/ftp/flinkx", + "protocol": "sftp", + "port": 22, + "writeMode": "append", + "host": "localhost", + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "password": "pass", + "fieldDelimiter": ",", + "encoding": "utf-8", + "username": "user" + }, + "name": "ftpwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 2、指定文件大小 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "sliceRecordCount": [ + "0" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "/data/ftp/flinkx", + "protocol": "sftp", + "port": 22, + "writeMode": "append", + "host": "localhost", + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "password": "pass", + "fieldDelimiter": ",", + "encoding": "utf-8", + "username": "user", + "maxFileSize" : 5242880 + }, + "name": "ftpwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + + diff --git a/docs/offline/writer/gbasewriter.md b/docs/offline/writer/gbasewriter.md new file mode 100644 index 0000000000..e1ed14c344 --- /dev/null +++ b/docs/offline/writer/gbasewriter.md @@ -0,0 +1,211 @@ +# GBase Writer + + +## 一、插件名称 +名称:**gbasewriter**
+ +## 二、支持的数据源版本 + +#### Gbase 8A + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者` merge into` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 默认值:无 + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + +** + +## 四、配置示例 + +#### 1、insert
+```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "gbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/dtstack", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "INT" + }, + { + "name": "age", + "type": "INT" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "gbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/dtstack", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "INT" + }, + { + "name": "age", + "type": "INT" + }], + "writeMode": "update", + "batchSize": 1024, + "updateKey": {"key": ["id"]}, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` diff --git a/docs/offline/writer/hbasewriter.md b/docs/offline/writer/hbasewriter.md new file mode 100644 index 0000000000..3429e34bea --- /dev/null +++ b/docs/offline/writer/hbasewriter.md @@ -0,0 +1,209 @@ +# HBase Writer + + +## 一、插件名称 +名称:**hbasewriter**
+ +## 二、支持的数据源版本 +**HBase 1.3及以上**
+ +## 三、参数说明 + +- **tablename** + - 描述:hbase表名 + - 必选:是 + - 默认值:无 + + + +- **hbaseConfig** + - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: + +Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal + + - 必选:是 + - 默认值:无 + + + +- **nullMode** + - 描述:读取的null值时,如何处理。支持两种方式: + - skip:表示不向hbase写这列; + - empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0] + - 必选:否 + - 默认值:skip + + + +- **encoding** + - 描述:字符编码 + - 必选:无 + - 默认值:UTF-8 + +
+ +- **walFlag** + - 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能 + - 必选:否 + - 默认值:false + + + +- **writeBufferSize** + - 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求 + - 必选:否 + - 默认值:8388608(8M) + + + +- **scanCacheSize** + - 描述:一次RPC请求批量读取的Results数量 + - 必选:无 + - 默认值:256 + + + +- **scanBatchSize** + - 描述:每一个result中的列的数量 + - 必选:无 + - 默认值:100 + + + +- **column** + - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 + - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; + - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 + - 必选:是 + - 默认值:无 + + + +- **rowkeyColumn** + - 描述:用于构造rowkey的描述信息,支持两种格式,每列形式如下 + - 字符串格式 +
字符串格式为:$(cf:col),可以多个字段组合:$(cf:col1)_$(cf:col2), +
可以使用md5函数:md5($(cf:col)) + - 数组格式 + - 普通列 +``` +{ + "index": 0, // 该列在column属性中的序号,从0开始 + "type": "string" 列的类型,默认为string +} +``` + + - 常数列 +``` +{ + "value": "ffff", // 常数值 + "type": "string" // 常数列的类型,默认为string +} +``` + + - 必选:否 +
如果不指定idColumns属性,则会随机产生文档id + - 默认值:无 + + + +- **versionColumn** + - 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: +``` +"versionColumn":{ +"index":1 +} +``` + + -
或者 +``` +"versionColumn":{ +"value":"123456789" +} +``` + +
+ + +## 三、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "hbasewriter", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase" + }, + "table": "tb1", + "rowkeyColumn": "col1#col2", + "column": [ + { + "name": "cf1:id", + "type": "int" + }, + { + "name": "cf1:user_id", + "type": "int" + }, + { + "name": "cf1:name", + "type": "string" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +
+ diff --git a/docs/offline/writer/hdfswriter.md b/docs/offline/writer/hdfswriter.md new file mode 100644 index 0000000000..97390a0a3d --- /dev/null +++ b/docs/offline/writer/hdfswriter.md @@ -0,0 +1,389 @@ +# HDFS Writer + + +## 一、插件名称 +名称:**hdfswriter**
+ + +## 二、数据源版本 +| 协议 | 是否支持 | +| --- | --- | +| Hadoop 2.x | 支持 | +| Hadoop 3.x | 支持 | + + + + +## 三、数据源配置 +单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html)
集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html)
+ + +## 四、参数说明 + +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 + - 必选:是 + - 默认值:无 + + + +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:否 + - 默认值:无 + + + +- **path** + - 描述:数据文件的路径 + - 必选:是 + - 默认值:无 + + + +- **filterRegex** + - 描述:文件过滤正则表达式 + - 必选:否 + - 默认值:无 + + + +- **fileType** + - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` + - text:textfile文件格式 + - orc:orcfile文件格式 + - parquet:parquet文件格式 + - 必选:否 + - 默认值:text + + + +- **fieldDelimiter** + - 描述:`fileType`为`text`时字段的分隔符 + - 必选:否 + - 默认值:`\001` + + + +- **encoding** + - 描述:`fileType`为`text`时可配置编码格式 + - 必选:否 + - 默认值:UTF-8 + + + +- **maxFileSize** + - 描述:写入hdfs单个文件最大大小,单位字节 + - 必须:否 + - 默认值:1073741824‬(1G) + + + +- **compress** + - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 + - text:支持`GZIP`、`BZIP2`格式 + - orc:支持`SNAPPY`、`GZIP`、`BZIP`、`LZ4`格式 + - parquet:支持`SNAPPY`、`GZIP`、`LZO` + - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** + - 必选:否 + - 默认值:无 + + + +- **compress** + - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 + - text:支持`GZIP`、`BZIP2`格式 + - orc:支持`SNAPPY`、`GZIP`、`BZIP`、`LZ4`格式 + - parquet:支持`SNAPPY`、`GZIP`、`LZO` + - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** + - 必选:否 + - 默认值:无 + + + +- **fileName** + - 描述:写入的目录名称 + - 必须:否 + - 默认值:无 + + + +- **writeMode** + - 描述:hdfswriter写入前数据清理处理模式: + - append:追加 + - overwrite:覆盖 + - 注意:overwrite模式时会删除hdfs当前目录下的所有文件 + - 必选:否 + - 默认值:append + + + +- **column** + - 描述:需要读取的字段。 + - 格式:指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - 必选:是 + - 默认值:无 + + + + +## 五、使用示例 + +#### 1、写入text文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/flinkx/text", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", + "dfs.nameservices": "ns1" + }, + "column": [ + { + "name": "col1", + "index": 0, + "type": "string" + }, + { + "name": "col2", + "index": 1, + "type": "string" + }, + { + "name": "col3", + "index": 2, + "type": "int" + }, + { + "name": "col4", + "index": 3, + "type": "int" + } + ], + "fieldDelimiter": ",", + "fileType": "text", + "writeMode": "append" + }, + "name": "hdfswriter" + } + } + ], + "setting": { + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 2、写入orc文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/flinkx/text", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", + "dfs.nameservices": "ns1" + }, + "column": [ + { + "name": "col1", + "index": 0, + "type": "string" + }, + { + "name": "col2", + "index": 1, + "type": "string" + }, + { + "name": "col3", + "index": 2, + "type": "int" + }, + { + "name": "col4", + "index": 3, + "type": "int" + } + ], + "fileType": "orc", + "writeMode": "append" + }, + "name": "hdfswriter" + } + } + ], + "setting": { + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + +#### 3、写入parquet文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "col1", + "type": "string" + }, + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": "int" + }, + { + "name": "col4", + "type": "int" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/flinkx/text", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", + "dfs.nameservices": "ns1" + }, + "column": [ + { + "name": "col1", + "index": 0, + "type": "string" + }, + { + "name": "col2", + "index": 1, + "type": "string" + }, + { + "name": "col3", + "index": 2, + "type": "int" + }, + { + "name": "col4", + "index": 3, + "type": "int" + } + ], + "fileType": "parquet", + "writeMode": "append" + }, + "name": "hdfswriter" + } + } + ], + "setting": { + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` + + diff --git a/docs/offline/writer/hivewriter.md b/docs/offline/writer/hivewriter.md new file mode 100644 index 0000000000..49c9842da8 --- /dev/null +++ b/docs/offline/writer/hivewriter.md @@ -0,0 +1,369 @@ +# Hive Writer + + +## 一、插件名称 +名称:**hivewriter**
+ +## 二、支持的数据源版本 +**Hive 2.X**
+ +## 三、参数说明
+ +- **jdbcUrl** + - 描述:连接Hive JDBC的字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:Hive认证用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:Hive认证密码 + - 必选:否 + - 默认值:无 + + + +- **fileType** + - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` + - text:textfile文件格式 + - orc:orcfile文件格式 + - parquet:parquet文件格式 + - 必选:是 + - 默认值:无 + + + +- **fieldDelimiter** + - 描述:hivewriter中`fileType`为`text`时字段的分隔符, + - 注意:用户需要保证与创建的Hive表的字段分隔符一致,否则无法在Hive表中查到数据 + - 必选:是 + - 默认值:`\u0001` + + + +- **writeMode** + - 描述:hivewriter写入前数据清理处理模式: + - append:追加 + - overwrite:覆盖 + - 注意:overwrite模式时会删除Hive当前分区下的所有文件 + - 必选:否 + - 默认值:append + + + +- **compress** + - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 + - text:支持`GZIP`、`BZIP2`格式 + - orc:支持`SNAPPY`、`GZIP`、`BZIP`、`LZ4`格式 + - parquet:支持`SNAPPY`、`GZIP`、`LZO` + - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** + - 必选:否 + - 默认值:无 + + + +- **charsetName** + - 描述:写入text文件的编码配置 + - 必选:否 + - 默认值:UTF-8 + + + +- **maxFileSize** + - 描述:写入hdfs单个文件最大大小,单位字节 + - 必须:否 + - 默认值:1073741824‬(1G) + + + +- **tablesColumn** + - 描述:写入hive表的表结构信息,**若表不存在则会自动建表**。示例: +```json +{ + "kudu":[ + { + "key":"id", + "type":"int" + }, + { + "key":"user_id", + "type":"int" + }, + { + "key":"name", + "type":"string" + } + ] +} +``` + + - 必选:是 + - 默认值:无 + + + +- **partition** + - 描述:分区字段名称 + - 必选:是 + - 默认值:`pt` + + + +- **partitionType** + - 描述:分区类型,包括 DAY、HOUR、MINUTE三种。**若分区不存在则会自动创建,自动创建的分区时间以当前任务运行的服务器时间为准** + - DAY:天分区,分区示例:pt=202000101 + - HOUR:小时分区,分区示例:pt=2020010110 + - MINUTE:分钟分区,分区示例:pt=202001011027 + - 必选:是 + - 默认值:无 + + + +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 + - 必选:是 + - 默认值:无 + + + +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 + +#### 1、写入text +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + } + }, + "writer": { + "name" : "hivewriter", + "parameter" : { + "jdbcUrl" : "jdbc:hive2://kudu3:10000/tudou", + "username" : "", + "password" : "", + "fileType" : "text", + "fieldDelimiter" : "\u0001", + "writeMode" : "overwrite", + "compress" : "", + "charsetName" : "UTF-8", + "maxFileSize" : 1073741824, + "tablesColumn" : "{\"kudu\":[{\"key\":\"id\",\"type\":\"int\"},{\"key\":\"user_id\",\"type\":\"int\"},{\"key\":\"name\",\"type\":\"string\"}]}", + "partition" : "pt", + "partitionType" : "DAY", + "defaultFS" : "hdfs://ns1", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", + "dfs.nameservices" : "ns1", + "fs.hdfs.impl.disable.cache" : "true", + "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + } + } + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + }, + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` + +#### 2、写入orc +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + } + }, + "writer": { + "name" : "hivewriter", + "parameter" : { + "jdbcUrl" : "jdbc:hive2://kudu3:10000/tudou", + "username" : "", + "password" : "", + "fileType" : "orc", + "fieldDelimiter" : "", + "writeMode" : "overwrite", + "compress" : "GZIP", + "charsetName" : "UTF-8", + "maxFileSize" : 1073741824, + "tablesColumn" : "{\"kudu\":[{\"key\":\"id\",\"type\":\"int\"},{\"key\":\"user_id\",\"type\":\"int\"},{\"key\":\"name\",\"type\":\"string\"}]}", + "partition" : "pt", + "partitionType" : "DAY", + "defaultFS" : "hdfs://ns1", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", + "dfs.nameservices" : "ns1", + "fs.hdfs.impl.disable.cache" : "true", + "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + } + } + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + }, + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` + +#### 3、写入parquet +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + } + }, + "writer": { + "name" : "hivewriter", + "parameter" : { + "jdbcUrl" : "jdbc:hive2://kudu3:10000/tudou", + "username" : "", + "password" : "", + "fileType" : "parquet", + "fieldDelimiter" : "", + "writeMode" : "overwrite", + "compress" : "SNAPPY", + "charsetName" : "UTF-8", + "maxFileSize" : 1073741824, + "tablesColumn" : "{\"kudu\":[{\"key\":\"id\",\"type\":\"int\"},{\"key\":\"user_id\",\"type\":\"int\"},{\"key\":\"name\",\"type\":\"string\"}]}", + "partition" : "pt", + "partitionType" : "DAY", + "defaultFS" : "hdfs://ns1", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", + "dfs.nameservices" : "ns1", + "fs.hdfs.impl.disable.cache" : "true", + "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + } + } + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 0, + "percentage": 0 + }, + "speed": { + "bytes": 1048576, + "channel": 1 + } + } + } +} +``` diff --git a/docs/offline/writer/kuduwriter.md b/docs/offline/writer/kuduwriter.md new file mode 100644 index 0000000000..8bba48abf3 --- /dev/null +++ b/docs/offline/writer/kuduwriter.md @@ -0,0 +1,194 @@ +# Kudu Writer + + +## 一、插件名称 +名称:**kuduwriter**
+ +## 二、支持的数据源版本 +**kudu 1.10及以上**
+ + +## 三、参数说明 + +- **column** + - 描述:需要生成的字段 + - 属性说明: + - name:字段名称; + - type:字段类型; + - 必选:是 + - 默认值:无 + + + +- **masterAddresses** + - 描述: master节点地址:端口,多个以,隔开 + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述: kudu表名 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述: kudu数据写入模式: + - 1、insert + - 2、update + - 3、upsert + - 必选:是 + - 默认值:无 + + + +- **flushMode** + - 描述: kudu session刷新模式: + - 1、auto_flush_sync + - 2、auto_flush_background + - 3、manual_flush + - 必选:否 + - 默认值:auto_flush_sync + + + +- **batchInterval** + - 描述: 单次批量写入数据条数 + - 必选:否 + - 默认值:1 + + + +- **authentication** + - 描述: 认证方式,如:Kerberos + - 必选:否 + - 默认值:无 + + + +- **principal** + - 描述: 用户名。 + - 必选:否 + - 默认值:无 + + + +- **keytabFile** + - 描述: keytab文件路径 + - 必选:否 + - 默认值:无 + + + +- **workerCount** + - 描述: worker线程数 + - 必选:否 + - 默认值:默认为cpu*2 + + + +- **bossCount** + - 描述: boss线程数 + - 必选:否 + - 默认值:1 + + + +- **operationTimeout** + - 描述: 普通操作超时时间 + - 必选:否 + - 默认值:30000 + + + +- **adminOperationTimeout** + - 描述: 管理员操作(建表,删表)超时时间 + - 必选:否 + - 默认值:30000 + + + +- **queryTimeout** + - 描述: 连接scan token的超时时间 + - 必选:否 + - 默认值:与operationTimeout一致 + + + +- **batchSizeBytes** + - 描述: kudu scan一次性最大读取字节数 + - 必选:否 + - 默认值:1048576 + + + + +## 四、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "id" + }, { + "name" : "data", + "type" : "string" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "parameter": { + "column": [ + { + "name": "id", + "type": "long" + } + ], + "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", + "table": "kudu", + "writeMode": "insert", + "flushMode": "manual_flush", + "batchInterval": 10000, + "authentication": "", + "principal": "", + "keytabFile": "", + "workerCount": 2, + "bossCount": 1, + "operationTimeout": 30000, + "adminOperationTimeout": 30000, + "queryTimeout": 30000, + "batchSizeBytes": 1048576 + } + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/mongodbwriter.md b/docs/offline/writer/mongodbwriter.md new file mode 100644 index 0000000000..fc49b2cd74 --- /dev/null +++ b/docs/offline/writer/mongodbwriter.md @@ -0,0 +1,309 @@ +# MongoDB Writer + + +## 一、插件名称 +名称:**mongodbwriter**
+ +## 二、支持的数据源版本 +**MongoDB 3.4及以上**
+ +## 三、参数说明
+ +- **url** + - 描述:MongoDB数据库连接的URL字符串,详细请参考[MongoDB官方文档](https://docs.mongodb.com/manual/reference/connection-string/) + - 必选:否 + - 默认值:无 + + + +- **hostPorts** + - 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:否 + - 默认值:无 + + + +- **database** + - 描述:数据库名称 + - 必选:否 + - 默认值:无 + + + +- **collectionName** + - 描述:集合名称 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列 + - name:Column 的名字 + - type:Column 的类型 + - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中 + - 必选:是 + - 默认值:无 + + + +- **replaceKey** + - 描述:replaceKey 指定了每行记录的业务主键,用来做覆盖时使用(不支持 replaceKey为多个键,一般是指Monogo中的主键) + - 必选:否 + - 默认值:无 + + + +- **writeMode** + - 描述:写入模式,当 batchSize > 1 时不支持 replace 和 update 模式 + - 必选:是 + - 所有选项:insert/replace/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与MongoDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1 + + + + +## 四、配置示例 + +#### 1、insert +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "terminal_id", + "type" : "id" + }, { + "name" : "longitude", + "type" : "double" + }, { + "name" : "latitude", + "type" : "double" + }, { + "name" : "created_at", + "type" : "date" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "parameter" : { + "url": "mongodb://root:admin@kudu4:27017/admin?authSource=admin", + "collectionName" : "tudou", + "column" : [ { + "name" : "terminal_id", + "type" : "int" + }, { + "name" : "longitude", + "type" : "double" + }, { + "name" : "latitude", + "type" : "double" + }, { + "name" : "created_at", + "type" : "date" + } ], + "writeMode": "insert", + "batchSize": 100 + }, + "name" : "mongodbwriter" + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、update +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "terminal_id", + "type" : "id" + }, { + "name" : "longitude", + "type" : "double", + "value": 1.0 + }, { + "name" : "latitude", + "type" : "double", + "value": 2.0 + }, { + "name" : "created_at", + "type" : "date" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "parameter" : { + "url": "mongodb://root:admin@kudu4:27017/admin?authSource=admin", + "collectionName" : "tudou", + "column" : [ { + "name" : "terminal_id", + "type" : "int" + }, { + "name" : "longitude", + "type" : "double" + }, { + "name" : "latitude", + "type" : "double" + }, { + "name" : "created_at", + "type" : "date" + } ], + "writeMode": "update", + "replaceKey": "terminal_id", + "batchSize": 1 + }, + "name" : "mongodbwriter" + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、replace +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "terminal_id", + "type" : "id" + }, { + "name" : "longitude", + "type" : "double", + "value": 1.0 + }, { + "name" : "latitude", + "type" : "double", + "value": 3.0 + }, { + "name" : "created_at", + "type" : "date" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "parameter" : { + "url": "mongodb://root:admin@kudu4:27017/admin?authSource=admin", + "collectionName" : "tudou", + "column" : [ { + "name" : "terminal_id", + "type" : "int" + }, { + "name" : "longitude", + "type" : "double" + }, { + "name" : "latitude", + "type" : "double" + }, { + "name" : "created_at", + "type" : "date" + } ], + "writeMode": "replace", + "replaceKey": "terminal_id", + "batchSize": 1 + }, + "name" : "mongodbwriter" + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/mysqlwriter.md b/docs/offline/writer/mysqlwriter.md new file mode 100644 index 0000000000..c7342458cb --- /dev/null +++ b/docs/offline/writer/mysqlwriter.md @@ -0,0 +1,305 @@ +# MySQL Writer + + +## 一、插件名称 +名称:**mysqlwriter**
+ +## 二、支持的数据源版本 +**MySQL 5.X**
+ +## 三、参数说明
+ +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + - 必选:是 + - 所有选项:insert/replace/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": ["truncate table kudu;"], + "postSql": ["update kudu set user_id = 1;"], + "writeMode": "insert", + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "update", + "updateKey": {"key": ["id"]}, + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、replace +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "replace", + "updateKey": {"key": ["id"]}, + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/writer/odpswriter.md b/docs/offline/writer/odpswriter.md new file mode 100644 index 0000000000..64e14e4c4e --- /dev/null +++ b/docs/offline/writer/odpswriter.md @@ -0,0 +1,118 @@ +# ODPS Writer + + +## 一、插件名称 +名称:**odpswriter**
+ +## 二、参数说明 + +- **odpsConfig** + - 描述:ODPS系统配置参数,包含以下参数 + - accessId:ODPS系统登录ID + - accessKey:ODPS系统登录Key + - project:读取数据表所在的 ODPS 项目名称(大小写不敏感) + - packageAuthorizedProject:ODPS认证项目,不填默认为project值 + - accountType:ODPS账户类型,默认为aliyun + - odpsServer:ODPS服务URL,默认为[http://service.odps.aliyun.com/api](http://service.odps.aliyun.com/api) + - 必选:是 + - 默认值:无 + + + +- **table** + - 描述:读取数据表的表名称(大小写不敏感) + - 必选:是 + - 默认值:无 + + +
+ +- **partition** + - 描述:需要写入数据表的分区信息,必须指定到最后一级分区。把数据写入一个三级分区表,必须配置到最后一级分区,例如pt=20150101/type=1/biz=2。 + - 必选:**如果是分区表,该选项必填,如果非分区表,该选项不可填写。** + - 默认值:空 + + +
+ +- **column** + - 描述:需要导入的字段列表,当导入全部字段时,可以配置为"column": ["*"], 当需要插入部分odps列填写部分列,例如"column": ["id", "name"]。ODPSWriter支持列筛选、列换序,例如表有a,b,c三个字段,用户只同步c,b两个字段。可以配置成["c","b"], 在导入过程中,字段a自动补空,设置为null。 + - 必选:否 + - 默认值:无 + + +
+ +- **writeMode** + - 描述:写入模式,支持append和overwrite + - 必填:否 + - 默认值:append + + + +- **bufferSize** + - 描述:写入缓存大小,单位兆,odps写入数据时会先缓存,达到一定值后才会写入数据,如果写入数据时出现内存溢出,可以降低此参数的值。 + - 必填:否 + - 默认值:64 + + + + +## 三、配置示例 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "data", + "type" : "string" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer" : { + "name": "odpswriter", + "parameter": { + "odpsConfig": { + "accessId": "${odps.accessId}", + "accessKey": "${odps.accessKey}", + "project": "${odps.project}" + }, + "table": "tableTest", + "partition": "pt='xx'", + "writeMode": "append", + "bufferSize": 64, + "column": [{ + "name": "col1", + "type": "string" + }] + } + } + } ], + "setting" : { + "restore" : { + "maxRowNumForCheckpoint" : 0, + "isRestore" : false, + "restoreColumnName" : "", + "restoreColumnIndex" : 0 + }, + "errorLimit" : { + "record" : 100 + }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/oraclewriter.md b/docs/offline/writer/oraclewriter.md new file mode 100644 index 0000000000..3161b3dd3d --- /dev/null +++ b/docs/offline/writer/oraclewriter.md @@ -0,0 +1,230 @@ +# Oracle Writer + + +## 一、插件名称 +名称:**oraclewriter**
+ +## 二、支持的数据源版本 +**Oracle 9 及以上**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + - 必选:否 + - 默认值:无 + +** + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "oraclewriter", + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:oracle:thin:@kudu5:1521:helowin", + "table": ["TUDOU.KUDU"] + } + ], + "preSql": ["delete from TUDOU.KUDU"], + "postSql": ["update TUDOU.KUDU set USER_ID = 1"], + "writeMode": "insert", + "column": ["ID","USER_ID","NAME"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "oraclewriter", + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:oracle:thin:@kudu5:1521:helowin", + "table": ["TUDOU.KUDU"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "update", + "updateKey": {"key": ["ID"]}, + "column": ["ID","USER_ID","NAME"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/phoenixwriter.md b/docs/offline/writer/phoenixwriter.md new file mode 100644 index 0000000000..5a1583dceb --- /dev/null +++ b/docs/offline/writer/phoenixwriter.md @@ -0,0 +1,248 @@ +# Phoenix Writer + + +## 一、插件名称 +名称:**phoenixwriter**
+ +## 二、支持的数据源版本 +phoenix4.12.0-HBase-1.3及之后
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "id", + "type": "int", + "value": "400" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "phoenixwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:phoenix:node01,node02,node03:2181", + "table": [ + "tableTest" + ] + }], + "username": "", + "password": "", + "column": [ + { + "name": "id", + "type": "BIGINT", + "key": "id" + }, + { + "name": "user_id", + "type": "BIGINT", + "key": "user_id" + }, + { + "name": "name", + "type": "varchar", + "key": "name" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "id", + "type": "int", + "value": "400" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "phoenixwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:phoenix:node01,node02,node03:2181", + "table": [ + "tableTest" + ] + }], + "username": "", + "password": "", + "column": [ + { + "name": "id", + "type": "BIGINT", + "key": "id" + }, + { + "name": "user_id", + "type": "BIGINT", + "key": "user_id" + }, + { + "name": "name", + "type": "varchar", + "key": "name" + }], + "writeMode": "update", + "updateKey": {"key": ["id"]}, + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} +``` diff --git a/docs/offline/writer/polardbwriter.md b/docs/offline/writer/polardbwriter.md new file mode 100644 index 0000000000..4028d820fc --- /dev/null +++ b/docs/offline/writer/polardbwriter.md @@ -0,0 +1,303 @@ +# PolarDB Writer + + +## 一、插件名称 +名称:**polardbwriter**
+ +## 二、支持的数据源版本 +**PolarDB MySQL 8.0**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + - 必选:是 + - 所有选项:insert/replace/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "polarwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": ["truncate table kudu;"], + "postSql": ["update kudu set user_id = 1;"], + "writeMode": "insert", + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "polardbwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "update", + "updateKey": {"key": ["id"]}, + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、replace +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "polardbwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": ["kudu"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "replace", + "updateKey": {"key": ["id"]}, + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/postgresqlwriter.md b/docs/offline/writer/postgresqlwriter.md new file mode 100644 index 0000000000..cc567de731 --- /dev/null +++ b/docs/offline/writer/postgresqlwriter.md @@ -0,0 +1,228 @@ +# PostgreSql Writer + + +## 一、插件名称 +名称:**postgresqlwriter**
+ +## 二、支持的数据源版本 +**PostgreSql 9.4及以上**
+ +## 三、参数说明
+ +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:仅支持`insert`操作,可以搭配insertSqlMode使用 + - 必选:是 + - 默认值:无 + + + +- **insertSqlMode** + - 描述:控制写入数据到目标表采用  `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 + - 注意: + - 此参数只针对PostgreSQL写入插件有效 + - 目前该参数值固定传入 `copy`,否则抛出提示为`not support insertSqlMode`的`RuntimeException` + - 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 + - 必选:否 + - 默认值:无 + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + +** + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type:"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "postgresqlwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:postgresql://0.0.0.1:5432/postgres", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、 insert with copy mode +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type:"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "postgresqlwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:postgresql://0.0.0.1:5432/postgres", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "insertSqlMode": "copy" + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + + diff --git a/docs/pulsarwriter.md b/docs/offline/writer/pulsarwriter.md similarity index 100% rename from docs/pulsarwriter.md rename to docs/offline/writer/pulsarwriter.md diff --git a/docs/offline/writer/rediswriter.md b/docs/offline/writer/rediswriter.md new file mode 100644 index 0000000000..416981a19a --- /dev/null +++ b/docs/offline/writer/rediswriter.md @@ -0,0 +1,167 @@ +# Redis Writer + + +## 一、插件名称 +名称:**rediswriter**
+ +## 二、支持的数据源版本 +**Redis 2.9及以上**
+ +## 三、参数说明 + +- **hostPort** + - 描述:Redis的IP地址和端口 + - 必选:是 + - 默认值:localhost:6379 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **database** + - 描述:要写入Redis数据库 + - 必选:否 + - 默认值:0 + + + +- **keyFieldDelimiter** + - 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 + - 必选:否 + - 默认值:\u0001 + + + +- **dateFormat** + - 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” + - 必选:否 + - 默认值:将日期以long类型写入 + + + +- **expireTime** + - 描述:Redis value 值缓存失效时间(如果需要永久有效则可以不填该配置项)。 + - 注意:如果过期时间的秒数大于 60_60_24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 + - 必选:否 + - 默认值:0(0 表示永久有效) + + + +- **timeout** + - 描述:写入 Redis 的超时时间。 + - 单位:毫秒 + - 必选:否 + - 默认值:30000 + + + +- **type和mode** + - 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 + - 选项:string/list/set/zset/hash +| type | 描述 | mode | 说明 | 注意 | +| --- | --- | --- | --- | --- | +| string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | +| list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | +| list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | +| set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | +| zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | +| hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | + + - 必选:是 + - 默认值:无 + + + +- **valueFieldDelimiter** + - 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 + - 必选:否 + - 默认值:\u0001 + + + +- **keyIndexes** + - 描述:keyIndexes 表示源端哪几列需要作为 key(第一列是从 0 开始)。如果是第一列和第二列需要组合作为 key,那么 keyIndexes 的值则为 [0,1]。 + - 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。例如:Redis中的数据为 "test,redis,First,Second",keyIndexes = [0,1] ,因此得到的key为 "test\\u0001redis", value为 "First\\u0001Second" + - 必选:是 + - 默认值:无 + + +
+ + +## 四、 使用示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "key1", + "type": "string" + }, + { + "name": "key2", + "type": "string" + }, + { + "name": "key3", + "type": "string" + }, + { + "name": "key4", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "hostPort": "180.153.62.13:6379", + "type": "string", + "mode": "set", + "keyIndexes": [0,1], + "password": "123456", + "database": 0, + "timeout": 30000 + }, + "name": "rediswriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` + + + diff --git a/docs/offline/writer/saphanawriter.md b/docs/offline/writer/saphanawriter.md new file mode 100644 index 0000000000..487c69e0b7 --- /dev/null +++ b/docs/offline/writer/saphanawriter.md @@ -0,0 +1,162 @@ +# Saphana Writer + + +## 一、插件名称 +名称:**saphanawriter**
+ +## 二、支持的数据源版本 +SAP HANA 2.0及以上
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + - 必选:否 + - 默认值:无 + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "id" + }, { + "name" : "CONTEXT", + "type" : "string" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer": { + "name": "saphanawriter", + "parameter": { + "connection": [ + { + "jdbcUrl": "jdbc:sap://kudu3:39017", + "table": [ + "SYS.P_ROLES_" + ] + } + ], + "username": "username", + "password": "password", + "column": [ + { + "name": "ROLE_ID", + "type": "BIGINT" + }, + { + "name": "CONTEXT", + "type": "NVARCHAR" + } + ], + "writeMode": "insert", + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} + +``` + + diff --git a/docs/offline/writer/sqlserverwriter.md b/docs/offline/writer/sqlserverwriter.md new file mode 100644 index 0000000000..cd1d8aed9c --- /dev/null +++ b/docs/offline/writer/sqlserverwriter.md @@ -0,0 +1,229 @@ +# SqlServer Writer + + +## 一、插件名称 +名称:**sqlserverwriter**
+ +## 二、支持的数据源版本 +**Microsoft SQL Server 2012及以后**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:使用开源的jtds驱动连接 而非Microsoft的官方驱动
jdbcUrl参考文档:[jtds驱动官方文档](http://jtds.sourceforge.net/faq.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **presql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + +
+ +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者` merge into` 语句 + - 必选:是 + - 所有选项:insert/update + - 默认值:insert + + + +- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 默认值:无 + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + +
** + +## 四、配置示例 + +#### 1、insert +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type:"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "sqlserverwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、update +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type:"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "sqlserverwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack", + "table": [ + "tableTest" + ] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "update", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {"key": ["id"]} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + + diff --git a/docs/offline/writer/streamwriter.md b/docs/offline/writer/streamwriter.md new file mode 100644 index 0000000000..91f8e22f34 --- /dev/null +++ b/docs/offline/writer/streamwriter.md @@ -0,0 +1,80 @@ +# Stream Writer + + +## 一、 简介 +Stream writer插件仅仅用来测试reader插件的读取效果,处理方式为简单的将读到的数据弃掉,可选择是否通过LOG显示读取到的数据
+ + +## 二、插件名称 +名称:**streamwriter**
+ +## 三、 参数说明 + +- **print** + - 描述:boolean值,表明是否通过LOG INFO打印采集到的数据信息 + - 必选:否 + - 默认值:false + + + + +## 四、 配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/offline/writer/teradatawriter.md b/docs/offline/writer/teradatawriter.md new file mode 100644 index 0000000000..e3f6616d53 --- /dev/null +++ b/docs/offline/writer/teradatawriter.md @@ -0,0 +1,156 @@ +# Teradata Writer + + +## 一、插件名称 +名称:**teradatawriter**
+ +## 二、支持的数据源版本 +**Teradata 15.0及以上**
+ +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 + - 必选:是 + - 所有选项:只支持insert + - 默认值:insert + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + + + +- **updateKey** + - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 + - 注意: + - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; + - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; + - 必选:否 + - 默认值:无 + +** + +## 四、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "teradatawriter", + "parameter": { + "username": "tudou", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:teradata://kudu3/testbase", + "table": ["Employee"] + } + ], + "preSql": ["delete from TUDOU.KUDU"], + "postSql": ["update TUDOU.KUDU set USER_ID = 1"], + "writeMode": "insert", + "column": ["ID","USER_ID","NAME"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/pgwalreader.md b/docs/pgwalreader.md deleted file mode 100644 index 56fbadbc84..0000000000 --- a/docs/pgwalreader.md +++ /dev/null @@ -1,184 +0,0 @@ -# PostgreSQL WAL读取插件(*reader) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [{ - "reader" : { - "parameter" : { - "username" : "postgres", - "password" : "abc123", - "url" : "jdbc:postgresql://172.16.8.122:5432/postgres", - "databaseName" : "postgres", - "cat" : "update,insert,delete", - "tableList" : [ - "changepk.test_table" - ], - "statusInterval" : 10000, - "lsn" : 0, - "slotName" : "", - "allowCreateSlot" : true, - "temporary" : true, - "pavingData" : true - }, - "name" : "pgwalreader" - }, - "writer": { - - } - }] - }, - "setting": { - - } -} -``` -## 2. 使用说明 - 1、PostgreSQL数据库版本至少为10.0及以上 - - 2、预写日志级别(wal_level)必须为logical - - 3、该插件基于PostgreSQL逻辑复制及逻辑解码功能实现的,因此PostgreSQL账户至少拥有replication权限,若允许创建slot,则至少拥有超级管理员权限 - - 4、详细原理请参见[PostgreSQL官方文档](http://postgres.cn/docs/10/index.html) - -## 3. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称。 - - * 必选:是 - - * 默认值:无 - -* **jdbcUrl** - - * 描述:PostgreSQL数据库的jdbc连接字符串,参考文档:[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **tableList** - - * 描述:需要解析的数据表,格式为schema.table - - * 必选:否 - - * 默认值:无 - -* **cat** - - * 描述:需要解析的数据更新类型,包括insert、update、delete三种。 - - * 注意:以英文逗号分割的格式填写。 - - * 必选:是 - - * 默认值:无 - -* **statusInterval** - - * 描述:复制期间,数据库和使用者定期交换ping消息。如果数据库或客户端在配置的超时时间内未收到ping消息,则复制被视为已停止,并且将引发异常,并且数据库将释放资源。在PostgreSQL中,ping超时由属性wal_sender_timeout配置(默认= 60秒)。可以将pgjdc中的复制流配置为在需要时或按时间间隔发送反馈(ping)。建议比配置的wal_sender_timeout更频繁地向数据库发送反馈(ping)。在生产环境中,我使用等于wal_sender_timeout / 3的值。它避免了网络潜在的问题,并且可以在不因超时而断开连接的情况下传输更改。 - - * 必选:否 - - * 默认值:2000 - -* **lsn** - - * 描述:要读取PostgreSQL WAL日志序列号的开始位置。 - - * 必选:否 - - * 默认值:0 - -* **slotName** - - * 描述:复制槽名称,根据该值去寻找或创建复制槽 - - * 注意:当allowCreateSlot为false时,该值不能为空 - - * 必选:否 - - * 默认值:无 - -* **allowCreateSlot** - - * 描述:是否允许创建复制槽 - - * 必选:否 - - * 默认值:true - -* **temporary** - - * 描述:复制槽是否为临时性的,true:是;false:否。 - - * 必选:否 - - * 默认值:true - -* **pavingData** - - * 描述:是否将解析出的json数据拍平 - - * 示例:假设解析的表为tb1,schema为dbo,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: - - ```json - { - "type":"update", - "schema":"dbo", - "table":"customers", - "lsn":207967352, - "ts": 1576487525488, - "ingestion":1475129582923642, - "before_id":1, - "after_id":2 - } - ``` - - pavingData为false时: - - ```json - { - "type":"update", - "schema":"dbo", - "table":"customers", - "lsn":207967352, - "ts": 1576487525488, - "ingestion":1481628798880038, - "before":{ - "id":1 - }, - "after":{ - "id":2 - } - } - ``` - - 其中:ts是数据库中数据的变更时间,ingestion是插件解析这条数据的纳秒时间,lsn是该数据变更的日志序列号 - - * 必选:否 - - * 默认值:false - diff --git a/docs/quickstart.md b/docs/quickstart.md new file mode 100644 index 0000000000..fe581442a0 --- /dev/null +++ b/docs/quickstart.md @@ -0,0 +1,248 @@ +## 下载代码 + +1.使用git工具把项目clone到本地 + +``` +git clone https://github.com/DTStack/flinkx.git +cd flinkx +``` + +2.直接下载源码 + +``` +wget https://github.com/DTStack/flinkx/archive/1.8.5.zip +unzip flinkx-1.8.5.zip +cd flink-1.8.5 +``` + +## 编译插件 + +```bash +mvn clean package -DskipTests +``` + +## 运行任务 + +首先准备要运行的任务json,这里以stream插件为例: + +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ] + } + }, + "writer" : { + "parameter" : { + "print": false + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : false + }, + "errorLimit" : { + }, + "speed" : { + "bytes" : 0, + "channel" : 1, + "rebalance" : true + } + } + } +} +``` + +### Local模式运行任务 + +命令模板: + +```bash +bin/flinkx -mode local \ + -job $FLINK_HOME/examples/stream_example.json \ + -pluginRoot $FLINK_HOME/plugins \ + -confProp "{\"flink.checkpoint.interval\":60000}" +``` + +可以在flink的配置文件里配置端口: + +```bash +## web服务端口,不指定的话会随机生成一个 +rest.bind-port: 8888 +``` + +使用下面的命令运行任务: + +```bash +./bin/flinkx -job ./job/stream.json + -flinkconf $FLINK_HOME/conf +``` + +任务运行后可以通过8888端口访问flink界面查看任务运行情况: + +
+ +
+ +### Standalone模式运行 + +命令模板: + +```bash +bin/flinkx -mode standalone \ + -job $FLINK_HOME/examples/stream_example.json \ + -pluginRoot $FLINK_HOME/plugins \ + -flinkconf $FLINK_HOME/conf \ + -confProp "{\"flink.checkpoint.interval\":60000}" +``` + +首先启动flink集群: + +```bash +# flink集群默认端口是8081 +$FLINK_HOME/bin/start-cluster.sh +``` + +通过8081端口检查集群是否启动成功 + +
+ +
+ +把任务提交到集群上运行: + +```bash +./bin/flinkx -mode standalone \ + -job ./job/stream.json \ + -flinkconf $FLINK_HOME/conf +``` + +在集群上查看任务运行情况 + +
+ +
+ +### 以Yarn Session模式运行任务 + +命令示例: + +```bash +bin/flinkx -mode yarn \ + -job $FLINK_HOME/examples/stream_example.json \ + -pluginRoot $FLINK_HOME/plugins \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -confProp "{\"flink.checkpoint.interval\":60000}" +``` + +首先确保yarn集群是可用的,然后手动启动一个yarn session: + +```bash +$FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024 +``` + +
+ +
+ +
+ +
+ +把任务提交到这个yarn session上: + +```bash +./bin/flinkx -mode yarn \ + -job ./job/stream.json \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop +``` + +然后在flink界面查看任务运行情况: + +
+ +
+ +### 以Yarn Perjob模式运行任务 + +命令示例: + +```bash +bin/flinkx -mode yarnPer \ + -job ${FLINK_HOME}/examples/stream_example.json \ + -pluginRoot $FLINK_HOME/plugins \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -flinkLibJar $FLINK_HOME/lib \ + -confProp "{\"flink.checkpoint.interval\":60000}" \ + -queue default \ + -pluginLoadMode classpath +``` + +首先确保yarn集群是可用的,启动一个Yarn Application运行任务: + +```bash +bin/flinkx -mode yarnPer \ + -job ./job/stream.json \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -flinkLibJar $FLINK_HOME/lib \ + -pluginLoadMode classpath +``` + +然后在集群上查看任务运行情况 + +
+ +
+ +
+ +
+ +## 参数说明 + +| 名称 | 说明 | 可选值 | 是否必填 | 默认值 | +| ------------------ | ------------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---- | ----------------------- | +| **model** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | +| **job** | 数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息 | 无 | 是 | 无 | +| **jobid** | 任务名称 | 无 | 否 | Flink Job | +| **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/plugins | +| **flinkconf** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | +| **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | +| **yarnconf** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | +| **queue** | yarn队列,如default | 无 | 否 | default | +| **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | +| **confProp** | checkpoint配置 | **flink.checkpoint.interval**:快照生产频率
**flink.checkpoint.stateBackend**:快照存储路径 | 否 | 无 | +| **s** | checkpoint快照路径 | | 否 | 无 | + +## 常见问题 + +### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 + +解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: + +```bash +## windows平台 +./install_jars.bat + +## unix平台 +./install_jars.sh +``` diff --git a/docs/rdbreader.md b/docs/rdbreader.md deleted file mode 100644 index a932b6f8c4..0000000000 --- a/docs/rdbreader.md +++ /dev/null @@ -1,276 +0,0 @@ -# 关系数据库读取插件(*reader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [{ - "reader": { - "parameter": { - "username": "username", - "password": "password", - "connection": [{ - "jdbcUrl": [ - "jdbc:mysql://127.0.0.1:3306/test?useUnicode=true&characterEncoding=utf8" - ], - "table": [ - "tableTest" - ] - }], - "column": [{ - "name": "id", - "type": "int", - "values": 123 - },{ - "name":"", - "index":1, - "type":"", - "value":"", - "format":"" - }], - "where": "id > 1", - "splitPk": "id", - "fetchSize": 1000, - "queryTimeOut": 1000, - "customSql": "select * from tableTest", - "requestAccumulatorInterval": 2, - "increColumn": "id", - "startLocation": null, - "useMaxFunc": true, - "orderByColumn": "id", - "polling": true, - "pollingInterval": 3000 - }, - "name": "mysqlreader" - }, - "writer": { - - } - }], - "setting": { - - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,当前支持的关系数据库插件包括:mysqlreader,oraclereader,sqlserverreader,postgresqlreader,db2reader,gbasereader, clickhousereader, polardbreader,teradatareader,saphanareader,phoenixreader,dmreader。 - * 必选:是 - - * 默认值:无 - -* **jdbcUrl** - - * 描述:针对关系型数据库的jdbc连接字符串 - - jdbcUrl参考文档: - - - [Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - - - [Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) - - - [SqlServer官方文档](https://docs.microsoft.com/zh-cn/sql/connect/jdbc/overview-of-the-jdbc-driver?view=sql-server-2017) - - - [PostgreSql官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) - - - [Db2官方文档](https://www.ibm.com/analytics/us/en/db2/) - - - [Gbase官方文档](http://www.gbase.cn/download.html) - - - [ClickHouse官方文档](https://clickhouse.yandex/docs/zh/) - - - [polarDB官方文档](https://help.aliyun.com/product/58609.html) - - - [达梦官方文档](http://www.dameng.com/down.aspx) - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **where** - - * 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - - * 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - - * 必选:否 - - * 默认值:无 - -* **splitPk** - - * 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - - * 注意: - - * 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - * 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,MysqlReader将报错! - * 如果channel大于1但是没有配置此参数,任务将置为失败。 - - * 必选:否 - - * 默认值:空 - -* **fetchSize** - - * 描述:读取时每批次读取的数据条数。 - - * 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - - * 必选:否 - - * 默认值:mysql为0,表示流式读取,其它数据库为1000 - -* **queryTimeOut** - - * 描述:查询超时时间,单位秒。 - - * 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - - * 必选:否 - - * 默认值:1000s - -* **customSql** - - * 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 - - * 注意: - - * 只能是查询语句,否则会导致任务失败; - - * 查询语句返回的字段需要和column列表里的字段对应; - - * 当指定了此参数时,connection里指定的table无效,但是在一些情况下依然必须指定,比如使用增量同步的时候; - - * 必选:否 - - * 默认值:null - -* **increColumn** - - * 描述:当需要增量同步或间隔轮询时指定此参数,任务运行过程中会把此字段的值存储到flink的Accumulator里,如果配置了指标,名称为:endLocation,类型为string,日期类型会转为时间戳,精度最多到纳秒,数值类型的为字段的值,程序结束时由外部应用获取。 - - * 注意: - - * 指定的字段必须在column列表里存在,否则任务会失败; - - * 增量字段支持数值类型和日期类型,并且是升序的,推荐使用表主键; - - * 必选:否 - - * 默认值:无 - -* **startLocation** - - * 描述:此配置参数和increColumn参数配合使用,表示本次任务获取数据的开始位置。 - - * 注意: - - * 此参数为空时进行全量同步 - - * 必选:否 - - * 默认值:无 - -* **useMaxFunc** - - * 描述:进行增量同步任务时,如果指定的字段值存在重复值,比如字段类型为时间,精度到秒,就可能出现重复的时间,需要指定此字段为true,读取数据前会获取增量字段的最大值作为此次任务的结束位置,防止数据丢失。 - - * 注意: - - * 此参数设为true时,会执行select max(increCol) from tb语句,会影响数据库负载,配置时需要考虑数据库的使用情况; - - * 此参数设置为true时,本次任务不会读取 increCol = max(increCol)的记录,会在任务下次运行时读取; - - * 必选:否 - - * 默认:false - -* **column** - - * 描述:需要读取的字段。 - - * 格式:支持3中格式 - - 1.读取全部字段,如果字段数量很多,可以使用下面的写法: - - ``` - "column":[*] - ``` - - 2.只指定字段名称: - - ``` - "column":["id","name"] - ``` - - 3.指定具体信息: - - ``` - "column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" - }] - ``` - - * 属性说明: - - * name:字段名称 - - * type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - * format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - * value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - * 必选:是 - - * 默认值:无 - -* **orderByColumn** - - * 描述:排序字段,读取PostgreSQL数据时,如果中途任务失败,没有关闭事务,会导致表里的数据顺序改变,再次运行任务时由于数据顺序不对会影响数据的准确性,因此使用orderByColumn指定的字段进行排序避免这种情况。 - - * 必选:否 - - * 默认值:无 - -* **polling** - - * 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - - * 必选:否 - - * 默认值:false - -* **pollingInterval** - - * 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - - * 必选:否 - - * 默认值:5000 diff --git a/docs/rdbwriter.md b/docs/rdbwriter.md deleted file mode 100644 index 6a19f80655..0000000000 --- a/docs/rdbwriter.md +++ /dev/null @@ -1,151 +0,0 @@ -# 关系数据库写入插件(*writer) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [{ - "reader": {}, - "writer": { - "name": "*writer", - - "parameter": { - "connection": [{ - "jdbcUrl": "jdbc:mysql://127.0.0.1:3306/test?useCursorFetch=true", - "table": [ - "tableTest" - ] - }], - "username": "username", - "password": "password", - "column": [], - - "writeMode": "insert", - "batchSize": 1024, - "preSql": "", - "postSql": "", - "updateKey": "" - } - } - }], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处可填写:mysqlwriter,oraclewriter,sqlserverwriter,postgresqlwriter,db2writer,gbasewriter,clickhousewriter, polardbwriter,teradatawriter,saphanawriter,phoenixwriter,dmwriter。 - - * 必选:是 - - 默认值:无 - -* **jdbcUrl** - - * 描述:针对关系型数据库的jdbc连接字符串 - - * 必选:是 - - * 默认值:无 - -* **username** - - * 描述:数据源的用户名 - - * 必选:是 - - * 默认值:无 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **column** - - * 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - - * 必选:是 - - * 默认值:否 - - * 默认值:无 - -* **preSql** - - * 描述:写入数据到目的表前,会先执行这里的一组标准语句。 - - * 必选:否 - - * 默认值:无 - -* **postSql** - - * 描述:写入数据到目的表后,会执行这里的一组标准语句。 - - * 必选:否 - - * 默认值:无 - -* **table** - - * 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表。 - - * 必选:是 - - * 默认值:无 - -* **writeMode** - - * 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - - * 必选:是 - - * 所有选项:insert/replace/update - - * 默认值:insert - -* **batchSize** - - * 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况。 - - * 必选:否 - - * 默认值:1024 - -* **updateKey** - - * 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段。 - - * 注意: - - * 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - - * 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - - * 必选:否 - - * 默认值:无 - - * **insertSqlMode** - - * 描述:控制写入数据到目标表采用 `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 - - * 注意: - - * 此参数只针对PostgreSQL写入插件有效 - - * 目前该参数值固定传入 `copy`,否则抛出提示为`not support insertSqlMode`的`RuntimeException` - - * 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 - - * 必选:否 - - * 默认值:无 diff --git a/docs/realTime/reader/binlogreader.md b/docs/realTime/reader/binlogreader.md new file mode 100644 index 0000000000..5a7269f3cc --- /dev/null +++ b/docs/realTime/reader/binlogreader.md @@ -0,0 +1,347 @@ +# MySQL Binlog Reader + + +## 一、插件名称 +名称:**binlogreader**
+ + +## 二、支持的数据源版本 +**MySQL 5.X**
+ + +## 三、数据库配置 +**1.修改配置文件** +```sql +server_id=109 +log_bin = /var/lib/mysql/mysql-bin +binlog_format = ROW +expire_logs_days = 30 +``` + +
**2.添加权限** +```sql +GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal'; +``` +
+ +## 四、参数说明
+ +- **jdbcUrl** + - 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **host** + - 描述:启动MySQL slave的机器ip + - 必选:是 + - 默认值:无 + + + +- **port** + - 描述:启动MySQL slave的端口 + - 必选:否 + - 默认值:3306 + + + +- **table** + - 描述:需要解析的数据表。 + - 注意:指定此参数后filter参数将无效 + - 必选:否 + - 默认值:无 + + + +- **filter** + - 描述:过滤表名的Perl正则表达式 + - 例子: + - 所有表:`_.*_` + - canal schema下所有表: `canal\..*` + - canal下的以canal打头的表:`canal\.canal.*` + - canal schema下的一张表:`canal\.test1` + - 必选:否 + - 默认值:无 + + + +- **cat** + - 描述:需要解析的数据更新类型,包括insert、update、delete三种 + - 注意:以英文逗号分割的格式填写。 + - 必选:否 + - 默认值:无 + + + +- **start** + - 描述:要读取的binlog文件的开始位置 + - 参数: + - journalName:采集起点按文件开始时的文件名称; + - timestamp:采集起点按时间开始时的时间戳; + - 默认值:无 + + + +- **pavingData** + - 描述:是否将解析出的json数据拍平 + - 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: +```json +{ + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before_id":1, + "after_id":2 +} +``` + +
pavingData为false时: +```json +{ + "message":{ + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before":{ + "id":1 + }, + "after":{ + "id":2 + } + } +} +``` +其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 + + - 必选:否 + - 默认值:false + + + +- **bufferSize** + - 描述:并发缓存大小 + - 注意:必须为2的幂 + - 必选:否 + - 默认值:1024 + +## 五、配置示例 + +#### 1、单表监听 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "schema" : "tudou", + "password" : "abc123", + "cat" : "insert,delete,update", + "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", + "host" : "kudu3", + "start" : { + }, + "table" : [ "binlog" ], + "pavingData" : true, + "username" : "dtstack" + }, + "name" : "binlogreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true + }, + "errorLimit" : { }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "trace", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、多表监听 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "schema" : "tudou", + "password" : "abc123", + "cat" : "insert,delete,update", + "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", + "host" : "kudu3", + "start" : { + }, + "table" : ["kudu1", "kudu2"], + "filter" : "", + "pavingData" : true, + "username" : "dtstack" + }, + "name" : "binlogreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true + }, + "errorLimit" : { }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "trace", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、正则监听 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "schema" : "tudou", + "password" : "abc123", + "cat" : "insert,delete,update", + "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", + "host" : "kudu3", + "start" : { + }, + "filter" : "tudou\\.kudu.*", + "pavingData" : true, + "username" : "dtstack" + }, + "name" : "binlogreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true + }, + "errorLimit" : { }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "trace", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、指定起始位置 +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "schema" : "tudou", + "password" : "abc123", + "cat" : "insert,delete,update", + "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", + "host" : "kudu3", + "start" : { + "journalName": "mysql-bin.000002", + "timestamp" : 1589353414000 + }, + "table" : ["kudu"], + "pavingData" : true, + "username" : "dtstack" + }, + "name" : "binlogreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true + }, + "errorLimit" : { }, + "speed" : { + "bytes" : 0, + "channel" : 1 + }, + "log" : { + "isLogger": false, + "level" : "trace", + "path" : "", + "pattern":"" + } + } + } +} +``` + + + +## 六、问题排查 +采集mysql binlog 发现采集不到数据
1、查看binlog是否开启
       `show variables like '%log_bin%' ; ` 
2、binlog_format 是否设置为ROW
        注意 binlog_format 必须设置为 ROW, 因为在 STATEMENT 或 MIXED 模式下, Binlog 只会记录和传输 SQL 语句(以减少日志大小),而不包含具体数据,我们也就无法保存了。
3、从节点通过一个专门的账号连接主节点,这个账号需要拥有全局的 REPLICATION 权限。我们可以使用 GRANT 命令创建这样的账号:
     GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT
    ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal';
  参考:[https://blog.csdn.net/zjerryj/article/details/77152226](https://blog.csdn.net/zjerryj/article/details/77152226) diff --git a/docs/realTime/reader/emqxreader.md b/docs/realTime/reader/emqxreader.md new file mode 100644 index 0000000000..a4b6ae60b2 --- /dev/null +++ b/docs/realTime/reader/emqxreader.md @@ -0,0 +1,123 @@ +# Emqx Reader + + +## 一、插件名称 +名称:**emqxreader**
+ + +## 二、支持的数据源版本 +**Emqx 4.0及以上**
+ +## 三、参数说明
+ +- **broker** + - 描述:连接URL信息 + - 必选:是 + - 默认值:无 + + + +- **topic** + - 描述:订阅主题 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:认证用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:认证密码 + - 必选:否 + - 默认值:无 + + + +- **isCleanSession** + - 描述:是否清除session + - false:MQTT服务器保存于客户端会话的的主题与确认位置; + - true:MQTT服务器不保存于客户端会话的的主题与确认位置 + - 必选:否 + - 默认值:true + + + +- **qos** + - 描述:服务质量 + - 0:AT_MOST_ONCE,至多一次; + - 1:AT_LEAST_ONCE,至少一次; + - 2:EXACTLY_ONCE,精准一次; + - 必选:否 + - 默认值:2 + + + +- **codec** + - 描述:编码解码器类型,支持 json、plain + - plain:将kafka获取到的消息字符串存储到一个key为message的map中,如:`{"message":"{\"key\": \"key\", \"message\": \"value\"}"}` + - plain:将kafka获取到的消息字符串按照json格式进行解析 + - 若该字符串为json格式 + - 当其中含有message字段时,原样输出,如:`{"key": "key", "message": "value"}` + - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,如:`{"key": "key", "value": "value", "message": "{\"key\": \"key\", \"value\": \"value\"}"}` + - 若改字符串不为json格式,则按照plain类型进行处理 + - 必选:否 + - 默认值:plain + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "broker" : "tcp://0.0.0.1:1883", + "topic" : "test", + "username" : "username", + "password" : "password", + "isCleanSession": true, + "qos": 2, + "codec": "plain" + }, + "name" : "emqxreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +
diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md new file mode 100644 index 0000000000..6e110b52e7 --- /dev/null +++ b/docs/realTime/reader/kafkareader.md @@ -0,0 +1,396 @@ +# Kafka Reader + + +## 一、插件名称 +kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: + +| kafka版本 | 插件名称 | +| --- | --- | +| kafka 0.9 | kafka09reader | +| kafka 0.10 | kafka10reader | +| kafka 0.11 | kafka11reader | +| kafka 1.0及以后 | kafkareader | + +
+ +二、参数说明
+ +- **topic** + - 描述:要消费的topic + - 必选:是 + - 默认值:无 + + + +- **groupId** + - 描述:kafka消费组Id + - 注意:该参数对kafka09reader插件无效 + - 必选:是 + - 默认值:无 + + + +- **encoding** + - 描述:字符编码 + - 注意:该参数只对kafka09reader插件有效 + - 必选:否 + - 默认值:UTF-8 + + + +- **codec** + - 描述:编码解码器类型,支持 json、plain + - plain:将kafka获取到的消息字符串存储到一个key为message的map中,如:`{"message":"{\"key\": \"key\", \"message\": \"value\"}"}` + - json:将kafka获取到的消息字符串按照json格式进行解析 + - 若该字符串为json格式 + - 当其中含有message字段时,原样输出,如:`{"key": "key", "message": "value"}` + - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,如:`{"key": "key", "value": "value", "message": "{\"key\": \"key\", \"value\": \"value\"}"}` + - 若改字符串不为json格式,则按照plain类型进行处理 + - 必选:否 + - 默认值:plain + + + +- **blankIgnore** + - 描述:是否忽略空值消息 + - 必选:否 + - 默认值:false + + + +- **consumerSettings** + - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +## 三、配置示例 + +#### 1、kafka09 +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "topic" : "kafka09", + "codec": "plain", + "encoding": "UTF-8", + "consumerSettings" : { + "zookeeper.connect" : "0.0.0.1:2182/kafka09", + "group.id" : "default", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "smallest" + } + }, + "name" : "kafka09reader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、kafka10 +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "topic" : "kafka10", + "groupId" : "default", + "codec": "plain", + "blankIgnore": false, + "consumerSettings" : { + "zookeeper.connect" : "0.0.0.1:2182/kafka", + "bootstrap.servers" : "0.0.0.1:9092", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "latest" + } + }, + "name" : "kafka10reader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、kafka11 +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "topic" : "kafka11", + "groupId" : "default", + "codec": "plain", + "blankIgnore": false, + "consumerSettings" : { + "zookeeper.connect" : "0.0.0.1:2182/kafka", + "bootstrap.servers" : "0.0.0.1:9092", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "latest" + } + }, + "name" : "kafka11reader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、kafka +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "topic" : "kafka", + "groupId" : "default", + "codec": "plain", + "blankIgnore": false, + "consumerSettings" : { + "zookeeper.connect" : "0.0.0.1:2182/kafka", + "bootstrap.servers" : "0.0.0.1:9092", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "latest" + } + }, + "name" : "kafkareader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、kafka->MySQL +```json +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "codec": "json", + "groupId" : "default", + "topic" : "tudou", + "consumerSettings" : { + "zookeeper.connect" : "kudu1:2181/kafka", + "bootstrap.servers" : "kudu1:9092", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "earliest" + }, + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "VARCHAR" + } + ] + }, + "name" : "kafkareader" + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "dtstack", + "password": "abc123", + "batchSize": 1, + "connection": [ + { + "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", + "table": [ + "kudu" + ] + } + ], + "session": [], + "preSql": [], + "postSql": [], + "writeMode": "insert", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "VARCHAR" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +
diff --git a/docs/realTime/reader/mongodboplogreader.md b/docs/realTime/reader/mongodboplogreader.md new file mode 100644 index 0000000000..ece557f274 --- /dev/null +++ b/docs/realTime/reader/mongodboplogreader.md @@ -0,0 +1,171 @@ +# MongoDB Oplog Reader + + +## 一、插件名称 +名称:**mongodboplogreader**
+ + +## 二、数据源版本 +**MongoDB 4.0及以上**
+ + +## 三、数据源配置 +[MongoDB 4.0副本集搭建](https://dtstack.yuque.com/rd-center/udi643/gufhya)
+ + +## 四、参数说明 + +- **hostPorts** + - 描述:Mongodb集群地址 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述: 用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述: 密码 + - 必选:是 + - 默认值:无 + + + +- **authenticationMechanism** + - 描述: 认证机制,可选:GSSAPI、PLAIN、MONGODB-X509、MONGODB-CR、SCRAM-SHA-1、SCRAM-SHA-256 + - 必选:否 + - 默认值:无 + + + +- **clusterMode** + - 描述: 集群模式,可选:REPLICA_SET、MASTER_SLAVE + - 必选:是 + - 默认值:无 + + + +- **monitorDatabases** + - 描述: 要监听的库 + - 必选:否 + - 默认值:无 + + + +- **monitorCollections** + - 描述:要监听的集合 + - 必选:否 + - 默认值:无 + + + +- **operateType** + - 描述:要监听的操作类型,可选:insert、update、delete + - 必选:否 + - 默认值:无 + + + +- **excludeDocId** + - 描述:是否排除_id字段 + - 必选:否 + - 默认值:false + + + +- **pavingData** + - 描述:是否将解析出的json数据拍平 + - 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: +```json +{ + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before_id":1, + "after_id":2 +} +``` + + -
pavingData为false时: +```json +{ + "message":{ + "type":"update", + "schema":"test", + "table":"tb1", + "ts":1231232, + "ingestion":123213, + "before_id":{ + "id":1 + }, + "after_id":{ + "id":2 + } + } +} +``` +其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 + + - 必选:否 + - 默认值:false + + + + +## 五、使用示例 + +#### 采集test库下的所有集合 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "mongodboplogreader", + "parameter": { + "hostPorts": "127.0.0.1:30001,127.0.0.1:30002,127.0.0.1:30003", + "username": "root", + "password": "123456", + "database": "admin", + "clusterMode": "REPLICA_SET", + "authenticationMechanism": "SCRAM-SHA-256", + "monitorDatabases": ["test"], + "monitorCollections":[], + "operateType":["insert","update","delete"], + "pavingData":true, + "excludeDocId": false + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 1048576 + }, + "errorLimit": { + "record": 100 + }, + "restore" : { + "isRestore" : true, + "isStream" : true + } + } + } +} +``` + + diff --git a/docs/realTime/reader/pgwalreader.md b/docs/realTime/reader/pgwalreader.md new file mode 100644 index 0000000000..a30ba5f610 --- /dev/null +++ b/docs/realTime/reader/pgwalreader.md @@ -0,0 +1,187 @@ +# PostgreSQL WAL Reader + + +## 一、插件名称 +名称:**pgwalreader**
+ + +## 二、数据源版本 +**PostgreSQL数据库版本至少为10.0及以上**
+ + +## 三、使用说明 +1、预写日志级别(wal_level)必须为logical
2、该插件基于PostgreSQL逻辑复制及逻辑解码功能实现的,因此PostgreSQL账户至少拥有replication权限,若允许创建slot,则至少拥有超级管理员权限
3、详细原理请参见[PostgreSQL官方文档](http://postgres.cn/docs/10/index.html)
+ + +## 四、参数说明
+ +- **jdbcUrl** + - 描述:PostgreSQL数据库的jdbc连接字符串,参考文档:[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **tableList** + - 描述:需要解析的数据表,格式为schema.table + - 必选:否 + - 默认值:无 + + + +- **cat** + - 描述:需要解析的数据更新类型,包括insert、update、delete三种 + - 注意:以英文逗号分割的格式填写。 + - 必选:是 + - 默认值:无 + + + +- **statusInterval** + - 描述:复制期间,数据库和使用者定期交换ping消息。如果数据库或客户端在配置的超时时间内未收到ping消息,则复制被视为已停止,并且将引发异常,并且数据库将释放资源。在PostgreSQL中,ping超时由属性wal_sender_timeout配置(默认= 60秒)。可以将pgjdc中的复制流配置为在需要时或按时间间隔发送反馈(ping)。建议比配置的wal_sender_timeout更频繁地向数据库发送反馈(ping)。在生产环境中,我使用等于wal_sender_timeout / 3的值。它避免了网络潜在的问题,并且可以在不因超时而断开连接的情况下传输更改 + - 必选:否 + - 默认值:2000 + + + +- **lsn** + - 描述:要读取PostgreSQL WAL日志序列号的开始位置 + - 必选:否 + - 默认值:0 + + + +- **slotName** + - 描述:复制槽名称,根据该值去寻找或创建复制槽 + - 注意:当allowCreateSlot为false时,该值不能为空 + - 必选:否 + - 默认值:无 + + + +- **allowCreateSlot** + - 描述:是否允许创建复制槽 + - 必选:否 + - 默认值:true + + + +- **temporary** + - 描述:复制槽是否为临时性的,true:是;false:否 + - 必选:否 + - 默认值:true + + + +- **pavingData** + - 描述:是否将解析出的json数据拍平 + - 示例:假设解析的表为tb1,schema为dbo,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: +```json +{ + "type":"update", + "schema":"dbo", + "table":"customers", + "lsn":207967352, + "ts": 1576487525488, + "ingestion":1475129582923642, + "before_id":1, + "after_id":2 +} +``` +pavingData为false时: +```json +{ + "type":"update", + "schema":"dbo", + "table":"customers", + "lsn":207967352, + "ts": 1576487525488, + "ingestion":1481628798880038, + "before":{ + "id":1 + }, + "after":{ + "id":2 + } +} +``` +其中:ts是数据库中数据的变更时间,ingestion是插件解析这条数据的纳秒时间,lsn是该数据变更的日志序列号 + + - 必选:否 + - 默认值:false + + + + +## 五、配置示例 +```json +{ + "job": { + "content": [{ + "reader" : { + "parameter" : { + "username" : "username", + "password" : "password", + "url" : "jdbc:postgresql://0.0.0.1:5432/postgres", + "databaseName" : "postgres", + "cat" : "update,insert,delete", + "tableList" : [ + "changepk.test_table" + ], + "statusInterval" : 10000, + "lsn" : 0, + "slotName" : "", + "allowCreateSlot" : true, + "temporary" : true, + "pavingData" : true + }, + "name" : "pgwalreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/realTime/writer/emqxwriter.md b/docs/realTime/writer/emqxwriter.md new file mode 100644 index 0000000000..b5774574bd --- /dev/null +++ b/docs/realTime/writer/emqxwriter.md @@ -0,0 +1,124 @@ +# Emqx Writer + + +## 一、插件名称 +名称:**emqxwriter**
+ +## 二、支持的数据源版本 +**Emqx 4.0及以上**
+ +## 三、参数说明
+ +- **broker** + - 描述:连接URL信息。 + - 必选:是 + - 默认值:无 + + + +- **topic** + - 描述:订阅主题 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:认证用户名 + - 必选:否 + - 默认值:无 + + + +- **password** + - 描述:认证密码 + - 必选:否 + - 默认值:无 + + + +- **isCleanSession** + - 描述:是否清除session + - false:MQTT服务器保存于客户端会话的的主题与确认位置 + - true:MQTT服务器不保存于客户端会话的的主题与确认位置 + - 必选:否 + - 默认值:true + + + +- **qos** + - 描述:服务质量 + - 0:AT_MOST_ONCE,至多一次; + - 1:AT_LEAST_ONCE,至少一次; + - 2:EXACTLY_ONCE,精准一次; + - 必选:否 + - 默认值:2 + + + + +## 四、配置示例 +```json +{ + "job": { + "content": [{ + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "writer" : { + "parameter" : { + "broker" : "tcp://0.0.0.1:1883", + "topic" : "test", + "username" : "username", + "password" : "password", + "isCleanSession": true, + "qos": 2 + }, + "name" : "emqxwriter" + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +
diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md new file mode 100644 index 0000000000..97355138cf --- /dev/null +++ b/docs/realTime/writer/kafkawriter.md @@ -0,0 +1,388 @@ +# Kafka Writer + + +## 一、插件名称 +kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: + +| kafka版本 | 插件名称 | +| --- | --- | +| kafka 0.9 | kafka09writer | +| kafka 0.10 | kafka10writer | +| kafka 0.11 | kafka11writer | +| kafka 1.0及以后 | kafkawriter | + + + + +## 二、参数说明
+ +- **timezone** + - 描述:时区 + - 必选:否 + - 默认值:无 + + + +- **topic** + - 描述:topic + - 必选:是 + - 默认值:无 + + + +- **encoding** + - 描述:编码 + - 注意:该参数只对kafka09reader插件有效 + - 必选:否 + - 默认值:UTF-8 + + + +- **brokerList** + - 描述:kafka broker地址列表 + - 注意:该参数只对kafka09writer插件有效 + - 必选:是 + - 默认值:无 + + + +- **producerSettings** + - 描述:kafka连接配置,支持所有`org.apache.kafka.clients.producer.ProducerConfig`中定义的配置 + - 必选:是 + - 默认值:无 + + + +- **tableFields** + - 描述:字段映射配置。从reader插件传递到writer插件的的数据只包含其value属性,配置该参数后可将其还原成键值对类型json字符串输出。 + - 注意: + - 若配置该属性,则该配置中的字段个数必须不少于reader插件中读取的字段个数,否则该配置失效; + - 映射关系按该配置中字段的先后顺序依次匹配; + - 必选:否 + - 默认值:无 + + + + +## 二、配置示例 + +#### 1、kafka09 +```json +{ + "job": { + "content": [{ + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer" : { + "parameter": { + "timezone": "UTC", + "topic": "kafka09", + "encoding": "UTF_8", + "brokerList": "0.0.0.1:9092", + "producerSettings": { + "zookeeper.connect" : "0.0.0.1:2182", + "bootstrap.servers" : "0.0.0.1:9092" + }, + "tableFields": ["id","user_id","name"] + }, + "name": "kafka09writer" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 2、kafka10 +```json +{ + "job": { + "content": [{ + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer" : { + "parameter": { + "timezone": "UTC", + "topic": "kafka10", + "producerSettings": { + "zookeeper.connect" : "0.0.0.1:2182", + "bootstrap.servers" : "0.0.0.1:9092" + }, + "tableFields": ["id","user_id","name"] + }, + "name": "kafka10writer" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 3、kafka11 +```json +{ + "job": { + "content": [{ + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer" : { + "parameter": { + "timezone": "UTC", + "topic": "kafka11", + "producerSettings": { + "zookeeper.connect" : "0.0.0.1:2182", + "bootstrap.servers" : "0.0.0.1:9092" + }, + "tableFields": ["id","user_id","name"] + }, + "name": "kafka11writer" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 4、kafka +```json +{ + "job": { + "content": [{ + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer" : { + "parameter": { + "timezone": "UTC", + "topic": "kafka", + "producerSettings": { + "zookeeper.connect" : "0.0.0.1:2182", + "bootstrap.servers" : "0.0.0.1:9092" + }, + "tableFields": ["id","user_id","name"] + }, + "name": "kafkawriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + +#### 5、MySQL->kafka +```json +{ + "job" : { + "content" : [ { + "reader": { + "name": "mysqlreader", + "parameter": { + "column": ["id","user_id","name"], + "username": "dtstack", + "password": "abc123", + "connection": [ + { + "jdbcUrl": [ + "jdbc:mysql://kudu3:3306/tudou" + ], + "table": [ + "kudu" + ] + } + ] + } + }, + "writer" : { + "parameter" : { + "tableFields" : ["id","user_id","name"], + "producerSettings" : { + "zookeeper.connect" : "kudu1:2182/kafka", + "bootstrap.servers" : "kudu1:9092" + }, + "topic" : "tudou" + }, + "name" : "kafkawriter" + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` + + diff --git a/docs/rediswriter.md b/docs/rediswriter.md deleted file mode 100644 index 366664ab63..0000000000 --- a/docs/rediswriter.md +++ /dev/null @@ -1,145 +0,0 @@ -# Redis写入插件(rediswriter) - -## 1. 配置样例 - -```json -{ - "job":{ - "content":[{ - "reader":{}, - "writer":{ - "parameter":{ - "hostPort":"localhost:6379", - "password": "密码", - "database":1, - "keyIndexes": [0,2], - "writeMode":"", - "keyFieldDelimiter": "\u0001", - "expireTime": 1000, - "timeout": 10000, - "dateFormat": "yyyy-MM-dd HH:mm:ss", - "type": "string", - "mode": "set", - "valueFieldDelimiter": "\u0001" - }, - "name":"rediswriter" - } - }] - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处只能填rediswriter,否则Flinkx将无法正常加载该插件包。 - * 必选:是 - * 默认值:无 - -* **hostPort** - - * 描述:Redis的IP地址和端口 - - * 必选:是 - - * 默认值:localhost:6379 - -* **password** - - * 描述:数据源指定用户名的密码 - - * 必选:是 - - * 默认值:无 - -* **database** - - * 描述:要写入Redis数据库 - - * 必选:否 - - * 默认值:0 - -* **keyIndexes** - - * 描述:keyIndexes 表示源端哪几列需要作为 key(第一列是从 0 开始)。如果是第一列和第二列需要组合作为 key,那么 keyIndexes 的值则为 [0,1]。 - - * 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。 - - * 必选:是 - - * 默认值:无 - -* **keyFieldDelimiter** - - * 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 - - * 必选:否 - - * 默认值:\u0001 - -* **expireTime** - - * 描述:Redis value 值缓存失效时间(如果需要永久有效则可以不填该配置项)。 - - * 注意:如果过期时间的秒数大于 60*60*24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 - - * 必选:否 - - * 默认值:0(0 表示永久有效) - -* **timeout** - - * 描述:写入 Redis 的超时时间。 - - * 单位:毫秒 - - * 必选:否 - - * 默认值:30000 - -* **dateFormat** - - * 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” - - * 必选:否 - - * 默认值:将日期以long类型写入 - -* **writeMode** - - * 描述:写入模式,由于 Redis 的数据结构为key-value模式,因此只要key相同,就会覆盖value值 - - * 必选:是 - - * 所有选项:insert - - * 默认值:insert - -* **valueFieldDelimiter** - - * 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 - - * 必选:否 - - * 默认值:\u0001 - -* **type和mode** - - * 描述:type 表示 value 的类型,mode 表示在选定的数据类型下的写入模式。 - - * 选项:string/list/set/zset/hash - - | type | 描述 | mode | 说明 | 注意 | - | ------ | ------- | ----- | ---------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------ | - | string | 字符串 | set | 存储这个数据,如果已经存在则覆盖 | | - | list | 字符串列表 | lpush | 在 list 最左边存储这个数据 | | - | list | 字符串列表 | rpush | 在 list 最右边存储这个数据 | | - | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | - | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | - | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | - - * 必选:是 - - * 默认值:无 diff --git a/docs/restore.md b/docs/restore.md index 682ad95036..792a19ad99 100644 --- a/docs/restore.md +++ b/docs/restore.md @@ -56,12 +56,12 @@ and id > ${offset} offset存在时 ```sql -第一个通道: +#第一个通道: select * from data_test where id mod 2=0 and id > ${offset_0}; -第二个通道 +#第二个通道: select * from data_test where id mod 2=1 and id > ${offset_1}; @@ -70,11 +70,11 @@ and id > ${offset_1}; offset不存在时 ```sql -第一个通道: +#第一个通道: select * from data_test where id mod 2=0; -第二个通道 +#第二个通道: select * from data_test where id mod 2=1; ``` diff --git a/docs/streamreader.md b/docs/streamreader.md deleted file mode 100644 index 68886faa7a..0000000000 --- a/docs/streamreader.md +++ /dev/null @@ -1,60 +0,0 @@ -# Stream读取插件(streamreader) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "type": "int", - "value":"xxx" - } - ], - "sliceRecordCount":10000 - }, - "name": "streamreader" - }, - "writer": {} - } - ], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,streamreader。 - - * 必选:是 - - * 默认值:无 - -* **sliceRecordCount** - - * 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止 - - * 必选:否 - - * 默认值:0 - -* **column** - - * 描述:需要生成的字段。 - - * 属性说明: - - * type:字段类型,程序根据指定的字段类型生成模拟数据,支持基本数据类型以及基本类型的数组,"int[]"表示生成一个长度随机的整形数组; - - * value:常量值,程序使用此字段的值直接返回; - - * 必选:是 - - * 默认值:无 diff --git a/docs/streamwriter.md b/docs/streamwriter.md deleted file mode 100644 index 0d786d2f6b..0000000000 --- a/docs/streamwriter.md +++ /dev/null @@ -1,41 +0,0 @@ -# Stream写入插件(streamwriter) - -## 1. 配置样例 - -``` -{ - "job": { - "content": [ - { - "reader": {}, - "writer": { - "parameter": { - "print":true - }, - "name": "streamwriter" - } - } - ], - "setting": {} - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,此处填写插件名称,streamwriter,此插件用来单独测试reader插件,对读到的数据不做任务处理; - - * 必选:是 - - * 默认值:无 - -* **print** - - * 描述:是否在控制台打印数据 - - * 必选:否 - - * 默认值:false - diff --git a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MapMakerHelper.java b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MapMakerHelper.java index 6d8fb46b79..27b707c0ea 100644 --- a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MapMakerHelper.java +++ b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MapMakerHelper.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MigrateMap.java b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MigrateMap.java index a68bf0637e..0b8fb14c42 100644 --- a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MigrateMap.java +++ b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/google/common/collect/MigrateMap.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java index 694c83d8e4..56f20ca9cb 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java index 597b1cb30f..6db0787bbe 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java index 1c9af53602..3708ac023e 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java index 8d3aea49f6..f5fdbe4ae2 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java @@ -20,6 +20,7 @@ import com.datastax.driver.core.*; import com.dtstack.flinkx.cassandra.CassandraUtil; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.google.common.base.Preconditions; @@ -180,7 +181,7 @@ else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { private String getQueryString(CassandraInputSplit inputSplit) { StringBuilder columns = new StringBuilder(); if (columnMeta == null) { - columns.append("*"); + columns.append(ConstantValue.STAR_SYMBOL); } else { for(MetaColumn column : columnMeta) { if(columns.length() > 0 ) { diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java index 412ad1e796..40304f54aa 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java @@ -48,6 +48,21 @@ public void openInternal(InputSplit inputSplit) throws IOException { LOG.info("inputSplit = {}", inputSplit); ClassUtil.forName(driverName, getClass().getClassLoader()); dbConn = ClickhouseUtil.getConnection(dbUrl, username, password); + initMetric(inputSplit); + String startLocation = incrementConfig.getStartLocation(); + if (incrementConfig.isPolling()) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); + } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { + getMaxValue(inputSplit); + } + + if(!canReadData(inputSplit)){ + LOG.warn("Not read data when the start location are equal to end location"); + hasNext = false; + return; + } + Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); statement.setFetchSize(fetchSize); statement.setQueryTimeout(queryTimeOut); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 47e20d2b28..bcac73e949 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -20,11 +20,7 @@ import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.classloader.ClassLoaderManager; -import com.dtstack.flinkx.config.ContentConfig; -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.SpeedConfig; -import com.dtstack.flinkx.config.RestartConfig; -import com.dtstack.flinkx.config.TestConfig; +import com.dtstack.flinkx.config.*; import com.dtstack.flinkx.constants.ConfigConstant; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.reader.BaseDataReader; @@ -44,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; @@ -116,14 +113,19 @@ public static void main(String[] args) throws Exception { env.setRestartStrategy(RestartStrategies.noRestart()); BaseDataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); - dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); + if(speedConfig.getReaderChannel() > 0){ + dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); + } if (speedConfig.isRebalance()) { dataStream = dataStream.rebalance(); } BaseDataWriter dataWriter = DataWriterFactory.getDataWriter(config); - dataWriter.writeData(dataStream).setParallelism(speedConfig.getWriterChannel()); + DataStreamSink dataStreamSink = dataWriter.writeData(dataStream); + if(speedConfig.getWriterChannel() > 0){ + dataStreamSink.setParallelism(speedConfig.getWriterChannel()); + } if(env instanceof MyLocalStreamEnvironment) { if(StringUtils.isNotEmpty(savepointPath)){ @@ -156,6 +158,8 @@ private static void configRestartStrategy(StreamExecutionEnvironment env, DataTr } else { env.setRestartStrategy(RestartStrategies.noRestart()); } + } else { + env.setRestartStrategy(RestartStrategies.noRestart()); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java index e38d99a4dd..df46175089 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ContentConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ContentConfig.java index 6d275590bd..da56f4372f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ContentConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ContentConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java index 9230bc7ea2..5e58d85e5a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DirtyConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DirtyConfig.java index b7b35b99d0..c06bf8bc74 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DirtyConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DirtyConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java index ed7ad555fe..a22a816835 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/JobConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/JobConfig.java index 8ff9d7478d..1708fae975 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/JobConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/JobConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ReaderConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ReaderConfig.java index 36ef2bccb4..3247f81454 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ReaderConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ReaderConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java index 520f5f78e0..35068840ab 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java index 9cff52c598..c85c1bb376 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/SettingConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/SettingConfig.java index d982c78bab..632ac8e6ed 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/SettingConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/SettingConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java index f680887022..2a15601654 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java index 2df69d966b..1c798b951a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/WriterConfig.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java index e60495d808..952cac39aa 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/Metrics.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java index a12446768c..16690d43a9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ClusterMode.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java index f09bd0a039..12009ede3a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/exception/WriteRecordException.java b/flinkx-core/src/main/java/com/dtstack/flinkx/exception/WriteRecordException.java index 27d08a8162..f7dae28b6b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/exception/WriteRecordException.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/exception/WriteRecordException.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 64bb73f2c3..1ce9844175 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java index ef1dc18337..8ba04e855b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/BaseLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/BaseLatch.java index 406792d799..adcdd44fb8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/BaseLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/BaseLatch.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/LocalLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/LocalLatch.java index dae07d3d5c..c34bb06a13 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/LocalLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/LocalLatch.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 8d1eec3005..7a86f3f2c2 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 9b57f5583b..0123d3f955 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java index 6e298859d9..0bdcca624a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java index f00482eb59..5d42ccdec0 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java index 59d3843d2e..548a1f0057 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java index ae05d25229..d328115334 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/restore/FormatState.java b/flinkx-core/src/main/java/com/dtstack/flinkx/restore/FormatState.java index 673893f71a..aa485ca084 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/restore/FormatState.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/restore/FormatState.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java index 350ba0bbff..bba853e451 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index ccaa76abe3..8ce417196d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java index abb4958910..ef25d54718 100755 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RowUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RowUtil.java index 8b1bab9611..e3f9e87d35 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RowUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RowUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index 3066b0c2c6..b276d856cd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/SysUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/SysUtil.java index 4988b304d9..d98895b88c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/SysUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/SysUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java index 16dc734257..dd93b58e49 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/TelnetUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java index d1202865e8..7465d84fdc 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ValueUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/BaseDataWriter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/BaseDataWriter.java index edfe5da320..55b7c547a5 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/BaseDataWriter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/BaseDataWriter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java index 73477191dc..145e5f35cc 100644 --- a/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java +++ b/flinkx-es/flinkx-es-core/src/main/java/com/dtstack/flinkx/es/EsConfigKeys.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java index c8c513def9..72d96de165 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java index 08dd266801..08613f7650 100644 --- a/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java +++ b/flinkx-es/flinkx-es-writer/src/main/java/com/dtstack/flinkx/es/writer/EsOutputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-examples/examples/binlog_to_hdfs.json b/flinkx-examples/examples/binlog_to_hdfs.json deleted file mode 100644 index d261eb165a..0000000000 --- a/flinkx-examples/examples/binlog_to_hdfs.json +++ /dev/null @@ -1,60 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "test", - "password" : "abc123", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.10.45:3306/test?charset=utf8", - "host" : "172.16.10.45", - "start" : { }, - "table" : [ "es_sink" ], - "pavingData" : false, - "username" : "dtstack" - }, - "name" : "binlogreader", - "type" : 1 - }, - "writer" : { - "parameter" : { - "path" : "/tmp/nanqitesthdfs", - "fileName" : "nanqi0903", - "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - "column" : [ { - "name" : "message", - "index" : 0, - "type" : "string" - } ], - "defaultFS" : "hdfs://ns1", - "writeMode" : "APPEND", - "fieldDelimiter" : ",", - "fileType" : "orc", - "charsetName" : "utf-8" - }, - "name" : "hdfswriter", - "type" : 6 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/binlog_to_hive.json b/flinkx-examples/examples/binlog_to_hive.json deleted file mode 100644 index 96d4261834..0000000000 --- a/flinkx-examples/examples/binlog_to_hive.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "DTstack1", - "password" : "123456", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.8.193:3306/DTstack1", - "host" : "172.16.8.193", - "start" : { }, - "table" : [ "employee_0", "employee_1", "employee_10", "employee_11", "employee_12", "employee_13", "employee_14", "employee_15", "employee_16", "employee_17", "employee_18", "employee_19", "employee_2", "employee_20", "employee_21", "employee_22", "employee_23", "employee_24", "employee_25", "employee_26", "employee_27", "employee_28", "employee_29", "employee_3", "employee_30", "employee_31", "employee_32", "employee_33", "employee_34", "employee_35", "employee_36", "employee_37", "employee_38", "employee_39", "employee_4", "employee_40", "employee_41", "employee_42", "employee_43", "employee_44", "employee_45", "employee_46", "employee_47", "employee_48", "employee_49", "employee_5", "employee_50", "employee_51", "employee_52", "employee_53", "employee_54", "employee_55", "employee_56", "employee_57", "employee_58", "employee_59", "employee_6", "employee_60", "employee_61", "employee_62", "employee_63", "employee_64", "employee_65", "employee_66", "employee_67", "employee_68", "employee_69", "employee_7", "employee_70", "employee_71", "employee_72", "employee_73", "employee_74", "employee_75", "employee_76", "employee_77", "employee_78", "employee_79", "employee_8", "employee_80", "employee_81", "employee_82", "employee_83", "employee_84", "employee_85", "employee_86", "employee_87", "employee_88", "employee_89", "employee_9", "employee_90", "employee_91", "employee_92", "employee_93", "employee_94", "employee_95", "employee_96", "employee_97", "employee_98", "employee_99" ], - "pavingData" : true, - "username" : "root" - }, - "name" : "binlogreader", - "type" : 1 - }, - "writer" : { - "parameter" : { - "fileName" : "pt", - "writeMode" : "append", - "fieldDelimiter" : "\u0001", - "partitionType" : "HOUR", - "path" : "", - "analyticalRules" : "stream_${schema}_${table}", - "password" : "", - "tablesColumn" : "{\"employee\":[{\"type\":\"VARCHAR\",\"key\":\"before_id\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_id\"},{\"type\":\"INT\",\"key\":\"before_deptno\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_deptno\"},{\"type\":\"INT\",\"key\":\"before_empno\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_empno\"},{\"type\":\"VARCHAR\",\"key\":\"before_ename\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_ename\"},{\"type\":\"VARCHAR\",\"key\":\"before_job\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_job\"},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"before_sal\",\"precision\":12},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"after_sal\",\"precision\":12},{\"type\":\"INT\",\"key\":\"before_hiredate\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_hiredate\"},{\"type\":\"VARCHAR\",\"key\":\"before_language19\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_language19\"},{\"type\":\"INT\",\"key\":\"before_health\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_health\"},{\"type\":\"VARCHAR\",\"key\":\"before_character19\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_character19\"},{\"type\":\"INT\",\"key\":\"before_id0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_id0\"},{\"type\":\"INT\",\"key\":\"before_deptno0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_deptno0\"},{\"type\":\"INT\",\"key\":\"before_empno0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_empno0\"},{\"type\":\"VARCHAR\",\"key\":\"before_ename0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_ename0\"},{\"type\":\"VARCHAR\",\"key\":\"before_job0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_job0\"},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"before_sal0\",\"precision\":12},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"after_sal0\",\"precision\":12},{\"type\":\"INT\",\"key\":\"before_hiredate0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_hiredate0\"},{\"type\":\"VARCHAR\",\"key\":\"before_language0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_language0\"},{\"type\":\"INT\",\"key\":\"before_health0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_health0\"},{\"type\":\"VARCHAR\",\"key\":\"before_character0\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_character0\"},{\"type\":\"INT\",\"key\":\"before_id1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_id1\"},{\"type\":\"INT\",\"key\":\"before_deptno1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_deptno1\"},{\"type\":\"INT\",\"key\":\"before_empno1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_empno1\"},{\"type\":\"VARCHAR\",\"key\":\"before_ename1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_ename1\"},{\"type\":\"VARCHAR\",\"key\":\"before_job1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_job1\"},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"before_sal1\",\"precision\":12},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"after_sal1\",\"precision\":12},{\"type\":\"INT\",\"key\":\"before_hiredate1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_hiredate1\"},{\"type\":\"VARCHAR\",\"key\":\"before_language1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_language1\"},{\"type\":\"INT\",\"key\":\"before_health1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_health1\"},{\"type\":\"VARCHAR\",\"key\":\"before_character1\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_character1\"},{\"type\":\"INT\",\"key\":\"before_id2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_id2\"},{\"type\":\"INT\",\"key\":\"before_deptno2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_deptno2\"},{\"type\":\"INT\",\"key\":\"before_empno2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_empno2\"},{\"type\":\"VARCHAR\",\"key\":\"before_ename2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_ename2\"},{\"type\":\"VARCHAR\",\"key\":\"before_job2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_job2\"},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"before_sal2\",\"precision\":12},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"after_sal2\",\"precision\":12},{\"type\":\"INT\",\"key\":\"before_hiredate2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_hiredate2\"},{\"type\":\"VARCHAR\",\"key\":\"before_language2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_language2\"},{\"type\":\"INT\",\"key\":\"before_health2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_health2\"},{\"type\":\"VARCHAR\",\"key\":\"before_character2\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_character2\"},{\"type\":\"INT\",\"key\":\"before_id3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_id3\"},{\"type\":\"INT\",\"key\":\"before_deptno3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_deptno3\"},{\"type\":\"INT\",\"key\":\"before_empno3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_empno3\"},{\"type\":\"VARCHAR\",\"key\":\"before_ename3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_ename3\"},{\"type\":\"VARCHAR\",\"key\":\"before_job3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_job3\"},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"before_sal3\",\"precision\":12},{\"scale\":31,\"comment\":\"\",\"type\":\"FLOAT\",\"key\":\"after_sal3\",\"precision\":12},{\"type\":\"INT\",\"key\":\"before_hiredate3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_hiredate3\"},{\"type\":\"VARCHAR\",\"key\":\"before_language3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_language3\"},{\"type\":\"INT\",\"key\":\"before_health3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"INT\",\"key\":\"after_health3\"},{\"type\":\"VARCHAR\",\"key\":\"before_character3\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR\",\"key\":\"after_character3\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", - "partition" : "pt", - "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "dfs.namenode.rpc-address.ns1.nn2" : "node2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "node1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - "jdbcUrl" : "jdbc:hive2://node1:10000/data_science", - "defaultFS" : "hdfs://ns1", - "distributeTable" : "{\"employee\":[\"employee_0\",\"employee_1\",\"employee_10\",\"employee_11\",\"employee_12\",\"employee_13\",\"employee_14\",\"employee_15\",\"employee_16\",\"employee_17\",\"employee_18\",\"employee_19\",\"employee_2\",\"employee_20\",\"employee_21\",\"employee_22\",\"employee_23\",\"employee_24\",\"employee_25\",\"employee_26\",\"employee_27\",\"employee_28\",\"employee_29\",\"employee_3\",\"employee_30\",\"employee_31\",\"employee_32\",\"employee_33\",\"employee_34\",\"employee_35\",\"employee_36\",\"employee_37\",\"employee_38\",\"employee_39\",\"employee_4\",\"employee_40\",\"employee_41\",\"employee_42\",\"employee_43\",\"employee_44\",\"employee_45\",\"employee_46\",\"employee_47\",\"employee_48\",\"employee_49\",\"employee_5\",\"employee_50\",\"employee_51\",\"employee_52\",\"employee_53\",\"employee_54\",\"employee_55\",\"employee_56\",\"employee_57\",\"employee_58\",\"employee_59\",\"employee_6\",\"employee_60\",\"employee_61\",\"employee_62\",\"employee_63\",\"employee_64\",\"employee_65\",\"employee_66\",\"employee_67\",\"employee_68\",\"employee_69\",\"employee_7\",\"employee_70\",\"employee_71\",\"employee_72\",\"employee_73\",\"employee_74\",\"employee_75\",\"employee_76\",\"employee_77\",\"employee_78\",\"employee_79\",\"employee_8\",\"employee_80\",\"employee_81\",\"employee_82\",\"employee_83\",\"employee_84\",\"employee_85\",\"employee_86\",\"employee_87\",\"employee_88\",\"employee_89\",\"employee_9\",\"employee_90\",\"employee_91\",\"employee_92\",\"employee_93\",\"employee_94\",\"employee_95\",\"employee_96\",\"employee_97\",\"employee_98\",\"employee_99\"]}", - "fileType" : "orc", - "charsetName" : "utf-8", - "username" : "", - "bufferSize" : 1048576 - }, - "name" : "hivewriter", - "type" : 7 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : 2097152, - "channel" : 3 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/binlog_to_kafka09.json b/flinkx-examples/examples/binlog_to_kafka09.json deleted file mode 100644 index 0c56f2d8a9..0000000000 --- a/flinkx-examples/examples/binlog_to_kafka09.json +++ /dev/null @@ -1,41 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "password" : "abc123", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.10.45:3306/test", - "host": "172.16.10.45", - "start" : { }, - "table" : [ "date_test", "es_sink", "inserttype", "myresult", "pv1", "sales", "stmresult", "subtable3", "subtable4", "test_text", "testmuyun", "time_test", "totalsales", "user_list", "user_sink", "user_source", "xcpv1", "xctime", "ysq_order_fact20_carbon_3" ], - "username" : "dtstack" - }, - "name": "binlogreader" - }, - "writer": { - "parameter": { - "timezone" : "",//选填 - "encoding" : "utf-8",//选填 - "producerSettings" : {//必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka" - }, - "topic" : "mufeng_est",//必填 - "brokerList" : "172.16.8.107:9092"//必填 - }, - "name": "kafka09writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/binlog_to_kafka10.json b/flinkx-examples/examples/binlog_to_kafka10.json deleted file mode 100644 index 6df130fc0c..0000000000 --- a/flinkx-examples/examples/binlog_to_kafka10.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "password" : "abc123", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.10.45:3306/test", - "host": "172.16.10.45", - "start" : { }, - "table" : [ "date_test", "es_sink", "inserttype", "myresult", "pv1", "sales", "stmresult", "subtable3", "subtable4", "test_text", "testmuyun", "time_test", "totalsales", "user_list", "user_sink", "user_source", "xcpv1", "xctime", "ysq_order_fact20_carbon_3" ], - "username" : "dtstack" - }, - "name": "binlogreader" - }, - "writer": { - "parameter": { - "timezone" : "",//选填 - "producerSettings" : {//必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092" //必填 - }, - "topic" : "mufeng_est"//必填 - }, - "name": "kafka10writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/binlog_to_kafka11.json b/flinkx-examples/examples/binlog_to_kafka11.json deleted file mode 100644 index da33403270..0000000000 --- a/flinkx-examples/examples/binlog_to_kafka11.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "test", - "password" : "abc123", - "cat" : "insert,update,delete", - "jdbcUrl" : "jdbc:mysql://172.16.10.45:3306/test", - "host" : "172.16.10.45", - "start" : {"timestamp":1567760579123}, - "table" : [ "myresult" ], - "pavingData" : true, - "username" : "dtstack" - }, - "name" : "binlogreader", - "type" : 1 - }, - "writer" : { - "parameter" : { - "producerSettings" : { - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092" - }, - "topic" : "mufeng_est" - }, - "name" : "kafka11writer", - "type" : 14 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/clickhouse_to_clickhouse.json b/flinkx-examples/examples/clickhouse_to_clickhouse.json deleted file mode 100644 index d8a8617553..0000000000 --- a/flinkx-examples/examples/clickhouse_to_clickhouse.json +++ /dev/null @@ -1,91 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "username": "username", - "password": "password", - "connection": [{ - "jdbcUrl": [ - "jdbc:clickhouse://ip:8123/test" - ], - "table": [ - "test1" - ] - }], - "column":[{ - "name": "id", - "type": "INT", - "key": "id" - }, - { - "name": "user_id", - "type": "INT", - "key": "user_id" - }, - { - "name": "name", - "type": "VARCHAR", - "key": "name" - }], - "splitPk": "id", - "where": " id > 121 " - }, - "name": "clickhousereader" - }, - "writer": { - "name": "clickhousewriter", - "parameter": { - "username": "username", - "password": "password", - "connection": [ - { - "jdbcUrl" : "jdbc:clickhouse://ip:8123/test", - "table": [ - "test2" - ] - } - ], - "session": [], - "preSql": [], - "postSql": [], - "writeMode": "insert", - "column": [ - { - "name": "id", - "type": "INT", - "key": "id" - }, - { - "name": "user_id", - "type": "INT", - "key": "user_id" - }, - { - "name": "name", - "type": "VARCHAR", - "key": "name" - } - ], - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/dm_to_dm.json b/flinkx-examples/examples/dm_to_dm.json deleted file mode 100644 index bec9a0a2cf..0000000000 --- a/flinkx-examples/examples/dm_to_dm.json +++ /dev/null @@ -1,96 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "name": "dmreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "NUMBER", - "key": "id" - }, - { - "name": "user_id", - "type": "NUMBER", - "key": "user_id" - }, - { - "name": "name", - "type": "VARCHAR2", - "key": "name" - } - ], - "increColumn": "", - "startLocation": "", - "username": "SYSDBA", - "password": "SYSDBA", - "connection": [ - { - "jdbcUrl": [ - "jdbc:dm://0.0.0.0:5236" - ], - "table": [ - "kudu_data" - ] - } - ], - "where": "" - } - }, - "writer": { - "name": "dmwriter", - "parameter": { - "username": "SYSDBA", - "password": "SYSDBA", - "connection": [ - { - "jdbcUrl": "jdbc:dm://0.0.0.0:5236", - "table": [ - "kudu" - ] - } - ], - "session": [], - "preSql": [], - "postSql": [], - "writeMode": "update", - "column": [ - { - "name": "id", - "type": "NUMBER", - "key": "id" - }, - { - "name": "user_id", - "type": "NUMBER", - "key": "user_id" - }, - { - "name": "name", - "type": "VARCHAR2", - "key": "name" - } - ] - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/es_to_mysql.json b/flinkx-examples/examples/es_to_mysql.json deleted file mode 100644 index 1c1bb883fb..0000000000 --- a/flinkx-examples/examples/es_to_mysql.json +++ /dev/null @@ -1,59 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 2, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "esreader", - "parameter": { - "address": "rdos1:9200,rdos2:9200", - "query": { - "match": { - "col2": "hallo" - } - }, - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb333" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/ftp_to_mysql.json b/flinkx-examples/examples/ftp_to_mysql.json deleted file mode 100644 index e8eaa041d9..0000000000 --- a/flinkx-examples/examples/ftp_to_mysql.json +++ /dev/null @@ -1,59 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 50 - } - }, - "content": [ - { - "reader": { - "name": "ftpreader", - "parameter": { - "protocol": "ftp", - "host": "node01" , - "port": 21, - "username": "test1", - "password": "oh1986mygod", - "column": [ - { - "index": 0 - }, - { - "index": 1 - } - ], - "path": "/", - "encoding": "UTF-8", - "fieldDelimiter": "\\t" - } - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ] - } -} diff --git a/flinkx-examples/examples/hbase_to_mysql.json b/flinkx-examples/examples/hbase_to_mysql.json deleted file mode 100644 index d904d95565..0000000000 --- a/flinkx-examples/examples/hbase_to_mysql.json +++ /dev/null @@ -1,67 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 2, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 2 - } - }, - "content": [ - { - "reader": { - "name": "hbasereader", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "node01,node02,node03", - "zookeeper.znode.parent": "/hbase" - }, - "table": "sb5", - "encodig": "utf-8", - "column": [ - { - "name": "rowkey", - "type": "string" - }, - { - "name": "cf1:id", - "type": "string" - } - ], - "range": { - "startRowkey": "", - "endRowkey": "", - "isBinaryRowkey": true - } - } - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ] - } -} diff --git a/flinkx-examples/examples/hdfs_to_carbon.json b/flinkx-examples/examples/hdfs_to_carbon.json deleted file mode 100644 index 4283cdbf69..0000000000 --- a/flinkx-examples/examples/hdfs_to_carbon.json +++ /dev/null @@ -1,67 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "name": "hdfsreader", - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/mufeng_test.db/tmp_toutian_test", - "defaultFS": "hdfs://ns1", - "column": [ - { - "name": "name", - "index": 1, - "type": "string" - }, - { - "name": "id", - "index": 0, - "type": "int" - } - ], - "fileType": "orc", - "fieldDelimiter": "\u0001", - "encoding": "utf-8", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node1:9000", - "dfs.nameservices": "ns1" - } - } - }, - "writer": { - "name": "carbondatawriter", - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/carbon.store/qatest111/tmp_maoxiu_2", - "database": "qatest111", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node2:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node1:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - "name", - "age" - ], - "defaultFS": "hdfs://ns1", - "writeMode": "overwrite", - "table": "tmp_maoxiu_2" - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 1048576 - }, - "errorLimit": { - "record": 100 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/hive_orc_to_mysql.json b/flinkx-examples/examples/hive_orc_to_mysql.json deleted file mode 100644 index b50d28fc70..0000000000 --- a/flinkx-examples/examples/hive_orc_to_mysql.json +++ /dev/null @@ -1,66 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/wujing_test.db/kepa_250", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - } - ], - "defaultFS": "hdfs://ns1", - "fieldDelimiter": "", - "encoding": "utf-8", - "fileType": "orc" - }, - "name": "hdfsreader" - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/kafka09_to_kafka09.json b/flinkx-examples/examples/kafka09_to_kafka09.json deleted file mode 100644 index 2091fe0376..0000000000 --- a/flinkx-examples/examples/kafka09_to_kafka09.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "topic" : "yxabc",//必填 - "codec" : "plain",//选填 - "encoding" : "utf-8",//选填 - "consumerSettings" : {//必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka",//必填 - "group.id" : "default", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "smallest" - } - }, - "name": "kafka09reader" - }, - "writer": { - "parameter": { - "timezone" : "",//选填 - "encoding" : "utf-8",//选填 - "producerSettings" : {//必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka" - }, - "topic" : "mufeng_est",//必填 - "brokerList" : "172.16.8.107:9092"//必填 - }, - "name": "kafka09writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/kafka10_to_kafka10.json b/flinkx-examples/examples/kafka10_to_kafka10.json deleted file mode 100644 index 8714dc2c9e..0000000000 --- a/flinkx-examples/examples/kafka10_to_kafka10.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "groupId" : "default",//必填 - "topic" : "mq03",//必填 - "blankIgnore" : false, //选填 - "codec" : "plain", //选填 - "consumerSettings" : {//必填 - "bootstrap.servers" : "172.16.8.107:9092", //必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name": "kafka10reader" - }, - "writer": { - "parameter": { - "timezone" : "",//选填 - "producerSettings" : {//必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092" //必填 - }, - "topic" : "mufeng_est"//必填 - }, - "name": "kafka10writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/kafka11_to_hive.json b/flinkx-examples/examples/kafka11_to_hive.json deleted file mode 100644 index 998a55cd9d..0000000000 --- a/flinkx-examples/examples/kafka11_to_hive.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "groupId" : "default", - "topic" : "kafka10_hdfs", - "consumerSettings" : { - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name" : "kafka11reader", - "type" : 14 - }, - "writer" : { - "parameter" : { - "fileName" : "pt", - "writeMode" : "append", - "fieldDelimiter" : "\u0001", - "partitionType" : "HOUR", - "path" : "hdfs://ns1/user/hive/warehouse/data_science.db/kafka_hdfs", - "analyticalRules" : "stream_${schema}_${table}", - "password" : "", - "tablesColumn" : "{\"kafka_hdfs\":[{\"type\":\"string\",\"key\":\"message\",\"comment\":\"\"}]}", - "partition" : "pt", - "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "fs.defaultFS" : "hdfs://ns1", - "dfs.namenode.rpc-address.ns1.nn2" : "node2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "node1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" - }, - "jdbcUrl" : "jdbc:hive2://node1:10000/data_science", - "defaultFS" : "hdfs://ns1", - "table" : "kafka_hdfs", - "fileType" : "orc", - "charsetName" : "utf-8", - "username" : "", - "bufferSize" : 1048576 - }, - "name" : "hivewriter", - "type" : 7 - } - } ], - "setting" : { - "restore" : { - "isRestore" : true, - "isStream" : true - }, - "errorLimit" : { - }, - "speed" : { - "bytes" : 2097152, - "channel" : 3 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/kafka11_to_kafka11.json b/flinkx-examples/examples/kafka11_to_kafka11.json deleted file mode 100644 index 070101c583..0000000000 --- a/flinkx-examples/examples/kafka11_to_kafka11.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "groupId" : "default",//必填 - "topic" : "mq03",//必填 - "blankIgnore" : false, //选填 - "codec" : "plain", //选填 - "consumerSettings" : {//必填 - "bootstrap.servers" : "172.16.8.107:9092", //必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name": "kafka11reader" - }, - "writer": { - "parameter": { - "timezone" : "",//选填 - "producerSettings" : { //必填 - "zookeeper.connect" : "172.16.8.107:2181/kafka", - "bootstrap.servers" : "172.16.8.107:9092" //必填 - }, - "topic" : "mufeng_est"//必填 - }, - "name": "kafka11writer" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/kafka_stream.json b/flinkx-examples/examples/kafka_stream.json deleted file mode 100644 index 99562dc858..0000000000 --- a/flinkx-examples/examples/kafka_stream.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "groupId" : "default", - "topic" : "tudou", - "consumerSettings" : { - "zookeeper.connect" : "ip:2181/kafka100", - "bootstrap.servers" : "ip:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name" : "kafkareader" - }, - "writer" : { - "parameter" : { - "print" : true - }, - "name" : "streamwriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : 0, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mongodb_to_mysql.json b/flinkx-examples/examples/mongodb_to_mysql.json deleted file mode 100644 index 396f47a620..0000000000 --- a/flinkx-examples/examples/mongodb_to_mysql.json +++ /dev/null @@ -1,58 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } - }, - "content": [ - { - "reader": { - - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_kafka.json b/flinkx-examples/examples/mysql_kafka.json deleted file mode 100644 index 7bfce6c2a9..0000000000 --- a/flinkx-examples/examples/mysql_kafka.json +++ /dev/null @@ -1,77 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader": { - "name": "mysqlreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "BIGINT", - "key": "id" - }, - { - "name": "user_id", - "type": "bigint", - "key": "user_id" - }, - { - "name": "name", - "type": "varchar", - "key": "name" - }, - { - "name": "content", - "type": "varchar", - "key": "name" - } - ], - "increColumn": "", - "startLocation": "", - "password": "password", - "username": "username", - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://ip:3306/tudou" - ], - "table": [ - "binlog" - ] - } - ], - "splitKey":"id", - "customSql": "" - } - }, - "writer" : { - "parameter" : { - "tableFields" : [ - "id", - "user_id", - "name", - "content" - ], - "producerSettings" : { - "zookeeper.connect" : "ip:2181/kafka100", - "bootstrap.servers" : "ip:9092" - }, - "topic" : "tudou" - }, - "name" : "kafkawriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : false - }, - "errorLimit" : { - }, - "speed" : { - "bytes" : 0, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_es.json b/flinkx-examples/examples/mysql_to_es.json deleted file mode 100644 index e3c32d1eb5..0000000000 --- a/flinkx-examples/examples/mysql_to_es.json +++ /dev/null @@ -1,65 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 20 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ] - } - }, - "writer": { - "name": "eswriter", - "parameter": { - "address": "rdos1:9200,rdos2:9200", - "index": "yoshi", - "type": "nani", - "bulkAction": 3, - "idColumn": [ - { - "index": 0, - "type": "int" - } - ], - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_ftp.json b/flinkx-examples/examples/mysql_to_ftp.json deleted file mode 100644 index 5f3452dd2e..0000000000 --- a/flinkx-examples/examples/mysql_to_ftp.json +++ /dev/null @@ -1,57 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ], - "scliceRecordCount": ["1000"] - }, - "name": "streamreader" - }, - "writer": { - "name": "ftpwriter", - "parameter": { - "protocol": "sftp", - "host": "cdh_impala2", - "port": 22, - "privateKeyPath": "/root/.ssh/id_rsa", - "username": "root", - "password": "abc123", - "writeMode": "append", - "path": "/root/sftp", - "fieldDelimiter": ",", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_hbase.json b/flinkx-examples/examples/mysql_to_hbase.json deleted file mode 100644 index 253af486a9..0000000000 --- a/flinkx-examples/examples/mysql_to_hbase.json +++ /dev/null @@ -1,63 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ] - } - }, - "writer": { - "name": "hbasewriter", - "parameter": { - "hbaseConfig": { - "hbase.zookeeper.property.clientPort": "2181", - "hbase.rootdir": "hdfs://ns1/hbase", - "hbase.cluster.distributed": "true", - "hbase.zookeeper.quorum": "node01,node02,node03", - "zookeeper.znode.parent": "/hbase" - }, - "table": "tb1", - "rowkeyColumn": "col1#col2", - "column": [ - { - "name": "cf1:id", - "type": "string" - }, - { - "name": "cf1:vv", - "type": "string" - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_hdfs_orc.json b/flinkx-examples/examples/mysql_to_hdfs_orc.json deleted file mode 100644 index 6abb990b38..0000000000 --- a/flinkx-examples/examples/mysql_to_hdfs_orc.json +++ /dev/null @@ -1,90 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.String", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb1" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "path": "hdfs://ns1/user/hive/warehouse/wujing_test.db/kepa_250", - "fileName": "", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - } - ], - "defaultFS": "hdfs://ns1", - "connection": [ - { - "jdbcUrl": "jdbc:hive2://node03:10000/wujing_test", - "table": [ - "kepa_250" - ] - } - ], - "writeMode": "overwrite", - "fieldDelimiter": "", - "encoding": "utf-8", - "fullColumnName": [ - "col1", - "col2" - ], - "fileType": "orc", - "fullColumnType": [ - "string", - "string" - ] - }, - "name": "hdfswriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_mongodb.json b/flinkx-examples/examples/mysql_to_mongodb.json deleted file mode 100644 index 30d9ee1044..0000000000 --- a/flinkx-examples/examples/mysql_to_mongodb.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_mysql.json b/flinkx-examples/examples/mysql_to_mysql.json deleted file mode 100644 index f9c2bb2a0a..0000000000 --- a/flinkx-examples/examples/mysql_to_mysql.json +++ /dev/null @@ -1,78 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "id", - "v1" - ], - "where": "id > 1", - "connection": [ - { - "table": [ - "sb9" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_odps.json b/flinkx-examples/examples/mysql_to_odps.json deleted file mode 100644 index bd36870980..0000000000 --- a/flinkx-examples/examples/mysql_to_odps.json +++ /dev/null @@ -1,62 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "username": "dtstack", - "password": "abc123", - "column": [ - "col1", - "col2" - ], - // "splitPk": "col1", - "connection": [ - { - "table": [ - "tb2" - ], - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true" - ] - } - ] - } - }, - "writer": { - "name": "odpswriter", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tb252", - "partition": "pt='xxooxx'", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_oracle.json b/flinkx-examples/examples/mysql_to_oracle.json deleted file mode 100644 index b82fb216bd..0000000000 --- a/flinkx-examples/examples/mysql_to_oracle.json +++ /dev/null @@ -1,63 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "parameter": { - "password": "wujing", - "column": [ - "ID", - "C1" - ], - "connection": [ - { - "jdbcUrl": "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack", - "table": [ - "SB2" - ] - } - ], - "writeMode": "insert", - "username": "wujing" - }, - "name": "oraclewriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 3 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_phoenix.json b/flinkx-examples/examples/mysql_to_phoenix.json deleted file mode 100644 index 71f3d5077d..0000000000 --- a/flinkx-examples/examples/mysql_to_phoenix.json +++ /dev/null @@ -1,72 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://127.0.0.1:3306/cake?useCursorFetch=true" - ], - "table": [ - "userinfo" - ] - } - ], - "username": "root", - "password": "745631289", - "column": [ - { - "name": "name", - "type": "varchar" - }, - { - "name": "age", - "type": "int" - } - ] - }, - "name": "mysqlreader" - }, - "writer": { - "name": "phoenixwriter", - "parameter": { - "username": "", - "password": "", - "connection": [ - { - "jdbcUrl": "jdbc:phoenix:myhbase:2181/hbase", - "table": [ - "USERINFO" - ] - } - ], - "column": [ - { - "name": "NAME", - "type": "varchar" - }, - { - "name": "AGE", - "type": "INTEGER" - } - ], - "writeMode": "insert", - "batchSize": 1024 - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/mysql_to_postgresql.json b/flinkx-examples/examples/mysql_to_postgresql.json deleted file mode 100644 index a8dd08d68f..0000000000 --- a/flinkx-examples/examples/mysql_to_postgresql.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_redis.json b/flinkx-examples/examples/mysql_to_redis.json deleted file mode 100644 index 30d9ee1044..0000000000 --- a/flinkx-examples/examples/mysql_to_redis.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_sftp.json b/flinkx-examples/examples/mysql_to_sftp.json deleted file mode 100644 index a1073ab358..0000000000 --- a/flinkx-examples/examples/mysql_to_sftp.json +++ /dev/null @@ -1,66 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.String", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb1" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "ftpwriter", - "parameter": { - "protocol": "sftp", - "host": "node03", - "port": 22, - "username": "mysftp", - "password": "oh1986mygod", - "writeMode": "overwrite", - "path": "/upload/xxx", - "fieldDelimiter": ",", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/mysql_to_sqlserver.json b/flinkx-examples/examples/mysql_to_sqlserver.json deleted file mode 100644 index 8ad7320536..0000000000 --- a/flinkx-examples/examples/mysql_to_sqlserver.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 4 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "sqlserverwriter", - "parameter": { - "batchSize": 2048, - "username": "sa", - "password": "Dtstack201610!", - "column": [ - "id", - "v" - ], - "writeMode": "replace", - "connection": [ - { - "jdbcUrl": "jdbc:jtds:sqlserver://172.16.10.46:1433;DatabaseName=dq", - "table": [ - "tb1" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/odps_to_mysql.json b/flinkx-examples/examples/odps_to_mysql.json deleted file mode 100644 index 113db8184d..0000000000 --- a/flinkx-examples/examples/odps_to_mysql.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 10000000 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - }, - "reader": { - "name": "odpsreader", - "parameter": { - "odpsConfig": { - "accessId": "${odps.accessId}", - "accessKey": "${odps.accessKey}", - "project": "${odps.project}" - }, - "table": "tb252", - "partition": "pt='xxooxx'", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/oracle_to_hdfs_text.json b/flinkx-examples/examples/oracle_to_hdfs_text.json deleted file mode 100644 index 288f12c67e..0000000000 --- a/flinkx-examples/examples/oracle_to_hdfs_text.json +++ /dev/null @@ -1,71 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1 - }, - "errorLimit": { - "record": 0, - "percentage": 0.02 - } - }, - "content": [ - { - "reader": { - "parameter": { - "password": "abc123", - "columnTypes": [ - "java.lang.Integer", - "java.lang.String" - ], - "column": [ - "col1", - "col2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:mysql://172.16.8.104:3306/test?charset=utf8" - ], - "table": [ - "tb2" - ] - } - ], - "splitPk": "col1", - "username": "dtstack" - }, - "name": "mysqlreader" - }, - "writer": { - "name": "hdfswriter", - "parameter": { - "hadoopConfig": { - "dfs.nameservices":"ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider" - }, - "defaultFS": "hdfs://ns1", - "fileType": "text", - "fileName": "hallo", - "column": [ - { - "name": "col1", - "type": "STRING" - }, - { - "name": "col2", - "type": "STRING" - } - ], - "path": "/hyf", - "writeMode": "append", - "fieldDelimiter": "\\001" - } - } - } - ] - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/oracle_to_oracle.json b/flinkx-examples/examples/oracle_to_oracle.json deleted file mode 100644 index 7397f14aef..0000000000 --- a/flinkx-examples/examples/oracle_to_oracle.json +++ /dev/null @@ -1,61 +0,0 @@ -{"job": { - "content": [ - { - "reader": { - "parameter": { - "password": "wujing", - "where": "3 > 1", - "column": [ - "ID1", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": [ - "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack" - ], - "table": [ - "SB1" - ] - } - ], - "splitPk": "ID1", - "username": "wujing" - }, - "name": "oraclereader" - }, - "writer": { - "parameter": { - "password": "wujing", - "column": [ - "ID", - "C1", - "C2" - ], - "connection": [ - { - "jdbcUrl": "jdbc:oracle:thin:@//172.16.8.121:1521/dtstack", - "table": [ - "SB2" - ] - } - ], - "writeMode": "replace", - "username": "wujing" - }, - "name": "oraclewriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 2 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/pgwal_to_stream.json b/flinkx-examples/examples/pgwal_to_stream.json deleted file mode 100644 index 86924263e5..0000000000 --- a/flinkx-examples/examples/pgwal_to_stream.json +++ /dev/null @@ -1,42 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "username" : "postgres", - "password" : "abc123", - "url" : "jdbc:postgresql://172.16.8.122:5432/postgres", - "databaseName" : "postgres", - "cat" : "update,insert,delete", - "tableList" : [ - "changepk.test_table" - ], - "statusInterval" : 10000, - "lsn" : 0, - "slotName" : "", - "allowCreateSlot" : true, - "temporary" : true, - "pavingData" : true - }, - "name" : "pgwalreader" - }, - "writer" : { - "parameter" : { - "print" : true - }, - "name" : "streamwriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : -1048576, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/phoenix_to_mysql.json b/flinkx-examples/examples/phoenix_to_mysql.json deleted file mode 100644 index 59534d987a..0000000000 --- a/flinkx-examples/examples/phoenix_to_mysql.json +++ /dev/null @@ -1,72 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "username": "", - "password": "", - "connection": [ - { - "jdbcUrl": [ - "jdbc:phoenix:myhbase:2181/hbase" - ], - "table": [ - "USERINFO" - ] - } - ], - "column": [ - { - "name": "NAME", - "type": "varchar" - }, - { - "name": "AGE", - "type": "INTEGER" - } - ] - }, - "name": "phoenixreader" - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "connection": [ - { - "jdbcUrl": "jdbc:mysql://127.0.0.1:3306/cake?useCursorFetch=true", - "table": [ - "userinfo" - ] - } - ], - "username": "root", - "password": "745631289", - "column": [ - { - "name": "name", - "type": "varchar" - }, - { - "name": "age", - "type": "int" - } - ], - "writeMode": "insert", - "batchSize": 1024 - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/polarDB_to_polarDb.json b/flinkx-examples/examples/polarDB_to_polarDb.json deleted file mode 100644 index c4144e76d9..0000000000 --- a/flinkx-examples/examples/polarDB_to_polarDb.json +++ /dev/null @@ -1,77 +0,0 @@ -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "customSql" : "", - "startLocation" : "", - "increColumn" : "", - "column" : [ { - "name" : "id", - "type" : "BIGINT", - "key" : "id" - }, { - "name" : "user_id", - "type" : "BIGINT", - "key" : "user_id" - }, { - "name" : "name", - "type" : "VARCHAR", - "key" : "name" - } ], - "username" : "username", - "password" : "password", - "connection" : [ { - "jdbcUrl" : [ "jdbc:mysql://aaaa:3306/bbbb" ], - "table" : [ "kudu_data" ] - } ], - "splitPk" : "id" - }, - "name" : "polardbreader" - }, - "writer" : { - "parameter" : { - "postSql" : [ ], - "username" : "username", - "password" : "password", - "session" : [ ], - "column" : [ { - "name" : "id", - "type" : "BIGINT", - "key" : "id" - }, { - "name" : "user_id", - "type" : "BIGINT", - "key" : "user_id" - }, { - "name" : "name", - "type" : "VARCHAR", - "key" : "name" - } ], - "connection" : [ { - "jdbcUrl" : "jdbc:mysql://aaaa:3306/bbbb", - "table" : [ "kudu" ] - } ], - "writeMode" : "insert", - "preSql" : [ ] - }, - "name" : "polardbwriter" - } - } ], - "setting" : { - "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : true, - "restoreColumnName" : "id", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 - }, - "speed" : { - "bytes" : 0, - "channel" : 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/examples/postgresql_to_mysql.json b/flinkx-examples/examples/postgresql_to_mysql.json deleted file mode 100644 index 955d71eda5..0000000000 --- a/flinkx-examples/examples/postgresql_to_mysql.json +++ /dev/null @@ -1,65 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - } - }, - "content": [ - { - "reader": { - "name": "postgresqlreader", - "parameter": { - "username": "postgres", - "password": "admin123", - "column": [ - "id", - "name" - ], - "where": "", - "connection": [ - { - "table": [ - "jb_tb1" - ], - "jdbcUrl": [ - "jdbc:postgresql://localhost:5432/jb_test" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "root", - "password": "admin123", - "column": [ - "id", - "name" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://localhost:3306/jiangbo?useCursorFetch=true", - "table": [ - "postgresql_test" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/sftp_to_mysql.json b/flinkx-examples/examples/sftp_to_mysql.json deleted file mode 100644 index d2632c6ec7..0000000000 --- a/flinkx-examples/examples/sftp_to_mysql.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 1, - "bytes": 10000 - }, - "errorLimit": { - "record": 0, - "percentage": 50 - } - }, - "content": [ - { - "reader": { - "name": "ftpreader", - "parameter": { - "protocol": "sftp", - "host": "node01" , - "port": 22, - "username": "mysftp", - "password": "oh1986mygod", - "column": [ - { - "index": 0 - }, - { - "index": 1 - }, - { - "value": "youcan", - "type": "string" - } - ], - "path": "/upload", - "encoding": "UTF-8", - "fieldDelimiter": "\\t" - } - }, - "writer": { - "parameter": { - "password": "abc123", - "column": [ - "col1", - "col2", - "col3" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?charset=utf8", - "table": [ - "sb5" - ] - } - ], - "writeMode": "insert", - "username": "dtstack" - }, - "name": "mysqlwriter" - } - } - ] - } -} diff --git a/flinkx-examples/examples/sqlserver_to_mysql.json b/flinkx-examples/examples/sqlserver_to_mysql.json deleted file mode 100644 index 760e7b0859..0000000000 --- a/flinkx-examples/examples/sqlserver_to_mysql.json +++ /dev/null @@ -1,78 +0,0 @@ -{ - "job": { - "setting": { - "speed": { - "channel": 3, - "bytes": 0 - }, - "errorLimit": { - "record": 10000, - "percentage": 100 - }, - "dirty": { - "path": "/tmp", - "hadoopConfig": { - "fs.default.name": "hdfs://ns1", - "dfs.nameservices": "ns1", - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn1": "node02:9000", - "dfs.namenode.rpc-address.ns1.nn2": "node03:9000", - "dfs.ha.automatic-failover.enabled": "true", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "fs.hdfs.impl.disable.cache": "true" - } - } - }, - "content": [ - { - "reader": { - "name": "sqlserverreader", - "parameter": { - "username": "sa", - "password": "Dtstack201610!", - "column": [ - "id", - "v" - ], - "where": "id > 1", - "connection": [ - { - "table": [ - "tb1" - ], - "jdbcUrl": [ - "jdbc:jtds:sqlserver://172.16.10.46:1433;DatabaseName=dq" - ] - } - ], - "splitPk": "id" - } - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "writeMode": "insert", - "username": "dtstack", - "password": "abc123", - "column": [ - "c1", - "c2" - ], - "batchSize": 1, - "session": [ - "set session sql_mode='ANSI'" - ], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", - "table": [ - "tb3" - ] - } - ] - } - } - } - ] - } -} diff --git a/flinkx-examples/examples/stream_to_hbase.json b/flinkx-examples/examples/stream_to_hbase.json deleted file mode 100644 index 420d071f41..0000000000 --- a/flinkx-examples/examples/stream_to_hbase.json +++ /dev/null @@ -1,77 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "sliceRecordCount": ["0"], - "column": [ - { - "name": "name", - "type": "string", - "value": "xiangqin" - }, - { - "name": "age", - "type": "int", - "value": "26" - }, - { - "name": "father", - "type": "string", - "value": "yongfang" - }, - { - "name": "adress", - "type": "string", - "value": "zhejiang" - } - ] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "column": [ - { - "name": "cf1:name", - "type": "string" - }, - { - "name": "cf1:age", - "type": "int" - }, - { - "name": "cf1:father", - "type": "string" - }, - { - "name": "cf1:adress", - "type": "string" - } - ], - "rowkeyColumn": "md5($(cf1:name)_$(cf1:father))", - "writeBufferSize": -1, - "table": "hbase_writer", - "sourceIds": [ - 371 - ], - "hbaseConfig": { - "hbase.zookeeper.quorum": "172.16.8.193" - } - }, - "name": "hbasewriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/flinkx-examples/pom.xml b/flinkx-examples/pom.xml deleted file mode 100644 index b6068aa503..0000000000 --- a/flinkx-examples/pom.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-examples - - - - commons-cli - commons-cli - 1.2 - - - - \ No newline at end of file diff --git a/flinkx-examples/src/main/java/com/dtstack/flinkx/examples/ExampleGenerator.java b/flinkx-examples/src/main/java/com/dtstack/flinkx/examples/ExampleGenerator.java deleted file mode 100644 index 0feb6334d0..0000000000 --- a/flinkx-examples/src/main/java/com/dtstack/flinkx/examples/ExampleGenerator.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.examples; - -import org.apache.commons.cli.BasicParser; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.FilenameFilter; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.OutputStreamWriter; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * The class used for generating data-transfer-task json files using variable substitution - * - * Company: www.dtstack.com - * @author huyifan.zju@163.com - */ -public class ExampleGenerator { - private static final String OPTION_CONF_DIR = "c"; - private static final String OPTION_TEMPLATE_DIR = "t"; - private static final String OPTION_OUTPUT_DIR = "o"; - private Properties substituteMap = new Properties(); - private String confDir; - private String templateDir; - private String outputDir; - private List tempList = new ArrayList<>(); - - public ExampleGenerator(String confDir, String templateDir, String outputDir) { - this.confDir = confDir; - this.templateDir = templateDir; - this.outputDir = outputDir; - } - - public void generate() throws IOException { - File dir = new File(outputDir); - if(!dir.exists()) { - boolean result = dir.mkdir(); - if (!result) { - throw new RuntimeException("目录创建失败:" + outputDir); - } - } else if(!dir.isDirectory()) { - boolean deleteResult = dir.delete(); - if (!deleteResult) { - throw new RuntimeException("目录删除失败:" + outputDir); - } - - boolean createResult = dir.mkdir(); - if (!createResult) { - throw new RuntimeException("目录创建失败:" + outputDir); - } - } - - initVarMap(); - - initTempList(); - - for(String tempFile : tempList) { - String[] part = tempFile.split(File.separatorChar=='\\' ? "\\\\" : File.separator); - String outputPath = outputDir + File.separator + part[part.length - 1]; - try(BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(tempFile), StandardCharsets.UTF_8))) { - try(BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(outputPath), StandardCharsets.UTF_8))) { - String line; - while((line = br.readLine()) != null) { - bw.write(substituteVars(line)); - bw.write("\n"); - } - } - } - } - - } - - public void initTempList() throws IOException { - File dir = new File(templateDir); - if(dir.exists() && dir.isDirectory()) { - File[] tempFiles = dir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.toLowerCase().endsWith(".json"); - } - }); - if(tempFiles != null) { - for(File tempFile : tempFiles) { - tempList.add(tempFile.getPath()); - } - } - } - } - - public void initVarMap() throws IOException { - File dir = new File(confDir); - if(dir.exists() && dir.isDirectory()) { - File[] confFiles = dir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.toLowerCase().endsWith(".conf"); - } - }); - for(File confFile : confFiles) { - substituteMap.load(new FileInputStream(confFile)); - } - } - } - - public String substituteVars(String str) { - StringBuilder sb = new StringBuilder(); - String pattern = "\\$\\{(.+?)\\}"; - Pattern r = Pattern.compile(pattern); - Matcher m = r.matcher(str); - int start = 0; - while(m.find()) { - String var = m.group(1); - if(substituteMap.containsKey(var)) { - sb.append(str.substring(start, m.start())); - sb.append(substituteMap.get(var)); - start = m.end(); - } - } - if(start < str.length()) { - sb.append(str.substring(start)); - } - return sb.toString(); - } - - private static ExampleGenerator getInstance(String[] args) throws ParseException { - Options options = new Options(); - options.addOption(OPTION_CONF_DIR, true, "Variable configuration directory"); - options.addOption(OPTION_TEMPLATE_DIR, true, "Task template directory"); - options.addOption(OPTION_OUTPUT_DIR, true, "Output Directory"); - BasicParser parser = new BasicParser(); - CommandLine cmdLine = parser.parse(options, args); - String confDir = cmdLine.getOptionValue(OPTION_CONF_DIR); - String templateDir = cmdLine.getOptionValue(OPTION_TEMPLATE_DIR); - String outputDir = cmdLine.getOptionValue(OPTION_OUTPUT_DIR); - return new ExampleGenerator(confDir, templateDir, outputDir); - } - - public static void main(String[] args) throws IOException, ParseException { - ExampleGenerator generator = getInstance(args); - generator.generate(); - } - -} diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigConstants.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigConstants.java index 03f0253d22..337d0e43bc 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigConstants.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigConstants.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigKeys.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigKeys.java index e0aea9ca1a..9c1e3df970 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigKeys.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfigKeys.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index c16807eda1..d201625068 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java index 5ff1d9eb61..c8da4124b1 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java index 4fce4b12ab..702dcf3c65 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java index 2f71951349..d83d83b3ee 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigKeys.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigKeys.java index 4fefcc3218..0eb4cfc4ee 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigKeys.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigKeys.java @@ -1,5 +1,5 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java index c0efd9463b..0bce2c0c69 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 89cb0a7503..495a89c4e6 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java index 3d446fce2f..cfae99b025 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java index 20126e65fe..913c0fd41c 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java index ee7ff67c95..fc6ced4a25 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index 84bc487084..b678812f68 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -98,12 +98,12 @@ public static Object string2col(String str, String type, SimpleDateFormat custom public static Object getWritableValue(Object writable) { Class clz = writable.getClass(); - Object ret = null; + Object ret; if(clz == IntWritable.class) { ret = ((IntWritable) writable).get(); } else if(clz == Text.class) { - ret = ((Text) writable).toString(); + ret = writable.toString(); } else if(clz == LongWritable.class) { ret = ((LongWritable) writable).get(); } else if(clz == ByteWritable.class) { @@ -120,7 +120,7 @@ public static Object getWritableValue(Object writable) { } public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) { - ObjectInspector objectInspector = null; + ObjectInspector objectInspector; switch(columnType) { case TINYINT: objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Byte.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java index ad0f0ba063..5d82539795 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java index 69213c0468..f6d802cf1b 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java index 6b2f5c108e..c13b6c3898 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java index ea30887cda..16fb276495 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -140,12 +140,14 @@ protected Row nextRecordInternal(Row row) throws IOException { Object val = null; if(metaColumn.getIndex() != -1){ - if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ - val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); - } - - if (val == null && metaColumn.getValue() != null){ - val = metaColumn.getValue(); + if (metaColumn.getIndex() < currentLine.getType().getFieldCount()) { + if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ + val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); + } + + if (val == null && metaColumn.getValue() != null){ + val = metaColumn.getValue(); + } } } else if (metaColumn.getValue() != null){ val = metaColumn.getValue(); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java index 52ba5549f2..5e3cea2a94 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java index 898fcf0895..2ba2b1c363 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java index 4b9058e4ec..38ff34f667 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 9a465dafd4..90f84054d5 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java index d0448207d9..ec30366e16 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 217fd45ad6..7c4cd0e1fa 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index 3b638288ad..a3daa372c4 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java index 9990b95d0c..684d2789eb 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsWriter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/StringUtil.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/StringUtil.java index 679f589117..d4f25dd030 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/StringUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/StringUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/ECompressType.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/ECompressType.java index a6204e2045..dba3700e60 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/ECompressType.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/ECompressType.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/EStoreType.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/EStoreType.java index 503f5ef78f..c713299422 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/EStoreType.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/EStoreType.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/HiveConfigKeys.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/HiveConfigKeys.java index 4a4bb278fe..86163be6d1 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/HiveConfigKeys.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/HiveConfigKeys.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,14 +28,13 @@ public class HiveConfigKeys { public static final String KEY_DEFAULT_FS = "defaultFS"; public static final String KEY_FS_DEFAULT_FS = "fs.defaultFS"; -// -// public static final String KEY_PATH = "path"; public static final String KEY_HADOOP_CONFIG = "hadoopConfig"; public static final String KEY_FILE_TYPE = "fileType"; public static final String KEY_PARTITION_TYPE = "partitionType"; + public static final String KEY_PARTITION = "partition"; public static final String KEY_WRITE_MODE = "writeMode"; @@ -52,26 +51,14 @@ public class HiveConfigKeys { public static final String KEY_PASSWORD = "password"; -// public static final String KEY_FULL_COLUMN_NAME_LIST = "fullColumnName"; -// -// public static final String KEY_FULL_COLUMN_TYPE_LIST = "fullColumnType"; -// -// public static final String KEY_COLUMN_NAME = "name"; -// -// public static final String KEY_COLUMN_TYPE = "type"; - public static final String KEY_COMPRESS = "compress"; public static final String KEY_INTERVAL = "interval"; public static final String KEY_BUFFER_SIZE = "bufferSize"; -// public static final String KEY_FILE_NAME = "fileName"; - public static final String KEY_CHARSET_NAME = "charsetName"; -// public static final String KEY_ROW_GROUP_SIZE = "rowGroupSize"; - public static final String KEY_MAX_FILE_SIZE = "maxFileSize"; } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TableInfo.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TableInfo.java index 386370ee73..7c31ed22e6 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TableInfo.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TableInfo.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java index 8a4f21ffdc..0de6b79e84 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java index c2b2158eb7..14d8d3fc8b 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/DateUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/DateUtil.java index 2189a9a4d3..6ac0e5a16c 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/DateUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/DateUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java index e57e8eb74e..2c36ff3c74 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java index 2be26c5cd9..7e0c0193fa 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/PathConverterUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/PathConverterUtil.java index fa34b932b7..19b125bcaa 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/PathConverterUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/PathConverterUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/RetryUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/RetryUtil.java index 2ac7faa006..a4abd4f9d4 100755 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/RetryUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/RetryUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -23,9 +23,6 @@ import java.util.List; import java.util.concurrent.Callable; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; /** * @author toutian diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/org/apache/hive/jdbc/HiveConnection.java b/flinkx-hive/flinkx-hive-core/src/main/java/org/apache/hive/jdbc/HiveConnection.java index e86482c22f..5c3c5725ea 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/org/apache/hive/jdbc/HiveConnection.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/org/apache/hive/jdbc/HiveConnection.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java index 8b85e65939..f8c69886b1 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java index 2259077ec5..5168b850a6 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java index 165ffd7ad3..ce1c5786d5 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java index aaa0150475..62abf5f7d6 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java index 9508fbbf14..448ae9768d 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java index ef77f72d4b..5b89c335d0 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java index aa59f328f3..0424bb2801 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java index 72748a7a14..758258e730 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java index 65355a6544..ec610798dc 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java @@ -22,7 +22,10 @@ import com.dtstack.flinkx.kafkabase.decoder.IDecode; import com.dtstack.flinkx.kafkabase.decoder.JsonDecoder; import com.dtstack.flinkx.kafkabase.decoder.PlainDecoder; +import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -30,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.BlockingQueue; @@ -51,6 +55,7 @@ public class KafkaBaseInputFormat extends BaseRichInputFormat { protected boolean blankIgnore; protected String encoding; protected Map consumerSettings; + protected List metaColumns; protected volatile boolean running = false; protected transient BlockingQueue queue; protected transient KafkaBaseConsumer consumer; @@ -95,7 +100,19 @@ protected void closeInternal() throws IOException { public void processEvent(Map event) { try { - queue.put(Row.of(event)); + Row row; + if(CollectionUtils.isEmpty(metaColumns)){ + row = Row.of(event); + }else{ + row = new Row(metaColumns.size()); + for (int i = 0; i < metaColumns.size(); i++) { + MetaColumn metaColumn = metaColumns.get(i); + Object value = event.get(metaColumn.getName()); + Object obj = StringUtil.string2col(String.valueOf(value), metaColumn.getType(), metaColumn.getTimeFormat()); + row.setField(i , obj); + } + } + queue.put(row); } catch (InterruptedException e) { LOG.error("takeEvent interrupted event:{} error:{}", event, e); } @@ -150,6 +167,10 @@ public void setConsumerSettings(Map consumerSettings) { this.consumerSettings = consumerSettings; } + public void setMetaColumns(List metaColumns) { + this.metaColumns = metaColumns; + } + @Override public void setRestoreConfig(RestoreConfig restoreConfig) { this.restoreConfig = restoreConfig; diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java index 5d72540717..7f3739089a 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java @@ -20,10 +20,12 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.reader.MetaColumn; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import java.util.List; import java.util.Map; import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.*; @@ -41,6 +43,7 @@ public class KafkaBaseReader extends BaseDataReader { protected String codec; protected boolean blankIgnore; protected Map consumerSettings; + protected List metaColumns; @SuppressWarnings("unchecked") public KafkaBaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { @@ -51,6 +54,7 @@ public KafkaBaseReader(DataTransferConfig config, StreamExecutionEnvironment env codec = readerConfig.getParameter().getStringVal(KEY_CODEC, "plain"); blankIgnore = readerConfig.getParameter().getBooleanVal(KEY_BLANK_IGNORE, false); consumerSettings = (Map) readerConfig.getParameter().getVal(KEY_CONSUMER_SETTINGS); + metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); } @Override @@ -62,7 +66,7 @@ public DataStream readData() { format.setBlankIgnore(blankIgnore); format.setConsumerSettings(consumerSettings); format.setRestoreConfig(restoreConfig); - + format.setMetaColumns(metaColumns); return createInput(format); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index dc952713f6..1b823c77eb 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java index c0cf771e7b..1d64181e81 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java index 9e919df0e2..3ef4726e79 100644 --- a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java +++ b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java @@ -119,7 +119,7 @@ protected String makeValues(List column) { @Override public int getFetchSize(){ - return 1000; + return Integer.MIN_VALUE; } @Override diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index 2bec64ac8e..7915c9bd64 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -22,6 +22,8 @@ import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -64,6 +66,8 @@ public void openInternal(InputSplit inputSplit) throws IOException { } querySql = buildQuerySql(inputSplit); + //MySQL流式读取 + fetchSize = Integer.MIN_VALUE; executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); @@ -118,6 +122,7 @@ public Row nextRecordInternal(Row row) throws IOException { } return super.nextRecordInternal(row); }catch (Exception e) { + LOG.error("error to get next record, row = {}, descColumnTypeList = {}, e = {}", row, new Gson().toJson(descColumnTypeList), ExceptionUtil.getErrorMessage(e)); throw new IOException("Couldn't read data - " + e.getMessage(), e); } } diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigConstants.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigConstants.java index 745fcd7770..a36d048ea2 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigConstants.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigConstants.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java index 580996b24b..71f06d6ac6 100755 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsConfigKeys.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index 707b82df7d..dd54b0ba18 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java index f6346492de..f22abeab47 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormat.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java index 8b1e650ff3..1bd4db402b 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index a90729ee0d..581d2030d9 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -25,6 +25,7 @@ import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.SysUtil; import com.dtstack.flinkx.util.TelnetUtil; +import com.google.gson.Gson; import org.apache.commons.lang.StringUtils; import org.apache.flink.util.CollectionUtil; import org.slf4j.Logger; @@ -40,9 +41,9 @@ import java.util.regex.Pattern; /** - * * Utilities for relational database connection and sql execution * company: www.dtstack.com + * * @author huyifan_zju@ */ public class DbUtil { @@ -92,15 +93,16 @@ public class DbUtil { /** * 获取jdbc连接(超时10S) - * @param url url - * @param username 账号 - * @param password 密码 + * + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; - synchronized (ClassUtil.LOCK_STR){ + synchronized (ClassUtil.LOCK_STR) { DriverManager.setLoginTimeout(10); // telnet @@ -118,9 +120,10 @@ private static Connection getConnectionInternal(String url, String username, Str /** * 获取jdbc连接(重试3次) - * @param url url - * @param username 账号 - * @param password 密码 + * + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ @@ -154,9 +157,10 @@ public static Connection getConnection(String url, String username, String passw /** * 关闭连接资源 - * @param rs ResultSet - * @param stmt Statement - * @param conn Connection + * + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection * @param commit */ public static void closeDbResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { @@ -178,7 +182,7 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con if (null != conn) { try { - if(commit){ + if (commit) { commit(conn); } @@ -191,36 +195,38 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con /** * 手动提交事物 + * * @param conn Connection */ - public static void commit(Connection conn){ + public static void commit(Connection conn) { try { - if (!conn.isClosed() && !conn.getAutoCommit()){ + if (!conn.isClosed() && !conn.getAutoCommit()) { conn.commit(); } - } catch (SQLException e){ + } catch (SQLException e) { LOG.warn("commit error:{}", ExceptionUtil.getErrorMessage(e)); } } /** * 批量执行sql + * * @param dbConn Connection * @param sqls sql列表 */ public static void executeBatch(Connection dbConn, List sqls) { - if(sqls == null || sqls.size() == 0) { + if (sqls == null || sqls.size() == 0) { return; } try { Statement stmt = dbConn.createStatement(); - for(String sql : sqls) { + for (String sql : sqls) { stmt.addBatch(sql); } stmt.executeBatch(); } catch (SQLException e) { - throw new RuntimeException("execute batch sql error:{}",e); + throw new RuntimeException("execute batch sql error:{}", e); } finally { commit(dbConn); } @@ -228,19 +234,20 @@ public static void executeBatch(Connection dbConn, List sqls) { /** * 获取某数据库某表的主键和唯一索引 - * @param table 表名 - * @param dbConn 数据库连接 + * + * @param table 表名 + * @param dbConn 数据库连接 * @return * @throws SQLException */ - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { - Map> keyMap = new HashMap<>(16); + public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { + Map> keyMap = new HashMap<>(16); DatabaseMetaData meta = dbConn.getMetaData(); - ResultSet rs = meta.getIndexInfo(null,null,table,true,false); - while(rs.next()) { + ResultSet rs = meta.getIndexInfo(null, null, table, true, false); + while (rs.next()) { String pkName = rs.getString(6); String columnName = rs.getString(9); - if(!keyMap.containsKey(pkName)) { + if (!keyMap.containsKey(pkName)) { keyMap.put(pkName, new ArrayList<>()); } keyMap.get(pkName).add(columnName); @@ -250,13 +257,14 @@ public static Map> getPrimaryOrUniqueKeys(String table, Conn /** * 封装channel通道顺序 + * * @param channels * @return */ - public static Object[][] getParameterValues(final int channels){ + public static Object[][] getParameterValues(final int channels) { ParameterValuesProvider provider = () -> { Integer[][] parameters = new Integer[channels][]; - for(int i = 0; i < channels; ++i) { + for (int i = 0; i < channels; ++i) { parameters[i] = new Integer[2]; parameters[i][0] = channels; parameters[i][1] = i; @@ -269,17 +277,17 @@ public static Object[][] getParameterValues(final int channels){ /** * 获取表列名类型列表 + * * @param dbUrl jdbc url * @param username 数据库账号 * @param password 数据库密码 * @param databaseInterface DatabaseInterface * @param table 表名 - * @param metaColumns MetaColumn列表 - * @return + * @param metaColumns MetaColumn列表 + * @return 字段类型list列表 */ - public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, - String table, List metaColumns) { - List ret = new ArrayList<>(metaColumns.size()); + public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, String table, List metaColumns) { + List descColumnTypeList = new ArrayList<>(metaColumns.size()); Connection dbConn = null; Statement stmt = null; ResultSet rs = null; @@ -289,35 +297,43 @@ public static List analyzeTable(String dbUrl, String username, String pa rs = stmt.executeQuery(databaseInterface.getSqlQueryFields(databaseInterface.quoteTable(table))); ResultSetMetaData rd = rs.getMetaData(); - Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); - for(int i = 0; i < rd.getColumnCount(); ++i) { - nameTypeMap.put(rd.getColumnName(i+1),rd.getColumnTypeName(i+1)); + boolean flag = ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName()); + Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); + for (int i = 1; i <= rd.getColumnCount(); i++) { + if(flag){ + descColumnTypeList.add(rd.getColumnTypeName(i)); + }else{ + nameTypeMap.put(rd.getColumnName(i), rd.getColumnTypeName(i)); + } } - - for (MetaColumn metaColumn : metaColumns) { - if(metaColumn.getValue() != null){ - ret.add("string"); - } else { - ret.add(nameTypeMap.get(metaColumn.getName())); + if(!flag){ + for (MetaColumn metaColumn : metaColumns) { + if (metaColumn.getValue() != null) { + descColumnTypeList.add("string"); + } else { + descColumnTypeList.add(nameTypeMap.get(metaColumn.getName())); + } } } } catch (SQLException e) { + LOG.error("error to analyzeTable, dbUrl = {}, table = {}, metaColumns = {}, e = {}", dbUrl, table, new Gson().toJson(metaColumns), ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); } finally { closeDbResources(rs, stmt, dbConn, false); } - return ret; + return descColumnTypeList; } /** * 占位符设值 - * @param param 参数 - * @param statement PreparedStatement - * @param i 占位符位置 + * + * @param param 参数 + * @param statement PreparedStatement + * @param i 占位符位置 * @throws SQLException */ - public static void setParameterValue(Object param,PreparedStatement statement,int i) throws SQLException{ + public static void setParameterValue(Object param, PreparedStatement statement, int i) throws SQLException { if (param instanceof String) { statement.setString(i + 1, (String) param); } else if (param instanceof Long) { @@ -346,24 +362,25 @@ public static void setParameterValue(Object param,PreparedStatement statement,in statement.setArray(i + 1, (Array) param); } else { //extends with other types if needed - throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)." ); + throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)."); } } /** * clob转string - * @param obj clob + * + * @param obj clob * @return * @throws Exception */ - public static Object clobToString(Object obj) throws Exception{ + public static Object clobToString(Object obj) throws Exception { String dataStr; - if(obj instanceof Clob){ - Clob clob = (Clob)obj; + if (obj instanceof Clob) { + Clob clob = (Clob) obj; BufferedReader bf = new BufferedReader(clob.getCharacterStream()); StringBuilder stringBuilder = new StringBuilder(); String line; - while ((line = bf.readLine()) != null){ + while ((line = bf.readLine()) != null) { stringBuilder.append(line); } dataStr = stringBuilder.toString(); @@ -376,32 +393,34 @@ public static Object clobToString(Object obj) throws Exception{ /** * 获取纳秒字符串 + * * @param timeStr * @return */ - public static String getNanosTimeStr(String timeStr){ - if(timeStr.length() < 29){ - timeStr += StringUtils.repeat("0",29 - timeStr.length()); + public static String getNanosTimeStr(String timeStr) { + if (timeStr.length() < 29) { + timeStr += StringUtils.repeat("0", 29 - timeStr.length()); } return timeStr; } /** * 将边界位置时间转换成对应饿的纳秒时间 + * * @param startLocation 边界位置(起始/结束) * @return */ - public static int getNanos(long startLocation){ + public static int getNanos(long startLocation) { String timeStr = String.valueOf(startLocation); int nanos; - if (timeStr.length() == SECOND_LENGTH){ + if (timeStr.length() == SECOND_LENGTH) { nanos = 0; - } else if (timeStr.length() == MILLIS_LENGTH){ - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MILLIS_LENGTH)) * 1000000; - } else if (timeStr.length() == MICRO_LENGTH){ - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MICRO_LENGTH)) * 1000; - } else if (timeStr.length() == NANOS_LENGTH){ - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,NANOS_LENGTH)); + } else if (timeStr.length() == MILLIS_LENGTH) { + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MILLIS_LENGTH)) * 1000000; + } else if (timeStr.length() == MICRO_LENGTH) { + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MICRO_LENGTH)) * 1000; + } else if (timeStr.length() == NANOS_LENGTH) { + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, NANOS_LENGTH)); } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); } @@ -411,19 +430,20 @@ public static int getNanos(long startLocation){ /** * 将边界位置时间转换成对应饿的毫秒时间 - * @param startLocation 边界位置(起始/结束) + * + * @param startLocation 边界位置(起始/结束) * @return */ - public static long getMillis(long startLocation){ + public static long getMillis(long startLocation) { String timeStr = String.valueOf(startLocation); long millisSecond; - if (timeStr.length() == SECOND_LENGTH){ + if (timeStr.length() == SECOND_LENGTH) { millisSecond = startLocation * 1000; - } else if (timeStr.length() == MILLIS_LENGTH){ + } else if (timeStr.length() == MILLIS_LENGTH) { millisSecond = startLocation; - } else if (timeStr.length() == MICRO_LENGTH){ + } else if (timeStr.length() == MICRO_LENGTH) { millisSecond = startLocation / 1000; - } else if (timeStr.length() == NANOS_LENGTH){ + } else if (timeStr.length() == NANOS_LENGTH) { millisSecond = startLocation / 1000000; } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); @@ -434,23 +454,24 @@ public static long getMillis(long startLocation){ /** * 格式化jdbc连接 - * @param dbUrl 原jdbc连接 - * @param extParamMap 需要额外添加的参数 - * @return 格式化后jdbc连接URL字符串 + * + * @param dbUrl 原jdbc连接 + * @param extParamMap 需要额外添加的参数 + * @return 格式化后jdbc连接URL字符串 */ - public static String formatJdbcUrl(String dbUrl, Map extParamMap){ + public static String formatJdbcUrl(String dbUrl, Map extParamMap) { String[] splits = DB_PATTERN.split(dbUrl); - Map paramMap = new HashMap<>(16); - if(splits.length > 1) { + Map paramMap = new HashMap<>(16); + if (splits.length > 1) { String[] pairs = splits[1].split("&"); - for(String pair : pairs) { + for (String pair : pairs) { String[] leftRight = pair.split("="); paramMap.put(leftRight[0], leftRight[1]); } } - if(!CollectionUtil.isNullOrEmpty(extParamMap)){ + if (!CollectionUtil.isNullOrEmpty(extParamMap)) { paramMap.putAll(extParamMap); } paramMap.put("useCursorFetch", "true"); @@ -459,8 +480,8 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap) StringBuffer sb = new StringBuffer(dbUrl.length() + 128); sb.append(splits[0]).append("?"); int index = 0; - for(Map.Entry entry : paramMap.entrySet()) { - if(index != 0) { + for (Map.Entry entry : paramMap.entrySet()) { + if (index != 0) { sb.append("&"); } sb.append(entry.getKey()).append("=").append(entry.getValue()); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java index 14c2c058f1..27cecb89db 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java @@ -160,6 +160,13 @@ private String buildQuerySqlWithCustomSql(){ querySql.append(" ").append(RESTORE_FILTER_PLACEHOLDER); } + if (customFilter != null){ + customFilter = customFilter.trim(); + if (customFilter.length() > 0){ + querySql.append(" AND ").append(customFilter); + } + } + return querySql.toString(); } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index 8578075390..7e0bd337b5 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -800,7 +800,9 @@ protected void queryForPolling(String startLocation) throws SQLException { * @throws SQLException */ protected void executeQuery(String startLocation) throws SQLException { - dbConn = DbUtil.getConnection(dbUrl, username, password); + if(dbConn==null){ + dbConn = DbUtil.getConnection(dbUrl, username, password); + } // 部分驱动需要关闭事务自动提交,fetchSize参数才会起作用 dbConn.setAutoCommit(false); if (incrementConfig.isPolling()) { diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java index 35929bd5c1..bd2dc5afe1 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java @@ -27,6 +27,7 @@ import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; +import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.ObjectUtils; import org.apache.flink.types.Row; @@ -383,6 +384,7 @@ protected boolean needWaitBeforeWriteRecords() { @Override protected void beforeWriteRecords() { if(taskNumber == 0) { + LOG.info("start to execute preSql, preSql = {}", new Gson().toJson(preSql)); DbUtil.executeBatch(dbConn, preSql); } } @@ -396,6 +398,7 @@ protected boolean needWaitBeforeCloseInternal() { protected void beforeCloseInternal() { // 执行postsql if(taskNumber == 0) { + LOG.info("start to execute postSql, postSql = {}", new Gson().toJson(postSql)); DbUtil.executeBatch(dbConn, postSql); } } diff --git a/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/MockDataUtil.java b/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/MockDataUtil.java index 3521a4f5c9..c8630a4260 100644 --- a/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/MockDataUtil.java +++ b/flinkx-stream/flinkx-stream-reader/src/main/java/com/dtstack/flinkx/stream/reader/MockDataUtil.java @@ -20,15 +20,19 @@ import com.dtstack.flinkx.reader.MetaColumn; import com.github.jsonzou.jmockdata.JMockData; +import com.github.jsonzou.jmockdata.MockConfig; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.types.Row; +import java.io.ByteArrayInputStream; import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.util.Date; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; /** * @author jiangbo @@ -36,9 +40,15 @@ */ public class MockDataUtil { + private static AtomicLong id = new AtomicLong(0L); + private static int minSize = 320; + private static int maxSize = 320; + private static MockConfig mockConfig = new MockConfig().subConfig(String.class).sizeRange(minSize,maxSize).globalConfig(); + public static Object mockData(String type){ Object mockData; switch (type.trim().toLowerCase()){ + case "id": mockData = id.incrementAndGet();break; case "int": case "integer": mockData = JMockData.mock(int.class);break; case "byte": mockData = JMockData.mock(byte.class);break; @@ -63,6 +73,9 @@ public static Object mockData(String type){ case "float[]": mockData = JMockData.mock(float[].class);break; case "double[]": mockData = JMockData.mock(double[].class);break; case "string[]": mockData = JMockData.mock(String[].class);break; + case "binary": + String str = JMockData.mock(String.class, mockConfig); + mockData = new ByteArrayInputStream(str.getBytes(StandardCharsets.UTF_8));break; default: mockData = JMockData.mock(String.class);break; } diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml new file mode 100644 index 0000000000..dd01212dd4 --- /dev/null +++ b/flinkx-test/pom.xml @@ -0,0 +1,370 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-test + + + 1.8.1 + 1.6 + 0.3.0 + + + + + ch.qos.logback + logback-classic + 1.1.7 + + + com.google.guava + guava + 19.0 + + + com.google.code.gson + gson + 2.7 + + + hadoop-mapreduce-client-core + org.apache.hadoop + ${hadoop.version} + + + hadoop-common + org.apache.hadoop + ${hadoop.version} + + + hadoop-hdfs + org.apache.hadoop + ${hadoop.version} + + + org.apache.flink + flink-metrics-core + ${flink.version} + + + org.apache.flink + flink-metrics-prometheus_2.11 + ${flink.version} + + + io.prometheus + simpleclient + ${prometheus.version} + + + io.prometheus + simpleclient_httpserver + ${prometheus.version} + + + io.prometheus + simpleclient_pushgateway + ${prometheus.version} + + + com.dtstack.flinkx + flinkx-core + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-stream-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-stream-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-carbondata-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-carbondata-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-db2-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-db2-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-es-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-es-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-ftp-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-ftp-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-hbase-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-hbase-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-hdfs-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-hdfs-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-mongodb-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-mongodb-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-mysql-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-mysql-dreader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-mysql-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-odps-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-odps-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-oracle-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-oracle-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-postgresql-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-postgresql-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-sqlserver-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-sqlserver-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-redis-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-rdb-core + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-rdb-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-rdb-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-gbase-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-gbase-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-binlog-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-hive-writer + ${flinkx.version} + + + org.apache.hive + hive-serde + 2.1.0 + + + com.dtstack.flinkx + flinkx-kafka11-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka11-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka10-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka10-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka09-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka09-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kudu-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kudu-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-kafka-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-clickhouse-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-clickhouse-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-polardb-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-polardb-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-phoenix-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-phoenix-writer + ${flinkx.version} + + + + com.dtstack.flinkx + flinkx-emqx-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-emqx-writer + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-dm-reader + ${flinkx.version} + + + com.dtstack.flinkx + flinkx-dm-writer + ${flinkx.version} + + + + + \ No newline at end of file diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java new file mode 100644 index 0000000000..33fc1dd035 --- /dev/null +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.test; + +import ch.qos.logback.classic.Level; +import ch.qos.logback.classic.LoggerContext; +import com.dtstack.flink.api.java.MyLocalStreamEnvironment; +import com.dtstack.flinkx.binlog.reader.BinlogReader; +import com.dtstack.flinkx.carbondata.reader.CarbondataReader; +import com.dtstack.flinkx.carbondata.writer.CarbondataWriter; +import com.dtstack.flinkx.clickhouse.reader.ClickhouseReader; +import com.dtstack.flinkx.clickhouse.writer.ClickhouseWriter; +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.SpeedConfig; +import com.dtstack.flinkx.constants.ConfigConstant; +import com.dtstack.flinkx.db2.reader.Db2Reader; +import com.dtstack.flinkx.db2.writer.Db2Writer; +import com.dtstack.flinkx.dm.reader.DmReader; +import com.dtstack.flinkx.dm.writer.DmWriter; +import com.dtstack.flinkx.emqx.reader.EmqxReader; +import com.dtstack.flinkx.emqx.writer.EmqxWriter; +import com.dtstack.flinkx.es.reader.EsReader; +import com.dtstack.flinkx.es.writer.EsWriter; +import com.dtstack.flinkx.ftp.reader.FtpReader; +import com.dtstack.flinkx.ftp.writer.FtpWriter; +import com.dtstack.flinkx.gbase.reader.GbaseReader; +import com.dtstack.flinkx.gbase.writer.GbaseWriter; +import com.dtstack.flinkx.hbase.reader.HbaseReader; +import com.dtstack.flinkx.hbase.writer.HbaseWriter; +import com.dtstack.flinkx.hdfs.reader.HdfsReader; +import com.dtstack.flinkx.hdfs.writer.HdfsWriter; +import com.dtstack.flinkx.hive.writer.HiveWriter; +import com.dtstack.flinkx.kafka.reader.KafkaReader; +import com.dtstack.flinkx.kafka.writer.KafkaWriter; +import com.dtstack.flinkx.kafka09.reader.Kafka09Reader; +import com.dtstack.flinkx.kafka09.writer.Kafka09Writer; +import com.dtstack.flinkx.kafka10.reader.Kafka10Reader; +import com.dtstack.flinkx.kafka10.writer.Kafka10Writer; +import com.dtstack.flinkx.kafka11.reader.Kafka11Reader; +import com.dtstack.flinkx.kafka11.writer.Kafka11Writer; +import com.dtstack.flinkx.kudu.reader.KuduReader; +import com.dtstack.flinkx.kudu.writer.KuduWriter; +import com.dtstack.flinkx.mongodb.reader.MongodbReader; +import com.dtstack.flinkx.mongodb.writer.MongodbWriter; +import com.dtstack.flinkx.mysql.reader.MysqlReader; +import com.dtstack.flinkx.mysql.writer.MysqlWriter; +import com.dtstack.flinkx.mysqld.reader.MysqldReader; +import com.dtstack.flinkx.odps.reader.OdpsReader; +import com.dtstack.flinkx.odps.writer.OdpsWriter; +import com.dtstack.flinkx.oracle.reader.OracleReader; +import com.dtstack.flinkx.oracle.writer.OracleWriter; +import com.dtstack.flinkx.phoenix.reader.PhoenixReader; +import com.dtstack.flinkx.phoenix.writer.PhoenixWriter; +import com.dtstack.flinkx.polardb.reader.PolardbReader; +import com.dtstack.flinkx.polardb.writer.PolardbWriter; +import com.dtstack.flinkx.postgresql.reader.PostgresqlReader; +import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; +import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.redis.writer.RedisWriter; +import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; +import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; +import com.dtstack.flinkx.stream.reader.StreamReader; +import com.dtstack.flinkx.stream.writer.StreamWriter; +import com.dtstack.flinkx.util.ResultPrintUtil; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.nio.charset.StandardCharsets; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +/** + * @author jiangbo + */ +public class LocalTest { + + private static final int FAILURE_RATE = 3; + private static final int FAILURE_INTERVAL = 6; + private static final int DELAY_INTERVAL = 10; + public static Logger LOG = LoggerFactory.getLogger(LocalTest.class); + public static Configuration conf = new Configuration(); + + public static void main(String[] args) throws Exception { + setLogLevel(Level.INFO.toString()); + + Properties confProperties = new Properties(); +// confProperties.put("flink.checkpoint.interval", "10000"); +// confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); +// + conf.setString("akka.ask.timeout", "180 s"); + conf.setString("web.timeout", "100000"); +// conf.setString("metrics.reporter.promgateway.class","org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"); +// conf.setString("metrics.reporter.promgateway.host","172.16.8.178"); +// conf.setString("metrics.reporter.promgateway.port","9091"); +// conf.setString("metrics.reporter.promgateway.jobName","kanata"); +// conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); +// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","false"); + + String jobPath = "D:\\dtstack\\flinkx-all\\flinkx-examples\\examples\\clickhouse_stream.json"; + String savePointPath = ""; + JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, savePointPath); + ResultPrintUtil.printResult(result); + } + + public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savePointPath) throws Exception { + String jobContent = readJob(jobFile); + return runJob(jobContent, confProperties, savePointPath); + } + + public static JobExecutionResult runJob(String job, Properties confProperties, String savePointPath) throws Exception { + DataTransferConfig config = DataTransferConfig.parse(job); + MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); + openCheckpointConf(env, confProperties); + env.setParallelism(config.getJob().getSetting().getSpeed().getChannel()); + env.setRestartStrategy(RestartStrategies.noRestart()); + + BaseDataReader reader = buildDataReader(config, env); + DataStream dataStream = reader.readData(); + SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); + if (speedConfig.getReaderChannel() > 0) { + dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); + } + + if (speedConfig.isRebalance()) { + dataStream = dataStream.rebalance(); + } + + BaseDataWriter dataWriter = buildDataWriter(config); + DataStreamSink dataStreamSink = dataWriter.writeData(dataStream); + if (speedConfig.getWriterChannel() > 0) { + dataStreamSink.setParallelism(speedConfig.getWriterChannel()); + } + + if (StringUtils.isNotEmpty(savePointPath)) { + env.setSettings(SavepointRestoreSettings.forPath(savePointPath)); + } + + return env.execute(); + } + + private static String readJob(File file) { + try(FileInputStream in = new FileInputStream(file);) { + byte[] fileContent = new byte[(int) file.length()]; + in.read(fileContent); + return new String(fileContent, StandardCharsets.UTF_8); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env) { + String readerName = config.getJob().getContent().get(0).getReader().getName(); + BaseDataReader reader; + switch (readerName) { + case PluginNameConstants.STREAM_READER: + reader = new StreamReader(config, env); + break; + case PluginNameConstants.CARBONDATA_READER: + reader = new CarbondataReader(config, env); + break; + case PluginNameConstants.ORACLE_READER: + reader = new OracleReader(config, env); + break; + case PluginNameConstants.POSTGRESQL_READER: + reader = new PostgresqlReader(config, env); + break; + case PluginNameConstants.SQLSERVER_READER: + reader = new SqlserverReader(config, env); + break; + case PluginNameConstants.MYSQLD_READER: + reader = new MysqldReader(config, env); + break; + case PluginNameConstants.MYSQL_READER: + reader = new MysqlReader(config, env); + break; + case PluginNameConstants.DB2_READER: + reader = new Db2Reader(config, env); + break; + case PluginNameConstants.GBASE_READER: + reader = new GbaseReader(config, env); + break; + case PluginNameConstants.ES_READER: + reader = new EsReader(config, env); + break; + case PluginNameConstants.FTP_READER: + reader = new FtpReader(config, env); + break; + case PluginNameConstants.HBASE_READER: + reader = new HbaseReader(config, env); + break; + case PluginNameConstants.HDFS_READER: + reader = new HdfsReader(config, env); + break; + case PluginNameConstants.MONGODB_READER: + reader = new MongodbReader(config, env); + break; + case PluginNameConstants.ODPS_READER: + reader = new OdpsReader(config, env); + break; + case PluginNameConstants.BINLOG_READER: + reader = new BinlogReader(config, env); + break; + case PluginNameConstants.KAFKA09_READER: + reader = new Kafka09Reader(config, env); + break; + case PluginNameConstants.KAFKA10_READER: + reader = new Kafka10Reader(config, env); + break; + case PluginNameConstants.KAFKA11_READER: + reader = new Kafka11Reader(config, env); + break; + case PluginNameConstants.KAFKA_READER: + reader = new KafkaReader(config, env); + break; + case PluginNameConstants.KUDU_READER: + reader = new KuduReader(config, env); + break; + case PluginNameConstants.CLICKHOUSE_READER: + reader = new ClickhouseReader(config, env); + break; + case PluginNameConstants.POLARDB_READER: + reader = new PolardbReader(config, env); + break; + case PluginNameConstants.PHOENIX_READER: + reader = new PhoenixReader(config, env); + break; + case PluginNameConstants.EMQX_READER: + reader = new EmqxReader(config, env); + break; + case PluginNameConstants.DM_READER: + reader = new DmReader(config, env); + break; + default: + throw new IllegalArgumentException("Can not find reader by name:" + readerName); + } + + return reader; + } + + private static BaseDataWriter buildDataWriter(DataTransferConfig config) { + String writerName = config.getJob().getContent().get(0).getWriter().getName(); + BaseDataWriter writer; + switch (writerName) { + case PluginNameConstants.STREAM_WRITER: + writer = new StreamWriter(config); + break; + case PluginNameConstants.CARBONDATA_WRITER: + writer = new CarbondataWriter(config); + break; + case PluginNameConstants.MYSQL_WRITER: + writer = new MysqlWriter(config); + break; + case PluginNameConstants.SQLSERVER_WRITER: + writer = new SqlserverWriter(config); + break; + case PluginNameConstants.ORACLE_WRITER: + writer = new OracleWriter(config); + break; + case PluginNameConstants.POSTGRESQL_WRITER: + writer = new PostgresqlWriter(config); + break; + case PluginNameConstants.DB2_WRITER: + writer = new Db2Writer(config); + break; + case PluginNameConstants.GBASE_WRITER: + writer = new GbaseWriter(config); + break; + case PluginNameConstants.ES_WRITER: + writer = new EsWriter(config); + break; + case PluginNameConstants.FTP_WRITER: + writer = new FtpWriter(config); + break; + case PluginNameConstants.HBASE_WRITER: + writer = new HbaseWriter(config); + break; + case PluginNameConstants.HDFS_WRITER: + writer = new HdfsWriter(config); + break; + case PluginNameConstants.MONGODB_WRITER: + writer = new MongodbWriter(config); + break; + case PluginNameConstants.ODPS_WRITER: + writer = new OdpsWriter(config); + break; + case PluginNameConstants.REDIS_WRITER: + writer = new RedisWriter(config); + break; + case PluginNameConstants.HIVE_WRITER: + writer = new HiveWriter(config); + break; + case PluginNameConstants.KAFKA09_WRITER: + writer = new Kafka09Writer(config); + break; + case PluginNameConstants.KAFKA10_WRITER: + writer = new Kafka10Writer(config); + break; + case PluginNameConstants.KAFKA11_WRITER: + writer = new Kafka11Writer(config); + break; + case PluginNameConstants.KUDU_WRITER: + writer = new KuduWriter(config); + break; + case PluginNameConstants.CLICKHOUSE_WRITER: + writer = new ClickhouseWriter(config); + break; + case PluginNameConstants.POLARDB_WRITER: + writer = new PolardbWriter(config); + break; + case PluginNameConstants.KAFKA_WRITER: + writer = new KafkaWriter(config); + break; + case PluginNameConstants.PHOENIX_WRITER: + writer = new PhoenixWriter(config); + break; + case PluginNameConstants.EMQX_WRITER: + writer = new EmqxWriter(config); + break; + case PluginNameConstants.DM_WRITER: + writer = new DmWriter(config); + break; + default: + throw new IllegalArgumentException("Can not find writer by name:" + writerName); + } + + return writer; + } + + private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties) { + if (properties == null) { + return; + } + + if (properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null) { + return; + } else { + long interval = Long.parseLong(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY).trim()); + + //start checkpoint every ${interval} + env.enableCheckpointing(interval); + + LOG.info("Open checkpoint with interval:" + interval); + } + + String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); + if (checkpointTimeoutStr != null) { + long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); + //checkpoints have to complete within one min,or are discard + env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); + + LOG.info("Set checkpoint timeout:" + checkpointTimeout); + } + + env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); + env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + env.setRestartStrategy(RestartStrategies.failureRateRestart( + FAILURE_RATE, + Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), + Time.of(DELAY_INTERVAL, TimeUnit.SECONDS) + )); + } + + private static void setLogLevel(String level) { + LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); + ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); + logger.setLevel(Level.toLevel(level)); + } +} diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java new file mode 100644 index 0000000000..f0f047e6b5 --- /dev/null +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.test; + +/** + * @author jiangbo + */ +@SuppressWarnings("all") +public class PluginNameConstants { + + public static final String STREAM_READER = "streamreader"; + public static final String CARBONDATA_READER = "carbondatareader"; + public static final String DB2_READER = "db2reader"; + public static final String ES_READER = "esreader"; + public static final String FTP_READER = "ftpreader"; + public static final String HBASE_READER = "hbasereader"; + public static final String HDFS_READER = "hdfsreader"; + public static final String MONGODB_READER = "mongodbreader"; + public static final String MYSQLD_READER = "mysqldreader"; + public static final String MYSQL_READER = "mysqlreader"; + public static final String ODPS_READER = "odpsreader"; + public static final String ORACLE_READER = "oraclereader"; + public static final String POSTGRESQL_READER = "postgresqlreader"; + public static final String SQLSERVER_READER = "sqlserverreader"; + public static final String GBASE_READER = "gbasereader"; + public static final String KUDU_READER = "kudureader"; + public static final String BINLOG_READER = "binlogreader"; + public static final String KAFKA09_READER = "kafka09reader"; + public static final String KAFKA10_READER = "kafka10reader"; + public static final String KAFKA11_READER = "kafka11reader"; + public static final String KAFKA_READER = "kafkareader"; + public static final String CLICKHOUSE_READER = "clickhousereader"; + public static final String POLARDB_READER = "polardbreader"; + public static final String ORACLE_LOG_MINER_READER = "oraclelogminerreader"; + public static final String PHOENIX_READER = "phoenixreader"; + public static final String EMQX_READER = "emqxreader"; + public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; + public static final String METADATAHIVE2_READER = "metadatahive2reader"; + public static final String DM_READER = "dmreader"; + + public static final String STREAM_WRITER = "streamwriter"; + public static final String CARBONDATA_WRITER = "carbondatawriter"; + public static final String DB2_WRITER = "db2writer"; + public static final String ES_WRITER = "eswriter"; + public static final String FTP_WRITER = "ftpwriter"; + public static final String HBASE_WRITER = "hbasewriter"; + public static final String HDFS_WRITER = "hdfswriter"; + public static final String MONGODB_WRITER = "mongodbwriter"; + public static final String MYSQL_WRITER = "mysqlwriter"; + public static final String ODPS_WRITER = "odpswriter"; + public static final String ORACLE_WRITER = "oraclewriter"; + public static final String POSTGRESQL_WRITER = "postgresqlwriter"; + public static final String REDIS_WRITER = "rediswriter"; + public static final String SQLSERVER_WRITER = "sqlserverwriter"; + public static final String GBASE_WRITER = "gbasewriter"; + public static final String KUDU_WRITER = "kuduwriter"; + public static final String HIVE_WRITER = "hivewriter"; + public static final String KAFKA09_WRITER = "kafka09writer"; + public static final String KAFKA10_WRITER = "kafka10writer"; + public static final String KAFKA11_WRITER = "kafka11writer"; + public static final String KAFKA_WRITER = "kafkawriter"; + public static final String CLICKHOUSE_WRITER = "clickhousewriter"; + public static final String POLARDB_WRITER = "polardbwriter"; + public static final String PHOENIX_WRITER = "phoenixwriter"; + public static final String EMQX_WRITER = "emqxwriter"; + public static final String RESTAPI_WRITER = "restapiwriter"; + public static final String DM_WRITER = "dmwriter"; +} diff --git a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java new file mode 100644 index 0000000000..5fbf58b645 --- /dev/null +++ b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.metrics.prometheus; + +import io.prometheus.client.Collector; +import io.prometheus.client.CollectorRegistry; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; +import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.FILTER_LABEL_VALUE_CHARACTER; + +/** + * base prometheus reporter for prometheus metrics. + */ +@PublicEvolving +public abstract class AbstractPrometheusReporter implements MetricReporter { + + private static final Pattern UNALLOWED_CHAR_PATTERN = Pattern.compile("[^a-zA-Z0-9:_]"); + private static final CharacterFilter CHARACTER_FILTER = new CharacterFilter() { + @Override + public String filterCharacters(String input) { + return replaceInvalidChars(input); + } + }; + private static final char SCOPE_SEPARATOR = '_'; + private static final String SCOPE_PREFIX = "flink" + SCOPE_SEPARATOR; + protected final Logger log = LoggerFactory.getLogger(getClass()); + private final Map> collectorsWithCountByMetricName = new HashMap<>(); + private CharacterFilter labelValueCharactersFilter = CHARACTER_FILTER; + + @VisibleForTesting + static String replaceInvalidChars(final String input) { + // https://prometheus.io/docs/instrumenting/writing_exporters/ + // Only [a-zA-Z0-9:_] are valid in metric names, any other characters should be sanitized to an underscore. + return UNALLOWED_CHAR_PATTERN.matcher(input).replaceAll("_"); + } + + private static String getScopedName(String metricName, MetricGroup group) { + return SCOPE_PREFIX + getLogicalScope(group) + SCOPE_SEPARATOR + CHARACTER_FILTER.filterCharacters(metricName); + } + + @SuppressWarnings("unchecked") + private static String getLogicalScope(MetricGroup group) { + return ((FrontMetricGroup>) group).getLogicalScope(CHARACTER_FILTER, SCOPE_SEPARATOR); + } + + private static io.prometheus.client.Gauge.Child gaugeFrom(Counter counter) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + return (double) counter.getCount(); + } + }; + } + + private static io.prometheus.client.Gauge.Child gaugeFrom(Meter meter) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + return meter.getRate(); + } + }; + } + + private static List addToList(List list, String element) { + final List result = new ArrayList<>(list); + result.add(element); + return result; + } + + private static String[] toArray(List list) { + return list.toArray(new String[list.size()]); + } + + @Override + public void open(MetricConfig config) { + boolean filterLabelValueCharacters = config.getBoolean( + FILTER_LABEL_VALUE_CHARACTER.key(), FILTER_LABEL_VALUE_CHARACTER.defaultValue()); + + if (!filterLabelValueCharacters) { + labelValueCharactersFilter = input -> input; + } + } + + @Override + public void close() { + CollectorRegistry.defaultRegistry.clear(); + } + + @Override + public void notifyOfAddedMetric(final Metric metric, final String metricName, final MetricGroup group) { + + List dimensionKeys = new LinkedList<>(); + List dimensionValues = new LinkedList<>(); + for (final Map.Entry dimension : group.getAllVariables().entrySet()) { + final String key = dimension.getKey(); + dimensionKeys.add(CHARACTER_FILTER.filterCharacters(key.substring(1, key.length() - 1))); + dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); + } + + final String scopedMetricName = getScopedName(metricName, group); + final String helpString = metricName + " (scope: " + getLogicalScope(group) + ")"; + + final Collector collector; + Integer count = 0; + + synchronized (this) { + if (collectorsWithCountByMetricName.containsKey(scopedMetricName)) { + final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); + collector = collectorWithCount.getKey(); + count = collectorWithCount.getValue(); + } else { + collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); + try { + collector.register(); + } catch (Exception e) { + log.warn("There was a problem registering metric {}.", metricName, e); + } + } + addMetric(metric, dimensionValues, collector); + collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count + 1)); + } + } + + private Collector createCollector(Metric metric, List dimensionKeys, List dimensionValues, String scopedMetricName, String helpString) { + Collector collector; + if (metric instanceof Gauge || metric instanceof Counter || metric instanceof Meter) { + collector = io.prometheus.client.Gauge + .build() + .name(scopedMetricName) + .help(helpString) + .labelNames(toArray(dimensionKeys)) + .create(); + } else if (metric instanceof Histogram) { + collector = new HistogramSummaryProxy((Histogram) metric, scopedMetricName, helpString, dimensionKeys, dimensionValues); + } else { + log.warn("Cannot create collector for unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + collector = null; + } + return collector; + } + + private void addMetric(Metric metric, List dimensionValues, Collector collector) { + if (metric instanceof Gauge) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Gauge) metric), toArray(dimensionValues)); + } else if (metric instanceof Counter) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Counter) metric), toArray(dimensionValues)); + } else if (metric instanceof Meter) { + ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Meter) metric), toArray(dimensionValues)); + } else if (metric instanceof Histogram) { + ((HistogramSummaryProxy) collector).addChild((Histogram) metric, dimensionValues); + } else { + log.warn("Cannot add unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + } + } + + private void removeMetric(Metric metric, List dimensionValues, Collector collector) { + if (metric instanceof Gauge) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Counter) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Meter) { + ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); + } else if (metric instanceof Histogram) { + ((HistogramSummaryProxy) collector).remove(dimensionValues); + } else { + log.warn("Cannot remove unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", + metric.getClass().getName()); + } + } + + @Override + public void notifyOfRemovedMetric(final Metric metric, final String metricName, final MetricGroup group) { + + List dimensionValues = new LinkedList<>(); + for (final Map.Entry dimension : group.getAllVariables().entrySet()) { + dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); + } + + final String scopedMetricName = getScopedName(metricName, group); + synchronized (this) { + final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); + final Integer count = collectorWithCount.getValue(); + final Collector collector = collectorWithCount.getKey(); + + removeMetric(metric, dimensionValues, collector); + + if (count == 1) { + try { + CollectorRegistry.defaultRegistry.unregister(collector); + } catch (Exception e) { + log.warn("There was a problem unregistering metric {}.", scopedMetricName, e); + } + collectorsWithCountByMetricName.remove(scopedMetricName); + } else { + collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count - 1)); + } + } + } + + @VisibleForTesting + io.prometheus.client.Gauge.Child gaugeFrom(Gauge gauge) { + return new io.prometheus.client.Gauge.Child() { + @Override + public double get() { + final Object value = gauge.getValue(); + if (value == null) { + log.debug("Gauge {} is null-valued, defaulting to 0.", gauge); + return 0; + } + if (value instanceof Double) { + return (double) value; + } + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + if (value instanceof Boolean) { + return ((Boolean) value) ? 1 : 0; + } + log.debug("Invalid type for Gauge {}: {}, only number types and booleans are supported by this reporter.", + gauge, value.getClass().getName()); + return 0; + } + }; + } + + @VisibleForTesting + static class HistogramSummaryProxy extends Collector { + static final List QUANTILES = Arrays.asList(.5, .75, .95, .98, .99, .999); + + private final String metricName; + private final String helpString; + private final List labelNamesWithQuantile; + + private final Map, Histogram> histogramsByLabelValues = new HashMap<>(); + + HistogramSummaryProxy(final Histogram histogram, final String metricName, final String helpString, final List labelNames, final List labelValues) { + this.metricName = metricName; + this.helpString = helpString; + this.labelNamesWithQuantile = addToList(labelNames, "quantile"); + histogramsByLabelValues.put(labelValues, histogram); + } + + @Override + public List collect() { + // We cannot use SummaryMetricFamily because it is impossible to get a sum of all values (at least for Dropwizard histograms, + // whose snapshot's values array only holds a sample of recent values). + + List samples = new LinkedList<>(); + for (Map.Entry, Histogram> labelValuesToHistogram : histogramsByLabelValues.entrySet()) { + addSamples(labelValuesToHistogram.getKey(), labelValuesToHistogram.getValue(), samples); + } + return Collections.singletonList(new MetricFamilySamples(metricName, Type.SUMMARY, helpString, samples)); + } + + void addChild(final Histogram histogram, final List labelValues) { + histogramsByLabelValues.put(labelValues, histogram); + } + + void remove(final List labelValues) { + histogramsByLabelValues.remove(labelValues); + } + + private void addSamples(final List labelValues, final Histogram histogram, final List samples) { + samples.add(new MetricFamilySamples.Sample(metricName + "_count", + labelNamesWithQuantile.subList(0, labelNamesWithQuantile.size() - 1), labelValues, histogram.getCount())); + for (final Double quantile : QUANTILES) { + samples.add(new MetricFamilySamples.Sample(metricName, labelNamesWithQuantile, + addToList(labelValues, quantile.toString()), + histogram.getStatistics().getQuantile(quantile))); + } + } + } +} diff --git a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java new file mode 100644 index 0000000000..52fd24d46f --- /dev/null +++ b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.metrics.prometheus; + +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.PushGateway; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.util.AbstractID; + +import java.io.IOException; + +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.DELETE_ON_SHUTDOWN; +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST; +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.JOB_NAME; +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.PORT; +import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; + +/** + * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. + */ +@PublicEvolving +public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { + + private PushGateway pushGateway; + private String jobName; + private boolean deleteOnShutdown; + + @Override + public void open(MetricConfig config) { + super.open(config); + + String host = config.getString(HOST.key(), HOST.defaultValue()); + int port = config.getInteger(PORT.key(), PORT.defaultValue()); + String configuredJobName = config.getString(JOB_NAME.key(), JOB_NAME.defaultValue()); + boolean randomSuffix = config.getBoolean(RANDOM_JOB_NAME_SUFFIX.key(), RANDOM_JOB_NAME_SUFFIX.defaultValue()); + deleteOnShutdown = config.getBoolean(DELETE_ON_SHUTDOWN.key(), DELETE_ON_SHUTDOWN.defaultValue()); + + if (host == null || host.isEmpty() || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + if (randomSuffix) { + this.jobName = configuredJobName + new AbstractID(); + } else { + this.jobName = configuredJobName; + } + + pushGateway = new PushGateway(host + ':' + port); + log.info("Configured PrometheusPushGatewayReporter with {host:{}, port:{}, jobName: {}, randomJobNameSuffix:{}, deleteOnShutdown:{}}", host, port, jobName, randomSuffix, deleteOnShutdown); + } + + @Override + public void report() { + try { + pushGateway.push(CollectorRegistry.defaultRegistry, jobName); + log.info("push metrics to PushGateway with jobName {}.", jobName); + } catch (Exception e) { + log.warn("Failed to push metrics to PushGateway with jobName {}.", jobName, e); + } + } + + @Override + public void close() { + if (deleteOnShutdown && pushGateway != null) { + try { + pushGateway.delete(jobName); + log.info("delete metrics from PushGateway with jobName {}.", jobName); + } catch (IOException e) { + log.warn("Failed to delete metrics from PushGateway with jobName {}.", jobName, e); + } + } + super.close(); + } +} diff --git a/pom.xml b/pom.xml index 78e34c817b..1ed77d8ade 100644 --- a/pom.xml +++ b/pom.xml @@ -13,7 +13,7 @@ flinkx-core flinkx-launcher - flinkx-examples + flinkx-test flinkx-stream @@ -65,7 +65,7 @@ 2.7.3 4.5.3 ${basedir}/dev - release_1.8.5 + release_1.8.6 @@ -95,14 +95,14 @@ org.apache.hadoop hadoop-yarn-common ${hadoop.version} - + provided org.apache.hadoop hadoop-yarn-client ${hadoop.version} - + provided org.apache.hadoop @@ -114,7 +114,7 @@ org.apache.hadoop hadoop-yarn-api ${hadoop.version} - + provided org.apache.avro @@ -122,11 +122,6 @@ 1.8.2 provided - - commons-lang - commons-lang - 2.6 - org.testng From fda120056a2fc1062a8a1f5cf9e3123c7f37eef2 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 18 May 2020 20:28:37 +0800 Subject: [PATCH 053/136] =?UTF-8?q?=E4=BE=9D=E8=B5=96=E4=BC=98=E5=8C=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-test/pom.xml | 2 -- pom.xml | 1 + 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index dd01212dd4..0d36c0ef17 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -12,8 +12,6 @@ flinkx-test - 1.8.1 - 1.6 0.3.0 diff --git a/pom.xml b/pom.xml index 1ed77d8ade..8524112890 100644 --- a/pom.xml +++ b/pom.xml @@ -62,6 +62,7 @@ UTF-8 2.10 1.8.1 + 1.6 2.7.3 4.5.3 ${basedir}/dev From f39fd55437c0be5be266399a7b54c2590905cf1c Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 18 May 2020 20:32:15 +0800 Subject: [PATCH 054/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=96=87=E6=A1=A3?= =?UTF-8?q?=E6=8F=8F=E8=BF=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/offline/reader/phoenixreader.md | 2 +- docs/offline/reader/sqlserverreader.md | 2 +- docs/offline/writer/cassandrawriter.md | 2 +- docs/offline/writer/phoenixwriter.md | 2 +- docs/offline/writer/sqlserverwriter.md | 2 +- docs/realTime/reader/kafkareader.md | 2 +- docs/realTime/writer/kafkawriter.md | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/offline/reader/phoenixreader.md b/docs/offline/reader/phoenixreader.md index 5a3de8b392..14ceca3099 100644 --- a/docs/offline/reader/phoenixreader.md +++ b/docs/offline/reader/phoenixreader.md @@ -5,7 +5,7 @@ 名称:**phoenixreader**
## 二、支持的数据源版本 -phoenix4.12.0-HBase-1.3及之后 +phoenix4.12.0-HBase-1.3及以上 ## 三、参数说明 diff --git a/docs/offline/reader/sqlserverreader.md b/docs/offline/reader/sqlserverreader.md index a3f2ac056e..d7d522d9e1 100644 --- a/docs/offline/reader/sqlserverreader.md +++ b/docs/offline/reader/sqlserverreader.md @@ -5,7 +5,7 @@ 名称:**sqlserverreader** ## 二、支持的数据源版本 -**Microsoft SQL Server 2012及以后** +**Microsoft SQL Server 2012及以上** ## 三、参数说明 diff --git a/docs/offline/writer/cassandrawriter.md b/docs/offline/writer/cassandrawriter.md index 4a93d69050..2e999d0242 100644 --- a/docs/offline/writer/cassandrawriter.md +++ b/docs/offline/writer/cassandrawriter.md @@ -5,7 +5,7 @@ 名称:**cassandrawriter**
## 二、支持的数据源版本 -**Cassandra 3.0以以上**
+**Cassandra 3.0及以上**
## 三、参数说明 diff --git a/docs/offline/writer/phoenixwriter.md b/docs/offline/writer/phoenixwriter.md index 5a1583dceb..88100c2ac3 100644 --- a/docs/offline/writer/phoenixwriter.md +++ b/docs/offline/writer/phoenixwriter.md @@ -5,7 +5,7 @@ 名称:**phoenixwriter**
## 二、支持的数据源版本 -phoenix4.12.0-HBase-1.3及之后
+phoenix4.12.0-HBase-1.3及以上
## 三、参数说明 diff --git a/docs/offline/writer/sqlserverwriter.md b/docs/offline/writer/sqlserverwriter.md index cd1d8aed9c..320042de7d 100644 --- a/docs/offline/writer/sqlserverwriter.md +++ b/docs/offline/writer/sqlserverwriter.md @@ -5,7 +5,7 @@ 名称:**sqlserverwriter**
## 二、支持的数据源版本 -**Microsoft SQL Server 2012及以后**
+**Microsoft SQL Server 2012及以上**
## 三、参数说明 diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index 6e110b52e7..531d681ee8 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -9,7 +9,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.9 | kafka09reader | | kafka 0.10 | kafka10reader | | kafka 0.11 | kafka11reader | -| kafka 1.0及以后 | kafkareader | +| kafka 1.0及以上 | kafkareader |
diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md index 97355138cf..fc82240795 100644 --- a/docs/realTime/writer/kafkawriter.md +++ b/docs/realTime/writer/kafkawriter.md @@ -9,7 +9,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.9 | kafka09writer | | kafka 0.10 | kafka10writer | | kafka 0.11 | kafka11writer | -| kafka 1.0及以后 | kafkawriter | +| kafka 1.0及以上 | kafkawriter | From 99f78bacae3dbe0ea640a24bb83530eed40bb843 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 13 Jul 2020 16:21:09 +0800 Subject: [PATCH 055/136] =?UTF-8?q?commit=20release=5F1.8.7:=20=20=201?= =?UTF-8?q?=E3=80=81=E5=A2=9E=E5=8A=A0Greenplum=E5=92=8CRestApi=E6=8F=92?= =?UTF-8?q?=E4=BB=B6=20=20=202=E3=80=81=E6=94=AF=E6=8C=81-p=E5=8F=82?= =?UTF-8?q?=E6=95=B0=E5=A1=AB=E5=85=85=E8=84=9A=E6=9C=AC=E5=8D=A0=E4=BD=8D?= =?UTF-8?q?=E7=AC=A6=20=20=203=E3=80=81BUG=E4=BF=AE=E5=A4=8D?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitignore | 4 +- README.md | 6 +- docs/offline/reader/greenplumreader.md | 340 +++++ docs/offline/writer/greenplumwriter.md | 139 ++ docs/offline/writer/pulsarwriter.md | 76 - docs/realTime/reader/restapireader.md | 75 + docs/realTime/writer/restapiwriter.md | 115 ++ flinkx-binlog/flinkx-binlog-core/pom.xml | 6 - flinkx-binlog/flinkx-binlog-reader/pom.xml | 10 +- .../flinkx/binlog/reader/BinlogConfig.java | 2 +- .../flinkx-carbondata-reader/pom.xml | 8 +- .../carbondata/reader/CarbonExpressUtil.java | 2 - .../reader/CarbonFlinkInputSplit.java | 36 +- .../carbondata/reader/CarbondataReader.java | 7 +- .../flinkx-carbondata-writer/pom.xml | 8 +- .../carbondata/writer/CarbonOutputFormat.java | 4 +- .../writer/CarbondataOutputFormatBuilder.java | 2 + .../carbondata/writer/dict/DateTimeUtils.java | 100 +- .../writer/dict/ExternalCatalogUtils.java | 7 - .../recordwriter/AbstractRecordWriter.java | 15 +- .../CarbonPartitionRecordWriter.java | 3 - .../HivePartitionRecordWriter.java | 14 +- .../writer/dict/DateTimeUtilsTest.java | 29 +- .../flinkx-cassandra-core/pom.xml | 4 +- .../flinkx/cassandra/CassandraConfigKeys.java | 17 + .../flinkx/cassandra/CassandraUtil.java | 3 +- .../flinkx/cassandra/TestCassandraUtil.java | 20 + .../flinkx-cassandra-reader/pom.xml | 4 +- .../cassandra/reader/CassandraConstants.java | 17 + .../reader/CassandraInputFormat.java | 18 +- .../flinkx-cassandra-writer/pom.xml | 4 +- .../writer/CassandraOutputFormat.java | 32 +- .../writer/CassandraOutputFormatBuilder.java | 17 + .../cassandra/writer/CassandraWriter.java | 19 +- flinkx-cassandra/pom.xml | 4 +- .../clickhouse/core/ClickhouseUtil.java | 7 +- .../flinkx-clickhouse-reader/pom.xml | 10 + .../flinkx-clickhouse-writer/pom.xml | 10 + flinkx-core/pom.xml | 35 +- .../api/java/MyLocalStreamEnvironment.java | 1 - .../flinkx/authenticate/KerberosUtil.java | 12 +- .../flinkx/authenticate/SftpHandler.java | 23 +- .../classloader/ClassLoaderManager.java | 8 +- .../flinkx/classloader/PluginUtil.java | 2 - .../dtstack/flinkx/config/AbstractConfig.java | 14 +- .../dtstack/flinkx/config/RestartConfig.java | 2 +- .../dtstack/flinkx/config/RestoreConfig.java | 5 +- .../com/dtstack/flinkx/config/TestConfig.java | 2 +- .../flinkx/constants/ConstantValue.java | 11 + .../dtstack/flinkx/decoder/DecodeEnum.java | 36 +- .../com/dtstack/flinkx}/decoder/IDecode.java | 2 +- .../dtstack/flinkx}/decoder/JsonDecoder.java | 6 +- .../dtstack/flinkx}/decoder/PlainDecoder.java | 2 +- .../dtstack/flinkx/enums/EDatabaseType.java | 1 + .../inputformat/BaseRichInputFormat.java | 8 +- .../flinkx/metrics/AccumulatorCollector.java | 11 +- .../metrics/CustomPrometheusReporter.java | 22 +- .../dtstack/flinkx/options/OptionParser.java | 14 +- .../outputformat/BaseFileOutputFormat.java | 10 +- .../outputformat/BaseRichOutputFormat.java | 8 +- .../BaseRichOutputFormatBuilder.java | 19 +- .../flinkx/reader/ByteRateLimiter.java | 15 +- .../com/dtstack/flinkx/reader/MetaColumn.java | 12 +- .../com/dtstack/flinkx/util/ClassUtil.java | 2 +- .../java/com/dtstack/flinkx/util/Clock.java | 5 + .../com/dtstack/flinkx/util/DateUtil.java | 20 +- .../dtstack/flinkx/util/ExceptionUtil.java | 35 +- .../dtstack/flinkx/util/FileSystemUtil.java | 26 +- .../com/dtstack/flinkx/util/GsonUtil.java | 144 ++ .../dtstack/flinkx/util/JsonModifyUtil.java | 48 + .../dtstack/flinkx/util/ResultPrintUtil.java | 10 +- .../com/dtstack/flinkx/util/RetryUtil.java | 2 +- .../com/dtstack/flinkx/util/StringUtil.java | 16 +- .../flinkx/writer/DirtyDataManager.java | 11 +- .../dtstack/flinkx/writer/ErrorLimiter.java | 3 +- .../StreamExecutionEnvironment.java | 14 +- .../flinkx/util/ColumnTypeUtilTest.java | 4 +- .../com/dtstack/flinkx/util/DateUtilTest.java | 112 ++ .../flinkx/util/ExceptionUtilTest.java | 23 + .../com/dtstack/flinkx/util/MapUtilTest.java | 4 +- .../com/dtstack/flinkx/util/Md5UtilTest.java | 4 +- .../com/dtstack/flinkx/util/RowUtilTest.java | 4 +- .../flinkx/util/SnowflakeIdWorkerTest.java | 28 + .../dtstack/flinkx/util/StringUtilTest.java | 54 +- .../dtstack/flinkx/util/ValueUtilTest.java | 12 +- flinkx-db2/flinkx-db2-reader/pom.xml | 8 +- flinkx-db2/flinkx-db2-writer/pom.xml | 8 +- .../com/dtstack/flinkx/dm/DmDatabaseMeta.java | 2 +- flinkx-dm/flinkx-dm-reader/pom.xml | 8 +- .../flinkx/dm/format/DmInputFormat.java | 4 +- flinkx-dm/flinkx-dm-writer/pom.xml | 8 +- .../flinkx/dm/format/DmOutputFormat.java | 43 +- .../flinkx/emqx/decoder/JsonDecoder.java | 56 - flinkx-emqx/flinkx-emqx-reader/pom.xml | 10 + .../flinkx/emqx/format/EmqxInputFormat.java | 13 +- .../flinkx/emqx/reader/EmqxReader.java | 8 +- flinkx-emqx/flinkx-emqx-writer/pom.xml | 10 + .../flinkx/emqx/format/EmqxOutputFormat.java | 2 +- .../flinkx/emqx/writer/EmqxWriter.java | 7 +- flinkx-es/flinkx-es-reader/pom.xml | 14 + .../flinkx/es/reader/EsInputFormat.java | 6 +- .../es/reader/EsInputFormatBuilder.java | 5 + .../dtstack/flinkx/es/reader/EsReader.java | 7 +- .../flinkx/es/reader/test/EsReadDemo.java | 90 -- .../flinkx/es/reader/test/EsReaderTest.java | 20 - flinkx-es/flinkx-es-writer/pom.xml | 14 + .../dtstack/flinkx/es/writer/test/EsDemo.java | 146 -- .../com/dtstack/flinkx/ftp/FtpConfig.java | 2 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 15 +- .../dtstack/flinkx/ftp/FtpHandlerFactory.java | 18 +- .../com/dtstack/flinkx/ftp/IFtpHandler.java | 4 +- flinkx-ftp/flinkx-ftp-reader/pom.xml | 10 + .../flinkx/ftp/reader/FtpInputFormat.java | 16 +- .../dtstack/flinkx/ftp/reader/FtpReader.java | 5 +- .../ftp/reader/FtpSeqBufferedReader.java | 12 +- .../flinkx/ftp/reader/FtpServerDemo.java | 41 - .../flinkx/ftp/reader/SftpServerDemo.java | 42 - flinkx-ftp/flinkx-ftp-writer/pom.xml | 10 + .../flinkx/ftp/writer/FtpOutputFormat.java | 101 +- .../ftp/writer/FtpOutputFormatBuilder.java | 4 +- .../dtstack/flinkx/ftp/writer/FtpWriter.java | 2 +- flinkx-gbase/flinkx-gbase-reader/pom.xml | 8 +- .../flinkx/gbase/format/GbaseInputFormat.java | 9 +- flinkx-gbase/flinkx-gbase-writer/pom.xml | 8 +- .../flinkx-greenplum-core/pom.xml | 39 + .../greenplum/GreenplumDatabaseMeta.java | 50 + .../GreenplumDatabaseMetaInsert.java | 33 + .../flinkx-greenplum-reader/pom.xml | 112 ++ .../format/GreenplumInputFormat.java | 30 + .../greenplum/reader/GreenplumReader.java | 83 ++ .../flinkx-greenplum-writer/pom.xml | 110 ++ .../format/GreenplumOutputFormat.java | 31 + .../greenplum/writer/GreenplumWriter.java | 93 ++ flinkx-greenplum/pom.xml | 36 + .../flinkx/hbase/HbaseConfigConstants.java | 2 +- .../com/dtstack/flinkx/hbase/HbaseHelper.java | 70 +- .../flinkx/hbase/test/HbaseHelperTest.java | 75 - flinkx-hbase/flinkx-hbase-reader/pom.xml | 10 + .../flinkx/hbase/reader/HbaseInputFormat.java | 27 +- .../flinkx/hbase/reader/HbaseReader.java | 7 +- .../flinkx/hbase/reader/HbaseSplitDemo.java | 34 - flinkx-hbase/flinkx-hbase-writer/pom.xml | 10 + .../hbase/writer/HbaseOutputFormat.java | 104 +- .../writer/HbaseOutputFormatBuilder.java | 2 + .../flinkx/hbase/writer/HbaseWriter.java | 19 +- .../RowKeyFunctionTest.java | 5 +- .../dtstack/flinkx/hdfs/ECompressType.java | 1 - .../dtstack/flinkx/hdfs/HdfsConfigKeys.java | 2 + .../com/dtstack/flinkx/hdfs/HdfsUtil.java | 146 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 10 + .../hdfs/reader/BaseHdfsInputFormat.java | 47 +- .../hdfs/reader/HdfsOrcInputFormat.java | 197 +-- .../hdfs/reader/HdfsParquetInputFormat.java | 78 +- .../flinkx/hdfs/reader/HdfsReader.java | 4 +- .../hdfs/reader/HdfsTextInputFormat.java | 124 +- .../hadoop/hive/ql/io/orc/OrcInputFormat.java | 1221 +++++++++++++++++ .../hdfs/reader/HdfsOrcInputFormatTest.java | 22 + flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 10 + .../hdfs/writer/BaseHdfsOutputFormat.java | 25 +- .../hdfs/writer/HdfsOrcOutputFormat.java | 55 +- .../hdfs/writer/HdfsOutputFormatBuilder.java | 4 + .../hdfs/writer/HdfsParquetOutputFormat.java | 110 +- .../hdfs/writer/HdfsTextOutputFormat.java | 4 +- .../flinkx/hdfs/writer/HdfsWriter.java | 8 +- .../flinkx/hive/TimePartitionFormat.java | 6 +- .../hive/util/AbstractHiveMetadataParser.java | 4 +- .../dtstack/flinkx/hive/util/HiveDbUtil.java | 13 +- .../dtstack/flinkx/hive/util/HiveUtil.java | 210 +-- .../dtstack/flinkx/hive/test/DBUtilTest.java | 64 - flinkx-hive/flinkx-hive-writer/pom.xml | 10 + .../flinkx/hive/writer/HiveOutputFormat.java | 12 +- .../hive/writer/HiveOutputFormatBuilder.java | 3 + .../flinkx/hive/writer/HiveWriter.java | 70 +- flinkx-kafka/flinkx-kafka-reader/pom.xml | 10 + .../flinkx/kafka/reader/KafkaClient.java | 2 +- .../flinkx/kafka/reader/KafkaInputFormat.java | 2 + flinkx-kafka/flinkx-kafka-writer/pom.xml | 10 + flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 10 + .../flinkx/kafka09/reader/Kafka09Client.java | 2 +- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 10 + flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 10 + .../flinkx/kafka10/reader/Kafka10Client.java | 2 +- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 10 + flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 10 + .../flinkx/kafka11/reader/Kafka11Client.java | 2 +- .../kafka11/reader/Kafka11InputFormat.java | 2 + flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 10 + .../reader/KafkaBaseInputFormat.java | 9 +- .../kafkabase/reader/KafkaBaseReader.java | 6 +- .../writer/KafkaBaseOutputFormat.java | 10 +- .../kafkabase/writer/KafkaBaseWriter.java | 5 +- flinkx-kudu/flinkx-kudu-core/pom.xml | 6 - .../dtstack/flinkx/kudu/core/KuduUtil.java | 13 +- flinkx-kudu/flinkx-kudu-reader/pom.xml | 8 +- .../flinkx/kudu/reader/KuduInputFormat.java | 6 +- .../kudu/reader/KuduInputFormatBuilder.java | 5 + .../flinkx/kudu/reader/KuduReader.java | 14 +- flinkx-kudu/flinkx-kudu-writer/pom.xml | 8 +- .../flinkx/kudu/writer/KuduOutputFormat.java | 7 +- .../flinkx/kudu/writer/KuduWriter.java | 11 +- .../flinkx/launcher/ClassLoaderType.java | 17 + .../flinkx/launcher/ClusterClientFactory.java | 10 +- .../com/dtstack/flinkx/launcher/Launcher.java | 25 +- .../flinkx/launcher/YarnConfLoader.java | 3 +- .../perjob/FlinkPerJobResourceUtil.java | 4 +- .../launcher/perjob/PerJobSubmitter.java | 4 +- .../test/java/bigdata/FlinkConfigTest.java | 19 - .../flinkx/mongodb/MongodbClientUtil.java | 8 +- .../dtstack/flinkx/mongodb/MongodbUtil.java | 6 +- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 8 +- .../mongodb/reader/MongodbInputFormat.java | 7 +- .../reader/MongodbInputFormatBuilder.java | 5 + flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 8 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 8 +- .../flinkx/mysqld/reader/MysqldReader.java | 5 +- flinkx-mysql/flinkx-mysql-reader/pom.xml | 10 +- .../flinkx/mysql/format/MysqlInputFormat.java | 14 +- .../src/test/java/bigdata/TestMysqlPk.java | 22 - flinkx-mysql/flinkx-mysql-writer/pom.xml | 8 +- .../com/dtstack/flinkx/odps/OdpsUtil.java | 2 +- flinkx-odps/flinkx-odps-reader/pom.xml | 10 + .../odps/reader/OdpsInputFormatBuilder.java | 4 +- .../flinkx/odps/reader/OdpsReader.java | 4 +- flinkx-odps/flinkx-odps-writer/pom.xml | 10 + .../flinkx/odps/writer/OdpsOutputFormat.java | 2 +- .../odps/writer/OdpsOutputFormatBuilder.java | 2 + .../flinkx/oracle/OracleDatabaseMeta.java | 2 +- flinkx-oracle/flinkx-oracle-reader/pom.xml | 8 +- .../oracle/reader/test/OracleLocalTest.java | 22 - flinkx-oracle/flinkx-oracle-writer/pom.xml | 8 +- .../oracle/format/OracleOutputFormat.java | 41 +- flinkx-pgwal/flinkx-pgwal-reader/pom.xml | 10 + flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 10 + .../phoenix/format/PhoenixInputFormat.java | 3 +- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 10 + flinkx-polardb/flinkx-polardb-dreader/pom.xml | 8 + .../polardbd/reader/PolardbdReader.java | 1 - flinkx-polardb/flinkx-polardb-reader/pom.xml | 8 + flinkx-polardb/flinkx-polardb-writer/pom.xml | 8 + .../postgresql/PostgresqlDatabaseMeta.java | 3 - .../postgresql/PostgresqlTypeConverter.java | 29 +- .../flinkx-postgresql-reader/pom.xml | 8 +- .../format/PostgresqlInputFormat.java | 9 +- .../flinkx-postgresql-writer/pom.xml | 8 +- .../dtstack/flinkx/rdb/BaseDatabaseMeta.java | 8 +- .../com/dtstack/flinkx/rdb/util/DbUtil.java | 244 ++-- .../DistributedJdbcDataReader.java | 4 +- .../DistributedJdbcInputFormat.java | 20 +- .../DistributedJdbcInputFormatBuilder.java | 3 +- .../DistributedJdbcInputSplit.java | 6 +- .../JdbcInputFormat.java | 34 +- .../JdbcInputFormatBuilder.java | 5 + .../JdbcDataWriter.java | 10 +- .../rdb/outputformat/JdbcOutputFormat.java | 28 +- .../com/dtstack/flinkx/redis/JedisUtil.java | 9 +- flinkx-redis/flinkx-redis-writer/pom.xml | 8 +- .../redis/writer/RedisOutputFormat.java | 16 +- .../writer/RedisOutputFormatBuilder.java | 2 + .../flinkx/redis/writer/RedisWriter.java | 13 +- flinkx-restapi/flinkx-restapi-core/pom.xml | 20 + .../flinkx/restapi/common/HttpMethod.java | 44 + .../flinkx/restapi/common/HttpUtil.java | 115 ++ .../common/MyHttpRequestRetryHandler.java | 96 ++ .../MyServiceUnavailableRetryStrategy.java | 73 + .../flinkx/restapi/common/RestapiKeys.java | 32 + flinkx-restapi/flinkx-restapi-reader/pom.xml | 99 ++ .../inputformat/RestapiInputFormat.java | 109 ++ .../RestapiInputFormatBuilder.java | 46 + .../flinkx/restapi/reader/RestapiReader.java | 71 + flinkx-restapi/flinkx-restapi-writer/pom.xml | 99 ++ .../outputformat/RestapiOutputFormat.java | 150 ++ .../RestapiOutputFormatBuilder.java | 71 + .../flinkx/restapi/writer/RestapiWriter.java | 103 ++ flinkx-restapi/pom.xml | 28 + flinkx-saphana/flinkx-saphana-core/pom.xml | 4 +- flinkx-saphana/flinkx-saphana-reader/pom.xml | 4 +- flinkx-saphana/flinkx-saphana-writer/pom.xml | 4 +- flinkx-saphana/pom.xml | 4 +- .../flinkx-sqlserver-reader/pom.xml | 8 +- .../flinkx-sqlserver-writer/pom.xml | 8 +- flinkx-stream/flinkx-stream-reader/pom.xml | 10 + flinkx-stream/flinkx-stream-writer/pom.xml | 10 + .../stream/writer/StreamOutputFormat.java | 20 +- .../flinkx/stream/writer/StreamWriter.java | 2 +- flinkx-test/pom.xml | 195 ++- .../com/dtstack/flinkx/test/LocalTest.java | 291 ++-- .../flinkx/test/PluginNameConstants.java | 5 + pom.xml | 77 +- 288 files changed, 6820 insertions(+), 2303 deletions(-) create mode 100644 docs/offline/reader/greenplumreader.md create mode 100644 docs/offline/writer/greenplumwriter.md delete mode 100644 docs/offline/writer/pulsarwriter.md create mode 100644 docs/realTime/reader/restapireader.md create mode 100644 docs/realTime/writer/restapiwriter.md rename flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java => flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java (73%) rename flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java => flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java (69%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/IDecode.java (96%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/JsonDecoder.java (97%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/PlainDecoder.java (96%) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java delete mode 100644 flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java delete mode 100644 flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java delete mode 100644 flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java delete mode 100644 flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java delete mode 100644 flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java delete mode 100644 flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java create mode 100644 flinkx-greenplum/flinkx-greenplum-core/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java create mode 100644 flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java create mode 100644 flinkx-greenplum/pom.xml delete mode 100644 flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java delete mode 100644 flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java create mode 100644 flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java create mode 100644 flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java delete mode 100644 flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java delete mode 100644 flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java delete mode 100644 flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java delete mode 100644 flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java create mode 100644 flinkx-restapi/flinkx-restapi-core/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java create mode 100644 flinkx-restapi/pom.xml diff --git a/.gitignore b/.gitignore index d8cb487ce5..1e7d443821 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,7 @@ # Created by .ignore support plugin (hsz.mobi) .idea/ plugins/ +syncplugins/ *.iml target/ lib/ @@ -8,4 +9,5 @@ jobs/ nohup.out flinkconf/ hadoopconf/ -/default_task_id_output \ No newline at end of file +/default_task_id_output +/syncplugins \ No newline at end of file diff --git a/README.md b/README.md index c2a3a77d5e..38e4d79917 100644 --- a/README.md +++ b/README.md @@ -51,6 +51,7 @@ The following databases are currently supported: | | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | @@ -65,11 +66,12 @@ The following databases are currently supported: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon| | -| | SqlServer CDC | Coming Soon | | +| | Oracle Logminer| Coming Soon | | +| | SqlServer CDC | Coming Soon | | # Quick Start diff --git a/docs/offline/reader/greenplumreader.md b/docs/offline/reader/greenplumreader.md new file mode 100644 index 0000000000..a8d89386e7 --- /dev/null +++ b/docs/offline/reader/greenplumreader.md @@ -0,0 +1,340 @@ +# Greenplum Reader + + +## 一、插件名称 +名称:**greenplumreader**
+ +## 二、支持的数据源版本 +**Greenplum 5及以上**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[greenplum官方文档](https://gpdb.docs.pivotal.io/590/datadirect/datadirect_jdbc.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +** + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 3、 指定customsql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "select id from performance", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "increColumn": "id", + "startLocation": "20", + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "polling": true, + "pollingInterval": 3000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "where": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +
说明:表名不应该有'.'号,否则会报如下错误:
`java.sql.SQLException: SQLException: [Pivotal][Greenplum JDBC Driver][Greenplum]cross-database references are not implemented: "public.public.test". ` diff --git a/docs/offline/writer/greenplumwriter.md b/docs/offline/writer/greenplumwriter.md new file mode 100644 index 0000000000..57a069ce88 --- /dev/null +++ b/docs/offline/writer/greenplumwriter.md @@ -0,0 +1,139 @@ +# Greenplum Writer + + +## 一、插件名称 +名称:**greenplumwriter**
+ +## 二、支持的数据源版本 +**Greenplum 5及以上**
** + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:仅支持`insert`操作,可以搭配insertSqlMode使用 + - 必选:是 + - 默认值:无, + + + +- **insertSqlMode** + - 描述:控制写入数据到目标表采用  `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 + - 注意: + - 为了避免`insert`过慢带来的问题,此参数被固定为`copy` + - 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 + - 必选:否 + - 默认值:无 + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + +** + +## 四、配置示例 + +#### 1、insert with copy mode +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "int", + "value": 1 + } + ], + "sliceRecordCount": ["100"] + }, + "name" : "streamreader" + }, + "writer": { + "name": "greenplumwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb", + "table": ["tbl_pay_log_copy"] + }], + "username": "gpadmin", + "password": "gpadmin", + "column": [ + { + "name": "id", + "type": "int" + }], + "writeMode": "insert", + "insertSqlMode": "copy", + "batchSize": 100, + "preSql": ["TRUNCATE tbl_pay_log_copy"], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` diff --git a/docs/offline/writer/pulsarwriter.md b/docs/offline/writer/pulsarwriter.md deleted file mode 100644 index 413e1e0f30..0000000000 --- a/docs/offline/writer/pulsarwriter.md +++ /dev/null @@ -1,76 +0,0 @@ -# Pulsar写入插件(**writer) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "parameter": { - "producerSettings" : { - "producerName":"test-producer" - }, - "topic" : "pulsar_test", - "pulsarServiceUrl" : "pulsar://127.0.0.1:6650" - - }, - "name": "pulsarwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,pulsarwriter。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:topic。 - - * 必选:是 - - * 默认值:无 - - -* **pulsarServiceUrl** - - * 描述:pulsar地址列表 - - * 必选:是 - - * 默认值:无 - - - -* **producerSettings** - - * 描述:pulsar生产者配置 - - * 必选:是 - - * 默认值:无 - -参考: https://pulsar.apache.org/docs/en/client-libraries-java/#configure-producer \ No newline at end of file diff --git a/docs/realTime/reader/restapireader.md b/docs/realTime/reader/restapireader.md new file mode 100644 index 0000000000..f354c823db --- /dev/null +++ b/docs/realTime/reader/restapireader.md @@ -0,0 +1,75 @@ +# Restapi Reader + + +## 一、插件名称 +名称:restapireader
+ + +## 二、参数说明 + +- **url** + - 描述:连接的url + - 必选:是 + - 默认值:无 + + + +- **method** + - 描述:request的类型,`post`、`get` + - 必选:是 + - 默认值:无 + + + +- header + - 描述:需要添加的报头信息 + - 必选:否 + - 默认值:无 + + + + +## 三、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", + "body": "", + "method": "get", + "params": "" + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "errorLimit": {}, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "trace", + "path": "", + "pattern": "" + } + } + } +} +``` + + diff --git a/docs/realTime/writer/restapiwriter.md b/docs/realTime/writer/restapiwriter.md new file mode 100644 index 0000000000..6a515b9a1e --- /dev/null +++ b/docs/realTime/writer/restapiwriter.md @@ -0,0 +1,115 @@ +# Restapi Writer + + +## 一、插件名称 +**名称:restapiwriter**
+ +## 二、参数说明 + +- **url** + - 描述:连接的url + - 必选:是 + - 默认值:无 + + + +- **method** + - 描述:request的类型,`post`、`get` + - 必选:是 + - 默认值:无 + + + +- **header** + - 描述:需要添加的报头信息 + - 必选:否 + - 默认值:无 + + + +- **body** + - 描述:发送的数据中包括params + - 必选:否 + - 默认值:无 + + + +- **params** + - 描述:发送的数据中包括params + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:如果column不为空,那么将数据和字段名一一对应。如果column为空,则返回每个数据的第一个字段。 + - 必选:否 + - 默认值:无 + + + + +## 三、使用示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "data", + "type": "string" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", + "header": [], + "body": [], + "method": "post", + "params": {}, + "column": ["id","data"] + }, + "name": "restapiwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` + + diff --git a/flinkx-binlog/flinkx-binlog-core/pom.xml b/flinkx-binlog/flinkx-binlog-core/pom.xml index d16755f37d..ba3a2485dd 100644 --- a/flinkx-binlog/flinkx-binlog-core/pom.xml +++ b/flinkx-binlog/flinkx-binlog-core/pom.xml @@ -15,12 +15,6 @@ - - com.google.guava - guava - 19.0 - - com.alibaba.otter canal.parse diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index ad9b9893df..302234b486 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -69,6 +69,10 @@ slf4j-api org.slf4j + + guava + com.google.guava + @@ -100,7 +104,11 @@ com.google.common - shade.binlog.com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java index 69d6d51add..0fcbdade5b 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java @@ -32,7 +32,7 @@ public class BinlogConfig implements Serializable { public long period = 1000L; - public int bufferSize = 1024; + public int bufferSize = 256; public boolean pavingData = true; diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index 111fc621c3..7b38ba5c8b 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -61,8 +61,12 @@ shade.carbondatareader.io.netty - com.google - shade.carbondatareader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java index f655d14934..77dd6dedba 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java @@ -19,7 +19,6 @@ import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.scan.expression.LiteralExpression; @@ -30,7 +29,6 @@ import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression; import org.apache.commons.lang3.StringUtils; -import java.util.ArrayList; import java.util.List; diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java index 5f440b5f33..91547fdfed 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; /** @@ -46,15 +45,24 @@ public class CarbonFlinkInputSplit implements InputSplit { public CarbonFlinkInputSplit(List carbonInputSplits, int splitNumber) throws IOException { this.splitNumber = splitNumber; rawSplits = new ArrayList<>(); - rawSplits.addAll(carbonInputSplits.stream().map(this::carbonSplitToRawSplit).collect(Collectors.toList())); + List list = new ArrayList<>(); + for (CarbonInputSplit carbonInputSplit : carbonInputSplits) { + byte[] bytes = carbonSplitToRawSplit(carbonInputSplit); + list.add(bytes); + } + rawSplits.addAll(list); } public List getCarbonInputSplits() throws IOException { - List carbonInputSplits = rawSplits.stream().map(this::rawSplitToCarbonSplit).collect(Collectors.toList()); + List carbonInputSplits = new ArrayList<>(); + for (byte[] rawSplit : rawSplits) { + CarbonInputSplit carbonInputSplit = rawSplitToCarbonSplit(rawSplit); + carbonInputSplits.add(carbonInputSplit); + } return carbonInputSplits; } - private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) { + private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) throws IOException{ ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos); try { @@ -62,17 +70,14 @@ private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) { } catch (IOException e) { throw new RuntimeException(e); } finally { - try { - baos.close(); - dos.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } + baos.close(); + dos.close(); } + return baos.toByteArray(); } - private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) { + private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) throws IOException{ ByteArrayInputStream bais = new ByteArrayInputStream(rawSplit); DataInputStream dis = new DataInputStream(bais); CarbonInputSplit carbonInputSplit = new CarbonInputSplit(); @@ -81,13 +86,10 @@ private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) { } catch (IOException e) { throw new RuntimeException(e); } finally { - try { - bais.close(); - dis.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } + bais.close(); + dis.close(); } + return carbonInputSplit; } diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java index 6b009c38f8..4942980eeb 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java @@ -25,6 +25,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,6 +40,8 @@ */ public class CarbondataReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(CarbondataReader.class); + protected String table; protected String database; @@ -78,7 +83,7 @@ public CarbondataReader(DataTransferConfig config, StreamExecutionEnvironment en columnValue.add((String) sm.get("value")); columnName.add((String) sm.get("name")); } - System.out.println("init column finished"); + LOG.info("init column finished"); } else if (!ConstantValue.STAR_SYMBOL.equals(columns.get(0)) || columns.size() != 1) { throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index 9b414eafd3..1f45de883a 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -63,8 +63,12 @@ shade.carbondatawriter.io.netty - com.google - shade.carbondatawriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java index cb78b18f62..0dece097c9 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java @@ -24,7 +24,6 @@ import com.dtstack.flinkx.carbondata.writer.recordwriter.RecordWriterFactory; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; -import com.dtstack.flinkx.util.DateUtil; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; @@ -215,8 +214,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { @Override protected void writeMultipleRecordsInternal() throws Exception { - // CAN NOT HAPPEN - throw new IllegalArgumentException("It can not happen."); + notSupportBatchWrite("CarbondataWriter"); } @Override diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java index b621038fd4..f2ef5d821c 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java @@ -89,5 +89,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("CarbondataWriter"); } } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java index 743acbcb23..6999225271 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java @@ -58,6 +58,24 @@ public class DateTimeUtils { public static final int TO_YEAR_ZERO = TO_2001 + 7304850; + public static final int TIMESTAMP_STRING_LENGTH = 19; + + public static final String TIMESTAMP_ZERO_SUFFIX = ".0"; + + public static final int DAY_IN_YEAR_29 = 29; + public static final int DAY_IN_YEAR_31 = 31; + public static final int DAY_IN_YEAR_59 = 59; + public static final int DAY_IN_YEAR_60 = 60; + public static final int DAY_IN_YEAR_90 = 90; + public static final int DAY_IN_YEAR_120 = 120; + public static final int DAY_IN_YEAR_151 = 151; + public static final int DAY_IN_YEAR_181 = 181; + public static final int DAY_IN_YEAR_212 = 212; + public static final int DAY_IN_YEAR_243 = 243; + public static final int DAY_IN_YEAR_273 = 273; + public static final int DAY_IN_YEAR_304 = 304; + public static final int DAY_IN_YEAR_334 = 334; + public static final ThreadLocal THREAD_LOCAL_LOCAL_TIMEZONE = new ThreadLocal() { @Override public TimeZone initialValue() { @@ -91,8 +109,9 @@ public static String timestampToString(long us) { String timestampString = ts.toString(); String formatted = THREAD_LOCAL_TIMESTAMP_FORMAT.get().format(ts); - if(timestampString.length() > 19 && !".0".equals(timestampString.substring(19))) { - formatted += timestampString.substring(19); + if(timestampString.length() > TIMESTAMP_STRING_LENGTH + && !TIMESTAMP_ZERO_SUFFIX.equals(timestampString.substring(TIMESTAMP_STRING_LENGTH))) { + formatted += timestampString.substring(TIMESTAMP_STRING_LENGTH); } return formatted; } @@ -231,33 +250,33 @@ public static int getMonth(int date) { int year = tuple2.getField(0); int dayInYear = tuple2.getField(1); if (isLeapYear(year)) { - if (dayInYear == 60) { + if (dayInYear == DAY_IN_YEAR_60) { return 2; - } else if (dayInYear > 60) { + } else if (dayInYear > DAY_IN_YEAR_60) { dayInYear = dayInYear - 1; } } - if (dayInYear <= 31) { + if (dayInYear <= DAY_IN_YEAR_31) { return 1; - } else if (dayInYear <= 59) { + } else if (dayInYear <= DAY_IN_YEAR_59) { return 2; - } else if (dayInYear <= 90) { + } else if (dayInYear <= DAY_IN_YEAR_90) { return 3; - } else if (dayInYear <= 120) { + } else if (dayInYear <= DAY_IN_YEAR_120) { return 4; - } else if (dayInYear <= 151) { + } else if (dayInYear <= DAY_IN_YEAR_151) { return 5; - } else if (dayInYear <= 181) { + } else if (dayInYear <= DAY_IN_YEAR_181) { return 6; - } else if (dayInYear <= 212) { + } else if (dayInYear <= DAY_IN_YEAR_212) { return 7; - } else if (dayInYear <= 243) { + } else if (dayInYear <= DAY_IN_YEAR_243) { return 8; - } else if (dayInYear <= 273) { + } else if (dayInYear <= DAY_IN_YEAR_273) { return 9; - } else if (dayInYear <= 304) { + } else if (dayInYear <= DAY_IN_YEAR_304) { return 10; - } else if (dayInYear <= 334) { + } else if (dayInYear <= DAY_IN_YEAR_334) { return 11; } else { return 12; @@ -274,39 +293,38 @@ public static int getDayOfMonth(int date) { int year = tuple2.getField(0); int dayInYear = tuple2.getField(1); if (isLeapYear(year)) { - if (dayInYear == 60) { - return 29; - } else if (dayInYear > 60) { + if (dayInYear == DAY_IN_YEAR_60) { + return DAY_IN_YEAR_29; + } else if (dayInYear > DAY_IN_YEAR_60) { dayInYear = dayInYear - 1; } } - if (dayInYear <= 31) { + if (dayInYear <= DAY_IN_YEAR_31) { return dayInYear; - } else if (dayInYear <= 59) { - return dayInYear - 31; - } else if (dayInYear <= 90) { - return dayInYear - 59; - } else if (dayInYear <= 120) { - return dayInYear - 90; - } else if (dayInYear <= 151) { - return dayInYear - 120; - } else if (dayInYear <= 181) { - return dayInYear - 151; - } else if (dayInYear <= 212) { - return dayInYear - 181; - } else if (dayInYear <= 243) { - return dayInYear - 212; - } else if (dayInYear <= 273) { - return dayInYear - 243; - } else if (dayInYear <= 304) { - return dayInYear - 273; - } else if (dayInYear <= 334) { - return dayInYear - 304; + } else if (dayInYear <= DAY_IN_YEAR_59) { + return dayInYear - DAY_IN_YEAR_31; + } else if (dayInYear <= DAY_IN_YEAR_90) { + return dayInYear - DAY_IN_YEAR_59; + } else if (dayInYear <= DAY_IN_YEAR_120) { + return dayInYear - DAY_IN_YEAR_90; + } else if (dayInYear <= DAY_IN_YEAR_151) { + return dayInYear - DAY_IN_YEAR_120; + } else if (dayInYear <= DAY_IN_YEAR_181) { + return dayInYear - DAY_IN_YEAR_151; + } else if (dayInYear <= DAY_IN_YEAR_212) { + return dayInYear - DAY_IN_YEAR_181; + } else if (dayInYear <= DAY_IN_YEAR_243) { + return dayInYear - DAY_IN_YEAR_212; + } else if (dayInYear <= DAY_IN_YEAR_273) { + return dayInYear - DAY_IN_YEAR_243; + } else if (dayInYear <= DAY_IN_YEAR_304) { + return dayInYear - DAY_IN_YEAR_273; + } else if (dayInYear <= DAY_IN_YEAR_334) { + return dayInYear - DAY_IN_YEAR_304; } else { - return dayInYear - 334; + return dayInYear - DAY_IN_YEAR_334; } - } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java index 1b83ff80fe..dbff692bbb 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java @@ -68,11 +68,4 @@ public static String escapePathName(String path) { private static boolean needsEscaping(char c) { return c >= 0 && c < charToEscape.size() && charToEscape.get(c); } - - public static void main(String[] args) { - String s = "2018-01-01 08:08:08"; - String d = escapePathName(s); - System.out.println(d); - } - } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java index 188601b450..63e01e580c 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java @@ -39,11 +39,22 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import java.io.IOException; -import java.util.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Random; +import java.util.UUID; +import java.util.Map; +import java.util.Collections; +import java.util.HashMap; /** * Abstract record writer wrapper diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java index 3a3236ca76..b2a6b20d30 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java @@ -21,10 +21,7 @@ import com.dtstack.flinkx.carbondata.writer.dict.CarbonTypeConverter; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.StringUtil; import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.schema.PartitionInfo; import org.apache.carbondata.core.metadata.schema.partition.PartitionType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java index 38dc8e42e8..a74982de1b 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java @@ -34,11 +34,21 @@ import org.apache.carbondata.processing.util.CarbonLoaderUtil; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import java.io.IOException; -import java.util.*; +import java.util.List; +import java.util.Map; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Random; +import java.util.UUID; import java.util.stream.Collectors; diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java b/flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java similarity index 73% rename from flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java rename to flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java index ce63e6fd15..11a8f7136e 100644 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java @@ -6,32 +6,29 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.emqx.decoder; -import java.util.Map; + +package com.dtstack.flinkx.carbondata.writer.dict; + +import org.junit.Test; /** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou + * @author jiangbo + * @date 2020/3/27 */ -public interface IDecode { - - /** - * 消息解码 - * @param message - * @return - */ - Map decode(String message); +public class DateTimeUtilsTest { + @Test + public void testGetMonth() { + DateTimeUtils.getMonth(10); + } } diff --git a/flinkx-cassandra/flinkx-cassandra-core/pom.xml b/flinkx-cassandra/flinkx-cassandra-core/pom.xml index 6dc4304d67..86b7f12abf 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-core/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java index 072a2b0096..a5a67df83d 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra; /** diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java index fa89243336..bc0e631faa 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java @@ -32,7 +32,6 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.net.InetAddress; -import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.sql.Time; import java.util.Date; @@ -222,7 +221,7 @@ private static Optional objectToBytes(T obj){ * @param pos 位置 * @param sqlType cql类型 * @param value 值 - * @throws RuntimeException 对于不支持的数据类型,抛出异常 + * @throws Exception 对于不支持的数据类型,抛出异常 */ public static void bindColumn(BoundStatement ps, int pos, DataType sqlType, Object value) throws Exception { if (value != null) { diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java index 179e420193..aea0c06c90 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java @@ -1,3 +1,23 @@ +<<<<<<< HEAD +======= +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +>>>>>>> 1.8_release_3.10.x package com.dtstack.flinkx.cassandra; import com.datastax.driver.core.*; diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index feca24f0f9..804af7c5d6 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java index cd8da1f553..df6dd94a03 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.reader; /** diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java index f5fdbe4ae2..52f1311861 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java @@ -20,7 +20,6 @@ import com.datastax.driver.core.*; import com.dtstack.flinkx.cassandra.CassandraUtil; -import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.google.common.base.Preconditions; @@ -97,7 +96,6 @@ public Row nextRecordInternal(Row row) { Object value = CassandraUtil.getData(cqlRow, definitions.get(i).getType(), definitions.get(i).getName()); row.setField(i, value); } - LOG.info(row.toString()); return row; } @@ -108,7 +106,7 @@ protected void closeInternal() { } @Override - public InputSplit[] createInputSplitsInternal(int minNumSplits) { + protected InputSplit[] createInputSplitsInternal(int minNumSplits) { ArrayList splits = new ArrayList<>(); try { @@ -130,12 +128,12 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) { private InputSplit[] splitJob(int minNumSplits, ArrayList splits) { if(minNumSplits <= 1) { splits.add(new CassandraInputSplit()); - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } if(whereString != null && whereString.toLowerCase().contains(CassandraConstants.TOKEN)) { splits.add(new CassandraInputSplit()); - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } Session session = CassandraUtil.getSession(cassandraConfig, ""); String partitioner = session.getCluster().getMetadata().getPartitioner(); @@ -152,8 +150,7 @@ private InputSplit[] splitJob(int minNumSplits, ArrayList s } splits.add(new CassandraInputSplit(l.toString(), r.toString())); } - } - else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { + }else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { BigDecimal minToken = BigDecimal.valueOf(Long.MIN_VALUE); BigDecimal maxToken = BigDecimal.valueOf(Long.MAX_VALUE); BigDecimal step = maxToken.subtract(minToken) @@ -166,11 +163,10 @@ else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { } splits.add(new CassandraInputSplit(String.valueOf(l), String.valueOf(r))); } - } - else { + }else { splits.add(new CassandraInputSplit()); } - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } /** @@ -181,7 +177,7 @@ else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { private String getQueryString(CassandraInputSplit inputSplit) { StringBuilder columns = new StringBuilder(); if (columnMeta == null) { - columns.append(ConstantValue.STAR_SYMBOL); + columns.append("*"); } else { for(MetaColumn column : columnMeta) { if(columns.length() > 0 ) { diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index 565ef3ca4f..73593ae1c9 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java index f16eca03ab..a280dbf059 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.datastax.driver.core.*; @@ -102,7 +119,6 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { throw new WriteRecordException("类型转换失败", e.getCause(), i, row); } } - LOG.info("insertSql: {}" + boundStatement); session.execute(boundStatement); } @@ -110,12 +126,14 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { protected void writeMultipleRecordsInternal() throws Exception { if (batchSize > 1) { BoundStatement boundStatement = pstmt.bind(); - for (Row row : rows) { - for (int i = 0; i < columnMeta.size(); i++) { - Object value = row.getField(i); - CassandraUtil.bindColumn(boundStatement, i, columnTypes.get(i), value); + for (int rowIndex = 0; rowIndex < rows.size(); rowIndex++) { + for (int columnIndex = 0; columnIndex < columnMeta.size(); columnIndex++) { + Object value = rows.get(rowIndex).getField(columnIndex); + CassandraUtil.bindColumn(boundStatement, columnIndex, columnTypes.get(columnIndex), value); + } + if ((rowIndex % 1000) == 0) { + LOG.info("insertSql: {}", boundStatement); } - LOG.info("insertSql: {}" + boundStatement); if(asyncWrite) { unConfirmedWrite.add(session.executeAsync(boundStatement)); if (unConfirmedWrite.size() >= batchSize) { @@ -152,7 +170,7 @@ protected void writeMultipleRecordsInternal() throws Exception { } @Override - public void closeInternal() throws IOException { + public void closeInternal() { CassandraUtil.close(session); } } diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java index 4446f1633f..56a170811e 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java index 2e2f0bbc9c..1e89082846 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.dtstack.flinkx.config.DataTransferConfig; @@ -13,8 +30,6 @@ import java.util.Map; import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; -import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_CONSITANCY_LEVEL; -import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_MAX_PENDING_CONNECTION; /** * diff --git a/flinkx-cassandra/pom.xml b/flinkx-cassandra/pom.xml index d788a4bea1..917b1dc9f8 100644 --- a/flinkx-cassandra/pom.xml +++ b/flinkx-cassandra/pom.xml @@ -1,6 +1,6 @@ - flinkx-all diff --git a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java index b33160d10d..c6103d9dfa 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java +++ b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java @@ -23,6 +23,7 @@ import java.sql.Connection; import java.sql.SQLException; +import java.sql.Statement; import java.util.Properties; /** @@ -43,8 +44,10 @@ public static Connection getConnection(String url, String username, String passw for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { try { conn = new BalancedClickhouseDataSource(url, properties).getConnection(); - conn.createStatement().execute("select 111"); - failed = false; + try (Statement statement = conn.createStatement()) { + statement.execute("select 111"); + failed = false; + } } catch (Exception e) { if (conn != null) { conn.close(); diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 01d798e35d..4c5ef8cf4c 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 94e72635a6..8c3d65ad12 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 223aae3824..a0bd729335 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -110,33 +110,6 @@ ${flink.version} - - junit - junit - 4.11 - test - - - - org.powermock - powermock-api-mockito - 1.7.0 - test - - - - org.powermock - powermock-module-junit4 - 1.7.0 - test - - - - org.hamcrest - hamcrest-all - 1.3 - test - org.apache.httpcomponents httpcore @@ -200,6 +173,10 @@ org.slf4j:* org.apache.httpcomponents:* io.prometheus:* + org.apache.avro:* + org.codehaus.jackson:* + com.fasterxml.jackson.core:* + commons-*:* @@ -207,6 +184,10 @@ com.google.common shade.core.com.google.common + + com.google.thirdparty + shade.core.com.google.thirdparty + org.apache.http shade.core.org.apache.http diff --git a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java index 56fb01de1c..c3dcf4ca9e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java +++ b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java index c89f0b2274..8fde360336 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java @@ -46,6 +46,7 @@ public class KerberosUtil { private static final String SP = "/"; private static final String KEY_SFTP_CONF = "sftpConf"; + private static final String KEY_PRINCIPAL = "principal"; private static final String KEY_REMOTE_DIR = "remoteDir"; private static final String KEY_USE_LOCAL_FILE = "useLocalFile"; public static final String KEY_PRINCIPAL_FILE = "principalFile"; @@ -89,6 +90,15 @@ public static UserGroupInformation loginAndReturnUgi(Configuration conf, String return UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); } + public static String getPrincipal(Map configMap, String keytabPath) { + String principal = MapUtils.getString(configMap, KEY_PRINCIPAL); + if (StringUtils.isEmpty(principal)) { + principal = findPrincipalFromKeytab(keytabPath); + } + + return principal; + } + private static void reloadKrb5Conf(Configuration conf){ String krb5File = conf.get(KEY_JAVA_SECURITY_KRB5_CONF); LOG.info("set krb5 file:{}", krb5File); @@ -188,7 +198,7 @@ private static String loadFromSftp(Map config, String fileName){ } } - public static String findPrincipalFromKeytab(String keytabFile) { + private static String findPrincipalFromKeytab(String keytabFile) { KeyTab keyTab = KeyTab.getInstance(keytabFile); for (KeyTabEntry entry : keyTab.getEntries()) { String principal = entry.getService().getName(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java index 90c0e127fb..dc1a70a1b4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java @@ -20,13 +20,18 @@ package com.dtstack.flinkx.authenticate; import com.dtstack.flinkx.util.RetryUtil; -import com.jcraft.jsch.*; +import com.jcraft.jsch.ChannelSftp; +import com.jcraft.jsch.JSch; +import com.jcraft.jsch.Session; +import com.jcraft.jsch.SftpException; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.File; +import java.io.FileOutputStream; +import java.io.OutputStream; import java.util.Map; import java.util.Properties; import java.util.concurrent.Callable; @@ -131,21 +136,11 @@ private void downloadFile(String ftpPath, String localPath){ throw new RuntimeException("File not exist on sftp:" + ftpPath); } - OutputStream os = null; - try { - os = new FileOutputStream(new File(localPath)); + try (OutputStream os = new FileOutputStream(new File(localPath))){ channelSftp.get(ftpPath, os); + os.flush(); } catch (Exception e){ throw new RuntimeException("download file from sftp error", e); - } finally { - if(os != null){ - try { - os.flush(); - os.close(); - } catch (IOException e) { - LOG.warn("", e); - } - } } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java index 9cb8f0220a..a6276ff42a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java @@ -25,7 +25,13 @@ import java.net.URL; import java.net.URLClassLoader; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Arrays; import java.util.concurrent.ConcurrentHashMap; /** diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java index fd51f7e65c..efee645720 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java @@ -24,9 +24,7 @@ import java.io.File; import java.net.MalformedURLException; import java.net.URL; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; import java.util.Set; /** diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java index df46175089..087c5302bd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java @@ -76,9 +76,9 @@ public Object getVal(String key) { Object obj = internalMap.get(key); if (obj instanceof LinkedTreeMap) { LinkedTreeMap treeMap = (LinkedTreeMap) obj; - Map newMap = new HashMap<>(treeMap.size()); - newMap.putAll(treeMap); - return newMap; + Map map = new HashMap<>(Math.max((int) (treeMap.size()/.75f) + 1, 16)); + map.putAll(treeMap); + return map; } return obj; } @@ -109,10 +109,10 @@ public int getIntVal(String key, int defaultValue) { return defaultValue; } if(ret instanceof Integer) { - return ((Integer)ret).intValue(); + return (Integer) ret; } if(ret instanceof String) { - return Integer.valueOf((String)ret).intValue(); + return Integer.parseInt((String) ret); } if(ret instanceof Long) { return ((Long)ret).intValue(); @@ -144,7 +144,7 @@ public long getLongVal(String key, long defaultValue) { return ((Integer)ret).longValue(); } if(ret instanceof String) { - return Long.valueOf((String)ret); + return Long.parseLong((String)ret); } if(ret instanceof Float) { return ((Float)ret).longValue(); @@ -176,7 +176,7 @@ public double getDoubleVal(String key, double defaultValue) { return ((Integer) ret).doubleValue(); } if (ret instanceof String) { - return Double.valueOf((String) ret); + return Double.parseDouble((String) ret); } if (ret instanceof Float) { return ((Float) ret).doubleValue(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java index 35068840ab..f83a394399 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java @@ -49,7 +49,7 @@ public RestartConfig(Map map) { } public static RestartConfig defaultConfig(){ - Map map = new HashMap<>(1); + Map map = new HashMap<>(16); return new RestartConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java index c85c1bb376..ef886bf7e7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx.config; -import java.util.HashMap; +import java.util.Collections; import java.util.Map; /** @@ -40,8 +40,7 @@ public RestoreConfig(Map map) { } public static RestoreConfig defaultConfig(){ - Map map = new HashMap<>(1); - map.put(KEY_IS_RESTORE, false); + Map map = Collections.singletonMap(KEY_IS_RESTORE, false); return new RestoreConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java index 2a15601654..dbcd774858 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java @@ -37,7 +37,7 @@ public TestConfig(Map map) { } public static TestConfig defaultConfig(){ - Map map = new HashMap<>(1); + Map map = new HashMap<>(16); return new TestConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index 46de3749fb..d94589fbc6 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -51,4 +51,15 @@ public class ConstantValue { public static final String SYSTEM_PROPERTIES_KEY_FILE_ENCODING = "file.encoding"; public static final String OS_WINDOWS = "windows"; + + public static final String TIME_SECOND_SUFFIX = "sss"; + public static final String TIME_MILLISECOND_SUFFIX = "SSS"; + + public static final String FILE_SUFFIX_XML = ".xml"; + + public static final int MAX_BATCH_SIZE = 200000; + + public static final long STORE_SIZE_G = 1024L * 1024 * 1024; + + public static final long STORE_SIZE_M = 1024L * 1024; } diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java similarity index 69% rename from flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java index 63d92b8463..ce360c6bc0 100644 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java @@ -6,31 +6,41 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.emqx.decoder; -import java.util.Collections; -import java.util.Map; +package com.dtstack.flinkx.decoder; /** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou + * @author jiangbo + * @date 2020/3/23 */ -public class PlainDecoder implements IDecode { +public enum DecodeEnum { + + /** + * json format + */ + JSON("json"), - @Override - public Map decode(final String message) { - return Collections.singletonMap("message", message); + /** + * text format + */ + PLAIN("plain"); + + private String name; + + DecodeEnum(String name) { + this.name = name; } + public String getName() { + return name; + } } diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java similarity index 96% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java index 0d117d3956..5b195bd668 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import java.util.Map; diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java similarity index 97% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java index 8d6fad179e..10a954e601 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; @@ -31,12 +31,10 @@ * @author tudou */ public class JsonDecoder implements IDecode { + private static final String KEY_MESSAGE = "message"; private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - private static ObjectMapper objectMapper = new ObjectMapper(); - private static final String KEY_MESSAGE = "message"; - @Override @SuppressWarnings("unchecked") public Map decode(final String message) { diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java similarity index 96% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java index 5a2a027369..63771cfd6e 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import java.util.Collections; import java.util.Map; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java index 475f6923d1..8de25bd051 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java @@ -33,6 +33,7 @@ public enum EDatabaseType { SQLServer, Oracle, PostgreSQL, + Greenplum, DB2, MongoDB, Redis, diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 1ce9844175..2c55ee3a19 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -28,7 +28,6 @@ import com.dtstack.flinkx.metrics.CustomPrometheusReporter; import com.dtstack.flinkx.reader.ByteRateLimiter; import com.dtstack.flinkx.restore.FormatState; -import org.apache.commons.lang.StringUtils; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; @@ -376,14 +375,13 @@ public RestoreConfig getRestoreConfig() { return restoreConfig; } - public void setLogConfig(LogConfig logConfig) { - this.logConfig = logConfig; - } - public void setRestoreConfig(RestoreConfig restoreConfig) { this.restoreConfig = restoreConfig; } + public void setLogConfig(LogConfig logConfig) { + this.logConfig = logConfig; + } public void setTestConfig(TestConfig testConfig) { this.testConfig = testConfig; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java index c701707ade..feb64ae62f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java @@ -35,8 +35,13 @@ import org.slf4j.LoggerFactory; import java.io.InputStream; -import java.util.*; -import java.util.concurrent.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * Regularly get statistics from the flink API @@ -247,7 +252,7 @@ private void checkErrorTimes() { } } - class ValueAccumulator{ + static class ValueAccumulator{ private long global; private LongCounter local; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java index 3735cc7c64..dfe35d84fe 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java @@ -27,7 +27,13 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.*; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; @@ -39,7 +45,14 @@ import java.io.IOException; import java.lang.reflect.Field; -import java.util.*; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.regex.Pattern; /** @@ -121,7 +134,6 @@ public void open() { if (StringUtils.isNullOrWhitespaceOnly(host) || port < 1) { return; -// throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); } if (randomSuffix) { @@ -190,6 +202,10 @@ private void notifyOfAddedMetric(final Metric metric, final String metricName, f count = collectorWithCount.getValue(); } else { collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); + if (null == collector) { + return; + } + try { collector.register(defaultRegistry); } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java index f5991f1f9c..c46d451f48 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java @@ -100,19 +100,15 @@ public List getProgramExeArgList() throws Exception { continue; }else if(OPTION_JOB.equalsIgnoreCase(key)){ File file = new File(value.toString()); - FileInputStream in = new FileInputStream(file); - byte[] filecontent = new byte[(int) file.length()]; - in.read(filecontent); - value = new String(filecontent, Charsets.UTF_8.name()); + try (FileInputStream in = new FileInputStream(file)) { + byte[] filecontent = new byte[(int) file.length()]; + in.read(filecontent); + value = new String(filecontent, Charsets.UTF_8.name()); + } } args.add("-" + key); args.add(value.toString()); } return args; } - - private void printUsage() { - System.out.print(options.toString()); - } - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java index 6d92035700..f9735c0ab3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java @@ -294,11 +294,12 @@ protected void afterCloseInternal() { moveAllTemporaryDataFileToDirectory(); LOG.info("The task ran successfully,clear temporary data files"); + closeSource(); clearTemporaryDataFiles(); } + }else{ + closeSource(); } - - closeSource(); } catch(Exception ex) { throw new RuntimeException(ex); } @@ -328,11 +329,6 @@ public void tryCleanupOnError() throws Exception { } } - @Override - protected void writeMultipleRecordsInternal() throws Exception { - // CAN NOT HAPPEN - } - @Override protected boolean needWaitAfterCloseInternal() { return true; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java index 1b42017236..a73df9b854 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java @@ -51,7 +51,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.writer.WriteErrorTypes.*; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_FORMAT_TRANSFORM; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_NULL_POINTER; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_PRIMARY_CONFLICT; /** * Abstract Specification for all the OutputFormat defined in flinkx plugins @@ -418,6 +420,10 @@ protected void writeMultipleRecords() throws Exception { */ protected abstract void writeMultipleRecordsInternal() throws Exception; + protected void notSupportBatchWrite(String writerName) { + throw new UnsupportedOperationException(writerName + "不支持批量写入"); + } + protected void writeRecordInternal() { try { writeMultipleRecords(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java index 0bdcca624a..38306365f2 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.outputformat; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.constants.ConstantValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; @@ -71,6 +72,12 @@ public void setInitAccumulatorAndDirty(boolean initAccumulatorAndDirty) { this.format.initAccumulatorAndDirty = initAccumulatorAndDirty; } + protected void notSupportBatchWrite(String writerName) { + if (this.format.getBatchInterval() > 1) { + throw new IllegalArgumentException(writerName + "不支持批量写入"); + } + } + /** * Check the value of parameters */ @@ -78,7 +85,17 @@ public void setInitAccumulatorAndDirty(boolean initAccumulatorAndDirty) { public BaseRichOutputFormat finish() { checkFormat(); + + /** + * 200000条限制的原因: + * 按照目前的使用情况以及部署配置,假设写入字段数量平均为50个,一个单slot的TaskManager内存为1G, + * 在不考虑各插件批量写入对内存特殊要求并且只考虑插件缓存这么多条数据的情况下,batchInterval为400000条时出现fullGC, + * 为了避免fullGC以及OOM,并且保证batchInterval有足够的配置空间,取最大值的一半200000。 + */ + if (this.format.getBatchInterval() > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量写入条数必须小于[200000]条"); + } + return format; } - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index f66d45b7f9..13c21f8129 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -25,7 +25,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.*; +import java.math.BigDecimal; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * This class is user for speed control @@ -80,11 +84,12 @@ private void updateRate(){ long thisRecords = accumulatorCollector.getLocalAccumulatorValue(Metrics.NUM_READS); long totalRecords = accumulatorCollector.getAccumulatorValue(Metrics.NUM_READS); - double thisWriteRatio = (totalRecords == 0 ? 0 : thisRecords / (double)totalRecords); + BigDecimal thisWriteRatio = BigDecimal.valueOf(totalRecords == 0 ? 0 : thisRecords / (double) totalRecords); - if (totalRecords > MIN_RECORD_NUMBER_UPDATE_RATE && totalBytes != 0 && thisWriteRatio != 0) { - double bpr = totalBytes / totalRecords; - double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio; + if (totalRecords > MIN_RECORD_NUMBER_UPDATE_RATE && totalBytes != 0 + && thisWriteRatio.compareTo(new BigDecimal(0)) == 0) { + double bpr = totalBytes / (double)totalRecords; + double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio.doubleValue(); rateLimiter.setRate(permitsPerSecond); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java index d328115334..a7436f894f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.lang.StringUtils; import java.io.Serializable; @@ -48,6 +47,8 @@ public class MetaColumn implements Serializable { private String splitter; + private Boolean isPart; + public String getSplitter() { return splitter; } @@ -124,6 +125,7 @@ public static List getMetaColumns(List columns, boolean generateInde mc.setType(sm.get("type") != null ? String.valueOf(sm.get("type")) : null); mc.setValue(sm.get("value") != null ? String.valueOf(sm.get("value")) : null); mc.setSplitter(sm.get("splitter") != null ? String.valueOf(sm.get("splitter")) : null); + mc.setPart(sm.get("isPart") != null ? (Boolean) sm.get("isPart") : false); if(sm.get("format") != null && String.valueOf(sm.get("format")).trim().length() > 0){ mc.setTimeFormat(DateUtil.buildDateFormatter(String.valueOf(sm.get("format")))); @@ -151,6 +153,14 @@ public static List getMetaColumns(List columns, boolean generateInde return metaColumns; } + public Boolean getPart() { + return isPart; + } + + public void setPart(Boolean part) { + isPart = part; + } + public static List getMetaColumns(List columns){ return getMetaColumns(columns, true); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java index bba853e451..b073524412 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java @@ -33,7 +33,7 @@ public class ClassUtil { private static final Logger LOG = LoggerFactory.getLogger(ClassUtil.class); - public final static String LOCK_STR = "jdbc_lock_str"; + public final static Object LOCK_STR = new Object(); public static void forName(String clazz, ClassLoader classLoader) { synchronized (LOCK_STR){ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java index 62f3a7e98c..c003af9457 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java @@ -46,9 +46,14 @@ static Clock system() { return SYSTEM; } + /** + * Get current time + * @return time + */ default Instant currentTime() { return Instant.ofEpochMilli(currentTimeInMillis()); } + /** * Get the current time in nanoseconds. * @return the current time in nanoseconds. diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index 8ce417196d..a9b2ce6e53 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -21,7 +21,11 @@ import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.Calendar; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; /** * Date Utilities @@ -93,7 +97,12 @@ public static java.sql.Date columnToDate(Object column,SimpleDateFormat customTi if (((String) column).length() == 0){ return null; } - return new java.sql.Date(stringToDate((String)column,customTimeFormat).getTime()); + + Date date = stringToDate((String)column, customTimeFormat); + if (null == date) { + return null; + } + return new java.sql.Date(date.getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Date(getMillSecond(rawData.toString())); @@ -120,7 +129,12 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma if (((String) column).length() == 0){ return null; } - return new java.sql.Timestamp(stringToDate((String)column,customTimeFormat).getTime()); + + Date date = stringToDate((String)column,customTimeFormat); + if (null == date) { + return null; + } + return new java.sql.Timestamp(date.getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java index cddcf1ba29..d770364db7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java @@ -33,30 +33,19 @@ public class ExceptionUtil { private static Logger logger = LoggerFactory.getLogger(ExceptionUtil.class); public static String getErrorMessage(Throwable e) { - StringWriter stringWriter = null; - PrintWriter writer = null; - try{ - stringWriter= new StringWriter(); - writer = new PrintWriter(stringWriter); - e.printStackTrace(writer); - writer.flush(); - stringWriter.flush(); - StringBuffer buffer= stringWriter.getBuffer(); - return buffer.toString(); - }catch(Throwable ee){ - logger.error("",ee); + if (null == e) { + return null; + } - }finally { - if(writer!=null){ - writer.close(); - } - if(stringWriter!=null){ - try{ - stringWriter.close(); - }catch (Throwable ee){ - logger.error("",ee); - } - } + try (StringWriter stringWriter = new StringWriter(); + PrintWriter writer = new PrintWriter(stringWriter)) { + e.printStackTrace(writer); + writer.flush(); + stringWriter.flush(); + StringBuffer buffer = stringWriter.getBuffer(); + return buffer.toString(); + } catch (Throwable ee) { + logger.error("", ee); } return null; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java index c545ef3d94..0e7214fb58 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.security.PrivilegedAction; import java.util.HashMap; import java.util.Map; @@ -76,7 +77,7 @@ public static void setHadoopUserName(Configuration conf){ } } - private static boolean isOpenKerberos(Map hadoopConfig){ + public static boolean isOpenKerberos(Map hadoopConfig){ if(!MapUtils.getBoolean(hadoopConfig, KEY_HADOOP_SECURITY_AUTHORIZATION, false)){ return false; } @@ -85,13 +86,7 @@ private static boolean isOpenKerberos(Map hadoopConfig){ } private static FileSystem getFsWithKerberos(Map hadoopConfig, String defaultFs) throws Exception{ - String keytabFileName = KerberosUtil.getPrincipalFileName(hadoopConfig); - - keytabFileName = KerberosUtil.loadFile(hadoopConfig, keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); - KerberosUtil.loadKrb5Conf(hadoopConfig); - - UserGroupInformation ugi = KerberosUtil.loginAndReturnUgi(getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); + UserGroupInformation ugi = getUGI(hadoopConfig, defaultFs); UserGroupInformation.setLoginUser(ugi); return ugi.doAs(new PrivilegedAction() { @@ -106,6 +101,19 @@ public FileSystem run(){ }); } + public static UserGroupInformation getUGI(Map hadoopConfig, String defaultFs) throws IOException { + String keytabFileName = KerberosUtil.getPrincipalFileName(hadoopConfig); + + keytabFileName = KerberosUtil.loadFile(hadoopConfig, keytabFileName); + String principal = KerberosUtil.getPrincipal(hadoopConfig, keytabFileName); + KerberosUtil.loadKrb5Conf(hadoopConfig); + + UserGroupInformation ugi = KerberosUtil.loginAndReturnUgi(getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); + UserGroupInformation.setLoginUser(ugi); + + return ugi; + } + public static Configuration getConfiguration(Map confMap, String defaultFs) { confMap = fillConfig(confMap, defaultFs); @@ -134,7 +142,7 @@ public static JobConf getJobConf(Map confMap, String defaultFs){ private static Map fillConfig(Map confMap, String defaultFs) { if (confMap == null) { - confMap = new HashMap<>(8); + confMap = new HashMap<>(); } if (isHaMode(confMap)) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java new file mode 100644 index 0000000000..b94765dc4a --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.util; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.TypeAdapter; +import com.google.gson.TypeAdapterFactory; +import com.google.gson.internal.LinkedTreeMap; +import com.google.gson.internal.bind.ObjectTypeAdapter; +import com.google.gson.reflect.TypeToken; +import com.google.gson.stream.JsonReader; +import com.google.gson.stream.JsonToken; +import com.google.gson.stream.JsonWriter; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Date: 2020/06/12 + * Company: www.dtstack.com + * + * Gson工具类,用于对json的序列化及反序列化,及解决int类型在map中被转换成double类型问题 + * + * @author tudou + */ +public class GsonUtil { + + public static Gson GSON = getGson(); + public static Type gsonMapTypeToken = new TypeToken>(){}.getType(); + + @SuppressWarnings("unchecked") + private static Gson getGson() { + GSON = new GsonBuilder().create(); + try { + Field factories = Gson.class.getDeclaredField("factories"); + factories.setAccessible(true); + Object o = factories.get(GSON); + Class[] declaredClasses = Collections.class.getDeclaredClasses(); + for (Class c : declaredClasses) { + if ("java.util.Collections$UnmodifiableList".equals(c.getName())) { + Field listField = c.getDeclaredField("list"); + listField.setAccessible(true); + List list = (List) listField.get(o); + int i = list.indexOf(ObjectTypeAdapter.FACTORY); + list.set(i, new TypeAdapterFactory() { + @Override + public TypeAdapter create(Gson gson, TypeToken type) { + if (type.getRawType() == Object.class) { + return new TypeAdapter() { + @Override + public Object read(JsonReader in) throws IOException { + JsonToken token = in.peek(); + //判断字符串的实际类型 + switch (token) { + case BEGIN_ARRAY: + List list = new ArrayList<>(); + in.beginArray(); + while (in.hasNext()) { + list.add(read(in)); + } + in.endArray(); + return list; + + case BEGIN_OBJECT: + Map map = new LinkedTreeMap<>(); + in.beginObject(); + while (in.hasNext()) { + map.put(in.nextName(), read(in)); + } + in.endObject(); + return map; + case STRING: + return in.nextString(); + case NUMBER: + String s = in.nextString(); + if (s.contains(".")) { + return Double.valueOf(s); + } else { + try { + return Integer.valueOf(s); + } catch (Exception e) { + return Long.valueOf(s); + } + } + case BOOLEAN: + return in.nextBoolean(); + case NULL: + in.nextNull(); + return null; + default: + throw new IllegalStateException(); + } + } + + @Override + public void write(JsonWriter out, Object value) throws IOException { + if (value == null) { + out.nullValue(); + return; + } + //noinspection unchecked + TypeAdapter typeAdapter = gson.getAdapter((Class) value.getClass()); + if (typeAdapter instanceof ObjectTypeAdapter) { + out.beginObject(); + out.endObject(); + return; + } + typeAdapter.write(out, value); + } + }; + } + return null; + } + }); + break; + } + } + } catch (Exception e) { + e.printStackTrace(); + } + return GSON; + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java new file mode 100644 index 0000000000..218ca60eea --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.util; + +import java.util.HashMap; + +/** + * @author tiezhu + */ +public class JsonModifyUtil { + + public static String JsonValueReplace(String json, HashMap parameter){ + for(String item: parameter.keySet()){ + if(json.contains("${"+item+"}")){ + json = json.replace("${"+item+"}", parameter.get(item)); + } + } + return json; + } + + /** + * 将命令行中的修改命令转化为HashMap保存 + */ + public static HashMap CommandTransform(String command) { + HashMap parameter = new HashMap<>(); + for (String item : command.split(",")) { + String[] temp = item.split("="); + parameter.put(temp[0], temp[1]); + } + return parameter; + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java index 78a328fdac..b4656af3ac 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java @@ -22,6 +22,8 @@ import com.google.common.collect.Lists; import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.JobExecutionResult; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; @@ -31,6 +33,8 @@ */ public class ResultPrintUtil { + private static Logger LOG = LoggerFactory.getLogger(ResultPrintUtil.class); + public static void printResult(JobExecutionResult result){ List names = Lists.newArrayList(); List values = Lists.newArrayList(); @@ -56,8 +60,8 @@ public static void printResult(JobExecutionResult result){ } } - System.out.println("---------------------------------"); - System.out.println(builder.toString()); - System.out.println("---------------------------------"); + LOG.info("---------------------------------"); + LOG.info(builder.toString()); + LOG.info("---------------------------------"); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java index 000d7a7ba2..5f8329a2e4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java @@ -33,7 +33,7 @@ public final class RetryUtil { private static final Logger LOG = LoggerFactory.getLogger(RetryUtil.class); - private static final long MAX_SLEEP_MILLISECOND = 256 * 1000; + private static final long MAX_SLEEP_MILLISECOND = 256 * 1000L; /** * 重试次数工具方法. diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index b276d856cd..5ae0ca309c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -38,6 +38,8 @@ */ public class StringUtil { + public static final int STEP_SIZE = 2; + /** * Handle the escaped escape charactor. * @@ -192,14 +194,14 @@ public static String col2string(Object column, String type) { } - public static String row2string(Row row, List columnTypes, String delimiter, List columnNames) throws WriteRecordException { + public static String row2string(Row row, List columnTypes, String delimiter) throws WriteRecordException { // convert row to string - int cnt = row.getArity(); - StringBuilder sb = new StringBuilder(); + int size = row.getArity(); + StringBuilder sb = new StringBuilder(128); int i = 0; try { - for (; i < cnt; ++i) { + for (; i < size; ++i) { if (i != 0) { sb.append(delimiter); } @@ -212,9 +214,9 @@ public static String row2string(Row row, List columnTypes, String delimi sb.append(col2string(column, columnTypes.get(i))); } - } catch(Exception ex) { + } catch(Exception e) { String msg = "StringUtil.row2string error: when converting field[" + i + "] in Row(" + row + ")"; - throw new WriteRecordException(msg, ex, i, row); + throw new WriteRecordException(msg, e, i, row); } return sb.toString(); @@ -228,7 +230,7 @@ public static byte[] hexStringToByteArray(String hexString) { int length = hexString.length(); byte[] bytes = new byte[length / 2]; - for (int i = 0; i < length; i += 2) { + for (int i = 0; i < length; i += STEP_SIZE) { bytes[i / 2] = (byte) ((Character.digit(hexString.charAt(i), 16) << 4) + Character.digit(hexString.charAt(i+1), 16)); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java index dd4ff9308a..3cc4719ea9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java @@ -34,9 +34,16 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.ArrayList; +import java.util.Date; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; -import static com.dtstack.flinkx.writer.WriteErrorTypes.*; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_FORMAT_TRANSFORM; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_NULL_POINTER; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_PRIMARY_CONFLICT; /** * The class handles dirty data management diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 1d2f7d9ae8..189f198a15 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -29,7 +29,8 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -public class ErrorLimiter { +public class +ErrorLimiter { private final Integer maxErrors; private final Double maxErrorRatio; diff --git a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index e331b3e672..b8ca7cdbc7 100644 --- a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -57,7 +57,19 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.source.*; +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction; +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator; +import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction; +import org.apache.flink.streaming.api.functions.source.FileProcessingMode; +import org.apache.flink.streaming.api.functions.source.FileReadFunction; +import org.apache.flink.streaming.api.functions.source.FromElementsFunction; +import org.apache.flink.streaming.api.functions.source.FromIteratorFunction; +import org.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunction; +import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.StoppableStreamSource; diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java index 7b68e5fcc0..0210f3c05c 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java new file mode 100644 index 0000000000..6607984c4e --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java @@ -0,0 +1,112 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; +import java.sql.Date; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class DateUtilTest { + + @Test + public void testColumnToDate() { + Date result = DateUtil.columnToDate(null, null); + Assert.assertNull(result); + + result = DateUtil.columnToDate("", null); + Assert.assertNull(result); + + result = DateUtil.columnToDate("2020-03-18 10:56:00", null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(1584500160, null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(1584500160000L, null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new Date(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new Timestamp(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new java.util.Date(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + try { + DateUtil.columnToDate(true, null); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + @Test + public void testColumnToTimestamp() { + Timestamp result = DateUtil.columnToTimestamp(null, null); + Assert.assertNull(result); + + result = DateUtil.columnToTimestamp("", null); + Assert.assertNull(result); + + result = DateUtil.columnToTimestamp("2020-03-18 10:56:00", null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(1584500160, null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(1584500160000L, null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new Date(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new Timestamp(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new java.util.Date(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + try { + DateUtil.columnToTimestamp(true, null); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + @Test + public void testGetMillSecond() { + long result = DateUtil.getMillSecond("1584500160000"); + Assert.assertEquals(result, 1584500160000L); + + result = DateUtil.getMillSecond("1584500160000000"); + Assert.assertEquals(result, 1584500160000L); + + result = DateUtil.getMillSecond("1584500160000000000"); + Assert.assertEquals(result, 1584500160000L); + + long expect = 57600000; // 1970-01-02 00:00:00:000 + result = DateUtil.getMillSecond("1"); + Assert.assertEquals(result, expect); + } + + @Test + public void testStringToDate() { + java.util.Date result = DateUtil.stringToDate("", null); + Assert.assertNull(result); + + result = DateUtil.stringToDate("2020/03/18 13:26:00", new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")); + Assert.assertEquals(result, new java.util.Date(1584509160000L)); + + try { + DateUtil.stringToDate("xxxx", null); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("can't parse date")); + } + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java new file mode 100644 index 0000000000..5d988bd64d --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java @@ -0,0 +1,23 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class ExceptionUtilTest { + + @Test + public void testGetErrorMessage() { + String result = ExceptionUtil.getErrorMessage(null); + Assert.assertNull(result); + + try { + ExceptionUtil.getErrorMessage(new IllegalArgumentException("error test")); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java index c7339dd3fa..1391e2a357 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java @@ -1,8 +1,8 @@ package com.dtstack.flinkx.util; import com.google.gson.internal.LinkedTreeMap; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; import java.util.HashMap; import java.util.Map; diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java index b38fc07328..b781529383 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java index b373be8f19..7a68496791 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java @@ -1,8 +1,8 @@ package com.dtstack.flinkx.util; import org.apache.flink.types.Row; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java new file mode 100644 index 0000000000..69da0176bd --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java @@ -0,0 +1,28 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class SnowflakeIdWorkerTest { + + SnowflakeIdWorker snowflakeIdWorker = new SnowflakeIdWorker(1L, 1L); + + @Test + public void testNextId() { + Set idSet = new HashSet<>(); + int i = 0; + while (i++ < 100) { + long result = snowflakeIdWorker.nextId(); + idSet.add(result); + } + + Assert.assertEquals(idSet.size(), 100); + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java index b072f9168a..bcff236c57 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java @@ -1,9 +1,11 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; +import java.sql.Timestamp; import java.text.SimpleDateFormat; +import java.util.Date; /** * @author jiangbo @@ -66,4 +68,52 @@ public void testString2col() { result = StringUtil.string2col("xxx", "xxx", null); Assert.assertEquals(result, "xxx"); } + + @Test + public void testCol2string() { + String result = StringUtil.col2string(null, null); + Assert.assertEquals(result, ""); + + result = StringUtil.col2string("test", null); + Assert.assertEquals(result, "test"); + + result = StringUtil.col2string(Byte.valueOf("1"), "TINYINT"); + Assert.assertEquals(result, "1"); + + result = StringUtil.col2string(Short.valueOf("1"), "SMALLINT"); + Assert.assertEquals(result, "1"); + + result = StringUtil.col2string(123, "INTEGER"); + Assert.assertEquals(result, "123"); + + result = StringUtil.col2string(123L, "LONG"); + Assert.assertEquals(result, "123"); + + result = StringUtil.col2string(new Timestamp(1584510286187L), "LONG"); + Assert.assertEquals(result, "1584510286187"); + + result = StringUtil.col2string(123.123, "FLOAT"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string(123.123, "DOUBLE"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string(123.123, "DECIMAL"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string("string", "STRING"); + Assert.assertEquals(result, "string"); + + result = StringUtil.col2string(new Timestamp(1584510286187L), "STRING"); + Assert.assertEquals(result, "2020-03-18 13:44:46"); + + result = StringUtil.col2string(true, "BOOLEAN"); + Assert.assertEquals(result, "true"); + + result = StringUtil.col2string(new Date(1584510286187L), "DATE"); + Assert.assertEquals(result, "2020-03-18"); + + result = StringUtil.col2string(new Date(1584510286187L), "DATETIME"); + Assert.assertEquals(result, "2020-03-18 13:44:46"); + } } \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java index d22a8853d7..75e254d9a5 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo @@ -9,7 +9,7 @@ */ public class ValueUtilTest { - @Test(expectedExceptions = RuntimeException.class) + @Test public void testGetInt() { Integer result = ValueUtil.getInt(null); Assert.assertNull(result); @@ -20,6 +20,10 @@ public void testGetInt() { result = ValueUtil.getInt(new Long(100)); Assert.assertEquals(result, new Integer(100)); - ValueUtil.getInt(new Object()); + try { + ValueUtil.getInt(new Object()); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Unable to convert")); + } } } \ No newline at end of file diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 6a6f40a2c7..483fc23905 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -63,8 +63,12 @@ shade.db2reader.io.netty - com.google - shade.db2reader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 2ff75d22e0..b3d103b159 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -63,8 +63,12 @@ shade.db2writer.io.netty - com.google - shade.db2writer.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java index 3c4d2883dc..da6829b005 100644 --- a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java +++ b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java @@ -47,7 +47,7 @@ protected String makeValues(List column) { public String quoteTable(String table) { table = table.replace("\"",""); String[] part = table.split("\\."); - if(part.length == 2) { + if(part.length == DB_TABLE_PART_SIZE) { table = getStartQuote() + part[0] + getEndQuote() + "." + getStartQuote() + part[1] + getEndQuote(); } else { table = getStartQuote() + table + getEndQuote(); diff --git a/flinkx-dm/flinkx-dm-reader/pom.xml b/flinkx-dm/flinkx-dm-reader/pom.xml index 15ce92929b..c2feddd9c8 100644 --- a/flinkx-dm/flinkx-dm-reader/pom.xml +++ b/flinkx-dm/flinkx-dm-reader/pom.xml @@ -63,8 +63,12 @@ shade.dmreader.io.netty - com.google - shade.dmreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java index 8a114752f1..67e70d499d 100644 --- a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java +++ b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java @@ -25,8 +25,6 @@ import java.io.IOException; import java.sql.Timestamp; -import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; - /** * Date: 2020/03/18 * Company: www.dtstack.com @@ -49,7 +47,7 @@ public Row nextRecordInternal(Row row) throws IOException { || obj.getClass().getSimpleName().toUpperCase().contains("TIMESTAMP")) ) { obj = resultSet.getTimestamp(pos + 1); } - obj = clobToString(obj); + obj = DbUtil.clobToString(obj); } row.setField(pos, obj); diff --git a/flinkx-dm/flinkx-dm-writer/pom.xml b/flinkx-dm/flinkx-dm-writer/pom.xml index abe284fc04..768be50eae 100644 --- a/flinkx-dm/flinkx-dm-writer/pom.xml +++ b/flinkx-dm/flinkx-dm-writer/pom.xml @@ -63,8 +63,12 @@ shade.dmwriter.io.netty - com.google - shade.dmwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java b/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java index c3912a218c..9612294ec8 100644 --- a/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java +++ b/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java @@ -18,11 +18,16 @@ package com.dtstack.flinkx.dm.format; import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; import com.dtstack.flinkx.util.DateUtil; import org.apache.flink.types.Row; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.HashMap; @@ -61,7 +66,7 @@ protected List probeFullColumns(String table, Connection dbConn) throws String schema =null; String[] parts = table.split("\\."); - if(parts.length == 2) { + if(parts.length == BaseDatabaseMeta.DB_TABLE_PART_SIZE) { schema = parts[0].toUpperCase(); table = parts[1]; } @@ -76,25 +81,27 @@ protected List probeFullColumns(String table, Connection dbConn) throws @Override protected Map> probePrimaryKeys(String table, Connection dbConn) throws SQLException { - Map> map = new HashMap<>(); - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); - ResultSet rs = ps.executeQuery(); + Map> map = new HashMap<>(16); - while(rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if(!map.containsKey(indexName)) { - map.put(indexName,new ArrayList<>()); + try (PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); + ResultSet rs = ps.executeQuery()) { + while(rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if(!map.containsKey(indexName)) { + map.put(indexName,new ArrayList<>()); + } + map.get(indexName).add(rs.getString("COLUMN_NAME")); } - map.get(indexName).add(rs.getString("COLUMN_NAME")); - } - Map> retMap = new HashMap<>(); - for(Map.Entry> entry: map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if(v!=null && v.size() != 0 && v.get(0) != null) { - retMap.put(k, v); + + Map> retMap = new HashMap<>(16); + for(Map.Entry> entry: map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if(v!=null && v.size() != 0 && v.get(0) != null) { + retMap.put(k, v); + } } + return retMap; } - return retMap; } } diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java b/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java deleted file mode 100644 index ecb26a2f3e..0000000000 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.emqx.decoder; - -import org.codehaus.jackson.map.ObjectMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.Map; - -/** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou - */ -public class JsonDecoder implements IDecode { - private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - - private static ObjectMapper objectMapper = new ObjectMapper(); - - private static final String KEY_MESSAGE = "message"; - - @Override - @SuppressWarnings("unchecked") - public Map decode(final String message) { - try { - Map event = objectMapper.readValue(message, Map.class); - if (!event.containsKey(KEY_MESSAGE)) { - event.put(KEY_MESSAGE, message); - } - return event; - } catch (Exception e) { - LOG.error(e.getMessage()); - return Collections.singletonMap(KEY_MESSAGE, message); - } - - } - -} diff --git a/flinkx-emqx/flinkx-emqx-reader/pom.xml b/flinkx-emqx/flinkx-emqx-reader/pom.xml index 0e106143fd..205e8a9c5c 100644 --- a/flinkx-emqx/flinkx-emqx-reader/pom.xml +++ b/flinkx-emqx/flinkx-emqx-reader/pom.xml @@ -53,6 +53,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java index 5534601f51..7e92921dcc 100644 --- a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java +++ b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java @@ -17,16 +17,21 @@ */ package com.dtstack.flinkx.emqx.format; -import com.dtstack.flinkx.emqx.decoder.IDecode; -import com.dtstack.flinkx.emqx.decoder.JsonDecoder; -import com.dtstack.flinkx.emqx.decoder.PlainDecoder; +import com.dtstack.flinkx.decoder.IDecode; +import com.dtstack.flinkx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.PlainDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.eclipse.paho.client.mqttv3.*; +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; +import org.eclipse.paho.client.mqttv3.MqttCallback; +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttConnectOptions; +import org.eclipse.paho.client.mqttv3.MqttException; +import org.eclipse.paho.client.mqttv3.MqttMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java index 83d70d80d1..32121eec6a 100644 --- a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java +++ b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java @@ -8,7 +8,13 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -import static com.dtstack.flinkx.emqx.EmqxConfigKeys.*; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_BROKER; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_CODEC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_IS_CLEAN_SESSION; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_QOS; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_TOPIC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_USERNAME; /** * Date: 2020/02/12 diff --git a/flinkx-emqx/flinkx-emqx-writer/pom.xml b/flinkx-emqx/flinkx-emqx-writer/pom.xml index a6117433f5..fcf861883a 100644 --- a/flinkx-emqx/flinkx-emqx-writer/pom.xml +++ b/flinkx-emqx/flinkx-emqx-writer/pom.xml @@ -53,6 +53,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java index 79dec0f172..f1e87baba3 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java @@ -17,7 +17,7 @@ */ package com.dtstack.flinkx.emqx.format; -import com.dtstack.flinkx.emqx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.JsonDecoder; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java index 0ba2a1a12d..52a311b76e 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java @@ -25,7 +25,12 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; -import static com.dtstack.flinkx.emqx.EmqxConfigKeys.*; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_BROKER; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_IS_CLEAN_SESSION; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_QOS; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_TOPIC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_USERNAME; /** * Date: 2020/02/12 diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 227dc25d10..8ccef94149 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -53,6 +53,20 @@ + + + com.fasterxml.jackson.databind + shade.es.com.fasterxml.jackson.databind + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 282e2ac5f8..1c54a5e115 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -25,7 +25,11 @@ import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.elasticsearch.action.search.*; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.query.QueryBuilders; diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 33f9e85d21..57584faeb4 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.es.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import java.util.List; import java.util.Map; @@ -103,5 +104,9 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + if (format.batchSize > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取数量不能大于[200000]条"); + } } } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index da95c5b0ee..e57fa881a4 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -28,6 +28,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.HashMap; @@ -42,6 +44,8 @@ */ public class EsReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(EsReader.class); + private String address; private String username; private String password; @@ -87,7 +91,8 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { columnValue.add((String) sm.get("value")); columnName.add((String) sm.get("name")); } - System.out.println("init column finished"); + + LOG.info("init column finished"); } else if (!ConstantValue.STAR_SYMBOL.equals(columns.get(0)) || columns.size() != 1) { throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java b/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java deleted file mode 100644 index 06e304c46a..0000000000 --- a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java +++ /dev/null @@ -1,90 +0,0 @@ -package com.dtstack.flinkx.es.reader.test; - - -import com.google.gson.Gson; -import org.apache.http.HttpHost; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.aggregations.AggregationBuilder; -import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.Aggregations; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import static org.elasticsearch.index.query.QueryBuilders.termQuery; - -public class EsReadDemo { - - public static void searchAll() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - SearchRequest searchRequest = new SearchRequest(); -// SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); -// searchSourceBuilder.query(QueryBuilders.matchAllQuery()); -// searchSourceBuilder().from(0); -// searchSourceBuilder().size(100); - - SearchResponse searchResponse = client.search(searchRequest); - SearchHits searchHits = searchResponse.getHits(); - - for(SearchHit searchHit : searchHits) { - Map source = searchHit.getSourceAsMap(); - System.out.println(source); - } - - - } - - public static void searchPart() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); - - sourceBuilder.from(0); - sourceBuilder.size(100); - -// QueryBuilder qb2 = QueryBuilders.wrapperQuery("{\"match_all\": {}}"); - Map map = new HashMap<>(); - Map match = new HashMap<>(); - map.put("match",match); - match.put("col2", "hallo"); - Gson gson = new Gson(); - - //QueryBuilder qb2 = QueryBuilders.wrapperQuery("{\"match\": {\"col2\":\"hallo\"}}"); - QueryBuilder qb2 = QueryBuilders.wrapperQuery(gson.toJson(map)); - sourceBuilder.query(qb2); - searchRequest.source(sourceBuilder); - - SearchResponse searchResponse = client.search(searchRequest); - System.out.println(searchResponse); - SearchHits searchHits = searchResponse.getHits(); - for(SearchHit searchHit : searchHits.getHits()) { - System.out.println(searchHit.docId() + ":" + searchHit.getSourceAsMap()); - } - long total = searchHits.getTotalHits(); - System.out.println("total: " + total); - - client.close(); - } - - - public static void main(String[] args) throws IOException { - //searchAll(); - searchPart(); - //searchAll(); - } - -} diff --git a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java b/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java deleted file mode 100644 index bf98581615..0000000000 --- a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.dtstack.flinkx.es.reader.test; - -import com.google.gson.Gson; - -import java.util.HashMap; -import java.util.Map; - -/** - * Created by softfly on 18/2/8. - */ -public class EsReaderTest { - public static void main(String[] args) { - Gson gson = new Gson(); - Map map = new HashMap<>(); - map.put("xxx", 111); - map.put("yyyy", "fff"); - String json = gson.toJson(map); - System.out.println(json); - } -} diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 1cdfab73fb..8e3e15a822 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -54,6 +54,20 @@ + + + com.fasterxml.jackson.databind + shade.es.com.fasterxml.jackson.databind + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java b/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java deleted file mode 100644 index 503c09fa7e..0000000000 --- a/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java +++ /dev/null @@ -1,146 +0,0 @@ -package com.dtstack.flinkx.es.writer.test; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.action.update.UpdateResponse; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.builder.SearchSourceBuilder; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -public class EsDemo { - - public static void test1() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - - IndexRequest request = new IndexRequest( - "nani222", - "doc222", - "id2"); - - String jsonString = "{" + - "\"user\":\"user2\"," + - "\"postDate\":\"2014-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; - - - request.source(jsonString, XContentType.JSON); - IndexResponse response = client.index(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test3() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - - IndexRequest request = new IndexRequest( - "nani", - "doc"); - -// String jsonString = "{" + -// "\"user\":\"xxxx\"," + -// "\"postDate\":\"2013-01-30\"," + -// "\"message\":\"trying out Elasticsearch\"" + -// "}"; - Map jsonMap = new HashMap<>(); - jsonMap.put("xxx", "asfdasdf"); - jsonMap.put("zzz", "asdfsadf"); - request.source(jsonMap); - IndexResponse response = client.index(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test2() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("localhost", 9200, "http"), - new HttpHost("localhost", 9201, "http"))); - - UpdateRequest request = new UpdateRequest( - "nani250", - "doc", - "2"); - - String jsonString = "{" + - "\"user\":\"kimchy\"," + - "\"postDate\":\"2013-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; - - request.doc(jsonString, XContentType.JSON); - UpdateResponse response = client.update(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test4() throws IOException { - - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchSourceBuilder.query(QueryBuilders.matchAllQuery()); - - SearchResponse searchResponse = client.search(searchRequest); - System.out.println(searchResponse.getTotalShards()); - } - - public static void test5() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - BulkRequest bulkRequest = new BulkRequest(); - - IndexRequest request = new IndexRequest("nani", "doc1"); - Map jsonMap = new HashMap<>(); - jsonMap.put("xxx", "8888"); - jsonMap.put("yyy", "9999"); - - bulkRequest.add(request.source(jsonMap)); - // bulkRequest.setRefreshPolicy(null); - // WriteRequest.RefreshPolicy; - - BulkResponse bulkResponse = client.bulk(bulkRequest); - - System.out.println(bulkResponse); - } - - public static void test6() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchResponse resp = client.search(searchRequest); - resp.getAggregations(); - } - - public static void main(String[] args) throws Exception { - test1(); - } - - -} diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java index fc58ca41e7..afc45b081a 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java @@ -59,7 +59,7 @@ public class FtpConfig implements Serializable { public int timeout = FtpConfigConstants.DEFAULT_TIMEOUT; - public long maxFileSize = 1024 * 1024 * 1024; + public long maxFileSize = 1024 * 1024 * 1024L; public String getUsername() { return username; diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index d201625068..4b99e46257 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -89,23 +89,14 @@ public void loginFtpServer(FtpConfig ftpConfig) { } @Override - public void logoutFtpServer() { + public void logoutFtpServer() throws IOException{ if (ftpClient.isConnected()) { try { ftpClient.logout(); - } catch (IOException e) { - LOG.error(DISCONNECT_FAIL_MESSAGE); - throw new RuntimeException(e); - }finally { + } finally { if(ftpClient.isConnected()){ - try { - ftpClient.disconnect(); - } catch (IOException e) { - LOG.error(DISCONNECT_FAIL_MESSAGE); - throw new RuntimeException(e); - } + ftpClient.disconnect(); } - } } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java index cc83e0343a..8c775f062d 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java @@ -19,6 +19,8 @@ package com.dtstack.flinkx.ftp; +import org.apache.commons.lang3.StringUtils; + /** * @author jiangbo * @date 2019/11/21 @@ -28,8 +30,8 @@ public class FtpHandlerFactory { public static IFtpHandler createFtpHandler(String protocolStr){ IFtpHandler ftpHandler; - EProtocol protocol = EProtocol.getByName(protocolStr); - if(EProtocol.SFTP.equals(protocol)) { + Protocol protocol = Protocol.getByName(protocolStr); + if(Protocol.SFTP.equals(protocol)) { ftpHandler = new SftpHandler(); } else { ftpHandler = new FtpHandler(); @@ -37,4 +39,16 @@ public static IFtpHandler createFtpHandler(String protocolStr){ return ftpHandler; } + + enum Protocol{ + FTP, SFTP; + + public static Protocol getByName(String name) { + if (StringUtils.isEmpty(name)) { + return SFTP; + } + + return Protocol.valueOf(name.toUpperCase()); + } + } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java index c8da4124b1..16faf8e6d8 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.ftp; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; @@ -40,8 +41,9 @@ public interface IFtpHandler { /** * 登出服务器 + * @throws IOException logout error */ - void logoutFtpServer(); + void logoutFtpServer() throws IOException; /** * 判断给定的目录是否存在 diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 1771055c60..42ae5c1123 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -71,6 +71,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index 6f0f4471d3..3b3b62013c 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -19,7 +19,12 @@ package com.dtstack.flinkx.ftp.reader; import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.ftp.*; +import com.dtstack.flinkx.ftp.EProtocol; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandler; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; +import com.dtstack.flinkx.ftp.IFtpHandler; +import com.dtstack.flinkx.ftp.SftpHandler; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; @@ -54,13 +59,7 @@ public class FtpInputFormat extends BaseRichInputFormat { public void openInputFormat() throws IOException { super.openInputFormat(); - if(EProtocol.SFTP.name().equalsIgnoreCase(ftpConfig.getProtocol())) { - ftpHandler = new SftpHandler(); - } else if(EProtocol.FTP.name().equalsIgnoreCase(ftpConfig.getProtocol())){ - ftpHandler = new FtpHandler(); - } else { - throw new RuntimeException("协议名称错误:" + ftpConfig.getProtocol()); - } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); ftpHandler.loginFtpServer(ftpConfig); } @@ -157,5 +156,4 @@ public void closeInternal() throws IOException { ftpHandler.logoutFtpServer(); } } - } diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java index 411389206d..c889b249cf 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpConfigConstants; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; @@ -30,8 +31,6 @@ import java.util.List; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.*; - /** * The reader plugin of Ftp * @@ -57,7 +56,7 @@ public FtpReader(DataTransferConfig config, StreamExecutionEnvironment env) { ftpConfig.setDefaultPort(); } - if(!DEFAULT_FIELD_DELIMITER.equals(ftpConfig.getFieldDelimiter())){ + if(!FtpConfigConstants.DEFAULT_FIELD_DELIMITER.equals(ftpConfig.getFieldDelimiter())){ String fieldDelimiter = StringUtil.convertRegularExpr(ftpConfig.getFieldDelimiter()); ftpConfig.setFieldDelimiter(fieldDelimiter); } diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java index a19e566fa6..34755a4109 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java @@ -20,8 +20,13 @@ import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.ftp.FtpHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.util.Iterator; /** @@ -32,6 +37,8 @@ */ public class FtpSeqBufferedReader { + private static Logger LOG = LoggerFactory.getLogger(FtpSeqBufferedReader.class); + private IFtpHandler ftpHandler; private Iterator iter; @@ -76,7 +83,8 @@ private void nextStream() throws IOException{ br = new BufferedReader(new InputStreamReader(in, charsetName)); for (int i = 0; i < fromLine; i++) { - br.readLine(); + String skipLine = br.readLine(); + LOG.info("Skip line:{}", skipLine); } } else { br = null; diff --git a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java b/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java deleted file mode 100644 index ed9177c3b7..0000000000 --- a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.dtstack.flinkx.ftp.reader; - -import org.apache.commons.net.ftp.FTPClient; -import org.apache.commons.net.ftp.FTPFile; - -import java.io.IOException; -import java.io.InputStream; - -/** - * Created by softfly on 17/11/22. - */ -public class FtpServerDemo { - - public static void main(String[] args) throws IOException { - FTPClient ftp = new FTPClient(); - String username = "test"; - String password = "qbI#5pNd"; - ftp.connect("node02", 21); - ftp.login(username, password); - FTPFile[] ftpFiles = ftp.listFiles(); - for(FTPFile ftpFile : ftpFiles) { - System.out.println(ftpFile.getName()); - } - - - String[] xxx = ftp.listNames(); - - InputStream is1 = ftp.retrieveFileStream("hyf/ttt"); - ftp.getReply(); - InputStream is2 = ftp.retrieveFileStream("xxx"); - ftp.remoteRetrieve("/hyf/ttt"); - ftp.getReply(); - - ftp.changeWorkingDirectory("/hyf"); - System.out.println(ftp.printWorkingDirectory()); - - ftp.logout(); - - } - -} diff --git a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java b/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java deleted file mode 100644 index 6defecb105..0000000000 --- a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java +++ /dev/null @@ -1,42 +0,0 @@ -package com.dtstack.flinkx.ftp.reader; - -import com.jcraft.jsch.*; - -import java.util.Properties; -import java.util.Vector; - -/** - * Created by softfly on 17/11/23. - */ -public class SftpServerDemo { - - public static void main(String[] args) throws JSchException, SftpException { - JSch jsch = new JSch(); - - Session session = jsch.getSession("mysftp", "node02"); - session.setPassword("oh1986mygod"); - session.setPort(22); - //session.setTimeout(10); - Properties config = new Properties(); - config.put("StrictHostKeyChecking", "no"); - - session.setConfig(config); - session.connect(); - - ChannelSftp channelSftp = (ChannelSftp) session.openChannel("sftp"); // 打开SFTP通道 - channelSftp.connect(); // 建立SFTP通道的连接 - - Vector vector = channelSftp.ls("/"); - - for(int i = 0; i < vector.size(); ++i) { - ChannelSftp.LsEntry le = (ChannelSftp.LsEntry) vector.get(i); - System.out.println(le.getFilename() ); - System.out.println(le.getLongname()); - } - - - //session.disconnect(); - - } - -} diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 61d8ffaede..5c5b2bb6f7 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -72,6 +72,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java index aeddb31bcb..4d66e893b1 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java @@ -19,22 +19,27 @@ package com.dtstack.flinkx.ftp.writer; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.ftp.*; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; +import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; +import java.io.BufferedWriter; import java.io.IOException; -import java.io.OutputStream; -import java.util.Arrays; +import java.io.OutputStreamWriter; +import java.io.UnsupportedEncodingException; +import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.SFTP_PROTOCOL; - /** * The OutputFormat Implementation which reads data from ftp servers. * @@ -54,21 +59,18 @@ public class FtpOutputFormat extends BaseFileOutputFormat { private transient IFtpHandler ftpHandler; - private transient OutputStream os; + private static final int FILE_NAME_PART_SIZE = 3; private static final String DOT = "."; private static final String FILE_SUFFIX = ".csv"; private static final String OVERWRITE_MODE = "overwrite"; + private transient BufferedWriter writer; @Override protected void openSource() throws IOException { - if(SFTP_PROTOCOL.equalsIgnoreCase(ftpConfig.getProtocol())) { - ftpHandler = new SftpHandler(); - } else { - ftpHandler = new FtpHandler(); - } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); ftpHandler.loginFtpServer(ftpConfig); } @@ -106,7 +108,7 @@ public boolean test(String file) { } String[] splits = fileName.split("\\."); - if (splits.length == 3) { + if (splits.length == FILE_NAME_PART_SIZE) { return Integer.parseInt(splits[2]) <= fileIndex; } @@ -125,11 +127,16 @@ public boolean test(String file) { protected void nextBlock(){ super.nextBlock(); - if (os != null){ + if (writer != null){ return; } - - os = ftpHandler.getOutputStream(tmpPath + SP + currentBlockFileName); + String path = tmpPath + SP + currentBlockFileName; + try { + writer = new BufferedWriter(new OutputStreamWriter(ftpHandler.getOutputStream(path), ftpConfig.getEncoding())); + } catch (UnsupportedEncodingException e) { + LOG.error("exception when create BufferedWriter, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(e); + } blockIndex++; } @@ -156,29 +163,42 @@ public void moveTemporaryDataBlockFileToDirectory(){ @Override public void writeSingleRecordToFile(Row row) throws WriteRecordException { - if(os == null){ - nextBlock(); - } - - String line = StringUtil.row2string(row, columnTypes, ftpConfig.getFieldDelimiter(), columnNames); try { - byte[] bytes = line.getBytes(ftpConfig.getEncoding()); - this.os.write(bytes); - this.os.write(NEWLINE); - this.os.flush(); + if(writer == null){ + nextBlock(); + } + + String line = StringUtil.row2string(row, columnTypes, ftpConfig.getFieldDelimiter()); + this.writer.write(line); + this.writer.write(NEWLINE); if(restoreConfig.isRestore()){ lastRow = row; rowsOfCurrentBlock++; } - } catch(Exception ex) { - throw new WriteRecordException(ex.getMessage(), ex); + } catch(Exception e) { + LOG.error("error happened when write single record to file, row = {}, columnTypes = {}, e = {}", row, GsonUtil.GSON.toJson(columnTypes), ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(e.getMessage(), e); } } @Override - protected void createFinishedTag() throws IOException { - LOG.info("Subtask [{}] finished, create dir {}", taskNumber, finishedPath); + protected void createFinishedTag() { + LOG.info("SubTask [{}] finished, create dir {}", taskNumber, finishedPath); + String path = outputFilePath + SP + FINISHED_SUBDIR; + if(taskNumber == 0){ + ftpHandler.mkDirRecursive(path); + } + final int maxRetryTime = 15; + int i = 0; + try { + while(!(ftpHandler.isDirExist(path) || i > maxRetryTime)){ + i++; + TimeUnit.MILLISECONDS.sleep(10); + } + }catch (Exception e){ + LOG.error("exception when createFinishedTag, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); + } ftpHandler.mkDirRecursive(finishedPath); } @@ -238,7 +258,7 @@ protected void waitForAllTasksToFinish(){ if (i == maxRetryTime) { ftpHandler.deleteAllFilesInDir(finishedPath, null); - throw new RuntimeException("timeout when gathering finish tags for each subtasks"); + throw new RuntimeException("timeout when gathering finish tags for each subTasks"); } } @@ -246,7 +266,7 @@ protected void waitForAllTasksToFinish(){ protected void coverageData(){ boolean cleanPath = restoreConfig.isRestore() && OVERWRITE_MODE.equalsIgnoreCase(ftpConfig.getWriteMode()) && !SP.equals(ftpConfig.getPath()); if(cleanPath){ - ftpHandler.deleteAllFilesInDir(ftpConfig.getPath(), Arrays.asList(tmpPath)); + ftpHandler.deleteAllFilesInDir(ftpConfig.getPath(), Collections.singletonList(tmpPath)); } } @@ -286,15 +306,15 @@ protected void moveAllTemporaryDataFileToDirectory() throws IOException { @Override protected void closeSource() throws IOException { - if (os != null){ - os.flush(); - os.close(); - os = null; + if (writer != null){ + writer.flush(); + writer.close(); + writer = null; } } @Override - protected void clearTemporaryDataFiles() throws IOException { + protected void clearTemporaryDataFiles() { ftpHandler.deleteAllFilesInDir(tmpPath, null); LOG.info("Delete .data dir:{}", tmpPath); @@ -307,6 +327,12 @@ public void flushDataInternal() throws IOException { closeSource(); } + @Override + public void closeInternal() throws IOException { + closeSource(); + super.closeInternal(); + } + @Override public float getDeviation() { return 1.0F; @@ -316,4 +342,9 @@ public float getDeviation() { protected String getExtension() { return ".csv"; } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("FtpWriter"); + } } diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java index dee19b9e1c..dc7bcae4f9 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java @@ -19,9 +19,7 @@ package com.dtstack.flinkx.ftp.writer; import com.dtstack.flinkx.ftp.FtpConfig; -import com.dtstack.flinkx.ftp.FtpConfigConstants; import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; -import org.apache.commons.lang.StringUtils; import java.util.List; /** @@ -53,7 +51,7 @@ public void setFtpConfig(FtpConfig ftpConfig){ @Override protected void checkFormat() { - + notSupportBatchWrite("FtpWriter"); } } diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java index edfb666f4a..9588e3a3c0 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.*; +import static com.dtstack.flinkx.ftp.FtpConfigConstants.DEFAULT_FIELD_DELIMITER; /** * The Writer Plugin of Ftp diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 3082049696..9a7903f7fc 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -62,8 +62,12 @@ shade.gbasereader.io.netty - com.google - shade.gbasereader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index 06b22d6f4c..b2b05b62b9 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.gbase.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; @@ -49,7 +50,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -63,6 +66,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { fetchSize = Integer.MIN_VALUE; querySql = buildQuerySql(inputSplit); + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index da6f21b4a2..d28e0af276 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -62,8 +62,12 @@ shade.gbasewriter.io.netty - com.google - shade.gbasewriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-greenplum/flinkx-greenplum-core/pom.xml b/flinkx-greenplum/flinkx-greenplum-core/pom.xml new file mode 100644 index 0000000000..3a79da783d --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/pom.xml @@ -0,0 +1,39 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-core + + + com.dtstack.flinkx + flinkx-postgresql-core + 1.6 + + + com.pivotal + greenplum-jdbc + 5.1.4 + + + org.postgresql + postgresql + 42.2.2 + + + + + + HandChina RDC + HandChina RDC + http://nexus.saas.hand-china.com/content/repositories/rdc/ + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java new file mode 100644 index 0000000000..95dcb1b887 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum; + +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.postgresql.PostgresqlDatabaseMeta; + +import java.util.List; +import java.util.Map; + +/** + * The class of Greenplum database prototype + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumDatabaseMeta extends PostgresqlDatabaseMeta { + + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + throw new UnsupportedOperationException("Greenplum not support update mode"); + } + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.Greenplum; + } + + @Override + public String getDriverClass() { + return "com.pivotal.jdbc.GreenplumDriver"; + } +} diff --git a/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java new file mode 100644 index 0000000000..cb811c6ef0 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum; + +/** + * when use insert mode, use org.postgresql.Driver + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumDatabaseMetaInsert extends GreenplumDatabaseMeta{ + @Override + public String getDriverClass() { + return "org.postgresql.Driver"; + } +} diff --git a/flinkx-greenplum/flinkx-greenplum-reader/pom.xml b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml new file mode 100644 index 0000000000..0fb695ce58 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml @@ -0,0 +1,112 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-reader + + + + com.dtstack.flinkx + flinkx-greenplum-core + 1.6 + + + com.dtstack.flinkx + flinkx-postgresql-reader + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.greenplumreader.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java new file mode 100644 index 0000000000..c1fba4eb0b --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.format; + +import com.dtstack.flinkx.postgresql.format.PostgresqlInputFormat; + +/**Simple inheritance of PostgresqlInputFormat + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumInputFormat extends PostgresqlInputFormat { +} diff --git a/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java new file mode 100644 index 0000000000..09f547469f --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.greenplum.format.GreenplumInputFormat; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; +import com.dtstack.flinkx.postgresql.reader.PostgresqlQuerySqlBuilder; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.datareader.QuerySqlBuilder; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import com.dtstack.flinkx.greenplum.GreenplumDatabaseMeta; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +/** + * The reader plugin for Greenplum database + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumReader extends JdbcDataReader { + public GreenplumReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + setDatabaseInterface(new GreenplumDatabaseMeta()); + setTypeConverterInterface(new PostgresqlTypeConverter()); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return new JdbcInputFormatBuilder(new GreenplumInputFormat()); + } + + @Override + public DataStream readData() { + JdbcInputFormatBuilder builder = new JdbcInputFormatBuilder(new GreenplumInputFormat()); + builder.setDriverName(databaseInterface.getDriverClass()); + builder.setDbUrl(dbUrl); + builder.setUsername(username); + builder.setPassword(password); + builder.setBytes(bytes); + builder.setMonitorUrls(monitorUrls); + builder.setTable(table); + builder.setDatabaseInterface(databaseInterface); + builder.setTypeConverter(typeConverter); + builder.setMetaColumn(metaColumns); + builder.setFetchSize(fetchSize == 0 ? databaseInterface.getFetchSize() : fetchSize); + builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); + builder.setIncrementConfig(incrementConfig); + builder.setSplitKey(splitKey); + builder.setNumPartitions(numPartitions); + builder.setCustomSql(customSql); + builder.setRestoreConfig(restoreConfig); + builder.setHadoopConfig(hadoopConfig); + builder.setTestConfig(testConfig); + + QuerySqlBuilder sqlBuilder = new PostgresqlQuerySqlBuilder(this); + builder.setQuery(sqlBuilder.buildSql()); + + BaseRichInputFormat format = builder.finish(); + return createInput(format, (databaseInterface.getDatabaseType() + "reader").toLowerCase()); + } + +} diff --git a/flinkx-greenplum/flinkx-greenplum-writer/pom.xml b/flinkx-greenplum/flinkx-greenplum-writer/pom.xml new file mode 100644 index 0000000000..db6738d9e5 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/pom.xml @@ -0,0 +1,110 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-writer + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + com.dtstack.flinkx + flinkx-greenplum-core + 1.6 + + + com.dtstack.flinkx + flinkx-postgresql-writer + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.greenplumwriter.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java new file mode 100644 index 0000000000..a1d02138d2 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.format; + +import com.dtstack.flinkx.postgresql.format.PostgresqlOutputFormat; + +/**Simple inheritance of PostgresqlOutputFormat + * + * Date: 2020/6/4 + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumOutputFormat extends PostgresqlOutputFormat { +} diff --git a/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java new file mode 100644 index 0000000000..8d3267968b --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.greenplum.format.GreenplumOutputFormat; +import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.greenplum.GreenplumDatabaseMetaInsert; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +/** + * The writer plugin for Greenplum database + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumWriter extends JdbcDataWriter { + + public static final String INSERT_SQL_MODE_TYPE = "copy"; + public static final String DATABASE_NAME = ";DatabaseName="; + public static final String JDBC_POSTGRESQL_PREFIX = "jdbc:postgresql"; + public static final String JDBC_GREENPLUM_PREFIX = "jdbc:pivotal:greenplum"; + + public GreenplumWriter(DataTransferConfig config) { + super(config); + //统一固定为copy模式 + insertSqlMode = INSERT_SQL_MODE_TYPE; + dbUrl = changeToPostgresqlUrl(); + setDatabaseInterface(new GreenplumDatabaseMetaInsert()); + setTypeConverterInterface(new PostgresqlTypeConverter()); + } + + String changeToPostgresqlUrl(){ + dbUrl = StringUtils.replaceOnce(dbUrl, JDBC_GREENPLUM_PREFIX, JDBC_POSTGRESQL_PREFIX); + dbUrl = StringUtils.replaceOnce(dbUrl, DATABASE_NAME, "/"); + return dbUrl; + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + GreenplumOutputFormat greenplumOutputFormat = new GreenplumOutputFormat(); + JdbcOutputFormatBuilder builder = new JdbcOutputFormatBuilder(greenplumOutputFormat); + builder.setDriverName(databaseInterface.getDriverClass()); + builder.setDbUrl(dbUrl); + builder.setUsername(username); + builder.setPassword(password); + builder.setBatchInterval(batchSize); + builder.setMonitorUrls(monitorUrls); + builder.setPreSql(preSql); + builder.setPostSql(postSql); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setDatabaseInterface(databaseInterface); + builder.setMode(mode); + builder.setTable(table); + builder.setColumn(column); + builder.setFullColumn(fullColumn); + builder.setUpdateKey(updateKey); + builder.setTypeConverter(typeConverter); + builder.setRestoreConfig(restoreConfig); + builder.setInsertSqlMode(insertSqlMode); + + DataStreamSink dataStreamSink = createOutput(dataSet, builder.finish()); + String sinkName = (databaseInterface.getDatabaseType() + "writer").toLowerCase(); + dataStreamSink.name(sinkName); + return dataStreamSink; + } +} diff --git a/flinkx-greenplum/pom.xml b/flinkx-greenplum/pom.xml new file mode 100644 index 0000000000..5c3fb85de9 --- /dev/null +++ b/flinkx-greenplum/pom.xml @@ -0,0 +1,36 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum + pom + + flinkx-greenplum-reader + flinkx-greenplum-core + flinkx-greenplum-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java index d83d83b3ee..d7f0d61ba6 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java @@ -44,7 +44,7 @@ public class HbaseConfigConstants { public static final String DEFAULT_NULL_MODE = "skip"; - public static final long DEFAULT_WRITE_BUFFER_SIZE = 8 * 1024 * 1024; + public static final long DEFAULT_WRITE_BUFFER_SIZE = 8 * 1024 * 1024L; public static final boolean DEFAULT_WAL_FLAG = false; diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java index 0bce2c0c69..3c88f828f5 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java @@ -27,15 +27,17 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.security.PrivilegedAction; -import java.util.Arrays; -import java.util.List; import java.util.Map; /** @@ -51,21 +53,9 @@ public class HbaseHelper { private final static String AUTHENTICATION_TYPE = "Kerberos"; private final static String KEY_HBASE_SECURITY_AUTHENTICATION = "hbase.security.authentication"; private final static String KEY_HBASE_SECURITY_AUTHORIZATION = "hbase.security.authorization"; - private final static String KEY_HBASE_MASTER_KERBEROS_PRINCIPAL = "hbase.master.kerberos.principal"; - private final static String KEY_HBASE_MASTER_KEYTAB_FILE = "hbase.master.keytab.file"; - private final static String KEY_HBASE_REGIONSERVER_KEYTAB_FILE = "hbase.regionserver.keytab.file"; - private final static String KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL = "hbase.regionserver.kerberos.principal"; - - private static List KEYS_KERBEROS_REQUIRED = Arrays.asList( - KEY_HBASE_SECURITY_AUTHENTICATION, - KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, - KEY_HBASE_MASTER_KEYTAB_FILE, - KEY_HBASE_REGIONSERVER_KEYTAB_FILE, - KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL - ); public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map hbaseConfigMap) { - Validate.isTrue(hbaseConfigMap != null && hbaseConfigMap.size() !=0, "hbaseConfig不能为空Map结构!"); + Validate.isTrue(MapUtils.isEmpty(hbaseConfigMap), "hbaseConfig不能为空Map结构!"); if(openKerberos(hbaseConfigMap)){ return getConnectionWithKerberos(hbaseConfigMap); @@ -81,43 +71,43 @@ public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map hbaseConfigMap){ - for (String key : KEYS_KERBEROS_REQUIRED) { - if(StringUtils.isEmpty(MapUtils.getString(hbaseConfigMap, key))){ - throw new IllegalArgumentException(String.format("Must provide [%s] when authentication is Kerberos", key)); - } + try { + UserGroupInformation ugi = getUgi(hbaseConfigMap); + return ugi.doAs(new PrivilegedAction() { + @Override + public Connection run() { + try { + Configuration hConfiguration = getConfig(hbaseConfigMap); + return ConnectionFactory.createConnection(hConfiguration); + } catch (IOException e) { + LOG.error("Get connection fail with config:{}", hbaseConfigMap); + throw new RuntimeException(e); + } + } + }); + } catch (Exception e){ + throw new RuntimeException("Login kerberos error", e); } + } + public static UserGroupInformation getUgi(Map hbaseConfigMap) throws IOException{ String keytabFileName = KerberosUtil.getPrincipalFileName(hbaseConfigMap); keytabFileName = KerberosUtil.loadFile(hbaseConfigMap, keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); + String principal = KerberosUtil.getPrincipal(hbaseConfigMap, keytabFileName); KerberosUtil.loadKrb5Conf(hbaseConfigMap); Configuration conf = FileSystemUtil.getConfiguration(hbaseConfigMap, null); - UserGroupInformation ugi; - try { - ugi = KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); - } catch (Exception e){ - throw new RuntimeException("Login kerberos error", e); - } - - return ugi.doAs(new PrivilegedAction() { - @Override - public Connection run() { - try { - Configuration hConfiguration = getConfig(hbaseConfigMap); - return ConnectionFactory.createConnection(hConfiguration); - } catch (IOException e) { - LOG.error("Get connection fail with config:{}", hbaseConfigMap); - throw new RuntimeException(e); - } - } - }); + return KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); } public static Configuration getConfig(Map hbaseConfigMap){ Configuration hConfiguration = HBaseConfiguration.create(); + if (MapUtils.isEmpty(hbaseConfigMap)) { + return hConfiguration; + } + for (Map.Entry entry : hbaseConfigMap.entrySet()) { if(entry.getValue() != null && !(entry.getValue() instanceof Map)){ hConfiguration.set(entry.getKey(), entry.getValue().toString()); diff --git a/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java b/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java deleted file mode 100644 index c1a67b5750..0000000000 --- a/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.hbase.test; - -import com.dtstack.flinkx.hbase.HbaseHelper; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; - -import java.util.HashMap; -import java.util.Map; - -/** - * @author jiangbo - * @date 2019/8/29 - */ -public class HbaseHelperTest { - - public static void main(String[] args) throws Exception{ - Map sftpConf = new HashMap<>(); - sftpConf.put("host", "172.16.10.79"); - sftpConf.put("port", "22"); - sftpConf.put("username", "root"); - sftpConf.put("password", "abc123"); - - Map hbaseConfig = new HashMap<>(); - hbaseConfig.put("hbase.security.authorization", "true"); - hbaseConfig.put("hbase.security.authentication", "kerberos"); - hbaseConfig.put("hbase.master.kerberos.principal", "hbase/cdh01@HADOOP.COM"); - hbaseConfig.put("hbase.master.keytab.file", "D:\\cdh_cluster\\cdh2\\hbase.keytab"); - hbaseConfig.put("hbase.regionserver.kerberos.principal", "hbase/cdh01@HADOOP.COM"); - hbaseConfig.put("hbase.regionserver.keytab.file", "D:\\cdh_cluster\\cdh2\\hbase.keytab"); - hbaseConfig.put("java.security.krb5.conf", "D:\\cdh_cluster\\cdh2\\krb5.conf"); - hbaseConfig.put("useLocalFile", "true"); -// hbaseConfig.put("sftpConf", sftpConf); -// hbaseConfig.put("remoteDir", "/home/sftp/keytab/jiangbo"); - -// hbaseConfig.put("hbase.zookeeper.quorum", "cdh01:2181,cdh02:2181,cdh03:2181"); - hbaseConfig.put("hbase.zookeeper.quorum", "172.16.10.201:2181"); - hbaseConfig.put("hbase.rpc.timeout", "60000"); - hbaseConfig.put("ipc.socket.timeout", "20000"); - hbaseConfig.put("hbase.client.retries.number", "3"); - hbaseConfig.put("hbase.client.pause", "100"); - hbaseConfig.put("zookeeper.recovery.retry", "3"); - - Connection connection = HbaseHelper.getHbaseConnection(hbaseConfig); - Table table = connection.getTable(TableName.valueOf("tb1")); - - ResultScanner rs = table.getScanner(new Scan()); - Result result = rs.next(); - if(result != null){ - System.out.println(result.getRow()); - } - - HbaseHelper.getRegionLocator(connection, "tb1"); - - connection.close(); - } -} diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index f77a9613ca..5cdc499262 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -65,6 +65,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 495a89c4e6..f17383ca87 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -29,16 +29,24 @@ import org.apache.flink.types.Row; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import java.io.IOException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; import java.util.Map; +import com.google.common.collect.Maps; +import org.apache.hadoop.security.UserGroupInformation; /** @@ -87,7 +95,17 @@ public void openInputFormat() throws IOException { @Override public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { try (Connection connection = HbaseHelper.getHbaseConnection(hbaseConfig)) { - return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + if(HbaseHelper.openKerberos(hbaseConfig)) { + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + return ugi.doAs(new PrivilegedAction() { + @Override + public HbaseInputSplit[] run() { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } + }); + } else { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } } } @@ -132,9 +150,10 @@ private List doSplit(byte[] startRowkeyByte, // 当前的region为最后一个region // 如果最后一个region的start Key大于用户指定的userEndKey,则最后一个region,应该不包含在内 // 注意如果用户指定userEndKey为"",则此判断应该不成立。userEndKey为""表示取得最大的region - if (Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0 + boolean isSkip = Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0 && (endRowkeyByte.length != 0 && (Bytes.compareTo( - regionStartKey, endRowkeyByte) > 0))) { + regionStartKey, endRowkeyByte) > 0)); + if (isSkip) { continue; } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java index 04c5150990..b1d4186132 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java @@ -26,6 +26,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -39,6 +41,8 @@ */ public class HbaseReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(HbaseReader.class); + protected List columnName; protected List columnType; protected List columnValue; @@ -82,7 +86,8 @@ public HbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { columnValue.add((String) sm.get("value")); columnFormat.add((String) sm.get("format")); } - System.out.println("init column finished"); + + LOG.info("init column finished"); } else{ throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java b/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java deleted file mode 100644 index 47f153621b..0000000000 --- a/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.dtstack.flinkx.hbase.reader; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionLocator; -import java.io.IOException; - -/** - * Created by softfly on 17/7/25. - */ -public class HbaseSplitDemo { - - private static void split() { - - } - - public static void main(String[] args) throws IOException { - - org.apache.hadoop.conf.Configuration conf = HBaseConfiguration.create(); - conf.set("hbase.zookeeper.property.clientPort", "2181"); - conf.set("hbase.zookeeper.quorum", "172.16.1.151" ); - conf.set("zookeeper.znode.parent", "/hbase2"); - - Connection conn = ConnectionFactory.createConnection(conf); - //Table table = conn.getTable(TableName.valueOf("tb2")); - - RegionLocator regionLocator = conn.getRegionLocator(TableName.valueOf("tb2")); - regionLocator.getStartEndKeys(); - - } - -} diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index d406061b54..d5dd35eba4 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -54,6 +54,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java index a3e75a9caa..d4db300d08 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.hbase.writer; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.enums.ColumnType; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.hbase.HbaseHelper; @@ -27,6 +28,7 @@ import com.dtstack.flinkx.util.DateUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.Validate; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; @@ -34,10 +36,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -93,8 +97,40 @@ public class HbaseOutputFormat extends BaseRichOutputFormat { private transient ThreadLocal timeMillisecondFormatThreadLocal; + private boolean openKerberos = false; + @Override public void configure(Configuration parameters) { + } + + @Override + public void openInternal(int taskNumber, int numTasks) throws IOException { + openKerberos = HbaseHelper.openKerberos(hbaseConfig); + if (openKerberos) { + sleepRandomTime(); + + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + openConnection(); + return null; + } + }); + } else { + openConnection(); + } + } + + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long)(10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + + public void openConnection() { LOG.info("HbaseOutputFormat configure start"); nameMaps = Maps.newConcurrentMap(); nameByteMaps = Maps.newConcurrentMap(); @@ -103,9 +139,9 @@ public void configure(Configuration parameters) { Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); try { - connection = HbaseHelper.getHbaseConnection(hbaseConfig); - org.apache.hadoop.conf.Configuration hConfiguration = HbaseHelper.getConfig(hbaseConfig); + connection = ConnectionFactory.createConnection(hConfiguration); + bufferedMutator = connection.getBufferedMutator( new BufferedMutatorParams(TableName.valueOf(tableName)) .pool(HTable.getDefaultExecutor(hConfiguration)) @@ -129,11 +165,6 @@ public void configure(Configuration parameters) { LOG.info("HbaseOutputFormat configure end"); } - @Override - public void openInternal(int taskNumber, int numTasks) throws IOException { - - } - @Override public void writeSingleRecordInternal(Row record) throws WriteRecordException { int i = 0; @@ -159,17 +190,19 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { String name = columnNames.get(i); String[] cfAndQualifier = nameMaps.get(name); byte[][] cfAndQualifierBytes = nameByteMaps.get(name); - if(cfAndQualifier == null || cfAndQualifierBytes==null){ - String promptInfo = "Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name; + if(cfAndQualifier == null || cfAndQualifierBytes == null){ cfAndQualifier = name.split(":"); - Validate.isTrue(cfAndQualifier != null && cfAndQualifier.length == 2 - && org.apache.commons.lang3.StringUtils.isNotBlank(cfAndQualifier[0]) - && org.apache.commons.lang3.StringUtils.isNotBlank(cfAndQualifier[1]), promptInfo); - nameMaps.put(name,cfAndQualifier); - cfAndQualifierBytes = new byte[2][]; - cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); - cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); - nameByteMaps.put(name,cfAndQualifierBytes); + if(cfAndQualifier.length == 2 + && StringUtils.isNotBlank(cfAndQualifier[0]) + && StringUtils.isNotBlank(cfAndQualifier[1])){ + nameMaps.put(name,cfAndQualifier); + cfAndQualifierBytes = new byte[2][]; + cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); + cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); + nameByteMaps.put(name,cfAndQualifierBytes); + } else { + throw new IllegalArgumentException("Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name); + } } ColumnType columnType = ColumnType.getType(type); @@ -180,8 +213,6 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { cfAndQualifierBytes[0], cfAndQualifierBytes[1], columnBytes); - }else{ - continue; } } @@ -195,20 +226,21 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { } private SimpleDateFormat getSimpleDateFormat(String sign){ - SimpleDateFormat format = null; - if("sss".equalsIgnoreCase(sign)){ + SimpleDateFormat format; + if(ConstantValue.TIME_SECOND_SUFFIX.equals(sign)){ format = timeSecondFormatThreadLocal.get(); if(format == null){ format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); timeSecondFormatThreadLocal.set(format); } - }else if("SSS".equalsIgnoreCase(sign)){ + } else { format = timeMillisecondFormatThreadLocal.get(); if(format == null){ format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss SSS"); timeMillisecondFormatThreadLocal.set(format); } } + return format; } @@ -219,7 +251,7 @@ protected String recordConvertDetailErrorMessage(int pos, Row row) { @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new IllegalArgumentException(); + notSupportBatchWrite("HbaseWriter"); } private byte[] getRowkey(Row record) throws Exception{ @@ -249,8 +281,8 @@ public long getVersion(Row record){ if(record.getField(index) == null){ throw new IllegalArgumentException("null verison column!"); } - SimpleDateFormat dfSeconds = getSimpleDateFormat("sss"); - SimpleDateFormat dfMs = getSimpleDateFormat("SSS"); + SimpleDateFormat dfSeconds = getSimpleDateFormat(ConstantValue.TIME_SECOND_SUFFIX); + SimpleDateFormat dfMs = getSimpleDateFormat(ConstantValue.TIME_MILLISECOND_SUFFIX); Object column = record.getField(index); if(column instanceof Long){ Long longValue = (Long) column; @@ -370,7 +402,7 @@ private byte[] intToBytes(Object column) { if(column instanceof Integer) { intValue = (Integer) column; } else if(column instanceof Long) { - intValue = Integer.valueOf(((Long)column).intValue()); + intValue = ((Long) column).intValue(); } else if(column instanceof Double) { intValue = ((Double) column).intValue(); } else if(column instanceof Float) { @@ -378,7 +410,7 @@ private byte[] intToBytes(Object column) { } else if(column instanceof Short) { intValue = ((Short) column).intValue(); } else if(column instanceof Boolean) { - intValue = ((Boolean) column).booleanValue() ? 1 : 0; + intValue = (Boolean) column ? 1 : 0; } else if(column instanceof String) { intValue = Integer.valueOf((String) column); } else { @@ -401,7 +433,7 @@ private byte[] longToBytes(Object column) { } else if(column instanceof Short) { longValue = ((Short) column).longValue(); } else if(column instanceof Boolean) { - longValue = ((Boolean) column).booleanValue() ? 1L : 0L; + longValue = (Boolean) column ? 1L : 0L; } else if(column instanceof String) { longValue = Long.valueOf((String) column); }else if (column instanceof Timestamp){ @@ -426,7 +458,7 @@ private byte[] doubleToBytes(Object column) { } else if(column instanceof Short) { doubleValue = ((Short) column).doubleValue(); } else if(column instanceof Boolean) { - doubleValue = ((Boolean) column).booleanValue() ? 1.0 : 0.0; + doubleValue = (Boolean) column ? 1.0 : 0.0; } else if(column instanceof String) { doubleValue = Double.valueOf((String) column); } else { @@ -449,7 +481,7 @@ private byte[] floatToBytes(Object column) { } else if(column instanceof Short) { floatValue = ((Short) column).floatValue(); } else if(column instanceof Boolean) { - floatValue = ((Boolean) column).booleanValue() ? 1.0f : 0.0f; + floatValue = (Boolean) column ? 1.0f : 0.0f; } else if(column instanceof String) { floatValue = Float.valueOf((String) column); } else { @@ -472,7 +504,7 @@ private byte[] shortToBytes(Object column) { } else if(column instanceof Short) { shortValue = (Short) column; } else if(column instanceof Boolean) { - shortValue = ((Boolean) column).booleanValue() ? (short) 1 : (short) 0 ; + shortValue = (Boolean) column ? (short) 1 : (short) 0 ; } else if(column instanceof String) { shortValue = Short.valueOf((String) column); } else { @@ -484,15 +516,15 @@ private byte[] shortToBytes(Object column) { private byte[] boolToBytes(Object column) { Boolean booleanValue = null; if(column instanceof Integer) { - booleanValue = (Integer)column == 0 ? false : true; + booleanValue = (Integer) column != 0; } else if(column instanceof Long) { - booleanValue = (Long) column == 0L ? false : true; + booleanValue = (Long) column != 0L; } else if(column instanceof Double) { - booleanValue = (Double) column == 0.0 ? false : true; + booleanValue = new Double(0.0).compareTo((Double) column) != 0; } else if(column instanceof Float) { - booleanValue = (Float) column == 0.0f ? false : true; + booleanValue = new Float(0.0f).compareTo((Float) column) != 0; } else if(column instanceof Short) { - booleanValue = (Short) column == 0 ? false : true; + booleanValue = (Short) column != 0; } else if(column instanceof Boolean) { booleanValue = (Boolean) column; } else if(column instanceof String) { diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java index 4da4c10754..ec96e04c17 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java @@ -110,5 +110,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("HbaseWriter"); } } diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java index 42091f62d3..3cb5bce6ce 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java @@ -31,8 +31,23 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hbase.HbaseConfigConstants.*; -import static com.dtstack.flinkx.hbase.HbaseConfigKeys.*; + +import static com.dtstack.flinkx.hbase.HbaseConfigConstants.DEFAULT_WAL_FLAG; +import static com.dtstack.flinkx.hbase.HbaseConfigConstants.DEFAULT_WRITE_BUFFER_SIZE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_COLUMN_NAME; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_COLUMN_TYPE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ENCODING; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_HBASE_CONFIG; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_NULL_MODE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN_INDEX; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN_VALUE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN_INDEX; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN_VALUE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_WAL_FLAG; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_WRITE_BUFFER_SIZE; /** * The Writer plugin of HBase diff --git a/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java b/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java index dbf12dfa2e..75143994f5 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java @@ -18,7 +18,10 @@ package com.dtstack.flinkx.hbase.writer; -import com.dtstack.flinkx.hbase.writer.function.*; +import com.dtstack.flinkx.hbase.writer.function.FunctionParser; +import com.dtstack.flinkx.hbase.writer.function.FunctionTree; +import com.dtstack.flinkx.hbase.writer.function.Md5Function; +import com.dtstack.flinkx.hbase.writer.function.StringFunction; import org.junit.Assert; import org.junit.Test; diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java index 913c0fd41c..2f0c7ec101 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java @@ -19,7 +19,6 @@ package com.dtstack.flinkx.hdfs; import org.apache.commons.lang.StringUtils; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; /** * @author jiangbo diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java index fc6ced4a25..a4b44360e3 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java @@ -60,4 +60,6 @@ public class HdfsConfigKeys { public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + public static final String KEY_ENABLE_DICTIONARY = "enableDictionary"; + } diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index b678812f68..0acaa924b8 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -19,15 +19,20 @@ package com.dtstack.flinkx.hdfs; import com.dtstack.flinkx.enums.ColumnType; -import com.dtstack.flinkx.util.DateUtil; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.io.*; +import org.apache.hadoop.io.ByteWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.parquet.io.api.Binary; -import java.sql.Date; -import java.text.SimpleDateFormat; /** * Utilities for HdfsReader and HdfsWriter @@ -39,62 +44,13 @@ public class HdfsUtil { public static final String NULL_VALUE = "\\N"; - public static Object string2col(String str, String type, SimpleDateFormat customDateFormat) { - if (str == null || str.length() == 0){ - return null; - } - - if(type == null){ - return str; - } + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; - ColumnType columnType = ColumnType.fromString(type.toUpperCase()); - Object ret; - switch(columnType) { - case TINYINT: - ret = Byte.valueOf(str.trim()); - break; - case SMALLINT: - ret = Short.valueOf(str.trim()); - break; - case INT: - ret = Integer.valueOf(str.trim()); - break; - case BIGINT: - ret = Long.valueOf(str.trim()); - break; - case FLOAT: - ret = Float.valueOf(str.trim()); - break; - case DOUBLE: - case DECIMAL: - ret = Double.valueOf(str.trim()); - break; - case STRING: - case VARCHAR: - case CHAR: - if(customDateFormat != null){ - ret = DateUtil.columnToDate(str,customDateFormat); - ret = DateUtil.timestampToString((Date)ret); - } else { - ret = str; - } - break; - case BOOLEAN: - ret = Boolean.valueOf(str.trim().toLowerCase()); - break; - case DATE: - ret = DateUtil.columnToDate(str,customDateFormat); - break; - case TIMESTAMP: - ret = DateUtil.columnToTimestamp(str,customDateFormat); - break; - default: - throw new IllegalArgumentException("Unsupported field type:" + type); - } + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; - return ret; - } + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; public static Object getWritableValue(Object writable) { Class clz = writable.getClass(); @@ -166,4 +122,78 @@ public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) return objectInspector; } + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision){ + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static byte[] longToByteArray(long data){ + long nano = data * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return dst; + } + + private static byte[] getBytes(long i) { + byte[] bytes=new byte[8]; + bytes[0]=(byte)((i >> 56) & 0xFF); + bytes[1]=(byte)((i >> 48) & 0xFF); + bytes[2]=(byte)((i >> 40) & 0xFF); + bytes[3]=(byte)((i >> 32) & 0xFF); + bytes[4]=(byte)((i >> 24) & 0xFF); + bytes[5]=(byte)((i >> 16) & 0xFF); + bytes[6]=(byte)((i >> 8) & 0xFF); + bytes[7]=(byte)(i & 0xFF); + return bytes; + } + + /** + * @param bytes + */ + private static void flip(byte[] bytes) { + for(int i=0,j=bytes.length-1;i + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java index f6d802cf1b..e5e6a8ee37 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java @@ -21,10 +21,15 @@ import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.security.UserGroupInformation; +import java.io.File; import java.io.IOException; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -36,6 +41,8 @@ */ public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { + private static final String PARTITION_SPLIT_CHAR = "="; + protected Map hadoopConfig; protected List metaColumns; @@ -61,14 +68,25 @@ public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { protected Object value; - protected boolean isFileEmpty = false; - protected String filterRegex; + protected transient UserGroupInformation ugi; + + protected boolean openKerberos; + + protected String currentPartition; + + protected transient FileSystem fs; + @Override public void openInputFormat() throws IOException { super.openInputFormat(); conf = buildConfig(); + + openKerberos = FileSystemUtil.isOpenKerberos(hadoopConfig); + if (openKerberos) { + ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + } } protected JobConf buildConfig() { @@ -81,7 +99,7 @@ protected JobConf buildConfig() { @Override public boolean reachedEnd() throws IOException { - return isFileEmpty || !recordReader.next(key, value); + return !recordReader.next(key, value); } @Override @@ -91,4 +109,27 @@ public void closeInternal() throws IOException { } } + /** + * 从hdfs路径中获取当前分区信息 + * @param path hdfs路径 + */ + public void findCurrentPartition(Path path){ + Map map = new HashMap<>(16); + String pathStr = path.getParent().toString(); + int index; + while((index = pathStr.lastIndexOf(PARTITION_SPLIT_CHAR)) > 0){ + int i = pathStr.lastIndexOf(File.separator); + String name = pathStr.substring(i + 1, index); + String value = pathStr.substring(index + 1); + map.put(name, value); + pathStr = pathStr.substring(0, i); + } + + for (MetaColumn column : metaColumns) { + if(column.getPart()){ + column.setValue(map.get(column.getName())); + } + } + } + } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index a87d704310..6f48d4e404 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -22,21 +22,25 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.hive.ql.io.orc.*; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.ql.io.orc.OrcSplit; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reporter; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.*; +import java.security.PrivilegedAction; import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; /** * The subclass of HdfsInputFormat which handles orc files @@ -46,99 +50,102 @@ */ public class HdfsOrcInputFormat extends BaseHdfsInputFormat { - private transient OrcSerde orcSerde; - private transient String[] fullColNames; - private transient String[] fullColTypes; - private transient StructObjectInspector inspector; private transient List fields; private static final String COMPLEX_FIELD_TYPE_SYMBOL_REGEX = ".*(<|>|\\{|}|[|]).*"; + private AtomicBoolean isInit = new AtomicBoolean(false); + @Override - public void openInputFormat() throws IOException{ + public void openInputFormat() throws IOException { super.openInputFormat(); + inputFormat = new OrcInputFormat(); + } + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); - FileSystem fs; try { - fs = FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); + if (!isInit.get()) { + init(orcSplit.getPath()); + isInit.set(true); + } } catch (Exception e) { - throw new RuntimeException(e); + throw new IOException("初始化[inspector]出错", e); } - orcSerde = new OrcSerde(); - inputFormat = new OrcInputFormat(); - org.apache.hadoop.hive.ql.io.orc.Reader reader = null; - try { - OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(conf); - readerOptions.filesystem(fs); - - Path path = new Path(inputPath); - String typeStruct = null; - - if(fs.isDirectory(path)) { - RemoteIterator iterator = fs.listFiles(path, true); - while(iterator.hasNext()) { - FileStatus fileStatus = iterator.next(); - if(fileStatus.isFile() && fileStatus.getLen() > 49) { - Path subPath = fileStatus.getPath(); - reader = OrcFile.createReader(subPath, readerOptions); - typeStruct = reader.getObjectInspector().getTypeName(); - if(StringUtils.isNotEmpty(typeStruct)) { - break; - } + if (openKerberos) { + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + openOrcReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); } - } - if(reader == null) { - //throw new RuntimeException("orcfile dir is empty!"); - LOG.error("orc file {} is empty!", inputPath); - isFileEmpty = true; - return; + return null; } + }); + } else { + openOrcReader(inputSplit); + } + } - } else { - reader = OrcFile.createReader(path, readerOptions); - typeStruct = reader.getObjectInspector().getTypeName(); - } + private void openOrcReader(InputSplit inputSplit) throws IOException{ + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); + recordReader = inputFormat.getRecordReader(orcSplit, conf, Reporter.NULL); + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } - if (StringUtils.isEmpty(typeStruct)) { - throw new RuntimeException("can't retrieve type struct from " + path); - } + private void init(Path path) throws Exception { + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(conf); + readerOptions.filesystem(fs); + org.apache.hadoop.hive.ql.io.orc.Reader reader = OrcFile.createReader(path, readerOptions); + String typeStruct = reader.getObjectInspector().getTypeName(); - int startIndex = typeStruct.indexOf("<") + 1; - int endIndex = typeStruct.lastIndexOf(">"); - typeStruct = typeStruct.substring(startIndex, endIndex); + if (StringUtils.isEmpty(typeStruct)) { + throw new RuntimeException("can't retrieve type struct from " + path); + } - if(typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)){ - throw new RuntimeException("Field types such as array, map, and struct are not supported."); - } + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); - List cols = parseColumnAndType(typeStruct); + if(typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)){ + throw new RuntimeException("Field types such as array, map, and struct are not supported."); + } - fullColNames = new String[cols.size()]; - fullColTypes = new String[cols.size()]; + List cols = parseColumnAndType(typeStruct); - for(int i = 0; i < cols.size(); ++i) { - String[] temp = cols.get(i).split(":"); - fullColNames[i] = temp[0]; - fullColTypes[i] = temp[1]; - } + fullColNames = new String[cols.size()]; + String[] fullColTypes = new String[cols.size()]; - Properties p = new Properties(); - p.setProperty("columns", StringUtils.join(fullColNames, ",")); - p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); - orcSerde.initialize(conf, p); + for(int i = 0; i < cols.size(); ++i) { + String[] temp = cols.get(i).split(":"); + fullColNames[i] = temp[0]; + fullColTypes[i] = temp[1]; + } - this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + Properties p = new Properties(); + p.setProperty("columns", StringUtils.join(fullColNames, ",")); + p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); - } catch (Throwable e) { - throw new RuntimeException(e); - } + OrcSerde orcSerde = new OrcSerde(); + orcSerde.initialize(conf, p); + + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); } private List parseColumnAndType(String typeStruct){ @@ -175,12 +182,25 @@ private List parseColumnAndType(String typeStruct){ @Override public HdfsOrcInputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { - try { - FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); - } catch (Exception e) { - throw new IOException(e); + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + LOG.info("user:{}, ", ugi.getShortUserName()); + return ugi.doAs(new PrivilegedAction() { + @Override + public HdfsOrcInputSplit[] run() { + try { + return createOrcSplit(minNumSplits); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + return createOrcSplit(minNumSplits); } + } + private HdfsOrcInputSplit[] createOrcSplit(int minNumSplits) throws IOException{ JobConf jobConf = FileSystemUtil.getJobConf(hadoopConfig, defaultFs); org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, inputPath); org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(buildConfig(), HdfsPathFilter.class); @@ -204,24 +224,6 @@ public HdfsOrcInputSplit[] createInputSplitsInternal(int minNumSplits) throws IO return null; } - - @Override - public void openInternal(InputSplit inputSplit) throws IOException { - - if(isFileEmpty){ - return; - } - - numReadCounter = getRuntimeContext().getLongCounter("numRead"); - HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; - OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); - recordReader = inputFormat.getRecordReader(orcSplit, conf, Reporter.NULL); - key = recordReader.createKey(); - value = recordReader.createValue(); - fields = inspector.getAllStructFieldRefs(); - } - - @Override public Row nextRecordInternal(Row row) throws IOException { if(metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ @@ -239,17 +241,17 @@ public Row nextRecordInternal(Row row) throws IOException { MetaColumn metaColumn = metaColumns.get(i); Object val = null; - if(metaColumn.getIndex() != -1){ + if(metaColumn.getValue() != null){ + val = metaColumn.getValue(); + }else if(metaColumn.getIndex() != -1){ val = inspector.getStructFieldData(value, fields.get(metaColumn.getIndex())); if (val == null && metaColumn.getValue() != null){ val = metaColumn.getValue(); } - } else if(metaColumn.getValue() != null){ - val = metaColumn.getValue(); } if(val instanceof String || val instanceof org.apache.hadoop.io.Text){ - val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + val = StringUtil.string2col(String.valueOf(val), metaColumn.getType(), metaColumn.getTimeFormat()); } else if(val != null){ val = HdfsUtil.getWritableValue(val); } @@ -291,5 +293,4 @@ public int getSplitNumber() { return splitNumber; } } - } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java index 16fb276495..565c0a49b2 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java @@ -20,9 +20,9 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.enums.ColumnType; -import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.StringUtil; import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -43,6 +43,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Iterator; @@ -75,6 +76,8 @@ public class HdfsParquetInputFormat extends BaseHdfsInputFormat { private static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); + private static final int TIMESTAMP_BINARY_LENGTH = 12; + @Override protected void openInternal(InputSplit inputSplit) throws IOException { currentSplitFilePaths = ((HdfsParquetSplit)inputSplit).getPaths(); @@ -82,14 +85,33 @@ protected void openInternal(InputSplit inputSplit) throws IOException { private boolean nextLine() throws IOException{ if (currentFileReader == null && currentFileIndex <= currentSplitFilePaths.size()-1){ - nextFile(); + if (openKerberos) { + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + nextFile(); + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + nextFile(); + } } if (currentFileReader == null){ return false; } - currentLine = currentFileReader.read(); + if (openKerberos) { + currentLine = nextLineWithKerberos(); + } else { + currentLine = currentFileReader.read(); + } + if (fullColNames == null && currentLine != null){ fullColNames = new ArrayList<>(); fullColTypes = new ArrayList<>(); @@ -117,11 +139,24 @@ private boolean nextLine() throws IOException{ return currentLine != null; } + private Group nextLineWithKerberos() { + return ugi.doAs(new PrivilegedAction() { + @Override + public Group run() { + try { + return currentFileReader.read(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } + private void nextFile() throws IOException{ - String path = currentSplitFilePaths.get(currentFileIndex); - ParquetReader.Builder reader = ParquetReader.builder(new GroupReadSupport(), new Path(path)).withConf(conf); + Path path = new Path(currentSplitFilePaths.get(currentFileIndex)); + findCurrentPartition(path); + ParquetReader.Builder reader = ParquetReader.builder(new GroupReadSupport(), path).withConf(conf); currentFileReader = reader.build(); - currentFileIndex++; } @@ -139,22 +174,20 @@ protected Row nextRecordInternal(Row row) throws IOException { MetaColumn metaColumn = metaColumns.get(i); Object val = null; - if(metaColumn.getIndex() != -1){ - if (metaColumn.getIndex() < currentLine.getType().getFieldCount()) { - if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ - val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); - } + if (metaColumn.getValue() != null){ + val = metaColumn.getValue(); + }else if(metaColumn.getIndex() != -1){ + if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ + val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); + } - if (val == null && metaColumn.getValue() != null){ - val = metaColumn.getValue(); - } + if (val == null && metaColumn.getValue() != null){ + val = metaColumn.getValue(); } - } else if (metaColumn.getValue() != null){ - val = metaColumn.getValue(); } if(val instanceof String){ - val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + val = StringUtil.string2col(String.valueOf(val), metaColumn.getType(), metaColumn.getTimeFormat()); } row.setField(i,val); @@ -169,7 +202,7 @@ public boolean reachedEnd() throws IOException { return !nextLine(); } - private Object getData(Group currentLine,String type,int index){ + public Object getData(Group currentLine,String type,int index){ Object data = null; ColumnType columnType = ColumnType.fromString(type); @@ -302,13 +335,12 @@ private static List getAllPartitionPath(String tableLocation, FileSystem private String getTypeName(String method){ String typeName; switch (method){ + case "getBoolean": case "getInteger" : typeName = "int";break; case "getInt96" : typeName = "bigint";break; case "getFloat" : typeName = "float";break; case "getDouble" : typeName = "double";break; case "getBinary" : typeName = "binary";break; - case "getString" : typeName = "string";break; - case "getBoolean" : typeName = "int";break; default:typeName = "string"; } @@ -319,11 +351,11 @@ private String getTypeName(String method){ * @param timestampBinary * @return */ - private long getTimestampMillis(Binary timestampBinary) - { - if (timestampBinary.length() != 12) { + private long getTimestampMillis(Binary timestampBinary) { + if (timestampBinary.length() != TIMESTAMP_BINARY_LENGTH) { return 0; } + byte[] bytes = timestampBinary.getBytes(); long timeOfDayNanos = Longs.fromBytes(bytes[7], bytes[6], bytes[5], bytes[4], bytes[3], bytes[2], bytes[1], bytes[0]); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java index 5e3cea2a94..d4291741f0 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java @@ -43,9 +43,9 @@ public class HdfsReader extends BaseDataReader { protected String fileType; protected String path; protected String fieldDelimiter; - private List metaColumns; + protected List metaColumns; protected Map hadoopConfig; - private String filterRegex; + protected String filterRegex; public HdfsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java index 2ba2b1c363..f7d2dac2b7 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java @@ -22,25 +22,25 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; -import jodd.util.StringUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.io.output.ByteArrayOutputStream; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hadoop.security.UserGroupInformation; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.charset.Charset; -import java.nio.charset.UnsupportedCharsetException; -import java.util.Map; +import java.security.PrivilegedAction; /** * The subclass of HdfsInputFormat which handles text files @@ -59,12 +59,25 @@ public void openInputFormat() throws IOException { @Override public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { - try { - FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); - } catch (Exception e) { - throw new IOException(e); + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + LOG.info("user:{}, ", ugi.getShortUserName()); + return ugi.doAs(new PrivilegedAction() { + @Override + public InputSplit[] run() { + try { + return createTextSplit(minNumSplits); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + return createTextSplit(minNumSplits); } + } + private InputSplit[] createTextSplit(int minNumSplits) throws IOException{ JobConf jobConf = buildConfig(); org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(jobConf, HdfsPathFilter.class); @@ -88,8 +101,30 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti @Override public void openInternal(InputSplit inputSplit) throws IOException { + + if(openKerberos){ + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + openHdfsTextReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return null; + } + }); + }else{ + openHdfsTextReader(inputSplit); + } + + } + + private void openHdfsTextReader(InputSplit inputSplit) throws IOException{ HdfsTextInputSplit hdfsTextInputSplit = (HdfsTextInputSplit) inputSplit; org.apache.hadoop.mapred.InputSplit fileSplit = hdfsTextInputSplit.getTextSplit(); + findCurrentPartition(((FileSplit) fileSplit).getPath()); recordReader = inputFormat.getRecordReader(fileSplit, conf, Reporter.NULL); key = new LongWritable(); value = new Text(); @@ -98,7 +133,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { @Override public Row nextRecordInternal(Row row) throws IOException { String line = new String(((Text)value).getBytes(), 0, ((Text)value).getLength(), charsetName); - String[] fields = line.split(delimiter); + String[] fields = StringUtils.splitPreserveAllTokens(line, delimiter); if (metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ row = new Row(fields.length); @@ -121,7 +156,7 @@ public Row nextRecordInternal(Row row) throws IOException { } if(value != null){ - value = HdfsUtil.string2col(String.valueOf(value),metaColumn.getType(),metaColumn.getTimeFormat()); + value = StringUtil.string2col(String.valueOf(value), metaColumn.getType(),metaColumn.getTimeFormat()); } row.setField(i, value); @@ -135,69 +170,7 @@ public Row nextRecordInternal(Row row) throws IOException { public boolean reachedEnd() throws IOException { key = new LongWritable(); value = new Text(); - return isFileEmpty || !recordReader.next(key, value); - } - - - public static class HdfsTextInputFormatBuilder { - - private HdfsTextInputFormat format; - - private HdfsTextInputFormatBuilder() { - format = new HdfsTextInputFormat(); - } - - public HdfsTextInputFormatBuilder setHadoopConfig(Map hadoopConfig) { - format.hadoopConfig = hadoopConfig; - return this; - } - - public HdfsTextInputFormatBuilder setInputPaths(String inputPaths) { - format.inputPath = inputPaths; - return this; - } - - public HdfsTextInputFormatBuilder setBytes(long bytes) { - format.bytes = bytes; - return this; - } - - public HdfsTextInputFormatBuilder setMonitorUrls(String monitorUrls) { - format.monitorUrls = monitorUrls; - return this; - } - - public HdfsTextInputFormatBuilder setDelimiter(String delimiter) { - if(delimiter == null) { - delimiter = "\\001"; - } - format.delimiter = delimiter; - return this; - } - - public HdfsTextInputFormatBuilder setDefaultFs(String defaultFs) { - format.defaultFs = defaultFs; - return this; - } - - public HdfsTextInputFormatBuilder setcharsetName (String charsetName) { - if(StringUtil.isNotEmpty(charsetName)) { - if(!Charset.isSupported(charsetName)) { - throw new UnsupportedCharsetException("The charset " + charsetName + " is not supported."); - } - this.format.charsetName = charsetName; - } - - return this; - } - - public HdfsTextInputFormat finish() { - return format; - } - } - - public static HdfsTextInputFormatBuilder buildHdfsTextInputFormat() { - return new HdfsTextInputFormatBuilder(); + return !recordReader.next(key, value); } static class HdfsTextInputSplit implements InputSplit { @@ -229,5 +202,4 @@ public int getSplitNumber() { return splitNumber; } } - } \ No newline at end of file diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java new file mode 100644 index 0000000000..a01ea427df --- /dev/null +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -0,0 +1,1221 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.io.orc; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface; +import org.apache.hadoop.hive.ql.io.*; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.shims.HadoopShims; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.*; +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +/** + * A MapReduce/Hive input format for ORC files. + *

+ * This class implements both the classic InputFormat, which stores the rows + * directly, and AcidInputFormat, which stores a series of events with the + * following schema: + *

+ *   class AcidEvent<ROW> {
+ *     enum ACTION {INSERT, UPDATE, DELETE}
+ *     ACTION operation;
+ *     long originalTransaction;
+ *     int bucket;
+ *     long rowId;
+ *     long currentTransaction;
+ *     ROW row;
+ *   }
+ * 
+ * Each AcidEvent object corresponds to an update event. The + * originalTransaction, bucket, and rowId are the unique identifier for the row. + * The operation and currentTransaction are the operation and the transaction + * that added this event. Insert and update events include the entire row, while + * delete events have null for row. + */ +public class OrcInputFormat implements InputFormat, + InputFormatChecker, VectorizedInputFormatInterface, + AcidInputFormat, + CombineHiveInputFormat.AvoidSplitCombination { + + static final HadoopShims SHIMS = ShimLoader.getHadoopShims(); + static final String MIN_SPLIT_SIZE = + SHIMS.getHadoopConfNames().get("MAPREDMINSPLITSIZE"); + static final String MAX_SPLIT_SIZE = + SHIMS.getHadoopConfNames().get("MAPREDMAXSPLITSIZE"); + static final String SARG_PUSHDOWN = "sarg.pushdown"; + private static final Log LOG = LogFactory.getLog(OrcInputFormat.class); + private static final long DEFAULT_MIN_SPLIT_SIZE = 16 * 1024 * 1024; + private static final long DEFAULT_MAX_SPLIT_SIZE = 256 * 1024 * 1024; + + private static final PerfLogger perfLogger = PerfLogger.getPerfLogger(); + private static final String CLASS_NAME = ReaderImpl.class.getName(); + + /** + * When picking the hosts for a split that crosses block boundaries, + * any drop any host that has fewer than MIN_INCLUDED_LOCATION of the + * number of bytes available on the host with the most. + * If host1 has 10MB of the split, host2 has 20MB, and host3 has 18MB the + * split will contain host2 (100% of host2) and host3 (90% of host2). Host1 + * with 50% will be dropped. + */ + private static final double MIN_INCLUDED_LOCATION = 0.80; + + /** + * Get the root column for the row. In ACID format files, it is offset by + * the extra metadata columns. + * @param isOriginal is the file in the original format? + * @return the column number for the root of row. + */ + private static int getRootColumn(boolean isOriginal) { + return isOriginal ? 0 : (OrcRecordUpdater.ROW + 1); + } + + public static RecordReader createReaderFromFile(Reader file, + Configuration conf, + long offset, long length + ) throws IOException { + Reader.Options options = new Reader.Options().range(offset, length); + boolean isOriginal = + !file.hasMetadataValue(OrcRecordUpdater.ACID_KEY_INDEX_NAME); + List types = file.getTypes(); + setIncludedColumns(options, types, conf, isOriginal); + setSearchArgument(options, types, conf, isOriginal); + return file.rowsOptions(options); + } + + /** + * Recurse down into a type subtree turning on all of the sub-columns. + * @param types the types of the file + * @param result the global view of columns that should be included + * @param typeId the root of tree to enable + * @param rootColumn the top column + */ + private static void includeColumnRecursive(List types, + boolean[] result, + int typeId, + int rootColumn) { + result[typeId - rootColumn] = true; + OrcProto.Type type = types.get(typeId); + int children = type.getSubtypesCount(); + for(int i=0; i < children; ++i) { + includeColumnRecursive(types, result, type.getSubtypes(i), rootColumn); + } + } + + /** + * Take the configuration and figure out which columns we need to include. + * @param options the options to update + * @param types the types for the file + * @param conf the configuration + * @param isOriginal is the file in the original format? + */ + static void setIncludedColumns(Reader.Options options, + List types, + Configuration conf, + boolean isOriginal) { + int rootColumn = getRootColumn(isOriginal); + if (!ColumnProjectionUtils.isReadAllColumns(conf)) { + int numColumns = types.size() - rootColumn; + boolean[] result = new boolean[numColumns]; + result[0] = true; + OrcProto.Type root = types.get(rootColumn); + List included = ColumnProjectionUtils.getReadColumnIDs(conf); + for(int i=0; i < root.getSubtypesCount(); ++i) { + if (included.contains(i)) { + includeColumnRecursive(types, result, root.getSubtypes(i), + rootColumn); + } + } + options.include(result); + } else { + options.include(null); + } + } + + static void setSearchArgument(Reader.Options options, + List types, + Configuration conf, + boolean isOriginal) { + int rootColumn = getRootColumn(isOriginal); + String serializedPushdown = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR); + String sargPushdown = conf.get(SARG_PUSHDOWN); + String columnNamesString = + conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR); + if ((sargPushdown == null && serializedPushdown == null) + || columnNamesString == null) { + LOG.debug("No ORC pushdown predicate"); + options.searchArgument(null, null); + } else { + SearchArgument sarg; + if (serializedPushdown != null) { + sarg = SearchArgumentFactory.create + (Utilities.deserializeExpression(serializedPushdown)); + } else { + sarg = SearchArgumentFactory.create(sargPushdown); + } + LOG.info("ORC pushdown predicate: " + sarg); + String[] neededColumnNames = columnNamesString.split(","); + String[] columnNames = new String[types.size() - rootColumn]; + boolean[] includedColumns = options.getInclude(); + int i = 0; + for(int columnId: types.get(rootColumn).getSubtypesList()) { + if (includedColumns == null || includedColumns[columnId - rootColumn]) { + // this is guaranteed to be positive because types only have children + // ids greater than their own id. + columnNames[columnId - rootColumn] = neededColumnNames[i++]; + } + } + options.searchArgument(sarg, columnNames); + } + } + + /** + * Get the list of input {@link Path}s for the map-reduce job. + * + * @param conf The configuration of the job + * @return the list of input {@link Path}s for the map-reduce job. + */ + static Path[] getInputPaths(Configuration conf) throws IOException { + String dirs = conf.get("mapred.input.dir"); + if (dirs == null) { + throw new IOException("Configuration mapred.input.dir is not defined."); + } + String [] list = StringUtils.split(dirs); + Path[] result = new Path[list.length]; + for (int i = 0; i < list.length; i++) { + result[i] = new Path(StringUtils.unEscapeString(list[i])); + } + return result; + } + + static List generateSplitsInfo(Configuration conf) + throws IOException { + // use threads to resolve directories into splits + Context context = new Context(conf); + for(Path dir: getInputPaths(conf)) { + FileSystem fs = dir.getFileSystem(conf); + context.schedule(new FileGenerator(context, fs, dir)); + } + context.waitForTasks(); + // deal with exceptions + if (!context.errors.isEmpty()) { + List errors = + new ArrayList(context.errors.size()); + for(Throwable th: context.errors) { + if (th instanceof IOException) { + errors.add((IOException) th); + } else { + throw new RuntimeException("serious problem", th); + } + } + throw new InvalidInputException(errors); + } + if (context.cacheStripeDetails) { + LOG.info("FooterCacheHitRatio: " + context.cacheHitCounter.get() + "/" + + context.numFilesCounter.get()); + } + return context.splits; + } + + static Path findOriginalBucket(FileSystem fs, + Path directory, + int bucket) throws IOException { + for(FileStatus stat: fs.listStatus(directory)) { + String name = stat.getPath().getName(); + String numberPart = name.substring(0, name.indexOf('_')); + if (org.apache.commons.lang3.StringUtils.isNumeric(numberPart) && + Integer.parseInt(numberPart) == bucket) { + return stat.getPath(); + } + } + throw new IllegalArgumentException("Can't find bucket " + bucket + " in " + + directory); + } + + @Override + public boolean shouldSkipCombine(Path path, + Configuration conf) throws IOException { + return (conf.get(AcidUtils.CONF_ACID_KEY) != null) || AcidUtils.isAcid(path, conf); + } + + @Override + public boolean validateInput(FileSystem fs, HiveConf conf, + ArrayList files + ) throws IOException { + + if (Utilities.isVectorMode(conf)) { + return new VectorizedOrcInputFormat().validateInput(fs, conf, files); + } + + if (files.size() <= 0) { + return false; + } + for (FileStatus file : files) { + try { + OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(conf).filesystem(fs)); + } catch (IOException e) { + return false; + } + } + return true; + } + + @Override + public InputSplit[] getSplits(JobConf job, + int numSplits) throws IOException { + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ORC_GET_SPLITS); + List result = generateSplitsInfo(job); + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ORC_GET_SPLITS); + return result.toArray(new InputSplit[result.size()]); + } + + @SuppressWarnings("unchecked") + private org.apache.hadoop.mapred.RecordReader + createVectorizedReader(InputSplit split, JobConf conf, Reporter reporter + ) throws IOException { + return (org.apache.hadoop.mapred.RecordReader) + new VectorizedOrcInputFormat().getRecordReader(split, conf, reporter); + } + + @Override + public org.apache.hadoop.mapred.RecordReader + getRecordReader(InputSplit inputSplit, JobConf conf, + Reporter reporter) throws IOException { + boolean vectorMode = Utilities.isVectorMode(conf); + + // if HiveCombineInputFormat gives us FileSplits instead of OrcSplits, + // we know it is not ACID. (see a check in CombineHiveInputFormat.getSplits() that assures this) + if (inputSplit.getClass() == FileSplit.class) { + if (vectorMode) { + return createVectorizedReader(inputSplit, conf, reporter); + } + return new OrcRecordReader(OrcFile.createReader( + ((FileSplit) inputSplit).getPath(), + OrcFile.readerOptions(conf)), conf, (FileSplit) inputSplit); + } + + OrcSplit split = (OrcSplit) inputSplit; + reporter.setStatus(inputSplit.toString()); + + Options options = new Options(conf).reporter(reporter); + final RowReader inner = getReader(inputSplit, options); + + + /*Even though there are no delta files, we still need to produce row ids so that an + * UPDATE or DELETE statement would work on a table which didn't have any previous updates*/ + if (split.isOriginal() && split.getDeltas().isEmpty()) { + if (vectorMode) { + return createVectorizedReader(inputSplit, conf, reporter); + } else { + return new NullKeyRecordReader(inner, conf); + } + } + + if (vectorMode) { + return (org.apache.hadoop.mapred.RecordReader) + new VectorizedOrcAcidRowReader(inner, conf, (FileSplit) inputSplit); + } + return new NullKeyRecordReader(inner, conf); + } + + @Override + public RowReader getReader(InputSplit inputSplit, + Options options) throws IOException { + final OrcSplit split = (OrcSplit) inputSplit; + final Path path = split.getPath(); + Path root; + if (split.hasBase()) { + if (split.isOriginal()) { + root = path.getParent(); + } else { + root = path.getParent().getParent(); + } + } else { + root = path; + } + final Path[] deltas = AcidUtils.deserializeDeltas(root, split.getDeltas()); + final Configuration conf = options.getConfiguration(); + final Reader reader; + final int bucket; + Reader.Options readOptions = new Reader.Options(); + readOptions.range(split.getStart(), split.getLength()); + if (split.hasBase()) { + bucket = AcidUtils.parseBaseBucketFilename(split.getPath(), conf) + .getBucket(); + reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); + final List types = reader.getTypes(); + setIncludedColumns(readOptions, types, conf, split.isOriginal()); + setSearchArgument(readOptions, types, conf, split.isOriginal()); + } else { + bucket = (int) split.getStart(); + reader = null; + } + String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY, + Long.MAX_VALUE + ":"); + ValidTxnList validTxnList = new ValidReadTxnList(txnString); + final OrcRawRecordMerger records = + new OrcRawRecordMerger(conf, true, reader, split.isOriginal(), bucket, + validTxnList, readOptions, deltas); + return new RowReader() { + OrcStruct innerRecord = records.createValue(); + + @Override + public ObjectInspector getObjectInspector() { + return ((StructObjectInspector) records.getObjectInspector()) + .getAllStructFieldRefs().get(OrcRecordUpdater.ROW) + .getFieldObjectInspector(); + } + + @Override + public boolean next(RecordIdentifier recordIdentifier, + OrcStruct orcStruct) throws IOException { + boolean result; + // filter out the deleted records + do { + result = records.next(recordIdentifier, innerRecord); + } while (result && + OrcRecordUpdater.getOperation(innerRecord) == + OrcRecordUpdater.DELETE_OPERATION); + if (result) { + // swap the fields with the passed in orcStruct + orcStruct.linkFields(OrcRecordUpdater.getRow(innerRecord)); + } + return result; + } + + @Override + public RecordIdentifier createKey() { + return records.createKey(); + } + + @Override + public OrcStruct createValue() { + return new OrcStruct(records.getColumns()); + } + + @Override + public long getPos() throws IOException { + return records.getPos(); + } + + @Override + public void close() throws IOException { + records.close(); + } + + @Override + public float getProgress() throws IOException { + return records.getProgress(); + } + }; + } + + @Override + public RawReader getRawReader(Configuration conf, + boolean collapseEvents, + int bucket, + ValidTxnList validTxnList, + Path baseDirectory, + Path[] deltaDirectory + ) throws IOException { + Reader reader = null; + boolean isOriginal = false; + if (baseDirectory != null) { + Path bucketFile; + if (baseDirectory.getName().startsWith(AcidUtils.BASE_PREFIX)) { + bucketFile = AcidUtils.createBucketFile(baseDirectory, bucket); + } else { + isOriginal = true; + bucketFile = findOriginalBucket(baseDirectory.getFileSystem(conf), + baseDirectory, bucket); + } + reader = OrcFile.createReader(bucketFile, OrcFile.readerOptions(conf)); + } + return new OrcRawRecordMerger(conf, collapseEvents, reader, isOriginal, + bucket, validTxnList, new Reader.Options(), deltaDirectory); + } + + private static class OrcRecordReader + implements org.apache.hadoop.mapred.RecordReader, + StatsProvidingRecordReader { + private final RecordReader reader; + private final long offset; + private final long length; + private final int numColumns; + private final Reader file; + private final SerDeStats stats; + private float progress = 0.0f; + + + OrcRecordReader(Reader file, Configuration conf, + FileSplit split) throws IOException { + List types = file.getTypes(); + this.file = file; + numColumns = (types.size() == 0) ? 0 : types.get(0).getSubtypesCount(); + this.offset = split.getStart(); + this.length = split.getLength(); + this.reader = createReaderFromFile(file, conf, offset, length); + this.stats = new SerDeStats(); + } + + @Override + public boolean next(NullWritable key, OrcStruct value) throws IOException { + if (reader.hasNext()) { + reader.next(value); + progress = reader.getProgress(); + return true; + } else { + return false; + } + } + + @Override + public NullWritable createKey() { + return NullWritable.get(); + } + + @Override + public OrcStruct createValue() { + return new OrcStruct(numColumns); + } + + @Override + public long getPos() throws IOException { + return offset + (long) (progress * length); + } + + @Override + public void close() throws IOException { + reader.close(); + } + + @Override + public float getProgress() throws IOException { + return progress; + } + + @Override + public SerDeStats getStats() { + stats.setRawDataSize(file.getRawDataSize()); + stats.setRowCount(file.getNumberOfRows()); + return stats; + } + } + + /** + * The global information about the split generation that we pass around to + * the different worker threads. + */ + static class Context { + private static Cache footerCache; + private final Configuration conf; + private final ExecutorService threadPool; + private final List splits = + new ArrayList(10000); + private final int numBuckets; + private final List errors = new ArrayList(); + private final long maxSize; + private final long minSize; + private final boolean footerInSplits; + private final boolean cacheStripeDetails; + private final AtomicInteger cacheHitCounter = new AtomicInteger(0); + private final AtomicInteger numFilesCounter = new AtomicInteger(0); + private Throwable fatalError = null; + private ValidTxnList transactionList; + + /** + * A count of the number of threads that may create more work for the + * thread pool. + */ + private int schedulers = 0; + + Context(Configuration conf) { + this.conf = conf; + minSize = conf.getLong(MIN_SPLIT_SIZE, DEFAULT_MIN_SPLIT_SIZE); + maxSize = conf.getLong(MAX_SPLIT_SIZE, DEFAULT_MAX_SPLIT_SIZE); + footerInSplits = HiveConf.getBoolVar(conf, + ConfVars.HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS); + numBuckets = + Math.max(conf.getInt(hive_metastoreConstants.BUCKET_COUNT, 0), 0); + LOG.debug("Number of buckets specified by conf file is " + numBuckets); + int cacheStripeDetailsSize = HiveConf.getIntVar(conf, + ConfVars.HIVE_ORC_CACHE_STRIPE_DETAILS_SIZE); + int numThreads = HiveConf.getIntVar(conf, + ConfVars.HIVE_ORC_COMPUTE_SPLITS_NUM_THREADS); + + cacheStripeDetails = (cacheStripeDetailsSize > 0); + + threadPool = Executors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("ORC_GET_SPLITS #%d").build()); + + synchronized (Context.class) { + if (footerCache == null && cacheStripeDetails) { + footerCache = CacheBuilder.newBuilder().concurrencyLevel(numThreads) + .initialCapacity(cacheStripeDetailsSize).softValues().build(); + } + } + String value = conf.get(ValidTxnList.VALID_TXNS_KEY, + Long.MAX_VALUE + ":"); + transactionList = new ValidReadTxnList(value); + } + + int getSchedulers() { + return schedulers; + } + + /** + * Get the Nth split. + * @param index if index >= 0, count from the front, otherwise count from + * the back. + * @return the Nth file split + */ + OrcSplit getResult(int index) { + if (index >= 0) { + return splits.get(index); + } else { + return splits.get(splits.size() + index); + } + } + + List getErrors() { + return errors; + } + + /** + * Add a unit of work. + * @param runnable the object to run + */ + synchronized void schedule(Runnable runnable) { + if (fatalError == null) { + if (runnable instanceof FileGenerator || + runnable instanceof SplitGenerator) { + schedulers += 1; + } + threadPool.execute(runnable); + } else { + throw new RuntimeException("serious problem", fatalError); + } + } + + /** + * Mark a worker that may generate more work as done. + */ + synchronized void decrementSchedulers() { + schedulers -= 1; + if (schedulers == 0) { + notify(); + } + } + + synchronized void notifyOnNonIOException(Throwable th) { + fatalError = th; + notify(); + } + + /** + * Wait until all of the tasks are done. It waits until all of the + * threads that may create more work are done and then shuts down the + * thread pool and waits for the final threads to finish. + */ + synchronized void waitForTasks() { + try { + while (schedulers != 0) { + wait(); + if (fatalError != null) { + threadPool.shutdownNow(); + throw new RuntimeException("serious problem", fatalError); + } + } + threadPool.shutdown(); + threadPool.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS); + } catch (InterruptedException ie) { + throw new IllegalStateException("interrupted", ie); + } + } + } + + /** + * Given a directory, get the list of files and blocks in those files. + * A thread is used for each directory. + */ + static final class FileGenerator implements Runnable { + private final Context context; + private final FileSystem fs; + private final Path dir; + + FileGenerator(Context context, FileSystem fs, Path dir) { + this.context = context; + this.fs = fs; + this.dir = dir; + } + + private void scheduleSplits(FileStatus file, + boolean isOriginal, + boolean hasBase, + List deltas) throws IOException{ + FileInfo info = null; + if (context.cacheStripeDetails) { + info = verifyCachedFileInfo(file); + } + new SplitGenerator(context, fs, file, info, isOriginal, deltas, + hasBase).schedule(); + } + + /** + * For each path, get the list of files and blocks that they consist of. + */ + @Override + public void run() { + try { + AcidUtils.Directory dirInfo = AcidUtils.getAcidState(dir, + context.conf, context.transactionList); + List deltas = + AcidUtils.serializeDeltas(dirInfo.getCurrentDirectories()); + Path base = dirInfo.getBaseDirectory(); + List original = dirInfo.getOriginalFiles(); + + boolean[] covered = new boolean[context.numBuckets]; + boolean isOriginal = base == null; + + // if we have a base to work from + if (base != null || !original.isEmpty()) { + + // find the base files (original or new style) + List children = original; + if (base != null) { + children = SHIMS.listLocatedStatus(fs, base, + AcidUtils.hiddenFileFilter); + } + + // for each child, schedule splits and mark off the bucket + for(FileStatus child: children) { + //update by tudou on 20200529, redmine = http://redmine.prod.dtstack.cn/issues/26286 + // make sure the file length on HDFS > 0(is not empty), otherwise an IndexOutOfBoundsException will be thrown in org.apache.hadoop.hive.ql.io.orc.ReaderImpl.extractMetaInfoFromFooter:362 + if(child.getLen() > 0){ + AcidOutputFormat.Options opts = AcidUtils.parseBaseBucketFilename + (child.getPath(), context.conf); + scheduleSplits(child, isOriginal, true, deltas); + int b = opts.getBucket(); + // If the bucket is in the valid range, mark it as covered. + // I wish Hive actually enforced bucketing all of the time. + if (b >= 0 && b < covered.length) { + covered[b] = true; + } + } + } + } + + // Generate a split for any buckets that weren't covered. + // This happens in the case where a bucket just has deltas and no + // base. + if (!deltas.isEmpty()) { + for (int b = 0; b < context.numBuckets; ++b) { + if (!covered[b]) { + synchronized (context.splits) { + context.splits.add(new OrcSplit(dir, b, 0, new String[0], null, + false, false, deltas)); + } + } + } + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } finally { + context.decrementSchedulers(); + } + } + + private FileInfo verifyCachedFileInfo(FileStatus file) { + context.numFilesCounter.incrementAndGet(); + FileInfo fileInfo = Context.footerCache.getIfPresent(file.getPath()); + if (fileInfo != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Info cached for path: " + file.getPath()); + } + if (fileInfo.modificationTime == file.getModificationTime() && + fileInfo.size == file.getLen()) { + // Cached copy is valid + context.cacheHitCounter.incrementAndGet(); + return fileInfo; + } else { + // Invalidate + Context.footerCache.invalidate(file.getPath()); + if (LOG.isDebugEnabled()) { + LOG.debug("Meta-Info for : " + file.getPath() + + " changed. CachedModificationTime: " + + fileInfo.modificationTime + ", CurrentModificationTime: " + + file.getModificationTime() + + ", CachedLength: " + fileInfo.size + ", CurrentLength: " + + file.getLen()); + } + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Info not cached for path: " + file.getPath()); + } + } + return null; + } + } + + /** + * Split the stripes of a given file into input splits. + * A thread is used for each file. + */ + static final class SplitGenerator implements Runnable { + private final Context context; + private final FileSystem fs; + private final FileStatus file; + private final long blockSize; + private final TreeMap locations; + private final FileInfo fileInfo; + private final boolean isOriginal; + private final List deltas; + private final boolean hasBase; + private List stripes; + private ReaderImpl.FileMetaInfo fileMetaInfo; + private Metadata metadata; + private List types; + private OrcFile.WriterVersion writerVersion; + + SplitGenerator(Context context, FileSystem fs, + FileStatus file, FileInfo fileInfo, + boolean isOriginal, + List deltas, + boolean hasBase) throws IOException { + this.context = context; + this.fs = fs; + this.file = file; + this.blockSize = file.getBlockSize(); + this.fileInfo = fileInfo; + locations = SHIMS.getLocationsWithOffset(fs, file); + this.isOriginal = isOriginal; + this.deltas = deltas; + this.hasBase = hasBase; + } + + /** + * Compute the number of bytes that overlap between the two ranges. + * @param offset1 start of range1 + * @param length1 length of range1 + * @param offset2 start of range2 + * @param length2 length of range2 + * @return the number of bytes in the overlap range + */ + static long getOverlap(long offset1, long length1, + long offset2, long length2) { + long end1 = offset1 + length1; + long end2 = offset2 + length2; + if (end2 <= offset1 || end1 <= offset2) { + return 0; + } else { + return Math.min(end1, end2) - Math.max(offset1, offset2); + } + } + + Path getPath() { + return file.getPath(); + } + + void schedule() throws IOException { + if(locations.size() == 1 && file.getLen() < context.maxSize) { + String[] hosts = locations.firstEntry().getValue().getHosts(); + synchronized (context.splits) { + context.splits.add(new OrcSplit(file.getPath(), 0, file.getLen(), + hosts, fileMetaInfo, isOriginal, hasBase, deltas)); + } + } else { + // if it requires a compute task + context.schedule(this); + } + } + + @Override + public String toString() { + return "splitter(" + file.getPath() + ")"; + } + + /** + * Create an input split over the given range of bytes. The location of the + * split is based on where the majority of the byte are coming from. ORC + * files are unlikely to have splits that cross between blocks because they + * are written with large block sizes. + * @param offset the start of the split + * @param length the length of the split + * @param fileMetaInfo file metadata from footer and postscript + * @throws IOException + */ + void createSplit(long offset, long length, + ReaderImpl.FileMetaInfo fileMetaInfo) throws IOException { + String[] hosts; + Map.Entry startEntry = locations.floorEntry(offset); + BlockLocation start = startEntry.getValue(); + if (offset + length <= start.getOffset() + start.getLength()) { + // handle the single block case + hosts = start.getHosts(); + } else { + Map.Entry endEntry = locations.floorEntry(offset + length); + BlockLocation end = endEntry.getValue(); + //get the submap + NavigableMap navigableMap = locations.subMap(startEntry.getKey(), + true, endEntry.getKey(), true); + // Calculate the number of bytes in the split that are local to each + // host. + Map sizes = new HashMap(); + long maxSize = 0; + for (BlockLocation block : navigableMap.values()) { + long overlap = getOverlap(offset, length, block.getOffset(), + block.getLength()); + if (overlap > 0) { + for(String host: block.getHosts()) { + LongWritable val = sizes.get(host); + if (val == null) { + val = new LongWritable(); + sizes.put(host, val); + } + val.set(val.get() + overlap); + maxSize = Math.max(maxSize, val.get()); + } + } else { + throw new IOException("File " + file.getPath().toString() + + " should have had overlap on block starting at " + block.getOffset()); + } + } + // filter the list of locations to those that have at least 80% of the + // max + long threshold = (long) (maxSize * MIN_INCLUDED_LOCATION); + List hostList = new ArrayList(); + // build the locations in a predictable order to simplify testing + for(BlockLocation block: navigableMap.values()) { + for(String host: block.getHosts()) { + if (sizes.containsKey(host)) { + if (sizes.get(host).get() >= threshold) { + hostList.add(host); + } + sizes.remove(host); + } + } + } + hosts = new String[hostList.size()]; + hostList.toArray(hosts); + } + synchronized (context.splits) { + context.splits.add(new OrcSplit(file.getPath(), offset, length, + hosts, fileMetaInfo, isOriginal, hasBase, deltas)); + } + } + + /** + * Divide the adjacent stripes in the file into input splits based on the + * block size and the configured minimum and maximum sizes. + */ + @Override + public void run() { + try { + populateAndCacheStripeDetails(); + + // figure out which stripes we need to read + boolean[] includeStripe = null; + // we can't eliminate stripes if there are deltas because the + // deltas may change the rows making them match the predicate. + if (deltas.isEmpty()) { + Reader.Options options = new Reader.Options(); + setIncludedColumns(options, types, context.conf, isOriginal); + setSearchArgument(options, types, context.conf, isOriginal); + // only do split pruning if HIVE-8732 has been fixed in the writer + if (options.getSearchArgument() != null && + writerVersion != OrcFile.WriterVersion.ORIGINAL) { + SearchArgument sarg = options.getSearchArgument(); + List sargLeaves = sarg.getLeaves(); + List stripeStats = metadata.getStripeStatistics(); + int[] filterColumns = RecordReaderImpl.mapSargColumns(sargLeaves, + options.getColumnNames(), getRootColumn(isOriginal)); + + if (stripeStats != null) { + // eliminate stripes that doesn't satisfy the predicate condition + includeStripe = new boolean[stripes.size()]; + for(int i=0; i < stripes.size(); ++i) { + includeStripe[i] = (i >= stripeStats.size()) || + isStripeSatisfyPredicate(stripeStats.get(i), sarg, + filterColumns); + if (LOG.isDebugEnabled() && !includeStripe[i]) { + LOG.debug("Eliminating ORC stripe-" + i + " of file '" + + file.getPath() + "' as it did not satisfy " + + "predicate condition."); + } + } + } + } + } + + // if we didn't have predicate pushdown, read everything + if (includeStripe == null) { + includeStripe = new boolean[stripes.size()]; + Arrays.fill(includeStripe, true); + } + + long currentOffset = -1; + long currentLength = 0; + int idx = -1; + for(StripeInformation stripe: stripes) { + idx++; + + if (!includeStripe[idx]) { + // create split for the previous unfinished stripe + if (currentOffset != -1) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + continue; + } + + // if we are working on a stripe, over the min stripe size, and + // crossed a block boundary, cut the input split here. + if (currentOffset != -1 && currentLength > context.minSize && + (currentOffset / blockSize != stripe.getOffset() / blockSize)) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + // if we aren't building a split, start a new one. + if (currentOffset == -1) { + currentOffset = stripe.getOffset(); + currentLength = stripe.getLength(); + } else { + currentLength = + (stripe.getOffset() + stripe.getLength()) - currentOffset; + } + if (currentLength >= context.maxSize) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + } + if (currentOffset != -1) { + createSplit(currentOffset, currentLength, fileMetaInfo); + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } finally { + context.decrementSchedulers(); + } + } + + private void populateAndCacheStripeDetails() { + try { + Reader orcReader; + if (fileInfo != null) { + stripes = fileInfo.stripeInfos; + fileMetaInfo = fileInfo.fileMetaInfo; + metadata = fileInfo.metadata; + types = fileInfo.types; + writerVersion = fileInfo.writerVersion; + // For multiple runs, in case sendSplitsInFooter changes + if (fileMetaInfo == null && context.footerInSplits) { + orcReader = OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(context.conf).filesystem(fs)); + fileInfo.fileMetaInfo = ((ReaderImpl) orcReader).getFileMetaInfo(); + fileInfo.metadata = orcReader.getMetadata(); + fileInfo.types = orcReader.getTypes(); + fileInfo.writerVersion = orcReader.getWriterVersion(); + } + } else { + orcReader = OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(context.conf).filesystem(fs)); + stripes = orcReader.getStripes(); + metadata = orcReader.getMetadata(); + types = orcReader.getTypes(); + writerVersion = orcReader.getWriterVersion(); + fileMetaInfo = context.footerInSplits ? + ((ReaderImpl) orcReader).getFileMetaInfo() : null; + if (context.cacheStripeDetails) { + // Populate into cache. + Context.footerCache.put(file.getPath(), + new FileInfo(file.getModificationTime(), file.getLen(), stripes, + metadata, types, fileMetaInfo, writerVersion)); + } + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } + } + + private boolean isStripeSatisfyPredicate(StripeStatistics stripeStatistics, + SearchArgument sarg, + int[] filterColumns) { + List predLeaves = sarg.getLeaves(); + TruthValue[] truthValues = new TruthValue[predLeaves.size()]; + for (int pred = 0; pred < truthValues.length; pred++) { + if (filterColumns[pred] != -1) { + + // column statistics at index 0 contains only the number of rows + ColumnStatistics stats = stripeStatistics.getColumnStatistics()[filterColumns[pred]]; + truthValues[pred] = RecordReaderImpl.evaluatePredicate(stats, predLeaves.get(pred)); + } else { + + // parition column case. + // partition filter will be evaluated by partition pruner so + // we will not evaluate partition filter here. + truthValues[pred] = TruthValue.YES_NO_NULL; + } + } + return sarg.evaluate(truthValues).isNeeded(); + } + } + + /** + * FileInfo. + * + * Stores information relevant to split generation for an ORC File. + * + */ + private static class FileInfo { + long modificationTime; + long size; + List stripeInfos; + ReaderImpl.FileMetaInfo fileMetaInfo; + Metadata metadata; + List types; + private OrcFile.WriterVersion writerVersion; + + + FileInfo(long modificationTime, long size, + List stripeInfos, + Metadata metadata, List types, + ReaderImpl.FileMetaInfo fileMetaInfo, + OrcFile.WriterVersion writerVersion) { + this.modificationTime = modificationTime; + this.size = size; + this.stripeInfos = stripeInfos; + this.fileMetaInfo = fileMetaInfo; + this.metadata = metadata; + this.types = types; + this.writerVersion = writerVersion; + } + } + + /** + * Return a RecordReader that is compatible with the Hive 0.12 reader + * with NullWritable for the key instead of RecordIdentifier. + */ + public static final class NullKeyRecordReader implements AcidRecordReader { + private final RecordIdentifier id; + private final RowReader inner; + + private NullKeyRecordReader(RowReader inner, Configuration conf) { + this.inner = inner; + id = inner.createKey(); + } + + public RecordIdentifier getRecordIdentifier() { + return id; + } + + @Override + public boolean next(NullWritable nullWritable, + OrcStruct orcStruct) throws IOException { + return inner.next(id, orcStruct); + } + + @Override + public NullWritable createKey() { + return NullWritable.get(); + } + + @Override + public OrcStruct createValue() { + return inner.createValue(); + } + + @Override + public long getPos() throws IOException { + return inner.getPos(); + } + + @Override + public void close() throws IOException { + inner.close(); + } + + @Override + public float getProgress() throws IOException { + return inner.getProgress(); + } + } + + +} diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java new file mode 100644 index 0000000000..6df5ceb002 --- /dev/null +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java @@ -0,0 +1,22 @@ +package com.dtstack.flinkx.hdfs.reader; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +/** + * @author jiangbo + * @date 2020/3/16 + */ +public class HdfsOrcInputFormatTest { + + @Test + public void testParseColumnAndType() { + HdfsOrcInputFormat hdfsOrcInputFormat = new HdfsOrcInputFormat(); + + String struct = "int,float(10,2),char(12)"; + List result = hdfsOrcInputFormat.parseColumnAndType(struct); + Assert.assertEquals(result.size(), 3); + } +} \ No newline at end of file diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 89f059b99d..ff3e37a7d6 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -98,6 +98,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java index 38ff34f667..f611487af5 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java @@ -43,6 +43,8 @@ */ public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { + private static final int FILE_NAME_PART_SIZE = 3; + protected int rowGroupSize; protected FileSystem fs; @@ -66,14 +68,29 @@ public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { protected Configuration conf; + protected boolean enableDictionary; + protected transient Map decimalColInfo; @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskmanager里同时认证kerberos + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + sleepRandomTime(); + } + initColIndices(); super.openInternal(taskNumber, numTasks); } + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long)(10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + @Override protected void checkOutputDir() { try{ @@ -120,7 +137,7 @@ protected void waitForActionFinishedBeforeWrite() { n++; } } catch (Exception e){ - + LOG.warn("Call method waitForActionFinishedBeforeWrite error", e); } } @@ -142,7 +159,7 @@ public boolean accept(Path path) { } String[] splits = fileName.split("\\."); - if (splits.length == 3) { + if (splits.length == FILE_NAME_PART_SIZE) { return Integer.parseInt(splits[2]) > fileIndex; } @@ -308,4 +325,8 @@ protected void moveAllTemporaryDataFileToDirectory() throws IOException { } } + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("HdfsWriter"); + } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 90f84054d5..a112570a66 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -34,7 +34,11 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.compress.*; +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.mapred.FileOutputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordWriter; @@ -213,17 +217,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe recordList.add(Integer.valueOf(rowData)); break; case BIGINT: - if (column instanceof Timestamp){ - column=((Timestamp) column).getTime(); - recordList.add(column); - break; - } - BigInteger data = new BigInteger(rowData); - if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ - recordList.add(data); - } else { - recordList.add(Long.valueOf(rowData)); - } + recordList.add(getBigint(column, rowData)); break; case FLOAT: recordList.add(Float.valueOf(rowData)); @@ -232,16 +226,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe recordList.add(Double.valueOf(rowData)); break; case DECIMAL: - ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(fullColumnNames.get(index)); - HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(rowData)); - hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); - if(hiveDecimal == null){ - String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", index, decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); - throw new WriteRecordException(msg, new IllegalArgumentException()); - } - - HiveDecimalWritable hiveDecimalWritable = new HiveDecimalWritable(hiveDecimal); - recordList.add(hiveDecimalWritable); + recordList.add(getDecimalWritable(index, rowData)); break; case STRING: case VARCHAR: @@ -270,6 +255,32 @@ private void getData(List recordList, int index, Row row) throws WriteRe } } + private Object getBigint(Object column, String rowData) { + if (column instanceof Timestamp){ + column = ((Timestamp) column).getTime(); + return column; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ + return data; + } else { + return Long.valueOf(rowData); + } + } + + private HiveDecimalWritable getDecimalWritable(int index, String rowData) throws WriteRecordException { + ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(fullColumnNames.get(index)); + HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(rowData)); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); + if(hiveDecimal == null){ + String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", index, decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + + return new HiveDecimalWritable(hiveDecimal); + } + @Override protected String recordConvertDetailErrorMessage(int pos, Row row) { return "\nHdfsOrcOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java index ec30366e16..0d8b40b951 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java @@ -83,6 +83,10 @@ public void setDefaultFs(String defaultFs) { format.defaultFs = defaultFs; } + public void setEnableDictionary(boolean enableDictionary) { + format.enableDictionary = enableDictionary; + } + @Override protected void checkFormat() { super.checkFormat(); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 7c4cd0e1fa..2dcb955c74 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -21,13 +21,13 @@ import com.dtstack.flinkx.enums.ColumnType; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.hdfs.ECompressType; +import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.ColumnTypeUtil; import com.dtstack.flinkx.util.DateUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.example.data.Group; @@ -38,12 +38,16 @@ import org.apache.parquet.hadoop.example.GroupWriteSupport; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.*; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; -import java.util.*; +import java.util.Date; +import java.util.HashMap; /** * The subclass of HdfsOutputFormat writing parquet files @@ -59,22 +63,8 @@ public class HdfsParquetOutputFormat extends BaseHdfsOutputFormat { private MessageType schema; - private static Calendar cal = Calendar.getInstance(); - - private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; - - private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; - private static ColumnTypeUtil.DecimalInfo PARQUET_DEFAULT_DECIMAL_INFO = new ColumnTypeUtil.DecimalInfo(10, 0); - static { - try { - cal.setTime(DateUtil.getDateFormatter().parse("1970-01-01")); - } catch (Exception e){ - throw new RuntimeException("Init calendar fail:",e); - } - } - @Override protected void openSource() throws IOException{ super.openSource(); @@ -101,6 +91,7 @@ protected void nextBlock(){ .withCompressionCodec(getCompressType()) .withConf(conf) .withType(schema) + .withDictionaryEncoding(enableDictionary) .withRowGroupSize(rowGroupSize); writer = builder.build(); @@ -162,7 +153,7 @@ public void writeSingleRecordToFile(Row row) throws WriteRecordException { try { for (; i < fullColumnNames.size(); i++) { Object valObj = row.getField(colIndices[i]); - if(valObj == null){ + if(valObj == null || valObj.toString().length() == 0){ continue; } @@ -233,7 +224,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ case "boolean" : group.add(colName,Boolean.parseBoolean(val));break; case "timestamp" : Timestamp ts = DateUtil.columnToTimestamp(valObj,null); - byte[] dst = longToByteArray(ts.getTime()); + byte[] dst = HdfsUtil.longToByteArray(ts.getTime()); group.add(colName, Binary.fromConstantByteArray(dst)); break; case "decimal" : @@ -246,7 +237,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ throw new WriteRecordException(msg, new IllegalArgumentException()); } - group.add(colName,decimalToBinary(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale())); + group.add(colName, HdfsUtil.decimalToBinary(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale())); break; case "date" : Date date = DateUtil.columnToDate(valObj,null); @@ -256,29 +247,6 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ } } - private Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec,int scale) { - byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); - - // Estimated number of bytes needed. - int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; - if (precToBytes == decimalBytes.length) { - // No padding needed. - return Binary.fromReusedByteArray(decimalBytes); - } - - byte[] tgt = new byte[precToBytes]; - if (hiveDecimal.signum() == -1) { - // For negative number, initializing bits to 1 - for (int i = 0; i < precToBytes; i++) { - tgt[i] |= 0xFF; - } - } - - // Padding leading zeroes/ones. - System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); - return Binary.fromReusedByteArray(tgt); - } - @Override protected String recordConvertDetailErrorMessage(int pos, Row row) { return "\nHdfsParquetOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; @@ -292,6 +260,7 @@ protected void closeSource() throws IOException { } private MessageType buildSchema(){ + decimalColInfo = new HashMap<>(16); Types.MessageTypeBuilder typeBuilder = Types.buildMessage(); for (int i = 0; i < fullColumnNames.size(); i++) { String name = fullColumnNames.get(i); @@ -317,10 +286,10 @@ private MessageType buildSchema(){ .as(OriginalType.DECIMAL) .precision(decimalInfo.getPrecision()) .scale(decimalInfo.getScale()) - .length(computeMinBytesForPrecision(decimalInfo.getPrecision())) + .length(HdfsUtil.computeMinBytesForPrecision(decimalInfo.getPrecision())) .named(name); - decimalColInfo = Collections.singletonMap(name, decimalInfo); + decimalColInfo.put(name, decimalInfo); } else { typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); } @@ -330,55 +299,4 @@ private MessageType buildSchema(){ return typeBuilder.named("Pair"); } - - private int computeMinBytesForPrecision(int precision){ - int numBytes = 1; - while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { - numBytes += 1; - } - return numBytes; - } - - private static byte[] longToByteArray(long data){ - long nano = data * 1000_000; - - int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); - byte[] julianDaysBytes = getBytes(julianDays); - flip(julianDaysBytes); - - long lastDayNanos = nano % NANO_SECONDS_PER_DAY; - byte[] lastDayNanosBytes = getBytes(lastDayNanos); - flip(lastDayNanosBytes); - - byte[] dst = new byte[12]; - - System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); - System.arraycopy(julianDaysBytes, 0, dst, 8, 4); - - return dst; - } - - private static byte[] getBytes(long i) { - byte[] bytes=new byte[8]; - bytes[0]=(byte)((i >> 56) & 0xFF); - bytes[1]=(byte)((i >> 48) & 0xFF); - bytes[2]=(byte)((i >> 40) & 0xFF); - bytes[3]=(byte)((i >> 32) & 0xFF); - bytes[4]=(byte)((i >> 24) & 0xFF); - bytes[5]=(byte)((i >> 16) & 0xFF); - bytes[6]=(byte)((i >> 8) & 0xFF); - bytes[7]=(byte)(i & 0xFF); - return bytes; - } - - /** - * @param bytes - */ - private static void flip(byte[] bytes) { - for(int i=0,j=bytes.length-1;i writeData(DataStream dataSet) { builder.setRestoreConfig(restoreConfig); builder.setMaxFileSize(maxFileSize); builder.setFlushBlockInterval(flushInterval); + builder.setEnableDictionary(enableDictionary); return createOutput(dataSet, builder.finish()); } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java index 0de6b79e84..33917f375d 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java @@ -29,9 +29,9 @@ public class TimePartitionFormat { - private static final long CONSTANT_TWO_DAY_TIME = 1000 * 60 * 60 * 24 * 2; - private static final long CONSTANT_TWO_HOUR_TIME = 1000 * 60 * 60 * 2; - private static final long CONSTANT_TWO_MINUTE_TIME = 1000 * 60 * 2; + private static final long CONSTANT_TWO_DAY_TIME = 1000 * 60 * 60 * 24 * 2L; + private static final long CONSTANT_TWO_HOUR_TIME = 1000 * 60 * 60 * 2L; + private static final long CONSTANT_TWO_MINUTE_TIME = 1000 * 60 * 2L; private static PartitionEnum partitionEnum; diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java index 21cb769d04..10d49b992c 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java @@ -25,7 +25,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hive.EStoreType.*; +import static com.dtstack.flinkx.hive.EStoreType.ORC; +import static com.dtstack.flinkx.hive.EStoreType.PARQUET; +import static com.dtstack.flinkx.hive.EStoreType.TEXT; /** * @author jiangbo diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java index 2c36ff3c74..4cb1fc6d09 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.authenticate.KerberosUtil; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.RetryUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.collections.MapUtils; @@ -55,6 +56,8 @@ public final class HiveDbUtil { public static final String SQLSTATE_CANNOT_ACQUIRE_CONNECT = "08004"; + public static final int JDBC_PART_SIZE = 2; + public static final String JDBC_REGEX = "[\\?|;|#]"; public static final String KEY_VAL_DELIMITER = "="; public static final String PARAM_DELIMITER = "&"; @@ -103,7 +106,7 @@ private static Connection getConnectionWithKerberos(ConnectionInfo connectionInf String keytabFileName = KerberosUtil.getPrincipalFileName(connectionInfo.getHiveConf()); keytabFileName = KerberosUtil.loadFile(connectionInfo.getHiveConf(), keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); + String principal = KerberosUtil.getPrincipal(connectionInfo.getHiveConf(), keytabFileName); KerberosUtil.loadKrb5Conf(connectionInfo.getHiveConf()); Configuration conf = FileSystemUtil.getConfiguration(connectionInfo.getHiveConf(), null); @@ -126,7 +129,7 @@ public Connection run(){ private static boolean openKerberos(final String jdbcUrl){ String[] splits = jdbcUrl.split(JDBC_REGEX); - if (splits.length != 2) { + if (splits.length != JDBC_PART_SIZE) { return false; } @@ -232,10 +235,10 @@ private static Connection getHiveConnection(String url, Properties prop) throws url = String.format("jdbc:hive2://%s:%s/%s", host, port, param); Connection connection = DriverManager.getConnection(url, prop); if (StringUtils.isNotEmpty(db)) { - try { - connection.createStatement().execute("use " + db); + try (Statement statement = connection.createStatement()) { + statement.execute("use " + db); } catch (SQLException e) { - if (connection != null) { + if (null != connection) { connection.close(); } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java index 7e0c0193fa..75209ebdfa 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java @@ -28,7 +28,8 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hive.EStoreType.*; +import static com.dtstack.flinkx.hive.EStoreType.ORC; +import static com.dtstack.flinkx.hive.EStoreType.TEXT; /** * @author toutian @@ -52,26 +53,27 @@ public class HiveUtil { private HiveDbUtil.ConnectionInfo connectionInfo; - enum HiveReleaseVersion{ - /** - * apache hive 1.x - */ - APACHE_1, - - /** - * apache hive 2.x - */ - APACHE_2, - - /** - * cdh hive 1.x - */ - CDH_1, - - /** - * cdh hive 2.x - */ - CDH_2 + public static String getHiveColumnType(String originType) { + originType = originType.trim(); + int indexOfBrackets = originType.indexOf(LEFT_BRACKETS); + if (indexOfBrackets > -1) { + String params = originType.substring(indexOfBrackets); + int index = params.indexOf(","); + int right = Integer.parseInt(params.substring(index+1, params.length()-1).trim()); + if(right == 0){ + int left = Integer.parseInt(params.substring(1, index).trim()); + if(left <= 4){ + return "SMALLINT"; + }else if(left <= 9){ + return "INT"; + }else if(left <= 18){ + return "BIGINT"; + } + } + return "DECIMAL" + params; + } else { + return convertType(originType); + } } public HiveUtil() { @@ -176,70 +178,6 @@ private AbstractHiveMetadataParser getMetadataParser(HiveReleaseVersion hiveVers } } - public HiveReleaseVersion getHiveVersion(Connection connection){ - HiveReleaseVersion version = HiveReleaseVersion.APACHE_2; - try { - ResultSet resultSet = connection.createStatement().executeQuery("select version()"); - if (resultSet.next()) { - String versionMsg = resultSet.getString(1); - if (versionMsg.contains("cdh")){ - // 结果示例:2.1.1-cdh6.3.1 re8d55f408b4f9aa2648bc9e34a8f802d53d6aab3 - if (versionMsg.startsWith("2")) { - version = HiveReleaseVersion.CDH_2; - } else if(versionMsg.startsWith("1")){ - version = HiveReleaseVersion.CDH_1; - } - } else { - // FIXME spark thrift server不支持 version()函数,所以使用默认的版本 - } - } - } catch (Exception ignore) { - } - - return version; - } - - public static String getCreateTableHql(TableInfo tableInfo) { - //不要使用create table if not exist,可能以后会在业务逻辑中判断表是否已经存在 - StringBuilder fieldsb = new StringBuilder("CREATE TABLE %s ("); - for (int i = 0; i < tableInfo.getColumns().size(); i++) { - fieldsb.append(String.format("`%s` %s", tableInfo.getColumns().get(i), tableInfo.getColumnTypes().get(i))); - if (i != tableInfo.getColumns().size() - 1) { - fieldsb.append(","); - } - } - fieldsb.append(") "); - if (!tableInfo.getPartitions().isEmpty()) { - fieldsb.append(" PARTITIONED BY ("); - for (String partitionField : tableInfo.getPartitions()) { - fieldsb.append(String.format("`%s` string", partitionField)); - } - fieldsb.append(") "); - } - if (TEXT.name().equalsIgnoreCase(tableInfo.getStore())) { - fieldsb.append(" ROW FORMAT DELIMITED FIELDS TERMINATED BY '"); - fieldsb.append(tableInfo.getDelimiter()); - fieldsb.append("' LINES TERMINATED BY '\\n' STORED AS TEXTFILE "); - } else if(ORC.name().equalsIgnoreCase(tableInfo.getStore())) { - fieldsb.append(" STORED AS ORC "); - }else{ - fieldsb.append(" STORED AS PARQUET "); - } - return fieldsb.toString(); - } - - public static String getHiveColumnType(String originType) { - originType = originType.trim(); - int indexOfBrackets = originType.indexOf(LEFT_BRACKETS); - if (indexOfBrackets > -1) { - String type = originType.substring(0, indexOfBrackets); - String params = originType.substring(indexOfBrackets); - return convertType(type) + params; - } else { - return convertType(originType); - } - } - private static String convertType(String type) { switch (type.toUpperCase()) { case "BIT": @@ -258,6 +196,8 @@ private static String convertType(String type) { case "INT8": type = "INT"; break; + case "NUMERIC": + case "NUMBER": case "BIGINT": type = "BIGINT"; break; @@ -272,8 +212,6 @@ private static String convertType(String type) { case "BINARY_DOUBLE": type = "DOUBLE"; break; - case "NUMERIC": - case "NUMBER": case "DECIMAL": type = "DECIMAL"; break; @@ -310,4 +248,102 @@ private static String convertType(String type) { } return type; } + + public static String getCreateTableHql(TableInfo tableInfo) { + //不要使用create table if not exist,可能以后会在业务逻辑中判断表是否已经存在 + StringBuilder fieldsb = new StringBuilder("CREATE TABLE %s ("); + for (int i = 0; i < tableInfo.getColumns().size(); i++) { + fieldsb.append(String.format("`%s` %s", tableInfo.getColumns().get(i), tableInfo.getColumnTypes().get(i))); + if (i != tableInfo.getColumns().size() - 1) { + fieldsb.append(","); + } + } + fieldsb.append(") "); + if (!tableInfo.getPartitions().isEmpty()) { + fieldsb.append(" PARTITIONED BY ("); + for (String partitionField : tableInfo.getPartitions()) { + fieldsb.append(String.format("`%s` string", partitionField)); + } + fieldsb.append(") "); + } + if (TEXT.name().equalsIgnoreCase(tableInfo.getStore())) { + fieldsb.append(" ROW FORMAT DELIMITED FIELDS TERMINATED BY '"); + fieldsb.append(tableInfo.getDelimiter()); + fieldsb.append("' LINES TERMINATED BY '\\n' STORED AS TEXTFILE "); + } else if(ORC.name().equalsIgnoreCase(tableInfo.getStore())) { + fieldsb.append(" STORED AS ORC "); + }else{ + fieldsb.append(" STORED AS PARQUET "); + } + return fieldsb.toString(); + } + + public HiveReleaseVersion getHiveVersion(Connection connection){ + HiveReleaseVersion version = HiveReleaseVersion.APACHE_2; + try (ResultSet resultSet = connection.createStatement().executeQuery("select version()")) { + if (resultSet.next()) { + String versionMsg = resultSet.getString(1); + if (versionMsg.contains(HiveReleaseVersion.CDH_1.getName())){ + // 结果示例:2.1.1-cdh6.3.1 re8d55f408b4f9aa2648bc9e34a8f802d53d6aab3 + if (versionMsg.startsWith(HiveReleaseVersion.CDH_2.getVersion())) { + version = HiveReleaseVersion.CDH_2; + } else if(versionMsg.startsWith(HiveReleaseVersion.CDH_1.getVersion())){ + version = HiveReleaseVersion.CDH_1; + } + } else { + //spark thrift server不支持 version()函数,所以使用默认的版本 + } + } + } catch (Exception ignore) { + } + + return version; + } + + enum HiveReleaseVersion{ + /** + * apache hive 1.x + */ + APACHE_1("apache", "1"), + + /** + * apache hive 2.x + */ + APACHE_2("apache", "2"), + + /** + * cdh hive 1.x + */ + CDH_1("cdh", "1"), + + /** + * cdh hive 2.x + */ + CDH_2("cdh", "2"); + + private String name; + + private String version; + + HiveReleaseVersion(String name, String version) { + this.name = name; + this.version = version; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + } } diff --git a/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java b/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java deleted file mode 100644 index 0b7c2e533d..0000000000 --- a/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.hive.test; - -import com.dtstack.flinkx.hive.util.HiveDbUtil; - -import java.sql.Connection; -import java.sql.ResultSet; -import java.util.HashMap; -import java.util.Map; - -/** - * @author jiangbo - * @date 2019/8/29 - */ -public class DBUtilTest { - - public static void main(String[] args) throws Exception{ - Map sftpConf = new HashMap<>(); - sftpConf.put("host", "172.16.10.79"); - sftpConf.put("port", "22"); - sftpConf.put("username", "root"); - sftpConf.put("password", "abc123"); - - Map hiveConf = new HashMap<>(); - hiveConf.put("hive.server2.authentication.kerberos.principal", "hive/cdh02@HADOOP.COM"); - hiveConf.put("hive.server2.authentication.kerberos.keytab", "D:\\cdh_cluster\\hive.keytab"); - hiveConf.put("java.security.krb5.conf", "D:\\cdh_cluster\\krb5.conf"); - hiveConf.put("useLocalFile", "true"); - hiveConf.put("sftpConf", sftpConf); - hiveConf.put("remoteDir", "/home/sftp/keytab/jiangbo"); - - HiveDbUtil.ConnectionInfo connectionInfo = new HiveDbUtil.ConnectionInfo(); - connectionInfo.setJdbcUrl("jdbc:hive2://172.16.10.75:10000/default;principal=hive/cdh02@HADOOP.COM"); - connectionInfo.setUsername(""); - connectionInfo.setPassword(""); - connectionInfo.setHiveConf(hiveConf); - - Connection connection = HiveDbUtil.getConnection(connectionInfo); - ResultSet rs = connection.createStatement().executeQuery("show tables"); - while (rs.next()) { - System.out.println(rs.getObject(2)); - } - - connection.close(); - } -} diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 25357e479f..e93421f32a 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -92,6 +92,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java index 217163c261..67731f0503 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java @@ -155,7 +155,7 @@ public FormatState getFormatState() { } private Map flushOutputFormat() { - Map formatStateMap = new HashMap<>(); + Map formatStateMap = new HashMap<>(outputFormats.size()); Iterator> entryIterator = outputFormats.entrySet().iterator(); while (entryIterator.hasNext()) { Map.Entry entry = entryIterator.next(); @@ -178,6 +178,7 @@ private Map flushOutputFormat() { @Override protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("HiveWriter"); } @Override @@ -187,6 +188,11 @@ public void writeRecord(Row row) throws IOException { Map event = null; if (row.getField(0) instanceof Map) { event = (Map) row.getField(0); + + if (null != event && event.containsKey("message")) { + event = MapUtils.getMap(event, "message"); + } + tablePath = PathConverterUtil.regaxByRules(event, tableBasePath, distributeTableMapping); fromLogData = true; } else { @@ -210,11 +216,11 @@ public void writeRecord(Row row) throws IOException { //row包含map嵌套的数据内容和channel, 而rowData是非常简单的纯数据,此处补上数据差额 if (fromLogData && bytesWriteCounter != null) { - bytesWriteCounter.add(row.toString().length() - rowData.toString().length()); + bytesWriteCounter.add((long)row.toString().length() - rowData.toString().length()); } } catch (Exception e) { // 写入产生的脏数据已经由hdfsOutputFormat处理了,这里不用再处理了,只打印日志 - if (numWriteCounter.getLocalValue() % 1000 == 0) { + if (numWriteCounter.getLocalValue() % LOG_PRINT_INTERNAL == 0) { LOG.warn("写入hdfs异常:", e); } } diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java index f8c69886b1..529d8d743b 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java @@ -131,9 +131,12 @@ protected void checkFormat() { if (this.format.tableBasePath == null || this.format.tableBasePath.length() == 0) { throw new IllegalArgumentException("No tableBasePath supplied."); } + if (this.format.tableInfos.isEmpty()){ throw new IllegalArgumentException("No tableInfos supplied."); } + + notSupportBatchWrite("HiveWriter"); } } diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java index 5168b850a6..a3483caaea 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java @@ -19,31 +19,34 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.hive.TableInfo; import com.dtstack.flinkx.hive.TimePartitionFormat; import com.dtstack.flinkx.hive.util.HiveUtil; import com.dtstack.flinkx.writer.BaseDataWriter; import com.dtstack.flinkx.writer.WriteMode; -import com.google.gson.Gson; +import com.google.gson.internal.LinkedTreeMap; +import com.google.gson.reflect.TypeToken; import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; import parquet.hadoop.ParquetWriter; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import static com.dtstack.flinkx.hdfs.HdfsConfigKeys.KEY_ROW_GROUP_SIZE; import static com.dtstack.flinkx.hive.HiveConfigKeys.*; +import static com.dtstack.flinkx.util.GsonUtil.GSON; /** * @author toutian */ public class HiveWriter extends BaseDataWriter { + private String readerName; + private String defaultFs; private String fileType; @@ -80,10 +83,9 @@ public class HiveWriter extends BaseDataWriter { private boolean autoCreateTable; - private Gson gson = new Gson(); - public HiveWriter(DataTransferConfig config) { super(config); + readerName = config.getJob().getContent().get(0).getReader().getName(); WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); hadoopConfig = (Map) writerConfig.getParameter().getVal(KEY_HADOOP_CONFIG); defaultFs = writerConfig.getParameter().getStringVal(KEY_DEFAULT_FS); @@ -95,9 +97,9 @@ public HiveWriter(DataTransferConfig config) { partition = writerConfig.getParameter().getStringVal(KEY_PARTITION, "pt"); delimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER, "\u0001"); charSet = writerConfig.getParameter().getStringVal(KEY_CHARSET_NAME); - maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, 1024 * 1024 * 1024); + maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); - bufferSize = writerConfig.getParameter().getLongVal(KEY_BUFFER_SIZE, 128 * 1024 * 1024); + bufferSize = writerConfig.getParameter().getLongVal(KEY_BUFFER_SIZE, 128 * ConstantValue.STORE_SIZE_M); rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE, WriteMode.APPEND.name()); @@ -120,18 +122,18 @@ public HiveWriter(DataTransferConfig config) { } } + /** + * 分表的映射关系 + * distributeTableMapping 的数据结构为 + * tableInfos的数据结构为 + */ private void formatHiveDistributeInfo(String distributeTable) { - /** - * 分表的映射关系 - * distributeTableMapping 的数据结构为 - * tableInfos的数据结构为 - */ distributeTableMapping = new HashMap<>(32); if (StringUtils.isNotBlank(distributeTable)) { - Map distributeTableMap = gson.fromJson(distributeTable, Map.class); - for (Map.Entry entry : distributeTableMap.entrySet()) { + Map> distributeTableMap = GSON.fromJson(distributeTable, new TypeToken>>(){}.getType()); + for (Map.Entry> entry : distributeTableMap.entrySet()) { String groupName = entry.getKey(); - List groupTables = (List) entry.getValue(); + List groupTables = entry.getValue(); for (String tableName : groupTables) { distributeTableMapping.put(tableName, groupName); } @@ -142,10 +144,12 @@ private void formatHiveDistributeInfo(String distributeTable) { private void formatHiveTableInfo(String tablesColumn) { tableInfos = new HashMap<>(16); if (StringUtils.isNotEmpty(tablesColumn)) { - Map tableColumnMap = gson.fromJson(tablesColumn, Map.class); - for (Map.Entry entry : tableColumnMap.entrySet()) { + Map>> tableColumnMap = GSON.fromJson(tablesColumn, new TypeToken> >>(){}.getType()); + List> extraTableColumnList = getExtraTableColumn(); + for (Map.Entry>> entry : tableColumnMap.entrySet()) { String tableName = entry.getKey(); - List> tableColumns = (List>) entry.getValue(); + List> tableColumns = entry.getValue(); + tableColumns.addAll(extraTableColumnList); TableInfo tableInfo = new TableInfo(tableColumns.size()); tableInfo.addPartition(partition); tableInfo.setDelimiter(delimiter); @@ -162,6 +166,32 @@ private void formatHiveTableInfo(String tablesColumn) { } } + /** + * 增加hive表字段 + */ + @SuppressWarnings("unchecked") + private List> getExtraTableColumn(){ + if(StringUtils.equalsIgnoreCase(readerName, "oraclelogminerreader")){ + List> list = new ArrayList<>(2); + Map opTime = new LinkedTreeMap<>(); + opTime.put("type", "BIGINT"); + opTime.put("key", "opTime"); + opTime.put("comment", ""); + + Map scn = new LinkedTreeMap<>(); + scn.put("type", "BIGINT"); + scn.put("key", "scn"); + scn.put("comment", ""); + + list.add(opTime); + list.add(scn); + + return list; + }else{ + return Collections.EMPTY_LIST; + } + } + @Override public DataStreamSink writeData(DataStream dataSet) { HiveOutputFormatBuilder builder = new HiveOutputFormatBuilder(); diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index d6cd16c26f..d9a711a6ab 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java index 61b483abd0..95016e4956 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java @@ -16,7 +16,7 @@ */ package com.dtstack.flinkx.kafka.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java index f209be623c..7f4c37f576 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java @@ -15,6 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + + package com.dtstack.flinkx.kafka.reader; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index b62d13da11..dad9361a4f 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index 980db83475..8c3e855d9c 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java index 8ad201b782..9633e6e4ad 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java @@ -16,7 +16,7 @@ */ package com.dtstack.flinkx.kafka09.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index cea09d8368..42fb23d6b2 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 8fbbdb883d..31467b287f 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java index 69c5747cf0..f0d10d6f13 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java @@ -17,7 +17,7 @@ */ package com.dtstack.flinkx.kafka10.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index f28e37fe50..4d86dbb7d0 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -39,6 +39,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index 4fa02eb7e5..ae62ee9515 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java index f27889ab25..2e3d2d569f 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java @@ -1,6 +1,6 @@ package com.dtstack.flinkx.kafka11.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java index 0424bb2801..5a7253860f 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java @@ -15,6 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + + package com.dtstack.flinkx.kafka11.reader; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index fe3808dd63..22d37d38b4 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -34,6 +34,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java index ec610798dc..38983693fb 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java @@ -18,10 +18,11 @@ package com.dtstack.flinkx.kafkabase.reader; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.decoder.DecodeEnum; +import com.dtstack.flinkx.decoder.IDecode; +import com.dtstack.flinkx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.PlainDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.decoder.JsonDecoder; -import com.dtstack.flinkx.kafkabase.decoder.PlainDecoder; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.StringUtil; @@ -66,7 +67,7 @@ public void openInputFormat() throws IOException { super.openInputFormat(); queue = new SynchronousQueue<>(false); - if ("json".equals(codec)) { + if (DecodeEnum.JSON.getName().equalsIgnoreCase(codec)) { decode = new JsonDecoder(); } else { decode = new PlainDecoder(); diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java index 7f3739089a..4e4f20cf02 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java @@ -28,7 +28,11 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.*; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_BLANK_IGNORE; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CODEC; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CONSUMER_SETTINGS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_GROUP_ID; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TOPIC; /** * Date: 2019/11/21 diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java index 482c1ca40a..6b338a13ad 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java @@ -18,8 +18,8 @@ package com.dtstack.flinkx.kafkabase.writer; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.decoder.JsonDecoder; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.kafkabase.decoder.JsonDecoder; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.flink.configuration.Configuration; @@ -29,7 +29,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.*; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; /** * Date: 2019/11/21 @@ -100,7 +104,7 @@ public void closeInternal() throws IOException { @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new UnsupportedOperationException(); + notSupportBatchWrite("KafkaWriter"); } @Override diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java index d8823c01f0..3ae4c05c7f 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java @@ -27,7 +27,10 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.*; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_PRODUCER_SETTINGS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TABLE_FIELDS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TIMEZONE; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TOPIC; /** * Date: 2019/11/21 diff --git a/flinkx-kudu/flinkx-kudu-core/pom.xml b/flinkx-kudu/flinkx-kudu-core/pom.xml index abafab1604..e419766ddc 100644 --- a/flinkx-kudu/flinkx-kudu-core/pom.xml +++ b/flinkx-kudu/flinkx-kudu-core/pom.xml @@ -24,11 +24,5 @@ 1.6 provided - - junit - junit - 4.12 - test - \ No newline at end of file diff --git a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java index 928580a166..4eb6ec865a 100644 --- a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java +++ b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java @@ -26,13 +26,22 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Type; -import org.apache.kudu.client.*; +import org.apache.kudu.client.AsyncKuduClient; +import org.apache.kudu.client.AsyncKuduScanner; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduPredicate; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduTable; import java.io.IOException; import java.math.BigDecimal; import java.security.PrivilegedExceptionAction; import java.sql.Timestamp; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index c8a99fea22..f9a1580d71 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -49,8 +49,12 @@ shade.kudureader.io.netty - com.google - shade.kudureader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java index f0ddce2857..6be5434a52 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java @@ -26,7 +26,11 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.apache.kudu.Type; -import org.apache.kudu.client.*; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; import java.io.IOException; import java.util.List; diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java index 48d0e43fc5..4959731c76 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.kudu.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.kudu.core.KuduConfig; import com.dtstack.flinkx.reader.MetaColumn; @@ -50,5 +51,9 @@ protected void checkFormat() { if (format.columns == null || format.columns.size() == 0){ throw new IllegalArgumentException("columns can not be empty"); } + + if (format.kuduConfig.getBatchSizeBytes() > ConstantValue.STORE_SIZE_G) { + throw new IllegalArgumentException("批量读取字节数必须小于[1G]"); + } } } diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java index 7b09c9048d..604b920a6e 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java @@ -32,7 +32,19 @@ import java.util.List; -import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.*; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BATCH_SIZE_BYTES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BOSS_COUNT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_FILTER; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_KEYTABFILE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_MASTER_ADDRESSES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_PRINCIPAL; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_QUERY_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_READ_MODE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_WORKER_COUNT; /** * @author jiangbo diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index 742fae63ab..256505e5ab 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -49,8 +49,12 @@ shade.kuduwriter.io.netty - com.google - shade.kuduwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java index df7a521186..69f346dcba 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java @@ -28,7 +28,12 @@ import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.types.Row; -import org.apache.kudu.client.*; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; +import org.apache.kudu.client.SessionConfiguration; import java.io.IOException; import java.util.List; diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java index 265e5610c2..2cb5532262 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java @@ -32,7 +32,16 @@ import java.util.List; -import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.*; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BOSS_COUNT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_FLUSH_MODE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_KEYTABFILE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_MASTER_ADDRESSES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_PRINCIPAL; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_WORKER_COUNT; /** * @author jiangbo diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java index 540db38056..d287c44a70 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.launcher; /** diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index 1b823c77eb..c5318c00eb 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -40,8 +40,6 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; import java.net.InetSocketAddress; @@ -97,7 +95,7 @@ public static ClusterClient createYarnClient(Options launcherOptions) { ApplicationId applicationId; if (StringUtils.isEmpty(launcherOptions.getAppId())) { - applicationId = getAppIdFromYarn(yarnClient); + applicationId = getAppIdFromYarn(yarnClient, launcherOptions); if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } @@ -156,7 +154,7 @@ private static AbstractYarnClusterDescriptor getClusterDescriptor(Options launch return yarnClusterDescriptor; } - private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exception{ + private static ApplicationId getAppIdFromYarn(YarnClient yarnClient, Options launcherOptions) throws Exception{ Set set = new HashSet<>(); set.add("Apache Flink"); EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); @@ -175,6 +173,10 @@ private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exce continue; } + if(!report.getQueue().equals(launcherOptions.getQueue())) { + continue; + } + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 91a4bd1409..a9382f251c 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -25,6 +25,7 @@ import com.dtstack.flinkx.launcher.perjob.PerJobSubmitter; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.options.Options; +import com.dtstack.flinkx.util.JsonModifyUtil; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; @@ -41,6 +42,7 @@ import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; /** @@ -94,8 +96,24 @@ public static void main(String[] args) throws Exception { String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + // 将argList转化为HashMap,方便通过参数名称来获取参数值 + HashMap temp = new HashMap<>(16); + for (int i = 0; i < argList.size(); i += 2) { + temp.put(argList.get(i), argList.get(i + 1)); + } + // 对json中的值进行修改 + HashMap parameter = JsonModifyUtil.CommandTransform(temp.get("-p")); + temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); + + // 清空list,填充修改后的参数值 + argList.clear(); + for (int i = 0; i < temp.size(); i++) { + argList.add(temp.keySet().toArray()[i].toString()); + argList.add(temp.values().toArray()[i].toString()); + } + if(mode.equals(ClusterMode.local.name())) { - String[] localArgs = argList.toArray(new String[argList.size()]); + String[] localArgs = argList.toArray(new String[0]); com.dtstack.flinkx.Main.main(localArgs); } else { String pluginRoot = launcherOptions.getPluginRoot(); @@ -237,9 +255,8 @@ public boolean accept(File dir, String name) { } private static String readJob(String job) { - try { - File file = new File(job); - FileInputStream in = new FileInputStream(file); + File file = new File(job); + try (FileInputStream in = new FileInputStream(file)) { byte[] fileContent = new byte[(int) file.length()]; in.read(fileContent); in.close(); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java index 6707c8f138..b88eb078b8 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.launcher; +import com.dtstack.flinkx.constants.ConstantValue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -42,7 +43,7 @@ public static YarnConfiguration getYarnConf(String yarnConfDir) { if(dir.exists() && dir.isDirectory()) { File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { - if(name.endsWith(".xml")){ + if(name.endsWith(ConstantValue.FILE_SUFFIX_XML)){ return true; } return false; diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java index 1d64181e81..508098067d 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java @@ -52,10 +52,10 @@ public static ClusterSpecification createClusterSpecification(Properties conProp int slotsPerTaskManager = 1; if(conProp != null){ - if(conProp.contains(JOBMANAGER_MEMORY_MB)){ + if(conProp.containsKey(JOBMANAGER_MEMORY_MB)){ jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); } - if(conProp.contains(TASKMANAGER_MEMORY_MB)){ + if(conProp.containsKey(TASKMANAGER_MEMORY_MB)){ taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index d9025e2ed9..81c6387395 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -21,7 +21,7 @@ import com.dtstack.flinkx.launcher.ClassLoaderType; import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; -import com.dtstack.flinkx.util.MapUtil; +import com.dtstack.flinkx.util.GsonUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; @@ -64,7 +64,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) Configuration flinkConfig = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); flinkConfig.setString("classloader.resolve-order", "child-first"); - Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); + Properties conProp = GsonUtil.GSON.fromJson(options.getConfProp(), Properties.class); ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); clusterSpecification.setCreateProgramDelay(true); clusterSpecification.setConfiguration(flinkConfig); diff --git a/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java b/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java deleted file mode 100644 index b974511769..0000000000 --- a/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java +++ /dev/null @@ -1,19 +0,0 @@ -package bigdata; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.JobManagerOptions; - -import java.io.File; - -/** - * Created by softfly on 18/4/24. - */ -public class FlinkConfigTest { - public static void main(String[] args) { - //Configuration config = GlobalConfiguration.loadConfiguration("/hadoop/flink-1.4.0/conf"); - //System.out.println(config.getString(JobManagerOptions.ADDRESS)); - String msg = "xxx" + File.separator; - System.out.println(msg); - } -} diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java index 26438d51fe..b5d78ffecc 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java @@ -19,7 +19,13 @@ package com.dtstack.flinkx.mongodb; -import com.mongodb.*; +import com.mongodb.AuthenticationMechanism; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientOptions; +import com.mongodb.MongoClientURI; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import com.mongodb.WriteConcern; import com.mongodb.client.MongoCursor; import org.apache.commons.lang.StringUtils; import org.bson.Document; diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java index 425dc21aa4..c419a2672e 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java @@ -29,9 +29,11 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.Arrays; +import java.util.Date; +import java.util.List; -import static com.dtstack.flinkx.enums.ColumnType.*; +import static com.dtstack.flinkx.enums.ColumnType.getType; /** * Utilities for mongodb database connection and data format conversion diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 09814f6c3f..a9caa60251 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -56,8 +56,12 @@ shade.mongodbreader.io.netty - com.google - shade.mongodbreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index 3f3ec2d863..ac8e65d705 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.mongodb.MongodbClientUtil; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.StringUtil; import com.mongodb.BasicDBObject; import com.mongodb.MongoClient; @@ -37,7 +38,8 @@ import org.bson.conversions.Bson; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; /** * Read plugin for reading static data @@ -149,6 +151,9 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti if(size * minNumSplits < docNum){ splits.add(new MongodbInputSplit((int)(size * minNumSplits), (int)(docNum - size * minNumSplits))); } + } catch (Exception e){ + LOG.error("error to create inputSplits, e = {}", ExceptionUtil.getErrorMessage(e)); + throw e; } finally { MongodbClientUtil.close(client, null); } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java index 918fb6b8f3..e1661838c2 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.mongodb.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; @@ -55,5 +56,9 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + if (format.mongodbConfig.getFetchSize() > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取条数必须小于[200000]条"); + } } } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 1d81ef36ef..a411920d71 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -56,8 +56,12 @@ shade.mongodbwriter.io.netty - com.google - shade.mongodbwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index 047e273350..21e5e39d4f 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -62,8 +62,12 @@ shade.mysqldreader.io.netty - com.google - shade.mysqldreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java index 9eae9b3dee..60181af17c 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java +++ b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.List; /** * @author toutian @@ -47,8 +46,8 @@ protected DistributedJdbcInputFormatBuilder getBuilder(){ } @Override - protected List buildConnections(){ - List sourceList = new ArrayList<>(connectionConfigs.size()); + protected ArrayList buildConnections(){ + ArrayList sourceList = new ArrayList<>(connectionConfigs.size()); for (ReaderConfig.ParameterConfig.ConnectionConfig connectionConfig : connectionConfigs) { String curUsername = (connectionConfig.getUsername() == null || connectionConfig.getUsername().length() == 0) ? username : connectionConfig.getUsername(); diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 8856cdf063..a52536b7ec 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -62,9 +62,13 @@ shade.mysqlreader.io.netty - com.google - shade.mysqlreader.com.google - + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index 7915c9bd64..630923f304 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -18,12 +18,11 @@ package com.dtstack.flinkx.mysql.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -53,7 +52,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -66,8 +67,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { } querySql = buildQuerySql(inputSplit); - //MySQL流式读取 - fetchSize = Integer.MIN_VALUE; + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); @@ -122,7 +125,6 @@ public Row nextRecordInternal(Row row) throws IOException { } return super.nextRecordInternal(row); }catch (Exception e) { - LOG.error("error to get next record, row = {}, descColumnTypeList = {}, e = {}", row, new Gson().toJson(descColumnTypeList), ExceptionUtil.getErrorMessage(e)); throw new IOException("Couldn't read data - " + e.getMessage(), e); } } diff --git a/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java b/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java deleted file mode 100644 index 50dff8f343..0000000000 --- a/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java +++ /dev/null @@ -1,22 +0,0 @@ -package bigdata; - -import com.dtstack.flinkx.mysql.MySqlDatabaseMeta; -import com.dtstack.flinkx.rdb.util.DbUtil; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.Map; - - -public class TestMysqlPk { - public static void main(String[] args) throws ClassNotFoundException, SQLException { - MySqlDatabaseMeta databaseMeta = new MySqlDatabaseMeta(); - Class.forName(databaseMeta.getDriverClass()); - Connection conn = DriverManager.getConnection("jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", "dtstack", "abc123"); - //List list = databaseMeta.listUniqueKeys("sb250", conn); - //System.out.println(list); - Map map = DbUtil.getPrimaryOrUniqueKeys("sb252", conn); - System.out.println(map); - } -} diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 82e3aa55c4..94e5d95a50 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -62,8 +62,12 @@ shade.mysqlwriter.io.netty - com.google - shade.mysqlwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index dd54b0ba18..1cb857e3ec 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -56,7 +56,7 @@ public class OdpsUtil { public static int MAX_RETRY_TIME = 3; - public static final long BUFFER_SIZE_DEFAULT = 64 * 1024 * 1024; + public static final long BUFFER_SIZE_DEFAULT = 64 * 1024 * 1024L; public static Odps initOdps(Map odpsConfig) { String odpsServer = odpsConfig.get(OdpsConfigKeys.KEY_ODPS_SERVER); diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 866fd622fe..8007e473c0 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -50,6 +50,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java index 1bd4db402b..eba6fc7175 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java @@ -19,11 +19,11 @@ package com.dtstack.flinkx.odps.reader; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.odps.OdpsConfigKeys; import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.odps.OdpsConfigKeys.*; /** * The Builder of OdpsInputFormat @@ -41,7 +41,7 @@ public OdpsInputFormatBuilder() { public void setOdpsConfig(Map odpsConfig) { format.odpsConfig = odpsConfig; - format.projectName = odpsConfig.get(KEY_PROJECT); + format.projectName = odpsConfig.get(OdpsConfigKeys.KEY_PROJECT); } public void setTableName(String tableName) { diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java index 4a076ce122..f45ee2d76a 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java @@ -29,7 +29,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.odps.OdpsConfigKeys.*; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_ODPS_CONFIG; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_PARTITION; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_TABLE; /** * The reader plugin of Odps diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index b268bf43b1..f1622adc0b 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -55,6 +55,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 7e6510104d..cdf8b9c6e8 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -116,7 +116,7 @@ public void writeSingleRecordInternal(Row row) throws WriteRecordException{ @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new UnsupportedOperationException(); + notSupportBatchWrite("OdpsWriter"); } private Record row2record(Row row, String[] columnTypes) throws WriteRecordException { diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java index 95c7dad246..ee3d21d10a 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java @@ -76,5 +76,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("OdpsWriter"); } } diff --git a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java index 5033bf64af..bc4b88f3ba 100644 --- a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java +++ b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java @@ -36,7 +36,7 @@ public class OracleDatabaseMeta extends BaseDatabaseMeta { public String quoteTable(String table) { table = table.replace("\"",""); String[] part = table.split("\\."); - if(part.length == 2) { + if(part.length == DB_TABLE_PART_SIZE) { table = getStartQuote() + part[0] + getEndQuote() + "." + getStartQuote() + part[1] + getEndQuote(); } else { table = getStartQuote() + table + getEndQuote(); diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 52ff1880be..225d8e2554 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -62,8 +62,12 @@ shade.oracleReader.io.netty - com.google.guava - shade.oracleReader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java b/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java deleted file mode 100644 index d4227ff774..0000000000 --- a/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.dtstack.flinkx.oracle.reader.test; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; - -/** - * Created by softfly on 18/2/1. - */ -public class OracleLocalTest { - public static void main(String[] args) throws ClassNotFoundException, SQLException { - final String DRIVER = "oracle.jdbc.OracleDriver"; - final String URL = "jdbc:oracle:thin:dev/pass1234@172.16.8.121:1521:dtstack"; - final String USER = "dev"; - final String PASSWORD = "pass1234"; - Connection connection = null; - Class.forName(DRIVER); - connection = DriverManager.getConnection(URL); - - - } -} diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index c6fc7b4e2b..6e9c1850e7 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -63,8 +63,12 @@ shade.oracleWriter.io.netty - com.google.guava - shade.oracleWriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java index ed77a77ea4..98840d5b10 100644 --- a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java +++ b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java @@ -18,11 +18,16 @@ package com.dtstack.flinkx.oracle.format; import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.oracle.OracleDatabaseMeta; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; import com.dtstack.flinkx.util.DateUtil; import org.apache.flink.types.Row; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.HashMap; @@ -63,7 +68,7 @@ protected List probeFullColumns(String table, Connection dbConn) throws String schema =null; String[] parts = table.split("\\."); - if(parts.length == 2) { + if(parts.length == OracleDatabaseMeta.DB_TABLE_PART_SIZE) { schema = parts[0].toUpperCase(); table = parts[1]; } @@ -79,24 +84,26 @@ protected List probeFullColumns(String table, Connection dbConn) throws @Override protected Map> probePrimaryKeys(String table, Connection dbConn) throws SQLException { Map> map = new HashMap<>(16); - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); - ResultSet rs = ps.executeQuery(); - while(rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if(!map.containsKey(indexName)) { - map.put(indexName,new ArrayList<>()); + try (PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL, table)); + ResultSet rs = ps.executeQuery()) { + while(rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if(!map.containsKey(indexName)) { + map.put(indexName,new ArrayList<>()); + } + map.get(indexName).add(rs.getString("COLUMN_NAME")); } - map.get(indexName).add(rs.getString("COLUMN_NAME")); - } - Map> retMap = new HashMap<>((map.size()<<2)/3); - for(Map.Entry> entry: map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if(v!=null && v.size() != 0 && v.get(0) != null) { - retMap.put(k, v); + + Map> retMap = new HashMap<>((map.size()<<2)/3); + for(Map.Entry> entry: map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if(v!=null && v.size() != 0 && v.get(0) != null) { + retMap.put(k, v); + } } + return retMap; } - return retMap; } } diff --git a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml index 182ea07067..04f327d64e 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index 180385dc01..b067a02a7e 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 4d5c46149d..a8fc44fdb5 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.sql.SQLException; -import java.sql.Statement; import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -67,7 +66,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { // 部分驱动需要关闭事务自动提交,fetchSize参数才会起作用 dbConn.setAutoCommit(false); - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + statement = dbConn.createStatement(resultSetType, resultSetConcurrency); statement.setFetchSize(0); diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index b97dab27c8..1dc16435a8 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index d477787cac..26060ebcf6 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -56,6 +56,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java b/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java index b7b3e95855..5e21aec025 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java +++ b/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java @@ -21,7 +21,6 @@ import com.dtstack.flinkx.mysqld.reader.MysqldReader; import com.dtstack.flinkx.polardbd.PolardbDatabaseMeta; import com.dtstack.flinkx.polardbd.format.PolardbdInputFormat; -import com.dtstack.flinkx.rdb.inputformat.DistributedJdbcInputFormat; import com.dtstack.flinkx.rdb.inputformat.DistributedJdbcInputFormatBuilder; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 093bdb5152..77ca955ae3 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -55,6 +55,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index d088128085..4ca3fb7511 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -56,6 +56,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index 985b99efaa..4af93b9480 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -20,10 +20,7 @@ import com.dtstack.flinkx.enums.EDatabaseType; import com.dtstack.flinkx.rdb.BaseDatabaseMeta; -import org.apache.commons.lang3.StringUtils; -import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.Map; diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java index 190ff818b1..04f5616f23 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java @@ -19,8 +19,10 @@ package com.dtstack.flinkx.postgresql; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; +import org.apache.commons.lang3.StringUtils; import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -35,9 +37,9 @@ public class PostgresqlTypeConverter implements TypeConverterInterface { private List byteTypes = Arrays.asList("bytea","bit varying"); - private List bitTypes = Arrays.asList("bit"); + private List bitTypes = Collections.singletonList("bit"); - private List doubleTypes = Arrays.asList("money"); + private List doubleTypes = Collections.singletonList("money"); private List intTypes = Arrays.asList("int","int2","int4","int8"); @@ -46,19 +48,24 @@ public Object convert(Object data,String typeName) { if (data == null){ return null; } - + String dataValue = data.toString(); + if(stringTypes.contains(typeName)){ + return dataValue; + } + if(StringUtils.isBlank(dataValue)){ + return null; + } if(doubleTypes.contains(typeName)){ - data = Double.parseDouble(String.valueOf(data)); + if(StringUtils.startsWith(dataValue, "$")){ + dataValue = StringUtils.substring(dataValue, 1); + } + data = Double.parseDouble(dataValue); } else if(bitTypes.contains(typeName)){ // - } else if(stringTypes.contains(typeName)){ - data = String.valueOf(data); - } else if(byteTypes.contains(typeName)){ - data = Byte.valueOf(String.valueOf(data)); + }else if(byteTypes.contains(typeName)){ + data = Byte.valueOf(dataValue); } else if(intTypes.contains(typeName)){ - if(data instanceof String){ - data = Integer.parseInt(data.toString()); - } + data = Integer.parseInt(dataValue); } return data; diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index f103f55d85..6e3b8612d6 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -63,8 +63,12 @@ shade.postgresqlreader.io.netty - com.google - shade.postgresqlreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java index 9f3f96c342..44ffade153 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.postgresql.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; @@ -51,7 +52,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -64,6 +67,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { } querySql = buildQuerySql(inputSplit); + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 2264b0dddb..7f6857ea04 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -62,8 +62,12 @@ shade.postgresqlwriter.io.netty - com.google - shade.postgresqlwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java index ad667ceb13..758be23ace 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java @@ -21,7 +21,11 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * Abstract base parent class of other database prototype implementations @@ -31,6 +35,8 @@ */ public abstract class BaseDatabaseMeta implements DatabaseInterface, Serializable { + public static final int DB_TABLE_PART_SIZE = 2; + @Override public String getStartQuote() { return "\""; diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 581d2030d9..f82521a483 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -25,15 +25,19 @@ import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.SysUtil; import com.dtstack.flinkx.util.TelnetUtil; -import com.google.gson.Gson; import org.apache.commons.lang.StringUtils; import org.apache.flink.util.CollectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedReader; -import java.math.BigDecimal; -import java.sql.*; +import java.sql.Clob; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -41,9 +45,9 @@ import java.util.regex.Pattern; /** + * * Utilities for relational database connection and sql execution * company: www.dtstack.com - * * @author huyifan_zju@ */ public class DbUtil { @@ -72,6 +76,9 @@ public class DbUtil { */ private static int NANOS_LENGTH = 19; + public static int NANOS_PART_LENGTH = 9; + private static int FORMAT_TIME_NANOS_LENGTH = 29; + /** * jdbc连接URL的分割正则,用于获取URL?后的连接参数 */ @@ -93,16 +100,15 @@ public class DbUtil { /** * 获取jdbc连接(超时10S) - * - * @param url url - * @param username 账号 - * @param password 密码 + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; - synchronized (ClassUtil.LOCK_STR) { + synchronized (ClassUtil.LOCK_STR){ DriverManager.setLoginTimeout(10); // telnet @@ -120,10 +126,9 @@ private static Connection getConnectionInternal(String url, String username, Str /** * 获取jdbc连接(重试3次) - * - * @param url url - * @param username 账号 - * @param password 密码 + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ @@ -136,8 +141,10 @@ public static Connection getConnection(String url, String username, String passw for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { try { dbConn = getConnectionInternal(url, username, password); - dbConn.createStatement().execute("select 111"); - failed = false; + try (Statement statement = dbConn.createStatement()){ + statement.execute("select 111"); + failed = false; + } } catch (Exception e) { if (dbConn != null) { dbConn.close(); @@ -157,10 +164,9 @@ public static Connection getConnection(String url, String username, String passw /** * 关闭连接资源 - * - * @param rs ResultSet - * @param stmt Statement - * @param conn Connection + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection * @param commit */ public static void closeDbResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { @@ -182,7 +188,7 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con if (null != conn) { try { - if (commit) { + if(commit){ commit(conn); } @@ -195,76 +201,49 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con /** * 手动提交事物 - * * @param conn Connection */ - public static void commit(Connection conn) { + public static void commit(Connection conn){ try { - if (!conn.isClosed() && !conn.getAutoCommit()) { + if (!conn.isClosed() && !conn.getAutoCommit()){ conn.commit(); } - } catch (SQLException e) { + } catch (SQLException e){ LOG.warn("commit error:{}", ExceptionUtil.getErrorMessage(e)); } } /** * 批量执行sql - * * @param dbConn Connection * @param sqls sql列表 */ public static void executeBatch(Connection dbConn, List sqls) { - if (sqls == null || sqls.size() == 0) { + if(sqls == null || sqls.size() == 0) { return; } - try { - Statement stmt = dbConn.createStatement(); - for (String sql : sqls) { + try (Statement stmt = dbConn.createStatement()) { + for(String sql : sqls) { stmt.addBatch(sql); } stmt.executeBatch(); } catch (SQLException e) { - throw new RuntimeException("execute batch sql error:{}", e); + throw new RuntimeException("execute batch sql error:{}",e); } finally { commit(dbConn); } } - /** - * 获取某数据库某表的主键和唯一索引 - * - * @param table 表名 - * @param dbConn 数据库连接 - * @return - * @throws SQLException - */ - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { - Map> keyMap = new HashMap<>(16); - DatabaseMetaData meta = dbConn.getMetaData(); - ResultSet rs = meta.getIndexInfo(null, null, table, true, false); - while (rs.next()) { - String pkName = rs.getString(6); - String columnName = rs.getString(9); - if (!keyMap.containsKey(pkName)) { - keyMap.put(pkName, new ArrayList<>()); - } - keyMap.get(pkName).add(columnName); - } - return keyMap; - } - /** * 封装channel通道顺序 - * * @param channels * @return */ - public static Object[][] getParameterValues(final int channels) { + public static Object[][] getParameterValues(final int channels){ ParameterValuesProvider provider = () -> { Integer[][] parameters = new Integer[channels][]; - for (int i = 0; i < channels; ++i) { + for(int i = 0; i < channels; ++i) { parameters[i] = new Integer[2]; parameters[i][0] = channels; parameters[i][1] = i; @@ -277,110 +256,65 @@ public static Object[][] getParameterValues(final int channels) { /** * 获取表列名类型列表 - * * @param dbUrl jdbc url * @param username 数据库账号 * @param password 数据库密码 * @param databaseInterface DatabaseInterface * @param table 表名 - * @param metaColumns MetaColumn列表 - * @return 字段类型list列表 + * @param metaColumns MetaColumn列表 + * @return */ - public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, String table, List metaColumns) { - List descColumnTypeList = new ArrayList<>(metaColumns.size()); + public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, + String table, List metaColumns) { + List ret = new ArrayList<>(metaColumns.size()); Connection dbConn = null; Statement stmt = null; ResultSet rs = null; try { dbConn = getConnection(dbUrl, username, password); + if (null == dbConn) { + throw new RuntimeException("Get hive connection error"); + } + stmt = dbConn.createStatement(); rs = stmt.executeQuery(databaseInterface.getSqlQueryFields(databaseInterface.quoteTable(table))); ResultSetMetaData rd = rs.getMetaData(); - boolean flag = ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName()); - Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); - for (int i = 1; i <= rd.getColumnCount(); i++) { - if(flag){ - descColumnTypeList.add(rd.getColumnTypeName(i)); - }else{ - nameTypeMap.put(rd.getColumnName(i), rd.getColumnTypeName(i)); - } + Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); + for(int i = 0; i < rd.getColumnCount(); ++i) { + nameTypeMap.put(rd.getColumnName(i+1),rd.getColumnTypeName(i+1)); } - if(!flag){ - for (MetaColumn metaColumn : metaColumns) { - if (metaColumn.getValue() != null) { - descColumnTypeList.add("string"); - } else { - descColumnTypeList.add(nameTypeMap.get(metaColumn.getName())); - } + + for (MetaColumn metaColumn : metaColumns) { + if(metaColumn.getValue() != null){ + ret.add("string"); + } else { + ret.add(nameTypeMap.get(metaColumn.getName())); } } } catch (SQLException e) { - LOG.error("error to analyzeTable, dbUrl = {}, table = {}, metaColumns = {}, e = {}", dbUrl, table, new Gson().toJson(metaColumns), ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); } finally { closeDbResources(rs, stmt, dbConn, false); } - return descColumnTypeList; - } - - /** - * 占位符设值 - * - * @param param 参数 - * @param statement PreparedStatement - * @param i 占位符位置 - * @throws SQLException - */ - public static void setParameterValue(Object param, PreparedStatement statement, int i) throws SQLException { - if (param instanceof String) { - statement.setString(i + 1, (String) param); - } else if (param instanceof Long) { - statement.setLong(i + 1, (Long) param); - } else if (param instanceof Integer) { - statement.setInt(i + 1, (Integer) param); - } else if (param instanceof Double) { - statement.setDouble(i + 1, (Double) param); - } else if (param instanceof Boolean) { - statement.setBoolean(i + 1, (Boolean) param); - } else if (param instanceof Float) { - statement.setFloat(i + 1, (Float) param); - } else if (param instanceof BigDecimal) { - statement.setBigDecimal(i + 1, (BigDecimal) param); - } else if (param instanceof Byte) { - statement.setByte(i + 1, (Byte) param); - } else if (param instanceof Short) { - statement.setShort(i + 1, (Short) param); - } else if (param instanceof Date) { - statement.setDate(i + 1, (Date) param); - } else if (param instanceof Time) { - statement.setTime(i + 1, (Time) param); - } else if (param instanceof Timestamp) { - statement.setTimestamp(i + 1, (Timestamp) param); - } else if (param instanceof Array) { - statement.setArray(i + 1, (Array) param); - } else { - //extends with other types if needed - throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)."); - } + return ret; } /** * clob转string - * - * @param obj clob + * @param obj clob * @return * @throws Exception */ - public static Object clobToString(Object obj) throws Exception { + public static Object clobToString(Object obj) throws Exception{ String dataStr; - if (obj instanceof Clob) { - Clob clob = (Clob) obj; + if(obj instanceof Clob){ + Clob clob = (Clob)obj; BufferedReader bf = new BufferedReader(clob.getCharacterStream()); StringBuilder stringBuilder = new StringBuilder(); String line; - while ((line = bf.readLine()) != null) { + while ((line = bf.readLine()) != null){ stringBuilder.append(line); } dataStr = stringBuilder.toString(); @@ -393,34 +327,32 @@ public static Object clobToString(Object obj) throws Exception { /** * 获取纳秒字符串 - * - * @param timeStr + * @param timeStr 2020-03-23 11:03:22.000000000 * @return */ - public static String getNanosTimeStr(String timeStr) { - if (timeStr.length() < 29) { - timeStr += StringUtils.repeat("0", 29 - timeStr.length()); + public static String getNanosTimeStr(String timeStr){ + if(timeStr.length() < FORMAT_TIME_NANOS_LENGTH){ + timeStr += StringUtils.repeat("0",FORMAT_TIME_NANOS_LENGTH - timeStr.length()); } return timeStr; } /** * 将边界位置时间转换成对应饿的纳秒时间 - * * @param startLocation 边界位置(起始/结束) * @return */ - public static int getNanos(long startLocation) { + public static int getNanos(long startLocation){ String timeStr = String.valueOf(startLocation); int nanos; - if (timeStr.length() == SECOND_LENGTH) { + if (timeStr.length() == SECOND_LENGTH){ nanos = 0; - } else if (timeStr.length() == MILLIS_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MILLIS_LENGTH)) * 1000000; - } else if (timeStr.length() == MICRO_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MICRO_LENGTH)) * 1000; - } else if (timeStr.length() == NANOS_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, NANOS_LENGTH)); + } else if (timeStr.length() == MILLIS_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MILLIS_LENGTH)) * 1000000; + } else if (timeStr.length() == MICRO_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MICRO_LENGTH)) * 1000; + } else if (timeStr.length() == NANOS_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,NANOS_LENGTH)); } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); } @@ -430,20 +362,19 @@ public static int getNanos(long startLocation) { /** * 将边界位置时间转换成对应饿的毫秒时间 - * - * @param startLocation 边界位置(起始/结束) + * @param startLocation 边界位置(起始/结束) * @return */ - public static long getMillis(long startLocation) { + public static long getMillis(long startLocation){ String timeStr = String.valueOf(startLocation); long millisSecond; - if (timeStr.length() == SECOND_LENGTH) { + if (timeStr.length() == SECOND_LENGTH){ millisSecond = startLocation * 1000; - } else if (timeStr.length() == MILLIS_LENGTH) { + } else if (timeStr.length() == MILLIS_LENGTH){ millisSecond = startLocation; - } else if (timeStr.length() == MICRO_LENGTH) { + } else if (timeStr.length() == MICRO_LENGTH){ millisSecond = startLocation / 1000; - } else if (timeStr.length() == NANOS_LENGTH) { + } else if (timeStr.length() == NANOS_LENGTH){ millisSecond = startLocation / 1000000; } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); @@ -454,24 +385,23 @@ public static long getMillis(long startLocation) { /** * 格式化jdbc连接 - * - * @param dbUrl 原jdbc连接 - * @param extParamMap 需要额外添加的参数 - * @return 格式化后jdbc连接URL字符串 + * @param dbUrl 原jdbc连接 + * @param extParamMap 需要额外添加的参数 + * @return 格式化后jdbc连接URL字符串 */ - public static String formatJdbcUrl(String dbUrl, Map extParamMap) { + public static String formatJdbcUrl(String dbUrl, Map extParamMap){ String[] splits = DB_PATTERN.split(dbUrl); - Map paramMap = new HashMap<>(16); - if (splits.length > 1) { + Map paramMap = new HashMap<>(16); + if(splits.length > 1) { String[] pairs = splits[1].split("&"); - for (String pair : pairs) { + for(String pair : pairs) { String[] leftRight = pair.split("="); paramMap.put(leftRight[0], leftRight[1]); } } - if (!CollectionUtil.isNullOrEmpty(extParamMap)) { + if(!CollectionUtil.isNullOrEmpty(extParamMap)){ paramMap.putAll(extParamMap); } paramMap.put("useCursorFetch", "true"); @@ -480,8 +410,8 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap StringBuffer sb = new StringBuffer(dbUrl.length() + 128); sb.append(splits[0]).append("?"); int index = 0; - for (Map.Entry entry : paramMap.entrySet()) { - if (index != 0) { + for(Map.Entry entry : paramMap.entrySet()) { + if(index != 0) { sb.append("&"); } sb.append(entry.getKey()).append("=").append(entry.getValue()); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java index 4faa32e0ce..f4442111d7 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java @@ -107,8 +107,8 @@ protected DistributedJdbcInputFormatBuilder getBuilder(){ throw new RuntimeException("子类必须覆盖getBuilder方法"); } - protected List buildConnections(){ - List sourceList = new ArrayList<>(connectionConfigs.size()); + protected ArrayList buildConnections(){ + ArrayList sourceList = new ArrayList<>(connectionConfigs.size()); for (ReaderConfig.ParameterConfig.ConnectionConfig connectionConfig : connectionConfigs) { String curUsername = (StringUtils.isBlank(connectionConfig.getUsername())) ? username : connectionConfig.getUsername(); String curPassword = (StringUtils.isBlank(connectionConfig.getPassword())) ? password : connectionConfig.getPassword(); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java index 85fd1e8db0..aec41f2601 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java @@ -30,7 +30,11 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import java.io.*; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; @@ -65,7 +69,7 @@ public class DistributedJdbcInputFormat extends BaseRichInputFormat { protected List descColumnTypeList; - protected List sourceList; + protected ArrayList sourceList; protected transient int sourceIndex; @@ -198,7 +202,7 @@ protected void closeCurrentSource(){ @Override protected void closeInternal() throws IOException { - + closeCurrentSource(); } @Override @@ -209,7 +213,7 @@ public InputSplit[] createInputSplitsInternal(int minPart) throws IOException { Object[][] parmeter = DbUtil.getParameterValues(numPartitions); for (int j = 0; j < numPartitions; j++) { DistributedJdbcInputSplit split = new DistributedJdbcInputSplit(j,numPartitions); - List sourceCopy = deepCopyList(sourceList); + ArrayList sourceCopy = deepCopyList(sourceList); for (int i = 0; i < sourceCopy.size(); i++) { sourceCopy.get(i).setSplitByKey(true); sourceCopy.get(i).setParameterValues(parmeter[j]); @@ -222,7 +226,9 @@ public InputSplit[] createInputSplitsInternal(int minPart) throws IOException { if (partNum == 0){ for (int i = 0; i < sourceList.size(); i++) { DistributedJdbcInputSplit split = new DistributedJdbcInputSplit(i,numPartitions); - split.setSourceList(Arrays.asList(sourceList.get(i))); + ArrayList arrayList = new ArrayList<>(); + arrayList.add(sourceList.get(i)); + split.setSourceList(arrayList); inputSplits[i] = split; } } else { @@ -251,7 +257,7 @@ public boolean reachedEnd() throws IOException { return readNextRecord(); } - public List deepCopyList(List src) throws IOException{ + public ArrayList deepCopyList(ArrayList src) throws IOException{ try { ByteArrayOutputStream byteOut = new ByteArrayOutputStream(); ObjectOutputStream out = new ObjectOutputStream(byteOut); @@ -259,7 +265,7 @@ public List deepCopyList(List src) throws IOException{ ByteArrayInputStream byteIn = new ByteArrayInputStream(byteOut.toByteArray()); ObjectInputStream in = new ObjectInputStream(byteIn); - List dest = (List) in.readObject(); + ArrayList dest = (ArrayList) in.readObject(); return dest; } catch (Exception e){ diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java index 28f2c2c5f1..0c8b21c07f 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java @@ -24,6 +24,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import org.apache.commons.lang.StringUtils; +import java.util.ArrayList; import java.util.List; /** @@ -65,7 +66,7 @@ public void setSplitKey(String splitKey){ format.splitKey = splitKey; } - public void setSourceList(List sourceList){ + public void setSourceList(ArrayList sourceList){ format.sourceList = sourceList; } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java index 724f9532d5..43df7f5c88 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java @@ -32,7 +32,7 @@ */ public class DistributedJdbcInputSplit extends GenericInputSplit { - private List sourceList; + private ArrayList sourceList; public DistributedJdbcInputSplit(int partitionNumber, int totalNumberOfPartitions) { super(partitionNumber, totalNumberOfPartitions); @@ -54,11 +54,11 @@ public void addSource(DataSource source){ this.sourceList.add(source); } - public List getSourceList() { + public ArrayList getSourceList() { return sourceList; } - public void setSourceList(List sourceList) { + public void setSourceList(ArrayList sourceList) { this.sourceList = sourceList; } } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index 7e0bd337b5..abea76ad8c 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -27,15 +27,15 @@ import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.util.*; import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.UrlUtil; import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.LongMaximum; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -47,9 +47,17 @@ import org.codehaus.jackson.map.ObjectMapper; import java.io.IOException; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Date; -import java.util.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; /** @@ -162,7 +170,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { initMetric(inputSplit); String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -260,6 +270,12 @@ public Row nextRecordInternal(Row row) throws IOException { boolean isUpdateLocation = incrementConfig.isPolling() || (incrementConfig.isIncrement() && !incrementConfig.isUseMaxFunc()); if (isUpdateLocation) { Object incrementVal = resultSet.getObject(incrementConfig.getColumnName()); + if(incrementVal != null) { + if((incrementVal instanceof java.util.Date + || incrementVal.getClass().getSimpleName().toUpperCase().contains("TIMESTAMP")) ) { + incrementVal = resultSet.getTimestamp(incrementConfig.getColumnName()); + } + } String location; if(incrementConfig.isPolling()){ location = String.valueOf(incrementVal); @@ -379,7 +395,7 @@ public String getMaxValueFromApi(){ } String url = monitorUrls; - if (monitorUrls.startsWith("http")) { + if (monitorUrls.startsWith(ConstantValue.PROTOCOL_HTTP)) { url = String.format("%s/jobs/%s/accumulators", monitorUrls, jobId); } @@ -715,10 +731,10 @@ private String getLocation(String columnType, Object columnVal) { long time = ((Timestamp) columnVal).getTime() / 1000; String nanosStr = String.valueOf(((Timestamp) columnVal).getNanos()); - if (nanosStr.length() == 9) { + if (nanosStr.length() == DbUtil.NANOS_PART_LENGTH) { location = time + nanosStr; } else { - String fillZeroStr = StringUtils.repeat("0", 9 - nanosStr.length()); + String fillZeroStr = StringUtils.repeat("0", DbUtil.NANOS_PART_LENGTH - nanosStr.length()); location = time + fillZeroStr + nanosStr; } } else { @@ -816,7 +832,7 @@ protected void executeQuery(String startLocation) throws SQLException { queryForPolling(startLocation); } } else { - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + statement = dbConn.createStatement(resultSetType, resultSetConcurrency); statement.setFetchSize(fetchSize); statement.setQueryTimeout(queryTimeOut); resultSet = statement.executeQuery(querySql); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java index cb48f98b59..ae788328f2 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.rdb.inputformat; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.datareader.IncrementConfig; @@ -128,6 +129,10 @@ protected void checkFormat() { if (StringUtils.isEmpty(format.splitKey) && format.numPartitions > 1){ throw new IllegalArgumentException("Must specify the split column when the channel is greater than 1"); } + + if (format.fetchSize > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取条数必须小于[200000]条"); + } } } diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java index 2a94afb8ff..a07eb3473c 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java @@ -32,7 +32,15 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.*; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_BATCH_SIZE; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_FULL_COLUMN; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_INSERT_SQL_MODE; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_POST_SQL; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_PRE_SQL; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_UPDATE_KEY; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_USERNAME; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_WRITE_MODE; /** * The Writer plugin for any database that can be connected via JDBC. diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java index bd2dc5afe1..47ab767ef3 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java @@ -30,12 +30,19 @@ import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.ObjectUtils; +import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -107,6 +114,7 @@ public class JdbcOutputFormat extends BaseRichOutputFormat { "AND t.table_name = '%s'"; protected final static String CONN_CLOSE_ERROR_MSG = "No operations allowed"; + protected static List STRING_TYPES = Arrays.asList("CHAR", "VARCHAR","TINYBLOB","TINYTEXT","BLOB","TEXT", "MEDIUMBLOB", "MEDIUMTEXT", "LONGBLOB", "LONGTEXT"); protected PreparedStatement prepareTemplates() throws SQLException { if(CollectionUtils.isEmpty(fullColumn)) { @@ -202,7 +210,13 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { int index = 0; try { for (; index < row.getArity(); index++) { - preparedStatement.setObject(index+1,getField(row,index)); + Object object = row.getField(index); + if( object instanceof String && StringUtils.isBlank((String) object)){ + if(!STRING_TYPES.contains(columnType.get(index))){ + object = null; + } + } + preparedStatement.setObject(index+1, object); } preparedStatement.execute(); @@ -233,8 +247,14 @@ protected String recordConvertDetailErrorMessage(int pos, Row row) { protected void writeMultipleRecordsInternal() throws Exception { try { for (Row row : rows) { - for (int j = 0; j < row.getArity(); ++j) { - preparedStatement.setObject(j + 1, getField(row, j)); + for (int index = 0; index < row.getArity(); index++) { + Object object = row.getField(index); + if( object instanceof String && StringUtils.isBlank((String) object)){ + if(!STRING_TYPES.contains(columnType.get(index))){ + object = null; + } + } + preparedStatement.setObject(index+1, object); } preparedStatement.addBatch(); diff --git a/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java b/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java index 31f40ecb64..9aed6f527b 100644 --- a/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java +++ b/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java @@ -19,13 +19,18 @@ package com.dtstack.flinkx.redis; import com.dtstack.flinkx.util.TelnetUtil; -import redis.clients.jedis.*; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisPoolConfig; import java.util.Properties; import java.util.regex.Matcher; import java.util.regex.Pattern; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; /** * Utilities for redis database connection diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 4a92436c6b..891f05acf5 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -56,8 +56,12 @@ shade.rediswriter.io.netty - com.google - shade.rediswriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java index 7fe2fe9358..2fc85d1fd3 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java @@ -30,9 +30,15 @@ import java.io.IOException; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Properties; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; /** * OutputFormat for writing data to redis database. @@ -70,6 +76,8 @@ public class RedisOutputFormat extends BaseRichOutputFormat { private static final int CRITICAL_TIME = 60 * 60 * 24 * 30; + private static final int KEY_VALUE_SIZE = 2; + @Override public void configure(Configuration parameters) { super.configure(parameters); @@ -138,7 +146,7 @@ private void processTimeFormat(Row row){ } private List getFieldAndValue(Row row){ - if(row.getArity() - keyIndexes.size() != 2){ + if(row.getArity() - keyIndexes.size() != KEY_VALUE_SIZE){ throw new IllegalArgumentException("Each row record can have only one pair of attributes and values except key"); } @@ -185,7 +193,7 @@ private String concatKey(Row row){ @Override protected void writeMultipleRecordsInternal() throws Exception { - // Still not supported + notSupportBatchWrite("RedisWriter"); } @Override diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java index e9480525ae..3b291ece1c 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java @@ -95,5 +95,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("RedisWriter"); } } diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java index c64e2a3b29..fa33f2c739 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java @@ -31,7 +31,18 @@ import java.util.ArrayList; import java.util.List; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_BATCH_SIZE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DATE_FORMAT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_EXPIRE_TIME; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_KEY_FIELD_DELIMITER; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_KEY_INDEXES; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_MODE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TYPE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_VALUE_FIELD_DELIMITER; /** * The writer plugin for redis database diff --git a/flinkx-restapi/flinkx-restapi-core/pom.xml b/flinkx-restapi/flinkx-restapi-core/pom.xml new file mode 100644 index 0000000000..742c77488d --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/pom.xml @@ -0,0 +1,20 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-core + + + org.apache.httpcomponents + httpclient + ${http.version} + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java new file mode 100644 index 0000000000..338ba84b1c --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.restapi.common; + +/** + * @author : tiezhu + * @date : 2020/3/13 + */ +public enum HttpMethod { + // http 请求方式 + GET, + POST, + PUT, + PATCH, + DELETE, + COPY, + HEAD, + OPTIONS, + LINK, + UNLINK, + PURGE, + LOCK, + UNLOCK, + PROPFIND, + VIEW + ; + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java new file mode 100644 index 0000000000..b37770ecb6 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import com.dtstack.flinkx.util.GsonUtil; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/16 + */ +public class HttpUtil { + protected static final Logger LOG = LoggerFactory.getLogger(HttpUtil.class); + private static final int COUNT = 32; + private static final int TOTAL_COUNT = 1000; + private static final int TIME_OUT = 5000; + private static final int EXECUTION_COUNT = 5; + + public static CloseableHttpClient getHttpClient() { + // 设置自定义的重试策略 + MyServiceUnavailableRetryStrategy strategy = new MyServiceUnavailableRetryStrategy + .Builder() + .executionCount(EXECUTION_COUNT) + .retryInterval(1000) + .build(); + // 设置自定义的重试Handler + MyHttpRequestRetryHandler retryHandler = new MyHttpRequestRetryHandler + .Builder() + .executionCount(EXECUTION_COUNT) + .build(); + // 设置超时时间 + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(TIME_OUT) + .setConnectionRequestTimeout(TIME_OUT) + .setSocketTimeout(TIME_OUT) + .build(); + // 设置Http连接池 + PoolingHttpClientConnectionManager pcm = new PoolingHttpClientConnectionManager(); + pcm.setDefaultMaxPerRoute(COUNT); + pcm.setMaxTotal(TOTAL_COUNT); + + return HttpClientBuilder.create() + .setServiceUnavailableRetryStrategy(strategy) + .setRetryHandler(retryHandler) + .setDefaultRequestConfig(requestConfig) + .setConnectionManager(pcm) + .build(); +// return HttpClientBuilder.create().build(); + } + + public static HttpRequestBase getRequest(String method, + Map requestBody, + Map header, + String url) { + LOG.debug("current request url: {} current method:{} \n", url, method); + HttpRequestBase request = null; + + if (HttpMethod.GET.name().equalsIgnoreCase(method)) { + request = new HttpGet(url); + } else if (HttpMethod.POST.name().equalsIgnoreCase(method)) { + HttpPost post = new HttpPost(url); + post.setEntity(getEntityData(requestBody)); + request = post; + } else { + throw new RuntimeException("Unsupported method:" + method); + } + + for (Map.Entry entry : header.entrySet()) { + request.addHeader(entry.getKey(), entry.getValue()); + } + return request; + } + + public static void closeClient(CloseableHttpClient httpClient) { + try { + httpClient.close(); + } catch (IOException e) { + throw new RuntimeException("close client error"); + } + } + + public static StringEntity getEntityData(Map body) { + StringEntity stringEntity = new StringEntity(GsonUtil.GSON.toJson(body), StandardCharsets.UTF_8); + stringEntity.setContentEncoding(StandardCharsets.UTF_8.name()); + return stringEntity; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java new file mode 100644 index 0000000000..e0689cb658 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.http.HttpEntityEnclosingRequest; +import org.apache.http.HttpRequest; +import org.apache.http.NoHttpResponseException; +import org.apache.http.client.HttpRequestRetryHandler; +import org.apache.http.client.protocol.HttpClientContext; +import org.apache.http.protocol.HttpContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.UnknownHostException; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class MyHttpRequestRetryHandler implements HttpRequestRetryHandler { + protected static final Logger LOG = LoggerFactory.getLogger(MyHttpRequestRetryHandler.class); + + private int executionMaxCount; + + public MyHttpRequestRetryHandler(Builder builder) { + this.executionMaxCount = builder.executionMaxCount; + } + + @Override + public boolean retryRequest(IOException exception, int executionCount, HttpContext context) { + LOG.info("第" + executionCount + "次重试"); + + if (executionCount >= this.executionMaxCount) { + // Do not retry if over max retry count + return false; + } + if (exception instanceof InterruptedIOException) { + // Timeout + return true; + } + if (exception instanceof UnknownHostException) { + // Unknown host + return true; + } + if (exception instanceof SSLException) { + // SSL handshake exception + return true; + } + if (exception instanceof NoHttpResponseException) { + // No response + return true; + } + + HttpClientContext clientContext = HttpClientContext.adapt(context); + HttpRequest request = clientContext.getRequest(); + boolean idempotent = !(request instanceof HttpEntityEnclosingRequest); + // Retry if the request is considered idempotent + return !idempotent; + } + + + public static final class Builder { + private int executionMaxCount; + + public Builder() { + executionMaxCount = 5; + } + + public Builder executionCount(int executionCount) { + this.executionMaxCount = executionCount; + return this; + } + + public MyHttpRequestRetryHandler build() { + return new MyHttpRequestRetryHandler(this); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java new file mode 100644 index 0000000000..c215926588 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.http.HttpResponse; +import org.apache.http.client.ServiceUnavailableRetryStrategy; +import org.apache.http.protocol.HttpContext; + +/** + * @author : tiezhu + * @date : 2020/3/12 + * 自定义httpClient重试策略,默认重试次数为5,重试时间间隔为2s + */ +public class MyServiceUnavailableRetryStrategy implements ServiceUnavailableRetryStrategy { + private int executionCount; + private long retryInterval; + + public MyServiceUnavailableRetryStrategy(Builder builder) { + this.executionCount = builder.executionCount; + this.retryInterval = builder.retryInterval; + } + + @Override + public boolean retryRequest(HttpResponse httpResponse, int executionCount, HttpContext httpContext) { + int successCode = 200; + return httpResponse.getStatusLine().getStatusCode() != successCode + && executionCount < this.executionCount; + } + + @Override + public long getRetryInterval() { + return this.retryInterval; + } + + public static final class Builder { + private int executionCount; + private long retryInterval; + + public Builder() { + executionCount = 5; + retryInterval = 2000; + } + + public Builder executionCount(int executionCount) { + this.executionCount = executionCount; + return this; + } + + public Builder retryInterval(long retryInterval) { + this.retryInterval = retryInterval; + return this; + } + + public MyServiceUnavailableRetryStrategy build() { + return new MyServiceUnavailableRetryStrategy(this); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java new file mode 100644 index 0000000000..8821e3777c --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * @author : tiezhu + * @date : 2020/3/19 + */ +public class RestapiKeys { + public static final String KEY_METHOD = "method"; + public static final String KEY_HEADER = "header"; + public static final String KEY_BODY = "body"; + public static final String KEY_PARAMS = "params"; + public static final String KEY_COLUMN = "column"; + public static final String KEY_URL = "url"; + public static final String KEY_BATCH_INTERVAL = "batchInterval"; +} diff --git a/flinkx-restapi/flinkx-restapi-reader/pom.xml b/flinkx-restapi/flinkx-restapi-reader/pom.xml new file mode 100644 index 0000000000..8e27c23493 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/pom.xml @@ -0,0 +1,99 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-reader + + + + com.dtstack.flinkx + flinkx-restapi-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.restapireader.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java new file mode 100644 index 0000000000..59c6b535de --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.inputformat; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.restapi.common.HttpUtil; +import com.dtstack.flinkx.util.GsonUtil; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; + +import java.io.IOException; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiInputFormat extends BaseRichInputFormat { + + protected String url; + + protected String method; + + protected transient CloseableHttpClient httpClient; + + protected Map header; + + protected Map entityDataToMap; + + protected boolean getData; + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + httpClient = HttpUtil.getHttpClient(); + } + + @Override + public void closeInputFormat() { + HttpUtil.closeClient(httpClient); + } + + + @Override + @SuppressWarnings("unchecked") + protected void openInternal(InputSplit inputSplit) throws IOException { + HttpUriRequest request = HttpUtil.getRequest(method, header,null, url); + try { + CloseableHttpResponse httpResponse = httpClient.execute(request); + HttpEntity entity = httpResponse.getEntity(); + if (entity != null) { + String entityData = EntityUtils.toString(entity); + entityDataToMap = GsonUtil.GSON.fromJson(entityData, Map.class); + getData = true; + } else { + throw new RuntimeException("entity is null"); + } + } catch (Exception e) { + throw new RuntimeException("get entity error"); + } + } + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { + InputSplit[] inputSplits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + inputSplits[i] = new GenericInputSplit(i, minNumSplits); + } + return inputSplits; + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + row = new Row(1); + row.setField(0, entityDataToMap); + getData = false; + return row; + } + + @Override + protected void closeInternal() throws IOException { + } + + @Override + public boolean reachedEnd() throws IOException { + return !getData; + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java new file mode 100644 index 0000000000..ac636b4038 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.inputformat; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; + +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiInputFormatBuilder extends BaseRichInputFormatBuilder { + protected RestapiInputFormat format; + + public RestapiInputFormatBuilder(){ super.format = format = new RestapiInputFormat();} + + public void setUrl(String url){this.format.url = url;} + public void setHeader(Map header){ this.format.header = header;} + public void setMethod(String method){ this.format.method = method;} + + @Override + protected void checkFormat() { + if(format.url.isEmpty()){ + throw new IllegalArgumentException("缺少url"); + } + if (format.method.isEmpty()) { + throw new IllegalArgumentException("缺少method"); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java new file mode 100644 index 0000000000..162de36054 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.restapi.inputformat.RestapiInputFormatBuilder; +import com.google.common.collect.Maps; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiReader extends BaseDataReader { + + private String url; + + private String method; + + private Map header = Maps.newHashMap(); + + private ArrayList> temp; + + @SuppressWarnings("unchecked") + public RestapiReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + + url = readerConfig.getParameter().getStringVal("url"); + method = readerConfig.getParameter().getStringVal("method"); + temp = (ArrayList>) readerConfig.getParameter().getVal("header"); + if (temp != null) { + for (Map map : temp) { + header.putAll(map); + } + } + } + + @Override + public DataStream readData() { + RestapiInputFormatBuilder builder = new RestapiInputFormatBuilder(); + + builder.setHeader(header); + builder.setMethod(method); + builder.setUrl(url); + + return createInput(builder.finish()); + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/pom.xml b/flinkx-restapi/flinkx-restapi-writer/pom.xml new file mode 100644 index 0000000000..00a801c2b1 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/pom.xml @@ -0,0 +1,99 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-writer + + + + com.dtstack.flinkx + flinkx-restapi-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.restapiwriter.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java new file mode 100644 index 0000000000..ce96f57a74 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.outputformat; + +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.restapi.common.HttpUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.google.common.collect.Maps; +import org.apache.flink.types.Row; +import org.apache.http.HttpStatus; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.impl.client.CloseableHttpClient; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + * 当前只考虑了元数据读取,和带有字段名column读取的情况,其他情况暂未考虑 + */ +public class RestapiOutputFormat extends BaseRichOutputFormat { + + protected String url; + + protected String method; + + protected ArrayList column; + + protected Map params; + + protected Map body; + + protected Map header; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + // Nothing to do + } + + @Override + protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + LOG.info("start write single record"); + CloseableHttpClient httpClient = HttpUtil.getHttpClient(); + int index = 0; + Map requestBody = Maps.newHashMap(); + Object dataRow; + try { + dataRow = getDataFromRow(row, column); + if (!params.isEmpty()) { + Iterator iterator = params.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = (Map.Entry) iterator.next(); + body.put((String) entry.getKey(), entry.getValue()); + } + } + body.put("data", dataRow); + requestBody.put("json", body); + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + sendRequest(httpClient, requestBody, method, header, url); + } catch (Exception e) { + requestErrorMessage(e, index, row); + } finally { + // 最后不管发送是否成功,都要关闭client + HttpUtil.closeClient(httpClient); + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + LOG.info("start write multiple records"); + try { + CloseableHttpClient httpClient = HttpUtil.getHttpClient(); + List dataRow = new ArrayList<>(); + Map requestBody = Maps.newHashMap(); + for (Row row : rows) { + dataRow.add(getDataFromRow(row, column)); + } + if (!params.isEmpty()) { + Iterator iterator = params.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = (Map.Entry) iterator.next(); + body.put((String) entry.getKey(), entry.getValue()); + } + } + body.put("data", dataRow); + requestBody.put("json", body); + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + sendRequest(httpClient, requestBody, method, header, url); + } catch (Exception e) { + LOG.warn("write record error !", e); + } + } + + private void requestErrorMessage(Exception e, int index, Row row) { + if (index < row.getArity()) { + recordConvertDetailErrorMessage(index, row); + LOG.warn("添加脏数据:" + row.getField(index)); + } + } + + private Object getDataFromRow(Row row, List column) throws IOException { + Map columnData = Maps.newHashMap(); + int index = 0; + if (!column.isEmpty()) { + // 如果column不为空,那么将数据和字段名一一对应 + for (; index < row.getArity(); index++) { + columnData.put(column.get(index), row.getField(index)); + } + return GsonUtil.GSON.toJson(columnData); + } else { + return row.getField(index); + } + } + + + private void sendRequest(CloseableHttpClient httpClient, + Map requestBody, + String method, + Map header, + String url) throws IOException { + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + HttpRequestBase request = HttpUtil.getRequest(method, requestBody, header, url); + CloseableHttpResponse httpResponse = httpClient.execute(request); + // 重试之后返回状态码不为200 + if (httpResponse.getStatusLine().getStatusCode() != HttpStatus.SC_OK) { + LOG.warn("重试之后当前请求状态码为" + httpResponse.getStatusLine().getStatusCode()); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java new file mode 100644 index 0000000000..3ab4751ea0 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.outputformat; + +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; + +import java.util.ArrayList; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private RestapiOutputFormat format; + + public RestapiOutputFormatBuilder() { + super.format = format = new RestapiOutputFormat(); + } + + public void setUrl(String url) { + this.format.url = url; + } + + public void setHeader(Map header) { + this.format.header = header; + } + + public void setMethod(String method) { + this.format.method = method; + } + + public void setBody(Map body) { + this.format.body = body; + } + + public void setColumn(ArrayList column) { + format.column = column; + } + + public void setParams(Map params){ + format.params = params; + } + + + @Override + protected void checkFormat() { + if (format.url.isEmpty()) { + throw new IllegalArgumentException("缺少url"); + } + if (format.method.isEmpty()) { + throw new IllegalArgumentException("缺少method"); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java new file mode 100644 index 0000000000..f050abb1b9 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.restapi.common.RestapiKeys; +import com.dtstack.flinkx.restapi.outputformat.RestapiOutputFormatBuilder; +import com.dtstack.flinkx.writer.BaseDataWriter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiWriter extends BaseDataWriter { + + protected String url; + + protected String method; + + protected Map header = Maps.newHashMap(); + + protected Map body =Maps.newHashMap(); + + protected ArrayList column = Lists.newArrayList(); + + protected Map params = Maps.newHashMap(); + + protected int batchInterval; + + @SuppressWarnings("unchecked") + public RestapiWriter(DataTransferConfig config) { + super(config); + Object tempObj; + + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + + url = writerConfig.getParameter().getStringVal(RestapiKeys.KEY_URL); + method = writerConfig.getParameter().getStringVal(RestapiKeys.KEY_METHOD); + batchInterval = writerConfig.getParameter().getIntVal(RestapiKeys.KEY_BATCH_INTERVAL, 1); + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_COLUMN); + if (tempObj != null) { + column.addAll((ArrayList) tempObj); + } + + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_HEADER); + if (tempObj != null) { + for (Map map : (ArrayList>) tempObj) { + header.putAll(map); + } + } + + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_BODY); + if (tempObj != null) { + for (Map map : (ArrayList>) tempObj) { + body.putAll(map); + } + } + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_PARAMS); + if (tempObj != null) { + params = (HashMap) tempObj; + } + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + RestapiOutputFormatBuilder builder = new RestapiOutputFormatBuilder(); + + builder.setHeader(header); + builder.setMethod(method); + builder.setUrl(url); + builder.setBody(body); + builder.setColumn(column); + builder.setParams(params); + builder.setBatchInterval(batchInterval); + + return createOutput(dataSet, builder.finish()); + } +} diff --git a/flinkx-restapi/pom.xml b/flinkx-restapi/pom.xml new file mode 100644 index 0000000000..876877c87a --- /dev/null +++ b/flinkx-restapi/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi + pom + + flinkx-restapi-core + flinkx-restapi-writer + flinkx-restapi-reader + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-core/pom.xml b/flinkx-saphana/flinkx-saphana-core/pom.xml index bed09b714f..5ac7f520b5 100644 --- a/flinkx-saphana/flinkx-saphana-core/pom.xml +++ b/flinkx-saphana/flinkx-saphana-core/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index ba9e5c4375..8b170aecf7 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 8c19fad040..233c1fc5ae 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/pom.xml b/flinkx-saphana/pom.xml index a9ecb0501e..e36168c903 100644 --- a/flinkx-saphana/pom.xml +++ b/flinkx-saphana/pom.xml @@ -1,6 +1,6 @@ - flinkx-all diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index 7300b9251f..1cd8f983b1 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -62,8 +62,12 @@ shade.sqlserverreader.io.netty - com.google - shade.sqlserverreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index e4f6ed9b36..f9d45aeb84 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -63,8 +63,12 @@ shade.sqlserverwriter.io.netty - com.google - shade.sqlserverwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index d2dfa853cc..3586e32383 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -49,6 +49,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index f106065763..8466152676 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -45,6 +45,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java index da9a167b59..ac578d2c86 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java @@ -48,7 +48,12 @@ protected void openInternal(int taskNumber, int numTasks) throws IOException { @Override protected void writeSingleRecordInternal(Row row) throws WriteRecordException { if (print) { - LOG.info("subTaskIndex[{}]:{}", taskNumber, row); + LOG.info("subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); + } + + if (restoreConfig.isRestore()) { + formatState.setState(row.getField(restoreConfig.getRestoreColumnIndex())); + LOG.info("print data subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); } } @@ -56,8 +61,19 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { protected void writeMultipleRecordsInternal() throws Exception { if (print) { for (Row row : rows) { - LOG.info(String.valueOf(row)); + LOG.info(rowToStringWithDelimiter(row, writeDelimiter)); + } + } + } + + public String rowToStringWithDelimiter(Row row, String writeDelimiter) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getArity(); i++) { + if (i > 0) { + sb.append(writeDelimiter); } + sb.append(StringUtils.arrayAwareToString(row.getField(i))); } + return sb.toString(); } } diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java index 246a4eb8bc..05344b6494 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java @@ -46,7 +46,7 @@ public StreamWriter(DataTransferConfig config) { super(config); print = config.getJob().getContent().get(0).getWriter().getParameter().getBooleanVal("print",false); writeDelimiter = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal("writeDelimiter", "|"); - batchInterval = config.getJob().getContent().get(0).getWriter().getParameter().getIntVal("batchInterval", 20); + batchInterval = config.getJob().getContent().get(0).getWriter().getParameter().getIntVal("batchInterval", 1); List column = config.getJob().getContent().get(0).getWriter().getParameter().getColumn(); metaColumns = MetaColumn.getMetaColumns(column); diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index 0d36c0ef17..b37ab81b88 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -16,351 +16,432 @@ + ch.qos.logback logback-classic 1.1.7 + com.google.guava guava 19.0 + com.google.code.gson gson 2.7 + hadoop-mapreduce-client-core org.apache.hadoop ${hadoop.version} + hadoop-common org.apache.hadoop ${hadoop.version} + hadoop-hdfs org.apache.hadoop ${hadoop.version} + org.apache.flink flink-metrics-core - ${flink.version} + 1.8.1 + org.apache.flink flink-metrics-prometheus_2.11 ${flink.version} + io.prometheus simpleclient ${prometheus.version} + io.prometheus simpleclient_httpserver ${prometheus.version} + io.prometheus simpleclient_pushgateway ${prometheus.version} + com.dtstack.flinkx flinkx-core - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-stream-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-stream-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-carbondata-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-carbondata-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-db2-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-db2-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-es-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-es-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-ftp-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-ftp-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hbase-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-hbase-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hdfs-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-hdfs-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-mongodb-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mongodb-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-dreader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-odps-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-odps-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-oracle-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-oracle-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-postgresql-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-postgresql-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-sqlserver-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-sqlserver-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-redis-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-core - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-gbase-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-gbase-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-binlog-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hive-writer - ${flinkx.version} + 1.6 + org.apache.hive hive-serde 2.1.0 + com.dtstack.flinkx flinkx-kafka11-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka11-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka10-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka10-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka09-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka09-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kudu-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kudu-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-clickhouse-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-clickhouse-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-polardb-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-polardb-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-phoenix-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-phoenix-writer - ${flinkx.version} + 1.6 + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test-jar + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test-jar + + + + org.apache.derby + derby + 10.14.2.0 + test + com.dtstack.flinkx flinkx-emqx-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-emqx-writer - ${flinkx.version} + 1.6 + + com.dtstack.flinkx + flinkx-restapi-writer + 1.6 + + com.dtstack.flinkx flinkx-dm-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-dm-writer - ${flinkx.version} + 1.6 + + + com.dtstack.flinkx + flinkx-greenplum-reader + 1.6 + + + com.dtstack.flinkx + flinkx-greenplum-writer + 1.6 diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 33fc1dd035..27883b37a4 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -41,6 +41,8 @@ import com.dtstack.flinkx.ftp.writer.FtpWriter; import com.dtstack.flinkx.gbase.reader.GbaseReader; import com.dtstack.flinkx.gbase.writer.GbaseWriter; +import com.dtstack.flinkx.greenplum.reader.GreenplumReader; +import com.dtstack.flinkx.greenplum.writer.GreenplumWriter; import com.dtstack.flinkx.hbase.reader.HbaseReader; import com.dtstack.flinkx.hbase.writer.HbaseWriter; import com.dtstack.flinkx.hdfs.reader.HdfsReader; @@ -73,6 +75,7 @@ import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.redis.writer.RedisWriter; +import com.dtstack.flinkx.restapi.writer.RestapiWriter; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; import com.dtstack.flinkx.stream.reader.StreamReader; @@ -84,7 +87,9 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -112,21 +117,18 @@ public class LocalTest { public static Logger LOG = LoggerFactory.getLogger(LocalTest.class); public static Configuration conf = new Configuration(); - public static void main(String[] args) throws Exception { - setLogLevel(Level.INFO.toString()); - + public static void main(String[] args) throws Exception{ + setLogLevel(Level.DEBUG.toString()); Properties confProperties = new Properties(); // confProperties.put("flink.checkpoint.interval", "10000"); // confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); -// - conf.setString("akka.ask.timeout", "180 s"); - conf.setString("web.timeout", "100000"); + // conf.setString("metrics.reporter.promgateway.class","org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"); -// conf.setString("metrics.reporter.promgateway.host","172.16.8.178"); +// conf.setString("metrics.reporter.promgateway.host","127.0.0.1"); // conf.setString("metrics.reporter.promgateway.port","9091"); -// conf.setString("metrics.reporter.promgateway.jobName","kanata"); +// conf.setString("metrics.reporter.promgateway.jobName","108job"); // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); -// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","false"); +// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); String jobPath = "D:\\dtstack\\flinkx-all\\flinkx-examples\\examples\\clickhouse_stream.json"; String savePointPath = ""; @@ -134,17 +136,29 @@ public static void main(String[] args) throws Exception { ResultPrintUtil.printResult(result); } - public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savePointPath) throws Exception { + public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savepointPath) throws Exception{ String jobContent = readJob(jobFile); - return runJob(jobContent, confProperties, savePointPath); + return runJob(jobContent, confProperties, savepointPath); } - public static JobExecutionResult runJob(String job, Properties confProperties, String savePointPath) throws Exception { + public static JobExecutionResult runJob(String job, Properties confProperties, String savepointPath) throws Exception{ DataTransferConfig config = DataTransferConfig.parse(job); + + conf.setString("akka.ask.timeout", "180 s"); + conf.setString("web.timeout", String.valueOf(100000)); + MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); + openCheckpointConf(env, confProperties); + env.setParallelism(config.getJob().getSetting().getSpeed().getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); + + if (needRestart(config)) { + env.setRestartStrategy(RestartStrategies.fixedDelayRestart( + 10, + Time.of(10, TimeUnit.SECONDS) + )); + } BaseDataReader reader = buildDataReader(config, env); DataStream dataStream = reader.readData(); @@ -163,209 +177,110 @@ public static JobExecutionResult runJob(String job, Properties confProperties, S dataStreamSink.setParallelism(speedConfig.getWriterChannel()); } - if (StringUtils.isNotEmpty(savePointPath)) { - env.setSettings(SavepointRestoreSettings.forPath(savePointPath)); + if(StringUtils.isNotEmpty(savepointPath)){ + env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); } return env.execute(); } + private static boolean needRestart(DataTransferConfig config){ + return config.getJob().getSetting().getRestoreConfig().isRestore(); + } + private static String readJob(File file) { - try(FileInputStream in = new FileInputStream(file);) { + try(FileInputStream in = new FileInputStream(file)) { byte[] fileContent = new byte[(int) file.length()]; in.read(fileContent); return new String(fileContent, StandardCharsets.UTF_8); - } catch (Exception e) { + } catch (Exception e){ throw new RuntimeException(e); } } - private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env) { + private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env){ String readerName = config.getJob().getContent().get(0).getReader().getName(); - BaseDataReader reader; - switch (readerName) { - case PluginNameConstants.STREAM_READER: - reader = new StreamReader(config, env); - break; - case PluginNameConstants.CARBONDATA_READER: - reader = new CarbondataReader(config, env); - break; - case PluginNameConstants.ORACLE_READER: - reader = new OracleReader(config, env); - break; - case PluginNameConstants.POSTGRESQL_READER: - reader = new PostgresqlReader(config, env); - break; - case PluginNameConstants.SQLSERVER_READER: - reader = new SqlserverReader(config, env); - break; - case PluginNameConstants.MYSQLD_READER: - reader = new MysqldReader(config, env); - break; - case PluginNameConstants.MYSQL_READER: - reader = new MysqlReader(config, env); - break; - case PluginNameConstants.DB2_READER: - reader = new Db2Reader(config, env); - break; - case PluginNameConstants.GBASE_READER: - reader = new GbaseReader(config, env); - break; - case PluginNameConstants.ES_READER: - reader = new EsReader(config, env); - break; - case PluginNameConstants.FTP_READER: - reader = new FtpReader(config, env); - break; - case PluginNameConstants.HBASE_READER: - reader = new HbaseReader(config, env); - break; - case PluginNameConstants.HDFS_READER: - reader = new HdfsReader(config, env); - break; - case PluginNameConstants.MONGODB_READER: - reader = new MongodbReader(config, env); - break; - case PluginNameConstants.ODPS_READER: - reader = new OdpsReader(config, env); - break; - case PluginNameConstants.BINLOG_READER: - reader = new BinlogReader(config, env); - break; - case PluginNameConstants.KAFKA09_READER: - reader = new Kafka09Reader(config, env); - break; - case PluginNameConstants.KAFKA10_READER: - reader = new Kafka10Reader(config, env); - break; - case PluginNameConstants.KAFKA11_READER: - reader = new Kafka11Reader(config, env); - break; - case PluginNameConstants.KAFKA_READER: - reader = new KafkaReader(config, env); - break; - case PluginNameConstants.KUDU_READER: - reader = new KuduReader(config, env); - break; - case PluginNameConstants.CLICKHOUSE_READER: - reader = new ClickhouseReader(config, env); - break; - case PluginNameConstants.POLARDB_READER: - reader = new PolardbReader(config, env); - break; - case PluginNameConstants.PHOENIX_READER: - reader = new PhoenixReader(config, env); - break; - case PluginNameConstants.EMQX_READER: - reader = new EmqxReader(config, env); - break; - case PluginNameConstants.DM_READER: - reader = new DmReader(config, env); - break; - default: - throw new IllegalArgumentException("Can not find reader by name:" + readerName); + BaseDataReader reader ; + switch (readerName){ + case PluginNameConstants.STREAM_READER : reader = new StreamReader(config, env); break; + case PluginNameConstants.CARBONDATA_READER : reader = new CarbondataReader(config, env); break; + case PluginNameConstants.ORACLE_READER : reader = new OracleReader(config, env); break; + case PluginNameConstants.POSTGRESQL_READER : reader = new PostgresqlReader(config, env); break; + case PluginNameConstants.SQLSERVER_READER : reader = new SqlserverReader(config, env); break; + case PluginNameConstants.MYSQLD_READER : reader = new MysqldReader(config, env); break; + case PluginNameConstants.MYSQL_READER : reader = new MysqlReader(config, env); break; + case PluginNameConstants.DB2_READER : reader = new Db2Reader(config, env); break; + case PluginNameConstants.GBASE_READER : reader = new GbaseReader(config, env); break; + case PluginNameConstants.ES_READER : reader = new EsReader(config, env); break; + case PluginNameConstants.FTP_READER : reader = new FtpReader(config, env); break; + case PluginNameConstants.HBASE_READER : reader = new HbaseReader(config, env); break; + case PluginNameConstants.HDFS_READER : reader = new HdfsReader(config, env); break; + case PluginNameConstants.MONGODB_READER : reader = new MongodbReader(config, env); break; + case PluginNameConstants.ODPS_READER : reader = new OdpsReader(config, env); break; + case PluginNameConstants.BINLOG_READER : reader = new BinlogReader(config, env); break; + case PluginNameConstants.KAFKA09_READER : reader = new Kafka09Reader(config, env); break; + case PluginNameConstants.KAFKA10_READER : reader = new Kafka10Reader(config, env); break; + case PluginNameConstants.KAFKA11_READER : reader = new Kafka11Reader(config, env); break; + case PluginNameConstants.KAFKA_READER : reader = new KafkaReader(config, env); break; + case PluginNameConstants.KUDU_READER : reader = new KuduReader(config, env); break; + case PluginNameConstants.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; + case PluginNameConstants.POLARDB_READER : reader = new PolardbReader(config, env); break; + case PluginNameConstants.PHOENIX_READER : reader = new PhoenixReader(config, env); break; + case PluginNameConstants.EMQX_READER : reader = new EmqxReader(config, env); break; + case PluginNameConstants.DM_READER : reader = new DmReader(config, env); break; + case PluginNameConstants.GREENPLUM_READER : reader = new GreenplumReader(config, env); break; + default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } return reader; } - private static BaseDataWriter buildDataWriter(DataTransferConfig config) { + private static BaseDataWriter buildDataWriter(DataTransferConfig config){ String writerName = config.getJob().getContent().get(0).getWriter().getName(); BaseDataWriter writer; - switch (writerName) { - case PluginNameConstants.STREAM_WRITER: - writer = new StreamWriter(config); - break; - case PluginNameConstants.CARBONDATA_WRITER: - writer = new CarbondataWriter(config); - break; - case PluginNameConstants.MYSQL_WRITER: - writer = new MysqlWriter(config); - break; - case PluginNameConstants.SQLSERVER_WRITER: - writer = new SqlserverWriter(config); - break; - case PluginNameConstants.ORACLE_WRITER: - writer = new OracleWriter(config); - break; - case PluginNameConstants.POSTGRESQL_WRITER: - writer = new PostgresqlWriter(config); - break; - case PluginNameConstants.DB2_WRITER: - writer = new Db2Writer(config); - break; - case PluginNameConstants.GBASE_WRITER: - writer = new GbaseWriter(config); - break; - case PluginNameConstants.ES_WRITER: - writer = new EsWriter(config); - break; - case PluginNameConstants.FTP_WRITER: - writer = new FtpWriter(config); - break; - case PluginNameConstants.HBASE_WRITER: - writer = new HbaseWriter(config); - break; - case PluginNameConstants.HDFS_WRITER: - writer = new HdfsWriter(config); - break; - case PluginNameConstants.MONGODB_WRITER: - writer = new MongodbWriter(config); - break; - case PluginNameConstants.ODPS_WRITER: - writer = new OdpsWriter(config); - break; - case PluginNameConstants.REDIS_WRITER: - writer = new RedisWriter(config); - break; - case PluginNameConstants.HIVE_WRITER: - writer = new HiveWriter(config); - break; - case PluginNameConstants.KAFKA09_WRITER: - writer = new Kafka09Writer(config); - break; - case PluginNameConstants.KAFKA10_WRITER: - writer = new Kafka10Writer(config); - break; - case PluginNameConstants.KAFKA11_WRITER: - writer = new Kafka11Writer(config); - break; - case PluginNameConstants.KUDU_WRITER: - writer = new KuduWriter(config); - break; - case PluginNameConstants.CLICKHOUSE_WRITER: - writer = new ClickhouseWriter(config); - break; - case PluginNameConstants.POLARDB_WRITER: - writer = new PolardbWriter(config); - break; - case PluginNameConstants.KAFKA_WRITER: - writer = new KafkaWriter(config); - break; - case PluginNameConstants.PHOENIX_WRITER: - writer = new PhoenixWriter(config); - break; - case PluginNameConstants.EMQX_WRITER: - writer = new EmqxWriter(config); - break; - case PluginNameConstants.DM_WRITER: - writer = new DmWriter(config); - break; - default: - throw new IllegalArgumentException("Can not find writer by name:" + writerName); + switch (writerName){ + case PluginNameConstants.STREAM_WRITER : writer = new StreamWriter(config); break; + case PluginNameConstants.CARBONDATA_WRITER : writer = new CarbondataWriter(config); break; + case PluginNameConstants.MYSQL_WRITER : writer = new MysqlWriter(config); break; + case PluginNameConstants.SQLSERVER_WRITER : writer = new SqlserverWriter(config); break; + case PluginNameConstants.ORACLE_WRITER : writer = new OracleWriter(config); break; + case PluginNameConstants.POSTGRESQL_WRITER : writer = new PostgresqlWriter(config); break; + case PluginNameConstants.DB2_WRITER : writer = new Db2Writer(config); break; + case PluginNameConstants.GBASE_WRITER : writer = new GbaseWriter(config); break; + case PluginNameConstants.ES_WRITER : writer = new EsWriter(config); break; + case PluginNameConstants.FTP_WRITER : writer = new FtpWriter(config); break; + case PluginNameConstants.HBASE_WRITER : writer = new HbaseWriter(config); break; + case PluginNameConstants.HDFS_WRITER : writer = new HdfsWriter(config); break; + case PluginNameConstants.MONGODB_WRITER : writer = new MongodbWriter(config); break; + case PluginNameConstants.ODPS_WRITER : writer = new OdpsWriter(config); break; + case PluginNameConstants.REDIS_WRITER : writer = new RedisWriter(config); break; + case PluginNameConstants.HIVE_WRITER : writer = new HiveWriter(config); break; + case PluginNameConstants.KAFKA09_WRITER : writer = new Kafka09Writer(config); break; + case PluginNameConstants.KAFKA10_WRITER : writer = new Kafka10Writer(config); break; + case PluginNameConstants.KAFKA11_WRITER : writer = new Kafka11Writer(config); break; + case PluginNameConstants.KUDU_WRITER : writer = new KuduWriter(config); break; + case PluginNameConstants.CLICKHOUSE_WRITER : writer = new ClickhouseWriter(config); break; + case PluginNameConstants.POLARDB_WRITER : writer = new PolardbWriter(config); break; + case PluginNameConstants.KAFKA_WRITER : writer = new KafkaWriter(config); break; + case PluginNameConstants.PHOENIX_WRITER : writer = new PhoenixWriter(config); break; + case PluginNameConstants.EMQX_WRITER : writer = new EmqxWriter(config); break; + case PluginNameConstants.RESTAPI_WRITER : writer = new RestapiWriter(config);break; + case PluginNameConstants.DM_WRITER : writer = new DmWriter(config); break; + case PluginNameConstants.GREENPLUM_WRITER : writer = new GreenplumWriter(config); break; + default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } return writer; } - private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties) { - if (properties == null) { + private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties){ + if(properties == null){ return; } - if (properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null) { + if(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ return; - } else { + }else{ long interval = Long.parseLong(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY).trim()); //start checkpoint every ${interval} @@ -375,7 +290,7 @@ private static void openCheckpointConf(StreamExecutionEnvironment env, Propertie } String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); - if (checkpointTimeoutStr != null) { + if(checkpointTimeoutStr != null){ long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); //checkpoints have to complete within one min,or are discard env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); @@ -387,6 +302,8 @@ private static void openCheckpointConf(StreamExecutionEnvironment env, Propertie env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); env.getCheckpointConfig().enableExternalizedCheckpoints( CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + + env.setStateBackend(new FsStateBackend(new Path("file:///tmp/flinkx_checkpoint"))); env.setRestartStrategy(RestartStrategies.failureRateRestart( FAILURE_RATE, Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index f0f047e6b5..21d07e7c9e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -53,6 +53,10 @@ public class PluginNameConstants { public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; public static final String METADATAHIVE2_READER = "metadatahive2reader"; public static final String DM_READER = "dmreader"; + public static final String METADATATIDB_READER = "metadatatidbreader"; + public static final String METADATAORACLE_READER = "metadataoraclereader"; + public static final String METADATAMYSQL_READER = "metadatamysqlreader"; + public static final String GREENPLUM_READER = "greenplumreader"; public static final String STREAM_WRITER = "streamwriter"; public static final String CARBONDATA_WRITER = "carbondatawriter"; @@ -81,4 +85,5 @@ public class PluginNameConstants { public static final String EMQX_WRITER = "emqxwriter"; public static final String RESTAPI_WRITER = "restapiwriter"; public static final String DM_WRITER = "dmwriter"; + public static final String GREENPLUM_WRITER = "greenplumwriter"; } diff --git a/pom.xml b/pom.xml index 8524112890..584610d49e 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ flinkx-clickhouse flinkx-saphana flinkx-teradata - flinkx-cassandra + flinkx-greenplum flinkx-hdfs flinkx-hive @@ -40,22 +40,21 @@ flinkx-phoenix flinkx-carbondata flinkx-kudu + flinkx-cassandra flinkx-redis flinkx-mongodb flinkx-binlog + flinkx-pgwal flinkx-kb flinkx-kafka09 flinkx-kafka10 flinkx-kafka11 flinkx-kafka - flinkx-emqx - flinkx-pgwal - flinkx-pulsar - + flinkx-restapi @@ -66,7 +65,7 @@ 2.7.3 4.5.3 ${basedir}/dev - release_1.8.6 + release_1.8.7 @@ -125,9 +124,16 @@ - org.testng - testng - 6.14.3 + junit + junit + 4.12 + test + + + + org.mockito + mockito-core + 3.0.0 test @@ -143,9 +149,24 @@ + + org.jacoco + jacoco-maven-plugin + 0.7.8 + + + + prepare-agent + report + + + + + org.apache.maven.plugins maven-source-plugin + 3.2.1 attach-sources @@ -164,8 +185,46 @@ flinkx-java-docs + + pl.project13.maven + git-commit-id-plugin + 2.2.6 + + + + revision + + + + + yyyy.MM.dd HH:mm:ss + true + true + + false + -dirty + false + + + + + + + org.jacoco + jacoco-maven-plugin + + + + report + + + + + + + \ No newline at end of file From 055e3e6de6d53ac75b844cec891832ce4a687099 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 13 Jul 2020 16:26:31 +0800 Subject: [PATCH 056/136] =?UTF-8?q?-p=E5=8F=82=E6=95=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/config/DataTransferConfig.java | 2 +- .../java/com/dtstack/flinkx/options/Options.java | 14 +++++++++++--- .../java/com/dtstack/flinkx/launcher/Launcher.java | 1 + 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java index 5e58d85e5a..f3e732da0b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java @@ -90,7 +90,7 @@ private static void checkConfig(DataTransferConfig config) { Preconditions.checkNotNull(readerParameter, "Must specify parameter for reader"); - // 检查我writer配置 + // 检查writer配置 WriterConfig writerConfig = content.getWriter(); Preconditions.checkNotNull(writerConfig, "Must specify a writer element"); Preconditions.checkNotNull(writerConfig.getName(), "Must specify the writer name"); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 0123d3f955..26a1189ecd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -64,9 +64,9 @@ public class Options { @OptionRequired(description = "env properties") private String confProp = "{}"; - /** - * savepoint - */ + @OptionRequired(description = "json modify") + private String p = ""; + @OptionRequired(description = "savepoint path") private String s; @@ -195,4 +195,12 @@ public String getPluginLoadMode() { public void setPluginLoadMode(String pluginLoadMode) { this.pluginLoadMode = pluginLoadMode; } + + public String getP() { + return p; + } + + public void setP(String p) { + this.p = p; + } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index a9382f251c..c5c19fc9f4 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -96,6 +96,7 @@ public static void main(String[] args) throws Exception { String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + // 将argList转化为HashMap,方便通过参数名称来获取参数值 HashMap temp = new HashMap<>(16); for (int i = 0; i < argList.size(); i += 2) { From d10875d1a42cc642c772dc78d6343038623d85d2 Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 30 May 2020 11:00:45 +0800 Subject: [PATCH 057/136] =?UTF-8?q?=E5=9C=A8DatabaseInterface=E4=B8=AD?= =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E2=80=9C=E6=A0=B9=E6=8D=AE=E7=BB=99=E5=AE=9A?= =?UTF-8?q?=E7=9A=84Sql=E8=BF=94=E5=9B=9E=E6=9F=A5=E8=AF=A2=E8=A1=A8?= =?UTF-8?q?=E7=BB=93=E6=9E=84=E7=9A=84sql=E2=80=9D=E7=9A=84=E6=8E=A5?= =?UTF-8?q?=E5=8F=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/clickhouse/core/ClickhouseDatabaseMeta.java | 5 +++++ .../main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java | 5 +++++ .../main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java | 5 +++++ .../java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java | 5 +++++ .../java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java | 5 +++++ .../com/dtstack/flinkx/oracle/OracleDatabaseMeta.java | 5 +++++ .../main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java | 5 +++++ .../dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java | 5 +++++ .../java/com/dtstack/flinkx/rdb/DatabaseInterface.java | 8 ++++++++ .../com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java | 5 +++++ .../dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java | 5 +++++ .../com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java | 5 +++++ 12 files changed, 63 insertions(+) diff --git a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java index 7d3c0d81da..898802d7dd 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java +++ b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java @@ -50,6 +50,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 1"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 1"; + } + @Override public String getStartQuote() { return "`"; diff --git a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java index 0334bedfa3..e4b8c2244a 100644 --- a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java +++ b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java @@ -60,6 +60,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " FETCH FIRST 1 ROWS ONLY"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " FETCH FIRST 1 ROWS ONLY"; + } + @Override public String getSplitFilter(String columnName) { return String.format("mod(%s,${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); diff --git a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java index 3c4d2883dc..23d0c95b7b 100644 --- a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java +++ b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java @@ -75,6 +75,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 1"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 1"; + } + @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java b/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java index cad2bc11f6..1d620287de 100644 --- a/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java +++ b/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java @@ -97,6 +97,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; + } + @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java index 3ef4726e79..f5fff8e282 100644 --- a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java +++ b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java @@ -53,6 +53,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; + } + @Override public String getStartQuote() { return "`"; diff --git a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java index 5033bf64af..8eb66da40a 100644 --- a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java +++ b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java @@ -64,6 +64,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT /*+FIRST_ROWS*/ " + quoteColumns(column) + " FROM " + quoteTable(table) + " WHERE ROWNUM < 1"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT /*+FIRST_ROWS*/ * FROM " + "("+ sql +") tmp" + " WHERE ROWNUM < 1"; + } + @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java b/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java index 2b16850a6e..37521672c5 100644 --- a/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java +++ b/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java @@ -53,6 +53,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; + } + @Override public String getStartQuote() { // 对于字段名和表名的quote得用双引号,对于字段值为字符串的得用单引号表示常量 diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index 985b99efaa..71a312e9cf 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -77,6 +77,11 @@ public String getSqlQueryColumnFields(List column, String table) { return String.format(sql,table); } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; + } + @Override public String getUpsertStatement(List column, String table, Map> updateKey) { throw new UnsupportedOperationException("PostgreSQL not support update mode"); diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java index 6617635a10..97a2a608ee 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java @@ -62,6 +62,14 @@ public interface DatabaseInterface { */ String getSqlQueryColumnFields(List column, String table); + /** + * 根据给定的Sql返回查询表结构的sql + * + * @param sql sql + * @return 查询表结构的sql + */ + String getSqlQuerySqlFields(String sql); + /** * 获取左引号 * diff --git a/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java index 8fd3137074..8bf9d8e563 100644 --- a/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java +++ b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java @@ -55,6 +55,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM " + "(" + sql + ") tmp" + " LIMIT 0"; + } + @Override public String getStartQuote() { return "\""; diff --git a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java index c942c4740f..e20ac5e44d 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java +++ b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java @@ -56,6 +56,11 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT TOP 1 " + quoteColumns(column) + " FROM " + quoteTable(table); } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT TOP 1 * FROM " + "(" + sql +") tmp"; + } + @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java index 0f6535497b..992f909935 100644 --- a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java @@ -56,6 +56,11 @@ public String getSqlQueryColumnFields(List column, String table) { " PRECEDING) BETWEEN 0 AND 0"; } + @Override + public String getSqlQuerySqlFields(String sql) { + return "SELECT * FROM " + "(" + sql + ") tmp" + " QUALIFY SUM(1) OVER (ROWS UNBOUNDED PRECEDING) BETWEEN 0 AND 0"; + } + @Override public String getStartQuote() { return "\""; From 3eed2a58b64cbed708b7a8f1ef25fcb47b54113b Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 30 May 2020 11:04:31 +0800 Subject: [PATCH 058/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DJdbcInputFormat?= =?UTF-8?q?=E5=8F=8A=E5=85=B6=E5=AD=90=E7=B1=BB=E7=9A=84descColumnTypeList?= =?UTF-8?q?=E4=B8=BA=E6=9C=80=E7=BB=88=E7=9A=84querySql=E6=9F=A5=E8=AF=A2?= =?UTF-8?q?=E8=BF=94=E5=9B=9E=E7=9A=84schema=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../format/ClickhouseInputFormat.java | 9 +------ .../flinkx/gbase/format/GbaseInputFormat.java | 9 +------ .../flinkx/mysql/format/MysqlInputFormat.java | 9 +------ .../phoenix/format/PhoenixInputFormat.java | 9 +------ .../format/PostgresqlInputFormat.java | 9 +------ .../com/dtstack/flinkx/rdb/util/DbUtil.java | 27 +++++-------------- .../JdbcInputFormat.java | 9 +------ .../saphana/format/SaphanaInputFormat.java | 9 +------ 8 files changed, 13 insertions(+), 77 deletions(-) diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java index 40304f54aa..67baaca862 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java @@ -77,14 +77,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, this.querySql); } catch (Exception e) { LOG.error("open failed,e = {}", ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index 06b22d6f4c..3d10d72d8c 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -71,14 +71,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount-1; } - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index 7915c9bd64..532f6f4894 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -76,14 +76,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount-1; } - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 4d5c46149d..d7812491ae 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -83,14 +83,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java index 9f3f96c342..329662530c 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java @@ -72,14 +72,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount-1; } - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 581d2030d9..753e6fc18d 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -283,40 +283,25 @@ public static Object[][] getParameterValues(final int channels) { * @param password 数据库密码 * @param databaseInterface DatabaseInterface * @param table 表名 - * @param metaColumns MetaColumn列表 + * @param sql 查询sql * @return 字段类型list列表 */ - public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, String table, List metaColumns) { - List descColumnTypeList = new ArrayList<>(metaColumns.size()); + public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, String table, String sql) { + List descColumnTypeList = new ArrayList<>(); Connection dbConn = null; Statement stmt = null; ResultSet rs = null; try { dbConn = getConnection(dbUrl, username, password); stmt = dbConn.createStatement(); - rs = stmt.executeQuery(databaseInterface.getSqlQueryFields(databaseInterface.quoteTable(table))); + rs = stmt.executeQuery(databaseInterface.getSqlQuerySqlFields(sql)); ResultSetMetaData rd = rs.getMetaData(); - boolean flag = ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName()); - Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); for (int i = 1; i <= rd.getColumnCount(); i++) { - if(flag){ - descColumnTypeList.add(rd.getColumnTypeName(i)); - }else{ - nameTypeMap.put(rd.getColumnName(i), rd.getColumnTypeName(i)); - } - } - if(!flag){ - for (MetaColumn metaColumn : metaColumns) { - if (metaColumn.getValue() != null) { - descColumnTypeList.add("string"); - } else { - descColumnTypeList.add(nameTypeMap.get(metaColumn.getName())); - } - } + descColumnTypeList.add(rd.getColumnTypeName(i)); } } catch (SQLException e) { - LOG.error("error to analyzeTable, dbUrl = {}, table = {}, metaColumns = {}, e = {}", dbUrl, table, new Gson().toJson(metaColumns), ExceptionUtil.getErrorMessage(e)); + LOG.error("error to analyzeTable, dbUrl = {}, table = {}, metaColumns = {}, e = {}", dbUrl, table, ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); } finally { closeDbResources(rs, stmt, dbConn, false); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index 7e0bd337b5..b7626f0748 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -188,14 +188,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount - 1; } - if (StringUtils.isEmpty(customSql)) { - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java index 0bac77a5d7..d4d9516787 100644 --- a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java +++ b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java @@ -81,14 +81,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); From ea8006ffc775f8eb4e0bb859a11a1b9cecc61737 Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 30 May 2020 11:06:03 +0800 Subject: [PATCH 059/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DTeradataInputFormat?= =?UTF-8?q?=E7=B1=BB=E7=9A=84descColumnTypeList=E4=B8=BA=E6=9C=80=E7=BB=88?= =?UTF-8?q?=E7=9A=84querySql=E6=9F=A5=E8=AF=A2=E8=BF=94=E5=9B=9E=E7=9A=84s?= =?UTF-8?q?chema=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flinkx/teradata/util/DBUtil.java | 25 ++++++------------- .../teradata/format/TeradataInputFormat.java | 15 +++-------- .../teradata/format/TeradataOutputFormat.java | 5 ++-- 3 files changed, 14 insertions(+), 31 deletions(-) diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java index 08e90eed83..3b42c064c3 100644 --- a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java @@ -16,7 +16,7 @@ */ public class DBUtil { /** - * 获取数据库连接,不使用DBUtil里的getConnection为了避免Telnet,因为jdbc4与jdbc3不同 + * 获取数据库连接,不使用DbUtil里的getConnection为了避免Telnet,因为jdbc4与jdbc3不同 * @param url 连接url * @param username 用户名 * @param password 密码 @@ -45,32 +45,23 @@ public static Connection getConnection(String url, String username, String passw * @param password 数据库密码 * @param databaseInterface DatabaseInterface * @param table 表名 - * @param metaColumns MetaColumn列表 + * @param sql sql * @return */ public static List analyzeTable(String dbURL, String username, String password, DatabaseInterface databaseInterface, - String table, List metaColumns) { - List ret = new ArrayList<>(metaColumns.size()); + String table, String sql) { + List descColumnTypeList = new ArrayList<>(); Connection dbConn = null; Statement stmt = null; ResultSet rs = null; try { dbConn = getConnection(dbURL, username, password); stmt = dbConn.createStatement(); - rs = stmt.executeQuery(databaseInterface.getSqlQueryFields(databaseInterface.quoteTable(table))); + rs = stmt.executeQuery(databaseInterface.getSqlQuerySqlFields(sql)); ResultSetMetaData rd = rs.getMetaData(); - Map nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); - for(int i = 0; i < rd.getColumnCount(); ++i) { - nameTypeMap.put(rd.getColumnName(i+1),rd.getColumnTypeName(i+1)); - } - - for (MetaColumn metaColumn : metaColumns) { - if(metaColumn.getValue() != null){ - ret.add("string"); - } else { - ret.add(nameTypeMap.get(metaColumn.getName())); - } + for (int i = 1; i <= rd.getColumnCount(); i++) { + descColumnTypeList.add(rd.getColumnTypeName(i)); } } catch (SQLException e) { throw new RuntimeException(e); @@ -78,6 +69,6 @@ public static List analyzeTable(String dbURL, String username, String pa DbUtil.closeDbResources(rs, stmt, dbConn, false); } - return ret; + return descColumnTypeList; } } diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java index cb0f896f43..116ad259b7 100644 --- a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx.teradata.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; -import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.teradata.util.DBUtil; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.collections.CollectionUtils; @@ -29,8 +29,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.sql.*; -import java.util.ArrayList; +import java.sql.SQLException; +import java.sql.Statement; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -83,14 +83,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - if (StringUtils.isEmpty(customSql)){ - descColumnTypeList = DBUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,metaColumns); - } else { - descColumnTypeList = new ArrayList<>(); - for (MetaColumn metaColumn : metaColumns) { - descColumnTypeList.add(metaColumn.getName()); - } - } + descColumnTypeList = DBUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java index dd3757fa06..e84473a9cf 100644 --- a/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java +++ b/flinkx-teradata/flinkx-teradata-writer/src/main/java/com/dtstack/flinkx/teradata/format/TeradataOutputFormat.java @@ -19,6 +19,7 @@ import com.dtstack.flinkx.enums.EWriteMode; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; +import com.dtstack.flinkx.teradata.util.DBUtil; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.collections.CollectionUtils; import org.slf4j.Logger; @@ -26,8 +27,6 @@ import java.sql.SQLException; -import static com.dtstack.flinkx.teradata.util.DBUtil.getConnection; - /** * Company: www.dtstack.com * @@ -39,7 +38,7 @@ public class TeradataOutputFormat extends JdbcOutputFormat { protected void openInternal(int taskNumber, int numTasks) { try { ClassUtil.forName(driverName, getClass().getClassLoader()); - dbConn = getConnection(dbUrl, username, password); + dbConn = DBUtil.getConnection(dbUrl, username, password); if (restoreConfig.isRestore()){ dbConn.setAutoCommit(false); From f21091394454566a986c7dabc485953a9e896dfc Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 30 May 2020 11:07:20 +0800 Subject: [PATCH 060/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DDistributedJdbcInputF?= =?UTF-8?q?ormat=E5=8F=8A=E5=85=B6=E5=AD=90=E7=B1=BB=E7=9A=84descColumnTyp?= =?UTF-8?q?eList=E4=B8=BA=E6=9C=80=E7=BB=88=E7=9A=84querySql=E6=9F=A5?= =?UTF-8?q?=E8=AF=A2=E8=BF=94=E5=9B=9E=E7=9A=84schema=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java | 2 +- .../DistributedJdbcInputFormat.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java index a70bba4ddc..ba5cadf44a 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java @@ -68,7 +68,7 @@ protected void openNextSource() throws SQLException { if(descColumnTypeList == null) { descColumnTypeList = DbUtil.analyzeTable(currentSource.getJdbcUrl(), currentSource.getUserName(), - currentSource.getPassword(),databaseInterface, currentSource.getTable(),metaColumns); + currentSource.getPassword(),databaseInterface, currentSource.getTable(),queryTemplate); } LOG.info("open source: {} ,table: {}", currentSource.getJdbcUrl(), currentSource.getTable()); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java index 85fd1e8db0..4829f0d761 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java @@ -134,7 +134,7 @@ protected void openNextSource() throws SQLException{ if(descColumnTypeList == null) { descColumnTypeList = DbUtil.analyzeTable(currentSource.getJdbcUrl(), currentSource.getUserName(), - currentSource.getPassword(),databaseInterface, currentSource.getTable(),metaColumns); + currentSource.getPassword(),databaseInterface, currentSource.getTable(),queryTemplate); } LOG.info("open source: {} ,table: {}", currentSource.getJdbcUrl(), currentSource.getTable()); From 82bffb37d540c2a5e26713d71bb4230fd8851082 Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 6 Jun 2020 09:40:47 +0800 Subject: [PATCH 061/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E6=9F=A5=E8=AF=A2=E6=95=B0=E6=8D=AE=E6=BA=90=E7=9A=84=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../format/ClickhouseInputFormat.java | 4 +-- .../flinkx/gbase/format/GbaseInputFormat.java | 4 +-- .../mysqld/format/MysqldInputFormat.java | 3 +- .../flinkx/mysql/format/MysqlInputFormat.java | 5 +-- .../phoenix/format/PhoenixInputFormat.java | 4 +-- .../format/PostgresqlInputFormat.java | 5 +-- .../com/dtstack/flinkx/rdb/util/DbUtil.java | 31 +++++-------------- .../DistributedJdbcInputFormat.java | 3 +- .../JdbcInputFormat.java | 3 +- .../saphana/format/SaphanaInputFormat.java | 4 +-- .../teradata/format/TeradataInputFormat.java | 2 +- 11 files changed, 18 insertions(+), 50 deletions(-) diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java index 67baaca862..1eb3c27123 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.clickhouse.core.ClickhouseUtil; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; @@ -30,7 +29,6 @@ import java.io.IOException; import java.sql.SQLException; import java.sql.Statement; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -77,7 +75,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (Exception e) { LOG.error("open failed,e = {}", ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index 3d10d72d8c..357646075a 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -19,7 +19,6 @@ import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -27,7 +26,6 @@ import java.io.IOException; import java.sql.SQLException; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -71,7 +69,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount-1; } - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java index ba5cadf44a..c212e0a09c 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/format/MysqldInputFormat.java @@ -67,8 +67,7 @@ protected void openNextSource() throws SQLException { columnCount = currentResultSet.getMetaData().getColumnCount(); if(descColumnTypeList == null) { - descColumnTypeList = DbUtil.analyzeTable(currentSource.getJdbcUrl(), currentSource.getUserName(), - currentSource.getPassword(),databaseInterface, currentSource.getTable(),queryTemplate); + descColumnTypeList = DbUtil.analyzeColumnType(currentResultSet); } LOG.info("open source: {} ,table: {}", currentSource.getJdbcUrl(), currentSource.getTable()); diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index 532f6f4894..d66c2766ad 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -19,7 +19,6 @@ import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; import com.dtstack.flinkx.util.ExceptionUtil; @@ -31,7 +30,6 @@ import java.io.IOException; import java.sql.SQLException; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -70,13 +68,12 @@ public void openInternal(InputSplit inputSplit) throws IOException { fetchSize = Integer.MIN_VALUE; executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); - boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); if(splitWithRowCol){ columnCount = columnCount-1; } - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index d7812491ae..33f48ff496 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.phoenix.util.PhoenixUtil; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; import org.apache.commons.collections.CollectionUtils; @@ -31,7 +30,6 @@ import java.io.IOException; import java.sql.SQLException; import java.sql.Statement; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -83,7 +81,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java index 329662530c..6ad0468b6e 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -29,7 +28,6 @@ import java.io.IOException; import java.sql.SQLException; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -66,13 +64,12 @@ public void openInternal(InputSplit inputSplit) throws IOException { querySql = buildQuerySql(inputSplit); executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); - boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); if(splitWithRowCol){ columnCount = columnCount-1; } - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 753e6fc18d..26269c7c34 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -276,38 +276,23 @@ public static Object[][] getParameterValues(final int channels) { } /** - * 获取表列名类型列表 + * 获取结果集的列类型信息 * - * @param dbUrl jdbc url - * @param username 数据库账号 - * @param password 数据库密码 - * @param databaseInterface DatabaseInterface - * @param table 表名 - * @param sql 查询sql + * @param resultSet 查询结果集 * @return 字段类型list列表 */ - public static List analyzeTable(String dbUrl, String username, String password, DatabaseInterface databaseInterface, String table, String sql) { - List descColumnTypeList = new ArrayList<>(); - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; + public static List analyzeColumnType(ResultSet resultSet){ + List columnTypeList = new ArrayList(); try { - dbConn = getConnection(dbUrl, username, password); - stmt = dbConn.createStatement(); - rs = stmt.executeQuery(databaseInterface.getSqlQuerySqlFields(sql)); - ResultSetMetaData rd = rs.getMetaData(); - + ResultSetMetaData rd = resultSet.getMetaData(); for (int i = 1; i <= rd.getColumnCount(); i++) { - descColumnTypeList.add(rd.getColumnTypeName(i)); + columnTypeList.add(rd.getColumnTypeName(i)); } } catch (SQLException e) { - LOG.error("error to analyzeTable, dbUrl = {}, table = {}, metaColumns = {}, e = {}", dbUrl, table, ExceptionUtil.getErrorMessage(e)); + LOG.error("error to analyzeSchema, resultSet = {}, e = {}", resultSet, ExceptionUtil.getErrorMessage(e)); throw new RuntimeException(e); - } finally { - closeDbResources(rs, stmt, dbConn, false); } - - return descColumnTypeList; + return columnTypeList; } /** diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java index 4829f0d761..2d33a29161 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java @@ -133,8 +133,7 @@ protected void openNextSource() throws SQLException{ columnCount = currentResultSet.getMetaData().getColumnCount(); if(descColumnTypeList == null) { - descColumnTypeList = DbUtil.analyzeTable(currentSource.getJdbcUrl(), currentSource.getUserName(), - currentSource.getPassword(),databaseInterface, currentSource.getTable(),queryTemplate); + descColumnTypeList =DbUtil.analyzeColumnType(currentResultSet); } LOG.info("open source: {} ,table: {}", currentSource.getJdbcUrl(), currentSource.getTable()); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index b7626f0748..bcf4e1f684 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -35,7 +35,6 @@ import com.dtstack.flinkx.util.UrlUtil; import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.LongMaximum; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -188,7 +187,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { columnCount = columnCount - 1; } - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password, databaseInterface, table, this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); diff --git a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java index d4d9516787..14c5c2fc35 100644 --- a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java +++ b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java @@ -19,7 +19,6 @@ import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -29,7 +28,6 @@ import java.io.IOException; import java.sql.SQLException; import java.sql.Statement; -import java.util.ArrayList; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -81,7 +79,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - descColumnTypeList = DbUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java index 116ad259b7..625c4249e0 100644 --- a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java @@ -83,7 +83,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { hasNext = resultSet.next(); - descColumnTypeList = DBUtil.analyzeTable(dbUrl, username, password,databaseInterface,table,this.querySql); + descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); From 8a0f6da6532cb7a7fdf9e9f601bccdaf1932fe4b Mon Sep 17 00:00:00 2001 From: daemin Date: Sat, 6 Jun 2020 09:48:45 +0800 Subject: [PATCH 062/136] =?UTF-8?q?Revert=20"=E5=9C=A8DatabaseInterface?= =?UTF-8?q?=E4=B8=AD=E6=B7=BB=E5=8A=A0=E2=80=9C=E6=A0=B9=E6=8D=AE=E7=BB=99?= =?UTF-8?q?=E5=AE=9A=E7=9A=84Sql=E8=BF=94=E5=9B=9E=E6=9F=A5=E8=AF=A2?= =?UTF-8?q?=E8=A1=A8=E7=BB=93=E6=9E=84=E7=9A=84sql=E2=80=9D=E7=9A=84?= =?UTF-8?q?=E6=8E=A5=E5=8F=A3"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit c87d164b25cd310f4b31b8dea55508e384407011. --- .../flinkx/clickhouse/core/ClickhouseDatabaseMeta.java | 5 ----- .../main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java | 5 ----- .../main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java | 5 ----- .../java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java | 5 ----- .../java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java | 5 ----- .../com/dtstack/flinkx/oracle/OracleDatabaseMeta.java | 5 ----- .../main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java | 5 ----- .../dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java | 5 ----- .../java/com/dtstack/flinkx/rdb/DatabaseInterface.java | 8 -------- .../com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java | 5 ----- .../dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java | 5 ----- .../com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java | 5 ----- 12 files changed, 63 deletions(-) diff --git a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java index 898802d7dd..7d3c0d81da 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java +++ b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseDatabaseMeta.java @@ -50,11 +50,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 1"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 1"; - } - @Override public String getStartQuote() { return "`"; diff --git a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java index e4b8c2244a..0334bedfa3 100644 --- a/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java +++ b/flinkx-db2/flinkx-db2-core/src/main/java/com/dtstack/flinkx/db2/Db2DatabaseMeta.java @@ -60,11 +60,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " FETCH FIRST 1 ROWS ONLY"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " FETCH FIRST 1 ROWS ONLY"; - } - @Override public String getSplitFilter(String columnName) { return String.format("mod(%s,${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); diff --git a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java index 23d0c95b7b..3c4d2883dc 100644 --- a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java +++ b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java @@ -75,11 +75,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 1"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 1"; - } - @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java b/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java index 1d620287de..cad2bc11f6 100644 --- a/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java +++ b/flinkx-gbase/flinkx-gbase-core/src/main/java/com/dtstack/flinkx/gbase/GbaseDatabaseMeta.java @@ -97,11 +97,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; - } - @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java index f5fff8e282..3ef4726e79 100644 --- a/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java +++ b/flinkx-mysql/flinkx-mysql-core/src/main/java/com/dtstack/flinkx/mysql/MySqlDatabaseMeta.java @@ -53,11 +53,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; - } - @Override public String getStartQuote() { return "`"; diff --git a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java index 8eb66da40a..5033bf64af 100644 --- a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java +++ b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java @@ -64,11 +64,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT /*+FIRST_ROWS*/ " + quoteColumns(column) + " FROM " + quoteTable(table) + " WHERE ROWNUM < 1"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT /*+FIRST_ROWS*/ * FROM " + "("+ sql +") tmp" + " WHERE ROWNUM < 1"; - } - @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java b/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java index 37521672c5..2b16850a6e 100644 --- a/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java +++ b/flinkx-phoenix/flinkx-phoenix-core/src/main/java/com/dtstack/flinkx/phoenix/PhoenixMeta.java @@ -53,11 +53,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; - } - @Override public String getStartQuote() { // 对于字段名和表名的quote得用双引号,对于字段值为字符串的得用单引号表示常量 diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index 71a312e9cf..985b99efaa 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -77,11 +77,6 @@ public String getSqlQueryColumnFields(List column, String table) { return String.format(sql,table); } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM (" + sql + ") tmp" + " LIMIT 0"; - } - @Override public String getUpsertStatement(List column, String table, Map> updateKey) { throw new UnsupportedOperationException("PostgreSQL not support update mode"); diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java index 97a2a608ee..6617635a10 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/DatabaseInterface.java @@ -62,14 +62,6 @@ public interface DatabaseInterface { */ String getSqlQueryColumnFields(List column, String table); - /** - * 根据给定的Sql返回查询表结构的sql - * - * @param sql sql - * @return 查询表结构的sql - */ - String getSqlQuerySqlFields(String sql); - /** * 获取左引号 * diff --git a/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java index 8bf9d8e563..8fd3137074 100644 --- a/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java +++ b/flinkx-saphana/flinkx-saphana-core/src/main/java/com/dtstack/flinkx/saphana/SaphanaDatabaseMeta.java @@ -55,11 +55,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM " + "(" + sql + ") tmp" + " LIMIT 0"; - } - @Override public String getStartQuote() { return "\""; diff --git a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java index e20ac5e44d..c942c4740f 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java +++ b/flinkx-sqlserver/flinkx-sqlserver-core/src/main/java/com/dtstack/flinkx/sqlserver/SqlServerDatabaseMeta.java @@ -56,11 +56,6 @@ public String getSqlQueryColumnFields(List column, String table) { return "SELECT TOP 1 " + quoteColumns(column) + " FROM " + quoteTable(table); } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT TOP 1 * FROM " + "(" + sql +") tmp"; - } - @Override public String quoteValue(String value, String column) { return String.format("'%s' as %s",value,column); diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java index 992f909935..0f6535497b 100644 --- a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/TeradataDatabaseMeta.java @@ -56,11 +56,6 @@ public String getSqlQueryColumnFields(List column, String table) { " PRECEDING) BETWEEN 0 AND 0"; } - @Override - public String getSqlQuerySqlFields(String sql) { - return "SELECT * FROM " + "(" + sql + ") tmp" + " QUALIFY SUM(1) OVER (ROWS UNBOUNDED PRECEDING) BETWEEN 0 AND 0"; - } - @Override public String getStartQuote() { return "\""; From bae47099e0538c98f7603c58753aac42a7ed3c46 Mon Sep 17 00:00:00 2001 From: daemin Date: Mon, 8 Jun 2020 21:57:40 +0800 Subject: [PATCH 063/136] =?UTF-8?q?fix:=20=E6=B7=BB=E5=8A=A0reader?= =?UTF-8?q?=E9=85=8D=E7=BD=AE=E4=B8=ADcolumn=E6=95=B0=E7=BB=84=E9=95=BF?= =?UTF-8?q?=E5=BA=A6=E5=92=8C=E6=9F=A5=E8=AF=A2=E7=BB=93=E6=9E=9C=E5=88=97?= =?UTF-8?q?=E9=95=BF=E5=BA=A6=E7=9A=84=E6=A0=A1=E9=AA=8C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../clickhouse/format/ClickhouseInputFormat.java | 2 +- .../flinkx/gbase/format/GbaseInputFormat.java | 2 +- .../flinkx/mysql/format/MysqlInputFormat.java | 2 +- .../flinkx/phoenix/format/PhoenixInputFormat.java | 2 +- .../postgresql/format/PostgresqlInputFormat.java | 2 +- .../JdbcInputFormat.java | 15 ++++++++++++++- .../flinkx/saphana/format/SaphanaInputFormat.java | 2 +- .../teradata/format/TeradataInputFormat.java | 2 +- 8 files changed, 21 insertions(+), 8 deletions(-) diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java index 1eb3c27123..2ae594c138 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java @@ -72,7 +72,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); hasNext = resultSet.next(); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index 357646075a..7361843030 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -68,7 +68,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index d66c2766ad..9836df273a 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -72,7 +72,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 33f48ff496..6d6b416559 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -78,7 +78,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); hasNext = resultSet.next(); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java index 6ad0468b6e..5ba10faad7 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java @@ -68,7 +68,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index bcf4e1f684..706600d712 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -186,7 +186,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if (splitWithRowCol) { columnCount = columnCount - 1; } - + checkSize(columnCount, metaColumns); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); } catch (SQLException se) { @@ -832,4 +832,17 @@ private void queryStartLocation() throws SQLException{ } } + /** + * 校验columnCount和metaColumns的长度是否相等 + * @param columnCount + * @param metaColumns + */ + protected void checkSize(int columnCount, List metaColumns) { + if (!ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName()) && columnCount != metaColumns.size()) { + LOG.error("error config: column size for reader is {},but columns size for query result is {}." + + " And the query sql is '{}'.", + metaColumns.size(), columnCount, querySql); + throw new RuntimeException(""); + } + } } \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java index 14c5c2fc35..9946803ffe 100644 --- a/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java +++ b/flinkx-saphana/flinkx-saphana-reader/src/main/java/com/dtstack/flinkx/saphana/format/SaphanaInputFormat.java @@ -76,7 +76,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); hasNext = resultSet.next(); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java index 625c4249e0..68e19a54c4 100644 --- a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java @@ -80,7 +80,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if(splitWithRowCol){ columnCount = columnCount-1; } - + checkSize(columnCount, metaColumns); hasNext = resultSet.next(); descColumnTypeList = DbUtil.analyzeColumnType(resultSet); From 42bf87c378bed2e15ab4d1cd053a20c08b323c52 Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 13 Jul 2020 16:41:55 +0800 Subject: [PATCH 064/136] =?UTF-8?q?=E6=9B=B4=E6=96=B0=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 2 +- README_CH.md | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 38e4d79917..c391b42f5c 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ The following databases are currently supported: | | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | -| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | +| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | diff --git a/README_CH.md b/README_CH.md index 28ac7c27b2..8bb8c11dd0 100644 --- a/README_CH.md +++ b/README_CH.md @@ -51,6 +51,7 @@ FlinkX目前支持下面这些数据库: | | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | @@ -65,11 +66,12 @@ FlinkX目前支持下面这些数据库: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon | | -| | SqlServer CDC | Coming Soon | | +| | Oracle Logminer| Coming Soon | | +| | SqlServer CDC | Coming Soon | | # 快速开始 From b85371190a311e885f9da792052d5594dc95d74a Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 13 Jul 2020 18:59:17 +0800 Subject: [PATCH 065/136] add pulsar license --- flinkx-pulsar/flinkx-pulsar-writer/pom.xml | 3 --- .../flinkx/pulsar/writer/Constants.java | 17 +++++++++++++++ .../pulsar/writer/PulsarOutputFormat.java | 19 ++++++++++++++++- .../writer/PulsarOutputFormatBuilder.java | 17 +++++++++++++++ .../flinkx/pulsar/writer/PulsarWriter.java | 21 ++++++++++++++++++- pom.xml | 1 + 6 files changed, 73 insertions(+), 5 deletions(-) diff --git a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml index 72c8493094..88396482f4 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml +++ b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml @@ -11,11 +11,8 @@ flinkx-pulsar-writer - - - org.apache.maven.plugins maven-shade-plugin diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java index b12d1f8ece..28cd09eaad 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; /** diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java index 55aeee7dda..fdfab032da 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.exception.WriteRecordException; @@ -92,7 +109,7 @@ protected void emit(Map event) throws IOException { } @Override - protected void writeMultipleRecordsInternal() throws Exception { + protected void writeMultipleRecordsInternal() { throw new UnsupportedOperationException(); } diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java index 637a97e67a..c6e35cc79c 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java index ae84b857f1..2068128992 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java @@ -1,8 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.writer.BaseDataWriter; -import static com.dtstack.flinkx.pulsar.writer.Constants.*; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; @@ -10,6 +26,8 @@ import java.util.List; import java.util.Map; +import static com.dtstack.flinkx.pulsar.writer.Constants.*; + /** * @author: pierre @@ -22,6 +40,7 @@ public class PulsarWriter extends BaseDataWriter { protected List tableFields; protected Map producerSettings; + @SuppressWarnings("unchecked") public PulsarWriter(DataTransferConfig config){ super(config); topic = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal(KEY_TOPIC); diff --git a/pom.xml b/pom.xml index 584610d49e..e7c14772ab 100644 --- a/pom.xml +++ b/pom.xml @@ -54,6 +54,7 @@ flinkx-kafka11 flinkx-kafka flinkx-emqx + flinkx-pulsar flinkx-restapi From b43961c4fc0e446b3c63a52aefbaa3af71a2548d Mon Sep 17 00:00:00 2001 From: tudou Date: Mon, 13 Jul 2020 20:29:18 +0800 Subject: [PATCH 066/136] update pom --- pom.xml | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/pom.xml b/pom.xml index e7c14772ab..821397ba25 100644 --- a/pom.xml +++ b/pom.xml @@ -186,28 +186,6 @@ flinkx-java-docs - - pl.project13.maven - git-commit-id-plugin - 2.2.6 - - - - revision - - - - - yyyy.MM.dd HH:mm:ss - true - true - - false - -dirty - false - - - From 4723ddaa9220099baf139b95c61764e88bbfb919 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 14 Jul 2020 09:21:09 +0800 Subject: [PATCH 067/136] fix bug,update doc --- docs/quickstart.md | 3 ++- .../java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java | 3 --- .../main/java/com/dtstack/flinkx/util/JsonModifyUtil.java | 5 ++++- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index fe581442a0..71779f930d 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -18,7 +18,7 @@ cd flink-1.8.5 ## 编译插件 ```bash -mvn clean package -DskipTests +mvn clean package -Dmaven.test.skip=true ``` ## 运行任务 @@ -232,6 +232,7 @@ bin/flinkx -mode yarnPer \ | **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | | **confProp** | checkpoint配置 | **flink.checkpoint.interval**:快照生产频率
**flink.checkpoint.stateBackend**:快照存储路径 | 否 | 无 | | **s** | checkpoint快照路径 | | 否 | 无 | +| **p** | 自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102| | 否 | 无 | ## 常见问题 diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java index aea0c06c90..6d5dde5398 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java @@ -1,5 +1,3 @@ -<<<<<<< HEAD -======= /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -17,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ ->>>>>>> 1.8_release_3.10.x package com.dtstack.flinkx.cassandra; import com.datastax.driver.core.*; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java index 218ca60eea..2e912373a7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.util; +import org.apache.commons.lang3.StringUtils; + import java.util.HashMap; /** @@ -39,7 +41,8 @@ public static String JsonValueReplace(String json, HashMap param */ public static HashMap CommandTransform(String command) { HashMap parameter = new HashMap<>(); - for (String item : command.split(",")) { + String[] split = StringUtils.split(command, ","); + for (String item : split) { String[] temp = item.split("="); parameter.put(temp[0], temp[1]); } From 62387267aa3aa01f5e390165fd9c97cd2e95c659 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 14 Jul 2020 16:04:04 +0800 Subject: [PATCH 068/136] =?UTF-8?q?commit=20release=5F1.8.7:=20=20=201?= =?UTF-8?q?=E3=80=81=E5=A2=9E=E5=8A=A0Greenplum=E5=92=8CRestApi=E6=8F=92?= =?UTF-8?q?=E4=BB=B6=20=20=202=E3=80=81=E6=94=AF=E6=8C=81-p=E5=8F=82?= =?UTF-8?q?=E6=95=B0=E5=A1=AB=E5=85=85=E8=84=9A=E6=9C=AC=E5=8D=A0=E4=BD=8D?= =?UTF-8?q?=E7=AC=A6=20=20=203=E3=80=81BUG=E4=BF=AE=E5=A4=8D?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitignore | 4 +- README.md | 6 +- README_CH.md | 6 +- docs/offline/reader/greenplumreader.md | 340 +++++ docs/offline/writer/greenplumwriter.md | 139 ++ docs/offline/writer/pulsarwriter.md | 76 - docs/quickstart.md | 3 +- docs/realTime/reader/restapireader.md | 75 + docs/realTime/writer/restapiwriter.md | 115 ++ flinkx-binlog/flinkx-binlog-core/pom.xml | 6 - flinkx-binlog/flinkx-binlog-reader/pom.xml | 10 +- .../flinkx/binlog/reader/BinlogConfig.java | 2 +- .../flinkx-carbondata-reader/pom.xml | 8 +- .../carbondata/reader/CarbonExpressUtil.java | 2 - .../reader/CarbonFlinkInputSplit.java | 36 +- .../carbondata/reader/CarbondataReader.java | 7 +- .../flinkx-carbondata-writer/pom.xml | 8 +- .../carbondata/writer/CarbonOutputFormat.java | 4 +- .../writer/CarbondataOutputFormatBuilder.java | 2 + .../carbondata/writer/dict/DateTimeUtils.java | 100 +- .../writer/dict/ExternalCatalogUtils.java | 7 - .../recordwriter/AbstractRecordWriter.java | 15 +- .../CarbonPartitionRecordWriter.java | 3 - .../HivePartitionRecordWriter.java | 14 +- .../writer/dict/DateTimeUtilsTest.java | 29 +- .../flinkx-cassandra-core/pom.xml | 4 +- .../flinkx/cassandra/CassandraConfigKeys.java | 17 + .../flinkx/cassandra/CassandraUtil.java | 3 +- .../flinkx/cassandra/TestCassandraUtil.java | 17 + .../flinkx-cassandra-reader/pom.xml | 4 +- .../cassandra/reader/CassandraConstants.java | 17 + .../reader/CassandraInputFormat.java | 18 +- .../flinkx-cassandra-writer/pom.xml | 4 +- .../writer/CassandraOutputFormat.java | 32 +- .../writer/CassandraOutputFormatBuilder.java | 17 + .../cassandra/writer/CassandraWriter.java | 19 +- flinkx-cassandra/pom.xml | 4 +- .../clickhouse/core/ClickhouseUtil.java | 7 +- .../flinkx-clickhouse-reader/pom.xml | 10 + .../flinkx-clickhouse-writer/pom.xml | 10 + flinkx-core/pom.xml | 35 +- .../api/java/MyLocalStreamEnvironment.java | 1 - .../main/java/com/dtstack/flinkx/Main.java | 2 +- .../flinkx/authenticate/KerberosUtil.java | 12 +- .../flinkx/authenticate/SftpHandler.java | 23 +- .../classloader/ClassLoaderManager.java | 8 +- .../flinkx/classloader/PluginUtil.java | 2 - .../dtstack/flinkx/config/AbstractConfig.java | 14 +- .../flinkx/config/DataTransferConfig.java | 2 +- .../dtstack/flinkx/config/RestartConfig.java | 2 +- .../dtstack/flinkx/config/RestoreConfig.java | 5 +- .../com/dtstack/flinkx/config/TestConfig.java | 2 +- .../flinkx/constants/ConstantValue.java | 11 + .../dtstack/flinkx/decoder/DecodeEnum.java | 36 +- .../com/dtstack/flinkx}/decoder/IDecode.java | 2 +- .../dtstack/flinkx}/decoder/JsonDecoder.java | 6 +- .../dtstack/flinkx}/decoder/PlainDecoder.java | 2 +- .../dtstack/flinkx/enums/EDatabaseType.java | 1 + .../inputformat/BaseRichInputFormat.java | 8 +- .../flinkx/metrics/AccumulatorCollector.java | 13 +- .../metrics/CustomPrometheusReporter.java | 22 +- .../dtstack/flinkx/options/OptionParser.java | 14 +- .../com/dtstack/flinkx/options/Options.java | 14 +- .../outputformat/BaseFileOutputFormat.java | 10 +- .../outputformat/BaseRichOutputFormat.java | 10 +- .../BaseRichOutputFormatBuilder.java | 19 +- .../flinkx/reader/ByteRateLimiter.java | 15 +- .../com/dtstack/flinkx/reader/MetaColumn.java | 12 +- .../com/dtstack/flinkx/util/ClassUtil.java | 2 +- .../java/com/dtstack/flinkx/util/Clock.java | 5 + .../com/dtstack/flinkx/util/DateUtil.java | 20 +- .../dtstack/flinkx/util/ExceptionUtil.java | 35 +- .../dtstack/flinkx/util/FileSystemUtil.java | 26 +- .../com/dtstack/flinkx/util/GsonUtil.java | 144 ++ .../dtstack/flinkx/util/JsonModifyUtil.java | 51 + .../dtstack/flinkx/util/ResultPrintUtil.java | 10 +- .../com/dtstack/flinkx/util/RetryUtil.java | 2 +- .../com/dtstack/flinkx/util/StringUtil.java | 16 +- .../flinkx/writer/DirtyDataManager.java | 11 +- .../dtstack/flinkx/writer/ErrorLimiter.java | 3 +- .../StreamExecutionEnvironment.java | 14 +- .../flinkx/util/ColumnTypeUtilTest.java | 4 +- .../com/dtstack/flinkx/util/DateUtilTest.java | 112 ++ .../flinkx/util/ExceptionUtilTest.java | 23 + .../com/dtstack/flinkx/util/MapUtilTest.java | 4 +- .../com/dtstack/flinkx/util/Md5UtilTest.java | 4 +- .../com/dtstack/flinkx/util/RowUtilTest.java | 4 +- .../flinkx/util/SnowflakeIdWorkerTest.java | 28 + .../dtstack/flinkx/util/StringUtilTest.java | 54 +- .../dtstack/flinkx/util/ValueUtilTest.java | 12 +- flinkx-db2/flinkx-db2-reader/pom.xml | 8 +- flinkx-db2/flinkx-db2-writer/pom.xml | 8 +- .../com/dtstack/flinkx/dm/DmDatabaseMeta.java | 2 +- flinkx-dm/flinkx-dm-reader/pom.xml | 8 +- .../flinkx/dm/format/DmInputFormat.java | 4 +- flinkx-dm/flinkx-dm-writer/pom.xml | 8 +- .../flinkx/dm/format/DmOutputFormat.java | 43 +- .../flinkx/emqx/decoder/JsonDecoder.java | 56 - flinkx-emqx/flinkx-emqx-reader/pom.xml | 10 + .../flinkx/emqx/format/EmqxInputFormat.java | 13 +- .../flinkx/emqx/reader/EmqxReader.java | 8 +- flinkx-emqx/flinkx-emqx-writer/pom.xml | 10 + .../flinkx/emqx/format/EmqxOutputFormat.java | 2 +- .../flinkx/emqx/writer/EmqxWriter.java | 7 +- flinkx-es/flinkx-es-reader/pom.xml | 14 + .../flinkx/es/reader/EsInputFormat.java | 6 +- .../es/reader/EsInputFormatBuilder.java | 5 + .../dtstack/flinkx/es/reader/EsReader.java | 7 +- .../flinkx/es/reader/test/EsReadDemo.java | 90 -- .../flinkx/es/reader/test/EsReaderTest.java | 20 - flinkx-es/flinkx-es-writer/pom.xml | 14 + .../dtstack/flinkx/es/writer/test/EsDemo.java | 146 -- .../com/dtstack/flinkx/ftp/FtpConfig.java | 2 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 15 +- .../dtstack/flinkx/ftp/FtpHandlerFactory.java | 18 +- .../com/dtstack/flinkx/ftp/IFtpHandler.java | 4 +- flinkx-ftp/flinkx-ftp-reader/pom.xml | 10 + .../flinkx/ftp/reader/FtpInputFormat.java | 16 +- .../dtstack/flinkx/ftp/reader/FtpReader.java | 5 +- .../ftp/reader/FtpSeqBufferedReader.java | 12 +- .../flinkx/ftp/reader/FtpServerDemo.java | 41 - .../flinkx/ftp/reader/SftpServerDemo.java | 42 - flinkx-ftp/flinkx-ftp-writer/pom.xml | 10 + .../flinkx/ftp/writer/FtpOutputFormat.java | 101 +- .../ftp/writer/FtpOutputFormatBuilder.java | 4 +- .../dtstack/flinkx/ftp/writer/FtpWriter.java | 2 +- flinkx-gbase/flinkx-gbase-reader/pom.xml | 8 +- .../flinkx/gbase/format/GbaseInputFormat.java | 9 +- flinkx-gbase/flinkx-gbase-writer/pom.xml | 8 +- .../flinkx-greenplum-core/pom.xml | 39 + .../greenplum/GreenplumDatabaseMeta.java | 50 + .../GreenplumDatabaseMetaInsert.java | 33 + .../flinkx-greenplum-reader/pom.xml | 112 ++ .../format/GreenplumInputFormat.java | 30 + .../greenplum/reader/GreenplumReader.java | 83 ++ .../flinkx-greenplum-writer/pom.xml | 110 ++ .../format/GreenplumOutputFormat.java | 31 + .../greenplum/writer/GreenplumWriter.java | 93 ++ flinkx-greenplum/pom.xml | 36 + .../flinkx/hbase/HbaseConfigConstants.java | 2 +- .../com/dtstack/flinkx/hbase/HbaseHelper.java | 70 +- .../flinkx/hbase/test/HbaseHelperTest.java | 75 - flinkx-hbase/flinkx-hbase-reader/pom.xml | 10 + .../flinkx/hbase/reader/HbaseInputFormat.java | 27 +- .../flinkx/hbase/reader/HbaseReader.java | 7 +- .../flinkx/hbase/reader/HbaseSplitDemo.java | 34 - flinkx-hbase/flinkx-hbase-writer/pom.xml | 10 + .../hbase/writer/HbaseOutputFormat.java | 104 +- .../writer/HbaseOutputFormatBuilder.java | 2 + .../flinkx/hbase/writer/HbaseWriter.java | 19 +- .../RowKeyFunctionTest.java | 5 +- .../dtstack/flinkx/hdfs/ECompressType.java | 1 - .../dtstack/flinkx/hdfs/HdfsConfigKeys.java | 2 + .../com/dtstack/flinkx/hdfs/HdfsUtil.java | 146 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 10 + .../hdfs/reader/BaseHdfsInputFormat.java | 47 +- .../hdfs/reader/HdfsOrcInputFormat.java | 197 +-- .../hdfs/reader/HdfsParquetInputFormat.java | 78 +- .../flinkx/hdfs/reader/HdfsReader.java | 4 +- .../hdfs/reader/HdfsTextInputFormat.java | 124 +- .../hadoop/hive/ql/io/orc/OrcInputFormat.java | 1221 +++++++++++++++++ .../hdfs/reader/HdfsOrcInputFormatTest.java | 22 + flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 10 + .../hdfs/writer/BaseHdfsOutputFormat.java | 25 +- .../hdfs/writer/HdfsOrcOutputFormat.java | 55 +- .../hdfs/writer/HdfsOutputFormatBuilder.java | 4 + .../hdfs/writer/HdfsParquetOutputFormat.java | 110 +- .../hdfs/writer/HdfsTextOutputFormat.java | 4 +- .../flinkx/hdfs/writer/HdfsWriter.java | 8 +- .../flinkx/hive/TimePartitionFormat.java | 6 +- .../hive/util/AbstractHiveMetadataParser.java | 4 +- .../dtstack/flinkx/hive/util/HiveDbUtil.java | 13 +- .../dtstack/flinkx/hive/util/HiveUtil.java | 210 +-- .../dtstack/flinkx/hive/test/DBUtilTest.java | 64 - flinkx-hive/flinkx-hive-writer/pom.xml | 10 + .../flinkx/hive/writer/HiveOutputFormat.java | 12 +- .../hive/writer/HiveOutputFormatBuilder.java | 3 + .../flinkx/hive/writer/HiveWriter.java | 70 +- flinkx-kafka/flinkx-kafka-reader/pom.xml | 10 + .../flinkx/kafka/reader/KafkaClient.java | 2 +- .../flinkx/kafka/reader/KafkaInputFormat.java | 2 + flinkx-kafka/flinkx-kafka-writer/pom.xml | 10 + flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 10 + .../flinkx/kafka09/reader/Kafka09Client.java | 2 +- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 10 + flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 10 + .../flinkx/kafka10/reader/Kafka10Client.java | 2 +- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 10 + flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 10 + .../flinkx/kafka11/reader/Kafka11Client.java | 2 +- .../kafka11/reader/Kafka11InputFormat.java | 2 + flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 10 + .../reader/KafkaBaseInputFormat.java | 9 +- .../kafkabase/reader/KafkaBaseReader.java | 6 +- .../writer/KafkaBaseOutputFormat.java | 10 +- .../kafkabase/writer/KafkaBaseWriter.java | 5 +- flinkx-kudu/flinkx-kudu-core/pom.xml | 6 - .../dtstack/flinkx/kudu/core/KuduUtil.java | 13 +- flinkx-kudu/flinkx-kudu-reader/pom.xml | 8 +- .../flinkx/kudu/reader/KuduInputFormat.java | 6 +- .../kudu/reader/KuduInputFormatBuilder.java | 5 + .../flinkx/kudu/reader/KuduReader.java | 14 +- flinkx-kudu/flinkx-kudu-writer/pom.xml | 8 +- .../flinkx/kudu/writer/KuduOutputFormat.java | 7 +- .../flinkx/kudu/writer/KuduWriter.java | 11 +- .../flinkx/launcher/ClassLoaderType.java | 17 + .../flinkx/launcher/ClusterClientFactory.java | 10 +- .../com/dtstack/flinkx/launcher/Launcher.java | 26 +- .../flinkx/launcher/YarnConfLoader.java | 3 +- .../perjob/FlinkPerJobResourceUtil.java | 4 +- .../launcher/perjob/PerJobSubmitter.java | 4 +- .../test/java/bigdata/FlinkConfigTest.java | 19 - .../flinkx/mongodb/MongodbClientUtil.java | 8 +- .../dtstack/flinkx/mongodb/MongodbUtil.java | 6 +- flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 8 +- .../mongodb/reader/MongodbInputFormat.java | 7 +- .../reader/MongodbInputFormatBuilder.java | 5 + flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 8 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 8 +- .../flinkx/mysqld/reader/MysqldReader.java | 5 +- flinkx-mysql/flinkx-mysql-reader/pom.xml | 10 +- .../flinkx/mysql/format/MysqlInputFormat.java | 14 +- .../src/test/java/bigdata/TestMysqlPk.java | 22 - flinkx-mysql/flinkx-mysql-writer/pom.xml | 8 +- .../com/dtstack/flinkx/odps/OdpsUtil.java | 2 +- flinkx-odps/flinkx-odps-reader/pom.xml | 10 + .../odps/reader/OdpsInputFormatBuilder.java | 4 +- .../flinkx/odps/reader/OdpsReader.java | 4 +- flinkx-odps/flinkx-odps-writer/pom.xml | 10 + .../flinkx/odps/writer/OdpsOutputFormat.java | 2 +- .../odps/writer/OdpsOutputFormatBuilder.java | 2 + .../flinkx/oracle/OracleDatabaseMeta.java | 2 +- flinkx-oracle/flinkx-oracle-reader/pom.xml | 8 +- .../oracle/reader/test/OracleLocalTest.java | 22 - flinkx-oracle/flinkx-oracle-writer/pom.xml | 8 +- .../oracle/format/OracleOutputFormat.java | 41 +- flinkx-pgwal/flinkx-pgwal-reader/pom.xml | 10 + flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 10 + .../phoenix/format/PhoenixInputFormat.java | 3 +- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 10 + flinkx-polardb/flinkx-polardb-dreader/pom.xml | 8 + .../polardbd/reader/PolardbdReader.java | 1 - flinkx-polardb/flinkx-polardb-reader/pom.xml | 8 + flinkx-polardb/flinkx-polardb-writer/pom.xml | 8 + .../postgresql/PostgresqlDatabaseMeta.java | 3 - .../postgresql/PostgresqlTypeConverter.java | 29 +- .../flinkx-postgresql-reader/pom.xml | 8 +- .../format/PostgresqlInputFormat.java | 9 +- .../flinkx-postgresql-writer/pom.xml | 8 +- flinkx-pulsar/flinkx-pulsar-writer/pom.xml | 3 - .../flinkx/pulsar/writer/Constants.java | 17 + .../pulsar/writer/PulsarOutputFormat.java | 19 +- .../writer/PulsarOutputFormatBuilder.java | 17 + .../flinkx/pulsar/writer/PulsarWriter.java | 21 +- .../dtstack/flinkx/rdb/BaseDatabaseMeta.java | 8 +- .../com/dtstack/flinkx/rdb/util/DbUtil.java | 197 +-- .../DistributedJdbcDataReader.java | 4 +- .../DistributedJdbcInputFormat.java | 20 +- .../DistributedJdbcInputFormatBuilder.java | 3 +- .../DistributedJdbcInputSplit.java | 6 +- .../JdbcInputFormat.java | 28 +- .../JdbcInputFormatBuilder.java | 5 + .../JdbcDataWriter.java | 10 +- .../rdb/outputformat/JdbcOutputFormat.java | 28 +- .../com/dtstack/flinkx/redis/JedisUtil.java | 9 +- flinkx-redis/flinkx-redis-writer/pom.xml | 8 +- .../redis/writer/RedisOutputFormat.java | 16 +- .../writer/RedisOutputFormatBuilder.java | 2 + .../flinkx/redis/writer/RedisWriter.java | 13 +- flinkx-restapi/flinkx-restapi-core/pom.xml | 20 + .../flinkx/restapi/common/HttpMethod.java | 44 + .../flinkx/restapi/common/HttpUtil.java | 115 ++ .../common/MyHttpRequestRetryHandler.java | 96 ++ .../MyServiceUnavailableRetryStrategy.java | 73 + .../flinkx/restapi/common/RestapiKeys.java | 32 + flinkx-restapi/flinkx-restapi-reader/pom.xml | 99 ++ .../inputformat/RestapiInputFormat.java | 109 ++ .../RestapiInputFormatBuilder.java | 46 + .../flinkx/restapi/reader/RestapiReader.java | 71 + flinkx-restapi/flinkx-restapi-writer/pom.xml | 99 ++ .../outputformat/RestapiOutputFormat.java | 150 ++ .../RestapiOutputFormatBuilder.java | 71 + .../flinkx/restapi/writer/RestapiWriter.java | 103 ++ flinkx-restapi/pom.xml | 28 + flinkx-saphana/flinkx-saphana-core/pom.xml | 4 +- flinkx-saphana/flinkx-saphana-reader/pom.xml | 4 +- flinkx-saphana/flinkx-saphana-writer/pom.xml | 4 +- flinkx-saphana/pom.xml | 4 +- .../flinkx-sqlserver-reader/pom.xml | 8 +- .../flinkx-sqlserver-writer/pom.xml | 8 +- flinkx-stream/flinkx-stream-reader/pom.xml | 10 + flinkx-stream/flinkx-stream-writer/pom.xml | 10 + .../stream/writer/StreamOutputFormat.java | 20 +- .../flinkx/stream/writer/StreamWriter.java | 2 +- .../dtstack/flinkx/teradata/util/DBUtil.java | 45 +- flinkx-test/pom.xml | 195 ++- .../com/dtstack/flinkx/test/LocalTest.java | 291 ++-- .../flinkx/test/PluginNameConstants.java | 5 + pom.xml | 54 +- 299 files changed, 6861 insertions(+), 2340 deletions(-) create mode 100644 docs/offline/reader/greenplumreader.md create mode 100644 docs/offline/writer/greenplumwriter.md delete mode 100644 docs/offline/writer/pulsarwriter.md create mode 100644 docs/realTime/reader/restapireader.md create mode 100644 docs/realTime/writer/restapiwriter.md rename flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java => flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java (73%) rename flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java => flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java (69%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/IDecode.java (96%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/JsonDecoder.java (97%) rename {flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase => flinkx-core/src/main/java/com/dtstack/flinkx}/decoder/PlainDecoder.java (96%) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java delete mode 100644 flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java delete mode 100644 flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java delete mode 100644 flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java delete mode 100644 flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java delete mode 100644 flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java delete mode 100644 flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java create mode 100644 flinkx-greenplum/flinkx-greenplum-core/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java create mode 100644 flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java create mode 100644 flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/pom.xml create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java create mode 100644 flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java create mode 100644 flinkx-greenplum/pom.xml delete mode 100644 flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java delete mode 100644 flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java create mode 100644 flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java create mode 100644 flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java delete mode 100644 flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java delete mode 100644 flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java delete mode 100644 flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java delete mode 100644 flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java create mode 100644 flinkx-restapi/flinkx-restapi-core/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/pom.xml create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java create mode 100644 flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java create mode 100644 flinkx-restapi/pom.xml diff --git a/.gitignore b/.gitignore index d8cb487ce5..1e7d443821 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,7 @@ # Created by .ignore support plugin (hsz.mobi) .idea/ plugins/ +syncplugins/ *.iml target/ lib/ @@ -8,4 +9,5 @@ jobs/ nohup.out flinkconf/ hadoopconf/ -/default_task_id_output \ No newline at end of file +/default_task_id_output +/syncplugins \ No newline at end of file diff --git a/README.md b/README.md index c2a3a77d5e..c391b42f5c 100644 --- a/README.md +++ b/README.md @@ -51,6 +51,7 @@ The following databases are currently supported: | | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | @@ -65,11 +66,12 @@ The following databases are currently supported: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon| | -| | SqlServer CDC | Coming Soon | | +| | Oracle Logminer| Coming Soon | | +| | SqlServer CDC | Coming Soon | | # Quick Start diff --git a/README_CH.md b/README_CH.md index 28ac7c27b2..8bb8c11dd0 100644 --- a/README_CH.md +++ b/README_CH.md @@ -51,6 +51,7 @@ FlinkX目前支持下面这些数据库: | | Teradata | [doc](docs/offline/reader/teradatareader.md) | [doc](docs/offline/writer/teradatawriter.md) | | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | +| | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | @@ -65,11 +66,12 @@ FlinkX目前支持下面这些数据库: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | +| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon | | -| | SqlServer CDC | Coming Soon | | +| | Oracle Logminer| Coming Soon | | +| | SqlServer CDC | Coming Soon | | # 快速开始 diff --git a/docs/offline/reader/greenplumreader.md b/docs/offline/reader/greenplumreader.md new file mode 100644 index 0000000000..a8d89386e7 --- /dev/null +++ b/docs/offline/reader/greenplumreader.md @@ -0,0 +1,340 @@ +# Greenplum Reader + + +## 一、插件名称 +名称:**greenplumreader**
+ +## 二、支持的数据源版本 +**Greenplum 5及以上**
+ + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[greenplum官方文档](https://gpdb.docs.pivotal.io/590/datadirect/datadirect_jdbc.html) + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **where** + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 默认值:无 + + + +- **splitPk** + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 默认值:无 + + + +- **fetchSize** + - 描述:读取时每批次读取的数据条数。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 默认值:1000 + + + +- **queryTimeOut** + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 默认值:1000 + + + +- **customSql** + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + - 只能是查询语句,否则会导致任务失败; + - 查询语句返回的字段需要和column列表里的字段对应; + - 当指定了此参数时,connection里指定的table无效; + - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:需要读取的字段。 + - 格式:支持3种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +```bash +"column":["*"] +``` +2.指定字段名称: +``` +"column":["id","name"] +``` +3.指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 默认值:无 + + + +- **polling** + - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + + + +- **pollingInterval** + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 默认值:5000 + + + +- **requestAccumulatorInterval** + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 默认值:2 + +** + +## 四、配置示例 + +#### 1、基础配置 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 2、多通道 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 3, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 3、 指定customsql +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "where": "", + "customSql": "select id from performance", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 4、增量同步指定startLocation +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "increColumn": "id", + "startLocation": "20", + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +#### 5、间隔轮询 +```json +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "gpadmin", + "password" : "gpadmin", + "connection" : [ { + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "table" : [ "performance" ] + } ], + "polling": true, + "pollingInterval": 3000, + "customSql": "", + "increColumn": "id", + "startLocation": "20", + "where": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` + +
说明:表名不应该有'.'号,否则会报如下错误:
`java.sql.SQLException: SQLException: [Pivotal][Greenplum JDBC Driver][Greenplum]cross-database references are not implemented: "public.public.test". ` diff --git a/docs/offline/writer/greenplumwriter.md b/docs/offline/writer/greenplumwriter.md new file mode 100644 index 0000000000..57a069ce88 --- /dev/null +++ b/docs/offline/writer/greenplumwriter.md @@ -0,0 +1,139 @@ +# Greenplum Writer + + +## 一、插件名称 +名称:**greenplumwriter**
+ +## 二、支持的数据源版本 +**Greenplum 5及以上**
** + +## 三、参数说明 + +- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 默认值:无 + + + +- **username** + - 描述:数据源的用户名 + - 必选:是 + - 默认值:无 + + + +- **password** + - 描述:数据源指定用户名的密码 + - 必选:是 + - 默认值:无 + + + +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 默认值:无 + + + +- **preSql** + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **postSql** + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 默认值:无 + + + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 默认值:无 + + + +- **writeMode** + - 描述:仅支持`insert`操作,可以搭配insertSqlMode使用 + - 必选:是 + - 默认值:无, + + + +- **insertSqlMode** + - 描述:控制写入数据到目标表采用  `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 + - 注意: + - 为了避免`insert`过慢带来的问题,此参数被固定为`copy` + - 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 + - 必选:否 + - 默认值:无 + + + +- **batchSize** + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 默认值:1024 + +** + +## 四、配置示例 + +#### 1、insert with copy mode +```json +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "int", + "value": 1 + } + ], + "sliceRecordCount": ["100"] + }, + "name" : "streamreader" + }, + "writer": { + "name": "greenplumwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb", + "table": ["tbl_pay_log_copy"] + }], + "username": "gpadmin", + "password": "gpadmin", + "column": [ + { + "name": "id", + "type": "int" + }], + "writeMode": "insert", + "insertSqlMode": "copy", + "batchSize": 100, + "preSql": ["TRUNCATE tbl_pay_log_copy"], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` diff --git a/docs/offline/writer/pulsarwriter.md b/docs/offline/writer/pulsarwriter.md deleted file mode 100644 index 413e1e0f30..0000000000 --- a/docs/offline/writer/pulsarwriter.md +++ /dev/null @@ -1,76 +0,0 @@ -# Pulsar写入插件(**writer) - -## 1. 配置样例 - -```json -{ - "job": { - "content": [ - { - "reader": { - - }, - "writer": { - "parameter": { - "producerSettings" : { - "producerName":"test-producer" - }, - "topic" : "pulsar_test", - "pulsarServiceUrl" : "pulsar://127.0.0.1:6650" - - }, - "name": "pulsarwriter" - } - } - ], - "setting": { - "errorLimit": { - "record": 1 - }, - "speed": { - "bytes": 1048576, - "channel": 1 - } - } - } -} -``` - -## 2. 参数说明 - -* **name** - - * 描述:插件名,pulsarwriter。 - - * 必选:是 - - * 默认值:无 - -* **topic** - - * 描述:topic。 - - * 必选:是 - - * 默认值:无 - - -* **pulsarServiceUrl** - - * 描述:pulsar地址列表 - - * 必选:是 - - * 默认值:无 - - - -* **producerSettings** - - * 描述:pulsar生产者配置 - - * 必选:是 - - * 默认值:无 - -参考: https://pulsar.apache.org/docs/en/client-libraries-java/#configure-producer \ No newline at end of file diff --git a/docs/quickstart.md b/docs/quickstart.md index fe581442a0..71779f930d 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -18,7 +18,7 @@ cd flink-1.8.5 ## 编译插件 ```bash -mvn clean package -DskipTests +mvn clean package -Dmaven.test.skip=true ``` ## 运行任务 @@ -232,6 +232,7 @@ bin/flinkx -mode yarnPer \ | **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | | **confProp** | checkpoint配置 | **flink.checkpoint.interval**:快照生产频率
**flink.checkpoint.stateBackend**:快照存储路径 | 否 | 无 | | **s** | checkpoint快照路径 | | 否 | 无 | +| **p** | 自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102| | 否 | 无 | ## 常见问题 diff --git a/docs/realTime/reader/restapireader.md b/docs/realTime/reader/restapireader.md new file mode 100644 index 0000000000..f354c823db --- /dev/null +++ b/docs/realTime/reader/restapireader.md @@ -0,0 +1,75 @@ +# Restapi Reader + + +## 一、插件名称 +名称:restapireader
+ + +## 二、参数说明 + +- **url** + - 描述:连接的url + - 必选:是 + - 默认值:无 + + + +- **method** + - 描述:request的类型,`post`、`get` + - 必选:是 + - 默认值:无 + + + +- header + - 描述:需要添加的报头信息 + - 必选:否 + - 默认值:无 + + + + +## 三、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", + "body": "", + "method": "get", + "params": "" + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "errorLimit": {}, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "trace", + "path": "", + "pattern": "" + } + } + } +} +``` + + diff --git a/docs/realTime/writer/restapiwriter.md b/docs/realTime/writer/restapiwriter.md new file mode 100644 index 0000000000..6a515b9a1e --- /dev/null +++ b/docs/realTime/writer/restapiwriter.md @@ -0,0 +1,115 @@ +# Restapi Writer + + +## 一、插件名称 +**名称:restapiwriter**
+ +## 二、参数说明 + +- **url** + - 描述:连接的url + - 必选:是 + - 默认值:无 + + + +- **method** + - 描述:request的类型,`post`、`get` + - 必选:是 + - 默认值:无 + + + +- **header** + - 描述:需要添加的报头信息 + - 必选:否 + - 默认值:无 + + + +- **body** + - 描述:发送的数据中包括params + - 必选:否 + - 默认值:无 + + + +- **params** + - 描述:发送的数据中包括params + - 必选:否 + - 默认值:无 + + + +- **column** + - 描述:如果column不为空,那么将数据和字段名一一对应。如果column为空,则返回每个数据的第一个字段。 + - 必选:否 + - 默认值:无 + + + + +## 三、使用示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "data", + "type": "string" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", + "header": [], + "body": [], + "method": "post", + "params": {}, + "column": ["id","data"] + }, + "name": "restapiwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` + + diff --git a/flinkx-binlog/flinkx-binlog-core/pom.xml b/flinkx-binlog/flinkx-binlog-core/pom.xml index d16755f37d..ba3a2485dd 100644 --- a/flinkx-binlog/flinkx-binlog-core/pom.xml +++ b/flinkx-binlog/flinkx-binlog-core/pom.xml @@ -15,12 +15,6 @@ - - com.google.guava - guava - 19.0 - - com.alibaba.otter canal.parse diff --git a/flinkx-binlog/flinkx-binlog-reader/pom.xml b/flinkx-binlog/flinkx-binlog-reader/pom.xml index ad9b9893df..302234b486 100644 --- a/flinkx-binlog/flinkx-binlog-reader/pom.xml +++ b/flinkx-binlog/flinkx-binlog-reader/pom.xml @@ -69,6 +69,10 @@ slf4j-api org.slf4j + + guava + com.google.guava + @@ -100,7 +104,11 @@ com.google.common - shade.binlog.com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java index 69d6d51add..0fcbdade5b 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogConfig.java @@ -32,7 +32,7 @@ public class BinlogConfig implements Serializable { public long period = 1000L; - public int bufferSize = 1024; + public int bufferSize = 256; public boolean pavingData = true; diff --git a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml index 111fc621c3..7b38ba5c8b 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-reader/pom.xml @@ -61,8 +61,12 @@ shade.carbondatareader.io.netty - com.google - shade.carbondatareader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java index f655d14934..77dd6dedba 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonExpressUtil.java @@ -19,7 +19,6 @@ import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.scan.expression.LiteralExpression; @@ -30,7 +29,6 @@ import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression; import org.apache.commons.lang3.StringUtils; -import java.util.ArrayList; import java.util.List; diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java index 5f440b5f33..91547fdfed 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbonFlinkInputSplit.java @@ -28,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; /** @@ -46,15 +45,24 @@ public class CarbonFlinkInputSplit implements InputSplit { public CarbonFlinkInputSplit(List carbonInputSplits, int splitNumber) throws IOException { this.splitNumber = splitNumber; rawSplits = new ArrayList<>(); - rawSplits.addAll(carbonInputSplits.stream().map(this::carbonSplitToRawSplit).collect(Collectors.toList())); + List list = new ArrayList<>(); + for (CarbonInputSplit carbonInputSplit : carbonInputSplits) { + byte[] bytes = carbonSplitToRawSplit(carbonInputSplit); + list.add(bytes); + } + rawSplits.addAll(list); } public List getCarbonInputSplits() throws IOException { - List carbonInputSplits = rawSplits.stream().map(this::rawSplitToCarbonSplit).collect(Collectors.toList()); + List carbonInputSplits = new ArrayList<>(); + for (byte[] rawSplit : rawSplits) { + CarbonInputSplit carbonInputSplit = rawSplitToCarbonSplit(rawSplit); + carbonInputSplits.add(carbonInputSplit); + } return carbonInputSplits; } - private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) { + private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) throws IOException{ ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos); try { @@ -62,17 +70,14 @@ private byte[] carbonSplitToRawSplit(CarbonInputSplit carbonInputSplit) { } catch (IOException e) { throw new RuntimeException(e); } finally { - try { - baos.close(); - dos.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } + baos.close(); + dos.close(); } + return baos.toByteArray(); } - private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) { + private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) throws IOException{ ByteArrayInputStream bais = new ByteArrayInputStream(rawSplit); DataInputStream dis = new DataInputStream(bais); CarbonInputSplit carbonInputSplit = new CarbonInputSplit(); @@ -81,13 +86,10 @@ private CarbonInputSplit rawSplitToCarbonSplit(byte[] rawSplit) { } catch (IOException e) { throw new RuntimeException(e); } finally { - try { - bais.close(); - dis.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } + bais.close(); + dis.close(); } + return carbonInputSplit; } diff --git a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java index 6b009c38f8..4942980eeb 100644 --- a/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java +++ b/flinkx-carbondata/flinkx-carbondata-reader/src/main/java/com/dtstack/flinkx/carbondata/reader/CarbondataReader.java @@ -25,6 +25,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -37,6 +40,8 @@ */ public class CarbondataReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(CarbondataReader.class); + protected String table; protected String database; @@ -78,7 +83,7 @@ public CarbondataReader(DataTransferConfig config, StreamExecutionEnvironment en columnValue.add((String) sm.get("value")); columnName.add((String) sm.get("name")); } - System.out.println("init column finished"); + LOG.info("init column finished"); } else if (!ConstantValue.STAR_SYMBOL.equals(columns.get(0)) || columns.size() != 1) { throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml index 9b414eafd3..1f45de883a 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/pom.xml +++ b/flinkx-carbondata/flinkx-carbondata-writer/pom.xml @@ -63,8 +63,12 @@ shade.carbondatawriter.io.netty - com.google - shade.carbondatawriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java index cb78b18f62..0dece097c9 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbonOutputFormat.java @@ -24,7 +24,6 @@ import com.dtstack.flinkx.carbondata.writer.recordwriter.RecordWriterFactory; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; -import com.dtstack.flinkx.util.DateUtil; import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; @@ -215,8 +214,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { @Override protected void writeMultipleRecordsInternal() throws Exception { - // CAN NOT HAPPEN - throw new IllegalArgumentException("It can not happen."); + notSupportBatchWrite("CarbondataWriter"); } @Override diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java index b621038fd4..f2ef5d821c 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/CarbondataOutputFormatBuilder.java @@ -89,5 +89,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("CarbondataWriter"); } } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java index 743acbcb23..6999225271 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtils.java @@ -58,6 +58,24 @@ public class DateTimeUtils { public static final int TO_YEAR_ZERO = TO_2001 + 7304850; + public static final int TIMESTAMP_STRING_LENGTH = 19; + + public static final String TIMESTAMP_ZERO_SUFFIX = ".0"; + + public static final int DAY_IN_YEAR_29 = 29; + public static final int DAY_IN_YEAR_31 = 31; + public static final int DAY_IN_YEAR_59 = 59; + public static final int DAY_IN_YEAR_60 = 60; + public static final int DAY_IN_YEAR_90 = 90; + public static final int DAY_IN_YEAR_120 = 120; + public static final int DAY_IN_YEAR_151 = 151; + public static final int DAY_IN_YEAR_181 = 181; + public static final int DAY_IN_YEAR_212 = 212; + public static final int DAY_IN_YEAR_243 = 243; + public static final int DAY_IN_YEAR_273 = 273; + public static final int DAY_IN_YEAR_304 = 304; + public static final int DAY_IN_YEAR_334 = 334; + public static final ThreadLocal THREAD_LOCAL_LOCAL_TIMEZONE = new ThreadLocal() { @Override public TimeZone initialValue() { @@ -91,8 +109,9 @@ public static String timestampToString(long us) { String timestampString = ts.toString(); String formatted = THREAD_LOCAL_TIMESTAMP_FORMAT.get().format(ts); - if(timestampString.length() > 19 && !".0".equals(timestampString.substring(19))) { - formatted += timestampString.substring(19); + if(timestampString.length() > TIMESTAMP_STRING_LENGTH + && !TIMESTAMP_ZERO_SUFFIX.equals(timestampString.substring(TIMESTAMP_STRING_LENGTH))) { + formatted += timestampString.substring(TIMESTAMP_STRING_LENGTH); } return formatted; } @@ -231,33 +250,33 @@ public static int getMonth(int date) { int year = tuple2.getField(0); int dayInYear = tuple2.getField(1); if (isLeapYear(year)) { - if (dayInYear == 60) { + if (dayInYear == DAY_IN_YEAR_60) { return 2; - } else if (dayInYear > 60) { + } else if (dayInYear > DAY_IN_YEAR_60) { dayInYear = dayInYear - 1; } } - if (dayInYear <= 31) { + if (dayInYear <= DAY_IN_YEAR_31) { return 1; - } else if (dayInYear <= 59) { + } else if (dayInYear <= DAY_IN_YEAR_59) { return 2; - } else if (dayInYear <= 90) { + } else if (dayInYear <= DAY_IN_YEAR_90) { return 3; - } else if (dayInYear <= 120) { + } else if (dayInYear <= DAY_IN_YEAR_120) { return 4; - } else if (dayInYear <= 151) { + } else if (dayInYear <= DAY_IN_YEAR_151) { return 5; - } else if (dayInYear <= 181) { + } else if (dayInYear <= DAY_IN_YEAR_181) { return 6; - } else if (dayInYear <= 212) { + } else if (dayInYear <= DAY_IN_YEAR_212) { return 7; - } else if (dayInYear <= 243) { + } else if (dayInYear <= DAY_IN_YEAR_243) { return 8; - } else if (dayInYear <= 273) { + } else if (dayInYear <= DAY_IN_YEAR_273) { return 9; - } else if (dayInYear <= 304) { + } else if (dayInYear <= DAY_IN_YEAR_304) { return 10; - } else if (dayInYear <= 334) { + } else if (dayInYear <= DAY_IN_YEAR_334) { return 11; } else { return 12; @@ -274,39 +293,38 @@ public static int getDayOfMonth(int date) { int year = tuple2.getField(0); int dayInYear = tuple2.getField(1); if (isLeapYear(year)) { - if (dayInYear == 60) { - return 29; - } else if (dayInYear > 60) { + if (dayInYear == DAY_IN_YEAR_60) { + return DAY_IN_YEAR_29; + } else if (dayInYear > DAY_IN_YEAR_60) { dayInYear = dayInYear - 1; } } - if (dayInYear <= 31) { + if (dayInYear <= DAY_IN_YEAR_31) { return dayInYear; - } else if (dayInYear <= 59) { - return dayInYear - 31; - } else if (dayInYear <= 90) { - return dayInYear - 59; - } else if (dayInYear <= 120) { - return dayInYear - 90; - } else if (dayInYear <= 151) { - return dayInYear - 120; - } else if (dayInYear <= 181) { - return dayInYear - 151; - } else if (dayInYear <= 212) { - return dayInYear - 181; - } else if (dayInYear <= 243) { - return dayInYear - 212; - } else if (dayInYear <= 273) { - return dayInYear - 243; - } else if (dayInYear <= 304) { - return dayInYear - 273; - } else if (dayInYear <= 334) { - return dayInYear - 304; + } else if (dayInYear <= DAY_IN_YEAR_59) { + return dayInYear - DAY_IN_YEAR_31; + } else if (dayInYear <= DAY_IN_YEAR_90) { + return dayInYear - DAY_IN_YEAR_59; + } else if (dayInYear <= DAY_IN_YEAR_120) { + return dayInYear - DAY_IN_YEAR_90; + } else if (dayInYear <= DAY_IN_YEAR_151) { + return dayInYear - DAY_IN_YEAR_120; + } else if (dayInYear <= DAY_IN_YEAR_181) { + return dayInYear - DAY_IN_YEAR_151; + } else if (dayInYear <= DAY_IN_YEAR_212) { + return dayInYear - DAY_IN_YEAR_181; + } else if (dayInYear <= DAY_IN_YEAR_243) { + return dayInYear - DAY_IN_YEAR_212; + } else if (dayInYear <= DAY_IN_YEAR_273) { + return dayInYear - DAY_IN_YEAR_243; + } else if (dayInYear <= DAY_IN_YEAR_304) { + return dayInYear - DAY_IN_YEAR_273; + } else if (dayInYear <= DAY_IN_YEAR_334) { + return dayInYear - DAY_IN_YEAR_304; } else { - return dayInYear - 334; + return dayInYear - DAY_IN_YEAR_334; } - } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java index 1b83ff80fe..dbff692bbb 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/ExternalCatalogUtils.java @@ -68,11 +68,4 @@ public static String escapePathName(String path) { private static boolean needsEscaping(char c) { return c >= 0 && c < charToEscape.size() && charToEscape.get(c); } - - public static void main(String[] args) { - String s = "2018-01-01 08:08:08"; - String d = escapePathName(s); - System.out.println(d); - } - } diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java index 188601b450..63e01e580c 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/AbstractRecordWriter.java @@ -39,11 +39,22 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import java.io.IOException; -import java.util.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Random; +import java.util.UUID; +import java.util.Map; +import java.util.Collections; +import java.util.HashMap; /** * Abstract record writer wrapper diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java index 3a3236ca76..b2a6b20d30 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/CarbonPartitionRecordWriter.java @@ -21,10 +21,7 @@ import com.dtstack.flinkx.carbondata.writer.dict.CarbonTypeConverter; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.StringUtil; import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.schema.PartitionInfo; import org.apache.carbondata.core.metadata.schema.partition.PartitionType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java index 38dc8e42e8..a74982de1b 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/recordwriter/HivePartitionRecordWriter.java @@ -34,11 +34,21 @@ import org.apache.carbondata.processing.util.CarbonLoaderUtil; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import java.io.IOException; -import java.util.*; +import java.util.List; +import java.util.Map; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Random; +import java.util.UUID; import java.util.stream.Collectors; diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java b/flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java similarity index 73% rename from flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java rename to flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java index ce63e6fd15..11a8f7136e 100644 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/IDecode.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/test/java/com/dtstack/flinkx/carbondata/writer/dict/DateTimeUtilsTest.java @@ -6,32 +6,29 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.emqx.decoder; -import java.util.Map; + +package com.dtstack.flinkx.carbondata.writer.dict; + +import org.junit.Test; /** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou + * @author jiangbo + * @date 2020/3/27 */ -public interface IDecode { - - /** - * 消息解码 - * @param message - * @return - */ - Map decode(String message); +public class DateTimeUtilsTest { + @Test + public void testGetMonth() { + DateTimeUtils.getMonth(10); + } } diff --git a/flinkx-cassandra/flinkx-cassandra-core/pom.xml b/flinkx-cassandra/flinkx-cassandra-core/pom.xml index 6dc4304d67..86b7f12abf 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-core/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java index 072a2b0096..a5a67df83d 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraConfigKeys.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra; /** diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java index fa89243336..bc0e631faa 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java @@ -32,7 +32,6 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.net.InetAddress; -import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.sql.Time; import java.util.Date; @@ -222,7 +221,7 @@ private static Optional objectToBytes(T obj){ * @param pos 位置 * @param sqlType cql类型 * @param value 值 - * @throws RuntimeException 对于不支持的数据类型,抛出异常 + * @throws Exception 对于不支持的数据类型,抛出异常 */ public static void bindColumn(BoundStatement ps, int pos, DataType sqlType, Object value) throws Exception { if (value != null) { diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java index 179e420193..6d5dde5398 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/test/java/com/dtstack/flinkx/cassandra/TestCassandraUtil.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra; import com.datastax.driver.core.*; diff --git a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml index feca24f0f9..804af7c5d6 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-reader/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java index cd8da1f553..df6dd94a03 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraConstants.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.reader; /** diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java index f5fdbe4ae2..52f1311861 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java @@ -20,7 +20,6 @@ import com.datastax.driver.core.*; import com.dtstack.flinkx.cassandra.CassandraUtil; -import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.google.common.base.Preconditions; @@ -97,7 +96,6 @@ public Row nextRecordInternal(Row row) { Object value = CassandraUtil.getData(cqlRow, definitions.get(i).getType(), definitions.get(i).getName()); row.setField(i, value); } - LOG.info(row.toString()); return row; } @@ -108,7 +106,7 @@ protected void closeInternal() { } @Override - public InputSplit[] createInputSplitsInternal(int minNumSplits) { + protected InputSplit[] createInputSplitsInternal(int minNumSplits) { ArrayList splits = new ArrayList<>(); try { @@ -130,12 +128,12 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) { private InputSplit[] splitJob(int minNumSplits, ArrayList splits) { if(minNumSplits <= 1) { splits.add(new CassandraInputSplit()); - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } if(whereString != null && whereString.toLowerCase().contains(CassandraConstants.TOKEN)) { splits.add(new CassandraInputSplit()); - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } Session session = CassandraUtil.getSession(cassandraConfig, ""); String partitioner = session.getCluster().getMetadata().getPartitioner(); @@ -152,8 +150,7 @@ private InputSplit[] splitJob(int minNumSplits, ArrayList s } splits.add(new CassandraInputSplit(l.toString(), r.toString())); } - } - else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { + }else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { BigDecimal minToken = BigDecimal.valueOf(Long.MIN_VALUE); BigDecimal maxToken = BigDecimal.valueOf(Long.MAX_VALUE); BigDecimal step = maxToken.subtract(minToken) @@ -166,11 +163,10 @@ else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { } splits.add(new CassandraInputSplit(String.valueOf(l), String.valueOf(r))); } - } - else { + }else { splits.add(new CassandraInputSplit()); } - return splits.toArray(new CassandraInputSplit[splits.size()]); + return splits.toArray(new CassandraInputSplit[0]); } /** @@ -181,7 +177,7 @@ else if(partitioner.endsWith(CassandraConstants.MURMUR3_PARTITIONER)) { private String getQueryString(CassandraInputSplit inputSplit) { StringBuilder columns = new StringBuilder(); if (columnMeta == null) { - columns.append(ConstantValue.STAR_SYMBOL); + columns.append("*"); } else { for(MetaColumn column : columnMeta) { if(columns.length() > 0 ) { diff --git a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml index 565ef3ca4f..73593ae1c9 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/pom.xml +++ b/flinkx-cassandra/flinkx-cassandra-writer/pom.xml @@ -1,6 +1,6 @@ - flinkx-cassandra diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java index f16eca03ab..a280dbf059 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormat.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.datastax.driver.core.*; @@ -102,7 +119,6 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { throw new WriteRecordException("类型转换失败", e.getCause(), i, row); } } - LOG.info("insertSql: {}" + boundStatement); session.execute(boundStatement); } @@ -110,12 +126,14 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { protected void writeMultipleRecordsInternal() throws Exception { if (batchSize > 1) { BoundStatement boundStatement = pstmt.bind(); - for (Row row : rows) { - for (int i = 0; i < columnMeta.size(); i++) { - Object value = row.getField(i); - CassandraUtil.bindColumn(boundStatement, i, columnTypes.get(i), value); + for (int rowIndex = 0; rowIndex < rows.size(); rowIndex++) { + for (int columnIndex = 0; columnIndex < columnMeta.size(); columnIndex++) { + Object value = rows.get(rowIndex).getField(columnIndex); + CassandraUtil.bindColumn(boundStatement, columnIndex, columnTypes.get(columnIndex), value); + } + if ((rowIndex % 1000) == 0) { + LOG.info("insertSql: {}", boundStatement); } - LOG.info("insertSql: {}" + boundStatement); if(asyncWrite) { unConfirmedWrite.add(session.executeAsync(boundStatement)); if (unConfirmedWrite.size() >= batchSize) { @@ -152,7 +170,7 @@ protected void writeMultipleRecordsInternal() throws Exception { } @Override - public void closeInternal() throws IOException { + public void closeInternal() { CassandraUtil.close(session); } } diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java index 4446f1633f..56a170811e 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraOutputFormatBuilder.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; diff --git a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java index 2e2f0bbc9c..1e89082846 100644 --- a/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java +++ b/flinkx-cassandra/flinkx-cassandra-writer/src/main/java/com/dtstack/flinkx/cassandra/writer/CassandraWriter.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.cassandra.writer; import com.dtstack.flinkx.config.DataTransferConfig; @@ -13,8 +30,6 @@ import java.util.Map; import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.*; -import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_CONSITANCY_LEVEL; -import static com.dtstack.flinkx.cassandra.CassandraConfigKeys.KEY_MAX_PENDING_CONNECTION; /** * diff --git a/flinkx-cassandra/pom.xml b/flinkx-cassandra/pom.xml index d788a4bea1..917b1dc9f8 100644 --- a/flinkx-cassandra/pom.xml +++ b/flinkx-cassandra/pom.xml @@ -1,6 +1,6 @@ - flinkx-all diff --git a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java index b33160d10d..c6103d9dfa 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java +++ b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java @@ -23,6 +23,7 @@ import java.sql.Connection; import java.sql.SQLException; +import java.sql.Statement; import java.util.Properties; /** @@ -43,8 +44,10 @@ public static Connection getConnection(String url, String username, String passw for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { try { conn = new BalancedClickhouseDataSource(url, properties).getConnection(); - conn.createStatement().execute("select 111"); - failed = false; + try (Statement statement = conn.createStatement()) { + statement.execute("select 111"); + failed = false; + } } catch (Exception e) { if (conn != null) { conn.close(); diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index 01d798e35d..4c5ef8cf4c 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index 94e72635a6..8c3d65ad12 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 223aae3824..a0bd729335 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -110,33 +110,6 @@ ${flink.version} - - junit - junit - 4.11 - test - - - - org.powermock - powermock-api-mockito - 1.7.0 - test - - - - org.powermock - powermock-module-junit4 - 1.7.0 - test - - - - org.hamcrest - hamcrest-all - 1.3 - test - org.apache.httpcomponents httpcore @@ -200,6 +173,10 @@ org.slf4j:* org.apache.httpcomponents:* io.prometheus:* + org.apache.avro:* + org.codehaus.jackson:* + com.fasterxml.jackson.core:* + commons-*:* @@ -207,6 +184,10 @@ com.google.common shade.core.com.google.common + + com.google.thirdparty + shade.core.com.google.thirdparty + org.apache.http shade.core.org.apache.http diff --git a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java index 56fb01de1c..c3dcf4ca9e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java +++ b/flinkx-core/src/main/java/com/dtstack/flink/api/java/MyLocalStreamEnvironment.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index bcac73e949..b36613bdb7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -30,7 +30,7 @@ import com.dtstack.flinkx.writer.DataWriterFactory; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.io.Charsets; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java index c89f0b2274..8fde360336 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java @@ -46,6 +46,7 @@ public class KerberosUtil { private static final String SP = "/"; private static final String KEY_SFTP_CONF = "sftpConf"; + private static final String KEY_PRINCIPAL = "principal"; private static final String KEY_REMOTE_DIR = "remoteDir"; private static final String KEY_USE_LOCAL_FILE = "useLocalFile"; public static final String KEY_PRINCIPAL_FILE = "principalFile"; @@ -89,6 +90,15 @@ public static UserGroupInformation loginAndReturnUgi(Configuration conf, String return UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); } + public static String getPrincipal(Map configMap, String keytabPath) { + String principal = MapUtils.getString(configMap, KEY_PRINCIPAL); + if (StringUtils.isEmpty(principal)) { + principal = findPrincipalFromKeytab(keytabPath); + } + + return principal; + } + private static void reloadKrb5Conf(Configuration conf){ String krb5File = conf.get(KEY_JAVA_SECURITY_KRB5_CONF); LOG.info("set krb5 file:{}", krb5File); @@ -188,7 +198,7 @@ private static String loadFromSftp(Map config, String fileName){ } } - public static String findPrincipalFromKeytab(String keytabFile) { + private static String findPrincipalFromKeytab(String keytabFile) { KeyTab keyTab = KeyTab.getInstance(keytabFile); for (KeyTabEntry entry : keyTab.getEntries()) { String principal = entry.getService().getName(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java index 90c0e127fb..dc1a70a1b4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/SftpHandler.java @@ -20,13 +20,18 @@ package com.dtstack.flinkx.authenticate; import com.dtstack.flinkx.util.RetryUtil; -import com.jcraft.jsch.*; +import com.jcraft.jsch.ChannelSftp; +import com.jcraft.jsch.JSch; +import com.jcraft.jsch.Session; +import com.jcraft.jsch.SftpException; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.File; +import java.io.FileOutputStream; +import java.io.OutputStream; import java.util.Map; import java.util.Properties; import java.util.concurrent.Callable; @@ -131,21 +136,11 @@ private void downloadFile(String ftpPath, String localPath){ throw new RuntimeException("File not exist on sftp:" + ftpPath); } - OutputStream os = null; - try { - os = new FileOutputStream(new File(localPath)); + try (OutputStream os = new FileOutputStream(new File(localPath))){ channelSftp.get(ftpPath, os); + os.flush(); } catch (Exception e){ throw new RuntimeException("download file from sftp error", e); - } finally { - if(os != null){ - try { - os.flush(); - os.close(); - } catch (IOException e) { - LOG.warn("", e); - } - } } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java index 9cb8f0220a..a6276ff42a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/ClassLoaderManager.java @@ -25,7 +25,13 @@ import java.net.URL; import java.net.URLClassLoader; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Arrays; import java.util.concurrent.ConcurrentHashMap; /** diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java index fd51f7e65c..efee645720 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java @@ -24,9 +24,7 @@ import java.io.File; import java.net.MalformedURLException; import java.net.URL; -import java.util.ArrayList; import java.util.HashSet; -import java.util.List; import java.util.Set; /** diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java index df46175089..087c5302bd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java @@ -76,9 +76,9 @@ public Object getVal(String key) { Object obj = internalMap.get(key); if (obj instanceof LinkedTreeMap) { LinkedTreeMap treeMap = (LinkedTreeMap) obj; - Map newMap = new HashMap<>(treeMap.size()); - newMap.putAll(treeMap); - return newMap; + Map map = new HashMap<>(Math.max((int) (treeMap.size()/.75f) + 1, 16)); + map.putAll(treeMap); + return map; } return obj; } @@ -109,10 +109,10 @@ public int getIntVal(String key, int defaultValue) { return defaultValue; } if(ret instanceof Integer) { - return ((Integer)ret).intValue(); + return (Integer) ret; } if(ret instanceof String) { - return Integer.valueOf((String)ret).intValue(); + return Integer.parseInt((String) ret); } if(ret instanceof Long) { return ((Long)ret).intValue(); @@ -144,7 +144,7 @@ public long getLongVal(String key, long defaultValue) { return ((Integer)ret).longValue(); } if(ret instanceof String) { - return Long.valueOf((String)ret); + return Long.parseLong((String)ret); } if(ret instanceof Float) { return ((Float)ret).longValue(); @@ -176,7 +176,7 @@ public double getDoubleVal(String key, double defaultValue) { return ((Integer) ret).doubleValue(); } if (ret instanceof String) { - return Double.valueOf((String) ret); + return Double.parseDouble((String) ret); } if (ret instanceof Float) { return ((Float) ret).doubleValue(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java index 5e58d85e5a..f3e732da0b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java @@ -90,7 +90,7 @@ private static void checkConfig(DataTransferConfig config) { Preconditions.checkNotNull(readerParameter, "Must specify parameter for reader"); - // 检查我writer配置 + // 检查writer配置 WriterConfig writerConfig = content.getWriter(); Preconditions.checkNotNull(writerConfig, "Must specify a writer element"); Preconditions.checkNotNull(writerConfig.getName(), "Must specify the writer name"); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java index 35068840ab..f83a394399 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestartConfig.java @@ -49,7 +49,7 @@ public RestartConfig(Map map) { } public static RestartConfig defaultConfig(){ - Map map = new HashMap<>(1); + Map map = new HashMap<>(16); return new RestartConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java index c85c1bb376..ef886bf7e7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/RestoreConfig.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx.config; -import java.util.HashMap; +import java.util.Collections; import java.util.Map; /** @@ -40,8 +40,7 @@ public RestoreConfig(Map map) { } public static RestoreConfig defaultConfig(){ - Map map = new HashMap<>(1); - map.put(KEY_IS_RESTORE, false); + Map map = Collections.singletonMap(KEY_IS_RESTORE, false); return new RestoreConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java index 2a15601654..dbcd774858 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/TestConfig.java @@ -37,7 +37,7 @@ public TestConfig(Map map) { } public static TestConfig defaultConfig(){ - Map map = new HashMap<>(1); + Map map = new HashMap<>(16); return new TestConfig(map); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index 46de3749fb..d94589fbc6 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -51,4 +51,15 @@ public class ConstantValue { public static final String SYSTEM_PROPERTIES_KEY_FILE_ENCODING = "file.encoding"; public static final String OS_WINDOWS = "windows"; + + public static final String TIME_SECOND_SUFFIX = "sss"; + public static final String TIME_MILLISECOND_SUFFIX = "SSS"; + + public static final String FILE_SUFFIX_XML = ".xml"; + + public static final int MAX_BATCH_SIZE = 200000; + + public static final long STORE_SIZE_G = 1024L * 1024 * 1024; + + public static final long STORE_SIZE_M = 1024L * 1024; } diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java similarity index 69% rename from flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java index 63d92b8463..ce360c6bc0 100644 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/PlainDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java @@ -6,31 +6,41 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.emqx.decoder; -import java.util.Collections; -import java.util.Map; +package com.dtstack.flinkx.decoder; /** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou + * @author jiangbo + * @date 2020/3/23 */ -public class PlainDecoder implements IDecode { +public enum DecodeEnum { + + /** + * json format + */ + JSON("json"), - @Override - public Map decode(final String message) { - return Collections.singletonMap("message", message); + /** + * text format + */ + PLAIN("plain"); + + private String name; + + DecodeEnum(String name) { + this.name = name; } + public String getName() { + return name; + } } diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java similarity index 96% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java index 0d117d3956..5b195bd668 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/IDecode.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/IDecode.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import java.util.Map; diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java similarity index 97% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java index 8d6fad179e..10a954e601 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/JsonDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; @@ -31,12 +31,10 @@ * @author tudou */ public class JsonDecoder implements IDecode { + private static final String KEY_MESSAGE = "message"; private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - private static ObjectMapper objectMapper = new ObjectMapper(); - private static final String KEY_MESSAGE = "message"; - @Override @SuppressWarnings("unchecked") public Map decode(final String message) { diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java similarity index 96% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java index 5a2a027369..63771cfd6e 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/decoder/PlainDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.decoder; +package com.dtstack.flinkx.decoder; import java.util.Collections; import java.util.Map; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java index 475f6923d1..8de25bd051 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java @@ -33,6 +33,7 @@ public enum EDatabaseType { SQLServer, Oracle, PostgreSQL, + Greenplum, DB2, MongoDB, Redis, diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 1ce9844175..2c55ee3a19 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -28,7 +28,6 @@ import com.dtstack.flinkx.metrics.CustomPrometheusReporter; import com.dtstack.flinkx.reader.ByteRateLimiter; import com.dtstack.flinkx.restore.FormatState; -import org.apache.commons.lang.StringUtils; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; @@ -376,14 +375,13 @@ public RestoreConfig getRestoreConfig() { return restoreConfig; } - public void setLogConfig(LogConfig logConfig) { - this.logConfig = logConfig; - } - public void setRestoreConfig(RestoreConfig restoreConfig) { this.restoreConfig = restoreConfig; } + public void setLogConfig(LogConfig logConfig) { + this.logConfig = logConfig; + } public void setTestConfig(TestConfig testConfig) { this.testConfig = testConfig; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java index c701707ade..a1f67a983f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/AccumulatorCollector.java @@ -25,7 +25,7 @@ import com.google.common.collect.Lists; import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.http.impl.client.CloseableHttpClient; @@ -35,8 +35,13 @@ import org.slf4j.LoggerFactory; import java.io.InputStream; -import java.util.*; -import java.util.concurrent.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * Regularly get statistics from the flink API @@ -247,7 +252,7 @@ private void checkErrorTimes() { } } - class ValueAccumulator{ + static class ValueAccumulator{ private long global; private LongCounter local; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java index 3735cc7c64..dfe35d84fe 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/CustomPrometheusReporter.java @@ -27,7 +27,13 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.*; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; @@ -39,7 +45,14 @@ import java.io.IOException; import java.lang.reflect.Field; -import java.util.*; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.regex.Pattern; /** @@ -121,7 +134,6 @@ public void open() { if (StringUtils.isNullOrWhitespaceOnly(host) || port < 1) { return; -// throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); } if (randomSuffix) { @@ -190,6 +202,10 @@ private void notifyOfAddedMetric(final Metric metric, final String metricName, f count = collectorWithCount.getValue(); } else { collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); + if (null == collector) { + return; + } + try { collector.register(defaultRegistry); } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java index f5991f1f9c..c46d451f48 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/OptionParser.java @@ -100,19 +100,15 @@ public List getProgramExeArgList() throws Exception { continue; }else if(OPTION_JOB.equalsIgnoreCase(key)){ File file = new File(value.toString()); - FileInputStream in = new FileInputStream(file); - byte[] filecontent = new byte[(int) file.length()]; - in.read(filecontent); - value = new String(filecontent, Charsets.UTF_8.name()); + try (FileInputStream in = new FileInputStream(file)) { + byte[] filecontent = new byte[(int) file.length()]; + in.read(filecontent); + value = new String(filecontent, Charsets.UTF_8.name()); + } } args.add("-" + key); args.add(value.toString()); } return args; } - - private void printUsage() { - System.out.print(options.toString()); - } - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 0123d3f955..26a1189ecd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -64,9 +64,9 @@ public class Options { @OptionRequired(description = "env properties") private String confProp = "{}"; - /** - * savepoint - */ + @OptionRequired(description = "json modify") + private String p = ""; + @OptionRequired(description = "savepoint path") private String s; @@ -195,4 +195,12 @@ public String getPluginLoadMode() { public void setPluginLoadMode(String pluginLoadMode) { this.pluginLoadMode = pluginLoadMode; } + + public String getP() { + return p; + } + + public void setP(String p) { + this.p = p; + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java index 6d92035700..f9735c0ab3 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java @@ -294,11 +294,12 @@ protected void afterCloseInternal() { moveAllTemporaryDataFileToDirectory(); LOG.info("The task ran successfully,clear temporary data files"); + closeSource(); clearTemporaryDataFiles(); } + }else{ + closeSource(); } - - closeSource(); } catch(Exception ex) { throw new RuntimeException(ex); } @@ -328,11 +329,6 @@ public void tryCleanupOnError() throws Exception { } } - @Override - protected void writeMultipleRecordsInternal() throws Exception { - // CAN NOT HAPPEN - } - @Override protected boolean needWaitAfterCloseInternal() { return true; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java index 1b42017236..39ce43bc7b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java @@ -34,7 +34,7 @@ import com.dtstack.flinkx.writer.ErrorLimiter; import com.google.gson.JsonObject; import com.google.gson.JsonParser; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; import org.apache.flink.configuration.Configuration; @@ -51,7 +51,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.writer.WriteErrorTypes.*; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_FORMAT_TRANSFORM; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_NULL_POINTER; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_PRIMARY_CONFLICT; /** * Abstract Specification for all the OutputFormat defined in flinkx plugins @@ -418,6 +420,10 @@ protected void writeMultipleRecords() throws Exception { */ protected abstract void writeMultipleRecordsInternal() throws Exception; + protected void notSupportBatchWrite(String writerName) { + throw new UnsupportedOperationException(writerName + "不支持批量写入"); + } + protected void writeRecordInternal() { try { writeMultipleRecords(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java index 0bdcca624a..38306365f2 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormatBuilder.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.outputformat; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.constants.ConstantValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; @@ -71,6 +72,12 @@ public void setInitAccumulatorAndDirty(boolean initAccumulatorAndDirty) { this.format.initAccumulatorAndDirty = initAccumulatorAndDirty; } + protected void notSupportBatchWrite(String writerName) { + if (this.format.getBatchInterval() > 1) { + throw new IllegalArgumentException(writerName + "不支持批量写入"); + } + } + /** * Check the value of parameters */ @@ -78,7 +85,17 @@ public void setInitAccumulatorAndDirty(boolean initAccumulatorAndDirty) { public BaseRichOutputFormat finish() { checkFormat(); + + /** + * 200000条限制的原因: + * 按照目前的使用情况以及部署配置,假设写入字段数量平均为50个,一个单slot的TaskManager内存为1G, + * 在不考虑各插件批量写入对内存特殊要求并且只考虑插件缓存这么多条数据的情况下,batchInterval为400000条时出现fullGC, + * 为了避免fullGC以及OOM,并且保证batchInterval有足够的配置空间,取最大值的一半200000。 + */ + if (this.format.getBatchInterval() > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量写入条数必须小于[200000]条"); + } + return format; } - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java index f66d45b7f9..13c21f8129 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/ByteRateLimiter.java @@ -25,7 +25,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.*; +import java.math.BigDecimal; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; /** * This class is user for speed control @@ -80,11 +84,12 @@ private void updateRate(){ long thisRecords = accumulatorCollector.getLocalAccumulatorValue(Metrics.NUM_READS); long totalRecords = accumulatorCollector.getAccumulatorValue(Metrics.NUM_READS); - double thisWriteRatio = (totalRecords == 0 ? 0 : thisRecords / (double)totalRecords); + BigDecimal thisWriteRatio = BigDecimal.valueOf(totalRecords == 0 ? 0 : thisRecords / (double) totalRecords); - if (totalRecords > MIN_RECORD_NUMBER_UPDATE_RATE && totalBytes != 0 && thisWriteRatio != 0) { - double bpr = totalBytes / totalRecords; - double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio; + if (totalRecords > MIN_RECORD_NUMBER_UPDATE_RATE && totalBytes != 0 + && thisWriteRatio.compareTo(new BigDecimal(0)) == 0) { + double bpr = totalBytes / (double)totalRecords; + double permitsPerSecond = expectedBytePerSecond / bpr * thisWriteRatio.doubleValue(); rateLimiter.setRate(permitsPerSecond); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java index d328115334..a7436f894f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/MetaColumn.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.lang.StringUtils; import java.io.Serializable; @@ -48,6 +47,8 @@ public class MetaColumn implements Serializable { private String splitter; + private Boolean isPart; + public String getSplitter() { return splitter; } @@ -124,6 +125,7 @@ public static List getMetaColumns(List columns, boolean generateInde mc.setType(sm.get("type") != null ? String.valueOf(sm.get("type")) : null); mc.setValue(sm.get("value") != null ? String.valueOf(sm.get("value")) : null); mc.setSplitter(sm.get("splitter") != null ? String.valueOf(sm.get("splitter")) : null); + mc.setPart(sm.get("isPart") != null ? (Boolean) sm.get("isPart") : false); if(sm.get("format") != null && String.valueOf(sm.get("format")).trim().length() > 0){ mc.setTimeFormat(DateUtil.buildDateFormatter(String.valueOf(sm.get("format")))); @@ -151,6 +153,14 @@ public static List getMetaColumns(List columns, boolean generateInde return metaColumns; } + public Boolean getPart() { + return isPart; + } + + public void setPart(Boolean part) { + isPart = part; + } + public static List getMetaColumns(List columns){ return getMetaColumns(columns, true); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java index bba853e451..b073524412 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java @@ -33,7 +33,7 @@ public class ClassUtil { private static final Logger LOG = LoggerFactory.getLogger(ClassUtil.class); - public final static String LOCK_STR = "jdbc_lock_str"; + public final static Object LOCK_STR = new Object(); public static void forName(String clazz, ClassLoader classLoader) { synchronized (LOCK_STR){ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java index 62f3a7e98c..c003af9457 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/Clock.java @@ -46,9 +46,14 @@ static Clock system() { return SYSTEM; } + /** + * Get current time + * @return time + */ default Instant currentTime() { return Instant.ofEpochMilli(currentTimeInMillis()); } + /** * Get the current time in nanoseconds. * @return the current time in nanoseconds. diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index 8ce417196d..a9b2ce6e53 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -21,7 +21,11 @@ import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.Calendar; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; /** * Date Utilities @@ -93,7 +97,12 @@ public static java.sql.Date columnToDate(Object column,SimpleDateFormat customTi if (((String) column).length() == 0){ return null; } - return new java.sql.Date(stringToDate((String)column,customTimeFormat).getTime()); + + Date date = stringToDate((String)column, customTimeFormat); + if (null == date) { + return null; + } + return new java.sql.Date(date.getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Date(getMillSecond(rawData.toString())); @@ -120,7 +129,12 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma if (((String) column).length() == 0){ return null; } - return new java.sql.Timestamp(stringToDate((String)column,customTimeFormat).getTime()); + + Date date = stringToDate((String)column,customTimeFormat); + if (null == date) { + return null; + } + return new java.sql.Timestamp(date.getTime()); } else if (column instanceof Integer) { Integer rawData = (Integer) column; return new java.sql.Timestamp(getMillSecond(rawData.toString())); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java index cddcf1ba29..d770364db7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ExceptionUtil.java @@ -33,30 +33,19 @@ public class ExceptionUtil { private static Logger logger = LoggerFactory.getLogger(ExceptionUtil.class); public static String getErrorMessage(Throwable e) { - StringWriter stringWriter = null; - PrintWriter writer = null; - try{ - stringWriter= new StringWriter(); - writer = new PrintWriter(stringWriter); - e.printStackTrace(writer); - writer.flush(); - stringWriter.flush(); - StringBuffer buffer= stringWriter.getBuffer(); - return buffer.toString(); - }catch(Throwable ee){ - logger.error("",ee); + if (null == e) { + return null; + } - }finally { - if(writer!=null){ - writer.close(); - } - if(stringWriter!=null){ - try{ - stringWriter.close(); - }catch (Throwable ee){ - logger.error("",ee); - } - } + try (StringWriter stringWriter = new StringWriter(); + PrintWriter writer = new PrintWriter(stringWriter)) { + e.printStackTrace(writer); + writer.flush(); + stringWriter.flush(); + StringBuffer buffer = stringWriter.getBuffer(); + return buffer.toString(); + } catch (Throwable ee) { + logger.error("", ee); } return null; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java index c545ef3d94..0e7214fb58 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.security.PrivilegedAction; import java.util.HashMap; import java.util.Map; @@ -76,7 +77,7 @@ public static void setHadoopUserName(Configuration conf){ } } - private static boolean isOpenKerberos(Map hadoopConfig){ + public static boolean isOpenKerberos(Map hadoopConfig){ if(!MapUtils.getBoolean(hadoopConfig, KEY_HADOOP_SECURITY_AUTHORIZATION, false)){ return false; } @@ -85,13 +86,7 @@ private static boolean isOpenKerberos(Map hadoopConfig){ } private static FileSystem getFsWithKerberos(Map hadoopConfig, String defaultFs) throws Exception{ - String keytabFileName = KerberosUtil.getPrincipalFileName(hadoopConfig); - - keytabFileName = KerberosUtil.loadFile(hadoopConfig, keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); - KerberosUtil.loadKrb5Conf(hadoopConfig); - - UserGroupInformation ugi = KerberosUtil.loginAndReturnUgi(getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); + UserGroupInformation ugi = getUGI(hadoopConfig, defaultFs); UserGroupInformation.setLoginUser(ugi); return ugi.doAs(new PrivilegedAction() { @@ -106,6 +101,19 @@ public FileSystem run(){ }); } + public static UserGroupInformation getUGI(Map hadoopConfig, String defaultFs) throws IOException { + String keytabFileName = KerberosUtil.getPrincipalFileName(hadoopConfig); + + keytabFileName = KerberosUtil.loadFile(hadoopConfig, keytabFileName); + String principal = KerberosUtil.getPrincipal(hadoopConfig, keytabFileName); + KerberosUtil.loadKrb5Conf(hadoopConfig); + + UserGroupInformation ugi = KerberosUtil.loginAndReturnUgi(getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); + UserGroupInformation.setLoginUser(ugi); + + return ugi; + } + public static Configuration getConfiguration(Map confMap, String defaultFs) { confMap = fillConfig(confMap, defaultFs); @@ -134,7 +142,7 @@ public static JobConf getJobConf(Map confMap, String defaultFs){ private static Map fillConfig(Map confMap, String defaultFs) { if (confMap == null) { - confMap = new HashMap<>(8); + confMap = new HashMap<>(); } if (isHaMode(confMap)) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java new file mode 100644 index 0000000000..b94765dc4a --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.util; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.TypeAdapter; +import com.google.gson.TypeAdapterFactory; +import com.google.gson.internal.LinkedTreeMap; +import com.google.gson.internal.bind.ObjectTypeAdapter; +import com.google.gson.reflect.TypeToken; +import com.google.gson.stream.JsonReader; +import com.google.gson.stream.JsonToken; +import com.google.gson.stream.JsonWriter; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Date: 2020/06/12 + * Company: www.dtstack.com + * + * Gson工具类,用于对json的序列化及反序列化,及解决int类型在map中被转换成double类型问题 + * + * @author tudou + */ +public class GsonUtil { + + public static Gson GSON = getGson(); + public static Type gsonMapTypeToken = new TypeToken>(){}.getType(); + + @SuppressWarnings("unchecked") + private static Gson getGson() { + GSON = new GsonBuilder().create(); + try { + Field factories = Gson.class.getDeclaredField("factories"); + factories.setAccessible(true); + Object o = factories.get(GSON); + Class[] declaredClasses = Collections.class.getDeclaredClasses(); + for (Class c : declaredClasses) { + if ("java.util.Collections$UnmodifiableList".equals(c.getName())) { + Field listField = c.getDeclaredField("list"); + listField.setAccessible(true); + List list = (List) listField.get(o); + int i = list.indexOf(ObjectTypeAdapter.FACTORY); + list.set(i, new TypeAdapterFactory() { + @Override + public TypeAdapter create(Gson gson, TypeToken type) { + if (type.getRawType() == Object.class) { + return new TypeAdapter() { + @Override + public Object read(JsonReader in) throws IOException { + JsonToken token = in.peek(); + //判断字符串的实际类型 + switch (token) { + case BEGIN_ARRAY: + List list = new ArrayList<>(); + in.beginArray(); + while (in.hasNext()) { + list.add(read(in)); + } + in.endArray(); + return list; + + case BEGIN_OBJECT: + Map map = new LinkedTreeMap<>(); + in.beginObject(); + while (in.hasNext()) { + map.put(in.nextName(), read(in)); + } + in.endObject(); + return map; + case STRING: + return in.nextString(); + case NUMBER: + String s = in.nextString(); + if (s.contains(".")) { + return Double.valueOf(s); + } else { + try { + return Integer.valueOf(s); + } catch (Exception e) { + return Long.valueOf(s); + } + } + case BOOLEAN: + return in.nextBoolean(); + case NULL: + in.nextNull(); + return null; + default: + throw new IllegalStateException(); + } + } + + @Override + public void write(JsonWriter out, Object value) throws IOException { + if (value == null) { + out.nullValue(); + return; + } + //noinspection unchecked + TypeAdapter typeAdapter = gson.getAdapter((Class) value.getClass()); + if (typeAdapter instanceof ObjectTypeAdapter) { + out.beginObject(); + out.endObject(); + return; + } + typeAdapter.write(out, value); + } + }; + } + return null; + } + }); + break; + } + } + } catch (Exception e) { + e.printStackTrace(); + } + return GSON; + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java new file mode 100644 index 0000000000..2e912373a7 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.util; + +import org.apache.commons.lang3.StringUtils; + +import java.util.HashMap; + +/** + * @author tiezhu + */ +public class JsonModifyUtil { + + public static String JsonValueReplace(String json, HashMap parameter){ + for(String item: parameter.keySet()){ + if(json.contains("${"+item+"}")){ + json = json.replace("${"+item+"}", parameter.get(item)); + } + } + return json; + } + + /** + * 将命令行中的修改命令转化为HashMap保存 + */ + public static HashMap CommandTransform(String command) { + HashMap parameter = new HashMap<>(); + String[] split = StringUtils.split(command, ","); + for (String item : split) { + String[] temp = item.split("="); + parameter.put(temp[0], temp[1]); + } + return parameter; + } +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java index 78a328fdac..b4656af3ac 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ResultPrintUtil.java @@ -22,6 +22,8 @@ import com.google.common.collect.Lists; import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.JobExecutionResult; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; @@ -31,6 +33,8 @@ */ public class ResultPrintUtil { + private static Logger LOG = LoggerFactory.getLogger(ResultPrintUtil.class); + public static void printResult(JobExecutionResult result){ List names = Lists.newArrayList(); List values = Lists.newArrayList(); @@ -56,8 +60,8 @@ public static void printResult(JobExecutionResult result){ } } - System.out.println("---------------------------------"); - System.out.println(builder.toString()); - System.out.println("---------------------------------"); + LOG.info("---------------------------------"); + LOG.info(builder.toString()); + LOG.info("---------------------------------"); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java index 000d7a7ba2..5f8329a2e4 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java @@ -33,7 +33,7 @@ public final class RetryUtil { private static final Logger LOG = LoggerFactory.getLogger(RetryUtil.class); - private static final long MAX_SLEEP_MILLISECOND = 256 * 1000; + private static final long MAX_SLEEP_MILLISECOND = 256 * 1000L; /** * 重试次数工具方法. diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index b276d856cd..5ae0ca309c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -38,6 +38,8 @@ */ public class StringUtil { + public static final int STEP_SIZE = 2; + /** * Handle the escaped escape charactor. * @@ -192,14 +194,14 @@ public static String col2string(Object column, String type) { } - public static String row2string(Row row, List columnTypes, String delimiter, List columnNames) throws WriteRecordException { + public static String row2string(Row row, List columnTypes, String delimiter) throws WriteRecordException { // convert row to string - int cnt = row.getArity(); - StringBuilder sb = new StringBuilder(); + int size = row.getArity(); + StringBuilder sb = new StringBuilder(128); int i = 0; try { - for (; i < cnt; ++i) { + for (; i < size; ++i) { if (i != 0) { sb.append(delimiter); } @@ -212,9 +214,9 @@ public static String row2string(Row row, List columnTypes, String delimi sb.append(col2string(column, columnTypes.get(i))); } - } catch(Exception ex) { + } catch(Exception e) { String msg = "StringUtil.row2string error: when converting field[" + i + "] in Row(" + row + ")"; - throw new WriteRecordException(msg, ex, i, row); + throw new WriteRecordException(msg, e, i, row); } return sb.toString(); @@ -228,7 +230,7 @@ public static byte[] hexStringToByteArray(String hexString) { int length = hexString.length(); byte[] bytes = new byte[length / 2]; - for (int i = 0; i < length; i += 2) { + for (int i = 0; i < length; i += STEP_SIZE) { bytes[i / 2] = (byte) ((Character.digit(hexString.charAt(i), 16) << 4) + Character.digit(hexString.charAt(i+1), 16)); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java index dd4ff9308a..3cc4719ea9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DirtyDataManager.java @@ -34,9 +34,16 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.ArrayList; +import java.util.Date; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; -import static com.dtstack.flinkx.writer.WriteErrorTypes.*; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_FORMAT_TRANSFORM; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_NULL_POINTER; +import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_PRIMARY_CONFLICT; /** * The class handles dirty data management diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 1d2f7d9ae8..189f198a15 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -29,7 +29,8 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -public class ErrorLimiter { +public class +ErrorLimiter { private final Integer maxErrors; private final Double maxErrorRatio; diff --git a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index e331b3e672..b8ca7cdbc7 100644 --- a/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flinkx-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -57,7 +57,19 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.source.*; +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction; +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator; +import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction; +import org.apache.flink.streaming.api.functions.source.FileProcessingMode; +import org.apache.flink.streaming.api.functions.source.FileReadFunction; +import org.apache.flink.streaming.api.functions.source.FromElementsFunction; +import org.apache.flink.streaming.api.functions.source.FromIteratorFunction; +import org.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunction; +import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; +import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.StoppableStreamSource; diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java index 7b68e5fcc0..0210f3c05c 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ColumnTypeUtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java new file mode 100644 index 0000000000..6607984c4e --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/DateUtilTest.java @@ -0,0 +1,112 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; +import java.sql.Date; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class DateUtilTest { + + @Test + public void testColumnToDate() { + Date result = DateUtil.columnToDate(null, null); + Assert.assertNull(result); + + result = DateUtil.columnToDate("", null); + Assert.assertNull(result); + + result = DateUtil.columnToDate("2020-03-18 10:56:00", null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(1584500160, null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(1584500160000L, null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new Date(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new Timestamp(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + result = DateUtil.columnToDate(new java.util.Date(1584500160000L), null); + Assert.assertEquals(result, new Date(1584500160000L)); + + try { + DateUtil.columnToDate(true, null); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + @Test + public void testColumnToTimestamp() { + Timestamp result = DateUtil.columnToTimestamp(null, null); + Assert.assertNull(result); + + result = DateUtil.columnToTimestamp("", null); + Assert.assertNull(result); + + result = DateUtil.columnToTimestamp("2020-03-18 10:56:00", null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(1584500160, null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(1584500160000L, null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new Date(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new Timestamp(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + result = DateUtil.columnToTimestamp(new java.util.Date(1584500160000L), null); + Assert.assertEquals(result, new Timestamp(1584500160000L)); + + try { + DateUtil.columnToTimestamp(true, null); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + @Test + public void testGetMillSecond() { + long result = DateUtil.getMillSecond("1584500160000"); + Assert.assertEquals(result, 1584500160000L); + + result = DateUtil.getMillSecond("1584500160000000"); + Assert.assertEquals(result, 1584500160000L); + + result = DateUtil.getMillSecond("1584500160000000000"); + Assert.assertEquals(result, 1584500160000L); + + long expect = 57600000; // 1970-01-02 00:00:00:000 + result = DateUtil.getMillSecond("1"); + Assert.assertEquals(result, expect); + } + + @Test + public void testStringToDate() { + java.util.Date result = DateUtil.stringToDate("", null); + Assert.assertNull(result); + + result = DateUtil.stringToDate("2020/03/18 13:26:00", new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")); + Assert.assertEquals(result, new java.util.Date(1584509160000L)); + + try { + DateUtil.stringToDate("xxxx", null); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("can't parse date")); + } + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java new file mode 100644 index 0000000000..5d988bd64d --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ExceptionUtilTest.java @@ -0,0 +1,23 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class ExceptionUtilTest { + + @Test + public void testGetErrorMessage() { + String result = ExceptionUtil.getErrorMessage(null); + Assert.assertNull(result); + + try { + ExceptionUtil.getErrorMessage(new IllegalArgumentException("error test")); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java index c7339dd3fa..1391e2a357 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/MapUtilTest.java @@ -1,8 +1,8 @@ package com.dtstack.flinkx.util; import com.google.gson.internal.LinkedTreeMap; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; import java.util.HashMap; import java.util.Map; diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java index b38fc07328..b781529383 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/Md5UtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java index b373be8f19..7a68496791 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/RowUtilTest.java @@ -1,8 +1,8 @@ package com.dtstack.flinkx.util; import org.apache.flink.types.Row; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java new file mode 100644 index 0000000000..69da0176bd --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/SnowflakeIdWorkerTest.java @@ -0,0 +1,28 @@ +package com.dtstack.flinkx.util; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; + +/** + * @author jiangbo + * @date 2020/3/18 + */ +public class SnowflakeIdWorkerTest { + + SnowflakeIdWorker snowflakeIdWorker = new SnowflakeIdWorker(1L, 1L); + + @Test + public void testNextId() { + Set idSet = new HashSet<>(); + int i = 0; + while (i++ < 100) { + long result = snowflakeIdWorker.nextId(); + idSet.add(result); + } + + Assert.assertEquals(idSet.size(), 100); + } +} \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java index b072f9168a..bcff236c57 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/StringUtilTest.java @@ -1,9 +1,11 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; +import java.sql.Timestamp; import java.text.SimpleDateFormat; +import java.util.Date; /** * @author jiangbo @@ -66,4 +68,52 @@ public void testString2col() { result = StringUtil.string2col("xxx", "xxx", null); Assert.assertEquals(result, "xxx"); } + + @Test + public void testCol2string() { + String result = StringUtil.col2string(null, null); + Assert.assertEquals(result, ""); + + result = StringUtil.col2string("test", null); + Assert.assertEquals(result, "test"); + + result = StringUtil.col2string(Byte.valueOf("1"), "TINYINT"); + Assert.assertEquals(result, "1"); + + result = StringUtil.col2string(Short.valueOf("1"), "SMALLINT"); + Assert.assertEquals(result, "1"); + + result = StringUtil.col2string(123, "INTEGER"); + Assert.assertEquals(result, "123"); + + result = StringUtil.col2string(123L, "LONG"); + Assert.assertEquals(result, "123"); + + result = StringUtil.col2string(new Timestamp(1584510286187L), "LONG"); + Assert.assertEquals(result, "1584510286187"); + + result = StringUtil.col2string(123.123, "FLOAT"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string(123.123, "DOUBLE"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string(123.123, "DECIMAL"); + Assert.assertEquals(result, "123.123"); + + result = StringUtil.col2string("string", "STRING"); + Assert.assertEquals(result, "string"); + + result = StringUtil.col2string(new Timestamp(1584510286187L), "STRING"); + Assert.assertEquals(result, "2020-03-18 13:44:46"); + + result = StringUtil.col2string(true, "BOOLEAN"); + Assert.assertEquals(result, "true"); + + result = StringUtil.col2string(new Date(1584510286187L), "DATE"); + Assert.assertEquals(result, "2020-03-18"); + + result = StringUtil.col2string(new Date(1584510286187L), "DATETIME"); + Assert.assertEquals(result, "2020-03-18 13:44:46"); + } } \ No newline at end of file diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java index d22a8853d7..75e254d9a5 100644 --- a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ValueUtilTest.java @@ -1,7 +1,7 @@ package com.dtstack.flinkx.util; -import org.testng.Assert; -import org.testng.annotations.Test; +import org.junit.Assert; +import org.junit.Test; /** * @author jiangbo @@ -9,7 +9,7 @@ */ public class ValueUtilTest { - @Test(expectedExceptions = RuntimeException.class) + @Test public void testGetInt() { Integer result = ValueUtil.getInt(null); Assert.assertNull(result); @@ -20,6 +20,10 @@ public void testGetInt() { result = ValueUtil.getInt(new Long(100)); Assert.assertEquals(result, new Integer(100)); - ValueUtil.getInt(new Object()); + try { + ValueUtil.getInt(new Object()); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().contains("Unable to convert")); + } } } \ No newline at end of file diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 6a6f40a2c7..483fc23905 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -63,8 +63,12 @@ shade.db2reader.io.netty - com.google - shade.db2reader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 2ff75d22e0..b3d103b159 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -63,8 +63,12 @@ shade.db2writer.io.netty - com.google - shade.db2writer.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java index 3c4d2883dc..da6829b005 100644 --- a/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java +++ b/flinkx-dm/flinkx-dm-core/src/main/java/com/dtstack/flinkx/dm/DmDatabaseMeta.java @@ -47,7 +47,7 @@ protected String makeValues(List column) { public String quoteTable(String table) { table = table.replace("\"",""); String[] part = table.split("\\."); - if(part.length == 2) { + if(part.length == DB_TABLE_PART_SIZE) { table = getStartQuote() + part[0] + getEndQuote() + "." + getStartQuote() + part[1] + getEndQuote(); } else { table = getStartQuote() + table + getEndQuote(); diff --git a/flinkx-dm/flinkx-dm-reader/pom.xml b/flinkx-dm/flinkx-dm-reader/pom.xml index 15ce92929b..c2feddd9c8 100644 --- a/flinkx-dm/flinkx-dm-reader/pom.xml +++ b/flinkx-dm/flinkx-dm-reader/pom.xml @@ -63,8 +63,12 @@ shade.dmreader.io.netty - com.google - shade.dmreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java index 8a114752f1..67e70d499d 100644 --- a/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java +++ b/flinkx-dm/flinkx-dm-reader/src/main/java/com/dtstack/flinkx/dm/format/DmInputFormat.java @@ -25,8 +25,6 @@ import java.io.IOException; import java.sql.Timestamp; -import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; - /** * Date: 2020/03/18 * Company: www.dtstack.com @@ -49,7 +47,7 @@ public Row nextRecordInternal(Row row) throws IOException { || obj.getClass().getSimpleName().toUpperCase().contains("TIMESTAMP")) ) { obj = resultSet.getTimestamp(pos + 1); } - obj = clobToString(obj); + obj = DbUtil.clobToString(obj); } row.setField(pos, obj); diff --git a/flinkx-dm/flinkx-dm-writer/pom.xml b/flinkx-dm/flinkx-dm-writer/pom.xml index abe284fc04..768be50eae 100644 --- a/flinkx-dm/flinkx-dm-writer/pom.xml +++ b/flinkx-dm/flinkx-dm-writer/pom.xml @@ -63,8 +63,12 @@ shade.dmwriter.io.netty - com.google - shade.dmwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java b/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java index c3912a218c..9612294ec8 100644 --- a/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java +++ b/flinkx-dm/flinkx-dm-writer/src/main/java/com/dtstack/flinkx/dm/format/DmOutputFormat.java @@ -18,11 +18,16 @@ package com.dtstack.flinkx.dm.format; import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; import com.dtstack.flinkx.util.DateUtil; import org.apache.flink.types.Row; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.HashMap; @@ -61,7 +66,7 @@ protected List probeFullColumns(String table, Connection dbConn) throws String schema =null; String[] parts = table.split("\\."); - if(parts.length == 2) { + if(parts.length == BaseDatabaseMeta.DB_TABLE_PART_SIZE) { schema = parts[0].toUpperCase(); table = parts[1]; } @@ -76,25 +81,27 @@ protected List probeFullColumns(String table, Connection dbConn) throws @Override protected Map> probePrimaryKeys(String table, Connection dbConn) throws SQLException { - Map> map = new HashMap<>(); - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); - ResultSet rs = ps.executeQuery(); + Map> map = new HashMap<>(16); - while(rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if(!map.containsKey(indexName)) { - map.put(indexName,new ArrayList<>()); + try (PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); + ResultSet rs = ps.executeQuery()) { + while(rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if(!map.containsKey(indexName)) { + map.put(indexName,new ArrayList<>()); + } + map.get(indexName).add(rs.getString("COLUMN_NAME")); } - map.get(indexName).add(rs.getString("COLUMN_NAME")); - } - Map> retMap = new HashMap<>(); - for(Map.Entry> entry: map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if(v!=null && v.size() != 0 && v.get(0) != null) { - retMap.put(k, v); + + Map> retMap = new HashMap<>(16); + for(Map.Entry> entry: map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if(v!=null && v.size() != 0 && v.get(0) != null) { + retMap.put(k, v); + } } + return retMap; } - return retMap; } } diff --git a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java b/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java deleted file mode 100644 index ecb26a2f3e..0000000000 --- a/flinkx-emqx/flinkx-emqx-core/src/main/java/com/dtstack/flinkx/emqx/decoder/JsonDecoder.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.emqx.decoder; - -import org.codehaus.jackson.map.ObjectMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.Map; - -/** - * Date: 2020/02/12 - * Company: www.dtstack.com - * - * @author tudou - */ -public class JsonDecoder implements IDecode { - private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - - private static ObjectMapper objectMapper = new ObjectMapper(); - - private static final String KEY_MESSAGE = "message"; - - @Override - @SuppressWarnings("unchecked") - public Map decode(final String message) { - try { - Map event = objectMapper.readValue(message, Map.class); - if (!event.containsKey(KEY_MESSAGE)) { - event.put(KEY_MESSAGE, message); - } - return event; - } catch (Exception e) { - LOG.error(e.getMessage()); - return Collections.singletonMap(KEY_MESSAGE, message); - } - - } - -} diff --git a/flinkx-emqx/flinkx-emqx-reader/pom.xml b/flinkx-emqx/flinkx-emqx-reader/pom.xml index 0e106143fd..205e8a9c5c 100644 --- a/flinkx-emqx/flinkx-emqx-reader/pom.xml +++ b/flinkx-emqx/flinkx-emqx-reader/pom.xml @@ -53,6 +53,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java index 5534601f51..7e92921dcc 100644 --- a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java +++ b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java @@ -17,16 +17,21 @@ */ package com.dtstack.flinkx.emqx.format; -import com.dtstack.flinkx.emqx.decoder.IDecode; -import com.dtstack.flinkx.emqx.decoder.JsonDecoder; -import com.dtstack.flinkx.emqx.decoder.PlainDecoder; +import com.dtstack.flinkx.decoder.IDecode; +import com.dtstack.flinkx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.PlainDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.eclipse.paho.client.mqttv3.*; +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; +import org.eclipse.paho.client.mqttv3.MqttCallback; +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttConnectOptions; +import org.eclipse.paho.client.mqttv3.MqttException; +import org.eclipse.paho.client.mqttv3.MqttMessage; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java index 83d70d80d1..32121eec6a 100644 --- a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java +++ b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/reader/EmqxReader.java @@ -8,7 +8,13 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -import static com.dtstack.flinkx.emqx.EmqxConfigKeys.*; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_BROKER; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_CODEC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_IS_CLEAN_SESSION; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_QOS; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_TOPIC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_USERNAME; /** * Date: 2020/02/12 diff --git a/flinkx-emqx/flinkx-emqx-writer/pom.xml b/flinkx-emqx/flinkx-emqx-writer/pom.xml index a6117433f5..fcf861883a 100644 --- a/flinkx-emqx/flinkx-emqx-writer/pom.xml +++ b/flinkx-emqx/flinkx-emqx-writer/pom.xml @@ -53,6 +53,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java index 79dec0f172..f1e87baba3 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java @@ -17,7 +17,7 @@ */ package com.dtstack.flinkx.emqx.format; -import com.dtstack.flinkx.emqx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.JsonDecoder; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java index 0ba2a1a12d..52a311b76e 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java @@ -25,7 +25,12 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; -import static com.dtstack.flinkx.emqx.EmqxConfigKeys.*; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_BROKER; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_IS_CLEAN_SESSION; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_QOS; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_TOPIC; +import static com.dtstack.flinkx.emqx.EmqxConfigKeys.KEY_USERNAME; /** * Date: 2020/02/12 diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 227dc25d10..8ccef94149 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -53,6 +53,20 @@ + + + com.fasterxml.jackson.databind + shade.es.com.fasterxml.jackson.databind + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 282e2ac5f8..1c54a5e115 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -25,7 +25,11 @@ import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.elasticsearch.action.search.*; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.query.QueryBuilders; diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java index 33f9e85d21..57584faeb4 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.es.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import java.util.List; import java.util.Map; @@ -103,5 +104,9 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + if (format.batchSize > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取数量不能大于[200000]条"); + } } } diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java index da95c5b0ee..e57fa881a4 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsReader.java @@ -28,6 +28,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.HashMap; @@ -42,6 +44,8 @@ */ public class EsReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(EsReader.class); + private String address; private String username; private String password; @@ -87,7 +91,8 @@ public EsReader(DataTransferConfig config, StreamExecutionEnvironment env) { columnValue.add((String) sm.get("value")); columnName.add((String) sm.get("name")); } - System.out.println("init column finished"); + + LOG.info("init column finished"); } else if (!ConstantValue.STAR_SYMBOL.equals(columns.get(0)) || columns.size() != 1) { throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java b/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java deleted file mode 100644 index 06e304c46a..0000000000 --- a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReadDemo.java +++ /dev/null @@ -1,90 +0,0 @@ -package com.dtstack.flinkx.es.reader.test; - - -import com.google.gson.Gson; -import org.apache.http.HttpHost; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.aggregations.AggregationBuilder; -import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.Aggregations; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import static org.elasticsearch.index.query.QueryBuilders.termQuery; - -public class EsReadDemo { - - public static void searchAll() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - SearchRequest searchRequest = new SearchRequest(); -// SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); -// searchSourceBuilder.query(QueryBuilders.matchAllQuery()); -// searchSourceBuilder().from(0); -// searchSourceBuilder().size(100); - - SearchResponse searchResponse = client.search(searchRequest); - SearchHits searchHits = searchResponse.getHits(); - - for(SearchHit searchHit : searchHits) { - Map source = searchHit.getSourceAsMap(); - System.out.println(source); - } - - - } - - public static void searchPart() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); - - sourceBuilder.from(0); - sourceBuilder.size(100); - -// QueryBuilder qb2 = QueryBuilders.wrapperQuery("{\"match_all\": {}}"); - Map map = new HashMap<>(); - Map match = new HashMap<>(); - map.put("match",match); - match.put("col2", "hallo"); - Gson gson = new Gson(); - - //QueryBuilder qb2 = QueryBuilders.wrapperQuery("{\"match\": {\"col2\":\"hallo\"}}"); - QueryBuilder qb2 = QueryBuilders.wrapperQuery(gson.toJson(map)); - sourceBuilder.query(qb2); - searchRequest.source(sourceBuilder); - - SearchResponse searchResponse = client.search(searchRequest); - System.out.println(searchResponse); - SearchHits searchHits = searchResponse.getHits(); - for(SearchHit searchHit : searchHits.getHits()) { - System.out.println(searchHit.docId() + ":" + searchHit.getSourceAsMap()); - } - long total = searchHits.getTotalHits(); - System.out.println("total: " + total); - - client.close(); - } - - - public static void main(String[] args) throws IOException { - //searchAll(); - searchPart(); - //searchAll(); - } - -} diff --git a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java b/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java deleted file mode 100644 index bf98581615..0000000000 --- a/flinkx-es/flinkx-es-reader/src/test/java/com/dtstack/flinkx/es/reader/test/EsReaderTest.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.dtstack.flinkx.es.reader.test; - -import com.google.gson.Gson; - -import java.util.HashMap; -import java.util.Map; - -/** - * Created by softfly on 18/2/8. - */ -public class EsReaderTest { - public static void main(String[] args) { - Gson gson = new Gson(); - Map map = new HashMap<>(); - map.put("xxx", 111); - map.put("yyyy", "fff"); - String json = gson.toJson(map); - System.out.println(json); - } -} diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 1cdfab73fb..8e3e15a822 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -54,6 +54,20 @@ + + + com.fasterxml.jackson.databind + shade.es.com.fasterxml.jackson.databind + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java b/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java deleted file mode 100644 index 503c09fa7e..0000000000 --- a/flinkx-es/flinkx-es-writer/src/test/java/com/dtstack/flinkx/es/writer/test/EsDemo.java +++ /dev/null @@ -1,146 +0,0 @@ -package com.dtstack.flinkx.es.writer.test; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.action.update.UpdateResponse; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.search.builder.SearchSourceBuilder; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -public class EsDemo { - - public static void test1() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("rdos1", 9200, "http"), - new HttpHost("rdos2", 9200, "http"))); - - IndexRequest request = new IndexRequest( - "nani222", - "doc222", - "id2"); - - String jsonString = "{" + - "\"user\":\"user2\"," + - "\"postDate\":\"2014-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; - - - request.source(jsonString, XContentType.JSON); - IndexResponse response = client.index(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test3() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - - IndexRequest request = new IndexRequest( - "nani", - "doc"); - -// String jsonString = "{" + -// "\"user\":\"xxxx\"," + -// "\"postDate\":\"2013-01-30\"," + -// "\"message\":\"trying out Elasticsearch\"" + -// "}"; - Map jsonMap = new HashMap<>(); - jsonMap.put("xxx", "asfdasdf"); - jsonMap.put("zzz", "asdfsadf"); - request.source(jsonMap); - IndexResponse response = client.index(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test2() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("localhost", 9200, "http"), - new HttpHost("localhost", 9201, "http"))); - - UpdateRequest request = new UpdateRequest( - "nani250", - "doc", - "2"); - - String jsonString = "{" + - "\"user\":\"kimchy\"," + - "\"postDate\":\"2013-01-30\"," + - "\"message\":\"trying out Elasticsearch\"" + - "}"; - - request.doc(jsonString, XContentType.JSON); - UpdateResponse response = client.update(request); - System.out.println(response.getResult()); - client.close(); - } - - public static void test4() throws IOException { - - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchSourceBuilder.query(QueryBuilders.matchAllQuery()); - - SearchResponse searchResponse = client.search(searchRequest); - System.out.println(searchResponse.getTotalShards()); - } - - public static void test5() throws Exception { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - BulkRequest bulkRequest = new BulkRequest(); - - IndexRequest request = new IndexRequest("nani", "doc1"); - Map jsonMap = new HashMap<>(); - jsonMap.put("xxx", "8888"); - jsonMap.put("yyy", "9999"); - - bulkRequest.add(request.source(jsonMap)); - // bulkRequest.setRefreshPolicy(null); - // WriteRequest.RefreshPolicy; - - BulkResponse bulkResponse = client.bulk(bulkRequest); - - System.out.println(bulkResponse); - } - - public static void test6() throws IOException { - RestHighLevelClient client = new RestHighLevelClient( - RestClient.builder( - new HttpHost("study", 9200, "http"), - new HttpHost("study", 9201, "http"))); - SearchRequest searchRequest = new SearchRequest(); - SearchResponse resp = client.search(searchRequest); - resp.getAggregations(); - } - - public static void main(String[] args) throws Exception { - test1(); - } - - -} diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java index fc58ca41e7..afc45b081a 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java @@ -59,7 +59,7 @@ public class FtpConfig implements Serializable { public int timeout = FtpConfigConstants.DEFAULT_TIMEOUT; - public long maxFileSize = 1024 * 1024 * 1024; + public long maxFileSize = 1024 * 1024 * 1024L; public String getUsername() { return username; diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index d201625068..4b99e46257 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -89,23 +89,14 @@ public void loginFtpServer(FtpConfig ftpConfig) { } @Override - public void logoutFtpServer() { + public void logoutFtpServer() throws IOException{ if (ftpClient.isConnected()) { try { ftpClient.logout(); - } catch (IOException e) { - LOG.error(DISCONNECT_FAIL_MESSAGE); - throw new RuntimeException(e); - }finally { + } finally { if(ftpClient.isConnected()){ - try { - ftpClient.disconnect(); - } catch (IOException e) { - LOG.error(DISCONNECT_FAIL_MESSAGE); - throw new RuntimeException(e); - } + ftpClient.disconnect(); } - } } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java index cc83e0343a..8c775f062d 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandlerFactory.java @@ -19,6 +19,8 @@ package com.dtstack.flinkx.ftp; +import org.apache.commons.lang3.StringUtils; + /** * @author jiangbo * @date 2019/11/21 @@ -28,8 +30,8 @@ public class FtpHandlerFactory { public static IFtpHandler createFtpHandler(String protocolStr){ IFtpHandler ftpHandler; - EProtocol protocol = EProtocol.getByName(protocolStr); - if(EProtocol.SFTP.equals(protocol)) { + Protocol protocol = Protocol.getByName(protocolStr); + if(Protocol.SFTP.equals(protocol)) { ftpHandler = new SftpHandler(); } else { ftpHandler = new FtpHandler(); @@ -37,4 +39,16 @@ public static IFtpHandler createFtpHandler(String protocolStr){ return ftpHandler; } + + enum Protocol{ + FTP, SFTP; + + public static Protocol getByName(String name) { + if (StringUtils.isEmpty(name)) { + return SFTP; + } + + return Protocol.valueOf(name.toUpperCase()); + } + } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java index c8da4124b1..16faf8e6d8 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.ftp; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; @@ -40,8 +41,9 @@ public interface IFtpHandler { /** * 登出服务器 + * @throws IOException logout error */ - void logoutFtpServer(); + void logoutFtpServer() throws IOException; /** * 判断给定的目录是否存在 diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 1771055c60..42ae5c1123 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -71,6 +71,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index 6f0f4471d3..3b3b62013c 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -19,7 +19,12 @@ package com.dtstack.flinkx.ftp.reader; import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.ftp.*; +import com.dtstack.flinkx.ftp.EProtocol; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandler; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; +import com.dtstack.flinkx.ftp.IFtpHandler; +import com.dtstack.flinkx.ftp.SftpHandler; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; @@ -54,13 +59,7 @@ public class FtpInputFormat extends BaseRichInputFormat { public void openInputFormat() throws IOException { super.openInputFormat(); - if(EProtocol.SFTP.name().equalsIgnoreCase(ftpConfig.getProtocol())) { - ftpHandler = new SftpHandler(); - } else if(EProtocol.FTP.name().equalsIgnoreCase(ftpConfig.getProtocol())){ - ftpHandler = new FtpHandler(); - } else { - throw new RuntimeException("协议名称错误:" + ftpConfig.getProtocol()); - } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); ftpHandler.loginFtpServer(ftpConfig); } @@ -157,5 +156,4 @@ public void closeInternal() throws IOException { ftpHandler.logoutFtpServer(); } } - } diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java index 411389206d..c889b249cf 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpReader.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpConfigConstants; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; @@ -30,8 +31,6 @@ import java.util.List; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.*; - /** * The reader plugin of Ftp * @@ -57,7 +56,7 @@ public FtpReader(DataTransferConfig config, StreamExecutionEnvironment env) { ftpConfig.setDefaultPort(); } - if(!DEFAULT_FIELD_DELIMITER.equals(ftpConfig.getFieldDelimiter())){ + if(!FtpConfigConstants.DEFAULT_FIELD_DELIMITER.equals(ftpConfig.getFieldDelimiter())){ String fieldDelimiter = StringUtil.convertRegularExpr(ftpConfig.getFieldDelimiter()); ftpConfig.setFieldDelimiter(fieldDelimiter); } diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java index a19e566fa6..34755a4109 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java @@ -20,8 +20,13 @@ import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.ftp.FtpHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.util.Iterator; /** @@ -32,6 +37,8 @@ */ public class FtpSeqBufferedReader { + private static Logger LOG = LoggerFactory.getLogger(FtpSeqBufferedReader.class); + private IFtpHandler ftpHandler; private Iterator iter; @@ -76,7 +83,8 @@ private void nextStream() throws IOException{ br = new BufferedReader(new InputStreamReader(in, charsetName)); for (int i = 0; i < fromLine; i++) { - br.readLine(); + String skipLine = br.readLine(); + LOG.info("Skip line:{}", skipLine); } } else { br = null; diff --git a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java b/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java deleted file mode 100644 index ed9177c3b7..0000000000 --- a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/FtpServerDemo.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.dtstack.flinkx.ftp.reader; - -import org.apache.commons.net.ftp.FTPClient; -import org.apache.commons.net.ftp.FTPFile; - -import java.io.IOException; -import java.io.InputStream; - -/** - * Created by softfly on 17/11/22. - */ -public class FtpServerDemo { - - public static void main(String[] args) throws IOException { - FTPClient ftp = new FTPClient(); - String username = "test"; - String password = "qbI#5pNd"; - ftp.connect("node02", 21); - ftp.login(username, password); - FTPFile[] ftpFiles = ftp.listFiles(); - for(FTPFile ftpFile : ftpFiles) { - System.out.println(ftpFile.getName()); - } - - - String[] xxx = ftp.listNames(); - - InputStream is1 = ftp.retrieveFileStream("hyf/ttt"); - ftp.getReply(); - InputStream is2 = ftp.retrieveFileStream("xxx"); - ftp.remoteRetrieve("/hyf/ttt"); - ftp.getReply(); - - ftp.changeWorkingDirectory("/hyf"); - System.out.println(ftp.printWorkingDirectory()); - - ftp.logout(); - - } - -} diff --git a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java b/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java deleted file mode 100644 index 6defecb105..0000000000 --- a/flinkx-ftp/flinkx-ftp-reader/src/test/java/com/dtstack/flinkx/ftp/reader/SftpServerDemo.java +++ /dev/null @@ -1,42 +0,0 @@ -package com.dtstack.flinkx.ftp.reader; - -import com.jcraft.jsch.*; - -import java.util.Properties; -import java.util.Vector; - -/** - * Created by softfly on 17/11/23. - */ -public class SftpServerDemo { - - public static void main(String[] args) throws JSchException, SftpException { - JSch jsch = new JSch(); - - Session session = jsch.getSession("mysftp", "node02"); - session.setPassword("oh1986mygod"); - session.setPort(22); - //session.setTimeout(10); - Properties config = new Properties(); - config.put("StrictHostKeyChecking", "no"); - - session.setConfig(config); - session.connect(); - - ChannelSftp channelSftp = (ChannelSftp) session.openChannel("sftp"); // 打开SFTP通道 - channelSftp.connect(); // 建立SFTP通道的连接 - - Vector vector = channelSftp.ls("/"); - - for(int i = 0; i < vector.size(); ++i) { - ChannelSftp.LsEntry le = (ChannelSftp.LsEntry) vector.get(i); - System.out.println(le.getFilename() ); - System.out.println(le.getLongname()); - } - - - //session.disconnect(); - - } - -} diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 61d8ffaede..5c5b2bb6f7 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -72,6 +72,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java index aeddb31bcb..4d66e893b1 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java @@ -19,22 +19,27 @@ package com.dtstack.flinkx.ftp.writer; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.ftp.*; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; +import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; +import java.io.BufferedWriter; import java.io.IOException; -import java.io.OutputStream; -import java.util.Arrays; +import java.io.OutputStreamWriter; +import java.io.UnsupportedEncodingException; +import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.SFTP_PROTOCOL; - /** * The OutputFormat Implementation which reads data from ftp servers. * @@ -54,21 +59,18 @@ public class FtpOutputFormat extends BaseFileOutputFormat { private transient IFtpHandler ftpHandler; - private transient OutputStream os; + private static final int FILE_NAME_PART_SIZE = 3; private static final String DOT = "."; private static final String FILE_SUFFIX = ".csv"; private static final String OVERWRITE_MODE = "overwrite"; + private transient BufferedWriter writer; @Override protected void openSource() throws IOException { - if(SFTP_PROTOCOL.equalsIgnoreCase(ftpConfig.getProtocol())) { - ftpHandler = new SftpHandler(); - } else { - ftpHandler = new FtpHandler(); - } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); ftpHandler.loginFtpServer(ftpConfig); } @@ -106,7 +108,7 @@ public boolean test(String file) { } String[] splits = fileName.split("\\."); - if (splits.length == 3) { + if (splits.length == FILE_NAME_PART_SIZE) { return Integer.parseInt(splits[2]) <= fileIndex; } @@ -125,11 +127,16 @@ public boolean test(String file) { protected void nextBlock(){ super.nextBlock(); - if (os != null){ + if (writer != null){ return; } - - os = ftpHandler.getOutputStream(tmpPath + SP + currentBlockFileName); + String path = tmpPath + SP + currentBlockFileName; + try { + writer = new BufferedWriter(new OutputStreamWriter(ftpHandler.getOutputStream(path), ftpConfig.getEncoding())); + } catch (UnsupportedEncodingException e) { + LOG.error("exception when create BufferedWriter, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(e); + } blockIndex++; } @@ -156,29 +163,42 @@ public void moveTemporaryDataBlockFileToDirectory(){ @Override public void writeSingleRecordToFile(Row row) throws WriteRecordException { - if(os == null){ - nextBlock(); - } - - String line = StringUtil.row2string(row, columnTypes, ftpConfig.getFieldDelimiter(), columnNames); try { - byte[] bytes = line.getBytes(ftpConfig.getEncoding()); - this.os.write(bytes); - this.os.write(NEWLINE); - this.os.flush(); + if(writer == null){ + nextBlock(); + } + + String line = StringUtil.row2string(row, columnTypes, ftpConfig.getFieldDelimiter()); + this.writer.write(line); + this.writer.write(NEWLINE); if(restoreConfig.isRestore()){ lastRow = row; rowsOfCurrentBlock++; } - } catch(Exception ex) { - throw new WriteRecordException(ex.getMessage(), ex); + } catch(Exception e) { + LOG.error("error happened when write single record to file, row = {}, columnTypes = {}, e = {}", row, GsonUtil.GSON.toJson(columnTypes), ExceptionUtil.getErrorMessage(e)); + throw new WriteRecordException(e.getMessage(), e); } } @Override - protected void createFinishedTag() throws IOException { - LOG.info("Subtask [{}] finished, create dir {}", taskNumber, finishedPath); + protected void createFinishedTag() { + LOG.info("SubTask [{}] finished, create dir {}", taskNumber, finishedPath); + String path = outputFilePath + SP + FINISHED_SUBDIR; + if(taskNumber == 0){ + ftpHandler.mkDirRecursive(path); + } + final int maxRetryTime = 15; + int i = 0; + try { + while(!(ftpHandler.isDirExist(path) || i > maxRetryTime)){ + i++; + TimeUnit.MILLISECONDS.sleep(10); + } + }catch (Exception e){ + LOG.error("exception when createFinishedTag, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); + } ftpHandler.mkDirRecursive(finishedPath); } @@ -238,7 +258,7 @@ protected void waitForAllTasksToFinish(){ if (i == maxRetryTime) { ftpHandler.deleteAllFilesInDir(finishedPath, null); - throw new RuntimeException("timeout when gathering finish tags for each subtasks"); + throw new RuntimeException("timeout when gathering finish tags for each subTasks"); } } @@ -246,7 +266,7 @@ protected void waitForAllTasksToFinish(){ protected void coverageData(){ boolean cleanPath = restoreConfig.isRestore() && OVERWRITE_MODE.equalsIgnoreCase(ftpConfig.getWriteMode()) && !SP.equals(ftpConfig.getPath()); if(cleanPath){ - ftpHandler.deleteAllFilesInDir(ftpConfig.getPath(), Arrays.asList(tmpPath)); + ftpHandler.deleteAllFilesInDir(ftpConfig.getPath(), Collections.singletonList(tmpPath)); } } @@ -286,15 +306,15 @@ protected void moveAllTemporaryDataFileToDirectory() throws IOException { @Override protected void closeSource() throws IOException { - if (os != null){ - os.flush(); - os.close(); - os = null; + if (writer != null){ + writer.flush(); + writer.close(); + writer = null; } } @Override - protected void clearTemporaryDataFiles() throws IOException { + protected void clearTemporaryDataFiles() { ftpHandler.deleteAllFilesInDir(tmpPath, null); LOG.info("Delete .data dir:{}", tmpPath); @@ -307,6 +327,12 @@ public void flushDataInternal() throws IOException { closeSource(); } + @Override + public void closeInternal() throws IOException { + closeSource(); + super.closeInternal(); + } + @Override public float getDeviation() { return 1.0F; @@ -316,4 +342,9 @@ public float getDeviation() { protected String getExtension() { return ".csv"; } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("FtpWriter"); + } } diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java index dee19b9e1c..dc7bcae4f9 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormatBuilder.java @@ -19,9 +19,7 @@ package com.dtstack.flinkx.ftp.writer; import com.dtstack.flinkx.ftp.FtpConfig; -import com.dtstack.flinkx.ftp.FtpConfigConstants; import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; -import org.apache.commons.lang.StringUtils; import java.util.List; /** @@ -53,7 +51,7 @@ public void setFtpConfig(FtpConfig ftpConfig){ @Override protected void checkFormat() { - + notSupportBatchWrite("FtpWriter"); } } diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java index edfb666f4a..9588e3a3c0 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpWriter.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.ftp.FtpConfigConstants.*; +import static com.dtstack.flinkx.ftp.FtpConfigConstants.DEFAULT_FIELD_DELIMITER; /** * The Writer Plugin of Ftp diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index 3082049696..9a7903f7fc 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -62,8 +62,12 @@ shade.gbasereader.io.netty - com.google - shade.gbasereader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index 7361843030..eb1b86a8a4 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.gbase.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.lang3.StringUtils; @@ -47,7 +48,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -61,6 +64,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { fetchSize = Integer.MIN_VALUE; querySql = buildQuerySql(inputSplit); + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index da6f21b4a2..d28e0af276 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -62,8 +62,12 @@ shade.gbasewriter.io.netty - com.google - shade.gbasewriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-greenplum/flinkx-greenplum-core/pom.xml b/flinkx-greenplum/flinkx-greenplum-core/pom.xml new file mode 100644 index 0000000000..3a79da783d --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/pom.xml @@ -0,0 +1,39 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-core + + + com.dtstack.flinkx + flinkx-postgresql-core + 1.6 + + + com.pivotal + greenplum-jdbc + 5.1.4 + + + org.postgresql + postgresql + 42.2.2 + + + + + + HandChina RDC + HandChina RDC + http://nexus.saas.hand-china.com/content/repositories/rdc/ + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java new file mode 100644 index 0000000000..95dcb1b887 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMeta.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum; + +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.postgresql.PostgresqlDatabaseMeta; + +import java.util.List; +import java.util.Map; + +/** + * The class of Greenplum database prototype + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumDatabaseMeta extends PostgresqlDatabaseMeta { + + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + throw new UnsupportedOperationException("Greenplum not support update mode"); + } + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.Greenplum; + } + + @Override + public String getDriverClass() { + return "com.pivotal.jdbc.GreenplumDriver"; + } +} diff --git a/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java new file mode 100644 index 0000000000..cb811c6ef0 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-core/src/main/java/com/dtstack/flinkx/greenplum/GreenplumDatabaseMetaInsert.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum; + +/** + * when use insert mode, use org.postgresql.Driver + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumDatabaseMetaInsert extends GreenplumDatabaseMeta{ + @Override + public String getDriverClass() { + return "org.postgresql.Driver"; + } +} diff --git a/flinkx-greenplum/flinkx-greenplum-reader/pom.xml b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml new file mode 100644 index 0000000000..0fb695ce58 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml @@ -0,0 +1,112 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-reader + + + + com.dtstack.flinkx + flinkx-greenplum-core + 1.6 + + + com.dtstack.flinkx + flinkx-postgresql-reader + 1.6 + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.greenplumreader.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java new file mode 100644 index 0000000000..c1fba4eb0b --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumInputFormat.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.format; + +import com.dtstack.flinkx.postgresql.format.PostgresqlInputFormat; + +/**Simple inheritance of PostgresqlInputFormat + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumInputFormat extends PostgresqlInputFormat { +} diff --git a/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java new file mode 100644 index 0000000000..09f547469f --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-reader/src/main/java/com/dtstack/flinkx/greenplum/reader/GreenplumReader.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.greenplum.format.GreenplumInputFormat; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; +import com.dtstack.flinkx.postgresql.reader.PostgresqlQuerySqlBuilder; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.datareader.QuerySqlBuilder; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import com.dtstack.flinkx.greenplum.GreenplumDatabaseMeta; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +/** + * The reader plugin for Greenplum database + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumReader extends JdbcDataReader { + public GreenplumReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + setDatabaseInterface(new GreenplumDatabaseMeta()); + setTypeConverterInterface(new PostgresqlTypeConverter()); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return new JdbcInputFormatBuilder(new GreenplumInputFormat()); + } + + @Override + public DataStream readData() { + JdbcInputFormatBuilder builder = new JdbcInputFormatBuilder(new GreenplumInputFormat()); + builder.setDriverName(databaseInterface.getDriverClass()); + builder.setDbUrl(dbUrl); + builder.setUsername(username); + builder.setPassword(password); + builder.setBytes(bytes); + builder.setMonitorUrls(monitorUrls); + builder.setTable(table); + builder.setDatabaseInterface(databaseInterface); + builder.setTypeConverter(typeConverter); + builder.setMetaColumn(metaColumns); + builder.setFetchSize(fetchSize == 0 ? databaseInterface.getFetchSize() : fetchSize); + builder.setQueryTimeOut(queryTimeOut == 0 ? databaseInterface.getQueryTimeout() : queryTimeOut); + builder.setIncrementConfig(incrementConfig); + builder.setSplitKey(splitKey); + builder.setNumPartitions(numPartitions); + builder.setCustomSql(customSql); + builder.setRestoreConfig(restoreConfig); + builder.setHadoopConfig(hadoopConfig); + builder.setTestConfig(testConfig); + + QuerySqlBuilder sqlBuilder = new PostgresqlQuerySqlBuilder(this); + builder.setQuery(sqlBuilder.buildSql()); + + BaseRichInputFormat format = builder.finish(); + return createInput(format, (databaseInterface.getDatabaseType() + "reader").toLowerCase()); + } + +} diff --git a/flinkx-greenplum/flinkx-greenplum-writer/pom.xml b/flinkx-greenplum/flinkx-greenplum-writer/pom.xml new file mode 100644 index 0000000000..db6738d9e5 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/pom.xml @@ -0,0 +1,110 @@ + + + + flinkx-greenplum + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum-writer + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + com.dtstack.flinkx + flinkx-greenplum-core + 1.6 + + + com.dtstack.flinkx + flinkx-postgresql-writer + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.greenplumwriter.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java new file mode 100644 index 0000000000..a1d02138d2 --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/format/GreenplumOutputFormat.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.format; + +import com.dtstack.flinkx.postgresql.format.PostgresqlOutputFormat; + +/**Simple inheritance of PostgresqlOutputFormat + * + * Date: 2020/6/4 + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumOutputFormat extends PostgresqlOutputFormat { +} diff --git a/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java new file mode 100644 index 0000000000..8d3267968b --- /dev/null +++ b/flinkx-greenplum/flinkx-greenplum-writer/src/main/java/com/dtstack/flinkx/greenplum/writer/GreenplumWriter.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.greenplum.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.greenplum.format.GreenplumOutputFormat; +import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.greenplum.GreenplumDatabaseMetaInsert; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +/** + * The writer plugin for Greenplum database + * + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class GreenplumWriter extends JdbcDataWriter { + + public static final String INSERT_SQL_MODE_TYPE = "copy"; + public static final String DATABASE_NAME = ";DatabaseName="; + public static final String JDBC_POSTGRESQL_PREFIX = "jdbc:postgresql"; + public static final String JDBC_GREENPLUM_PREFIX = "jdbc:pivotal:greenplum"; + + public GreenplumWriter(DataTransferConfig config) { + super(config); + //统一固定为copy模式 + insertSqlMode = INSERT_SQL_MODE_TYPE; + dbUrl = changeToPostgresqlUrl(); + setDatabaseInterface(new GreenplumDatabaseMetaInsert()); + setTypeConverterInterface(new PostgresqlTypeConverter()); + } + + String changeToPostgresqlUrl(){ + dbUrl = StringUtils.replaceOnce(dbUrl, JDBC_GREENPLUM_PREFIX, JDBC_POSTGRESQL_PREFIX); + dbUrl = StringUtils.replaceOnce(dbUrl, DATABASE_NAME, "/"); + return dbUrl; + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + GreenplumOutputFormat greenplumOutputFormat = new GreenplumOutputFormat(); + JdbcOutputFormatBuilder builder = new JdbcOutputFormatBuilder(greenplumOutputFormat); + builder.setDriverName(databaseInterface.getDriverClass()); + builder.setDbUrl(dbUrl); + builder.setUsername(username); + builder.setPassword(password); + builder.setBatchInterval(batchSize); + builder.setMonitorUrls(monitorUrls); + builder.setPreSql(preSql); + builder.setPostSql(postSql); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setDatabaseInterface(databaseInterface); + builder.setMode(mode); + builder.setTable(table); + builder.setColumn(column); + builder.setFullColumn(fullColumn); + builder.setUpdateKey(updateKey); + builder.setTypeConverter(typeConverter); + builder.setRestoreConfig(restoreConfig); + builder.setInsertSqlMode(insertSqlMode); + + DataStreamSink dataStreamSink = createOutput(dataSet, builder.finish()); + String sinkName = (databaseInterface.getDatabaseType() + "writer").toLowerCase(); + dataStreamSink.name(sinkName); + return dataStreamSink; + } +} diff --git a/flinkx-greenplum/pom.xml b/flinkx-greenplum/pom.xml new file mode 100644 index 0000000000..5c3fb85de9 --- /dev/null +++ b/flinkx-greenplum/pom.xml @@ -0,0 +1,36 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-greenplum + pom + + flinkx-greenplum-reader + flinkx-greenplum-core + flinkx-greenplum-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java index d83d83b3ee..d7f0d61ba6 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java @@ -44,7 +44,7 @@ public class HbaseConfigConstants { public static final String DEFAULT_NULL_MODE = "skip"; - public static final long DEFAULT_WRITE_BUFFER_SIZE = 8 * 1024 * 1024; + public static final long DEFAULT_WRITE_BUFFER_SIZE = 8 * 1024 * 1024L; public static final boolean DEFAULT_WAL_FLAG = false; diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java index 0bce2c0c69..3c88f828f5 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java @@ -27,15 +27,17 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.BufferedMutator; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.security.PrivilegedAction; -import java.util.Arrays; -import java.util.List; import java.util.Map; /** @@ -51,21 +53,9 @@ public class HbaseHelper { private final static String AUTHENTICATION_TYPE = "Kerberos"; private final static String KEY_HBASE_SECURITY_AUTHENTICATION = "hbase.security.authentication"; private final static String KEY_HBASE_SECURITY_AUTHORIZATION = "hbase.security.authorization"; - private final static String KEY_HBASE_MASTER_KERBEROS_PRINCIPAL = "hbase.master.kerberos.principal"; - private final static String KEY_HBASE_MASTER_KEYTAB_FILE = "hbase.master.keytab.file"; - private final static String KEY_HBASE_REGIONSERVER_KEYTAB_FILE = "hbase.regionserver.keytab.file"; - private final static String KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL = "hbase.regionserver.kerberos.principal"; - - private static List KEYS_KERBEROS_REQUIRED = Arrays.asList( - KEY_HBASE_SECURITY_AUTHENTICATION, - KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, - KEY_HBASE_MASTER_KEYTAB_FILE, - KEY_HBASE_REGIONSERVER_KEYTAB_FILE, - KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL - ); public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map hbaseConfigMap) { - Validate.isTrue(hbaseConfigMap != null && hbaseConfigMap.size() !=0, "hbaseConfig不能为空Map结构!"); + Validate.isTrue(MapUtils.isEmpty(hbaseConfigMap), "hbaseConfig不能为空Map结构!"); if(openKerberos(hbaseConfigMap)){ return getConnectionWithKerberos(hbaseConfigMap); @@ -81,43 +71,43 @@ public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map hbaseConfigMap){ - for (String key : KEYS_KERBEROS_REQUIRED) { - if(StringUtils.isEmpty(MapUtils.getString(hbaseConfigMap, key))){ - throw new IllegalArgumentException(String.format("Must provide [%s] when authentication is Kerberos", key)); - } + try { + UserGroupInformation ugi = getUgi(hbaseConfigMap); + return ugi.doAs(new PrivilegedAction() { + @Override + public Connection run() { + try { + Configuration hConfiguration = getConfig(hbaseConfigMap); + return ConnectionFactory.createConnection(hConfiguration); + } catch (IOException e) { + LOG.error("Get connection fail with config:{}", hbaseConfigMap); + throw new RuntimeException(e); + } + } + }); + } catch (Exception e){ + throw new RuntimeException("Login kerberos error", e); } + } + public static UserGroupInformation getUgi(Map hbaseConfigMap) throws IOException{ String keytabFileName = KerberosUtil.getPrincipalFileName(hbaseConfigMap); keytabFileName = KerberosUtil.loadFile(hbaseConfigMap, keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); + String principal = KerberosUtil.getPrincipal(hbaseConfigMap, keytabFileName); KerberosUtil.loadKrb5Conf(hbaseConfigMap); Configuration conf = FileSystemUtil.getConfiguration(hbaseConfigMap, null); - UserGroupInformation ugi; - try { - ugi = KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); - } catch (Exception e){ - throw new RuntimeException("Login kerberos error", e); - } - - return ugi.doAs(new PrivilegedAction() { - @Override - public Connection run() { - try { - Configuration hConfiguration = getConfig(hbaseConfigMap); - return ConnectionFactory.createConnection(hConfiguration); - } catch (IOException e) { - LOG.error("Get connection fail with config:{}", hbaseConfigMap); - throw new RuntimeException(e); - } - } - }); + return KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); } public static Configuration getConfig(Map hbaseConfigMap){ Configuration hConfiguration = HBaseConfiguration.create(); + if (MapUtils.isEmpty(hbaseConfigMap)) { + return hConfiguration; + } + for (Map.Entry entry : hbaseConfigMap.entrySet()) { if(entry.getValue() != null && !(entry.getValue() instanceof Map)){ hConfiguration.set(entry.getKey(), entry.getValue().toString()); diff --git a/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java b/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java deleted file mode 100644 index c1a67b5750..0000000000 --- a/flinkx-hbase/flinkx-hbase-core/src/test/java/com/dtstack/flinkx/hbase/test/HbaseHelperTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.hbase.test; - -import com.dtstack.flinkx.hbase.HbaseHelper; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; - -import java.util.HashMap; -import java.util.Map; - -/** - * @author jiangbo - * @date 2019/8/29 - */ -public class HbaseHelperTest { - - public static void main(String[] args) throws Exception{ - Map sftpConf = new HashMap<>(); - sftpConf.put("host", "172.16.10.79"); - sftpConf.put("port", "22"); - sftpConf.put("username", "root"); - sftpConf.put("password", "abc123"); - - Map hbaseConfig = new HashMap<>(); - hbaseConfig.put("hbase.security.authorization", "true"); - hbaseConfig.put("hbase.security.authentication", "kerberos"); - hbaseConfig.put("hbase.master.kerberos.principal", "hbase/cdh01@HADOOP.COM"); - hbaseConfig.put("hbase.master.keytab.file", "D:\\cdh_cluster\\cdh2\\hbase.keytab"); - hbaseConfig.put("hbase.regionserver.kerberos.principal", "hbase/cdh01@HADOOP.COM"); - hbaseConfig.put("hbase.regionserver.keytab.file", "D:\\cdh_cluster\\cdh2\\hbase.keytab"); - hbaseConfig.put("java.security.krb5.conf", "D:\\cdh_cluster\\cdh2\\krb5.conf"); - hbaseConfig.put("useLocalFile", "true"); -// hbaseConfig.put("sftpConf", sftpConf); -// hbaseConfig.put("remoteDir", "/home/sftp/keytab/jiangbo"); - -// hbaseConfig.put("hbase.zookeeper.quorum", "cdh01:2181,cdh02:2181,cdh03:2181"); - hbaseConfig.put("hbase.zookeeper.quorum", "172.16.10.201:2181"); - hbaseConfig.put("hbase.rpc.timeout", "60000"); - hbaseConfig.put("ipc.socket.timeout", "20000"); - hbaseConfig.put("hbase.client.retries.number", "3"); - hbaseConfig.put("hbase.client.pause", "100"); - hbaseConfig.put("zookeeper.recovery.retry", "3"); - - Connection connection = HbaseHelper.getHbaseConnection(hbaseConfig); - Table table = connection.getTable(TableName.valueOf("tb1")); - - ResultScanner rs = table.getScanner(new Scan()); - Result result = rs.next(); - if(result != null){ - System.out.println(result.getRow()); - } - - HbaseHelper.getRegionLocator(connection, "tb1"); - - connection.close(); - } -} diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index f77a9613ca..5cdc499262 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -65,6 +65,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 495a89c4e6..f17383ca87 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -29,16 +29,24 @@ import org.apache.flink.types.Row; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import java.io.IOException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; import java.util.Map; +import com.google.common.collect.Maps; +import org.apache.hadoop.security.UserGroupInformation; /** @@ -87,7 +95,17 @@ public void openInputFormat() throws IOException { @Override public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { try (Connection connection = HbaseHelper.getHbaseConnection(hbaseConfig)) { - return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + if(HbaseHelper.openKerberos(hbaseConfig)) { + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + return ugi.doAs(new PrivilegedAction() { + @Override + public HbaseInputSplit[] run() { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } + }); + } else { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } } } @@ -132,9 +150,10 @@ private List doSplit(byte[] startRowkeyByte, // 当前的region为最后一个region // 如果最后一个region的start Key大于用户指定的userEndKey,则最后一个region,应该不包含在内 // 注意如果用户指定userEndKey为"",则此判断应该不成立。userEndKey为""表示取得最大的region - if (Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0 + boolean isSkip = Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0 && (endRowkeyByte.length != 0 && (Bytes.compareTo( - regionStartKey, endRowkeyByte) > 0))) { + regionStartKey, endRowkeyByte) > 0)); + if (isSkip) { continue; } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java index 04c5150990..b1d4186132 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java @@ -26,6 +26,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -39,6 +41,8 @@ */ public class HbaseReader extends BaseDataReader { + private static Logger LOG = LoggerFactory.getLogger(HbaseReader.class); + protected List columnName; protected List columnType; protected List columnValue; @@ -82,7 +86,8 @@ public HbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { columnValue.add((String) sm.get("value")); columnFormat.add((String) sm.get("format")); } - System.out.println("init column finished"); + + LOG.info("init column finished"); } else{ throw new IllegalArgumentException("column argument error"); } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java b/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java deleted file mode 100644 index 47f153621b..0000000000 --- a/flinkx-hbase/flinkx-hbase-reader/src/test/java/com/dtstack/flinkx/hbase/reader/HbaseSplitDemo.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.dtstack.flinkx.hbase.reader; - -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionLocator; -import java.io.IOException; - -/** - * Created by softfly on 17/7/25. - */ -public class HbaseSplitDemo { - - private static void split() { - - } - - public static void main(String[] args) throws IOException { - - org.apache.hadoop.conf.Configuration conf = HBaseConfiguration.create(); - conf.set("hbase.zookeeper.property.clientPort", "2181"); - conf.set("hbase.zookeeper.quorum", "172.16.1.151" ); - conf.set("zookeeper.znode.parent", "/hbase2"); - - Connection conn = ConnectionFactory.createConnection(conf); - //Table table = conn.getTable(TableName.valueOf("tb2")); - - RegionLocator regionLocator = conn.getRegionLocator(TableName.valueOf("tb2")); - regionLocator.getStartEndKeys(); - - } - -} diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index d406061b54..d5dd35eba4 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -54,6 +54,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java index a3e75a9caa..d4db300d08 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.hbase.writer; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.enums.ColumnType; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.hbase.HbaseHelper; @@ -27,6 +28,7 @@ import com.dtstack.flinkx.util.DateUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.Validate; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; @@ -34,10 +36,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -93,8 +97,40 @@ public class HbaseOutputFormat extends BaseRichOutputFormat { private transient ThreadLocal timeMillisecondFormatThreadLocal; + private boolean openKerberos = false; + @Override public void configure(Configuration parameters) { + } + + @Override + public void openInternal(int taskNumber, int numTasks) throws IOException { + openKerberos = HbaseHelper.openKerberos(hbaseConfig); + if (openKerberos) { + sleepRandomTime(); + + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + openConnection(); + return null; + } + }); + } else { + openConnection(); + } + } + + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long)(10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + + public void openConnection() { LOG.info("HbaseOutputFormat configure start"); nameMaps = Maps.newConcurrentMap(); nameByteMaps = Maps.newConcurrentMap(); @@ -103,9 +139,9 @@ public void configure(Configuration parameters) { Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); try { - connection = HbaseHelper.getHbaseConnection(hbaseConfig); - org.apache.hadoop.conf.Configuration hConfiguration = HbaseHelper.getConfig(hbaseConfig); + connection = ConnectionFactory.createConnection(hConfiguration); + bufferedMutator = connection.getBufferedMutator( new BufferedMutatorParams(TableName.valueOf(tableName)) .pool(HTable.getDefaultExecutor(hConfiguration)) @@ -129,11 +165,6 @@ public void configure(Configuration parameters) { LOG.info("HbaseOutputFormat configure end"); } - @Override - public void openInternal(int taskNumber, int numTasks) throws IOException { - - } - @Override public void writeSingleRecordInternal(Row record) throws WriteRecordException { int i = 0; @@ -159,17 +190,19 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { String name = columnNames.get(i); String[] cfAndQualifier = nameMaps.get(name); byte[][] cfAndQualifierBytes = nameByteMaps.get(name); - if(cfAndQualifier == null || cfAndQualifierBytes==null){ - String promptInfo = "Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name; + if(cfAndQualifier == null || cfAndQualifierBytes == null){ cfAndQualifier = name.split(":"); - Validate.isTrue(cfAndQualifier != null && cfAndQualifier.length == 2 - && org.apache.commons.lang3.StringUtils.isNotBlank(cfAndQualifier[0]) - && org.apache.commons.lang3.StringUtils.isNotBlank(cfAndQualifier[1]), promptInfo); - nameMaps.put(name,cfAndQualifier); - cfAndQualifierBytes = new byte[2][]; - cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); - cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); - nameByteMaps.put(name,cfAndQualifierBytes); + if(cfAndQualifier.length == 2 + && StringUtils.isNotBlank(cfAndQualifier[0]) + && StringUtils.isNotBlank(cfAndQualifier[1])){ + nameMaps.put(name,cfAndQualifier); + cfAndQualifierBytes = new byte[2][]; + cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); + cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); + nameByteMaps.put(name,cfAndQualifierBytes); + } else { + throw new IllegalArgumentException("Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name); + } } ColumnType columnType = ColumnType.getType(type); @@ -180,8 +213,6 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { cfAndQualifierBytes[0], cfAndQualifierBytes[1], columnBytes); - }else{ - continue; } } @@ -195,20 +226,21 @@ public void writeSingleRecordInternal(Row record) throws WriteRecordException { } private SimpleDateFormat getSimpleDateFormat(String sign){ - SimpleDateFormat format = null; - if("sss".equalsIgnoreCase(sign)){ + SimpleDateFormat format; + if(ConstantValue.TIME_SECOND_SUFFIX.equals(sign)){ format = timeSecondFormatThreadLocal.get(); if(format == null){ format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); timeSecondFormatThreadLocal.set(format); } - }else if("SSS".equalsIgnoreCase(sign)){ + } else { format = timeMillisecondFormatThreadLocal.get(); if(format == null){ format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss SSS"); timeMillisecondFormatThreadLocal.set(format); } } + return format; } @@ -219,7 +251,7 @@ protected String recordConvertDetailErrorMessage(int pos, Row row) { @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new IllegalArgumentException(); + notSupportBatchWrite("HbaseWriter"); } private byte[] getRowkey(Row record) throws Exception{ @@ -249,8 +281,8 @@ public long getVersion(Row record){ if(record.getField(index) == null){ throw new IllegalArgumentException("null verison column!"); } - SimpleDateFormat dfSeconds = getSimpleDateFormat("sss"); - SimpleDateFormat dfMs = getSimpleDateFormat("SSS"); + SimpleDateFormat dfSeconds = getSimpleDateFormat(ConstantValue.TIME_SECOND_SUFFIX); + SimpleDateFormat dfMs = getSimpleDateFormat(ConstantValue.TIME_MILLISECOND_SUFFIX); Object column = record.getField(index); if(column instanceof Long){ Long longValue = (Long) column; @@ -370,7 +402,7 @@ private byte[] intToBytes(Object column) { if(column instanceof Integer) { intValue = (Integer) column; } else if(column instanceof Long) { - intValue = Integer.valueOf(((Long)column).intValue()); + intValue = ((Long) column).intValue(); } else if(column instanceof Double) { intValue = ((Double) column).intValue(); } else if(column instanceof Float) { @@ -378,7 +410,7 @@ private byte[] intToBytes(Object column) { } else if(column instanceof Short) { intValue = ((Short) column).intValue(); } else if(column instanceof Boolean) { - intValue = ((Boolean) column).booleanValue() ? 1 : 0; + intValue = (Boolean) column ? 1 : 0; } else if(column instanceof String) { intValue = Integer.valueOf((String) column); } else { @@ -401,7 +433,7 @@ private byte[] longToBytes(Object column) { } else if(column instanceof Short) { longValue = ((Short) column).longValue(); } else if(column instanceof Boolean) { - longValue = ((Boolean) column).booleanValue() ? 1L : 0L; + longValue = (Boolean) column ? 1L : 0L; } else if(column instanceof String) { longValue = Long.valueOf((String) column); }else if (column instanceof Timestamp){ @@ -426,7 +458,7 @@ private byte[] doubleToBytes(Object column) { } else if(column instanceof Short) { doubleValue = ((Short) column).doubleValue(); } else if(column instanceof Boolean) { - doubleValue = ((Boolean) column).booleanValue() ? 1.0 : 0.0; + doubleValue = (Boolean) column ? 1.0 : 0.0; } else if(column instanceof String) { doubleValue = Double.valueOf((String) column); } else { @@ -449,7 +481,7 @@ private byte[] floatToBytes(Object column) { } else if(column instanceof Short) { floatValue = ((Short) column).floatValue(); } else if(column instanceof Boolean) { - floatValue = ((Boolean) column).booleanValue() ? 1.0f : 0.0f; + floatValue = (Boolean) column ? 1.0f : 0.0f; } else if(column instanceof String) { floatValue = Float.valueOf((String) column); } else { @@ -472,7 +504,7 @@ private byte[] shortToBytes(Object column) { } else if(column instanceof Short) { shortValue = (Short) column; } else if(column instanceof Boolean) { - shortValue = ((Boolean) column).booleanValue() ? (short) 1 : (short) 0 ; + shortValue = (Boolean) column ? (short) 1 : (short) 0 ; } else if(column instanceof String) { shortValue = Short.valueOf((String) column); } else { @@ -484,15 +516,15 @@ private byte[] shortToBytes(Object column) { private byte[] boolToBytes(Object column) { Boolean booleanValue = null; if(column instanceof Integer) { - booleanValue = (Integer)column == 0 ? false : true; + booleanValue = (Integer) column != 0; } else if(column instanceof Long) { - booleanValue = (Long) column == 0L ? false : true; + booleanValue = (Long) column != 0L; } else if(column instanceof Double) { - booleanValue = (Double) column == 0.0 ? false : true; + booleanValue = new Double(0.0).compareTo((Double) column) != 0; } else if(column instanceof Float) { - booleanValue = (Float) column == 0.0f ? false : true; + booleanValue = new Float(0.0f).compareTo((Float) column) != 0; } else if(column instanceof Short) { - booleanValue = (Short) column == 0 ? false : true; + booleanValue = (Short) column != 0; } else if(column instanceof Boolean) { booleanValue = (Boolean) column; } else if(column instanceof String) { diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java index 4da4c10754..ec96e04c17 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseOutputFormatBuilder.java @@ -110,5 +110,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("HbaseWriter"); } } diff --git a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java index 42091f62d3..3cb5bce6ce 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/main/java/com/dtstack/flinkx/hbase/writer/HbaseWriter.java @@ -31,8 +31,23 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hbase.HbaseConfigConstants.*; -import static com.dtstack.flinkx.hbase.HbaseConfigKeys.*; + +import static com.dtstack.flinkx.hbase.HbaseConfigConstants.DEFAULT_WAL_FLAG; +import static com.dtstack.flinkx.hbase.HbaseConfigConstants.DEFAULT_WRITE_BUFFER_SIZE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_COLUMN_NAME; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_COLUMN_TYPE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ENCODING; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_HBASE_CONFIG; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_NULL_MODE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN_INDEX; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_ROW_KEY_COLUMN_VALUE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN_INDEX; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_VERSION_COLUMN_VALUE; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_WAL_FLAG; +import static com.dtstack.flinkx.hbase.HbaseConfigKeys.KEY_WRITE_BUFFER_SIZE; /** * The Writer plugin of HBase diff --git a/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java b/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java index dbf12dfa2e..75143994f5 100644 --- a/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java +++ b/flinkx-hbase/flinkx-hbase-writer/src/test/java/com.dtstack.flinkx.hbase.writer/RowKeyFunctionTest.java @@ -18,7 +18,10 @@ package com.dtstack.flinkx.hbase.writer; -import com.dtstack.flinkx.hbase.writer.function.*; +import com.dtstack.flinkx.hbase.writer.function.FunctionParser; +import com.dtstack.flinkx.hbase.writer.function.FunctionTree; +import com.dtstack.flinkx.hbase.writer.function.Md5Function; +import com.dtstack.flinkx.hbase.writer.function.StringFunction; import org.junit.Assert; import org.junit.Test; diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java index 913c0fd41c..2f0c7ec101 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java @@ -19,7 +19,6 @@ package com.dtstack.flinkx.hdfs; import org.apache.commons.lang.StringUtils; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; /** * @author jiangbo diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java index fc6ced4a25..a4b44360e3 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsConfigKeys.java @@ -60,4 +60,6 @@ public class HdfsConfigKeys { public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + public static final String KEY_ENABLE_DICTIONARY = "enableDictionary"; + } diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index b678812f68..0acaa924b8 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -19,15 +19,20 @@ package com.dtstack.flinkx.hdfs; import com.dtstack.flinkx.enums.ColumnType; -import com.dtstack.flinkx.util.DateUtil; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.io.*; +import org.apache.hadoop.io.ByteWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.parquet.io.api.Binary; -import java.sql.Date; -import java.text.SimpleDateFormat; /** * Utilities for HdfsReader and HdfsWriter @@ -39,62 +44,13 @@ public class HdfsUtil { public static final String NULL_VALUE = "\\N"; - public static Object string2col(String str, String type, SimpleDateFormat customDateFormat) { - if (str == null || str.length() == 0){ - return null; - } - - if(type == null){ - return str; - } + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; - ColumnType columnType = ColumnType.fromString(type.toUpperCase()); - Object ret; - switch(columnType) { - case TINYINT: - ret = Byte.valueOf(str.trim()); - break; - case SMALLINT: - ret = Short.valueOf(str.trim()); - break; - case INT: - ret = Integer.valueOf(str.trim()); - break; - case BIGINT: - ret = Long.valueOf(str.trim()); - break; - case FLOAT: - ret = Float.valueOf(str.trim()); - break; - case DOUBLE: - case DECIMAL: - ret = Double.valueOf(str.trim()); - break; - case STRING: - case VARCHAR: - case CHAR: - if(customDateFormat != null){ - ret = DateUtil.columnToDate(str,customDateFormat); - ret = DateUtil.timestampToString((Date)ret); - } else { - ret = str; - } - break; - case BOOLEAN: - ret = Boolean.valueOf(str.trim().toLowerCase()); - break; - case DATE: - ret = DateUtil.columnToDate(str,customDateFormat); - break; - case TIMESTAMP: - ret = DateUtil.columnToTimestamp(str,customDateFormat); - break; - default: - throw new IllegalArgumentException("Unsupported field type:" + type); - } + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; - return ret; - } + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; public static Object getWritableValue(Object writable) { Class clz = writable.getClass(); @@ -166,4 +122,78 @@ public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) return objectInspector; } + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision){ + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static byte[] longToByteArray(long data){ + long nano = data * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return dst; + } + + private static byte[] getBytes(long i) { + byte[] bytes=new byte[8]; + bytes[0]=(byte)((i >> 56) & 0xFF); + bytes[1]=(byte)((i >> 48) & 0xFF); + bytes[2]=(byte)((i >> 40) & 0xFF); + bytes[3]=(byte)((i >> 32) & 0xFF); + bytes[4]=(byte)((i >> 24) & 0xFF); + bytes[5]=(byte)((i >> 16) & 0xFF); + bytes[6]=(byte)((i >> 8) & 0xFF); + bytes[7]=(byte)(i & 0xFF); + return bytes; + } + + /** + * @param bytes + */ + private static void flip(byte[] bytes) { + for(int i=0,j=bytes.length-1;i + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java index f6d802cf1b..e5e6a8ee37 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java @@ -21,10 +21,15 @@ import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.security.UserGroupInformation; +import java.io.File; import java.io.IOException; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -36,6 +41,8 @@ */ public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { + private static final String PARTITION_SPLIT_CHAR = "="; + protected Map hadoopConfig; protected List metaColumns; @@ -61,14 +68,25 @@ public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { protected Object value; - protected boolean isFileEmpty = false; - protected String filterRegex; + protected transient UserGroupInformation ugi; + + protected boolean openKerberos; + + protected String currentPartition; + + protected transient FileSystem fs; + @Override public void openInputFormat() throws IOException { super.openInputFormat(); conf = buildConfig(); + + openKerberos = FileSystemUtil.isOpenKerberos(hadoopConfig); + if (openKerberos) { + ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + } } protected JobConf buildConfig() { @@ -81,7 +99,7 @@ protected JobConf buildConfig() { @Override public boolean reachedEnd() throws IOException { - return isFileEmpty || !recordReader.next(key, value); + return !recordReader.next(key, value); } @Override @@ -91,4 +109,27 @@ public void closeInternal() throws IOException { } } + /** + * 从hdfs路径中获取当前分区信息 + * @param path hdfs路径 + */ + public void findCurrentPartition(Path path){ + Map map = new HashMap<>(16); + String pathStr = path.getParent().toString(); + int index; + while((index = pathStr.lastIndexOf(PARTITION_SPLIT_CHAR)) > 0){ + int i = pathStr.lastIndexOf(File.separator); + String name = pathStr.substring(i + 1, index); + String value = pathStr.substring(index + 1); + map.put(name, value); + pathStr = pathStr.substring(0, i); + } + + for (MetaColumn column : metaColumns) { + if(column.getPart()){ + column.setValue(map.get(column.getName())); + } + } + } + } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java index a87d704310..6f48d4e404 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormat.java @@ -22,21 +22,25 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.hive.ql.io.orc.*; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.ql.io.orc.OrcSplit; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reporter; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.*; +import java.security.PrivilegedAction; import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; /** * The subclass of HdfsInputFormat which handles orc files @@ -46,99 +50,102 @@ */ public class HdfsOrcInputFormat extends BaseHdfsInputFormat { - private transient OrcSerde orcSerde; - private transient String[] fullColNames; - private transient String[] fullColTypes; - private transient StructObjectInspector inspector; private transient List fields; private static final String COMPLEX_FIELD_TYPE_SYMBOL_REGEX = ".*(<|>|\\{|}|[|]).*"; + private AtomicBoolean isInit = new AtomicBoolean(false); + @Override - public void openInputFormat() throws IOException{ + public void openInputFormat() throws IOException { super.openInputFormat(); + inputFormat = new OrcInputFormat(); + } + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); - FileSystem fs; try { - fs = FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); + if (!isInit.get()) { + init(orcSplit.getPath()); + isInit.set(true); + } } catch (Exception e) { - throw new RuntimeException(e); + throw new IOException("初始化[inspector]出错", e); } - orcSerde = new OrcSerde(); - inputFormat = new OrcInputFormat(); - org.apache.hadoop.hive.ql.io.orc.Reader reader = null; - try { - OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(conf); - readerOptions.filesystem(fs); - - Path path = new Path(inputPath); - String typeStruct = null; - - if(fs.isDirectory(path)) { - RemoteIterator iterator = fs.listFiles(path, true); - while(iterator.hasNext()) { - FileStatus fileStatus = iterator.next(); - if(fileStatus.isFile() && fileStatus.getLen() > 49) { - Path subPath = fileStatus.getPath(); - reader = OrcFile.createReader(subPath, readerOptions); - typeStruct = reader.getObjectInspector().getTypeName(); - if(StringUtils.isNotEmpty(typeStruct)) { - break; - } + if (openKerberos) { + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + openOrcReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); } - } - if(reader == null) { - //throw new RuntimeException("orcfile dir is empty!"); - LOG.error("orc file {} is empty!", inputPath); - isFileEmpty = true; - return; + return null; } + }); + } else { + openOrcReader(inputSplit); + } + } - } else { - reader = OrcFile.createReader(path, readerOptions); - typeStruct = reader.getObjectInspector().getTypeName(); - } + private void openOrcReader(InputSplit inputSplit) throws IOException{ + numReadCounter = getRuntimeContext().getLongCounter("numRead"); + HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; + OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); + recordReader = inputFormat.getRecordReader(orcSplit, conf, Reporter.NULL); + key = recordReader.createKey(); + value = recordReader.createValue(); + fields = inspector.getAllStructFieldRefs(); + } - if (StringUtils.isEmpty(typeStruct)) { - throw new RuntimeException("can't retrieve type struct from " + path); - } + private void init(Path path) throws Exception { + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(conf); + readerOptions.filesystem(fs); + org.apache.hadoop.hive.ql.io.orc.Reader reader = OrcFile.createReader(path, readerOptions); + String typeStruct = reader.getObjectInspector().getTypeName(); - int startIndex = typeStruct.indexOf("<") + 1; - int endIndex = typeStruct.lastIndexOf(">"); - typeStruct = typeStruct.substring(startIndex, endIndex); + if (StringUtils.isEmpty(typeStruct)) { + throw new RuntimeException("can't retrieve type struct from " + path); + } - if(typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)){ - throw new RuntimeException("Field types such as array, map, and struct are not supported."); - } + int startIndex = typeStruct.indexOf("<") + 1; + int endIndex = typeStruct.lastIndexOf(">"); + typeStruct = typeStruct.substring(startIndex, endIndex); - List cols = parseColumnAndType(typeStruct); + if(typeStruct.matches(COMPLEX_FIELD_TYPE_SYMBOL_REGEX)){ + throw new RuntimeException("Field types such as array, map, and struct are not supported."); + } - fullColNames = new String[cols.size()]; - fullColTypes = new String[cols.size()]; + List cols = parseColumnAndType(typeStruct); - for(int i = 0; i < cols.size(); ++i) { - String[] temp = cols.get(i).split(":"); - fullColNames[i] = temp[0]; - fullColTypes[i] = temp[1]; - } + fullColNames = new String[cols.size()]; + String[] fullColTypes = new String[cols.size()]; - Properties p = new Properties(); - p.setProperty("columns", StringUtils.join(fullColNames, ",")); - p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); - orcSerde.initialize(conf, p); + for(int i = 0; i < cols.size(); ++i) { + String[] temp = cols.get(i).split(":"); + fullColNames[i] = temp[0]; + fullColTypes[i] = temp[1]; + } - this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); + Properties p = new Properties(); + p.setProperty("columns", StringUtils.join(fullColNames, ",")); + p.setProperty("columns.types", StringUtils.join(fullColTypes, ":")); - } catch (Throwable e) { - throw new RuntimeException(e); - } + OrcSerde orcSerde = new OrcSerde(); + orcSerde.initialize(conf, p); + + this.inspector = (StructObjectInspector) orcSerde.getObjectInspector(); } private List parseColumnAndType(String typeStruct){ @@ -175,12 +182,25 @@ private List parseColumnAndType(String typeStruct){ @Override public HdfsOrcInputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { - try { - FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); - } catch (Exception e) { - throw new IOException(e); + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + LOG.info("user:{}, ", ugi.getShortUserName()); + return ugi.doAs(new PrivilegedAction() { + @Override + public HdfsOrcInputSplit[] run() { + try { + return createOrcSplit(minNumSplits); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + return createOrcSplit(minNumSplits); } + } + private HdfsOrcInputSplit[] createOrcSplit(int minNumSplits) throws IOException{ JobConf jobConf = FileSystemUtil.getJobConf(hadoopConfig, defaultFs); org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, inputPath); org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(buildConfig(), HdfsPathFilter.class); @@ -204,24 +224,6 @@ public HdfsOrcInputSplit[] createInputSplitsInternal(int minNumSplits) throws IO return null; } - - @Override - public void openInternal(InputSplit inputSplit) throws IOException { - - if(isFileEmpty){ - return; - } - - numReadCounter = getRuntimeContext().getLongCounter("numRead"); - HdfsOrcInputSplit hdfsOrcInputSplit = (HdfsOrcInputSplit) inputSplit; - OrcSplit orcSplit = hdfsOrcInputSplit.getOrcSplit(); - recordReader = inputFormat.getRecordReader(orcSplit, conf, Reporter.NULL); - key = recordReader.createKey(); - value = recordReader.createValue(); - fields = inspector.getAllStructFieldRefs(); - } - - @Override public Row nextRecordInternal(Row row) throws IOException { if(metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ @@ -239,17 +241,17 @@ public Row nextRecordInternal(Row row) throws IOException { MetaColumn metaColumn = metaColumns.get(i); Object val = null; - if(metaColumn.getIndex() != -1){ + if(metaColumn.getValue() != null){ + val = metaColumn.getValue(); + }else if(metaColumn.getIndex() != -1){ val = inspector.getStructFieldData(value, fields.get(metaColumn.getIndex())); if (val == null && metaColumn.getValue() != null){ val = metaColumn.getValue(); } - } else if(metaColumn.getValue() != null){ - val = metaColumn.getValue(); } if(val instanceof String || val instanceof org.apache.hadoop.io.Text){ - val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + val = StringUtil.string2col(String.valueOf(val), metaColumn.getType(), metaColumn.getTimeFormat()); } else if(val != null){ val = HdfsUtil.getWritableValue(val); } @@ -291,5 +293,4 @@ public int getSplitNumber() { return splitNumber; } } - } diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java index 16fb276495..565c0a49b2 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsParquetInputFormat.java @@ -20,9 +20,9 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.enums.ColumnType; -import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.StringUtil; import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -43,6 +43,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Iterator; @@ -75,6 +76,8 @@ public class HdfsParquetInputFormat extends BaseHdfsInputFormat { private static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); + private static final int TIMESTAMP_BINARY_LENGTH = 12; + @Override protected void openInternal(InputSplit inputSplit) throws IOException { currentSplitFilePaths = ((HdfsParquetSplit)inputSplit).getPaths(); @@ -82,14 +85,33 @@ protected void openInternal(InputSplit inputSplit) throws IOException { private boolean nextLine() throws IOException{ if (currentFileReader == null && currentFileIndex <= currentSplitFilePaths.size()-1){ - nextFile(); + if (openKerberos) { + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + nextFile(); + return null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + nextFile(); + } } if (currentFileReader == null){ return false; } - currentLine = currentFileReader.read(); + if (openKerberos) { + currentLine = nextLineWithKerberos(); + } else { + currentLine = currentFileReader.read(); + } + if (fullColNames == null && currentLine != null){ fullColNames = new ArrayList<>(); fullColTypes = new ArrayList<>(); @@ -117,11 +139,24 @@ private boolean nextLine() throws IOException{ return currentLine != null; } + private Group nextLineWithKerberos() { + return ugi.doAs(new PrivilegedAction() { + @Override + public Group run() { + try { + return currentFileReader.read(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } + private void nextFile() throws IOException{ - String path = currentSplitFilePaths.get(currentFileIndex); - ParquetReader.Builder reader = ParquetReader.builder(new GroupReadSupport(), new Path(path)).withConf(conf); + Path path = new Path(currentSplitFilePaths.get(currentFileIndex)); + findCurrentPartition(path); + ParquetReader.Builder reader = ParquetReader.builder(new GroupReadSupport(), path).withConf(conf); currentFileReader = reader.build(); - currentFileIndex++; } @@ -139,22 +174,20 @@ protected Row nextRecordInternal(Row row) throws IOException { MetaColumn metaColumn = metaColumns.get(i); Object val = null; - if(metaColumn.getIndex() != -1){ - if (metaColumn.getIndex() < currentLine.getType().getFieldCount()) { - if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ - val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); - } + if (metaColumn.getValue() != null){ + val = metaColumn.getValue(); + }else if(metaColumn.getIndex() != -1){ + if(currentLine.getFieldRepetitionCount(metaColumn.getIndex()) > 0){ + val = getData(currentLine,metaColumn.getType(),metaColumn.getIndex()); + } - if (val == null && metaColumn.getValue() != null){ - val = metaColumn.getValue(); - } + if (val == null && metaColumn.getValue() != null){ + val = metaColumn.getValue(); } - } else if (metaColumn.getValue() != null){ - val = metaColumn.getValue(); } if(val instanceof String){ - val = HdfsUtil.string2col(String.valueOf(val),metaColumn.getType(),metaColumn.getTimeFormat()); + val = StringUtil.string2col(String.valueOf(val), metaColumn.getType(), metaColumn.getTimeFormat()); } row.setField(i,val); @@ -169,7 +202,7 @@ public boolean reachedEnd() throws IOException { return !nextLine(); } - private Object getData(Group currentLine,String type,int index){ + public Object getData(Group currentLine,String type,int index){ Object data = null; ColumnType columnType = ColumnType.fromString(type); @@ -302,13 +335,12 @@ private static List getAllPartitionPath(String tableLocation, FileSystem private String getTypeName(String method){ String typeName; switch (method){ + case "getBoolean": case "getInteger" : typeName = "int";break; case "getInt96" : typeName = "bigint";break; case "getFloat" : typeName = "float";break; case "getDouble" : typeName = "double";break; case "getBinary" : typeName = "binary";break; - case "getString" : typeName = "string";break; - case "getBoolean" : typeName = "int";break; default:typeName = "string"; } @@ -319,11 +351,11 @@ private String getTypeName(String method){ * @param timestampBinary * @return */ - private long getTimestampMillis(Binary timestampBinary) - { - if (timestampBinary.length() != 12) { + private long getTimestampMillis(Binary timestampBinary) { + if (timestampBinary.length() != TIMESTAMP_BINARY_LENGTH) { return 0; } + byte[] bytes = timestampBinary.getBytes(); long timeOfDayNanos = Longs.fromBytes(bytes[7], bytes[6], bytes[5], bytes[4], bytes[3], bytes[2], bytes[1], bytes[0]); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java index 5e3cea2a94..d4291741f0 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java @@ -43,9 +43,9 @@ public class HdfsReader extends BaseDataReader { protected String fileType; protected String path; protected String fieldDelimiter; - private List metaColumns; + protected List metaColumns; protected Map hadoopConfig; - private String filterRegex; + protected String filterRegex; public HdfsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java index 2ba2b1c363..f7d2dac2b7 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsTextInputFormat.java @@ -22,25 +22,25 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.FileSystemUtil; -import jodd.util.StringUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.io.output.ByteArrayOutputStream; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hadoop.security.UserGroupInformation; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.charset.Charset; -import java.nio.charset.UnsupportedCharsetException; -import java.util.Map; +import java.security.PrivilegedAction; /** * The subclass of HdfsInputFormat which handles text files @@ -59,12 +59,25 @@ public void openInputFormat() throws IOException { @Override public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { - try { - FileSystemUtil.getFileSystem(hadoopConfig, defaultFs); - } catch (Exception e) { - throw new IOException(e); + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + LOG.info("user:{}, ", ugi.getShortUserName()); + return ugi.doAs(new PrivilegedAction() { + @Override + public InputSplit[] run() { + try { + return createTextSplit(minNumSplits); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }); + } else { + return createTextSplit(minNumSplits); } + } + private InputSplit[] createTextSplit(int minNumSplits) throws IOException{ JobConf jobConf = buildConfig(); org.apache.hadoop.mapred.FileInputFormat.setInputPathFilter(jobConf, HdfsPathFilter.class); @@ -88,8 +101,30 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti @Override public void openInternal(InputSplit inputSplit) throws IOException { + + if(openKerberos){ + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + try { + openHdfsTextReader(inputSplit); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return null; + } + }); + }else{ + openHdfsTextReader(inputSplit); + } + + } + + private void openHdfsTextReader(InputSplit inputSplit) throws IOException{ HdfsTextInputSplit hdfsTextInputSplit = (HdfsTextInputSplit) inputSplit; org.apache.hadoop.mapred.InputSplit fileSplit = hdfsTextInputSplit.getTextSplit(); + findCurrentPartition(((FileSplit) fileSplit).getPath()); recordReader = inputFormat.getRecordReader(fileSplit, conf, Reporter.NULL); key = new LongWritable(); value = new Text(); @@ -98,7 +133,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { @Override public Row nextRecordInternal(Row row) throws IOException { String line = new String(((Text)value).getBytes(), 0, ((Text)value).getLength(), charsetName); - String[] fields = line.split(delimiter); + String[] fields = StringUtils.splitPreserveAllTokens(line, delimiter); if (metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ row = new Row(fields.length); @@ -121,7 +156,7 @@ public Row nextRecordInternal(Row row) throws IOException { } if(value != null){ - value = HdfsUtil.string2col(String.valueOf(value),metaColumn.getType(),metaColumn.getTimeFormat()); + value = StringUtil.string2col(String.valueOf(value), metaColumn.getType(),metaColumn.getTimeFormat()); } row.setField(i, value); @@ -135,69 +170,7 @@ public Row nextRecordInternal(Row row) throws IOException { public boolean reachedEnd() throws IOException { key = new LongWritable(); value = new Text(); - return isFileEmpty || !recordReader.next(key, value); - } - - - public static class HdfsTextInputFormatBuilder { - - private HdfsTextInputFormat format; - - private HdfsTextInputFormatBuilder() { - format = new HdfsTextInputFormat(); - } - - public HdfsTextInputFormatBuilder setHadoopConfig(Map hadoopConfig) { - format.hadoopConfig = hadoopConfig; - return this; - } - - public HdfsTextInputFormatBuilder setInputPaths(String inputPaths) { - format.inputPath = inputPaths; - return this; - } - - public HdfsTextInputFormatBuilder setBytes(long bytes) { - format.bytes = bytes; - return this; - } - - public HdfsTextInputFormatBuilder setMonitorUrls(String monitorUrls) { - format.monitorUrls = monitorUrls; - return this; - } - - public HdfsTextInputFormatBuilder setDelimiter(String delimiter) { - if(delimiter == null) { - delimiter = "\\001"; - } - format.delimiter = delimiter; - return this; - } - - public HdfsTextInputFormatBuilder setDefaultFs(String defaultFs) { - format.defaultFs = defaultFs; - return this; - } - - public HdfsTextInputFormatBuilder setcharsetName (String charsetName) { - if(StringUtil.isNotEmpty(charsetName)) { - if(!Charset.isSupported(charsetName)) { - throw new UnsupportedCharsetException("The charset " + charsetName + " is not supported."); - } - this.format.charsetName = charsetName; - } - - return this; - } - - public HdfsTextInputFormat finish() { - return format; - } - } - - public static HdfsTextInputFormatBuilder buildHdfsTextInputFormat() { - return new HdfsTextInputFormatBuilder(); + return !recordReader.next(key, value); } static class HdfsTextInputSplit implements InputSplit { @@ -229,5 +202,4 @@ public int getSplitNumber() { return splitNumber; } } - } \ No newline at end of file diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java new file mode 100644 index 0000000000..a01ea427df --- /dev/null +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java @@ -0,0 +1,1221 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.io.orc; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.ValidReadTxnList; +import org.apache.hadoop.hive.common.ValidTxnList; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface; +import org.apache.hadoop.hive.ql.io.*; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.shims.HadoopShims; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.*; +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +/** + * A MapReduce/Hive input format for ORC files. + *

+ * This class implements both the classic InputFormat, which stores the rows + * directly, and AcidInputFormat, which stores a series of events with the + * following schema: + *

+ *   class AcidEvent<ROW> {
+ *     enum ACTION {INSERT, UPDATE, DELETE}
+ *     ACTION operation;
+ *     long originalTransaction;
+ *     int bucket;
+ *     long rowId;
+ *     long currentTransaction;
+ *     ROW row;
+ *   }
+ * 
+ * Each AcidEvent object corresponds to an update event. The + * originalTransaction, bucket, and rowId are the unique identifier for the row. + * The operation and currentTransaction are the operation and the transaction + * that added this event. Insert and update events include the entire row, while + * delete events have null for row. + */ +public class OrcInputFormat implements InputFormat, + InputFormatChecker, VectorizedInputFormatInterface, + AcidInputFormat, + CombineHiveInputFormat.AvoidSplitCombination { + + static final HadoopShims SHIMS = ShimLoader.getHadoopShims(); + static final String MIN_SPLIT_SIZE = + SHIMS.getHadoopConfNames().get("MAPREDMINSPLITSIZE"); + static final String MAX_SPLIT_SIZE = + SHIMS.getHadoopConfNames().get("MAPREDMAXSPLITSIZE"); + static final String SARG_PUSHDOWN = "sarg.pushdown"; + private static final Log LOG = LogFactory.getLog(OrcInputFormat.class); + private static final long DEFAULT_MIN_SPLIT_SIZE = 16 * 1024 * 1024; + private static final long DEFAULT_MAX_SPLIT_SIZE = 256 * 1024 * 1024; + + private static final PerfLogger perfLogger = PerfLogger.getPerfLogger(); + private static final String CLASS_NAME = ReaderImpl.class.getName(); + + /** + * When picking the hosts for a split that crosses block boundaries, + * any drop any host that has fewer than MIN_INCLUDED_LOCATION of the + * number of bytes available on the host with the most. + * If host1 has 10MB of the split, host2 has 20MB, and host3 has 18MB the + * split will contain host2 (100% of host2) and host3 (90% of host2). Host1 + * with 50% will be dropped. + */ + private static final double MIN_INCLUDED_LOCATION = 0.80; + + /** + * Get the root column for the row. In ACID format files, it is offset by + * the extra metadata columns. + * @param isOriginal is the file in the original format? + * @return the column number for the root of row. + */ + private static int getRootColumn(boolean isOriginal) { + return isOriginal ? 0 : (OrcRecordUpdater.ROW + 1); + } + + public static RecordReader createReaderFromFile(Reader file, + Configuration conf, + long offset, long length + ) throws IOException { + Reader.Options options = new Reader.Options().range(offset, length); + boolean isOriginal = + !file.hasMetadataValue(OrcRecordUpdater.ACID_KEY_INDEX_NAME); + List types = file.getTypes(); + setIncludedColumns(options, types, conf, isOriginal); + setSearchArgument(options, types, conf, isOriginal); + return file.rowsOptions(options); + } + + /** + * Recurse down into a type subtree turning on all of the sub-columns. + * @param types the types of the file + * @param result the global view of columns that should be included + * @param typeId the root of tree to enable + * @param rootColumn the top column + */ + private static void includeColumnRecursive(List types, + boolean[] result, + int typeId, + int rootColumn) { + result[typeId - rootColumn] = true; + OrcProto.Type type = types.get(typeId); + int children = type.getSubtypesCount(); + for(int i=0; i < children; ++i) { + includeColumnRecursive(types, result, type.getSubtypes(i), rootColumn); + } + } + + /** + * Take the configuration and figure out which columns we need to include. + * @param options the options to update + * @param types the types for the file + * @param conf the configuration + * @param isOriginal is the file in the original format? + */ + static void setIncludedColumns(Reader.Options options, + List types, + Configuration conf, + boolean isOriginal) { + int rootColumn = getRootColumn(isOriginal); + if (!ColumnProjectionUtils.isReadAllColumns(conf)) { + int numColumns = types.size() - rootColumn; + boolean[] result = new boolean[numColumns]; + result[0] = true; + OrcProto.Type root = types.get(rootColumn); + List included = ColumnProjectionUtils.getReadColumnIDs(conf); + for(int i=0; i < root.getSubtypesCount(); ++i) { + if (included.contains(i)) { + includeColumnRecursive(types, result, root.getSubtypes(i), + rootColumn); + } + } + options.include(result); + } else { + options.include(null); + } + } + + static void setSearchArgument(Reader.Options options, + List types, + Configuration conf, + boolean isOriginal) { + int rootColumn = getRootColumn(isOriginal); + String serializedPushdown = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR); + String sargPushdown = conf.get(SARG_PUSHDOWN); + String columnNamesString = + conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR); + if ((sargPushdown == null && serializedPushdown == null) + || columnNamesString == null) { + LOG.debug("No ORC pushdown predicate"); + options.searchArgument(null, null); + } else { + SearchArgument sarg; + if (serializedPushdown != null) { + sarg = SearchArgumentFactory.create + (Utilities.deserializeExpression(serializedPushdown)); + } else { + sarg = SearchArgumentFactory.create(sargPushdown); + } + LOG.info("ORC pushdown predicate: " + sarg); + String[] neededColumnNames = columnNamesString.split(","); + String[] columnNames = new String[types.size() - rootColumn]; + boolean[] includedColumns = options.getInclude(); + int i = 0; + for(int columnId: types.get(rootColumn).getSubtypesList()) { + if (includedColumns == null || includedColumns[columnId - rootColumn]) { + // this is guaranteed to be positive because types only have children + // ids greater than their own id. + columnNames[columnId - rootColumn] = neededColumnNames[i++]; + } + } + options.searchArgument(sarg, columnNames); + } + } + + /** + * Get the list of input {@link Path}s for the map-reduce job. + * + * @param conf The configuration of the job + * @return the list of input {@link Path}s for the map-reduce job. + */ + static Path[] getInputPaths(Configuration conf) throws IOException { + String dirs = conf.get("mapred.input.dir"); + if (dirs == null) { + throw new IOException("Configuration mapred.input.dir is not defined."); + } + String [] list = StringUtils.split(dirs); + Path[] result = new Path[list.length]; + for (int i = 0; i < list.length; i++) { + result[i] = new Path(StringUtils.unEscapeString(list[i])); + } + return result; + } + + static List generateSplitsInfo(Configuration conf) + throws IOException { + // use threads to resolve directories into splits + Context context = new Context(conf); + for(Path dir: getInputPaths(conf)) { + FileSystem fs = dir.getFileSystem(conf); + context.schedule(new FileGenerator(context, fs, dir)); + } + context.waitForTasks(); + // deal with exceptions + if (!context.errors.isEmpty()) { + List errors = + new ArrayList(context.errors.size()); + for(Throwable th: context.errors) { + if (th instanceof IOException) { + errors.add((IOException) th); + } else { + throw new RuntimeException("serious problem", th); + } + } + throw new InvalidInputException(errors); + } + if (context.cacheStripeDetails) { + LOG.info("FooterCacheHitRatio: " + context.cacheHitCounter.get() + "/" + + context.numFilesCounter.get()); + } + return context.splits; + } + + static Path findOriginalBucket(FileSystem fs, + Path directory, + int bucket) throws IOException { + for(FileStatus stat: fs.listStatus(directory)) { + String name = stat.getPath().getName(); + String numberPart = name.substring(0, name.indexOf('_')); + if (org.apache.commons.lang3.StringUtils.isNumeric(numberPart) && + Integer.parseInt(numberPart) == bucket) { + return stat.getPath(); + } + } + throw new IllegalArgumentException("Can't find bucket " + bucket + " in " + + directory); + } + + @Override + public boolean shouldSkipCombine(Path path, + Configuration conf) throws IOException { + return (conf.get(AcidUtils.CONF_ACID_KEY) != null) || AcidUtils.isAcid(path, conf); + } + + @Override + public boolean validateInput(FileSystem fs, HiveConf conf, + ArrayList files + ) throws IOException { + + if (Utilities.isVectorMode(conf)) { + return new VectorizedOrcInputFormat().validateInput(fs, conf, files); + } + + if (files.size() <= 0) { + return false; + } + for (FileStatus file : files) { + try { + OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(conf).filesystem(fs)); + } catch (IOException e) { + return false; + } + } + return true; + } + + @Override + public InputSplit[] getSplits(JobConf job, + int numSplits) throws IOException { + perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ORC_GET_SPLITS); + List result = generateSplitsInfo(job); + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ORC_GET_SPLITS); + return result.toArray(new InputSplit[result.size()]); + } + + @SuppressWarnings("unchecked") + private org.apache.hadoop.mapred.RecordReader + createVectorizedReader(InputSplit split, JobConf conf, Reporter reporter + ) throws IOException { + return (org.apache.hadoop.mapred.RecordReader) + new VectorizedOrcInputFormat().getRecordReader(split, conf, reporter); + } + + @Override + public org.apache.hadoop.mapred.RecordReader + getRecordReader(InputSplit inputSplit, JobConf conf, + Reporter reporter) throws IOException { + boolean vectorMode = Utilities.isVectorMode(conf); + + // if HiveCombineInputFormat gives us FileSplits instead of OrcSplits, + // we know it is not ACID. (see a check in CombineHiveInputFormat.getSplits() that assures this) + if (inputSplit.getClass() == FileSplit.class) { + if (vectorMode) { + return createVectorizedReader(inputSplit, conf, reporter); + } + return new OrcRecordReader(OrcFile.createReader( + ((FileSplit) inputSplit).getPath(), + OrcFile.readerOptions(conf)), conf, (FileSplit) inputSplit); + } + + OrcSplit split = (OrcSplit) inputSplit; + reporter.setStatus(inputSplit.toString()); + + Options options = new Options(conf).reporter(reporter); + final RowReader inner = getReader(inputSplit, options); + + + /*Even though there are no delta files, we still need to produce row ids so that an + * UPDATE or DELETE statement would work on a table which didn't have any previous updates*/ + if (split.isOriginal() && split.getDeltas().isEmpty()) { + if (vectorMode) { + return createVectorizedReader(inputSplit, conf, reporter); + } else { + return new NullKeyRecordReader(inner, conf); + } + } + + if (vectorMode) { + return (org.apache.hadoop.mapred.RecordReader) + new VectorizedOrcAcidRowReader(inner, conf, (FileSplit) inputSplit); + } + return new NullKeyRecordReader(inner, conf); + } + + @Override + public RowReader getReader(InputSplit inputSplit, + Options options) throws IOException { + final OrcSplit split = (OrcSplit) inputSplit; + final Path path = split.getPath(); + Path root; + if (split.hasBase()) { + if (split.isOriginal()) { + root = path.getParent(); + } else { + root = path.getParent().getParent(); + } + } else { + root = path; + } + final Path[] deltas = AcidUtils.deserializeDeltas(root, split.getDeltas()); + final Configuration conf = options.getConfiguration(); + final Reader reader; + final int bucket; + Reader.Options readOptions = new Reader.Options(); + readOptions.range(split.getStart(), split.getLength()); + if (split.hasBase()) { + bucket = AcidUtils.parseBaseBucketFilename(split.getPath(), conf) + .getBucket(); + reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); + final List types = reader.getTypes(); + setIncludedColumns(readOptions, types, conf, split.isOriginal()); + setSearchArgument(readOptions, types, conf, split.isOriginal()); + } else { + bucket = (int) split.getStart(); + reader = null; + } + String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY, + Long.MAX_VALUE + ":"); + ValidTxnList validTxnList = new ValidReadTxnList(txnString); + final OrcRawRecordMerger records = + new OrcRawRecordMerger(conf, true, reader, split.isOriginal(), bucket, + validTxnList, readOptions, deltas); + return new RowReader() { + OrcStruct innerRecord = records.createValue(); + + @Override + public ObjectInspector getObjectInspector() { + return ((StructObjectInspector) records.getObjectInspector()) + .getAllStructFieldRefs().get(OrcRecordUpdater.ROW) + .getFieldObjectInspector(); + } + + @Override + public boolean next(RecordIdentifier recordIdentifier, + OrcStruct orcStruct) throws IOException { + boolean result; + // filter out the deleted records + do { + result = records.next(recordIdentifier, innerRecord); + } while (result && + OrcRecordUpdater.getOperation(innerRecord) == + OrcRecordUpdater.DELETE_OPERATION); + if (result) { + // swap the fields with the passed in orcStruct + orcStruct.linkFields(OrcRecordUpdater.getRow(innerRecord)); + } + return result; + } + + @Override + public RecordIdentifier createKey() { + return records.createKey(); + } + + @Override + public OrcStruct createValue() { + return new OrcStruct(records.getColumns()); + } + + @Override + public long getPos() throws IOException { + return records.getPos(); + } + + @Override + public void close() throws IOException { + records.close(); + } + + @Override + public float getProgress() throws IOException { + return records.getProgress(); + } + }; + } + + @Override + public RawReader getRawReader(Configuration conf, + boolean collapseEvents, + int bucket, + ValidTxnList validTxnList, + Path baseDirectory, + Path[] deltaDirectory + ) throws IOException { + Reader reader = null; + boolean isOriginal = false; + if (baseDirectory != null) { + Path bucketFile; + if (baseDirectory.getName().startsWith(AcidUtils.BASE_PREFIX)) { + bucketFile = AcidUtils.createBucketFile(baseDirectory, bucket); + } else { + isOriginal = true; + bucketFile = findOriginalBucket(baseDirectory.getFileSystem(conf), + baseDirectory, bucket); + } + reader = OrcFile.createReader(bucketFile, OrcFile.readerOptions(conf)); + } + return new OrcRawRecordMerger(conf, collapseEvents, reader, isOriginal, + bucket, validTxnList, new Reader.Options(), deltaDirectory); + } + + private static class OrcRecordReader + implements org.apache.hadoop.mapred.RecordReader, + StatsProvidingRecordReader { + private final RecordReader reader; + private final long offset; + private final long length; + private final int numColumns; + private final Reader file; + private final SerDeStats stats; + private float progress = 0.0f; + + + OrcRecordReader(Reader file, Configuration conf, + FileSplit split) throws IOException { + List types = file.getTypes(); + this.file = file; + numColumns = (types.size() == 0) ? 0 : types.get(0).getSubtypesCount(); + this.offset = split.getStart(); + this.length = split.getLength(); + this.reader = createReaderFromFile(file, conf, offset, length); + this.stats = new SerDeStats(); + } + + @Override + public boolean next(NullWritable key, OrcStruct value) throws IOException { + if (reader.hasNext()) { + reader.next(value); + progress = reader.getProgress(); + return true; + } else { + return false; + } + } + + @Override + public NullWritable createKey() { + return NullWritable.get(); + } + + @Override + public OrcStruct createValue() { + return new OrcStruct(numColumns); + } + + @Override + public long getPos() throws IOException { + return offset + (long) (progress * length); + } + + @Override + public void close() throws IOException { + reader.close(); + } + + @Override + public float getProgress() throws IOException { + return progress; + } + + @Override + public SerDeStats getStats() { + stats.setRawDataSize(file.getRawDataSize()); + stats.setRowCount(file.getNumberOfRows()); + return stats; + } + } + + /** + * The global information about the split generation that we pass around to + * the different worker threads. + */ + static class Context { + private static Cache footerCache; + private final Configuration conf; + private final ExecutorService threadPool; + private final List splits = + new ArrayList(10000); + private final int numBuckets; + private final List errors = new ArrayList(); + private final long maxSize; + private final long minSize; + private final boolean footerInSplits; + private final boolean cacheStripeDetails; + private final AtomicInteger cacheHitCounter = new AtomicInteger(0); + private final AtomicInteger numFilesCounter = new AtomicInteger(0); + private Throwable fatalError = null; + private ValidTxnList transactionList; + + /** + * A count of the number of threads that may create more work for the + * thread pool. + */ + private int schedulers = 0; + + Context(Configuration conf) { + this.conf = conf; + minSize = conf.getLong(MIN_SPLIT_SIZE, DEFAULT_MIN_SPLIT_SIZE); + maxSize = conf.getLong(MAX_SPLIT_SIZE, DEFAULT_MAX_SPLIT_SIZE); + footerInSplits = HiveConf.getBoolVar(conf, + ConfVars.HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS); + numBuckets = + Math.max(conf.getInt(hive_metastoreConstants.BUCKET_COUNT, 0), 0); + LOG.debug("Number of buckets specified by conf file is " + numBuckets); + int cacheStripeDetailsSize = HiveConf.getIntVar(conf, + ConfVars.HIVE_ORC_CACHE_STRIPE_DETAILS_SIZE); + int numThreads = HiveConf.getIntVar(conf, + ConfVars.HIVE_ORC_COMPUTE_SPLITS_NUM_THREADS); + + cacheStripeDetails = (cacheStripeDetailsSize > 0); + + threadPool = Executors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("ORC_GET_SPLITS #%d").build()); + + synchronized (Context.class) { + if (footerCache == null && cacheStripeDetails) { + footerCache = CacheBuilder.newBuilder().concurrencyLevel(numThreads) + .initialCapacity(cacheStripeDetailsSize).softValues().build(); + } + } + String value = conf.get(ValidTxnList.VALID_TXNS_KEY, + Long.MAX_VALUE + ":"); + transactionList = new ValidReadTxnList(value); + } + + int getSchedulers() { + return schedulers; + } + + /** + * Get the Nth split. + * @param index if index >= 0, count from the front, otherwise count from + * the back. + * @return the Nth file split + */ + OrcSplit getResult(int index) { + if (index >= 0) { + return splits.get(index); + } else { + return splits.get(splits.size() + index); + } + } + + List getErrors() { + return errors; + } + + /** + * Add a unit of work. + * @param runnable the object to run + */ + synchronized void schedule(Runnable runnable) { + if (fatalError == null) { + if (runnable instanceof FileGenerator || + runnable instanceof SplitGenerator) { + schedulers += 1; + } + threadPool.execute(runnable); + } else { + throw new RuntimeException("serious problem", fatalError); + } + } + + /** + * Mark a worker that may generate more work as done. + */ + synchronized void decrementSchedulers() { + schedulers -= 1; + if (schedulers == 0) { + notify(); + } + } + + synchronized void notifyOnNonIOException(Throwable th) { + fatalError = th; + notify(); + } + + /** + * Wait until all of the tasks are done. It waits until all of the + * threads that may create more work are done and then shuts down the + * thread pool and waits for the final threads to finish. + */ + synchronized void waitForTasks() { + try { + while (schedulers != 0) { + wait(); + if (fatalError != null) { + threadPool.shutdownNow(); + throw new RuntimeException("serious problem", fatalError); + } + } + threadPool.shutdown(); + threadPool.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS); + } catch (InterruptedException ie) { + throw new IllegalStateException("interrupted", ie); + } + } + } + + /** + * Given a directory, get the list of files and blocks in those files. + * A thread is used for each directory. + */ + static final class FileGenerator implements Runnable { + private final Context context; + private final FileSystem fs; + private final Path dir; + + FileGenerator(Context context, FileSystem fs, Path dir) { + this.context = context; + this.fs = fs; + this.dir = dir; + } + + private void scheduleSplits(FileStatus file, + boolean isOriginal, + boolean hasBase, + List deltas) throws IOException{ + FileInfo info = null; + if (context.cacheStripeDetails) { + info = verifyCachedFileInfo(file); + } + new SplitGenerator(context, fs, file, info, isOriginal, deltas, + hasBase).schedule(); + } + + /** + * For each path, get the list of files and blocks that they consist of. + */ + @Override + public void run() { + try { + AcidUtils.Directory dirInfo = AcidUtils.getAcidState(dir, + context.conf, context.transactionList); + List deltas = + AcidUtils.serializeDeltas(dirInfo.getCurrentDirectories()); + Path base = dirInfo.getBaseDirectory(); + List original = dirInfo.getOriginalFiles(); + + boolean[] covered = new boolean[context.numBuckets]; + boolean isOriginal = base == null; + + // if we have a base to work from + if (base != null || !original.isEmpty()) { + + // find the base files (original or new style) + List children = original; + if (base != null) { + children = SHIMS.listLocatedStatus(fs, base, + AcidUtils.hiddenFileFilter); + } + + // for each child, schedule splits and mark off the bucket + for(FileStatus child: children) { + //update by tudou on 20200529, redmine = http://redmine.prod.dtstack.cn/issues/26286 + // make sure the file length on HDFS > 0(is not empty), otherwise an IndexOutOfBoundsException will be thrown in org.apache.hadoop.hive.ql.io.orc.ReaderImpl.extractMetaInfoFromFooter:362 + if(child.getLen() > 0){ + AcidOutputFormat.Options opts = AcidUtils.parseBaseBucketFilename + (child.getPath(), context.conf); + scheduleSplits(child, isOriginal, true, deltas); + int b = opts.getBucket(); + // If the bucket is in the valid range, mark it as covered. + // I wish Hive actually enforced bucketing all of the time. + if (b >= 0 && b < covered.length) { + covered[b] = true; + } + } + } + } + + // Generate a split for any buckets that weren't covered. + // This happens in the case where a bucket just has deltas and no + // base. + if (!deltas.isEmpty()) { + for (int b = 0; b < context.numBuckets; ++b) { + if (!covered[b]) { + synchronized (context.splits) { + context.splits.add(new OrcSplit(dir, b, 0, new String[0], null, + false, false, deltas)); + } + } + } + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } finally { + context.decrementSchedulers(); + } + } + + private FileInfo verifyCachedFileInfo(FileStatus file) { + context.numFilesCounter.incrementAndGet(); + FileInfo fileInfo = Context.footerCache.getIfPresent(file.getPath()); + if (fileInfo != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Info cached for path: " + file.getPath()); + } + if (fileInfo.modificationTime == file.getModificationTime() && + fileInfo.size == file.getLen()) { + // Cached copy is valid + context.cacheHitCounter.incrementAndGet(); + return fileInfo; + } else { + // Invalidate + Context.footerCache.invalidate(file.getPath()); + if (LOG.isDebugEnabled()) { + LOG.debug("Meta-Info for : " + file.getPath() + + " changed. CachedModificationTime: " + + fileInfo.modificationTime + ", CurrentModificationTime: " + + file.getModificationTime() + + ", CachedLength: " + fileInfo.size + ", CurrentLength: " + + file.getLen()); + } + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Info not cached for path: " + file.getPath()); + } + } + return null; + } + } + + /** + * Split the stripes of a given file into input splits. + * A thread is used for each file. + */ + static final class SplitGenerator implements Runnable { + private final Context context; + private final FileSystem fs; + private final FileStatus file; + private final long blockSize; + private final TreeMap locations; + private final FileInfo fileInfo; + private final boolean isOriginal; + private final List deltas; + private final boolean hasBase; + private List stripes; + private ReaderImpl.FileMetaInfo fileMetaInfo; + private Metadata metadata; + private List types; + private OrcFile.WriterVersion writerVersion; + + SplitGenerator(Context context, FileSystem fs, + FileStatus file, FileInfo fileInfo, + boolean isOriginal, + List deltas, + boolean hasBase) throws IOException { + this.context = context; + this.fs = fs; + this.file = file; + this.blockSize = file.getBlockSize(); + this.fileInfo = fileInfo; + locations = SHIMS.getLocationsWithOffset(fs, file); + this.isOriginal = isOriginal; + this.deltas = deltas; + this.hasBase = hasBase; + } + + /** + * Compute the number of bytes that overlap between the two ranges. + * @param offset1 start of range1 + * @param length1 length of range1 + * @param offset2 start of range2 + * @param length2 length of range2 + * @return the number of bytes in the overlap range + */ + static long getOverlap(long offset1, long length1, + long offset2, long length2) { + long end1 = offset1 + length1; + long end2 = offset2 + length2; + if (end2 <= offset1 || end1 <= offset2) { + return 0; + } else { + return Math.min(end1, end2) - Math.max(offset1, offset2); + } + } + + Path getPath() { + return file.getPath(); + } + + void schedule() throws IOException { + if(locations.size() == 1 && file.getLen() < context.maxSize) { + String[] hosts = locations.firstEntry().getValue().getHosts(); + synchronized (context.splits) { + context.splits.add(new OrcSplit(file.getPath(), 0, file.getLen(), + hosts, fileMetaInfo, isOriginal, hasBase, deltas)); + } + } else { + // if it requires a compute task + context.schedule(this); + } + } + + @Override + public String toString() { + return "splitter(" + file.getPath() + ")"; + } + + /** + * Create an input split over the given range of bytes. The location of the + * split is based on where the majority of the byte are coming from. ORC + * files are unlikely to have splits that cross between blocks because they + * are written with large block sizes. + * @param offset the start of the split + * @param length the length of the split + * @param fileMetaInfo file metadata from footer and postscript + * @throws IOException + */ + void createSplit(long offset, long length, + ReaderImpl.FileMetaInfo fileMetaInfo) throws IOException { + String[] hosts; + Map.Entry startEntry = locations.floorEntry(offset); + BlockLocation start = startEntry.getValue(); + if (offset + length <= start.getOffset() + start.getLength()) { + // handle the single block case + hosts = start.getHosts(); + } else { + Map.Entry endEntry = locations.floorEntry(offset + length); + BlockLocation end = endEntry.getValue(); + //get the submap + NavigableMap navigableMap = locations.subMap(startEntry.getKey(), + true, endEntry.getKey(), true); + // Calculate the number of bytes in the split that are local to each + // host. + Map sizes = new HashMap(); + long maxSize = 0; + for (BlockLocation block : navigableMap.values()) { + long overlap = getOverlap(offset, length, block.getOffset(), + block.getLength()); + if (overlap > 0) { + for(String host: block.getHosts()) { + LongWritable val = sizes.get(host); + if (val == null) { + val = new LongWritable(); + sizes.put(host, val); + } + val.set(val.get() + overlap); + maxSize = Math.max(maxSize, val.get()); + } + } else { + throw new IOException("File " + file.getPath().toString() + + " should have had overlap on block starting at " + block.getOffset()); + } + } + // filter the list of locations to those that have at least 80% of the + // max + long threshold = (long) (maxSize * MIN_INCLUDED_LOCATION); + List hostList = new ArrayList(); + // build the locations in a predictable order to simplify testing + for(BlockLocation block: navigableMap.values()) { + for(String host: block.getHosts()) { + if (sizes.containsKey(host)) { + if (sizes.get(host).get() >= threshold) { + hostList.add(host); + } + sizes.remove(host); + } + } + } + hosts = new String[hostList.size()]; + hostList.toArray(hosts); + } + synchronized (context.splits) { + context.splits.add(new OrcSplit(file.getPath(), offset, length, + hosts, fileMetaInfo, isOriginal, hasBase, deltas)); + } + } + + /** + * Divide the adjacent stripes in the file into input splits based on the + * block size and the configured minimum and maximum sizes. + */ + @Override + public void run() { + try { + populateAndCacheStripeDetails(); + + // figure out which stripes we need to read + boolean[] includeStripe = null; + // we can't eliminate stripes if there are deltas because the + // deltas may change the rows making them match the predicate. + if (deltas.isEmpty()) { + Reader.Options options = new Reader.Options(); + setIncludedColumns(options, types, context.conf, isOriginal); + setSearchArgument(options, types, context.conf, isOriginal); + // only do split pruning if HIVE-8732 has been fixed in the writer + if (options.getSearchArgument() != null && + writerVersion != OrcFile.WriterVersion.ORIGINAL) { + SearchArgument sarg = options.getSearchArgument(); + List sargLeaves = sarg.getLeaves(); + List stripeStats = metadata.getStripeStatistics(); + int[] filterColumns = RecordReaderImpl.mapSargColumns(sargLeaves, + options.getColumnNames(), getRootColumn(isOriginal)); + + if (stripeStats != null) { + // eliminate stripes that doesn't satisfy the predicate condition + includeStripe = new boolean[stripes.size()]; + for(int i=0; i < stripes.size(); ++i) { + includeStripe[i] = (i >= stripeStats.size()) || + isStripeSatisfyPredicate(stripeStats.get(i), sarg, + filterColumns); + if (LOG.isDebugEnabled() && !includeStripe[i]) { + LOG.debug("Eliminating ORC stripe-" + i + " of file '" + + file.getPath() + "' as it did not satisfy " + + "predicate condition."); + } + } + } + } + } + + // if we didn't have predicate pushdown, read everything + if (includeStripe == null) { + includeStripe = new boolean[stripes.size()]; + Arrays.fill(includeStripe, true); + } + + long currentOffset = -1; + long currentLength = 0; + int idx = -1; + for(StripeInformation stripe: stripes) { + idx++; + + if (!includeStripe[idx]) { + // create split for the previous unfinished stripe + if (currentOffset != -1) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + continue; + } + + // if we are working on a stripe, over the min stripe size, and + // crossed a block boundary, cut the input split here. + if (currentOffset != -1 && currentLength > context.minSize && + (currentOffset / blockSize != stripe.getOffset() / blockSize)) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + // if we aren't building a split, start a new one. + if (currentOffset == -1) { + currentOffset = stripe.getOffset(); + currentLength = stripe.getLength(); + } else { + currentLength = + (stripe.getOffset() + stripe.getLength()) - currentOffset; + } + if (currentLength >= context.maxSize) { + createSplit(currentOffset, currentLength, fileMetaInfo); + currentOffset = -1; + } + } + if (currentOffset != -1) { + createSplit(currentOffset, currentLength, fileMetaInfo); + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } finally { + context.decrementSchedulers(); + } + } + + private void populateAndCacheStripeDetails() { + try { + Reader orcReader; + if (fileInfo != null) { + stripes = fileInfo.stripeInfos; + fileMetaInfo = fileInfo.fileMetaInfo; + metadata = fileInfo.metadata; + types = fileInfo.types; + writerVersion = fileInfo.writerVersion; + // For multiple runs, in case sendSplitsInFooter changes + if (fileMetaInfo == null && context.footerInSplits) { + orcReader = OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(context.conf).filesystem(fs)); + fileInfo.fileMetaInfo = ((ReaderImpl) orcReader).getFileMetaInfo(); + fileInfo.metadata = orcReader.getMetadata(); + fileInfo.types = orcReader.getTypes(); + fileInfo.writerVersion = orcReader.getWriterVersion(); + } + } else { + orcReader = OrcFile.createReader(file.getPath(), + OrcFile.readerOptions(context.conf).filesystem(fs)); + stripes = orcReader.getStripes(); + metadata = orcReader.getMetadata(); + types = orcReader.getTypes(); + writerVersion = orcReader.getWriterVersion(); + fileMetaInfo = context.footerInSplits ? + ((ReaderImpl) orcReader).getFileMetaInfo() : null; + if (context.cacheStripeDetails) { + // Populate into cache. + Context.footerCache.put(file.getPath(), + new FileInfo(file.getModificationTime(), file.getLen(), stripes, + metadata, types, fileMetaInfo, writerVersion)); + } + } + } catch (Throwable th) { + if (!(th instanceof IOException)) { + LOG.error("Unexpected Exception", th); + } + synchronized (context.errors) { + context.errors.add(th); + } + if (!(th instanceof IOException)) { + context.notifyOnNonIOException(th); + } + } + } + + private boolean isStripeSatisfyPredicate(StripeStatistics stripeStatistics, + SearchArgument sarg, + int[] filterColumns) { + List predLeaves = sarg.getLeaves(); + TruthValue[] truthValues = new TruthValue[predLeaves.size()]; + for (int pred = 0; pred < truthValues.length; pred++) { + if (filterColumns[pred] != -1) { + + // column statistics at index 0 contains only the number of rows + ColumnStatistics stats = stripeStatistics.getColumnStatistics()[filterColumns[pred]]; + truthValues[pred] = RecordReaderImpl.evaluatePredicate(stats, predLeaves.get(pred)); + } else { + + // parition column case. + // partition filter will be evaluated by partition pruner so + // we will not evaluate partition filter here. + truthValues[pred] = TruthValue.YES_NO_NULL; + } + } + return sarg.evaluate(truthValues).isNeeded(); + } + } + + /** + * FileInfo. + * + * Stores information relevant to split generation for an ORC File. + * + */ + private static class FileInfo { + long modificationTime; + long size; + List stripeInfos; + ReaderImpl.FileMetaInfo fileMetaInfo; + Metadata metadata; + List types; + private OrcFile.WriterVersion writerVersion; + + + FileInfo(long modificationTime, long size, + List stripeInfos, + Metadata metadata, List types, + ReaderImpl.FileMetaInfo fileMetaInfo, + OrcFile.WriterVersion writerVersion) { + this.modificationTime = modificationTime; + this.size = size; + this.stripeInfos = stripeInfos; + this.fileMetaInfo = fileMetaInfo; + this.metadata = metadata; + this.types = types; + this.writerVersion = writerVersion; + } + } + + /** + * Return a RecordReader that is compatible with the Hive 0.12 reader + * with NullWritable for the key instead of RecordIdentifier. + */ + public static final class NullKeyRecordReader implements AcidRecordReader { + private final RecordIdentifier id; + private final RowReader inner; + + private NullKeyRecordReader(RowReader inner, Configuration conf) { + this.inner = inner; + id = inner.createKey(); + } + + public RecordIdentifier getRecordIdentifier() { + return id; + } + + @Override + public boolean next(NullWritable nullWritable, + OrcStruct orcStruct) throws IOException { + return inner.next(id, orcStruct); + } + + @Override + public NullWritable createKey() { + return NullWritable.get(); + } + + @Override + public OrcStruct createValue() { + return inner.createValue(); + } + + @Override + public long getPos() throws IOException { + return inner.getPos(); + } + + @Override + public void close() throws IOException { + inner.close(); + } + + @Override + public float getProgress() throws IOException { + return inner.getProgress(); + } + } + + +} diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java new file mode 100644 index 0000000000..6df5ceb002 --- /dev/null +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/test/com/dtstack/flinkx/hdfs/reader/HdfsOrcInputFormatTest.java @@ -0,0 +1,22 @@ +package com.dtstack.flinkx.hdfs.reader; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +/** + * @author jiangbo + * @date 2020/3/16 + */ +public class HdfsOrcInputFormatTest { + + @Test + public void testParseColumnAndType() { + HdfsOrcInputFormat hdfsOrcInputFormat = new HdfsOrcInputFormat(); + + String struct = "int,float(10,2),char(12)"; + List result = hdfsOrcInputFormat.parseColumnAndType(struct); + Assert.assertEquals(result.size(), 3); + } +} \ No newline at end of file diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 89f059b99d..ff3e37a7d6 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -98,6 +98,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java index 38ff34f667..f611487af5 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java @@ -43,6 +43,8 @@ */ public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { + private static final int FILE_NAME_PART_SIZE = 3; + protected int rowGroupSize; protected FileSystem fs; @@ -66,14 +68,29 @@ public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { protected Configuration conf; + protected boolean enableDictionary; + protected transient Map decimalColInfo; @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { + // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskmanager里同时认证kerberos + if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { + sleepRandomTime(); + } + initColIndices(); super.openInternal(taskNumber, numTasks); } + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long)(10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + @Override protected void checkOutputDir() { try{ @@ -120,7 +137,7 @@ protected void waitForActionFinishedBeforeWrite() { n++; } } catch (Exception e){ - + LOG.warn("Call method waitForActionFinishedBeforeWrite error", e); } } @@ -142,7 +159,7 @@ public boolean accept(Path path) { } String[] splits = fileName.split("\\."); - if (splits.length == 3) { + if (splits.length == FILE_NAME_PART_SIZE) { return Integer.parseInt(splits[2]) > fileIndex; } @@ -308,4 +325,8 @@ protected void moveAllTemporaryDataFileToDirectory() throws IOException { } } + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("HdfsWriter"); + } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 90f84054d5..a112570a66 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -34,7 +34,11 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.compress.*; +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.mapred.FileOutputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordWriter; @@ -213,17 +217,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe recordList.add(Integer.valueOf(rowData)); break; case BIGINT: - if (column instanceof Timestamp){ - column=((Timestamp) column).getTime(); - recordList.add(column); - break; - } - BigInteger data = new BigInteger(rowData); - if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ - recordList.add(data); - } else { - recordList.add(Long.valueOf(rowData)); - } + recordList.add(getBigint(column, rowData)); break; case FLOAT: recordList.add(Float.valueOf(rowData)); @@ -232,16 +226,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe recordList.add(Double.valueOf(rowData)); break; case DECIMAL: - ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(fullColumnNames.get(index)); - HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(rowData)); - hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); - if(hiveDecimal == null){ - String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", index, decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); - throw new WriteRecordException(msg, new IllegalArgumentException()); - } - - HiveDecimalWritable hiveDecimalWritable = new HiveDecimalWritable(hiveDecimal); - recordList.add(hiveDecimalWritable); + recordList.add(getDecimalWritable(index, rowData)); break; case STRING: case VARCHAR: @@ -270,6 +255,32 @@ private void getData(List recordList, int index, Row row) throws WriteRe } } + private Object getBigint(Object column, String rowData) { + if (column instanceof Timestamp){ + column = ((Timestamp) column).getTime(); + return column; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ + return data; + } else { + return Long.valueOf(rowData); + } + } + + private HiveDecimalWritable getDecimalWritable(int index, String rowData) throws WriteRecordException { + ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(fullColumnNames.get(index)); + HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(rowData)); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); + if(hiveDecimal == null){ + String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", index, decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + + return new HiveDecimalWritable(hiveDecimal); + } + @Override protected String recordConvertDetailErrorMessage(int pos, Row row) { return "\nHdfsOrcOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java index ec30366e16..0d8b40b951 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java @@ -83,6 +83,10 @@ public void setDefaultFs(String defaultFs) { format.defaultFs = defaultFs; } + public void setEnableDictionary(boolean enableDictionary) { + format.enableDictionary = enableDictionary; + } + @Override protected void checkFormat() { super.checkFormat(); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 7c4cd0e1fa..2dcb955c74 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -21,13 +21,13 @@ import com.dtstack.flinkx.enums.ColumnType; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.hdfs.ECompressType; +import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.ColumnTypeUtil; import com.dtstack.flinkx.util.DateUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.example.data.Group; @@ -38,12 +38,16 @@ import org.apache.parquet.hadoop.example.GroupWriteSupport; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.*; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; -import java.util.*; +import java.util.Date; +import java.util.HashMap; /** * The subclass of HdfsOutputFormat writing parquet files @@ -59,22 +63,8 @@ public class HdfsParquetOutputFormat extends BaseHdfsOutputFormat { private MessageType schema; - private static Calendar cal = Calendar.getInstance(); - - private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; - - private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; - private static ColumnTypeUtil.DecimalInfo PARQUET_DEFAULT_DECIMAL_INFO = new ColumnTypeUtil.DecimalInfo(10, 0); - static { - try { - cal.setTime(DateUtil.getDateFormatter().parse("1970-01-01")); - } catch (Exception e){ - throw new RuntimeException("Init calendar fail:",e); - } - } - @Override protected void openSource() throws IOException{ super.openSource(); @@ -101,6 +91,7 @@ protected void nextBlock(){ .withCompressionCodec(getCompressType()) .withConf(conf) .withType(schema) + .withDictionaryEncoding(enableDictionary) .withRowGroupSize(rowGroupSize); writer = builder.build(); @@ -162,7 +153,7 @@ public void writeSingleRecordToFile(Row row) throws WriteRecordException { try { for (; i < fullColumnNames.size(); i++) { Object valObj = row.getField(colIndices[i]); - if(valObj == null){ + if(valObj == null || valObj.toString().length() == 0){ continue; } @@ -233,7 +224,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ case "boolean" : group.add(colName,Boolean.parseBoolean(val));break; case "timestamp" : Timestamp ts = DateUtil.columnToTimestamp(valObj,null); - byte[] dst = longToByteArray(ts.getTime()); + byte[] dst = HdfsUtil.longToByteArray(ts.getTime()); group.add(colName, Binary.fromConstantByteArray(dst)); break; case "decimal" : @@ -246,7 +237,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ throw new WriteRecordException(msg, new IllegalArgumentException()); } - group.add(colName,decimalToBinary(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale())); + group.add(colName, HdfsUtil.decimalToBinary(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale())); break; case "date" : Date date = DateUtil.columnToDate(valObj,null); @@ -256,29 +247,6 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ } } - private Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec,int scale) { - byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); - - // Estimated number of bytes needed. - int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; - if (precToBytes == decimalBytes.length) { - // No padding needed. - return Binary.fromReusedByteArray(decimalBytes); - } - - byte[] tgt = new byte[precToBytes]; - if (hiveDecimal.signum() == -1) { - // For negative number, initializing bits to 1 - for (int i = 0; i < precToBytes; i++) { - tgt[i] |= 0xFF; - } - } - - // Padding leading zeroes/ones. - System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); - return Binary.fromReusedByteArray(tgt); - } - @Override protected String recordConvertDetailErrorMessage(int pos, Row row) { return "\nHdfsParquetOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; @@ -292,6 +260,7 @@ protected void closeSource() throws IOException { } private MessageType buildSchema(){ + decimalColInfo = new HashMap<>(16); Types.MessageTypeBuilder typeBuilder = Types.buildMessage(); for (int i = 0; i < fullColumnNames.size(); i++) { String name = fullColumnNames.get(i); @@ -317,10 +286,10 @@ private MessageType buildSchema(){ .as(OriginalType.DECIMAL) .precision(decimalInfo.getPrecision()) .scale(decimalInfo.getScale()) - .length(computeMinBytesForPrecision(decimalInfo.getPrecision())) + .length(HdfsUtil.computeMinBytesForPrecision(decimalInfo.getPrecision())) .named(name); - decimalColInfo = Collections.singletonMap(name, decimalInfo); + decimalColInfo.put(name, decimalInfo); } else { typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); } @@ -330,55 +299,4 @@ private MessageType buildSchema(){ return typeBuilder.named("Pair"); } - - private int computeMinBytesForPrecision(int precision){ - int numBytes = 1; - while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { - numBytes += 1; - } - return numBytes; - } - - private static byte[] longToByteArray(long data){ - long nano = data * 1000_000; - - int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); - byte[] julianDaysBytes = getBytes(julianDays); - flip(julianDaysBytes); - - long lastDayNanos = nano % NANO_SECONDS_PER_DAY; - byte[] lastDayNanosBytes = getBytes(lastDayNanos); - flip(lastDayNanosBytes); - - byte[] dst = new byte[12]; - - System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); - System.arraycopy(julianDaysBytes, 0, dst, 8, 4); - - return dst; - } - - private static byte[] getBytes(long i) { - byte[] bytes=new byte[8]; - bytes[0]=(byte)((i >> 56) & 0xFF); - bytes[1]=(byte)((i >> 48) & 0xFF); - bytes[2]=(byte)((i >> 40) & 0xFF); - bytes[3]=(byte)((i >> 32) & 0xFF); - bytes[4]=(byte)((i >> 24) & 0xFF); - bytes[5]=(byte)((i >> 16) & 0xFF); - bytes[6]=(byte)((i >> 8) & 0xFF); - bytes[7]=(byte)(i & 0xFF); - return bytes; - } - - /** - * @param bytes - */ - private static void flip(byte[] bytes) { - for(int i=0,j=bytes.length-1;i writeData(DataStream dataSet) { builder.setRestoreConfig(restoreConfig); builder.setMaxFileSize(maxFileSize); builder.setFlushBlockInterval(flushInterval); + builder.setEnableDictionary(enableDictionary); return createOutput(dataSet, builder.finish()); } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java index 0de6b79e84..33917f375d 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/TimePartitionFormat.java @@ -29,9 +29,9 @@ public class TimePartitionFormat { - private static final long CONSTANT_TWO_DAY_TIME = 1000 * 60 * 60 * 24 * 2; - private static final long CONSTANT_TWO_HOUR_TIME = 1000 * 60 * 60 * 2; - private static final long CONSTANT_TWO_MINUTE_TIME = 1000 * 60 * 2; + private static final long CONSTANT_TWO_DAY_TIME = 1000 * 60 * 60 * 24 * 2L; + private static final long CONSTANT_TWO_HOUR_TIME = 1000 * 60 * 60 * 2L; + private static final long CONSTANT_TWO_MINUTE_TIME = 1000 * 60 * 2L; private static PartitionEnum partitionEnum; diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java index 21cb769d04..10d49b992c 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AbstractHiveMetadataParser.java @@ -25,7 +25,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hive.EStoreType.*; +import static com.dtstack.flinkx.hive.EStoreType.ORC; +import static com.dtstack.flinkx.hive.EStoreType.PARQUET; +import static com.dtstack.flinkx.hive.EStoreType.TEXT; /** * @author jiangbo diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java index 2c36ff3c74..4cb1fc6d09 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveDbUtil.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.authenticate.KerberosUtil; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.RetryUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.collections.MapUtils; @@ -55,6 +56,8 @@ public final class HiveDbUtil { public static final String SQLSTATE_CANNOT_ACQUIRE_CONNECT = "08004"; + public static final int JDBC_PART_SIZE = 2; + public static final String JDBC_REGEX = "[\\?|;|#]"; public static final String KEY_VAL_DELIMITER = "="; public static final String PARAM_DELIMITER = "&"; @@ -103,7 +106,7 @@ private static Connection getConnectionWithKerberos(ConnectionInfo connectionInf String keytabFileName = KerberosUtil.getPrincipalFileName(connectionInfo.getHiveConf()); keytabFileName = KerberosUtil.loadFile(connectionInfo.getHiveConf(), keytabFileName); - String principal = KerberosUtil.findPrincipalFromKeytab(keytabFileName); + String principal = KerberosUtil.getPrincipal(connectionInfo.getHiveConf(), keytabFileName); KerberosUtil.loadKrb5Conf(connectionInfo.getHiveConf()); Configuration conf = FileSystemUtil.getConfiguration(connectionInfo.getHiveConf(), null); @@ -126,7 +129,7 @@ public Connection run(){ private static boolean openKerberos(final String jdbcUrl){ String[] splits = jdbcUrl.split(JDBC_REGEX); - if (splits.length != 2) { + if (splits.length != JDBC_PART_SIZE) { return false; } @@ -232,10 +235,10 @@ private static Connection getHiveConnection(String url, Properties prop) throws url = String.format("jdbc:hive2://%s:%s/%s", host, port, param); Connection connection = DriverManager.getConnection(url, prop); if (StringUtils.isNotEmpty(db)) { - try { - connection.createStatement().execute("use " + db); + try (Statement statement = connection.createStatement()) { + statement.execute("use " + db); } catch (SQLException e) { - if (connection != null) { + if (null != connection) { connection.close(); } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java index 7e0c0193fa..75209ebdfa 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/HiveUtil.java @@ -28,7 +28,8 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.hive.EStoreType.*; +import static com.dtstack.flinkx.hive.EStoreType.ORC; +import static com.dtstack.flinkx.hive.EStoreType.TEXT; /** * @author toutian @@ -52,26 +53,27 @@ public class HiveUtil { private HiveDbUtil.ConnectionInfo connectionInfo; - enum HiveReleaseVersion{ - /** - * apache hive 1.x - */ - APACHE_1, - - /** - * apache hive 2.x - */ - APACHE_2, - - /** - * cdh hive 1.x - */ - CDH_1, - - /** - * cdh hive 2.x - */ - CDH_2 + public static String getHiveColumnType(String originType) { + originType = originType.trim(); + int indexOfBrackets = originType.indexOf(LEFT_BRACKETS); + if (indexOfBrackets > -1) { + String params = originType.substring(indexOfBrackets); + int index = params.indexOf(","); + int right = Integer.parseInt(params.substring(index+1, params.length()-1).trim()); + if(right == 0){ + int left = Integer.parseInt(params.substring(1, index).trim()); + if(left <= 4){ + return "SMALLINT"; + }else if(left <= 9){ + return "INT"; + }else if(left <= 18){ + return "BIGINT"; + } + } + return "DECIMAL" + params; + } else { + return convertType(originType); + } } public HiveUtil() { @@ -176,70 +178,6 @@ private AbstractHiveMetadataParser getMetadataParser(HiveReleaseVersion hiveVers } } - public HiveReleaseVersion getHiveVersion(Connection connection){ - HiveReleaseVersion version = HiveReleaseVersion.APACHE_2; - try { - ResultSet resultSet = connection.createStatement().executeQuery("select version()"); - if (resultSet.next()) { - String versionMsg = resultSet.getString(1); - if (versionMsg.contains("cdh")){ - // 结果示例:2.1.1-cdh6.3.1 re8d55f408b4f9aa2648bc9e34a8f802d53d6aab3 - if (versionMsg.startsWith("2")) { - version = HiveReleaseVersion.CDH_2; - } else if(versionMsg.startsWith("1")){ - version = HiveReleaseVersion.CDH_1; - } - } else { - // FIXME spark thrift server不支持 version()函数,所以使用默认的版本 - } - } - } catch (Exception ignore) { - } - - return version; - } - - public static String getCreateTableHql(TableInfo tableInfo) { - //不要使用create table if not exist,可能以后会在业务逻辑中判断表是否已经存在 - StringBuilder fieldsb = new StringBuilder("CREATE TABLE %s ("); - for (int i = 0; i < tableInfo.getColumns().size(); i++) { - fieldsb.append(String.format("`%s` %s", tableInfo.getColumns().get(i), tableInfo.getColumnTypes().get(i))); - if (i != tableInfo.getColumns().size() - 1) { - fieldsb.append(","); - } - } - fieldsb.append(") "); - if (!tableInfo.getPartitions().isEmpty()) { - fieldsb.append(" PARTITIONED BY ("); - for (String partitionField : tableInfo.getPartitions()) { - fieldsb.append(String.format("`%s` string", partitionField)); - } - fieldsb.append(") "); - } - if (TEXT.name().equalsIgnoreCase(tableInfo.getStore())) { - fieldsb.append(" ROW FORMAT DELIMITED FIELDS TERMINATED BY '"); - fieldsb.append(tableInfo.getDelimiter()); - fieldsb.append("' LINES TERMINATED BY '\\n' STORED AS TEXTFILE "); - } else if(ORC.name().equalsIgnoreCase(tableInfo.getStore())) { - fieldsb.append(" STORED AS ORC "); - }else{ - fieldsb.append(" STORED AS PARQUET "); - } - return fieldsb.toString(); - } - - public static String getHiveColumnType(String originType) { - originType = originType.trim(); - int indexOfBrackets = originType.indexOf(LEFT_BRACKETS); - if (indexOfBrackets > -1) { - String type = originType.substring(0, indexOfBrackets); - String params = originType.substring(indexOfBrackets); - return convertType(type) + params; - } else { - return convertType(originType); - } - } - private static String convertType(String type) { switch (type.toUpperCase()) { case "BIT": @@ -258,6 +196,8 @@ private static String convertType(String type) { case "INT8": type = "INT"; break; + case "NUMERIC": + case "NUMBER": case "BIGINT": type = "BIGINT"; break; @@ -272,8 +212,6 @@ private static String convertType(String type) { case "BINARY_DOUBLE": type = "DOUBLE"; break; - case "NUMERIC": - case "NUMBER": case "DECIMAL": type = "DECIMAL"; break; @@ -310,4 +248,102 @@ private static String convertType(String type) { } return type; } + + public static String getCreateTableHql(TableInfo tableInfo) { + //不要使用create table if not exist,可能以后会在业务逻辑中判断表是否已经存在 + StringBuilder fieldsb = new StringBuilder("CREATE TABLE %s ("); + for (int i = 0; i < tableInfo.getColumns().size(); i++) { + fieldsb.append(String.format("`%s` %s", tableInfo.getColumns().get(i), tableInfo.getColumnTypes().get(i))); + if (i != tableInfo.getColumns().size() - 1) { + fieldsb.append(","); + } + } + fieldsb.append(") "); + if (!tableInfo.getPartitions().isEmpty()) { + fieldsb.append(" PARTITIONED BY ("); + for (String partitionField : tableInfo.getPartitions()) { + fieldsb.append(String.format("`%s` string", partitionField)); + } + fieldsb.append(") "); + } + if (TEXT.name().equalsIgnoreCase(tableInfo.getStore())) { + fieldsb.append(" ROW FORMAT DELIMITED FIELDS TERMINATED BY '"); + fieldsb.append(tableInfo.getDelimiter()); + fieldsb.append("' LINES TERMINATED BY '\\n' STORED AS TEXTFILE "); + } else if(ORC.name().equalsIgnoreCase(tableInfo.getStore())) { + fieldsb.append(" STORED AS ORC "); + }else{ + fieldsb.append(" STORED AS PARQUET "); + } + return fieldsb.toString(); + } + + public HiveReleaseVersion getHiveVersion(Connection connection){ + HiveReleaseVersion version = HiveReleaseVersion.APACHE_2; + try (ResultSet resultSet = connection.createStatement().executeQuery("select version()")) { + if (resultSet.next()) { + String versionMsg = resultSet.getString(1); + if (versionMsg.contains(HiveReleaseVersion.CDH_1.getName())){ + // 结果示例:2.1.1-cdh6.3.1 re8d55f408b4f9aa2648bc9e34a8f802d53d6aab3 + if (versionMsg.startsWith(HiveReleaseVersion.CDH_2.getVersion())) { + version = HiveReleaseVersion.CDH_2; + } else if(versionMsg.startsWith(HiveReleaseVersion.CDH_1.getVersion())){ + version = HiveReleaseVersion.CDH_1; + } + } else { + //spark thrift server不支持 version()函数,所以使用默认的版本 + } + } + } catch (Exception ignore) { + } + + return version; + } + + enum HiveReleaseVersion{ + /** + * apache hive 1.x + */ + APACHE_1("apache", "1"), + + /** + * apache hive 2.x + */ + APACHE_2("apache", "2"), + + /** + * cdh hive 1.x + */ + CDH_1("cdh", "1"), + + /** + * cdh hive 2.x + */ + CDH_2("cdh", "2"); + + private String name; + + private String version; + + HiveReleaseVersion(String name, String version) { + this.name = name; + this.version = version; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + } } diff --git a/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java b/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java deleted file mode 100644 index 0b7c2e533d..0000000000 --- a/flinkx-hive/flinkx-hive-core/src/test/java/com/dtstack/flinkx/hive/test/DBUtilTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package com.dtstack.flinkx.hive.test; - -import com.dtstack.flinkx.hive.util.HiveDbUtil; - -import java.sql.Connection; -import java.sql.ResultSet; -import java.util.HashMap; -import java.util.Map; - -/** - * @author jiangbo - * @date 2019/8/29 - */ -public class DBUtilTest { - - public static void main(String[] args) throws Exception{ - Map sftpConf = new HashMap<>(); - sftpConf.put("host", "172.16.10.79"); - sftpConf.put("port", "22"); - sftpConf.put("username", "root"); - sftpConf.put("password", "abc123"); - - Map hiveConf = new HashMap<>(); - hiveConf.put("hive.server2.authentication.kerberos.principal", "hive/cdh02@HADOOP.COM"); - hiveConf.put("hive.server2.authentication.kerberos.keytab", "D:\\cdh_cluster\\hive.keytab"); - hiveConf.put("java.security.krb5.conf", "D:\\cdh_cluster\\krb5.conf"); - hiveConf.put("useLocalFile", "true"); - hiveConf.put("sftpConf", sftpConf); - hiveConf.put("remoteDir", "/home/sftp/keytab/jiangbo"); - - HiveDbUtil.ConnectionInfo connectionInfo = new HiveDbUtil.ConnectionInfo(); - connectionInfo.setJdbcUrl("jdbc:hive2://172.16.10.75:10000/default;principal=hive/cdh02@HADOOP.COM"); - connectionInfo.setUsername(""); - connectionInfo.setPassword(""); - connectionInfo.setHiveConf(hiveConf); - - Connection connection = HiveDbUtil.getConnection(connectionInfo); - ResultSet rs = connection.createStatement().executeQuery("show tables"); - while (rs.next()) { - System.out.println(rs.getObject(2)); - } - - connection.close(); - } -} diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 25357e479f..e93421f32a 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -92,6 +92,16 @@ under the License. + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java index 217163c261..67731f0503 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java @@ -155,7 +155,7 @@ public FormatState getFormatState() { } private Map flushOutputFormat() { - Map formatStateMap = new HashMap<>(); + Map formatStateMap = new HashMap<>(outputFormats.size()); Iterator> entryIterator = outputFormats.entrySet().iterator(); while (entryIterator.hasNext()) { Map.Entry entry = entryIterator.next(); @@ -178,6 +178,7 @@ private Map flushOutputFormat() { @Override protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("HiveWriter"); } @Override @@ -187,6 +188,11 @@ public void writeRecord(Row row) throws IOException { Map event = null; if (row.getField(0) instanceof Map) { event = (Map) row.getField(0); + + if (null != event && event.containsKey("message")) { + event = MapUtils.getMap(event, "message"); + } + tablePath = PathConverterUtil.regaxByRules(event, tableBasePath, distributeTableMapping); fromLogData = true; } else { @@ -210,11 +216,11 @@ public void writeRecord(Row row) throws IOException { //row包含map嵌套的数据内容和channel, 而rowData是非常简单的纯数据,此处补上数据差额 if (fromLogData && bytesWriteCounter != null) { - bytesWriteCounter.add(row.toString().length() - rowData.toString().length()); + bytesWriteCounter.add((long)row.toString().length() - rowData.toString().length()); } } catch (Exception e) { // 写入产生的脏数据已经由hdfsOutputFormat处理了,这里不用再处理了,只打印日志 - if (numWriteCounter.getLocalValue() % 1000 == 0) { + if (numWriteCounter.getLocalValue() % LOG_PRINT_INTERNAL == 0) { LOG.warn("写入hdfs异常:", e); } } diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java index f8c69886b1..529d8d743b 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java @@ -131,9 +131,12 @@ protected void checkFormat() { if (this.format.tableBasePath == null || this.format.tableBasePath.length() == 0) { throw new IllegalArgumentException("No tableBasePath supplied."); } + if (this.format.tableInfos.isEmpty()){ throw new IllegalArgumentException("No tableInfos supplied."); } + + notSupportBatchWrite("HiveWriter"); } } diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java index 5168b850a6..a3483caaea 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java @@ -19,31 +19,34 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.hive.TableInfo; import com.dtstack.flinkx.hive.TimePartitionFormat; import com.dtstack.flinkx.hive.util.HiveUtil; import com.dtstack.flinkx.writer.BaseDataWriter; import com.dtstack.flinkx.writer.WriteMode; -import com.google.gson.Gson; +import com.google.gson.internal.LinkedTreeMap; +import com.google.gson.reflect.TypeToken; import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; import parquet.hadoop.ParquetWriter; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import static com.dtstack.flinkx.hdfs.HdfsConfigKeys.KEY_ROW_GROUP_SIZE; import static com.dtstack.flinkx.hive.HiveConfigKeys.*; +import static com.dtstack.flinkx.util.GsonUtil.GSON; /** * @author toutian */ public class HiveWriter extends BaseDataWriter { + private String readerName; + private String defaultFs; private String fileType; @@ -80,10 +83,9 @@ public class HiveWriter extends BaseDataWriter { private boolean autoCreateTable; - private Gson gson = new Gson(); - public HiveWriter(DataTransferConfig config) { super(config); + readerName = config.getJob().getContent().get(0).getReader().getName(); WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); hadoopConfig = (Map) writerConfig.getParameter().getVal(KEY_HADOOP_CONFIG); defaultFs = writerConfig.getParameter().getStringVal(KEY_DEFAULT_FS); @@ -95,9 +97,9 @@ public HiveWriter(DataTransferConfig config) { partition = writerConfig.getParameter().getStringVal(KEY_PARTITION, "pt"); delimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER, "\u0001"); charSet = writerConfig.getParameter().getStringVal(KEY_CHARSET_NAME); - maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, 1024 * 1024 * 1024); + maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); - bufferSize = writerConfig.getParameter().getLongVal(KEY_BUFFER_SIZE, 128 * 1024 * 1024); + bufferSize = writerConfig.getParameter().getLongVal(KEY_BUFFER_SIZE, 128 * ConstantValue.STORE_SIZE_M); rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE, WriteMode.APPEND.name()); @@ -120,18 +122,18 @@ public HiveWriter(DataTransferConfig config) { } } + /** + * 分表的映射关系 + * distributeTableMapping 的数据结构为 + * tableInfos的数据结构为 + */ private void formatHiveDistributeInfo(String distributeTable) { - /** - * 分表的映射关系 - * distributeTableMapping 的数据结构为 - * tableInfos的数据结构为 - */ distributeTableMapping = new HashMap<>(32); if (StringUtils.isNotBlank(distributeTable)) { - Map distributeTableMap = gson.fromJson(distributeTable, Map.class); - for (Map.Entry entry : distributeTableMap.entrySet()) { + Map> distributeTableMap = GSON.fromJson(distributeTable, new TypeToken>>(){}.getType()); + for (Map.Entry> entry : distributeTableMap.entrySet()) { String groupName = entry.getKey(); - List groupTables = (List) entry.getValue(); + List groupTables = entry.getValue(); for (String tableName : groupTables) { distributeTableMapping.put(tableName, groupName); } @@ -142,10 +144,12 @@ private void formatHiveDistributeInfo(String distributeTable) { private void formatHiveTableInfo(String tablesColumn) { tableInfos = new HashMap<>(16); if (StringUtils.isNotEmpty(tablesColumn)) { - Map tableColumnMap = gson.fromJson(tablesColumn, Map.class); - for (Map.Entry entry : tableColumnMap.entrySet()) { + Map>> tableColumnMap = GSON.fromJson(tablesColumn, new TypeToken> >>(){}.getType()); + List> extraTableColumnList = getExtraTableColumn(); + for (Map.Entry>> entry : tableColumnMap.entrySet()) { String tableName = entry.getKey(); - List> tableColumns = (List>) entry.getValue(); + List> tableColumns = entry.getValue(); + tableColumns.addAll(extraTableColumnList); TableInfo tableInfo = new TableInfo(tableColumns.size()); tableInfo.addPartition(partition); tableInfo.setDelimiter(delimiter); @@ -162,6 +166,32 @@ private void formatHiveTableInfo(String tablesColumn) { } } + /** + * 增加hive表字段 + */ + @SuppressWarnings("unchecked") + private List> getExtraTableColumn(){ + if(StringUtils.equalsIgnoreCase(readerName, "oraclelogminerreader")){ + List> list = new ArrayList<>(2); + Map opTime = new LinkedTreeMap<>(); + opTime.put("type", "BIGINT"); + opTime.put("key", "opTime"); + opTime.put("comment", ""); + + Map scn = new LinkedTreeMap<>(); + scn.put("type", "BIGINT"); + scn.put("key", "scn"); + scn.put("comment", ""); + + list.add(opTime); + list.add(scn); + + return list; + }else{ + return Collections.EMPTY_LIST; + } + } + @Override public DataStreamSink writeData(DataStream dataSet) { HiveOutputFormatBuilder builder = new HiveOutputFormatBuilder(); diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index d6cd16c26f..d9a711a6ab 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java index 61b483abd0..95016e4956 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java @@ -16,7 +16,7 @@ */ package com.dtstack.flinkx.kafka.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java index f209be623c..7f4c37f576 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java @@ -15,6 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + + package com.dtstack.flinkx.kafka.reader; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index b62d13da11..dad9361a4f 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index 980db83475..8c3e855d9c 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java index 8ad201b782..9633e6e4ad 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java @@ -16,7 +16,7 @@ */ package com.dtstack.flinkx.kafka09.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index cea09d8368..42fb23d6b2 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 8fbbdb883d..31467b287f 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java index 69c5747cf0..f0d10d6f13 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java @@ -17,7 +17,7 @@ */ package com.dtstack.flinkx.kafka10.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index f28e37fe50..4d86dbb7d0 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -39,6 +39,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index 4fa02eb7e5..ae62ee9515 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java index f27889ab25..2e3d2d569f 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java @@ -1,6 +1,6 @@ package com.dtstack.flinkx.kafka11.reader; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; +import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.kafkabase.reader.IClient; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java index 0424bb2801..5a7253860f 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java @@ -15,6 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + + package com.dtstack.flinkx.kafka11.reader; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index fe3808dd63..22d37d38b4 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -34,6 +34,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java index ec610798dc..38983693fb 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java @@ -18,10 +18,11 @@ package com.dtstack.flinkx.kafkabase.reader; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.decoder.DecodeEnum; +import com.dtstack.flinkx.decoder.IDecode; +import com.dtstack.flinkx.decoder.JsonDecoder; +import com.dtstack.flinkx.decoder.PlainDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.kafkabase.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.decoder.JsonDecoder; -import com.dtstack.flinkx.kafkabase.decoder.PlainDecoder; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.StringUtil; @@ -66,7 +67,7 @@ public void openInputFormat() throws IOException { super.openInputFormat(); queue = new SynchronousQueue<>(false); - if ("json".equals(codec)) { + if (DecodeEnum.JSON.getName().equalsIgnoreCase(codec)) { decode = new JsonDecoder(); } else { decode = new PlainDecoder(); diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java index 7f3739089a..4e4f20cf02 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java @@ -28,7 +28,11 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.*; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_BLANK_IGNORE; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CODEC; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CONSUMER_SETTINGS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_GROUP_ID; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TOPIC; /** * Date: 2019/11/21 diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java index 482c1ca40a..6b338a13ad 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java @@ -18,8 +18,8 @@ package com.dtstack.flinkx.kafkabase.writer; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.decoder.JsonDecoder; import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.kafkabase.decoder.JsonDecoder; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.flink.configuration.Configuration; @@ -29,7 +29,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.*; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; /** * Date: 2019/11/21 @@ -100,7 +104,7 @@ public void closeInternal() throws IOException { @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new UnsupportedOperationException(); + notSupportBatchWrite("KafkaWriter"); } @Override diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java index d8823c01f0..3ae4c05c7f 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java @@ -27,7 +27,10 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.*; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_PRODUCER_SETTINGS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TABLE_FIELDS; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TIMEZONE; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TOPIC; /** * Date: 2019/11/21 diff --git a/flinkx-kudu/flinkx-kudu-core/pom.xml b/flinkx-kudu/flinkx-kudu-core/pom.xml index abafab1604..e419766ddc 100644 --- a/flinkx-kudu/flinkx-kudu-core/pom.xml +++ b/flinkx-kudu/flinkx-kudu-core/pom.xml @@ -24,11 +24,5 @@ 1.6 provided - - junit - junit - 4.12 - test - \ No newline at end of file diff --git a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java index 928580a166..4eb6ec865a 100644 --- a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java +++ b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java @@ -26,13 +26,22 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.kudu.ColumnSchema; import org.apache.kudu.Type; -import org.apache.kudu.client.*; +import org.apache.kudu.client.AsyncKuduClient; +import org.apache.kudu.client.AsyncKuduScanner; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduPredicate; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduTable; import java.io.IOException; import java.math.BigDecimal; import java.security.PrivilegedExceptionAction; import java.sql.Timestamp; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index c8a99fea22..f9a1580d71 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -49,8 +49,12 @@ shade.kudureader.io.netty - com.google - shade.kudureader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java index f0ddce2857..6be5434a52 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java @@ -26,7 +26,11 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.apache.kudu.Type; -import org.apache.kudu.client.*; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduScanToken; +import org.apache.kudu.client.KuduScanner; +import org.apache.kudu.client.RowResult; +import org.apache.kudu.client.RowResultIterator; import java.io.IOException; import java.util.List; diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java index 48d0e43fc5..4959731c76 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.kudu.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.kudu.core.KuduConfig; import com.dtstack.flinkx.reader.MetaColumn; @@ -50,5 +51,9 @@ protected void checkFormat() { if (format.columns == null || format.columns.size() == 0){ throw new IllegalArgumentException("columns can not be empty"); } + + if (format.kuduConfig.getBatchSizeBytes() > ConstantValue.STORE_SIZE_G) { + throw new IllegalArgumentException("批量读取字节数必须小于[1G]"); + } } } diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java index 7b09c9048d..604b920a6e 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java @@ -32,7 +32,19 @@ import java.util.List; -import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.*; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BATCH_SIZE_BYTES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BOSS_COUNT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_FILTER; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_KEYTABFILE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_MASTER_ADDRESSES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_PRINCIPAL; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_QUERY_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_READ_MODE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_WORKER_COUNT; /** * @author jiangbo diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index 742fae63ab..256505e5ab 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -49,8 +49,12 @@ shade.kuduwriter.io.netty - com.google - shade.kuduwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java index df7a521186..69f346dcba 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java @@ -28,7 +28,12 @@ import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.types.Row; -import org.apache.kudu.client.*; +import org.apache.kudu.client.KuduClient; +import org.apache.kudu.client.KuduException; +import org.apache.kudu.client.KuduSession; +import org.apache.kudu.client.KuduTable; +import org.apache.kudu.client.Operation; +import org.apache.kudu.client.SessionConfiguration; import java.io.IOException; import java.util.List; diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java index 265e5610c2..2cb5532262 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java @@ -32,7 +32,16 @@ import java.util.List; -import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.*; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_BOSS_COUNT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_FLUSH_MODE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_KEYTABFILE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_MASTER_ADDRESSES; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_OPERATION_TIMEOUT; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_PRINCIPAL; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_TABLE; +import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_WORKER_COUNT; /** * @author jiangbo diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java index 540db38056..d287c44a70 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClassLoaderType.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.launcher; /** diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index 1b823c77eb..c5318c00eb 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -40,8 +40,6 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.util.ConverterUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; import java.net.InetSocketAddress; @@ -97,7 +95,7 @@ public static ClusterClient createYarnClient(Options launcherOptions) { ApplicationId applicationId; if (StringUtils.isEmpty(launcherOptions.getAppId())) { - applicationId = getAppIdFromYarn(yarnClient); + applicationId = getAppIdFromYarn(yarnClient, launcherOptions); if(applicationId != null && StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } @@ -156,7 +154,7 @@ private static AbstractYarnClusterDescriptor getClusterDescriptor(Options launch return yarnClusterDescriptor; } - private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exception{ + private static ApplicationId getAppIdFromYarn(YarnClient yarnClient, Options launcherOptions) throws Exception{ Set set = new HashSet<>(); set.add("Apache Flink"); EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); @@ -175,6 +173,10 @@ private static ApplicationId getAppIdFromYarn(YarnClient yarnClient) throws Exce continue; } + if(!report.getQueue().equals(launcherOptions.getQueue())) { + continue; + } + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 91a4bd1409..c5c19fc9f4 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -25,6 +25,7 @@ import com.dtstack.flinkx.launcher.perjob.PerJobSubmitter; import com.dtstack.flinkx.options.OptionParser; import com.dtstack.flinkx.options.Options; +import com.dtstack.flinkx.util.JsonModifyUtil; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; @@ -41,6 +42,7 @@ import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; /** @@ -94,8 +96,25 @@ public static void main(String[] args) throws Exception { String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + + // 将argList转化为HashMap,方便通过参数名称来获取参数值 + HashMap temp = new HashMap<>(16); + for (int i = 0; i < argList.size(); i += 2) { + temp.put(argList.get(i), argList.get(i + 1)); + } + // 对json中的值进行修改 + HashMap parameter = JsonModifyUtil.CommandTransform(temp.get("-p")); + temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); + + // 清空list,填充修改后的参数值 + argList.clear(); + for (int i = 0; i < temp.size(); i++) { + argList.add(temp.keySet().toArray()[i].toString()); + argList.add(temp.values().toArray()[i].toString()); + } + if(mode.equals(ClusterMode.local.name())) { - String[] localArgs = argList.toArray(new String[argList.size()]); + String[] localArgs = argList.toArray(new String[0]); com.dtstack.flinkx.Main.main(localArgs); } else { String pluginRoot = launcherOptions.getPluginRoot(); @@ -237,9 +256,8 @@ public boolean accept(File dir, String name) { } private static String readJob(String job) { - try { - File file = new File(job); - FileInputStream in = new FileInputStream(file); + File file = new File(job); + try (FileInputStream in = new FileInputStream(file)) { byte[] fileContent = new byte[(int) file.length()]; in.read(fileContent); in.close(); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java index 6707c8f138..b88eb078b8 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/YarnConfLoader.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.launcher; +import com.dtstack.flinkx.constants.ConstantValue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -42,7 +43,7 @@ public static YarnConfiguration getYarnConf(String yarnConfDir) { if(dir.exists() && dir.isDirectory()) { File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { - if(name.endsWith(".xml")){ + if(name.endsWith(ConstantValue.FILE_SUFFIX_XML)){ return true; } return false; diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java index 1d64181e81..508098067d 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java @@ -52,10 +52,10 @@ public static ClusterSpecification createClusterSpecification(Properties conProp int slotsPerTaskManager = 1; if(conProp != null){ - if(conProp.contains(JOBMANAGER_MEMORY_MB)){ + if(conProp.containsKey(JOBMANAGER_MEMORY_MB)){ jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); } - if(conProp.contains(TASKMANAGER_MEMORY_MB)){ + if(conProp.containsKey(TASKMANAGER_MEMORY_MB)){ taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index d9025e2ed9..81c6387395 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -21,7 +21,7 @@ import com.dtstack.flinkx.launcher.ClassLoaderType; import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; -import com.dtstack.flinkx.util.MapUtil; +import com.dtstack.flinkx.util.GsonUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; @@ -64,7 +64,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) Configuration flinkConfig = StringUtils.isEmpty(options.getFlinkconf()) ? new Configuration() : GlobalConfiguration.loadConfiguration(options.getFlinkconf()); flinkConfig.setString("classloader.resolve-order", "child-first"); - Properties conProp = MapUtil.jsonStrToObject(options.getConfProp(), Properties.class); + Properties conProp = GsonUtil.GSON.fromJson(options.getConfProp(), Properties.class); ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); clusterSpecification.setCreateProgramDelay(true); clusterSpecification.setConfiguration(flinkConfig); diff --git a/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java b/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java deleted file mode 100644 index b974511769..0000000000 --- a/flinkx-launcher/src/test/java/bigdata/FlinkConfigTest.java +++ /dev/null @@ -1,19 +0,0 @@ -package bigdata; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.JobManagerOptions; - -import java.io.File; - -/** - * Created by softfly on 18/4/24. - */ -public class FlinkConfigTest { - public static void main(String[] args) { - //Configuration config = GlobalConfiguration.loadConfiguration("/hadoop/flink-1.4.0/conf"); - //System.out.println(config.getString(JobManagerOptions.ADDRESS)); - String msg = "xxx" + File.separator; - System.out.println(msg); - } -} diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java index 26438d51fe..b5d78ffecc 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbClientUtil.java @@ -19,7 +19,13 @@ package com.dtstack.flinkx.mongodb; -import com.mongodb.*; +import com.mongodb.AuthenticationMechanism; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientOptions; +import com.mongodb.MongoClientURI; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import com.mongodb.WriteConcern; import com.mongodb.client.MongoCursor; import org.apache.commons.lang.StringUtils; import org.bson.Document; diff --git a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java index 425dc21aa4..c419a2672e 100644 --- a/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java +++ b/flinkx-mongodb/flinkx-mongodb-core/src/main/java/com/dtstack/flinkx/mongodb/MongodbUtil.java @@ -29,9 +29,11 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.Arrays; +import java.util.Date; +import java.util.List; -import static com.dtstack.flinkx.enums.ColumnType.*; +import static com.dtstack.flinkx.enums.ColumnType.getType; /** * Utilities for mongodb database connection and data format conversion diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 09814f6c3f..a9caa60251 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -56,8 +56,12 @@ shade.mongodbreader.io.netty - com.google - shade.mongodbreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index 3f3ec2d863..ac8e65d705 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.mongodb.MongodbClientUtil; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.StringUtil; import com.mongodb.BasicDBObject; import com.mongodb.MongoClient; @@ -37,7 +38,8 @@ import org.bson.conversions.Bson; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.List; /** * Read plugin for reading static data @@ -149,6 +151,9 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti if(size * minNumSplits < docNum){ splits.add(new MongodbInputSplit((int)(size * minNumSplits), (int)(docNum - size * minNumSplits))); } + } catch (Exception e){ + LOG.error("error to create inputSplits, e = {}", ExceptionUtil.getErrorMessage(e)); + throw e; } finally { MongodbClientUtil.close(client, null); } diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java index 918fb6b8f3..e1661838c2 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.mongodb.reader; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; @@ -55,5 +56,9 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + if (format.mongodbConfig.getFetchSize() > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取条数必须小于[200000]条"); + } } } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index 1d81ef36ef..a411920d71 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -56,8 +56,12 @@ shade.mongodbwriter.io.netty - com.google - shade.mongodbwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index 047e273350..21e5e39d4f 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -62,8 +62,12 @@ shade.mysqldreader.io.netty - com.google - shade.mysqldreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java index 9eae9b3dee..60181af17c 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java +++ b/flinkx-mysql/flinkx-mysql-dreader/src/main/java/com/dtstack/flinkx/mysqld/reader/MysqldReader.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.List; /** * @author toutian @@ -47,8 +46,8 @@ protected DistributedJdbcInputFormatBuilder getBuilder(){ } @Override - protected List buildConnections(){ - List sourceList = new ArrayList<>(connectionConfigs.size()); + protected ArrayList buildConnections(){ + ArrayList sourceList = new ArrayList<>(connectionConfigs.size()); for (ReaderConfig.ParameterConfig.ConnectionConfig connectionConfig : connectionConfigs) { String curUsername = (connectionConfig.getUsername() == null || connectionConfig.getUsername().length() == 0) ? username : connectionConfig.getUsername(); diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index 8856cdf063..a52536b7ec 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -62,9 +62,13 @@ shade.mysqlreader.io.netty - com.google - shade.mysqlreader.com.google - + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index 9836df273a..148d4bbb95 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -18,11 +18,10 @@ package com.dtstack.flinkx.mysql.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -51,7 +50,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -64,8 +65,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { } querySql = buildQuerySql(inputSplit); - //MySQL流式读取 - fetchSize = Integer.MIN_VALUE; + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); @@ -112,7 +115,6 @@ public Row nextRecordInternal(Row row) throws IOException { } return super.nextRecordInternal(row); }catch (Exception e) { - LOG.error("error to get next record, row = {}, descColumnTypeList = {}, e = {}", row, new Gson().toJson(descColumnTypeList), ExceptionUtil.getErrorMessage(e)); throw new IOException("Couldn't read data - " + e.getMessage(), e); } } diff --git a/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java b/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java deleted file mode 100644 index 50dff8f343..0000000000 --- a/flinkx-mysql/flinkx-mysql-reader/src/test/java/bigdata/TestMysqlPk.java +++ /dev/null @@ -1,22 +0,0 @@ -package bigdata; - -import com.dtstack.flinkx.mysql.MySqlDatabaseMeta; -import com.dtstack.flinkx.rdb.util.DbUtil; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.Map; - - -public class TestMysqlPk { - public static void main(String[] args) throws ClassNotFoundException, SQLException { - MySqlDatabaseMeta databaseMeta = new MySqlDatabaseMeta(); - Class.forName(databaseMeta.getDriverClass()); - Connection conn = DriverManager.getConnection("jdbc:mysql://172.16.8.104:3306/test?useCursorFetch=true", "dtstack", "abc123"); - //List list = databaseMeta.listUniqueKeys("sb250", conn); - //System.out.println(list); - Map map = DbUtil.getPrimaryOrUniqueKeys("sb252", conn); - System.out.println(map); - } -} diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 82e3aa55c4..94e5d95a50 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -62,8 +62,12 @@ shade.mysqlwriter.io.netty - com.google - shade.mysqlwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java index dd54b0ba18..1cb857e3ec 100644 --- a/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java +++ b/flinkx-odps/flinkx-odps-core/src/main/java/com/dtstack/flinkx/odps/OdpsUtil.java @@ -56,7 +56,7 @@ public class OdpsUtil { public static int MAX_RETRY_TIME = 3; - public static final long BUFFER_SIZE_DEFAULT = 64 * 1024 * 1024; + public static final long BUFFER_SIZE_DEFAULT = 64 * 1024 * 1024L; public static Odps initOdps(Map odpsConfig) { String odpsServer = odpsConfig.get(OdpsConfigKeys.KEY_ODPS_SERVER); diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 866fd622fe..8007e473c0 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -50,6 +50,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java index 1bd4db402b..eba6fc7175 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsInputFormatBuilder.java @@ -19,11 +19,11 @@ package com.dtstack.flinkx.odps.reader; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.odps.OdpsConfigKeys; import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.odps.OdpsConfigKeys.*; /** * The Builder of OdpsInputFormat @@ -41,7 +41,7 @@ public OdpsInputFormatBuilder() { public void setOdpsConfig(Map odpsConfig) { format.odpsConfig = odpsConfig; - format.projectName = odpsConfig.get(KEY_PROJECT); + format.projectName = odpsConfig.get(OdpsConfigKeys.KEY_PROJECT); } public void setTableName(String tableName) { diff --git a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java index 4a076ce122..f45ee2d76a 100644 --- a/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java +++ b/flinkx-odps/flinkx-odps-reader/src/main/java/com/dtstack/flinkx/odps/reader/OdpsReader.java @@ -29,7 +29,9 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.odps.OdpsConfigKeys.*; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_ODPS_CONFIG; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_PARTITION; +import static com.dtstack.flinkx.odps.OdpsConfigKeys.KEY_TABLE; /** * The reader plugin of Odps diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index b268bf43b1..f1622adc0b 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -55,6 +55,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java index 7e6510104d..cdf8b9c6e8 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormat.java @@ -116,7 +116,7 @@ public void writeSingleRecordInternal(Row row) throws WriteRecordException{ @Override protected void writeMultipleRecordsInternal() throws Exception { - throw new UnsupportedOperationException(); + notSupportBatchWrite("OdpsWriter"); } private Record row2record(Row row, String[] columnTypes) throws WriteRecordException { diff --git a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java index 95c7dad246..ee3d21d10a 100644 --- a/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java +++ b/flinkx-odps/flinkx-odps-writer/src/main/java/com/dtstack/flinkx/odps/writer/OdpsOutputFormatBuilder.java @@ -76,5 +76,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("OdpsWriter"); } } diff --git a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java index 5033bf64af..bc4b88f3ba 100644 --- a/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java +++ b/flinkx-oracle/flinkx-oracle-core/src/main/java/com/dtstack/flinkx/oracle/OracleDatabaseMeta.java @@ -36,7 +36,7 @@ public class OracleDatabaseMeta extends BaseDatabaseMeta { public String quoteTable(String table) { table = table.replace("\"",""); String[] part = table.split("\\."); - if(part.length == 2) { + if(part.length == DB_TABLE_PART_SIZE) { table = getStartQuote() + part[0] + getEndQuote() + "." + getStartQuote() + part[1] + getEndQuote(); } else { table = getStartQuote() + table + getEndQuote(); diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index 52ff1880be..225d8e2554 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -62,8 +62,12 @@ shade.oracleReader.io.netty - com.google.guava - shade.oracleReader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java b/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java deleted file mode 100644 index d4227ff774..0000000000 --- a/flinkx-oracle/flinkx-oracle-reader/src/test/java/com/dtstack/flinkx/oracle/reader/test/OracleLocalTest.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.dtstack.flinkx.oracle.reader.test; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; - -/** - * Created by softfly on 18/2/1. - */ -public class OracleLocalTest { - public static void main(String[] args) throws ClassNotFoundException, SQLException { - final String DRIVER = "oracle.jdbc.OracleDriver"; - final String URL = "jdbc:oracle:thin:dev/pass1234@172.16.8.121:1521:dtstack"; - final String USER = "dev"; - final String PASSWORD = "pass1234"; - Connection connection = null; - Class.forName(DRIVER); - connection = DriverManager.getConnection(URL); - - - } -} diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index c6fc7b4e2b..6e9c1850e7 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -63,8 +63,12 @@ shade.oracleWriter.io.netty - com.google.guava - shade.oracleWriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java index ed77a77ea4..98840d5b10 100644 --- a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java +++ b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/format/OracleOutputFormat.java @@ -18,11 +18,16 @@ package com.dtstack.flinkx.oracle.format; import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.oracle.OracleDatabaseMeta; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; import com.dtstack.flinkx.util.DateUtil; import org.apache.flink.types.Row; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.HashMap; @@ -63,7 +68,7 @@ protected List probeFullColumns(String table, Connection dbConn) throws String schema =null; String[] parts = table.split("\\."); - if(parts.length == 2) { + if(parts.length == OracleDatabaseMeta.DB_TABLE_PART_SIZE) { schema = parts[0].toUpperCase(); table = parts[1]; } @@ -79,24 +84,26 @@ protected List probeFullColumns(String table, Connection dbConn) throws @Override protected Map> probePrimaryKeys(String table, Connection dbConn) throws SQLException { Map> map = new HashMap<>(16); - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL,table)); - ResultSet rs = ps.executeQuery(); - while(rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if(!map.containsKey(indexName)) { - map.put(indexName,new ArrayList<>()); + try (PreparedStatement ps = dbConn.prepareStatement(String.format(GET_INDEX_SQL, table)); + ResultSet rs = ps.executeQuery()) { + while(rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if(!map.containsKey(indexName)) { + map.put(indexName,new ArrayList<>()); + } + map.get(indexName).add(rs.getString("COLUMN_NAME")); } - map.get(indexName).add(rs.getString("COLUMN_NAME")); - } - Map> retMap = new HashMap<>((map.size()<<2)/3); - for(Map.Entry> entry: map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if(v!=null && v.size() != 0 && v.get(0) != null) { - retMap.put(k, v); + + Map> retMap = new HashMap<>((map.size()<<2)/3); + for(Map.Entry> entry: map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if(v!=null && v.size() != 0 && v.get(0) != null) { + retMap.put(k, v); + } } + return retMap; } - return retMap; } } diff --git a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml index 182ea07067..04f327d64e 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/pom.xml +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -33,6 +33,16 @@ false + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index 180385dc01..b067a02a7e 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 6d6b416559..89bdc80968 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.sql.SQLException; -import java.sql.Statement; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -65,7 +64,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { // 部分驱动需要关闭事务自动提交,fetchSize参数才会起作用 dbConn.setAutoCommit(false); - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + statement = dbConn.createStatement(resultSetType, resultSetConcurrency); statement.setFetchSize(0); diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index b97dab27c8..1dc16435a8 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -56,6 +56,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index d477787cac..26060ebcf6 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -56,6 +56,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java b/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java index b7b3e95855..5e21aec025 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java +++ b/flinkx-polardb/flinkx-polardb-dreader/src/main/java/com/dtstack/flinkx/polardbd/reader/PolardbdReader.java @@ -21,7 +21,6 @@ import com.dtstack.flinkx.mysqld.reader.MysqldReader; import com.dtstack.flinkx.polardbd.PolardbDatabaseMeta; import com.dtstack.flinkx.polardbd.format.PolardbdInputFormat; -import com.dtstack.flinkx.rdb.inputformat.DistributedJdbcInputFormat; import com.dtstack.flinkx.rdb.inputformat.DistributedJdbcInputFormatBuilder; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 093bdb5152..77ca955ae3 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -55,6 +55,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index d088128085..4ca3fb7511 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -56,6 +56,14 @@ com.mysql shade.polardb.com.mysql + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index 985b99efaa..4af93b9480 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -20,10 +20,7 @@ import com.dtstack.flinkx.enums.EDatabaseType; import com.dtstack.flinkx.rdb.BaseDatabaseMeta; -import org.apache.commons.lang3.StringUtils; -import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.Map; diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java index 190ff818b1..04f5616f23 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java @@ -19,8 +19,10 @@ package com.dtstack.flinkx.postgresql; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; +import org.apache.commons.lang3.StringUtils; import java.util.Arrays; +import java.util.Collections; import java.util.List; /** @@ -35,9 +37,9 @@ public class PostgresqlTypeConverter implements TypeConverterInterface { private List byteTypes = Arrays.asList("bytea","bit varying"); - private List bitTypes = Arrays.asList("bit"); + private List bitTypes = Collections.singletonList("bit"); - private List doubleTypes = Arrays.asList("money"); + private List doubleTypes = Collections.singletonList("money"); private List intTypes = Arrays.asList("int","int2","int4","int8"); @@ -46,19 +48,24 @@ public Object convert(Object data,String typeName) { if (data == null){ return null; } - + String dataValue = data.toString(); + if(stringTypes.contains(typeName)){ + return dataValue; + } + if(StringUtils.isBlank(dataValue)){ + return null; + } if(doubleTypes.contains(typeName)){ - data = Double.parseDouble(String.valueOf(data)); + if(StringUtils.startsWith(dataValue, "$")){ + dataValue = StringUtils.substring(dataValue, 1); + } + data = Double.parseDouble(dataValue); } else if(bitTypes.contains(typeName)){ // - } else if(stringTypes.contains(typeName)){ - data = String.valueOf(data); - } else if(byteTypes.contains(typeName)){ - data = Byte.valueOf(String.valueOf(data)); + }else if(byteTypes.contains(typeName)){ + data = Byte.valueOf(dataValue); } else if(intTypes.contains(typeName)){ - if(data instanceof String){ - data = Integer.parseInt(data.toString()); - } + data = Integer.parseInt(dataValue); } return data; diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index f103f55d85..6e3b8612d6 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -63,8 +63,12 @@ shade.postgresqlreader.io.netty - com.google - shade.postgresqlreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java index 5ba10faad7..790f66f7e9 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlInputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.postgresql.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.util.ClassUtil; import org.apache.commons.collections.CollectionUtils; @@ -49,7 +50,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -62,6 +65,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { } querySql = buildQuerySql(inputSplit); + JdbcInputSplit jdbcInputSplit = (JdbcInputSplit) inputSplit; + if (null != jdbcInputSplit.getStartLocation()) { + startLocation = jdbcInputSplit.getStartLocation(); + } executeQuery(startLocation); columnCount = resultSet.getMetaData().getColumnCount(); boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 2264b0dddb..7f6857ea04 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -62,8 +62,12 @@ shade.postgresqlwriter.io.netty - com.google - shade.postgresqlwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml index 72c8493094..88396482f4 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml +++ b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml @@ -11,11 +11,8 @@ flinkx-pulsar-writer - - - org.apache.maven.plugins maven-shade-plugin diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java index b12d1f8ece..28cd09eaad 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; /** diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java index 55aeee7dda..fdfab032da 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.exception.WriteRecordException; @@ -92,7 +109,7 @@ protected void emit(Map event) throws IOException { } @Override - protected void writeMultipleRecordsInternal() throws Exception { + protected void writeMultipleRecordsInternal() { throw new UnsupportedOperationException(); } diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java index 637a97e67a..c6e35cc79c 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java index ae84b857f1..2068128992 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java @@ -1,8 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.writer.BaseDataWriter; -import static com.dtstack.flinkx.pulsar.writer.Constants.*; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; @@ -10,6 +26,8 @@ import java.util.List; import java.util.Map; +import static com.dtstack.flinkx.pulsar.writer.Constants.*; + /** * @author: pierre @@ -22,6 +40,7 @@ public class PulsarWriter extends BaseDataWriter { protected List tableFields; protected Map producerSettings; + @SuppressWarnings("unchecked") public PulsarWriter(DataTransferConfig config){ super(config); topic = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal(KEY_TOPIC); diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java index ad667ceb13..758be23ace 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java @@ -21,7 +21,11 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * Abstract base parent class of other database prototype implementations @@ -31,6 +35,8 @@ */ public abstract class BaseDatabaseMeta implements DatabaseInterface, Serializable { + public static final int DB_TABLE_PART_SIZE = 2; + @Override public String getStartQuote() { return "\""; diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 26269c7c34..88477b16b3 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -18,21 +18,17 @@ package com.dtstack.flinkx.rdb.util; import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.ParameterValuesProvider; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.SysUtil; import com.dtstack.flinkx.util.TelnetUtil; -import com.google.gson.Gson; import org.apache.commons.lang.StringUtils; import org.apache.flink.util.CollectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedReader; -import java.math.BigDecimal; import java.sql.*; import java.util.ArrayList; import java.util.HashMap; @@ -41,9 +37,9 @@ import java.util.regex.Pattern; /** + * * Utilities for relational database connection and sql execution * company: www.dtstack.com - * * @author huyifan_zju@ */ public class DbUtil { @@ -72,6 +68,9 @@ public class DbUtil { */ private static int NANOS_LENGTH = 19; + public static int NANOS_PART_LENGTH = 9; + private static int FORMAT_TIME_NANOS_LENGTH = 29; + /** * jdbc连接URL的分割正则,用于获取URL?后的连接参数 */ @@ -93,16 +92,15 @@ public class DbUtil { /** * 获取jdbc连接(超时10S) - * - * @param url url - * @param username 账号 - * @param password 密码 + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { Connection dbConn; - synchronized (ClassUtil.LOCK_STR) { + synchronized (ClassUtil.LOCK_STR){ DriverManager.setLoginTimeout(10); // telnet @@ -120,10 +118,9 @@ private static Connection getConnectionInternal(String url, String username, Str /** * 获取jdbc连接(重试3次) - * - * @param url url - * @param username 账号 - * @param password 密码 + * @param url url + * @param username 账号 + * @param password 密码 * @return * @throws SQLException */ @@ -136,8 +133,10 @@ public static Connection getConnection(String url, String username, String passw for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { try { dbConn = getConnectionInternal(url, username, password); - dbConn.createStatement().execute("select 111"); - failed = false; + try (Statement statement = dbConn.createStatement()){ + statement.execute("select 111"); + failed = false; + } } catch (Exception e) { if (dbConn != null) { dbConn.close(); @@ -157,10 +156,9 @@ public static Connection getConnection(String url, String username, String passw /** * 关闭连接资源 - * - * @param rs ResultSet - * @param stmt Statement - * @param conn Connection + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection * @param commit */ public static void closeDbResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { @@ -182,7 +180,7 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con if (null != conn) { try { - if (commit) { + if(commit){ commit(conn); } @@ -195,76 +193,49 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con /** * 手动提交事物 - * * @param conn Connection */ - public static void commit(Connection conn) { + public static void commit(Connection conn){ try { - if (!conn.isClosed() && !conn.getAutoCommit()) { + if (!conn.isClosed() && !conn.getAutoCommit()){ conn.commit(); } - } catch (SQLException e) { + } catch (SQLException e){ LOG.warn("commit error:{}", ExceptionUtil.getErrorMessage(e)); } } /** * 批量执行sql - * * @param dbConn Connection * @param sqls sql列表 */ public static void executeBatch(Connection dbConn, List sqls) { - if (sqls == null || sqls.size() == 0) { + if(sqls == null || sqls.size() == 0) { return; } - try { - Statement stmt = dbConn.createStatement(); - for (String sql : sqls) { + try (Statement stmt = dbConn.createStatement()) { + for(String sql : sqls) { stmt.addBatch(sql); } stmt.executeBatch(); } catch (SQLException e) { - throw new RuntimeException("execute batch sql error:{}", e); + throw new RuntimeException("execute batch sql error:{}",e); } finally { commit(dbConn); } } - /** - * 获取某数据库某表的主键和唯一索引 - * - * @param table 表名 - * @param dbConn 数据库连接 - * @return - * @throws SQLException - */ - public static Map> getPrimaryOrUniqueKeys(String table, Connection dbConn) throws SQLException { - Map> keyMap = new HashMap<>(16); - DatabaseMetaData meta = dbConn.getMetaData(); - ResultSet rs = meta.getIndexInfo(null, null, table, true, false); - while (rs.next()) { - String pkName = rs.getString(6); - String columnName = rs.getString(9); - if (!keyMap.containsKey(pkName)) { - keyMap.put(pkName, new ArrayList<>()); - } - keyMap.get(pkName).add(columnName); - } - return keyMap; - } - /** * 封装channel通道顺序 - * * @param channels * @return */ - public static Object[][] getParameterValues(final int channels) { + public static Object[][] getParameterValues(final int channels){ ParameterValuesProvider provider = () -> { Integer[][] parameters = new Integer[channels][]; - for (int i = 0; i < channels; ++i) { + for(int i = 0; i < channels; ++i) { parameters[i] = new Integer[2]; parameters[i][0] = channels; parameters[i][1] = i; @@ -295,62 +266,20 @@ public static List analyzeColumnType(ResultSet resultSet){ return columnTypeList; } - /** - * 占位符设值 - * - * @param param 参数 - * @param statement PreparedStatement - * @param i 占位符位置 - * @throws SQLException - */ - public static void setParameterValue(Object param, PreparedStatement statement, int i) throws SQLException { - if (param instanceof String) { - statement.setString(i + 1, (String) param); - } else if (param instanceof Long) { - statement.setLong(i + 1, (Long) param); - } else if (param instanceof Integer) { - statement.setInt(i + 1, (Integer) param); - } else if (param instanceof Double) { - statement.setDouble(i + 1, (Double) param); - } else if (param instanceof Boolean) { - statement.setBoolean(i + 1, (Boolean) param); - } else if (param instanceof Float) { - statement.setFloat(i + 1, (Float) param); - } else if (param instanceof BigDecimal) { - statement.setBigDecimal(i + 1, (BigDecimal) param); - } else if (param instanceof Byte) { - statement.setByte(i + 1, (Byte) param); - } else if (param instanceof Short) { - statement.setShort(i + 1, (Short) param); - } else if (param instanceof Date) { - statement.setDate(i + 1, (Date) param); - } else if (param instanceof Time) { - statement.setTime(i + 1, (Time) param); - } else if (param instanceof Timestamp) { - statement.setTimestamp(i + 1, (Timestamp) param); - } else if (param instanceof Array) { - statement.setArray(i + 1, (Array) param); - } else { - //extends with other types if needed - throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)."); - } - } - /** * clob转string - * - * @param obj clob + * @param obj clob * @return * @throws Exception */ - public static Object clobToString(Object obj) throws Exception { + public static Object clobToString(Object obj) throws Exception{ String dataStr; - if (obj instanceof Clob) { - Clob clob = (Clob) obj; + if(obj instanceof Clob){ + Clob clob = (Clob)obj; BufferedReader bf = new BufferedReader(clob.getCharacterStream()); StringBuilder stringBuilder = new StringBuilder(); String line; - while ((line = bf.readLine()) != null) { + while ((line = bf.readLine()) != null){ stringBuilder.append(line); } dataStr = stringBuilder.toString(); @@ -363,34 +292,32 @@ public static Object clobToString(Object obj) throws Exception { /** * 获取纳秒字符串 - * - * @param timeStr + * @param timeStr 2020-03-23 11:03:22.000000000 * @return */ - public static String getNanosTimeStr(String timeStr) { - if (timeStr.length() < 29) { - timeStr += StringUtils.repeat("0", 29 - timeStr.length()); + public static String getNanosTimeStr(String timeStr){ + if(timeStr.length() < FORMAT_TIME_NANOS_LENGTH){ + timeStr += StringUtils.repeat("0",FORMAT_TIME_NANOS_LENGTH - timeStr.length()); } return timeStr; } /** * 将边界位置时间转换成对应饿的纳秒时间 - * * @param startLocation 边界位置(起始/结束) * @return */ - public static int getNanos(long startLocation) { + public static int getNanos(long startLocation){ String timeStr = String.valueOf(startLocation); int nanos; - if (timeStr.length() == SECOND_LENGTH) { + if (timeStr.length() == SECOND_LENGTH){ nanos = 0; - } else if (timeStr.length() == MILLIS_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MILLIS_LENGTH)) * 1000000; - } else if (timeStr.length() == MICRO_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, MICRO_LENGTH)) * 1000; - } else if (timeStr.length() == NANOS_LENGTH) { - nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH, NANOS_LENGTH)); + } else if (timeStr.length() == MILLIS_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MILLIS_LENGTH)) * 1000000; + } else if (timeStr.length() == MICRO_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,MICRO_LENGTH)) * 1000; + } else if (timeStr.length() == NANOS_LENGTH){ + nanos = Integer.parseInt(timeStr.substring(SECOND_LENGTH,NANOS_LENGTH)); } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); } @@ -400,20 +327,19 @@ public static int getNanos(long startLocation) { /** * 将边界位置时间转换成对应饿的毫秒时间 - * - * @param startLocation 边界位置(起始/结束) + * @param startLocation 边界位置(起始/结束) * @return */ - public static long getMillis(long startLocation) { + public static long getMillis(long startLocation){ String timeStr = String.valueOf(startLocation); long millisSecond; - if (timeStr.length() == SECOND_LENGTH) { + if (timeStr.length() == SECOND_LENGTH){ millisSecond = startLocation * 1000; - } else if (timeStr.length() == MILLIS_LENGTH) { + } else if (timeStr.length() == MILLIS_LENGTH){ millisSecond = startLocation; - } else if (timeStr.length() == MICRO_LENGTH) { + } else if (timeStr.length() == MICRO_LENGTH){ millisSecond = startLocation / 1000; - } else if (timeStr.length() == NANOS_LENGTH) { + } else if (timeStr.length() == NANOS_LENGTH){ millisSecond = startLocation / 1000000; } else { throw new IllegalArgumentException("Unknown time unit:startLocation=" + startLocation); @@ -424,24 +350,23 @@ public static long getMillis(long startLocation) { /** * 格式化jdbc连接 - * - * @param dbUrl 原jdbc连接 - * @param extParamMap 需要额外添加的参数 - * @return 格式化后jdbc连接URL字符串 + * @param dbUrl 原jdbc连接 + * @param extParamMap 需要额外添加的参数 + * @return 格式化后jdbc连接URL字符串 */ - public static String formatJdbcUrl(String dbUrl, Map extParamMap) { + public static String formatJdbcUrl(String dbUrl, Map extParamMap){ String[] splits = DB_PATTERN.split(dbUrl); - Map paramMap = new HashMap<>(16); - if (splits.length > 1) { + Map paramMap = new HashMap<>(16); + if(splits.length > 1) { String[] pairs = splits[1].split("&"); - for (String pair : pairs) { + for(String pair : pairs) { String[] leftRight = pair.split("="); paramMap.put(leftRight[0], leftRight[1]); } } - if (!CollectionUtil.isNullOrEmpty(extParamMap)) { + if(!CollectionUtil.isNullOrEmpty(extParamMap)){ paramMap.putAll(extParamMap); } paramMap.put("useCursorFetch", "true"); @@ -450,8 +375,8 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap StringBuffer sb = new StringBuffer(dbUrl.length() + 128); sb.append(splits[0]).append("?"); int index = 0; - for (Map.Entry entry : paramMap.entrySet()) { - if (index != 0) { + for(Map.Entry entry : paramMap.entrySet()) { + if(index != 0) { sb.append("&"); } sb.append(entry.getKey()).append("=").append(entry.getValue()); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java index 4faa32e0ce..f4442111d7 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/DistributedJdbcDataReader.java @@ -107,8 +107,8 @@ protected DistributedJdbcInputFormatBuilder getBuilder(){ throw new RuntimeException("子类必须覆盖getBuilder方法"); } - protected List buildConnections(){ - List sourceList = new ArrayList<>(connectionConfigs.size()); + protected ArrayList buildConnections(){ + ArrayList sourceList = new ArrayList<>(connectionConfigs.size()); for (ReaderConfig.ParameterConfig.ConnectionConfig connectionConfig : connectionConfigs) { String curUsername = (StringUtils.isBlank(connectionConfig.getUsername())) ? username : connectionConfig.getUsername(); String curPassword = (StringUtils.isBlank(connectionConfig.getPassword())) ? password : connectionConfig.getPassword(); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java index 2d33a29161..6e9534b554 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormat.java @@ -30,7 +30,11 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import java.io.*; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; @@ -65,7 +69,7 @@ public class DistributedJdbcInputFormat extends BaseRichInputFormat { protected List descColumnTypeList; - protected List sourceList; + protected ArrayList sourceList; protected transient int sourceIndex; @@ -197,7 +201,7 @@ protected void closeCurrentSource(){ @Override protected void closeInternal() throws IOException { - + closeCurrentSource(); } @Override @@ -208,7 +212,7 @@ public InputSplit[] createInputSplitsInternal(int minPart) throws IOException { Object[][] parmeter = DbUtil.getParameterValues(numPartitions); for (int j = 0; j < numPartitions; j++) { DistributedJdbcInputSplit split = new DistributedJdbcInputSplit(j,numPartitions); - List sourceCopy = deepCopyList(sourceList); + ArrayList sourceCopy = deepCopyList(sourceList); for (int i = 0; i < sourceCopy.size(); i++) { sourceCopy.get(i).setSplitByKey(true); sourceCopy.get(i).setParameterValues(parmeter[j]); @@ -221,7 +225,9 @@ public InputSplit[] createInputSplitsInternal(int minPart) throws IOException { if (partNum == 0){ for (int i = 0; i < sourceList.size(); i++) { DistributedJdbcInputSplit split = new DistributedJdbcInputSplit(i,numPartitions); - split.setSourceList(Arrays.asList(sourceList.get(i))); + ArrayList arrayList = new ArrayList<>(); + arrayList.add(sourceList.get(i)); + split.setSourceList(arrayList); inputSplits[i] = split; } } else { @@ -250,7 +256,7 @@ public boolean reachedEnd() throws IOException { return readNextRecord(); } - public List deepCopyList(List src) throws IOException{ + public ArrayList deepCopyList(ArrayList src) throws IOException{ try { ByteArrayOutputStream byteOut = new ByteArrayOutputStream(); ObjectOutputStream out = new ObjectOutputStream(byteOut); @@ -258,7 +264,7 @@ public List deepCopyList(List src) throws IOException{ ByteArrayInputStream byteIn = new ByteArrayInputStream(byteOut.toByteArray()); ObjectInputStream in = new ObjectInputStream(byteIn); - List dest = (List) in.readObject(); + ArrayList dest = (ArrayList) in.readObject(); return dest; } catch (Exception e){ diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java index 28f2c2c5f1..0c8b21c07f 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputFormatBuilder.java @@ -24,6 +24,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import org.apache.commons.lang.StringUtils; +import java.util.ArrayList; import java.util.List; /** @@ -65,7 +66,7 @@ public void setSplitKey(String splitKey){ format.splitKey = splitKey; } - public void setSourceList(List sourceList){ + public void setSourceList(ArrayList sourceList){ format.sourceList = sourceList; } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java index 724f9532d5..43df7f5c88 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/DistributedJdbcInputSplit.java @@ -32,7 +32,7 @@ */ public class DistributedJdbcInputSplit extends GenericInputSplit { - private List sourceList; + private ArrayList sourceList; public DistributedJdbcInputSplit(int partitionNumber, int totalNumberOfPartitions) { super(partitionNumber, totalNumberOfPartitions); @@ -54,11 +54,11 @@ public void addSource(DataSource source){ this.sourceList.add(source); } - public List getSourceList() { + public ArrayList getSourceList() { return sourceList; } - public void setSourceList(List sourceList) { + public void setSourceList(ArrayList sourceList) { this.sourceList = sourceList; } } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index 706600d712..d71fa41d16 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -27,12 +27,8 @@ import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.util.*; import com.dtstack.flinkx.restore.FormatState; -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.StringUtil; -import com.dtstack.flinkx.util.UrlUtil; +import com.dtstack.flinkx.util.*; import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.LongMaximum; @@ -48,7 +44,9 @@ import java.io.IOException; import java.sql.*; import java.util.Date; -import java.util.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; /** @@ -161,7 +159,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { initMetric(inputSplit); String startLocation = incrementConfig.getStartLocation(); if (incrementConfig.isPolling()) { - endLocationAccumulator.add(Long.parseLong(startLocation)); + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(Long.parseLong(startLocation)); + } isTimestamp = "timestamp".equalsIgnoreCase(incrementConfig.getColumnType()); } else if ((incrementConfig.isIncrement() && incrementConfig.isUseMaxFunc())) { getMaxValue(inputSplit); @@ -252,6 +252,12 @@ public Row nextRecordInternal(Row row) throws IOException { boolean isUpdateLocation = incrementConfig.isPolling() || (incrementConfig.isIncrement() && !incrementConfig.isUseMaxFunc()); if (isUpdateLocation) { Object incrementVal = resultSet.getObject(incrementConfig.getColumnName()); + if(incrementVal != null) { + if((incrementVal instanceof java.util.Date + || incrementVal.getClass().getSimpleName().toUpperCase().contains("TIMESTAMP")) ) { + incrementVal = resultSet.getTimestamp(incrementConfig.getColumnName()); + } + } String location; if(incrementConfig.isPolling()){ location = String.valueOf(incrementVal); @@ -371,7 +377,7 @@ public String getMaxValueFromApi(){ } String url = monitorUrls; - if (monitorUrls.startsWith("http")) { + if (monitorUrls.startsWith(ConstantValue.PROTOCOL_HTTP)) { url = String.format("%s/jobs/%s/accumulators", monitorUrls, jobId); } @@ -707,10 +713,10 @@ private String getLocation(String columnType, Object columnVal) { long time = ((Timestamp) columnVal).getTime() / 1000; String nanosStr = String.valueOf(((Timestamp) columnVal).getNanos()); - if (nanosStr.length() == 9) { + if (nanosStr.length() == DbUtil.NANOS_PART_LENGTH) { location = time + nanosStr; } else { - String fillZeroStr = StringUtils.repeat("0", 9 - nanosStr.length()); + String fillZeroStr = StringUtils.repeat("0", DbUtil.NANOS_PART_LENGTH - nanosStr.length()); location = time + fillZeroStr + nanosStr; } } else { @@ -808,7 +814,7 @@ protected void executeQuery(String startLocation) throws SQLException { queryForPolling(startLocation); } } else { - Statement statement = dbConn.createStatement(resultSetType, resultSetConcurrency); + statement = dbConn.createStatement(resultSetType, resultSetConcurrency); statement.setFetchSize(fetchSize); statement.setQueryTimeout(queryTimeOut); resultSet = statement.executeQuery(querySql); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java index cb48f98b59..ae788328f2 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.rdb.inputformat; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.datareader.IncrementConfig; @@ -128,6 +129,10 @@ protected void checkFormat() { if (StringUtils.isEmpty(format.splitKey) && format.numPartitions > 1){ throw new IllegalArgumentException("Must specify the split column when the channel is greater than 1"); } + + if (format.fetchSize > ConstantValue.MAX_BATCH_SIZE) { + throw new IllegalArgumentException("批量读取条数必须小于[200000]条"); + } } } diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java index 2a94afb8ff..a07eb3473c 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java @@ -32,7 +32,15 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.*; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_BATCH_SIZE; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_FULL_COLUMN; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_INSERT_SQL_MODE; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_POST_SQL; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_PRE_SQL; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_UPDATE_KEY; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_USERNAME; +import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_WRITE_MODE; /** * The Writer plugin for any database that can be connected via JDBC. diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java index bd2dc5afe1..47ab767ef3 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java @@ -30,12 +30,19 @@ import com.google.gson.Gson; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.ObjectUtils; +import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.*; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -107,6 +114,7 @@ public class JdbcOutputFormat extends BaseRichOutputFormat { "AND t.table_name = '%s'"; protected final static String CONN_CLOSE_ERROR_MSG = "No operations allowed"; + protected static List STRING_TYPES = Arrays.asList("CHAR", "VARCHAR","TINYBLOB","TINYTEXT","BLOB","TEXT", "MEDIUMBLOB", "MEDIUMTEXT", "LONGBLOB", "LONGTEXT"); protected PreparedStatement prepareTemplates() throws SQLException { if(CollectionUtils.isEmpty(fullColumn)) { @@ -202,7 +210,13 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { int index = 0; try { for (; index < row.getArity(); index++) { - preparedStatement.setObject(index+1,getField(row,index)); + Object object = row.getField(index); + if( object instanceof String && StringUtils.isBlank((String) object)){ + if(!STRING_TYPES.contains(columnType.get(index))){ + object = null; + } + } + preparedStatement.setObject(index+1, object); } preparedStatement.execute(); @@ -233,8 +247,14 @@ protected String recordConvertDetailErrorMessage(int pos, Row row) { protected void writeMultipleRecordsInternal() throws Exception { try { for (Row row : rows) { - for (int j = 0; j < row.getArity(); ++j) { - preparedStatement.setObject(j + 1, getField(row, j)); + for (int index = 0; index < row.getArity(); index++) { + Object object = row.getField(index); + if( object instanceof String && StringUtils.isBlank((String) object)){ + if(!STRING_TYPES.contains(columnType.get(index))){ + object = null; + } + } + preparedStatement.setObject(index+1, object); } preparedStatement.addBatch(); diff --git a/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java b/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java index 31f40ecb64..9aed6f527b 100644 --- a/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java +++ b/flinkx-redis/flinkx-redis-core/src/main/java/com/dtstack/flinkx/redis/JedisUtil.java @@ -19,13 +19,18 @@ package com.dtstack.flinkx.redis; import com.dtstack.flinkx.util.TelnetUtil; -import redis.clients.jedis.*; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisPoolConfig; import java.util.Properties; import java.util.regex.Matcher; import java.util.regex.Pattern; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; /** * Utilities for redis database connection diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 4a92436c6b..891f05acf5 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -56,8 +56,12 @@ shade.rediswriter.io.netty - com.google - shade.rediswriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java index 7fe2fe9358..2fc85d1fd3 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java @@ -30,9 +30,15 @@ import java.io.IOException; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Properties; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; /** * OutputFormat for writing data to redis database. @@ -70,6 +76,8 @@ public class RedisOutputFormat extends BaseRichOutputFormat { private static final int CRITICAL_TIME = 60 * 60 * 24 * 30; + private static final int KEY_VALUE_SIZE = 2; + @Override public void configure(Configuration parameters) { super.configure(parameters); @@ -138,7 +146,7 @@ private void processTimeFormat(Row row){ } private List getFieldAndValue(Row row){ - if(row.getArity() - keyIndexes.size() != 2){ + if(row.getArity() - keyIndexes.size() != KEY_VALUE_SIZE){ throw new IllegalArgumentException("Each row record can have only one pair of attributes and values except key"); } @@ -185,7 +193,7 @@ private String concatKey(Row row){ @Override protected void writeMultipleRecordsInternal() throws Exception { - // Still not supported + notSupportBatchWrite("RedisWriter"); } @Override diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java index e9480525ae..3b291ece1c 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormatBuilder.java @@ -95,5 +95,7 @@ protected void checkFormat() { if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ throw new UnsupportedOperationException("This plugin not support restore from failed state"); } + + notSupportBatchWrite("RedisWriter"); } } diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java index c64e2a3b29..fa33f2c739 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisWriter.java @@ -31,7 +31,18 @@ import java.util.ArrayList; import java.util.List; -import static com.dtstack.flinkx.redis.RedisConfigKeys.*; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_BATCH_SIZE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DATE_FORMAT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_DB; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_EXPIRE_TIME; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_HOST_PORT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_KEY_FIELD_DELIMITER; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_KEY_INDEXES; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_MODE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_PASSWORD; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TIMEOUT; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_TYPE; +import static com.dtstack.flinkx.redis.RedisConfigKeys.KEY_VALUE_FIELD_DELIMITER; /** * The writer plugin for redis database diff --git a/flinkx-restapi/flinkx-restapi-core/pom.xml b/flinkx-restapi/flinkx-restapi-core/pom.xml new file mode 100644 index 0000000000..742c77488d --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/pom.xml @@ -0,0 +1,20 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-core + + + org.apache.httpcomponents + httpclient + ${http.version} + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java new file mode 100644 index 0000000000..338ba84b1c --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpMethod.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.restapi.common; + +/** + * @author : tiezhu + * @date : 2020/3/13 + */ +public enum HttpMethod { + // http 请求方式 + GET, + POST, + PUT, + PATCH, + DELETE, + COPY, + HEAD, + OPTIONS, + LINK, + UNLINK, + PURGE, + LOCK, + UNLOCK, + PROPFIND, + VIEW + ; + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java new file mode 100644 index 0000000000..b37770ecb6 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import com.dtstack.flinkx.util.GsonUtil; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/16 + */ +public class HttpUtil { + protected static final Logger LOG = LoggerFactory.getLogger(HttpUtil.class); + private static final int COUNT = 32; + private static final int TOTAL_COUNT = 1000; + private static final int TIME_OUT = 5000; + private static final int EXECUTION_COUNT = 5; + + public static CloseableHttpClient getHttpClient() { + // 设置自定义的重试策略 + MyServiceUnavailableRetryStrategy strategy = new MyServiceUnavailableRetryStrategy + .Builder() + .executionCount(EXECUTION_COUNT) + .retryInterval(1000) + .build(); + // 设置自定义的重试Handler + MyHttpRequestRetryHandler retryHandler = new MyHttpRequestRetryHandler + .Builder() + .executionCount(EXECUTION_COUNT) + .build(); + // 设置超时时间 + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(TIME_OUT) + .setConnectionRequestTimeout(TIME_OUT) + .setSocketTimeout(TIME_OUT) + .build(); + // 设置Http连接池 + PoolingHttpClientConnectionManager pcm = new PoolingHttpClientConnectionManager(); + pcm.setDefaultMaxPerRoute(COUNT); + pcm.setMaxTotal(TOTAL_COUNT); + + return HttpClientBuilder.create() + .setServiceUnavailableRetryStrategy(strategy) + .setRetryHandler(retryHandler) + .setDefaultRequestConfig(requestConfig) + .setConnectionManager(pcm) + .build(); +// return HttpClientBuilder.create().build(); + } + + public static HttpRequestBase getRequest(String method, + Map requestBody, + Map header, + String url) { + LOG.debug("current request url: {} current method:{} \n", url, method); + HttpRequestBase request = null; + + if (HttpMethod.GET.name().equalsIgnoreCase(method)) { + request = new HttpGet(url); + } else if (HttpMethod.POST.name().equalsIgnoreCase(method)) { + HttpPost post = new HttpPost(url); + post.setEntity(getEntityData(requestBody)); + request = post; + } else { + throw new RuntimeException("Unsupported method:" + method); + } + + for (Map.Entry entry : header.entrySet()) { + request.addHeader(entry.getKey(), entry.getValue()); + } + return request; + } + + public static void closeClient(CloseableHttpClient httpClient) { + try { + httpClient.close(); + } catch (IOException e) { + throw new RuntimeException("close client error"); + } + } + + public static StringEntity getEntityData(Map body) { + StringEntity stringEntity = new StringEntity(GsonUtil.GSON.toJson(body), StandardCharsets.UTF_8); + stringEntity.setContentEncoding(StandardCharsets.UTF_8.name()); + return stringEntity; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java new file mode 100644 index 0000000000..e0689cb658 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyHttpRequestRetryHandler.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.http.HttpEntityEnclosingRequest; +import org.apache.http.HttpRequest; +import org.apache.http.NoHttpResponseException; +import org.apache.http.client.HttpRequestRetryHandler; +import org.apache.http.client.protocol.HttpClientContext; +import org.apache.http.protocol.HttpContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.UnknownHostException; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class MyHttpRequestRetryHandler implements HttpRequestRetryHandler { + protected static final Logger LOG = LoggerFactory.getLogger(MyHttpRequestRetryHandler.class); + + private int executionMaxCount; + + public MyHttpRequestRetryHandler(Builder builder) { + this.executionMaxCount = builder.executionMaxCount; + } + + @Override + public boolean retryRequest(IOException exception, int executionCount, HttpContext context) { + LOG.info("第" + executionCount + "次重试"); + + if (executionCount >= this.executionMaxCount) { + // Do not retry if over max retry count + return false; + } + if (exception instanceof InterruptedIOException) { + // Timeout + return true; + } + if (exception instanceof UnknownHostException) { + // Unknown host + return true; + } + if (exception instanceof SSLException) { + // SSL handshake exception + return true; + } + if (exception instanceof NoHttpResponseException) { + // No response + return true; + } + + HttpClientContext clientContext = HttpClientContext.adapt(context); + HttpRequest request = clientContext.getRequest(); + boolean idempotent = !(request instanceof HttpEntityEnclosingRequest); + // Retry if the request is considered idempotent + return !idempotent; + } + + + public static final class Builder { + private int executionMaxCount; + + public Builder() { + executionMaxCount = 5; + } + + public Builder executionCount(int executionCount) { + this.executionMaxCount = executionCount; + return this; + } + + public MyHttpRequestRetryHandler build() { + return new MyHttpRequestRetryHandler(this); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java new file mode 100644 index 0000000000..c215926588 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MyServiceUnavailableRetryStrategy.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.http.HttpResponse; +import org.apache.http.client.ServiceUnavailableRetryStrategy; +import org.apache.http.protocol.HttpContext; + +/** + * @author : tiezhu + * @date : 2020/3/12 + * 自定义httpClient重试策略,默认重试次数为5,重试时间间隔为2s + */ +public class MyServiceUnavailableRetryStrategy implements ServiceUnavailableRetryStrategy { + private int executionCount; + private long retryInterval; + + public MyServiceUnavailableRetryStrategy(Builder builder) { + this.executionCount = builder.executionCount; + this.retryInterval = builder.retryInterval; + } + + @Override + public boolean retryRequest(HttpResponse httpResponse, int executionCount, HttpContext httpContext) { + int successCode = 200; + return httpResponse.getStatusLine().getStatusCode() != successCode + && executionCount < this.executionCount; + } + + @Override + public long getRetryInterval() { + return this.retryInterval; + } + + public static final class Builder { + private int executionCount; + private long retryInterval; + + public Builder() { + executionCount = 5; + retryInterval = 2000; + } + + public Builder executionCount(int executionCount) { + this.executionCount = executionCount; + return this; + } + + public Builder retryInterval(long retryInterval) { + this.retryInterval = retryInterval; + return this; + } + + public MyServiceUnavailableRetryStrategy build() { + return new MyServiceUnavailableRetryStrategy(this); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java new file mode 100644 index 0000000000..8821e3777c --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestapiKeys.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * @author : tiezhu + * @date : 2020/3/19 + */ +public class RestapiKeys { + public static final String KEY_METHOD = "method"; + public static final String KEY_HEADER = "header"; + public static final String KEY_BODY = "body"; + public static final String KEY_PARAMS = "params"; + public static final String KEY_COLUMN = "column"; + public static final String KEY_URL = "url"; + public static final String KEY_BATCH_INTERVAL = "batchInterval"; +} diff --git a/flinkx-restapi/flinkx-restapi-reader/pom.xml b/flinkx-restapi/flinkx-restapi-reader/pom.xml new file mode 100644 index 0000000000..8e27c23493 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/pom.xml @@ -0,0 +1,99 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-reader + + + + com.dtstack.flinkx + flinkx-restapi-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.restapireader.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java new file mode 100644 index 0000000000..59c6b535de --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.inputformat; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.restapi.common.HttpUtil; +import com.dtstack.flinkx.util.GsonUtil; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; + +import java.io.IOException; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiInputFormat extends BaseRichInputFormat { + + protected String url; + + protected String method; + + protected transient CloseableHttpClient httpClient; + + protected Map header; + + protected Map entityDataToMap; + + protected boolean getData; + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + httpClient = HttpUtil.getHttpClient(); + } + + @Override + public void closeInputFormat() { + HttpUtil.closeClient(httpClient); + } + + + @Override + @SuppressWarnings("unchecked") + protected void openInternal(InputSplit inputSplit) throws IOException { + HttpUriRequest request = HttpUtil.getRequest(method, header,null, url); + try { + CloseableHttpResponse httpResponse = httpClient.execute(request); + HttpEntity entity = httpResponse.getEntity(); + if (entity != null) { + String entityData = EntityUtils.toString(entity); + entityDataToMap = GsonUtil.GSON.fromJson(entityData, Map.class); + getData = true; + } else { + throw new RuntimeException("entity is null"); + } + } catch (Exception e) { + throw new RuntimeException("get entity error"); + } + } + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { + InputSplit[] inputSplits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + inputSplits[i] = new GenericInputSplit(i, minNumSplits); + } + return inputSplits; + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + row = new Row(1); + row.setField(0, entityDataToMap); + getData = false; + return row; + } + + @Override + protected void closeInternal() throws IOException { + } + + @Override + public boolean reachedEnd() throws IOException { + return !getData; + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java new file mode 100644 index 0000000000..ac636b4038 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.inputformat; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; + +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiInputFormatBuilder extends BaseRichInputFormatBuilder { + protected RestapiInputFormat format; + + public RestapiInputFormatBuilder(){ super.format = format = new RestapiInputFormat();} + + public void setUrl(String url){this.format.url = url;} + public void setHeader(Map header){ this.format.header = header;} + public void setMethod(String method){ this.format.method = method;} + + @Override + protected void checkFormat() { + if(format.url.isEmpty()){ + throw new IllegalArgumentException("缺少url"); + } + if (format.method.isEmpty()) { + throw new IllegalArgumentException("缺少method"); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java new file mode 100644 index 0000000000..162de36054 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.restapi.inputformat.RestapiInputFormatBuilder; +import com.google.common.collect.Maps; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiReader extends BaseDataReader { + + private String url; + + private String method; + + private Map header = Maps.newHashMap(); + + private ArrayList> temp; + + @SuppressWarnings("unchecked") + public RestapiReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + + url = readerConfig.getParameter().getStringVal("url"); + method = readerConfig.getParameter().getStringVal("method"); + temp = (ArrayList>) readerConfig.getParameter().getVal("header"); + if (temp != null) { + for (Map map : temp) { + header.putAll(map); + } + } + } + + @Override + public DataStream readData() { + RestapiInputFormatBuilder builder = new RestapiInputFormatBuilder(); + + builder.setHeader(header); + builder.setMethod(method); + builder.setUrl(url); + + return createInput(builder.finish()); + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/pom.xml b/flinkx-restapi/flinkx-restapi-writer/pom.xml new file mode 100644 index 0000000000..00a801c2b1 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/pom.xml @@ -0,0 +1,99 @@ + + + + flinkx-restapi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi-writer + + + + com.dtstack.flinkx + flinkx-restapi-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.restapiwriter.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java new file mode 100644 index 0000000000..ce96f57a74 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.outputformat; + +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.restapi.common.HttpUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.google.common.collect.Maps; +import org.apache.flink.types.Row; +import org.apache.http.HttpStatus; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.impl.client.CloseableHttpClient; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + * 当前只考虑了元数据读取,和带有字段名column读取的情况,其他情况暂未考虑 + */ +public class RestapiOutputFormat extends BaseRichOutputFormat { + + protected String url; + + protected String method; + + protected ArrayList column; + + protected Map params; + + protected Map body; + + protected Map header; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + // Nothing to do + } + + @Override + protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + LOG.info("start write single record"); + CloseableHttpClient httpClient = HttpUtil.getHttpClient(); + int index = 0; + Map requestBody = Maps.newHashMap(); + Object dataRow; + try { + dataRow = getDataFromRow(row, column); + if (!params.isEmpty()) { + Iterator iterator = params.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = (Map.Entry) iterator.next(); + body.put((String) entry.getKey(), entry.getValue()); + } + } + body.put("data", dataRow); + requestBody.put("json", body); + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + sendRequest(httpClient, requestBody, method, header, url); + } catch (Exception e) { + requestErrorMessage(e, index, row); + } finally { + // 最后不管发送是否成功,都要关闭client + HttpUtil.closeClient(httpClient); + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + LOG.info("start write multiple records"); + try { + CloseableHttpClient httpClient = HttpUtil.getHttpClient(); + List dataRow = new ArrayList<>(); + Map requestBody = Maps.newHashMap(); + for (Row row : rows) { + dataRow.add(getDataFromRow(row, column)); + } + if (!params.isEmpty()) { + Iterator iterator = params.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = (Map.Entry) iterator.next(); + body.put((String) entry.getKey(), entry.getValue()); + } + } + body.put("data", dataRow); + requestBody.put("json", body); + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + sendRequest(httpClient, requestBody, method, header, url); + } catch (Exception e) { + LOG.warn("write record error !", e); + } + } + + private void requestErrorMessage(Exception e, int index, Row row) { + if (index < row.getArity()) { + recordConvertDetailErrorMessage(index, row); + LOG.warn("添加脏数据:" + row.getField(index)); + } + } + + private Object getDataFromRow(Row row, List column) throws IOException { + Map columnData = Maps.newHashMap(); + int index = 0; + if (!column.isEmpty()) { + // 如果column不为空,那么将数据和字段名一一对应 + for (; index < row.getArity(); index++) { + columnData.put(column.get(index), row.getField(index)); + } + return GsonUtil.GSON.toJson(columnData); + } else { + return row.getField(index); + } + } + + + private void sendRequest(CloseableHttpClient httpClient, + Map requestBody, + String method, + Map header, + String url) throws IOException { + LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); + HttpRequestBase request = HttpUtil.getRequest(method, requestBody, header, url); + CloseableHttpResponse httpResponse = httpClient.execute(request); + // 重试之后返回状态码不为200 + if (httpResponse.getStatusLine().getStatusCode() != HttpStatus.SC_OK) { + LOG.warn("重试之后当前请求状态码为" + httpResponse.getStatusLine().getStatusCode()); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java new file mode 100644 index 0000000000..3ab4751ea0 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormatBuilder.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.outputformat; + +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; + +import java.util.ArrayList; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private RestapiOutputFormat format; + + public RestapiOutputFormatBuilder() { + super.format = format = new RestapiOutputFormat(); + } + + public void setUrl(String url) { + this.format.url = url; + } + + public void setHeader(Map header) { + this.format.header = header; + } + + public void setMethod(String method) { + this.format.method = method; + } + + public void setBody(Map body) { + this.format.body = body; + } + + public void setColumn(ArrayList column) { + format.column = column; + } + + public void setParams(Map params){ + format.params = params; + } + + + @Override + protected void checkFormat() { + if (format.url.isEmpty()) { + throw new IllegalArgumentException("缺少url"); + } + if (format.method.isEmpty()) { + throw new IllegalArgumentException("缺少method"); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java new file mode 100644 index 0000000000..f050abb1b9 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.restapi.common.RestapiKeys; +import com.dtstack.flinkx.restapi.outputformat.RestapiOutputFormatBuilder; +import com.dtstack.flinkx.writer.BaseDataWriter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +/** + * @author : tiezhu + * @date : 2020/3/12 + */ +public class RestapiWriter extends BaseDataWriter { + + protected String url; + + protected String method; + + protected Map header = Maps.newHashMap(); + + protected Map body =Maps.newHashMap(); + + protected ArrayList column = Lists.newArrayList(); + + protected Map params = Maps.newHashMap(); + + protected int batchInterval; + + @SuppressWarnings("unchecked") + public RestapiWriter(DataTransferConfig config) { + super(config); + Object tempObj; + + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + + url = writerConfig.getParameter().getStringVal(RestapiKeys.KEY_URL); + method = writerConfig.getParameter().getStringVal(RestapiKeys.KEY_METHOD); + batchInterval = writerConfig.getParameter().getIntVal(RestapiKeys.KEY_BATCH_INTERVAL, 1); + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_COLUMN); + if (tempObj != null) { + column.addAll((ArrayList) tempObj); + } + + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_HEADER); + if (tempObj != null) { + for (Map map : (ArrayList>) tempObj) { + header.putAll(map); + } + } + + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_BODY); + if (tempObj != null) { + for (Map map : (ArrayList>) tempObj) { + body.putAll(map); + } + } + tempObj = writerConfig.getParameter().getVal(RestapiKeys.KEY_PARAMS); + if (tempObj != null) { + params = (HashMap) tempObj; + } + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + RestapiOutputFormatBuilder builder = new RestapiOutputFormatBuilder(); + + builder.setHeader(header); + builder.setMethod(method); + builder.setUrl(url); + builder.setBody(body); + builder.setColumn(column); + builder.setParams(params); + builder.setBatchInterval(batchInterval); + + return createOutput(dataSet, builder.finish()); + } +} diff --git a/flinkx-restapi/pom.xml b/flinkx-restapi/pom.xml new file mode 100644 index 0000000000..876877c87a --- /dev/null +++ b/flinkx-restapi/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-restapi + pom + + flinkx-restapi-core + flinkx-restapi-writer + flinkx-restapi-reader + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + \ No newline at end of file diff --git a/flinkx-saphana/flinkx-saphana-core/pom.xml b/flinkx-saphana/flinkx-saphana-core/pom.xml index bed09b714f..5ac7f520b5 100644 --- a/flinkx-saphana/flinkx-saphana-core/pom.xml +++ b/flinkx-saphana/flinkx-saphana-core/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index ba9e5c4375..8b170aecf7 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 8c19fad040..233c1fc5ae 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -1,6 +1,6 @@ - flinkx-saphana diff --git a/flinkx-saphana/pom.xml b/flinkx-saphana/pom.xml index a9ecb0501e..e36168c903 100644 --- a/flinkx-saphana/pom.xml +++ b/flinkx-saphana/pom.xml @@ -1,6 +1,6 @@ - flinkx-all diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index 7300b9251f..1cd8f983b1 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -62,8 +62,12 @@ shade.sqlserverreader.io.netty - com.google - shade.sqlserverreader.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index e4f6ed9b36..f9d45aeb84 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -63,8 +63,12 @@ shade.sqlserverwriter.io.netty - com.google - shade.sqlserverwriter.com.google + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index d2dfa853cc..3586e32383 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -49,6 +49,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index f106065763..8466152676 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -45,6 +45,16 @@ + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java index da9a167b59..ac578d2c86 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java @@ -48,7 +48,12 @@ protected void openInternal(int taskNumber, int numTasks) throws IOException { @Override protected void writeSingleRecordInternal(Row row) throws WriteRecordException { if (print) { - LOG.info("subTaskIndex[{}]:{}", taskNumber, row); + LOG.info("subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); + } + + if (restoreConfig.isRestore()) { + formatState.setState(row.getField(restoreConfig.getRestoreColumnIndex())); + LOG.info("print data subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); } } @@ -56,8 +61,19 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { protected void writeMultipleRecordsInternal() throws Exception { if (print) { for (Row row : rows) { - LOG.info(String.valueOf(row)); + LOG.info(rowToStringWithDelimiter(row, writeDelimiter)); + } + } + } + + public String rowToStringWithDelimiter(Row row, String writeDelimiter) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getArity(); i++) { + if (i > 0) { + sb.append(writeDelimiter); } + sb.append(StringUtils.arrayAwareToString(row.getField(i))); } + return sb.toString(); } } diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java index 246a4eb8bc..05344b6494 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamWriter.java @@ -46,7 +46,7 @@ public StreamWriter(DataTransferConfig config) { super(config); print = config.getJob().getContent().get(0).getWriter().getParameter().getBooleanVal("print",false); writeDelimiter = config.getJob().getContent().get(0).getWriter().getParameter().getStringVal("writeDelimiter", "|"); - batchInterval = config.getJob().getContent().get(0).getWriter().getParameter().getIntVal("batchInterval", 20); + batchInterval = config.getJob().getContent().get(0).getWriter().getParameter().getIntVal("batchInterval", 1); List column = config.getJob().getContent().get(0).getWriter().getParameter().getColumn(); metaColumns = MetaColumn.getMetaColumns(column); diff --git a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java index 3b42c064c3..1c84adcfa4 100644 --- a/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java +++ b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/DBUtil.java @@ -1,15 +1,10 @@ package com.dtstack.flinkx.teradata.util; -import com.dtstack.flinkx.rdb.DatabaseInterface; -import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; -import java.sql.*; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; /** * @author wuhui @@ -37,38 +32,4 @@ public static Connection getConnection(String url, String username, String passw return dbConn; } - - /** - * 获取表列名类型列表 - * @param dbURL jdbc url - * @param username 数据库账号 - * @param password 数据库密码 - * @param databaseInterface DatabaseInterface - * @param table 表名 - * @param sql sql - * @return - */ - public static List analyzeTable(String dbURL, String username, String password, DatabaseInterface databaseInterface, - String table, String sql) { - List descColumnTypeList = new ArrayList<>(); - Connection dbConn = null; - Statement stmt = null; - ResultSet rs = null; - try { - dbConn = getConnection(dbURL, username, password); - stmt = dbConn.createStatement(); - rs = stmt.executeQuery(databaseInterface.getSqlQuerySqlFields(sql)); - ResultSetMetaData rd = rs.getMetaData(); - - for (int i = 1; i <= rd.getColumnCount(); i++) { - descColumnTypeList.add(rd.getColumnTypeName(i)); - } - } catch (SQLException e) { - throw new RuntimeException(e); - } finally { - DbUtil.closeDbResources(rs, stmt, dbConn, false); - } - - return descColumnTypeList; - } } diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index 0d36c0ef17..b37ab81b88 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -16,351 +16,432 @@ + ch.qos.logback logback-classic 1.1.7 + com.google.guava guava 19.0 + com.google.code.gson gson 2.7 + hadoop-mapreduce-client-core org.apache.hadoop ${hadoop.version} + hadoop-common org.apache.hadoop ${hadoop.version} + hadoop-hdfs org.apache.hadoop ${hadoop.version} + org.apache.flink flink-metrics-core - ${flink.version} + 1.8.1 + org.apache.flink flink-metrics-prometheus_2.11 ${flink.version} + io.prometheus simpleclient ${prometheus.version} + io.prometheus simpleclient_httpserver ${prometheus.version} + io.prometheus simpleclient_pushgateway ${prometheus.version} + com.dtstack.flinkx flinkx-core - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-stream-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-stream-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-carbondata-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-carbondata-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-db2-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-db2-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-es-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-es-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-ftp-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-ftp-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hbase-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-hbase-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hdfs-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-hdfs-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-mongodb-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mongodb-writer - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-dreader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-mysql-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-odps-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-odps-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-oracle-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-oracle-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-postgresql-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-postgresql-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-sqlserver-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-sqlserver-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-redis-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-core - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-rdb-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-gbase-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-gbase-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-binlog-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-hive-writer - ${flinkx.version} + 1.6 + org.apache.hive hive-serde 2.1.0 + com.dtstack.flinkx flinkx-kafka11-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka11-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka10-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka10-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka09-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka09-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kudu-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kudu-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-kafka-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-clickhouse-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-clickhouse-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-polardb-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-polardb-writer - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-phoenix-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-phoenix-writer - ${flinkx.version} + 1.6 + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test-jar + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + test-jar + + + + org.apache.derby + derby + 10.14.2.0 + test + com.dtstack.flinkx flinkx-emqx-reader - ${flinkx.version} + 1.6 com.dtstack.flinkx flinkx-emqx-writer - ${flinkx.version} + 1.6 + + com.dtstack.flinkx + flinkx-restapi-writer + 1.6 + + com.dtstack.flinkx flinkx-dm-reader - ${flinkx.version} + 1.6 + com.dtstack.flinkx flinkx-dm-writer - ${flinkx.version} + 1.6 + + + com.dtstack.flinkx + flinkx-greenplum-reader + 1.6 + + + com.dtstack.flinkx + flinkx-greenplum-writer + 1.6 diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 33fc1dd035..27883b37a4 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -41,6 +41,8 @@ import com.dtstack.flinkx.ftp.writer.FtpWriter; import com.dtstack.flinkx.gbase.reader.GbaseReader; import com.dtstack.flinkx.gbase.writer.GbaseWriter; +import com.dtstack.flinkx.greenplum.reader.GreenplumReader; +import com.dtstack.flinkx.greenplum.writer.GreenplumWriter; import com.dtstack.flinkx.hbase.reader.HbaseReader; import com.dtstack.flinkx.hbase.writer.HbaseWriter; import com.dtstack.flinkx.hdfs.reader.HdfsReader; @@ -73,6 +75,7 @@ import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.redis.writer.RedisWriter; +import com.dtstack.flinkx.restapi.writer.RestapiWriter; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; import com.dtstack.flinkx.stream.reader.StreamReader; @@ -84,7 +87,9 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -112,21 +117,18 @@ public class LocalTest { public static Logger LOG = LoggerFactory.getLogger(LocalTest.class); public static Configuration conf = new Configuration(); - public static void main(String[] args) throws Exception { - setLogLevel(Level.INFO.toString()); - + public static void main(String[] args) throws Exception{ + setLogLevel(Level.DEBUG.toString()); Properties confProperties = new Properties(); // confProperties.put("flink.checkpoint.interval", "10000"); // confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); -// - conf.setString("akka.ask.timeout", "180 s"); - conf.setString("web.timeout", "100000"); + // conf.setString("metrics.reporter.promgateway.class","org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"); -// conf.setString("metrics.reporter.promgateway.host","172.16.8.178"); +// conf.setString("metrics.reporter.promgateway.host","127.0.0.1"); // conf.setString("metrics.reporter.promgateway.port","9091"); -// conf.setString("metrics.reporter.promgateway.jobName","kanata"); +// conf.setString("metrics.reporter.promgateway.jobName","108job"); // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); -// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","false"); +// conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); String jobPath = "D:\\dtstack\\flinkx-all\\flinkx-examples\\examples\\clickhouse_stream.json"; String savePointPath = ""; @@ -134,17 +136,29 @@ public static void main(String[] args) throws Exception { ResultPrintUtil.printResult(result); } - public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savePointPath) throws Exception { + public static JobExecutionResult runJob(File jobFile, Properties confProperties, String savepointPath) throws Exception{ String jobContent = readJob(jobFile); - return runJob(jobContent, confProperties, savePointPath); + return runJob(jobContent, confProperties, savepointPath); } - public static JobExecutionResult runJob(String job, Properties confProperties, String savePointPath) throws Exception { + public static JobExecutionResult runJob(String job, Properties confProperties, String savepointPath) throws Exception{ DataTransferConfig config = DataTransferConfig.parse(job); + + conf.setString("akka.ask.timeout", "180 s"); + conf.setString("web.timeout", String.valueOf(100000)); + MyLocalStreamEnvironment env = new MyLocalStreamEnvironment(conf); + openCheckpointConf(env, confProperties); + env.setParallelism(config.getJob().getSetting().getSpeed().getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); + + if (needRestart(config)) { + env.setRestartStrategy(RestartStrategies.fixedDelayRestart( + 10, + Time.of(10, TimeUnit.SECONDS) + )); + } BaseDataReader reader = buildDataReader(config, env); DataStream dataStream = reader.readData(); @@ -163,209 +177,110 @@ public static JobExecutionResult runJob(String job, Properties confProperties, S dataStreamSink.setParallelism(speedConfig.getWriterChannel()); } - if (StringUtils.isNotEmpty(savePointPath)) { - env.setSettings(SavepointRestoreSettings.forPath(savePointPath)); + if(StringUtils.isNotEmpty(savepointPath)){ + env.setSettings(SavepointRestoreSettings.forPath(savepointPath)); } return env.execute(); } + private static boolean needRestart(DataTransferConfig config){ + return config.getJob().getSetting().getRestoreConfig().isRestore(); + } + private static String readJob(File file) { - try(FileInputStream in = new FileInputStream(file);) { + try(FileInputStream in = new FileInputStream(file)) { byte[] fileContent = new byte[(int) file.length()]; in.read(fileContent); return new String(fileContent, StandardCharsets.UTF_8); - } catch (Exception e) { + } catch (Exception e){ throw new RuntimeException(e); } } - private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env) { + private static BaseDataReader buildDataReader(DataTransferConfig config, StreamExecutionEnvironment env){ String readerName = config.getJob().getContent().get(0).getReader().getName(); - BaseDataReader reader; - switch (readerName) { - case PluginNameConstants.STREAM_READER: - reader = new StreamReader(config, env); - break; - case PluginNameConstants.CARBONDATA_READER: - reader = new CarbondataReader(config, env); - break; - case PluginNameConstants.ORACLE_READER: - reader = new OracleReader(config, env); - break; - case PluginNameConstants.POSTGRESQL_READER: - reader = new PostgresqlReader(config, env); - break; - case PluginNameConstants.SQLSERVER_READER: - reader = new SqlserverReader(config, env); - break; - case PluginNameConstants.MYSQLD_READER: - reader = new MysqldReader(config, env); - break; - case PluginNameConstants.MYSQL_READER: - reader = new MysqlReader(config, env); - break; - case PluginNameConstants.DB2_READER: - reader = new Db2Reader(config, env); - break; - case PluginNameConstants.GBASE_READER: - reader = new GbaseReader(config, env); - break; - case PluginNameConstants.ES_READER: - reader = new EsReader(config, env); - break; - case PluginNameConstants.FTP_READER: - reader = new FtpReader(config, env); - break; - case PluginNameConstants.HBASE_READER: - reader = new HbaseReader(config, env); - break; - case PluginNameConstants.HDFS_READER: - reader = new HdfsReader(config, env); - break; - case PluginNameConstants.MONGODB_READER: - reader = new MongodbReader(config, env); - break; - case PluginNameConstants.ODPS_READER: - reader = new OdpsReader(config, env); - break; - case PluginNameConstants.BINLOG_READER: - reader = new BinlogReader(config, env); - break; - case PluginNameConstants.KAFKA09_READER: - reader = new Kafka09Reader(config, env); - break; - case PluginNameConstants.KAFKA10_READER: - reader = new Kafka10Reader(config, env); - break; - case PluginNameConstants.KAFKA11_READER: - reader = new Kafka11Reader(config, env); - break; - case PluginNameConstants.KAFKA_READER: - reader = new KafkaReader(config, env); - break; - case PluginNameConstants.KUDU_READER: - reader = new KuduReader(config, env); - break; - case PluginNameConstants.CLICKHOUSE_READER: - reader = new ClickhouseReader(config, env); - break; - case PluginNameConstants.POLARDB_READER: - reader = new PolardbReader(config, env); - break; - case PluginNameConstants.PHOENIX_READER: - reader = new PhoenixReader(config, env); - break; - case PluginNameConstants.EMQX_READER: - reader = new EmqxReader(config, env); - break; - case PluginNameConstants.DM_READER: - reader = new DmReader(config, env); - break; - default: - throw new IllegalArgumentException("Can not find reader by name:" + readerName); + BaseDataReader reader ; + switch (readerName){ + case PluginNameConstants.STREAM_READER : reader = new StreamReader(config, env); break; + case PluginNameConstants.CARBONDATA_READER : reader = new CarbondataReader(config, env); break; + case PluginNameConstants.ORACLE_READER : reader = new OracleReader(config, env); break; + case PluginNameConstants.POSTGRESQL_READER : reader = new PostgresqlReader(config, env); break; + case PluginNameConstants.SQLSERVER_READER : reader = new SqlserverReader(config, env); break; + case PluginNameConstants.MYSQLD_READER : reader = new MysqldReader(config, env); break; + case PluginNameConstants.MYSQL_READER : reader = new MysqlReader(config, env); break; + case PluginNameConstants.DB2_READER : reader = new Db2Reader(config, env); break; + case PluginNameConstants.GBASE_READER : reader = new GbaseReader(config, env); break; + case PluginNameConstants.ES_READER : reader = new EsReader(config, env); break; + case PluginNameConstants.FTP_READER : reader = new FtpReader(config, env); break; + case PluginNameConstants.HBASE_READER : reader = new HbaseReader(config, env); break; + case PluginNameConstants.HDFS_READER : reader = new HdfsReader(config, env); break; + case PluginNameConstants.MONGODB_READER : reader = new MongodbReader(config, env); break; + case PluginNameConstants.ODPS_READER : reader = new OdpsReader(config, env); break; + case PluginNameConstants.BINLOG_READER : reader = new BinlogReader(config, env); break; + case PluginNameConstants.KAFKA09_READER : reader = new Kafka09Reader(config, env); break; + case PluginNameConstants.KAFKA10_READER : reader = new Kafka10Reader(config, env); break; + case PluginNameConstants.KAFKA11_READER : reader = new Kafka11Reader(config, env); break; + case PluginNameConstants.KAFKA_READER : reader = new KafkaReader(config, env); break; + case PluginNameConstants.KUDU_READER : reader = new KuduReader(config, env); break; + case PluginNameConstants.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; + case PluginNameConstants.POLARDB_READER : reader = new PolardbReader(config, env); break; + case PluginNameConstants.PHOENIX_READER : reader = new PhoenixReader(config, env); break; + case PluginNameConstants.EMQX_READER : reader = new EmqxReader(config, env); break; + case PluginNameConstants.DM_READER : reader = new DmReader(config, env); break; + case PluginNameConstants.GREENPLUM_READER : reader = new GreenplumReader(config, env); break; + default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } return reader; } - private static BaseDataWriter buildDataWriter(DataTransferConfig config) { + private static BaseDataWriter buildDataWriter(DataTransferConfig config){ String writerName = config.getJob().getContent().get(0).getWriter().getName(); BaseDataWriter writer; - switch (writerName) { - case PluginNameConstants.STREAM_WRITER: - writer = new StreamWriter(config); - break; - case PluginNameConstants.CARBONDATA_WRITER: - writer = new CarbondataWriter(config); - break; - case PluginNameConstants.MYSQL_WRITER: - writer = new MysqlWriter(config); - break; - case PluginNameConstants.SQLSERVER_WRITER: - writer = new SqlserverWriter(config); - break; - case PluginNameConstants.ORACLE_WRITER: - writer = new OracleWriter(config); - break; - case PluginNameConstants.POSTGRESQL_WRITER: - writer = new PostgresqlWriter(config); - break; - case PluginNameConstants.DB2_WRITER: - writer = new Db2Writer(config); - break; - case PluginNameConstants.GBASE_WRITER: - writer = new GbaseWriter(config); - break; - case PluginNameConstants.ES_WRITER: - writer = new EsWriter(config); - break; - case PluginNameConstants.FTP_WRITER: - writer = new FtpWriter(config); - break; - case PluginNameConstants.HBASE_WRITER: - writer = new HbaseWriter(config); - break; - case PluginNameConstants.HDFS_WRITER: - writer = new HdfsWriter(config); - break; - case PluginNameConstants.MONGODB_WRITER: - writer = new MongodbWriter(config); - break; - case PluginNameConstants.ODPS_WRITER: - writer = new OdpsWriter(config); - break; - case PluginNameConstants.REDIS_WRITER: - writer = new RedisWriter(config); - break; - case PluginNameConstants.HIVE_WRITER: - writer = new HiveWriter(config); - break; - case PluginNameConstants.KAFKA09_WRITER: - writer = new Kafka09Writer(config); - break; - case PluginNameConstants.KAFKA10_WRITER: - writer = new Kafka10Writer(config); - break; - case PluginNameConstants.KAFKA11_WRITER: - writer = new Kafka11Writer(config); - break; - case PluginNameConstants.KUDU_WRITER: - writer = new KuduWriter(config); - break; - case PluginNameConstants.CLICKHOUSE_WRITER: - writer = new ClickhouseWriter(config); - break; - case PluginNameConstants.POLARDB_WRITER: - writer = new PolardbWriter(config); - break; - case PluginNameConstants.KAFKA_WRITER: - writer = new KafkaWriter(config); - break; - case PluginNameConstants.PHOENIX_WRITER: - writer = new PhoenixWriter(config); - break; - case PluginNameConstants.EMQX_WRITER: - writer = new EmqxWriter(config); - break; - case PluginNameConstants.DM_WRITER: - writer = new DmWriter(config); - break; - default: - throw new IllegalArgumentException("Can not find writer by name:" + writerName); + switch (writerName){ + case PluginNameConstants.STREAM_WRITER : writer = new StreamWriter(config); break; + case PluginNameConstants.CARBONDATA_WRITER : writer = new CarbondataWriter(config); break; + case PluginNameConstants.MYSQL_WRITER : writer = new MysqlWriter(config); break; + case PluginNameConstants.SQLSERVER_WRITER : writer = new SqlserverWriter(config); break; + case PluginNameConstants.ORACLE_WRITER : writer = new OracleWriter(config); break; + case PluginNameConstants.POSTGRESQL_WRITER : writer = new PostgresqlWriter(config); break; + case PluginNameConstants.DB2_WRITER : writer = new Db2Writer(config); break; + case PluginNameConstants.GBASE_WRITER : writer = new GbaseWriter(config); break; + case PluginNameConstants.ES_WRITER : writer = new EsWriter(config); break; + case PluginNameConstants.FTP_WRITER : writer = new FtpWriter(config); break; + case PluginNameConstants.HBASE_WRITER : writer = new HbaseWriter(config); break; + case PluginNameConstants.HDFS_WRITER : writer = new HdfsWriter(config); break; + case PluginNameConstants.MONGODB_WRITER : writer = new MongodbWriter(config); break; + case PluginNameConstants.ODPS_WRITER : writer = new OdpsWriter(config); break; + case PluginNameConstants.REDIS_WRITER : writer = new RedisWriter(config); break; + case PluginNameConstants.HIVE_WRITER : writer = new HiveWriter(config); break; + case PluginNameConstants.KAFKA09_WRITER : writer = new Kafka09Writer(config); break; + case PluginNameConstants.KAFKA10_WRITER : writer = new Kafka10Writer(config); break; + case PluginNameConstants.KAFKA11_WRITER : writer = new Kafka11Writer(config); break; + case PluginNameConstants.KUDU_WRITER : writer = new KuduWriter(config); break; + case PluginNameConstants.CLICKHOUSE_WRITER : writer = new ClickhouseWriter(config); break; + case PluginNameConstants.POLARDB_WRITER : writer = new PolardbWriter(config); break; + case PluginNameConstants.KAFKA_WRITER : writer = new KafkaWriter(config); break; + case PluginNameConstants.PHOENIX_WRITER : writer = new PhoenixWriter(config); break; + case PluginNameConstants.EMQX_WRITER : writer = new EmqxWriter(config); break; + case PluginNameConstants.RESTAPI_WRITER : writer = new RestapiWriter(config);break; + case PluginNameConstants.DM_WRITER : writer = new DmWriter(config); break; + case PluginNameConstants.GREENPLUM_WRITER : writer = new GreenplumWriter(config); break; + default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } return writer; } - private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties) { - if (properties == null) { + private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties){ + if(properties == null){ return; } - if (properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null) { + if(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ return; - } else { + }else{ long interval = Long.parseLong(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY).trim()); //start checkpoint every ${interval} @@ -375,7 +290,7 @@ private static void openCheckpointConf(StreamExecutionEnvironment env, Propertie } String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); - if (checkpointTimeoutStr != null) { + if(checkpointTimeoutStr != null){ long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); //checkpoints have to complete within one min,or are discard env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); @@ -387,6 +302,8 @@ private static void openCheckpointConf(StreamExecutionEnvironment env, Propertie env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); env.getCheckpointConfig().enableExternalizedCheckpoints( CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + + env.setStateBackend(new FsStateBackend(new Path("file:///tmp/flinkx_checkpoint"))); env.setRestartStrategy(RestartStrategies.failureRateRestart( FAILURE_RATE, Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index f0f047e6b5..21d07e7c9e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -53,6 +53,10 @@ public class PluginNameConstants { public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; public static final String METADATAHIVE2_READER = "metadatahive2reader"; public static final String DM_READER = "dmreader"; + public static final String METADATATIDB_READER = "metadatatidbreader"; + public static final String METADATAORACLE_READER = "metadataoraclereader"; + public static final String METADATAMYSQL_READER = "metadatamysqlreader"; + public static final String GREENPLUM_READER = "greenplumreader"; public static final String STREAM_WRITER = "streamwriter"; public static final String CARBONDATA_WRITER = "carbondatawriter"; @@ -81,4 +85,5 @@ public class PluginNameConstants { public static final String EMQX_WRITER = "emqxwriter"; public static final String RESTAPI_WRITER = "restapiwriter"; public static final String DM_WRITER = "dmwriter"; + public static final String GREENPLUM_WRITER = "greenplumwriter"; } diff --git a/pom.xml b/pom.xml index 8524112890..821397ba25 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ flinkx-clickhouse flinkx-saphana flinkx-teradata - flinkx-cassandra + flinkx-greenplum flinkx-hdfs flinkx-hive @@ -40,22 +40,22 @@ flinkx-phoenix flinkx-carbondata flinkx-kudu + flinkx-cassandra flinkx-redis flinkx-mongodb flinkx-binlog + flinkx-pgwal flinkx-kb flinkx-kafka09 flinkx-kafka10 flinkx-kafka11 flinkx-kafka - flinkx-emqx - flinkx-pgwal flinkx-pulsar - + flinkx-restapi @@ -66,7 +66,7 @@ 2.7.3 4.5.3 ${basedir}/dev - release_1.8.6 + release_1.8.7 @@ -125,9 +125,16 @@ - org.testng - testng - 6.14.3 + junit + junit + 4.12 + test + + + + org.mockito + mockito-core + 3.0.0 test @@ -143,9 +150,24 @@ + + org.jacoco + jacoco-maven-plugin + 0.7.8 + + + + prepare-agent + report + + + + + org.apache.maven.plugins maven-source-plugin + 3.2.1 attach-sources @@ -168,4 +190,20 @@ + + + + org.jacoco + jacoco-maven-plugin + + + + report + + + + + + + \ No newline at end of file From 178532114b6b0500c5d3f8d67b0a056e617c456e Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 17 Jul 2020 11:01:33 +0800 Subject: [PATCH 069/136] fix error word --- docs/quickstart.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index 71779f930d..626ea18091 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -221,7 +221,7 @@ bin/flinkx -mode yarnPer \ | 名称 | 说明 | 可选值 | 是否必填 | 默认值 | | ------------------ | ------------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---- | ----------------------- | -| **model** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | +| **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | | **job** | 数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息 | 无 | 是 | 无 | | **jobid** | 任务名称 | 无 | 否 | Flink Job | | **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/plugins | From 5ccd38836ec6b9546f173a6e21468039dece2ea4 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 17 Jul 2020 11:05:43 +0800 Subject: [PATCH 070/136] fix conProp setting --- .../perjob/FlinkPerJobResourceUtil.java | 36 +++++++++---------- .../launcher/perjob/PerJobSubmitter.java | 2 +- .../deployment/ClusterSpecification.java | 11 +++--- 3 files changed, 23 insertions(+), 26 deletions(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java index 508098067d..bef34a032e 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/FlinkPerJobResourceUtil.java @@ -19,6 +19,10 @@ import com.dtstack.flinkx.util.ValueUtil; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; import java.util.Properties; @@ -33,46 +37,42 @@ public class FlinkPerJobResourceUtil { * the minimum memory should be higher than the min heap cutoff */ public final static int MIN_JM_MEMORY = 768; - public final static int MIN_TM_MEMORY = 768; - public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; - public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; - public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; /** * the specification of this per-job mode cost * @param conProp taskParams * @return */ - public static ClusterSpecification createClusterSpecification(Properties conProp){ - int jobmanagerMemoryMb = 768; - int taskmanagerMemoryMb = 768; - int numberTaskManagers = 1; - int slotsPerTaskManager = 1; + public static ClusterSpecification createClusterSpecification(Properties conProp, Configuration flinkConfig){ + int jobManagerMemoryMb = ConfigurationUtils.getJobManagerHeapMemory(flinkConfig).getMebiBytes(); + int taskManagerMemoryMb = ConfigurationUtils.getTaskManagerHeapMemory(flinkConfig).getMebiBytes(); + int numberTaskManagers = flinkConfig.getInteger(NUMBER_TASK_MANAGERS, 1); + int slots = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, 1); if(conProp != null){ - if(conProp.containsKey(JOBMANAGER_MEMORY_MB)){ - jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); + if(conProp.containsKey(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.key())){ + jobManagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.key()))); } - if(conProp.containsKey(TASKMANAGER_MEMORY_MB)){ - taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); + if(conProp.containsKey(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY.key())){ + taskManagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY.key()))); } if (conProp.containsKey(NUMBER_TASK_MANAGERS)){ numberTaskManagers = ValueUtil.getInt(conProp.get(NUMBER_TASK_MANAGERS)); } - if (conProp.containsKey(SLOTS_PER_TASKMANAGER)){ - slotsPerTaskManager = ValueUtil.getInt(conProp.get(SLOTS_PER_TASKMANAGER)); + if (conProp.containsKey(TaskManagerOptions.NUM_TASK_SLOTS.key())){ + slots = ValueUtil.getInt(conProp.get(TaskManagerOptions.NUM_TASK_SLOTS.key())); } } return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobmanagerMemoryMb) - .setTaskManagerMemoryMB(taskmanagerMemoryMb) + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) .setNumberTaskManagers(numberTaskManagers) - .setSlotsPerTaskManager(slotsPerTaskManager) + .setSlotsPerTaskManager(slots) .createClusterSpecification(); } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java index 81c6387395..d3d044a21e 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perjob/PerJobSubmitter.java @@ -65,7 +65,7 @@ public static String submit(Options options, File jarFile, String[] programArgs) flinkConfig.setString("classloader.resolve-order", "child-first"); Properties conProp = GsonUtil.GSON.fromJson(options.getConfProp(), Properties.class); - ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp); + ClusterSpecification clusterSpecification = FlinkPerJobResourceUtil.createClusterSpecification(conProp, flinkConfig); clusterSpecification.setCreateProgramDelay(true); clusterSpecification.setConfiguration(flinkConfig); clusterSpecification.setClasspaths(new ArrayList<>()); diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java index 2e12164d90..2139dd5251 100644 --- a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java @@ -20,10 +20,7 @@ import com.dtstack.flinkx.launcher.ClassLoaderType; import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.*; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -192,10 +189,10 @@ public String toString() { } public static ClusterSpecification fromConfiguration(Configuration configuration) { - int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + int slots = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, 1); - int jobManagerMemoryMb = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); - int taskManagerMemoryMb = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); + int jobManagerMemoryMb = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); + int taskManagerMemoryMb = ConfigurationUtils.getTaskManagerHeapMemory(configuration).getMebiBytes(); return new ClusterSpecificationBuilder() .setMasterMemoryMB(jobManagerMemoryMb) From 577127e47bee705988ca353b9e66d68c9b11e549 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 18 Aug 2020 10:06:45 +0800 Subject: [PATCH 071/136] =?UTF-8?q?=E5=88=A0=E9=99=A4=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E9=85=8D=E7=BD=AE=E7=9A=84=E9=87=8D=E5=90=AF=E7=AD=96=E7=95=A5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/src/main/java/com/dtstack/flinkx/Main.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index b36613bdb7..dc6177a5f8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -110,7 +110,6 @@ public static void main(String[] args) throws Exception { SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); env.setParallelism(speedConfig.getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); BaseDataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); if(speedConfig.getReaderChannel() > 0){ From 9ad1e69638108275f6ab7e503840df9ec3166094 Mon Sep 17 00:00:00 2001 From: yanghuaiGit <2774584057@qq.com> Date: Sat, 24 Oct 2020 10:26:41 +0800 Subject: [PATCH 072/136] =?UTF-8?q?=E4=BF=AE=E6=94=B9bug=20=E4=BC=98?= =?UTF-8?q?=E5=8C=96kerberos?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/constants/ConstantValue.java | 5 - flinkx-kingbase/flinkx-kingbase-core/pom.xml | 0 .../kingbase/constants/KingbaseCons.java | 41 ---- .../kingbase/util/KingBaseDatabaseMeta.java | 124 ----------- .../kingbase/util/KingBaseTypeConverter.java | 54 ----- .../flinkx-kingbase-reader/pom.xml | 107 --------- .../kingbase/format/KingbaseInputFormat.java | 89 -------- .../kingbase/reader/KingbaseReader.java | 49 ---- .../flinkx-kingbase-writer/pom.xml | 106 --------- .../kingbase/format/KingbaseOutputFormat.java | 198 ----------------- .../kingbase/writer/KingbaseWriter.java | 85 ------- flinkx-kingbase/pom.xml | 35 --- .../com/dtstack/flinkx/launcher/Launcher.java | 51 +---- .../launcher/perJob/FlinkPerJobUtil.java | 1 - .../flinkx-metadata-hive1-reader/pom.xml | 0 flinkx-metadata-hive1/pom.xml | 0 .../flinkx-metadata-hive2-reader/pom.xml | 0 .../constants/Hive2MetaDataCons.java | 0 .../metadatahive2/constants/HiveDbUtil.java | 210 ------------------ .../inputformat/Metadatahive2InputFormat.java | 0 .../flinkx-metadata-mysql-reader/pom.xml | 0 .../constants/MysqlMetadataCons.java | 0 .../inputformat/MetadatamysqlInputFormat.java | 0 .../flinkx-metadata-oracle-reader/pom.xml | 0 .../constants/OracleMetaDataCons.java | 0 .../MetadataoracleInputFormat.java | 0 .../flinkx-metadata-sqlserver-reader/pom.xml | 0 .../constants/SqlServerMetadataCons.java | 0 .../MetadatasqlserverInputFormat.java | 0 .../flinkx-metadata-tidb-reader/pom.xml | 0 .../constants/TidbMetadataCons.java | 0 .../inputformat/MetadatatidbInputFormat.java | 0 .../dtstack/flinkx/metadata/MetaDataCons.java | 0 .../inputformat/BaseMetadataInputFormat.java | 0 .../flinkx/oraclelogminer/format/LogFile.java | 0 .../oraclelogminer/format/LogMinerConfig.java | 0 .../format/LogMinerConnection.java | 0 .../flinkx/oraclelogminer/util/SqlUtil.java | 0 flinkx-phoenix5/flinkx-phoenix5-core/pom.xml | 0 .../flinkx/phoenix5/util/PhoenixUtil.java | 0 .../JdbcInputFormat.java | 2 - .../flinkx-sqlservercdc-reader/pom.xml | 0 .../reader/SqlservercdcReader.java | 0 .../teradata/format/TeradataInputFormat.java | 7 +- .../com/dtstack/flinkx/test/LocalTest.java | 4 +- 45 files changed, 10 insertions(+), 1158 deletions(-) delete mode 100644 flinkx-kingbase/flinkx-kingbase-core/pom.xml delete mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-reader/pom.xml delete mode 100644 flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-writer/pom.xml delete mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java delete mode 100644 flinkx-kingbase/pom.xml delete mode 100644 flinkx-metadata-hive1/flinkx-metadata-hive1-reader/pom.xml delete mode 100644 flinkx-metadata-hive1/pom.xml delete mode 100644 flinkx-metadata-hive2/flinkx-metadata-hive2-reader/pom.xml delete mode 100644 flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/Hive2MetaDataCons.java delete mode 100644 flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/HiveDbUtil.java delete mode 100644 flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java delete mode 100644 flinkx-metadata-mysql/flinkx-metadata-mysql-reader/pom.xml delete mode 100644 flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java delete mode 100644 flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/inputformat/MetadatamysqlInputFormat.java delete mode 100644 flinkx-metadata-oracle/flinkx-metadata-oracle-reader/pom.xml delete mode 100644 flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/constants/OracleMetaDataCons.java delete mode 100644 flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java delete mode 100644 flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/pom.xml delete mode 100644 flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java delete mode 100644 flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java delete mode 100644 flinkx-metadata-tidb/flinkx-metadata-tidb-reader/pom.xml delete mode 100644 flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java delete mode 100644 flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/inputformat/MetadatatidbInputFormat.java delete mode 100644 flinkx-metadata/flinkx-metadata-core/src/main/java/com/dtstack/flinkx/metadata/MetaDataCons.java delete mode 100644 flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/pom.xml delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index 2fb92424c6..02822b8222 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -66,9 +66,4 @@ public class ConstantValue { public static final long STORE_SIZE_M = 1024L * 1024; - public static final String SHIP_FILE_PLUGIN_LOAD_MODE = "shipfile"; - public static final String CLASS_PATH_PLUGIN_LOAD_MODE = "classpath"; - - public static final String CLASSLOADER_CHILD_FIRST = "child-first"; - public static final String CLASSLOADER_PARENT_FIRST = "parent-first"; } diff --git a/flinkx-kingbase/flinkx-kingbase-core/pom.xml b/flinkx-kingbase/flinkx-kingbase-core/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java deleted file mode 100644 index 52e5733a22..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.constants; - -/** - * kingbase常量 - * - * Company: www.dtstack.com - * @author kunni@dtstack.com - */ - -public class KingbaseCons { - - public static final String DRIVER = "com.kingbase8.Driver"; - public static final String KEY_UPDATE_KEY = "key"; - public static final String INSERT_SQL_MODE_TYPE = "copy"; - - /** - * copy语法分隔符 - */ - public static final String DEFAULT_FIELD_DELIM = "\001"; - public static final String DEFAULT_NULL_DELIM = "\002"; - public static final String LINE_DELIMITER = "\n"; - -} diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java deleted file mode 100644 index bb2cfa9108..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.util; - -import com.dtstack.flinkx.enums.EDatabaseType; -import com.dtstack.flinkx.rdb.BaseDatabaseMeta; -import org.apache.commons.lang3.StringUtils; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import static com.dtstack.flinkx.constants.ConstantValue.COMMA_SYMBOL; -import static com.dtstack.flinkx.constants.ConstantValue.LEFT_PARENTHESIS_SYMBOL; -import static com.dtstack.flinkx.constants.ConstantValue.RIGHT_PARENTHESIS_SYMBOL; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DRIVER; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.KEY_UPDATE_KEY; - -/** - * The class of KingBase database prototype - * - * Company: www.dtstack.com - * @author kunni@dtstack.com - */ - -public class KingBaseDatabaseMeta extends BaseDatabaseMeta { - - @Override - protected String makeValues(List column) { - StringBuilder sb = new StringBuilder(); - sb.append(LEFT_PARENTHESIS_SYMBOL); - for(int i = 0; i < column.size(); ++i) { - if(i != 0) { - sb.append(COMMA_SYMBOL); - } - sb.append(quoteColumn(column.get(i))); - } - sb.append(RIGHT_PARENTHESIS_SYMBOL); - return sb.toString(); - } - - @Override - public EDatabaseType getDatabaseType() { - return EDatabaseType.KingBase; - } - - @Override - public String getDriverClass() { - return DRIVER; - } - - @Override - public String getSqlQueryFields(String tableName) { - return "SELECT * FROM " + tableName + " LIMIT 0"; - } - - @Override - public String getSqlQueryColumnFields(List column, String table) { - return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; - } - - @Override - public String getUpsertStatement(List column, String table, Map> updateKey) { - return "INSERT INTO " + quoteTable(table) - + " (" + quoteColumns(column) + ") VALUES " - + makeValues(column.size()) - + " ON CONFLICT " +makeValues(updateKey.get(KEY_UPDATE_KEY)) + " DO UPDATE SET " - + makeUpdatePart(column); - } - - private String makeValues(int nCols) { - return LEFT_PARENTHESIS_SYMBOL + StringUtils.repeat("?", ",", nCols) + RIGHT_PARENTHESIS_SYMBOL; - } - - private String makeUpdatePart (List column) { - List updateList = new ArrayList<>(); - for(String col : column) { - String quotedCol = quoteColumn(col); - updateList.add(quotedCol + "=EXCLUDED." + quotedCol); - } - return StringUtils.join(updateList, COMMA_SYMBOL); - } - - @Override - public String quoteValue(String value, String column) { - return String.format("\"%s\" as %s",value,column); - } - - @Override - public String getSplitFilter(String columnName) { - return String.format("mod(%s, ${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); - } - - @Override - public String getSplitFilterWithTmpTable(String tmpTable, String columnName) { - return String.format("mod(%s.%s, ${N}) = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); - } - - @Override - public String getStartQuote() { - return ""; - } - - @Override - public String getEndQuote() { - return ""; - } -} diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java deleted file mode 100644 index 586b6892c1..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.dtstack.flinkx.kingbase.util; - -import com.dtstack.flinkx.rdb.type.TypeConverterInterface; -import org.apache.commons.lang3.StringUtils; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** - * kingbase中不常用类型转换为java类型 - * @Company: www.dtstack.com - * @author kunni@dtstack.com - */ -public class KingBaseTypeConverter implements TypeConverterInterface { - - private List stringTypes = Arrays.asList("uuid", "xml", "cidr", "inet", "macaddr", "text", "character", "character varying"); - - private List byteTypes = Arrays.asList("bytea","bit varying"); - - private List bitTypes = Collections.singletonList("bit"); - - private List doubleTypes = Arrays.asList("double precision","double","float8","money"); - - private List intTypes = Arrays.asList("int","int2","int4","int8","integer","bigint","bigserial","smallint"); - - @Override - public Object convert(Object data,String typeName) { - if (data == null){ - return null; - } - String dataValue = data.toString(); - if(stringTypes.contains(typeName)){ - return dataValue; - } - if(StringUtils.isBlank(dataValue)){ - return null; - } - if(doubleTypes.contains(typeName)){ - if(StringUtils.startsWith(dataValue, "$")){ - dataValue = StringUtils.substring(dataValue, 1); - } - data = Double.parseDouble(dataValue); - } else if(bitTypes.contains(typeName)){ - // - }else if(byteTypes.contains(typeName)){ - data = Byte.valueOf(dataValue); - } else if(intTypes.contains(typeName)){ - data = Integer.parseInt(dataValue); - } - - return data; - } -} diff --git a/flinkx-kingbase/flinkx-kingbase-reader/pom.xml b/flinkx-kingbase/flinkx-kingbase-reader/pom.xml deleted file mode 100644 index 613433025e..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-reader/pom.xml +++ /dev/null @@ -1,107 +0,0 @@ - - - - flinkx-kingbase - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-kingbase-reader - - - com.dtstack.flinkx - flinkx-rdb-reader - 1.6 - provided - - - com.dtstack.flinkx - flinkx-kingbase-core - 1.6 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.kingbasereader.io.netty - - - com.google.common - shade.kingbasereader.com.google.common - - - com.google.thirdparty - shade.kingbasereader.com.google.thirdparty - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java deleted file mode 100644 index 0ff86b4a5e..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.format; - -import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; -import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.types.Row; - -import java.io.IOException; -import java.sql.ResultSet; - -import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; - -/** - * - * - * Company: www.dtstack.com - * @author kunni@Dtstack.com - */ - -public class KingbaseInputFormat extends JdbcInputFormat { - - /** - * 避免Operation requires a scrollable ResultSet问题 - * @param inputSplit 分片 - * @throws IOException IO异常 - */ - @Override - public void openInternal(InputSplit inputSplit) throws IOException { - if(incrementConfig.isPolling()){ - resultSetType = ResultSet.TYPE_SCROLL_INSENSITIVE; - } - super.openInternal(inputSplit); - } - - @Override - public Row nextRecordInternal(Row row) throws IOException { - if (!hasNext) { - return null; - } - row = new Row(columnCount); - - try { - for (int pos = 0; pos < row.getArity(); pos++) { - Object obj = resultSet.getObject(pos + 1); - if(obj != null) { - if(CollectionUtils.isNotEmpty(columnTypeList)) { - String columnType = columnTypeList.get(pos); - if("year".equalsIgnoreCase(columnType)) { - java.util.Date date = (java.util.Date) obj; - obj = DateUtil.dateToYearString(date); - } else if("tinyint".equalsIgnoreCase(columnType) - || "bit".equalsIgnoreCase(columnType)) { - if(obj instanceof Boolean) { - obj = ((Boolean) obj ? 1 : 0); - } - } - } - obj = clobToString(obj); - } - - row.setField(pos, obj); - } - return super.nextRecordInternal(row); - }catch (Exception e) { - throw new IOException("Couldn't read data - " + ExceptionUtil.getErrorMessage(e), e); - } - } - -} diff --git a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java deleted file mode 100644 index a6625225a8..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.reader; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.kingbase.util.KingBaseDatabaseMeta; -import com.dtstack.flinkx.kingbase.format.KingbaseInputFormat; -import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; -import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** - * KingBase reader plugin - * - * Company: www.dtstack.com - * @author kunni@dtstack.com - */ - -public class KingbaseReader extends JdbcDataReader { - - public KingbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - String schema = config.getJob().getContent().get(0).getReader().getParameter().getConnection().get(0).getSchema(); - table = schema + ConstantValue.POINT_SYMBOL + table; - setDatabaseInterface(new KingBaseDatabaseMeta()); - } - - @Override - protected JdbcInputFormatBuilder getBuilder() { - return new JdbcInputFormatBuilder(new KingbaseInputFormat()); - } -} diff --git a/flinkx-kingbase/flinkx-kingbase-writer/pom.xml b/flinkx-kingbase/flinkx-kingbase-writer/pom.xml deleted file mode 100644 index a4a8831300..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-writer/pom.xml +++ /dev/null @@ -1,106 +0,0 @@ - - - - flinkx-kingbase - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-kingbase-writer - - - com.dtstack.flinkx - flinkx-rdb-writer - 1.6 - provided - - - com.dtstack.flinkx - flinkx-kingbase-core - 1.6 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - org.slf4j:slf4j-api - log4j:log4j - ch.qos.logback:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - shade.kingbasewriter.io.netty - - - com.google.common - shade.kingbasewriter.com.google.common - - - com.google.thirdparty - shade.kingbasewriter.com.google.thirdparty - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java deleted file mode 100644 index 6ff0f8832f..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java +++ /dev/null @@ -1,198 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.format; - -import com.dtstack.flinkx.enums.EWriteMode; -import com.dtstack.flinkx.exception.WriteRecordException; -import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.StringUtil; -import com.kingbase8.copy.CopyManager; -import com.kingbase8.core.BaseConnection; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.types.Row; - -import java.io.ByteArrayInputStream; -import java.nio.charset.StandardCharsets; -import java.sql.PreparedStatement; -import java.sql.SQLException; - -import static com.dtstack.flinkx.constants.ConstantValue.COMMA_SYMBOL; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DEFAULT_FIELD_DELIM; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DEFAULT_NULL_DELIM; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.INSERT_SQL_MODE_TYPE; -import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.LINE_DELIMITER; - -/** - * 写入数据到kingbase - * Company: www.dtstack.com - * @author kunni@dtstack.com - */ - -public class KingbaseOutputFormat extends JdbcOutputFormat { - - private static final String COPY_SQL_TEMPL = "COPY %s(%s) FROM STDIN DELIMITER '%s' NULL AS '%s'"; - - private String copySql = ""; - - private CopyManager copyManager; - - - @Override - protected PreparedStatement prepareTemplates() throws SQLException { - if(CollectionUtils.isEmpty(fullColumn)) { - fullColumn = column; - } - - //check is use copy mode for insert - if (EWriteMode.INSERT.name().equalsIgnoreCase(mode) && checkIsCopyMode(insertSqlMode)) { - copyManager = new CopyManager((BaseConnection) dbConn); - copySql = String.format(COPY_SQL_TEMPL, table, String.join(COMMA_SYMBOL, column), DEFAULT_FIELD_DELIM, DEFAULT_NULL_DELIM); - return null; - } - - return super.prepareTemplates(); - } - - @Override - protected void openInternal(int taskNumber, int numTasks){ - super.openInternal(taskNumber, numTasks); - try { - if (batchInterval > 1) { - dbConn.setAutoCommit(false); - } - } catch (Exception e) { - LOG.warn(ExceptionUtil.getErrorMessage(e)); - } - } - - @Override - protected void writeSingleRecordInternal(Row row) throws WriteRecordException { - if(!checkIsCopyMode(insertSqlMode)){ - if (batchInterval == 1) { - super.writeSingleRecordInternal(row); - } else { - writeSingleRecordCommit(row); - } - - return; - } - - //write with copy - int index = 0; - try { - StringBuilder sb = new StringBuilder(); - for (; index < row.getArity(); index++) { - Object rowData = getField(row, index); - sb.append(rowData) - .append(DEFAULT_FIELD_DELIM); - } - - String rowVal = sb.toString(); - ByteArrayInputStream bi = new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8)); - copyManager.copyIn(copySql, bi); - } catch (Exception e) { - processWriteException(e, index, row); - } - } - - private void writeSingleRecordCommit(Row row) throws WriteRecordException { - try { - super.writeSingleRecordInternal(row); - try { - dbConn.commit(); - } catch (Exception e) { - // 提交失败直接结束任务 - throw new RuntimeException(e); - } - } catch (WriteRecordException e) { - try { - dbConn.rollback(); - } catch (Exception e1) { - // 回滚失败直接结束任务 - throw new RuntimeException(e); - } - - throw e; - } - } - - @Override - protected void writeMultipleRecordsInternal() throws Exception { - if(!checkIsCopyMode(insertSqlMode)){ - writeMultipleRecordsCommit(); - return; - } - - StringBuilder sb = new StringBuilder(128); - for (Row row : rows) { - int lastIndex = row.getArity() - 1; - for (int index = 0; index < row.getArity(); index++) { - Object rowData = getField(row, index); - sb.append(rowData==null ? DEFAULT_NULL_DELIM : rowData); - if(index != lastIndex){ - sb.append(DEFAULT_FIELD_DELIM); - } - } - - sb.append(LINE_DELIMITER); - } - - String rowVal = sb.toString(); - ByteArrayInputStream bi = new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8)); - copyManager.copyIn(copySql, bi); - - if(restoreConfig.isRestore()){ - rowsOfCurrentTransaction += rows.size(); - } - } - - private void writeMultipleRecordsCommit() throws Exception { - try { - super.writeMultipleRecordsInternal(); - dbConn.commit(); - } catch (Exception e){ - dbConn.rollback(); - throw e; - } - } - - @Override - protected Object getField(Row row, int index) { - Object field = super.getField(row, index); - String type = columnType.get(index); - field = typeConverter.convert(field,type); - - return field; - } - - private boolean checkIsCopyMode(String insertMode){ - if(StringUtils.isEmpty(insertMode)){ - return false; - } - - if(!INSERT_SQL_MODE_TYPE.equalsIgnoreCase(insertMode)){ - throw new RuntimeException("not support insertSqlMode:" + insertMode); - } - - return true; - } - -} diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java deleted file mode 100644 index 3e2acf2a63..0000000000 --- a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kingbase.writer; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.kingbase.util.KingBaseDatabaseMeta; -import com.dtstack.flinkx.kingbase.util.KingBaseTypeConverter; -import com.dtstack.flinkx.kingbase.format.KingbaseOutputFormat; -import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; -import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; -import com.dtstack.flinkx.streaming.api.functions.sink.DtOutputFormatSinkFunction; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.types.Row; - -import static com.dtstack.flinkx.constants.ConfigConstant.KEY_WRITER; - -/** - * KingBase writer plugin - * - * Company: www.dtstack.com - * @author kunni@dtstack.com - */ - -public class KingbaseWriter extends JdbcDataWriter { - - public KingbaseWriter(DataTransferConfig config) { - super(config); - String schema = config.getJob().getContent().get(0).getWriter().getParameter().getConnection().get(0).getSchema(); - table = schema + ConstantValue.POINT_SYMBOL + table; - setDatabaseInterface(new KingBaseDatabaseMeta()); - setTypeConverterInterface(new KingBaseTypeConverter()); - } - - @Override - public DataStreamSink writeData(DataStream dataSet) { - KingbaseOutputFormat kingBaseOutputFormat = new KingbaseOutputFormat(); - JdbcOutputFormatBuilder builder = new JdbcOutputFormatBuilder(kingBaseOutputFormat); - builder.setDriverName(databaseInterface.getDriverClass()); - builder.setDbUrl(dbUrl); - builder.setUsername(username); - builder.setPassword(password); - builder.setBatchInterval(batchSize); - builder.setMonitorUrls(monitorUrls); - builder.setPreSql(preSql); - builder.setPostSql(postSql); - builder.setErrors(errors); - builder.setErrorRatio(errorRatio); - builder.setDirtyPath(dirtyPath); - builder.setDirtyHadoopConfig(dirtyHadoopConfig); - builder.setSrcCols(srcCols); - builder.setDatabaseInterface(databaseInterface); - builder.setMode(mode); - builder.setTable(table); - builder.setColumn(column); - builder.setFullColumn(fullColumn); - builder.setUpdateKey(updateKey); - builder.setTypeConverter(typeConverter); - builder.setRestoreConfig(restoreConfig); - builder.setInsertSqlMode(insertSqlMode); - - DtOutputFormatSinkFunction sinkFunction = new DtOutputFormatSinkFunction(builder.finish()); - DataStreamSink dataStreamSink = dataSet.addSink(sinkFunction); - String sinkName = (databaseInterface.getDatabaseType() + KEY_WRITER).toLowerCase(); - dataStreamSink.name(sinkName); - return dataStreamSink; - } -} diff --git a/flinkx-kingbase/pom.xml b/flinkx-kingbase/pom.xml deleted file mode 100644 index 9915d66707..0000000000 --- a/flinkx-kingbase/pom.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - pom - - flinkx-kingbase-core - flinkx-kingbase-reader - flinkx-kingbase-writer - - - flinkx-kingbase - - - - com.dtstack.flinkx - flinkx-core - 1.6 - provided - - - com.dtstack.flinkx - flinkx-rdb-core - 1.6 - provided - - - - \ No newline at end of file diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index a5f9817e91..65b954a53f 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -73,55 +73,7 @@ public static void main(String[] args) throws Exception { findDefaultConfigDir(launcherOptions); List argList = optionParser.getProgramExeArgList(); - switch (ClusterMode.getByName(launcherOptions.getMode())) { - case local: - com.dtstack.flinkx.Main.main(argList.toArray(new String[0])); - break; - case standalone: - case yarn: - ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - argList.add("-monitor"); - argList.add(clusterClient.getWebInterfaceURL()); - ClientUtils.submitJob(clusterClient, buildJobGraph(launcherOptions, argList.toArray(new String[0]))); - break; - case yarnPer: - String confProp = launcherOptions.getConfProp(); - if (StringUtils.isBlank(confProp)) { - throw new IllegalArgumentException("per-job mode must have confProp!"); - } - String libJar = launcherOptions.getFlinkLibJar(); - if (StringUtils.isBlank(libJar)) { - throw new IllegalArgumentException("per-job mode must have flink lib path!"); - } - argList.add("-monitor"); - argList.add(""); - PerJobSubmitter.submit(launcherOptions, new JobGraph(), argList.toArray(new String[0])); - } - } - List argList = optionParser.getProgramExeArgList(); - - private static JobGraph buildJobGraph(Options launcherOptions, String[] remoteArgs) throws Exception { - String pluginRoot = launcherOptions.getPluginRoot(); - String content = launcherOptions.getJob(); - String coreJarName = getCoreJarFileName(pluginRoot); - File jarFile = new File(pluginRoot + File.separator + coreJarName); - List urlList = analyzeUserClasspath(content, pluginRoot); - SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.none(); - if (StringUtils.isNotEmpty(launcherOptions.getS())) { - savepointRestoreSettings = SavepointRestoreSettings.forPath(launcherOptions.getS()); - } - PackagedProgram program = PackagedProgram.newBuilder() - .setJarFile(jarFile) - .setUserClassPaths(urlList) - .setEntryPointClassName(MAIN_CLASS) - .setConfiguration(launcherOptions.loadFlinkConfiguration()) - .setSavepointRestoreSettings(savepointRestoreSettings) - .setArguments(remoteArgs) - .build(); - return PackagedProgramUtils.createJobGraph(program, launcherOptions.loadFlinkConfiguration(), Integer.parseInt(launcherOptions.getParallelism()), false); - } - public static List analyzeUserClasspath(String content, String pluginRoot) { // 将argList转化为HashMap,方便通过参数名称来获取参数值 HashMap temp = new HashMap<>(16); for (int i = 0; i < argList.size(); i += 2) { @@ -164,7 +116,7 @@ public static List analyzeUserClasspath(String content, String pluginRoot) } } - private static JobGraph buildJobGraph(Options launcherOptions, String[] remoteArgs) throws Exception { + public static JobGraph buildJobGraph(Options launcherOptions, String[] remoteArgs) throws Exception { String pluginRoot = launcherOptions.getPluginRoot(); String content = launcherOptions.getJob(); String coreJarName = getCoreJarFileName(pluginRoot); @@ -210,7 +162,6 @@ public static List analyzeUserClasspath(String content, String pluginRoot) return urlList; } - private static void findDefaultConfigDir(Options launcherOptions) { findDefaultPluginRoot(launcherOptions); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java index 405e1cc0c0..d9c5c057a8 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java @@ -48,7 +48,6 @@ public class FlinkPerJobUtil { public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; - private static final Logger LOG = LoggerFactory.getLogger(FlinkPerJobUtil.class); /** * the specification of this per-job mode cost diff --git a/flinkx-metadata-hive1/flinkx-metadata-hive1-reader/pom.xml b/flinkx-metadata-hive1/flinkx-metadata-hive1-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-hive1/pom.xml b/flinkx-metadata-hive1/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/pom.xml b/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/Hive2MetaDataCons.java b/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/Hive2MetaDataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/HiveDbUtil.java b/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/HiveDbUtil.java deleted file mode 100644 index f98b7860f8..0000000000 --- a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/constants/HiveDbUtil.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.metadatahive2.constants; - -import com.dtstack.flinkx.authenticate.KerberosUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.FileSystemUtil; -import com.dtstack.flinkx.util.RetryUtil; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.security.PrivilegedAction; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.Map; -import java.util.concurrent.locks.ReentrantLock; - -/** - * @author toutian - */ - -public final class HiveDbUtil { - - public static final String SQLSTATE_USERNAME_PWD_ERROR = "28000"; - public static final String SQLSTATE_CANNOT_ACQUIRE_CONNECT = "08004"; - public static final int JDBC_PART_SIZE = 2; - public static final String JDBC_REGEX = "[?|;|#]"; - public static final String KEY_VAL_DELIMITER = "="; - public static final String PARAM_DELIMITER = "&"; - public static final String KEY_PRINCIPAL = "principal"; - private static Logger LOG = LoggerFactory.getLogger(HiveDbUtil.class); - private static ReentrantLock lock = new ReentrantLock(); - - private HiveDbUtil() { - } - - public static Connection getConnection(ConnectionInfo connectionInfo) { - if(openKerberos(connectionInfo.getJdbcUrl())){ - return getConnectionWithKerberos(connectionInfo); - } else { - return getConnectionWithRetry(connectionInfo); - } - } - - private static Connection getConnectionWithRetry(ConnectionInfo connectionInfo){ - try { - return RetryUtil.executeWithRetry(() -> connect(connectionInfo), 1, 1000L, false); - } catch (Exception e1) { - throw new RuntimeException(String.format("连接:%s 时发生错误:%s.", connectionInfo.getJdbcUrl(), ExceptionUtil.getErrorMessage(e1))); - } - } - - private static Connection getConnectionWithKerberos(ConnectionInfo connectionInfo){ - if(connectionInfo.getHiveConf() == null || connectionInfo.getHiveConf().isEmpty()){ - throw new IllegalArgumentException("hiveConf can not be null or empty"); - } - - String keytabFileName = KerberosUtil.getPrincipalFileName(connectionInfo.getHiveConf()); - - keytabFileName = KerberosUtil.loadFile(connectionInfo.getHiveConf(), keytabFileName); - String principal = KerberosUtil.getPrincipal(connectionInfo.getHiveConf(), keytabFileName); - KerberosUtil.loadKrb5Conf(connectionInfo.getHiveConf()); - - Configuration conf = FileSystemUtil.getConfiguration(connectionInfo.getHiveConf(), null); - - UserGroupInformation ugi; - try { - ugi = KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); - } catch (Exception e){ - throw new RuntimeException("Login kerberos error:", e); - } - - LOG.info("current ugi:{}", ugi); - return ugi.doAs((PrivilegedAction) () -> getConnectionWithRetry(connectionInfo)); - } - - private static boolean openKerberos(final String jdbcUrl){ - String[] splits = jdbcUrl.split(JDBC_REGEX); - if (splits.length != JDBC_PART_SIZE) { - return false; - } - - String paramsStr = splits[1]; - String[] paramArray = paramsStr.split(PARAM_DELIMITER); - for (String param : paramArray) { - String[] keyVal = param.split(KEY_VAL_DELIMITER); - if(KEY_PRINCIPAL.equalsIgnoreCase(keyVal[0])){ - return true; - } - } - - return false; - } - - public static Connection connect(ConnectionInfo connectionInfo) { - lock.lock(); - try { - Class.forName(connectionInfo.getDriver()); - DriverManager.setLoginTimeout(connectionInfo.getTimeout()); - if(StringUtils.isNotBlank(connectionInfo.getUsername())){ - return DriverManager.getConnection(connectionInfo.getJdbcUrl(), connectionInfo.getUsername(), connectionInfo.getPassword()); - }else{ - return DriverManager.getConnection(connectionInfo.getJdbcUrl()); - } - } catch (SQLException e) { - if (SQLSTATE_USERNAME_PWD_ERROR.equals(e.getSQLState())) { - throw new RuntimeException("用户名或密码错误."); - } else if (SQLSTATE_CANNOT_ACQUIRE_CONNECT.equals(e.getSQLState())) { - throw new RuntimeException("应用程序服务器拒绝建立连接."); - } else { - throw new RuntimeException("连接信息:" + connectionInfo.getJdbcUrl() + " 错误信息:" + ExceptionUtil.getErrorMessage(e)); - } - } catch (Exception e1) { - throw new RuntimeException("连接信息:" + connectionInfo.getJdbcUrl() + " 错误信息:" + ExceptionUtil.getErrorMessage(e1)); - } finally { - lock.unlock(); - } - } - - public static class ConnectionInfo{ - private String jdbcUrl; - private String username; - private String password; - private String driver; - private int timeout = 30000; - private Map hiveConf; - - public String getJdbcUrl() { - return jdbcUrl; - } - - public void setJdbcUrl(String jdbcUrl) { - this.jdbcUrl = jdbcUrl; - } - - public String getUsername() { - return username; - } - - public void setUsername(String username) { - this.username = username; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } - - public Map getHiveConf() { - return hiveConf; - } - - public void setHiveConf(Map hiveConf) { - this.hiveConf = hiveConf; - } - - public int getTimeout() { - return timeout; - } - - public void setTimeout(int timeout) { - this.timeout = timeout; - } - - public String getDriver(){ - return driver; - } - - public void setDriver(String driver){ - this.driver = driver; - } - - @Override - public String toString() { - return "ConnectionInfo{" + - "jdbcUrl='" + jdbcUrl + '\'' + - ", username='" + username + '\'' + - ", password='" + password + '\'' + - ", timeout='" + timeout + '\'' + - ", driver='" + driver + '\'' + - ", hiveConf=" + hiveConf + - '}'; - } - } - -} diff --git a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java b/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/pom.xml b/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java b/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/inputformat/MetadatamysqlInputFormat.java b/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/inputformat/MetadatamysqlInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/pom.xml b/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/constants/OracleMetaDataCons.java b/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/constants/OracleMetaDataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java b/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/pom.xml b/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java b/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java b/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/pom.xml b/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java b/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/inputformat/MetadatatidbInputFormat.java b/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/inputformat/MetadatatidbInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata/flinkx-metadata-core/src/main/java/com/dtstack/flinkx/metadata/MetaDataCons.java b/flinkx-metadata/flinkx-metadata-core/src/main/java/com/dtstack/flinkx/metadata/MetaDataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java b/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index 446d48ae15..ee2468b017 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -59,8 +59,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static com.dtstack.flinkx.rdb.util.DbUtil.analyzeTable; - /** * InputFormat for reading data from a database and generate Rows. *

diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/reader/SqlservercdcReader.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java index 68e19a54c4..12dd9b833e 100644 --- a/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java +++ b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/format/TeradataInputFormat.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.sql.SQLException; import java.sql.Statement; +import java.util.List; import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; @@ -42,6 +43,7 @@ public class TeradataInputFormat extends JdbcInputFormat { private static final Logger LOG = LoggerFactory.getLogger(TeradataInputFormat.class); + protected List descColumnTypeList; @Override public void openInternal(InputSplit inputSplit) throws IOException { try { @@ -83,7 +85,10 @@ public void openInternal(InputSplit inputSplit) throws IOException { checkSize(columnCount, metaColumns); hasNext = resultSet.next(); - descColumnTypeList = DbUtil.analyzeColumnType(resultSet); + if(descColumnTypeList == null) { + descColumnTypeList = DbUtil.analyzeColumnType(resultSet,metaColumns); + } + } catch (SQLException se) { throw new IllegalArgumentException("open() failed. " + se.getMessage(), se); diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 6c074cbd40..4e83dcfc1c 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -17,6 +17,8 @@ */ package com.dtstack.flinkx.test; +import ch.qos.logback.classic.Level; +import ch.qos.logback.classic.LoggerContext; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.binlog.reader.BinlogReader; import com.dtstack.flinkx.carbondata.reader.CarbondataReader; @@ -127,7 +129,7 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - String jobPath = "D:\\dtstack\\flinkx-all\\flinkx-test\\src\\main\\resources\\dev_test_job\\metadatasqlserver_stream.json"; + String jobPath = "/Users/yanghuai/Library/Application Support/JetBrains/IntelliJIdea2020.1/scratches/stream_stream.json"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); System.exit(0); From c7e3668083cf1d61f7ea4415829ff9cc32b9cda3 Mon Sep 17 00:00:00 2001 From: yanghuaiGit <2774584057@qq.com> Date: Sat, 24 Oct 2020 10:47:50 +0800 Subject: [PATCH 073/136] =?UTF-8?q?=E4=BC=98=E5=8C=96=E4=BB=A3=E7=A0=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/options/Options.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index ef296a979e..1f379cca2a 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -247,4 +247,28 @@ public String getP() { public void setP(String p) { this.p = p; } + + @Override + public String toString() { + return "Options{" + + "mode='" + mode + '\'' + + ", job='" + job + '\'' + + ", monitor='" + monitor + '\'' + + ", jobid='" + jobid + '\'' + + ", flinkconf='" + flinkconf + '\'' + + ", pluginRoot='" + pluginRoot + '\'' + + ", yarnconf='" + yarnconf + '\'' + + ", parallelism='" + parallelism + '\'' + + ", priority='" + priority + '\'' + + ", queue='" + queue + '\'' + + ", flinkLibJar='" + flinkLibJar + '\'' + + ", confProp='" + confProp + '\'' + + ", p='" + p + '\'' + + ", s='" + s + '\'' + + ", pluginLoadMode='" + pluginLoadMode + '\'' + + ", appId='" + appId + '\'' + + ", remotePluginPath='" + remotePluginPath + '\'' + + ", flinkConfiguration=" + flinkConfiguration + + '}'; + } } From 027cb4b03f75cb7ead9d897edfcc31efb4635d2f Mon Sep 17 00:00:00 2001 From: yanghuaiGit <2774584057@qq.com> Date: Tue, 27 Oct 2020 17:23:45 +0800 Subject: [PATCH 074/136] =?UTF-8?q?=E7=89=88=E6=9C=AC=E4=BF=AE=E6=94=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index ebd632451a..c57526a48e 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -12,7 +12,7 @@ flinkx-test - 1.10.1 + 1.10.0 0.3.0 From 5c0be9e48d68cc9eb9d74c877832b2c9b4f5ac2d Mon Sep 17 00:00:00 2001 From: yanghuaiGit <2774584057@qq.com> Date: Thu, 29 Oct 2020 20:59:23 +0800 Subject: [PATCH 075/136] =?UTF-8?q?=E5=8D=87=E7=BA=A7=E4=B8=BArelease=5F1.?= =?UTF-8?q?10.2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 06e33b3561..dc22ac123a 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ 2.7.3 4.5.3 ${basedir}/dev - release_1.10.1 + release_1.10.2 From 29f0b9262433bb79d346a0be33fe9ec253d346b6 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Thu, 26 Nov 2020 11:09:58 +0800 Subject: [PATCH 076/136] [opt][restapi] remove empty dir --- .../com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java | 0 .../com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java | 0 .../java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java | 0 .../metadatahive2/inputformat/Metadatahive2InputFormat.java | 0 .../dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java | 0 .../flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java | 0 .../inputformat/MetadatasqlserverInputFormat.java | 0 .../dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java | 0 .../flinkx/metadata/inputformat/BaseMetadataInputFormat.java | 0 .../dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java | 0 .../dtstack/flinkx/oraclelogminer/format/LogMinerListener.java | 0 flinkx-oraclelogminer/pom.xml | 0 .../java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java | 0 .../flinkx/sqlservercdc/listener/SqlServerCdcListener.java | 0 14 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java delete mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java delete mode 100644 flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java delete mode 100644 flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java delete mode 100644 flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java delete mode 100644 flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java delete mode 100644 flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java delete mode 100644 flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java delete mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java delete mode 100644 flinkx-oraclelogminer/pom.xml delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java delete mode 100644 flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java b/flinkx-metadata-hive2/flinkx-metadata-hive2-reader/src/main/java/com/dtstack/flinkx/metadatahive2/inputformat/Metadatahive2InputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java b/flinkx-metadata-mysql/flinkx-metadata-mysql-reader/src/main/java/com/dtstack/flinkx/metadatamysql/constants/MysqlMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java b/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/constants/SqlServerMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java b/flinkx-metadata-sqlserver/flinkx-metadata-sqlserver-reader/src/main/java/com/dtstack/flinkx/metadatasqlserver/inputformat/MetadatasqlserverInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java b/flinkx-metadata-tidb/flinkx-metadata-tidb-reader/src/main/java/com/dtstack/flinkx/metadatatidb/constants/TidbMetadataCons.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java b/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-oraclelogminer/pom.xml b/flinkx-oraclelogminer/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-core/src/main/java/com/dtstack/flinkx/sqlservercdc/SqlServerCdcUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java b/flinkx-sqlservercdc/flinkx-sqlservercdc-reader/src/main/java/com/dtstack/flinkx/sqlservercdc/listener/SqlServerCdcListener.java deleted file mode 100644 index e69de29bb2..0000000000 From 067e0e01d6f466dcfdcef874c7c681555b3f12b4 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Thu, 26 Nov 2020 11:16:06 +0800 Subject: [PATCH 077/136] [opt][restapi] remove empty dir --- flinkx-phoenix5/flinkx-phoenix5-core/pom.xml | 0 .../main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java | 0 flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml | 0 .../com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java | 0 flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml | 0 5 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/pom.xml delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java delete mode 100644 flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml deleted file mode 100644 index e69de29bb2..0000000000 From ae3d6db70aaf6477bd6ef98df8fda8c178a1cf42 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Thu, 26 Nov 2020 11:58:40 +0800 Subject: [PATCH 078/136] [opt][restapi] remove unUse variable --- .../com/dtstack/flinkx/restapi/inputformat/HttpClient.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java index ea6139798b..a3cc581dd5 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java @@ -162,7 +162,9 @@ public Map initData(HashMap data, String[] names if (i != names.length - 1) { HashMap objectObjectHashMap = new HashMap(4); Object value = tempHashMap.putIfAbsent(names[i], objectObjectHashMap); - if (value instanceof String) { + if(Objects.isNull(value)){ + tempHashMap = objectObjectHashMap; + }else if (value instanceof String) { try { Map o = GsonUtil.GSON.fromJson((String) value, GsonUtil.gsonMapTypeToken); tempHashMap.put(names[i], o); From c94fdab285ab0f08341c7164c64e23d81ef2caf6 Mon Sep 17 00:00:00 2001 From: lvyanquan Date: Tue, 15 Dec 2020 19:52:23 +0800 Subject: [PATCH 079/136] =?UTF-8?q?[merge-32861][launcher]=20=E8=A7=A3?= =?UTF-8?q?=E5=86=B3launcher=E6=8F=90=E4=BA=A4=E5=A4=B1=E8=B4=A5=E7=9A=84?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/launcher/Launcher.java | 11 ----- .../perJob/PerJobClusterClientBuilder.java | 15 +++--- .../src/main/resources/log4j.properties | 48 +++++++++++++++++++ 3 files changed, 56 insertions(+), 18 deletions(-) create mode 100644 flinkx-launcher/src/main/resources/log4j.properties diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 37dfaf746e..960157ca34 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -17,8 +17,6 @@ */ package com.dtstack.flinkx.launcher; -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.enums.ClusterMode; @@ -35,7 +33,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.Preconditions; -import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileInputStream; @@ -64,7 +61,6 @@ public class Launcher { public static final String MAIN_CLASS = "com.dtstack.flinkx.Main"; public static void main(String[] args) throws Exception { - setLogLevel(Level.DEBUG.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); findDefaultConfigDir(launcherOptions); @@ -248,11 +244,4 @@ private static String readJob(String job) { throw new RuntimeException(e); } } - - private static void setLogLevel(String level) { - LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); - //设置全局日志级别 - ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel(level)); - } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index 97983dfa5b..cf8c8caef5 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -98,13 +98,14 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptio } else { throw new IllegalArgumentException("The Flink jar path is null"); } - File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - if(logback.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - }else{ - File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); - if(log4j.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + + File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + if(log4j.exists()){ + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + } else{ + File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + if(logback.exists()){ + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); } } diff --git a/flinkx-launcher/src/main/resources/log4j.properties b/flinkx-launcher/src/main/resources/log4j.properties new file mode 100644 index 0000000000..07e589ff7e --- /dev/null +++ b/flinkx-launcher/src/main/resources/log4j.properties @@ -0,0 +1,48 @@ +############# +# 输出到控制台 +############# + +# log4j.rootLogger日志输出类别和级别:只输出不低于该级别的日志信息 DEBUG < INFO < WARN < ERROR < FATAL +# WARN:日志级别 CONSOLE:输出位置自己定义的一个名字 logfile:输出位置自己定义的一个名字 +log4j.rootLogger=INFO,CONSOLE,logfile +# 配置CONSOLE输出到控制台 +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# 配置CONSOLE设置为自定义布局模式 +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +# 配置CONSOLE日志的输出格式 2019-08-22 22:52:12,000 %r耗费毫秒数 %p日志的优先级 %t线程名 %C所属类名通常为全类名 %L代码中的行号 %x线程相关联的NDC %m日志 %n换行 +log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} - %-4r %-5p [%t] %C:%m%n + +################ +# 输出到日志文件中 +################ + +## 配置logfile输出到文件中 文件大小到达指定尺寸的时候产生新的日志文件 +#log4j.appender.logfile=org.apache.log4j.RollingFileAppender +## 保存编码格式 +#log4j.appender.logfile.Encoding=UTF-8 +## 输出文件位置此为项目根目录下的logs文件夹中 +#log4j.appender.logfile.File=logs/root.log +## 后缀可以是KB,MB,GB达到该大小后创建新的日志文件 +#log4j.appender.logfile.MaxFileSize=10MB +## 设置滚定文件的最大值3 指可以产生root.log.1、root.log.2、root.log.3和root.log四个日志文件 +#log4j.appender.logfile.MaxBackupIndex=3 +## 配置logfile为自定义布局模式 +#log4j.appender.logfile.layout=org.apache.log4j.PatternLayout +#log4j.appender.logfile.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %F %p %m%n +# +########################### +## 对不同的类输出不同的日志文件 +########################### +# +## club.bagedate包下的日志单独输出 +#log4j.logger.club.bagedate=DEBUG,bagedate +## 设置为false该日志信息就不会加入到rootLogger中了 +#log4j.additivity.club.bagedate=false +## 下面就和上面配置一样了 +#log4j.appender.bagedate=org.apache.log4j.RollingFileAppender +#log4j.appender.bagedate.Encoding=UTF-8 +#log4j.appender.bagedate.File=logs/bagedate.log +#log4j.appender.bagedate.MaxFileSize=10MB +#log4j.appender.bagedate.MaxBackupIndex=3 +#log4j.appender.bagedate.layout=org.apache.log4j.PatternLayout +#log4j.appender.bagedate.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %F %p %m%n From d857e928d96855464f340572ec57c9f7d9da3b85 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Wed, 16 Dec 2020 17:26:24 +0800 Subject: [PATCH 080/136] =?UTF-8?q?[readme]=E4=BF=AE=E6=94=B9=E6=8B=9B?= =?UTF-8?q?=E8=81=98=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README_CH.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/README_CH.md b/README_CH.md index 8bb8c11dd0..74c4fa5a17 100644 --- a/README_CH.md +++ b/README_CH.md @@ -7,7 +7,18 @@ FlinkX # 技术交流 -- 招聘**大数据平台开发工程师**,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) +- 招聘**Flink研发工程师**,如果有兴趣可以联系思枢(微信号:ysqwhiletrue)
+Flink开发工程师JD要求:
+1.负责袋鼠云基于Flink的衍生框架数据同步flinkx和实时计算flinkstreamsql框架的开发;
+2.调研和把握当前最新大数据实时计算技术,将其中的合适技术引入到平台中,改善产品,提升竞争力;
+职位要求:
+1、本科及以上学历,3年及以上的Flink开发经验,精通Java,熟悉Scala、Python优先考虑;
+2、熟悉Flink原理,有基于Flink做过二次源码的开发,在github上贡献者Flink源码者优先;
+3、有机器学习、数据挖掘相关经验者优先;
+4、对新技术有快速学习和上手能力,对代码有一定的洁癖;
+加分项:
+1.在GitHub或其他平台上有过开源项目
+可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) - 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群 From fbe541b050994d47835db42cf7e774632c6dfcee Mon Sep 17 00:00:00 2001 From: lvyanquan Date: Wed, 16 Dec 2020 19:37:22 +0800 Subject: [PATCH 081/136] =?UTF-8?q?[feat][launcher]=20=E9=BB=98=E8=AE=A4?= =?UTF-8?q?=E7=BB=91=E5=AE=9Alog4j=E5=B9=B6=E6=B7=BB=E5=8A=A0=E9=85=8D?= =?UTF-8?q?=E7=BD=AE=E6=96=87=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/launcher/Launcher.java | 8 ---- .../perJob/PerJobClusterClientBuilder.java | 14 +++--- .../src/main/resources/log4j.properties | 48 +++++++++++++++++++ 3 files changed, 55 insertions(+), 15 deletions(-) create mode 100644 flinkx-launcher/src/main/resources/log4j.properties diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 65b954a53f..88512a9797 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -67,7 +67,6 @@ public class Launcher { public static final String MAIN_CLASS = "com.dtstack.flinkx.Main"; public static void main(String[] args) throws Exception { - setLogLevel(Level.DEBUG.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); findDefaultConfigDir(launcherOptions); @@ -267,11 +266,4 @@ private static String readJob(String job) { throw new RuntimeException(e); } } - - private static void setLogLevel(String level) { - LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); - //设置全局日志级别 - ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel(level)); - } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index 97983dfa5b..e63c25b874 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -98,13 +98,13 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptio } else { throw new IllegalArgumentException("The Flink jar path is null"); } - File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - if(logback.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - }else{ - File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); - if(log4j.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + if(log4j.exists()){ + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + }else { + File logback = new File(launcherOptions.getFlinkconf() + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + if (logback.exists()) { + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf() + File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); } } diff --git a/flinkx-launcher/src/main/resources/log4j.properties b/flinkx-launcher/src/main/resources/log4j.properties new file mode 100644 index 0000000000..5e1396994a --- /dev/null +++ b/flinkx-launcher/src/main/resources/log4j.properties @@ -0,0 +1,48 @@ +############# +# ̨ +############# + +# log4j.rootLogger־ͼֻڸü־Ϣ DEBUG < INFO < WARN < ERROR < FATAL +# WARN־ CONSOLEλԼһ logfileλԼһ +log4j.rootLogger=INFO,CONSOLE,logfile +# CONSOLĘ +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# CONSOLEΪԶ岼ģʽ +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +# CONSOLE־ʽ 2019-08-22 22:52:12,000 %rķѺ %p־ȼ %t߳ %CͨΪȫ %Lек %x߳NDC %m־ %n +log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} - %-4r %-5p [%t] %C:%m%n + +################ +# ־ļ +################ + +## logfileļ ļСָߴʱµ־ļ +#log4j.appender.logfile=org.apache.log4j.RollingFileAppender +## ʽ +#log4j.appender.logfile.Encoding=UTF-8 +## ļλôΪĿĿ¼µlogsļ +#log4j.appender.logfile.File=logs/root.log +## ׺KB,MB,GBﵽôС󴴽µ־ļ +#log4j.appender.logfile.MaxFileSize=10MB +## ùļֵ3 ָԲroot.log.1root.log.2root.log.3root.logĸ־ļ +#log4j.appender.logfile.MaxBackupIndex=3 +## logfileΪԶ岼ģʽ +#log4j.appender.logfile.layout=org.apache.log4j.PatternLayout +#log4j.appender.logfile.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %F %p %m%n +# +########################### +## Բͬͬ־ļ +########################### +# +## club.bagedateµ־ +#log4j.logger.club.bagedate=DEBUG,bagedate +## Ϊfalse־ϢͲ뵽rootLogger +#log4j.additivity.club.bagedate=false +## ͺһ +#log4j.appender.bagedate=org.apache.log4j.RollingFileAppender +#log4j.appender.bagedate.Encoding=UTF-8 +#log4j.appender.bagedate.File=logs/bagedate.log +#log4j.appender.bagedate.MaxFileSize=10MB +#log4j.appender.bagedate.MaxBackupIndex=3 +#log4j.appender.bagedate.layout=org.apache.log4j.PatternLayout +#log4j.appender.bagedate.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %F %p %m%n \ No newline at end of file From dd33c2488b899da58ffe1f4cc075be671e4fd244 Mon Sep 17 00:00:00 2001 From: lvyanquan Date: Wed, 16 Dec 2020 19:43:39 +0800 Subject: [PATCH 082/136] =?UTF-8?q?[feat][core]=20=E8=A7=A3=E5=86=B3?= =?UTF-8?q?=E5=9C=A81.10.1=E4=B8=8A=E6=8F=90=E4=BA=A4=E5=87=BA=E7=8E=B0?= =?UTF-8?q?=E5=86=B2=E7=AA=81=E7=9A=84=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/api/java/ExecutionEnvironment.java | 1278 ----------------- 1 file changed, 1278 deletions(-) delete mode 100644 flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java diff --git a/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java deleted file mode 100644 index fb7435c175..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ /dev/null @@ -1,1278 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.java; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.Public; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.operators.OperatorInformation; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.io.CollectionInputFormat; -import org.apache.flink.api.java.io.CsvReader; -import org.apache.flink.api.java.io.IteratorInputFormat; -import org.apache.flink.api.java.io.ParallelIteratorInputFormat; -import org.apache.flink.api.java.io.PrimitiveInputFormat; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.io.TextValueInputFormat; -import org.apache.flink.api.java.operators.DataSink; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.operators.Operator; -import org.apache.flink.api.java.operators.OperatorTranslation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.NumberSequenceIterator; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SplittableIterator; -import org.apache.flink.util.Visitor; - -import com.esotericsoftware.kryo.Serializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * The ExecutionEnvironment is the context in which a program is executed. A - * {@link LocalEnvironment} will cause execution in the current JVM, a - * {@link RemoteEnvironment} will cause execution on a remote setup. - * - *

The environment provides methods to control the job execution (such as setting the parallelism) - * and to interact with the outside world (data access). - * - *

Please note that the execution environment needs strong type information for the input and return types - * of all operations that are executed. This means that the environments needs to know that the return - * value of an operation is for example a Tuple of String and Integer. - * Because the Java compiler throws much of the generic type information away, most methods attempt to re- - * obtain that information using reflection. In certain cases, it may be necessary to manually supply that - * information to some of the methods. - * - * @see LocalEnvironment - * @see RemoteEnvironment - */ -@Public -public abstract class ExecutionEnvironment { - - /** The logger used by the environment and its subclasses. */ - protected static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); - - /** The environment of the context (local by default, cluster if invoked through command line). */ - private static ThreadLocal contextEnvironmentFactory = new ThreadLocal<>(); - - /** The default parallelism used by local environments. */ - private static int defaultLocalDop = Runtime.getRuntime().availableProcessors(); - - // -------------------------------------------------------------------------------------------- - - private final List> sinks = new ArrayList<>(); - - private final List> cacheFile = new ArrayList<>(); - - private final ExecutionConfig config = new ExecutionConfig(); - - /** Result from the latest execution, to make it retrievable when using eager execution methods. */ - protected JobExecutionResult lastJobExecutionResult; - - /** The ID of the session, defined by this execution environment. Sessions and Jobs are same in - * Flink, as Jobs can consist of multiple parts that are attached to the growing dataflow graph. */ - protected JobID jobID; - - /** The session timeout in seconds. */ - protected long sessionTimeout; - - /** Flag to indicate whether sinks have been cleared in previous executions. */ - private boolean wasExecuted = false; - - /** - * Creates a new Execution Environment. - */ - protected ExecutionEnvironment() { - jobID = JobID.generate(); - } - - // -------------------------------------------------------------------------------------------- - // Properties - // -------------------------------------------------------------------------------------------- - - /** - * Gets the config object that defines execution parameters. - * - * @return The environment's execution configuration. - */ - public ExecutionConfig getConfig() { - return config; - } - - /** - * Gets the parallelism with which operation are executed by default. Operations can - * individually override this value to use a specific parallelism via - * {@link Operator#setParallelism(int)}. Other operations may need to run with a different - * parallelism - for example calling - * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire - * set will insert eventually an operation that runs non-parallel (parallelism of one). - * - * @return The parallelism used by operations, unless they override that value. This method - * returns {@link ExecutionConfig#PARALLELISM_DEFAULT}, if the environment's default parallelism should be used. - */ - public int getParallelism() { - return config.getParallelism(); - } - - /** - * Sets the parallelism for operations executed through this environment. - * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with - * x parallel instances. - * - *

This method overrides the default parallelism for this environment. - * The {@link LocalEnvironment} uses by default a value equal to the number of hardware - * contexts (CPU cores / threads). When executing the program via the command line client - * from a JAR file, the default parallelism is the one configured for that setup. - * - * @param parallelism The parallelism - */ - public void setParallelism(int parallelism) { - config.setParallelism(parallelism); - } - - /** - * Sets the restart strategy configuration. The configuration specifies which restart strategy - * will be used for the execution graph in case of a restart. - * - * @param restartStrategyConfiguration Restart strategy configuration to be set - */ - @PublicEvolving - public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { - config.setRestartStrategy(restartStrategyConfiguration); - } - - /** - * Returns the specified restart strategy configuration. - * - * @return The restart strategy configuration to be used - */ - @PublicEvolving - public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { - return config.getRestartStrategy(); - } - - /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of {@code -1} indicates that the system - * default value (as defined in the configuration) should be used. - * - * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #setRestartStrategy}. The - * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of - * execution retries. - */ - @Deprecated - @PublicEvolving - public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { - config.setNumberOfExecutionRetries(numberOfExecutionRetries); - } - - /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of {@code -1} indicates that the system default value (as defined in the configuration) - * should be used. - * - * @return The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #getRestartStrategy}. The - * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of - * execution retries. - */ - @Deprecated - @PublicEvolving - public int getNumberOfExecutionRetries() { - return config.getNumberOfExecutionRetries(); - } - - /** - * Returns the {@link org.apache.flink.api.common.JobExecutionResult} of the last executed job. - * - * @return The execution result from the latest job execution. - */ - public JobExecutionResult getLastJobExecutionResult(){ - return this.lastJobExecutionResult; - } - - // -------------------------------------------------------------------------------------------- - // Session Management - // -------------------------------------------------------------------------------------------- - - /** - * Gets the JobID by which this environment is identified. The JobID sets the execution context - * in the cluster or local environment. - * - * @return The JobID of this environment. - * @see #getIdString() - */ - @PublicEvolving - public JobID getId() { - return this.jobID; - } - - /** - * Gets the JobID by which this environment is identified, as a string. - * - * @return The JobID as a string. - * @see #getId() - */ - @PublicEvolving - public String getIdString() { - return this.jobID.toString(); - } - - /** - * Sets the session timeout to hold the intermediate results of a job. This only - * applies the updated timeout in future executions. - * - * @param timeout The timeout, in seconds. - */ - @PublicEvolving - public void setSessionTimeout(long timeout) { - throw new IllegalStateException("Support for sessions is currently disabled. " + - "It will be enabled in future Flink versions."); - // Session management is disabled, revert this commit to enable - //if (timeout < 0) { - // throw new IllegalArgumentException("The session timeout must not be less than zero."); - //} - //this.sessionTimeout = timeout; - } - - /** - * Gets the session timeout for this environment. The session timeout defines for how long - * after an execution, the job and its intermediate results will be kept for future - * interactions. - * - * @return The session timeout, in seconds. - */ - @PublicEvolving - public long getSessionTimeout() { - return sessionTimeout; - } - - /** - * Starts a new session, discarding the previous data flow and all of its intermediate results. - */ - @PublicEvolving - public abstract void startNewSession() throws Exception; - - // -------------------------------------------------------------------------------------------- - // Registry for types and serializers - // -------------------------------------------------------------------------------------------- - - /** - * Adds a new Kryo default serializer to the Runtime. - * - *

Note that the serializer instance must be serializable (as defined by java.io.Serializable), - * because it may be distributed to the worker nodes by java serialization. - * - * @param type The class of the types serialized with the given serializer. - * @param serializer The serializer to use. - */ - public & Serializable>void addDefaultKryoSerializer(Class type, T serializer) { - config.addDefaultKryoSerializer(type, serializer); - } - - /** - * Adds a new Kryo default serializer to the Runtime. - * - * @param type The class of the types serialized with the given serializer. - * @param serializerClass The class of the serializer to use. - */ - public void addDefaultKryoSerializer(Class type, Class> serializerClass) { - config.addDefaultKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with a Kryo Serializer. - * - *

Note that the serializer instance must be serializable (as defined by java.io.Serializable), - * because it may be distributed to the worker nodes by java serialization. - * - * @param type The class of the types serialized with the given serializer. - * @param serializer The serializer to use. - */ - public & Serializable>void registerTypeWithKryoSerializer(Class type, T serializer) { - config.registerTypeWithKryoSerializer(type, serializer); - } - - /** - * Registers the given Serializer via its class as a serializer for the given type at the KryoSerializer. - * - * @param type The class of the types serialized with the given serializer. - * @param serializerClass The class of the serializer to use. - */ - public void registerTypeWithKryoSerializer(Class type, Class> serializerClass) { - config.registerTypeWithKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with the serialization stack. If the type is eventually - * serialized as a POJO, then the type is registered with the POJO serializer. If the - * type ends up being serialized with Kryo, then it will be registered at Kryo to make - * sure that only tags are written. - * - * @param type The class of the type to register. - */ - public void registerType(Class type) { - if (type == null) { - throw new NullPointerException("Cannot register null type class."); - } - - TypeInformation typeInfo = TypeExtractor.createTypeInfo(type); - - if (typeInfo instanceof PojoTypeInfo) { - config.registerPojoType(type); - } else { - config.registerKryoType(type); - } - } - - // -------------------------------------------------------------------------------------------- - // Data set creations - // -------------------------------------------------------------------------------------------- - - // ---------------------------------- Text Input Format --------------------------------------- - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * The file will be read with the UTF-8 character set. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFile(String filePath) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new TextInputFormat(new Path(filePath)), BasicTypeInfo.STRING_TYPE_INFO, Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * The {@link java.nio.charset.Charset} with the given name will be used to read the files. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param charsetName The name of the character set used to read the file. - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFile(String filePath, String charsetName) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - TextInputFormat format = new TextInputFormat(new Path(filePath)); - format.setCharsetName(charsetName); - return new DataSource<>(this, format, BasicTypeInfo.STRING_TYPE_INFO, Utils.getCallLocationName()); - } - - // -------------------------- Text Input Format With String Value------------------------------ - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * This method is similar to {@link #readTextFile(String)}, but it produces a DataSet with mutable - * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations - * to be less object and garbage collection heavy. - * - *

The file will be read with the UTF-8 character set. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFileWithValue(String filePath) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new TextValueInputFormat(new Path(filePath)), new ValueTypeInfo<>(StringValue.class), Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * This method is similar to {@link #readTextFile(String, String)}, but it produces a DataSet with mutable - * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations - * to be less object and garbage collection heavy. - * - *

The {@link java.nio.charset.Charset} with the given name will be used to read the files. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param charsetName The name of the character set used to read the file. - * @param skipInvalidLines A flag to indicate whether to skip lines that cannot be read with the given character set. - * - * @return A DataSet that represents the data read from the given file as text lines. - */ - public DataSource readTextFileWithValue(String filePath, String charsetName, boolean skipInvalidLines) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - TextValueInputFormat format = new TextValueInputFormat(new Path(filePath)); - format.setCharsetName(charsetName); - format.setSkipInvalidLines(skipInvalidLines); - return new DataSource<>(this, format, new ValueTypeInfo<>(StringValue.class), Utils.getCallLocationName()); - } - - // ----------------------------------- Primitive Input Format --------------------------------------- - - /** - * Creates a {@link DataSet} that represents the primitive type produced by reading the given file line wise. - * This method is similar to {@link #readCsvFile(String)} with single field, but it produces a DataSet not through - * {@link org.apache.flink.api.java.tuple.Tuple1}. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param typeClass The primitive type class to be read. - * @return A {@link DataSet} that represents the data read from the given file as primitive type. - */ - public DataSource readFileOfPrimitives(String filePath, Class typeClass) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new PrimitiveInputFormat<>(new Path(filePath), typeClass), TypeExtractor.getForClass(typeClass), Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the primitive type produced by reading the given file in delimited way. - * This method is similar to {@link #readCsvFile(String)} with single field, but it produces a DataSet not through - * {@link org.apache.flink.api.java.tuple.Tuple1}. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param delimiter The delimiter of the given file. - * @param typeClass The primitive type class to be read. - * @return A {@link DataSet} that represents the data read from the given file as primitive type. - */ - public DataSource readFileOfPrimitives(String filePath, String delimiter, Class typeClass) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new PrimitiveInputFormat<>(new Path(filePath), delimiter, typeClass), TypeExtractor.getForClass(typeClass), Utils.getCallLocationName()); - } - - // ----------------------------------- CSV Input Format --------------------------------------- - - /** - * Creates a CSV reader to read a comma separated value (CSV) file. The reader has options to - * define parameters and field types and will eventually produce the DataSet that corresponds to - * the read and parsed CSV input. - * - * @param filePath The path of the CSV file. - * @return A CsvReader that can be used to configure the CSV input. - */ - public CsvReader readCsvFile(String filePath) { - return new CsvReader(filePath, this); - } - - // ------------------------------------ File Input Format ----------------------------------------- - - public DataSource readFile(FileInputFormat inputFormat, String filePath) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - if (filePath == null) { - throw new IllegalArgumentException("The file path must not be null."); - } - - inputFormat.setFilePath(new Path(filePath)); - try { - return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); - } - catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be automatically determined. " + - "Please specify the TypeInformation of the produced type explicitly by using the " + - "'createInput(InputFormat, TypeInformation)' method instead."); - } - } - - // ----------------------------------- Generic Input Format --------------------------------------- - - /** - * Generic method to create an input {@link DataSet} with in {@link InputFormat}. The DataSet will not be - * immediately created - instead, this method returns a DataSet that will be lazily created from - * the input format once the program is executed. - * - *

Since all data sets need specific information about their types, this method needs to determine - * the type of the data produced by the input format. It will attempt to determine the data type - * by reflection, unless the input format implements the {@link ResultTypeQueryable} interface. - * In the latter case, this method will invoke the {@link ResultTypeQueryable#getProducedType()} - * method to determine data type produced by the input format. - * - * @param inputFormat The input format used to create the data set. - * @return A {@link DataSet} that represents the data created by the input format. - * - * @see #createInput(InputFormat, TypeInformation) - */ - public DataSource createInput(InputFormat inputFormat) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - - try { - return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); - } - catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be automatically determined. " + - "Please specify the TypeInformation of the produced type explicitly by using the " + - "'createInput(InputFormat, TypeInformation)' method instead.", e); - } - } - - /** - * Generic method to create an input DataSet with in {@link InputFormat}. The {@link DataSet} will not be - * immediately created - instead, this method returns a {@link DataSet} that will be lazily created from - * the input format once the program is executed. - * - *

The {@link DataSet} is typed to the given TypeInformation. This method is intended for input formats that - * where the return type cannot be determined by reflection analysis, and that do not implement the - * {@link ResultTypeQueryable} interface. - * - * @param inputFormat The input format used to create the data set. - * @return A {@link DataSet} that represents the data created by the input format. - * - * @see #createInput(InputFormat) - */ - public DataSource createInput(InputFormat inputFormat, TypeInformation producedType) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - - if (producedType == null) { - throw new IllegalArgumentException("Produced type information must not be null."); - } - - return new DataSource<>(this, inputFormat, producedType, Utils.getCallLocationName()); - } - - // ----------------------------------- Collection --------------------------------------- - - /** - * Creates a DataSet from the given non-empty collection. The type of the data set is that - * of the elements in the collection. - * - *

The framework will try and determine the exact type from the collection elements. - * In case of generic elements, it may be necessary to manually supply the type information - * via {@link #fromCollection(Collection, TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @return A DataSet representing the given collection. - * - * @see #fromCollection(Collection, TypeInformation) - */ - public DataSource fromCollection(Collection data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.size() == 0) { - throw new IllegalArgumentException("The size of the collection must not be empty."); - } - - X firstValue = data.iterator().next(); - - TypeInformation type = TypeExtractor.getForObject(firstValue); - CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource<>(this, new CollectionInputFormat<>(data, type.createSerializer(config)), type, Utils.getCallLocationName()); - } - - /** - * Creates a DataSet from the given non-empty collection. Note that this operation will result - * in a non-parallel data source, i.e. a data source with a parallelism of one. - * - *

The returned DataSet is typed to the given TypeInformation. - * - * @param data The collection of elements to create the data set from. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the given collection. - * - * @see #fromCollection(Collection) - */ - public DataSource fromCollection(Collection data, TypeInformation type) { - return fromCollection(data, type, Utils.getCallLocationName()); - } - - private DataSource fromCollection(Collection data, TypeInformation type, String callLocationName) { - CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource<>(this, new CollectionInputFormat<>(data, type.createSerializer(config)), type, callLocationName); - } - - /** - * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until - * the actual execution happens, the type of data returned by the iterator must be given - * explicitly in the form of the type class (this is due to the fact that the Java compiler - * erases the generic type information). - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @param type The class of the data produced by the iterator. Must not be a generic class. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromCollection(Iterator, TypeInformation) - */ - public DataSource fromCollection(Iterator data, Class type) { - return fromCollection(data, TypeExtractor.getForClass(type)); - } - - /** - * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until - * the actual execution happens, the type of data returned by the iterator must be given - * explicitly in the form of the type information. This method is useful for cases where the type - * is generic. In that case, the type class (as given in {@link #fromCollection(Iterator, Class)} - * does not supply all type information. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromCollection(Iterator, Class) - */ - public DataSource fromCollection(Iterator data, TypeInformation type) { - return new DataSource<>(this, new IteratorInputFormat<>(data), type, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains the given elements. The elements must all be of the same type, - * for example, all of the {@link String} or {@link Integer}. The sequence of elements must not be empty. - * - *

The framework will try and determine the exact type from the collection elements. - * In case of generic elements, it may be necessary to manually supply the type information - * via {@link #fromCollection(Collection, TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The elements to make up the data set. - * @return A DataSet representing the given list of elements. - */ - @SafeVarargs - public final DataSource fromElements(X... data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.length == 0) { - throw new IllegalArgumentException("The number of elements must not be zero."); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForObject(data[0]); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + data[0].getClass().getName() - + "; please specify the TypeInformation manually via " - + "ExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - - return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains the given elements. The framework will determine the type according to the - * based type user supplied. The elements should be the same or be the subclass to the based type. - * The sequence of elements must not be empty. - * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param type The base class type for every element in the collection. - * @param data The elements to make up the data set. - * @return A DataSet representing the given list of elements. - */ - @SafeVarargs - public final DataSource fromElements(Class type, X... data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.length == 0) { - throw new IllegalArgumentException("The number of elements must not be zero."); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForClass(type); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + type.getName() - + "; please specify the TypeInformation manually via " - + "ExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - - return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the - * framework to create a parallel data source that returns the elements in the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type of data - * returned by the iterator must be given explicitly in the form of the type class (this is due to the - * fact that the Java compiler erases the generic type information). - * - * @param iterator The iterator that produces the elements of the data set. - * @param type The class of the data produced by the iterator. Must not be a generic class. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromParallelCollection(SplittableIterator, TypeInformation) - */ - public DataSource fromParallelCollection(SplittableIterator iterator, Class type) { - return fromParallelCollection(iterator, TypeExtractor.getForClass(type)); - } - - /** - * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the - * framework to create a parallel data source that returns the elements in the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type of data - * returned by the iterator must be given explicitly in the form of the type information. - * This method is useful for cases where the type is generic. In that case, the type class - * (as given in {@link #fromParallelCollection(SplittableIterator, Class)} does not supply all type information. - * - * @param iterator The iterator that produces the elements of the data set. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromParallelCollection(SplittableIterator, Class) - */ - public DataSource fromParallelCollection(SplittableIterator iterator, TypeInformation type) { - return fromParallelCollection(iterator, type, Utils.getCallLocationName()); - } - - // private helper for passing different call location names - private DataSource fromParallelCollection(SplittableIterator iterator, TypeInformation type, String callLocationName) { - return new DataSource<>(this, new ParallelIteratorInputFormat<>(iterator), type, callLocationName); - } - - /** - * Creates a new data set that contains a sequence of numbers. The data set will be created in parallel, - * so there is no guarantee about the order of the elements. - * - * @param from The number to start at (inclusive). - * @param to The number to stop at (inclusive). - * @return A DataSet, containing all number in the {@code [from, to]} interval. - */ - public DataSource generateSequence(long from, long to) { - return fromParallelCollection(new NumberSequenceIterator(from, to), BasicTypeInfo.LONG_TYPE_INFO, Utils.getCallLocationName()); - } - - // -------------------------------------------------------------------------------------------- - // Executing - // -------------------------------------------------------------------------------------------- - - /** - * Triggers the program execution. The environment will execute all parts of the program that have - * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, - * writing results (e.g. {@link DataSet#writeAsText(String)}, - * {@link DataSet#write(org.apache.flink.api.common.io.FileOutputFormat, String)}, or other generic - * data sinks created with {@link DataSet#output(org.apache.flink.api.common.io.OutputFormat)}. - * - *

The program execution will be logged and displayed with a generated default name. - * - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception Thrown, if the program executions fails. - */ - public JobExecutionResult execute() throws Exception { - return execute(getDefaultName()); - } - - /** - * Triggers the program execution. The environment will execute all parts of the program that have - * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, - * writing results (e.g. {@link DataSet#writeAsText(String)}, - * {@link DataSet#write(org.apache.flink.api.common.io.FileOutputFormat, String)}, or other generic - * data sinks created with {@link DataSet#output(org.apache.flink.api.common.io.OutputFormat)}. - * - *

The program execution will be logged and displayed with the given job name. - * - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception Thrown, if the program executions fails. - */ - public abstract JobExecutionResult execute(String jobName) throws Exception; - - /** - * Creates the plan with which the system will execute the program, and returns it as - * a String using a JSON representation of the execution data flow graph. - * Note that this needs to be called, before the plan is executed. - * - * @return The execution plan of the program, as a JSON String. - * @throws Exception Thrown, if the compiler could not be instantiated, or the master could not - * be contacted to retrieve information relevant to the execution planning. - */ - public abstract String getExecutionPlan() throws Exception; - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - */ - public void registerCachedFile(String filePath, String name){ - registerCachedFile(filePath, name, false); - } - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - * @param executable flag indicating whether the file should be executable - */ - public void registerCachedFile(String filePath, String name, boolean executable){ - this.cacheFile.add(new Tuple2<>(name, new DistributedCacheEntry(filePath, executable))); - } - - /** - * Registers all files that were registered at this execution environment's cache registry of the - * given plan's cache registry. - * - * @param p The plan to register files at. - * @throws IOException Thrown if checks for existence and sanity fail. - */ - protected void registerCachedFilesWithPlan(Plan p) throws IOException { - for (Tuple2 entry : cacheFile) { - p.registerCachedFile(entry.f0, entry.f1); - } - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * This automatically starts a new stage of execution. - * - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan() { - return createProgramPlan(null); - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * This automatically starts a new stage of execution. - * - * @param jobName The name attached to the plan (displayed in logs and monitoring). - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan(String jobName) { - return createProgramPlan(jobName, true); - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * - * @param jobName The name attached to the plan (displayed in logs and monitoring). - * @param clearSinks Whether or not to start a new stage of execution. - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan(String jobName, boolean clearSinks) { - if (this.sinks.isEmpty()) { - if (wasExecuted) { - throw new RuntimeException("No new data sinks have been defined since the " + - "last execution. The last execution refers to the latest call to " + - "'execute()', 'count()', 'collect()', or 'print()'."); - } else { - throw new RuntimeException("No data sinks have been created yet. " + - "A program needs at least one sink that consumes data. " + - "Examples are writing the data set or printing it."); - } - } - - if (jobName == null) { - jobName = getDefaultName(); - } - - OperatorTranslation translator = new OperatorTranslation(); - Plan plan = translator.translateToPlan(this.sinks, jobName); - - if (getParallelism() > 0) { - plan.setDefaultParallelism(getParallelism()); - } - plan.setExecutionConfig(getConfig()); - - // Check plan for GenericTypeInfo's and register the types at the serializers. - if (!config.isAutoTypeRegistrationDisabled()) { - plan.accept(new Visitor>() { - - private final Set> registeredTypes = new HashSet<>(); - private final Set> visitedOperators = new HashSet<>(); - - @Override - public boolean preVisit(org.apache.flink.api.common.operators.Operator visitable) { - if (!visitedOperators.add(visitable)) { - return false; - } - OperatorInformation opInfo = visitable.getOperatorInfo(); - Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes); - return true; - } - - @Override - public void postVisit(org.apache.flink.api.common.operators.Operator visitable) {} - }); - } - - try { - registerCachedFilesWithPlan(plan); - } catch (Exception e) { - throw new RuntimeException("Error while registering cached files: " + e.getMessage(), e); - } - - // clear all the sinks such that the next execution does not redo everything - if (clearSinks) { - this.sinks.clear(); - wasExecuted = true; - } - - // All types are registered now. Print information. - int registeredTypes = config.getRegisteredKryoTypes().size() + - config.getRegisteredPojoTypes().size() + - config.getRegisteredTypesWithKryoSerializerClasses().size() + - config.getRegisteredTypesWithKryoSerializers().size(); - int defaultKryoSerializers = config.getDefaultKryoSerializers().size() + - config.getDefaultKryoSerializerClasses().size(); - LOG.info("The job has {} registered types and {} default Kryo serializers", registeredTypes, defaultKryoSerializers); - - if (config.isForceKryoEnabled() && config.isForceAvroEnabled()) { - LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer"); - } - if (config.isForceKryoEnabled()) { - LOG.info("Using KryoSerializer for serializing POJOs"); - } - if (config.isForceAvroEnabled()) { - LOG.info("Using AvroSerializer for serializing POJOs"); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Registered Kryo types: {}", config.getRegisteredKryoTypes().toString()); - LOG.debug("Registered Kryo with Serializers types: {}", config.getRegisteredTypesWithKryoSerializers().entrySet().toString()); - LOG.debug("Registered Kryo with Serializer Classes types: {}", config.getRegisteredTypesWithKryoSerializerClasses().entrySet().toString()); - LOG.debug("Registered Kryo default Serializers: {}", config.getDefaultKryoSerializers().entrySet().toString()); - LOG.debug("Registered Kryo default Serializers Classes {}", config.getDefaultKryoSerializerClasses().entrySet().toString()); - LOG.debug("Registered POJO types: {}", config.getRegisteredPojoTypes().toString()); - - // print information about static code analysis - LOG.debug("Static code analysis mode: {}", config.getCodeAnalysisMode()); - } - - return plan; - } - - /** - * Adds the given sink to this environment. Only sinks that have been added will be executed once - * the {@link #execute()} or {@link #execute(String)} method is called. - * - * @param sink The sink to add for execution. - */ - @Internal - void registerDataSink(DataSink sink) { - this.sinks.add(sink); - } - - /** - * Gets a default job name, based on the timestamp when this method is invoked. - * - * @return A default job name. - */ - private static String getDefaultName() { - return "Flink Java Job at " + Calendar.getInstance().getTime(); - } - - // -------------------------------------------------------------------------------------------- - // Instantiation of Execution Contexts - // -------------------------------------------------------------------------------------------- - - /** - * Creates an execution environment that represents the context in which the program is currently executed. - * If the program is invoked standalone, this method returns a local execution environment, as returned by - * {@link #createLocalEnvironment()}. If the program is invoked from within the command line client to be - * submitted to a cluster, this method returns the execution environment of this cluster. - * - * @return The execution environment of the context in which the program is executed. - */ - public static ExecutionEnvironment getExecutionEnvironment() { - return contextEnvironmentFactory == null ? - createLocalEnvironment() : contextEnvironmentFactory.get().createExecutionEnvironment(); - } - - /** - * Creates a {@link CollectionEnvironment} that uses Java Collections underneath. This will execute in a - * single thread in the current JVM. It is very fast but will fail if the data does not fit into - * memory. parallelism will always be 1. This is useful during implementation and for debugging. - * @return A Collection Environment - */ - @PublicEvolving - public static CollectionEnvironment createCollectionsEnvironment(){ - CollectionEnvironment ce = new CollectionEnvironment(); - ce.setParallelism(1); - return ce; - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. The default - * parallelism of the local environment is the number of hardware contexts (CPU cores / threads), - * unless it was specified differently by {@link #setDefaultLocalParallelism(int)}. - * - * @return A local execution environment. - */ - public static LocalEnvironment createLocalEnvironment() { - return createLocalEnvironment(defaultLocalDop); - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. It will use the - * parallelism specified in the parameter. - * - * @param parallelism The parallelism for the local environment. - * @return A local execution environment with the specified parallelism. - */ - public static LocalEnvironment createLocalEnvironment(int parallelism) { - return createLocalEnvironment(new Configuration(), parallelism); - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. It will use the - * parallelism specified in the parameter. - * - * @param customConfiguration Pass a custom configuration to the LocalEnvironment. - * @return A local execution environment with the specified parallelism. - */ - public static LocalEnvironment createLocalEnvironment(Configuration customConfiguration) { - return createLocalEnvironment(customConfiguration, -1); - } - - /** - * Creates a {@link LocalEnvironment} for local program execution that also starts the - * web monitoring UI. - * - *

The local execution environment will run the program in a multi-threaded fashion in - * the same JVM as the environment was created in. It will use the parallelism specified in the - * parameter. - * - *

If the configuration key 'rest.port' was set in the configuration, that particular - * port will be used for the web UI. Otherwise, the default port (8081) will be used. - */ - @PublicEvolving - public static ExecutionEnvironment createLocalEnvironmentWithWebUI(Configuration conf) { - checkNotNull(conf, "conf"); - - conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true); - - if (!conf.contains(RestOptions.PORT)) { - // explicitly set this option so that it's not set to 0 later - conf.setInteger(RestOptions.PORT, RestOptions.PORT.defaultValue()); - } - - return createLocalEnvironment(conf, -1); - } - - /** - * Creates a {@link LocalEnvironment} which is used for executing Flink jobs. - * - * @param configuration to start the {@link LocalEnvironment} with - * @param defaultParallelism to initialize the {@link LocalEnvironment} with - * @return {@link LocalEnvironment} - */ - private static LocalEnvironment createLocalEnvironment(Configuration configuration, int defaultParallelism) { - final LocalEnvironment localEnvironment = new LocalEnvironment(configuration); - - if (defaultParallelism > 0) { - localEnvironment.setParallelism(defaultParallelism); - } - - return localEnvironment; - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the cluster's default parallelism, unless the parallelism is - * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment(String host, int port, String... jarFiles) { - return new RemoteEnvironment(host, port, jarFiles); - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The custom configuration file is used to configure Akka specific configuration parameters - * for the Client only; Program parallelism can be set via {@link ExecutionEnvironment#setParallelism(int)}. - * - *

Cluster configuration has to be done in the remotely running Flink instance. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param clientConfiguration Configuration used by the client that connects to the cluster. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment( - String host, int port, Configuration clientConfiguration, String... jarFiles) { - return new RemoteEnvironment(host, port, clientConfiguration, jarFiles, null); - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the specified parallelism. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param parallelism The parallelism to use during the execution. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) { - RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles); - rec.setParallelism(parallelism); - return rec; - } - - // -------------------------------------------------------------------------------------------- - // Default parallelism for local execution - // -------------------------------------------------------------------------------------------- - - /** - * Gets the default parallelism that will be used for the local execution environment created by - * {@link #createLocalEnvironment()}. - * - * @return The default local parallelism - */ - public static int getDefaultLocalParallelism() { - return defaultLocalDop; - } - - /** - * Sets the default parallelism that will be used for the local execution environment created by - * {@link #createLocalEnvironment()}. - * - * @param parallelism The parallelism to use as the default local parallelism. - */ - public static void setDefaultLocalParallelism(int parallelism) { - defaultLocalDop = parallelism; - } - - // -------------------------------------------------------------------------------------------- - // Methods to control the context environment and creation of explicit environments other - // than the context environment - // -------------------------------------------------------------------------------------------- - - /** - * Sets a context environment factory, that creates the context environment for running programs - * with pre-configured environments. Examples are running programs from the command line, and - * running programs in the Scala shell. - * - *

When the context environment factory is set, no other environments can be explicitly used. - * - * @param ctx The context environment factory. - */ - protected static void initializeContextEnvironment(ExecutionEnvironmentFactory ctx) { - contextEnvironmentFactory.set(Preconditions.checkNotNull(ctx)); - } - - /** - * Un-sets the context environment factory. After this method is called, the call to - * {@link #getExecutionEnvironment()} will again return a default local execution environment, and - * it is possible to explicitly instantiate the LocalEnvironment and the RemoteEnvironment. - */ - protected static void resetContextEnvironment() { - contextEnvironmentFactory.remove(); - } - - /** - * Checks whether it is currently permitted to explicitly instantiate a LocalEnvironment - * or a RemoteEnvironment. - * - * @return True, if it is possible to explicitly instantiate a LocalEnvironment or a - * RemoteEnvironment, false otherwise. - */ - @Internal - public static boolean areExplicitEnvironmentsAllowed() { - return contextEnvironmentFactory.get() == null; - } -} From 81ae641edf19857d4acd99c89c4b2d6f6eff8814 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 21 Aug 2020 14:59:11 +0800 Subject: [PATCH 083/136] fix doc problem --- docs/quickstart.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index 4625c44d82..f5f702280c 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -106,7 +106,7 @@ rest.bind-port: 8888 ```bash bin/flinkx -mode standalone \ -job $FLINK_HOME/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ + -pluginRoot $FLINKX_HOME/plugins \ -flinkconf $FLINK_HOME/conf \ -confProp "{\"flink.checkpoint.interval\":60000}" ``` @@ -145,7 +145,7 @@ $FLINK_HOME/bin/start-cluster.sh ```bash bin/flinkx -mode yarn \ -job $FLINK_HOME/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ + -pluginRoot $FLINKX_HOME/plugins \ -flinkconf $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop \ -confProp "{\"flink.checkpoint.interval\":60000}" @@ -187,7 +187,7 @@ $FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024 ```bash bin/flinkx -mode yarnPer \ -job ${FLINK_HOME}/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ + -pluginRoot $FLINKX_HOME/plugins \ -flinkconf $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop \ -flinkLibJar $FLINK_HOME/lib \ From 7d140cda3c78f058d6b091609ae840226676b252 Mon Sep 17 00:00:00 2001 From: huangrunxing Date: Mon, 14 Sep 2020 13:36:52 +0800 Subject: [PATCH 084/136] add lzo conpress for hdfs wtire with text type --- .../com/dtstack/flinkx/hdfs/ECompressType.java | 1 + flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 18 ++++++++++++++++++ .../hdfs/writer/HdfsTextOutputFormat.java | 6 ++++++ 3 files changed, 25 insertions(+) diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java index 2f0c7ec101..3ff1b66de2 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/ECompressType.java @@ -32,6 +32,7 @@ public enum ECompressType { */ TEXT_GZIP("GZIP", "text", ".gz", 0.331F), TEXT_BZIP2("BZIP2", "text", ".bz2", 0.259F), + TEXT_LZO("LZO", "text", ".lzo", 1.0F), TEXT_NONE("NONE", "text", "", 0.637F), /** diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 7212c0cef0..3961f1a032 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -34,6 +34,24 @@ under the License. + + + org.anarres.lzo + lzo-core + 1.0.2 + + + org.anarres.lzo + lzo-hadoop + 1.0.5 + + + hadoop-core + org.apache.hadoop + + + + com.dtstack.flinkx flinkx-hdfs-core diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index dc592d5563..4b8666e338 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -26,8 +26,11 @@ import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + import java.io.IOException; import java.io.OutputStream; import java.math.BigDecimal; @@ -93,6 +96,9 @@ protected void nextBlock(){ stream = new GzipCompressorOutputStream(fs.create(p)); } else if(compressType == ECompressType.TEXT_BZIP2){ stream = new BZip2CompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_LZO) { + CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration()); + stream = factory.getCodecByClassName("com.hadoop.compression.lzo.LzopCodec").createOutputStream(fs.create(p)); } } From b1ad2195ea59978328537962647fd4aad4c49a15 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 12 Jan 2021 17:02:58 +0800 Subject: [PATCH 085/136] =?UTF-8?q?1=E3=80=81=E5=A2=9E=E5=8A=A0Kingbase?= =?UTF-8?q?=E6=8F=92=E4=BB=B6=202=E3=80=81fix=20#301=203=E3=80=81fix=20#27?= =?UTF-8?q?6=204=E3=80=81fix=20#244=205=E3=80=81fix=20=E5=85=B6=E4=BB=96BU?= =?UTF-8?q?G?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitignore | 3 +- README.md | 8 +- README_CH.md | 21 +- bin/install_jars.bat | 3 + bin/install_jars.sh | 8 +- docs/example/kafka09_stream.json | 36 + docs/example/kafka10_stream.json | 42 + docs/example/kafka11_stream.json | 36 + docs/example/kafka_hive.json | 52 + docs/example/kafka_mysql.json | 65 + docs/example/kafka_stream.json | 34 + docs/example/stream_stream.json | 30 + docs/generalconfig.md | 23 +- docs/offline/reader/mongodbreader.md | 1 - docs/offline/reader/postgresqlreader.md | 2 - docs/offline/reader/sqlserverreader.md | 1 + docs/offline/writer/postgresqlwriter.md | 4 +- docs/offline/writer/sqlserverwriter.md | 4 +- docs/questions.md | 24 + docs/quickstart.md | 134 +- docs/realTime/reader/kafkareader.md | 466 +++--- docs/realTime/reader/pgwalreader.md | 215 +++ docs/realTime/writer/kafkawriter.md | 213 +-- docs/restore.md | 2 +- .../com/dtstack/flinkx/binlog/BinlogUtil.java | 207 +++ .../{reader => format}/BinlogInputFormat.java | 132 +- .../format/BinlogInputFormatBuilder.java | 171 +++ .../BinlogAlarmHandler.java | 3 +- .../{reader => listener}/BinlogEventSink.java | 24 +- .../BinlogPositionManager.java | 3 +- .../HeartBeatController.java | 3 +- .../flinkx/binlog/reader/BinlogReader.java | 16 +- flinkx-binlog/pom.xml | 10 + .../writer/dict/DictionaryWriterTask.java | 12 +- .../clickhouse/core/ClickhouseUtil.java | 4 +- .../flinkx-clickhouse-reader/pom.xml | 1 + .../format/ClickhouseInputFormat.java | 4 - .../clickhouse/reader/ClickhouseReader.java | 1 - .../flinkx-clickhouse-writer/pom.xml | 1 + flinkx-core/pom.xml | 11 +- .../main/java/com/dtstack/flinkx/Main.java | 38 +- .../flinkx/classloader/PluginUtil.java | 72 +- .../dtstack/flinkx/config/AbstractConfig.java | 36 +- .../flinkx/config/DataTransferConfig.java | 2 +- .../flinkx/config/ErrorLimitConfig.java | 4 +- .../flinkx/constants/ConfigConstant.java | 3 - .../flinkx/constants/ConstantValue.java | 8 +- .../dtstack/flinkx/decoder/DecodeEnum.java | 2 +- .../dtstack/flinkx/decoder/JsonDecoder.java | 9 +- .../{PlainDecoder.java => TextDecoder.java} | 2 +- .../com/dtstack/flinkx/enums/ColumnType.java | 28 +- .../dtstack/flinkx/enums/EDatabaseType.java | 5 +- .../flinkx/exception/DataSourceException.java | 27 +- .../inputformat/BaseRichInputFormat.java | 6 +- .../BaseRichInputFormatBuilder.java | 5 +- .../com/dtstack/flinkx/latch/MetricLatch.java | 4 +- .../dtstack/flinkx/metrics/BaseMetric.java | 2 +- .../com/dtstack/flinkx/options/Options.java | 37 +- .../outputformat/BaseFileOutputFormat.java | 4 +- .../outputformat/BaseRichOutputFormat.java | 30 +- .../dtstack/flinkx/reader/BaseDataReader.java | 3 +- .../flinkx/reader/DataReaderFactory.java | 3 +- .../source/DtInputFormatSourceFunction.java | 25 +- .../com/dtstack/flinkx/util/DateUtil.java | 5 - .../dtstack/flinkx/util/FileSystemUtil.java | 7 +- .../com/dtstack/flinkx/util/GsonUtil.java | 1 + .../dtstack/flinkx/util/JsonModifyUtil.java | 5 +- .../java/com/dtstack/flinkx/util/MapUtil.java | 6 + .../dtstack/flinkx/util/RangeSplitUtil.java | 48 + .../com/dtstack/flinkx/util/RetryUtil.java | 11 +- .../com/dtstack/flinkx/util/StringUtil.java | 41 +- .../com/dtstack/flinkx/util/ZkHelper.java | 94 ++ .../flinkx/writer/DataWriterFactory.java | 2 +- .../dtstack/flinkx/writer/ErrorLimiter.java | 3 +- .../flink/api/java/ExecutionEnvironment.java | 1278 ----------------- .../com/dtstack/flinkx/util/ZkHelperTest.java | 84 ++ flinkx-db2/flinkx-db2-reader/pom.xml | 1 + flinkx-db2/flinkx-db2-writer/pom.xml | 1 + flinkx-dm/flinkx-dm-reader/pom.xml | 1 + flinkx-dm/flinkx-dm-writer/pom.xml | 1 + flinkx-emqx/flinkx-emqx-reader/pom.xml | 1 + .../flinkx/emqx/format/EmqxInputFormat.java | 4 +- flinkx-emqx/flinkx-emqx-writer/pom.xml | 1 + .../flinkx/emqx/writer/EmqxWriter.java | 3 + flinkx-es/flinkx-es-reader/pom.xml | 1 + .../flinkx/es/reader/EsInputFormat.java | 2 - flinkx-es/flinkx-es-writer/pom.xml | 1 + .../com/dtstack/flinkx/ftp/EProtocol.java | 12 +- .../com/dtstack/flinkx/ftp/FtpConfig.java | 5 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 58 +- .../com/dtstack/flinkx/ftp/IFtpHandler.java | 6 + .../com/dtstack/flinkx/ftp/SftpHandler.java | 9 + flinkx-ftp/flinkx-ftp-reader/pom.xml | 1 + .../flinkx/ftp/reader/FtpInputFormat.java | 5 +- .../ftp/reader/FtpSeqBufferedReader.java | 10 +- flinkx-ftp/flinkx-ftp-writer/pom.xml | 1 + .../flinkx/ftp/writer/FtpOutputFormat.java | 94 +- flinkx-gbase/flinkx-gbase-reader/pom.xml | 1 + .../flinkx/gbase/format/GbaseInputFormat.java | 5 - flinkx-gbase/flinkx-gbase-writer/pom.xml | 1 + .../flinkx-greenplum-reader/pom.xml | 1 + .../flinkx/hbase/HbaseConfigConstants.java | 6 - flinkx-hbase/flinkx-hbase-reader/pom.xml | 10 +- .../flinkx/hbase/reader/HbaseInputFormat.java | 5 +- .../hbase/reader/HbaseInputFormatBuilder.java | 7 - .../flinkx/hbase/reader/HbaseReader.java | 3 - flinkx-hbase/flinkx-hbase-writer/pom.xml | 10 +- .../com/dtstack/flinkx/hdfs/HdfsUtil.java | 2 +- flinkx-hdfs/flinkx-hdfs-reader/pom.xml | 1 + .../flinkx/hdfs/reader/HdfsReader.java | 12 +- flinkx-hdfs/flinkx-hdfs-writer/pom.xml | 1 + .../hdfs/writer/BaseHdfsOutputFormat.java | 6 + .../hdfs/writer/HdfsOrcOutputFormat.java | 26 + .../hdfs/writer/HdfsParquetOutputFormat.java | 24 +- .../hdfs/writer/HdfsTextOutputFormat.java | 7 + .../dtstack/flinkx/hive/util/AddressUtil.java | 3 +- flinkx-hive/flinkx-hive-writer/pom.xml | 13 + .../flinkx/hive/writer/HiveOutputFormat.java | 10 +- flinkx-kafka/flinkx-kafka-reader/pom.xml | 1 + .../flinkx/kafka/client/KafkaClient.java | 221 +++ .../{reader => client}/KafkaConsumer.java | 28 +- .../flinkx/kafka/format/KafkaInputFormat.java | 111 ++ .../flinkx/kafka/reader/KafkaClient.java | 104 -- .../flinkx/kafka/reader/KafkaReader.java | 11 +- flinkx-kafka/flinkx-kafka-writer/pom.xml | 1 + .../{writer => format}/KafkaOutputFormat.java | 20 +- .../flinkx/kafka/writer/KafkaWriter.java | 6 + flinkx-kafka/pom.xml | 10 + flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 1 + .../{reader => client}/Kafka09Client.java | 27 +- .../{reader => client}/Kafka09Consumer.java | 9 +- .../Kafka09InputFormat.java | 21 +- .../flinkx/kafka09/reader/Kafka09Reader.java | 21 +- flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 1 + .../Kafka09OutputFormat.java | 30 +- .../flinkx/kafka09/writer/Kafka09Writer.java | 5 + flinkx-kafka09/pom.xml | 10 + flinkx-kafka10/flinkx-kafka10-reader/pom.xml | 1 + .../{reader => client}/Kafka10Client.java | 17 +- .../{reader => client}/Kafka10Consumer.java | 15 +- .../Kafka10InputFormat.java | 14 +- .../flinkx/kafka10/reader/Kafka10Reader.java | 11 +- flinkx-kafka10/flinkx-kafka10-writer/pom.xml | 1 + .../Kafka10OutputFormat.java | 20 +- .../flinkx/kafka10/writer/Kafka10Writer.java | 7 +- flinkx-kafka10/pom.xml | 10 + flinkx-kafka11/flinkx-kafka11-reader/pom.xml | 1 + .../{reader => client}/Kafka11Client.java | 17 +- .../{reader => client}/Kafka11Consumer.java | 15 +- .../Kafka11InputFormat.java | 14 +- .../flinkx/kafka11/reader/Kafka11Reader.java | 11 +- flinkx-kafka11/flinkx-kafka11-writer/pom.xml | 1 + .../Kafka11OutputFormat.java | 20 +- .../flinkx/kafka11/writer/Kafka11Writer.java | 7 +- flinkx-kafka11/pom.xml | 10 + .../flinkx/kafkabase/KafkaConfigKeys.java | 8 + .../flinkx/kafkabase/KafkaInputSplit.java | 40 +- .../flinkx/kafkabase/entity/kafkaState.java | 106 ++ .../flinkx/kafkabase/enums/KafkaVersion.java | 33 +- .../flinkx/kafkabase/enums/StartupMode.java | 80 ++ .../kafkabase/{ => util}/Formatter.java | 2 +- .../flinkx/kafkabase/util/KafkaUtil.java | 103 ++ .../kafkabase/{reader => client}/IClient.java | 11 +- .../{reader => client}/KafkaBaseConsumer.java | 32 +- .../KafkaBaseInputFormat.java | 143 +- .../format/KafkaBaseInputFormatBuilder.java | 126 ++ .../kafkabase/reader/KafkaBaseReader.java | 61 +- .../KafkaBaseOutputFormat.java | 38 +- .../flinkx/kafkabase/writer/AddressUtil.java | 62 - .../writer/HeartBeatController.java | 12 +- .../kafkabase/writer/KafkaBaseWriter.java | 5 +- flinkx-kb/pom.xml | 10 + flinkx-kingbase/flinkx-kingbase-core/pom.xml | 24 + .../kingbase/constants/KingbaseCons.java | 47 + .../kingbase/util/KingBaseDatabaseMeta.java | 140 ++ .../kingbase/util/KingBaseTypeConverter.java | 54 + .../flinkx-kingbase-reader/pom.xml | 107 ++ .../kingbase/format/KingbaseInputFormat.java | 89 ++ .../kingbase/reader/KingbaseReader.java | 49 + .../flinkx-kingbase-writer/pom.xml | 106 ++ .../kingbase/format/KingbaseOutputFormat.java | 246 ++++ .../kingbase/writer/KingbaseWriter.java | 87 ++ flinkx-kingbase/pom.xml | 35 + .../dtstack/flinkx/kudu/core/KuduUtil.java | 23 +- flinkx-kudu/flinkx-kudu-reader/pom.xml | 1 + .../flinkx/kudu/reader/KuduInputFormat.java | 15 +- .../kudu/reader/KuduInputFormatBuilder.java | 5 + .../flinkx/kudu/reader/KuduReader.java | 6 + flinkx-kudu/flinkx-kudu-writer/pom.xml | 1 + .../flinkx/kudu/writer/KuduOutputFormat.java | 13 +- .../kudu/writer/KuduOutputFormatBuilder.java | 4 + .../flinkx/kudu/writer/KuduWriter.java | 9 + flinkx-launcher/pom.xml | 24 +- .../flinkx/launcher/ClusterClientFactory.java | 5 + .../dtstack/flinkx/launcher/KerberosInfo.java | 149 ++ .../com/dtstack/flinkx/launcher/Launcher.java | 14 +- .../launcher/perJob/FlinkPerJobUtil.java | 2 +- .../perJob/PerJobClusterClientBuilder.java | 31 +- .../src/main/resources/log4j.properties | 13 + .../dtstack/mongodb/oplog/util/SqlUtil.java | 0 flinkx-mongodb/flinkx-mongodb-reader/pom.xml | 1 + .../mongodb/reader/MongodbInputFormat.java | 25 +- flinkx-mongodb/flinkx-mongodb-writer/pom.xml | 1 + .../flinkx-mongodboplog-reader/pom.xml | 88 ++ .../format}/MongodbEventHandler.java | 2 +- .../format}/MongodbOperation.java | 2 +- .../format/MongodboplogInputFormat.java} | 4 +- .../MongodboplogInputFormatBuilder.java} | 10 +- .../reader/MongodboplogReader.java} | 18 +- flinkx-mongodb/pom.xml | 2 +- flinkx-mysql/flinkx-mysql-dreader/pom.xml | 1 + flinkx-mysql/flinkx-mysql-reader/pom.xml | 1 + .../flinkx/mysql/format/MysqlInputFormat.java | 3 - flinkx-mysql/flinkx-mysql-writer/pom.xml | 1 + flinkx-odps/flinkx-odps-reader/pom.xml | 1 + flinkx-odps/flinkx-odps-writer/pom.xml | 1 + flinkx-oracle/flinkx-oracle-reader/pom.xml | 1 + .../flinkx/oracle/reader/OracleReader.java | 6 + flinkx-oracle/flinkx-oracle-writer/pom.xml | 1 + .../flinkx/oracle/writer/OracleWriter.java | 11 +- flinkx-pgwal/flinkx-pgwal-core/pom.xml | 22 + .../com/dtstack/flinkx/pgwal/PgDecoder.java | 274 ++++ .../flinkx/pgwal/PgMessageTypeEnum.java | 51 + .../flinkx/pgwal/PgRelicationSlot.java | 161 +++ .../dtstack/flinkx/pgwal/PgWalConfigKeys.java | 51 + .../com/dtstack/flinkx/pgwal/PgWalUtil.java | 254 ++++ .../java/com/dtstack/flinkx/pgwal/Table.java | 124 ++ .../flinkx-pgwal-reader}/pom.xml | 36 +- .../flinkx/pgwal/format/PgWalInputFormat.java | 202 +++ .../pgwal/format/PgWalInputFormatBuilder.java | 113 ++ .../flinkx/pgwal/listener/PgWalListener.java | 148 ++ .../flinkx/pgwal/reader/PgwalReader.java | 88 ++ flinkx-pgwal/pom.xml | 28 + flinkx-phoenix/flinkx-phoenix-reader/pom.xml | 1 + .../phoenix/format/PhoenixInputFormat.java | 2 +- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 3 +- flinkx-phoenix5/flinkx-phoenix5-core/pom.xml | 82 ++ .../flinkx/phoenix5/Phoenix5ConfigKeys.java | 22 +- .../flinkx/phoenix5/Phoenix5DatabaseMeta.java | 146 ++ .../flinkx/phoenix5/Phoenix5InputSplit.java | 50 + .../flinkx/phoenix5/util/IPhoenix5Helper.java | 215 +++ .../flinkx/phoenix5/util/PhoenixUtil.java | 99 ++ .../flinkx-phoenix5-reader/pom.xml | 102 ++ .../phoenix5/format/Phoenix5InputFormat.java | 288 ++++ .../reader/Phoenix5InputFormatBuilder.java | 98 ++ .../phoenix5/reader/Phoenix5Reader.java | 64 + .../flinkx-phoenix5-writer/pom.xml | 103 ++ .../phoenix5/format/Phoenix5OutputFormat.java | 124 ++ .../phoenix5/writer/Phoenix5Writer.java | 48 + flinkx-phoenix5/pom.xml | 36 + flinkx-polardb/flinkx-polardb-dreader/pom.xml | 1 + flinkx-polardb/flinkx-polardb-reader/pom.xml | 1 + flinkx-polardb/flinkx-polardb-writer/pom.xml | 1 + .../postgresql/PostgresqlTypeConverter.java | 14 +- .../flinkx-postgresql-reader/pom.xml | 1 + .../reader/PostgresqlQuerySqlBuilder.java | 3 +- .../flinkx-postgresql-writer/pom.xml | 1 + .../format/PostgresqlOutputFormat.java | 111 +- flinkx-pulsar/flinkx-pulsar-writer/pom.xml | 4 +- .../pulsar/{writer => format}/Constants.java | 2 +- .../PulsarOutputFormat.java | 12 +- .../PulsarOutputFormatBuilder.java | 2 +- .../flinkx/pulsar/writer/PulsarWriter.java | 3 +- .../pulsar/writer/decoder/JsonDecoder.java | 54 - flinkx-rdb/flinkx-rdb-core/pom.xml | 1 + .../com/dtstack/flinkx/rdb/util/DbUtil.java | 60 +- flinkx-rdb/flinkx-rdb-reader/pom.xml | 1 + .../JdbcConfigKeys.java | 2 + .../JdbcDataReader.java | 5 + .../QuerySqlBuilder.java | 21 +- .../JdbcInputFormat.java | 122 +- .../JdbcInputFormatBuilder.java | 5 + flinkx-rdb/flinkx-rdb-writer/pom.xml | 1 + .../JdbcConfigKeys.java | 1 + .../JdbcDataWriter.java | 4 + .../rdb/outputformat/JdbcOutputFormat.java | 98 +- .../outputformat/JdbcOutputFormatBuilder.java | 16 + flinkx-redis/flinkx-redis-writer/pom.xml | 1 + .../redis/writer/RedisOutputFormat.java | 4 +- flinkx-restapi/flinkx-restapi-core/pom.xml | 6 + .../flinkx/restapi/common/ConstantParam.java | 112 ++ .../restapi/common/ConstantVarible.java | 22 + .../restapi/common/CurrentTimeVarible.java | 43 + .../flinkx/restapi/common/DymaticParam.java | 260 ++++ .../flinkx/restapi/common/HttpUtil.java | 2 +- .../restapi/common/InnerVaribleFactory.java | 53 + .../restapi/common/IntervalTimeVarible.java | 43 + .../flinkx/restapi/common/MapUtils.java | 37 + .../restapi/common/ParamDefinition.java | 56 + .../common/ParamDefinitionNextAble.java | 31 + .../flinkx/restapi/common/ParamFactory.java | 105 ++ .../flinkx/restapi/common/ParamType.java | 32 + .../flinkx/restapi/common/Paramitem.java | 31 + .../restapi/common/ReplaceParamItem.java | 48 + .../flinkx/restapi/common/RestContext.java | 188 +++ .../flinkx/restapi/common/UuidVarible.java | 21 + .../common/exception/ReadRecordException.java | 10 + .../exception/ResponseBreakException.java | 22 + .../exception/ResponseRetryException.java | 22 + .../common/handler/BreakDataHandler.java | 26 + .../restapi/common/handler/DataHandler.java | 23 + .../common/handler/DataHandlerFactory.java | 29 + .../common/handler/RetryDataHandler.java | 24 + .../flinkx/restapi/common/httprequestApi.java | 76 + flinkx-restapi/flinkx-restapi-reader/pom.xml | 1 + .../restapi/inputformat/HttpClient.java | 194 +++ .../inputformat/RestapiInputFormat.java | 83 +- .../RestapiInputFormatBuilder.java | 32 +- .../flinkx/restapi/reader/HttpRestConfig.java | 113 ++ .../flinkx/restapi/reader/RestapiReader.java | 49 +- flinkx-restapi/flinkx-restapi-writer/pom.xml | 1 + .../outputformat/RestapiOutputFormat.java | 16 +- .../flinkx/restapi/writer/RestapiWriter.java | 3 + flinkx-saphana/flinkx-saphana-reader/pom.xml | 1 + flinkx-saphana/flinkx-saphana-writer/pom.xml | 1 + .../flinkx-sqlserver-reader/pom.xml | 1 + .../reader/SqlserverQuerySqlBuilder.java | 3 +- .../flinkx-sqlserver-writer/pom.xml | 1 + flinkx-stream/flinkx-stream-reader/pom.xml | 1 + flinkx-stream/flinkx-stream-writer/pom.xml | 1 + .../stream/writer/StreamOutputFormat.java | 43 +- .../flinkx/teradata/Phoenix5ConfigKeys.java | 0 .../flinkx/teradata/Phoenix5DatabaseMeta.java | 0 .../flinkx/teradata/Phoenix5InputSplit.java | 0 .../flinkx/teradata/util/IPhoenix5Helper.java | 0 .../reader/Phoenix5InputFormatBuilder.java | 0 flinkx-test/pom.xml | 34 +- .../com/dtstack/flinkx/test/LocalTest.java | 86 +- .../flinkx/test/PluginNameConstants.java | 10 +- .../AbstractPrometheusReporter.java | 315 ---- .../PrometheusPushGatewayReporter.java | 94 -- flinkx-test/src/main/resources/logback.xml | 2 +- jars/vertica-jdbc-9.1.1-0.jar | Bin 0 -> 769344 bytes pom.xml | 9 +- 334 files changed, 9988 insertions(+), 3538 deletions(-) create mode 100644 docs/example/kafka09_stream.json create mode 100644 docs/example/kafka10_stream.json create mode 100644 docs/example/kafka11_stream.json create mode 100644 docs/example/kafka_hive.json create mode 100644 docs/example/kafka_mysql.json create mode 100644 docs/example/kafka_stream.json create mode 100644 docs/example/stream_stream.json create mode 100644 docs/questions.md create mode 100644 flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java rename flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/{reader => format}/BinlogInputFormat.java (63%) create mode 100644 flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java rename flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/{reader => listener}/BinlogAlarmHandler.java (94%) rename flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/{reader => listener}/BinlogEventSink.java (88%) rename flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/{reader => listener}/BinlogPositionManager.java (95%) rename flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/{reader => listener}/HeartBeatController.java (96%) rename flinkx-core/src/main/java/com/dtstack/flinkx/decoder/{PlainDecoder.java => TextDecoder.java} (95%) rename flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java => flinkx-core/src/main/java/com/dtstack/flinkx/exception/DataSourceException.java (63%) create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/util/ZkHelper.java delete mode 100644 flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java create mode 100644 flinkx-core/src/test/java/com/dtstack/flinkx/util/ZkHelperTest.java create mode 100644 flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java rename flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/{reader => client}/KafkaConsumer.java (58%) create mode 100644 flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/format/KafkaInputFormat.java delete mode 100644 flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java rename flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/{writer => format}/KafkaOutputFormat.java (77%) rename flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/{reader => client}/Kafka09Client.java (69%) rename flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/{reader => client}/Kafka09Consumer.java (82%) rename flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/{reader => format}/Kafka09InputFormat.java (76%) rename flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/{writer => format}/Kafka09OutputFormat.java (78%) rename flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/{reader => client}/Kafka10Client.java (82%) rename flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/{reader => client}/Kafka10Consumer.java (71%) rename flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/{reader => format}/Kafka10InputFormat.java (71%) rename flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/{writer => format}/Kafka10OutputFormat.java (77%) rename flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/{reader => client}/Kafka11Client.java (78%) rename flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/{reader => client}/Kafka11Consumer.java (71%) rename flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/{reader => format}/Kafka11InputFormat.java (71%) rename flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/{writer => format}/Kafka11OutputFormat.java (77%) rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/entity/QueueData.java => flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaInputSplit.java (56%) create mode 100644 flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java rename flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java => flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/KafkaVersion.java (71%) create mode 100644 flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/StartupMode.java rename flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/{ => util}/Formatter.java (98%) create mode 100644 flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java rename flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/{reader => client}/IClient.java (74%) rename flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/{reader => client}/KafkaBaseConsumer.java (64%) rename flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/{reader => format}/KafkaBaseInputFormat.java (55%) create mode 100644 flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java rename flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/{writer => format}/KafkaBaseOutputFormat.java (77%) delete mode 100644 flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/AddressUtil.java rename {flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09 => flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase}/writer/HeartBeatController.java (81%) create mode 100644 flinkx-kingbase/flinkx-kingbase-core/pom.xml create mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java create mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java create mode 100644 flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java create mode 100644 flinkx-kingbase/flinkx-kingbase-reader/pom.xml create mode 100644 flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java create mode 100644 flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java create mode 100644 flinkx-kingbase/flinkx-kingbase-writer/pom.xml create mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java create mode 100644 flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java create mode 100644 flinkx-kingbase/pom.xml create mode 100644 flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/KerberosInfo.java create mode 100644 flinkx-launcher/src/main/resources/log4j.properties delete mode 100644 flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/util/SqlUtil.java create mode 100644 flinkx-mongodb/flinkx-mongodboplog-reader/pom.xml rename flinkx-mongodb/{flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader => flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format}/MongodbEventHandler.java (98%) rename flinkx-mongodb/{flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader => flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format}/MongodbOperation.java (96%) rename flinkx-mongodb/{flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java => flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormat.java} (98%) rename flinkx-mongodb/{flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java => flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormatBuilder.java} (81%) rename flinkx-mongodb/{flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java => flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/reader/MongodboplogReader.java} (78%) create mode 100644 flinkx-pgwal/flinkx-pgwal-core/pom.xml create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java create mode 100644 flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java rename {flinkx-mongodb/flinkx-mongodb-oplog-reader => flinkx-pgwal/flinkx-pgwal-reader}/pom.xml (61%) create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java create mode 100644 flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java create mode 100644 flinkx-pgwal/pom.xml create mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/pom.xml rename flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java => flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5ConfigKeys.java (73%) create mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5DatabaseMeta.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5InputSplit.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/IPhoenix5Helper.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml create mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5InputFormatBuilder.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5Reader.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml create mode 100644 flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java create mode 100644 flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java create mode 100644 flinkx-phoenix5/pom.xml rename flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/{writer => format}/Constants.java (96%) rename flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/{writer => format}/PulsarOutputFormat.java (91%) rename flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/{writer => format}/PulsarOutputFormatBuilder.java (97%) delete mode 100644 flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java create mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java create mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogFile.java => flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5ConfigKeys.java (100%) rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerConnection.java => flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5DatabaseMeta.java (100%) rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerListener.java => flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5InputSplit.java (100%) rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogParser.java => flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/IPhoenix5Helper.java (100%) rename flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/PositionManager.java => flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/Phoenix5InputFormatBuilder.java (100%) delete mode 100644 flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java delete mode 100644 flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java create mode 100644 jars/vertica-jdbc-9.1.1-0.jar diff --git a/.gitignore b/.gitignore index 1e7d443821..09167d98ec 100644 --- a/.gitignore +++ b/.gitignore @@ -10,4 +10,5 @@ nohup.out flinkconf/ hadoopconf/ /default_task_id_output -/syncplugins \ No newline at end of file +/syncplugins +/ci/ diff --git a/README.md b/README.md index c391b42f5c..0a57779807 100644 --- a/README.md +++ b/README.md @@ -66,12 +66,10 @@ The following databases are currently supported: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | -| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | +| | RestApi || [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon | | -| | SqlServer CDC | Coming Soon | | # Quick Start @@ -89,6 +87,10 @@ Please click [Statistics Metric](docs/statistics.md) Please click [Kerberos](docs/kerberos.md) +# Questions + +Please click [Questions](docs/questions.md) + # How to contribute FlinkX Please click [Contribution](docs/contribution.md) diff --git a/README_CH.md b/README_CH.md index 8bb8c11dd0..cec944897d 100644 --- a/README_CH.md +++ b/README_CH.md @@ -7,7 +7,18 @@ FlinkX # 技术交流 -- 招聘**大数据平台开发工程师**,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) +- 招聘**Flink研发工程师**,如果有兴趣可以联系思枢(微信号:ysqwhiletrue)
+Flink开发工程师JD要求:
+1.负责袋鼠云基于Flink的衍生框架数据同步flinkx和实时计算flinkstreamsql框架的开发;
+2.调研和把握当前最新大数据实时计算技术,将其中的合适技术引入到平台中,改善产品,提升竞争力;
+职位要求:
+1、本科及以上学历,3年及以上的Flink开发经验,精通Java,熟悉Scala、Python优先考虑;
+2、熟悉Flink原理,有基于Flink做过二次源码的开发,在github上贡献者Flink源码者优先;
+3、有机器学习、数据挖掘相关经验者优先;
+4、对新技术有快速学习和上手能力,对代码有一定的洁癖;
+加分项:
+1.在GitHub或其他平台上有过开源项目
+可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) - 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群 @@ -66,12 +77,10 @@ FlinkX目前支持下面这些数据库: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | -| | RestApi | [doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | +| | RestApi | | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle Logminer| Coming Soon | | -| | SqlServer CDC | Coming Soon | | # 快速开始 @@ -89,6 +98,10 @@ FlinkX目前支持下面这些数据库: 请点击[Kerberos](docs/kerberos.md) +# Questions + +请点击[Questions](docs/questions.md) + # 如何贡献FlinkX 请点击[如何贡献FlinkX](docs/contribution.md) diff --git a/bin/install_jars.bat b/bin/install_jars.bat index c3910d5ff5..7b00a7d7da 100644 --- a/bin/install_jars.bat +++ b/bin/install_jars.bat @@ -6,3 +6,6 @@ call mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversi call mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar +call mvn install:install-file -DgroupId=com.kingbase8 -DartifactId=kingbase8 -Dversion=8.2.0 -Dpackaging=jar -Dfile=../jars/kingbase8-8.2.0.jar + +call mvn install:install-file -DgroupId=fakepath -DartifactId=vertica-jdbc -Dversion=9.1.1-0 -Dpackaging=jar -Dfile=../jars/vertica-jdbc-9.1.1-0.jar diff --git a/bin/install_jars.sh b/bin/install_jars.sh index cac9471eed..fdeafa149d 100644 --- a/bin/install_jars.sh +++ b/bin/install_jars.sh @@ -10,4 +10,10 @@ mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversi mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar ## dm driver -mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar \ No newline at end of file +mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar + +## kingbase driver +mvn install:install-file -DgroupId=com.kingbase8 -DartifactId=kingbase8 -Dversion=8.2.0 -Dpackaging=jar -Dfile=../jars/kingbase8-8.2.0.jar + +## vertica driver +mvn install:install-file -DgroupId=fakepath -DartifactId=vertica-jdbc -Dversion=9.1.1-0 -Dpackaging=jar -Dfile=../jars/vertica-jdbc-9.1.1-0.jar \ No newline at end of file diff --git a/docs/example/kafka09_stream.json b/docs/example/kafka09_stream.json new file mode 100644 index 0000000000..cb69ecbee6 --- /dev/null +++ b/docs/example/kafka09_stream.json @@ -0,0 +1,36 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "topic": "kafka09", + "groupId": "default", + "codec": "text", + "encoding": "UTF-8", + "blankIgnore": false, + "consumerSettings": { + "zookeeper.connect": "localhost:2181/kafka09" + } + }, + "name": "kafka09reader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/kafka10_stream.json b/docs/example/kafka10_stream.json new file mode 100644 index 0000000000..d25f6859bd --- /dev/null +++ b/docs/example/kafka10_stream.json @@ -0,0 +1,42 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "topic": "kafka10", + "groupId": "default", + "codec": "text", + "encoding": "UTF-8", + "blankIgnore": false, + "consumerSettings": { + "bootstrap.servers": "localhost:9092" + } + }, + "name": "kafka10reader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + }, + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } +} \ No newline at end of file diff --git a/docs/example/kafka11_stream.json b/docs/example/kafka11_stream.json new file mode 100644 index 0000000000..398b2c4e77 --- /dev/null +++ b/docs/example/kafka11_stream.json @@ -0,0 +1,36 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "topic": "kafka11", + "groupId": "default", + "codec": "text", + "encoding": "UTF-8", + "blankIgnore": false, + "consumerSettings": { + "bootstrap.servers": "localhost:9092" + } + }, + "name": "kafka11reader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/kafka_hive.json b/docs/example/kafka_hive.json new file mode 100644 index 0000000000..4e724b1fcf --- /dev/null +++ b/docs/example/kafka_hive.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [ + { + "reader" : { + "parameter" : { + "topic" : "test", + "mode": "timestamp", + "timestamp": 1609812275000, + "codec": "text", + "consumerSettings" : { + "bootstrap.servers" : "ip1:9092,ip2:9092,ip3:9092" + } + }, + "name" : "kafkareader" + }, + "writer": { + "parameter" : { + "jdbcUrl" : "jdbc:hive2://ip:10000/test", + "fileType" : "parquet", + "writeMode" : "overwrite", + "compress" : "", + "charsetName" : "UTF-8", + "maxFileSize" : 1073741824, + "tablesColumn" : "{\"message\":[{\"part\":false,\"comment\":\"\",\"type\":\"string\",\"key\":\"message\"}]}", + "partition" : "pt", + "partitionType" : "DAY", + "defaultFS" : "hdfs://ns", + "hadoopConfig": { + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn2": "ip1:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns.nn1": "ip2:9000", + "dfs.nameservices": "ns" + } + }, + "name" : "hivewriter" + } + } + ], + "setting": { + "restore": { + "isRestore": true, + "isStream": true + }, + "speed": { + "readerChannel": 3, + "writerChannel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/kafka_mysql.json b/docs/example/kafka_mysql.json new file mode 100644 index 0000000000..2fa5a950af --- /dev/null +++ b/docs/example/kafka_mysql.json @@ -0,0 +1,65 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "topic": "tudou", + "mode": "timestamp", + "timestamp": 1609812275000, + "offset": "partition:0,offset:0;partition:1,offset:1;partition:2,offset:2", + "codec": "text", + "blankIgnore": false, + "consumerSettings": { + "bootstrap.servers": "ip1:9092,ip2:9092,ip3:9092" + }, + "column": ["id","user_id","name"] + }, + "name": "kafkareader" + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "root", + "password": "abc123", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://localhost:3306/test", + "table": [ + "test" + ] + } + ], + "preSql": ["truncate table test;"], + "postSql": [], + "writeMode": "insert", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + } + ] + } + } + } + ], + "setting": { + "restore": { + "isRestore": true, + "isStream": true + }, + "speed": { + "readerChannel": 3, + "writerChannel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/kafka_stream.json b/docs/example/kafka_stream.json new file mode 100644 index 0000000000..a3d8f45a72 --- /dev/null +++ b/docs/example/kafka_stream.json @@ -0,0 +1,34 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "topic": "test", + "mode": "timestamp", + "timestamp": 1609812275000, + "offset": "partition:0,offset:0;partition:1,offset:1;partition:2,offset:2", + "codec": "text", + "blankIgnore": false, + "consumerSettings": { + "bootstrap.servers": "ip1:9092,ip2:9092,ip3:9092" + } + }, + "name": "kafkareader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "readerChannel": 3, + "writerChannel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_stream.json b/docs/example/stream_stream.json new file mode 100644 index 0000000000..6785e2f0ed --- /dev/null +++ b/docs/example/stream_stream.json @@ -0,0 +1,30 @@ +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "column" : [ { + "name": "id", + "type" : "id" + }, { + "name": "string", + "type" : "string" + } ], + "sliceRecordCount" : [ "10"] + }, + "name" : "streamreader" + }, + "writer" : { + "parameter" : { + "print" : true + }, + "name" : "streamwriter" + } + } ], + "setting" : { + "speed" : { + "channel" : 1 + } + } + } +} \ No newline at end of file diff --git a/docs/generalconfig.md b/docs/generalconfig.md index d289482516..f0d1ed44d2 100644 --- a/docs/generalconfig.md +++ b/docs/generalconfig.md @@ -8,22 +8,18 @@ "job" : { "content" :[{ "reader" : { - ...... }, "writer" : { - ...... } }], "setting" : { "restore" : { - ...... }, "speed" : { }, "dirty" : { }, "log" : { - ...... } } } @@ -68,12 +64,13 @@ reader用于配置数据的输入源,即数据从何而来。具体配置如 writer用于配置数据的输出源,即数据写往何处。具体配置如下所示: ```json +{ "writer" : { "name" : "xxwriter", "parameter" : { - ...... } } +} ``` | 名称 | 说明 | 是否必填 | | --- | --- | --- | @@ -89,6 +86,7 @@ writer用于配置数据的输出源,即数据写往何处。具体配置如 restore用于配置同步任务类型(离线同步、实时采集)和断点续传功能。具体配置如下所示: ```json +{ "restore" : { "isStream" : false, "isRestore" : false, @@ -96,6 +94,7 @@ restore用于配置同步任务类型(离线同步、实时采集)和断点 "restoreColumnIndex" : 0, "maxRowNumForCheckpoint" : 10000 } +} ``` | 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | | --- | --- | --- | --- | --- | @@ -111,10 +110,12 @@ restore用于配置同步任务类型(离线同步、实时采集)和断点 speed用于配置任务并发数及速率限制。具体配置如下所示: ```json +{ "speed" : { "channel": 1, "bytes": 0 } +} ``` | 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | | --- | --- | --- | --- | --- | @@ -127,10 +128,12 @@ speed用于配置任务并发数及速率限制。具体配置如下所示: errorLimit用于配置任务运行时数据读取写入的出错控制。具体配置如下所示: ```json +{ "errorLimit" : { "record": 100, "percentage": 10.0 } +} ``` | 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | | --- | --- | --- | --- | --- | @@ -143,12 +146,13 @@ errorLimit用于配置任务运行时数据读取写入的出错控制。具体 dirty用于配置脏数据的保存,通常与上文出错控制联合使用。具体配置如下所示: ```json +{ "dirty" : { "path" : "xxx", "hadoopConfig" : { - ...... } } +} ``` | 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | | --- | --- | --- | --- | --- | @@ -159,6 +163,7 @@ dirty用于配置脏数据的保存,通常与上文出错控制联合使用。 参考模板如下: ```json +{ "dirty" : { "path" : "/user/hive/warehouse/xx.db/xx", "hadoopConfig" : { @@ -169,7 +174,8 @@ dirty用于配置脏数据的保存,通常与上文出错控制联合使用。 "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", "dfs.nameservices" : "ns1" } - } + } +} ``` @@ -177,13 +183,14 @@ dirty用于配置脏数据的保存,通常与上文出错控制联合使用。 log用于配置Flinkx中定义的插件日志的保存与记录。具体配置如下所示: ```json +{ "log" : { "isLogger": false, "level" : "info", "path" : "/tmp/dtstack/flinkx/", "pattern":"" } - +} ``` | 名称 | 说明 | 是否必填 | 默认值 | 参数类型 | | --- | --- | --- | --- | --- | diff --git a/docs/offline/reader/mongodbreader.md b/docs/offline/reader/mongodbreader.md index 89c15c32b1..af8cb1a7ad 100644 --- a/docs/offline/reader/mongodbreader.md +++ b/docs/offline/reader/mongodbreader.md @@ -68,7 +68,6 @@ - 格式:支持3中格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: ```json -"column":["*"] {"column":["*"]} ``` 2.只指定字段名称: diff --git a/docs/offline/reader/postgresqlreader.md b/docs/offline/reader/postgresqlreader.md index 39c5bfc3aa..302eb54d36 100644 --- a/docs/offline/reader/postgresqlreader.md +++ b/docs/offline/reader/postgresqlreader.md @@ -331,7 +331,6 @@ "customSql": "", "increColumn": "id", "startLocation": "20", - "queryTimeOut": 1000, "requestAccumulatorInterval": 2 }, "name" : "postgresqlreader" @@ -391,7 +390,6 @@ "customSql": "", "increColumn": "id", "startLocation": "20", - "queryTimeOut": 1000, "requestAccumulatorInterval": 2 }, "name" : "postgresqlreader" diff --git a/docs/offline/reader/sqlserverreader.md b/docs/offline/reader/sqlserverreader.md index c17decd8d0..e5cebc2803 100644 --- a/docs/offline/reader/sqlserverreader.md +++ b/docs/offline/reader/sqlserverreader.md @@ -396,6 +396,7 @@ "path" : "", "pattern":"" } + } } } ``` diff --git a/docs/offline/writer/postgresqlwriter.md b/docs/offline/writer/postgresqlwriter.md index cc567de731..4df7a7c9c5 100644 --- a/docs/offline/writer/postgresqlwriter.md +++ b/docs/offline/writer/postgresqlwriter.md @@ -105,7 +105,7 @@ }, { "name":"name", - "type:"string" + "type":"string" } ] }, @@ -174,7 +174,7 @@ }, { "name":"name", - "type:"string" + "type":"string" } ] }, diff --git a/docs/offline/writer/sqlserverwriter.md b/docs/offline/writer/sqlserverwriter.md index 320042de7d..4c3317ab49 100644 --- a/docs/offline/writer/sqlserverwriter.md +++ b/docs/offline/writer/sqlserverwriter.md @@ -105,7 +105,7 @@ }, { "name":"name", - "type:"string" + "type":"string" } ] }, @@ -175,7 +175,7 @@ }, { "name":"name", - "type:"string" + "type":"string" } ] }, diff --git a/docs/questions.md b/docs/questions.md new file mode 100644 index 0000000000..64073f9c6c --- /dev/null +++ b/docs/questions.md @@ -0,0 +1,24 @@ +## 常见问题 + +### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 + +解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: + +```bash +## windows平台 +./install_jars.bat + +## unix平台 +./install_jars.sh +``` + +### 2.FlinkX版本需要与Flink版本保持一致 +1.8_release版本对应flink1.8 +1.10_release版本对应flink1.10 版本 +不对应在standalone和yarn session模式提交时,会报错: +Caused by: java.io.InvalidClassException: org.apache.flink.api.common.operators.ResourceSpec; incompatible types for field cpuCores + +### 3.移动FlinkX lib目录下的Launcher包后,任务启动报错:错误: 找不到或无法加载主类 +FlinkX启动脚本里面找的是lib目录下的所有jar包,而移动后lib中含有其他的jar包,这些jar包没有主类,因此报错 +可以使用如下的命令运行: +java -cp /opt/flink/flink/deps/lib/flinkx-launcher-1.6.jar com.dtstack.flinkx.launcher.Launcher -mode local -job /opt/flink/flink/deps/job/stream.json -pluginRoot /opt/flink/flink/deps/syncplugins \ No newline at end of file diff --git a/docs/quickstart.md b/docs/quickstart.md index 4625c44d82..bc054b5444 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -10,9 +10,9 @@ cd flinkx 2.直接下载源码 ``` -wget https://github.com/DTStack/flinkx/archive/1.8.5.zip -unzip flinkx-1.8.5.zip -cd flink-1.8.5 +wget https://github.com/DTStack/flinkx/archive/1.10_release.zip +unzip 1.10_release.zip +cd 1.10_release ``` ## 编译插件 @@ -21,6 +21,20 @@ cd flink-1.8.5 mvn clean package -DskipTests ``` +## 常见问题 + +### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 + +解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: + +```bash +## windows平台 +./install_jars.bat + +## unix平台 +./install_jars.sh +``` + ## 运行任务 首先准备要运行的任务json,这里以stream插件为例: @@ -59,9 +73,7 @@ mvn clean package -DskipTests "errorLimit" : { }, "speed" : { - "bytes" : 0, - "channel" : 1, - "rebalance" : true + "channel" : 1 } } } @@ -73,10 +85,11 @@ mvn clean package -DskipTests 命令模板: ```bash -bin/flinkx -mode local \ - -job $FLINK_HOME/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ - -confProp "{\"flink.checkpoint.interval\":60000}" +bin/flinkx \ + -mode local \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINKX_HOME/syncplugins \ + -confProp "{\"flink.checkpoint.interval\":60000}" ``` 可以在flink的配置文件里配置端口: @@ -89,8 +102,10 @@ rest.bind-port: 8888 使用下面的命令运行任务: ```bash -./bin/flinkx -job ./job/stream.json - -flinkconf $FLINK_HOME/conf +bin/flinkx \ + -mode local \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINK_HOME/syncplugins \ ``` 任务运行后可以通过8888端口访问flink界面查看任务运行情况: @@ -104,11 +119,12 @@ rest.bind-port: 8888 命令模板: ```bash -bin/flinkx -mode standalone \ - -job $FLINK_HOME/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ - -flinkconf $FLINK_HOME/conf \ - -confProp "{\"flink.checkpoint.interval\":60000}" +bin/flinkx \ + -mode standalone \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINKX_HOME/syncplugins \ + -flinkconf $FLINK_HOME/conf \ + -confProp "{\"flink.checkpoint.interval\":60000}" ``` 首先启动flink集群: @@ -127,9 +143,10 @@ $FLINK_HOME/bin/start-cluster.sh 把任务提交到集群上运行: ```bash -./bin/flinkx -mode standalone \ - -job ./job/stream.json \ - -flinkconf $FLINK_HOME/conf +./bin/flinkx \ + -mode standalone \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -flinkconf $FLINK_HOME/conf ``` 在集群上查看任务运行情况 @@ -143,12 +160,13 @@ $FLINK_HOME/bin/start-cluster.sh 命令示例: ```bash -bin/flinkx -mode yarn \ - -job $FLINK_HOME/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ - -flinkconf $FLINK_HOME/conf \ - -yarnconf $HADOOP_HOME/etc/hadoop \ - -confProp "{\"flink.checkpoint.interval\":60000}" +bin/flinkx \ + -mode yarn \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINKX_HOME/syncplugins \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -confProp "{\"flink.checkpoint.interval\":60000}" ``` 首先确保yarn集群是可用的,然后手动启动一个yarn session: @@ -162,16 +180,17 @@ $FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024

- +
把任务提交到这个yarn session上: ```bash -./bin/flinkx -mode yarn \ - -job ./job/stream.json \ - -flinkconf $FLINK_HOME/conf \ - -yarnconf $HADOOP_HOME/etc/hadoop +bin/flinkx \ + -mode yarn \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop ``` 然后在flink界面查看任务运行情况: @@ -185,26 +204,28 @@ $FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024 命令示例: ```bash -bin/flinkx -mode yarnPer \ - -job ${FLINK_HOME}/examples/stream_example.json \ - -pluginRoot $FLINK_HOME/plugins \ - -flinkconf $FLINK_HOME/conf \ - -yarnconf $HADOOP_HOME/etc/hadoop \ - -flinkLibJar $FLINK_HOME/lib \ - -confProp "{\"flink.checkpoint.interval\":60000}" \ - -queue default \ - -pluginLoadMode classpath +bin/flinkx \ + -mode yarnPer \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINKX_HOME/syncplugins \ + -flinkconf $FLINK_HOME/conf \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -flinkLibJar $FLINK_HOME/lib \ + -confProp "{\"flink.checkpoint.interval\":60000}" \ + -queue default \ + -pluginLoadMode classpath ``` 首先确保yarn集群是可用的,启动一个Yarn Application运行任务: ```bash -bin/flinkx -mode yarnPer \ - -job ./job/stream.json \ - -flinkconf $FLINK_HOME/conf \ - -yarnconf $HADOOP_HOME/etc/hadoop \ - -flinkLibJar $FLINK_HOME/lib \ - -pluginLoadMode classpath +bin/flinkx \ + -mode yarnPer \ + -job $FLINKX_HOME/docs/example/stream_stream.json \ + -pluginRoot $FLINKX_HOME/syncplugins \ + -yarnconf $HADOOP_HOME/etc/hadoop \ + -flinkLibJar $FLINK_HOME/lib \ + -pluginLoadMode classpath ``` 然后在集群上查看任务运行情况 @@ -221,29 +242,20 @@ bin/flinkx -mode yarnPer \ | 名称 | 说明 | 可选值 | 是否必填 | 默认值 | | ------------------ | ------------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---- | ----------------------- | -| **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | +| **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | | **job** | 数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息 | 无 | 是 | 无 | | **jobid** | 任务名称 | 无 | 否 | Flink Job | | **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/plugins | | **flinkconf** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | -| **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.8.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | +| **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | | **yarnconf** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | | **queue** | yarn队列,如default | 无 | 否 | default | | **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | | **confProp** | checkpoint配置 | **flink.checkpoint.interval**:快照生产频率
**flink.checkpoint.stateBackend**:快照存储路径 | 否 | 无 | | **s** | checkpoint快照路径 | | 否 | 无 | | **p** | 自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102| | 否 | 无 | +| **appId** | yarn模式下,提交到指定的的flink session的application Id | | 否 | 无 | +| **krb5conf** | 提交到开启kerberos的Hadoop集群的krb5文件路径 | | 否 | 无 | +| **keytab** | 提交到开启kerberos的Hadoop集群的keytab文件路径 | | 否 | 无 | +| **principal** | kerberos认证的principal | | 否 | 无 | -## 常见问题 - -### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 - -解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: - -```bash -## windows平台 -./install_jars.bat - -## unix平台 -./install_jars.sh -``` diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index 531d681ee8..bd94bc72ad 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -1,6 +1,5 @@ # Kafka Reader - ## 一、插件名称 kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: @@ -9,101 +8,151 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.9 | kafka09reader | | kafka 0.10 | kafka10reader | | kafka 0.11 | kafka11reader | -| kafka 1.0及以上 | kafkareader | +| kafka 1.0及以后 | kafkareader | + -
- -二、参数说明
+ +## 二、参数说明 - **topic** - - 描述:要消费的topic - - 必选:是 - - 默认值:无 + - 描述:要消费的topic,多个以,分割,当`mode`为`timestamp`、`specific-offsets`时不支持多topic + - 必选:是 + - 字段类型:String + - 默认值:无 +
+- **mode** + - 描述:kafka消费端启动模式,目前仅支持`kafkareader`插件 + - 可选值: + - group-offsets:     从ZK / Kafka brokers中指定的消费组已经提交的offset开始消费 + - earliest-offset:    从最早的偏移量开始(如果可能) + - latest-offset:     从最新的偏移量开始(如果可能) + - timestamp:         从每个分区的指定的时间戳开始 + - specific-offsets: 从每个分区的指定的特定偏移量开始 + - 必选:否 + - 字段类型:String + - 默认值:group-offsets -- **groupId** - - 描述:kafka消费组Id - - 注意:该参数对kafka09reader插件无效 - - 必选:是 - - 默认值:无 +
+- **timestamp** + - 描述:指定的kafka时间戳采集起点,目前仅支持`kafkareader`插件 + - 必选:当`mode`为`timestamp`时必选 + - 字段类型:Long + - 默认值:无 +
-- **encoding** - - 描述:字符编码 - - 注意:该参数只对kafka09reader插件有效 - - 必选:否 - - 默认值:UTF-8 +- **offset** + - 描述:消费的分区及对应的特定偏移量,目前仅支持`kafkareader`插件 + - 必选:当`mode`为`specific-offsets`时必选 + - 字段类型:String + - 格式:partition:0,offset:42;partition:1,offset:300;partition:2,offset:300 + - 默认值:无 + +
+- **groupId** + - 描述:kafka消费组Id + - 必选:否 + - 字段类型:String + - 默认值:default +
-- **codec** - - 描述:编码解码器类型,支持 json、plain - - plain:将kafka获取到的消息字符串存储到一个key为message的map中,如:`{"message":"{\"key\": \"key\", \"message\": \"value\"}"}` - - json:将kafka获取到的消息字符串按照json格式进行解析 - - 若该字符串为json格式 - - 当其中含有message字段时,原样输出,如:`{"key": "key", "message": "value"}` - - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,如:`{"key": "key", "value": "value", "message": "{\"key\": \"key\", \"value\": \"value\"}"}` - - 若改字符串不为json格式,则按照plain类型进行处理 - - 必选:否 - - 默认值:plain +- **encoding** + - 描述:字符编码 + - 必选:否 + - 字段类型:String + - 默认值:UTF-8 +
+- **codec** + - 描述:编码解码器类型,支持 json、text + - text: + 将kafka获取到的消息字符串存储到一个key为message的map中,如:kafka中的消息为:{"key":"key","message":"value"}, + 则发送至下游的数据格式为: -- **blankIgnore** - - 描述:是否忽略空值消息 - - 必选:否 - - 默认值:false + ```json + [ + { + "message":"{\"key\": \"key\", \"value\": \"value\"}" + } + ] + ``` + - json:将kafka获取到的消息字符串按照json格式进行解析 + - 若该字符串为json格式 + - 当其中含有message字段时,发送至下游的数据格式为: + ```json + [ + { + "key":"key", + "message":"value" + } + ] + ``` + - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,发送至下游的数据格式为: + ```json + [ + { + "key":"key", + "value":"value", + "message":"{\"key\": \"key\", \"value\": \"value\"}" + } + ] + ``` + - 若改字符串不为json格式,则按照text类型进行处理 + - 必选:否 + - 字段类型:String + - 默认值:text -- **consumerSettings** - - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 - - 必选:是 - - 默认值:无 +
+- **blankIgnore** + - 描述:是否忽略空值消息 + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
-- **column** - - 描述:需要读取的字段。 - - 格式: +- **consumerSettings** + - 描述:kafka连接配置,支持所有`kafka.consumer.ConsumerConfig.ConsumerConfig`中定义的配置 + - 必选:是 + - 字段类型:Map + - 默认值:无 + - 注意: + - kafka09 reader插件: consumerSettings必须至少包含`zookeeper.connect`参数 + - kafka09 reader以外的插件:consumerSettings必须至少包含`bootstrap.servers`参数 + - 如: ```json -"column": [{ - "name": "col", - "type": "datetime", - "format": "yyyy-MM-dd hh:mm:ss", - "value": "value" -}] +{ + "consumerSettings":{ + "bootstrap.servers":"host1:9092,host2:9092,host3:9092" + } +} ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - 必选:是 - - 默认值:无 - - ## 三、配置示例 - #### 1、kafka09 ```json { - "job": { - "content": [{ + "job" : { + "content" : [ { "reader" : { "parameter" : { "topic" : "kafka09", - "codec": "plain", + "groupId" : "default", + "codec" : "text", "encoding": "UTF-8", + "blankIgnore": false, "consumerSettings" : { - "zookeeper.connect" : "0.0.0.1:2182/kafka09", - "group.id" : "default", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "smallest" + "zookeeper.connect" : "localhost:2181/kafka09" } }, "name" : "kafka09reader" @@ -115,101 +164,71 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "name" : "streamwriter" } } ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : true, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 2、kafka10 ```json { "job": { - "content": [{ - "reader" : { - "parameter" : { - "topic" : "kafka10", - "groupId" : "default", - "codec": "plain", - "blankIgnore": false, - "consumerSettings" : { - "zookeeper.connect" : "0.0.0.1:2182/kafka", - "bootstrap.servers" : "0.0.0.1:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" - } - }, - "name" : "kafka10reader" - }, - "writer" : { - "parameter" : { - "print" : true + "content": [ + { + "reader": { + "parameter": { + "topic": "kafka10", + "groupId": "default", + "codec": "text", + "encoding": "UTF-8", + "blankIgnore": false, + "consumerSettings": { + "bootstrap.servers": "localhost:9092" + } + }, + "name": "kafka10reader" }, - "name" : "streamwriter" + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } } - } ], + ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, "restore": { - "maxRowNumForCheckpoint": 0, "isRestore": false, - "isStream" : true, - "restoreColumnName": "", - "restoreColumnIndex": 0 + "isStream": true }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed": { + "channel": 1 } } } } ``` - #### 3、kafka11 ```json { - "job": { - "content": [{ + "job" : { + "content" : [ { "reader" : { "parameter" : { "topic" : "kafka11", - "groupId" : "default", - "codec": "plain", + "groupId": "default", + "codec": "text", + "encoding": "UTF-8", "blankIgnore": false, - "consumerSettings" : { - "zookeeper.connect" : "0.0.0.1:2182/kafka", - "bootstrap.servers" : "0.0.0.1:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" + "consumerSettings": { + "bootstrap.servers": "localhost:9092" } }, "name" : "kafka11reader" @@ -221,48 +240,33 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "name" : "streamwriter" } } ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : true, - "restoreColumnName": "", - "restoreColumnIndex": 0 + "setting" : { + "restore" : { + "isRestore" : false, + "isStream" : true }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 4、kafka ```json { - "job": { - "content": [{ + "job" : { + "content" : [ { "reader" : { "parameter" : { - "topic" : "kafka", - "groupId" : "default", - "codec": "plain", + "topic" : "test", + "mode": "timestamp", + "timestamp": 1609812275000, + "offset": "partition:0,offset:0;partition:1,offset:1;partition:2,offset:2", + "codec": "text", "blankIgnore": false, "consumerSettings" : { - "zookeeper.connect" : "0.0.0.1:2182/kafka", - "bootstrap.servers" : "0.0.0.1:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "latest" + "bootstrap.servers" : "ip1:9092,ip2:9092,ip3:9092" } }, "name" : "kafkareader" @@ -275,122 +279,70 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 + "restore" : { + "isRestore" : false, + "isStream" : true }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : true, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed": { + "readerChannel": 3, + "writerChannel": 1 } } } } ``` - -#### 5、kafka->MySQL +#### 5、kafka->Hive ```json { - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "codec": "json", - "groupId" : "default", - "topic" : "tudou", - "consumerSettings" : { - "zookeeper.connect" : "kudu1:2181/kafka", - "bootstrap.servers" : "kudu1:9092", - "auto.commit.interval.ms" : "1000", - "auto.offset.reset" : "earliest" - }, - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "VARCHAR" + "job": { + "content": [ + { + "reader" : { + "parameter" : { + "topic" : "test", + "mode": "timestamp", + "timestamp": 1609812275000, + "codec": "text", + "consumerSettings" : { + "bootstrap.servers" : "ip1:9092,ip2:9092,ip3:9092" } - ] + }, + "name" : "kafkareader" }, - "name" : "kafkareader" - }, - "writer": { - "name": "mysqlwriter", - "parameter": { - "username": "dtstack", - "password": "abc123", - "batchSize": 1, - "connection": [ - { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": [ - "kudu" - ] - } - ], - "session": [], - "preSql": [], - "postSql": [], - "writeMode": "insert", - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "VARCHAR" + "writer": { + "parameter" : { + "jdbcUrl" : "jdbc:hive2://ip:10000/test", + "fileType" : "parquet", + "writeMode" : "overwrite", + "compress" : "", + "charsetName" : "UTF-8", + "maxFileSize" : 1073741824, + "tablesColumn" : "{\"message\":[{\"part\":false,\"comment\":\"\",\"type\":\"string\",\"key\":\"message\"}]}", + "partition" : "pt", + "partitionType" : "DAY", + "defaultFS" : "hdfs://ns", + "hadoopConfig": { + "dfs.ha.namenodes.ns": "nn1,nn2", + "dfs.namenode.rpc-address.ns.nn2": "ip1:9000", + "dfs.client.failover.proxy.provider.ns": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns.nn1": "ip2:9000", + "dfs.nameservices": "ns" } - ] + }, + "name" : "hivewriter" } } - } ], + ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : true, - "restoreColumnName": "", - "restoreColumnIndex": 0 + "isRestore": true, + "isStream": true }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed": { + "readerChannel": 3, + "writerChannel": 1 } } } } -``` -
+``` \ No newline at end of file diff --git a/docs/realTime/reader/pgwalreader.md b/docs/realTime/reader/pgwalreader.md index a30ba5f610..9390b88ec2 100644 --- a/docs/realTime/reader/pgwalreader.md +++ b/docs/realTime/reader/pgwalreader.md @@ -183,5 +183,220 @@ pavingData为false时: } } ``` + +## PostgreSQL实时采集原理 +PostgreSQL 实时采集是基于 PostgreSQL的逻辑复制以及逻辑解码功能来完成的。逻辑复制同步数据的原理是,在wal日志产生的数据库上,由逻辑解析模块对wal日志进行初步的解析,它的解析结果为ReorderBufferChange(可以简单理解为HeapTupleData),再由pgoutput plugin对中间结果进行过滤和消息化拼接后,然后将其发送到订阅端,订阅端通过逻辑解码功能进行解析。 + +### 版本限制 +逻辑复制是pgsql10.0版本之后才支持的,因此此方案只支持10.0之后版本 + + +### 主要涉及模块说明 +| Logical Decoding | PostgreSQL 的逻辑日志来源于解析物理 WAL 日志。
解析 WAL 成为逻辑数据的过程叫 Logical Decoding。 | +| :--- | :--- | +| Replication Slots | 保存逻辑或物理流复制的基础信息。类似 Mysql 的位点信息。
一个 逻辑 slot 创建后,它的相关信息可以通过 pg_replication_slots 系统视图获取。
如果它在 active 状态,则可以通过系统视图 pg_stat_replication 看到一些 slot 的实时的状态信息。 | +| Output Plugins | PostgreSQL 的逻辑流复制协议开放一组可编程接口,用于自定义输数据到客户端的逻辑数据的格式。
这部分实现使用插件的方式被内核集成和使用,称作 Output Plugins。 | +| Exported Snapshots | 当一个逻辑流复制 slot 被创建时,系统会产生一个快照。客户端可以通过它订阅到数据库任意时间点的数据变化。 | + + + +对于修改一条数据之后 ,pgsql订阅端decode解析后的数据格式为 +```json +{"id":"schema1.test1", + "schema":"schema1", +"table":"test1", + "columnList":[ + {"name":"id","type":"int4","index":0}, + {"name":"name","type":"varchar","index":1} + ], + "oldData":["2","23"], + "newData":["2","name1"], + "type":"UPDATE", + "currentLsn":23940928, + "ts":1596358573614 +} +``` +主要包含schema table以及类型`INSERT`, `UPDATE`和`DELETE`以及WAL日志id等相关信息
+
+ + +### 逻辑复制 +逻辑复制使用_发布_和_订阅_模型, 其中一个或多个_订阅者_订阅_发布者_ 节点上的一个或多个_发布_。 订阅者从他们订阅的发布中提取数据,逻辑复制是根据复制标识(通常是主键)复制数据对象及其更改的一种方法,因此在上面订阅端收到消息数据实例中可以发现 具备数据库以及表信息外 还具备修改前数据,修改后数据信息以及执行的type和对应的WAL日志ID + +发布可以选择将它们所产生的改变限制在`INSERT`, `UPDATE`和`DELETE`的任意组合上, 类似于触发器被特定事件类型触发。默认情况下,复制所有操作类型。
已发布的table必须配置一个“副本标识”以便能够复制 `UPDATE`和`DELETE`操作, 这样可以在订阅者端识别适当的行来更新或删除。默认情况下,这是主键, 如果有的话。另外唯一的索引(有一些额外的要求)也可以被设置为副本标识。 如果表没有任何合适的键,那么它可以设置为复制标识“full”, 这意味着整个行成为键。但是,这是非常低效的, 并且只能在没有其他可能的解决方案时用作后备
+ + +### 创建发布 +为哪些表设置创建一个发布 +```sql +CREATE PUBLICATION name + [ FOR TABLE [ ONLY ] table_name [ * ] [, ...] + | FOR ALL TABLES ] + [ WITH ( publication_parameter [= value] [, ... ] ) ] +``` + + + +### WAL日志 +WAL 是 Write Ahead Log的缩写,中文称之为预写式日志。WAL log也被简称为xlog,每一次change操作都是先写日志再写数据,保证了事务持久性和数据完整性同时又尽量地避免了频繁IO对性能的影响。WAL的中心概念是**数据文件(存储着表和索引)的修改必须在这些动作被日志记录之后才被写入**
WAL日志保存在pg_xlog下,每个xlog文件默认是16MB,为了满足恢复需求,在xlog目录下会产生多个WAL日志,不需要的WAL日志将会被覆盖
WAL具备归档功能,通过归档的WAL文件可以恢复数据库到WAL日志覆盖时间内的任意一个时间点的状态并且有了WAL日志之后,逻辑复制就可以在WAL日志生成之后,对其进行一系列操作之后传递给订阅客户端,使得订阅客户端能实时获取到源服务器上的修改数据
+ + +#### WAL何时被写入 +WAL也有个内存缓冲区WAL Buffer,WAL都是先写入缓存中,对于事务操作,缓存的WAL日志是在事务提交的时候写入磁盘的,对于非事务型的由一个异步线程追加进日志文件或者在checkPoint(数据脏页缓存写入磁盘需要先刷新WAL缓存)的时候写入。
+ + +#### WAL主要配置 +``` +wal_level 可以选择为minimal, replica, or logical 使用逻辑复制需要设置为logical + +fsync boolean类型 表示是否使用fsync()系统调用把WAL文件刷新到物理磁盘,确保数据库在操作系统或硬件奔溃的情况下可恢复到最终状态 默认是on + +synchronous_commit boolean类型 声明提交一个事务是否需要等待其把WAL日志写入磁盘后再返回,默认值是’on’ + +on:默认值,为on且没有开启同步备库的时候,会当wal日志真正刷新到磁盘永久存储后才会返回客户端事务已提交成功, + 当为on且开启了同步备库的时候(设置了synchronous_standby_names),必须要等事务日志刷新到本地磁盘,并且还要等远程备库也提交到磁盘才能返回客户端已经提交. + +remote_apply:提交将等待, 直到来自当前同步备用数据库的回复表明它们已收到事务的提交记录并应用它, 以便它对备用数据库上的查询可见。 + +remote_write:提交将等待,直到来自当前同步的后备服务器的一个回复指示该服务器已经收到了该事务的提交记录并且已经把该记录写出到后备服务器的操作系统。 + +local:当事务提交时,仅写入本地磁盘即可返回客户端事务提交成功,而不管是否有同步备库。 + +off:写到缓存中就会向客户端返回提交成功,但也不是一直不刷到磁盘,延迟写入磁盘,延迟的时间为最大3倍的wal_writer_delay参数的(默认200ms)的时间,所有如果即使关闭synchronous_commit,也只会造成最多600ms的事务丢失 可能会造成一些最近已提交的事务丢失,但数据库状态是一致的,就像这些事务已经被干净地中止。但对高并发的小事务系统来说,性能来说提升较大。 + + +wal_sync_method enum类型 用来指定向磁盘强制更新WAL日志数据的方法open_datasync fdatasync fsync_writethrough fsync open_sync + + + +Wal_writer_delay 指定wal writer process 把WAL日志写入磁盘的周期 在每个周期中会先把缓存中的WAL日志刷到磁盘 + +``` + + + +### 复制槽 +每个订阅都将通过一个复制槽接收更改,记录某个订阅者的WAL接收情况。
在源数据库写入修改频繁导致WAL日志的写入速度很快,导致大量WAL日志生成,或者订阅者接受日志很慢,在消费远远小于生产的时候,会导致源数据库上的WAL日志还没有传递到备库就被回卷覆盖掉了,如果被覆盖掉的WAL日志文件又没有归档备份,那么订阅者就再也无法消费到此数据。
复制槽则保存了此订阅的接收信息,使得未被接收的WAL日日志不会被回收 + +注意
数据库会记录slot的wal复制位点,并在wal文件夹中保留所有未发送的wal文件,如果客户创建了slot但是后期不再使用就有可能导致数据库的wal日志爆仓,需要及时删除不用的slot
+
可通过以下SQL获取相关信息 +```sql +select * from pg_replication_slots; +``` +字段含义 +```text +Name Type References Description +slot_name name 复制槽的唯一的集群范围标识符 +plugin name 正在使用的包含逻辑槽输出插件的共享对象的基本名称,对于物理插槽则为null。 +slot_type text 插槽类型 - 物理或逻辑 +datoid oid 该插槽所关联的数据库的OID,或为空。 只有逻辑插槽才具有关联的数据库。 +database text 该插槽所关联的数据库的名称,或为空。 只有逻辑插槽才具有关联的数据库。 +active boolean 如果此插槽当前正在使用,则为真 +active_pid integer 如果当前正在使用插槽,则使用此插槽的会话的进程ID。 NULL如果不活动。 +xmin xid 此插槽需要数据库保留的最早事务。 VACUUM无法删除任何后来的事务删除的元组。 +catalog_xmin xid 影响该插槽需要数据库保留的系统目录的最早的事务。 VACUUM不能删除任何后来的事务删除的目录元组。 +restart_lsn pg_lsn 最老的WAL的地址(LSN)仍然可能是该插槽的使用者所需要的,因此在检查点期间不会被自动移除 +``` + + + +### 局限性 + +- 不复制数据库模式和DDL命令。初始模式可以使用`pg_dump --schema-only` 手动复制。后续的模式更改需要手动保持同步。(但是请注意, 两端的架构不需要完全相同。)当实时数据库中的模式定义更改时,逻辑复制是健壮的: 当模式在发布者上发生更改并且复制的数据开始到达订阅者但不符合表模式, 复制将错误,直到模式更新。在很多情况下, 间歇性错误可以通过首先将附加模式更改应用于订阅者来避免。
+- 不复制序列数据。序列支持的序列或标识列中的数据当然会作为表的一部分被复制, 但序列本身仍然会显示订阅者的起始值。如果订阅者被用作只读数据库, 那么这通常不成问题。但是,如果打算对订阅者数据库进行某种切换或故障切换, 则需要将序列更新为最新值,方法是从发布者复制当前数据 (可能使用`pg_dump`)或者从表中确定足够高的值。
+- 不复制`TRUNCATE`命令。当然,可以通过使用`DELETE` 来解决。为了避免意外的`TRUNCATE`调用,可以撤销表的 `TRUNCATE`权限。
+- 不复制大对象 没有什么解决办法,除非在普通表中存储数据。 +- 复制只能从基表到基表。也就是说,发布和订阅端的表必须是普通表,而不是视图, 物化视图,分区根表或外部表。对于分区,您可以一对一地复制分区层次结构, 但目前不能复制到不同的分区设置。尝试复制基表以外的表将导致错误 + + + + +### PostgreSQL实时采集配置 + +#### postgresql.conf设置 +``` +wal_level = logical +``` + + +用于复制链接的角色必须具有`REPLICATION`属性(或者是超级用户) 需要在pg_hba.conf做出如下配置 +``` +host replication all 10.0.3.0/24 md5 +``` + + +### 部分核心代码分析 + + + +#### 执行发布SQL +逻辑复制流是发布/订阅模型,因此生成流之前 先进行发布 +```java +public static final String PUBLICATION_NAME = "dtstack_flinkx"; +public static final String CREATE_PUBLICATION = "CREATE PUBLICATION %s FOR ALL TABLES;"; +public static final String QUERY_PUBLICATION = "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s';"; + +先执行查找sql 判断是否存在 dtstack_flinkx 的 PUBLICATION +如果不存在 执行创建sql语句 +conn.createStatement() + .execute(String.format(CREATE_PUBLICATION, PUBLICATION_NAME)); +``` + + + +#### 创建一个逻辑复制流 +```java + ChainedLogicalStreamBuilder builder = conn.getReplicationAPI() + .replicationStream() //定义一个逻辑复制流 + .logical() //级别是logical + .withSlotName(format.getSlotName())//复制槽名称 + //协议版本。当前仅支持版本1 + .withSlotOption("proto_version", "1")//槽版本号 + //逗号分隔的要订阅的发布名称列表(接收更改)。 单个发布名称被视为标准对象名称,并可根据需要引用 + .withSlotOption("publication_names", PgWalUtil.PUBLICATION_NAME)//关联的发布名称 + .withStatusInterval(format.getStatusInterval(), TimeUnit.MILLISECONDS); + long lsn = format.getStartLsn(); + if(lsn != 0){ + builder.withStartPosition(LogSequenceNumber.valueOf(lsn)); + } + stream = builder.start(); +``` + +#### 业务处理 +逻辑复制流接收到订阅的消息后 进行编码 获取到相应信息处理 +```java + public void run() { + LOG.info("PgWalListener start running....."); + try { + init(); + while (format.isRunning()) { + //接收到流对象 + ByteBuffer buffer = stream.readPending(); + if (buffer == null) { + continue; + } + //解码为table对象 具体信息为库 表 字段信息 WAL id等 + //然后就可以对其进行处理了 + Table table = decoder.decode(buffer); + if(StringUtils.isBlank(table.getId())){ + continue; + } + String type = table.getType().name().toLowerCase(); + if(!cat.contains(type)){ + continue; + } + if(!tableSet.contains(table.getId())){ + continue; + } + LOG.trace("table = {}",gson.toJson(table)); + ............... + } + } + } +``` + +
+ + diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md index fc82240795..453e011411 100644 --- a/docs/realTime/writer/kafkawriter.md +++ b/docs/realTime/writer/kafkawriter.md @@ -1,6 +1,5 @@ # Kafka Writer - ## 一、插件名称 kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: @@ -9,63 +8,66 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.9 | kafka09writer | | kafka 0.10 | kafka10writer | | kafka 0.11 | kafka11writer | -| kafka 1.0及以上 | kafkawriter | +| kafka 1.0及以后 | kafkawriter | - -## 二、参数说明
- -- **timezone** - - 描述:时区 - - 必选:否 - - 默认值:无 - - +## 二、参数说明 - **topic** - - 描述:topic - - 必选:是 - - 默认值:无 + - 描述:消息发送至kafka的topic名称,不支持多个topic + - 必选:是 + - 字段类型:String + - 默认值:无 +
+- **timezone** + - 描述:时区 + - 必选:否 + - 字段类型:String + - 默认值:无 -- **encoding** - - 描述:编码 - - 注意:该参数只对kafka09reader插件有效 - - 必选:否 - - 默认值:UTF-8 +
+- **encoding** + - 描述:编码 + - 注意:该参数只对kafka09reader插件有效 + - 必选:否 + - 字段类型:String + - 默认值:UTF-8 +
- **brokerList** - - 描述:kafka broker地址列表 - - 注意:该参数只对kafka09writer插件有效 - - 必选:是 - - 默认值:无 - + - 描述:kafka broker地址列表 + - 注意:该参数只对kafka09writer插件有效 + - 必选:kafka09writer必选,其它kafka writer插件不用填 + - 字段类型:String + - 默认值:无 +
- **producerSettings** - - 描述:kafka连接配置,支持所有`org.apache.kafka.clients.producer.ProducerConfig`中定义的配置 - - 必选:是 - - 默认值:无 - + - 描述:kafka连接配置,支持所有`org.apache.kafka.clients.producer.ProducerConfig`中定义的配置 + - 必选:对于非kafka09 writer插件,该参数必填,且producerSettings中至少包含`bootstrap.servers`参数 + - 字段类型:Map + - 默认值:无 +
- **tableFields** - - 描述:字段映射配置。从reader插件传递到writer插件的的数据只包含其value属性,配置该参数后可将其还原成键值对类型json字符串输出。 - - 注意: - - 若配置该属性,则该配置中的字段个数必须不少于reader插件中读取的字段个数,否则该配置失效; - - 映射关系按该配置中字段的先后顺序依次匹配; - - 必选:否 - - 默认值:无 + - 描述:字段映射配置。从reader插件传递到writer插件的的数据只包含其value属性,配置该参数后可将其还原成键值对类型json字符串输出。 + - 注意: + - 若配置该属性,则该配置中的字段个数必须不少于reader插件中读取的字段个数,否则该配置失效; + - 映射关系按该配置中字段的先后顺序依次匹配; + - 必选:否 + - 字段类型:String[] + - 默认值:无 - -## 二、配置示例 - +## 三、配置示例 #### 1、kafka09 ```json { @@ -95,43 +97,24 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "parameter": { "timezone": "UTC", "topic": "kafka09", - "encoding": "UTF_8", + "encoding": "UTF-8", "brokerList": "0.0.0.1:9092", - "producerSettings": { - "zookeeper.connect" : "0.0.0.1:2182", - "bootstrap.servers" : "0.0.0.1:9092" - }, "tableFields": ["id","user_id","name"] }, "name": "kafka09writer" } } ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 + "restore" : { + "isStream" : true }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 2、kafka10 ```json { @@ -162,7 +145,6 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "timezone": "UTC", "topic": "kafka10", "producerSettings": { - "zookeeper.connect" : "0.0.0.1:2182", "bootstrap.servers" : "0.0.0.1:9092" }, "tableFields": ["id","user_id","name"] @@ -171,31 +153,16 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : false, - "restoreColumnName": "", - "restoreColumnIndex": 0 + "restore" : { + "isStream" : true }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 3、kafka11 ```json { @@ -235,31 +202,16 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 + "restore" : { + "isStream" : true }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 4、kafka ```json { @@ -290,7 +242,6 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "timezone": "UTC", "topic": "kafka", "producerSettings": { - "zookeeper.connect" : "0.0.0.1:2182", "bootstrap.servers" : "0.0.0.1:9092" }, "tableFields": ["id","user_id","name"] @@ -299,31 +250,16 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ], "setting": { - "speed": { - "channel": 1, - "bytes": 0 + "restore" : { + "isStream" : true }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } } ``` - #### 5、MySQL->kafka ```json { @@ -333,15 +269,15 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "name": "mysqlreader", "parameter": { "column": ["id","user_id","name"], - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { "jdbcUrl": [ - "jdbc:mysql://kudu3:3306/tudou" + "jdbc:mysql://0.0.0.1:3306/test" ], "table": [ - "kudu" + "test" ] } ] @@ -351,34 +287,19 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 "parameter" : { "tableFields" : ["id","user_id","name"], "producerSettings" : { - "zookeeper.connect" : "kudu1:2182/kafka", - "bootstrap.servers" : "kudu1:9092" + "bootstrap.servers" : "0.0.0.1:9092" }, - "topic" : "tudou" + "topic" : "kafka" }, "name" : "kafkawriter" } } ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - }, - "errorLimit": { - "record": 100 - }, - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "isStream" : false, - "restoreColumnName": "", - "restoreColumnIndex": 0 + "setting": { + "restore" : { + "isStream" : true }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "speed" : { + "channel" : 1 } } } diff --git a/docs/restore.md b/docs/restore.md index 792a19ad99..fc2c607197 100644 --- a/docs/restore.md +++ b/docs/restore.md @@ -91,7 +91,7 @@ where id mod 2=1; 3. 检测 /data_test/.data 目录是否存在,如果存在就先删除,再创建,确保没有其它任务因异常失败遗留的脏数据文件; - 数据写入hdfs是单条写入的,不支持批量写入。数据会先写入/data_test/.data/目录下,数据文件的命名格式为: channelIndex.jobId.fileIndex,包含通道索引,jobId,文件索引三个部分,文件最掐灭。 + 数据写入hdfs是单条写入的,不支持批量写入。数据会先写入/data_test/.data/目录下,数据文件的命名格式为: channelIndex.jobId.fileIndex,包含通道索引,jobId,文件索引三个部分。 ##### **3**)checkpoint触发时 diff --git a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java new file mode 100644 index 0000000000..c587b22fa4 --- /dev/null +++ b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.binlog; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +/** + * Date: 2019/12/03 + * Company: www.dtstack.com + * + * @author tudou + */ +public class BinlogUtil { + private static final Logger LOG = LoggerFactory.getLogger(BinlogUtil.class); + + public static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; + //是否开启binlog + private static final String CHECK_BINLOG_ENABLE = "show variables where variable_name = 'log_bin';;"; + //查看binlog format + private static final String CHECK_BINLOG_FORMAT = "show variables where variable_name = 'binlog_format';;"; + //校验用户是否有权限 + private static final String CHECK_USER_PRIVILEGE = "show master status ;"; + + + private static final String AUTHORITY_TEMPLATE = "SELECT count(1) FROM %s LIMIT 1"; + + private static final String QUERY_SCHEMA_TABLE_TEMPLATE = "SELECT TABLE_NAME From information_schema.TABLES WHERE TABLE_SCHEMA='%s' LIMIT 1"; + + + public static final int RETRY_TIMES = 3; + + public static final int SLEEP_TIME = 2000; + + + /** + * 校验是否开启binlog + * + * @param conn + * @return + * @throws SQLException + */ + public static boolean checkEnabledBinlog(Connection conn) throws SQLException { + try (Statement statement = conn.createStatement()) { + try (ResultSet rs = statement.executeQuery(CHECK_BINLOG_ENABLE)) { + if (rs.next()) { + String binLog = rs.getString("Value"); + if (StringUtils.isNotBlank(binLog)) { + return "ON".equalsIgnoreCase(binLog); + } + } + return false; + } + } catch (SQLException e) { + LOG.error("error to query BINLOG is enabled , sql = {}, e = {}", CHECK_BINLOG_ENABLE, ExceptionUtil.getErrorMessage(e)); + throw e; + } + } + + /** + * 校验binlog的format格式 + * + * @param conn + * @return + * @throws SQLException + */ + public static boolean checkBinlogFormat(Connection conn) throws SQLException { + try (Statement statement = conn.createStatement()) { + try (ResultSet rs = statement.executeQuery(CHECK_BINLOG_FORMAT)) { + if (rs.next()) { + String logFormat = rs.getString("Value"); + if (StringUtils.isNotBlank(logFormat)) { + return "row".equalsIgnoreCase(logFormat); + } + } + return false; + } + } catch (SQLException e) { + LOG.error("error to query binLog format, sql = {}, e = {}", CHECK_BINLOG_FORMAT, ExceptionUtil.getErrorMessage(e)); + throw e; + } + } + + /** + * 效验用户的权限 + * + * @param conn + * @return + */ + public static boolean checkUserPrivilege(Connection conn) { + try (Statement statement = conn.createStatement()) { + statement.execute(CHECK_USER_PRIVILEGE); + } catch (SQLException e) { + LOG.error("'show master status' has an error!,please check. you need (at least one of) the SUPER,REPLICATION CLIENT privilege(s) for this operation, e = {}", ExceptionUtil.getErrorMessage(e)); + return false; + } + return true; + } + + public static List checkTablesPrivilege(Connection connection, String database, String filter, List tables) throws SQLException { + if (CollectionUtils.isNotEmpty(tables)) { + HashMap checkedTable = new HashMap<>(tables.size()); + //按照.切割字符串需要转义 + String regexSchemaSplit = "\\" + ConstantValue.POINT_SYMBOL; + tables.stream() + //每一个表格式化为schema.tableName格式 + .map(t -> formatTableName(database, t)) + //只需要每个schema下的一个表进行判断 + .forEach(t -> checkedTable.putIfAbsent(t.split(regexSchemaSplit)[0], t)); + + //检验每个schema下的第一个表的权限 + return checkSourceAuthority(connection, null, checkedTable.values()); + } else if (StringUtils.isBlank(filter)) { + //检验schema下任意一张表的权限 + return checkSourceAuthority(connection, database, null); + } + return null; + } + + /** + * @param schema 需要校验权限的schemaName + * @param tables 需要校验权限的tableName + * schemaName权限验证 取schemaName下第一个表进行验证判断整个schemaName下是否具有权限 + */ + public static List checkSourceAuthority(Connection connection, String schema, Collection tables) throws SQLException { + try (Statement statement = connection.createStatement()) { + //Schema不为空且用户没有指定tables 就获取一张表判断权限 + if (StringUtils.isNotBlank(schema) && CollectionUtils.isEmpty(tables)) { + try (ResultSet resultSet = statement.executeQuery(String.format(QUERY_SCHEMA_TABLE_TEMPLATE, schema))) { + if (resultSet.next()) { + String tableName = resultSet.getString(1); + if (StringUtils.isNotBlank(tableName)) { + tables = Collections.singletonList(formatTableName(schema, tableName)); + } + } + } + } + if (CollectionUtils.isEmpty(tables)) { + return null; + } + + List failedTables = new ArrayList<>(tables.size()); + for (String tableName : tables) { + try { + //判断用户是否具备tableName下的读权限 + statement.executeQuery(String.format(AUTHORITY_TEMPLATE, tableName)); + } catch (SQLException e) { + failedTables.add(tableName); + } + } + + return failedTables; + } catch (SQLException sqlException) { + LOG.error("error to check table select privilege error, sql = {}, e = {}", AUTHORITY_TEMPLATE, ExceptionUtil.getErrorMessage(sqlException)); + throw sqlException; + } + } + + + public static String getDataBaseByUrl(String jdbcUrl) { + int idx = jdbcUrl.lastIndexOf('?'); + + if (idx != -1) { + return StringUtils.substring(jdbcUrl, jdbcUrl.lastIndexOf('/') + 1, idx); + } else { + return StringUtils.substring(jdbcUrl, jdbcUrl.lastIndexOf('/') + 1); + } + } + + private static String formatTableName(String schemaName, String tableName) { + StringBuilder stringBuilder = new StringBuilder(); + if (tableName.contains(ConstantValue.POINT_SYMBOL)) { + return tableName; + } else { + return stringBuilder.append(schemaName).append(ConstantValue.POINT_SYMBOL).append(tableName).toString(); + } + } +} diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormat.java similarity index 63% rename from flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java rename to flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormat.java index 6d5464096d..a5e2b5f0b2 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogInputFormat.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormat.java @@ -15,19 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.binlog.reader; +package com.dtstack.flinkx.binlog.format; import com.alibaba.otter.canal.filter.aviater.AviaterRegexFilter; import com.alibaba.otter.canal.parse.inbound.mysql.MysqlEventParser; import com.alibaba.otter.canal.parse.support.AuthenticationInfo; import com.alibaba.otter.canal.protocol.position.EntryPosition; import com.dtstack.flinkx.binlog.BinlogJournalValidator; +import com.dtstack.flinkx.binlog.BinlogUtil; +import com.dtstack.flinkx.binlog.listener.BinlogAlarmHandler; +import com.dtstack.flinkx.binlog.reader.BinlogConfig; +import com.dtstack.flinkx.binlog.listener.BinlogEventSink; +import com.dtstack.flinkx.binlog.listener.BinlogPositionManager; +import com.dtstack.flinkx.binlog.listener.HeartBeatController; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.RetryUtil; -import com.google.common.base.Joiner; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -40,18 +43,9 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.nio.charset.Charset; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Objects; import java.util.concurrent.locks.LockSupport; import java.util.stream.Collectors; @@ -72,15 +66,6 @@ public class BinlogInputFormat extends BaseRichInputFormat { private final String SCHEMA_SPLIT = "."; - private final String AUTHORITY_REPLICATION_TEMPLATE = "show master status"; - - private final String AUTHORITY_TEMPLATE = "SELECT count(1) FROM %s LIMIT 1"; - - private final String QUERY_SCHEMA_TABLE_TEMPLATE = "SELECT TABLE_NAME From information_schema.TABLES WHERE TABLE_SCHEMA='%s' LIMIT 1"; - - private static final int RETRY_TIMES = 3; - - private static final int SLEEP_TIME = 2000; /** * internal fields */ @@ -122,34 +107,18 @@ public void openInputFormat() throws IOException { List tables = binlogConfig.getTable(); String jdbcUrl = binlogConfig.getJdbcUrl(); if (jdbcUrl != null) { - int idx = jdbcUrl.lastIndexOf('?'); - String database; - if (idx != -1) { - database = StringUtils.substring(jdbcUrl, jdbcUrl.lastIndexOf('/') + 1, idx); - } else { - database = StringUtils.substring(jdbcUrl, jdbcUrl.lastIndexOf('/') + 1); - } + String database = BinlogUtil.getDataBaseByUrl(jdbcUrl); if (CollectionUtils.isNotEmpty(tables)) { - HashMap checkedTable = new HashMap<>(tables.size()); - //按照.切割字符串需要转义 - String regexSchemaSplit = "\\" + SCHEMA_SPLIT; String filter = tables.stream() //每一个表格式化为schema.tableName格式 .map(t -> formatTableName(database, t)) - //只需要每个schema下的一个表进行判断 - .peek(t -> checkedTable.putIfAbsent(t.split(regexSchemaSplit)[0], t)) .collect(Collectors.joining(",")); binlogConfig.setFilter(filter); - - //检验每个schema下的第一个表的权限 - checkSourceAuthority(null, checkedTable.values()); } else if (StringUtils.isBlank(binlogConfig.getFilter())) { //如果table未指定 filter未指定 只消费此schema下的数据 binlogConfig.setFilter(database + "\\..*"); - //检验schema下任意一张表的权限 - checkSourceAuthority(database, null); } LOG.info("binlog FilterAfter:{},tableAfter: {}",binlogConfig.getFilter(),binlogConfig.getTable()); } @@ -302,89 +271,4 @@ private String formatTableName(String schemaName, String tableName) { return stringBuilder.append(schemaName).append(SCHEMA_SPLIT).append(tableName).toString(); } } - - /** - * @param schema 需要校验权限的schemaName - * @param tables 需要校验权限的tableName - * schemaName权限验证 取schemaName下第一个表进行验证判断整个schemaName下是否具有权限 - */ - private void checkSourceAuthority(String schema, Collection tables) { - Connection connection = null; - try { - connection = RetryUtil.executeWithRetry(() -> DriverManager.getConnection(binlogConfig.getJdbcUrl(), binlogConfig.getUsername(), binlogConfig.getPassword()), RETRY_TIMES, SLEEP_TIME, false); - } catch (Exception e) { - try { - connection.close(); - } catch (Exception exception) { - String message = String.format(" closed connection error,params jdbcUrl【%s】user 【%s】, errorMessage %s", - binlogConfig.getJdbcUrl(), - binlogConfig.getUsername(), - ExceptionUtil.getErrorMessage(e)); - LOG.error(message); - } - - String message = String.format(" get connection failed,params jdbcUrl【%s】user 【%s】 make sure that the database configuration is correct , errorMessage %s", - binlogConfig.getJdbcUrl(), - binlogConfig.getUsername(), - ExceptionUtil.getErrorMessage(e)); - LOG.error("{}", message); - throw new RuntimeException(message, e); - } - - try (Statement statement = connection.createStatement()) { - - //判断用户是否具有REPLICATION权限 没有的话会直接抛出异常MySQLSyntaxErrorException - statement.execute((AUTHORITY_REPLICATION_TEMPLATE)); - //Schema不为空 就获取一张表判断权限 - if (StringUtils.isNotBlank(schema)) { - try (ResultSet resultSet = statement.executeQuery(String.format(QUERY_SCHEMA_TABLE_TEMPLATE, schema))) { - if (resultSet.next()) { - String tableName = resultSet.getString(1); - if (CollectionUtils.isNotEmpty(tables)) { - tables.add(formatTableName(schema, tableName)); - } else { - tables = Collections.singletonList(formatTableName(schema, tableName)); - } - } - } - } - if (CollectionUtils.isEmpty(tables)) { - return; - } - - List failedTables = new ArrayList<>(tables.size()); - SQLException sqlException = null; - for (String tableName : tables) { - try { - //判断用户是否具备tableName下的读权限 - statement.executeQuery(String.format(AUTHORITY_TEMPLATE, tableName)); - } catch (SQLException e) { - failedTables.add(tableName); - if (Objects.isNull(sqlException)) { - sqlException = e; - } - } - } - - if (CollectionUtils.isNotEmpty(failedTables)) { - String message = String.format("user【%s】is not granted table 【%s】select permission %s", - binlogConfig.getUsername(), - Joiner.on(",").join(failedTables), - ExceptionUtil.getErrorMessage(sqlException)); - - LOG.error("{}", message); - throw new RuntimeException(message, sqlException); - } - - } catch (SQLException sqlException) { - String message = String.format("params jdbcUrl【%s】user 【%s】 create statement failed or user user 【%s】 has no REPLICATION permission errorMessage %s", - binlogConfig.getJdbcUrl(), - binlogConfig.getUsername(), - binlogConfig.getUsername(), - ExceptionUtil.getErrorMessage(sqlException)); - - LOG.error("{}", message); - throw new RuntimeException(message, sqlException); - } - } } diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java new file mode 100644 index 0000000000..2e59ef17f9 --- /dev/null +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.binlog.format; + +import com.dtstack.flinkx.binlog.BinlogUtil; +import com.dtstack.flinkx.binlog.reader.BinlogConfig; +import com.dtstack.flinkx.config.SpeedConfig; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RetryUtil; +import com.dtstack.flinkx.util.TelnetUtil; +import com.dtstack.flinkx.util.ValueUtil; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang.StringUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; + +import static com.dtstack.flinkx.binlog.BinlogUtil.DRIVER_NAME; + +/** + * Date: 2020/12/16 + * Company: www.dtstack.com + * + * @author dujie + */ +public class BinlogInputFormatBuilder extends BaseRichInputFormatBuilder { + + protected BinlogInputFormat format; + + public BinlogInputFormatBuilder() { + super.format = this.format = new BinlogInputFormat(); + } + + public void setBinlogConfig(BinlogConfig binlogConfig) { + this.format.setBinlogConfig(binlogConfig); + } + + @Override + protected void checkFormat() { + StringBuilder sb = new StringBuilder(256); + BinlogConfig binlogConfig = format.getBinlogConfig(); + if (StringUtils.isBlank(binlogConfig.username)) { + sb.append("No username supplied;\n"); + } + + if (StringUtils.isBlank(binlogConfig.jdbcUrl)) { + sb.append("No url supplied;\n"); + } else { + //检测数据源连通性 + TelnetUtil.telnet(binlogConfig.jdbcUrl); + } + + if (StringUtils.isBlank(binlogConfig.host)) { + sb.append("No host supplied;\n"); + } + + if (StringUtils.isBlank(binlogConfig.cat)) { + sb.append("No cat supplied;\n"); + } + + if (sb.length() > 0) { + throw new IllegalArgumentException(sb.toString()); + } + + SpeedConfig speed = format.getDataTransferConfig().getJob().getSetting().getSpeed(); + + if (speed.getReaderChannel() > 1) { + sb.append("binLog can not support readerChannel bigger than 1, current readerChannel is [") + .append(speed.getReaderChannel()) + .append("];\n"); + } else if (speed.getChannel() > 1) { + sb.append("binLog can not support channel bigger than 1, current channel is [") + .append(speed.getChannel()) + .append("];\n"); + } + + //校验binlog cat + if (StringUtils.isNotEmpty(binlogConfig.getCat())) { + HashSet set = Sets.newHashSet("INSERT", "UPDATE", "DELETE"); + List cats = Lists.newArrayList(binlogConfig.getCat().toUpperCase().split(",")); + cats.removeIf(s -> set.contains(s.toUpperCase(Locale.ENGLISH))); + if (CollectionUtils.isNotEmpty(cats)) { + sb.append("binlog cat not support-> ") + .append(GsonUtil.GSON.toJson(cats)) + .append(",just support->") + .append(GsonUtil.GSON.toJson(set)) + .append(";\n"); + } + } + + //校验binlog的start参数 + if (MapUtils.isNotEmpty(binlogConfig.getStart())) { + try { + MapUtils.getLong(binlogConfig.getStart(),"timestamp"); + } catch (Exception e) { + sb.append("binlog start parameter of timestamp must be long type, but your value is -> ").append(binlogConfig.getStart().get("timestamp")).append(";\n"); + } + try { + MapUtils.getLong(binlogConfig.getStart(),"position"); + } catch (Exception e) { + sb.append("binlog start parameter of position must be long type, but your value is -> ").append(binlogConfig.getStart().get("timestamp")).append(";\n"); + } + } + + ClassUtil.forName(DRIVER_NAME, getClass().getClassLoader()); + try (Connection conn = RetryUtil.executeWithRetry(() -> DriverManager.getConnection(binlogConfig.getJdbcUrl(), binlogConfig.getUsername(), binlogConfig.getPassword()), BinlogUtil.RETRY_TIMES, BinlogUtil.SLEEP_TIME, false)) { + + //校验用户权限 + if (!BinlogUtil.checkUserPrivilege(conn)) { + sb.append("\nyou need (at least one of) the SUPER,REPLICATION CLIENT privilege(s) for this operation; you can execute sql ->") + .append("GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO '") + .append(binlogConfig.getUsername()) + .append("'@'%' IDENTIFIED BY 'password';\n\n"); + } + + //校验数据库是否开启binlog + if (!BinlogUtil.checkEnabledBinlog(conn)) { + sb.append("binlog has not enabled, please click my.cnf Add the following to the file: \n") + .append("server_id=109\n") + .append("log_bin = /var/lib/mysql/mysql-bin\n") + .append("binlog_format = ROW\n") + .append("expire_logs_days = 30\n\n"); + } + + //校验数据库binlog_format是否设置为row + if (!BinlogUtil.checkBinlogFormat(conn)) { + sb.append(" binlog_format must be set ROW ;\n"); + } + + //校验用户表是否有select权限 + String database = BinlogUtil.getDataBaseByUrl(binlogConfig.jdbcUrl); + List failedTable = BinlogUtil.checkTablesPrivilege(conn, database, binlogConfig.filter, binlogConfig.table); + if (CollectionUtils.isNotEmpty(failedTable)) { + sb.append("user has not select privilege on " + GsonUtil.GSON.toJson(failedTable)); + } + + if (sb.length() > 0) { + throw new IllegalArgumentException(sb.toString()); + } + + } catch (Exception e) { + throw new RuntimeException("error to check binlog config, e = " + ExceptionUtil.getErrorMessage(e), e); + } + } +} diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogAlarmHandler.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogAlarmHandler.java similarity index 94% rename from flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogAlarmHandler.java rename to flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogAlarmHandler.java index 81d23427f4..7a1b6252fb 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogAlarmHandler.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogAlarmHandler.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.binlog.reader; +package com.dtstack.flinkx.binlog.listener; import com.alibaba.otter.canal.common.AbstractCanalLifeCycle; import com.alibaba.otter.canal.common.alarm.CanalAlarmHandler; +import com.dtstack.flinkx.binlog.format.BinlogInputFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogEventSink.java similarity index 88% rename from flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java rename to flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogEventSink.java index ccf7149ecc..0dbbc4ae15 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogEventSink.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogEventSink.java @@ -15,15 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.binlog.reader; +package com.dtstack.flinkx.binlog.listener; import com.alibaba.otter.canal.common.AbstractCanalLifeCycle; import com.alibaba.otter.canal.protocol.CanalEntry; import com.alibaba.otter.canal.sink.exception.CanalSinkException; -import com.dtstack.flinkx.util.SnowflakeIdWorker; +import com.dtstack.flinkx.binlog.format.BinlogInputFormat; import com.dtstack.flinkx.log.DtLogger; import com.dtstack.flinkx.util.ExceptionUtil; -import com.google.gson.Gson; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.SnowflakeIdWorker; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,10 +113,14 @@ private void processRowChange(CanalEntry.RowChange rowChange, String schema, Str if (pavingData){ for (CanalEntry.Column column : rowData.getAfterColumnsList()) { - message.put("after_" + column.getName(), column.getValue()); + if(!column.getIsNull()){ + message.put("after_" + column.getName(), column.getValue()); + } } for (CanalEntry.Column column : rowData.getBeforeColumnsList()){ - message.put("before_" + column.getName(), column.getValue()); + if(!column.getIsNull()){ + message.put("before_" + column.getName(), column.getValue()); + } } } else { message.put("before", processColumnList(rowData.getBeforeColumnsList())); @@ -129,8 +134,7 @@ private void processRowChange(CanalEntry.RowChange rowChange, String schema, Str LOG.error("takeEvent interrupted message:{} error:{}", message, e); } if(DtLogger.isEnableTrace()){ - //log level is trace, so don't care the performance,just new it. - LOG.trace("message = {}", new Gson().toJson(message)); + LOG.trace("message = {}", GsonUtil.GSON.toJson(message)); } } @@ -139,7 +143,9 @@ private void processRowChange(CanalEntry.RowChange rowChange, String schema, Str private Map processColumnList(List columnList) { Map map = new HashMap<>(columnList.size()); for (CanalEntry.Column column : columnList) { - map.put(column.getName(), column.getValue()); + if(!column.getIsNull()){ + map.put(column.getName(), column.getValue()); + } } return map; } @@ -152,7 +158,7 @@ public Row takeEvent() throws IOException { Row row = null; try { Map map = queue.take(); - //@see com.dtstack.flinkx.binlog.reader.HeartBeatController.onFailed 检测到异常之后 会添加key为e的错误数据 + //@see com.dtstack.flinkx.binlog.listener.HeartBeatController.onFailed 检测到异常之后 会添加key为e的错误数据 if(map.size() == 1 && map.containsKey("e")){ throw new RuntimeException((String) map.get("e")); }else{ diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogPositionManager.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogPositionManager.java similarity index 95% rename from flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogPositionManager.java rename to flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogPositionManager.java index 0edfd64e0e..8a354047f2 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogPositionManager.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/BinlogPositionManager.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.binlog.reader; +package com.dtstack.flinkx.binlog.listener; import com.alibaba.otter.canal.parse.exception.CanalParseException; import com.alibaba.otter.canal.parse.index.AbstractLogPositionManager; import com.alibaba.otter.canal.protocol.position.LogPosition; +import com.dtstack.flinkx.binlog.format.BinlogInputFormat; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import org.slf4j.Logger; diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/HeartBeatController.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/HeartBeatController.java similarity index 96% rename from flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/HeartBeatController.java rename to flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/HeartBeatController.java index a871265f0b..77fc487801 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/HeartBeatController.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/listener/HeartBeatController.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.binlog.reader; +package com.dtstack.flinkx.binlog.listener; import com.alibaba.otter.canal.common.AbstractCanalLifeCycle; import com.alibaba.otter.canal.parse.ha.CanalHAController; import com.alibaba.otter.canal.parse.ha.HeartBeatHAController; import com.alibaba.otter.canal.parse.inbound.HeartBeatCallback; +import com.dtstack.flinkx.binlog.listener.BinlogEventSink; import com.dtstack.flinkx.util.ExceptionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java index 471206a673..2f35e70389 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/reader/BinlogReader.java @@ -17,6 +17,7 @@ */ package com.dtstack.flinkx.binlog.reader; +import com.dtstack.flinkx.binlog.format.BinlogInputFormatBuilder; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.reader.BaseDataReader; @@ -47,13 +48,14 @@ public BinlogReader(DataTransferConfig config, StreamExecutionEnvironment env) { @Override public DataStream readData() { - BinlogInputFormat format = new BinlogInputFormat(); - format.setDataTransferConfig(dataTransferConfig); - format.setBinlogConfig(binlogConfig); - format.setRestoreConfig(restoreConfig); - format.setLogConfig(logConfig); - format.setTestConfig(testConfig); - return createInput(format); + + BinlogInputFormatBuilder builder = new BinlogInputFormatBuilder(); + builder.setDataTransferConfig(dataTransferConfig); + builder.setBinlogConfig(binlogConfig); + builder.setRestoreConfig(restoreConfig); + builder.setLogConfig(logConfig); + builder.setTestConfig(testConfig); + return createInput(builder.finish()); } } diff --git a/flinkx-binlog/pom.xml b/flinkx-binlog/pom.xml index fd6290739d..85753848ab 100644 --- a/flinkx-binlog/pom.xml +++ b/flinkx-binlog/pom.xml @@ -23,6 +23,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + diff --git a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DictionaryWriterTask.java b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DictionaryWriterTask.java index 58840f1da8..ab6fd9dbab 100644 --- a/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DictionaryWriterTask.java +++ b/flinkx-carbondata/flinkx-carbondata-writer/src/main/java/com/dtstack/flinkx/carbondata/writer/dict/DictionaryWriterTask.java @@ -20,6 +20,7 @@ package com.dtstack.flinkx.carbondata.writer.dict; +import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.carbondata.core.cache.dictionary.Dictionary; import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier; import org.apache.carbondata.core.constants.CarbonCommonConstants; @@ -28,6 +29,9 @@ import org.apache.carbondata.core.service.DictionaryService; import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.core.writer.CarbonDictionaryWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -43,6 +47,8 @@ */ public class DictionaryWriterTask { + private static final Logger LOG = LoggerFactory.getLogger(DictionaryWriterTask.class); + private Set valuesBuffer; private Dictionary dictionary; @@ -99,7 +105,11 @@ public List execute() throws IOException { } } finally { if (null != writer) { - writer.close(); + try { + writer.close(); + }catch (IOException e){ + LOG.error(ExceptionUtil.getErrorMessage(e)); + } } } return distinctValues; diff --git a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java index c6103d9dfa..baf5bcb3e7 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java +++ b/flinkx-clickhouse/flinkx-clickhouse-core/src/main/java/com/dtstack/flinkx/clickhouse/core/ClickhouseUtil.java @@ -37,8 +37,8 @@ public class ClickhouseUtil { public static Connection getConnection(String url, String username, String password) throws SQLException { Properties properties = new Properties(); - properties.put(ClickHouseQueryParam.USER, username); - properties.put(ClickHouseQueryParam.PASSWORD, password); + properties.put(ClickHouseQueryParam.USER.getKey(), username); + properties.put(ClickHouseQueryParam.PASSWORD.getKey(), password); boolean failed = true; Connection conn = null; for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml index a77eeddeec..8f1e494eb8 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/pom.xml @@ -88,6 +88,7 @@ + diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java index f394b3cb8c..ec01fc9cd6 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/format/ClickhouseInputFormat.java @@ -25,10 +25,6 @@ import java.io.IOException; import java.sql.Connection; import java.sql.SQLException; -import java.util.ArrayList; -import java.sql.Statement; - -import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; /** * Date: 2019/11/05 diff --git a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/reader/ClickhouseReader.java b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/reader/ClickhouseReader.java index 215ed022c7..51cf3821d4 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/reader/ClickhouseReader.java +++ b/flinkx-clickhouse/flinkx-clickhouse-reader/src/main/java/com/dtstack/flinkx/clickhouse/reader/ClickhouseReader.java @@ -23,7 +23,6 @@ import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - /** * Date: 2019/11/05 * Company: www.dtstack.com diff --git a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml index ddb84e6148..8427d72b1b 100644 --- a/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml +++ b/flinkx-clickhouse/flinkx-clickhouse-writer/pom.xml @@ -88,6 +88,7 @@ + diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index dae86d7427..fd42d2ca17 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -26,8 +26,8 @@ org.slf4j - slf4j-api - 1.7.20 + slf4j-log4j12 + 1.7.10 @@ -135,6 +135,13 @@ simpleclient 0.5.0 + + + org.apache.curator + curator-test + 2.6.0 + test + diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index 26254f836a..bb67c3ddec 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -18,7 +18,7 @@ package com.dtstack.flinkx; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; -import com.dtstack.flinkx.classloader.ClassLoaderManager; +import com.dtstack.flinkx.classloader.PluginUtil; import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.RestartConfig; @@ -37,7 +37,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; -import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; @@ -51,12 +50,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URL; import java.net.URLDecoder; -import java.util.ArrayList; import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.concurrent.TimeUnit; /** @@ -74,9 +70,7 @@ public class Main { public static final String STREAM_READER = "streamreader"; public static final String STREAM_WRITER = "streamwriter"; - private static final String CLASS_FILE_NAME_FMT = "class_path_%d"; - - private static ObjectMapper objectMapper = new ObjectMapper(); + private static final ObjectMapper objectMapper = new ObjectMapper(); public static void main(String[] args) throws Exception { com.dtstack.flinkx.options.Options options = new OptionParser(args).getOptions(); @@ -100,15 +94,15 @@ public static void main(String[] args) throws Exception { config.setPluginRoot(pluginRoot); } + if (StringUtils.isNotEmpty(remotePluginPath)) { + config.setRemotePluginPath(remotePluginPath); + } + Configuration flinkConf = new Configuration(); if (StringUtils.isNotEmpty(options.getFlinkconf())) { flinkConf = GlobalConfiguration.loadConfiguration(options.getFlinkconf()); } - if (StringUtils.isNotEmpty(remotePluginPath)) { - config.setRemotePluginPath(remotePluginPath); - } - StreamExecutionEnvironment env = (StringUtils.isNotBlank(monitor)) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(flinkConf); @@ -118,9 +112,10 @@ public static void main(String[] args) throws Exception { SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); + PluginUtil.registerPluginUrlToCachedFile(config, env); + env.setParallelism(speedConfig.getChannel()); env.setRestartStrategy(RestartStrategies.noRestart()); - BaseDataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); if(speedConfig.getReaderChannel() > 0){ @@ -133,18 +128,16 @@ public static void main(String[] args) throws Exception { BaseDataWriter dataWriter = DataWriterFactory.getDataWriter(config); DataStreamSink dataStreamSink = dataWriter.writeData(dataStream); - if(speedConfig.getWriterChannel() > 0){ dataStreamSink.setParallelism(speedConfig.getWriterChannel()); } + if(env instanceof MyLocalStreamEnvironment) { if(StringUtils.isNotEmpty(savepointPath)){ ((MyLocalStreamEnvironment) env).setSettings(SavepointRestoreSettings.forPath(savepointPath)); } } - addEnvClassPath(env, ClassLoaderManager.getClassPath()); - JobExecutionResult result = env.execute(jobIdString); if(env instanceof MyLocalStreamEnvironment){ ResultPrintUtil.printResult(result); @@ -211,19 +204,6 @@ private static void speedTest(DataTransferConfig config) { config.getJob().getSetting().getSpeed().setBytes(-1); } - private static void addEnvClassPath(StreamExecutionEnvironment env, Set classPathSet) throws Exception{ - int i = 0; - for(URL url : classPathSet){ - String classFileName = String.format(CLASS_FILE_NAME_FMT, i); - env.registerCachedFile(url.getPath(), classFileName, true); - i++; - } - - if(env instanceof MyLocalStreamEnvironment){ - ((MyLocalStreamEnvironment) env).setClasspaths(new ArrayList<>(classPathSet)); - } - } - private static Properties parseConf(String confStr) throws Exception{ if(StringUtils.isEmpty(confStr)){ return new Properties(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java index 2c49d1a973..f41b3b65a7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/classloader/PluginUtil.java @@ -19,16 +19,18 @@ package com.dtstack.flinkx.classloader; -import com.dtstack.flinkx.util.SysUtil; -import org.apache.commons.lang3.StringUtils; +import com.dtstack.flink.api.java.MyLocalStreamEnvironment; +import com.dtstack.flinkx.config.DataTransferConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import java.io.File; -import java.io.FilenameFilter; import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Objects; import java.util.Set; /** @@ -47,25 +49,28 @@ public class PluginUtil { private static final String JAR_PREFIX = "flinkx"; + private static final String FILE_PREFIX = "file:"; + private static final String SP = File.separator; + private static final String CLASS_FILE_NAME_FMT = "class_path_%d"; + public static Set getJarFileDirPath(String pluginName, String pluginRoot, String remotePluginPath) { Set urlList = new HashSet<>(); - List pathDir = new ArrayList<>(); - - if (pluginRoot != null) { - pathDir.add(new File(pluginRoot + File.separator + pluginName)); - pathDir.add(new File(pluginRoot + File.separator + COMMON_DIR + File.separator)); - } - if (remotePluginPath != null) { - pathDir.add(new File(remotePluginPath + File.separator + pluginName)); - pathDir.add(new File(remotePluginPath + File.separator + COMMON_DIR + File.separator)); - } + String pluginPath = Objects.isNull(remotePluginPath) ? pluginRoot : remotePluginPath; try { - for(File path : pathDir) { - urlList.addAll(SysUtil.findJarsInDir(path)); + String pluginJarPath = pluginRoot + SP + pluginName; + String commonJarPath = pluginRoot + SP + COMMON_DIR; + // 获取jar包名字,构建对应的URL地址 + for (String jarName : getJarNames(new File(pluginJarPath))) { + urlList.add(new URL(FILE_PREFIX + pluginPath + SP + pluginName + SP + jarName)); + } + + // 获取common jar包名字,构建对应的URL地址 + for (String jarName : getJarNames(new File(commonJarPath))) { + urlList.add(new URL(FILE_PREFIX + pluginPath + SP + COMMON_DIR + SP + jarName)); } return urlList; @@ -74,6 +79,19 @@ public static Set getJarFileDirPath(String pluginName, String pluginRoot, S } } + private static List getJarNames(File pluginPath) { + List jarNames = new ArrayList<>(); + if (pluginPath.exists() && pluginPath.isDirectory()) { + File[] jarFiles = pluginPath.listFiles((dir, name) -> + name.toLowerCase().startsWith(JAR_PREFIX) && name.toLowerCase().endsWith(".jar")); + + if (Objects.nonNull(jarFiles) && jarFiles.length > 0) { + Arrays.stream(jarFiles).forEach(item -> jarNames.add(item.getName())); + } + } + return jarNames; + } + public static String getPluginClassName(String pluginName) { String pluginClassName; if (pluginName.toLowerCase().endsWith(READER_SUFFIX)) { @@ -98,4 +116,28 @@ private static String camelize(String pluginName, String suffix) { sb.append(suffix.substring(0, 1).toUpperCase() + suffix.substring(1)); return sb.toString(); } + + public static void registerPluginUrlToCachedFile(DataTransferConfig config, StreamExecutionEnvironment env) { + String readerPluginName = config.getJob().getContent().get(0).getReader().getName(); + Set readerUrlList = PluginUtil.getJarFileDirPath(readerPluginName, config.getPluginRoot(), config.getRemotePluginPath()); + + String writerPluginName = config.getJob().getContent().get(0).getWriter().getName(); + Set writerUrlList = PluginUtil.getJarFileDirPath(writerPluginName, config.getPluginRoot(), config.getRemotePluginPath()); + + Set urlSet = new HashSet<>(); + + urlSet.addAll(readerUrlList); + urlSet.addAll(writerUrlList); + + int i = 0; + for (URL url : urlSet) { + String classFileName = String.format(CLASS_FILE_NAME_FMT, i); + env.registerCachedFile(url.getPath(), classFileName, true); + i++; + } + + if (env instanceof MyLocalStreamEnvironment) { + ((MyLocalStreamEnvironment) env).setClasspaths(new ArrayList<>(urlSet)); + } + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java index 087c5302bd..a1dabf4557 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/AbstractConfig.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.config; +import com.dtstack.flinkx.util.GsonUtil; import com.google.gson.internal.LinkedTreeMap; import java.io.Serializable; @@ -25,6 +26,7 @@ import java.math.BigInteger; import java.util.HashMap; import java.util.Map; +import java.util.Properties; /** * Abstract Config @@ -129,7 +131,7 @@ public int getIntVal(String key, int defaultValue) { if(ret instanceof BigDecimal) { return ((BigDecimal)ret).intValue(); } - throw new RuntimeException("can't cast " + key + " from " + ret.getClass().getName() + " to Integer"); + throw new RuntimeException(String.format("cant't %s from %s to int, internalMap = %s", key, ret.getClass().getName(), GsonUtil.GSON.toJson(internalMap))); } public long getLongVal(String key, long defaultValue) { @@ -158,7 +160,7 @@ public long getLongVal(String key, long defaultValue) { if(ret instanceof BigDecimal) { return ((BigDecimal)ret).longValue(); } - throw new RuntimeException("can't cast " + key + " from " + ret.getClass().getName() + " to Long"); + throw new RuntimeException(String.format("cant't %s from %s to long, internalMap = %s", key, ret.getClass().getName(), GsonUtil.GSON.toJson(internalMap))); } public double getDoubleVal(String key, double defaultValue) { @@ -187,7 +189,7 @@ public double getDoubleVal(String key, double defaultValue) { if (ret instanceof BigDecimal) { return ((BigDecimal) ret).doubleValue(); } - throw new RuntimeException("can't cast " + key + " from " + ret.getClass().getName() + " to Long"); + throw new RuntimeException(String.format("cant't %s from %s to double, internalMap = %s", key, ret.getClass().getName(), GsonUtil.GSON.toJson(internalMap))); } @@ -199,7 +201,33 @@ public boolean getBooleanVal(String key, boolean defaultValue) { if (ret instanceof Boolean) { return (Boolean) ret; } - throw new RuntimeException("can't cast " + key + " from " + ret.getClass().getName() + " to Long"); + throw new RuntimeException(String.format("cant't %s from %s to boolean, internalMap = %s", key, ret.getClass().getName(), GsonUtil.GSON.toJson(internalMap))); + } + + /** + * 从指定key中获取Properties配置信息 + * @param key + * @param p + * @return + */ + @SuppressWarnings("unchecked") + public Properties getProperties(String key, Properties p ){ + Object ret = internalMap.get(key); + if(p == null){ + p = new Properties(); + } + if (ret == null) { + return p; + } + if(ret instanceof Map){ + Map map = (Map) ret; + for (Map.Entry entry : map.entrySet()) { + p.setProperty(entry.getKey(), String.valueOf(entry.getValue())); + } + return p; + }else{ + throw new RuntimeException(String.format("cant't %s from %s to map, internalMap = %s", key, ret.getClass().getName(), GsonUtil.GSON.toJson(internalMap))); + } } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java index a487a64430..1e31635cda 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/DataTransferConfig.java @@ -83,7 +83,7 @@ private static void checkConfig(DataTransferConfig config) { Preconditions.checkNotNull(config); JobConfig jobConfig = config.getJob(); - Preconditions.checkNotNull(jobConfig, "Must spedify job element"); + Preconditions.checkNotNull(jobConfig, "Must specify job element"); List contentConfig = jobConfig.getContent(); Preconditions.checkNotNull(contentConfig, "Must specify content array"); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java index a22a816835..9692be0d51 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/config/ErrorLimitConfig.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.config; +import com.dtstack.flinkx.util.ValueUtil; + import java.util.HashMap; import java.util.Map; @@ -57,7 +59,7 @@ public void setRecord(Integer record) { } public Double getPercentage() { - return (Double) getVal(KEY_ERROR_PERCENTAGE_LIMIT); + return ValueUtil.getDoubleVal(getVal(KEY_ERROR_PERCENTAGE_LIMIT)); } public void setPercentage(Double percentage) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java index fb6aead1ab..303dce6cda 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConfigConstant.java @@ -15,9 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - - - package com.dtstack.flinkx.constants; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index 02822b8222..b996539e7f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -27,10 +27,13 @@ public class ConstantValue { public static final String STAR_SYMBOL = "*"; public static final String POINT_SYMBOL = "."; + public static final String TWO_POINT_SYMBOL = ".."; public static final String EQUAL_SYMBOL = "="; + public static final String COLON_SYMBOL = ":"; public static final String SINGLE_QUOTE_MARK_SYMBOL = "'"; public static final String DOUBLE_QUOTE_MARK_SYMBOL = "\""; public static final String COMMA_SYMBOL = ","; + public static final String SEMICOLON_SYMBOL = ";"; public static final String SINGLE_SLASH_SYMBOL = "/"; public static final String DOUBLE_SLASH_SYMBOL = "//"; @@ -38,6 +41,10 @@ public class ConstantValue { public static final String LEFT_PARENTHESIS_SYMBOL = "("; public static final String RIGHT_PARENTHESIS_SYMBOL = ")"; + + public static final String DATA_TYPE_UNSIGNED = "UNSIGNED"; + + public static final String KEY_HTTP = "http"; public static final String PROTOCOL_HTTP = "http://"; @@ -65,5 +72,4 @@ public class ConstantValue { public static final long STORE_SIZE_G = 1024L * 1024 * 1024; public static final long STORE_SIZE_M = 1024L * 1024; - } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java index ce360c6bc0..43f4782796 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/DecodeEnum.java @@ -32,7 +32,7 @@ public enum DecodeEnum { /** * text format */ - PLAIN("plain"); + TEXT("text"); private String name; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java index 40e950413c..1f8112d875 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/JsonDecoder.java @@ -17,7 +17,7 @@ */ package com.dtstack.flinkx.decoder; -import com.fasterxml.jackson.databind.ObjectMapper; +import com.dtstack.flinkx.util.GsonUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,15 +31,14 @@ * @author tudou */ public class JsonDecoder implements IDecode { - private static final String KEY_MESSAGE = "message"; private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - private static ObjectMapper objectMapper = new ObjectMapper(); + + private static final String KEY_MESSAGE = "message"; @Override - @SuppressWarnings("unchecked") public Map decode(final String message) { try { - Map event = objectMapper.readValue(message, Map.class); + Map event = GsonUtil.GSON.fromJson(message, GsonUtil.gsonMapTypeToken); if (!event.containsKey(KEY_MESSAGE)) { event.put(KEY_MESSAGE, message); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/TextDecoder.java similarity index 95% rename from flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/decoder/TextDecoder.java index 63771cfd6e..ed963d1d85 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/PlainDecoder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/decoder/TextDecoder.java @@ -26,7 +26,7 @@ * * @author tudou */ -public class PlainDecoder implements IDecode { +public class TextDecoder implements IDecode { @Override public Map decode(final String message) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java index 31e1272dfd..1291dbd493 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/ColumnType.java @@ -19,9 +19,11 @@ package com.dtstack.flinkx.enums; import com.dtstack.flinkx.constants.ConstantValue; +import org.apache.commons.lang3.StringUtils; import java.util.Arrays; import java.util.List; +import java.util.Locale; /** * Define standard column type for all the readers or writers that do not @@ -66,6 +68,12 @@ public enum ColumnType { STRING, VARCHAR, VARCHAR2, CHAR, NVARCHAR, TEXT, KEYWORD, BINARY ); + /** + * 根据字段类型的字符串找出对应的枚举 + * 找不到直接报错 IllegalArgumentException + * @param type + * @return + */ public static ColumnType fromString(String type) { if(type == null) { throw new RuntimeException("null ColumnType!"); @@ -75,15 +83,31 @@ public static ColumnType fromString(String type) { type = type.substring(0, type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL)); } - return valueOf(type.toUpperCase()); + type = type.toUpperCase(Locale.ENGLISH); + //为了支持无符号类型 如 int unsigned + if(StringUtils.contains(type,ConstantValue.DATA_TYPE_UNSIGNED)){ + type = type.replace(ConstantValue.DATA_TYPE_UNSIGNED,"").trim(); + } + return valueOf(type); } + /** + * 根据字段类型的字符串找到对应的枚举 找不到就直接返回ColumnType.STRING; + * @param type + * @return + */ public static ColumnType getType(String type){ + type = type.toUpperCase(Locale.ENGLISH); if(type.contains(ConstantValue.LEFT_PARENTHESIS_SYMBOL)){ type = type.substring(0, type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL)); } - if(type.toLowerCase().contains(ColumnType.TIMESTAMP.name().toLowerCase())){ + //为了支持无符号类型 如 int unsigned + if(StringUtils.contains(type,ConstantValue.DATA_TYPE_UNSIGNED)){ + type = type.replaceAll(ConstantValue.DATA_TYPE_UNSIGNED,"").trim(); + } + + if(type.contains(ColumnType.TIMESTAMP.name())){ return TIMESTAMP; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java index 323d49cf0c..a7d40a7ad6 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/EDatabaseType.java @@ -18,8 +18,6 @@ package com.dtstack.flinkx.enums; -import org.apache.commons.net.ftp.FTP; - /** * Database type * @@ -40,7 +38,6 @@ public enum EDatabaseType { MongoDB, Redis, ES, - SapHana, TeraData, /** @@ -56,4 +53,6 @@ public enum EDatabaseType { polarDB, Phoenix, dm, + SapHana, + KingBase } diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/exception/DataSourceException.java similarity index 63% rename from flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java rename to flinkx-core/src/main/java/com/dtstack/flinkx/exception/DataSourceException.java index 7f4c37f576..cf02033a0b 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/exception/DataSourceException.java @@ -16,26 +16,21 @@ * limitations under the License. */ - -package com.dtstack.flinkx.kafka.reader; - -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; - -import java.io.IOException; -import java.util.Properties; +package com.dtstack.flinkx.exception; /** - * Date: 2019/11/21 - * Company: www.dtstack.com - * - * @author tudou + * 数据源异常类型 */ -public class KafkaInputFormat extends KafkaBaseInputFormat { +public class DataSourceException extends RuntimeException { + private String dataSourceName; + + public DataSourceException(String sourceName, String message, Throwable cause) { + super(message, cause); + this.dataSourceName = sourceName; + } @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - Properties props = geneConsumerProp(); - consumer = new KafkaConsumer(props); + public String toString() { + return "datasourceName【" + dataSourceName+"】" + super.toString() + "\n" + getCause().toString(); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 431b6be981..885dbbac4b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -288,7 +288,7 @@ public Row nextRecord(Row row) throws IOException { numReadCounter.add(1); } if(bytesReadCounter!=null){ - bytesReadCounter.add(internalRow.toString().length()); + bytesReadCounter.add(internalRow.toString().getBytes().length); } } @@ -422,4 +422,8 @@ public void setTestConfig(TestConfig testConfig) { public void setDataTransferConfig(DataTransferConfig dataTransferConfig){ this.dataTransferConfig = dataTransferConfig; } + + public DataTransferConfig getDataTransferConfig() { + return dataTransferConfig; + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java index bb0d925f94..38b6fd2690 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormatBuilder.java @@ -67,7 +67,10 @@ public void setDataTransferConfig(DataTransferConfig dataTransferConfig){ public BaseRichInputFormat finish() { Preconditions.checkNotNull(format); - checkFormat(); + boolean check = format.getDataTransferConfig().getJob().getContent().get(0).getReader().getParameter().getBooleanVal("check", true); + if(check){ + checkFormat(); + } return format; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java index 7a86f3f2c2..d6c57d18ec 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/latch/MetricLatch.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.latch; import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.UrlUtil; import com.google.gson.Gson; import com.google.gson.internal.LinkedTreeMap; @@ -76,10 +77,11 @@ private int getIntMetricVal(String requestUrl) { try(InputStream inputStream = UrlUtil.open(requestUrl)) { try(Reader rd = new InputStreamReader(inputStream, StandardCharsets.UTF_8)) { Map map = gson.fromJson(rd, Map.class); + LOG.info("requestUrl = {}, and return map = {}", requestUrl, GsonUtil.GSON.toJson(map)); List userTaskAccumulators = (List) map.get("user-task-accumulators"); for(LinkedTreeMap accumulator : userTaskAccumulators) { if(metricName != null && metricName.equals(accumulator.get("name"))) { - return Integer.valueOf((String )accumulator.get("value")); + return Integer.parseInt((String )accumulator.get("value")); } } } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java index f7019f3996..942e254f8c 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/metrics/BaseMetric.java @@ -56,7 +56,7 @@ public void addMetric(String metricName, LongCounter counter, boolean meterView) metricCounters.put(metricName, counter); flinkxOutput.gauge(metricName, new SimpleAccumulatorGauge<>(counter)); if (meterView){ - flinkxOutput.meter(metricName + Metrics.SUFFIX_RATE, new SimpleLongCounterMeterView(counter, 60)); + flinkxOutput.meter(metricName + Metrics.SUFFIX_RATE, new SimpleLongCounterMeterView(counter, 20)); } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 1f379cca2a..68b59e1624 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -82,6 +82,15 @@ public class Options { @OptionRequired(description = "plugin load mode, by classpath or shipfile") private String pluginLoadMode = "shipfile"; + @OptionRequired(description = "kerberos krb5conf") + private String krb5conf ; + + @OptionRequired(description = "kerberos keytabPath") + private String keytab ; + + @OptionRequired(description = "kerberos principal") + private String principal ; + @OptionRequired(description = "applicationId on yarn cluster") private String appId; @@ -248,6 +257,30 @@ public void setP(String p) { this.p = p; } + public String getKrb5conf() { + return krb5conf; + } + + public void setKrb5conf(String krb5conf) { + this.krb5conf = krb5conf; + } + + public String getKeytab() { + return keytab; + } + + public void setKeytab(String keytab) { + this.keytab = keytab; + } + + public String getPrincipal() { + return principal; + } + + public void setPrincipal(String principal) { + this.principal = principal; + } + @Override public String toString() { return "Options{" + @@ -268,7 +301,9 @@ public String toString() { ", pluginLoadMode='" + pluginLoadMode + '\'' + ", appId='" + appId + '\'' + ", remotePluginPath='" + remotePluginPath + '\'' + - ", flinkConfiguration=" + flinkConfiguration + + ", krb5conf='" + krb5conf + '\'' + + ", keytab='" + keytab + '\'' + + ", principal='" + principal + '\'' + '}'; } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java index f9735c0ab3..e2ba4b7266 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseFileOutputFormat.java @@ -106,7 +106,7 @@ protected void openInternal(int taskNumber, int numTasks) throws IOException { nextBlock(); } - private void initPath(){ + protected void initPath(){ if(StringUtils.isNotBlank(fileName)) { outputFilePath = path + SP + fileName; } else { @@ -122,7 +122,7 @@ private void initPath(){ taskNumber, currentBlockFileNamePrefix, tmpPath, finishedPath); } - private void initFileIndex() { + protected void initFileIndex() { if (null != formatState && formatState.getFileIndex() > -1) { blockIndex = formatState.getFileIndex() + 1; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java index ac91a879fa..239ec59e86 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java @@ -29,27 +29,29 @@ import com.dtstack.flinkx.metrics.BaseMetric; import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.UrlUtil; import com.dtstack.flinkx.writer.DirtyDataManager; import com.dtstack.flinkx.writer.ErrorLimiter; -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; +import com.google.gson.reflect.TypeToken; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; import org.apache.flink.configuration.Configuration; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClientBuilder; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.types.Row; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_FORMAT_TRANSFORM; import static com.dtstack.flinkx.writer.WriteErrorTypes.ERR_NULL_POINTER; @@ -452,7 +454,7 @@ public void writeRecord(Row row) throws IOException { updateDuration(); if(bytesWriteCounter!=null){ - bytesWriteCounter.add(row.toString().length()); + bytesWriteCounter.add(row.toString().getBytes().length); } } @@ -541,20 +543,26 @@ protected String getTaskState() throws IOException{ return RUNNING_STATE; } - String taskState; + String taskState = null; CloseableHttpClient httpClient = HttpClientBuilder.create().build(); - String monitors = String.format("%s/jobs/%s", monitorUrl, jobId); + String monitors = String.format("%s/jobs/overview", monitorUrl); LOG.info("Monitor url:{}", monitors); - JsonParser parser = new JsonParser(); int retryNumber = 5; for (int i = 0; i < retryNumber; i++) { try{ String response = UrlUtil.get(httpClient, monitors); LOG.info("response:{}", response); - - JsonObject obj = parser.parse(response).getAsJsonObject(); - taskState = obj.get("state").getAsString(); + HashMap>> map = GsonUtil.GSON.fromJson(response, new TypeToken>>>() {}.getType()); + List> list = map.get("jobs"); + + for (HashMap hashMap : list) { + String jid = (String)hashMap.get("jid"); + if(Objects.equals(jid, jobId)){ + taskState = (String) hashMap.get("state"); + break; + } + } LOG.info("Job state is:{}", taskState); if(taskState != null){ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java index 197903dd83..dfd9e07929 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/BaseDataReader.java @@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.datastream.DataStream; -import com.dtstack.flinkx.streaming.api.functions.source.DtInputFormatSourceFunction; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import com.dtstack.flinkx.streaming.api.functions.source.DtInputFormatSourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import com.fasterxml.jackson.databind.ObjectMapper; @@ -92,7 +92,6 @@ public void setSrcCols(List srcCols) { protected BaseDataReader(DataTransferConfig config, StreamExecutionEnvironment env) { this.env = env; this.dataTransferConfig = config; - this.numPartitions = config.getJob().getSetting().getSpeed().getChannel(); this.numPartitions = Math.max(config.getJob().getSetting().getSpeed().getChannel(), config.getJob().getSetting().getSpeed().getReaderChannel()); this.bytes = config.getJob().getSetting().getSpeed().getBytes(); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java index 69ea028371..6104bc0c4b 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/reader/DataReaderFactory.java @@ -22,7 +22,6 @@ import com.dtstack.flinkx.classloader.PluginUtil; import com.dtstack.flinkx.config.DataTransferConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - import java.lang.reflect.Constructor; import java.net.URL; import java.util.Set; @@ -42,7 +41,7 @@ public static BaseDataReader getDataReader(DataTransferConfig config, StreamExec try { String pluginName = config.getJob().getContent().get(0).getReader().getName(); String pluginClassName = PluginUtil.getPluginClassName(pluginName); - Set urlList = PluginUtil.getJarFileDirPath(pluginName, config.getPluginRoot(), config.getRemotePluginPath()); + Set urlList = PluginUtil.getJarFileDirPath(pluginName, config.getPluginRoot(), null); return ClassLoaderManager.newInstance(urlList, cl -> { Class clazz = cl.loadClass(pluginClassName); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java b/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java index fd476f923d..537e4e1cd9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java @@ -20,6 +20,7 @@ import com.dtstack.flinkx.config.RestoreConfig; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.restore.FormatState; +import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; @@ -110,6 +111,7 @@ public void open(Configuration parameters) throws Exception { @Override public void run(SourceContext ctx) throws Exception { + Exception tryException = null; try { Counter completedSplitsCounter = getRuntimeContext().getMetricGroup().counter("numSplitsProcessed"); @@ -146,12 +148,27 @@ public void run(SourceContext ctx) throws Exception { isRunning = splitIterator.hasNext(); } } + } catch (Exception exception){ + tryException = exception; } finally { - format.close(); - if (format instanceof RichInputFormat) { - ((RichInputFormat) format).closeInputFormat(); - } isRunning = false; + try { + format.close(); + if (format instanceof RichInputFormat) { + ((RichInputFormat) format).closeInputFormat(); + } + }catch (Exception finallyException){ + if(null != tryException){ + LOG.error(ExceptionUtil.getErrorMessage(finallyException)); + tryException.addSuppressed(finallyException); + throw tryException; + }else { + throw finallyException; + } + } + if(null != tryException) { + throw tryException; + } } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index f53592c342..919a515ecd 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -28,11 +28,6 @@ import java.util.HashMap; import java.util.Map; import java.util.TimeZone; -import java.util.Calendar; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; import java.util.regex.Pattern; /** diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java index 10152a122d..7283371b20 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/FileSystemUtil.java @@ -69,8 +69,9 @@ public static void setHadoopUserName(Configuration conf){ } try { - String ticketCachePath = conf.get("hadoop.security.kerberos.ticket.cache.path"); - UserGroupInformation ugi = UserGroupInformation.getBestUGI(ticketCachePath, hadoopUserName); + String previousUserName = UserGroupInformation.getLoginUser().getUserName(); + LOG.info("Hadoop user from '{}' switch to '{}' with SIMPLE auth", previousUserName, hadoopUserName); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(hadoopUserName); UserGroupInformation.setLoginUser(ugi); } catch (Exception e) { LOG.warn("Set hadoop user name error:", e); @@ -87,7 +88,6 @@ public static boolean isOpenKerberos(Map hadoopConfig){ private static FileSystem getFsWithKerberos(Map hadoopConfig, String defaultFs) throws Exception{ UserGroupInformation ugi = getUGI(hadoopConfig, defaultFs); - UserGroupInformation.setLoginUser(ugi); return ugi.doAs(new PrivilegedAction() { @Override @@ -110,7 +110,6 @@ public static UserGroupInformation getUGI(Map hadoopConfig, Stri KerberosUtil.refreshConfig(); UserGroupInformation ugi = KerberosUtil.loginAndReturnUgi(getConfiguration(hadoopConfig, defaultFs), principal, keytabFileName); - UserGroupInformation.setLoginUser(ugi); return ugi; } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java index 9824c6c7df..47a41a150e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java @@ -51,6 +51,7 @@ public class GsonUtil { @SuppressWarnings("unchecked") private static Gson getGson() { GSON = new GsonBuilder() + .disableHtmlEscaping() .setPrettyPrinting() .create(); try { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java index 2e912373a7..5c603d46c1 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/JsonModifyUtil.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.util; +import com.dtstack.flinkx.constants.ConstantValue; import org.apache.commons.lang3.StringUtils; import java.util.HashMap; @@ -41,9 +42,9 @@ public static String JsonValueReplace(String json, HashMap param */ public static HashMap CommandTransform(String command) { HashMap parameter = new HashMap<>(); - String[] split = StringUtils.split(command, ","); + String[] split = StringUtils.split(command, ConstantValue.COMMA_SYMBOL); for (String item : split) { - String[] temp = item.split("="); + String[] temp = item.split(ConstantValue.EQUAL_SYMBOL); parameter.put(temp[0], temp[1]); } return parameter; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/MapUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/MapUtil.java index 7fef2a4543..73916facec 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/MapUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/MapUtil.java @@ -2,6 +2,7 @@ import com.fasterxml.jackson.core.JsonGenerationException; import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonMappingException; import com.google.gson.internal.LinkedHashTreeMap; import com.google.gson.internal.LinkedTreeMap; @@ -55,4 +56,9 @@ public static Map objectToMap(Object obj) throws Exception{ public static T jsonStrToObject(String jsonStr, Class clazz) throws JsonParseException, JsonMappingException, JsonGenerationException, IOException { return objectMapper.readValue(jsonStr, clazz); } + + public static String writeValueAsString(Object obj) throws JsonProcessingException { + return objectMapper.writeValueAsString(obj); + } + } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java index ef25d54718..9492aef130 100755 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RangeSplitUtil.java @@ -19,6 +19,9 @@ package com.dtstack.flinkx.util; import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; /** * Range Split Utilities @@ -89,4 +92,49 @@ public static BigInteger[] doBigIntegerSplit(BigInteger left, BigInteger right, return result; } } + + /** + * 分隔数组 根据段数分段 + * 多出部分在最后一个数组 + * 不够的最后为空数组 + * + * @param data 被分隔的数组 + * @param segments 需要分隔的段数 + * @return + */ + @SuppressWarnings("unchecked") + public static List> subListBySegment(List data, int segments) { + List> result = new ArrayList<>(); + // 数据长度 + int size = data.size(); + // segments == 0 ,不需要分隔 + if (size > 0 && segments > 0) { + // 每段List + List cutList; + if(size <= segments){ + for (int i = 0; i < segments; i++) { + if(i < size){ + cutList = Collections.singletonList(data.get(i)); + }else{ + cutList = Collections.EMPTY_LIST; + } + result.add(cutList); + } + }else{ + // 每段数量 + int count = size / segments; + for (int i = 0; i < segments; i++) { + if (i == segments - 1) { + cutList = new ArrayList<>(data.subList(count * i, size)); + } else { + cutList = new ArrayList<>(data.subList(count * i, count * (i + 1))); + } + result.add(cutList); + } + } + } else { + result.add(data); + } + return result; + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java index 0737a14a61..7d795819b8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/RetryUtil.java @@ -48,15 +48,14 @@ public final class RetryUtil { public static T executeWithRetry(Callable callable, int retryTimes, long sleepTimeInMilliSecond, - boolean exponential) throws Exception { + boolean exponential){ Retry retry = new Retry(); return retry.doRetry(callable, retryTimes, sleepTimeInMilliSecond, exponential, null); } private static class Retry { - public T doRetry(Callable callable, int retryTimes, long sleepTimeInMilliSecond, boolean exponential, List> retryExceptionClasss) - throws Exception { + public T doRetry(Callable callable, int retryTimes, long sleepTimeInMilliSecond, boolean exponential, List> retryExceptionClasss){ if (null == callable) { throw new IllegalArgumentException("系统编程错误, 入参callable不能为空 ! "); @@ -86,7 +85,7 @@ public T doRetry(Callable callable, int retryTimes, long sleepTimeInMilli } } if (!needRetry) { - throw saveException; + throw new RuntimeException(saveException); } } @@ -114,12 +113,12 @@ public T doRetry(Callable callable, int retryTimes, long sleepTimeInMilli long realTimeSleep = System.currentTimeMillis()-startTime; LOG.error(String.format("Exception when calling callable, 即将尝试执行第%s次重试.本次重试计划等待[%s]ms,实际等待[%s]ms, 异常Msg:[%s]", - i+1, timeToSleep,realTimeSleep, e.getMessage())); + i+1, timeToSleep,realTimeSleep, ExceptionUtil.getErrorMessage(e))); } } } - throw saveException; + throw new RuntimeException(saveException); } protected T call(Callable callable) throws Exception { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index 7fd10f9df9..473e2816a0 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -201,12 +201,12 @@ public static String col2string(Object column, String type) { public static String row2string(Row row, List columnTypes, String delimiter) throws WriteRecordException { // convert row to string - int size = row.getArity(); + int cnt = row.getArity(); StringBuilder sb = new StringBuilder(128); int i = 0; try { - for (; i < size; ++i) { + for (; i < cnt; ++i) { if (i != 0) { sb.append(delimiter); } @@ -219,9 +219,9 @@ public static String row2string(Row row, List columnTypes, String delimi sb.append(col2string(column, columnTypes.get(i))); } - } catch(Exception e) { + } catch(Exception ex) { String msg = "StringUtil.row2string error: when converting field[" + i + "] in Row(" + row + ")"; - throw new WriteRecordException(msg, e, i, row); + throw new WriteRecordException(msg, ex, i, row); } return sb.toString(); @@ -301,22 +301,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ return tokensList; } - /** - * 调用{@linkplain com.dtstack.flinkx.util.StringUtil}的splitIgnoreQuota处理 并对返回结果按照.拼接 - * @param table [dbo.schema1].[table] - * @return dbo.schema1.table - */ - public static String splitIgnoreQuotaAndJoinByPoint(String table) { - List strings = StringUtil.splitIgnoreQuota(table, ConstantValue.POINT_SYMBOL.charAt(0)); - StringBuffer stringBuffer = new StringBuffer(64); - for(int i =0 ;i strings = StringUtil.splitIgnoreQuota(table, ConstantValue.POINT_SYMBOL.charAt(0)); + StringBuffer stringBuffer = new StringBuffer(64); + for(int i =0; i < strings.size(); i++){ + stringBuffer.append(strings.get(i)); + if(i != strings.size()-1){ + stringBuffer.append(ConstantValue.POINT_SYMBOL); + } + } + return stringBuffer.toString(); + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ZkHelper.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ZkHelper.java new file mode 100644 index 0000000000..c8f80392d4 --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ZkHelper.java @@ -0,0 +1,94 @@ +package com.dtstack.flinkx.util; + +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; + +/** + * ZooKeeper Util类 + * @author kunni@dtstack.com + */ +public class ZkHelper { + + private static final Logger LOG = LoggerFactory.getLogger(ZkHelper.class); + + public static final int DEFAULT_TIMEOUT = 5000; + + public static final String APPEND_PATH = "/table"; + + public static final String DEFAULT_PATH = "/hbase"; + + private ZkHelper(){} + + /** + * @param hosts ip和端口 + * @param timeOut 创建超时时间 + */ + public static ZooKeeper createZkClient(String hosts, int timeOut) { + try { + LOG.info("try to create zookeeper client... "); + return new ZooKeeper(hosts, timeOut, null, true); + }catch (IOException e){ + LOG.error("create zookeeper client failed. error {} ", ExceptionUtil.getErrorMessage(e)); + return null; + } + } + + /** + * 获取某个节点的创建时间 + * @param zooKeeper zookeeper + * @param path 节点路径 + * @return 创建时间 + */ + public static long getCreateTime(ZooKeeper zooKeeper, String path) { + Stat stat = new Stat(); + if(zooKeeper != null){ + try{ + zooKeeper.getData(path, null, stat); + return stat.getCtime(); + }catch (Exception e){ + LOG.error("failed to get create time of {}, {}", path, ExceptionUtil.getErrorMessage(e)); + return 0L; + } + }else { + return 0L; + } + } + + /** + * 获取某个目录下的所有子节点 + * @param zooKeeper zookeeper + * @param path 目录 + * @return 子节点路径名集合 + */ + public static List getChildren(ZooKeeper zooKeeper, String path) { + if(zooKeeper != null){ + try{ + return zooKeeper.getChildren(path,false); + }catch (Exception e){ + LOG.error("failed to get children, path :{}, {} ",path, ExceptionUtil.getErrorMessage(e)); + return null; + } + + }else { + return null; + } + + } + + public static void closeZooKeeper(ZooKeeper zooKeeper) { + if (zooKeeper != null) { + try { + zooKeeper.close(); + } catch (InterruptedException e) { + LOG.error(ExceptionUtils.getMessage(e)); + } + } + } + +} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriterFactory.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriterFactory.java index 4558aee1a0..36e8d2ede7 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriterFactory.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/DataWriterFactory.java @@ -40,7 +40,7 @@ public static BaseDataWriter getDataWriter(DataTransferConfig config) { try { String pluginName = config.getJob().getContent().get(0).getWriter().getName(); String pluginClassName = PluginUtil.getPluginClassName(pluginName); - Set urlList = PluginUtil.getJarFileDirPath(pluginName, config.getPluginRoot(), config.getRemotePluginPath()); + Set urlList = PluginUtil.getJarFileDirPath(pluginName, config.getPluginRoot(), null); return ClassLoaderManager.newInstance(urlList, cl -> { Class clazz = cl.loadClass(pluginClassName); diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java index 189f198a15..1d2f7d9ae8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/writer/ErrorLimiter.java @@ -29,8 +29,7 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -public class -ErrorLimiter { +public class ErrorLimiter { private final Integer maxErrors; private final Double maxErrorRatio; diff --git a/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java deleted file mode 100644 index fb7435c175..0000000000 --- a/flinkx-core/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ /dev/null @@ -1,1278 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.java; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.Public; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; -import org.apache.flink.api.common.io.FileInputFormat; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.operators.OperatorInformation; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.io.CollectionInputFormat; -import org.apache.flink.api.java.io.CsvReader; -import org.apache.flink.api.java.io.IteratorInputFormat; -import org.apache.flink.api.java.io.ParallelIteratorInputFormat; -import org.apache.flink.api.java.io.PrimitiveInputFormat; -import org.apache.flink.api.java.io.TextInputFormat; -import org.apache.flink.api.java.io.TextValueInputFormat; -import org.apache.flink.api.java.operators.DataSink; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.operators.Operator; -import org.apache.flink.api.java.operators.OperatorTranslation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.core.fs.Path; -import org.apache.flink.types.StringValue; -import org.apache.flink.util.NumberSequenceIterator; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SplittableIterator; -import org.apache.flink.util.Visitor; - -import com.esotericsoftware.kryo.Serializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * The ExecutionEnvironment is the context in which a program is executed. A - * {@link LocalEnvironment} will cause execution in the current JVM, a - * {@link RemoteEnvironment} will cause execution on a remote setup. - * - *

The environment provides methods to control the job execution (such as setting the parallelism) - * and to interact with the outside world (data access). - * - *

Please note that the execution environment needs strong type information for the input and return types - * of all operations that are executed. This means that the environments needs to know that the return - * value of an operation is for example a Tuple of String and Integer. - * Because the Java compiler throws much of the generic type information away, most methods attempt to re- - * obtain that information using reflection. In certain cases, it may be necessary to manually supply that - * information to some of the methods. - * - * @see LocalEnvironment - * @see RemoteEnvironment - */ -@Public -public abstract class ExecutionEnvironment { - - /** The logger used by the environment and its subclasses. */ - protected static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); - - /** The environment of the context (local by default, cluster if invoked through command line). */ - private static ThreadLocal contextEnvironmentFactory = new ThreadLocal<>(); - - /** The default parallelism used by local environments. */ - private static int defaultLocalDop = Runtime.getRuntime().availableProcessors(); - - // -------------------------------------------------------------------------------------------- - - private final List> sinks = new ArrayList<>(); - - private final List> cacheFile = new ArrayList<>(); - - private final ExecutionConfig config = new ExecutionConfig(); - - /** Result from the latest execution, to make it retrievable when using eager execution methods. */ - protected JobExecutionResult lastJobExecutionResult; - - /** The ID of the session, defined by this execution environment. Sessions and Jobs are same in - * Flink, as Jobs can consist of multiple parts that are attached to the growing dataflow graph. */ - protected JobID jobID; - - /** The session timeout in seconds. */ - protected long sessionTimeout; - - /** Flag to indicate whether sinks have been cleared in previous executions. */ - private boolean wasExecuted = false; - - /** - * Creates a new Execution Environment. - */ - protected ExecutionEnvironment() { - jobID = JobID.generate(); - } - - // -------------------------------------------------------------------------------------------- - // Properties - // -------------------------------------------------------------------------------------------- - - /** - * Gets the config object that defines execution parameters. - * - * @return The environment's execution configuration. - */ - public ExecutionConfig getConfig() { - return config; - } - - /** - * Gets the parallelism with which operation are executed by default. Operations can - * individually override this value to use a specific parallelism via - * {@link Operator#setParallelism(int)}. Other operations may need to run with a different - * parallelism - for example calling - * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire - * set will insert eventually an operation that runs non-parallel (parallelism of one). - * - * @return The parallelism used by operations, unless they override that value. This method - * returns {@link ExecutionConfig#PARALLELISM_DEFAULT}, if the environment's default parallelism should be used. - */ - public int getParallelism() { - return config.getParallelism(); - } - - /** - * Sets the parallelism for operations executed through this environment. - * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with - * x parallel instances. - * - *

This method overrides the default parallelism for this environment. - * The {@link LocalEnvironment} uses by default a value equal to the number of hardware - * contexts (CPU cores / threads). When executing the program via the command line client - * from a JAR file, the default parallelism is the one configured for that setup. - * - * @param parallelism The parallelism - */ - public void setParallelism(int parallelism) { - config.setParallelism(parallelism); - } - - /** - * Sets the restart strategy configuration. The configuration specifies which restart strategy - * will be used for the execution graph in case of a restart. - * - * @param restartStrategyConfiguration Restart strategy configuration to be set - */ - @PublicEvolving - public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { - config.setRestartStrategy(restartStrategyConfiguration); - } - - /** - * Returns the specified restart strategy configuration. - * - * @return The restart strategy configuration to be used - */ - @PublicEvolving - public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { - return config.getRestartStrategy(); - } - - /** - * Sets the number of times that failed tasks are re-executed. A value of zero - * effectively disables fault tolerance. A value of {@code -1} indicates that the system - * default value (as defined in the configuration) should be used. - * - * @param numberOfExecutionRetries The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #setRestartStrategy}. The - * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of - * execution retries. - */ - @Deprecated - @PublicEvolving - public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { - config.setNumberOfExecutionRetries(numberOfExecutionRetries); - } - - /** - * Gets the number of times the system will try to re-execute failed tasks. A value - * of {@code -1} indicates that the system default value (as defined in the configuration) - * should be used. - * - * @return The number of times the system will try to re-execute failed tasks. - * - * @deprecated This method will be replaced by {@link #getRestartStrategy}. The - * {@link RestartStrategies.FixedDelayRestartStrategyConfiguration} contains the number of - * execution retries. - */ - @Deprecated - @PublicEvolving - public int getNumberOfExecutionRetries() { - return config.getNumberOfExecutionRetries(); - } - - /** - * Returns the {@link org.apache.flink.api.common.JobExecutionResult} of the last executed job. - * - * @return The execution result from the latest job execution. - */ - public JobExecutionResult getLastJobExecutionResult(){ - return this.lastJobExecutionResult; - } - - // -------------------------------------------------------------------------------------------- - // Session Management - // -------------------------------------------------------------------------------------------- - - /** - * Gets the JobID by which this environment is identified. The JobID sets the execution context - * in the cluster or local environment. - * - * @return The JobID of this environment. - * @see #getIdString() - */ - @PublicEvolving - public JobID getId() { - return this.jobID; - } - - /** - * Gets the JobID by which this environment is identified, as a string. - * - * @return The JobID as a string. - * @see #getId() - */ - @PublicEvolving - public String getIdString() { - return this.jobID.toString(); - } - - /** - * Sets the session timeout to hold the intermediate results of a job. This only - * applies the updated timeout in future executions. - * - * @param timeout The timeout, in seconds. - */ - @PublicEvolving - public void setSessionTimeout(long timeout) { - throw new IllegalStateException("Support for sessions is currently disabled. " + - "It will be enabled in future Flink versions."); - // Session management is disabled, revert this commit to enable - //if (timeout < 0) { - // throw new IllegalArgumentException("The session timeout must not be less than zero."); - //} - //this.sessionTimeout = timeout; - } - - /** - * Gets the session timeout for this environment. The session timeout defines for how long - * after an execution, the job and its intermediate results will be kept for future - * interactions. - * - * @return The session timeout, in seconds. - */ - @PublicEvolving - public long getSessionTimeout() { - return sessionTimeout; - } - - /** - * Starts a new session, discarding the previous data flow and all of its intermediate results. - */ - @PublicEvolving - public abstract void startNewSession() throws Exception; - - // -------------------------------------------------------------------------------------------- - // Registry for types and serializers - // -------------------------------------------------------------------------------------------- - - /** - * Adds a new Kryo default serializer to the Runtime. - * - *

Note that the serializer instance must be serializable (as defined by java.io.Serializable), - * because it may be distributed to the worker nodes by java serialization. - * - * @param type The class of the types serialized with the given serializer. - * @param serializer The serializer to use. - */ - public & Serializable>void addDefaultKryoSerializer(Class type, T serializer) { - config.addDefaultKryoSerializer(type, serializer); - } - - /** - * Adds a new Kryo default serializer to the Runtime. - * - * @param type The class of the types serialized with the given serializer. - * @param serializerClass The class of the serializer to use. - */ - public void addDefaultKryoSerializer(Class type, Class> serializerClass) { - config.addDefaultKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with a Kryo Serializer. - * - *

Note that the serializer instance must be serializable (as defined by java.io.Serializable), - * because it may be distributed to the worker nodes by java serialization. - * - * @param type The class of the types serialized with the given serializer. - * @param serializer The serializer to use. - */ - public & Serializable>void registerTypeWithKryoSerializer(Class type, T serializer) { - config.registerTypeWithKryoSerializer(type, serializer); - } - - /** - * Registers the given Serializer via its class as a serializer for the given type at the KryoSerializer. - * - * @param type The class of the types serialized with the given serializer. - * @param serializerClass The class of the serializer to use. - */ - public void registerTypeWithKryoSerializer(Class type, Class> serializerClass) { - config.registerTypeWithKryoSerializer(type, serializerClass); - } - - /** - * Registers the given type with the serialization stack. If the type is eventually - * serialized as a POJO, then the type is registered with the POJO serializer. If the - * type ends up being serialized with Kryo, then it will be registered at Kryo to make - * sure that only tags are written. - * - * @param type The class of the type to register. - */ - public void registerType(Class type) { - if (type == null) { - throw new NullPointerException("Cannot register null type class."); - } - - TypeInformation typeInfo = TypeExtractor.createTypeInfo(type); - - if (typeInfo instanceof PojoTypeInfo) { - config.registerPojoType(type); - } else { - config.registerKryoType(type); - } - } - - // -------------------------------------------------------------------------------------------- - // Data set creations - // -------------------------------------------------------------------------------------------- - - // ---------------------------------- Text Input Format --------------------------------------- - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * The file will be read with the UTF-8 character set. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFile(String filePath) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new TextInputFormat(new Path(filePath)), BasicTypeInfo.STRING_TYPE_INFO, Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * The {@link java.nio.charset.Charset} with the given name will be used to read the files. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param charsetName The name of the character set used to read the file. - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFile(String filePath, String charsetName) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - TextInputFormat format = new TextInputFormat(new Path(filePath)); - format.setCharsetName(charsetName); - return new DataSource<>(this, format, BasicTypeInfo.STRING_TYPE_INFO, Utils.getCallLocationName()); - } - - // -------------------------- Text Input Format With String Value------------------------------ - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * This method is similar to {@link #readTextFile(String)}, but it produces a DataSet with mutable - * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations - * to be less object and garbage collection heavy. - * - *

The file will be read with the UTF-8 character set. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @return A {@link DataSet} that represents the data read from the given file as text lines. - */ - public DataSource readTextFileWithValue(String filePath) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new TextValueInputFormat(new Path(filePath)), new ValueTypeInfo<>(StringValue.class), Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the Strings produced by reading the given file line wise. - * This method is similar to {@link #readTextFile(String, String)}, but it produces a DataSet with mutable - * {@link StringValue} objects, rather than Java Strings. StringValues can be used to tune implementations - * to be less object and garbage collection heavy. - * - *

The {@link java.nio.charset.Charset} with the given name will be used to read the files. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param charsetName The name of the character set used to read the file. - * @param skipInvalidLines A flag to indicate whether to skip lines that cannot be read with the given character set. - * - * @return A DataSet that represents the data read from the given file as text lines. - */ - public DataSource readTextFileWithValue(String filePath, String charsetName, boolean skipInvalidLines) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - TextValueInputFormat format = new TextValueInputFormat(new Path(filePath)); - format.setCharsetName(charsetName); - format.setSkipInvalidLines(skipInvalidLines); - return new DataSource<>(this, format, new ValueTypeInfo<>(StringValue.class), Utils.getCallLocationName()); - } - - // ----------------------------------- Primitive Input Format --------------------------------------- - - /** - * Creates a {@link DataSet} that represents the primitive type produced by reading the given file line wise. - * This method is similar to {@link #readCsvFile(String)} with single field, but it produces a DataSet not through - * {@link org.apache.flink.api.java.tuple.Tuple1}. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param typeClass The primitive type class to be read. - * @return A {@link DataSet} that represents the data read from the given file as primitive type. - */ - public DataSource readFileOfPrimitives(String filePath, Class typeClass) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new PrimitiveInputFormat<>(new Path(filePath), typeClass), TypeExtractor.getForClass(typeClass), Utils.getCallLocationName()); - } - - /** - * Creates a {@link DataSet} that represents the primitive type produced by reading the given file in delimited way. - * This method is similar to {@link #readCsvFile(String)} with single field, but it produces a DataSet not through - * {@link org.apache.flink.api.java.tuple.Tuple1}. - * - * @param filePath The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path"). - * @param delimiter The delimiter of the given file. - * @param typeClass The primitive type class to be read. - * @return A {@link DataSet} that represents the data read from the given file as primitive type. - */ - public DataSource readFileOfPrimitives(String filePath, String delimiter, Class typeClass) { - Preconditions.checkNotNull(filePath, "The file path may not be null."); - - return new DataSource<>(this, new PrimitiveInputFormat<>(new Path(filePath), delimiter, typeClass), TypeExtractor.getForClass(typeClass), Utils.getCallLocationName()); - } - - // ----------------------------------- CSV Input Format --------------------------------------- - - /** - * Creates a CSV reader to read a comma separated value (CSV) file. The reader has options to - * define parameters and field types and will eventually produce the DataSet that corresponds to - * the read and parsed CSV input. - * - * @param filePath The path of the CSV file. - * @return A CsvReader that can be used to configure the CSV input. - */ - public CsvReader readCsvFile(String filePath) { - return new CsvReader(filePath, this); - } - - // ------------------------------------ File Input Format ----------------------------------------- - - public DataSource readFile(FileInputFormat inputFormat, String filePath) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - if (filePath == null) { - throw new IllegalArgumentException("The file path must not be null."); - } - - inputFormat.setFilePath(new Path(filePath)); - try { - return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); - } - catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be automatically determined. " + - "Please specify the TypeInformation of the produced type explicitly by using the " + - "'createInput(InputFormat, TypeInformation)' method instead."); - } - } - - // ----------------------------------- Generic Input Format --------------------------------------- - - /** - * Generic method to create an input {@link DataSet} with in {@link InputFormat}. The DataSet will not be - * immediately created - instead, this method returns a DataSet that will be lazily created from - * the input format once the program is executed. - * - *

Since all data sets need specific information about their types, this method needs to determine - * the type of the data produced by the input format. It will attempt to determine the data type - * by reflection, unless the input format implements the {@link ResultTypeQueryable} interface. - * In the latter case, this method will invoke the {@link ResultTypeQueryable#getProducedType()} - * method to determine data type produced by the input format. - * - * @param inputFormat The input format used to create the data set. - * @return A {@link DataSet} that represents the data created by the input format. - * - * @see #createInput(InputFormat, TypeInformation) - */ - public DataSource createInput(InputFormat inputFormat) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - - try { - return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat)); - } - catch (Exception e) { - throw new InvalidProgramException("The type returned by the input format could not be automatically determined. " + - "Please specify the TypeInformation of the produced type explicitly by using the " + - "'createInput(InputFormat, TypeInformation)' method instead.", e); - } - } - - /** - * Generic method to create an input DataSet with in {@link InputFormat}. The {@link DataSet} will not be - * immediately created - instead, this method returns a {@link DataSet} that will be lazily created from - * the input format once the program is executed. - * - *

The {@link DataSet} is typed to the given TypeInformation. This method is intended for input formats that - * where the return type cannot be determined by reflection analysis, and that do not implement the - * {@link ResultTypeQueryable} interface. - * - * @param inputFormat The input format used to create the data set. - * @return A {@link DataSet} that represents the data created by the input format. - * - * @see #createInput(InputFormat) - */ - public DataSource createInput(InputFormat inputFormat, TypeInformation producedType) { - if (inputFormat == null) { - throw new IllegalArgumentException("InputFormat must not be null."); - } - - if (producedType == null) { - throw new IllegalArgumentException("Produced type information must not be null."); - } - - return new DataSource<>(this, inputFormat, producedType, Utils.getCallLocationName()); - } - - // ----------------------------------- Collection --------------------------------------- - - /** - * Creates a DataSet from the given non-empty collection. The type of the data set is that - * of the elements in the collection. - * - *

The framework will try and determine the exact type from the collection elements. - * In case of generic elements, it may be necessary to manually supply the type information - * via {@link #fromCollection(Collection, TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @return A DataSet representing the given collection. - * - * @see #fromCollection(Collection, TypeInformation) - */ - public DataSource fromCollection(Collection data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.size() == 0) { - throw new IllegalArgumentException("The size of the collection must not be empty."); - } - - X firstValue = data.iterator().next(); - - TypeInformation type = TypeExtractor.getForObject(firstValue); - CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource<>(this, new CollectionInputFormat<>(data, type.createSerializer(config)), type, Utils.getCallLocationName()); - } - - /** - * Creates a DataSet from the given non-empty collection. Note that this operation will result - * in a non-parallel data source, i.e. a data source with a parallelism of one. - * - *

The returned DataSet is typed to the given TypeInformation. - * - * @param data The collection of elements to create the data set from. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the given collection. - * - * @see #fromCollection(Collection) - */ - public DataSource fromCollection(Collection data, TypeInformation type) { - return fromCollection(data, type, Utils.getCallLocationName()); - } - - private DataSource fromCollection(Collection data, TypeInformation type, String callLocationName) { - CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource<>(this, new CollectionInputFormat<>(data, type.createSerializer(config)), type, callLocationName); - } - - /** - * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until - * the actual execution happens, the type of data returned by the iterator must be given - * explicitly in the form of the type class (this is due to the fact that the Java compiler - * erases the generic type information). - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @param type The class of the data produced by the iterator. Must not be a generic class. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromCollection(Iterator, TypeInformation) - */ - public DataSource fromCollection(Iterator data, Class type) { - return fromCollection(data, TypeExtractor.getForClass(type)); - } - - /** - * Creates a DataSet from the given iterator. Because the iterator will remain unmodified until - * the actual execution happens, the type of data returned by the iterator must be given - * explicitly in the form of the type information. This method is useful for cases where the type - * is generic. In that case, the type class (as given in {@link #fromCollection(Iterator, Class)} - * does not supply all type information. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The collection of elements to create the data set from. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromCollection(Iterator, Class) - */ - public DataSource fromCollection(Iterator data, TypeInformation type) { - return new DataSource<>(this, new IteratorInputFormat<>(data), type, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains the given elements. The elements must all be of the same type, - * for example, all of the {@link String} or {@link Integer}. The sequence of elements must not be empty. - * - *

The framework will try and determine the exact type from the collection elements. - * In case of generic elements, it may be necessary to manually supply the type information - * via {@link #fromCollection(Collection, TypeInformation)}. - * - *

Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param data The elements to make up the data set. - * @return A DataSet representing the given list of elements. - */ - @SafeVarargs - public final DataSource fromElements(X... data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.length == 0) { - throw new IllegalArgumentException("The number of elements must not be zero."); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForObject(data[0]); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + data[0].getClass().getName() - + "; please specify the TypeInformation manually via " - + "ExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - - return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains the given elements. The framework will determine the type according to the - * based type user supplied. The elements should be the same or be the subclass to the based type. - * The sequence of elements must not be empty. - * Note that this operation will result in a non-parallel data source, i.e. a data source with - * a parallelism of one. - * - * @param type The base class type for every element in the collection. - * @param data The elements to make up the data set. - * @return A DataSet representing the given list of elements. - */ - @SafeVarargs - public final DataSource fromElements(Class type, X... data) { - if (data == null) { - throw new IllegalArgumentException("The data must not be null."); - } - if (data.length == 0) { - throw new IllegalArgumentException("The number of elements must not be zero."); - } - - TypeInformation typeInfo; - try { - typeInfo = TypeExtractor.getForClass(type); - } - catch (Exception e) { - throw new RuntimeException("Could not create TypeInformation for type " + type.getName() - + "; please specify the TypeInformation manually via " - + "ExecutionEnvironment#fromElements(Collection, TypeInformation)", e); - } - - return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName()); - } - - /** - * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the - * framework to create a parallel data source that returns the elements in the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type of data - * returned by the iterator must be given explicitly in the form of the type class (this is due to the - * fact that the Java compiler erases the generic type information). - * - * @param iterator The iterator that produces the elements of the data set. - * @param type The class of the data produced by the iterator. Must not be a generic class. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromParallelCollection(SplittableIterator, TypeInformation) - */ - public DataSource fromParallelCollection(SplittableIterator iterator, Class type) { - return fromParallelCollection(iterator, TypeExtractor.getForClass(type)); - } - - /** - * Creates a new data set that contains elements in the iterator. The iterator is splittable, allowing the - * framework to create a parallel data source that returns the elements in the iterator. - * - *

Because the iterator will remain unmodified until the actual execution happens, the type of data - * returned by the iterator must be given explicitly in the form of the type information. - * This method is useful for cases where the type is generic. In that case, the type class - * (as given in {@link #fromParallelCollection(SplittableIterator, Class)} does not supply all type information. - * - * @param iterator The iterator that produces the elements of the data set. - * @param type The TypeInformation for the produced data set. - * @return A DataSet representing the elements in the iterator. - * - * @see #fromParallelCollection(SplittableIterator, Class) - */ - public DataSource fromParallelCollection(SplittableIterator iterator, TypeInformation type) { - return fromParallelCollection(iterator, type, Utils.getCallLocationName()); - } - - // private helper for passing different call location names - private DataSource fromParallelCollection(SplittableIterator iterator, TypeInformation type, String callLocationName) { - return new DataSource<>(this, new ParallelIteratorInputFormat<>(iterator), type, callLocationName); - } - - /** - * Creates a new data set that contains a sequence of numbers. The data set will be created in parallel, - * so there is no guarantee about the order of the elements. - * - * @param from The number to start at (inclusive). - * @param to The number to stop at (inclusive). - * @return A DataSet, containing all number in the {@code [from, to]} interval. - */ - public DataSource generateSequence(long from, long to) { - return fromParallelCollection(new NumberSequenceIterator(from, to), BasicTypeInfo.LONG_TYPE_INFO, Utils.getCallLocationName()); - } - - // -------------------------------------------------------------------------------------------- - // Executing - // -------------------------------------------------------------------------------------------- - - /** - * Triggers the program execution. The environment will execute all parts of the program that have - * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, - * writing results (e.g. {@link DataSet#writeAsText(String)}, - * {@link DataSet#write(org.apache.flink.api.common.io.FileOutputFormat, String)}, or other generic - * data sinks created with {@link DataSet#output(org.apache.flink.api.common.io.OutputFormat)}. - * - *

The program execution will be logged and displayed with a generated default name. - * - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception Thrown, if the program executions fails. - */ - public JobExecutionResult execute() throws Exception { - return execute(getDefaultName()); - } - - /** - * Triggers the program execution. The environment will execute all parts of the program that have - * resulted in a "sink" operation. Sink operations are for example printing results ({@link DataSet#print()}, - * writing results (e.g. {@link DataSet#writeAsText(String)}, - * {@link DataSet#write(org.apache.flink.api.common.io.FileOutputFormat, String)}, or other generic - * data sinks created with {@link DataSet#output(org.apache.flink.api.common.io.OutputFormat)}. - * - *

The program execution will be logged and displayed with the given job name. - * - * @return The result of the job execution, containing elapsed time and accumulators. - * @throws Exception Thrown, if the program executions fails. - */ - public abstract JobExecutionResult execute(String jobName) throws Exception; - - /** - * Creates the plan with which the system will execute the program, and returns it as - * a String using a JSON representation of the execution data flow graph. - * Note that this needs to be called, before the plan is executed. - * - * @return The execution plan of the program, as a JSON String. - * @throws Exception Thrown, if the compiler could not be instantiated, or the master could not - * be contacted to retrieve information relevant to the execution planning. - */ - public abstract String getExecutionPlan() throws Exception; - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - */ - public void registerCachedFile(String filePath, String name){ - registerCachedFile(filePath, name, false); - } - - /** - * Registers a file at the distributed cache under the given name. The file will be accessible - * from any user-defined function in the (distributed) runtime under a local path. Files - * may be local files (which will be distributed via BlobServer), or files in a distributed file system. - * The runtime will copy the files temporarily to a local cache, if needed. - * - *

The {@link org.apache.flink.api.common.functions.RuntimeContext} can be obtained inside UDFs via - * {@link org.apache.flink.api.common.functions.RichFunction#getRuntimeContext()} and provides access - * {@link org.apache.flink.api.common.cache.DistributedCache} via - * {@link org.apache.flink.api.common.functions.RuntimeContext#getDistributedCache()}. - * - * @param filePath The path of the file, as a URI (e.g. "file:///some/path" or "hdfs://host:port/and/path") - * @param name The name under which the file is registered. - * @param executable flag indicating whether the file should be executable - */ - public void registerCachedFile(String filePath, String name, boolean executable){ - this.cacheFile.add(new Tuple2<>(name, new DistributedCacheEntry(filePath, executable))); - } - - /** - * Registers all files that were registered at this execution environment's cache registry of the - * given plan's cache registry. - * - * @param p The plan to register files at. - * @throws IOException Thrown if checks for existence and sanity fail. - */ - protected void registerCachedFilesWithPlan(Plan p) throws IOException { - for (Tuple2 entry : cacheFile) { - p.registerCachedFile(entry.f0, entry.f1); - } - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * This automatically starts a new stage of execution. - * - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan() { - return createProgramPlan(null); - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * This automatically starts a new stage of execution. - * - * @param jobName The name attached to the plan (displayed in logs and monitoring). - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan(String jobName) { - return createProgramPlan(jobName, true); - } - - /** - * Creates the program's {@link Plan}. The plan is a description of all data sources, data sinks, - * and operations and how they interact, as an isolated unit that can be executed with a - * {@link org.apache.flink.api.common.PlanExecutor}. Obtaining a plan and starting it with an - * executor is an alternative way to run a program and is only possible if the program consists - * only of distributed operations. - * - * @param jobName The name attached to the plan (displayed in logs and monitoring). - * @param clearSinks Whether or not to start a new stage of execution. - * @return The program's plan. - */ - @Internal - public Plan createProgramPlan(String jobName, boolean clearSinks) { - if (this.sinks.isEmpty()) { - if (wasExecuted) { - throw new RuntimeException("No new data sinks have been defined since the " + - "last execution. The last execution refers to the latest call to " + - "'execute()', 'count()', 'collect()', or 'print()'."); - } else { - throw new RuntimeException("No data sinks have been created yet. " + - "A program needs at least one sink that consumes data. " + - "Examples are writing the data set or printing it."); - } - } - - if (jobName == null) { - jobName = getDefaultName(); - } - - OperatorTranslation translator = new OperatorTranslation(); - Plan plan = translator.translateToPlan(this.sinks, jobName); - - if (getParallelism() > 0) { - plan.setDefaultParallelism(getParallelism()); - } - plan.setExecutionConfig(getConfig()); - - // Check plan for GenericTypeInfo's and register the types at the serializers. - if (!config.isAutoTypeRegistrationDisabled()) { - plan.accept(new Visitor>() { - - private final Set> registeredTypes = new HashSet<>(); - private final Set> visitedOperators = new HashSet<>(); - - @Override - public boolean preVisit(org.apache.flink.api.common.operators.Operator visitable) { - if (!visitedOperators.add(visitable)) { - return false; - } - OperatorInformation opInfo = visitable.getOperatorInfo(); - Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes); - return true; - } - - @Override - public void postVisit(org.apache.flink.api.common.operators.Operator visitable) {} - }); - } - - try { - registerCachedFilesWithPlan(plan); - } catch (Exception e) { - throw new RuntimeException("Error while registering cached files: " + e.getMessage(), e); - } - - // clear all the sinks such that the next execution does not redo everything - if (clearSinks) { - this.sinks.clear(); - wasExecuted = true; - } - - // All types are registered now. Print information. - int registeredTypes = config.getRegisteredKryoTypes().size() + - config.getRegisteredPojoTypes().size() + - config.getRegisteredTypesWithKryoSerializerClasses().size() + - config.getRegisteredTypesWithKryoSerializers().size(); - int defaultKryoSerializers = config.getDefaultKryoSerializers().size() + - config.getDefaultKryoSerializerClasses().size(); - LOG.info("The job has {} registered types and {} default Kryo serializers", registeredTypes, defaultKryoSerializers); - - if (config.isForceKryoEnabled() && config.isForceAvroEnabled()) { - LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer"); - } - if (config.isForceKryoEnabled()) { - LOG.info("Using KryoSerializer for serializing POJOs"); - } - if (config.isForceAvroEnabled()) { - LOG.info("Using AvroSerializer for serializing POJOs"); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Registered Kryo types: {}", config.getRegisteredKryoTypes().toString()); - LOG.debug("Registered Kryo with Serializers types: {}", config.getRegisteredTypesWithKryoSerializers().entrySet().toString()); - LOG.debug("Registered Kryo with Serializer Classes types: {}", config.getRegisteredTypesWithKryoSerializerClasses().entrySet().toString()); - LOG.debug("Registered Kryo default Serializers: {}", config.getDefaultKryoSerializers().entrySet().toString()); - LOG.debug("Registered Kryo default Serializers Classes {}", config.getDefaultKryoSerializerClasses().entrySet().toString()); - LOG.debug("Registered POJO types: {}", config.getRegisteredPojoTypes().toString()); - - // print information about static code analysis - LOG.debug("Static code analysis mode: {}", config.getCodeAnalysisMode()); - } - - return plan; - } - - /** - * Adds the given sink to this environment. Only sinks that have been added will be executed once - * the {@link #execute()} or {@link #execute(String)} method is called. - * - * @param sink The sink to add for execution. - */ - @Internal - void registerDataSink(DataSink sink) { - this.sinks.add(sink); - } - - /** - * Gets a default job name, based on the timestamp when this method is invoked. - * - * @return A default job name. - */ - private static String getDefaultName() { - return "Flink Java Job at " + Calendar.getInstance().getTime(); - } - - // -------------------------------------------------------------------------------------------- - // Instantiation of Execution Contexts - // -------------------------------------------------------------------------------------------- - - /** - * Creates an execution environment that represents the context in which the program is currently executed. - * If the program is invoked standalone, this method returns a local execution environment, as returned by - * {@link #createLocalEnvironment()}. If the program is invoked from within the command line client to be - * submitted to a cluster, this method returns the execution environment of this cluster. - * - * @return The execution environment of the context in which the program is executed. - */ - public static ExecutionEnvironment getExecutionEnvironment() { - return contextEnvironmentFactory == null ? - createLocalEnvironment() : contextEnvironmentFactory.get().createExecutionEnvironment(); - } - - /** - * Creates a {@link CollectionEnvironment} that uses Java Collections underneath. This will execute in a - * single thread in the current JVM. It is very fast but will fail if the data does not fit into - * memory. parallelism will always be 1. This is useful during implementation and for debugging. - * @return A Collection Environment - */ - @PublicEvolving - public static CollectionEnvironment createCollectionsEnvironment(){ - CollectionEnvironment ce = new CollectionEnvironment(); - ce.setParallelism(1); - return ce; - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. The default - * parallelism of the local environment is the number of hardware contexts (CPU cores / threads), - * unless it was specified differently by {@link #setDefaultLocalParallelism(int)}. - * - * @return A local execution environment. - */ - public static LocalEnvironment createLocalEnvironment() { - return createLocalEnvironment(defaultLocalDop); - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. It will use the - * parallelism specified in the parameter. - * - * @param parallelism The parallelism for the local environment. - * @return A local execution environment with the specified parallelism. - */ - public static LocalEnvironment createLocalEnvironment(int parallelism) { - return createLocalEnvironment(new Configuration(), parallelism); - } - - /** - * Creates a {@link LocalEnvironment}. The local execution environment will run the program in a - * multi-threaded fashion in the same JVM as the environment was created in. It will use the - * parallelism specified in the parameter. - * - * @param customConfiguration Pass a custom configuration to the LocalEnvironment. - * @return A local execution environment with the specified parallelism. - */ - public static LocalEnvironment createLocalEnvironment(Configuration customConfiguration) { - return createLocalEnvironment(customConfiguration, -1); - } - - /** - * Creates a {@link LocalEnvironment} for local program execution that also starts the - * web monitoring UI. - * - *

The local execution environment will run the program in a multi-threaded fashion in - * the same JVM as the environment was created in. It will use the parallelism specified in the - * parameter. - * - *

If the configuration key 'rest.port' was set in the configuration, that particular - * port will be used for the web UI. Otherwise, the default port (8081) will be used. - */ - @PublicEvolving - public static ExecutionEnvironment createLocalEnvironmentWithWebUI(Configuration conf) { - checkNotNull(conf, "conf"); - - conf.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, true); - - if (!conf.contains(RestOptions.PORT)) { - // explicitly set this option so that it's not set to 0 later - conf.setInteger(RestOptions.PORT, RestOptions.PORT.defaultValue()); - } - - return createLocalEnvironment(conf, -1); - } - - /** - * Creates a {@link LocalEnvironment} which is used for executing Flink jobs. - * - * @param configuration to start the {@link LocalEnvironment} with - * @param defaultParallelism to initialize the {@link LocalEnvironment} with - * @return {@link LocalEnvironment} - */ - private static LocalEnvironment createLocalEnvironment(Configuration configuration, int defaultParallelism) { - final LocalEnvironment localEnvironment = new LocalEnvironment(configuration); - - if (defaultParallelism > 0) { - localEnvironment.setParallelism(defaultParallelism); - } - - return localEnvironment; - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the cluster's default parallelism, unless the parallelism is - * set explicitly via {@link ExecutionEnvironment#setParallelism(int)}. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment(String host, int port, String... jarFiles) { - return new RemoteEnvironment(host, port, jarFiles); - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The custom configuration file is used to configure Akka specific configuration parameters - * for the Client only; Program parallelism can be set via {@link ExecutionEnvironment#setParallelism(int)}. - * - *

Cluster configuration has to be done in the remotely running Flink instance. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param clientConfiguration Configuration used by the client that connects to the cluster. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment( - String host, int port, Configuration clientConfiguration, String... jarFiles) { - return new RemoteEnvironment(host, port, clientConfiguration, jarFiles, null); - } - - /** - * Creates a {@link RemoteEnvironment}. The remote environment sends (parts of) the program - * to a cluster for execution. Note that all file paths used in the program must be accessible from the - * cluster. The execution will use the specified parallelism. - * - * @param host The host name or address of the master (JobManager), where the program should be executed. - * @param port The port of the master (JobManager), where the program should be executed. - * @param parallelism The parallelism to use during the execution. - * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses - * user-defined functions, user-defined input formats, or any libraries, those must be - * provided in the JAR files. - * @return A remote environment that executes the program on a cluster. - */ - public static ExecutionEnvironment createRemoteEnvironment(String host, int port, int parallelism, String... jarFiles) { - RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles); - rec.setParallelism(parallelism); - return rec; - } - - // -------------------------------------------------------------------------------------------- - // Default parallelism for local execution - // -------------------------------------------------------------------------------------------- - - /** - * Gets the default parallelism that will be used for the local execution environment created by - * {@link #createLocalEnvironment()}. - * - * @return The default local parallelism - */ - public static int getDefaultLocalParallelism() { - return defaultLocalDop; - } - - /** - * Sets the default parallelism that will be used for the local execution environment created by - * {@link #createLocalEnvironment()}. - * - * @param parallelism The parallelism to use as the default local parallelism. - */ - public static void setDefaultLocalParallelism(int parallelism) { - defaultLocalDop = parallelism; - } - - // -------------------------------------------------------------------------------------------- - // Methods to control the context environment and creation of explicit environments other - // than the context environment - // -------------------------------------------------------------------------------------------- - - /** - * Sets a context environment factory, that creates the context environment for running programs - * with pre-configured environments. Examples are running programs from the command line, and - * running programs in the Scala shell. - * - *

When the context environment factory is set, no other environments can be explicitly used. - * - * @param ctx The context environment factory. - */ - protected static void initializeContextEnvironment(ExecutionEnvironmentFactory ctx) { - contextEnvironmentFactory.set(Preconditions.checkNotNull(ctx)); - } - - /** - * Un-sets the context environment factory. After this method is called, the call to - * {@link #getExecutionEnvironment()} will again return a default local execution environment, and - * it is possible to explicitly instantiate the LocalEnvironment and the RemoteEnvironment. - */ - protected static void resetContextEnvironment() { - contextEnvironmentFactory.remove(); - } - - /** - * Checks whether it is currently permitted to explicitly instantiate a LocalEnvironment - * or a RemoteEnvironment. - * - * @return True, if it is possible to explicitly instantiate a LocalEnvironment or a - * RemoteEnvironment, false otherwise. - */ - @Internal - public static boolean areExplicitEnvironmentsAllowed() { - return contextEnvironmentFactory.get() == null; - } -} diff --git a/flinkx-core/src/test/java/com/dtstack/flinkx/util/ZkHelperTest.java b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ZkHelperTest.java new file mode 100644 index 0000000000..a32ed68c30 --- /dev/null +++ b/flinkx-core/src/test/java/com/dtstack/flinkx/util/ZkHelperTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.util; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.test.TestingServer; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooKeeper; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +public class ZkHelperTest { + + private static TestingServer server; + + private static CuratorFramework client; + + private static ZooKeeper zooKeeper; + + @Before + public void createZkServer() throws Exception { + server = new TestingServer(2181, true); + server.start(); + client = CuratorFrameworkFactory.builder() + .connectString("localhost:2181") + .connectionTimeoutMs(5000) + .retryPolicy(new ExponentialBackoffRetry(1000,3)) + .build(); + client.start(); + client.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath("/hbase/table/test1", "init".getBytes()); + client.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath("/hbase/table/test2", "init".getBytes()); + + zooKeeper = ZkHelper.createZkClient("localhost:2181", ZkHelper.DEFAULT_TIMEOUT); + } + + @Test + public void testCreateSingleZkClient(){ + Assert.assertNotNull(zooKeeper); + } + + @Test + public void testGetChildren() { + List list = ZkHelper.getChildren(zooKeeper, "/hbase/table"); + Assert.assertNotNull(list); + Assert.assertEquals(2, list.size()); + } + + @Test + public void testGetCreateTime(){ + Assert.assertNotEquals(ZkHelper.getCreateTime(zooKeeper, "/hbase/table/test1"), 0L); + } + + + @After + public void closeZkServer() throws IOException { + ZkHelper.closeZooKeeper(zooKeeper); + client.close(); + server.close(); + } + +} diff --git a/flinkx-db2/flinkx-db2-reader/pom.xml b/flinkx-db2/flinkx-db2-reader/pom.xml index 37e16d9a6c..fe6415dab9 100644 --- a/flinkx-db2/flinkx-db2-reader/pom.xml +++ b/flinkx-db2/flinkx-db2-reader/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-db2/flinkx-db2-writer/pom.xml b/flinkx-db2/flinkx-db2-writer/pom.xml index 3ed4b0b54a..a01e3ee391 100644 --- a/flinkx-db2/flinkx-db2-writer/pom.xml +++ b/flinkx-db2/flinkx-db2-writer/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-dm/flinkx-dm-reader/pom.xml b/flinkx-dm/flinkx-dm-reader/pom.xml index 16bc404e6e..2aec50182f 100644 --- a/flinkx-dm/flinkx-dm-reader/pom.xml +++ b/flinkx-dm/flinkx-dm-reader/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-dm/flinkx-dm-writer/pom.xml b/flinkx-dm/flinkx-dm-writer/pom.xml index 7830ae0871..d1061ba007 100644 --- a/flinkx-dm/flinkx-dm-writer/pom.xml +++ b/flinkx-dm/flinkx-dm-writer/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-emqx/flinkx-emqx-reader/pom.xml b/flinkx-emqx/flinkx-emqx-reader/pom.xml index ff644d64c0..1db1b443de 100644 --- a/flinkx-emqx/flinkx-emqx-reader/pom.xml +++ b/flinkx-emqx/flinkx-emqx-reader/pom.xml @@ -86,6 +86,7 @@ + diff --git a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java index 7e92921dcc..6dc2bea7d6 100644 --- a/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java +++ b/flinkx-emqx/flinkx-emqx-reader/src/main/java/com/dtstack/flinkx/emqx/format/EmqxInputFormat.java @@ -19,7 +19,7 @@ import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.decoder.JsonDecoder; -import com.dtstack.flinkx.decoder.PlainDecoder; +import com.dtstack.flinkx.decoder.TextDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang3.StringUtils; @@ -70,7 +70,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if (DEFAULT_CODEC.equals(codec)) { decode = new JsonDecoder(); } else { - decode = new PlainDecoder(); + decode = new TextDecoder(); } try { diff --git a/flinkx-emqx/flinkx-emqx-writer/pom.xml b/flinkx-emqx/flinkx-emqx-writer/pom.xml index d0be154b1a..68ddb5ac70 100644 --- a/flinkx-emqx/flinkx-emqx-writer/pom.xml +++ b/flinkx-emqx/flinkx-emqx-writer/pom.xml @@ -86,6 +86,7 @@ + diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java index 52a311b76e..03d0adfc77 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/writer/EmqxWriter.java @@ -66,6 +66,9 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setPassword(password); builder.setCleanSession(isCleanSession); builder.setQos(qos); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); return createOutput(dataSet, builder.finish()); } } diff --git a/flinkx-es/flinkx-es-reader/pom.xml b/flinkx-es/flinkx-es-reader/pom.xml index 374d90178e..2fc873788c 100644 --- a/flinkx-es/flinkx-es-reader/pom.xml +++ b/flinkx-es/flinkx-es-reader/pom.xml @@ -90,6 +90,7 @@ + diff --git a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java index 1c54a5e115..7221549e91 100644 --- a/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java +++ b/flinkx-es/flinkx-es-reader/src/main/java/com/dtstack/flinkx/es/reader/EsInputFormat.java @@ -88,8 +88,6 @@ public class EsInputFormat extends BaseRichInputFormat { @Override public void openInputFormat() throws IOException { super.openInputFormat(); - - } @Override diff --git a/flinkx-es/flinkx-es-writer/pom.xml b/flinkx-es/flinkx-es-writer/pom.xml index 8877e10e13..3004e63002 100644 --- a/flinkx-es/flinkx-es-writer/pom.xml +++ b/flinkx-es/flinkx-es-writer/pom.xml @@ -91,6 +91,7 @@ + diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java index 2cad866aba..60d2fd8861 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/EProtocol.java @@ -14,15 +14,5 @@ public enum EProtocol { /** * 安全的FTP协议 */ - SFTP; - - public static EProtocol getByName(String name) { - for (EProtocol value : EProtocol.values()) { - if (value.name().equalsIgnoreCase(name)) { - return value; - } - } - - return SFTP; - } + SFTP } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java index afc45b081a..886ae88e99 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpConfig.java @@ -20,7 +20,6 @@ package com.dtstack.flinkx.ftp; import org.codehaus.jackson.annotate.JsonIgnoreProperties; -import org.codehaus.jackson.annotate.JsonProperty; import java.io.Serializable; @@ -43,11 +42,11 @@ public class FtpConfig implements Serializable { public String fieldDelimiter = FtpConfigConstants.DEFAULT_FIELD_DELIMITER; - public String path; + public String path = "/"; public String encoding = "UTF-8"; - public String connectPattern = FtpConfigConstants.DEFAULT_FTP_CONNECT_PATTERN; + public String connectPattern = FtpConfigConstants.STANDARD_FTP_PROTOCOL; public String host; diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index 6fda60da00..aa8a3d083a 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.ftp; import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; @@ -28,6 +29,8 @@ import org.apache.commons.net.ftp.FTPReply; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -83,6 +86,7 @@ public void loginFtpServer(FtpConfig ftpConfig) { //设置命令传输编码 String fileEncoding = System.getProperty(ConstantValue.SYSTEM_PROPERTIES_KEY_FILE_ENCODING); ftpClient.setControlEncoding(fileEncoding); + ftpClient.setListHiddenFiles(true); } catch (Exception e) { throw new RuntimeException(e); } @@ -106,7 +110,13 @@ public boolean isDirExist(String directoryPath) { String originDir = null; try { originDir = ftpClient.printWorkingDirectory(); - return ftpClient.changeWorkingDirectory(new String(directoryPath.getBytes(StandardCharsets.UTF_8), FTP.DEFAULT_CONTROL_ENCODING)); + ftpClient.enterLocalPassiveMode(); + FTPFile[] ftpFiles = ftpClient.listFiles(new String(directoryPath.getBytes(StandardCharsets.UTF_8), FTP.DEFAULT_CONTROL_ENCODING)); + if(ftpFiles.length == 0){ + throw new FileNotFoundException("file or path is not exist, please check the path or the permissions of account, path = " + directoryPath); + }else { + return FTPReply.isPositiveCompletion(ftpClient.cwd(directoryPath)); + } } catch (IOException e) { String message = String.format("进入目录:[%s]时发生I/O异常,请确认与ftp服务器的连接正常", directoryPath); LOG.error(message); @@ -124,25 +134,13 @@ public boolean isDirExist(String directoryPath) { @Override public boolean isFileExist(String filePath) { - boolean isExitFlag = false; - try { - ftpClient.enterLocalPassiveMode(); - FTPFile[] ftpFiles = ftpClient.listFiles(new String(filePath.getBytes(StandardCharsets.UTF_8),FTP.DEFAULT_CONTROL_ENCODING)); - if (ftpFiles.length == 1 && ftpFiles[0].isFile()) { - isExitFlag = true; - } - } catch (IOException e) { - String message = String.format("获取文件:[%s] 属性时发生I/O异常,请确认与ftp服务器的连接正常", filePath); - LOG.error(message); - throw new RuntimeException(e); - } - return isExitFlag; + return !isDirExist(filePath); } - @Override public List getFiles(String path) { List sources = new ArrayList<>(); + ftpClient.enterLocalPassiveMode(); if(isFileExist(path)) { sources.add(path); return sources; @@ -150,10 +148,13 @@ public List getFiles(String path) { path = path + SP; } try { - ftpClient.enterLocalPassiveMode(); FTPFile[] ftpFiles = ftpClient.listFiles(new String(path.getBytes(StandardCharsets.UTF_8),FTP.DEFAULT_CONTROL_ENCODING)); if(ftpFiles != null) { for(FTPFile ftpFile : ftpFiles) { + // .和..是特殊文件 + if(StringUtils.endsWith(ftpFile.getName(), ConstantValue.POINT_SYMBOL) || StringUtils.endsWith(ftpFile.getName(), ConstantValue.TWO_POINT_SYMBOL)){ + continue; + } sources.addAll(getFiles(path + ftpFile.getName(), ftpFile)); } } @@ -166,7 +167,6 @@ public List getFiles(String path) { /** * 递归获取指定路径下的所有文件(暂无过滤) - * isDirExist()、isFileExist()方法在Windows系统下判断有误 * @param path * @param file * @return @@ -182,6 +182,9 @@ private List getFiles(String path, FTPFile file)throws IOException { FTPFile[] ftpFiles = ftpClient.listFiles(new String(path.getBytes(StandardCharsets.UTF_8),FTP.DEFAULT_CONTROL_ENCODING)); if(ftpFiles != null) { for(FTPFile ftpFile : ftpFiles) { + if(StringUtils.endsWith(ftpFile.getName(), ConstantValue.POINT_SYMBOL) || StringUtils.endsWith(ftpFile.getName(), ConstantValue.TWO_POINT_SYMBOL)){ + continue; + } sources.addAll(getFiles(path + ftpFile.getName(), ftpFile)); } } @@ -236,7 +239,7 @@ public OutputStream getOutputStream(String filePath) { try { this.printWorkingDirectory(); String parentDir = filePath.substring(0, - StringUtils.lastIndexOf(filePath, IOUtils.DIR_SEPARATOR)); + StringUtils.lastIndexOf(filePath, IOUtils.DIR_SEPARATOR_UNIX)); this.ftpClient.changeWorkingDirectory(parentDir); this.printWorkingDirectory(); OutputStream writeOutputStream = this.ftpClient @@ -282,6 +285,9 @@ public void deleteAllFilesInDir(String dir, List exclude) { if(CollectionUtils.isNotEmpty(exclude) && exclude.contains(ftpFile.getName())){ continue; } + if(StringUtils.endsWith(ftpFile.getName(), ConstantValue.POINT_SYMBOL) || StringUtils.endsWith(ftpFile.getName(), ConstantValue.TWO_POINT_SYMBOL)){ + continue; + } deleteAllFilesInDir(dir + ftpFile.getName(), exclude); } } @@ -328,6 +334,9 @@ public List listDirs(String path) { FTPFile[] ftpFiles = ftpClient.listFiles(new String(path.getBytes(StandardCharsets.UTF_8),FTP.DEFAULT_CONTROL_ENCODING)); if(ftpFiles != null) { for(FTPFile ftpFile : ftpFiles) { + if(StringUtils.endsWith(ftpFile.getName(), ConstantValue.POINT_SYMBOL) || StringUtils.endsWith(ftpFile.getName(), ConstantValue.TWO_POINT_SYMBOL)){ + continue; + } sources.add(path + ftpFile.getName()); } } @@ -344,4 +353,17 @@ public List listDirs(String path) { public void rename(String oldPath, String newPath) throws IOException { ftpClient.rename(oldPath, newPath); } + + @Override + public void completePendingCommand() throws IOException { + try { + // throw exception when return false + if(!ftpClient.completePendingCommand()){ + throw new IOException("I/O error occurs while sending or receiving data"); + }; + } catch (IOException e) { + LOG.error("I/O error occurs while sending or receiving data"); + throw new IOException(ExceptionUtil.getErrorMessage(e)); + } + } } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java index 16faf8e6d8..cc2fbc8076 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/IFtpHandler.java @@ -116,4 +116,10 @@ public interface IFtpHandler { * @throws Exception 可能会出现文件不存在,连接异常等 */ void rename(String oldPath, String newPath) throws Exception; + + /** + * 关闭ftp输入流 + * @throws IOException 文件句柄操作异常 + */ + void completePendingCommand() throws IOException; } diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java index 702dcf3c65..4599a4ef38 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/SftpHandler.java @@ -30,6 +30,7 @@ import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; @@ -379,4 +380,12 @@ public boolean mkDirSingleHierarchy(String directoryPath) throws SftpException { public void rename(String oldPath, String newPath) throws SftpException { channelSftp.rename(oldPath, newPath); } + + /** + * 仅ftp输入流需要显示关闭 + */ + @Override + public void completePendingCommand() { + + } } diff --git a/flinkx-ftp/flinkx-ftp-reader/pom.xml b/flinkx-ftp/flinkx-ftp-reader/pom.xml index 6296d7cf1a..29e56eb80f 100644 --- a/flinkx-ftp/flinkx-ftp-reader/pom.xml +++ b/flinkx-ftp/flinkx-ftp-reader/pom.xml @@ -104,6 +104,7 @@ under the License. + diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index 364dc2a2e9..f90c860ef1 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -26,7 +26,6 @@ import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.StringUtil; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -44,8 +43,6 @@ public class FtpInputFormat extends BaseRichInputFormat { protected FtpConfig ftpConfig; - protected String charsetName = "utf-8"; - protected List metaColumns; private transient FtpSeqBufferedReader br; @@ -103,7 +100,7 @@ public void openInternal(InputSplit split) throws IOException { br = new FtpSeqBufferedReader(ftpHandler,paths.iterator()); br.setFromLine(0); } - br.setCharsetName(charsetName); + br.setFileEncoding(ftpConfig.getEncoding()); } @Override diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java index 34755a4109..49badaec33 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java @@ -47,7 +47,7 @@ public class FtpSeqBufferedReader { private BufferedReader br; - private String charsetName = "utf-8"; + private String fileEncoding; public FtpSeqBufferedReader(IFtpHandler ftpHandler, Iterator iter) { this.ftpHandler = ftpHandler; @@ -77,10 +77,10 @@ private void nextStream() throws IOException{ String file = iter.next(); InputStream in = ftpHandler.getInputStream(file); if (in == null) { - throw new NullPointerException(); + throw new RuntimeException(String.format("can not get inputStream for file [%s], please check file read and write permissions", file)); } - br = new BufferedReader(new InputStreamReader(in, charsetName)); + br = new BufferedReader(new InputStreamReader(in, fileEncoding)); for (int i = 0; i < fromLine; i++) { String skipLine = br.readLine(); @@ -106,7 +106,7 @@ public void setFromLine(int fromLine) { this.fromLine = fromLine; } - public void setCharsetName(String charsetName) { - this.charsetName = charsetName; + public void setFileEncoding(String fileEncoding) { + this.fileEncoding = fileEncoding; } } diff --git a/flinkx-ftp/flinkx-ftp-writer/pom.xml b/flinkx-ftp/flinkx-ftp-writer/pom.xml index 68cb5ceeeb..beb6d76b2a 100644 --- a/flinkx-ftp/flinkx-ftp-writer/pom.xml +++ b/flinkx-ftp/flinkx-ftp-writer/pom.xml @@ -105,6 +105,7 @@ under the License. + diff --git a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java index ce5fd20267..c1b30da63f 100644 --- a/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java +++ b/flinkx-ftp/flinkx-ftp-writer/src/main/java/com/dtstack/flinkx/ftp/writer/FtpOutputFormat.java @@ -25,22 +25,21 @@ import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.SysUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.flink.types.Row; import java.io.BufferedWriter; import java.io.IOException; import java.io.OutputStream; -import java.util.Collections; import java.io.OutputStreamWriter; import java.io.UnsupportedEncodingException; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; import java.util.function.Predicate; /** @@ -62,17 +61,33 @@ public class FtpOutputFormat extends BaseFileOutputFormat { private transient IFtpHandler ftpHandler; - private static final int FILE_NAME_PART_SIZE = 3; + private transient BufferedWriter writer; - private static final String DOT = "."; + private transient OutputStream os; private static final String FILE_SUFFIX = ".csv"; private static final String OVERWRITE_MODE = "overwrite"; - private transient BufferedWriter writer; + + private static final int FILE_NAME_PART_SIZE = 3; + + /** + * 避免ftp没有数据时阻塞 + * @param taskNumber 通道索引 + * @param numTasks 通道数量 + * @throws IOException IO异常 + */ + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + initFileIndex(); + initPath(); + openSource(); + actionBeforeWriteData(); + } @Override protected void openSource() throws IOException { + writeMode = ftpConfig.writeMode; ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); ftpHandler.loginFtpServer(ftpConfig); } @@ -133,13 +148,14 @@ protected void nextBlock(){ if (writer != null){ return; } - String path = tmpPath + SP + currentBlockFileName; - try { - writer = new BufferedWriter(new OutputStreamWriter(ftpHandler.getOutputStream(path), ftpConfig.getEncoding())); - } catch (UnsupportedEncodingException e) { - LOG.error("exception when create BufferedWriter, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); - throw new RuntimeException(e); + + os = ftpHandler.getOutputStream(tmpPath + SP + currentBlockFileName); + try{ + writer = new BufferedWriter(new OutputStreamWriter(os, ftpConfig.getEncoding())); + }catch (UnsupportedEncodingException e){ + LOG.error(ExceptionUtils.getMessage(e)); } + blockIndex++; } @@ -174,35 +190,29 @@ public void writeSingleRecordToFile(Row row) throws WriteRecordException { String line = StringUtil.row2string(row, columnTypes, ftpConfig.getFieldDelimiter()); this.writer.write(line); this.writer.write(NEWLINE); - + rowsOfCurrentBlock++; if(restoreConfig.isRestore()){ lastRow = row; - rowsOfCurrentBlock++; } - } catch(Exception e) { - LOG.error("error happened when write single record to file, row = {}, columnTypes = {}, e = {}", row, GsonUtil.GSON.toJson(columnTypes), ExceptionUtil.getErrorMessage(e)); - throw new WriteRecordException(e.getMessage(), e); + } catch(Exception ex) { + throw new WriteRecordException(ex.getMessage(), ex); } } + /** + * 直接创建目录会失败,增加等待和重试 + * @throws IOException 创建目录异常 + */ @Override - protected void createFinishedTag() { - LOG.info("SubTask [{}] finished, create dir {}", taskNumber, finishedPath); - String path = outputFilePath + SP + FINISHED_SUBDIR; - if(taskNumber == 0){ - ftpHandler.mkDirRecursive(path); - } - final int maxRetryTime = 15; - int i = 0; - try { - while(!(ftpHandler.isDirExist(path) || i > maxRetryTime)){ - i++; - TimeUnit.MILLISECONDS.sleep(10); - } + protected void createFinishedTag() throws IOException { + LOG.info("Subtask [{}] finished, create dir {}", taskNumber, finishedPath); + try{ + RetryUtil.executeWithRetry(() -> {ftpHandler.mkDirRecursive(finishedPath); + return null; + }, 3, 5000, false); }catch (Exception e){ - LOG.error("exception when createFinishedTag, path = {}, e = {}", path, ExceptionUtil.getErrorMessage(e)); - } - ftpHandler.mkDirRecursive(finishedPath); + throw new IOException(e); + }; } @Override @@ -261,7 +271,7 @@ protected void waitForAllTasksToFinish(){ if (i == maxRetryTime) { ftpHandler.deleteAllFilesInDir(finishedPath, null); - throw new RuntimeException("timeout when gathering finish tags for each subTasks"); + throw new RuntimeException("timeout when gathering finish tags for each subtasks"); } } @@ -313,11 +323,19 @@ protected void closeSource() throws IOException { writer.flush(); writer.close(); writer = null; + os.close(); + os = null; + try { + //avoid Failure of FtpClient operating + this.ftpHandler.completePendingCommand(); + }catch (Exception e) { + throw new IOException(ExceptionUtil.getErrorMessage(e)); + } } } @Override - protected void clearTemporaryDataFiles() { + protected void clearTemporaryDataFiles() throws IOException { ftpHandler.deleteAllFilesInDir(tmpPath, null); LOG.info("Delete .data dir:{}", tmpPath); @@ -330,12 +348,6 @@ public void flushDataInternal() throws IOException { closeSource(); } - @Override - public void closeInternal() throws IOException { - closeSource(); - super.closeInternal(); - } - @Override public float getDeviation() { return 1.0F; diff --git a/flinkx-gbase/flinkx-gbase-reader/pom.xml b/flinkx-gbase/flinkx-gbase-reader/pom.xml index cb775310ff..bb90458df0 100644 --- a/flinkx-gbase/flinkx-gbase-reader/pom.xml +++ b/flinkx-gbase/flinkx-gbase-reader/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java index be3fed3c9d..87470fbfbe 100644 --- a/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java +++ b/flinkx-gbase/flinkx-gbase-reader/src/main/java/com/dtstack/flinkx/gbase/format/GbaseInputFormat.java @@ -18,11 +18,6 @@ package com.dtstack.flinkx.gbase.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; -import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; -import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.util.ClassUtil; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import java.io.IOException; diff --git a/flinkx-gbase/flinkx-gbase-writer/pom.xml b/flinkx-gbase/flinkx-gbase-writer/pom.xml index 8edfbde076..077c8947e4 100644 --- a/flinkx-gbase/flinkx-gbase-writer/pom.xml +++ b/flinkx-gbase/flinkx-gbase-writer/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-greenplum/flinkx-greenplum-reader/pom.xml b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml index a95b84d109..72a28b084e 100644 --- a/flinkx-greenplum/flinkx-greenplum-reader/pom.xml +++ b/flinkx-greenplum/flinkx-greenplum-reader/pom.xml @@ -98,6 +98,7 @@ + diff --git a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java index d7f0d61ba6..efe8a007e1 100644 --- a/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java +++ b/flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseConfigConstants.java @@ -28,16 +28,10 @@ public class HbaseConfigConstants { public static final int DEFAULT_SCAN_CACHE_SIZE = 256; - public static final int DEFAULT_SCAN_BATCH_SIZE = 100; - public static final int MAX_SCAN_CACHE_SIZE = 1000; public static final int MIN_SCAN_CACHE_SIZE = 1; - public static final int MAX_SCAN_BATCH_SIZE = 100; - - public static final int MIN_SCAN_BATCH_SIZE = 1; - public static final String DEFAULT_ENCODING = "UTF-8"; public static final String DEFAULT_DATA_FORMAT = "yyyy-MM-dd HH:mm:ss"; diff --git a/flinkx-hbase/flinkx-hbase-reader/pom.xml b/flinkx-hbase/flinkx-hbase-reader/pom.xml index fc79c7d2f6..49429b3834 100644 --- a/flinkx-hbase/flinkx-hbase-reader/pom.xml +++ b/flinkx-hbase/flinkx-hbase-reader/pom.xml @@ -12,6 +12,11 @@ flinkx-hbase-reader + + com.google.guava + guava + 12.0.1 + com.dtstack.flinkx flinkx-hbase-core @@ -68,11 +73,11 @@ com.google.common - shade.core.com.google.common + shade.hbase.com.google.common com.google.thirdparty - shade.core.com.google.thirdparty + shade.hbase.com.google.thirdparty @@ -98,6 +103,7 @@ + diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index f17383ca87..5e9c274864 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -69,8 +69,10 @@ public class HbaseInputFormat extends BaseRichInputFormat { protected List columnTypes; protected boolean isBinaryRowkey; protected String encoding; + /** + * 客户端每次 rpc fetch 的行数 + */ protected int scanCacheSize; - protected int scanBatchSize; private transient Connection connection; private transient Scan scan; private transient Table table; @@ -238,7 +240,6 @@ public void openInternal(InputSplit inputSplit) throws IOException { scan.setStartRow(startRow); scan.setStopRow(stopRow); scan.setCaching(scanCacheSize); - scan.setBatch(scanBatchSize); resultScanner = table.getScanner(scan); } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java index cfae99b025..b12db12207 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormatBuilder.java @@ -82,10 +82,6 @@ public void setScanCacheSize(int scanCacheSize) { format.scanCacheSize = scanCacheSize; } - public void setScanBatchSize(int scanBatchSize) { - format.scanBatchSize = scanBatchSize; - } - @Override protected void checkFormat() { Preconditions.checkNotNull(format.columnTypes); @@ -96,9 +92,6 @@ protected void checkFormat() { Preconditions.checkArgument(format.scanCacheSize <= HbaseConfigConstants.MAX_SCAN_CACHE_SIZE && format.scanCacheSize >= HbaseConfigConstants.MIN_SCAN_CACHE_SIZE, "scanCacheSize should be between " + HbaseConfigConstants.MIN_SCAN_CACHE_SIZE + " and " + HbaseConfigConstants.MAX_SCAN_CACHE_SIZE); - Preconditions.checkArgument(format.scanBatchSize <= HbaseConfigConstants.MAX_SCAN_BATCH_SIZE && format.scanBatchSize >= HbaseConfigConstants.MIN_SCAN_BATCH_SIZE, - "scanBatchSize should be between " + HbaseConfigConstants.MIN_SCAN_BATCH_SIZE + " and " + HbaseConfigConstants.MAX_SCAN_BATCH_SIZE); - for(int i = 0; i < format.columnTypes.size(); ++i) { Preconditions.checkArgument(StringUtils.isNotEmpty(format.columnTypes.get(i))); Preconditions.checkArgument(StringUtils.isNotEmpty(format.columnNames.get(i)) diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java index 5b998b1bce..27dfb76396 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseReader.java @@ -54,7 +54,6 @@ public class HbaseReader extends BaseDataReader { protected boolean isBinaryRowkey; protected String tableName; protected int scanCacheSize; - protected int scanBatchSize; public HbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); @@ -71,7 +70,6 @@ public HbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { encoding = readerConfig.getParameter().getStringVal(HbaseConfigKeys.KEY_ENCODING); scanCacheSize = readerConfig.getParameter().getIntVal(HbaseConfigKeys.KEY_SCAN_CACHE_SIZE, HbaseConfigConstants.DEFAULT_SCAN_CACHE_SIZE); - scanBatchSize = readerConfig.getParameter().getIntVal(HbaseConfigKeys.KEY_SCAN_BATCH_SIZE, HbaseConfigConstants.DEFAULT_SCAN_BATCH_SIZE); List columns = readerConfig.getParameter().getColumn(); if(columns != null && columns.size() > 0) { @@ -110,7 +108,6 @@ public DataStream readData() { builder.setBytes(bytes); builder.setMonitorUrls(monitorUrls); builder.setScanCacheSize(scanCacheSize); - builder.setScanBatchSize(scanBatchSize); builder.setMonitorUrls(monitorUrls); builder.setTestConfig(testConfig); builder.setLogConfig(logConfig); diff --git a/flinkx-hbase/flinkx-hbase-writer/pom.xml b/flinkx-hbase/flinkx-hbase-writer/pom.xml index e78909ce62..05ce45986e 100644 --- a/flinkx-hbase/flinkx-hbase-writer/pom.xml +++ b/flinkx-hbase/flinkx-hbase-writer/pom.xml @@ -12,6 +12,11 @@ flinkx-hbase-writer + + com.google.guava + guava + 12.0.1 + com.dtstack.flinkx flinkx-hbase-core @@ -57,11 +62,11 @@ com.google.common - shade.core.com.google.common + shade.hbase.com.google.common com.google.thirdparty - shade.core.com.google.thirdparty + shade.hbase.com.google.thirdparty @@ -87,6 +92,7 @@ + diff --git a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java index fed523dde7..2593eb7000 100644 --- a/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java +++ b/flinkx-hdfs/flinkx-hdfs-core/src/main/java/com/dtstack/flinkx/hdfs/HdfsUtil.java @@ -57,7 +57,7 @@ public class HdfsUtil { public static Object getWritableValue(Object writable) { Class clz = writable.getClass(); - Object ret; + Object ret = null; if(clz == IntWritable.class) { ret = ((IntWritable) writable).get(); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml index 2918b4d6b3..57a6ad9d67 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-reader/pom.xml @@ -128,6 +128,7 @@ under the License. + diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java index 5d66b7dc1e..38ed6a9878 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsReader.java @@ -21,10 +21,12 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.hdfs.HdfsConfigKeys; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; @@ -51,7 +53,15 @@ public HdfsReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); defaultFs = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_DEFAULT_FS); - path = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_PATH); + + String fileName = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_FILE_NAME); + if(StringUtils.isNotBlank(fileName)){ + //兼容平台逻辑 + path = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_PATH) + ConstantValue.SINGLE_SLASH_SYMBOL + fileName; + }else{ + path = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_PATH); + } + fileType = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_FILE_TYPE); hadoopConfig = (Map) readerConfig.getParameter().getVal(HdfsConfigKeys.KEY_HADOOP_CONFIG); filterRegex = readerConfig.getParameter().getStringVal(HdfsConfigKeys.KEY_FILTER, ""); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml index 7212c0cef0..a01c5aa9be 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/pom.xml +++ b/flinkx-hdfs/flinkx-hdfs-writer/pom.xml @@ -131,6 +131,7 @@ under the License. + diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java index f611487af5..33f9da70aa 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java @@ -23,6 +23,8 @@ import com.dtstack.flinkx.util.ColumnTypeUtil; import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.SysUtil; +import com.google.gson.Gson; +import com.google.gson.JsonParser; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -72,8 +74,12 @@ public abstract class BaseHdfsOutputFormat extends BaseFileOutputFormat { protected transient Map decimalColInfo; + //如果key为string类型的值是map 或者 list 会使用gson转为json格式存入 + protected transient Gson gson; + @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { + gson = new Gson(); // 这里休眠一段时间是为了避免reader和writer或者多个任务在同一个taskmanager里同时认证kerberos if (FileSystemUtil.isOpenKerberos(hadoopConfig)) { sleepRandomTime(); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 9712b405b4..22925d99eb 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -25,8 +25,12 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.ColumnTypeUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.ReflectionUtils; import org.apache.flink.types.Row; import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.orc.OrcFile; import org.apache.hadoop.hive.ql.io.orc.OrcSerde; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -44,6 +48,7 @@ import org.apache.hadoop.mapred.RecordWriter; import org.apache.hadoop.mapred.Reporter; import java.io.IOException; +import java.lang.reflect.Field; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.charset.StandardCharsets; @@ -53,6 +58,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Map; /** * The subclass of HdfsOutputFormat writing orc files @@ -138,6 +144,7 @@ protected void nextBlock(){ recordWriter = outputFormat.getRecordWriter(null, jobConf, currentBlockTmpPath, Reporter.NULL); blockIndex++; + setFs(); LOG.info("nextBlock:Current block writer record:" + rowsOfCurrentBlock); LOG.info("Current block file name:" + currentBlockTmpPath); } catch (Exception e){ @@ -235,6 +242,8 @@ private void getData(List recordList, int index, Row row) throws WriteRe if (column instanceof Timestamp){ SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); recordList.add(fm.format(column)); + }else if (column instanceof Map || column instanceof List){ + recordList.add(gson.toJson(column)); }else { recordList.add(rowData); } @@ -296,4 +305,21 @@ protected void closeSource() throws IOException { this.recordWriter = null; } } + + /** + * 数据源开启kerberos时 + * 如果这里不通过反射对 writerOptions 赋值fs,则在recordWriter.writer时 会初始化一个fs 此fs不在ugi里获取的 + * 导致开启了kerberos的数据源在checkpoint时进行 recordWriter.close() 操作,会出现kerberos认证错误 + * @throws IllegalAccessException + */ + private void setFs() throws IllegalAccessException { + if(FileSystemUtil.isOpenKerberos(hadoopConfig)){ + Field declaredField = ReflectionUtils.getDeclaredField(recordWriter, "options"); + assert declaredField != null; + declaredField.setAccessible(true); + OrcFile.WriterOptions writerOptions = (OrcFile.WriterOptions) declaredField.get(recordWriter); + writerOptions.fileSystem(fs); + declaredField.setAccessible(false); + } + } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 351e5eee12..84cfa835a1 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -24,11 +24,15 @@ import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.ColumnTypeUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.GsonUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroupFactory; @@ -45,9 +49,12 @@ import java.io.IOException; import java.math.BigDecimal; +import java.security.PrivilegedAction; import java.sql.Timestamp; import java.util.Date; import java.util.HashMap; +import java.util.List; +import java.util.Map; /** * The subclass of HdfsOutputFormat writing parquet files @@ -93,8 +100,21 @@ protected void nextBlock(){ .withType(schema) .withDictionaryEncoding(enableDictionary) .withRowGroupSize(rowGroupSize); - writer = builder.build(); + //开启kerberos 需要在ugi里进行build + if(FileSystemUtil.isOpenKerberos(hadoopConfig)){ + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig, defaultFs); + ugi.doAs((PrivilegedAction) () -> { + try { + writer = builder.build(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return null; + }); + }else{ + writer = builder.build(); + } blockIndex++; } catch (Exception e){ throw new RuntimeException(e); @@ -220,6 +240,8 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ if (valObj instanceof Timestamp){ val=DateUtil.getDateTimeFormatter().format(valObj); group.add(colName,val); + }else if (valObj instanceof Map || valObj instanceof List){ + group.add(colName,gson.toJson(valObj)); }else { group.add(colName,val); } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index dc592d5563..3ad063e429 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -23,6 +23,9 @@ import com.dtstack.flinkx.hdfs.ECompressType; import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.google.gson.JsonElement; +import com.google.gson.JsonParser; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.apache.flink.types.Row; @@ -35,6 +38,8 @@ import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.Date; +import java.util.List; +import java.util.Map; /** * The builder class of HdfsOutputFormat writing text files @@ -200,6 +205,8 @@ private void appendDataToString(StringBuilder sb, Object column, ColumnType colu if (column instanceof Timestamp){ SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); sb.append(fm.format(column)); + }else if (column instanceof Map || column instanceof List){ + sb.append(gson.toJson(column)); }else { sb.append(rowData); } diff --git a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java index 14d8d3fc8b..c28a1e4785 100644 --- a/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java +++ b/flinkx-hive/flinkx-hive-core/src/main/java/com/dtstack/flinkx/hive/util/AddressUtil.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.hive.util; +import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.net.telnet.TelnetClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,7 +48,7 @@ public static boolean telnet(String ip,int port){ client.disconnect(); } } catch (Exception e){ - logger.error("{}",e); + logger.error("{}", ExceptionUtil.getErrorMessage(e)); } } } diff --git a/flinkx-hive/flinkx-hive-writer/pom.xml b/flinkx-hive/flinkx-hive-writer/pom.xml index 9c5e35828c..de65c7b5e3 100644 --- a/flinkx-hive/flinkx-hive-writer/pom.xml +++ b/flinkx-hive/flinkx-hive-writer/pom.xml @@ -92,6 +92,18 @@ under the License. shade.core.com.google.thirdparty + + + + META-INF/services/java.sql.Driver + + + META-INF/services + java.sql.hive2.Driver + + @@ -115,6 +127,7 @@ under the License. + diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java index 2b858bd78d..1a0ea6f90b 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java @@ -41,6 +41,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import static com.dtstack.flinkx.hive.HiveConfigKeys.KEY_SCHEMA; @@ -202,7 +203,8 @@ public void writeRecord(Row row) throws IOException { event = GsonUtil.GSON.fromJson((String) tempObj, GsonUtil.gsonMapTypeToken); }catch (JsonSyntaxException e){ // is not a json string - LOG.warn("bad json string:【{}】", tempObj); + //tempObj 不是map类型 则event直接往下传递 + // LOG.warn("bad json string:【{}】", tempObj); } } } @@ -230,7 +232,7 @@ public void writeRecord(Row row) throws IOException { //row包含map嵌套的数据内容和channel, 而rowData是非常简单的纯数据,此处补上数据差额 if (fromLogData && bytesWriteCounter != null) { - bytesWriteCounter.add((long) row.toString().length() - rowData.toString().length()); + bytesWriteCounter.add((long)row.toString().getBytes().length - rowData.toString().getBytes().length); } } catch (Exception e) { // 写入产生的脏数据已经由hdfsOutputFormat处理了,这里不用再处理了,只打印日志 @@ -250,11 +252,11 @@ private Row setChannelInformation(Map event, Object channel, Lis //防止kafka column和 hive column大小写不一致,获取不到值 ,全部转为小写进行获取 HashMap newEvent = new HashMap<>(event.size() * 2); event.entrySet().forEach(data->{ - newEvent.put(data.getKey().toLowerCase(),data.getValue()); + newEvent.put(data.getKey().toLowerCase(Locale.ENGLISH),data.getValue()); }); for (int i = 0; i < columns.size(); i++) { - rowData.setField(i, newEvent.get(columns.get(i).toLowerCase())); + rowData.setField(i, newEvent.get(columns.get(i).toLowerCase(Locale.ENGLISH))); } rowData.setField(rowData.getArity() - 1, channel); return rowData; diff --git a/flinkx-kafka/flinkx-kafka-reader/pom.xml b/flinkx-kafka/flinkx-kafka-reader/pom.xml index cc677ef642..b2b936be42 100644 --- a/flinkx-kafka/flinkx-kafka-reader/pom.xml +++ b/flinkx-kafka/flinkx-kafka-reader/pom.xml @@ -66,6 +66,7 @@ + diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java new file mode 100644 index 0000000000..48eccb16c2 --- /dev/null +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java @@ -0,0 +1,221 @@ +/* + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.kafka.client; + +import com.dtstack.flinkx.decoder.IDecode; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.IClient; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Date: 2019/12/25 + * Company: www.dtstack.com + * + * @author tudou + */ +public class KafkaClient implements IClient { + protected static Logger LOG = LoggerFactory.getLogger(KafkaClient.class); + //ck state指针 + private final AtomicReference> stateReference; + private volatile boolean running = true; + private long pollTimeout; + private boolean blankIgnore; + private IDecode decode; + private KafkaBaseInputFormat format; + private KafkaConsumer consumer; + //是否触发checkpoint,需要提交offset指针 + private AtomicBoolean commit; + + @SuppressWarnings("unchecked") + public KafkaClient(Properties clientProps, long pollTimeout, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { + this.pollTimeout = pollTimeout; + this.blankIgnore = format.getBlankIgnore(); + this.format = format; + this.decode = format.getDecode(); + this.commit = new AtomicBoolean(false); + this.stateReference = new AtomicReference<>(); + consumer = new org.apache.kafka.clients.consumer.KafkaConsumer<>(clientProps); + StartupMode mode = format.getMode(); + List stateList = kafkaInputSplit.getList(); + Map partitionMap = new HashMap<>(Math.max((int) (stateList.size()/.75f) + 1, 16)); + Object stateMap = format.getState(); + boolean needToSeek = true; + if(stateMap instanceof Map && MapUtils.isNotEmpty((Map)stateMap)){ + Map map = (Map) stateMap; + for (kafkaState state : map.values()) { + TopicPartition tp = new TopicPartition(state.getTopic(), state.getPartition()); + //ck中保存的是当前已经读取的offset,恢复时从下一条开始读 + partitionMap.put(tp, state.getOffset() + 1); + } + LOG.info("init kafka client from [checkpoint], stateMap = {}", map); + }else if(CollectionUtils.isEmpty(stateList)){ + running = false; + LOG.warn("\n" + + "****************************************************\n" + + "******************* WARN *********************\n" + + "| this stateList in KafkaInputSplit is empty, |\n" + + "| this channel will not assign any kafka topic, |\n" + + "| therefore, no data will be read in this channel! |\n" + + "****************************************************"); + return; + }if(StartupMode.TIMESTAMP.equals(mode)){ + Map timestampMap = new HashMap<>(Math.max((int) (stateList.size()/.75f) + 1, 16)); + for (kafkaState state : stateList) { + TopicPartition tp = new TopicPartition(state.getTopic(), state.getPartition()); + timestampMap.put(tp, state.getTimestamp()); + partitionMap.put(tp, null); + } + Map offsets = consumer.offsetsForTimes(timestampMap); + for (TopicPartition tp : partitionMap.keySet()) { + OffsetAndTimestamp offsetAndTimestamp = offsets.get(tp); + if (offsetAndTimestamp != null) { + partitionMap.put(tp, offsetAndTimestamp.offset()); + } + } + LOG.info("init kafka client from [timestamp], offsets = {}", offsets); + }else if(StartupMode.SPECIFIC_OFFSETS.equals(mode)){ + for (kafkaState state : stateList) { + TopicPartition tp = new TopicPartition(state.getTopic(), state.getPartition()); + partitionMap.put(tp, state.getOffset()); + } + LOG.info("init kafka client from [specific-offsets], stateList = {}", stateList); + }else{ + for (kafkaState state : stateList) { + partitionMap.put(new TopicPartition(state.getTopic(), state.getPartition()), null); + } + needToSeek = false; + LOG.info("init kafka client from [split], stateList = {}", stateList); + } + LOG.info("partitionList = {}", partitionMap.keySet()); + consumer.assign(partitionMap.keySet()); + if(needToSeek){ + for (Map.Entry entry : partitionMap.entrySet()) { + consumer.seek(entry.getKey(), entry.getValue()); + } + } + } + + @Override + public void run() { + Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { + LOG.error("KafkaClient run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); + }); + try { + while (running) { + if(this.commit.getAndSet(false)){ + final Collection kafkaStates = stateReference.getAndSet(null); + if(kafkaStates != null){ + LOG.info("submit kafka offset, kafkaStates = {}", kafkaStates); + Map offsets = new HashMap<>(Math.max((int) (kafkaStates.size()/.75f) + 1, 16)); + for (kafkaState state : kafkaStates) { + offsets.put(new TopicPartition(state.getTopic(), state.getPartition()), new OffsetAndMetadata(state.getOffset(), "no metadata")); + } + try { + consumer.commitAsync(offsets, (o, ex) -> { + if (ex != null) { + LOG.warn("Committing offsets to Kafka failed, This does not compromise Flink's checkpoints. offsets = {}, e = {}", o, ExceptionUtil.getErrorMessage(ex)); + } else { + LOG.info("Committing offsets to Kafka async successfully, offsets = {}", o); + } + }); + }catch (Exception e){ + LOG.warn("Committing offsets to Kafka failed, This does not compromise Flink's checkpoints. offsets = {}, e = {}", offsets, ExceptionUtil.getErrorMessage(e)); + try { + consumer.commitSync(offsets); + LOG.info("Committing offsets to Kafka successfully, offsets = {}", offsets); + }catch (Exception e1){ + LOG.warn("Committing offsets to Kafka failed, This does not compromise Flink's checkpoints. offsets = {}, e = {}", offsets, ExceptionUtil.getErrorMessage(e1)); + } + } + } + } + + ConsumerRecords records = consumer.poll(pollTimeout); + for (ConsumerRecord r : records) { + boolean isIgnoreCurrent = r.value() == null || blankIgnore && StringUtils.isBlank(r.value()); + if (isIgnoreCurrent) { + continue; + } + + try { + processMessage(r.value(), r.topic(), r.partition(), r.offset(), r.timestamp()); + } catch (Throwable e) { + LOG.warn("kafka consumer fetch is error, message:{}, e = {}", r.value(), ExceptionUtil.getErrorMessage(e)); + } + } + } + } catch (WakeupException e) { + LOG.warn("WakeupException to close kafka consumer, e = {}", ExceptionUtil.getErrorMessage(e)); + } catch (Throwable e) { + LOG.warn("kafka consumer fetch is error, e = {}", ExceptionUtil.getErrorMessage(e)); + } finally { + consumer.close(); + } + } + + @Override + public void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp) { + Map event = decode.decode(message); + if (event != null && event.size() > 0) { + format.processEvent(Pair.of(event, new kafkaState(topic, partition, offset, timestamp))); + } + } + + /** + * 提交kafka offset + * @param kafkaStates + */ + public void submitOffsets(Collection kafkaStates){ + this.commit.set(true); + this.stateReference.getAndSet(kafkaStates); + } + + @Override + public void close() { + try { + running = false; + consumer.wakeup(); + } catch (Exception e) { + LOG.error("close kafka consumer error, e = {}", ExceptionUtil.getErrorMessage(e)); + } + } + +} diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaConsumer.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaConsumer.java similarity index 58% rename from flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaConsumer.java rename to flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaConsumer.java index 51d36158b7..ba9564b740 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaConsumer.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaConsumer.java @@ -14,12 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka.reader; +package com.dtstack.flinkx.kafka.client; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseConsumer; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; -import java.util.Arrays; +import java.util.Collection; import java.util.Properties; /** @@ -34,12 +37,23 @@ public KafkaConsumer(Properties properties) { } @Override - public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format) { + public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { Properties clientProps = new Properties(); clientProps.putAll(props); - clientProps.put("group.id", group); + clientProps.put(KafkaConfigKeys.GROUP_ID, group); - client = new KafkaClient(clientProps, Arrays.asList(topic.split(",")), Long.MAX_VALUE, format); + client = new KafkaClient(clientProps, 100L, format, kafkaInputSplit); return this; } + + /** + * 提交kafka offset + * @param kafkaStates + */ + public void submitOffsets(Collection kafkaStates){ + if(client != null){ + KafkaClient kafkaClient = (KafkaClient) this.client; + kafkaClient.submitOffsets(kafkaStates); + } + } } diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/format/KafkaInputFormat.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/format/KafkaInputFormat.java new file mode 100644 index 0000000000..6358d830ad --- /dev/null +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/format/KafkaInputFormat.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.kafka.format; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kafka.client.KafkaConsumer; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.util.KafkaUtil; +import com.dtstack.flinkx.restore.FormatState; +import com.dtstack.flinkx.util.RangeSplitUtil; +import org.apache.commons.collections.MapUtils; +import org.apache.flink.core.io.InputSplit; +import org.apache.kafka.common.PartitionInfo; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Date: 2019/11/21 + * Company: www.dtstack.com + * + * @author tudou + */ +public class KafkaInputFormat extends KafkaBaseInputFormat { + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) { + List stateList = new ArrayList<>(); + org.apache.kafka.clients.consumer.KafkaConsumer consumer = new org.apache.kafka.clients.consumer.KafkaConsumer<>(KafkaUtil.geneConsumerProp(consumerSettings, mode)); + if(StartupMode.TIMESTAMP.equals(mode)){ + List partitionInfoList = consumer.partitionsFor(topic); + for (PartitionInfo p : partitionInfoList) { + stateList.add(new kafkaState(p.topic(), p.partition(), null, timestamp)); + } + }else if(StartupMode.SPECIFIC_OFFSETS.equals(mode)){ + stateList = KafkaUtil.parseSpecificOffsetsString(topic, offset); + }else{ + String[] topics = topic.split(ConstantValue.COMMA_SYMBOL); + if(topics.length == 1){ + List partitionInfoList = consumer.partitionsFor(topic); + for (PartitionInfo p : partitionInfoList) { + stateList.add(new kafkaState(p.topic(), p.partition(), null, null)); + } + }else{ + for (String tp : topics) { + List partitionInfoList = consumer.partitionsFor(tp); + for (PartitionInfo p : partitionInfoList) { + stateList.add(new kafkaState(p.topic(), p.partition(), null, null)); + } + } + } + } + + List> list = RangeSplitUtil.subListBySegment(stateList, minNumSplits); + InputSplit[] splits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new KafkaInputSplit(i, list.get(i)); + } + + return splits; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + Properties props = KafkaUtil.geneConsumerProp(consumerSettings, mode); + consumer = new KafkaConsumer(props); + } + + @Override + public FormatState getFormatState() { + super.getFormatState(); + if (formatState != null && MapUtils.isNotEmpty(stateMap)) { + KafkaConsumer kafkaConsumer = (KafkaConsumer) this.consumer; + List list = new ArrayList<>(stateMap.size()); + for (kafkaState kafkaState : stateMap.values()) { + list.add(kafkaState.clone()); + } + kafkaConsumer.submitOffsets(list); + } + return formatState; + } + + @Override + public KafkaVersion getKafkaVersion() { + return KafkaVersion.kafka; + } +} diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java deleted file mode 100644 index 95016e4956..0000000000 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaClient.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka.reader; - -import com.dtstack.flinkx.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.reader.IClient; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.commons.lang.StringUtils; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.errors.WakeupException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - * Date: 2019/12/25 - * Company: www.dtstack.com - * - * @author tudou - */ -public class KafkaClient implements IClient { - protected static Logger LOG = LoggerFactory.getLogger(KafkaClient.class); - private volatile boolean running = true; - private long pollTimeout; - private boolean blankIgnore; - private IDecode decode; - private KafkaBaseInputFormat format; - private KafkaConsumer consumer; - - public KafkaClient(Properties clientProps, List topics, long pollTimeout, KafkaBaseInputFormat format) { - this.pollTimeout = pollTimeout; - this.blankIgnore = format.getBlankIgnore(); - this.format = format; - this.decode = format.getDecode(); - consumer = new org.apache.kafka.clients.consumer.KafkaConsumer<>(clientProps); - consumer.subscribe(topics); - } - - @Override - public void run() { - try { - while (running) { - ConsumerRecords records = consumer.poll(pollTimeout); - for (ConsumerRecord r : records) { - boolean isIgnoreCurrent = r.value() == null || blankIgnore && StringUtils.isBlank(r.value()); - if (isIgnoreCurrent) { - continue; - } - - try { - processMessage(r.value()); - } catch (Throwable e) { - LOG.error("kafka consumer fetch is error, message:{}, e = {}", r.value(), ExceptionUtil.getErrorMessage(e)); - } - } - } - } catch (WakeupException e) { - LOG.warn("WakeupException to close kafka consumer, e = {}", ExceptionUtil.getErrorMessage(e)); - } catch (Throwable e) { - LOG.error("kafka consumer fetch is error, e = {}", ExceptionUtil.getErrorMessage(e)); - } finally { - consumer.close(); - } - } - - @Override - public void processMessage(String message) { - Map event = decode.decode(message); - if (event != null && event.size() > 0) { - format.processEvent(event); - } - } - - @Override - public void close() { - try { - running = false; - consumer.wakeup(); - } catch (Exception e) { - LOG.error("close kafka consumer error, e = {}", ExceptionUtil.getErrorMessage(e)); - } - } - -} diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaReader.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaReader.java index 441a5762ac..b284af7419 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaReader.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/reader/KafkaReader.java @@ -18,10 +18,10 @@ package com.dtstack.flinkx.kafka.reader; import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka.format.KafkaInputFormat; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseReader; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.kafka.clients.producer.ProducerConfig; /** * Date: 2019/11/21 @@ -33,13 +33,10 @@ public class KafkaReader extends KafkaBaseReader { public KafkaReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); - if (!consumerSettings.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) { - throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must set in consumerSettings"); - } } @Override - public KafkaBaseInputFormat getFormat(){ - return new KafkaInputFormat(); + public KafkaBaseInputFormatBuilder getBuilder(){ + return new KafkaBaseInputFormatBuilder(new KafkaInputFormat()); } } diff --git a/flinkx-kafka/flinkx-kafka-writer/pom.xml b/flinkx-kafka/flinkx-kafka-writer/pom.xml index c107bca101..84efdd2b15 100644 --- a/flinkx-kafka/flinkx-kafka-writer/pom.xml +++ b/flinkx-kafka/flinkx-kafka-writer/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaOutputFormat.java b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java similarity index 77% rename from flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaOutputFormat.java rename to flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java index d9503a7c58..fdd075b0b4 100644 --- a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaOutputFormat.java +++ b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java @@ -16,11 +16,12 @@ * limitations under the License. */ -package com.dtstack.flinkx.kafka.writer; +package com.dtstack.flinkx.kafka.format; -import com.dtstack.flinkx.kafkabase.Formatter; -import com.dtstack.flinkx.kafkabase.writer.KafkaBaseOutputFormat; +import com.dtstack.flinkx.kafkabase.util.Formatter; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -30,6 +31,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** * Date: 2019/11/21 @@ -45,7 +47,7 @@ public void configure(Configuration parameters) { super.configure(parameters); props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 86400000); + props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 60000); props.put(ProducerConfig.RETRIES_CONFIG, 1000000); props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1); if (producerSettings != null) { @@ -56,12 +58,15 @@ public void configure(Configuration parameters) { @Override protected void emit(Map event) throws IOException { + heartBeatController.acquire(); String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), objectMapper.writeValueAsString(event)), (metadata, exception) -> { + producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { if(Objects.nonNull(exception)){ String errorMessage = String.format("send data failed,data 【%s】 ,error info %s",event,ExceptionUtil.getErrorMessage(exception)); LOG.warn(errorMessage); - throw new RuntimeException(errorMessage); + heartBeatController.onFailed(exception); + } else { + heartBeatController.onSuccess(); } }); } @@ -69,6 +74,7 @@ protected void emit(Map event) throws IOException { @Override public void closeInternal() { LOG.warn("kafka output closeInternal."); - producer.close(); + //未设置具体超时时间 关闭时间默认是long.value 导致整个方法长时间等待关闭不了,因此明确指定20s时间 + producer.close(KafkaBaseOutputFormat.CLOSE_TIME, TimeUnit.MILLISECONDS); } } diff --git a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java index 36fe32d8f9..f382cd087c 100644 --- a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java +++ b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.kafka.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.kafka.format.KafkaOutputFormat; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.kafkabase.writer.KafkaBaseWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -47,6 +49,10 @@ public DataStreamSink writeData(DataStream dataSet) { format.setProducerSettings(producerSettings); format.setRestoreConfig(restoreConfig); format.setTableFields(tableFields); + format.setDirtyPath(dirtyPath); + format.setDirtyHadoopConfig(dirtyHadoopConfig); + format.setSrcFieldNames(srcCols); + format.setHeartBeatController(new HeartBeatController()); return createOutput(dataSet, format); } diff --git a/flinkx-kafka/pom.xml b/flinkx-kafka/pom.xml index aa6ebda990..bc85942890 100644 --- a/flinkx-kafka/pom.xml +++ b/flinkx-kafka/pom.xml @@ -23,6 +23,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.kafka diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml index 0e52e92ab7..7e90aa9af0 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java similarity index 69% rename from flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java rename to flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java index 9633e6e4ad..faacd2710d 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Client.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java @@ -14,14 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka09.reader; +package com.dtstack.flinkx.kafka09.client; import com.dtstack.flinkx.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.reader.IClient; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.client.IClient; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import kafka.consumer.ConsumerIterator; import kafka.consumer.KafkaStream; +import kafka.message.MessageAndMetadata; +import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,17 +53,21 @@ public Kafka09Client(KafkaStream aStream, KafkaBaseInputFormat f @Override public void run() { + Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { + LOG.warn("KafkaClient run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); + }); try { while (running) { ConsumerIterator it = mStream.iterator(); while (it.hasNext()) { String m = null; try { - m = new String(it.next().message(), format.getEncoding()); - Map event = this.decode.decode(m); - if (event != null && event.size() > 0) { - this.format.processEvent(event); - } + MessageAndMetadata next = it.next(); + processMessage(new String(next.message(), format.getEncoding()), + next.topic(), + next.partition(), + next.offset(), + null); } catch (Exception e) { LOG.error("process event = {}, e = {}", m, ExceptionUtil.getErrorMessage(e)); } @@ -72,10 +79,10 @@ public void run() { } @Override - public void processMessage(String message) { + public void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp) { Map event = decode.decode(message); if (event != null && event.size() > 0) { - format.processEvent(event); + format.processEvent(Pair.of(event, new kafkaState(topic, partition, offset, timestamp))); } } diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Consumer.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java similarity index 82% rename from flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Consumer.java rename to flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java index ce2fccbc05..110ef449d5 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Consumer.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka09.reader; +package com.dtstack.flinkx.kafka09.client; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseConsumer; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import kafka.consumer.KafkaStream; import java.util.Properties; @@ -37,7 +38,7 @@ public Kafka09Consumer(KafkaStream aStream) { } @Override - public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format) { + public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { client = new Kafka09Client(mStream, format); return this; } diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java similarity index 76% rename from flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java rename to flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java index ce1c5786d5..bc4bcd9992 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09InputFormat.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java @@ -15,9 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka09.reader; +package com.dtstack.flinkx.kafka09.format; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka09.client.Kafka09Consumer; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.util.KafkaUtil; import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaStream; import kafka.javaapi.consumer.ConsumerConnector; @@ -41,12 +45,12 @@ public class Kafka09InputFormat extends KafkaBaseInputFormat { @Override public void openInputFormat() throws IOException { super.openInputFormat(); - Properties props = geneConsumerProp(); + Properties props = KafkaUtil.geneConsumerProp(consumerSettings, mode); consumerConnector = kafka.consumer.Consumer.createJavaConsumerConnector(new ConsumerConfig(props)); } @Override - protected void openInternal(InputSplit inputSplit) throws IOException { + protected void openInternal(InputSplit inputSplit) { Map topicCountMap = Collections.singletonMap(topic, 1); Map>> consumerMap = consumerConnector.createMessageStreams(topicCountMap); @@ -54,12 +58,12 @@ protected void openInternal(InputSplit inputSplit) throws IOException { for (final KafkaStream stream : streams) { consumer = new Kafka09Consumer(stream); } - consumer.createClient(topic, groupId, this).execute(); + consumer.createClient(topic, groupId, this, (KafkaInputSplit)inputSplit).execute(); running = true; } @Override - protected void closeInternal() throws IOException { + protected void closeInternal() { if (running) { consumerConnector.commitOffsets(true); consumerConnector.shutdown(); @@ -68,4 +72,9 @@ protected void closeInternal() throws IOException { LOG.warn("input kafka release."); } } + + @Override + public KafkaVersion getKafkaVersion() { + return KafkaVersion.kafka09; + } } diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java index 62abf5f7d6..31c6c6c5b4 100644 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java +++ b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java @@ -18,32 +18,31 @@ package com.dtstack.flinkx.kafka09.reader; import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.kafka09.format.Kafka09InputFormat; import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseReader; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import java.nio.charset.StandardCharsets; - /** * @company: www.dtstack.com * @author: toutian * @create: 2019/7/4 */ public class Kafka09Reader extends KafkaBaseReader { - private String encoding; public Kafka09Reader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); - ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - encoding = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_ENCODING, StandardCharsets.UTF_8.name()); + //兼容历史脚本 + String id = consumerSettings.get(KafkaConfigKeys.GROUP_ID); + if(StringUtils.isNotBlank(id)){ + super.groupId = id; + } } @Override - public KafkaBaseInputFormat getFormat(){ - Kafka09InputFormat format = new Kafka09InputFormat(); - format.setEncoding(encoding); - return format; + public KafkaBaseInputFormatBuilder getBuilder(){ + return new KafkaBaseInputFormatBuilder(new Kafka09InputFormat()); } } diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml index 32a3dd0bf9..b774b700be 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09OutputFormat.java b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java similarity index 78% rename from flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09OutputFormat.java rename to flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java index 87478df174..7d792f2619 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09OutputFormat.java +++ b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java @@ -15,15 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka09.writer; +package com.dtstack.flinkx.kafka09.format; -import com.dtstack.flinkx.kafkabase.Formatter; -import com.dtstack.flinkx.kafkabase.writer.AddressUtil; -import com.dtstack.flinkx.kafkabase.writer.KafkaBaseOutputFormat; -import com.dtstack.flinkx.util.GsonUtil; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; +import com.dtstack.flinkx.kafkabase.util.Formatter; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; +import com.dtstack.flinkx.util.MapUtil; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -32,6 +29,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** * @company: www.dtstack.com @@ -61,23 +59,16 @@ public void configure(Configuration parameters) { props.putAll(producerSettings); } props.put("metadata.broker.list", brokerList); - props.put("bootstrap.servers", brokerList); producer = new KafkaProducer<>(props); - LOG.info("brokerList {}", brokerList); - String broker = brokerList.split(",")[0]; - String[] split = broker.split(":"); - - if (split.length != 2 || !AddressUtil.telnet(split[0], Integer.parseInt(split[1]))) { - throw new RuntimeException("telnet error,brokerList" + brokerList); - } + super.configure(parameters); } @Override protected void emit(Map event) throws IOException { heartBeatController.acquire(); String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), objectMapper.writeValueAsString(event)), (metadata, exception) -> { + producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { if (Objects.nonNull(exception)) { LOG.warn("kafka writeSingleRecordInternal error:{}", exception.getMessage(), exception); heartBeatController.onFailed(exception); @@ -89,8 +80,9 @@ protected void emit(Map event) throws IOException { @Override public void closeInternal() { - LOG.warn("kafka output closeInternal."); - producer.close(); + LOG.info("kafka output closeInternal."); + //未设置具体超时时间 关闭时间默认是long.value 导致整个方法长时间等待关闭不了,因此明确指定20s时间 + producer.close(KafkaBaseOutputFormat.CLOSE_TIME, TimeUnit.MILLISECONDS); } public void setEncoding(String encoding) { diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java index c91e0053ec..4caa61ba91 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java +++ b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java @@ -19,7 +19,9 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.kafka09.format.Kafka09OutputFormat; import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.kafkabase.writer.KafkaBaseWriter; import org.apache.commons.lang.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; @@ -60,6 +62,9 @@ public DataStreamSink writeData(DataStream dataSet) { format.setRestoreConfig(restoreConfig); format.setHeartBeatController(new HeartBeatController()); + format.setDirtyPath(dirtyPath); + format.setDirtyHadoopConfig(dirtyHadoopConfig); + format.setSrcFieldNames(srcCols); return createOutput(dataSet, format); } } diff --git a/flinkx-kafka09/pom.xml b/flinkx-kafka09/pom.xml index b24079eeea..3a8a141fcc 100644 --- a/flinkx-kafka09/pom.xml +++ b/flinkx-kafka09/pom.xml @@ -23,6 +23,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.kafka diff --git a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml index 3715a3df95..9170712ee0 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-reader/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Client.java similarity index 82% rename from flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java rename to flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Client.java index f0d10d6f13..18baae6eca 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Client.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Client.java @@ -15,13 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka10.reader; +package com.dtstack.flinkx.kafka10.client; import com.dtstack.flinkx.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.reader.IClient; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.client.IClient; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -59,6 +61,9 @@ public Kafka10Client(Properties clientProps, List topics, long pollTimeo @Override public void run() { + Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { + LOG.warn("KafkaClient run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); + }); try { while (running) { ConsumerRecords records = consumer.poll(pollTimeout); @@ -69,7 +74,7 @@ public void run() { } try { - processMessage(r.value()); + processMessage(r.value(), r.topic(), r.partition(), r.offset(), r.timestamp()); } catch (Throwable e) { LOG.error("kafka consumer fetch is error, message = {}, e = {}", r.value(), ExceptionUtil.getErrorMessage(e)); } @@ -85,10 +90,10 @@ public void run() { } @Override - public void processMessage(String message) { + public void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp) { Map event = decode.decode(message); if (event != null && event.size() > 0) { - format.processEvent(event); + format.processEvent(Pair.of(event, new kafkaState(topic, partition, offset, timestamp))); } } diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Consumer.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Consumer.java similarity index 71% rename from flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Consumer.java rename to flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Consumer.java index 1e9618c673..44999b3d79 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Consumer.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/client/Kafka10Consumer.java @@ -14,10 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka10.reader; +package com.dtstack.flinkx.kafka10.client; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseConsumer; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import java.util.Arrays; import java.util.Properties; @@ -34,12 +37,12 @@ public Kafka10Consumer(Properties properties) { } @Override - public Kafka10Consumer createClient(String topic, String group, KafkaBaseInputFormat format) { + public Kafka10Consumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { Properties clientProps = new Properties(); clientProps.putAll(props); - clientProps.put("group.id", group); + clientProps.put(KafkaConfigKeys.GROUP_ID, group); - client = new Kafka10Client(clientProps, Arrays.asList(topic.split(",")), Long.MAX_VALUE, format); + client = new Kafka10Client(clientProps, Arrays.asList(topic.split(ConstantValue.COMMA_SYMBOL)), Long.MAX_VALUE, format); return this; } } diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10InputFormat.java similarity index 71% rename from flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java rename to flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10InputFormat.java index 448ae9768d..850a9b4f2c 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10InputFormat.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10InputFormat.java @@ -15,9 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka10.reader; +package com.dtstack.flinkx.kafka10.format; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka10.client.Kafka10Consumer; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.util.KafkaUtil; import java.io.IOException; import java.util.Properties; @@ -32,7 +35,12 @@ public class Kafka10InputFormat extends KafkaBaseInputFormat { @Override public void openInputFormat() throws IOException { super.openInputFormat(); - Properties props = geneConsumerProp(); + Properties props = KafkaUtil.geneConsumerProp(consumerSettings, mode); consumer = new Kafka10Consumer(props); } + + @Override + public KafkaVersion getKafkaVersion() { + return KafkaVersion.kafka10; + } } diff --git a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java index 5b89c335d0..0f606c9e94 100644 --- a/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java +++ b/flinkx-kafka10/flinkx-kafka10-reader/src/main/java/com/dtstack/flinkx/kafka10/reader/Kafka10Reader.java @@ -18,10 +18,10 @@ package com.dtstack.flinkx.kafka10.reader; import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka10.format.Kafka10InputFormat; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseReader; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.kafka.clients.producer.ProducerConfig; /** * @company: www.dtstack.com @@ -32,13 +32,10 @@ public class Kafka10Reader extends KafkaBaseReader { public Kafka10Reader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); - if (!consumerSettings.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)){ - throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must set in consumerSettings"); - } } @Override - public KafkaBaseInputFormat getFormat(){ - return new Kafka10InputFormat(); + public KafkaBaseInputFormatBuilder getBuilder(){ + return new KafkaBaseInputFormatBuilder(new Kafka10InputFormat()); } } diff --git a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml index a2d7870814..40234ddd1c 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/pom.xml +++ b/flinkx-kafka10/flinkx-kafka10-writer/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10OutputFormat.java b/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10OutputFormat.java similarity index 77% rename from flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10OutputFormat.java rename to flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10OutputFormat.java index 46f196b462..4a3b8bccbd 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10OutputFormat.java +++ b/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/format/Kafka10OutputFormat.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka10.writer; +package com.dtstack.flinkx.kafka10.format; -import com.dtstack.flinkx.kafkabase.Formatter; -import com.dtstack.flinkx.kafkabase.writer.KafkaBaseOutputFormat; +import com.dtstack.flinkx.kafkabase.util.Formatter; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -29,6 +30,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** * @company: www.dtstack.com @@ -44,7 +46,7 @@ public void configure(Configuration parameters) { super.configure(parameters); props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 86400000); + props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 60000); props.put(ProducerConfig.RETRIES_CONFIG, 1000000); props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1); if (producerSettings != null) { @@ -55,12 +57,15 @@ public void configure(Configuration parameters) { @Override protected void emit(Map event) throws IOException { + heartBeatController.acquire(); String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), objectMapper.writeValueAsString(event)), (metadata, exception) -> { + producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { if(Objects.nonNull(exception)){ String errorMessage = String.format("send data failed,data 【%s】 ,error info %s",event,ExceptionUtil.getErrorMessage(exception)); LOG.warn(errorMessage); - throw new RuntimeException(errorMessage); + heartBeatController.onFailed(exception); + }else{ + heartBeatController.onSuccess(); } }); } @@ -68,6 +73,7 @@ protected void emit(Map event) throws IOException { @Override public void closeInternal() { LOG.warn("kafka output closeInternal."); - producer.close(); + //未设置具体超时时间 关闭时间默认是long.value 导致整个方法长时间等待关闭不了,因此明确指定20s时间 + producer.close(KafkaBaseOutputFormat.CLOSE_TIME, TimeUnit.MILLISECONDS); } } diff --git a/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10Writer.java b/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10Writer.java index 03ad3cd108..72a3f235a4 100644 --- a/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10Writer.java +++ b/flinkx-kafka10/flinkx-kafka10-writer/src/main/java/com/dtstack/flinkx/kafka10/writer/Kafka10Writer.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.kafka10.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.kafka10.format.Kafka10OutputFormat; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.kafkabase.writer.KafkaBaseWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -46,7 +48,10 @@ public DataStreamSink writeData(DataStream dataSet) { format.setTableFields(tableFields); format.setProducerSettings(producerSettings); format.setRestoreConfig(restoreConfig); - + format.setHeartBeatController(new HeartBeatController()); + format.setDirtyPath(dirtyPath); + format.setDirtyHadoopConfig(dirtyHadoopConfig); + format.setSrcFieldNames(srcCols); return createOutput(dataSet, format); } } diff --git a/flinkx-kafka10/pom.xml b/flinkx-kafka10/pom.xml index 314f352a9d..b4fea25a0c 100644 --- a/flinkx-kafka10/pom.xml +++ b/flinkx-kafka10/pom.xml @@ -23,6 +23,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.kafka diff --git a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml index dcdaa4d2cf..4ce9b9fd74 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-reader/pom.xml @@ -65,6 +65,7 @@ + diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java similarity index 78% rename from flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java rename to flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java index 2e3d2d569f..2b6ff0bf0d 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Client.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java @@ -1,10 +1,12 @@ -package com.dtstack.flinkx.kafka11.reader; +package com.dtstack.flinkx.kafka11.client; import com.dtstack.flinkx.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.reader.IClient; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.client.IClient; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -42,6 +44,9 @@ public Kafka11Client(Properties clientProps, List topics, long pollTimeo @Override public void run() { + Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { + LOG.warn("KafkaClient run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); + }); try { while (running) { ConsumerRecords records = consumer.poll(pollTimeout); @@ -52,7 +57,7 @@ public void run() { } try { - processMessage(r.value()); + processMessage(r.value(), r.topic(), r.partition(), r.offset(), r.timestamp()); } catch (Throwable e) { LOG.error("kafka consumer fetch is error, message = {}, e = {}", r.value(), ExceptionUtil.getErrorMessage(e)); } @@ -68,10 +73,10 @@ public void run() { } @Override - public void processMessage(String message) { + public void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp) { Map event = decode.decode(message); if (event != null && event.size() > 0) { - format.processEvent(event); + format.processEvent(Pair.of(event, new kafkaState(topic, partition, offset, timestamp))); } } diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Consumer.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Consumer.java similarity index 71% rename from flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Consumer.java rename to flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Consumer.java index cc3f3f0204..1206ea7178 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Consumer.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Consumer.java @@ -14,10 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka11.reader; +package com.dtstack.flinkx.kafka11.client; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseConsumer; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import java.util.Arrays; import java.util.Properties; @@ -34,12 +37,12 @@ public Kafka11Consumer(Properties properties) { } @Override - public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format) { + public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { Properties clientProps = new Properties(); clientProps.putAll(props); - clientProps.put("group.id", group); + clientProps.put(KafkaConfigKeys.GROUP_ID, group); - client = new Kafka11Client(clientProps, Arrays.asList(topic.split(",")), Long.MAX_VALUE, format); + client = new Kafka11Client(clientProps, Arrays.asList(topic.split(ConstantValue.COMMA_SYMBOL)), Long.MAX_VALUE, format); return this; } } diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11InputFormat.java similarity index 71% rename from flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java rename to flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11InputFormat.java index 5a7253860f..55d1029f1b 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11InputFormat.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11InputFormat.java @@ -17,9 +17,12 @@ */ -package com.dtstack.flinkx.kafka11.reader; +package com.dtstack.flinkx.kafka11.format; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka11.client.Kafka11Consumer; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.util.KafkaUtil; import java.io.IOException; import java.util.Properties; @@ -34,7 +37,12 @@ public class Kafka11InputFormat extends KafkaBaseInputFormat { @Override public void openInputFormat() throws IOException { super.openInputFormat(); - Properties props = geneConsumerProp(); + Properties props = KafkaUtil.geneConsumerProp(consumerSettings, mode); consumer = new Kafka11Consumer(props); } + + @Override + public KafkaVersion getKafkaVersion() { + return KafkaVersion.kafka11; + } } diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java index 758258e730..6cd8e3a7ab 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/reader/Kafka11Reader.java @@ -18,10 +18,10 @@ package com.dtstack.flinkx.kafka11.reader; import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafka11.format.Kafka11InputFormat; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; import com.dtstack.flinkx.kafkabase.reader.KafkaBaseReader; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.kafka.clients.producer.ProducerConfig; /** * @company: www.dtstack.com @@ -32,13 +32,10 @@ public class Kafka11Reader extends KafkaBaseReader { public Kafka11Reader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); - if (!consumerSettings.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)){ - throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must set in consumerSettings"); - } } @Override - public KafkaBaseInputFormat getFormat(){ - return new Kafka11InputFormat(); + public KafkaBaseInputFormatBuilder getBuilder(){ + return new KafkaBaseInputFormatBuilder(new Kafka11InputFormat()); } } diff --git a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml index 94bced62b2..907e45992d 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/pom.xml +++ b/flinkx-kafka11/flinkx-kafka11-writer/pom.xml @@ -66,6 +66,7 @@ + diff --git a/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11OutputFormat.java b/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11OutputFormat.java similarity index 77% rename from flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11OutputFormat.java rename to flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11OutputFormat.java index 90002eeb86..1b3336bfda 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11OutputFormat.java +++ b/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/format/Kafka11OutputFormat.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka11.writer; +package com.dtstack.flinkx.kafka11.format; -import com.dtstack.flinkx.kafkabase.Formatter; -import com.dtstack.flinkx.kafkabase.writer.KafkaBaseOutputFormat; +import com.dtstack.flinkx.kafkabase.util.Formatter; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -29,6 +30,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** * @company: www.dtstack.com @@ -44,7 +46,7 @@ public void configure(Configuration parameters) { super.configure(parameters); props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 86400000); + props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 60000); props.put(ProducerConfig.RETRIES_CONFIG, 1000000); props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1); if (producerSettings != null) { @@ -55,12 +57,15 @@ public void configure(Configuration parameters) { @Override protected void emit(Map event) throws IOException { + heartBeatController.acquire(); String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), objectMapper.writeValueAsString(event)), (metadata, exception) -> { + producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { if(Objects.nonNull(exception)){ String errorMessage = String.format("send data failed,data 【%s】 ,error info %s",event,ExceptionUtil.getErrorMessage(exception)); LOG.warn(errorMessage); - throw new RuntimeException(errorMessage); + heartBeatController.onFailed(exception); + } else { + heartBeatController.onSuccess(); } }); } @@ -68,6 +73,7 @@ protected void emit(Map event) throws IOException { @Override public void closeInternal() { LOG.warn("kafka output closeInternal."); - producer.close(); + //未设置具体超时时间 关闭时间默认是long.value 导致整个方法长时间等待关闭不了,因此明确指定20s时间 + producer.close(KafkaBaseOutputFormat.CLOSE_TIME, TimeUnit.MILLISECONDS); } } diff --git a/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11Writer.java b/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11Writer.java index 22227e6513..6d4a0e0a52 100644 --- a/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11Writer.java +++ b/flinkx-kafka11/flinkx-kafka11-writer/src/main/java/com/dtstack/flinkx/kafka11/writer/Kafka11Writer.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.kafka11.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.kafka11.format.Kafka11OutputFormat; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.kafkabase.writer.KafkaBaseWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -46,7 +48,10 @@ public DataStreamSink writeData(DataStream dataSet) { format.setProducerSettings(producerSettings); format.setRestoreConfig(restoreConfig); format.setTableFields(tableFields); - + format.setHeartBeatController(new HeartBeatController()); + format.setDirtyPath(dirtyPath); + format.setDirtyHadoopConfig(dirtyHadoopConfig); + format.setSrcFieldNames(srcCols); return createOutput(dataSet, format); } } diff --git a/flinkx-kafka11/pom.xml b/flinkx-kafka11/pom.xml index e9d3b3a452..4d42dcce6a 100644 --- a/flinkx-kafka11/pom.xml +++ b/flinkx-kafka11/pom.xml @@ -23,6 +23,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + org.apache.kafka diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java index 714f85e95a..ccc5bf84db 100755 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java @@ -34,10 +34,18 @@ public class KafkaConfigKeys { public static final String KEY_CONSUMER_SETTINGS = "consumerSettings"; public static final String KEY_PRODUCER_SETTINGS = "producerSettings"; public static final String KEY_TABLE_FIELDS = "tableFields"; + public static final String KEY_MODE = "mode"; + //partition:0,offset:42;partition:1,offset:300 + public static final String KEY_OFFSET = "offset"; + public static final String KEY_TIMESTAMP = "timestamp"; /** * kafka 09 */ public static final String KEY_ENCODING = "encoding"; public static final String KEY_BROKER_LIST = "brokerList"; + + public static final String GROUP_ID = "group.id"; + + public static final String BOOTSTRAP_SERVERS= "bootstrap.servers"; } diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/entity/QueueData.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaInputSplit.java similarity index 56% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/entity/QueueData.java rename to flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaInputSplit.java index 8a39cdeb7b..fe37990aec 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/entity/QueueData.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaInputSplit.java @@ -15,40 +15,46 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.oraclelogminer.entity; +package com.dtstack.flinkx.kafkabase; -import com.google.gson.Gson; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import org.apache.flink.core.io.InputSplit; -import java.util.Map; +import java.util.List; /** - * Date: 2020/06/01 + * Date: 2020/12/30 * Company: www.dtstack.com * * @author tudou */ -public class QueueData { - private long scn; - private Map data; +public class KafkaInputSplit implements InputSplit { - public QueueData(long lsn, Map data) { - this.scn = lsn; - this.data = data; + private static final long serialVersionUID = 1L; + + private int splitNumber; + + private List list; + + public KafkaInputSplit(int splitNumber, List list) { + this.splitNumber = splitNumber; + this.list = list; } - public long getScn() { - return scn; + @Override + public int getSplitNumber() { + return splitNumber; } - public Map getData() { - return data; + public List getList() { + return list; } @Override public String toString() { - return "QueueData{" + - "scn=" + scn + - ", data=" + new Gson().toJson(data) + + return "KafkaInputSplit{" + + "splitNumber=" + splitNumber + + ", list=" + list + '}'; } } diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java new file mode 100644 index 0000000000..7ee7669f54 --- /dev/null +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.kafkabase.entity; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Date: 2020/12/26 + * Company: www.dtstack.com + * + * @author tudou + */ +public class kafkaState implements Serializable { + private static final long serialVersionUID = 1L; + + private String topic; + private Integer partition; + private Long offset; + private Long timestamp; + + public kafkaState(String topic, Integer partition, Long offset, Long timestamp) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.timestamp = timestamp; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public Integer getPartition() { + return partition; + } + + public void setPartition(Integer partition) { + this.partition = partition; + } + + public Long getOffset() { + return offset; + } + + public void setOffset(Long offset) { + this.offset = offset; + } + + public Long getTimestamp() { + return timestamp; + } + + public void setTimestamp(Long timestamp) { + this.timestamp = timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + kafkaState that = (kafkaState) o; + return partition == that.partition && + offset == that.offset && + timestamp == that.timestamp && + topic.equals(that.topic); + } + + @Override + public kafkaState clone() { + return new kafkaState(this.topic, this.partition, this.offset, this.timestamp); + } + + @Override + public int hashCode() { + return Objects.hash(topic, partition, offset, timestamp); + } + + @Override + public String toString() { + return "kafkaState{" + + "topic='" + topic + '\'' + + ", partition=" + partition + + ", offset=" + offset + + ", timestamp=" + timestamp + + '}'; + } +} diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/KafkaVersion.java similarity index 71% rename from flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java rename to flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/KafkaVersion.java index 74961c83c1..7c24b740a5 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/PlainDecoder.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/KafkaVersion.java @@ -6,31 +6,42 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.pulsar.writer.decoder; - -import java.util.Collections; -import java.util.Map; +package com.dtstack.flinkx.kafkabase.enums; /** - * Date: 2019/11/21 + * Date: 2020/12/26 * Company: www.dtstack.com * * @author tudou */ -public class PlainDecoder implements IDecode { +public enum KafkaVersion { + + unknown("-1"), + + kafka09("0.9"), - @Override - public Map decode(final String message) { - return Collections.singletonMap("message", message); + kafka10("0.10"), + + kafka11("0.11"), + + kafka("1"); + + private String code; + + KafkaVersion(String code) { + this.code = code; } + public String getCode() { + return code; + } } diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/StartupMode.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/StartupMode.java new file mode 100644 index 0000000000..1c99905dfb --- /dev/null +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/enums/StartupMode.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kafkabase.enums; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Locale; + +/** + * Date: 2019/11/21 + * Company: www.dtstack.com + * + * @author tudou + */ +public enum StartupMode { + + /** + * Start from committed offsets in ZK / Kafka brokers of a specific consumer group (default). + */ + GROUP_OFFSETS("group-offsets"), + /** + * Start from the earliest offset possible. + */ + EARLIEST("earliest-offset"), + /** + * Start from the latest offset. + */ + LATEST("latest-offset"), + /** + * Start from user-supplied timestamp for each partition. + */ + TIMESTAMP("timestamp"), + /** + * Start from user-supplied specific offsets for each partition + */ + SPECIFIC_OFFSETS("specific-offsets"), + + UNKNOWN("unknown"); + + public String name; + + StartupMode(String name) { + this.name = name; + } + + /** + * 根据名称获取启动模式 + * @param name + * @return + */ + public static StartupMode getFromName(String name){ + if(StringUtils.isBlank(name)){ + throw new IllegalArgumentException("StartupMode name is blank."); + } + switch (name.toLowerCase(Locale.ENGLISH)){ + case "earliest-offset": return EARLIEST; + case "latest-offset": return LATEST; + case "timestamp": return TIMESTAMP; + case "specific-offsets": return SPECIFIC_OFFSETS; + case "group-offsets": return GROUP_OFFSETS; + default: return UNKNOWN; + } + } +} diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/Formatter.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/Formatter.java similarity index 98% rename from flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/Formatter.java rename to flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/Formatter.java index 19b97ceeb9..1e492a00ae 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/Formatter.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/Formatter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase; +package com.dtstack.flinkx.kafkabase.util; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java new file mode 100644 index 0000000000..8cfa41b3c6 --- /dev/null +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.kafkabase.util; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Date: 2020/12/31 + * Company: www.dtstack.com + * + * @author tudou + */ +public class KafkaUtil { + + /** + * 解析kafka offset字符串 + * @param topic + * @param offsetString + * @return + * @throws IllegalArgumentException + */ + public static List parseSpecificOffsetsString(String topic, String offsetString) throws IllegalArgumentException{ + final String[] pairs = offsetString.split(ConstantValue.SEMICOLON_SYMBOL); + final String validationExceptionMessage = "Invalid properties [offset] should follow the format 'partition:0,offset:42;partition:1,offset:300', but is '" + offsetString + "';"; + + if (pairs.length == 0) { + throw new IllegalArgumentException(validationExceptionMessage); + } + + List list = new ArrayList<>(); + for (String pair : pairs) { + if (null == pair || pair.length() == 0 || !pair.contains(ConstantValue.COMMA_SYMBOL)) { + throw new IllegalArgumentException(validationExceptionMessage); + } + + final String[] kv = pair.split(ConstantValue.COMMA_SYMBOL); + if (kv.length != 2 || + !kv[0].startsWith("partition:") || + !kv[1].startsWith("offset:")) { + throw new IllegalArgumentException(validationExceptionMessage); + } + + String partitionValue = kv[0].substring(kv[0].indexOf(ConstantValue.COLON_SYMBOL) + 1); + String offsetValue = kv[1].substring(kv[1].indexOf(ConstantValue.COLON_SYMBOL) + 1); + try { + final Integer partition = Integer.valueOf(partitionValue); + final Long offset = Long.valueOf(offsetValue); + list.add(new kafkaState(topic, partition, offset, null)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException(validationExceptionMessage, e); + } + } + return list; + } + + /** + * 构造kafka properties + * @param consumerSettings + * @param mode + * @return + */ + public static Properties geneConsumerProp(Map consumerSettings, StartupMode mode) { + Properties props = new Properties(); + props.put("max.poll.interval.ms", "86400000"); + props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + props.put("enable.auto.commit", "false"); + for (Map.Entry entry : consumerSettings.entrySet()) { + String k = entry.getKey(); + String v = entry.getValue(); + props.put(k, v); + } + switch (mode){ + case EARLIEST: + props.put("auto.offset.reset", "earliest"); + case LATEST: + props.put("auto.offset.reset", "latest"); + } + return props; + } +} diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/IClient.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/IClient.java similarity index 74% rename from flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/IClient.java rename to flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/IClient.java index b3b9ad2b3e..9a3ffcc941 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/IClient.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/IClient.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.reader; +package com.dtstack.flinkx.kafkabase.client; /** * Date: 2019/12/25 @@ -32,10 +32,13 @@ public interface IClient extends Runnable { /** * 处理消息的方法 - * - * @param message 待处理的消息 + * @param message 待处理的消息 + * @param topic kafka topic + * @param partition kafka分区 + * @param offset kafka offset + * @param timestamp kafka timestamp */ - void processMessage(String message); + void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp); /** * 关闭连接 diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseConsumer.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/KafkaBaseConsumer.java similarity index 64% rename from flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseConsumer.java rename to flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/KafkaBaseConsumer.java index 8342f19fe7..89810927f0 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseConsumer.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/client/KafkaBaseConsumer.java @@ -14,8 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.reader; +package com.dtstack.flinkx.kafkabase.client; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; import org.apache.commons.lang3.concurrent.BasicThreadFactory; import java.util.Properties; @@ -28,33 +30,30 @@ * * @author tudou */ -public class KafkaBaseConsumer { +public abstract class KafkaBaseConsumer { protected Properties props; protected IClient client; protected ExecutorService executor = new ScheduledThreadPoolExecutor(1, new BasicThreadFactory .Builder() - .namingPattern("KafkaConsumerThread-%d") + .namingPattern("KafkaConsumerThread-" + Thread.currentThread().getName()) .daemon(true) .build()); public KafkaBaseConsumer(Properties properties) { - Properties props = new Properties(); - props.put("max.poll.interval.ms", "86400000"); - props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("auto.offset.reset", "earliest"); - if (properties != null) { - props.putAll(properties); - } - - this.props = props; + this.props = properties; } - public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format){ - throw new RuntimeException("KafkaBaseConsumer.createClient() should be override by subclass!"); - } + /** + * 创建kafka consumer + * @param topic kafka topic 多个,分割 + * @param group kafka消费组 + * @param format InputFormat + * @param kafkaInputSplit kafka数据分片 + * @return + */ + public abstract KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit); public void execute() { executor.execute(client); @@ -65,5 +64,4 @@ public void close() { client.close(); } } - } diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java similarity index 55% rename from flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java rename to flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java index 4f9dc3c288..678832cd3f 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java @@ -15,14 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.reader; +package com.dtstack.flinkx.kafkabase.format; import com.dtstack.flinkx.decoder.DecodeEnum; import com.dtstack.flinkx.decoder.IDecode; import com.dtstack.flinkx.decoder.JsonDecoder; -import com.dtstack.flinkx.decoder.PlainDecoder; +import com.dtstack.flinkx.decoder.TextDecoder; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.kafkabase.KafkaInputSplit; +import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; +import com.dtstack.flinkx.kafkabase.entity.kafkaState; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; @@ -30,8 +40,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.concurrent.BlockingQueue; import java.util.concurrent.SynchronousQueue; @@ -50,32 +61,48 @@ public class KafkaBaseInputFormat extends BaseRichInputFormat { protected String codec; protected boolean blankIgnore; protected String encoding; + protected StartupMode mode; + protected String offset; + protected Long timestamp; protected Map consumerSettings; + protected List metaColumns; + protected Map stateMap; protected volatile boolean running = false; protected transient BlockingQueue queue; protected transient KafkaBaseConsumer consumer; protected transient IDecode decode; + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) { + InputSplit[] splits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new GenericInputSplit(i, minNumSplits); + } + return splits; + } + @Override public void openInputFormat() throws IOException { super.openInputFormat(); - queue = new SynchronousQueue<>(false); + stateMap = new HashMap<>(16); if (DecodeEnum.JSON.getName().equalsIgnoreCase(codec)) { decode = new JsonDecoder(); } else { - decode = new PlainDecoder(); + decode = new TextDecoder(); } } @Override - protected void openInternal(InputSplit inputSplit) throws IOException { - consumer.createClient(topic, groupId, this).execute(); + protected void openInternal(InputSplit inputSplit) { + LOG.info("inputSplit = {}", inputSplit); + consumer.createClient(topic, groupId, this, (KafkaInputSplit) inputSplit).execute(); running = true; } @Override - protected Row nextRecordInternal(Row row) throws IOException { + protected Row nextRecordInternal(Row row) { try { row = queue.take(); } catch (InterruptedException e) { @@ -84,81 +111,85 @@ protected Row nextRecordInternal(Row row) throws IOException { return row; } - @Override - protected void closeInternal() throws IOException { - if (running) { - consumer.close(); - running = false; - LOG.warn("input kafka release."); - } - } - - public void processEvent(Map event) { + public void processEvent(Pair, kafkaState> pair) { try { - queue.put(Row.of(event)); + Row row; + if(CollectionUtils.isEmpty(metaColumns)){ + row = Row.of(Row.of(pair.getLeft())); + }else{ + row = new Row(metaColumns.size()); + for (int i = 0; i < metaColumns.size(); i++) { + MetaColumn metaColumn = metaColumns.get(i); + Object value = pair.getLeft().get(metaColumn.getName()); + Object obj = StringUtil.string2col(String.valueOf(value), metaColumn.getType(), metaColumn.getTimeFormat()); + row.setField(i , obj); + } + } + queue.put(row); + + kafkaState state = pair.getRight(); + stateMap.put(String.format("%s-%s", state.getTopic(), state.getPartition()), state); } catch (InterruptedException e) { - LOG.error("takeEvent interrupted event:{} error:{}", event, e); + LOG.error("takeEvent interrupted pair:{} error:{}", pair, e); } } - protected Properties geneConsumerProp() { - Properties props = new Properties(); - for (Map.Entry entry : consumerSettings.entrySet()) { - String k = entry.getKey(); - String v = entry.getValue(); - props.put(k, v); - } - return props; - } - @Override - protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { - InputSplit[] splits = new InputSplit[minNumSplits]; - for (int i = 0; i < minNumSplits; i++) { - splits[i] = new GenericInputSplit(i, minNumSplits); + public FormatState getFormatState() { + super.getFormatState(); + if (formatState != null) { + formatState.setState(stateMap); } - return splits; + return formatState; } @Override - public boolean reachedEnd() throws IOException { + public boolean reachedEnd() { return false; } + @Override + protected void closeInternal() { + if (running) { + consumer.close(); + running = false; + LOG.warn("input kafka release."); + } + } - public void setTopic(String topic) { - this.topic = topic; + /** + * 获取kafka版本信息 + * 0.9: kakfa09 + * 0.10: kafka10 + * @return + */ + public KafkaVersion getKafkaVersion() { + return KafkaVersion.unknown; } - public void setGroupId(String groupId) { - this.groupId = groupId; + /** + * 获取kafka state + * @return + */ + public Object getState(){ + return formatState == null ? null : formatState.getState(); } - public void setCodec(String codec) { - this.codec = codec; + public String getEncoding() { + return encoding; } - public void setBlankIgnore(boolean blankIgnore) { - this.blankIgnore = blankIgnore; + public IDecode getDecode() { + return decode; } public boolean getBlankIgnore() { return blankIgnore; } - public void setConsumerSettings(Map consumerSettings) { - this.consumerSettings = consumerSettings; + public StartupMode getMode() { + return mode; } - public String getEncoding() { - return encoding; - } - public void setEncoding(String encoding) { - this.encoding = encoding; - } - - public IDecode getDecode() { - return decode; - } } diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java new file mode 100644 index 0000000000..b140a6747d --- /dev/null +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.kafkabase.format; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; +import com.dtstack.flinkx.kafkabase.util.KafkaUtil; +import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Map; + + +/** + * Date: 2020/12/07 + * Company: www.dtstack.com + * + * @author tudou + */ +public class KafkaBaseInputFormatBuilder extends BaseRichInputFormatBuilder { + + private KafkaBaseInputFormat format; + + public KafkaBaseInputFormatBuilder(KafkaBaseInputFormat format) { + super.format = this.format = format; + } + + public void setTopic(String topic) { + format.topic = topic; + } + + public void setGroupId(String groupId) { + format.groupId = groupId; + } + + public void setCodec(String codec) { + format.codec = codec; + } + + public void setBlankIgnore(boolean blankIgnore) { + format.blankIgnore = blankIgnore; + } + + public void setConsumerSettings(Map consumerSettings) { + format.consumerSettings = consumerSettings; + } + + public void setEncoding(String encoding) { + format.encoding = encoding; + } + + public void setMode(StartupMode mode) { + format.mode = mode; + } + + public void setOffset(String offset) { + format.offset = offset; + } + + public void setTimestamp(Long timestamp) { + format.timestamp = timestamp; + } + + public void setMetaColumns(List metaColumns) { + format.metaColumns = metaColumns; + } + + @Override + protected void checkFormat() { + StringBuilder sb = new StringBuilder(128); + if(StringUtils.isBlank(format.topic)){ + sb.append("No kafka topic supplied;\n"); + } + + if(StartupMode.TIMESTAMP.equals(format.mode) || StartupMode.SPECIFIC_OFFSETS.equals(format.mode)){ + if(format.topic.split(ConstantValue.COMMA_SYMBOL).length > 1){ + sb.append("mode [") + .append(format.mode) + .append("] is not supported when the number of kafka topic bigger than 1;\n"); + }else if(StartupMode.TIMESTAMP.equals(format.mode) && format.timestamp < 0){ + sb.append("No kafka timestamp supplied when mode is [") + .append(StartupMode.TIMESTAMP.name()) + .append("];\n"); + }else if(StartupMode.SPECIFIC_OFFSETS.equals(format.mode)){ + try { + KafkaUtil.parseSpecificOffsetsString(format.topic, format.offset); + }catch (IllegalArgumentException e){ + sb.append(e.getMessage()).append("\n"); + } + } + }else if(StartupMode.UNKNOWN.equals(format.mode)){ + sb.append("parameter [mode] config error, the value of mode must in [group-offsets, earliest-offset, latest-offset, timestamp, specific-offsets]\n"); + } + + if(!KafkaVersion.kafka09.equals(format.getKafkaVersion())){ + if (!format.consumerSettings.containsKey(KafkaConfigKeys.BOOTSTRAP_SERVERS)) { + sb.append("parameter [") + .append(KafkaConfigKeys.BOOTSTRAP_SERVERS) + .append("] must set in consumerSettings;\n"); + } + } + + if(sb.length() > 0){ + throw new IllegalArgumentException(sb.toString()); + } + } +} diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java index f9e6c593a7..dacfc4ba0e 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java @@ -19,21 +19,20 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; +import com.dtstack.flinkx.kafkabase.enums.StartupMode; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.reader.MetaColumn; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_BLANK_IGNORE; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CODEC; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_CONSUMER_SETTINGS; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_GROUP_ID; -import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TOPIC; - /** * Date: 2019/11/21 * Company: www.dtstack.com @@ -41,11 +40,14 @@ * @author tudou */ public class KafkaBaseReader extends BaseDataReader { - protected String topic; protected String groupId; protected String codec; protected boolean blankIgnore; + protected String encoding; + protected String mode; + protected String offset; + protected Long timestamp; protected Map consumerSettings; protected List metaColumns; @@ -53,28 +55,41 @@ public class KafkaBaseReader extends BaseDataReader { public KafkaBaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - topic = readerConfig.getParameter().getStringVal(KEY_TOPIC); - groupId = readerConfig.getParameter().getStringVal(KEY_GROUP_ID); - codec = readerConfig.getParameter().getStringVal(KEY_CODEC, "plain"); - blankIgnore = readerConfig.getParameter().getBooleanVal(KEY_BLANK_IGNORE, false); - consumerSettings = (Map) readerConfig.getParameter().getVal(KEY_CONSUMER_SETTINGS); + topic = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_TOPIC); + groupId = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_GROUP_ID, "default"); + codec = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_CODEC, "text"); + blankIgnore = readerConfig.getParameter().getBooleanVal(KafkaConfigKeys.KEY_BLANK_IGNORE, false); + encoding = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_ENCODING, StandardCharsets.UTF_8.name()); + mode = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_MODE, StartupMode.GROUP_OFFSETS.name); + offset = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_OFFSET, ""); + timestamp = readerConfig.getParameter().getLongVal(KafkaConfigKeys.KEY_TIMESTAMP, -1L); + consumerSettings = (Map) readerConfig.getParameter().getVal(KafkaConfigKeys.KEY_CONSUMER_SETTINGS); metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); } @Override public DataStream readData() { - KafkaBaseInputFormat format = getFormat(); - format.setDataTransferConfig(dataTransferConfig); - format.setTopic(topic); - format.setGroupId(groupId); - format.setCodec(codec); - format.setBlankIgnore(blankIgnore); - format.setConsumerSettings(consumerSettings); - format.setRestoreConfig(restoreConfig); - return createInput(format); + KafkaBaseInputFormatBuilder builder = getBuilder(); + builder.setDataTransferConfig(dataTransferConfig); + builder.setRestoreConfig(restoreConfig); + builder.setTopic(topic); + builder.setGroupId(groupId); + builder.setCodec(codec); + builder.setBlankIgnore(blankIgnore); + builder.setEncoding(encoding); + builder.setConsumerSettings(consumerSettings); + builder.setMode(StartupMode.getFromName(mode)); + builder.setOffset(offset); + builder.setTimestamp(timestamp); + builder.setMetaColumns(metaColumns); + return createInput(builder.finish()); } - public KafkaBaseInputFormat getFormat(){ - return new KafkaBaseInputFormat(); + /** + * 获取不同版本的kafkaInputFormat + * @return + */ + public KafkaBaseInputFormatBuilder getBuilder(){ + return new KafkaBaseInputFormatBuilder(new KafkaBaseInputFormat()); } } diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java similarity index 77% rename from flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java rename to flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java index 3c8f7c076d..9d71909132 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseOutputFormat.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java @@ -15,16 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafkabase.writer; +package com.dtstack.flinkx.kafkabase.format; import com.dtstack.flinkx.config.RestoreConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.decoder.JsonDecoder; +import com.dtstack.flinkx.exception.DataSourceException; import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.TelnetUtil; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; -import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +47,8 @@ public class KafkaBaseOutputFormat extends BaseRichOutputFormat { protected static final Logger LOG = LoggerFactory.getLogger(KafkaBaseOutputFormat.class); + //producer.close最长等待时间 + protected static final long CLOSE_TIME = 20000L; protected Properties props = new Properties(); protected String timezone; @@ -51,20 +56,21 @@ public class KafkaBaseOutputFormat extends BaseRichOutputFormat { protected Map producerSettings; protected List tableFields; protected static JsonDecoder jsonDecoder = new JsonDecoder(); - protected static ObjectMapper objectMapper = new ObjectMapper(); - //连续发送数据错误次数 - protected int failedTimes = 0; + //和kafkaBroker连通性控制器 + protected HeartBeatController heartBeatController; @Override public void configure(Configuration parameters) { - if(producerSettings != null && producerSettings.get("bootstrap.servers") != null){ + if(producerSettings != null && producerSettings.containsKey("bootstrap.servers")){ String brokerList = producerSettings.get("bootstrap.servers"); LOG.info("brokerList->{}",brokerList); - String broker = brokerList.split(",")[0]; - String[] split = broker.split(":"); + String broker = brokerList.split(ConstantValue.COMMA_SYMBOL)[0]; + String[] split = broker.split(ConstantValue.COLON_SYMBOL); - if( split.length !=2 ||!AddressUtil.telnet(split[0], Integer.parseInt(split[1]))){ - throw new RuntimeException("telnet error,brokerList"+brokerList+" please check dataSource is running"); + try { + TelnetUtil.telnet(split[0], Integer.parseInt(split[1])); + }catch (Exception e){ + throw new RuntimeException("telnet error, brokerList = " + brokerList); } } } @@ -98,15 +104,14 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { } } emit(map); - //只要有正常的 重置为0 - failedTimes =0; } catch (Throwable e) { String errorMessage = ExceptionUtil.getErrorMessage(e); LOG.error("kafka writeSingleRecordInternal error:{}", errorMessage); - //连续发送3次数据错误 或者出现broker 连接异常,就直接认为数据源异常,退出任务 或者出现broker连接不上,超时直接抛出异常 - if(++failedTimes >= 3 || e.getMessage().contains("Broker may not be available")||e.getMessage().contains("TimeoutException")){ - throw new RuntimeException("Error data is received 3 times continuously or datasource has error"+errorMessage, e); + //如果是数据源错误 直接抛出异常,而不是封装为WriteRecordException + // 否则WriteRecordException会被上层捕获,导致任务无法结束 + if(e instanceof DataSourceException){ + throw (DataSourceException)e; } throw new WriteRecordException(errorMessage, e); } @@ -150,4 +155,7 @@ public void setTableFields(List tableFields) { this.tableFields = tableFields; } + public void setHeartBeatController(HeartBeatController heartBeatController) { + this.heartBeatController = heartBeatController; + } } diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/AddressUtil.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/AddressUtil.java deleted file mode 100644 index 54a3c28f6d..0000000000 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/AddressUtil.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.kafkabase.writer; - -import org.apache.commons.net.telnet.TelnetClient; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.InetAddress; - -/** - * @author jiangbo - * @date 2020/3/2 - */ -public class AddressUtil { - - private static Logger logger = LoggerFactory.getLogger(AddressUtil.class); - - public static boolean telnet(String ip,int port){ - TelnetClient client = null; - try{ - client = new TelnetClient(); - client.setConnectTimeout(3000); - client.connect(ip,port); - return true; - }catch(Exception e){ - return false; - } finally { - try { - if (client != null){ - client.disconnect(); - } - } catch (Exception e){ - logger.error("{}",e); - } - } - } - - public static boolean ping(String ip){ - try{ - return InetAddress.getByName(ip).isReachable(3000); - }catch(Exception e){ - return false; - } - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/HeartBeatController.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/HeartBeatController.java similarity index 81% rename from flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/HeartBeatController.java rename to flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/HeartBeatController.java index ebff662163..7aca7879c2 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/HeartBeatController.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/HeartBeatController.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.kafka09.writer; +package com.dtstack.flinkx.kafkabase.writer; +import com.dtstack.flinkx.exception.DataSourceException; import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.kafka.common.errors.TimeoutException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,11 +63,11 @@ public void acquire() { if (Objects.isNull(e)) { return; } - //连续发送3次数据错误或出现连接异常 - if (failedTimes.get() >= detectingRetryTimes || e instanceof TimeoutException ) { - String message = "Error data is received 3 times continuously or datasource has error" + ExceptionUtil.getErrorMessage(e); + //连续发送3次数据错误 + if (failedTimes.get() >= detectingRetryTimes ) { + String message = "Failed to send data for three consecutive times,Please check whether the data source is normal,errorInfo->" + ExceptionUtil.getErrorMessage(e); logger.error(message); - throw new RuntimeException(message, e); + throw new DataSourceException("kafka",message, e); } } } diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java index 3ae4c05c7f..b320303c0b 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java @@ -19,6 +19,7 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; import com.dtstack.flinkx.writer.BaseDataWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -62,7 +63,9 @@ public DataStreamSink writeData(DataStream dataSet) { format.setProducerSettings(producerSettings); format.setRestoreConfig(restoreConfig); format.setTableFields(tableFields); - + format.setDirtyPath(dirtyPath); + format.setDirtyHadoopConfig(dirtyHadoopConfig); + format.setSrcFieldNames(srcCols); return createOutput(dataSet, format); } } diff --git a/flinkx-kb/pom.xml b/flinkx-kb/pom.xml index c11d9c32a8..c322923a89 100644 --- a/flinkx-kb/pom.xml +++ b/flinkx-kb/pom.xml @@ -24,6 +24,16 @@ flinkx-core 1.6 provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + diff --git a/flinkx-kingbase/flinkx-kingbase-core/pom.xml b/flinkx-kingbase/flinkx-kingbase-core/pom.xml new file mode 100644 index 0000000000..899f3ecdfd --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-core/pom.xml @@ -0,0 +1,24 @@ + + + + flinkx-kingbase + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-kingbase-core + + + + com.kingbase8 + kingbase8 + 8.2.0 + + + + + + \ No newline at end of file diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java new file mode 100644 index 0000000000..e271291b15 --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/constants/KingbaseCons.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.constants; + +/** + * kingbase常量 + * + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class KingbaseCons { + + public static final String DRIVER = "com.kingbase8.Driver"; + /** + * kingbase 主键索引名后缀 + */ + public static final String KEY_UPDATE_KEY = "key"; + + public static final String KEY_PRIMARY_SUFFIX = "_PKEY"; + + public static final String INSERT_SQL_MODE_TYPE = "copy"; + + /** + * copy语法分隔符 + */ + public static final String DEFAULT_FIELD_DELIM = "\001"; + public static final String DEFAULT_NULL_DELIM = "\002"; + public static final String LINE_DELIMITER = "\n"; + +} diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java new file mode 100644 index 0000000000..3a6a27811e --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseDatabaseMeta.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.util; + +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.constants.ConstantValue.COMMA_SYMBOL; +import static com.dtstack.flinkx.constants.ConstantValue.LEFT_PARENTHESIS_SYMBOL; +import static com.dtstack.flinkx.constants.ConstantValue.RIGHT_PARENTHESIS_SYMBOL; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DRIVER; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.KEY_PRIMARY_SUFFIX; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.KEY_UPDATE_KEY; + +/** + * The class of KingBase database prototype + * + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class KingBaseDatabaseMeta extends BaseDatabaseMeta { + + @Override + protected String makeValues(List column) { + StringBuilder sb = new StringBuilder(); + sb.append(LEFT_PARENTHESIS_SYMBOL); + for(int i = 0; i < column.size(); ++i) { + if(i != 0) { + sb.append(COMMA_SYMBOL); + } + sb.append(quoteColumn(column.get(i))); + } + sb.append(RIGHT_PARENTHESIS_SYMBOL); + return sb.toString(); + } + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.KingBase; + } + + @Override + public String getDriverClass() { + return DRIVER; + } + + @Override + public String getSqlQueryFields(String tableName) { + return "SELECT * FROM " + tableName + " LIMIT 0"; + } + + @Override + public String getSqlQueryColumnFields(List column, String table) { + return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; + } + + /** + * Kingbase 的主键索引名为TABLE_PKEY格式 + * @param column column名 + * @param table 表名 + * @param updateKey 索引 + * @return updateSql + */ + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + List columnList = new LinkedList<>(); + updateKey.forEach((key, value) -> { + // 兼顾查询主键索引名或者填入key map的情况 + if (StringUtils.endsWith(key, KEY_PRIMARY_SUFFIX) || StringUtils.equals(key, KEY_UPDATE_KEY)) { + columnList.addAll(value); + } + }); + return "INSERT INTO " + quoteTable(table) + + " (" + quoteColumns(column) + ") VALUES " + + makeValues(column.size()) + + " ON CONFLICT " +makeValues(columnList) + " DO UPDATE SET " + + makeUpdatePart(column); + } + + private String makeValues(int nCols) { + return LEFT_PARENTHESIS_SYMBOL + StringUtils.repeat("?", ",", nCols) + RIGHT_PARENTHESIS_SYMBOL; + } + + private String makeUpdatePart (List column) { + List updateList = new ArrayList<>(); + for(String col : column) { + String quotedCol = quoteColumn(col); + updateList.add(quotedCol + "=EXCLUDED." + quotedCol); + } + return StringUtils.join(updateList, COMMA_SYMBOL); + } + + @Override + public String quoteValue(String value, String column) { + return String.format("\"%s\" as %s",value,column); + } + + @Override + public String getSplitFilter(String columnName) { + return String.format("mod(%s, ${N}) = ${M}", getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName) { + return String.format("mod(%s.%s, ${N}) = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getStartQuote() { + return ""; + } + + @Override + public String getEndQuote() { + return ""; + } +} diff --git a/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java new file mode 100644 index 0000000000..586b6892c1 --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-core/src/main/java/com/dtstack/flinkx/kingbase/util/KingBaseTypeConverter.java @@ -0,0 +1,54 @@ +package com.dtstack.flinkx.kingbase.util; + +import com.dtstack.flinkx.rdb.type.TypeConverterInterface; +import org.apache.commons.lang3.StringUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * kingbase中不常用类型转换为java类型 + * @Company: www.dtstack.com + * @author kunni@dtstack.com + */ +public class KingBaseTypeConverter implements TypeConverterInterface { + + private List stringTypes = Arrays.asList("uuid", "xml", "cidr", "inet", "macaddr", "text", "character", "character varying"); + + private List byteTypes = Arrays.asList("bytea","bit varying"); + + private List bitTypes = Collections.singletonList("bit"); + + private List doubleTypes = Arrays.asList("double precision","double","float8","money"); + + private List intTypes = Arrays.asList("int","int2","int4","int8","integer","bigint","bigserial","smallint"); + + @Override + public Object convert(Object data,String typeName) { + if (data == null){ + return null; + } + String dataValue = data.toString(); + if(stringTypes.contains(typeName)){ + return dataValue; + } + if(StringUtils.isBlank(dataValue)){ + return null; + } + if(doubleTypes.contains(typeName)){ + if(StringUtils.startsWith(dataValue, "$")){ + dataValue = StringUtils.substring(dataValue, 1); + } + data = Double.parseDouble(dataValue); + } else if(bitTypes.contains(typeName)){ + // + }else if(byteTypes.contains(typeName)){ + data = Byte.valueOf(dataValue); + } else if(intTypes.contains(typeName)){ + data = Integer.parseInt(dataValue); + } + + return data; + } +} diff --git a/flinkx-kingbase/flinkx-kingbase-reader/pom.xml b/flinkx-kingbase/flinkx-kingbase-reader/pom.xml new file mode 100644 index 0000000000..38c18a7e05 --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-reader/pom.xml @@ -0,0 +1,107 @@ + + + + flinkx-kingbase + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-kingbase-reader + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + com.dtstack.flinkx + flinkx-kingbase-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.kingbasereader.io.netty + + + com.google.common + shade.kingbasereader.com.google.common + + + com.google.thirdparty + shade.kingbasereader.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java new file mode 100644 index 0000000000..0ff86b4a5e --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseInputFormat.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.format; + +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.sql.ResultSet; + +import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; + +/** + * + * + * Company: www.dtstack.com + * @author kunni@Dtstack.com + */ + +public class KingbaseInputFormat extends JdbcInputFormat { + + /** + * 避免Operation requires a scrollable ResultSet问题 + * @param inputSplit 分片 + * @throws IOException IO异常 + */ + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + if(incrementConfig.isPolling()){ + resultSetType = ResultSet.TYPE_SCROLL_INSENSITIVE; + } + super.openInternal(inputSplit); + } + + @Override + public Row nextRecordInternal(Row row) throws IOException { + if (!hasNext) { + return null; + } + row = new Row(columnCount); + + try { + for (int pos = 0; pos < row.getArity(); pos++) { + Object obj = resultSet.getObject(pos + 1); + if(obj != null) { + if(CollectionUtils.isNotEmpty(columnTypeList)) { + String columnType = columnTypeList.get(pos); + if("year".equalsIgnoreCase(columnType)) { + java.util.Date date = (java.util.Date) obj; + obj = DateUtil.dateToYearString(date); + } else if("tinyint".equalsIgnoreCase(columnType) + || "bit".equalsIgnoreCase(columnType)) { + if(obj instanceof Boolean) { + obj = ((Boolean) obj ? 1 : 0); + } + } + } + obj = clobToString(obj); + } + + row.setField(pos, obj); + } + return super.nextRecordInternal(row); + }catch (Exception e) { + throw new IOException("Couldn't read data - " + ExceptionUtil.getErrorMessage(e), e); + } + } + +} diff --git a/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java new file mode 100644 index 0000000000..a6625225a8 --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-reader/src/main/java/com/dtstack/flinkx/kingbase/reader/KingbaseReader.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kingbase.util.KingBaseDatabaseMeta; +import com.dtstack.flinkx.kingbase.format.KingbaseInputFormat; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +/** + * KingBase reader plugin + * + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class KingbaseReader extends JdbcDataReader { + + public KingbaseReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + String schema = config.getJob().getContent().get(0).getReader().getParameter().getConnection().get(0).getSchema(); + table = schema + ConstantValue.POINT_SYMBOL + table; + setDatabaseInterface(new KingBaseDatabaseMeta()); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + return new JdbcInputFormatBuilder(new KingbaseInputFormat()); + } +} diff --git a/flinkx-kingbase/flinkx-kingbase-writer/pom.xml b/flinkx-kingbase/flinkx-kingbase-writer/pom.xml new file mode 100644 index 0000000000..5a1bd1a2bc --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-writer/pom.xml @@ -0,0 +1,106 @@ + + + + flinkx-kingbase + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-kingbase-writer + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + com.dtstack.flinkx + flinkx-kingbase-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.kingbasewriter.io.netty + + + com.google.common + shade.kingbasewriter.com.google.common + + + com.google.thirdparty + shade.kingbasewriter.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java new file mode 100644 index 0000000000..8696d1db82 --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.format; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.enums.EWriteMode; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.kingbase8.copy.CopyManager; +import com.kingbase8.core.BaseConnection; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.types.Row; + +import java.io.ByteArrayInputStream; +import java.nio.charset.StandardCharsets; +import java.sql.PreparedStatement; +import java.sql.SQLException; + +import static com.dtstack.flinkx.constants.ConstantValue.COMMA_SYMBOL; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DEFAULT_FIELD_DELIM; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.DEFAULT_NULL_DELIM; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.INSERT_SQL_MODE_TYPE; +import static com.dtstack.flinkx.kingbase.constants.KingbaseCons.LINE_DELIMITER; + +/** + * 写入数据到kingbase + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class KingbaseOutputFormat extends JdbcOutputFormat { + + private static final String COPY_SQL_TEMPL = "COPY %s(%s) FROM STDIN DELIMITER '%s' NULL AS '%s'"; + + private String copySql = ""; + + private CopyManager copyManager; + + /** + * schema名 + */ + public String schema; + + @Override + protected void openInternal(int taskNumber, int numTasks){ + try { + ClassUtil.forName(driverName, getClass().getClassLoader()); + dbConn = DbUtil.getConnection(dbUrl, username, password); + + if (restoreConfig.isRestore()){ + dbConn.setAutoCommit(false); + } + // 查询主键时用table格式 + if(CollectionUtils.isEmpty(fullColumn)) { + fullColumn = probeFullColumns(table, dbConn); + } + + if (!EWriteMode.INSERT.name().equalsIgnoreCase(mode)){ + if(updateKey == null || updateKey.size() == 0) { + updateKey = probePrimaryKeys(table, dbConn); + } + } + // 其他情况,使用schema.table作为表名 + table = schema + ConstantValue.POINT_SYMBOL + table; + if(fullColumnType == null) { + fullColumnType = analyzeTable(); + } + + for(String col : column) { + for (int i = 0; i < fullColumn.size(); i++) { + if (col.equalsIgnoreCase(fullColumn.get(i))){ + columnType.add(fullColumnType.get(i)); + break; + } + } + } + + preparedStatement = prepareTemplates(); + readyCheckpoint = false; + + LOG.info("subTask[{}}] wait finished", taskNumber); + } catch (SQLException sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } + try { + if (batchInterval > 1) { + dbConn.setAutoCommit(false); + } + } catch (Exception e) { + LOG.warn(ExceptionUtil.getErrorMessage(e)); + } + } + + @Override + protected PreparedStatement prepareTemplates() throws SQLException { + if(CollectionUtils.isEmpty(fullColumn)) { + fullColumn = column; + } + + //check is use copy mode for insert + if (EWriteMode.INSERT.name().equalsIgnoreCase(mode) && checkIsCopyMode(insertSqlMode)) { + copyManager = new CopyManager((BaseConnection) dbConn); + copySql = String.format(COPY_SQL_TEMPL, table, String.join(COMMA_SYMBOL, column), DEFAULT_FIELD_DELIM, DEFAULT_NULL_DELIM); + return null; + } + + return super.prepareTemplates(); + } + + @Override + protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + if(!checkIsCopyMode(insertSqlMode)){ + if (batchInterval == 1) { + super.writeSingleRecordInternal(row); + } else { + writeSingleRecordCommit(row); + } + + return; + } + + //write with copy + int index = 0; + try { + StringBuilder sb = new StringBuilder(); + for (; index < row.getArity(); index++) { + Object rowData = getField(row, index); + sb.append(rowData) + .append(DEFAULT_FIELD_DELIM); + } + + String rowVal = sb.toString(); + ByteArrayInputStream bi = new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8)); + copyManager.copyIn(copySql, bi); + } catch (Exception e) { + processWriteException(e, index, row); + } + } + + private void writeSingleRecordCommit(Row row) throws WriteRecordException { + try { + super.writeSingleRecordInternal(row); + try { + dbConn.commit(); + } catch (Exception e) { + // 提交失败直接结束任务 + throw new RuntimeException(e); + } + } catch (WriteRecordException e) { + try { + dbConn.rollback(); + } catch (Exception e1) { + // 回滚失败直接结束任务 + throw new RuntimeException(e); + } + + throw e; + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + if(!checkIsCopyMode(insertSqlMode)){ + writeMultipleRecordsCommit(); + return; + } + + StringBuilder sb = new StringBuilder(128); + for (Row row : rows) { + int lastIndex = row.getArity() - 1; + for (int index = 0; index < row.getArity(); index++) { + Object rowData = getField(row, index); + sb.append(rowData==null ? DEFAULT_NULL_DELIM : rowData); + if(index != lastIndex){ + sb.append(DEFAULT_FIELD_DELIM); + } + } + + sb.append(LINE_DELIMITER); + } + + String rowVal = sb.toString(); + ByteArrayInputStream bi = new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8)); + copyManager.copyIn(copySql, bi); + + if(restoreConfig.isRestore()){ + rowsOfCurrentTransaction += rows.size(); + } + } + + private void writeMultipleRecordsCommit() throws Exception { + try { + super.writeMultipleRecordsInternal(); + dbConn.commit(); + } catch (Exception e){ + dbConn.rollback(); + throw e; + } + } + + @Override + protected Object getField(Row row, int index) { + Object field = super.getField(row, index); + String type = columnType.get(index); + field = typeConverter.convert(field,type); + + return field; + } + + private boolean checkIsCopyMode(String insertMode){ + if(StringUtils.isEmpty(insertMode)){ + return false; + } + + if(!INSERT_SQL_MODE_TYPE.equalsIgnoreCase(insertMode)){ + throw new RuntimeException("not support insertSqlMode:" + insertMode); + } + + return true; + } + + @Override + public void setSchema(String schema){ + this.schema = schema; + } +} diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java new file mode 100644 index 0000000000..d11b9e1fcc --- /dev/null +++ b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.kingbase.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.kingbase.util.KingBaseDatabaseMeta; +import com.dtstack.flinkx.kingbase.util.KingBaseTypeConverter; +import com.dtstack.flinkx.kingbase.format.KingbaseOutputFormat; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.streaming.api.functions.sink.DtOutputFormatSinkFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import static com.dtstack.flinkx.constants.ConfigConstant.KEY_WRITER; + +/** + * KingBase writer plugin + * + * Company: www.dtstack.com + * @author kunni@dtstack.com + */ + +public class KingbaseWriter extends JdbcDataWriter { + + public String schema; + + public KingbaseWriter(DataTransferConfig config) { + super(config); + schema = config.getJob().getContent().get(0).getWriter().getParameter().getConnection().get(0).getSchema(); + setDatabaseInterface(new KingBaseDatabaseMeta()); + setTypeConverterInterface(new KingBaseTypeConverter()); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + KingbaseOutputFormat kingBaseOutputFormat = new KingbaseOutputFormat(); + kingBaseOutputFormat.setSchema(schema); + JdbcOutputFormatBuilder builder = new JdbcOutputFormatBuilder(kingBaseOutputFormat); + builder.setDriverName(databaseInterface.getDriverClass()); + builder.setDbUrl(dbUrl); + builder.setUsername(username); + builder.setPassword(password); + builder.setBatchInterval(batchSize); + builder.setMonitorUrls(monitorUrls); + builder.setPreSql(preSql); + builder.setPostSql(postSql); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setDatabaseInterface(databaseInterface); + builder.setMode(mode); + builder.setTable(table); + builder.setColumn(column); + builder.setFullColumn(fullColumn); + builder.setUpdateKey(updateKey); + builder.setTypeConverter(typeConverter); + builder.setRestoreConfig(restoreConfig); + builder.setInsertSqlMode(insertSqlMode); + + DtOutputFormatSinkFunction sinkFunction = new DtOutputFormatSinkFunction(builder.finish()); + DataStreamSink dataStreamSink = dataSet.addSink(sinkFunction); + String sinkName = (databaseInterface.getDatabaseType() + KEY_WRITER).toLowerCase(); + dataStreamSink.name(sinkName); + return dataStreamSink; + } +} diff --git a/flinkx-kingbase/pom.xml b/flinkx-kingbase/pom.xml new file mode 100644 index 0000000000..14ceabbcf7 --- /dev/null +++ b/flinkx-kingbase/pom.xml @@ -0,0 +1,35 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + pom + + flinkx-kingbase-core + flinkx-kingbase-reader + flinkx-kingbase-writer + + + flinkx-kingbase + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java index 4eb6ec865a..3fc1f31960 100644 --- a/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java +++ b/flinkx-kudu/flinkx-kudu-core/src/main/java/com/dtstack/flinkx/kudu/core/KuduUtil.java @@ -19,8 +19,10 @@ package com.dtstack.flinkx.kudu.core; +import com.dtstack.flinkx.authenticate.KerberosUtil; import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.FileSystemUtil; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.math.NumberUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -57,10 +59,18 @@ public class KuduUtil { public final static String AUTHENTICATION_TYPE = "Kerberos"; - public static KuduClient getKuduClient(KuduConfig config) throws IOException,InterruptedException { - if(AUTHENTICATION_TYPE.equals(config.getAuthentication())){ - UserGroupInformation.loginUserFromKeytab(config.getPrincipal(), config.getKeytabFile()); - return UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction() { + /** + * 获取kudu的客户端 + * @param config kudu的配置信息 + * @param hadoopConfig hadoop相关信息 主要需要kerberos相关验证信息 + * @return + * @throws IOException + * @throws InterruptedException + */ + public static KuduClient getKuduClient(KuduConfig config, Map hadoopConfig) throws IOException,InterruptedException { + if(AUTHENTICATION_TYPE.equals(config.getAuthentication()) && FileSystemUtil.isOpenKerberos(hadoopConfig)){ + UserGroupInformation ugi = FileSystemUtil.getUGI(hadoopConfig,null); + return ugi.doAs(new PrivilegedExceptionAction() { @Override public KuduClient run() throws Exception { return getKuduClientInternal(config); @@ -81,9 +91,9 @@ private static KuduClient getKuduClientInternal(KuduConfig config) { .syncClient(); } - public static List getKuduScanToken(KuduConfig config, List columns, String filterString) throws IOException{ + public static List getKuduScanToken(KuduConfig config, List columns, String filterString, Map hadoopConfig) throws IOException{ try ( - KuduClient client = getKuduClient(config) + KuduClient client = getKuduClient(config, hadoopConfig) ) { KuduTable kuduTable = client.openTable(config.getTable()); @@ -157,6 +167,7 @@ public static Type getType(String columnType){ case "varchar": case "text": case "string" : return Type.STRING; + case "unixtime_micros": case "timestamp" : return Type.UNIXTIME_MICROS; default: throw new IllegalArgumentException("Not support column type:" + columnType); diff --git a/flinkx-kudu/flinkx-kudu-reader/pom.xml b/flinkx-kudu/flinkx-kudu-reader/pom.xml index 0c97772391..53568ff52a 100644 --- a/flinkx-kudu/flinkx-kudu-reader/pom.xml +++ b/flinkx-kudu/flinkx-kudu-reader/pom.xml @@ -80,6 +80,7 @@ + diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java index 6be5434a52..0f637dcd84 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormat.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; /** * @author jiangbo @@ -45,6 +46,8 @@ public class KuduInputFormat extends BaseRichInputFormat { protected KuduConfig kuduConfig; + protected Map hadoopConfig; + private transient KuduClient client; private transient KuduScanner scanner; @@ -57,7 +60,7 @@ public void openInputFormat() throws IOException { super.openInputFormat(); try { - client = KuduUtil.getKuduClient(kuduConfig); + client = KuduUtil.getKuduClient(kuduConfig, hadoopConfig); } catch (IOException | InterruptedException e) { throw new RuntimeException("Get KuduClient error", e); } @@ -122,7 +125,7 @@ private Object getValue(Type type, RowResult rowResult, String name) { @Override public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { LOG.info("execute createInputSplits,minNumSplits:{}", minNumSplits); - List scanTokens = KuduUtil.getKuduScanToken(kuduConfig, columns, kuduConfig.getFilterString()); + List scanTokens = KuduUtil.getKuduScanToken(kuduConfig, columns, kuduConfig.getFilterString(), hadoopConfig); KuduTableSplit[] inputSplits = new KuduTableSplit[scanTokens.size()]; for (int i = 0; i < scanTokens.size(); i++) { inputSplits[i] = new KuduTableSplit(scanTokens.get(i).serialize(), i); @@ -168,4 +171,12 @@ public void closeInputFormat() throws IOException { client = null; } } + + public Map getHadoopConfig() { + return hadoopConfig; + } + + public void setHadoopConfig(Map hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } } diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java index 4959731c76..68f0c1d00d 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduInputFormatBuilder.java @@ -25,6 +25,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; +import java.util.Map; /** * @author jiangbo @@ -46,6 +47,10 @@ public void setKuduConfig(KuduConfig kuduConfig){ format.kuduConfig = kuduConfig; } + public void setHadoopConfig(Map hadoopConfig) { + format.setHadoopConfig(hadoopConfig); + } + @Override protected void checkFormat() { if (format.columns == null || format.columns.size() == 0){ diff --git a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java index 55d86d1af8..26fce0f837 100644 --- a/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java +++ b/flinkx-kudu/flinkx-kudu-reader/src/main/java/com/dtstack/flinkx/kudu/reader/KuduReader.java @@ -31,6 +31,7 @@ import org.apache.kudu.client.AsyncKuduClient; import java.util.List; +import java.util.Map; import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; @@ -56,6 +57,8 @@ public class KuduReader extends BaseDataReader { private KuduConfig kuduConfig; + protected Map hadoopConfig; + public KuduReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); @@ -78,6 +81,8 @@ public KuduReader(DataTransferConfig config, StreamExecutionEnvironment env) { .withBatchSizeBytes(parameterConfig.getIntVal(KEY_BATCH_SIZE_BYTES, 1024*1024)) .withFilter(parameterConfig.getStringVal(KEY_FILTER)) .build(); + + hadoopConfig = (Map) readerConfig.getParameter().getVal("hadoopConfig"); } @Override @@ -90,6 +95,7 @@ public DataStream readData() { builder.setKuduConfig(kuduConfig); builder.setTestConfig(testConfig); builder.setLogConfig(logConfig); + builder.setHadoopConfig(hadoopConfig); return createInput(builder.finish()); } diff --git a/flinkx-kudu/flinkx-kudu-writer/pom.xml b/flinkx-kudu/flinkx-kudu-writer/pom.xml index a2d5dc9967..c0030ab1bd 100644 --- a/flinkx-kudu/flinkx-kudu-writer/pom.xml +++ b/flinkx-kudu/flinkx-kudu-writer/pom.xml @@ -80,6 +80,7 @@ + diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java index f7420e4f15..be730c4aff 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormat.java @@ -43,6 +43,7 @@ import java.sql.Date; import java.sql.Timestamp; import java.util.List; +import java.util.Map; /** * @author jiangbo @@ -58,6 +59,8 @@ public class KuduOutputFormat extends BaseRichOutputFormat { private transient KuduClient client; + protected Map hadoopConfig; + private transient KuduSession session; private transient KuduTable kuduTable; @@ -65,7 +68,7 @@ public class KuduOutputFormat extends BaseRichOutputFormat { @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { try{ - client = KuduUtil.getKuduClient(kuduConfig); + client = KuduUtil.getKuduClient(kuduConfig, hadoopConfig); } catch (Exception e){ throw new RuntimeException("Get KuduClient error", e); } @@ -197,4 +200,12 @@ public void closeInternal() throws IOException { client.close(); } } + + public Map getHadoopConfig() { + return hadoopConfig; + } + + public void setHadoopConfig(Map hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } } diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormatBuilder.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormatBuilder.java index 320e2df95a..16f8881f7c 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormatBuilder.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduOutputFormatBuilder.java @@ -24,6 +24,7 @@ import com.dtstack.flinkx.reader.MetaColumn; import java.util.List; +import java.util.Map; /** * @author jiangbo @@ -49,6 +50,9 @@ public void setWriteMode(String writeMode){ format.writeMode = writeMode; } + public void setHadoopConfig(Map hadoopConfig) { + format.setHadoopConfig(hadoopConfig); + } @Override protected void checkFormat() { if (format.columns == null || format.columns.size() == 0){ diff --git a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java index 2cb5532262..72c71baee2 100644 --- a/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java +++ b/flinkx-kudu/flinkx-kudu-writer/src/main/java/com/dtstack/flinkx/kudu/writer/KuduWriter.java @@ -31,6 +31,7 @@ import org.apache.kudu.client.AsyncKuduClient; import java.util.List; +import java.util.Map; import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_ADMIN_OPERATION_TIMEOUT; import static com.dtstack.flinkx.kudu.core.KuduConfigKeys.KEY_AUTHENTICATION; @@ -57,6 +58,8 @@ public class KuduWriter extends BaseDataWriter { private int batchInterval; + protected Map hadoopConfig; + public KuduWriter(DataTransferConfig config) { super(config); @@ -77,6 +80,8 @@ public KuduWriter(DataTransferConfig config) { .withTable(parameterConfig.getStringVal(KEY_TABLE)) .withFlushMode(parameterConfig.getStringVal(KEY_FLUSH_MODE)) .build(); + + hadoopConfig = (Map) parameterConfig.getVal("hadoopConfig"); } @Override @@ -89,6 +94,10 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setBatchInterval(batchInterval); builder.setErrors(errors); builder.setErrorRatio(errorRatio); + builder.setHadoopConfig(hadoopConfig); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); return createOutput(dataSet,builder.finish()); } } diff --git a/flinkx-launcher/pom.xml b/flinkx-launcher/pom.xml index 8ac6150544..9c52b9d958 100644 --- a/flinkx-launcher/pom.xml +++ b/flinkx-launcher/pom.xml @@ -12,17 +12,25 @@ flinkx-launcher + + ch.qos.logback + logback-classic + 1.1.7 + + com.dtstack.flinkx flinkx-core 1.6 + + + org.slf4j + slf4j-log4j12 + + - - ch.qos.logback - logback-classic - 1.1.7 - + com.google.code.gson @@ -104,12 +112,6 @@ avro 1.8.2 - - junit - junit - 4.12 - test - diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index 09f530cd18..d97ee6ce73 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; @@ -76,6 +77,10 @@ public static ClusterClient createYarnClient(Options launcherOptions) { try { FileSystem.initialize(flinkConfig); + //进行kerberos验证 如果需要的话 + KerberosInfo kerberosInfo = new KerberosInfo(launcherOptions.getKrb5conf(), launcherOptions.getKeytab(), launcherOptions.getPrincipal(), flinkConfig); + kerberosInfo.verify(); + YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); YarnClient yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/KerberosInfo.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/KerberosInfo.java new file mode 100644 index 0000000000..e557ea5e88 --- /dev/null +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/KerberosInfo.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.launcher; + +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +/** + * KerberosInfo + * + * @author by dujie@dtstack.com + * @Date 2020/8/21 + */ +public class KerberosInfo { + + private static final Logger LOG = LoggerFactory.getLogger(KerberosInfo.class); + + + private final String krb5confPath; + private final String keytab; + private final String principal; + private final Configuration config; + private final org.apache.hadoop.conf.Configuration hadoopConfiguration; + + public KerberosInfo(String krb5confPath, String keytab, String principal, Configuration config) { + this.krb5confPath = krb5confPath; + this.config = config; + this.hadoopConfiguration = HadoopUtils.getHadoopConfiguration(this.config); + + //keytab, launcherOptions.getKeytab() 比flinkConfiguration里配置的优先级高 + if (StringUtils.isBlank(keytab)) { + this.keytab = this.config.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + } else { + this.keytab = keytab; + } + //principal信息, launcherOptions.getPrincipal() 比flinkConfiguration里配置的优先级高 + if (StringUtils.isBlank(principal)) { + this.principal = this.config.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + } else { + this.principal = principal; + } + } + + public void verify() { + if (!isVerify()) { + return; + } + + check(); + + //如果指定了Krb5conf位置 + if (StringUtils.isNotBlank(this.getKrb5confPath())) { + System.setProperty("java.security.krb5.conf", this.getKrb5confPath()); + } + + String keyTabpath; + try { + keyTabpath = (new File(keytab)).getAbsolutePath(); + } catch (Exception e) { + String message = String.format("can not get the file 【%s】,error info-> %s ", + keytab, + ExceptionUtil.getErrorMessage(e)); + LOG.error("{}", message); + throw new RuntimeException(message, e); + } + + + LOG.info("kerberos info:Krb5confPath ->{}, Principal ->{}, keytab->{}", System.getProperty("java.security.krb5.conf"), principal, keyTabpath); + + //开始kerberos验证 + UserGroupInformation.setConfiguration(hadoopConfiguration); + try { + UserGroupInformation.getCurrentUser().setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS); + } catch (IOException e) { + String message = "UserGroupInformation getCurrentUser has error," + ExceptionUtil.getErrorMessage(e); + LOG.error("{}", message); + throw new RuntimeException(message, e); + } + + + try { + UserGroupInformation.loginUserFromKeytab(principal, keyTabpath); + } catch (IOException e) { + String message = String.format("Unable to set the Hadoop login principal【%s】,keytab 【%s】error info-> %s ", + principal, + keyTabpath, + ExceptionUtil.getErrorMessage(e)); + LOG.error("{}", message); + throw new RuntimeException(message, e); + } + } + + //是否需要kerberos验证 + public boolean isVerify() { + UserGroupInformation.AuthenticationMethod authenticationMethod = SecurityUtil.getAuthenticationMethod(hadoopConfiguration); + return UserGroupInformation.AuthenticationMethod.SIMPLE != authenticationMethod; + } + + protected void check() { + if (StringUtils.isBlank(getKeytab())) { + throw new RuntimeException("keytabPath can not be null"); + } + + if (StringUtils.isBlank(getPrincipal())) { + throw new RuntimeException("principal can not be null"); + } + } + + + public String getKrb5confPath() { + return krb5confPath; + } + + public String getKeytab() { + return keytab; + } + + + public String getPrincipal() { + return principal; + } + + +} diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 65b954a53f..2a464515f5 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -17,8 +17,6 @@ */ package com.dtstack.flinkx.launcher; -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; import com.dtstack.flinkx.config.ContentConfig; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.enums.ClusterMode; @@ -36,7 +34,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.Preconditions; -import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileInputStream; @@ -62,12 +59,11 @@ public class Launcher { public static final String KEY_FLINK_HOME = "FLINK_HOME"; public static final String KEY_HADOOP_HOME = "HADOOP_HOME"; - public static final String PLUGINS_DIR_NAME = "plugins"; + public static final String PLUGINS_DIR_NAME = "syncplugins"; public static final String CORE_JAR_NAME_PREFIX = "flinkx"; public static final String MAIN_CLASS = "com.dtstack.flinkx.Main"; public static void main(String[] args) throws Exception { - setLogLevel(Level.DEBUG.toString()); OptionParser optionParser = new OptionParser(args); Options launcherOptions = optionParser.getOptions(); findDefaultConfigDir(launcherOptions); @@ -162,6 +158,7 @@ public static List analyzeUserClasspath(String content, String pluginRoot) return urlList; } + private static void findDefaultConfigDir(Options launcherOptions) { findDefaultPluginRoot(launcherOptions); @@ -267,11 +264,4 @@ private static String readJob(String job) { throw new RuntimeException(e); } } - - private static void setLogLevel(String level) { - LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); - //设置全局日志级别 - ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel(level)); - } } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java index d9c5c057a8..2ee9028014 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java @@ -38,7 +38,6 @@ * @author tudou */ public class FlinkPerJobUtil { - private static final Logger LOG = LoggerFactory.getLogger(FlinkPerJobUtil.class); /** * Minimum memory requirements, checked by the Client. * the minimum memory should be higher than the min heap cutoff @@ -48,6 +47,7 @@ public class FlinkPerJobUtil { public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; + private static final Logger LOG = LoggerFactory.getLogger(FlinkPerJobUtil.class); /** * the specification of this per-job mode cost diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index f4649d336a..cf8c8caef5 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -17,6 +17,7 @@ */ package com.dtstack.flinkx.launcher.perJob; +import com.dtstack.flinkx.launcher.KerberosInfo; import com.dtstack.flinkx.launcher.YarnConfLoader; import com.dtstack.flinkx.options.Options; import org.apache.commons.lang3.StringUtils; @@ -54,6 +55,9 @@ public class PerJobClusterClientBuilder { private Configuration flinkConfig; + //kerberos验证信息 + private KerberosInfo kerberosInfo; + /** * init yarnClient * @param launcherOptions flinkx args @@ -66,6 +70,9 @@ public void init(Options launcherOptions, Properties conProp) throws Exception { } flinkConfig = launcherOptions.loadFlinkConfiguration(); conProp.forEach((key, val) -> flinkConfig.setString(key.toString(), val.toString())); + this.kerberosInfo = new KerberosInfo(launcherOptions.getKrb5conf(),launcherOptions.getKeytab(),launcherOptions.getPrincipal(),this.flinkConfig); + kerberosInfo.verify(); + SecurityUtils.install(new SecurityConfiguration(flinkConfig)); yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); @@ -91,13 +98,14 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptio } else { throw new IllegalArgumentException("The Flink jar path is null"); } - File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - if(logback.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); - }else{ - File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); - if(log4j.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + + File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + if(log4j.exists()){ + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + } else{ + File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + if(logback.exists()){ + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); } } @@ -124,4 +132,13 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptio descriptor.addShipFiles(shipFiles); return descriptor; } + + + public KerberosInfo getKerberosInfo() { + return kerberosInfo; + } + + public void setKerberosInfo(KerberosInfo kerberosInfo) { + this.kerberosInfo = kerberosInfo; + } } \ No newline at end of file diff --git a/flinkx-launcher/src/main/resources/log4j.properties b/flinkx-launcher/src/main/resources/log4j.properties new file mode 100644 index 0000000000..f252ab2262 --- /dev/null +++ b/flinkx-launcher/src/main/resources/log4j.properties @@ -0,0 +1,13 @@ +############# +# 输出到控制台 +############# + +# log4j.rootLogger日志输出类别和级别:只输出不低于该级别的日志信息 DEBUG < INFO < WARN < ERROR < FATAL +# WARN:日志级别 CONSOLE:输出位置自己定义的一个名字 logfile:输出位置自己定义的一个名字 +log4j.rootLogger=INFO,CONSOLE,logfile +# 配置CONSOLE输出到控制台 +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# 配置CONSOLE设置为自定义布局模式 +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +# 配置CONSOLE日志的输出格式 2019-08-22 22:52:12,000 %r耗费毫秒数 %p日志的优先级 %t线程名 %C所属类名通常为全类名 %L代码中的行号 %x线程相关联的NDC %m日志 %n换行 +log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} - %-4r %-5p [%t] %C:%m%n diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/util/SqlUtil.java b/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/util/SqlUtil.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml index 83cd68b3c4..9e532bf9ad 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-reader/pom.xml @@ -87,6 +87,7 @@ + diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index ac8e65d705..d4e0343395 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -24,7 +24,12 @@ import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.StringUtil; +import com.google.gson.Gson; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; import com.mongodb.BasicDBObject; import com.mongodb.MongoClient; import com.mongodb.client.FindIterable; @@ -40,6 +45,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * Read plugin for reading static data @@ -94,7 +100,8 @@ public Row nextRecordInternal(Row row) throws IOException { row = new Row(doc.size()); String[] names = doc.keySet().toArray(new String[0]); for (int i = 0; i < names.length; i++) { - row.setField(i,doc.get(names[i])); + Object tempData =doc.get(names[i]); + row.setField(i, conventDocument(tempData)); } } else { row = new Row(metaColumns.size()); @@ -103,7 +110,8 @@ public Row nextRecordInternal(Row row) throws IOException { Object value = null; if(metaColumn.getName() != null){ - value = doc.get(metaColumn.getName()); + Object tempData = doc.get(metaColumn.getName()); + value = conventDocument(tempData); if(value == null && metaColumn.getValue() != null){ value = metaColumn.getValue(); } @@ -171,4 +179,17 @@ private void buildFilter(){ filter = BasicDBObject.parse(mongodbConfig.getFilter()); } } + + /** + * 如果是 map 或者 list 数据结构 使用gson转为json格式 + * 主要针对 mongodb的 Document(继承Map) 类型 ,其原有document.tostring 格式不符合正常json格式 + * @param object + * @return + */ + private Object conventDocument(Object object){ + if( object instanceof List || object instanceof Map){ + return GsonUtil.GSON.toJson(object); + } + return object; + } } diff --git a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml index d3fffb6dc6..83b9d3003a 100644 --- a/flinkx-mongodb/flinkx-mongodb-writer/pom.xml +++ b/flinkx-mongodb/flinkx-mongodb-writer/pom.xml @@ -87,6 +87,7 @@ + diff --git a/flinkx-mongodb/flinkx-mongodboplog-reader/pom.xml b/flinkx-mongodb/flinkx-mongodboplog-reader/pom.xml new file mode 100644 index 0000000000..6ff375839c --- /dev/null +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/pom.xml @@ -0,0 +1,88 @@ + + + + flinkx-mongodb + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-mongodboplog-reader + + + + flinkx-mongodb-core + com.dtstack.flinkx + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbEventHandler.java similarity index 98% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java rename to flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbEventHandler.java index 914b07c812..7a0c6c694b 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbEventHandler.java +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbEventHandler.java @@ -17,7 +17,7 @@ */ -package com.dtstack.mongodb.oplog.reader; +package com.dtstack.flinkx.mongodboplog.format; import com.dtstack.flinkx.util.SnowflakeIdWorker; import org.apache.flink.types.Row; diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbOperation.java similarity index 96% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java rename to flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbOperation.java index c5f80add50..adfac99650 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOperation.java +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodbOperation.java @@ -1,4 +1,4 @@ -package com.dtstack.mongodb.oplog.reader; +package com.dtstack.flinkx.mongodboplog.format; import java.util.ArrayList; diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormat.java similarity index 98% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java rename to flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormat.java index a1461eef51..acd66c4dba 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormat.java @@ -17,7 +17,7 @@ */ -package com.dtstack.mongodb.oplog.reader; +package com.dtstack.flinkx.mongodboplog.format; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.mongodb.MongodbClientUtil; @@ -47,7 +47,7 @@ * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogInputFormat extends BaseRichInputFormat { +public class MongodboplogInputFormat extends BaseRichInputFormat { private final static String OPLOG_DB = "local"; private final static String REPLICA_SET_COLLECTION = "oplog.rs"; diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormatBuilder.java similarity index 81% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java rename to flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormatBuilder.java index 790b6e407b..f1975fe397 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogInputFormatBuilder.java +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/format/MongodboplogInputFormatBuilder.java @@ -17,7 +17,7 @@ */ -package com.dtstack.mongodb.oplog.reader; +package com.dtstack.flinkx.mongodboplog.format; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.mongodb.MongodbConfig; @@ -26,12 +26,12 @@ * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogInputFormatBuilder extends BaseRichInputFormatBuilder { +public class MongodboplogInputFormatBuilder extends BaseRichInputFormatBuilder { - private MongodbOplogInputFormat format; + private MongodboplogInputFormat format; - public MongodbOplogInputFormatBuilder() { - super.format = this.format = new MongodbOplogInputFormat(); + public MongodboplogInputFormatBuilder() { + super.format = this.format = new MongodboplogInputFormat(); } public void setMongodbConfig(MongodbConfig mongodbConfig){ diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/reader/MongodboplogReader.java similarity index 78% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java rename to flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/reader/MongodboplogReader.java index e3873636ea..3d67f6b4ca 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/reader/MongodbOplogReader.java +++ b/flinkx-mongodb/flinkx-mongodboplog-reader/src/main/java/com/dtstack/flinkx/mongodboplog/reader/MongodboplogReader.java @@ -17,12 +17,13 @@ */ -package com.dtstack.mongodb.oplog.reader; +package com.dtstack.flinkx.mongodboplog.reader; import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.mongodb.MongodbConfig; import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.mongodboplog.format.MongodboplogInputFormatBuilder; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; @@ -31,30 +32,31 @@ * @author jiangbo * @date 2019/12/5 */ -public class MongodbOplogReader extends BaseDataReader { +public class MongodboplogReader extends BaseDataReader { private MongodbConfig mongodbConfig; - public MongodbOplogReader(DataTransferConfig config, StreamExecutionEnvironment env) { + public MongodboplogReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); try { mongodbConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), MongodbConfig.class); } catch (Exception e) { - throw new RuntimeException("解析mongodb配置出错:", e); + throw new RuntimeException("parse mongodb config error:", e); } } @Override public DataStream readData() { - MongodbOplogInputFormatBuilder builder = new MongodbOplogInputFormatBuilder(); + MongodboplogInputFormatBuilder builder = new MongodboplogInputFormatBuilder(); + builder.setDataTransferConfig(dataTransferConfig); + builder.setMongodbConfig(mongodbConfig); builder.setMonitorUrls(monitorUrls); builder.setBytes(bytes); builder.setRestoreConfig(restoreConfig); - - builder.setMongodbConfig(mongodbConfig); - + builder.setLogConfig(logConfig); + builder.setTestConfig(testConfig); return createInput(builder.finish()); } } diff --git a/flinkx-mongodb/pom.xml b/flinkx-mongodb/pom.xml index 876dc99887..f16a24b779 100644 --- a/flinkx-mongodb/pom.xml +++ b/flinkx-mongodb/pom.xml @@ -14,7 +14,7 @@ flinkx-mongodb-core flinkx-mongodb-reader - flinkx-mongodb-oplog-reader + flinkx-mongodboplog-reader flinkx-mongodb-writer diff --git a/flinkx-mysql/flinkx-mysql-dreader/pom.xml b/flinkx-mysql/flinkx-mysql-dreader/pom.xml index ae15ba2241..0751f5c80a 100644 --- a/flinkx-mysql/flinkx-mysql-dreader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-dreader/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-mysql/flinkx-mysql-reader/pom.xml b/flinkx-mysql/flinkx-mysql-reader/pom.xml index c573d6ff70..5f82074c6f 100644 --- a/flinkx-mysql/flinkx-mysql-reader/pom.xml +++ b/flinkx-mysql/flinkx-mysql-reader/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java index febd28def0..01418c8d0e 100644 --- a/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java +++ b/flinkx-mysql/flinkx-mysql-reader/src/main/java/com/dtstack/flinkx/mysql/format/MysqlInputFormat.java @@ -18,9 +18,6 @@ package com.dtstack.flinkx.mysql.format; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; -import com.dtstack.flinkx.rdb.inputformat.JdbcInputSplit; -import com.dtstack.flinkx.rdb.util.DbUtil; -import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; diff --git a/flinkx-mysql/flinkx-mysql-writer/pom.xml b/flinkx-mysql/flinkx-mysql-writer/pom.xml index 2d8e5c8c93..de701fc94d 100644 --- a/flinkx-mysql/flinkx-mysql-writer/pom.xml +++ b/flinkx-mysql/flinkx-mysql-writer/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-odps/flinkx-odps-reader/pom.xml b/flinkx-odps/flinkx-odps-reader/pom.xml index 483dbf8843..33d7cbd1c5 100644 --- a/flinkx-odps/flinkx-odps-reader/pom.xml +++ b/flinkx-odps/flinkx-odps-reader/pom.xml @@ -83,6 +83,7 @@ + diff --git a/flinkx-odps/flinkx-odps-writer/pom.xml b/flinkx-odps/flinkx-odps-writer/pom.xml index 16b74aca80..b1575373aa 100644 --- a/flinkx-odps/flinkx-odps-writer/pom.xml +++ b/flinkx-odps/flinkx-odps-writer/pom.xml @@ -88,6 +88,7 @@ + diff --git a/flinkx-oracle/flinkx-oracle-reader/pom.xml b/flinkx-oracle/flinkx-oracle-reader/pom.xml index a16de5f343..ab8164a41c 100644 --- a/flinkx-oracle/flinkx-oracle-reader/pom.xml +++ b/flinkx-oracle/flinkx-oracle-reader/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-oracle/flinkx-oracle-reader/src/main/java/com/dtstack/flinkx/oracle/reader/OracleReader.java b/flinkx-oracle/flinkx-oracle-reader/src/main/java/com/dtstack/flinkx/oracle/reader/OracleReader.java index 2f79eeff76..4e7a1262ea 100644 --- a/flinkx-oracle/flinkx-oracle-reader/src/main/java/com/dtstack/flinkx/oracle/reader/OracleReader.java +++ b/flinkx-oracle/flinkx-oracle-reader/src/main/java/com/dtstack/flinkx/oracle/reader/OracleReader.java @@ -19,10 +19,12 @@ package com.dtstack.flinkx.oracle.reader; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.oracle.OracleDatabaseMeta; import com.dtstack.flinkx.oracle.format.OracleInputFormat; import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** @@ -35,6 +37,10 @@ public class OracleReader extends JdbcDataReader { public OracleReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); + String schema = config.getJob().getContent().get(0).getReader().getParameter().getConnection().get(0).getSchema(); + if(StringUtils.isNotBlank(schema)){ + table = schema + ConstantValue.POINT_SYMBOL + table; + } setDatabaseInterface(new OracleDatabaseMeta()); } diff --git a/flinkx-oracle/flinkx-oracle-writer/pom.xml b/flinkx-oracle/flinkx-oracle-writer/pom.xml index b4d3389992..45ef58b1fd 100644 --- a/flinkx-oracle/flinkx-oracle-writer/pom.xml +++ b/flinkx-oracle/flinkx-oracle-writer/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/writer/OracleWriter.java b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/writer/OracleWriter.java index 6dc096c611..f478508659 100644 --- a/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/writer/OracleWriter.java +++ b/flinkx-oracle/flinkx-oracle-writer/src/main/java/com/dtstack/flinkx/oracle/writer/OracleWriter.java @@ -19,10 +19,12 @@ package com.dtstack.flinkx.oracle.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.oracle.OracleDatabaseMeta; import com.dtstack.flinkx.oracle.format.OracleOutputFormat; import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import org.apache.commons.lang3.StringUtils; /** * Oracle writer plugin @@ -32,13 +34,20 @@ */ public class OracleWriter extends JdbcDataWriter { + protected String schema; public OracleWriter(DataTransferConfig config) { super(config); + schema = config.getJob().getContent().get(0).getWriter().getParameter().getConnection().get(0).getSchema(); + if(StringUtils.isNotBlank(schema)){ + table = schema + ConstantValue.POINT_SYMBOL + table; + } setDatabaseInterface(new OracleDatabaseMeta()); } @Override protected JdbcOutputFormatBuilder getBuilder() { - return new JdbcOutputFormatBuilder(new OracleOutputFormat()); + JdbcOutputFormatBuilder jdbcOutputFormatBuilder = new JdbcOutputFormatBuilder(new OracleOutputFormat()); + jdbcOutputFormatBuilder.setSchema(schema); + return jdbcOutputFormatBuilder; } } diff --git a/flinkx-pgwal/flinkx-pgwal-core/pom.xml b/flinkx-pgwal/flinkx-pgwal-core/pom.xml new file mode 100644 index 0000000000..ebbdb1958a --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/pom.xml @@ -0,0 +1,22 @@ + + + + flinkx-pgwal + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-pgwal-core + + + + org.postgresql + postgresql + 42.2.8 + + + + \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java new file mode 100644 index 0000000000..ffa87be7ef --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgDecoder.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import com.dtstack.flinkx.reader.MetaColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.sql.SQLException; +import java.time.Instant; +import java.time.LocalDate; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * reference to https://github.com/debezium/debezium & http://www.postgres.cn/docs/10/protocol-logicalrep-message-formats.html + * + * @author tudou + */ +public class PgDecoder { + private static final Logger LOG = LoggerFactory.getLogger(PgDecoder.class); + + private static Instant PG_EPOCH = LocalDate.of(2000, 1, 1).atStartOfDay().toInstant(ZoneOffset.UTC); + + private Map tableMap = new HashMap<>(64); + private Map pgTypeMap; + private volatile long currentLsn; + private volatile long ts; + + public PgDecoder(Map pgTypeMap) { + this.pgTypeMap = pgTypeMap; + } + + private static String readColumnValueAsString(ByteBuffer buffer) { + //Int32 列值的长度 + int length = buffer.getInt(); + byte[] value = new byte[length]; + //Byte(n) 该列的值,以文本格式显示。n是上面的长度 + buffer.get(value, 0, length); + return new String(value); + } + + private static String readString(ByteBuffer buffer) { + StringBuilder sb = new StringBuilder(); + byte b = 0; + while ((b = buffer.get()) != 0) { + sb.append((char) b); + } + return sb.toString(); + } + + public static String unquoteIdentifierPart(String identifierPart) { + if (identifierPart == null || identifierPart.length() < 2) { + return identifierPart; + } + + Character quotingChar = deriveQuotingChar(identifierPart); + if (quotingChar != null) { + identifierPart = identifierPart.substring(1, identifierPart.length() - 1); + identifierPart = identifierPart.replace(quotingChar.toString() + quotingChar.toString(), quotingChar.toString()); + } + + return identifierPart; + } + + private static Character deriveQuotingChar(String identifierPart) { + char first = identifierPart.charAt(0); + char last = identifierPart.charAt(identifierPart.length() - 1); + + if (first == last && (first == '"' || first == '\'' || first == '`')) { + return first; + } + + return null; + } + + public Table decode(ByteBuffer buffer) throws SQLException { + Table table = new Table(); + PgMessageTypeEnum type = PgMessageTypeEnum.forType((char) buffer.get()); + switch (type) { + case BEGIN: + //Byte1('B') 将消息标识为开始消息 + handleBeginMessage(buffer); + break; + case COMMIT: + //Byte1('C') 将消息标识为提交消息 + handleCommitMessage(buffer); + break; + case RELATION: + //Byte1('R') 将消息标识为关系消息 + handleRelationMessage(buffer); + break; + case INSERT: + //Byte1('I') 将消息标识为插入消息 + table = decodeInsert(buffer); + break; + case UPDATE: + //Byte1('U') 将消息标识为更新消息 + table = decodeUpdate(buffer); + break; + case DELETE: + //Byte1('D') 将消息标识为删除消息 + table = decodeDelete(buffer); + break; + default: + break; + } + table.setType(type); + return table; + } + + private void handleBeginMessage(ByteBuffer buffer) { + //Int64 事务的结束LSN + long lsn = buffer.getLong(); + //Int64 提交事务的时间戳。自PostgreSQL纪元(2000-01-01)以来的数值是微秒数 + Instant plus = PG_EPOCH.plus(buffer.getLong(), ChronoUnit.MICROS); + //Int32 事务的Xid + int anInt = buffer.getInt(); + currentLsn = lsn; + ts = plus.toEpochMilli(); + LOG.trace("handleBeginMessage result = { lsn = {}, plus = {}, anInt = {}}", lsn, plus, anInt); + } + + private void handleCommitMessage(ByteBuffer buffer) { + if(LOG.isTraceEnabled()){ + //Int8 标志;目前未使用(必须为0) + int flags = buffer.get(); + //Int64 提交的LSN + long lsn = buffer.getLong(); + //Int64 事务的结束LSN + long endLsn = buffer.getLong(); + //Int64 提交事务的时间戳。自PostgreSQL纪元(2000-01-01)以来的数值是微秒数 + Instant commitTimestamp = PG_EPOCH.plus(buffer.getLong(), ChronoUnit.MICROS); + LOG.trace("handleCommitMessage result = { flags = {}, lsn = {}, endLsn = {}, commitTimestamp = {}}", flags, lsn, endLsn, commitTimestamp); + } + } + + private void handleRelationMessage(ByteBuffer buffer) throws SQLException { + //Int32 关系的ID + int relationId = buffer.getInt(); + //String 命名空间(pg_catalog的空字符串) + String schemaName = readString(buffer); + //String 关系名称 + String tableName = readString(buffer); + //Int8 该关系的副本标识设置(与pg_class 中的relreplident相同) + int replicaIdentityId = buffer.get(); + //Int16 列数 + short columnCount = buffer.getShort(); + LOG.debug("handleRelationMessage result = { schemaName = {}, tableName = {}}", schemaName, tableName); + if(!tableMap.containsKey(relationId)){ + List columnList = new ArrayList<>(columnCount); + for (int i = 0; i < columnCount; i++) { + //Int8 列的标志。当前可以是0表示没有标记或1表示将列标记为关键字的一部分 + byte flags = buffer.get(); + //String 列的名称 + String name = unquoteIdentifierPart(readString(buffer)); + //Int32 列的数据类型的ID + String type = pgTypeMap.get(buffer.getInt()); + MetaColumn metaColumn = new MetaColumn(); + metaColumn.setIndex(i); + metaColumn.setName(name); + metaColumn.setType(type); + columnList.add(metaColumn); + //Int32 列的类型修饰符(atttypmod) + int attypmod = buffer.getInt(); + } + Table table = new Table(schemaName, tableName, columnList); + tableMap.put(relationId, table); + } + } + + private Table decodeInsert(ByteBuffer buffer) { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + //Byte1('N') 将以下TupleData消息标识为新元组 + char tupleType = (char) buffer.get(); + //TupleData TupleData消息部分表示新元组的内容 + Object[] newData = resolveColumnsFromStreamTupleData(buffer); + Table table = tableMap.get(relationId); + table.setOldData(new Object[newData.length]); + table.setNewData(newData); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + private Table decodeUpdate(ByteBuffer buffer) throws SQLException { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + Table table = tableMap.get(relationId); + //Byte1('K') 将以下TupleData子消息标识为键。该字段是可选的, 并且只有在更新改变了REPLICA IDENTITY索引一部分的任何一列中的数据时才存在 + //Byte1('O') 将以下TupleData子消息标识为旧元组。此字段是可选的, 并且仅当发生更新的表的REPLICA IDENTITY设置为FULL时才存在 + //更新消息可以包含'K'消息部分或者'O'消息部分或者都不包含它们,但不同时包括它们两者 + char tupleType = (char) buffer.get(); + if ('O' == tupleType || 'K' == tupleType) { + //TupleData TupleData消息部分表示旧元组或主键的内容。 只有在前面的'O'或'K'部分存在时才存在 + Object[] oldData = resolveColumnsFromStreamTupleData(buffer); + table.setOldData(oldData); + // Read the 'N' tuple type + // This is necessary so the stream position is accurate for resolving the column tuple data + //Byte1('N') 将以下TupleData消息标识为新元组 + tupleType = (char) buffer.get(); + } + //TupleData TupleData消息部分表示新元组的内容 + Object[] newData = resolveColumnsFromStreamTupleData(buffer); + table.setNewData(newData); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + private Table decodeDelete(ByteBuffer buffer) throws SQLException { + //Int32 与关系消息中的ID对应的关系的ID + int relationId = buffer.getInt(); + Table table = tableMap.get(relationId); + //Byte1('K') 将以下TupleData子消息标识为键。 如果发生删除的表使用索引作为REPLICA IDENTITY,则此字段存在 + //Byte1('O') 将以下TupleData消息标识为旧元组。 如果发生删除的表的REPLICA IDENTITY设置为FULL,则此字段存在 + //删除消息可能包含'K'消息部分或'O'消息部分,但不会同时包含这两个部分 + char tupleType = (char) buffer.get(); + //TupleData TupleData消息部分,表示旧元组或主键的内容,具体取决于前一个字段 + Object[] oldData = resolveColumnsFromStreamTupleData(buffer); + table.setOldData(oldData); + table.setNewData(new Object[oldData.length]); + table.setCurrentLsn(currentLsn); + table.setTs(ts); + return table; + } + + private Object[] resolveColumnsFromStreamTupleData(ByteBuffer buffer) { + //Int16 列数 + short numberOfColumns = buffer.getShort(); + Object[] data = new Object[numberOfColumns]; + for (int i = 0; i < numberOfColumns; i++) { + + //Byte1('n') 将数据标识为NULL值 + //Byte1('u') 识别未更改的TOASTed值(实际值未发送) + //Byte1('t') 将数据标识为文本格式的值 + char type = (char) buffer.get(); + if (type == 't') { + data[i] = readColumnValueAsString(buffer); + } else if (type == 'u') { + data[i] = null; + } else if (type == 'n') { + data[i] = null; + } + } + return data; + } + + +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java new file mode 100644 index 0000000000..c12a614149 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgMessageTypeEnum.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * reference to https://github.com/debezium/debezium & http://www.postgres.cn/docs/10/protocol-logicalrep-message-formats.html + * + * @author tudou + */ +public enum PgMessageTypeEnum { + RELATION, + BEGIN, + COMMIT, + INSERT, + UPDATE, + DELETE, + TYPE, + ORIGIN; + + public static PgMessageTypeEnum forType(char type) { + switch (type) { + case 'R': return RELATION; + case 'B': return BEGIN; + case 'C': return COMMIT; + case 'I': return INSERT; + case 'U': return UPDATE; + case 'D': return DELETE; + case 'Y': return TYPE; + case 'O': return ORIGIN; + default: throw new IllegalArgumentException("Unsupported message type: " + type); + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java new file mode 100644 index 0000000000..f002f50d6d --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgRelicationSlot.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgRelicationSlot { + private String slotName; + private String plugin; + private String slotType; + private Integer datoid; + private String database; + private String temporary; + private String active; + private Integer activePid; + private String xmin; + private String catalogXmin; + private String restartLsn; + private String confirmedFlushLsn; + + public boolean isActive(){ + return "t".equalsIgnoreCase(active); + } + + public boolean isNotActive(){ + return !isActive(); + } + + public String getSlotName() { + return slotName; + } + + public void setSlotName(String slotName) { + this.slotName = slotName; + } + + public String getPlugin() { + return plugin; + } + + public void setPlugin(String plugin) { + this.plugin = plugin; + } + + public String getSlotType() { + return slotType; + } + + public void setSlotType(String slotType) { + this.slotType = slotType; + } + + public Integer getDatoid() { + return datoid; + } + + public void setDatoid(Integer datoid) { + this.datoid = datoid; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTemporary() { + return temporary; + } + + public void setTemporary(String temporary) { + this.temporary = temporary; + } + + public String getActive() { + return active; + } + + public void setActive(String active) { + this.active = active; + } + + public Integer getActivePid() { + return activePid; + } + + public void setActivePid(Integer activePid) { + this.activePid = activePid; + } + + public String getXmin() { + return xmin; + } + + public void setXmin(String xmin) { + this.xmin = xmin; + } + + public String getCatalogXmin() { + return catalogXmin; + } + + public void setCatalogXmin(String catalogXmin) { + this.catalogXmin = catalogXmin; + } + + public String getRestartLsn() { + return restartLsn; + } + + public void setRestartLsn(String restartLsn) { + this.restartLsn = restartLsn; + } + + public String getConfirmedFlushLsn() { + return confirmedFlushLsn; + } + + public void setConfirmedFlushLsn(String confirmedFlushLsn) { + this.confirmedFlushLsn = confirmedFlushLsn; + } + + @Override + public String toString() { + return "PgRelicationSlots{" + + "slotName='" + slotName + '\'' + + ", plugin='" + plugin + '\'' + + ", slotType='" + slotType + '\'' + + ", datoid=" + datoid + + ", database='" + database + '\'' + + ", temporary='" + temporary + '\'' + + ", active='" + active + '\'' + + ", activePid='" + activePid + '\'' + + ", xmin='" + xmin + '\'' + + ", catalogXmin='" + catalogXmin + '\'' + + ", restartLsn='" + restartLsn + '\'' + + ", conFirmedFlushLsn='" + confirmedFlushLsn + '\'' + + '}'; + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java new file mode 100644 index 0000000000..c0d3f0a03f --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalConfigKeys.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalConfigKeys { + public static final String KEY_USER_NAME = "username"; + + public static final String KEY_PASSWORD = "password"; + + public static final String KEY_URL = "url"; + + public final static String KEY_DATABASE_NAME = "databaseName"; + + public final static String KEY_CATALOG = "cat"; + + public final static String KEY_PAVING_DATA = "pavingData"; + + public final static String KEY_TABLE_LIST = "tableList"; + + public final static String KEY_STATUS_INTERVAL = "statusInterval"; + + public final static String KEY_LSN = "lsn"; + + public final static String KEY_SLOT_NAME = "slotName"; + + public final static String KEY_ALLOW_CREATE_SLOT = "allowCreateSlot"; + + public final static String KEY_TEMPORARY = "temporary"; +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java new file mode 100644 index 0000000000..4f0e9dd212 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/PgWalUtil.java @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.TelnetUtil; +import org.postgresql.PGProperty; +import org.postgresql.core.ServerVersion; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.replication.ReplicationSlotInfo; +import org.postgresql.replication.fluent.logical.ChainedLogicalCreateSlotBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalUtil { + + public static final String DRIVER = "org.postgresql.Driver"; + public static final String SLOT_PRE = "flinkx_"; + public static final String PUBLICATION_NAME = "dtstack_flinkx"; + public static final String QUERY_LEVEL = "SHOW wal_level;"; + public static final String QUERY_MAX_SLOT = "SHOW max_replication_slots;"; + public static final String QUERY_SLOT = "SELECT * FROM pg_replication_slots;"; + public static final String QUERY_TABLE_REPLICA_IDENTITY = "SELECT relreplident FROM pg_catalog.pg_class c LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid WHERE n.nspname='%s' and c.relname='%s';"; + public static final String UPDATE_REPLICA_IDENTITY = "ALTER TABLE %s REPLICA IDENTITY FULL;"; + public static final String QUERY_PUBLICATION = "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s';"; + public static final String CREATE_PUBLICATION = "CREATE PUBLICATION %s FOR ALL TABLES;"; + public static final String QUERY_TYPES = "SELECT t.oid AS oid, t.typname AS name FROM pg_catalog.pg_type t JOIN pg_catalog.pg_namespace n ON (t.typnamespace = n.oid) WHERE n.nspname != 'pg_toast' AND t.typcategory <> 'A';"; + private static final Logger LOG = LoggerFactory.getLogger(PgWalUtil.class); + + public static PgRelicationSlot checkPostgres(PgConnection conn, boolean allowCreateSlot, String slotName, List tableList) throws Exception{ + ResultSet resultSet; + PgRelicationSlot availableSlot = null; + + //1. check postgres version + // this Judge maybe not need? + if (!conn.haveMinimumServerVersion(ServerVersion.v10)){ + String version = conn.getDBVersionNumber(); + LOG.error("postgres version must > 10, current = [{}]", version); + throw new UnsupportedOperationException("postgres version must >= 10, current = " + version); + } + + //2. check postgres wal_level + resultSet = conn.execSQLQuery(QUERY_LEVEL); + resultSet.next(); + String wal_level = resultSet.getString(1); + if(!"logical".equalsIgnoreCase(wal_level)){ + LOG.error("postgres wal_level must be logical, current = [{}]", wal_level); + throw new UnsupportedOperationException("postgres wal_level must be logical, current = " + wal_level); + } + + //3.check postgres slot + resultSet = conn.execSQLQuery(QUERY_MAX_SLOT); + resultSet.next(); + int maxSlot = resultSet.getInt(1); + int slotCount = 0; + resultSet = conn.execSQLQuery(QUERY_SLOT); + while(resultSet.next()){ + PgRelicationSlot slot = new PgRelicationSlot(); + String name = resultSet.getString("slot_name"); + slot.setSlotName(name); + slot.setActive(resultSet.getString("active")); + + if(name.equalsIgnoreCase(slotName) && slot.isNotActive()){ + slot.setPlugin(resultSet.getString("plugin")); + slot.setSlotType(resultSet.getString("slot_type")); + slot.setDatoid(resultSet.getInt("datoid")); + slot.setDatabase(resultSet.getString("database")); + slot.setTemporary(resultSet.getString("temporary")); + slot.setActivePid(resultSet.getInt("active_pid")); + slot.setXmin(resultSet.getString("xmin")); + slot.setCatalogXmin(resultSet.getString("catalog_xmin")); + slot.setRestartLsn(resultSet.getString("restart_lsn")); + slot.setConfirmedFlushLsn(resultSet.getString("confirmed_flush_lsn")); + availableSlot = slot; + break; + } + slotCount++; + } + + if(availableSlot == null){ + if(!allowCreateSlot){ + String msg = String.format("there is no available slot named [%s], please check whether slotName[%s] is correct, or set allowCreateSlot = true", slotName, slotName); + LOG.error(msg); + throw new UnsupportedOperationException(msg); + }else if(slotCount >= maxSlot){ + LOG.error("the number of slot reaches max_replication_slots[{}], please turn up max_replication_slots or remove unused slot", maxSlot); + throw new UnsupportedOperationException("the number of slot reaches max_replication_slots[" + maxSlot + "], please turn up max_replication_slots or remove unused slot"); + } + } + + //4.check table replica identity + for (String table : tableList) { + //schema.tableName + String[] tables = table.split("\\."); + resultSet = conn.execSQLQuery(String.format(QUERY_TABLE_REPLICA_IDENTITY, tables[0], tables[1])); + resultSet.next(); + String identity = parseReplicaIdentity(resultSet.getString(1)); + if(!"full".equals(identity)){ + LOG.warn("update {} replica identity, set {} to full", table, identity); + conn.createStatement().execute(String.format(UPDATE_REPLICA_IDENTITY, table)); + } + } + + //5.check publication + resultSet = conn.execSQLQuery(String.format(QUERY_PUBLICATION, PUBLICATION_NAME)); + resultSet.next(); + long count = resultSet.getLong(1); + if(count == 0L){ + LOG.warn("no publication named [{}] existed, flinkx will create one", PUBLICATION_NAME); + conn.createStatement().execute(String.format(CREATE_PUBLICATION, PUBLICATION_NAME)); + } + + closeDBResources(resultSet, null, null, false); + return availableSlot; + } + + public static PgRelicationSlot createSlot(PgConnection conn, String slotName, boolean temporary) throws SQLException{ + ChainedLogicalCreateSlotBuilder builder = conn.getReplicationAPI() + .createReplicationSlot() + .logical() + .withSlotName(slotName) + .withOutputPlugin("pgoutput"); + if(temporary){ + builder.withTemporaryOption(); + } + ReplicationSlotInfo replicationSlotInfo = builder.make(); + PgRelicationSlot slot = new PgRelicationSlot(); + slot.setSlotName(slotName); + slot.setConfirmedFlushLsn(replicationSlotInfo.getConsistentPoint().asString()); + slot.setPlugin(replicationSlotInfo.getOutputPlugin()); + return slot; + } + + public static Map queryTypes(PgConnection conn) throws SQLException{ + Map map = new HashMap<>(512); + ResultSet resultSet = conn.execSQLQuery(QUERY_TYPES); + while (resultSet.next()){ + int oid = (int) resultSet.getLong("oid"); + String typeName = resultSet.getString("name"); + map.put(oid, typeName); + } + closeDBResources(resultSet, null, null, false); + return map; + } + + public static String parseReplicaIdentity(String s) { + switch (s) { + case "n": + return "nothing"; + case "d": + return "default"; + case "i" : + return "index"; + case "f" : + return "full"; + default: + return "unknown"; + } + } + + /** + * 获取jdbc连接(超时10S) + * @param url url + * @param username 账号 + * @param password 密码 + * @return + * @throws SQLException + */ + public static PgConnection getConnection(String url, String username, String password) throws SQLException { + Connection dbConn; + ClassUtil.forName(PgWalUtil.DRIVER, PgWalUtil.class.getClassLoader()); + Properties props = new Properties(); + PGProperty.USER.set(props, username); + PGProperty.PASSWORD.set(props, password); + PGProperty.REPLICATION.set(props, "database"); + PGProperty.PREFER_QUERY_MODE.set(props, "simple"); + //postgres version must > 10 + PGProperty.ASSUME_MIN_SERVER_VERSION.set(props, "10"); + synchronized (ClassUtil.LOCK_STR) { + DriverManager.setLoginTimeout(10); + // telnet + TelnetUtil.telnet(url); + dbConn = DriverManager.getConnection(url, props); + } + + return dbConn.unwrap(PgConnection.class); + } + + /** + * 关闭连接资源 + * + * @param rs ResultSet + * @param stmt Statement + * @param conn Connection + * @param commit + */ + public static void closeDBResources(ResultSet rs, Statement stmt, Connection conn, boolean commit) { + if (null != rs) { + try { + rs.close(); + } catch (SQLException e) { + LOG.warn("Close resultSet error: {}", ExceptionUtil.getErrorMessage(e)); + } + } + + if (null != stmt) { + try { + stmt.close(); + } catch (SQLException e) { + LOG.warn("Close statement error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + + if (null != conn) { + try { + if (commit && !conn.isClosed()) { + conn.commit(); + } + conn.close(); + } catch (SQLException e) { + LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + } + +} diff --git a/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java new file mode 100644 index 0000000000..442ae67718 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-core/src/main/java/com/dtstack/flinkx/pgwal/Table.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.pgwal; + +import com.dtstack.flinkx.reader.MetaColumn; +import org.postgresql.replication.LogSequenceNumber; + +import java.util.List; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * @author tudou + */ +public class Table { + private String id; + private String schema; + private String table; + private List columnList; + private Object[] oldData; + private Object[] newData; + private PgMessageTypeEnum type; + + private long currentLsn; + private long ts; + + public Table(String schema, String table, List columnList) { + this.schema = schema; + this.table = table; + this.columnList = columnList; + this.id = schema + "." + table; + } + + public Table() { + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public List getColumnList() { + return columnList; + } + + public void setColumnList(List columnList) { + this.columnList = columnList; + } + + public Object[] getOldData() { + return oldData; + } + + public void setOldData(Object[] oldData) { + this.oldData = oldData; + } + + public Object[] getNewData() { + return newData; + } + + public void setNewData(Object[] newData) { + this.newData = newData; + } + + public PgMessageTypeEnum getType() { + return type; + } + + public void setType(PgMessageTypeEnum type) { + this.type = type; + } + + public long getCurrentLsn() { + return currentLsn; + } + + public void setCurrentLsn(long currentLsn) { + this.currentLsn = currentLsn; + } + + public long getTs() { + return ts; + } + + public void setTs(long ts) { + this.ts = ts; + } +} diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml similarity index 61% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml rename to flinkx-pgwal/flinkx-pgwal-reader/pom.xml index 259af09b8d..363722cc11 100644 --- a/flinkx-mongodb/flinkx-mongodb-oplog-reader/pom.xml +++ b/flinkx-pgwal/flinkx-pgwal-reader/pom.xml @@ -1,26 +1,43 @@ - - flinkx-mongodb + flinkx-pgwal com.dtstack.flinkx 1.6 4.0.0 - flinkx-mongodb-oplog-reader + flinkx-pgwal-reader - flinkx-mongodb-core com.dtstack.flinkx + flinkx-pgwal-core 1.6 - + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + + + maven-antrun-plugin 1.2 @@ -34,13 +51,13 @@ - + - + @@ -48,4 +65,5 @@ + \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java new file mode 100644 index 0000000000..0c66ead84b --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.format; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.pgwal.PgRelicationSlot; +import com.dtstack.flinkx.pgwal.PgWalUtil; +import com.dtstack.flinkx.pgwal.listener.PgWalListener; +import com.dtstack.flinkx.restore.FormatState; +import com.dtstack.flinkx.util.ExceptionUtil; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.postgresql.jdbc.PgConnection; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.SynchronousQueue; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalInputFormat extends BaseRichInputFormat { + protected String username; + protected String password; + protected String url; + protected String databaseName; + protected boolean pavingData = false; + protected List tableList; + protected String cat; + protected Integer statusInterval; + protected Long lsn; + protected String slotName; + protected boolean allowCreateSlot; + protected boolean temporary; + + private PgConnection conn; + private volatile long startLsn; + + private transient BlockingQueue> queue; + private transient ExecutorService executor; + private volatile boolean running = false; + + @Override + public void openInputFormat() throws IOException{ + super.openInputFormat(); + executor = Executors.newFixedThreadPool(1); + queue = new SynchronousQueue<>(true); + } + + @Override + protected void openInternal(InputSplit inputSplit) throws IOException { + if (inputSplit.getSplitNumber() != 0) { + LOG.info("PgWalInputFormat openInternal split number:{} abort...", inputSplit.getSplitNumber()); + return; + } + LOG.info("PgWalInputFormat openInternal split number:{} start...", inputSplit.getSplitNumber()); + try { + conn = PgWalUtil.getConnection(url, username, password); + if(StringUtils.isBlank(slotName)){ + slotName = PgWalUtil.SLOT_PRE + jobId; + } + PgRelicationSlot availableSlot = PgWalUtil.checkPostgres(conn, allowCreateSlot, slotName, tableList); + if(availableSlot == null){ + PgWalUtil.createSlot(conn, slotName, temporary); + } + if(lsn != 0){ + startLsn = lsn; + }else if(formatState != null && formatState.getState() != null){ + startLsn = (long)formatState.getState(); + } + + executor.submit(new PgWalListener(this)); + running = true; + }catch (Exception e){ + LOG.error("PgWalInputFormat open() failed, e = {}", ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException("PgWalInputFormat open() failed, e = " + ExceptionUtil.getErrorMessage(e)); + } + LOG.info("PgWalInputFormat[{}]open: end", jobName); + + } + + @Override + protected Row nextRecordInternal(Row row) throws IOException { + try { + Map map = queue.take(); + if(map.size() == 1){ + throw new IOException((String) map.get("e")); + }else{ + startLsn = (long) map.get("lsn"); + row = Row.of(map); + } + } catch (InterruptedException e) { + LOG.error("takeEvent interrupted error:{}", ExceptionUtil.getErrorMessage(e)); + } + return row; + + } + + @Override + public FormatState getFormatState() { + if (!restoreConfig.isRestore()) { + LOG.info("return null for formatState"); + return null; + } + + super.getFormatState(); + if (formatState != null) { + formatState.setState(startLsn); + } + return formatState; + } + + @Override + protected void closeInternal() throws IOException { + if (running) { + executor.shutdownNow(); + running = false; + LOG.warn("shutdown SqlServerCdcListener......"); + } + + } + + @Override + public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { + InputSplit[] splits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new GenericInputSplit(i, minNumSplits); + } + return splits; + } + + @Override + public boolean reachedEnd() throws IOException { + return false; + } + + public void processEvent(Map event) { + try { + queue.put(event); + } catch (InterruptedException e) { + LOG.error("takeEvent interrupted event:{} error:{}", event, ExceptionUtil.getErrorMessage(e)); + } + } + + + public boolean isPavingData() { + return pavingData; + } + + public List getTableList() { + return tableList; + } + + public String getCat() { + return cat; + } + + public Integer getStatusInterval() { + return statusInterval; + } + + public String getSlotName() { + return slotName; + } + + public PgConnection getConn() { + return conn; + } + + public long getStartLsn() { + return startLsn; + } + + public boolean isRunning() { + return running; + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java new file mode 100644 index 0000000000..c32f225e55 --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with format work for additional information + * regarding copyright ownership. The ASF licenses format file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use format file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.format; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalInputFormatBuilder extends BaseRichInputFormatBuilder { + + protected PgWalInputFormat format; + + public PgWalInputFormatBuilder() { + super.format = this.format = new PgWalInputFormat(); + } + + public void setUsername(String username) { + format.username = username; + } + + public void setPassword(String password) { + format.password = password; + } + + public void setUrl(String url) { + format.url = url; + } + + public void setDatabaseName(String databaseName) { + format.databaseName = databaseName; + } + + public void setPavingData(boolean pavingData) { + format.pavingData = pavingData; + } + + public void setTableList(List tableList) { + format.tableList = tableList; + } + + public void setCat(String cat) { + format.cat = cat; + } + + public void setStatusInterval(Integer statusInterval) { + format.statusInterval = statusInterval; + } + + public void setLsn(Long lsn) { + format.lsn = lsn; + } + + public void setAllowCreateSlot(Boolean allowCreateSlot) { + format.allowCreateSlot = allowCreateSlot; + } + + public void setSlotName(String slotName) { + format.slotName = slotName; + } + + public void setTemporary(Boolean temporary) { + format.temporary = temporary; + } + + @Override + protected void checkFormat() { + if (StringUtils.isBlank(format.username)) { + throw new IllegalArgumentException("No username supplied"); + } + if (StringUtils.isBlank(format.password)) { + throw new IllegalArgumentException("No password supplied"); + } + if (StringUtils.isBlank(format.url)) { + throw new IllegalArgumentException("No url supplied"); + } + if (StringUtils.isBlank(format.databaseName)) { + throw new IllegalArgumentException("No databaseName supplied"); + } + if (CollectionUtils.isEmpty(format.tableList)) { + throw new IllegalArgumentException("No tableList supplied"); + } + if (StringUtils.isBlank(format.cat)) { + throw new IllegalArgumentException("No cat supplied"); + } + if(!format.allowCreateSlot && StringUtils.isBlank(format.slotName)){ + throw new IllegalArgumentException("slotName can not be null if allowCreateSlot is false"); + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java new file mode 100644 index 0000000000..8e5efb743f --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/listener/PgWalListener.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.listener; + +import com.dtstack.flinkx.pgwal.PgDecoder; +import com.dtstack.flinkx.pgwal.PgWalUtil; +import com.dtstack.flinkx.pgwal.Table; +import com.dtstack.flinkx.pgwal.format.PgWalInputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.replication.PGReplicationStream; +import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * Date: 2019/12/14 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgWalListener implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(PgWalListener.class); + private static Gson gson = new Gson(); + + private PgWalInputFormat format; + private PgConnection conn; + private Set tableSet; + private Set cat; + private boolean pavingData; + + private PGReplicationStream stream; + private PgDecoder decoder; + + public PgWalListener(PgWalInputFormat format) { + this.format = format; + this.conn = format.getConn(); + this.tableSet = new HashSet<>(format.getTableList()); + this.cat = new HashSet<>(); + for (String type : format.getCat().split(",")) { + cat.add(type.toLowerCase()); + } + this.pavingData = format.isPavingData(); + } + + public void init() throws Exception{ + decoder = new PgDecoder(PgWalUtil.queryTypes(conn)); + ChainedLogicalStreamBuilder builder = conn.getReplicationAPI() + .replicationStream() + .logical() + .withSlotName(format.getSlotName()) + //协议版本。当前仅支持版本1 + .withSlotOption("proto_version", "1") + //逗号分隔的要订阅的发布名称列表(接收更改)。 单个发布名称被视为标准对象名称,并可根据需要引用 + .withSlotOption("publication_names", PgWalUtil.PUBLICATION_NAME) + .withStatusInterval(format.getStatusInterval(), TimeUnit.MILLISECONDS); + long lsn = format.getStartLsn(); + if(lsn != 0){ + builder.withStartPosition(LogSequenceNumber.valueOf(lsn)); + } + stream = builder.start(); + TimeUnit.SECONDS.sleep(1); + stream.forceUpdateStatus(); + LOG.info("init PGReplicationStream successfully..."); + } + + @Override + public void run() { + LOG.info("PgWalListener start running....."); + try { + init(); + while (format.isRunning()) { + ByteBuffer buffer = stream.readPending(); + if (buffer == null) { + continue; + } + Table table = decoder.decode(buffer); + if(StringUtils.isBlank(table.getId())){ + continue; + } + String type = table.getType().name().toLowerCase(); + if(!cat.contains(type)){ + continue; + } + if(!tableSet.contains(table.getId())){ + continue; + } + LOG.trace("table = {}",gson.toJson(table)); + Map map = new LinkedHashMap<>(); + map.put("type", type); + map.put("schema", table.getSchema()); + map.put("table", table.getTable()); + map.put("lsn", table.getCurrentLsn()); + map.put("ts", table.getTs()); + map.put("ingestion", System.nanoTime()); + if(pavingData){ + int i = 0; + for (MetaColumn column : table.getColumnList()) { + map.put("before_" + column.getName(), table.getOldData()[i]); + map.put("after_" + column.getName(), table.getNewData()[i]); + i++; + } + }else { + Map before = new LinkedHashMap<>(); + Map after = new LinkedHashMap<>(); + int i = 0; + for (MetaColumn column : table.getColumnList()) { + before.put(column.getName(), table.getOldData()[i]); + after.put(column.getName(), table.getNewData()[i]); + i++; + } + map.put("before", before); + map.put("after", after); + } + format.processEvent(map); + } + }catch (Exception e){ + String errorMessage = ExceptionUtil.getErrorMessage(e); + LOG.error(errorMessage); + format.processEvent(Collections.singletonMap("e", errorMessage)); + + } + } +} diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java new file mode 100644 index 0000000000..5dcfaf888c --- /dev/null +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/reader/PgwalReader.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.pgwal.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.pgwal.PgWalConfigKeys; +import com.dtstack.flinkx.pgwal.format.PgWalInputFormatBuilder; +import com.dtstack.flinkx.reader.BaseDataReader; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * Date: 2019/12/13 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PgwalReader extends BaseDataReader { + private String username; + private String password; + private String url; + private String databaseName; + private String cat; + private boolean pavingData; + private List tableList; + private Integer statusInterval; + private Long lsn; + private String slotName; + private boolean allowCreateSlot; + private boolean temporary; + + @SuppressWarnings("unchecked") + public PgwalReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + username = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_USER_NAME); + password = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_PASSWORD); + url = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_URL); + databaseName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_DATABASE_NAME); + cat = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_CATALOG); + pavingData = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_PAVING_DATA, false); + tableList = (List) readerConfig.getParameter().getVal(PgWalConfigKeys.KEY_TABLE_LIST); + statusInterval = readerConfig.getParameter().getIntVal(PgWalConfigKeys.KEY_STATUS_INTERVAL, 20000); + lsn = readerConfig.getParameter().getLongVal(PgWalConfigKeys.KEY_LSN, 0); + slotName = readerConfig.getParameter().getStringVal(PgWalConfigKeys.KEY_SLOT_NAME); + allowCreateSlot = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_ALLOW_CREATE_SLOT, true); + temporary = readerConfig.getParameter().getBooleanVal(PgWalConfigKeys.KEY_TEMPORARY, true); + } + + @Override + public DataStream readData() { + PgWalInputFormatBuilder builder = new PgWalInputFormatBuilder(); + builder.setUsername(username); + builder.setPassword(password); + builder.setUrl(url); + builder.setDatabaseName(databaseName); + builder.setCat(cat); + builder.setPavingData(pavingData); + builder.setTableList(tableList); + builder.setRestoreConfig(restoreConfig); + builder.setStatusInterval(statusInterval); + builder.setLsn(lsn); + builder.setSlotName(slotName); + builder.setAllowCreateSlot(allowCreateSlot); + builder.setTemporary(temporary); + return createInput(builder.finish(), "pgwalreader"); + } +} diff --git a/flinkx-pgwal/pom.xml b/flinkx-pgwal/pom.xml new file mode 100644 index 0000000000..01b1f0d7c8 --- /dev/null +++ b/flinkx-pgwal/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + pom + + flinkx-pgwal + + + flinkx-pgwal-core + flinkx-pgwal-reader + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + \ No newline at end of file diff --git a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml index 8c8d2d901a..195da1bee4 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-reader/pom.xml @@ -89,6 +89,7 @@ + diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 0cb861e1b2..656763e9b5 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -107,7 +107,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { if (splitWithRowCol) { columnCount = columnCount - 1; } - checkSize(columnCount, metaColumns); + hasNext = resultSet.next(); if (StringUtils.isEmpty(customSql)) { diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index 94c3d0fe5c..a284085fa6 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -89,8 +89,9 @@ + + tofile="${basedir}/../../syncplugins/phoenixwriter/${project.name}-${git.branch}.jar" /> diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml new file mode 100644 index 0000000000..e292c9b6e0 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-core/pom.xml @@ -0,0 +1,82 @@ + + + + flinkx-phoenix5 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-phoenix5-core + + + + org.apache.phoenix + phoenix-core + 5.0.0-HBase-2.0 + + + guava + com.google.guava + + + hadoop-auth + org.apache.hadoop + + + hbase-protocol-shaded + org.apache.hbase + + + hbase-protocol + org.apache.hbase + + + hbase-zookeeper + org.apache.hbase + + + + + org.apache.hbase + hbase-client + 2.1.10 + + + hadoop-common + org.apache.hadoop + + + hadoop-mapreduce-client-core + org.apache.hadoop + + + + + + org.apache.hadoop + hadoop-auth + 3.0.0 + + + + org.apache.hbase + hbase-zookeeper + 2.1.10 + + + org.apache.hbase + hbase-common + 2.1.10 + + + + org.codehaus.janino + janino + 3.1.1 + + + + \ No newline at end of file diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5ConfigKeys.java similarity index 73% rename from flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java rename to flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5ConfigKeys.java index 80ffe47276..bcb938c9be 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/IDecode.java +++ b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5ConfigKeys.java @@ -6,33 +6,27 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - *

+ * * http://www.apache.org/licenses/LICENSE-2.0 - *

+ * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.pulsar.writer.decoder; - -import java.util.Map; +package com.dtstack.flinkx.phoenix5; /** - * Date: 2019/11/21 + * Date: 2020/09/29 * Company: www.dtstack.com * * @author tudou */ -public interface IDecode { +public class Phoenix5ConfigKeys { + public static final String KEY_READ_FROM_HBASE = "readFromHbase"; - /** - * 解码给定的数据 - * - * @param message 待解码的内容 - * @return 解码后的数据 - */ - Map decode(String message); + public static final String KEY_SCAN_CACHE_SIZE = "scanCacheSize"; + public static final String KEY_SCAN_BATCH_SIZE = "scanBatchSize"; } diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5DatabaseMeta.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5DatabaseMeta.java new file mode 100644 index 0000000000..49c7cc3315 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5DatabaseMeta.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.phoenix5; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.enums.EDatabaseType; +import com.dtstack.flinkx.rdb.BaseDatabaseMeta; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.commons.lang.StringUtils; + +import java.util.List; +import java.util.Map; + +/** + * The class of Phoenix prototype + * + * Company: www.dtstack.com + * @author wuhui + */ +public class Phoenix5DatabaseMeta extends BaseDatabaseMeta { + + @Override + public EDatabaseType getDatabaseType() { + return EDatabaseType.Phoenix; + } + + @Override + public String getDriverClass() { + return "org.apache.phoenix.jdbc.PhoenixDriver"; + } + + @Override + public String getSqlQueryFields(String tableName) { + return "SELECT * FROM " + tableName + " LIMIT 0"; + } + + @Override + public String getSqlQueryColumnFields(List column, String table) { + return "SELECT " + quoteColumns(column) + " FROM " + quoteTable(table) + " LIMIT 0"; + } + + /** + * phoenix查询表结构SQL + * @param metaColumns 字段信息 + * @param table schema.table + * @return + */ + public String getSqlWithLimit1(List metaColumns, String table) { + String columnStr; + List column = DbUtil.buildSelectColumns(this, metaColumns); + if(column.size() == 1 && ConstantValue.STAR_SYMBOL.equals(column.get(0))){ + columnStr = ConstantValue.STAR_SYMBOL; + }else{ + columnStr = quoteColumns(column); + } + return new StringBuilder(256) + .append("SELECT ") + .append(columnStr) + .append(" FROM ") + .append(quoteTable(table)) + .append(" LIMIT 1") + .toString(); + } + + @Override + public String getStartQuote() { + // 对于字段名和表名的quote得用双引号,对于字段值为字符串的得用单引号表示常量 + return ""; + } + + @Override + public String getEndQuote() { + return ""; + } + + @Override + public String quoteValue(String value, String column) { + return String.format("\"%s\" as %s",value,column); + } + + @Override + public String getInsertStatement(List column, String table) { + throw new UnsupportedOperationException(); + } + + @Override + public String getReplaceStatement(List column, List fullColumn, String table, Map> updateKey) { + // phoenix只支持upsert插入 + throw new UnsupportedOperationException(); + } + + @Override + public String getUpsertStatement(List column, String table, Map> updateKey) { + return "UPSERT INTO " + quoteTable(table) + + " (" + quoteColumns(column) + ") values (" + + StringUtils.repeat("?", ",", column.size()) + ")"; + } + + @Override + public String getSplitFilter(String columnName) { + // phoenix不支持mod,只支持%取余 + return String.format("%s %% ${N} = ${M}", getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getSplitFilterWithTmpTable(String tmpTable, String columnName){ + return String.format("%s.%s %% ${N} = ${M}", tmpTable, getStartQuote() + columnName + getEndQuote()); + } + + @Override + public String getRowNumColumn(String orderBy) { + throw new RuntimeException("Not support row_number function"); + } + + @Override + protected String makeValues(List column) { + throw new UnsupportedOperationException(); + } + + @Override + public int getFetchSize(){ + return 1000; + } + + @Override + public int getQueryTimeout(){ + return 1000; + } +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5InputSplit.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5InputSplit.java new file mode 100644 index 0000000000..4d81749188 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/Phoenix5InputSplit.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.core.io.GenericInputSplit; + +import java.util.Vector; + +/** + * Date: 2020/09/29 + * Company: www.dtstack.com + * + * @author tudou + */ +public class Phoenix5InputSplit extends GenericInputSplit { + private static final long serialVersionUID = 1L; + + private Vector> splits; + + /** + * Creates a generic input split with the given split number. + * + * @param partitionNumber The number of the split's partition. + * @param totalNumberOfPartitions The total number of the splits (partitions). + */ + public Phoenix5InputSplit(int partitionNumber, int totalNumberOfPartitions, Vector> splits) { + super(partitionNumber, totalNumberOfPartitions); + this.splits = splits; + } + + public Vector> getSplits() { + return splits; + } +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/IPhoenix5Helper.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/IPhoenix5Helper.java new file mode 100644 index 0000000000..237b4c2c04 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/IPhoenix5Helper.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5.util; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.types.Row; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Properties; + +/** + * Date: 2020/10/10 + * Company: www.dtstack.com + * + * @author tudou + */ +public interface IPhoenix5Helper { + + String CLASS_STR = "public transient RowProjector rowProjector;\n" + + " public List instanceList;\n" + + "\n" + + " @Override\n" + + " public Connection getConn(String url, Properties properties) throws SQLException {\n" + + " Connection dbConn;\n" + + " synchronized (ClassUtil.LOCK_STR) {\n" + + " DriverManager.setLoginTimeout(10);\n" + + " // telnet\n" + + " TelnetUtil.telnet(url);\n" + + " dbConn = DriverManager.getConnection(url, properties);\n" + + " }\n" + + "\n" + + " return dbConn;\n" + + " }\n" + + "\n" + + " @Override\n" + + " public List> getRangeList(PreparedStatement ps) {\n" + + " List rangeList = ((PhoenixPreparedStatement) ps).getQueryPlan().getSplits();\n" + + " List> list = new ArrayList(rangeList.size());\n" + + " for (KeyRange keyRange : rangeList) {\n" + + " list.add(Pair.of(keyRange.getLowerRange(), keyRange.getUpperRange()));\n" + + " }\n" + + " return list;\n" + + " }\n" + + "\n" + + " @Override\n" + + " public Map> getFamilyMap(ResultSet resultSet) throws Exception {\n" + + " Field field = PhoenixResultSet.class.getDeclaredField(\"rowProjector\");\n" + + " field.setAccessible(true);\n" + + " rowProjector = (RowProjector) field.get(resultSet);\n" + + " field.setAccessible(false);\n" + + " Field c = PhoenixResultSet.class.getDeclaredField(\"context\");\n" + + " c.setAccessible(true);\n" + + " StatementContext context = (StatementContext)c.get(resultSet);\n" + + " c.setAccessible(false);\n" + + " return context.getScan().getFamilyMap();\n" + + " }\n" + + "\n" + + " @Override\n" + + " public void initInstanceList(List typeList) {\n" + + " instanceList = new ArrayList(typeList.size());\n" + + " for (String type : typeList) {\n" + + " instanceList.add(getPDataType(type));\n" + + " }\n" + + " }\n" + + "\n" + + "\n" + + " @Override\n" + + " public byte[] getScanProjector(ResultSet resultSet) {\n" + + " StatementContext context = ((PhoenixResultSet)resultSet).getContext();\n" + + " return context.getScan().getAttribute(\"scanProjector\");\n" + + " }\n" + + "\n" + + " @Override\n" + + " public Row getRow(byte[] bytes, int offset, int length) throws SQLException {\n" + + " int size = instanceList.size();\n" + + " Row row = new Row(size);\n" + + " ImmutableBytesWritable pointer = new ImmutableBytesWritable();\n" + + " for (int pos = 0; pos < size; pos++) {\n" + + " PDataType pDataType = (PDataType)instanceList.get(pos);\n" + + " row.setField(pos, rowProjector.getColumnProjector(pos).getValue(new ResultTuple(Result.create(Collections.singletonList(new NoTagsKeyValue(bytes, offset, length)))), pDataType, pointer));\n" + + " }\n" + + " return row;\n" + + " }\n" + + "\n" + + " @Override\n" + + " public Map analyzePhoenixUrl(String url) throws SQLException {\n" + + " PhoenixEmbeddedDriver.ConnectionInfo info = PhoenixEmbeddedDriver.ConnectionInfo.create(url);\n" + + " Map map = new HashMap(8);\n" + + " //zk地址\n" + + " map.put(\"quorum\", info.getZookeeperQuorum());\n" + + " //zk端口\n" + + " map.put(\"port\", info.getPort());\n" + + " //hbase zk节点名称\n" + + " map.put(\"rootNode\", info.getRootNode());\n" + + " map.put(\"principal\", info.getPrincipal());\n" + + " map.put(\"keytabFile\", info.getKeytab());\n" + + " return map;\n" + + " }\n" + + "\n" + + " /**\n" + + " * 根据字段类型获取Phoenix转换实例\n" + + " * phoenix支持以下数据类型\n" + + " * @param type\n" + + " * @return\n" + + " */\n" + + " public PDataType getPDataType(String type){\n" + + " if(StringUtils.isBlank(type)){\n" + + " throw new RuntimeException(\"type[\" + type + \"] cannot be blank!\");\n" + + " }\n" + + " switch (type.toUpperCase()){\n" + + " case \"INTEGER\" : return PInteger.INSTANCE;\n" + + " case \"UNSIGNED_INT\" : return PUnsignedInt.INSTANCE;\n" + + " case \"BIGINT\" : return PLong.INSTANCE;\n" + + " case \"UNSIGNED_LONG\" : return PUnsignedLong.INSTANCE;\n" + + " case \"TINYINT\" : return PTinyint.INSTANCE;\n" + + " case \"UNSIGNED_TINYINT\" : return PUnsignedTinyint.INSTANCE;\n" + + " case \"SMALLINT\" : return PSmallint.INSTANCE;\n" + + " case \"UNSIGNED_SMALLINT\" : return PUnsignedSmallint.INSTANCE;\n" + + " case \"FLOAT\" : return PFloat.INSTANCE;\n" + + " case \"UNSIGNED_FLOAT\" : return PUnsignedFloat.INSTANCE;\n" + + " case \"DOUBLE\" : return PDouble.INSTANCE;\n" + + " case \"UNSIGNED_DOUBLE\" : return PUnsignedDouble.INSTANCE;\n" + + " case \"DECIMAL\" : return PDecimal.INSTANCE;\n" + + " case \"BOOLEAN\" : return PBoolean.INSTANCE;\n" + + " case \"TIME\" : return PTime.INSTANCE;\n" + + " case \"DATE\" : return PDate.INSTANCE;\n" + + " case \"TIMESTAMP\" : return PTimestamp.INSTANCE;\n" + + " case \"UNSIGNED_TIME\" : return PUnsignedTime.INSTANCE;\n" + + " case \"UNSIGNED_DATE\" : return PUnsignedDate.INSTANCE;\n" + + " case \"UNSIGNED_TIMESTAMP\" : return PUnsignedTimestamp.INSTANCE;\n" + + " case \"VARCHAR\" : return PVarchar.INSTANCE;\n" + + " case \"CHAR\" : return PChar.INSTANCE;\n" + + " //不支持二进制字段类型\n" + + " case \"BINARY\" : throw new RuntimeException(\"type [BINARY] is unsupported!\");\n" + + " case \"VARBINARY\" : throw new RuntimeException(\"type [VARBINARY] is unsupported!\");\n" + + " default: throw new RuntimeException(\"type[\"+ type +\"] is unsupported!\");\n" + + " }\n" + + " }"; + + /** + * 获取phoenix jdbc连接 + * @param url + * @param properties + * @return + * @throws SQLException + */ + Connection getConn(String url, Properties properties) throws SQLException; + + /** + * 获取region边界 + * @param ps + * @return + */ + List> getRangeList(PreparedStatement ps); + + /** + * 获取HBase列族 + * @param resultSet + * @return + */ + Map> getFamilyMap(ResultSet resultSet) throws Exception; + + /** + * 初始化Phoenix数据格式实例 + * @param typeList + */ + void initInstanceList(List typeList); + + /** + * 获取ScanProjector属性 + * @param resultSet + * @return + * @throws Exception + */ + byte [] getScanProjector(ResultSet resultSet); + + /** + * 获取HBase值 + * @param bytes + * @param offset + * @param length + * @return + * @throws SQLException + */ + Row getRow(byte[] bytes, int offset, int length) throws SQLException; + + /** + * 解析Phoenix JDBC URL + * @param url + * @return + * @throws SQLException + */ + Map analyzePhoenixUrl(String url) throws SQLException; +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java new file mode 100644 index 0000000000..44eb1ebdaf --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-core/src/main/java/com/dtstack/flinkx/phoenix5/util/PhoenixUtil.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5.util; + +import org.codehaus.commons.compiler.CompileException; +import org.codehaus.janino.ClassBodyEvaluator; + +import java.io.IOException; +import java.io.StringReader; + +/** + * Date: 2020/02/28 + * Company: www.dtstack.com + * + * @author tudou + */ +public class PhoenixUtil { + + /** + * 通过指定类加载器获取helper + * @param parentClassLoader + * @return + * @throws IOException + * @throws CompileException + */ + public static IPhoenix5Helper getHelper(ClassLoader parentClassLoader) throws IOException, CompileException { + ClassBodyEvaluator cbe = new ClassBodyEvaluator(); + cbe.setParentClassLoader(parentClassLoader); + cbe.setDefaultImports("com.dtstack.flinkx.util.ClassUtil", + "com.dtstack.flinkx.util.TelnetUtil", + "org.apache.commons.lang3.StringUtils", + "org.apache.commons.lang3.tuple.Pair", + "org.apache.flink.types.Row", + "org.apache.hadoop.hbase.NoTagsKeyValue", + "org.apache.hadoop.hbase.client.Result", + "org.apache.hadoop.hbase.io.ImmutableBytesWritable", + "org.apache.phoenix.compile.RowProjector", + "org.apache.phoenix.compile.StatementContext", + "org.apache.phoenix.jdbc.PhoenixEmbeddedDriver", + "org.apache.phoenix.jdbc.PhoenixPreparedStatement", + "org.apache.phoenix.jdbc.PhoenixResultSet", + "org.apache.phoenix.query.KeyRange", + "org.apache.phoenix.schema.tuple.ResultTuple", + "org.apache.phoenix.schema.types.PBoolean", + "org.apache.phoenix.schema.types.PChar", + "org.apache.phoenix.schema.types.PDataType", + "org.apache.phoenix.schema.types.PDate", + "org.apache.phoenix.schema.types.PDecimal", + "org.apache.phoenix.schema.types.PDouble", + "org.apache.phoenix.schema.types.PFloat", + "org.apache.phoenix.schema.types.PInteger", + "org.apache.phoenix.schema.types.PLong", + "org.apache.phoenix.schema.types.PSmallint", + "org.apache.phoenix.schema.types.PTime", + "org.apache.phoenix.schema.types.PTimestamp", + "org.apache.phoenix.schema.types.PTinyint", + "org.apache.phoenix.schema.types.PUnsignedDate", + "org.apache.phoenix.schema.types.PUnsignedDouble", + "org.apache.phoenix.schema.types.PUnsignedFloat", + "org.apache.phoenix.schema.types.PUnsignedInt", + "org.apache.phoenix.schema.types.PUnsignedLong", + "org.apache.phoenix.schema.types.PUnsignedSmallint", + "org.apache.phoenix.schema.types.PUnsignedTime", + "org.apache.phoenix.schema.types.PUnsignedTimestamp", + "org.apache.phoenix.schema.types.PUnsignedTinyint", + "org.apache.phoenix.schema.types.PVarchar", + "java.lang.reflect.Field", + "java.sql.Connection", + "java.sql.DriverManager", + "java.sql.PreparedStatement", + "java.sql.ResultSet", + "java.sql.SQLException", + "java.util.ArrayList", + "java.util.Collections", + "java.util.HashMap", + "java.util.List", + "java.util.Map", + "java.util.NavigableSet", + "java.util.Properties"); + cbe.setImplementedInterfaces(new Class[]{IPhoenix5Helper.class}); + StringReader sr = new StringReader(IPhoenix5Helper.CLASS_STR); + return (IPhoenix5Helper) cbe.createInstance(sr); + } +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml new file mode 100644 index 0000000000..ab6fc2962a --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml @@ -0,0 +1,102 @@ + + + + flinkx-phoenix5 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-phoenix5-reader + + + com.dtstack.flinkx + flinkx-rdb-reader + 1.6 + provided + + + com.dtstack.flinkx + flinkx-phoenix5-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.phoenix5.com.google.common + + + com.google.thirdparty + shade.phoenix5.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java new file mode 100644 index 0000000000..e9516404ab --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java @@ -0,0 +1,288 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5.format; + +import com.dtstack.flinkx.phoenix5.Phoenix5DatabaseMeta; +import com.dtstack.flinkx.phoenix5.Phoenix5InputSplit; +import com.dtstack.flinkx.phoenix5.util.IPhoenix5Helper; +import com.dtstack.flinkx.phoenix5.util.PhoenixUtil; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormat; +import com.dtstack.flinkx.rdb.util.DbUtil; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RangeSplitUtil; +import com.dtstack.flinkx.util.ReflectionUtils; +import com.google.common.collect.Lists; +import org.apache.commons.io.FilenameUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NoTagsKeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.phoenix.query.QueryConstants; +import org.codehaus.commons.compiler.CompileException; +import sun.misc.URLClassPath; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.URL; +import java.net.URLClassLoader; +import java.sql.Connection; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Vector; + +import static com.dtstack.flinkx.rdb.util.DbUtil.clobToString; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class Phoenix5InputFormat extends JdbcInputFormat { + + //是否直接读取HBase的数据 + public boolean readFromHbase; + //一次读取返回的Results数量,默认为HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING + public int scanCacheSize; + //限定了一个Result中所包含的列的数量,如果一行数据被请求的列的数量超出Batch限制,那么这行数据会被拆成多个Results + //默认为-1,表示返回所有行 + public int scanBatchSize; + + public String sql; + private transient Scan scan; + private transient Table hTable; + private transient Iterator> keyRangeIterator; + private transient Iterator resultIterator; + + private transient URLClassLoader childFirstClassLoader; + private transient IPhoenix5Helper helper; + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + } + + @Override + public InputSplit[] createInputSplitsInternal(int minNumSplits) { + if(readFromHbase){ + LOG.warn("phoenix5reader config [readFromHbase] is true, FlinkX will read data from HBase directly!"); + Phoenix5DatabaseMeta metaData = (Phoenix5DatabaseMeta) this.databaseInterface; + sql = metaData.getSqlWithLimit1(metaColumns, table); + List> rangeList; + try { + dbConn = getConnection(); + ps = dbConn.prepareStatement(sql); + resultSet = ps.executeQuery(); + rangeList = helper.getRangeList(ps); + LOG.info("region's count = {}", rangeList.size()); + } catch (Exception e) { + String message = String.format("failed to query rangeList, sql = %s, dbUrl = %s, properties = %s, e = %s", sql, dbUrl, GsonUtil.GSON.toJson(properties), ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(message, e); + }finally { + DbUtil.closeDbResources(resultSet, ps, dbConn, false); + } + if(rangeList.size() < minNumSplits){ + String message = String.format("region's count [%s] must be less than or equal to channel number [%s], please reduce [channel] in FlinkX config!", rangeList.size(), minNumSplits); + throw new IllegalArgumentException(message); + } + List>> list = RangeSplitUtil.subListBySegment(rangeList, minNumSplits); + InputSplit[] splits = new InputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new Phoenix5InputSplit(i, minNumSplits, new Vector<>(list.get(i))); + } + return splits; + }else{ + return super.createInputSplitsInternal(minNumSplits); + } + } + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + if(readFromHbase){ + try { + Phoenix5DatabaseMeta metaData = (Phoenix5DatabaseMeta) this.databaseInterface; + sql = metaData.getSqlWithLimit1(metaColumns, table); + dbConn = getConnection(); + ps = dbConn.prepareStatement(sql); + resultSet = ps.executeQuery(); + ResultSetMetaData meta = ps.getMetaData(); + columnCount = meta.getColumnCount(); + List typeList = new ArrayList<>(columnCount); + for (int i = 1; i <= columnCount; i++) { + String name = meta.getColumnName(i); + String type = meta.getColumnTypeName(i); + typeList.add(type); + LOG.info("field count = {}, name = {}, type = {}", i, name, type); + } + helper.initInstanceList(typeList); + Vector> keyRangeList = ((Phoenix5InputSplit) inputSplit).getSplits(); + + scan = new Scan(); + keyRangeIterator = keyRangeList.iterator(); + Pair pair = keyRangeIterator.next(); + //for yarn session mode + scan.setStartRow(pair.getLeft()); + scan.setStopRow(pair.getRight()); + scan.setFamilyMap(helper.getFamilyMap(resultSet)); + scan.setLoadColumnFamiliesOnDemand(true); + scan.setAttribute("scanProjector", helper.getScanProjector(resultSet)); + scan.setAttribute("_NonAggregateQuery", QueryConstants.TRUE); + scan.setCaching(scanCacheSize); + scan.setBatch(scanBatchSize); + + Map map = helper.analyzePhoenixUrl(dbUrl); + Configuration hConfiguration = HBaseConfiguration.create(); + hConfiguration.set(HConstants.ZOOKEEPER_QUORUM, (String) map.get("quorum")); + Object port = map.get("port"); + if(port == null){ + hConfiguration.setInt(HConstants.CLIENT_ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT); + }else{ + hConfiguration.setInt(HConstants.CLIENT_ZOOKEEPER_CLIENT_PORT, (int)port); + } + Object rootNode = map.get("rootNode"); + if(rootNode == null){ + hConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); + }else{ + hConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, (String) rootNode); + } + hConfiguration.setBoolean(HConstants.CLUSTER_DISTRIBUTED, true); + org.apache.hadoop.hbase.client.Connection hConn = ConnectionFactory.createConnection(hConfiguration); + hTable = hConn.getTable(TableName.valueOf(table)); + resultIterator = hTable.getScanner(scan).iterator(); + } catch (Exception e) { + String message = String.format("openInputFormat() failed, dbUrl = %s, properties = %s, e = %s", dbUrl, GsonUtil.GSON.toJson(properties), ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(message, e); + }finally { + DbUtil.closeDbResources(resultSet, ps, dbConn, false); + } + }else{ + super.openInternal(inputSplit); + } + } + + @Override + public Row nextRecordInternal(Row row) throws IOException { + try { + row = new Row(columnCount); + if(readFromHbase){ + NoTagsKeyValue cell = (NoTagsKeyValue)resultIterator.next().listCells().get(0); + return helper.getRow(cell.getBuffer(), cell.getOffset(), cell.getLength()); + }else{ + if (!hasNext) { + return null; + } + for (int pos = 0; pos < row.getArity(); pos++) { + row.setField(pos, clobToString(resultSet.getObject(pos + 1))); + } + return super.nextRecordInternal(row); + } + } catch (Exception e) { + throw new IOException(String.format("Couldn't read data, e = %s", ExceptionUtil.getErrorMessage(e)), e); + } + } + + @Override + public boolean reachedEnd() throws IOException{ + if(readFromHbase){ + if(resultIterator.hasNext()){ + return false; + }else{ + if(keyRangeIterator.hasNext()){ + Pair pair = keyRangeIterator.next(); + LOG.info("switch regions, current region is [{}}] to [{}]", GsonUtil.GSON.toJson(pair.getLeft()), GsonUtil.GSON.toJson(pair.getRight())); + //for yarn session mode + scan.setStartRow(pair.getLeft()); + scan.setStopRow(pair.getRight()); + resultIterator = hTable.getScanner(scan).iterator(); + return reachedEnd(); + }else{ + return true; + } + } + }else{ + return super.reachedEnd(); + } + } + + /** + * 获取数据库连接,用于子类覆盖 + * @return connection + */ + @Override + protected Connection getConnection() throws SQLException { + Field declaredField = ReflectionUtils.getDeclaredField(getClass().getClassLoader(), "ucp"); + assert declaredField != null; + declaredField.setAccessible(true); + URLClassPath urlClassPath; + try { + urlClassPath = (URLClassPath) declaredField.get(getClass().getClassLoader()); + } catch (IllegalAccessException e) { + String message = String.format("cannot get urlClassPath from current classLoader, classLoader = %s, e = %s", getClass().getClassLoader(), ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(message, e); + } + declaredField.setAccessible(false); + + List needJar = Lists.newArrayList(); + for (URL url : urlClassPath.getURLs()) { + String urlFileName = FilenameUtils.getName(url.getPath()); + if (urlFileName.startsWith("flinkx-phoenix5-reader")) { + needJar.add(url); + break; + } + } + + ClassLoader parentClassLoader = getClass().getClassLoader(); + List list = new LinkedList<>(); + list.add("org.apache.flink"); + list.add("com.dtstack.flinkx"); + + childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, list.toArray(new String[0])); + + ClassUtil.forName(driverName, childFirstClassLoader); + if(StringUtils.isNotEmpty(username)){ + properties.setProperty("user", username); + } + if(StringUtils.isNotEmpty(password)){ + properties.setProperty("password", password); + } + + try { + helper = PhoenixUtil.getHelper(childFirstClassLoader); + return helper.getConn(dbUrl, properties); + } catch (IOException | CompileException e) { + String message = String.format("cannot get phoenix connection, dbUrl = %s, properties = %s, e = %s", dbUrl, GsonUtil.GSON.toJson(properties), ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(message, e); + } + } +} \ No newline at end of file diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5InputFormatBuilder.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5InputFormatBuilder.java new file mode 100644 index 0000000000..b125fbdabb --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5InputFormatBuilder.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5.reader; + +import com.dtstack.flinkx.phoenix5.format.Phoenix5InputFormat; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import org.apache.commons.lang3.StringUtils; + +/** + * Date: 2020/09/30 + * Company: www.dtstack.com + * + * @author tudou + */ +public class Phoenix5InputFormatBuilder extends JdbcInputFormatBuilder { + protected Phoenix5InputFormat format; + + //use for check + private String where; + private String customSql; + private String orderByColumn; + + public Phoenix5InputFormatBuilder(Phoenix5InputFormat format) { + super(format); + this.format = format; + } + + public void setReadFromHbase(boolean readFromHbase){ + format.readFromHbase = readFromHbase; + } + + public void setScanCacheSize(int scanCacheSize){ + format.scanCacheSize = scanCacheSize; + } + + public void setScanBatchSize(int scanBatchSize){ + format.scanBatchSize = scanBatchSize; + } + + @Override + protected void checkFormat() { + if(format.readFromHbase){ + StringBuilder sb = new StringBuilder(256); + if (StringUtils.isBlank(format.dbUrl)) { + sb.append("No database URL supplied\n"); + } + if(StringUtils.isNotBlank(where)){ + sb.append("cannot config [where] when readFromHbase is true, current where is : [") + .append(where) + .append("]\n"); + } + if(StringUtils.isNotBlank(customSql)){ + sb.append("cannot config [customSql] when readFromHbase is true, current customSql is : [") + .append(customSql) + .append("]\n"); + } + if(StringUtils.isNotBlank(orderByColumn)){ + sb.append("cannot config [orderByColumn] when readFromHbase is true, current orderByColumn is : [") + .append(orderByColumn) + .append("]"); + } + if(sb.length() > 0){ + throw new IllegalArgumentException(sb.toString()); + } + }else{ + super.checkFormat(); + } + } + + public void setWhere(String where) { + this.where = where; + } + + @Override + public void setCustomSql(String customSql) { + super.setCustomSql(customSql); + this.customSql = customSql; + } + + public void setOrderByColumn(String orderByColumn) { + this.orderByColumn = orderByColumn; + } +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5Reader.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5Reader.java new file mode 100644 index 0000000000..9f73d41a16 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/reader/Phoenix5Reader.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.phoenix5.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.phoenix5.Phoenix5ConfigKeys; +import com.dtstack.flinkx.phoenix5.Phoenix5DatabaseMeta; +import com.dtstack.flinkx.phoenix5.format.Phoenix5InputFormat; +import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; +import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.hadoop.hbase.HConstants; + +/** + * phoenix reader plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class Phoenix5Reader extends JdbcDataReader { + private boolean readFromHbase; + private int scanCacheSize; + private int scanBatchSize; + + public Phoenix5Reader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + readFromHbase = readerConfig.getParameter().getBooleanVal(Phoenix5ConfigKeys.KEY_READ_FROM_HBASE, false); + scanCacheSize = readerConfig.getParameter().getIntVal(Phoenix5ConfigKeys.KEY_SCAN_CACHE_SIZE, HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING); + scanBatchSize = readerConfig.getParameter().getIntVal(Phoenix5ConfigKeys.KEY_SCAN_BATCH_SIZE, -1); + setDatabaseInterface(new Phoenix5DatabaseMeta()); + } + + @Override + protected JdbcInputFormatBuilder getBuilder() { + Phoenix5InputFormatBuilder builder = new Phoenix5InputFormatBuilder(new Phoenix5InputFormat()); + builder.setReadFromHbase(readFromHbase); + builder.setScanCacheSize(scanCacheSize); + builder.setScanBatchSize(scanBatchSize); + builder.setWhere(where); + builder.setCustomSql(customSql); + builder.setOrderByColumn(orderByColumn); + return builder; + } + +} + diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml new file mode 100644 index 0000000000..07912fc063 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml @@ -0,0 +1,103 @@ + + + + flinkx-phoenix5 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-phoenix5-writer + + + + com.dtstack.flinkx + flinkx-rdb-writer + 1.6 + provided + + + com.dtstack.flinkx + flinkx-phoenix5-core + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.phoenix5.com.google.common + + + com.google.thirdparty + shade.phoenix5.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java new file mode 100644 index 0000000000..07efa749a3 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.phoenix5.format; + +import com.dtstack.flinkx.enums.EWriteMode; +import com.dtstack.flinkx.phoenix5.util.PhoenixUtil; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ReflectionUtils; +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.io.FilenameUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.phoenix.query.QueryServices; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.misc.URLClassPath; + +import java.lang.reflect.Field; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.List; + +/** + * Company: www.dtstack.com + * + * @author wuhui + */ +public class Phoenix5OutputFormat extends JdbcOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(Phoenix5OutputFormat.class); + + private static final String PHOENIX5_WRITER_PREFIX = "flinkx-phoenix5-writer"; + + @Override + protected void openInternal(int taskNumber, int numTasks){ + try { + Field declaredField = ReflectionUtils.getDeclaredField(getClass().getClassLoader(), "ucp"); + declaredField.setAccessible(true); + URLClassPath urlClassPath = (URLClassPath) declaredField.get(getClass().getClassLoader()); + declaredField.setAccessible(false); + + List needJar = Lists.newArrayList(); + for(URL url : urlClassPath.getURLs()){ + String urlFileName = FilenameUtils.getName(url.getPath()); + if(urlFileName.startsWith(PHOENIX5_WRITER_PREFIX)){ + needJar.add(url); + } + } + + ClassLoader parentClassLoader = getClass().getClassLoader(); + String[] alwaysParentFirstPatterns = new String[2]; + alwaysParentFirstPatterns[0] = "org.apache.flink"; + alwaysParentFirstPatterns[1] = "com.dtstack.flinkx"; + URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns); + + ClassUtil.forName(driverName, childFirstClassLoader); + if(StringUtils.isNotEmpty(username)){ + properties.setProperty("user", username); + } + if(StringUtils.isNotEmpty(password)){ + properties.setProperty("password", password); + } + if(properties.get(QueryServices.MUTATE_BATCH_SIZE_ATTRIB) == null){ + //执行过程中被批处理并自动提交的行数 + properties.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, "100000"); + } + if(properties.get(QueryServices.MAX_MUTATION_SIZE_ATTRIB) == null){ + //客户端批处理的最大行数 + properties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "1000000"); + } + if(properties.get(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB) == null){ + //客户端批处理的最大数据量(单位:B)1GB + properties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "1073741824"); + } + dbConn = PhoenixUtil.getHelper(childFirstClassLoader).getConn(dbUrl, properties); + + if (restoreConfig.isRestore()){ + dbConn.setAutoCommit(false); + } + + if(CollectionUtils.isEmpty(fullColumn)) { + fullColumn = probeFullColumns(table, dbConn); + } + + if(fullColumnType == null) { + fullColumnType = analyzeTable(); + } + + for(String col : column) { + for (int i = 0; i < fullColumn.size(); i++) { + if (col.equalsIgnoreCase(fullColumn.get(i))){ + columnType.add(fullColumnType.get(i)); + break; + } + } + } + mode = EWriteMode.UPDATE.name(); + preparedStatement = prepareTemplates(); + readyCheckpoint = false; + + LOG.info("subTask[{}}] wait finished", taskNumber); + } catch (Exception sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } + } + +} diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java new file mode 100644 index 0000000000..25d76cb568 --- /dev/null +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.phoenix5.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.phoenix5.Phoenix5DatabaseMeta; +import com.dtstack.flinkx.phoenix5.format.Phoenix5OutputFormat; +import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; +import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; +import com.dtstack.flinkx.rdb.util.DbUtil; + +import java.util.Collections; + +/** + * phoenix writer plugin + * + * Company: www.dtstack.com + * @author wuhui + */ +public class Phoenix5Writer extends JdbcDataWriter { + + public Phoenix5Writer(DataTransferConfig config) { + super(config); + setDatabaseInterface(new Phoenix5DatabaseMeta()); + dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); + } + + @Override + protected JdbcOutputFormatBuilder getBuilder() { + return new JdbcOutputFormatBuilder(new Phoenix5OutputFormat()); + } +} diff --git a/flinkx-phoenix5/pom.xml b/flinkx-phoenix5/pom.xml new file mode 100644 index 0000000000..cc8aeef584 --- /dev/null +++ b/flinkx-phoenix5/pom.xml @@ -0,0 +1,36 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + pom + + flinkx-phoenix5 + + + flinkx-phoenix5-core + flinkx-phoenix5-reader + flinkx-phoenix5-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + com.dtstack.flinkx + flinkx-rdb-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-polardb/flinkx-polardb-dreader/pom.xml b/flinkx-polardb/flinkx-polardb-dreader/pom.xml index 278cf2009b..d4bd90d2c4 100644 --- a/flinkx-polardb/flinkx-polardb-dreader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-dreader/pom.xml @@ -98,6 +98,7 @@ + diff --git a/flinkx-polardb/flinkx-polardb-reader/pom.xml b/flinkx-polardb/flinkx-polardb-reader/pom.xml index 948a6c1cfd..60e39a8821 100644 --- a/flinkx-polardb/flinkx-polardb-reader/pom.xml +++ b/flinkx-polardb/flinkx-polardb-reader/pom.xml @@ -97,6 +97,7 @@ + diff --git a/flinkx-polardb/flinkx-polardb-writer/pom.xml b/flinkx-polardb/flinkx-polardb-writer/pom.xml index f97f7afcf4..100ecc20ad 100644 --- a/flinkx-polardb/flinkx-polardb-writer/pom.xml +++ b/flinkx-polardb/flinkx-polardb-writer/pom.xml @@ -98,6 +98,7 @@ + diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java index 04f5616f23..b4bff44466 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java @@ -21,9 +21,11 @@ import com.dtstack.flinkx.rdb.type.TypeConverterInterface; import org.apache.commons.lang3.StringUtils; +import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Locale; /** * The type converter for PostgreSQL database @@ -43,6 +45,9 @@ public class PostgresqlTypeConverter implements TypeConverterInterface { private List intTypes = Arrays.asList("int","int2","int4","int8"); + protected static List STRING_TYPES = Arrays.asList("CHAR", "VARCHAR","TINYBLOB","TINYTEXT","BLOB","TEXT", "MEDIUMBLOB", "MEDIUMTEXT", "LONGBLOB", "LONGTEXT"); + + @Override public Object convert(Object data,String typeName) { if (data == null){ @@ -53,6 +58,10 @@ public Object convert(Object data,String typeName) { return dataValue; } if(StringUtils.isBlank(dataValue)){ + //如果是string类型 还应该返回空字符串而不是null + if(STRING_TYPES.contains(typeName.toUpperCase(Locale.ENGLISH))){ + return dataValue; + } return null; } if(doubleTypes.contains(typeName)){ @@ -65,7 +74,10 @@ public Object convert(Object data,String typeName) { }else if(byteTypes.contains(typeName)){ data = Byte.valueOf(dataValue); } else if(intTypes.contains(typeName)){ - data = Integer.parseInt(dataValue); + if(dataValue.contains(".")){ + dataValue = new BigDecimal(dataValue).stripTrailingZeros().toPlainString(); + } + data = Long.parseLong(dataValue); } return data; diff --git a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml index 3ea22aa8be..296ca62033 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-reader/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlQuerySqlBuilder.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlQuerySqlBuilder.java index 6868aeb8e4..ad07571d4f 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlQuerySqlBuilder.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlQuerySqlBuilder.java @@ -19,6 +19,7 @@ import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; import com.dtstack.flinkx.rdb.datareader.QuerySqlBuilder; +import com.dtstack.flinkx.rdb.util.DbUtil; import org.apache.commons.lang3.StringUtils; import java.util.List; @@ -37,7 +38,7 @@ public PostgresqlQuerySqlBuilder(JdbcDataReader reader){ @Override protected String buildQuerySql(){ - List selectColumns = buildSelectColumns(databaseInterface, metaColumns); + List selectColumns = DbUtil.buildSelectColumns(databaseInterface, metaColumns); boolean splitWithRowNum = addRowNumColumn(databaseInterface, selectColumns, isSplitByKey, splitKey); StringBuilder sb = new StringBuilder(); diff --git a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml index 3f7039e64b..5a64580055 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/pom.xml +++ b/flinkx-postgresql/flinkx-postgresql-writer/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlOutputFormat.java b/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlOutputFormat.java index 203be225f2..0ced5ac9bb 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlOutputFormat.java +++ b/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/format/PostgresqlOutputFormat.java @@ -20,6 +20,7 @@ import com.dtstack.flinkx.enums.EWriteMode; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormat; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.types.Row; import org.postgresql.copy.CopyManager; import org.postgresql.core.BaseConnection; @@ -46,6 +47,8 @@ public class PostgresqlOutputFormat extends JdbcOutputFormat { private static final String LINE_DELIMITER = "\n"; + private boolean isCopyMode = false; + /** * now just add ext insert mode:copy */ @@ -63,7 +66,8 @@ protected PreparedStatement prepareTemplates() throws SQLException { } //check is use copy mode for insert - if (EWriteMode.INSERT.name().equalsIgnoreCase(mode) && checkIsCopyMode(insertSqlMode)) { + isCopyMode = checkIsCopyMode(insertSqlMode); + if (EWriteMode.INSERT.name().equalsIgnoreCase(mode) && isCopyMode) { copyManager = new CopyManager((BaseConnection) dbConn); copySql = String.format(COPY_SQL_TEMPL, table, String.join(",", column), DEFAULT_FIELD_DELIM, DEFAULT_NULL_DELIM); return null; @@ -72,27 +76,10 @@ protected PreparedStatement prepareTemplates() throws SQLException { return super.prepareTemplates(); } - @Override - protected void openInternal(int taskNumber, int numTasks){ - super.openInternal(taskNumber, numTasks); - try { - if (batchInterval > 1) { - dbConn.setAutoCommit(false); - } - } catch (Exception e) { - LOG.warn("", e); - } - } - @Override protected void writeSingleRecordInternal(Row row) throws WriteRecordException { - if(!checkIsCopyMode(insertSqlMode)){ - if (batchInterval == 1) { - super.writeSingleRecordInternal(row); - } else { - writeSingleRecordCommit(row); - } - + if(!isCopyMode){ + super.writeSingleRecordInternal(row); return; } @@ -106,17 +93,23 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { if(rowData==null){ sb.append(DEFAULT_NULL_DELIM); }else{ - String data = String.valueOf(rowData); - if(data.contains("\\")){ - data= data.replaceAll("\\\\","\\\\\\\\"); - } - sb.append(data); + sb.append(rowData); } if(index != lastIndex){ sb.append(DEFAULT_FIELD_DELIM); } } String rowVal = sb.toString(); + if(rowVal.contains("\\")){ + rowVal= rowVal.replaceAll("\\\\","\\\\\\\\"); + } + if(rowVal.contains("\r")){ + rowVal= rowVal.replaceAll("\r","\\\\r"); + } + + if(rowVal.contains("\n")){ + rowVal= rowVal.replaceAll("\n","\\\\n"); + } ByteArrayInputStream bi = new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8)); copyManager.copyIn(copySql, bi); } catch (Exception e) { @@ -124,54 +117,41 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { } } - private void writeSingleRecordCommit(Row row) throws WriteRecordException { - try { - super.writeSingleRecordInternal(row); - try { - dbConn.commit(); - } catch (Exception e) { - // 提交失败直接结束任务 - throw new RuntimeException(e); - } - } catch (WriteRecordException e) { - try { - dbConn.rollback(); - } catch (Exception e1) { - // 回滚失败直接结束任务 - throw new RuntimeException(e); - } - - throw e; - } - } - @Override protected void writeMultipleRecordsInternal() throws Exception { - if(!checkIsCopyMode(insertSqlMode)){ - writeMultipleRecordsCommit(); + if(!isCopyMode){ + super.writeMultipleRecordsInternal(); return; } StringBuilder sb = new StringBuilder(128); for (Row row : rows) { int lastIndex = row.getArity() - 1; + StringBuilder tempBuilder = new StringBuilder(128); for (int index =0; index < row.getArity(); index++) { Object rowData = getField(row, index); if(rowData==null){ - sb.append(DEFAULT_NULL_DELIM); + tempBuilder.append(DEFAULT_NULL_DELIM); }else{ - String data = String.valueOf(rowData); - if(data.contains("\\")){ - data= data.replaceAll("\\\\","\\\\\\\\"); - } - sb.append(data); + tempBuilder.append(rowData); } if(index != lastIndex){ - sb.append(DEFAULT_FIELD_DELIM); + tempBuilder.append(DEFAULT_FIELD_DELIM); } } + // \r \n \ 等特殊字符串需要转义 + String tempData = tempBuilder.toString(); + if(tempData.contains("\\")){ + tempData= tempData.replaceAll("\\\\","\\\\\\\\"); + } + if(tempData.contains("\r")){ + tempData= tempData.replaceAll("\r","\\\\r"); + } - sb.append(LINE_DELIMITER); + if(tempData.contains("\n")){ + tempData= tempData.replaceAll("\n","\\\\n"); + } + sb.append(tempData).append(LINE_DELIMITER); } String rowVal = sb.toString(); @@ -183,16 +163,6 @@ protected void writeMultipleRecordsInternal() throws Exception { } } - private void writeMultipleRecordsCommit() throws Exception { - try { - super.writeMultipleRecordsInternal(); - dbConn.commit(); - } catch (Exception e){ - dbConn.rollback(); - throw e; - } - } - @Override protected Object getField(Row row, int index) { Object field = super.getField(row, index); @@ -202,8 +172,13 @@ protected Object getField(Row row, int index) { return field; } + /** + * 判断是否为copy模式 + * @param insertMode + * @return + */ private boolean checkIsCopyMode(String insertMode){ - if(insertMode == null || insertMode.length() == 0){ + if(StringUtils.isBlank(insertMode)){ return false; } @@ -213,6 +188,4 @@ private boolean checkIsCopyMode(String insertMode){ return true; } - - } diff --git a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml index 44a622ca75..49ec0fab1d 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/pom.xml +++ b/flinkx-pulsar/flinkx-pulsar-writer/pom.xml @@ -1,6 +1,6 @@ - flinkx-pulsar diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/Constants.java similarity index 96% rename from flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java rename to flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/Constants.java index 28cd09eaad..029ac9ecc8 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/Constants.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/Constants.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.pulsar.writer; +package com.dtstack.flinkx.pulsar.format; /** * @author: pierre diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormat.java similarity index 91% rename from flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java rename to flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormat.java index fdfab032da..efbc945c7b 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormat.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormat.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.pulsar.writer; +package com.dtstack.flinkx.pulsar.format; +import com.dtstack.flinkx.decoder.JsonDecoder; import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; -import com.dtstack.flinkx.pulsar.writer.decoder.JsonDecoder; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.flink.types.Row; +import org.apache.flink.util.StringUtils; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import org.codehaus.jackson.map.ObjectMapper; import java.io.IOException; import java.util.Collections; @@ -49,7 +50,6 @@ public class PulsarOutputFormat extends BaseRichOutputFormat { protected List tableFields; protected static JsonDecoder jsonDecoder = new JsonDecoder(); - protected static ObjectMapper objectMapper = new ObjectMapper(); @Override protected void openInternal(int taskNumber, int numTasks) throws IOException { @@ -81,7 +81,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { if (tableFields != null && tableFields.size() >= arity) { map = new LinkedHashMap<>((arity << 2) / 3); for (int i = 0; i < arity; i++) { - map.put(tableFields.get(i), org.apache.flink.util.StringUtils.arrayAwareToString(row.getField(i))); + map.put(tableFields.get(i), StringUtils.arrayAwareToString(row.getField(i))); } } else { if (arity == 1) { @@ -105,7 +105,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { } protected void emit(Map event) throws IOException { - producer.send(objectMapper.writeValueAsString(event)); + producer.send(MapUtil.writeValueAsString(event)); } @Override diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormatBuilder.java similarity index 97% rename from flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java rename to flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormatBuilder.java index c6e35cc79c..c2ed5acf97 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarOutputFormatBuilder.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/format/PulsarOutputFormatBuilder.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flinkx.pulsar.writer; +package com.dtstack.flinkx.pulsar.format; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java index 2068128992..dc09d6fb7a 100644 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java +++ b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/PulsarWriter.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.pulsar.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.pulsar.format.PulsarOutputFormatBuilder; import com.dtstack.flinkx.writer.BaseDataWriter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -26,7 +27,7 @@ import java.util.List; import java.util.Map; -import static com.dtstack.flinkx.pulsar.writer.Constants.*; +import static com.dtstack.flinkx.pulsar.format.Constants.*; /** diff --git a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java b/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java deleted file mode 100644 index 5efc3cf298..0000000000 --- a/flinkx-pulsar/flinkx-pulsar-writer/src/main/java/com/dtstack/flinkx/pulsar/writer/decoder/JsonDecoder.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.pulsar.writer.decoder; - -import org.codehaus.jackson.map.ObjectMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.Map; - -/** - * Date: 2019/11/21 - * Company: www.dtstack.com - * - * @author tudou - */ -public class JsonDecoder implements IDecode { - private static Logger LOG = LoggerFactory.getLogger(JsonDecoder.class); - - private static ObjectMapper objectMapper = new ObjectMapper(); - - private static final String KEY_MESSAGE = "message"; - - @Override - @SuppressWarnings("unchecked") - public Map decode(final String message) { - try { - Map event = objectMapper.readValue(message, Map.class); - if (!event.containsKey(KEY_MESSAGE)) { - event.put(KEY_MESSAGE, message); - } - return event; - } catch (Exception e) { - LOG.error(e.getMessage()); - return Collections.singletonMap(KEY_MESSAGE, message); - } - } -} diff --git a/flinkx-rdb/flinkx-rdb-core/pom.xml b/flinkx-rdb/flinkx-rdb-core/pom.xml index 189de1ac45..36a9305c05 100644 --- a/flinkx-rdb/flinkx-rdb-core/pom.xml +++ b/flinkx-rdb/flinkx-rdb-core/pom.xml @@ -31,6 +31,7 @@ + diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 891a60533d..1c5ea0d024 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.rdb.util; import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.rdb.ParameterValuesProvider; import com.dtstack.flinkx.reader.MetaColumn; import com.dtstack.flinkx.util.ClassUtil; @@ -40,6 +41,7 @@ import java.sql.Statement; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.regex.Pattern; @@ -191,6 +193,8 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con try { if(commit){ commit(conn); + }else { + rollBack(conn); } conn.close(); @@ -206,7 +210,7 @@ public static void closeDbResources(ResultSet rs, Statement stmt, Connection con */ public static void commit(Connection conn){ try { - if (!conn.isClosed() && !conn.getAutoCommit()){ + if (null != conn && !conn.isClosed() && !conn.getAutoCommit()){ conn.commit(); } } catch (SQLException e){ @@ -214,6 +218,20 @@ public static void commit(Connection conn){ } } + /** + * 手动回滚事物 + * @param conn Connection + */ + public static void rollBack(Connection conn){ + try { + if (null != conn && !conn.isClosed() && !conn.getAutoCommit()){ + conn.rollback(); + } + } catch (SQLException e){ + LOG.warn("rollBack error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + /** * 批量执行sql * @param dbConn Connection @@ -266,18 +284,23 @@ public static List analyzeColumnType(ResultSet resultSet, List nameTypeMap = new HashMap<>((rd.getColumnCount() << 2) / 3); + Map nameTypeMap = new LinkedHashMap<>((rd.getColumnCount() << 2) / 3); for(int i = 0; i < rd.getColumnCount(); ++i) { nameTypeMap.put(rd.getColumnName(i+1),rd.getColumnTypeName(i+1)); } - for (MetaColumn metaColumn : metaColumns) { - if(metaColumn.getValue() != null){ - columnTypeList.add("VARCHAR"); - } else { - columnTypeList.add(nameTypeMap.get(metaColumn.getName())); + if (ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ + columnTypeList.addAll(nameTypeMap.values()); + }else{ + for (MetaColumn metaColumn : metaColumns) { + if(metaColumn.getValue() != null){ + columnTypeList.add("VARCHAR"); + } else { + columnTypeList.add(nameTypeMap.get(metaColumn.getName())); + } } } + } catch (SQLException e) { String message = String.format("error to analyzeSchema, resultSet = %s, columnTypeList = %s, e = %s", resultSet, @@ -408,4 +431,27 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap) return sb.toString(); } + + /** + * 构造select字段list + * @param databaseInterface + * @param metaColumns + * @return + */ + public static List buildSelectColumns(DatabaseInterface databaseInterface, List metaColumns){ + List selectColumns = new ArrayList<>(); + if(metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ + selectColumns.add(ConstantValue.STAR_SYMBOL); + } else { + for (MetaColumn metaColumn : metaColumns) { + if (metaColumn.getValue() != null){ + selectColumns.add(databaseInterface.quoteValue(metaColumn.getValue(),metaColumn.getName())); + } else { + selectColumns.add(databaseInterface.quoteColumn(metaColumn.getName())); + } + } + } + + return selectColumns; + } } \ No newline at end of file diff --git a/flinkx-rdb/flinkx-rdb-reader/pom.xml b/flinkx-rdb/flinkx-rdb-reader/pom.xml index ecd46d3def..1bd6ff5a5d 100644 --- a/flinkx-rdb/flinkx-rdb-reader/pom.xml +++ b/flinkx-rdb/flinkx-rdb-reader/pom.xml @@ -40,6 +40,7 @@ + diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcConfigKeys.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcConfigKeys.java index e11716d894..00919f7e0a 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcConfigKeys.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcConfigKeys.java @@ -53,4 +53,6 @@ public class JdbcConfigKeys { public static final String KEY_POLLING = "polling"; public static final String KEY_POLLING_INTERVAL = "pollingInterval"; + + public static final String KEY_PROPERTIES = "properties"; } diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcDataReader.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcDataReader.java index ebf962474a..7eb7fc27cf 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcDataReader.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/JdbcDataReader.java @@ -34,6 +34,7 @@ import org.apache.flink.types.Row; import java.util.List; +import java.util.Properties; import java.util.Objects; /** @@ -47,6 +48,8 @@ public class JdbcDataReader extends BaseDataReader { protected String username; protected String password; protected String dbUrl; + protected Properties properties; + protected String table; protected String where; @@ -77,6 +80,7 @@ public JdbcDataReader(DataTransferConfig config, StreamExecutionEnvironment env) splitKey = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_SPLIK_KEY); customSql = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_CUSTOM_SQL,null); orderByColumn = readerConfig.getParameter().getStringVal(JdbcConfigKeys.KEY_ORDER_BY_COLUMN,null); + properties = readerConfig.getParameter().getProperties(JdbcConfigKeys.KEY_PROPERTIES, null); buildIncrementConfig(readerConfig); } @@ -101,6 +105,7 @@ public DataStream readData() { builder.setSplitKey(splitKey); builder.setNumPartitions(numPartitions); builder.setCustomSql(customSql); + builder.setProperties(properties); builder.setRestoreConfig(restoreConfig); builder.setHadoopConfig(hadoopConfig); builder.setTestConfig(testConfig); diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java index 31ecf742bb..e82cf3e6b2 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.datareader/QuerySqlBuilder.java @@ -21,10 +21,10 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.rdb.DatabaseInterface; +import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.reader.MetaColumn; import org.apache.commons.lang3.StringUtils; -import java.util.ArrayList; import java.util.List; /** @@ -92,7 +92,7 @@ public String buildSql(){ } protected String buildQuerySql(){ - List selectColumns = buildSelectColumns(databaseInterface, metaColumns); + List selectColumns = DbUtil.buildSelectColumns(databaseInterface, metaColumns); boolean splitWithRowNum = addRowNumColumn(databaseInterface, selectColumns, isSplitByKey, splitKey); StringBuilder sb = new StringBuilder(); @@ -170,23 +170,6 @@ protected String buildQuerySqlWithCustomSql(){ return querySql.toString(); } - protected static List buildSelectColumns(DatabaseInterface databaseInterface, List metaColumns){ - List selectColumns = new ArrayList<>(); - if(metaColumns.size() == 1 && ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())){ - selectColumns.add(ConstantValue.STAR_SYMBOL); - } else { - for (MetaColumn metaColumn : metaColumns) { - if (metaColumn.getValue() != null){ - selectColumns.add(databaseInterface.quoteValue(metaColumn.getValue(),metaColumn.getName())); - } else { - selectColumns.add(databaseInterface.quoteColumn(metaColumn.getName())); - } - } - } - - return selectColumns; - } - protected static boolean addRowNumColumn(DatabaseInterface databaseInterface, List selectColumns, boolean isSplitByKey,String splitKey){ if(!isSplitByKey || !splitKey.contains(ConstantValue.LEFT_PARENTHESIS_SYMBOL)){ return false; diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index ee2468b017..d4b91d450d 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -32,6 +32,7 @@ import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.UrlUtil; import com.fasterxml.jackson.databind.ObjectMapper; @@ -49,6 +50,7 @@ import java.math.BigInteger; import java.sql.Connection; import java.sql.Date; +import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; @@ -57,6 +59,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.concurrent.TimeUnit; /** @@ -68,54 +71,53 @@ */ public class JdbcInputFormat extends BaseRichInputFormat { - protected static final long serialVersionUID = 1L; - - protected static int resultSetType = ResultSet.TYPE_FORWARD_ONLY; - protected static final int resultSetConcurrency = ResultSet.CONCUR_READ_ONLY; - - protected DatabaseInterface databaseInterface; - - protected String table; - protected String queryTemplate; - protected String customSql; - protected String querySql; - - protected String splitKey; - protected int fetchSize; - protected int queryTimeOut; - protected IncrementConfig incrementConfig; - - protected String username; - protected String password; - protected String driverName; - protected String dbUrl; - protected transient Connection dbConn; - protected transient Statement statement; - protected transient PreparedStatement ps; - protected transient ResultSet resultSet; - protected boolean hasNext; - - - protected List metaColumns; - protected List columnTypeList; - protected int columnCount; - protected MetaColumn restoreColumn; - protected Row lastRow = null; + public static final long serialVersionUID = 1L; + public static final int resultSetConcurrency = ResultSet.CONCUR_READ_ONLY; + public static int resultSetType = ResultSet.TYPE_FORWARD_ONLY; + public DatabaseInterface databaseInterface; + + public String table; + public String queryTemplate; + public String customSql; + public String querySql; + + public String splitKey; + public int fetchSize; + public int queryTimeOut; + public IncrementConfig incrementConfig; + + public String username; + public String password; + public String driverName; + public String dbUrl; + public Properties properties; + public transient Connection dbConn; + public transient Statement statement; + public transient PreparedStatement ps; + public transient ResultSet resultSet; + public boolean hasNext; + + + public List metaColumns; + public List columnTypeList; + public int columnCount; + public MetaColumn restoreColumn; + public Row lastRow = null; //for postgre - protected TypeConverterInterface typeConverter; + public TypeConverterInterface typeConverter; - protected int numPartitions; + public int numPartitions; - protected StringAccumulator maxValueAccumulator; - protected BigIntegerMaximum endLocationAccumulator; - protected BigIntegerMaximum startLocationAccumulator; + public StringAccumulator maxValueAccumulator; + public BigIntegerMaximum endLocationAccumulator; + public BigIntegerMaximum startLocationAccumulator; //轮询增量标识字段类型 - protected ColumnType type; + public ColumnType type; //The hadoop config for metric - protected Map hadoopConfig; + public Map hadoopConfig; @Override public void openInputFormat() throws IOException { @@ -145,7 +147,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { querySql = buildQuerySql(inputSplit); try { executeQuery(((JdbcInputSplit) inputSplit).getStartLocation()); - columnCount = resultSet.getMetaData().getColumnCount(); + if(!resultSet.isClosed()){ + columnCount = resultSet.getMetaData().getColumnCount(); + } } catch (SQLException se) { throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } @@ -170,7 +174,7 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) { } @Override - public boolean reachedEnd() { + public boolean reachedEnd() throws IOException{ if (hasNext) { return false; } else { @@ -211,6 +215,7 @@ public boolean reachedEnd() { @Override public Row nextRecordInternal(Row row) throws IOException { try { + updateColumnCount(); if (!ConstantValue.STAR_SYMBOL.equals(metaColumns.get(0).getName())) { for (int i = 0; i < columnCount; i++) { Object val = row.getField(i); @@ -305,6 +310,11 @@ protected void initMetric(InputSplit inputSplit) { //endLocation设置为数据库中查询的最大值 String endLocation = ((JdbcInputSplit) inputSplit).getEndLocation(); endLocationAccumulator.add(new BigInteger(StringUtil.stringToTimestampStr(endLocation, type))); + }else{ + //增量任务,且useMaxFunc设置为false,如果startLocation不为空,则将endLocation初始值设置为startLocation的值,防止数据库无增量数据时下次获取到的startLocation为空 + if (StringUtils.isNotEmpty(startLocation)) { + endLocationAccumulator.add(new BigInteger(startLocation)); + } } //将累加器信息添加至prometheus @@ -370,7 +380,7 @@ private String getMaxValueFromDb() { LOG.info(String.format("Query max value sql is '%s'", queryMaxValueSql)); - conn = DbUtil.getConnection(dbUrl, username, password); + conn = getConnection(); st = conn.createStatement(resultSetType, resultSetConcurrency); rs = st.executeQuery(queryMaxValueSql); if (rs.next()) { @@ -742,7 +752,11 @@ protected void queryForPolling(String startLocation) throws SQLException { ps.setDate(1, date); break; default: - ps.setString(1, startLocation); + if(isNumber){ + ps.setLong(1, Long.parseLong(startLocation)); + }else { + ps.setString(1, startLocation); + } } resultSet = ps.executeQuery(); hasNext = resultSet.next(); @@ -802,6 +816,8 @@ private void queryStartLocation() throws SQLException{ //执行到此处代表轮询任务startLocation为空,且数据库中无数据,此时需要查询增量字段的最小值 ps.setFetchDirection(ResultSet.FETCH_FORWARD); resultSet.close(); + //如果事务不提交 就会导致数据库即使插入数据 也无法读到数据 + dbConn.commit(); resultSet = ps.executeQuery(); hasNext = resultSet.next(); //每隔五分钟打印一次,(当前时间 - 任务开始时间) % 300秒 <= 一个间隔轮询周期 @@ -832,10 +848,9 @@ private void queryStartLocation() throws SQLException{ /** * 获取数据库连接,用于子类覆盖 * @return connection - * @throws SQLException */ protected Connection getConnection() throws SQLException { - return DbUtil.getConnection(dbUrl, username, password); + return RetryUtil.executeWithRetry(() -> DriverManager.getConnection(dbUrl, username, password), 3, 2000,false); } /** @@ -870,4 +885,19 @@ protected void checkSize(int columnCount, List metaColumns) { throw new RuntimeException(message); } } + + /** + * 兼容db2 在间隔轮训场景 且第一次读取时没有任何数据 + * 在openInternal方法调用时 由于数据库没有数据,db2会自动关闭resultSet,因此只有在间隔轮训中某次读取到数据之后,进行更新columnCount + * @throws SQLException + */ + private void updateColumnCount() throws SQLException { + if(columnCount == 0){ + columnCount =resultSet.getMetaData().getColumnCount(); + boolean splitWithRowCol = numPartitions > 1 && StringUtils.isNotEmpty(splitKey) && splitKey.contains("("); + if (splitWithRowCol) { + columnCount = columnCount - 1; + } + } + } } \ No newline at end of file diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java index ae788328f2..b9e85760c2 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormatBuilder.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Properties; /** * The builder of JdbcInputFormat @@ -99,6 +100,10 @@ public void setCustomSql(String customSql){ format.customSql = customSql; } + public void setProperties(Properties properties){ + format.properties = properties; + } + public void setHadoopConfig(Map dirtyHadoopConfig) { format.hadoopConfig = dirtyHadoopConfig; } diff --git a/flinkx-rdb/flinkx-rdb-writer/pom.xml b/flinkx-rdb/flinkx-rdb-writer/pom.xml index d4c61d86f9..87cdb442d3 100644 --- a/flinkx-rdb/flinkx-rdb-writer/pom.xml +++ b/flinkx-rdb/flinkx-rdb-writer/pom.xml @@ -40,6 +40,7 @@ + diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcConfigKeys.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcConfigKeys.java index dd027a9b46..30e1afeef4 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcConfigKeys.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcConfigKeys.java @@ -35,4 +35,5 @@ public class JdbcConfigKeys { public static final String KEY_UPDATE_KEY = "updateKey"; public static final String KEY_FULL_COLUMN = "fullColumn"; public static final String KEY_INSERT_SQL_MODE = "insertSqlMode"; + public static final String KEY_PROPERTIES = "properties"; } diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java index a07eb3473c..0780f2c928 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com.dtstack.flinkx.rdb.datawriter/JdbcDataWriter.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; +import java.util.Properties; import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_BATCH_SIZE; import static com.dtstack.flinkx.rdb.datawriter.JdbcConfigKeys.KEY_FULL_COLUMN; @@ -63,6 +64,7 @@ public class JdbcDataWriter extends BaseDataWriter { protected Map> updateKey; protected List fullColumn; protected TypeConverterInterface typeConverter; + protected Properties properties; /** * just for postgresql,use copy replace insert @@ -100,6 +102,7 @@ public JdbcDataWriter(DataTransferConfig config) { fullColumn = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN); insertSqlMode = writerConfig.getParameter().getStringVal(KEY_INSERT_SQL_MODE); + properties = writerConfig.getParameter().getProperties(JdbcConfigKeys.KEY_PROPERTIES, null); } @Override @@ -127,6 +130,7 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setTypeConverter(typeConverter); builder.setRestoreConfig(restoreConfig); builder.setInsertSqlMode(insertSqlMode); + builder.setProperties(properties); return createOutput(dataSet, builder.finish()); } diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java index 47ab767ef3..8d94f93bc6 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java @@ -27,7 +27,8 @@ import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ClassUtil; import com.dtstack.flinkx.util.DateUtil; -import com.google.gson.Gson; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.ObjectUtils; import org.apache.commons.lang.StringUtils; @@ -46,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; /** * OutputFormat for writing data to relational database. @@ -102,6 +104,13 @@ public class JdbcOutputFormat extends BaseRichOutputFormat { protected long rowsOfCurrentTransaction; + public Properties properties; + + /** + * schema名 + */ + public String schema; + protected final static String GET_INDEX_SQL = "SELECT " + "t.INDEX_NAME," + "t.COLUMN_NAME " + @@ -143,17 +152,16 @@ protected void openInternal(int taskNumber, int numTasks){ ClassUtil.forName(driverName, getClass().getClassLoader()); dbConn = DbUtil.getConnection(dbUrl, username, password); - if (restoreConfig.isRestore()){ - dbConn.setAutoCommit(false); - } + //默认关闭事务自动提交,手动控制事务 + dbConn.setAutoCommit(false); if(CollectionUtils.isEmpty(fullColumn)) { - fullColumn = probeFullColumns(table, dbConn); + fullColumn = probeFullColumns(getTable(), dbConn); } if (!EWriteMode.INSERT.name().equalsIgnoreCase(mode)){ if(updateKey == null || updateKey.size() == 0) { - updateKey = probePrimaryKeys(table, dbConn); + updateKey = probePrimaryKeys(getTable(), dbConn); } } @@ -176,6 +184,8 @@ protected void openInternal(int taskNumber, int numTasks){ LOG.info("subTask[{}}] wait finished", taskNumber); } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); + }finally { + DbUtil.commit(dbConn); } } @@ -210,17 +220,13 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { int index = 0; try { for (; index < row.getArity(); index++) { - Object object = row.getField(index); - if( object instanceof String && StringUtils.isBlank((String) object)){ - if(!STRING_TYPES.contains(columnType.get(index))){ - object = null; - } - } - preparedStatement.setObject(index+1, object); + preparedStatement.setObject(index+1, getField(row, index)); } preparedStatement.execute(); + DbUtil.commit(dbConn); } catch (Exception e) { + DbUtil.rollBack(dbConn); processWriteException(e, index, row); } } @@ -233,7 +239,9 @@ protected void processWriteException(Exception e, int index, Row row) throws Wri } if(index < row.getArity()) { - throw new WriteRecordException(recordConvertDetailErrorMessage(index, row), e, index, row); + String message = recordConvertDetailErrorMessage(index, row); + LOG.error(message, e); + throw new WriteRecordException(message, e, index, row); } throw new WriteRecordException(e.getMessage(), e); } @@ -248,13 +256,7 @@ protected void writeMultipleRecordsInternal() throws Exception { try { for (Row row : rows) { for (int index = 0; index < row.getArity(); index++) { - Object object = row.getField(index); - if( object instanceof String && StringUtils.isBlank((String) object)){ - if(!STRING_TYPES.contains(columnType.get(index))){ - object = null; - } - } - preparedStatement.setObject(index+1, object); + preparedStatement.setObject(index+1, getField(row, index)); } preparedStatement.addBatch(); @@ -272,15 +274,22 @@ protected void writeMultipleRecordsInternal() throws Exception { if(restoreConfig.isRestore()){ rowsOfCurrentTransaction += rows.size(); + }else{ + //手动提交事务 + DbUtil.commit(dbConn); } + preparedStatement.clearBatch(); } catch (Exception e){ - if (restoreConfig.isRestore()){ - LOG.warn("writeMultipleRecordsInternal:Start rollback"); - dbConn.rollback(); - LOG.warn("writeMultipleRecordsInternal:Rollback success"); - } - + LOG.warn("write Multiple Records error, row size = {}, first row = {}, e = {}", + rows.size(), + rows.size() > 0 ? GsonUtil.GSON.toJson(rows.get(0)) : "null", + ExceptionUtil.getErrorMessage(e)); + LOG.warn("error to writeMultipleRecords, start to rollback connection, e = {}", ExceptionUtil.getErrorMessage(e)); + DbUtil.rollBack(dbConn); throw e; + }finally { + //执行完后清空batch + preparedStatement.clearBatch(); } } @@ -302,7 +311,9 @@ public FormatState getFormatState(){ }else{ preparedStatement.executeBatch(); } + //若事务提交失败,抛出异常 dbConn.commit(); + preparedStatement.clearBatch(); LOG.info("getFormatState:Commit connection success"); snapshotWriteCounter.add(rowsOfCurrentTransaction); @@ -320,7 +331,10 @@ public FormatState getFormatState(){ return null; } catch (Exception e){ try { + //执行完后清空batch + preparedStatement.clearBatch(); LOG.warn("getFormatState:Start rollback"); + //若事务回滚失败,抛出异常 dbConn.rollback(); LOG.warn("getFormatState:Rollback success"); } catch (SQLException sqlE){ @@ -331,9 +345,23 @@ public FormatState getFormatState(){ } } + /** + * 获取转换后的字段value + * @param row + * @param index + * @return + */ protected Object getField(Row row, int index) { Object field = row.getField(index); String type = columnType.get(index); + + //field为空字符串,且写入目标类型不为字符串类型的字段,则将object设置为null + if(field instanceof String + && StringUtils.isBlank((String) field) + &&!STRING_TYPES.contains(type)){ + return null; + } + if(type.matches(DateUtil.DATE_REGEX)) { field = DateUtil.columnToDate(field,null); } else if(type.matches(DateUtil.DATETIME_REGEX) || type.matches(DateUtil.TIMESTAMP_REGEX)){ @@ -403,8 +431,8 @@ protected boolean needWaitBeforeWriteRecords() { @Override protected void beforeWriteRecords() { + // preSql if(taskNumber == 0) { - LOG.info("start to execute preSql, preSql = {}", new Gson().toJson(preSql)); DbUtil.executeBatch(dbConn, preSql); } } @@ -416,11 +444,21 @@ protected boolean needWaitBeforeCloseInternal() { @Override protected void beforeCloseInternal() { - // 执行postsql + // 执行postSql if(taskNumber == 0) { - LOG.info("start to execute postSql, postSql = {}", new Gson().toJson(postSql)); DbUtil.executeBatch(dbConn, postSql); } } + /** + * 获取table名称,如果table是schema.table格式,可重写此方法 只返回table + * @return + */ + protected String getTable(){ + return table; + } + + public void setSchema(String schema){ + this.schema = schema; + } } diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormatBuilder.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormatBuilder.java index fe4cf0e302..8b969e7026 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormatBuilder.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormatBuilder.java @@ -17,12 +17,18 @@ */ package com.dtstack.flinkx.rdb.outputformat; +import com.dtstack.flinkx.enums.EWriteMode; import com.dtstack.flinkx.rdb.DatabaseInterface; import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.Properties; /** * @Company: www.dtstack.com @@ -68,6 +74,10 @@ public void setDatabaseInterface(DatabaseInterface databaseInterface) { format.databaseInterface = databaseInterface; } + public void setProperties(Properties properties){ + format.properties = properties; + } + public void setMode(String mode) { format.mode = mode; } @@ -92,6 +102,12 @@ public void setInsertSqlMode(String insertSqlMode){ format.insertSqlMode = insertSqlMode; } + + public void setSchema(String schema){ + format.setSchema(schema); + } + + @Override protected void checkFormat() { if (format.username == null) { diff --git a/flinkx-redis/flinkx-redis-writer/pom.xml b/flinkx-redis/flinkx-redis-writer/pom.xml index 44be362663..725c3a1df3 100644 --- a/flinkx-redis/flinkx-redis-writer/pom.xml +++ b/flinkx-redis/flinkx-redis-writer/pom.xml @@ -87,6 +87,7 @@ + diff --git a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java index 2fc85d1fd3..80a6c6e878 100644 --- a/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java +++ b/flinkx-redis/flinkx-redis-writer/src/main/java/com/dtstack/flinkx/redis/writer/RedisOutputFormat.java @@ -84,7 +84,9 @@ public void configure(Configuration parameters) { Properties properties = new Properties(); properties.put(KEY_HOST_PORT,hostPort); - properties.put(KEY_PASSWORD,password); + if(StringUtils.isNotBlank(password)){ + properties.put(KEY_PASSWORD,password); + } properties.put(KEY_TIMEOUT,timeout); properties.put(KEY_DB,database); diff --git a/flinkx-restapi/flinkx-restapi-core/pom.xml b/flinkx-restapi/flinkx-restapi-core/pom.xml index 742c77488d..359dc9293d 100644 --- a/flinkx-restapi/flinkx-restapi-core/pom.xml +++ b/flinkx-restapi/flinkx-restapi-core/pom.xml @@ -16,5 +16,11 @@ httpclient ${http.version} + + + com.github.pfmiles + dropincc.java + 0.2.2 + \ No newline at end of file diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java new file mode 100644 index 0000000000..787b6bf43b --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.commons.lang3.StringUtils; + +import java.sql.Date; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Objects; + +/** + * ConstantParan + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class ConstantParam implements ParamDefinition { + + private final String name; + + private final ParamType paramType; + + private final Class valueClass; + + protected Object value; + + private final String description; + + private String formatDescription; + + private final DateTimeFormatter format; + + public ConstantParam(String name, ParamType paramType, Class valueClass, Object value, String description, String format) { + this.name = name; + this.paramType = paramType; + this.valueClass = valueClass; + this.description = description; + this.value = value; + this.formatDescription = format; + if (StringUtils.isNotBlank(format)) { + this.format = DateTimeFormatter.ofPattern(format); + } else { + this.format = null; + } + + } + + @Override + public String getName() { + return name; + } + + @Override + public ParamType getType() { + return paramType; + } + + @Override + public Object getValue() { + return value; + } + + @Override + public String getValueType() { + return "valueClass"; + } + + + @Override + public String getDescription() { + return description; + } + + @Override + public String getFormat() { + return formatDescription; + } + + @Override + public Object format(Object data) { + if (Objects.isNull(format) || Objects.isNull(data)) { + return data; + } + if (getValueType().equals(Date.class)) { + Date value1 = (Date) data; + LocalDateTime ldt = value1.toInstant() + .atZone(ZoneId.systemDefault()) + .toLocalDateTime(); + return format.format(ldt); + } + return null; + } + + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java new file mode 100644 index 0000000000..6cafee83d4 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java @@ -0,0 +1,22 @@ +package com.dtstack.flinkx.restapi.common; + +public class ConstantVarible implements Paramitem { + private final T object; + private final String name; + + public ConstantVarible(T object, String name) { + this.object = object; + this.name = name; + } + + @Override + public T getValue(RestContext restContext) { + return object; + } + + @Override + public String getName() { + return name; + } + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java new file mode 100644 index 0000000000..36267d991d --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.Date; + +/** + * CurrentTimeVarible + * + * @author by dujie@dtstack.com + * @Date 2020/9/28 + */ +public class CurrentTimeVarible implements Paramitem { + + public CurrentTimeVarible() { + + } + + @Override + public Date getValue(RestContext restContext) { + return new Date(); + } + + @Override + public String getName() { + return "currenttime"; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java new file mode 100644 index 0000000000..49deeadc21 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import com.github.pfmiles.dropincc.*; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +/** + * DymaticParam + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class DymaticParam extends ConstantParam implements ParamDefinitionNextAble { + + private RestContext restContext; + + private final List dymaticNowString; + private final List dymaticNextString; + + + private final String initValueExpression; + private final String nextValueExpression; + + private Exe exe = getExe(); + + + public DymaticParam(String name, ParamType paramType, Class valueClass, String nowValue, String nextValue, String description, String format, RestContext context) { + super(name, paramType, valueClass, null, description, format); + // nowvalue一定是存在的 + initValueExpression = nowValue; + + dymaticNowString = ParamFactory.getVarible(nowValue); + + if (StringUtils.isBlank(nextValue)) { + dymaticNextString = dymaticNowString; + nextValueExpression = initValueExpression; + } else { + dymaticNextString = ParamFactory.getVarible(nextValue); + nextValueExpression = nextValue; + } + this.restContext = context; + } + + @Override + public Object getValue() { + return getDymaticValue(dymaticNowString, initValueExpression); + } + + /** + * 字符串解析为后缀表达式 + * `如何判断是动态变量 + * 如何判断是符合后缀表达式的 + * 后缀表达式计算 + * + * @return + */ + @Override + public Object getNextValue() { + return getDymaticValue(dymaticNextString, nextValueExpression); + } + + //判断是否是运算符 + public boolean isOperator(String oper) { + return oper.equals("+") || oper.equals("-") || oper.equals("*") || oper.equals("/"); + } + + //计算 如何变量1 2 只要有一个是字符串 那么就直接拼接 或者为null 为null 2+2+ + public int calculation(int num1, int num2, String oper) { + switch (oper) { + case "+": + return num2 + num1; + case "-": + return num2 - num1; + case "*": + return num2 * num1; + case "/": + return num2 / num1; + default: + return 0; + } + } + + public int operationLv(char operation) {//给运算符设置优先级 + switch (operation) { + case '+': + case '-': + return 1; + case '*': + case '/': + return 2; + case '(': + case ')': + return 3; + default: + return 0; + } + } + + + @Override + public void init() { + + } + + public Exe getExe() { + Lang c = new Lang("Calculator"); + Grule expr = c.newGrule(); + c.defineGrule(expr, CC.EOF).action(new Action() { + public BigDecimal act(Object matched) { + return new BigDecimal(((Object[]) matched)[0].toString()); + } + }); + TokenDef a = c.newToken("\\+"); + Grule addend = c.newGrule(); + expr.define(addend, CC.ks(a.or("\\-"), addend)).action(new Action() { + public BigDecimal act(Object matched) { + Object[] ms = (Object[]) matched; + BigDecimal a0 = (BigDecimal) ms[0]; + Object[] aPairs = (Object[]) ms[1]; + for (Object p : aPairs) { + String op = (String) ((Object[]) p)[0]; + BigDecimal a = (BigDecimal) ((Object[]) p)[1]; + if ("+".equals(op)) { + a0 = a.add(a0); + } else { + a0 = a0.subtract(a); + } + } + return a0; + } + }); + TokenDef m = c.newToken("\\*"); + Grule factor = c.newGrule(); + addend.define(factor, CC.ks(m.or("/"), factor)).action(new Action() { + public BigDecimal act(Object matched) { + Object[] ms = (Object[]) matched; + BigDecimal f0 = (BigDecimal) ms[0]; + Object[] fPairs = (Object[]) ms[1]; + for (Object p : fPairs) { + String op = (String) ((Object[]) p)[0]; + BigDecimal f = (BigDecimal) ((Object[]) p)[1]; + if ("*".equals(op)) { + f0 = f0.multiply(f); + } else { + f0 = f0.divide(f, BigDecimal.ROUND_HALF_UP, 3); + } + } + return f0; + } + }); + factor.define("\\(^[\\-]", expr, "\\)").action(new Action() { + public BigDecimal act(Object matched) { + return (BigDecimal) ((Object[]) matched)[1]; + } + }).alt("\\(\\-\\d+(\\.\\d+)?\\)|\\d+(\\.\\d+)?").action(new Action() { + public BigDecimal act(Object matched) { + if( matched.toString().startsWith("(") && matched.toString().endsWith(")")){ + return new BigDecimal(matched.toString().substring(1,matched.toString().length()-1)); + }else{ + return new BigDecimal(matched.toString()); + } + + } + }); + Exe exe = c.compile(); + return exe; + } + + private Object getDymaticValue(List dymaticString, String expression) { + if (CollectionUtils.isEmpty(dymaticString)) { + return expression; + } + Map tempReplaceValue = new HashMap<>(16); + AtomicReference tempExpression = new AtomicReference<>(expression); + Object tempValue; + dymaticString.forEach(k -> { + Object value = k.getValue(restContext); + tempReplaceValue.put(escapeExprSpecialWord("${" + k.getName() + "}"), value); + }); + tempReplaceValue.forEach((k, v) -> { + if(Objects.isNull(v)){ + tempExpression.set(tempExpression.get().replaceFirst(k, 0+"")); + }else if (NumberUtils.isNumber(v.toString())) { + tempExpression.set(tempExpression.get().replaceFirst(k, v.toString())); + } else if (v instanceof Date) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((Date) v).getTime() + "")); + } + if (v instanceof java.sql.Date) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((java.sql.Date) v).getTime() + "")); + } + if (v instanceof Timestamp) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((Timestamp) v).getTime() + "")); + } + }); + + + try { + tempValue = exe.eval(tempExpression.get()); + } catch (Exception e) { + tempReplaceValue.forEach((k, v) -> { + if(Objects.isNull(v)){ + tempExpression.set(tempExpression.get().replaceFirst(k, "")); + }else if (NumberUtils.isNumber(v.toString())) { + tempExpression.set(tempExpression.get().replaceFirst(k, v.toString())); + } else if (v instanceof Date) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((Date) v).toString() + "")); + } + if (v instanceof java.sql.Date) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((java.sql.Date) v).toString() + "")); + } + if (v instanceof Timestamp) { + tempExpression.set(tempExpression.get().replaceFirst(k, ((Timestamp) v).toString() + "")); + } + }); + tempValue = initValueExpression; + } + restContext.updateValue(getType().name().toLowerCase(Locale.ENGLISH) + "." + getName(), tempValue); + return tempValue; + } + /** + * 转义正则特殊字符 ($()*+.[]?\^{},|) + * + * @param keyword + * @return + */ + public static String escapeExprSpecialWord(String keyword) { + if (StringUtils.isNotBlank(keyword)) { + String[] fbsArr = { "\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|" }; + for (String key : fbsArr) { + if (keyword.contains(key)) { + keyword = keyword.replace(key, "\\" + key); + } + } + } + return keyword; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java index c975283507..8b7b9b1b23 100644 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java @@ -92,7 +92,7 @@ public static HttpRequestBase getRequest(String method, post.setEntity(getEntityData(requestBody)); request = post; } else { - throw new RuntimeException("Unsupported method:" + method); + throw new UnsupportedOperationException("Unsupported method:" + method); } for (Map.Entry entry : header.entrySet()) { diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java new file mode 100644 index 0000000000..62102263ea --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.HashMap; +import java.util.Map; + +/** + * InnerVaribleFactory + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class InnerVaribleFactory { + + private static Map InnerVaribles; + + static { + InnerVaribles = new HashMap<>(18); + InnerVaribles.put("uuid", new UuidVarible()); + InnerVaribles.put("currenttime", new CurrentTimeVarible()); + + } + + public static Paramitem createInnerVarible(String name) { + return InnerVaribles.get(name); + } + + public static boolean isInnerVariable(String name) { + return InnerVaribles.containsKey(name); + } + + + public static void addVarible(String name,Paramitem paramitem) { + InnerVaribles.put(name,paramitem); + } + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java new file mode 100644 index 0000000000..8adc73ab6f --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * CurrentTimeVarible + * + * @author by dujie@dtstack.com + * @Date 2020/9/28 + */ +public class IntervalTimeVarible implements Paramitem { + + private final Long intervalTime; + + public IntervalTimeVarible(Long intervalTime) { + this.intervalTime = intervalTime; + } + + @Override + public Long getValue(RestContext restContext) { + return intervalTime; + } + + @Override + public String getName() { + return "intervalTime"; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java new file mode 100644 index 0000000000..c1bdfc5b13 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java @@ -0,0 +1,37 @@ +package com.dtstack.flinkx.restapi.common; + +import com.dtstack.flinkx.util.GsonUtil; + +import java.util.Map; +import java.util.Objects; + +public class MapUtils { + + + public static Object getData(Map data, String[] names) { + Map tempHashMap = data; + for (int i = 0; i < names.length; i++) { + if (tempHashMap.containsKey(names[i]) && i != names.length - 1) { + if (Objects.isNull(tempHashMap.get(names[i]))) { + return null; + } + if (tempHashMap.get(names[i]) instanceof Map) { + tempHashMap = (Map) tempHashMap.get(names[i]); + } else if (tempHashMap.get(names[i]) instanceof String) { + try { + tempHashMap = GsonUtil.GSON.fromJson((String) tempHashMap.get(names[i]), GsonUtil.gsonMapTypeToken); + } catch (Exception e) { + return null; + } + } else { + return null; + } + } else if (i == names.length - 1) { + return tempHashMap.get(names[i]); + } else { + return null; + } + } + return null; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java new file mode 100644 index 0000000000..0439ca70db --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * ParamDefinition + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public interface ParamDefinition { + + /** + * key + * + * @return + */ + String getName(); + + ParamType getType(); + + Object getValue(); + + String getValueType(); + + /** + * 用户定义的json + * + * @return + */ + String getDescription(); + + String getFormat(); + + Object format(Object data); + + + default void init(){ + + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java new file mode 100644 index 0000000000..b51b828176 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.List; + +/** + * DymaticParam + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public interface ParamDefinitionNextAble extends ParamDefinition { + + Object getNextValue(); +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java new file mode 100644 index 0000000000..d7f8f902a5 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * ParamFactory + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class ParamFactory { + + public static Pattern valueExpression = + Pattern.compile("(?(\\$\\{(.*?)\\}))"); + + public static List createDefinition(ParamType paramType, Map> variableDescriptions, RestContext context) { + + //2 如果是动态变量 还需要解析为后缀表达式 + //先用一个正则 将动态变量拿出来 参数拿出来 + if (variableDescriptions == null || variableDescriptions.isEmpty()) { + return Collections.emptyList(); + } + List data = new ArrayList<>(variableDescriptions.size()); + + Iterator>> iterator = variableDescriptions.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + String name = entry.getKey(); + Map entryValue = entry.getValue(); + String value = entryValue.get("value"); + String next = entryValue.get("next"); + String format = entryValue.get("format"); + String type = entryValue.get("type"); + + + //1 含有next 2 value里含有动态函数 isDymatic(); + //valueItems + //nextItems 如果不是空 或者 直接返回动态的 + if (isDymatic(value) || StringUtils.isNotBlank(next)) { + DymaticParam dymaticParam = new DymaticParam(name, paramType, null, value, next, variableDescriptions.toString(), format, context); + data.add(dymaticParam); + } else { + data.add(new ConstantParam(name, paramType, null, value, variableDescriptions.toString(), format)); + } + } + return data; + } + + + public static List getVarible(String text) { + List valueItems = new ArrayList<>(16); + Matcher matcher = valueExpression.matcher(text); + while (matcher.find()) { + String varible = matcher.group("varible"); + valueItems.add(parsr(varible)); + } + if (CollectionUtils.isEmpty(valueItems)) { + return Collections.emptyList(); + } + return valueItems; + } + + public static boolean isDymatic(String text) { + return valueExpression.matcher(text).find(); + } + + + public static Paramitem parsr(String value) { + + if (value.startsWith("${") && value.endsWith("}")) { + String substring = value.substring(2, value.length() - 1); + if (substring.startsWith("body.") || substring.startsWith("param.") || substring.startsWith("header.") || substring.startsWith("response.")) { + return new ReplaceParamItem(substring); + } else if (InnerVaribleFactory.isInnerVariable(substring)) { + return InnerVaribleFactory.createInnerVarible(substring); + } else { + return new ConstantVarible(value, value); + } + } else { + return new ConstantVarible(value, value); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java new file mode 100644 index 0000000000..8f0d25c26b --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * ParamType + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public enum ParamType { + + HEADER, + BODY, + PARAM, + RESPONSE; +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java new file mode 100644 index 0000000000..7c2b4b7a81 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +/** + * Paramitem + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public interface Paramitem { + T getValue(RestContext restContext); + + String getName(); + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java new file mode 100644 index 0000000000..1d5062a758 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.Collections; +import java.util.Locale; +import java.util.Optional; + +/** + * ReplaceParamItem + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class ReplaceParamItem implements Paramitem { + private final String name; + + public ReplaceParamItem(String name) { + this.name=name; + } + + @Override + public Object getValue(RestContext restContext) { + return restContext.getValue(name); + } + + @Override + public String getName() { + return name; + } + + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java new file mode 100644 index 0000000000..b1b0373c4e --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * RestContext + * + * @author by dujie@dtstack.com + * @Date 2020/9/26 + */ +public class RestContext { + + private String requestType; + + private String url; + + private String format; + + private boolean first; + + private ThreadLocal> setThreadLocal = new ThreadLocal<>(); + + private Map prevRequestValue = new HashMap<>(16); + + private Map requestValue; + + private Map> paramDefinitions; + + private Object object = new Object(); + + public RestContext(String requestType,String url,String format) { + this.requestType =requestType; + this.url =url; + this.requestValue = new HashMap<>(32); + this.paramDefinitions = new HashMap<>(32); + this.first = true; + this.format=format; + } + + public void updateValue(){ + prevRequestValue = requestValue; + requestValue=new HashMap<>(requestValue.size()); + } + + public Map getPreValue() { + return requestValue; + } + + + public httprequestApi.Httprequest build() { + // 根据当前value计算出httprequest + httprequestApi.Httprequest httprequest = new httprequestApi.Httprequest(); + + Map body = new HashMap<>(16); + Map header = new HashMap<>(16); + Map param = new HashMap<>(16); + if (first) { + paramDefinitions.get(ParamType.BODY).forEach((k, v) -> { + body.put(k, v.getValue()); + }); + + paramDefinitions.get(ParamType.HEADER).forEach((k, v) -> { + header.put(k, v.getValue().toString()); + }); + + paramDefinitions.get(ParamType.PARAM).forEach((k, v) -> { + param.put(k, v.getValue()); + }); + first = false; + } else { + paramDefinitions.get(ParamType.BODY).forEach((k, v) -> { + if (v instanceof ParamDefinitionNextAble) { + body.put(k, ((ParamDefinitionNextAble) v).getNextValue()); + } else { + body.put(k, v.getValue()); + } + }); + + paramDefinitions.get(ParamType.HEADER).forEach((k, v) -> { + if (v instanceof ParamDefinitionNextAble) { + header.put(k, ((ParamDefinitionNextAble) v).getNextValue().toString()); + } else { + header.put(k, v.getValue().toString()); + } + }); + + paramDefinitions.get(ParamType.PARAM).forEach((k, v) -> { + if (v instanceof ParamDefinitionNextAble) { + param.put(k, ((ParamDefinitionNextAble) v).getNextValue()); + } else { + param.put(k, v.getValue()); + } + }); + } + httprequest.buildBody(body).buildHeader(header).buildParam(param); + return httprequest; + } + + public httprequestApi.Httprequest buildNext() { + // 计算下一次的请求 但是不会更新preValue值 + return null; + } + + public void parseAndInt(Map> job, ParamType paramType) { + paramDefinitions.put(paramType, ParamFactory.createDefinition(paramType, job, this).stream().collect(Collectors.toMap(ParamDefinition::getName, Function.identity()))); + } + + + + protected void updateValue(String key, Object value) { + requestValue.put(key, value); + } + + private Object getValueQuick(String key) { + return prevRequestValue.get(key); + } + + protected Object getValue(String key) { + if (setThreadLocal.get() == null) { + setThreadLocal.set(new HashSet<>(paramDefinitions.size() * 8)); + } + if (setThreadLocal.get().contains(key)) { + throw new RuntimeException("循环依赖"); + } + setThreadLocal.get().add(key); + Object data = getValueQuick(key); + if (Objects.isNull(data) && first) { + String[] s = key.split("\\."); + ParamType paramType = ParamType.valueOf(s[0].toUpperCase(Locale.ENGLISH)); + if (Objects.isNull(paramDefinitions.get(paramType))) { + data = null; + } else { + ParamDefinition definition = paramDefinitions.get(paramType).get(s[1]); + //没查到对应的动态变量 + if (Objects.isNull(definition)) { + prevRequestValue.put(key, object); + } else { + Object value = definition.getValue(); + data=value; + prevRequestValue.put(key,value ); + } + } + } + setThreadLocal.get().remove(key); + + if (data == object) { + data = null; + } + return data; + } + //循环依赖使用有向图判断 + + public Map> getParamDefinitions() { + return paramDefinitions; + } + + public String getRequestType() { + return requestType; + } + + public String getUrl() { + return url; + } + + public String getFormat() { + return format; + } + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java new file mode 100644 index 0000000000..3fd1be6283 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java @@ -0,0 +1,21 @@ +package com.dtstack.flinkx.restapi.common; + +import java.util.UUID; + +public class UuidVarible implements Paramitem { + + + public UuidVarible() { + } + + @Override + public String getValue(RestContext restContext) { + return UUID.randomUUID().toString(); + } + + @Override + public String getName() { + return "currentTime"; + } + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java new file mode 100644 index 0000000000..cbb9c9cc4e --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java @@ -0,0 +1,10 @@ +package com.dtstack.flinkx.restapi.common.exception; + +public class ReadRecordException extends RuntimeException { + public ReadRecordException() { + } + + public ReadRecordException(String message) { + super(message); + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java new file mode 100644 index 0000000000..bd780a05b9 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java @@ -0,0 +1,22 @@ +package com.dtstack.flinkx.restapi.common.exception; + +public class ResponseBreakException extends RuntimeException{ + public ResponseBreakException() { + } + + public ResponseBreakException(String message) { + super(message); + } + + public ResponseBreakException(String message, Throwable cause) { + super(message, cause); + } + + public ResponseBreakException(Throwable cause) { + super(cause); + } + + public ResponseBreakException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java new file mode 100644 index 0000000000..19ca560944 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java @@ -0,0 +1,22 @@ +package com.dtstack.flinkx.restapi.common.exception; + +public class ResponseRetryException extends RuntimeException { + public ResponseRetryException() { + } + + public ResponseRetryException(String message) { + super(message); + } + + public ResponseRetryException(String message, Throwable cause) { + super(message, cause); + } + + public ResponseRetryException(Throwable cause) { + super(cause); + } + + public ResponseRetryException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java new file mode 100644 index 0000000000..1b61b82756 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java @@ -0,0 +1,26 @@ +package com.dtstack.flinkx.restapi.common.handler; + +import com.dtstack.flinkx.restapi.common.MapUtils; +import com.dtstack.flinkx.restapi.common.exception.ResponseBreakException; +import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; + +import java.util.Map; +import java.util.Set; + +public class BreakDataHandler extends DataHandler { + + public BreakDataHandler(String key, Set value) { + super(key, value); + } + + @Override + public void execute(Map responseData) { + String[] strings = new String[0]; + strings[0] = key; + Object data = MapUtils.getData(responseData, strings); + if (value.contains(data.toString())) { + throw new ResponseBreakException("key:"+key+" contains"+data.toString()+" ,need break"); + } + + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java new file mode 100644 index 0000000000..fe0a4cf643 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java @@ -0,0 +1,23 @@ +package com.dtstack.flinkx.restapi.common.handler; + +import java.util.Map; +import java.util.Set; + +public abstract class DataHandler { + + protected String key; + + protected Set value; + + public DataHandler(String key, Set value) { + this.key = key; + this.value = value; + } + + public boolean isPipei(Map responseData) { + return responseData.containsKey(key) && value.contains(responseData.get(key).toString()); + } + + public abstract void execute(Map responseData); + +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java new file mode 100644 index 0000000000..94b94b9ca1 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java @@ -0,0 +1,29 @@ +package com.dtstack.flinkx.restapi.common.handler; + +import org.apache.commons.lang.StringUtils; + +import java.util.HashMap; +import java.util.Map; + +public class DataHandlerFactory { + + + private static HashMap handlerHashMap = new HashMap<>(16); + + static { + + } + + public static DataHandler getDataHandler(Map dataHandlerParam) { + if (StringUtils.isNotBlank(dataHandlerParam.get("key").toString())) { + return handlerHashMap.get(dataHandlerParam.get("key").toString()); + }else{ + throw new IllegalArgumentException("dataHandler key not allow blank"); + } + } + + public static void destroy() { + //gc + handlerHashMap = null; + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java new file mode 100644 index 0000000000..2ae704a97c --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java @@ -0,0 +1,24 @@ +package com.dtstack.flinkx.restapi.common.handler; + +import com.dtstack.flinkx.restapi.common.MapUtils; +import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; + +import java.util.Map; +import java.util.Set; + +public class RetryDataHandler extends DataHandler { + + public RetryDataHandler(String key, Set value) { + super(key, value); + } + + @Override + public void execute(Map responseData) { + String[] strings = new String[0]; + strings[0] = key; + Object data = MapUtils.getData(responseData, strings); + if (value.contains(data.toString())) { + throw new ResponseRetryException("key:"+key+" contains"+data.toString()+" ,need retry"); + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java new file mode 100644 index 0000000000..69e203c660 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.common; + +import java.util.Map; + +/** + * httprequestApi + * + * @author by dujie@dtstack.com + * @Date 2020/9/25 + */ +public abstract class httprequestApi { + + abstract R execute(); + + + abstract void getHttpRequest(); + + public static class Httprequest { + private Map body; + private Map param; + private Map header; + + public Httprequest buildBody(Map body) { + this.body = body; + return this; + } + + public Httprequest buildParam(Map param) { + this.param = param; + return this; + } + + public Httprequest buildHeader(Map header) { + this.header = header; + return this; + } + + @Override + public String toString() { + return "Httprequest{" + + "body:" + body + + ", param:" + param + + ", header:" + header + + '}'; + } + + public Map getBody() { + return body; + } + + public Map getParam() { + return param; + } + + public Map getHeader() { + return header; + } + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/pom.xml b/flinkx-restapi/flinkx-restapi-reader/pom.xml index f4d8801a9d..151b903e3f 100644 --- a/flinkx-restapi/flinkx-restapi-reader/pom.xml +++ b/flinkx-restapi/flinkx-restapi-reader/pom.xml @@ -87,6 +87,7 @@ + diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java new file mode 100644 index 0000000000..a3cc581dd5 --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.inputformat; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restapi.common.HttpUtil; +import com.dtstack.flinkx.restapi.common.MapUtils; +import com.dtstack.flinkx.restapi.common.RestContext; +import com.dtstack.flinkx.restapi.common.exception.ReadRecordException; +import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; +import com.dtstack.flinkx.restapi.common.handler.DataHandler; +import com.dtstack.flinkx.restapi.common.httprequestApi; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.types.Row; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * httpClient + * + * @author by dujie@dtstack.com + * @Date 2020/9/25 + */ +public class HttpClient { + private static final Logger LOG = LoggerFactory.getLogger(HttpClient.class); + + private ScheduledExecutorService scheduledExecutorService; + protected transient CloseableHttpClient httpClient; + private final long intervalTime; + private BlockingQueue queue; + private RestContext restContext; + private static final String THREAD_NAME = "restApiReader-thread"; + private List metaColumns; + private List handlers; + + + public HttpClient(RestContext restContext, Long intervalTime) { + this.restContext = restContext; + this.intervalTime = intervalTime; + queue = new SynchronousQueue<>(false); + this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1, new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, THREAD_NAME); + } + }); + this.httpClient = HttpUtil.getHttpClient(); + } + + public void start() { + + scheduledExecutorService.scheduleAtFixedRate( + this::execute, + 0, + intervalTime, + TimeUnit.MILLISECONDS + ); + } + + public Row takeEvent() { + Row row = null; + try { + row = queue.take(); + } catch (InterruptedException e) { + LOG.error("takeEvent interrupted error:{}", ExceptionUtil.getErrorMessage(e)); + } + return row; + } + + public void execute() { + httprequestApi.Httprequest build = restContext.build(); + doExecute(build, 2); + restContext.updateValue(); + } + + public void doExecute(httprequestApi.Httprequest build, int retryTime) { + + HttpUriRequest request = HttpUtil.getRequest(restContext.getRequestType(), build.getBody(), build.getHeader(), restContext.getUrl()); + try { + CloseableHttpResponse httpResponse = httpClient.execute(request); + HttpEntity entity = httpResponse.getEntity(); + if (entity != null) { + String entityData = EntityUtils.toString(entity); + if (restContext.getFormat().equals("json")) { + Map map = HttpUtil.gson.fromJson(entityData, Map.class); + + if(CollectionUtils.isNotEmpty(handlers)){ + for (DataHandler handler : handlers) { + if (handler.isPipei(map)) { + handler.execute(map); + } + } + } + + if (CollectionUtils.isEmpty(metaColumns) || (metaColumns.size() == 1 && metaColumns.get(0).getName().equals(ConstantValue.STAR_SYMBOL))) { + queue.put(Row.of(map)); + } else { + HashMap stringObjectHashMap = new HashMap<>(); + for (MetaColumn metaColumn : metaColumns) { + String[] names = metaColumn.getName().split("\\."); + Map keyToMap = initData(stringObjectHashMap, names); + if (Objects.nonNull(keyToMap)) { + Object data = MapUtils.getData(map, names); + keyToMap.put(names[names.length - 1], data); + } + } + queue.put(Row.of(stringObjectHashMap)); + } + } else { + queue.put(Row.of(entityData)); + } + } else { + throw new RuntimeException("entity is null"); + } + } catch (ResponseRetryException e) { + //todo 重试 + if (--retryTime > 0) { + doExecute(build, retryTime); + } + } catch (Exception e) { + //todo 脏数据处理 + throw new ReadRecordException("get entity error"); + } + + } + + public void close() { + HttpUtil.closeClient(httpClient); + scheduledExecutorService.shutdown(); + } + + public Map initData(HashMap data, String[] names) { + Map tempHashMap = data; + for (int i = 0; i < names.length; i++) { + if (i != names.length - 1) { + HashMap objectObjectHashMap = new HashMap(4); + Object value = tempHashMap.putIfAbsent(names[i], objectObjectHashMap); + if(Objects.isNull(value)){ + tempHashMap = objectObjectHashMap; + }else if (value instanceof String) { + try { + Map o = GsonUtil.GSON.fromJson((String) value, GsonUtil.gsonMapTypeToken); + tempHashMap.put(names[i], o); + tempHashMap = o; + } catch (Exception e) { + return null; + } + } else if (value instanceof Map) { + tempHashMap = (Map) value; + } else { + return null; + } + } else { + tempHashMap.put(names[i], null); + } + } + return tempHashMap; + } + + public void setMetaColumns(List metaColumns) { + this.metaColumns = metaColumns; + } + + public void setHandlers(List handlers) { + this.handlers = handlers; + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java index 59c6b535de..d093e35af8 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java @@ -18,18 +18,18 @@ package com.dtstack.flinkx.restapi.inputformat; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.restapi.common.HttpUtil; -import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restapi.common.ParamType; +import com.dtstack.flinkx.restapi.common.RestContext; +import com.dtstack.flinkx.restapi.common.handler.DataHandler; +import com.dtstack.flinkx.restapi.common.handler.DataHandlerFactory; +import com.dtstack.flinkx.restapi.reader.HttpRestConfig; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; -import org.apache.http.HttpEntity; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.util.EntityUtils; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -38,47 +38,37 @@ */ public class RestapiInputFormat extends BaseRichInputFormat { - protected String url; + protected HttpClient myHttpClient; - protected String method; + protected RestContext restContext; - protected transient CloseableHttpClient httpClient; + protected Long intervalTime; - protected Map header; + protected HttpRestConfig httpRestConfig; - protected Map entityDataToMap; + protected List metaColumns ; - protected boolean getData; + protected List handlers; @Override public void openInputFormat() throws IOException { super.openInputFormat(); - httpClient = HttpUtil.getHttpClient(); } - @Override - public void closeInputFormat() { - HttpUtil.closeClient(httpClient); - } @Override @SuppressWarnings("unchecked") protected void openInternal(InputSplit inputSplit) throws IOException { - HttpUriRequest request = HttpUtil.getRequest(method, header,null, url); - try { - CloseableHttpResponse httpResponse = httpClient.execute(request); - HttpEntity entity = httpResponse.getEntity(); - if (entity != null) { - String entityData = EntityUtils.toString(entity); - entityDataToMap = GsonUtil.GSON.fromJson(entityData, Map.class); - getData = true; - } else { - throw new RuntimeException("entity is null"); - } - } catch (Exception e) { - throw new RuntimeException("get entity error"); - } + restContext = new RestContext(httpRestConfig.getType(),httpRestConfig.getUrl(),httpRestConfig.getFormat()); + myHttpClient = new HttpClient(restContext, intervalTime); + myHttpClient.setMetaColumns(metaColumns); + myHttpClient.setHandlers(handlers); + restContext.parseAndInt(httpRestConfig.getBody(), ParamType.BODY); + restContext.parseAndInt(httpRestConfig.getHeader(), ParamType.HEADER); + restContext.parseAndInt(httpRestConfig.getParam(), ParamType.PARAM); + + myHttpClient.start(); } @Override @@ -92,18 +82,37 @@ protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Except @Override protected Row nextRecordInternal(Row row) throws IOException { - row = new Row(1); - row.setField(0, entityDataToMap); - getData = false; - return row; + return myHttpClient.takeEvent(); + } @Override protected void closeInternal() throws IOException { + myHttpClient.close(); } @Override public boolean reachedEnd() throws IOException { - return !getData; + return false; + } + + public void setHttpRestConfig(HttpRestConfig httpRestConfig) { + this.httpRestConfig = httpRestConfig; + } + + public void setIntervalTime(Long intervalTime) { + this.intervalTime = intervalTime; + } + + public void setMetaColumns(List metaColumns) { + this.metaColumns = metaColumns; + } + + public List getHandlers() { + return handlers; + } + + public void setHandlers(List handlers) { + this.handlers = handlers; } } diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java index ac636b4038..48c6198715 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java @@ -18,7 +18,12 @@ package com.dtstack.flinkx.restapi.inputformat; import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restapi.common.handler.DataHandler; +import com.dtstack.flinkx.restapi.reader.HttpRestConfig; +import org.apache.commons.lang3.StringUtils; +import java.util.List; import java.util.Map; /** @@ -28,18 +33,33 @@ public class RestapiInputFormatBuilder extends BaseRichInputFormatBuilder { protected RestapiInputFormat format; - public RestapiInputFormatBuilder(){ super.format = format = new RestapiInputFormat();} - public void setUrl(String url){this.format.url = url;} - public void setHeader(Map header){ this.format.header = header;} - public void setMethod(String method){ this.format.method = method;} + public RestapiInputFormatBuilder() { + super.format = format = new RestapiInputFormat(); + } + + public void setHttpRestConfig(HttpRestConfig httpRestConfig) { + this.format.setHttpRestConfig(httpRestConfig); + } + + public void setIntervalTime(long intervalTime) { + this.format.setIntervalTime(intervalTime); + } + + public void setMetaColumns(List metaColumns) { + this.format.setMetaColumns(metaColumns); + } + + public void setHandlers(List handlers) { + this.format.setHandlers(handlers); + } @Override protected void checkFormat() { - if(format.url.isEmpty()){ + if(StringUtils.isBlank(format.httpRestConfig.getUrl())){ throw new IllegalArgumentException("缺少url"); } - if (format.method.isEmpty()) { + if (StringUtils.isBlank(format.httpRestConfig.getType())) { throw new IllegalArgumentException("缺少method"); } } diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java new file mode 100644 index 0000000000..a7f647e77e --- /dev/null +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.restapi.reader; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +/** + * HttpRestConfig + * + * @author by dujie@dtstack.com + * @Date 2020/9/28 + */ +public class HttpRestConfig implements Serializable { + private String url; + private String type; + private Long intervalTime; + private List columns; + private String format; + private Map> header; + private Map> body; + private Map> param; + protected List handlers; + + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public List getColumns() { + return columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + + public Map> getHeader() { + return header; + } + + public void setHeader(Map> header) { + this.header = header; + } + + public Map> getBody() { + return body; + } + + public void setBody(Map> body) { + this.body = body; + } + + public Map> getParam() { + return param; + } + + public void setParam(Map> param) { + this.param = param; + } + + public Long getIntervalTime() { + return intervalTime; + } + + public void setIntervalTime(Long intervalTime) { + this.intervalTime = intervalTime; + } + + public String getFormat() { + return format; + } + + public void setFormat(String format) { + this.format = format; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public List getHandlers() { + return handlers; + } + + public void setHandlers(List handlers) { + this.handlers = handlers; + } +} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java index d5bc81c8de..be5b719ef9 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java @@ -20,13 +20,18 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.reader.BaseDataReader; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restapi.common.InnerVaribleFactory; +import com.dtstack.flinkx.restapi.common.IntervalTimeVarible; +import com.dtstack.flinkx.restapi.common.handler.DataHandlerFactory; import com.dtstack.flinkx.restapi.inputformat.RestapiInputFormatBuilder; -import com.google.common.collect.Maps; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; import java.util.ArrayList; +import java.util.List; import java.util.Map; /** @@ -35,36 +40,52 @@ */ public class RestapiReader extends BaseDataReader { - private String url; + private HttpRestConfig httpRestConfig; - private String method; + private List metaColumns; - private Map header = Maps.newHashMap(); + protected Long intervalTime; - private ArrayList> temp; + protected List handlers; @SuppressWarnings("unchecked") public RestapiReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); - url = readerConfig.getParameter().getStringVal("url"); - method = readerConfig.getParameter().getStringVal("method"); - temp = (ArrayList>) readerConfig.getParameter().getVal("header"); - if (temp != null) { - for (Map map : temp) { - header.putAll(map); + try { + this.httpRestConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), HttpRestConfig.class); + } catch (Exception e) { + throw new RuntimeException("解析httpRest Config配置出错:", e); + } + metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); + List handlers = httpRestConfig.getHandlers(); + if (CollectionUtils.isNotEmpty(handlers)) { + handlers = new ArrayList(handlers.size() * 2); + for (Object handlerParam : handlers) { + try { + handlers.add(DataHandlerFactory.getDataHandler((Map) handlerParam)); + } catch (Exception e) { + throw new IllegalArgumentException("dataHandler param error" + httpRestConfig.getHandlers()); + } } + DataHandlerFactory.destroy(); } + + InnerVaribleFactory.addVarible("intervalTime", new IntervalTimeVarible(httpRestConfig.getIntervalTime())); + intervalTime = httpRestConfig.getIntervalTime(); + } @Override public DataStream readData() { RestapiInputFormatBuilder builder = new RestapiInputFormatBuilder(); builder.setDataTransferConfig(dataTransferConfig); - builder.setHeader(header); - builder.setMethod(method); - builder.setUrl(url); + builder.setIntervalTime(intervalTime); + builder.setMetaColumns(metaColumns); + builder.setHandlers(handlers); + builder.setHttpRestConfig(httpRestConfig); + return createInput(builder.finish()); } diff --git a/flinkx-restapi/flinkx-restapi-writer/pom.xml b/flinkx-restapi/flinkx-restapi-writer/pom.xml index 4ffe9cd76e..723c37691b 100644 --- a/flinkx-restapi/flinkx-restapi-writer/pom.xml +++ b/flinkx-restapi/flinkx-restapi-writer/pom.xml @@ -87,6 +87,7 @@ + diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java index 95665df334..543727261f 100644 --- a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/outputformat/RestapiOutputFormat.java @@ -20,10 +20,6 @@ import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.restapi.common.HttpUtil; -import com.dtstack.flinkx.util.GsonUtil; -import com.google.common.collect.Maps; -import org.apache.flink.types.Row; -import org.apache.http.HttpStatus; import com.dtstack.flinkx.util.ExceptionUtil; import com.google.common.collect.Maps; import com.google.gson.Gson; @@ -63,7 +59,7 @@ public class RestapiOutputFormat extends BaseRichOutputFormat { protected Map header; - protected static final int DEFAULT_TIME_OUT = 1800000; + protected static final int DEFAULT_TIME_OUT = 300000; protected Gson gson; @@ -79,9 +75,9 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { CloseableHttpClient httpClient = HttpUtil.getHttpClient(); int index = 0; Map requestBody = Maps.newHashMap(); - Object dataRow; + List dataRow = new ArrayList<>(); try { - dataRow = getDataFromRow(row, column); + dataRow.add(getDataFromRow(row, column)); params.put(KEY_BATCH, UUID.randomUUID().toString().substring(0, 8)); if (!params.isEmpty()) { Iterator iterator = params.entrySet().iterator(); @@ -160,8 +156,12 @@ private void sendRequest(CloseableHttpClient httpClient, Map header, String url) throws IOException { LOG.debug("send data:{}", gson.toJson(requestBody)); - LOG.debug("当前发送的数据为:{}", GsonUtil.GSON.toJson(requestBody)); HttpRequestBase request = HttpUtil.getRequest(method, requestBody, header, url); + //设置请求和传输超时时间 + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(DEFAULT_TIME_OUT).setConnectionRequestTimeout(DEFAULT_TIME_OUT) + .setSocketTimeout(DEFAULT_TIME_OUT).build(); + request.setConfig(requestConfig); CloseableHttpResponse httpResponse = httpClient.execute(request); // 重试之后返回状态码不为200 if (httpResponse.getStatusLine().getStatusCode() != HttpStatus.SC_OK) { diff --git a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java index f050abb1b9..b4c589fb5d 100644 --- a/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java +++ b/flinkx-restapi/flinkx-restapi-writer/src/main/java/com/dtstack/flinkx/restapi/writer/RestapiWriter.java @@ -97,6 +97,9 @@ public DataStreamSink writeData(DataStream dataSet) { builder.setColumn(column); builder.setParams(params); builder.setBatchInterval(batchInterval); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); return createOutput(dataSet, builder.finish()); } diff --git a/flinkx-saphana/flinkx-saphana-reader/pom.xml b/flinkx-saphana/flinkx-saphana-reader/pom.xml index c095c9df02..7c68672a0a 100644 --- a/flinkx-saphana/flinkx-saphana-reader/pom.xml +++ b/flinkx-saphana/flinkx-saphana-reader/pom.xml @@ -89,6 +89,7 @@ + diff --git a/flinkx-saphana/flinkx-saphana-writer/pom.xml b/flinkx-saphana/flinkx-saphana-writer/pom.xml index 3a733bae57..db4debcc3e 100644 --- a/flinkx-saphana/flinkx-saphana-writer/pom.xml +++ b/flinkx-saphana/flinkx-saphana-writer/pom.xml @@ -89,6 +89,7 @@ + diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml index d0bd44a29d..a826c51267 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/pom.xml @@ -93,6 +93,7 @@ + diff --git a/flinkx-sqlserver/flinkx-sqlserver-reader/src/main/java/com/dtstack/flinkx/sqlserver/reader/SqlserverQuerySqlBuilder.java b/flinkx-sqlserver/flinkx-sqlserver-reader/src/main/java/com/dtstack/flinkx/sqlserver/reader/SqlserverQuerySqlBuilder.java index 50818fbe74..84161e15be 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-reader/src/main/java/com/dtstack/flinkx/sqlserver/reader/SqlserverQuerySqlBuilder.java +++ b/flinkx-sqlserver/flinkx-sqlserver-reader/src/main/java/com/dtstack/flinkx/sqlserver/reader/SqlserverQuerySqlBuilder.java @@ -19,6 +19,7 @@ import com.dtstack.flinkx.rdb.datareader.JdbcDataReader; import com.dtstack.flinkx.rdb.datareader.QuerySqlBuilder; +import com.dtstack.flinkx.rdb.util.DbUtil; import com.dtstack.flinkx.sqlserver.SqlServerConstants; import org.apache.commons.lang3.StringUtils; @@ -43,7 +44,7 @@ public SqlserverQuerySqlBuilder(JdbcDataReader reader) { @Override protected String buildQuerySql() { - List selectColumns = buildSelectColumns(databaseInterface, metaColumns); + List selectColumns = DbUtil.buildSelectColumns(databaseInterface, metaColumns); boolean splitWithRowNum = addRowNumColumn(databaseInterface, selectColumns, isSplitByKey, splitKey); StringBuilder sb = new StringBuilder(); diff --git a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml index 2b2ae09d82..9ae14cbb76 100644 --- a/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml +++ b/flinkx-sqlserver/flinkx-sqlserver-writer/pom.xml @@ -94,6 +94,7 @@ + diff --git a/flinkx-stream/flinkx-stream-reader/pom.xml b/flinkx-stream/flinkx-stream-reader/pom.xml index 9be42e64d3..2ec8b70719 100644 --- a/flinkx-stream/flinkx-stream-reader/pom.xml +++ b/flinkx-stream/flinkx-stream-reader/pom.xml @@ -82,6 +82,7 @@ + diff --git a/flinkx-stream/flinkx-stream-writer/pom.xml b/flinkx-stream/flinkx-stream-writer/pom.xml index 276a46b582..540c3d9d43 100644 --- a/flinkx-stream/flinkx-stream-writer/pom.xml +++ b/flinkx-stream/flinkx-stream-writer/pom.xml @@ -78,6 +78,7 @@ + diff --git a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java index ac578d2c86..c19e618846 100644 --- a/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java +++ b/flinkx-stream/flinkx-stream-writer/src/main/java/com/dtstack/flinkx/stream/writer/StreamOutputFormat.java @@ -18,13 +18,12 @@ package com.dtstack.flinkx.stream.writer; -import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.restore.FormatState; import org.apache.flink.types.Row; import org.apache.flink.util.StringUtils; -import java.io.IOException; import java.util.List; /** @@ -40,40 +39,52 @@ public class StreamOutputFormat extends BaseRichOutputFormat { protected List metaColumns; + protected Row lastRow; + @Override - protected void openInternal(int taskNumber, int numTasks) throws IOException { + protected void openInternal(int taskNumber, int numTasks) { // do nothing } @Override - protected void writeSingleRecordInternal(Row row) throws WriteRecordException { + protected void writeSingleRecordInternal(Row row) { if (print) { LOG.info("subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); } - - if (restoreConfig.isRestore()) { - formatState.setState(row.getField(restoreConfig.getRestoreColumnIndex())); - LOG.info("print data subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(row, writeDelimiter)); - } + lastRow = row; } @Override - protected void writeMultipleRecordsInternal() throws Exception { + protected void writeMultipleRecordsInternal() { if (print) { for (Row row : rows) { LOG.info(rowToStringWithDelimiter(row, writeDelimiter)); } } + if(rows.size() > 1){ + lastRow = rows.get(rows.size() - 1); + } + } + + public FormatState getFormatState(){ + if(lastRow != null){ + LOG.info("subTaskIndex[{}]:{}", taskNumber, rowToStringWithDelimiter(lastRow, writeDelimiter)); + } + return super.getFormatState(); } public String rowToStringWithDelimiter(Row row, String writeDelimiter) { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < row.getArity(); i++) { - if (i > 0) { - sb.append(writeDelimiter); + if(row == null){ + return ""; + }else{ + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getArity(); i++) { + if (i > 0) { + sb.append(writeDelimiter); + } + sb.append(StringUtils.arrayAwareToString(row.getField(i))); } - sb.append(StringUtils.arrayAwareToString(row.getField(i))); + return sb.toString(); } - return sb.toString(); } } diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogFile.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5ConfigKeys.java similarity index 100% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogFile.java rename to flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5ConfigKeys.java diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerConnection.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5DatabaseMeta.java similarity index 100% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerConnection.java rename to flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5DatabaseMeta.java diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerListener.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5InputSplit.java similarity index 100% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogMinerListener.java rename to flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/Phoenix5InputSplit.java diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogParser.java b/flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/IPhoenix5Helper.java similarity index 100% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/LogParser.java rename to flinkx-teradata/flinkx-teradata-core/src/main/java/com/dtstack/flinkx/teradata/util/IPhoenix5Helper.java diff --git a/flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/PositionManager.java b/flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/Phoenix5InputFormatBuilder.java similarity index 100% rename from flinkx-mongodb/flinkx-mongodb-oplog-reader/src/main/java/com/dtstack/mongodb/oplog/format/PositionManager.java rename to flinkx-teradata/flinkx-teradata-reader/src/main/java/com/dtstack/flinkx/teradata/reader/Phoenix5InputFormatBuilder.java diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index c57526a48e..a544ea0ca3 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -12,7 +12,6 @@ flinkx-test - 1.10.0 0.3.0 @@ -395,18 +394,6 @@ 1.6 - - com.dtstack.flinkx - flinkx-phoenix-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-phoenix-writer - 1.6 - - org.apache.hadoop hadoop-hdfs @@ -464,7 +451,26 @@ flinkx-greenplum-writer 1.6 + + com.dtstack.flinkx + flinkx-phoenix5-reader + 1.6 + + + com.dtstack.flinkx + flinkx-phoenix5-writer + 1.6 + + + com.dtstack.flinkx + flinkx-kingbase-reader + 1.6 + + + com.dtstack.flinkx + flinkx-kingbase-writer + 1.6 + - \ No newline at end of file diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 4e83dcfc1c..c6d83f82db 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -17,8 +17,6 @@ */ package com.dtstack.flinkx.test; -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; import com.dtstack.flinkx.binlog.reader.BinlogReader; import com.dtstack.flinkx.carbondata.reader.CarbondataReader; @@ -55,6 +53,8 @@ import com.dtstack.flinkx.kafka10.writer.Kafka10Writer; import com.dtstack.flinkx.kafka11.reader.Kafka11Reader; import com.dtstack.flinkx.kafka11.writer.Kafka11Writer; +import com.dtstack.flinkx.kingbase.reader.KingbaseReader; +import com.dtstack.flinkx.kingbase.writer.KingbaseWriter; import com.dtstack.flinkx.kudu.reader.KuduReader; import com.dtstack.flinkx.kudu.writer.KuduWriter; import com.dtstack.flinkx.mongodb.reader.MongodbReader; @@ -66,15 +66,14 @@ import com.dtstack.flinkx.odps.writer.OdpsWriter; import com.dtstack.flinkx.oracle.reader.OracleReader; import com.dtstack.flinkx.oracle.writer.OracleWriter; -import com.dtstack.flinkx.phoenix.reader.PhoenixReader; -import com.dtstack.flinkx.phoenix.writer.PhoenixWriter; +import com.dtstack.flinkx.phoenix5.reader.Phoenix5Reader; +import com.dtstack.flinkx.phoenix5.writer.Phoenix5Writer; import com.dtstack.flinkx.polardb.reader.PolardbReader; import com.dtstack.flinkx.polardb.writer.PolardbWriter; import com.dtstack.flinkx.postgresql.reader.PostgresqlReader; import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.redis.writer.RedisWriter; -import com.dtstack.flinkx.restapi.writer.RestapiWriter; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; import com.dtstack.flinkx.stream.reader.StreamReader; @@ -86,9 +85,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -104,7 +101,6 @@ import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.concurrent.TimeUnit; - /** * @author jiangbo */ @@ -117,7 +113,6 @@ public class LocalTest { public static Configuration conf = new Configuration(); public static void main(String[] args) throws Exception{ -// setLogLevel(Level.DEBUG.toString()); Properties confProperties = new Properties(); // confProperties.put("flink.checkpoint.interval", "10000"); // confProperties.put("flink.checkpoint.stateBackend", "file:///tmp/flinkx_checkpoint"); @@ -129,7 +124,7 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - String jobPath = "/Users/yanghuai/Library/Application Support/JetBrains/IntelliJIdea2020.1/scratches/stream_stream.json"; + String jobPath = "your json file's absolute path"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); System.exit(0); @@ -158,11 +153,10 @@ public static JobExecutionResult runJob(String job, Properties confProperties, S Time.of(10, TimeUnit.SECONDS) )); } - + SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); BaseDataReader reader = buildDataReader(config, env); DataStream dataStream = reader.readData(); - SpeedConfig speedConfig = config.getJob().getSetting().getSpeed(); - if (speedConfig.getReaderChannel() > 0) { + if(speedConfig.getReaderChannel() > 0){ dataStream = ((DataStreamSource) dataStream).setParallelism(speedConfig.getReaderChannel()); } @@ -172,7 +166,7 @@ public static JobExecutionResult runJob(String job, Properties confProperties, S BaseDataWriter dataWriter = buildDataWriter(config); DataStreamSink dataStreamSink = dataWriter.writeData(dataStream); - if (speedConfig.getWriterChannel() > 0) { + if(speedConfig.getWriterChannel() > 0){ dataStreamSink.setParallelism(speedConfig.getWriterChannel()); } @@ -224,10 +218,11 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstants.KUDU_READER : reader = new KuduReader(config, env); break; case PluginNameConstants.CLICKHOUSE_READER : reader = new ClickhouseReader(config, env); break; case PluginNameConstants.POLARDB_READER : reader = new PolardbReader(config, env); break; - case PluginNameConstants.PHOENIX_READER : reader = new PhoenixReader(config, env); break; case PluginNameConstants.EMQX_READER : reader = new EmqxReader(config, env); break; case PluginNameConstants.DM_READER : reader = new DmReader(config, env); break; case PluginNameConstants.GREENPLUM_READER : reader = new GreenplumReader(config, env); break; + case PluginNameConstants.PHOENIX5_READER : reader = new Phoenix5Reader(config, env); break; + case PluginNameConstants.KINGBASE_READER : reader = new KingbaseReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } @@ -261,11 +256,11 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ case PluginNameConstants.CLICKHOUSE_WRITER : writer = new ClickhouseWriter(config); break; case PluginNameConstants.POLARDB_WRITER : writer = new PolardbWriter(config); break; case PluginNameConstants.KAFKA_WRITER : writer = new KafkaWriter(config); break; - case PluginNameConstants.PHOENIX_WRITER : writer = new PhoenixWriter(config); break; case PluginNameConstants.EMQX_WRITER : writer = new EmqxWriter(config); break; - case PluginNameConstants.RESTAPI_WRITER : writer = new RestapiWriter(config);break; case PluginNameConstants.DM_WRITER : writer = new DmWriter(config); break; case PluginNameConstants.GREENPLUM_WRITER : writer = new GreenplumWriter(config); break; + case PluginNameConstants.PHOENIX5_WRITER : writer = new Phoenix5Writer(config); break; + case PluginNameConstants.KINGBASE_WRITER : writer = new KingbaseWriter(config); break; default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } @@ -273,46 +268,23 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ } private static void openCheckpointConf(StreamExecutionEnvironment env, Properties properties){ - if(properties == null){ - return; - } - - if(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ - return; - }else{ - long interval = Long.parseLong(properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY).trim()); - - //start checkpoint every ${interval} - env.enableCheckpointing(interval); - - LOG.info("Open checkpoint with interval:" + interval); + if(properties!=null){ + String interval = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_INTERVAL_KEY); + if(StringUtils.isNotBlank(interval)){ + env.enableCheckpointing(Long.parseLong(interval.trim())); + LOG.info("Open checkpoint with interval:" + interval); + } + String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); + if(checkpointTimeoutStr != null){ + long checkpointTimeout = Long.parseLong(checkpointTimeoutStr.trim()); + //checkpoints have to complete within one min,or are discard + env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); + + LOG.info("Set checkpoint timeout:" + checkpointTimeout); + } + env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); } - - String checkpointTimeoutStr = properties.getProperty(ConfigConstant.FLINK_CHECKPOINT_TIMEOUT_KEY); - if(checkpointTimeoutStr != null){ - long checkpointTimeout = Long.parseLong(checkpointTimeoutStr); - //checkpoints have to complete within one min,or are discard - env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); - - LOG.info("Set checkpoint timeout:" + checkpointTimeout); - } - - env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); - env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); - env.getCheckpointConfig().enableExternalizedCheckpoints( - CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); - - env.setStateBackend(new FsStateBackend(new Path("file:///tmp/flinkx_checkpoint"))); - env.setRestartStrategy(RestartStrategies.failureRateRestart( - FAILURE_RATE, - Time.of(FAILURE_INTERVAL, TimeUnit.MINUTES), - Time.of(DELAY_INTERVAL, TimeUnit.SECONDS) - )); - } - - private static void setLogLevel(String level) { - LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); - ch.qos.logback.classic.Logger logger = loggerContext.getLogger("root"); - logger.setLevel(Level.toLevel(level)); } } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index 21d07e7c9e..472cf2623e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -21,7 +21,6 @@ /** * @author jiangbo */ -@SuppressWarnings("all") public class PluginNameConstants { public static final String STREAM_READER = "streamreader"; @@ -50,13 +49,10 @@ public class PluginNameConstants { public static final String ORACLE_LOG_MINER_READER = "oraclelogminerreader"; public static final String PHOENIX_READER = "phoenixreader"; public static final String EMQX_READER = "emqxreader"; - public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; - public static final String METADATAHIVE2_READER = "metadatahive2reader"; public static final String DM_READER = "dmreader"; - public static final String METADATATIDB_READER = "metadatatidbreader"; - public static final String METADATAORACLE_READER = "metadataoraclereader"; - public static final String METADATAMYSQL_READER = "metadatamysqlreader"; public static final String GREENPLUM_READER = "greenplumreader"; + public static final String PHOENIX5_READER = "phoenix5reader"; + public static final String KINGBASE_READER = "kingbasereader"; public static final String STREAM_WRITER = "streamwriter"; public static final String CARBONDATA_WRITER = "carbondatawriter"; @@ -86,4 +82,6 @@ public class PluginNameConstants { public static final String RESTAPI_WRITER = "restapiwriter"; public static final String DM_WRITER = "dmwriter"; public static final String GREENPLUM_WRITER = "greenplumwriter"; + public static final String PHOENIX5_WRITER = "phoenix5writer"; + public static final String KINGBASE_WRITER = "kingbasewriter"; } diff --git a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java deleted file mode 100644 index 5fbf58b645..0000000000 --- a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/AbstractPrometheusReporter.java +++ /dev/null @@ -1,315 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.metrics.prometheus; - -import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.metrics.CharacterFilter; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.Histogram; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; -import org.apache.flink.runtime.metrics.groups.FrontMetricGroup; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.FILTER_LABEL_VALUE_CHARACTER; - -/** - * base prometheus reporter for prometheus metrics. - */ -@PublicEvolving -public abstract class AbstractPrometheusReporter implements MetricReporter { - - private static final Pattern UNALLOWED_CHAR_PATTERN = Pattern.compile("[^a-zA-Z0-9:_]"); - private static final CharacterFilter CHARACTER_FILTER = new CharacterFilter() { - @Override - public String filterCharacters(String input) { - return replaceInvalidChars(input); - } - }; - private static final char SCOPE_SEPARATOR = '_'; - private static final String SCOPE_PREFIX = "flink" + SCOPE_SEPARATOR; - protected final Logger log = LoggerFactory.getLogger(getClass()); - private final Map> collectorsWithCountByMetricName = new HashMap<>(); - private CharacterFilter labelValueCharactersFilter = CHARACTER_FILTER; - - @VisibleForTesting - static String replaceInvalidChars(final String input) { - // https://prometheus.io/docs/instrumenting/writing_exporters/ - // Only [a-zA-Z0-9:_] are valid in metric names, any other characters should be sanitized to an underscore. - return UNALLOWED_CHAR_PATTERN.matcher(input).replaceAll("_"); - } - - private static String getScopedName(String metricName, MetricGroup group) { - return SCOPE_PREFIX + getLogicalScope(group) + SCOPE_SEPARATOR + CHARACTER_FILTER.filterCharacters(metricName); - } - - @SuppressWarnings("unchecked") - private static String getLogicalScope(MetricGroup group) { - return ((FrontMetricGroup>) group).getLogicalScope(CHARACTER_FILTER, SCOPE_SEPARATOR); - } - - private static io.prometheus.client.Gauge.Child gaugeFrom(Counter counter) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - return (double) counter.getCount(); - } - }; - } - - private static io.prometheus.client.Gauge.Child gaugeFrom(Meter meter) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - return meter.getRate(); - } - }; - } - - private static List addToList(List list, String element) { - final List result = new ArrayList<>(list); - result.add(element); - return result; - } - - private static String[] toArray(List list) { - return list.toArray(new String[list.size()]); - } - - @Override - public void open(MetricConfig config) { - boolean filterLabelValueCharacters = config.getBoolean( - FILTER_LABEL_VALUE_CHARACTER.key(), FILTER_LABEL_VALUE_CHARACTER.defaultValue()); - - if (!filterLabelValueCharacters) { - labelValueCharactersFilter = input -> input; - } - } - - @Override - public void close() { - CollectorRegistry.defaultRegistry.clear(); - } - - @Override - public void notifyOfAddedMetric(final Metric metric, final String metricName, final MetricGroup group) { - - List dimensionKeys = new LinkedList<>(); - List dimensionValues = new LinkedList<>(); - for (final Map.Entry dimension : group.getAllVariables().entrySet()) { - final String key = dimension.getKey(); - dimensionKeys.add(CHARACTER_FILTER.filterCharacters(key.substring(1, key.length() - 1))); - dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); - } - - final String scopedMetricName = getScopedName(metricName, group); - final String helpString = metricName + " (scope: " + getLogicalScope(group) + ")"; - - final Collector collector; - Integer count = 0; - - synchronized (this) { - if (collectorsWithCountByMetricName.containsKey(scopedMetricName)) { - final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); - collector = collectorWithCount.getKey(); - count = collectorWithCount.getValue(); - } else { - collector = createCollector(metric, dimensionKeys, dimensionValues, scopedMetricName, helpString); - try { - collector.register(); - } catch (Exception e) { - log.warn("There was a problem registering metric {}.", metricName, e); - } - } - addMetric(metric, dimensionValues, collector); - collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count + 1)); - } - } - - private Collector createCollector(Metric metric, List dimensionKeys, List dimensionValues, String scopedMetricName, String helpString) { - Collector collector; - if (metric instanceof Gauge || metric instanceof Counter || metric instanceof Meter) { - collector = io.prometheus.client.Gauge - .build() - .name(scopedMetricName) - .help(helpString) - .labelNames(toArray(dimensionKeys)) - .create(); - } else if (metric instanceof Histogram) { - collector = new HistogramSummaryProxy((Histogram) metric, scopedMetricName, helpString, dimensionKeys, dimensionValues); - } else { - log.warn("Cannot create collector for unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - collector = null; - } - return collector; - } - - private void addMetric(Metric metric, List dimensionValues, Collector collector) { - if (metric instanceof Gauge) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Gauge) metric), toArray(dimensionValues)); - } else if (metric instanceof Counter) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Counter) metric), toArray(dimensionValues)); - } else if (metric instanceof Meter) { - ((io.prometheus.client.Gauge) collector).setChild(gaugeFrom((Meter) metric), toArray(dimensionValues)); - } else if (metric instanceof Histogram) { - ((HistogramSummaryProxy) collector).addChild((Histogram) metric, dimensionValues); - } else { - log.warn("Cannot add unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - } - } - - private void removeMetric(Metric metric, List dimensionValues, Collector collector) { - if (metric instanceof Gauge) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Counter) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Meter) { - ((io.prometheus.client.Gauge) collector).remove(toArray(dimensionValues)); - } else if (metric instanceof Histogram) { - ((HistogramSummaryProxy) collector).remove(dimensionValues); - } else { - log.warn("Cannot remove unknown metric type: {}. This indicates that the metric type is not supported by this reporter.", - metric.getClass().getName()); - } - } - - @Override - public void notifyOfRemovedMetric(final Metric metric, final String metricName, final MetricGroup group) { - - List dimensionValues = new LinkedList<>(); - for (final Map.Entry dimension : group.getAllVariables().entrySet()) { - dimensionValues.add(labelValueCharactersFilter.filterCharacters(dimension.getValue())); - } - - final String scopedMetricName = getScopedName(metricName, group); - synchronized (this) { - final AbstractMap.SimpleImmutableEntry collectorWithCount = collectorsWithCountByMetricName.get(scopedMetricName); - final Integer count = collectorWithCount.getValue(); - final Collector collector = collectorWithCount.getKey(); - - removeMetric(metric, dimensionValues, collector); - - if (count == 1) { - try { - CollectorRegistry.defaultRegistry.unregister(collector); - } catch (Exception e) { - log.warn("There was a problem unregistering metric {}.", scopedMetricName, e); - } - collectorsWithCountByMetricName.remove(scopedMetricName); - } else { - collectorsWithCountByMetricName.put(scopedMetricName, new AbstractMap.SimpleImmutableEntry<>(collector, count - 1)); - } - } - } - - @VisibleForTesting - io.prometheus.client.Gauge.Child gaugeFrom(Gauge gauge) { - return new io.prometheus.client.Gauge.Child() { - @Override - public double get() { - final Object value = gauge.getValue(); - if (value == null) { - log.debug("Gauge {} is null-valued, defaulting to 0.", gauge); - return 0; - } - if (value instanceof Double) { - return (double) value; - } - if (value instanceof Number) { - return ((Number) value).doubleValue(); - } - if (value instanceof Boolean) { - return ((Boolean) value) ? 1 : 0; - } - log.debug("Invalid type for Gauge {}: {}, only number types and booleans are supported by this reporter.", - gauge, value.getClass().getName()); - return 0; - } - }; - } - - @VisibleForTesting - static class HistogramSummaryProxy extends Collector { - static final List QUANTILES = Arrays.asList(.5, .75, .95, .98, .99, .999); - - private final String metricName; - private final String helpString; - private final List labelNamesWithQuantile; - - private final Map, Histogram> histogramsByLabelValues = new HashMap<>(); - - HistogramSummaryProxy(final Histogram histogram, final String metricName, final String helpString, final List labelNames, final List labelValues) { - this.metricName = metricName; - this.helpString = helpString; - this.labelNamesWithQuantile = addToList(labelNames, "quantile"); - histogramsByLabelValues.put(labelValues, histogram); - } - - @Override - public List collect() { - // We cannot use SummaryMetricFamily because it is impossible to get a sum of all values (at least for Dropwizard histograms, - // whose snapshot's values array only holds a sample of recent values). - - List samples = new LinkedList<>(); - for (Map.Entry, Histogram> labelValuesToHistogram : histogramsByLabelValues.entrySet()) { - addSamples(labelValuesToHistogram.getKey(), labelValuesToHistogram.getValue(), samples); - } - return Collections.singletonList(new MetricFamilySamples(metricName, Type.SUMMARY, helpString, samples)); - } - - void addChild(final Histogram histogram, final List labelValues) { - histogramsByLabelValues.put(labelValues, histogram); - } - - void remove(final List labelValues) { - histogramsByLabelValues.remove(labelValues); - } - - private void addSamples(final List labelValues, final Histogram histogram, final List samples) { - samples.add(new MetricFamilySamples.Sample(metricName + "_count", - labelNamesWithQuantile.subList(0, labelNamesWithQuantile.size() - 1), labelValues, histogram.getCount())); - for (final Double quantile : QUANTILES) { - samples.add(new MetricFamilySamples.Sample(metricName, labelNamesWithQuantile, - addToList(labelValues, quantile.toString()), - histogram.getStatistics().getQuantile(quantile))); - } - } - } -} diff --git a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java b/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java deleted file mode 100644 index 52fd24d46f..0000000000 --- a/flinkx-test/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.metrics.prometheus; - -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.exporter.PushGateway; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.metrics.Metric; -import org.apache.flink.metrics.MetricConfig; -import org.apache.flink.metrics.reporter.MetricReporter; -import org.apache.flink.metrics.reporter.Scheduled; -import org.apache.flink.util.AbstractID; - -import java.io.IOException; - -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.DELETE_ON_SHUTDOWN; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.HOST; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.JOB_NAME; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.PORT; -import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterOptions.RANDOM_JOB_NAME_SUFFIX; - -/** - * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. - */ -@PublicEvolving -public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { - - private PushGateway pushGateway; - private String jobName; - private boolean deleteOnShutdown; - - @Override - public void open(MetricConfig config) { - super.open(config); - - String host = config.getString(HOST.key(), HOST.defaultValue()); - int port = config.getInteger(PORT.key(), PORT.defaultValue()); - String configuredJobName = config.getString(JOB_NAME.key(), JOB_NAME.defaultValue()); - boolean randomSuffix = config.getBoolean(RANDOM_JOB_NAME_SUFFIX.key(), RANDOM_JOB_NAME_SUFFIX.defaultValue()); - deleteOnShutdown = config.getBoolean(DELETE_ON_SHUTDOWN.key(), DELETE_ON_SHUTDOWN.defaultValue()); - - if (host == null || host.isEmpty() || port < 1) { - throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); - } - - if (randomSuffix) { - this.jobName = configuredJobName + new AbstractID(); - } else { - this.jobName = configuredJobName; - } - - pushGateway = new PushGateway(host + ':' + port); - log.info("Configured PrometheusPushGatewayReporter with {host:{}, port:{}, jobName: {}, randomJobNameSuffix:{}, deleteOnShutdown:{}}", host, port, jobName, randomSuffix, deleteOnShutdown); - } - - @Override - public void report() { - try { - pushGateway.push(CollectorRegistry.defaultRegistry, jobName); - log.info("push metrics to PushGateway with jobName {}.", jobName); - } catch (Exception e) { - log.warn("Failed to push metrics to PushGateway with jobName {}.", jobName, e); - } - } - - @Override - public void close() { - if (deleteOnShutdown && pushGateway != null) { - try { - pushGateway.delete(jobName); - log.info("delete metrics from PushGateway with jobName {}.", jobName); - } catch (IOException e) { - log.warn("Failed to delete metrics from PushGateway with jobName {}.", jobName, e); - } - } - super.close(); - } -} diff --git a/flinkx-test/src/main/resources/logback.xml b/flinkx-test/src/main/resources/logback.xml index 6e1771d94c..b408ae60df 100644 --- a/flinkx-test/src/main/resources/logback.xml +++ b/flinkx-test/src/main/resources/logback.xml @@ -15,7 +15,7 @@ - + diff --git a/jars/vertica-jdbc-9.1.1-0.jar b/jars/vertica-jdbc-9.1.1-0.jar new file mode 100644 index 0000000000000000000000000000000000000000..2c663c095b5302aa4b8c0c5da91791d93b2f6c06 GIT binary patch literal 769344 zcmbTc1yp29vIUB}yL;o%xVyX4xVu~7(73z1ySqCy?(Xi5G}<`)z9aA7*YoDi{9kL; zs#CRMXJo|Q5t()JC`f~X!2kh40RaVs{*(dw(+e613`kZ~MUYNXPVDph7!Z)cKTM&3 z{C}CQ&&!%D{4x&&1_DC+_4z-hvVwAwVxr0_^s-_>G4fU*j7Wn&{JcUqmxP;9ZR##% zmm~Zy@&>I!SK}aV&9IH~w!5josjTvb-G8v(9$i@9h;v~!QSiNFYR=lr4Wqh9Xi<0+a>f~XGGdi_ljwRvq3-!GFzC>At~J$j z9X@CRrN^n9WZ9#Of1X&jQnKDdqEw>(2#K2EpSMj_%tX`s-&rtc6SBej75yJ?;y-7> z$;8pc!pOwwv!#KH0lkxhHNA+Vg^P)!^wjhsjWiwgsEpjgxRj*Sk`vV!*iGC2Bp2v! z$^A9b|GdC{T^ZTgeEz57e`N~p9~Qrc2Ur*x{Dmd*Z^;s*E&n*T4R{y-R*031#JO7{PkD*v0< zey_3rDlYur;`$TF(%8_5`7Z+e4^WmrL4R)th`)KYu=|to?^XJHtNCMIe_{Sd1N}2o ze>4B}gN>c-|5uc`yP>0n@n4{RPx;?`{inSBThN~~_InckSQ7pvsy{rjeEx5=@wd! z-?IAOsQUM!_SA z*|^&8aR3Xy*IV(bn@-|#1bM$8CF*RR!--?^CUb4eZobMihUfDIc^$cCqeCuJXAFwc}ORhvU z)*;f$7gA^&D;DvMEOXIR7qIiID_V==?S15^$g}7uOew0Iz}1W$HX( zkez4!zPe$cTEDV;GYyGw>4>nr4!)Gv@ZphV5A zhq*EGGt8D#EvHG07@|q7%FG_ysXXpu>X|m-`RY#s1KF< z#ir~pHd%fX(4X0qv2X&Ai5XZ}n;0wG8CjVCL~Q|%?tdrOJFf2!a)UoC=7xsUwW^vd zwLDMqq73iIDgDuc1Ox@IG!5Gq7TDs9sUGDB&}jW3{~(uw5fZTpwxfah1MU|7_Wtb} zdIze*-B@ucZb0m{;@fm_FdaOrKi{cmckFS1phW4N*wjMZO3OJ5W&TNEY{ng7fC9>h zsrZ(s;#us>Nn78y^l$}_YNIwm%F4#VV^Jo0?t3r!Y~<+m2<794ChiunCR{kMJO;Fz z7_ri3?vGi^OwX(x>WgR_ee7UA^aXfxb*9*m>O4UX^}vO8KxZA+2H0glEM_-c4pb2* z-0Eg3~c+^VSW& z8ACG$#zN>oB?n6=70qw@S+8WRP9Rv6PFt4OS$OyKs#P*-u~L_rNJX8-2x-#$* z=@Y+ZT)LA%=ZluVa+Dit0QQP^{$+Oyqw&%hd5mT~4DNF7uB*Fq$lU3b44sp3?H;VR zGl>2~@}s+OXZ%qLiTANQvW77;mT@$(X8aLiz^*>rZuGI)pH2I~Tvj)Gci&b0S#&Ea ze%+VxNuH=K?+u_1=6X79rO z9Be7|Vk-L&sh@sw`B=~tCTZVJNn?T=nCM}xX{osi&6G~B`PQ-6tOsObF|({JPaCRe z#MZ+$b1_w*WW5_cStDrcA*qrT0l0S)$?W#>f zDz{WqhNWb8Q86E$qqkCdA0HzPp0~;@I=8Vf?=*^ttWq)pQCz@GI&HkjhdNC{(G_x1bWC- zXt?-n&EBS4OgKk$>B)enNoV1n_ah(cW`0va%n;ROLUaN;5X~!a)9Vc$A(nJeLU7ey zNNfI{_#@-JMI<}FB4ZE@AP?jI{^ZnPdppS%S-mT~qLpOd1f747hN)N;jiF!nGdZTNDxt#HZAVO^QA*Z6% z@4+cD03_pw0t^P0$~D)P!ZnBV8;BG2*U|Bst8{_Qi$)id`l;58^}$$`&M>_SHxy{} z9)n|_iaqkLC3_DXLMhIYtgh(r6^{c_hG!(#F^}{9sx3@Mr53y<1kH|eVT&ycg?rN* z$cXQ*YMBFX5#c@c%Rc)AP&J==$QH%3c|xIEc_wZn|N@dYXA5YbwQtX5w!zyN2B7(%T$`f5q zKy;wVE3s%N^Q5GB4ppT8_*7Q-nlN+Wv_ZnloeW=ROn(Sn%(M!))AwO_h-w`=lU{W+ z(#Td-%qJ+aF3CjWZfpP%GL!Of;`{tO$p6ghe#fFJFWHZ9fh> zR6_+$YSKx5BA4kXWa4a$bQXX};7lJ0ZTxQKGsvF!sm} zVU%t;@5{~#r#m12M~%h?tGd=ETD1;)K!;KNQI06QMfu_Z$P#949-D}CtFI@_%E{DNez*@z$4CgmJjC*d1kq#gR>P{20z%&f{<$nf>7|4 zIPfYvBYa@_KDg#@{y3*AYvqB7;SpCMTIhT=F{+}C)W}LOE6KxK>a`0LY949IeN1wP z$62qhrxl40z?s9~&$rj|$QT*WN@GvX_vOBHD)NMVQ4K46D~%1d5Zs0+W!F|p!0^M> zAl1DNnUu+Nq9)2vKSSM2Jl&Me&5l5Dilj@go8xd-2&PH#@}yMVOUcC)xm@LXDT;5d z9>S@(_#Vl>lfa!~X7ve+*rbz$m4`j0Xa|SX zga$SG*-+s&1ca`RXq5W%{zZyn5xCxPwd6OR32yp=bjB*wtK&Mld~o4-D?8Me;XEro z#Fl`9X(i<=*1aj znTcOx8Ns4^fq29@;HuX8=ZfJiWZ49K`m~}Y;8}Nq{%WFv<{l~c)jx#hp-ut##W0>6 zy~CU0$A+z4#1o4i@DCE57Yc=em*^`|utIN@o7Yz@=Y-fIp29KkA06gZpEhk+v=C6v z`yOUG^j_Hi_Y&_|+5(vPwFT1!1p=b~zn1tf!gjW{CPsh!ZzNMOaI*Tx*S|0KLCRWo z^9*S38!oab;3G?nwjIuB$;G*Fy4*Q@q z0_i|Mf62}#8yWsXcY3GbE6WWyD`rr$lVlx4r=$^!*?3H^R=t&Z3vBG{;q;p8npz2bbDhu^m_MdO1=C&Q>~ykIi?^a#pP^OzEG~T5;x1H6AdrY9?hZ`k9!a`$EKAw=l%h*Ndg$1u-T$yPac=pk$|nBhqkF@B zTh8yWVS0L zABa1*sd$NaPssRAi7C=3ZN&Xui0BYWv&C5~D3+vl3Lv4^g6ul#&J{}*Xx_w<2%O@X zEUcRES{o^(J%l`%v^=Cda1we{i?%t7y5{)2cnYL?rsLm<-jFliC6lt9aJobpQXa(tU_H6;iN4Qk}V;}EwRiW=5u;1 zr#d1(3HaW_joxs(yJ{o}&F8kS?}aJc6`v&%_?8OgSx%K@pW&SNhc`Z}oJz}h6-j=K z#CIomJ4kK&3lJUWq~B$czrpJDEEoEaz6vJ17a%?7r0Aa!GCuT_-zAX0Mb^BqrkU3R z)zp`cq6ldupv$`Welt-`U@gh~#Jwx{65GJ4;W`#@(3d(gYmC{iE$c#hXYNSQ?@(l%Icm-9y6p?Lmo4GQ1|E&OCAF{I|gZF*hwxgpk6qNiu}yd zhGh^&o|o8>y{r=s zHpR*@OSw7)w{{LjR<~@d!gu*5PJN|xtb*}_Qk8E_Qa8bN;lL=fpZKOueWh@8*uv3Q zI@A|zN#CV`yoCtP(vha0&3Ih1mXjCz30Y(NfUR$SwK(&p3i6 ze5+JBBst#`^MY;hm_sUaNr_ZSw!{iL|n3zcUEXJto58=VZ=es7I zB}Cdv#3a`ZN0?GjTO+W1-A%1`ik4njRqmBUt{2tr0{^Ji70G!`g$0U_?VYPfWF5!cK zQa8I+w|qAtA)@|`V99H2=O)#USi+$e@hf=zy3Cnx?6r!Bd@dGy8R?Nz>R(k7JXxRE zOBZ7dx`r8@5W9JI6LS)`h7Xh<-EiV>>>N6K2F5&D-3j~6YR@0iUm<1R3#dON6LLgt zc}hm72VRMKa>qWCRHv{!naF+>P0*W6)=T)P9IG*dbse?S-;?#~3>w+~a_EZmK1j7e z(-SedrF_`K6$E+Mv%BwowQ>V#Zs`MQNAGiz$RE%DOO_10bq!X=O*o zi=hy4 z1+0fyvYZ$I5eucH{mebI(_aV8-uy=pMA8}o7wN>MiZc#0g0dEiiEOeO6;#Bs8YNU` ztVKG?Tq>W=(Z-44lt`pX%JC4_@UG7*cP>%~zcaI)t&#{7s8LA8ac{co~04+u& z^fy)Iuf_>IUSE@XM)g!XY7OE90xkXE{GaRGi175V&40j1?d1G~B^&Mm7|hqVmxC{g zvn?hst3ZFd(8F9oO5GrUYLTsu5f|QVW|HNMSgxeCE49?DZLBWt6eJ>5pLDZV_Hy(t zEok6X#}0+W6v@Z;b4wH8sVg=WGoVDN4h015_mv8SYZO$wm3lLwvAc>E^x1nk5)qt2 zvC7kzY!gBSRcnwhBT2s?|qD-Y)1v_DgysGOQx#xz17==(So8r=lzvk+J!$k>Jj z0co*tV2`G1Yv^6XG%~_*RrS!$bn?w$ph;lJ4=sb_aHVq<#JLr6&LM!pLvVpHae-S= zGP!F=f%m3t&!QC=>1?4#+E%ZjSzm^W>uY~1y93+v({H6iLvWlbT!3J)6+nfwb?)7f z-QwU%UlKDUQm85@Gm+((K`wOHVWeF+k@oH)ENs&xk|QEG=wd{%$xGrQRW#=)EGT1b zq*^hI?P9`;0)7BzN{!}5vxRa7Bh^u0?kJ!_V^$_0qL}OC!_q@ChX7$;p6kv-rbHvT zf?NVyw`fXMKqKSgN3zq^!iVVJ1JKUuS61-y^;9&mVpSn#0oeo%f{YfBu~$Yx1dM~s zau9*r7-lIBO7$@{7Eoa!R9YY|0gFP#nGi3S#UvfZHn41Tr@^x#S?b|b`(+F7N=Vyd zpIAp?0KnMjB1F~#=+KKng!&|2{7w4I=^8O4{A`|JER`9=&hwe-89S+w!b$fjNAdgI zfP!?8&2@3_LNo&$H1ml&GC(h`($@tibNR3!sn-qfhg+0UKxcFLQEZJczKB9@33X_I zmai>p5JuWEoZuK68=^cps1%1O8O9$S#eiuZLFNiEHxdSW5P*$}1kfGQ(GgXwfFOy! zn5uH(t`!;66`%(+>WGF*Dc@?Q9l(~VQ=r)tZy9S7N{mt7==kK+k7QAjvb>A-6VF`gHZ%(bn?oR_qtA`JCO zEomj8CiY;v&v%oxrd4p(;|4Ei!;>9hVHZEaK^_JkFtRvV_zVzOV-~H;|SdfQCm_F|!xP&(bgMH7@Nv zhk6x7v!)gR*X^!sCz+XYYiC5$%TSVNie#LO*NUM>@Dc=1Vtlx#%qrZ@oJrZ%PP=5{ zB@Zsj>a)kc=K$d>o&H$~v7&>0FVacJTv=|8#!Z+pYA4M(Vd}?Z*~`#S%mygJ(Yp2U zdpxILvj;shN;vbsLhuSR+GJYe3*_TpTZE3U%+Zq#!qpcGOb6j@0^zRe zCQOL~t$eq#rqNW!xiZ>S%}lUqEkwFReYL8kMMMw4oQi>{k9Z76??! zLy~?j!siFRx?Dn7W}>7Q7aT3*;3z9d=aGR5L{DtbeiQbh0T+D&6Pm2YkJS=9>YE^r zFipV%BAV`_9J+OV7{}Rjwf=!bdJUaAD{eB;SJ* zt3IjAn_1=cd@z5#qyOmUwcoZw9r2v);4m{o1i|cV^R3s4B6hHm+#}!s4DwcR7I@0K zGbcH5sk#>j>O{FdxUuLiHdBbAnA`6w`K&)#ngugHZJBjy93MVO9k`A6K30mDG@%o9 z^k5EGS^dv}^uFf&X3c-}BXNaPzRW5dE&1Hov3~WdwvXacwt_Qami;>`3ZXV<+~R6j7WhsMU}dAuKkVX38B<=qN$&OhKKrR z=zw5m?^b5C3Rnoum2X*~QL}|hy<|f&0zsLgfaAWfU<>P54x?ld1e!W)Vu6sIMkmA5 zCr3rrto+!a3E7~|R2vrr{A%Y0;u1PuGo+)1mq|sFeJ=xBe|_&AF=yI20{sSWHquGL z4d1GU^~y5Ap_RuWBtild)H67i3^d}HH!rH3`Q&V7knfczWN zha-)ZNOSTc-0mTxy+lX5PmwUp0+P&$(h>Q1xXuCru11lPlcG9OlQ zjU+0G#7UNWqy@Gb6I80KLE2NVZqXDbh5=~P%LJ=Db#Pls-?B1><(3dO`+LfkO9Ju_a*9TMSFOdkB z2E_+;b-dsRV`z@Q?}4k^$GO$f>xS7a@9lFTyN(shq=|g+n}pME5wV@_k*Z^$WvCqT z=y2+_38>O9Bt;}fI3J>1qpG&4sL?CaFAp9MVgl+^!Pxhu#wvRI|AHIR|RR?_G7^;OzKb35zju8XRP&@=&n_gr1q2ZsfQSH4Tfh1Za zn0}u#y|xJrdESJY8FVIL2?yOzw5sJI1^KS}nN`h?iq^2aG6K0yl4!4wu6^YbK@s)x z_c7-&44u_-?HU^P(43R8sF9oT#0$Y8=#4R(ODG=_Qc8E^09#So7j3CHJ8vXkTmYz@HV!7mn0!TOdrv(|Am2$yU1#p zbVzuPFMf%W<5#uPE^su#bb9#Zq`)~+B!Bx90gOE`LTf=W1H6u6UNo&Uc(jEA?ha9=Yr0!ZZ>rbpV(_98jI3u(&N!0GI~#vh7*UKO=9Ym ziT0n5wR6$k_Q9t?IenYK@rHhUD{;QYt`$9mW*OnnKj4*gAQ6vi*eNjKv0X}{NGbsw zTA-CJ^)Cl048>$6luR%&38{UJX!4w)ALUYy}-ioL_olSCX zBFkTOQ-<>DzAK=~(RbGa-;+garVJUuqgw*{Bg>=*7)}k%rmRjP@dL26)ZU5k^QD!G zEI`tDe4#M+_ylq{Nkp;;VusUL0A&g|en!U#!5(?1?TyIXQzjILRo3_6EN63+TwwGG zF3p;%OHGv)^ovRS`L4)Ps)zFJH(i6BC5t!Vbwf^_5D0TF4tRyl8L4&~us@ z@>d`wD2_{_ryI7$vo@ns-+PNZy>Dhi$50%$PGgi^gB2OLiIK-6D*aEo@a&b7@OdlCY z`-LgLCshgaGkD_8x$gX6mLUip1i@@ z{CWGcD@1!N#|HYhtD~CsN2Z?zyD~_5G|8QAjy)`i8zN4%5wz_pWba9~(0gM+PrLW2 z1*8E+UZRE4ReS60H|&SrI!^*m9o1f?UWGJ-G)ThU<=SvK$d{IO!|(jcNz=}Uu!qt* zPcQ(Xr;r=gRNt&+UG&gk;HGk~0-9~=4-s0eV$IJz|d;fg$4)pT3VF zVljO{zC0ec*8ZfG)YbwotSaMtYdl zPcp>I^nU_h^KTE}0NW(q>-6vo^Z#@95gn(f3etu)PCFci>+ zBejk&^^je>o0ISlX3~u%91KIwzwsL&j{W|DUKFN46!Af!oz(}lNE(hovikvyoeF^Z zj6uZZhxZK2FiXtHo%1f8VG|jA-5s%dZ8?Urm z8MMGL#+l!h;ho?@Vx`W-Wb}l3_TKhD%L^1zAi|KsO{4{;pxjWmLv-;-@e2ejV$uPT zmsl6Re0=Eqpz=Vw8816j%39=Uh_B2`DGO=p6W)TNg%w?cxY6IbvfTD$%|2mBrJ;ht zUXZ(=DL1eOuPGXEf#lUKQ zI!PM+-9H4ciFG#U`4XGqh05gk$PJ7E}c?ORzm+y`OY4dzyR0)Y{1Qx(6Y!*r=lYyZL}wnwh6=;o#sj8K-P{lpSBwX z?=$F}BfFmNZSL{59{}e`loysor?nelSPodXR~%9bOJ|eDmc5D74IS;o@0doU2itom z%EusedA$YYVPp`n85t0Bf4;i`75;pQk)hgaju>K|m4kNkh&U<21I%uaZw}Vxy9vmv zOas&!(bMwci|?%SFql@(jmm*0Y}O23I*_i6=Mzn20cB?9MjRU`0&}n3-u!C6)kC=# zSu3|!-P1n+*Yp4>3d1HI7v1Bc2%OJai$LKC-xB(0=Mj%&ZW03vv@w4OfRRQ9utmB) zN#eRPGuFOp@6!&AFEjhTvYfvGtTXLgpFvosa#Js&HRAEOP)Uyho;kxMzwp9VL|^VJ zkZW96SF+K*xshvpMM#dB;3$=>0?>cKG4YWwJ>2$MRLtWp$Abk}B(P{Z(=Omg;4LD> z-L*gnY)|P}q_rfugQ;*%#9-U@b(k`{zA%vPLZ{8r&4?x-HB^HCz_(7j2-1g2hg>^t zhEqW*>Gik9aebqU(}%IN6W=b(W27(UMpl65&U=eG++>DjWC!c)^ckvCR#8gNR1u_!7FV`aa>t|tY=VqFF9MB6<)%Ihq#3|m zryZmoj14KQ+btbS+mqf7(}2eg6(6)bGjv3y7*3Z54_B?;UO9WFr>S^p_HF?CT|#hZ zR9;EWeQ6-uSp3*X-0{~vu*_+cc9ve}=8z!ORxZo_LuBFI@Tw^5M$@6kBoRhR1FZ&@(OXX?QQieL`-3H2?IVxwKSt!D0}|i)5|Rbusx+%pbHVb7-kN7PQ!Y3T_$yDv}*glgi7kJw@(YWhqkiv z+T2{D`D|fyv}D~%p~oYlVH!F-tdv=KwAg0d8t}rnL+TV@csAMyj45xs$T%LTS12U` zgz6*5jd#8#7cQySamd}d(TD-LHDMfawQ@f$dfZQY3OOS8F#+|+>`N8fFO~$_`MSuW}nSY z-tNwjhrpto-7uJDJZ5S{r1ZXb(-FJWOe8`ff$hF?RUZ56F~+LV(Amui<*=RjOI5h5=F6!R5elWh00NyR4^VbkT6mBe{<6!U!$S{S zJ@Q@)TJX&aK{Z;4A$!+{vr{ZGE}LHVT#(3QpjE8M1XIPkT{pap}Z#Y5N(>eofp zCU{x&!$5k0QvsNzwXuP0t^O(xUgk*jd}Aa`{7f zvoA(EZq-~YS;L*_0$*P3?(Q}@vDo;*Q*BKY=N=(hE3k^-P+Mlvq^aIuPiP zOe%oy!3itDH+#TM>Wx&?Xv*~p>dAxd34c(ls=O)jsWVGEK1tfIZ@i-I%GALgP=;17 zS6Nc6sVukh@A0^Lqk6fZ`Jw_kpT!{DEW)Kuvbe!mLu(?J5p#9{MBq4&FUf52f{*Z( za1nt9xikHURV=Ir35Z!q)kN9d#6(Bcn^#7hb@5${2&|>@yMVPDEDhRvEHvD8 z+-2v-96!RT4~8wzYTI-K3+g~05R}<$V^V>0dMFznfXTDo^Q9J#3-KHz@{!3Cn zF_NH|u=+IRxnFE=Tabu5T|&qM^oltSLNDH&>Uob2>&}ERg_l8dpj# zLm$$>4HOeaUm7!&#C`!8UVzaW*O2V7hNf_)xUi(=MX4q5pCOmB@>GNKdY%I=h-^&i z^X(Hv=z{VaKfSRZ7lJb!)=0u;$2?EEs|7BJ=#Jzgqz=?%Hxt1!d(9EW>ba3NekP_P zr6-*k$xGQ{A1)W10dBv-j^A^N^m=tqozS*Vu>!4I+aZ7Lroaw=)jpffES`aHhHTI5KNbt0~3sT~RR}Bsb z6*t_C4{GDQ*55nDUV=0rqMgvZ8Iaf;usrJ3<~Qe;eRa1HRYjEvWALdBL+vC{Ya@#JtL z^}J4(EKNMx)`&>q?z_D+^}HA*csmfs@$7SZj0;Fs?DJVM`2IQTz&>@*ya2MZI#7RQ ztMI~Q`dpZ9e9k_Sx6c&5yxCjCdI{asl4$D(f@IFFQT=-=obc`gq2c}`ta|1X`9AOT zo!&v6O^}0gPLjP7o9R|{ZncAc;9_ewC0Fj}l#}66rcT$Lm`f2hFkBLjilyD5v@!mu zA#}<6y+KG*vt(ti_R2meggdYRF8kU?lZYn^F?0AXKUQDcdeo)^OB64c$oH9r@U>4( z(rSxB2Q0B3>jk~r0|k0RX`X;j-X|)AS`MI)F_5*QCR{`~Jev9XufW0#mC5>W0bI8b zH2G&tb}hkuK6_QHS6wW&(Yvn|&0_iIxc!pe!&q!|<10%Wuagc7+#Q4uscs8iOk$PD zYREzuvAcVg*OVlo;`^Vw!-oZ))$2xO2*oqqcu}@O_JbsS!5{B2=&aD!DYeJvi4m5= z&ZvIfE}(>)cOm(jAYwofGVFwfZiF`E-VS>r*y=YBg=f2W&vxrUVedvV%`eH5*01dL ztb!oFe+598-no*AexOB|KDxp~m_EGX*_wXB)9qc?8=1!p!&G*Ip6*fU^zWx}f2P*` zVTn+)d!?!S6+cAF%}0x~GkXBn?G4xcnOyhd*Uh%r`Kr4&w#SaF7s^gOFW-L1!4S%gsa=F?0~o^H3zu+c7|(Ayfvv&2yfzYc;F<*Jp1V)D z5{JmZKH;h6-QrLxDa{Rp6hn3p7gi&c!L)qzm%tn9`;PfLc1Q^g-h;d%*Q4BF0~F_E zk*29nNaVjp93zmtJ2@jBaES+TBivdMhH;HYzT!gB6~7=W*j-$we28V_4m)v*k&O+Ybl*djAc@sM z3~)mU?uW7=#pxp?tO_yRN4oCg9(6lG{U9fkP-G88j~))KQS^`8dlp;O=`Zy&KBZ@Z zHc!{yE=8jlNqokUdLtI_mh_j|Y{2K5A5~z=lsVYs_N8!6^ey>Tf(_i?6P92M>;(o% z4Qz>32;iu-$S~bM(3bpKPikdDb|BPqC zTglu!u#cNahDDm4XmE{f>lj65(ok*Bm_QvVx4oIPHo;8GAwusetapGca|B=2f4o99 z21c5d#sJB|6fEZw#OO#N3*{HI^rdyR@%Eb&PQFa5LZgJP_i8Tay(TML7pjNBlaFoZ zO)Wnzd*2H-9x@xdM5x9O)A@JXP#WL@l}5#VGZR z9To>QijG4R_oEo8BW?DVB9~BRr*b45Zd;QTn^=kmZ`V-B^$S$3Pqrp8i*5KVGRG8Sr1 zEWk;CPc8r?n$pXlg`%c6Ax4{6yW_pdXxM$euqz6F66Th)V0T!4CW6$>Q<6l~_~F!& z;e3x@NGjN0>EYxr;&mz4px{Jkhauv9nOdYDrh)X())B9B90y-%tn6=nNxpCC9Sd5c z0Nqjit?=M+gCf03H7p^Eh{73FtxDb>v$Xenmi6g7TEmO7_vO6xDc9yPUjT-{yNIy- z6xim9;FN#Ki?!Q1$uZeTn~a{@fTo^>f9A1MPL5C#YJ^K1EXsjU&P%>=ob|6;(8t|{ zF<(-iH&X5u0d3RXqJ*Mq&l_s@v&)No53J1t-Al*a)rbHsugj`rw&e^P+p9LaMzjrM zjE!@r?S9yw@q)BJzBIyJNthO`dtI)-oLnR1yiD4(lU)>aDgF-TIoPJU*@?IvlZW*tYcm=gq=cw>V^fe03~AuIR`KZ8LC2r>SY}g!*FFs1S-R? zoFi1UbBDP?x-Z~4+xfo+=LKuKg@iYL{p{X}d>A$l`&xF@0_k;t$~~IE_n)z##j&g)gD5Cm^%SbCeS7(!1**P3)mtyI#-!qdVl)fc>dJ5_gCwYi=o0b>#pgIw%2%MdZJ5=5{d&1q zZAS?W{zou)bAxdEV@_^re(;K#Je5gZK)#011RL~wOtLJIbTVcP6Ru&Y6m zueB3Ab?U(Ho4|Z8q*tYF2mQJ?BpH#OBht_}OwSGY2a8o?9Vwj<8;`dHcV=ard=F>A z%XQ@&+9HQoZ9%0|wj^d3Bz8b!t4zjlKihb~nFg~b^|sH+g)M=RZ&O!?DyH9(!!)@B z7OKfCQ^4=FBp2)$7wmB3vqPu2V>+D{F_>VQtgEM4GwEG~0>M;!1FHJ| ziK(=(LHlMjr}a05W;P)TGGtH2SnfeWgp|v)jxe0~b5NiC?^F-qZ`cZvzk;3C1fKTG zNRU<%odrS$QUsQa(x7!i94@sgf193ihr$O7N(N^0fA<0NAv7wyg%;t0fqz27`Jvw_ z8S+L&=)!9((J^9n-!Xa&Q$}a9bPeA4QJ>-k4m{5<TaLYQ^m($5BS;3f)$D9*FUnUZ$m2=>Z$xxQ1#o@zp2S-)_qq8Z6gtUgAEr7f^J zb`MJKjP~zC6LVXYkk&90WD6R`o8B@z z;`Hg3?fRYV_N&~^Tjq!ZfpMQZ$~5;%yem_UPdKAXLWwXPPm!t|-H1WBD-Ny?J!imS ztC+EQ{xmC-@S&K=LFClPwdnzxV}HF;(*`TKEOH-&Yyld1rX2jzF-}HaHKfTM1N{IR z!qf5asJvo0rUUt=XnFwyJ{0p~Sa6(E`)#0MgmNth$Pv?w;G^~+_I|pA=QeR7 z4mO{W$5K>gJe;nd_MJCCP4Y=G?a ziIG^lIuXMlt?bq(8Kwq4NxKwj`q7DY7SI=ivTWKE?c`bfHXdO{(>1IC+6m@*4rH zvd3IU8iBPORa5bejr>u;>2T;}vK4g;Olzmfv-Hf7$Zst#IXgWGM@>BN;yPlV?=4ty z=1#QZzH~Fm>Q>CniJMZ|7->#@zS9+^zifY0tO(9j=Lwda~F; zz*OPq+o_nnX;EBzYymR^e!@^#u}nlaR{-a*gBEZ92vi~G2Sj*EQ7hZTwUCJI);5y5ppKQ#4x4375U8>b5|eysgpkgG1bOiVQZSbMP`Py zZL`znm(54NFCC2D;TYlYb=(P3@%g5>a7F${>?RXeKMEz&EC?yS6h+yNB0benk%X{~ zGRoFQS_>D(csm9R%OnG0hibrmuw9fsjTZX!vdTOkh|B3%H!pDRxR9nIdmmI zq9?{CaB|WPC8zVrw&`q015P2v+4d+EeonX$4Z3nSgYFl{i56l!@MkC)9?*{;=*VI z0%!l$zoDB~h8*aT+4Gne8PNCZh`kj_%}-w;RJ!(%k7;|E1`Ow^xkOw&fC5s;Nyxm9 zxwyfCdWELZpJffDUsJ*}5NRFl0(t3z%supwSZ*s~Px|%B4qMS4QvyrvIYP{g?iq8(IS`--u4scauw%4aX!%NOq)evn>eXNs?v&rLkRn7VVhM4|S8!xFJktW3?as7cSoDM0 zJ{zh9H5JHp2RatMc`>Ht3Z1e-*kkl74W=?0PH+-T9W{3=vS?owgu8&q(b@T4Seh+((#Yq4%BZZCvPG4M!-V!3TbYbdb*d}zFdnQ_6t3#^6uy6 zF4=@a1KTTEX6!t$>M4sV$Se-msNCNb#eVYsIi>l-bqZ-#2R=teM*;#n%onIrRt2KX z<=1&eLC>KeBHtC!4Irqy{?T67P}_ZW-+gj`V@m?8$_GlRRvJB@n7=_rqZ1p(BPNR0cYcdlOOw6?_B+=t{x$9hP~w0cVwMO zQL0{3SZteggYJdHe4;EQhKs!)2l8=>4CEEghY>+AC!5`VK)~) za+YE2W;T%|bAU8~#x>C|@@dt~pHLf%8t}hIoI#hcs@gu`AXaY(%BqFaDtyBjwZ(5R zZ0cu|9=Zy-ZwML>_@ls`gOcsdGYq76$}%Onj4GAKv}nS9{+ni$&i=M>RZyIbJp|n+ zV9K3KbbZt^BYe{UjNZrm~NN)nNl;00E35lW93(=?&Yj>CU> z#0gM6gg{zNTRPotOgFr_JA&~_5}lYU^lLjh^QY?zVUQ|^8~V%|(7f-ULj+0o}5N#eaI9MKzTXdc}m{NWMkT$7QHFN!vI^~ zb0J)dk-7t)K<(oai|{j0cVR@ z!Y!6KOTq|VaLlNk>9%`9V}$wK6$LBM8>){~64S0Ew)mmajsKk_WbA+($0K(xIn{XCd3S%WUV~HczTk0_5Bqv&VWhhzJ zybB1Xc+q*%_h|18RJ$yQfvKco`XEE+wbkmhUf*y_T>oM6G6ZWH;3`CxdQP zC!3exE#OwRkWn&TFIY3Oqmk(>+coN08{&e|PR$--#vTqaX(8@x!irmPCCB9Ka67`D zxod#5W5PO6%q;Q^oYA_n?+o(^jJEOuysT)kbyO*97h-&~hYD1W$0h_(@fL{rGbo8avl?z+ zOOGS^4-`TKHVe9AgRx;PBoU`dW>^rhDSG7(pt|etkr?&bp_TJD)lhG&vlmW+>zQgl z%<@iBh+A8HS#sUe6~ve3@252PwEC~$1V)gKkhTwvOFS#}nr_3DcxPh^lJC$Zzu-mw zf-gJbUb4dHV@i|GveVe`kFynWI!RlUr{s=L$$~ye>J0C(K;fF9q({Bk5eVoofzCN# zXG-MAo!+U<7b$+M={El$6l2N&p$_x-qi!46Huc#UK5Q+&)eD_sZAZTHg$SA0m9Y+4 z`Xx^7G5e;5_|_k&T8O;7rF~e&&_T^uHb(sPioD+s>hQ;CY{~Ov{_?e8ap^4dHm|H@i6^XqHCW-Z!}0yWY{f25sfzg- zyS?@;L&;ZDp!UsQaP4!(aBkLr@zY0;O#sW%e3QX8><8UO%1aQw=Ls0pw>!O4s z+X{>Wkyd>#*DhHP(jV%C+Z8<2y9IX!oec+~921h9nIt)I(`@qy88Y`y(V;=N&L|s# z2{)>J=ReYf+s6MOi`WG(`vMC_JhNQwlcqYR(Lt9`s9EPfKv~(esboq3Tct{fXfI{2 z_e=2lm`mnuo7J8ut+g;d8*YQ@Mx}ecaCp^hZzqn4UhHUgs1ZeS7>`JK1L4^{^1nJ$ zh6#X3gXW!hl?L-?{tZ>4ezk(`F~hZ)^8o_NDi}#=2JEC#R+yynoGGbA&*HRFWT|BD zus!jFv-`*{<%F|6lc%rZiGG7z;8|`j(3Nx+3A)Ay1X0%B5n-LvyAY+`$@W(Pai=!B zyMVaEG52_x4F@#FCRz;ek_`tPTiXUn_V%%8C0!exTAVQ%xeOnym&ZnZw1Ph-zvXz+yw(qr0P)0(KHdq^;88 z;%mh3lKd8^4t=V;AST_ zj0sE(`P&I+P3)KnO%3gv$9xlm-T*15xPmWEV;$q8y!J%-DRk|>&3A-r?oOl)l+zfP z)ws<>$g7skOrfhzXz&}|{ItRc*B(nSAals}grs`;LPRTuBF(q;bHy3y%Xc$-2K$F| zBDJABUHVKb00bKx1bg=K;_-)}u~utm^p+QEzb#x zb7!bT`&7@Y9Ja*w&&GfBt&^;EiD=6b8u;c+G&ohZ4G3Q|3 z{t4O0OG|Zut6HgTk06{ZI3IDC92+6(&+1B}Gl_5#c@q!dq<(5qS+#dY+UAe2w7~e8 zug%eq#J)kcn1MbxSKA%En&?wV&@`ul9-E@dY=JM{_=v4x>cpmb+dQ2Y%ndnCK{jP; zbYrTb@DF}*UPJwa4n=V04fA9^0b}Yx@FW=>!ebP(Ydw_`@iA7;ojD~x1M>JF=&)76 z?0Heb@2)W=q&9WV32}aojQEIUOwcG5b#>Y7aQ9E}43(qfOh{{{&mJQ8je!u?Pd4V< zEuGz=x<;Q7;`vij-2HNeM7sg$)n`oZJw%hxIoQ`Ac#VvCeTN>hMkkz+DKQ^dxY=Ao zr9Ld#Gn?f7svim?=RNutt*lIqbE96Mfb-F$)CG??JB&Gmu-Xx?g zHO7}!&KiRpw-k5Q(cP`2_!D~lo|LCywNyo>1-6jBQI|xmRAgKD9>1*q6?%qn=0L54 z?sp^7=x^RY%Ykd7A}xl8$furBh{Y`Q+jJz2V0gf%@}{F}jxiq_CILl=lKq$|0#`7h z2gk4rz*U`MEm8ucFGI>8CS!qC(VW@^FYN*5oAAH)z0Y8u%BlW?=cTa&0nz`z@DLQ8 z9UT9WR=s4*{x^%NjT_n%ckI{0S~7e3$*9&L83J%S5(=gKA&UK}rKy&@=rL)stqGLU z+VYluEv@#{__>sfjKbM2NLksVB;wZcV)kNds{Z2gM?a$W)_=X<_h#c;<#PwDets5S zyq@-EdpU3NIeWZb3vPM}nB9Xoa7>vysV38h$AmBfWwMd_*e132S-f=#0_|}&F-&C) zd4za^FMAqO3Ms)2HGqbJ&xrp*@pUu z4~5Qz8&n0dMOb&vq=@J8K0XhEK@YxULE(fDTdrmVDkIuPh_LL$1B!610V~8YHHt)H z5t*3=n@g5spg7}Z5g`L|5dy#|l!GH@qh1sYA5;l>;_&8;umW@!4zlp_#}2|NJUMtH z{@onr&m4H+;W@^z`?Bwg22(O3&Ozutqj7z z;q3**EBt5r$PM5#e}oY5nKV)W_{{x)t#obnvlDFm z>)D;@2ifP_DC)tx)@Iv~*0np^Lv3Wb)~4IcbXP#Pum^odrYu1+2?o}3nS1_GaGPnh zsJFwW>;|<-#)4ItVbhRVHQEC2iX2(ExHHed*U$)e^5P{VikD5!%3@##y;Cn>4=i*OVOVHHFRXovuAMS2}Hrt+}9RXS6oBbR1 z>(+HXsC>cP7*X9K^lYNzu6O8E2CA9BTxE)J8$c+9+TIRy*w;}KNk@yC+Mcj^tVnW2 zEiVjg=hJ6_npuL7FowUGsJOJ)Q0{97KE3R&DrFbj2d$KT{ z+JH)iW?ZS7a1^Cg#vrsaGPn}5ag%`9@_DpsB#=#aE9ShaQnevnYw4{N@D^oy6M{Mcud(Np2M0=J@QS)F$9WEX8|3jX zbLb3R|NAqNSqzKcbfXxXai5Uipz=TJ?#6`K#PjTLh~J!_3I_OFP2@ImnZIWlGW;GpS0!6Z8K;%)J0mg5XCjT$?yov6Jem~@DjqzLhIP^A+0Q} z8m=;rxM*SF>V;WYbMT$9LAI9;kZ1AR=`E1^n~e1hYANR8pgNtAbpaIwn}heAnUZ3YdGE<9E}j)9ITgDWbY}zQl-H z84}^>oFI#QeJCg)gdG`jF_+VA?1=5c+F8J61QD%jTAvf!wRup{1(I~2v<;;sbwX$j zx$5|0D%AtyC9b5R+?O*OckahPh`=xh*Am^^-Q@*Yg||>!T|jYQ+SPdkO6ykEAX^bs zyW}Hh*TrGeSjNkLf5DHzfY;znDZO(M+l93Am)ea&=7K`Z!)yirW2DF%w&{Bn>+`Ac zWRSzYSJ0#>dq`gVBPz-&h?Ca>oYY0oghsuFK&=S!@q$z1l)Bv>>d%P0kt*W<; zzC5hWQ~o=OEIoPf97kNZfAtD&Hg2gEcM_{Uig$;SG6*;Wk!a_Nt;F8Be;E-Nt)YZ zSj((2o5@8dPg@J(4&K8uhPTGE<1o>w$3&g%Sr5o+F|XAXynwapVZVF;0jRRjs$fY% zp)>8~CT9sz%u$VFXn=BAVbZwo(Woo6Vnw-wM=8z9>ZC!ls!*q&!W6T$r_)0wRNeg@ z4OMf$99h-Ppl8*gJ5~&Zn3w|A(axgL!AM_)S&rOdxM-oiG~p4W2H=b~%{rS;r!j?4 zxE@;t{&E4|BvP3&L@IGbbw_{=+i%06l{|4QlRPv|C}2!e#TTkxb7E!Hqc56L>KE#g2nzK4&+$V_3kplhe`y-tQ*cF?7zjfLLRfY> z&Wu?)o&U7L1s3)sxjke0jy;=$I?=Ic8ImQk%}{mHU#UVxOu$I5%D=3v z^p{Jgv_g-RC~0)14cHYA<0IEEbD6A=Tv%Iqq-H;QHPuWJ2`jDm;ICn(>xUxh1%8SD zz3WnC7Na^WdI`YDl$rSwFKH`Gwm!S+Y}6B}NP!Zu2!N>grG? zRk6QHyGiE~Y^_5PNot>Pd$QUjjwK^tjC(W8BRgRyN*8(?{(@sXH-UHnxDc1Ot zc)%4Or!9{|LcJ?E@}X(xp;Q8=RT>HFBS31~dDts(jnaCC;)GKE%p{*pyDFuCD zM41~R6qRd!HJz%&=q%pue#MKFP9PDpd=PIoG>C<9a8=mmDBQ|IGCKxM@@pPFW`!j) z&iX(RU1l^iu2js)g>4kr>|iOOh$njo6Vpm->@{<6&B(~U(7SC^2m*~>WdlQ-JtD&z zGZ59!pwedVg1d&$Pmqbrq`ES=(^y`owdR7ig;#4%#TE)wiBJjIx6ZeUrGf+n-(CUB zvRZ3(luEU#oL}2TMVehyypNP+yotfItSVyKRE{>MwouHff!>qhS0r~M@Z;@sOB^XvozCA+YVER={XzIf>>231Vw%RKhE-+l ziyL{e%A*QOYEzwZx9rsG;v1&ZaiaIiT1jFe?AdK%l^8G@EH2JA9g<2+%lH4N0&67q z)u}q$--MYW26LeZ_?(ns8zgNeLaLF8<0jp*cdD_sZIv7fG%^kIhb4cpcoI=m&ahv#BA0?F~zQzyURwch`WrY zGhxEY6%r&}!0);838d+#=IfcRzo7G>8pHzU-oIT^BV!6(j8aMiGUhKe^ua#7>Z77q zNWGVJUDO1CfTdV~PP0YdUKqV-gd~DYPo_rFp!6^fox5q0zqt$fy&9-%ja(xSx+ZFQ=}YkiMvk7XMSUiTCN;(GHbci5U!~_rckRfB zzgB;ky%tm8%w2NilF-#>H293kHDLLSGU4#cWZD&?_JZb6kHs&N3d*zy*t)r$&eH<6 z7Fi9weIKfmFb#JOWw+_}&{ezOUs*%jB(LDj_!w;hMFn?#{g)knlb9cJBdpd%_fV8d zFL{ELzt8B2m?J}aSuz`GH*+z0*Nf63=mSq#u3xnRaE!mc)`LLb4OqJ?S+?XD$$_h> z0w5{2d^Jb+rOl0T4fHO)o0+x{b@(Nfa2%s8+yJj@(q=1JsHK>+UGg(-kv4+v-Q3Dg zzSl$|xb|e9I;F1fsO+fsZ)L`}an(myaHqiu`Uw7S_u=gml|B%6P%|xuRz_I^1HPZ9 z{Y&uQPCFiFdUuDWxeP|z7?P;8sb<=K-`sNuYWpSDD*#n=zG zwgynpuOHzHHA>cV^bVkRbH*Q$QAOiM4**j3 z!0kRJ$#*5B0`a#9B7g8}0`Du;p}_M#BvQa^2qn@_6eOX@X9dBh^=QDKp>^VK7esra zZ&<{=pjSq|SN6!C>5$)%uMK>!{-Xg$Lo1O#YaxW9pB@ChPn#d@bg7KjKQ_gf-4eQvo% zMF>8bbmR;au?Llhi_q`P3fFI0kqoYfwvh}DhBQd}r$ahO`@Mnv68jsxULns_bo~0= z(EUn90F&8gD^&Bmz)g9x6Q8d=ZAKQSA#{!B#D5$*Yg|^CA$I%S4Mx@=wS0OOA_Te} zPQ`JEZ1b&3EOy7v3{d?f-M#uCy})l@=y6ZALb#NQ%ZGH4A^Jj-WVp zht1=g3v>5=zaha6B4b?UTFgmqUGYo{JNwCW4D2efyl(p$UnxT)(`sX+&ZN6?wsS56 z?(M8!;PH(72PJZ50D`e&h66vKbWCfF)LehIb9=Xc8Pez8ST}<6l^xu4;VvOKa-Jhg z1o42I1f1?xN>uFVF4c!TC1Vf>UuEpy{M*>#Tgpscn%97JdoKCQO)%rzdlT}2FEQF< z3Qq#s(0aJM^x5Rv?VsyrUdVT(-IfdoVE)WsJU3XqT-cm$zHQ;t?eh%#!YgbC&1`)g z56!)EH+{ot5A<&*bY`C+q1sp)kC1qPpN7&)>kec6g@lG;{H( z>Ws-CZ+1zF1ZUN&WRRmI*II|tz1RY(n zu)Uw$#Z73w=*RZNv0T4%MhOam+Tw0p+6Nq%uMc4r zn0DWt&Q54yV!3?VPnsaS?!}!MEvrRLXs0v+nhM>Gbaox4(1>r@9b8q|5ZI#zSt{F% zWz(@9@+D9CU(k^0OvUHnjsID{13 z1MS;Ew0b6B-x)fS7z_1
Hv%oYLr%XOc5d@3@?A#cd|;WAohSFtzZihDl25czp; zeB^izlp74^m+p#yJw=`URG$ns^MQMT=iS1uw*ca zCt7qsxNW`i44*#;o~O(DQV8#SGqcDK@Y6ffv z_ywyKXqFf2G>+Z<+^2eZFC<}M(&&9Io$oE_C6kckj0AEU;;}1!ba&d$I}#BE|SC4U+{5#bjQIAl(5aIe`p8E{>}9 z7IdaMAQ@nAa6Dx}Enqcp08Ay4pccgf--~Bna+p+ftwAWth!V{1fy@@fOKUw23vw0Y7YzvceofZiz@7^jc%wk|<2 zQSEws0+UYoO+EC^5%I(+iQ%9+nDfXfvmg150o|(>kOQ_?|5sU(u+Gj!(ar@k$WQn# zq_nf<<*M|BNAe~IL+vr|{VMsR=P`%Ou>)kUZb0P9qp|&WEer|WuRD`V%b)TO4-VZ! zn}<9%N}ZOfxehhF)`v?1+YHNdcz)hKLh>qe9`u9}e_B=GBS0vHrpQ@7FUQ zgQqn?|I|XX!GVAz{y$#Ns2kb3nUVc(VHj~EJ1bi+MQ1Z}GiNh<6SM!Yi|bVN6jlt; z{Pnl$^r`^;91*h$tA2&3V4@rlGohwaXvtyNkJahTsjhYH+bGvxsC(W=^s!)(;=^qN zBfpFCJ8~f4C@AXX_EucxS5A3vyc9m3J}OXwR5W3PM)km75)+Y95|x=E*$gPqr*}?g zK*6zjj|Zu-MR-~0Nr%5ASc-S%B{R*C3V7Owkmzmgy24NEtld_&^gQD1M%ZohSgfbo z;X5p?@+~~!#+t8JS?NdYGQ0oPBCk)X#vlkihM#X-)IWqP_o&TTZE z`GZv&WnI+Uq-^B*UQPKpgaMhj^-HvwGuC)%v)Ejc|89$8Bu3Or!o~_)ro|>pdW8K- zn`0CAj%$Sj*DHJyoLqW}Mx@es&9-jIRVpcQAK7f%?Jqu5NbX7IC_uP!jqs8 zf6XBbVNYf>7jHV#g5s-#Ho6n9k`q3sYV1LEG`K1Gu__%&gE3Ueu5PG+81pofqrVLt z@b0~Z7g-W-%T?Pt4_U~2_NPT0T#JRy&4T}P@ezMRp!FpD={j|;#*$wc@ux+s@c4GQ zxmQ`XTyb=^#FGSLc74W-%Ht-&GNYhw{9QVbDXAXr2?y_ZLO2p|s3-0ale~T?-9ReG z2JyQ*@#UL;WVU}@OjPlL#1o>3EIz3tOcUteqqvH+kZ|Hfc*&~ASLEOi{{3>qZgA|5 zfR;HGGC}uyEM+dgW~$?(BSIvFi3-foOiw5-x5z~wFq6YQo*=)^yE^zX75Q;6CpXRf zCI2_IS2!n^N&`T!H_(^|LsUQOK2HW{KPrMhu>w)`bos`(pc(6;c&9T^`|n*TiNE-< zx8{n-$Wm~O6aFCo-O4FC<}h6#fPlK;fq)qQ|5*8dthp@X%>``&eZqh=3O@=QmMNk! zj(L1%2?y2Kwo&fJmJx@Ir)74wKG9iLyoZ#x@p8I_7I^;PL;iAf-sp%Y;%&2XS~aZt z_~n!QjaK>X*)m~GHLQ4Az_LQ%OMk%s=IMg7B;eByT40dp8&yCy8IRk8_Z(udr-ZIrWAA53*0ef}2jctUDGjS+U#GoEWW z3MY>Hq~5X%ot2GhHoKOydepIeG4~p4aqCfK z;q?SSCPY#ba0O%fZnIdF0o9xP1UFxKW^O;DmDo)~grE3`OKqTQ`V?~BY`wSMev+*7 zZD&2v>kD&mJeDyG=ba^SoOH`OlQ`Sk;#zk>>ErW@$z0?!I3>tn z#)Mx%t)cPL=z~pn=xm?j8)-MSy;w6>}_`KVFh%^ci zC<_a|nS0C6J05=WtfC_o-nbo6t0r4`I+w<7&!edxZHAqk8KK_3MMOA2yWcrZM~KVg zi*w2CQLeMhl}oN(f{b?EnxagH*K;i!P(dLoQ)$0&Q!wdB=1Drn|L}s^XfzyA@nyhb zSjYkSNVm{nEz4~s@5SEE(${(UiVRJwbydWl+V=&YAkh^onS@6@`I7+@J{w^mzF2=~ z5M>~EwOLL@1?_wSy|-{SxZC#W%sHd%C%TCNIRaaWks8ZS?z=(-xh8mFT3CRb1`yJ&KlBqie}* zB0Xb$_3TCZEdxL3n8y(~EHaE?S)1jbSd5%XRofwxhM0(Tc$@sHb^6g7iRgODTIgkNs^^A($+bDi^nTtr4j+AZU?3?LpzBQfns_%3X4vYqFDBKRQqSct;Xnf@c{jn zexiGHD*s;dWtYzQIRPWK?kyufP*{gTCpV=I{7C_P*<9LVw>W-Wy3Bg(N3N1424_?a zGhKHAUFwjW@!Vc;96#WJ6We7Q4V@+`%SZBTR}C%qi0kv>x62g99L;>q9;CLAH*rmi z)^FFc@bBhKokDT5_ zGCju&zt4Tsr(Fr?Z}t%I9MVmeLmZeqc~8Q0(e-P>Vx$DSVAk5j5|-v67}@t=bw-_N z=aWV3(|MCIjSEc|420QYUtj87@#t)2-R+p(p0^1&oH7n@V&!>4{}IOs7ROyES{nwTN>vhkvqs{)YYx|5)wL8-L_Ez1B+UaP z>f0*!X2upW)yq#VDy!7U+sY}(>FbNXD;(ZHqrDTEOijQV*F;=VAzXaX_2d<{tjfb! z$1jC=eBcG4)RU7mYRh@Hd=(m%D;%#D(eTf$9n8d@QD+xkwSq$LQvdEPgx1o;QdkUm zR1EOvd^HMt2_!7)^YcgRRXCU!{6O!49E3zqzZaqQiBP#nD7_?o} znn?Z12O?Cs`WZnhklg%GlKwW8_E%1t>E*AxrM^91eYa4)D3H5zsdC{;Nq$iM^b#|G z#6-i`FCMlSQh~wUcY#p|nOpTQL(`Gw<-oF|s={RpIX-tQKo@I($ya&>@l-$Hx=ld; zhJLMp`GLEn{d93GyyfjIcf`k|=>_VqynWc0JdoL+f#Dp~P<|<_a)j-zyoKG@Jh*(7 z-)u`^PKhHNm?@m|WO+MxdF5zwIDzD|a&2SOFEmR2%{s5H{elm_TR3|wheM!UbfCl5 zEPUahWFE%GF*nP@#?8}ecFAFvsgG~Y%;vD1zD$qPFNAx&SiZUK`g;JOVevHyee09n z-4m?rgUwF%J7y#NH#2V&@@|0nVZeVszr><~Pc8l_Nn`#K5&WNWi~md4P5nO_$p1O6 z(lnxcP}h-v=#%^In>U*AP-VGfp+q?h!!}xn%GowBsw!=xO|A+}kp1-N3VJH5LOYIt&3zJo`b36#?idjFu7-R>c}GiMqCE>j-2l`^ zpnHsmpeP&%rICozuSf*-BN2DV<{y=jDz!gsjQkXbM<`o&(E7g0qZMSAfd@ks*bmtH zyd;M$Zm)R!j0YIGNr@9$$fsnzXckxx;Lrx68KSbS&#Xv^hB3bI7)MP0L@#gS!sb?t7 zUlkU!waHWBX{z$*L^(|(T=Z#JERRwugMUdVGJYuoQ&fqotH=%i%3_}5UHrYhPAv!^ zE9A$yh=GF>g9r&JX;KtG7Ga3SmwJ3gj3^#Dz`{jA5Dzf|#vOLz!_PD}aw0eULqX|A ziVJly)yQ2|F6k}DoxYp`(oSb6o2gAyk83R`qfid=MM^A8rWW$bXHI8FAraddcr_ye zp}L}){5LY6^^k@*XLdvNHoucQ>(`?!V}UL^*ro2WVB4q#L^Uy;6&Yam6b~o<@%Iv^ zHBa9V|JS!7K(lmp8|^(Vlq3YKSBY6pT`c&6U4<)G85JZu#o8T^F{PNfF{kv4DJn?! z&_Tx#FGBpN5Dd@OG`>ecA@RlL#S)voM7 zX@g?uA|<#~hgR6XM_#bqq{b0%MUVxIrxVO}!Wv_-hDrQ~Qwx-Rd3fA+g-7Ohdpwke z)oUD&t*J!b;n?%>`8cZ5j~I`h091x@WfYgRW=4xlt|H?Lj#+YVU4ipvrHFr)tBb}+ zTlw;r6Vgvo&c+ZJ{zOI^Oumx)lOL|L_=i-+@(6Zq`d?vYT)o%DGRub#{&=ZjfAC;8 z4B1e_XH?s}(q1CTr(3c3CkRZwl75rzhr?J3vdVQAj#OJ^si~76F0gzD4@|$}GQ_<0 z#0EH$uxUxkA%>E~1K>zV4hvYm$F#cIQ4qVowCL-&dAF>^%Gt|kp_jL`TbtXA%^=g% z$2~GVD!9cF!X@CzO{`Pr5bS=&S;LrgQWYGu2#Bq_w&ybhNfGUI_lY;wi?G-jN<< z|K_K3@QMIJVW-+2@hxsI%NK^tJq09=aQRVN|8ji}wYCn=rqY;As!g8DQco$1Znf`P zhi4XhzNyf?CFCAm-u*^Gl-s$!VrgC=%TXn#FSM)j44+XWW6heu(CRvH(K4i@>ssLM zMbMg{I+YBPVV`uGGn+)Js*E%(86JDCrkK9A(GJc+F;CX0ifx%zdtC84qTVHI1?Wq! z(QEMQj%}KXZKMC4QErP&VKn8lJSr_RG1_8`q5osQoYkY#U|c9yybv_^Cz^p+({33?rKlGe06Qk z$SrDFSI)|n?;mkd<6%`5)jDfGd;IBn%cNmFbyX9NovfuyRLg2A5wE!WEEeoL-s2H?OyY%sOg1;({l_GMN$ z^ntJ9W_k}r4+#aSq)k9eyIUmkNve$262a0eiQ?#FH+BW$QU_l1?$l$~7L2E3+vZKk znb^P3eX?5SNPJ{!0BnoMlaaSww%g2Qxr8Gfi}cG>v=^E^~Ky~vi-+)(GZC&`h;XUu2Z{x{bNYV|TAS`qZI zUF3-Tu-QL>!)^$KYga1zc0sikUimw^s&(gm{Qo_GP(*IO-{*Q|yX76t0>|p=@ z$Own2+9{w4A^E4b%kZ?S{_Gn~DJoLr|qTY67l+1z`cVa5UVZJ29i3 z-(>QD0=jK}rr(Y?V_=lIxF*|Cd?S-;byiN>*`J1vL7k4G19uRsa|maBoibGuL<+8e zO;Ud4U5aRhvg!b1|H^yBjlfABX(TP<>gXG}gIA=UGR(n&LJ5 z;*?e`M+0z2!w!o)#w3m<{u1ja(g9ylTk(U=Lm5h2tT|+ZeGfk}jJ|Y_(1ekC^btbi z;BBGVJ8WK}elX0!dG5LHF3_$SXQ1XfMS(v?X1_9K0P!c+Po^_|R%OT<4cipEbzVah ze5obOHwJpdmv*&At)~7~M+?%->L7dhJK%@D4vVM3^#HW%UuQdf&p4+Cc6h6>aotre zj=AVehb5=Sb0tDh1AoHDO{h-UB4n~;KWmCi9U4}R*>!`<0_(J{ct6qxX1YrG)gg;D0LFb3*C^+90u*{YL5ObP={6EsFgoZZGb8V`4x$w>v{>Bs&n2l(BO?aC&TRVa*2Wd%Pm(0XK<+rrr;Iw#phjpSrX*?JSA4P~ADJA4X{`^p!vCgbLy zZ${wK^9ljE{VLX5pnq;sa;-@ryyVfnJnOzG5a?k@B2-+BT>qD>HW~AOJa<2(C20^LG`^CWnw&q{6zN+0uZH5W{Fr@_ zh(X9?8xI@^-R;+tmsH>CifDX~z<#Q(>w><{`@>zQ1lPH&4-X$t@P#1{baZ&!l10ab zk)*h0xDy=mSj*;z5^)?Nb&RXAjAb1Cx1EaG_XmTX;`64K^-`Wk5&d0em7E?ljNx$V z8gXPH^cu=^a7Cu9<#~J3lU;XeSeunBXx>Fos>cyWK2{XgrblcMIQLu|lgEj1OW1h# z?MK7n5x{U=JxrG=pe4>ARK70Q_4(yqv1k05)00G`V4VF%VeX!L(v z%HT6_TY4+nmT@28e&PFLDLYZYTzLvlMiO-Fis$hYiE*qL*5gi;1V|~G9U%2cRLc6z z0_tWw8rL9E8X^(JMDi(`%0=ho`UPn`;$i@Ta^p#T99{~vRJ|MztKXENe6ymaw?;r}39vY6gWc7o9$NlGW11<^5Yt&zor zle({flaLW@U{JPD1KqUG0`b_Fa-U{i0FANTuPkv$^@`zxnv=Dfr&I zyqpR?c`(}Z8TjxXc-gM_{@6eTdOi|Bi62aN%|<|;ow5&?bMEBkD&-2llBgwzdt2lZ!B#fr{whQnvZ0U)`)SB^5E+=Fw%z%%*hFs zKN8VtGD6H(bX0=ttq(<5Gtm+Ho+4@opOqMjnXgvxQEv9`5k zuRx3dSs^!nUZd=U>c2T{QsrVOtP7N26`_QIiku=WSewfX+yzfn(vh)QJcQomGt7>N{h>5nb!=u=!YWQe#dYoxnEVGEOEsEVGhC9i~pzs8tqg#csDl~^qK zzq&uVk_%(GcIDQKOJ#t;KER`tf}vn$vp`;)@0>O%-g}w3k^j3il!XY{nJR@2M5&vM z2k{s-w5!{@rxeb>FxsR8CB2$KsF4eJH^}i^ScW^ho-5lrV%$t71wI0W%2;iXeGHi* zzL&}-yhNL|Svd9}I8dfpSg@z0qa#D2N3^j8+1aM-zOlHcY>X)iI$X&&xrh*yguucR z#0D+|Fn&wzn|%Pmn@~-RIiIv*G3A3?VDw7IKXFSU%NpvGOAM1{1V)J!&NO@LD~e@* zcqp(SpdgHX2OiFNDD!`D_Rc|~1l^Zl+qP}nw(aiMwr$(CZTq!t+qSLO*q;5(?#_I> zJF&4*k^fXgRb*skoxJy)3mEzRNfawLvEPel=-x!r*caN)!^FeGHAiMmn%>^vp2k+ z**jng82+I+4Bx4*cSv66+k2;P_~_VOhO6W&YfgxH0)r#ULV| zQTw}H3NO|hO-(RBFE)n8c{scm94qdE?ViBcUEm(8HRbEA)+;mRl*3zm3U@r$G##5I zc;y#?BbfVk%R5N41y5GXx|MPeUQ~-VkXNJ{0`Z1!>k#(GL>qn$-S4Eh?uy@^|GvIc%-UaGU;fTt6FE6x1EIW=O6X|eij54AvH794Qxnv zn{=XFXd_Pkiqa2au7_V#YX9RoP$m`bF#vX|sgb}sPgfs{0HMsM!_fGdsqy>h0uwh? zCQ1=nOJ@YtQ57011?{)7P;ypnC2G7b1_xW!+0m7NlzjQbq5PwX-TB_^i7kb3 z$=6+zwL|-A!B(gnR04-d(GwLyySD6Nf*A3eRFp;lscO6Yn&k|w8a6{a4D?w3?k%h) z0&a$^@@hE~O?){ucB-{XZw?`)Ft-$)aGqcq?FN*hPP2WFD4e&cuRFC|hFx#-mqSdZ zjfif{QwcYN(8!`W5~Wa5T^0_hbOz&4*$`M(G{NOg=zwprSc{S4UWKm#oWEJoN{^jX zaX+ckpl3Z7`S3fKU|(^$^e;`Ok7;|SBsTZ?=;-s)W;K$D_dvlPOnlp|6ftbKlJns| z7KRS$SbAG2rkMF15&S03wFP>#MHs|yl3cQd*{*#G^;VcB{k5dY;6ZWuJfOQspF6FF zkVMx*52K2I18MP8vGEI4!DJklzSkvTx4 zFtw!q;E+9@*mriGbSjR7Db2=+k3-_upsUZb{f&Ov$f3NF!tSOHF>^dsnz4XXA3f|D>y{nuz5bRpgfGglT1^lfx~qqXYL7m*0*AV!c*^EzOYRqQQlLAOb^` z0VYkBE&lZvEeT}(mEt>wEIuHD#$_@yl7~UrMYh~%=>$a`1VLkAaCk@g`+CRncy_px zk^wlgCFC4;UvGR~d%Swxc4fZ5U*mcKp7sneg4{WxUEE26TijV>h%iUktWPZ>hY@~T zr7+RB)~}5lszVH}O&bQ6d1+j!-f3X`W2oG70F|e3Pi&D7i6361an6msFXa?FqzMlE zVtO#3=?4S_iv!o>Day~!FYi;)r=+;wfCM$cv(eq7D8J*V*h4t=*62sXi)(z)#GD;> z*BzKLbd_lvz9A+}U>qTpkc2ESTq*FF$9k>jRn%u*4RgZKs++n*VGs1{Ue?W?wb=MgAyq14}+C_EP9=U2SG%U7NcfhCC z8=FM6qkm5p)uz!Elf<;6cu%*Ze9spJe^(G0ch>+i_9il1_7>fqJ$2$NPOsqsk4|g2 zK&9N1q&es;vc3dORcbFaa!S?HmqyiNs5oqas&|Nux_b~EdCSl@s;^a=Z=_-GL#s_K zW}uV&H}RIV8JRGvkZR@t9F2d$bbe#qHd5*=94Rh0(5#?5hnpah%ZT*i?ALRr6_@R% z;Fyr_auHvuMuF;49G>ME4e%o@Eb<&~pekilMEp+el~O1wX&l33%fY?$9fX9lId$Is z$K&J`P&DGG#>hxH5u31xd^nWI@EA&o^rcpy!~!0iLyfHqEurBxt+Z*bqw5eIxQ{VV z%LxZ5Zsvqc&$z7;;aoy7Ax0;B@&$T1=iDi&v2-@vkG!Qp3}OwbRZHVM+R3f{Dmjwr z>Qq@$j4k{uwT93M%HG4#bjhyIRrB1SEnRTx$E`THSEuzsJPnfuC&M-)wadDWikR6nZ3Bl zDusgWYD997lU$`GWl=n2Sr<81+usvlO_sAkZs1}CD(i*J{UG7##nr@qo#!sR)k53o z#F{P+-;&@L6&6#3Qj%kf#(!zH>EaVf!~ve(ZsQU*qDzP-fxRS6YOs)%->JE~onOhz zUy(iBa5O!&b^OhT`#z#}X7OCg`lRaCWYle8S;Wpw;gr^~awz)RvD@%DtReM- zyfWXL%{I07id%sB8TcsXxjDZE7okhk&9Vz40SGcDxZ!t@@Dkq0;0(H8GL=70+B|$n z2Sf(}c8GY4bmZO!P1-Xc8ZOGwvmh$&O-QSFn4rJMB81NPNE3+8I5~XfE<%=q!x1Hr z6~(DVUb7-+kixbKKH2QSr%^5H&7gjH^&*hYmdmgDxQ!4iKA?=_6@dPa4ITQhx5y?| zc~}g7O_hWvU!@^CKxY_99AV4a`(Rj`yletK#PJiA3y-g3m4=-QO-+f)E7yKSU z@&GJ{MQRGe18mj!SykKR}FG`N}Y5O%lX%DZJI!-{NHsY(*aR`V(VtU?J? zQmO|L?uV8m4`WD|IhcZk_P!^7+d787Yk5wrPHoZgd7*#PtNwQ+RzRtgnBrr*`~7l! z)%5tKsWoXg@)RHf6IfIoc}Ic?QYMyv>!T&%w&gnx~VetD_;eqbJ84jju5F z5u4FcOuChQ2-QTB(?yKCpEZtakcJAHAlX3dQ+Sc-tRuc-h`|`!iZ#suQWmAz-zORq zZh9zYIl0j~)pdL_)%=9R&fb8=dp6xeoj@M zy$eC4FT%{V6I5|Xw6m|Va!<;HL(O$#l(K(N^=4|GTRolfkr`WuGP?REHrm-g*lIGq}9k8SP2I1tsNnNN3Ky`6k0jT%ul7-rn=+X$qHViLMW-6h2xqNKY;4Q0M}r*}Jo=fv~IjkyzDjuDz(T=X!GOR9872k0{-N=Ye6}8@uZ$BJp_M`k+PXp8(hR zBQqzVI?XI8txMGaYO?X6Lf>@p46ab(cZm_EVSRLxYr>&N;X?6orpF7WEneh)yS!*i zjTO0)F^4qUCJ*YeC=Ml0Rcb^4aQ@$so{q>gxARF@~{ZL?2byZIq(A zP5p(T3k$pI;PdvZ_7vhjd42moor4&^EauGb;P`(>2{mCmdkgTjse& zj#C<*YaMX zZXuaAlxUF-=jv2^H6(^Mut|f|c|@7kP)uBgnQm0b5v#^iXV$}7CmAMX$-5SEW>qBP zjdtoEB(1yH&qbwfk3{d5gHL0>*g~AWF;GobKIF#xSZQVYh9KU1>tEbKCYU+s9l@K_ z^5l9ERt}GlXBZSq^l7DeCJC68Ky(fHVdkFyP*K{<-vhCJ`C=vz007SaU#so(U&~#h z{O4C&LE&Ar?YwUEFY!ke{dC=otjh06j!z#6O9VrbWEca+Y)dkQ*e2~%2Kpw$XF<%A zIDo$s$Gp9=MV;;`|7UM@ljC&D`)btd^Yeh(r_96dFbXvI2Rf;}%8lmBO8%^k7*x$dV$_5fudH zNzMo9j1jlV&V=_{o|cqP!AAd_I^#QBLu{}oZ*A?~m5q6%+E3!V{YA%_rI@=@e>a@-uUkg-zu0ghzwVjn|L&gsA6i9DlD5^N zKE~+J?Fy|8`sx%e&*jzyT<*n^M|K#UtXXR-r42hqXE`IRwwP?m^hr%VzQ8;@0fqY| zBqAc>(*(63zyI&8Ijeh*4%_uRA&lnUpMR78jwE;3$IJNbb^tL1$)da0(1qANg;_Rc zH^>X}NGCY@F!W)VLNNxTgQA6_2cwCji=r_OF^qy8bOs4{2t`*FvwEW{hM;K<6U3(WgzA)!4S* zHA)lp8qpGC|Ik@1Tb58ABBlK?9l|_>NlDN*k0vn+bORAgbQ&0K2R8M!R%hC>liQ7+@I5 z_Ga=;i5KOz^v}Y}&-cqKy{y?zlF7NGYqt0I`|j!6PL-G0$&9Pn$&TBJ9eoanKkHj4 zgxlaFT068Z!afFx2MghVgL{HFy#w?3G-R$9hPjDZ_v2Hv%d%O>pBRg8P5<6-d)QBI z=zDcU6|hfu=uh&`G|+GCZ!#q(F%a@>S}}rJL|P61kmgV_#HcwLp?m=$BgR?HaI{FE z)bM&(T?2Q?J`XgS;di1`15S2SwCW)$+iD``10cjI2g1RhlmwgTW{JLGA`trsfAk1~ z!$dDcA;RtE{!4S`p|wgrkN6$`bcqCKr2ySrKb$^k-e=%4n;H$iarX*8fU0fjqoyg$>^V!G`kfvbkl zf6;9X?{l`Aq0W%*xwdHb&;Yj~{0aUNBZ|6&9&5-%5GKQb7&>s)!cZh|42b6HJLg;U z7LEuD{-wlFO)0(pt~fRgE({Rh*odoVg05vpTQCp?4%#nO>)k}YzExKVq{oJUshoFb z!GQgDSV;s=Kr<->E1I>@1OOq41p!htWXymUDTa`GvOk`oySgvj;Hrw5tXhH&=vG>~ zmC#-&cEk`E;`edbO-Q7!Iha@#jco-xe8g)Y#$*Q}Ds&$ML4DO!sc&}e3X;UUQ5DY= zxNjc)A;c}8I-KloB|%!N1D>K!BnLelk)F8GVytjFD~lzxrvJJ{bDNem;8@%naRev& z5#iV>cidhr@*#}4;j1mA2!F0pRINJi!cY!li9`TFIRP>4Q9hGXo?O@@84LbOq?1CF z8Xh(vo2x+Mi&sqZt>7mTOP*fAMw|otSH_kHeq#mDBkw>Ju9?yyWB+qL7^~PPcFjqL zDG25kFET7yLzykNvxPILo6V!1Gp*Api^{7Gq4Q53PVw~CdMDyPKVPS{Eqg+h?0I`b zf(6P`WQ)U});+OK7m5avbfTFmyFei=6;;0C_p|Ka^vzJd7Ij5#cvvd2(`-%iKV-cF z2@f&s%C%i)CqHr3+B2xsresDuJKD3jgC}y;?!V2WLR=t29jZWX!{8y@yMytpllB1J z6$I*n`K;6S_}cXX#rF&#@e0A(48lP0gW(lb@=iwDPEDb1fIdjChUfs>q}O7&N!(HJ zN=Z`2i9!@77-X-vo-Ji0*Mhi7w?fz%sM)35k#SNP{5ugVnN-o&%bps6AR_^df1~LY z`g9>tuafHVU!^bxvPieW3?RWV2!O&6v6wN#Nx{I2EW0EnPb0|ED#ezacj6f)(u3Ny z`?N_^5|GA+N4lo2+$h>ek~O+yBZiNBLo(u|)BC7g z0FmH{&d=4ntAgzYg(0k*-V?x48e4mk$iO}J2_(ykExK@b3HNQbx{!BC_dD--kzg7E z(B!ZiO{jjXxDxJMc}Vtfk#-{7Am4N$Z%4dHy3lvS=JfRoh&}aNr+4Fy^zf7KgnuK1 zUzt<1?*W4+VSc|>03hcAH+BiQLt8LBBlNS_?rc4!b~v2GMkP?>ze1+p(b@@0F52B9 zk}nwlB(|U2!0aJ9$tMwhIsLt2HJezyY=lBO5RI^(W*;+>FQ^{kT@{jVOwhpUJ+`KU z*l5CC)*CIAX%^`xo3D7EI_W2!DEud$FTTN-H6WvEP$wAJVOV1mH1i!Wq~BQQcX-4X zh9A+s)tfS;-sy?i)+df1;T}EdC+fFWVA(zk0$gV~6S(ckVD9rRezrW)d8 zI^@C}rhP}B?z#Fxqa;DbwH4gInOTJXTR z)Kh6}k!Gt(wtNf$_MS^5#289-lG+Kx+O}1bECVQ)$bvt3Mh8nW4I-V0ig!uNx=1bb zm2mNOmw5P#wn|4FMCS#vd=mX?d4*n9E*d*;rP8oYGYfOQbGhBVV8+V%iEPL zxM?9C5SPNAN;#awz$r!5sXkQGC|t#teNYULn~gZrJ}g$cl3_{@Hl2wGCo?~U3_75* zt~<9fNv$63lewFugJ>fVsVl-|490DFwAV{QN^!Y>Jfx(xD4yHd+T4L~>HIopFWnMC z+%U&W&dgDnBxOTyTySwn%G63T+DG`PwPxQ>G>QcF!&KlAy?yIa4eeZmwE7 zw>pNI)4&S5o6T8?>LJ~8;~#LND`ngt5bTI>#HW*_q`pAG^7>Z*p(*(OWU_A4hHpAs86N(&v?-XJ}qCh4c&kZClkSJ+8xl z5AM_+m;1)EGntYF1P;sVv<#OY8qT`Rq& zbgfgT2&XB!HfLAY_A-;#*G{V{BHw#<0sH)Oeccl)hkox;VMdp1KAO6

GPd{XPho zSFzMd0L}@2JEF)){?=Mq@P)HUZ4~fzj>MySL0CPGeaB3142EL3TWV$4p#$IC@kN`( zQyoS{MdhUb7wii z{(TFNudzNRK!Urzt8W;AQKf)7*{t-!5~OWAXEx!a)rYL&q(Uhf|NLp`k4W?1kBpkc zT(S{^tnO4x!aDlB>j!s=Y4+)u1WFxW76Tbqu3)q48)(Ht(6OH9z`WXpY%7<3YxE+X zE4i!&uF~b5=!;kwvV)v+%LU`rolj}jP2+YOR@pOFz1CBrEq<>GwlpfU(u}Uz9A)xmWw@iEtS!Q;I>eDB&6C zMb}VYmglPy5gauoq`IbC9(lfWu@p7V)1~hd5s_Yx^sCIcRMT5|^4L$wlos&_P)(C? zZr0frp&J@i4HT*F2|*%*Nb^50)8wiT2;dKU1j~~s5)WeEn7Pyt_ZwJ00SF$hSv+Io zU1TM3SX`Q{i{w>gK8=>qb?XqII%T>Jg7W($`YPA37J7H>rp&$%9%nyyN%CU2>5mmg z?^o(Z*LX4q&gT~N7;Z>RyA8qyjn#Ngh9Ao{c~R2Te}DV_xH=}6V)z|{Rt7v>K6!L( zbcqx#tS_t|+}pstiOs?+MoULEx=? zh%0>+ZKcs3ksE)|0uC_nh+T)Buf`~Vge=)rRpkLxSMh^^x;#r#@PL(ebfEVo2NxRY z*J?Y$1b2`Q5~Pdui$o1qh)yyhSdgG~!;tcZJb&XYCLR*J#|?I!`u`yC0fi}R(~_vg zW9kPB361$v5^%depm4%>0f8~gYHCHhE>@6B*6WR=W_akWG$&N#LRVIh_`Zi zNGjPb5%#CjZBndY$bhYezA;1ooSC|u8jp&jfLLehXj<(Na-MK4c;L_;@|F>jm6uJq zTQ=Q4MY3HbPitA(oLSbNmse35XQjk91{kxZKpQ_11h&kerD#npJD4miq;nF7$ju6e zbgRZi;;`NpqtZGxiS7OfB3WkOSmY*Bd2vzcq*>0L7twX9XGiEKD9YM^_mSe ze`Wu_(mjjY7s7OL_b}z2&K_^1)&v>cSSTOlpO1-wXH~V-_d^Cg$Q<4ks>K_JvqsHo z)hZhzcG8u~mQjO$Pg4$OJ;GV6>m*jo9fbAKgRzRKo#Mcm{yA_})0Qd&oISrKf) z`igGcEwfPJvaR(d^lNo2K8elZ$tI|$OX++oGZ(!z=J#(s)h(vV*++AFbZO z=5(*XM@MPwD zjNpLW2g7uK3GJ^a1kpRT9*7``3PVg-TR{%k%3pGnxWQ-}7S-WKY_>;jH9k0)@|Mj= zi^n&S!yAe(W8(%<3$?tLotuo^UH~ub3#9|D+Bb5=&8i{SF=gc7XHO4$95#lH$sP$V zF;8Hf1jkUG;HN9)%RbZLPYTy&Ug7=XolN9)R=9;}r*PXp?Jxu2@vqiFdr?sr;GE$$ zD>^IuFt>P+Hdx_jjxJ^V1eM!^d4zD-kn~_Tx7Ct0%&=FPzUy~4zt-vyYiAY{1vzI> ztwQ2o<%l9vKd#T|UH^89Rf;jX4Ean+b~soW(v5V#TN!-MLxVlVdRM$g;)_?sNkH>{ z_dggDdkE5if4@vg)vto^zv8q-ZH?tkg$$gH%>PqC`0t!{O#JMxgn%&8D{EzFbh#U~ z!=omOhUjTQX?{*yNP)rwH?Adq1;)0hgg>b)*bjbJFeFPyBw`=7z5b`?efrMFx7!DR zW6(A@1_SCE!|F_AExNu00dB1k2KA+L9GO;PHYD#k#OW!~p75(ELdYrEOsv3VR9TwI z?b}zo3XjBsZxQ-1Sb0ISpNp8#sLWcH)Wksf4BL@~*>EcbkqY(8VA-q=Y+Wq;`e99x z<45#|P=181)3=>O5`R=W$y2~q{4Z)#vQWWmU4|tT_O@gBqNmDv+Rg>Q&{$cPoG^*b&B*ng@78{6nED!+v7&Hp-%Morel+2Fq^xyp7f zjz%W`2XR}HYGti#f-#zRTCKf$Q;*8F)`u+>5|HMAi4N#c!iyXBY`PgEyQH+yfhx_t zY5PbwfznkZ)5i@Z^{HQ|51}X|!@}54Plkp3Wn4)1F)f1hW$gXEeatJ%z!>k``}3aj z%6scm#``l{b{cLD+I?Fb<6IS_wm=r9vki9Lm;~O4c?ZPT2kLp(|| zr*%g*Wk+!4V-hICkOV_2qmT6FYcl26$?piOmZ0vP3x(ta%IAniY%W5$BFPJbkTtYR=57G1A8%+yWtM z64s+xQ%8#;4H^^|PlRPWAUrTUKs-=9;1_^6V0i$q0Ea!vJxYE2dVF^*cQAJ}cQ|)E zcR+VU4uCF@E*T?Nu5dv4v!w{YUZF5&E*Km@T~J+MUGN=%9grQM9k5uSTYy`jQUAj} zFX!-zKWaTjmdD&nm2OX}^sQc?$c*Qve0h|fPUA|H%7%^=tkAbxDUvA}Xg|F#>1=AM zG$u7EX02U^{B2#fEE(AS7(Ca~ZQyieHLx->?PgAW5<}obV2h=bHq*KkrA_a~wT>9a z*{dU61}xQHybkgNTR9LOfrYR;_WA4>nM~W*tUM0Hi1#AAf~8`4OEoa&m0i>Lu~?>V z6!S(fB$e2fHB^mk7mcYp#g0>*!w{aOQdirG@)WjHIw3`k45njdPxA0<(W+4OYD@$E znCykQ0z^oxpQgRXV%4j9f9FvkyhLk~!bPU@$+_$sBvnvVWe2nz6DCxX(#>cywwA{O zMJ+$m&t!6rE&!+?Ngtmyw1oScW2 zl5wo(`Z&6n95j`!Xs%Sx!eDHNZY|QD3OA4MVn2*b+f$^jU2I-mC!^-}Ft!OvQ?XMi%)DEvtb9_b6VSGzc98bjtRHs*dtdHY_73cruD#Zq%{%ph^G@t| z`i|_F);{gw@7?;uY5ysOT?jk1T~HfE9(-PvH=uX!1NC0bL|t$jWiJ$;-uv<$C@R13 zs>+M`zNB|8uk5PqZNeGRyjF?rJXgtT)J6&&T8(;#$dXCLHe8#E9#q?s9+(?NFW`-e zFL9UPyR7}6Up)P7QMVT}kv5o>T<0%;Ge6oKjk6?Fqatn79F8VUmoV6)!q)@hl~MND71cSMUEES;IrAYnSw^;4VV}0Pz1WG=l%Y=l>J0l}wymteurj z{=0OZqh_Owtd8Q#W|w7+hKgbyuVq~m3ZhY?#7IC3QZF5(G9aCxrjul)k#%M2${zl; zvpb*Rdk;Oclo85xEpwf0_obMV>xC>d7c3a|m;RLJmGAU8H|coy=lc!1kJXwoJ|Q+A znb|v2l_8AV`xQ0M8S~NmE3=B33C(y*HD5nLe)sM?H;^I~#khgSNOhzmn7KrD)R${; zE88fQcd?bk*(?`hGgR$FQ+L<^>86>BIbrJPDzSAMXLbrx zYlpR^YaerpDjjW6X3}WJh-R(^9>Y8)$%WF_9dFD1XFm}kIzV*`m(I{Uy^?J{wE02f&=d8$~OcbD#Np)`W#GwaV97vTb4$KE&(23X-%%f;|gv=bmD(uY9p>$HMQZJy10A-87`7>YR?KB=RvfI$=F5?0TFogu#oE+@LFLwE?KUpV zF%}*`J58s~L4`NR+pTjLsLeP;N z{`l45m&91?_@~LgNMPjGWy5%4fLJK3&-Y~Y#PdDHZ*7wJ2?SlpUsddwUmu7{TCwN+ z^^cR_7g%t4JBM#ie8W{D(hQ+zR0-i{Yz`AZ?GLVtR5*|yy97~3^o4|glJiOk=Nln+ zDP($MN7In|1lxV*75NTLOe|4JKf3dzdVsI-#Ucb0Ro>Sy{^6{Fs$o2Kk=Y5FD1+n6 z!8(H+5^7+$&-M*r?j*`55exSV*1<#6)hR?Iz~?J*z*ILs@!hHgcUD}Rwr?WD>KFJn z8w)svcuV@%2|fN*uqr~e>a-~TeuiGWlmuG|?ozp=t*L@XP{ah+!66J7n*0S zGhF^ge-f%8&Pb+aYlRTu`N1WbXb$`E^tF+EOi~fXpyO7N#qy~UY?hRJR)+3Unh?5L zb|KT#XN$Wy79>nF#P|c~dVTE$d(~uAe>tkpGvm5;+O9du6)baTf{SGZzikO$vLSqU zT@djyXCuQV*kgJI?Y$0{IpEBFrk#nZ#oZ9dO7R2yKjWR070I9<1^_^p2mk=h|7X1W zFI{w-8`2qNwBb)pN^1*NEhr!mCcOqhK@hez$N-vVK)5*kFrwT@i&Z<^iY-&>9dKO|q(*Q=W;JRyDZ+upj@Qmc;6 zj*iOT%d_?Q+|Tb0a%4^EyFH8-d>Ze?O+zozjuvkXpysJ{9`={i1F0>Q!bV+HXvm0k z1;oW`1?K{_F8EL*a;(8_^~i_rs93u(P}T>3eopRpnUl5a4tC*{tzQK0+#|JhufYDq zW5c%1x6O&%#)A-uhZ=`XgmwL|yNSA&O7HEfJPZsC~(Z0@~9ojw0 z{~yFH!+s6Aeej-k1%%h0&7CXYYf>#f!W|soElp6-T`iM40@&8w4U;Qymv~>f|0_~R z7zmHd9U|bU#D{pr4KLX`4LKMcLUkCcHNmpRiqC$$^uKcWxwo z`+k(jrL>CoI_$5op$e#neCGnVi*P3kxQldW1GtNL_Xltn`OXKBAK^|IkRR#J2#_D~ zS6k*szHi|U7PCl2U`bY}+Whj^zBI1Blf z<=+ea7Uq8n`Wff%2lbZb?+496&Vh^ravF#Zd4trYafj5u9SGeoEopQq{WmNv#Q_!! zB+sALn+P=tB#-6|rN28+9@RT@?>JB%-8*!T8MF?;J9V!Kv<~GHss9whw_=a4{}hT( z{%#Va4#~S`4;rKn?YCQVNZ=}zF1dU69&8{zl6S+NG>9J68&iK8bdS(oN?;w*r(^%` zi(2+#0_z+eTkv*%|RHMiQYg|`MjX-tI!KHi)$9C#|kU2aBe5TV>M z9fUKV!HXc!BCr!1=sRryC-e^wim}*&Jq-s)ENOHVk#6^btrZLY^aLaF&T$!nAl*x# zXmlGvB(M*BnXYMUnZUt?L|a|2N#q;P;qx;X19Wb2(wT0Ik{y_iMQ4dwJ4Bw=N)TZ& zY_JhT90>YREWlg)!hw!6$acC5|MKg_y}S98(%@f;l#Emc>l#g&`|XI=i@+Q<+FIfg z`h_G2ahXcc5++Lj@O(7Vbm(s8!pvjon)s_nkrk6=++J$CF$8}r4-e_LYv1Y>g@-)Z z{~XX66K?gIZe#tm6{E?A$D{@3q6=h4hhA$I^vlB(O2{;_5hTW8|0e_Nuz!NQp}`BnOcInSP8EUgVD!LJfg^{ zhE?eF8<)P@YV>tD`Z>3zX_ghsU)ojrEx@T|B*;PoFD-GPk@#ca-5S6hk-TLKH`syB`n9kuqR554QmrPzN$&7YOje613&9SaYImLn&LS_JAcC9B@^b z)-SQGRjptcN3Fm>q6s1t5lKZJ~!l z_X}93hAQJPIxaQjr265+{kLP>C%p~Pu5=TMVYCVTguIp`MEkVDVk&9pgRPn9BNDo> zXy8JKO7`!pJ88}&S(9XSmcn5><}xndGH*bW~gv$n8U!Jp3p zrbhzOkA+fLjp{m>Oh{tS6KvK{)@w23TRcp#EedJ3&>OJIsDL8srMjzEEYR2!{1Jze zsh4&31zeT#j&TmIP^^Fn|2|qO06LYYS2g%GIobB;9@2UTfzeTbsd0bHpj+Ij!6wnp z2?$hx;Y6~eLnPH(q>ZpY$o9R}{=ssUpVPgfFSuSf;#Ht^xcZo58$00%S-tco{TF}j0s4SJ^a`xj%9~vg<}QB(cIWiu z3k!d*bmaA}8w>tk&=cBiJoeIH(6J=JrrAW|V_YP3UkCCuh)RUw`QJoKX)zbHpizzZ%}~tb??(Vn>{`WrQIR z=Auzi5&uORH}rX~a6etoWz=QYcf)F_yyen+8ME3_k0es&pDX@YX1|zbkXSpAz9}6x zhp3+l{HRgXL2|3sqX}bbRH`j3#j1Ak=pkiyAB++~}*yrH>fIsLE@r>%J9-1juwcj^qPC zCwtxl>hVXl8YfZNvQlu19A+y~=WD*^+jM^2cp6?8MW74rx_v9lI;#|{yFT{l#mc{{ zr~?H*ElRBFP-+$p>0vG^y0XHENa_ZGpb?+_bgkF1cr3}zAV2z%cTu=Z%)!|9J{&Un61yEft|n> zodU#kQ?g03v7KXh$YWAajA^V>Gp;Wbn^30kO)h}g9t}h(xSjfajf8-g(>Wda2~}<; zB8nmrH!;6;tni?I+YPkNr{^y4IqD<B+6q=AOky?u3*a8Cr40 zDKa#)It{H(*We&1z2ae~NKDxT&W$v3pBY z>J8bwIB7F@$vNFCzu%-prB>vFo)QfZ4)Qeb+kaC6Zdt=%dd0%(4SJ{%4(8+Jc361J z!I@o9sJl;lRNsRxm2gu$0fD}^7+}tpWMPnIW2Dy(hNPh>rD+Z8WlHJZ;6G6WdZCKx z&19PCe79oJ0Gr(YY{Bxwtsb(bxkSz$(UborYbgF~4(^uUv)_B2UF*j5X7mnE-xf_6 z#NhuXSHEpSTwZ_r#_;#ujuN}O!dVvcM-nmzNU%g&iI=56jWWTi>4QO6g(Hg-;Yh>4 z60*BOG0n=vfw)E-H&xVMxxE3`hxA;T8;ZAfLf3=xozj_%Hoqg)V|j_i5Bl2}k$=w- z$Ud3>D3F+z21aLZu7!Q6N>JfMiMu>~b{(k@)ZthqfwWQ^y@0wXD%=S ztrweaNc0z4MB6eRRcM}V4CZr5oH2u}y2|-0v?k6KqLFK-A4&L+AuOiaId&p(u)l0j z-!kVR&W6{?d$Os5G?c>UNgSDwm|qAX$PCOtlP!yL#2nx1zMaM^X2z*blUP zEuQ5Y&bMl0cRbGW^N)AZ2!3&!+hY;j53PAWC~b2*tm&tllw2#Qb>OVUaptT+53;vh z)7B?ZuHqR_o{LO_EQLH(JE;k-+7ZF2M@Z&Cgy0Xt4aIjk9*cPWc#v!XLPTK7f(jY+ zpZ-h$S_S#wgH|WjRtXR!nFnL3SX$0S9WN)6G*euRxuS0g0BzmV$rZ^QFpXGIOyTW8n_kYhKIJ}hpK*Y4geA8{go=A*K23nCk z+!0@UK-KSo@T2R2K%G+)vkQ}#8sAHS+6$BCB{>qD>eQ4b)Knac304Lf37lvrC%5x& z6n=GC{K1e)yrz!u$_I}S{9x9t(}~JaxUoY%7$KJaF537U#g4CKnhA>tVuKT&yN#Z^ zsIDyIvR2Qv7f)^o(Kceq&Vc~}TE|h+Y@xY?<1?@Oo|nZgKk;~?-G+WLSZu7=m5N-8 zN|r6_x^1%1a%!n7|5AylVJGkVUqeyfXqw&{S$au^5$5|4kV{kW$r_cHe36--C^|wi zD+icFhA!Z-1$*&ELEPMkAG&W&dsMH>gyOSHTk zEx4fIsEMJ{8Bv8Mn!4F*eKG#h9JYe?f-1J-Ncrr_R~UlXj6GTGPS7&lH|kWdF%Kl( zX3`M#1cCc3vo3=$;d(an3pz-^MM@@zJQSm03mJS&`47)$dy5~iWMG7-yTmFPL~E3I z0bGuvmNW)W=X$AWEjL4%x>F#k!wE0Ux;l#?If&bZ6siBo737ciT9^XDdQM&YCD$0lxmgZ++Pdu%JnHrRV0`NYN z+g2s1Bs+Emb@Tt{oZId>54W9%)!KSoZLTrLnyd9bzRsVu3LXv|Rhnyl*yL*rm-!Kt z&%X!F5a)ZOgviFxo7c7dt{fvIDP_4u9m;qYf=VPh`r4G3L_y><_Nuz2(!AT?phlv4 zXL+gClT%xwe%e74|IcJ|`+j>Z-%v?5+7DJB^hw#_Ux_-c@MkXnb2m2*Mw&jGwY5yf zR`l{tNH)BoL+2$)0GvblL{ws1f~USqK^xh8^|Hb)=2$Q|UA=76a#?etq%br-^Py&c zj5i5UL7w;2O6aUX^5f`s-tJ1_#>*^=a{{+DT!n9@pZzgFzGAMzt3IfMhG}B&Y^xqb z&6v8@%sAbG)RLUDxD_f_^E)Q$EH=flY4gnAz=1vyXWrBm9_$oRcE7thG52RutkDM! z0;sa#dOZ6d9YiBRT)85^s&5i@^Y2hQ1tYKiCCyxqC&`X6+8rtLeF?L7H32jIld-%t^s^^K}PgY(z7ws$Yyf;EaBOupw9 zdIY_jUi~APpC*1zFP%-!5%-HM5;QUvTpmq{Ju+$sA&T$L`T%jF;r7D zQD2jY8Yc`vzuc^ti*bGen9s!?_p~rENrPx=TjE`t%e7ix+|{{ zAO@%-9WDh;cSZTQQE$ zE#22$SbAx4Z5z{4aZCu9}Krq`MNy^L?)W0VTyp4(tCw;cG9A2;!ye^nm@3&IoMuU?XF(zZEs#UT3@`m8ZEP>V+3mpZ zdVwc6>2aH9n&p*N9iOkpTG*zZJrTWzpt7?EoHIg&+YaDs($=X%7FPEK}h z>NdlnYA&5W$wBYn4Z*G|P`et&!qlD%J=!8mO;>YI>7bH^i8)9WVK-bJfv&-cbgQl* zLXRwbwjpeH0_ed>$41%4ZcF-;^yC^X;obV zm&6qH_FM~eIz0o%2`T~$hwI_5p&uxuOf}tuPYSXelO(B)<|ezJjsU)iZZ$c9k$O%UiL)yW)!~H^vD&hSfGZro6QB6K&jgzXN z#3Z}eB6n+LRm#E{X|_G(*1TQ1B)cnBI|IuhTzA})VJ3{G%7*sB7lbTi5X(h=8&rR; z$w~|*CZ&wV(jTf`Y%aLva_Pp+A-ivj9nFLCJv^PC9JbPp5(4WyWrO-&nT(~Ie>%pH zopA&_{y^%wgrXwC|5kq92}Uh5PZu93((q6aB!Wv+B9aL!dq(090CbZhTu(#}`+ zdq+BsKIZ8#e^? zyTA!;a`~Fl&(vl%677|Zs?K*ZSA1%~x74m1c^%Wdt4L2er844HPcgQxc7iT#D7_Ow zyuS2;Gu{{7Uq$q&W|AeG)xk93uG6bxaKM{u4gsdlLM+?p6%FUY7&T!`L54R5Uf$Z- z7fmJV|JKZb8c(INoP21Rc4&zPel1(yAIaBX;2qY3F%SN3_79V3>dvu<+xnH8CSd-o zr?H{<4kkYmYi9u9K%dncsH?jMFXQK_AlM`?k5F4&oot8 zJ)WI(ZRx@L^?;-L1Iqg4 z!K*?2$ipWpU~qZphhjs5WE(=Xs#VcE4NhSOfoG!4h{RX~UB*I8?^!J{ksUHXLgQzpVh#4)Y_qoNuFz;oxhvPJ?T*v4fYuuG zKSf-=wZq}9^GLoNzV55!1-gn&TZtfdSHCKEsYF6Nb9If&Sqh(5nkmKCpJ`1ce`N>|z zUe29fHU)-pWy@V%BQOMCAEH`{`?INO50vi@hxmmqKT-zva`RQFHe_~}Iwe&+bI8Yrxm{P3DS{QYM=`&}i-@CU>D8j{$*835j2 z$NP$aw>#aqX=k1UMU_0)5O9Fx3_$<@P?r4rAIQcx z_Z3^hFPl=x*F@z1%nA8FMwyPCWFmxB8rTE>csyzG}j$uFx^2K`71 znGr+L+a|ct^EUBNf;r*uw-kcz@l6I}N*T%>tan8fRE&WTp?+dY1)xs%N>Q2@`*io% zwCS#p!5$c>;A}ATG(Suo-)BpFtXjnKe1N6VNJ5kQhjw2t|u#Z06dXm}C zBrpGH#=qZ_K0^k|^$Mr+w6_v+=>5kFr}#N{2y@aKSn5K0c2XO z2B2@b#3r~w6b)Sju@*YQ@Lb@)cNabG|~U=anc(}Xx&4;53PV$>%^p$cizQ}-cZCP~%Y!Zu%9bm!&7%;+emT@TI677EuQkxey0n*j2XDbMn=rLaW^Ko) zlTc5XPon2$TG0H?e3)QnX`Y3|oQ;ofAjw!Njcs)LAh{EF%dJ|;H|cLx5+4=Cp4E}I ztqEm?=Voy@pmM-XuXn=tK-g|!iKGqd<+L1X@RL%z)GlaGb}siU={~zt@(QYVLSDkU zItl`WkF!L7%JTq+SmGy7!f(czWO2vIJt}{y!dl287}Z+<_FYCj3L~TtDxlje>20^w z5%dw4rnF!zN6HcrxG+OnB5(t0Pt@A$eEh^z>iCm_73lf z`j&Z#@u)(rMOShfNYw$@<(N%q$!d1^yN1=o@{U=3gUGn`c0cPxlp&P<94bApiRIp} zMHo3UoUCzS^$}=ZbD3|SM%8gjekTP#!4b`y$Z2-k!l*oUHo{x2-@G=zv2=&Kn$Xeu zToSY}8G)p-_0}w((1bGvDQEHy?dWVF6ar77dq#FX5OJcw&xcf7vG?E0SMOEb)8ngA z2!ECE|Gs?L{-aIADC;vr{NLGpd)+7rzkp_~W<`91$h6Ah0`UlJMN^N(*y;=m58b~} z{N$*pw-E0%&TR?%LDCjiJ(efGr+X}Sck>Lt+4WBhjang5jk>7CP}duokbO&{DwUgN ztLCL%V|&s;+;!*oJ9d@8k&WY1;X_1~=%!Whu^))%tz$Q9>k+2Iy=Ez%N@0gCs`Tq5 zsGwz@Pf(i)i_qQVX0{2kw>Bt9{SmWpe^?3LE_O;i3^DbKZgH5Vhu5MGQ> zM@uBqIOngE(3_JV(XWC`71(D`JNcl&rKD$$R`@Ix1Wt9Q3RyKYKTv$wD4>E^Uk{GC z6!ssbRE*!L%;hIeK>k#S!kut~-VdRdoDPZQN{8ojpz)_;ZiSd39nnJ%#V>Y0WDwei zX6k3$#!{r4jD#l&)-kDQm$WKWa$diI;T^K{4mJG;%n(l}pkXf@0+QX&GaTa#g9U(4 zI`yZ}UoSiHQz!&{S{VNuV_kZ1hr0g6`B(3etzpk(D z|DUf@ZO?H_4dc_{Xt?&&-GFNez-ZU6lek>7p#jc7zJu4PZ1of(I@LKpAzO#0SfNT_ zY18r;DZ?L!4T%hKizW{;mYyezr@Bn(3+gj>a;Sw@Pg4;8d-dA;YwpLI;OS_9;M)gm zU{ZSlIRKR~{5C8Wq48tbg-+6K9cKx3tlcO!j2pmHagZFggh$2ET|&z2>uBpKHPnSS z3!^^ojCCEUB0-qMQ)cpWq!|d0a6TCmxb}i4X7iNlv*or~u;$dsbPB`|v{-I%PZ0Hn3V|2UbT40+ zr6KW_j>GK}kw5DCmK%urS8M!veD|7L{91ViM=S2@2nu?|4iVU-N~XBb9(>Wbd#555 z2dz5YBSxRkDrt_phw3&mE|y}gOOj9N+uNPQowomGLrGInhwwJZvmGFM#^^9DaJGE3 z1RNte_%cV=^O-^)uPWGGCD_;xZ~Tm#)I37&MfPjqEuXw)8>HAv zu$GQOuA8Cy=SP zTZ^sk;D+Bc&hp;JgdDphMFJI;2J3gYxKu_a$seOBO-$|PDuR-MIRm;N0Q<~->@_L75WysAiZUVZZ<|PAmb^`gM9{f zAUmj1e2tQeb|vW@a-;cIm%kK%UAsan2U zt#rP?9P#t=^&+C#wq01b*0vxqE{fq_w~f`QThZ#VaU?B*UmSWmT;mD4Xy zAh(xSXgrD$iEuoaF%_wuBmowH!ia++{+o$#jMyDaQBnpAyU1bher>y2O(fqJX%JqT zYQ8zJW`t?gMzgC@-Bwrk(m>b#b*|%jMleXY?|O~-xwqqa``4VnoBqpFK%XxR*h$3B zh8@|K?1&&{^hkGKVGf-Xkum`zeeg)kw--pHY$UHt{K0#no;hViH84IZ# za2$=rVLqW_mf{NSk`V~a7z>#Sl^tTky_LWzJkevm?P_fBf%Jp_a$;U9-wI+9udibO zJ|qFJAWHD}c&iV_k9Q=$k60srMML5W5Hcw4;ocPBS2&0k{15%DY4UUM^XE*!RFKV+ z2jC;*JV52PV(?Rf#c=9R4ehP3w~05$qNu zKC>v-eI#67aWHmTun?81akPh=@Yg;n(!>1z7Kyj$Tt1QZR^(niGZDD-O_o z(S{S!1#G>j0GL$|kZ83GB%*3c3&ViNYSsCN1UheJpCnm-_{$#>Ba_DLbvUAMc z{6R5aOi~^B{Fr5s#tR-M(Wz~P7a=FN`Tf1uFD}D5?s6?T^;f8OQSgokAXes;3BX6q zGuKhgvubcl%`@R*qdMJDl*$X%)!6XHQ ztMiyuH`3!H-PQ~jOhpXZ!(D3}VEX^S)Kqe&)*4WM-+TQSbMU?zq2!5_K^`#e?vpapCswsOO*2z&x~vB8~@E zIwH*%bjYS8AhrEFZDn5Tt!4R@>I)NoK&tE2kHRZ2z}KS+HK*HQKutaY>>z!Vj|~rf z?0oAcpKsIbe1)v-`eLN)d;@!SC3~e*sCE3)r5?_!h9jtZZ$!%z#4BBko}`tGoTx@CMe(F;PTIdx>}d* z1^DHqw-YHuNs+m18WG4M-q0h`vZ`$G^tMg2eMqFPkFe_(uTJ+Tiixx1E$jG&)focx zV9;tHGO4TL_mByhdrt_=szfjZjDmXVYL7pkEFqCHH;JWgS61AjadLsR&#r$O%~dtq z>C3s#3z!UXumvXbo9RK1>eW|^U9JRamrt#40+6HmY%KxGZ0!8TaaWGdf1Pb0%5;&L z^@63FV^deb?%Fs(h%dNulft&@)x-zz@x-|#Dv{NO72yJ*86w7m53(U5iXJ)K;a>U(1#`;0uND>q8p+)*nQ5_n<+!kAjp25`&5+zWm; zMz&0#KO7}m6?EnyID4ZlDrs6;lHk;q%LlRJ)K6SIhjHO!nNAnVZqw*(gh3Sh+LqA% zOuA`qnJBFk#?HX3GH21qM4CNytb=oP=XO08i6gy54V{F%2Bl70m2E7{iX!?OdQ8KK zQdRS9YkVO&$E07Ep>zZUZ6e@lx{4^v4TD<)Ltesu9gA{#r-d!J^9`w76gC?hGM&j< zdUUgXjS(!&7@f0JW5hHTN$jmww0RAAtD?ZW_{Zq`o1DAkeHSoPh7QcMuj#D#z{EW+ zl=`MLJzzwFgI}*?V`8ak)wHV-L!s`a!=0r1Q|T%tS0S{Myn|AC%T`%`d@C1YScd0& zVHNG4Gx4Az8O8*Q6h%FjN6r%-MU91-pwcq<^HPV=o0yau-JYP@_>GqBkZp&g!^gif zTos-Xg`p-IN^BjQ_;+^6f!a|@&qJ~}+q&M{ZV*VnORHc zO9BIQu;DVQRm>=4XsqYo`JSTK*q%OBS-(hjfl9^fPIbj-3<_ctQ~4ReROY7TbH=Y zW*oebxMe*9Bua@ddr%{PyD4NqeGMD@+St#mSjxUpV=@y4)9EfD-h2b!5hYpEm3gBB zDKH&eUCS$6*g{#_iytUVXJ}7I6T?K4G9-C%ke5tno*odYoY+epdm+nqBPj}4B=d7A zPq3iUgPTi4FjeSp74O(~YXMGpHrx`#P(^vp<^0(YNeFd9NEvx%Ql-v}a<+QOod*%* zV=;0tm)sb4%@>mX(M{vi0GGfQxuc@PpVE~kVVs{8Pqx4QWKf%K3ouEP7`EL!%~OA z*+YaKxnY}FEq~1R=-;vN{!#u*8O^SiGhNz1h)GdVRv7Vh?;PDbxwf*S`eg-@ zd!I<1ABNu`i^&NxeAi#CxHY_gPCQ}ZfWF_`^!AWH!i(vf6PkA_YX;LaT&b_@D5)*5 zvCbLQP?;(>s~MaL8J)G9K~NAX!4NjZUx%Ve2>W^ zt43*(28N|w@Hs!JKP_Ya0n9v6Z}02ung;(#HokU^GEsd-N8P(wyT*~OC?EWre-V29 zila;${TJh6GzB|U_iAY+A=AYg6EOR)==NWu?R*A>6hRCzdOqx^kP!=e$^k5NJ=XV7 z6X<+c1q`96VYmX=sCpYRcx*C^r+O+oGF|k_@r89b_gFFx`Dl{!TFKEv4gVmjFM2lO z?V04?O$9>kgESl&n0pRZI9>5xn)zWXp`Yb_H;LbmWW)ipPx7?qFhn1 znKWqVoUD=a3RFy5skQn``H?1{uz%mxM)X&YA6w2uLvIoX zkXhK8#0pEXazd%P0@#it$WZTxeI>GMBY!%FQDH@NWb!1zr3uF!%MHv&bjWf$K+4&u zurQ)UaW;r&mqa#=s6Zc69}{yn$f;QuW(IgJYS@+fWtJNg(qct$FyH^(TEuP37LlRz zDa6&M0|8x&uN(qw>GoAx<-Rg`U3Mi$E{hA==GPuU(DK=0-fFT!ZS_d<*;9Kdf6CbCsqlaO{{bZj7QW;+c06O=pg-!VCM&quCbv1+X?;(j>6t6UV>Jo0n# z^_iU+z#|iR&tXaV=0x`Pke!N#uJxRB5kz{h)M-)-L>Xg?8H>rjiXM4%-XT4c&H6=4 zc{{RwnF%Si0V8a}OG+B&F5$G_l=HY1=f{q=>lF8C%jT=>$p@5&Dr~%YRJ_ue6-7r< zJ{64*NhVNQbyU0)gbt)fh$^Uq!8>Aj9^9DEXv#(C0zV|!*(+hD3r~;g`(s+ zI+z+#!VFWHRyqUA7|92=#?fQt<-m`xHqu2*tJZDgU8ANrodNAC=$Z+l1~rkk!z~Ea zO{eXavK0p56Spkcod^7;;gzh{{sJLh)P39WR1PRx*6%UtAyl=(rg05poX+SYk=-Va zq9S{?e`;E2CpP}qUk$&o1`%0t&JIZ^|Rk*b{ScKNAV;+1V;U_fPy2~?M@w0u3 zVb&8V-Ze|xwcGz_oa(bHmMO#1j3;#3cZQ$NA$tn^Ti%sO=%fa(u-*Ew5rTu?#UhE5 zmNS($>0--5HH>L3^L%gj6mw+dZFtBqjatmKl%sMm8nSux&hFr!T_5#HsORKlPK(aN z*3j;>r624_mP%SWeC&h*rK5j@`Fl~mnWg8n*5ceLZJI%tCdT&I%(!>1$~a0$Iy{A* zAIq1HFjp;<;L@)1fQo{a-yV&3t5J}X#8)F%3%=8vcz2X`B_aI__HL*DgFNySLCYVR z`%a=biX-)^J{rp*(iasM;6uXf=o_>un{ch~e`KoW!nRZ^D@hGY?~OcWG*Cel!9Fh2 zE|DKnZcMaXwg<^l&fk*I&?H>c(OYF&)SWn6?3sZfv2PL&h1(w7~8b=wi@Cg+1tHT`Ki;9x?1NttW?9fx_SNZZ&kPHoespD#LjKUoWXqTvu!oxs8{ANR@#8hRck!SO+;5<>UBF*zExlkM^lxoGx zCCz;{pQ<9GfclvBDxvP&FSPtQ)aj*{_9K2pC49wJoHyl4tNX5%cmr*j4_sF!VaK+r z(q?--FYE80c$JIg)@>yfHU11)vP0{^NW+Is1TT@kvx9QEW=o59KhDCASMbP_6wP*o z(UQ}o6_INVU>Hw@^_z+(wpTlvOEoALcatZlGIfZ+6)dgaB@&&LOr4 z<4Vg_$<(<`r7AHN62SBzlDqSa9&ZWwgt@xp(XWoHe>aioPWmHPD?m+2vdj4`dpp-K z(?wX^gg8xlGIM62-<(2fgR^GXF9RC_!-n^^W4O3x7@tPwF3DGnF=rKW>rW4yz@<#T zv?8o2OwFADr?x5GgC42pZD9Dm7Hc#WjQ!} ziD&Q>#R!(ErKWX+3WFgW$rdhC83OGkY7YF%O3TeOeT+L!wF}gKUa{P2gaXYaN3gub zh{{B(tF6o7?#7D6&2;0sBWx2z^du!`^C&eNQg16oz zt!2616TNN!gx$uD%r|sH?y&5AB?22!XJwL|;4F-7_Zayr%srVS5b_Pb(z)2mYLZl! zePNBLbB4D*9JZz1aPAMWFg+TZu?^zCEwMOn(Y*L{Ke#%JlBHJ8`MDonApGlIQNDt2 z@!14_Dx!#1_T;*=KTp=x>US2JOUV1llk=+mDm&p(MgWoui&}_>oKLDF+Fn$l7uos^ zWnw*Vs#oFRm1yVr4H9Y+T%$a3vx!JP#JRDwx1N!5w6(6oLzzZ>S6vq$haRO8gVp-O z2nXimZ9#USqOJSKE%N=+%>2%yV?!`XeSdwU(Glr0ck1Ne*~&W&&9*Mo*E^EzU{21; zFdDkU7;1QUPfBZU&t*Te+#MZIiGH_pDf(W$NCih++f+bc@Gd$1b+EXm_ z=kUyWfHkQfF&38=w)F;*aqeb4v78fb*>jnu4>e=XUx#_Le}Kl;`6v|aHl-Zy*xiu* zYZrneV|-^hday3^MiT6c3jzodfUB(yrkZ(oFg1@~1{%g(eguI8;Z4LGQrbe%PRY+=wL=bP;R#55 zg|qC+ap&Q_;R02}a0Gyj50*{iu?aXrK&wBF7zvb=v4*ib^YrlbUsO;eM z!9U%&LhW)=gvNY-vf-2eD`9U655Iceua|P7XPT)-%34hWJdHI)0 zj>2WY>zWqlmfnPs(eyhZ5oQ?`?Zbb3D@izwSPD|BEYhy=MkGDe9lR}=4r&AiGjtu-PvIpYG;o6B=eGQeV69~c9_IzA6t)(EB+lf5 z^GxHxJhNsBGIL^$L(gOW7?=B@y#wvjKJF_iPQ@G+@q z@7?8!mE?Y{U#WgeZ}jL=xQCQuDtfKROtFFCB;Soqn|^`mP(Nkfl7ichO2hZCakO~Q z**IdYlA|vh#1F)2S<@V0hPj^$)Q@P*loFK}f4)D#*j;^7qh$u3q1m9RiSMhFFoO5O zxc+`?h!8e1C!S6d<2{aFL=d}kDKc$MH7Em8X8_-d z7de*7=L1iZ+6N9TdzaXiq-XRY*8=-jr1tSDn6=$)A!{*HVe~Au&79bY6ZH4K7ilJL zw8Kc|M>;U`@f&jkts&px1*MEfd=a(-gWKl$NP_nAfQ!$}A0R@pY~=A0(TIiNiG{V8I9%}M6>gVvLGFB}5Lv9MlKW*mDDl7Cml z&^Y?o4Lh_SrbwdH^hxk`BXKq)TlP zpWdlPn5CGdd9rJ+?NA%fVggW?rSe__G9=dZgUq_mTRyKs7Rly~>6kLjkd0j3vR&rV z9R@EOwVO8dKD^%|;T6mBx)yvtbUTeaQuGc1{9q9Za9?^lrc0i%kFs^W?|=j!YN*_< zcnnvz)g9%^G#*8_kPY972OWn53!fIJRQ~apYIxMiw z9J72KBv|f}FxU1Vy=H(G#N5ep+DxC$tThi#WH(5Vh$B3YO(w(;MfnVhF!RZCwf7sV zB?aS{vHOkIdKp6WTYazLc(@ux51G3ZW>>V1Ti&NmIEB(D&j?4ljw3$&oS=b{;y8__ zIoz4^<0fL`G2$T0>^HE#2s&|$B5_Q!@RDEonnq|DUsl?Ktbf)3$I#@At}u@^RP?lc#$UbL_LX4%c*IPg~w14Ue3 zW+4Jax=e4QZ*Ro+sMc!y(!gdZDWIb#M3BGNC#g( z2_sXeFFJ(hbE|$gZfA*8*;4zMI)~gemBwGx(*^bYwJKP#zp;MU0fb(d6X=E$tP7;M z7l`SOg|(cxDh%9i`Al%e(@%^)vQ`?maBeRDfRPVz(UZqg7V}RbNOCA2f%%pxo&-re z(1X_{h~Wob10PuYzhC`8Bk>c1chQ>|f zG`HyjK=&O?(zRLQ;!wnlF;VyEY!EPts7YL&PvfcpV z#NhxnAs8Vl%zu$nhsnu0Xf%w`lEc{MeA;clkncRevytEv?OC|`0K1==yZ^L*^I7ka zpA|gEQaml}zK$2>pgXrrE9=kI9@aT`Rx}BuU-HRW$_5H@sv}nWUSnSa zLWg7IgD~E|48lARlQMrn?v|{G@2^$hyDMB@&-8QPZ*f%1TB-iw=K#Ads5PR=z}XT2 z$we%s+x7H~MdC|c6ihNGS76WqwD}HxbhRjDTr~awwSIf)PqCki?RWIvFBI|Y%g7XT zt#IcA?;8=AAnBgBD=l!xFUjd+U!48+@?jkOTFwnIO^jyL2zwqU&oMF4`_)?>Tx|D@JzRiGJ*T z8|x&wgR?oQeP*A;mltrC8~>-k2`LAiqp?4T|8%aCt8$X-6Yr8o3WanIk}u>#erhp^ zzSm&a{sg@Epbbm?*{uI71+Df^4*Wg&eN{PK!uIP6Q7$+@6J7NPyYu>Qlxg@OW5fP; zWZh6!dK0xvSOat8cXqSLO``kq;xEYBA3?NB{|Etsi<5?{Im7s!q78B!@FgitaYcWk z80{6VWLVs;ICnTKFX??J{4OY0-+u#?AKbai6o8p-t9AZWGY6e%qx6wzND!(V{Y0x! z+8)CEppb`IQAU~WmMO@2$g*icT9M0Ky4a>&_TA}Na1@Efz_4HZvm0VsrN0_ARo$9) zSkCQdupC@EnJhjy{$hnks+1O&MDi<65Rx_%)zS=I6idr@;Qi@FjY%zkF@MqgBK)sQ zZ64*&<9(A-Y?c|&C6v!MeAKssji&mz{9G1hlz~OY(4XW?-9s1*FG`r&dp2AZx z?x}xJB)d0_r9s`K0P4*1p zJL7iXEIV&@qrl#eakzx1Z(LM^ql>-!G|iisqVDBvx@wgG8)L38=B^s0oQp|XQ7?vd z*pVj!Yl(X}=eY!Q<0IfCYjmUXnugAs5t>_}ery2|ACxS9!V zdDlzDf=|JjT-Fu7J^uZ17r8&*WgqsNnT!7F7M}NAc$SWx8Omj@Hq181{xaRF)>{6- zj^_%ZpPF8}s)$|pdQoMuKyk3z9UC>JFhgfSL#ij+ChbWqeDmUZBJkOzDD97S0uFT$ zEg8!ofrBiV&GJ((xw;I^41eMaeH+6EGcxrtN$^V~h>@c=LMOLVapikJV~AzD;$Rpn z(!p$sMDrre8ZQk7Zn?bf{m(?Fk#dWP-(TKE@Gtk|GohHb0WUxFu*M##ml z=Ufl^U73VQy_T)_Tz`H*HRR31lfXsv=!G@vO`ss*U_LEYG1SWvhCDKmCC7ph+lpJM zc&J8aoyUfudnp#CB1Qe+Tv2wYhF=lpXDr2gJO6<4S8FURj$P}aAu=_N%Yl`F_?|XS zr8aCqRv>-2=v1XxhI!8I>Znz=>*#lFl8*veGtSpI&_h)#%|R+WXYx<+@V7xZ6+RTM zZ4H*&8rJjRNb(ae-087pSk324^s4Ws<{f=Csj313$uId#O%qI z&n8;09+_qL;&i#Qn?L=YI@MNt1q;5g!4Bp1a`a+S%!?+P={#u}DLTrh&#n$$yy@w3 z<>D00iwj4rny}ibaP-0W92kO#u<&?ShV3xw45TDNvJY4iK(Jd}(1YyU*e) z^)um5W(zG+hPegu>gk4`e`XpawQ@{zSY9$dh%l$2G0ZvUhoN7oi1tZe$@#_7y2Aw! z#*uH}Sbr$>|5!wuO{!hcEepmbfLbB`?>1Lr!A{2_VlXf~2{16~|4pgnz{*1ucC`c zrz66iU*4dSremvHQ&Xc~y#unSasL(aFZ*p5gr52daJ=%L6FA*&+PS~mUXUXC9$3QL zgA13j2kF2iDnW9c{uJje!j5mArc7^UVT@Ivk*uRz(K0jL+Qe$3&(sj7{NYR|C^CYH z*0qR0N%yVUf9k(N$Seq=?`GyI%TP0!R~t?{DVO(4e;qQzmlOJ!(yT5w3vcC=tl z**R%gQ1RZ7VMpX~t>PstVaM9UqXCwDOV(=7-t2k)r6+VmFxZ7S$4MyXMxKtSzWxIg z2oM-2Dl}0sVHLe6!3>8P98%*Kl|@#)P__|}Y>HE5l`vIAiFVA`NCHI+_u7X!JSArc z%;YAlH(C6lblEl9D@ZtiBdO@x6 z&+xo8)%igGvXp5Fm2BkUZd5gYo=QH!lfRWDd;q^B`GGCanninFkHctDfS8XzeZyXc zG2dHH#z#;OX@gMafhkB)?L7m>Mc5>OboO%vfw_Tu2IL50wE4(E>NPZR%rgG&6Albe zx>Y29OH26J8vhjY?3K0bl@jk&r1lfW39vGgf*mlGM3S71&OX9eJNQ{Umck#0!WT^6 z{w?iSBwW$3yhVFVmi4+QIa|$Knr9I1)jmMd!0h*kg~Kr=hafrfu+R>^qyfp1Q) zY5cnr>I&6QrQeIj4SF+qZ7HtUhT<~vXdDKy9J%gW-u38aliIh&*HDd5ml%DcR-2U; ze(G*1#d&olc;LK3xyqGg_;{ubuc52#s6Fg6`4=QSFFNBZJ-kXr8dBUwQTj&?zu|yE zY!JfGkf!b7Bhec}vuAB({@q-Aa^vz@ZF5y#S=c?Xeqt%#USA|8sMxCvCF}Kf>|xgW zx*OUhEwQyz`^unXSv*C4ax^xw-vI5L*F5_`h&^6ranXOEX3rFDkV4-t1tNs{J$?+t zt6bybKCW?lZ8HuOkWz+T94GXCl9;R#-$FqkoD|!m>vrl#dg(dhp216XqneAuileL% zObOgt>*;eWa^t~C27ie3QKPAJlOvp1sH`Qlf^lQUJ|IexFT^iagcg0muvKZ5aNym( zUNA?{DVu~$9O1)xTH}UCu;L1ORPF7lIuu6#4%~L&R zoW7WKmN)qr2B`z0_P`1Z%4OB{0+2qUiEL-tza3ouX+;wnj7H2E_wYK%6u?rAu%C!% z63GlgdLa|6+w=Gla}xJ}=rdG9W0eDbt&)ss*pE%50nLkf9mvd596QzAL|z>2E!HgW z>O6`WowyEHK#Y3}4c*x-b_NVpi4az!jfNyQ5E>w#k67S^rlc*pia--pCPwL20uLEb z>RUYDsU-{C;zqVRH>?u>8!iyhHJ?3THV4ecE^Mdb zE;P&6_?jvx*_iSWM|;Fb^|3h<2qeI>-K~9)A|IFsGQh|CS^mKEQ09i{q2~!y6MWgr zb#qz${TqdsGD>9p^f5`4WnNvCS*Ti^SH+>WnfFagM>G0@mz`m&`rP`*a`Lb9WM@WM zX+u3+r~sv=thQ%*6Uxdk38mk_utAaJCedLI-px|GI0`Iy&ILPQuQ!AUS7d_*hL0Oe zOo&XNa`;JM257I53+Eip3&Ob0Nn;SXBP@Xxxc-}r$HH(hQ;B+niX;HFoTFHL+>Di{ zYO}{dKNVfx82lE_nD)SXa~}OJHnva@1qC;#_z%6PyX?CP0diT)Mb%E|A_4(r76F{+ z2W^uC%y7}m!Yx(w=k&S65)ZZJRfejoVk*bd&mIaN1~Jk6#AzL=QJ!}7GdN%uyn`x0 zwMWCn0%5jX@W&_RGYoCwauEukgj!4^%#dwC01_HfwDg;qkvQt*p^-59%zo=*QN;IK zEom8=L^d<5Td+$}c5~PS5Au5nflQS&T}J8xv;gp-_vK>8~2#2o2FCi+09!6XW}t}>QBg7 zys{xXV{Du!t?+yq4+Y~Iu_7H)UcX>jC#LT%kpXfi#HV?}3*#Ih+d{#AV3DI47ZAGO z1NPte2HI51t@cjt-!8K-_%ge`J=D89u3xo;s|Of4dT{?gjJ@-Br%{ydnN)1s<~KGg zwrx8V+qP}nwr$&1#a1OZclGqF+daKz=7;wmIQyJ;pR=F6pU*nZT|mvAnxZ`#$fn^( zM+qz4$M3Pwwv9Us(WJ{B2Hjf66?OH4^g3D?FcJ9M#L$ zFvt%?ZcisCN>b}yZ(2;aV=mMBDc9Rd6Cb!<)ss~ul$z7rxOMA}O%dyoA=|jO=uc+k z@2V;O(9{XrgIqI)n;<@QQ@ko2)6=@xiBFhM({$hXq1Gs4BZ_9zd0>o$q_(=rUg11j zgV)!OhN)gbTza))7XCr`I2Nn~AF206&fUbmd`u0-{*@u7uO9>`?E}BG86rCpHJyaP$t@1Jn_T zmX!Pp^0_NYb#YN`bCvALOC3>q_T)@qs#@u2Ol(9^6D2TG?0R>gxj$%|=ltged$dSW z9bO>|H!pmh0>kBY#U41BpP46waZl_n#lwdJ%`5vG2-B`yhT?!J0|##)Q2=sRz5*%& zS${mpr$9JA>4OCRV5;Q^I#YsqTq7IaQK<13gOz|%&Qn_a8wsgc=vyK3Pk3V=sM7iS zdy~Bdsx95J;ol@SJcA-^8}UJNJH{+t;8+UUYnvAv*|AkOiI6djImy#>KT^ozkDRW{3iBAR&Jrr?+Q)7D4nGeB36n5VYmG;-Jju zWxVSk)~EyebT$p?ICsPWwUD;Y$YM)t{+jOk{*}7MS?SI1!^pXVO zm1C0x@jY-RAs^80_oluPyy7r@lV~IkW8N`9xQ~&2%E4MX3a^c?kx8L_PjQ)iVSK5D z_fkFd-nEi`xu*9{@5XB|HWnnD#H^ZpA$-ZDXUoPIjhBPZF*!CmGks(9Y94%-%$@Ty z`KDG~EJ%w}+2?Tk-Pu6W`{Cn2nd;X8K9d;wjq8w8JnFk8upSExkym2CE4{tdM&Ic)Mx8J*>F3WeX}RH#|I8zi9sRW(Js0q` z(}KPlJ?x6zDGtbY<<;rC_Wmv;>?w_Sig-poHimAps$PieCdKMYiHUMu}Eu@=kqlfX*p+p ze_k>vJVQ^G_YRPJGSEd;B%YTi%yKB^M3H#>jfcE7!coM#M<8VlX_kf3OFm@it*v!u zaEbt59SshOHVxdM_0R3S!9uqM8w{tB-ztn$S6!wAwtTGv3H(J=}>Q}G_VhcRFF16u1ey)@_E9)IT>VljhiOL39$?as5Q>FoM$rXiy@#AR4 zfKImL_Zox8{znd(#egG92LQJGTMdBW&wVN@)uzzGvjJ_ zj{>4FVHaUp1#q@TGFYVPFV3!E%*t3zzN%W7ahk8Q)kaQ|peSTu4AV~TAZ2N24cA@M z8DKS(sv=zEafn^NiYyy!UgmPh;T=H8P2~Wj$)^1(=zu?5Ach4jT(37BhoexDktCzA zWNX2Kt#L@Ez{?wITJpmg^R>D$L93g1s=2ZRqF21313y(Jc&P=I6+F6MLrr@9t|n?BWdtY)?-lPU%P};GFrPCAnVOuVHmrBY=%bOMRW3`p zR^!}(of)oH7uv$R2PPRRy+zjIL{Jv;&mGPmsgQ7*O-*cIedqX(4VtSp^)fuR)_wKq0)H_~6-xH9pl%C&k9 zhwHJgHzhqqW{&$jw>f}Ems9m3ZR86(Y;4C^rDvE$lC=U* z&y}+YCyG^2(O&CUrnt}B(X){*hj!JHDqHJX+?`k7XHI%ro@@AK-(zB)XKGGL+(@bP zBYN8e$%&JD;+W#b>PKbV^WQnQQQ(wFW&1VrrKaymkLL^8b)GW0dw?83cf(p=ilnE; zqxSIH9I+_UAZ#c>;cfeO4S?4uDPVzA-wVhuey)rPy$A$h??N2Z7e-NEKC_Z~^L~tY zEu4#o^s!jT-cY`xrpSL9RkfNlP2a+u!Cq;k*VTf04NcLmklBQmiZyjcOd~u}R5zPE zTFR;)1>}~{GP=qM zwJ2=RNchgvzLN%CQR8Y2q#Ue?)xGc?0sjL&+Qo(N=X@o#6SE(8PRw(h0{TaaiOb3u zN8W#C#{)WonMU4U#R^CyR8oo&_##E+fEO~LA4m%#hOnV68HflZ7e!q+xG`SO5$q_QJ{V=>I8)zB}Vi7it-G7X9 z-UHGUn?Q)vi8V*pySkLUc4pD4?wq$fVbBp$cGMlmX3!BqaM>pTGn#cWj>xQzxt=sc zxe&;q$_$YQZ=kT6F)g;V^%6oE166@0g5t_nf3jKUAuhbBGRftfmDb!0E`bbwFmT4u zjH0b(u`T8S#(s7HQ<65Q;bMP(P*B)n?64i4rRA@!hSh+9MQniAFC)?okTYCvf}tCzaL1@}4fFJcH06c0!fIOK zgFR)!3{|ww8r_4kYh6A!tg|^-!-21w-GynS=-ZKOI55P^4{&y?EViR_BEkQwUsy(V>rY_C9~FVUCQ1892?2xnJ;L#I z2_0lG;^9)U1gYCXPGu={SIOCc@zs#qTmhO=X%&`=cgTQnd>LRb{I#~p(dSx7{`0_F zoY^Mamp;|4lj{Tg38>;Ct*mjTN0shdrW!hD8Mac|_i*WNr{3%c)1#>h(shUZ>BaD{ zM#Bxc*2NZxJ`3*;zL2Mv90#v<1!5c7E69N=bHW~KX9Kx=uwdAP9b1p|jPtfPl^m0{ zkjzs)VL_H0p(>RSIZEi1`A~4Ih_BQ=(q`5EWpGr8(JtwX*%JA#>4v3?RPxQj^6Xdb z8~(1t@^k~O#7`iD9m9;bJmp8e6=zSA)q`iL=nQv3DzSM?jq)B;)r9LOLTVy0hOO8-jj z(s3HJ^R(XS}Bcr%9CZkiL7^4kRoF330rhB&F zq6Rl8y#o@qHjgS~TBQH_5f*?K6;%4SLFo3ZcCD-?i z8O15XrUXluTktiSWr}m~YF}DR8;hO0wy%@p+ME*~AKs59yH0MUYJk)nq|vsJZriwa zZ)BIOI=q({x~IhW`Z!dddZwO92Ci1<sN+{!LYiUTOOlNq>%%3b$H z$WE!;C)5?$MzR-zNHW)!p@R5@M`D^I3%~RW<_Oo&&XKx05Jes%_nY`mSdM31f@hq> z{#`MntnB$H1=FjYBBr8EZ|ISR5mPPN065w!#ZEwd5Hs;I!%szd9x<}NS`Z+qV%(ci zfW@|mVy6Y`Lh@BS5@^@iVHjHy0V=It;$X(8QF1_4n!GerWq9A*S)H*>oxh$ybsE*S zol#G~+NyK9nE$l;=A5b!{m;soUH=<=TSnQ;{ME8WblFZ9Mzl$zFVf)*km0JqdebV9 zWAh_-2TC(iJpXl>1J!L4aU!4lH2Nm=>Smp}vMsC0K%V_zAS-x;pO~Vd0{0EE_+7CL1}Ou$N5M zPor@wsp{_B>dwz(N(3JGLfro{08$|9dM#6>VI<8TV=y*0gLI7|a7h4B*kk`|hYJ=W zj1g@G0%1T%_^n9EHX14V?`ZylhdhkM(?*XF!r&k*`Ci-;(w}LEzxzYjTVH1!NfuI2 zYlF9#qsOZhF8s2c^sTZMoj6r;q%M`ZS^7l~=o0F?#Ol#26wT9CePV{=}CcP z`Vx(i(DD6#_*-^Wv&GWBI(g1G`y}vOkh?PADKNRR(QJunkWe?v#H0hqEYl}+E4P~2 z4g;2%Ur=uNhh8`pM=!Tn+`y4ker_}V;&@91(9Ra=6EiHv<`$ts^t$3@C5_qO$gqP+ zN)_Vv!3#}9UD&6Yj5F+qg%SZ|mkge`^*&j6`Pd(|b!zqB)Sl6n+6qHE{T*rDBCMXF z|Fj;JSh3G$*_*|j7#aG1MYx_^DuIZ<3t<8>da?A>Uq=F}r>LO%9kC_u*{n?8t>m46 zbUA=_WBU;dZ-tO^roCe0f-tfdK$GUUuuE>II%QqQfn+;TC>4kv@UQxP}XPP9s%1vx+?% zJTa+Ovx-@k4t=yDpB&b3%9CBZQHDM!$r)6?S{Yq!Q2n)!B0aXYm;9R;b1&hh!D}h| zS*94p6M=X9&aE6ijMkx9w6Uh@S!5GY1=t~`6(XF>6IePAb>1_$*rpw9c3PWC8oW%; zgT946$2JSgu(|oQQf`px(GEi(E&=Lhr-3oy*HNYG*^o@Cp_a9WlpP;i>c>$|2($eJ zrQl$tf3X0U0B%1+XAdEBSEV=stoq2mJOH{BmJxPl%8TtnPxvdTwVjhD!Y)A#9dx4!Ggy8U1P7{wS5$fL|4X^s=BjP75cxvIT!V5xhd*xk7_HZs8FZ!rm;BVN4 zuqm{9a}V%BOh9E#{mGc_^Jav1MgSI`HL`Be_(w|bwSdU6|d*GI>7bf zsS`*8-g7TGt;0FFo(uM`ClC;?{C% z8S}E0D-JUWrxccBmJ^Oi4qze`fAGU1?JRHb?8rG?p}A3BDkRWjj`667LK}e#U3R0k zw+>H%vlH+d08xl_Bev&?LeOe zJ0KAIjJUfXv3*i&zOcIa8PkRjYl7;2&S+LcUxFC&@}X^6BG^W>&p;H)}Q0q)Kz|$*nd0L-~i+%-}wG)jIy2D-HF-`SE*rFeZE?o(i-D_W;fi26734Zt|*mVWm8(ed}jRzD2;ZWkZLFUjGWX zDS1n3&SHqN81Pc1>LK9m`S(Jle1k|auhgeQcffOl^NYz0mgp zr}?G#5k}c`MYS$7W*tJ5ssPAbLNdoGlNLtT-v?SIPfU%~I?fUyf&p;%O$76{VyFZ^dP3w_j##}a z_L<>eV7;|c25DAWiJe*qNrhtR-Kz};Pr1iWs$sdcq#>2gL@Gu%Wqn`grlIB+c>kEZ64h!*RGm?jmA7)D6f8~*btu)~1D?p| z9r%OOE(a zPz%lStYf=e0zU23Joq6g+$CjUirn0xCay~X5d(Fe@*Drwi4_s~vw0JSBce3woY#8)fnq>Qd%Ym5ALhh)_|UM<{A zZV>zp%Z1egWkbZ0H>DRZnACM`Eo4Fv|9R7WwUYhUy%_`N#>o?BlC^tY*u&8|*Gzq< za|j(qp;MyRIC_5JREbui)gb2T`S%Fn7{$AD8K3Zj$veTKzuJ6wo8_OT9!YL0(lZ!s ziRkFk(8Iy(L7l;kzIOj>?>c2`1ibQBe8W!pT*!TF1D#Pg>|~!17tCqV3#@~Bz4A=bZ9|H$0huV+8c={?@<6UdQ<`CLH4WXpX(LF zx8)HM`xMyTHO->TvjzU~V3Yy@{rLw$LITMLfk=>HWdJMxbgE7Nd}ROqx8!I+FJxyY zWNqN+sODf`XZN3WY$EcLvIFvnys7B#c@u?Uuhvm?d6^@;jErX?(c$En>uG|uMMZNZ z)n4H=K7GkBxuh9qJF!f%FcaPF%d$9#B+ zImuAZr!%EUXDH34F^usQHAS)B19oI2@~TK)z%8I9xO{AAz_aI&sMgYR$@!Q6xGwIpWIgPn`Hv~WxYmtOeWcU{N zlSozitO2kq1AFmL16-kS@SK8@oL0N78`F!K@2oW34vqa|lRaFj!#{RZLorr3rZ-`T zm0Bf-6(yBrf$_4Co{4G{r_*fyM+CI&Jh&BDCHwd&pTh})kW1BcnR`|n`U5cHgc4!+ zszni`Dit3K!2Paj`*l?MFz)c1zg+5Q2t(TZz=ohWr#( zxI;{kksI0B3En$SzVb_$AO*}g7}P7}KWpoqxaC|OM#f7jUz{?juqs*#nZ?CtJHgIxS(ppY_5l!f&{T@oNreGur7hd%|Hp97!lumzh3>PaCU9-EV4 ze9$ng&H(-*mPZKO$==j2+F_VAqm&>FrYUQjnf`X^Kk0B?9C8DZX=7eAboz=g=?x-a zZJ3i-9RXP*lFx+KTvZMIURC63^`Zl0Y-Pa+=?R)48WWf)j3|@qMqnWHDYpq;U8rhI zI+dhdQ{Xi9H9%^I*ifq*`T#J{)qM}sM+YdKsX1_Rb=yI3C#khDQ1*?!=;f(n8DtOZ zz5eOTdJ(}|w9GURK5%xawSj@A<&f$#vrAx_7LEGs_Cc*sw1(}c#^Hyl~14x4qt7^YH$V(Ox?Ye+{o~i*!5)C zSo$8y_x2h#&EakgsK&U>H5~38yhv&|No@;7unegmolCd+4A@XFmHQk63E?`9rEe1c zg}qj>$Z$$P3lCbPl;LFH;KcK8)ulp5BnDQ#YLpArK}*WN(;J(0;Y#ILQAvezZet+% zmjz?EIpe*2>)5M?P!txHsTEg^YnO%NE95jx@-@3-U{L|`hK%F_R$GWsRGUJU2#N=U zN7Yavfp^YLK*y&T-4QOJj^0I~+AZxbE|nk|4p0_&mz&Ew`e~EDK$SQTXGwnW#s`#_ zF%?OeJ|^;bVW8_9$w_8%N#RLDOK#q#m@h=M1cTd*gMCRdSI5conx{+|R*q9=)BC|K zX)*T^RmZT#2Yl%wP7D;t^=?J{cp()Ia>K$E8_DMK^Y`7PQbPK>K!CjTCbtCeG*(|y zq$^c#N&fhVNk?$fT=i~`XAGD{nPXoc+G)coy4im8Az90eC3Wd@l|<3?4CSL8|6c^> z^yKU4WReOF4lo%70-39{Vbu5?AuFDY=3?4+2@YzWEV^6NQ&yzH=qYWgX z4bRb183e-IhDwJlmOr(+$5G8!pfix~EAKekV-crWC&OJBvzLApc@>t*ob04L;l=3tqYFY8!LF%Z=OplAW-F&Pm?i-BZEJ30!32ftww> zy^${8LM%3pB9|%-3S_$WS0mk`Ipo`-KXi8E{WCGOlyVD>WjmCKsA~LPR3dw(saKFI zIKM4@gqJ^-v$PJ_pJwubwvE4CWZH>n9e3mIBHG7e+6i}+aBGcv+3R+a!xU?#kamns zptw|*ph~%8mfrNfwCCR;I|PxwF*ET(5>(~;-6*Vp*M9wpu>oLum}ia-%0?w8&&WAzT1-g0z} zU;3o066D4ifX>LvNY(~TwwbjT&WkILP~6qQ>IL>D+(F_e-(h-sf$65)Rlc$YX|z+< z>H3k8<{!Q$Mafy=n_d&0^f4pn#NCnR6f9)$>9bSS^GUKm8@&FDIe}=M0>xRK7 z@eZBr_H^7rr(J{xm{HBWa%^x9N7s_WUhMnvisSk{Z~3w(6W2P&TYm3mTKnjw-hsKL z(}5+@q&XGq%ns=$-O(=4QA>_SJ)Gmb%0TIrwC}eXzxutYo>^MlHC&G^UOu#EY(f47 z@wU`RyU(L-D5RA&GSMd#9!U>rxE4;*;THUuU_p3JY{Zu9Xw(?!dS%;=`rD1)N@7U)p z;#bn+bdG$ryHGsjg!J!?^b;S&BYPy;i61viv||~+`g}an`KA0HbO70m2Z~`^M9jk$ zd+wLN{GY(x#2E~zUiw3XJq%6<=D#&bDHD`joNPfYflBT|SI3~~a?Uv(X0rOabRgvq zRPJU}d7_lw*nrG2GNU!qB0fB{aoveF%h<)muop*{7unFkNmbix^;YeIXUo*h25sX4 zhc8qGVixj&3{}@OZz28FcQm6dHs|S0huy`K*vwz!%n2JE>KFnZzHv_ggDwUeaZn-N zjiH7FfR6S;$N64f=8xZ?qf}~b_4xKh`HiKGYb%>aCifX%wlg=D5SRvYX!A*hM}rUf zW>b2Psh(Qi^6^9C)sBkX@Py8+rRqs(Y`AjuOk1-wu`uLfqI*j`E0|!$P0rZ{XFhj( zqs>%x+lG3OODg-^`oXp3{cEPS|GSLxxc9U_ot#qS>b5YE1G_xP(wPg`hUT`825LCl z?}qVgYNx+TZ3_9q>a6W9#5f|*;?5(gxUt)km+xlmVi}kD9m|u%^l(g?5Ad2GE7xZB zXaC%o9#KJJV}Bi5oEb8199)xv!Ci~0Wu{i#lO%R#moobIm>VNhj(?v@Y4RG}p}%_> zq{!5ubJ+*fJE>_9i&8)B7i4uzOR!ZYJVytLBw@czJBbG`+1YlgmBYA%(&l}4|6V^?Y9~No|M=~8@{;*eLA5UcCxg%5Hu&bh*owbB%2 zu_-=Ta>EmMJ518HE2Q9kdC}#&L_j_dL@brOEx?NS^K~#}j95JO!`?bSM`v>sdlZE4D z1DhKzSdJFgRL^NKf!#q%m<|x7nR|hPvrDMIA=hI22TGLoc)&!252$o4*sW|!8i@2;sw)FUD+1LqfdRAJnP?V32Ca5?GGa}XU`;c!hIV- z*C@)~$udZ5$u-k=No5BbvDhcYUUhL@@QUA6 zBb9Wni(1;9#`?=7gJeCW2Z29Lea4AL>N@l`gsxRJwQ`TQNe#BYi>U1^NG=Wm86Bqpf?tWHDpVbHSf3XF-WS3*caSD>f0@6J-l-v0aI~V&!L(0T z8l7@S3O-&=MJ!(*)v}zLmRIV$bIN6pQ2fM5pAFA;;VZs=y-A_M0;SOvkZ#!VjkNPA z{&tK0xk$QV+FsAr8`MhCZ^O9e`*hd(T+6R(|Mo_rD)*}Pnd?f-uC#WYB;FUQrI=$20JzT6aw4g21U$a`)DX2HDZ*&8HOg zX%HX}k~(1WXu86Oxk%KzKL>%rVf+P(moebOz97^UZCwv$wZgojbqpIdkR@x@ryDWT zVqh+JVw2n`B;?BQh=$vtB;?BUm~7GH<|Av+D@_J>(hG%5gYd?0fMvqQ*$UUMVr?#g z@IYIF)TRkh)MeOa3R<%)V3h-aTYCsAv#U!=ofS}xWE^2S zSK@?Il9|yp;)zbAKjLlL1gnJi#7CNhoulH=CjCtopkEIpb1JPzC_at&YMLHL@Nt<=p2qxnQYF-1B3^m6C+#&9F0yFmuSd)@=j2Kvx)-J`dzovjQh(j}Y z9E)uvncQp$Qw;{&GAIqO|F{h8B?V@l4g|o+L;K}a52QoW+&}ZpAT$4uho))I2b*=jm6qKwdjonSvi1^c*?xJ>;Az z@CNa3-AIrbd%MKFXt}*V=;i=z)E4L;$Wg9d9vEnujNb-lG@dqXEn*v2a94N23i*u2 zrYYG)9J zL0uM*7?VDGgJP6lENT0H@dIuS_qwrKUQEt*a)I^4Zb5Lpc>56?ZsD*#*-h7aC8DB1 zY-WVLBp>I1Ta6vOG^;@2+m?@b;(pnXTnOB%bJhZ5wOA;}>fLg6{EBOgWTFFVJp zq|))hE!mA&+O3Z)3|Xj&!AV$i6-nQP!N3eXN5HxzXur+)GM;tZ0+Mj(V69td{t~hD z#$$PB*7=vfzR9VnC%*bc%)tXH27iwC&{#)G1@(W@iT-y*r&z_y5D5YZhy?L}7~K8u z6&+y*i=Uz1|0upCt3kQqETesmU|WnFN)Z3hkr6YBEq>eM(zHlOWYlpqz&RX&P1?(= zZ;hK*NzaW;QSPOr6bGYs2(fEElU>GP8sfl$ZZv$y+wkX?UUh4Vr5s@99=iXOZBlRO zZn`V!zdv?DfjoXl_VBogc8vk#Lz(p$qhwnJeUV1eTNQJsY*4MtjN4H+f=XwO1z23j z(D40n41<#rraV;rM|Vj9K-Y2cigBW%0_dDjuGDq9L*#t$?eKZ5bs`X~q!o+IF|7XmQ;ez|VSFV=ClHiq7uFEXA1bYJrQ zQ)rJn7_8o-J2fVs=|fv%o{1hl^21gpABz2#%Mk+7PlY~gPY*(LEJkZBDeM-%oU$C5 zP+6Q5qYKTknNtUiNhmDWzoIjYjab|)>N)r`ndLzSc&LP>rHd|1rsuypEQe59uofCq zkeC+RmCGzCQl_;!1QH}YHrQ}Sr%3-ws~JDUq%+|y)(QQKF-oVnk73N?+z3k{0qYSm z9dW8b+k?8!Kqd@ngbG|z{k?&Vk1@@UFe#3SHM*oIlyPlbtpV)auLFnI0>^8c zK4VR(NFl76nHYjHeW4j)T}=U3+O((%Y*yALN$X=+ey7@phEcDAk&K>z*V@2&KoZvZ z^N>!Bm`XBD*Bmj>*aU2%MOCRL4Ysl+FC&v#Bv$s@Vo5Bzz*cG!-V^bvgtbQqz(SpA zdk<(F$Wg!BtN%W;I?)ut1feEhH{Ny7P;-wtSqw3`FlypWoY+yJ0!90&bV2#5-ht*Q z-=XHD+(t#1-DO3b-u+V9g_b125;Vi=VCKwZ)~41oRuTJ4xQwPSC7nBW0WC3%)iuQE zuHna}pH8YvO8`|{=7v@);ifZ;)1N(iX*HdVRA~Z(wxl7K1C27XtngrOs&howB3)p!O99Z?Phkal(gOqr(t$ zlHym84O;F2bHiyzy|PzK>G&E$wLM)=o;o}NF1#9ysV zGzO}y7oV;m;2MIluz0&Qg=V8^vw4H?P5_Y*4i^52AX*msO!uj=Kf&0-=uWbye+)Av z#Uc23MxVE?wzTUG;5jdiOAFyr!Ow+7K#+wWJ|y~1G03h+HZP*hmnH$MMepLjXwz_u zYy(p>)_1-evLw}I>nO3b+`RHj@ zT_<+JOn}wmsxH1CS%Wo6-L2w+hnFV_L&d`=Xi;Y4OqxA|bz(SF*9;+;?MWu_lyZ85 z(|~;L7^5^{Wn!3y!apY{UFmD8oGlb*h^0T|V#2I#w~Z9Y!!CC&M}8#at$za3A`li_ zRMmmYU83~>B31!;DHqU=Vz&N8@x! zMp?BV&pmRV?Wk|gF?*{dx7wIt8B<~-^LFMtcs zG5@6GRyuI89O6DR=s?G}K7h#eFpB?xvq@|B&#O;YEwd+j2QM%!TeT^$Z*v3|JVnkp zC};zkbq^DEJA}E-4i)yims-Qh4rB^VaW8XJXO3;gh=BRJ_{G?gL*kukgyRxNhF?3t zwcu+69PS7C58)8fd))(KzbF;;0{Pe7jC&J6|KGZxCp}?AJhB>hbW}ZAodFU~VJF7C zt||7gdU$!VsOs^R7lfgECYOIgZfo*qS9o<8K_1EVdRLfCy@)n>T#ys^A22hv&a8YB zVtQX(ON;mDMyWOAml*>Z`}o%%Fn6y!!|?UnSRcBB{8351v5Vi-f-T0SS>sN{dfS?_ z#G=l_>q&dWv*B$<8!+t!`iVGuE0I~6)}se{y%GMd;lN)j-Ju!e{Rhw*N*OY_y@>_o zXV8G+^4-&vCltIj5zI-a2PREjQ5v7zGh!_-$>kQkWC>@pbj?Gp*zFpfUW&R>9$ROl zXNWFcj3Ty#zb;obwfo!hLVuh#ZJ$23;qsOrVm0q`n~!=e$9-4*egFoeI%!-yg6ehe`f{s%qlzbGaDkyk|xjGSy8-2WrJMk|fm{NU)kugHw#$n7s_ zg8gl3t!t*_6R3Q^P@f_$~G>=5KS}13OAFxW?{_vL|A=b^$QADed+(#D@g+VIQ zpaBMjnwyCRv5WoSinp?;c39ZMAb*lM5fw$clDSZdIzLn^df)`{ILnZ2i>x2I(uOZ~ zEe(N#@z!Nf-P45D2Wcf?y|Q>bwr@g599WI_W0O=xokF`{UtFT0qH~J`=6oi;<;8I_ zTr&d;>5X`M$OU@ftA8FIMjIJ{9ied69%}<^gYUj4GYtF_7L+3(V(I(W!Xk-9pc%K( z2`OcN6U#mS(b=|WnKc-u;nefHy9)!sYJI!43oos80%RzqXlkS^oU)bU`#B+HG$)%S znzy(yen&G_FK4F2RLvz6pjRC7mFqiDKT++pRL9lkV0-9PDgK?EQ_CBP==68t+)#3y z)6Q4w`YGx>Db>Y>i6K3x|0ZW-M|SmVlpE_By<|7eRdZNZI`W+x#*ohLdkY^39eqM+ z3%BQDBtN5h9E?Ei;?UO-gu|%g(bpx9z}%3^&we*ERs;sq<|(AdcV}ZcMEI|AsW1{` z``6F58vBt+i;gPQV5u{dRQDs0D$AOff1SKZl5K_zccOC;MVXDFxyfJ~&&m3^D;;FQ zV6M=&r7;Tyj>MyA!CDLCF8PuOf2Z}0hVlBis_&*4_FSG9*Nc?vm_VIN z+7wY11BQm5Bxypu`I1IMNuspGYIthgfqRi$QD%nL`Q3hP7e4rSzyVT4NMiwI#q|ghrzbw zdEU`Hij+N*ZoU-oZ=^~zD2lztJRf#CCvR+SdOkmP^}d1DLiNC^{D%(H{J_O!ZVxzc z1g#-qScKh#!G`Pt0HSVDY(z2jeiSj^uaOQwGS94 z1;$9FhZ~duCJt*Dk{4LJQHJW4iqA6X74+pe5%gx|7R!ogi>Ix$LcE!cNkr=jgAOi! zLQ>@M$#5c#A+X6DoKD;@3anC$CC8)^@z6y>TPYISNpR6kp~{PjEF-%r{kA6!8ryYV z3&Lc0jGpl28J2h_bEi7Z6W8|eHGX`>amW2q5@c589j5ng;X+mm^2Sn7XBj1=>ujM) zx?;2?2~$`VtbY@Kg)oVfR61smAeVK#>Dg+8K&{(0gND=qhGsWM7YCAGBWjpZ!4@HnVtT+)*J^QWby$l1Mu zM?ZS!+8)W%J}K~q4}_usqMP`=PHPDNRrr-rT*6*AC*IpaL+Y)-Y*s1f&P`*8#!Y1i zsie49Gg6TjNRKpC0hSf3`0%{Rr(PYKM;GOeYiIASB&?Nz;^0r$;Gqsu>K%UUi-?yW z!uf%^KvPjXD|mnStZ8zAJ`#aJjA0*x-o1r`s7c*8Clhz{*@}68@`N&0N~{P^427E) z@qv;T`GF(&8H^W|Ay6pCr5^CG{`_0xX!1G}rW(WOAr1Mh6?(L-n-|Ribxrsy{I26| zEg^G8?(rkTHV&r{(VQZArYcjCkx8A%XP-V_pqa>Lm;gRMqsi*}%q)+OK_pI+#7w&Q zy_Zm-qI{+ToQ_dKx#)Fz6WUQ{HgeNcM4rK!C=wu4mcoO&u>RQTXf9_(X^<1(n+WzZ zUBv3*`Qu~b(FrOSTJgZFH^8mR;?=IYngOM`zN#-%>d;{M=EzW5QhJZpR;j5}-Y3~W z#mQkE6)kKOKBmsWK@5gGf44VgN6EFnv~dwe_=j%7!eIiugGkaPRuRvt+`j^|1~eB9 z4#J0$ zjMCgY>N0xdK#}DDfh;8O%Z(U0nxe>#c1VD9AYk$yuDKYruy5y@SJeh|)7TcMmXveV z+0%mEPL}PcXc)J59fvYikq67}PwD6#wtRGcJbwAo?wS2|pQg4?37wcDbM{R!g)7MO z;zUrUxqo;L_nt?}9D_SVbnE`bzRln?xSSmQhbGX?JB}4Oj;9#LyztP#%^FrtC`W8F zjCuNK2Uyb%z`Bb5&lWLjr^Gt&4{?=nLtKOZpEl?r&_DPCMw|)wfQ8==CDbNT<`nWc z*T2{*L&xCf7ITcEqeMrx1enDd=N=%od<{uLGu=Jlx|o^Hzg_LTRJHHlaFQ20gR-HFA}N+OFiBwpK04R;ypA7Vj#fFu`U5R8|0msGgyb**VvIf4I+vUmLMC5*OoW9;lWzu2~IJJ~UJY}>YN@7T6&+qP}p z^gXx7J?Hd??jED+52zY7YQAf(xt?`^>%1|E(8k?I=uG=Bf@b4tNaUgpEn6ifq(ih^ z6EJIPaPtlMUxj8ic86!-N3u_UPKN)>4#EHKwn*EUm>AjrPf%J^{G|0fKT6OvvGYYi zaQe5LWbS|!(wB6cE~wzIumI{gFb-hUP2CImy|G8(Cgki8fCaxGR4y+Nd{7HAWf}nA zZK^fL>#&uRiO<`c3!)~(gC%n~h;qrcR&tRf%n3Gy#8SMK&`5DinL-n@%B-b#reWwh zk3049-GK^0Y$?9j4dq3pDRo*TT@#c6=Y}K(LlTlU;EO#ppLi^XK2h^h^|z#IV5Z)t zguj}hJmco^t8^eHCMGo~*9wd6uVPr{S;mf>LqV zjd-i(!i_k4)$l0PemEJE9BGpfbwfzPLuW_kn9Hq|LWW}Q#wYY1-4=4!CQQt2Z{YN=kWPU3m_(_F$bO#XYl~27pHk~o(3*IJk81$B(76_;b z`~Sd-`;U*~KMo+3=rFE|hmV=8r=1dXg$2HWgiQ(s1vI(jzrgVdWe8%6W>CxpDpL-c zV2RltkAhQd2A?%q*;1!uLM-6`;WE8b!`aqXpN<(G-E7C&vTU3ESq|4S2~D|5-G}dY zIp20$-yFx^-|uriIc{)3XSd#D7k&hg*dfSpE3ng{wk-X6*1RM=^|A=AZfxAg1xId( z+O~bgyG*h#)ISyn-}EFJ_6#f?r$7Lm{XL1OS6Bue(|)z>b@_&8R7Ukrv$^G!lx-Vn zcl~!aQIAU&Jz8+fYc>!aS1wYp@C$E09{X;{O*jB6z zIwCI_pjH0M>O~Nt1~C9{AVKt{yMHbj4gu z+PnLgw>qbmgLZydCb*1jrB*z-3$>vtd#K|73~81~XsJBP*i#qMBh5CwfOc7L58`y` zj%=d}BkXN57=<7SrD8#m?QP+N7m4-Pv8A0||1oW2i#>t8;k{zjhu0dj>WmX^5*E!x*`eGAG*9;$ZN|Zn4q7{hrV`KP+}_%%bVyipF>fP zAyvvLB4K81VNP*lE@e8GYfw#kOcbpsAEK_tz7B;6Eig4wiaJA!y*ayuf?*?ANk>9l zYEYfk6t&FM?xfGr!3jjEvBVR=u=1+)(|3o<^^cX^)f8hsRXnU@An}(|>$GL!NX?wB zu#|Zud^nVMS73S#q-^h#NSv?-+dEP9Kq^XwlQ9mKjpk5Sgof5L?=E^6Pe)R4%pxVP!5+P!HO?b+ zK~A%=V|2lyDks|#DN5B>sF)7?O3V&KO@SqqPa5)&?cM)A`txt-VrlRaaSo0ZWF$!SxqkBBMS+ijO<21X zp4@k(l$VAoOUdH4@Yu*V+|Vh#+R)YqM=C9{{_8u=cWSKEm@G0|lO&@4TsyMRyb#Yw z<;FCU#_0q^JND36x^g`Lq^f~?k`@f>xE<2HU^H6?0&QYFqN?~EdTxSVU{^W+X0@ID z8wuz++iEcQ+l>tFX9zZ%e(gXrmXC5drMm9E6RDK}7$fz4&bNjE*(y9F)_i!qBeizb zLU;ESFA)F$P^V3x2UCS=8x`oi65=Zhz04@ZP%nYR?R!pMAE~?aPV9;a0c{f(2y7|B zZpHm=N}Xpqks=FNCzRy1hnIJtkeQR`&&KsNK;nk}BVh{`O*p~odNM}4i4{KdZXj7- zK}93l6>IeaYXroJ52mxg-nt4gwvwa#y48-Gm#WL?L%}fmElML#f~uY)F3O<*jkiwx zHwH`Gfw2`w2+4Uz6wE`)K+M4RUv+0^WtOgqfk%!1KfaZ@p_mNc48f4Lbsf*leh?eG-n6`I{ca+nt7K6(X`oNSI$X*?sr7V2r> zk+!*`N8Rc`ubMx7hi2>$dG+c!8ftL$>+8cF@YW%!G@)6#dD6=%L@;c7LtV z;m?xDqtTP>!fjCO(yoy0SVKSWB!Os?>@K>(_Er2nYf9Y8nUk;(iLN(Zws+Vp3B@Xf zMADvq+wx`m)uP&$abpZ^)2j*IA=9_69>R4aE4fk&Oth<20Mh}#MYp?&^5zQf(25o< z>57Lfl$xR`NH|);mN|n$HyqZ?aX{AnP)Fsk(qy$MS_qWOt#gS%kje+$r2GiwKw74& z6tt$HQ8IHVR2QAmcZ_Q$wO}nN@!D;GO zeaTEoG@ZVKFK>~;hLS9$LRKo&*`z-`$&|_2Q0kO0nS)>WhnMjkNV0a2$oV+%bbSUl2%Q#7MC4L)_-Z3yHRmEI#seIrtJIN#DK(*CmS8LP^+lbG@(Bsfv32j zeU#{6eoqq)5|hn+>z!q7%~byRY9SL>7a>Q1cX@jmSK?eIMDWA#jNfFg`%QTErl@Vp zv}rjN9g(h7gzr(`u}o3$c~LZ^TT_|eUb?xwvU^@6_GDfruz8NfwS*rDzmRaV%b1H< zJ+(GUxnA3zGZSR2Beg%o580O|kujP`Jo8SwDad7nD^d2HeR8N~HU2WpzMtihWF>+*6gQj@FdJ;Rr5#1A=PISLBNQ)qu)|}c zW^in>tx6NSosiG@{jtQo+cGpg_w+P6iofUs2M-!a+ePHYy7((V`UA009$3FjeL3Bb zB{v?=z%zr)>jb|^VPgwfisbztl5;;Ue+!bZZGWRJ&SN#CFUX1is;$tJEA;1Smf^#x zodLi5Df*3yg>*F;=$vBM$ra<93$o4u6?fzF>_m{9cs-DbM=%f%s2n|15N?VU!UZmN zAmKDUl$H=aoPWP!0z~n!vL|-IKD`lwWHiV(<*xUN2?N)#zDiVfvEG?L>bNeD6@!-I5;z_C;& ztc#N-JmQ#^OCj5z)JASt+Vt4op}|#l>TAe7(qwk(iQbDiT-+uhB^lu|Hfde%x-r74 zeJF3b>f9-^l)GV|DOwym^boZ~t8j))o?faHfs?3ETst`|0Je`_Y3^kLV%E9JAh@4~ z`x%PbHloVStmMow!dfrbKmzk9gJ@a(_cQ0s3>TfphCU|hf$6QO_F_4iLT5Z5ftoNE zg}W74bGv0_RS?RNz@JUK8#%!ibJK;hQXK!aB}`QBp8Sf;Ryo^GmvT5MyuH-nmGmpMgs$XtW^uV!60tQnEp~-r>r}`N~JL zjwuFhA%X1iGi0KtNPQ!=WtbPHkk)8z`?!D|?y04`;ht!*8o$_(osCbmzNPGDya>j2 zBHaM?-{Wn6$3y<*3OLT|g40wu&VOZXE3<>473}uOHt)2Bx|jlCj(#ae;F6_v4juW{ z65OA5r+!hLYzuzFK6D3$rl+1DRODILukysS@z!ffZKBp|N^C9~yO269r*7(>y+r}l z4zvgD*RaB#&moLRFKC=Qkyc>wIIdUNm9sQu#DYZbKZG83^xp&?Zr}$=67(vgVD=*O z$1qf>nW^#p#u}lytYqm&I3vV!F3>YzRbsVkko&!yz?o(2Yv{@c{YSlnsv;_5iuOQ3 zbZ1t+u)>#b>i~skh;|Tjt-r6I1>4B1g*|`+Xt)A4M}$++H-B>fj1w+H#}`5Mh1qfy zzFL#vrcAljhSD%KSPD0oXTc%US1N9-g0tu!-JzYH8z!vl}z&%{X4 z#>w8m=s&6}u1A|?UI+*XAqZMp2xACmX9zfF2w71G){Kdc2_!C22$iT3@=1D82-M?` zL?v7ZR0tJOh{}VHzn^srdGQkm!k>wI(w_$(c}kxX8~)f{DY}2Q7xwlB9u{^!=JF1d zL?M(72L}tm!Cys4-pI|l{V*|5Fj9S=0)8?XAeF%Jmi*cDawx=8RU|a^S$=jzYS5tn zZKN%uqJ&%-90+AkoZg8n_^ErK0ryx*Mc=8T1K| z7cS34HJp?nvuH%r>}3laojeJMHCnWKIodmnp#R#V=~s_4mcVqQFlK2D>}HBLU$nJ~ zE-;2YqCTK09|$IwDPS}#=L`cvR~slN9}(;p?*d!(O!IW2NA?YCPq&aqavFbZQV4NS zUK08xql#B~E+v(Q=EuG6?CeNdf;Xc!8w9oVW%g~X-V3q;jRq}6&jC_RbUs<epYBYsZYO*oS$Gej`=d4m_NfWaIOv zqbH7~WmULoi3p=Qpm0>$gR_O{ zl>{rkyPCB2rt3+`%YRi_oiF~eANbO(%|=vpyP8YN#_U2$Ch1c>M(Q`MBcPJIgceV3 zb>Sh@^0SdX*MSh`nkL{6*%`=DM>NE+uuz3kqaM2(y{J^TC8{gC4dM67>7=VX;5Ko_ ziDh_v@NXk3{MrF&(_fJJEoeKJ2hf=eef@~9s7k4~NE+Z-^9)CBIjH*P2>|XkY`~|k_!YzHrvq^_{_YENvM1UVH_jY|> zy$_g8L7Hks)4agUBPyZap022;=I`wQDeYj(w+Rt z0^=uA!Id`Mn{pL?TH@Te2~r#&9%~eq4_oJ)06fTHs>j~8_qbYO=YT1HaI`-PI$w@> z$3qbM!@t2ZG9k-OX-&?I2KYs7=$(Yz+6FU{NDtgHY8)>A0N)2IwRzUK))A^?f&#Cw zM|DH$g)G~gZH@;BLjxxIt^bV{L!{`LQwBI%gg!osZNR9kNG6rpt zE}6*VxoQTR?F(4MkPq_-!EsDa_DMbY1?E9HUOv8U2+bw*Y|}N&k;8guS|r3;^R73# z@N)a_fo+4`C-uWAe*SpIlf@4FLfK+SB(W@*b@y$RTkp!~C{d5>( ze>#k;|GN(3|0hBYI)+9%!Xp3qN3E*$XI&D(t9s$PMnNYWEVKyav8fH7)UJoHR%;zt zUTf5TPvRn?{-|oXpzzG6+hoZZ3F=c6Bxu0H_-G=nJ^@W%t;{G-L^}-Tjrt3fZd3T! z#gIz5%sXu^8(mSE1cnnO8bv}LsuD>Z;98cLM1Cx2!p=PnvxYXE)7jJg04C0z9Vky- zL~PzlJ@1*VNQsnY?G-z&E?=C8%j;4b#EitOBluRd94SXKVs zrc_#S2(oIiL;F=46Ma-KcnstBPqM;1DL(dpr03uE1HKBiNO3q87{ zoK+N_VZ&YG7lUt4=&7Y-nfTf3`{l_VXsrIepnAZ8-rL8kNA`2VFas zR~GWbGId%=C5&xPm#!Cq22^J`%@Wvr)DK*_FGeJJ)6FmMjw^0L;Yen(@cdXE6eRQE zI*07W;!@UErz&KMlxxz>8Hh2Nwa8W)_ID*DB3%=Y*H1=lGeKUP5bp0v_#L#E7iEHN z%5XehRGA;%9Zm(RF-oLe*j|A0zlE2t8s?!wYzz-P$Oo@+`>jjlxN9mX?b;|@20jaY zkKPzp3z!2cDBu;*s0X$9=mVjImw`8ji_67`GKJ^Afia$^X`Ux=8)i;+Y>2u|KGQIA z_oY!=F)$oPlw4KIj?^cn{TT&aY_nS=?ON~ET^ho)%hgFMO3RK1T9x3_;oU4cL@1?x zMNMP1=F7S;Vh(zvG&FXy^#4VH#Dgwg^ix)498GRm^eCU67}yPoyUgaO+RXldAtPIt z6m%AM>%DK!F1hK}k3AVkhLPU&z^w`2N#+bV6IMnYkr~m@8?xWE%&u?^1vZsDCv_6} z%OaY{#1PY1upOttzy({ZnFhH9;I0^Lc4AI+rn7k zwV1Ar@T@?i%$bhr^`nw=hClOEKD&cZ+T+HO3}|CM&7%HUC!H%KZjxLJd6O%*uP>qG zeskiOw6F*=6wD3sE`+@%ABa8C1J@b0uB=xfI=QP?)bB#M>N$^X3PGhnsykbS|a{!jC>-r};uI+H$qF>6l?LCbtUARYbWOt3_d)pRM z?a>X=2K#(N=^mAw9X}wxmwHq5Jr@PB$D6Z;mNbps#P?CWi`Fo#`Qy}JYgb|zr=Ve_>S*TdT9I8)N z=&lE&fxHW#womW;*OD}UB&>fVJaPi7VxNfUPCAzqSZH@qJUl}*E}rfPt1B@mr^HZV z0#lHgcMwdC7Hu2eR&iw$WuObt=!xgB{oxkRtK?%2;m<4J$}MhzId{3}OX2s@w*HMs zq3Q=(mjzS zX?rwwnWcxfQwS(eOvKjtPiQrx=#{D!6SFBkg;1z<*@T|NU`rFCus%$z8AHypykS1A zH>gp97VO$zb||RD+x&SM?rPP#UE=92d1=%%P!w687rm|wLv&vN{xe|MmZ^bS5w(AMmAQ#V^xo~27^X5qS@ zM(J_4s-IeLZ+-nwYjyrdkh==K`s(nf2&n(RQWzr#9TCO+cL(V9jEo~w?htJ`2dTOCl+E|bNJ`*E} zg|l}WG2<60Wh1pn*_V~9to0wMWQit5_aI-w1OOFF|IfjIO6QmoITCl_(u6c!GPzM0 zEnNKqcdnnsQSXR5*)1Z$_KRJN^{{L2k1>>`vFwN<*X z_i&&IJ^2jj&x*#u+P?{qqzLJH5}*lIlAWU4$w;znEH_#1t}97)A?UAfaY3JD^TN`; zq&N%ZTvu3I5lU57FimJ8HQvzL5LZSg9=5GL?=SqT)*xakCD@S=ZtnD;{tmoXnUv?s zNzGbjv~$)k{B41@U+ll{l=}mXa%$J4S|y&^ke68xsuQ+x#`{PIMs-o!{jQ}Gpk4L` z;P(a|jpu25P7r~4*;B%hhY8}gaJW2n)I5Smf6(j8=iB|i5b!Z{xV`}07a)FxVnc3! zSiAre;_4Olw+YOxbH?R)k^o9H6`5v3_=+L4M#NW!eXq=~S(RLO`EjH>lmi4BWV8W| zuEnG5laE)&^ohNicHrB|Yl8XBE$g0)T!@KBUmg=_apKuxis$OTIW7#e2>lYxz`i|l zfhupa%^wO2{wY$*lD@eP;|ipDf^M|G1YCTbiO)5SSg!zE3h{t%Y zhC3BRi8*X+nXr`+{HbRMWwJRoi`AI5FRdTe`ky;xaq^`gwf!b9*IN?nv8Kr9Y9lKWyJbL=3u^PUy;dM z3A3VV_~Cw*Pxq0_Kh6}WnXlpWNp=+UX{u^Iu8GLzn?*gMi-EWT3~Eb5d9Q?@Uu+m(0D1J7tz)AUP9j#jtZ@lukfu&sniJo+e=G zQ6O5YTBe0U+?n8b?fG*3<{dtU>?`zMof70t8x+(H+GP-DYDgr+)MCm{j7rbMiwx!# zyf)qU?@bWlYrROj_HT&~Zn+=g&iZBS0U}yR`skZ~&-C>!_CSf3DC?P=Hb*+FkJ)fy zmGPdy@3GOUuDQNanaGp64HzoV0zR{n6pUT-%)X0^{o_Mzj3 zB6a;qwEo; z_nd~H77NHbWIAUY;adYJ4m_N{Rf`bNj}od0E{s88$`AY9^9O56-puuy zMtoRA{&vkJq|lVfZrZ%byAWqKbvTj6l&@+?dXR432YBu2it8RZ4>j-7>q-=q6&Z|{ z7)i{da+dTDo@pc1m=j&Rp_(NpOcvhmKP0JMg?4-018~=YiiX?99fG$S6j&_f?MY2h z3TBww_GHvuQL1Ehc_Iqubhz0n6PslQYWwR&iQQbMPEe@NCo4(Esf4TCb+^xkYMH+0 zLMg^d)#tKQqa)&cR2H7FK2#a&qFn-J2_NQlH3o7%99d)dxXuk~yp1Lslqi~vqex|% zn{*_;3Io7i+W?YwH4p@`W7#xr{Uis2a6!T+!AVC5{`y(OI-J&dfh`(!Ak^hnhOv&` z880H}hPf$!Z2lS|hf=$4*~xg!-#ofL9POzRl}5J(pV-haF|dfC0}9|=E`wx3pyuif zD@B1Ms5tkpd~d}>sE!2C58CIlE>o(GEUI?z*~VWM<{`hA+&IPl=r&b&Xf7fa=;BTR zayR>@JE>+bHb93DXS&@{dX^rs4ovKUp0D%q3J_sm_QDaDV(+rY0HjD{ZU|yq9jUL{ z+QN3)V0w%$M?>GM8-*^)u5Q_cahY*p;9m$hkvQa(=WliOMe(2!pvGQ4_Xg;+D|AI{ z+K!hx9DH&+-@}umvAMq}l@;N-t76FGEH{@HlJEQxVPgdMD4NbFicYI8Wh*ot0?b`e zT+&&x?5@PMoh#36e~I8dAbDh;R*Rp2%U_9UGe1HnPWt@0%cAs8UEHH0o}?6M;I~S! z>|a&Qx0==GrHkiw>LJkigtTPnP}fTUUZwXhDu-0MFADXF^U8dmbwYT(dISIaEa`y8 zvCBEcN=dYu@)mA>#zwW}>z*WSA)1+Y>VaTfAx-jvK)BXaamAoA6iwJW6$#P^WErS* zgq+DQeq@MGkafhO`lo;(iZ9@8ZDx3dc2-3vNFcL%W`Z@YchQV{TMLJ;QzKl1*(2pP zF?x##7}#l9)B8kd7u#ZZ1!or*+ZKy?>oBInUYws;^=m?N;$9zDM+mqoE395@uLF<~ z5iUW*CZ)aQK$bq+ zZ9U^P7Fg|b7W;*fu;INKj_=VnY|11MOq6-No% z{k|?5=WGYtT-oe1R*@+yz0u^N2M%=aET@1t><&K!#x+s37xgpMIgPMMx z@%O5>ve4F;cQ)Mf7nRzqU5gSu`L~PkmOzdsHUK5v z#2DIPOyeqGj)c;G(xXBZt(bXn1W&O&zQ_Skd;*ldI*r(?UiPV~pza!+ z8A&qr03Hd^V^{kp_6?f7Zk@-K=q&1!7EN3^R=Z_KRppdpXw5SBVU`5gcldBd1f!79fsC}uIh|h_hl5fiDj5YOVT#-9m$2Ry|{et{(OV*{ER+H z1sLyyYMPfy1j`C2k*b3Skm`pw&7QpqxzC)1VxW4vw36XFBsPYXW65d_-K`vG}+B-z`Ada;BOOy*aud7#%&o-sz#f) zdOF*tcOO{_;(7b+sMhL10oIeh4B5wm7S=WNh*Z-8Q)-jk%P#YA>*GYt?EezEN(X&( zUwQ8!kDJ80KyDlQrrW4x<9cJd3+7Bh>w>3iuwV&-g}OvNdF?%eR3`J_uJLm6E+x7~H|838 zO(ZQsRLoM9h)|bd68W%FL9x-WGI%E@WszRx!kBbRmA@fCM9;I}lm$9LK{l_fXZ*ym zkSlzZ>0<0YXU9;iQ+wJ6*dp8>^RYS?Jz4r0NEcQ>25&Th#myd_SJjCoqqA>4AQ6M| zF1~-r%Xp=n9oj(X@^z%(PH^cnT~9G1;D<^4PRHD0y{52zp8dQDFZ!6Zz?jeiS|M-M zdt>cKoVgT%C&7MeVK|taV`%NG+LqF$9o)$cjJ501FoZ?9P+SKL zmcwECw6elNf=Ls2(qb6O)}+Q6Ro!g*1{@kJTu)nWrDS&L?nHr8gxOV{5Jgs~VLG$} zH9r{?E5`us0|{r>9$NGHcor`BoMo1?PS>!)?!u2b$R~uDaJOqoTqZozWSXorw!mA` zvlDOAUVxI6?BL-h@FnQn(_<+gT9fL^<snqs=2Rk@&O)nXP_=Gc_z9yMU-0pXqCG!^4_+ zJ;22!)9f*%WF?qepn|X%iWsp~M)p<81^V$tes7><-N^5dOZ}ptoE_-xef@${!hIuR ziU<(e%e$Zk8(e{*9RXA%);@v9pF_WGkJ#x4Sb+-w&vKzI9KSk%_^mGNXM?h-d&)`L zm5xQ7ykPhbd06io*HdUNovIePY4=K{fj@v|yts_kaB*8CY(#GncO@Jk5{$O)P#bx( z&d97vK_+XHYp?%&sU?F>S#$o!4u{%k%Y-EenWVmzwMJ!JK@i{F>gPo}yb?@S{X6J( za_8W>ZdfcT_mpDJZs1{dUm`jDbF3{uG(#1GhoU_)5(_D7i%fc2edEQEBV9Xa5iKeZ z!$tjDYiIgo`^z)=w#Yy1?x`we_rIB0&fN0wk3TgD7{>pN2AJ#VNXq>u4d_(+kD3HN z)(2rG1yv0NiRMf}7f(q{HgGMTYLo^91%W)`4D)7fWx&iu{$V>MlYnHcZvP=2QYiN{ ztv`*)B-q_prpd*pKC7{W_CFVuh_5#v|9} zVF=_lfj$|D6S=DAijd~(Y1lOWZ*x~?Jw9&8i4|xnf|Z8R!=M4>!U*+DvVBSqwBrP` zC+%Ef-?{uQnvYd5YQ~yHTHpjL?)fj`7JcN~@>qOpwPN2DaBQT)rqe~t&(_euKI5Ly zm*gb^{M+B}F`072xgA-_0cK!~@x z#StdPpG+FO69`pLO_S~XgRp0X*f5$Y zMmNLPGQMXVoio*t*(<4-tEh-y^(^B~&^di|a#R*EN*f1J`I=g;*!>g#5pmyGy*G@* zH*`h|1tQ7P6*hPNA}N4W@bwuAwhcnLwCUWyU12uLnX>L5m3??#M?BQZDVs4+SI*GLG}@~uYu zSx@9tNFZ7Nq-n-maF~m_%#=FeFTkeQ&ONK3mJ#Du!XV!{Rg=N((Y|@V6TR8y-6zpC z8rK+ofG#8XE2hxdbEqZZrjzmE>gJM76t*5%v@?7pf&c7Vz?q+x$RHlIgIXy8aY^ww z0uSlu=oJB%mQ7(*%tkDRi^kGE$AmweUUnziJs3pvPl(I_2@F3HC7##(Q;2Baat(`s zEK+Xo>oqza~&Kx7!@T?TzPn$!mg>5u3d@?FLIK- z;#ndxa%e8;#FZ}N+~Y>y30TL#yB90v0V9e$&?v)r8ig#Ji-v}zI5e%<(~&tDvR@7D z3t-Qp(w0vdv9J(AC9;&*irdur$YBv z5}-u}IO!b!&w37)HBDfBOC~S z9s-K3CWg$3KihNq9JS9{);)GxEthQgW_UxM(4Rj?pDrE`q1#(0>Q_ z7i^35TwIHAP?5}PKMU?Ts|u~(P{-`q3aD;39R@YCb((L2*qi?*IkUX_~X zRSL0;PkLj-M`6;$?ebk#b&y*JcAW!*NFox&nycO%y0)+0`nc%AQSl@?M9)7R;5ET;D_A`EHDwqG zoxQ)Qz9&o^iXa|e>k~ZAQdixz{5>e`d46kZqx}^my%hTL$AQ-rE8j;>XaHWvE=-4Y ziE4h8$pV!`zePIV5nEHxYfn{$VjCZ`8c+#)9Hq88w(RfnF(2*|yzHMx&{fI| zgYtTPhaWpRV&?JpXdD~$L7hdL!IgJjZ-APu(+!y(D!Tt9T?R#&U!x~ULU6y-OCbCtEgA$YPyRR`9nG_!r52;6Z}=SP^A#X z?vSTE)^z{Chbg~JU7qK!6z;o*f7$H%JZAwRl)bPZsB0UKDHXp4Y{D z0|=}24?h33m}+WG&ntum0(wCM0;2xkcs; zl}eY+X^qCMWuMQI>q*(V`u2p_7lxeaDVwcN@9WO(_s-G$@2&@WU#FL1FuFRzt3)}! z3M|dsun`H{Bj7ANB4zf8b3||Kq!R|pJtHLTXjS?fNyKXVecFz-sO6nTBp&VKp>w7v+zx<^!N&UP0oTi|EaJ8PL^}zX?cYB8j*wmt9#^_8vOO@+ zX9-t6SxL13_?mRZk0*hpH-e=bJ@5C>&=|(^Vd2als5D8#PGo3I`z^F*JVj1?4Xz2# z3p1|m+|w=ITfz-SWNowq($KB?(mOsDFP*;NKmia1V3MM(stB0__`L5{n$lKn){zAl z=J(M8KX*QbAEK5_SzFlwh*euvDWs@lAyLsVr@^WUuv4k2%DAC~F>^(cl2>>a^%FHZ zuq7qX4eIXY#TJ%9msgXn;4vEg^AbMj-l_cG+q0^IMEgZOtj%O`~?+>6ghn6k;6y-LRJXJx%1MXC%je57NYs z>3JGi1s8?-tK2JfEKE@Tn68?(YN{|jUU#{n4u|FCxS3$7j$Bo{5au>h$<==8uXf7B z07NCY!l?;qH#m$Flw(u@icFLXvKx6C z2je`MFgNi&BQC0)$k4H^^3d@t)gBCES14WBOI>934q_YFksOZXd@zfBNX$p1K8B8w z6-I2Bm=Skyb=XT=q|FX1Mhw$oM(FrT3@`Bcv0PUrIRN!?TOTK8cRe%kdgO(+D*Poo z@_HZI{2xpwiQ$Lup%3}K*bmbxXQ!CaJ2+?j1-u*9A1Psg0K)~JQulJ>%D(N?SN-oN zNfjyyoe!ho$FI(d&uVb$dM>ad420W)$n6~pjP664kXtUq?vz{RFZ{h2Y=6F52Ord1 zG>ES;Y@dW%mydud-1p?PF;Y;RI62{%Q^|15O5Y)iClwfnh^+c#bTAg0{7Fll9!W0m zvuBT@djS)IGzm*Q39L@8)TKr>&MMpP?E9(}G+G|`Aq*X!<03BRM|w6~8(K-Z;`s4F zB+1o8gXZ$nWh)|c`P28(3smtmL zx5+&J3XZuUz!tX--#uE*3gX6A6Pa|eJs zZE#SlfzR!vhEwG{!?g{}GwcIceparyiKfoGY@6sOfmnTDbx>VTl64leedX0s7zjr;j5Ra}|%h~}v>W=uGz z;J4r7TX_N*w7m8yYeCyb!b$m-i)se4uv0jGg>hgW$SsXJ3E^7(jpjBt_L zyfDfrwhmW>qp=jFMKJMqO-<3JnmRgk#o1kSuG&l>ef_~QJ5Q@()`QqM9DEkh*)-YkLYl@%VJJzQ1`hC*?mtzs&bBEbOO&bnvf3ptT%>My%nbBw-;0*t*Zf9`2Ow2Mr)Nwp zwGP-3L-JgHgZpZXer;5419Kq$=+%5;P0jJm0t@ICe4YmaY@gT%KJcH`BGsqcunlu3 zp_r^{5)i8^0kK0b>)qm{{=ACDg@^!MiQZ8oG`VT-Ux)S{BVR0sl+($hvtUwxlJ957 zcqI?i7Dc$F__xA+fj6x0s2SZdCzxcmLQ+;@apn&`nYh+$+?~rOqt*7jqv^oi(@16l zNpFIYV<(qQVr2AQeI2l=LL&xLW%U=6(yUrXdh87hn30brf8iJ|NtMT0ne<-x=DV#m zuv@)xh!lw&wTCVOnEk6yku-{0t*LEU{_GSPVBydpx46~UUb-KHeHhmh zbHbKxiCAU`@0qV$u(oWFrVRYW&DGYY4Qu5N9?=Iqd<^5AzJ1?88TxJ6_=oswpKQ0x zB?<$3kr&=T7Eu&4f9;ZzX0d8czi7`+j|R6dA@OdU`St#dJ!+wY@k>W(Ch|DxwgVMy zQVOwtw8eyT9}?j^$2<ZLulLa2e$ zaI~0d1X1nB32kzy9T^;FDMC~Lbp4zHzbW~K;0IgtrJQ;s+<$v4-D#3HMQ zCQhPaoDFNt1v%p;Q`9$wq5=(ug5(sQwv6}p8(L;4_x4X*}3&!_yUr`fGDLFY^$a7n4eei#u-PvToZ5WhTC8V|3d=gVx~w}**w%0Yadg3jK*L3N~K zw)*>Q^{+l-^=vSbh+j`(e0ANm%PMlBE`O)*sdEf1ya$dt$o*;f^hC$}*^P~G3q*eZ zF*56|gQc?Zb44EfcsY#F*+ItZQJeBw)~o}VD5o2qns~w7u;<>6N0lC8KEaseyE{jo*tE1oJ zSNjyf--+q6)XMl+(-a4{(>G=meu}sl7kw(%VS`GQOS<9)QMd2El1^O-$e|kBB^6lW z0NUAI4MiSordWHRzh8R39Nu&Uv!`KgN?)^Sz7kVSd^w=!n5op2ZG9!z5JndQc!_^DvFT{ILHa@ztPpyK8yHX05E2>yR4JICfs0BuXh zR>!t&+qP}nHs9E`?R0G0wrzIO$?f?z_e|APou9GSe%7<7_xhtWaRcMeHvFP}O6P1P z3})fVLwrA6kI^7&gD#7@2lA}*+MaRuvMlrX?ONHzc{xz?&sNd6uv@lgnh3*dAFT*5gviMh&p)Ox!OFwslx(Ui_308&dx!GC1+$CUVs{Im7t*0m#H?3<(7)Z~?O7+)X5jIiErG89A{9Cz~p05B4BZfn9N=^AG8dHB?$|CrrLwhdS_z zb2{-^blQ!RaxJVzY6?heC;+?D6)LL_=_^a6+G!~J!y)BgmISL1yT}&6DQIo=l+`$1 zC7LK)NA!EcRIQU=@bWUj&mNua`5-H-|MN*Gp#fg* z;i)uIIyEH>+ymkB2|Hf&XYmEMhu-9@Y6wCVCcd;028-|BZvcQ?#PusNxUTh3*aDS) z{nvQP&T!hfB^ZdGs;O0x0qM(V$gG+P2Xr7Wl%6@}0mKX2nkg2qMr|jAQ89VE;twUt z6*QTSlFrsMT9hD$g7Mh9HMgHz2kEwwnzWbk9VQQO`lk*NuXZ10Xg_gGD=^q#}wApZtaTe%a~y=5NPCx+)B z7%@ht@Xv%}=YqRzQCC9GP6=?N`lqcLoB@{4VD?x-!e&)CByWO-El3+Ku9i|CWM?h^>(R0k~|s`A-x%k=GHeOeQDf8JCAHLjb{H6 zAws{MjrLYcHI9nT+!CRSvnuaSCv{AR+r_N7SYkQlE>WClM$uCm6avQ6NZj*VGxHrw zv8)n9T^a(NJaR^%m0CtPMu>}@Tl1;c{w!l!C>&layOZvF#o25oaS!2Rmr#M734t!M zmE+-9-X+0Avkf#$m)`dSD-HiQ?}&=$`s*nW{m}{lxQoYRO)3h<2_TZ`d*ag9LzEQ^*y83K0j#5BKv)l*o7iQ0pU(TUR&nA3-PDMX6hW zco%93+IU6xR5Xc*zlThyD(D98lr1VMh^}bUSPza*YtmJoZMZJAeJ4Dvca!mG5f^A# zTHZuez=Dhc#gB${x%Yg|qf=lix{7(V7SSlbolE(y%HI@BZQo3O0dUh=GcgTltIQ~=EN$IlLs{RkcOE@L0Ob(s zL3+v6Yv!&NEEg(g1{RGH2=^Adb^3PN|C5io)X{-a*$G6o*ppqEvqQV{Jm*vHn6d#Z zvAON8&TBf9xYe-}AmCRj+|xsk{MnsyEi~7Vi>*!%FmyMGOt|p(H^T6Soor>&Z^gY} zV7l)#TgfbWsssd6gbWf(-9ES1e==}^Wc>T;Z`%jX195=fz*k8#TKF$GZk=yeA(7U6 zDT!0yPyBCD0(|vwdwdJq0vLO`T)KnNT3QHt$az%uR&+zjT(M|)1()-=m8RrTx^jjtW;H8dHc&57tq3++M8i;95p?kG007?m#0fy_ubW$tLJ2x98VsWT}NBfnrpU zCxtujD}UFzp%!>C9Yn$wpoo%($h)b$h1+L|x0L2QV$PC*RQdERv!=Xxsj{mxh{{k6 zoVZN^wd?X7F25J=J4^6^#j!tF6@G#4=lj)o@X0LT&mRP+s~$qYtCb(wd(J~Byth{T zB}~l9c4~vGNQb9#6Sq4RwX$96UL?=|<8fOITiP}BO*wbyi54306{ho6hPMk^o}0DE z-L`WdUvR`1W)e*$PaVc?KBEcT5g}44bD{%1s%i>>d%HZUJ3>keY_ptUA(FvKh^(E8 zuz#V?B8-*RuB6OkxM?aS+5I45#<|bQx`eMBws{AN2<}>$IerWA*s#h?Ls!LXCJtrn zIy=_+`^rcVo+RFJ;^83VZ>xAwYBC^{xmcw|NO5lG3V;maiTzBS*p0J_E^eVIlKJ&^$jndP8FHUje1e8ET9kxd|O>2fEV7-Fax7m+^(DQ&B+iu9@}L(9jQPd&}L zi(5=P2R?z=7Z1wqBD_iUtM`OfvjlE+00nJsrkMjDsiX`*P{#H>JZYOZ?Fn&rsr6nK zPQ2(9!|k0|`}ms$$c74l<~*$(f3E&(L-tkiXcj=$4K4uH7@(t;{cnAz!D|b$!GdMJ z6HIaJ)3!NibG{CPT3!YL)n;VMXL+A$#>3cz#K@=kID?&jM=5DmBBH1dCc{L8-hl5j zlOqNjI_qtF)M4*1=rF^fFyH=_k3yDafWp1Boqe3k9mkOwl*7YbVWGD)Ku0idbghmS zng<=L4hzz#XwV1Fh=iA-7h5pyMa`u^Tz00^$!_DNgrkfP!MFN4yU2cZ#ENCJOiTVQ zBn((O@(7`<;Ex&IJd7Wk5@rr1twkHlfSkw@Sgh62(g|#=$-0b=>z0t;+!7}&0^3<8 zNfkF7$May;aZDcD6nF%o`koBv$OL;7>njU>!%rJ*#vH34cWzt#BGoe4{%#k;KQ+0s zgJek3L0qwExd0uGDz`-P4T55JY;>Ib$dtnkchYL55cXhBaLRxBk5@`%Mri&K#WE~F zKpg+y$HV`9iT*#0#~z)3_oz7XkN*F;NB_=4Fv80?;Zh6$OGu>*X%nj@0giU@z@)XM zYbs0Q5xbO}em-ZeLv#VI*Q4pMw?JB(xtY0R4)|Sv6JwTaG0;h8C&>N^e~bIO4-)ql ztEZK2FzPL7m`XqKZqJ(-vmld0Q#B==K-WlX$uCBt>a63AK=TnZ_`m=9F2NH;U+N&B z9ZyP1x>(_+@PjfbM^Z=j0nPl6QIZC(9~Sga*Q>8k>B5=VTOucNxPop zrAR@0bCqLhiiQ>bDX4BL_&UosNNkwqVEtp^<&0e0rX{4@YM(mKs8Y?&dLT3VxY9=W z#;T(Nu^n-1%I}6wvXw74eHHdO@i>a|CM~@HNtNR)(TwR1tXA^RNNtrCY6ccT_-&Vg zni6KKa0E|zLX0B&J{XBirSASda-6K>Q{dpJ&hVp5LbtbQx<5 z54z!wJ{W$fO*Hj~m}b*%7D~n=+$7>%w&dC`f{s zRa0SjJM_d75cR-7izt@NrHt>t}>PuDBITEWS~UF5Q#xby&(>&K^^fsWAC{GJKM|wT-9*6($rLu zx6dApf+}2940JPyf_r3B-w9B8Fy2vtJhpBs=`7NH@xB-c5fOk@V2Id5*>osSYO|

hM((wNC_Q~CI8RO_dm1Hvmu_kzp{W-F6uI9O$ zw`xM2pI25v-Uzem5p_t2k}iNoBTTn~F&pI{JjQnI$4s%Ww`%(D$nbMz!cp*q^r9-7 z*t85}+C$8~Sr|;oI2Qc+#57FJ@8v%jn;ri`_akQkYOo3-5cIO?f5O#WkhzPcqHEhX zjoF5>G=}2`#Rv>xt}=yq51aFf=>)Xr3MwpK^~-JP*X65CVuR)JWk4=M8x*6=;=G4M z&P*D|^8j}w!ueXDi=i`22_GaDQ7mQN#MI0s9a}}00HDI7_TwQXIZsug!)A#+7zKqd zvJ+f{tWx+I_I?B5)ms2YKZcZnYVb!dL0N6Q%PH`lL2{8I2=7O}rS4kgU!c+l%JoBf z1%LOm;O&XuCF*v%Qry?P@=r(HMNJ^g!;4@nstgR|2aKqin&PUW-Zh53#3M3MXXdqL zAfptzE~yD8M29;xI#?oJ%g_x(nYBpJstF_TOrr+fZ5gULd;q@O z&3OkGT#Gxh&lu*4=RT>2SF}Vo+{ZwR1e$!{RdcTSWe_`;PO?8Dg8_YSBdb-E>dA%OBl--g zFh{^W{&HB%^OnWj@>h1efdRfxQ1e6GYJ4nS0m;&uC$IxPady*RBZ@T^I4GZF@pI=E zTKNM#CA7FqRS#N-3b4zoqC&Xpv-dOR^bg6o`bDJ^&TI2Y1~K=0>ZRhN9ZK)@%Gri$ zf<^8f_0y@b6K?(FqE&OQ>3{>XLBz?S3-^cX(pyzSITFFH=f>HCYRaYcC0*@=dtj{0 zRd1GGPD|4_TXgAygxJ-pVreE!fBkpVLwbv|U8WqE=hn%7U+C6#cId|i_V?dsuwR?2 zTz8Kq^ibhHzjdI=nDLocP&@j<5Yz!X=eYzZnh?GQ}P>w1W`70i9$ozGh-xdg>!;%0Uh~^ z56jLJY9&c1O-VJdA%g7syhU2%)mRAaR|dBP+;Xi#!`MihKR6&>ViN+I601fQ+V`tYyFuS&sy~x^?yWW83w>Hz0bIa7zt`Jqk}i5EOhEMi<%I zM8@UM3Q@H!ctd6QDF&jOPGz}%?!j-@8w|rNJk=^Bh$zaOr|tzf=z+`KCp_&A^;5QKzF;=~b`4>kSGwF+tC zsqq0<>msf~xC4&d$EBbxA01gg&)#-JiF)r6-LOP8Uh)3(=bLa%V%_M`lXw%0K1pMr zw-BE9FX3w(0$MB>O&KYrF7zn73tOz_VUkiS`G?40zzB)5*+Dd5)#S?7$lyOq_OOSx zexwq*G3Hy0T90DDaSVlHu`26WEUk^?fcG*2BTM-dHtsUiZ@G+6Wj2-J-0pUarH^nE z_K>K#I5K>9ZW)>|+&=&U*^uz$w|rJ%&9MY`7t>e{eJaRMk*b#!q4>E7o3|wkAYJ*K zH%3}4$N42fK&4csFxdcZLIV)E(4j-IiGJ;xOl*Iic7jR?t6r%k(w$|Mc`)QGRR9aV z(_#llA{_U%&ZnMzzp!0)w|k-vPq<_Q+ZdJ`-8OC||@*(UfTTfL+wAzHLz~%{94*%+m=tATV8|ZWo#l?hvFwrXu;B3C+WV$bXkk z1X+=iW07yE3AOTUyyPWkF-`t=x&;Mn&GjmxVLloNnrokJMF zkU07}tPQ6(mpvD(-PEjM3N}muq-h(krc|THNQ>6KwB!aCHZ>%?!MRO*>W#r7deb1u zqNKzpVkj$rm?V#)-Y1l0C^rpUN!E+}qHg#?jr@pjw326{`J{?<6uZQJTt4tb{O7w5KLWBd7dl>m8zt-wN zJ)Agec=ZQy5Ui?2t+EgKv_*^Z zw;~_p^L(;E74|2rEY0lrvun5C4zOSmSQXM2a$_2gu^*6>k_(;xOFvH9-n(5RuyLbf zSFv^NCCxG2&~i>xY&nIs$%-1LGnnm{df7%+(4C+HRw*Ef@)Fxm`mUwdnZN2Of*4C$ zj6(-Xe)`8PSj1+lBO$bE715vR8FoRM#Ib2o^L(wU5iF_BL@NJO682?f+a=}TJ#`4& z%!q2T(l!J>=~VPUSzTBA2~2wJs{+zrwN)kF^Oq#ckS%=ASE<_#Ls>63aR9Gl^b1tR z>zQod?Hq93Fl-!}qk{fdfySTc${U3Tyr_bqJ0x-3!V&{TeVH?{?k%=Eb&pN#(XHU3 z-hME%3RW^46h%Dj@=o^Ko{QgA%C+vm2~<~?a1)-l^N)Tuu8z+9dl5t78_0JtfH4FE zwVgw59b~R#!~P>vtR|fMB{_5nC46YhlMkqRrf4`%++KoTWQFa(4k_oz=%qg~HBo4; zdX{hRngPmu%~TJCk-R>5t+*TT1}&KPmCjT|(v2GSF7NM!X@a{~?vjq=3tF-Kz1dKY z>XkBJ&rr?1oPY3%zqrR>j{76tX?G`y1pb+D^6DOVIIQNTP(FQ#SIz%-Kv?qBE zk8r-x#dPh$Z3*m?VVJDbohXunxVm|L50;NyoESzX<}OcpdY!5|>(an{Z}mJh4X!x9 zEtvOv%Fh765vs3Sr#LLe@t$e=%BWC(f+zs%hC>su@@>XL&5*MSx9=*666YSeO2PT6 zB;Z@WJWjmVMktoJ=yG-i z8M%Y5a+Mf=-VWE0d3r3C6mF5us(c8iT0DqIt9fbI9O?ZC;;03!75)l1SN}w4<=hi9 z(g9V>4^n~nbAP$BF(9uS^0)WbIc~M;;_6}7Z#j6^%eq|xU;C^38yvMf0`gQy)Xn*` zXaaH2<}BZuf|$d~AI#8&m<$tOFh1#oIae?W;_n%z9t(uJ>`fJdp-RfDr_xJg<4 z&FJ6v`_jhDc;yom47(4dtBY#fEW9<<;TYh(8fj%efB51#WJdbnMy7X=T|cr;-i|K+ z{hhO;IT?K+)%p@qpWh5>Z}E8HDachIYKuWjYp5$g){F%4W?0*Im~!Tqz!!E{Vw|jF zP2F-Mc<}@BNz!Z4Y`7+2Bl_~|b??=Le#q^8XHq$dDEtonSUSi{u`MKLZj2x2Pex`w zk3m}6=b)_(>XCmPzTs4RJ^W{zm!-2V{oH-DR z5@{jr6fKD0T?vk!zWgc`=*i+cO#JeMxCbP9;*M7^plU9~pp;?R2e?v48b6PdCxdw& zQZIz$apcfn!;Y$gMDi7c4s=8yKxP|Wjv0pIIA~8|;d<9%P(t+);x9iKv;hGe%bW>v z!P{N{fz|hTJ%`BRwyL=6C#gz$ur%CG)KBG4M9BAJ;9&cJxzFJ8DAEnLNY+=;U!b+D z0o-(Ahj)!ES}vrKw=d0>&$MNia&_Bh?@FI&BBDH%)^gaAMD|Ba_z^VKM>ZU7KsAtR=wC!mbrep-PLe+4BW#&S zn+YUt<>hrk|J?gs)tPOLubjhqgrSAnb$QiU-5J1S?$v0NnwT=`D{{d?uaa_yTLQJg;VVgP;pPcL++O!gj3^*lc^2ja)-l6#*7>%s%)P;Jx;pi3 z;L#%PP01M>w492;$ZMJ%dai1F{Box6^2p@7O0sBszaNftfE|vYPJ*j|={Bj!-0Cq^ z44Tg`^I(Zm;Yw~ahK@$lV!`I9Z|sMJ@u>r^ZrbcTDu65oCGg<1ea&trO0`X1P%liV z07e~~YN-56W+*W|wV#S9xWPq4r)y~Rk{ZJYTTN;hD&k5zdH!{KW%vQahT16y7@>R$ zN^xh91pztk>DAc5sKfapY#u9`Ln_uK2Haw2J%HpTxl(yre<$r0=83r@i~CK0?(<~<*k#3BugUMOo^JuBQf{K1nH7kytG4YFFd ze$6RL|9jIDq9C6L6-hV6p{@UzyY!tfWP-b+61Is@7u<;$M5~+-v}xI#rB8u@LINVx zcdJ!o%k?@S(0oSf{DWCPenAuzfXvia?WJAlfy2RjB!@X(FW`d8i5voya4OLrOw3a= zp>iKRWaBDeO4-IUgIOHh3^?+^uZgO{vL$1 zU2mW~Xj&Qy(f^ww)O9g%-37wICFZfR!7B(>-!8`h5+l6VuLG{ss#kQ0wVJpioYc^k zt6f99lIB+YH;L6TAVD^#ji;4gN*i<9Q3%z-3E7G7qSdMgzw9K-1^lXJ!n0u|%Nh4i zK$=4xZ0H`2dhFH+6)4bA<p#8QR<9Mj&j=$A`|&$q-C| zr&y2oK9u&QEv#+^U~Y=L7DPW)JPN@N_OAG>l+rkMPN!k`1jIR^6-IbZ8Eus6L$4)% z3_FobYXDEO-)Yg5=urw2}w!1f)U=3S5wA&u58KT zQ?UCf%^zoCBqU**`5xnlodG{&((}!pp`qhO7}s8ihvyr?Qc-G6?VHoFo+G>G9R#*& z)(U6uo##IFt88@pSu?OcWItcR=H=~9sGE;0KNnZ<4`u1?0wocq@&!>BhM+%y`|TcZSiQ?rvTepr{8%>fsuI;*q|3At?7J=`$lxIOq0M>!x%H> zs%ST?VL|1O5Y&?lEUH5oxw;xML{`BDwvDalh^UnqNz1=&qJ+m>OfFPXlkhr_oFuqy z@;jHpN;N{yZf^TFz%lam!V$OdOjH_jM_Yu7H_JYqNKS|q-cI#{AWpcVXmcVPx@C&3 z26hMu+)ECY;nYqG5l0(q zQd-JX$?^TzNl^&8Hz?m*(xdjSI@+PNcj#z@o$v0^!-g2wQS`|}m&tY||{gS!{)Put7DB4zwBlVk*hINdn5+Zr|7p|>=n0NYXJuh-*09Q!?8x zA`p^0WQhNC26vKmmQe3i9wvFBR{I-IQ;Y5(u+6YCNyQ!PwE^-|4Gc zW+G>z!(a$xp_t;%>qa}PpLD4V|LP44$sJ$tmsAr;luFp5$s|H_$D26hr_-E1qtM)y z%#!SD(@)s!>d^L?3+57$qCdUn|dx z8%>H(B(k~f-5S!If62Yx9zxMEX;EN)6E*}fiD-@=dp2vi*AJQiR!;ebj9d^}?PSNG zvP()f7hlwyjsgYWR-NhgY=^1jMN)gb zsXVOzcqf#pq*7fe$Oh^s4yhNB&Q%;JlHr)L=Z&n#Q+8%)Ei?jN$PTwwsld&xkkxY8 zt5q3D=af!Qg%b)?POH*1x; z(5}>)0iO2}At=6Eel*be&t2wav+)OJ!sKAk`0|}=D3xd0i!7AkB7VQ0^~``NEbyX_ z4G47^WrKA!%WdIH)zT)3VWDs}fvkH-jQV-N0kYFl{Z6^d85)+mn=W_0tu2(l<-k) zh2C=WZ3+OvD$2?ddHa>y0GB?dEl2#vqlO&4m!mPbKg*q6nu6$)115n4;nHHxsaOt_ zunt5LMmbxDwPTJ|?*KI>(GRTJsR5*_!(4Ul-+GrNo#IY|4Hod@?ZW3TW>Sv;jfgt0knwGB;x;{8njdk}aN@#nsdEoyYnX`D&6SjxdKv5#AT)Y*-wh3o{c^q#`VEULPTzfKD{}!=JyzDY7eSOEgOR=YcXsC=vZZw+|B%x zIg4UK2?axty~?ZFQ|-6lud{qt;!zp9bE-RQW8?@#dt971XDC`P_fa;Jtkt$LHepm@ z5U<4T`x@x?N99iwdn69dESu+}bOtf+*_C-sU0``-{~={2?iuQb<%BhtOw2ps6LX8L zP8h>_+veh4*IETwNnTIztyJAn=f}atNHYVE9DSVo(_SY5d47i`-5KMO2@IIW>8H-1?#EHUN;XI=Fk_ReK06+`_t%4=q%4pNQMDD*So%YSkq8D9_G1F zNIOrW!BG?1nt(mj>w!k#VlfNF#q(iSNrCIGI+=UD3Wq-D3)7pYN*i1ufIRl3e_l{J zt^hV2sn=f033-h>b~$F@uzklpOwm1s{GuL%BIPLQ1#*SW6OIR}jdoH)I zx{U34QsSG^YBr!}yg4qnUT+GUv@`4_^F_L!f_*kW^Zsp8zjjnerWS3J!k z`}DE+tbh7@yWu8-e+d`0Rde{KK$QkId4ty?2ibbvONW_AK4%-~#gwy!TM z``z$-mHM>cXp#0LMTiBGL>2Idd`7slP^ysZBs?Hh%s{3pjF60uTwAD!Gd%u>KEfhI zrb%C7054JnmP#Lfky8%2%5I6d`c9nt_&WCWbAteG%Gdr6jo$^&S0Bg-R{_JYoTR>}i-d3Os616U`#ZH_(yU*61(?o)23i z?py|pK1oWCbg<)tfizdW6}f&s`vm7df2`=J9kG>{LUZsf($st3<6~N&ZG}5*!$D=J zC>F{dyW9~u!&(L%%`|9By3`0w@f9-$6xv7AnNSx#ze9W7QJt($wm{5bThVHnbS1uhct2kb$S} zd zW{fhjxphZQLkVwqhS@oD+yNr}`n3a`8*ssZLNbYSyZPfs=Po%Gr%dPHQ#*^?ophyT z(!kn20zFy%V(kg2DfcAeLA%kF+jt^m2JMm!Rnck=J8|1^4 zIG4f+zdyMFoHzan1PZUNG>a98HgCTQ=Jr8GB5(;&P=n!Mi76f5itU4Duvc5qM8l8j z2$79RNwVoYpD3v;A;x&iVo>Lq%4zgC)o{8~BF=iaq3Y*B__Rk2)!sEcj}Gb0RwY{D zLm{lv4Fqe(`bm1I0%>5O6F^CkcwjjGLIi>GIxI#uY|+KiJkbr%uTg0#Ts#xfY%C(F5LGF3XRF^WyzN2L|=%Z@6su)jCI@?POYSGNLT(FUG{i>4lvGzqr)%q9l z{OQ-D5RimiY<%eeV>h<2DMkh^HAPArm*bKAUEvAQ{&W>q659F7#O!)#mH*L`VpDZy zk#uR_ZS|3zjIhY7u4Vv--Q7O*qPO!Z+rl~UkmJR5kD2ILol3BidRug&6=vr8?rA5| zpOzu8K5AC?{?Hu*t9K$+OJ`(&5ZyBv=YRda}a_bH5 zI|1_St&p1BzYY!Q2m;DI>50PPvOV9`<8pnq)e$o113W8a>8*F0Q5S&xVmS~x?Tm(h z@9c_u9)v3HFc6*&8|gVZ`J^XeE`o904MF#3Z2CMj^<;YExazkN7nNshyRTtX$P?AhNdSGu3Ca+I?;dWo|h^Hdwz)OgXgVo8V+nPfln*+$-2>d%*PLpQ3XxnuB} z@`d98>uaR#t6mZ$=CKbj<*Q$r{fk5`YksQ?`@D(E2&kDRpLbdJubJNvC<_t;S#vy2 zt`n7ets=VK$XIxU1vVlMo#AVK?D3Y7Ns#<7nFs-Y9Aj_%FE5gNz-GAG8?xSjq}V?%4I(07_u9z#a|bjM9IiCbDr?i z1#=CdCzf*E>p&}ISSBjD!=+CGrj_wm>$eiFZ8n0b+DkYR#FL3*nKwsApE!c-Tq)ZU zp0p&SmNybSace$OflrO!xlc_%u(9v!h4HD%oHe0DQIkv6sgNS)4t40{n{}UuUs(f8 z*<_cSEHTh*{&?XokzEl8drJwb)qdk;Tq{+T)vI{`mvo?oHA(Mj{aII41uV;?iKhdv zE-sEz9k(gcX~0lgDoSoRv-L9_n%y82-4-SGD2CK3eQuR(SmMI1mmE-uZkZQ1aQ>j^h3Z3mWO!FvMX_R zYnMXzL&MZ)XjX0#{;WPXsMK~ZwXX8Tz=nk+Zr_g(Z;&Lp z77PxBMm_sY3ah4HeoewDWF z2XFw)mjhHXth37YLrDI0%l_aEU^6TRsUG_M+Gf(OUlL;Wm+N^Z#;Z<^A00`UT%8{` z8uHws%fI~+{J@lqJgACy?;nRp06zKDNr#^d6+bn^&fVaL@yhG5U#Quc2E=3?gYfV< zi=L2y1BjUC<_8eva#NhgSEq3?5qg;>r_vnVG!?lbq$sQYlYBu*v>yEW0V|OvLgGa8 z&B7>g+jS=?-y>G2?kmF*qB(h0ka=e4J8+`iF9}k4E}DT~@WtLHOSs+;=(xA>B{^+!LMBkU zO(_Brl^hud#E%WRHWU5E)uE{Hc?FP)B7z8k)8ZxXKGuhf2u@hc!zeOssV?ebK@kO9 z`GAAp^{5VDJD64jy;O(zoq>z>O)(sE25g9W&Hm#H?z<{+LDjJ6q5D^b$s*cvQz_f} zb$Z!shgN8`nk$!?D@8g;#R)AFih!8XyZP3s|K{=L(S(7a;)iNfNbRdU!&4 zyoylDqd$P>D4!;+l7?&gRCQU)8j&+5w^&f$t(zSzvOa+*8Bg*?INsRjWguya|rm zBSbyyz?F!~6NV}yATvGtG&zl#Zhi*?SJ@TOg7?GwX4Jk?i;7iXf-3;wZXrP?ful=m zr`OTZ1xKt3DNx{CyFQZ|&T5x6vu^1ecXzXPxGaxP~9e}Oa;?P&Wk*7dAxCnF0D zWke4CIr}PIk4p3roz_K$bDztvY~8T7_jb`GKT@R^ZJLEJXziWL38!c4V|FVwJ`R~W3ubv$5DIf2%?qaT!GN3SQH_cL>&XjY+x^Q?=yDx-Tz~vhJNR0B9cw8?2k9{U~n8~ z;=gvVLrtX1UStKbe&KOgO^~^itG3*Pf&-<$y}Cd1&!Sm&yra){b1-B1+AO0@3*ToA zpUG~(t`u#=ZfixEVMhd9oP$nJR+v1zN27A7>15Uec3n?B1GZnysC|Jh+#Y^jy$1Q% zs;R2xgCtt@8O@o0coc4`^&(DfCYheG<9$6o1bLQRB9K-G=}cr^;)^44P?q=Up07gt zayVe|INlf;d@Vq2nJquNw*MRky`T09x%FW>wUZZCZ2JngZYgSiUE#y06wFMQZKRN7 zO|J?!dMO-+Eb@wSkj<-dFA2Eas4>PiV@(M3&iV{yy(#-U+gSD|)hg3C%C{c^9;Jya zC?8zaJy76mjspqL);NyX%nbJyKc zmC}jPk&e7LDAoo+v^tP~pr1uexv1~RQ%i7qL#o?~ftu3c?3p5LrM6G+R~=EJSP`(o ze4OjC?M>>7UXf>)nJH1U1xjc`+yw2_nlmFBCxClTCmET!9Tj2~1%24=!za0nN0&5f z3jshOP31F-)L6H=u^Q6$jYx#I^WcainyO^h{PGRgqu8~>*w(qJC$b5@vkIWPeow_k zJ~S*3RAS_}N9+yO`a5d+ewli{b|l>uBk43Pum{qvG83_8))$T}1Xr&bMP1To-b4qB zNGi^s=&*d2ch$ z79KvrSf!9pXnucomP+>1RH!LW9Yw2w6%JvVWr~2ZAl_(C4)wMiCNJ|yzJW)_Y-8xa z5|Q#NQ;0mE1&yE;R%sK$7l?dT$2hBl*JFw98wD}QO82J)^u|6nvEYG+RKNdNb~V1S z{@+Yw1Lzc3AOsE~O!CoRI;LMRx0c}nBgl=sQ7?0|SwkJ&lX=8#pH#Th8^w(xSqGQI zcVXqVmw|AKo*t?p-69o_I0M5R(1gij6hDyvn{=3td#E=5M>_a0{#Rkee+qM3we4J% zB$2=6(SC$R{0%E%I@UuC*E_3JY`H8G$w(#yUtcHL8pmd@Cz3Q64#rpizFpmBCW<%Z z0v)^y(^~!Wjl5jseN13me1xZ_QPy~TMZb7lS>&f4hVyvkoebGd-5>;0yS>+K&zWb`gj?n>EEYDAMdfd4HRBM!5o{M%&m-IJAQ&J>dQvT}h{8wvH=D z|3olUCFLJ!2naOq6Ukuk0tcOZiVbSj?SfPr?az>Z`yPv{%=P9#6682ugA<@l@i)X& z(amjymkSd9!clUjsb%NIh@O`AGOm>j$#f$D0LTts({_TbIT>Y|Sk2ID+C=|u;9S)w z{~+&849A|m(jpw^1CFTdottL>!n#57wN-{(i<>|uuwj7)&_lT;4L-)tK5ou~kOcy; ztgy%_&-=s1YMFcyh#I(@C^CB<;f*-&PlXtwvyzRxplQ4MNn#Vyo=$Xv0QMlm%XjM! zLl3SHzM(<%RNabl8AYiyveod?eRZ#F`|}u!R#U!~_xXbCkUdq&*cM6H0sUX}u7iwF zXLSO(BTtohnvK#VQ9B zAH|h|?e;Vk1m`J8hwfIaTeVl|6_pt)DGj3BQmbP?w%XE^pA96=L_BlP3aAnO4OPVI zZMg)1Zfy*4La8lY7bhFx(wpL%6SCejQ0W<~8%Hxw^=WQ+&&YTlYa|>bzBjTlGuJd$ zNPD0sGE#>MYNQZja9NOzaSW^w_$tMnU{7{b9N&lB1Z8vWRwTQ8hRDx?bAYC={||TX z6rF1ntZT-}if!9ov2EM7ZLZk1ZQHhO+ji19y}SFIy<4Ym`{p0xyZx))npLxE)}v4_ zoejBa3)>kV2?2=W%G{!?a8Cdzgl^Sz(#=;B-pfP{xYqEmx4xW)4QVEZ8W~q9tPV1R zdf<}D!!E8UZJWi2re2?Lh4SK~DBbdj7O4xYG{LKQpY`i)wp`vu5*hPKbT zraZnLhMq}MBq7}CcrH;X&EYHVK^&n#B93+_pW^RJ^~+UQGAPc#49nF}?1N1t;gUKI zdn^!LNjeZFUx}CzXCT8UrnB~N4~i;gxn^t?OdEW*6)gRvGGrokY>ELQ<(5u z3S*Q|-MsSpapmIj%fciWiuY!q0+T1q94*76wORq;bR#ww_yQq#`*0*mr+>8g7k-m* zf3b>yYg3p`nHeu)6nTM|VPxh64Yf0EoZ+DFj&Ud4P z5NOqP*<(a!jymNv-!LCWmDIUhfe$2_o(CUN4dgLKqnT9 zh{Yb=!T`ZwH6E#AR{6)7gQV}5nl8d?^qWhk9=jdOTHs~E?vF|5Ugwl~V_c((fnY-O zXZ7=1GD&k>gW{UzIo!Wp9J@M2HgJGL;$NA4TM zNT+0iWa;kq%pqh<$}-@mxp>43NDp=SdW)Zc967y`VN~HcIpzeI(o_BY}*`M0U|_p9qzI}rNl zEB__g$?Tp?=Bh|(z7XHZd(?>>k`zP)htQ&cHipx}tU|!nm{WUF$aN?|{n`=VL892a zueX3TuoI+m7dGk~Pi`}1RTpIXNBO2ok&V!y zkO&Jqry_=Q`k|#fi$;OR63$c{sUHyGwX)JQ^X`UdAn`>gG1im_(LXkOR=rgq4MG2M zf8&MB0Z}gu|9ZCPNX~j@@AV7+>xFd%?9;`=``u4fQ}V!kQNsnkU$oK04#~p#iw86G z7pb??8AhW8eOJ$pCDQuN9sf`5LeZmH>KlwUWFTn)H37E9Ke|l_GV9>ZKpCSGFB&_x zqzsq2=j&@8r~O%H9yYMPY)li@2>q~WV0icrp{Ya*-*Pf+?sPlME_~W=wE{q8M*z^Usc6|I?(UT)kAxY`=Q#EX z%O~>eL7JV*oh|+!^*f;Fq7aLr>_a}{j5C**;I!&E`{&1jQhk0R7;$wq=XL*?JR7oo zH0SZ!gcQg?*Gm~}vHkG6$Q_uFsvF|v=S@HZRUem_o+$y@Igx`sI!vm+J`F8XDhIwz zh*wYiaQhc&N7YDNK8J0B;IY6{mNQuq>e72k!6d!GR^s~u?d8nW#hI~15!9x;Y5OXE zc;AFS`)wSafHB$zFl64)dDRW^pQMNxR3<}S_E`c|YEU|U&8Q>eKPiN0EiHPU$+!D> zpgx|^UARqop!@AuqOjbj0n`j~{pkNF z?K~!zF}xtsL+jSaE7LHW$K(Mc@2$sbWl* zotZDDH~(4R-vc@ZP8wQ?rFkID6R02UF-zN7fcuol`d4o#Sr5Yl8<^HJ(NxoFk`tny z^etCh7wpjD+8+zj$k6dY#L4`C2U90HX;ogQu?HBWND* z@dQQ;X3_VwZSKhBzDTVZ4m-$r%N!TJ?a9XH`~@W(9NxeI8<;YfT4o}lf<`}|pC1Jj z*Xx=nsQ?7x#R%nXO@?VC+eCH?M1&Fnh`Y5?CFJstCm_|hrOoV3;>NGW4_=WvYPPuB zk+$HCc;1u00=fp&FT5wr24=L`=`%ibBu1)`GpBV9rGBD6YZsTc9KjwImGq*|^c~Zn06HJgSJX78@gju{8=lT-PVH711 zg~O~=zm#iS$@0(IqTH#7F_k;*$OZ98Zv`M|FDb=`Ze75NG+wqw+37hh6{4y`MBD-U zUH8ExE=(-9j&4ORHCaq;8Q@|_YUXCOUnJN)vkcIanDOq(5Fr|-Csp(Ls|}ncCWFm)<<{+kMFCz*Un6Bq$~EpWv$_%fjvx8?V*NQCCODQx8e(%(V_XH zA~m^9#(RD=V+Tf$g!!l6UycON+=X}g+fGN%*J@AsBG=`iVMKvr^?0qip&mRpnTS)nPGLuFS964|!9)5sJ8(+Pa(?#+sRO6w-G z-W5IO7H;G{#PGx|5F{8L!-6W!2uvEJa1$I#x%;sf7bdL?HOgXGY@7WuM84N zYNh;71L6?Q(N_80wJ2m}h{s$ry%aLLctTst)b=hz)4!D#i}&r>bW41{$n81$QnY-Y zt0@$fw+@xgmtGu0N63Z9*IN>lsxFZ8^4f~Yl^Tw5W;(|-4J)(39G9lI^P{eHD}?GG z&rdU_=yB$n$0swh)1_OJLi}Ek$8cX)FbcOWx`o;j$qLzrf#78&1H9}V*-8J!<$NLk zf!p!6HtSb zc9mv8E=9S37MWc2lmYIcVkWfH1vVR3+T0gdac^mDI3BqdX*>tP+T{H1ops;4d~SH; zKjhdP%Z60Ps}bQ?phM*fvov*?wV4qIyQPMaX*XW!0%^TeN!|If2Yv8)RL#xH5`|Ag z7;*ImP@KuaFLwnoE)dQZhf`Znyt6c(J#-%u+_e8*)ZR4{)||t?AfHkrEbkvzQl%)y z&LUDM0Z#8`cO9&5alrTB!6p9AUo2qPPJO29SGYvj7utvt8f`Gqf3uhqtI|xdCEfya zD7F`u7h*V7msA%n*){XwAQSLntistRY+V@ZY0QAWVPk$$&BQH$heIM^vaEC}m7;?K zAKPV<+_E*u;8N2f@6D>{!KipPS-!|=ul!aPAb%*lOoA*h&5!;o6?!sSfr<$c0H8$W z|5NGyUpWUyT;?$wBFQ^2ue|VMA!7iRfrW0EYI>`P#!PWyL4^h{mEb@z2J;a%W3h{O zV13^&RpqKfl(-~ibl?$ZZ~^Ir7dTJXFX20u4_n{WY!vbB#}5Z*JC4 zKEiY^4ja;EWe*;@tnp7Cx-8FN+xD3c6v(%+h3HOElxA;8wfo6(bi(?WlcM@waX|rksXby; z#qiI*+TY%nnAIy&?o3dPD}?R3#C`rOyb->Qy6G$Euoa1`I-WKZN$+VqdJDjZ|^FSWPx*!}M#3T~LBml~1$|w)T<-gKQbvRr!*a^8+2gu6sRox3PXKD#q6VCvgE%T4sO-EK88g9q7pg+zo_U#CA=xi=F z_`!LCUQpGl!*JP6(`urSc+Kgd%K{nzjUE!jQdGq6pLu#=2Dwhv!xW`hAPKN3 z2Oe4;{0q|tq*ZOxCmB{<8YC18l!2oMF+(>}uy;4MTy;`R2o?kt(YBBca*_ss=lRI1 z-QkrB&^P?_Grvs~h+1dawlzkrAr`v_`PTS&Ov-)+O|7idLNMY$`lt=taXud$HC2gP z3xe-Mp9+Hh3zkC)sHJS9S1$s0vLxivF31ycOq*l*^}H2N9S`XO+@ci_luy*h=>V?P z!!wR-Fn2bje5(KfB;Em#^j}K{8UZANpJD-w;5N9FW!g8<=wR8~_7vF6+FOn1rlf0PG%B#N z70}8m@?_M!=~+yz@_ao}b+YPF#K>*r)7qD@X$EiPE$*mEgcn@pGEpQUX)b+AtG4g+os|F?VX3PK!(Pi^G7z>SY-~sWl5-eHtfiR$$%IU+vYe zb;1uf-6I1VR`A;kMzsQ9`;c&8#-BO-X6oJ78xwcXxR7b7uxXkLL7&a5r4Ge7@+xjZ zHPtgFfQ_nF#m*+{3xrWx_!DV_dcy?;iY%#WqqOp5X4?qWhcUp}$QwmK3IF%d9NrXR zjxjjaLQ+|0>nS6iOdJ_556cM6Y6qZWjYGD&cyT7l>ZTA{Wt?Dm@TJ38m^y4B^~P^c+OU9k3h<<_TB6kBL{3?4**4fN-UoVK z9g(g&O&Dy@ny_=q=BH_OW!=W}BP2P0;OmM(a3oN}RO1F*Uxnlk$`;}wWCnrZ9d5;m zM^IpoUBZ$1=mX`)zcMBgO{V&0`hWGN#$L-o7^vnU2h6B{C?wx2E``ge_L>9OO&8E;LeL^BgU`t&-|pzdrB{o%t+9eqNoZE&n z$DU&p?Akk-;qi2WP)v{y*YSm*iyoPyGpNO6iYDA$4}`e7y3+BBqsk<>Ef)zD-9;j< z)J&CFK@=ZqCx5MM1pnf1+x}pQr57j+l0~dOOrQz-NcAR$Z_nQ{Gk79k6f8#mNmF<+ zw#de}*b$f(aT!xi^Cz>Ke7J!W^fJ=bO1ek-%xNZ~>%kGv^gUH5SoBOeEQldAZ1b|| z9BJk+x|H4hnPB|kfNp15Q8sUl(z>uG{=CBc`MM&3#|w`xQ8J4B&L9(q&)c7B2%BRR zA7Gwbj^NtZrLsiz^pFrX2~}IFIc?p%LLf!~$0bP^gy&zlv$7S*m%~S{L4#6}H7JFr z+1%dQ9@jcWji0b7JM+4gu8{)qpe8GA?zqANDkrx^;8%D7s0|Vm9>~TIZ-)iN6$8!1 z6w~K}hQ*jl!#v9s5*iBz36HyiK0hK(pu-AFi2mL5RHp47BrZpz9&z_WK$NUXpi)K; zC&n|F+V?>$R-O~7`lF1I79>>$t{{L(@lbTd>c(@1&8L`AG}cswl=ZYRA8Lukm&HoD zRe(HaV+i*6TFGPa-VCpSQi87?d{eR>o&pVHml^LY!;!1dG<%X2vw%On0zCqngr;D< zKxa&KkBuTQz6MKk&bX#n#u);<_j}1pFO(UPx~{Lr34f1meYAw@P4Qeh9X*7wm3Jjy z>bI~ixth=*CL!$5ZVP&*Oid6g|n75OikgeQdF+r5S(MR+-4wf|-Owz37TRg` zJI?p|U#@XkMwYMpSn9kslpr_~(!F1*6__YkRjsA*6_!pG=Ig4@Q-s0ffd#$3A(^c;eQjy+?Acz(rhECgLa=_2B;!*|xxk9-P)1t$ zu89HTq_)X><81~R(iN^{YSJj+i7Ii3aIy1pB3VWbvJz-o)|VJ2ePmY#Fw@#QG+kD+ za*n%U2*0!}s6V%($*o{y$MH1otD_`Ckc_D!V&#gkggFDR?{Lzo4`vwqr)hVRSC~hB zi0jIgdq;NHnJo?}mG|(HtukoPw#I*3n^a%7zYnZ}<7w>k=_sXRYs2KlziG4P&Mr0x z(}V579^_Q_KAa;$K=T09CoD#Vq7UK^D>)CKDiW%K%9JvuIkh<&-8{T0c=0qlE&n!T zbkxr}D19|=ZC(3x@q^Q>tJpT-!4A=Id>-)m_-I~T9~KiY0OlvKjFyr}5BY5Z_m>c- zGSPoFhAXg01h1Lq80Kgh`{!VHWCdA{TqZiI_;U9>m?78{i>Ftsw&iZuf>5V*D5NWRq059rQNAX*KCK0ero$pnk}x3BQs zfijNW*)QLbXlADrUiP10@vq}&DsT%MwPe)H14Kk;tBZ+xYcA$EpTYXf#cmP#m#ezX zL!Y{nA(COKSPj)pe9*%#^SF8y(w*!LMGr#KuOgv%HYG0WI-&m&WFv)HI3nnsO-2DIHCd>$M{)X>_ZH3HW#v-Z3T zdOBT!Ww%+SOE>#nJPW0IoD!Rwl@DZ1!xZCz=6FOGwe>&wHGUTfxFDx+QqT+@-!BQ; zZJT8IX5tb)edi2i`#m9$iQ|vJ)=KAvwv%~cR87>ZOm?fvJ{w9Zkjm|-qi_g!&(Xo! znH7&UN)?iHyM*c0y-<{Eie*aaV*TWxhHa7By8i3DRUc>ZCQ$1sB-q(nl(y_r*it8w95Us{At}VJkLZQw=A4T|7@}yLX>@|=Io?mnq2PQavjK#$CZNO6oRu--_CR9ye#b6AW zny*M?QTpLA1A7!H-}4+`*45D+$!y@18TLA)+4&5Ra8Xop-5v`pc;I}kC5n?1g7Sqy zw^zH+NOpF!Skl=pV}ZfouUvkpg1b-p6lGVwh(n1^O8oRcg@v@+1G1VF^2Mb<%Lt-t zn7NZkkQ>Umn_X|9@URcfBN#(=hoUxQt%fzQwZHsObMVrq_R0)rz7b>;=lZU%Kt-3e zKGu@9NXD>4=~{dtoda@Sc+z$27+&5`_}<<+{!eA&6~zG&MiVyeB2=;h8IYd@ufx|m z3LYyO1YfG)3pZ5unKICdQ-;)ukn^ypGu!ckR{voY+y)(|PI2}Y`)#_agPi?C!^xPE zD#SCp^ga_XeIaPKWtS2O{|vTFQas?Rk!fs&7pcfSbqVx?6FsGi{;)<2m-_hfy0nhE zrIC>F+M5#8e)`2&9^wdfI}{qi=WNVCn1-i3Cu|aKjEz|O{xtZ!*qF=;0Acld2oMIV zVXH)CxC`uo%QoP1up+F`*^n(!m`rg$|#|bjGF;0TI`iX*e{rYp^&Q{zZ750aeqr)$U9?_gQto$lsL$@ zAvUlqzzmd;d}>&;uxQXX83LX7RM!1*L!Defq(>eJ(kuu~l#hK_oVjnhUW&m!NpON$ ztQorS50UtVr~EA>!MALgu{myGO22RaBpv*ml=RFg7 zw|#F1QY>W75nT6D$;f%l(9E2nYG5VQ$O}8YF8N0h1YOazzX0uC%m)_SISme(W!Qc3 z6Q^6Ja*xtwbKCVkLlC(}JR{>n0A-?lkQ8R;DT?oobo)x+oJ&(CT^wyz6fTylvD`nT zup+`WJ=V&t2xnMua2OBnh7NZ>a8ZSW@2z z*=BRk6JHFR*YftF!V{m9y4xn{KE6DVIv9N(c=Jzp<&*=KI3YPIY~7?a{^1xtq%nYF z;q)@|>+s8oLL2FwEcdhdB1~LcxGGT|db&OD=!I;?sWdEeA8dP7@(D8Lai~aP(W@8P zd{(DJWsm=eMT?0okwFqSa1-*I8d4wxG4YC#u9qLa8fuz>Jv1#qWiXsz|f3W0S_ z#Dc+T3Eqg;WhK={bUVygk=8FV*iKlo81=IS81blgmkcBSR2r?ibCWbtI)NbNuPogB z@l5xD)%6WcY=n)Iv4e@ep|K;KxS*|#jj^GVxvdSYp_RU)W0tbIDiQ;7xBO*js9!*9 z1eT02`VdNm8I7`MWou&K$t!Aco=8aogSh8m&D$>At*Cpac#FOXB{`gt!^76ibxV)W z&+8w6mc1lkVeo9E)J+2zTpKY=%5zIv0U+?LPf~xPqZ^)xNFjDPLc3A8e+D29-DWIp zq`R+#&}lKcR5mC?Tr6V9FuNAz= zgpI7|h4sd-C{rWdN&kX~Sau>PY{jQOL>+z{F#lweoNDS6q@$|L4JO)`7B!l=WUUd@ z?y3YEIq@}#la#PzFi`|YWkKNJ@+V7-`2D_sr8#SGULPzmXJuk9fT)7Z0kaQ~dr*D%$9V$f9Jtm%}JRe(aUEjuC)y82AzjF&V z-$2N?S`!OObbUdK(@#sJ)>9fXeXm94uND5PS^Z0C2v0wGfsv5d1cB}5-&Hg}<+)IG z+2RT_Aqi9xTZGs=q)Hm0--Rq!R35M_-PzIU-th#QB1AyLFc)3UfAg%O+UX2qA_cP! zW~))D7$scORg;fYRT}M#Zk2ysJ~)N{?b`I6kREO~(J;NOzS`gm=3|hT@Q0u_b%~_% z$8bNX2)X{}AYPw7*jwgeDES*(knRU(OY+XLnMsObpopJoI5GM8R?@OfJg{3J8o(b| z@B3K9E?FPsCgj#+8e0PU6Nn40L`P&#Of_CGn1PBD^5D-RL#mhvCo|b?EE-f)Jn*Q{?ZotHICy?rud)15yVQS0j#qN3w*JkYr z(56%^O@F-i-EK0sZHa-7jBsma|E7Nj!x(qsW(YWgEw08QBv(BP@S(;la zY8%c_?1SA^mTEO&H9J)uo){IoazQRk`(}fj^6xtwn(EPtSkF~t&e+^yY$8^dh!|B(Utq=prNED+0Gv{l@qsk?Kr@&b z!y#h232 zw@*fOMN_x9mWju}`TJu4s%0 zBy$=zqhF`=+m-tFMPzI_5C}D&eK`tXgxsD_SVul8Dw2pN#)Ab`iF5F?X=F`R|y;B*hIIBnEiyU}h{#+9Iv- zEY?N@G$@@9CAnA? zhYeQ%x;wVMED%=|mHT;ch1S$)Qb4VLCAaL{D)@`aN9^Z(?o%wZ|E8Ep(5ff~j(@^PTIq_R z!fz-*6lbez*i&q>snd8CC!_ZUFXUSB)>H#D$+@H(m>w_8B(bVSWuY|>^2K=N)X?6^ z2BzB$HiE^X)d;vHz5Y<#V@U}&93Tmcl)zmHMIA2(=WPr&Fn_GziX?!*A#owj>z44i zI8JO&9p)PydYafcnQP^_c3+MelXLM-(c*N0aGN}PJu(_CONV)1O-`~X4|bZv^1QZw zWJoQ*zAx4H$Q-Shc&vVLYB-6}O_Ny3sH{eOdA~D$;epC4M4U&U4I}197b~dB-@xBj|~iZGN&#Q`~A2X+8%-+QJsR@ zn~2-oT-ki$E1}I3V#0%=d6xtT9>-7!a*A;#)m0HUa+p&mXF->dnskwEOcm2})~L8? zgj4ij-EP_}#f@i$lZ5$&ary%(Z>@>YUsC9p1DhrjgUUn08pb0O6fB5}f6s-pmE>Nj zk)?W3O6}0jkS|i>wOG*m!v%45*{&4n5|R=bi5~csi`~|Z$>McrU_RqO<0i+VIS68= zMYC^PlK)cmqx`zZl}Ru8j0`QR@myy(_JAr@b2;#WmKxWy9dsFTi=xzzPQ6jqmqD+b0U5wGZsKs1Dzs^9Z(kE4^!=hCaB7Q<&)FZk1+ikq>P9Ud_HdhC47KTHCN}e$0p-A_RXew5KejjB+kxa(bsh&kT4t zw$%*X${N}p8AU%n?i8wSqv>QKoQ$bDk|efQ*xnwE5;>~_*++)S1s=X%QyZo9F=4$uS^{Rs#0bKzYpET=-%ag};~ zv_`_W%O<7ex=0EaDuv4n9GA*JdCGN{ThqTDnK8A|NO5`XwtYe|4R1hSF*o* z;{LBC>+EE1W$tADoA&ZMIXRdcI6D~&={xC5>)ZXu;Tol6skp$0%)NmML?9Py3W)bg zZtX5K2(BQ6Ajgr%2R@fZqpO#r=Ol2P^sJCmVg9FmO( z!v*rB^_~RP#1za7V?#96IG3bGKa*;!rM+fxiFmHhcs&jK6ub5U*-(;5Y1r}!EkeB2O9lD+>G?VVleib#4%EbG7&RaZ$V@+ zCVzehD4eRE;*lM-?oY;xs!FhW>0R1G+0MbTV3nQzJbRRoN4F2sf5S~f8RJuR2u){Wf#JXFeZ zBiy~5DIR53A4%S<=dNe_TH&Vq7L;$#M&DvE!U;z+Llxj; z#!Q$w-4ago`DYH&Sp3|l01i=gP4tNpH^Y~<6@_a+VFW&fTu<->8ppkOm5h*7&Y*E7 zoD&oIHhK(M|INu9Y~jXBqdQ3A3l)x%ZELnFctCgB@eNCR5Ds^F=ReF)dzKJ` z*{iI}9Sw0#{*23~dxf_a!hpd}JZQ!_WlOo^5|!<-uQd|I$a`J!s36*ptO+YVFoOO0 zU(PP=U))&jZ)oxO&0>Cm4gYUK5VSHkws8`-F|qw`O8DPOKm}>p4L*49i$OdPGJj;V zh56(fl`wyeu4WXpRw0QAK#CPJ`(&p$Ub*_P_W}=cLs$M(U8u&I{4j{-|c^ALWgly%mKj@#w&9Joa(FQ&_3BV;Co2r;L=duSiv%%mKIQ zC)M(oxGJ?!T!Xae^?gID)Updwuw~|9gRqj%=z&WMoM67st&@p(@@a(ZtHd>%Cy=VH zJ04*!M=7|PdRT1jh2*h9nw!e-4mAL^KW@!V2{ipx<)tgUV*dil2)3D3Kou6HGZGo$ z{?C!n$@*m9@7Dnoexo7B{}%_4bFj7hO>*2Njotsl51i%4WPteLy{A}019!AG{}C9m zafgpX)QF3FY+hj`(_5?8&13~%JW8u>aWs^NtAEdwYK}w40d%W`o#2d0dHAps4^|W~fF~goU zejpp!#jIz}9Q~y04|d|(O?=E1<|L&Hp+bJ4!V6(4hWgJVLPfT6^bHIiAaMy~z>o=< z#&b3Ni=@|b1N7{-la5pTXx}bA0qmdZX98NvNA(gp#VyN6#l^6YkBGGgCwcUfyR}D- zk^Rqht;}CK?SlHz;CV?2T?@NPEcXX8`)|_WWj2Whji%bXNmrESk z|6H~pB1hp0N2Y0H4J7}vcSD7Zd2zu;m8UxhnA~csR>i!v7wH?YB-{-DO+y`#(_dB2 zfg0;K3PbC#QK;jL3QoAzYq`{-Ort~)pK4h&NVp#=oJ!!YSSiwX=z|ya*|N=Fi3SUeWtdKyq^lc3&^i;*F!b4}xRuxQ4speRNw4h`&m&Zicd@8%5Ka^xJ& z>|gM_O<0C$n23&_RWFihC);s#z3fc{)dvexZ=-;O2P4`_MS8c<0rEUk2v!|{%{QR9 z&Z>!|xo9;0TQ6$@Wu@LPg1-N|uyXRUvux^8XfL(iLQpfyOyzE^leEx!hg!I-O8w^p z1txFJd}){J}_>tytA_Jm5Tuv@FRT=2A8qB}i`G#sfVMchl7&9*ZBV;s z8ULX~G|l|lfK+p3>Yy{c$$;{jX6BSZTiAj6vjn@FUUT!7{-8~w5S1q6K7N<;1ng9A zS?&{o`(I88G%W#x@Om(xuGEL;OLlXNxVykN#~?4dC3m@GljfAWJ(FrUf;vl{Ow)Hc zZP_z3hM5p!`PgRaiI5BG4XO+O%iU7ff8=+-6R?z&prWT{s=&O}WM`{!@5YYQojo!k zfd^?Kr7m`fuvARWBi+^BXLgOSeiB0S#xKu$3`xa?kHI}e0i z{>~mbR&HbF9J1kfubkB*8BLcOny-W|Y~jcnR5vb1CW&d(n3h+Uf5>&`rk{e~V4P{# za}k|RSy0+rqsR&NY|a{Y?xny=fxhqcO)R#!-@Y5W-0N+6x8v};^i zLMf;$x2P}HHDn*BBMnO9-a;%A-F-Qfeb+#GWg}(XLa*UW98q*oGTo5T?dm_=YCj`) zRN+cv&@)x&qvXP@FD+&*HdrKL`D@XFUai(c{&%hAc742OR!6B2Yb}}Kbg`~n;vdaM z2J_3QGdG1^t9WVDS);^<1dgTi1bJhLP8Shf^_ssS>XUs^_HqlQ38Kq{XcsJRDaQM? zbG{AwH)JP;bHRSAyLo0IQyhx1J4$wC2fhu^t;QbbS$oMTkX#izLqPAT@)`4&TF-D8 zk6Z&+J=S_zdmyT9qqm31Qe$O3Y>S4iFq#m{7N)kj2V2ig)zs!yZr%fV$cMB0Kj#gn zStN%%xehkzkqY9(3#a=5a=T6o2AGSdxc=6893siHI54qENzn^u-q@1(1r_YgYC`X= zKMfcxk^2?zx)C>L#3m_k(Wy>EVeCXW%92)hTm_2_n4BsVCwgTiN!$65*@DE+baz6l zmcbk6UM6R6G`uo92?00i8dBliIMue1p}ocP+~f=mq9FQAVMRyN{J@;oO$aUNH^Ny} zaIgtxz6szoo!I9#i}$Ojh!LVZq@H>M5aWtHNrV0JcodSCYej&g9^w%X>_u%uLv}*# z&2emcg{Gl{?Ic6qHfNp21ne<`pANE*ZyH@oU!U^}!qLCX!-7C63_rRi|M#&?#nsLaQCa>PN z`;)kme4A!*IP4kA?X^i&V0OxEYw~C!;Ne8cL1lM7iUzbjtlhb>{&u?Bj)+^|;(j$n zr1M&+KX5k4>AaYPmwLf#ZIpT3Z^mMw=|+pgmw>wNlt!QJIgETjMXUD+)s4_d^2T=A zj3eMVCL5Z_d(d|TkZYpk1eHJIj;9ATmHCQ4m-;j)w$x`#7`{)*aLX!`9f~^;bcnt(RPe z&YJ6P@?E&?P49kg$^c(qfwn%k-C^XawbygY^>yrEXJ%8|VRjiAqtAfM7>fqNfZ~n3y50VX zgs}^%`_qiRu?st_Y)57Y-WgzzwrZ>e>>258mu1h&riUFsG|bT2*9zVlZ%^lja)8uF z#=wK&1$hACMR~w;YcXuuYl-m^C~MdW+nId_;0AoK(1(SQ1Bh$H1;UeXr@kv4$r>1f zVGC$$?8WkmJ-pF-!pH;X#d84Fi^kXu;l*$u*_*=H4eiB#K-H_t*bV9R_rS7ug|Qpf zi~9hpmzA;mj~CN{W^W5)H;fnO0cWo(V>gr+>w#zQ3u8B&7w-XhFC3!}#3%iMXm1Ro z5A-M70a>rD{-?_A26_+9C+zSINKeFFN1rUC58P{4WDZbI%-u&Hp57<_Z3sFq%s2f( zj9-rar~K^*x-a%8^?|G(@7Sy2t}N3x^lL*TFZegt0i9pZ=&R)J3A!)FC(5ucsBhHW zO5cv&r}S+Lx-Zrz&hQSXZ_M3GAFtl0;B5^05Bew4FfZs&Wx0R zUFH@7vJuaF{%fNbl2K}C1{nYVQR@HU!NGsO@cM6U-v7NEP|>$?Hg^1v8ZSxXPb#*k z^EIw`axKkOb@h@2m1aH3TYqv>v%M6wy{+qvm8WOBsCT4`v|g;WzLC|KSUo!oA^bHH z!5`vGg3T9mJ48e=puSCFK72k{dQ}M5R)$WFk0suQ3%d7Dbq~?5^7owz)f?p*?U&6r zpP7f}CR*+#&AIiK_P-77&S5vwLdJ}zDDJlKB7rWk$#pkxsvq0yx2JIjWU#pGbG&W~ zRUshNM1Ewa?szK#PG69#ko_^PF=UG&SmYE0hKrf(C(ZQKniYOLlnl zZ?N+_U6njic1bI}I>SrwmmR@<7}5U999FA`6X5-nQU0c|YE4=IHOL;EB5vkFx77&k zjEa+p#pbm2D;m-w_Lvd)TvM|kazr!(0fcie%a-Y+W8?TrapsBVI>ZO7@%VbX6D3&D?zPr9ysbBiN14jNBC zkeZc7-8Tyw&p4GgC~O4^ohI%s&)7 zTMlv8@5b*K?h^}(df}1re|xStgge>L#<{^hoi5}fVB?;O5c2$ayJiJ7$X$6 zuzJ2wO{a`lHc*4Omx4$PseESN>~tC?Oh1Z+5yfBLMyMR%uB(CCH*MZzl6WQ)81oBZ&Fx?&`Ehesd|xWjQV7{Cjq74?HLGopikpxD8MmM-4%5w-ZQrY){^ z(X^d$q_Rc0_|%Z(2DM!q=1{Bd=_qBVC@c37YKH?A{Yq|HRl;#YqW`JH`B4zp^ss^i zS{%`;0nOQbmHHm3s!vjqy3Sc2khDeitDnK!XNK zr#I0B*&Sd;_*jQM8}Fzm*ZoH^bA(mCvS0cknd$Dtufz8BlV0?m+ab{AZ<3w3Q zy{M}aCxW9`fs|qXrtAu&GS`C*g9RKKL4e3_QPq$&U7F-2_}qw?i)rX(^-1f>id1SBbz0Nk~dKa!`X zRz$?e9Zq5tMAPe}G?0sfm+?sN`V3!1neOU4(kTmMPaCNIGo-hq52xQ zOfoaZjISJ1|LjSyO5_e`k}y>EYQteR^4WrLM;5}Uh4Q=OOO5Jf8hZ6|FcK-V)E)aQ zYdQ04i`%QO<+GEw*Q(dD2x~k8*M|5zdtm3|`J@f?PoxqRN0mSf3hK5oi_ezwr!4y+ zvufjG*Yk9=D&=IwgHw*dg#fv##WUhmS_O?8awi&$bW%)9EW+W_^>4dn&H-Ip!>*K^od+jJz8gWAb3S_-Wd4=V@bxHrwyQzbbZqe7o8 zse_82G)pWM%7?)Ftk#gr;t`HVDH5RBl=TAs*_)&zs@3+h<#BW$$dE;KM2_R24rLKW z^Hb@Zwmrz3CZJWzq`PY27+iL&F6CuQ>k6mONp3Q5qu_e+3-hVEfu+vshi%3oR|E_t zjTa`gJ-`vDUyFu_nX%f5@Uzp^(gD9M1v1-YJcpJBd}{ygZ;p&?N*_P9TL(a5&Kmq* z7&{A~IJ#}$LvVL@x1hnD;O_1o+=DZ?OK_J!aDpTRcL)vx!2-eEXK-iUeCM2d?>YBX zz4ujhSMBQV>gic~?cKZg{{PnMirpSqe;QKxrs5o+l#V}0n|&Kj%fd2$X;!Um+CBs) z)qV=ycmK7|!-cKNlN>)qEm9x%oaHU61?=_YP9VKw%^jmDEy;C9WC^VxFY2mt{%W_X zX%osTtnF?VqQ~iI+x(u#MWO2TN7Zslq zUU1&%UbvC5&)7mD=@x<`2CW-pn88Qpe}PjU&)z3fAh8*`fc?_b=1=w*F}^ zwXzn~YL>5Li7}RW;YaI4*&j~MZM#o(Nnd93#_ubL7BbS5)y(2i-zs#xdFr7$llc8ffOw?R0@o-(QXi-*)Hu`N%1T6fHaAek?1KU^j^NryzD(oM!}| zUU9#p!_gU(A4_Clo2zQR8c23(F|53TBU!N*ep}9*7lnwdFRi-cSg`e>**YFqUrKBW z;oTGyjK9c47yb$pQyLn)=`4v{#vLmT$xEAoTv*4hFo!9ENc)jinxp2a=^K3xn(RM1 zmo8c48N*l#l5NV9H_COS{YGW zIV>cJygWsmrbXa9oW(Ix=$O(sX?IbgzAQKNR>`owU7{=3kv2No{WP!Oov`R;hLjgw z59&eB5YEb^)(iVwe4}JHCQc#0K^+lB)KvQ6-9-__?5HUAPk-Y$Qk` z4rz_d5~^|1z>EMuybL#MWib3573YA?M{e1=o)7d=$9ZG|q{sbbsNTYvbuUNM({s~?Frpz8h^4UPxgDT6J_R|Bb0klp>(vD zG7?V|$hWWaI7%b%4e4b9=b2mTi*_Sz1B=9@zqY>3n+?)f|B$;~+ZoJW?Ed@IFx>B^ zaV#KXETDNTFE^uHGxz3?TJ4Wn&%zeRam!BCg|^IPE0=WJ_LuR>GTg2}kvJ7eL z2Z40s5XuaQnBi+mJ|KJE*Iw1S-r&dAm{_&+hql}jws0xm8ZrvXDYofPm?9H;hDs^F z`IT&tk{uk>h_py!_d|@l@;GdFF;y9L&X01?eppeJ#MW-K%#%1|@Z|%PXzQm@Y1V>_ zzS$atS9trwTphoG$>cL;bK^JhAGn(AnRb6Fbk?8tiSJo^mqV{TtlD@zEk)z_5R8y-#3v#%WS;7xCO>6m{ z>ksGjJ87;gT(a8R%YEEG#^S$~)SxNt$4l5gd^0w;3eY`Zs~0wrP3LPSjm$JFk0_`@ zXU8mqs}BvGgN91bOCO{a`!K0AiPp+3&I+F}OJk6T-fvZn{q~#zKEd{VqX8}a&1gtf zjmm7yOxE~*rL@-YTziR2{JM={Y40qMWibc}o5Zl(~pg4h}ug+KU#1rq7 zlPS!8l{7^|@!D#UL*gJrw{qgfLqr3@G`D-kf(2%dv2l?%cwd5g(kyuPRmk{N_8KgMO ziLjDb@>Fs1G8ti`uO9+NORma&14a+EtXzKMxKZ$k8-0+-NyK#=RPoC_$z$a_RB~F% zeJeZUlsEZpyRg$_lJ!`uS()<%lB`{yb<_O7g7DLaTw`VtD)3v0bsEAdNhc`s?K}vq zH1WS5-NM7V?j|@WTDt4b+?qYXDqQcV`muC(YwCNw-`z?f{BaxXgLG#Gcm}VZBvXns z-!00m>;8ZybOUBTka^MdF(X4wNcM1^2SE*^7sb)%yN<74eqx=b=2_i$QF| z2rOkvj9(8081QEEMkY2UJuw7#Rz3ROA9V;s6%Hsuy|~~7kId1S6~kUUs>DL<`yc%H zhag_WFD9kMz(XSJmyyH}npptq{VPcwL{_qw`1#?0>yqYX*C3 zhH~dL zG!_Fw@1Nxx@6;_o($o;HM2P(i1o?0S;WB05Q5^B)NpSN8r4N)Ix>QMBB#v<(jQD)R zx!C|f00a|mA9!I)6vsS2yL~e6g?WO|ozXQ{EFyLY#6SlsOP+j>w*a6$66@> z#w5~9B!G2P9qy@XqvZbDzgOyD>EH#i(7gDL%sV7_-EHRK272%&0{Da>417hr6MZ6E zyE?rJi9E(DzN7Q*c~O+p7Ta>OjxhlF#+>(E#h-(=+@%AFf@1r2=*BW-J%|*BOaz>9ymfD&e5(+k#9NP1O{3@ z|Fi&5Z3@7>cEN+Y@SmYEnn59rMH8@)jj$V2*!xwg#kFa^woUWRo{~=PA|$S$g$6j608JU%yB9J_dvh62uKErlY_<4?^ls8E|3L_1^WHFNWh|R{3u<{)ggqXR7dN*uiSS% zHz}QWUjYN%#V^K|FRiezDVJ1dlXp4MSyy6Nvjgiped{~@>t~V@u-C=Nuc9cA>Te*g zu!#VkL;&jOXTWVS>Lawe)R6$9aL-RFU@TCvY?nCM_K6U-IWX$pk8)__ibmWF<^m#* zozev%JUEMTUw?(4Cc8cmzdm?PfGH+|6Yr8BI!L%X*zSUwHGpOFfu|3&7rMa))2sg` zq)y__T=u#a7Ptgc)PNvrj^x)v2@ZnWyproXPz?pyLj z(32aEdNBe7fcXsZ&ndp!pbhDPIeDP1i-8tjd*Dx=cvO?;eHod zfHZ`6H}v0k_P;J!J_}g9Y(%~Q(7=!Un5Q;vuM>j}kDDV!Z7!=V>71~V|-c1$O zO%>4%feQ+GtSly4775>-QRd~rQUvH=2UA8g5Izv@6aK~O=Qv+anhGDaRYY8ci{fJ z#q)CGx@qW=Yo8O=rVr-k&nMq_;G%ycTe=F>L`8}fQ|}9rK{|kRZOp)r!{ir@T2Rk*PI>r zEXnNHdjB-9te~_;&CdTAtpCi%o{xn|GO%g-wf;}EYy2~26B25)Lo5ZN8>6v3nBc4Q zIG%%1#SF&xr|@DPdY(Cs&F-6>p@4V;QF63@YVSg4*_FLNdC`3s7r?=i#E9NLdz4aFAts;L<;PcgDd{>Sg=i?wa}U z?urw+{dG|p!DvnIt6sG#dMVA~Vxy{R@u(7$jngtir!?NPs<=)WucKAatvm^^&x!`D z@oj8!`Q64t&%1+yhpW30PMBIh!k_qr_^Yv-jIOdZ=!=V|*i2EIswvR5}(8ddzZ0o!{8 zqKB0ROUBRnaF|`SK%JC3@{H@txAw5Nk0G?9DPqLw#X1Q%Amb5Ntt^-5>}5?I*Gnht zuObRhj_7B*b!(1o`||7Ts<||8Pbo>HzX1-UPzX@yN$lhU$}Mz%28(_QjbW|eF0;Z& z!57tQN^Xv4@*8jDafoI`{jK3au%t7&BRsewMBMco)i=LoZ~Tg z84tYC`qA|@yXEH6)xSY!t-6a@_t2u`MYeO6_c1+nELkEo6C-%jWjtz}k5_!RwD0~? z1gqOCl{COqm=H;1b)x+fnGa6aGVT2%@qIgTqaUaMo*3TPXJOyqghc znmr}#w@{f^{I?Lio)=DVF)bLQA1Ikk9mR{FFIWVLw?O&~(jZprXrz21r?7V) z+bN<;N%^QdkS&Z1k=q@j>rP~&WW|sOmaI!KI;VcDZ&1P84ofTYY^x-H+NJprG)y;N zg#G%z;KJ!fw0T#khl)W(*8G3ZL;r~u{>xAQwdAC2`d42nHh78gGb^j5C1e4JFz@oh z&&wAp`UzXAD(5!J*;2Y9Ox70(7n0=Gb6*noTL$khoXOALw@H_OR{V5&H4rG~H9a+% zQf%G}_j_PR(5j6>8s7*$--Fq;kpbXq<0lXvh_h;+^JlF#`_GR@*Yevg16vD%@yzH7 zc5m(G>(^0@SR8GOvgd8va?KFN^hBCH`jaH1>Q(2CWf*DinTn6x-$-a9KAKB#J)$2S ze->Z-(c^mCT+!IJ%qBS6BpUIFq*WZZtAT)B8DPBELak6Y7RZ=th&YHRG5PVwJMs_K zgScH~PNw-zg`aH&sX5v#o!Hm5Z7Le2owrcPsDjfE}@9)wtF3UvLAhtgRx zh!cu$tytn>{DvAIusk~!D`ry%KajrvL=*jBo$@0>npIeu_9cG(?3E9mNMrQeI-*E3 z@jM_0e>k;dDK$fY$_u|zZOERvNCJm*M*d@g*w{zCOUWAo`(7pI#DK-aUbS3|fNEHy zckOhItB>$us{+h{^2ln~kMMVvMT7oWJ(6r?d>K!PVEKk1Sfb$ZFf6kNTngMAmY7vn z{RLYk^M2hF+gu(kZIgjZb8LS;0#A%3u}M-IdR?Vd`EF_YdrB$WfZZfRlTob7>Fw~4 zwzp7t@5}-1;1Vygvg5Mt$nd%B9AoCj5AA=GJ!VPkIayF;bS2W?^5B1xcmK6+`aihTn=oyB|(XlKUDoBg^KD) z5FEycOBR1QV8f@-VzajSf=dmSe-(OHK%&+A@-)!^14h1dsFLDG2tY-T5YlAktaDU8 z@z3M2i0^m)B_T@bjfr$N-Cyg9(Fz-}SF#%?U7g+WwQe?dz8-m^j%O%AtnhvknZypr zmBX~p2i*2f>c`FHw-UJ!L{`Szfz|zPU%MQwRL+UEo@mEN;vBV{2&$=@>{|873S=wP z#g)ci_1NHAtke=PWjoEWa?d^=ohaS&a`oreVfv~Cx_>X0atq{cQ(f4}8u@WRVuEEf zWgQUjrrpav+kV}ic}NpqL9&Wm>Ss}2*o;x6P)wXOo_IoE!@<=?ptEzHNRWfYhE_0y zOHA?Q9eO>2!&V#NDr)hU}g9W8Qmsxnt{-7k}}V4S_oIEh+Uk=e+%v0D@wIhWv^`PZ8B zvAC*F)hZ@klhXN8Hv$p))-`o1Ylp!D;ncED5U!k{RgqJSBCaZzb+>mp3Xhx}@4y(R zTz+jEkk79jfxL%jzQ>%&fj2L&nys&nGa`o=0c$%Qg2!8qA@dL%KDUn@Zp3f9PP@+M zv4mqbmZp3;JD=bTV%}kUw!+qO!P;MZEkJLeQlTlP3!?PT10(E`416E;iabaqLN=E% zqwk6giUZs3vPEX6>LWL7`zC>Pcby^|=r*XIF)vjIPDaTh-Dox#gODCLwgN}T2*m$P(f*_ zV5eVy$nN5~7@*or(E^^jo}6b`KQd%T*;0;(o2e0LQS||OpK{Fe5@XTa#mXnl-*z}l zMzD-|R>*n~s)N%`geS`o)he0751qtf4rf?Sul$6X4v^$?<19GJOUCGpO4jyOgR}Ivd zt;80es(w!@J6;eLF(W5(kGctl-jfQUr9WvaL-@zH3Jv`#Cj9l^19?pFnJ2XN!R{## z^6Sv!aCgn^j0YFf>l97`Uz`cgTMu#%!~aU(OGW8V9`T1js7kT)yX(ElGvT+MUgVa*SpiQ58W9wMX4P&qyuXLGO2C# zU9>d$J+TPfSGcxum7r}QK@Q52l0>1mAAb>Dl&ne|t$SpcbOrD>%2$|o%;Up<9TMed zv5GMr0RI`RT~RgaStXG9I;>J)bdQcLWa264mGk40$`}gWdJ!l#aHj`TGZU}W2pku> zVBl8A*kT~w?0dN2qf0hxx`ZedL(zqRA8F#QOLGf$wOQgP^Xeys0bdLGjeNei);^Gk znLk$=kx_9lWnL*gHvhWM`6ym6UqfLK4*N+WkuN7qN@07CEDcCTEEmjBDzA|ucTIeJ z!nHn!SLG&itvcUI8-RP0k>+bWmY;k`5(FGDfo88RwS6^ZaFZG{*11pn)@#e)#!$K! zpilQmdHkfb9uced*(~8uzKTlz8gE`b^D*&Tkx!HxG$YDDG23(0fH~`P<}aYCEu?|q zRc^}|6;KOyiR#t9PacRH_>Q_{O%5sCT^Iq9C#&3(Qn%kyG`sZ!*3AF24F+??2ckkz z6l>`IH&`SnPyK(rYE@IFlm;d7qkt|ea!+M0XTwRGY6v8&2^#5d!}(ZLNmrwCFnJgl z8FCRH4eH^E#6z(d)ZS#8k&wZDkKp9vLQlrl2 z;{DeANs~2#dD%nNRx*d&`Fz~G?s?{-TZv}ir<8wDcjW#r%sSf-ZOPUoJ!ir{YGrd? z52oblbSTy1GAnR8?q~@v!%9p|ks@1Cp+F~-Aj9hLvIsPIv@Cg(4~;>=UcuQHg-HA^ z3q2!_OXphjC#D{zM1D<@CVHifoTrYhrXdESs#k{{lfq=9a${9^(+kar6oNa_RejM9 zU=1G~jPmo;0FotiZ&z_Cs;U#L|1|Aqj(o)#Xt(wTYTCas6#qBV{(Il1n$#UX@RxP@ z9G?Y@jaJ(E%es+Eap3zm&Dp+td`cCh8zTlc)xjqOhXz;KGO^gWd9-hAne-HIED40t zbdi$tnl_s>qf9N^c_1&=ZZ?{Ng5Iz@^DnS+o8*e6xDGU#%Y5igxEk}^v1dWvOV#<6 zh@uAaQfF4X(d5;TF62VKi7Kb%=h;2YQ6M}vXv|Md<<+8})OWx>sbydFQhyJg7TtTh zeLSRRZjVFittnZ-S`tC4l*`SubPI}o>Cuf<@!ajcc@8GC4PA_r+@d+l0C6GGLcm>$5vUo>uinOwiW#p zxuS8bJ2CS1w`V6={(ir$JmyN36>y*@366ZFQKLmuD%Tx%7>_$UPS27Wu^QL1K!JPC zv939-y3La3BN1D3#*6`Esh4U@0}oxm+WR#OyUw4hXzUg4^Vws*hJP|+3Jx>auKrnP zYypP*mQX+6ZRFXtv8Brw8Y5qF3nSV$!|4KHnbx_iB~heq$T6USt0?LvhN$*quzDz8OP(lEk#AsEh!fj~5JuFfQJOWF zIYqe#$p*nQG=q3pqFiv#2MR`C%YQW4{~~!V@w$~b=ndL{6a#>dNdWU+6bq~nPHU<( zsXQO0!?Z1Fo=Dg0(W-tNlJ3Ku?;$Ai4}#P?#OqT4L^#3e?2SQRtc&8iXy`P#BJn<1 z@jfnJD4y;p#3eoTy)X=WXgZHSSt=K4WU{tY7Zpiv$Z$Bc95jpf z!wv-~o6gAT_map|Lox3KW3aA@Qxxv26757MfM=nrM4s3&Pfr|I|(r1_p-+1R1 zzeg|B8z09dX4zSckklCaNVihigJnq(iJvc=jjRnMYK%M<^xiF}_N?CfEkXlk$){DH z8aRjND^@o(myc)}Y9|~;f6ZlD$gul~dr@ckf#tlSJS=f3Emz#$vdtFRsgZpL5yC9ZUE!ab!z2Ek#Q=&-{?2*I0bsC)w&)(=Y11EySFg%)=Eg+x&iC z5Dwh&M={p`%LXa-XM2QwU||uuX1PZ-`vlfSFdvjf$OgY5ST6~0*pWrp=o26aLd#|d z%D1d!>MHnZ?`_QF4OdfL8$DI_oE#3?WquihI}gp|Cv}K)m^v$t(oQT*2IPS)f~IyYF8ui z+_9pMsbK9%4H}NJJTByV#jz6!k@|Lgz%W5AyQUuQzTl)2djEyh4}~24#?TX7SLS=6 z5Sp!uaO1YGo7wjSgM}$*x5ItkzG~ zrn+Z79_9U1v~d7=}oxr!O-t@I8nYr*tLtR2w~&6+}K|5Kz+h6Wmr-~g~t@hz8g zEHAxs(?L0V0vO*{&z{$Mh>~E9M`}-AF3qq<5~$-2>*}pjV2XRqUFVfMI#G_omp<2_ z?%5EWuzKlp$!Z+L)90Hiah6+<_j$0FJMFE#zV~m^OM{OUB($l~nS~*!;o4Psd6vG! zcpbTX!ym>2UvPCgw6}gyE34Kn1Kr)|x57S9Z+uWzbSZUVn`F`NskGrK&R!X{6c{s6 zXcFJ0IQi|zxVWDELEr{OZO6zdr%0G@LQ9@^yq{8CYi+@#fO7IEw|TR*XH}c!JngKK z_k9hyvlEr;2?#_%`9YaS=dw4FQRsr~I7crlY;RzPBAT_Sdq?BXtc}FY7y;Z)-Y!>* z5-sx@i?+7Q8H(A*`m70eaV`IAleAaOcY%54`du6L%EO1jFj=Rfkq*}CPz|*E((H8d zx=c`8U`g4X#1`8>$9vfMR1QrgxfWa2>XGV#x$Cr(zR5}GaN(fbxl`tgL;Lh*%U?J zy2eHgDt?W7&xTLwBOk!bw_}rq`KP7)5%X&@GA1S1s6zL^<-M9xHf40T*+NqR+B&?6 zszA}_K+e>(#kYm^2Ggn)K)thqs55W04S4}C!up+7%Dk^|JK5Ba!5Kx1a|%l+I^Bc!BH_D%b^18)NmK9hkl%pxFJoTQi>C5{;T8e&LCJT*b@=+QDK z!e<+`J|S?JMD7-cZwRDcPtDjXuz%b}_OD|~338T7A*R{?$&9ImE*fu2sVTGcw2r@+ z{yHo)n!R>g9f>1r-EUTF?N$|f94b(gffU{>5YG3eKP0R~H*s|?WMs~AWvlGM5;O|h zr8@`%(#;>aWg)K!8*szv6IvGJ!mW zE)q7o3uFejXz7r4U1XH{6>V(QFiVp3Mp6D?qC8(+fXvhg8lQN%LajG)yL!_#@zD!GEh#O#7Aae(a9>@P;X%-yMXU!1EWDY65#9l43bwxzgVVMT`S ziD+#Q0R(iM01ZasZ5flmRKe$3$<>ouy81XCCigK{lPv|3T8##lwZ=Llt7lO1wdl%C!n^ zls61^w<3uA*<{E8RZk}4zS%tvj~3nPi47=^6&LbS8@%iEMs^nh+#D64xK2$@->j-^ z5lGC<b>;_ed8KMyHetE$?=D_HTUx8kRkjVKM(8<^g&93$XSfw_ zP>nh{qAR~%f2un)M_YSLTZJ~(C0!n`QgfpY=ICpbj;OAD zB>>YK>2LhT+vQvcvYAUN2m|*U@3gikUQri!@3wpYbI^fJZEw?t2LppH|NjVm_di60 zwudtmitzHbboKt%NH5A5DThMer^r98u6`}E`B!~BJK0#1YWNJJg$lag`eavYS)src z@18^U?I+uxqoZ2RW0(xGpXuS>pmg6Ap?((3QG27P;2w%E{P}JjH*c)nXZmI1&kyln zvqDc`_g(Knk*EDjn1~yo^1wd^8d}}2)<|Tj#cr|1F4!&DpPu4lcynSfP(8@a%WwaB z*=t0tn2cP+XCh9Qci^hq=Y2KnvB69a$OYG%c-bRn$qx!exXkl>cKzLBhZJVx1SjJF zXUt%z@1JuQpvX5VLa~wm-flmTjK1L=`mg!lJ<%RWrq&(r(65AR6SXGp7##zVS zv#l&8f)Em-JM%o!C*7C$R==H-oPPnGrLM-l$_+Au5ESLE>}_>tdRaH}DG1~}n5wkX zOzUiuUb)Z$nlu!?eNh@}xpOS68hzS11(DK0AqeY#K@c-)XDo$x-7hPgXW*E~ejq-D+OwbRKeC5_;(b0eEu}u$WjuiJ~ z=tR`Z+X&+hHXTIw0>^ed2*2cqacXOl)Yk*RFVLhiC|iqGg+}hLn^K@>utIuP+zRZ<)7Gi|n#!gJ0R?K9Cg680sBM8=q)ri_NOeD}5Y7|FQ_hAA(HK z4<>5#`IKx8K4Il==3-8;GP6=>#3p`Mc=}*~RaTUt@4-1;qxA%H@gz$*tZ6y~zeJ67 z6ZN~g{21SVlPirrVOv)Imw^hRBB}412WxC=j3K+8s3r4zEsquyHO>?(PyCKK`Gg_Y z-mQJZj@>}x-AySzXC_ns4PrX9c!qbmHaduur2`Ko7B;PF8AuOBVxs-}qv{4cnCWJp zzB!b3xmNB6A(W;6lWX_@iRzc}ikJF?nY~?7LSehmbqj!U4a$S8M-Ih#_!D{jN)AOn zNK%%!Z5a2CULR2m*Ak98np|+go8uIvvzjY)2JHEUCWVzr(@ZLs?Cn@)*~g^x64)KW zFXlbpzL!VnmWx4MS@_6iF=1Xt7cr9MKc8Ai#B+pSs6ydTgt0wf;g%!i&>+Q+&I8SP z@80&?pi;TO`;c(NN@Bbr!cu`t_kxRAnTJixw>Pz}APKYX%P`Muk!Ckz)}i}O{F=M< z+sJam+R&J7q@PHrUwT>x<88y~{7L6&$)`xA@-%MnnK)}SuCYyaJ7uM3WjjQ&ECEdI!Q$K5jLuKfT@9ImzG%IM92pR19d0b9T@I2$Z@g?=x%QeWzxVN*Pj3<*N=Z2^n+P~NjT-Mi zjD1dxouvh(U1oPjMdNeJwaaU?4Nrr&zqrJ;v;$R2Dw-O-aK?}H{s9{-`~x;nb!$}; zamLkZS|R$GOvPWvD?v(Bt5Ymm?Z3#T-#!>PW<~y0i~ZZ=+*tVZ92u{y25%`dVFp3m zw&tU~)_A9;MqlZv-i(|)yzSd~6xb;}b`3HD1cQ2_4BO8ficIVdwK_s<9ySi$ns`@= zzCAdz(@EtsI=jZ+9`9wBxjtm8{-7nXOfKLV3)My#HYT>Oua1IeOT=S6>^S%S)izPJ zYN@%(zD?Y`{j*1eDznXh>-dZR(gMx!=fDnOoz9b-FC;HR6mvw$+ZfK&vr%?FUPC zuBTubUV>6dJlUO--=I)VxD`O1s?2V$xfqeywX`_<@&4Y1RbIeX_39UOB;5?=dE@+b z5cHd}n3=rIY&mynoc@)Bzt_X}B~+Z>=Y9G7;&QFElqsjhRg!0yDPJe82b^S)7#PR&mI9 z)l%^eBkK(I26jodzG^2!^P&2mE=yi~P)F@N!{h`V7COVVtnhd)e9T$)Yr^%-*&=d~ zWFi|zKAv^=)*I7QUR`y6m)gD@L}+8TPR8>XNBnpsf zvIwxV(!@6}Q5?jG3h}4=s7n}zmWF36TaR= zj-$1Dw{v8SK%+c3u>fxa>*r%$w6l)YajItA)4(9Z{-K>5y-oo9xB88Dw5U~-sI_Uq zEQNw@l;;YZ8Tumbge-ni#jd}temRK`j=5-erKY5pww8YK{zA!C?l`C9o2-x|gJ4}_ z+6239h0tMK+>G9-xrTz7w-j69M&8DWQ{IEDgw59Q3#h>G-KLm%Qp%|!2uFKE>xAcU zc(ovHg>=$GR_2*Quh2s^c=d5quZFXwo+8!N4gj8 zHD#)f?4DCI#;IOqR2##q`VuIua0I0lqM)<_Q-PGg%Adbz1=os?b`yG^wNrkJso5Z@ zeQ;1SSF)g2N!ur{7A5=CreZPp)iy)kCJ-h1_Hv&~hP58Bz3;`8LM7t2ww#WXQr!M9 z)NedpDC%>$I)-PDl_gNf)7#;nMcuab`|Aj1k?_aI$B2g?0tJF0)1AMa{A)#ly@UBT zdVx-&c`C1-3r3|&+yP>oV7ba_ZP(2-kWu8;I8*XNv8vF-z?w#U~C;WzH=HE}3MFCZK2*Wfm<_EpFNZAXN4wLzKu7 zS5G2F9gsS!EF*yo4&f>QO2a^623}*_DY4+Xl3|OM27*Q`drl-n-t6`woU~IeUb!0p z6~Pd)zE?WQ5Y%0PH?NgjqQJff2!CR4+^hgzFy`|P;cwvA&6Hj^KXKgKCuTvQ{JTeP zDAiB^0YyMwU^fFPr%$k8fg|C)Vwlfo7n{B6FpyU|@P>rlHC|i@ZdtL)wKnmKPAP;O z986jVIg*rso4(8CdNq`kAiI4c?}N050xaN`hAb)C?^1dI>>^%cthAt7vVmDViH1B0?Brlctw5QY3N_y2<`5Tdg-DB`@!8` zZS=nc(cN$4!=wnnQM$oC^7rk4ECFr^j~{;n|5!-4#k_REJzo`CfJ`kwYt#U&L;&X@ z1nv+*H@bmzc^C0qZ1Dm?KYEdQbmAV`N0rb7tP zAw*#2U(#r#2c}I?ag8xA;>F>@MhzHIHdM0`UP7{sMu6JfjRH6c4;W zNLKHPrA40+c7Y-^l+VACBzg{psV;hrFP=|ZSN(PyJg=&ISEe>zNd+VxiP|0(%}@4~ z0fl64FE_4TH~tb*H;KKevpOmF`Kv)EZXYl1THEfQ@Hk#yza;cZ>}`YGe4vNg@1JIh z@8`HSFmv3PCI#TGroy{TE!+eWH{fpX-h>bh_?yA|okv`C5>JD04S;Jfke9|DxKLnu zV~=^*iy7km<*U;D5<#Q+x;YAD>m^eVaM5nn+;y)!I%C3#&b9j zFcPwkcI)fb2U6-g5f26GHf}tHzq*CLE?7W9BsZvrC18fl;jR!GU-=8vkq^jV*SR;> zU`0DanLt!g0LI4kL3gvRUi{bWfsla4V)Wa=C-p5*uporay$9j67amdv10n^JgaXK; zp2nVxJ*cu_fJkqOu&f@u*k`dmUNu0l1vVbBe6GGqcwc>oa2Y_j`yfV;ws#voiG)xO zX>J^OG9Ax;y|I97SwQG#Wzrti-Is4|fG%^wyOg_$?N>2Eus`|c zm0#ZhN+=d6`)BYD^t6OJJXFLB4#M5Mv*`n=!}eCfgs8iGfB<7dNH-t6`>ser58lB7 z?O}_6(NA~eZ2(}@(~W4~)kEk38Vr!=O%dlCQFC;^9_TIpK@(i4NeP;pRe6rc-jxEC5T1!=gKXjr=U47&AV4fj$F zUxgUoGyQeJCA5GCEl5c9p8a)j2gy>*w=cjC`tuLcjXhX{kK%7mc3xWAB)T>T?_!`& zv;lpz^Evtc^%0Bbk^WaEi)SXwm*Vgj04ewpee*#gbi*FD0SIeWj1<%Zb036r8jNxp zOmZ5Ga2iZ-8jJyc5Z=7N=sU>|1-6F1^8bFqOoVVuLd5#+o-9B}p$Ay75UTzF0Qy1y zlRpl`PlEG#k16y(9Tr#&TO@%b3V_`KMPBrJkF<*V22J>mv>Kh?FR~K%k`6a`TsWI< zZf_W!Y|j}T6v;b@P4>d>Zy;Z}g+F{Qc9W9W*nSaR$p5NXQW?6AdbN!YIuejcz z7|O-veKGO+=|ahX`*85T1EbJ9GNylruK40XWz=u~|H2>WG@y>ZyX`+$9D`>k)c$B- zBW?nac|Uog`F>!z6PZ8jc^{9PAcM+Juco1wih@Xw0cQM2n#@Uo2 zb{@O*eZ44X3cdp$5ca7*D^N!#3HFS9#tujvPc~vd->)?;H;I-MGE)w89TYI_br#On zt6O${Ww`FaKQI4Y{k}0kT_L)5PiIT$UY!^-qO0B|4qP0>T1n#gxnO+6uNmHQS5@O{ z)iZG=q2;wp$5$sg{#dNNEKUzbU6g^c(P_i(`QsoJ;)KJ3Ii+aQ-H$eAI&&RaD3`*P ztO4gWpJVWfuo>x?&})k9J!k{P2%_i;Lv#q$_Hir9HlBP$K!Y8P8zh#u-JCZamj&RX z?6Z|uMw*r`Eym``Zn#>qS_8Vbi(T4#;~n6$ylQe@=kk41gYj}Vkrp>&jB>wo;$Gl= z%H7fwo3@9_ox<$hiQd!*`UO5Ti{!QrOde;U^Fvh0ol)3f6hx;zw} z0o8W_Lwl4JuKCesvxNnYwVFN|joiCmdt&7J4^?#_Y&Q5$G%1a@PB;<4=h5=J9CwkY zU;?a`fO{sUyDJoHrjX49?f1@*lJ~JrmTSsZc*c=sM)zO5yDGp=^;&KZO6<^(2gdgp z=J#gGO+)Ez&e6bvLo+|qfz`4<9+}Z(t8C?e%9iFYz86q{e8@MsBMj|=wENKIfc4Xoz#Z1_Z4?xML0 z-b9jIjfh`$D{w7G-c)R{`F62C#v<3y*`Fg@Gx_0MAEX}e6wpKo`oLn?eAzZiq`1ymqp2;ue>NR>gAt~bGC05-L4e?tx6*UO3_x!&i*?(K0 z6)J+NJT9Smum46*t7qxy09|bPXHOfYf10Iy$=nrmyR zoR=JXZ=~nCL1JD9Z{Tdd22U?Poqh+d_kO?Y861IhKTw8xK99wHEcmfgK*1?uxwYqe z>i08Q|4a}RnV6G#4Q6>+_UK1JurC}P@I7lglzeAqz3*dYy1j)#*>ezykzLrh9se;) zUBq&P0Ik8mIwMBl;gHz;B3NYo7q9vZyIzlDB=KVV_#l?S1?r!Krd;BQVgH=XBx1M3 zi{qH>7Q7COmH{KTE3PqdQ!iQ!ZE|pPx(#TwZ91(DU01F*v5e;pPdE>vZKSQtS=*M3 z*z|bL)`L8c>!m2Wwcc>bnP0YQ_sAeSNh`(f0L4k{Ib0o+Hx=@aVg-ZsBiEyA7WQDaWi&%h;%LrRx^W6J*3%s*dM`rm8aVvE6yB-{W+P%GRydzrngS_tyO% z(%vyj)@aKXOq{fB+qP}nwsq1rPujL^+qP}%q|sS*U%m0}?ORU{dBOxHC<0;qG5xy8z6{Pd9y#B;fWrI`phvZ2y zOH0P+TqfelmX? z5>;a|Kb;b_NXp)l@E~)Ec<#@dG~#EXPJmECqq7G$bu7s$s7L-l;A8QVf40%vMf>9k zq+6i3Nmb~d@@{gTq^f1SM={rVBh*&3YqZ}I;&$MF=Pk?*n0Q_2&r6~R3cM{F^LDV3 z^(n}$@u>zE9;_<0cAkO)yfJ6oMZFjWz-I#bhZMy~m!7`l)mxTis!(Q>vo;-;@JvJSy z=t6yx52n-vxn9Mi%WCZyu8Wihh`s1FENSok9f<7-#7)_z};@m_FvJ z5tBgIuArIp)+Z$@qD^HVCm*cF5N7X zf_n_Ci|gTjdq*FoN5MW=fs5-Cy;JLyL(U$p1lb*-gjo4FShRusf!b5xnEeg=SLlns z22?JE?XqNfdCH`gNU!ha4lF+T&}ylch>sl^oLk_`1iaKO+*hC1=h5DBO@0yYR_xQV zP+DE(R>n)JPYRu2bG$`GWy!j#e7(NO@gn(ZY3YdNXbJKw?3%+pza%9{CVdo-@&Z~k z1QhISX=z(@{EzPab|PjB08jbp#P6x0WunDL$aFLs9I!UtQhY#eQaa${BsuW5^>=<2bGdU*RvyXMHzJ|I`iE~HPlw6Wx5rwPkZHPdo)Rd zhK_8>WiIwoP*0M?;(2v^L<|1m1vu8`CPN}`yNxoY>V8e<+yqQ#G&U3VaNo(n8?tP~ z>}4yCEeqX|@Tw`fh&Hjw(7{WT2Lp!p!A+{mQg*}D8=~%=!S`@J%Y8mMuiim8cs%Ai zf_*TV52e9S8Lyz1(r`S&{D`Gxt|p|j#|IEakvSOgHnk?Z70$b< zlyFLr9SLLtj|_7*1-dzs^$F^>NZ-Uw0ZnM%jNqeQn8F>xWz2ybnM#_FKhl*vAsM8q zF$4qpc0w@&3OT?h!%k?s_SW3bn6fj1d0WzXUYlM|+^l5N=It0E)wEfC_1;FSQso-$ zmZ}rT$tlBe-hgRJ_IKmXme8*d7m8!#s>G>-+KHYxR1uQZL?`JXb15%@((0P0p=Cj4 z4%)lx!maZu>Xft|U{uxTe0lfMtzSB~D#Bplwg^1Z%+kmOT}#fiC#=>EmDnpO%j~L4 znvQe>$)2X%d)3`T4%g#-=iGvm*{+sf%{(zpP1P=KcK8S1IWx|OSskh4lF4R}Z9@>7 z%V;x0(W^6=2-(F``U8l^+z$*glfSlj#t7PC$v8;RW=Q=|0+jk8qQL!bkN`H~0S@>` z{dUl}fj27y%KWc%pQ2){QVMi%$&Z%V&FOzv^5c6cKL(|680OxLFx{% zD{1ihFbHt_IGmUi2~-gTswhe{&tG&-kU4@Z-dvg*w?62hHHykfaOvW9;w1w7FL1nl z>56n-Ou8Tsg&S;B^r=QZXugmzL4IVhd|8q-=3H{B`kWbQ!dcy^>GC0UJ#5&mZLYIp z*==`-+HE<4E~mr`j@&+o0&rD6D5Z>yX#+*8FJyM0XXYvGT!Aq39LiijVN0}8V^v_!sH4=My|(Ny>OQ@I)-9e{>o%k%Fq>!3jKF- z{n0LDkInq3z6~0P66IZ!-8`R3g6i-GqACeNrhtmgHFSAXN=$*NqA$G1 z5Sj?%@dEW+W&*Pk@`&n6Gkde#^n0dP7ITllmaHp&hAD5^7AGxq4(mq->Q}S`E)mL8 zq;M?~54oFc5KeyTE7kR0rR&vgRNm-c_~;DnCT3fE@8n^Y>MZ7VZMcQf*57mHPD}1V zCl_W4a`ddNxZwqH+XZt?86oWHq-fTcdHw~5O`ghIZEGp#1t$RjNdqNnBS zL%o2s(kr~>V;0mof)mWz+ARy}ayg7PaZ)fETFQ!{%cv+2W1kRX#n_FOk@?u%7Pu5&Sdv}Vkr<`sV+cX-z!-=^7{(o*iPA&wVe~N~kLrq2 z80H=J8TKb0j%O#fS7iPEP2^c^sFQfLOXuDPLA26*sJfA)rE1LaRe8a6?{(;5d&p_i zwl9pjLm&7$`};AUe}zK_-(~ShdnY;u${FrYCW?5ezy!jmW%m_`%}P#DjFSLW_<#g< z9DPgyBCCEukP$oW-Y;25>vnw_^$4@q21yqBldHD)3pW}X=s z17snyOjBuw@hHfGae8!?S@J(~)=5Kal+0Qr*;S6)e+kK~sYI0Y(rwEYlWKXBZe9@l>uMf#x8aW_V9s=EJKb^T;`ZX4j_f=} zTYLR9&5_54jTe;VERqBlN|Vig!TLNr#0#uCdQV(*;y(@_suXbRl(Aa5d{*i%ld>dO z^0v$+DKnm@^J>`0S};{At57VOp|a>zV3n(8R$EOS;RY6+m9Ni6pDQXo{J_mT_^}$a z#pRcZOZQim8Vkx@@-aDy3|PS-wye%c#4F^|iA^d6iOH(R5Alyxnz>)H-bTg6G^~%m zF-x`D7fRKnK${m{zx<~=Ms0S5dey`YkbDX^r5(YxcGavuec|PB&2V!sczxTyMFk?|vJEIkG(&MBTf-<&Y~{I*-bXgMRtR zto4gDQV?kmqm297qpXnE9Dv#?ZujeO=X!;exQV`E)7?efvuy9u9C3AyfQ|nF%E#g% zBycqJ!#?dtH3ZTM2RqBii^ejZDEj&rr^Fn4IDW$)BU2Y5004pie}VUpu-j69_r+F0 z`IfC;lVDZSoG+fj)U?L3j^A>A4-iCBWF0Hk$zW#4XeL(acXo}JUAqj~k|5ZH0#p$8 zqKYQe)KV>1#gsI}Dx!>lgcP|87PcMK__LN(XfdzPN1F4){NMDP=Gk$Y z=6T-seu`d@162RBf_+P<8Bm2zDS%|_f*;Zv$jd@HAPvjd7Cm+)Et3gyXf#+df`{Oc zNX9dX_vbz}D6HoGfdPzlFEy-&qc$x^&+M2DyA^gM!eP^&>Q3gEn|414*vL(APy#^? z(hpz^FPyyYxEA{ll>?wjptq1@B_Af1waP#`q7o4>P>?QQWQCisv{FHwTO?-6b8u2D zfM+14LX#+9^Z9oA)}wh$2r?j40E5UfvJQxacWpZjI?f}u@oVlS#E^=d__8qxW3DLj zOl@(9ij2yB_HU7dHX?YO7!VX+Den!y*W66(T??gy^gEnnBH(Pl;b9GXkE-DKovV9! zo{h8HEnMZ0czxk0C~}BWU<3`U7>a<}bHX17Xgke=N&seQGr68?qd$7Czda(5H!Xro zfEKV};NV2tO{gv~6JR@~VT0Qf+3Nx>MF8K+X>xUf%47_|qIk*a@Zx`Z^k2pdMG9OFuilU`Bmn!r;eOkb~|Watv>>{Ht{H4Od z9jwitz~Y#MIr*4a;w8AmXL zclM|F6|1;(Wug5u4T^jyrI9KUE5P6YTvuJEL973_F!9SQr2)_mqJw0=g|y^qG!LRj z>LcUY#GxrEV}iGIfV80W6O%$3b}tQKH_aZGo8%w{#Lmz&u@|14)C*uX>YWh8%&r)r z5A`0Io9dtwqEAGQgc~lM_-cqMf*DC?n1tpR#(P(pXAm7~ zEvz#{tLg6(qU7u3TeD`PPibZ7@x5BU{G!6|e)8sn8X#^aR(zES(*UiM-9g!@7dBQE zb+Wyis875P6+U1Q8bDu48yA@>NBPKMQQ;4z{+`!WGbSliBsUv$t+1JZpMYZR^-<0WD9j0#j?`G`N}W);)4KWgsdTI9_jaz0PoWiDVInbYaqPrc&> zvsG^k%$cW7sFLzq7B5;}tS+}KY+NJFvbSW#Md8SgHs4$JkhIV9Km69~SV^6nyRxx- zz`!e-!m;UCcg560$h5rbFs2)!0Ry!Cv6TWnv+_a3{dAyp>;rgdDJ~nVM`Y z!RrE+&kCMl>qeqnHjO8xSp`pamUhPJy*(4jOYIk#&G&>QTNJXn!j z4oC&;{-H8a@xO5C8L%NXEesP$?Ry7f5lE}|23x89ry#5u?@&o=+n1|)=p^;m z&}s36*4R-KYLcV!ys)KaPGxOMOo>vigvA({vldSWMG}P+B(9t1obq-#Ar} zTaHD$(&ox+mT2|VzHWzD%<5kUx-4_)$P_`kHHO*|uA-yS7F!BTustGPKFhyoS{*;r>l*e+zZVSJ;P8-E}K%#0aQ;6 zqmv|I1F3Z7AempJs?00`ncYtd8V(wrqE_wMk7foKBco;YSIL#asw}&W3CuyO5Au?& zHv@}feIc5HfhhvZ@)5B@sx4h$7&==v$`}`vd^5V%h{FK6uoh5rKiRJIkFIDK=jocT zD>*gg7u`R%@7Jj{vCDrFH^843{=YNQ{Wq19#6Rk#f8=Y?F+7kzP)v{+3n?TjE>WEZ zb>%#G1U@+`WWbiw12oqCh-t!rh8qAJ2`PE!?B0ysNG^1w#%?vvQ%-z0D53;(YKHyg4&G7b(?!b8^4 zb^}>Xch@^WE*zRmqmddD;wsG;E)GQ@$(Z;&k%VfzE{>;xWW8L3cl-w37$6MT(UcIT zBj^5V(Ozu~5qtSwYRCaJ%~Dys)k^vFuNr#jsKm{34zkW(N{5C^Ez>kGZFZCi1XG$? zYpU8HyO;<+sex9l2T8&n=bTN8r7{lNm0v=Jhom_N#Zy@k9Jp@DL{m0fo6kkAD|eyP zIdU8MlfBi9gKsn=kf7+o=iiPmESELukHmrLt`mu-*2DF{dkolb6RkppotJlO8hbc$ zO8cTBzh0AIVTg|E7h7q^zOXuhLR+yVWxa+*6rM`#8D#8mQXcGipOWOHJ93N z^w=U0i(TTG)DQ33TB(&bs~tAzRIyi(+^-{dl!Z1*7m6gVX|}bb`ok`ilToUuT<=-E z!~4qQJ}}4lf;rz1s}ff| z2WvWq+W(Amo8%c`(=HnVDG}(7RKc4B;jFHWy5ZzSp_BX>f2hqRb47sf(# zY1|4eR%Aay(L{?JasB7x=Su}0aP|i|Z1`#1{5$mU--Mx(je)hbgst;G2cgza5W;`S z+EI}3=lSizw2^n^0s`78|C0O(LRk!KZv#P((;$h3^unkwWOtWAG+bm5?5sK0A7|j~ z#wO@Tz))`E8J~=Xj+2d>!{?kEKES9w3Ggd)b3rW7AuIBridNKq`h1x%0$f2LqW&&g zRklINIC@H6oCJHOAXI2^bT|W`1Vmej)~Q)-^@Rx0HvQ^S_mDG@Ds}nh)}zz(@(fouhE3I;cW9Yac85b!P=%q<2jD|bRdbX#Vm^u>ql!D?6M3O>a9t+Ayo{oGQ*g5NbA#( z0h*98CDk;t+u)*P$wi;2^~8rwi6w}P@Zo7m4f-;Sr2@}_I+dS6>6E!(TMPte$yTBR z+G~l___#rDhy_Mma4*qe>SN&_$s?GMJ#*k_=0vM|>IRcrGo^`{dFX;~Di*|Y zTRs}BkP3)+9eK8_rSTy`e4*!LXZ3>2JgW89ZvbB%RTwR=S~oA)>dcLE^YPq@vDvJH zX4|_>J{V)1C5nyHk{)p`bk5yiG4g~jaa;l={7oWc;qE{CUF>oB%-!9?mi4xEB(d@Znaqm~52~|rH;60E_ZZ*G zX~Gg1>wOgdCTtkMtAzzgzSLv=;v_WZ;&<0U&9TGre)V2L2QYkxAHWW}0Y4!BD}{l& zjS4|~h{c_xh?b} zMr^fO_fT_et_jh#jeeq?qA|S!tGxnBObeAZB_u$lA(GG)nQtmrne}(80uprAt4;R+ zOJIgc)$V#b#gxv#ro-Vy^4a7C><+VVxPKK>PdOfAL!owqdlTj58FErta&U&#K1+sl7HXY+v-wLESbUiZ3IufNsc=ew79}Zt z*-sA)(zc>Cu!FL=&=gYx<(Zf*P$J|mfhFujMB%Uea`I%p$wvNZb!TZrFi}^8Zl;Mi zb6+|a=4614R#5CyN;f7sCvR_LcHwS!gvKPpV8$Qb?aksz=rdn8Ym2!YnUBL?7GAj@{B$7{vL^u%U6lb46mZzV2i>9PqPag$OyspMFix zd1n_wzk-B4mB97QrvW$C#;JuBiJgOE1MY5{VM3(U30s=$h6-+n2Ia+H zH%H|A7ac>qu3x8=KlHH8|3(|?zuofE7Eb?sGpKASVW}X0>-<^^m)$q?4D_3>-qRR!1aT&A8><6(R6@JA;CyKjE3KaE`YPMK!*?-DBX;b zxc?1)fLJ5WNHA82+=?lRsT40Q#Yi;9g}I$H%!pYXJQppcc-zG~X($`-4zp79^^VC4 zegImJiOTFXI;*PvqubJEPIC)1@M1kpTg)1KfWeVH-XTgk!$VscU=Xgn#w0z!6qJ#U z^wX5k=(s!-_R#Z9Bv97gIyBX2g`Qi3=^pd0S}Z)ceQyZ4h!eM3CRg92;tC+xmNu=N zW^PjvH{ltng)BHIT+!8T!U-hYJI;6RgXf?eopn#{sFv<+-lP~fvML^Bs?C7zV;ErU zbO^D&e4Xb$XYC%%ExzIgLFqI~&9+CU`Fr?+^R7RXO&+K1>>X*W6G$us#y)KSA zniM^kr0y;?0ubWV9n!#|J3ztt3{wb>=R~a&2C<$;EW3}3G1**nfNGIw*Bcp8pR1!B zBirR%j$MddA+XmP!J*KIHO5I?2p+%)^W;#t_e>mD=h$!%p4MvBLtJ?G!f_x*_kfk; zY1>)07^%a-0cB&opq z7F%h~q_!9mTYC=^r`pna{6+@7RVQ6s1{Zwt5L&q;wX4PT?K_eoP_UM4Q(ZHEk?Cfj zN#j`3%^{niu|dgg5Clz0_-ri}@_KVZxUj2~8dsq0SUbJt?lAKJB|KN2+~eM!9&KGZ zwr`u3VOy-f79;MILTJqgFO~6FEDzwmhZ!@F-h<&7(v|7YqW6-q0Q8dJ5&~PeXp>94P)> z!%xLh??(xilCwLWVJDYPy<+JLzf<&uY)_@X2V4x?D*om~)bq{RU3Q)9nDUf33ceZP zHzFm9^OSGoDR3i?=<_|T?u*1)a(BgP8r}zB7qk|;L?jx9kLXU=dIf0(dmBllQrTy@ zbwUc=EtZd1%FP}3C&r4EZ-BfgJigWVicg(TYgpd!oy9kNnDf3eN(mmn5|wk2p9{LAI}ha(f^s-~rOvzbOe~ps51IOqmV}8<%lY^7ry1(8O#5$6zHOgB0ES#iBcVQkYI?Ml<}{f_9gx@oENpT&C&6Qrd0(~!?*^GMkXbdO%MaR!n$)-2lwLXnx(5+q9I|-;d9CkuG9P6l=^hVkBm( zT(=7rOih)+iPaIH6+1h25Ul;qOpL%qSHfQDe%Gm+3UjZsxAGs1xos#fYQ3FE;=Ng6Fi1XXfAND%3M!TBY#z`O}`DT}IhtVn$ytaYG9 z_DMQ9OnkOZLZesN4!C6gTu||zD-h&)lBjw$GJa<&6zDh!7A*}4q?Hi76e9-Qq7wM` znPs5(BxRl=t@|jg$g{iyFk=*b5R0qIt%S%ln zV)m8SIQ*L6Oho(I!b%63&nPOAbH=ycDfx#!DJmZ4b7*>2lHJ|I9m9( z&X6JC;skdBo7G;&O?%aPX5^b?c z$be(Rx&!$J`0EoOAmC*hi;$4;lu8fc9}C^6IVBy1gy4`@H@-e&PO>u`9B1!2bb7$( z!>{bk`c}~1tEdIbzn5_k6{)!}KBEAK@r}ajBI`2hLhDlM{^Zd=F@<1&p@U(B5=V)% zQ_`n1*0lNaB(#_+{l2oa+%5yFwkaxyVN+=Wnz$>`*^!t6sXhU%UJyUZ>fOgOwW+q{ z@(|6v%BO|rvGA^O--eY-`jU3eE zJ>3*UzNqteBpMTH41`-cM}x1?8_LPQg796>Awp!p#b=ybl%0=}Cz+p?FDO(jWtcZF zb(j~HazXQyYMb0v>gwF7&j0J8@mN(?>g!51o^ly%OMYH z4I(oWo{!+_m%h-nH}o*(~>Qu_p#tIlw2}N`E*2SPBFi z(F#F9$BI9NmyX8CNZ5sgXwZwS9}XY)*APAQ^&a7igIHg9(2#d$5>1c55JGnhuVz1| z=J=0RryGp!#n6Sla^XmoZYQoCyaU$uC`V5?-}dEz)~zbucF2avrhf~RTd1F_M;(G| z#NZ7r;0Pq_&OF=L6u6I{`pE#5;q{0X#f}xd6}CdohvC(06b4 z<5^skTZZbh5fE*zVnPH)#NflM4YbSIS0XQSAv$zQw!eib6UJ7pU|eaJOm0(#vP=^z zQxK5T-rVZM?)5AfGN>G5Q3i7dke7poX17E*MQ8_#6mBeydy1i{Qeofc(cx=zK1^9U z;<%=zWRvoK+(_hw22q(X=SDu*k--`$coS9#H=>FRRZD7oQ=a@Lqq0qD|M>Rs8hdj2 z(I~oF>)GIKTw#2jvQB~DJ8>XeHLOK_5{vjiS@5EPQrlq0cc=%C%k`BW&H%0vKLa8W zMv;xK2haP*?HI?L$|$;|=y6;tjXbT;60xf(iV?Awu!JeimO0Wbg(NS>xRx=MyPtto z?jUFLz==er!MC#_iVe5j{oi6km-3MK#!Qj=DMZb2Pp)Ee;BkPW@?PESOv_N@NJhMY zasC61`~BBn?Sa(!8h3`6x=7?`&db7+WtVX<(T#`gRG4yO{%NUlf!0(r#j=l#n=Jqe zJ2Ymf>^qrb(SzS7{b6kuyvm%9%B)o+hmw>u&iOP-UPtl_pjk-R5@w3Ol}A5p4cu(= zxY_v-N^)+T;(Au|PVCsJ!%)mntmS18aPsF&dNPC00=6*zT5v2#K#VCaK$Zv=9nIfg z%Cky`^|{Efs>I0)#>!M20=*2C!$=N$$Cll4j{_zH6Ypw$Ip8s*j8BM=Dz5g zSM={5z#zWeIRciDc){kN-GV^q4W2Cq?66<%p;{lYDc;U@9Rp-;;ERuH&F z<{L6X%krApBGHAK z-e;ktlV((fa8~*|1TtWfB%JqV;>zn1>s|(RIc_N-%TuWlQhB#;;H|&BZl>q_mewTq zncikXRo-UEQ%66r-d5c95!Vc|ntwqdLb?Dx;7D2V2(9>*r%9^`Dq#LK4=wu2TeKRn zF|Wr{6B)YRBfJx1?@PSAG*D}h-GVldUziuWlgTaQ)8-9G+t2Z6fD#9Z@{4T-B?*|$ z_ZPHFDq3~qqz{)&YBBfVfR)Y=J0%ay>k5qq3yZ#5V?~J*rKMJ3mkx%3I`)OcqgcDE z@W9rdI-&G5c@KaYKPj)RjI0v8F1$H)3%HO|Qvg?a_SrQE+pq&PKF!~N0oKz*D9J`uO%?3Sxd`D) z@UK$PUVF%kLDITCX9kRLK7m-izc@2;gRqdlypa2H`I?yXuPtZdOggD7dC%HIo~@5!k?y}ggyCM1ha~`N&>9sFs?0kaAfGQtDMcJUl}-Wq^l@& z($1$o*pwkb&fnZKlXUtM5u^^e`Ki%WRX%XcG10$O77ppUJ|LS0sC9=6V=;9|(F|Ph z74x7=i5H>C&Z8=bWDAudhW^yg!2Vuvl#J77AzRH;IJ-^-O#HCNefOSom`m>^Sl$A94qfptEfr6V*JZCHGSx6}y|7k4kEZx;uLb%FS` z=(sG?cZ}NOnPvvv&wehu>bJ6BkMyD{Y)#R!R=VL->Y5LWIMw0+>cTPhwmEgjD<{=i zvwRct3^Bo-aO>zGCva(%?N_6M9*E2hYL}pw+Dq8Y?f;;8A#1?Uwb^Mm~8WH%gyA-D*Ypr$XB22t9>?Dt zV!Guyul^sd)kvN_#Q?G{|MM8K>asp|$`)`R4)MTmXm!_p&{gEpA(SBoy3hf8x@-a}ZKWVU%7zn^u6o}mBh2~r z*_aEDWUOsETt~Hz02$e+J>7gCijsMCUPvd&ux5s0Zt#NIXU`dn>|su3Ep3c z&QhOOF2fP-WD>94f=Bw>tZ@CB2aq7~V_kU6I25?J@ z=KH~5ynlK^|3g3Mzj81U8+&K>|KgY{*||6x{jVA3B&8o}IX^s42D?MX&v6Y7BQHKd zgBGDh;6|}e0;8DUm?TaP$(=j2hN%{_QooY-Nv0dysw{pB=!I;E9WN-Eq&U%Wki}v4 z1C?!H^w}hL(!&W}*G!!6H@6s%0XZ@mp%fRirpj*+p>anOaaN6}ET%ve7Dz#f z4HL`v05$e|8xhY)kGKenf-4*YJ-2M|n(F&E@wLC0Zp@e2Ck<<14! zE5}bB1$Vw5Fpk~etv4h6dhro0+?$F1ZlpWGz*~nCPsVzUN6D*dhphVgF$T7Op4-kG zXINWyE+M4OT8(yTBS0Bb=M)&_MMXw#TdIu>73Lw%mvv1O|sywPrDgy6z4Bgnn zPbm;C9V5SreKK1d74rt?`ZAjEWBU{cN*l@c%Bp9?z=rANKJS_DD^ukklzY3}3cwOP z_f;NV9AMUspZ>H^@MH+wII$sabTrD;DOmovYJ;iGXevuecrWqZ`nnoqE$7K=#a$>S zpk5T=P|sb7`x&H*5?14E1?~yHct~ZTZ2_@vz+=DMuB}-ZW zWzCWs>AyGT3)JO#{mMz?(8evEgd;Kjj<*Lg@CUzS2J-nAq=a1eIH2)|yrce8dH)Xx z`af+uM{ z&cA!m06zng45w1O8OLAL_O6?Yc@gw$84;&sT~QmoBAUOzn<00TB5Mtv?$BA}H;$1;cn{udGwcyGV@58FCkksP6A17aD&LJs74l z$N?^nEIAz7<@G}KKben?7hzLsyiywrA+17O8e3285hg)ET2A8-*dHa?m`4`Qn|v03 z99`W-o5kI0G9NtrGrMzUrRTq_<7w`-vC!tW?nG>i8r!&JI|Y+!f_*LFeSo=_-I~S} z?6ucB{9jaoR2DU0CEcU-=;Vq-DIRWi9g>D19`(-=l)JYJepy{GDZjv32EMlz1%#`8 zp3|KZG|N+zT~VmK`|lpWP_8_$+T8ewar<`4r*KJ!w;}(lmVWvitoDBlVn_a*QvYtS z{@+xIB94xBj{n#Q|JVgdiaJt2KgM4T71g%M>m>?w0&o`*FmINRioD>0vG|Y38=N#q z?H!jf^D~s>K>_tkAcSkyt4x4-~NrjG;-xRYL)KyQm zF}S>TUvO{A#ou%rK37!E7z(EYyYB>BcLWwM0Ko>RBHShJ`Ksm2uD(BM4E>ia?`s;ZB8`hwh-(lWdsyx^QA}Yy< zi3}=d(7_4|5m1k#&IR{DAsYq}35cXOW@d3q47~m&M5S9Hf~J3fOxd3)Y|;N`EaAUG z^uMl+{~e{o|9i#G^&kILlUled(h0(M6^(n*U!Y<9-Z~0dDNK#}I09JaW(S6P3lVRF ziQ-sL*ErTEM2n)!YUoEvr@ylEtY-oq>=1zKqBhY`j6uolJPVKA?{JTL=I?L17t@#3 zC_6L!`_CU!UJu?*Gi(hWpT2Z@V6{lT&qi?93HER?46KtAU?k-1w1fnC&|{9Jt$KrD zzxYH&%y~0o8BE|g`l1>lhj=JJ4J*+13CB$$1(Vj42yvW4~-$Tq)$J#iK zHGvW!N^F=(^s{;C$QUlvSWUH(xFa0aMC)X0W16G0y{I_l=$A8Uj;)!7GCy-vJ^vFAsoFt4|Xc;_&S_XBt5^-$Gq`P*6B}Xb#afzd_ zmIG2CdNpUYVN`0DY{^m$sfp9X@}|Neg)>SH`$)-%rxi=n97A%8B}d6HONoUf<&!`T z8)=EM_9M`PQa^tC7{BcUvc(`1F-1j0h$ZG0Hda>1upSJNrdF#W)QW5&hjBTMY(+IM zlZu7yWx|84Rk8x$oy55SRg8B86b9M@4sJUA&wN#a1CUIVyZk68cDborQ6{1TplQg` zv;+8_xl%*TAwqa~+{Lv4O)8~}WLuiQ?oMJfQsqtLu1ywG(p0L3&R|&IG#ERko?3QC zWuvEpEm1eS?ih3gbO#+T*-^NU5ii|Q5ZZ}ze*iHOMk8PRqolCQCKw@8;~hZMOgho% zCeR0C!>l@A^Zc@)U=$;hWD#VHp#$e9n$(?jhyU1%4;ysq4Z2{|=xYvFuu~n3M12OO z`SkwyA~So=Eu6c(JZo3EKyZI78-X_r+j(0pUN*ECYUNRq=b zG>Tkhi54FmEUKmArZxir>Fe!U&;9pKJJ3`uv=jJBWC1E3Sw{x>3 ziN9C$?5MV`Al#a}0UCIB*Oo3}^aA9MXnMJ#>wyLFu3k{v4B zLhgx@?va-}5zF(DOx3xfU6e{!uc%;Nh80k6kMJJA(M+?+r#3{BZ>^o8-|0XG#BNmx zw0qwk;Caw?34nv&+O+I?AE>rC$-YeHnrlP53xuU+;KAmiH(3YG;jPU ztS6B~+5>tqu)%{%YQr<&b z>ykjMX?AU1P2oCpk0^4JyGry38IKo@dZez|*R@K022sVXYei#2SdLX{GuVqPaGrOxEfoSO^ zT3K?KtjhheYJC_*4$#Rws3=96)$v6k05r2_BQWqs+9G0>N1V=MoU8?av>MohitSJ0 zLWfclweFv^LOvXZU~76**;l>b4Y-s;I?=Zq@*O>O3A(1%M+b-Z$lj3ZLHn)j8GJ`OOX5??;U4}L!ObA_Da?V{Oen!Hfy&Bq8TyMpdANl zeW7RaWN?RBU}CHDQdg3~`=!Ue+q2gk?zSeu~1Bivs29k^fI?Juq_z5!s{qYdFq=zbLN{uxTcDu&Jpfdv3i z`w?y7{QsCS{3A=)(tvQ&UUK>RW9mLXnLtffpSBudBR%P6tid`w92YHM&S3Nhl{r56 za(9oxBY9+!a678=|Dx<0y98m9F3Yx!F56a@ZQHhO+qP}1%jmLgcG=d}oSE5op4mM+ zvmf#gWJF|S+_-VWuMvTOFrPwPVr`AsS};<$FF#KMo0h?Po<{nq$aBtVd!1RjgKdiA ziYMA65d%_!x8iEP`)Sj2`f=NFn&)W>z2|Q181OugjCmBU_c-*6VfUXI!thgH)Kr8bcLwxS7J0 z!EVt~ZBv509-%N*4|ALqse9Q_)lYkAqr4*6C~IXi46QA?Se(sF^ni)@;P zo1HXdktvbm-7MVH*dQMA%_wJ`B-j}1%a)Sxa52R((@XS5akeDOq=k@JOYJNc!cK!U zZ^c69>Payd69?*xwooKZoc7z$7U!k=rnm)6RFcDRVu##;&g8H;%d_V)M40N`6vAP$R^I7Lh^g>rxI=2&0rH4e3#Z#R17$%HVsF4kUO&)AnUedI> z!^JWHPgz4ix^mctm-~nIwn||$fl|nRl(G>zq?Cy{XoRjzgHCL z2MQ2{bPC#$5sF$c7b&o1z#2g!x+!U~bHVj|xHscQUSeW9U|+6Unw1}zNyNN_Bn`c1 zqGmJePev+xor<~6u%Uy`xm1U;L^V9QY~j+dH6VH%N~oXk=~w*_jw+xa+J5TCq$wm* z%kP#{Pt;ovFoI@x&sU{eI-m)KoLX8y7&tWNLtYeRtVijjci-k^rE{pH1i&hQ53ATZ*skA^qdGqQ#HfAKB6ItdFzmFj>DcF zo($%?lr5G)H;nlCoD`WZqY>nO>}gX@{l>J!oek|)07-s!A;VDl3Ad=@YhcNCE~^11 z&)HDq({7+B$dYlF(ucVC<^y5mss#4?s*WY11@TanxxGr?X<4SGe^$Cza6-#aIirU_ z?ACNQ<9W0=2P|oY~hSavJG($ z6z+)fV2}aEH+^X+kP;6_cBMeU4@-&I=v=W$Ss028)k4V}n}woDyBOCgQGiRi12b3e z>`2I{Q7qYI)+X3S-aNMMgMzzerc>FXq#No&@kC{u2kP^2bP9R&sm13b<0e(}s?IT);R_U~uUKaiLE;B%!mlQ?^`ep=YG-l?r3l!S)K!wBwPx*w$DOO-a zDPraVXuP|z?HHUTnVGwM#+y>uT#Q89j#p?z8LMT6J6}QSRb~#be~M&7O1T4Hwi2Z@ zNZ{Q*O@T6c2AC2vNhRzsc8(%hW*#$n3hD>^9tSG=HS=zVLSLR}Wc^i6VW?8qsMjj6 z+bi@lMt=RmJr2S;ERpenpR3HlWx-R$J+DVk-Me4JG{ku7b&~*m*-`4_kH3DLcL42R@RQ!p;oQ`2ukiaG1g`+(Xi92WtKUqQ9GS4yM2O;+SA z6NMj*kyBHsvaqgEq=i5>o{WZO;_fl49I|4q9Qd%PBG^o8taN?ToK0q=##x7X7%WaL z48BRIUf0!Pzm6m+XlbJco=z3yvLdZ$qT?@$two|n`f-e&52Q(|wIXG=qMFxWL1UDV z3?soKJL)IOQT-I=O`n3EDj%OzZlnKTewUIacLDFfB~wzqc|$w{U*ZmhlO9j|99Th`s!U0K^*t(Qn$)Y{x)$+@Qa z`s%q(BplU}+->)mxDQ-Bp@kAVwBBPWPRHPdpEl1tIzNH#eVBxXzt|ajCox7#o6)yt zal~1D(KhEx?^@zyQfe%St}7pM2vRj}YKc$jMKM!QW%L?yvEGXOiq-{^^wgmB!$qYY zETvZ) z7Hzo0;nKDXqxz#8XK>;J{qpK$*NwE(wy_7~vKT>mZH-IgRnR*~6b+8ab~dI8(goKB z5_J7X^=5v}rwo|tdtiP<@sMs+(nyDDM38!G@_rS&TWSzps+Dc{`0q2MbXhR{vk#&- z`o7c}2h_AH^8pF4qoSx3_uSmrZ3W$&$8N-O9Lff44IgPQWCTdECO7UhW+J_lHjUr) zuXRGYq+-WN8)^pFNpNB;AG?oRVlk(i#q(<~-cQcRWTC9g#H>kje)4=rd>RRX8&KOx zWVan0cu{p-=ovkB_V)%1NV3qu5c0-R@ccAHFMFq5rCoWObkp`j-8ib>I_mCS@!Y?w zHkLr=Cb&{}QdS^flL9THhfb#@&EGIZa61iP^R%7nmejIcC-Y(C&Lj3dv2w9aO z8faF;i)`30eg0yevhz&l21$PqJnDkwKE~OC2HzUdqGalFT zlqm(NpLqx_)gNlzUpp{V0-y^amH*=>vK>ekvYo^@YFV0Jr#={z;4>v9I43%Kj!KJ2w=GWuB)%D90FCa4PyRERzwkkXj+8SeC=~+0zfyOJJnC*aOa!W9(ZD zG0ll;WI+$A?O)k+jm}`a{-t$ThHij#4%${<;4f!5c=Z>$19k0fj~cxUR%em}k7eSI zY0{54$P`+9&Gc56Kr6S1mMFa-tixl&Gqm9_h`zQyIKEbMNr> zZv)&a`fK$u7lbvI((G*XS^F#O;b*mIdOvUHMv7;w3^nrGbq+XI1X?26s(B+16(;qN z`Qzkhh{f?Btlc%O)?N)X77M!l<=Oy($WrpIpLPvdCnLLtm9=yc@>9ravxhs~lv$+= zndl}gO4XTyj-a%USh`@IU~H;-Ew5z_$2B@}%~ZtRt9R8GUN+{|EPP}N>|qvNriojk z^UK@k7VNkz|9GHsf#DvlFxwc^vrbNlZ{Xw%)>OtwR@q{M$FOHWRXzJ~n``bi=2uSn z$3Q|)ezMyFMCw&b*HMS+p}P3igdHS41FlyuuM<-_inXb~VLf&!IjC|EUC8c&Ti-gf zx?A(UM9eku4PM|6yRTQKIMOr$b?=j`+tcotucwJ*J`8&;?cLR2c4b~ib>l`LzA!W8 zF9*gQ$j_xXH(nTiYN-cOPq<&jhc^g)in%l@Wp*)OM?vy&e$B-0Ha^ZkEfob=y5{){ zj-Nj>sS8?I)loLAeLVvi^40gQG`5*(dP9dY$FCBoa9#8crNkDdZ+b_cOnRoo-A3Hv_c1C?%BrTgohHT(ObaUTM-abBQ z+?Y86@>ym&V@K*oWDeVRrjhJ7a{V$!a?KL$#?*4;(j=Q5#@@6q&+F+{m!V!PwNksd zw}&{9tLsegw3s5|jsfO9`~k<|qACv2@C#22en;<8nA*uXQCf##wRRq0&&Wvn- z4@i67V`2~RVRo%fC{!%bZDH!W_NvAPUl^O*GfFZNCUz}|;k;sKDb|?*aF6@;LTG zaL&vN(#(qtt(L6&u!ebdg|<@4gD2EXCqMML-w%Y?B?$9wrglj)?ap9Vh$Jp{`*;RX zG0x!@(M(}+?i$Q#wA?eJ58bFZvDz_`hReSWsNviNHy-USjyKq$9yQ?b@z_{`MM#KMJ+ZL;6I<6}rd>T<9x*U*F1j;0 zbtUO}J-$K*sOgO|MA99k_oo}Omfx@$zDBhfx@P$mK7ZkY;~E!}_=BiP{U#}Lm-ZKG zoEOfMo$k*R6ciL{Dsl-7%O%!yQ>)}^-82viI{gOq^|{T4QpebIj(nn*5RhhdrnM}l zbt;Z^UMtPx7UShq>SaULeI#0HOc}**Qo4FbP5Xt)_-;;$E?1al61~TK1I<-RkeP#z zazK`g)Ggk;CXGqvuNSw+`aX@E!>^#<{3%6*0(L=Z9T{pHB4*7T1U?A<=p1+PnO``X*I=&3Z^cb z4e3jdRUwD!6N|5=Ual3Mej*&{=yNNqt=uX3cSttGcVvy!JC<4~$#h%m2!kY2WVkik zZ6_#jNLgiX>LAu7d$uK`W*%3%%P?VGvYRdH?SEyB$E4}cPyy)Ot zC@W`Tl+W@*vd`+mu$54FWivN&0I-m%vA_E)TCg-qvgjhgRSax8cazgam3@j_ovQQK zovWX(5t+oCUf8)w%{2Ds$$zotHWN4!*5wQ&_lOzkp#o_%Bz2(E0fg}wd zCRe4d-lR3Q7&Q?u+zfVt$UM{}gBUWLIde64N&TCIQ2GOs z0(DbZXrq9_9Q~FlEH~&00{3Q?36QfsFdjGRl7Dpoo6dXX&FOd1yI)Ns&mH78fc9I< z)PvO|Sj(`W&J%(stI@^4vNK<3YyfZovb`Sh1I|sUVDDWU!XPHtm@i1Zju2yAA>nD| zu^8?PDl&yT-@$ggST&!hlFv-Euhq)<`>a(_j0(-p{!izOfUp_P3=Ig>LnP4U#at+Z zSfO074%wJl^N~Na^Y}S7{l^sgyN)K{r~S5Rw~(?3yU@P73wXo2aH%Jhys&Ujz&V@3 zUKsAMxxrAJulS(J#y9r)rF91#xx(9JQP@iNyBc8SySjgUa30pe*tx!2qetI|=l|=$ zv2_)+b+&Q(*B+_29FHszKZ3U>4o(($0Df6=y&zIUJw59%7)2g~|6u=c)kuVB%=-96 zz$ZD46f-H>CxCB~eF}g)G+Vki*WFgvx&WLeUJK6x&pG{b;+fQD z>moA)Ox1`SdvwBR7DHR~)-DOZCM<~{Z&Whr1dEluHLokb~^HR=GmQN zG^u8y%NMMdR>?%*J!Gb_byXPoUmoritnU-Va7vD~zv&%Yqc;AR7 z7lcFyiw4g?;Xc{d^t1idDyQg;>*3!PS^RET|Ceo1C4B=c_ZXfhQx@fm3JRzI{c^(8@<}PH! z=Pk_zSJycU58Cijku0nYc_xi6lHVKLKL%|n_{XS{nAb2J}V!>CImYg zWa*=Zaia!2vuX2!7(?!xVMLl(=qs$%Z6!?|&6Os6|81TQ#nFUI`+G111qJ{>=)ViW z|A(>vng3T+kySCiq><8yVZz!LTUyG@rGYpVB-zNJ#Onef)sXYzTi8jL>0KBTrvmg` zEuVVwYQ4+gAB1BHmgd)vKjnEoE{TJJgcb}POpa)`U*A$Zzdo*}djMf}Q|X1eum$*r zpU;ZI6!k$WsP=!5_KOkxCMzc(?X<(X0?h|uAlZ{bIHC85gPgx?3QMP-+WSC&+ttEQ zKVZM93Tvl7y5Wa#&w|$n#1A=nKqPx+4-(db5F8Sg(Lj(#=@ZgI0su@fz?hKuQ(?`N zS#l^mnPViQFU}+Z6m$pVF%}8B+>o|QE)rioQ?NJ7aVngBs6v%h&E~DHtzx`9>%>vL zIu+(8r$U8Uv9BTZ(`fwB%(QT|Guc&Un3A zNoqVkxv5pUD7!EtpTiL7wQyZw_y`09$s|6C{TI)n3Ou{%ZMoW-$Iy@yiqkwP^t(Lo z*{uoWa)Y!3bCw20T|{9zSl^1WflLU8g}k#8piSdrLG^F1KunTJy&R3n*)$qkJ0t03 z%DSNPPGh3d6p@mWoG3o=$h#Dca=R|_9c{^qYz7*asV)Ogk$DZ*RV@<^$Bb(BCTjlS zx}a~(-5{bcsFmjM!8YUaN*p*yhCvDzR{r#Mw{%n-L=s5J8Dg9vhOr_m-b_j3SI2$i zlccb5Bwl}+GSn%m#S~)=XNDgzkffAIyul;^#2j`c)cX>Nj*DTDYVmaul|OhBuQ`4p+!P1ZUN;BX z+DZ4fwvq021YKQshv5^yhvgE#!>IPIu;$N=p=!&RByWVUF@^kE!*K>2Scjy?G0|3l zMzJ!c+AcR$#KA1UvI-?z96c9_kPnd4m+cVo2hL!`kz{GRL|$KB7VxmVbUR#_*K$cJ zu&5{3lWl_KD8Ae(o6|_kbm&@a`rVl==6=wCX%W>~pjhv8$I6mDAAgP-tzjrqEi`&5 zHhBq=kG3jAqflElN<^vXXY23r=TD;&o4Tcj@T8n`t&n(DZU(q0e|Ym=owo;b6Ho+8 za66&ho78f7+f%i&;>Sp!FfM#NAL;adcH{+e{dk)K^d;_N4Es& zdF{aSpRJu9gRct5L)~quS)w{C_ce0>wPr1ebEr5aBi`?&c{SukrR90)N0T7~dkT*M zJ#KNmxxO(|edQMdd6i;STrPeMoe{gqZ1QS5SC>QvLcZVoBDIFcupo1-VI|O9GorpW z*x8{h>u@uvHbE<^qeIeFlpcvW0EqXqcW7HR4Apswbs$bt3+im$UwAIYcetLw?~)En zb)I^-o-rTw`VDfPuk1Sa&sT=E@Lzh{pHn_l4;zy#=eh%TXKwy++oB4~HwG@S;7d1i z18AOGePWY%?MH*VB4lD-J3y*V_;k?Eum*l{*V|TzOcDka5IOne_6%g1L(e|Kw%k|Y zg=L#RUAg7pcdf6?O9N_p?&0R25R5530>6O&nyZ~Iy*A>$b2Y>_%aG|mJl{6o^Uo&w zhQ^L`;)1qT&eq>uWGj6~$LNG~&K%i50KdBVexqtcuisC+4po+- zK+Cq^X8)v{DLR6P3{mvm;-A`{r#7rtClg^p> z-Gh{48jQPUjvrd#5XC{6`A9XLXwI`Oblx#R+e%m?HIQ#R0{X4iAE4HI($w-bX0b1W zn83c-YHm%MvBQ7ab?-hhV;Fw(wS5ydb4hOaa>*`o@LUiauiXwv~K^}z+LiY9E+J^1_dFuA!sTPv6Y5XgjpdlD=b z4i?}DJS|cx%}epwGO)eZ7VRveIAgpHsa@k(S}pQj?f9$?u^lq^2j4*IXm!Rqiz^V| zzq1EB3D@=Nz72f(pM{vr|HZ)nGr;+GBTp)=|INtR!G7({hYB_yK!p_=H#k8`Nn(Ee zEMee-<1ReXsbK3FxQe&@ULf4pfL%{SugGQldIBJb5d{_fbS+cq9Hz8O8=Ko3K)sY+ zP26B&I5jv_X?$#a_UOJB1ev3C974G)>U~#rCW(&T{)1CkxbCHnSoltQgjkO0#Z|8)OLzku@j|_=K8MQt&PJiZ_nV=e17?W zSKPf?pBv?^1t!t6yyQ0H{#rQKyd@B{P^OzT$&%ynGX_-*z75fT6YMitS+w>f1dgC$ z_oiKRGyxu$^*o3mYI+A4voZnk*w~^q2#OC-a7(~1UF6lK1X5kEW9Lrpe7cZ|Lh(9D zxlzv`9mL{t{izy`bh-fwnwz`sGIWW3%NQtgUj_s!7{|vi9z&S~aKHmp7yuTK5`-UU z|AG||&dRhJn2vOW+l5YSdbW>?61YfKVGyrewpt|P`EN#A4P}*j9O|Wm?~vd;711e?4b5~l3`+c zjkq(L11maSuOOzS81Y!YT4t_C^6SeBey;5mfF0Nejt2)0)u@#?AnG}M$P-jYrgauO z0@WX>CA70YqoFu52O4@=`pnSC(E-eD4N51d4o*vdgzY*w^60WevkR!SoiOg7MsP`5 z*h?6deQY9TonCvsp#9;kPO;pXy^>JqNGHN>QgjALTkCX$sj}u-Y6|s0kA2y>@Z)o% zQP0M=S5fX$O^lt)z4C2yQp^ei=5tjD7g_8V+Gs>%)52SYn^DPE0!uq^!r}0;o2{Xk{_E`(!YhEczVDCsl{Pw>9RxOg&FUAB@`6H zR);4Yx zc~-v>pd}*-GZ!AE-jG%}sJFlbU}p9$!B8L-qYQN!#2I(S^Lt}fIN4El5(cR%DCCd? zJPK+3U8RbwAJ{qNjcYa~xYAkCwAWVg$LaR^UO^&MJf~D=Vu|4$fdDxm*UWyVfZXVH z97>saw`QgD(lvgtR74o1|8oSUBvIeE2Y?|LvfDM0ali?$X;v#24p4sIN$ zA79#aOI)>tYrF3>w5O5`kNjtv_>O0*FUeN%oG#sliK$Hiu+o zcR;_K)ML}P9CpOVQ_2Br)adHx`|Cer*GlGl(g6VA;@g8*|NH3tH(>am-g6N=BS{cE zr*P?tBVkW<XXX?aoOxjXN+1N=;`*A+guR4rF;6n!*t zh%+7K`|){$4Dh`P0l*e&2T2X0+vShvx=!ty**$CJ1tp*YnOv24J}jz9<}gL(CzD4* zK;C*KXtqh`vbm21*e0oeu?CqHsQ#7Y%7bF-3pd0ox4fk|356-?S{Q3#UT@$-WRQNr6Lpkj#V z!iDAKBg62ZAM{a4$jF9W$!1kvC8LWn#T7!@2&v|TP$$i2wmJ>3!;df&mUJjX2ArRe z^=JvAC8YaKx@JJ*GVS%E@MYFlD#-;Sur<=4#x2i3(WI+Ucdd3lS+F zL%ovWS6ppUPp1`(w`;5GvI$*&w6xK|t7)>X#@7pK#2G|F*SOA}5s&&sNGLSHC?BCT zRjmLBGb|QM*fETvF!tH)86QW-Bqjn0Cz_VQhyp}=Z>Y7!JcTqDF3zn9p zW7;%%XA!(<9y^S4m4DYauM?LgIDCSpaHwonUKn4#Xb;&|RcEOiGNVpd`^G!xk}i`B zsK9K`9igcLdKU$&^*6{i}AC(-wU?wAo--a-HB!-_^U-o4+x`!`A5 z|Bno#@PEM?m1B5pfEW;hbM^IK#N}e*QOtz;AQ3>B#ZXXAqFfs=XdcHN1*TF0fbGh{ z2o(^d?uu+hw!6l!rZIa#3ULN-=%5C&V=R3$UWQ46AKykw>k{UXH7Q|^wnNU0x+6Js zP&rF(EV`gBH)=5+r$q4F>3;agY9Sl{~pmL$HRK0yYAY%`XY zU)#0#Wnm!l=o`abRQaal? zx&V6kEhN;E-}OGF9_z!OcT~>vrQgu(v?^R~v6ODenXc zDX--Fu*DU!@!ahZ(gC)5C}mEM@D9*4h763d_r=bo&rc5G@I`@Skebv=qCuB9-M^6O zb5TdRhrk~%#U22M|{#{z*iZuvV^h? z?-DdH3Su4MV@UK|g-{I@b%qgM%|JR*;s$cu{Z$l&Ci6SGb5nSfG+k(2dUlx{c~Ri+ zGEfNgqPE$;ZnR{6!jZ1z8rAl;1W}AOvP^zrOxXFL9bshucJ%T=$lZnOe(pClK}=zX zy85eCS=i)V1iz~WsBgY1$N$@^&Q9i5=1%6{mEqqD!+(CN{+{!*Rn8QVRZ+fZf~=5K z@c?Naf|r&;U^-)6ETq47 z`zozFvUa*(vd>POb3AQrclUTcWo@@-`hI?#*#dxHD?)h2C)6W-uMy$r_Fn{GeoS5I z>Ju?dG%*B;=TFg04IJ$|NI@0Cv)t(DKcdWZW?dF=L$bE3#l#OcAL;qGVHXd7Cuy+Wl{wr zeqD+V%R)nJB|}H5EV;-;rRbt7PM7gpIbJ1lQ~E?$QhQFgep#aXp^BZ3+BqFf93xB0 zWUihizOktYOOv%3BNp0Fm`RA**1W^SygzDEg&M0t{7&5lxjyJ4+Maj=Rd7fUiX~Nx zh7rdsicm*G0EH0hr_ORDNc}KGNfxM}3vmJ_PcAE|c}g=Se?}cRizBCtB`0l#6=NX~@vldu5I;vL*PZ9G=5%y2>- zLZ(jL_}M-Yj!K6LgmEkN>i3*RWe1)Oa_eZlbar`_^F9&t3T90jSSXJ0!fF{T?F3=z!&x^yY+XVNwi; zw8Qtz;`hwt7rvFe5Xlo$9iLE2z_djm_5}iS;m>pxhDubuB}zXsB_oHrKI^gYra%MZ zhdgWxC=wI8aK(xBdMB0wnI!QN0gCdm5p>ui4$J!5*~nGpRWf(h6<5&&mL>4 zS7{#B+O>Z_M-tyee9z}M3g;vtjWq8TUt#hqecFVHqmmKBuwAPp3#Su_q7)du6qqt} zaG|~Yq@Ci`gS$xKWGyBZb~-8xmzRm-ap$SB7O$c;#I`PIJ<;x7o|hT6EnBi z@O9Bj1;gp%-gA@CSujQ+q%QO``#d;^O#Dk)3-G?SBs{YB5pm>+llRoE0%Ki~eU3Li9=?;kCwao`-pITlnt7Oc(H%Sz9qc zkG1hSm=$)DHMs|NCY0Q$HEmw9mL}lEtR5il>=97E6{qbDA2ZDCeqnUx9}fRSBlkPF zTcta*@K66INbWN zubgfo(<%qV^8wL)A@5>ne&En0OHdA(3C|R^!N)NK;k(qAi^CNT5Tb2wysEU zqydIXyvUhTD>)1vYY8LD_F(|uGxV+J6ucQZL}Bt$D@}U@K6r9cgN6l6+54oC7VPbU2cK1Jq!fsfhO^TI72yWWDW!) z{kRK>6w>Ydw^$SpIgE!9yboAaq9)!Rpa1q1RoTD^Rs6O-^7od5|1cc>Ke_pDoAGRI zoXiyM^#1|fS}SThA{(RlY+ahBSewr;2_#DJCo%gESCF@$_@gn?th6Kl2 zd+q()sR)Gh%IIU~HMEZ&I))1uZb_lmYOO!-?Ni2PQ}f;3A+@P@&Qw&TuF_~T+^q?w z0~Ntq=SLZTtiktrYa;;-BGY4S!I$iz(PL@Bo+RHww4$t(aNKEk@X#MO3}f4Twiyz&$Hk{&Oa+&L_;6DY)VLb7 zpx)5wurv)sQAKC5*>JyVaM}tMql>+R3k$*hJEev=hvQAllfLv9=Fs%<-*xv5gQ8e@F1q6fWbQrJbcZZG#R-Yg?m()h|)0Ul57x=#SJ z_tRypGE=Aj!DPHNYt{!3WcaP)1zvDfG<`3JPB@gr{IxnI$^u9gEI8Y)ygPO4G&g5e z0%P+z$I}CtHsHQk;--vC;U$^IzH z5*vR&WfHv)=;3jnB&HcUYAxo#^DK_U9i=lb>05LW-+uRVZjxpS`K;_)B=0fLE#Du` zI=yGvfXO?D3a4RWi3W z{+}@QR?=3){ti>tw{(OxgpOiXap>TsIDmYW$`l3b{APLH1p?G}2+0QG$QQ;{dY55I zx>w+?d~}&%9lLqxtoof5Og@|Ln=!sU_`00Nb1;a%eWQiV>L`Pm-Nfk*Ze5`LARsCVX17@_{%sc1#d|TP(Z^^Yw9RE6e!7| zMa_B%)a=k|wSw7to&npWMk9aYCr76|n?s9oH3Wo0{&0Xgy_IF;c$LDdbfv*+D>;3R zi%(_YI8af#MGjC>fcZ3II0in+7#`=ACA#8?)3!dZw92bmpwI0@=VV>>O$UdBtZ=)3 z$T;E9ywv7AyU5(Nn8$Ak{NE{iGNqrxh$X{>i6*K{E}Io7AX(wDQ`T6TA{9nTQ02f^ zD8h2BDqA~3i|t&giH^kRAV?F^hyqr*L)a~P?#(`9}`EmUE0ryA}A&9ZB6 z%;g-uvQ9kJ^0p6Lozq5_q?j!}{}J-X(hd-aRv`C#H9Emv5Nh1&@`d+~RUqHYM3?Di^cu>G3zfKN$HjVb=mbXdh0P{VZ9DIabEtOp( zl|xIFT}PE-7yMeb7h_nvg?FyW_ZzD8lGS!ApsI4StDvIxxHeK`wsinL|zC zdB)PY0II}$$ExYPrjMQ#^&mjpJxU8WYiqo9Pju_fMkuk9k;72sevhUFbytdy2`EUt zCbe;*TEY8R@I^TwB@%^_(CMH265k>HVH0ppGIQJ~{fzd5crG@Fs1E6JRn#sfv^n}G z<3?wwB7B0@AF?b`E}UDOg0nw&I0~hLF1Y$G*CsRWya)K}g1(qH;HJHFI3ibg(O5v`6w^FiL2J98h2ah4 zYew+O=ALDlQ`|&TZ~d~4#&U3#__^UfNA|J! zC}AzBlNsZo$rIU0tGjy0l+Z);h|%!O{DN(8r)3+JgEqGTl>Q6NhJh*F0D8HNIb8wT zzDBxaLm(=XSkattqC5V?w%w37Z@A~L2tf|oSy;mNFC^hx1LHpcVt;+zzKLGS|AY+w zjuBKVZ`-abeQ#U37!2QUky&?x#U`>cETR}V^J$`{348$GNw5~%j9|^#7(C@JmNDW& zegTeq1-~~=27rXTAffY1Qub?U;dYSPg)osC<=>aj3I@V0{f%l}%`Ve7yhPr0Kk=bH+0=r$a_RHFx zv-N9@<;;0JTUR@~pjWgX;jF}yeQXwiVliuyw+yKuvc<4NIz;lt7HvDJ-z-&NWZg~n zPYf#*W`myEN3KlY4$eWc?YIgLIgB9XSoMi!6Xh>`3;*pwfp#n<75@+-sI4K$ko1AA zDjT~^yD^sK=CRLclg-lv1v{uKinO00`nLpURM|Ef_L~`0`uu$(ZPdHSpt*Yy0m9Vc zYASUB)9knH0d#s|-}~oDBw*0Lv{fv@jvX2cg^CyO0rW{6KctE11%&kB z7vJwBgFM6o(;Q)6kD5mOI~ifLaPxz#@T0KL=Ua{@EC3_*Ki9Chyn~@h$kaqlJ&SmCCr4aXf>*6eq^k zJ3;tW&0x{(N%4%N#-oHm6=c^-qGJ0Oo*47r7ktRV!MDFDfoLA{pc>1;M-{NX0EFPe z8YyyO`)=esOUw=gC2uTFuBhwlN28zW>RV72b!ldcdT~0^1r=uO3b9PT)B_Uzp zEHs*}VNuqIh=B0m7+5I}>$&=m#gbs!4gK`m(+KiU#)fe@ zZGd_EVpR&xgwa_h;X6Pw#FwTEFS>3QnOD_EO5cy~(=VcrDHluDOYgh$0Gfpt2qxX5 z6fq`#D5mOczRvnaR+_EVt0(n5>mPO1_3Lz16Vg0lg6*zclKb>Giv+XeER#+9f6h(c zzibmZV{~a{X+T9sND?v7ljS!^-}z(a9l2~{1um_3jgGs*V9~+JVlscM5})-Ot{7jN zaKG(o8>1WZ%5?wcej+tbHyKjbG1@Xtm%U@0KGKN|Py;QDTvctT9KKJ{X={B^hycaWIIj9}F9B#|*mFPCGE4`-)w@vzM z|5|&`6KF)IS-;UJ( zeFXno?BajeDF1XM6@4pbW5<8^Qrxt}z)yrw-}w?1$;by=O&)hex=2@431kc5&A1>+ zo&F43V@au^RWvVZ*#L0--cY1g8h^Qsfx7MURxYl)ySE3}9iS1Gq}iGR#UaFe<2T@A z3WzH=J|`Zs0~`7_d~Pm~4iaV$^91wwKlx#%4FQV<&6XSR0@J%W7jEqeKbn@4-%mE` zKkYYauGZgg2whnr-#|x>_*b4fE~S08h^@3Lgib64Uhb{L>O4-wmhAlm`<0V^HMD10 zphB{ap}p2*Cz8TQb1PC`tEr$M7d8cjTp3yENnk*A*k zqB`lNn~l+R1Y?r5Kxr{)(b)}LyZC2@xI%ZCx->mEh0`RisyCE7enP&gJ-q(gV((MG zj5!h%0081|;OM`vNWKGE)j{9xJBI&}t~DUNu!k0Xr&GFaJ*;buxx&wb*nc7suChZC zgK&X31O)CE*~f5#^Vg$l5=e?90Ou#^{G|JXXjVQS$YnzziuBWk9(E>Dz?QrTbZt(- zjH0-y7(tBSzI$~wSw(|o>gvredo%rI`m5WW>1q-?oh*je4FSLey&cs`9dwnKoV7}WXAz?TKSqAbR9)2f9D0Ng{n=V4-Qn7qD`X@ zp1e(@PX=_k!j7ngah{WQsd8r+mJ($j9Q9lwA5gQs!9SC&0%WCV&t#(@AOih*S%sGN z;K!i#x)S+_4Jz}wes@2m&4|v?g(Ck*7?e&|SReY`GAO0eO&H~8MIU+VL#13K1K5Z% z`4AwQJ7gm0Yyz zR_V?$%6k`-<~1~wk9=@7GmPu*!@DyudaB2@WYtkBYFGTWuDqRCFfXey#pO|Ltpnmw zL1rWbI>XYnW!3KTs@1mKH7Wd7l>m~h{@d2rkhRvvl*eXY$JK7FKJBmHjcv)1>uqh> zk+$~sWa!&Rd$hxw7k!>;Q`K4ncY2ga);gG2X2RqbQTM|F z<$XdJ`oZ0<{_@cw2Hlg^^-67QS!R<0O`-|qR-*}KxO&BpF$Q60&y}3XD9tmpWVM5b@713SZ~sD22GO z5i4X#F?WS1kzz?wMH2F@XYJqxl*>v@X?`PN1tk`73eH3EV1g@bK}8=8c>G!Zly~I1%ZIYQJC>6lM=X#H>GD1 zFVESz7V)7tr)8eUd(cM1CmcF}VpfCPAQ-tb<4IsLPUA-18%b+We|9)qOBTzSvj#D$ zi=XSEo^LEEoPiib{x8nnF-Vti$rf(gcJFrgZriqP+t_W}wr$(CZQJhNZQh;tP@Cpv{`x4egWlEb}>*qBi+D(odv{dEV=y$o2&JT}!j;sWfxnuYS#0j#96m-8 zt=#0UnWVuPPE6ZlCwdYytj@(n_nl=K4kUiG%gR67vW>!9Jj>Q|u=Oz%k|&?&GA1iu z&&NE`C68M%C=DwcCw5FET0EmYV0j&0QygrgcxN`GycoVrH0Lj^`y7J@C^`Y$D1ULq zV4J?iSib*7pf-0HN}^h`!?PRFy$H%g zF^c;keJW}GibY7lOC*A|z`OX5zn7{pho?{nO>*HR;ih0W@?OD$Qg+P3bR{05rAq3E zm4y@GylnGMUzxv?f-y6aD+Q>{-@tLd3;}B1d1~`!((O&>`J+Pzbr1tp=&N$7U@7c9 z&MP1{#n9?VzKXEfh^Wt620Y0}C%DkjVQi;@Goj9zNM{G&2pT6k7KHQeRR|sT;(0`d z2bB-XLzNV4^l!hG-m&^bt~y z9%zoD#xg2dVL|h8vJdmq&66sr+}i}p%k#UIw@$F#fUM>(UYUaz zAHd+*wMoqhpB)EbAVIP+UX_m7;9qiruoI}BQUmyV2UuLgp0d3SSYH@E`FpZu5*;z+ zyaQmv-&zB@y$MiX3Ik+!6tFhInoJ+4*@b%xOgK4y8;~(Q$NQ{UUudScxu$KdG65~S zvB5J3ipL^E6Ij8FKrO+jYGr#jsV?wbHTsyBE4R-t;2XzCcz(3I_^2NV!L&?lnZ)Pt@c!%bllU6krFX*oEJ#dQHot1mKdkDDUZIZ37bV#V>o&G)rn#-skcjaKR z1B{M?L*$~fw>N_UHG8~LrbX(FRrQe);C@Y(U@f-rWc}Rs&PMSoQ#u(!A`~yOgvQ6J zx4Mbs^K)}dx&uB3*pB^afhfA*;9#TH{oE5~!X9RhCiG!nyv^L=kdrLgV;NTYEw$N* z-ZVg1n@}yYUE~4UMmoePBuIbm7lxa>_JVc7J?er}Lw#U@?;u9ngN<}O>DCtzB1*8D zgRnW4n_6x`tPbB_DiAYV@3!Y^F-^^H*|_rd^sm|aeqI_3}F#6 z2(jLRdak!>Ct|S@3jhNf^%0`icq^MgXQN9z&2?!6Y_QQ>w6R0s+B<^N#>USJ3m`IT zHEA22O}EtBZ`i^MiRpvucLV1IK$K*dm@5^M?yG|1YfC|CKUGw9UtV;w7EH&>G2vW5 zslSQ!y8%5PKUHkJ3dM~t&nNZYX~h1M-DO#`*}<_!(>Q}B;p|Vj8~R16^Lt}sld9-Tid6YWej+T7NEew?pzeo4 z1L>g`>1kfSrD)Zc?r3J;j_?idN%8ojDt|C9QQ{WUnbBfQlcD|`_IM0v)I_lOp27yX zt48C)v|mV&<`71&d~j(%2TAf%vUvCWvX*YTu_ zWAYwT!wiFk!fl<$ypA4mj|US=PeCgp+N4C@M7oPumk|VNz<8z#5ZZ173E6v_a7)1i z*O6tz%p0}FCCn<5v!H`)&_~~d0I-o`#|I7EQlj*)nHR*2d?_Xt>7%Qj+@iIV3>;kxk>kFbaG6B1e zAm*9?WZ(cAqva0P=}sWHexdtzk2A`dR)@pvN{jk#?3Xv@j?*tXb3U1#?Mum-hs{%$ z{spO^Dr6OGUqjTe=C`b*U3x1YhuPU@(`w0W!aUU%Jq3a0parlX^A#Kc>O6wDH3sOy1W+w_vgql;YEXT@Tx#v&(5n%-T#~$8 ziokiSoH%XUiiY2)KsPOa_xS{thz6zf7T$jyF!9Aa>gjVJM>;^Rnw9Zi`(H zKkHK2F@n*tfl=9@Zw24%>nfo0GA3Gc6TvcJIX50Mzhym7zx&OqgZhC5+*CmiTN25a zTe}JVi~1;=Dk(?FV4$hdsma~G_jUn#st9$MAHpFQ(j55^8aY$mL z8TU7fZCGYNO~-sSGAW{it>@i$w~C6@MRMB8Vfft;`&$Ke*X|K|6%~yNO$t?2*#Ro3 zg`hV40DST5jzaXEN?Vpu7tF$K0yVu@H+Ge29dSI4kgivzCll4A`W)O<>dn;yBpof1 zu9t@{FySSH$s&wdJSNxjdAgKgu%p#n4D80Ary6|=_i9AZ18>Q!)MR{(f(`p3w-^5ZnTPn=5SX`Q>8$ug3yA6a6RM{)Bnf^3$oy{;KPE z>@N!N86Vrt>~G46zl4#oI|&KZ9m$~@RZrArlNB#tBlYnnH9SDkz^aRcgHeC9Ce#hcw{ z^)b|f-@tF2k>&V7XxO*#uZ}l|^If|_C7iSWc0ZZ_MEoL_MB{2o6|e`omiKkWL4}2prYu zrJ*%U@A!m}_@)K&8$Uo?n`Z?gQ!vSr0)#sMS@{ttFN{D*R{mT6JT zQx-fFCVVb^uC2Q-EM3>oQNb?z$Vqt)S5a$f{#iIciNfY+o6<5+Cy39O?u2yP#Dl6U zi7mCHJ-<(YDv8Z|?hxJx?i0ZgA% zJJ6)hWRwRp{3WJO;r%bUV_o6Gh@PaX9SDsYJD8^VZ4mCzUoBxF{PXmacF5o97TO*2 zwblc-?=`Ri9YexE9nVOjs32lMe?8<&K;Wu8kT--GQ`W zUWZu;8Nq4Yzu^rls9}>~yLPBtBL~7dmN75Z)R}1z zPr(>gL`IlmQP|ura+E4Mew<6yJ{d(#)^OXGU~*1MmMs)ks)Z7wL1jD^5!knYyjE^H z=NHoZ;7JS!yNJu-E~6W@zvw`wbK%V<-km!7J2{Qch4INNmXxi^AuRgDfJkG*FN7sp zp#cp^&5j)aN{?WI^8Z0*LjFW@%;)kVoh@H+lIN-}+H~|y@ZoWJ7XP5Yh;`)^{xcpn z#D30r+jPAiuVD;AGlq*=7+EAH@{ME<8vgZ8P={SES+ZT5!MJCX!paO30H4t_raZcH zNDkVpu+*z`|ADJ4@gU%?d%k~PkGPta;F+KDjSoujs3dvLTu3oL zqna?H-a47hQoQdY98NzFUXnJgvP0C%0P42+ zGxT-%p%nj*!NPwm51HHA2>uY4j9vf3)1pdA()Pz^B@skn?&v7C)=N#yAV1P(&+9eCyKcYQ($znTaP zm5pmP>+$W9g#7hcXG|OBU_-v@LK83Td(b6&Mo`iDb-@Uok|e@cclHqRp-A$rv+Q!< zk>t1LUj}4SODC{+WPy6p^0mow@eGAw5xzEj0?KnjY0UNIpn*wblqg}zO2>;~3fB!* z?Hty|nX7LY6D{;(UO1;j2*VFoh331jqGsvLR)x_7v^ zkhB@ns!1C(c5Ug{a?-D4K67<8ELV69g0j&fbnuXatMP@<&B1JD?S~&S8cVS`QsncU zPsDP_tdk1G?dRShkGpXCZgryc+;`-$@;+eSI)qxK)m&=%I-)a(#+cHdfsYy%qx2Ca z*B-e*F^DO#7y|TLBRtgC@9Yk&s3yD^ct@QWN4D^Q{#9LK4-d`v@+WA(emFGz|5LQ! zUxP-`QQy(&Kd^!##U0yuIg~FN6b*@ymm^Y#uWXIf^~=5zLj5u?I{vX!HDaDrjo)YU zNr5_}UsT3nj963Ip9PU8Xn1P%tTqXMk1TsUrd@1iKHqOn@PFJ;(gFg(f8*`PMV3d7 z{DxXZ-V5`WPU7J~4L6070CEB~_H)971zS7QOuk}+so~synM{2}#&fNvj|4V6wO@%5 z;XlMy#0vz3|M)RP@QPVws~<*uo@pD<=82L9yYhN|N%-t*ej9nh4vyTqaPm;nyE&{~ zXer~nI7mmWXJpNMqZibysmYM69~R8nB%$cg#fW~GF5`5FnvD_(9w6gqVz{`#vH_Ft z)r8SxCHB$e)gNW=BPV@0M;6ESz+vsZO7PQaC(Nw-`xPs93cz*CO(beGwuMU)Y| z;hLup?vkvJEDTG6I8M%hQq5WN=eGZkkhIFUuoFE?b0ZGeWNg`nba*tZqjA)IxfG!H(O%A&rCD zL&_Hg9wO%(Bh|-!N>g!{0Ah+c9A@#GM8lo{S;_lTM{It>|1()(c>Ou+-~4o5f40zg z{{M!O!+)yLWW{UBBJ-n+d@s;!HCHR3=IOLWX=Sq@AR^I*nJ))IQ2QmP@WlRcnTJz1 z8NF`oC)-z-jwA^F=>!GyXDz(SdAeOjO2&bEU8gT6>yaeAY!$|40}tnW}x44EB8Kn3K=@Db}~u zuW!A6L(y)MF+X!$6{wU*NAImYmcNinCZe5{`Yq1Q-?;aAFJ51=#=z}dIucIK2`QWP zt5sdUoe-`=+9iBY=hK9v0A}d+Rd*}%7Tk_i?W4QuTmWr%ASOeQ4WA+>i@e0mQ>OOA zkjjI$6bnL!}a5OLE>%S1~e@!j_ zkyWZxwiL1d%?RG}%0yi$`5~5gHKf`Ka$OxGg+sf;S z_jXEDhhyQ6A6G1s;oxD7SAyHLA<*{>1~zD|<9XZRDRbv_%R%OP=bP&tx)+5C)Yd@6 z55Ivwa|j21g;?t(3sYxvcTE)O7!~Hg1mVO^k{~qtAhTt9JG1Wcxl$q_7%f5S0A(uw9#^{^Taq-v=RZV+^8X$xkx;yf)1P^#{BcX|5gj^Eb=Rr}VNs)$3Nehb_Pjz0Y5FK8FPS@;^> z$;HUaXV?A$Ct{2WqS9;0#p%Ha!UflxBrB+MG{oDK(Z!;*zS&PZz?H}lHpEDy?&MM3 z>g&1{v zNnDE+@@w(Ar@>|2zMe<9MITpA(2E>&$IG>HOc#pPx^fw`9C3Q`4;JIzrP}i3Q$>V- z22OWtz0P870&9=%yqMf6AX{#VUPw6AAVbV$UxJ(>J40*WM^kpMA=W?$yCdc5xFwD6 zxV`Gju_dTI>}Ko^bU{~$XPWo6L8n!^=Y79^G8agk)#)uF{1boRu zc=sL|KrAMfo#2NMB2wx?*BCPQ(Td&G6SiU=IZ7?WG0N)Wl5hJrn-FZf1iEs7Lz_Nm zAapV~-Swb_W-5aC6(Ged)A8kgmX~fc-7vagj&yh(U#>-HT^Yw-`YkO|SNqOb>kqec zgxA(}^}KU2S?w&M-U6{(gb7sFj64S+@%O*L_r(>{dF4N+Sk4a*=fBy+|9@ny|6A@Z zOJTxdgAaupc_i6*VyeMa6O#5*iil1%Md%wl|{E5D3;3k&X9IC=?x3SMIpqu>Z{Z}rf zW?>bBk%tb~111`_>Q|Gk(`T(AJGv@YEXvHH5cwg2VB3QnCZYtM?i=6|d;I}hxTtXX z$$JS*!(*wmSGR{_a9rz{vGN#Y%!)AUqZz_@VJp%FZgwloMA4@>k%f>sEUP+E|6&Kz zEl38|kBu3T?XQx_Xa+DSFlW?t!R);&CRB{$5l&OFq{YA^bI#(-<_nJMll7?tkqOIn z>fpA+arVFG(_{2ohu6W6K7j63>h6`jFrQ492y>;5#eHVz4n`+qoVo8PSr^CfwN0|* zsXKHI5sRjW5K~AD@HTN@P>&RQ0)$j>ionnkVvCbr=ledT>ugSt6OhWrrsd*)Ip=lb zZxy*R5M-kIU!ec_+|teHK858=4j?V$SzKCdUc6rljCP!;YT+nKcdVz_%k?NaL`BQhBQx3m!AA75D((XNw zIP@nTA4evjwb?1}5>_Usbcz~mkrfOY*`xik@1w!30@bW;8zeJZ)w0ZNb)7>H5L(N} zBqYeIvC+J4*Y0a@{{fAY)pYf8(^tbKn38JV<#S4P>$Yy}Yh)#wv5$6dp_u{4k#1BG zMGIneLq>&(r+wwDBH?!Ytj6kVV_ii#{g&c#z9D4)6IJP=U5<1pn_Uwe7CAXNqr5=@ zOi=&Fh+#;DX4X6Nm?90Yva;)U!x6YM5pK=xoMux{Kj$@ht;up+rn9c;g&ZQM+A%8t zlnDA{11iQz+q&yAgJ9>l$D@u`BN$f?~IYEH`rLlH=8f9Xz;S04H1%;YCD8#svpuqO`MH z8Ey|iy1twAy5`W9XsdJL`4V9jGqj*bKxhc@%QDbR$@Mf}EQRwu(>-1gFZ z(ptK{v;LvX-Uao^N^QrzyrC{IR!T>YhSE};r4L7LOqb8C0U}g|GU{^P&%Zh+-$l7j zd^2IX$iGgHjHoM3r{qKWBMHYCp*dh#LfSBY5YIe#s6Ym}{A324U+f=hZ8H4nfB<&6Z>@Ocrfzwx zHt(meo0Nlc`6L$;Y`c2IWYKw%KfS(FgWPfO2($CZzHhyR9@|n1<0J1wekM`PR&gv~ zdU7*8-!VyrbrG&l0_#&7?RCb8bRq~^*OPl%yKUixd0`TH_oQInp*!S|k6}c~>3V+U z3lk$HRl%s_QO!D7&sm-Nt77}9?y+p6@OTthu~WqEAxf5{Dz7>upl}Ep{xQArce6La z%Z18XvozOO9}S&q@&}LcTEwu($k;kr02vOANm(Z8Q}n1HO>D~r6U$QeLz zgKOwzpO9=q=BZ}k(lxAU=%y$}Qrc$?-8Nod!7k7ctOO7Qnqu~U3HOPT@A~`N({ckCx&LDoU?>VCxSE!`OtZX&gp$l1*%j#(sXq*1$a?^A@aq8K zfR1Q@tslb}#~8;DCxwo5FR{1KVB1m9BhPG0?sB)i{**~zr8&`kLHS(8jQy+gf>VOh zk(<>qt60HNAcAu;cc~Van{3ue9X(WbHT$AtkKONjFHRDOp1G702squ=gMZ=i#C%^s zF2zoxzcrFEHMR~A+c7NHttAY5hf|l z&@7V6F#V5T9wQBd2?nT0oV)=>B!gKBvXXVNqIGcjl+cn%a>W#tas`;qMSD)mC7s!( zf(YoZUl-jsOiD8rELp{Tzt_9|Jf<6l03B3(!}`_jq~mZ$ZwpUY-USbbh|-{|G>u~V z{j|v@5k(qp`MUGa>y0K*nOct)3bUmh-A7HjaPFFgITZ`(_S>aqZ;oXw1ygZwho+Ji z%B&J5R|Yl`+%$BJjz!C94T}T|%i(L|TBauLTJ%cPteSHc6A@%7HFxtS>WYt6`ob;h zQz}jk(3F7&8m5ApI0JI)s;@} zNLS4|14;dhp+$Z^jNWJLISHxXPzG@{uiy2!8i zm!EO)_3%R7$=s-~98ZoLIYdz)%k!|8c^NI(9u&`ebB|Pi~uPWUA>ymjAxqJ!_8UTQ&_J1qt@?V|C|MS2056bYB7PPC< z&|^-TRXyt-5CIqQeXQtUEGVHYD#F^OO@Gjo%_=w&!U~CczrkqmRB+1AW?_%6@C9&{ zZ&Pzr8EmeyMn%<)XqA`Ql8UD0r>15_Rg=aQyTi?P*Ly1`cJEhD^SzhL^yOscPNu{0 zWSc2z2iNNwFg&tAaf7vSVWW>sD@!#zuu|Bw<@vA%r7 zP@S68hyfTHV_j;{kQLRj-X9#ak(7Dl(@h1Am}%O8eIXESISJW(=!w#-5FRI130x$sGbkiMYVdDhgfuZI z0#?JsmK!!=L1+fTHZl&Vs?0*Fk28Yqrw7!L;soU}6LnNfx-&CbNx1OxVrGR}UCbVza zPaRjSF_Z%^{TLTL*jZT@ih5=i;Vfj)fjswQMhFEnCjL=(?J6nsXFB6j2Ynmkn8@Pl z&eb)7b3G|lR6NbnL9&n$F)EL^=*vt~lx+?d-2CRz(^!)lVa9U&NhpH4Y2(MxiFuEBW)Mpn5q{%^K&QTG;7xKvoGG)2UJm;*Y zqBifY189j?uJNfQtZd;#4ZO$FohdqC&yichy?kt|j;)UWX6EMW>*jmwj@$|^|*ygjS;PY zYdFtYk3rewJqysk5Js@u5H(0H(#dQcLqGCK1J-o!0jLzW2!%Q73)3cv&Z zqj78n`VNLuTy-FAv$%QCDylYe)Sz_n*mh`t=!nmhyc$gt)bNSd^fG6QG2bo?5(_!4z*%IvmmJu{XakMNBss&KUwdYyfP}p{{z?ZpcW%w2tbH)DaoFP&^uNA=-HE92aZ zj)kF_^{pxX$}{j?J#(V9VHvmkP5Gl}Sl_Q=Ycoja?s_*Ph;VdX>lKJv?t~Im!SZyF z?LLkRdo?7xCk?a{HDU4lZPM|CZ#t|2X=B6NFoSH2NOqoFmaP~0Z z#R9;gij#Ad#s|~yM;e7EqJMl`|5!JvW#ww8`VATfIQznt*3T`k3 z3OFoTS$bx)F-YxR?F3)79$Pzj#Vj8MQ+-v24UhR&#ah3_H@pTzI4-@_C{w_n4;bhM zMm$-1CawddCtD84RuvEf5n=IC7>#~7ND|s~0Nme;O6qV^T!SyT&W?rBo^5u|7R#{I z4IO>Mk@nafU3655P!5%$5i6yxi6Uy^u!U;YjkK&R8f{cigYG*F%9bJfC&wF;&7)S7 z5$bE044esB#>S=-mpHCV4GwE{iId!+&pKYMB#05Vc}8D-jH=Qzchk(B#9Uh!M0aKp z=iQr|mTT$=(WKMqsM0%AJUm%m?kw%9$>sZ8; z2zI7jmITIHwYFxlr8Ho4DzECAjl8eU#Rdm(sS#^cza%r0t|xJ9@wQ=UD(`K>Ni~*P zf;lUEyCe*2fTGG3E|iOERF#jJ;YqBD1ja4{+KK$dFe6$-IQy?Ys*&tSk6A9>rk^xQ zbpHtSN3>n_RwH;?L^?VJZOjrbIF#E=O2;p?Uz>SQvOdtKF%}I<=?|m_bfLmqVGm(i zID%e&;D*P`PIJ_x0`zJnN8h@ueu3Rs$C-V zQlzQWg1&6UvB6I?rh0?Mrsc}pdpk9&uxq@L=v`YIq+9z7EWEjRfo0g&HB2M=kGfj< z*lz3bclB=UAf)_J3C;xB{&H?<@dJ01K*L#|!`6{J6Lniv+*?+W2v$8=eW%=Ynn-}{ zV0oA0dJ8^4s8JZPvP(F(;+=7!nL+e(`0eK`rA??@Xp^E4xz6_MWwS8mb?h>w;+G?B_n4Be-GG6R%hFne>7a*vTI&=shazt~ve1Y%xx&3dr zNxxrr4#r>`Ky=6Yz0t&xxeZW>TLAv^=*=SmS*w#L6hVhmnD_3RQ=qmES=isXzy{c} zCTQeNP(VS+0E4(XxaKpfVH#S8g6j_iDq4nL%@~_WowHKgaq8vhf9LRMd7ocj5ohfL z@W&S-lkGelEen-B0Qonk>C61tMwBTj;cP-+=VjwHZZT}w<_f1h6CzN4E|!}1KG;L$vmoH(*5uu`q>FQBoRkIX&A8Yad{uN-M~fC6 zz-7BD>=!9N0SQudwP8qzB(qH*e1s+q+|Qy;^zoM!^i{|mV)lUSih(C5@gcn#(7W{Z zLqP_6s7wD0>?3$1P*~JbISR~gNL*Dnn?7CkG|Y= zoQDBR2p@Px&59i=;*VP7NH?TfMyN0L;Bd#VqbF zjorD+#`wik+!@*WcnXlZ9woS;Opak)5@#IR#(033WMkQh20k%k`oxi!5$R-m%49mF zvP74WGk#jYLo1Ix*F5su=%z%OfewC}E+p{VXqcA~e-V?r2R?h*d^jw$EDOL=xbGz< zpP`iRxe#A8Ozs{UGI2@t;dqo%PMDIF|KhKBb9qvQFvxhqc=JDbVv<5)*D!_ph4ifP zs)gE$X!8od3PnvNf5!cW!NqIcB=FT$8dEYo0l1dOX^hbOZ1~(Mc6!n{4|8@Ga!9<| zU(8Pve4kJb7sPdG#vY6fMq*6dmF2W4dVB3@^^KtdY>J$QOB!bEMM#{ldD$2>RqdX;P zSA@9+0Fcg z%34M#G;vgraXyX_2rS_UEMbkSR2mu7gqY1}?JTCDvC}Ao;=)=L4MJk3CyV9v#4eSz z-BmBnk&k>gOi_<4@cH!UK6 zDy&P49Qe~sm|S{Nz75A;sy4ZQ8GIWta-vXB!y0LT#Wl5oeNYkX7$w$b3EepFM3{T=^MEDXvY}d( z&_yJ;B7dNEOn^8ukso|7ALVKZMX*67*dz^SXo)~@MmuMA35~s`TCCSaCfHi4AveS{ zFB>EWc5F0H)OT;ztro$2+e1e;hU!{F6iE}Xbq$<*rYaQ%bOQ>1XH30-43l z4a3=1@S`Rr4JJ40^iDKs9mydta#|R_#=8wLkJn-ojB$)Fxp7mB}YeN(Q2tGxl{eInKKZ4^ndt7+AjDdRSPMZywf7?v}mhpm_u@_^+RN zf?dWI2w{z-vzp?<#0&cZamk^vo9d=%!<%9{hzdXXVjk$aYL$v5qbJj(?m(JbPQp7Z z#oVXLWL>ucyTlG&i!R7x7iR)x2^2o&!kjiv=f2b$9dla}5164eD(_@cS&%U1EIsce7U)!wq9tbToQvUxa} zXh`I1jr~Ox<3%wtIC3@et}!z*eNgs9-;-2(f0!wZ`OEb0Ew3uT_g}Xc-O&--R#V1? zm-WnW%Shg_ffoaw;hbZAPYV2lnfubWzn+mjql^c%f9bn3O@~$Q-G*xJ>AE$1@q3a( zz8^i2zB%9zZ13r9Mhy#Gd4{XG7)kG~+r;pOnO{vjiQo@1+AtMGc^HQq8Za!~H=BWN zBAr7R};g-Em^Fgie#q5zZ6dQ9slH;@sla>?Bmdt&fqN3fibUOHJwhEV@ zVykZVkh)$Ox}1fdEK0`D6NgZeD$2|jh{DQ6;{Vuc*ufgr{zhL^ zv1oe86&UN_l9IcJbIXXeKhynZ97qWb2_+jQlrb=MV~YASRpBcXsrjO_~5B=zJENBuL%J{FurkW0-_>0lv`oS(;Gza14T4%qyh!*2 z1g1DTa3V|#=XJJ^!O>Ue@5Sj|LFlfadj_lMTms9=R-=^7CZ@%^ERv{c`IH2=f*Y~m z1tTRY1Sgb}?Oy`-0`o?^)vGv9i{-FPC&Umo6A2^zC0v!)n| zyGkH$Bkz`BdGIsk#&wU>bpbz}{t@f)cb_O={cRX8;Lr01o25T+4MyjW zS@pS=$xiqOxl^=9n?oj=`7-xdN1B8Dp{ZM(DYq#ZrGh0)jXnIcYr0PHwE8;tAAj1C zHF>}aIDuJyQFc z#9tg?J^cH&NIpS_JA|GYgF6I|*D5b~_g?YezK1pOAI$r_#IG?Byri$PVqc2$GaPrU zVqMbvyhL9P`#rzDCJ=|=3%a~sdVjFi1Gt4`K{$VHhE zgatjZ-{Q5)8kSW}|9+#ngrccT@zPDiMbWa`sj|N6ZfS1fKA8k1_5Of=@$}yDyys5$%K9!u{6?ycd@1ui z8uWoyb2Pa2C3wC=j%NXit!~2-2$m3aYnC=P9@zHH-HC}MO*;o zlAc7kg^`C)lbuAV^-CquBcj*rB{hilof^~v)C_e4LZR2~Rmn%GjZ*ui$5|)W3uh4L zn>CmXXB501oAr z>QAiiNoW8l4hbBhT_l@Q@1K1-K(f#Q&`jcJ1iNrHg!FN@`ujRUV$vu{9QxVKk^%Ad5fOAw=pjM>>GGLAg*>8wJgC9|Gu;gAK) z@omBd7sR>#AIjb_ywY{)7VZu@wrzH7qhs4<2Q#)iGqyUmZQHi389V8)qdr;Z)miJ? z*Y)l9@AG$5)m=~3s4-UTBR@sce? zktID?r-*>*)uH;L-ovC}vBo6TVfcFO_67xB{Ox_8N6VA8S}lgP)#*kh87pb6GL)bO zWQ&ee(UW2GanToMlZ^S3WY`+Yq1S$-k<68P1PiUZL$Z3@O`I_~qnlG-fKY@^U^slR zUHzMsl}54%z5Im~{|*DuiQ}1>l|o&qo6krsz~DL}N&e!r&DlyJxv+g=>0rSF)%x;i zNVMNsXHS0N&u;}U`fis}iJ}K`s)@mbI{e|1`Zrf{@dI4gp?qO4+l9Wedu{iQ-xE(8 zCA-aMHgI1VzOLpG1s0IU^r` zEF4(R`_ZFai}74z`wCCmCK&{VBpUYplxQ)mX$91W^x%j=Gssfc!`XM;ar>Hy7H8P} zl}kA7trnCpE>}IazUINK7<%2gD$AJ`02waK$;%J79-_X(1qn({L#T^S0#Kgbwmm7? zr$HP7aZ?@pU2^LO)+f7ZByCbBM$6%zBF@bn>P&r{?A#s$*SnB<6?X;ZC-agIyI;YM z1~=;tK~4_83;nT(2DcU@7?u(Ce%&{D)H>gQJ6pOUUul2B8ji;eUXPq3Szu_Hdo+8V z6P5XH(he0__v>pG-6TL~CG=&FTV&a*hvt@1nodr&fm}PCb|ac%vQkd%K%?W}^l<4g z8keq@F=dk>JLF2fxwVvicD5CQbG|cDThv7);$4;NKt$d82z2^&T|rI_@fFxb>65X? z%&(86YB1`|eH}JD=D*!{<;({gbI`*Qet`a!T3xv!PgAZo7~~L6MZKh4k?$T4q6sgi z$;#DKsP#7{8>9@MMAxB4A5;&wL0_kFRXi_P5vxr%MjF%)_d$Q5K~%&q;1YF60YMMe zqfaa07jlU^WPl(BrNULvi)p45GYZ|aK#<{1=tngA1@2)Wvv3V+2E`8P8{EM&bUVck zsT<6}b@VI64w)Om!54HQMQ_O)6uw|2ljX6Vqwd_oN{on-?^+(xi-A=-3oA4JiCo-` zpC*$;y}TB&7QL3W%FWe4&ASrU1)S*|L|s#hp42zf4A@rL%Eb+s>+{>999j+AeS8wyHQKAKFgxr+KkIT3#;m<#;IG z#vU3?lBG>s2^4#%-liNnOroVxu`5_+E%D`hsNZ@_VzEtIW-WUw-BL^@vm4BI zv`(wG?#lgLbf}YNXWfZrEtzTNEUt~VZ zbcj+%my#>9CydXgIfTsX5lr#OomRfoBgC}=L{@`G z`I>eCWTpb^l1(f$t7EI;qhL+5025P^brH^CAk*(wbZucdd%2yA^S(>od+3PZF zKCdV5Q#V30)XBT zY~A={v>aC%bz$v@x*i3P**9%+Z9?P@s!Xsc%@}k2Fq%DJYf4}3A}Oxl?0b;o4!BTn zvKyssT3_j+FJHC&{t}%r>I${)W`?ky3D{0(1i01u7)%+rr(ZzL8McR9gw#b}=r<8b zC0q!6=&UoYzX1e4CS>3M!Nhw2Qr(;E^9rHyOKZmDLrueDzMIAq23=Cob@h!VQjmeA z$qDSQP0e5+7$+qJ9I>e2|KRbkeXDX*Iu*> z7zbUNgRhih$d;$rt0@`>gcVhQObpeBr6dihODvXgA156F=g0(_< zL&pw%hm{@p1c_;!14d*V3x>z6*^57b4EcFL5t3wp3DOCrAEPA{1vW1<4z^1CCrv2H z7zUkCq_RD|aRZRcI1&h9Tmk$tzy%o<3JWP0dIOt=k_c-nB&%41&8#GhXB-d2LrK7B zAS9?XrlZVbYyeYochp4_f$PYNu(lEts`gNM;=P%n+K_G}<}@vMM$|1( zO2s?+fu%bFff~JU&@#Q?p`MUPq~5TE-zLHrG~rXue~{U3 z{x|Bm|3$F*j~-A;^go)v%%3(YxEk2OR~SG0Bice4*%{^L$)hY(tWXU|PV4Ae!i81` zUDKO>64z=DsLyOK+M379^Im+gvIWcLZ~5?5GDoS54!c^WJRWw>xoiHszPuQG!gL@M zM}9%^4y*pQ0@E9@!`xNm&TvgmO4XekZU>nZx(!2zb_Vu!;0jVb48flr1n2K!tU@fu zhdhF;12!GX2+0u|0+v2N1kn^~^EtqsJeJ~}+7Oi&j2l$@L+l)OKa2_t5~KjOK9DWK zwa5g*UCleN(4^r|h*mkOocC>Cl_a=ZD-96uo;@XOYOh?->fW?`T(8u(s$R6433PIF zT?4GPHUy_wwusm2mL$uFDTpjEH&^nDtYSa9b%qyyaV9L$D0}GAYk<95B&pY}xJcYD ze%hH&T-EH-Z9M;0+GXug;AlHt;we3x8E&>FUi)4pG2no(?5ev?UaC4?lvu7-s`8ro zYCv#UxLW9~B?1D8S8WIn*l(6)R<=iMlizJIQWp4>{St$4a$yTYp1n{ZI_Drg}VJ=DO&&&G&Pq)j&zfM~Sm^T~K={ zjHtL=bvR6eTa_F(oELPZ3MPEUO{!X?3PznWF4=*@P`G9brU*9+#|T#oXAgG?XA=ts z;S7Emq#VQy5{@fmqH#pW6wAsJszvx6fz#fGgd3cmnNYP+H`=;i$WPnNWySvDxSh;k z*Sud9Ast{ zCuq~cxy59dt2GrjC{H<$bq~S=QU)2qXKEzQk_$8jLLP8M>`pDy+66BDG!kb*SeY$E1s3Li6@fSDBca zP4gGUn7U1)7s;5~O|utGBD3~kR|6swubzi6fvJaB8~Nqcc)+b^hK>J)iVyGtW&OJ+ zkbdx?RbaHKMB$Bwe`j2<3+wK{WD)Oy`P7}P%RA4DgvenSO#n5zcs!Q|hMR+o1nN6Q zpMb6ONJ1w)%DsaG>ET}0KuL@js)+!hWaNuGb?%F#^neeYr0kPBdE9kee9B$zaR-ot zOG7Rba3T2jfyc7te&^N)9l!RYUi|;qH2)77Pu>LpFfs<1{YQ;RRF?beXx;~pw|1|^ zvdcn>W*hSSr*ZhesL~?x!4Lb)fW*8|a7O|(n*$zYz>%9Utr7&-{Y1>Sa~^>LWk!7& zR^Ny8RL=U_$L(psPlz{V@F+3pd_!n3^e_q-sP?bzTLsdlbmdAKfvf#KXf-L9c&>QtVoE}UO=biOm zAm&Ei4Uq)b&TXz5zDIqb+dWfgWlh4gjeSDt#gm&RqJQ{~mQ5p>0N;&#$9}e<*CM_X ztkNHL#Fmlf>8qrNJNFc7c$VaRb*}=f6DiKcqjFN$n|Z=&J%iLzwWGH{ro`IJ0E?Ez zkn^pj$9Qp1XaW(6;;y^Wo5qTPZZs_ z`ar++M3LS$k!(*`5farUme=`U#H{On?(ycoMYy!)?U0!g!gH&E1lk0*97$*ay(M;W z`!ZGnm9dey`sTPJIEc|+N2eYXP(nO>QvUAu2wP}JW5h|Z84?LKQB{}8 z1o?XBD~!)cz6<+{VvI&{09{o%2UDd+>3K4?##Dw0Gr5Fo4}^+0MKMwaF&fVI`j1;@ z=%1fsu0O(M`y*V~{@wYQf}^R~-~E9U<`uDhr2n^(-)5 zmB_U^X4)1D3!IBClAfv?$c@=>tKO7`9y1nA$v0}|?|hycre9Vcr?-i^KeJuafy3Zw z0YJGQck&Wi`!^|ga=ng80!uAVr&}Q_wGrVUHcpkY*E%o%3U0%Bj^<>ua{9UG%ty6h zQlg7?rFIcWV!|Ub4LdAQs9^OPHFw@#yWAV8vW}O{Zl~Rwm7CZ1$5AaPnjrWh9a7N& z4*|o?o|9vox`f(M1ymFX)(=`F#ZmTg+gB2#sN|Rzdh9RVM6#r2H2*&9SV_!Oh4-*n zGUOZ3#HjE!W6faK>{H+%whVI;_o0TzbivOWppG+Z1!_6+uY{oJw*m$dIWd?m0WAhvQYkFr<>n=Z*mqip$G~F&bUtFtQ61wl@TB8nU>@MD( zm?E|pWqV2iWuy zkt5j1IxT$t9RoMqEUM*m{*bF6Ov0yXSr)~JT*LdYgIV5*vP>8=fEZ$v{O6q6UGQ%< zdS0QMpJ^N!2QCY#qi85o1q@}&s+pp5%87Dm!3QSeKASw!(}`b7!KTEQrm?EN=3Exe z`W9Xd6a76{bMDfRtUgLz-^WDt-(Cm&cd-6fOZ}grs-~o}E{^;@g_&fd8YxBmIjz7l zU7b3YHZT?p#$R!5%>f@(meAcO&G1Pk zvSLz$PX68|)n={gBF}U5&3G?!7|4+Ry3?Kq%Sq^()N1Jnx0-ETdW{ol-_zO0n+u!_ zdFKExuA2`wC#gD}N?*aqqWH7Trwi?ByL;ZoZ&W>j@|Ig zduLk6{L`EYPmjjUyH92cr?L-r&`J~RLKvTtD4oSWf04ZJsKn1M$gY+d~x=D z<|1R6ewmIrdq@rTNRw~~J zv9G=m3NA<-q+IvgIkep(Vey(h0ae@BzbQtFo?uybA?ZOzfmA2cPxNhb$bU!#1W-US znq5D(DVV=`L&kuvC3A*rhh;O0Vq*TD!^+BQFF!w8mf4Ry7W{WxmjBLS1sCUkyf`YW zAD#f{{PTkrk?Lo2ZE9y}10f|PExQq2(CUzdpoGW^Mddl|xv(7r8Vg8B!JZh@s|FX3 z>)Z?Y3AxMGqp8u<(cz;r-XCf!JkI3oktfh3s{w?OtvVae7uX zF73s}Cks@o23D_l*{&wOz_*(hqgK#~?C)?eVmAO|LtL28iWb8%^sQolJD3$OQvUg! zrxwHq!xuvW(AENHCdXD^TQtoO+7dvRMW;3g2UPv7*^*~?9 z6#@M>fg7Jb16MJpSX=^pdZrJfNMuHPOLge>gn0SoTVb3-xlIBtR5goB{q%`IiiA3* z<%c}u8y-p8zTsQgc>#7VPX8zDFDLX-Oe8&_Z(!!#B+mE1YgW>SU#xjCiUB0ix#qu# z#l)Gd1WA6)1cosGLc)IwmWZ?Hqgj*gbr@8qV0T{3S)AEifid_y;H*A67=Jzh|4Z75 z>wgXS|6H(VDt}zCE28sP|8>FM^w$M@cQIuqE-${XBu;{ogW^(cK>#S^H^5|6QblSW z_$SeIM^x$ILvY3Wiv1hZi?P87{}uVsrrYJg`}^%1)(_k!*4S&IK}y20k`ea;(vV@k zV_0>yhxwiGG9cJ6J)~H)0{TxbfyA3gVi$Y&1y^W&w)TF1ZioJnGv#{DUlW8;C3tt8 zJ?z7AM_zG9(Gz62bN+a(xwj|z)+iRBC(H7SP`VR8yo;9Ih8QFT1%38)gbyWd005ZT z9HI9!OVRH<8e0E!2uM@#W;AW4FIuz&1?2R+jg&7rna{CXQ#yw98P}ZA--PS1NY3$= zn*!epBsTGS`ysv;t0j5*8limkd{esY6oT99u4phYSXG!i0hOEAJKzMo_@j`wOKF8` zyX%Ut9=EKxlRF{;;{iMP;Zp&#rxte-4+NB^tcGfmzFllwM zxDtD~Qc(P`)PnHCh-An(k|Mg*MW0heTC*SF0kwiv-FzgO2l99L07ZN@T{XDSv^UwRIOGA)z27x=#$eTexFS@C^P7}l@M_-}S{eykT zpH8L_N)*C%Knoh-BnT~exH{55rgYq90dsLhBWsxp=rXQU{Da{>h%D`i1CLvO{vnQV zULv#w@zJ*a6;%JWGpdT2k)w&l|NJR`M3mj!hb96dEPWwfWVWq*u`MlAW8E>;2#b2U zWr_nu^hv-X^D7@^g-T_!l?@*M!aTvkM> z5Q|CtEi+V8O!hWqkK$EZ`JY{0yUuu@x$OBl?K9(Sxt%TiioR#5=s+8H*f>Kg|KUW$ zQ=cbRq_3Yf3JIZ464~OmX^@wVFuZz0JprMeMrES(mMeSU_>E%U;j0Hh&%cWrCTSlz ze{+d>_Su9DuG+3nOs1*0e73DoP;4PFtoN3Vt=Z^O2)Kzoh)H^iR&MT`8BcLydAkvN zTcS8$W{;v@;>>|{0UZUU%oJ1kfY*3A);tf|E?bLXjC4#usOsESpX6N) zx+(DZGdiECE9M1R`H$+o?ASIOZcoSqR2X4Zsn`NyZP{21Vf8&=!M;JHSTR7fZ4O<@g^BCSgq~r^Ls@g!1t>#Qg8ps=quORXzWstBg_|SN_mI z;FqIJMTVmId_WSOW6UBFdZmFbCImwoN({!}EW5_Ez!WDN5@fKm=9o{EgNLbI=s!0% z7X{ZnP+il(pNfuboR})TeRAm}zpXJ{EtuW?nJXv+#d)^^e@e_A`3MiC50bmH1f(2D znSAg|tTjH#j!m~$jho69VE!e z<-O8?qfM1G$dU1q?U-`hA^7>8k>OT^3j z)gTnvfIhWpyk>j3L5?ar&9iQzo-|X_6q+&hQdx`=S-}+h;h+_Jj6PYn*Xfsiumzve z9+C)Lu6eWZFGN!=MX{w|Ow1r0rG^YXNP|K5ELC+Fl!-NRwe}0%G9xfIHvImAV44E9 zJ5Wh57b?7XQ%$_~o|uu@r0P_oGPg>~#;6YBV@FNkpx*S^>Jjc2a;=bDP}M@ll09lM zbRJp#;aZ zAYS;M_2eMCO)@NIyze6SAaYq=Yb|_PycI)R%_A7DSIY0)FRj?N{hZeE!-z1F%w2!1 z+`6u{IBU%KdsFh5M1fmDt@ho_{5OX|bogXYd&Hj_`R_~CC}T0_7PqUcjwvKs5>}kk z)e5-$&bRHyOPo1qA@5>X{Ib@=_8CJ0X%xsgP+SFgW;FlilTh@LDqeK0jI!6oj+pBvpTFRyt4UFl)!nrK&xHl!1?n~>tmcTq)(5g4 zy9hVoh%hL=DhY>`*v?bjG#8@;zIsQ`IjjZhi>rKnsOoCLwN~#UZ9_g@69t%~(I*(p zwAJ{(B_|>V{c_t84negpUojp0Ijf&oJPsPi^QCN)>Iwj5#p=I`E|d%`{w9E;2@nqo z?f!cQq5hRaJpZHE-+vVQf7{CFudMu_f7#gocWSEW{iWo`FVdYt=}s zAs9j2o6I4KR>_Brrn?S9w`yD1e&GLSH{kt~&#rJjCMcj6i|AEphq86a1?f}0W;7X$96S#5u>&;|NYbz^*<34be9$obE>iMH)LebSq9u&GHUn$? zH@5AdeLBhRO`8q3b;so@`=iG$3iwx+yXpnq>1JKONk?41UuJJ+Rd*b$5<#bJb{fBJ zCl9_R_1hb{jIY&B1l#daQncvQo_Yiy125LzNds~lKW#OcB!x<*t8eeO$FW$$?Nj}Zp+<>1TKAq@<{v-g4m&Kd0lpG}3P>bv1$9RlWj~fPtL#3!X z!1U8r8D6WMQc3iHTjaRrgY1Mj1wBt36gR`$Ux4Hpp8_Gd#E4#FF%yea%eIght zExk#1N zUT_|Lyii-P)aPfVQRuC+y@WzwZnoz_B6Th07U$x!D-O-CrHQ3NXWl~k0q#|*#@VPiG+y~?`TLvXrhnz2Fz9yGR|S9Qua;5 z&~j4{xt>#yWrBE9J3VPPgbNR27w|LlLO_AG=VT9raWh#rXk)za#|9|uh`DUD>!RL_ z4OFcQc=5c_b& z%rGLKhu(xw9MbVf@K^UBANOu7q8=!pyO!~g_8St$%ia*57A&Hh9f9DOUp-{-kF5A+ zPuTt7@M6Ix@WVLrOx_v0!oeZf0=?H#!3pq4tlr_f^YGb5Pjnx|B!TH`$Y4emzP@XU zUNGuZ3d0Z)XeXGlJOBv%rSi zZ|P=SiP&3-*f}eMh*?_}-ozLV4-{m2VO%!HSUy9_m97abN%`~I6-v(9tVybPgWL!h zm!f=)<4w_fQX!;9FENLdZv{iRe0)ivkMO}Sm0sxv95`5m+pgX@oO47MAI_$NeRicu z4dj~|vIj}OeRy_#egQAj7XZC)jec?R7-yu)u)WJ;NeXwVv&tX&5%xaJXV7{ ziE}2Q-`x;uAWKTcMj}vjf}~GCjM{HRb}L;~qAw3Ff?Z2dpIPvh%#he_EVnie#Jm?P zVm&qW_jTF|5R_iQ488QQg=?E$uG2PxqA(%W(vyrIwHPMu?A7x2YN_{7r57!){)Xs0 zV@ADS!8QlU%0d|>b02HfQLL@PTGh!tv#Yvnw0iw)6YB&d2 z?hpm#r+oeWbugft)*aSQ<=PGZ*N!FRpB_Q}r(~eto*?0?Mlb7Ac3{-^44R*gqY)mX0NZh5WlifHn@rDWXTTnblDl=N{n3jj!1Lal< zRZ;E-BjZc#8+Fm{Ta*{e=Sy`T#sCYQL$Dlh*@WcyMN6i8#J*?O2V2b}wzQnx>Qow~ zCfwXIo5g*W6-(-7Xvt0w+Gp%)?v>m!1Qi%Q^>cP?HDiStNPvdkCED~EsQdF3`i{jh zjy!{kMG1F(fpSCCYUNmwW+2$|uUFHtk-n zBm-yM*F+bXujktq=A^~J4L75;if$}xne!tO?fW5No{QcB{AGtH5r)e?a7_&DmN>yLxoxSZv*$8A>-? zJ%75>R>sTq7HT0YGS86MPU|CxwECX8yRihKnqV?_y?)m8SlsbjsZSOT2d48KD>f6* zeRuAOo)6Y2n0M_IRF+s}I~o%-jGOI!FeH1spOy0Sw&^u?&c#C|6=lChy+UtEYLPGu za!>AY8Bp>+LhIJti3l|TTSj|?J*SF>+Yrh?Zi#DA*r5bPKng^(80;JjNcm4w%ENdg z-u&qK8VUtzOSMIREe%SB?g)36+*us(fR^34aCBxv8`?mI2hccqHq`GVFV1hKD zJAQD{Uw;G1Kn3U4iqar4- zO#FRIZ591Z;ETo?%mgp&BeOf8&F7CMQm!sv=h6LylgG zfn-&<12xuqBBLedE|aY(He>(RqJS~?kU9QyJg-z1X)1{a&~zcRR+O+}KdmzHwPS+8 zk!%z;p1wFtPn>x=_MD&hd!P6;!d7JzteJ#4* zZJ=SH%`|x=Tqa4Zs{Y1X$Gg)n@!Wty`}N&(wOg%$O;ro`?V_$*uYv8iPa+=sI)_#5 z4X`W<>zTjKFU~@1z?AZraq~H5t6I{V!0e28a5#?4t8I5XNLR+@NYi|;-qa&&WZcHP zDQbF`S5;Geg>=oPqt0rGR%hU}Ec6F`UOp|UMTDlwUZdl|TSfhiXT*_W5&3ifiMFS5 zYM%L-fBI0H;rd!3eN~bN3%z5vfPaYI_h5{YBoB)XEWT*Pn#xdw-bOJ1JOjP4O3k3! zz@g!I2Wb<8&W0EO;R{huFdA2Ls7=u`kaUW55eu_+Q+1jV1g5g7G!RtuoJZQX1XDGP zH-W}BBk6;4OKmaUS@aUs(%aiP>exLx+qvr4y*k_3>(~t$g1>zsTHFnevW4Zq9p9-@9lLreF zk`+oqmk6WdPDwGw(1{WZ}&=k<$fx+@q z^B?=H>GfI% zf9IFCdYuxbT~Cq{MO)9Dp87r}>SCTh0pLn~T|T(Rd$QqvMfAPu9VUW!A~X3j3ho!& z{j*=`sT}J4iyvbM#H+{;KTpFi9Tq(HOp|`8C9?q4lql)?T@E~k&+ixhzpwM4|J?L& zJlR2ZKn#m=G4#Qp$cJCoL05;`O9yI*xyuDEi*eDTBEz7BiIB_3wTLQHqBcuH#D$X5 zV1-GLW01y)vPh99l9r;vU`RDeT8S!ujpvD5h}Z{mGj&svJR|>m#h3D2d`{Kpo-p*F7gmj&&{K z&F061HYNGIhI5Js9cavNWbj9bu7N}HMJZXdm5DRav_Hy(w*J^hVt|bcqf(F zV3|x?N>kka#d@5GmNIx}?9+9b#oAs$!Bm{%`K248Y^Clcf7jh@U`Npg7gIgotdkq% ztO8c9p|Cz>!8jJE*Q=jyXJfE}4-Ikp>#MPgvH1E}6i)icSX44d-n{=DM_RaD1_lmunz1f${&9!jtjSDIX`OhqSywBm{<=dxLO3gMODQ*Hj)yokb=o zFG2gtFdrnI9Uiyj-{iHAr~z+LTnIN zDNoW<>;NqFS>Pfw{z2ozor$YyBxi;b|@)GCHNl%|WFhQ^HtK2k z*dcS}NQ;+DONBfjJCqFyxTROqH2fti_VXZWJgqwneC(m@D7igku zEH&v`CdFMZb0q&QMVbPgyL9gh``I?>_kCe}W43;4bqCZ;l~Qx?6`5=v<2(I)^8j9@ z-Z@6sQ1=AOKmy#G5nkrKT-LEg0M}Y+Grv{Sjvj8@{7N<;Fz>TS?jUg@Gbo>t)*ntM zT%)aVz;mat#DppL!W{;&hYYM$mNYp*47C-=kD2h5!VyD@G!`*TS1t*5A2&=_#vAp7 z+?NjbL4X{U*w-I|qAt=I#*Vn1G zB07F>j%ni*(J~M9I_}en8WvCJUlA|K*h{b6`&91JDH}N6ugHaaGUmh=Isai#PZ>ZNj1VkXSA-BFGwsg`-h{0LRp@?KZ zbcRb@g3c}!H{oUg9h^1orc@`lV4c&|qp#~}Nnb;J>y%uDD)LHK9@yJVOiFvVs;uV( z6eaG+0}7oW=cwYiwDcNAnRTRVpt^|T*!Ow(LJJJ~v>W7sAx7B{(>?R3e8Ys({qre; za8T4NX_9vZs!1O+8ZpwdFauJ&l|4@u;Ih{Rys`R^>b_-i zcd)DmQ~%?l$}z*}p23dq`L@kpfA=j{Wj?-!!q9PD8HY zJRrzam*=!~k(hXKNFpty)iQ=KGoXiU*efE=ZZzp6I)$;~Ma?0ZRp>w7`fbkd>0S)Q?pSXlqQ; zH+h-e@XuGueX7Kxck2O~ETnEIYxgm%Cdo5nt?`&Wj)Pu`;Fc{R9_<-06p9lgKRE8x zU%#k~W>iqtKf}_g_WX-C}>~D1dbXz z6jRKR0>4fI4pDb*g6SA4!QBCVgMhyk=8hF&C;Cc>xE=U3iQk2Dqr>&8KG1CcWD~kQ zwX1`&J+!-y(jCx$CF&hA`KO)Bn+oLS^u!-}1-kY^>7M3ze_TKGYo7cA#q}yYKvXyR zNSOW}!u^Ue@VIu^zRuAdvo`d|y#6FGwW~l(SpU0yn(|Kp!nWP~pP%&l*@xHPkp&Iw z{btPnXzB0?e4qCV?a!9rv(-XkY!@JY&kKGe9V!jxMKo4glk9xz<~E{e+KnBDh4z@AG`e3;+y2G)gi=gGu7WCape2 zM&n)ueHn~)$q7be=>mh}Y?+S{3)-U|^b5YHKkUfc zGq}jNBL(u;R;=T_hEYb#6~bH-*5Er!jhF!y;JJ}Pq?g*n2#2NltnvCTJ36%@6RR|`OAKe?^*9s`qH*lPi>N1)})UJ$2>$e6S(^Ty0&b6*+=auhu=^?Mo zX;KQeFbkT%BFsP@keV^q@p-Sdhi@>Bz5%HfkgQs8gC{F0lEWB-%DKcC5%Dg{9tg#?IQ8S(vbchHZ@P9dGMua@! zL8Q+=<6aQsw6|y#V>O-CHJa6xQgEb}Rj@BfCp=ONv^`Y(NsE7)yUx`Rj_6d9Ov#0w zPlkV*FNHXGf{9pJM1xqFYl)~Qp@w)re}S(p(~LPItl6xwF8iGMa@LZV^@HlZ5<+fg z!L%{N$mfs1iam4{l{*3v7c`znzMUcH52g2bOV+lX?)23xBkI#AM2)<{a}9ze1dZ7> zRbG4SB@@8*TN?%p8me!V|HDlUxFy!yq@??Fz`CAh^2XD!wAyd3`8_j=i^O&>Hevhp>%m$H44(_hAR@4VZ`PMb$FQ9N<6cq3?j zeC$-N*D9Vg?=-LNwY_=k7*y>CdX5hX-L+k$365BQfu9_db?U+*^NLIEV)^3K0m5eU zV%}2exLd$pFzIMrfpd)!obUoL^T{v1F0cqDP9SM#*6HwPH=nW^w{5IA2$K)4NWI@u zXc!&+q2BJNGtot8L7GB-z>>BJLrA;yY`=`Tg(~Ww$&B$bf`0tgXg$frf6|FGc1@hs z(~PG;#Xp4|?cfQ&ohI#D{|CZ%%)^mMq2S`P>w#)bnT-Gt(&6b70LDbS?HIWPaRtH5 zv^u(pr-smW9;qH$aQu(sj5+@ONnFruG)y68&vVdeGv^Y#j!6Q0Q?%BOMtF(~;- z-k}^j$is=pf{EeUGK!pmNhE*v^=692Xuk_S;?Y+b#3?kyl4G<4Rl%d;dCNaXZU?(K7()k5(uL+>2_Z^V0* zC5ea!A6|Tf)qCKO+kV>Kpc0We5cmDKjH4Q;ry>^DZ6h_vp^tVxkt{T5(xFrfq^i{t zl^4rO2Xlm2ZGT>A=h^D{{;>@35%+p|opbrN6!Tbbv+{UO%ye~jyIFClpBSZ=G+EK2 zaR~eA@V1uGK3iwk%<;>Zdmsfe9qRzU!5G9oFKRP#mpIqt80~O|89AwX5p?Y@@KK#m z6=k5t1RWIyNexmg?Rq(SOXw_sY~$;LH`2b6PUyCEn;alBs`j5D0T$%F~bR|`_Jx%)AahwE!+^vS9XL*&XTLm*}tHyIxpLgFW&%b86dg=#M}9Uqm=Pue!IkHUWZ61BiiC z3nU`))sIi1cs`ddQNi(jXX@v9{m7-y1j)H2>-9YcfU;wYk)l`;fgtiFD)RuN>I(xL zG7Z|ySGmi8ldRQFkdk<^X57W8dCk9{EpCrOOzAL1olSK8 z203HDiG>5wShzub`)Gw!HJO#n-(q@-8NvqbC>0V(_Y=^}otpW?pWyob;X!eyq0r_1 zfsgk`QvWN~#Xsy74^H8A3y=P^6?>jnFhXg zU$@ENVdh5)##P`A_Ln&rW66nBV}AWEp?21wOS-})L9a}_q9I=Fhn;%gjVc&<%7_(# zPy46Kw{w8$djca+~kbh*(6|0tahhE{Ec~8YJSg>V4 zHVPq%V9Ze^Lm-l?XU}1*y9w1Z;2rH}6pyoYYlLWa@7KSl+GmZC#L^EaQ9hvjSGEv; zhf>tc+{gvs{D0Y<|9~{{zkK%sbe5LXg(5bz6J`3E0s)~h*a<=OJZ-TqZiMy}Ug!Vh~<<$=*_=y3+UC;2P%aK`>a3a4o*!2s(XZ<_P?x zFLe{wWf*RoBBfDQFwL3N8ib9e(=%?UZZZu%t&GpPrLgpEiS1ay_eRwP+h>Do@LuW- z(f7z#uK|yV#|x26s$-GUoy~Gx%hzC+oNt*sD)py6cdj^Ny(LNgnt3WeN?2*iwvO44TlWDX0xXbbgbBclJ>QoM#YJ!|1eR>1F-GD9O*zaZa z@6+>0ScC&?e^gnW{%DtGG-qC zANaSW!GzI=x@Q;VW|hBwO{~(8Nor#!b)skzrLZAsixtLNt6EV=em z0UNc{d5qECIBsfRD(JdB%I4(@p2!%yh)8b0hpNyHdqk6UhNDAwLD3NsuIK7VvU6oz z^>PeeqE%17YTyON36Ph4n+9eXKmG3B%{_zt|o74An^LOesvep@aY}0uXyTYoC+HW;E#-*q%>?~ZB;W(b5*imAxlw7^QN{j9|O_{zJMDc$Z zd#CV9*R%~VuGmh+ww+XL+eyW?jf!pCNyT=>wrx8Vd)97E@BMdA&%wJ6*7Y4dydML1 zSf2XRbk|WUpvzESq6Ql-6!pcPwp-V{Kj0v1dqgx zBoRe2MDLToFzD!Ui@1R|jJZfZtaOuCqO4aRB*iZIf z_0B1ErUZ*A3zl0WT&pfK*+96zI`~t4&m>AAGa8d=cuxQ z9|*I*wDHB%Rsx5$HwmWso+>2K<|k9Wq6&V#m${tp_x`nA3Zq7XO-y|1c?pj_;4d|+@W5+3jO{xX@FPRVEPJ_*xedr;kwGGEYI*m!W@1f_m!*HCP?qMFy~jzcz(xXrP|g_BZ! zSLl&>^!R0TuqRqGNQ7h|SbFyYoW~`@zAgd>ua8+=`)jWinedif@yQGSxAciRU=k)7 zEl#sGbKsU<=J>O=A}MrVE`knr1Bbv#KB%m^C=OW5_l_>tRfqWk0b;xdSb3M3{?Am& zIeNs;0GQuiz@mu%frI}gAZiZ!b^ssx&zUxJB@}?1fcnFYL1Q9z@DEUv1m2_DB7p=D ze&MH<56s!W9Ge@=s3AddwY6* zhVh#dAK{2pfJ!l9(QklzZyBFJv#A=L*G{n03C3q%wL>vnZznYb0?aKMof}vvi;|PA zaFXscMFNSYaZGe#@m(^|4F15ZH*F1bWFAU%hjJw)Nv$E!SIFf}nb#}13$RXg_vfv} zSS>F$Y4j3}T3lBHduugXls4DUA=nm_Y>HDcjE*_vb{xsnrLn;B?xE4348V;j8qaJi z*J_HJoi`|xXQ0)}%H9YiGB{Hqwq0HV-|fL|{>V^ig28U1-^O8|D-^1#Rza0#YU9;6 zO9R)cH@D&7XxMwX_d{LaN^cnLe{exRCbSW{4zR|0hSleSWY!opIcovp+)cNxI8_*9 zqE%CNv@=^(Nnp?$kZMKS$j6+uTOUt$cupUh!mCs!32qayXfv}))Zp8XFG7QE6On9- z%&#|Wh|^hp$O}klA!nGL#v^aNEskrxhfwW7f~DdJH-wJf3k-m1p+I~Vs$sh-A&At6 zdPJky@ehDZM&s#+44Pb7W9TfU)d-#_xu|MA8nJBGRecrjG2}ryUcc@)|@JP`@y?e>c(MIF50?{&R^=&Q?bp+JLB?X*Ky3N zsB-!%Z{PbJ@WdUWQms;T*x5#{@^7QH?LDtKw@^`}x^H!2f@g?CZ}1l?EVR`H&z2M& z5PB4K!O7Ft*qVHPzJxDci11;T!1r!(fGH6xdRn4R5wT+wudVIs+{!m3hZX&omTcZP z$PvT0+=#3DYa`7Q%d*_g@ha#6B@@JZvTm03TSpdO0^v)X zPr+TqXL>)o>3lK&#)rgE)iPzA)O~M02ZNW1gc9#L4;Au5fS|B%5D!OOIY>5ZPFG5C zgv#bNJGykP#AnJ}qUT-rphAfMC%8^iHYJ@y(uqku!{utcN_1LbqeguD0{y~=>rcsy z021|bBjpif3b7O?Av~RFOlDh(8;d>HhY&raazbGObe<1ExLvlN=%+_eQy7_Sk(TzL<`aB>)l1ilnlYPO~>AOJq!H zEgLUn0+jr5hj4>0`JdqFE1~%8cpl3-hr98Zo#@2xuKHIUPL#?d=y9krfjDeag`6UVaYCxrOFYj#~dFZO&>+WT1#ZYv~DnL)^Q(Gs1h zFMjIsfVOpL64oxlOz%BJYcgvIJA=)!LGh8#NLq%^*Q~-%4Q-Cgia0YjFNlC^(g{M( zY1=#1ePIdCF%k+I01&a|sK7dby9_SOh%%A!!Orfn&tO+w4nxqrt&p{h2CVbGjje z?Cgv_NSj~lpps3fO@8UyXOS6E+Z@ZHaP~+FkK1m9%+%Jd;$m3b;ag^~*s9jf>KQR=AJ9cG5;6R=t#VJ~Fr8*( zbtunk;BmJ)T4j6a1^+v$DZ_rGd^ANRC>+P0O8+O34p;5;@pm+9=ghGdWQTb}22HJ~ zDYIw9;1dzZzE1s?sNP!M247gDGLz7L;DF6+L`DI*{8h;XMBFQ+4Ta0&Wjq!k_sEO9 zgw@K%5WYV{ut9@a#|wa9mjVXB{zv{w!p244%G^l6!PME>*v9Gq2>0KoP3F*{jTO4A zl;u$cRu$Ee`1lda8Wxk#Byhy1Lrjzg+ln_x+m-S^P(J;-o(O~qW{wLGKFFh7j}@s< zfvsnm@gFytO^-IY`Fy-TAikqI5;_sgM!E=-_6J~Eh<@#x!1Tu^YC-Qeg9O(8AfWtQtmCY$mbzfa3AARAA&%PEIyv*s^7>X7LlO)k)6#mr%& zl!}eR&4cZP=k%~GgEd(y0@Bf<&C$pTq+tES9UHRWHQr*W(w+ zf=AU7E&z^Q3N&ajrDs2}TF(l&qhvw4gE~rT6E?5Q8~H{`mA1fSER{;Ci^BgXClU7b zrdDE(!YGLFIGj6PlPF87{iz3CoP-kG=}=_x50GR9qJfnc5e2o48yK&{Pjxv-dvW$4J`&Ed_wG!sJfyMH8u+@Z7tpS_DEywA)~n4JkHLrPgg{xWVVg(-2E?Dn6PmS za<)IW)5=nK9aBtgi0W-&M#ZKKw;5scco5gLCzewP`7$dQZpHw zg4p7<;zG9mtV4#eOESX%M+pu9IREK%{hyq}-*iY05W(pGQGof25_!u3;$8*B%z@UY zVzeaapa@|~Y}ArC#HC;B);ICDvX{KOB~LJtXbMljK8bE5iA~EJX%o^ecvEY*-9U{* zbrkKTEv3k(=}MHleoC|`jSnSY=_phQ!sf(@P{JLCC7s9GawUclvG@|C6REKmPN0G| z@xt1oz#_d2(rlaAO2k3{(tCJ8x?5IMK6o`^)JCFf$qrGK#%29DYKS0+k2}m88__mxh`{LZjM)Zp`O3DOFhS zmnY&VOk_!=@$ocAlf(6#B^*EWa4hv<&S7o}L`lv5Sp;4r%d-3fBjH?1LO$g!<&4}FMMt}pjn$7-^| z1*>IVs~zAJC^pl4bdqi)bcWEBvRlJamck7Py?oG!{Y+!~;bk6o?V>;n?n(6!r<6y8 z&rj$F8Zejrnj3|M;Syuvy|uV(7~?L;wrwdBV~^d!hg~dBQcBrHNJJW_Rm#r~B0;8};B!qZQ3Y{-o{v7B0_`<4;9{k*kJ} z$MB#}3ReayS9pS9fu8gRdv|C;|C&1$5>lICaR?+VVB`bS*tZ+7Jo0U!5(7u1qf!Qe zr;LP`eWL?!B=aRxXDSq76OsOqiQoNiLhydnb&ddUBt*@#HR@Rba@}wwQ{Y zz)*fPNsm=GSV>i%aJTo>G8J+xXh2!O6?nYIY@SVuNELlf_2s2pIoOQ#rG2YVEX(Lo zgP2T;2kl&PfM;c9l09TYz?KnMu)g<~E;Xj;=pqh9z#M2a+QPJ&k-QwrK90}YIU}L_ zkMs$@jPv{+E=5+{)TBH`*BFHrwDaK<)%LwQwp$X*YP zAKMX`O;!>`9yaA>2ajh-7M0pt)CZRL1(KCExT<5Pd`n|{tNV~HQ8y=gAT^)*H$tMO zd_jf56#0V?1nU0LP(=x+sDje8`N-*tuMOoSd4v@#O+GMnwX0&s5SW`0hKD~ilpa}& z!hjPt=kbuf!I@rsYj8!k{WC*f6n|<>0vHJbVCsL`wDfO^__ykvfR%%>zLC4Uv!mJn z;Uf3F?b0_02nZnvA36vV2s{@ET^ERNq7eSO^ZDcbnKkqIyW?*i@^ToW5Tf*=5V}Wi z9lP=U9q~2&nX}4^d(rH_$sAwljf}q6kBpDe)EXLnUz~;o{Mt{-NgU{J8*LjHF-T2L zEznERQjJK<&dX6w&dXJb%hFN+V{295$Eqi$o~0@!rl%@m$!liDrSC!~7#U6X7ce%E zFqdFfp&xsZ<(^?-Vqz6ys$;7Av*5%I^>@P?F$QD;leGF11$_So3i@-=1!6D?_B+sD zYs1CAcsu1z%PNVBol4~8S0wP~CurzVo9dZXNdLcRclf$Mb=5)lP_dnP&lf(ZC) z_otg|u}N}^EptfD^vZNign8wIjB76x~g^4 zYl1$dJj}dA@m?y$s7SF2lhQx=Hkg0ERO?R4=c0xGtoyqzL179^Hh(is>j2Y{X~(Kn zE8?IX>=#DlKB|i79Yj$>j!0q<%cD$iat$ZRlz0;j9eY!9uWI#DK9XQYS?*DeL5@j| z9*&V|%stqwI0NxPFvEDpAO(LNW#u{r<@j((2ocgxIx0pIApcYaDtENbW96T^Dg<~m zjGp$Bj+P)lK6#4xUjvjndT-BhF?n)6j&O#+nlOvwImad~&AF{9#un?NL}+C1no_8rGW5GP)_cnZdh4honBZ!SR^M;%Cfa(j7a9hV^z+yD`YX# zS`Q|-X38*|wb3Qj-HNhRYKSft8b@17vX$osb`VajIs{89+!fI>ih@)1W$x3mD>)RW ztvzdH^kv;iP8y9Lx~cWyKrKs)Ig#ROAnEntb2KCNVBaZdOF7_H&V%@L__@7Py`a*{ zJz021%Tv4}<|#Q_lUu{ENU!3V1+siK`(jQqO=0e(@WAvSa>Do^@|<;!>+f1C26|

jj!xHI zq~6U_j%wNBpMh$-!*(g(5#iY)cLR*KLpfmT_M;hfYX|h3;#=UHXQ>MEg9wiUP*_g#h!;^#9)uphGBlQVRj^V;R$Hm3f z`4HpaHMN7O9~==se*5ve<7lJ)D%-Kn_wyF8Be~aY-yi169Z9S=3XMHpH3Ye(2|*ojDuYzkfWZ8jPt4t($S_Yb$7fh_0UUi0QRUUTw@YJdEDNo)FAto z7!4*fIb38iMuo=yP(_&Is$Id(Pz)ILXu9B`_64jFxxr0mCL zEsqA1TNW(y?eGduwh{GGKlGhL!w=6jh3&d&{B}(7j0pbj0l5~UV?~Q4}=N1k-CM5MSMdD91&C!pU?Kl<5 zjV)Sq><0XLv1Wk|2qJRpn;X32wlpA519oTxl;=*Mdz9FBu+p6F&v&rs$AVnz43|lr z%8TVBFLzdH`T_+HQ;VNMUIRe8fkxsEicB<|q@1*z#GKTe!)oqOsT`_amY| z72@^=e!Q=}t)70?M#aoOX4C16hT~kDa%eJzU+4sn}9cqPFX>qGKB zZ)F*duAG{$VWAW@6MPem{^3XMIym&C+aOp5tS@d_2a+pN_LH7T%5qxy)JNXN;?a7JzEdOA)XT4Jzi zhJ1J(wjsD9HCmP-Lz7{V4HhsZDAZy;R%2ac`OEI4p^V3>1r(8P#yR>QWzdN}gWIdj z`!}n397wWqU^+u{Nbf83O_-6t#e1ZcKbdX<%iTpdujtfLFC;H?I+RE098ex<)`yyc zzT&C3L>KjYrlyLKxzH@D7qDjujt#)|ZE|qo-M#7;4l#??4P8I@UZat=WuvE)*s8U> z_C9Xnv23vWG|pU~!EMo|{LDUTcI!N3bAR{CQ_e_^%N za7LK1Qvg&sU>ja?jfCqhX4OF)m-!(y7-*u2BG{avo*_04YNTC)@_7azvZRXWzh`&~ zjVHPMBD}{Mk@RDy1N}w^tN#YB&HDvCwirSEW1__dHtqw|1I6-anj~nZRsJP>-gVp; zoACNC+sE`u&W^eOUTO>Q{)e~vV~W)AFQF-KbSFHp0BX>bPCJ{r@63sl8JWjSZtnDo zB(nwzmPUOYbMj)l8*q{^p~clMT7RrvxaX8B;O#kg0w3pT3d8%y)WtyO6^l^0e+^e2 z6P_R^29bd}(9dt!84cvwgtGP3<|sKjmmPjAiZ|gOZsaEH!6lPI%Jn14Lm|aU4x1ki ztuIQJ{z_EK`cPrwNNmXa!I)Qjh-JzMsN~CSpnuhZBVgI-t>?BK@wdhG_F2}blXxHV zTk7_dTpl=GhHfT5U z(3AT3%(t5vSPy%*7r5PM9y0UO4>LD2q%2j;NohEOw2&oWS52wRc}Zr;!8t7z4niC- zy&;8eNYsmDQiZ80Y1@rReIp#28McdoGC9+9E3mW5jur>RA}z2kRHQf)G+&*S%leIi zun}JMrQJSazSFa&k9O>ubIu7%wg2whErPqReX&-fF-6pR`b=%M_k!0mw1XZq z{noQb+hn(k`4?bRvDh*66F^&IfED^*^cA$VwKCQRB+={~{!HJ52^&CXR?rlQIcVOr z@2H%=Dix84-vV3)K_McMLKzD`Df))Yg%(RCH_C*xBwjciH3>Qo5JFH(_V+jfEp*+d zXzxkDg+QBYo&CPd;>EpCOutkHiE%B_vyG8ZW zB8UB9@TuDF#ylrF#vErgau9^IFSV_qbR!;UV-)sO`L!`5?oz1EuU4l%osxE zNz9tt5+YC4>`czH(z@`l^)}SlCMJD3BGSYK)BIv{Nd6jcy4`28r4d|<4@kbbp3%dkNNtg`m?O^2DZlGY2>DtU-VW+T^jhU|T0=nLYe2KCDLSKQ6~ZRTqS?MgeGhMaz=ZAPFUsYr z`S!jAh%OfZ(IxSJM8!_@e?JeWXl(M277*nvC1eHE54v2KvGQE=^s)mvev5_k5)(p2 zg-~X*$d}#4KGKpSkCPr6dot6}qmR;JM_^+`hYIhO_Q!R{uG=ifU-f*x z-mmcAzuIG|F`Nfr5F6Z#ToZ5iQUoy3Hj}I;6AC0Tsxrn%rN-_I;_mQL?o>sZz{2Es zB43Ufc@gXp;z^_$*c~aJ1mMvhF1GY^=ff(}8Jk;jnWlyo!Eicf;Ha`ymSHjfG+~k2 zdycZo{<+dI5amoeA;X%!ZKBYO-YVi}{UhBxJXUZsb1WT%QmFeUY}DL}*J8T<^=tl7 z>~nO{B&{ehw<_Ftj*(p;3|4-=l^wPAp84Z*SXn}(6k2)0x6jX5>u&-C=U{rKp)aRQ zb1lYpRl!6OGW5TPCzBbSzvbAGWE#-P$%9>H%APHRhuJC$um__y6bSXGI7z0eUSl#N zXH&R^#|CHwVnNVJ>-noOgoFl~LT3~APy6bNxAK6~BdSaXS;A&ztTq*&YWfJlWvwgs zGe8Fiq#PR_x7N@p3^+qoqxLgQZI}7eR#^XT7+~;fg^0f^eYS7gC~DTy4$b!wlOvYX zE~^(xaWJwYO;OqjT0VQ`=o>DY!FkTeFV29y=vrR+UHnF}4=pe@64s>RLPZ)NaGoJl9TY2Z?_C z&5st5$M2oFzi&FCtGpm(Q#$ZVSMNiahZiXO+1Itl(IK7VA^v+em}rM|>0Gd(Z`hJ% zULgnFjD^c)P?N~oud!o;HOu&YIsr_-Vv2R#)UQF;#msa$g~wv*%96Zzt}F+M-P8gp z-9i_6RYxxS34+U4Jjdq!7TMgS%+`YTpCCitVbZCbciI2!d`6V1gK7c#(E!j7%KxGt ze`{2AQZ#l1G@2_JJN+YGRMz^VSM?(q-9V}(R9pBaptJ!Jtiv)6%j`!&EiF8)_jOTy zA)q;)jCIA~hW(icfn)DS*K0w4f?}?AAXD1JisOUV(d)xGneWHP8~AthTLtYtbufe* zihv=^m|1Fcfv+eEJUxX0l;CsFEWyCUTI%9BWd;tMo#`X!^*_@@yo3C)V09#xZb)5? z{MT5KKpq~kxDIKC;%{N|)H}KnW!p%5rghMnMo61C9Fo;2rtu?6*q_LjtS%E9Gw*0h zGQZ1H+DeE!@|t?1-DCHoEHuj~B%#+*rV{E(j;5XE_aaz;=kn;)5BRUOPQFK;g`v5Q zRi`}lPhUdmi3h|xU&zEcU1$#IhYrmaYt66)k<%tvl5WlRH_a<@3T-7*_g1ekksI}v zj$N-;Ce{R@k`fUP*?IMX>((@A(%!TcypTU2Mq}LsgxlOvq^jJYPNDLs-GI=mbO`1I zEKzN4R|W)zb9NYFS_&WuUD-b-FR=wmHGR#~h(_HswSm@SPA`{c6?)Tb zJoSiwf?umV$v*NByWkVckce8WkX(RAE#`?>EFJTS>xjJe)FHUc;lG1GO-&@7+hZmA_;nZVc!k_aHojSZn%{omJdPhig>y>?HOollWvC%9f(E9Fx6_l-A&Z81?b{9LIkya=3F>(PR3F?O3W@lcO<`kD z7dR&-Br+wF_==DK+e!x9JcCW6Uz<*f9cmt905{w^q~IM;XS>;&h#A6;PQz8+-s1=r z%Ox@fcS^+0Cb;Dzy*kdokKpJkkFjYpX%xnfzqF)0tOEN+0pz_J@cx_JOZ;;SoVc;o zzZwLsqqyOF84$s9ghDAPT0r5H(OEHo!t?}0nrl;r<0N7=h$cmQfH_cm<0#xGSZrCY zrf$K%`^Smq<)M+GjYZ=-gjO26##At5SzCEB;)>lQN4C2sz?)pOAo)IgO}S0zeImyz zgYCRiTd9jB&O-~K9#Z~UMvPKgHr`dSBAJ&Po2Qpd4I2v@)|cJ=mvD>l^cW=)0Cdm+ zr5}?2qRSFSfGUBxiTOW)qqV}=-_+U8VNDULd9{HJWl zFb*ZDB&Mo40*~?t`j`WF&%moXBrznTW$Jh`iF38uz3LND_mc;U?nGA*myX=j)>s*OKtIcDh$BjcZVQPks*l?_wTl*<}fpFu``D_ zdb)apZj&D}G2Br8(D%BWGazXaWfnIa?{5i%67-ls@NN_qUv2dcs6|#uWw?|?50A(u zUqKm->4ig51xXGBZ9f?o@wa5PfPLLsT`o34^6RqK04>&Z18>BTg_GNO5s>tUAL7 z5vTs!5BqS`tf?d4e%VuC`jPVo?(b+Se_wuG z5r~GpIvq|CC_j*X7R1)~Wd8bQTS&iq>1q%^XkQ5D{Msqlr(b$%S^-rL>JjpGF2tOF5@Ry)jbLCh}Y_ zlZnbnGg>T8VVpH5);$$<_b=60#jkE4VE~4V0~qq3GgnUZ|Cb^Ee#S?kT^10KXGSEX zf`zCFLLzRWx}!n_n(?y~3JCK12PBel#S?Sw^BCnIc|qg*nNjcu5GY&&d!ZW8v!RAU zSecu4I__m{cz(KhgVsbZkOVU%`GsY|&jF?@RYN-99mx)5Xc%Cd&t`vyI$m;z`!8Vb zUU(!>PuAE^A2(e{?NF_3yUZmtRYk~3xddKNpNNwNnW~k#o&MI9YgsGYjKeK+!D@o` zO<4+82|m_sroDrSAl`AD6&bJ^8st5v$ zM-SFdQ*VeBSnWklrj<4b%6pp&Y*BouU9%1?z6v+I{M{tq3)dc{b1h}V-7!}eGf5}> zqpAI2i86pTQ~MF*@wFkd*QZu8Te?k~K#-mj&&Vj90Ydagr2N;SYK?N7z=9%-8pRK+ zIK^kILE<|hbVfY9`Qe6+JFq>%Lh{&;py>4wr^5{0BDVBNM%rH@l0x`1IH@z-!cp`C zBz1WPq{`Ak;r4L;REuIf454y=Ry6O>gI#I>Px}EzmFd5@NG0<>s_FkN(^9FVV~Y&f zfvkkyv=L2ESz20Z#+QuHZvF0tI2Wi~p7C9VDt~8yh9TZM89Zg9_yg6Kf6Hy=3(Ggm z9KgjdZd6!=!0)>}>lv+%hvRA7ratckpCuvka>bA_GgOjqaBPx^0HIwNdxMimS{JeK4m}8R- zJOb=2HhZuV%6thX^^$`W>O}x)-dV$4^3r6mOvCOwC6QZs2r}h_t^N!$BWqyUrGb@; zm@}X1kezPaSX6Jw&v2mRqTKg8wf)EmEU@*0!wVvI;1{qUoVj3HO^TAZ7lYC(odMH}(wM9uff z&o?E1-&P4+a))$$h>JEWc5h9-R+uB;i>Ku$X*EO=##Myg)n;N=xR%L>a|+ z^_K!33P51>0py#=QOSvVl6TqLmE~Q3HNl{_g`Wd7DbE@!ROCmqSTYnGO^e$O^3zAn z9ZE#8Q7{E8I-_&iRTPy}SB}f(oRdZNtAvE`zBYUMuT~5YkzZ#6w5MZ%+Q2hN!+IY6 zDcVrWiw;Ctnpy-}InU^a{VYER9W1>i({GD3RINF6P#_+p_wzE;1q6n=x}WoN5VHA{ zolt_qjixaX3be3V!t)E1%V-x+a58**+X-?@tBk%X(a7Ts$2ld!zD&OF`FV^~0AHTp z@g;NYm+iYLV{K|bpAQZ&Ta05;1Y~iR|85^tuiGL#Pk>^2=mA@$E{-?~YhUO2JNg_!$*C3#_3+AF_>^8_wA5dl;BcIpGg#hl}Fyits7S zyMP0jm^GaBDyZ~BTroF|47H#6qlj8{U&r)JLP?ZxU?mr6QOqg?FY8|WN7_8mE;Uq^ z^i!qKUDkPTYp=a5LoyD^jl)a2MuM)O^i3SB0W(VxJHipfSL-*zX@Xt`*$9LuKwoI~ zXp;>lnz-G@5wq#sx}E9<8f-39tY%v4b377^b=sSbD#E@*@NU3mXH;3jT<&M9c4)0r zuJvYI(zsns7mqgu&(=!{6?v0T5?+4FnmS<=ot0JvOS<>tJJug+HKpC~d{lf_dG}+j zwYlJ0uXgX>#Sbz*@kRCGjJ@`G9E9J)B^}bYVqqKk3Z1)}GjbRj^?G8tdjVp8_OUKdqHE!9QCC^E`HC-+=o6GxA5j5xP zk5ec?)srQYE!2>;3!IH;FqNWIt&v0rC-pp6rNy8CG9iIcl6Ol6_-BkS3AoXNk7jI= z$#GG|dVcTdK&4^3%6${k{jmhxtXf0U*4RWM##QhKYHY>hUsN+9=8<&d4?$5##jOZU zDCqO(N=22^Tk__SK5@gi8uaXfv2Ko@xUKv3^q;Eo-k^AYgA7G3OGVDxd;Nm)p$5`p zM$N~qG^{}Th7$Pcg-Ev{oady-Amj}{73yNdIvb1)CXDbKd?X)tM4H%xVx?a|SuI@@ zKAus~hnNmS_FU{CV<4mZ#Kc-`7(Nb{>)P87!SfYWgzcs{3;LX864^x5z%Kewq7Wq7 z)qH*!elDyF^!+dOf@{t8feJu6tR4vni2T1{1bZuCz(O1m8&h+ee;kdSP&;#&Q$_qh zRLC(62GuFBxZAnY+VNRX|b2z%H6d8;f zt<2I>wB3B2v~{ILZK?sKMA28I?Ti`N2`=hxlyq(M?JVj8sBef{@)|3%lAR-K&>=eVksXuc?jntnCNlgjUlb9* z1;5fUjM-=3$p!Aq`mtT5peZTAMmUM-w4p)(&*_Zw@w#|MjKU=wG-Gel)k&%}gS{c1 z0_~06gE|VZ-PIM9moagoOK7VtqIn8&SfoqR=Bdz#l~RimW|BgS=0zAaCUlMEo2ZX> z6qgz)RymF{;X(MSQe%dzu%7h$zS46}IWh6P^1a*~)yKH{Ld=;|H_VG zb|>qKZswJe8cmlqf4+5mqp)t2_{QV&ZPM}xK2lzSrIewZ#ug*!XPytNi)7oEv4|rb6#0Mw zh4Qz$Xf0&HRMhK1HjerUhIzW0bQm*G;cBuS7_X6n+oZs1&QqF0Fi6U8Pr(q>L|q4- zHZUBs_Yxgtx9S5IwGuXZ;i>Z(pDy)#wj1!!YjzITSjO_N&7agxy{5~^hsL6dlG85mJJRw{G1 zE{>iZ8;aEITFXw+^Nc3wb7|VAicDRA{)E}Fn}KKHmpZ@6qxvsk8n@VzarwNmw3b5& zn-4X20(#wD9+p1QSo0R`pd63$L{IZV(Jb1!(PYW0OgGw!lYn6iM;H*75lZCI+OK+Y zqRe+Qc`BL%eNu4m7Hd^7H%YW3nn%LUo;ou>X=Ju2J)ceaQ%VC%QSWaz^RZoW^%=RE zNlKJj&j*BgoMmMPz|RpzY*Jt94XC4<3d4{~8pxeCWddP{H;M1uIe>xbXMO!JC|{ys zd&ho$orsyHfkTM$J1P5uSd6Ft{Q#KfSFhAuPmUdjJ^rpYtab0NuaeIu1 z@AzI1@0h_*$7nB}BkKw;_y6cj{p;|aa?V^3kt&G4hUG#$5c6Ljv`fJKm;ie!m3#9ofi6^fDFmQ}=-yK9#em>Bw$KBGY|n)G}fICoj1s zKH;e*wjKS9W$%t#0IcV+Ps^i3_Im%OrxrzjXfKd&H5Y*^1#Zj9)w55Gsnr(Q!)@LY zvr9J?d{bdpvhv+C{%$Tamc_`AuEmGkV{C-hL7yd#ZES=k`z8RYI{A(8XJme#=0ds) zK;{Dg^!ML#!AgLS7?0?0Y0ihQ;~8Y)|J#5+92Yr7GbzT!S4$&UJ4bYv~?$ zIJh_dNn>bVYmHkboMsrTwsH0*L)3}%!#N&8c`RA!yQr-!v=xJQ2gS(sd zrUQ54;l70{*M_c+uPTMD0)gRjX32X==8pCIh|EA|f+!eC=pADPTJr}tdWaytak3a0 zwaHAoYLlNaa=CCabfN;&-h187{1|klE{ito@&N65YfbHWFJ&@YGM3933xZ6!aZRiU z75(e|&Rp6ZTwjk0!x(vjGETq|=z({oQYR*ZQ2iQ?2KSw%)?H498LHs0B`TBTS>j@_ z2$xQ*$Evl`I?>vVkDvRLshJzSiJ8Y2X7w!!_=v*@l~GMvtj!+-tp5-b#SN)7dDPDhGFEf!H47b`_m5+3Mg2Sk5e5;|iy!hd%jKx}BvG}( z22|FXO`jG&8A9rIZa+$E9p}C&Bc8)$_Km!IXF6WJzh0dn_<{Up3{_XH2~dGzMqWfM zRX@@f8f*~E$oLwU5V!xF5J(%mZ2r7`T<)Q0&S0f_{*X1}%H?mVHR!y=npy3y(43yF z)am;6((E~8BriBPaAMy%PH-@ZKxKM>Ep+KX^q{=;+e0=*UuurrFf`=CG9**b;67Mi z?B}|s_t-VfFXQ=u=(A>NZ>HP$v-Tc5p+mSZ!}rYSg2N^+>SO0&h3O~!RwGYhRr^Tg z2T#``DCtLOG)f;FW~y&OhU&g41-qb5Xtz%s&~xIY;zT{MZW};C^AD@v?zu`%3@21F znxk>2y9aJK5qN*Dv2Jm-FO?tq_-BVm+Da}@IXL<`#2?Lr;0L=18!<+(45yXD8h0fE zFUtw*5_ln;Vami~=^r9NDcrF1ztIcp5x@%|eIpTpmlVoHi2c1L6_fgc1X4$62F!#` z3BTY{2HMrdBV;u$*-O0trN37YsgDTnT_%Ykm)bfMwBHipyAUs%qV~&QjD;w&w0JiF z6-ol+0I(1LKg?-3(d!#KIT#z$3y2698U0-Z{@(_E0a*xm2w6G^bO<^Z2pbm&JW&X? zjPZ_fWNuLibU6A627-gkocPhn?y|(uto^-Di>{@6Rx4xqt*Zr=KL8UR|BM4fF6{}@jDJ28S{0W~LifZ41~Nlmrr zL|t!H#L@YxqZVjD-yn`bCD2Amh}#P+Bk4eOVR#~Hw*&elzbo9+7G9sa<8_N~a`^e= z<@v=fIv5Uv^^38U=5n1pxiY;|#RTb21!o6*g4lh3*^GW}ahhre1J(Bgrk^gTMV;K|6?Pin;%VNl+U_s1Dyn96cM-ZdsVb8)JUjc5KV zkS!VNNLm1h#egQn_^-kAWA^K3R*8EAyv_nflEy?J8EC^$Jt`*s!ng5_Ct`|X(Qsfw z?TztM5~%B`3UM!ldm@pXd_$1{i`TT3>aB&550jhu#C`^^gRLO28WA$XfHRPa@dX)K zvIGT6YBPt>c&n@vX*ZHv_PRuEQ}DkB4hLY*T%An?BLpBLH(K z#l&_eMcC}H5DraC^+2s^ofHu1ZoAUEZl&E+k7OS9%Sw_sdInvW#32oovhKvy>%49r z2jy21Xp5S2We~p?lawZ+d(ZvBjz@Z+YN95SV_us(n>9vJt4N$D_4o~NG_wDr`)>Q=zDFwZB6cdGWjwT!1zz226CA<|lw*@A5yYyJJ7dIY z@QsNX#kexsz#yS6<#xY^4r!_GVq{|(I(G7gz|Awuf#?#uUlf&x0=%m%Q>4NadG&I$ z60b;!)G2{6L8&H2G;!=Czm2P(OR1g4FCPOe7eEuQDPuUg%R&nBk|D%)JNyuumwD3p z+%4o@MK)ZrX!!_y|Cfy6DQEM5vm@r-kg;XC zwOXaLzrl>63qL}YQ9?UxXE$z9jIU;z@M37AbwwNXqO~W-#wutI{*b!H`E+6Dx2>P$ zL5iA)x#+N(C}Wh9sMfaEWQM!~yQpF5t5KN^(D5b>9G^lJK<>R6gr_565zH^BUhFdX z?a7E+pnN$0@-aJkM>Y%HdHKvEaU~D;Wxsp;L}5VEiT=!|%}1o2P}}I*nq=MI-jnSN zOWN4Zok(`&4edtlXc9hsOXICsd0*kQ=qgRb39Go4jy1+gZlQio*xdO1-pN5|9}K%$ z2ai=8WMq_^*e>?AS5Uxf(yzRJcWTFaY_Iq^0AnWrkqQ!%k_G){Se2e zc1i_~51HvK^^uANT%d;UeIc8_7>vv4SAisI&?t+bWm~x4)DQP^f(q(E67GlolSdSt z_yCkMMhf3zFg7a-6aXx*HnN=-7P3D^RFHI=d%IRK#T0uAWQ$>Ax(nDky`^Yd9z;gR_(~j>f!-4TD@Vm+EdMO8}0FM(Xq@ zr0)sA{RdQE;-g2WIU#yyP{?Y~MaU4V@p70H`D>a^ZG8PN@;FoGwj`e|phS%9RTPy> z2end6yqF)9mf(EpH=(XC7uKHtODR(!K8irOm5eR!S}Ms{w0-1rn$3(^N@D>;TIC4!9>KEt7%L22#qr|mDr2&nZ#vA6TmnwKRYurK z6ALfJXAc@uf)qv@g7Sk8pCM^FtklaV{hw{k4dxKkJMXqBX|%Zoz<)xh10nK1{p(bp zIsK@w|GzC;>_4__3EA24X(>3$s%bI>%D?@QYOONYJ|hSvj1c(!8(8T3tNZ&a+nYET zSla{Nn>P7BH*LXqxGcZcw9Tf-bg2P-h67-I{x_EU`-S;!B=)ys&Qy7A{AsXiS5X%+~(;9-L!5CWI&o$6cg-1jh*a zj5}^W+~G9CBY~%FHHbZGGr3(T5$QY3S~r#SIgh$5d?IsR=NzwRgF$b2A;xP{H9!}c z@Ud;stC(EoLVmW#VgBNqu1@G{vaBXw#nFnfs)+fXLDeh|(7)F=l9Jbd8CU?I( zq3dk3aga68GUyFv{$Qs~Uso+O4O{Mw;H>rwyn;dIBqXHc(d-1=Ao;DQ8Dero9O2%0 zWj&ciu}tOV4+b=prY%A1CH>LjR3}#t;PRIrp${k*qzI=iZ*_=O@47m|COb+umkx9s znx0z3DTtzT_=SvP%rnSyB1+{*mq2WjkwNUv%OI^5c35od)@9K=#lUa#F{jz=`)hA4X8?T7}!kR4|)2)wbGw4G4_{Nh;<^ONib3a z{EQqc+unrK2d_gHkZ3kR<1UViXrjwfqj32t?_Cz_aK6zcpUpRsAsF5iXR2Hz@P*FA zJK)dw5LDR~m~;`cN~U&qnODr_cJL!0HuHek{Lfoc|CEPBpW4_NKC^WANhcD2ew7&K z)jShP;r%l53rJI!8m5a1cIv_dMdioPMM`>oV^P&6TT}CaOA4`qOJ#W?zlG<5uLvtE zwsRb>%X*f(!`TAcYcKC>6a#z(qsB0F-biaB^)%aGi-CM*UeH!U&~|LHN_f2 zhy9LcFGpW8n;lwevoYKqw1wP0Y?I76wR2_vZryY7*z{v{)TL`j3^xIHP-s+#GleSO zs8TvPnfaV|G={YotQ(X0)e@JET@*zLL56)~$_6LYnU zP8snO^DJ(EZAOjb!ev5f#nunYwjD8GncW2qbxyGd8LZOzN+s1!y@hYgB9^@a$nHtY z)iGe}UcT`ZZ_|kCB0g`#CX_Kw%`(d%rWz9uhk*nY6jE;%!AHPlQ@Z=gbJN!O z&^>WP-GR`ZA|}_*(bNLFZ!U;`{L0wYC*Za~c7wcK1}*085pabeWmv36C(N7Q_IO#a zt`jHLHc&-OEXer2TG(}fGpOkVHRKs$=LA-xtUQ(m%bZ^nA(ZbM%H_*3mt6Y&Trq*% z%Mjd~JJk_Fra)iu2ouZ&#|kdKL@OLk zixqy{6)shuX4oNQ1P6zTzyd|a08J2M0)Hgtn>Qqsrt-)O6MDrNt|-z#9CQaK)x?y@ zaR8^=b$tKIVnv@lb#Rb*YWYJ8`2WNvC+Q!t(MizP*-6Y!?d>xN(a8Nx9#OQCHrm zmJW`VfKYO`yZYZrMrr?uOq>=FC&@aUeFi=nx)!&x(O+>P%fI%qy;s60Wooj75*Ffe z2tqkLaX>LNQ9LJ;NqdXEj>53Yz zXBQ+(@FooRPPkagWhK1@uuudr`PW87%+lG?Wl2YS>ZEnVR;6EX(Oj1 zKp*k-#7f_Tpy4psSX_mC&=IuoW_&1lV`vbuOyMRnw2#tpaU0lJY+x9S?so4SEfmJL z*;fyM=3fb(xGQjX925bI2G}X-_mpdigd z*3tUBoRC;y>?&o!92F6IBp(SxZt?CG30Jsa7+5GphocbygU1+ylVjDU9?|H zE>Gp?(n0`)MuFddvm<`*=>Ooof27?1Yubih#c@kzM684)UebbDAc^3hB7_jw@uX@Y zpLYszAjOzX$4$?VbTp!u2SDD*3N}6WH93K!_5|9%e4_+pYX`=OgUL26etuhzc2AIdsPNatnDia6^LHqfG7X{ZGg!1IgFKA zJsW7R93C-|usMk{@&0haS{!0%c@V!{@A+h`iW&}W?NwNi@BV`mc>AwRG?kxe31+&J zAci4Dm%+cdK#E&pPz9VF<=8=?fpwxYb`z3tk&PQYvEstw%Kk&l<48Qzq zJpEC^BY$5)YTHsiN3V%4y!uMF1G}3cBtbdOR!Vvd4WNg+Qb~-ubGisuSYIH&meM9$ z7 zHINl2T%oU!m9kd|48268#+EH{^|6FS#hc8Gw3%jl%`=A$>KYVx>7B#TGr(!mIpZi1b5f??i3apn(`MX_H@uC)E2U+3Arg&^~03eV`J z^9Pi5+==G*Xb~s)qU@iO5L^b65X#-K^WB=?@Gvcx21W$;UhB)KHcY6it>G=d6n~*E zB&JYB=UVWIHOr#W_XT2>Ws2~NAQZZ|#>!lzd3>|O46^#+^QNtXqomX(uJnwJrmP5} zF_vH)e7Erz2701!NFcX-T0A%rUDI8}TKxO5#wj3$C&^7Njr4>}kk4MEd2_KB3C)+B ziD`MZ8CI+!O=cdjwIk$06C=@d3bw}~`Xkq6*sUTxNnN`lKa^{oU5OeUD`eJpHZXN~ zjOCxojt&-H&Dkn-eQa6c^uFm4`=eIe#s#wZm^>O+QbcUGC^YfP$^d%X7MM7v%{&&z zxla1iM1(pKZe(zQqy3etY#d~gzsGO?}FFS*6G;EN(| zjH}t$@p?ltNUf?d*$FV&c_elabTR8Z5ZCy%+bM3=L7~~%WP(A&bERwBq{C921p4UL zs%+=W&cl2PIR($tV%xqzgSxl^-J+5lEMD71o|RcAmZyFu=(nUIcqBcc;fVUxk}QfB zx7c*QY!!|t-hy9z7Cm88Qj9_$tqkhzoguEBHLIc#ATMeIDTM|2_a7TCS<5G;cM{4U zN%DW!eI@>5?TAm!#Q$D77=NoAGk>ofqQ6xRF7fV5SGN|pzt)ZEKh+IVplo_8RQtgI$=N6$yNa|Pz`9dy`I9F2~)W|2fa{l*XB%YH! z;^lahbl;@`#yji^Qor7cHEpv6$hR+a=hISxGFL}N<{b9snl?s8_4Qvv&j<+wJ!hoF z?Ty5>T~-I#&)ugk8+)eI69Z1TF)IO09j{~DOYPJoktix)M&Ei ztLP!Mc}0-WB^K}g{!_t8hVz#?3`qBBhjR6JGPx^?<4Jbwm+#r(S^^TDVOI>X;GtO+ z&F3B$^&bj~)G0Pxrw2aMmg^zHhMQ881&nuH>~oe9BasxCIQqR6NKtA8)hm^%xRp90 z;dGTlv}o0#WqeWRZlZ81L2umQcE1~JrvCf&z5tv7Z`T2 z%0Pqp^u*Q0^bGmp#GQzdlBUf zcL)HP@wLw$Ak2OzL*XIk$fx%XK?k8GeNS&BHpCLAv zZisz_aYgR|(RSFW!IHMwMl#?!LYqW|)OL}bmF3{Yzj3ma{aVlqQnPmj*vJ_1rW<3B zC`mbEm1&UQd7pr`7hOCsTU9FwnJ>f`whWHZ*QDxwQoAGEJkbXgsI6Vm=Zp^|&sx;q zx$Jtmh<5DMx1K$DACJQwsyCPnAJu#SrrLTwaJnnJ=x*jtxQP&w427BQym*$4$e2tp zyDL1(vtIYAjwj2e1amQNz$LNv)i$qx)JJ#s0uK61D7Q#H$gZ;s*zSn|?7YI}fht`a zr)C|zY3sP7G>ztm{fkJmnRJ{zN#dNd@?%VGPZzsEwbazCrpwfi(oB&*QyqF#mHZ;_B}ip>!uhQKdO8{RDlsPUN?%0 z;_|-i6Ctv1e|3MFXtoULtWwr)>BzUN^!AfQSIvTVv3dT56rKL>bOX#uKC;i2Xl12e z;lk_(pWJ``NHsTpSbaldgsgO#fSeIO{N|p=cr6Rg>WemsG^$M?R~2ByY$kiOPL1 zjd67|pU|@po%`lYG#%C2_rZ1Ku_nUBTQVXkbAv3h=ZK=-sJfR>D@IQ*?zDRjS&pr@ zD|l`vS^kxW%inID`9I=S|IEXa)3P*gd3cb9+Mfk1uN9xOV1R%luL1=8+`-8O^dAVg za3=@^`lb6PC`*0c0=RpGz_#*#?Rn*&{*)pjRdqEc&Ct9S=7pR^uOI^QNTq~_BpSx2 zybz>x@0UNC*2RW>t*s!XXUX@-Io*VRsoq;ef%&7}Bmr}IVxr1uN!wo4CUZ$$e;dd)pNDr0w%uwSs700l?J zKS(-qLH4C{T0NdX`=lKl?5p7;B~{iJxyiZ@hswAH1@tYsg5$zqAw74Kjx#kJ!cU7x z?uEqpymB6_YB@TR5Je=U<2`-&xe`P^TJwPOgt3WO>s164T}7t{m3>B7mr;SxS18Xl z-ZR}TvkTpNLBGIGtT zH%`7ImZRr+c)s~PBI$28l3)_Z=V8EQVEShSP?!>Q(-IG%S{Aw~DT^Wr$7jT+1|AbT zNDye=iw~>mfFSA-p_8N-kmao}pc2uWV2!GBBJ9WstBpG5DmD&@HjLQ3Kr`l{FGd7) z?@U>Hzo&1_wGu$shJ+@hfm`fqk0O*M!m(58rDELX@{3m0nX-pXaKTd}(lA7furW^d z(5Tc%S2{goblML({v2g($3!$Ow1PWd^F%&P!(Oyk&Vz#GXwk2Wm-(YZHVOL09dT(<0gm`K|N9PHRz)c_oC{EswvF?V63-&P3tZ*p*_u z7?@z}-;HY}8ubY>+DW-8-xWLcVPU68gs^ma6cfs=!Zig{ejT}*B=fu- zmtt-~)+=1H8!+=Dq;fSK(hjTK0jVFw20X$}sIgS6^H%71Nb;B!lI~B!Z@T#o)o=8* zvmdV>GF;?-LyUkKrl~`TW9akZugOZ)P;F_|0Qf&pzhW?(empELyVT-SJmc8 zi&3_QPm$&a&sCPk%U;8Om=qaR$f!>iI%_i-G(ht7&if{v(UR`M*Y6qj3@PplR#Tx@ zcS`@#;|m|{u-Q$+OVs@nq3PaxLrR2?44S3Epy{%1t!{|6k$Aq2Y(+)~4;tzRgbJNO zE2KFuaIPPK_yS7#{NiqqGpn>ZU;lL$CjvX5iU`P7TmO@${`Ru|cn|+`Q*5N-Z_5ti z3o<#1Ua5UHHpL;g;S|NG#L5jo#3pI+8)U_^lF9JxS3Ie5R8u^FzM?%WYP8QMYTS3< zYjY5Iy&wPqvPaAFlZ!G#Kz$I!(TgG(oa>gfHI&JA(RPzS79v+lo32Sk#NFFAc%x=S z{8mIOnM%1poO&`IQ<=Lg0XOm~g5AIv;b%%t@gno-0sfQ(V`eDH$lOaBkAio#PqPgV z_S2A3xs0=;gFpr$)Y=RL8@05mZ^qIF1F@PW+3ei~HA0QKwA_~qK+8v?JxTH2fo)E` zXr5xzSh_IX95qst?v!HC;!Z~&O}1QB;T~0MBt(;S_b4=p3T0bg6D`bt!nheMeJd)) zBjkU=(SP66Ppwdb7k{QA=+4+v=((ye69Dri@B`-m{+U1h1!|TbQ~_8l0nLiVpDNG0 zJM&S!B07;tOk1~Gt5F#Sk6zVdsb~p2(w#Oz9TzJ8O-2J2RwJBV zK>j=0VlxOz-9|oc2&j<|!w?LR+Mq&;!bDy(+B#lOI6;n*0xP3JyM&#(QJ#%ZyyMV1 zc`Cmbsmn+G_GFofaoLW{p>}C zW+GlD<}}{f&AdB3hWXC1bKNLJkd?v{%azzZa4IW%kP-QX)F$NXW}<;^R>C3v+GKw& zjW;tMN6a%>E}(ldP-iu;*(Wr*TWow_j3C9pT554OzOWXRYK_%kvhGfW7V&PVw*$Pb z>BDr0%r6;;OIJN7`g8mP+;darzOXWR;9acGnR0RG^Y!Dsyg{4OCZ{`dkp z0+rd))0&AusmC~?_x~K|M*k(yM>DyPznq>yl6LC(;7@thL33?hY<`_N8aFP^UZZc= z0NHx8m>w8q$_!9TAU+yV`u5B&aZ^$txo16e$uia9i%C<=lvKau3IRTssYSG!@+Mg0 zLwy}8&uI;2r=&mW4IznXYAoJvX|oGn)HhvcGK{j}V?!)fAF>QuDmk5y+5{?5bLw?J z91+v2GcRh)LAzS;Y60T%_SQ$$zSo^Mr7Yd#H5_jt`N%W-3P|jvLXgAKR-O7M)|3dC zI`ou?f*_svwnj#y@8GrGG61abA45U%=ArFY>*zdC+!Juki9YrC`Rwj zUMU%$LUGC+*s}@B_fFOg8?roZUF0=`5am;pNpqwxPfW^vh|c!rg-GO=-D@$ks81%A zeZqJJYwzmmV!Z?VYriAjNeuNJ@q=5$w^!?L5$DeEZ_DEYp{sih5aFNnPc43$-yQk% zno*mty8xP9sw%!*iCR5pN=!Md6r7FvPV|84UYBL{~OXgVPU61}c*LXad)_`=8}nS7Hh7FJdeOj~J4zKzBFTqdk(c;d=NZVv;Mev^Lm5fKhO_ja5v+7Gl z;((b3K>P5Q5oiCeh+6=N=lnpN0yg>GmZD1BJxb#m@4_;_@5_QX-$oMbc6cs2><^`m zRyCkP5LrK`HT)nx?;jWxUKIWi=RwesWdVglj{*FyfInuDC;&N~Vc!R&%8%1vxeGB< z>Nr2tQ$*BOlBlH;qBAVG90crFB-K=9z;}d7ahYXs(v&q>Ijt(8}^W z<3AlN+9Zplv8V4jN8xrXo!6;r;fC{RaElr|MB z!vUz{hrSYUUPAlaP3`F`F5;-DN>wEdpgwVn`dh;T0CijRG3v*j!Kc;y{@38AHB0f% ziWJ{bH`gl}i^888T)2Zer$809E`WFzK=OaBkUso@y7G^vr6@H`;_|>Z+NuIu+6ldR zQwF_&w%}q$>djgT%ELnHIMCah#7gbo0=*=xu_V*bY{WBzcc|(3=I{tEC#v*b6rbdq zfS|lkaYibORO3M7NaM^mTu0FQL~I2++UdeigNUT{nBzpyXI?(8cVKRHfdebI(`3l+ z@E$2~irWwbU5&$l)tNDQd& zvOCx>H1AX~WS^_x#O*L>5+=5*yJdA@ ztE|+qpkYD|&@tv;tH9AE(}g*3Rx!VPRvdplE0uQzQ&uF}9*@OiCzVBTobwTs>1o^T zl*Z;$bgX1#4Zsh+YBZP9e*fl<%3Nbj<|hI3k<`JqdEQ+$K{hc00 zIE)Do7_~I-v!Gs&O-n+lIM-7c)R9MXeWs5~%hbIrciWSQ9`Q*y)7^5d*&mB{rnD4a zC2U~Evm=dp8f=U9)@!DX^cwB?eRYQv369{0~nO*dLyKj#(fOX(iXv0h@V8H{hk1wP-#U-Hv;&`tHKV{&7iiB-sQrg%F}c zIr3_JI3v|aN`5?OpHF>g=gw4hD-YQj8E|TQzns@Nf1KLywBTpABp$1ZBZ{*uf=;Pg ze4&l46Vyo%hmo07lQtY6vuhFAcN9swz>V`kiEg4#%sK$LJ zS6O*tN#e9^%n%TbFM2+D$eT|Ct})%`==$`!j7iL>{`uZnG+qZ9N@5)Kh`3T8P)UH= z?U37Zb(Y(0%B79}OcT!ji@lA^s=i?zBgkGVBeEJknwEw>)H^t!F{fBwFP)JHs3U|x z9Wh3zK2q5m%X#$b24%y`1?Ajye3_JTH&@s#(9e#X+jGV#3ewiwBOx`as{ZU|@wO0Zn3cqVFd&Pz4{XkMhIw<&?ZF9`P z`ug)Kmm!ECgBa6~VkL}-x-4})&*;VJU&c0i;$kv&jd#ZQgF^huBou1oRwu6F7QHZ;^uvls z0kb>I$UM12nzd}$il~5ty6v$3)pYzbY5o(5_}S7v{z2PjhjW%>(CdIa^u5ySeb*;6 z|Ey1p$vG{x>pP5IUwC-JSBFO;z%m9EiytVWjw94yJkzkVg|RcDBc_X_eA8q0PBb#{lb5!Vd;%PE8YY{^`&`N`Pj?tWuO2>*^Vehq>p0yf1;DNOt9JwZ zyG|kf2zZQmqt{%}^i)nZlI*H|V4b%aX?R{#93=8ufSTjbmgZOuZ49v3M1csYfU$)lrZSk{ zs0w)dRc@{I;BpRmKa)&+z@hM6IDfFjKx_dy`9^_#|Gi+Sw-QTV4R>I#k(6Bw6(rw_ zp|C*_co!}9RVq*?)x>pZz&hXTI#cUnOGKw%wm_k=qzelv%_rYhYD?1Ki9^#dm&#`7 zXNeteCaKaxl@!cS^9znp-=)2Z=&;V7v~1zVi^BANOfHH~;UstHU-hxTHS*Q_G(=06 z>L?J2y3hv06WLblSD>i@#^(?FtfR@YV)aQhi8N493QC{f zN}ra9VJ*9kx$#qsqE+xSLMmY{K8>$T_%p>#ePjwVUteDHxY^YNoz}VfSD!OamBjFb zWxqO4he4Z@$1QDqcwagXL*zLnbtg%<-KPFRRKjF{f0LxM@{2@U;g(pCsG*mSm5<mrS9X5eE68&hz2j0i8iD>0xk$|4wv+EfnTnE1sPgAf>@SF^d&b%er^MarJ zDd{eqZ$%^rx`OxXUvd`BAw;$FFwN+Sg_%Oc&Eb_}H)`4CyL=z?o>v(CW zKRc>o(RKnIN_s?&2TD3(Ms~M}itznLu%jf`EFjOqj#yzdFX>zmHsdN@cQq~QcX(>O z;zE#iw}@Gw{E!pTP(EoP2|fFMiQuof$ib*b)DJ*<^_PSGhoq6HJPR!2MdP0>itMpq z*2aEXq5Y)PqBVy(j8u&*?^Kv}rXhvNzK}Ym9B4$dvA%1j=50Rhu39OXA5t^fTFlqK ztbMxI0X9Q+Br_)sZ1wd5(U#>HD{heyb@50zMX(ldij%J17o8~Oz^&7sSzbLQ-;)8; zPkEhoATjTtL>Cr2nBftTkN7AjOa;Y7_i4?Y?)LVb%Fd|6#++!^3Yj47Vr zD4UkjSr-a6!z~qrvc2@GbHQ^}4)FCWW*PeS)sKHV-q2B3PsiJL% zzYA~WyfgHo01y90mWk#iTR}d`!?%*SbgNG#KUcAU+60obzddsfvC5nJ5*Wd(TAY9- zOxtF`E$FiLnAqusMmH~clAlhmPW!Yim{IHUn~$YB0j=74)FD{Bv>1pG(4V6v`5HRQq zEZVU7Dz}tXCKxGa^ZqCpjuF; z7H;s@46=<^ZlLE1r&QwpVu*4*S+kb9uvXnP;BiAzS3%?^k1-voOsRh6kGfD9VAYTI zXeI0s@YBkRW$l4M{^a*@oo@o%>G7kD;6KmwZFAzG+z`3YK~=={&gUD$v2b}24t9r& zr41zH@?c@Up+HBy7@k*zOZ#N9m*lnX`r(MapXb}N`RB7}&qzzCyMr6!S@MIfS z8r)`rq?rw?HwEEXGeh~(%keEg-dn%H+QGwPW$gSS&54jiq3h;vKWp7i$|-(h3FI*` za5T8zbeTtdNUKQ7VL+2%=FT+Nx<{ke1K=R8emRf*6j78OJrV&vl)mnEx-F-|;9hQp zo?DNLp@Z$KgB3v>E@Z(S*|i%s(lMvBt^Zzzb`$<2KhL31F8$e3>!{1fLFV@c_{9uT zY_FS-6#{BK#3}IF0Us22EJm~EM>cJZjCa)+0VFatK%bp#wbqqpTXQhgC5I&`yr-cM zg|t?#+&bGr)`2Hl|6;AXAT+8f(VaaXg4Yy=7F_o&K+>#eJ`?j5LGLTUcN0_+=6$+B z@!mM+fsQ=_B(Hcu2DGSk>Kthy>}qtPHjL|j3IS1HcDppCd&jh%uaY9=MZg)q?0?<4 zUTPeoWMmftec=c}o>ciM7L5%nv9Bz?aFoKk6~bNo3|FN5I)MOYDE2{HSIeS4>3Oa7 zp(6EezKC1Ai9h(JR=I!(+Zw0y4&Cd(-rxUU(LL}_QRHq7QC2K?|28Lw;CG>xS~N4x z%m57YAB5WdjPKCH&mbti$%{6FfPX+@L&6}y1A4ELNaxqn8aZ0J}bW$@RWzK!S5ZV7t3l8-e6u(5~PooKnS5#>asB z^KIUd&Iqs~Mn17Z<8}%sNn;k0kD88*;fISHsG3lCowFp>m~y4L13GQq(c;GhzIJ^nl0+mU+%(7hlCB!KQg>eimRaGj^CG%~-u4^L)^iaaq$`6%JgED?X#!k`}C zKQ-ENP3Z6WDO^?Ysu83*hxfTVGl*1c~UNImwMCtP#2J) zD7C__89c3&ZhcOBqR52GbxLvWwzF03T=UPn1yj$uccaHXbBMEPr%i=g!kVdio(Ad41JtX@RB zT$?MaQi24tp8*pN^Ve#?WG=6L!~!nKc&l-U z7`ctl1J`bDdufuT7&JUAwf-k@g@gHZoUrcV_P9`tR|$_{9Q)N5Kh4?7e`+7nCJZRx zpOj*bXH@D*SEIkRNS__tv5&x3iIumV?JtW^;Gdev-?I`Wkd>B1u!Mi_Yype(KWR`a zbH7PYDAnjWLEwEkC%D@$Qb%_1{0UAaL3%orVYL&qfO{0lbg5D6x{7IP zO%1Cs5K%@m7+5Po=OxgX>x&t5CN$Ve%a*CfaH&WsFeP$f!D9W5zfu{hRQon3i8rH7 z++TmbagYKD`lR^8bXg~%kRFf6m=&E5hi9$f+?eV?*Z3Smuo{}iHzwKoFB-VREkU%D z0}?ZSk`b8kVy{I-R23&Fs~#mqN!mO`JPX?fI@=4nbBi&mS&v|c2JXO1@RRS>5qXR2+t+*Rhz+Mma8b=Jmvj```Q8r=_H)xl?%SBf6t38w=;r$(#_uGgG-_Wun6J zlEe=9>&C{yU3o#K$PA$Ek`4Ex2r^6U29OMQG15_U?owQW?n^7(X5rbrFhw_}hR?C) z;gXgPKswqvmWf1^#|2|A+Q7zca8?F*AZ+7LfnEU-#P}^t^ zRz5xmn#fkrx6^<#e7X?=2?|=dBxh`J#R?4}n_bnlZD?GquG&xUOa%2}Qt;F$MLkF8 z#C_h)P7`R@_b}k<%Zyx=XdS&+S(ibi@E6@Y^r2*D10G{{u(ott&G5G1QN?4cL>d3C zm?Aovv0Q<;hjQ9cwnq3R$|O5@hvc>_$3#q|;>WP7;n0+DQd4@lbJ*vXQvApcm?=~x zq@vI0Mku0rA_^?ct4p`D|Ju3EJg?-ty&j)|-!BnAwx6=d@7Lp}M(YP?v`XEHb~)lPdRVL*EK)gnyZ-cGi-32J1DrVwBZj; z)hY|A%sE?k^h&H5sd*Yg38P_$DI+ToO zZU}BticD8{x^7;(GZoZo-LyJ5c0dH>JdGI1SG^y|gsV-#9ZQ~QMr*q-sjU^X(j<8{ zNJ{UWxwZx<;#~W<syw$aTn4D>D;e|j-T_M0MZ z>wtJ}I<-tndaH=1nuLNE%5N3%a`o#wEfN_DM=6vL~p^u`48)o(L!#y`0ap3%;3S!YTuu@mm`X=PZqSp-+q)32|UGG1O zG{CD3)UY@|{mKS*Fd@NNWXF;+SE6i_S1ycbFJ85jM}N$H*T;L|=OFsOYFXKSmpy*) zHtF>5gJLTMfI|^1{uW4BK=eZrBN9)W6w__N#T9P8o=_2fL$VZ&LGv~oXScI;BR%!{ z`{)v(L6<0OI8z!iW1UW&Zh7M-0ucV0DQu9undg}A7cwiwMKa#E~cbt|{k6IaOuCTfHD1G9hil}l1GP>tWT8mEX9 zS^GTT8scK>KY7e33oGe#_R;!_tLj-1S`}@!PjF!NvtVD2*{{6im&0^B$y(vC+4cT# zpG}jfMz~(eM7pq7YoE5?`I=IJ@SobfgFdenVL_`S%#>ATps46<$UWefHB%NgKK7? z5tHl58kV2FdO#)4=AT$_2K(145qfVbM-(_Vyk9=H-}Q~Vc}t11BAj$v1~5)74t)_| zThggTJkWek$4*5Z!dycjZHGOyTP^yi|gX4T_?;$6usv(!-YO+<2 z#PA~L31zaH?3hK9eeP+`DW`$eSl#(Bzg~zV$b(6ywq)O{042#s7mP#|Xg4xD4e&eK zdJ5)t5-@9iJjG4A#F&(_4KAf?a97|wz&zuC4eWreNP*}HZ_ zz?ET&i!_rdV8pT{NP2W;#z=~8n6Y4F8Qso?&>+tx_-hsc*XAtG7y*ycmaz~aR5p8t zisKU0Jip87J`)Hm%4FG;#lKf09xJUE$asCv?FrnO-w7*Y*H^k}ge0=ZuEEawRjgp+Tn#nGFdJf~7{51Z*rz2hKFeCR8o1Wx3asRY~aLdTsE*FvO3K(Lk4 z2qVoEdD|;O+n7;wTew;9fSXi__?HveYhyG2Ep-_VO5PhJbRt2)CzOkN7x8=1o#5o%GOqNH5->+-A?|vpqJp}VX{7RUtGg&N4d}Eo>K&uP z$)@&)VC%jbbRU3AoSR?HuXNz!%|f;I+g@?Rdd zWc#Ua{GGi?m4yq7ao$2+xXrZmOg1PD&^OE&|AU)7lJVWEunVbxz-GL(fU_Gr8b7O9 zzqPXm!-lfh8t3(fmc;3S@qs#ky|wMBJxwXTmn~xhj1Nqj=m=;r6}(qI3ct0p!R(jf zaSZX;`M~&q>b5ZjFg_rB#Z4T3rG&y}Jh&$ph9PH5oD)`r7YRRB7q+? zr6~(LD$cqL(1}#dpp`%}cvG}Q1&wCF(00ZNB)cD4j|&*uj^y!P9b{x2w59RW)Q^%S zO}*g&p_~Oq^DCVTKqqb|(Hj1zlW1er`RLRD?FGL#&|tdNzcElGr>;PwCZgVNwo4) zl*6!P1Q^XPbPaf$wH57M&u22bJhgP}6^(auN8B<^BiVqdjZb(^)=77#HjI|}dME(A zf7zSTvupBPq7 z7Wblyfr$&l0@*2JO}vyQ*<}B?yH_1rzzXWq#8GTJYl2;$a_Z{`W8WO+Qp0R)v(Dh5 z&@3YAs4-2yw?qo>W&35fLR;I@m91gnzU^_*M;6cVbD9Fq^wAGzn)Pgn^}91|2sqQF zFQ*IvXSz@@*j4xo3{<&CaEs1nZTagrP4fWYdIek-Eazh;?IU$P#81&n4kM^J%HG^@ zrawBXv)`&h$G;ln|BzMyz(sybD+21U$;Qu0N-Uw`|1?o7XTKuBYq#jIm&tv8JC)Qy z=FF#By`8u9Mr2TY_H9sjl+Y=AKZXOf0sMTd59T_nJ;y`=h!_5isSEC!3O9vKoK%0K z6eT}iNuLGz^M%)D9ic)mv&Aj-C_h|;4T%!B%p85nqaM4&6p$s}Qdb!g7o{?~6cpn= z5D}-I9=+@Z+E92c2_9KBB}KBawBl4C#(R#|y+mTsxwL^82X!8uH!<;kL-aSDWr>xY zLvXiCaaoT~HmG4CBYgki$;-9g{E2kY);vBa;@g>GxR2JK_pA4uH>&mTYcO7v6R~Sn zy)3uy(NeYI17?a#&+mvr5_`UIgaF*X{66HTaC(>Yet#e0k7S<<@Gw=69Ym&HU=Yj* zj#{+j&jYSy&P(n4P3I$DZoXVV+4=dSVOu|A7j?C;uj6*+0+i}-c7=H=|HaSIcz!dK zX|)4ZTYcPn^CwRepQE$c=#~~pg{f{fVJh>M)i>fVU@m3kau2DQg|TPPS)N;CM71Fh zJ`CDJY-G4R!-Hd()_JrR;mOZq3?a%s;9#>N`ZJaVCtLm9>TG#`a3{E#?~8r7EtNA%8HU2>1%xv7jXD@=4K z!=%_yWi>2SggR=gQj^l!8X%vJ^drXGw@;hDnofS#JLPVzNYC^iPeN8yQOPZt!{Jf$Zex3L0id+Q$|~DT z)=9TsB<#;Y7ri>9WB#XRXP&>+OuYcaDN%L`SBpT4;2Fem!{V$p9+wS}*8~9@OiBL_ z11v@7|MB*gL3M8Fx;7Bp-Gf62?(P=c-6!tumf-I0?jGFTHMj-{?iyUanO>{UUcJ6k zo!w`jIzND#MN!3g-)D^R=zRez`R+Tw0vH{{R9Hqia6rL9^`j4CGkNz%VaJ@^arBm~ zHOf2vugKhtLUY)ypL;(wTCEx%ytVBriUS95dSKpR|Hxe!En-$cFVi=Z8PL8+okrtp z&89lUj#A6wpz*JN{#yGho%8hw+aP)g*N_l2R)!4qngFI&;GgN7SFm3Vjv+O-zZ&H~ z0NoSUMV;=?8bITg_+{L{I{u%=y*hvJw>o|%9{@HJIP>%d{NnykW}hT}^>hh47z2Q? zh(9~;B5PrL8L$J+7t_p~FNj5s(D5@e3MeV_rjTa|%}Wu0;~?cg<^W%UnJQMt))^m1 z^ZxNG2*Z%PkT^VaFUNu}%&ZmrDxRkTZJRu9C9ED*CuAWfhe?Abp;eDmXhO|>%c>b# zMVm|`%9R`7J>Vu%rjM?=;qC-xTe3-WzLm3lyVL*;db#Tp7?BgQC`@iR8QTqimQUQY zKlzFFDY($GQ5ka(I*{?z2od&Bas)%n5TCw^h3(#!Yimv@iZH#gp(| zflZ8KL1?DZ!_+hCVTLg1p&aOUQcaI|$q|)$?_0t{# zy8(O2V4qU;IH9GbCqH!5Mqo`%n5xg3(=)T-O__95_+u=`J6;9`Dtqk7KhIU{=e+fI z_CISojg0LmhPtt&%7}I_x0tPs8S~I{aZRS5Mtm@j=G+HsR~D?7Z6+0>QEm zl~rr0NLC*&a4^kBJ1o4nqH-t73Y~K2q-XY_*rL7Agxd1z662?XvV-6UzaaH-n7;1n z^np5cuReAFNpH7Ai&_q7v+m_(+tN+9%a&>0py1ieA$@DLYLs-Vs~% z4@U}~9NFJ{th%nwUP^sNdeRD#WsKW*GBE*MSW~nnDtdj^h^+6z*HjvUN!{xej5ZrtPDW*J!Wtb;4(u=r8uAIXz+Jp(79kPY zZs0*wEJ-Smg(*Cez)g^^h&9TLQw*hu`zaY9!YmPv+M2H#had@tYE1?p-$g})Xt_~?NJN2_M4||iM0Ff1syEq-WZBOm zR@8SaO`;OwQ1uewn4XAx&T8VQK5iwRuiu8<byp}E#T>a4DrOyJP`6o{ zltw&4b;#NhS)`<6P84QHYLkt!<5WAG{QG^<(v@c@7)2cfB-DQpjd)9vaIIH;%dJ(& zS29TR3r?_#3YIr1XOcI=cKRE{!xuyFA0VE$nCU=e$IDUY#)v4F9nR>e@n zFnzFO&?S;F^8IA}D*_jXV6w~ODs}6&rCl; zjhx@{U3=Zp++}$J%T=Lg#IC#Tcl@vNA$XUcqWCXk`5m7JqPsBj zKt3xqm&QDwYd;6~PKE7x25P0esWvAb`1h(7$pP@{qI+FTqIwZcs(NKj zvU;t@y#iNLeFC{tB)4Que0z1qg#z1BMPct4!ZFT_lPK4QPRi~0>y)=Zi&U2ePKs;N zm-TuW#?WsWpeT8U@k=g{9bS_Vfc^#$f}4QxjpKQiRbFnS!Z!dhv_wLO8Xz z7#st7QTD_G^T!onhf?w}2#li?+#)ztx9s;g0zaf|V?;53pyU~;P;?8^sJvjQ6WaoJ z%|KAw?#0qa=QTRyGdB99hrL1_CnCr@ z1=(DuiLHilc6ogv4YA_P0?vr;2(f^9Ju_9B=s;w#a*~X#a%%2DUQ7o!qu*gDlx--p z1`n13pYYLv6*HWp$BZPaKnMmysJ3WNf9}e72&&9*8K|5mnrNvN}T>wo32-g|3w-%ayD4Mp9$Pq^A zcj71#XwFXQq@`B-B|5^NrEd9*it`f@**ND$f@d&WZZS%;uD5LLKS&~cedm#@Plml; zx7V1377?5@{Q~RZ>^ zHsGudsYRD&l6*?yMtb_1b5-P9>!-R2$qo@E17ilEWYhDcBuUD%X~pJmHFZC3;;dCx z%IAJ?jhPoqk>f|RuIj$gF7c7`23kc6T@G@bTC@_Oupe=HX&QHof7gr7Vko^m5!E~z zE>)1F4HsWC52baMjdZXW>8qDRdsCBP3*OC2Ua|1YYSSlMa65V~{h`oC;gNX1PFYze zwOVJpINEUd^LUkOfo!IHg_%z|(|gbs>rK5y!-?|&Z}t?ANr!JvStZ;3Z4KGG2vFd7 zqD^>IMzUYXUi9wtFp3qlx-EXPtm-i$c&ys?*z8z$voA*ud$7`GD|4=~h;+RpH{N8T z^heldnWFTz>(n+=zQi(H>!$dLQ`Y3!T7dNn4&v9GDs288Jvc)2CtNRL!of_1E(5Ov zIIL~cdw0g~LwDhq@4N&ypdAo(QWp?1t=H4RY@le(SC96bOg*fNxMvTw=h8Gb)*Gw* zxqXa0H~Kql_m;+M1Gh1wv^X0bthSdXYyAyB3|ZqhIj;5Ce;CricLjo{?dgZS@$VdY zx@*G?*@L&1`Ywrvgz-H&Z|zU^-BPweh6JNNaUj~`?{W0nCv0O6<>N=&^GsfX57|e3 z;LQ5rmbMK(#Ed`Xkg?+ywe22-#i6_77P@U5rN*JV>lV4~9L3J@ea9_)+d9gXrXj;y?MGfgLbK~DV+2*p&j>?`2Ot)?njxpzF zs{8Q{KOMx0Vt5Gyi!({-v`y;Y4cyCjRmZH8R%u<;&+6MJu3W~rY0m50$F6L~JZa7w z+NZ93#-M0iHO?AYl00b$>%E2@s(1OuUXqZs2x@rD9zu4B$NG{!(L`$!)OpQaF^&z< zX4ml8JQVCICCO=J)p!kFX^h1uO=~sme+t9t?49n2@0dbGa-(WrZ?UN=( zB5{)m|5X$T`)UbNmp)~S0*=E0;l-jfek;;LTVW4(H>tTD8-mq!XzYK82}-&mSn(h8o1zW2V!viQb{Xg~-gm3|I^l9WvD( z8Zh60)TSEL-3Zh!8#vw&)wUR1JGH21s&^33u<2R(Rv~LqHUeuW5Cgb3$h9%*>PQ%l z{g}r9gGd9Z8#_j|6B=G2t5S85Xmm@|@FN8S##$3zO{-LO?P!$iepHVNb;SzNXf!Rl zMxkgdsZei8Ib?3Fli60u6a3n#1B=>3gY0=$1M^xHk39AAB@12&8d8_4ESg5K6Gx9Q z_4FloE+Uss^|makM%4=8#jr{=t?J@X9ioNO;&A6s=VKfLyneQ$R0FIqodVNPYt*Oq zELxY^C9E6#TB-idlKn6ps#|hixf7;Z`eUxzv4PIK zev()juYzn2y8s3aK@%n(o+)4i(h^z;egzuT{~AUKVFA?wVF9HGM_VApW?D1_P~*!m zyD8L5<1Z7i42gxdhpjD?Vrlmgkkw-xz~w&%^Al1Hc@IxpLS5eu1CZM@A0P_3i0lWs zh>8neMH&EK^&tSkhKS>)>h89?iEv}}UE+)1#d3AX#`E9ww!GCu!&`yr1TkO+=eOiI zSzD(+0_*>&>fOI5$0h3lfpES+a-5Hktc5|=pNi<9vxN`Q-}g&tOy*yd#`?ZA0+q&q zO8GUxK=sYQ$9slt-5K~ms@K>Ig> zq+Kln!T(S{N4EnS1l-R{7-$FQODC6qYOq!R#WVzgg7EnNk!kErRc_2{{!AF4o4_ZJC7tlPQ{=fwE z{KYtk0FaQ7|B-RX6_J1&0$0BTY{d92LE&!@+C(5-v=>Mh)nR9KPV`p4{m85ANRn?# zio6;wK0*njXLK+Cd1TZE_mu5XBJM}<`0g=3ug(~p7rqT^vSr_u@9g36A3Q}rA=u;L z!q{#xSbHl-vh03U)K&OYz>ve!YfN;&P!3z`q5F9gmBwYY(xAgp;%{xFBk;^9@+`OIRWxNtlY4S<`} zQeS$It`AV~6T*4#$OqqPsQr*-Vl2(x#~K%Y=$rA&y-F{^xVe_q))yC2K`pd=sls6T zvMe`z7Dq}EEQuK+G_VuPK+_tveipXC0tuyIO<6><)SVCD!A1VNsT;8Nm?E>fM7)!8 z%gC?gNrXbyS?y}VSp%A1y#ChggqwF<)8+Ptlrrrcn zu?QrLaREuI5kF2QJvUp>8L#7FFYeVnBF2C`S+bMbn=H!(_Szr%7zj`l3-}d={x#g` zNi{mquiByKy2G}%sWqc<-!2t$1tHA=YcBpZub3mYNf6XxqRwWe~^60(Yrdj{7C z^T=@-b>&~bTylv1y}LV8!WT>p_;{6o3dO$_g9kohH}}5+s}h00Dm37*PY|6|TZBU? zNuDCTb|YN#CIo#B3N;0bI<&>Rb{1joIPz7x<@LBLwR_ON0IM>VF=6OMUE>^?nD;Zz zSFbZ4UiMd!K+xRcx}Db;>u_>`%{!>}Ay2Kbseq766p#!yl+-Yd$#hd#Z)%cUKrJ*> z9zy9AM$Ad%Cm`yXxEY8R!m5^3azrJ(TCZf^TMS=m;zE~cI?bdj!F1t_VccfMdn{l5NDoTpso@**HWFYaJ2-Y~(m)v)R85AIm;E|6L}&&cR-Dv_ur2_Uz@C zZDIn(2=>5<(62O^gL~pXz=Y3(mQWk1sg7^Q?bS?xbRZ0qupTQ-*KcdSlKE*abJxE4ipCNCJLDVUQf&M`|Pgi2#Xaze;PmGAEfF) zD>&?38I1G7f{Echep9)UQh?|EoZdtEs2IS66*-O!9W^v^}HGbdzATdH?lB zEqXB*x?Hf|EHMkMI2gQtIu8=;C_fd1%QG>r-tryksmHKxo(?Q7Es=SlOAq3-LBk2L zNiDBgcD_*>sJ0EbYpm&xR%LaFx0UwYmgM#$XYSyz98>G!cf+}rPshY7SsJCn2VKa; zd2Qk+M9a`-Z-|DwAgBEEw@S)Y$(vPi=StU7Q1&-yLmn@$(D(P>X)#{ zj+E`e|Fw56zk~kqJMgml)%N&XmZO3*z{bhk#N62E@9?Vs;WeiJZ~Yrw^|TbVxYVT7 zv>zGSS~2Pg8X3RPK)!0x|LaQogA!cmBXIY}ofg6^+7A`P^%&^Z>fYMQ@$>}A0qG9; z^z@Hg&N$#N_dMtzJUrEZbk7fhUND;hcQq<-SO2ytUBTJd!5zracK8SW@(*V)QBg_` zO#%J&%;}(;K_E8_16Ezs3Th0b6I4dr_}icWvOlQdc%fqt@oB=ibm++oMnweNQuD|fM=%iX?C46FHg$t3JGYM;3Y z+Gp2zBd2*uRnrX3LtJr~x|~LvWn~>QvPR;sNff=Ji6)p2-RIVr!tH2y`T-^nV7QDQ zS#Bbw$Z9M#pdFg)UZx$oQt%}D4l9OfI+X66L@NU$3Lf zNQ5n%=1?t24+f&K4YnAuF5l||6Y=iO^`pmHRwIA*no4&rPR|H8X;!4kSa=kXHmrQHvA2bG@yY=!N zx)E;*t|M(H&BOCffOi~XqmM9agcab57wnAh4E5~bitCb9BT>Bw6m!P#gD@xZQc|o6 zpS;@{3TgTd{fLiS2(g#?fK&S(K6Q_8uG);m>BL8qfS^lM=!iwjt+Hlak6*~_{2oy; z*FNAL6~eqD*o;qLOlbSoTbW8`okdbco>|(*J9M3v_1dQ^AifrZXTP=djdLZsoz`|hj-_;m#Q_#JC#cfEy zo&H<1{oko`|JNQT_{GVTb@>PS`+vmAjP>QMTcT%VXAO+?)evS5`^C+~FhkWf^iy<1 zm14TSMs?Jth7E%*MWm6TExVx?rv*C(C>ZI3aS?#0U}Fn5f=aLv$zczhfXpMLsp1)~*>ZGdX!|9678e=8V`n~;{A zlK%e}jMfDb&z^u$>=#IfAmEwS^2qtv`UiDqhc1ypYUiuB#B`qW@uz4%I`yE7|B`fb?7V)A>w5Ef24jdc^$|gVBLW|Zme9T?4dK92 z(jEqFZKKoY0wFX3b8_?#PuE^UwM(ydZEmLC)CH+aL1UOAQdOF%6#& zYcEY}R8`)ID;zmWqNOggE-TC6XQ{vu)hdsBz!%=H^^`3wfzLf@{aQNQYLKwoZfzl! zTG5*yW^6xR2B5V_`dU^_4tbl_!(EM&*3r%IejP4us;5SPP)jBK5CV#JM8JSou?(FHLDvE%<&Ku-Gm`wXm%(X1S z^t2yJy{|>r&5;vTknQyP!3aVQhJ-<3P@<1Y@8-j(o z=hm(yU+OSLQnCX)TIK$#U4D^lW07V&?T6(ZPKqg_0>?m+r1SQp)wDb#y@4=lM6p{N2Sait{NUt zmnNB%6_PUTSPxWb+-SkfAVJvWMqY=wSZ7EIF!U~qm=a zBeN1{iDVOc+Cw^E-UN5f=CSS6fwaZD^11i!2;>zgPeC{+v9JGgcGT=N<`(51q~`_3 zX3=!W3c4?98M&~3Lda;L69H*87FzS!KkEI^EmIXo<2qu&r$cI@t)V@bBv}D9&^wL0mRJTZ{Aej8RMgU`uNqOLLR%v+sT%@!7K2t z#^UcuE&|_XcU<5>Ndnv8%_M`Pa4?x6&ooZe|q6?M`l$a_K1SUb*Mo{ zvc|CeN)}F6u#B2K^Qv|v*yHbZ8;#0N>k*OJhX~ijKJNKzWErTOu7!y|XRTaomhVT7 zhJZFkp$a{(6btJt#<1pcEY>72m?7V_Up9nBP%2QwxHW9;Ou~5~#-l`Hp!-b8!>3BH zO|`wojYMA{%^}ry$#5-G!KE%d>l^9}g||f3d|d{^9e7|x-D^+mocYyiW*0_DsE8&N z;X7@Dw)ytUh$OK!yKRJv-W8dPC*{4+Ke{I;yh)Kg5;3mbKWvGoFxv1G|oG)PWM zwNm8f%`?GxZWI4B1lk6*+T%eCSM72G-1#Kqizv&KHD?a?Mh6ID!)1iJq+^y$c>hNg zof*Xf!pAB)%LiqNo{TW?J(UvivJ28263SjMOq`jwX=8PQVLpp&!p4?aV|BdYRrcfd z$FB0bk@{_$_}7o>9JGWOO-SJ_y*PbG`Wt+!!?pEEZ=`C{K)K=(T}{?27tz%hW3D`A ze==Sa!j2r`$Kf4a%#&O!76y1F_;0eo@JL%3SN$1yBVSOhqx{KXwGF!;?ba zJsCI=lHMH8kn&Vte82gK5^?(xMM&`S;4Y@>Y4;%7xJAvxo(umNpiFe}w}e96fCTS< zn~9?2Y~bVo_-97pzg&y|GfdIHWuo9}rztBYqyQn2zh|N#Av(jtI5_=p0FnPW69xU= zTdWIc;w(TDr~F+$@Gm^X$T%6fRRLsR7X)HDhYd`DKn}X*XmC6%PO~2s#6>Rj&lnLD zJHb`AKy6N#uUHAM7(N1?yXB&JiY?dqRD;CzP<-fGE6bVQrlz;49S@#Ap}J8|Si~4C zNFu#(!3-}M=q^@7P|?6_rKJrdH)(sUPTXVl67|4DQ`XCxa3^D78}yP9pWe?eOp5Ks zuPzv4yoYSfyMZNUC(Wl+ZAkkTL^#kx;OheOQtmUmNC*SrSy$W#%j#GA{ffiWM9=*D z*#eU-mPf^4BfDo}k?OHvQ+I$)2~2w(L*yN%uEC&RXaoctJ@iBMl{lBL)fso;Q+g|4 z3D$6z0q|JsqvrKG#CgWSf}(7cZtw`}Wy8 z?4A_fpahS9IBAI@97{aI$>sWF{E{qC4f|}Af}2F266zAJ)DU_bDx#k=qK(x4Zwg`_ z8kT7;KqEi`8o_U?CCavTe;#phI#OT?Kof{BKRz-s)-XxEH9q#Jb+QI0)+LLjAlZtf z@hA!W>Fw*i80ASVP~4tifpAt=Hk zu-F=s5`ut|PVxi3M|J%mv3Y*eI^B03+v}6qYE3jwyJT)s6%%xeY&~hw$2Hv6W^1Pv4M^a;>-x#{bB-nOkcEA& zDM<8bI8d0GE672gVMWnmLKPo{HqZ(R_!9B{gvsdj8?LZLBOi7gT90Hg_vtcAro zSAX<}ru%_M3ZRKbhH^%(euSQ?OpjFb7zpkBn;eEsk8%4paNC9e-`;PLMk+P{2lsy; zG=csKnt-TEOh_9SNCFo~-Tz_aM9tgK-$m3w$w(g1 zy1)Zp0dK_v?*;!XuqrV<{0o2T{0osw-inEsBb5o9L;iB6JbOX5DIcR&se$gh)jtjb zCJi4!)C4Bs0*tCCC&3Dq8+M$5U{f1#YS5MLZWt9J*oE%y?$UwwNZ&({ch`_LsE~hq zY$k-aTPHxT=@v+$`E3nA6=3CT{9j1V%!DUk#uItSRO&JZ9|;2@8yc)}*PcfdLV-r5 zw9rD5_%U2#rLdyqd#0-v*sDyMS)M-2CDKJS)8vhZxcI7a`iZM)Q+Dd|YUa9LSJxZ( z7jripU%uvbL@a=yKR1e*ZyVX2I3_7ahlt;Tw|Y?v6d96)syIO`nYgM%l6V|hv_w`6 z54pxHQDF}=!VLV)OnSSuP0N~nz?eQ&W|P!lmR;hwWB#Uu@Cmg)+%7D@s5#D)1dzTP zr}9(X$j(GUQ<0V#JG=~9;T|<9E-U<0IlPj`XlYe zcF;&=UaU%0eOO9idb8wSeNC7}vY-@8($$Mv;eN_95%os?XR?-Ttf;ugUGtjB90XsM z4NQu62r~Q<&cw{EF~2-Mav9BeCWa+)W*j9 zG!IoYR8McX$&?-7cOt6lSp`Sk?-IV3E#;;m@AV{6mw#L>Os=DOuJA)t+bly&-emG0 z`Ap63FmYh_9@429)SpYby8v>^8QTwwD`NVFa)dVI2$_@`(U2=5-ncs<%GF9NSR;-z z&OHUduu@~j-lBmy;tfSVq%S;eN`Y%tUY}s+>PB$qJn>ec)UM`&zp8@GP6;4AY;E+^ zo^i9z``F6MEQ2LThs~%S^7`qtz@YSksFTaPJBLs!1QAiMoS^;^U!i)-3|o^p!uzwA z{tsyN4DUesXQ=viTn;~mm26+12-&KmZ{fUsowTnILc!CIrIMSq1ZcVO{i)F^2GF7M zfk0?c1T?P?G;U*~?ev)vdu`xl7r=`2r-PVZM*h!`s4v}qWikh#DCJoB%tSTlmS{Sl zjJot5JDb2`XREv@c(NX_VpsH z7!i2T&;obPe{a`F14+SuTo#eBa({#zjf|`PB@LQyKZyt`WO!+^NQEdl>%r+4)`aU0 zgc`_o3Vo=5>i>#3ghT5J-DF(!la^l)>Ylz&o$q6Pfy3tnLS5maS|wKT5wO@?I9xc> z;BN~(lpn)-j^LjI8o!gp!#8l3dfr)NFlM+u`tYRjgqWsUvPz1wDG5zBh!;*4q5~2N zwm;xZ6uWm%(-l;k3v}1+0|=oAEI4jzq7W=-+54w;KXR!uPuMBm;|9$NncOpxPfGOO zvxPme3kTKxuuUuKrsSX;d4$n0iI+1`5#;O6Jy8U9Dt!1d%j*NBm2=e?kfN|nSSpWQ z&>-*=Z=sy{C^yzMkVQ3Q-P&8<+8ycnZ$7D<0ag7k@^U)xx&C%U^B<=X&@ugctRat{ zAq7`h$cL=pMP$?8NQ1kGjYdvGHVBTKXC)qpXRS#2aiE*OMSh#q9-#fCFsRfYh&Mgq zzB4}Fk?Q!5Yxo49HHeA|OGYymLR2cR9$Q02Ob&2g)uffQe6~tK_*gXH z9qo8sqYiV>xZ05MCFX;DWdS=`Mfp;95iEo`hQcsaV+>&UML41-C9Fzo@{1Z&^7c8C zk*R-|ifa*=+(>$zLiBU!{8akEdPQ06RN%E60KWy}7h7b96cOr}lKo{n0cW@7 z5_-#pNw@olQAPo~U#%7N6!Hd2CmyL{0Wwk<{lD%3_tfCCZs2=j0cND=f8#HknmUL% z0~zz?wl)$XfBHe+gmLG(%8gC#_$25^cMr7IKueJM(~c>NsPUW9}E+=;cyJ`ubBqj zzl8;W`O`OD#{n!VmCLC<1)Zjl>nvBNhuNP{wr(v2lOu z!-;9^)K*73h;x6gX%WmLgst3i`4`xn>@3V-1l>N`sPOZ!R9+=@sxe|0ZLrSEOU6vQ z2pU2Y93}=Sn2LW;nlQh=Zty;$f$^QVgO?gz19=j6_3=i9@6EIcq;VS_C?m~^g(k6)>mzdwm+W_l5plw|<9hslUZT~D z%uVmlW#-4PXuP5AAJ2rwUcA&mwqk<&2%_9Lj&->G z!VxR6?Y@;__|q9^#@?>wV;Ez3Ycfho#ks(`M7=5vt0^(tp0#}b*Ug|8K$5ZyJh;e# zal>!2_5W3i`_)PRPe@o|+?36%0Pq#T z9dE#ZE^C!G_t$lh=)O>9BH{pf!Y{!><7^PgTCo+jd!|#9K5bdqU7dtr>O-0c`;66w zkZ@eaSY&i%B~s0l#uB6CACn8U73@_8oP#`*p4i4;zF`t-`FypN5lIW?Yx1M1?{8cC zK_VAe!!`1nQVdAy|Nc~~PkCy^U>5yM8PWI5DBQB_F_7aF_l0kgtCo;l1N+=2)GGm; z$_86K$aoWe61O=t$QOeJC+YS+U==0W^;Gu1nfN&YsigI|uT;B@3EXB3 z6>)M;OT{vL)>r0o-Yaj9^$R%N`d0n5;@l`xOtFk91642Vp1o5zn#DFF@|hP00=F8L zjV`Uly7w_&Ja`<|Q#kM)`=Hwb^oEr5S+u%JG?bo;8 z@4fDQfTj8j>*;O#h`4BG1D{4}vam}Zhr=@7Ti_OT@NP5#^atM3R`d#10^eBiGe+3V z8?^B?ba_A`m`bmOFK(`8j%*2rIw?!>e71aGlsiqHA6KM|{}0%+AAAE)X4^2LQ1qLM zHy5~kqB#Maih*KyPWU-uL8fG>D}U2$QbBbay=WrxSH3S0v!jihcT!cGzZ0nX<#fGHizJ?t?hIr zv^;1IurrD%g!Xnkrv!>uU5=xei!pV=7oB4U!B?;J)-|C^ zB#0nwpZT7fs3IueQM@^(yxZxlzpIthmd@@SJx5yrl^21IzjzrAJGddFB5ONMkUCjq z`S3(gi7$DHC|0UgP8=VfP1!v8mb2?o&wIyts3mh1b2z7^OWh~f^p8a5A3sjc<2gEw zq*pdtb}i=}-FmkjDuq|!^GE+MR(D&~;4k&PqQl#JH)Q+jG135L@b%Y?G)sp{s!l;ZbnXupVT|blgwSq*Da0^ zt(S`8{`CG=|2BwX z@CKa(Lz72YS3P#}Kp<%5JI6k`X@HpQ^M+qY7gbALD6=353fZ7v|1yz1op=OP=(V>;kg??X>>Gs$Xcv!g zbF3S^Q$VON8H5`@O?LqZAv~O&<`*PVuzrYpaSVMCkW3f`s_vmnpA1~JHwyeCVso}$ zUg@+*K-rR&NQp_E;3|&+8}_OHa>#6g1%Z%z7+kP^wDE_h%17Aji#~x-getDd*S{$x zAr3SjMFCx`c3|tmZ^{~yw&pf}naonMmYbEw=5;+@sESzJvNQexXNy+?fsVqC;wNm1 z$>0=2<3nUTS7>ycR7rYaD}4i{FA-oS!uOOPG1Z~`4u)`nd}^e{d(vIkOANDQuAP54h)x7C-PIQ3%{2EZ87U6Ld0n zMb$y6AbB5e-lh=-UCKqvN8fp;)32rH4OZ)1PR_Fq=}CoM<*TxnB>-dWIU$?M=DJ#y zoxG}v=k%tx)Ye87%4bpsA8*gXn9mc_vn!>kkwK@*VS6_71}XQ`09yA5nnnjm;rs52 z#!5_Ks>Cy8+RBByk~EqEJVG2yr9kiN?z!?JrEdsL67P*?-lqlDH5#@_!C&G@UmaQp zg#gs>XRX;wO&A8`cEWZYIM2B@%!$dhsvZk5tCUAMPc<7!!waR+T^~eoLJeT5vBkLM zMH$0(DF+!*L{XdaoBdIOBow-g6IW$cd1tN>D`&oY+Okcg?P^Y}0hTSAUDmgH6RHV* z9DyU@0==oVKJQk@#o*j@wW@{pnv?oe-p13B>)z!jGbDS}bRG>BjhdhV-KtF2z zgyY42XBYe&f=K$l`7Nw0xZJ}o)M6Pv#wjW@MtKORQMROw=Z`JRWKuNPON^x zf)Y2*kRvUf^MiQ3wFmqsTs$xtu=(#NFFr4jndbaocqf&MWh+152|GV_j#>0vA_b;y z#P1v50hQc7ygfhd^MWYdbB2Ie5Ni);anT)>1T**>M?tzivaWE790`}j0eqCizb zuJCLnzRJVeqh_M9t8>GGy2IL-vs_<);fF33Z+hpCscbqMty+55nz$50*+qgxEnmeg zj~rZH!-5Vk$5mX{Bj?u#6~}GZ&Gjxri`_(L8^kJkg+E=T@GbQ7@FKT0&e^S6%4jw^ zW>7!IE$$t*yB>dBK%Jd$)jZuP8WTCPLmE+z_&8BCpQ1>o6gD}g6n1Jj@uN_Oc!nUN)Gl#(J1d8Za?3UK z%%^4GBjv2PW}bJ;bsJpF-2K~v5!tWP#3n3&K#yOkYS)(o8zWiTVh7d6-GYbJ^&X6CoS^O3rad#9o!CQ z=#@8p!L`&o#uo`2e&Fq6Fy>Tte2cQ7JPIO-^r$nFBM^^CNJvBZA^f*dIo(}j}6Uw8O z3yOLkfrHBsVk@DI#hNJ4Et+-5+tNBnoWY8jp&vMiV%O`G~AIjpTZV2Z!6%n3@A8Fy3P1TY8_qQA*Q~daOud^Pz$cBX}upx>I9Akrf#8 z50MJy_U5yLZ04tz;ic-$a*g9Lj;hB#ISD|MMLg6B(bmTBN$G$ zIRO1bFuf(w8~{At_fMGh+-|#={ezyfbK#!Xe@hu6&yRC6uZwlGFyqqHx3c;?U*@{Q_uU^Lz(! z+WMzoL)-9RZ;el%UfMo?$vm9Hk@V1QY?+2aG8{C5L zoPtuJuP7OygEWFTzo(coEcMMZe2@GQQf(ww0}`G)EdQgy5}wV!Z!<_C$^hj&uoivr ztDiUA?3sgvHfojbw2n?si*jdk4lV2|>s70&9em_sAgigm5OrISd2^6medtiaBl=v^ zK~x`?=ok(gzV{$H1jHzY1H*?kj@a=Z+M0UEu!!|h8PYoQSk4o3m7KK4D?zNEC~lc* z&62PxD(o}eRQmBQe~uW8%q744l&7Xw8wiI$%?mATr*yaJYp%*kTl(lNmIB**bKqIA zRs7Z5LAi6i;G8Z!8w;*g?#d{4^}#*6D2}Hm`qhOt{7pg`t7}NsVIawo9MkJl5E^yc zK!&k4_1Fcabwd!^;`>;PBw;a3b5a8i&apgw%39AygfTlPuiFB@pao|o&9<}|bco$a z>I1WNgK@#m;n9}Lql&`ldakB2mDtQWt$8gOK(-oMLLFou4rW~j4uDQ>IY`>h`SbUL zLz3?WVLoIk?S4@V-_`6P2}?o@)kUdg%kGiiQ|b1bW1LPaJ6$PZ3A0ZIU&e+-+Y1kT zfM>TC?t!5{jT%lK+4!1NkLf&18F9?^K7smV{>q5Z9D~=YELq08><6<~)epEJdYn#! z14AKxUDab;HOjbWgTR}Epd7nNb>*fvbs-8o^{eIR1W@Bn#1Ph+8h zxbnl^hQR3=uHCVV8;@Z4}F;iu`8NGZFy`kS0HQ-mA{I z%J;aA7Ak&o`IwOt%6h)5S|_$!vx_?l5$AWd@IE2SbycLqaL}a6r2k~E`qn64KC^JB zXtqU}s$5{zC0?b@#ELR8C!9K&KqgwIaAcfqHOgk}Qrdt6Ct8NAXx7l2X`$AC!Fyc% z*?>m)Zg{0vo+dqFwj*7ppQ-*3;#tJq`g?W4t~3c74{wsJj;fu>(|V^TReT0fSA*rm z7Dl0;Kg1SxBW1;{=4VrLAGK7>k9c75+yl%{53Aw+T8 z?2@wZP)Km(SI-1Bmr!Zgi?7QCO0iZ9NeLDl6afieDBE#>Qfp+n^eNiMob?_3S#8s^(d9bq$ zNx}1h?}$JmoEPuGi|S%IJUlKojt|Ov@U^>`!aE4;GgFijb;*Zp1Qx_;4)wz}ThfhS zu#Fp>*a%$>vQx>`^CfRQT5Gn>n6-6o;9ckb!Q*_1WY7(}TC2Q}eJV z?8v>*C3=0Qm!<3nQ6>XC#w-NvQk6l4krGl{5sx#rUKRdaDs+Zmwy(}>rpnu>GjaCg zt5CO#`MU&JUF0$^L47`~+j{?+QJbDDxcj9`p#apS_)SLb&vwVZCzu!9z0h~x4_%(e zSlvp9Z~EGiuel0+k%4yYtzFI}gG*npse=bGo4i0w zpVnpP`BOI@h9#2L>G|F2C*Aw9H<0SZbKdNcI_AEUlOS)s^Cg6wQHg!h#Y+qh_o$6% z`j;rGZf9A(Xs}laTY^BJ;u4StS;D|$xRR$U*~%K<)8d-g(`vhSS`~Jr+yYw^E;+pu zCq#i-OFgjZOT@70%cu~2NmN#VV5+rIM5X4)_f<8C?{jM+-7+X~`TGsdgR~scsg70ANbVGdij47Li$a!NIMy1>+Lc8-Jo5SiFQASiGzm zXu9MMJN5M%Bg+trl4pue+07S$s(pCjV|#xPx~@qdY;{dlZ@E=RuceiL@3B>WFP@di zmW)emuhxk{VAFC*FSk{EZ=01gfFDoIE!tYuEn6q=0yJIzKISJnpFunP_1>^C~Elrl&d%b%2mjo z{wY_{Z7I`K%zf<1rP;Q+;3Cq?is&F5*%RR3F3*DrOJPomdc6j2eV><`vp?fkqBZrD zZ&N|Me9%o8IV5@E4 zP_3bg?pDeb?SpMu!pCy}-l=?x`kbpwB5gWqLryAFuOhAm!=9v_`qGm-{#t)XxBP4Jq?WTEwoO{voz^Xpm*^WEMdCCA z$y-k@9Vxu>vNW9$i0xz;(quZiCoN8OeNGd>viR^ynD0mJ3Oci*GI=u%RK?mkPDip> zKB(_2fD7CAg!iWyF$~Iq_Snh{M~WO$%ou2h#&0Cntj5^s*tl}8gX(#+5bVDb+eUb% z7{n{1Doy+0#M+T|2)JS$O1>n2LhG?3!@j<^h9B%rT%bzYkmc&aGS9i*H=-ZZ>Xnm^ z(vFj)P<%G~mMMRu3vs+dI}tuNr1*`pvY~*76J(Dd{n{Zs?H=2$E#(XMzTFDTQ~gga;QTss*PP2-7qic4*68)%H&7k-XX zGg<3oyo;|{qOZXL3j33y$J$nus!f&KC0))Vwx8(+9jpW-dGWe6hlR&j?>%FficQ`AhPAsbYtzNrJU9 z7r3wwC-*1-J&v#(yNp-ErR&tRZQnc6^|Ncuwnnd}UDv7EvDLBJv3=Wd%Vf)@cerb^ zYolwdYsIJhp7DHZ*|+Ae?p5=)lmEbTfk*y5$@%cIR871tQL~_<|Il;EJTb0qeIQ37S9}u&$K;qY8iOJt{)PcLk1!-$Ia3)0%hd(K!n9Pmi!4jdoDQ z={08tF`F_QdmGW47e$7zTN24w7-Zq^pOG75xQ)cAnf@^hYferMf9%vEUehL?vrg+& z;~MXg>oCPIH~`Al>$G*8O0L{tJ@PfpfqIopN<|T|SJ*Ju3*SHBM9I*V>W!*6@G8fF zdNYVLQm_-%pQ1&nC|~@8KhiW3EFkg+YH$kBIFf|7yH31H-3kja6}POJE?h2l_DPkj zRPq$Oa!%FE-Knxs^}xMw5l{)!C2d=zo_R^8!N?)=H?=AutEP1ky|S_^UtPFL^>2Dr zQdU`XX;o?2IhZP2iRIZ3)#eh*a}iaylB2UhRSwlwpNK{N%BL=I+aHUf2DBI{RSROJ zqN*kW@)N05E5oI@CAjzBDkBz21ypSt)z#yZY?Go+=Tu!PEd?ZPBh^LYQ*TO=d~(!f zZ7T_xTIFp!^N5#7JyPT4JbWg78|?tZA%wje3I04&j1D_OSG-BN@)|)xdr9Y zwQ0{k@@SVRJ&F*ebIF(L=t`HRJx*vVTUF2R*ejDuc|BUy)79Ja7?;Re4QCQ~CC=0Z z)Xvx|iOzIX%jMprXVQ4Z&c=M=7Bf|a=G;oBlb$Q<5_$E`thzMM@GJAry;YB8-$ZB9 zd6mx6eSR!HsrJvql`+x!uk5p( z9Jl_)LHA|26JVdUQ-KQZ#Z>5){kG1PQjj|g^K6PMGyC_qby54EcG0Xczlu3?;)$8E z;vGi?amyfjP~R%JcwjJX(P}{!wlQ-swl#A!wmEY+w!N|#JWGaD3N}5lEJ6@aIT)E` zYJegPj~$r|o)jn@JOZQ+rZjC4$-y>j2*I}7MLB?11p7w5i#ClZ zSbh>KsAIRFCL^!{(Sw(O>cO0(#?GiJHVt7FtlDz&Y}}a@oF}AZcz4Kb@QNgsxH*zj ztlYU74$e?4W4rREb%Q$Mv4d;kwSzoB{$RJrC#Yi-IQVHQOk928uLz9fXYh=qqMx1> zCc$#wR(4g3ws%>J26tVH=5}F8O`Rzk$9HW_8wWYWQv%)qdb{>xM(x5DfmS>Z?V6YW z-o$Fr>0>qa6X6#7ztGP-+->Y_+-)q}n8cKoHU8t_e^tY^>URGKZUw~j%Kp?TSKKJm zM@2ZGU@H5iW(bocmZ-vuk_O;ikN>V$LvqWX1o1wR(j+06yDw*Q9 z(@=h!FP?Jg_;Z$aN`Hp9!P+4y%n8GL?5^fTgeKn9i4bHxH_tsdt9Hp*-TwXFx4h?%j+RmW7Z5E zm1TWBF1pZkD*3}&hq|s%eY&WE`k^fi%VvU0*O#Y&RHlpi2$;cSdpiD=l)R&ps8RjV zUr~-Xx}sgxZc;Y;Xe>VBF&HzOtPx%uIh=mTr{_tTP#R+*b6Dd=#;d z*Pae?6zGe}#20V&MH}ty1_@y-n*wEgF314;ZdEF3kum5#=I({r#s!I&eigM88*TPc zGB~hh;tdZ267CjOfNY*n zV_u&ek)^{%TDoEjQr_y*ORG)vSbGV;yC&h|(~VH3@)mXGkR1x0)2_2%z8;$UR-xnq zwFn2*UtG@xQwus?alPEL?Z!mm;%b*$I3+%k#kIal+YY3cJr)!9sgAU3O}!>Yu%ror zdqaQ75XVN}AM%^-L$ZdSa0`3ALzgf`^l*#b=>_;%n=(JcG-7WrzcCipU#`T*;nt=| zTD_B)+r3wx84b@e}aj`ES6Uf0@==Ca#V)j#mFL#4H>wT>k~} ziB0~WqVlm1HAcDgP|_9+5IFTvTLj&1?`jA;zc4c@3o=o-MTOYJrKM$bI@9m!o^NyF zJ8#OPY#XZ7i*nhS$+=ItA5)!EGe$=J5E)`?VZk_fZyk1JI&53byK5(d{dl%wcZ-`As&ZlJQ5{Dm%h-VBlE|3by6x7rcR}?WIF*jTl=&L zA{uP)?L@>L^3pzHnh=C57^l+)UoiRd7E-<3Ll;^LE6_KzlKdFuVH1QjqC;G1)q=WN}a zbSjlw*WbwpaFMFE&)aGJ#@ZFAGQzK2eXmki6;xYg{R1cF!miC=d-Dm&Q{xTG8&d>J zn8l5TPl|+Q_3-(?`9eNBbj>L&GRLBr+i;qv@nf&`?PHb1&E4~uR6-_P$-Czxr@NGT z>qb9@<#9=!I3R1;lrYQktvL5qrWWg!j+ezQPqEYtm#rdfzCz5W6WcGU^EIkp168BR z)4IqbV{CVNX83i;5}h`=4fuoJU>>p>*@O_21)`fm$<2Zt)@Tw)J(6{v|IO3^BfIQ}IJ4Q23{sl{FDOnq}7C*9t|i_^<4Wlpcqf z27mIL{=Ph%1M(8>{)@Vu*7*#ocGcHa4(fe|eIz(n!qAdJbvGO_jhrIrB9s zI#)qU*r|KZfUl(8?&t>YrPbi7_vSpuMtsln zcX&~pL}Ygd6-(#~D&+(!l5&c+s9I4d(bpy-KiWtOsJw$i8hvEyDvKS&_4`zZoxn{{ z(CnY>mo*ooFheS2Ztrm&*9+p4kkVsE8sgs;mp96$kn!&dL$}iM2T22~W5II#qiB)k z+Ns*Z@$`v^g{SUqXFWK1W-DC2d09(v0_|Sx*Hw5)^gmu3H>GBbIurrYw}TJg%unFi zA@H$g;)Xz=v$*mECyV(S6`~`uKmez|=>}su&YB^Ol_Nw+bI2NT%I!*pLbw^Tm zPb&9yXjac&OX&Ze5Mhjc1r>cVDvdvn2^jvLJBPTMjDm%mn~9Z$oQ3!QUU#WaY8#U1 zWBJ@z%nh4dA@~RhB`IY#S)!i;sbXU}$s`g)8FYxS4DPZK;q2{CKkUC-@0+x23gx+$ zH*OyliaWmaFsIB8;A}fc`S%M8m$r!?&o6)NzA%3k(h!E>*Mx~5MmtPk3a6smBj=X@ zwS141eZ9u10%!{7Jm!3fe(d>@{TTH9D@Xz?8JPZ+8<-B+0o3{m0BS*!N7ln+koGYe zXFY~}R|9fEmP9zga4Y!f4Zwp4z^o#!V7cY|lm>YAU%>tLp~0uogwPWx1IUfj9vi>& z>{EdsMdrhfBMYH52*2tNkb>aAup*3L$5H!H8^m9A2PF2nz+Hh(lY4v!whDH#@tI}; zLyc%I0)--h_Kr{UyFQi{9DM17IZWozE1J!n7q*;g*!XV00AERz9_aTY% z1(EMJwdR}7Qw>f^IEv(!N=|WRjp?%30h{gzU08%G1#60cGKW(ZA=8sQr?ot%Q8X3= z(c-k4d?h2h7KqC31()lz@u3lNzG~} zHwH&@u>~H%mQ%>}!bX20?MDN_!H@M{k+?3yH`bJ`W|HDcqx?daQjK!x$Gh0ceRcps zTqb%`Du%4vjLTEOE3tXk5l z__)OL#yiT(JI38Qo+CO+9>+o_xsSPMWOt7=51Aoz&=1eq^yUeeXbTF!b@_(daRVa! z>kRRW46Gy&Scf^9Q?c4j+P`q+U$r zPVeuG8FP*B-LV%?rDwPN%*xe&!hBzER> z*#agdBm@4;k2~xW$Nsy|@%%Jo;g=g27?yU((N&@WL+`QFUUVnF|a+2W(Xt zWK>606__kkK=Ayza+8X>_%UK~6m9`_AlL--gNI>glTn9pZZshnSiGrgdyySC1TNzH z_at?HZ_sCl*MgwZrjXJo3iS!q&rS`as*|LXI%nEvka7@m#hVp&rgoa44ED9I#Z9rb z2AT~X4IUG+ZH#S>IwQ-&ZDL`oK5L#6wrvLX-4+NNgl*mC3>$`Rf;4N-Yf$C{8-i_b zO?iL5Y-~0Ca7g$=xHirS^sKixoleT+x$lk0e`+V#q)v=2yfv5?b(ARXG{{9|jRsUwn`YzxbdmfnA6$er3FDBw%U!aPpzEhh@eJS&WY>8#7tyJ@WBKp@dL6*|N_8(8Le-wxR?=0&x z*nb*MeI5k6d;gzkuaNNTA7ORq%;6a3SIAeRE-@|yyI2`DT#C5a@26=o7C~aNa_c`r z##}A-hoG+uW7_S_lR9~JHr@%{DrVT0e{h&k`w(jJt&@okWCg zN>io%+|107r@H0i^tMUD%gj(7wX--#QAR5stGpI7TID8Ng@n31PmXnaz+!hddX?Gr z>5GLM8theU@g`2`<{~NFFJ)UAK#7CdqBh{gOT{6Ou}ZVEa&WJS)$ClzhYzARaB+I6 zX$x5n?3mjSl?aAu`{ojE8|npZ0_O>cWI0MJ~18yu>*c^>eEn#Mq(b z##e*EZ0;T7IC>@|V&!`1_!$+2(b4u;+G|$e^8)icaQW}55*0-IxS-EH_0Q*V(toE^ z`2UhwHFwv~N=(ej(ev}4{*P=nSHr*!%>q4uvpf53yltzbCxtW8PrFdT0q&c~GexO7 zf{}^{yF{s+rw!J8i)B0fHn$=7`&X$bMG-mzt26kt%z+4qzcJ~Cfb-nI0loX4ZkdI8 zIv-G8^D+N%&p-F+ZD623>@Kn_@55e3+=@#M5yUI@V&V|;N?0dn(E&ov?ijU@A}#j8 zO-da0CM=98HjGVba=DOXQxHjZ~IJ@J{G_C8nI4@8h^~Wui zxZmwP7OAdc9x{oaht?n@7+){l3a#dxm(`@^>89ULuVdLdH5BB|9JJZr2932dJ^D|Z z_G|f=v+FkAd1`ysS1IQ@T5@p=!WqcO1g?{HLZreMXMN$dj+KE^v^*9YRq&qSl2;3=LP+@N1yDM!7DmDwoHDMR zm-=s2(U()v1|txsRY$G#c}~BkV6c4})kWA37orvvL6XwKEp$?)ii@!(m1oM0=*(6H za9gm9xB#Mbc)#yR2DX0oqg8 z3}6(zPN`PCvqNa?y0~<6*~PT>V{UYDy_~~veTOsQnN$twgfD{6y(n|jRKmK@$3)Ct zAcg`b4DcOhUbZ_7Dw>28ws_)K@b~gU==$YNA+f?QyLkznNIHOoN>bM_L>F$e2ThV# z@%iZ4`LZ^2z{s81xK5;>jZIUOwPqZ%Hwp`7`5)_VB+$t`v{|OETjS<7tRu0o$=WG2 z@rCrx;j?Z;cZGR>_B?-DMo~v;SPNLP!MwqkhUU@6NLl+R)8!J%)ZWcRLS&j-VElqR z!C)0KRbst|NczcVEw?3Nm-H7TU`oT7j_ezhHbj;&lSJbvzb>Mdh~W}~=fn1sc3&q_ z=4jA&Rt6ub$3`umP{yrVfaj)^nEvChluxOqoDkx&t+!z@h^9P|QFd0Mc4j%5N0~}Z zK`MP&WEq=aHEj;c8Ll2-Mbp!Zo_!M`?qH&dIyB)PzZrLo?;M|KXs>~3PwC7K`sYsj zDcx%Po^;d$*+`}S3L2AT$d}3n3YsDE?WB}-ZLZEt;CUL`5g}#hu62k6mWguWd>JFm zK#@o`5Ys{Kix!#up>wJP*C5P*I^qhqKEj?)$SHmun+ICX7y09B?a>P}pKn~GQ`#Y< z3}UFw3VtbTmdoyVv}|c4%}4fdb|%kd^5~Pg0Lo-Bwwg?wHghX9`o=^`OXP-3h=a*l zpuPSS<-9zi;g|z+StB-^O@b&Rpi3XnzwJ@s2FLfA|wDWKrT|`A|i-8MpMHx)ek?A*( zJdJ!t5tI+V;bcvn`U*yoGM<8PVxt3-mF2S)mljVKOJ2oVvK8))llRTp6bALx5R z;y;G}7S4ZZsi@&;eA*$&beK1*S3v61VcHgEe~HzI<}hAToppd{2~|aJWz61iw`t10 zbj)BCz8l=xyK?5|f=Lh-{b^jwm)!l?tYX@$&+T%{&FTI1 zrJdo+@gD7o7;BVaPo1)x;UFGB?O{7=4}8?(wk;?cKoy2gi7vlyu%@qiO}L%@3mQKi zLw+yHzU-4V?H3VW5=mlGH*th3m1UMjaq1DZt*9q=R& z;_2PWUDwPrw$5)M;OhV)&mYLa@6m9PePf{m9|l1zG+MKljp+@=Jhd(#-(kj3gq!{7 zb|&j(``J)YzB1*(mdTcTQ8y<}P-8@ouFoWm+paLvm}|fO0I??xjH>-0POp)m$U=$( z-_WM}W7!P-?6=5{qgAh;mxa;iWI4jo=z@+J%i7bSf{aGT0h+F_gQ4U8Zgc9*TH8Pi z<;lonhp9_OXwc9chl4IKi2d^Ml|=__SacWVLT%4do{x%kMWCJ7BP38QLogIj+3NTE z4l3`jOw6*W-R0$^)t=)_fc52Dyw%m)j{f-35Bzq16R4s-z~Oa~=V=>?V8K$|!ojN1-IXfb>)^-W+BtB8~<5xQT{HjOz{OXQiCT@9*>IQXlVXMQ+E##x9Cyl*d9fsTRp5L4|H4;Yz zFgTa76$umM;$CQ7$E%$iYD~WtC+X&p1e`2n6xuoEnqXU}sPX=FoMT;OjZ7)d$pXUzE?Uvyq|9X!@^ zN|9P9MnLFp=Q`!bAp3{Vocqn4*WGHO)3S`#O#1xg6#JzyJ0Id6zUyHEyW@{JaQLZP z+>mP@s&tsC?s@5_;TAJ4xX0B5JQcD|CzRjT6h^2x??%Db#UXWt-&*E;gGuakJAARc zVbOmpZY;2QYm1KOB)17ytlCK_Din1`w|*(TsKpcMo1HB*is;cO&1L6VIchup&}1vT zQmUR4HXo0rb(&Bo9akV83U;TvDi-wc4Lyr88;}BonT+XbV`$t00+t4JEXb`Qxpf(2CvgtJ?7jeI}%27x(LOMJ$aSm0q zvH_ED<7+F%YQ6Nmyhbb1AB!dI$~YDVeNl5Qn+QMI(yPHanJXvB{DNGPrMgzN_=O`J znnJN?ei8N{bfj1>>mE5K5uA8cDn>q8r533tuwe&*xhv)ZP<^qytZYQ7sjfYmEllnK zK|HcdWj3HNgpp=D$~-{IOLoZ=MFGL)Y?(5SYJ4dg_v&$9>}L(I{)xE$dxH9(BATqYsM!C8d_+BL z?9KlpL9JE$Uk*!T#*k+aJC6b1G1+bO&Gf(HwVn-$qZ!0nh|2y`2%Jzoa&^t^4k29^ z$F#2(L_`{JN4v4H9dG&`Z}ZhVeOfGeHhd+yDmm~9a=v=-ef|M9Q;ingcMvg zX=F;!=bF`4Zj7LSoTSxks4`X<;dKT3jRPBatm&xZsR4jD6Cz{#UG!UiqFynllVPSL zz}`A^hiG%S_?be+u7k2&hP^V>$w6jvSx6}|DPG+;3oe|i&-c_}aee2Mxx^t|t6R*Y z=)T--_KNSh0ecQ#(k)zr`p!QH0|JgRh*_Zu^ZGW7;2If0`gukD^Zd`CsDGzt=aWINax-?-y|Xiwoyp~7x&89H zGM)&a{Y@pe*}h^t^gMhpeJ_l^gC41m6+UWL64t0RDVzL}+|L2~y(jba991@=U7DIA zKx{}FrhR_KNtQa<8Sm{iEkp&lK@pSY4E?riWpaR=fB>x__QV9cyHSgP{W988eoCS-k6tj0Ab%K_ zeIau42qI}~(swmcHjo$$129lV8dWH%A@9)%1}}n0ZdLjg}fw!2el0u zTE?2wF4g&a1l$L) zOHYWHxc+^3WXJD#WS8E2@f`X+^Gq>00jsJIwTK1s99f`80Io||NHH)IRgo%9jypIa z`c>XEG^9DAnIcCahingjUmxZe^;m{GUYlmm&ongzE@BClk$gJc8O7yCNOUCJt*fmt z`m(M|pcw5wP|cGYS1Bk1$M$^9z`MZT$jD01&f=x8?LojlS756*)efTd-Pza4$xh1G z>}Btec;Z!&8YQj-T!ag^DT${1XWUX~jwP~002MyssG26}B|u1l5w-CRRDdS)Bo=1} zE&SNz%F@kfk`$^VLuD%^95W2x_D6LzZX`#W7CVx1zp?_Od5BmNFalEFEHp(PITa^0 zOm{__t3VqDOL<0P!K!aP+xMi$XxaqON(M6=_|+O5125DKn9~?RkTo5tQDdkm{TS^K zh6jDZ^bxVyG`D&x)tu&L8y;;KO1n!bhlIZnpvVRRv#^jWX2WShmq6#R$siPbkz|aa zz6)hyyX{d(vJ6BhRm`1$uUCtud@Khv6X;_Hjm8e7aRTWjZm-NiiQl(^)R>708i(3W zJB^lGUFMTiULKzfJ>4nK6;${wB{qo>er;xHX<9A7x`Hr8ML7s%85Ta7FTz-!kL45(0=^C?nyf<(q)1GOPSfX14NeHnw3R-=HCK50tij z9kV}L>$~KpnY3z(+7$RPyNe(BvURc1%n6n3mmRWiyjec``(Oh8qP1WK3fy(e-!01m zRYSoLVukwh&?32BIyV+T6cR%ov708 zlz3b1KtKqVroiXbcl_Ci^r=m>%g}edvO8@V|Jl;p>26ZGc&dmkbGJTbwn@#&@6DsF zSr-+-@!!NnD39Au7S(Ws-tk=y_5haLeE~XxZEXId!6tae*uZOdN)$MY)Z4juGg!zs zPC*N|iRJZsm??}hH%~cs>|!=Ab6zzp_%%S@h-qYn7j`+Ebyp7-%yJwzjiDw0?cNg@ zr&H!hgTehPfC>dCBfdOc+{cqfXo{Z8v2{w?Cl*wQ+g8LaQU>&m8e29l{MmIb>fQiJ zcE1{g*63|~z(i-n ze)Q;oWz?p_fNrT7^*3O6A=wvcm4q)~Gn>I+;;LMU`f$NV*hXc_Fufo=b*cYqRgBxS zdW3}vy{h#!_N|XxmQ}9FsOv&I%aEL9Y{TwH00rd%22`Ut{6qR0|6s)5Ai4VWMxC@K zQFWx5`;Nmc;O`67O(e^lx=s!B@-_FI2zJSwBkAWi9FKXYnIdhG>$}gVn-tPR}Su; zGCie6^@8?nmLPfY*AC=9FAd8~GgMpa+bI0aK%u)L!SJ}kyCB`GZ=@u&Nm z240A{A1pFGon98wi6i_JZ_*uU?4ENf6&9D~hKMq}&dwUV@ClBai$3$Xh|E&&?{b{T=i|)hG!?VB~J)`d2&eQdv_1g3_{u;XU~Hy7XQiu7+Jw@+!P) z*m{b4<25_DY$y(!71>R zm&>~tiRz|wl92Kzb+VAs2dsqQ9i=rjG)$oCrlE<@kHug@RjEgtswgxx;BHJx5})Mc zM_6eI4>6ePs&rG3ihK8;$}X{K3C}UKx;2)bnB!Ov*gTlo2BpR3mHsIF?kWSxsC%)7 z(x8{AW{kF0R~;P>yH89NFei)hq-$vHy-gL#Lx!%QA1P;~p z>KP*M^gjh*DeG8(wf5WPj5dD`bGk`&t*^BC(G%NQFKGZiZp3l&>A zb2)1{OF5ervlZ)PW^)F{)Y=r!Cw<3zO4bsiIg2O1^LuB4&aggEe@cHye?os$e_DS~ z|EK?o5FxfvMgp|0zRD zA~&JiFSo?E$hXY5-)_l|eom=nl(Qj zL_L>$?AMC%d)mWUIy^EBTqdYUeRGwO3zxr9BpA-^aj(GQ>7MDs&$=2yxx}z@>I{3wa8fj;^rXK+ZXzE?>;s?o#$~)!i>u z`E_A0me~0=J^F_)mN@myf_fN_N7(sxetEcqf3$GxRYNb9Mvv)6`UW0FHGqa@eu+0v zn0Q8}ewDOGHx{Gm)u?^_7^7NMst+2YTGfet9HR^etyMAgfANqiR4&>;yc}7*+kjx) zVM6vk%~%U3c0$#I)!|VxTmEh{{wm6bdNx8ev^9eR7mi2)lO7obODE3lqOu06*m^Cs zH7@L(in2y0);=8tD zCC^27tXFmCg}~_F3h%z&(Z4uX8mao*R(}0eHw4T9vyUVifB5WQBS6WEeszIOAhkvC zfWC5EAq9rt&r-4Zb+#R^@CQ~^5`J$wqcfV(o$oxOGa7QMGi(8Ux)b+I%mIbEqXCJa z=q1+&xtKu7+TC*|j(|+vpNHvqA2qro0ihteCEJJUm_Wr^lm7L`kEkLjtu0)j#H6}%64v4I*(%{%6=c2z&}XMTXtNygD9U-*%2Y_R^kp&pf`35-Fc|ya$i=*P31+ z;RI@TUc_Sus*kJ@vAq+d8V7DZdTF4&ppEvIXRYP4zxyxkzx)7|$9yPLtmSjQleOlZ z3z@x+*+RTDj|LWTy1pI9e5gB^25vvjELzrm1cCCa(etI4a6dxV_g_ju=`kO&t*&qB zu^+k)7J(y=PU?6s)uVycZH@hwuU=KTZypEzS{rLS%N z<9|^h&zPws0g<8}A!xF>cgST>HY}S8N46@vl+bo6CzRpVdv3|zx+z^Ul@giR>KVAf z1Dk-alvJc)wk(i8h$t42hg&lu$5)ELRSHJ*f!`QtQ9^aq$x<-kkR@~%iT!lxMbPaU z;Z=%8V8tsjtC7iC8;~t@A5n^18c>Q`E2LuPNF%VTCCE&3Fin5}27KUGhGh~;+dWcB zn?zyxsFQeX?Lb7C z-S-Kp9YKiTWpETiNRMSCH4w*au_N=JMnwsd>qPnP!~#N#oENFS_0D5mw8L}U*($PtDJJQ0Ej zoe^)iVZ?~m2@%1#VI+t)<5Kv`%-}BK!LC>6#)6d~y0w>z_xZqoL$G*r&nCS;G5J>5 zFJE~7jkffka1L8@Q!_TEe{d|e{~J6%qN%6#kH#6199|wO>f8FFTqrFiI03D#U4iyD z?V>$WVswW&5*~!$&;HU_=)O-xPmm9D4$QYd6;wjN3!o#OzLqfr6LAPDB|l zH^p%nB@huxW~MAQVBTP%#oUjXJQ3HWM>EYtpESV9HceK{a zvbDAp**ZY7a?@1V+Hb2qW?uryQcQX-O*L6&cj=6u#Z4+WB2oF*G}}~|)DFWF@MPOD z3Rzdi1rb-8a0m^0IM*Fs~MVzAB^iMcy(ej#%B4` z%}OABIbpecoNia!B0{ki9ublD5f?J7kPO4YCM=Jr87{BmyYdzSz86NDL#UW;c%3oC z3_3Px2q`sDT2PYn@kPIr<>o>xe>;UeZ9`}WgjP_wu^fJ0^dV#$Mh}n+wr|7>o@?kb zWOH@A{xEFO7wzD*P{3+FjfTAQR|5Z6O{Iq$nf3I--hnNfOK#mCOxk(5#p_oJxF<2d^+mSVk62RZfKIN$YPYA6z&JCZ#O7`vBvwS1so#fYT`I^ zC#?hxMAhKzRnEUYrbdw^*yghR`-=->j{0_Hue&}FaB(cPX+<}saUdXok*Z=JA!V>7 zT-I^K82-^=seqOVTbs=vIZ-`B`ClVaU2+$Js&jzFmEXm5%6`(2wFC}9J!l01#OmGg zw)GOZ>k=N16$@Y!lc0WGh_?=GzH2?oQpt1kYfX4n` z1KNLh`{kw@IBl4tk9{a*FUp)X!RuyWa6)}uAJRTKfwN!8I2T;&q4adlu{RJ&OV_rO zZ6Kri1L%|r@+E)#65$dIL{<~A_L3eDK=u3b`9|6~n44pX=<>FbO04&3ZvfTGmf?iL#mx`Mw3_1D)md(H(R-iR53o@ANOH9 zr>#Q;+38+c(|80%#^wvi+`d<*67@?D*{ReQ&M`Z!{`GiJ;FSxpo^;mt)8AsIf2}~G z`mB~bLz~Z9hi_>*w_f~h8!8KGhGoa7p4Gq~9j09-)xQBIM>d<8#K2tQS zUcK2+z>zR1+2;?(p~oV|sBw!-iVE)#$0>u2E*-i-jsB%-;i7Fz?~xWwpRUcR%{Yv% zU8aHsRm=!1X0Yd~XUBxYTGqTD=|56Hf{42-ehvEaRh{hHtF)8vR^1N9_-JrLo&8H} z-oSo96&yW6IYOE#_uEdIsr`!Gm9v)O^e?Bh2aVq;-&NUG6wXN%rYGL*LR6YD0Ds=| z3jEUu7TxLEG!omINtjhuWI0Z8DLWe_Uk*9>Jbx{6NX=)PeSq%!9Zo`hvF{9M7H3>~ z&7~I#4bd5X9DHyIG?wvX4Y;e=D*j5Vvr(yZEhV2+PYHJh5(`=3O@~$MSuQnl`0Ons zRG09CNvQF2VE0!q<=mz@=o9wjGczO~iLzC13jNqLm>Q9<+(9I0FLhiht*uQXcbGT} zJGD;Nx7kQjC|_Wim~XXJH*LzVyKmOtpiR}}@HZe9xJ&&R_zWI5-^wADz9r2$R*pz4 zG^J=o97+ZVd8Hnt!iFS@$rOmtGqMmI9!Cg}bRtb`5EM1-jW-MViH$MTh^-bPOY}G4 z66@#YuGK5$BA>)Js&C0G+fv+^2VkNUqx2@Uqro9-g(okW;X_2mgi-nCievmEIgw8y zN1%-I%%u#sDK!6gbm+uU^0GwnxDSSLD;IFN-;=W^lJY|(8>bdGS`DFEKQ}tUxPI;o zZ2-bw4SK@={uCMr?@)Ya5ga3Y!g|ErNeGO-0zo{*Kf(wle*_tU@Sd(nX;h`7S~5_1 zOm&s`WwQp%N~})O3zztSDGF~e|GKm>b6rM^KJyfE{Quuv<^Q_0{}opMZw2L@ zNSdDBE+#g9oC~oH5**G?4|xpRSpj|s%pQxcwj0wAa3}3kW;LI%?V#T- zgS20QT*U5?lxMTf-g{N%;L|Nrf9IHlQ`?{^O?(mX1iRzge0w$3CjNdv3a$R1F#%ZnaRz-9L+*XkoLA%w}b!QEWD8S7>%;( zQyoA&5)1iDkA}PO3FYpCgoZ9`0-mi=oR zzyhA_S*G^~P5DE@J701_X0VD0j8d*C9?mAG_=kdwOyY@B=YH@-Uu*cqc8_(u@Os6| zJ%+_N?dzfTG3@c7_R;P6qV_TEL8Asx?@6Hs(C<;92GH)oe9nt&H?VKiwTTgyoS{qB z>d~<1>?bH&>!b-+wVPzSwyZuJKTs!6x|(Va0n&Ia?qsC1e;oK~2sPeJIOdOq;#b;3 zW57OCezn{soW+kqK_e{Ov|nociXc3-TPRMGG+WYI@0ues>0hwz9k@uStV~N2TuyMS zVVopayU&z9e`l9RFxYaaBB`aQL9j-l>$X~dB%WKu?^1%dmAPe+Lwea*8l=HzJez+Z zt-@fVkyd)|f3qKF4M23;&=9D(zhsRSNt-Lm&B85>bjUQFcuznviC)v_jxB3YM6-#- zXSMd!jIp)N zXz%;3@hH?;wrd*#d1UENw}%Cr2V{awE1`!n+q-5Ii`UkrU55<_HIjRb4cgiFf(?jw zLwzsUrQxsLjprTT)TT5J@1sv+57n2@4L;)SG98Ht^Am{Fvb5vUSV z=%rB^6qa&D@+fZHymGmnQ)dyO*rnoA7}wku*&1>lz{zvJb_2OrOh|<4$-LU-bL3jL zs5VubyUZ?+qDWGcOxn~5!mV%B^k?!?U2(as8B0zX&p$)#R|+s)$J6-^gIk2MY{{$2 z%U8KF;X6_1MA7)Qk5VodMWHCv1+bSX(N1|$8W#^vEl3;2Y36H@tR<(i=%Zi)7ek10 zCAs|)0*VQWj#wIEQqG@jraAB634dUh>n17BoU-+*aQa!8*W9pT@Et6C zeI4I+EnS+i_M&<;Uo09+Rcc*yB6Y2DWx1t=3c-5Iy_HHXNbJG+YH*e}@L8n#r(nEw zneZXG?V354{uDBX2ROz;rvgmMgN^YRV{d6=?@OI; z=-~*hT#+#L50=wK91EVydr>s_uV|uDj!nOFw@EUtJ{{AEo~VF)3oOg2YEQ+*&@7P} z)f$T5 z15j=lxs1hMW1*i`NX=}+{*)b8>w=o69ZJ`G~5a;R?k{ApCFZ3YZ+(gJ(d$f`h)LRO5w}Ro)a)!xFO<(ZEQff+5Shuw{ zwekG9j5La^%)w_k4GL*M!6DW{oK6=mh2)F9rG|UA-kQ&XV@z~B0qVLv|K<=lyBFXP zkXR+&DlcHQiWoEXDs?oHkTjJ#TobnwxY?J}7uPKm7^?E+KoV1+uCmcYL&m}bafs~M zT)|}hDiy3+sIYnx?Uc2x)5W8%yuLG3eyD0Y_WOI?v#{U$3U=S&8n*h2wc z_zcwVuS8F0a_^s<8Y5r}uPT%xI#EDDk+B|ZW)=2AMQiW)BD#ptUZzMK+PvYPL4}YV zk|csR;UUSXA&FeKuOib$r01 zs#$`t8)ecbnwoF0tWwsrOER>9|BL49+GbMK#B-Sd@ZhE+a9CalwtZL^4oL!M>n$X6 zrW``Yyp}u$>_Ez11_mr)FZ5G-tUh7?`qBLdG-#oUBF%}GV-51%b{G{I@%rT5lnZIR zac^-Yf5z1K9p!Cknur*~p$ilqXaY{dnqAQaJ*+|G1M))syqFFH@M1vO zftFfz&w%_?jA+=WZ<7n&WxrH-#4jPOIaE4x);1Ri3LiL|Da-ieE>}5y`4)SI4&JFM zB$QAS6>s~6tp4lyRdllG<$o_JZd?tn&b zJdmtv3wHxN1a=?nMxOl@!aFY{C>W$Dc=je_l=^(fqt$BC2QAR*_yh`!9!$( zY$43UE}jg@1z@QyY>Cl^U}J1P1FZ1*Li&_lmOSh^t$V%AQfLv$?0)+a(ZF(lbMS~z z5Hf#KdBplT&FBkkb@ME@aIJKKjjbzv0i#sGxveW-0i#?&maQwG0804;5_ePbWL(*@ zSNt4CO~o=eR$u(253OTJoRN|>S-5r0lC+?kx@lEh2YONYgdNuo=g3`;P!&`c+@;c2^}S1I*0?O={I1Q4oO4lM$csx>q4Bgm&l;*E)NE4va!>u9hOsUXHHHW-g9E{j1qOm16%hsro6&Df9~> z`N4&vS3&y|f$p`U@ghxQp%N2FvQikfITE3>Yl>Kj$dGnmMr|veARb0KUO$8cyur5Rfel&>>s313>o|?Y zvSv2IB9n{NAuhp)tzN$B7%t2)9!e3c+}sf))Qyk*ewE^g*DX;~xJZW8?Rf&d&Q=vT zkZ7{Qx6A&95~ZFXX1Zf;J0G2`p3VrEq_}N}qQX$KHIWzsCBf^Yd8nIq%HKnhfPZcu zIVOFq;l^9H*W+Yg#BBV2c$6*xUHRB?Oxk?D`hiVizMgeIneZZm9CmOG-K>p?Z{(Sr zXHzXbxlH-)F{0|Y1S;|%U}xE8lu0l6_3Pe@T!C1Sk|?=D=pIPJ?%pl#3hE9Z6{)Cf zaQ;AwSKU`Az?})pJCtRY{#7@O^Ag-F(^NJAXs(^~eu65+Q6;zggr=U0oSFXw`R~Zx zB+yIy0g)>JA_w>zm+(*IBve#@GG$jQdpEQHfZsnH#`rw+x~+ArOENOe@+y(%8j*{| zpv4k)qY>**huG)PJ@n2D)b2r$-vj(rSvzU@HHV#n_uaVwF2@tgkFS&gdbXi`$cV%N zAwPAXNh61llqFqEIa9HObH-b&2&_|wiP*;A1Npiubzb4Eeid7}MzsLXk*d(1iP!5l zz6hTu-?>4y8#jRmJG0F==#=)eHnX=kXuI{^%<8G!(`}w);%VKj_jA`Qb}=+$db~JB z>1{kxQCA*~@F2i(drApmslWz7Q)>>xO33%oEO9QO+Zui~vY>l5G}KdRjhbz~zY@%b zYm`7lE|M@GdHN3R!jXANve-CFRHNKN`ANh%i4{KbJ2K9bE^Da0ieC-;n3_h}!a+MW ziAgY%>F;?Mt~I98=6MqQ<`ar}i9IDzVu#c{lvP2aSA+{>M{J-_w=DR0)aAyL_#N3d zP}MUmKJGx~yYRyaGDj$}G^IF1cRzDOOx3oLK$|EE@=p;??vO*uCqA@)Ipq0Q9`@Oh z^bP>gTn3`~cPq^QMf1-nD(n5@Vo9qW&t-=s2wcg~9L)&^Av?c_EHuhIxQMYFs#n&P z{8^G+>*^}XZr&*2jT&>JD5~GF1oD+Vrevn&krDq?yTw#?yTY&=P3nCj!c2}aj75jU$VU4)aq4=*OP{Q$Dd5wCzJ?1a@s+Y1J`V^bLv#|I*Brq)aFZZ1l157l`VQP&|*>V2Vn{<=A%b zoZMAyD1Qf}Q1PO!)YU~EhfHRydr;m|GI-VA15FGkVtV1=*w<`5!vsVQ$W%09Tt8py z%ayG_J=!C_j$F{iRzF;ixX5N;_pg13WPV@Xf_ox`8PA>bFYf9z?5K~S;U>tA*x6A&1%le=Lb?h3_q3iN*f3fUjp-^(@emMToFg;uGzK1|C*vbxfcwrh=PkOD)7dbsu_y~? zoe=+t1pRvY5gJW1x2+a>A6>me2>S&uB|keIb1C#o)gh#X%-cAHa#)N4xI584l>^aa za+ql}I+IYN;!}En&(xK1PfO>RjoKNJz}1!F*Ik(4!EykqN~C(rdG08aV{7wKVXLhZ6yOt@|yz?aeB$-0l8-vB*!|6vbY20Bs zmUq+FFw@y}8e%mx0Y4|I3#Ie&fDb#Y_brB(EXh;Q8hw&1aL7K# z++yT8sK^nOA}B-ZQy){pKvg;lPbRju_!*1dSe_6#Q(9m54Z2?MGuo*c81NFvewb4* z3}s=lq^5XiJ<|FWA9zBN4ku`@{ZDp!(qNaMmOHj!_)rn=_rc)7x!f0rDx29&B(7mj zv$ZW>IZ3a~-w$wDbo&o698Zc64XjS_dG84N7BL(TMs-By!cF+`UV`MOvA_H+0}xq4 zth9mEkQkT+G=J`GGHid2|HOZ;uL_m~h#>(by|S1Lf~YtcHB1!4bwtLjmXf-0m&FT$ zrB+Zu@q0RDxF~Y2*CN=Zp>@3{w)W;z9FNBvwb!kPpX5{<6_ph0GsKyr!KJZn3#yED zhCU0O4gS(~lFnc5z#z;(#eiNe{5C%MI~E!Xzxm7$^e}rpMzE8xQ{*)%l-TV+t=x@- zc|BKq0+ZoU7$EbyQ4+LOKQ$`HA09aYDNHugPm2RyxRuF;U?(362Y^=x9#e38|8E05^4thtw zz*MYp+lLHaaFMl&IOnP-PWevx_1jb>vg!vw@QQ+512u;%c@#mC@vxezH93LV&65?( zD-s6ceCEKW_&h28zUz0$1`mqI&;k6|G2X-wQ{tyS$h+ud2yCh1F5^qCqZ_b)pFU37 z&`z}eJ3XbHT#qI!L$aXgXR-ZOy!`OIX^!0=LX*oKSk zxXIE^RQLix>lqjjp!Q_J&KytD#~|@UJ~WM?3rQeAT&z0;KS@E|B%gno5G`bC87Q>B zQU6K}V`Tv^@$td2$=RYo5cc8xBGEar1H@67h%mxIAz0)a}be&MUA8DF!I860-u55Zo znYiNn^r!l)WX)J!DXKXHXvU#xs+MFVW;a(}sJ{Tpni;HIaO;!ds}&%Aj2o0=<_f=S z|K!3f3ocj}`o>}=BubK^Vaq#$=l{19!YHbg{>$`^2be+}e@-DqBNrDB2WL|eOCx&= zGt)n#rSv~rAYdt=7(z&d$q5Y66i7>0s3MvPmSn(WT5<U!C58^1x3N%0YL3THAvbv&huZk} zdtB`*ucEv^l!1P-9F=~MCFFQh`J1wvi7r#Yy*P?x={ZQUt;g(nk{X^sXUUzrGk;;r zp{g1kh7{thg#^*40!N!kk}8+_1|v&XKjKh&aq;!XPy>MX#?e&_&#>mxh9jL7wzH)d zWS&ABeY*p_HF7rXsIc#k|%GVzvf%XF)Hhjj2{b#J>~Q zpb88&zJRgRM(7>y!%C{?tZH}tR+kLf}6#gch@DPhbS%!(3HNW|*#AAHgXiEQTTjPmMmVNqiu&_rU#d7-_Dx z9HvF~EXD9=|@!J6y{k=VC{i?7&~w-vbHe`hi9eVZ_npp8jt%j#- zp}{LB>#&E3T-ew7x-gX)A35pclsxPdwb3;V^B{`SUdxl%a}e@v^_RwsNj$MoUz4ZX zt=#T&h@MQ{u>74K*PXo-GPPIusJQ4D|G;UGR60}qgJdZXqlAkN(xyj2088^9w}i0@%NZ{5aes z9>{kaZxXCn;WnTz0c>N3vFBU`9Eab-$wnQ}D9YX}qSrKa>FGF? z5_(D9N>x=AJEO}u{9U^~dBI>a+mcrMgXwC#eVRT~80{i;)`?(1dVM1|7iLic55hN1 zOXB*K^vP>z$z^olUw-8OLTt2|y;B#6*k7KjfAV+e*;aUzIY%OsH3K7H+F?sz z`t|nt4l+N~8iNLwre&VWJW-H37(ZFAUO9NTwJrtGoHCT%GajSMqpX;?@$7 zQ^|Lgv8Z3-Gbkap)C1Rb%!-edN4m)eenL!cV7tV?Ag1ps3L!8T=Jp-&H$u2%Ds}jw zL*dZOz~I=VD10+^6&kpSgObk+?YT&)J}Rs3`ySA+8oiFE;$SJIqw9dRuKeu7Z?!a? z3BLF-k>Df!gH4PQhTIu)PQ#0KESB6?tgGH#V(v0wf{)lN5^g@C<9t#RHIKEWr>&T9hav!~?csg|E)ep9*(OjBnDhm#L$ z)J0+&jk!eB>O+44$YVMoavABE0|98k(!`7T*rGyto|Po2h+;?Uas5)eb}6x?es~NZ zTrS2PtD(;5zFef6%Pp1~u6%cGgZ*jc0igrBY@mES;qiG3CFfR|FW3IQP~~!| z646@mIkGkG24lXh`~GLj7J-Hk(p-Tk{PH)xOeQ0>xcrwq2xA`VdOWWr#bQDv`zRfy z3ue2;@jr{kmUh zE`MjFJUomn|C-wWLoNwh*_-}lKWJ;_`X374-*Tyl#77lusBxC~mIx~}bjE4juO4Po z%bl`5ey+8QL0AYl7tUT#pomW%njJk)z8-rC@EdG>YEqICk&qv8M|oy7rYf!rE{xN| z_koWjy>j-NP^>|)Wwx6fP1|VE)@>5UoTIc^E04!&s&S0yYwOuZJx&hl#i^`RKMyzV z;As8vRTpSu%}?Rwbxqx@U0GYI`y2jOTp*b;e5j5MhShtc8cx_lJ7aO((BQ>$&rQ}* zg6$r_d7}L0hbx^`#@(>9T&9nyjB}<@haWx^q%;+Pb%`d=arrc=oP$_O1eSM^7MMvj zz)Ui>Lu%%#-*A#y?ca1Vl{vkBPSz5zVNv}J5V@o-SVGI?Y?{HQGFYUqWyerqp5@dH z;1df8lfyb=TNsZ*{AK$LlA$qJb9W2T^yt8Z+9wMFuQSS^a-?cna2FFh5;1`J@3VV( zlI8sX&?@pA8F=pC|LjtK;RMvdxS6@Q{u|r7sd?&wx{Ur2%XT-qZ=r)_N@6@e^R32q z6xC*aP=Fcwod}2S6RzM%!XiEQ@Pc=tHXRKtDH(ln_OnKLk4CS!=Cu=yG&3H z2Zlkxq$qJ?Wkbv`4oNUy01_1BvOmhg_xhaphxsILlz#1i6Mkj+Rv^Vmy-zRc6xS33 z-|%a6mmh|^l-V?WbUeLJhkS#v2%*$8dwAWn3;#OdR-U(a*2C$@V|gIn1TSF^YSZzz z^+kMVjea-HE}vWa5H|Mp(CgU^(iS7Cr)EEtscj!86yg1ji0>N{6yo?3G(Qb#M7)z7 z;qeYC9&dji<@pUYaD&u>EE|1CO2IlqcZ(JLiG(r*n>?ALS=BPWhPIR>)93-FEZXWv zjPa41T5o|h>wBUnJAS5;n@TioK}bO;N>q7SMscc=o$};@DyggDY)X@Lg?BnFo`$kl zGjKK6;`E2^R=PTB@F019x#&7rZUS9MNX3nDYa|bf;N!ow*G;Rq;cT3 zW(&RCsnmd~EL*ktgi(XKpi5*+dbv%8Y2QJWJq>mEw-$qO%OiUbC{Q&M%_F6f!Lf`u zCy>1@+o0l)>WY~Unv;*pfSRTG_dpgu4%~*cafo+=&`5GM7e&5Yg)jhRgKAx1ToHN5 z>^|ipC!^-9A^--Y6;fTZ$V_CPOh#?4h0bgH-HFY@dKEsUM>I%6;gD*T9g=&jJxp~# zF^R_D+5~!EDXjAbGOQcRbS1tM`>JmfbV*%GIc$258Dz_u6kD8m_=by@!SHy}9%%r0 zxZbVKztrc#{0VgA`~~$4hCGkOY7Z|DDSU(kIgD{n2Bl}>27LnNgi50VqX-!Xc=Vi^ z8>MILhK=_J-xulrW$fHx4l>`bI$!h!_s75uJs~m4e4}@@CS>Z`ir@~4Y`h*Bz7$$`?gJ<`2C4? z!{G(`(P0G4pJ4pQZUDfef-iRyIHNY@Ar*i((C=uU_M}+5+$S5c|k#}GK zf01RQ2pj5BuJuR!47=rUe&e~jwW7>Y%&srZVR4H~oh_}*rCLg906oW;2(=_Pz79kc zoa!$_lX6uazt3k{0Nm+s7xzUkssJbJ50j!-4_HeWRV!#-5K&m9Ps)S;g@Y=V=+keU%MhUBgC#8aJcv zS6IeHQ#2O>*SE7lTm(ee6~+4U?DDz-8&BmBsK@hF*9*nL(-z-Xs#TqeN2@iWz16Hq zQ>ru7RZx4{pG>)n<&nBYL>Ah$t@0(3D34?)MWqsrMcSEBic-f>edM&~@M_D+$$F70 zfms5)1G}V$5-*8yci6B+Ns>1#wMibTi><0JEM+zT-cG(@(>2NA`?i1@SXw(}`W%^1 zpu9vlEz<0#x9+75e+(XJgA=(}+}K0Ab!&W55Ir$*%eqWT@l%z%_AWC*EG8?SqJomd z76%Ioeg+;iBBFepc+uCy6Q-Ql%fv;;pTp+nId=>Wcte#I^w-R9o(>2bI#P<3N21Gm z)n>ZkJubh5ieD7WQbgv8 zLq~u2j1X)#VGMVN5E518i2trwpK1S80N{+zoT~9^*5kK~IA2IqPWc3i#4n6gkZ5ke`|51>SGb<@7@LG@^cOq(JeUe+E8 zfp}&)3-6?_Yb---Vdoe;stcVFhq5DaMJ>>W(i5?CsY{{Aq^}|$+@zgaf_Bl!`N zGG#e8k#Lr$DZYC@ooh$J+-e9`DPIR&+WH}$Oh>}EyuWl`4tZy~Js*fweUH$wK4Cb_ z3#Phf>FMh0ug1{!!LcoFo!x#yS(`Gh-roQE9-B66Ze-&683FH+jc!j(=}Y(YFgq0e zhPlz#ympM+vn`762M!H4nFGJM^Dd$~yu0{ozn_xWW1f(7Bq|WBUpU-(G43^Qgglg# zpH!vo-AhNWpYM)ml}=IdY&5b=+BrDY4xB^pn~adgn~acyQ!?b+AaCmb0%g~Mj%*4= zt&v$_sSl{8w}8#yzG-k8ffG_WXDU zQ5k8~nQSy$OI-QGe%ybqrT@oadJYF@(0vhy2ys?7E3EG!=*u+E2g5VRzs~`M=JSlA zz-by3xMD&5f6f6SM)oFVwtt!g{8VflP?V7P(w0Ftwe=zBn+lsdF4l~j)kD|VF$8P} zn~>)Rp9-{6X;)!8p=C@yQgCEYV}E}VcnOYi!MqV_US%4{1$dZWWvn^~@cTj3MO{+u zjY=R5pc~bQ*GJg&(@C`vkRODBlSeV#8KH$}gwBSvV-Gun(^5&dksGBD&+?by9@*G) zJaokU@Zsz3?{y6;e+aZ%p>W{a9f2=E7;o8-3(GG&5A_S{jho*&@ODiiu2hvP)#KgM zTV{zZJVbl6u}{OjY>0^E_qVx=v*B`A=(mGx|FH>M`f+fcyn>D@^v9b*xmwW=2!mASXdV_6$uUV{Y9qN}y{hrHaJR4(Q)_Iv zwuyx2Se4q;P2Yu9ZG^3o%Ig?L{ju@3c&Pql)8u*~(2LU)W&M5;?5q`I&1_3s&j;); z^T&K13*-ZDq6=hs_nE}P+wbHU!3M~8B_H(kr<$ae7FV0CQLbQcg$fYbQM zc=-IuGBAN6tDu!`*Z&T1MxCC?7pO3q)B8biBh0vm0%6(}FIfzl+N5b45P*T`OGs2U zG*8=KPPmIoMqoJp-Ioja3>UF= zZ~$p``JaRlJzG&ORn+cYMK7?bO z)ghuWQ%B}UzAj&;-k*HEz-Gh%i0EeM>sqRJk_DLqpRH-!)h?J#S@G;;8AK(&E!)UpCG)pN7C8DHw>?mcC32-`h!GAO`!lstA$@)%g>;kNyW>wd zE7;Noa?04}-JlJl%&HEPT7?MH>Q-1C$HbT$do}FPA_VSD%vuK%qf5L5N#i83~qMDWSH0E5Wxa z4F{2`hr~@yiZQ?IOYji5ISP=Zw906pBaKe6u;rz|`#VV~xEHt93PkBUuxL>Ip=kVt z)Sqj{rSiNYux9v^#VXkp9l8?6e1|S}`wW>ntxU#Hfbsn2eZ*6u9`1|vLGR3JuY+}I>N8$Ec!Wu`=&M?zVq%(1l zlcDw7|G-Lf(&@*CpbQ3y{Mj0u54F~<%%q+h3V36peR-TO^go)4)s!=Y<0FwQp zk`u!6=?s@QLRjw7{5Na$^4?0-B=U;w`-CV9x^TIkW5#?X@swo2>$RZ(UKWL=8& zpiyiaNzNSyZN8!3%bk{OzD+aD2wM;F;KCp$X#G|ZgUq8if-Ox+4Qi3*8I3lD#FFl(Kqw) zy|2&*yTk^@bw*@=@u?7|Xb~~f`~E9_z8>MTUsG;8VbIVoFxSCX#GO}Y*0SI>S#Y6j z(o7=xVS%E3xVLC;PCW_1G>Q0@_jNaLH&EYe;&7K&=n#a!*d^jq{ul1g91E)7g|C@v z*a4a%0b~KPH#`SbSJBJJEUsSw+M)%PiGVPIC&*ui#8cu^i_}_U#3(iUL589a#4u3L z=O?W9hzQISH4-W$vb3msc{6S?dw+X5zh_Mf!vTYT5V*th2h9zzLfIMp-$)K+VDAL7 zyOO&k;Xmt*|>1Eb_v(*%~5fENn0G*6en~;X)$XpJ{R=1ZRlI zEq1)Hx^B$X3ype)Cte~+SCjgf>iDgDRJpS{`dNf0`0c&e-zV1@w{!$r#2yeiceWft zE`fY4zPw!c483MQ^A&uS%l)-G&6%?apTOE&C2%;}Ll0Zreh1NQgW@DEM!y>|Ly^r! zYSFX4-^-_O;kd!&g7v)>#OOy=_w#wOPY&s7>;iR4;h9k)ghdBZx`b43KR$jIsD~1A zx`@(9YA@dgCEC{{P%~XJT}f#^mCGN2D89YJx2fwkpoq7UJIpj8v7&&^z~fc_c$t;7 z#ErCp1i6G#NHx?q?Ed}(@EF!dc{#F!?t$3_U*YE}P3Z9tap+LL)s!nq`79D$ZXU3H z-9niDvk!nw7R`MXe%Rb~J>`)tNh9&l)ZHD>l)UuG9P`N-EF^g>`h#UgHI=uqw+Lmr zw9WiH7Ij9VOk#i})l+@<5DW9yi$(FUI7azrs#8Tv0aXd@w>{5glUabWI!L|!3Itsn ziI^g+WwWF*d}QVd{AI~!BlFBD#lzC4qbx@LvRNP>1oc!Db36qfVds?8Zt*Jc|mG1UbZQJM9V7$!(|q%M#Gy#3a>5z5cFG{8!~4nCNQv_)TSv~v5K>` z3U zOvRr&x1wLolT@DNJ9voKCLV&_-l{!{#s6x=l*p@fIsCxDxUL1U&sn*{pK5l1R%NIx zBmjw4Wh~c66ov+GA~*~Nfhg1+-6F2&h@6HMrh}@7p||UX{DCZ&2%8yC>lqUiE%U-5snT7+4TK88@ZNU%e#C9vC3?VGA^J<5Mkq)1 zztSHNnGrk%Iuc(hupzI?%52a6xWm~v1WBau=Foe+b=`rE?`XT&wR&Xu*W22E z{mQ_b5cVtp8A>_CKXUyd4vt=;My^JGnlrl9|H)FC7jxjm#zM;)_PT#LaM2Pm*Pp%W86ZO!;PKJ)e}yaHbDh-y1l050^dvJ)f`C;HAXi zrQk#8q6 zRqUt|#Rx0jhy<%InhEbXQ+;~2;Z~aL}ejO$CFlHETl@{U^rE5m(fg z7-SOSIoPhRmjDWL+s>)qMIHuG_!Yt?VW?sL>>b+H(-`38Whwm!6_qkp@zpFtO9CNAI-I8ueYD>-wmRm0Cqt~2UXD|h@fBwVsxu1V?at}bj? ztKwL(iP9nwN3kT_eYA5~gp%B2oN8#$rt|Ve`1c0{6`_&KK`v28Y2^w8sF{sviW>@^ z9&s^3Hg>BFLma39S+N#%JBgIsgr<^U0Zga#$VpDkT5`4^S)%3f80Mh7TpILbi{z_z zxQKg+vD`|R!l~M0+;JE0kHiNh{BEq-Th82>*;or#-pIB%#6p=7{fqfqItSi(Fm6aE z!5tayo|sx_U$v3t8#nyzUM~D?(>t7=lKpk8f-}vq)>7S;Z`daI+kNL3(PZDFwO|ys z9m3oucDv%PnV~nDYN6UaZoV87W==SMwd*6Vo^*d+$V!av`Y}chbUq~l3 z@jP;a-^~})8&T8!T4y4c`4UxPC4U$3zL%_5?yZHYIdjWCC1a^a`dzyQghylVMp;eD z^JZ&NB1MdRfWfC!KKH=ATy#Tv%`x8ln;cGAy4ZmxqZ(c_IGdE)Du-2j6byq|4%Zd} zDQ|MI&hK2v%hwT?p{U8nelDBNW(jjs*dT6XElnr6aE0uKk#??1`NM>DAw4JSV~|p3 zN1C)E@?Ty{ojTD&OWSUT)Cu#tyaSXliHln}a)C%%i=1Q^U1`rgLqF0Ql;}}9mNt67 z7;GTuv7mn!kn`5+`dl~Rbte#OQi2nDbMmG7;2WhBxgayyCClk&$`^85EvwPcl$6i& zA==B{N#y14hsONR{X53jd(zUZG>wWxj+-d{mXkum(PkZ`<0xn@<35-3i^S1({?eZstpNXzLFRIIOKr{J2IK zRB`nJr=GU_{5GA99u7~uz6npSp_kfPq`fFLNE>jYpGn}~=RlX;t4VZ6nj<=Xcu1W6 zPPjpv^z+PIx<{g`ew@kN0@sMkh4t`F{nek<4%LA{Zhh-VwysdAkMqttT%x5>WM{6#W1i=dVb%&NjS?90*oG*j@tK9dRm7@!@XzS#yOS8J2KPfRPO*S`| zpm3M`G?03^acu7RlTrt)?VYIfZjgSswr(kQq8)9`bi6Gi%g^gh2#>Oz!0&=C$=o;a z&Y&mk2R$AY=RhhPL@QUGa)j_s7>8;0LK1HE4KpmNo--ZYz{ev_?NN(RVc%gR(l|{* z>kBp(gsPFI^SLn8WE4vo;eXkf(dSBI0aM1$id(qhSstcaSdPpV z0(VJP-(7_pl+;t^%qx!L)-+OBK}lXia>j!8#e{-37CYM0ma7J*A`4&}CFYl#SwNGq zyk~tmwT&U-$A)2NrOAq3$iNcQZL3@|w517Y8QMn-FjT~GS5l@T?$e1FDGUi+BgW}r z69EJhgqcsk<7k8NujS4{_k5o!6J@NXHzWMA=DGY6c@M&JW6N2Dk7@*+>x$!z9@FT8 z{Y?kqu*9VtTP)}HYsUBMo`bc-ldl&ErTE{Jj62hIFDpDjNVl%ieINo4AYB(`xsLVX zXjce6Zz|3^KN-?ghxjO4ec?=zhU$-0)eezRv0~#*kPZhkiY>cARvM5K4Q3Ges-l5( zt3C90uBeoFNyib$6~zFDKnIhAsE{2zP*{o-Lyw0u8G6iO{eiG?i&%w{KTPDraL$=c z6Ank6Y?(wa<(F^Z!xTHEx9S&LyHLZb{O!5jy~rSQ&2Y@dsarF*1hL#pR*V&`Q)MGWBRW z+4!h9*=rwVlBZ@yZzN0=FEL8~6X?Ga&i703RXjlC(16JOLBjD55PVp&?*`BK>`x*wov0rK1__ z!QP$JL;Qa3S2Et%PP*drIwG+}tnO*~wH8eC$Hh-+<*r1kr2G@cr-{4jUdfp90gV|z zYJmwtq*(hzx*@B!mqnbY%tWHDJ9ajX$}*SBisp|)4d*qurqYs?zfl;ycd`C|37%;G zOZNN&gFj`@-^A5K#k63VGj+plLKtLN2v{g-&E0)CC#;u}J4(KAZX$(|SAJCcO(byt zAX$0-Kay3H908pReO>BcKB6F@|8vwoNLDg=yk+E4_(SdKJya9v9!I))$2Mj>=)4K3 zN`6ztwG9f$%cCvq^*mtR6e+e{r zTyW~IhG-B(e$E{S>G)Q8_~yMLbSq$RsWkk8d6a6EGdkN_a&^n}GKFct1qw8PS|^}D z1FuHS-o?RxSno__=Lf$Pn~0m-lZ;~3uR@HGZ#=+Nf;gTiI_h^m+<~P4-yi-JK!rx5 zust<#8c?B;JcB8zaY)bji2jw?2`EeRi9&;AFZi*XkXFKfuzL3|QqX=`GIM33S0VQd z7D`gMAAdbxoonpK&QIKo{ zoUu(R0Gh-&sBAF5JjhdOClpGHUFFA^IVJ}#xIiD`4bQ@y<#R8snt!sHLQrDVsQTo+ zY~Iz@)jXfTmv>H3rs%F1CUN1m2qkQjHWCv&R3TmTJ8h(8vyI;3whgp88q&Csl_cXV zSR)(&iZY5lvt+ZzhLdf$k>O=?OKpklWJR&G+K7MUWjgNwk|qn?fSx&)>U2|t1Itr` zd&bu?m&ysIs+v=G`sQi;<(cMWqf++hnt^8RarG(u6W?hz+e!wbRF*QwR@XL(D--Lk zb*G%=bL~(#-XfwPE{L=l%H5VAuZ_gI{2c}iY?3TSoDHrCe8=wHwW%0M)jmEDgj2Pe z>lz#i4QjZ73fb7}m6jiDv<$@6M8V_`kM&hMAQV6EiY+=z4P7^jw+pk4|ho zZWY*H3bKFLY&e(O4IKBv^Pwp!D$iJ>)#OD8*fD61HGOnUZC2>nXT|2!RPN%KpS7EG zw%BLkx|XZhw3o~i*(R$;K9;>WOgt?~^IT*a-3k}A4CJWqdeP`w+UWseJ-RjuXuKbY zn8Bl2tye3=fAh03rmKWdIJ9^5=XlLw=ro>aw5o%1-+%NOL%V7JU~=AqHV)d-P2VC4 zmVhV8>2i(B&wBvr5Rod6f%}sAAMVRwJ}$V6P*_ZL!HQO0RgoI8AV6J4Qd_u? zj9sLiRfHQb>8fpXO~_QU(&4l(97{jEjOchws63s6Aq(4cud)pmCOR+`tWd(5sTd8z3ZkP|)EyFo}~O@sH_J%B@M1p69GcFrAfL zq!BzkOxKEUBDb;+h`liEIyVkTJ<16#eBk%FPT*FPCdWSa2>W{n0Q98s?LhAUQ%UL) zKI{)$VkXApSy=dF8G;`XaRVo6x;EmSS$LA57M6QCSpnbwJ^CT#hf(Z+(H{ql{y&UJ z|6siwJpM!YuiGmNq3MYPg5A%|RW>Vj83$Hqp5xn=tUx8PQ&?tV5sspXM{!|-LP>x8 zfYPzEVZ8-=quS5i6{KlOnpg!iF6qCrzy1EXj~SqLAOdZoCBg!YUyj79ncCU(Ru$ri z`FL>E1I{AE+gq{0w@~+O7g=P5PQ$jyP8*`rjtjnOl*v9d>!Giu&U%V}U{8J7>TXg* z0oVJ=HZ#zmq&_dkwct`M{(KXmKl__ zy1)?JYN!Db+_PMRyNEw)eT#$N{i6_EL}xR~lGAP7+R9TFdnrT(DU4V|5k>T+u6!2; zf;68DRa^EOgM!p`{fk%(xr%DDc-r-6Tr^hxvvUeEr#r+i#Yx()eFcmr=kw{E;c0Q~v%S>{kFglV@fL3Ii6uiVZNR-WZ1__*Bq#C-`^Nr^aQYADr^ z%_8vH=!Bm+WYeDqd0?q~u$#UbU3TeIZd$I$QIbMlQe;HQF>M0|xoY#ZuI|G;`Yv8g z%+eK!5$qRPcyycB4)oaS3T@kQioE^2yuG8-9<#sgr%oche%P|Y>P_N8+Rs!X&I=66 z*noab_#V+fyz0?Fzy;m_!o`uQyJIvX5$h`_Vk=k2O0b(W}J=M&&--c(-L2z z{J}>u;6^hA0EOa@KiL?h*Pz@(=^lapWoA#hJ2uf_DJaf0^fa5G{K+F?{TflFOknQ= z_Dg)^-=a48h_XWt_$M|E{JZ#5)c)hs!uZdFz3RIAyg1q>>Ub*2w}o_YXo_8BhK7`d zkU3!~Nf~1>`#0e%$qf5W;)FqkasE!3)5M#>}06v46Zak zhF8sx9-m(ry>uaQsOk7(7-T3fXl(toZ%tfpbh}Xudf64432LxkRuqjVf!E3pGMei!=uzosm1P!imX%r@?GQ zEm2x6V45W*cQ>L^n?FQv7e$q)uMh3@ifXcg=wPthU+C4~F@FpF4Ykc|!q~P^fjDo@ zd3*YfTe$|pOb}&=gfjhTIQik1M$){8c{%e*+%M?h_!-mi`NAdUpcF{(AR-0o1qgkfbBZh$zKhCWcqll?BfdrXVP^Mo zncC27DF2YiBE@xw+2s0*Ims%zYvPn&6(H0KV#ok$Hab5&=&C!wy~rxuknIgtZA@|J z7o+_RrO3=xcpR^8;=tx+67z}9rB2~c`L}FG{Qtw)JI2@Db?d^7oraAY+csBh+qSJH zX_6*sY};02+qP{xY4EP@=j?sXbMNloem}0SYtH`|b6n#BV21b>vs5{{BdRy?*)2YN z_(Lb1NG5i8PmY1D+LB_+&^jZO3rv>Uj-eyPJ%D&Y30Gzy%4mTsNGAH^DI2a0>fd{b zJk&ue8?cx3{v`kkNUHhYR*6B1qqegGXuL+PtPawON8bYg7CC+yiqbhe`S}UiB1DSf z3SnU*g-jDj+N2J~tqQG&Ym*_T-fT;sP1{Ybjn6yv&c+Z-+@$ zpT5}30bocg-Q08Pa^v5@E~B&H4;1ZsW9rB}$NrPY{oFRJy6dxX8> z4yzWN53+mnXSo=noWXr9YBn6xvRznTes6hC-=Dhvgo-)jYB*fQxpuhkR+`b;&q>oF z3*By8yvj9!?8Svvj{?1zb6SA>Ub~}qgq(qkj{MR0Xa!$O$#A}1Ihq!8$S#Dg@ls5X z@MKF85;P!{Ww1~m`4JhZ5PmFfBs_>)S(3cyfXCK*lr?q%_+c z*~29Lien(2)I8e$C4qiN}b?Ci^aBF=s?+`-5Q`a1G}< z5zP~;<*^UG-VhV3y&n@oq5hq51RlXEj4W9Y1%c2j%oar!P{@2A`rvSrPM9UkmXdMm zXR>T2j!&2v5K2cO9(6<(H*Ur+sQQ?*7tttg_L;HX!kIyu@kVjLJ&LlY08pG3u-F|6 zMnb~lMO*VwgA_+E>&*a)}1^xiAzAajWef+ z)8bAu>#v*oF;x_``R`Xyu5VNGk7$Pycn$xVMGU-xA~yEU9ttidjvjx1(fp^sOo9+? z0D zS18*cpPERPgBfd;V7Qw+c0WTjE?vfhz&=kABSx;am(KoN+>|1QQn7sRGQ2Mr)F_=F zLrB5bG>dtvU2Y7Wnv@Q6P?Ix%kbWIJpZ67^aIR4_TxrxT$STx}z5a+MH#6H8FnoT> z!Jr@-)CDAUeHAhEWzF&4$lDcnl~hxjTPW^?CT+!G=XCD%CzgTf5ww>Z9-t z*~m~O2$p~_MM_c(#U2EX9OLJI91?+6_19eBAq@c!=`S8YBDThIra}hJMnK1j|E>!0 zQM8epl}Gcv&^=NuL&Jhz?UGU8dN3f|#t9TovP_gVE5;)9W{`1PYq4IprU!pVr-vaS zdjWe=7;qB%ZaTf5Kk4K(=`?ZdVSDN8^8ureqk_kkd92=D8sPoK0*wU?y^2zOlIZ6l zwj}ll=&j7}x3q&Ku+`_em!qY~t`&z_90aQ(G|5dTtM}pu|J{ewi?D14Fnb?acG;k#U5No=4B%`<^{!JEK0c?CRc->CvJ|lnOs$Ls z>5=pg>$4Ks)1H{J`p=tWSD1*a%9qWA+@`GYS3cP4$X7uiWsOmkfWc+CDU_M>E6JO^Y+jW(uq|qCs>&r zF8^`ZDzE&ie{|u0n3jM0BLAPm{{MjhN{T0mpkgmzVqgOiKzJYmXv8gq;GF78@2ZWm)sFixa+811s{dNY|Q|C zXBuA+`|4+H2n6&oB|08#t`7gQSam5T03q2@Z(>S>(H?;c{!^@cI zML6d`Jk*$dadt4K(xkpq)^Y%61jTws7#ITj5%_`Te#7CNt!zp4pwE|=TBU~6+=k$H z&=FI1TSuYx7*ZLBb_Hw6{mWx$Sg}(-K0VZT`-x8)1%SFjz`I z#NeWflI5#mWI9A!=$^$VPd^W}6FI0z)BRyuHe-^Z0KZD#lv)Df(0Cxza^Sx)Eu*@b zB>vRRTHq-77kc*p!TyaUW5zt==PK4|Y0N zoJV9pM?_M{e%8Kdw6x)ZuXo@m@r;21Bj1MCl!P zcS8pA*+1;D;lnnsg$DP0E*O<%HJrT^&;~*cBJB+i-y-oKyGdpdjcn^5p|@Rkg8lXR zoWR(D#`X&bN^&{2=Jipr$y&ZfCn}oG7(WIhG}8oE;G#IczZ4#N4nno9{=!uxtsoUB zc%C?jfu2gJ0nq#)AMY$!p0WH;!c8gHZI2qVG3oK6#hugQ&+AD z{7H4OlNtV->gyIE;9id~C{2v&i@olE5&9qa&xy!dl5cN4L~(bwo*K2BIdDd+EQ_cD zV|Ruj{+Gr+h6v511qcET`XFwUtV^zdQgG3h+Y8_lzN~JCsa#qN^9y40_&Ogs(3L*e z#PU~4;l75y5q#8ex77LB@oVdNmb^mPujO&Gp#olmCN4YSBOQ!m4KNL3;UCBJ#E_&3 zEYu{VFOp5*k`eVTTtJJ=(|c3~pF?UUcrfGpbCCQf0n6%uQDIbPJP$_%<-5tDgieBQ z+24M=w#Zs0COQW=fuRMvmX*-qmXdgPdaHolZRf;U<25@CDCIyD76kmem z=29H7b+$!MyXd+N9~54|-?U^5mE!QTKhHZ~1Vs3=!5^OS9=c7Qx?NOlZbp3yH_#Df z4+cBxK_FDdLKZeL$3!+%4oL&)vzO{gmiAjkxfn9WW?{q@>HL8=}WecsaU{Z`vg$T3N z$BvCFuyP`7m^^i=k+~jJYh#ReCCB9|;pf-2HpchK2K{W%WA}#I`ulsniAmPIOw=e} znZ!M8tJ>+O??G5+ydC`2s5~>__|XfIoFgxlZz*qo^-mRuvdN*<*Vm2KxoQjIf29Nx ze1R}EH4rWXHzggT6gSlEMkUN7BHd02I230FPy0^M6i^1wJHAdjY{3%kjv*#Wm6M{Cmyt_<-SZp=Z z#-bH6i%s!+esfTQK)%mC!}o}Cz+F*+tubP(Y0oXhyBuPqE9xT7hT4^)p??1(s(<}K zi%jDp00y5B3S{jf(unznk+B$_#Jn?Cq$kxsxo2cM z)DEn`1_=%P6#hbR7qxczW4-m4j{vwF17$=Dnhf&qQAY=^C~wVJhYSsO5O;&kNv4xS z4EX&agNNWJ&~ohx=olQf!Jgy?TuJE2EaM*Xc@H<%=z$6ZFDUy!UQ~-c>AjkqD4{Pv zZ3qREHR_)ggLN1C&s?x zOD5EOCRp4re_6=y$dm*6#q|Q@*?DOsTS~W*ZDj62@J9Q7whjYJ1y&JMB%FjG(VpsF~ba- za0Vp7AMN$JfxJ+xo(#`MGxqy0V6#|4882RZ#?gz?qs|LhbQTGP>y(J@lZHre2?K3(8W zGf*9EuQ_=rhM!K&Q;3~ZyrmN$yva#N z3CLH%cdyV-=C6=|cOpj(PNA}Mbn0>$-L}~%^Yocr2UUDD&RgC8GVi2OCY_iUpT_iz zp6e&Iw-1l&YCYK(JyRHzug=qc=*;y>rI=nf0a-c28Zv7GGbM%a!(H)$27vMdbj~yh zLOUXz9@pQ;)dl6CMnJW@4!XsgFzX}Ck4fkXC;wxC&~7xhHKG!(U8?DfyDD17dL#Oq zP$gQxGjhD9;EVrN8nH0SH1b4D(G>Iv^r2PI6sGo-9}8sG8m2U^?50z=DVd>f_gJzi zg+pP{-^&$lH@n>Vh&z* z%tkZEQXaMX_H5pxvJ$ji**;b|TOCdX-K`&xY%l}jr9WF?Mo(p#DR~kMC`6e?+-NX| zu$OA(Mf-)&(ZN{ZTv(uGg%;uZuEe!rHWLnSUPOZ}6+-2u`v~(vM(lnf%;g+cCeoF&bS>W;wtI`nA-0F0n# z?lbyFy@&uT7XOSl2SN+~b`XCn7>Y7>Ky)izX3d&UJC+OrloCYJ-}_UcH@djFzhbDk zU^bdaZqmpvwQh<~;?;<)s=E|Xv)IMOKV#ayIj_`red2yMIyD}Yc>wFR_`%u$K77! z7h60#^D4X|=P&2Ho%Oi`Yas!)f^It7?uqMeSsIlc^XO!RIKpLGKBHQu6!EFS>$AbJ zCP%3zRyH#zo>i*0%oJq}l$diBHf2^FNR?%7<|XmiDAJYf?0sDrhfU*M+&x}z1_FVG zB|MebUlZMFv-uun#!y@W5G369e}^QF?2+iQw-^l8PpdNijH3eT)Aphw`8n-d4>%f# zC2pxMw&(_V(WRQIh`wW?f0kEv0JVtxL1kIY96#7%v(oacOfCbc59B8JLB!H^E#A+h zI<&Wx!bntb$u26syKev#9Bdpgh$n0E(|H9e#UTefOJVk4hly^Y;%(OlILHgn5+WH7 z**`qN$lIafGL5}`vjpnd)LNbEilvmA5{bupw-$NVS6$x-ZKh=8rbVFEFGIr9N<(lz zMGucoPjf+v$fD#6m(gq^H*1w#eD?|RMoSmXbS3TBqXwDR5q6EHyowTk6TTbgi{DHb zagzoHlX-3V@(Ou>4i(r7rx%Wy((2NFYK8Yn5q6Dv-MPh_$id8ICgc(^3Z{4oc8g4lEeBYB5IBq;Xu54bf21NG^TIH(&4xBi8*@#jK{T37?i z!e4LWf2x99mk}k80p`>Vd@B&;Ruz+muUpiFIc$`Lq1S-Au6S#(R`3R4prW=QRP>u4 z|A;$OL!ZDYS)JQKCiCIM8gu7|_XmVN95)#qVUhsKqBKyvFkVS+5614Y#^g~`Y1A7C zGAf|kAMk?ivbIno!La&6%vP$yVI?-Y5|`U&(3Vqowh~fj(JB%8x~TFHauRw}wcw)h zyyFbgcD+VtlTSa^d8ObLM9^!SU0rBSc1_ht?ZHzvE-6(-HT(GlpyLX<_~^-Xfn=J> zH3FBo3>AMdcbY43jh>f2 z3+j$p`EsA-Q~S?*P5jL;J7@;B>S-}I8%;j~g@0G`pYReo#ZU+K-y#b+>MazkCgq(X zCDL8cb#rj@8ti|t3DcUmt{pq)FA`X5&Nj$$jQ&{-MZP4yH?QqHMqm+`9}oAGl77trTvx3$*iNfZqxhY`WeU1$Q`qr zLiWyxT^80+GI|y{y<0L4?!!(v7IHG5CJ z>Q^E?Ca3=$W4N`@0bGBNBAb~xC<0U9Nr9nFp zoJR1~bE%38{fyxJ{c;}>K)z0mlktbvp1$&U(z)jgr(Mf@ZA#_Ju}H6tV_hb4L(JMJ zlO6P7WFY8w+V3=wQltY5JvG!7e$iV>+g0@BqJ+T4Y>ZTzNp7PNH3c+}t_whdS zrP?+sR%LC&4C;HbWe3=65vR+?O#!Xy$a|Rqzug6tC~wSYFWgmR2C-gtsOQSCrbS`} zNC$tVp4+1kU3;{ecMdrgNw>Glkxn?LtzT;!CLh^^Ejzy=`F_J3@Ur&2KG$UhKMJz_ zzyd8z9va##>LAm>&Ii|r1@BnZ0f$zR# z&C-^Ts(9PJJ4Es>|BRaZ(l_gN>Ne!$mziuP0nR|EK6mKoli zAOrg0YlIu{;oodSzmN0!?3#9d@(jPux@T?;*dKufA0)>_`(> zMQ?h!hNt5{PP0l%1bR^WhwyoEukXKkDy*vmp&g!8+JH;F5{ zfhxE*dh0Kii(P+kVn8;Pk@j5hI1}kf1DfT)gT#ShiqiBJ|o3PuMqqSCF z*($i>eM-H;BDkZvlAfuWskium3vbi;6`^i<-ywDsa{#J0>CWXmlfV!VsH#zR5+Kg7 z7l{N{ma2;!&T|)Hi)j^1Pzyh-I$j6NI8==Qe_BTjCB(2xX~=8V86@`~g)h-+F#6qy zMRtQeYJ>jRe7KsZQNO+;I{(gci3 zc3QE8I$^(8TDmDid}#>UhIO4nD!Qbqt|8x+f{>|4(;lo3;55LEJ|NOf+M+Vwr4Bx` zSA@0Q_6cU*Mo??6O2jx+DRp#jH^fnDgtk)VB1}>kB&?uw*jz!#F1M6vM73rvHRbwZ z2T6c#DyefFvaPL7*z#rmXDJK*p3v)^y@_-cz;Q*{XX7%wBq!SlMA`8qJ$~8N=|{_* zX9tsZwBmL$bty7y7m3S;llih60HYw5|EoF*ty2%2gaWW=e*T@gJ!65|ZNF_CY%sR< zrSZ{Y`P$>Vmna7b3m7r7f?lmK)Cgd)XqB%>;7o09M7*BjlX-JRMn0ZrgCT_q z;YZeY&N~>!-@=wwp9l;;1eu1e?T+2T)l7GpdG|`|LVI~86*GmDlPz7?JjB!CLKTP{ z4=u8gw$khf59z$4j4z0qLF3MT)8k!B`F!jXzn8C(SK^Y^`WWWQ0ojVq1&A)~NGI5* z22=z+;n(>8UO%KBVG-Yes$CP{Xb4Pz`($KiL+@(h=xkwRKo6Wo{>@0uMy^}37Z{td zYnw(^xl&4-32rAr|MM^mqaBUtE1DqB5L=~{WLk(0+p_Xs1SQ@^H@?a6 z2`L|AGv;0Eg3oB0(CDeq(C3?iuvX@%o*$Q^c)p@nao)D*UD8a4JeA09c%p{Zcbfu} z0=$~RFT|bt$2EyC)ap7zS}xexdr*DNN1Z~MJSt=o^BU;C{~>x-ad)gM29{iLpwN@x zfBf5&J?u^Xl{gZV2<46{_GNI>a2fm2p?hFpz*+l~QCEC2lE6}6Z)A6F-?n43h;AB$ z1`0LIRKb+jW&0zdQ5U6H&wcBjC-BDxee+}HL4c3mMRn)pN80AuYDR5qtnRnleKnBm z5pfAf3~v$$K3IIGUMbrrcUT^%tZa9ku3Jj?D}>Ugd_VL)IT&?`*LpvdzDa0xnb&eZ z*1ky?b*a}jKbJmwXmz>QTt9?9dYC2g*IGZBzAES?>DN*}+P*57CCS$oKbt-~=q1_L zLO;AdJD4Sj*9JeGzDwvOnb%4`-o8tiC8^gAKc7CJ&6C_~jvrVbJ`9KWYmFbEFAJJO z`nAN5xGxKaL-MuR&!|rinnU)rzz?-g4~9eH^_QPg-zGGN%xi@obKfQmhtzAkpHm+{ zG>6=4o*!HvKTMPOYn`7|Ung{v^lO~-b!7x zJ6fP*YSVSJz-CgO*f$6%3;hA59uL69j1E6Mj z-}RR~w^Ej@*;`eOPcutYmkyV5Lh#_XczCqkByGGXP}0zUr6+^T zSaCK{{_;;8SfDiR`ynM&?vkGU-E6Gd&>5w&mCw9kZrK-?Sg_MPx@K5Dr)VQ#aJ@v_u z>2qK%)4bX3MO5G7evw!PU2910dHMHxyf&bRIAe86K}BUr%d}@P`BtB*!f|Pvt@v6G zs5SUy`mgLK(o1^9svkzp%q9+b1&MtATCZFs!(G%!)_+>@&CAnv&5#@gVf(ec#_zKk z?4@@0t4uOr^!i-ylHuqD?vPm}kM6B*k4MPKwncb-3c{7g)Jr_lM8Uw=ke zKqprnv@l}E(typ7TkjjwJXhYAQm^7k9EoRMM(H(Vk9y{Zmtrok!JW?CZcSIYX4Sae zl81~&CccSFUw~x`z?6UL{5iXo8IulK)$M~f z*$Zb;9bib>PZc(gW<{}}Se~aYSL1I;+)oxZj;2M?plF@9BwO>{ki4HRY#q&u;z98& zZ%NLU1NzuH_<3Yx0c@eeWcgx zljdC8?C$JaDQjHmRqGz-tDrn{iJvX(C&!=z$#Ip`GS)@&%6aXgW?iT62cqLLscEc@ z=C<=|h0QunKMrKaby8bdAI&f3*NU3;Zv74f#{sE7Sfk8=ZXJal+P8rR65~p#MXZzN zne#kF9=f*y2O{HAsY$F==DPE3g&sP$!3Q$qTB(wCfZK)xyz$o53}Fp%JMs+;43gf_ zD_G1c-+CUBw*m@=3!$jr(e`n?K;Oi&7BYNKdlgH>rV+iszTaD6828yp7;#~a1Rx(E zYy0RT8PU!^;rO~l@cZyd@q6~lnm4+zrdV{nAhgGqF9Kze@S?|JcMK&=B&3Wb%!b7c zhmB)w6KN8yyJLG}sdrs?mv5X6__XM;s%Z47j0Ri6UN#-^x2{`C-6LS5U$y2;wn1Fg<_xs? zx*As-Uy*W7sWHHob-rGkes0n5I0V~ZwzNV0qc+7FZjXTtU+X9CglWyM5y=L&h9F1q zG@$xhwT9O8b~MKN+!};4jCSZtY~FHWHD_%q?bw$XqorJHseSbZ6YJ4^jt0!t-+S&2 zX8XjCQ>bXZO6;v@@1+*9%F!1#vWn5iHm=k|H*9baoExU0I=skBNl{%Bpu)8Q8_?dJ zKaopdC6O#(UkGQ=;RTE-Faw8_8NdnA;|XiglnH9l;Q@OT#lm}3#X@(=46rdp+1N2v z`ZT#o0oNg*5Ue3W5K%;))b8*K`T9(`>i$sEVuG}}!rd&wdJwb7Z)jrzcWP69nAzFL zjPl(yeFPA?efkhiNC31kKs=SXXgpjdBM8W~e5W)d!_k~pu(O0mEDSQX~_kS48 zaX1%-s{N<+9W5laP-ZKW|L&8Yu`vF)5eJ!dsq)_JVFZ@Ds^H zNo(^%qOAWq8I8Ri!?ZgdA`+ADDB1--)u#DF0R#GaZ3F7?f_bcyc@G%u#b6?_k&#sJ0dEB(hp);9cj{!d66Lj5OX2#6*8)dhr^ z{NnH>@4P82E~*jP4_P2f0MSY^K8xrj1P?1Fs$trKy);ga^nf-Jc!;v8eY!{|Flqm;Wn;MmVN?6G+8Mm}E+vWyLB@GL@C*T%v-jdu-3t4K~Z8EqSQ? z%41*3y~EVp{Rth3a@(}1G#c^>c+!{N4cW|6Oo^GzmbFJoptm^(tEIcO8Kip&$FY9# z-h*4JYX(bhdope)O2*djHRDB@V&(__0;o_`xkmI%P|+vd^EAh%qUoFO+e9V_2h@f` z!FThxEwpt=mrQ>ghf5P`T}q-MSwIsc=<@jwqNn!)u%ggt2y?b*2GGKZ%k{CSnZ8oH z2Nc1}GXxq0#|!|pP+pp8J7Sfu;3tD5DAoH@16;}(rC+5gLVR-c!Sk5d(DG{}RM;pg zRk9BDQSJlrAE{>qtcc*7JYgg%@gL(2w0Y0hgdYc2)A3{&S=n58oKO+QZ5Um;4I31H zbM+oscwhx<5yvx(JFOL8^;u=i{GKzP`9(5Ul_a^g2DC*?qgSf880DrKL03gR-?F7Q zvdJ=#w=w-WdZO+!APNU^{(I(KGxTIY7GIO9-50Ot+%)ao9PADNM&cauT+oQfXm%kP zcTvf5MhD1c4ENjS@xU4^nwH`F7`1+^odsh4x954PK4QEav$W)Z z^E@#w;Y*Mg%K$PR1$d0(lpfNmSU%0;m)y7`pDZ1sUK8!>E#1$F9l4#sJM{lP&ne}8 zdQ1X-5?Dao^)I;qe?AFQ;B4n?VfR01(c(^JQ3ZkfXh35|13c)yFR}t!V@1=BPFS(_ zoFxcYa3Ihib0tSZ$Suv);x6MgTU0W48tN)1+|0ERAyFtTgR`#FX(D~~{rUGUyPt>k z0Aq+UJec^P1Dum&O9cr)!flQTzgK5%w+X+f7cBCJgY&KeTGafzFfE+ruy74#zfvS= zq#69SV}~I+mU?1NYdSrL>$D^7psDyC;#i9XZmR3>l51I_&Ip6>h{x!Lo49(t^{9?v z@{|)qk@cL({4dx-1I__N3#qzvi3LxlX);eJ^nMo>4z2EB5=q_3j zyPRP35|3|1u$0or!>(2SH9J&@iatp?CeX(cVQo>7)Zl zrEA<&&x3@#&)f0)2OIu0O<4X8($H;POoL%vR14t23^Z3Lb)dwU=$O3}gVg?u^2oou zkeb4*;EB%}Y0E~2N^d@bR|F+!rQzB{WNA-9agqxr>Vjx(&KFx=S=2h^W1S+&-Zlxg`s zmI(2JT#a@y`1xL&+B>=_7_= zN8k)U9)|;sAEzO;*Kv(L3DV2L%c9%O>Qm1w(n0vrKWmiSeed@G#UedXKMB&|zl_K+ zQ{Sv#NLl>*9IPOHjD?~$}x zVMLmT3uQ`*6v_=bh5MQH{)Ke<;%7)d1W+ZT&z17#5_uY8Ep*-e$#tKg;CrR?C;@Bm zHqRih6#8mO()M#qUoS6~?H<$HPpUdUo>03?oL!8s7oaD0Z166(bP`Qd#u%GRcd&0g zGKRdQH-M~sIBDDP?7%cCKK=V2RD3frf*QSpWBk)KA*6{^*rnKs&#xuH=m)ImWmJ5q znCI3D6&>2^=OY~ghIup$Gs}0bt$2+RuBE(MCq)$v@Dm((gj{h&zCmZx-^_>wq3)xYY&$Z%FJ8)S(##0&FC7*ve)d>6uXM;R?iP& z&mJqj;Gx%zW&7Uh0$h1lU0$e;${X?r6=2huoq8LCC&FiMAyX2KeOgsqFPcD=OfT%? z=U)=7-Hq8qO-EIuP_B*XY5as5lmkKuDJTZ^3~c)EsRT8ae!w!bdh4j?<{b7CVKwb+GB@*k=0|!dh%0YJNDt zDYD9UQG=5IsM6Z19sMfinC7WUoAgr04B6pdVK~~QQ=j5ojGjJ@lA!MzjHs|tn~2yuO{UPsS?!7zWvdB@ z{nw=)cl~*SfqASLGP@0CjXxK?c&(DY2;HuAg=ret_7yQ$43rN0@_uPDA1HSZC!Ls0 zVv@?zfq>ayk40&7Be&SEOySER z7~cQEYDxZumMs8mck)0o#b1mq|Em%xIU6{;05^_*+k`0n2LsFZf&$Hw+%8S}N617H z=mghh5b(X?O(19%Ovvb~4y!GJVGe_q3k(FV70&w;2+Y$++k|GC_ORxa<#gdO;dJr# zcKDnAlh%?_bXNlA&XT-6`eNTv%hP&@AV5%5AxA=DKn8-*TGT;GUFX$StcG+qGQ1T& zW+TVsjssBYY8;NxZl#XbegQ9CreqN9^}{HSn~0;Q)d+KOPd8QOpS=ZbAy2EH?L4JTYSojBbRRlp3$;!kt+DtZ0eA` zEn`o$cbC>0u2U6XFkF$<1kz`;?&!B~Q9`L4M96uYhpK5T(+88?uhUl|g;&K!UQ*7i zJI>60q@E?;E-21dph>2UhGY%?b1v!o3koM*>^J#d1pWM7k*=Hffb-m<039Ub2GNk_zCZlR+U8wyA ze{TyZk&hYH=>65=`>hLh@Wc>?VzQy>osZrVt&ud@vQsU**psokwj*G}ela^ptdOJ( zy^wp(Azly*J@`O$ANdjN1ZFTTCZ7=^(=u`4`yAVmoR_Vub4qJl!Pgprz%_D^Px)W7-?w=*e9JX}bnM6XpY)rr~8ERmdjP^drGB8XA z_L}rhm<-OJFd6Ql3dIAJKVdQz|2<5Gk~2>P3iyT)1U{7ivN-*ZgZe8&LCNM1XWX}v zhGxn%-!POuA%QkK*TmS8z9UOqPQtq;wSjs5_}uz-&0{BNDA~X?U+|aNQ@5nv5aTrZO4PO z;ZF`Y4ulubj8uI(_@J0^jCA%w@)yHR^t&JcCn`qv6ippz^OU=QME#?Qf*VuLhH=`~ zcbBXq;#(Q-Mc38&o zhA>fo^URx4K@u)$w1`}vEW+~4JMNNU(q3x#nfhiy?Sigztd0&9aqLN;OFi}ko>Bgy z-c_5-W^C!&lKzo04z)&3wl;m#&*=BQGM=g>k=f!AckyQ>YlY8%DD3xdaBZm1itW%z z*t!Z+5GsP?La=mOMEASDtt5Arr;rd#6}!<2(VYo|+pVNkyIt;vlYx8zVQPIj* z>LKLfl*t%H5cya_AF>jHkQNYv^~`r{O(xD|BN&_!g_etyEW40JqF!Zj)f8FTntv2aDG0`djH6f@=(1Ia9w+PW%`XDlZ!n&#J(^X|z?GF+LQ z-@XY>aM{k9HY?_u1yJ8@tsD_b)kCn5aIuDpI7C16bjw1mXD=O*Edv^H`l6w^iS@dO zrg>RJARo5E?i$5uFR*#MqWOhhF@e?$8FON2qL?>ly=g8YmRA-Uls8y1nhP=nR`|G) z!WQ>4KOhciwXcvbpc|{MCgJ#dAeuk)Wi)Y+dfLh27F?>Eza-`QOCy-&3YI{UuITkH z7fY={Mao8+A&i3#uHjvLZyH2T`tCS&i+GNAk_a^`6(WaC9e*1{` zJGv7H2xt+(8M+C)!i$25Vi&8SW27mXbK+>1^PRQ_7}_7#SO1`pb#d}mNB~>#B=AB0 z7ajfoO``kn^z#U1Ej1h!w0Fc5EiwRS0iJP_a@$-mxO6XRfuButJW9|PiGadlq*%sK zvXo@_Ur$p-?;!a1(-^u|zhd@JPU5VZRrOwx-#?%7#j!LXw@7~3OsRWxd*JiBv^zNF z@8kzT0>V{%(nraC@%Utg5Pi-8QlO^J+5_c&oFKQp-hR|Q4l2E@cyQ`-)g%ihAd5j} zB0THzHB?0XD&ECA#E=Dh9D0{(SS<}nm~%I>HNM0#`udGno+I4z((UgRN~5%2n_P9G zzZ?&F*C;wyF(h~Ys$z0uicu_#jUrjotclISSvQPMOiv#tfNI3(cfG@0`Jk1l6fu*% zk#K2^HNu3iG|6z>i_~%Yxh$Q3B-{{UYBr*4`}VS4uH*7kwxH55f@);d_9k0$krM;= zoTtfl@ie84_SH!_NoUv-<8%8(kO+JDK}p|9kF#@t!B##fxZ5i$*rx6EJ@52Gex2o} z$(7RHdB3~1a>=-vR0ZKvK%N$4FtSRG?G~9W7G_^G2pkxsC^8e$KynxY7{1Z0fRF$R zF_j`@!&T^3xKjN6z6E*~w)l6*Dn-^^K3S& zG+9G_Ep>{0yp=B_=kGY~U~lk;+@2lNigSf;;~hF~TEzO87h5hblkOVB$er`Df&icE zxehHF5jM-nQ{xQ9Ft&X4cFs&?N0Z>3OOEa^(MP;03QOkzmPX~FIdNaXs{7#TKI+Ph zMk^&_8TVXV4h;>oJ&$pCKvm+_)@X$dXVOoP6L5fEJ#N+9x-_YjLf#;H-9y+Xp`G9J z+5-i>l9lL6u$~C=-(@VOe+=edB5naXlks_7U?@GPkHZ!|nX7Y%J&~Q$)VfWt;JqNw zpz#SU5o$dQ=Av;U>l~p~8COyfle3X#YbrPb_Xq8lD5Vc7Ys>T9M~83do9YH5xX_ zbY-|g2a;oQlh}ubPPBz-zDh}Otx>E|s7I`kQ$ElYk7yNjwWL2&Sf-4I_vR_(Ff!^$ zCTBinsnGP#co+Hx&L*Dm4hKPc(t4E3DA3N&2xITaNBbIHJI>6IWD>_eJKrE-1lC+& z!DTW6ilawol)sYBJ%LZ}E&CPlX5V-}p~^V~ht2jE6{uX1v1;bIm{1hH7{*c}d;~s6 zQ_5rx)Oj&3$Io9HN-im1mYoq!q>juzN+=!hK5|7olDa;fMIAh`Xp^Y0@2c?(fAUJ_ zPn_lRWlTtPXGutW7N)AW8dlcaS1PF=C{nVhyu!>Gp+~m#4xXB%M?K*IU~YK@LQbN% z-r{ak+M+&O(JOUsSu?Z$b1-NQrjh+)7sm`74*oV6*f}~ISSz_W*_+rJ|4(_e|Bq|j zpJ)!!!V3Z7K-Pf>U^GXJAE5(h1G_p{Mj;+FUzlBvEfB(Q|7?=3+5?(b*POz+xt8U` z->JI^0?ac7i0J}j{epn8eudTFWB(QF$2{n=$N631T+?tSIffqZCp+#mCStyv_e?1Q zRy~{nD&WBNB>Y?o7L-p>tzreH-`UGHh8%Aj68i&sZ<4~kLU`R|^pG@sg`lo%V?K)J$kQucj3w?DCd61AuQ5$lH)Z@`dV zsd53McXO?e?Y_5S{78AF+**&lZy)gi5nsAqJj>`NhOi&L8%bm~@-1p8T8nyA{&^Aw zmoO(0{ZFtTK84$SMfL+Q*w4Yc_LEm6G(y@?@uECC8vY6AE>OtbXA67%N67uJx0u!t ztGnL3LdT z*f>6PZfXo#8wB7;e=VUYrG0YoSFM*XVUd?WfY{Ib{`AY5Zn($PZGBz-s&3-x;OPk9 za~FiQnLa&L!>pqBmdBNu_8Fx^XngEo11RR3cwTq9XmY)Ht2i-n-nema5dR3-QC6+N7H!Lok3d6mQt*1Oj2&*B=pCR%8fc8-$>P5R0;FN4KDLcX-kz?cAr=o*gtc8 z=Oq9v=I*284Z8YdM$CUo(Dd%LuRA`$mbOat0)~Xv*_9}z+P*&_fnmnin>etWC~h61NCrRmzi_YFshQbm3*@2oyxI4(qR-)ZTYS;L-!JF z@f0kcZM@Q4|I3sao}39P&7Q_7{(I`TZtikU#Kzh=b1`-D&!MF{GIN?vYgKo!q|6SO z644=cY6mQ&iE-Y)lFQ62#f?uMK1P)}rKBj+8lve=qy_2NW|cfc4syvQsXf*(#E>(@ z#b|%!1os9k-^Zok)$BEVX9`y8$QZG@2t2i1b$%g_rKUtdNm-fh1)ACCgZs}4yB5w= z!mT3GUskYOm;R(qp~=D_DYCfd;E?Z>q-L$q)7uHM!|OY$>=0`IjjgVg)}(#}YnMYF z$`|`ETmEw+NciIP1FYN*gSJ!fwK&Zof+f&^f5rtb=eqSh#ouh#KStlI`Vd1E_Xr*_ zDj1Y_+x!S09B3Gc3*!d<+ZW0!UgSXX8UlW1WiKVdU+ChL9S}VNm16;1eZ>;=>d#6-?Ptm0?6#@vCO( zt%E1s9H@3Nr2Dgg-%sx3N(0Zx?zYM4azCd3@D(^UasgSm z|9iD!`d77zP+e9-Rm1$yL1`Lhu#C@z{H5NH9fB0FvZAk$sH_C09#AhXcg)Zn+|7V$ zMn<##=%c6m@_DY~S02_Iv*|u$Tq06>y+qP}nwr$()vTbzX)SQ`{*|TTOMc(FGnJY3fBEEQ^_v7xS zmK{jwt{VTJy%Ik*p5iS>j6L_Bpp-v)H_j-koj6|=9XDAe0C&8Q=^2IG3zLs~+B zBF?O%lw;S}-0lpPj3&FFr9$g5G0cDqtac=Z%3+mtdy0**VhwH6n+_6>O}#eyEEo|LBcTqfa^2L6JBwvr;4LKfr>Y|i6BD(zMwJXf)%Xy( zI|{f8$UR_DkzpD@NOekfIL1IXLi!|!jM0+VB4>d%&kyg~$yK;8G!~C%fwYxJ!8S&J zlt%~}gdcDiZKbhBMMpHrSUV6@OcW{!xGaLkMhh(*NM7pNqA61wN{BG-D%v*|_Yt8O6r2t#E^}t#lKmyV>VKy*U600ZCq;!!ez2STx*M(BZhQv%{e!F3QBOc!k7w z^eWg_^eWkR^y;5e&;Oh2YARc-5SVg?Wug-WMJ@`3WX!C)PaRXM>=g*3#D=XRpa~Cx zu@qzu?a3hN0~JarmyR70_|04kpuJcWh)EHqirGJOWHZ?o2X#(Lwy_L!aH zuGK~XB@`r*qHL}}@6nuve@?5IK~8|JZ=AT(pDyy-07PT%kx7j%OJz^^uw(Y-(x_X) zky2Avuh~WX;ekCeL(~Bm{=sQZ9URK(NU}>-}P&|!el?rA% z8l5XU?BnRm#y-L$yXm2t%%Pd9+iIljNuBbVAh8l4vohRZ5+6#F{<5&C7m7~p$$i2a zVQ0}sc1JC4Rk=&$S+htTQTjV=slIORX^v|<5z|d3wrt+0?ABtBHa1qEjmH+us2uM@o$ay=l#+ichGp7^ zyezIwV#}C?=mQ3F2U{4G#QhXBBe#I)wl3{1qQKqL?c-9BdqF;o6(N8_3^h_iKA#tB z@~s(2?1H;FY0_J%rQ-%GUI)-Z8H!zry_Z<7|~ID3n-H=`mlytT}-B1GkKv z4;ue%p*Bk>3bCu}cO28q%YlWsfOqzQHj8l&)VX1`sujFzFNDj%?!kax_$L!`W5;v~ z>mVDq(eTF-g< zkMX!SDkSf_HEqEi=4-~%EhxTQQ;uYgHv}IAS`hhb$Qrue`a{_ZIQ!9-Kn9)YHm?(7 zjl$#ftk|}J*o_}Nd7!PABKrG3);j({wyahD)a{u+CE@=&!7~5P1nZrsE&FqD9kS!U zU@e(1EQi1^H<7A~im3DlMe?l6 z9~ljzd`5m5#xV|InBeH+7~>e?m}cn>VUn{(nQ_q0;V-sIt&SBN6Pm1H*GnI@l@J=1 z;7V2+$k&u7JMcp-RntL(mPB=#r(ZYy`JB6T2GSjvS(p}EV7hmiZ(3QHm|jAt?dnT0JU>Iw@{O(`|T4h|gqJ=FN1%*@QaahR-z7($g}UVxVJ0 zic%LzU>r>|rC~tBOl2gT6O10Jj2@Eyonvpc%68dx%_U(oBmzc&pwmQiL~`GtrX=iN z_R^YSTSGmFdivKH>-;eHpKo=lui~*JxBcwg?RoNdY!(XSOwyLt044fqwst4{vK6_d z29vJ3?*>9N{xY(;2fk`K;|4Junh~QYtwVY~cvN3Hx^(6UxM~Kc!mzuoy||C~`$k_7JX=H+#Hz za=b&t+$CNCrfVf$;jV77Z$P=X>Gv=_``AaWAAPbCd2kS6yiSOsXVCK^P4&VczW)*J z?dL6&?LT2YuJY>_@&Bv&`Jb_Vru{LxpMMF%$@{lS}p_1n7-B_m|=c7@z zDqE_W9ax*^P#joW(6H36x=1ohFvX19l>u~!7 z_xF6)8t!_*y))33;dQ+$(9LeKa?ua2NB$%e$BWsSd$hl^Y-@j46ZZod&Zj0Y_hffF z$g`)Pr)#Lc6;U8I&=Ysd;r7bCEA__CE;mqjclUSs^v$j}9G=~6Uf_%3tn6|9r2Cj( ztJ|8M4_-bxQUub!PBErV@h^0)*Iz*Bh_{s4z8ij#Z>FctjNQB!yQ!xi3LtbRsq+^O z5WN8t)VLpxa6YKfj%3c?U4g#-e)L?g%R`%~W%rZd-_YMwj4!Mn-r46p`45%=AKG5J zT;D`r$&=NSJ07oEpw$Ok=Q!kMpd75R}WpVd1;hi~<5k;^xPEUxxJ7vom)OA@#U zj!#l|_UuCyysIy_6y0V6{S`*((=zZon2Tx9a;lHepx!Lof0zeD?REU+bG@&?|j* zuXSJNhg0_Vrr)>1TMW>R{96qWzT6u=2yelCFSw!7*Q{Q^#0wxr@aP}-qIqYxbS?HP z1lRYR-rLfBd_Qy@ie6o~5>udbK>~JFT3jd;0(fRXc92P(Ip_(76<2aQICZH4`ZTVgMw`h z$s)_DPO*%Jam|6M6yuVm;Puy2nnWi6{e<=_Kv0T!_l1I9UZVD_>~}#_&xCxQ0eJV^ z-=oOHvKes$(L?m1@yCqd!myvu>j1$K*C0o5ik#LXJxqyX2$+f*w;*D|BD5^uAzyGw zp?!=HO|gAQ?xN#rucl($)2ksplop%I`BzskwyipOSNhOxpEj2&Yo;} z>7u}YLPVPdF?1TPJ_NL3g>WVRQ5eQrbZJ*}NFA;|2eJiePJ|x)+HhXR89Qufon#P> z1$dx3Qf4dBfOy%8=pfeN18^-~ubx#HNeAB7X9WdXC0DqE45z6N{`Q3Zd)3*MC3kzf zCkMzJyv z&n8njzq*Y)1a=nw8UO^&tmzrx-P~CvH?(?Cegzd$wILln{)`H^fA4T@CnlU+3~j^- zzJyt^Y0yNVEv?l4%}@jT3t4* zCvgFmqn)#1Hgr;J{RHNs@mEEcLjb4EyBkQB1WTZ3v4N;s>ZEcI`lV1t-CBj?@~l~< zN`whK2E*nNVlV}L*#<_Uyw)?6dy0Ry-)KETPncI?3i-7DmZ;#4Nkvm&JW_gj$iXU!Fv zRA+sm_)DDA*2De6a6U?-C$%2!6_dkNjliyw@=Dr3mSj8_%=9sMO(5Hbo2rPZ?c4D( zR2e%WR)w2b5iv}RshI#?k6#1mc=Qta2+~0;a>RvuQF9(B!Burhsw8X3kCKcGYAM2+ ztwvMSj$FtCzr8=P<%vIV+n7LD3hf^!N{yu&lq-a`@WxPV!s)f}@gN^BA;CcU%{bKg z;KX;8O4ZSyHFdNi4+EkQ{t_l@UX!z=5GDwsIGc-u@3ACVA*EWZM;Sf|DGYUU1u8}O zRAYpu&6KUZe~a}unVvxtK|3@N1QK_WZe~Tui8EFCVEHpn^CG5g9kg=i1CTg*4#(@Z zu46-7^TJzz^&jJ|KG8N3i8f!6=Gq3&*TZ@2VG4X};PY?+b8XrDsi^$9&CKSE7*+Bc zk;oS2vMZ%f6#Z4nPEgI$`Em>_0kKnV9%B(x>P=?xMHw~F3}UHqSnWu!-iM%(FwfX#}cy0dcwr*8E|PA^v+Eq(e*;I&fHVl`d@oPQTeem%H?hcMsl#_6qEediqGF65-%zMxQ-li4e z1TjLV9!hS5Ogay#J=7G+78s?>fk*dD1Bhi==~~;c_YWOvBWQfj85H!e?8p1e0@=rX z9HBlG{^y2qpRfeP&&MZ7Z|?-qCyef#yw$}^d2ni3hWL~|y4al)QhF@F8{K_W2DnFp zle5Ix#%}z!LY*aVibEz5$v{U~8j-}-4N@l~ilin-l)FeLeqwi@k99+9`Didn7SMYK zq;QnOThjPn<8OVrc#&df5)Fw$GG1CUOAwm6G$)RGDBBBh(%%QoO!G{$~ zgnQKSDX%!5YbdJWlG4uwuU(WqeK~@mH~@M;p6uK(y0m@sD-v8Z3T)Ld=K4HozMBA# z+0K0d#)O0uxZ?ttjVM#$aFCD3*?@7!YXfBuQIAE#=z6Twv!Mqrs_tNQ606>l8H%D; zw$a|L2T_i74g>TDHmK}Scg*k7&4H$2YTAmP`Q?SRze`bDtoi?3MZrHRHfAc+<%vjG zbc^tsWmp^dh%1b22_zjRE4l4Y)!5YfwkRZm2m5{QCdeZMf`LmK%(a$gcrS<|d1|MlnQu8XN zrCt5IFuJ+@FF9_LhTOW`0&ru=t&bl-;sgo&wjn4orXsh^?^^uy{ZXUFTm;#SOVdkA z?T;Z8K2$zK2Hz^FDxW4-)?vl{lM*)#E9PNll11IL{nREoV@eY-dGWKyDy@IWc9{X+ zID5HHTXH5d1{w3UGGQ%4D6^613tQ!YSVlp0WPrX33f5TMQ|+*=)Sc#?!7-eMEo@%$ zqtT-q6c5qwu^Oh~$w%if?cfz^zsc&zo%H>eefW+)i;vLj6O`$Uqt91Hb>BidPqX{V z+8@Ixwc)Fjk8Xn~zQTjC@2~7x7hwK@o*3m9969JiWQ2+T`9-g63Q=` z2e%ZO(+_)-rlmXll2>vG|IbXGipW-}JNoAay1#70y~Dpt+*I|#;yO##+bvi#2EHS{ z&-Qts=l-$}LmXDZGi)W{N5qcxevK~qO?{1@*&reJIr)71ndlmF1dPb{vhwR@qzvjE z-z5?$PNLoz)zoY4?p_0ovJE=<>iJA8kV}H~sqI%%;#*Zp^x0p5|1!oeicdE7FxHzE zmj;cEL5-Gj+jcfIVUC^IlrygJ&7n*sM6VhVJU_X|FxLyO-(^j%=`*mhsBDO%ONR$5bhnz=(7)^tTQ##f&^WQr$$hwsz+LB+Zw`U;vLgMus zGsJ90o~hs@QIQl2G{=T(42<<>12PqAesmDJ`Pdm+P#m@($Y(rY>akla?Q~VP%p1CP z9uB5Kjf|H5x4o;OCiva(Xu4)2?z~qOZ#0I~v&osFXBMxeU=EEWkENl;Ttt3_)c~X+ zHrNh|p-vuVNeX>~m^p~ct=BE5jx+{~p$>xgCD^3pl6d;H7BqC`cnfv-_j+8Ws1Km3u3ME05bxvN}yKu&|2 z2wy&mFrF=a%RasXtnZf!V=Q9kAZ81Zi-Vd-s)vs3m?XOlhUtQ_tg9N=bzOO~nd01h z>#}Q`v^ABtY~77AQhIo~bRlFrK|C{W)@G&FvpmWLSe`M zaU=OKfUo3e2}gd(DYL`{tSVwrj35eroEz5_10`G zSGG_#KaW3BDv*aR1}o-b>QIrs&)|KQFCG7@hXErkr!2!|5 zZ>-25hkX}d<7)6@sbdk3!fJzJo$ofZWFZdafJUqH^Wz?eYa}jd33D=Jq@0$w4j;cB zEECd{$Z&+tCf_ckHLQCCWxlBM2w0OkNu^ZOjInQYp&FUqFfVc85@sh4(@8bBw(A1< zGo|BEon%<2($STr%lNlZ%(O*a?_wlKWKX0IQk%|mqq%R4-~3rnj}<)oQzh$%928FicOLBw)EBXwT-H$sp|tL?X}D!J2_Xcbb@;P@!(;OB9#6WSuxu^nv;d2 zsHkTsPHlU&-6x1^<`;(PS58p`P{3sO^{8+amh177X%9`w374M;X!X`C&Kc8*U& ze_IbKZTj{SCCPKmxC^s8&W8a`{;~=C(X+njbK!f{g5@|4!z87mg!D;_oE^J)ht`$3 zIdr;%rpBKHyH+-cB~rR=DxrVd{W;kxLo@zzdt?L(-K*FFD`#q$$xTF)gl^Cy5muf? zD8b`|ktLOvA6T)o^tOAEjKP0oy!LR*v8gtL9nQ_}k=h1(iUfM`3N*X6d0AMq`Q4?t zr&lOHu3cPLT%ByT;*=^PB4`?>eeSA)iye}TaZbs7D*uQfqhg<{G=g;IGc_|Xw8rz2 zsODO*uTQkni2={sD<&={vumZ}WSa?MuW4ud%%Qp^Sa9TTEgR~%LeP+)iq+ggkFO~i zSw-InOlH>kWFnP%l#f1x?1cqmTJ?DMZpMD>!R4^mT-6iNA}6V$hxZq~NOCF;(Pj6e zbwJO&SN=k|S&Fbie}Y0>Ooy+5^LCaLd&PsQe&h%&&gIFW1 zrQhX$F4^3m^-VXdbaoZ=R&@2g*l_7Uz{$NL5SJb&5#kFO2(c0>6SYAJ94IknJFto6 z6Nu>6vka_8B603?=x_6}tZ()=OHr}1#;C03P8@{IoX55{JIkBa!nyvN>=ZEZWq@Wk zpgGb+SCoOxwefosdsDOX3X#3T4&hOhL(gq>AXt89^G%BeKmX1JKUUd?L&mlD2htxZ5ca2MU-k zvTbNaRGk`ih~+Z4s7R#~HW5*enp&CXnwl9_#1|rkp)%}?mo?6v$h}w?(h%9e&{P=1 z+!mpHD7fi@$5LREW6LEqFORFZvLL@Em;!pAYM!U*P~VaZc*H)0Shs_xt2yfF zlhh4HX0aTR`GBSbTfBT0#Br!3_@dAHqPS5`>*(Uhjh~;0GAg#41hV_dblpm=|0J*| z+azj`et!XNE31hd**JS_q;YdM3nP?h(~+_eW~GdvkX7q3A!emjNeD$KHhhVpjwc6E zW=X*%7HJ5x8F)dIQCks0s=!ldJ1PSw!Wa8NsImpcf1JSP-f3-84@EpIpV@a*E} zakm?H;fyB(N=+k4TWCw&!hIYPQC7ET{1c?g{j%;A{Ga!((pMq zamQCqV(I^_358x`h~{OFcDGMXaG;fqn<&OKj#CVZopQHpzpPhJv||ywOG-4EkM5!# z?5lL3kB;pa8Rby2wX}3n8cv~M`F!L#MsN{XTVB{$+gGRZPmRDep_0}d&rDz8UL+IB zLxEaIN7NSg>Ifj|P)oV0jXQd8GW3n#0<<809Wi8k$7q8^dFy;xFQ6lQ89;ln4(Lkw z_eR~SD(u!0eKe!IUO@U1gY|&#St{;cAbgoXf6xZi%KE<$bg_edu?_5q`}aoPz9{Ts zBlIwW^+4@qD(-3`^ss{Uz~*kQ=Y8}9Zc|Qnu5CATg?i)P`4IHp44dL_()Yrb-SaBr z33({tDHun=j^8ux4tTLRx$~YXqOtA~3 z19c@FP;FI&zYnC4F%FKlV4N!?;A85hf*aBca!OE6juOEYVWm~sFq;E_d?2^x{EG}# z10s(Ce7Idy-y!u66X`V_Eu3Q9gi=t+a9)#{2VlZL?)rM7Jf)WZ z0P5Ez#%DX`3c@ISeJJYaDsZS4jVnT|R29_@%=VBxKxPAkOJ>kqkQ^B$P#qF=NRN$e zs^653_n}ES5TEC)0<*-I&A~WZrD8ZQBJQlv1L3hFD`T%e)@D7H`*2W#1LSzJ0f@?I zJ<7{@pxw?VK@~!x0oan9pf|Qo6t%aL5P}o@lz)9W?-$xg-pa9gyf%%Fy2ypLu&+a? zY~#C2`9k#UINNaoKSj8pFnYKvWkjplJ38NA@iGEU;ywrmL<^bv#BhE%sZ1|c z3#NfF2O$c<;z1qG8)tL#M2XLaS&d3Ey300XEke=g?&Gq3Sz$6gMgS^F6=nxJIQ4P{ zTJ^v%NqhB3!zlnPOMFtxmcEhJd=@=9hJL$Y3BJPxB~`U2U$Ic5Qr67Pa=*|w zBX-e#w2%d3Tl>Ydsp7=u(`mOGQ`127qJP>2H8pXbLAb^N>3(mxh+ri$yhi`+?;2 z2#X2^rC>hsZd-Vc2;jgtMHG@Cui!b^1FwZEm*q?1w>%}EtT^bd!lHW*{jYCy1OV(t@*o-08}piH2@3UX5L?%GjucMNj1G6(2X1=d{)c~?o;39vq3#&;7rvHmy~iH}%r3EIuudCba2uRkoV zIYRO>Yk33cV){*0H;+i2Gd}IjwPogwT~b3ZJ@v?J3MM<3CJg!|nhD#0U1ta?Q>2SI z!#AjnuaXL8%m0SwoZ{9Nkh#(KZ=EETn7~+X(MRng7rzY$&3AK|J4N;+<#B=P;stpN z>&l-(>{deTT85R9AOzOJZ#MPnqpu?5TOtCjzID*W%Pr8sdipMUgZ3nY)L><#`X&oeQpZT{34a-q3Gze89DPCa`kY zL)wx4yGN*22XQaaY*?q)rX&dt@nFyPGWXG&tAx(9%4hd8Yn5ciu9@rxagcNjy>P-Y z+R1(=xJ-hhitYo_^h-nr?n-`E1YA23$G)^E!<$|>Qh4U770miPE)}0Tq7Q1T(Issf zG6KescA?v#pC_5%AQp{Yke?NeVzBchI&su%`i|r%)R2H3LFRAK%PVEQ zIC|yCcw~2T01tyIw1b{c>W3{}+YE1OCpUD?s$h5i$|du%V-|E?W^^L}7ge1%nhXx> z5Byhe(N>KF=6Q+0Ep#*(x*Pw4v46)l(=T>G*lR&#^}&grEyRFRrZ&d=I86X-yN7%V zOz*^g9Lpg%VEDL!;u-qAKX6!_#%>H*kbE2*ZA2kmnUqxs@O=&GyE(w4s7&5qd>B0t zpzKocec}x#xvN65z0Nm6eqS=0J4&gz0Xr%;lGuLpJG|vU z9`+$De%Q&o!WpIjoehvkJ2-lLzn$UT7f*ec0iq8e>0fLgcMM$#7J#p!f^I&=tZ%B3 zm@@eAHTgj2I$q+(FWC2hAO}aveQpwcf~#J!e*J?;@gz#oU}qH`xOU-1&B>QPj2U-7_4epj^1hkYAFnSq4U?fq z{55>k=*Fa}ltz_yXR&VhzD$ZL9tJSfZ!m?A-dM;1n{S9_-$y6ZvrVHkpcz1YR|3u+ z8IHd|T$x6)>Pz2{5KT1HLL~=-OjG5RzGu%jH@I!H%0Y``TLpeOW{n5HIX?oaHZV@` zJmCRDs#M_qu6DDyXcB@f?ZJmubzk`p9*BdG5RzF{vKdabGcV)}r(4fcFx)q6-pV%3U3h zn|ts;#!G+j0HpB;IkzF6w+jj175Z`z$$iBnmT@M5=Afmx|My+H56*`V(5U4^75a@H zCnwxZh1V%xbzlK8+_&=XH-c5vpeM8!e!t@Wtv@iIkev+a%spyB`0lj5N?son zqzqpC#t5(@H$JdTEy;v)M9A! zB9cU_R;NIhZo#&-CQyVdIWWblN!s)qz&3S;^FP-~rwCY>NKmgl;lf zoTzS00krmPlKzc|RgsOH&{>fLT=|itWw7<`kg244`h#!wY`XcXO>hJM>|6f$CqsVG z&QzeNMej=5s}%MHL-H2W_%c=s2kI7h6OW)p2h|_Wts6L!N}`Mx zi`_5UyF>t|60V_}PNKR|1)L18n?rNs_CIrQ=k;-c&Pp{j0z**Z!?T}@g}s&)Jb<+{ zSd0+1RCLuHh)|TBbB(E-oe*a^c_kkOOSAW79sa2-o-{Nq>g;MVgIX!G3xB*|0b68- z$<3IkFuY76bU7w)8J(pRtnxu1i=R*_hs<+)S(QkXZ0i$-4fIvo?yaMvjbh* zV2he5<0XunFDQ$tcRnlV>qz0bmzz3u5D!(7mgxZBj`#2XATHjhi+Wm9Cmxm9;2DcY z4?qy;5vY?)8MZ^xig|DrD5k00MN*H6Dbz8ntz3-sj0e9Kcj2U)duQ`%?nszw5jV-N zN^4(O911x%Siv*I9=kh0T0#Ls2&->|SeLLm8?&;9Z%|)bpwuKAOTpOW7=x~JNUX}D zp%#wP?|QSrr5n^-bCMw|uHkpX(`*?_nY`RMVuGQjiK(ws9d&$+_QKsuMt*Y(15PocI$La-^Y8HlC zCvY5pWTn=((olWL?cyIb_*c(YSwg+S;37ccisHdFoXR!o$rxPQwyw7vuC>6MY|^uv zJ2FpcOrqp8xwH@Q3H(@E#ZwVZY#^kf9 zanT7gTC6SPx#@~gk1Z<8PD*9H#=FXM(T*v*TwYxY>n7`Go#09X=7s61k4|Ypwk3Yf z?l;RWvVuCNPlz^aY@p5*QO!gk&AZz`u{ZV$obp^9#&EJlQr{}U7oC?2d_9cuMow7R zcmG#;2#rEWz|6Kf%lsGGF&pufR7+#hvoGl;y&U4{`4aaX!QnNKbuaEF>t7(_z+I`5 zVOPI!mFg?Fr1v^Wytf@nG5QGojwt6y#^dVZF>FISDoe(uD&>Qo?5Xy1l${5WlT!K; zk>0}CQgQ*zsj0By@9ZMAViOOf2z$L)K#+Tsc94026K3uJX;i%|e5t8bRCl7V+6Ez; z9!%t-FJk^issYU(saaV(#JX80Ts?Gj4Tv?<#1o{p*VMN8DP5}NuDSlN$*)pKDTOzf z;60-G4?IskNoH9V-#*@{g{jL$Ac%hycj_rW;rkN?1!gw$ahvi$5hp8m*?8UN4CF9ldE6dg} zPdhX67(pdbDbZ|RbBxgJHhRh~plSeN(_TxoOHVA_vt^aa$(;aalZr!UPfr(g%L0`d z0w6J7xd_$0VxRG(+O^$+RYYYX9H7x|>Zr?+vuToMX?JInKxAPe0jOwt@?KapG2xo# z$QX*`N>DRUd)IRoth)C=%Fw56Vxw)9&zrz$y;DFN1aUVD)PIrQg_?cN5aERE(p#i> z@v?!@VQ-1j(6uWZ-+>vlk6xkB|6Ar9q`zKZ$O`3%y}xF+5fUSfQq)$9d;)KD8_I>Z zS06#nm1CeeN<>%gR&u}@0cbCGso1J`aqZr<4u$xnvi&y%oy(AtbC=^lFc3DMcJyzn zn%OGNb5R?enYn}!QE8fMdyLag%Z(U$uz~YJ43D~$v?v0py71Zz4}&h3l@d@{O3Z?$ zY~xruDLJZw;3D4W`gAF-m-Zbic7EB_o}a0aHwwh?e&!)Y^*GNEZ=Bvl!V?8qU2kOr z`HTw5qf#fK5^kGFcIPGW7W4>36iOb32bWJm4?vKF&4WY0C%XHc`3Yz3*N#6YD!(!y z7A1qs*v2$II-p^2SmXPF z0RQ{Rc9j<#99$6GhYs8X+{OhQ&jnmZ1e`5%s%r}Aw+Ofi>@~nwCORMw%t8!sL-ge3ph>LVB+1)J)&#VBbLB#27(j z2_Z#u9ZPfD5PNeQ2P3Xo14ah7mH-EJJ%DEa44yqzINl%XR1HB77z@7159ri@Ko&ua z8QAV; zbu05!-AQk*?l0md8ptIn6uDHlI5(2CJ)&&#-$$WAGc;iW?8==&5IvQ|tDRJa0_Fwm zT<&B}*`6gmq6qZ!3W%4p-8%F;N;v;?qXyDV3Z6V5m)Pd#$x_VL;>D79{md<>O(=D;PC2I$z5wPJuI#-?d{ELUHOtS- z+`Gu!Wye&~u4v-)b+GLp!NjKm}p-}rzTDZy)i$>Bq_$y1my=r z0r|@lBID-vhROOIy3Hzy*j-0y%Eaax=!f8Ad46S zMOwPnCRI-*g9NIpnq2G_8rF$~Og3vRkY=n`Rp!{SSY0AAl7MYRlSQX%Mj^xRWVW6^ z;mj)0igQ~g$+hkJ7#`TP(&J(I48Ww$zHQ5l=XK&f1wkODkYo!NFD;$_XM=QMw`!!BQEi_6)(e zB%ntaDt4hoFujhN0xB`cOYpI=MH=*3F2^N>RhjiwBR+x z;95p8AhYBM35p0XCXwMl!{<3xKB<}=*fV-UBCxTst3JEsvOQZ;JqLS|x<_e>OkT_& z3yb-bwO3y(OT`%^EX}V<$tX992o@l%ot{=HQPlYI&oPvHIDr)y1fhodlQ!-zbQMV3XQ zu75aKnE|KB?dEb}?iIMH#(;VmKIc&|!g@?VN@umnq}92QDjI9FS^NdR7z%Y7B0I%# zVP*kG$B#hESms5R)^;T!F;c)JMKR(Ks)j_i3`e(C!9f*@yprm&p5 zcVSWJ37>GjDHu1izyO92oH^?JKXIl6?ST)7)wuANF-6^y zLjhdEW^7Kme<>-}pdX#oSk7~k7uXC#T4WY*$@#)ya{+hLFaEzN6StZQJJ6evqv z(scR^>E+Cc!25L9odqX-f$%+#%5n>ZJai|o$H=~6c;6s*o$2Bdi++Itwq!SO#X-oFZ=Q$2mzCZLOSpjPTUo{jHu9fscB6`xCx`k{81GcgKgYoBnSFn zvK2oR3GSG(u)VUROcgt+2s~AWLIWNd3Xbw!O4KV3!ZX>>JAld2ofSfc(5(1+R8WIL zE#w}G1GG%#oV`fDBcOhu$~XrX(Pd=e1b3SSW53X}35-_R>Cww#uA%9nu;c2HF{tD* zE1y$Xk^!)CaE3pf7Q7Tthr?4?+>lWSMO=za#A%ehrSQ2C6aWt34K45j64&TMt!_6LYcTPEi>G?fh_w%yv#=+^g{8=5G(LAy(<-4x4|&uXZCPR(I=P# z2g4@H?!keBlSi}4NHzTIKr-JiKLk9go@JOv?SElGh%ubRi7&o-(CfeVLb2I*HXA}YeYY}!J=rwd#AN7Ll&K;x!mh0dZ zGK<+^dEVQxkvNc3N)WV)3QcG4huHGhY0gs~}9M6zKA23S8) z0uU{x^%of^NR7QUEj>65HHcWd2lzV*5{ZZ(!Q$C);*nR1ev7e`WGR^#PvWylXB38w zo|vTSYZuqrj&qj7cv=;f7!~O74Aq^<(9MY@EQ(s&A2yK==}$Y1GzVNmEy{#VycK_C zE?V)UNN7uOvC+*WpDbseUs;;EY=DV|kfc_Da<)F@x?yCDZ6Ty5q(Sj$J_jTqKCc8` z@Mke8(Yw*Mqy774F!xw7IP!TgU$1)9yFyA_hdz&~V;OBK6IY?lX$mE$*IY3$|IS{q zN>`-nmH(=$v2~VZvv_zIAb|HcAL?ME{EL#HH^ew;*|xyPKU(E?)vDX76t|(=c{a1j z*|9%hm*c~d6joK|ENN6(X*7|KpC#>kkNZdr;bj&_jR?qgviyVI-6ZP22HqY6xhlq@ z(dP75wiT^_UChdK6=_X*Y&4cqN@JkH%rnk_C?a4JkW1vP)LARFdvsB zNe{qSpdSjc5S?JL z>>^Q-Seb!quEcXe_uhH7YQ7k%MM4}=z8NYyF-23R+Wxe*Dy3!bzUxOg$I+R;Lk-6Bd?!A< z2j;W_Ebt=_-Ya7Gym3?F#|Wc&XOQ#(&yFGUpz$L@UGN_IkjLjXp5ml7sg?ArWLstr zsa4^_&;78qd7)o(%IWw>4L6lm*^jUo+Yfmn8Q+=CCV$jET#EKXHk+7ig4kqK_Wi`C~S)FYy1pcC%>BUSj_#Y%zX1OS1nn=_)Ep z{kN*#`+oteL4t!dk|Xl&ud2uL=iLU~81cjy$X4uZ^#m;hBArOzYnMRC@IHX>KKVz? zNiDE`EX`84-EEm{kDor@!1{PKxEd%<8R`*(4e&B4ppw8^AlHd!UzQjNt(zf0uShV& zL*(9x1(wg@1H*YznU7$#H(@UQ3r!ZXorr6EJbouB9)bB#)JY>4mebvo(AcV772QgO z_f zJ;I={oT>;=OI-=l@R<3_`!dyJvMY&}KS07F%FAe}a3E~0VL{JyPwlA&Bhsrr)&1K_pVVfNAJy9F@*x5fj?vu#!(Q zXJtP1jrPouKLJK}2(M731{e-;=d1GLG(yi1jv&0gD;9z3})7eJQyxz}NZ z|G0nZc3=cli0N6~N+jW=wqk{dL#Y0vZn|HwbKYBLbH8Kx7l$Gr^Y5wIyE3206?ca* zdxXrIyMidYIK$S4TT@RA)-rN~325-2A7O!FDI{i+Pcw~8F{I@BvwL=@qtS9_HHwPm zBycuI_b$Nn9ch~3srNW9lu67FNk@qIKU{cf;J!EIpFz{~e`lrt05Tn%obCQ=YEatIz*a{1&NfPgy-o*+W1}+-_UH?f1?ZkhJxv{B)Wu)phsd{qYLRrq7>Qrx6pp9L`~OJZ3jN z{xC7g{rucS^s6+h3?pWZh!mrC6BP!(8HrQA!>g$n+slZ!aMKmW!w)+OQR781#fQ4c z>f1qC=!!!NdaPID#T+iBpion?lMIeYk^o8xYQh;bHTc%zm+wah)USwaI!&vD>KACx zUS)Wiw3u&1Tdh0Nb~27KI4m~M+WPBHV1SFMU=#_PMlP=GJ);c@7b>}Gtn*xZsBBMZ z$g~Kk*|Y#xbu{UoSkF3K#stt~CV{`Kz-=7z8j?WZs=AdLvbEJIl40@N-c8#Sn%Vva zbi{LK&z>LC)p|N&(d8Er`eTNOC5x%e+hJD%46Rr1vN{&G@_yk))b7TF)Y_^3(0;L2 zP?cD=2ED=HS-6Fi2UiW=z;`Ux`n+LT)3Dg&SL;Yg=!&daST6gN^vu_WT~SOKNDXY_ zngv!ZZ0uDBHQh7?k!iu%YWu-?E+&3K7|%L&+x(;Z?wS0pGX^PW@Qcli4}XY6(EcgN zKBSD$g28d4^+J96GCMBQS)mM7$!4{@=)!W69fK^IP`MT@$J6{fEhD&rVHmc8MUO*tSlh`e(~rNT5oBA=RV%0ZBQ(z^tanVtTa2B~+7`w+_WvX79iuzzx@XZO z9jjxbW81cE+qP}nwr$(C`HO9JI-Q)n=lzdy?|IHM?)|XG*q`@abFErcHD}@T)i2ih z^b6molVl-L;JA0sle9|-8L(!_?mUH$!iX|xV1Xb7E!=pDWKB|K)mq`ne_;dDD51D* zY@TYF8CsfPAvglyZgkP#3d=`_9gJ$zyJEdS&{iKFc{`T$r+DFLA&SVbR|4A^NtZ`( zlXe;Zei4Zbp|hep{61K~X4ty^C1|S7l+6MCCkw|Y*J&FYc~oN`hjyusLiS5AQ-0knYM$c&=qUq;lddGBBeL>{LUg>6M1 z5@{fR#jebpC3X=bX^#s#gbc5QG1GjaKm5;u2v-K3D1dkdtvACCh*Hypx*Wq5aE8qAdHHKpP1;(%7-SM zN7J5qcW1`FW86+0eX0F=+4CYXK9PApt3zyL66Hcp9b^Ae@;9Q7hpllr5Gnkt%O9xh z6&bO3JCDyja3sl@`M{tOOG_*mT3dpwNYfly1j=xDIQVAO{Ny5W##=hoLr(Haz4J5t zpSLpP@2hU-H$Zs&KB>t51t56X8vW}wkhmqg#g7pBC4s}r08=c%k#|1fps69=Q*;4$ zEkaCLj73nOs~oGLEf$e83ZBwtS%pMg`~v(guVNMx60g;MmwS?DXX^JIQ?AwrWPKO{ zg&d_qy&^x<*dICZSQ)`meQ8h>Dq{`BjpQzk5ESQASM?q|syoN^nB9vG7Mm42NWM-W z-3?tt?;}I5RU;JGfQ?TzSp8h58fKEe$X4lvr!TC*>W8_VIy<;B12eIV`A&{Vi*I*TN z-3AGsnLFFusWk@awN(5LxlaBE+&EIH{G5mrHRZjvY2gda%nxSA8O~_fqthlNJX?*-wLuRFO$g|(syY7?3=5_t~ezEt4XV$9p`aaXLcK0*FZ ze3)mPX!aj`i17XU|0V8M*~HPt!q&j~|1$>s8zpvHB8nVI$(4==q<_Oi0z7eMQo-;w za8h+-O>4F9ImtZw2Lt%~ag4T9fcf(881U&{wj>X(JiS|dpjZceLYOIGU@ZmmlZF`F zfW&@U0yvnW=f4bAkQ>;@l`kXXd99~7RsJ~^WQ&=naYVTk zYp;Rq+JLlnO#B`LqlDya0Lywtf8z&)5U2 zUcY^Lnm-5^PzH%~yBv?{XF4?}#L2#|LgUXZhc`9I=8vv)jbue>xn=Hy15n>skwDOw zg|y%$&!s0ediOtA@eJR2`m#Ddf!Q>6m!Gbo-#>z-*s!yjp+upzItxsZrh)%mjKqk< zKxYrmns!TL%5{_~`cys4?*IG&$xhsM`7l60ju`(+uB!fb%-R2tt7=yIC@ZL+wytbV z8L;ue#P!yK;3!ZY2*7adz$k*`Rlw8{PmEpnALidNPm zs@2rX@X_a0_I*P-SKzlh9KP8r@MbLx}Yn$NxdOV`KgaGZxaD@F;UqG z4~g32Ht?SheKFGvq(`Z_e*uiR{j^7e*eLug_-&m9XvOw9z8cvK6L8go(ShN8CSMZNOw1e zPQraSmS)_1IK}N9^7CB(f$xCnYdtP+{EfnQ)R6BKfPU4W_pqvWe@FPJYb=8HY=!4L zG-OBeAyl%zNLo*$X$5bK` zYSE)CmzBzIBdgyuG9f`8DpDi!`f5cHkcev)!PdQ0<8JGwjBmSDR5Gp3yok1^ zyUl6v<`$E=IjeK-tJuz%8lF)FcEZy8UMr#n)}7;-Ct7xe6=5lHf`#NuCqIDDAkOXD zV~Vew-`TK#2P)IKsI9~5{z8RnqQ(y|GGG_4MXZnOG5BjY|nC|J~ddv ztfh{GjHc2n5@p%IlUWlvi-496xWmb&X@vrG+qdOn)Cbd{ri*r`n?81SUD>4;h>xjP zhPH0XMvEc+uwm^A($Pw$;ngdG3G~9^h-pOZkw>w{6Vsl2t$1L()~a z!~akl(7-|QC;SymC>b0^nDVM`#%+iZc?X?GEQv}0Fqu62csSANI^U1Hi((AWn8tT7 zd43mPUREkf?){D1abbV76QG$DJMxDe3uZM5riFK9d8N6aVxigKVyMy}pz<%PrMOBk z^(1P+em-Go#Cc+RK{&L+gb?Zqhsa1Y$s(u_Dt-Bj#oq+yZD)BjDZ^62N|6{s%Z5Ch ztrQyiMxmKk7*+wKCh{Q7<>FjMnvy0|M1&L&Cr1^W(gEHkCBGvh1~Ehdg|cu@1~o)D z!IJ8kyeWl&V#1S%-;wNM*;&shZoM-NMfl~|gQipncW zmc$H|H<7|&ixMn`S*__dTKVCR!-8N!;Yn2w0Zh^E*e;P!_e#PO zG-aYLfLv&zKbIP|_JFrQGYXch?o__Em6C5}Vl`>`V=j8W-nCM4BU?K60eZn;_!JUW6iUq2j9?F3Tzp zAtekeiclk?BXqaR57ZRu2n1U_)@YZsJ>J1rk~QEYe^K!(l)I?$Vm7YOk_EhL<7p>g zNS=8iV>=Hyht%DyXomJb1!!;DZ{-$FwPSm=3O|;J)bZ=x7|pr4PE*O5dkrYpKx0~zux+$y_6)&{TI!N0+8=lD*`E}rG@#JZ9G<-~ z#Lz&^mO<}iJBEI2d1dJqHBH#7@oc&}>DU@sN z>T0{*R7y^b?k_6&OE{eR+~t9-!=8UpXa(G(5OYdowCH&P)x2_XUr1E__S{?jUps%i zLtRfL*aeMoiA<=J&&Qd9uZ@|K8t3<}kf=X2VvC`7fv`{X9EmcF4oF zpb4V-K%5u0>b7{d1+&_Y9_Th&Q3K<|3`f)?e1yY55Gu2|{H&@~X<&{1bX` z5EC*5?8G09r{7L$eb$ zTl1&c!yUjRVsz!4b0e=jArK(FuxO!@9+uW=jr>s<;qaTnPN*B)b)u8`BQcwtu_Id` zB`k|j%^}HHUk2pBjCUV0M)7inP)aqxBta!c>S=8D88B{=*gqymOt>@CBd}kmSdUpzC1D%d#yPo?&!!!>K8Ft{(cnYdt~FAQ$=Q&~8pq zY~p2R$rT>Ls=Bf{XMT||>8JvekqW-Zff@AgRjk0xKsEo?JE~L_K=9Ak~T@K{o@Y<5#+uAbT&|>%l zZe~50CZX9~XUagAJZ~)AQPSP{{z71m z-Ehq8Y}Gle^~Q0oO76<@0Fn9`e?1~?JeHkKh~H7E3Iwz@f6_PsPrdUGIpJ=ii}W(ENMJKYG&LVLPcI zHknb$Yv5yXti#vq=THNDnBb3~+m|`joZ{ytWp~6%(3|U!K40WTbvR?ES~u|H^8bLc z@@j+LAQDtq-l_Lo2} zifsQ8RL;<;zpa79oj?4WsM>948e7j#t*GfBNlW!cZ(y0?mYr;=XP9we*|oZ~>h{6Z z-o2^1m#MEDb+hSmySi!$aa*YA=v9*9ICkhv-i{x*tfNfsg|jP2C_$g6oy`Z5u=Bt| zoqguc>FEyfc!BEo%W*0ptETa;QMkONppMycYJ9}q(9~*ObA3g1J>B8-Fk{&bZQ}}6 z;pFnMmS2T@U%R1Zo1e~5uqD#Kdh${rd{i8T$FumN{4|N|#S%-sw+U3_eqj4D1J1NwH9V!&QW&mB+ zEWWe`gv`{bseitV=Amg_kcLlACX)bF&8p)55{Y;=ER7-1!eD&J>8zm(Ow zO4;E}a(2V;tWbk>RT3G8{kQ1|gXBU=e`Sk1{AWp%~Up{Qk6z5bPZ~J zy%zx;l-{1@&>C(+mFB#deARL71-?5}_G1;vl(X%ay~~xCHLnX|ew+dfj0c!ehS$eQ^0`8E=-qLq-vJrVLf@pwGMCLuUTF~zSvPcLje zsVIE^NWdYdMLU81CYt|z3;%TgH@)<~i@@W5O5k67RzAubir;e_d2*x~ljyAtP?3gH zs0~5gSWzLdj2DRIOL+ScGMD9zk&^f)y6_9?l2y7ZNeU#t!-4r4J|Lc7rh`de?#3?1Gsa{C3I;w`}SFptjTW=otlF3{+bpIgvWo)$(q z$69Rw<_UwC6SPQLI~yzPs%IYz6{h6L!45&5Mq?u<(+$mM?-aWUJ!Pk$0X75k8H-!t zS?nb`IN#z-=xouLY>WPa$z)mUWAkM}kV?$DCzvdWa-r-)%8c~3`qTb7Aj)Ar|8C`f;k36Da z`hd}ns6)Bhh!inKK4AokkPu<2rua08;Zj3{vbEv}$a{I#;TFaP>LIfuc$q_JW!MDF z`)2ETAt>ER#BAK?y3$AuL34U1RKy%UPi(SjS}S!Fy>qS^Ug%2`V(2KSoRd$~SPatK zo#5buQg*>Ecr};d8K++^tP2^Z(cPEd%WVE#mDbd0N5kYie)Y3@&Gi+*f&-ghSA|Ww zZEcJ68B37jq=wnq&b(|Lct2E_J=8VMAkoPMn_9-_lQAQ`cJa8F2Sm+{y)ph3%&?ds z9j$`uzLT;_Yt)&?27pY=gfa#C5V87^r$E*YDj4*RN0bEdbupVU6KR zul1@~yWn0`yJ?kqX$lTRTKUTcF{8HF9%j9EK#Bp9jEmVFYOlpdfWA91^WLC!t{ z&KDt$*O?B#pX>35OWjMstY|iX@gixt0_rB81t~stCKjSb*H1)RShF?m4QtXIh0;gH zgxN3n;PHBqMn3Stq?wqe_cDA>bH=XeO)|ueV)P$Q&1dO};^A-F?&$k``4?IDyR5#$ zZViM5l|AfD{^P3oFMnjCC?oex+WI7=r>Da~&C5qxK>ZzbfL5JT=8T z1eJUBEJCyjtA)B6P!VB9nu~0yV8FzG1Lm;j zl?Sh~y+;^cUY0l{49^lLG8R0Ejav0YGvN9%C~y>WM6Y77UlHIo%!w(ibxjm_Z)-mY zZT0FzKBvSIj*Lv_zIE^7+Df^W)gwST(yECnjdTxp3plountBHrA4U?ed#81K)T(IEsaO{(&jU zlIWW~*^7gBl;bGXLYUA+#%)jfB&`0~byudG25ELxu@*$DLA7$4u=_iYDMDF2z z(5ZXu-Urdgjo9Vm^p)d=^&)iGt4H2zNAKJ#|p4|>kPCLzL)Wq8fYUChRFaMLP_!z5I2|8nMZ$auh9Da)_jVg z`dJ=aDmaRWF1+eQ!ko<|pdAtoC=F3_tZtS=;DB1xfU?Y|BB}0n%Z*J$!h*qEnw4s& zOKE|*7Xi47s>Ypa6XmwxBCR%BTeg4pNU{lz z!%PLhNw|00g>eqW&LwsU(&6~%nZBD%%!lU`VOKqp3{A^GuB|)c^o0<&+!0U z7nfa=q|LlI>1Rc7w`N3l{dvDE5h5v7V9=W+s{zYWPCR36t?*ctvW9M$p+=XGqdTvN zP(ALICR-hEs6K~K9XWifjWlwX9zlq$bjPD3f6I-$HIf_&yAO}V-w&gV^)Jpp5PNHg zj5|<3#UCm{da>?s=+5j3K3s>t#8q>^z*o3O)=r&Qy{r$UqV$c-t9awkRlbMmF5XY0 ztUVxJ37nJGB`afHA~XO@dl*<~+d`bWs2w#YR46%Ra1f(wU=sa5MVNPkbmYtVc&_!$qUA<4OR3-e(&aFsPqxKLOVEC&>=^L|G zt`0bE!$_;|VznF6HW@6~&7d`D8q^`rIAQEC^%7LN=laM^;}c&mo1s&JsAtpYYM18i zBw2#tR=S5S1H`$jjO5*AMp~(>+;v8>)tBa18{)3H1+&uE?8~6q83>7BEiDUq2N7ik z>zofypFbIMIuJ52CSj=DwMM$x3l08Y8F37q$}3qMq)`+LmwYB~uQ-_dKJ&05utcFC z+L+o)3>RFx-%JI!+x1Ac61#3&|IX}4{IFo)r(kF^N}gBABP<}HX&h8X5%4ZgqH<4s zPahTH#Q4G7J$Mk2Xxq7M+}4Wm%4EejTqwSw%A0c6mA$E6<)($jQ^qq?tcg-w9~63^ zJ~8=DW9SiUYeFIY^S0YnrJ1w!;)x@yD1_4|IMSrOxxSxdbjY~f(^`WNdsw*&^T(%t z_Rc|m_;nw0PWH(J@)PQo)P}}Rb%czGWP_HD%j_JZKt*tttIug@(r3TTlNt3uwM7@` zkY@uqHP&y4ON+ka@g((}+3M%;#eD z(%&iKWo3&QOy@_^j*i?zMJ!CIN6>wI&%Vs@C2wEnIImPKWn`lvBse?UtZ=le6LTG$ z6kD7b9JlLI+GS#Y_L#)EM1OMhzP{Lz#N^Hxh?lsYsbn*CEqP= z!2^>9`z+IHuP2s=jcuN14C#UMMfRyXJRkr&>^L_7TMhAsgSu&+zdQ-__1Sr{qhJ7N zqwnFS4}9t@;ah2xV(To^p*qxMRT_oZX}P*7?3kob*3fahsn^n$A9d*b2PF)W0l;$ZJ7TPyl30BzuX1Xaz;s@~6(N($?03}tmk z!ayuRnVet!H!qj`U*K^l4-2T|k5I=8)(XIZ`~NwGUcz@^HW(HN2<5vkhTtCut);P{ z5#4`UAe8J}9F0u=i?vEHAly+*UV4ltq*;GZ5I4&QDhh&7|DsF}FO@t94Yn2sg_*sP zO$^w{a4|&-*S+m|)qRF`KeK@JJgqp*$yMv#?fQPI+π%vxuz@ZWk5|Dz%~YO9EUndJBcx(E@$efeqf#ys>(Di-q2X0=({gUfu#=`H}%*j34TudE<9Vn0RRb zGsX{YSfr0Yig%Jfo`F2RBtvy3Z!xfaDF8Rd58Y6{@jEfhPvW6GleZ{XzZ8HS$5o;raM34j4*6ij{OS7LNy@QOitzdc50 zLntt6BMAEHK4Epu0B5yX95@vETK_LvgSZj{ZBf$$M{UGWQ|$@7szLh5DiUqvQ8aDr zQJ6bH%(V~)9@}{PqTm`NTS587Fs9Uu1!}Q1>)_2Z#OHLZ2~JN+#$r=Wgi53L55Th;uG~wh84u%D*{ZFoX!F+F9b?0wiu#d&uL-_D@B?bEf=JvAE7eu z*Sb54NbqEw!SYNc96gmqqbgbE*jj90B1GcC&1=mO6`8$-rh+nv4m4l6i!ls5H5nRe zM?g>n?$gpZE7`s*fb`XAUKW|7C@AjAjg91`oSf|C>U7_xQ4Me%Frw1q$Z&$-p1mL7 z0fU`9l@${PL@Yt|@lb2ZR17?YoJ+Nc-$9%%OW~CcG$HlLzl>5X*Ne0Bf3#*OZa^3~ zcTruY^O$nha1DvRO4qSgB=RKV?Zj?W@CwD0H4_guR@XeIhZZd=tSEp zyn{N};je%cuPGyjb#)mDH&e`W%&BS%swWY4YcU;RnQG49R{s>CVD3n=5TG8RTJlu@7LdGH{=XFfV-|#{Isr%T@P;fDuo;&4- z(3$NLU7|pwq1i~4(Ok~+PH|kVlPVYC5Y=v%nsb{iqQvXZrlFBB!f07Ie<2c&g^sim z+hAIqLHW5NXvA$$N;J0KD^ZTfl{F!6CFY2N>k)8+Q3#cwrU|N%nUCb!Jg|F{3wT2e z^jrcph>{yfFy=-i;NN5>1b%>bvy4Nz?lbA&8FJGGUIKsA!&xDB?j9-ZgGv#!Yp1xiC^Mi z7vXkCI! z5{HK2HrEP=YERKlnpF`~}bBhF87g|gLtV#>YjbYsuj4p^-gI^_s`L`F$Gx>T%T2md*l>-<{?9dCc zN`;a8ABzPk zP@9?$r}GriOh%61XHElj@0`TKTVzl085_R4|j0ruc zEiJ@sJAm-o!fu*$JzeH_%ovJ zOQ#7HXDB@of}Rx69*hi~9l!xqr?5|rr@YV0UA*6N_bX<~5E;3&L}b*9CI<*fw`~78 zKKTwD?59)Ts175PowB@1{E5HcRW~M66I#?(E0C)vsqOj1xbEjgq!K^@Hc|sCBaDpj zMWzBtLT1|mJmxy?q>Oy#GtpmuZoyw{eI6^j(gecM2Y7YnLYl>qir}m1s^Ygi6qOEf zGsTRdPHAc5<{k0isH64*5Daz&^YSPr_U#o8uVtgEo*O(xbfnI0`C2JAW{U{y(#LSm zCypRHrj{)0pSxk^g^|K-lN!b$Tad2F;a6^=UE#EF$H1(!LV5UJ3iX_j9~o_NX52+u zo`2rE;DyKFmjs~lSBz3(N*m7Q~W*W|#*GZ5SGWDk?RD0j;HxjrJhx=tP0 zKwC}65r$Y^RIJAlQ_YoG!g)p9R2=NxAA~F?gdkbNTC@458Tm@o;!f>c5B=epbcz3N zvM5ro&$Ju!kZ(a@E?4fR-TykEV-$}oYVZJe0saUGG7bEv<>>|^_H0y=s3P(oBJqqR zotn6vze(X<2R6t#%~Nov^jv?=EhF`=*|L+8><}x(T-BPXm#_*Q)ymG$GH27lSD>1J zRtG_~g~D2bp85iD@y>m4fqe9S5@hkMV*m3k_-Y5v)Xijo9k(QhO3CiRpztJ4m>81U>%`?kkC)G)U!pvvM*7RgiLp?RBI>GJMn;iz8 zFg;M1Hjgd=Q$SfYyvQ->m(t9qv__zG4K<5-z#gL3{&=NguY3eitiQ#x_pTY3wAG0b zC+S_^$nBQ{I{>k^5YKq=Qim_|RR3C|Js*kGQHGu9mkt7P52o$CxSD%$)b^?sU5Qty zKkZeKSE+yPsMssDyQ>B=D&zg2;UcuVtnQI!Tyw==165&#wnOj4JeavDDR@L7S;xPb z9O`P4@7@83QXiu9i|E;O=P+OkdU(;FQJ@Q!FTbA*1IAYb66g@)Ykwy8D(Cxz$o-iD zb*AC%OoLIbgR1lm$yZsOTT8A(kxql_J#TEF*U=@R_X(PNQiBn0dl$B&4EK%Ucdn9q zvH~mI8fLn80a>u&U8tKAa|j~y1S_+eiz*eppUGl_OkBp3Uc?4FV4EZQlmWTV3Od-M zzH^M4n`(muX&1riZ-bt+y7W$ZvjJIZ1DEmZ;`cS~eeen6_a!F($_X53reU|)C0W1Q z^2Y9cfci4@ZsVVL$xnPeN@@0zTx~Bdp(wX{x^%DOAB)Y8eT91Y6T6r`>5F=c4!TdM z2V~!r_xJBObNm^6#lDv zsz)37vF;h~Ix7s1Sw5SX3;Oo8LJYD>_->H%+Qng5PSbpxJzA|GMj=6W8eC`16N7)T zy*>V-aCc^k_KJ-h->xiQAtbFiS}3Wl*Ly=ZNt4H+)%gr{Me)iB!CZfKnDwu|b>PZM zpGTalDUx^UZJ~Lb-oV4H6aDq$zO*U+w_vw`4`QD^h&;+M2FFI~vA<83!0z)+1+z@_ z%n*Nc;26C*{cvqg*d2`+qGtuI;YYFHrw7{8h^-%#;RlJ<)m7lWLdY8aOdkJJkL*eO ztL^#C7khr|k+}bQg>kfSHF5kOtD(}C;&=OrPFQN`C{T%fM>sj*yknHR+B_yXdn`(% zGFK3tEY|EYy@jHjEarWf_q~8_J`t0zKmKvM95xEnYtjIZC{4_0-SD#|*s}_c#_e0IxMrzDLJsj;{j522^1%*rOL!hMW#n zLx;`62ALslp>I>S5x~8|ZjaRmK)8zsOxeT+h`CS4TcqH7+s`ak4Q`5Mt(jEKZipAY zLZAY!GU+r;JCMW$D&?BwOuve#EKy4f87OGtujIa6yXYP3p;tQzG?`3c z5am;e{NxWMcq07HIFQ9X6AtXPKtJ;dvGtr_Y|L32(+`uE!});$3lJFAtak{E%{eC; zjz#>!Y7C{vY4jZY)!=+eA>M@)XfcUJl$D-gLwHXzY%wCpjmkNeHNUwlq5V~3m)niX z=^c19%Ru`qz&ChI8ttrxypxBh$7c<|Ox!HW{--^i^!JHc@V9N#eZDwDyF(4@dnfm#_2vJ2)73JAfhjicPZ7@Cj?7~#YXnga1HR+SHpiS(uGh;| z$7|jioi0fFfE)C|cP|yRDG`Q{Xr~j7+IObrcVr`~z4l>y?Jq(b)d30%JNs>Ad#p#C zHg;PojPC~5!a#!~_EePNdg_WBd+#EB6e^r5=vk?*QHL}F#rFmBI@Gpj4e*yALSJx0 zjN2ZeQV}jYv=tD84sbe3uR24_+Cs@IEhQt`0_-M3t&I~PLhNSfU_%@`h?!K5nHH8? zrQ=;k$-&2rADQZVEAY%K%hk9n-?Dp4k_~`16;7oMw*V`ZDtjzbRzvj?w5FkpI3-Yu zWJ~dpW2-UvMv|mz0^Z;{vk`Xkp@8;zc&&=38`>eK6Lcl&LhnU6wd6CC? zIQJy{AWXT_pEQwcf66A&V_m2uRzbNbJwfx@dJARbq%BSADVgjX^-Z4U%dqmA818*^ zGtv}Q#`3kSXTh!9+N&|Qb3`sL#t;V?=e4js4nn~@a;`9a_!QP?24|OxoVj0tdl+1Z zGrnw}YZt>jxGwcYI>S5(UgfZ^B<;*&hw)x4CbnYy zGYom?EY?1aW+UzE<#e>yo;;QyS$xr=?QNGR7rsBo2!C+^ywu~Z&&1Nkgl}opOZMq& zbHL%QSH|x1ZOj2!Z&8F=wv8#2fWz=Sf{2=ou15RUEr=kyfGcqdFNjph*KmZ{J^*^K z;1u#N@Nbx@4rv8O-{2cZALA5dPfJcTUcomvx|2e5F*VxvzGz_@&wy_FjgBX(r@t|u zznAqO|Aq!ozP+AT5M@{EQ{VhXoKvF=S9O^5iUp5I5?={FoxhzBe-Cb;ltngwN99e- z1gFHPG@RGtAW{nrG`5gl88;1=tn5BZf>UFVmOqHZ2UvpzsSI;}!Tu%^Aj~vnDI{oY zO#j=Mej1oS*#pnVD4`o}-K`t1V7G+t1;&2Mhv;(p{jJn3xr76$$v3qQ68;+ zW=Z_)fR%kYYvN$gvh2)kq+PHirfmFpw9V}Le|UzfpdcSmzqP!t?>Q^~oRRRqX~zFi zcK@Gmq-1X3Xksk!-AnpEyrcRz@3@AT%aNiJ3l>Z~xv))-_KTo@9URJN**{1KiIV^p z<7j0^Qqy^p$5pMc9o<^nz7FHnQTB=$V@%K?F3Df&>dBC-;qa-QxIrmrO0clYX#*={%6?M&8Jj_$|NeHpMm9xpqH>wt~#I^3TRd#-JQq3_A=`?P%9aIbFw z-FCUqKOK+$uu!Q#C+|p7^-=)@sXwRgP*e3%09MqmQ+JS4KNz_uA1)cWM;qsULKNtQ z+S#8vVCL%oE{2tB@ZS76*WbkGoy0XaaYvK7mkQvb{yAZve*DPT-TxqhscZOK2UFMd zcO>kd(QhA2-~NX%#*gshIFmQ1sl3TMrc}N(fJ$oJ#2rj(Uot?AS}zR%m)e&KkV)-J z0JKs2kqzByd2Zm{j8lk6 z?|<2mQ1KWgM;E-!Z$KRk(!+JBwym_Lqlxsx zK@xE>lbvzoYB4^F>zuBpD_~{xsk1qoK?%xEOrx#Y6q$@#oWTcgCc@SsE-g-Kfq$rk zwwCu;dPuWyP7cMEVB@ARE#-%{(in>={l07!Yi@@W98}C4mdz`^2`Bq~h4VcnMAK8e zB}a2bVM|%7s)>3@Q2m+XCTzDIR!pF1r&#`IBq4pB9*j0<3bh@)Hl<#AfJ0r@p^_(7 zVz<*pY@mZ(ZtypBZa54vx5PlwJxXwc!cL*bmNMC*goVH(c{>97FLFdpM4Kp~`weK? zLFi1XOl|+blAl|!OogN3PL=i18Mq8JYEcyJ!GEBi;aaqs4lRSNWsRf> zscSiNplM|?&O(datW1A$)C^ieS-GdQG)rzpJ6p(hVL3=!L2xP7%T60vSS~Xga-nFC zR?*focafbmKR35orPkaMcOAD24*ufOl%R^u!U0fLmqOXhv|0=qSz^~Q4SzaXXYr~v zQT5eSs1`5j81PVyvFD_gj;Kr(pw73FulI+J#Fk*SQo_r%&7@D_L$f~MbnrBnK-U5Q zNz3NT8|UdbggQw#>>h>UIGW{wGFKMb!&8Cj6{1_q741WPZt5;5w*a zSsT}9G?kW1{jNPjJLxUaXf772S*97@8eY%gCzYD&D)EZ<=N)gHqfs_tu1PQ&IwE@@0shNnfsVJShlqzmnhwMcJg_-t)XP7rKvj!x)Sw;_iKc zKf|5zwCgCCK#lEUQo`1PyDbo8F=|!I*o{(JiBurY-mNh2q7wlU6i^F=XQ|~3TE=08iG4KFIUOkL#&vhTNH0zRk##w9K$18&=W1@1I%$VNAAIv($a>L?nA&PsGTH6 zb&B=cvce6p!VS!|CQ2plq*}<`3@F_U8&e9Q%d$hIi-Mx1R7>ZE^-Hjwi!3|@U_hW5 zfC-`viE$ogrJlebIviq)J~g$hOpz>!kt)$!HlLTfcOXm4$8tibDRwKKR8V7BZ>?Fg zl$w{$7QRCTj(&G!AQLWDxJr!0v}(%7da*tqxUiLj^umsp|NMp#XRT7m3X?NNq z3z@<2TcTw5+ZC0Q5@bXOCqNgL5f;L&?c+*!OER8(PD{GHR+5P`SZF8OnW%{(E$CI5 zM(!rDGYrFnB$_mLcFENZf7x$VAJtP!E=pqyE~Rgx=}un|w=C1Q1Sc*ZfRBi_2vc8% zt7n2?AIIW zo$CPJmD>me_A{)Xxd9!Dp1TP4O~cEPE|o90^-vj?Dpj4VX>q9mbehJJZLM3g)ssja zY1CdMm<=TZR3%E{eR(!AUxO)Hn|kFO&v}Fp;~F6P=ts>oTIrAZz+ky~8hQK2Et7M+ z{e?TvLKV)Ds|rqs5rhJ@nHqcW9M3c2pV9`NU0rMGUO*-0)Y{aX}NdsqPyJd-Kzn+uAC)l}>u3ALtlv zQ^Jw=M9Gsk`IUq>bHAv3Ti~;cf9X76tXStza6NC)T$Rhl@q|Z&lE|mSbm_AJz3C%; z&4B0Y6MnH06XP4N|4;)STv?|2HSy*66X*tAkN5LYR*@@}`0~*d7j%zEFvcENTTZli z)+50fFQ&b+8Q<|@3jTL%D(cCLz-|q_qbvT=Yfy7vwfu`eBJDo9zjVL&1%<91%Co3f z55qFmou^TS+Szktx!Tq9sDfShS{8p##BOqcZ@O^5u^dl`rxMRIvzv&RNB_cKdsyHx z&whv6p!3|I!wR+`rM!Jk^800`3brYwIoFjEJo%H(tiz2oCA-?P+|$juSAS$3{QxsR zq-WU|&oX(tf;BDS9_TtXiJp<%A|)L$Deiq{boC2hm7B#%p8V04@@IceboCS9sT~3; z?q6phR6P1AFu~mVEilVo{6R3xp8c7;=7hhlfM>XEKulHSV_UC666E?jpCJHGX7+yP z1mv_d+P50E`cjgInDihZyl|_E4|7P4tkfuuh}6iA9Lz8sXq&;&{MzF+_OKIpuukIUH;ODWq&#w#^Qk8`L~92vqwbzreU zbzpV{M+4vq+zv_vZ$`00x}oq97y}?euEtJ7yn*#lA6UGl#@p_N2>2c-sC|mOl|r&l zvWSd3ZkY+v-myZo50yqdfo=B*`P=@8zWX6>U+voG0V7?Lys;)Lp4e88!yTPW*Nwv* zoy^va|2)DCo%m}%3cT(N#QG{gRq{K-8QK!RnFo2C4|Wao;hjK{0IXQ3p7qzWz!Swh z<&i&1e&+lqwex)2_Rky&e9qtFWJ%8)CSd}!+lfFG)jvB2FP)@@E+5yKMKu<3G6`9x z^9?+NV5OLT9+5~bq;CLI%9!F0Bby^$sAebxD_(DA_Rj=&w1^T^|EvbBB`jx$Yea63 z|J7!T0_itf%X=oV&j6ZM%ylKRPd;dBBHKd)jE{gdE8Ir5FC%zrTF8ZThg8s#aW)Ie zEqd0-ynG(igk(=vK*hMwglvylK*hA+grtv*H7PtN(@!dR$+Yl@RG&RRl3~Tp2mA=B zlNg~Jk0Blf15qLZ0!>A%2wsfx0C~LS)O*~UV!x12-3?|dQ$&EjgaFsMZbE_i;z_MS zm<*&8=?@P`?D=&}`it#U!Q2DHRiUE&Nui(E!ys9cFUITV7DFTR-9OB;%WgMU7~ftF z46OgZi>STF|GWLVMjggmNe%NWk99)Ejt&eOc+v_OyqK=>TObTjSnMBM1kD7YQko*^ zNQU`eq`hO5W!;voof)>x3_CJxE5o*J+qP|F*tTuk&amx>yWgt%s?K?<+PQVx_hbE^ zZLdA&eAXC!^lpZ9`z_o-kTt7qvN=N-mji$C#IZlF@AR9F$IH@W^Bq0@9s4#%R{vEy zt5gP4&?n2=+XLs3=h0i&UFUSG+umS^G;oWeqnK*|jGu&5#v?;s)ie zGbYky>^0h!8`XSAl!=YWH5}NL;i-$#{$@);uNKl4Q9cwfQnd1Jr>hR?R!qAQ(6Hge z0J!T_0L-P{b*{N52EHCWz7Z}M<`X?8{%|J)kDm-fOH6O^^N!5j^k6pG`unGL_ z5Tr7J$uL5`%otlx2r>8_D;9sRvu zu8-ur! za)WJ~Tmysg8eS8D@fu$@;PUD{8k{%g)Gd3H?jD`jo2c&4Lw4`f`;6|WbbOqdb46^A ztoUt8He6lgy_Dv12;Yz(KEi_=u<=3?5 zA_F;wY#R&Z&3&|QZ|$EK4Kxu%V}X~_gA0+sJHsrwh<>od$ta)Man~#BCY6s&bk1ra zKCw8^4>9j=s&`BE>`4iYlCQdu=aHmEJHmDexf$X1hCJ&e_ssIuRAhbOt04q}m_j73Ce& zC+V6M)MuC%Whhc$QiJWkbT8Tm%Orq=|7%wli9|FVPqXF*%yUvtMOgBkhVZJ5>Jw(*d)F`HW$yTO(JI*YTrSGXNcBcbL5mM;!Hlv2+XA&MV|J-^nO^8 zGgq?v5gs#c!9ATjL8*Qic0^j$>rL65H8tWY*-jfDN%NWc&j;$u5yrI-D{K!IOIUZ0 zJ`I0uotWx+h2I2pYLFYA!*lUK=90XY3%MQ|A<5F^ookI)jVWrx>?g=_opkCvA( zR_FIY6rtll35Y$?6T}md6ph_83Cpdd)h*rcPCSA#&2mTASXuohEO6$J8}j$A%ci5y z6ZsVw0dxiSEM|65K+VPs>ql77A@=`Pt;lne^`oDUp1vuDqwl{wcpY~su6@`DGZ$G#28X3GkRM_qBt zf!rCS$ax%?q$CO($&jn+kDuA+L`%t086=9l$p+^nYz&3BhgQz4yj1UY?zaO(1$+)+ zt4BtMb!3+p=9JRm(IPWSgoeZkA0&wbI)X~{MS4(j9|Ug{{V`ZqM@93?QU zMr3BmPec7clK$L6B6t17bZ8Fw5L&b_J*O^)ys$1@DmHP9(sr|_p3P8^+Non8ps|tq zgs3t|<8c2pM^)u0eTFWnvLlVmor*p_!gFzxS!#dp+$rnMu4+$zzXKQv)iq^Vo7iB9Gv}m7qv%fwTLoo=R&ynE1M}l z=TO7ZQ~F%@`<=v`?Kxve2eU~Yw|TT*Y%|Rdx+l+f7Aqg+sGFnX5u=7I>tJV^qNfuP z#+uSm;z7T+9z!1D+iIqGO+*;PX4Zw--K#uHQFE-u+;nN3<)tazVw;5N^#hBeD^s12 zZEUDOf1&zZ#)V$|-;Raxue(*wym}Y;19ev%)iw?(yeI`0&S7@*ApjE#fO{9swqS}~ z#vFwC#)(p&t2pab$v+Y^kGb1q)5)(4Whh_ z{sB2!%qBfVX8}-MK^mX7*@|S|L(__#YBmF@?MQ?bi=D^Wbe?mlGq#U;71YUc$Yy*B z^rt?lg{sadX5h&*P&i#Qp;S_hqu5bNCznD468T4|GK#GlqkVhI4MEaS>X8<=9U4(-+ z`?VKLDnP|ft#@YMvO=uR5en*;pt@vAn{u?nfjnWuoa5%ZK_XXS-6Ltxk)_!A-37y- zHUb`9(mmVtoQY?!iA&sjGS?enSVV~^h8bcbYW^SLO&6mJOXwuw_$KE9nZ z{dUTkDe9SHWhJwpB;Y9ffKf!LI}xUXwpnCgXYsj~W&F*Bgw7a(I6H)Rayl5sE972{0kwo(O-z?cN~zvvV0o4#K%HvyVBB z;9yaiDyquv3$edqZKM|aWKPx0xekj;!|aMHgp`9s3t+)+m4P+!hM>W5n_T=@s@?u0 zZZCB9qsWB`^}Vgl%rg~W9XgF-uWuef!dqQW=iKY1$g-#!nPk`ku42F))p1E6tRio- zGdqDY6^k8JyV8?kOKh{_s1AwMs%w793Ng9nN|USeYl2WlS?)*vonGulq=qEL9!A;k z8P|2XCC7wLi(+L?^2QK(GSR;q5O9{c z)egpL_0LQ1dyXcTWDmV2LNjtp*W{wY5*%W?na0niUXdv2v_!uuL{s$U9loqz zt78~>iUI2l`bC%s|Cy91inAzE&rOXz6m6|A@o;zz#dDkF_=lon%%9*slY$2A{cmph zFzqN5gMjuH0-(+H_lehkY~lPR7}wWxFcP-6x3L$qHn#cq{*7?7lypBIOi)g)<1xN| zFG2cZD6>SkP~i{0xE0RQFj7j!xq-1=RoicHLAL8ZI$pNwvwgaJc)GuB?2YN2)DTW+ z%#xjWSO?kG(~d=hKYB!0Rx^dHY#|yauZG&MAZMNoBP7+(DRm|%)vX=RZ3s?LY!?Gu zMVxB6g|+4JnF>TzKZM0G2b58zUatcTVPW5_RaieS3|T5P9exg9vviU5J$z|QJCQ|6 zLk2S9j7gp2k^0`~|8Xq9b_-jLyNinu#D50;XN#*DzP|TH$Zy~9(Ej_7+CTpyZi;4B zMq<{EM)uBn|IyE?RB`h}UPk;};&7%xqXvO)B8daBq+`&HM*zQ`Wov>kkCdV@FrORJ zE)b3}x$BhW zVcKS^!|iTzBI9Z6e16e~`whq!`-T-A>*ia+Q_D;=GHnO?Re8bgo7&@dMEr_@kn3G> zPY`@3L^s)iVIi(R{*f_^FM)wF_FojF@NL&-oV1)c5lGo_*A;;hNZI=(+2}X^(CuvR&TzSiDO#?GDncE^d)PpKWMzD6 zY<}l^l$0CD`XV6JbTDR(9V{w(^GQXUv50F*`j$(K8#acfFXsU%Yw!zxbP9TSz~9&E z%lD+OI=l&RgTDrFadJ#svaPJH&kxp9+-hdhU_?TzYLHSlr)Jx<88Q(4E@RUYAOTsJ zKuXzDqpPk3XTf;v$U@yh24-S;9Ri`RNv>0I{$8s=%Fx{+Bq~RNbC7DNL zE`_{yj49bSpI7CHvpiigq505iRoHIQZAdz_xe#?HV(clTD5 zgIRitLw>dhV+ih z-dWuhHR(Cos$UVMF9R)jF?e|>Dh_r;&c)7-l2V5t&8nMDy#dgW%qjj+K;4IvJsMua{WMcVAHGi-&o z8%6HmeLs^>78oTyAX$xzii$EX^}OB+P2N2ZdybAwxzR#YG4h139$|}ahPIu4ltDLm zth2_KAkJ4$?%e*l|K_Unt>w!v6o1SWmUk}=e;h*~Br>PwWEn8x!P`&uigqVGCmovq z`iXQDI-jjz6H_*9&u}RyAcB<7ATK#5$sifc@WMX*=|B22?LAy9bF@0`Pi)E!_PcUV z5y>SX?t5k8vem-LnRtii2|mN zBC=aZn&kylxSO^<%%p19QAaX-%dY*P$wn0Lwqr-0_6DL;RR$Y&pM!_|wNu{sDRU`c zEwwgNwa)vFTyaD%!5yP^>NcT1okf4=u_LZp#=gi^7DYuv z5>;g#DU-uC1s}0RbdN*6t*nq=SdooN*?1?)!C7`FA@QX%r)AWsd(U2okf%-?x0p9r zjx$yMQj25#yI7tRq~kl4T)_H_AJm34{w0)S9`w6Pz;EHUV)D*@!cC3DVCp$mv_cp4 zf=%HQo`{BG@{2~%C-g#_I?=cAxfz*3B~r63Iuc5^W?B91xbicxxc%No!wD659YF=+JW^YxT92*r?st% z)yKcK`92X+TD}*HGS+AXegNTor&=@nt@W{N%Qfj)^f%f@^%L%d0E;ZsMs9q(yRhTrk^oDDP=NfMr9Jl zh~xn$oah=wIcKq$>RHf6QU#~D#ViYz;)-J7Ra!6V3xzBE10|U%S|JwY$n1HnUzR-5 ztU}KueEz}{g~g-3ge`+aINetRakI*`-c_KD#p&WI{7%o6LTV4QV(E^AG$iAJXZ~FF zyjV`h+TF@7DEqX}AEmwh0fx)_GNJ!s!zlgYbFhXPv?3>0iHDbHJ=A#3D`M9ink}b2 z-B$ZFzQ0n4(Q0eZb_4i^8Ycg1mx{==Eh#jCh*ld&B2r~^56G>esI!`MPgy(IiT z@Y2FJ{j=VG4eT>b_wn2#Am)7T*6U{1Sz~=pzF71!PVD?vW(Q5Wl~)?u#6o8;8?=M* z8r+jn75#d8r_EPCr}yFeAbkKfQ23o$1?>i_duiIRBhg1u#f~__wLIFg!a84Nv^sZV z)Sw3Gb-y)Sup)~adT#-GPYu--Q0fFBhVwxdxC4qBa=jm22foEP-TByA{kP*uGa$fS zn8n8K1{?HJyT^^^r^V{p+?ix!pE0fV0-@}B($tt;f>2cT5nySP?W;;3#KQUJs4~1+ z`F;gCuG@_;Ag6X03(gwAZfTi=e}y2=ukl zK(&UOuQm794BNz!s(&40dkQSw4f`}5*Ro*M%rfTBDg$Upu{o>K1Z2y~qZpF1lwOwF z0yj!S#)0NWGv=UI^B-n$@j=w_kb)!uw@5=)1&@);efv9@QH#<}$%`CfUQ_kQv}_z^ z!n6k_%%-UJv-Z%7-H4VO911HYfJ7mq!u-=;J#B0sx+`TFb>-XY(Iy$1Hmul1 z-Y;GybDc)+O(>{)@RoWWAuu95)W7*upf)k&BSWSN&s8T08f-@q%EyWZV@o#El&53b zpq4Ncmv5(VSM#}^U^rjgGq&wJ`-LS~xrWgy$7KIh%oarkIGS6iHggN8KtN;46PG(I zbME2U*e;i`M>}FT-v!l5y#{25TByzb7BEKvpN)!mm8q`f4`2Wlx!e8uN|BzAaZ!rEAu~@ALf*+& zr%KuFiz;yy%0=`F@s$R7O-w^c8fyp~rtB%upm<4yCY)N_2FxSKBcP3Y?IkI6jnG#r zXz1k`$3+CFYw?h(@$EzZ^Na0$m)x8OD9YafF7khmEc{2X|J#fGFR@yd(wPi0Kf)Ic z4Yw)0@ORXKTk@DWDje1AorosW+OYHp;sN+Wcka{PhpH`Hp)bO2@jOr|PtV5~hN-zB z9tz!rtgOzoZ5v|~m!|G7&$sWokTL-+zM>Ecsc3Y_1Jrpyhwig6stB90{-Flh5umh& zv}mn*Vf$4MG`)@3K-N$#Bs(ktvYz$DtI2~1Q|8WcTl2WnGm2{UxLRAH@xseo{Y}Z3 zKvcm-uJ9Pj`LI=P%Ox$?$;F%+Wkv1wMQ~}G%1?@`ik1EquF-m^$ww?0?_!+Qj3Dz@ z%$)-tnt(!t0O+R2?h47?XAd2+6>!ztJy)|7KY4Sk27HDUu!ZmqH5i4avQuZ`X({{S z;(W9LC(PY!YcP%VkwbcioElS$aRp6AN^oXG@bFHA{Sw|V%VcKRMJPre$#)mw4dB)X z)}tS~!UJBiQolG+bq2nHu@4J=j}hV;Dhov<7LFt&A&hJYWtI%ZINF0mPe|2`jb88L zI0?h@DpF;@B&6BWBsMIZO3=Gyn{qGC&G%}%l2mg``*7)zg3PkDDp7Z2X>?kmrHf?F zifGrMT#Y6(8Bw3XrMz+FWK`2}3pR5NzGQ$&b=I?5=FpeY$j(>C(i#wIAi6tyd@^aG z$XzQcDY6$YB|+nDwH?z~oV`n$-|>pA)vS)+&<}ZAfs?tBpPl;f92*kPNh|Wo&Ccp^ zX!5?xXlfF)YPPToy7c1Zwtzu5bG*^cU1s>kYr~_iQ?y9f>BW;?tS|rmP4YFIHu&=D z+Uk|qXdYh$ZZ}4(d*%bw&+B>{70Y`MyNKSuTqzsl#&H%W+Dqg&Os`Jik7ok+=U=6f zTQW2-`7O#uv1MtIxYGzfiB{5uFbs&<2btvP(?iNpc=Aem$03yV7(ek8^Q-13?zBw< z6_xeRzj>iPZ2Y=6DlfZt$e|NN;Sv~u_&5_%k5A|F3cUM5{{DCQI?~1hL_FZwTY>oY zjqU$-^vT*A+5RQw1f(xG8d(`xJO0;3sZ_FZKnBF>(L%1)X_3=3<;mw#wz!pQl?-5Il!)qT?S7|4Q}toT$fL=<+9~phrxf62RpE)8P>afHy)WXe9FE9 zgp^;te68|+eP@fwaiZS&AW9tu$C^47-O0@JEFnK+0@S)kPDk|AKD-M0`@*9sGF5cY zQ5=={d6#VEhDkgD9%htdCQx6k!5zWlSY5iJT91Kb-G(LUkJE8}6fQc6wRfB0%-$8tRn+PTeBCKnL7WkIutZkBI2^>-_kH?$o>_iCNI5b8*# zV8&Z`2QeN8fz#TM+baOG>t2IzSc!Qr#$9p?q@85t1y@5HFSQ^$jzTJwind1Ex=r?o zd0Op2RWW4ZUN6SMp)1MQmUk@f_1Xl~NxBN(&G6(~Wy| zGix&1;x_rgZp_#!;IhD4?s@=|Y%th@Nq{x+XX9Rx-5hRmBQ4vm_rO2T_IN-!f^(jp zkp)8wLQ=WFEh}!Z0B_j>X`hg^sM;)GFc{Qx5%7*p`Q`AKxa6t)M`@hD1+KQ&o{4x$`K@b$7p#0 zx@Tx*y{`kJ+qPuMjOG1yqfW@ z_Aw^(f$mSPzJqz;fhr+spAr^%RC5-eZ2P*lf_-jc>Y-^$1U>O6e!}NY=4SYm9B+8a zd`i7T1bs5bLDPx`%>m=bB$X_dUDVLU|FQ<%w*L3?b8cer<7B{Ogl^zsbcRD`{1c0Y3y@p1apwW*85Q{ zP+Wh@`YGPm5e^+E3hT;Q&{zLJoJbGjz*!*nlb|$&WYth8lp<{xACxpRu>`b{Xi;j#+NKZp+8vH3evAl)9=Sm~4I$Vfmk00hi*p0?j zAL92Z+NIO7SC+S4k45HECd=@t2ohi_d$n#J*-}8%4|N}XdCrzIm9bAHYeh8XeiH@n za^Z>+jm#-$7d)Xvjg8D=s5Xd8ya3Z#;1lV@d0e|C>2j)XbOp|CYO%{-3zp z-v!mioWJll(zSAv88yX)wUqy{1_inN&jAa3@yvzaNe!trjP3Q?yrVF8H!rVXy6|U^ zl6rFdq;Q~WsEE`r#;dICpj1%Gn56A0MJWDy<%RYsBhIxT-bLx4bN9+}ZD29uBxDmdM^z0jc zOEvr+_4A*(n~B3#{eR2d|L|h}3n%`GyFvexyCHab(XD3nk(trOLh&aa+!q4Qv^qIM zKZU)MgrL)(pU?4)G7F=;LXTFTVXR?q2_kyW7ChR^^_Q0Nma27k3}17_(x&Lvj`~f+zsEd*t7_+wCvz z7XFL7ajm{rz1(xP3;r8-LuZ(Mw-5nvH@T*WQ)j_xsoLVA!e89&FuU4DbNQz(y+ciV zp|z;e>M!nw{&()C1#ovo*+n?UfYE<&H_PDjOI(S9*2KHDwS=?`Ty`PQsE24ohIO$O4ObI_Y9GIxsri z{?1D5IM$eJKD(;8jLA7ow`NOwsa=yiZuw2Rp`K4Y`?^1ywT z%53BoZEog>n389erqkTT%Logp#4603lpxMr`+O3luvHeFO_c+770__te((ovZDUy3 zH9RG<>FS7)Mfr4tsS^6(93}hiA!ECtQOmrYJAw+M7`+A_CdKsGt$M}+^G8%VJZ_6X zIjH-aCzH%j!81YXJt{6t*7olDt=L#HZxv22eu8)I9W)U5PIo!e`v9wu0md>V7yaJJ zZ+4uwNIppaZUO#lBFCE;dC1M2bPN&O&b}W?44B+$1lj^M>|t~wV)o$_dHPK7O5~nG zazQBwl*1Jzg2UZB&$4-M{jz<+0PmaFpfl;^y#v;BhhcS^b+dV>{ ze~|(GJ=UgZ`=*l2H@`B|2OXD_=~&W$%pbk34hgc?NlK?U$4o#lT1bo z@9nI))euf{BC35P0H^;%`LuCuvq)K4^&()a;@&laJ`adnP1o6lao^1y#}40E;Df_> zCvee-*KsKD$dNU6FD}`Y0BU2t1ktkZQ>2uyBGS}mv_;avYzeo3IRf)^D(DwpMBbpZ zX>`4|AXvBn2V1&s^b_u|`0eWiyN&UTCLF9FcSAQ}zik7%47=fLr3=i;ydfVV1+4Kr z#Ru68o7o%hXBBHQ{3h1nWD_&Myg`4>?M-)`4YWBJ{rVcW+b7#Mmeuc+)I90Sn(3?J z4oSO&%^W)7| zT-5dULtQ+0Tril>DJC7;)Q_v*GGRQAeBkw>s*XWx+v>4C-Xlp?e=#l6m`(1$KB4S~ zA+J2xxMcT=F)=gDs-4p1D&?li;3dvyke7e=L7-7;QQ-^VP}?6@4|BRa4iS1eaghO>?g; z)YHVxG5yo{3Wzk!53M1n$|+E~4zYE1v1zf6AVJp*X=pY{_}?aVtp_iEh-!Fqy{`gA z*QiwG*NY;WPO^SKi68#NarYOhiz2RPjImw0lt&VJ!L7%RGq|*SgpcP8XJ2EKZ720~ zu`CF6fGQSTlV2WFs>`j=BqS*<&aGY8{{P9{e{Z(aPWwy#Z@HW4UvW1y^my41|LL)Q z@O3MrfxY@;bpO$G(gDj70`DK=&L!)XCTEuoug#J$ipU?|Kgstu^(m>F23kgW4$spZ zuDBgc05%7&xIL8FXKW$txL~w0dr(#8Ek*kr*2g2ehakOokb3;Xgvs%{m4D8ff;q7+ z%~g#tG@c5pwdpJ~P=8w9Mu#R|wRWJrZ9o(CdX)+_T)7h-Hsx!QqzjbYLR#lH9X;S# zY^be66ryC`3aT~TDFvU4s3OC>9{!SPKw?|cmDik3%7(yTFzzzn0lEdFMqxsM-ROtv zSM0xN;j%T#vbkUul>=tA#V^Og!ywn2>=EQ*u!XG0!ozm;F9yrb&HQl&vs4OF{wqL5 zEc-*?+&IEs6-!krXQYNr^ajJEDaG-fXJ_h6ynFA9HO1GMyfuMkxdhsY`aL}DD}{|B zCAi*flw6ghf^nj_Z{R|iDiWpn#70Y~fb+xT#uAArmjA4UU&Hc(aU#vR%RZcc9__P8 zJa2sr-t!=craXgKyW4>7JqWnElOvxnPMM#l%G+#>;$gA8aT#_KaA5HtDy@6s0o>dqqB&E zRwRd3q?drbRspAkJ)`6N)+de9v0hKd1EzX#5%M<2fGavO8oxUbzAgiepUywy*#&EB zk0GGg;{$m9l>+3y-8=uT0O5SFS>^!)0}}*ub_PRp22*whV-W$H{PV8)QMZs6Kk-NC zqav}u@PiJ5>^FJik6LnO62Pb~F*7T3bzmI-52{*fIbJgQpIY0z6jaQZs-@g6xf~7QA+%*Auo?ga6Y)|K$Qf{%wIF z<-k}&BRwDk6EM1eU!cRPrtlP?j8+OLqy4Kw^M5Yz?^Ds0${G&y$_Ss*c|u`%VBip? z(1n@;w$|h`?OAiAO~7CVWXk#wbo+?6rE){b{=_D~eez{4@Q*FTOmK=MC9+><00MqI z_ND{kSh`B%it)=+&h5+9k3ZdCFCQ}BmTy#hF-XDa!^@$95fkW3`T7zg%`h1QQU@FP zO;Z()9ku)-v7vYdO`v#&(HXi>;B`iy)jP0L;$-P!Yz9sMy?#`Itk@ge0C_64rX%d8 zQB33dB8B;;mNSi_T&u*KRQ1}I-*3Uw78{Qb#lhp_X=}#Jy)(QvtI0kIxvVT|hOzO| zX|lnT8`BN+jVpgV^0GG=%9PW%Mk!f0g!@@@7~V0?1n6;7ECve<);I2HhNT_d#g){~a5p_~j)h2QRLNUZN|;||f) ztXUnI)^5VzCF2Z}^oX44#26NEH0IaUNa3_-9JXz(#;XizrjJ-$wvm#YC{gXhIjF3l zvLiu*{oLxYECa)1Y?TK?Ld#Tz;&;VWlsHf+oRj#4#X)|tMA8VBV>t`Yd!=B|6MqW2 z85Sr}?eGg=Y_s4bpM)> zYOmT4hTxICpt-$VR8`cyJW?U=b?k<_WI0koheduXihA~$6xx)IR@$C zxZ#j0*>*T9@i?V`YP;{q1!8b{119)=e}PNsJJRBE7b?oF#Tu=duLDMh`WO2HD;}Ih z!!T&^i6S{dmBmidYRtNop>Ux&xGQwEBue>h8q;(*;rM>VU`#QAQmeTLw-FRThn!d) zNy&|m+VZ)MsW%TOwzNtlw8pG(_J=LgJ^Wfm;Eo7lX&~zT3du8u&sqmLJSJin$u-W8T0wgI{Kgp z`}hO0Hcd+u_WmRCuG8l?FKuUVM^}^|#P*Obu(;w{udH=b5a6R^2KfigY>vNo+tv>p zW?JFGZiFbLIq-Ij=qKT~5l?;eI4}`!eq!3rzS0sV#`T^P)}O$^==9MEj-Avk@kW_- zEpyWLk;;g}GZv9mS346n$7V(0f3L9z4~Ko|0@{m5fchH|-~=M{-6x9xw&T+hO2~@Juc+9Omod~yPryeQHU|O?NhL2Xi~uc8a#O3B z)^h#b6Tf*6B>*On#q4#`$9T%D+na~ydnVCE2=*@?oHttm2$XR3F2zyJW&1QrKd3<6 zCW{BF%{nUgqd`Xpg#?78GxK!ycKLAp7Jc4(8nKt#n0ZF&EE3rqZF)Oz|a~?*HTngSXifHK)v_>j2uOES= z8d2pgATq6*BdsC*yR3SbvCYR;l05_kClT+G56fJexFvs~WJK1WVuf zJucF`1PUtQ9Sid{q=YM*rTRsnXxTghV|~H3uU(U&c^>nwk=B29Ou#{auM+IiaT0d) zX`^Zz{0drLrQc=q5M=9pD!NAV&|n+gLfkUCf-HWH%FG#<7Obj>gmInT;i*~J zB^48DGzQQjWCR#n{Hyy=%}CE)%EsE!^q&=gQOaiy*ea-BS`v>{tY<=T84FDX$xL=N zsMv*teAt{|%<0YQ4>ijL_C5p5NUY}!S&X)skJ!^n-u^No)b(NG0bl>?D^6pT?x90U~( zCXK^rA+xQqU)b6};M}nU-6oyakqugE@i~nt=j*E)5~g>XSz?qXTd0?I3}+x>@EEW~ zH)thzj-^K1ZR&#$DEaxR)myhYG^CCkR7sSDih;dRDd$UHpM;Hi7S?93 z={8e}oEks4lPKM@T23;?mbcGs(NRP3J7I$^vz#&e4U0O;G*6QZ)TTSG15q`n8yb|% zmDGPacB;qHt`?=T`Sy;LRaCrSM2EfOv19=&fAp8X!GiV1O4OrLjo^XmEYOEEL{mKn z`rDj5vvlR}0~tF~>x&J$&{yw!NOr~GLrCIrIO9U67VAqv^X=Sx_;t@PRF*ZYvbkLg zzKj00=_uPL`2$M$)~S4LfXmw}gZeHxJbeRQcIoQm{{GIm0H^mwGIf?ZDC`%-uP&1 zT2wEZW;+S%gW=i|X{@kRzHttY{iq8MUPuf)+%3muc0%;WV00YJiJHI+eeG5q;;#I zLtk{5-n?$cbQM*4FU`8a#Urg6dxO1H|C^WC7Lur8N=i!dc*H9DJXNcH(?%=Wk;AH| z$d2VFo%@LP?e!KUPA!%Q^UlRVRXenONa^uJmq!gOwe#L7Y;lL;1nu-uiZ+RG8NU$-9oYd+aTAAU^4XZ# zOX63=qUPktiov}wtrthOQO04cq)p0NrM19P>8l<0-Me(d+aI6zP6=Un2DA-7tbe zWSZffaR2BXq(sgO`(rj2dUr*KTdqSQ<|+4&TK*NT`)8ky;Jv0>Xg(l5C}NwmmjKBy zhiGydJ&{r7q}~_*ufy_ZJc7(qfXw|h!&k>p3ee^=r0L{Uac?da~kp+ zngM)5xwaUx%Y0tKtNl^j4jb@d=bHsi&7fA0qgQ*CJQ>iiY*-|2+=yR}XKR+ndn`r% z%(J9c$Tt`dr;QQ0t5!HyQ9ql4ZnXQ0M{9E8kJI|#hE@ZfJ#wd-D1BZ7P7id<`lryyrcVcH9<6(dJ>mTy1WCMZ4 zgQGpbpYHEA@kuJ+0C_v+7q7;x*zNbG2|7O{ zLEvPP08x2)WijrEP%iydHd2~PHJ%AMHIqw2k+vs@&5Y;*;ElMZAz!OK(t`FvkE^UE zvy986s)lVBjuT=vhG-w&ZWs4$&#xB1%jtKg@?qrUE9I)^ak^cD)S6-ell_1 zV?yRU=uz<;^}}iI?%tjEqPsp5!t3nq{uT5bQZ4?R4zBZ*3#!8P3^u*}=fUP?xtYXs zIOJ;21F7rhE;y1mdCdgQ!i1>2gqgTNyC5za0yxMY?4{4{Lb4^fE-V#wFxeQAO^e_&;r?n)tBz3Q>t@Dt(8~E2-86eR3Jn z>Huem%JkeJg*Ll*%P6>^;mJgvHLGI->f-c2+G~C{iOCQydkEa&L9jmP~$RV$Md9w1}D9c_j} zQjN!>#1}!1P!&*21%s3MVKsuf*K5)Pi}d4lJynDv;i)vaW$96egd#tzDG!}%f6x-` zn&N$q14GlBk7`>fE)FX!B@+Q9oj7rBI1zg$`Hq|Uc?(LEiogk_<6V+!lrrh5s5`i@ z^qN1XR4D}7f7dpV$f?n#C`gDu9R6k{MxtG}X=3tZFNnJsQyS!}l5rOx+s(HA_HvIZ z(yKk{w~iW0XHl3I(QJ9>r-n@)ZX`0rU+?b?n8r4frw+JsaB%*ZviM&9tJHPs3YMX)Jk$Doh6QXme<*GBfymKn;?yaQ1|8*FiTA0y-t zqYo$2_GZwD4;?U+s|PBQ+$`BU6;398W|tl2%Tg*7FzY_ zT!!Z6LR&ePM$_UthneYevm$}{rxI5EkTbD>p>KpW(cfIh!a+iv>x^3mF|Hs@KEC5- z5_%!Ut?yi5LpCXq<4f5LlDbS<>tM-LQwT>t(H@d{P(%=PixWw#Y&Rsa#XQ)GLSDOD zRJn{iP%gwgJp;9x=Kh882(H?qSJ#*H)f=Q?nx=Xsq#Gq>TgIV{NUJOWv&frxB%28(khIUHHqA zhkc}O)dewQC%05qt*C~{)o%k0T?si&iPAl2*21G?cE7M5!cAJdUVn`;HWe`+>z{5T z&}Z~IhbPftlmi$~^y?9dp~6d%*x}BNDkrQ|SyhL`Id!6828YC#-h(rV#Mm1RY|?G| zt=sI9$(2oPA{j-UEF{k-DCDkf*J`oq>uVXWd7VIE!VjTw!)V!awXyRgKNYZBC>&2s z40LBfLs{hvm~U_HFRaOMm=ir6n#u1m8c;YiPPi?PhhM^y&^1hwwuHnpxtS}cekvlR z_E|UV`X*rBnF{-zDKk}JZRu^&t^n2UB~9&hsL#>N3{x5ynA^8}3yZ(1b-d%Z$K$wI zg5n0jAw3H&vanEPKFOuayx7#Ia0ky&KE$=HELQPGF*@cWs|RfP2JKppdO*>j+_(x6L}+foqAo) zx|RvdL>3PPeRr>SQc&0yYEIbI-}_4V+25fAC)LjrQG!9egoDsXE#-X~D^6Dj0MEMQUQVORfS>4!sQyrA^bV}}g z1o<#}QTy_QzF_eCMCKo<2#{wrl_=+jpMfk z4l$fNwkx+6*S=W0erRyK?@mFsxU_+SAHb9eM7`5ToxPFCVw&H3yd;EKlUg6fz}PvY zW9*4a?g!5hw<^aNH|SBlobDaPDiItz8IeiFWJP%t$vR_Ek?+28ku9@;bk?>F|s`*NljQsL9IIq}qRKvpNyslscuj z$`J}{CBrJH&JgQQkm;u(4Y*%iK>Rc2aL1GmJ>L|G@849GA&U5maD=>NN>(Z_Oj>5# zx7!KKYBsjp3rmIo+r#XXtTRr)3a^7&lV6t`o)`=H#0GB z&JV~u8IciZpS{=m){h^m*gt;I{lD5w|97SLztOa7{y{l)R?xre=szbOT~T=CjmU|W zK*$=eMWMl<+@M@-Ai)}K=tf41hkaA^@xZkd7QCgQc9L#;^1YGV=h?yTiNF_kl+SZr z=EdGhy#?nyy&6MH*57+SKEFnN&vJdgoofla9(y5w*kGou8UEnta=>eAznA5WGRUjh z=VIW+oO3>Wm+#I;U>uGD{uzPMKNRM(b3H8a@y|eIJo}{8ZAWP9yqE5l9;njOoMi;{ zfH%;eN?Ut4dGYZKg0&ycx_mI}9tmkXY~|jLZEKirqxFEC@+b>*=X$WiWZ&oW@vMWr z8~Q}w9!RFtOp(Im|BXlkROIsk)(7%rY)M?+EBQ=^xMd%obfupr2X?)m<fy#s0X+z<5Nd+PW2;Ctv6|IAfzkRKvN2fG2a z1GSQW^95Dz>%CN8D3_1h1#C)>CLZJiLhbl0)2$Zqlu@ze1aEQSViDk8B|f@7gXDQ> z>X)%%a05)XqpKo-^xCq5+eGz?SUK}>BKT{aFA>SP%ZX-^IYbaKs6}JTE{W}sN1+wu zEzQ3fu~sO*BwCuW<0`+08XC&;Fesjj1$cF3WpHZn*QR$}v=!b`PWG;C>CMB6bmA5? zG<@Wh0%*I@Puya#&9q}O`7StpZ?oqbT*K2}cJWq}&9Zq#t+372tV7y(<2LoG7wiJ8 z>PD1fSY(2_bqx{U)gvw^iZKA-wR9p0dK2jn!Tv-wW$T2Bpj0|BkXJ8Wt@>G7_MxMIVx)}L9$D_YlKAtBBM=>`$rK}wxNIO}#AU^iX9(EW?qjC;NK}U-%eV?HH7p!|dg~>p+K%(miPe;SnVWKqJm(t2KtD zVG316&Q%t`hG_oV(1UH3JdPm7Y|)bbl*^4Ocb?bMHXAm8?}i!CGB^E123#No_%ELd z$mJ)bPYXYpE6e*2Pl|$rpzL~^1|;_rL*$VD!huBl5DgL z@_iFMNPI~RNWB-dG1dD^`#{#@Exj0tq?vn$#+AGpS78h-7fF*yR{79Qf~UM3t{u=iI2?}c zI2qM^8b>){52^N8BM@KrgBtAs?VbU{zqlNgL2OxuSPYb!I#sekl`g3;f&7k&Hp4c$ ziMD1E*1A**4zh(5dnSKTMrwg9m9|hT65XhTL|tluWIIxmegp?E!HAw!UuArKyja<5 z^LeXM%8>v;vO&IDBUmC5-lz!eOeKF9oU=R}wq%`*9h1wR0g)PIkXEefp1M#j(hZ|6 zWQ3k}hKRZqrVnjv*e&4THdy-IQxd={weuWpslAvav9i7z5Iky`ZSWY4|0cR38kp3I zrO|d8(Osjg8%$Z;u6Z4Qh_WCE(4V!iMR59qv3VMUoR)}7Wox9YOW}r6d~Hs2Z$iF9 zRd4C2X+Y8z?|`_GqL9Nyr>zZQyKLiu+0ybDAxU(02MyUc?5vwv?r)Mt_t&gzbP{FN z*D6sPR!TEbBrtp2%$|BsJAXyHT)5!QDb57Z-wfjg?EYufhD6Idl=tdPaQzLe80{Mt zS1!q`w8GtPt?xj1|LT6nwFw=^i!d9yr0TUu2yK9KlFB?2Y)QhGHXvm&k=actp_xrBl662B?PiLO-59MO?# zkh&w0nu1_g)gHjn8H3cvmEI8Ju6OGZN3;io<(!RP*~KHD_rS=a63a*6AAkaRXw}K*=e6&M!P6O zIUq@f)Ja2E&)kj#ih@Fg2u?mpHyr`mNwYR{>RH_qn5-PeoR(R%L24yKAvWY9R5m+Q zcB@xjJQZt^14|;bz#fF)7drY0#-JJfp>^GWDK%57E3YLuqq_YXsX;Y!bj+Mm3Ym!- z$qshyDnImIwd`%_2CD?Uke}FWSL@Tbz-7+M5f0`GQt^_~7~31GQ*lZCmeG4V z_UyRT?Sn}}^V}3sABZb=4q@3wCw*kvNt-}*9&zGK2YC`L3a%pGR52$Iu&C;7myv+@ z5Gb)FSL#{Hkq1)K6qpk7Fo$?n(HY#wHk71pIEu z;eiq4`T8&qwi>`qu;dGU=|GBrbVAibU&Qm5OI|3uIrs7&u3U7+z&$?Os?&SZOz|X3 zsO89xlcJ&99->7~I;rUc?Xq+qr#Gsdlc*v2Jb?LBf)+4L`>$~@A;YV+y17(Nm=S7Y zl|G0<_=khOWC|^H2i`HQBw&9J{#tLwO&#ScRoI9otIX z-lRXVMdpBt(s~V+U0(9}600|D3{myDnbV}H5)w~DVXg3s&+`&(_Gu@`^_X_-kEVd~ zjKQGFJzU;sD6*gBCTGkO51!o^8pC~QvRGyS{Z z%U`)i2lDqdl~a=w%7U$1pFrM&!cSw^_NI|VBWHD3dw}J@p)F!l|0?T!!?ad2w99fN zf>^t?LccT44XMcrT`N@f;Ho8%$7%;~yQyVU1;RJKdSmXeK*vxk95#5x17_w3&Ft7Y zftm)h9UAlHlF^TSeO>OXQmJp(aE+8Mbat!~*SN0tsrNkKjcXvxA`S0i)m@ zAKeRvc!aF0l?X5 zv-#q}xw#$d#$M?T;~hqs=S7!wuJ(nRe)A~e#w;#B;K-wHB6DA<3oUQdMsy^<--xZt z7xcSsJzy;!^lqe6su@h7$SfFzhcl+P*UbwGzD-~pyqh3rJ;plf+U0Ovm%6sOU{Fnt zXnt8hqZUZ`U&FiqD0>$XPgQOae*9=8`)|yf|Fi5B{)aO8FAe$s5yjD?1?z#klJXr# z?|u9b%;|P03MTwp@OoVw3<{B}fU^|@rqzayXS9GVLlRXYf{r>fAxn_j_b2_o)x1ny z_m49`3T~vmF&YAp!RB z_or=+AMqDoPsyP=Z^0n~p3*`bGq{1%(G-m?FcQwd^{AZAPYxq5`GA@uH}#Ae{m4T+ zv~KDi9vxd2>Y_N18r{{SHBz7y-A6^NCwo5V42{qcBm76t)4K_0Z|JS+0km5!hVbxTqGoR3p2kA7fZ{_Ku*m8Q zOF-#i42XL0UDkL}M)M8l&oE>#exD|6MEIsL`1UVh6Bk`zXl@4>V(}yc-lyCE{f}60 zAM(LL^NrxoX#ZC+#(&uQ@l|03X#N%D^IZk^xAbuDfw1w9zz&cR-THoy~^~%;bJcvhv+GyPJ=^eWm@;51yzSU3q($&JER%vk?46^#pSX}6O%o5f{neY2sX$9qkm z)W&vUNS(wretvOYv-oUxytY1#G8EPEj@N0mo+j}Z(-E%(6o6BBxky5#27xA-mooWi zy!-+BlCHK?n4g~+4L|ZqoqNPXE|uTWctLtNn&mC|AZC=Fv_%IIx0FNCA)2*85EFgA zTNsChE(-h(@LC4Hbh(&Rv-XW(MUTT${RMW=7>cr{hNChlQr2_v7ZPJot6Dz+ozZm2 z$w7?1qD(f3I_QJRz(UR=4taOYI$f&QE+_-pjXgURt%*kmpBW17yn++eOFl(%}xn(STs`h%uLporZCoSV6d?5o1$0*engSLg%R;u2?9&XGAa?OE} zzw*G4w@``J*2Ksf97N1`Zy{L#=SxSphMmmRh6t3s8e zM2X|YftL%Ob>W=iQjMxuiIBhKAcCNNOYA!gc8JOZ5)LXRRexVk#TP&BYCc`yo zw!`)qn=DfWcL#(W`JNIhFJn3sjwYJ{muhr3TCDC3Z3cyUv zeRfOssaA@~rV)m2J^Cqx6gSA!VKLBcd~xXWLm7|m{n>YPpo5wzjPq$Da2#@(d%W_W@W`A)D)%5&Z9K2?^JfqY-N=WEVvr{ zltS(yPk0TLrUF08oO4_{b$R0!>O~9}q{H)}@V%*egJ1{FDf&>5ppDRK*93Z|Otu}r zf~qb=b2i;jYTw3+>ZhZjTJp}wW3?U;by`22%4?1@??JaRLvqcJ(~sF{mb%xuZIr7C zEvhi5-WDpD^D^#1w_BSi&Go`-21ERk%rMs57^_GxQrwh6qO;_m8TzT<^BsKG)Dt7S z)uN7(*R)_ZZhi&uVr9mMC(f^b8Eh1M4aw(p8#nq&y4Lzg31v3I%p#i1)=I`U2%S>L z@tV}A!@98zqA0q-PiUNLI5v8sTP4Y8#?Dzgo^7qeGH8r-*b!_ZMx$-^#;BL}<>$x4 zgvMw{+8EO!5hE2KcG+)jmsFJa7y?cXLzJAh>jRKN+U?dSsmyW)2ni!fQ63CMEEx18 zIjPncv|08!m*&2umvUK17UM1(HrV_;Bd`o{4%qe-MokV)b~Oj>s2c+LdG?fHZT7}+Aw zQ8smxU4no!Q<=EY8Qc=D?ek+K*tS13??MzBc6A08I!f_M&DG3iIug{+*~Qe7+w3{n z)~B-Lp7~-_r+gR8U0*!`o`dld^>#E|8yw2K5XQD z5SYzaHxZgP6dS2m3E^@ZIn$$>mJPCh9h3)m#mE=4mM=OUM_V(|Ne1*%n?3(A_-yrkhh7KWC?uHM2pv!JtTmw3hd@fSG!wf= z2CT9_CZ+i@*bb4QP043Gz7>!`q}=0<=UZn%1g*Wjy0-olC?Zd`fX~R(`O65@%|`q! zd*q1_+9m_@bT+WpVM4oy{ESovlEZ$R2iF@7?C2gHcl%%e#dumuDt?CNLSwXx(DF{t z%L(haa&{!Non&kbDHtVGK#bBbf1`8O-G}P05ybQ^6)=-@OSz|^7rITdp1qzU2`J!1 zZMizFFy6*+Ye z)=uSDM`gz47k*hdt3aGc*lv~p!EWakc1L)2zP1J6*M6zkj)2hw(PLBbr}3#Tck(gp zaz^%ZD~Lg;6}8t&>8oN_x_7$iYiU)zC!3~B_kJ){W~Hi`)sL6S9#%`y5rk~=Mr6XJ zRzFeAU@sHk2k8JAO_fB0cGjb~Q+LS)e7CH4e?w|{V8kqETl8U%HHwrMl#-U`ivQUM zOC>y8QqTTPxy};O1kH~rXLN#GUVYD%#F^%Ue(Z=p(6OWm7$-YJ9&PXfIcriUSVLo= z5`lGYzXw~DJt$-W(r9wUS%YAnm93=;N#a7OSR9km@TI6|N??tEKdy>$2AJg?DzR8( z43n$eqfj_rtWOX&QPEdleUxdz7?{d0QtML4jw&9gjH`Hx?sw=S%k97 z?1rTO(s%Vhu-}&(fXr)cAohVk+`xF$`o8`N_RGGG$q40~(9eIIR*;b7KB4TD*X?kH4Mqt<3BDobtpluq%;A&^~@vBvwXa4O# zBNT44Ryo~|bxFgs66@lXWtsLeM-4G{a0(W$-M3Q07yBibCXO4tFTbN?8bbIOVf*ZrYAlPwF-YNnd0)w@&CKD5O5JRa+p}l!Xg1#(Wq{fx%@Bt# zh_^&`zqHJk!H~SvXT)*T3X1WNeDn$ZEB@~#&1N>Pqrvc2m#z#AB(SchuHUTF%(gV|1*vFbG_YwMv`BONP^r|x85?4B zz173c0S#aS-;@-!&S}Ok&p$$CkC|F)ywq;$fOlt8>w(aQkea4&s8=AA_9uNf7Wx|P zs(OWW9nkMRD8|0Muk?rlJu*Exk@uJzJ{_I&mB9HU&lSqsbFqFR+40(qpQ$UGPk}vm zApl(-5`lidKa!Ove#RKjJ97-#bwgr&<{I9#{;nOL`pHa*;&0UC$TTy1iL@U-QE!TK z$0H-Omq0eLNv;D#q_bSDSO_Qt}!YPS2VKrG#f2u zhE*9%NB5)!a*_)SA1%qoXu^JK|5d6x%y9A)i=&q_D-h=f4m4XQjEBYJj+m;JxqUKk zpwV)hYrNpucW(dBhAr@a^Md@x^%-tYbujgxFxd|BztO(_&+GGlmtv`=qp&Q9_6-5! zt;XPld{-{%j!g5>J`9FAe@SwVg5pEmv}35UXuIY{2HelPmjGUqCy1i@zDZy!bgk3t zVPGBw`KHfibDi<*aXFt($K@IH{!Ab1K|^Uz=$}C=b)c0Sa>3zKT#_V76XXxT4oztYtcuR_W-EuYj)D$~|r+#xh&k zuBc8aR3UGQ#stq$Q3y!bGLdAw!!0D18+*n2z?9uee#7=s^S9r208c9%r;pO0Q%Y?v zht*-}R+H)SiL=INv)0z?EY+jMbJTs2eY}F1X)bY~78qQ5M#`U0FzGm7XICQQ2k&v_ zpR~<}f!;R5;x)kwWV$Z*r@(OBBv_N8({_V1cHE0=*K0YK-F4wlq7pVQaUE+A1r%|iRhIkvPN#4$3XZI1E_E@puX;^tpRICJhqolMC32Q1{)mk`csQxXBv^rXz z)cqH9IC9I925Ypr)}5BR>J|LgzHW4j{g9A?Od^o~s@KpV1)Ra#h&tj~X0 zdC#klssJX0aHa49@7qw^r@5=IMQ2ydw`Y{j;A(o~u9dZRfe=^T@%m8=qV!^nGg`_b zbtj&fi*#A%$8MQg;0mFYm|d_!^yiPlJJc(GI=LeT=yNNuOGPp84|J-3*g@U-21(EE zoFc;f8{c2(Ymr|p=b-UMh%w0*BIm>(pQFC-pwEzzk%7)w#0U04N{P{iycAq<5$VSu z45MUe4qOx+fmO^C09kQzw2c?iJq^K3D395G%W;8l9~=C&mT2>loU_O zU}6=jZ*dz?-`ebHwa}|1X%Q2n`H42tsnNA=Rn_uuwYIt~sCR8#`p^2_&2(qHQFua^ z?7a<*`M&MA&vux1{p%0i-Y<8(@j?BF0y-0L_RrD=dTVA43KKXqxOr(w%y@;^-v2I) zu_-PUG0ft%VddEiV#Gwc3|YQqS=x*-T~VdaZn2EY7f^N^olxA zUzO-*4vbn_^9!*h1%maE@iFC7X7Z$(`$dF~9#%vshwHM}#vV>C-^LZrHe3HhBuwl;eev^LUJ!$3dT!6m(Q&THH zshK+7YxGy^%O2?&`|@{sK%LSPkCmUo%oW)*Vfcf9@|{BQvH)fi$9mTCSIF@nT52Hu zWsjTz`SxDXPfwu2#VDbikNgJG% z9GDPg%eI7~9`FN7C;~xydf?V6AShp4UvZ1*K`XukmOmWserK8}qO@~%^Xc-$N4~B? z6izI|K!_DsxYK3hNS4FlIP=%Ri5~j4Y^{jW;6Sgrkr%b-9jI!?xJnpT21k|_S~a#J zLvoM}{`q5NG-5@@wZZ0}wHXt<9K1R5t2U(D=MH|X)B5V@K_7sZ1SF^UQY71s%8ubE zU=RC5w$OSF&qx(WRqjf_%pxW+XlKERDphAaf6XKkt`KEg?HMF-B+FH_E(*_G$BE6a z|5l{QR{mVpq``eXtffZ=yS<7Uvl4-S(3FiSE__&r`TO+(w+;WmkUV&_g{<^xiDk>o z9|%IZcshifmi7zXH4Fz^vf>fHhWh#K!+~ZosJQ!y;}hLa`EqrMH*-T2<80h&rA9$`vKvLL&Td|}d zoof@;A(iJqi5>Lg>gmU-wKRnwoAFO!yIRVi_1}w-B%8lT#eztwU8qO`VHIY`|KkC< zYxdei@a^G7dI!TWwEIj=7m8EYsga}~V#DS>zhvrahemn1ZI)fumvIrx*v3nF<2m)i z;3Xu%RP7{6ELlZ7mVSn58FDXEsi((M|2yX@91H|* zzcI{CAi=7gh8up~;OE+cx}2PQaLiALthRtp6QHQ5v~aRDhnf7QAb(~9UCMcmr6FzD z_Je((<|)VRgYS5G9nh9&c>fnlPfdmR;wZHpn3Q0FHn!|Qm_d>^TIeXN&Sbho1Rxlt zXma2ETJO-IFpNuEZ*VXwWA37~-28b%wnv7VP~7mg5wMIpo^Il`*0i)_2CW(?2+wDc z0(Iid4eM6LRP&o1upKgyxwkL8b#}d&oP86DG>b(ry~=^yLY1*mb!}qDX0qx|&67@X z;UMMfPjcG=KaSuQL!6)5we&nn;bNsvJ-qQN}8VyCWR7??kUe<3#? zhlg8OK>(ek>VX~F_L?jDrOf;6B6i^rWc7CRPD#uP8VSFaxe-4GJyG!I*RF`#$3B44 zn|m7p5Bemjn>5gkQ@Zw-O)I_eSOJ;0)87!TVi8FJO`(j{zsfR-0SFZ<$Sefr{A&tj z$6TM@GS%*70;?sr{V@X{YLi0PReQuLr!b&i#L8^c;!FBzlAu-_W3!3k68uJLX-H?f539WAVTrSD3~iI;xeM^GkAatXw|D1K0&eD z(DT;btAOPVnA7;A?l-;T4V*oC!U(82Fy=9Okk_GfXRbkcz!6~j{N$~?*B-!p6bocL zErae2R+N`e^Z>~nwqeKj6g~rAGzBYcQtel{Z{D8*Vh83;q`~$EDr%6>SO;)``9Sv4 zYO=!8_@wMty=%b;=-hMb+e=z8#6#`&Y498A&?0oqv*RN9<)cH)W_S{9zcFx(tR#q!v5I<`_H){*uXH4 zuHTpkWr1}^SK!@X0+{zKU&{To_m%s&K=I2z+V=>(QvKHVJOdn0PvG4M{zzXSeg(e; z2J*n~jxnJ!z?oA*OdE#3V1PVTAOP&1dZ7Dj3Fa?ckB@r2 z^f%W)rB=nDfC8xjQgx`mQoVL&UFxaMcwNb<%(9QxK^OET_dx3X6}Zl1AIUoQ7lf4^ znhkhgL}nJa&P*Q-kiNO!?j$B~PrQfk%Pr&%{mZ(U^LF0=^9z^kFTY>ezPr+$U9Z8t z_kR99B#;BVFOAT}YS!iV(0sbGdoh*Zz0d$6a>LMZoy54o9qN~#pX$N*bkJA&KH`Ht z^_QV;?HBR@;Yr(3ZCXzY%r};QHpVe(pn%jAa7*-xn&n_H0fK8F?{p9Os|yXPSHh$> zPANoS$e##^_-+YD^Q1w*tRBW3AZHIM^0SR%`I`ws2KByKHfKV*f<#5S$mqD!FPXGy z6aCvpKHj03*|2hBZ3Fl2wiH!IY}GFw@BD5Iu6dO(K9Bi%uKqLHU=3S)xIeHt6T+zR z<1(rQbvD)YrsissXUpddD00)wZa`YP_ii>tA=pK!hfijd9+J3L z$$Z$u;?LhejEe!Sx<$Px&%X(Ke~)7W?bh110xsfbP@gjZq9bxVH1=*YtUU(3cc!xtW172gK8N^i<(gLBb0p>l2Tj*xi7S9#V<{x??-Z-|8%*$2q`CnV& zgB_$Nr6*HWoTfY_;q*Ee_Ze!VtPD(_0=Ib5Nv(Ut=KXv`*`tJ|TG z$v1FNZZL)KCN7h-K1}9A)mYM1%uvoIx9`I^QfO)L1T4)1lGx#4I$hA7{z+esRjYdR z_;KWIVMs|I-IwSWu}nl6r*i~L0MaEFBR!~`3aCMWs=63Z3j1_^mI5^&9w22}K|J9X zgS(t$2Y$vndxlK<25#8nOFalzKg`%>XrsL@$D;nCUy9Lg812$$;|Kng{c)si+i>(QTUNGcvDik6Cj}1q zu`J3~SaKf%-NM+L!ys%rD;7i?gH7`{M(!*Q-Zj~fD~lti$x;Ok801lGSZ@;Ss`MxN z*Y^Yxyi%uiy(wPObM%1nD9$R}hg$tH>Jyo+ajxV#XU|1jbbC$L2c$Ar8v4hfdwOg~ zVrBRH15a?Cnc(IQw)~WIi&HIFnFq&A8y`bH-Q%r&_C=Ab8im)@zv!7sTkL_W7g+V1 zTKmQb@+bOTQDfe#%3IYbi{2?T6+iRipGiG2X0gqNF_fMP9Q;pmdkd7FS<_ffsv;JX z30hlhlwND;2bPMU21OV1v+|qd&bgm$6|DweT-%pj!({eUwDpVS?7rjNJr`c|GAkE6 z%_{Uo@2Y$NOcrNyb=$wxgeA(L+dgxq-;RY7mW~6RNefe|?@Ob8S2Yre^0b^N@(YIC ze{?vwDhvoEZj!Yn9A938lZ0F$|M-wm5GHlEUR5YoRpqy3Fk&^k_AOgRCgNs@W+BF9 z4ai4}xkzLB<{(!hy+9l$$iy4)@Rz>2iYI+*1_c+PWGl@MYDP^mN4jl!H9aip+;L); z$bq*yhkX_l4dE8QsIEErrBE{dlERQf*`$@$(e|}YWH6b=`!z z_+kCNNH8or!q+tG_|8a?GqG zi$4{8xf%Jf`KO@wLV!54oS}Ci-hvHd{%{0q4_66Vv2!IGz@xe#t+0aQAF}-qi=a#i z{P2Utg6ISPVpi40kgO*8VNWRcrE(s@xlQ{r9*MF=qw#-NNaRNk*`?E%XOzyc=;ei2 zq}?IzQ)bpCL%|JBj1}ik?ZahDW;G*GmjAXtYE~FnB1N_6tPw1`o}$E9}tDEBr`@f4RF@KD+cuGcDJUT1-(3VlqcM zJ#A!9x^;Cu<}r+R>8yHcVpnKX^1_mbKWqx;C;Z7M(U;S=Ju|uOonKr!%?FhN`Gc|= zL0uhD7@pg1<~tv)QXSRGFhIx`xzlpMP!o?N2tX?9np&f)h(K%+20pG1<<@VxzDJzb zA^i2jP%0dqDiAl1Evbx%R5Yw>J0AQ*6U7AJEx4R)?Q?71;q_kNLpeCdt$Kq}WtA61 zn8Q?s`a;NKa^pg4{jbjAeq6C|RZhR`zm(Cyg{WEQ2o{o&4sAULDNn)B`@PBw|9SJ6 zF}1INC*3xUBcLCg7c1P!iYNV6ca(N32Fwt-m~EDRL;2E`=v7W*;4O9uGwf&z(R{p} zPgE05Nv=lGRw|^i(VxIjgLTf2nm_H65E6>Zc^_xN{sLrvMT(@C7xG-%o()1{{BfZ;9}MK$bL zG;Gt}5VE@jd?w42T88+*Kdb%i}%GmrI| z83gU|EGz8nU|i^j2}F~StZ-m2&h%JQT_peeDpz2U$OX=AqvRVUuqoCjg+VV3^a3Ym zL}zR;q}YM@efs1MK5X#4`ruU#I7NlmRI!U>cbX%Tahr3*v6FSL5w@7)Tqf4pwr`97 zF0R)(6`F|9m&3EO1AzJt`z2S8Q?#TlxBFwX^3tUmNB9tWl2s&17ZG0h*ekrS9DZG# zo}>@lf9a_Q;MLoY5 z@W5F7jPUgdb{l=y?~ezwVTAsu=V6Z))wi;7=#JO+#Q5!w&53r)J9Su=7jZe>^>V_@ zG;7if>&wXlQV2Ap7^|eULEdmLFC|A*gdTpTinryQ8uAg z|CGJ!fv=Gph`5d9{KpL>t5~N@Y>pG>v7+|0osLZ_uTqIJM1ay=bc`vXo}Tv`LeAVM=kzpbfh@o3@k| z^^ED0_~6uU{16pam=7VXxl?Xr5cnksj{zU3placY7uoT84Sx1t8U{=D!kKD z#+6%b$fP?Lr4QiTP9>>)mSYDWkr^lAv?i*5(Mk-CtGS`6$D}7z)tIO^HHi<7k2zB_ zxw(D7xPKg5;U}&apN*D@>_XOU8pAEDHok z>@UI1<)^H_!+~Z{l(^u1)S`Fo>L>BHVQgrIc2jw#cmk+18RHpmuCTc#Uqnmrh&}Ni)m!G5gh;2 zC>|~Cw!QQ>gyCGni5O1oSoBc>=LnFFmS0`n1ul}=njs*fJT!tyjX!O`>%89_QkN?4 za~}$CkKgYacp8b{9fm^wktd+8r>*v2XkC1EL>)r2bMj;BUQT$D-oSIOzb3 z*njhQ_}PiK1hHtkuIEK=7A7n7bbRFf63tCX=w9PgQ=9Gwn|x(~j$#@2T&>68QmdhN z!f{2`Tbc*rB$=$k*P8+Ex@%S-v7ntrqT>8`ALx|R6t*9`?VOPHUDh0ny;#CM)>5ha zcB}IX;brnA5|(>!&9TMLn1kx%HtKbv<-Ysa#Q>vBEpaBhg7XiDqu$<6hlv(P_KQ!b z?TP*tC+^p~PViIg%jFm4MD~fD^u;?P&J)<=Q^J5haLpwn>mBP}_Z!1yjn13Kv)HrL)#rZP#^3;LT zb#}(qf9&t)Y>$*(Vpi+<_cvr;w)35*&)rkb9naZoh`;|Q^$*SQJAe?kP9}8N1`Ew> z{-G?*HA+=+Cryx=iXLf0lk6|$(OkI^E>+HQP5BWRnrxIRfKV;Vq?i~!`ip9|jFe{E3(m#@W~nNcTg zWVI=l?$j7lxk1P%hb=z}0!WU*)L_?|5Sb6%5L(VCtH7uFtlSV0*(=+cp_e?tV5Vbn zPcGVp=i2bC@!Qf8sOmLSL(a93msfaX z-C}bcc3jop==`rEFdxhT-*hPa@Mc^RL~OQwT_paq)?{c?_lTn+)%PtTh!+U5v^;UYblUtG(PyH-$TzIY!x5a5%|ml zCpOeAyfP#1sb`{4Q=HfTVOov zcXEp2bzvKXQ1d*+x=LROF21TpcZ*Yb7XKzrfAujwS-tZj5zB$`k;coPn1O$hErE5- zinRrnhF+IU14V3Waz{?kc7)!Ek%pJR^vFKb`o5$GACdzmBUvIVDO~`I5 z(=3Zy3p;@Gh-c@3NWKy&Ido9(0OF3@e*Vh-DYRY+dkyu~HI>Vk7$o81wdT!xkWZ+% zjk>@)0>Wm#tG0UMY6A(pjV!hZc5AIQ#ZW|}-{90OjL*kjR zqlktZ@8Hy>#qJU851Eq-Zx+gTcf*9e@$_eIl z6Gu*Emy8BlwY=_K+}5y!^%g$EVcrE4Y^=W1(S`b4hkR*rxQme!tHD(oHfh~ExJUXY z7fiC)3ODQG;e0rW-1LE138FJF?kzrR-<;livjD4DvQQiPM#(C6%Z(R#Fj%km2L$g|J7@?zJj61CH3t z8WxWsINiCg-2Rk}nGvy*YgZ!;aHpCsva$-7T027a9Gg7!Fc7fH3ZyI+NLszC4DzcP zDCVMGNf;OMIJWAo(p96A62$HL8S}Lvjf#kGRL>~#IUa@OX3?!t)h7(X!)=gp*BIhmc z#+#rYt;Cjtqm6P{gvvvCb%FG3lJl&wq|QsGuX9IZ@Hk?ou9r>D7Vh9V8Q-#CWNV_h zMO#p1;ANT=eK@31!q+~#yej@kUs*k{-P+gGSi2DeduO(kxPVzur_BT;M{7`7UqLIM zhnb}}=^KY4_;ina7wy^T}VAfcm~_*W~QCfTwJ9- zM8Mt+b~oV!nVY=Vqv?g*OFQ85An1_M4Pfd1!N~Fh+wX_Kum^=%`s=UxrVKFBp}2i= zY(E&rse71is-e04GnQ_czW4(`AQv$)Rnmd0he)Ua@~Oel6U$d9#UG5^wDmqyw`|a$ zr02G~TDxtJ9V?$W;)t!amer>2&c;dr#oGHhV!V(#l_Q1o?9#kz85>>++ zku#}+#1@wIHjQ(Hxg@X3DMhXrl1WC~(UB>CfccaSF^|?E7t2@HnKZA$>AO$iFsz9_ z6yc-~3WxD@g?RuCLWaVrii!Sk6e%|ncVws!N*>LVrBCOy`F#cp5BSTJ!eC&;A)E3W zWO%q*e;7(0`O}6^?C{y-_TbqZv<&%c={=1pZ2o8#7hY@1DsVuMX%Z2+MKiRIn8*V&cA`65 zW84QiKzH&?$_Lw!TjbQ-scV`RS`WQ0)dL>z@8wPJTqy4IQ%)Im`F50bDeY0(mnQmi zb19ynK4i`uGxlWlQ65Rt8joffb%)LlJC;FI@LKc^x~Em1HPdVLY!P{sPfzb@rtC>z zyOWIZ8383z+e2&8-EyZl+q6w^sZwI8(`(3uFFpm+JWylP3rVOMpOVIylRlt{Qxx^W z8SW?odfRkO|MYWr7BO;>mg59coWY`wNHY&lKg8WTt0FN<8Lx-Zu~2REARP}cg+Omk zamfkD27*n1VBf@!#ie1@EtoS|_J_VKd(#|7MlnV`;pv^}F3lkx-NBJE742{hMIpKj zPqe_4XP1=ew#D|UhO``*JIODc3fTeSeWGR^$Hn^j$GH*mGwLRZo=c zryb4*x(8&*yl|+WLf~EG5UR5>${xd$ACRdrUF^Ja1BXqfyMYvIU6h!f;*BzXnJPl* zdPhoWu~~IX_j%Z)Q=j9NT`U^RQUualJncY<`P+X)8 zl9@nA0$28=37ZaAggwWq6@@3lhY{_u`i&-w(o~dM6oZ37$*BQ=o{(IWIdELbomN#^ zxuA4rWm{R-%CXwqX^n%6U)!`}kvqHag9t?ehI$N@m7C0n9W}!hVw2ZI>)r9aY*Bhl zvtT_wBJ~&<%xqQjkOIc#h>2><f{tQ;zX5d{ll1EbY-V zOaM)NRn0Q+PYh>3L>#015w4`Qpz3LPA+8Xc%8A;0 z<$WDB4V^0sn-&hhDs>#zU6~P&AB#~_42G5{0kxTgN<}T#)b6n;+LP;s3;>lGUUO7( z`7GW}`q_(ttpwuqe^B<0F``7>n{Vs1ZQHhO+kM)$ZQHhO+qP}n=9!-P&)nR1GP#qR z)Q75M?+=xgwRhEC&-y*F$KcYLL$gxHq2Ww_wl&UJf%~V&Svd~ug0|T}d$9|RrMUu# zSpL}1XxX19njA_zHnox*Ef{pFRQGs zX&TqIs&QJ*`DdvZ+FDtZAymg~c$msqM_1N^wgy6PiHk&y>@6W_6vZEhS~2_N3}

Y%c#v)K0$_yDbvn#IIC>BM)6F-}w3`?v#f;mvi+HS{pDG8J{ooa2_8_8Ef7FGme zqsmrAMD~1U8;;!!vxfOHFHvEMO!dzawmE8R( ztCoqd($UVb6Vs?F6NBZmfLH;xujUUf6eHZizcey_D?X=XjUn z9x;JO*Ai!)n6V>lmFGYhz`5Zp%`PB1wjy@%h*mH)A&Zd2Zo}W*_oH$#>DEKm3Kthg z8pvSKv@3LcXfZU+Z0#mHbEcvWE8ItF=gJdNC>vuuze?}N;Vfm-Eu&lFf>Rrp;hgxU z!&kd?Yd5+ooKHDpPJGan$-F_pte~mQpE<8;QmYP&IiNG*;0O_0P{?__Ag4@qDZrIj6;5@)cE_zL3H(l!Q%CfFey$(JbBNW-d_cvi4! zT2%~iJz-JP&Xi?M*SwZR9G!_w8mG*Q4_NaAvJQ)dBBr#_rWLlOKTpdpsH~_V zmZi1I%fB-zs-Yb!>y^l2iFnJ_St!D^qJ|kfTmo<5ytY^4e)*kbOYwHAsCaY3r_`0A zTFJTDNf?|N(K;&Z-Q7E~P0-bPSZc%V=Sd2N3At!$3xL<3(9yW`e-%>-w1Onw6j!AQ z7Lc#eI}@AM7A|a>*unrP=@R-sXdhZC=UW{PsRt;JHMc5UvrEh)a%#ytAi`~>7giRJ zy*nHr@~xVhIB|ZQ{ev#^%NSJ6cm$hJa7IX{q&?44n8g}(Dzcny4D=QMUM*o_`Fnx8 zQ^Xa#g2gx^l#b@f@&U@YmvDNpmxrefRUD#122o-MuQgVdOfVzwl4}!}$8U zR69U7l(q_jE&B>O@7_DKTMNP~+X@aeYpcxNTl{(@;lvzqYl>gQwohZ_i{-}>i0}`% zl9B_Uia502XHPO&b>Jm1pV*oYT?$sf1M1}#telw5kk7~<5CW^=B~ZF>WEeE6^c6zc zXmJVqYJ51S>pM=@U=0qboIDJMu1FwJvPwvgGz(Pq0R$a)q&-y6LFy(6SYDV}FY{g^ zE5XFM72WD4Q2wt05oJ}-1w@)1cu5QX_hN`1Cs-C|VX$VX9%-Pr-2z~m$Pzc;dYB1o zpg>xo)2uQ5FTwnT!7dqA03z_39s{om8pVQ&Y--(P5I+eP+c)xL?h}6Mlpf%{Ca^9q z2{WuqK%Kn{ExA5;+7v1udicvTpAnSz>l<)KVTdMZw!USdi1{eo4I?NKVg{dDuuc}l zQU(M{9^izy&NnLHdLV=Mz0tCt7f>`h5acj|q91s?3k8uN&~C#e zEfOTF2d5zjM?LmR#OAd>HEh?rE{zmM@SVQrjz&s$!L}rKnnSrX4p)8 z?U$0`SD*=EvKTQ#;vpG&$T20TKNG-3q!KcAEHo^#6_QX_0!eM?3H;~@EM%mgXZvQ|~ z*SZGFe+;i|Z$_)KMoz+N<(UBeXCbbRDIXF+rGLEuR2^`JxYHdiraY)S%t%5VZYX6f zDNsJTvMP)+c`B=kPi7s$Wa+QWd|;eXkkgHyySjHBunFX!oMQ4jD;0lQO|+0eJK1x) zB$2ES!AxdA>5<*?rx@V))Da$aHk_bjWi}#Tuy=mER>verJ8 zf*nYnWff2?SdH7Uk0+gakuQV0}5xA=_yRVJ9ZW+RYFrW9`PVR%zC!f^wIN4 zKCq@83$%r(uM#8j^kd0JG9 zS#c_(&q2`!D8XM)*maZv?xGQ)lH{yB3z*x$FZ`vxaBxajRtuC`a#pty zKRDIB<0D^e4Nhc%e58`v6PvY!=Bln(tfL%ZgZlS*z)xkrRr>a2%U%;<6>hPQx^1^V zfE=O?5FBo_C{i7xkL5}g68#B@<)sG3QdL0M*BujQBR`Q+;L-EbS;cL3*YWOG`pr zCiI4~L1!AvFe>W4^~+f&n$4BS#fVN6mS_Fin)R@OA>QB<`*{+UIQN=%)B*(?<$*-3 z_5Dbi*Fvxtan9#nisAB~0I$x$|NICHu1N4)sMGR%E@Dk~*5;|)OKGd{>C1rkxz|yI=gW-SM$=;$X5x&M{VLy{ji3u(BUM`EmOc?S ze~5|3-r^p zxIt1|vHhq*Q1;I^=aW5LoQw-_$_sKbjHJ_Yl+dyn!1b}Lo2suw#H3B0yTZdcpI=O8%bh?ROG1{v?&z*}07 zH(~phh3sTBCmK2ZlIIeRU?0|nRaPCB2jkfcIz*#ZNy|JX`KSk0FGKb`LYyuxc;+P_ zx|vp!uIWPu%nm3`xuzDlHZlF|(+$A8;dRg3?scoPq;CSTK=(xv(oWN1gzS_si&iQW z<|`DIDL-f(JLW7HNNr*xjUE^j7C##aU)`jaVhyQZs=!;RhT5MHr-CrkSkUql|C?;X#{QBuTQ-&R8gzhen}u z@(idH)S5;*5d)1(B7j;y;8F6H%VXYTg)HVhlP++Aq*A2#jNZN_tqeGEP zFQhKyN>@ibfbm5NrrR3Xhku&K0d$wBPlD~!ZYH5w(`(b~HJcV|?CM+>J;j&sjDHrV z2zL2%)>eb;IgQfh3BbEcb0M4)p^_7_r2Sko{}O|AG>LgUhG}?8RpaI&8J?}U8PCJ{ zqca3EHfPiZPK)5<4Yd^zq!R$@D1|7bM06P%A>}m;oMv<^J(}-Jl<4Ve|}M~P~8EJcA4gXW0UXuq2K@SUVI1; zG1K262csKZGt|2&hLQ!U6wfMxW;&*ogeRoxpdt*~hYpbk0r;FFOl5%HBYX{V2EI83 zGOxE@AvWViskWFlOb-o~6fXP>4}imC22m&=^R$~JV{lxXKDa`lO^!?FFUN>jz&Smx zLp(3u0xM2AhycYhiaZ31_#>g&Kor`YYfjlXu4%s*XQ)BC=hDskJX0g1QIJzgss50M zQ7b#)FfGIgflr@qOquyZRvE00g7-R+rY>r*5`6gQ%_>ATQO}NUaId;qUC*$=L8r;i!?!oxE2eZ_JUpm94!wx8TmEBu9M;dCi|^qWvRJ0cJC3zeJ9e1~cK5l)TB8d$KvXOi- zr@>8z+AymGm4687glYQKL~v#gy2vAURa~pUdcjc{`aU-pMKU`hdO;7Dor>M}-aHMB zWakY9lzm?>&uWhxGcN9Y2b52}2uduL(<<&GnN&n#5pelBSw{^jSXC=L&S(Z;sI;Q1 z?ug;~2ce;_CnIiuy`8(=z(LuuK|S03(_uRGP`x+q$MoHrw#KGLwByWK4IUVDa6X)2 zr?nPi;IR1+a~TkFhN_49PQS`33}0$e-kk4=uwp z*0UM3d?Gh`a~rgHA_3TabV{k5#g>~mRe$&=GR2i}2b zeo0MS{Z=bef+o=SWz|#oeaJ70HTgV{10stp-J__y(KqM8G$3cQevma#REj-j*>oA=3nRjfBDCJG@sEqV>#&r% z&WdqyOizotd$sTxgzN>c#?Nh~RmRXNVxA{29@v`bXgbugrjF2H2Iln7h;Z3#L+@3q zdZW|%IPombL%A(Df*9J^Vd+t?cP8usc`H3>Q;LQk%nTaPHyi3usVpU*()3k|%|}{% z>=*F9zl5I=N{i*Bd(kS&j14@NWW=YauLIgmBvh zlrC1O(x09sPwoJoOb`cdlE7!@Ma-o5(`euo?~8C(2v91+>oVXiU6`ZDs^>u&Q3U6p zo&@@a3wlsu1{kVh^@BX6!5?bP(8DRK!$x{b3-rd-O~U3D5I>s;Un!QBZpMyZ>ICXz z!_QjIBZvZN!ZFX3=Yiug!9v0I&&QhY5AD(oGVgl-v+HdRuPU(mx2e+-^Z&cC^Z!*L z7T32ivNHa!I&rpoxG&Nw!p}-dJPjX~G(IdKrXu86g8%`Abs!F#Xj&frBuHmIiS>4K z6Dkx{byufL-t;`xxp(f|XAzh+%%5X3BoDlE@8Z|q1zpo7OLLOzn(O1&L(k6JX)oKI zuKB4d-M8JwS>UzUeHAdBy2fvq zgR2;{3|@iznb5liZ>)pHA-D8isr#PLQqXGXHH==l`@_}P>Cn=0`SV*3Zm_=A+CqyM^P|mt}W)g5XUcfMhyedfay=UZ&q|zI}BP7F3 zY+~SyX2v2N7P3|on%y~vXzrahOwFNa-m9OHku9zI7nEjVjKH@=emyoCNGv@B21<@; zSlX5GjRU1gr*#rIDlFaIRhN}7y;N9e@*_)_Jo59<5p9TMT5G3*9W>agj5bj56KoQ4 z>BqWV*=l2(a%Nc)7mOCMC8A(sIYC!zt9$oZamU!Huv47dqbxKf!@m(; zAH@=h&s|r1u^E=R^T*!D6GUZi3WUlcA+JgFUAvZTW-Bjfu_QHElai{;5pV9)4h`$# zxGO1c$lDi%iV2Fkl5QT9H6<(q4J)Zm)ac*Zs^eT6uUmX9iz9ElWHRbK7>v{8knmR6Qm{z|N^RcEq`h4ooUBgDUurG}){O4hEN z6^NHCS&JjYxu+kunRVf;1#D?6F`2Zdsee%Z;BL@t*nW^gFDBEoj$}Uk!E&X?Wt32M z?9owmOXb)eqoClbbUl@&YHvJmm(WB+A4Krij{@HcRPa|j)bVdU6i4PmK_5kMDB-GK zvIy;)y?Z40A-Iz=Dtt@stGy5CvrW5hg%-nDXv^J47ZTM2((sqSMsouo1Mvynh3W!@ zgUmWwT|ICHsvX4w`SIg}>Jr&Q$nivu6}cz#=R2fR_)^)kbyL`jP76P{1^U;=N1=Bq zo0G*OV&{R@!dQ!)vzT$1hb4!-OACT}6gAk&_Qtwc^$Ehl5rcvyb^WaMJ<@aCpEfcuM8Rxyty zptE(y7)cM)Y@@zr@HU|4O2(a)on2x}sheHA_fg);*68?bAo=?%&$wE4Cbw)H)h@YW z$HKh}mxLrt9aTynFTWp*-#jlo2OuZ=ZNQqzc=ATsAJV{~a9v2tD`mhI#8P}}2^?|3 zWFyH5NDBKare$!)xq&3KiqH=935JN`L|l{K9pIcweseuK zV>(B&PuOwJ!l*7_bvJ^d7@LRHExaV|p&QLCTPS?Us%Y8C54r}bLQ$!V5~f-h1B$VX zv_M_~4SOyS>g1Z;pi@xDx0{XMfYo;mKV+#h&%L z@E0sv$a&@(#Y&wM-P{Oesv{U1V-vC7-Lt+YI2AJg-}xtPC_9geitk9Y{+9&FM(%w{ z^Co&fa>;o(2HX6Y|Jpe5YMpsG<}W}O_{Y-*)-fVcptOx6sL`gu&0jlYN4tSDAjSe2;`c3DCI__Ljgv zTkMb*?bbxG*ZN#96wMKg$#rcCYi@Nwn#REo$tB1hh-J%%!zIq)>RR7{`uWyB#@k$c zbb+iNGG-fNqh3UpE)G~1bq1;!e@0uM{hVrSAKMuSe;?9sTeZwm&T=m1{r54b0Fv9s9nPWpDL&QezwDZA0b#N@@K+%A2P0 z+m49TR=!YRl&%BOf^5#WlN=m3t?07+JGHwBlXdWa&7<)+4y@byzIbbyJr&N2Dc?VE z|GD$;`qhcl{QAu${bC*dk7ktr6X~Gz-@RT8ezAx$#!mYG{hpZNzxMuUMQu4G0c4(3 zRF_U_JV4}rx`MLphdP4${vz=*>u8cl;+pDMlMbB8B;%AT>CCqi-3O6qIMSXEx;L;N zpu?#G2ubNIqmvxRo1+sS@6Y>pC_n2Z?l<+vx|mB`6MI9D{)ggOtckSXS&U z5^BmEsFWAC8e!7TY*haj2*+t@%ITt;dZ($lOxQLnqG@37hP?xfF>E z3L&^Q-U(1xR&Q$k^?n|_*ZV#(6u^FK79G|c=7W~;^2aB4wg_p~?Y8fLx@04={h(7# zVrp!d4+*oNMClOwNb?obh|IC6gK29Za33+r95$sAWVN7WUu1spA1+4ypIHltnW2}5 zI~e`;4!o~z=*9WPdQ$ERojdFoG4A>2Nl816fe?&#IIUUE6MEaLfNUOAn1JiNdMeQC zd}VStOFV(*2{P$T=|v#<|8j?|LDUzZ8VJ!xh`)f0=G^9C?sj zTi*76K>joMl2D0N6Tg9H{qz4f_>BKE_(>YJhziJlvwuWO#Hlt$Mn;0${%8`l%&&=W z`q2*y=xMgBYy*}!k!P1r`E{QEkI?@BxiKI3LC1{~Az@;b;F2)hLAf$q2ag&96ccWbeE#F%Tbuu$oXl#_xh2q;kUvVY@>e zmD{b^dpnO}g+^%>&d(UZS5SwCpCY#qpgFF)uR}>unKDWKtD!KKJk4U&tvT1ks3|a* zf|&BSIv0US8*D2xyp++BuNSDtk*%^wf?1PAt6#0WVu1{d!V)_8(d}7Qm1>Xt?@so} zqFr^Os4~K@y`(!69i3;@>LOgWa$Z``V6_D?x$5QuW}CsTgY#ab%b6QNf4Lz_r=*fA zW*H)Z>&=F~TJ7d|q?k@;J24Fc*WZbD$}} zU!ivSk2_drZ2y@U*4%xOqyzu}Lf-$E|DpeVRs65CxYl;}S{zOGWm?gnWE>L(jgp}c z8bn6|LySQeyd&VppB#BW5JFT_=ZoS`i6RD;&4??K*!;_*n2}N>futyynZQ~KJkC1a z;@P~ty|b#beQo3FY23NJ?eYtxn7*y^3BAfSdeD9M@?m<+JpA10a(_ITo|-+N>h->c z0Zsgr9J-$=0_qu%*iR>Gm@ddd2Aq4kH^fHS46%MxFBi*mYeN}lSC$*mI zz6bs3IPsnl>YeQ!sJQDC@KviUV?}4UYp`3=aO<|_y8c=={~CUemsH=Y=c_UD6Yek;9nW`vYT0mH#FfZ z>wzy*pm&nupWwECDB~x(=g+%kd%F~GEPs0QTmA`@|0~+-Eqm})+W9wk`A_lAcj=`y zspIHm&mWvicLHN1OG#dFBHrKDp6m}T!awhF|X)Qw)^JeFUp<}oy^Ji zWv}SW+Q&XecBMBK+&Avw)sA;iP$-wD;7!j!?frat`UNgz2&dMUrWcoI`Z7qI7~Du! zvV!sW6LD4?xfm;b`S~14lZJVt7QPJh)VlXGc@x6KWKq{Bve3qRWkzhAWX?iE=#z=* z(Z+jWCKJ8Q25~Q9jOm!t`fZkvg=&r_L5d=yl5Q9T)EHQjQL$ntuQlNs!Ui_xB}_<| zXIK|7LTL3mVO%;ol%UR#0q=ZyVeerlOD%GK74$%kq*c_un^z|o5H55#a^S^pf82@z zj5l&J^c(Ok_9Zmfn`TGyd11%c2m;~I zym~WMO!1hZM8TiI^7#`(MNTZd7}in7dlSlJVRJ|B^yCGw!a>CnjUS9T zis{S>)Oh*U!gXYaL$%7w>(i|AiEc#?W%F#ws94g@#e2j1AZ9_vC$shlKS$HafWNX& zkAW5?O6k(EPVxg1tWlZ{uf|w08qn)NMp4w`{$`-l&-=<9x{{{Crt+hKArEehrCvU= z5VDDDZ+0nGyn&dG{wrsYl!>sS@jfvLU`*x0%BcD*D*WzJ<~(d1@0_%hOT}Nt5|0LLsUw|2X`ZOUHGw*!Xd#=0%hi`Li3_Vc&HN6ft`D0)r5GXl7PZ? zX?F@Tj6_+GL@=dcxeV`jQWzL0&4?B^#*7gZ2#zRH!L}MBxbea$3R;PoGiR6tOS{%TR`(#j^8y-(vnKw7DE132Gi`_4a?Qajmz$-=>xcO zri$l(Ns}i6{;@4S@- zy{Js=C!q>g5M;u#^*lR2nMAM=eg?X~*&{f~iIuZ)Uc^jT#<>$o2gSV3V=v;b%%38af-7|BVt9yV|`m*wSM!UP*+ho4uFUvnLj6{+{UXzKsG zcpDnNpIZi_UZ;}VxR$WQtB~&Vb9N3HP?^67tJUvR6j;>RSL#nE!zaU<_NYpL*RUYOni%C8R^Ur~5`fNx zyOb0bwmH$4O;U(Ut=UqbV({5HVXI*!EPyt*3AM(X%Zx+~EZCZ3I&2Wui#3R}&kPlm8*Dj_)ZuY#FMY^aECzQLKiw#2 zp=+0+UL`efnO9;if3e79WcVssj+rNtDvqahc+oK@y^x^zGpg61#=FE6+1tYq!-?1@ z^2ZFsoYgh#Sb*ohAxP*TZXz|c3a=VhfV-wBQn$isVtCBl&)+Ld-Z!(tOiB-5-&rQ0%-> z7M&ueAgv`x5ows8h%X^E$T}N5G~ck5RY$q(YPV3TKaekfk3VBDhkh^gUj!+8^Hp7$ z|2t}ht(JD7zBxjI`utQfFFkFA!Q!28dK9CqiIPpKVh!D?R3229!6D+rnRrPfXJ5QN zgpYF*C9-%QmDe^_A-X5bow!_|sDv(k0=&b8n@ono(X-KzGKy(&P`6zBtfp@l@*%v>jxaf%ljJX>y!nuX z5MQ7oc9|Ivo0?6J#`>sOKzk|GG)$!_@BC7B$5gU_!;ElkLtwloB_|tAbV-a11`X#? zy+d?4*m9Zspjd}YXj$^0IRwIalcN|X3#IIEDjT}&zN=P_DtE!)Slt#R*!wPtE5cb+vCkO zqBD|>Zoa#Q*g7R0AK=}XsC!j5+s6l&o&96L9-(7J`6b!6ZZERWxYXMN0lMAk*EaXD zNiO%b$xgS{)a(&Pml*Gj=~oYTxUOkG7!UWLSC93@D&%rCwHu z_2E$}k&=v#4_fc$kVIJ6N6LS0MIt{RFp2_tzkE8eY-74?0+!ZxE+T z1jw_(TYf5^py49}-9L7|>yucIcf&6vo1eB+K7rueBe{cz;?_2uALQ}U++2bJxl3eC z)UZ_F2zE{>!^Pp;dxw09T+%{Ji=1y-+ z7@ERRW?eb`7Msl~(?k>*lj`WH9M#yah#PumCmOf;P8V%mAFEn0=Hx63r5U7FVO$!| z^pP&l3idn>GJ{QMuI;8~4Fm|5cgd_aATR@&{jvOb{4Mt!HbLxD!#pNV!U<^LTcw!X zP|>>DJWaj*+}{hUi_7VZhZSjN0s}j&8?VN5_7YM=M)ZroBbgh^?&^#>i-;1r)KTS| zrl45VS(IFVm18=i4*Y<{V9== zm>(+vBi_~iV@=WLISiQ#a+!s}PU~auPXQ1bGs;E=#XPnGyw|F=Reu-e@5!y3E3l|f zaA{$ZmD&&R1A~rvMRNdHnZg?&fm( z@Ox%pLO!*G`eO)=Kk=|}JlV9&f+rFd!mI8U=gOi*7BM8)KbBDej^5QOEa0P@f;Bz= z)JNEsVNJXFdvh)Qc3ct>MPo8iSC~xzsl{o64U{8N14hZ=`Vxg!xSxFa2{R$D|(;JLd6M6L}qwvPcOe9HI#fZbw z1{~AH!3McH;x2n&lWu;I!@vBNq{}Mo2-)l)5hD>%{KpoDjzWlr6c&&oKE;K-6_lUp+}Ed)}F|Tu*#bR1PaRruZb$;$TnfJaoYDh5SF&WYNokO&L!N72#`5xx+QaF zN&|Lmv()5n<{Wb}Gyt%!g3e|!WU6XPr%_yRU*Qb+QM%*{Hr9s~@g9<8V|ZHca{Qi` z^x8^X<%iZX&p+C<7A!Fm=%@5AJQRfAqk@{HlZa&>6QL=OPfW^L6LDN=t3l`ae<6-> zdy#Q~MS?v|4-t>pl8zsad*;jr6&Z``3JuR*#R4861VH?e6jj@%Tx4gCRK+clrz;+! zTfB@?93S|RCP3S>P)C~WOFm+X?||I=`Xl9GBKJt#FcxdWZWBKB2(Df4zXRNQ!)|NO zchLGT?>sfX>GnRcLSES%cXkI}ciFmp8BV{Y-hRMBUVP2U{)vS=@U0}T_U9hm^`h_g zWz)?~dNfW5TTF^NqRH4Xll$!4ZJn{f-La?@b$O; zLEQa;$U2C;tYs~*!qf%heWOg$OBQ{BWT&uG*el6P3TPN1h13aQJVoncQg0w-JTr^< z>-R9iqHH})>_#W}$NpTfi6bzL;g#^sZtgO^+8Pl@}NVy`4Q!nr%`-qy-A>tLnvk@YOT^D3Cfl*J5#Xa#~d)^@8 z6?lf^us7vzNOgmOpUw38>|uj{*&*FZ{XHXL+7P$7K_pt*&2j;o_ORF_?t7^Nj7VRx zR)d|9{{KAR!2bMDx1Gg8=ml)B2jMR3__znKqVp11=T zfd62Av;n=)Kh+}cah66-8!qA)W;J$zi3~CaTCTiunpnjnMe1xqVfF$k47E3KKY>Xyb*#>P)zp zIcQ1O=j?8EqI1Sj|A>dlb*s9@;l#0ykrrO+gHC_{l*@W@bU}D>w;6>f6MDUjeQq2e@}W7QKuow#dBIzECf+|DVOwRv|r;ssl$l zM&0;4@83hCm>)T%94zEx4a1|+;4kS?E|xML(ei&TCTk0XdUOo(dUBLpi9|~yX0R|X zaK{9je4$qVup7P|=^crqa)r?OHrT?f19PeYjp-fN45mh)xz*pGjkZ8-q!8Ljh9l+$ z4N@9tWTRkDl-E2E(h1?7!oP6M2AOfoigzi#h7s^FP zF=cBd`D+XV(gHrI*EU*P*85yj$6Il+w#mS@XM~f@&B8 zGyX+C;>L>84i@Zl_#2E+Sv%z5;frp^e_<;WhFItD|d`h*)>H7oR|z?+`9*U^J7A7(=ad#O9H^GEy7 z?Z^7TBvqW=K2ZBWNbXw=;nxsJhDnaHi{GbA8yxm{#|;ORI9X#W_eQ}9h<;?YZ3Y`L8U`3`RH?cvs{Q) z!S!gDmV%cRs#5f44o&P8bNlZn%`WVKv0+Fv#CuKXK#v;isOq5|Mx5MAW}yZxg`((1 zbi}1NTKkdQQF-dYcB3R=p?u^HsNyu^?S(-LY}5SCk^2!OyTMEN{)}##nuO>Z3$~MN{|-zwl$!7Fw&-em z<<3>52V6ET*K@HpllHi8K=)Bi?Zc`L1z8V^tO^E$eA?Gwik!3fGesaRFKjE zm6Gzgs(J*~HM8+i(V6PmAM-QTJ_{w&T9eC&HjU2mzD$-6LLGuem2>Gpd)Y9m%go0* zUOdz^`r3NKAUyH`aNWx3`d2DPYD?P~j%7=}%fS}J>a}=JnNUhEsU3_r+DCWV&Ae@- zT&0_E+d@{REhI3Q*CD5GH*Jh`kMHTbxP2ar(-Jqb3?ja(06>IYa3Vrp5PO8n_1&^< z z*p{Fz|K*j&WeXdGGaz*{pzi%_fbAbyzm^`PJnLH!vrjI-jePS}Sdq?^%%)4u1=>?- zUFhXOMI?_t&=3EDdDpdL?~&TAk_@?*1jPS}z}#M|$0G;wX@pjXC1lDv8O6r6Sx< z;*bbpgnGGxVsXUo7YQqx1mRqmSy$Qm9NS(PnTqUa(&t9Nbic>XhyH-aW6^7Pg{t1b zn2s%DH?4J2Fl+0#!c%T<15Z{YFr{E!uk&Nt|a4mM%5)9pBd@U%Sci&%1R#p>Bxdadg~D?)x`bTP`rX zSDgMkeO;hIFz-V2p4HTWJK0_Ikk*eFgS6*j2I&t*bleA=!8^ZO_7JvrS-rK}%)YwK z-`0~(?^gY_eNVQKO|MqNwslvqkV|he2CIi?!?uOg;kq8go1wB2#G8S#3dEa{vaQsi zISq#%*`K*oUlK9C-muxox2_PlbJ`goA2!~$$dk_dc04>fUY_6A;PiUgUksDO;tjnw z?>`0Ly-eo3c&yw1c8&cu#o#sfvdPZh!8m5Ps8-2w`@**tRC*)fi<1B9z&k+hJ<7l1 z;KR;;qsH`(z4w%F9)% zpL@**mTv!Pe1D2>)ocBR;fkSp$F_bqdud{SW8Vm4YNS*@Ge+a$JgO0O18sgL#{Xl> z2P%EWrqO1x3C(*WI(o3USNw~cH74-Orf;~bne>=JZ)RA@TxM#*Xlus21ZH)S%rtd| zWVgy4_>3roZ_KY*R7ogIa|GYs7fCYHBqN29nBDWWNjs zqdF}(TyC>Zmjmi63I?UST^(iGV$b~_r8VNsB(T}Nnz_|i!_2o~D#~@6M+^{I(&goj z`K*GS!-`J=_t!*%?h22D?&qG@H$?}VwXu}x-J=5BIdFWkU2<3^4Ies3{&wY7=+>O| zy<6dNcSKd*YN}P`H+7hA+b+sNUd{@|d8J**lFT&xmKojMQv8VfghM0RX?r0?NLqM5 zM0CIQ#PDqM3Uopd^P_RJZSh8{#bD;^r``R1;49FUZr|&E>pFW}Ut+_B|^t$&kUcu-%Rgl$jqo|;p6KX zA1k@4)=jfqfx5MO?eBz{t=DxPUh)Mzw_cMZ--;^C*<%eB%6;BzqSDQyU?H!RHrQO* zyiG3--(JMZ+4zPJ3Na}!uw)|e*c%wJfx8~*SDR+cHjr^Sy-}M0B0|}|pfKCv;JAa& z{iZOy;8Ec!v_9{aSYJuIciOkNQ=e=Bih5>pyWZ_`BV&2oIr1Q6$jnFiI<{2R0ZOE3 zFuKc9J1v;po@mS+GTPVDU$5-;5H;S(o3S-(r%d6Q#(^&`x`9Aw%u=7uF)YN&fj@74`=?1|@rPAwHC&|ofiY_<8>4y* z-OI8Ap*sO^x>colZ9kvBw^G`MRtXLbzCa1ZSav1MDHE+og#S@&*U1gVgc*e2hEG2n$LJs#1FAOjC#csTcMH-g` zJMwLKIBo*vj+%!hNHSOgO^UwTvwed1+22Ib2V9@A@c`$*p6P+Hxgneen#vs3Wd)ZI zun~biKvyCd=m zjPxGZ&)Wqaoxzgg4JVOnD8)ZTGMnNs)JF4sW&TVe9B^Kf%5aIqvb1*<=aZSwzgCf- z&e5lbI2Cu*-t%_|yx}|>oqhyz<~79%TZAjOyip2I1DQmgKDVmq2G2dY>$0G8G6Qg& z_h&5d78TN}&C~wX!V|^iOvft2>L4vNVlJTmF;5;?6?vlu26X4;E5fUQi7UQp-0;$+ zWX8_>_%*FkxpbuK-&x&%O!N350UzTWKTAQZs5@|+nD}z%`0@Qu$%WqyH|6245RC!& z|Dr__wRLdScQ8^kc67FKQZjb>e}rfWn>LI5@FD+Xw$|)nSWf30Y9x>#(+K-vz$uDY zBRs@Wz-&Ae{(3I6J9LkQKDoCY{5Q(pDY}#R+xAV;Nji2qwr#6p+qP}nw$-t1+qP}z z7dt0=?6c1p|NC(6y>C@7HR@q~N3B|OuKB@?Ll7v1?-e94QPd!+CS3cTWaH}KT2I&Q z@x}h7Hh_$iYF`xNu2q%jDs3P*nrUyhoI*%5k{r2;|%WD*LS=HG9T@s^OcIHt78feUFrNv%myC2r;Y6{EJjAdwW2Vl zNI0IXCqd={Mox~rrZ3m2bx$64gtd#&D7W?&7xWp9>nTjJ9~ZMA_!J5gR=xgwU6 z5gmfJP%^il?{BWJGGF1N$dB@!f8?LCa;(K_L|IhwB$dgfh~*^9kWEO=0{6u4OcrU= zOvkgmj2T9%woE8I{(j#`EQfX68C?TdN$d{h!cMY`B9TnOZ3JCmxe)uT8$DSCqkMa(f=DJ z!00z)L^dU*rft%&($=V2Rw+`p_b&wlqa=t|(Y&(Iw6wOoGI6zRs@g)ce&jppW}~y+ z_GC&&==(aH_yP5JUUQuMcpKYJa$E-*Z217hKw0_Bp7?mVGr=dfTw0yIL#}NvxM({k z`@*z7y+e{(A90a*NBgk=GOc$CG&)BB6WiBjU{9}ZJ3tlpgXN19Ubk<@;~$Xjqulci z`Y{W9cTCUoF5Ydd`wRG2xaW6PsBd3SJHQtAgX=3W(%0iw_r(_XffUX5Xy5m?r|pHc zTd)h{R2jbus*feu4gzftRXs?|=me@|6cDDhH-_vkU>pY-BrJ7kfQ0s{C3T$N`%Ca8 zMN|q=v5YjvG145DT~X?UpDC?hO2o-4hTP~hztZ)=m)jskbMh&S!+_Q`JB~^Q<(@lV zZSrIV%*H<9w@UaKM}E)_nuGj!!Fa36bQ4rH&AtZ7ii}I7BKPhILCM{_0j!Y_;#qqjC^^#pxN|F4@pGWsv@vMQViXTBm z_!rtqf`l*;k;T3aUJHw(HOtEMfd(|1kB{|Hezfk2U0sUq%t`(h&9eKhM9#2>PmfQ^pxSl6~8xaQ6ica2;Fu6O1@llzM;t<8ae?R1^eliM7sbV+4GGj)MX%TA1|A(fZFn1U$8tAWsg;md@a<0Y*(C}lmFW%i19;b7^{xtQ zp8=NPT^AX`bv1F-LKKNUV~(9~O&(r?w(j4C%)i{LlBmN;3NGL%`WdquecqO0)Cu42 z6fd@=KS9pF%Y+ft|bu3k5Ejl80?7qBVnUQgs7*++PRqJa#FD5_1nFGr7?9IejOkbJYq9Bi)@spI<3Jmt)K z=^TFhpty`Hy$((^cyqikg6(?TsxeP;eY}wNWzaAjYdm0(Q!p>+B28ZpT9~qHuluWY zkL6FXhQysO_+v>Bs>Y*!t4{eoSakLaA%O zqI=4roCCm=OdoBNriH$OvDFLfD4a!4W1t#%QLZ-Su<_ZJ9sU_+3*5vNUEO3zVV0xk ziueUjRsjzo&g?bAq6N5_lLYa_u`-)ToSRj!wqpDt>-X(}#SkVg;o?9t4A72qoqI?b zI0X+GEEO!0Sj-jG6tgXph?R3t$F&RT8b&|np7sD2K2axVg$;hj-GqnYxP%>OQ5?2z z$l89*0Iad7gyml>@+aYERa(C?2pQA_TLOY)rusCissLdE&1EJLL?(@+2Dt?x{SMci zzQ^Xt8DhR_8~9j_EC)M6PgNKVe8CLwO+zM7e#PlkYuI;(670Ap4F&^z(I&FuF_kQf zmX1c7De*)Lz@qj>oQsI;jNXOi_ZX4CJ>C(DH`0=E99pJ=4nUp8POZrPR>VX-586?j zT6+_pdX+Z?3udESkTeT6|7%VWxj%>eeUQAERCg8)>bj6URs06&#pOreM5JMj9FBrr1e`bUhat<|X8yU?zuAj7an1_}`TL25yhJ+FLAF zb9wqMshBl34xJPf{))Q7F4`L1m^s6ZGRO+daCP^|k0jQ37&diL6F9qA*?*0Ll?fOn zX$&k*-67l9Czz{Mh$bGFGP7Wim&gAW3)sf#ts>Mp8FvruW#+Pet%Bkx60#ThIpDN# zfhE-w@oRvWgLW~ZHeQZm`IEkn(E_SdrxtK_n{o~sft%wV*%P}TL=*$oTkChD~}Y0h7nJ#E6rE3h%3T_Kz8z?wqA}d^%@hW4V6@& z^qRby`Q!1k6$aSQ^IMuDjnFt7d4#9gY6vJDuL?JShr*35ic=h!4ed217yd1e6dslY zGkLTp9t4u9`BWD^HZuFar7ZWIVJeEfo0y>Z#X3wXSxto`oqQPa8DPs;@aOa5T5f}RjCr=@fPE)Rzc{6}pSDjeI$u1fQ$7!k`l&MI} z2a&|jLI3yr+ecDFCb*|JlhO;VyQKelPZ-q)h_38b@FgCKFC@40R?$s8IJa*``33bu zBY4MdgX-3?g}pR_|0Nr$#sPr&>jU^ptfSDGp?!!z875p`qa$5}&j#?L&PC3&4V@?h z3f&^yP;})08|<<_tR6`lfG}zgNsddTsS<$1YnrCJGNLA|rhMOW|H1}PN!d-%E#82$ zHz@YlHq?TC?^yoQ?6ytQT@Zmb=&I}$BIJj5z}@olBP7%5I7Q->?vnrn!QbO4eW-Pm zhGbtJOSC-G{|(K6PQ6&WEztPnMd(K8whh%N1F!%Qnjb2tH4!^y`z!%NbbG9*!1BUq zCS@&#r+q(f7)tp?@pbg)TV*J7pT?$e zEvfi~l_N-gs)q6y*_IsxS_yy$&_ylj?@fn-J|TOprA(DcbB&&QPb8dYx$Ac|6mYog3~Am98jso29vag3iij`BFYab z$|E8H%*yHiq?7hk~ zip6E8118Gp)su2_vYc`zlp~JvrTds#6GN}%3WbyV2F(!{01#T*LOG_cM(UG)km5I# z;A&)8wTjBeyCTZzlGGGrcJ_p5_j2;d29&p0FVd**ZO!SCG@A^$(I4Da_EgYHN5m;Q zkeSEO;{vF3rW-vVHA;X_x2A}d-6l$#lXd48f)<5wIb%wlShm(=2cI%I0O?0q1ls`( z<#t3%nH!gnz!2B1OiO(jFs1UQf35=++5H3s<#zOQX)E%LBUOpRgaPUgPFs0%$_cMO zw9QRusOzOvlOq~9rL-0AC;XTlR|$XsSM~ymtCGBs%!ITpfXjFk9^f4xA!(nkNl=tK zRDJS%h>=`mB{Y#l8I(A3eyVKDF(#tUKcT{rX?b&ypAngS=$M4*z$Q@@p_?KjP^Osd zafGd`KBVwiPl<`{#K~kw%PUG72UR3*?m)%cJn#O8FX2%Cf@toin^K(^4ebH(M4B-W zGe<A){rB}%!R^jXRrGvY zdh?nc8%{?08h3hKk>1bh#)Q%D6}#DMZ-tP58rKCB=#gOUfnQ?2N#ZnO$#60OM3mbo z24W$SfKNICW!oy0>F@&I-Gf~=X$@iuuk=*lG*)+{uC_8O7<~o8$Zd>M2H&(S_;@$ur7`eC{0H(e~M=#SgeTyotX;Vmq^N))vrxDQUC0f zfSufM0L1PTdM3RX>DLv#f|?vMzZ9(t3S9RjfrwDfCb5!oxi&Lj9ewUa+{YPD^qe6u zS^0*9UJIIRwobrX=75JUG!?f_tFG`{>YLk(n%7oVx8=1L&aI!~n~ITC8B>h^%f;>)Yw=O!#9u0QKWXPpiFPv z1;aJ36;4VBMCfGb2mrB6MO#aE!|6IBLt4c*++QVKB;DvJ)tg9Fcv+a@*F`j?)$n4T zEjbEb7%i)DiRQ{PK3)9LXdfFlvwXKOSX9>B(vaqI;x>@Ucq%H|_Y#_(edj1qMLcGg zj2ap&3rK5bAk!HOjoLL2e+cae6HDx5-q%syQrnqp#?%)n_*vmU&TjoXnkPL_v$jbg z?tW0XM{yD-sUT%!yMo7qSkHq%lK#XKVWg5k&BDkqof(xK)A^|P8Ft#*lG$^fYywAf z^pC_>P;|pU+7+^z6G;OOZ!{*!2TodJMiEc=aYf?s0`j^*tR$yXi<^+?Kr}u=bv+_Q zfiJ=c6Cuea2AS+^+Je}#eNdl(J%I&&p)p%*V~0y;n5lq+v_q=yxNLOS2d*M6Kf^iFf)s=a3WzLvG7P;yR^|1R}|CQ8+ z1N+05^bmB%3(M{>2Xv>X#zy+k`VM&suKZMTvs8MPmgXfu^YI-a6QquGz3nXcM!)fO z`jMsP(RuMFY`F5g^s#yIY$PZiJJnqCL^4C#ziNFIrtysHWHG^msnANAw8~mSQ37+) zZZj=P*0o4c@oq@5~t7paM0kiF~XzRBz_S+-Pms|1B zro_Jrsv+?(gZYmXPQL7zPiH4PE32pbGrNGAbT~|!CElLY>Bxuf_V!>(k#D7m^uddY zDz04i8gokkTZ>cLLI@7clx2AHHdFnS*;TOlS0_lGGdD)uqUaHmWG5`FD8iq_O3KJj zO##5Pw6s6(bl#r}h=^#!p=IoHHCcivE~Kmdn~cV_#pWhA?-TqPa% z&*(;s+K8#;Wgewr zsl%`gEF zZ*}zmTQ@~hL*zPw)3tF!^f~rZHU-P{Xdl>5wX>c_+=cI0{+)^iU4uM2$RxE=PGs_8 zJSzWhOwj3KR#G_=f?hLl{XHP5CjBkCIhnw8)<#<_Gd>wv9quiYMkp+I&xM6W)-w6( z@H#?8AuEJ=%r1{mT0_A;dn7hFm zFgadOguWWFUK{y3q;P^$kqjq8z!+K^?e;-u)cL>d;efwKUCf5vY{T!CdZwy7T9Q=var~_I84Fmi8v(q|)TE>Q6Icp%oS>ni;6_Fn zOTug#L_xbZ?b_0$bJMtklM(H>;CHxz^ed}@;A&bADLzos*VRu`<;%ndt@0@cbrld_ zRBLdsiO*>!lRo7>l?;Fbqp?qQkwb+vh|T1b90=7V(yd+&?@K{DsK6O9J?gXx20F57 zNa#P0T)M}P`jCI5M=&RS2JEmZNNYoOyi=FVlrt?;vw@U-jENIwA(O;(NbxNFMjaL8 zeN<~oRq#|U{u=U5KBlm zFQaUixANN1Y$isbuixV4VHF9mbD91Gtc47NN43$5^#M|GxhINkQ^mzUG9N&zgPF{* zug(!O(Ox1Xi*=nN+X7UQ?P9#$=u)!STsDIwF*o-t!#vfpSeutdYu_G2X0By~^`}Gi z{!p}%;TmEa;rqEE8lb3dmbE9vt%2{B&)pxcld^}HmOOd5+mYexl3c#8;J>K0kJdr!zg~V`ZU!Fb- zXP3qED#lGl+t_bndtx8a1*0DtmSUJ*0VrR*P+rP^74ozDqOT)9aS+i45n}Cwxh2HB z7`Dql8r7OOxzhSANUa03@I}=|H@5MI%&?RvJ z&MRiqwR;woHS+Aa7Mf&pibRi6$&~f?o{8)4j}mq$Pm`XuDRNbO&`ivD&{nHVfm6-{ zC7ro)v#~>xV)608w9H22LXHlepU`+xWXd>xE{xUU^Wr#h8>H>!!6h%37^v$ce(gZv z=$3hhw}tK*1fi&zuFlMP1-gGkKyYWsAYKEg_Kmn%1cj~K7K_7%+}2-au(HnxBcn{y z-a6}6*#}}-2WTzz_WDWOIG5aRjXdhh8{BG}FXL}q58Ym z_30;5hAVqi@9nIde|e`5_2Bu@H%YC6A;bBNH*Nh$U3n1`eS1I1tR(tKlSVn-MAOYM zX$)&J#GtNpTQXiN+@6{%$rDSa?PbMx4ZP}UxwX_ac>5FmNZuD}ZJT=>!YvfUYm_vL zgW=dJ&i?H7(Q53#Q3du&g0SLfx1|S#Llip6IyhPqpNq22MgN;-m?HFsj&PP%6}*qQ z>+F`;6|NDc$LUuzqw_4od5)38h4JrK&Tr4KZz~TSKF)7m=x<4F)FUSCVu}a4DaIHe z^|M$Yv_dYB$-pJMoU8$hRe{c2(W5`19F3->y;Cw!T5Ksqy??ku9^L$7_V~B1`RIK* zRt{TPs@~|NYJBLJ#pc`z1PCqvF&ZFmd34SeepLp&F|l8fJaA=DSV`7=K`-P2G0A^> zbdn1`>RuOi#?+O5Lve*fGO^=!@rE48135nQ9&C;CYNuy;|KNNs7v4t_y%`nmfhU(r z7&A<&WsW2W(&4$?YhAFNs8;AH0Z~`J{Mu^s>C9=#`YMZR9PQ*=rmEO1JR1XhMKb2SNfz;H1(Dmsj2> zSK$k>hv3!Pm@PwA9r=oz=rU12KO}Dqo_$k6K{*oAhI?wMID8AUDgq!qka;UdLux;HK2_N7gjG0Mms5xx&0 z%5O8)Vz%Wb%+R#Yl)1qR@!!P|{EHD^B@gkDyg~T3N(s zzz%DNO^PK~bC-3)tGbu!@imGf-NZR3opd2>SDCZVBYU^(4CF9&!O3d0>1;BfjrH!G5+s~Kf&Y{7gc;|Tps-So)LR#8DmgwJIhZliJ%wIQ_?VMS0r@o!(EM-aCW!Sbq)~Np_#x)U; zd0SAx*i94;kX^&*!La3M7I9$mv7#2aGv}IU*|)xl=jue|^C7mpw|U358Ii@-tlNpn z#g1IPscZ21z3pzL-u>$a6MEu#U47d>myz4e1qxn=5tM638@~G&pLOMg6$SZ3JF}R? zvLyH(FV}_Z0Z##E(mp}!_m&g6>p|BUP6WRkqYgI&{3^yy__OgoB$HZtI#W0D4c-zm zscw?@?DWg8m)+7~Mpp=zu8!e5plb`}p&QO`zo?x0+Z$ajxXaW&5vqYn4g7;`#a538YWG(4<{L%h0%&Konb z&Zh%^DeNWFZE4XaJU*`Sf;Xvzf_1~x$%7BFz!8NX_Q?QNa$E9A$fnwrn#cs(?qEj32B`?M439S!_5{iPr~1cx}$tvDtC#fSgfsoxi? zFH-8Ot~8&Y&}U98uc+;7S3JI?Mc3DEZvi|~)sGNSguRN}o-IfMe&Fj#(9Z9>a;fwf zfLHuu4#7Qv#8xV2-}1WkDcxr%`jBS|UxQLDRgimn62nLy`S+2&g-%c;6 zl%d}HK2$U;{&g@NR-IkF#PKk>f|Sk+MTRnVA-f78+`n){+P(h+)S=UDWHnPF7~}Ff zto#Br6@9GW){}GXS-;4zWjgY7L_9M3L91$3Nfl4yGj8B7Vxf5P6;QMxvr7f;PpK_R z!hKe`wYxquqyIt}@F92vJj$lT2O&FwqCzW0cHeORSphyBZ|g+dYZUo`3G9QDy9BHn zsCm7j>|l@hz_bCtS+)>35tMW%azQ(uKC(U2br@J5ls-?d1{D>O2zIz@9)p4WinVm+ zq`L@TM}zB4RhJ9AO%~3|!r4Xbr3$FZ@yYc6-bdP{{Cw`~$Dz+lD)dAkh3O@Oz6lr; zb{$KxG@fuN_m*3xWqb5iZnnx_cTw*Md)pM|(u3~ zXU6p60k!e^hqq+LqrDoOwO{-WzB{)`*ZI8}u+$y0t|pdT7;4L$BCWEG9%2iH9%M^H z*B(kr544t8BjJqf{%5k;gUikT9M^*veg^pPS z>mK9nfo5*a_c-9j z6(ML%7tJY;8O&3KRLr|$m)K>4urBPMN27`FNpdP=pVUoF-&xyA5yz0|LQbzg#7}l6 z3}F4VBQvL>N2x5MmsbKn;416q6&m2Ef*rejdh6c20Gf5AH@oyoP=d3Ym##>)QCZ4`7G?97rdD{xJi4=z}`#xVrF{3C0qy z?D$v~_PEesbARTs76$?;#~IOIxqRu;Sw$uvEpe^B3E`)-d;5MwR^#A|OVLS{tihx0 z`r0YN+j{2m_@oeE#l#)+mKAD!c8zBUaV# zZgB%YiplFtpX%s71GfwZJj@Y2%(iE=E&jDnYEN)p%`21F@ARm9Fh(qI91@)P^0*j@ zsRT*|$3pco_xb+#Im1iQkcvyuVLZbbaXix*$y`0WL>)cVBH4PpM4v2gtv8hSpa=7( zfCr2x@iVg3SV_JyUJ~yVZ;dy>OD6AcD5g<;OfrT5b>3KtnO$rZa=PBX@$>nD!$l9b ztRUm8Hu<9I``%uEUkP~oN0M@P6}?1WIl2dF#y4JY`Q*OBaskjsH7~F|ieLVG0JNhW z5Ce1~LLw-G2nUVHNX4WKcB*Mb+DK)6g+Id5w;zpYx1bDxWZ^ZlY_eE}F;#J&X#aU) zqblP~`jCKthM9qYsQ*8o*#GUS;e>QcT538WPLyJN^#>LZz~`Uy8zUz8M=l8joA@s< zQGAlP&kx<##HW9kTK^BRM(Td|ti)A7s~`DbFo}>jxIBNAMpI$=CPfM*}hG+)NGKj_XX<^QPmC`hVGK-c5DhU(QZ|^nkynLD~9c@ASDolKN)Q zKeGDh&_5FTxJ2AmKyUhLTJJ59{h!+&v1~q5`hLUsq{8=v`WoC41p1oXA})O90ewMy z8wB~X@FRYNbNRCN>q_0)-rm4`1%kcE>g?5k-Rj=9fV|kg;1PY)K+1%EAJoN4-$H^Q z;?Fyz4OLlT{tZNQ6r-IyiizjXfxMqH>IX?`3?u{} zf+=Pq*QXEcoAhdhl$5^pq-NTq=8o<=gE3)Z59~{SRf}=Fv!~5Tjzv{77l2lYcTk%u zV1YSiC^e?f?E?WfWnvHQ697M9WRLA51dqg^)~`+C$D!(pO4gm$6P_kp$9tY8YuE=9 zsigCx)iM{LVvTF6n+vN@D~DG97b#^nuv9rW1ak?Kk+#_PFP$IyfL4EYd^3x?C7Q+J z{4E!NqU{w6d%cnqB(>!2t+rPYB&1jL)rmeV3`xEA4ONy%Gk2k^1dV48o~&j z{=MDf@ZUcCJtwSC0R2L@`ZncCNVA(IMjbyKMIFC}SAER|CCBOWOUG4?oEiZt!ZB$$ zX2in-XuzJvMurh*Mpnn{CL~K_M?kXBjJlP$Lti)d0K<67=hlo^4b`Ji~S`IH#_-0b>klnj1U9EoFT!w;%bk zg`&qmM1)$|`8dS-33h)a(ISd~I@=ikns5io0gG~teg$F2kpKQsB?fVAInpgXM_>Zy zBs*q~roN(mCqt9UTE)wjLoIDKeeMdR&?tj)8yPtq$#yh(FFtnG3RP z^d%VKBv|v+RE}nf`1{T+l#M;a`Ki-)N(q(1TDr?XTg9|+X%J1Q$>xl$Abp3mscCX# zeKHSXc2t|BeOGp&psxMeNSA&!VzZu{_i zqDYFZM!Xl@!+B`qf^r+*s#H%Ag@B>6hmt?rJzhfnC(Ya~+knQ9G1ntxU zh-PPYOF@6#7J|z5F+k?qY4yLG!2^6?Faf}jc!NAh*T%O9f!;5*L9x5IMAxaeKu_p( zvi+g}1|nYCJ!H4puu!$A0jpLbq^J9=0vK>~?y+48oqhB`{%bgfFh=(2U2||Ubnby& zb?_Mm_V5+4Tb}JEMVOosG-5Br9R5;_BjGAsM#!FMIt+j@q;B7h*vn6nRxQ3W7)g%F ztg%?NEQMeie4-7P&d7+E{J)}Zs=eo35=fce2BfTj6B0LIouu3MmtT-Oy?7EYL|%#k zD{g{1zX^RJpRjx<7j}jNk-ma7`HXLwuD`N0dv=ilk}mhIw;qr?N6VftK0~`JFx#OU zQ7>4KzL9wNSsayUq%jon{ToiEu#%XLiz#|~nm1xOU6crgD9bpqUI>&TEoQGEK~-Y_ zn($R|ip>22vj_FWnnh{}V&QJunp1+LXt5|ek79jH>)2o>-3tRIvsdh^WAX%E7MF^ZrSZ|>=%iD*Sv1IhZ(5K=u&5>5Y;bEO zVO%Gh_FL>u0~!|_=T}o`E-YM^*~Idd7mYfLt4cNc&_3q*vyYex$|+kmGDwW*uGSG)7t_ukZr$ns%f4ao%KiqId%)hK=+{72l+HNuypN*X(NX0{GKa#=WZfPo|I~)op}%4)GrZrPN=|t({}vZbWbL`To=moV=LD}uE`ptk_}VEMXTZF zK|ue-&(?J*7QvA&sCtR1CTv^(H_j&204=zrRspZr$VZU4l9)GFO8kl7@iMjaP$mDL zwAquVCo-~$xglY36DW@+kIwp-T!3Ci&fieJ76ay*3;p0cOS!z0`5}oKo2j+jcET>j zHdmR5DyO;N(99$`v1=7G`-Gk?`?>=6=z@x}x~gt}11q!<*`S6HXxTC7xWT3L4@O09 zMZRo|G|2wWsB7-c#`Z#Nz9*N=ObTjKkI6^nD22=!1l~|oo`oT9t1y~{X5Eg@h0Sty zcW!T?&JnAxX#c^mV}-*mUXInwON9mlL*Uf?;u3evfinuwyhi72r@psCxw|%srC|?p zq|xn0WkW?GCxVtStWF6|QRk=v1p~2$n6&sadYP5z5-FNR*t1z-%#5(aXjXL`Gh4F- zVR&6tdKXADN<;-OoCcAAkNG8EFNKpVRejCuul>(PGGccD7dG9$AKH<@k-@@n)CF2H ze9_C9sT^*tLJ56!1uEWiBgiS8cKA<6rZeO{Sf?!Lp>YLV-swF+?gHo95*(CU#fsK3 zF&-UNc@yhawzdvt3|Eblqr9&V5j-gO(1;MIFW9n?)vBaaSM-i_k)$KmKs_mxR%sS8qHEKu zb{j+(R@zx!w$NPL9?k|a^^`QU^b)&@S5zd+G(Ztv)c;=l=VQELx-^9PRx=lFYaOyy zi(>=ohdsJaXR|oIA8V7}jeL#t2=@ZD1pf6V@`!D56;y-R8R~H}JIP(H+ob0lt`9LA zFoGY+-%N6Wj^3I)fWLBwf*w61pkwNi+^`^bbvapP?{b`io+UrpqQsyKJ3~FDzn-vZ zs8_EBp0|rGmXbk`)~iQIKnm-Zo-)1;-|$Z&0U;`=)|Wah~fgJlsa*|2})$s9d^aIx9QCeqli{JStqXfR8PWlsj9 zQc{t!_35lV&k&S^^WKih7atr=brOKq1pJZ({5a+50zKqQpzaXn>hV<*@jGL2I>lrJ z;$#HRAnq#36_3)bC>9P0&ewe;F~pgBcH|4hl?ITKd)4h2?5lt(O0o2sN*Ca9?oLkZ zZHEFFtcM0}M_W0vc}Qj6_j=JejPauEIZt-g*dO}}Fv)s`%^yn~=Q~z{(mb!YN2`K= zFL+;g^?BEoruIU~lSu;5xp{3Sa=1t*c!7V9Or3Qy*mHu0j!c=)rLp7qf7v@~aEr)@ zvdCCAqRXZJ`Bffzq{0I#nE>bU2Zf(wvZUzbZ&hjH8fC)< zJw^M?2&r)2-lW2&Lz<>@=m^meK8(4-3p0h4Ay=t_6qLP81!L9%sz8`bSJhb)iGRxTw;Xz>%$x%yx8Qqk>RTlmtgIC8epNy1)?#C7Q^dmnYt_ z&>n~noNBNfoNLqevLWXZxKyo=e_&@*n-@7mpIpF`|8%luHiq>O}qWgoV`CCto zP^^S}tzaH1!gFk%Bg@)ppaOhS(tK!u>zMma1> zl85gH*Qh2g@h#&&Jz0K;{LK;^ux@mKCBI7EXhm3V(dFWVbXgniH72~TD%Y+8lAOe6 zZ%x9z(gpf#BKM_EQ(!Tdf7Ty#M-(!5okY(GIUZ&THkq%G?x2%{eAAe?$O|?T`<}DP zTy=EEX5_UCxcF6{xI$KDyw+~d zF?P9IVU{h=t@D0yl`KC`N|r4j+{SQE;vb^-#D;MtYX)3L za6mo`ciA$1c?;|{q_bY`wD}W%b?CR(TE$wd(Ri6RP{i{AWBVzrcmKnOC_=zS#XPls z_I~jn&hys_a8}Qvk+FJmeE%1BG4Iaw9I$sPt-ks^7HY6m+io!!w<@Q@ZG1)j6|KZQ zr@Yc!AGG$H8PH;JF<5xOa98a9J5$6c5T)d#cN2webblvLK62IAngF|QLV-NIY~Gmu zhJ+E_OLD>h_b^Q4NSg9S;Wcg8M82%BK5qOdD6$mHwxAzl;g=FF$tGMp5AghzUyTOL zOxlYj3PrrN*|lH~37P+>`SEz~3mi{gJVNvz$DH1Wy?M88#yJDp`8Y@LMX5ue>Bh83 zmcliIoKhCY{X`g1!@^s#NcVcqYRL?z16#Gpw)GduKlh+juTCJILUa;VUanqWTYnt$ zLORuD%gjM|?8kqe+a02nif&tn-n3i}I>Q;s5t41oklA>Q9qAsioE49sX2c-7kWrcP zq+Lz>rESCi(Xpw%?%MV#mtuEC(cdHqtI7;L@?{3ptUlP%;mB z4({q@62p2g_$Xxzpu<;wE+}M=M7CH(ckCepg%qYkSi|VpW!!7Eu+BFqVU*7j@DrfC z(#Y?cOmKyP@pdY96xJ47Mv4ZJj45p}{MZTjh55ZvHd1o6etDFk5jg8fC&eb4-Ud&a z7HP^W&=)b8(v>y83f#&mBcDYE0rdbX3BHRNt;ePYzAFikSp(9-x=H)J+y27I5pq=Y zJ6J9FgG!19A}PVGlG9ds+P*c%vFr!hHt}iBfA&WorBHRqg>dGb>pw?f1;0yJ@%}3s z9~20P@&7golhb$5w>EY%cKDwu_Wv1RD_+?k{p8~htbo|iJXWl}$zK%<(AbojNtd%| zF!1Y}gQTTlX6n0wNTzI=sMB|c(Cyh;&NB}`0DF-iWGhF$K}(t3w3$BTaJgPjdcA)g zf&N9zG@m5O?1!N(8P$w)lJ7$muP(NST{-G0LUCs!)=t`%Hcvq{gR)`Zp2xho?7dum z@i+1CwkH1QWole!MGUIGC1zLxg(SAn|G#{ul^J%(Bk6^9Zl#m8gn^KHYj8>=(S))3RmZ}Nz58<4` zK=bhKw-0Q^*tkXKt$Eu(k5ImMpH!%c^7G3>u_;YT>}e+{Z6+S`O2Aku)qPbMbeT3u zXOq6=RItgkNRZr`Ho{kp0)aQ}xA*VZzv$tE!=wZ59$jS9fCtVGIWVAn!4)SS>?a-; zQqB%iNJKU<#)H`$Uc1hf_Gv6y<#*ULmMjAKb2Td6x{W(FA>k_s6XlD^WGjp*P4czA zH*==KU2jw4D8p5wwp0@H9(?bFx|U!bEz(TC$%c@4QX3u`3DYY(setyfgg76Gr}wz` zzzpyZNZ{Q(VO;K5J~842Ir2rrhR*R2Ju&7%=iJ0%^Mdrp1UVE$#lGQ)JVMO8q?Gf+ z<9r0LDKube9cA^5<1#Jj}kj#cn?y0E~;>qzBW+jZ1xD<2J zfoReL?s}wSqtn*jDW<5n3&KhFZM{)+0UZ%wGNN|ICS~BRI_Pr}mR7xsHoZ~i zrJ*#NH6oC91&?L9D=ZExX)c1c4 z(-)fF#aa!O*qq)={V;q58(J2(vJ$!#K?d!31#LX@@)F>8TfULIwku6kce8KBW|^km z3%reoIhIDj6`REq4p}%HNUBh+cx=^>e8fcMh;nu5t$=ZTb(Q|kyd}3DfX4r2tCphq zV`jWET*e3;Tp~?VrCAs!5ExR{m{EYL-hu?(mgN!iJG*@dvD|o^l00mEp{=usLuqu> z`iz6-Q|iHsW7NV)InXzVrhDK3p*H((x-069-GbZ25ro8Cm7;~rj8Jrxi*TR3ltMQv z1TGl-0U}uJo*@)(3?f(oyTEi6uEyOa7>%{$OMC#@tsx|z{H>yvVqAY=0HhF!mg+52 z(8?}O-uSD;OIv{Mt}qzgo-tU>u02@IekxS4UBZwr$=(W-&%i>7okl+_SkKT6N@vK5 zdvWRRvzuTao}FTUCREC;C|HZzF>BlyU}!&m+mMTY!^YFqx6ty1EJ3kyZ%7XBmg!`sx948K%z$Q9%RbAUV4LPp8IbJ3r~>Hl`;4wnWqd!(Ky}hnA{;@BVului*-(mR7JeuA zPEpn;ZDwH2LaHs`#A30UD(jplVQKkCG6TJ;tqHM-gHrsyTu}~dBBN80b;7Nss`9oB zZJw;45Z!LKy=#128$DPMVw`G?^{`^HkkT(yvoXEm@tDuIj6a@9ovit`^f)sdNQ{(U z#3#8K`=XHQ%*BJI3e5|0*Y=k;LP3{B!W)(Bja6x+Ih}kZQDp)d8aX^vBQ+mrG!tV> z&JiR;vs$0{DyLJ?Pw}XrQsq(mE2JhxG=s%XuxqnGbC1Iym-J5j9=(ngjcR-*Nz#!$ zS2K;x0MvMf;~shHEjWIeM2F2k`b>qOgtj=V!F$p7cOjXYZ;OpRYkHE!QtIydWP8iD zVU>vNll#2uzz%T}yqpxP z@=mFRt%e=vSr-DsROi^)qCl5sh0z1&Du#3x)KI=*!!+cS2(n>(33QnB7bBE7$C+R_g)2$xf{f>KP5PE8D}-@gY0P zN7f(?hMii$^M=J~F6!B$&a_z73zExY+JWqudyA?Y#=X`O>{**;ez7r%tp-YL%q~nB`Pydp6@$F!LH71!6jO(un*Vc z%RXUEjr?#3CLhjNM97kDxye-ffHQXp-HU9+oGN>_y41i*%FXxE6TuuZ#0fCy#uMJF z^QU?s6KY?`lb{NJ&~feX!BpB^+=itfeqE1eCfGdNlUkqLON`$gW7WGdt(2<|flLOI zE}mVr3NYlVWVe)YLzsWLg+&u;FqbBl;1q*JWlw$o+iG3>)_|n=yD`Ls{9gnFa<;Zs z#zum+Ha5nFPUf~Y|FKwSt5_>yDkJ&OSgEa1A-f|}6pK~kXSS(WRr@Wm{K==GMc&L% z88&tlM@`i3=+r+0^9|$eJ6GjY%&R|~9AotjWc)^a-jc>6!VfbcdA-hh%yPWper^Bw z(B=Jx-h+&Q+@QxlRi+2ZD)bny)`LDwU4%46 zK%K!(|E0$CC{MN5V#X7%Kc=QM7g|i_QBNg8Y2YL}Qp%u$UJ%^ebq%()pEEmWY?ZND zq62-^Wi$7HBsf};?KqWpW*<`Nd~j~pQ}_}4=rQ4pWdqW3S=8E&h=x2@jy`ZkQ>89? z;J!G10wKjn)tYDPNLxp(xI0y&$w7^zC8{gEOpYkTIedQNuw{r=L;n=1sab=11|>p9 zvfN_6Ss#cVqIN|^Q!;W(FGkKPNl$)`9cde`tUF=~Eoj2BNkktqSDA+?l zgydYf?>uomhWDx5bA%XgDaQ|SqrWP9#Y9E0(uq97Uz|#lmeSVRD#$V5l(~h`8hMo; zP*#<{VU$Efq9xbY3<#)zs@D4lMP!-b;8zZeltkGo9m&h>_Zg#ot!5jcSyd1=tSj83 z<)FP&0L6W~Ee`71Z*$G!@CM~mxD?ji4rv>qaSA%qpE>o6A~*I(d*G~ElkV+=nA_{> zma)5D^Oh~9FD)T%1BJwMq(-|Q9f%tw>=Y~10cRqjKFxV&QgR$?y8z8PQB~B3ua%F% zMgN0@u;p7UY~%K%R9?wstnPcx`AaCz0~Ldt^hJEBz`mS@aM{Xb1KO3tkJiJDiO!^2 zQ5#u);-4*~cWP0dxaQXuY(eQV5W~$@?S`qS$f7mNK?FFyeV~MFZG3}w&{S?U#dpX5 zgTELgPo=a1!pmC2FcvH28-Du--&S5I0a3;Sjz30}N@B*$CwKtktbzwnzbxr9SP_k> zsojObLeYg#p_&6~+-|)gB4@O5fvh|0T8}MO#lt52+|0{=XCEbs51~ugP&^L%nPHq} zUphg%WSxqD38-fDwf}&2K4D$-lBTGwB~%RJ*S{WjIEydIo6o&iQQgBQ`U!$W-4l97 zvt;_Ww^hK)BlC|Uxw&|8$z*52Fo^{@w*r{dD=U|U~BNYK`TsKEv z>=Z(Z&=AHplp9=0umIr&{5k3$4&b)L96{Bo+^by;eE|`F()0+02{4RFZP-mHsN&ZT z(V}yRk)JDBIzy9mAN+m+3kA7hP#AI7L|qzLe&#->RvtgxVC|WbB{`MV1TIi)|W}Z5m=q8_~p28`5Ha za;K{6!)64~;8R|N{8O<5JvxQ^tPw8#*+Fp6e_z1=^)$h)q&x;j4ghfN{=c0j9E|Ps z9gK|>o%EfIt&MG*{!cE47lfD6^7CE$ZO;+$uOfxd?=KKPpdbT2SwsRJMgm@-IvrwP zUGT3WWz2|VN(yP|Mk95lgH{eXLZHGxQl2EdOs=`b!g^W5QZ;pBctxYSdL{XX{dL>@ z(L^$&%LhEr9RqwAZ87T2n-X6SaUzZLXjb93QJhAp^PT8$BJDd zreHG(eqzkZI4L2sI!fiCj1APZAgb8s{fRlIXJm2pk2HB*zxaeurb3xi!3BW;WBNtO z5!vvBk~I4a=;NBCSe^qH@nYHF6P9rZo+LZIBzZkKW3qxYd7+7Cib78D)Pw?sL(=+0 zc!mQ4JY8spOvw>affbFs3q?OK%zDLCA%yZtpFnYh-)yN*sbYiQ#Y|o;<9TJq2u)E@ zh};T-UwzUZ(?!`T!G$x5y*@qi>;e*4O>sYJrnI_&moj`%qrG^NhqQT~Kp{*b38q_8 zG1Co}L`LJfU_@e%hSdavaeGPCn97W@hV@@Lm4#Ib3r*>941+BVY4gdD>WIjM=Zu9M zU{p$M7M0(qt2;1pc1FWIsIEL0%atR}vQ@#oj`VrKmT6!ur4K<_CR$T^7|h@7(s|W5 z64ZiPoC`0Wc~?OB#^4A|=3@cOtvoHE;K=f8D4`=%=3`0rI_Y-?V%_|fT7=J9_)kf* zdD>=jT~(UGj>+;HNR4;O@?BP@<4l&dYAxYl+k9v;)x}lj42No%n-Pm+j$@CCp8k@p zvMnK9Rz(L%7MgvQe;YF6t<1+Nm|L}5BEg#w1F)G5i7Q`m57_@UgvYUyw#3J6BwoxU zXpeV`b#O8rQXN=fdPhsHly8b_Z{;7nGaF>z{iWAyw?v0OAm03~h^wCFiT;_(05f%h zl|AEmbC;3B$v+P+J^%f!3-vE_X&*lGq3i+8`ztJARD8h4O3|l)i+`B3{0&L^x>U(4 zBOzD*!BN_M1hn?o$G^`m;ge53ySX6_UFlfS7@iQy=RDXt)y-_2} z79P2>PuwQ+9+AQSTTLd7CrcqIRAs{f1~Vo%f|Q?q5Ho_ ze9~v&-(GXY@A59;YTHGmAK`(Qn3BBwaUo-o@6XVUo#mugtErODRK0Fl6nzT+Z@)e- zu`u3(mDuu2QlTlO3eErsFeITFJ2ch5Os5#Cj%t9jb8aOtQ0f@7E%gLK#h;0rb;A6X z8KaJ8mJ}2qtL6HChzZHjg~e%z3z>6F4-MbcgeD%Dd1MX`(WPKpKm-9}Ya7?MwGVGE zT!OhSxpidCEW3Am%S$u+%l>8fugeI%#;&77 zwrOjJc|6M(z+wPXO4;>CTs*6tTfn7{!R*C5@18+sP(x|wzeJCHBtt`XZmcMhLyU{5 zxB|`cs=~sTCng{mF-*LYlXFmvz;hGYO#|6n-Q3vOyfBP4>e)e7kI1ytw9qJKU(til zUaIZt_#n{yWNc$@0k+m|Yqz$K3WF|6DEX*nt|CW?h4}Ao<0nm>?if^&4CoxC zUWPwOIf4WR4c$Vmcm&iu>s;ir?;bt|q0pUCiEC~A@oD}5zJY|3P|*vl#1?mA6Re5r zoVaA?;F*NuNY7WB!I^x@zg5zP_>HhYh4s6%AdI)Dg&jNQ@JFMvS|? z&XhCe!bZV9x3lGMBL?cK-oA7ZPV!13Q0HGXddk?`OXq8201>FdO)c8thdE4S zszRV99@j_^TqQZ zDXcGp1L;sKA+S(M_!#}-;Tt+j*5f4LhW5_-X+>+pJ61ilGscaag4 zBg-fdE9Vj;Arl8YdTSMzsKS?+UfjBg1YkuSD#h0_R&qmC(vUVc#>e`5`b<4KO%mw>_*3_hnRo$7W;Br$T{p5Khs%A}=lr@BODl$wQJc$EbW-C2@Kse?gInYM3v`~ENNpuR*iZp!AKf?RJ4KGRR4zyE zs<(*+fDNV^PCTFty>8?{R?sBKKMM>870_?!$z0sK1?HuBF#Ycpd|hK#0oWzn3@bgc z6%l$}@%Lcvn{x-;Ac|4k%w-Rm2p9=WDv%+pi4G%}?(*Asa-E@n+(!(44oN=6wN zC0MBV>LTh76A@m>iwN1C4$vh4gJiHyfB{9M_9Sa+8qw09b!KknNl?q ztH_pIU~CL>$P;#t@XVjbdkYR(E z0dWm;vB0(kn#pP(RN*-UQ30FQ@8|HFqT!^jI_7V(pUFR-p#?L*W~L6UdS?)xRJ~v8 z*yj~6lF|+wixUX14b3dCi(A1WAX(c&v~7oJn(n4VkFJDyoVG%q(Y8UTzN5@$2liDXk3A7Qbmo+iE{K_eU_sE?xUkOC7BD8Tt(=NQLd)1F%bLbAtuDEmC z&bbplUR)s&p{O@Wn3f(qxgqLCd|SJ%sGpwRnM89s?+mP*d!g&rJR*;I`Ow%dze!}* z9#o=F@o1RlH_}}$u44o=h14#)WzMYHZ&lL;b?BU4|7mvkTUIY>Ms$CWHEQ?u?Ki{8 z7V+v9L?D0uYO0B)Iv~^(R z*uu1`%(}GPLS1E3uD;dM)NW($%!rar4MRO+AX|sF+{Si9%ai~7$MgPQj+{NiK!o&w zdC)8_#fy`(Cy0;hXqCaCT2mog#ojQgTbItfeRt>DobOfHJq*X~5hme3{GoX+SR(bi zvrjIC1mGsC%r}EB-&Q+O`|*}lQP%BFA)gBU*WZ3ghx*z=cP(}BkqhHcEZN_&9H!?AvNNj4b zn}~?Frm^xpmR^6=?30wk7wCOBS7@Fe_X(*zCHLaf{F_lKjGoXS=EmvU@!LC^SMcfA z7kByQi17o0oU{5)%l@#h`iTRa0g*dlT0JNGX6X`J0Vpu7Bz zYmTjR=)3TqyKc?#1NB20hzQn9c>T=>u+3->%4t9kl!V0-b2t%Xl?x{L7xcYJ&C$!&1uEC1*1V`v+0Y}q#qqBGNXD37X}JF|3h zn*iz95aTs`=H^&*;W_8zKG`qNfXuOW;-H9k{9xj-YN%KlS^Bu6q5{1mBV0hOp|;oE*oL zSA<@HRF78zaw)vb4-LCKxPwESdSSg*4r+P)0x6q0u|MqLS|V13M*>G^Ty|AuX5eO2t2^WiTnoA=5I%xGmc3gb)n6%UNddWmENC>b5eRb^<~ku3w4mX5An7GBYFCio zMB^j$^Jk;Db@JM-7+FC%bfXWuRNo#=vHqx~)ctx|6%p5FeY2B6AnUPho(wH>i(xhR zO;(+q!o;w#)@#f{u~?XTM>gGy5fJy) zEZh&|vvcOc6NgH+b2nnwPe|FROEgvHe3^R}PKPQusvY*~-rkrJ^3H;qg4&YK*3Od5 z6*d%F^@EiMSKKdyfLqqF(<@!^kT3 zt?8jlrjX{S7Q{AajGmjjdmi4j^*0CO3CnyrR@*8=BRMW+n_V?3CWWtU>KfHo782fE z%NseX7!XC2RX3K2jphzdk?M;win%`FXV0So7)mC1L~XrsI4Rk9q@pr~tWNToOJJ3U z=PT^vO+#Cu5sQbL+PO5ax(`<1JNB+aQd7L-3S1H-N>t0UGsqRDVz=$Yj2tgN$85BW z7{fqaX23X}3_aUeSy)+`*Vs65%xOokdYrAC)&HXH#?seDD&hxHJS?s*fR_}Ro(B@leW!Ucf~U4%tJS9N_Y*k zOt9hGSP6i#Qdye|cjf45tFjJz5k?svwVZSq&u~CHDCI`(43$OuK*F9tAF6oWA=WLe z9+Xj>820FOc%HD~g(x>Nud>fCH?v>LJyC=@i_=bLk!S;Jd&Ate=mJetTiV!I*fb)L zykd*A7SqW3)r5uy7h1%P#*o7Xo8N@hANY7^(UPD?qsl8%|7t`zR<_K0&`zWl-F8O)oDp~BFrBtBTy zKF;fs^|r&@2kfv1%zE)S7QN>X?eekE&{)-y4k@TE>8LNM$zh`3G~m@1r9_J_yep*h zJd|!kwY55dQG7Wso=mebq@;p1kvguy$jJk%G^ugGYmN4HXiTcKTC^JC{4fhMU1rvw zzYMLm5YIJ;jnz?Gee34d?h#=sJll&?Q=I;@GiG?H252GvKY~lQDRI4 z%8sesZ^mi8Q%J1C;0Ph1VBKiQ&SB8-%)BgF`TIeP*u7;n9meX8*=5d5RMn_6DMrv> z`K>w0UXJTDF3kK{o#5)!o&{V=)apzzDw>~V5%oB*)>@j9mkL6)7@C<}mT&88i_JNB z%&A1)0bNFIv94J0qr+i+*mq5gWMtxBHtP;37Ca(2stpSi0u4DY2*d~R4`Py=Du?@b z18IP2>c(?(2lv*+84ytYOzz8)a@T}~yT#VHpd zWVJ_#r?R6ZT#;lMa5;|%y9E-gWIGcv3AWnUJ)p6xmk=60V}CzGOzRBplWqfXOr^e@$uJ1|6kg^VK4Q>Iqq1eO zM!th`W}wiNSDKq!+Oc7hEwET=53J{S*Wnfu<%sDpL*KCRjJ9uZP^&l#j87ezFzcvK z;l_oz{=23@=n8+Dpj@^SFoY#AW`<%+5Lap7P8?<%(*=zZ2G+t=mHfxYZqk29&kd>V&RK(Z6th z^@S9rGFf)WrH2O9D9QbnMX!de0bH0DaN=Qcq<<2khYQAN7};E1j#Uu%n+|_gR+iLa zJ&&dykK5OS)Zk8EJ*E}Q>&GwTSGdFaiDmK)w+jVVREOC#Lf1710cz#DmgT((3I(sm zYK37)B!h)qhSL+7d`u$}A=^4XPdf=wZBjmFh6TU0n5J3p<8{rR|D2sMbWB^kGleRb z&lWOdrZj^CjzwyV6Fy;07^PwUi4mCEYa-fS{1caRm$0O6BPXgp;{#(~l4ni~DL018 zp0wprm2*W(UxXNl`Nh2hi+%wmZA%YvDUnzwWR1H6he@)`#C)(aBHn3aXli6usb&kE zl6h$tySNr2P$z|jcL2L~d=D~gEGbj^+-j99>V0ATYkYz6+@K6)wcU7Pegn24bXypQ zDm0JdjS~A%qj;Z17Dc5(v?TlzKtS_&(#AC=PWztam^3_8zJOplT^RHR*f+X zk)!AA>e|-}5||LD2`cn3FoIaM88CsRXbvGVRNFd7?9>*RF0eOk> zw;-`u?~CCoPXN>ixUN;=`uN=fw{9UYujvJl7l6XYWY6>jaIPR%y?8AN``S3t=OpRm z986D$oD<+bqN2~Z(b*4mi)p}gEdSDFr4J=;L+{h!`h!$-POOx#XI7#dOnH%#QsB!P z=%j{VgMsNFqD4V{P^<&Pzv$RkSpXD~zy<3&5E~}zcb7b@xNUf`a6`9qZU z=?d`-0c7;#BB~id^MdrVC+hNU{2{*R9DB|B?TxX*!2`ekCNI+^;rgi?*r;VKgCR2| zryW>1+mPf|9$P9kLe1psziXh$a6ywQn;e_k1VVtQ>!%@xB)0a0^ROavW+8;Mr%jiH zD1z-VfvD)wzv4{^rYK<`r*1`VPWCAcX=A)%aj;Oi~wogW9n+H1Bu^+moV}tzynCq?AZQSiQ(E2(i zGF5BDDQQ5y(h_7@BT9k+d8EP++R8VE7)1Ew^z1asG4co;=gOJ}U@Fke)it{_HaPf$ zE_26BK!dp{VvCB{w4ZC@Km_*FU=%JCi}8_1u7cpH&piD@M^$zDMsGu!gtkN_4Bz+6 z3#O`VSA$OOlG0*;>WnR+g?^u+LR|@`VKxLnbttz1HaQu!01490+JZx>hBC|O=eoB| z#AV6s7P65Hi1p~sB$jMI{Z4}BVuLk^QMY0Evwr=F-bhKi9Z%=yoCr~RZ=72d@<1!(8quPhe^*}fSl zR3D}~ULAZ;qrPM|J!%+Zp&q?c-p@X~6eDQi(Gjg{+pVTwN>^~pD>v;dRpp;tf1aHUR!dvbcKOw z7X@rh$-Dkq8}pFw)2pVU|{^9BzZC+TAsnFIvYXd&btZ8L_JX z=!iUx*2QK79971Bb*t9Lu{^&hQ#sXpj|&P zR`401P*bqv{KiI_ybrHd8?XZd`7T6znzj=f4UQiz9UGIm?Eqv%aeFrM9?*>*q7crk z_UdVMWsp&~&_aNi($s7bJ$gxQ<`SGP7LqFAN($*-MW1hY+XiusBT>zwq5XQO|5(fq z0=zR8rjShwe+X%8zQ($e8kC)>#zAK zS{{FUb|{!n8hF~eboi2lcHvTmoL(RUgD45P8ld#hUqqo8!p_ha@Vx^I>$?ztYx?YG zj!@MgG~y+0=>65D4X>4!_A1QFQ)rgc%_A7envj};VgUI%{vc4q3X(Bgz>v}4G-5UT zOhE{*uQ5R@&`OL@RA(cTbfDn(q-=yiuvS5^4j96kf^rz@3%C-DgFDOvkC3;B*msN9 z1#TUHopSBNCh&#S31y}81>PyhVs22jqCT-y?;H!f(r^JjVXe~?`UddK;`V<>yfUA- z@1PW)0dOS$Lq^ zQp#y-5LnL*K1E3KCgi+*Y#q%^Q|(xiPdL5(l)e8v5z5(RAe)UIXc1{__5x^OTmxl) zySD~Sy*(!PI+Xl{oKKD2gpcm6E;vvu_c={^>=7SvF;knvopGUrC^n~n+S*Bl3hP5W z=6h_Gc>aolm@NscVPW^F!Ey4A`^CotqSjH<11q&8Ym=24VCO{uJ}B)IWZe&6K{aSr zX4z{yjRZFZrHH4ZgzT%(KW;{)@AAA&$&UEmQ+*SBBnY{Izw%<7ye4Isf{_fYL=i;3 zfp-ac=WpPj8xJp7ze(cd~?Gx3l;u+CrW%G@xN`K>|d~Sc3zh!>Rmm1w| z(?j==QiJY-w-QZVT3#g>QkJDQ)Jg_n5)GXE2{Jo;8DXS%K_?B~-^ow8cVc4%{4l<9 z2=7)ja3vZ1?#8dv#SOWI3-$q2BF*a!&?cYPrI@$H3D~@E3wBFA^}!19?xq|6rhA|L z*avehuoRr%iS6e(euWXk>%Prj>MXGYNHBhbnW znZ}UUNb96%o`-~ME`TkQ6&P<$NZ_>NW~A%&Ad{GI6-;5u4{NSP#X@U%K?DKTcgw?t z#kkzm%}X(S(Q^kSR!_5wnO_$KvA7YN!lj|+0uR+FKi%hf%Ln*PT`cKCX(RK0x?Fos zc#>}8#9L4$iqlzui-j(s60#6S!}(2d1-Vs#nn{;SXh~bT&uB!To&`Y@K-Ko$y16IEVT=7#n)P`|>W1mLp~eVWB4vguE$D z5;RW)kKDPS2gDIdYG|c!=;H=C{8rj|%zn4X&=5I)7&nCh?=>Ud=M#8iNkC+{b8bl)r#gE`Z5D{fBt)LC$Oa(Y1NZ1EmB#&c4X zf?m7ax2ZiO=e)=;JQV`ThcL$|H_Zy1gfBDc!NN_fB}vT8G6B^L#n{AZSikW51}vI#oN`cE z?ro^9iV5;I@PL5-TK>YW&}ZS7sXIT)c5-)It%d#@vPesruE=s^e1+~S&?p7L5I~yM ze37I`gb!k;*~u{)EU}MpiZ!Db0PbQ#{o&eEo3Wu4=|{|#(((AFpHxR37I_NDt6b+A z_V|V<`nBSEw$*F>^O~ocEeHL%ZD^FZFDsVhkG4zdkWcy$4wjBw8B}$4l){rIA=PIf zG?Z?F{{Ez3d@G6;#L@Qj4Y~1_;_kmCGMJ&Qqq*vK`!h-Wwi^@ zajd=@H4N!85GBG`N2<4N8ni}Y2EQ)`Y~YRYZfcU3G{ac#l1>VofV2b2mpa`!t^S)O zh7<>?3UOQu-0Q4X+)Le)y6X?`rj`^3cTIJUj&-e@DasO`w zyk6B_25Rl=q}t-CiX7TE_SJ^sYxc5;zqydTIA*x`AiNLrU`PcK_wvqiMaI3hO*P|y z?lN(*0jQfG-iL+PTOqw`rDg~2r%%3-6heOu#yH@H)lf~FMO<~P8 z)bK0JrLU{nchgM-(3b*92i7OV7cmBl{Q-9>ci_`I-52i>&iFfiiZ|t*lz~@(9qmVc z#;9g2Yf>xOJ!Fc}xMrM7@}y(R(YQzAq|&%&WNaB(q=sA2y_O3kR(F3WS#uDFF4hy7 zf6Bnyfg_>LuT3jiwC?7oMeYn$DTo_C?1?q{oXtf46gHequaHLk4U|%eYsV~sq#uh41N7-a zJ$D^)qc1|<>wu+8`bNHID4Y9%ltSRf<%|oMqJd&unN6~elW(a_f^=a>in0xbH+Dh) zTuuFL4D^YuN#hTxG^hC2D!+ehAN4^V%nboc*9BixhMY$Y4cu{#amFn#^w)7!4Vo2y zm$O7X(~PYPNo|h3sXB3D>N5EHo#L3 zNk*GquWmF+!{&r;xfvd(~vfUeH^j z{wYd2u3WII-Ix{8pm^akq0LIJmB^*GEy}vbIbgUMvSx`C`pc~zz;6O7nkv}I&)>4=uN(`oweT!a|pmu$8 zD&2)@Ke5I>@7a&7)Z^Dm%AfnWQ##Q0QyZqNQ&7H3{e{Sy0Z54XzS<$4A*u-FcdXl1 z!-8vQ6Hz*i=&XWAxZlOF0f^|<6id=Kl=c>6}|&$u5{70f%A z%sT5XTQS+@n?!Lhp9SQnurLaxkJ<1IK2hi5s8Qah2gHO>YmR$JAF8%og@>j{vc2Pw^8vH9c4teV>kL3)T-t4D*Q1>cgQK6i+L~ge zi}^ad{IEm4!kk1`bMe8vLVKd%%bxXJym&KPU_=rw7EfKkF_ITs`TeW1dazd}X3;nM z&6{%TvID^XQG~w?O(Y(hj=Q9#fABo70eaZbcwYtmDkXkBtWWLj2bQE!eQMl`c&)Ev z6%7wzjF;RD?Nb(>K2Yjcu4UCD22zL^(f3DaER0hUt&?l7JSp?D6E}v4QE%Dw5nYdV zPl8i3I6RIBR>m&I*k?ar|23)d0I!`s^S4f6<}VfaKN5lePl3Y!P6Q?^Xj>xw5`l?U zo#I;7OB*Zy=GR~oBqrr@A(8nv!H1auQ(ubV7}l{9AQsivv!#6%ay9vdBz*mNCAtuY z-puq*c06yUzV^009#>;A0XX+ynCyFF{=!`taX;D74-vwp2vCKp8UIFq+Fi!a#C-O} zT}DVLTsh*osF`Go6DMGsmAjla%u1E}J&Wlb^pFIPpW^agNa!~#UQ_ZncDVGxUmBBe zP&ruW)$_0K#Mx8Srd8@a!B#83_rwe5Lx!AG3Hj2g zgd4g&|FEOC{Cr7mg`u+^(6SngOn(YxETH;q?{Ho3P`Jg4T|e`_#&m9{@o#akO+6@S znwJ3IBR-_>blz*oNAehgd-M!I{Fzdojd`fYgB|MWga3G&HW9 zi;~y10cAEw&f^z83c$oM6Pkir>@@}0rOX62o+L7MUcGl;-%Zb_EUE`N;>^D1I8A%f zSs86ls&hVW5oH0-27I&r!lUN1%Llt*v*7nn6R~~>oynllWs4c~CaSI8RYw}HK`94@=TD0a7m+}XblN1|Gux!&VsEbTkD)xy?};*;+9u`+yQ zraet>bq$CAs!^ezTylB>MQ@MhJKdbpGF=@Xacp)bF>en~z*%1%9|2fzYX?od{6yh6 z?2+K^&-OXE0^l4T@pEu02MgI|$Z$`ZWA6ZYCMvi`G^8K>c}Gg$+QMYR*zgbR;YmUG z{Eilxgk49C2IZguVeajS;3Y?zb%v9;8^{+Ed{oq+S(n4EA}}f_CD z{EXSJL>wF#@-u02cQ<|uuHD}Os;??pM-`J}JXTRnF(yfhPCc@eXIR*v(5)=JBM3I+d3dQ>aJ7hU&E}U4(nxiH*)2ogxg$G9+KH)@> zC6;8&LBr=F*?}->OsMV08|cKTOA#?E@;PgE%JLa{tQTS=(z{a>?T9bIA{SPoKSN{G!vFDtzwmM5wtbkvx|lXE0mxFvOC(MecYh^b14G?@X7 zt5I>ED;}B2MZyObVc-j|5FVCyYh)~N%NL-igW@@|^%%)tCR7{dDxu&N@#ZF8CHk+s zL7>5ka!}+OJvwVvz)jFIUn~|(@I&zYvT$sO{oEZivi3^iXZZ%sk7d+6}F z$#?93guHYHQ2!>}33E{%utVq$JQDO!?Nh#0gw@uk)iQpt5_ut>5_>_N3VUe|xYFV6 zcXl!O--^HS_~7k_x~UArx;+_+Yz3PVMcFlGp1;^VBJOl&XUg07MC6LUvHRd3q$2nX zOZAS}3Vo^!+}?RY`0ky=z9ok7O1?pTQ|SK+BBN6c!6n}~b4+1?5x)I>CEifKsr8(N zYf$FmrFr48qub2zbLO=2d4|PCrs51t5MG#5lB2!pRsE-D9^y4R$yAH#1vtOVbJbgO z6eb@8#KRDl`mEC8C&{m^q%Gi09xX~}Ea5wVE;l7eTH$}T${HBf3#KY!ql}WF?0c~! zDf+vCYNn(uSwMze3^n_mR&%QT6J)&8$DBx+c1*~hoff?^^h=s_w^KaFB^VNmKuic$ zJw7%*L2Lxl*}IF5uM%gR(a7hO?KSWYHy}nHT*detSwv_MOEr9W*Ra-+30wavyf9a7 z^aMZF1F0a@U$M-hxYin=J)|{g=Now@bDrUz!31&l%)$E3+Oe^nU2k5%!9uk4 zq&Yg^cbsa4s4Jp8m`s2djCMAg_U+Q@N%NB;9bV0dCzb;oF?3byY2nyXD*pTsYDHLR zoSh)UkkjX!{zTNX*_dXxp9&xF5$XZ4|k1p>$%Rit%Zl7FID^n)jzQgWl8&qYP+8R=)9zf)! z$aM)mkj9Lu#MhnH+UCOG9(dt@Kwjh(=Zxt&&P*!JdqcRc23}>@`KY|F)}#W0@Bloe?u!(gF~>h^U;@Fs!pbr zg+ADP#Wx>5iP}*CF}DTJmX~UceL{zKnFMpQrgofAW_D&A%TVnKTtOc>!%c7&v-<;} zf`GI_O~jVK&QJ4_v7a{6M_xA8w1ewWUtDtd2cc1r0c}SfxbAp@tnM<+t0Vp;Pbo_8 zMR`ERIJOD&HRb+QJ;rA-5XUj#1+@|acHZm78X%UcPvhywoL_Tz16*P~m`E9fD@Yg0 z9$KclB1$QF(=f4eC7m&s{5m_r;BS~We~CEi5K6QFUS;oeoy*Z*!)TN;ZC!5XKrbz3w`c=O8F@=vQ- zbipFkFfLhGN_z~ilQJUy^7r9J)^Ak~`f>=O?6WC-lH+-~M%HZgEpw{56bJu*~9 z1=NySx1!lI!?;7p-{%Td7FMQ4!iaa8+cqYILuMQlcS;~ufnp0}w!~?Klx^t+R94dL|PZwP%Z1 zcLioS30GPm2y7XYu~R$+p*fO{Ib+|_3ND@Z9*8Kgfe;!5A z=+BVw)L}k2Iurp)KHZZlNk?~cdcql=C5A!)618*i;kZX~JO%9LWN2)%LW|6jkfb9! z+j<>yse6z%b7Nt3m(dycSaLfmGueBOMVF%ViWPU^W`_xVlLGCCYZTHe3gcCmMN!z) zeP8;sSAG`JB)LxAf(WIu6h$#2Jq*psJgXdt@n8egw@8^AGp3>ea48wB&?$jrhH<&M zwfv^xnbcIxC{Fd*WTiNJMij5Qx&+NE&2Rymw0k^nHTm1hbfDa9k%M( z__O~$7nj{VrbKOHq^^Q#iRZW3OM;)kP_1ab^KujEGj7R1O)5m zaUJ0$FLLY+qkHK{Z%93U%#q;XAh>;^_%}=ZX#qmi(_5Z3x9zj<--nm10I~OZ(V#X} z`7t@|HW`+s;hWN1hGt-acX~|e!GRFKm<&#zS`^d8IS)?wR1p9n&3c~}C$VrkGEx5Y zRpRWDcG3b%xi?C%Hhy8db`kCL>zqv zx|1diZM~RK<9AHWxMYnAbm_P*V+*(*WrXq*!#wLHy3#CZj_e77Q^lwwR=E%+FhSor zd3D9KNj7&g8PLozJmAbq8P%WJC7wxfKSm>SvmWw@UYxCTOrwmM#vnxa$y4 z6YpY-W4}VBX=)anXh^EjXb!ko1LG#VpdYUeH$UE0^}4UBc99M-Ah-SOJ@XiBS0|n$ zYSXQ z9M5wwu@p4(L`Mgi+>fl&EYE)*v#+`U7H&BK0ohD9g;fP*2l^Q5TnHm`bvY%m$o^#( zV^F=7`FNj($g*fju7IxS4WhXs{iiB@o;qhBS^T_vTiDHBo?I@Y(6k&fbWEi;WKUR| zSWhz8i3o#!nJLH{`J7JfYC0!nh~dZgJ-0osqlmmk1oC8jh%yk(UJbkKHk#11yj4ng z?9E@{bFUj^RU>�XWaK5(R^?(6&)CeK{K?wvRpL!GsA~wN)fF`}VY3hGaVY?sDw< zV(L1%oBKvWkOv`SW-z0%GR>&4&F@BQYWzbw+M~|(h{D*i`W0>?av!!B*qdeNWcw;c zEAfl*s3m?_+m{V;Awh?e)f8yS!6dMsW2iy9Ybc2D;8yyGtbJpF4A2MSvwb}TF5z#e=%r<^neZlv^x+dn`AWU` z_7KLf5#jZ5$a!1`Pe2yVtxo<&>3_CT@;K-0Qm?YEsmcG&e#4P%XJPdExx^$vFGN5y zFw7w`s7Zbok?|B3pHnazRoNJXFkfO_QT+zP9pF}lFqMS*fvuP%|GVhm2#LlO3kLvT zR_1?O)c^DE^B)0_VNM9IMBx`(67fY6UD*u*t9GO!qH1L1I6pJ7K+p`N3?wQX;->sR z8%0Fq0hham!bA-5g`n;EqC-nPC+3Aw2LxG9MwRfsLB*8`r3(A zVbG01TDDrcI;zOa@6{xC=Okd#x(c&p7UqkI#m?nX)`kn%)(`h|rvPAUJ+}G8jL1u4 z=ZNerb6t&9l~O=?+}5yl$u*bh5nkh|bb5P|u4TcomZE1=@upU9a|0guCB8FVRacGg z){OIfy%zW_vWM?6kEt#~U5FTFN7-tDi5K|g#U&0zZJo&JMFz749a_v!fdEjx4X4;jdLb$~UMfATK!ix~N?u1ObTK+7i!%IA$_$2u{I!-0iv?K1c}~KTR#JPKb2>^MIgOpqIfY^ zOD>r}?nOXe$ll(V{N7=_K9k=5viSy871s0|X+2?ueuozIDBK~WKg&>l2@Ul?9r;As zg5LQsahV&Au*>$1dIs8c$U|IHTBai_d5BhT&s_YkQ_?r-oWBk-}HOGt@K z$Nze#4@$RJ1QL^-a`uV|`uJc^Nm*LT+zUh?p>y!KY3OSL5G#f2gl5Z7K`Fc&MalW+ z(F2DTEF-uhZ6#13Wlf+j`X4hQG^1>0UoNlZ&AkxO7i2)cu}@(O zRX9riSCkxt5$#5h)s)l(j}^_eaLL4(6EnT8bY`dmk(>^F`{c^f#Y!xYopXReURBZD zM)T}mlg{k1U9~MDM9awBrPTu~jdQD=^nv;ggpOAV-KyG6STJbcbQ|n2D;Ia;^ucX& z6SY~b8=lp3F?x%NfIdIsmX#h~$_RlFkpyw*wrOrfHv?iAkuQi$F4JkmbMPlD_I&UO zOLIv31sh;s?q5fZx7A4bv-{;KJop7qb}8NY&GWAU1$jU=7Xcr3#|BZI9z+wxKTaax zK&0oDU$)+^O*LBtWiG>hd;9y>%}-rlx1=M&J{~sAsK)eEp#MVX&L4hpf%YlVOC+oi zmd;-{Pk8SAh7M1?JFsS;KQT6QJJ%GGXyHY^b9&>++qUHrHx(bh7qyX*A;+z9X^%JA z<5zIW;GMs|xTAv#iEV`RcLMXOX#c<~1*O6~bZK!4;^Yd;y;~z>4c7xxTHabdfGMz) z?dS99SnMimYZH}B;Hq0Eu+lkz(u^LAxw5vnfkUZ0PL*a4!260SO<+fc9`AdWf*tSJ zP<}PtnjZ#i3x>WsevZzHuKNTE6Vp6|{PAB`kS{G-at*W{Kxd(@tw~!MDqHkl79E)VnfQ=Np(AfSRfQgG7CKnMFqR&NdA7 zD;ADya(VUAL;II!S_&QM;KUyf`{-?leOti@15H-P#X8X|q+dw7A?ln)L5ZJ)Yc=&4 zkr}3KN82!vLEAxe(Q$sjDTZBGZF$yLa6Lo#+l00{lcUW(I+H3hy@wO=(P?jP?tx-d zs)W}(V*j*UTP)teofVH}&}>Y^$x;eAh9aZ^)r7WiVGR*-#Q0#FlcYQ-?B@)1>S2sH zkD3$xolPk84mX~vw=JsNEG;x4n2mObedB1Om45f;Jhm0KzJ$XOOP?Yi;=ElD5$SLa zstoCk0YW6jFtymsOw<8~Oy9z+yo{I2>W)0p%MLc3ZT1S`Nw^60Aipn4)OtOkN^5)+ zJg*vlhmuh)gE;IC>82RJCoxP>!lN|hQWVRXTBgU?JS2htSja%KH?*(}HeJ^#5LTC1 z=oghmW*VjFa312xn#Vw#?CRCUaF{{um1sw-DS^uD;k;-u{yaKe0w5l>jy*n)HjdaS zZ*P%?9Nl@e&0TC&CIxfdQs{9I&0MmDu9LQoig`JXy{l}T22-Gb!Xp^~LVl9QQs5#7 zW?;w8&lbME0@zp~;c&q)tV90uN^_B&uM^!Yz32XI+~#^sFPTn`%B{hX z(|QFzn}R~7o_T~eo8OWW{eFnht*QD0B+CpLWHMRFZbeF#yff z5>%|9fp?#cNP6#Ny{ z9;<2r6XKOIx=DG9-dWWvvs(iF0qb4e%XM1~a??kL3LxpM>HT#}1?~+;@B>+QmGX-z z1*C?{>{@ZBo-tG2{sJia&6u7MlmSpY%TJ7hlI$b0S13E9(;h#VgH|3Eo;-v5XLe1_ zU)7*<#!Sv$ErW8VPI5&U=OaL{4wQptj?jZ<4o=8D|MIHL%qX-*(aEObeO1|V*~qpA zBvE-1^6Jdc$+jlYg^%&e;=52hc`_i)m)c^uT(~9lryvFvsn$ZgnIGeWN6R^2X;jyO z6Wh??)SS>j>kOw{yaqwr98sBm3b}jN_0#R8P?xOv~-gw4TA%?(%hO?hz7|hp4K~47lUV zxh7(F!Mf;z&beZZq?#JSO4{2qV>d?OX8P*%pZp?ahBsPC=h zA_7ugn_#Cg_rmM~2MC|=WdX+CS^d){81$*x3Vi6n{g`{C){H?qGVd%oQ%V51z%Su$ zbhc}fMVttDG;b7~n%j(58t4fVaOj>SDOmveen+;C(4G#7l7`5JJ5G<*%uVh! z7g~3c&B8-R=^abZ&XmpE!`0$y-mM3q3%VypMnRS7fxI%EMkzIe_1N3LEumnI)2neM zso!Pq?hv83>2;hRmM=*k=B z= z7bYJ2Yw(fEgh_CNv6AnoeQ*Hn1Jx4PBRNDhikA#?2tVg1FS>1-hHCl}H$C#CYb>Bj){6_Q{%^9kBLTn<_iMOoA%}bG9=|&0e2K*32V5vWzjr zgB-U~)|Mb!_N$GYWohhR0HJ|8DP?gy)%^9TAqw*Q;AgHL@;|L~by%(}(Xe27nzuJY zqWweF0B`2PKLfvRi(}4&sg*-XfBxv{IM-~1MB!7(a^xh0`BvWVEm|@}cZ3_-ljts^bLMT#41MDRU~-{T zq&SWRM|tC;OO!qmw5eFvR|8$1yg;)@3_y`}T$tBWsw_)$A{a?Y;7+mDphzWb9XE4fi!FnTDVIxA5*PKt(%?#J$#< zhV-}3c`V07tPy{fRk3l2MSi5!LD;^hq%bEBN=bH4Pfwm8^Lh+k1`vhW-A5tQNWqa@ ziGv_{*brl%OS4NANxv{^6g1>b9Y3*hVL+l}KD(m1xSxtqdhsw^ss(sL;bID4kkKT5 z9YCxM&Js3#6k|y3; z3*v=^;B>-MGJ;Y0B?>cwYik_fmL*VS+aSxUB0}#~VHN(f5~_Dx^p@{If6;eYuy1+e zN`NqF7?3vpskKOjMeg}0_F-Q%Wz>OaQw@>vL#!flAT?7IP>w1W-70bQ0SR50Tg?Tb zh`tHxCfPU$0_=wmS%s&x%AamC&@2PcTJN`)&gjX zW+gBGaBkG?2I6Unv9jmVXT#D`n1b)D+cadO+LAg4`aC`M49SqJpf42@Y4Rs!p~0v+ zSCPYe+$iRA+ zi}18YmYm4(Ol{!VHvH<|B?39+uUast@` z@SAr;Eqn;P5p68$J_ic#*y>JbTyZvdk*^30-pOtiK9=VXB1(7sT!cRHKlm4W0$+qa zfj`6-dx8;!K5+si=>N)J#4Nof5CeLK+hxojGQUO{4Wor0rJGeT>}X313W*;6sE*b zt6<4km=Y}HIxzXP2gC?7x=cm$@JEdnadT^GbFDT5nSo3fBJ z=KG%FCFMH2=s+Oy0HWjpWVLXIL@qg>p0v$ijQs>+DhKsVfc?tv#bsnlkR=4Ckeq*O zAek^Qz5Y+UL4#qW5a0ej6#$RAm|MCcA?(4NUMq7ePv8o>4Z2qh;F8c;uOvXOmVoBqG@>b$h&iSy$Ae*@-(uY`yC zpf^k)t%{@vtQA!&gLfqxW8?>Z?N(ZPkOy8jn?iM3zbZQc{?gv~aq6(8S}K z81-nfCc~r-u>FE>q$dx7d4@jDir)}oINn;-NZ|_te_8}-QeCf??Ng#uu27^FFUn^X z?hn)F1wa11OrbhG=CAd@J6LVR8J2Z{*a6a8J*KBPn;t1qM@sTptBM~g3=w6jc&p!J z8r(a@7r!J3Z5p@U9tet$p4ph_XMUz-epRi+` z@MXK{ndbP8JtQW>#!5T?w9nT{?$w)~GpF!-4l8~R3yky6GN}P_yZ?ZxJ(V7}WWQB< zV1~$(vMcf!#0MAgIq$BOE$T%}Ou&`I zs`_%26B|rKdt{SrPdTpJzZG=tDC(Fqr6XJrc|_kWom{2%)Jo(vyeMWaYi35L{^8Wg zV@@;8+V(?^R!7KnOx)t>B;_ixcU-`mUea3NbNN=`R*IS1DSj);P;&LnYvn(qQ@Er& zP@UfbcxkY&Ax5IC$_4c!!BXplz3iS`eou@~XW3j$^(e&WTn6ZYeyQenMXqaLW;#OGAI>T)*08iHOww7T;R zy^K-?O?BmPVTt;`B2Lf6#`W6ubOSRe`?vvol5fc|dPkJK4NATW^FYbwf7dhcw6XXl zLsXDL*kJvNIDm{1ON}byqlRh>d9X$#&DoP?Q3C5_PR9(L-;hw==--6J0y9!yrxj zG6`jcx28uC08^<%`zM*R@&y5)@$vMJz5-cl#OrO998r~lAVnPqB>`-B-=yM+n7!I0 zJ@mE?OUdTmboy*;*hlzR(o0_@8(w7q#1N3$$uCkzjRrbBfb{+W~gKq2Q3xgv`f zb(`^a{G|0_YK;Yg=D!RQ*2dI=qZp1jjsW8cY@A1F{%=%-*Rq7#X=U8t=S^pEJ37XP zh{-&#&gKgzh?*079IKn94Y4~D#!F2-v~F5;`Yu~VN(}`bz8i-pgWXMLR?ayZ|*lY z789GLPK+%Tw41Hw$EW4j+SHwtUEv36u|Y(Q$SRSX&XqW_(i%+bkQMApnAZx@zdG;jYh0#916+R&3M zLeF^dze7}|hu(2stz1&k3d!C9+Fbf|qVX`kXAw}lRi48P$yNDkAD1UDLDRMSoNc+8 zejlfjplvIzR%=I(BrP0V%*T`I_oQYERXM3^t4tcOwkFUl!Kr|y$)gY9RA6|!O&8s_ zW(TpOfW@8Y@e8yJwpxrn)oaLZbsC3c<0T7o18FsOn6^ z%UblqK3hVw0X}{mQ>hkON3v$Rx*e#JzSB26d3BO@KjNifZ5rMM*RwEZwAkeMaku$Z1wDBiX7~8jO>->Ykas;j%k{QPZl&KLa%$vi zcZc4oDjhiZOe6Mc)>c`QyVrqd`U}n#8Kec8PO{cnLYC@UL_Bi{&K1tQnwlLU%Cz)m zE(56J5Bgd(zE@%l&i0?)r?C(3i?3gUA2lXvIjaHOyOX4}AK-Tu*4U?D-%7AW)LWM} zg+*%fY;qFc-6C4ud~la4sT659?tH}O2EO=Wpv3a;bstZi>T!iNJn}BB=3I&jrnKl@ zOrTsNtFP3lQL4rTI6Pbd=@5m&*2n5fMagCEwd~dEiTG&cN;Y9-%eLzgVD0bGU9>4H z1v!IB~Kz}#@3*7T2QLRPPI z_V4KMiZbS+3+$h=GP&9n{wC383Q{5ViUyd8y}I%KDz4nY!V3CCT&wx%@tXiB%xnFd zE?LnoqxbZ;UQwN-AU0N|u`p3W&VN zr(ZPU_QrjoX@#Ha)4OAqC(9Nt)bfcwq_$dpC{uS_wdC9K;alRUV13SGGF!lxlUx;= zA$Rk~sPG1mp?0}OjE<})=6nV6BQBC9u2vZ=NtC|Qjg~}u)^!3|C;)?OoJ8}a76$Sp zwz7@nNXm!6V(|!v@yp4Pg9s5W7xxd4{^5x~wQCR$x+_I|%D6QF;9kt^=_oQW3IAcL z%MNS#lGL-49)?-!x3nbGC|JY>wY;A6I1qM-bXS}KtkJs#C7cyl zqpq{`Mancc$xVsEq)0kkp?5`^CQ&Wc$(I`zUuVRSSjgd?`31gv>zf}F4Li%&%>7k> z>UK;e`l;DO#}C|YA`)vg!AC!SFnLCRH*B!Zy+@fvy9CgXk1|r#)*|b_w^2~lHeQN9 zTN);hlf&TYDEhwLN6RIK2}#)%a((SIUyXcn;d}jWwy{w{y!DPBe@o4ezvX}QEhyUS zSv%(haw>tI>*Tlg_|%|#EyfGj0nnEA0>_;Y;T3%^8t?j}ts**)`*D|*^zpD~!+ zL;hOaW!cV8Cxu*$n6*H26p?yd#hQ3+KeO-UxRf4 z3G_u4QnN?=$0u3JG9rw@24##wo*R`gg^2zF*^vSCT&rGX#iL%M-l)9D&wZO84<^@F zkkVygW1&&Sb6M8aSS7OaZer}>Y{GU3_&fB$_3(Z3wd3*a@qO)Z!~IMQR2jQ{5}YAN z8JJP)`zs_;nvH#Sh>j6K!IeC5?rNAwag!ab1Z~eTQ!bFnE*ZzffB`gdlzk-1!R|-i zx?lL!*?_y5Wn zL|o#gHmJmg(3qB?!n7noD7mxcM5u68of^R6uGDK}Qp;BNt~$`8LFflDxx4kopLRCj zVWpoF#!okVY}cb{cQrjub8ux}iTbNK;&5`edy*G~q5fJn6?Jjv>22;}0K^3-O9+9p zZUtt$U-*UTih_`#7C~eA2xHh;3%Pz5V13yifhq1}zy=Xf`-=sur5>WIvX7N5*tP?Q zZkQfa5R-YlfK=TV)kb};FBX*h(r~$BvlWL_s6P*I8g92Yl&lW5V4lPq} zWJ6s7QzXZlM`n>?Zeyr^elojjZwNK48)J=)v@jnuyv9j#=kFyo-&fD)&6G49lWnZw zRa4lJf-$Eb<94+7AEFKjTE|-k`SkLtbB&*@h3{)(MTH}o;m5o+jNRE@z^lv)6Qx#1 zfg~1d4Mmq)jw;qAPhZ^ESo&>tYJnjS-bfu8INd&c67%#N|181wn;qML$55ihMMd^P zB3NJUe)1DiVewj!dNbz-P&v|)=v*ngB@X*x?k`ApOK&dM(#(Rsk}hDEoq?oSz(*s= zRe3|1W@}DgS#}p?&W#=ZJnx|wgT5S}ZVPgvj29(}bZ6G3UeACI_dIf-&fKaY`98|c zOp^;EVD>@*P4bf#k%S|`uGAmR3%omoCUHa*QT3b*9fW9%RxQ}gOGvMlQS)3tT5?9Z zOzduDMGSvRn7wXoPI7k3P|ZeO={IL&?nRFnab|_`au+3apDVk?p*s`KO-Vb%ol<9P zt)f6(@we35Dm45zqzN5JOBgQHj(^f)X#w>`@q6){m~@Mwf{(Y)m!;y;ahc&!`REj^uag z+h-ZoH}T?1*O^{yOVj$&k0k!76!x%qR3jYU)P(s9v*T<5E#b z4{@^?oX$FW$ZHOm1@~O9BK|y}0)6}XPy)1qe?hn<^1*j9hR*e_{IM_xkMiKzG8~ zi~$siUzxytqUp#1FrIRKI{P)KuD`aGZdU;Dk#tPGHmIK9K2!%cfTI`aFJK*|+ZX_u z_h*B@&t6GSee*Zb+^r2D-M{8w2dR8SHGx^411XFWfbdZg`7M9D0|@rFsh|FqyX^t+ zV(rM^T6zA`6JiyT(0tGDQ_Y;2i{A259|uMCOu3{2{!1kx3L+6tjYb|X5RW=W(S|A( ztb%F=J%C!z#!c5I2DGkFCYsMf{SzUgoL4kUCz3KL#NwU|^e{3*|1+apsL$xHNHir^ z*f!5CM45C+Htg~;Vk*I2Dj~d5laWl)fP12~UsdiVBMQRI8ZcQ_bP8WH)O%X4an$Z$=v zx0@{YAJ|b7x=}dVtHx|pTCJe2I%D$LJQw#Du8Bi71arprsZ}|z32Gc-No3iZfaN}-ePO+`jc?@g(&3W_ zSEp8$6`q#g^wY)0I#;I@7|vOE3CiX8olaP!w*1kc*o0xLRI+4))VKN7<2xK?H$$F1 z_4=Rj6E&%;Zn$k{zg5+q?(V*uBI}wpjh(r*>WWhfwVRh_R}JvKG&3golCrP>e*9hU ze5;Dn&Ez*L#KKkXfoPmok$ITvXXnDHbv7K)X)e`qtj_fJr;9<`! zX%_IK+fz8+DW|~v4xxz4Td>It_;GjlzFxet?VAaxIV|(d`D4>bnkoOa%oNx3%J|a- zC-)QMes=DBpPXz6CU=0oOaq$Qz%c|4p(y1Xf3o$s+SP`VC5}I!vTauo=olI? z=~Huk^yeHC%h*tLf!2RLyizTy8v7L zYbjl>#&V=;l!!>ULzR2`=LtMLF9LCHj>4xRqD1E&LoY86F9{Wm0ZtnO5tRtKS^1qg zSK9tXWlkxs^$QKqzv9}8ABXvg65IUyHrk8q#1DZE}AjKg%0CWI;nh;Fys?GL(1Ct{N&Skw%M zM3u`Wv`Q(4>+LdFR}yX*rNSJ+Y1mdpJ(yeS8hB=xx0iOUtP2JT*H-Z9p)=$a9vnI; z4!}6qyPjwuEx5a#E=X+|ny~T=7>l&O`2DvM{wTk+pm3v)E{a)JqqP1Zw4Tz+3(O8Y zJ_#x2na@)^@V%|G;R@JOp)YJQ$I3FtAC)*Ar!lK+mEtzm{`&%#nM9{1__3dS z2D|7YqD{aS{?rkAz_Au4J=(q%${GTQOw@B}Y{&7zip)o4q9xteHY4>Nq-)38?VIRx z3)tiAm0GDe~|%f0lx2%FIC|J(=V+EoPDHekq!x=d5Nq&Df*&~0>#>CZ0&Z_P-W2??Ei<-|#}*lh19C|6=yxJ&EUEx~ zNl`dObnj+)0onk4O}-_jI6^30O%?Nvp9fsJ3FHE>Vl@APj@fLbVfq6>U0jC^^5OX9 z;k8Vh1?%V%BSb-h6;-fqUuxM*g90x-lpe)`LU z;N#T8eQGTe^iJnq(gH*~^vUKhS9+lv)HlOYc`6>ls)k)4D2I5hRJsya0w&lV5(!Mb zO?`F_t8In)DGx+xC)fextx0^nYwSBJu+XZaJC#0Vvw`P;U0=a;sIn$j1qFK+mWYtOHBbx6*U8Mkp_*m=X&fg@yM z_H9{*C=^d#5MxSe=P>hBj`LJgKoeOo=nFlTOxzr62G`6Tvmy%4sWLW`Zs{y#nbsrh ztenG?T*C?l29?bQ+;B66^`+HjOPDMqG&8k1|7N6vx3RgpYI;@R+%rP61O9eJ?xPw> zuR&%~P~t6|O$6t%o-DV_TQ*r;$dvf3X?(K)+WTq;byZd6l$&nH2}>I!Uf^7(fwoP7 z*p8F0AuEfBuDq(2lmu**g|L(yZca|Ab59$?ibYNsRrM0CGTe-~HfT3mb~kRSj=Dj2 zSTLmdhtof9IEBFQ_bbZck3@X+S3}cqv&ix_HWEa zf?y#2h^0Qx;M{~=O)&YXUlWx`tG1Y|=)J-M^Wn;SFBy(KJDKuMFdbBTF}>ix@L-IF zFnr2J=EQOgvoITFJ;1Hm>YtWV-k>#@>0STQ+fE-&?ecAzr_>c#N6J6xyB6iDsMsN+ zj>8)k)mT6<;tvU^os@B)S2!W^I+rO>ujbLZMA>Nj%7F`F3~-V-86y)WT!>eJK5HdP zJqesroRfSiDk6sp>m{jdlnZGI%>6)Wwj4G99J1~aS|0*nex zZX2El99cNd>yo|ApqwKnGV&9%1w}y;=QFU-)qto(eOu$qKN;g!T}q^S6lB{yqG`=f zMoVI7qJ<<*EF(w=c1(R+7R`jB;6dJA=EYV5Xrbcv03uxzi#~G^C=_SBU@b_OzNHV% zebKS=LHU!vj)Fjc#H%n7le52JPTN8-USsu) z&}GR4iGMVHKTZ)7lhRU?G%8$nfHiT)cmlvA@zJp*^ciGM-L3NYkDGd}Il*Y-76B`| zY)R=tJRqeKBFMd;?fuxFr?ykqDs2c7W-uCS6wN3)f?Q3 zl>4M_Dqc~IM6Zhd&s9NGJiw&{BCX;SagU!kj`WYF6#C+QW&$aOa~S7xlf05{t zTl9zdMBEH;Ey%Q=6OoSwxZ`b&IUMi}@y;G%E9(h`inXG|FwcdEs9#LAvpC6Ne#$m9 z88qMDv3VFD{aw zjnVE(l)x>Pjg;s;6-~U4K6U&}poC?759m$z32Dzb5rB-$*gni>+X}KfB{NOnGJSIO zTzIcSc(?+EgN8?dqNH}0-g6w>L$@gQYr(A$!_G)Ht)IcRDRfo?LQnYI_BgUMWUA?{ zsEgJG)j3wjWIU>@zH^4&RVE#@B)K1AoNDFDkM#{8S*0*KrFPdqY~My`y?~*yq}z$-%n2Ne*~?o=-L11k0$@;n3a^RWVPITH3{_?~tPFIP=hDoAWzPy8;Dv;z}V!xAHveReJS8 zUVE7~lSSCT${%BVpNADo;!ye{EDE_?_g#-5th zt+Z}+-wrdv3ST6yv_4jy+O$!edfDx?Zk2R&lY4#WOiZD0*k}WzV(yih4pM99s`?P} zUvI@s;x)*XL!k|S_~vp)}qDL=Oq3`8XbHc8)rdaTxU35 z4AQil#;F93%MdjFZf(ZOa0{%_X7&;+jvB0w`NiYtedXE<%pJ65EJMo7P5I~XMW~ia zEfwKL7#8i5tg{13GzrR<8Rlh{Ysq3{)dE8zX@6Ol+RG_OhL+(bSd-fA3Q-|G;wjG1 zA5xUEij`n0F}2Q0>I#VCh3;~VDH?^s&F1ILc#7wKr!qvZ@t@g@*{jV=P+SnsxpYoq#7>gILWRv>TFbH$MG8|>*dpPe z$Rl<9xkbXkxE8kLPmNqn{4IEyZOf~_&<|@s(3cIC{C33M$2g={Is5V#k3xOY(g*zI zzm-;z(pLQw{E=2p!2ds&JL7*Y_rxum4Moi1oP}a1WbWj?^{Rt{t#hj@h*UI8f_I=9 z4lyY_uKFx^X2J=F%B7)(iorADRuO-Xp9^+Nq?w}W`VukXis-@@)}Ti`e8~bSCC7|| z_g|ar%m@4tpU;PPP@ofC6^57~Xk*(>{sI&b3=(n>X=o0T!^YSm%49p?#O$isQ#ii} z{m0_imcUZ}ZFGC-(eT(G`a9L0S1?n!2i5dIlyYrR;F^>X*ZGkhCi!RIvhwmoQUxDf z)B=wCaugLQsofyB?p)Hn(pfsoyRs}ySVGgt!iaxX%RNZW$^sMpX)>LW6)Ds@~PPDjfhvUdY$O|n=$G!ti-#Np(-ZbXHCMnD_KcPfZ=cN zv%~p&mWS_iI@t}!jr!JWtmY$^b*HI=?g^fm+H~51uuo!9wHllfK1OaX&bL1XVW>wf zcpX!CHx9vSD+M3uEuAH#@MtJjAuTK+H*F)l2ks*U9uEf-YhHjw9)z6x$L1V4A&YDm zo{;8KE@6kI2c8mxvvu1(6_$~FsNjPi(U!T=X^Q#$3lJ^RmoDOM)OxDcL1WCkP_hsI z@}x9nZqHef7F=-Fp)iDudHfiCupUj<&ZLfZ#3m{5a;jPt{Yo5~Av{rAD(19<+0@QQ zSP+@kY*2|qYGc1s&(m{en#}ew)%AkcZC!K6iGI37FJY4uwk~()12lRPxbR1Fn4y#Z zi;@o}ff)q<#+Rp6=MxRZWst=Up(x%9r>Kcg8bun797!5Ol{68$5-~ZXHQ=27A&6@~ z{${T?L9!nMssx%_lwC_VN|y2#QLn8x1=7M`i4F#zikI3|@f!v{*MDF!kYfFdNIEWO z7<&5;uZ&GBCx-|Q?JLG=Z^|K*^&ZuI%^h4f$v6DY!LNIQI~41L@5l~7l;)w&I>01!O6myU5qS=mo_*lcgzc)e_=eLU>s_yDi^Z)k%t5bt8qX=2{%f(CQ2 z5BJ%@XCLiyz-PPsIU9nBPHwT#+*dJ5y0f@(#h}&zW{>1@*pGiNQ zBMl|{T5U(06#%m0vmU_2elf(6iDBCpfJp0p$4GluGNGNXg?t;1VYRuX0^Uiw>jZ2= z@5>i?XkWjc5yvdGjOi)B6L2Tm$%}QtF1_;P!jt>|CTs0F0&9r zF-~`>PU%)o4AaFgl#@UW?IYTO& zA#F6~n6XPvXdw(g+b?}7I?v9VH7$un3Of$v)jyrc1ysq1sLq}i+0i5+%~O0lHn3;2 z-0x3&s+4&NV=_hX5Gg`lf0%&W1}HR9#v%zx0=a?JIH>}-(w5F>kt0ctf$GI=Xoj4l z&~iVkh#>)m`K7Xbcy|UNawUmxn^H4@QR#_G-)!voX#BV=#^Gqta2Z*O^r-vTShKDb zCC{IF{3hfU@>Ojbr>Q|?2kmk9{mL`-bkXKS-SYUN=BV(+_t2)n6PGvr<%MME!%eg! zoC%AcceqOkHp1s z%Fo8K{8x!^z!)E_lVYEylV%qW=|;c9_eHQX1_8{ITpz(5%#(QEz>{cS!IM-U5eg~i z00v2Cp9)Fm00&9swkmXhooJs+vq{U|Sak3u(*eVKcQ-vWYL$3b1c`Ug1b9X0!ym^H zj|mjr4&?yE8|@CKI=nT=E?CxWTX=y?UXS1Kq0P3xen{c>LMxo59^~@)CB66VU}@&O zS*(&(D)iVo&-(bRx)j+Ia25cNt&M`uF3|dqLbF5Q{!b z$NgF&_wa=r%*$Gg$fbS}Kf}g=GHb-2>AQu)OV?nuhDA<+h^?}5Hx4VksoF>jph*t&&WzN|lx6977 z;`vjd+?4O)cwqW@h!<&ey&rnM{<3TSop zPDGzFJV+a^uUjx=ea>2zen@+ED=ndD%OK7vT%bQcg&pKy(9Qd03nx=>O?xhINsG;H zdiZa*cgl}!u@5Ti@$h9kfDS)BP*l_M01m*9bEAikckTPF{uYVdF=kTcZa>s;OD$IL3NdiZ%-!*5+X z1{$hJ2*dhH!7XKM5`Q68pLR_ygdwmw&nPNZS>VNc(Hi!~g|~_BVgfy-S$psoefB#- zc7*wZ;SVrd*zDK==`KE7>t>dMs%hpGCIafiZp;1b>1ZDxZnjCWbKOc#V!T^pw|?+cbGG3!18d<#tgEcjOE558p8;O8116 z_s)6k!uADu?&yo=L96MW^2nW=-HzSe4NHLaebelI(2nx(4dT{;qj?3%N%KdAD*PU& z2oWr}H7-r_fcyYl?+cLCqV0vx2<5vnSJgYLUDcHJ+lUF|H^P6ND@)e2?5saV8m^zs z&Hu9~#?(#U-pr6r^gnKV`VNlvKW-Yq9}k6sjg!5B(SOdDxG8Z^2Gk&#%({XDfJ#%< zwPtCSzu*cQpN5d2Eg`z*bRnO@x*%iOYRh~S+D|`B2y#L=k#dCE;WqJ3+Zd0VmxnuW zZB!B>u(7})J%&F1D-aB!CpCmBwCwNED@ck-T0fO&958Pw77o5RvusRn&7GR8;{V0o zJ9lT=FI%HYCms8aZQHhO+qTu=9osfKwv&!+t79h}yOZ?E+E`;e&slr#_xkLc3fo;A>b4 zNc+~~i1kR;`=N5@95mVb!O2NvO~?hAgrO(ET~976^((}Z@ZtMsVRhkYHn8b7WLS!- z^K;V=WZl>r(N#arZaF+nT;K0zKA>Q*I3QcIZ){9_s;Ev?bpcgre~hFyuQh&YKsXM7 zIU%fZHB~0Y3=WsiLw@Hq{9F@z54d9rJ&C&IFSmHch&?lTvj=&ZwoK~=`>Q7or@V9+ zK0U$mUkSzjYfm^^x&M>R^dDZBaT|_}-8Wcj>2N7%5reLw32KQ9-x&{@kjG{nGAB)n z^rQKysdNqMOUZ<#R6Jyi-v073-QLb|MZn+y;$DwV-0K^Ti!Mw{OkhYA5_bqf0}2ze znX5W~0qAlrk-9#Y(m{&UjY>r>iy3(J+C<*OZEcfs&8t(X$-XI`xC~)WY}G8ob1_Hk zh%hGcjEP_2D=F@9_K+)3$)M~Xj+dGl#jz@uTw|z0H-cF}HfO@f$1Dg@`+_s(_cN~9 zwlRq|x~tbqbM4V{O(+}N?khYUo#^UYWxtW5P1n_shhB&<_2gXl&ffCs`dNf?1+B&l zL3KInyW8(j(9LVX8WwkiUp1yAYnW<~`DXy~++9BN>^Lk@f#fH#?*1RHiy%peSO=XD zol6vdH9ztXertozmIne31cdEh65*6HkhtlZCiY1c$=Rk89G7tx42 zB3!w0GFDlouBdoC3)w#pvbR4xT&C^)-`*badcX1>bH>nAf66|o>khbmRlx<_RCG)# z(puNSrE9C+L4Ztqq?bTifFZ{HH9zg>YN+0aJ7Q#Y<>YFt-}gnLwJ8c$g6x=o#L21% zw_8*BI-p2lJq#ulYki)99pL0Aw~95uj0I8Dk)pm0weiWf_xpuOHNYe)u9uw=i<(`E zndyL8y1QwNiLkUwpl<>tS{3q1Z?!AnA}%aZ*5)NMMsS<&cg5TaE6iHQk(0ABSB~Lo zF(NM;UyX<^I7_8d9dHflE;wZ!7e-dRq$v}fK5a6qs1G;G6!s`i40VKWm6l^w;P0%C zqQlBms_UOsiZ>u-whehy*DCDg`MlKazK_{KGm^V9DH^$auUAeV^kbpHtVFbMu$f-! zW!EEg$}$#xh1pn&Gt-tu^9K>_owv?IxDpB2p#`L}IDv|~^MXDwsiySwC7lUDXR{XHHS(%5u(P*XZQ9829DTO=_XHGMpod#jfeM)N@9W3BROW`A9DsGwS{lK z?#QGqe?}0MES~l+2<6MCWAb!0{tO?riRXE}-$P%rSbYU05CR|O>ThZU(d7tSEVoW634Wf` z$>awWol_3O;I7$_ZfG1d9nK0DXf>gPbXJ4u_j7(;a{&L@#!lSSSskn&B<)fzbmF*= zfV^9`!M#VO+dK4yNc6^e$U7>;5FzKhNXr#DUx(eh>jscs)to8efqsq26no_=@5z#`n1@7(3UtcVWiVeKSXTmi2OqdM+5S+qtC z3HjzTUpgsArls_ASsd|9zQO(J`qtG2su`RsgivHvjw$29j=vicIR;g#DqEROR&^GG zgQfioaHrkcMbl03rl?)I1z$oULiOd{(WA_r=uOa@p2%H`qPT@tqJw#pe2``dHc0u- zR%rCHJ$3)HVg;;iBzvr&u5qwE0F&s3)~whnb41DKJm zJHMF~Md3ijL#OCIPVo_YZSmhNr!pIyCt-KBpes;>mToI({!>iBg5V*Wtsh#ABvtWs zw+ceSHx`F!fMEuTM81Vn@u5TMhuh~Fpqwyqznk!>!BWNX#yne2&&Y$&69i1xG-_>{ z^ULTwr3rq}uv5AfhSPMwILpZ|w!83E5!XEh#pFgFN(nz%X7X3hILvcdLu}}SwP$9D z>usnXFb!|h*Hd++*h=&d(0}#ecDnb4(`VLQ`fP3K{?C2*@7Xp*)lLH-f$9$wrO;A` z&S#xM)?t+lN=GYTKyOM3hzwv5wG33qn!-#4>P=sWUKrdUJ{K5N@j<#AG(Ea}Q&qUK z71ob_Zs5ww=yuu8^~r1h^YL`#0Ft!IJR4?alk^kk!FDgMOPSp}4Yl|gTVvXQ zmphmsZ|GJz!>CIO>lWRJbm-@kcFQlK?{QJqqE_GCpr$zgFm8&QCR zIgJeJ(K5>tBn``dKamY>EA24JYzrd;SVd=I4toJjX)daZlo=#Xla9!uGc<=`L=0r8 zWY-7%cnu|+1`#89`jv1I<#_w4MIMuEMCI(;t?*fjwda>A$~N`pco&x_88uN!IIs`2 zFI}*IyI!VyyYRBt1DRso00R3M!1_np=IZ8m9iainjGg*WTgqerK%A<>}H zC?0j%hPjOw$CiB2@VJ^r$XZP)ctUC8lQmGQlK_MzlND z=%hWuF2l2}O$!4esX45u!>w$qbj6u&b{&Y2A-5?fXRg%d+p59$LpT_Rlv)$Sn50Uc z!OC<3PUaQ@fQL|i4-4Z>u5WCs1hWdb7>@N$UE<@fo;PUT!-z;@@Awrv*xh{Vblu0; zzb}XiacZ3kfk_PU6@g({9~$9&5?4ffuI63UVT||SRw3zPA^w7;pQGE!lABQjPPc%C zi8QMPkAT7PL4lKR?XKZ{@Y1)F8;s?{%z`%3aT&lemk>FDh`>fA=AYq0kQE3IN zKv}@U%Pb@#WB37Rk1qngU!diI!r<9UJyNoeP2^da=*8bS_+P|Wltez??IqHG0TNrA z8k?{%sIZ6_+1mbta{X@vd^@v$O1Nsg=l~=z{UK4*`@QoEjhiEvv4t54ty45- zq(P!uN@SRa4I3glaZq(^5Ph~i?D)Km6{1vyimN|134TL9oN`D(9nh8>dZn*T`aX4S z^K^cDb366`iFsZMg1Dd_Az9T!fYhdCy-1j~?(=Sva*P;qhn-xdG0OXjeu0K^33D+5 z3XKojb%I0#Nru~yj}k`as=XtUrg?!b68G#S%F9O$+=9U!VuHmV*hlmH;%DsHfD7{) zO7o5D%Z~7V+bWbar&25ugeWHk4IvRNk#V1RdB3_U920ceV3n34q#iF8${q>?*0?)x zF}^;E$npV>GB}c?s$3{+W>(mG_Vh-ABvz;fi|EH5)Cp0TK!uZpBwL%i?v2aO-AAm* zJ`!CvW;4hy@BS#UqJZ@cpr|M>FotT%$ z9z0jCor3#Jrl1M)F<~@#3^x^g;7J#;hsi93yNzR*)t;isV^jTT$pKLoJ|-s z+Qs$1Kt9J$*G*~*wgj1WZz`^j5)8b14V6Ah`BFs?D6^9Fe`o!A@BEm<9B9)R!om!6 zR3BC`_QXEMWItY1hLb}uQ1Ose%@>&5C|b7?gDnnDxPhByVMV1{opi}6c%sb(1oF-( z>de~Jv|>04^+z%5_fM#uV@`@s z80kKZxu<`Hxu>r##D=+N$fLjaiP^t54o>Ij3m3G&SX*qKS@aGtT1nTa4z0=@AxO04 z$7oxG4)ZXhzAOi%7_{H8L(sQZ{8^n2cs#H%Qy)vrS-``LwSh17WNwnc59G;<1A6g>d>(&klm@$rEIaEFy_frG4A%y z3eUn8Uc?+61ADqhL{fYr^QLl)=j*-*qyA5-8Q+V_p?*2B360P%&7v+Tyvi*X*5ggk zl{;$e3iQ2G{2W0XZp#dBp@~2q$nui%vWM{H?BLsZ zHfn@s>8LOimo}>$778ROf{(XW6{92(pOb`A>uE&kh9XFH(0G^bN(y0FE8Okw<%!iz z+r}$*N4Jk|j-S4UhK!Dk2JFDFaKH?!(44CHtGtED&qs%N`pM&$1X~s zS;ydKs!U`sOja((xSb?sl1-g)VN}h_mYn!X(7*ZA708F9ROT;YWi~-&9%v(BNGZpa z4z#Dt%0q}?v~}qt(sH$&^)7AL2O=+4T#ywTD&w^79{>E&jNpYvtrWF?YvwC2P-uY& z7hp+$wiblWJXx`RwJU9(SGTCGWsV766IXJEE#W}v*E!i1h0(Fv1coU~<115@cyZjV z4;yxbrz9~xt)C4mx+5n3B0q{L+Vp{+pfDQKuDM>963gvVStG-Gw3&so82`vK zH7I=Pn9VTCo2xU%#IUT$?UjB77C&Fzty9!R?h;Sh4vNlHV$~RbZO0pWM;*#kszPHQ zR&-g>=MHouYYTh8EAq_uhr=8uz!;9^bc$lIOu; zg&zU}LIgse9>N?#-W`J89l~B5f<0@pYZ8S=9D)Kq@8C`MyX)J$X;|{ot)o z^?kA{aj~#z@Ly*C$Vs{%hhIzQ5*vO_$^sg z9D;r&@?h~xe21|+1U^0lI)u77gqi7nqjF+mUr4=w9aF!iZAHMB_7|_$zR?QNz7`CO zqZ3HT*IqgkMN`8sukhfQfk>7#@iPzy&|qRhdl%a4+#6V!96>2{U`()zSdd^~Ab&N> zq8RH(@25GqKF#tk`=tN3StO0@O>O@J+xwr412p`rajKGwP5tr(?frB8>s8rO?8QJ|UbwzfNlLQ~U}9-z->E2LVYTVfB&xs)77ks&@$$ma!c|4XWnzj4LYcf9 znuHMHM|e?jeP3NP?tm{A#DSe?XkZMlNQ5zs3}k z^vix;5pyJug%Y)&+$HKCpF`l3WQ&!q=|H`5zNTSI@6kE0;K;b9mgU;97oC1!gwj4F zL+|fvh%S%^uW_Obdh*-(5C1>M_xJbNxefw9e7^>uITn7hs3?;QP(tBU$*u*W2nVLa z&0hJ?UeFtef~oG|n#nQAP0QoFWProz!OR!2T=WB5=@@37?8f4)p8V5hSo+Z~vcR+; ztQIda&o3{{JA}7Aq{asviQx(DmL&VFRGhJ!Fd{r~co0r2*sEPDbr3Z_JK$AfW#rjL!|+|YDtm9A7SFVw!rRI+TS26$mMxukmYYrRTezT8EDXiY9|0yCb2msOSF|5*l3TMeD&EDBMXfy)MS@> zRp(QxB7_t*?#c^pmWa9|HH?UOgVe0GS(=zr;oKxj@CTs0Q#+F2suhd3qLq^Bw5m0n z&+y5QVz-Hnms{m%)nU??B%XBbAL3NXmU|m@c$e%rbUuve%xVtXMzS`WufKEMRHJ>W zk}TkkU{ms4cBD=iUJ@it5WZ~lv82DckU3tLHBB#pL9j$9-(H3GFY#k}!z}}jsh7-G zti4X7q;8EvHmHI(qH`_qTNTk~Ha^ZPW-ApcqmI9%Vh{>KXX(F!+fvVn zBfU6FN~V(%lS-r94UI@8C?pJq$|g-hWstQKo5mhg#Cs*$2%}5Zz-p1U6CIF`*%5h2 zq6jCFP6%HnZUPdCwU8U6oef6862pX-rSPGpl57*Zo?7ieB0rj}6mLDA0lfG-0+C6n zSa$c@e~o-$HMNxQpOFvx$?5pNi~N6*I?k2X1yKWLU?ffJ3LpKO@p!&oBr9!vS4qwk zrX)lP1!?B!6dWWZM}Yo#M0>Hn;;ACEv2eVTUss%y+Xr!4v& zK!|12dAL&4MwU9;#mLJGt><(fB}GC42oe~-)wq5T1k8}e7;X7w)VL<&uT+t$Di!Oo z7@TSh+8f{aP}(P;*O@uACvyNvt` z(k-pewk!GBN66_qj3lg2`b}2U?&%>miQgcJ(;)OWNAO6(unXQ(ZS*1R)1uia_$+=9KL~I4n2Xw?Ko3x_4=UH zfHMYGf@aD}r^65etqR)7#jjwUbY(3DPr#U@!)uE}lH(^MYun{@M|?5nM$Gu4)!34H=qN0gg_ z7lFev#u0Qz8M}hHJjMVix1}cZcC#d49|p__j%&wU{yCWy8DPO|z|Eco7cP{8B683) zYc9%H+Rd}5+1x3Sw7n=%Jm=WG{gW%Yh-*E)oPztkJz)j@GB^J6WK(^RyFfC_OlJS) zyX62fx$8!R{l!p^jw1D{)#a@HWe@S&jXGO>=uD4v`FH1n9e9qHhK;rUrMbw;uiRC* zt)A28wI;eWwy0=1SB23;a1*ofCMPwJtesubWMqL8Z82TZXu5^c<>xjJBtmKRkvz;W zl;ezss0p-E|T5kp3~Pc3=cTzkCm%H~=IjLhd`%Pa6?E9QSRN9dF*wjx5 z+^th0cK28Ukq4Ojc?H*}0EVn$3_hcB{ptz;ah2gjm9mS9%i?@c0s5kA64Bkanww>g zrMyhJA;)a}DzJP1y9GeXI-y$%fYoLoi$C^HP_2-$T${gov~Led$8 zKRm1e6${%R=CRR}WUuKwT&Q3M%R62}cAH2G=RG(4Er@;l>Y6WDpdps;mjGF=5H_(@ z*lbnNoq{%`YZ2rx{H8_*(Ys$??BI^sXGxDe&%It%rdcFgv>P;gwigOgUd4z}m6Z#JRQ!WpG+vNTu z66{{#hzo&UOdQ;KLj|1S)ylDa`$z1G{rzu(lF`#}$Bx*Z$bq}I9Vbny7}$enJ>AW4 zYH4=j3Ke?p%0h~-&vm`{{>h{#yW}VS732=Ib2&rveUeEajaS@;#lF6s4Wt#bMaFaEbV`(LX# z#Hebi0ECb}fHp+59G(HXd&vFu#;9?G#Gs^Y;YgtkO_AE{BD%8XL_9(h1P6`UFv+Q} zmvIdj5q0swmdXo_e122Des8bu50C*&yzu#8Kiy7*9PrlX$|QdPqdqgXazOe-q^{CI z1u%adGDV&;00fB2Q)NUNpoAIOUOJj?Tx$bm)7GcRvI`|4&wK$Pw;_kq$xDK_!Q6s^ z(;@_ErGa_V)ACjHj)Np7 z{ggb37l5T}Rvy^_J&ly^XLDzF+Sb3|m{h<&c1WDI4ENRW1R5bzXPMq>RSK);j>JoJ z1S@-qv1T9L{=BVdR8JH$(?y(0hBT%mhR$9F9>9HRq&(s$d+Hfrg{LIT6cVXlE4dmT z7S(^6-d7dN#jEOc>^4}}JL`@+{w)$H!7WRB<{3?fr%+dA^3@60r&2W4p249`z6(iL z;**?a{1bto__^WKAh_U9aa@(Fa;z(*0xZkIoLcUUGxt&?EDK!vA60TwocZ%J`W`ns zZ@v^6Y7vjMY?k{Jl%)H$Pr;ic-zl@uYI^}1(_s|$B}q=r5rmmndMF7Zs%X&-*s!C=fr_~Lb^cB_+!*Vd9`>-QW2mh=7c)O~ z)tZy9WsLhqad9Y7s?{(=a#FUo1^Q44qH-7i=rZV55cwgfVrl7VNf7jyvXKSnI?BMwy z1coIwJ%xYV8V0k&7$*+TR|gkEl>ybDdktbzIs-G8OD1RIXWRJIhSk8M<53p=XXLp` zoNR7T1Mm^`tOYmbuN?aJg3vILSv8+A&j?)XI@9Hxg;^qCTc1KNax;sKI^jHtsYs zw7ZprUa3E;uRCL%D4`&24N>LwK5gS|Szh+qyjRPpB%#Zceinvip2ax{bzY6u?NMKM zKff{P=5yDw z0dXX&E3q}mc+^ap{yD9UAlo#gy!dHTr0)VyY8oE`lO?C!WIbiwVpke$DD4{Em{~CN;aqz5VI)Cn*b+<$i#%AkV+J zJWnA3m>U;$18tp}&7mnLb748~BF!b{IIfG|AUUD3b_(pg8dU#?&7Za3Uk=%$imD zbWJZMJhv?;l*Tmu&ZH@C8Nf^9D}07m3TqHg?<;cj^%Noo6gzlh)L&ehWXr>8_ix9O zKeb~RZa$;6=M$y+mm4Y$xM z$k&1@%FqwfAB4{v(@7-jl$1Dy?u$e)n6%Red0kN4=+mI-KyLc+L2fX*@-#vb02?i_ zI>0zYITR@TZ8~;ec=v`=XFk&v@Srit05`#7 zr)w`fi#j94&fb_Hf5Sebf;?=SKg|dKKY@IT4i2_vrl0q_{_hY_`2Q1-uh|@v1T4zM zUz(Z(EX_vD*&0-`DLp4JEG#fIFeO0n$`$nAU`|jga>A+7lJ0*)_P+iD*;D%$$Q~!y59j}* zaipTNnWK@jnd$$IBmZ|C`Tx6dZsU>f`{km_zie8q)|BfU7cO3b@+HEeK}60=ZZB(lY=um!dGPrBg~}mZj6r2 zZiwq9a&>c5@<*Su5@hWi0ps*+g4vZ=gfzogXJU?YcKB7yaGJ)9q5>p2+Ic>t@83ps zRYp#!o}b*nssHLU^nX@_fBTW=sM&Zdh$D65@Pt$Dq2W^nf)M$w#;PD^qid_=lOkXy zLpTDkEwiX^jg!}DNU8_+^R8iE(Y&RF$4>YEK?}4v6us%httv`s_Bo!un9Nxc_|x+a z=tcFPcf#EVdqyIF-vRTjTDXn?hQyL8W0NnMYD)-^v-wttj1&u}2Xlr|!5_PsGFF@O zFI>0BinQhL7REAMmt;;>>?V4@OAnaAYhXU=I)~CQu+Wmzj#Q?fZ>H6lWDWJQ;msVE z(vUt8tRo;!`U_Im&?nC$Coq7{zDbOkdQ*YOx(KjyU$%C0UQAhfW%+g?4Bk zx#a#Ce2hZQ0G>nG6V5o+j2vORU}s=|5@)E;X@e@5mS`%Zn{B7jZ`-BS_-z~+3*iv2P$VLj9cOwyG}#>q5aE2| z2;knr(SVJ5197N}&`l%-5VLdT`k_4P2kAytbDkPB92T`xv`zw0Oz&rO1B~|X4C(ic zwGD+q;ZojfLtn>);lqk&?Ue9sXk@X~|oWUsGbE`OsJjA!Yp; z@(AJIoG4t1Ii>q`aR9ijROR3=#9{4seYAChr36}~638?0w)5;&HmpM6SyN7jB`F;FAmgyp~(85ulIjYWQ|d?)4&l! z^@sSPlSpOf^&OoK^;`xe-zG|#%x(#yuCH2>q_4$NmtCYIeX_I@aR0ovy({s1$ve5n zbMfi}?pyeSJmH2Z!1yV<-Q{@NFT>}GrSaRJ-}ee27`wwvfh#~q{943axUF!27fWDk z{NjY_7pu?9+RMGVxEO|KLmQd8ixV;sNBf?TSJs_h<>+m0a|53qx-ds5nKL^J$JFF% zu%3H+!{CB7bWba6;C!SeZYqv)uT#!fkL;~Bn4}aRiZf>*# zXhKh=$=>ibkOMP1$%4Ga4#ftE%U7&(YbUc0QzrUniecdDt29vHZVs<-l~igdz`zKY zNXLbLqK@kqg1{w>B{$n#C4~J(V{zTuoN??5(ty`J9yN|I(MobjXRqtOVX}8TsI%Sy zAi#2P@effCSPtk7rJAus+Zx8~vsAfk3g_;-*gB%tjXB=7->yH_I3@=&aN6%9jOf`q z`9{=oMy?u5dY~#Ld9es=J6Xrp8l5LOxNuTBL0vc|JH_qJC+M?VckFGT)$hG*ulrNo zuTPb){lFLBH&VMhSJ#9Bo~m289jfadDw}Ae)mXsxBVu!dd9%$pZoL{D!VTP5D-lpP z<#Kl9!3`|FS806LFk7ya;kNk3oI>K%thv%29;Ej@DgfIh00wWNX;V-sd)FO0W4#20 zEVHRlZnF52F`7;9D23s|dcmoJjSN}%j!V5EcX6X%Ai!E4VmuWA)GIdswSU-kOJ#}p zLA0ZxDvqZ3F`fUF-oE8LG1z3uxWp+3bNaE%P8H{{x%I{0MostyIgK5=J}E3q~|5S<)Rn|MX;dEYT@CLKv}Gd>o-`bSoj&x>iP*b$k0 z_`j!ln~RX8%~@a;*7P6Fz|W9g>JSpR(U@XN!>wLrqFSa6m z=Mi@Ru#~#F21sCTk62aTG>R!%g!@DcPjFLuL83cyp#c8;O@z6=59ux+5df6KX)#nm><+tB6Rm+u2XHug_9NUNl z4%5?o|$UjM1%6U^S}6$_c?IywC=Qe-FSFDcJlt_ z!}tGz`vLMX3;GTGn=Zn)FW<%o@>2Tp1`Lw>4Ehb2t_|+iwAr_m;??2Qy6j-Uge>53oZ!IQI(K?xcO{M0>Vng#F_6 z4G7cu)QT<$4O7Qo$35wD&;2<*U{@8=itDi>?~M3LdM~pY=xA0fz=<12GbidnTmw!F zMdR0}Ip`-EK_T21ptGc|*2?kFj$BYo^(kQ7LITI_7X~Kr<PjP3ru^+xu%AyeL(VqntzR+K0?r@L-!qJL)22)Q8TC$4F&$ZSGXH6=#_h%Y&d#Ct%{Ecv`jOTPrCi8Xh+ z^{eye6^Vr@l5{o7Xy#ioT-$uSbVUVBgzAOvk2qF@?>Y6FoQ{ODN;F}M-8a3n@hA>) zLm6oimNuOE6*1Au%Xrw)5GVsq{pFKiteovu^3lts3$@^Z!Yuq1&ff4I-bEqi%>?Q$=icEj*i_}4OlNgq`iDm&zp>Ko~0iP?D zyVYX@IdIg6*jMQQ8oB|!`m5y}chQisT3fZMi4GQaNvn+i7|Skhg_Q|iRzV7KHFz^U z1sjbi*PP5q^;CNthL*1153}&BbqNd%4tR~yQjbk#7AilILl5BBI#W*XWcQTN&YnD3 znob1VQfc8VR}mEjk;m7^i|XZt1<9-!ahL%@hx6shS4o{N4WQ31Kh|X+qD-64F#Sbp zeOMUZHEn;4R%lF^+~uX553mh4!HWtF7k&^jdYsUN;Zm%PNa2AQLkY0;YgJ;NkSy$h zYt7tcBhEweSqSTD(4X2xy;1U+2n)izf7o8siM{kik5rQ1e1@!^};&ftv799X8=4-sNNV z$GkGz{B7DrKp?ovnpVNE8_etvEn zmfEP<8*PQ`jtet?blm7OrN4gvh$(Mu+{87)CuttqSGb>{&bIg^iJ3oCVd5DsOrx<} z04LL^U>FEUj{hmX#gxvnuZlGBOzWTUIGdaM_zJH>>MK9YuzN>41IyeKPQu(1!Z7h{ z#(wL>DNpq?amdv7tAyDztFQPl(?!}1P4qqWK;Gbrsb@NWFUfT|TwI?MmV7UOnHO4N z?3vqNZCK@*4SK}P6M@3i6PbeSjoZdwdzd9X`O^uhL2nsG%b-(LDx7q?IJV6$%spe| zK2c;oI9KsEy!wKx>D!+6)g9O&Zgyjg+ws?USldGmiG$ZDhzt{qpKP~PfTC ztXcB+=-pP!ofRg3KeQh)bq~a%jon%34?Ln^^0$JyoI{N&yg0{eoHOKxiD5~+v*%Wv zB)cAKVB1MX*jw?ta-K54yXV}H?X(+6+!&%aqaF|k_>krbBi1Kk!=la^2z_h$8}SN3 z$IO)tv}%E~aI%iDFrcy1*_d2Do4Nlf=Gxry{JISKg5kwm3$#)5Rbg-Pdu{lA<0gc= z*;!iwwiIs)%AzVmhW9Qp6|Df=UKBh6_2L(OIYO)tx$parr65qR;Ve(Plp~-O@z!(c zX>@678H2VOD19P73w%(km%YBjndg~8DNhA7aWyBcvk^mxew&Go`A!*P68vjG>6hnm z1o>W|W&AY<_uY7lz(%R|2nug8_|+}`CR?#6-B(4X>B!S{kz?tPN&cPvOpXe2<;`SD)hqTV;$d%uJgw8irz(ZEANMAf z>SiKs{CU)c`?Pt<9*8tI392!N!PD%pF}$FefothMvX2L4@K|H`yj1)jxjMh3JUiq9 zhzT}bWmw`nx$qIA>w;I;PR;ZS21F&7w8_`GDp~@U55fherToRzz=Fr+Lwjku@w%t@ z9dy2p;D3uMtK7aiwX(Olv?>)B(15Z?B(ypc?IVtS$xR?ak+Z&Q;<&nbP}GGQpvCx< zXL3QOTqG0ql7B2LPH>?>MRZng)K$Pd_1VxRCN|@fNGr>VUfKRd1jar|Meb%IO==uz zzJMF7LowLdtOK27U0r|eNGe^haei+GwET!xPVeLtV^E^QG8dd+yyQir=jpQyk+ht? z(;iVVtU&phPuIHDRWJBmu6>(ov*%A(DiUS<49eQAjl`|wH%$jJb^YQ5&E!}!1~Peg z32qc9#Ls+X?E!IQxk3b0ooH8`LjniwjTNCxQiO`?O%lb9Z(6CgxtYi1lGaBK-y=-M7&4P@m*@P3f)U@isc&=|tF#ETq&5XG&8+}tv zi9LaiI7*~DfVNF+ZM*;NvDGq7lO?00vAuF_dwXH!lqgK}HBK7`a>aHXLWg>dfmx!S zgF7F0|9~t${I&H^J53Y;4qj7@l7Nm9NXDiEyfDb?y^U~{Gf7&NEjDJ4>$<~ORGo+j^_6Y zJHrN}G!@@d-E?~`E`h`J&Ev)CHsKyc2-)LBX*QD;EM`ZsN0@IMECzJA`C{BnIZ*i~R+goo~Myi}wVG(4xpIbAruLy!4G;h)CE!|V{KD+KV-{^&sS2V7|N;7bW1h3-V zZNU+(f5C9X31>=!m=|#(;umR^@o^h!*>+yfO4 z>uQDVLLCs>ghzc%A_JQ}IXk%7Pt83wSSlyf4W6u~B9W9+3pLQ$_5)|43 z$xMufWIrhW5*nW{``Hcf@fw0qHCs&$g6_rAlJdLyr!y?#u)x+{by$T19_5^48Aa(z zkf%=hfd}@zH4R<9ub1{Y&F8e6I~DT%5a0Ar>}fNDViX44Sbr9;J>%M57Nyal=_J+t zqTO8Ftth*$B3Pl_5=6zJC>Y}xxj(w(*BN@WZ|TV+JbUJGVXJCJCMG>4U;WPK$0`mt zCL_58x!ag|k+)!GP_$7=P3Kch&Y5)pCe=oU*KKtl2EVlNz&9T+;-=iGwNOk}B2|gc zQ>y_WH&i})rt(8BHB+uQlFoXL!sRGCLV(0N*%M?$C}Cb0VX zq3ZQoEO=BpXOvim8$iehX+Z44sQi_veuu2NhU!|2D6dQl`B3p-o5z)1<^(6JVNp#_ zBN_Zph+reet{L8|-MB}|J@1%sN-_MlMdkTr?Py3 zOwvsvV#dq~XaZD`8P46V%a9fNA!yid{t^0Z3m2guL0yhPE$K_JML-PQ4h;2rv|k-) zS`>{G{G{wt(G=Eh3c5L25MCsN#LVWNCXfj`g*`bXtrqJV@kXqbQ>N(%mt^CV1B+42 zlj*wy_}X-!-CFH!h+dd zIW90gr7oRvE9FwAMn`?AYcywUl6k7N2_kt~FV}UemoB%vdc@lMX?cLeawY4*A7|DpYWacowes>vRbRkXrU%z{@sSC-Gr6d9 zZWwGKV5{ySh{>)0H^jRS$Ci@xB)kd4V9r4>%`{rRu*dn9O71W44EClp+?j&<)z~yK z<2TdsWVeg?zl)@>xQam0E-Hz12I90~86&p}BRIg)Les)-QizT~azAXH^q1ydn|@wl zakg#fwmg^YE&uk&I|M?Vco7+>h5AC&)+Ik*KG?9p3_R$O+(ra9I#NH z`ioxUALc8&Y*8^q;UaIW9e@SZmLv7#zuJGXxvjc2uWTO*Ij0C&hBhL93CUm)1Wwiy z7~(&TcVt)YB3u%`ef?`BkSMbQWne)-d~p7oy}Q3g!hdjfG@(6lhR{An8k$m@+Os($ z9F=2rTg*tDA-PCb8)QXIxilN(0+wxyV@M=Yqt%^Jyl3aU7_oFwg_Z7${o_%aWK|? zlXCYEq3cVn>n92_7OSz(Cfl$Mu5ILX!c+_?;qZ?r} z>`CT10IbrnG6a#k4uZgW50l?^1d4xeP~wJ(fj(h;|U8o&uoOBV)stIX~*gD zAd1&FV&K{=m#=T2JfxPh4z~4n0uMp+T8*a%SbKNmuzDlsI}z)!&x3&7GvWPX1i{~+ zA5rrVk?c*fKkjR>o-}i+;=&ha!VJ}go{&PT3Ub2@e1%~a9&qjRpt;3-3r#tTJ*6SQ z!d|W=b+z~2SajxAt*F`*;JU=pffb-YmK6*(UbiPjIp(jxnG{R$JL}vBoZ9buI`z&q zL2QM{qy+Yn*g{33lFq!0J?`m(s2*nb0w4;%>Re7 za|)8I3%hi8b=fw$Y}>YN+qP}nwr$%svuwMntEaw+n2CSB_z@G4d65_ADr2AhVy$Nt z7tzT1!lYxZdMr-Bhg--ZK&GIBoavdF(0H=#Ua5UZ4_;magLo%ik#Z95Oh1K37GEo)OmC?`7N#6m9;%{x0XK+W z_y@UeG6TIvT(M|GpPD1Sx4IC26YmIQCV>Q`7)j6&Fn)=R6$q#SXGEPSOv1txFJQ!S zI1g}rIdLTCaM&B=ml4gO5~HRDC2t|HI0%XrM1XZtVOs?wl@jAm8swzuu2p^%;dSe2 z5e9Wen?Mge3qR>6l6IA=p{krrLWA&8#SIClQz0%Vs8K}{7lWXsMYqupA8h+4|ex2`BrZZaB(%d5!2K=P)KD(3?iNn}sAa++vrP$AXPzqjs-aD)DrE>({SrBKvLG1828Bnx37@h~ zAJr;dJz5`trp64$mS&HjPnY}{6s_ev!q;5s@e{I`0g%NVatfsc^GvMp=Z{xnHFO3B z9{BmB5*`hmX34S*qz{dH8r^aRIpt;YfG!5x*={WQOjAm&VDrNWYRQmFo#%aFxN8i=Xe(#pEB83FrBA5MpKV zzm~X1AtjxDNlj*tU)N>e#)!UI={mUJQ9%0$CLT)?BZ8!9B&kJdGqL9Yhb5Oh3+)G5 zuzJ(S@9Ou~GA^N(cI8f8fZ%mcmw0hX?i17%FNvb-YceCt)0n^#Lr{ zvIWjo_gO1AZhtX7Kd@8$6e5rVuinh@@($H>otr^K73`or0Z$+1>d) z+}hcm-(ge;njiC^B=2|Gm?&oE<> zKJ}OgJtoE5;I~ibeZMl{`g6a^`kEhG&{5yv#9*#Rkl1i>9dv?X@zpdgi3 z{8_8qRyF#&Q#+aFf-U7aMz4HT!la|hintPU^j?v$6wd*V)KUZ7d(}`K!UZ&LY_l!V z3|hxEJi2PWnv!ehdl3`eAl?VmW{3Mn8jUl|NT#X{$z|-(s88okIOQ(BZwLIeejf(3S*7TYG{gTGw|mHLE?N$h;zddvh;);BGZE?WRUG z%EA>>`gHPO(DSt=`#7BhQSy*@_J}+gtm$ims_Fdx=v7Q7G;WHnq`31cw!Q^}WQI!( z=<+u~5nI#}B+1nW!@&f7aU|};=#PNq2BaP#_7M7-;SSCyD*BmL>CUvV0{N)^yrtfe z0QIzl{Zs+)ghXyZyX?xhVQQ&(@36r3@|c$*?ftI!Jur3#mdLgNuUKR z@aR!oLq)2t+L~xuG;3*XK;JnEpCuFu(%tctPwy0C7~8YJ6jKTPYASnOs>XJPtfc4^ z5x-F`N|wp1@$0Ptks?;^R0_dC(XN9kDA0InMX99SK%y~I0#FQ|szw(jXrdmC%{qOq zpP`$5qAkd(MT@P(WHX9p_55%D6nUT*`PR30O5i&Z_rLT{{l69Ye?&f1tA4m*EhBub zq;%DH%?yhFSPTLVz)^1~Bnkpj5vwMQ#X8ik?}QEFA)5oUov0-4 z!n0lk3nsG5YRXFa+~TbW#kkVYuynN!HT*cWaJ`;-ncm`by*~B*`}f87=jEU(!jOE} z?~sD91B9tPAz(Ow5vvSoY3eXU66;Ko`sAQ(;ZdUAMYjLq||YFZJ;=^(=)3Ix2eqex9T%0_~j$tiMsWd9& zHb`^D_(~Ulrdy-vk=~`8przrq+<60k{UXp*{4UTlDCbDM%qD1Z2}pa}>?l!Vz_br< zW^&Z^J@Y${QmHC~M0FfUrDnR0qq)GN=${9F*)1D95j=r~%Z$~WJZN~jy~dXvsbUxo zHSDJeIQPW@H7NyyIWm28LPVd)UHOD2%?LN00Vp^5AxT7?5lRA`QXA}&`#k~R^4h%8)SM@s~z(-Dd^PH!Ant^Cv5CRWih>$Dx zikvI{iu6UbPm4&iUu*S72;pmZx2t}Fhd6TA9wh#m#=sf3+2{;6p;0VCYD%_NRBZ+s zR*jpZ`jUIO9LQzl#x^yGfZKrB)0c`!a~Gg9vI*qUrdp6JGtC@N7mN+(4zbwQkhMu2 z8YuXvgB#f)-%6M$QP_3pR7s~>dl6Eslz}V%Cto3Lxl%b;VhNTa&<1sJMb^j)L}D)6 zXOYP>YbJbxG|0kLx>jb!62^BS!+1wGFpDSsGXGQ1pgS!wbTa@AzQ3HOd}k{Dx?g;9xiulgj+OkAwll5zrIdFp18 z3?7I3AS*K^{Dum1a!gNL7>#EZ=?1*##fQjhx1Jfa2_ugxw@8??&8kaR=>!3Ws&ZVHJhnilxz3Y!ozotiLoaATVk))&ZHKnlxb<1p`80d ziMRSGtDYLU(`@+^amV6@$|~rpiq411DjnuhLHlmf4TmCw3L{f82<1ZHoaD)f^f4+x z_^zKvp(jRV`AL$9-ICqobnDsb=BN&T*(-8Rl|pCKQWi zmIVfB+Psn(+LW@O#pHI2mAn_9XK^}>6h$6xuQ}<1#c?nlrsy^T$6^&Dpn7YG zBgb~9inQ?)iE^u+KCo8O|qW$;ui3W)b0G3#I6&kM4QnosZpL?F+jsB;Kka2&EOBT%G*gTa>I+Jdi;9cBVu z0nXV>t$x>A;FLOoMu(bk=RR9Ud0%mIoTr>46IK|0Dbbu}cc>a@>Nb5p2RbwDRuFG+ zw>nGQUi6)$?-JPBLOK@)ia}@lyK{TiXYmzo!o6Y6I6Tnmz7$e&(dzDW*NEi)jwyQI zfmo5So~6RCAkBreOqcA`o>_?!ZW_?YG3+>)Pk=e6$%R@k!K|I~d;c(mwMg1oTehyR zT>IYBH`Lo$=xO_*cZA8Eoa7*5i$-|qfAEkl)wo?^+SsgS<(0w~ouy|ehti8WVvn90 zU?UGvEAHm~CF1nKX@%7tg{v{xl(4_*labY0^5UR!PqZo^z_2%S;9(*Z zVl-I1EIPO;9%o}36-O<(sYjiD2H9f4bU=6t#V(njQ=kY(c^4d7#%maN0m$Gr48pEM z6r2~_Bt=?Lrr)5$be;`FI_dLv&@BdAcl?BYpTmLB643aQZP^mEeq?5&ONjXn1DH!P zyJNeGiCe}&v(r5Ax@kbT=?eIH5~e%4*IGT;{6WwHRthjCXbZFn>~4yp-slIl-pThK zBBx(pUPN?DatkbMc7K^WTA1OCF7!EZI8C@|=K}TmmsfJx&A9sJ+bikz-$7CSxhVe) zhxoq~*`)7aCpqMh)sT=7jiO&sQDuY&<{EaFeS2eL{1O>0<>tSnq1=Y%K*5ucGyP0M zp=`a;ZI|u_z;02ze%yEgk2WtT^I@1?;=lFy)@FAnULJ1N((3-ygWeI@qp1?w!`+JI z!^IVY_X^<;38hXsdkK4BKOCiwk?*0fXTL|nprYaA(>4r&#i+2(Nz7k4j)Qi|G-Rl1 zbyo|CMupzvgpE7aW5LH$2q-{+4QeBZ9`U z1)m4@LILT1X{M|=l|4qR+@%1Orlue9Xmf>3TC=TMzT*`)>kFMXCq6rGfG)H-=Bu_C zUdlYi95qCE1x#j5zSc9Vwv^sgp`7{FDsRg66V9ii>+i7`wW)h1h^1@CvRvy{bkU9- zYe`J|87q#INh$DSg-!F$tsGQK?|NAP^UHp_?7wp!j=}&p_8hv!VtbxQb1F}q>RcdHTzv&WbDW!Q@+e@ zg_&%iq1`{E1y0d1J<$~&JxpbiykaF%8J&IDv;CM|k-Sw=tsFQlw2&pH+0$&|&Z0vX z`8tu7cZ9gCwQ57*Jo#WH+qcJi9qBjZi>Kaq#%oI;G4nix$3bfT6) zyOcxn5vF(+F+anEL-rx4WP(wYah6fQ5y?1pig>~%l_C)%RU?rj6**!sv2=p|c!C)b zUI42wX;Pr1wtw+na6{Cl_owKsz=`Hf^1qNmbcQxCjo+t-=>Pll{2M6*lrY9uyy40D z%-{maMqOxFDkDFNJP-yX51!Y(4nhD-Ur+S933ev=D7$$U%fp8^amGGAQqPf1o_hI` z-Pk$4bN_L9LiizwE!-cd8RC`~ALE=S^oP-$cMLqMSIXHc@{`~gb${tt zCqFL&#xqcgu^X11LWnC5X?gAo0WSa7Z?%z|(Y!B0jAw6?q&K2pa0(X;O(3N7%Lt#~ z?UZgf>)_|P6L!DT^t(v;bpr%J#YfT8z1EmSV$3z875sm08htlvDF+Ud*7tz5G__Ae z?u%F?#K6-C^Ym-XKs$DK>pHc_~qnx+?^bzw;(hsw-^w+_=ok&sF+7D-;IL z&uerP?>S(-1TFP`Ymh5rIjx506|25$hg51nZC)}ZWz;Afb!4nounpcln94(zg*#Fw zMF?MR_Sz5io_LM&Oyr;wu3p|==51cPr50Oxm-lJ!9LM-S427SdVpQ978L5wu@gBw) zvO(MYHDpIG`rBy=y85NViM;}G1%2h$FJ-~#?QU*=J8GEP?{F?4VhI-&li7(Wt_uT@ zQMrv$dL{h>CUjm*v?IoK2`4j&cW}tcsChWJYXs{SjqJROB zJw8MNIlg#;IX)v1Yw#99JK-VCxHmqWm`N}f(f?)#WyO{L!wwQ-5I7dk@!aQLheDW5 zpaNGXQb)W`I82T+PFNLiAgUBt5nB=O@gU`e*4)8P$=~mGsc^&D;|*afd#%m=m&$#h z4IRGkTj8F9{y#KY|Gq!|&jRGl7iksYFHK6f+KwF-JcOy=SOSi|fIy@csZfeydx90! zddyfp&4K}Hk#=L}K%43UVX^oPOZP&jbS7?-vma(yX3&oH#-q$lZ*lI78L|3;M9kG~ zpz2N9OSb0@SMPlcI-XZg3xIf z_i%3G6`h_-p8)?5JL`&JhWSh_8TIBy&-c4TNmJiv{^hY3|?G$saI4U3yPH}MyUIp@ znFkkFMxORh0iix>d~HT*^~RASEq-(S-8PD)_*Wq28$LeG+=_CQUcWlljQIzD#|Zs$ z40byXPXeU8;I0}FMBs9A49VqYJ(zg)w2TE9O8D^e#KZ`MOH8yq$TM|?RUDpN`I6T# z@_kC6=Lrx=ID{MvmM(Uh7$bAdc@kx5at)PcrD5n>_8qZP5W+$s&;aF<*oqRA%LcTm#&*77c8rfp z>46)|!$}gZ){Tl0sCKRHj{4+?#h_)E%;mHaEcIczhNsUKDGC*`Qi=?ed6{#<5T4;I zl=}%2Tcj}yGQ1~_C*5@kq#6Yj)qON={9ck%EQstyVv^jIX!~Kh za7lVPLS@eimFCtM715_NF_wxPB`RiFKyUSk!LlcDV%#%+k(KwMQh&M>p_2i+1ao8mnO+Ma>#4AOQ zNK17n;+-?ZZXYW#7nJ%y>#eoJB#?aZW!U$!#PwDiVt4R`SZfH5=o6lo0tqegsZ4*2 zF{>WYgdQK@E)~{B1YoEMd#w***CJMApCIss1wjm*fDkHo@Cga-Apvmc#L>m((v6}K zcLh;?cc(B2yvetWpISq-L|rhw$aA~EF8H-rM+U@l#GjD8sCA|0(=uwv^zK0L@EQn} z8w~^Je?oYBk7B9zQ4o3)Z#}yx@T2&@8`?^OFEW7aYhzIFUM=GF06v88pd}*gL0nj` zq!<8?;ENof``Q{LdW-k-t7OD@Feh$$g1D|#b?-wWVouoznJ|oUkzoc8d9K;$T%^f( zlEa*nvOI?kwOZ)+urp6K|}@tNVuZXJo@}yOm=WyLe9RWwD1uhY$t^203=2-`QT@XCR*4SQlEKkx-VrrISGYKHoz@ zIm!@ZTfqwGrz=glEBdj@Z}veo>tWoxIKN+#mYfu#?WeSw}1Q`gdAER8(kob?gESRJU_qHvj=Er8RnUCz=19( z`*MfFi|UhymgUQZGMc+Io{suS%z~zC2DiISJlggEfEC?6TEI;D%2BLun=(Rqq4oY2 z21az$;7%yR5;?hSkTS}L!2+Wppt;>XSB3`zCnpTLXmmfkK0Y2WZt0jILQuhCPUj5KQpO|^xs?s5J&o?AKTc?vJ>{o%^_hwIn;6SfvLTH$KlhV0>$ml5oB zDg^v#0Gt6#X7`p}BK8!CuxIrnRtY1{*ELpL*f{lLYsnGfiq0qp!kv4S?3)L=`?9hX z+kQd6NUSAnCJ9%~Ivf3!eu?9069z6x;u1qs^wSS|A4&q^KLKWx8T&B=9daVE^lq$% zuCpWH9i5;Xg3r~f#%AUnRz2e#;E(4KDNgKC9d)(nE|b_TI-#O`PPm&2&jZ!0cAH!+ zrJt(xgU4NtD}ZIC5u{&X^_- zjX$gPo+u<-)!Rq>vhI8K>x8?+mv>Dwfai6>>Y}pJWdsT&w>lY3|9!h6q7~%u;bZT# z*{0=iWzzV`0}9E*@6UPpSbeIrj6<6mY^oq^Oo9h{jD8c*D0lJHnqn%3K_al4sJL2e ziko(m$=MP(b$C7E21f>c@&)2EyEc~$YReg@S#;rFKf+GgY5l+6Mg^QMGyR;KY9Tml zXK{djNSC&tZOzzjTr`P2?MT0yAsR3@$Wcl57XP%m_KnXT%UkU##d(>SP_s|F;<#;1 zhPqxIY);jh{@tO486&jQ=)Y!!{2dq8)=&u}7F{?D)wki>!X9~!xZKD^t~$pXNO|-A zlcx=m^?s%jbKJvk7QmOg|DHN`FVUTVK>YZDK=3~lL;pFoYgpCY3uzSP3)KoLGT5&_ z3nm=tw?Io_LfRaGgrQY?*cd~7aJZnRD|Pko#)5qYHLZVNx8VYtbq2WGaN)Jqzq{CU`)Se|-4USaOwFdY)kD%A ztxRsYvFpr7sO~FQ|Kp)~t~Fj-wZFMg@A4nUQs3&|m!{g8y589z(Y#|pwLMvNwHxKc zIM@Dsw;Az_#d6(8cI`?X{!r`69R^d&GP&bN#WT3mM>SR38s4XhjICoE-M2w?M?F!W zYIrS!dTFz(4NB?GsKNLknsO7ES6W~#G=ZkBcr7tNlE))YAm3(XFdz%bP;wsB)km7N z=t(mmtH51#S}r{c$w*f!uvkIER%vVxqAyU19y?w*QyRGi5iEDpntpYSk}x}$99Dw3 z?xU7xwl*W@z8DxKs*0){@mC@!kurmrW+T@xHx-o|a8Q7&2w9NzA-1}^1eYdf1oq+r zG-U^GCPbM!*ijh)E|2+Jl!Y=%STwO--`}+dy2gD7=wDY+4#BV>&1emLv76IV+WXs~;mws)HCE z3{Ok?jrb%ug~8At_lZ2b4pw9Di_)1zkY-?s|;US{|wMeB>mtn0>}-q3s^uc1q-M~ig? z_+X&j+GFJE-HCfi3{;$V?+uh1dx7RA-?4N3K6r4O3#J`=C3s$pS72z7wLt43I+%Qo zh^pQH4nK&o9eX9|AwJl2^K00)&#gPXi+*I2vB22vcV(Zs8Gq&R*&Bc}{siZB0gy8I zM1*R+rbp!-=)2rOVe}5w8hv8+#v}CkMg4}I0N0lrfrlMO9|?=v-FKz=)EPnel%N;z>U9h{%`flTCi_@rSGI0!-zVUut<`oyv=YM%0WtmII2B{;#S7~ z3R7!Idoo|bFcFl8G(I(9U{w^h3DISlI518CX_C~`v7nJ=S5S7#)sdw&Z3M7vyAK~T z+=AT%I=NPlMb%VO;g^QsjvP=&+v~GC<5`{beEZidqU?;-DQ~3^(nx`dUo^!63X>Ba-5|Prt zL_u?9)>^WKYMGeQWM~vlmt{d#Q^Mx>Aj;#G0;uTPvwZ>;B{mex+KnSQ^WLXor)v|6 zc38-B^+LLjYYKx4*b=@?vquEt!o_feg@%v{Mk7@ev0>nfDhR{k_zJ1cgQL;->RS09 zDhJE7NP+`WrN38F64fi9YHL_gEbE zu)rRKp`pJ@zB4ED<@D&_V}Reue+4bA2oTJca>0PU`5rn-zZ4vlo6OJ_{-TQ;xBJs& z$)j0wnK{W_epW9);^>Y~W`3$@sqUaPi7F!!K5>=JY*W7=tf%~JZ2IJ4YD|+x?tf!5 zCa1Kdm;`a*(e4y8X&mUh5xchEQ&@<)kxJnJo@ zFe}$X?q&5Qd6b<-WYkEeh5d?)dA;A$==<%S0n*GpbITv&WhJdy7eW3+n#qB_0V+O}c`ED7&G= zGHG2I6)!_3Zk{Y4vfWce;+gq5STfoMDg z6c^~nQ`s{fVZ=g^lNs*gti-%_(IU*rcj?txlhO_&O_6o8+8Q$Cl-5hjLaJmX=-CgT zV#d;~7SNNU*_j(c5|p5Wk=K6oqAxdiNWsD;RTwv%HI-oX=!r=Z8Ff!g3MSN0SP&E$ zL8}v&M;iiVd^ERF_2IA~Brb*&lw+WooVxSLTsTNE9Jcw*#8+96k-JNclg$@hswz*% zNvZ#&bZi%suozHXIzzI1LE@Ff{`>8;y_`?5C!atk1z~b$0R^EVaM{uPX~~Th*^lf+ zAw;Oh9v=D;E7pNbcp?g;ouV-G1We5>Ovp?w!=lYbtJgi6!kkbdd{!BUTsJ2qZ6VKz z+^NU#aXeHX*^WW_qQNKnbG(K)uaZNh{I@4_0S?^AWH&?lgJ&|DC1KY5ul&sr0U(v6 zA4hFO4f-}k0b2sZg83$GD&}vnQ1Ho9_}UVs#@=#^7&{@0xk(J^JSi4f%7_(4M$Q&~ z$|wY!-JMe9D^(s@brW6na#mS@(WQM zLCDNem0(6aQQ}Re!=^`>cNyaga5MIF%hKfb(1{f7Bwzgq6fYW=#jT#?3F?-<`|4H( zI8}vNA~!?jJ_qqh-paocpOa6^pI<0PL)dCP5S*80Dz?+v`-uE#Cc<8zFOh38U};_| zLxFB$eGB(Gc7TAeS-=B9k6);;7lN(>V3GjWKer{Rzz&djIG`%zg@PvR1*4002jitU z^gXCT*zF%wx_YMep}>t!FIr{Y(5zRCe5L6j-T!#?55wCRy*!N7ehm-H-8ZHAXXJES z2GxsvN69tf;zPa9`dS^f3wR*_g@>;P<45>_M?@H@3;TtRa0m7|AIuw$KA0h2P{g#%szI`*uRA5M0^)Z0$-lluKA1tD@ews)pLxpBS-t<_ zcGGwLMXx@a#z6hKM7b9-*EP^Y$n!Oni)WG9U=fzmc^Ut=(XCQ+0Pi^~#Lv>)Ap=UG z&snDS?5dxO=L$lcnm$e0P|M=y@*FXek4U7V~LRIXcy;px&XA+9v|tW)hr2I z@j#1_G2z~Fs(MF?#KNm0ZIyH&j|n~rAhKG+jAWQ-7!$t|oqAKIN?zR>nNGmaI%9|3)}*9~s?U2& zayCi^ICOFzYBm}O^`;bUK75@HHC414e_!F45Y-qeFi^tDW(Y=jS2Pp3q1P(dY<&qj zkPs3ddNV9Lee{XhQPD9V;KoQje!P6Rh5|7GlqgpetQi&$kV6%nMP*9dZSbI93Bg2x zO4|44zU1`CzAG?L;H|^7cndM3x1>@L(!Y`KH5NPOofR9upvr&z>X;GDtbS_ByV|i+r7rv6$!D_d3cUIod9m13Q+t!NDopm*DY%Gix z6&VC0&y*BstxPLTAzxPLxlQ3g!b(K}fBqT-%)JKn@*-lxDsIC8gZ=>%MA+Dh7{iY) zeLBkNc5$4#9OSSXX1U!}VOo5BMJ|vnE@igMPxh(3cX_cox4O1shXQb+ZdpcL=lwMS zKOK~Lctd@+Ek4mE@#V}30U;41wbwj#wTB%FGNTBEx1;<_8Iakh&eM%`1)Y|LD=~dr)meY6n~+(`K87o zGdp?OoBfrVI~ws(7eqXx^eP(I5PVc>m%*lL)tml_Wvjs$86!U_eSb}*(%@@aAF-X= zf1ZzsOC7rpI!1Yd9;puSv}uK z%I+>A8u9&$2YUH*omnPlp8dqZ9d+ajlUwuR5Cf-&Ce;*-PNUFtmBSsipI_3O`;P-_ zb&KmMyOt}O)^ur@V@HAKh+UY0{W$(Jou?s#Nb!o68h~1xVjw~zQ$%d} zy^;MC%h5QhsLX2Zj>=PC*Y*eYGH1qHHhp2N+^iaPiy#zBuXjUL zxr3%gfY4U!VRu}CV|4uy+&qeaH|u#|uL-4}ILk6Y)Jc@&1@>!NV$_Pd73ZEy}6=PilNo7peJfM9)X(zs_u3Y`Ur$7cC9M9Lmc}L^U4^x3u8}>nbvEN6u~G} zmHjO2%xg*qSXLEve9^nRC6JX$i(f8@juSNGe|Si%%!#nFzs^Q#UmsDl?#zqM)$d3- zYZbJW8`m)o5-N^ONU2aX3RC5TJ93b%>>WPH9z{C6k<&f4uALn=`C8^}u-I18Vj;f% zMWg2(a!IQFcDnt8v-&U5#Q#)b{}q}zVN(K&AA6)X++5|LC8yxoZ60b^$QT~DKZx%e zncdPvY##hsun8>rHL;dwZ!w?Bc^P>&u-IahIHbk$(CBI$& zB%*rzPi=V7c%~PxEi@5IoMM0ilJegHxt&PQ1pSIa0_v@?xv#1^y$hL{9OW+3tT+w@ z4U;kl35_p#bowNjjF*M*B(s*ks+BzrA0kBIOtyxDa#*eG@ErS|zx1sezmoIckBO4P z73*7YbLgl}>QoA|oTNjlRa{wXTOU9=v^ku!P4H0MaC6?% zMi}uL-Dc+6y!U)=eovKH@2vM6Y+K&ffRpH>Tkeqip*8Yc0{mQgKzyz8dTz>JUsgtX ze7>mZs*btG@wlDsPjdUZf>OyAb8~r+^vY$)^b>Y0ic-z9t*UITbo~B|rIBM2D}`r; z$Rd!l%Swc=On4k17C%IJiWN#N71|F$UzyUYBGcb>(;48t{>$73qVKOp^Zm#F`lb;6 zw^G{w9ITw3^qmMV2S56^6b5+1jusr!L7wSRKnXsXAVSS1#o??h&__=ovs`}zI5DYT z^zJ7%4`X3q@*Ttdw5r{RsQgQiK|8YDm=sGUx2dn`Y|i$>!AmLa4;z9r!5`fOHE(lJOqmuMxac;vhnVki6&{*i$w!c^re>f_MUhM0g z5$^~XIV8d+5Vq&C+sx55Cg!CHH(sLn(ODOp z3TRv67gpMqQRxTfN=g=wR1Hp(XJJDNH>s&jYb;6iHKYtBm>1R;g6^u!6{)g4z0i1V z%`Gd6lMU?0N3{)8j1e7Al~xzj9Pqzv2ZfU6LL#B1i>qSQQ{F>HCql0!%|s%DC1?~PNW$tizQ99ZLWQUv2!=ISmdE~(oK6QTCq~1^EnzH zEGL>qCrD^D*j1-%76&e_@#v}1=IYh4XWHsInq4FYy(^a(cA4iAeo|cH=K;s^ z_j7|npmxs!fg@020I2XNFI)^^18F@J7Rq`6m!&H@1PGJ;Ux`JPXDBStsH&emV?gc}vjM2KsaB_aZ<9f;N6WELFk{W4Bi;1c_DLEf^YumHsff zxecQ>;no79w%-!to&%ybl^)AB?p6fDb|4eu6}6`$xYp)qP}9;L%O>@XWlKJi^Uj5* zs}S^NsMEp@?b_tfmGP1raYr-o1%Rt!(1eNzE8*4~h4T8Nu{{;;MZ7I{7B6W&2<~n< zRQ<1@m$TXkf>%&2Zrt7P5hLtjwq0Dqq);PlQv|%^`k&trB4ARZtf5i0_}E5*+&!?| zq&sr2jBp-8KqCXqNa1DqDEd7+L~iPQuxR=rl+?ZO^I+=hdMdG(a0r}k@*T5RHj2B} z04tG~VxEPhQfx^pBy{Je=dyPNXUX~CwH^n`awz8~KxnI5RoYj--)TX$`bbBcmBSu0)}mY8o7!lT zw6Z%bYYx?H1Z9A`&s-WHnK-0i5M~Zz8|&KJFse}NOLuGnHpcl(c5>HlB9ZA>^-0uW z^nq({by03LrSy`h=7!2APCv+9fGDjBO#H2On^P?U74)Z5W7;`PG|J|Vf1ufYMj`w$dU0*&9!*Z$f&ERf#en=$62rU+8Q>+8H(EZGuK zDxeldK$iH(n+YM{1@g;}m*QK+U$sa3S{d-|fQ9_>_my}}4e&kSMa(&TP(;j8xz*Jw z1dr-2a+W-C{`neNbL(T$KV!r8A{qF${?N}Flm)2kVZfQZzWKPln}1MZ;gOjU$s=< z=Mb(!!vbE>AjU93qT3pW7zx?5vU2BNr|Dm@RvIFySMRS=Q-LSMB3Tbn853@5)WNA* zMYWF)D9clCzf)H`!b*vjL6h7^Y?&_H^E;)&YEBa2Y8LYu*}#{DR>wMYU+GDj_5GQO zvQwB|SjZX=YmzW76d_Uw(mYRL%TTh}!PyBqCWkaS?s@^7@{x;@zOluAhrhxptDT$u zF4Dtj{E%hh2s6EO3BziznKMrIHFR4ivS^QyF9jjFg563R$+z~QRqUFC=sD%2p(m7I0>l4z<|s0u=T)1_Dp9G?F;PlcA&C*!{^# zVtG)gcSTvHwKCG%LT30Dm)fh^_T{;#By$56`O$^Cw@tc5S>yhdUNpd!G}or6k)sJ^ zsF;dgxICt;R!Na#+RS%J?4gHbPNUCA;u5khd^F0*z+ytz6sOSLEqn~+o)S;D*E;#g zWuTyh+K!)H@zO<7N3(X!tUUhmbx+z(@nuBR3UO`u#bfZEN z(OI5lhD5eVN-DDLDu;xMz!BXG%SMuQ=4*-Kcv#iTp^t4`7Ax{&!mmwPa#F{cB(Y(t zS?bcXEp`#DYJO^w5@%wA^fsAt&b@nC>G^vcV-LW73D+vwDIG%NVB6$W`#z{4!}qIn zGxK7CNi%u-;xb@hRZoRXGfB16YVyVR6awqlgnNOSujBJ@Xw*1N#Z$Z2k9_IS*7$%I>`4l?bjSADpDF#l4X;lNB}}d^0UZQlJhRX;8y{n2{p)IxlAkS3eMjLKA`^1gi^#Dsv5g zp{Au*Zi~M^EmrGOs_XwK5jMIO<)=XT*k-k&Vd*-{P)Z>#r>T;u;$G(qLTr-g^x}zG z{tV||Iyrj*ZtI2#2L0$_nrk>Ho6#`sPJg92W~;nK5|y8v_I?}l(LI|UGGGr+`tgV6 zqn3No(NKQcw6S8RuFPD2QhJ58xy^d#%mpD%rvOqXe~O~|;`#+<&hL5~7xVo7(VgSZ zVdgt-EooEXuBCvtKTj{~-W7~{CrE`k`*Vpx*Jzb5O=O;NaO<0baR#~_igzCeb*egK zJ!5>BdBx08>muebcGf_fy+*Sxv;ZN{Dac={kiA;5vOW~2R$!TT5EItcrj}_(>|isV zfCw;o_cfvGQeH3w;4}1r80`35F^V&$S6n-TZ4{?Pv%}8|B9g+ zksVmpeq3M;8q`)W2+=cmO5Glz+631~-KCJ_5q*}Xf1=qXcffbvc%2t==4OMBnJW`p z48UtcC##^JePT<3cj}x>rA3vQRVi8@rP9C0|1}c!9ITI~fpL)H^CZ3QXyv&V)pnO5o$E1 zra59SAEzkM6vIH9XaKDVXK-h)#kIDvw#URTYA@a?6kAO125aEcnfRLimISw|zZl-S z*HYjHlX}lznhh87z{3{*(C=M(5U4JjQ-ah{C3BjSf@Cv%B`mX+=sLhcfee=2$`J2Z zp|0X7W!$Z9?CFrayE!6s(AIPn6n*i0sEtNowk$-?(|K_zR1fAjfSKj8P_Sl+w=#J- zfH~a!SGN4BQN($Zywrv2CXf+rS#8JBl(6D|vGz_;mPOl^XxO%GJ0iojZSSzn4BNJC z+qP{R8O(6zi*wJr_3E5gx2jsT`r56n{WsT|W6se>$9}So8(PGxRWE?5nwlE-DqBn17Ey8&boGAcIgm(Tb((PXBIh95~TbXOAy{6_wxWxjPIQL z0{(wN{y@Hd^KOI00hy)n{X6~f>cQ5`WI+y?Ajv?|zySXNAYdv;IAB$vP>@jIP*8Co zaS(A}1mI|hXy9mYJQyrUEW~smE;uG+2d;b+CTJ#XCU7QnCU_=%MeuTnWgs&kD_~Y& z37|_L2@o^jSP(q;bog}8bXYD32fBT(e%^kcevtlq{}6vvpeP_p5Dd^kf3JQNK~O_# zLu?`tO$1F~3$SW1HehvNbULyUorqv$S6SMEg=kQRV`|osd~mZt7YwWB-b%v9cN4Q`DID~Wn)Cvt2!bg3)f|g3 zanLFvMx91-D>9*sMGIB6BZ@DZ9)?=V%`0uZE!pO-BuKGm(dFoPYe-O9SR*B*h{{Bq$qu) zEPbTPeWboGS!z>T@RV)55@+vi2Gj^9GY~;VD)yV$wh0L>>1Xw%B5#fr2~0L_w5@~Z zhjbeC7A3tU18j%P3Ki2yY&uz<9!vyq5_N0%`mSN%5S*;D9B=`h0Nh#`(1RYBkl-HduwBN8^Q<>s^!SHm>4judESRyj44NufK28`pkz zgF}vdDNa$H%_BAn=kl8z50xWoT8d5cD>PErvk|Q|?g5mG*5S0yX|G5Y#f7C4AtvWJ zI59Kl8R~ju39>x9K(t}a<>Fz|p`=7f@C;*yKl&^iHuUX9XXN>Ei?%Cx@LhXIYqij8 zorK%ESlmL0B6PZKB#ym65_pQ>j?PoLG3_1IO!u$ut#=!EO1@lnXnGk{IMR5b0XDahY%wqR4MiPuU%!{C>LHS#8MU z7p?vC1XA&m?j~!tawJYt>YbVzI3eA|KC-(e!Vw6NQC8x!tZuC-Pmp$`HZJKHz)7XK z2GI=WB~$)j2L)>&w!4U(+TOQAr;8-x6(T$GC<6CNrJFlut1~EedIz_$rjZnhpmhap zHvs1haYo%cZsBuTmb_DkaL!!3=TesuEer~;vDzB$Jwx?W*06M$_rJPN??-mGU}Al# zbpSK|yBd7Xj*HuY=Im=a@hLmw6iw35jn&>*EiMxV@V>X7;2QcmJ^mg7k6dsPW0>L3 ziXs39@_B|pDj6MgC~F-+=TV=W=S*bRn^JT-X*6+oNQ(0&N32myQ{`E(oG7W=-x)DP z%1uNaZk~>2Q>`#ggwDQRJ*o`>GXH0OPkXIpWM{&vVx)_SV;Lw6@NkxlK`x7bfS+ zOGCYbjEHE<#=}R*SqhmS*|8eJ=EFK5ln7@MH^*pqZv``v*sq2C#N73N2}Ty+^#Zj*9lt>+aK2_LHDcF z8|Ve*9rR0S-q)osAM#vC2oac|R051oIu6<^I~VDR%+wF$8hDDJ{<@?Ls0--xm5qj7wo$~KbQ~M zxykDX=6hAX!nsOJ?DfZjZ$@r-fePFVqrfhQw*p^?0W+%Cix{3;kF!*3`0G=;jxKzt z_fLz)>{J()iJ77Blk(N49qpLPk5AyCoE2%0TfkXZPhCiaT8z`)Hn}dBYlQ7lJll)E z3NC0G8mlt7YrGVWZ6WYyS{fa)f#=slOUwxdIriLPnIj$H_*!J9j#O1fLlDIB@@;;= zPOw0V=1DcsNCL0Wk+sweJzITPBrkl}U-X9V?G?F?&Z6x#loVliaYXbpT5AiOf#D#S z2+k=Nk&sRu(@ydb4wnfR8CcJ`iA(uoS{DhAPJj>g{tf~|_Gryq9!J&b~L1nsQ+7v+{3dJ}j&m3jel_t4aDd z7coh%^`4K-Mn{EsN47NUVj)FY3up&Cz{O_7fxHtI;6jRa?m-AWJ2ZadJ2ym>t0`|_ z#J!skcp?5A#9)H^%O>Fx=mtWR@wwd$OO#BF{K0m z%Ty&AU6;1FdX1o4Omww)z_m8^1{^rcA2JwoCnMB4gz!a-PHb(BXvO#fo_ zh)i&_CJM;;&W22DPOss_ISV7sY)Ye^p#!wS%rwU^m~Yle@-VwJ3raZ6w4&Z@wN@ty zxM8YCfU8Fg=~Elk50A7?|1MY|-=s>ov|5RFAtEjOm_)we7Zg3Ck$9r${Qj5O3b{v$ zV8LXC1V+B&*Vyycx8cH$Fb|0BNzn-}`Y+o1rkH5QUYsLamoB8}pL;NP+f?2MY4%C0;9H(~9n*7)BGO#w9?yb8Qh9q0lQb5lMBWoBp2Pk~RLjt@fX zZ-sN6<8!_>bKH?W1`}Sn zmfgL4PQdfe&de11g)J7^ImMO-{5NRhyD=O07Ui2??2Q#!?MLxhG5ptK%D7&sMqHpkfm}em#opZ@^RLPBUicrxf4yAq zFmY}*eGeR?{#OIX|0+35dCG1{5RES>v|SGVDM2|G7DJ4T8b_@%ShUavL^35BiK`i< ztAbS%^@j;7tKAzIf2dd&NW(tpE7fody5d0XFo=w^xsT7RZ}$DmH{=4mKGefeYuFTn z&>HD5G}IcF1y@tBI2nV5KnTI=NaAs;ItU8bAqrpoA&@XyA?c81u4RQM0JX@KdYTv~ zF=$b8B+S6x&>03dIM!zk|KWF>r9DzM2wh=PLawnX^(Zg8_z>in zdF;%G^;LK41BiU&zs;c+}(<^yqzjA7KqW zz7u`8zd@NJKN5A~ZHc5mkB3?y-%+=qQ%|eJ-BKS;b$uV_tm~c=S_LbDS?N`GQc6sm zP?qz%moL5f_Z7!_B`ajBEg^^8IcxR$Wt!YoYBoRu|2Fx6MB)@F00RQT`0lW%{%76Q zzek^`>uBJrq5VMsgP>S~O~#;KOf{B)su0m>ZYj|0SZNOW9-%3shfNT}$(apK2g|Gw z9wL_e-WRx1=j-X`ekn1$NC~sZGN>Clo@USN?zNfk&3-<=)b)Y3$Jzs)4$XMP?|$;^ z4nVTdw~=k9-V@(ShDSY?M`LWKFj0t%>Jh(DMSH($4z@=_%fd_~2U}v}w=*7~Vng#H z4Tc$)L2qL_eyWbrotp9vnl#8rr>A`agt#`4urOI;Ms)Hj>kGhe2oFEWO?Tc%>V`)M~vAzS2pbu@NrqP4}NSjSbmAm*U? zG(hfi<(+@=x3!B@SA_Y@0k|+LOt#EAQL(p9s6$W3_hAAuZnJI8pIYELYSV8Mi_V<3 zGg|ROYbBA%igYYpNs7AcbkmzDXPQgAJ;Ufat5n5{Ou%dEr+Q$Vs2WLxmowYk!u=1V z;Vtq^-E5OcWd7(A=Jcu16IPk!M! zj18o6BfqBh2(3ehJHic}ay6*l7HCP1#ThSiY1OTnkkH;B59RN3EKwnlja8s?8lyjN z4qt*2>HA$bsj5WTSTz{Y=<42}x8sD2W+EoX_TZP9x*4>2Z?T77p|v6`*0f|PFXxt^ z`YCG1pVox0Acux7>CL0;2eFc&Vy|3qAKF}~gYM*<={+jG=ralnN;)vF#it>;#8-+P zi3V|4q74?@=YYmbGaQ|hJF62y%?W;)Y zPqtr7>dE;$zxLnn8r}-pJl8GwVm%J*=s_WldxDv5A$$!0?&$NMy<_;&8&D&ZK;^TC z?K<6ac!>Yp9J4G%7c?j`c>~HJSF}AZaSN{{=FUYg^v(VVnisSHyYPZ>q)2BzINYqG;|%F6iM&-q&2;F=S0|{N66=X%1Ad?IU(Y#F!UycY zZ+dei@&6t^KrL3jG*h65KvX|2Jm)x+3Jy{hZmn+KP!l6?uNhl!Nzt^6Y$)CbH_8U zKrUfF?49s!clbn9#|KKPUj-o}?EZ-dxV!#@Ngd4oiU+*A!PK#H2&e5rRlD;2`=|JrVK-;RL>XYj=U~Hvj?Cg$c${02V@FDzymV{q4@(eMc&hq z%1<9{;Tv(ewP-mM1aOmk7aAYQ860h0XhKQ2v7uqfl4l2lyaFsM>6MsRk_=IKSV}Yz zrDtS0jiF4t@(o{U-Ik><%A_q*xwm_4=or$<2KpRXsv&J<)>z4MgxYXv3nI5RD=Hnyy5Nxk^ajvLVh{qkQ2d;b zdjx1_sqQlCzQk`Pif&$lS_+zDkdk+9YN@4kpzQro6G|!sy_@oa#2{JC1t~rt5#>^L z+0<(teuN@@IyREat^w#sa|k69pX`MWda9=(5`3d__eqmY$tq={yIu{v!E5kh3%#p^ zhoYbYYWy&9ym3-8RyQZLsN{4xxWl(iI=*GwP<_I5`FhnQ^+1nHPl`P4%CO>{-&Ry3 z+9&nU_-DrW=sbl=B}ZKxvU&tNM`To!$*VsO=XjcH@sLV?K7wqwv~evPk2+aGnuufacFSx>$0rM zSvB%nENmTgt)$t&SyLOOBnzuir%{9IP!_nBB}=?jD!GmDA;2or4W2AnG+XL)oKq0A z#k<`gx=OrB`|P>?C5f%RZhz1>%uiNAZnBb-=<9`1IJ&;IZk5-EBtqCZ>(^r!2zPEdK?7#g2U$1G=-y>d|4L<;H=wl<8RAK`@=hXQC0(RFt zZ4qG?0PYY&t1_|jD^X!oze0TUM!6=s{Wz_JyL|aO&1fYFgt{O9@cOI)!b+-Alqh<$PqAUP>{M1*PuXiqsDs8FG@H zpulqvr$A@8f=SBU891=14$jZ&Q)5@y&23&V-&wU9B;UMT&EkBuTRk7#>jgNk`De~# zu&{6IkRDvFgNIxM=cM2mf8=by_At6s(VRdz<h^o&7Xn4!b2GPuOp!>4SUKx1GwkkrsVGh(+^qgLWGr}E0K ztxr)ck@aNR`%yu>$(6I1F{9=x4(F+19yHSEb5k0cXcjdznb7k4pwm$7cnj>H zpVHf>4@UrNr;W7e(hQr`@K#ZLLRsiqbyQVPuH4-99s)M(eYpLkHI4>Y$+%fbn5 z71SDy^T8eS4k2aoZ`?CqxzW*7JxDhHaTs)TEa) zH1?ONyXS(%?&anZe-$;(W@l#_pjHU<&BgC+l$RKs!q4L`1QHJ-y{wCy{M_w_RWWaN zv7=lhyxvK5s6c{ipM^rYMPu^vWQeIY8!%nopEAAC50=3e z6{2(Qi{IJ9lQFm`g)m%J+!ciOb_v_w(9Q|MLyvssli z-Dw!bXS%vQ%J03IoDu#A^~Ci-g=&7aC=dAaz-5-hBV~ot0K8U|yWcjyCy@>H)pkgs z^Co_%w!kTQiBm<`4?sxC`!=%h_iuqC^#xW!eA zCvgQm;6k%+KG{fcom~jfh@dOd7?IU#h4ocwzP3sLJS}HXPrWlUiA|+O(iOd|$!c+g z^Qko3WQ;fJIA-zrQv3c;z-Vu}WlP-y@B4-2voe;m5pul|MY*8uaf_EZRcXB?jM`qR z`Y^FvIf&F^A4Zai%uAfjENknPzWJMd0n59Vp}D;05>7oEtvXb{Hr6G^ld93>3Ojgs z@x$gKzK1sI?Yyc%uWf9N|DyM|V{-(vxJ~rRFd$|MyJ$6}p{&#u#IAhRu^P?j=0*Fe z#n>{gC3Ivlq=lir0kDN*|A}lK8e))sP$J@dy(j;`|894vS(y?25GeB&sL2*P7B$#o zUV?v1D|S%(qw1E@j{aJwuvIf`>Lf+gw%oF!A*E$Gyd^~BEbM6fukGm9n`{axDj*4yh&ws-& za3FCROTXJzpl=LA=D%)VU0p0~EL|)C&Ws{9mH;~!DLXU!ZvmSF!08_k{~>8JHF7ah zv3GSc0sN=mUQLV;%penzm|x|hHkdaU=B>y8ZZbnB61W1=QbXymXkv4i^8qk#a{49u z=g?oK>AStFFAor^gfh+IajeyrgiS9%CtpaFtqL8sm8HE88D=tCxBc7$E)PFgiJIcE zX9;@7e0mfgw|5@@#2CmzB$2$ELt-mAD3*E*^Dzy5H|3n-lQEhu?9%;E2PHOEr9gju zL^+rMdHP8981Ujb@9ugt`o_PR4Dx*5IEa5QMdG(rn%V!Q<@o=tn5T%njjOHQKi2U7 z^_EhWkr@(1`co>GmC1iTMCoz2U+|0qvrD5N7$z-NFmJs>vDHbwk!-5+WbTQ=`;TOm zLpC%tFa>~-dEeM^+I8;cnW_(SHt@SoXNh%2ydJ-v%xKv~MZN>8t9*&5` zDBuDdInsOHL@%RUC^j;-LQA4E64)8hVC=mr`|xFj<;ikQ=Rw+%lnx5Ct{&)theN?l zE)su5Tq}B1Ml?$rl|ftg8 zLA+c?#PuEobhnbw02^+TqQtQtdbbD7{SjAyuKDuexUJ~LA$*7ie?D}oPT%pMda15i zAif3bj$qgg`p=FaP_BP7y!-kkweRw6cIN^I1SIu;=S}|eWwof=I-{zg{j_VGDV^Y; z4*@}rpS(*f4psykE)I;xkhOtDp<1j2NZBpXPuOkmBxCHM`(U0xzJli%jSeW8QI)W` zyGo$Y$>-a60nb@_dLjgk24Z$BJ>XIqGICI&uzgu}f$pp=2V zWD-&r1G;x%7$JPI2J83bKO7+Jp>C^*unRopM?DF-9R}l&b?=Vkbd#BBc1>7z9}aB0 zN!>;Px(^Almm)}M;1e*3sKK!Is4NdG6aWi2Lq+Az*%?!z7CJj-8#xTN+AV6|K`s*< za=XnYsUk&&OXHB@s3R@&&ALq*k+J<`?wZlbR`qartJEArix~?~y_P8o^ZKjS4~Qml zW)3;=mTH`=NxfO_<9yPB4V*S(_*+ZI@GYy@Pe62I#tF|_B$K0Ra6KAX&e?J%@i84e z1)Vcr-9x1=44>9H!$Ak;~XR{LYwaY@y@szZT&@8U{}mS3)| z<41uRb-(IJ7-*ZHa!V*w1k66#v<1MHyuUR&fes5pAysaQ2_IXyl+PRvd+9TqdD?u~ z5(|VxAya$4)-k|8(3h*!kn=ZHIFu6}GqwgAnxbm{P`r}z9mDux$=@zrbj^A|GP>+h zWgxaFV49{q^aZUOQD*=$@LDX{$QRWR;8VU&>O*&g3Z55uYcApIN&O19Cgok-aA&%$ zc=w|hDeNne_+8bNO7WYsx%MhP&=%+e$vq=C;2TGXl{g2)9SUl_BG;fA{%o#{J#g2w zO@mpkT)_a5E4J!&1=KKEl4!Ka-0K4{1}(KrpVA07(x~~2WHr(xa#Nn!4E-@xPipP7 za%v4uoGIPj(AK&DD7NC1nyav>^so{QSaE93#$<8{Q9j5l&gIvE3>R1t6P9y(>xT$DZ;I-R4z5Fl5Rx!$TqpN-T#Xr3dj z(?%oJoUBQHJuZpyjBmBxdZ4Qls%TNoOn*ROEz{_5TpqAfbLvzWorPzf{HLPR63DJy zl(vj818GJVYAJkq+S@|$FIjKO>ThE4$1k=m5qr#@Hqp}o5*q{tq3wiRPsxNZ3*tWn z!z#>6GLw*Z#%ga89z;tYtEmQf8Cm7Lb~sy>WERo$Or$Rpx1b&vL|u1SyB`+tUY(rM zhzDJ`Pi(jE_PqXwy<6=It<|gTnBR<`J=Z}lNs&T-#09SCv`QiO#jHQCMBA-^k1Ug7xa`hCHm{ z8(0Om!N3{tm?HIHZ1m*CObb-5KLKu?R{D)^cZEm3e9$^GU~rI|Cn9*vK}B!a1&dkh zy^-VhGgcJyY+m0Wf~V^II$y>gf${6c-2~I;FUY@QQ7IE+6wTiU^uu>~7yUmi?f-2a z^UtatqiU`4k2wr%O8`=XKNP>9-~tDDk+Ki^z0!rOd0<^E*CixHu?(ebBlQFZp}_bY zH)-60Qr|}ro_mz-g|me@XI2m8cOAdo`Fec$U2hSIkEhO~X6A0|e%@TYls=h=AU%r#7gq*TXj) zyiJdvb&OOB+&+DM1sZ5TSY4cjPhefuL z2PSmJX6oR=yx(aY%he{(Ymo-)an$nXx8JEr4l*pXZhe8L{)ww_P-QF5gY=;U7uJ>5%SZ$oG^Xg5uf^`q=T_8`61kpQ_>kv zf#oNC)i!@Kz3MoCQqu`r--Y}A>#iv3%FC1f-W3qvyMpt7XIE%A8997k;MM-~YX8rT zkfN%ifGULcr?jnP3InbD5MR-;t=UEiEM#+d8AU2`wApaE>|6e?GiA4vQTT_Fa5>D% zXWrc}`CgX3q&cOZf|c@&XRl-WN7LiSTwESdYP?25KLB0BLCtOrO;^Jal$A7H4S7Hp zkJC|xq^*Rz&KSoX?HmR!RCD%OH^Ch0Y7{}Fc>mWAxXdX72=$@^Fx;R0Om%5is{w#j z`Z_C&ON_HCzW&FZOp_?rBX*JQYjr*uHqb~TIIa`rx4*9=nY-STWA$E@vO!2e+Mk9z z@CWY%8u7+d$28VK-Jga^DUjZkY$p^o6NIE&*Jn!yuHp2OE>s_`op2=3U03I>3t z;_acvzz-bt`pN@f0xM0>kf{UYp~RbCS&;HXx?3#P-d%3qIviPpHC=qx@U7Ed#Rp`JBdF%mKurusEwb%1zjV3OK#St0n=sv}~7yrcVDRb!T5wMaT zcq!f}_+Zs`9aaLz-l59b)RIr_Zo`|cq4}Y3(X)~)w(q5OSnLh@U{-VhX>74SIR+u7 zl#9kmV4p?<7gf^ap%WF|__gE^f&E66GyYU7q^#_P17JmQ4QY44!)w$B4Z%vc>iD#GPdsP zUNYD$--8Bbl9hSTl9hZhT1gBN7%?4QZEp{>s+N`-z8fP%ZyJ}=p0*8A z+~}{ax3Aycdz^dIK94igoUD6k=CJ}!P(ZUy9lZ5$`c6agugIK?&W|vh0vP*hi0#uI zc~EyMi0v~2)*zqLpou?Tzc>5-sVnf$ivX5abi|+Z{r0?yMu(r@?ERwyTmqawxki5b z2ZTL$lpag4exZF(cN&P<5Pqop^uT`@`USvV%Ygo(?C*hmhB5D+lJ6btIx=Pp-jNi> z;C6iDr*sowj_9#X?F?}Y}# z(tqR+CR8kllcjjA!c~7Gt*CO!52^HU;VYotyC8#--&XaisIKGm2@f|_tR{nEv_=S9qo!}Pcl?henQ$n8nbl4)QzJA*MJNS*#xGP*nOn3BE z!>K9z1(L)qyghJ?R!n!KTO-ONCEVI$qe@J7Dqr4ibt3~aXm2h#(O-|C9e6FOJbe9T z04a1Lv>axAsOrUOiPBhf7@ow=bVQ7AJ&xH4av2N{$q}_m)jVVjTsA!Y_|{B)7h`o> z9TDM>z4`;up-FBSv2mB5Y(EVAlj^PCzt&c>lnkYkKqNlRMv;RyWDsZF8Nh={LM zU*JBgMhs2E?r3Zll7aNq>1m>b~L2+$WgJ@CW{5xsEXve5| zgxP#lf+ksmD6Fk$IkhZ$$%My|Q_6cM3af)wWqE1vnY~!LvGNF~7%oq(E{mVFB);K+ zxwbe{n=lRi#Nlo!-2CUE?s)b67HxSsz(jXzIT%G9C9+_fM8S znjT<2z3XTa*8ny%B9|s{2MJ3j!&4JItUdImgu)B=LH5aohk7Rt{Vgg5 z?+$U~PuP|E-H>{h=SKwa_diXaNfWi%0ZEpo1D z^`5llID>vh=~wbkmO=8rRWc2xJY?Nb&+cu(JomI9Kh*}k@AQE!L-g)Q05Lwky1CJ! zL)8_Q8s!mSz#w0%AOQq&h*(MtLcks0XW@x*BAwEvzcL>OG{9xNgiM2M8K|Cx_kb zd%?4XDT*VwRGe&QXjhv?wgJv4OR#Np2<{F^F= zIDZLpACx?lJ4@(1qafcb$KIeh=~ul^0#1eb0-d5!t(3hoz#RF(mBoiZ{~4JNfq~$< zX++nLU33la56iH7h?+9WS_%-p&ec44M4Jm0S9ZuGcg%ErC~ethNsnk6T1%Z^X6f>P zjpDwCN`Dks$Z47@bO19;mL7iGA$>Ku1W3N-rdeB% zKbnIY_mn7q&y9ve$v;x z)=EaxDwA)bEP+vYB@?w+$tG;{Ges-;e=s@=?c`H} zOc_YNjoOVvmuHyE8AKV8K3Dsvr><6o` zRN>V&=|42`iz>(b11HeXGt=K*^40q@DA-T5b9ZK3+H#?Mvr6>rvPX(P&qORI*#0n& z{1X;WgsX&}hrqm5=$kZ525BLF($d-K0+7*23-&uD^K&rBxOBlpEI^VTR7~@ohS2+|@=$t<0(IX|y^{K*@un zL9bj!&<&GsTGu^-7-&6QZPjJ@D9Xn8mAyYul#NYrKY>RlT#ZHBs~xw83(Ju@SkVyRt}B9QPX5fWBq|J+JnDW(c?)#O`5 zY0xity~)QoTuz)KZzsHdVl`1_=uP$mr7WQ#!0e!m1yn01^HCR+R6Xkk#qjlolU?(+ z@NDo5peJ+uf?KE&|jP zCl!L1Xr@Et6t=;cRS>u=Egq~h?Y&=0_F1m6qRNo$vUV^|VLfnS&yA*-e!<@JI1e_< zt-H#UCy}*WV?l(PYTV1Ba3V@dB#eGfiLp|zEr+emVd_Wo*l|kwu{3*7Hm*F*gSULa z$txIDbs;nNYAzXx_$3_(!sa;vzVU8#q4I>V*op&N@x%RRQuhp5rk206PG(*ay;KVaHRn8%heJ-Gqz%){aDJbqg4)?pqh}E1)m4^3}|8v+&_eb;o>MEgzlAV7~1mZ z@}Wb>2n#16XHw5TkDQ!B8>4_-pYrG<3ww2Kx5`K=Yz#@9Bw5lC?DURwZ^4D5$_+-3 zbtx1?aH%Q^qAD~&ZPQ}ufI^kOf`PMP)(WA%alAsfQ{0&8p`0g@_ zh_iDeieZzXPO@DNj~`eEsmG@VM!f`20K4qVn(f8KO_a}gQzx=Q5OkJWnkB5YC&#&) z6TMWpc+#%y3CkuMa9|{a;df0pQj1mte>+s=8ihlEu7d&ocj#0B6`xKZf_vUZ>4sh9 z_CX){$p)kq29t=1qc1{7u%9FrJ}>?HR*Xk$d?r7^lue#KEPa5)$42AkUkwTVN;KUO z7Tp(QGv6I$G|VC&T@`eU9Y9<4NfmK%4G8V+W`6mz|3vA*L+td4}(>MDCb6O%qT%iL-=Ia~V*-cygyCuD#bJ_%4I}iPh zK4~}#PdkC4C6tu!1U`h<5DPah%VO`l!{%vFBTD7MFiZn(&WbT;=2qqDPUEpk5}Qpv+6cNz~P7%{i41VQyZik^MjwU|q%~<=?&zjxY?^ zf#TK%RJB+IJwC$Di&SF zasyLaJAIkzdVZpvHgPXb(T&&t)-!srcYEmU4WYK!K`=if#(AJC6n86aIAJ!6T5{M! zadtx6v=PK{i4o?h>M&xYbLEAYT3x449OSu{97qWRU~X9cNPum0Kr@>ZhT(=Q2;C*k zQql}C6uUQbWUMw-ugctMb=yF=fQYt~@*HdU|`fmvnq1jOz9k{ei=NxZ)I#=laj zZ|#s|V8LE`FpddwimX#{TY9AdVMiwn_Hq+|JQ^4KbNE$X+cEs|bAvt|YpXz)Az_Ck z#z&7@Qvf8g&jZRAd&jluHqrx?_*O~_Vq)bGDH)koU49*cGuGegM$xUy1Gl7+D(yOm zAR~IuI&38a{XM-d>`ffzUhhUHUHymU=6bpH>uZ%|x;<#bj&7=c_4LnPdr}b}r;MS? z)G2^`m5!19!^cb?rVW1nbOuk*Xr~o4*>o!O%|Q(AxIuB=8XB8x z{yh`=Y+Hz5H|unzykYV-eQ87#^x<;5$Z1EscrRUFyBAfpk=0EBNGzH)<2FmA(^4|$ ze~B(<4f2^d+A^*+*Yt7qx_*fbVUqADBi4vMSo5}w`>36F~Us^SRNi{)(u$?C( zc5M3uJ49q^19iq;p zJ&`u^*+G*0DbVl)AGGBVs_}`E!`hK7;sCwTbx5^5f!-VP(!YWCa2%=Bp6dpp!gY%u zjJrCH{`>2WnH3``o{%%x=0W8}yh-QcWecHSPEJ;6_bAT^$c{rV1e6?@W(;UZamA)U z2|PK>92a*=%qc|o^5tzx`7Yw%sO=b#ZLD5z^r5!h#1dODu3!fFC+Y#@Z;fZbo`W@k zj}_e~VqSY&=*s>_v(iqCAZf_+99BZ_Cy);Zc-|KHCX< zh&w_+)1Uf~3tZ`e-t?DoU0izd*14Uqx(~}kGfS7T zIL{PC*X{P31PYMuP_w*vlCb+_XRbo&!a-HBds7s&@}mfhMJj0R~yBAU_P>lIk( z*Rb_HDbXLye?`n9R1h!ezrkE$cp#u}={Aswy)C2LKL{5lMvPXb#wLvaW5n#A09lUu z)jy`xe|JtBq&mB$g$m34CAZkv^PyUjX~g3NY35he?>xIYY}b~tb`%aVi^BdCNw>@{z4o_}h z6Nd4~0|q=>@(?;ZxtMCAo~%yNiALN9Cj$Ad>o_*Zd1Gc|K&V4`lwu)Gut{X{fbCs|YCG&K`iGI}~3uEc8y zGrwiH(ZK_X`;Q=PCOM-s_*+84RA+$Mi+orAfOe`x$84l{`n=SW~EmlmF+{evWgNT1rDr9ZfYzhm7EZ3gn9h~C*=yxcRx-qOu5}7H50F66QU8zz1 zl7w2imS?gUDLN^#aZF6ESdV&yQ#7n{*rzG4OdkVB-#J&hM{}ZKaayC5!tkZ}eWVro z2eO6`9kYR`Mql}!VOQm@Hxki7Yb5Mda$t;xf1nJew!A%Z4y(0}qe!cDJm!r45>p?I zhJTD5+@Smw#h~I9XOHSt{apfa@4%Y|j!|jWQ)IX5WQx?_PyQhgLgo3GGYy2GCDyPJ zx=12#9A-I=$f$(cs%{Y+&#x^Cw9j^NgrV##_lr?n*)`foZSWdE=+vwBSY`C3%Yd5RGJnalw&9yu^!@mS;y!d%&?eD6l_3e>JnI7Xj+l=-anF<{7Ov9tIQ5Ja z9Obw=24VbV?MdO`$#s^?f~(T59?R!F*fH=l=5yAM^R}=e;IBdpcyjNcrNhl&ZzGuz z`@};W)@ZJ@Z^PTrYTjL8D5dg{TB&p%XGH974)clICWP6kcY;A^;)$L(K?;>MV}gz(^`;RhX+TV zh&A!6R5M+B__!7j%lx-~z2T_&&K7XVvELrFiKT{6Nj%yCA_e|DIst696GI>Gomc9p z4^TIocPKYq7HdKuZZvm%ssfCI!UPfjy_5StCKZrYaDiVNx4SZprL?|0ZlW)su+tkIGrJTWFH2d+50a7`wAi`4oeJrZ=d(tOk5# zc}(TB-lt38cYX`~*eE%}_PlW>&=1n^c8bBMyW1nwpGsI!74Bd;FaEpod-D9{)|+5E zkmK$|uIt8}U^}>VdEd^HXQn&k`08#i&ocwVrk4dYq8i>`& ztGKM@w;Xg#ya%Lo$ZKBqSI*>z>Yb~*%4YuX*zy}^nJbx^Tj>pr9pi3YEzXDMo4T+d zo0^iXoh;z-)KKWSYwJEK1s!tXRz4n=6@5yTKPwt9EpH`~^;p4WyoSY; zs#ttaBq8cwQUS_!yq>XpjlDqLlqi%GW}FJbB3AnIDER;{1&VLt%y%0(`zTGJwrwqu zpClD#Dun`PDNC2yoPyLguRLqAR0L{bS>#l98M1k_Xe(8iu5`4zE6=do?0r8ZY5VH4 zn-7ogGi1&kO6Elpcgj_inC4U&iz?df;Ze+cg9m&=+XN4}D`P~b)BYlIj*5WSphiQX zo~p8FD(LMV+9x{d9y(9|ZVQ)_$LjIet)0iNERPQv*sQ7840hyn)A%&?7nfJgW*zNu zs%g4ulan9&YwA^BKXU)Jk^dznHXe)QHX8;zV171b6J)6;p1sP~zXfLanrirv=nuHf#Q0Vj%N6DD&q_^}`6b*R zb7Kspmce0wv^_1fS-VS)i_F3^I0X&gNs7N+W~i9xXv2xC&PXch(nz=-tH}}t2jjdI zs42CZ7;LI^`^V9<9JpJuCF(Cy#?YEpw3)>4WIUcR*A$CMV?O1OQ$Ev6(q6Q0^M&d9BCOJa>w*Jy&3IQz8OVa&KJlBowJ$RLKF-gF zyQ?N!Znai{ro89c&OrTow>kJ(`*JHf@pQPYJ(}ut;^=4^0U;z-(6gC2A=x)}%sFIl z`oncX4IO&LRCxy4;vjV_n^vn9eL4- zV^{knGN6Zq^<||m%G`{(b}c9kgI)MP>m>&UDjjS}<+oWLLep^5zO}zv^7pR|Hs}^N zw;4ZUoDzFFmM`@tJh%isSBHf#B6JsZaow5rjOWF+hLJE-=;J#w-JK?foiN}qSm?9q zwPSBwIM5{F)CeDy<1=~6`=qop99iOMZdmsf820qt@Hm{YS7xtSZfUi>*c;SBKMrUF zvxSGTRPiyo!nI#(YA3DdEq<+Ydj^*)x`F2xDS|!od;eAhp{5G!8!;Swvr>il+}=e! zEKI3@V$Tm{SoMT%rYPJbK`kIjEs){A>buuva5McGJd2ZxT~=RmvZsDEQ`^U+TA0n~ z0xfer$PBdwR-vhN_OId45}$KY%^3 z+g*`0pq;O!k!z)@rf$;ES$I)dl(j#Ax=cWEEL0gg*d`e&SU_*j$wQe-J@B@>Y^A&D z0*y2FyDn*Jp@2Cou9FuIVX$>sq~DI%W*86eAOT`UH1KJVd>KP7cjO3Y`MDu3+=@4f zgR$JrVog*}88z`1xvPnikdHc6*!zT)7vxKZ=2Ibm#Eha|Y&LMzfWcC6Hk|rfT?hxv zuJS1Y6b`0i>f8a{wex4j#1~%h73u?Y^Ljw(t_D<(%wtEF*v(euMX&Y^U$8;(^2;+| zc-u4Q>|O%Y0i#Q##?7mIccn7=o4da_*r1Nysd$6JGoxd1ht4x$xBHBF<=emCK~at4 zgTLS8%sJy3#=rQ?!%MfuC(xW4W)&Id1{EN!TPe`qb#CqkcXsDIbvEwxrPAo!%yS z&yQRubGK}l_0ye0=nHfQJ)!n{fpce#O&MS6&tmH#;ozPi+Y)DH7XG?mgb?G$!iN9IFKQ!j{Adm$hx07Lq|8poWq3 zAmT>i4ke|Y!k%_@KMqFPiY}Sl2uL;D(yIh!BM&f}mH0ZVg;v<0E9^w-oEKYe6NuU9 zhX|Oqsm3<{qz*K1c#LxSuJNe7)3Yn>D{_O+MrmdjD|3t(NuK7*Ep*Z{>BIL{3Z5Sk zC1K)EPE4Je5-zci3W43X=~76KyU$K<4oLz|mkY{INum;V_ynq{E>;jyLPbMyqkHS; zprA79lZO_eusrZi6s^^S@RD&?h{eXL!;E+))VvNTy$ydZTGAy3qi#~7m60zgoW?h) zh8NL*rHlAQ9*bD?MO9?QnFm@brFvyg=;&;bZT%=?6!_p|IIvuMWVOosqHD z*Z6WQ-K8_PrnMIz)WXLUB8u`4n$N^+bEzOv%)8``&w1hwM=e~6^)T%7h3}hS{W*jZ z+2T&IeZor57Z>TTmkr`^!X1WfS;A>oS%KP|_0T78fc@+SQ#kIX7>SApXXfs@ zdEn?+paSH%%#S5=VLm} z@m`@)}WP}fvEJe!6+ceJr;Nn zLRH857Ek8VfN|sWLzrAo!b_;qY<`9kUu2~oaU?fPYq}(@y@@F$-Q`9(T;m&tgE^Pm zC7+|%;BVd@fyyDn|KXp1po5}eYjg>~JCFc`kpBz*QE+p0baVZupyrV+W|(hzbE?VX&T&{{Sbbjl7*JbIUHo>lo#v}Hpn=f<684F1!;#1&Vv7wrn-OK63o zjdIp0-9NGkwlrPlxuP<(P&;?TJH#+8!-LC)4~8VzaD?R7YWCOE;Oz7s9?qW2s{yjW zkP=>|VY`s0%C_}*?E^{{`{$(}ie~pUQ<6QMLd?l`K}}GD1X! z$HyHqiKvwgu!Q_!Hm_H{Ra#!eC)8Ey-{i~Ff51*qj}oow1N4OA3*Z#B`6Oxw?!bsW zVxu;8>JXopY|n>&z+7W;tw~HXP7_LKs0Z{_*87HR!5m2EZ~raW2R(|_?O*Hco5dQc zqjvUOewXhz;hRnm(TZhUHRgkVrueVBC9Xz^uLA&z0{|!l{>5(je+7kuv+I9B#f}V) z5C9b($N9HKOzV|P*4m|U*qpw!Q0O`3fklXv4^G`Yhx=X>IdXjheg6Q(sf^XHpM?DH z5^2+AbHZ|k%VF(Vc{$fP1e4nzZEe8yL210Ylko!sxqZI)@aUlVH~>(@0LuWI1#gX~ zlBYhYF58=qcc!54c&K4g1+U01NU|ZXor>ZK+wqLWf9U@kCh!25Fq+}$`v)ejM76_} zC(_<^*#;!}Zz-)xg1FTfr&pR08D(!4Xs|y;o*0vdRzM?!?of<90spK4k%P}*t#ATA<$e(L+Nb_&7SpS~xGto1i z=x2##S~b=Sen7#;uvo5$Mox993e3XhA=2>!qs4hMQv!&~Jw@Zh-!VN$3j^{{lrO=#!Y6 z<_=edBc#3LYGko+UuJnfI9|E_apb5lh*-v!D@+t4gbJ29zp=HYXQTye4mt_VS<|=t z(@1+5tKWtAx;om0db+BOP!|JCSJ_Ex7!B+mx|YQf$iO3S*9f(i36sXgH)ew$!TQM> z*GtGQzp%X2b7-Px;-`0irH+pK4r9FqEVJDprTOe&#TnS9hiIlTyokd_n&crE!;NvJ z7Om^@DIj+DmPw=xRb6b2)3zP!4Q5!rA>N6Wx~=elyv#LrH#6P{Xq2ha1eU_}5bTig z52GuipVh*BmA&YSf2|1=8D6VnjyssPSchlN)$$z*v8%)=X&Gj!Z{8P`fKV_7^c6F$ z`hc6yM4WNmz-6;_yK(R4Z!v&-%|%$hAIFgz{{47M73y2gc`pd;U;x6YJYt*D`0-U=1&bN?O38N&~Ty;-~fcQepVA>5E**$l&(h&}hy`ov>=O85aM)p$}e$YkF`XT=k z{%XN$CBH#Q1f#a&SDL_Qz-1v3+0wDAqh#(9xgl0(vh`Ovwc zIn|`Cx)TSg#g+kPaO9%P@`vmdNVa_;-C^KGzW<72oFEQ`X8^sy0dV|3JoEmEV`U?I z8!LN@e}A!6QLyIz z#23R5U>QNd0No}w4@V5Adf0El>F(#vYo24@=gs%`hj)b^I;+eLk!~PjtMj%eZ4SbN z9Z?V!<7$p|yoDO1xlX&AaNVe&hMG80Fh0Eb>|0`2B47s);UjuE`OE3`R!NT(evEa$?Z%Fh)~iVgOH- zjyHVt=LYi#iw=DOBX74g6uSLByU6KpRqplxD>FsBwUw#YySd@Lg9b>y@N|}qh_&TI z<6j~+^3UHJ^{3_x!F$}O-Z-w_i3E6rT5%s{)rmE;mC}%E#+xXNt=_Q1cK9Nn*+`kH z1t2+~sr|OkIH};D1cf#dl;2wZ!L_(TDgwu{XJXT~NC|Uow;=29D7qx!*~Ua+J-UP> zASH~-nFged55dW`e;1|m7aehW6=jX`Bs3g38lN3w#8djjybr4=bN`cd+ds;v*PA>0 zBmkuD0rbfG|DPHG)sr1S0{>qbF)tY);Z{Tpn_;ywv2Ie+*-3a4w1x)fu%`}oHB&;R zzH-=*k*(oK9(j|{jn?6?2d&x_)_pG?l2Sx+BLZiR$jW7TmAslbKR^j&qb)Ns+!RGd zQOb(_lNyx8dR`sG1p*da`%y*;d)*|e!D&~hTaFH%?>qrob6J2=@u!w#hL5;}o%e58 zPH4lvLO^qe^Vqa>*Np88MLBUI1V52V%2kx=UM;McN@%d59SvECw2mwHZxKoikJT$z zGemE=y{7EPaFBk3dIvJi@RnRH_D0CoI}z%QGeKqQ-Owl` z>F+Sy5u&;EbAlpqircgZ02ApAuiX-zyFj8PXri*#N*SP)*!IJr z0e+IAZML4bdSj2fp3SFNVX}7^p1XXmxpvmB+Mg#)zP(Op11D}BD1h5Gg3fQXDb{be zzd9l!e7rpQ5F8M1#VIbpS8_Z(Sr8nMZq<3Vk3`Y7>G$#ERS)v$yI`KzZtan-or=&^ zCOtASu08gEx-uf3xhxMp;UD%RQ3(1%Chvsw-xEPTUl{BBM2DJCFmRv+{`e3a>>bei z0!v_Brttdy5yBR*BVk}v2yn+WSvw$>U%kaL=c6(nk)qFcHbO7=8jJKC3!2J2*lX-^ zCO~**dB)@wRh?pn=FAK9oo@;%IT~G$=k#71847KtS?pSRA&>V5&H?R&>aA z*D)2?Ks>=ow{6v>Dx{dSYgcC^vrkfFrDDR)@7p(;6@ELWKHE>#3K_a%pVRj9yUJ9G zTqwEDtl2r$bKOO!iUWZ3yz0J4e=eyXIWZa?=pz<;&@?So;*pX{*NusvG0@rKMp)dD@V ztU-X%Q;eOh4HT|UGm3nTWF{h zHZN>%yn-A)1#Hb{c1KDSyg?efs}7ZLkQhoBPP#ccNRRJ+fKHv#wWXs%^-74tAk;5I zI>rmjqN0M&gBVAB;LIv%q|`zHNhQUu*H_j>a20trt?y}fh0lerU};Yc0z=^*vOo{u=&BYR0i;%Wz6`$@1c2u@1cl_~Oo=c_UFOv)Q|E#nC- zka)vtpX3NHfO-qNKp=%_HbQ_iA+Wbs?mBoR63|^OKl?~!1WN=ZaB%T~D?&Ui^8)5i zwWsmY9J;gHitHaj1e6G9=LBuX&ga6c>LEhA=UpR`<-$mn>+~*9NdnTTc(RT;Fif18 zr8GM@CrVF=Y)l>lbPZY35GD2%h`(s}7(9gs47docO0Crgb~w=k0KQ25$Q$X*I&_pH zwg8X*5*|M(#L7|_CkqP>C_jPCm(<-Mob$4fUK(k}1_%PF~U z{f@YF56GB1Ue1wD;9C8DAbP9ZTCE1lo*eC{^yz1ml2Me|6u1wm9dkWvyo8aUa zb;}p@T5j9+kTOYX%Pk80^J_LVP=*1qnRqHF`N^Cq2AkLKFLH@QS!G4QtdCDe8gXv- zpHyTSS?3f2rB(vTc_NqDJf=+M9z8AbPgJ#eysDv7!&dsLW9I8r`L@$%?H?aM0t;Ne ztVS3*^GJQO{FFOKvG`hW z0|YyxGNAzc(%p2W9+-EeBvmT1MV1_=Dz&QpGtqNhD)65=&ubbrQ-tKe+%16iB{P3$ z2rt6jqXJj60qMF_;61gnE+Yx+F$p8LW6x%oGIuR11_7D zC4d9Q7qSbOkm9v~u8J)085YdX7Z`0s@w~FCo*87N2fOD5DQkc%y9Le03=ek1wioWq zoXxhVH&S9<-xQ3@nEfurvTZuC)JjlMluIZ^b)}h83ZL0!rtTxD_!~tAq5vkORuKBE zsY2)Lp_?%!A{SE}b?1m(vlWNXw{3~H)PdvjL&C7p^}f0Rv463e$jssKQaQWRWUL8Bk3blMW`w2-hXw{r!(sr4mDXWb}k)_ zH!x&qI@m%oQDO6zOPBwmVE7(_)5cqpry5S3rNZvROtg9M$e~_Ik)^E&IAVOJxaR0X znolpGH5J-n63XGKp6OaGqCp$OTL@g^Tv(yS+)*w`(M7(ZW^ja`b(Q&qkxhO^z>cqy z$2j%$5;ZL20If`hC(0S_qTY6|xHyMpHhb7^ip%(sL$fhnw!d)nN!QKu6I3cn1+iwK zURxaIu`BtZ3)&;oJDbXNvE(c*DaNZD3y`+~=InvIqZc#bmgO8*>RhLO~+06UfQIGj;9st9-q9O_}qc1_{4r(+^J6i zV+#iE=u)=qxZrl>VirSwPb^$bHJ|m3lBV0WVMp+BS-~~Xl5S3@Xi3z8)Ei|kum1bL z%?q{!MI3+w%2_wShxT8Tv&s%`uK$qNia6NY|MOb(&l<;0Q4ZihgqpiDG_+h()BJc) zhGC^*bX)B?E`BOS^qQP0s+$aSu$SmtJE1HW@2(e=n-l4#fo4`z z#tc0Wz7DfyIk-dSmk0$xoC-N6oW%Wl&o)j4AqRzfBB9++1@Zen=50)As< z5{j+0VMbCH&ov9-#XTGx_sIVS9D0NdyWb3o!cnHJ?h&=;zg-U?fcf2-0a3D?2>&9} z5>PY#-|H!B<>LC!HBHli@<3TZ`x;5-Be4o393{23q{WtQUZo_plCl#921iC&<8%rJ zIZbY0jRdk9nTv&Wm3t=Bn@?vale3b-SPyS0YZk>_Ppq|C@cZWA`@D*+C-dT}+ ze!u#xCA^NX$sW8D4Y| zA7Qi?N=reaD~&Xt5w-{jZ**Mv?W63DR_H=w|n#!I$=VD(Zj`63BQFgzN1aS;3!8m<8ST7G!7fSn(|&+=j$I^rkKSdj4- z3(a8mMvUk^IlK|Or+SaZ_`6Zb7wtIokPFU0<+dxjjs|i~zF!q_B9GBL)^@x;W(}&& zcx{*nOixsc42%ObLv4oI%q9K&mYUZH&2F$teT-+-y_+V(hBf zKg(<#Fhsoo{Fe|f%N=)py;uD2K}lKlm0r4?S0|b*`8UBx>74fV>Y@=g`-Oz&1OdlA z1(m!Icb2wIBqm|OjBy~d4cnXs&RPJsgZ#uC-2Rz&n)NnH(542NZQZO`FsrY>?LX#1k}9*?V(Qri@CykgF}1EE9L&6xMGom_|~sYb%JpC$-TXKe%s z6))lO4*S9Jqc_39b~Y~JqpgmD!>w+<5IajwNDyki3Zu_Xf};qIilZ@Yq{tMmABY0; z4F>f7v~5W3d1z%5FHzA2XPJXw9e|K+pDOiVhU%yfwAON-^1{?4g~j3A1`JNIP!{MP z_h?`sQNME-qo<#S@H6^pRb&Mn+MzOlD9I{Y4EXjZ-SHTV^hYP!Di17Pyktty8^waI zvTJ2751D1S_50b-_X0^<;x%qz(0UhdX`E`p4bETS`~j&Q(W5K&`O=3Ojo)l@Z#*CQ z`(%GY;u5Po^+pM9B)obYUt8P@z$o$cfMyAp;V-cSg6n%dz#}KlxaNzyw-URQ0f)EQ%HX4@^HwgS|H%q ztF~?w8SclPXPJTF+%HKFH}P3%6i>oWJetjSyRB>tgj%R}v2`+Eadg6gBe0mK*nf7k z{bYLjT_h2=5aWAQT#I#L-g?Oj4R&k&OJVQCm*glP4}6Ic0#)FAXGasyd; z#qP4%#$rf>i+qEnI%eHe)mEbx(?v_0%t1$-)e$xa$apCqM>FUWrP*8(Z`*qF5f@N& zbEd8|rC&x*ETG5J@V;d2djD>4_9QtE_z> zMjYD=ZiY(d^#Y$DHjx$5)_Q7a5-+=Tls=YD`=mMr+aT>G;a*Lu+d=tj{v` zg5Ra`&#DIor~OcNTnvhNK|(UuPB^r9#-+wZH%+Ahb8tqU9&@Nh@~rq7f(m$)3KL~z zIiE&c$gM=HDsz8cVLBN;Y4DnT5pwC)@KlHop&k zd(W=(!rC#>SUlIJ>gd$5FT!6(G^g-mxCWiGRasREp^G@(x3f035i=LODDD=@M%3;M zP-{E~Z?sl8cQ#Bgv<>aioG>RH!cgZOWr^SMs96x52YL-Pw^!WrDR$f}G2DK1pZ%YL zwmw}p_-pd8l^RG~#H3D-(Dnq2mP6Byx*Mqaz9D!y13I$B=$u3Mcu4qHk}fMGbqguv zb`H8Zcrxz_D;ws6dUWiti?`}#EnMuDoNjg9(7{1j~5C?lZCPAt~pN#ktyY;ftvK&R7{t~c``EH0UK&JwizPxHF#RYc3Ca@2ZsrL6 zV&%ur4*1_h0oRy^xpLq|`GTi83*bf30<{wtW`c~pd!aXAKPE|oB8G_@w50Q@yFDA> zW2dm5t73<-uTX z3n&jRNKY92Z>0(Te~S&?PdNDBY7+7(T3+%J{26e+c8dkxP89gx8YMq5=5FhlKjgxD zCvFmce5QPnb$!tleT*g;!6UO!O#;;#XTaJA{+0k|fNTI(d;&`S^#1{drVf%>B@}LR zsACs+-iCkYJT?l)uTMTvUx4#U``V?8DQem*yw~X0rM#yG#C*~=b}Lrx*({h#QdKc< z%RRx*$#7^wcd967}SVa>;?IC4(b-y0u$KQC*UBC1+k;3-|L%3P*(dWjRf;MSxka zezyXcqi`2*M4BV34Z<`Wj-}9}JT}%}?u#>#(QRtpK{j3pXge5WqP}Ty*-7sKZXE{) zh=+}-lHy578)7|U>cm_-#mth)#g|}(ISVA53$)W})<*bx`XD6TV{8Zrw8BG?=7-lWpK@*|LONI_yrb@LXf=T=CYmXLb{_hRRbnJ>F+W|s}))k z+APZ*O9j3kcnxNbrLjn5OK4dJy~Qh-PJII;9ZLulcFgc49@bJDtbVgIOI_$At{w*{ zuAqtq?vDEW5I^v>dQK+Tw=PD`L z5wyvDSrJ;NSVwM+yzbYLtHuizS6F7d;P32Y6{d?Bhky8)Bf)=I?`9B>=3s!orVuXo zBDNcI^Fjsd#gwdllGeB8Cvhr-8|VEh9xIW7TUK8CMw=bpd88yww@Q7T$iSM?7Kzm5 z!S$$kAogw;IR5oO&R<|LXSx+rD3se^Vihv`hl7s3~{VV_Ayjq8Ks%?`p*Bney8w{>g`reCzqy_^?SJq7>bypo4`RCJ{jl-hloS7>}=U8aUn3cScMKhPe+F2@w{D8`ZXsR|7%xEYvI~FC+ z`3>GUIM|UTwOv-_3u6%TLo{y^vW&f=T-Cp$VvdnWyA=(yh3sNmxj?E2AAJ?X1ld~4 zTwMZsmMP7?oJX}oCH#28G4e|J1&ew|x(+qwT1l(8m>OXphXT{j{;{ipG**s<$Np4g zSh|W)6!!=zP2$u)EnmdrmuoBRbM$MnuuWb4YPoat=%0^vH%EraLg#F_w9$5%zn`|$ zu_yJMRd=8U(e#@RqQ{r%QQG3Ezo?+G$5Sb{(D(x3!7#*~5F)XyK8sGoC+0^-YEE9M z2@oytRK1jQ0eN-hcPvDt0x?%1JkfmKhCU8Tcx7rJl!WYJzlg~uix0w+j63R;SJRSD z@RyRUNO_De-B%Scw~1ce!f!jKj^=N_n}t-v!9J}iQ(U3-lyf<%3o)CXQ`7dsp)c{d z_M5De#0jX+@2WChlrQdRs6jYudYt-~DimaFo@%BLs^wl`7PF6ryUpyvWYRj@CEf&U zT=sD48Dk4yf@NHwJa2I^3x~@k=|6wwk7>WTAHx1FeSO$;sAq8%WT4RC4O6w!<-2%l z+)%1gY#Tsqov-=baS;zUc}sh`}mPMPp##-sxPes^=bvpC?LLB$Q1S|GW3;a3&_5j%OIeT6N-K zV?=LRR|&Sj3^i5{Ua85=u|)NttW-}g{sib#`)+gKxotS-E{^>dC%+_-6-zgkI_!vd@y`Mu)-DDwt@r%kNU$JHeFmytVizZs zQ)r0KR`Oo!7$OG)EqG}v;q_YiD=k3de)E-aAG6(LQucC_f{_`9T+psxK-*g&Yp$r2 z2>iJOET-IXrFb?{)q+H5vgSd=l&+&i~8c(!>-f`@D!eomp zEyDNrY<)xcVjHamisnjLQ>+^tH)Z~de6ztoZ;spoBhhE{l`48iR&X)GBhKGX!>H$y z5ZY6LvdaY-Zd;rCDVegl0e{;ZN(YBb$<(pvzN@1xux!6<=fSNHsJ$tlFryd87%&FY}CZ(4}cbJ=rl?l~33($Ibe4P`aE##`HMNd^V8@ z9cR>28@QUx(#EO^0R&`^)~R`E@J5*R&Bmt6=>2^5gvJ~3O(caO9+@h=CKu}~oIqZP ze%xqoRKn@@AJfGG`hw5svuW&gjW=P_F)z-pUr3cPp28So-c!?5l`S06w(`m-q3w)U z37*c2yfQXf`$3Y1>Bhp|an;(6sIj0fi`W_@Uy;z`i@42eB}e!XC5G-wXnf+A!Cuko z;=n%x8sLDZmyiNZ_9Y1=G6L5+^co^L_63A8MndYfF>`(vletfX&6}weIia)^rnqvM zo7Mkfm#C=;?rs)0 zkB>mgd&!!ZT$_EZpxiLfbD(3U3h#4=*%(Kfr|&v&A90D$VrJ2g58o8)x;+hp*p<9V z@TP1QAni!aL6Gexwz$r8U_Z&dS$=|c0J)%i7ftOy|f_ws>B%y1` zZHBK$^q7~OaT>=Y4c78ucRapKi>zKu;XrF*2OI55j&Jz z1=Uk1^T=^rp5EYA9yvZ%a-^&*Dfv}^=f4Q9x};`fcYpaL%yTt}JH)%oDlzK6Bl6Kn zbYlA<3SB?Q-_mU2HSad#^yWNj#@n|aZ*wH7rH-PP^%gNE5=$r~tVo=RWxhcSG$(af zd51HqZW`zEa9d+UJsJkBbW5BZAk5d)BFyLGS|!v{kK8hO8%(kr-+Cc%(VKec$;L^KFFY)vs$M3K9a=e+mLk0Ev$U+e3&PYadz;4& zD&0AafR*1vTORU+CEN(xqUp?Es0a6YXKhjR*v(8I>UwE9)1}4MNUwlbDV)sC`^nLZ zb{2-v$-JiRd?-7g=&KnU9|1t-AF z{bBf%7Gu!1E#*si71Dk2Pf1tLvZF7|=c1m!W4YNr#@1I2$ODgvJS<{C{G&#S8`;@D zL79#`mqj{1he7F)TN!S!KmXgC^3vueGdG}T9t#kl|0@Z=f4qSzxtTe8{qyRz#Rafu zTv|Hr(%PPwlw-!iylD_u#Uzgc8X`my6j=o(5)zgKp_Vt-9~d(wp9CJ#?v%8vS(mE; zui0M?Y^EcI6-Aw0ZeCg!v#VOFr7unEy6zsA9e^FWJ}20@7IC@me9pVf|J?FA z3W60D5{s9(`)*o`?1Geqn=XL|3-!-UkjoweNr@;KNCUkG$`Ccb>!|}o+a60JI z^$`K`1@*oEw-=AK>pB!+w=H%zP|0yJDl!>`&XP>qGG*>6aKUpU_%{j~?S4N|aU&$T zaa3$6d?xHDE+dQ{b2?f-Dz4arBkRKL4P*EZM_VMI+z|!pka&~CTUi%i0lC9&sY!?5 zAQQI_#JRy1W1i$v5f-7QI{hwCVB1Rd;Y!Dpp7|ihMFu0*554-DmPV=r|uBjK|FXR zxjbfF6F>;N$XM6p<(E)H&6dCU(c_47S1@DUG#uvk#~m3J{j@4bG`p>29<@!nbJH6& zMwQ0}B}7d;=>Xqr5!@DE5+JC8xJ~ICp0Evv0mc?eRRVaf_AK3$7x%KLl^qYNe;p-8 z43YZ&jw|VjI+)zHR7VC}IEYk9fP~=)TU^Hqm}y{HkXJev1#;nzxJh?))jV*~0XDQ) zJ2z33G*PlPdX(Nq&XNe!L0{D0VJv-6e$+NPj1^?zx6(#9VND&yC30vBDf%@$DN^*Q zVU#Ihwk}R&i6mCMX`9bY>F#j$1d9w=))+5$a@0kSpFStHJiTBWcdsu={ZiD^cq0pQ zshoifc};mpN@`3iSTnx4uyjVVoi<}y$)$Xq5N{f+M5P9}x3x<;oi?+_8m2s|_Hi`( z)~d8^ylUMt@-NBcaYEGQ(9WVDQ)+ox1A-CPZK;IK&3^P6Jx19kUP+kn7!HQI{A`u+ zd{i-2(j`1RDtQ@Neq0mK2zs_h+4OeC(qSK@BTtw-w^?8?OFdudNuIx4f|eY~mgyhgayBIR?LL=r8*#eC}&-Up0=eRI71K!OZYA8tk@a)HzIEb>M# z8%RP#OLQZdlL5)V(qTlFL1c=_(5MwuNI-|9yO6Vv>KJ0KZB6Mq$3Q97;J~Y-As~hSw7Gd}S&Ga#VU4(o>65ikg{wHjB$PetNi8h&C`>q64 zmfWbmx`R((_yHiSULwOE_{cYqxaco(AL+3^p!|V8V7S2%V7w3vV=q7Z{3L0`&1Q5k z5+J0wCOlLujZF#%JVZMa4i#V?d@86+;1ER8x< zAaimhoW_Y33apC6Q7bCfX|u?{1R=$c*4z1!X2>sgAMgXO1lr&vG*e1u{VULIn`tlzSG$3ZT3& z_Yfbh_j( zt!p$G*R*Qw;=GU}I4HGpZ)xaV(HQ*h@G@&}%>{P!e$7s+0{w!&CL=op1b*DZnme9J zgXahbjov()61cn4spiL~<+W2R2LKm=Xyc0&GbSJklT z$FvHgStA4qjn+*p({z=~4W_j9-@t^PKsrwFEa(&1D=_`QoNv}N=;zF zz)UD8C_PbE>GPPcc^N%vlw>~=C-tsUwCp%BX_aXTS!bfD6q0h%pTxDOiag_9;)?HBj4qnyKC z%K|m_nu*3|Zd`#G>TI-!Aado4a1zqJ?oVO8{aVsOr?rJlE-;G@=Wm`2 zl4v5u|4UdhO*t)bI=!+zyor7vxLGjW9MYcwDYVRS1Ez7*|NOlCF5&kV*N;!P?&7y zU`mB_1rCLD&}w*n_|}tF-m*6QBwdgccb)+_8+dETN_iWfJVcl`rV}K``Cg*b!w-Ku z&T9dyaOGo$zn2R4m0cv^P&fltq?W~XY#q3HS5#7&(`?bU{fx_t;Et&GS;F;QnYZMT z?7)8B`*Bh+o$1TaK))|6T#=NQTdC#jb{KL%_L@rj<8+WC9F+E25gLjD-Q$^bTtkz!K7$LHy&DlMiW*4yEvs4({?>h}>q{U$ zO1$sdhh9^a6Rk$Lp_%x@-XY(Fv-ofqZjNmvExR;Ejfc%(^YB1Zb{E{M80hKnHsKTw zRY_!5eMDGSLU_>$$I|5f#%8wD$b!=dKWxLWWHl`cRRf~CP)}YeVR`fKnfrrh#_<&7 zv|(&2Qari3r76BlAWBJ?G-PvSW$NMaRWqR~p#<4fx>&<1+6*K! z(U+vReUg$Vhryr zWR|2G=_E1Nh|I`DPNqp-9+W$yDf9>is<0Y=In@i1*xE);g!fZ(a;#ZpGa2$Zb#5n) ziJl&JI9uiM3$R-M50Va1S%qiQmj6ZBI|k_%EZd@O+qUgqZQHhO+qP}nt8Lr1-Mw0? z?c4kAbK|_d-`ywP`%yE#m=WKPnOT*URheUqq_Tc%Lo;-|G?mhQV5w%3=MaM@bMiD; zX8MKaV0QWxB~_#(0Nmqh#1dUg@GcL?`mU$86&YAY5iHy`!VOGngcT4?(cHY~QBSAs z62qZ15ov*Vxl2jbYabqs?IqF&c=f?o@3Pub4P5o0H`WC58(PHU!nRcD;i7gXbcK`H z-_C@CQ&CN^%4#QBE);;l3;Nnc!8?NEDu+YKNz*P0Oz%PK-eYvQ>mRC&Tv&DnH~78| z!;ob*x2@p~JAzR1o`yUX;l5f-D_L>Lxf;mT7yRWMSm(zSx@CI2*$~#kzjiauboksb z*QF@o8a0|!1Lre zXZEt?oa4?A*KLdhBfrUMLSDmO7;W<`4V_`(?2}^n%^z(jn~gSUo)a28_)Wt`rOvtG zi^RoQmz$^Wp~-y?vC! zvB&lzJ-%`^iMnU!lC~wS$Q@PR6h0;muE-r%-;_Qs3a!wrGvDfXe&O+u_KAK2I=@1U zs5Rh|=p85Jx6ytQsFn72;9IN6`P*w%F6iF?KkLTV#Q87SR#-|1%_Hiiz#ZPI`~g1o zMc_?Q9p>HTjein=lwo^wn$vT@l6(%j1pM@$UC2uj{<&Coy7j3)D=a4n$~iIA6ER7# zRIyoMNxs4iz3uXi#GiW+dP#`*m5M;i!sU|^u5LgX{$!N13k{Ze%)_*#o@qyNr;Q4{ zSNbL-mR$AL?*Jj%jPE}Wyw~8br+N&C5i|!*>Sa(J5<4r#F7Poy_$vla*$*)&M(CDH zc@!Yp?B)5@0w__3{4J-_319keln_Hk-{j|TrTE}Em}rEip&`vFIj&J;Z<8SHHn@9alq&CX@R3QUO*#c z&$G7?2@1Vl?5I?WwB5Fs{Dvyiz4)?d%w?47i^d%=1OF&s%fcbF|FGSxV!Wa z+^bpWoof?kzu#nADxt?ZMKXM79HmI8b6I={RI@z3POJG(CxC$Tzi#B<+I%e3a8rDL5Tm24N){@)>rsIa3`r))tRxScl1Y1_=->G8-yTcc` zi7Krj%Q_<$bj-157h#IJ8&k)06z5@gUkaUlwXvKB_*X^&8(a7s09Q7hVpVj0>vZXb zsaJ$6vCyta^5ckadZU>K&kU)j;eOF#fBc!2B`C>=Uplsp-gf<p2SxoKn$ zG&prMCmM%sGpsVi9Jpo`N0|pT4p@+JW?{-^CuuJ|TC@!#&YiY?r<`(o?2Ej4LHZh& z4pCF^jj`xiRtv7=?pI9zd8SE`a4F z#U?r;k0q8HI-(LbO%UpkJ81R7X#r3VOkMi8Gh@j$Q|9UKKq`tiYixM_wK@#TP8!&( z6OmmmEEr*?2d-8?&>H?P# zsi$ABJl&Fa)Q*nJxuLY8ShT89(=1qfHZ(BdQ~3;9TU`l78>{n*wLcuM78> zZ(yh#Jk9O&>__vv#`)5e?{T{m?QQ_P6WKewAZqcMe4? z#;~4m!dSeoBvaG$SiSfo$-lX7;QGoac7Fa`uo21(4a?LSnSL(aaw(mP*|rJ3x0u}q zeZQ5PcKQS}t7ZOe>9Z>fIc+n3CaGttkd~fVwEcsupS{giQE@2IwFBR46y9JtfNZuG zv881RE--Y1HBdf56IHo`(6v(vTOJF#<>1u%z5`#wti=`F;1hr2^rKXmBo(P7?luXB zEqG90wijG(560-8HP5xazLeof{#$LtC*}Flb$*Ih_NWMlJeWc0babQcE35EL)g{W6 z4T@LhwnYdVy(=J-@(wLMsb)ddej$Nz<>aTZ>*P0Hf7Q(Uoc8DzHvsFC+*6^|Iq%+> z(PDcMG|v01=Imc&kNkPV1;PHl)rvtN1G2CRUgZ7o`-b4ptvdWb%_l=xi`^w~imToI zrgyewPvyo`+M)OLrVAufHj~w0QS=l1iyIYl1L`9F-`(4px&Ge9^wV51udTi5`l-A< zE^qV~s;2?m9>~+={!X!9$#f)~)&0ZBI+t#V zCD(s`C`?vGKHSl9Z9!_y42;y!zy9{zogVtqQ z&uB?sU+ADkwiFxtNyIc68m2Zzwp^csoDKxq$pza+Nf!twzPEqsl~!FS;fIZUG-o=Yd={yvj;VI?%*^> zsU?URu0@C;rHdI+R5QB_vJb1kGQRJS;6OG&gGw}knzohy`Zu6^;i;S-?oT^w<xF$3i5YYh(3=7Um zqov+ZKcPR_KyRm|i&Z+leJ7$v1%;+=T5w(nvnc|Z8pYZM!{zHZ?UYeDhtB=eFxxxfN9h_y6m2pg zpT~j|@sgMaM|7U#dOwR9o+;r%G-{bJfOgeaoWsCmDO)Ei+9*F#pcmvFkg)^32K>YN z=LTjcR0Lh^6*tGLaK|Y(cRjAO<`K$USDu*9R85JU(bRCn39BGRJH zTJftMUgTas{$3msIo?BlA@dYA*vT~Z&gJGkv_CWq_u5&9xnXB$ZV)bbCkXWlM*;gx zAr+EGF`A)frZR0UL`6wn6u8g8_->)~Z28L7cT1$v(}HObw)O|&#aik%r0f7cM#wCI;&mX@ zjw+d`!*V@yKpSCVJik04-4?8gAlO8cYbgl+SepGl@QeHiC6Rvr(AxCJcUta!UVE2c zR}Ub?;Shg6e*%FOhoAzT@pQvy0|7;HrVwXAsmyTSABwmrf#bgk1Rl)I=~ZpS#R0&9 zL}~C#PZ@uBJF8Ekbt>IwO+nDHVN}IC*^N^(HA>s6H?22Hn736a^)mpK^b~-0^$u7d z9QnSiID7)TL|b!_*T5++W5wiiCw+!(8V8k4N^Be}oPq%b3J1XhsdkOPVwIQ&J0y6xzU&Hk*KSqLxzY(=i=SOrwYhR^!^BNvj2nthX&@XsBK7rFy*i+9?tcx@9pwG%5@?S; zEaLwG+rIx`xc?JE;a?p0zXiemtD8@G{fBY@g=ewY;Q$Mwprpi_a&BeaazHb=5?)-W##9*uTDX7INMk%aq6KIg;%`NXL(C%;6L6_*h z``|;XxA8m5o90&cG-)Q-s$+Y;+p#~}!KT;=xzjnRE4Mt87R!bXJ#YfHka<~Kr_C~v zsT=h;le@-qOMb5!23bsnon!{t7Lea-q(W*j0Zf=}Ba?5~*BlSEt?kDUb*kxm$S{AC zoEz%1f!BFzqf2j7JCp$>^a@7rqjxE*1rlN9pzfJ`w}4-qa65Yx--!w@TB z;_b!yb+(MszmbOz`+m#pQ?|u{!&uAo3socM_MQmGS<<~v?%$h+jTxn5M|i%wrvFK5 zVkB9LpY!#GjDC+bdg$fY;SFJ7D(ygd2W<5Md3A#;@>_zklL&C@9ALzxVkC)E@=TVd zN^82o1J6(~Y7@)#6iUS!MUkll2o7okHNcdhR|E!}Z{<&va@H*LD(6ze@kf;cPp5)Q zSi>an@IR*yA0eVkQY`P`rzMx@WD>S4Q>Rh`i|37VuAP%kT&!x+8+;^wR{0GLOn16a zB4o@fNaZ(hfWRb@P8gBMB_x$`LO3HDd;4#!7_u2;3rjyC#OEi3Q2d8rtSV;WOrrF! ztSB*xH&UDOC_Hr@tbABHRymrdCnp!u7pAv^q3J{^wJ-&9@CKmkW$C7*I?{wP?{3pj zN;ufQJNT@wdYYo3qi;aZj`uS^)$jM~y4_YjKpcHN(9zV;Q+L85io+5xToVIV6dFtC zS4jd_RFq(L<8+vX2qiQ59N+AwgIIj#voeLF(Z@Mlm=8dq_VY*j>I8i9fK$!xnRYWD zOpPmeU&Sw}-u*!`<}6;< zW6gr>7n-t2;0mj3r^$6-3h_A1)x7bCOV&~AAlqXyp^Orep)W_V2pgHadU zm7^`OCU=4&xTt@|-KXn6{c>YjyXF-JMwV!Z>Wcii2Nyo6Fn;T<#g*@&4rW)>yX@R| z2Y2Xxr(q3lu5%P?>GQE{y$8Bp`Ljuuu_jxRDxmZC5P16vJxH?SP^sQxuvkW_jz<;6 z^!E@tfuCDpRV^L8p)`g{M^2#jQ?rll1jzAY&hu#iQVABq$5VS{t*W<41Ij7Rk)d#t za*ewRqlPzr<9=9cq}CQ_3&;S+lnKzi2N79F%lVFsj&Z2zrz!PIjF`ETSe8cPY^qnL zd;$DxjQWZIp#BGf>ft9&;r$OYHhK)7{GR4F+>S3_;O24DI?=G$zU$ zH8lFd9Z$gkXprbadGAllOkg6+1s9D_P~>spxDn)I;=Ev&w5UNZ&o1mqaqs4G;_>Vw zsiTCVkEtBZr%Oc~zP#>ek|=AXLJU4Tl_KH00s*>!c0L;Mc#Bkmb4(HqxjbA>J2583 z2El`>Lx?H@T}^?@3T-62nx80Q+XDZ(Nqn=m9zww{kyk^|0e3_2*%WQ9Q7BT?>TRyogh%pJ6|PW;#OFcjdO%4UE*%kWVf-SQ zr3w#!0RK4}tP`CL!>{Fd*sW#wa5dW9(DMMGR-ft{&_KuD(G)<++~Xu=N;{zZGR0sM zxnGIfeu{9S-8z#bQ5^2M43BlYj{H$a#GO!QAx33SCPx*jR52T~?hL?xyMq(z=yDj* zS0Mji+lKM33M`7xChfaL$!Ii!0=C zLt*eIj5Lelp>%s!SLHzmM6sh6gy-!qhM#P(FC>r+aX=!J3w1CEW1p#qY&}kb%{W{K zo{GS^nabXIlFnw(=Ox2 zN154ZYOHl_q%?^p*>ti}XEkk~QCSi-FsA`Gb)K{v!wJ#Rpd1~|(?t1$qAAPw1G(H4 zmu7k(I+pZioLWiDOD~alM=@ue;hG9EgVt6VD~P!5l&{Zs)*n z&7xcjItXujMBU3bc5WX*u(&l`kG#aA;h$sKFFVI7iE92ib+mfs=#;}Ym6W+?wc^-P zN%>Ib71201F618*_;b6!#8STQ^IEN~jIo<>%aAXz|j7$7#t2!!tP#R;O>DNly4j##S8J5@VL~*L9${( zu<0=BR;bycFOlrJ)6=~fs9(PhROp===&YGnZqzB|3$`NCSM47DOIffVS5oUc+@pgA0D+Y~cv$dt8mYFsu4 z1GcUin`aeIqSDI?FIHeBY`o*LfKjE!X}!2)7WYKmRdSn~IO;uNI{TC+sW#G-hPdP^ zh&2TSENpi8s%9l8;qk}X5?apvwc3gF8v2RyD0xKf*m#qTtj;9k6b2LQ&+%AGP?wLi zGefM4Hh+opaVuXl8q4l!NA2Su@l%c$UiTBWi`kD)xY(73Q9#UBwmUc$5EwNC`tsWp z?22rjhn2)}fnhx_)G6RU%?ZT_=3WHrv%oCWn*#lvVh|Pz32jDB7aQ#O zQoqbhT})>Wd4puhcD@ZggN!%fyt2a}rYl^FwS1$6ZHPe_q+yB}VNRFpKyw})fF^G! zr>~DV5n5EInQJt~k^+e}q=uzKc1Mw72g!^Thz-d~cahj*$#q6~WW=L6%sAvY5TG&J z=skZyHq-GzR25F7cIQRw*?OH*9JJ8hci@_HWPH$vCc1U^fBA(9x&PEz=CG$a77|z$N_yryteW_j{#jbuCdG+QT4XQ=?U~8 z-UZnb8@xg0@EE~FPM+|0paahL4P2E&Hde;K#!ZTbA;hJcUNkdrO46uiQT;|t&!cUD zfm=rA!KCrB+g_r` z&Qd1xW0y5VB$@0`T0i9tr%G7+kkql-Bi4jbT@xrb1#RBp#F76ADe5pe2hrSi!Dp|{ zHPPa+tEF$D5*H)0U&tzgu)W{M7$~#%@t3}O9`#(`Qrz_3LiO@2+KELUooUb(E5JEE zT%1Tsc+-+&6V%&T^&Yt44(&+o1L0gxmDrMx^$^pwZ6_bKSUPuq8p$@B+nv^*%?Z$BPxTZNGU3Jd6#qo8^H7Xfx7&>ly#UXLPX~uKGYHcIxld_K^U)YOK zW=!g&sb7TFg-`KZ0yj`-Xl?khhbw4vNniL5VQr} zmnFU8iY4;{_*?W>t4^Rc{hB3$#;@L0VyOW4l3zZ1KM&2 zV763a7UKFM!?gQ$R_Y?U)z{|ggF2|7s3M@3PTBQM#5C`oZoVC1a~~#wpi~$2N6GasKAstfj77F4^24&Q)Kj zf5LkhSgLyUy~Av<%>qP2cz)N~K_%yJc%Q zXh+O&LNbvL+-TBqxaWvaq)em#$RAUb$yTT_{FIp8Ja%wACMPGNx=LJCMqEV?c99gT{vL^&R*EPx2&}U%ZT% zc+g=tXU-~U3E~sk!%|uTeb&xME4OH*^QJOxd335Uz*Fw`Z>O%@%FKYsAAy{-pW74s z|7pJcGg~iFhjdq7M&&hYc{cqE^i$;~3uYiVFcyR`2bB??3WYHhK*}dn&nBTa&`)KE zLa17%TC%KauF*lb473rep`JhxcBmk7S?y@Le(zcJk(%|mnV!lrF1>nYz5NFf<9D{# zk1*Me*Y(!(Htjorer~^o5VK}AMgABoH8OS*5H^WCQ_!26zgCDenZv1Z?fSHtdG{8T zAub$tcGR%6B!=n)t_d4A#N>2Y9Hrkbc1*_UthbV%*VR3dBnh>-vNFHu*lI^yV?q z7$Z>YrUJk$YkO`stp-pnjcG%&QAQ0zA$q#wjk^=Zt2i4AqXpYRxIZZN8L*zW5Y!3C z=``$mzx9o0&!yyOPgK#}T$#N=lTxHr*YzRs`jcLG-;2Ynsxlrn2x6_Q-HE>Zv#c_* zhla@Y@zZ`?aR zW?PXnuaoSVVq5nk+GZ2&!MAg#j9UPGg-8?7PDNrV*jAg$d24)1xRY5uvzMo7?GJ+vsjCEN`u?bhLCg zT)O{&jQ|!$5XfAfX{>InH5NA37CO4DSRML<<>&C@gpcH-r;AoNsWJ!l{r^A{rf*lJ zsV?%Dp4Tike>xf2$tT{Y~=K zzqZlW2pfs9U1Ee^U|jSh?`sz`pc-)_&xE>DfMid7brnYdL( z7+fAZ?$fPcTGFS@MX27=%YU?*6slP|IMr56MDUQ_hC-4eN~JUPkY+W8&p{~baAoG8 zW7&(M&@_vQ+xG6u%)ZNpr!`MCfzKEA;zfKCux$NXN}_6znW7Q-O(@H1B161QJm_dj zP6c=MM~nV>6srqLGJj7&jvhM<5mzN@5HT1oXWUsn)R;6Hi;PaHR);e|Y(tCvd2+Ct zglg|N{14)-DWqFJ57HK>U&@{>+OC{nItj`a zbSq?=R-Xc3gr9$k4lK(KZJXMFRzfEHfDjV*lww@}Ard#BoWw2s&vE7b))n%mkBhV` zij@a^R{WMGsVrho@6HzTrvDRZCkmY81A9lTipBRy#=%dYKUVyfs7qK1RdH2k;QbB| zBxrKF8N7mRQ-ocm*&klwg@IQvZ0=<^^;W@KdZ7345Q3R0XNR93?=L78u~0DwiWGf9 zupC0RsE=G9IjJ|)H{m|todsl`K~UcCDTuf3Pf0}tQtv>cC%W#IGrmgqhj`c<*TB^8 zm%7_;oQT(VAD2t;x3^2SZ0*)WU>nmDse+4qJ&{Mar}Aa^u7f9R4|p8L=aI z!Rd+LbIBD_5as8JApJ-Mg3}FIr4Sl(VG+MY^``CTI@InPtp~B|-z;_x9ETv-cN3SG z>#q=x=m@+smkxLwuSLj_uC*T`1rIYKhFC0e^f#OlIUV2F7kBUWbghn=5Q;4KQfSr( z821MQ7P*3XJAnWzGG@S>O6)g8012YNtY(1?nwBT%$56~0dYYL)RhnD?ZklA=^`}26 z>NFt0gNc>@z=9$jBypwPAQ?0c$|=;Jt_Um;XZp2=I4*BZLHHS}7`(hA@0ZTh{-N4A zaEEam>!h6lsDeND!3slbyBlsLVYG_m)?C$4Grc^&zOW&3A|I&fRjf~u1OtvaaZWYU z$$oYsmDhnH(veR!ozo^n~cmBXH6Z`{B|3{0cBc5-!H%F^0gG`zVcFu%Zt zDDwNH9;(ja^kSC1P+|P{9 zM|aiKeDDfEq6n*Y>6DrF!(F=NZkeN68N^+N&8@7}qLpx*hk4d<+ai{kDl-4c#`G$c zB9&dNPOWvPc4P?Mfm^a+sbvquo*oK%1zrD?1Mf9-!m=NkiaQFnr2}ld(jv41Hnk1* zfD)ZZ7JieBj;-zXr}aB(E#d_GV46HhRNqXC^<+mjZkiC%7@Mfo zT6IyY`Bdz|7DXD1*$@v~J6`Rb5}L^0){A4mZ^<|oIM!$vjf>UO8FV?HLtlc^5g;_> z24Au$XcZ8YcEn>6!tH|Zbx4Os-}*R zqs|lipULC$DyM+gt9$$!?98W+?~}}n9rQK5gnSGB>8y5D#S{4!7f8F#A6W|AHQbcDzhKGOfc}T&rN^!=Xd!;_Q}t?bZsPQbLiwl#T0uDHg9@KZ>eTw!f-cgco^w_`tYzjFCTR6CI z_Aket2rADfSU!rOWq8Lc20heEoW1a!OWwv?jr5OWiaRGI*IRKlf4$KWL6`M^6@o2i ze^_N*LIs`2`;&e+F)H6)Mp&ZS!)ilAtSW2+03{DT+6#5~5I4JBY_KDipaaZ{5d7VY z^>kIT;>AeG%i}@a1W8O|iOGl9H{>3n*P+OAlNk%xC-#`V8=oyBr4B%m6 z0U(NH700&KDz||;_l~3>{Ere9TuI>z<#C$@XVi`KCUcnbU0f+Mipfw`5lb-E3D z1&uCiCzTXiYY8c(+=7=5aB`w!%Mzy*ilol@dgMoD-dP+|H{W%j*9fsG zi7DeeqY)J?+2s&!8nPO6JNA-up%9&(CCHs^wtrdz5clQAKT%2q#4w~?a_I~6mAe@{ zsF>slgD1x&^(Yh|cIWS#!~-aI@dOxW_$aOLfRnp$55ip{uptpAZc=16LE#+-pF@Jv zKN`1gA&bu^k=_d{*{)sRcfgXS;hsb`vg4SUA~c@Ok!<~;l~(%Z#5rxP+#{43r>$f)i5~?JLs^W{iN{@w)s8#j;h-OVg>dajdk`a8ddru9o%=j{QmjN`QvLGVN z;KSqGs<0lc59!h)txw(jGl-AdoExbR`dn*9PtsWt3c%@tnlxum=?&`Jby-~9^VLI+ z$S#Q-9%pO&94TQ`BW~b;V{Ss!?sT8Nt;|EYqgJ+M7S#_@6^3SU%D86CoSpq<=r52#y zXVKTww<0;yoGtmx8lx>|G%F3$oIx(uNi({*o9RrgX+cQ?UNz?|k4FMu3fw-|1xOjH zv$hxnjIW}we;u(BT?VcL`%w;3SPHd;#PMdQ8N@LY;2x>EN(-_#5Vg)S3J0t?*DgI$ zG+fHr_okbe8Xc8TJxePXB*49(cXAm_W1z4X`gg{;Oe5fa2>8khELltzEw4j}U^(LA zA=9%J`P9f?c)f4&K~_zm@k)_D7`|!DF7)yHN3pC4`+wo)mM!vW%=6{g+VSapg^?J1 zu8_g6mcA?ZRnw?^OX3fRnh5tR2^C^9k@+MdBa8uj(HlM7zoGC>a;P!@zx0m$JsWw$ z=}F(a5K>WZhkG*F5qG$d*PCCjE~H{HN#bTrp}M|hf%}03st)g)St3Q+seOqkIQWzE zu(3T+F1sndlBSp`zM`f~mVanc_Y!WoDEn5W>{=XSq^Oy@!KR@&k39!P9Q)@bSdJUL zc}Q866z^Yg?B$xY)TNDBT6fNtlM{!XfA3~)K}#>T}>0GTOn#W)sX zbrW>PMu`!u-Vq)%@8$rc9g4OKGPE3B-a0onUA+#G;n9RUT$6@`{mMwF0kVm#h5#u8Zbi(kyMe;!d z2!5J9Gs~PLK0M)e4$>w*sWF8^Q8DBU<8jyaE&){xL}h2tdktQ8$n7}wiXPze%#!tM^x--H zI%bdMCSaGqKD8a(H;NxJ+XDV$24GQa|M~GCg<+g>N8YX*iAMJF>XEI?gKtgU6nj90 zEpxoUp7Kw=#vlJ+MW8~Y)NnA97SUbb9(OnoXJlHH*^AOja5KNUqk1lvzGqu9`@-Z31@>`0I-kiV@}$9a1UB zP9GL1U*5<|#c1zTdn~90VWxhh;oo5YdTm#E4+8W7U|bfKLgPe!+|J@C!ygl1%M(1vTDI0#)(M@ zT|rQc!$Sn8xck4}fV%ph^nd>*Vwc%#j!fch=q21$f@`hCD!I#yTr%y%-<5)xNv{$k zU3_Np9$=$}qOK_K%Iovr!^XXJJhqj<2@6-@#XwC%UCsv$ZIkr#5YaTNEDH(-vXehB z9CAib=aX?jo`jzXKwdflJw3lY>dRvcrPhf!D}j)RkCLF~f*Z?w1%L}~>C`e%hRgy{Ty$9OKK*A!j zVm&;ni>4bdKPU!kJ_59#TVO8=N54uF8HR>egsxH3Ku+&i>%%El6Jf-fezj-{180~WXA&w)O`W#b zNSixIv6DP};PW9lm&!ZV#8wv}(U^Y+E%Z7HB z>4UpUu61<{7>~U};q@1RXy_&|9z_RkyKDCY{?QpAm9>2rpK3V=2JO5}55CZj<_Y_# zxUht+*e_?mXjJPDqv@Jro8U&2FjBrnTY4jPGEc-=$$PAjtd>%6k;Z1rN z82G~&Ymh}rgi;ABVDOma6|}FuLZx4!$v_h8}58>SFv)Vo6nlku~?3?N!(@+1R zA~-Kdiid$glpnZjuM>DfH^GYX3Ia*@%e!(Y*h5!u_z6L%r)xK^>*buZ<(&j zAt4lq8Ko8IO(HT!2v(k1JL``}M%v`blRBXg-#YKwpw3RobX;dPZ6WJpyBw=fj%Bq) zg=yljMXR6XON4ie@F;p&+e%jFzA1R2MMP*?)l{}Iyp0L;OxZZ)xC4!WilP03dQ)`# zD~sevUBchY+loVYc(p2QnWTHcPZn zAO{cBrWJ4UK$5p~7z^Q|lqnYur@XfE)<4n0sA}vbTw3%Z?xJEIAhzKwckJh-(o9KS z;=uB{7t2eUGftCb7M+WhWj`3e2J% z$xaWNGqN&uDbfmAwVg^^qDmXGin}l)ZVZWy!a@=_!x1zEx!lf~>^u|dRNLqa^+@e# zmUTjpCBog!2IUC^j)d+rfJr{!+0aIRADxd zz95|Pjbt7r5!>AUBLKW?KWkFfg1o~q?JQL|!giWwp4%6fj8LVi5tLNB@Kk+KNXlZM z_g{CWCfu16jGwkP!G9F>{!@YFpF7k4dsF(aqDz8|>>vX|NS=jw{>LM@CxGe|kVbW< zKqVEMK)C2d;Pwz^Nb(-YXpJWu8jimZp|pTJDK-gO>SIra$025KcW)O^+mMC7tiP>) zE`KMBcH+KO3y|8Ny#RG0$GNiTiZLfH`#6R2aFTXILk3a1xFIRNn&UOj$c$Zws`f_p zoQi)Bzud9(NhmF;v4!A80I25RytckP?c{HUmdv)J>Ys9BH065m4d@$VSw`(>M|Bb> zHpQT^hbj&2Fo)b~b!rO((7fNt>w=3C`@t~&!ZtE%)hsK7voF`q2(%dJ3^v;CuWL+Puz7qpBXi{%B6>yVNf7ZT)5PP4ek=UjomcRdH-^R>!q54$F9>Y-{Abe?t?b zO7lts{TLeN$Iy)b-wZ9{Zerx(Z1-Q1v?1}`KW0V|2rp4e67uZhC)WMd8^Y63z#?)y!V7g`6>3p=CXw8(qzt1bP)6w+?Mq#I zsflLSrAlf+>Hd@`{-t^ElBE_xnt;)4{-m7@(>Q2=LD#wS$tDH^1N-PrQ=i-eifMNtSLJXl_4V%VTuvY^y zkbX3|iO9JVaiG(+8Va+c#(+xr77z$eY(9$7yA@@K^<3ocPlL}4WlDTDBINrX9>)Qe zVfKk*?U4f73J&0I zs5DhCK$(H4s2=g7I*;cHgbif&CmsdY7BifvQX+ttP|m2>Ys&m^=mixF2Fv-%hRO;9 zsZiLlQ0OM8c81EsY^o|j+E4wnatE~kVM9w#buSgTT+<{s}q`B3p3{2yUl{W zHWUq@PbL&QEK{0O`W4a#ZrI^@NPC7sSSB<@+AL4N1)XUVxP`4Q+{!D6 zn;^YoV(}3mzoO3om-O-qbLO~A&uf(SAasC?3e?Xa%+wvj{c=!WV7HC-6VzV-cq-Ii zAWnfuHn4{dTE9P1#A9&|KHOY#i@PcGp1C`&M_DCuLn3y{-5gzwjPwy*IT~vLCwcW|6g={V|ZlYnr+7&+qT&;JGO1RVzXo0w#|xdr(?T= zj_u^;%$d1&&Y3$u>ZyA6uU&7|e!ul%t@Z79O%dklkQsMiXUcnnE)fpdn&8mo5mQ1^ zjP%r8Rylsz|4+ljvMZRJRT?<*p2g?>|m0?;c$Z;gIDVZV^= zDX(imFGxOQdp8GrU|U0<5J4VMKB;&3_jaE$m&hs6%V3n^Hxee)yFUm3=?LcneL#YORT$I( z@d_6P=M}`G*^%610(lKn>h;~T2Z0JB1?Lr=M759ORO%%*>h>!QhX6%Dg@wQ)jzPJC ztC#3iG^+PA9z+2d2`>iCA&4zDP3f?J_MYTg#FVj0Q zC<&@ZEQfJLQ?Ju&JqQC*i%JX8DfqQ*AiJjmVuxG`?Vic0+siYE4JwCR37$nUjdsOQ zuhr`{s0q47IgN1z;t=D97|y0StOEv9P_xeiG9sr@bJ`jXNo&+rV$*BV(Ppf>IGl2? zTRkPKYJ)!|Yg3)d;nM8dq+X5C;_S*^ZKJvBs96m{?&BFIswS2-EYTk;P7gMGSJ}KI zQtf94;S%^3_J@P}V!Zp3QNoj|VXaBHF9;o)vrnO@&u{M{rc;CUZ|j1x{GT$D;-gjQ z84g_m4B3TD;&m7tg`_H&6=K>wN*!xex&ti&HrYjwl#VsW*9Ca_rF78nV)M~E zu&BR>!KWz?ZYE$o8NW^QTDAFZ_&Ino0BzXPTq{WYl_NEu)VBTRK9`HOt?rz4*X*3j znScL`d{tjg?+dcz;N<+lw{4P{camAYnm`ti5*v%fL{|;)pb99QlBNyk)*D6SLIyL@ zvfOu^XjX!q5Pd58BWCYdX_T`5<8I4^udBFcz!lW&Q^_tlI`d4se6C#qJT`gXz-ujb z<3dz=rN+E>^x_eAFuH}?&ynch|_g^-|8J6h7DAz93(d*GVoY= z4>L)}w=cL3!i;@7%upG)Al(h}L@=_dlwF2%yKE#d9+Z5l!M>jS>A3WwQJPHtz0#{B zMjkv%V({<#&lho&u(q822XB%lN5dC1P%|=v>-r@gAwr7#5c7dTb(h?igo}HL?Gcq~ zz3LoAvV_wt?oLa)1$1F0TVQj zw4RiMu$QJ;+kz_5@9nC>wQHZloQ-f1-KuVaT3AMBIr+>o>s=6FfTi7~ z<}=H?BqxO~4|{gw4|)mTPw5I`8@-c1v@)o)6!*1P_;|(XfS;FTh-_K)fyuVIvVUj6 z0TYAT19xB(uLxZQuaic#hFYB_l?vdOG6HSpwshmN6CJ4ncrHfBT`11Rt3EvRQtM?V zfd-?6pv zTuUo`?LRKnb~gIj+8#^78VhS%Yt7)xFK}H!H0ro<=gl=Xc7NCE2=uw@0;C2D4%y!q z-kL(I9^g+gvNyd_7Y_R$k3V|}|KQxhb#jeQAJ*$Ut!Tu-3WVeyx5C@b1k0jjQUet> z1RW@cbkW?DH-sG+hkVc;l>x;oV)fZZP(xs7h)TR7*GxmCXt7FXMJuA$SVQ}09n=pB zR1sZh@(R3?^|3|*Lm&~QR0v9V1@0MpOhd>K=2S6Cc!gg!&vZjhXfw*4KOF-0L?cqE z9rsDlUrR&43)#%1TA}rMIol5P5IwOWiv9_ zS}eBmJxULCGu*7V3O#BMeKH=cZ5KC6+H{>l55Y1Jt@w+c>kdgXVy!L~Hj1Cq4)-%U z*d8rtrn)lZE%?jZU3ytQuY4w0wKS!1mT=6P$Ls}7wqW~{sOJcMpV zr&3wvt-7n8BMvJwYAw5KoIsi)>MF0HytpSusCG6+#jnKy~u9a)HQ zyAcJxf%%51`4tIp!9@8m?NGsOOy$564#YZV3SsqDLmmkZCxOLzps^%GL(UEbNJifn z5Pd+5X0S9n3&5=n!26Cz`-TW4{E8XtGR^5l-pQR54kaSxC$Zk~Me-ufB&Q5CWs$AEis#r6UEGCYX>&2c*uZ<>6v9L??`;hysRKGOL5CRSxT-&2UMf8lz2d zDVa&nN%hgu7Uyt zxk9FlDpQSDaaK|$900^XbVuHGDRyl+1K38<0}=L6dYXw9wfZ(kl+2OCi7@~(aIqn) zI^HI_${?~ao0N8S+VnLltKXGi-TaRV5PbdUfE7pqy#3=0Gy}pOzyUFUu%=&TFu<9K zbU-x#bIb}Dxw>{h2Xq0PJF;s-tz#bG0i{=#CUAgqKnAW<8?bdTuGn)WAmjKmAhlkX z0kclG?}5B6?pzAUIH3isT+pc>1Q!5DgO4`kimlOW3|zzJdMn&fgV}YndNSkg642u`mynS-lT( zkQXAAs1e#7P6Z(c!hZWUi~m2_?f-%-ZT<-? zofzyL?d<>2!ZCHK*0i=;7sGh3(Pu6bVAbQ4I}p!uU51@yIj72(2rfD^Q~a4%7C#6- zq{wB0<`VzeeKZ|dKmz&kmTb*ynakp^|Bl020q-N~b!W+(!|e$_uON5f_u&|S6?d6$;mP6-I&u$D1Y_SGR!vF9p;FQo{Xg9SPD&N!YwKgtzn!a>X+nwuv1l{(FM!bCDvNv8xCf5lQ3rqqdfcYKuXt}AHpT?eD2 z-axZVt~-_mIw9vB#=VlYr%s$73$+P?AKGoo=~*zE^jjPtMqZ#T_=n?R|p21 zE1SffDG~e|W+|j^sm)WpYJzVFd})-=m`pn{a;DmG3%cI z|CDOgFS=-w;Z3Hhb>Ez*5``ti>r`lYn$WI<)ula})QML*1LxbkoC4?&j!@Hcn3LA| zF4&eVoK!+P%0v*}bvM-fN#b+ICl$?{A;z+6(5Ph^}8)L^kDRu&rW{ zXi=%;b?tg_mTeLx>^`k;H}t2(2np(?vZ++hb}o=_tSc%krd?X$4GAwx)HJd1(u(0K z2V(xR!bcV&IknKt)-XQfAgC-w3{T7Gpds>8hU~OMN+)SV`(=L9OmKU7nC`~s`(39U z`TLfHGdue=bMvJ~;vY}~?sp^CrTNwj+4zl$*KJ6rC7v41n%B9jzx00b*l)j^9zPg8 z0_pd%M9x;X5(;$1ww6sBNYr2b*B1cIRFcmog?P!JF>3hR(S(6l1A-ikU%c4U$$YgG z7k}{b8*TGNQ2GLqdn{Bqs+XLjEuJm+o02eg?PP_9L8^prMGJv7P zu}P{(C(>~o1A6Yx1%ftr(SCm;2LF1O)_`#B;r&EFA;OhI(BiQa7N_h6)qkr*F$ zKB714v`zg@;zeZidyprandA=l0PoUP9uu?%u8OLrBUu7xQa9K3pACs0Fg62;=KjS16&xl&9ue?KU+A-POmEv49P@fi;a16!)Zbi-`U_D`a>Nl z`pUk1{W>`T%^1vUh%w9f;zMI8?7hf#_dW`_xF>b!>6#&0g7=9UjV0xHbKayZQ0a}0Wn<)+$+b;l$IMy0 zfZ!9yzYmgon>j-$Q$Lc^pt6dxud(;;=Y2n4jqy7!4D~?oScV*WN*6^(DpgG{qFVVY-^Bb-FX~g9u z*bkU8-;+@~;Z-^VGFCQl_G&}?pnO{x$UH;q$gr{D^;faLZr>|Mr&`6_{6mU+fLJ9w z`lN}nLphDL+EE)u&#!e`LuWHEY`pVGPU2C0K79qwd7UyQCui2Z0Z_A~!4er6DqIRQ z_-x!~KE|-wP5zhwjUR2cn34o0N`Lud&Pi11vj^vUlyl6b=j3rW)m|_PQ%DfR^fqGm z1ObC2f&&!679DK%;K0m`ZG7+yn)tKI!y(TR!<@QhbiME0187;?} zIvob1;#WbRzEsToerrJD1R{{#Rmnx{c`E|0VvO`^t9{!xw_)_^6|oN9%>h$L#H!xs z?Bzmz(iWJ-T5V#k!H$-ivE+y4G;{4$^QUYxQd+hjh?U=-Gg}C5kl;AfYV)nmHU1ro z9~mNl$fYB-Nh%JR8_Y4c?vnj+a{^w;Ii6b1+^el0U_8L?ZLI67)sNd~d>Nvcdhd_a zWS7GS-@7G>ImO!wmL8#ya=7$d@3E%{Vm@CaH5*}`>|5E#nE7BlG2pVN7#0Du=a48z zde&cM_#;Q9Iaza6E+l7BssCfq-#Y3t|qDCP28WQBWiB`(4tAyc3?wIcmaPigl(>ijI4V~R)W5hl~D z(uYij$;R#7&MX>!PRRSdtpmTe-#Rf>8^x@+iY;@`-l} zF8lLxAQtPArggf*#^EKi%q*e|hI(=2qy<_cDuqRjrlj14R;TO-VLXvRNf{H)xNRcY-yJU~>Jg|FpSJ+WauC>7V|tyIpPBJK3|cFvWV zYQHYc%C@N|GiSG>f}oer2FQE~*2FeNYxSh}>BTHIO=}9%(p&xi2m1eEQfZ&A;1%^X z|8Ph7FZ~f88AX_WCbWr6sUxH_XcNMZ)NH}GE8zvw zg$rg5Gg=#bcQyfDzfcsOHx6QG;<(47XHbQFq%`P*pg&6)Tfs~0UNM;Ju#?cm4(e=m zvDLr-miQRd#uzV@^1P0;llaxGXioUfpCWN0nUq3Rd)nZ-aJm6V4DUb1Q%=YS5V0x7 zO2hpmXy(L>6QBRh@CCJ83Bd5bjGg~K-$&HR*wEfo+0?BoOc(MxQ2@h|0CGd#JtKUfCo+_T z+mNZD;8Ap5et>Fn1}i2r5+6ew;gZ^r;{q0}1J1ynp+KT&6M2Ruo|*TJ_OD%vHT4Te zummL({`hI{a79oC7+47yY>7Yjz;BeJTe_X_R`i+RFwnofFEdbXy(1fh*T=tm{=$pI z{a{T&AIxJnWqkOgC=g7XvkZa3ufTECbSj@MAsoX*@`bmHH|Z2o8X z{~7OObtrGtBlKUnE^|H!h{n0}KZpImf>s)$jT1NpDZn-PjfZf6Wcy(V_i?p(2GS8? zenlN8w(WBemWdSj7ZO?{AY9S7QkbkKTe6R~?T->rCtjtAbbwXXtJ}}5kM7H^M|eKh zkKOrivjZVE_Ou(u!P+}|0dA331i0Tt$Go7oDEph>IXiO)36XXK*~@~xByNal0sgOm z?~kbWhAdCtsm(;(dm*Elq)QMJF39?*jbOo@*x}xg6bZ@y8qQ58dij^dBJ&E_XYA~g zWbIx-(|V)u5A8m)Zha}8V{_8>bKy1(m{)Ty6q-C^O+7lXuYl!Ju3}0&h$i@imocJx z2k%sZ>ytil_ier*loe{9oc=v`{><`#rqiCox%I)->t>_7~Z|a z`{>)v#mhOkVnp~nydp&S{9~UppqvF_9@U65Et3VEmCn^@PvJPs)#zxpUzPG>3{nsX z19EXx71Owwwdl0en?9*oIGQ11hPjFZ+57W`B2jDjN4Z&}R*GPDLUg4iebr{ta)JZ= zy>dN3Zpuv<5KM2teGcQTE?~G2ljm$WBk-Tb6p53RH79_u3&1n6?=#G^@7Ud zlomy5SmmZATjC%oha6F3PjgXr*x4UA{;zORvjVE&^!oSq%Op?MByOS3)V}t*k#na|qd;R2`9WN`L7{Q?QC!7-I2>9Et|BwU z=S9}x2Zm&|Yxc2jkd^wD#(x)Pfcl*g7!VB+tfa$rug1ORNEs$N#48N~(K+HCf|$x$j0kg|dSR5eOf z%CiAat!EE!{!t2U7rKvzS>DpWPPQbtCKtdEa1Op%1@XA-IMH{$Ow%wyzBJ%EYoH!> zCIx+Aq|UtJeN7T4UOQM*;!q>#r#@g!223JmyuH#zYORgXYWM|5Iid-eMqD zdjRcb8B~&`9{$@snY;)#*{ zL-Xubg<`on`7+0rJh6|bbZ{TAb~hm<@%Pa~dOaf4qbuO?J}jUOFZ!N#{80j=7Z7E9uNVJghR5!p#FdthoiZ6FA?8 z#YO^NluN9Foq6!qbpJdzbiONTOBoIXEM!hvLY`HF^~uHXVseYxB2*(y z4Q#bS?M7xwUw!j0kYmjOxWnP7f!T6{R^5erZ0y3w4%M4m{H0|bmef!$fyX6|K``_p zCPph&U_7rlR;r(z=0Vx+c>4)S7zJw}^&);fWzAW7E1u|kbM3aI>w%bdER+0th3G`L z^cEoQ1|eVX4f^~&!38zB>HuSvMPJ%`kOf7|4Rxq&;PLI4@VWE%)yYppke*EET=>$T zI=`=F?V%8&V%W@dhH23FxDa{&@`f`|_OU{#5#Ba_<64rR^q(S6^PIX>h%_Wi9a^Bx z@N6qzw))%bf@Yw~YM+vQoDtj(TE&mhf!|NJfSG5Q3#9>%559TcZ`D)hg=xV?X-5h3zQ1v^oyTef z2;v#68#q}aRpLE0To%v z!)>Uxbeh_xChhAew3fIms4^oZz~L^Qo++6U_jH*|jz;>7Ta1j(_MlfzkPZ!{RkpZc zQ^(g2cUU6ir7h@DipWMn^P=g1I2f+$4t3a#>7fs0!gZNPy(b(}mCNF_&}zdWA1QJ^ zTSxmMr(d&~2pxD#x`sqR1pT$y8)_Z%R)CHFrCDL6Sp1KD^%G8L{KnbxTHhl&XX55} zI6(ehMu(ZgIv;H6>Dqnctg(e|fG{VN(GhDJe!t976u$yI-)!`i?Sa*mmn!oc0^c`^ z>xlG%o$-Vn8sS-O6dk?+vnVxw5I3CNjH`*OiAd}2L;it1{42Quqip6p5sXvVx?8xW z)V&?!wK^P*hq)`>RwiYiR&<#T4lKdTV0;l9D(+Vv`|1uD`*U|mMwtzGV2o$g;l=g3 z4B(|)yk5G77NLJvxApQ)&`ODj1d;99Vsvf168el0iK@1OA6IH?D$Vj%^Y9(ckcJRg@#KO;8T2cH}<0gmccmDBd*TSP%9xY@e{*+HK&ZyOF(F0yWVo5KekTCyHEG zoNgPmYz2iN7Dl4zGebM$0QW#)%_FLS6?rzpbhShLGiyj)I!~;2prwwJO-b0dRFHV4 zb(U{+=#C^S;1ak?M(FaFGD42f38!?T*ZpMn7f3}=?4YGZ2a{7)qNB}F5Ts))pwYTMzgnGA|fQ@hdD2ULQzG#^(COBXBoFe-#>S}IY=K~DI)CH#?t;2}c@HI6cKO+HviRT~*B z!^8;fT#@NwP()OOWJq0?j+ff>EnDAgvNp5&l*+2JRJrJ`mK^oKtRW+hi&vksY{zhK z101oJW|cbEK)6o6QK?!RZB;;eMTTv32RJK4OB?Bpitv+bMGfyC1pmV74;RB^<@p-x znxm?Wyo9R)RYw^`w{r8fTC+Jj1wGd(8DxG|ORkhOqp%?#N)yya?4(tx_!sM2YY07b zwW0Fz)^Vmo*yp34wYo)v!QwEO8`;$h_u`KQE5-gX z#%Xg7;P2cBh3Z3W_{+@I#*mG0K3+Bzq%lDeT==)G`Gttg_#DW_$M0VKGMhA*ZI$~N zV$LS)0$ALiO`VSx!I;O3$8tLLX2nnJ*919-w$i(b;zh2n z5`sZm%^n>zOI;PACgBF$CGM`2>ecmwhdZBGrMT{AYG?H0Bl{iL+o^!Qsp6!J`>YJH z78RX_!9*@SARv+reLs670WMe;(i;*o|uE z%2_xKC88l32UY1X3FX8iDoym_BKF$Dl={r=wq#niTjV+yB@f&EYbW4s1!p_wOHSDR zi%#_K!mB_!92`37U?Jip=JD{3^Ax1Yq zmRMF;wUBta6=)nTv#x5!J#VwTX{w50xZ4grnUv1YJ|^=^rSwP-THM3MVP=26?a2Fa zKKbMM@loI#U2g=j{~zDM`hHn8A-Mo-VYCSr{@o80j?}iX5v0CS7By2e6P^FrVx4UB0t%PUwb@UsjY=iodzwX)=>uD zAQ`>3czRZ!&IZ;!+ptQ(#y7FsxMfpoQL3d1XxuVy?IpXcS42&asCQkCios_xYd#AG z*(uq@d#2R^r?s2nG9tBHJ#6c_X|s&evB0hhTZ=fL;CLvy`FghQrM78^B?n26+TCrczB!a`ynjK3lCmYzHUU|7KoMedXTt4lQ2W#;Tedm<@q3NYqzd8&{ zYncIt6Ab@gK>&wG(xF*SCd4Fj*-TEBbG;UQNBsa(s<$5SDEV$iEI=;q=JVG5N%Wlk zi=dl7dr~PeD~mTsHNA&tnRicTlU$k3PDc_vTHMTf&PyYEH!g>Jj0X5W1(x z5OZa@J@e`-=sx@hBZtBN${BG~-17Hk-W#ZB>GS$tnFII(EfI{Q=jXNj-OTs+&pxbQ zVR6^r@?Q%MA*=ZnVJm{m#%^j?9vd!Iwa`EO=V-8YL%(0{2jt5|8f+<0iZa=hDU)z| zMU5^o7ifzg&ld@g?uQ-=h@5?|ee)2CVaSlb5XYH-_WlgQ%=%VOp1UHNyAmOP)*ug9 zq%^aV630EZ2*f{E5m)A+1eZevv8v_^siJ<3bC-$h4Oa;TK4%>-y=2*v(3$CSDD@EG za7nyJKh0>!z6U=guR`SnJrapNAVZtG7gRhNTusL^CGGgC$zZo%fYJ^si?aE+S&u8fvA1iVtU? zP{~G;_j-f+*C%@@0O_yKS5Hjo>qz@wjuPMMA#*htTjmO_(}rLe?$W5t*Ib z&GqvOf&o-AsEo`({D4VLXE|1>%j6&vE}|261UCj2kp5^ma>F(7k#6}?3k+E#XQG+_ zQ3zQ>ZjkghI!$})DRhWR<8&O4b8%_saTsw|o6A_9>N^H~%yJU#`>gUZJaY4UN{lF& zvoK=6}bY+}@n2ZU@_cFwaQ#TF?Rs+%4l$D7fe}1 zu%F@^Qj)zb%p=Jpo%Sr6&6K$VR(!-9&1Ircgi@Qh*~{=S3Q8R{U-rA@+hI9o!a(@s z#=o^Sf#Tcap!ZM*ZkdEni_H19COK-o-rOFZbO~%RaoFBA(rD*eIB`77cZz1n@ z+S56kd`F2NA4iWp-|qHHsrucSLwIc~Mk~@{4Zksl@BYEk>#KyTg=Q030TmXsi0?yd zAFU&EPR}QE4&HUi#~7$k$jql}>bI9J8`y!l$Epxo###!t4Kq{*+=gInp$%v=XL#{M zNV|*kc1_r89(%!2hY*Jth9HM1hA@XXgs={P3x^mc7)lMj((o&MgW$DZFG&Q#EwkxD zAa6EJ29)kN*C^t8QyVUoBquRtF{p#f&q5MFh?voe3iB?+YUZ5GimWmJ5pnOQR=4mS z*2cKC^0uYp$o!3-l`b%zPhuUSitfghdCWBj%B>j>RQ=PQE(!q_z7bJxcBdKACGuNt zOb_RbYi_ALLH8yEy@?=>c>PByQAKJBV4#w_$&iZJ%qUXMqZy;%r<2S}?ztYvX2qWq zipBFM!O*sHzhan_p7mfpWbBUi4DPq8fk3<&7#?#j{y}k=c0}!d=IB^1Gke(~O1#{? zAMLfYoBh{Z-8-#tiZZEmeeozW6`HpYBXNULQ+Rj-proQF;4u!8s+LeOO+^XL@FQiD znnb7eQQ=3P!mYu^LLulyrkpG+RTAC1X=pPrn|9!=GU2<@-b2@H-Mdd1+YZOAkWcCTg)5PdkMV*A*d9y|zOe}*<*QB=^SV;{R zsu@~8VyeRd552mc`!Akjv~=>g$(-oa7nsy~v`XIk@n{QE%6iRbXwpE#=!bbqE5Gi_ z=)&`U#&4G6%f$MjKKR!z3(L&1vC!`3z`dyX2#clrM}`Kw zpz7L-lcy_iWcqyX`nE?=hmxq4cBu*K*Y0}$I?Im%ykv1RHSSRqlGba)pj+ASS>3x0 zK^Y+yI)CZo>Qn_-9?bYWNq3>-S^8bx#95kr`O2YYNl&Fp2zJT8%U1F92|05G3EX6S z)k9=1lI?kH65P}$1;gaob20Cdcm)W5pP8EFW|oy_?!1FCijPF8?|BPjjy7(Ca|MfIl?D%YU`zovg zzQzRq3(i!rvo^K;X9{yk4%b%}M;iHsDAW+7o_{S;JwyN_D0Qw|w}zF2GbW1i^XfY@ zFLB1i!LDNSLrvGE@VU>b+PWu5HBil~<(G;;_UajG<>=9X?-2(NPh&^A>=(e?fcQ3X z-GY0{q!#IUwr9wJ=(WLm=CX^-@nG14D~D&luZUMwyvyuA0+C~Il*o(cyi1KBFj_Bv zdj*F$<9J5wf8>7QA@>r{8#$Q@u4mRwaz%5?=D(b7_q}t>TYAUlkO%aV1rZj{kO191 z_PM(kY7!BG-^iEl3i?*reNimcD#)Tb;x(^XB zRB%`2Ea*Fl4H5#8y1WtZ_YsSNb$Lb+o4fp48~q*x@Pb^w4q`-8jp1$viOE&MV{=)U zux1S_pR+X%+zz7vjknL@z=`9rAC4xPQZCV%+mn__^P+>#*?Cx~QSo9=CbW#zE|&2P zVpvX=M#x;1iGlf(&>>xJ6#vF&3^75ce`Fk=*fiHhk~pJ5%l4E}E9?CpygUrNyN<>T zk>yTTE#qXHw6aaN?P<(}Dt%n1ef_ayftgEXoKC)0+~!1=On=)?n<5Id<(OA9pCC%c zlT8+M*li9#ADgo6TfF45&zdsMRwK^<7~7=b{111so#B(+R1Us zFh9PaCEhG@f>>e<6RuoimK?e)zM@);bAGG3cx+QkG7J|!Gtjyr*g1k`sF+H70DJ9c zz))0zWpwfld7|A!k?~_g3yuElZ_3+wkmDSeyGF-iSP%Z_7M762t^Zudh&kTCbkV^N;t=H+fMuy{%AdR#vE7(hN2*Y|*R`Sqvc^%h{6X4>M-=DX%Hq z&I`6UI3%i!_Y!6)!%~8y6+XlHV^aE#O0w zjYX8RyC<`f8-J$p9kcqt`cpwnz=wF`_eh6_YR?rqT1H`PfkGnVx(IH?g9pe8er=ts zPjMR8`dK;&{dByYPlblN;Eme&8<`t&Q(3%lzA$tRr+f|m-6pxggl^xV&+ynL89vS$ z^oLx9@Sjqyr2U(GRx|~bTWO2#49=c=l)tfgB`+eUq6?v&s1|uB@TjfVL0ZE4tX5Mx zBu7=Zn z)i}p!z7F8r{?otN>xCcWGs1kaVfT>#r@O^}wg&zqWm|gwmv>>bnzl3Q5%R~9mLG>!({g=SgRaeJ1 zf&n{T&aflil@MeO1cRjT3u4C@$%Sa2u=+{+&ZgE@jhG`%-eHq0$*eIC3QS%JgMd%+ z^rEyP&36*q8LH8CL!-H9>8GO5QBw%@rzurL_I-$tM`|^Xdc0hL+C|#Pj+jQNq3s`^C_E)dz)6~0^ zpd38wdNf1?4tS|J;qo-W*dmA3B!yGEzu_emS`78YWm^dp+hQff0xo?L1W7w5?Rac+ z{ZK>InYycGIqaN5&@LbMtnxA?GW!Oo5Us>H(M8b~I`SIstAs}NS?p1;C`k@{>y$$F zPnw;I90Qi5l7=v_C^Tnyn&aRF&Sg~-S}ApolT5-G<7!qu3Nnws$osOpkmMhW_cU1= zR5?h|#QTrKDWesv0uX~^h=l8He@|_V1!3#IhV1E6mvxSobQqI`3lMTpZSbAqNzQA< z2i-ha^z)0YJ?2Fq<)~gmI?^EQheedm<`*CuqQ7G)Fb50MtT{-4s$}&_qR-@s4_QKr zr11sU1zeUt@kfSgcxOp+@aOMAJ>^GO9+0!9mQeDf%ayX2MJn=srqsw#BvUU11`w9F z#t_oFD-X$Er9@!ACBb;yyK+9|GCy-A;?h7Omw;{wT6XJiGTW;SjX1~*DLNxLu3C&J zO+{L$yn)(PK9MJ8%4`|3gtnX%sX;k9v*d(f7u_(}R$uBY1VpS;m8Ff~qxbZC;Ys`N zRhNcT)vAAaP&WoYmOEWw`o2{N4jk7+cy-Pv1O?4xa}=mxqnkyXwvZ=@bts)U zOlDowyJyK__FOm^SKMD!-glTEExD)i$1CBd$hYd6b0Cj8NaJFF2|bMX{LHd6*CC9& z+KI2LZ0y)ykuWK%?b*MRSRP9x-AyFL&sR_)l+4_m$)$6tTyvI{(M%9iyh|sHYpp=cz{Nz@>o8Qp@25tTuL-#L7cnd^7Rfrdw5M z<@c>wYywWLes$OEi_5>KI#N`U>A3sWL{01tt|M`@L=toe5HzpW7CpQ8T+te)M?m=@sEIIXilExUbz#BLcy+D!1yhIC3 zhxL=Lz#LV74+(z*5Yv|3?+aTYZwOD6*H0vHh7$p0ewjJ3;4t=bebk$961VpE;C6!X zrIu4aUGZQxgr0Y4+$wecSUSx)GILXQ>F|)B*Y2l!01)F^B0S2tRG_|m&0-O@#XKr` zNwf1ga2>HQM=e{;ifsMuV^Dr-2*J`?`}0~I&xlo%BidlI82@N6!1x?Av*5bu4u`qU z^6zK43I2@pHANs5)qfhX()VIT^qQ!xYo8m^Cp_3wUFrt2fdL$ma`8x3!%S#Tbd95# z_G-(O*CDBLl}Wdcm|vmgO1Q)bd$AxD>9kO@MI@hZ!l6c$Xtr(dn7Ar~Z2iVMO%;3I zjnr!!&la!rQTu=Z5M&HsOr;Ghfm$6r*4vEM{|s_tLew`QXnw4cNGWpBYhz%KZD*&1 zXy*%!R*Oig5MR@QU13()hg)hrtiLUx$|@CQR_dE8fL){EIQfR2DKlfX)kB5up>jZn zSZbUS*g;%T6A>ni#%M3A0s0|o4OOk%tmQ)xJ<(yT9eS#9oMcY#Ad@vuINHP;yuD;) zeygETXzz-1alfEfxrnpuG}{WvWyXbtju3LWZ&1Bf2ukQK(P^wk))dtfa#rJ$!6_%( zA5~O;ENHoHYm)9-dUm8-cuWw}?-l=j)+K2HvFKcv8O-m~lRSwcIhjx#)7ssyj{U1< zb;Xa*jZWVv=1%KDI&>_7zz1&4Gt-8U065eBn#vSUV%CzKjZ@jZ=Hp5yG}7`5Xpz#* zO`qJLxz-w*^UQUYQ2-wX@|77v@;%PFDTA~FdW4%Ta^G$;cP=B1V`J?Ryw(oZvhF-L z;b}W+krXW!=W&V_f1WZn;aIxN?b(l5!q0ksnGM}0PR``l%Q9FRmD?bEB%}P2gw;(2 z(vyo4Y|5LGc^uA~?4R#GZTmghQ66j;Kk<3I*@U~n_;@pg`c+>x++Bvn@6eU&&?+14 z{}`nwlJHoS7S27&G*X?_t1V!g9jtKhNCwT2uU?p@|JVebq5D|vC#*;4o1-c51bIDP zougPDWvc-t=!%wQYbZg#(m1i$A|Zix&t|Vh`jto_h@ZW&KaYWnCj29|C1|hi=UmR9 z9Jn6`c9XbeWhH0zAGH>A16r`ono|qlNb66pWa|i(UepD%7`4X=?Y&q%w-___Z5?=z z1ue)M6$RSqMCY^7^B^nQt>saU^KpDSb`*IPh6l4(CD%#rwQ4Hs^A1Y!dmUEdmaiL~ zZ&~ ze?{g0!=R6EzA0SI>vsUs8;FmV_XBkkGT72j?SLl~;3VOE>B&o6J#);_9Ure5rPf#{r@4tM76nWHf$nY^G`Ty>!9T)Pa0_BlwPn~+<< zGH!LzJY$eOuz5O(`$w}T26qF{aSl5LnSnsfky4cSypq$x7P>Unl3&)`fj=B0{&7I; zxU8Upt*@hf+FksuhZWx>VtM9Ls(kyW(D}+VxP&ARO7Q#C+H9kqbi*}qc*R23CeP-(%M3^3ug}|&+;3ZgYypyVLDx9^341^^LTLx_2vO>b z3JEbJAt{`b?^;n1N7$693B0Pfmlsm!O21nT{|_m6H#!5X3bMP8)z}d2f+iU z8@q5EPR%Lh0x0++We#1oxL-QQm4+HZm)v6W_Hc%$l_R7_*!^aUe>2GYPJ_r!F)SR{ zC2d`*Yl~X|-n8GFG@G+ycFbZDxV7Q(g`o!AFNl8)x)c_hSGg`zci4{O@#r*c?VbaV znKn2<8)GZUx>4{A7e!t&87Q}bn1Yf4W{az~r~x|xhS>jyv3FpsbY0f9J9av@jgD>G zwryJ-+jhrJ$F|e4ZQIrvYwN5v_nF`Ne!{3xb-j1hQ<$^~QG6fpTv5XnF!hWUkZ3wp zCXu&lvY!2Han7-XEaBVJx;atHtrIPS-o5F4Wc<B^zRk3#ktAz1MJw1HoAJC5M?*&lmi zhV0V~Dn2&~%Mn6dp;fI~rfjs>o`2C@f^z|RT;|RxL5_b{U0M5RM~%%U0<~eLYne7> z5&GSNakaW^2}bpSE=Am5OROw)}d(f)A59^P3mWVx4#3`FNcXj^9j@1Jj1Lzt`Cy>2tZaG%io>E z7@GWw3BPg;F_LrZQ0Gk4Jx6IIQWD!p|8NF{{rn4Pk8;QqO98N;0>FafZ$SIcuy8W6 zu=szILRm)*TLi;5#QTQ^S_JT5*hy$3YvD|l>Kb&IVBO$!5P<~WY`KH@kg-+W=t}lI z%GKNVjxx5#`w*>%hHXloi%z~_{9j%5a2ADG#C04VZig#JAHHUv&u61d-|%}D9dNEr z*+ZDHb`hnC^Ag!Bh<_FaY$+|jS(=E7XU$2R2*Cl)&czGzF4_wWBWlBbZ1|vfN`%Wf zkL#;e9Wr~D8zJ3Ayrd)LTB*zHp}9yC77|fmah$z#hO>>W}j48dN;b_FzF{MSxcaIAWe9lW)}b=ly#G>0`L`KC*7f5|Q~B5UFt!F=)7#TPHr+Cu8uOO8~*(Kag z9ZJoi^xcDmAeqSt2O{U$)adzO^;32T857mg3-X>!4WNNv?mggwO7?Cc ztOMWlPeCpCp5_J8&5W$i&-H5S^WH#u0=lAu^Iib$OQ7xJJ36K(h@ys1&qh(_+u|13 z1x&pT9io~>e4E4j{O3R^kL3fGJVWk306nC#a09+Wwaat8>GF&SaA1brMnqVDNJoGb zSPdwAX!!l3u#9hyL83$`14b1cvF)r0+rno)KQg(}s1UEJ%%m6GYjXx4}U=mTLVXrzlr}XH)Ii%&*Bx=%nLso)9eGe|%SIV=X^D*0!r(BhlF=+@hIRk|?8C0!A-c9Bar9mz@XUEq0_k2ZUI6_g5oV} zAi?5EE`6v2f;M^9@@xAE<@R%%Z$r#SNBIy0B-LJH7!?H3yEG1yMUs-KVvjg@95Za! z2sigQ%?u;Mgn9c`32XO1CDx+MXQd$r*~ssv1!)6imNYGHdt$+r?y<%ac2QX zPMzjt9`_`yM{|eQEf~dDR=gUcZ2X=xFvV6GEKmd4QDZ&%tYb5OYUTO`$7y;Oc-&ut_{q zZB$1LDsrwtvOz~P&xZ)tn$(lpeN$Pa-4SsRdw)+=Onn3&-YSGp112Ta69wL^JxWRacQ{&$e|EBz@D`V~aww_a&tv_JPtGLUB65;PS0Q$Aa0y4iNY> zBX;X(sOf(?zMW?2BuKiB(2Thz3Nrm!gavg;>0Y9S4I~kA7=oN+?a+2JY75dpeEL+96>q z@`@aI59~+M#9)s8AOHa<%nHFSA1j1j#}O4R+n;Uf({fVaIWV0yl3gcD1e`x!O|Qj2fT_-O1dDixBi zN%06XRG>*KfEZQ>ru1?~sOg6uKMPN0+ECj~@fZ$4U!duTou~!r2LAh;NHafOtzZ?t z7%$;=%o%8nUgE~$)goy6`7gspvQ% zt6=!b^0qm*C#^OZ{t}2IvM99{30_rgD5NYVk&&4d6oy;dkF(9(2-=Xbnb!LBXR{~X z1M9pI#93ue>Gc}M{0iq2@R?$j5i%H3OiZu8;yv=&ymESc>h|*i-`4S_-h(?0c$FW*5G9S0N;@79YxyyETMLVs8+IY#uY;(lOd$*9DBaf(^{x8# z%iH#zJ(6g(N5`iJsc%8)Y6ZdCF9fs-~c*wV6p)Uq=ua)Dk8l!~Cz-R78!Zz_zZaVsQgB3Gwd2?fOonEIOJgYq7MoPt=t4$8s*BiB z6K9D=MlY@{X=?#g^v&}$a3Gx~r6agso_6iY2^@?(rGr_@Zw?sER?%%?_hVF_feA~S zOqtrT+D72toSuUXr0c(=UAU+^P<7mLsUe9=wr#RZ*tafCmuxe(4$;_PxLr$%5mmcw zEf%PnL6e`^Zh`}xFjPzT&`G>U5$vs$7snG;9VdTX$^)D`q01IbiLlu?u&#QecB!424= zql`{WCG2s)SSqQWHGf#O4AOf}A5xU=RoA@0^%uDkHwo37FI&2V8dc1YCcuz1of~VD zhD3h#W@(eemC_-Aa3$i%HM&rCa_!g_uC0DKjlX4zZfF*xrF!&YqCX=NcjG$LQ_c?$ zR7?=Sxj)^JC+tYKhc^*srne!bJmn!?fsU22!ZuCZ4q)2^jnYVR{BMin%ohjE$QAR7# zG3E!YpKsq;Q@;fjC(GFEX|QB6G>V7B+Af{{I-VPbM~NRLO!q^D{pT}?D zu`MP_1dA~`*CKkYz1hksgy+isw( z3Hj+0%<_vFBhP|>27QvjSM{gb)9{dO(B9|cXVbpnW|70!rbGBHJ?RB_;CE`pId$dE zG(r{m_je<%e*6l?TM_Xhi{X1956<0@>2T#p@Cw+R=CKhv26u$JkA8WbmU|QHZjUX< znHJKv!SXN~7C+tu56Igw_4w`{D)X6UC#Zs4G=2}fLw*^-?K}_*22in z*yQiXr3P3#yOHWGjX|s4pSoPETESYljnb$NqyU*0LKYOlpQ#$1Y7YmJE|Hb)jJov# z(KV0W{z~xs)UuaU=V60<={ENnWZr2GK`}yVK*Q>CeWiY?>*(Ce_Vz57j}M4GfX<~O zLIC#&Va~o5R|sR#!d)u#dq@BsrM2+Hrfta%F7DNCT}HH}yPOz#HRz|MHnfl4%<#Om z_An@~e7yu5e#muv46k%qm<7uK%Q%ItV1Ff%&H^IfQbz*TAw~dlv5jXU?25Ui&N^a< z219~t;~|Y_49cQAqe`Xv5blY)sRBktwM8YgoOrwO>TIcp>B_zI2^LHrV#dtgqBR+7q8<#Kh170z$M!T2HW zKnHad)^xeZKl9vJnKxLBI4~`f@q_tDe*oLmbBr_$*z6ODW3{ksoxB6&ICzEpcV=r9 za|y3Qh|5(@D0ZeeK%sG{C0oJ*LR8?x>DVRJjo4M1yMCM}2y|t-hO?SdvT<=UmIuhh zxLDgK)%H&N@Y^i*8ticPCnsk7F(}AaEoaoWMRhon*~VJFNLpT_`Z=Y!y8XE-L(>mO~;|ol9-b( z`T+SPSUaaqVYtlnfU16$x_ZG%LeW9B*G7HStI(}V6~8Bmh-W08|NgOI)}k5Ob>hz= z1FM=^Nl@G4f|7BQSmYzCqh^NOy0-X$Q+T?dVNT^q6ST>W=S^cqXPz+H#P9ujxULh= zgcKX}7-H7QWU3eJ$fo5u4UJS0J8ln9<*mVAD%Y$PB!4K@eE+s^x^dmG?8rCG z6iDa@ORAhJCSq)Eok+8`H^fm-Q&ui1^Qwx+ZD!&znhrmxihR?+coh%C-3vnH%MIGP=h&2rmU-LL0y3Md0UiHGS<%Il_a z#8<;6+VP1WStNEA3(4W`d1)GHX2;8rQe~ie2VG z@7J1~ILb5j|81rE2RLdJvAOjDO}%dbc!>R5aM-yx+q*c^{cq_bY~XD0&&APyjIqqg zWB5)kR#(a@{Gw1qNs6^aruLOajv8fcD&)*Eh%#OQV_ieE`lYiP6Sg#jzfDJmh$xw|7HS5F*l zl?A->(E4O;{h=zOY>7weNC%}eTg&>j!L62beAph>6PB$8VZH^kpXM0WY;8KTxHAtq z%fxQL9@{MdG5~8(C&Za9X{mVs1a6L{xn)Xc$^ztM<;b`!en(=s-uU$}7an7X6YlZ8 zQShlpC*;QWAzJp(nY`ZGYeKZ&!AEy$g{4GRxqX8H)(t3vXE8|i7^Fj;edW&ZT9>Jl z+q3#^N0F?3oM&o%JN6kqP*GE7v?z5cdf4hDptd20^60J=HKeuA4^_PovF0QML$u+! zTN*=@rGT$Dv1ib`D@m(JVU?9Fv*S66h`GRS)V(`l;giCE##SHN%)G;b^Y{xgz7UFp zLwc1RloKd0+^0seE_hnL1#Aa7z%D3-e+RlsA&XDeBM?ZOM6Y%q0`@|v5;7O@1-LBO zzrW@cZ)V@WEi&CmE{S{BZ6vwo6}3bXmCyrW1kEsX2Zhp-hjdl|5yb($K7zhBa#R?m zL@7QF$peC+Aypalp$y4G%a^7mw%d=YZ&UA__M6zhI)kTo=(@B+;*?pVs>+4?D3TATmk2FnV2f9{C<|9r-@( zf4-bImwYRhXP}R81}x#3h?KC@I% zfK2_KV8m(PeaXo(s+r?xUa7eZ!?U;RdbKf$dZ`|!Lm~x-tTS_d|!_STiu8TUm(LvUwTNs7- z2tKT#eZU@=%E3EC4!gK>DxCVK?xQMYyqRRh78sXx7%7FrQC-1$iT(CDX^#noJwpA! z9mWXuD3;wt%F{*8)7e~QS##UkCY;|P+Cdv)p{2NmSIra`3>O=R?FMW(^*MsjLmfK? z$u@p>N$t0+tj@l@yeY;z_5w|I{~%+3aA0z9LZM}lDnd9uw8cc~=ur!!(34akWCUW6 z!RgB`(afU}r-{4F*reN-e#^a|C(-FoU%K)6%)CC{vub-6KDM|7TB5d~}Y ztnzY8)w~piuVMz(eVoEVvV^F{Nni}h;ZlGH47>hy%?%i z07q?K=UViR`YbU<;xkGD--Eax#7k>u$BT7sDj=O0cj>b>Z+4AKe~zvx*CgL*4*+qZl5t?sOXL zFu6@;b2tq>F4^UFeOnH8-Z5+sgs-O8Hfb5M&(52o*T2R+XK&frF6QUrqEcazwenw+ z%O^@CFRye#`O`AX2!AGjK<8XtIBBIqW>83l$;|>Mq_V~Ip%5Qaugwn%j5ij1BaomR zL9%g9K`r@^=6w7)ofJDHkuT{#bSDP9p}~80Z8{*|tqNTA#<}zW;?MQ*<%SL#Xa@YK zi)6{E@5%sKwr2}J6*}db zSfJx}^%r6vIx@4j6>wzl;{{Yk*F!6hKqORX-o=&u3>JEo(kosiO-+DGf;Lki=;BFh z5FFAGAx}iWF3xxJbjsojGY%;2PobV6vZi8MwPIkM9Y6j(IGi^c>$B{c9=Ct>7~BKw z&UJvNC)dr$0;7F?^Gj-RLl8%|2S41q*`#OU>(<-)W=sDk9{qTSLMVPm-N2qsb~mtZ zV{pAX#dCr;f>Z}$hDF41Da?q5w$6`qOdwMX9zRmuAq$JiL-jbhA_MidrE+hM03fVD0pC>me`~1! ztbYHqrIMAUoRGx;&*KT5vy9-_Tph5W;ytri>f_ksW1jq~BVpp8H5_S;t|W`3Rm$ze zJ?k%s>+JG_gXk32=nkm%ld+OB^$DoM4jkcy77zE%ejhG!SD!aWS9;&1w`*g3*sG1` zWX&<82*Q(O`=${G-2=wSli7=KWT_{VS25v`IoT(v#yChPsT~BuBf0FMQWs88;nY~9 zsm86as;ydjr~M`#v?cx&8RyMfr%eUZZA92``de`}H3X|U@fft_p!hCShxIf_5}7ZV zXpBgVN%9znnPkN1i!V>vG}}%sNyL&Vp#|o6X&};^uy*qI%Q)T#U(jEd$0eIR4MhIf z3GD*SUz-%Q31%cD;ZyCWY!30YuuQ%Ryu`rIu6C*alh=Gj^k;Qs2du2@1@pWX#QO0`xsg$PCoaypp^fF-Ed+{P>M>Jgke5f`vbe|xOcM-DpWRF?R~cZoA(Bdi z@s*w@=}%+m^Q^MAi||4(U~tg-`;wg_LH`+cN40iSbQ0O<-Dw;M`f{>2{%o*QUjnW;MIsRdbI*NXhv8Bs$p(rG1x%; z)Z(6ur9}{JwB*J{gy{pVlSZsj7&_<%G8ykFcGHl#?iBWG2nMti97iJ;AU>WEz<`vJ zYALi+jDL)^(CiV1)ackcHmf{g&>oG4RN34Wo27@SI?z-Zx`Vq}ru4In-MH6VrmZjB z2VBYTZ?bG{Sd$Ya z*{W1@j3P|gV6BCyvSnqc-m{DNA-mcRiA!|x3KA-6+uG19uH4~DgPa?AVh>fvENSo z<2h4wuIw_uriM8g;qs;yVx*l!w288yIwX~eDPsvPzG1rHOPjPqmiFp^g2gLapcB@f zD(&6U_8DR-;1g=Diha?du+WSr_IU>}^9j9|Hrx**UiVHRlpKBgu+b;lX_`5Z#rr+? z1ipEm_QhXvM*K-5^%e9?JRmtshq2tx^v3=`lJ<^RyYD7mfB}D@EN= zCasjTPA7ixc-hO{m6DKD@agjZp@#x|8A@dWv*a6^~}yqqr(2dX(A*aJ>6X*`r)iC<4~MQ?EdTuCCJYz(pWVz&`M zU%>yWlsU54bMkutvYJxWEDL8 z^6(sgAkEiQ%_vI#K-09r}r|D*c;KXA7A zNA~eQMTox^&XNILh5sR&+HW;)iS`DD_{GPG!6gt_N#as*hXqpvZY9B`njmqKL^2Xm z&D~Hh!uT)C{(K(^F2`XHhCwe~&Qna@&%sZgW=f$E9!;tby-d&X_2${&c>`R2dk5P_ zx*Uk_NyaF?4kyW#yiF&`mAVZkadH^nlLCQpSoKC7CSqjJ7GVt`-G}`lHQIRW*GUKc z5X~2}$9W454I6HFsA|KZHmdtO`L5Xs_Q#Yx1@?2Ey?5mjoV8~XANzF(sq&wbHn6VY~~w$HX5*EuNp zH~gst-Cgxn1X#%6{xo~=doOnpE5^O`Gy(UFCpBcal>{YorgqYEV0NS-d%QH9i+jHV zX_ot?Kz4JmRI`8zfHaJQSBa7+o=-K20P_pJd7Y3v#qlK6bXC!zXwrLiFk%MaivF5VATW4S(8girz!E z8Y^#QyuOdq4)oIMi+5rLUoDj8f=vW+r*yK3qNk|MDU_7^Tx%|DnoELZCP^v9DrW?{&{Z4U`f<1bSWHc8?q=mK0H>jbq?1(B_cQoAqB(XTFzHb_e zg+Y+ubWakuKPkSmI)8ni!hL&C&5&9VYUdZcIzz*KQMJgCf)Y*)7A*gjo8QbVaJ5hI z@PQ~DxI=M4#4h@__xpUypO!puu?+`)(9APSRQP7D(6b))$qHJ2LR8_@*n)%yeUV+SIDTZpeY)ppI!;GUvxXoFA#eOA+>A{W zufHud-IO>GJ%U(N9DSif4hkKG`h0IoX+o{}2jPb^(H{*|1(U{vP0@G6n692>r?h>K zeGYgFKcd>`n{IN4G?LY+So#9JEo^6?k_+`grRs{go2Th z^jq_5qjUeH4h3I<{#4|9egcvb!c#uF8p(mxOG`Py&LBp$$4qQtl7t6C3=~)*bTP(i z2<#j}KWr+ zK<0W3s2Rq(f>k_IDH!e0OL^K%Br4}4M1YKp^l1pL!S#^ZUB?gDF=rHMCihv<-g%Kh zvW1tWQg{}7G+ZIA`QXWk00YM=qNsdA*h_N;qRCayR%wPy67~6sfRu)DL1Xs(T_Nv6 zqUflAGb&U1itUhyAGYOb{cO-(DmS=}usUAl$S`BvbjssGfYFMmhz=AzP&Ee_Ffb`9 z*dI*JOQ#$PV+!RPstUpq!BA2nm~5DnofcG#mt0iVNmzmfISO(la|(p=3bg^yLRSJT z%Jyo5REvc}Jg8g!ir0Y=_&3P?!Qe$y7&sG$vPBd)w6IW!HUp#BA+S~uKVVbg3mW)g zdFW0bJj*%Q-6AwSsWhn3gxYt6BUAd(F}3{(RH)RielEooEI8+Vt0v46&1O)Z%Ari zV;6zN$G2U@=$qKjI7rfyru~xc+D2vaOHDGZy=CL1nhff{J9Arve=L?C~ zucLIPxQ`*-kkDXFWiKddSSxn7WK1lo9>rCmG^wQ4-cKz*mPA;W#6XV%BjP8j(2~`sjN;V^?Ph^7n`N;m&&5@`}#Hp#8~8C|A}? z#XmkG)og%Y zQVvw-C)?IU=D{^^3typW$L_%M2%H%;w0;KE`9Go$+`ZS0-F5vr!#MR?%yEQ67K3Yl zWwkeU$F=+3Cb6l;(&*o;8&oGZeUR%HQEkL7IdGYaZ&*F}WxjM?>totHlejU!6?jRT z%sz~tw|2BCYRB3EwFSPZBc}GzYKQ%yk-PS#5A!M0@#Hn~wN1JC6q@@L;*a+A`}mH} z3g0=D5+~IU287p*N?0Lo)XE~Ynr#Qf&+l8 zLqIRo-}OHKMbZ3c3FxoPN>=&5%mPd>P$4Rvh{zZ^TUT`=NCuNM{exLy4RAv2a3KG| ztm_TGY4kTmH9dD59q3BSht99ER3AamfL<^r>O_vWuBVqtHovd8BdKq>+y3|;Vb2cm zsqbOCb>G!~y_tI~h^@-8Xe;ebCH1-#y=iy)v_n5E0EkZav zkdehIsiK(!7n~WAMlVf1v@KVSDlC%GbYO{LbZs(`Z!%IYRDEaXfNMSdqpi5hTwF+9 zWDdzwC4*Ecqi+G3B+DU6GT3t6qQ+QVgvUl}a9KWT55#)}093d{9HkM<5r_35=3xNV z(hgc8IYO#=mTrPg6yk8U_8?;L8G7T@4)5sM(d4IVMhLHwg`O_ChuyrCZAD^8({RSr zcPRY9-5y%1O54V8QlO0r`W(KahHUnm7Ws2)iyx>Ss1HVjyx~}>o3$)%E3Y}Lxdmy_ zNT`V`2?*db(DZAly|~Hu+J8Xm_QMp?Lpyfs$=Q%yb|&OxOJB(b=WRIek*=Xe!sf_7 z(+au!0oTGH`Ye&#YDv=Igunx9|8XobK!s#Z{r>Z?Y7(^PE)*k{Omr7>r%hC09f`RV zd1YPMkv%cCWVn75w12R_j+s+2B=O+=V`#6>e#d+`f^c=bT0^e&0ovtC2KFcfvzi&s zFAclsB*}Fz#yTiwd20rXLzlFj`u-_ z3XR-C0)yn-OWQW{Oh6&;GDWhk-NU2$?W8bcjuhGLXsvLS7*TSQqwXR>?}@N1_T(g? zsd>?Aa#!R)1o%@KCpJ{VtF{UbZ}rFzW-Q*3TxW1WJ6q+%ssihc1Mg9L6Ynz1s?w&q zLy#H%iWL_uXqja*_Ly4T8fs;^LWyT(WCbd2nQ_;cE9-hfSzIhRLC0kv1irDc+$PrXWe2KYF0PB?AH0#V~gmahPDd$&a zTTE{k`UU0`f`$A1Tyr?G5VpHpkJO{TW~ofa9Tnpnicar~*)eBuBFXvZ3nC*;@9;xS z34D-GOwI~T2ix|=YbbLGejr50jLf}lkD3=0_i*~kPqt@!Y#%N5H!qUsYs`S&pjd>>_hb7vKi}OuO1SPhIMDn`l9ZLbR&S5;qcR_8)b6HC4y7JXk(3Mk20eEL+7x%xpU4v8l{ncm9u z^wjQTcA3rfcID^y0kW<0zVDBZ-{t^}?@+cRq?ilwag2Ih3jy*x5PgTS|C5kS5CaY2 zaE=(~nQ}>ITT!^D6ru{lX`Lo@tlIHl*n-h+sC5(yC^3s`Fdm)5E#oX7*bwiTeh7j| zn{6!AsiSp3;SQMA%N{J(!yY9U=2?(GD~}SJO_{U|3%2T(#Y7{%A&c2&>Yjo5(W=c9 zj{g3|D!BlWBYsSBtlYo?N`w*S#e1@*a3ss{I3p*4Wlqth56l!ZD^FX|#GaIqBhfQ7 zS1y8bAw%9yvtF{8KQZ*(zYV+uCE9BjYD>D*Lu%}#_$L^+o3hkNgbICF+cmW17n;oM zd|{y>WqHo5GeDAPiqx-0KmXT*vLIA0b_>YxTlb<8g^gBzuF{>CM_3Z@kx};zXdWRktd-SP{iud{2JP09onHsn|`!4TR zMXqCuR=o)B>E)`4u1vUs9N85dP3hJA!fZ4ma`ITYt{lhUB&5Rd41PHs3aQ8L)FR zvJ($u$S&y^kdsx-8y++bH6Cvrpt&`tF5MkPXnWgg z$I1;JQrdS2qSe)|hgp)&x*mj2+t%Ma%%`4YXfjWTmf@K0c@mB}#SMWvVl`phjk!S2 z(l&S2FGY7C9UFI#;X{u+z##W_Bwvh62hqs{3)NoRb8`zoE$2j=q0B)S!MwU$Ix4rz zKuxDbYq^p!E3n%L!M=U0P23Itw0)~Iyv7o!cy1Ot&y*n0?XKu)iuR)C;Nf9Hjhq?~ zWxcd(Tvo!Py?Ziuj9 z7WeDHHV*BalwHD7kE_ZjbWg}2|0z>6Tm;M~WQtYFhoT)wtR8SjxEkn}p9iXjD7#Y3 zOK8osy4?t;89+IaO7`5iW}or@ z6|9Wegp_Xpph^PXL&^RMRu_AFJ4ZmR%F#>}a1`9wz}e36KSzpHVtOS3_f0~+PROEV zlx%+BiFA^-{>VWU09-dY7dXz#S!5odu|R5C3$WW0>2SXRtnd6j9ZF?8oA0cDXlBZ5 z`nm9G2DOdDgO0XJ%k7MF&c#l^OlLy-fa(vn6Q}qKBDybD|vh_(tf{JbWUI?FjlC~LRkbo%oYSM#HJZtus5dH%(pU3xBhD=f%*Fz{X*0Nn8Z*awUeN;G zHBm5ilNl4q7 znf>=xo9y^0Ne~8vA#%<3jiyCQt*>nnyY`Z8g~wCq!2k%6UM<$4784msRxNMV@4P?1 zlaj3@LDLXT9I!EYr_-gsyna4|{)AzN+NStv1WIV;NkmDB5nzBKS8Ob6Uvvk&v;ZnF zEb$yPq2a$h9#-nz?^Nt83q@P8B25*I4(&uZKBgR3sXqnTRPYQXP!abYVS*}ljkSOW zhKRFekqndbUQsuh)TzWe*NQ?ao18$UvoG=fbK6v@B01v1^lMxC^g6@baEPG>Fs;ra z6RavM&onv)mt7^N&lxiqS!r{LS_`*E=1n4r5ua|?y1qi8sgr&&OX){>FE}{35IC(X zIJ_&ktSI(pm0Xy-tf6k3uf2qV zscu^%Sup}}Ad7=%L4M8YBeAL*bHXn_MO8~jDA%KB1T%u#AxhGtqtnzPv}9v4@>BFw zG?abx*xQvKbJRWoq3wXijlws3MbvHfqC^NAC%0%kx6)~jsWSXI?p_4=;3 zsB~R*7-pSFI{Lcjc@5aS8j4%}ZjRI)3)!I{J1-yjMs^Wm2q*eMg+D zR69(sPs@mH)|H?HIGJDWffSC*c<|6+)bJ@BYI9eFkI?DnE{{Itf!Es}f{qg{?DOh!^RfV~0 zID6S&l#8Pa))v79v}AhtC83*O9% z1Vsy}7D-k-QxPLW>2YRHACL_>tGskCoeY>bd=Px zjb?B=@ZN`)aQ&ekMnPzT$?^U>BVE8$)5iPJiF+%_x|ztOX4$;i`DyWwNXZQ;>(*PG zhh(QUhn6!AEgKT0KAWqw4G2-tzMKGQ|e!HeJ%K38(tR zh6Xu?dWCTv2cx--mgv|yBnA~P^JBE}k_B@j<{E1kB`YUM(#;&Z8qGg=sW5xYz@h!jF06dx!_3+z-`EdE-6YdyPFIh zr5b-3Afw-mYW5(x6zPSP>z3)9cmhle!h4DWSl8llE`V5GbKjaJrXI(tbA8dJ+p5T)9tOa3-odi3++;2 zlc`Z))TMvAAzGZ8+exDyqrql_b&+G>CLQp#r~0dk!PQL-?Rc+K(2pQ58PS%P-?5yo z(Egt?<$1fz|wn#KcMeu}pID$mbh@-*=M8U@asb{->t)Jpg4p@BC77Ek80r_R< zu?{DQz;_TCSR6wKVhAttN-F5`2ZlFh)>fAD{YX1?#qgf&!tNSMq1-=^Wi2ho`wjY# zGdRx;g0D$W=dEHH1k1lR-+_vrXuA5JFpBm`I;3PyNZ;AuGIG0fdh}F?yO2A4gJ5Xo zKWGtj5a3$)9eees!3fb|!)fDm5E8W!Tzl;c%}Rs^wxHfhtvgRkJRdB7md{rRl*{~9 z@d3gs4 zzvMTkoSxC2L7syhuPjNW9#_edC)3lt+`8U2xSX!WJMOp4egm=p^eb;c_e3s^Q*`(~ z3%$FJJ;5Fc8(Zv=bO3dP5>;o<$HuWZPQl(Qm;k+EI7KAvVxwulkqD4Z5DCRaeEI?(@>vz)65} zX{OT2jKHFrdz4PaMvFD)){<8gDbQeZbzD(*lm+@`6W+DNrW(bl6@Peg=0bN($&A!_ z#vMvod}VVNxEwus=NxcbCOx%ho9bT14+&xh9BLSG=52V$^&!4Xy`Ufu-!>H;DWN)s3?G)M`jEVuVp_EK0(#n|b4pj==e{2db zKxGQMX!bPK7qb+sfv6TU8|`1@|NN-Zy{yuqkKi#{ZdAc%jS%4xf^YV)izopL{y}n& z*`KGwm3I+^Zcgr>(3I20m%_AsG>ex;? zwrwXJ+q(JgcTWBHd+$Ae)v8^q)~>zQ)?9PU@r-8-n(y=*NIW0(J7Cu;GCrlTqq&gc zU9acv5M_?4T_!Y^sNIMVdEFhyQt;xCQk;9`WnF}mH2d}k*NIj0;MbU`*W|&F#Tq))!$|Nw{PGBY7$S7 zUD;Bb8nQ%aE0ERy1H*Ftm7&5T5EIvE0QQnojrNCDsvn6%an^w?a_<=rEydayo=}v z`j@m|%L{O89hPReV~vtS(N+19VPFbR#!@8Ye&=NJxpOr1K3<)QCmkcIe(@#F8ZKTb zW%fE4-E=au>>pX`zwL8XY4#}u#mDvVSSM?OU0#6=ihtDlV6Yz>(~m&3eu+62TjDyb z|JEv@I3X+dUT={cWCCbFDJc5auC3Q=|x#x~3$;fAe$yGeLq%vAharAqml^fM&alQSaM24$j_aR{YMAwD>89nHa8*J0C z9E}ko2?&I(9GGD@coIilBT0=Q&iW12^P7L1AxV+@METw;9e(|2vGc{}wu95;&M5=ugFNKTi+mlWZ7G!6N z*~@ECTo2y8#>f}&@qr}^8#OMXCgX@;b$@+{?8eerq`}+cV%k5Sb0?j0+SA|`C zI8XecAN`8qKQ<*TJyT{f!y+zB?nw)~1^g1k+}t9CB)p*}74;=(_GL#s8cF`{#~Z5{ zOC+%I**8W3oXjJnk|0L^n(VE`^#3g$Gja9&Y(F3VcIkd_d-F_1pa$4;cvfvMll~mk zrrVt|S2#aUah>kPPoN7Wailt5NgmjOCaRk4p~CPpBc*x&hk3|_9nfaXH($N>UGk^? z5BU0jtV|_s?EeL{qt&dwWtteiaBb(N4UGs8&eGO_Xc^EJDvc>YM!`bZhXr7Qf6!8E zgNjFOH>}fQw(s@t!Bu+Z(jDh#xh~lK0(%B;HqYruj9nzij(Jm$d9Hle{XXu_@PF_L z=>7`B3GMs^cOZl@(8{`T$Y?k!U$vXV>OBNdBH$d{?ItY+o7fS=fLGkPZu>=d;O=cW zs)^^6lGOr-&fz_}Pi^%f@rNj-a@PvzyMA;H8>@7)HV;ZK$7U5a^+bY78 z?u6M=Gd-0Ws{e3R*F%14Nwz5~JF&teWTrrC&bYyL=Hy~&-oZ;VrAetdFI`HK@m5wl zvLxTK_#HI7ZxK#IyT|; zBWE6l%dLE`ww=Q(9 z2Doo@-p{ZdYI$gHPL9-W$#tZNEE7P^FC5$TLi|Y)cg4mUuT_CKPF!5F99_WlQCX%U ztGbL)WYIqp?LG)6#mp)^OuNVaouVV6o4lzEqf9%ts@0{n2+Mp1>R2bJS*9{xGzYD2 z4#f0~q$bzC?wR4go;dDe;@>x$N)$)!=PRvv`G zq&GO; zw|a2yS9fT`wcNA|ElPT|)LPENO$*bpPtmtNWjqmhRsauG5&W%6<`a?q z#OI9QFa$2is!4v~r+FUO3o)TM?t!Y1BME(*1m#M=+eoi2vC1#wBF&xy{z15JJm^|$ z;kZ`+z-9Pe2hfqq-KmGCZiLYokLq&HE(*U{uRbS>jdCAyR?^F_V<{`I2E?y`f0!}+ z4fCTqhgt-mJ4l4;k(ACSs)AE*k2w@$BsdMl8%PmKvKKJmNO&wI7xtP=zwE+z^XC* z+r%dOXL#e_MclbEOwJA^%dN6-6=4-m)=W<+A`Ux=&nMXbTr$vPF;u~RGb*G1Kdg$p zk);X1S>4gl-u~YVl#PO{e4qeIR1p_+ArRmpkiCUT>jhJVFKP5 z-5L?BH@E3=ilk+jB_E_xp%gGkPk124R~|{^HiNY5>Rwokh0OEd1uk-r%}fW9K+}P= zj?fVEjgM$Hb}!Og@P=yV*{5|KO{L*zxWo#2XqjqfiMHn@cj(S`+u0sS%x@QntzkWjo*{SKcg(1{>8s57+U;`cXv_J zwnI`x_=M{icR`fuRjb=-8W)Ou2v|eHV_{@mFt67wUlAU=Q=I!|EDfb84HA58hjBpsic&yG3E5nf)&Q3g$9PXBY zl^s{!U+XwM{jnWD8vq5Zj;qO7nD7f9fWbkgSVxZEr|ov-+PAqQZp9Xxb*UMXM9o+b zs8$C^;R!^KPv2BOBJB~1F^tLRj}A-boVG0naB$aG^q=aeSqm=Sn9~sC7NK4RYff!+8q(iV5`s%4BC&|ET)->~+)=1RD^49Q zujBxP-uHmAxMnC(;hHU9dhiBn?h>xROVsYKhL*!{4BKPGi36-#-UR>u9QFR;TgO`8 zBOdSje@gowz>JcKk)xrl@i+PW55B-gab2!Y0D*VBidM}(pe&5SlIY%l$SfTGR)ocp z<<&B!zI<>Fsr_144@u^KyU*?`Fo^V%aWiw9O`Yp>ck}ZB;p{WO!iI!!+(JQFR2G&Q z5l72fms7|-GzTyV(|KdqzlC$}S&w{FpK-$siN?mZ9Xf2+*&fDulf#skoORqL5$aWJ zGm~fuswXaB?*VdsNz~wGT^KVZevB}JKNX>7xofhNS%o}AB|oX zowd!O<_F;7q8gu=8L^<8pRNce=WnDnU`cCmJv#h}(^YX=Jb~0^GDfRT8Sak@C`(3r z0?DPiq4Usx-<@pFV|4FxI{>hRumz&UaP8tyEv;RUMi)v+&zMuf6kALi4~~`~#_1HQ z$FLA*vuyY3G4NtBjZE!Yg93-E59AO_laA4^iV*ctdkm0EWMzsu1|&%?D@Zt#D{LcV znhgE}aXJU4b-;c1>IK65XTn?A#M$}#7g91YH2(LkIarUvKbjfmDggswwv}-@SYO-A7p{4o9Eem{Od&i z0s2!f@Z!(ee8Cbsmx5pNo*?Id2*zbf)dc>Fy3SeVx*=<=1ldLk-Du(8F(_ z^pS8r0zl}wZW2J~BUeHV!Sx(=RDp6tL}vR>%#%)DCAhW<#~6Y7G=Y-CC9>c?!zrXO zBQi+Lr#(D$9tRH(%IK}lr5qcx4Ln6+1~q6VU;-)He?@C*tSRhu^U2CGPAa6y%~0_c z$?^41g-2uo58Ap)61 z%a($zbSz+mS4Rw`SGo-CgK7U(8}`?rKthTY25PCe;lHS*&{1z1EhCv41-sOW9<7_w zklAHSb~*(+ry*>zXz~Ky(D{?Gdh4-P!njuNAjVgtB=4%q&Q>)x*qhwQYL-_bp)DxB z$>cF^Sx7hh2jps*MHO0!=s;0wF=QwK&Sn8t6+lY#!v9iNdGHc1gXl5jGAPc}(p}fs zl(o2u7@g@tx=Ybc(pV_mXXWk29>a0n9u&wy@&isM`TD0&(L_>lSvNo`%b}W`v@>`g zX=~_LklPH|u5Kr}!MxOdY)P_vlm*h(Fe~ITV_d&RJ@BZ?8UaNjG7?uCF8+=%Bz}J6 zZ5+r?&%0=vT|o5>!}2pEpUVz2WUiBN_n*`SFzQ9=G$DyTSQ+l?<-@Mgtz3>=hWSt-MYhHc?{N2p}7$VAgQV@@!W@R*F}tg%o9fHYYSr%~DA;mhO~i z8#13dp&_+mQpG_AKi1QYNNQ-@u$l%+y*8Hu31X!&1B&xJ`k{`@=qbr zj0*3`)kF4kRiEGVRT6VouB8=WnA7CPG@uI8FHaXgS3n&ZpGp4e$LHEYhR7Z1nToJ& z6BEYGld#gQ*DET+PPf+nr~lkVK$a$Msk(PT+q&#H2iQ~q|0zJ#8s_+E|OJo zfykGItz?le!Zb|*gzeFqXfdsezDg`ug2s*%1Y2J@t6sm{=$pH-2SWv%u4G4(Z(^_D9I9HngllwdE>&K-5f`*~kr1!c=x|W_9pJ z3!XLm!|ca+Ldm(K&uWD9X7!uhCf6U55!4`+=H!ih>{mzdzOx4Qmop02F!>&~f@ zpNjfZ&IE(M49VP);T7lc+v7u4!`&0V-4<}#kRD24EA}8lJ>shtpl)hU8Vx&p#n>ly z&<71Zt!HqLTF-ZKk1$fa^9eT6j`$gl^Yb+)c+7}9${XD^ug)Duj4+BE*z-Y0vnk?j zt=k@rc3b$rkx8bal;Y^fKKs# z6G$WHw6CzFLN6PIx5k$s?pFcHtTI~}6|XKmnA?au{06kL*F*_tIA50G@l>x?WkhM? zHTM@-DvoimfWi=i80p(lJ%~gJNz6g)2Z_!ACuAxnwi)3*RHOx@8D{H%7vP0R`_Jc?S$?5Wg)Zh1l1y#)iA2Y-`>w)EaR{Q}cycmv5Px+9y#T+7 zbVm-T)es5#ks5?J5LV|AA=rWqMPY@UGC~p=zCe)RWjXYzGK^np$a-ddW zp0@nfaJ)=|`5ioRYr8N6Q3Mv@oPk6QB(W}ohOkJH$We^a5@F#xglmn$Zvm$eiQTKe z;q-?mB-+{kk_^Q+PhPS9cAKdQWlf*MEDEKyhKZoF~oxMh@wLCTs zr@L@GWouWwho1V_UCT~z5_(F`gOXn2|9zhHKla%zgQkD(b3pG7miOLSfd>CvK)-a} z71eKD59aSrfTwA{KJG3T-B?ID0b=bZT=oo}YKMbYJjRTH&U&qZh@2+GyBEJX4j1=m zfp?GI4WRP|#{};dAD*B$Oy1R>8Qf~m74)Y33Llu}5O)jWp6I|iroZnQ0vB94-rWAt zJ@jj&qqdnd zB^zPBV!pch&aSN(&-MFw1^Z)CN~* z8*01R%-d3>HONins@BMX_g_kudOEzkzq}Y|*tq@tu>G3g`1)+D&NFY+nRXC94tsnp z0fdGC*lFgkQEAp_$Dv|fDoBM4(_>||p9X!+*Ye5VYiH|WxAJznk2_QtoF|X`@n7ja z7^NQM$>+tZJ*>3~D23Kt%i;1&M6F!d)`z}va^MM9yVM&rEHWR-!S$tX?rx@u<~+Wt;P=@L)?}DxKl)V zCII2Z14^dlwKNTtwUzaibt;-mnwB)!G@0&Y2DcYOs8PB1Q*X{g+Qqlnp^n{No}NsV zUuJ*AYMdMt{gwx|r?j@KX4l&Ua-;fep4#`etF?vM{)OmoJGa|cDE zvC_Pq+j zXo36}&1K>aFtK;Gu(Nfd7q+%Av2~WTHMJA{_rd=j$&+d8|5Y6M(-TaRsX&pMu+%t{ zqDBL1WZjj7CRH3~6eo=hwb&d1O><%-v1jVPq}_!6hR(4V($-h_xW^Y_ro3u@09?K9Xk(<1)rs#iq|VQ z0ul4{Gazdq>J9w5_~CTFTuCPp<5_=vq%JNm=ZHGMVQB zNq~$hQ~ALTgW>_^TtWsTji8%+67zL~>%)}v4wft}a^c)kY$QvPC10;HX0)UA!UYfI zT4%nI)Xi3Mrl+N{rc7C9D<2Ie`o+kP>4dZ9h^2`+&n^M#^Y1QwLy3FK;Y3mw zmB__=duOc%mKO^S7PZgIm8g6rlsB5fj0=KZuycTT;t1C#M~@)~vsredVU&y82#mW* z>DN&@3MAaD!CUL*WmI2%L(+o4dYTQr%g&elJxdYrBiBKi<{Vu=74eq82)1hGbQ<8- zPnPD@b>$Z9Y9?}%SXY2^mrJ?VG~KJ4?E7!k5y=G?X9v;cYRYyV)iJv&J5IzZ<{_E^`ftlnYl9Z+R7g!!qv4e$L8@7Rv!vW zrZF**bbYXDUc4^^`Jzmz*<@+)UDw1V3~p2?oz((2&W^e~UrM>R`iiYoD7v!n@zs2b ztNrYQ$qu(^9sO=;mg>Hz)1|SJFSR9RNs50wQ{Cjgs?$Ri`{Sg$)iMn~F0ZL{`5+jS zed0+ZgQ&kstQ8iYtxxc|4Dqoa$n%jAHCN$=64kBjIVxhhyj6OTm+Dga=2rwg35SVc zzqycdq8LkCLWK2p5`i(0s55Wzm3#N_C7Vb$0tT4 z9g3%ZHre~d2yfg{;%U{LaROX7mzHi0B@;QL2_H$$=(-PfTwG3bw`oCqKK-Si!7=vL zQ6+J=jI**P`4}iNVDvxbgy3{%U}%3Rh;LC<&HOx24OhQ7$m+})d2n*a{blRuivSmY z2j^Id%(Fu@ytE@vfO}d~u^!NF@E5(z5BQ!Q+J(rGBNT@)EEMO-Bwns3k_o5_QdPT@}%Shc|)Kwmoocgiwp;9Ge$Len%ZOsmX{Lr~7~8PtoeFT0Vc?y5p( zTz^bp>iZA4^FP2jwAY{@^^iRN2J(8j=pcL3*qHW&rf`zn;3d+gHqgZPNc2Gvv(6r?2st9>DD=7uod`mvxq4!MlW`W} zBZZ}FYlu|{plXP2)?!9RF*t>c8?=WrT(Q#qah#G6SUN7&VPvZ!WiK#`rjA%gF?beG z><`DLaXJKzL)wnMh>OVWfCyeud-_?D}5Hjkvo_bgEGmq#{sky*b@~6f}pjNl@R`NHD=uoP; zcxlKwNlc>Gw2L*LRues`)%~_IF1%q=T6Xs}xm)=2>Vlh&9ei3HJ~_`V(W%{fnV~U` z7Uxl;v3X%qSW3{O(R%q<|4D;rin)L@FI7$A=nU=`@`~@q(+qt54?;P@j1OJ+8^j@e zE69X~ zhXEEFHz6dn?hJG!g*3%D1vNzlg%!ma1!WS;7;zG8(lC-Si93;iz9##vmZ3QN*o%?_ zcOsGiK}*N7^k98C>LhNJeWjP5-{-^N)A2qzkT>*;(UuMp0m`H?d=s)Pt@)sj%Q(yJ z;K#v1%#S_KlPbDD1ZN&(fkAz~BAZ?w21#31c2MoA4he%jrZsqOqifun4zf<{EeG|W zEkB%wb+}MW&f)N3>hV-QO!+M9iGNNPs~46DLln zXv!Woeb6d1KaiLiFRf7e*0U8+&hd&CzzKPEN$Iv)#72GX z84Qkelx{z7D1PT84@%~LBB{N#v)R+kZp*cX%=t+GJq?Ty#F*re1eru31uQCmsz-8&zm$V9<+`X~X-5I0t7dv06AKHs)@Zp)SGl|vlf z`X%>E@h{;QGcW`LsLzaiHON6Ccy%Af3pWH|!SYD9;I$xl+fSoy0Sq~Jp3Ylrx650N6kgfnQr#WM1<5{Xn6X4PJ@Qmw2v zStcx&smh{FA8_^lCQGyw)W=1yvNudf&eeItkxM!4%cxYn*VesPSr*-=pcCWVJLEvJ zbEZY)L^dy&WyNuFjxIjqem|n}hz+M~#(aQk}P2WG3RJ@zKJ$G#lednP@dH+X+QWbjH zc@qA|k56M)3<*g%%I()oQ zGHKC4J1@C?<#uPDDW2Iqw>){dX`Jcfa%G!f11+s3Bx#{+t}K8eqKG2;?=5)uLkvY! z;r}}zs(3xo_D!F$y~e-S^=!N4m>uP$d+~cbFLGa@3bAl!14L1Ec;L3cuBtV1S;mT8v#!Ku=ID=S_A5&0fR2ODG+AbQvPP9z8b#i4L@f7ir z;ZZK=XHZ8}7q%g1AMemsWG(P6AK*${Bm5c_fG=P!^8C9_U)Ws8eP*SGn1$H8e4z@v#Ns0%98TjXQ{o{O*t;m7v)O-uy!V7#dZ$pxiG zeP9a#?_LdcN7cga;tzcy=R@zJ0=|OdM4nsfYKV#u^oY+Of>^i=Dg)yL&64U=AvB%E zu(^L4a&s*X5edO%*A2x8U(aINS%VVBPJsf*iBM)Zdo{vmqNhjzs6y~utV42Qe?`qA z>XZ>9{>m_?D0zZ&F-H~iY=uwJ04P1&@|lM<##WGL)6Hm#PbB}+Fr+v(1WgeEC<*1o z%);xEf_?tde80+$oy%oNsry2(v{yr_UU^2-kj6xfpDTJui7PuouAweT*lSkPgobp= z;%_6C^eWS@DJ=k{=wp7vmd04*{Hz6YN`fIjS7uZ(49Bl2G{8w6e8E1}y55C|piDUF zMvRg?UE(E0F?6}o(qtPYzKD*#kZn*2?^{`|}8<84eH2!iwKi&L?xoPdGR{ z+bG!ki-*wHkk@qq8C*6A7G;TJ>6`B_)IV()bu}wBI{Dq3N?*$Cw%DBT*vyfX|7`ZX zX@ltam)0-&qCu3kH+1#crlidCDpBN!l{B zk(P-rPddCBuo2g?NFroxUqt&6d7_-xthdL-PiYJGL&Ybh-lsmg5}uSBZ@9HH+2{@=EkO*BF$nLDx}TgMFaIermRa3t(`gtIO&+NpTnw2P$$*SC81)u%tC=8OqxtHvLF+D)qJUvWG8hk z9kbPd_-wgd`e3vy0KV#wLEfM);bjxyap|SXWQVWi?t}U(BSO z1zYjBhuv9oj_Td7wYSyB^5O{x36n&;CkS>sMFm}%<4wK96;iDb>}bz!e($&)%*Kvx zv+0QMhQDRx4PakRa4~ZMi#_r=TmrJ>>Bwd5N2UI;zK}6J$Wb9G+~!J5Jq$Kjy0jcn zUZ%BCb9KtZ!$sIg9*nAfX)n$(Uo8VNbvhZ3@`Owo({ukE-E{C(>+&vy*!e2c%PoL> zWe=~%y%F89f_r}E+va3CH$fU@;=@FFp>h&Sa56c)Zk|qYW=H2pYn##Y2$%y{)yw7a zdi!SKzz9CC>o9(n3|mIH$005IxQvT}alhEO8B=jmCSA&3q-fGE`@s41F_jz>JHrgN z`RH6~ zCHRcwE%enObXG(uuk1Nn#S-yJQO@C?228oP1I`fJ0XGwM*tng1x*5)bhYb@>OY5bw zZrbAAcPkf7{kPhkNnx`xhry~*axdzd(I_bf1@&gQD=gRXIK9cnu)<8TRJZ*Zqq87Od{Y%WbC&?U$3+e;4A8r?yzUQ;~Xzju*1OwSpA) zn_V8$E1l+SEW4@c?|*jpc13^AZwy;gZ~5%VVgI*gCHgq!|22w(C{<3GT5Gp7e*ONiU-bP z+kbk5f7!j1t**kh__E*6`EY$zY${nEGOX(CB?f2v{E}YXK=Wr$^JB}xb})h~*1Pyx zX9YgyzJbu$aTeLJ<^Uh~Thb&eJE;O}7ybt&%~D#6h=Dd}2|@P!;qjq?QZ#PDlgW{{_vx5e$ZQQdU+LQN!Hd3N2Vbm>w=Km|PbL zc##IypG+4Gcv2a=uN`6`IY0e%Ayi=87Q#P+7T!Ols{n>}!P*3(kvkfwa<42CSYvVq zM}S&55=?LUHv&b#BB8=U)%RujtT5ET3aelV#%k$)Vf4TL*n`2wkU`HVr3K9}13<7$ zm;BIboi>HWSx-SIn_gmKQI5t2UTjRw54J2IRFI|lMPM15-!Osz@Biu?B(UooB7m6y z;h(e7jPN(!IanZ;L^H$`iwhMPHXF{rZ!}2Y&R96lqd8Pyngt!0SD7C)f&d>_Q~|=D z=r5fxwCD^nu;>i-_h+K%_q1##2DNNv28D7~hd`MMCsg1d4;I)>63Vle`##l4!;JN# zr`z`%%t0`RmxJ^-I)nEo<7EZSaJGYBIk6y|;Q=vW!0w?3%{X?3Rz&Xgh4uge7s@jY z{7I&!7pl@>_SM_FPq#2}>GRj?P*B4v-=?Rx_gKh3D-%;i6DkYxotNpTSvH*$< znovsHEG<$2V$MRXR6E4j!Ynizit;VjoY3&lZSf!?0 zaiLbI22kUzaHzml0TW(qQTDC0WRdlIEs<~-w zTDYVxB3I=YHyJ+|e<#S4^P%1vuhWrDJ50_R{> z%DhSl=Xj^CoI0mzE+@TN8L$((Q*2hbXL{rW_W?~)q6hk2xMMR{c-w$?eOBRXnj7;~ zywf7(#K%PGn1?0(oy7!*WB1yyH`kqSn*go>o-rQ<`fuj6X=6ST^z!}*;R*2rBNw#0 z)rsQ~(o@)pW2PQOI#oL(zStMQcBk1&Pva^Ntbb2G-&`R|ynMQ(Raqx zO1>#?)H`zH$XAlP#fjX6iiOhj8pF>hohrU@Z`!+KFLv>x)p)3=FO<}ZPv z1^Dz_gU|T;h1m35eBY^emOFam(pST z{^57-)nec2cb+?ZW2KMy7qBD#(WF=AyT}Q@c#8-1qtvu7GUn5(33}!)VbewabbZ6m z)Q#%T$#<$dD<*Z2K9nD60a*4$`XGg^MN|}+w0EowA=1ACk&=S3cCb;0z)1H^++amo zjfE7L6dxDz|{6tAz7}?`hVUvoy@vW1J4EJ?tL`!_%!=NR58?>Y?c@bSO zD$!3;VF~?KcqA(95+&4W(U9h+%HfW>)21YfG@6s7=ZQj(iqL9F%FP(zhA$LE8z-P; zN+zMvNk*eFCX8x1$H~kZVUMcNDv``LjwHw|_mQXPi?WZ(&_<9nG_Hs;X-6WB;?TM# zPAnGW8t0&u#!oEoIVXT=!6eWu?8zjEYQe6KfU@K9E>7wfY-pqp`&Xo*ql3+Kr#u_K0!N*xI?FlB(YVnBF zt44}T=+NSkS}gA|CX}JYCJr>Z2Mo;bNhY+S$;UgJ->^uQqD{w1(G$&VnZ$34tWx#Uhcc z3Bmdz19~SF#0T$f8Q}i<2QkpFg-(6&?GHDI^Z&3U|G#TGqK=Msj{l_bG+*?wml40{ zmV!HvSTJBXR%8K$1RQc1uQ7rnD&IcA)}@59qeM#t>RR!esLjgcya{9{+4pR`8^qyW ztv2TvVf$>7S8QFGTW*YqQee8DAI@V;MLH4+>@Yh7R_vUmvL=J#df8uk)t-cS?w`o^QodOT;t ztT>lWJig(hshS_C{M`{Xj4$o@+ke=*o>S1L-)?NYha<0`uAiPBgT(s#`-S^~gTj8X z<~J6cBoyE$Bw%`=9ttBUY+*+lM{6jM~|6HdaL6sJSPXiKj7Nc7_yneS!E8m`bS_s-(Qk zxS^_SMgBdj#S*AZA@nqt4~xsHQR7tl7MbSDA22y6(Q zwY8=5-Z`0u^wx5f{et)l8rb}isg17l)z#-vb%vh536J#HjTFY=XF5tBk>?1dujreQ zau51i!`xNYl-atRglG9+cJHeGZ0%#JwHu$Oy4dDUo+gpT%T~IOTqBl{sR3FUym2wo zqXV1GEp@p?9wplz(&93K8eD}#PMMsAlqg0D`oUbq%*dmwz~G%lPUaw!)?fvI6+O7td2)EL(NCYHW}7%ikwxrB5Zpfp6D~| z4dY3a{*z>H`Z+SJXU|UWU3Y;0S{_JU)3?5=W%_TSy=^Ac-_8j(w!YDIuN!*AFE%wx z!u1)Q_cj0wqvh+ZZ$ofdEcKi4i3(L=95zTAKlKZ;Sh@to#w1HG6B_1_(MNvT-~#%FrY34vC|SA4t@x{4|V@{TOU{%u`; z|3ZX6L~hTamZ=PL@&k}3H8+r|c70gEIC3Yxk8;BA(E+p*jK1o|qI?n`gM?5AP3f{* z_fPj-yb<%2FxnNUJjm!)Ja&OUmJ?YOt*!&XS3q$aTJ=;UuLhrTFv4*^GI|e!Y zpy=N-DovWUz5X3<({g?e$0kvr?PvLnYaRYo6{yw$2G=Um30ETBh;t z<$9tS)4Xbli%FC4f*6(FUot2(!>dRQe!xPm@Q>SIiNNtFj0B+5Ruow*0z5nG+|Q#A zETeZLa6DHGOD#<9GMH{WA*aRnn+*z|=BPDbIuH~Y5toti%vy+AV#JrDm##zg=;)krjgi2_f-$CEm;m0mGgW8>ZOm#9i6rcTCY*3DNp0NR1%>sDZ0UT|k^%WKjXOGdzt|2d^uCA3$OnbU7aaJzU5S z;}@}~^F4k>8z1E6vu2A=!l>zsjN9C;T%Y%q!>20;CNX^0^oN|2@XK>amKy@Ju5haB zr@iiqi9MmWPV@8*h36PTci`M=XKjLn<;8fgX=`X*)FdwS&h2xD)&kPZQxKDYj6_w3wp~qCpiq70AM`)KaIb+!LjV%h%# zH2qutBIIIW{oS-d!ua1hnB>I&ov%6k1^&~^a6|+_L^NqMvcRP*FTlJ7)n-L&E?M-T zV5-_bp^a``po@7H-k<>gj|j4zY82xT0&>^w*ub+()HL&q(Oxjg`jwyWg~!SK{N2NG zsO%3&3PhZNd2Wv}bmdh!%9%GmY!oFhGU>imI4DO(Zj0j^mG_X%a@I!zK zRZ=SH71iQz_2@k!w~m1{BN=8oW>XAx40TK<;!z34xika`1;IJj|JoAt zr>A3zbSISp&vIJ)Qy18Gkh|xulbLFi2v+@3r=u%OK^AB-~@LGF2OapJHg%EB@i@7@VhqYKBxQi`A7GCTZ6~7S5Y;qzOQC} ziVCflZ~neHQW9Y8kL=V_}K#k>ChUX@So)TOXE5ge%;RJC`mwD>orunK6neFGQ=7 zDy%R|OrsBy_lsvhpR3Rf*4SyytH-_`>=0OG}{o*_!2U*PI6rpvkR1O!5`2@y}`g^B5sXk?k#*w@~=8WRj}z zW5FcRbi3Mj*6EM(MJSh~D&_(Y6bUVbPo;{O2knKdQ5oH2OyOw$UP!Ax{zgdN zOv%0LSpG6dBusccl~=eM7{O9R^mYwBfBP^bWh>D^4xR+1jq$e+I~BGPUF7gdueD?U z^8w)j*&{AsVofjJPXR%U3NP=&e>|+m`VH}~#|B9e(3>@U+4#GMA^we6Unv-R{077I zzj;vDe~<8>m7N3~L*}BzSQxAYsrfqR9hAzY_A%n+MUbC6FArm7n6~Bf7k+$rpcbV?#+~HAxpGK9yt3iA8d1iLShGe1c*=(Z-1C zZLz1DhEhc6Rv##MS+#B8yhkqwgTuY~r6hh29lqZjzn>{c~d zOAr@_U*IU*%clnMOq;Hyvh}X+Mb2!DyIZMz;PhYNy6+TA?f$m$b}kb2^HHC}uL)zP zkxJB`XTG&t4UzndHTlMC;`HM;PI8Fi6|c*LX>(ogN`m~D2Bi*Ye`vgKIYWl_4hcWM zoyGa7!@jxYIR__3XoZo~cBVo&P-R0s?Orp-N$(+;m&&zr_Nps}!BnW;E-nwnx+z%K zZffNMTCiXb-|N$RtNlmPz{kV44ZZQw&h%r(BFHgV~Im^=I4St#kZDvba_h(i~!_5ip$Ly3cbcr8HyfiXfT0D6yN@ZAPYk0BXN;jt{FEiA{Pl&Rh zBYrip{w!m>roo4GB?>*Ge3O;@NgpG0fHEKM9Ou$iu%L)CKbsRg5VqG5Yv!ha^Mj*q zCjWR|FtJ>aQOWFS%FnnFwxZ7zg*7-&IjjYBbWd`fqxWOreXrD`t86E3_H?TngvXaR zNml(vNzBEd?{$wHqc<2(bq+C?a2XUJv=#Lb;Y`O20U_Kgmf3@!o8P9yk~a5^es@@^Za0ZbLrF5WmI*F zl5*>BJj*)THQk4&5y`dojwiShV^zCEM_3)+CUVA_#Vj$zyeeRDYp~W+d@=AmP3N?M zcXXT2xT=q+9YG5JN28jibYm}hpCu_DZ)Gy`)~9p*VSi;aiX)D={N%oB42g)y_&eB0 zu`7}DiOMUt=yWG~ljP(+sYcy*yWnQdc{m5!%|L^w_QC?&?yn#5#(Vaoqp21qRwvS! zCs0(lskC`-UjkP-5*KT?3q#gd4xcJj->vs^@w%qxFwO1vuRDc`d8Cv7k}%6J-HS&k zpCftg#2;Zs|1+LgsVOqHxp7%Z^_Ys`GMXJwnT@?$mn6L<#Z@Tb;&&*y=5e zLC_YN@kT|9XoM+#RN~iJgg2}-B)v*)nQz>&EHL_DP*!V^Dq{~nalwSQ4m%)p^VYaKW*B-XRhqJd_Z;07qj~pRFwdL(P~0G1F!TnaySF6crM<;xC{nk^I;yZv&P z9YvpNq!&??9q-5)Z=ckagZ4VtTY-?wS0?kx>gi?<`j^gxSS2p=U)0l$a`m0+MnY$J ztkg?WtTanCEwxK`ms-@*4Im9_JKlUd&|$B7Wy4+N!V{f-DBzp{sZj#CAOu?_iek9O z6Roo@a76-J#SUAANZh}AC{Q==YZ($T?8zBYUG+XBr?X)Ud${y-drQ8XHanZb3q7s= ziO!8JI0x-MyAn5S7ki=|*VrwMNL()6#8tfHCB+aze!3)yb#>j>F$Qj510r|qLx_i* z;Yo&2LP&#6Ht~SDK|_+GQsY3z4gA$Nj>)2mXwkq9=;` zh3-j!t3T#ugzP7d1+)>4R}-TQCijx`^k^B(nELo+i4tgXqrUS46EX`)F{xa~yYroP z0b5i*H{Vq2^gzMq^!4KLzcJ`ZUFtK6Hf_)l3klLPOg8-$5RQqQ8Sk63p~S4n>ZutN z8Z_3gK4DvegoB)!Fp;ytj!#eUmSRh^d#_AwySYKRvCaQuEi$9WEQpNg~DTUN1L1a6m_5D!ORH7>kKHNzQulserlf{ow zXZ02D%1K(`qDiP2pQa~shdignnxf3>cQQQJ-_ivRlQ<`T{&esh_2Rl3S$n#F#Pwm; zHO%Vh*C(A}t(z(|`U<7g58n4c%jfx;nZ$t>QIiO|lY&UJvw(Rh;0ZFa=}AyZpUI5=hiiVHW$tntFV$(iiW-c<1V_BMzpb zHl9WjZy1K7ffDC~ISjfl*0)HvLkfiKcB}howYNw+LU!Bwd5d@1qCbb2mPzri5U&ou zxe1@JHuW{}b6UAX(e%b7)l41N!8Ri~Wa5obPr}BYE%mK@TmKRNMh;sf;DnV&VrDh) zWwpnZu}RI13k~&K+;@6fz1OazmE4kzp%rMwg&!zl?p(9nuJ10xaB`pq)Nba`0D)Gz|hDKv@S zhLpX7h@Fu^<$I3 zzBCTzeNHqMBitR4 zF@-H|XKpJorN_&M#Qf|&!*kc{PiZxxIXBJLcp^@0~VP6e>|JNG@&>tTPR`ZVR`FX zBM!Y0(xo1x&$o|Zr=jt$5urio#jnP{+eWAuUTi{ZKAe~PuJ!(a#j7*l7&tZ-=KrKG zgz87~p!dkwOS^7Cy}!Bp$>GCg*^k#vhwm*7W2f~EE1!J$6Xy%RFQJX!K;oL^@tIP>@_3@piO?2fM!B_!J@;>?_x;dh1{W6jSx0P9od_P`QysQt{40!j;!i zHdr4nb903(71T734YB)6)97w^q!`FE#^Sm9cDHUZi8pzS@&!!@GE=6-Q7{G4n#>(k zhLa-%`d)Lefbk~NOaH{B+KU!zLOJEv@%iY;+I@gu-0Ej}?&X2#*2K_!7E)a6o}P*w zG<<__(oSQJStCbI@;yeoS@hkIex!AVYzkhEmxLx8c;KxaX6{8ZE-7g;P!+{?}KQ7_IXLe?2Gq-edO=OYsqj8HXIQLO`!B~Q{wS7!valK0{CU_ zV~z}vlmwM)q9(&k5x*Lntio$kPz!Pi&I+@OA>oG6K41$%M6eocI3FS3x|`zvg8lc+ zJ&ZI7A7tR;rvwa3p8syni0PSG8X3yl+W_TXpweKVYsSIG`nN@su4H91DUZj8i$Et2 zvo$kv^A}%WM&af}ks&iXRZ!pMb zYRxF}C|7zV4ogC@qFb=^<1!d(V9Wn1%WYDTs%{ni3r@=#VPS;7cU2?_n+#G$+=huZ? zS9Z=MC|xZUc`Qf4D%#K^_N7S=wpvpvE@kwhrAm6pOHGfCcb}2hj z`_3dg_3=ejd#64aRyFu8#4wArP1S6XVh4=%MDT~WDhN^M-aWE4^d_NSGq1c|d87^6vibUmc_tU?B zWhy`8Li7R2LkesJ3H}dXnUS8Olf99wjib_U&)mrH4;A$LTdPf!Q34i(f}R(IGG%Fe zRYe6Q`Dx)O)dk*aR%%X|LF3XV!{JjZ5xzm*pUm7`Ot((9+dw_#|L!U`$zFf48Y{ot z;=4+*xB9bf56AoI;RX@R_(L<{WaTvELuGiC-#L;Iwh^)su@NM>f2`<&$byg?y`Zcg zE(DUYO6!++7nN4cNotIGT3g5ol>Lu68U#B)s?}{L0NL7gayj$#>{#;C$=S8l< z$Hz6{_$tqPRZR|wS!qT&{|gZgaq`UKuh`df`vj%)_lMKn^M=Qd9!CBf_uyByO zahs+5D@_n=Q@kYuXu=FAl>Z+*{CA-1N%SoL3flM)>nQ=Oz_AAB(H*4iJ$N}A~4OQiB&8|6ZX8{ zI-FdFe|)@ogzPZj)f9y0Q2pwcs%>qzuxXz{`~ULI*S3h20!Gw|-HI@w;P>hv{>OW;v%PLst7% zf%>~edqh*obUBi-k%0-xRrBst1+ynvJ(?Qv3@Xd$Td5zb*I}3oLI29 z4wv0C3eH-UHLnytElo*_Y=y00u)w&3MrGI|@^~Nl^u5$P> z7p(J*s$F0&t4D0KUQ8uR`!~n*GOXw>jIi&w9MO|?uOnV~L3eRf&nL@^ic!%Tj}}VG zW^|X!R$?f_KrB$HDt}WC?XXbd$F|=tIJ4S;8wz=-NlR1O?YM_3t+ZCbnKS9oH%B5D zs*vML>Q=3qDhoR&Qb(JklvvcJ5uE@dG+LFQ&S4jYw2bctAN@eC(EZ{YY@yf(4P~M= zl~ny-i2u$RnbABoW&meU0-PcF-#J6V+R@0w-ptWW7-$%9wAVAUc2qU9vD5=K4*c~y zOw>^VzQbqo*j_RtA1|g9{~io22tCA6re3y>)rrWE43`~Hgd7Jm&GY2{h~BCvW^xby zNZH3Hv1Iw(T7qh|#jILx^w@1}h5h5><2^FiROJFZmHNKlTXhi)Aq_DNK@Cw2VU2eZ zxrXu?IhI+xabz@|4&b*##o1K7#50r@?q)oxIKW=1$Lyuzg0kdpq$00EYgjuELf*FyUUj}>VX_Z>euzs!Zg^5aXb*84sYA2JJ2k6Q$=5PlH?q>a~;1Y;0$B(pDuU`Xz3`F#0&1!H+*gocs>F-rx)&tA`rcgY;OYRIWAbfGC@if-oux{a_R|NnSm6okQxiWLq126iU8Zg zEnWWM6&X5ORz=XjD$BlY$A%^!yhwpAL;*IjWq=L?IvOOfXsQg}4J?{=fB}o9EWn~^ z!}Fr4S}JJKH1&DWREzdW{{^sU>Xf?kp4RZ@onCIKHyzSL9MLJt5Puq*Lhwr2hfYaE zoe6bHyK-NqV(`ovFg;6uD)Sa0bV&R-q|S(qBHcxsOMUooi++=MU-!_zni0^o|rF? z$+l3Iy(ZM@#7+rJttlc|m@<#^Z!q*W9(bol0(Rg|P0 z#Ei?T0VtpFJVUv9Fim2m?W(vA_1#l8{ai}7#aei?O0c)i(7pY!F3-=Q*1V%(8;|Qo z;=RQ`WMHGP6-7oPNCr0lQwF~MhYV~5$-pG9x&;2cpvv>TRVdZRBN^Tjmg~5qU^T)1 z?n5$hMqlO-=0FTliLBTEM6m*nJPx#a)AlG;o>Fb$iHYoRcUAUBg|jJbtIzinZQGY= zdhy7iQBAFz5J*M|!{3g@tXb6hw_O|pyHX(}`X5}N!i7n1NHGL_6!3fm&xu~PyM4fW z(?5n0@M-mRX1EV(Q$Gvd9e9^|*A^G-m%!7s}fwViOT`e@=E~4BL5x5vcTQPKy$DIsPG41W^153{O<@^3m{~?p!WC+ zu*BRc$t)Fx=t+Nokh!K*OH~~QL4>T%f|mbAnqZuCJ>f|V1Y<3ed-O+?3ZiSNMPn%! zDao^4nN+XyWABstrQNybvIm&DCOz~-_bk#x1yy-fg<1Jo1$KF6N{x7;L8W+^coXmb zyNyOLNbFwuo%0NRWo4bE5ff;%{-f3KausO}mh0{FkWIdan>(reUpG-;ZR9FSBiP_w zUp+U(-zH2MY4C{Kqy`le)5;#flNn!Jhk|vNo_UglV0nCMs&_EmC*5%|>a|bPKeJh= zrT}sBEQpiK8fw-F7}bXJx|zNm*nUW6lOIaun${Nu9p`!x$RC7nm6@b#Ul-6&^z}9A z8~q|miw!)-T>e;+EK>PnLi+qHv~d+nCNOuP}X3nNUl`_0MhKO!$-(C!WJqYj~k8 z95+(5!QMAbNj;-oAfsoloO#Sq5X7<>^reusA{9XLt_yIc({!p{+8{Y?Uf7baE^Cso zb0}+e8+k!mckr8#Mu z`0c)H-DeqJtgOGclIJzt6g+peJUuOUjkT|}L3UtSI(G5WItnt-=Fn#Iiv3en59Dmi zXJnP;@+Ocmbh3co4i*DFPbEGkI$6+h|xin=}QJDG!k z!eP~^PK5f@^D|I*M`R!#%ZtC->~l&RKmXTPJ`Kj6fE{DVH?|ieUQMcSU56sQ6?fkD zWf7Y~{a)+pKq9KNI?iZ})x|-I1Z2m!4ClSF`*wWVNp>g*XzJWIJo=~F?p(U_EN_iv zm8T}J;-znX?e{a5h2Of#@m2LpC$MP9EUeXFNu}?F51V_Cr_ry~Xay{77<(je#Wi_- zS0N@vc>-x_*rYZ*mm)w@$KULvwW{tWc}Qw~UYi|tpc$>TKi}HBmzD3%I1S>K<-J01 zaA7|wgV-~q*ZpN>^3=<|x>t8`&?VYP&Udw-_?U*-HJIsvp?*`(epWi#B3B1BCM@rR z`vR_;au9EjQ<8{0Fjnld$RA}bm>+i2SnBomweoY4*i`E6XO_03F1adpq^B6E-X33? z0+lTz6{RQTd+heT30>9zay4ELxIS6=paYdHJbzZUq@tF0QQi;*!A)3fkCH)U`D+V@ zRFEMRcmi!JUf`8$NO1)y-yyE6n_at-K#+OZwOBKp!0&EBjP zQ8_0a*2qna_*dSMuLi{C6u+|g<`d|2Gl7)!%6x$^NJ+Z@N*ddajEcHjie;ZU?w*y( zyjUY8o!%uX(>O^=hC?cD(>SHTt(W`+^hi>9wf9TR=`1?Qq~FN+#!YcryPCII=NO%T z3<3J-F9|Y~!pWm909;%ExLp4oxQa#wHbA9|p1!5gf8h4dlQ8YS*r?R%ZDzgZq6Kie zuAKoXK+lriq|YPn$G}y_T$V>Lo0hYB*Y6D!W~p>5bnv~Y9OGX0&SQ5ObocjnN3dYS z9jY8(^uEI0>cJb~8$ti8kLng$6>=`^mj*Qk2)Z_*lKJu(d{ED&(${IyPU+R!smAS;_ju6PKO;1$7 zgES5si7+ItFfJETg>~)|5nKj;@uhI!N9nv0l)Fw+7!0X>_C<V6+W=UF1`_$?1ea5t{&`UhMxJiMUy69lw>}x(D)uoaGAA7&Fiqhv;`${Xypp@P4 zVRsYqQu;M#O#rygCLrKC?5NQFaTspHcPT*u4#PJ)%X}Ol1#P`Mx8mizm6n=H+PeN+ zvBWZMuir~G%%`!a0X;gR-E^@udxJ;}s#vZFooa(A-GD@N4 zhYQ}{Jp=}&M`Q$Hgh3t}D+yH1HAQ{Cogts|RwS(=> z9ngeCY@+3YI$?FT@jrP%>#=T}rTOk5`rH^tmumhN&YH@vDcSYxkqMzg0O5=b2xq}0 zZEc0@i#O&Ho*=N^aJUdai%FRhhV=ilKnjcfH<`wPAv2k0mLEc>^awgUm1g zqPt%++Ib?-qDtdaSO;6U@yVprbcBV|1mwlbR?(7b6CbV)F3wLY_prv^)nf}!PjB~6 z+W9|%3lLr1z5p+QWJChBp%DT`y-Yk^yj47wB*6~!Aj+T=JTt}*1V%-b+8jl0`C9X@ zimPK^pa7}dc_i6bOk?yQK$TuJx?g&P6cQer`gM;1}_u{zE#orV~^P=al zb!2SCj7#bA+N}+}xlAfjZ+0>! zt#|92e=bKjtfbwqW4B{YR|fyc&c`Hl);k`>xT3a|-Ce9*W53Qk-0-rNvZ>IC*P1lW z)Y%#1dQp1cNa36CJNtM9vX4PX4c8-ZhTkZKQg}(J0AwDVgD(d$2AKy#!NSzi8t><- z%C*0+mF46ex(uux(5#y8_&dwhff*DH?~ZQBx-+CN4U^J+dqSO3*Keev(W(6=K{7R!{BDF$ef+F&26d( zUhy1yU;v>95)gX0!F%CwPJlv>nXt{OoIshXL($fpxdo_Oiw|JSK z-V~u4L?{IcJ(8Y7kEosZ^HK3bIr-|3fe%3+vDilr<~qjt2Ov(W@4HoUW#;_}c1vm* z(Gc!(>5zxzksHDR#1cp=RI{1;(SCwKoq0yA9)MU@bLF_+MD0EG8AHX9L-S2Pwo$n% z`?$kTy#tC$>OUhwt#iY&Z<|bsdIKrnr4haBIw}{1pz{@5{0Gf0t`d*hbK?MPAYw>#VAIO8+eq{xMT6kZF2oKG-bdvbk9p= zo~M-Ik8_=tX%gZ0@!#cfoF{_QdjPPQ0L%UlnTN8Om64p2<9{&h-vyOOaMYln@~dVi z6)C+oBS;$80B76Gjjl3&A&%jOar~smR~(VoudN)UPUxt5M^f3{?2q|QJghgapO3Z( zvpg@Le*!&*h?HTJatev^zmK?+l=$%`K%?OpdB5bTkWX8c@xHm~?EbYq%*RHT;b1ohEZdabr7)vdQjqSLCN)1B_gV=P@S98W zu2aDLOZOIN0g8QfusZ9jaGTXeW2)Kb^{Fabh^YHnOsuGla1WF-+|6T|WLR zk0)clyZmM9P2oT!9&eIv=rF);lX2mQHo|L z3p348LS?h(lr_3_aO#oDuJplGf?BN?=RK-Z<1CT(js9}p;>DrlZj!^5=I3X>*;|We ztc-JRa3D(elt&QRW0=^S=WH5OV>tL+W7u#3)ENGJt}%S(*#D!(P#1;_wgRXzREP%^ z83Hwip@Lq20lB z*#h1a+WfXm)!hOBEU|0cGhoAhr!KMvLDY0}@yn{b)1KyO<$m9>l2H1PnW+L+@4Zxe4;vKO-xv2ymuRqoDgS@Fe zAf-fM83Izu-ISe$BU`y&ZmR#0QeLlujeh_T`#+8uJR4Jb&xVwujisf&o`J=GpjS)n z52~$&=0Zqt*~UmwCD4_Bk=w87>apL+Hc8w2$qqU9vi6|2)_ouHViwoj=?aMvmJ1g4 zYtj?t9>-nU!_$fX;N3#a^N4{IjT%y(B@xg&H^ax|cwypxFOS|)tw@U-!3GYoVR)flQt!Ra^~&J9Z%&ot|%G8A0>Otat9ZN9A(3%4HwH0x54ezUVW>AYLt zdbb?uaI}1Xio6EU?7Ot=M9f23z2kw{QEFT3-Gj2Vy3x#&b%16`twA)amua;#)(E0m z4uEDo{yAbW3(MuY2gxs;Bb_~*6`-I`X$K4!0PDl?qVeSMM$G;k%Z+?GRBG`MkFj%- zc?Nl>TuNHxAFFL#d2$a^WY6>pIJAAT9i0vCOnIu~25hz?nQ!jzc;WkR)S$uzNpElg zYZ?6?)^g+eTj|)$2Q;!IJHcumSy@bU#jOU@aQr6TOlLrX1q<+*aQ^vP#H)eC*AXw z{D(q5gs*lr`2U3OfAN)aEYpNSV9M7*Qf3b$Wc7>ZSUsh){<7(~1?AmR=EC-Hs= z01rY0n(oZi#{Tn=kn2vEU6T1!1r0DHv|wT6-B?hQaAS4ETfeDIL&0y~ zxd+AeJLom2XLK>4a>3@LaJZSYS-KmmKYLrBrrWxPfIlJVO8nepZT887r78E@I28L!0OXS~Y4Gu|fpzsq=&b1kqE zOn*$}kF!x1cmSbqC}%WQ!gk(wmt-^V6fB@Yctke1c}%W)DBCy{(8&%XQ!TsEs42iB zs>WS4NHX^fRWIr7f}AO#YF-Z_>zGWFz`si7tz#-%EdasTfMot3MydZuhW>+gmq7L3 zL24j7`Q1P4WLPO^Y}x`LQ2qDLdLXWJ4^lu8RUFI(pC`tpAAa8%xF=;_Yi!*@o<~yr z=13;{P1Tj|Q@iuV*ji>gI2p25YqW3PTf6|Hco}r~PKHH? z6_vn3uA+L;JzZyJdaGw_EC*(nd#-@}A^x&OUf|^bQ2V{a#Y~->P|Lrhy7rQlzyD=>B_?789F-J4y%AY|=*p5-M_X zveZGJ72AmlDFE1-Qx{MLgJ6pfz}5u>Tb+6kY+asV3(WRi{W04I!1iGBdA4ui&=3UM zxSOLNye^yC6cqgK11A7%nLzUypfIR9{CPgZbEp33A*snVb7^vk=J$NYyMLL_Sm>$d zcHm#YXh9G_*M%j6i=$Km;HnT0C5d5#Lmt5#%p6taQ$;Co8_QB}Kvb-5a1Noj^l;}$KgIR zt_Hxk>;U7MPvdEzA(Ip;ff(0cjBVhbVXiR{=GJCagoMpDFb)8|w=fEYl#Jd8EA?#| zJ_5WX`Geo(-$0lP2<9C?Fn@P8yW(Dc<(=Y%!Zru;q--*G zk%K*#L7r6C@A7XYQ2DpUI>5MLfG6eBVDfjKRFnlL#k3&DgI^klE8k?8M}RY;}_&QUUlhT*3c0EK~R zGtx9I1OFqoR_daR@l2rQ6hP`0mN-)KeEyN+t<&w;#o^r+W(WDGlPTw?S3!QVN(y~1 zBI8viZA&*qx~YWS-7OXrU602x%eUV`9x}bYc~uWNz*F)nNW4sq!ljg4`^D1X12UbC zO%HyTw}5n=>Pb{tqmeA_9MfIKAuIm5)@bKS@>1;bnZ-s!B9dp5*CnpH2Z^{J!*(3k zMX$DWL-#uizOogUMHxtmc;udJLh+7qc_!PSVqc^ye0ob8^U+3zKp8#%h2ft&HouN7 z>+T?gjn%M(Z$7Fh2d7X=nrjjHv)8`?CtYUQF0^c3RLCQp=%{@6s)RXfEUYqC60(>) zCMS+n13I7x(tV4#RKZPj-%XS;VJC>{r3g;6X$x76h-*l{5YuNA!3E?mAD~4EZ9dVn zGug8SM?Zo8yCKn5L{7|49y8kav;*TFpLfGEO5Xjxl^en}IV~&lz+wb2iA$mj- zQV=bmWGjX347M|u0S%SeH4;wjB^PF;aDFXG;n!X-6=IiQv62QEVHl zr;75#!`aOh$~qD(m$SnU^hkt>ka}%h(a#cjkwYoA+?=1)pY(AsW;EEHF#V#;&A-Yx zHnDPi{2|V?Ls}x?SYTieEEw#%41{Ip&fL!G4F?R+dNyD zh7k>ROVrK*4^{UKRk#KHR+fl~z_Z3xaoe|YakX^UO&esR1UjU}#Yj1a2Fexg9#<{0wM*f5_@BO@=hSVk?5Dk98@kyh8f!-A2`(G$Wsj&tJ0OmHZQ> zec<&30rjpqY@N(!ziP|Jm5XHH^H9lj^?u#0Qnb0G&Ovo7 zB>NPnH5L}qVXEOG#`b-b=sW+Jf?4KH+T#4nbQ^-DDo5qpx=l4VpvgeCOi#12!#00c za(2+v&51vkIm@5?R)Z58Owg(j z-77kjzX1MyUI5#i0{`vBDVbU6>wzZPK&UhTZc#A&6Qck4-~yE`Ak@{0VJ9lDVjXW>u1C6~Y$8F_9lYxMwlUeHHm4&Auuvo4-$Va7P1Jqw8;H{b8A* zN|F1Od%~A+LWx824L@3&hnnWhGu)E?nUQ!I{?JiZcZ9uQU+qO9J+Zlnd#<3z}16sGNX zY@#Q~(sv5z{O{Dr-@$gH7S@7uFm1tFzEocA!^r)V$dC-*f8;5;M86aQEB>=!eEaQy zO=Nn`;U}NPB5D7gdtTnhzZ8!`S&(hL0}c^Gz^=jnm2UsRanFeQ6GZ>UbLlFFpz+Yh zIyxvxG_*4Wzv~*6SL?`v%F$A_t|d8LsKyd=7;j|EtDLrzP}ebkLEL}C2S?ul9VDs) zsZ^xES>jgWRvg-X9yMBeKi<}XXAxH9jDMKU#b#pb$rWGEiKQ>vsDfb$rP-ey#}wsr zpRrZ>u5Hobv@G8}LV0T|CGkK2lVjRxh3rpo%(&n9*%D3KHpwJ#%$9fOMq>)+Jc3K^ za95qXwRR5JvQ4#Xu~evdW#q>7GyOEr%8rF{`)3Bjh+%PqOtDoXzO}|n2ELE;xcD2A zsXXA_GojADl?ZKgxWhH+sXJ6-o3fv=JK1)OJ!5`;!MoC?S{(`%slO6$b?$Js;_4qa zQr^K&HT;C+f!{fvf_-k&hm;hq?$kBF@0LLp?*YwKph`8P>YY74v0KHarH6Ia2EDM?;%JjLjwko@b_NlOG%E7(P*Ehx@{}D?pHrGZbI=Ro z?uLq7YL~C{aj$+} z>S3nL*_b=8R0Ywx&n&GOKRDS@m3yO(fVgVn9Gan-;5NJ%CShHYqp#hUp_Bb?(@GFZ zH1s;~Rh6R)p6KRCs+x|bI{6OwL5m?XJ715j`5esaOM!lgW32raE3ut<_Qg&(Bd^)7 z{#-FsID=T-Kc6CgjS7B}*~EHKV`U@~f_gDCb3w{oNRYZ4eQ>uaT_)#{e8FuS`%U)t zAo~(+WN)kFgI!&CG(p!{#V6X_HyP?;R`-bJ-TYZzqAMna!GrO<){BINFW^lXrZnF@ zSX@fj$hcxp{%)Fb9h4Q4V)(;Ui-8gZg5(~=~z4$g{_c9>2pGxDXB1;dt zN&gmlqo064xS_B|g!Ajyp#R6%I|gYMZCj$1wr$&XrES}`ZQGT$ZQHhO+o)vcxp;lg zd8a!%V(<8V?)cUk({qjqgix6)?p>3Ywucl`Sx^*Iw5)1x7jBKnC21RUnl?El44_!sH?Z=Xu8PXdSm-g{wLWTl*DyYsY;O6<1MjluvbAQa@r zoRwC+c`Tt&n0F#^HxF-DG+Yr56@eT2;rQM5<+}qE@C0& zD|LrtM0$;rKdh>VqAr@JL@Mqn^u_=DVdE8>_=DFQYtTx?<+2}oiE*WUvF&-?*5Rpw z$NTvL&`Y?i6s4~Zth<|z4b7pYT3c!WF>yZwlN3^e;#FV(yn{R9`l=%g$}6VJKP3H8 zx{vCkaK{t@X+uTM5VgpPZMzWL2?^wh@m-^yt$YM9V6+BT9x^agGpxb_9qn2Cw|0+; z9=pQ2)rTb}gLcDbYIV=w&T0a@;L4g8e%OM;MX^+6qZUYKFWZh?qS>swjiXe)$ zrIp6$GP%j{3X`CvpckPAh6=F3ZZh#GE2B8HV(t|MkE30#|nxRnACzvTpwarYi z&DI?yW!tC76pUU))y2A-273nzW%Fl0HFm>T>!D}BjRF-3&-IWUO^rn=p;cKE( z#o7`qJY^C5BTV zG?HtpWf$#wkEl%VMi-)q#7wn$>TqX>ede+qs40uoF7`l~3^|A^%jSSL%n3soL9Bbo z&|rvkNwc{#GfiX_rCY8(YL6MYq09g>ExB9i9s(|vTmBwJHU&Kk_Xt!-Q6I^8w{@H7 zx(0ft6R}VIg;3-4udIS+32YCpg$m%T8MJbEzY-z5pOs{5L z=bJq41>eW%^9@A;bv|v;s-8llr!wFn`pg>=3iryRA;J$45J)JZ8tGSjnQAn18H>x^ zbo!^?cH#X~u&r9fh?~%4ls*hrP!R<4NOz#(bpyJM*0>=GK>jzlr#+!s7}HN1;0eTA z9m%_)DLx{)(%sYaf;o{4Z~I1O#Ni88Qsvyu#7^~s3bS=21PM6BrkJ_`*AnS8iYDoo zTGu#gHtjKan_f6&THP^D*La0Y-$V`B;~5uc?v8i^wzIEOyMzD|zp|)_Jo!aR$kX%D zr;>QcgWRQLzGEC|=MsxoCmyx>mQ20?5ikio<&R4;mcS2qM3x=zwX;JOxk|V28&WL; zv}_Yv6ZrOZu}$5es<1>wo5LG|l%z~lNMo<{30KDdBzk9-8Y;ElTj7z#5+0E!+Dwrq z@QWqP9SKGet3_nz@#YzQ<=gD+E8&c1zzV!V|Fb-YF+~I1{K#|bk39dM5}p6-qx{>D zDM{Jd6-gQ1$Hpa5dVwacQBRjTm+-LYCzd7G5Hi;=BhH%-w$Na)VVh`YSsT0z2^`rW z%K6yUi~ux%sCSQxMmZ0dd)|HF$%;D!2FBdmQFEDnAU7%#~y9&uvUp9$Gg%j%`vKWnh=&(_OP>H5zugex=~4z~0ksUJB>Tz7xI5OH+Z z7Fh#RZIxg`R^JS_%x|zu38Pe9ZjnS>yD$*?<|HL0&mzVs$LBC!W zM}be|Dzk>5OiLG(3O0%uXJO#;YSKY6@zWSa*-WvAxEXJq9NTaUt)6jJJj|Y;HL9V} zo;r4joS2YQr=_)f0b?v?ATmT$WZWLCjCyG8wKI_$nd#TGbdo90-9HZ5saa3Fr+at&63Pjf@mY7B_{1uL#l}8H zp-IZX+_jqDVbtgRe#qEr9Svm&w=s9b?ct7R}j5cI2*0+>m`Jhti_0Tlde;KKEQO zKaDSJZ%6cPK4ggBLXQLzx8e8fQ;8$yg<+wue(-4r{7yM?_HAJFp3-w&F!=Q1e`uFQ1vSt3XZwb^E}^LnP9 z+%=ev)^a0z- zqwznMqsod4{bz>7I?>V=ksmo^2cW%!8Vb5erQ*p3KfU@8}HGGcWW zUpl!cyk~T#7|%-T@kdcanYu`DsSp@#pJei^;P`xgeIfN?mXkVFUbvYN28fbEk`L$Y z)y2Tz$f7^W92gIHZ|Uj53f6b>WCL%rpL!IW;Y(l`K-VQ)EC*#1T`{j;2v|a|H>6ui z&W|bj=(7d0lAEvCOw-F$M{HF?S|`|WJ;{-w?+di7yL=|mt3ihY3BE_ulVCtos#&{S z)%qg`avbPrt9e-n0*{?6D7DX(sbVijSGEPG!>@mHLz%Cb zKJn$jl@om*USmc&$KPDxS}AKpce9?@xVvA_V;^4$NzCOV(HxknLzDUh7#K%zmA@aw zvOUhSr(eZ>4Tg!Bix{}0U)2jUan!DxxPn8rMt(1twb`S+W-$7KXud^mj!6h+q(rsR zkJfMN`K0J!x((F#?Qs~(fTq;hm7iF=(?@*yT+WphJm=cC?!d!9ZIHRcXG0*Gi(6}HDiDWi`)OP2D4kO-95RFe-OrUq6pq7hMwUS=L6$*s3}za9fW#xx zk$Yvo%kL)(lcnqw+EIkeA=OcNg}ze{oBhOoc@rKC#-H%EFFP~Z-)Vb;xuhB#=?w&Wp*6_6g39^!7o)Sd zBv?NmYDLX<)Pj?Zmh6~5*fY0d;%*y8QaIgA=-(Bm*ykn*9`mL^q zM1hUujy3P2kQ&#PpzU~j5z?T!(+>%{#`^WCC8+@Whr69IrxJYPwi-7-lt+lrq8=3} zSObYF4F;90`efGI;1qSumEYj1Z7i3EhsB2)`BeR{E#-eT6hEfezPDcG4Y9Ib8Til% z`PmHUTyP5O^5Ik&YKF&5j1t2pIy~PsV{^g0hPnP(^ihlfu$_NSPxVLq3I3hv|JMmR z{x>0bw1T!I;7?YtprWAp$RxSUJ1v78SaL=fIT_!W--PbBbcbZMRY!dX=4XC3Ran?5 z*keI{hr%;52$SPX+dx+)hwC=4&h{@WeIgK$25~#1grH-A^rKX`F+kCng5KJ)X4^`> zo2Eg&r7cTmBM<)~C}(uF=5=~u`bVj6EPA1K>5#z~8_gJ$OGcrJCMeLdYfPuK-%d^& zPc$dx#-KbayW|ekV`_cnQN#+2%W?^IF^qzFbRq%`!Hz{)tyg*5mTJY;*dEyzV{$(| zItkcUk};^`bEFQ<=)L%CI3the&9~kQrgA=<**eeCJbI(w>j8O>p6_Fpe}RWY`*A~z zlPsz{KTOY9(-zN|;ihfDl(fE70bF?o9D;aQleCgG-c7BIqlZ&+?HP?38tH`wOYjvl zNnQNN33A_{pP$n`gboR`=E%h6&u|J07^0D)qtAE<7s(>A4b+4%P$F_Sp|cR@osP{- z$p-|BiBB5}>pD}uf&cl=AdZm~Pk-*L{KxGh`@i1Ve_h^x>g;F*`5#s!%1QsPMxLdnr=i>g9NRcK_y?)s%4UTdg2=44 zaL&!by*Ia^yKljLn#Jb135F-TPsW|Se$wO0*AsS^I8#God!NeJNZZHIaDzb3KFwesNyDsKXe<(7)+aIYg@QX#5} z`NZ3YXmn#4s>H{YRs!gJhAg7ILjQqZz^CE^O?U*9aCZGfJadF-rQ^qvyYsuOTqmG_ zmwfj55~#OI-Wp`LOChsfIhy3t@WC3x|YDTtWwr9?R!wgT4 z&-W{Y9@GXXyNX2W2oDfSWwPAxrxf?>9Q`f4C1)jc&Bf~M?BeYFtjRAR*Zfi0x4evE zpYptc+^$<~25B$QgWacFyHz|`muK`;M`yzqO1E+P6{vhZZbLbZs}^VrUR*^#hq##H zMDa{;-^p1kn9kS9-!DiAZq+;}{AMR@3RG;*SgDccRTGEtW|TN zd~Relo8iJvR3(}iTkSG*odrG?_KdKEY=5kYIz4D+_z@#lBL~=k>&MABaKjJjT?orN z*D*gjV6DOC?tC`*19dVM?8w!o(?ymYmT}7}6sR8G7PS*xA!y=?{rT!1U@n>JYl1N2 z>0)*B`JhgetO5RQ#jKjp*N@ zLP|mLUrgkW;{S)s$J63Py{jT`X<4_lzTED2wT^~YX&%5gNodeNFC7MF1c-#)FrBcK zC;y$~of|IV>u37~<12JfSqDK8%?S50xykXexfzRRs@n~;I^u&mp*FV9s^Ykg486%v zn=A-$sHwxADfBkj~O&%floA4c%RC9OmT9u zvAkAuk0kJx7AJs%m|S(t$qR z4NXv~<-~&r<=ZQW#CA72Nc#*PHW$Ma3vuze+vPneNb)v`OSQK#)vWnFdqPx1g^w-s zEqX9V4xThM^G(!&GAc7_MBJ*>E;06A=PS|s7Rx7Hn^%##AGDON=sxGDp;L6W=&@AL z{NBFh$dx3MC1){R;mEg|ef-9s^LHO9(!BX4LKFe(KIMa`ud-!>GjQSjEFFSt%DunQ zO9bTPQ>Q+nA&`i;(#^%371N#rzfwI~eSK%l?D?8x;2UkH>DV@eiR$w>!75wFQ_+@@ zk9?Q1+I;e21+e-8>r7jH{yF)XTNqY6+Or5rE;~n0KS1alD`87$?TWOAaEV9Q!w~R5 z#3bMnnBxUm_4mQdUzmSZ8Ab%GZ!tfhsQ!6z{}13$wRN!kPmLiR!vNX;vrYY{Ze~F# z8JipI#Vy3iFE{I<{kvX#GGxkTbd8$ZFBCD6lzr+}FEwrEea1#Gm{SDLclehN}Ko9iEa?M|BH8PxY+)8 ztyFZT=gmiB93nVVRK5|rpV96RUqMOlzXUd}e_utoe_($1^OF9%e>iKgM4^V!J*vR_1>UmeE1o1 z=45w*n1g5ZsnddimVIZ|BdFKAvYeX3Sb0X;FtndB#$5XX#z=X2;y$*Akr&xCsx~1z znCkOQTi3Azb@kBAj{++XG$j{hw%ZU*zACnbQE&q8ONm!+F_%QA;!$LlI+-IFUN$LMRV-*C+5s7Tlmqad8tzgY2!pnWPOD6CH!8d9&CwrnfK3( zax~R>!PE|&ah>TPozlxb?l{uOEVK+MEYkdxVgOUS=05x@neM8L2LccAep5={SuKdTv4R-HTs~h$5Nlz>(gP zAW4lkUrsoDQ(wRY`FU~0)Z%Kz=#yztAk&`qpG63CCrcjMt{jSM)23>W%-V1qXP7NW zD=;%Q8`N2Z>Em3~vNEDlrZfV_)d<0B)+di=zyasf((qGVDF5h~v zFIk5|FsDabXNy|9z(za83U}rw8fWUUd#H<0OXkX%g!2{cPZ0Ns(am45p!^g-Pd%CF z_VOY`-Kh1|g%Oc}3G_Kqlx*tAye*sF}ZzWe1(cJgU$$FO*z6wK&t zZsr|#PlZ7&VLDXuaQhU(=e$!i`paZGK&d<={cO6Jx;T$G9b0`cJ%v^40{-w;mb*om z`jE~*kp9q;t`^|7&@7nQb(mN=L8>5Tw3F|qg?((`ASgH)oB#fcHmgELDd%e0j6eBI zty6Ohmz@>9JzJiWdHMFHS(Q-nr>5&f=@dMpT2eh&uR7&HQZ!4Ui1v1a;v6`32@u?7mosFLo7%-2onNeSsLPxHo|@1HADI{UFX;Eh2)m7oo^cN?jvyHMcZp!7&9fYQK@;Rm~Dj!N06#pu5sPMf*ktDx9P=ligUxc(}VYP zc?KO-2Io$Rx@UPGb38NFwDw>sC|a4cc|)0NfX^>}UccPkDoz9h1pb;%#77|T{T29} z5*YF&F<^aYk{#c)2&?%modwr?hB)g$S03k@P=)em5NXw zIZKnHnmp%*YY3;w3czqAo28n+dw=*&Mo%mGsPU9ek15q1EBF^`v?<*81mwaG_^`ni z*n`HHm-W3dL+y#93)o3(@5&nv56fk)BxK;mBd-PKUdmC{%z&erV;t8YhQ)`G50DR- z1@)zv?xO$&86%>Y*a_*SY?CzID>e;IRUM6M__sBBc7tSGw0;4aH>o^e{w-gy@{snI zdXyVJs>sHIg#NT7@%O$Wp`w`-rkVVl>%S%`X&>ET{msZ(^EtD50C=I~tunV5P&KJ$ z3ZvLns#cx{74>-|aF;88XDQRv6x>(>BwVJ(RNUx1P`@5<0_y9ST{!VEGmooy9Ib9nxsYBcO+Is$ z?9i87Y@Q1ES;ZkyvB(M|ouP7n0-Rti`SR_(7F3j6Xgw(>EURO=rSg=|Pp26tY%J2i zDCkuDb`}*riT{o79s~?ZFV^Ni$WIpSXG=%0-6=y;aDd^}Fx-q_RY5ml7IX3#fsu*8ma1+iwkR9X2;%R$mx%#g0h3qY+ zzy&%h472L5{ilP0f20X2(Pe!Z5lM$>JaX;yy~@bV}1f|N7O1^?$Id z{~{p&=_w?kLAfa&J$+cudwRCwGCf#?RK3B6)6=%Vdm1@%z$w}AU<2Hn7aRfF#keRw125PsBy^TO|i z5`KiW;ziyf_4A_anG$|z1?h&}N+S4>^w$W#MuYbh4XFyfW`Xyh?ZFX#$OpZ|U&$wPhz`HJ6iesmyz0evUIf5`fML*0e|W9_p0D}tlL z>zM`kq15dwv}UgnZ=P`YuaTUZkN_ePw?@^H`={8I0srn#0QL?t72hSJmV_tf)(n8( z<3O|`;#Li~3Ys8p&9rL+R;@>rBCp6@>=s$%27C}!5qAUd4+ch+H-gPaMN$*cqwseG z&y?Kd22>vEt@f2RIB9?acF6i^#+-M0kUp*#>D;uXcOXHyjy}usIC~sTEQ<9~nvNSu9$j`< zS65ifR_;l3o{B3;st|omjj_pA-0Ym|%diXE_L;r=D{!3KH;X8uSjb6P5G8K}2b`U) z4dYMiCV$DuMDOy|9mrg34cjh+hle-x<>!_SZ~1CFw~l~LEL{BV%@*e2AN19I7~m{^ z;)%KCs~3*0z0ALv;%muOHoTxjZJ%E{zO4-_!KhGL*R&@pw)EI`?+oko)fiBd19~oG zztgITAL9`5MyRJSVPeb6%yRP2%Em(XIWANbh}s#^RqygRA;&hLO!O?$DePNeNImG& zBF^6FnXc-Dq;7gKmooL0JI6{@w+e=)WAOQNb&oFJ+)m!@=k&jBD#nOz_f^7fGebZP86>tN)(r zOX#V2E1kiI%sio1|baQl@dhK&rAiSU4w6a>779 zX$_x?09wLV@TtC~+m-}{lBMevX~#&lYR(BKLLFWnQk>F-M#9k3i(7415E=;sn!~w%-oZ2mrJfefrRe(c}wD=0n zqL8E&R=r=;W&({m(HQ_8)vPVv6^0&4u(#ZWR+-ry4ppA}BScwD`pDf7(?CDrJjo4K zVPdzdN+hmTT;YPM;_nTiy=Hl3$+QwIT7C=uIp*di7k)WyTWD(Ft>H0kHe&9$V5j1( z8Pc&!ieW==tKSN-k)kAQ-O?lx0vCU40@#A+2Fb{{DYAG4e2s5M<_`pXO=<_XNn9H? z8G`;i1EN?ne_Lyx7*c20sG`r5QeJL^S_H)*fG?tl=D!8J4=e&fyvmAASDA&plgQ{5 zpkbx_MfC9GS57d4mn8E$aNR3OTQD$wIIeH@J~u(3t(uG>)5xw*>bktnm&Ut7mA5-G zVO&i?ZE#r+Pwh@?dZWmy(w z|M1e03uw#RmcFhn{pH!&L%ZiDySu)vj(zh&uhj&7#0`&b4ITC)mp1@h`{3BriK8p# zLe|s~G$%%;q~JCwCaXD%UpqN?D0i1mNuKK2U2vvOEY;;E=H>O1XLH8=-_5E-PpEg# zuI=dQ>8V#zSm@V)X>0x8l>1urZ5*Jc5TqABq7}fQw*L!F6-nM3xT|m&Oz~LKgCEAE zXV#06Dq)ZmUb2w2Lak~X@TqhwW~qP9udiO9BG{tA>>U4e-hjOKA{jEaI3Iu}fM!#n zBu{BlE;>492um9vmV*T(;Xos(Tq(ICfek}`|9JI4AvEc(HJi=vs4Q3A`HR#(A^oOc zIIa!I_x^s9FJ_=!{>XTnscwkJ3c?t9fTE9IMH+ffyNn{v>R~A290o*m_3Y_lnp;+F z)=dE;J(InQMbU9Tt^GZwh4%8$TwN2Vm9tp16h&UAVk)U}O!)Pg4GUUn@OeUHo z_%}tdm|F(C9uIaC|NITa1K&0IxgD?Hwtv8~t-G*}fKwn1K zaQLxIzvmz1pYgr0_XJ0`z)A24x|^@TYullbVX*!8$@)}Oc>$iu@{IlhgI6{0qjn6q zbQTrP>!{!32Vpl$yq}}yDX{;l{&e{v=S7H4a_DyWB_Z!|yI6q%4<;uNxj&0QZcS6= z322282TB=CL_ZV3$Nnm(zOBtaRAw zZl0QRK)FzhKSColkdJ*LHn5M~LJTA6xXFP^ccQ^OBPpmH!^I1OcGwCeuPq*BpU@d^6^q7t^8^F|+ zd1F$Uxmk|i)3a%=VJglXn6>)IdMffBSt* zLjyN_s2sjC!jIp7j_RiSTdVv0`dbWbHFM7uS2Nywr2YkYIKgOVoDBD}Xqp&=r zc7SpMkq4C=l7*DZ4P~HmvykZVsb)IAVRZ_Vzb#JAmKtXMheos;UQ&S}2;bBMyiR6K z`Ztp{eK||&j7$le)e;wX)^@J)4TD&Q$AdHvg00jecoa4x3vSM|;^g^R_}Qb2`cpP_ z>%ny=4q5R7a*GEiW7c-5@{L&^4w5?;ZpI=z97_cJj9@geMI&7s*kD9mpq{1yIoh1* zpE5532d4+L`R0jeY0q=nfUZ}$Px)5Li_bHwOyP<jRD!n<8Qo66rSxGJ)p^vi(FVUR?8E5C=RVSan0u(QCPYFMe~CNRM4b_h+2LeoyYj*|#P)ldk&{F1%H6V;W?t_(1uf9Zz&W;O zECgm=A2|J4_{@y4Z;xE)V81$gw2^t|z&UXuD{*!3h+2S<(H%OImkDLh@|x@fVY`J0 z?t5at;!k@e{pVEDfJCq;+fNz~@+U5*|93I@|KLzibov=+_?Pr`Qv8I3;6Fyg9?y>! zRXjU$sa(6r!M0R38?}|r%TtQrWd|4>q$Cnl?l`;<*iePMe!XxLTcXUDkjjR_I7*22dXU~mem=U+g#PUm&5+B6kgdg5~< z?|-R{#)11;4P8ndAR;r&zhY2mZq_tO=RccLM_gcT`0l_ zT5nYpRDGOEiKJG7Oa#^g*>RBrUpc^`TfrNkAWKYzOzgK|bNByfx&23C#B6`I=JBVB zv?usOp8Rhw;b$}0|H$QCdAfUP4mI7T%{kN@TE4%LLdMe|fdR7XAOXtQWx|ixBl@Sn zb3w@D))`PV^Cd?ShlT{?Sqluz2Md@Zku&%kpf-g`vl2V!7X6t|Ua7aex{%S;g|f!; zzPVGopHAaSs6ku*^YZ!kZXzp<$?;~RgG-p5EN`f%22cP-HWolboHE_^xLt)`Vz3eb$04^U-Lqc4I5M~&f zX_KOU7wz&$KdlQw$7;0j;MAMCl0E|(EFAeIB3q^zBa&+au;hFtzr~fpYLU{<7ok>d z1X`u^a~!mSHT5#yY8Q1Zl7cH>pruP~zVjv1Ckh2sm>{ z?BGy5%UaXlexLk$tGTYKm<0XE+W;NEGG-;$>XOiZM|O$E!VLeQbfJmwiZRi?G08F2 z#lMY&d{4kyCi=bveLmuLA?gT^s3E^J!Tc)k*J@l|DRty+f`#9wVRoTv>b^E3T% znMD13|IS_=rBz8tPj%y();-ea@_rq~wF^kk0hsr-8Kq~({X<>zI^9{$9^zY7qv#%$ zHCv@;&Gn_5Qn$p2or=qy(eba`T3^?JtxILB_fL;n053lxU!}DirRU`FZI!hg#Pe5dfvc~ zVm0P?XraUmp!hy`AVWqIRmTGnP~k+b6=oM{L9Lx`2n17E^^u+CvIchzS5Zer&kama z$C4W7b|dTQMif~#)mtgNUS*TZ5#wS6Lc-&qG|)llnMR(}&+4UU=@XYmW|z1_l>&-h z5a>s(ljq}@;$fIp^=@mHzyK9r$;ckV_mJc0jUk6iY|!9j8AmpEiH{LRU6xLZF8~8 zl8#t)L0!u+>QI2;Lgd-d1t66WWPn_RdH9U&N&XE4*f3>(YR8ees+A^rF2W`7H~#N#@+20jU#>c9MB zrP0$e4GA)dR>OhZ!{zEJ;30$q{n-Qe9egK*{j~x3D_A$^(x@^80{a++zEinq1ga3? ztLn^3y-%iNorEYI)TN5v*NCYSEG#{Yj=I1Ftpg5jW1nY_b2Z()@3$Jq*kEjhb?$PbjOAh|AS>H7(!fXWajYT1 z>5CDJSLGS`ArJM&zNdCa2alAfP*H4p?D64egBz=#Ss2%rZpPwrhOY|u;qWwNNc$05 zTT-WAPD~RFWC{iG(=fiim{mfA<(lNtQ|VKt8#tRD$zf3^5Z+RZUhayuVm_svKA=gB%+ zxmKcoQy??DrjBK@qz;i%Vl5o=X8)8eX{5{P#CRD;0d8))_zvnd81ox=r~1|=8tfJJ z{EQzfW4D3<7w)o{z4$hFSkSAa-;-r~WPqIAXYTHL2a$lqhauLiGO5X|GpAq}wQdMA z)*ktk#H(cZfIIK@CvJvV9imfyC1dH%xEYKxXYeueOu3V#_;85U?aN^KEi{z$K^g6v z9x)?#r`|1k;Q4n6@;h_*<<1h>r)s$7V8!y|;R6cuo3K3F7XoF{b{7jh^A>^Ok5Eu` zeNa|j9g{ka^xyhn+e0?<52_E7Xx}8?=!j2D-rl)HG`t*q&JRLjCurb&%%0%pqsreT zVc(=}2JbPE0zWPOn(k_Qx+|~NAGyP!;-@VSg6NG$Q3uGsi@Qr5G!P5WWvIDG)absJ zG6Oif0!O#ShuBJ6BO?%ShvgVCkOo|zi0NklDL!nXcLr{Z<=B!+7sDbSm%w^!1rpfZP2Q;nl5HD}b>#4l5u-^_1P&-n~OeS$+UO zYIIg6L|8;u6@qVy4LLMNU2Dm6Q}_4P*K)Is?>jjpK-;a0BLzH*66S656*6pdiCJp8 zR8__d5fap+2gBxU!<)#*G{vjoi@zYpD^X+s-Qmv7YZ%OJ*JGQpIeK=aowGL}j2es2 zJdg?xE)6>QJjYQL1S(qf1E!D|8uvTS(mDdw3r;xi+Kad7L5V#qkARZRC2m#io#Nkd zSmzQgB1M8$VW%O>qylS-OW}>plV|Fw?0b|vPDW*2$&)z*Bbh8)U})c(Vwa5&n6e7m z<$1V_y6~&$uEB!H8cn7$JZ`YTgizG1BX_Z5cKl4nDU@M30!%)fGdP0UC+JhFY6$Z>x*e^E0k=`W)fbo;h*4!0**hiO57TuA(Dx>FDE)Ly^$#g+r zxtVy29_$pkRqwLYS<@pjJZQF;X_)HMgD=_GLJ0>ow@wpqsqo`5ma+5xiFCC?u^Xes2qIO<}m!j@hjvx+1F@Lu^7Jss#1}#4!1oF z6uijA(ULqK{4r!2CU4Pe3X84Tc5{2JDbL+g(Gx43KeVUI`h8u*SCR?xY%%$OAaJ5= zOsk%^!)sVm^NP0(EGqoi+CrNgGXjh-*)h*t#QIY)CI%WMx1JpIC~7&?N0;TM(C5x# zo#?s{%CvncKmKVRLyKHj{9|s}ju=rUOti^y6k@0=Wpac&8M-ads-EsrKsxnBEK|iC zVxMBXZ3->xWdcGa=q5au1Jy~tntZeF#Vq+5JA zvPs2Y;NN%s^1hvm4)b-~=28<&v9_eRLE+u2dc3iBgn<eV5w5&uXU(5LMflEp1#hc(W6mDh9eN1$g#+ z?s}O#i+vJF9zK}HP1$j^=V2)3s(ExW;%uT!b27a#+K+@jlsy{k^XM5^T4fR<1t{Sz0ik(9tsr0F}B6-fN5>tJO|vn zmjCoe!-l045t~32n@w6q45d1PR$LNN+3Zm4w>yJ`^M|s!*SUM!Un9fmKQ4?rJ-1B3 z%b0NPXZ0Htou#j!Fos_i1+K4AfmmBwbK@_}Ae9_7V(r%%Rg$41xYJy73DsETNoQG} znQ$Zqjg>%x)keEJ=AP`!6ygx>P2UD_6KKsy4G=#XMPaKF9~6(UGNh_@SD6>1ok2J> z_r?s~{qC3YA=1&f-g&#z2|BxM=~UX7;HobZ0wc9V#-`>d1Bq8y3U6^~2|FRondt5e z$4Nkz6YtON4`dk#B5bIs!z0L3za4$CiX?0vp46KBE|Pz-k=uB?6}E2rWPXf z7wZ22p~i*-akMQdT|U$UL>=Jb^Hp{LL>}<%_XP<276}`G0`IZ(&k=p~rM{&C?=kh) z6@B%F`d|v;t?lI%eHDcI!0sQRzMYVp>;>!zqHpM>Bb=nUwFKXwzO4Y?pt;2Y=SpRQ zVgk|ceAMqqZBy5QtpU)V0$d@=wDqS#psnqNLZHo53q9L47EX4)5niqBeK~VH`^4OM z@SC&bNhZTr3PNwyw^ND&^yK^a8yz`itV0|=@F6VM2h+Q4tn)-W`an5^vD>UP=Dqp? zzdzj{`$fI)Lg=px%3D&2qF&+f!_Qg>X1kXXstd{cwh@7MkOD{S<_B0P?H25ymEsh^ zX9w8|;R3R3U)dAnEHE!Ue#c+lb^gHtTf+!CTB&+hZm8E%)H$wi=j5)Op^D=RpIMM= z#8+g4>5Ajm=8M*sWqr7&;E3J8-(k*sctCjmx}9O;UsU}UJcWqaSNAqd<3<4X*cz?Z?45^>>Lx+ zE`vN?2$}FFkDKKq7>EkhWMWs^Zh;PO9?fD_mVKFQ4but+g+%i4JWx(e zfx!n{%M%@o_&I?3}=?8vo<;>_ehE$z(24wn1j7U%KB%rz+I+GEy#ALS-thmaFQ4=4e z9%P80DofK`Gd~orpSrvvG%pV^7y&ngDJ*b6Zkyy?yFq^soZVFLh+L^fnFx!eRl{1< zd5N5Qyw|{UJUQu&qU}m6(4oN2Q5eK#unF%HPAO(1kjksYuV%hQsD>N7p-kj{8bOJW zH)J2}{a!idJ>YnH9d_Vsl>t<;Tr|9b>EgS-o3s(u2FA#!^U;^m{#nsp)vLP5ZDD?^rQ89hmK*swS({sSI6nmzos-Ju zrOPasr#%aS1O1qh2MllJbA&mqa}qqN*0Mzrh-EZ1Wx zphQ(dT+kKmu!4pw*em3f&lUx zC*Y%P7sZ;2AQgX>AI%#{8DIDjGDG^*RvG1o4iIi(O>6{#PdUE=vPvtwGMfbEo zLwB8OwhFi*X|Qoam!TVZV6R@`i1OZ$Pj$#}{IGC?AK0zd3U*y`aO+gvu5?b&t+4A- zeiq-xoyB+r?{P;tWWD#DITzR2DuS}!h;l~cUY4qgTG7dE12vbBdk%Dc!l0mxylDsN z4!rSHej8k;(G0u+Rm$e^#tpCkV&eKMXJptbU}7*h&2}_EF1{Y})TXBE_mrfj8}?*d zC!*y8yUb<075S8<_Ci2cQ42fqMyRe!eMg8`qq{7xK0>_hS4o$b{@g|abaDf>bczWO zT8H8~=_|ao!Dt5nK!<2Y(BvalB`bFU@rS9o9GV!yEz*zQG08<>OPS6@ zOCyIYnY@66xkjXcH_d-bvol>M*?*$>_3`795R*1wASYj#xovc6S-6^D_kLs#QiJSg z_HU2*Kw_k2ubdXEF9$bp#>r`|?60~-OdoFglK$mqD0@@F{SxWX%vI>q@NX8ji}Y~y zm5AcN^p)`9#PpTO;!#XqUVc;^hVENmGKT}^Z54FgpYhF_EPT%`;I#{~Q4RAdrtU`u z(_^Td>0X%Adl>{tSfhiX6zPXSaJg8FWohkURgt~pg8EOE2I&U4uo%r!!9p9}rk-DH z{f^Hk!qT^NyA^=N>ASoJtwgC7dMwDGYBVA*!d7)sCh7F_a@Dns`xNX8&Hbtb1DpMj zJB9csW)(h=kdzgUo7a2jTR$fWn|OOB<_5tfh>JyMkXihC4Bu4G3Vv2i6hoWQZKjx#1qp^CLy1hO zMCk-@kuO-1?LkJWr#sVYO{{};-p_HZJRJDCCtNB}gCOaa#k!T|AE;TMHj!2-V{?c2 z5c@=+6kbBhd7#1FN$4RM)XWGv5TF@E|GzkU%b-YtH4S%g7~CBOcXxM}!QG*O#@z>Z zcXxNU!QI_$aCe76FK5qg+}(5c+#eg!^`oMzqbn*izsmgb{hn9zn)4{?91KJCpiOPeBxkr-t=)g8f}ZB(N18$ddLcEqJk;Q$jQwI zY@*`k&njPb_$?ex5Wadl;6tn!sI3p4c?h^|^k_Y)hZh?Xt0)rX5J;bFD^trzv}blv2aH z%i;J*$t`B^0H;smZMok<&`rNOr2M`5x+qgprv%iIB!uCh!8IkknHnq4D4!xC<3h2N z1UBNFI_H*puUZkScjDA$w(ZCqrW-7CsIv@(ER}ivDXh8rCuot{w9$i7g@Ew;YOS}9 z#luLT+Rg`+R;P@`1A%hI?oJ=sjbufPWL30ORkTbMKT*?5;lin*as}<{c3iU7FBOXi zDdmdm+=WvSrHU<9CE#8La8Db!_t6+9kBe9s1vNh;0^HkaypdcRDU`A-pz$)IA@7EW zZ{aC~WPpk@?lBYG$Ew8?jRxWH9bsaW4#xuvE_$St=*0tj^c!K?=0@Uz^aA<%+QdH9 z*x(y$)!x~V#+nF+Gx7Jt^4 zX}WFZuksX{Ud5+(N?5Ryv3YhcoP#NOV3(~*6q&|IP7=hS@KdwBmVRBiv_Q*iZGquU z%;u?L@o1>ra4qt6@af8&TovCxDII4Ww`+x$+*?z<`zLzm*^SR8ei%xB{wV6saVWg+ zMh@Pv1JgFNN##hN_IjHz+G%)mV);eZbgku2mm{}C3GU784R}u4RVRJ5Z+Kgva;(qc zhM<3j?)dzC-$RTFNVp9D1lbh+t19=`^c0m4(%np2U5R+DY8o{43&e%eLnztii4A(z+LwWIU)e$%?Pg!0&Dd3{f= zA>*mW6|SVub6Rlowz%E1waKY){Mt>wOt`F!qZB zk!VlV0@2{LiD-|_Vz~DGCH989>9zKKDE0=jiJ|v#Iyf>%ktgyZ2N7GKZARDoVJGB9R(Vr6dj;{|P69pL!5Oj`lb z?{m+~+$AyDO43)pb0`xQC)Oi-=Vvy#q*(JNH+r{cDAM_7=PK|9kk-Q$jl)EGLG=vV z^h*;<(nGcoH$L-y{BZ;rF}(~!+bQ?qp_BS~eqkFoL)@Nhf5^S9UDz?sUKkVObjv(; zp!f44FnjCraL@1-o${V1p))Bn$H}ZFkQ{xK>EdVM-kRH}!MZEtm)C>jo#&|dmOru^ zL(@>HOuu#R%MtB5TXpW=Pv0&+bC4SR<*&>?J}0%-CzGy`zb6R95lY>R9p2uel^1?0 z=#nO!LPmUbevC7o-X1XIEYd(E$_d#))Ey*fuW~~B_yztQ!u)OpM%GLpgn{4ueGuZ+ zbX3f-Qf&Ka$>Xz%?5#e)4rlQJ%8+1@4QdCxS5lQKew67}nW;cw0xP*& zcabX8utIr~Rn#P91_@SIbg&<{Xx@U7J@h)1YymWI+fn|4eI zDv5%#j5Zx_qeDKXO#(n5!E>onyG$`A=fLA`#M94ciLwj5@ST~AlX}1wi4VMs7t)7R zJ0T;52b14ri}d>=cZ#0iqyACzymA{9LsHvtP{XOtZr!VOjdIy<_(P=w`@Dr(Lw-OE zp64CoTUiAbLpPgV3BCZ7_Y{ub$>WN|PLeNao=`8^OWBGr;s&5_#*sY%K5{D zeb<)xK*Cy#-p6=)ANC1q3Bi~ajk`gP4%4h9N}y?%lL!>glBjVv?_~zBGBru8Cz*c1 za8u$h5G9=SU%RLewl)}bgbEebc%pr52zSvC>q3Zenk=eo?$A6yuw18%LG9GAzYW#t zT3qQ7q1~cvn&P7akp=INn~XLFCVPBR$15u7o+_T1>o^4$Y27Hme0Y9TJrPl=MwjZc zCrYH~>Tj$b%%nKEa6Pba6Z&_j*MMGk|G%;gg+MSVE&EX^+Aqll(N_f!_5Y$606GBy zjtnBs4vw}C3bu}4r?xi#RjzDnJ12KeP*7*SulIvL^$T=>J&`o*~{jh&FeaSb@^NWYI=*X8w9Nm)Bl^C z5WzYMJL!fxQa>0ZBA}AWsjdev#)K3L6F`bBkQ`_9CMuqOql}F`#DkKJT*_eipw{l9 zW83(E&}-Vj!$~LWA+mg$9#m(!nr!(X9(3x)i37KVAg+RtiQ=boqO%lQgRmaOQ3FHD zyfNyZstch zKS-xI9H3n(o@Y3|oN;&|v%Z7|jMr`a@_%M)XKf_$kF|NgpT|c$$@9a@h#S{kuUX5Y zD(Bjpzx6u0Oi`(9AneN9<#K_kze_A@jl!h9BtGlM*`-E-i-%K!n}cHxMWxk^5=S~; z0}I(CPM$Y4YT3-WZ=<&!oU(!{Z|1^Ds{jEgSh-ILA2jZ;s#=9p=A)2 zHzKPDek>7i(6T&Df<@p}fzm6JWiYSL@Ac)uw`A`_%p>AX_ zeq@jt6FDpbon8${r$7gi96f;2RZt`kS0;e|A{+o`D0BLqht;QA>?g@Xv+7E zK>Yxz zw}3a8#?Uw=+P<;IKzukhI-(KQu!tc6Ff~-M_6IY8r6jD*5+l`hSa4)d2~ItzI;$*< zj%wXdtlOY#x$`EaDh;|G^Vs6|{&Sm#Un<8<(R}Sd^tSAamFz#KXfACw^9}1Z?(HM7 zou%gIi_I-?n=`)+6KQIkgPrtaJEF8Ywf3C*O(ahsi@J8u_qqjli4`jD*3`1}Z*N*U z8qH#VW^HY){n=YEx%?yNB<_(sZ|#hQ;qVt1owDdL1mhvjc`*jg^7kK+3Nv)PaX!l9 z&AJN$v^x%qPMh}T79kORdl9bgX{5yqu;tsf;RVJ-R08YH6L3|UUt${<1XcQ5js;l5 zjc&Ls_PqU$!92@J1NN8M@j9!{3BCRKlx=I4s<4>bi zT>@5Yd|JI0pZjFPr%vJOba%H^gOdP{oW1+w#mPt{aHL|eOgKhV1NI&&zXDAr0aa?W z?x6KtbR-$%aMZirF2ilOJcf7);LGX{iYSmYw$#A-XZ#%&$|ET-BPJ6uB^06;$(7x? zdn#gmK%c*C8x;K0X!h1)8d~Vx2?d&?EPR$QkB~}yD)DG8=}j%ENIaYbDR!KRN)jp2 zXezms1M6*s`AA8MF*QG=Gt0TLBnJttBUYUeOZp$DpnO-S+t7370rthW1s}9;Xqn8PwKUO z<9pkA{_}YZ%2Ba<>5mmp!{E_O7Bg~3eZ5aUkQGrfFzg7NE+eVlLVo}}w_2d4DpR$h zgWQiX>f1ga<=Uy(4`(Nh*U?%v*L4oH=`+y{=SrJ_Uj+6HjiPNQ&cd4r=ir@#xrrH2 z4-I>eB7coYcQvMO+Vw_iZBh%E_O)KeNQzS3UNQ`aOZ9p9hHtZS0c`mQh+55D>266- zylcjk-U_<|R?H?i9q62jEh6FZL|vt-%Re>NMGE(FJ(cl=oTitUh>{u0_ITielYVk> z7aN|Y_F5SY#m&b_L=!!Afvg3bwSb*IL>MBlxq7%?{6>cF;Nvx$bnTn@INs{4vAD9G ztC%^Vl6oa0LWrG)6=0$W&WnshDOUJKrL02A5WNJ|E^&%IXN_lO9&hBICPXa`fQ_Z` ziSrKT{Z=#(YXVXFEsQ=_g&l){@sR>D(P#!wWg-DzWxWVt;O`CL?cYRVKEX~Aas*Z` zg!&VS@IwBod!0*?^;R(41!JkfAddgp=nw}kO85xhzEytJV*dN#BL{FY{6{h7{{ZiH zYC?J8E@FT3Yd2>DR_zmoGNqtlvQv>Hr6GUFm>uoTOD#!#-G*=-Lv$%^QGMo6R%4_R zC7m@;Ca=g^OB@rT#2H2D#~)9`wx>-`%$Lsg`ICDm`%J-%t!a*rt4`aGt*1_9{)d77 z7|G2O)U8yI%NfI}*GQMAn&klD88@2Hrz3ND9tdAwn-`{_~)i=N?1} zEU&~ZfguLMBS-#Th8&aYGRS9EuV98}+LUK5`lCCzt{(lak!=~ct_d@*pk8%suc+QV zxUMNPpMYKtY_Evk7HqGOURP|dnBFJsH{V`F?6(nE0n_UQ$Y-jzNSYUSus2Ze(H!r= z?PWOc{_Pex@8RtyxR0Lg7`Tst?P<7=zU>;gkEL%uAwG=H6zca3ke{a4gpfb&60%L8 z;o=P8`?Z1zf}P>cusu3Qu-F;znXF8X%~l3#)6}+kgX!RoSl}3SdhCe$#o#bmcq4XD zf~(+Cn01EixcbrI)R}b#udRYBu(SK^;QJl1s{yUiJMM7nSj{Zw1}mer5r!Q7ci4}N z9lkpr!I;b&J=eqxdJ}d9h}jVtBl=#D-8#b2+om6lK|5+6^-()Ta13|52YPVCv~Xfc zCZck{u-3}Q5;UocpH>or*oJXzXeq?K&%UNHNp_lYlJ80zLzpR|V*cU60gr-$a^gyP z=A6JUs@|vrDLHDAL#hgN$MWNk0 zDI#J-1Wja*<`c|DoY=Lck{?)KjIo`7on((1k*^n+kcVp{;WSQY(CcQ>&M5u1=u-ob z3T!$GGN7@LoMg|2$xc&-%{dyUW!g-Pp|QM({-n&w!kf02^Ku(6B7MS}k5phUI0|>L zvE^4AxjJ&V51Zo+RI^0sG`TaL)p04ex{$V!-h=X6Up! z_NDZiiPcLT*>}{|A&%oqa5f|OG5*x$B~$_jcdU$nS&8>3cPJiSxX4tW?@&1AZdsIx zyNGyOnj%5_3vKkq$*^2$Yiv%#Ta=KDS|v=EST?mZz9Q;;es~@=iA%>O>S9@f`NS$| zVQx-}Se&?%qC(x0b-4KPY>|mktrgig&$vS?9|RT!Q(GiZ4br-cD;ACdPs78BHIa~T z`eMif(gjOehVFJ0)00_-fxy_~irQuK2iz zfR{T-H z;HmfZJykScO@uQQx)=_b5gw7471xr}fR+|!c3+ou&t7&WvKB4Ati7nup*u5IesLm0 zgO)x>>Bc}~Q_4^|A-wMu-GBKs!`lseLzE=Awss+8rVu_pG~Oi;<4jk21evOc%oEri z)C{#(W**Ae9@Q+mXQ)YbQCCV%v0oZKLupx7KSWN4Mz$wHsu}i3@;vEE*O*0{H+pE< zm${KImT(ih-&jCx+%{=%w9u+I5-AFFvt|Fp%Xm+OU>Q??wx>XNwa3&u8LW7Fmfg4* zM$?n?3k7#|pcoN!5J*))Da*@^Ck=`+l!6^RNIj7aaE60cD(rl}$xRO(lKF}9Q-YQr zarAo@*9e=#1{Px$(9Wc)&IvBBBW`TdL_cRQcDF%%6omn0>v0=;(7zBOA*{4Am=YK- zZQ{-~ttl7ztUfP+dF9KRIaeqofOn_@lpS*GQo1QLYRi*+%xJOgsUN_l;jZ%u9hpf{5h-4lE0Umrk<*HlT%ri6EP@@ocLndw99=l($U^Q=)Jc3s&_ zalRxzx{w-?JZ$RfKyM;pv9^(gEftQsFqW{|i(Cm?@{F!DNJ(+ZfNo&epBx+B_0*`> zJFHnmCa02=awV@6rx{DSSWHBT6Z4y^W4Nxh&_3Th zzHZWrD7in1Touk#dBkcrwWEe5RNn2Y5@V;-xt6q7iHyd4%Xv+zmva#`;J}Y`=e-ajRxstNF9~CzMJ3(2Q65g226lF&vildL!0V;zA_=v_KJqr= za=4T8FhtvJc_jpUjb%wah|q?;-1NlQUpzQc5=mycEk{GFg>N(!XxvL&dRdJW(u8JE;65k z>9<**2S(xg`04!X)7fvuK%10g{m#Bq-wrvU_M zA5ajYD}FS8$sq8$5Kr*di&d573os@YtuqVari2YwR!9C?I9~U0g;V8RNDc6!7-ln} zr)z8s%k)G)5@{fE^hABzh9DHcnoEeunU8#(Tzbbi`*1k(u5k7Vve!o#i5YVGh2A+? z!ZX)dy$8K5((}pIQ?bLQ&>wOGzTh8ndBu;w)=w7^EqEa~ta$CTKp4=5)0*#cRu3!{ z0AJ@$#s{;7&-{jQlI|A_yq**+@W6ZPm&m1>;RU;EJFyMc0%Gz_9HsO;gg){8&Fk?8 zYH-3wM1P553huBn#f-pYpn1Mm0SwBnPr9A!|_3b4_38DcNL1RJ&kS_CuUi4qbx9%-#GTbHchBbQQz|+_}6|_$@1`>8p=X z|N6TijQ{6qTl3+z5VCC?g#7o>v>@Ui`c%J%<~CS4A7b4@yA-w!0K6?@0L&>7?)Q+pMQia5|Z^LzyKU7T%--;-~!ptxDrygTBJ9{=Uygs#_RsUKZ zD?=zc!ve@r&9HsHsOH{-Tw4dCT^5~MneyeeCDwi$N!Xx6t9pD0zh6Ao1$4DYt&tZrgH$C|2rohrIzBpO=^ zw6RM6IASk(9?qIGquVX_GYq}2nys#y#PY8O=o7x*u`v+B;=KI;0%`F9e8C*(`}SI| zc35)Qd~5D4$j@L32=4CDO&!G4I)5G-4$OnodykRmRgX|@Y(gY_)$+5Ni~b~&*E1re zo&G;AU5HwL*YKi`q2Sbxzdt>AWTWMNm`8YWBiH4k2H#;g*t(P!XTPyhfa zOn$`Q29x<$XQJERQ&f3$YYmyA+C{g=G`R;*o{7E0R+Mb(z>w-d?ZS|KJ=*nEt-VvM z&&`E#4GMC1t6We$dU%Ppvs@yh$W_{{fA>CbP92a9Rs3P!<&NPJZ0kQUY(yL zJi=Y>d0-QLm5^>^SU#CQwb6yVsPNE95%SEN4}u|Z>Z7sdybM$i!}#sQ*Wjr0-9;X_ZT%>ZUF$MNPTKs@5bJK<IVIix zlp%9<%uCg;hWdh?(W9RJq5$kl*Prld#XFEzz~h_&^vFz4Fy{;W48100RkZDC-5aBE z@tHTK<-B&`)BICX#2|t!CJ@`K^yODc;_s(@=AW^a3OjSZnq`)?>c(*BRiVy&9#(6y z9)-oB@86qM32Q6cwra=J0gI^->@8DW(}OROu+6Iw?L!crOO*up#0Y90_tC3eH#(WA z54cxfC_8zMpq*UgY+s^D>v7eK@lt3L-Ap{!UI%)O#Z_vCY?a3dmTbBy%Fv!bXf$2c zpWzOOp*>|jyGTCrRLdB6f~wH(?dQDwrb)i8Fs}zEv><**6J#kiUILUV3d0n=e2tx% z@_(Owu(jm@v<~EduMkvc<=YLE=`q@zAge!#_9%X1nIk+P#=boD1tfbwq$1Cc$Midf zXn!Dov5((*K&NBypx8H@!S!-oT(9ZnQUqO(;5uq=Ik4M0kAtyWo7XEH-dqI`PtJ+) z=bNKJWQGAcAsp96F(pfmTGV#XT|V$m>65&-N?)yyCxWQF~_ipQ}u-T*+GbuZ-c=7f7Ay-zI7PD?y|3 zRp|ERIrT*WasoK~Ghv;jYVC-sisnr()plZPsV_{4OtX~2E=LIqtvavNtHc#{)iC&)Hsm+B1y0?mx-d9NY5gdJ<3v0F;UUWbct@3)LCW#oa^^9MiwLVqE2 zIxHnSrQX`_@Py^&{ZtSos$PM}HdBF&;SC#^%GXTVO4NPNqJ!>*8NkGyS*#fP>*M@z z8LXWX00Dq&yVd_Ex-2v+d|L7eJnXa4FV=^c6?dwclU0&t0HEk7O>L(>lR;*A9E;4a zPQznKgYmxJ4C5gpS~ou0Oxm@UA*=)z;grV@itRpEefxP_LETQQ-z-(d zPPHG6fB71s8u*N|4B_DdlF5iSU&{**S)3*!FG4`-*(%*^t^x5&8{Kc@5xG8PZruf* zLjmcy2Js+ShHU-D={H>&|0!p)GF0)^u8+Ld98zMC&9*LBxy5Q7L>x6$N0i!*g{oXY(7-AsgZpPmfJpjEF}`iV}>@l&YTlhZHZ=1 zH~T|{Z}z&-8y{pYx$$g>l#yn=LElL>y|QMS1UVa*Wn&f5VUZ zF3YCty(lYj2&`z}LsRLW3}iJ#8}}09&OyFy+{2Oq&L5=1axf+&{>EOPxS6_EDQJz; zC^v*~oRFh%+OvL9x}ApVwPkB!N^eqTC`BAt{^4E5&02tH{}Ot8>k6G4_VRkR!txtZ zUu(ce_{R;So}Yl@F$Xw!G7)fwRKbtAIeNR0VN+j-WpXK({5|ZWwA+8?dvLt>mf(-m zPgJ^b^N1IY8KqttFhBIhhB{~LNh_;g9nsBFoAU%jS8PtadhWNsc8TiF2dN@wT=67l zwtgn75qm2_K*g2NkFqqcP^Ba#h7^Yi#idqbcqDW`@x_8eA&aJNrxkC zTS6}GF@mo#Mb#b7_xp}ex!bf>Yl&Xt=)<@=kX-zI(=G75;`kAs61j-`6B!D4qV9CZ zo3C;9|9*1|&xS zIknpZ0P0Q6Gni3=un>txsG7fkbfz2vtboQ!T9*4!+UnnDFK>|AXfa%={kiSbhS-)- z45vbK!|YyQww{xr=?%lGL z#E;s9zYi+6k0!~Qq|636x!!xItmDeLTC}-wrCstGIFWDW5TF$RneZ+;aU>SZoTvyW zv#2r-gjYEevRK39MB``wy5!gW5VNXOpjJi(tS~?UEkY>C<_c|5flWOhnX{_aiKgg(@K>7^Ez{h!VlC_YWgTxU?57EwA4Db2+{u0P)~?q zqk@*hgK9JO>^Hs<2W6ZHPaPNCU?TciMP7C(@;be?aC1~O4ps7YHG~)3R_7`7D_D}m z=wqq^GBO1iT_o%E!(i=(Bv^en*r+Ub3s)uOtf*sc@rW+v^u$2j4zDpdGpV8#v6s#C6_0=eTMm^FG5xOmaW60I3xa4u)3HVw5w5t^xpveTzJ z+kFB|ZFwz)J3n=uBb<1ngO*%UO{iR9h;a-#Op>;Umm5W|In*@LWGL zXICv!TO-BN+xR1IkV1xLBwbHJy~UG_D=tA2nqyN-@SV&tc)W7@U%DCFE)QdE3wJ+x zW&I~4O8yLmy4<{dp4w!e6k7DxMAG+(S@8I0IN&9CXYWFKUXn4DOo6KzXSc6KZn|-% z$%jis%5VOTu;)R)5+GzyNb!iYvAOCl?Ih@?{YpiyRT$NCP?7Hiq9=xUxitCP#$xh|%GKJw}}syVicIoc1eOXVGtrs%F( z4i&evIyY@h^@;BmCB38%)j_NH@GU#kOlN z2WS_JH-mv|_c};7;>V9|#9l&(H>PL#9@pL{&=2r8U{8*p0Aed-HPps84lq5)7EnEq zEkrl;4vcH==LlayUqZ+?vS-B}eZOvm)Bq09=-vxoh*rl)Z2o7h9UuRs!T7Ml%Xj&6 z4(iW;-Hm&t=G*`H8nK066TN>wV*fG0lQ*;m{O5?hNYquwRYePa3M6ub2Es~c(LDAm zS)i7)$pys_v# zt9qF^_y#Tke0(b(x~4PvJf}?_S3~r(y+2Wr;T5RFf8Apgv|#H3oT6rb6u4q_jvw=h zFJr?q;h5g@7vM(n8#pD*hKiT5;}ki;u@}#}MYfp}i)XQpQuZHX6i|t4iqA(nM4Dp# zF})`$;EBAA{2pnFJ;~HIeoRo{E)I)BXW^uucU*g*@c|#Cg)+(&w4m5Rigro~+g+=T zUeJ8v&eh;QGb&1+_J`3`wx2z#KJLjfS(!kIlKruivb^hnJ6rgoo}Myn9^FHPsBhk% znU8Y*4Dj7KC@=h^=8UQ$6GN=RXze?`z{EtGklvBukGd#v!W$h-SBrC zE~}LGA6uu0gC84hYPJrIq}sUfw-_l33RmS?wx-%5hy^SaxcQ;^kHyUc8QmIHPwE-Y z?D$Vaphvjcb~6;lK4TrlH5)#IMML?)-hZS^#7lyX(f~|kE=WSr0e13{IpAkBT0y1`e8ujB>RJO1)V^e`wj++4*kU z8ZE-?(kfbRp(=GDpsq<$1Q7RLj8F{NiNy``#$S98ci>Z9TwJ^dN6_hP`7J;`2%6<0 z9D4-#dy+9Pd=jM4|8V9a%#NLqxA&}jZAo$O2AU*?IM$=1t-dGfs7^k!C_;=wGj`R zsssVu0aVwBc35wy=TM)Zm_6NOIntMYoxK610Y^_kP^Iz{CCu1(_;yH<{tyaajz1F4 zS}U)hHxcRmX|{ja!RvrpEBF%nS;hmdkz473QZ|w{W4zkcG+*gp8qK7Qa^+X_%A1tg z3pQ_@u0DM~hHD!)zwYnTH!Rmu-Wb`tN89~xyk4>0kXJ{T+rK)XHpdzIrf>9~f!;7v zf6uNtyWwJPMb&I@-~YuH^ynwM@ygdkEd48paR1v^zVUyo4*#(S%^>yVP3++EWuo_A z3&#Hf##SZRegR|AhH}=6<+KXU+D0W#F1svjZ&>mtRYq_XyMQBZ%j#n zgE89_H%$nlqK_bN3cJ6Bk})r%m?g6vy^n%DFQu~I-X7k;d&mh?>--nLm$&NVore#w zp~J(d7qJ%UBsk~}!lR$TRWo@^EBmltN5cttn9=%k9(0*aC==W!Lcyp)praDN^dke2 z7&>Ogq!V-0UU;EP_7EjNL=2}5XR`kiVeZQ8-%I0-AQM5bV`>|w3T1$&JA0T+{Cx6&1 zln+yS#i`%Wpbfr*11S2L4XGrO zYBLMOOAj@O7MR~6QK&TraRwpTN5aV(YlDW-a&jXG#Bghv@}bdcWy^ErJrXXtRm7iOsC{XYB>r#xMbyy!%eKwd#zf7~%K1Mr^+ktvk++{N%x}*hj zyyQFTI-ELMeueTSLeNc|bygHw?u`!M6IOj?sUw|9tu1~}p^SoIyRZZ((Kxci37MFJ ztG`ayL05*u9I9U~k^8c&n?bmcGUnHE{~JupQJow~wg*WQ*K)#fxRD#@O_Pf@<^7jFlaEbfZv zcrVC~N6>c;Bd*ac4s^pYV~$Uhs-TfT%Xc{zjoq{br+&Ek|xuY7p{jf&elX@9(%Q2^;xY9LNwSBjb48K)I631n&ZvGgy z1rUTRyX6zO?0Emn@J?9olM?)Ds&8NPXX*c2bN$zV7Y72YO#U&G_~#0&O8Gxrv-rSZ z+j|~CRP=Un(M#K`o4S6;!61?eNw&>|dTX`NteUoCxe(pxYo==y%-%N$jB&D5!pOZ2 z27bxQxLmKk3>2m&t^~GxKvo9Ek-+!+DRU`_X?BIMn0QSN;{x4MKU6dZf&)qWXWrjNf5c`qTSt~BA94nZ%$T3_lK_k& zb(EJA>nyd3Epejr5v@<1Ci1G@WRwo*xuQl2u9(5Y1~uv)k|DwspIlOXXMx=g$4| z+eqdK62HUJ9d@k+4(&Zt; zR$mpH82h4?6KA6?V8m$-$#ulq8H}P z-R;cG=&p@aeWpm%@@+YN&`YOeJ-Aa9{egd3GP>V^be*sK=4&+9pBMr}WPwv>smyW; z5`15_!w2V}luqG0|CZc)JO)t_TL>@?#;+mD*X2JBQstk*jMSAlkst6^*BX_fft+cqPrm(&7V1_M zHjVKstPuX+!-^)r&_T}D#>xC&!pfHwR;3jCYHJu1;uZxW0Sbf6Km?^fc%W}g0Mg7j zv&|5vp(}<9`Kx|5IHI>NAz3mJ)Qu@t>S^`qiH!w+_ol!%xLU&8NSkknx`f*NF5qvh z9!tNSL5)ZiWJoG@CuG4-fhLD1Nou15N*$Qyg>lU-7SvhvXk`+9r@z_PbfFVH-|Kui zCdkKztG{XRGm9Vei)vM@>@>RKn*lXSH%>Hy9ZOIz8taLM;~KeHc1AdpFqyIGDThJ= z8a`V6AeVlf!{}7c>c(pX_%yOujHFm!5{phG>0ohgmR)LhLm;tU_ujD%)&!37s~WCZ?_43rrKI@1yum13oLnAhNV~FvKnQ)q0JF@H{b2ZJnAq4!^^sha@G8MC7 z537-&&eq`36Iou_C5NjvomOPr3l-~K3zh^7@+#RWgD&>uvjYnr8lCAy1m>5>Kdknx zeidd8Au*%kMFSbEBj^Bp=II}UZ?oO zo=-t}I+bcUaN_?WWjeF*f1c2tt-o!)ETuQ6KPIz-SBi0#H*?_p1kZ;_n-!%oKoG8* zSltKeZR!04v(QdIi{DEv!O0V}wl%LPvwmu1jWN@RjJ>L;AtIuiyeNseqA3~dukZA( zF_U9aWnXG(dOqW|+ZMmX;UX)0dyM*H10^^pr$O`6iMg6!IN` zSf%DB_rw$7ul+aTR7l??+D$$iA%%&kM6+;_Z%qa(mN{4#5;#n4EH-df2uef2^L5cT zMI*&&MN;W;w;x)TTK=y7SfYzxUlEa}aZkpL5wfEsP)<+tTTf|#6KU_WA;{%Y67OfANe~^3N`ov5faOLCT-!vFyT?nvzfhvhgMfe#_Pjdje6M@d z8zcwUPwSmYp42WfFw!)xZ7J~|4hrxKSK!q_{!R+2)JTJuME!!gBgu)Jyo-*fBd0{% zlQoWE;;`Hydz#TV`w^H39vdeGvrMXNC{EsV*P19l*rURi_FD-Z9dM}Rc~o+L%B@tG zC_7N{h*|P@N1W~ER{mU|{6gVEeucCjezU7=t&*w~MDy^d7X`c1SAQ?X@2`N(LAtrM zyo@WO;>Hecq=>SD6-`09wm$@sh?Csd4$)q;&A#_toH50n7w>jHfQc$L_S6-;s7th! zBis(S%BdQ^gj%Da-O|=96+R56JI4(?s;8P5`(8kS64wSAm#(4i6N4sq%n1E3LDyNA zEtpS)0qqbG?&;FTq5c#?LKJ^?Q!gYBSaGwA$ue0OYT})kB`Ye|$~N5X&`dwxu~vo; zT5#Nd=r$8H;YxuWx*=nuR>UtMLe~;qruL&Blpe+%QpWfh=DjPp^7*9uO;f7+&SR~g zFTRDLCr>RFq@{juzJ7Y8#7YkJfm$zbkEcl;Su?YYi^|d8M(((B!Nz*yd;)(ou9_8c za|a3X!$&?gQa&q>wSdBj$#xvuRHgeK`-=()O}xRLL`c308`JYhtG?=%xDUI2YarUy z9C}U38?5lxnIv1j0>(uoKD1q|1DXNj24%x|P)4i+?kUubtEmU~x;RvQJBe6t5Keus zMeGeBrP}aTmwb8mNz@HqPeS9isYG+&Ok?1Ic9+UcI53BWYhquDe45!keUwEu+w2xE z0W-B@WFJ7z(2EK!qr8ZFIE6|E6(Rx*iAJ|Yx)JY?8n{XT?uTcHZH&j-?3^B5^B{Xq z?<2}?27UcQ&M1^DzsW6fLQI-h-@Y<=4%1(M=#Ou<`4vM8W`y|SWg)HTw&*t@tuaCu z5+hX$AEJ0YgkhYa0*7h8pz^_nlY^o7<@o$63 z`AipMn|Aq#HzS_Ho|SK*fP4mG@obTZq1WF#!WHh+b1-Qigpd(;LqiK9J!A($t__h9 zccw+X0!IsE35;(!H)wXJZ(xuSSw1NIXx+m{=Vh}EZ)tCOwXe02bM{W}B44fDLKZuI z^wsQe5O<9@zQU3-FuA9VGNiqQkJhBU1&`A3KE_-Zh1SGVqFkqk);JF1%5Ltv^P;>C z*O(x034^|}cZobD><4l@Jy|6i!QYY%A*Ee~KNED>4kC)aAqmv)4ikI(=gh@r;H7c- zHo{WRWiXb+hnB^cDNvamN|UvyltWk+snx5VinSBis&L($X-C`5A-7t>2>(Hb8^2D( z9NTZN4!U%^jSAHBbP{R^*_&+qhPrbRztPCFE;sIWN+b(A{D2u$Ntd%=PFKZ5g%v&S zXasZrAmwF(p0Zfn{n~g&Ir$A<{Su}*@S|gdVfn3d`^jE-3D%_xMy#4_z%|bQFOSH=L?J~HOjQL9Et3KqDu_9mGbX$ zC${yG2`6KA?*<8udrJAuJV+H%Bh$@VMO8kII#E%YZ;9iiwY&*EQpC2wJ68G67e>Ra zW6a~tqUM@**Jx%&jG}ign`c_h3L3J0;;N&VnYAxeQS3ufQ(jAJG(E0+hsB;_!Kve+lNz zFZy(><*Pt7Mqq7=q+tkIOS&0-f`0Opc4SplySs@4H;VGn6UTVKz1^2xOEF0*D6A+5+gYa6fVU(5$!{Xm@oq;qKHyFZiIWPqC_TVumig& z5G&xg9CftPopRuw#!6zXB@BI6*=<;K-Ki<5Buf6qR6n~ot`T8`zIO!yP8a|H-r@UX z!C18{>VV{V7|nWWEiL2DUjyw4O>X202|DZeej1!e2Z&+AX2WHfb5WfgEwcb)x``m% zZPst8AVGFCZ%9awigZwq%D;LPK9Lup;2#K;-`PdpUCOuOln9v@t_oj%Zqs*qb1Yu@ zdBQvM2*f_8KzidU>OuEIen6Fejtkr))4XmWxs zb;e4hUCa^H?u%Sv?I-aZM*6n;y~LEGjNvEI_Zr4H=?z<3RQ;@q>S+WkpX+a%vy>_d zy5*B9V);y##$Tk~yQgMxO&exPXUX?el6ayKHY<87BW+v#q&RjGmq-d~r`^M%lM**d z%k!=Ol6iwR1pJs}O@B(X*_3Afnfo?S=>AAM!7}3EQGd)Oo#4DcS}pDIAs4kjn9n zFfzKZ4rOs4Ienr+UKDIY+4fFfor=a_5SI#P5UgWaZ7S?tDPs=ouwq^+Y>`T;JM`ge zpdqI>xO|TyrKeS#>9Gtprib^Kus-9>0O#0Zs3V|zMt_PLhg{Jj_CaQC0$np?F%8E# z#@928U-LGvY;s9Ux+!t0FfrGOtmsOaJ0M;^tCv9fNVzRFhUuwK%((X2HM`l6jg;l&Z(M2M&c^wE!rnE()4bh9c^o(d2_>4++}ake(b2 zngVPX;DSozIdWk-{+{m2Z#`z`Q&BUoy;QgF1g$A=J@l6wI^YJU`E@DQM`D*g-8jPo zSs|Wx=sWg-(@|Xywb-*}Kur+n(qhaMB61_0ZFO|2SXzZ4{U~JbS37*K8dgv*8I{US zEt-t@>j601`ei4AcjGLtQ`JiQmL zBV0OG3FDadOXm-iI~juK(^y{3hl)c?_^Gv-K% zVFDyT2<{%-U4y&3dvJGm5_oY4?oQC)?jGC%1a}YaZr|q2Wagf;zPr|4ncq;myQ{0J zpZ(qzvu)t(+n_a?(e!RRww)B#{BMikYXT>SDt;N#F(Rw!ul+`5F8(YIDimJ3P>cph{WEg0YIxao?Lh+}F!K+WB2} zTSozNgL>7ClMLJqeQC^)q`MCOB#9z}R|5`fQew4K7`!QxHfBVPggQraDT}Ew4&@n= zWh~L!5{bTyBX+uE+YhU z=dBdqT)=g3$8rV^R!a{uP;J{+re*}8lyK@^neb}%W)*alta*Lxb8ihjcfc}L?!hwB z`Lgx41#1$Qvke+YKny>*qlO-h8jeU_EcK_o*t-)5yj5q&s1?S<0WeRMPlT!+VD3Rg z=}vv5Kec~mU*+;UC&#MOk{Nrk-O#-8LWf%k;%p1v>yX9zn8`N27krQr6CHuK!#P1!Tu~r!0{6;T1+zc#X(C&?1 zwtlWX*9HUIphkSeDhG%+I-kK)Fs-vWF5qJ?6sxm_3CR5G=)X+{ZW zwYKxN*M|62%mK+)kOls)MKaH5wkMmZDF-@p8>3fM1btjTqD_7rJ3}2Q;5GQ;x(SFv zNghzdc7A5np4&paC;Kw6$sfQvVUE4C{@OhFDU0|Uk?=!?BT5deu`rDo-h0^Pd>B<@ z3#KRNk9+kHTX#OmF2y>9OBSx@rB^xI4v$R5+RzAH?lj>xNj82nRkCeoAGJw08$ho- zrYGa?V*GPs%UZd9*-IPh_<8L|Yc0_U{ED;vrM#qKlaTz~qWW9S16a1E2>tc>zyd!) zrtc5FW?g9xM~VW^&b-{oOGNDih=>P?wKJSwp(NbEyI>s_G0a-TJ6~GFRwXkB+~a4+ zIm)l@hDU8Zz0r4ETX>7lvCGG{AIfHSDmcnXEu4KK3rF1X#+R~(U55D)$%ZRqz*VE&hnFwsiUD|wO0`|2n~zDMmLK%>h%m_ z+T@%WmQGzAx_L(N?oCUEgx(@~Na6y6t+*P2q+h2bf+tVwsRt%GvH8nK44Z8e zgLsI#?D?*?@X!G^$P$9P!f>^A#NA%SzI44P!K)FB>pA5m3z-y#*+BwwTp!}{9Xnu# zTc`BX)boOlbGf952zbBoZdb19PxeV!-Oh@p52doTdZfyGlAv@g*T{psh-qD(+OeUR zInpOiDfJu2EqN75xxy%jXjt(vBiqx4emJLORu14OZ0Zc?!{M8_m*q)QsTG@3N6BI@ zM!wEx-DaCjFx0d>)cvg49%Z@ZL;Q>q5Sb`0haZO3ft-+xF2Cmb@(LOzh?Qm zy8R8bSD>t~wJe9~>eRIQ?Kq?#yQ5iTMTl@Y(Wnb@;l2t-_9As%tF0Jup`^3awxSE+ zQ9aR4%Fgq>_pKFfi z0M6XU8H!STv8vn@mP_!SpR|K!+S>}!En);rtI*y(HeYRlTsd<9RKZ1wN~k6mmhPYARQK);3;C{xxi4X20*l=Znn~B8a3u@Ja5$htZRCL!A%zj3b2A z=l{pgAYT%2r=RJ5MgA{1Gyld}P7YfD^_hoXwI5kPSj#$0B?Sy$)tXyh8vEi?_NlxS zitVIvs9|08$S{9b-zi+Sh}xeoa9#1h$o1hB$C^w#ygT$1qc-OoRucup<3zu?E!7 zQ~W1n1978mNUEu=)66PW)tK-wwc98gb2HRfO|tvosVzwE%kl4~*Yu-zG{y4_8q;dP z2XZ?f)lD>w%SP+oOt#dN3hJ)=VP^b}v`~%nnbrwhFa68ng;)$JAS|g+S)8`xnrB?bpjw$3vN5E~hntA4;Ya zdOYy1D-}#nKL-Wjv%!3h7Y%Wuh)_v>`r#{*#SRjspPYs{3?#xEYTQrV_HZDxg;8-W zv9j#zCI*N0`DkhGQBx_E<{k2abeP$J@;3nKP-dQFli=k{pZHcmr->SuW?YAl z=3DU4^z+rtu7My)dK?MC!*t0SJHu`-4i2PP^Wa~3jO$KWK;NcP%8o58PnXW!#kYF} zTkf1@qPn|T>eM&a9~9ZqM}+TbcIW0tOR!+Z*rG9Hd5Dy7TnMUQwxkejO!wD$K!E5> z-BjXykXE<8=5oz4GG)X@Qk{RsMBq#BpLOOEGdmMic2|>}{S@S$VObirClFN+e*1#7 zLDR5ykTq$3>RPpJ)(MoP{x0~|aiP~c|Bt?nj++JHo+5>4B}}DRoyXuSq|>aS1m|fj z9KQc3YC4OUsi^;@r~#M7O5f2jUU5hkhY^*B9R>?oM)>`ff|+}+fTZ7deo|Edag})y zc`EdPEEiLgMLiZbs)Z0?-~5#yBCrOXfp`Qem!n2XMOFp_ZS*USnR32I1Yie-%^)nA zM1}8?SoD>eCiWxU)ClD;D%26kr~KH($llYQa#@Cgd8uamAQuRv$W7^s$G)9D5A#HM zr^G?FEAA4M|6@ciUv+PCVoGf&Q}{(IRXRw{zn8vnLnh>Cy(Mc-{*`Nf;EH)BZ@%>f z8fs*BNee45LE5It z?ru`+7tLAnDE(_hvpq9rXloDraMt`-@$V6h_J1GI#}DAOw$YY5M^VoaUAJ<%lrMnq<4kq(fpnu2d;6Ts6B}A2a84smam)d z7urF7TuOCyTZZvqf$kr{IKjS!|csF1}uTle3-;^O@mG8l}zX!h%Pm;5aycAX&uAlcs(I19w!=2 z)xW2-R|(W?M=@Br!p0eyO!-xYyf+a_*L1EtNSlY&k zli}m@Sc9(z9r27snqE44PDk^)J{E7jds&Y(l?v6r+1Q7CmF?--ZN^fzl?QOv@>SN> zm3|!}-v^Z4fi;ac3;52M=ur{S7KRMT%gt>tOZ96$H3=u4X*4!LKB2!@X*l@|>-*_BKb5Y> z(Uwhc$w+UzW!r1#_QvTMOb>SdwX|MaFgJRr`gg_Xo_hRccQ-i-E%UhGrU`Yl{merl zaBtE$%}RiD&!CO-%Nnt-w|}gAejwglO0JrcK_GyBe`8&mXD-sEX%4fH&d>%O_}7?T z{XM2}MWB7OpAt%yohFp)Q{FWe)HvLO(&ulEF6jtoLOo;TZ_4g88$Ro|jeo%CMiREu z3vW1}*uN4T6#wP(V_Nnq&-};{Tq)BTvqyFJiG6o;Jp+Njm|#-y%i3Q^w{9eZmZv^9=ZC~>%jw95EY*e zoAQ$nh9>nuk~qz=e!JZ7#M}$BFvr9UbA#v&M$jELzaD_F${asYjYALei2a;=&c3t5 zwX^ux+W5M{f8?#nNlhqIhRP`}wFxaIV4cgF+fG<-8t@j;uiyHa!vhJ(O3gm zw;N}A`xiFi6wQvt`wM@qS6sk)B?|)9t7Vss3oRC4y;2t@ED}}VZQxc(z-SuXx z9bmm`n_Q3bl%%ego3)lcUOqsQJglLIiI23%3c$hI?mm(aCnwCg?hOtrtFp1pkp|%l z;~j4Dr%dx=Ft`?Okkm>d@QS-;TzTI>dkt^o1;G)NpUn5+{KQ(W*_v8GLMJE!Rq$9- z2v$aaPJagGG>)kfCoeG#JLia)d{yLj&jM2!Drr;Io-f!v6yIhu?6^$^E8fbAhiRUx~be-7RB;CBqko2`R$2 z_=fvE#SW)l2&)Qs>d731HU0(Agc+-Ap%e9zULErze)T4Y9soABltDe9GKJI(O)))tL=`KE~gW9 z1gFxe7Z%S?g2DV&rZiJzsb0EwOQKQYsXBH6GE!ukWW3TuuS9-d?7P!}DbqPhVR9MA z_7Osac-5oelN{Do&slP1&N+T$i{mMISazmiu~t~d3{db#Y~nHxfPIHBp?P4s*2W5% z&x>hyJXogZCaZYds5YHjt#*s88or9;vAUb>s*5ByV`?!UvcyMBI085sJAid#yuZ%P z3)VX?SeMi^t>ID3X}o&_+M4F;-H^jzvOhdQ1=zRBay&A?z5%SMb3G6@OZjeUmok+={vaY(0el+qZMK(jJRdUfq+zd;fKq^koQau>G@06(>SpQo2-VC)M z^oV=K6k1MUAGQC>SAQ!%rnYi9l{_J(U*)X z#8sS^*3@YwH*VYuTZh3k`E9Vh(4m2%BJJ;_1CVq>t`fWm*gMeD05mkrsF6bC&a~tFiJGust`e z4(R;1Nxjmq@J+m1&KSigG|c0~=O@(T_a3*&KG(sZD0~A%oLZ5ISYol#u+}kld9(`e}PlcK0Xy zw+L8pisU2BcTnKa0!o(VIl?c3(O$?Q^l@f^OcGl9Xuchh!X1mpfx+XMZxHf>&-VHe zJAN@sv7CCROBKF;{+XQp^^I_EnF7z|InPt;LDGltt{skE4fu`ls1H5%e&cT(v8pwj z-%Pb+_UjP(%2U5aptu(I_TV>uKB8|CzB)_7HN#L?|CJq{=r&T=F}R#%c!2-+royCv z*%1L?tM!N2GX5Le=w=&0y9s=Tute{l*TxO@+t7*wr>M$#L#P$y7u+v`OEc%ds7q$_ z!-OpbE2;$5=T*DFGNEG1cZW47yZCwuQ+M=mG)k6afEZ?u`_egjx_U!oeNNOU(c=j# zpfFy`I#^E*drMf}h)jcPmnTBPchLmtA*FsOtcXK4@+knN&EfWOY-X@N>>c40vSJb& zp++ejv@AKcgWz^6ri2OEphGL4iPE-QIP{};-mLpav&ob&!dT*VaySC}DJyoEMs_A> zV%p7LcI=NWz(qlxm`oj-L9EGd_p|yDfms^Za$kf(<++KQ1LwSh#j?-EsqaafTl_U;16#gz0)H>^I_+G8MUic-`dsldI4~4R ze2CnY$2?&rxh2UXotm=z?(bjuzUP%F8TqyuRX;B9N`$*Uu9h`zF6@Hm?+9~Q{S?dw z5U&11gnz5B(PO|OkJ#G|9i-4}3db5wSkoKr zX$gDn_k!=9L!Jl?0@niMop_Q}AAqk+q6ma;zfQW`BXDx0p2x- zzQM-}Qf3I_#!AQGKpIR;q}Rvs1_fM#sUww`jB^N_6DPcCrKRZEX99r=B>@M=4(F^# zjp`Q_kYnAjZgdGeC3;ic*t#ew`d-NI8kvZ27hs7S;Jeh}`<9@f3Rc`ak+rO;a=u~hbCvz?Fyc|!0bWJ+2JB9@)e-U3LbhGrn zXi_N;#StFf`Hz~t%)^r}kIhDLuTj_L)vGk%Q4*pK26`o=OjV`gjw7h$yPuyS8l^V* zSS2YNC~s!4mMnY1tDusV`z@ZC&XgRO2Batx1*aZ8gD}Kp@s++VQ=eEn$@fULR4Lb` zbA83h)A~E=uZt+FzXGUN{UPeVHQ8t>V6g}OnqAe0zwwS?8|~E^C3+o3cnM8J)DMzq z-01hO8IYSz7L!C{GYXrnL&7$}=1e=u5YRvHt2TMp2SV55cJJ|Q=a@D~ zei=zINk_19`4@5;!>)bT_v~L%CT|8E9G_$0G#D6&;IHBg)b0gS(A zQTVzr?3{1sh{%Lv<-r;fHv+@n!0z5z=zu3(Cg%`9w(pt+c;Vz(TsGk29DF5=_0S z!DRk$(mZkUO085jOx}&On`2xM{!2rnG!cdhLgbpe8`xj%ocr_ybpd3Oq4`55v~!zttXGib)TC+x!!PZe&6Kli`LPD3m?l(LtJ%*NPqo>} zRKkx-7Vkg@aj@3L`wQH(WU;p6oj7r_}&>cs5ypc+XYBJiW z%JbHeq8`J^6o^0fhZ!CKjEeC+N<`3cht;zz;e1ZR^nWu3;3Z*F)X8C>!6^aom#pNIs6tqDo;uo)x z$y-J@g$Y}9u5Awa9(M8S4je79ZZ8|51LhRkI}9DhIb9C}hVH=F=X~U-AbGC3II|tB z=Cmag8U34Ea1RwOO{3eULBOhI8D!YKws~~FYilB;hfESIN)cRa{@6Ma{chOf(M-gO zj9?>mE`t`x;OiVYhd$FPD4E@WA{mQvs5IS>&7`wGm}6EoH2nbDx=D%A!~ydncTK2_ zQwvYZ!0Aux=7RJkAV$*g|k9iko@b+%(!5cc0PZSh`v=pkP0zfgX`W#f{(u(YHe zC0%G<-$uI2wrOZ;9>RbRWSgm_jGZtsFytlaq2f%;DhUhSQVC4H7QzSZilHNcf~m_n zn9NwW5aw>*RjwPP)O~4Rs#=tMQJVw>gA2IbbEoKZJpJCS#C={75{O_}+)} zFH`@};gcqd!(J>@BFs@S@`MJ(uguDq@&X>Iow;;LitE?83yxkQrGD!UaOe zL)7bYO?Ns64okD|CK9dX^!k&_Wi*_@-BAWtmm(rJawJ}R+Uh=?FZ=qeJl()#eO42Z z&Q|4jgAUqvu44CY-SAgm`+>6 z+Uv;PqOCh;!&E@;kB-@=(C@VqD&%Iu)8Butfw3%^8A2bYl$y74%f0%^FLr%+lzqmB zgYv*qeZESp-#A~yvmSB|{JSvW8v-4|=%~9@YOT_a|4tj@g2n5}6CIicnjkE(ei}#n z>2VyjA*$3%+pMN*yvFi=-@-&cuE8aNx%Q_pwSHNAN5dZXIPDAhUhs?$RGd7viwZaY zI4kXk=3jqt)LyQD|745)EzCv_*nR1H7-{)HN`%x*N z%M|@gXXf~6wpV1uj2sy+gX0_x)lW6VEpU&nz^$DtX^dh!>iXqyW#^%-@!1^Eyz}XB4}5kBz{9bDJ&T6#+7sa6>Ieov*EZ?Mff-ZrUvhQzU3n(lD#(`xU(i=fmoV3!e zcQUgyH*UX${pnN|bsKV!XfmgzXaa+F?eaNMTn$~$G?X+BXw;o=GPI~Pc}!h$PX1w= zvV7vb`|>}4{9BIwZy~FzW0(a@t3!g^%!=(NQ}bm|sxRficeCYDY&VV?-f5VQ3_Hd8 z(4atk_b2#-LePz%9}d(p%D?K^YRs1u!o zo7tX>$bwmNrgx^6D|}w1U85)ASi2c99*Y9Y8smV%fFL zvxKDkl*y)_*<;Tb47^0CR~>6Le7MHZ(yW*H?TMMWa|BjMi~#0xyTUn=dRl{8rjYe{ z9@m>=CwjMTS&JO^KDtL6j;6cTzQ=^LL{m^dCBia6cTQ8meY_2_Hk~?-PTpsx?51A1 zDd3yuIaGU=x^c-E8oPYuxBwc$>1>SYW!h~>{L0^FF=sc+Stc3I*fK>>x3Gd=zlN93PO!E>?VOdk*Ed zg|5|o(Zt|Hn4)0`Z1*D0k7rClRa~ldjT>t)uOQpYYgt-fbgFFT;K}lFW$Xe2B$+hH zIHd_u`)>u+1Uaq+D@OEc=`16%iAqP*rHilh%UONhJjE$4C!$G|m%mk_+Zd?BLy;SB zN+Gqfh^E2EIo~rI5?yfYS(5inI3ZwmOjzQP7pZu7Q7qYWo~;p|RE8JeWAkPJ88-A3 z?d6t}lxENrR}`Ylwu}fxU6n!NbMfa@nQBTrZ^Dw=yF;#Tn|eSHVz1OUO)OK+=%DFP z&g?`q&Ry*-5QGrPuq2)NCJ7T}cE+GD{C@m0 z`77j1sOFT}+v6!&-dSvrt)6F*^wa8fO+tR^TgyDsntO>W!VK{ZWuo;;r6M2? z!7*@QbSLz1xB2w|{abAQ+>za&PG0;1F*l4Uf-M|$ybW6%Xr7vNqALP!Na0ysI-hdg z+wi-@M~!+e)_lRp^$ZsW5*gFTS2Wx+5-A#aV)S`jgMqwWki6*jW8L|Y_in2jgaG6v zCn?rlXRPAT3)T5VaosJmJSSe;9zin8 z!BoeRF#9JAV~(Y0nmo{Fs_!#d8VIC@6E)`bKZdwt^4z@*l~oA!8)A-Y&pu;&>+>A% zqa*P#N(qZSw1isSSXaRR_%*MTl_e_uHkO#cKQWgU{ytX-V9pFY@$v^=%fAs%SJZjo zV?6Z>F1t;jb*Q&QM5;^E-{VD*1!3p|zQ?d}7m}U;+#Coyv2jh3J)!x6Lg+16I$eEZ z35T5+%@uX`q_=(28XcJ&j~*It+~M``J_jMAb1(MrV1(i0(s(9yLSRtihpM}r{^+5~ zE^(t$U)(A}4)L0qC;h>zUzaSNm;6%)9l>qw_Vjqd^mzz9yMZ?M2NxuHp$p8di9;7m z9(xXXGJY=EQO($FQ~(6+`U$I20p9e=XjamI?rcoTwc9*qu?1{iIrSW4EJHo{R}KZi zD^Bf9GIBcn+a#g6o|MSGLc~3VGe+DqP zQ_yPH?XtLwz@pELNzCKs+e^HZB>XKFQFVl#rgN|z5`o?UqBt-At;-Mu(H=9Aw zZ{Ag&_8Ow`qum$jnCHMQ{n7_l5Gw7u<|g%lot@)%7ufoUv5ha4*CU!tT<}#2EoaQh zt&o3j@XH0;Lcsv`a(_nDn10(G(JL`&h$)#7y#gpRmNNCdWdftGKbwjz)k?}~c6$hCQBcgxZ9F_yr(!BvdExr0GoZq_p_!#Dh z@1)c{wKxI(3X(o9rue;yrlSm|4-ou?Y|2AL(;-Bv#N% z;z{if!Tpcl@q3f5n`5&w6MTLTj;d)dPV`g7Lr@a6=&g9JxZg7F8K7+^B*ixsKW?)d-k%Te3}gcO2}`BQcW~8E^&U+m{Aq9Q zD+SUxxEa45p|S}1vx@5kAn~NACSjOdBZN-wmHoK2eZ4cVg_ivTU2ztk@MAFojILS) zhyJcVrj!ZFVZ%6|F(oVt^}3y!rOP-*Z=GpR(t!PJY(iwCjk6|b%atiK9?79)?2w%G z?3AU08S)kW`;$}djyJlHZXq&)%!;P^gSBYm(z@!MGMJRfqfH1<7&|RoGPE=0E-8?R zei?Dvu7)P>*g^3hmXPX>3uI;uzACT<%W6F6l6qua1xOzpMjr%v7N4#BfaDv&<>4SK znen*%P{(1J&Zb{_HMZT^r-VPJ`Z%kx)|ff`rZ@*|rAH)jt6~#6*^({+x7EJ3GS)%aZE#(1z_M#OWv~G!_tU3p( z(7=H>m(8`T`e;rxn#P5wCEF2;OGOG#}+Bf4LhL($?c@={e(`7gAagIuR`Gim&TI_0`lPeYv>P%+W zOPuFHdD|aTA2(1kW@YzdtGgWeG~U(uz4gM_(NXR4BStN2RClh~un^f&xd;Dw)t2q_ z<*d(zr{(Je9?JuP?+JCNN^f_F0h}`i56cO(9NudH<+P+{xW6}RAyx(hKLF)F)VrDf z<8#Q+3jzM+%A_uxv-v)A8=t{A+xi38HQku30hcC!neEixC{MBcx z&%(YhCJ1-+T8%o6Cbu|_a@i7$(Sxa0LG7z#^0xj~d}_;;&~NmmU1 z8VHU_8%!ip^+5nH~+X{!jn?9$nM`NTBO za;XeGdzVH?Xw`Nf`x9<0$XpMpZ$IuMFLIoot)?+(7NS2hd{41Zfguhc81b(rLy&HC zvyibnARwcb$-*g%;9-->oqe=v9XjeF1moG`XDFH^WIx=(0wKDF0kePDd1XM z78}C4hsJ@8v)>SLBY{bE%zWk7?LTI)^7kq97r|5G58L!_8$&_orFePRHFRWVQp{8= zrc_S}Vb)_Nf&_)5rh-sKPKf<%Czlxix*Rbg z(Re+a`P}XUVWwZ|$2hEd^2QO(E$O;))iCjv$`;=Z8S^H&TWB~EH?fvv#=H80N64|6;z9#Mm4^qP!o3v^y{Ac6Jd7|Dm$F$WF zkc23O*9J$=FJ{Nr)#DE^79Q*rkNmeK8gNGnyZApJd|gkldnh+9LZ4`nd+Oxuje+%5lww2_a-j4A=lc!K@N| zqCi{=Vz*SAe0!tx_~ZdUf?^Cb1fE3QJ++gt;OdajER3BqpW~4YzT?N8mCGIiFdVm! z{P07sTZBu&0G*^inwNXGz=*grJWuw({5_l8G4Hr zOSBT*foXO7J$|$L;6IRKJ;=4bB zJgQebx)An@&`5u#AkYQzA-J1ymU{1hHf(2DGW+dRxl4&3R#h797MJG5M;1qBtjHkRbr1cPSVN}VYMDx|)t3Q=)mVS}mWtv9c`ahbo``Us~`2Wz9|6^DD zN6v1to|{O}4E~f8D71jccG#H|7ZnvRBmoCiL$cVrWS?%5Je9ml=7r7ckHL!j0G7e+ ze$tI;s>MiRXHR#0z1;G!)B5c7?EMJGPcZgPU9dP5%1mudZ=|Ow3{FtUkyM>&O)^Zm z4Xc&p2V*vR?q5O}DRUd@Gww%k+=nKzxT^rv?N9OtPUMi};>5CrX`$*&%t{HK8JrpkJU{aD>`=eWRN&rcZNZ^h=oXyRF0lb1VOLK`<># zf`pjFNgflaQJldK@R%7bzwp#F^M-ukizAZpqd0?5%p1uRIGDpE6zFCfDRS)b)K(JuM9@vqNU31sG z%jUJL1&!4L?Q;SSe7CnrLbO{hPTpn&VY$)ida8HgAlROMslX>Q^%(@DZ1umSY%(T1 zxnti}j-5^*Sl~T-BM#QN|J#ik=3&IsKXji5afansA;nkS1T&*8y3Vs(L?)0nm~R_h zbLm%(oAc`bwIiWpBhWRkyy0d!+^($csN_qcRTe`ki%?=4(w|3Ws<)!ix-+uWFs{AC z`RPy=bszAcU%V$n!Tver{uthuh|9eNlTuZ_c3c3iTD?xW%44+9a1Z_O=*nW!SiF?) zA37CG|8XjQCx+0`6(J;U3&8W_FQbH2)a(S`3JNaZ`|;y`Qob-@Bg3R+t4fdOxEhYf zSOfMWgda+o&}%Yagq^Q{FGs%D^{!wEC$*qg;fV~N2av&~1Crnwg($T2N#wM;)QhHc z&&fZtK@(T1PTOy3avz3%Zr{P6Wz;kTb*dEQY7wIr*P;^!J8K~Zx*Cbu&#-3fr}lAg zD9jA7XtD=pT1)P$-uJ{6%l=fBbpn}}utFQQE5}$+tC&hP5HGSlX4wU-AA2P8D6GLy ztv!pi2)Bh|9#VeEJ}~_BmUlh2AZ7;EgFF!bA1=k2O1XL#0C43WPT&8y6rDDc{6IJe zND;uNIG;-vBwo6GvWFvUh=0(F4m2%+ZauoxuO&ucO-MQ2+-&K3woQCJX{FqflN5bgg9*G4~yx?aKrX zDhlsj54^O|#fn)St%AdEAuFkeO1GTWQ!{UcF1_OG@sEXxlrBFX{Bks#IrL_ZAFV8c zH8sA%2xcz9O1ti2+#>m`?H=amrLT65)8{k$`UL+}|L5wgcFo$Xbpb+^(D`e>hqN@o zGY6`zA5QI$osLEY;qmUra6mH1%mA&PcFKBZ^g;CSAAivm`!q1`0f5d1fc^)P=I=#G zK}HG}2=Q)x`BL_E|1KouI|8KZ^(qCv8i>JTR_`Rr2%@o;`EZ`;O5F80=jN+0A{xRI zn4F()_X=Xk92|0R;zgdWiW<*etH*;kaQ_1_BywSx6D%Q(wx&Pd7asYcD+xH$^!hCH$(BAr3 z-AQ=Xg$j-jhj{cl&RMI~fXa#5%8XoV;%EInXrk=Pm+t&+CdWTPz-o z3B4R-6LwLKjJpOYQRBeioobvQO6Z%i20ug&^i#L!&Mm4g?D4#2qo#KyC^!eT(Gmtw zqA1=9rmsTU`=h*p7g1uGR7A(HR4W;INI;7Is1Uj|4Pw(;GWc#ZSuWX~svSQ_+SUK% z#Y3K3+}UQbX-GO{fF0|8&yC@Tzg@Yik0b@UWN5ou0Ls>77bx3%eVT=(!^f`+IQULL zwrj8#8_0H10oktaK(>qFs?P;9^=yyQtMS#|&ZkER#p_!A>VQ5OvOe9-%}$_yt2lr! zyuw72txNc+&or?`nWK6g&$C&0kmYW5e%7`c|D8JMJ2_4eO-gNTZVV-_U#Q0QBMc9M zc={r-JLNmqe9&s3W=qojM^#oTBpanB$|;}Li1E<@eayHS^MTynKsN*0AE&;xd`M^ z(#VJF&278mfl1&P`V+Xxq?w2>HUy@0cV|?XKE_v&F5na^BvaEGG^%Nm{@~1}4$fnf zPGJhOXt}(*w7Xm1dyM!|N0Q4h1X5;GBjXUzUITr>Q-~k_8Mk#Cp`>c+GZNenAgN;m zZJm$;AUQ=Z(Os;KmKX-gV0~GEs+tByB(&`rxAk!|f@A@inOoXmu$#pMI)H)qd>=rP z?eJen?vLy~E)6;2vq%Cmv%zODKeZQzYydFiU(be|U0u&DFj%TZR_7b5&7GR2K8NCV z`x2kD`as!Ml-FW!*(5AQFo&NS7CAY$vMIOW54OKNHW)=J)O6O)+7zQZsRxpfgy{ee ze$p!Rp1Edu^?Emo4bdvzC-C}?c+hh8XCV+L=BdPR<*dusSkmE5fSXKCRtfcn?2Gi9-e&g+S=SLvUu7gjNw#N^o1O^ZftYx zHo5JsVR0iw0;$`RieOTNu~WYaY|?_xPt?RY(X9-A`|l+;^?rJvJRLwF1OdyBv3CTJ zr%#9}W_WQBl034wx`v+5>)1*bmOjYF!JJc1w_J_ILApkj+1&>-XxXyk&-yRru@Vpe ze5SG?$T;b*!M=BK>GRRy9N-?bvo4t}`NSwCK4Y}ekVjpdHz;zB{_~=JCELz_UAH>s zbL!`3;sQR+4+Rebbn$2N^=S{`Y1(H@R7UcSe9yg>xdp*}-OJ>w3%{}gCAcepp?P8C&Er-#XTT&m!CbW7*L9HZjiJV)?Z2}I&W$m{v)CbnP;Zy2PbzusAN@mAH1=#lJo>f5 zH__MyD7i22!W_ET*@>!ZSRJd*@yMLgZV4gb`d4WSjEj%$^Y0v6R#uqib-8ny>pV6x z*{~KBc)LgN0s~Cyok!mvFj%t&-U32%YB>}UQ~I>yLW|)FovksFN(0;U!8Z^k^`>K$ z{k#o=72WZT%Kf{NLtkJ23Ch8-`C4$BSI#{C^?{zi=fUxZZT>%Mrj#Uv0BVV;sMtnf$j1j?MCmF!=%z4erlw#D zjY6cw+E})_aGr@Jm`$+d@30Lg{;&wEk}ZL*h@Qwu0_<1$dn)a8%f&voZXR#szbj%d zVM-XH8k%!px-zaAg#IL2xWEs>=#lvC%6Y znAZ^~ckseGOq22n7*}-avA1xUqPq!F=~SZiL4Z?O~6k?v|j}G{~X)Uc2monRqb!VkUO016KHbP$@Njja2v4**qsO zuW5eI>$5&5U|zeUh-#F&=qVEePI;vB*pp`LDqHM3YE#gu_J?ONL^j~?0(0oGp(IWB zlTOtjs>iD4L9GhL9gPrhOZ{p{wIqRJG5|`RaKDWwXCJ@FQ0im*FWQ*c zF1-akXsj^!D$L$oWApn(bJq)>fBf@pb&DAEvLOHA(G>H4)XTqKO$dB8EZ1Owy#W_R z1Sbha{r^b23b3lOu1!gYpdv^qjdXW+qqInOcb5oAg9_4uNQi)>goJd6gwiD?A)z23 zlJf7XpUzx{n)&bZ+yUp|yzg1D*IsMwz0VmvM;mYxU-PP{{!B1zBWB1JkLh8Ql72Ev$B zz3g9kJ9WS-(|jI%Pqgd!^=b*;UCWP!oj7}4sbchudPE_;L z%*DG%di|F_p@y^yx#0v~XHW6~79Qn^n@}&WWq`FZ$PA`Y35{4;5l;~dv4X&4(J@d-B6Seza2McVG1HpdWVTzgDQVtN!<1d(uheR4> z70~T=Ds$)wPXv<>DTh9059Ht_1N(6=uw_gUkjFi6MKcE>E)ChLimzB!H)LViES3DEgZmmKo zCpolLI}$*F+e-g_b|Axskx;M%qy4~2#ttbVLb5fW*&>~z9&PCvy_81w4`=s{Z!wZt zI=g1yU6;QbkJ6>X47Qjj%`zFHB&c$9-LlZ4SH|2&yT7t0_62E;Wbfg3p;tv4=5nd6 z8?%1xUsgQ%Bwg0r_?9_?9!6lcgI;TFXju}g7qj=FHYPO_jZPLYa~Ux6e<+E6hvh$; zH9xq>Y8%mByeJmqX6C8)z>W6W`)1Kgjlwi2VAFM;XsjkHt98TWukfoF71>{(xGDy$ zOa>`k_-M71Px-S?{bQvw>HE0X{ag{Y3X&u)))#h9CLwhbyx2X5M!#dj2icuX$+#Ye#utfI!RN*p^R z=pIV$UghSR*l*w-hFz;m7-%OlCwmeWzv`fN3 zR1~l7B(raLSlCv=$d)fBV<$O$#Wf0f{e{kMqV|yYNN9k`QtITFDds4-a&NtPh^0T$ zJw+WHN&F@gUmReiI-sD%Q*tY-ddUmMd5?tThDX$z&$GWbLqtF`Waw|r5Ie>nlibD1 zEuHo&tgA1a-__BN6{(az=KUBxVhf$H`4V>ty8+8X3a|fQ`QHKj##;siu)H|<(pi9S1Nn&!*%XyHxy8!~5IG6>xHE&&|5^q_G) z2R8RyQJ*bkNpqEjsyq1@;!B$tJ z++cT&?#F|>@;x5d_fQ2HzAJRO8?`qZR=qM5WY9K3*Q~(hwd%j8@b>;aqoUE`b09Hy zTfkOnuUb(w^AwtE>nUgS2-+}u*UR=vJXIXYeZ#P?INAFP8!wLlEVM2CsC%j)XQChn zT>pbD|HSGd8*t+7-T^4OMw8zpOkRZ}Kc88sZTIfoP^cItYWkClcOLXucM+JT8(#jL zAF)Qt2V*M)k4!FudR>RYSV&}fc+4+=pe(`m$M8No>+ZMt)^3S$Q$7BjF;4{ z=!&Dren`BO$+HoA+m(P$4YpF!h%c6%3f}{Y`~%}zxI^ystf36lv9UoBHUE6p zW0x2ZuopqLiuvotvXkwJs~_yw(Iq`?sx>>*w1zc!G#0(E^%}X z^u5E}yGWi+WnUDHU<34^ECxmo!*P0#o6`W zXmtM4Y1`*>$m#<%YHX0>6z2cU>VHG20iQqNE7HHtTjKT_%gL-`vdwSM+l9bxhOnlK$?4N8-3Iw5sMaKpxO zq5zo|Nd(nVN@|e6`|gD%c!f6Nurwo+3_8xLIIE7m2iOmMGcG1M1#%#m#i%I0QG<@#{E+1K-lP zV8ZQny~Hc|7L1^QZT|F{c{&xq@%6X=z8OqIYv$Fd<$$)W6W z0$DZvL30e0*`1oVi$lpHleWZ2V^(YC-|*!fmPTwW-QvD4b2j7WcDzzj-&X2`w52Cr_tOPwJv&!-qnm z_WQNZWSPJ++!LKhjhv~!aS;1w>P#2U9~2xz}CtPrX)WzUgg-OQOUgSyD8W=3NtSfSfN>SNPtcmrfBb2Ao+0EZ!Qn&8^GIaMt#y8Rq zHQP(Bn^A6NS4#PLAyFIOh*-RWBi1UmF`j~4w%C5GC!rr_R(YRL<$D3c{oTmDRu7tU zoaYV{B{**}r!NarUMe0jmhh2Lpme91y_sE6TksX_VRUO#;dg1sA!a?)D7r^rsbW=P zfdo3Dvj_;-yZ-mt`S z;69(<`Bf|zX4muD>dHLo`PPY(W+^JA&w=v%_G=NlyA|yy{76qJ6U?dKqE$H$82K~? zw@us()&7iX%`pND-N`81XnFe*XIOxr_0N`QS!<{JZD-~MeXy#mL+K}&Q=9PiD3#hBoW^>-K7sazaeeQp=GAyk4=xl9$wgH9CZ zVY9&7*cY+X(Z5P%!KWV~rSinm%Fa%{hbpN|o58zZs9|8< zI-d06`oFyo4eDvnzaB2~nIFYXMZBlKV4@?*-5n)50QR1&P~ZLYHi+oK_Yl~SQ664J z6vKOXEnESUH6*lkx-w;ne(8E1ZLpb|ctjy)y4z&Uc%@Em?ov(4lK0i8d#d{$CgVxf zdH1!yPaN$2SV_;VtjsM~1Ro#%sUnhsPVJsc)*@Q??R>+qG#2bZDUOKu#iVQayPDL4_u#qEIWw|x1R0A?sr8#p@2qmu^n!fy8U_ou$S?*X51U{ zk{7xSa=w`5ESa@)NtjAf%OcZdhEv281+a+p1=YuJ#P=Zf*2Ufc@B6-45D;R_UBq@? z7QTqhP5(JK_)L0{S4}yx7dwjd9(AWIS?PpWV50fWTfuxOJOeVHTZ660-YHBXz)|6F zh1AKwQALw0rr|1NqAH}TDW)$fu2a{(i&5oC&fa31b!y+EK-of<*}mi_VZf`b+|{Fh z3EQ}RxY8lgfuXr8eW(B>xq#;K`PWK8C6B_8Z_7>Mz{Nd8U`W5Ee&MYxwR}u<@*>IX z4IkMpR5*2u_A5jT>H|1iM1Hyd`%{O(LEB}CRZ6>xwen|0rRXZ}jB&j<|$YXLl_GryEnHp53 z*AN{Dni)t76Iv<*HGX8%lt&x1vK!N)j7xH(w7>FiBELvH zT@g1h;nH(&m-@309M*Za`*osCYfR0=*H)?EqHx^8>pmi+TwNvq$}e8@424g0QWMUX za+T0;9;I4nQvB8v#s{B%h|IiATnoiqp%Gv~D==@LaN4@>H0w+xK)H7hGxfZWa!@yN zZD!-;fQf-A+J==+s~A(R&O^dmD?H2OdnuSz$h8HHQ2~PQ@X!BHGDV$=a>~N_p%nc`Nfm7$@4Az~dGlAN1|d0(r(y}o&%lS+-)}*(^dgDCdcUO7?_74Qit4dG4V(aL%h1oT1OA!edby;j@(+MO-eO)ag7U*hh_1r?hO8qn?jFmbw z70tgVgvI!Em4;ZLzY(ivi|w(vz1`4+KMJA9lbAXuIAer%l#Z)HxJe{3lT2`z5KOSS zlIyJC9FbLUvPqfZ>aY>oZaEeYRNt%ff|JD25SpZhD@DM?(Ga`S!J#WRi3bM;6~%ed z!j8rzHl2c!ey+!Sc_nXA7)>XiYbupyWLdm1Ld?4>?4s$YoJZf+=VC-XLOl0U1mBIK zdq|H*z=v?IL#DSL1%0POfWt||%*im@y7|+>gL|P_Wp<(!&Zz!l#lxBX{Y`4hysj+| zxicN{H&_xzM@iqmcNC#-n|%?<#uhj-qSc5_IqLW<(TJ_DkExRGN@#Nyu0oQ5v@8cZ zM(&%-)-kH8#N38D&f=<>!?(r1U!p|Q;xdsmuvVb!6+`pCo;%k`^H{scuI1rc&UUY9 zx5iC&+czDfK~hxc&HnsntR_8PgDtwIi8zszpcw~-iwrSkZGsJ+jMUzIXnj7TQTH@l_qK|{0u7JiOV zp|9|uaO9`NraCiA31b?pLGeOjMEY|jONj)M%Ic)fY45)kwSC%A(xM*gJ8zDPmLM@g zIyLmkLGJduD;DNy2_>2h;)sk^aIq5GmuiFo3NuUy+3 zpJ*I3!EX)3=uZq3lg5`4@yB*2*cbTmgnhDKzkXTD{<4SfYEzqh%SYs_tPeM|)QFA6 zlj4SP_G8Q{bIUef$3*Jr4&tsEDa17ucU11`k$&U8V$PF~#>p*R)-)bN&39Rg1w(lY zU)$p9e9l{qnJ#@mk9O7wl72$;ud$sZ?GL*yUkn>epOM(4F)`D{SRe=X6CDRqCF*NBUyu zwR@D|m6jYeeH3w5KINsS(;jd1IE&;Ft0JwLE{abfR>2xBOO*N55Ai2$FQOJ9oX3*#hHx8~QYZ5QovMMz?dw-sfXDBN6nT7@W>B8fgAy!V1IE|Sr40DZiL z?20Yt7RsKOe`=HZl|3UL)O;E$dv0{rt@mc{__Dr4qx)PR=pZ(XO5fNr zNA&coFRq9Z zNHty6U0=rl&wW$+co~1sz$YXPPo}J%G4q;si#ov`CtLp8Z416>1}t$yegA`4-TFHb z%v+DDLtsf*Ezlpn#n`0U%&s3?Ek!&as%@uKe&tNC_kGVotDIg`yLE7MM`L)rB0OvA z%HkVDpBMF22kMW(HU~>w`mjB*0I4pu?{Y~U%gstfMX)D?g_yk=8 zU*;j&~2J-qpR!%8tt!ET^3^P8>4OJFKb6_ISj* zXHqe3XZ_hWE23{~pWvvAtU!Ne)lS_EznZ24gVnfgS~VS<%=&}S;q0RL z{p+ebSNv+R-H2ub@NMt62uR9)l}B96W;RqRAD^}**dzKuo;aM(P*p#XIqeI{arkfL za(?}e@dNhY-6RS2R(zcxvX{uDGF&S5R&w2CgxBP&m;4G)y!>Hiw~Es$E=5Rudy(cg zl#4h~Uz?uxnQiNP1^PC_5C3`{!va=n7nUcn&iv`6rLX#*&Rkw4`95To>B{53(jwmW z#pFBQ_3RG4?xJ}N)%yHw!pxi;I4sYWn$sj5DBEbS0_z>isE{uC@#?OUS+LE z-S{Az{z{nK*e+>(4I_&=_;q}j6v7O($<<;jVGAm$oTM<}}~qks<2T zi=VB)j<`o4^1f%_-e!xPa#FLp+YNEfY*8Z1?2mn4XV9i_d4(UFWeYpw*fdwn)9Q#z z24Py#D=nOp2uULj4(!8m4oy=~#p>xZzvoft5u6sA6Qt*Z?XeWp_56L`!o6ErvFU@E zT|(v3j}w+(FXV+qmBNdQWVkymdc1i0JX70eqMUp2&5B!1kMvqtQs&Kd*873tK4c4z zty(rj=L1C>FiNGO1@NvcrhS@x9`yXuJsw&9>R+?e~r_Umc-Tb^dsUWF@`~ zTn$0jkS3Zp(ei6*;lj9OA8j2|m$?{zMJtiD2B&eA8vnnZEQU+STEVEWF?OZHK9Ny&O*Oy}-=tnW6r z!z#7O6W+SDr;HQcvi0m_*OBMU5P6>a9#71`m{#OP1zf|6x$uey#vU0FVkVE``z$@2 z$~wZ^Rb>=E+>CneHT7{+^u30-sIsHw(CzZbTZ*}Fjo4|kL)cziWRzNtm0XVA(6Pk- z(pb#D{M|{;9Hq3SVXkFysDZ8zrapQsjp$XoJy(SZnS9#7qq)@@gDtV%wM&fY=yE(P zeTj14u1RBPJw#=BMoND$GwbDCAy21QtUjlH2PHPw{wv{|1)18VXo{T4cbiiDBl&Pa z<>EVQ19L4iqrp=j1j%2=oJx_%ZP@^eqn##IiClTH zG9G8)8h2Nlobt7sE1GLos?lTS{SpS7L{?*pHj&akVf3H1;ylVibR*SK75y=VY{S1kd&sJQ-Ezsm^gg>>5T<5$$(KEYZY# z_}Q>LdSUvh(@V9^t5I5O{e_S%#{qxaJyScZ$>q> zuEVPnVXEdkt?%j-#N+3hC0+=TAY(43D>0j&XI_gK;}V#n+OTMwA@h_^eUO>JR*NXx zu$M!?`nh*(z)qW-(uGOA{wd3p^Q6jCt@&n-wx+m5#RTzK&Vko-ltFSI@-7i&wKSJROeJVcY1bRG#dmA(>X+wQt1CY!f>9>u-# zXR%c9v0b5@xah&-8=o%6vcuW*+;`K*Xu#0DdG&%^0{_UMeRYB=x1;eusjh3!o5`ff zy+`GvFXVhgIWk_qEokj!v0x^l{w%EeCQfcTQ{Z|IRcz0AH4k60VVt#&(*5wU@mxy( zE4h)+yo+jld>&1`aVKTo*_WtDK~HVQzloM(&ZWtm`-VKS?~7CzulcUKA*&iwPMtga zCl_60Cv_OD0fRS0I^wL16tu3kmpB8+I*G}>krr<)hS{3bykVSI3C5cZLF?3&<4L~$ z1)qg=R>`8rBjL>_hXgSNx>^4Ol&q_Aw&)m>PC0GC!~10#mun*|I3E{P%bC(~d-dL( zH@@yx>=U-L>8vs_gOK$Yt<}HSs?%wC?FE)nK$OJxt99^Vl)PI?uL2_FrU!*g@ZBY> z%_XrFTv!@B-BuM+=ID2|=iqut<-biuw|U~duW7aPt|i;cBb?gMBxmn`Q!}z|Y_s!Y zu3CO%Hz$MSaT0f12iu!CL-v~+MBXCxI@KBvGH=#!^glFJ_#RrPg=Al1y64kmL!J3AT}?P(ygpgub8azFq!&p1KhI> z8@`wO-_1-L`e#EES`CWr)&j-^KfChm1t6=!%#+I1ZY~b=sT$o@F}gjFD9EwioveOK zP5FkZCXc6dXZ7|Y<4iFoyv2$a8lex>Uq5lunSI!SYKNLAa3Atpx3s(X*@(GY{?{tja&k>LZ1hM2Ya0up1^aykA+1DDYU*n4o6PZUEt)bw~~n$iaQU`+^k_(vaDqsLiq zjG_w|bw65mF9{%)gs=`yFlh$ zCOV-I(<#ImX+dvf?e9~nbU-<7px(-ojNl#7sN*u}@4h z*2@gsGwL%PAIMC)z7vJ=aoc-ckXY4yPnpWd>P}A4>$UURF$QLWO$mKnk2_K(QC5)| zv$&WG>wR`9Qy!PC6JbkUppgBz2KLuW^AQ&`uV8A=AVn#Oycg$OXZ$>oZz2yJ}6{v(#3_9>x$}Z9YN8LQP-96eIf&r zdlFBH8ZP0alB$K`CvPRUlZzUDbwPgmuE^c_!i0*Ig&Z8^W6y;w9CmSHxA&Q(-H8um zJ`)7tG$>CqIV<0^r|TI@c$kIh+R$ibN!1v(%2Vy>AVT1s?>R^YL5A zwXh^8d(2)mtLm#YbqOW*jH3+fg|}`ciJEg{J?-Z;!$43*GlxOJmv|E6s=swZgX)eq z(}rY<`AcdnN;@f2hljzsY+2KApMvB+w0?<`pOxLsxx~_!*VA>;vQqbZ5fb@2n-QfU zQ+^?``pAZinimnF&;`Y%6#;6|W{TUdaUD?RU!1dL9^gqY9(q0wN}%F#!^{WVJ$1dw}zThrTERGLs_s)Gj5;S3v zv$?EA)bvq0q?JNt``q((Mr2hVrOBX46u;Df`Ey+tCsK#fyQX{Trr%bcBXgWqat}M7 zI!7%~zfnsi@G&+|>*jQBW%I>}F)g{TMD}6+TQ7b6w?H{8`;}dGH;J+7nQ`KTyL~q9 z+@PkL+7en}GwExSUsk<0MfpFgI+kw^+W06B3>tW4etz+w%=NYZ=YFfK@7LJ!qi=*4 zG*|{dds^oz<+pMk&R1$OXo}k}Bx==iHKlH)@fn;i;sk0kH@0FSRVGa(3YAEDJ*qE~ zcPN?vOc|X9yf@Nz|F`>=tbz?wUvm$dgdSBOKCnu|gKrHPj&Q?^_k9ptD9kk}QgmOM zKc@s9F|LAjKQ4w^g94mG3L8KCR#_~o_Gkm9WJn2mpt-|&ldyN>G2N}NwYnnsT-Hu2Ak5gCpW#t)Ny@+H zAV6Pl{<6cN)U;C5)L%~edg~MN+LpGjcm)klZZ&wJZ{s;lmmu`tm3(ubu~7(pS9%%v z)12wM1CmOt3rsP3)cYu}8#g(E)Agu`XqPUW$HjI^jti3#g;NV|+d>c%o6&%cTDll% z)h8A~X@R5J30Fow>DK!$F5AqrL|->kgBxjwCOzg!U3859IRP$MlR&}Xe0v~gP;m)*^M40`r9F3q(@FVMVfxkkx% z&XlqL3k{o=Tz5#1RvY@Dlo#&37c5*~xgSa2yC9F&Q}$ddO271jvBu*yd^N4)d;L9@ zX0EHae7Lq#K5nQf5{y%}6}z>J)q!9a`8UZ5^~uj>k6lofHe%`&)3r(E_NoHPSG%z+ z7z;0VzIn&c+NTq&9LGmisk1iYjM>qjGLFpiG%g>uz?QO0lk-`EUxPY~HF$13nK<~W3p`Pq5Lx#OSCe~idhl)n<{b_^`wP=kb2Ao!bFVWanCS~}<>ZTbR|2}K z-&X`C2aEDccYKuDQ=gs|E}tN>`%nTa1J6+F>8141$C}KIgzo|yqE7?^vqGe(O^76h z+q4~OJMEL6{}NzS2yI+=EYPjZFc#m^QJ{)a;YG1- z`Qc(OvRm*5%7`RNMX~JdW;YLQc;&)-#W=2YL_xZ-6>#}n$=G(--*UN<@$F_UlDIkC z1~|B?Vp+O}QRmfA8R%TgG{00ok@z$yjFqmbtn3`=m&kCuQ{TxQ z*`k>ovweMRS;BUtFV22_qM-HE_A3vQ*vzMIlg(}CgdgvIv%Km_Fw>9N_rgy9*=T_+ zOgV3M$(<#Sx=Rk&7@<4PMR-w+RG4uHC1yT11IrpRDJ1IJs=@=7-=doAmUyQ`Fp=x& z8HEeVI+0@L(%@p4JWi;oGW@;->%Uv=<84mp)4M0mb0^qM7j}1B;u?Cen=dR9GM7@FFdQbX z9SLj@GM5T9qQcT_7V#d3dKd#nC5=a5<;8hCGWBUfvRnR+;bzuKR2x-#u~cSTAKcZg z0|NQX;(J%RcD>oSOLLwSD7bn*dA>)RKZ><-ece3z zCmv#OzJi+kd=YyD91T+pIX`p@zBtX)x{%4)D?5AYR>!P)S_hCuv>&kf=2OtuJREzy zqk>%&KbBqd93d*GOON`PYxJb)>1hW{wTfOXTnf9GldY>b`;CJpVX*bK z8cTt##e*rC?oIkEib&cIu598DY59|OOE~Lrd&(o~3F!hCKX(fgu`YdL%zmkGzr|3* z16O*R`5HggOUylPwfdgKx|kRvW@8eWB2;$10bEUm|jFa6ui=2z)4bG3B zzw}}8V?ZjnTqp{f6>k1_I8oiA?<65qYl2QjwQ8LLtnCFmM^omM8_8erK~M2P(C+w? z9yOE{5s4J=wC*o)6yFIqO3Ya4#LEluh$0226|j{G{EOyZ2~6T}rtIP{=%T^TV3-NK zcegZeZP59kz?rVlS>5nsaqM{o>-XG;yB0H>S6_$o1dbIe)-JwQoJ|1RG`&T3mf4wt zHoGH)IX|P{5+b`U%ut#&zx7cMajZRW)C|f=aAn=| zw5J?H9DV>0Bzd};QD5;8H$1jT->l!65l*;1 zp(oTWnGimuimqt5CA3B^YL4hFgt&i@wp)nWcuuJGp>Nmspc;&9tnUTadKFE}kkneI z>M=K|n@LbB_wsKe(&NSA=ny%jHB=%<25EMf;(AhJV!0OfPmf7ut`VER@id}oFhc3D zE@x+(%;Va7?oWTov`ac)Ab))I(^ou^04n|G0u1$bg)biOaPf`Thf?n#&<}O48WE;6 z#LLX>Enre^x3o?rV7ACVy@uKCQpZU%-A%OW$CW1#N4MpLRBfc_o(50Mg7&g$su@!% zBRGy{l9D{M>GDjiQ(ZsJ^ekotn|O7PUzQNPT%GR6RSTr#nDE*G*!MfmbRT3ln?A_AgO{8*|F>PJy$fenvr(R&<>El{PHjWZ8DG|<(OHa6A%7B*(a+#(? zy$fr>+~;p>LxxZ5?H-&_-|-F%sZYN^e8;hLes!xBV_cN%>kpz;*r0QB1m_<6&{Cm= zpZn@MtzT=sdTq_yi3?ZEvZDGO-JKX(d5W;wetFOCuuXv$k01Jst1dZpZo}iSqmvhA zc4J1;yPIPX7nY-kr+J^FX1ztdJ37glH|cnzK57%yr!cyza^Sm7&TD-K1n*$SXLxm& zHCE!lJF8t;r!R~76%^JDZo;i_b22Hf`ugupjBHQ1&gqhGxn4bRb@_pnOkR0m`}JkG zBE@Uo3+*v=L0uRYPwTZc-ebA^h)UkB#oZGmJ+R?DXr=j)_HZI<@2Ll3sTO`o$wA|* z9mM*AtAge0zSA8fm-eo&RW0wO@~m^B`f#pUy8P%)-cCZOMcS?bzdqhpNA@AwE?M5w zbV0L6CIIyDimn{$M2Gz^y(tXm8u)hi&q99NLfZ zwiUp!)Zv>qx#GX){dtc!dAsp$Ey)_v^4r^9Qd9 z;8(+`nefn-4H*k_li(Ob;Y17Tlsx@N2rq|A7GjBRdVgLP%o-xdghjSCYn-8(dtz2YGwpGMV! zK?zyYyyC=D;ialE(|jBf{x=-FzTw#F@wD}OO|S4ob#?-!UU4~E1v%K~Hb^A6nwTPd zG6;;wN)H;1qlhH5Nxf!Y*0i=}<$RhGkIcyI*zC3RXu2!5ZRQ9k~W6|;vb(Ov8n zjFq;4TT5luST3Jwy`X z9=~H!waC`(G4FzUS0lKthr(omIff&&y;mhvM&iJUB0WS(aad}dn~WyF7P&JqYpc|m ziEeVvhrTN-n0u}8&OA>WO2vL+C0kck;-Vw!hteI18`o@$U*>Op`{wq;c{j8t!_TOQ zyk_Z|F1ZKTv+-`<1F2$fTo1Q2&bZ7~DP*d9&2yk0e{AIg%-aw3R<%^|9|sE%I|~e( zmUI@oUm>CkN-Xtz*wdE?ybu|ZFR1KW`DV(f8b>+wQjgJhGhw)+2DW&=#%m{3uORY6 z>bdYc*g;|YHC?F9!}ZBFR_n~2p9lAemnH5J-86`=G>acBBezd)yFNlnGdxq}H~n&x z$l5)_;cD`8bV+MoG=@uCEN&{3t&`$@x=#=8b_f&jzo|kprnWF-?C_O0OJCGV-oAP_ z;?hX=!xs(%5|>IJJP!Egxmq1LZ*3;D_o9iHJc|wPQ2wwG|uK_d*xO-7opoO zZboD(a~zd!Q%(dkC~i&a{>yhvdG$JHDTv?D?E$B0ez3Fxi|* z@cJMk9**611Xt3vFT<_3;RVEyY!F_`eu?~0KpMqx@onno*SKS_wp1pboVQduX2Hu0 z(R=aMWLxYPKB8pzo)Gw}z`%Fb#RP|N(2deQ19kNwK zw_B5@lUjbRg^$3Kzm_HZYz;=mJ+cs+0yaJV3RGS+q$kWeN>3dwx+D{U!5THzVqwWy61ZqmM%@)T-i|v_ zA9+KFxWA|+Ifi$gz>#!vtaI|MsqETtkCEwp^w|<)++v2*eyz5`$L_BlzpC%5CQTKm zn<;WxtGH+$Ih?*B^Ns16XN5UI*mSY^7@Wsl_Dp1oOY0OGj(1DRhrXf=B(lezT(nN7_x%)U; zZ=Q0<9c6O~fmg7eL8=h#!tzsuDz174=|b5BSOtM9E82jvy6Dj%3#$W8EujxB46Q@9 z<#*l^o=Yc?c1{tLm3L&O>n{4L~r4Saup zGW?hU<{tz{V+R*=Lt{rq3w;-T21k1<1~CV77h{LN;9!2<(C^D3KYvRC0|Su-^1xj~ zTWj#N*5jo||8qIyik$yl0cf1e4fRjF5*zBuzC%|U+Bz6RT@KORpDVyGm;d|&LJSXf zqeE`>H?fhvlfJQysksd_$YI-^hXR2d1%?7~G>5u{!&hH|y4vK3tN$UzVTdbG5DCXY zm|NRfop>u(p>W|mk~@rsj-N&|n0V99{lREG(VdmD4;NPoGwOv1%4W&OgIE|D7_y$IAZVz;lcCt4pbm#iEU3G}L4;P3xh>Sihocm9s1`hQ zF+`3NFNaK2zdjI8u{r8;U@vQ1n}30laTJP~yMcqb(TOO~CLTEo<#fA1gJ2;+EAyv5 zApe23$YG^k`>m9pbFJ9%AZPPk=;7#p6r82YiK@Yex}nlT2v)X+`c~#1`jEf?1#{Tb zEd6+ z$LFm}2caMX%tNG542YN~GaQZQkX`Q70q$4_By9{(cncsa|HmDxiQ3xO7#l)j7rBbQ zqvikpc{H|``nTB%0T`I()Wi1*3udyDu#d)ji$5=~0)p!!cJOFGsC(U3I~h;O!PX8` zQtqV}V6grbCLyjU;BgfAubTE~ zEH7iM14t5n1o-m=l~QNa6RxSKxHR=r=P>_ z_xr^T54;<(@d{x;T5p`grTpEX69NAzFeF;#X(^Jr1B+CGFDaB|`1Yd6mhOrGvscps9)A)O;tFn{qtDf_{PvqDQ+*&0ZItOtBS@o=FfUfWcb-koss8g;O1R{K61mp(fwIg=DoqE_j^ENdziw z(?Q;TMH~i(>JKPnsw_i11MJcFT3nxI2I*Ouf?>oA1)4&6Jm{|g4CN^QgjWiHm!`l^ zgjE5m=mcL1D0p_-L-3%HD{5yaYNhY!sP3R|2a>9zA>AWgjf?;sD)2mVD98@Fe?o$E z#>m;wNe%QF!Ax;9Vp8&8b`Z#A6o3Mtec6HGpNK!x>z|!QF=gq$a@M1vGgZ9|ut82u z20}QrD~Ji50)1T81W~IICBqeLApbbO&*P&l$TAd3MODrP$EL3BO9Cz-SL>z6QUAxiU2N5V8{EwjvP>al52WK zbX_B3T`}+8E(Fan03Kxi_kzp_1VX47^FKQ~>Y6+0N~@f};lg(V>l=^*0sk<^ z$eputSm^7@D4oDj_A2pdCg{p+f&MCVCPW67B2Sd#Po;IVuM(J9f>(>e!2AHi7J4Sa z0gn_t9$7|AMD(|cPPp;@Oxb2yU|?C+Uyqvp?XMYr$>5b9XM;c0Tg7f5n0^B76G_lU z;QY-1erEU)7M+>oGF5|ph*yW&P;Ne z!*)d8q-6zV^)t1@B=u8gCOOSx-9HtjsDOD=6nHJPUrBEqNAlOo->EZvJ|d5xJeY92 zz=Q*xJS=}d1N=#|{I~aRUn&9g1h{+Xc{pJE4A7^}l^5scyRpD+bAY%6%>3t*;TO4k z2K-aT^SNiEFSY?@6_5bBKwH{B1GJ2Baxgb=b~1*Pn@ zUpw4MCPO@(2Q+C%Jz@v*>xpMjbLn!}9oRE}S<>Qw7zVUT*$1CB3% zgBwa=H(CEB$4U10W(xI01;fP_R#_!AU}>DZM)YbvDfKKR8smPv!V6ftrHF^H)ucDlUMoyX4_XR!sC%4rO~QP)9pD>DxFRBf_;E%IFfHq6=UQLT8<_ z;5F5!m;%5<9*%hQjqhx>H~0YL12FKQm(`@eJ9JM&cCvQ*ZG>aPZI!JPrxako1Psub z6drgF!oOfR$&ZTrNx4&ijn_an1+Adc+ozJK={uMM-5;$WOUm|VcwjX5gQz45HJTqA z{}cId4Xxk1yFi!ZcbD>4H|~GEhvN$ER68so1CXmZlYvDIC_-HD%;LlP`WJ*p%Qr%I zi!cY!zycFNCqaV_XQGonuEz+Oq|WKf2QvZfQGv1yIu>?2o(u(fm-w zgMA15s{|Mc+Q-yf&qSkat#4%|ZR2zj$tUaR)8lL;AkDHdJ&A&(2I13n zFviFJF?HB@{|gF8u6PnnN(}nJFrZlgZpH`YVk=;!_6#lt`kz1&C4DE*T(&uh$&nsi z01?>#)jxDb8gwGl?*=0zT|9n;2;yRPwOux>K#+@ICmQrT=o$JiDUKc?Fm4GEg23+H zKx5F_I*2|Kfw;Aull$)&a8kR9T4(O=OE3;KfX1NRI6n65On(~bM~jpA9`iON9)|&u zpd)@-{Mkqx9BdtaksQq+SAi1H1w2>*1Y-#(_u@@F6T@#^{gH(H&;PM?J(`ms3qAE7 z=$@#9=ntI@TR;78oJaFqWWs(;036{R@Hps2TUqc{)DwozZ-@T1Jp4zpqglR<1at8M z8_j_)Ka>WmOaBdvt*fZ5vyIb9##+2wG75Pmb3bqgXk*ysIwS|N~K14k27 zT^rSdRL6_|n1Lqh&P?!EusfRMJQLZ+NT9D5;0x`1GYw}Z5kGDC!doVOfCr+Kf-iKz z*VcGumfr*Rq;Pk?xVv`|kO+hE1U+WSo6k&ihDEq3D}{%4z`Z{M9_RwVr1i`^r;J!- zBYN*TU=~Ok1+BE}?PupW!+7=KWC(z~fkp~=vJfEr{mJkXeS2o2Gp}E=bKYl=Odi8s*ekiT-47rg%F3kY5lq!gT(0X*f{x6w+rFg%LrlN0pv}9Ig zgp|F&ZeRZ~Yq_)iCq8MhqlpI1XKKcQ!xsTE=uAJ8>x@JS&Q?~CqULA<odi|N{|k7qc42LI9EZ9V%jQ?mn-{%uIP)%2Ix7drR!{b> zmp46Z+`!(V2b70P-bTQ0R}H11GGmaTAs?_>;(3{|L^Ga z(eSY9ZI7M=mBnQdm7E(YLQb| z#!Q$iL7D^;d;aj|7Bc%2SwN|H3iO9}Qk#t+fms4y=xM~&=``q@pr;20^(hRuE)oqj z0Q5cZg&y1C&L=aR-3+Po-${~yf*QcI4(&`&T~6fqbMGhCPm&#?bXvg5ka`z-G7ksa zP!4Cx#=kw&k@0^Szm4f1o}<0BtC-!Y7U*Xme4)DnTwr{hKp<@c_9!}l9S4xv;S`c0 zeB`-%z)vecRtG&z$a$Ya0w(WYbKNNvEe1V~p8-WSm?ogJZ;CsoP`H32(?-G$rp}Nv zWv9>>-XD~21FAX)Yyw>&nfsrGMiK04|9Lw56dtTxi}5DFa{#{3;)Dd8$fE$Ha4RT3tYako_CbsU3aDiKIW{udo$3 z!*1lRSV;WYl9m=bt^*M1v$ZP$5)@I8L_nVr zdK~dD;E#>~EWO?LI)H{;0Ri;AGl8gcf)4;K{k-?zLFZ_A-DgYluYnQ%ueGy|j`H~W z_(Fl;?j9sqArylACLVys#DcvLsERhYnQ3{j*#VHhuI|M61iZxhq*Wy;7cyTK( z@Aqz!g?XNtM>ucJ*+WnN_}u&4xpU{v%$=Dl__4aT`Bf25nTSorttOsJ4$TkU&)~<7 zAsr|4;IXbPLv68ER28=D-P>yFZSRV$jnUzeqPoJ=Gh6o2AvqJQ!%EnIGAg-sK1uMf zSRzfvXz#=rtD=O2uJ7Jof$va#MP(Q%^DiZN?KHlG^qFR$sTbek2ZC$;gsnKkIsj{A z((rFe@=1(TJeBQ(b;c$c_Xr$qGrzcy<;QWrtiq4Q9-70)I?6s;n<9m$d7SUwDKZuY zE{+gXh0%5{mZ5ms`Z_8i_}~BJ?1(1Q%GT3=;vt>$rd55XuZ$^EEF;d}U{4Tdn*a+7N#_=`wWB&e4 zpD?~NJ?^@BPbxan+CtJZcehpw`28Z)wlV*k9$)nuEVKuTvt8-zIsp`iwM^j-N8;;x zqUlxF1^y*I}~;I1LJdg|EpI z6QAvX+Ws5jux20I%ab3|G?Y`CL2v&Z&hJ>n4^u>q|bT{-yXMq+2{Qd5@=tE z&G1`n(Cr=yq7o^bKmhGe^4ysA9T5L8yi&?M(UAt67w?iOz2Dujg&HPJAu6Iao)KL|*cOqr|A6J3;;$VuR!U5)Iztg&6Eez+=?{i%b{u@5N^Oi~F`S=Sr^x!%7h|r96*wdF63I9l@z5gve z`1*Y9gM1#lRs;Hp4FBhCJ$N@hzLEQ?Vm*MKgU-a--tAvK_y&CZ)3q{kWul84!pqav&Wv@PxbAcWy()_s)sI!346!>QLr)abiW$w zmo=a3!3$zSleT`hD*#V*u`K@g|McJm#<7L&^&A2;D%>Ra3NQ8G1u-G8?8Qb+fX69U z3I6dbJ$OM(xHTYcP;KDf$&8cqMh{*P6Sfz7^oZZ9{(GwjFNg_FM_LRHh32$n$u@1T z_j>Sxn6T#c!86pWkDE+N??No+DhLd)4o(rJ$QlbN98&*&lC7kGW_vedhmjnaN_*_WmSNe zyM1ULJ$OM(i1nJ0M0j}$csj2hydWkVU%jvpoqzrhKi2JA=huN3*e3K^)4h}d$_Lsl z1$58_rnwN`(TxJmGnwwS3+bQ>EaUSqIY(KL}#c!P2$3*LiAIx$e1(>Yxi^ z<=L;=d`**{J2J}*D5irh2$i$ljk#xI`bV4N?5_3g;yUQQ6qY@)$;-WxH z0BZ$K1*K~df0xiMh@S(7)|zaZn#*7aFCv|Qe^*WeTfAUDu#@+gO|Xm)Y|k#(yH`^M z$MTiE8IyJW<_sn~Gj3X4jfq-PKD1}URb!6;DqlEhT0;X=W#Q!3(N!0=flg}#Rq=1; zUkYA^RiU@H@)d5sxKcJ6qH&T%vS`_*mI_j3=ce@Z0;S4;Tp46`l_y7Sb#h-+;qWz{ z?VFqx_ahjBuzJKUZJX<;a}$%4)}R+|b$xRgR;7`X-B-x_jS9O%9Bvz+GwtDOtI>P2 zJg(mdAApD_$y=IZMNDzzx1Wo1?CkRGQ+is_fz($D;2YSq=!KgKUduYwftd$-W%?ky zPW7>YDn1&kVSKIQYi!#`r&aeLG_h-U*&C@~gYZ6t(dr{uJ1?J{mQ)(1s028xZ{;Rx zU=#s~ys*s7*(XK=>x8v6R^MUYs$t_o?d`}n#Kb+m&G?I$otmkG?R6l0aM4!-eP}1u z4H{R^j2!qkCAqh|3Rq!1zJHg98S$XfTTN_aj%=YuwFoYyir!(`avhqD1eaZIb@5PP zhS2t*7B}uy;l)4q;Sw*fF3N7!^zc-Xp|Y*_;qmAtRKHUjbe22`^-hZ2+fJr?9{4TT zKD6dK+V_|VGMiW4Yo&!8r094_(?>kSsb+&4dLX-jJ=Is4{@Ko$=m1C0VU?!QbwE|& zCTvhj@mGX@3Z|4Dh>FQNOqn?qYr}@MPEy?ux6^^=n?3)Yw7WmUFk29>*$DC`P!Uf1 zE8?6ctoggDD}and!o>Q?u@FUUMgfVKT=4Ya(U(CE$s^r=-`iD@+zxx<@uu_vQn7^h z_|cS?l<0LpqQ^Gpi=K)SGGcW`v4hB4PFX&l8wZPf$~xyIvmF2FNT6pxfJ&`@pBq^R zQIXRtn1`)FT^p%F?qsw^qpUIW%05N=ycp&_s5bm@GCqLKQDQBMVvLcd=y0Qzt~(P| z?-kpZ?a2c4(eKf^?B?nNs}h?II;UT*E44rbw|%*swGC(k(ZpaPHk z7+ZE6EOG%2%4&XXm=ONi1+E@(=13e3*p^LZ-#wHB zldMM(8C%tzGF1ppe}Rm2udmfQ<8JhgNQPyuz}4A;o`=X^KKn zZe~Woh$bOZO^HADgpfuM!X9A>*`z3>6{^j(`Q!fe4L94XfFOcRS)GtheVzyZ8d=t5{!MwCn<+#)ehPw1EXfqKIg~d0NRYQY+|41 zyO62gDSjgEXS39=4#=SCCtS2(^Zl{vM=EH;MXHia z@8?5uY^248^pOs;Xf#EOYd>H3187)PGAV6*gpb#v^!u4dJo13L8{-liW*Z!rQ}F^Q zPD|)gyU5-CTPPYfMKok*sK1;L3Ha<`Dos9^H>|>+r{E=R3rL-N_Gylcm?Wyu)?%F{ z&S$#Cph?H#1K3c$>2C$FhN$)YOy1OQK`sh1+aF`E@X3~FbeZ(4ij|yu`_plQD2z~v zKdbVqs|sKZDz7rsIz|UeM#2GE@|h+t>N?+E`*;=I-n|)Qc3hi&lS}rE*5nsC z+U+eI1@6z_vPPbIN5-}Lge5@}_}5$2Hnl~MIR>X@!ENu!!S-j?LN&EsfYYtbxV_kb z>R?F%ef^;TY=6H^lh*BbU+GF?Kn)0B=PbRR$l&%%8|{m4n%EV)tuDU+?EPS~nP8r0 za<=0kF)bb)@~(MD$|5MPvTDD2u87v))8lv4^Cfevfrqd|!2;a(Eg5t(Z;n$>UWbqV z%V!&~k7N#A__Z8seM}FHz(bjfyy2APIGW*|>1r0Sm1)kdH3U*|;*ObULpZv$VVWiPv^o-XhF;)9e*jw?eoj2N@G$;qIK<3=++wJ zb43S-*3*X<5Rv$6b~Nd19J_cdmG{eME0HB+w08Cf7c@1^p?>!l9U_my^qd`q@|NOo zW6c^9`=AAems7z7M3Ny_z*WkU7=she*cn67J&rwV(mKU$* zOSA#oLt0nkI3AT;qK9$1UEyBB1#iEHPGPdgQ6PasgILtR>76nSN^~Z_SCi<$s62e^DOh zZGv?t7C8h(6cRVb8tCUpPGuG$-{#zKZaGXa6PmKf>sVY-MsljL2)Xg@h8i2eI?z#i z-1;xPyP=5WuTu?QT_%i9XxxE!vFrPX!eqcex>CT$V%45kwNCrjvK|fe48O`uL$$*t zX%c<4xYm`EZuPDJMD==A7@${g4p755@$gdr!!!!JqZhE_gb5cSNL92XQ2M~PW$NZE zKclxB>cfTD6HBR)T+*ipj5P(BzJJU#pJqqTz^aL~0x_XXE5uY+Q^MZjoJan>N7?&!BfT#tt6+ z<=mv`FfGuz;S+n4aSH;EW-orj~hm3q@i{#nXFb)>DA_Muc{CA=4cH2imoS4FCa`T=dk zW*DisK;ZNCv43zz3t`718e}AU{1#fCE9cW(5cF}$%coN+Mh$l`=)_9}98cSw>*NrM^a>6~>EO*ffG1?CfT>n!;re7k$ z*L}x3|G!Vdjo%@lv&q7sQcTeI+`Q#t9XuyZ-3bAfqhRq!aM#JO-&IHm0N>y^Y( zmRvVBZH14hfiTAg(UBKbq-dazJN~gRzYWS=R;IgYYXi3?F@ZySQ0b=W{X;~61?7=g zXBvEMe0`^t%6(uwfaAc?xgQQj;R$dGb^2Ebi zbRDXKgA;v}^Q(Fnk@?IFkY+8N8!6Eh@(aQquaG))1*{Q)@t?Iw{}4@B(P&5TD}z^i zuSBo;6((mpWo!=xcxE0zw`9?O^A;Qk@0Vv%D&7{cKVQq@`}Nd==PW)DKC0>VF)q%i ztQ#mb&FD}aczORP_~3_4YhH(GzCp}lb$>rp0gk1E1f!Ko7H|mzD&It=afuJ!)3pLl zKO{(XcO9++&)2;{;oKj4L2o1C7dzuUvq%A+*_>GVNlX$cJ~s82UONgK40~}I5<7_; zxlAjYuk}KwB@HW(BVq=@X3~?kD3Bd1rkU!CG*3NPtkC!9NnMIayMNYgdiZ?p7rk0j zlumJ!LEvTk>FT{obh`Wq`}N{U203H1;E?@Jm#}?zV>}9y@%h>xJXiDgLU8srXj*n! zxBH_4J;)euia|}YpgcXxj-oYB!6u#{vqceQzH$=l>=+}eq-&3cjWs0-WZbyC>O&UT z94E9Sby1$#HOUC>W2N_)qQlMn`6;>1jiPlD%9cl}(N1cf=}uaN@QuFTx!i@#FwZKO zhb`%PTUCLo@5ylI>NnTtE{c~lQ1Z!coaCvlAjENhxil8@KjrCUv)_EQLNZ!`_0EW< z3Q{B$)S{r{QfEaAFBl0qzl7ekxemHu1e{TSQ5hs&hIPov*g${ZLpz;sm($U88aG4} zQ)wXE1H61W>^@1MR&%&(Yx_T$m$<5BsyB}$b#sLb+C^a_&rv_kG8FRFLfhQ?_Js9w zU;%|K3T*4I#@Cos77mKYmw->`iBDkT{>S!eA{_PoBJa5}t^fUg(El{rn625^)j>0y z-@IA9>vy0d2@eZO9rMQ!4m!Pkn zS%*Da<*OZL_}29a+~rTW3me*xSQN;zjFL|uGbACBHvaO-oH1`dCAy}Juklygto-ho z->L-HmhnBmuyNT_sGce_%-lo;y2gww)!1)JBBIt2Sw?mxNrCRjo%rT%GRie{HmGG} zOQ*93Do}N11HbPm_-tMQgJCwNH>}G{NES$Oa}9FS7#7{fEbL5alOMi=jRAvxQKCD- zVwLm&X62ixa$v*ZZLn$bLm6q*9XUx8o{z0DbU=+u==mPV#n|3odafp{Qq(=2Xj<_I zy+2VF7Cjd$z_mu*x?$1z)&gHehEHCm2QP@aJJU)Q+zyAFf~gH_nxUyW@O-;(JyrL1 zcB~Q>sw9O^Sg!!r7Inv_T^iREa4NN6qwcjo72q18Zu8yozpjFlQ)gwnP@nw@WQ|cb zyD_HT7>h_H2n%oW@=madzs(uy0Og2-cq=)EUAZ`{JS`{@ZsCV(6}yi3mfzGNt0*l;ou^{(l5}N_ES)7lgFshnDw`e6PkL zB*tQ5IjS4Dow>Cb;8T(S~l+dk&W?6i+)3%A%4`49shfhHObo`|xrp(s8S5q{YoD52&?@CvQVk2Agt?Krwt~wD3#QD zGauI9RhO{zZ%wyh<`?t7uS{~I_Al{$_WDIeq({9k5E*EUj=;qSd|9s_KaKtcvYz1+ zn=-N{yJ?n%=Ay^1l)q~2p)NCKpsRI6O3h|pgBxm<#Q%m3$s?}kz&FI<$CAUF^T~mh zIF$87ezrx+Cp%?zkFSTUt1+fWY;v@&g;p~EJ{amVZ{i;oX^5Ei7IPPt?$%l>Jpf4_ zU-#3OF4b!STaQJ^WJ6I}h%((?2+W5rxZVCoxDlxBB z(=14^6VL-HIdr3exMHF*k+0#L0ejr4L%aIOLuxWSW3qrpHz;87!q&%rEfgnVAU)^& zHY0g7l8>oq80?KZ{_u#*Pz5eu$pza-MYh1I#aJwevcY`X6ag%~CYP!O-Q+=0Tu`>I z+|kJ;(O@2A5{J@c-g>hYCCP>GKV^!)yMI;)kl*M%bB-nf>5q+CILk(4_M*ivS8s!k zMlWo$yCnDL>cWdnG-}SuQ-!fJzOI^OH&!Zeqteg9+3JY%%$Vr`Ka53VZorSt3(s$2$*9V-nXIl@zp$CY?Zbtr z9~M~_7)g&o$Q@u!QEzB5W3ZiO!9Khr|%76}@D~G|%&0uCWbX4D}CV)>g|2fgL z3Vw}4O_Ff=aGwHEVa87$_wHQjD|nO8kcj@thRf>*xzrAE7TzN_#M(2Y;9a_gCJ6-l z%FajKnD3e!{qaAc&XC{WVzQOYFJg3PRWpniv z9pfNeT-iC@PNIQFAYWvI@35PE9C{J%(?spJ%~9v*SP@i`bbF3V;8IAc2}$fFXutof zF3A>VHfs{|p>VmI(;@8`8-$8dnGo~HpFzqV;GJ9!(-4*4XjXbc+*2>ZIAn*wqjQzT9@A*Ql_L@DL>C+3b4yj5G z*rn5lw|}G8REIFO6ebus7q7+Sf_$U#kxD|!-7K^965N4agJSm?hJU3hgq&OnzBG8p zoR-i&8%)Jcy;fyYgQvT;68+DKd#XFbqp_ukBxn?l+IQA4XqIqtrG z`Trn=3i8-^xuUoxDd{b)CZ_0xKks~mm^K*e*a4tf8BJmwO|K@Y-fZU@Z_zdA9&r~Y zVos?jkrWpbgK3;Ao|4Bb#2hZZSN_%O;$_{zJAkO=%rH+o^O?S)HAad>{|>rgTmsA- zAlVcpu(AM`k@$*E+!6IdS(o~^L7fX~O-9M)RRmN?(HcK;R&j~Rb^$yixYZbLbX5U2 zIvy`p;>vJ(AdU~+(8j3&-C$J;y{Hlc9p}P_!l3@9coSzqhdzGxq#oWS!5Upq+Umd5 z6!H|7@rn7slI3r@i0A@3AiL=D;2S=ZX(j&bhrnOYs=I=!L=MO-y1OACoPL~9@mkit z!(&tXz(3bPV>X*C-k1*!Fq%0tKvFPy)aBcBw?ldG*p;kJP57)pQ>YaejbSNO#i+4E zr)|!Ufn{i3sWU^`Yw|%a~*M05f616b3 zFzhKS%?thtS7qC`om7;63+I}@C~&tVe{A)XyhPxC)>z12WS_;zpaNp0VjUPErp#5!d=+}cGHjA`6^+c z@GRfK`5=U9-Pv^f4{ZJO!oX+!!N-UL{`9zd#-1QTgYbFekY|%dg~D1D&?QVojrOq|t(D}I8~gOMp*#mP$FsODY~--R+rN&8sRK;n9chAg2W6usL_+3{^RSrZ5Yx3#T1A6){WP7U$^772`Zd zRZLJ)bgVH^OI)2aAa@)U)fd6nup#fnd{r@Y#2*{zDqr&Qvf=pm5Fzix!xpKFKw@IU zG6>zzWr>UoHHJl~h`X0-ZDJT#oaZua;&i#)^@lZLx`WS(fk@d%A!CS$GG)Q54;lmwTq^LRoZBEp?Rcu5O8~|5gOQyZ`_Oe3a zV$G4RbW)zu^7g#PM2OeDU6pP3V0$_M!#44~{T!Z;JfBIQTPOq%5>w2Sr7Ue{cY^B9 znNEv;Q<7^R6pOL(l4Qc=x>Y3T4sgH0+|?M|KZlfX8S2Z^B7(l@+iUrGK(heKZWiZ1 zssQD~{F1(YbqO#U1u8QdRyodr`3DB6STNkV;hUcUT8SS!J8yY{5A0;IM(~wOc>T(! zDU_o}GF=$u@|@(b(jUIDkL3LMDOu0&hu)*((7Ns6m{l0&m{S}ka&P->fRyEP`V$dt zeb2qK+ts?HES&T@!BxaG%WwQETpFLE9Ka$DNEV(DOO8R@r&VVUOh2x{*@5-Q)G zyZnA?MfafPhPHJXP0!roFr|l9Ra(B!wd1Rv!y*6ilwPIU?GBHQO-O-NX^86D?||>u z)C>}T|oVjJ+k8{8#o2zG>)n>5MJ+0%w|lX6eW%&@-@$C-+{` zd-#W@4{7Lqh@Ttdy{9UR@PVdYCM%Tlzv;Y*^LiCqS!-%tXM;YBwNsevH*FzeZ#PGu)z#q*~y!+wg?$!H!b;u zYvu(#4r}uw5UGCo=84g?x6l#2r4?h1UtGUPj+YT7z0f4wUJAu;U8jJ@v{ny)HhM?%`lR_71N6l!n{%aHD8c$Aw4=AGODwFW6n&#dx|$Xl|1^V9u_Tqz^{zCeoL z*x8fhI(No%rNsDuft31ui=8X|apDX}p(iC+i-olN0x52MDRufzJUxN?E#3iNAVu&L z__%Ga^G@bUsTKGIQUp(e8x7mCQUST#$BGAkffT{h;2-a9t~Q<9F7HFWK#Jgr@B_(x zJg0D_-0$=SQUp(hci(o#t1A3!5&|8&OmVJ@ekptpv(5P6Hv(T00y*qVuS0kJGV1b$ z)ay`kJ3XvN&orZyvKng_LdY7KknExQ zg$QDcOTp$1>B->^G9mB7^a~M$mSgkJ6{9Cl=*Sssu1hBULIjcJ!LNVL8UrEiWI|5% z)-Oa5Sf-s^K86B*uuRBWvwk6hxUxTIi)$2#Dbr!iH8n!N5J6aR8|?n(XReR|QTl}l zqDqrpZck@I$R>OVJ9723=oV5>G}fP5^P&iqbZn7H3F!9)QUu-6-)r~zrI1n|c`h3v zy=?lW@Z-wmCAU0BLPmR8%eRQrFGCO^I~Qs&jD}!ZRASw$T!MZff(YsHQ^zSZ0?UOs zCFvI;XcEr`bwh_hh&*^b=&xUhAVLQ3K3$yRN*9^Ab`Q`mL=Yk0lzo+J6okYt@+xo8H$GM96j}zwS$4B9D+Q zQFgCah#*2f4O!KB9)wWZ&9+IU;rfLLB4n$<2QKu7kcJS#MwJpHbqf)AaBS}FJNrQf zJv_;N$jwpuWeB3m(B*^o(}y&W30XcyzYsxGx%*ZBb}0}dPl@`E(=S92Rq~yVe{&E* zen6mS&1D;}Ux=VhP7L^CIIYh$mYHkfFZzWDqRQ2qm4{I`k>|pplk^J_M3wDpUZ16z zHobt(n#*&tej$RWQoU}C?-oPI1_)uhNzJLcg$S(mWBH>kX?v4An3Yjwx^5W)VHPxHU_+3_$?ie`mcnb;Q*&83*U+mLZ70_1xY2 z(T#rc_SiX3w+un}T@-zG$U`hVAF@doamOvtDflinkx-saphana flinkx-teradata flinkx-greenplum + flinkx-kingbase flinkx-hdfs flinkx-hive @@ -37,7 +38,8 @@ flinkx-ftp flinkx-odps flinkx-hbase - flinkx-phoenix + + flinkx-phoenix5 flinkx-carbondata flinkx-kudu flinkx-cassandra @@ -54,16 +56,17 @@ flinkx-kafka flinkx-emqx flinkx-pulsar + flinkx-pgwal flinkx-restapi UTF-8 - 1.10.0 + 1.10.1 2.7.3 4.5.3 ${basedir}/dev - release_1.10.1 + release_1.10.4 From 8251a50f36244a0e0ba6a7239340ad31b16253cc Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 12 Jan 2021 19:47:57 +0800 Subject: [PATCH 086/136] =?UTF-8?q?=E6=9B=B4=E6=96=B0quick=20start?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickstart.md | 36 +++++++++++++++++++++--------------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index bc054b5444..be01beac63 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -15,6 +15,13 @@ unzip 1.10_release.zip cd 1.10_release ``` +3.直接下载源码和编译好的插件包(推荐) +``` +wget https://github.com/DTStack/flinkx/releases/download/1.10.4/flinkx.7z +7za x flinkx.7z +cd flinkx +``` + ## 编译插件 ```bash @@ -87,9 +94,8 @@ mvn clean package -DskipTests ```bash bin/flinkx \ -mode local \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINKX_HOME/syncplugins \ - -confProp "{\"flink.checkpoint.interval\":60000}" + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins ``` 可以在flink的配置文件里配置端口: @@ -104,8 +110,8 @@ rest.bind-port: 8888 ```bash bin/flinkx \ -mode local \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINK_HOME/syncplugins \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins ``` 任务运行后可以通过8888端口访问flink界面查看任务运行情况: @@ -121,8 +127,8 @@ bin/flinkx \ ```bash bin/flinkx \ -mode standalone \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINKX_HOME/syncplugins \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ -flinkconf $FLINK_HOME/conf \ -confProp "{\"flink.checkpoint.interval\":60000}" ``` @@ -145,7 +151,7 @@ $FLINK_HOME/bin/start-cluster.sh ```bash ./bin/flinkx \ -mode standalone \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ + -job docs/example/stream_stream.json \ -flinkconf $FLINK_HOME/conf ``` @@ -162,8 +168,8 @@ $FLINK_HOME/bin/start-cluster.sh ```bash bin/flinkx \ -mode yarn \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINKX_HOME/syncplugins \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ -flinkconf $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop \ -confProp "{\"flink.checkpoint.interval\":60000}" @@ -188,7 +194,7 @@ $FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024 ```bash bin/flinkx \ -mode yarn \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ + -job docs/example/stream_stream.json \ -flinkconf $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop ``` @@ -206,8 +212,8 @@ bin/flinkx \ ```bash bin/flinkx \ -mode yarnPer \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINKX_HOME/syncplugins \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ -flinkconf $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop \ -flinkLibJar $FLINK_HOME/lib \ @@ -221,8 +227,8 @@ bin/flinkx \ ```bash bin/flinkx \ -mode yarnPer \ - -job $FLINKX_HOME/docs/example/stream_stream.json \ - -pluginRoot $FLINKX_HOME/syncplugins \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ -yarnconf $HADOOP_HOME/etc/hadoop \ -flinkLibJar $FLINK_HOME/lib \ -pluginLoadMode classpath From bc458f844ed406976c9cd262f0f08810aa4e0cb8 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 12 Jan 2021 19:51:15 +0800 Subject: [PATCH 087/136] =?UTF-8?q?=E6=9B=B4=E6=96=B0quick=20start?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/quickstart.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickstart.md b/docs/quickstart.md index be01beac63..7fda36c0d9 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -251,7 +251,7 @@ bin/flinkx \ | **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | | **job** | 数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息 | 无 | 是 | 无 | | **jobid** | 任务名称 | 无 | 否 | Flink Job | -| **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/plugins | +| **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/syncplugins | | **flinkconf** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | | **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | | **yarnconf** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | From 9cfa7ed62c154d75a0d1a5edeadd7a64ead41265 Mon Sep 17 00:00:00 2001 From: lvyanquan Date: Wed, 13 Jan 2021 17:54:29 +0800 Subject: [PATCH 088/136] =?UTF-8?q?[doc][doc]=20=E5=A2=9E=E5=8A=A0kingbase?= =?UTF-8?q?=E4=BD=BF=E7=94=A8=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 1 + README_CH.md | 1 + docs/example/kingbase_stream.json | 53 ++++ docs/example/stream_kingbase.json | 66 ++++ docs/offline/reader/kingbasereader.md | 439 ++++++++++++++++++++++++++ docs/offline/writer/kingbasewriter.md | 309 ++++++++++++++++++ 6 files changed, 869 insertions(+) create mode 100644 docs/example/kingbase_stream.json create mode 100644 docs/example/stream_kingbase.json create mode 100644 docs/offline/reader/kingbasereader.md create mode 100644 docs/offline/writer/kingbasewriter.md diff --git a/README.md b/README.md index 0a57779807..cd8aa4cea1 100644 --- a/README.md +++ b/README.md @@ -52,6 +52,7 @@ The following databases are currently supported: | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | | | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | +| | KingBase | [doc](docs/offline/reader/kingbasereader.md) | [doc](docs/offline/writer/kingbasewriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | diff --git a/README_CH.md b/README_CH.md index cec944897d..c181ce45c0 100644 --- a/README_CH.md +++ b/README_CH.md @@ -63,6 +63,7 @@ FlinkX目前支持下面这些数据库: | | Phoenix | [doc](docs/offline/reader/phoenixreader.md) | [doc](docs/offline/writer/phoenixwriter.md) | | | 达梦 | [doc](docs/offline/reader/dmreader.md) | [doc](docs/offline/writer/dmwriter.md) | | | Greenplum | [doc](docs/offline/reader/greenplumreader.md) | [doc](docs/offline/writer/greenplumwriter.md) | +| | KingBase | [doc](docs/offline/reader/kingbasereader.md) | [doc](docs/offline/writer/kingbasewriter.md) | | | Cassandra | [doc](docs/offline/reader/cassandrareader.md) | [doc](docs/offline/writer/cassandrawriter.md) | | | ODPS | [doc](docs/offline/reader/odpsreader.md) | [doc](docs/offline/writer/odpswriter.md) | | | HBase | [doc](docs/offline/reader/hbasereader.md) | [doc](docs/offline/writer/hbasewriter.md) | diff --git a/docs/example/kingbase_stream.json b/docs/example/kingbase_stream.json new file mode 100644 index 0000000000..db34cbfdfd --- /dev/null +++ b/docs/example/kingbase_stream.json @@ -0,0 +1,53 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} diff --git a/docs/example/stream_kingbase.json b/docs/example/stream_kingbase.json new file mode 100644 index 0000000000..5a34e7058b --- /dev/null +++ b/docs/example/stream_kingbase.json @@ -0,0 +1,66 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "kingbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:kingbase8://localhost:54321/test", + "table": [ + "tableTest" + ], + "schema":"test" + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/offline/reader/kingbasereader.md b/docs/offline/reader/kingbasereader.md new file mode 100644 index 0000000000..2ed22425dd --- /dev/null +++ b/docs/offline/reader/kingbasereader.md @@ -0,0 +1,439 @@ +# KingBase Reader + +## 一、插件名称 +名称:**kingbasereader** +
+## 二、支持的数据源版本 +**KingBase 8.2、8.3** + +## 三、参数说明 +- jdbcUrl + - 描述:针对KingBase数据库的jdbc连接字符串 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- username + - 描述:数据源的用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- password + - 描述:数据源指定用户名的密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- schema + - 描述:查询数据库所在schema + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- where + - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 + - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- splitPk + - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 + - 注意: + 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 + 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,FlinkX将报错! + 如果channel大于1但是没有配置此参数,任务将置为失败。 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- queryTimeOut + - 描述:查询超时时间,单位秒。 + - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 + - 必选:否 + - 字段类型:int + - 默认值:1000 + +
+ +- customSql + - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 + - 注意: + 只能是查询语句,否则会导致任务失败; + 查询语句返回的字段需要和column列表里的字段对应; + 当指定了此参数时,connection里指定的table无效; + 当指定此参数时,column必须指定具体字段信息,不能以*号代替; + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- column + - 描述:需要读取的字段。 + - 格式:支持3种格式 + 1.读取全部字段,如果字段数量很多,可以使用下面的写法: + "column":["*"] + 2.只指定字段名称: + "column":["id","name"] + 3.指定具体信息: +```json + "column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" + }] +``` +   属性说明: +   name:字段名称 +   type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 +   format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 +   value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- polling + - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 + - 必选:否 + - 默认值:false + +
+ +- pollingInterval + - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 + - 必选:否 + - 字段类型:int + - 默认值:5000 + +
+ +- requestAccumulatorInterval + - 描述:发送查询累加器请求的间隔时间。 + - 必选:否 + - 字段类型:int + - 默认值:2 + +
+ +## 四、配置示例 +1、基础配置 +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +2、多通道 +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 2, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +3、指定customSql +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": ["id","user_id","name"], + "customSql": "select * from kudu where id > 20", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +4、增量同步指定startLocation +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id < 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` +5、间隔轮询 +``` +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "dtstack", + "password": "abc123", + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], + "table": ["kudu"], + "schema":"test" + }], + "column": [{ + "name": "id", + "type": "bigint" + },{ + "name": "user_id", + "type": "bigint" + },{ + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id > 100", + "splitPk": "id", + "increColumn": "id", + "startLocation": "20", + "polling": true, + "pollingInterval": 3000, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "kingbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` \ No newline at end of file diff --git a/docs/offline/writer/kingbasewriter.md b/docs/offline/writer/kingbasewriter.md new file mode 100644 index 0000000000..5844b52b90 --- /dev/null +++ b/docs/offline/writer/kingbasewriter.md @@ -0,0 +1,309 @@ +# KingBase Writer +## 一、插件名称 +名称:**kingbasewriter** +
+## 二、支持的数据源版本 +**KingBase 8.2及8.3** +
+## 三、参数说明 +- jdbcUrl + - 描述:针对KingBase数据库的jdbc连接字符串 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- username + - 描述:数据源的用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- password + - 描述:数据源指定用户名的密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- schema + - 描述:写入数据库所在schema + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- column + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 字段类型:List + - 默认值:无 + +
+ +- preSql + - 描述:写入数据到目的表前,会先执行这里的一组标准语句 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
+ +- postSql + - 描述:写入数据到目的表后,会执行这里的一组标准语句 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
+ +- table + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- writeMode + - 描述:仅支持insert、update操作,可以搭配insertSqlMode使用 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- insertSqlMode + - 描述:控制写入数据到目标表采用 COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'语句,提高数据的插入效率 + - 注意: 目前该参数值固定传入 copy,否则抛出提示为not support insertSqlMode的RuntimeException。当指定此参数时,writeMode的值必须为 insert,否则设置无效 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- batchSize + - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 + - 必选:否 + - 字段类型:int + - 默认值:1024 + +
+ +## 四、配置示例 +1、insert +``` +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "kingbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:kingbase8://localhost:54321/test", + "table": [ + "tableTest" + ], + "schema":"test" + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` +2、 insert with copy mode +``` +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "kingbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:kingbase8://localhost:54321/ide", + "table": [ + "tableTest" + ], + "schema":"test" + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "insertSqlMode": "copy" + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` +3、 update +``` +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "kingbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:kingbase8://localhost:54321/ide", + "table": [ + "tableTest" + ], + "schema":"test" + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "update", + "updateKey": {"key": ["id"]}, + "column": ["id","user_id","name"], + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} +``` From e2035bf7e6049b19cc5a4f86f5061e9085d1aadf Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 13 Jan 2021 19:50:55 +0800 Subject: [PATCH 089/136] =?UTF-8?q?1=E3=80=81FlinkX=E5=8D=87=E7=BA=A7?= =?UTF-8?q?=E8=87=B31.11=202=E3=80=81=E5=88=A0=E9=99=A4kafka09=E6=8F=92?= =?UTF-8?q?=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/example/kafka09_stream.json | 36 - docs/questions.md | 9 +- docs/quickstart.md | 10 +- docs/realTime/reader/kafkareader.md | 53 +- docs/realTime/writer/kafkawriter.md | 81 +- flinkx-core/pom.xml | 33 +- .../com/dtstack/flinkx/util/DateUtil.java | 14 +- .../flinkx/emqx/format/EmqxOutputFormat.java | 5 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 2 + .../flinkx/ftp/reader/FtpInputFormat.java | 4 +- .../ftp/reader/FtpSeqBufferedReader.java | 22 +- .../hdfs/writer/HdfsOrcOutputFormat.java | 2 +- .../hdfs/writer/HdfsParquetOutputFormat.java | 2 +- .../hdfs/writer/HdfsTextOutputFormat.java | 5 +- .../flinkx-kafka09-reader/.gitignore | 13 - flinkx-kafka09/flinkx-kafka09-reader/pom.xml | 79 -- .../flinkx/kafka09/client/Kafka09Client.java | 93 -- .../kafka09/client/Kafka09Consumer.java | 45 - .../kafka09/format/Kafka09InputFormat.java | 80 -- .../flinkx/kafka09/reader/Kafka09Reader.java | 48 - .../flinkx-kafka09-writer/.gitignore | 13 - flinkx-kafka09/flinkx-kafka09-writer/pom.xml | 79 -- .../kafka09/format/Kafka09OutputFormat.java | 99 -- .../flinkx/kafka09/writer/Kafka09Writer.java | 70 -- flinkx-kafka09/pom.xml | 74 -- .../flinkx/kafkabase/KafkaConfigKeys.java | 6 - .../format/KafkaBaseInputFormat.java | 5 - .../format/KafkaBaseInputFormatBuilder.java | 4 - .../kafkabase/reader/KafkaBaseReader.java | 3 - flinkx-launcher/pom.xml | 8 +- .../launcher/perJob/FlinkPerJobUtil.java | 22 +- .../perJob/PerJobClusterClientBuilder.java | 10 +- .../deployment/ClusterSpecification.java | 19 +- .../flink/yarn/YarnClusterDescriptor.java | 899 ++++++++---------- .../src/main/resources/logback.xml | 22 + .../phoenix/format/PhoenixInputFormat.java | 3 +- .../phoenix/format/PhoenixOutputFormat.java | 3 +- .../phoenix5/format/Phoenix5InputFormat.java | 3 +- .../phoenix5/format/Phoenix5OutputFormat.java | 3 +- .../JdbcInputFormat.java | 4 +- flinkx-test/pom.xml | 12 - .../com/dtstack/flinkx/test/LocalTest.java | 4 - pom.xml | 8 +- 43 files changed, 548 insertions(+), 1461 deletions(-) delete mode 100644 docs/example/kafka09_stream.json delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/.gitignore delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/pom.xml delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java delete mode 100644 flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java delete mode 100644 flinkx-kafka09/flinkx-kafka09-writer/.gitignore delete mode 100644 flinkx-kafka09/flinkx-kafka09-writer/pom.xml delete mode 100644 flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java delete mode 100644 flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java delete mode 100644 flinkx-kafka09/pom.xml create mode 100644 flinkx-launcher/src/main/resources/logback.xml diff --git a/docs/example/kafka09_stream.json b/docs/example/kafka09_stream.json deleted file mode 100644 index cb69ecbee6..0000000000 --- a/docs/example/kafka09_stream.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "parameter": { - "topic": "kafka09", - "groupId": "default", - "codec": "text", - "encoding": "UTF-8", - "blankIgnore": false, - "consumerSettings": { - "zookeeper.connect": "localhost:2181/kafka09" - } - }, - "name": "kafka09reader" - }, - "writer": { - "parameter": { - "print": true - }, - "name": "streamwriter" - } - } - ], - "setting": { - "restore": { - "isRestore": false, - "isStream": true - }, - "speed": { - "channel": 1 - } - } - } -} \ No newline at end of file diff --git a/docs/questions.md b/docs/questions.md index 64073f9c6c..9d707038ba 100644 --- a/docs/questions.md +++ b/docs/questions.md @@ -12,9 +12,12 @@ ./install_jars.sh ``` -### 2.FlinkX版本需要与Flink版本保持一致 -1.8_release版本对应flink1.8 -1.10_release版本对应flink1.10 版本 +### 2.FlinkX版本需要与Flink版本保持一致,最好小版本也保持一致 +| FlinkX分支 | Flink版本 | +| --- | --- | +| 1.8_release | Flink1.8.3 | +| 1.10_release | Flink1.10.1 | +| 1.11_release | Flink1.11.3 | 不对应在standalone和yarn session模式提交时,会报错: Caused by: java.io.InvalidClassException: org.apache.flink.api.common.operators.ResourceSpec; incompatible types for field cpuCores diff --git a/docs/quickstart.md b/docs/quickstart.md index 7fda36c0d9..cb346ac199 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -10,14 +10,14 @@ cd flinkx 2.直接下载源码 ``` -wget https://github.com/DTStack/flinkx/archive/1.10_release.zip -unzip 1.10_release.zip -cd 1.10_release +wget https://github.com/DTStack/flinkx/archive/1.11_release.zip +unzip 1.11_release.zip +cd 1.11_release ``` 3.直接下载源码和编译好的插件包(推荐) ``` -wget https://github.com/DTStack/flinkx/releases/download/1.10.4/flinkx.7z +wget https://github.com/DTStack/flinkx/releases/download/1.11.0/flinkx.7z 7za x flinkx.7z cd flinkx ``` @@ -253,7 +253,7 @@ bin/flinkx \ | **jobid** | 任务名称 | 无 | 否 | Flink Job | | **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/syncplugins | | **flinkconf** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | -| **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | +| **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.11.3/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | | **yarnconf** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | | **queue** | yarn队列,如default | 无 | 否 | default | | **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index bd94bc72ad..826a136e7a 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -1,14 +1,14 @@ # Kafka Reader ## 一、插件名称 -kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: +kafka插件存在三个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: | kafka版本 | 插件名称 | | --- | --- | -| kafka 0.9 | kafka09reader | | kafka 0.10 | kafka10reader | | kafka 0.11 | kafka11reader | | kafka 1.0及以后 | kafkareader | +注:从FlinkX1.11版本开始不再支持kafka 0.9 @@ -125,9 +125,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 - 必选:是 - 字段类型:Map - 默认值:无 - - 注意: - - kafka09 reader插件: consumerSettings必须至少包含`zookeeper.connect`参数 - - kafka09 reader以外的插件:consumerSettings必须至少包含`bootstrap.servers`参数 + - 注意:consumerSettings必须至少包含`bootstrap.servers`参数 - 如: ```json { @@ -139,44 +137,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 ## 三、配置示例 -#### 1、kafka09 -```json -{ - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "topic" : "kafka09", - "groupId" : "default", - "codec" : "text", - "encoding": "UTF-8", - "blankIgnore": false, - "consumerSettings" : { - "zookeeper.connect" : "localhost:2181/kafka09" - } - }, - "name" : "kafka09reader" - }, - "writer" : { - "parameter" : { - "print" : true - }, - "name" : "streamwriter" - } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "speed" : { - "channel" : 1 - } - } - } -} -``` -#### 2、kafka10 +#### 1、kafka10 ```json { "job": { @@ -215,7 +176,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 3、kafka11 +#### 2、kafka11 ```json { "job" : { @@ -252,7 +213,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 4、kafka +#### 3、kafka ```json { "job" : { @@ -291,7 +252,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 5、kafka->Hive +#### 4、kafka->Hive ```json { "job": { diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md index 453e011411..6f958cc1de 100644 --- a/docs/realTime/writer/kafkawriter.md +++ b/docs/realTime/writer/kafkawriter.md @@ -1,15 +1,14 @@ # Kafka Writer ## 一、插件名称 -kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: +kafka插件存在三个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: | kafka版本 | 插件名称 | | --- | --- | -| kafka 0.9 | kafka09writer | | kafka 0.10 | kafka10writer | | kafka 0.11 | kafka11writer | | kafka 1.0及以后 | kafkawriter | - +注:从FlinkX1.11版本开始不再支持kafka 0.9 ## 二、参数说明 @@ -30,29 +29,12 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略
-- **encoding** - - 描述:编码 - - 注意:该参数只对kafka09reader插件有效 - - 必选:否 - - 字段类型:String - - 默认值:UTF-8 - -
- -- **brokerList** - - 描述:kafka broker地址列表 - - 注意:该参数只对kafka09writer插件有效 - - 必选:kafka09writer必选,其它kafka writer插件不用填 - - 字段类型:String - - 默认值:无 - -
- - **producerSettings** - 描述:kafka连接配置,支持所有`org.apache.kafka.clients.producer.ProducerConfig`中定义的配置 - - 必选:对于非kafka09 writer插件,该参数必填,且producerSettings中至少包含`bootstrap.servers`参数 + - 必选:是 - 字段类型:Map - 默认值:无 + - 注意:producerSettings中至少包含`bootstrap.servers`参数
@@ -68,54 +50,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 ## 三、配置示例 -#### 1、kafka09 -```json -{ - "job": { - "content": [{ - "reader": { - "name": "streamreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "id" - }, - { - "name": "user_id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "sliceRecordCount" : ["100"] - } - }, - "writer" : { - "parameter": { - "timezone": "UTC", - "topic": "kafka09", - "encoding": "UTF-8", - "brokerList": "0.0.0.1:9092", - "tableFields": ["id","user_id","name"] - }, - "name": "kafka09writer" - } - } ], - "setting": { - "restore" : { - "isStream" : true - }, - "speed" : { - "channel" : 1 - } - } - } -} -``` -#### 2、kafka10 +#### 1、kafka10 ```json { "job": { @@ -163,7 +98,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 3、kafka11 +#### 2、kafka11 ```json { "job": { @@ -212,7 +147,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 4、kafka +#### 3、kafka ```json { "job": { @@ -260,7 +195,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 5、MySQL->kafka +#### 4、MySQL->kafka ```json { "job" : { diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index fd42d2ca17..b09612dffc 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -27,7 +27,7 @@ org.slf4j slf4j-log4j12 - 1.7.10 + 1.7.30 @@ -51,32 +51,21 @@ org.apache.flink - flink-runtime-web_2.11 + flink-streaming-java_${scala.binary.version} ${flink.version} - org.apache.flink - flink-streaming-java_2.11 - ${flink.version} - - - org.xerial.snappy - snappy-java - - - - - - org.apache.flink - flink-clients_2.11 + flink-clients_${scala.binary.version} ${flink.version} + + org.apache.flink - flink-hadoop-compatibility_2.11 + flink-hadoop-compatibility_${scala.binary.version} ${flink.version} @@ -86,15 +75,9 @@ - - commons-cli - commons-cli - 1.2 - - org.apache.flink - flink-yarn_2.11 + flink-yarn_${scala.binary.version} ${flink.version} @@ -106,7 +89,7 @@ org.apache.flink - flink-queryable-state-runtime_2.11 + flink-queryable-state-runtime_${scala.binary.version} ${flink.version} diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index 919a515ecd..660afb3f2f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -42,6 +42,8 @@ public class DateUtil { private static final String STANDARD_DATETIME_FORMAT = "standardDatetimeFormatter"; + private static final String STANDARD_DATETIME_FORMAT_FOR_MILLISECOND= "standardDatetimeFormatterForMillisecond"; + private static final String UN_STANDARD_DATETIME_FORMAT = "unStandardDatetimeFormatter"; private static final String DATE_FORMAT = "dateFormatter"; @@ -67,7 +69,6 @@ public class DateUtil { TimeZone timeZone = TimeZone.getTimeZone(TIME_ZONE); Map formatterMap = new HashMap<>(); - SimpleDateFormat standardDatetimeFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); standardDatetimeFormatter.setTimeZone(timeZone); formatterMap.put(STANDARD_DATETIME_FORMAT,standardDatetimeFormatter); @@ -88,7 +89,11 @@ public class DateUtil { yearFormatter.setTimeZone(timeZone); formatterMap.put(YEAR_FORMAT,yearFormatter); - return formatterMap; + SimpleDateFormat standardDatetimeFormatterOfMillisecond = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + standardDatetimeFormatterOfMillisecond.setTimeZone(timeZone); + formatterMap.put(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND,standardDatetimeFormatterOfMillisecond); + + return formatterMap; }); private DateUtil() {} @@ -236,6 +241,11 @@ public static SimpleDateFormat getDateTimeFormatter(){ return datetimeFormatter.get().get(STANDARD_DATETIME_FORMAT); } + //获取毫秒级别的日期解析 + public static SimpleDateFormat getDateTimeFormatterForMillisencond(){ + return datetimeFormatter.get().get(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND); + } + public static SimpleDateFormat getDateFormatter(){ return datetimeFormatter.get().get(DATE_FORMAT); } diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java index fc15e5c688..9cac743adb 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java @@ -21,9 +21,9 @@ import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.types.Row; -import com.fasterxml.jackson.databind.ObjectMapper; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttConnectOptions; import org.eclipse.paho.client.mqttv3.MqttException; @@ -54,7 +54,6 @@ public class EmqxOutputFormat extends BaseRichOutputFormat { private transient MqttClient client; protected static JsonDecoder jsonDecoder = new JsonDecoder(); - protected static ObjectMapper objectMapper = new ObjectMapper(); @Override @@ -96,7 +95,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { }else{ map = Collections.singletonMap("message", row.toString()); } - MqttMessage message = new MqttMessage(objectMapper.writeValueAsString(map).getBytes()); + MqttMessage message = new MqttMessage(MapUtil.writeValueAsString(map).getBytes()); message.setQos(qos); client.publish(topic, message); } catch (Throwable e) { diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index aa8a3d083a..d51fd3adb4 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -69,6 +69,8 @@ public void loginFtpServer(FtpConfig ftpConfig) { // 不需要写死ftp server的OS TYPE,FTPClient getSystemType()方法会自动识别 ftpClient.setConnectTimeout(ftpConfig.getTimeout()); ftpClient.setDataTimeout(ftpConfig.getTimeout()); + //设置控制连接超时 + ftpClient.setSoTimeout(ftpConfig.getTimeout()); if (EFtpMode.PASV.name().equals(ftpConfig.getConnectPattern())) { ftpClient.enterRemotePassiveMode(); ftpClient.enterLocalPassiveMode(); diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index f90c860ef1..92042c9a25 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -94,10 +94,10 @@ public void openInternal(InputSplit split) throws IOException { List paths = inputSplit.getPaths(); if (ftpConfig.getIsFirstLineHeader()){ - br = new FtpSeqBufferedReader(ftpHandler,paths.iterator()); + br = new FtpSeqBufferedReader(ftpHandler,paths.iterator(),ftpConfig); br.setFromLine(1); } else { - br = new FtpSeqBufferedReader(ftpHandler,paths.iterator()); + br = new FtpSeqBufferedReader(ftpHandler,paths.iterator(),ftpConfig); br.setFromLine(0); } br.setFileEncoding(ftpConfig.getEncoding()); diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java index 49badaec33..277fe09225 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.ftp.reader; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.ftp.FtpHandler; import org.slf4j.Logger; @@ -49,9 +51,13 @@ public class FtpSeqBufferedReader { private String fileEncoding; - public FtpSeqBufferedReader(IFtpHandler ftpHandler, Iterator iter) { + //ftp配置信息 + private FtpConfig ftpConfig; + + public FtpSeqBufferedReader(IFtpHandler ftpHandler, Iterator iter, FtpConfig ftpConfig) { this.ftpHandler = ftpHandler; this.iter = iter; + this.ftpConfig = ftpConfig; } public String readLine() throws IOException{ @@ -97,7 +103,19 @@ public void close() throws IOException { br = null; if (ftpHandler instanceof FtpHandler){ - ((FtpHandler) ftpHandler).getFtpClient().completePendingCommand(); + try { + ((FtpHandler) ftpHandler).getFtpClient().completePendingCommand(); + } catch (Exception e) { + //如果出现了超时异常,就直接获取一个新的ftpHandler + LOG.warn("FTPClient completePendingCommand has error ->",e); + try{ + ftpHandler.logoutFtpServer(); + }catch (Exception exception){ + LOG.warn("FTPClient logout has error ->",exception); + } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); + ftpHandler.loginFtpServer(ftpConfig); + } } } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 22925d99eb..be11aacc12 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -240,7 +240,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe case VARCHAR: case CHAR: if (column instanceof Timestamp){ - SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); recordList.add(fm.format(column)); }else if (column instanceof Map || column instanceof List){ recordList.add(gson.toJson(column)); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 84cfa835a1..f635c61348 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -238,7 +238,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ case "varchar" : case "string" : if (valObj instanceof Timestamp){ - val=DateUtil.getDateTimeFormatter().format(valObj); + val=DateUtil.getDateTimeFormatterForMillisencond().format(valObj); group.add(colName,val); }else if (valObj instanceof Map || valObj instanceof List){ group.add(colName,gson.toJson(valObj)); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index 542f692704..e73ac5d850 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -23,9 +23,6 @@ import com.dtstack.flinkx.hdfs.ECompressType; import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.DateUtil; -import com.dtstack.flinkx.util.GsonUtil; -import com.google.gson.JsonElement; -import com.google.gson.JsonParser; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.apache.flink.types.Row; @@ -209,7 +206,7 @@ private void appendDataToString(StringBuilder sb, Object column, ColumnType colu case VARCHAR: case CHAR: if (column instanceof Timestamp){ - SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); sb.append(fm.format(column)); }else if (column instanceof Map || column instanceof List){ sb.append(gson.toJson(column)); diff --git a/flinkx-kafka09/flinkx-kafka09-reader/.gitignore b/flinkx-kafka09/flinkx-kafka09-reader/.gitignore deleted file mode 100644 index ca7ca55c4c..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/.gitignore +++ /dev/null @@ -1,13 +0,0 @@ -target -.idea/ -/.idea/* -*.pyc -*.swp -.DS_Store -/target -target -.class -.project -.classpath -*.eclipse.* -*.iml diff --git a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml b/flinkx-kafka09/flinkx-kafka09-reader/pom.xml deleted file mode 100644 index 7e90aa9af0..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/pom.xml +++ /dev/null @@ -1,79 +0,0 @@ - - - - flinkx-kafka09 - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-kafka09-reader - - - - com.dtstack.flinkx - flinkx-kb-reader - 1.6 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - com.google.common - shade.core.com.google.common - - - com.google.thirdparty - shade.core.com.google.thirdparty - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java deleted file mode 100644 index faacd2710d..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Client.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.client; - -import com.dtstack.flinkx.decoder.IDecode; -import com.dtstack.flinkx.kafkabase.client.IClient; -import com.dtstack.flinkx.kafkabase.entity.kafkaState; -import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; -import com.dtstack.flinkx.util.ExceptionUtil; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaStream; -import kafka.message.MessageAndMetadata; -import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; - -/** - * Date: 2019/12/25 - * Company: www.dtstack.com - * - * @author tudou - */ -public class Kafka09Client implements IClient { - - private static final Logger LOG = LoggerFactory.getLogger(Kafka09Client.class); - - private volatile boolean running = true; - private KafkaStream mStream; - private IDecode decode; - private KafkaBaseInputFormat format; - - public Kafka09Client(KafkaStream aStream, KafkaBaseInputFormat format) { - this.mStream = aStream; - this.decode = format.getDecode(); - this.format = format; - } - - @Override - public void run() { - Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { - LOG.warn("KafkaClient run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); - }); - try { - while (running) { - ConsumerIterator it = mStream.iterator(); - while (it.hasNext()) { - String m = null; - try { - MessageAndMetadata next = it.next(); - processMessage(new String(next.message(), format.getEncoding()), - next.topic(), - next.partition(), - next.offset(), - null); - } catch (Exception e) { - LOG.error("process event = {}, e = {}", m, ExceptionUtil.getErrorMessage(e)); - } - } - } - } catch (Exception t) { - LOG.error("kafka Consumer fetch error, e = {}", ExceptionUtil.getErrorMessage(t)); - } - } - - @Override - public void processMessage(String message, String topic, Integer partition, Long offset, Long timestamp) { - Map event = decode.decode(message); - if (event != null && event.size() > 0) { - format.processEvent(Pair.of(event, new kafkaState(topic, partition, offset, timestamp))); - } - } - - @Override - public void close() { - running = false; - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java deleted file mode 100644 index 110ef449d5..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/client/Kafka09Consumer.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.client; - -import com.dtstack.flinkx.kafkabase.KafkaInputSplit; -import com.dtstack.flinkx.kafkabase.client.KafkaBaseConsumer; -import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; -import kafka.consumer.KafkaStream; - -import java.util.Properties; - -/** - * @company: www.dtstack.com - * @author: toutian - * @create: 2019/7/5 - */ -public class Kafka09Consumer extends KafkaBaseConsumer { - private KafkaStream mStream; - - public Kafka09Consumer(KafkaStream aStream) { - super(new Properties()); - this.mStream = aStream; - } - - @Override - public KafkaBaseConsumer createClient(String topic, String group, KafkaBaseInputFormat format, KafkaInputSplit kafkaInputSplit) { - client = new Kafka09Client(mStream, format); - return this; - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java deleted file mode 100644 index bc4bcd9992..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09InputFormat.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.format; - -import com.dtstack.flinkx.kafka09.client.Kafka09Consumer; -import com.dtstack.flinkx.kafkabase.KafkaInputSplit; -import com.dtstack.flinkx.kafkabase.enums.KafkaVersion; -import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormat; -import com.dtstack.flinkx.kafkabase.util.KafkaUtil; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.KafkaStream; -import kafka.javaapi.consumer.ConsumerConnector; -import org.apache.flink.core.io.InputSplit; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - * @company: www.dtstack.com - * @author: toutian - * @create: 2019/7/5 - */ -public class Kafka09InputFormat extends KafkaBaseInputFormat { - - private transient ConsumerConnector consumerConnector; - - @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - Properties props = KafkaUtil.geneConsumerProp(consumerSettings, mode); - consumerConnector = kafka.consumer.Consumer.createJavaConsumerConnector(new ConsumerConfig(props)); - } - - @Override - protected void openInternal(InputSplit inputSplit) { - Map topicCountMap = Collections.singletonMap(topic, 1); - Map>> consumerMap = consumerConnector.createMessageStreams(topicCountMap); - - List> streams = consumerMap.get(topic); - for (final KafkaStream stream : streams) { - consumer = new Kafka09Consumer(stream); - } - consumer.createClient(topic, groupId, this, (KafkaInputSplit)inputSplit).execute(); - running = true; - } - - @Override - protected void closeInternal() { - if (running) { - consumerConnector.commitOffsets(true); - consumerConnector.shutdown(); - consumer.close(); - running = false; - LOG.warn("input kafka release."); - } - } - - @Override - public KafkaVersion getKafkaVersion() { - return KafkaVersion.kafka09; - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java b/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java deleted file mode 100644 index 31c6c6c5b4..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-reader/src/main/java/com/dtstack/flinkx/kafka09/reader/Kafka09Reader.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.reader; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.kafka09.format.Kafka09InputFormat; -import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; -import com.dtstack.flinkx.kafkabase.format.KafkaBaseInputFormatBuilder; -import com.dtstack.flinkx.kafkabase.reader.KafkaBaseReader; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** - * @company: www.dtstack.com - * @author: toutian - * @create: 2019/7/4 - */ -public class Kafka09Reader extends KafkaBaseReader { - - public Kafka09Reader(DataTransferConfig config, StreamExecutionEnvironment env) { - super(config, env); - //兼容历史脚本 - String id = consumerSettings.get(KafkaConfigKeys.GROUP_ID); - if(StringUtils.isNotBlank(id)){ - super.groupId = id; - } - } - - @Override - public KafkaBaseInputFormatBuilder getBuilder(){ - return new KafkaBaseInputFormatBuilder(new Kafka09InputFormat()); - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-writer/.gitignore b/flinkx-kafka09/flinkx-kafka09-writer/.gitignore deleted file mode 100644 index ca7ca55c4c..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-writer/.gitignore +++ /dev/null @@ -1,13 +0,0 @@ -target -.idea/ -/.idea/* -*.pyc -*.swp -.DS_Store -/target -target -.class -.project -.classpath -*.eclipse.* -*.iml diff --git a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml b/flinkx-kafka09/flinkx-kafka09-writer/pom.xml deleted file mode 100644 index b774b700be..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-writer/pom.xml +++ /dev/null @@ -1,79 +0,0 @@ - - - - flinkx-kafka09 - com.dtstack.flinkx - 1.6 - - 4.0.0 - - flinkx-kafka09-writer - - - - com.dtstack.flinkx - flinkx-kb-writer - 1.6 - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.0 - - - package - - shade - - - false - - - com.google.common - shade.core.com.google.common - - - com.google.thirdparty - shade.core.com.google.thirdparty - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java deleted file mode 100644 index 7d792f2619..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.format; - -import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; -import com.dtstack.flinkx.kafkabase.util.Formatter; -import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; -import com.dtstack.flinkx.util.MapUtil; -import org.apache.flink.configuration.Configuration; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.internals.DefaultPartitioner; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.TimeUnit; - -/** - * @company: www.dtstack.com - * @author: toutian - * @create: 2019/7/5 - */ -public class Kafka09OutputFormat extends KafkaBaseOutputFormat { - - private String encoding; - private String brokerList; - private transient KafkaProducer producer; - private HeartBeatController heartBeatController; - - @Override - public void configure(Configuration parameters) { - props.put("key.serializer", org.apache.kafka.common.serialization.StringSerializer.class.getName()); - props.put("value.serializer", org.apache.kafka.common.serialization.StringSerializer.class.getName()); - props.put("producer.type", "sync"); - props.put("compression.codec", "none"); - props.put("request.required.acks", "1"); - props.put("batch.num.messages", "1024"); - props.put("partitioner.class", DefaultPartitioner.class.getName()); - - props.put("client.id", ""); - - if (producerSettings != null) { - props.putAll(producerSettings); - } - props.put("metadata.broker.list", brokerList); - producer = new KafkaProducer<>(props); - - super.configure(parameters); - } - - @Override - protected void emit(Map event) throws IOException { - heartBeatController.acquire(); - String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { - if (Objects.nonNull(exception)) { - LOG.warn("kafka writeSingleRecordInternal error:{}", exception.getMessage(), exception); - heartBeatController.onFailed(exception); - } else { - heartBeatController.onSuccess(); - } - }); - } - - @Override - public void closeInternal() { - LOG.info("kafka output closeInternal."); - //未设置具体超时时间 关闭时间默认是long.value 导致整个方法长时间等待关闭不了,因此明确指定20s时间 - producer.close(KafkaBaseOutputFormat.CLOSE_TIME, TimeUnit.MILLISECONDS); - } - - public void setEncoding(String encoding) { - this.encoding = encoding; - } - - public void setBrokerList(String brokerList) { - this.brokerList = brokerList; - } - - public void setHeartBeatController(HeartBeatController heartBeatController) { - this.heartBeatController = heartBeatController; - } -} diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java deleted file mode 100644 index 4caa61ba91..0000000000 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/writer/Kafka09Writer.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.kafka09.writer; - -import com.dtstack.flinkx.config.DataTransferConfig; -import com.dtstack.flinkx.config.WriterConfig; -import com.dtstack.flinkx.kafka09.format.Kafka09OutputFormat; -import com.dtstack.flinkx.kafkabase.KafkaConfigKeys; -import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; -import com.dtstack.flinkx.kafkabase.writer.KafkaBaseWriter; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.types.Row; - -import java.nio.charset.StandardCharsets; - -/** - * @company: www.dtstack.com - * @author: toutian - * @create: 2019/7/4 - */ -public class Kafka09Writer extends KafkaBaseWriter { - - private String encoding; - private String brokerList; - - public Kafka09Writer(DataTransferConfig config) { - super(config); - WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); - encoding = writerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_ENCODING, StandardCharsets.UTF_8.name()); - brokerList = writerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_BROKER_LIST); - if (StringUtils.isBlank(brokerList)) { - throw new RuntimeException("brokerList can not be empty!"); - } - } - - @Override - public DataStreamSink writeData(DataStream dataSet) { - Kafka09OutputFormat format = new Kafka09OutputFormat(); - format.setTimezone(timezone); - format.setEncoding(encoding); - format.setTopic(topic); - format.setTableFields(tableFields); - format.setBrokerList(brokerList); - format.setProducerSettings(producerSettings); - format.setRestoreConfig(restoreConfig); - format.setHeartBeatController(new HeartBeatController()); - - format.setDirtyPath(dirtyPath); - format.setDirtyHadoopConfig(dirtyHadoopConfig); - format.setSrcFieldNames(srcCols); - return createOutput(dataSet, format); - } -} diff --git a/flinkx-kafka09/pom.xml b/flinkx-kafka09/pom.xml deleted file mode 100644 index 3a8a141fcc..0000000000 --- a/flinkx-kafka09/pom.xml +++ /dev/null @@ -1,74 +0,0 @@ - - - - flinkx-all - com.dtstack.flinkx - 1.6 - - 4.0.0 - pom - - flinkx-kafka09 - - - flinkx-kafka09-reader - flinkx-kafka09-writer - - - - - com.dtstack.flinkx - flinkx-core - 1.6 - provided - - - ch.qos.logback - logback-classic - - - ch.qos.logback - logback-core - - - - - org.apache.kafka - kafka_2.11 - 0.9.0.1 - - - slf4j-api - org.slf4j - - - slf4j-log4j12 - org.slf4j - - - log4j - log4j - - - scala-library - org.scala-lang - - - netty - io.netty - - - snappy-java - org.xerial.snappy - - - junit - junit - - - - - - \ No newline at end of file diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java index ccc5bf84db..afe9c94a22 100755 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java @@ -39,12 +39,6 @@ public class KafkaConfigKeys { public static final String KEY_OFFSET = "offset"; public static final String KEY_TIMESTAMP = "timestamp"; - /** - * kafka 09 - */ - public static final String KEY_ENCODING = "encoding"; - public static final String KEY_BROKER_LIST = "brokerList"; - public static final String GROUP_ID = "group.id"; public static final String BOOTSTRAP_SERVERS= "bootstrap.servers"; diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java index 678832cd3f..90ef168fa2 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java @@ -60,7 +60,6 @@ public class KafkaBaseInputFormat extends BaseRichInputFormat { protected String groupId; protected String codec; protected boolean blankIgnore; - protected String encoding; protected StartupMode mode; protected String offset; protected Long timestamp; @@ -175,10 +174,6 @@ public Object getState(){ return formatState == null ? null : formatState.getState(); } - public String getEncoding() { - return encoding; - } - public IDecode getDecode() { return decode; } diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java index b140a6747d..f46655dbf6 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormatBuilder.java @@ -64,10 +64,6 @@ public void setConsumerSettings(Map consumerSettings) { format.consumerSettings = consumerSettings; } - public void setEncoding(String encoding) { - format.encoding = encoding; - } - public void setMode(StartupMode mode) { format.mode = mode; } diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java index dacfc4ba0e..614dbbd7ae 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/reader/KafkaBaseReader.java @@ -29,7 +29,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -59,7 +58,6 @@ public KafkaBaseReader(DataTransferConfig config, StreamExecutionEnvironment env groupId = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_GROUP_ID, "default"); codec = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_CODEC, "text"); blankIgnore = readerConfig.getParameter().getBooleanVal(KafkaConfigKeys.KEY_BLANK_IGNORE, false); - encoding = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_ENCODING, StandardCharsets.UTF_8.name()); mode = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_MODE, StartupMode.GROUP_OFFSETS.name); offset = readerConfig.getParameter().getStringVal(KafkaConfigKeys.KEY_OFFSET, ""); timestamp = readerConfig.getParameter().getLongVal(KafkaConfigKeys.KEY_TIMESTAMP, -1L); @@ -76,7 +74,6 @@ public DataStream readData() { builder.setGroupId(groupId); builder.setCodec(codec); builder.setBlankIgnore(blankIgnore); - builder.setEncoding(encoding); builder.setConsumerSettings(consumerSettings); builder.setMode(StartupMode.getFromName(mode)); builder.setOffset(offset); diff --git a/flinkx-launcher/pom.xml b/flinkx-launcher/pom.xml index 9c52b9d958..8d26b84b5d 100644 --- a/flinkx-launcher/pom.xml +++ b/flinkx-launcher/pom.xml @@ -13,9 +13,9 @@ - ch.qos.logback - logback-classic - 1.1.7 + ch.qos.logback + logback-classic + 1.1.7 @@ -30,8 +30,6 @@ - - com.google.code.gson gson diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java index 2ee9028014..5718ba229a 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/FlinkPerJobUtil.java @@ -43,9 +43,9 @@ public class FlinkPerJobUtil { * the minimum memory should be higher than the min heap cutoff */ public final static int MIN_JM_MEMORY = 768; - public final static int MIN_TM_MEMORY = 768; - public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; - public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; + public final static int MIN_TM_MEMORY = 1024; + public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.process.size"; + public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.process.size"; public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; private static final Logger LOG = LoggerFactory.getLogger(FlinkPerJobUtil.class); @@ -56,16 +56,16 @@ public class FlinkPerJobUtil { * @return */ public static ClusterSpecification createClusterSpecification(Properties conProp) { - int jobmanagerMemoryMb = 768; - int taskmanagerMemoryMb = 768; + int jobManagerMemoryMb = 768; + int taskManagerMemoryMb = 1024; int slotsPerTaskManager = 1; if (conProp != null) { - if (conProp.containsKey(JOBMANAGER_MEMORY_MB)) { - jobmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); + if (conProp.contains(JOBMANAGER_MEMORY_MB)) { + jobManagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(JOBMANAGER_MEMORY_MB))); } - if (conProp.containsKey(TASKMANAGER_MEMORY_MB)) { - taskmanagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); + if (conProp.contains(TASKMANAGER_MEMORY_MB)) { + taskManagerMemoryMb = Math.max(MIN_JM_MEMORY, ValueUtil.getInt(conProp.getProperty(TASKMANAGER_MEMORY_MB))); } if (conProp.containsKey(SLOTS_PER_TASKMANAGER)) { slotsPerTaskManager = ValueUtil.getInt(conProp.get(SLOTS_PER_TASKMANAGER)); @@ -73,8 +73,8 @@ public static ClusterSpecification createClusterSpecification(Properties conProp } return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobmanagerMemoryMb) - .setTaskManagerMemoryMB(taskmanagerMemoryMb) + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) .setSlotsPerTaskManager(slotsPerTaskManager) .createClusterSpecification(); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java index cf8c8caef5..f13251a5ed 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/perJob/PerJobClusterClientBuilder.java @@ -26,8 +26,8 @@ import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; +import org.apache.flink.yarn.configuration.YarnLogConfigUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -99,13 +99,13 @@ public YarnClusterDescriptor createPerJobClusterDescriptor(Options launcherOptio throw new IllegalArgumentException("The Flink jar path is null"); } - File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + File log4j = new File(launcherOptions.getFlinkconf()+ File.separator + YarnLogConfigUtil.CONFIG_FILE_LOG4J_NAME); if(log4j.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME); + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + YarnLogConfigUtil.CONFIG_FILE_LOG4J_NAME); } else{ - File logback = new File(launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + File logback = new File(launcherOptions.getFlinkconf()+ File.separator + YarnLogConfigUtil.CONFIG_FILE_LOGBACK_NAME); if(logback.exists()){ - flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME); + flinkConfig.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, launcherOptions.getFlinkconf()+ File.separator + YarnLogConfigUtil.CONFIG_FILE_LOGBACK_NAME); } } diff --git a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java index 3f37518c41..f95543a97f 100644 --- a/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java +++ b/flinkx-launcher/src/main/java/org/apache/flink/client/deployment/ClusterSpecification.java @@ -19,10 +19,7 @@ package org.apache.flink.client.deployment; import org.apache.flink.client.program.PackagedProgram; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -62,20 +59,6 @@ private ClusterSpecification(int masterMemoryMB, int taskManagerMemoryMB, int nu this.priority = priority; } - public static ClusterSpecification fromConfiguration(Configuration configuration) { - int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - - int jobManagerMemoryMb = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); - int taskManagerMemoryMb = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - - return new ClusterSpecificationBuilder() - .setMasterMemoryMB(jobManagerMemoryMb) - .setTaskManagerMemoryMB(taskManagerMemoryMb) - .setNumberTaskManagers(1) - .setSlotsPerTaskManager(slots) - .createClusterSpecification(); - } - public PackagedProgram getProgram() { return program; } @@ -200,7 +183,7 @@ public String toString() { */ public static class ClusterSpecificationBuilder { private int masterMemoryMB = 768; - private int taskManagerMemoryMB = 768; + private int taskManagerMemoryMB = 1024; private int numberTaskManagers = 1; private int slotsPerTaskManager = 1; private int parallelism = 1; diff --git a/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 3827005790..3d13479ef5 100644 --- a/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flinkx-launcher/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -18,6 +18,9 @@ package org.apache.flink.yarn; +import com.dtstack.flinkx.constants.ConfigConstant; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.launcher.perJob.FlinkPerJobUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.cache.DistributedCache; @@ -26,32 +29,60 @@ import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClientProvider; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.client.program.rest.RestClusterClient; -import org.apache.flink.configuration.*; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.jobmanager.JobManagerProcessSpec; +import org.apache.flink.runtime.jobmanager.JobManagerProcessUtils; +import org.apache.flink.runtime.util.HadoopUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; +import org.apache.flink.yarn.configuration.YarnDeploymentTarget; +import org.apache.flink.yarn.configuration.YarnLogConfigUtil; +import org.apache.flink.yarn.entrypoint.YarnApplicationClusterEntryPoint; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -62,30 +93,36 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.*; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.net.URI; import java.net.URISyntaxException; import java.net.URLDecoder; import java.nio.charset.Charset; -import java.nio.file.FileVisitResult; -import java.nio.file.Files; -import java.nio.file.SimpleFileVisitor; -import java.nio.file.attribute.BasicFileAttributes; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; -import static com.dtstack.flinkx.constants.ConfigConstant.FLINK_PLUGIN_LOAD_MODE_KEY; -import static com.dtstack.flinkx.constants.ConstantValue.SHIP_FILE_PLUGIN_LOAD_MODE; -import static com.dtstack.flinkx.launcher.perJob.FlinkPerJobUtil.buildProgram; -import static com.dtstack.flinkx.launcher.perJob.FlinkPerJobUtil.getUrlFormat; import static org.apache.flink.configuration.ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; +import static org.apache.flink.yarn.YarnConfigKeys.LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR; /** * The descriptor with deployment information for deploying a Flink cluster on Yarn. @@ -106,11 +143,17 @@ public class YarnClusterDescriptor implements ClusterDescriptor { private final List shipFiles = new LinkedList<>(); private final String yarnQueue; + + private Path flinkJarPath; + private final Configuration flinkConfiguration; + private final String customName; + private final String nodeLabel; + private final String applicationType; - private Path flinkJarPath; + private String zookeeperNamespace; private YarnConfigOptions.UserJarInclusion userJarInclusion; @@ -142,161 +185,6 @@ public YarnClusterDescriptor( this.zookeeperNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, null); } - /** - * Uploads and registers a single resource and adds it to localResources. - * - * @param key - * the key to add the resource under - * @param fs - * the remote file system to upload to - * @param appId - * application ID - * @param localSrcPath - * local path to the file - * @param localResources - * map of resources - * - * @return the remote path to the uploaded resource - */ - private static Path setupSingleLocalResource( - String key, - FileSystem fs, - ApplicationId appId, - Path localSrcPath, - Map localResources, - Path targetHomeDir, - String relativeTargetPath) throws IOException { - Tuple2 resource = Utils.setupLocalResource( - fs, - appId.toString(), - localSrcPath, - targetHomeDir, - relativeTargetPath); - - localResources.put(key, resource.f1); - - return resource.f0; - } - - /** - * Match file name for "flink-dist*.jar" pattern. - * - * @param fileName file name to check - * @return true if file is a dist jar - */ - private static boolean isDistJar(String fileName) { - return fileName.startsWith("flink-dist") && fileName.endsWith("jar"); - } - - /** - * Recursively uploads (and registers) any (user and system) files in shipFiles except - * for files matching "flink-dist*.jar" which should be uploaded separately. - * - * @param shipFiles - * files to upload - * @param fs - * file system to upload to - * @param targetHomeDir - * remote home directory to upload to - * @param appId - * application ID - * @param remotePaths - * paths of the remote resources (uploaded resources will be added) - * @param localResources - * map of resources (uploaded resources will be added) - * @param localResourcesDirectory - * the directory the localResources are uploaded to - * @param envShipFileList - * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} - * - * @return list of class paths with the the proper resource keys from the registration - */ - static List uploadAndRegisterFiles( - Collection shipFiles, - FileSystem fs, - Path targetHomeDir, - ApplicationId appId, - List remotePaths, - Map localResources, - String localResourcesDirectory, - StringBuilder envShipFileList) throws IOException { - final List localPaths = new ArrayList<>(); - final List relativePaths = new ArrayList<>(); - for (File shipFile : shipFiles) { - if (shipFile.isDirectory()) { - // add directories to the classpath - final java.nio.file.Path shipPath = shipFile.toPath(); - final java.nio.file.Path parentPath = shipPath.getParent(); - Files.walkFileTree(shipPath, new SimpleFileVisitor() { - @Override - public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) { - localPaths.add(new Path(file.toUri())); - relativePaths.add(new Path(localResourcesDirectory, parentPath.relativize(file).toString())); - return FileVisitResult.CONTINUE; - } - }); - } else { - localPaths.add(new Path(shipFile.toURI())); - relativePaths.add(new Path(localResourcesDirectory, shipFile.getName())); - } - } - - final Set archives = new HashSet<>(); - final Set resources = new HashSet<>(); - for (int i = 0; i < localPaths.size(); i++) { - final Path localPath = localPaths.get(i); - final Path relativePath = relativePaths.get(i); - if (!isDistJar(relativePath.getName())) { - final String key = relativePath.toString(); - final Path remotePath = setupSingleLocalResource( - key, - fs, - appId, - localPath, - localResources, - targetHomeDir, - relativePath.getParent().toString()); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=").append(remotePath).append(","); - // add files to the classpath - if (key.endsWith("jar")) { - archives.add(relativePath.toString()); - } else { - resources.add(relativePath.getParent().toString()); - } - } - } - - // construct classpath, we always want resource directories to go first, we also sort - // both resources and archives in order to make classpath deterministic - final ArrayList classPaths = new ArrayList<>(); - resources.stream().sorted().forEach(classPaths::add); - archives.stream().sorted().forEach(classPaths::add); - return classPaths; - } - - private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(org.apache.flink.configuration.Configuration config) { - return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); - } - - private static boolean isUsrLibDirIncludedInShipFiles(List shipFiles) { - return shipFiles.stream() - .filter(File::isDirectory) - .map(File::getName) - .noneMatch(name -> name.equals(DEFAULT_FLINK_USR_LIB_DIR)); - } - - public static void logDetachedClusterInformation(ApplicationId yarnApplicationId, Logger logger) { - logger.info( - "The Flink YARN session cluster has been started in detached mode. In order to " + - "stop Flink gracefully, use the following command:\n" + - "$ echo \"stop\" | ./bin/yarn-session.sh -id {}\n" + - "If this should not be possible, then you can also kill Flink via YARN's web interface or via:\n" + - "$ yarn application -kill {}\n" + - "Note that killing Flink might not clean up all job artifacts and temporary files.", - yarnApplicationId, yarnApplicationId); - } - private Optional> decodeDirsToShipToCluster(final Configuration configuration) { checkNotNull(configuration); @@ -358,10 +246,6 @@ public Configuration getFlinkConfiguration() { return flinkConfiguration; } - // ------------------------------------------------------------- - // Lifecycle management - // ------------------------------------------------------------- - public void setLocalJarPath(Path localJarPath) { if (!localJarPath.toString().endsWith("jar")) { throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); @@ -369,27 +253,12 @@ public void setLocalJarPath(Path localJarPath) { this.flinkJarPath = localJarPath; } - // ------------------------------------------------------------- - // ClusterClient overrides - // ------------------------------------------------------------- - - /** - * Adds the given files to the list of files to ship. - * - *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by - * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, String, StringBuilder)} - * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. - * - * @param shipFiles files to ship - */ - public void addShipFiles(List shipFiles) { - checkArgument(userJarInclusion != YarnConfigOptions.UserJarInclusion.DISABLED || isUsrLibDirIncludedInShipFiles(shipFiles), - "This is an illegal ship directory : %s. When setting the %s to %s the name of ship directory can not be %s.", - ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR, - YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key(), - YarnConfigOptions.UserJarInclusion.DISABLED, - ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); - this.shipFiles.addAll(shipFiles); + private static String encodeYarnLocalResourceDescriptorListToString(List resources) { + return String.join( + LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR, + resources.stream() + .map(YarnLocalResourceDescriptor::toString) + .collect(Collectors.toList())); } private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws Exception { @@ -444,6 +313,10 @@ public String getNodeLabel() { return nodeLabel; } + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + @Override public void close() { if (!sharedYarnClient) { @@ -451,6 +324,10 @@ public void close() { } } + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + @Override public ClusterClientProvider retrieve(ApplicationId applicationId) throws ClusterRetrieveException { @@ -500,6 +377,10 @@ public ClusterClientProvider deploySessionCluster(ClusterSpecific } } + private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(org.apache.flink.configuration.Configuration config) { + return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + } + @Override public ClusterClientProvider deployJobCluster( ClusterSpecification clusterSpecification, @@ -517,13 +398,119 @@ public ClusterClientProvider deployJobCluster( } } + private static boolean isUsrLibDirIncludedInShipFiles(List shipFiles) { + return shipFiles.stream() + .filter(File::isDirectory) + .map(File::getName) + .noneMatch(name -> name.equals(DEFAULT_FLINK_USR_LIB_DIR)); + } + + public static void logDetachedClusterInformation(ApplicationId yarnApplicationId, Logger logger) { + logger.info( + "The Flink YARN session cluster has been started in detached mode. In order to " + + "stop Flink gracefully, use the following command:\n" + + "$ echo \"stop\" | ./bin/yarn-session.sh -id {}\n" + + "If this should not be possible, then you can also kill Flink via YARN's web interface or via:\n" + + "$ yarn application -kill {}\n" + + "Note that killing Flink might not clean up all job artifacts and temporary files.", + yarnApplicationId, yarnApplicationId); + } + + /** + * Adds the given files to the list of files to ship. + * + *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link YarnApplicationFileUploader#registerMultipleLocalResources(Collection, String)} + * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. + * + * @param shipFiles files to ship + */ + public void addShipFiles(List shipFiles) { + checkArgument(userJarInclusion != YarnConfigOptions.UserJarInclusion.DISABLED || isUsrLibDirIncludedInShipFiles(shipFiles), + "This is an illegal ship directory : %s. When setting the %s to %s the name of ship directory can not be %s.", + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR, + YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key(), + YarnConfigOptions.UserJarInclusion.DISABLED, + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); + this.shipFiles.addAll(shipFiles); + } + + @Override + public ClusterClientProvider deployApplicationCluster( + final ClusterSpecification clusterSpecification, + final ApplicationConfiguration applicationConfiguration) throws ClusterDeploymentException { + checkNotNull(clusterSpecification); + checkNotNull(applicationConfiguration); + + final YarnDeploymentTarget deploymentTarget = YarnDeploymentTarget.fromConfig(flinkConfiguration); + if (YarnDeploymentTarget.APPLICATION != deploymentTarget) { + throw new ClusterDeploymentException( + "Couldn't deploy Yarn Application Cluster." + + " Expected deployment.target=" + YarnDeploymentTarget.APPLICATION.getName() + + " but actual one was \"" + deploymentTarget.getName() + "\""); + } + + applicationConfiguration.applyToConfiguration(flinkConfiguration); + + final List pipelineJars = flinkConfiguration.getOptional(PipelineOptions.JARS).orElse(Collections.emptyList()); + Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + + try { + return deployInternal( + clusterSpecification, + "Flink Application Cluster", + YarnApplicationClusterEntryPoint.class.getName(), + null, + false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn Application Cluster", e); + } + } + + private void checkYarnQueues(YarnClient yarnClient) { + try { + List queues = yarnClient.getAllQueues(); + if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. + boolean queueFound = false; + for (QueueInfo queue : queues) { + if (queue.getQueueName().equals(this.yarnQueue)) { + queueFound = true; + break; + } + } + if (!queueFound) { + String queueNames = ""; + for (QueueInfo queue : queues) { + queueNames += queue.getQueueName() + ", "; + } + LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + + "Available queues: " + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + } catch (Throwable e) { + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Error details", e); + } + } + } + @Override public void killCluster(ApplicationId applicationId) throws FlinkException { try { yarnClient.killApplication(applicationId); - Utils.deleteApplicationFiles(Collections.singletonMap( - YarnConfigKeys.FLINK_YARN_FILES, - getYarnFilesDir(applicationId).toUri().toString())); + + try (final FileSystem fs = FileSystem.get(yarnConfiguration)) { + final Path applicationDir = YarnApplicationFileUploader + .getApplicationDirPath(fs.getHomeDirectory(), applicationId); + + Utils.deleteApplicationFiles(Collections.singletonMap( + YarnConfigKeys.FLINK_YARN_FILES, + applicationDir.toUri().toString())); + } + } catch (YarnException | IOException e) { throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); } @@ -545,18 +532,13 @@ private ClusterClientProvider deployInternal( @Nullable JobGraph jobGraph, boolean detached) throws Exception { - if (UserGroupInformation.isSecurityEnabled()) { - // note: UGI::hasKerberosCredentials inaccurately reports false - // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), - // so we check only in ticket cache scenario. + final UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + if (HadoopUtils.isKerberosSecurityEnabled(currentUser)) { boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); - UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); - if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS - && useTicketCache && !loginUser.hasKerberosCredentials()) { - LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); + if (!HadoopUtils.areKerberosCredentialsValid(currentUser, useTicketCache)) { throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + - "does not have Kerberos credentials"); + "does not have Kerberos credentials or delegation tokens!"); } } @@ -573,12 +555,12 @@ private ClusterClientProvider deployInternal( final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); if(clusterSpecification.isCreateProgramDelay()){ - String url = getUrlFormat(clusterSpecification.getYarnConfiguration(), yarnClient) + "/" + appResponse.getApplicationId().toString(); - PackagedProgram program = buildProgram(url,clusterSpecification); + String url = FlinkPerJobUtil.getUrlFormat(clusterSpecification.getYarnConfiguration(), yarnClient) + "/" + appResponse.getApplicationId().toString(); + PackagedProgram program = FlinkPerJobUtil.buildProgram(url,clusterSpecification); clusterSpecification.setProgram(program); jobGraph = PackagedProgramUtils.createJobGraph(program, clusterSpecification.getConfiguration(), clusterSpecification.getParallelism(), false); - String pluginLoadMode = clusterSpecification.getConfiguration().getString(FLINK_PLUGIN_LOAD_MODE_KEY); - if(StringUtils.equalsIgnoreCase(pluginLoadMode, SHIP_FILE_PLUGIN_LOAD_MODE)){ + String pluginLoadMode = clusterSpecification.getConfiguration().getString(ConfigConstant.FLINK_PLUGIN_LOAD_MODE_KEY); + if(StringUtils.equalsIgnoreCase(pluginLoadMode, ConstantValue.SHIP_FILE_PLUGIN_LOAD_MODE)){ jobGraph.getClasspaths().forEach(jarFile -> { try { shipFiles.add(new File(jarFile.toURI())); @@ -660,12 +642,8 @@ private ClusterSpecification validateClusterResources( int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); final int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); - if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { - LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " - + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + - "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + - "you requested will start."); - } + logIfComponentMemNotIntegerMultipleOfYarnMinAllocation("JobManager", jobManagerMemoryMb, yarnMinAllocationMB); + logIfComponentMemNotIntegerMultipleOfYarnMinAllocation("TaskManager", taskManagerMemoryMb, yarnMinAllocationMB); // set the memory to minAllocationMB to do the next checks correctly if (jobManagerMemoryMb < yarnMinAllocationMB) { @@ -705,33 +683,36 @@ private ClusterSpecification validateClusterResources( } - private void checkYarnQueues(YarnClient yarnClient) { + private void logIfComponentMemNotIntegerMultipleOfYarnMinAllocation( + String componentName, + int componentMemoryMB, + int yarnMinAllocationMB) { + int normalizedMemMB = (componentMemoryMB + (yarnMinAllocationMB - 1)) / yarnMinAllocationMB * yarnMinAllocationMB; + if (normalizedMemMB <= 0) { + normalizedMemMB = yarnMinAllocationMB; + } + if (componentMemoryMB != normalizedMemMB) { + LOG.info("The configured {} memory is {} MB. YARN will allocate {} MB to make up an integer multiple of its " + + "minimum allocation memory ({} MB, configured via 'yarn.scheduler.minimum-allocation-mb'). The extra {} MB " + + "may not be used by Flink.", componentName, componentMemoryMB, normalizedMemMB, yarnMinAllocationMB, + normalizedMemMB - componentMemoryMB); + } + } + + /** + * Kills YARN application and stops YARN client. + * + *

Use this method to kill the App before it has been properly deployed + */ + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { + LOG.info("Killing YARN application"); + try { - List queues = yarnClient.getAllQueues(); - if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. - boolean queueFound = false; - for (QueueInfo queue : queues) { - if (queue.getQueueName().equals(this.yarnQueue)) { - queueFound = true; - break; - } - } - if (!queueFound) { - String queueNames = ""; - for (QueueInfo queue : queues) { - queueNames += queue.getQueueName() + ", "; - } - LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + - "Available queues: " + queueNames); - } - } else { - LOG.debug("The YARN cluster does not have any queues configured"); - } - } catch (Throwable e) { - LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.debug("Error details", e); - } + yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); + } catch (Exception e) { + // we only log a debug message here because the "killApplication" call is a best-effort + // call (we don't know if the application has been deployed when the error occured). + LOG.debug("Error while killing YARN application", e); } } @@ -750,11 +731,7 @@ private ApplicationReport startAppMaster( configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); - // initialize file system - // Copy the application master jar to the filesystem - // Create a local resource to point to the destination jar path final FileSystem fs = FileSystem.get(yarnConfiguration); - final Path homeDir = fs.getHomeDirectory(); // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && @@ -765,10 +742,18 @@ private ApplicationReport startAppMaster( } ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + + final List providedLibDirs = getRemoteSharedPaths(configuration); + + final YarnApplicationFileUploader fileUploader = YarnApplicationFileUploader.from( + fs, + fs.getHomeDirectory(), + providedLibDirs, + appContext.getApplicationId(), + getFileReplication()); + // The files need to be shipped and added to classpath. Set systemShipFiles = new HashSet<>(shipFiles.size()); - // The files only need to be shipped. - Set shipOnlyFiles = new HashSet<>(); for (File file : shipFiles) { systemShipFiles.add(file.getAbsoluteFile()); } @@ -778,11 +763,6 @@ private ApplicationReport startAppMaster( systemShipFiles.add(new File(logConfigFilePath)); } - addLibFoldersToShipFiles(systemShipFiles); - - // Plugin files only need to be shipped and should not be added to classpath. - addPluginsFoldersToShipFiles(shipOnlyFiles); - // Set-up ApplicationSubmissionContext for the application final ApplicationId appId = appContext.getApplicationId(); @@ -814,21 +794,24 @@ private ApplicationReport startAppMaster( 1)); } - final Set userJarFiles = (jobGraph == null) - // not per-job submission - ? Collections.emptySet() - // add user code jars from the provided JobGraph - : jobGraph.getUserJars().stream().map(f -> f.toUri()).map(File::new).collect(Collectors.toSet()); + final Set userJarFiles = new HashSet<>(); + if (jobGraph != null) { + userJarFiles.addAll(jobGraph.getUserJars().stream().map(f -> f.toUri()).map(Path::new).collect(Collectors.toSet())); + } + + final List jarUrls = ConfigUtils.decodeListFromConfig(configuration, PipelineOptions.JARS, URI::create); + if (jarUrls != null && YarnApplicationClusterEntryPoint.class.getName().equals(yarnClusterEntrypoint)) { + userJarFiles.addAll(jarUrls.stream().map(Path::new).collect(Collectors.toSet())); + } // only for per job mode if (jobGraph != null) { for (Map.Entry entry : jobGraph.getUserArtifacts().entrySet()) { - org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(entry.getValue().filePath); // only upload local files - if (!path.getFileSystem().isDistributedFS()) { - Path localPath = new Path(path.getPath()); + if (!Utils.isRemotePath(entry.getValue().filePath)) { + Path localPath = new Path(entry.getValue().filePath); Tuple2 remoteFileInfo = - Utils.uploadLocalFileToRemote(fs, appId.toString(), localPath, homeDir, entry.getKey()); + fileUploader.uploadLocalFileToRemote(localPath, entry.getKey()); jobGraph.setUserArtifactRemotePath(entry.getKey(), remoteFileInfo.f0.toString()); } } @@ -836,45 +819,33 @@ private ApplicationReport startAppMaster( jobGraph.writeUserArtifactEntriesToConfiguration(); } - // local resource map for Yarn - final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); - // list of remote paths (after upload) - final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); - // ship list that enables reuse of resources for task manager containers - StringBuilder envShipFileList = new StringBuilder(); + if (providedLibDirs == null || providedLibDirs.isEmpty()) { + addLibFoldersToShipFiles(systemShipFiles); + } - // upload and register ship files, these files will be added to classpath. - List systemClassPaths = uploadAndRegisterFiles( - systemShipFiles, - fs, - homeDir, - appId, - paths, - localResources, - Path.CUR_DIR, - envShipFileList); + // Register all files in provided lib dirs as local resources with public visibility + // and upload the remaining dependencies as local resources with APPLICATION visibility. + final List systemClassPaths = fileUploader.registerProvidedLocalResources(); + final List uploadedDependencies = fileUploader.registerMultipleLocalResources( + systemShipFiles.stream().map(e -> new Path(e.toURI())).collect(Collectors.toSet()), + Path.CUR_DIR); + systemClassPaths.addAll(uploadedDependencies); // upload and register ship-only files - uploadAndRegisterFiles( - shipOnlyFiles, - fs, - homeDir, - appId, - paths, - localResources, - Path.CUR_DIR, - envShipFileList); - - final List userClassPaths = uploadAndRegisterFiles( + // Plugin files only need to be shipped and should not be added to classpath. + if (providedLibDirs == null || providedLibDirs.isEmpty()) { + Set shipOnlyFiles = new HashSet<>(); + addPluginsFoldersToShipFiles(shipOnlyFiles); + fileUploader.registerMultipleLocalResources( + shipOnlyFiles.stream().map(e -> new Path(e.toURI())).collect(Collectors.toSet()), + Path.CUR_DIR); + } + + // Upload and register user jars + final List userClassPaths = fileUploader.registerMultipleLocalResources( userJarFiles, - fs, - homeDir, - appId, - paths, - localResources, userJarInclusion == YarnConfigOptions.UserJarInclusion.DISABLED ? - ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR : Path.CUR_DIR, - envShipFileList); + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR : Path.CUR_DIR); if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { systemClassPaths.addAll(userClassPaths); @@ -896,17 +867,39 @@ private ApplicationReport startAppMaster( } // Setup jar for ApplicationMaster - Path remotePathJar = setupSingleLocalResource( - flinkJarPath.getName(), - fs, - appId, - flinkJarPath, - localResources, - homeDir, - ""); + final YarnLocalResourceDescriptor localResourceDescFlinkJar = fileUploader.uploadFlinkDist(flinkJarPath); + classPathBuilder.append(localResourceDescFlinkJar.getResourceKey()).append(File.pathSeparator); + + // write job graph to tmp file and add it to local resource + // TODO: server use user main method to generate job graph + if (jobGraph != null) { + File tmpJobGraphFile = null; + try { + tmpJobGraphFile = File.createTempFile(appId.toString(), null); + try (FileOutputStream output = new FileOutputStream(tmpJobGraphFile); + ObjectOutputStream obOutput = new ObjectOutputStream(output)) { + obOutput.writeObject(jobGraph); + } - paths.add(remotePathJar); - classPathBuilder.append(flinkJarPath.getName()).append(File.pathSeparator); + final String jobGraphFilename = "job.graph"; + configuration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); + + fileUploader.registerSingleLocalResource( + jobGraphFilename, + new Path(tmpJobGraphFile.toURI()), + "", + true, + false); + classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); + } catch (Exception e) { + LOG.warn("Add job graph to local resource fail."); + throw e; + } finally { + if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpJobGraphFile.toPath()); + } + } + } // Upload the flink configuration // write out configuration file @@ -916,16 +909,12 @@ private ApplicationReport startAppMaster( BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); String flinkConfigKey = "flink-conf.yaml"; - Path remotePathConf = setupSingleLocalResource( + fileUploader.registerSingleLocalResource( flinkConfigKey, - fs, - appId, new Path(tmpConfigurationFile.getAbsolutePath()), - localResources, - homeDir, - ""); - envShipFileList.append(flinkConfigKey).append("=").append(remotePathConf).append(","); - paths.add(remotePathConf); + "", + true, + true); classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); } finally { if (tmpConfigurationFile != null && !tmpConfigurationFile.delete()) { @@ -939,43 +928,6 @@ private ApplicationReport startAppMaster( } } - // write job graph to tmp file and add it to local resource - if (jobGraph != null) { - File tmpJobGraphFile = null; - try { - tmpJobGraphFile = File.createTempFile(appId.toString(), null); - try (FileOutputStream output = new FileOutputStream(tmpJobGraphFile); - ObjectOutputStream obOutput = new ObjectOutputStream(output);){ - obOutput.writeObject(jobGraph); - } - - final String jobGraphFilename = "job.graph"; - flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); - - Path pathFromYarnURL = setupSingleLocalResource( - jobGraphFilename, - fs, - appId, - new Path(tmpJobGraphFile.toURI()), - localResources, - homeDir, - ""); - paths.add(pathFromYarnURL); - classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); - } catch (Exception e) { - LOG.warn("Add job graph to local resource fail"); - throw e; - } finally { - if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { - LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); - } - } - } - - final Path yarnFilesDir = getYarnFilesDir(appId); - FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); - fs.setPermission(yarnFilesDir, permission); // set permission for path. - //To support Yarn Secure Integration Test Scenario //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML //and KRB5 configuration files. We are adding these files as container local resources for the container @@ -987,89 +939,93 @@ private ApplicationReport startAppMaster( File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); Path yarnSitePath = new Path(f.getAbsolutePath()); - remoteYarnSiteXmlPath = setupSingleLocalResource( + remoteYarnSiteXmlPath = fileUploader.registerSingleLocalResource( Utils.YARN_SITE_FILE_NAME, - fs, - appId, yarnSitePath, - localResources, - homeDir, - ""); + "", + false, + false).getPath(); String krb5Config = System.getProperty("java.security.krb5.conf"); if (krb5Config != null && krb5Config.length() != 0) { File krb5 = new File(krb5Config); LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); Path krb5ConfPath = new Path(krb5.getAbsolutePath()); - remoteKrb5Path = setupSingleLocalResource( + remoteKrb5Path = fileUploader.registerSingleLocalResource( Utils.KRB5_FILE_NAME, - fs, - appId, krb5ConfPath, - localResources, - homeDir, - ""); + "", + false, + false).getPath(); hasKrb5 = true; } } - // setup security tokens Path remotePathKeytab = null; + String localizedKeytabPath = null; String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); if (keytab != null) { - LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); - remotePathKeytab = setupSingleLocalResource( - Utils.KEYTAB_FILE_NAME, - fs, - appId, - new Path(keytab), - localResources, - homeDir, - ""); + boolean localizeKeytab = flinkConfiguration.getBoolean(YarnConfigOptions.SHIP_LOCAL_KEYTAB); + localizedKeytabPath = flinkConfiguration.getString(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + if (localizeKeytab) { + // Localize the keytab to YARN containers via local resource. + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = fileUploader.registerSingleLocalResource( + localizedKeytabPath, + new Path(keytab), + "", + false, + false).getPath(); + } else { + // // Assume Keytab is pre-installed in the container. + localizedKeytabPath = flinkConfiguration.getString(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + } } - final boolean hasLogback = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME); - final boolean hasLog4j = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOG4J_NAME); - + final JobManagerProcessSpec processSpec = JobManagerProcessUtils.processSpecFromConfigWithNewOptionToInterpretLegacyHeap( + flinkConfiguration, + JobManagerOptions.TOTAL_PROCESS_MEMORY); final ContainerLaunchContext amContainer = setupApplicationMasterContainer( yarnClusterEntrypoint, - hasLogback, - hasLog4j, hasKrb5, - clusterSpecification.getMasterMemoryMB()); + processSpec); + // setup security tokens if (UserGroupInformation.isSecurityEnabled()) { // set HDFS delegation tokens when security is enabled LOG.info("Adding delegation token to the AM container."); - Utils.setTokensFor(amContainer, paths, yarnConfiguration); + Utils.setTokensFor(amContainer, fileUploader.getRemotePaths(), yarnConfiguration); } - amContainer.setLocalResources(localResources); - fs.close(); + amContainer.setLocalResources(fileUploader.getRegisteredLocalResources()); + fileUploader.close(); // Setup CLASSPATH and environment variables for ApplicationMaster final Map appMasterEnv = new HashMap<>(); // set user specified app master environment variables appMasterEnv.putAll( - BootstrapTools.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); + ConfigurationUtils.getPrefixedKeyValuePairs(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); // set Flink app class path appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); // set Flink on YARN internal configuration values - appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); + appMasterEnv.put(YarnConfigKeys.FLINK_DIST_JAR, localResourceDescFlinkJar.toString()); appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fileUploader.getHomeDir().toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, encodeYarnLocalResourceDescriptorListToString(fileUploader.getEnvShipResourceList())); appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); - appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); + appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, fileUploader.getApplicationDir().toUri().toString()); // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - if (remotePathKeytab != null) { - appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); + if (localizedKeytabPath != null) { + appMasterEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localizedKeytabPath); String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remotePathKeytab.toString()); + } } //To support Yarn Secure Integration Test Scenario @@ -1113,7 +1069,7 @@ private ApplicationReport startAppMaster( setApplicationTags(appContext); // add a hook to clean up in case deployment fails - Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, yarnFilesDir); + Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, fileUploader.getApplicationDir()); Runtime.getRuntime().addShutdownHook(deploymentFailureHook); LOG.info("Submitting application master " + appId); yarnClient.submitApplication(appContext); @@ -1163,34 +1119,6 @@ private ApplicationReport startAppMaster( return report; } - /** - * Returns the Path where the YARN application files should be uploaded to. - * - * @param appId YARN application id - */ - private Path getYarnFilesDir(final ApplicationId appId) throws IOException { - final FileSystem fileSystem = FileSystem.get(yarnConfiguration); - final Path homeDir = fileSystem.getHomeDirectory(); - return new Path(homeDir, ".flink/" + appId + '/'); - } - - /** - * Kills YARN application and stops YARN client. - * - *

Use this method to kill the App before it has been properly deployed - */ - private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { - LOG.info("Killing YARN application"); - - try { - yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); - } catch (Exception e) { - // we only log a debug message here because the "killApplication" call is a best-effort - // call (we don't know if the application has been deployed when the error occured). - LOG.debug("Error while killing YARN application", e); - } - } - private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { List nodes = yarnClient.getNodeReports(NodeState.RUNNING); @@ -1289,38 +1217,30 @@ private void setApplicationNodeLabel(final ApplicationSubmissionContext appConte } } - @VisibleForTesting - void addLibFoldersToShipFiles(Collection effectiveShipFiles) { - // Add lib folder to the ship files if the environment variable is set. - // This is for convenience when running from the command-line. - // (for other files users explicitly set the ship files) - String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); - if (libDir != null) { - File directoryFile = new File(libDir); - if (directoryFile.isDirectory()) { - effectiveShipFiles.add(directoryFile); - } else { - throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + - "' is set to '" + libDir + "' but the directory doesn't exist."); - } - } else if (shipFiles.isEmpty()) { - LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + - "Not shipping any library files.", ENV_FLINK_LIB_DIR); - } + private int getFileReplication() { + final int yarnFileReplication = yarnConfiguration.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); + final int fileReplication = flinkConfiguration.getInteger(YarnConfigOptions.FILE_REPLICATION); + return fileReplication > 0 ? fileReplication : yarnFileReplication; } - @VisibleForTesting - void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { - final Optional pluginsDir = PluginConfig.getPluginsDir(); - pluginsDir.ifPresent(effectiveShipFiles::add); + private List getRemoteSharedPaths(Configuration configuration) throws IOException, FlinkException { + final List providedLibDirs = ConfigUtils.decodeListFromConfig( + configuration, YarnConfigOptions.PROVIDED_LIB_DIRS, Path::new); + + for (Path path : providedLibDirs) { + if (!Utils.isRemotePath(path.toString())) { + throw new FlinkException( + "The \"" + YarnConfigOptions.PROVIDED_LIB_DIRS.key() + "\" should only contain" + + " dirs accessible from all worker nodes, while the \"" + path + "\" is local."); + } + } + return providedLibDirs; } ContainerLaunchContext setupApplicationMasterContainer( String yarnClusterEntrypoint, - boolean hasLogback, - boolean hasLog4j, boolean hasKrb5, - int jobManagerMemoryMb) { + JobManagerProcessSpec processSpec) { // ------------------ Prepare Application Master Container ------------------------------ // respect custom JVM options in the YAML file @@ -1340,26 +1260,12 @@ ContainerLaunchContext setupApplicationMasterContainer( final Map startCommandValues = new HashMap<>(); startCommandValues.put("java", "$JAVA_HOME/bin/java"); - int heapSize = BootstrapTools.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); - String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); + String jvmHeapMem = JobManagerProcessUtils.generateJvmParametersStr(processSpec, flinkConfiguration); startCommandValues.put("jvmmem", jvmHeapMem); startCommandValues.put("jvmopts", javaOpts); - String logging = ""; - - if (hasLogback || hasLog4j) { - logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; + startCommandValues.put("logging", YarnLogConfigUtil.getLoggingYarnCommand(flinkConfiguration)); - if (hasLogback) { - logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; - } - - if (hasLog4j) { - logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; - } - } - - startCommandValues.put("logging", logging); startCommandValues.put("class", yarnClusterEntrypoint); startCommandValues.put("redirects", "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + @@ -1379,6 +1285,32 @@ ContainerLaunchContext setupApplicationMasterContainer( return amContainer; } + @VisibleForTesting + void addLibFoldersToShipFiles(Collection effectiveShipFiles) { + // Add lib folder to the ship files if the environment variable is set. + // This is for convenience when running from the command-line. + // (for other files users explicitly set the ship files) + String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); + if (libDir != null) { + File directoryFile = new File(libDir); + if (directoryFile.isDirectory()) { + effectiveShipFiles.add(directoryFile); + } else { + throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + + "' is set to '" + libDir + "' but the directory doesn't exist."); + } + } else if (shipFiles.isEmpty()) { + LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + + "Not shipping any library files.", ENV_FLINK_LIB_DIR); + } + } + + @VisibleForTesting + void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { + final Optional pluginsDir = PluginConfig.getPluginsDir(); + pluginsDir.ifPresent(effectiveShipFiles::add); + } + private void setClusterEntrypointInfoToConfig(final ApplicationReport report) { checkNotNull(report); @@ -1397,15 +1329,15 @@ private void setClusterEntrypointInfoToConfig(final ApplicationReport report) { flinkConfiguration.set(YarnConfigOptions.APPLICATION_ID, ConverterUtils.toString(clusterId)); } - private static class ClusterResourceDescription { - public final int totalFreeMemory; - public final int containerLimit; - public final int[] nodeManagersFree; + private static class YarnDeploymentException extends RuntimeException { + private static final long serialVersionUID = -812040641215388943L; - public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { - this.totalFreeMemory = totalFreeMemory; - this.containerLimit = containerLimit; - this.nodeManagersFree = nodeManagersFree; + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); } } @@ -1426,15 +1358,22 @@ private static class ApplicationSubmissionContextReflector { private static final ApplicationSubmissionContextReflector instance = new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); + + public static ApplicationSubmissionContextReflector getInstance() { + return instance; + } + private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; + private final Method applicationTagsMethod; private final Method attemptFailuresValidityIntervalMethod; private final Method keepContainersMethod; @Nullable private final Method nodeLabelExpressionMethod; + private ApplicationSubmissionContextReflector(Class clazz) { Method applicationTagsMethod; Method attemptFailuresValidityIntervalMethod; @@ -1488,10 +1427,6 @@ private ApplicationSubmissionContextReflector(Class applicationTags) throws InvocationTargetException, IllegalAccessException { @@ -1552,15 +1487,15 @@ public void setKeepContainersAcrossApplicationAttempts( } } - private static class YarnDeploymentException extends RuntimeException { - private static final long serialVersionUID = -812040641215388943L; - - public YarnDeploymentException(String message) { - super(message); - } + private static class ClusterResourceDescription { + public final int totalFreeMemory; + public final int containerLimit; + public final int[] nodeManagersFree; - public YarnDeploymentException(String message, Throwable cause) { - super(message, cause); + public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { + this.totalFreeMemory = totalFreeMemory; + this.containerLimit = containerLimit; + this.nodeManagersFree = nodeManagersFree; } } diff --git a/flinkx-launcher/src/main/resources/logback.xml b/flinkx-launcher/src/main/resources/logback.xml new file mode 100644 index 0000000000..0125d733de --- /dev/null +++ b/flinkx-launcher/src/main/resources/logback.xml @@ -0,0 +1,22 @@ + + + + + + + + + + ${CONSOLE_LOG_PATTERN} + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java index 656763e9b5..59602e71ee 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixInputFormat.java @@ -30,6 +30,7 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.types.Row; +import org.apache.flink.util.FlinkUserCodeClassLoader; import sun.misc.URLClassPath; import java.io.IOException; @@ -72,7 +73,7 @@ public void openInternal(InputSplit inputSplit) throws IOException { String[] alwaysParentFirstPatterns = new String[2]; alwaysParentFirstPatterns[0] = "org.apache.flink"; alwaysParentFirstPatterns[1] = "com.dtstack.flinkx"; - URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns); + URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns, FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER); ClassUtil.forName(driverName, childFirstClassLoader); diff --git a/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixOutputFormat.java b/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixOutputFormat.java index 35597aff05..303e438850 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixOutputFormat.java +++ b/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/format/PhoenixOutputFormat.java @@ -26,6 +26,7 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.io.FilenameUtils; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.util.FlinkUserCodeClassLoader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import sun.misc.URLClassPath; @@ -66,7 +67,7 @@ protected void openInternal(int taskNumber, int numTasks){ String[] alwaysParentFirstPatterns = new String[2]; alwaysParentFirstPatterns[0] = "org.apache.flink"; alwaysParentFirstPatterns[1] = "com.dtstack.flinkx"; - URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns); + URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns, FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER); ClassUtil.forName(driverName, childFirstClassLoader); dbConn = PhoenixUtil.getConnectionInternal(dbUrl, username, password, childFirstClassLoader); diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java index e9516404ab..8673089293 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java @@ -35,6 +35,7 @@ import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.types.Row; +import org.apache.flink.util.FlinkUserCodeClassLoader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; @@ -267,7 +268,7 @@ protected Connection getConnection() throws SQLException { list.add("org.apache.flink"); list.add("com.dtstack.flinkx"); - childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, list.toArray(new String[0])); + childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, list.toArray(new String[0]), FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER); ClassUtil.forName(driverName, childFirstClassLoader); if(StringUtils.isNotEmpty(username)){ diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java index 07efa749a3..6957a2c47b 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5OutputFormat.java @@ -27,6 +27,7 @@ import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.util.FlinkUserCodeClassLoader; import org.apache.phoenix.query.QueryServices; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +69,7 @@ protected void openInternal(int taskNumber, int numTasks){ String[] alwaysParentFirstPatterns = new String[2]; alwaysParentFirstPatterns[0] = "org.apache.flink"; alwaysParentFirstPatterns[1] = "com.dtstack.flinkx"; - URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns); + URLClassLoader childFirstClassLoader = FlinkUserCodeClassLoaders.childFirst(needJar.toArray(new URL[0]), parentClassLoader, alwaysParentFirstPatterns, FlinkUserCodeClassLoader.NOOP_EXCEPTION_HANDLER); ClassUtil.forName(driverName, childFirstClassLoader); if(StringUtils.isNotEmpty(username)){ diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index d4b91d450d..535f0ebbca 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -33,9 +33,9 @@ import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.RetryUtil; +import com.dtstack.flinkx.util.MapUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.UrlUtil; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -724,7 +724,7 @@ private void uploadMetricData() throws IOException { if (endLocationAccumulator != null) { metrics.put(Metrics.END_LOCATION, endLocationAccumulator.getLocalValue()); } - out.writeUTF(new ObjectMapper().writeValueAsString(metrics)); + out.writeUTF(MapUtil.writeValueAsString(metrics)); } catch (Exception e) { LOG.error("hadoop conf:{}", hadoopConfig); throw new IOException("Upload metric to HDFS error", e); diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index a544ea0ca3..f00d577077 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -334,18 +334,6 @@ 1.6 - - com.dtstack.flinkx - flinkx-kafka09-reader - 1.6 - - - - com.dtstack.flinkx - flinkx-kafka09-writer - 1.6 - - com.dtstack.flinkx flinkx-kudu-reader diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index c6d83f82db..e78846ad55 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -47,8 +47,6 @@ import com.dtstack.flinkx.hive.writer.HiveWriter; import com.dtstack.flinkx.kafka.reader.KafkaReader; import com.dtstack.flinkx.kafka.writer.KafkaWriter; -import com.dtstack.flinkx.kafka09.reader.Kafka09Reader; -import com.dtstack.flinkx.kafka09.writer.Kafka09Writer; import com.dtstack.flinkx.kafka10.reader.Kafka10Reader; import com.dtstack.flinkx.kafka10.writer.Kafka10Writer; import com.dtstack.flinkx.kafka11.reader.Kafka11Reader; @@ -211,7 +209,6 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstants.MONGODB_READER : reader = new MongodbReader(config, env); break; case PluginNameConstants.ODPS_READER : reader = new OdpsReader(config, env); break; case PluginNameConstants.BINLOG_READER : reader = new BinlogReader(config, env); break; - case PluginNameConstants.KAFKA09_READER : reader = new Kafka09Reader(config, env); break; case PluginNameConstants.KAFKA10_READER : reader = new Kafka10Reader(config, env); break; case PluginNameConstants.KAFKA11_READER : reader = new Kafka11Reader(config, env); break; case PluginNameConstants.KAFKA_READER : reader = new KafkaReader(config, env); break; @@ -249,7 +246,6 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ case PluginNameConstants.ODPS_WRITER : writer = new OdpsWriter(config); break; case PluginNameConstants.REDIS_WRITER : writer = new RedisWriter(config); break; case PluginNameConstants.HIVE_WRITER : writer = new HiveWriter(config); break; - case PluginNameConstants.KAFKA09_WRITER : writer = new Kafka09Writer(config); break; case PluginNameConstants.KAFKA10_WRITER : writer = new Kafka10Writer(config); break; case PluginNameConstants.KAFKA11_WRITER : writer = new Kafka11Writer(config); break; case PluginNameConstants.KUDU_WRITER : writer = new KuduWriter(config); break; diff --git a/pom.xml b/pom.xml index 274907c6a2..dabba8b30a 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,6 @@ flinkx-binlog flinkx-kb - flinkx-kafka09 flinkx-kafka10 flinkx-kafka11 flinkx-kafka @@ -62,11 +61,12 @@ UTF-8 - 1.10.1 + 1.11.3 + 2.12 2.7.3 4.5.3 ${basedir}/dev - release_1.10.4 + release_1.11.0 @@ -127,7 +127,7 @@ junit junit - 4.12 + 4.13.1 test From 0bf0f67ad0a802feeb90c297a7f5f5f47dbaf513 Mon Sep 17 00:00:00 2001 From: tudou Date: Thu, 14 Jan 2021 13:47:41 +0800 Subject: [PATCH 090/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DPhoenix=E6=8F=92?= =?UTF-8?q?=E4=BB=B6=E6=89=93=E5=8C=85=E5=90=8E=E5=8C=85=E5=90=8D=E6=A0=BC?= =?UTF-8?q?=E5=BC=8F=E4=B8=8D=E6=AD=A3=E7=A1=AE?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 2 +- flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml | 2 +- flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index a284085fa6..4f69ff0a3f 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../syncplugins/phoenixwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml index ab6fc2962a..e07787b6cf 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../syncplugins/phoenix5reader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml index 07912fc063..de68f42658 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../syncplugins/phoenix5writer/${project.name}-${package.name}.jar" /> From 3896bed2b635352c0a61fe4fc5c2e237e78e5b1b Mon Sep 17 00:00:00 2001 From: tudou Date: Thu, 14 Jan 2021 13:47:41 +0800 Subject: [PATCH 091/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8DPhoenix=E6=8F=92?= =?UTF-8?q?=E4=BB=B6=E6=89=93=E5=8C=85=E5=90=8E=E5=8C=85=E5=90=8D=E6=A0=BC?= =?UTF-8?q?=E5=BC=8F=E4=B8=8D=E6=AD=A3=E7=A1=AE?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-phoenix/flinkx-phoenix-writer/pom.xml | 2 +- flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml | 2 +- flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml index a284085fa6..4f69ff0a3f 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/pom.xml +++ b/flinkx-phoenix/flinkx-phoenix-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../syncplugins/phoenixwriter/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml index ab6fc2962a..e07787b6cf 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../syncplugins/phoenix5reader/${project.name}-${package.name}.jar" /> diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml index 07912fc063..de68f42658 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/pom.xml @@ -91,7 +91,7 @@ + tofile="${basedir}/../../syncplugins/phoenix5writer/${project.name}-${package.name}.jar" /> From 8743b8fb337cc506aeadf4f9577aa3af76c4764f Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 20 Jan 2021 19:28:05 +0800 Subject: [PATCH 092/136] =?UTF-8?q?=E4=BF=AE=E5=A4=8Dkafka=20reader?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E5=A4=9A=E5=8C=85=E8=A3=85=E4=BA=86=E4=B8=80?= =?UTF-8?q?=E5=B1=82=E4=BB=A5=E5=8F=8A=E5=88=A4=E6=96=AD=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java | 2 +- .../dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java index 48eccb16c2..18b9960fb9 100644 --- a/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java +++ b/flinkx-kafka/flinkx-kafka-reader/src/main/java/com/dtstack/flinkx/kafka/client/KafkaClient.java @@ -96,7 +96,7 @@ public KafkaClient(Properties clientProps, long pollTimeout, KafkaBaseInputForma "| therefore, no data will be read in this channel! |\n" + "****************************************************"); return; - }if(StartupMode.TIMESTAMP.equals(mode)){ + }else if(StartupMode.TIMESTAMP.equals(mode)){ Map timestampMap = new HashMap<>(Math.max((int) (stateList.size()/.75f) + 1, 16)); for (kafkaState state : stateList) { TopicPartition tp = new TopicPartition(state.getTopic(), state.getPartition()); diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java index 678832cd3f..c3cd2e307c 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java @@ -115,7 +115,7 @@ public void processEvent(Pair, kafkaState> pair) { try { Row row; if(CollectionUtils.isEmpty(metaColumns)){ - row = Row.of(Row.of(pair.getLeft())); + row = Row.of(pair.getLeft()); }else{ row = new Row(metaColumns.size()); for (int i = 0; i < metaColumns.size(); i++) { From f1b388ccbd27b92417927648d813563abeefcec3 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 26 Jan 2021 15:04:38 +0800 Subject: [PATCH 093/136] =?UTF-8?q?=E8=A7=A3=E5=86=B3standalone=E6=A8=A1?= =?UTF-8?q?=E5=BC=8F=E6=97=A5=E5=BF=97jar=E5=8C=85=E5=86=B2=E7=AA=81?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-core/pom.xml | 21 +++- .../inputformat/BaseRichInputFormat.java | 2 +- .../java/com/dtstack/flinkx/log/DtLogger.java | 108 +++++++++++------- flinkx-launcher/src/main/resources/log4j2.xml | 25 ++++ 4 files changed, 115 insertions(+), 41 deletions(-) create mode 100644 flinkx-launcher/src/main/resources/log4j2.xml diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index b09612dffc..316e487f70 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -24,12 +24,31 @@ 2.7 + + + org.apache.logging.log4j + log4j-core + 2.12.1 + + + + org.apache.logging.log4j + log4j-slf4j-impl + 2.12.1 + + org.slf4j - slf4j-log4j12 + slf4j-api 1.7.30 + + + + + + ch.qos.logback logback-classic diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java index 885dbbac4b..5c707926ed 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/inputformat/BaseRichInputFormat.java @@ -114,12 +114,12 @@ public final void configure(Configuration parameters) { @Override public void openInputFormat() throws IOException { - showConfig(); initJobInfo(); initPrometheusReporter(); startTime = System.currentTimeMillis(); DtLogger.config(logConfig, jobId); + showConfig(); } @Override diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/log/DtLogger.java b/flinkx-core/src/main/java/com/dtstack/flinkx/log/DtLogger.java index bc4f34b898..8bd3363018 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/log/DtLogger.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/log/DtLogger.java @@ -18,7 +18,6 @@ package com.dtstack.flinkx.log; import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.LoggerContext; import ch.qos.logback.classic.encoder.PatternLayoutEncoder; import ch.qos.logback.classic.filter.ThresholdFilter; import ch.qos.logback.core.rolling.RollingFileAppender; @@ -26,9 +25,17 @@ import ch.qos.logback.core.util.FileSize; import ch.qos.logback.core.util.OptionHelper; import com.dtstack.flinkx.config.LogConfig; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.PatternLayout; -import org.apache.log4j.varia.LevelRangeFilter; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.Appender; +import org.apache.logging.log4j.core.Filter; +import org.apache.logging.log4j.core.LoggerContext; +import org.apache.logging.log4j.core.appender.rolling.SizeBasedTriggeringPolicy; +import org.apache.logging.log4j.core.config.Configuration; +import org.apache.logging.log4j.core.config.LoggerConfig; +import org.apache.logging.log4j.core.filter.LevelRangeFilter; +import org.apache.logging.log4j.core.layout.PatternLayout; +import org.apache.logging.log4j.spi.StandardLevel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.impl.StaticLoggerBinder; @@ -45,14 +52,14 @@ public class DtLogger { private static Logger LOG = LoggerFactory.getLogger(DtLogger.class); private static boolean init = false; + public static final String LOG4J = "org.apache.logging.slf4j.Log4jLoggerFactory"; public static final String APPEND_NAME = "flinkx"; public static final String LOGGER_NAME = "com.dtstack"; - public static final String LOG4J = "org.slf4j.impl.Log4jLoggerFactory"; + private static boolean isLog4j2; public static final String LOGBACK = "ch.qos.logback.classic.util.ContextSelectorStaticBinder"; public static int LEVEL_INT = Integer.MAX_VALUE; - public static void config(LogConfig logConfig, String jobId) { if (logConfig == null || !logConfig.isLogger() || init) { return; @@ -67,58 +74,73 @@ public static void config(LogConfig logConfig, String jobId) { } String type = StaticLoggerBinder.getSingleton().getLoggerFactoryClassStr(); + LOG.info("current log type is {}", type); if (LOG4J.equalsIgnoreCase(type)) { configLog4j(logConfig, jobId); + isLog4j2 = true; } else if (LOGBACK.equalsIgnoreCase(type)) { configLogback(logConfig, jobId); + isLog4j2 = false; + }else{ + LOG.warn("log type {} is not [org.apache.logging.slf4j.Log4jLoggerFactory], either nor [ch.qos.logback.classic.util.ContextSelectorStaticBinder]", type); } init = true; } } - } + } private static void configLog4j(LogConfig logConfig, String jobId) { - org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(LOGGER_NAME); - org.apache.log4j.Level level = org.apache.log4j.Level.toLevel(logConfig.getLevel()); - LEVEL_INT = level.toInt(); + LOG.info("start to config log4j..."); + LoggerContext loggerContext = (LoggerContext) LogManager.getContext(); + Configuration config = loggerContext.getConfiguration(); + + org.apache.logging.log4j.Level level = org.apache.logging.log4j.Level.toLevel(logConfig.getLevel()); + LEVEL_INT = level.intLevel(); String pattern = logConfig.getPattern(); String path = logConfig.getPath(); - logger.removeAllAppenders(); - logger.setAdditivity(true); - logger.setLevel(level); - - org.apache.log4j.RollingFileAppender appender = new org.apache.log4j.RollingFileAppender(); - PatternLayout layout = new PatternLayout(); - if (StringUtils.isNotBlank(pattern)) { - layout.setConversionPattern(pattern); - } else { - layout.setConversionPattern(LogConfig.DEFAULT_LOG4J_PATTERN); + if (StringUtils.isBlank(pattern)) { + pattern = LogConfig.DEFAULT_LOG4J_PATTERN; } - LevelRangeFilter filter = new LevelRangeFilter(); - filter.setLevelMin(level); - appender.addFilter(filter); - appender.setLayout(layout); - appender.setFile(path + jobId + ".log"); - appender.setEncoding(StandardCharsets.UTF_8.name()); - appender.setMaxFileSize("1GB"); - appender.setMaxBackupIndex(1); - appender.setAppend(true); - appender.activateOptions(); - appender.setName(APPEND_NAME); + PatternLayout layout = PatternLayout.newBuilder() + .withCharset(StandardCharsets.UTF_8) + .withConfiguration(config) + .withPattern(pattern) + .build(); + + Filter filter = LevelRangeFilter.createFilter(org.apache.logging.log4j.Level.ERROR, + level, + Filter.Result.ACCEPT, + Filter.Result.DENY); + + Appender appender = org.apache.logging.log4j.core.appender.RollingFileAppender.newBuilder() + .withAppend(true) +// .setFilter(filter) + .withFileName(path + File.separator + jobId + ".log") + .withFilePattern(path + File.separator + jobId + ".%i.log") + .setName(APPEND_NAME) + .withPolicy(SizeBasedTriggeringPolicy.createPolicy("1GB")) + .setLayout(layout) + .setConfiguration(config) + .build(); + appender.start(); - logger.removeAllAppenders(); - logger.addAppender(appender); + for (final LoggerConfig loggerConfig : config.getLoggers().values()) { + loggerConfig.addAppender(appender, level, filter); + loggerConfig.setAdditive(false); + loggerConfig.setLevel(level); + } - logger.info("DtLogger config successfully, current log is [log4j]"); + LOG.info("DtLogger config successfully, current log is [log4j]"); } @SuppressWarnings("unchecked") private static void configLogback(LogConfig logConfig, String jobId) { - LoggerContext context = (LoggerContext) LoggerFactory.getILoggerFactory(); - ch.qos.logback.classic.Logger logger = context.getLogger(LOGGER_NAME); + LOG.info("start to config logback..."); + final ch.qos.logback.classic.LoggerContext context = (ch.qos.logback.classic.LoggerContext) LoggerFactory.getILoggerFactory(); + final ch.qos.logback.classic.Logger logger = context.getLogger(LOGGER_NAME); Level level = Level.toLevel(logConfig.getLevel()); LEVEL_INT = level.toInt(); @@ -166,14 +188,22 @@ private static void configLogback(LogConfig logConfig, String jobId) { logger.setAdditive(true); logger.addAppender(appender); - logger.info("DtLogger config successfully, current log is [logback]"); + LOG.info("DtLogger config successfully, current log is [logback]"); } public static boolean isEnableTrace(){ - return Level.TRACE_INT >= LEVEL_INT; + if(isLog4j2){ + return StandardLevel.TRACE.intLevel() >= LEVEL_INT; + }else{ + return Level.TRACE.levelInt >= LEVEL_INT; + } } public static boolean isEnableDebug(){ - return Level.DEBUG_INT >= LEVEL_INT; + if(isLog4j2){ + return StandardLevel.DEBUG.intLevel() >= LEVEL_INT; + }else{ + return Level.DEBUG.levelInt >= LEVEL_INT; + } } } diff --git a/flinkx-launcher/src/main/resources/log4j2.xml b/flinkx-launcher/src/main/resources/log4j2.xml new file mode 100644 index 0000000000..e9fc82c633 --- /dev/null +++ b/flinkx-launcher/src/main/resources/log4j2.xml @@ -0,0 +1,25 @@ + + + + + + %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n + + + + + + + + + + + + + + + ${pattern} + + + + \ No newline at end of file From 95e3d64a83893f8298e7c5beec2c768d5b4c73a3 Mon Sep 17 00:00:00 2001 From: cecotw <249508741@qq.com> Date: Wed, 3 Feb 2021 16:18:19 +0800 Subject: [PATCH 094/136] =?UTF-8?q?postgresql=E6=94=AF=E6=8C=81update?= =?UTF-8?q?=E5=A2=9E=E9=87=8F=E6=9B=B4=E6=96=B0=E6=93=8D=E4=BD=9C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../postgresql/PostgresqlDatabaseMeta.java | 23 +++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java index 4af93b9480..191dd0e493 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlDatabaseMeta.java @@ -20,7 +20,10 @@ import com.dtstack.flinkx.enums.EDatabaseType; import com.dtstack.flinkx.rdb.BaseDatabaseMeta; +import org.apache.commons.lang.StringUtils; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -73,10 +76,22 @@ public String getSqlQueryColumnFields(List column, String table) { "where attrelid = '%s' ::regclass and attnum > 0 and attisdropped = 'f'"; return String.format(sql,table); } - @Override public String getUpsertStatement(List column, String table, Map> updateKey) { - throw new UnsupportedOperationException("PostgreSQL not support update mode"); + return "INSERT INTO " + quoteTable(table) + + " (" + quoteColumns(column) + ") values " + + makeValues(column.size()) + + " ON CONFLICT (" + StringUtils.join(updateKey.get("key"), ",") + ") DO UPDATE SET " + + makeUpdatePart(column); + } + + private String makeUpdatePart (List column) { + List updateList = new ArrayList<>(); + for(String col : column) { + String quotedCol = quoteColumn(col); + updateList.add(quotedCol + "=excluded." + quotedCol); + } + return StringUtils.join(updateList, ","); } @Override @@ -98,4 +113,8 @@ public int getFetchSize(){ public int getQueryTimeout(){ return 1000; } + + private String makeValues(int nCols) { + return "(" + StringUtils.repeat("?", ",", nCols) + ")"; + } } From 5e6bd7ab885c1b2606bb41c541908731e8205ebc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 21 Jan 2021 00:47:42 +0000 Subject: [PATCH 095/136] Bump jackson-databind from 2.9.10.1 to 2.9.10.7 in /flinkx-core Bumps [jackson-databind](https://github.com/FasterXML/jackson) from 2.9.10.1 to 2.9.10.7. - [Release notes](https://github.com/FasterXML/jackson/releases) - [Commits](https://github.com/FasterXML/jackson/commits) Signed-off-by: dependabot[bot] --- flinkx-core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index fd42d2ca17..8cd5c95808 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -123,7 +123,7 @@ com.fasterxml.jackson.core jackson-databind - 2.9.10.1 + 2.9.10.7 io.prometheus From eb6907f03e2db31694ec2cb863913258b0b7b6fd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 12 Jan 2021 09:23:14 +0000 Subject: [PATCH 096/136] Bump junit from 4.12 to 4.13.1 Bumps [junit](https://github.com/junit-team/junit4) from 4.12 to 4.13.1. - [Release notes](https://github.com/junit-team/junit4/releases) - [Changelog](https://github.com/junit-team/junit4/blob/main/doc/ReleaseNotes4.12.md) - [Commits](https://github.com/junit-team/junit4/compare/r4.12...r4.13.1) Signed-off-by: dependabot[bot] --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 274907c6a2..a7d057c196 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,7 @@ junit junit - 4.12 + 4.13.1 test From f6ab97f55199ef90d55b8457afb60a5d4f65fbc5 Mon Sep 17 00:00:00 2001 From: huzk <1040080742@qq.com> Date: Thu, 4 Feb 2021 15:44:18 +0800 Subject: [PATCH 097/136] Update readme.md improve doc --- jars/readme.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/jars/readme.md b/jars/readme.md index e6a930809e..114b8a0ea6 100644 --- a/jars/readme.md +++ b/jars/readme.md @@ -20,6 +20,8 @@ mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversi mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=gbase-8.3.81.53.jar mvn install:install-file -DgroupId=com.dm -DartifactId=Dm7JdbcDriver18 -Dversion=7.6.0.197 -Dpackaging=jar -Dfile=Dm7JdbcDriver18.jar + +mvn install:install-file -DgroupId=com.kingbase8 -DartifactId=kingbase8 -Dversion=8.2.0 -Dpackaging=jar -Dfile=kingbase8-8.2.0.jar ``` 说明:这几个驱动包在我们自己搭建的仓库里有,并且这几个版本的驱动包在已经在生产环境中使用,所以不能很快修改版本,需要做相关测试,我们会在后期的版本中修改这两个驱动包的版本,可以先暂时下载安装驱动来解决。 From 84e19494a36545bb4056d38ce308b3b786eee810 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 5 Feb 2021 18:05:15 +0800 Subject: [PATCH 098/136] =?UTF-8?q?1=E3=80=81=E6=96=B0=E5=A2=9EOracle=20Lo?= =?UTF-8?q?gMiner=E5=AE=9E=E6=97=B6=E9=87=87=E9=9B=86=E6=8F=92=E4=BB=B6=20?= =?UTF-8?q?2=E3=80=81=E6=96=87=E6=A1=A3=E4=BC=98=E5=8C=96=203=E3=80=81fix?= =?UTF-8?q?=20#323=204=E3=80=81fix=20#309?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .gitlab-ci.yml | 11 - README.md | 21 +- README_CH.md | 20 +- bin/flinkx | 1 + ci/install_jars.sh | 13 - ci/sonar_notify.sh | 14 - docs/contribution.md | 40 +- docs/example/LogMiner_hive.json | 51 ++ docs/example/binlog_hive.json | 52 ++ docs/example/binlog_stream.json | 41 ++ docs/example/cassandra_stream.json | 54 ++ docs/example/clickhouse_stream.json | 52 ++ docs/example/db2_stream.json | 52 ++ docs/example/dm_stream.json | 55 ++ docs/example/gbase_stream.json | 48 ++ docs/example/greenplum_stream.json | 36 + docs/example/hbase_stream.json | 65 ++ docs/example/kingbase_stream.json | 10 +- docs/example/mysql_stream.json | 52 ++ docs/example/oracle_stream.json | 53 ++ docs/example/phoenix5_stream.json | 61 ++ docs/example/polardb_stream.json | 52 ++ docs/example/postgresql_stream.json | 48 ++ docs/example/saphana_stream.json | 87 +++ docs/example/sqlserver_stream.json | 59 ++ docs/example/stream_cassandra.json | 64 ++ docs/example/stream_clickhouse.json | 67 ++ docs/example/stream_db2.json | 56 ++ docs/example/stream_dm.json | 56 ++ docs/example/stream_gbase.json | 48 ++ docs/example/stream_greenplum.json | 45 ++ docs/example/stream_hbase.json | 104 +++ docs/example/stream_kingbase.json | 16 +- docs/example/stream_mysql.json | 67 ++ docs/example/stream_oracle.json | 67 ++ docs/example/stream_phoenix5.json | 73 ++ docs/example/stream_polardb.json | 67 ++ docs/example/stream_postgresql.json | 63 ++ docs/example/stream_redis.json | 65 ++ docs/example/stream_restapi.json | 59 ++ docs/example/stream_saphana.json | 67 ++ docs/example/stream_sqlserver.json | 64 ++ docs/images/LogMiner/LogMiner1.png | Bin 0 -> 41841 bytes docs/images/LogMiner/LogMiner10.png | Bin 0 -> 31812 bytes docs/images/LogMiner/LogMiner11.png | Bin 0 -> 45832 bytes docs/images/LogMiner/LogMiner12.png | Bin 0 -> 30107 bytes docs/images/LogMiner/LogMiner13.png | Bin 0 -> 31839 bytes docs/images/LogMiner/LogMiner14.png | Bin 0 -> 33540 bytes docs/images/LogMiner/LogMiner15.png | Bin 0 -> 31812 bytes docs/images/LogMiner/LogMiner16.png | Bin 0 -> 48114 bytes docs/images/LogMiner/LogMiner17.png | Bin 0 -> 30386 bytes docs/images/LogMiner/LogMiner18.png | Bin 0 -> 31839 bytes docs/images/LogMiner/LogMiner19.png | Bin 0 -> 65043 bytes docs/images/LogMiner/LogMiner2.png | Bin 0 -> 29283 bytes docs/images/LogMiner/LogMiner20.png | Bin 0 -> 31490 bytes docs/images/LogMiner/LogMiner21.png | Bin 0 -> 217935 bytes docs/images/LogMiner/LogMiner22.png | Bin 0 -> 1042855 bytes docs/images/LogMiner/LogMiner23.png | Bin 0 -> 715930 bytes docs/images/LogMiner/LogMiner3.png | Bin 0 -> 163416 bytes docs/images/LogMiner/LogMiner4.png | Bin 0 -> 32936 bytes docs/images/LogMiner/LogMiner5.png | Bin 0 -> 44106 bytes docs/images/LogMiner/LogMiner6.png | Bin 0 -> 68355 bytes docs/images/LogMiner/LogMiner7.png | Bin 0 -> 74021 bytes docs/images/LogMiner/LogMiner8.png | Bin 0 -> 110133 bytes docs/images/LogMiner/LogMiner9.png | Bin 0 -> 33540 bytes docs/images/quick_1.png | Bin 159395 -> 174058 bytes docs/images/quick_2.png | Bin 106263 -> 105940 bytes docs/images/quick_3.png | Bin 156375 -> 170235 bytes docs/images/quick_4.png | Bin 240317 -> 158846 bytes docs/images/quick_5.png | Bin 112928 -> 106919 bytes docs/images/quick_6.png | Bin 160393 -> 171128 bytes docs/images/quick_7.png | Bin 265959 -> 160656 bytes docs/images/quick_8.png | Bin 190881 -> 163608 bytes docs/offline/reader/carbondatareader.md | 23 +- docs/offline/reader/cassandrareader.md | 40 +- docs/offline/reader/clickhousereader.md | 176 +++-- docs/offline/reader/db2reader.md | 154 ++-- docs/offline/reader/dmreader.md | 152 ++-- docs/offline/reader/esreader.md | 36 +- docs/offline/reader/ftpreader.md | 172 +++-- docs/offline/reader/gbasereader.md | 169 +++-- docs/offline/reader/greenplumreader.md | 132 +++- docs/offline/reader/hbasereader.md | 174 +++-- docs/offline/reader/hdfsreader.md | 255 +++---- docs/offline/reader/kingbasereader.md | 172 +++-- docs/offline/reader/kudureader.md | 160 +++-- docs/offline/reader/mongodbreader.md | 27 +- docs/offline/reader/mysqlreader.md | 134 +++- docs/offline/reader/odpsreader.md | 120 ++-- docs/offline/reader/oraclereader.md | 136 +++- docs/offline/reader/phoenixreader.md | 147 ++-- docs/offline/reader/polardbreader.md | 136 +++- docs/offline/reader/postgresqlreader.md | 133 ++-- docs/offline/reader/saphanareader.md | 94 ++- docs/offline/reader/sqlserverreader.md | 153 ++-- docs/offline/reader/streamreader.md | 6 +- docs/offline/writer/carbondatawriter.md | 42 +- docs/offline/writer/cassandrawriter.md | 39 +- docs/offline/writer/clickhousewriter.md | 101 ++- docs/offline/writer/db2writer.md | 142 ++-- docs/offline/writer/dmwriter.md | 119 ++-- docs/offline/writer/eswriter.md | 26 +- docs/offline/writer/ftpwriter.md | 230 +++--- docs/offline/writer/gbasewriter.md | 102 +-- docs/offline/writer/greenplumwriter.md | 87 ++- docs/offline/writer/hbasewriter.md | 233 +++--- docs/offline/writer/hdfswriter.md | 529 +++++++------- docs/offline/writer/hivewriter.md | 317 +++++++-- docs/offline/writer/kingbasewriter.md | 145 ++-- docs/offline/writer/kuduwriter.md | 197 +++--- docs/offline/writer/mongodbwriter.md | 40 +- docs/offline/writer/mysqlwriter.md | 98 ++- docs/offline/writer/odpswriter.md | 114 +-- docs/offline/writer/oraclewriter.md | 90 ++- docs/offline/writer/phoenixwriter.md | 92 ++- docs/offline/writer/polardbwriter.md | 100 ++- docs/offline/writer/postgresqlwriter.md | 70 +- docs/offline/writer/rediswriter.md | 32 +- docs/offline/writer/saphanawriter.md | 76 +- docs/offline/writer/sqlserverwriter.md | 72 +- docs/quickstart.md | 217 ++++-- .../LogMiner\345\216\237\347\220\206.md" | 392 +++++++++++ .../LogMiner\351\205\215\347\275\256.md" | 434 ++++++++++++ ...06\345\217\212\351\205\215\347\275\256.md" | 237 +++++++ docs/realTime/reader/LogMiner.md | 236 +++++++ docs/realTime/reader/binlogreader.md | 477 ++++++++----- docs/realTime/reader/emqxreader.md | 49 +- docs/realTime/reader/kafkareader.md | 31 +- docs/realTime/reader/mongodboplogreader.md | 35 +- docs/realTime/reader/pgwalreader.md | 271 ++----- docs/realTime/reader/restapireader.md | 11 +- docs/realTime/writer/emqxwriter.md | 89 ++- docs/realTime/writer/kafkawriter.md | 28 +- docs/realTime/writer/restapiwriter.md | 18 +- .../flinkx/kafkabase/util/KafkaUtil.java | 2 + .../format/KafkaBaseInputFormat.java | 3 +- .../com/dtstack/flinkx/launcher/Launcher.java | 7 +- .../flinkx-oraclelogminer-reader/pom.xml | 106 +++ .../oraclelogminer/entity/QueueData.java | 52 ++ .../flinkx/oraclelogminer/format/LogFile.java | 87 +++ .../oraclelogminer/format/LogMinerConfig.java | 188 +++++ .../format/LogMinerConnection.java | 661 ++++++++++++++++++ .../format/LogMinerListener.java | 198 ++++++ .../oraclelogminer/format/LogParser.java | 249 +++++++ .../format/OracleLogMinerInputFormat.java | 108 +++ .../format/PositionManager.java | 37 + .../OracleLogMinerInputFormatBuilder.java | 241 +++++++ .../reader/OraclelogminerReader.java | 75 ++ .../flinkx/oraclelogminer/util/OraUtil.java | 65 ++ .../flinkx/oraclelogminer/util/SqlUtil.java | 401 +++++++++++ flinkx-oraclelogminer/pom.xml | 36 + flinkx-test/pom.xml | 6 + .../com/dtstack/flinkx/test/LocalTest.java | 2 + jars/readme.md | 28 +- pom.xml | 1 + 155 files changed, 10412 insertions(+), 2756 deletions(-) delete mode 100644 .gitlab-ci.yml delete mode 100644 ci/install_jars.sh delete mode 100644 ci/sonar_notify.sh create mode 100644 docs/example/LogMiner_hive.json create mode 100644 docs/example/binlog_hive.json create mode 100644 docs/example/binlog_stream.json create mode 100644 docs/example/cassandra_stream.json create mode 100644 docs/example/clickhouse_stream.json create mode 100644 docs/example/db2_stream.json create mode 100644 docs/example/dm_stream.json create mode 100644 docs/example/gbase_stream.json create mode 100644 docs/example/greenplum_stream.json create mode 100644 docs/example/hbase_stream.json create mode 100644 docs/example/mysql_stream.json create mode 100644 docs/example/oracle_stream.json create mode 100644 docs/example/phoenix5_stream.json create mode 100644 docs/example/polardb_stream.json create mode 100644 docs/example/postgresql_stream.json create mode 100644 docs/example/saphana_stream.json create mode 100644 docs/example/sqlserver_stream.json create mode 100644 docs/example/stream_cassandra.json create mode 100644 docs/example/stream_clickhouse.json create mode 100644 docs/example/stream_db2.json create mode 100644 docs/example/stream_dm.json create mode 100644 docs/example/stream_gbase.json create mode 100644 docs/example/stream_greenplum.json create mode 100644 docs/example/stream_hbase.json create mode 100644 docs/example/stream_mysql.json create mode 100644 docs/example/stream_oracle.json create mode 100644 docs/example/stream_phoenix5.json create mode 100644 docs/example/stream_polardb.json create mode 100644 docs/example/stream_postgresql.json create mode 100644 docs/example/stream_redis.json create mode 100644 docs/example/stream_restapi.json create mode 100644 docs/example/stream_saphana.json create mode 100644 docs/example/stream_sqlserver.json create mode 100644 docs/images/LogMiner/LogMiner1.png create mode 100644 docs/images/LogMiner/LogMiner10.png create mode 100644 docs/images/LogMiner/LogMiner11.png create mode 100644 docs/images/LogMiner/LogMiner12.png create mode 100644 docs/images/LogMiner/LogMiner13.png create mode 100644 docs/images/LogMiner/LogMiner14.png create mode 100644 docs/images/LogMiner/LogMiner15.png create mode 100644 docs/images/LogMiner/LogMiner16.png create mode 100644 docs/images/LogMiner/LogMiner17.png create mode 100644 docs/images/LogMiner/LogMiner18.png create mode 100644 docs/images/LogMiner/LogMiner19.png create mode 100644 docs/images/LogMiner/LogMiner2.png create mode 100644 docs/images/LogMiner/LogMiner20.png create mode 100644 docs/images/LogMiner/LogMiner21.png create mode 100644 docs/images/LogMiner/LogMiner22.png create mode 100644 docs/images/LogMiner/LogMiner23.png create mode 100644 docs/images/LogMiner/LogMiner3.png create mode 100644 docs/images/LogMiner/LogMiner4.png create mode 100644 docs/images/LogMiner/LogMiner5.png create mode 100644 docs/images/LogMiner/LogMiner6.png create mode 100644 docs/images/LogMiner/LogMiner7.png create mode 100644 docs/images/LogMiner/LogMiner8.png create mode 100644 docs/images/LogMiner/LogMiner9.png create mode 100644 "docs/realTime/other/LogMiner\345\216\237\347\220\206.md" create mode 100644 "docs/realTime/other/LogMiner\351\205\215\347\275\256.md" create mode 100644 "docs/realTime/other/PgWal\345\216\237\347\220\206\345\217\212\351\205\215\347\275\256.md" create mode 100644 docs/realTime/reader/LogMiner.md create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/entity/QueueData.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogParser.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/PositionManager.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/OraUtil.java create mode 100644 flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java create mode 100644 flinkx-oraclelogminer/pom.xml diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml deleted file mode 100644 index f29c0c55bc..0000000000 --- a/.gitlab-ci.yml +++ /dev/null @@ -1,11 +0,0 @@ -build: - stage: test - script: - - sh ci/install_jars.sh - - mvn clean org.jacoco:jacoco-maven-plugin:0.7.8:prepare-agent package -Dmaven.test.failure.ignore=true -q - - mvn sonar:sonar -Dsonar.projectKey="dt-insight-engine/flinkx" -Dsonar.host.url=http://172.16.100.198:9000 -Dsonar.jdbc.url=jdbc:postgresql://172.16.100.198:5432/sonar -Dsonar.java.binaries=target/sonar -Dsonar.login=11974c5e9a29625efa09fdc3c3fdc031efb1aab1 - - sh ci/sonar_notify.sh - only: - - 1.8_dev - tags: - - dt-insight-engine \ No newline at end of file diff --git a/README.md b/README.md index cd8aa4cea1..313ecf207e 100644 --- a/README.md +++ b/README.md @@ -11,12 +11,20 @@ English | [中文](README_CH.md) - We use [DingTalk](https://www.dingtalk.com/) to communicate,You can search the group number [**30537511**] or scan the QR code below to join the communication group -

- -
+
+ +
# Introduction +* **FlinkX is a distributed offline and real-time data synchronization framework based on flink widely used in 袋鼠云, which realizes efficient data migration between multiple heterogeneous data sources.** + +Different data sources are abstracted into different Reader plugins, and different data targets are abstracted into different Writer plugins. In theory, the FlinkX framework can support data synchronization of any data source type. As a set of ecosystems, every time a set of new data sources is connected, the newly added data sources can realize intercommunication with existing data sources. + +
+ +
+ FlinkX is a data synchronization tool based on Flink. FlinkX can collect static data, such as MySQL, HDFS, etc, as well as real-time changing data, such as MySQL binlog, Kafka, etc. FlinkX currently includes the following features: - Most plugins support concurrent reading and writing of data, which can greatly improve the speed of reading and writing; @@ -71,6 +79,13 @@ The following databases are currently supported: | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | +| | Oracle LogMiner| [doc](docs/realTime/reader/LogMiner.md) | | + +# Fundamental +In the underlying implementation, FlinkX relies on Flink, and the data synchronization task will be translated into StreamGraph and executed on Flink. The basic principle is as follows: +
+ +
# Quick Start diff --git a/README_CH.md b/README_CH.md index c181ce45c0..eb2b4ce4e3 100644 --- a/README_CH.md +++ b/README_CH.md @@ -22,11 +22,18 @@ Flink开发工程师JD要求:
- 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群 -
- -
+
+ +
# 介绍 +* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线和实时的数据同步框架,实现了多种异构数据源之间高效的数据迁移。** + +不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。 + +
+ +
FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集静态的数据,比如MySQL,HDFS等,也可以采集实时变化的数据,比如MySQL binlog,Kafka等。FlinkX目前包含下面这些特性: @@ -82,6 +89,13 @@ FlinkX目前支持下面这些数据库: | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | +| | Oracle LogMiner | [doc](docs/realTime/reader/LogMiner.md) | | + +# 基本原理 +在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,基本原理如下图: +
+ +
# 快速开始 diff --git a/bin/flinkx b/bin/flinkx index 5af671e003..f31a8ae7f0 100755 --- a/bin/flinkx +++ b/bin/flinkx @@ -38,3 +38,4 @@ CLASS_NAME=com.dtstack.flinkx.launcher.Launcher echo "flinkx starting ..." nohup $JAVA_RUN -cp $JAR_DIR $CLASS_NAME $@ & +tail -f nohup.out \ No newline at end of file diff --git a/ci/install_jars.sh b/ci/install_jars.sh deleted file mode 100644 index 23a0e3850c..0000000000 --- a/ci/install_jars.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -## db2 driver -mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=jars/db2jcc-3.72.44.jar - -## oracle driver -mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=jars/ojdbc8-12.2.0.1.jar - -## gbase driver -mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=jars/gbase-8.3.81.53.jar - -## dm driver -mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=jars/Dm7JdbcDriver18.jar \ No newline at end of file diff --git a/ci/sonar_notify.sh b/ci/sonar_notify.sh deleted file mode 100644 index 4ab9172260..0000000000 --- a/ci/sonar_notify.sh +++ /dev/null @@ -1,14 +0,0 @@ -#!/bin/bash -#参考钉钉文档 https://open-doc.dingtalk.com/microapp/serverapi2/qf2nxq - sonarreport=$(curl -s http://172.16.100.198:8082/?projectname=dt-insight-engine/flinkx) - curl -s "https://oapi.dingtalk.com/robot/send?access_token=e2718f7311243d2e58fa2695aa9c67a37760c7fce553311a32d53b3f092328ed" \ - -H "Content-Type: application/json" \ - -d "{ - \"msgtype\": \"markdown\", - \"markdown\": { - \"title\":\"sonar代码质量\", - \"text\": \"## sonar代码质量报告: \n -> [sonar地址](http://172.16.100.198:9000/dashboard?id=dt-insight-engine/flinkx) \n -> ${sonarreport} \n\" - } - }" \ No newline at end of file diff --git a/docs/contribution.md b/docs/contribution.md index 2063b96ddd..af06261f8a 100644 --- a/docs/contribution.md +++ b/docs/contribution.md @@ -13,8 +13,8 @@ ## 开发环境 -- Flink集群: 1.4及以上(单机模式不需要安装Flink集群) -- Java: JDK8及以上 +- Flink集群: 版本与FlinkX版本对应(单机模式不需要安装Flink集群) +- Java: JDK8 - 操作系统:理论上不限,但是目前只编写了shell启动脚本,用户可以可以参考shell脚本编写适合特定操作系统的启动脚本。 开发之前,需要理解以下概念: @@ -41,13 +41,13 @@ ## 插件入口类 -插件的入口类需继承**DataReader**和**DataWriter**,在内部获取任务json传来的参数,通过相应的**Builder**构建对应**InputFormat**和**OutputFormat**实例 +插件的入口类需继承**BaseDataReader**和**BaseDataWriter**,在内部获取任务json传来的参数,通过相应的**Builder**构建对应**InputFormat**和**OutputFormat**实例 ### DataReader ```java -public class SomeReader extends DataReader { +public class SomeReader extends BaseDataReader { protected String oneParameter; public SomeReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); @@ -59,7 +59,7 @@ public class SomeReader extends DataReader { } ``` -reader类需继承DataReader,同时重写readData方法。在构造函数中获取任务json中构建InputFormat所需要的参数,代码案例如下: +reader类需继承BaseDataReader,同时重写readData方法。在构造函数中获取任务json中构建InputFormat所需要的参数,代码案例如下: 构造方法 @@ -92,7 +92,7 @@ public DataStream readData() { ### DataWriter ```java -public class SomeWriter extends DataWriter { +public class SomeWriter extends BaseDataWriter { protected String oneParameter; public SomeWriter(DataTransferConfig config) { super(config); @@ -105,7 +105,7 @@ public class SomeWriter extends DataWriter { } ``` -和DataReader类似,writer需继承DataWriter,同时重写writeData方法。通常会创建一个ConfigKeys类,包含reader和writer所有需要的使用的任务json中参数的key。 +和DataReader类似,writer需继承BaseDataWriter,同时重写writeData方法。通常会创建一个ConfigKeys类,包含reader和writer所有需要的使用的任务json中参数的key。 构造方法 @@ -136,10 +136,10 @@ public DataStreamSink writeData(DataStream dataSet) { ### InputFormatBuilder的设计 -需继承**RichInputFormatBuilder** +需继承**BaseRichInputFormatBuilder** ```java -public class SomeInputFormatBuilder extends RichInputFormatBuilder { +public class SomeInputFormatBuilder extends BaseRichInputFormatBuilder { /** * 首先实例化一个InputFormat实例,通过构造函数传递,通过set方法设置参数 */ @@ -161,10 +161,10 @@ public class SomeInputFormatBuilder extends RichInputFormatBuilder { ### InputFormat的设计 -需继承**RichInputFormat**,根据任务逻辑分别实现 +需继承**BaseRichInputFormat**,根据任务逻辑分别实现 ```java -public class SomeInputFormat extends RichInputFormat { +public class SomeInputFormat extends BaseRichInputFormat { @override public void openInputFormat() { @@ -256,10 +256,10 @@ public class SomeInputFormat extends RichInputFormat { ### OutputFormatBuilder -需继承**RichOutputFormatBuilder**,和**InputFormatBuilder**相似 +需继承**BaseRichOutputFormatBuilder**,和**InputFormatBuilder**相似 ```java -public class SomeOutputFormatBuilder extends RichOutputFormatBuilder { +public class SomeOutputFormatBuilder extends BaseRichOutputFormatBuilder { /** * 首先实例化一个OutputFormat实例,通过构造函数传递,通过设计set方法设置参数 * 如下演示 @@ -282,10 +282,10 @@ public class SomeOutputFormatBuilder extends RichOutputFormatBuilder { ### OutputFormat -需继承**RichOutputFormat** +需继承**BaseRichOutputFormat** ```java -public class SomeOutputFormat extends RichOutputFormat { +public class SomeOutputFormat extends BaseRichOutputFormat { @Override protected void openInternal(int taskNumber, int numTasks) throws IOException {} @@ -451,10 +451,9 @@ public class Row implements Serializable{ ## 加载原理 -1. 框架扫描`plugin/reader`和`plugin/writer`目录,加载每个插件的`plugin.json`文件。 -1. 以`plugin.json`文件中`name`为key,索引所有的插件配置。如果发现重名的插件或者不存在的插件,框架会异常退出。 -1. 用户在插件中在`reader`/`writer`配置的`name`字段指定插件名字。框架根据插件的类型(`reader`/`writer`)和插件名称去插件的路径下扫描所有的jar,加入`classpath`。 -1. 根据插件配置中定义的入口类,框架通过反射实例化对应的`Job`对象。 +1. 用户在插件中在`reader`/`writer`配置的`name`字段指定插件名字。 +2. 框架根据插件的类型(`reader`/`writer`)和插件名称去插件的路径下扫描所有的jar,加入`classpath`。 +3. 根据插件配置中定义的入口类,框架通过反射实例化对应的`Job`对象。 ## 统一的目录结构 @@ -465,7 +464,6 @@ public class Row implements Serializable{ ``` ${Flinkx_HOME} |-- bin -| -- flink | -- flinkx.sh | |-- flinkx-somePlugin @@ -511,4 +509,4 @@ unix平台 mvn clean package -DskipTests -Prelease -DscriptType=sh ``` -打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,plugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! +打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,syncplugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! diff --git a/docs/example/LogMiner_hive.json b/docs/example/LogMiner_hive.json new file mode 100644 index 0000000000..c69d7492cf --- /dev/null +++ b/docs/example/LogMiner_hive.json @@ -0,0 +1,51 @@ +{ + "job" : { + "content" : [ { + "reader" : { + "parameter" : { + "schema" : "ROMA_LOGMINER", + "password" : "password", + "cat" : "insert,update,delete", + "jdbcUrl" : "jdbc:oracle:thin:@//localhost:1521/xib", + "readPosition" : "current", + "pavingData" : true, + "table" : [ "ROMA_LOGMINER.TEST" ], + "username" : "username" + }, + "name" : "oraclelogminerreader" + }, + "writer" : { + "parameter" : { + "writeMode" : "append", + "partitionType" : "DAY", + "tablesColumn" : "{\"TEST\":[{\"type\":\"VARCHAR2\",\"key\":\"before_ID\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR2\",\"key\":\"after_ID\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", + "partition" : "pt", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "fs.defaultFS" : "hdfs://ns1", + "dfs.namenode.rpc-address.ns1.nn2" : "ip2:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "ip1:9000", + "dfs.nameservices" : "ns1", + "fs.hdfs.impl.disable.cache" : "true", + "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + }, + "jdbcUrl" : "jdbc:hive2://ip3:8191/test", + "defaultFS" : "hdfs://ns1", + "fileType" : "parquet", + "charsetName" : "utf-8" + }, + "name" : "hivewriter" + } + } ], + "setting" : { + "restore" : { + "isRestore" : true, + "isStream" : true + }, + "speed" : { + "channel" : 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/binlog_hive.json b/docs/example/binlog_hive.json new file mode 100644 index 0000000000..1acd0e3b50 --- /dev/null +++ b/docs/example/binlog_hive.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "table": ["schema.table"], + "password": "passwd", + "database": "schema", + "port": 3306, + "cat": "insert,update,delete", + "host": "host", + "jdbcUrl": "jdbc:mysql://host:3306/schema", + "pavingData": true, + "username": "user" + }, + "name": "binlogreader" + }, + "writer": { + "parameter": { + "writeMode": "overwrite", + "partitionType": "DAY", + "tablesColumn" : "{\"CDC\":[{\"type\":\"VARCHAR2\",\"key\":\"before_ID\",\"comment\":\"\"},{\"comment\":\"\",\"type\":\"VARCHAR2\",\"key\":\"after_ID\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"type\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"schema\"},{\"comment\":\"\",\"type\":\"varchar\",\"key\":\"table\"},{\"comment\":\"\",\"type\":\"bigint\",\"key\":\"ts\"}]}", + "partition": "pt", + "hadoopConfig": { + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2": "host1:9000", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1": "host2:9000", + "dfs.nameservices": "ns1" + }, + "jdbcUrl": "jdbc:hive2://host1:10000/dev", + "defaultFS": "hdfs://ns1", + "fileType": "orc", + "charsetName": "utf-8", + "username": "admin" + }, + "name": "hivewriter" + } + } + ], + "setting": { + "restore": { + "isRestore": true, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/binlog_stream.json b/docs/example/binlog_stream.json new file mode 100644 index 0000000000..6606c89181 --- /dev/null +++ b/docs/example/binlog_stream.json @@ -0,0 +1,41 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "filter": "schema\\.table", + "password": "passwd", + "database": "database", + "port": 3306, + "start" : { + "journalName": "binlog.000031", + "timestamp": 1610525946000, + "position": 4 + }, + "cat": "DELETE,INSERT,UPDATE", + "host": "localhost", + "jdbcUrl": "jdbc:mysql://localhost:3306/schema", + "pavingData": true, + "username": "username" + }, + "name": "binlogreader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} \ No newline at end of file diff --git a/docs/example/cassandra_stream.json b/docs/example/cassandra_stream.json new file mode 100644 index 0000000000..b764be874c --- /dev/null +++ b/docs/example/cassandra_stream.json @@ -0,0 +1,54 @@ +{ + "job" : { + "content" : [ { + "reader": { + "name": "cassandrareader", + "parameter": { + "host": "localhost", + "port": 9042, + "username":"", + "password":"", + "useSSL":false, + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ] + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/clickhouse_stream.json b/docs/example/clickhouse_stream.json new file mode 100644 index 0000000000..eaa608cac7 --- /dev/null +++ b/docs/example/clickhouse_stream.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "int" + }, { + "name" : "user_id", + "type" : "int" + }, { + "name" : "name", + "type" : "string" + },{ + "name" : "eventDate", + "type" : "date" + } + ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:clickhouse://localhost:8123/database" ], + "table" : [ "test" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "clickhousereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/db2_stream.json b/docs/example/db2_stream.json new file mode 100644 index 0000000000..d835e22e86 --- /dev/null +++ b/docs/example/db2_stream.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint" + },{ + "name" : "name", + "type" : "varchar" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:db2://localhost:50002/database" ], + "table" : [ "TEST" ] + } ], + "where": "", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "db2reader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} \ No newline at end of file diff --git a/docs/example/dm_stream.json b/docs/example/dm_stream.json new file mode 100644 index 0000000000..5ca44a0240 --- /dev/null +++ b/docs/example/dm_stream.json @@ -0,0 +1,55 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "dmreader", + "parameter": { + "column": [ + { + "name": "ID", + "type": "int" + }, + { + "name": "AGE", + "type": "int" + } + ], + "increColumn": "", + "startLocation": "", + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] + } + ], + "where": "" + } + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} \ No newline at end of file diff --git a/docs/example/gbase_stream.json b/docs/example/gbase_stream.json new file mode 100644 index 0000000000..a8c26230d5 --- /dev/null +++ b/docs/example/gbase_stream.json @@ -0,0 +1,48 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint" + }, { + "name" : "user_id", + "type" : "bigint" + }, { + "name" : "name", + "type" : "varchar" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "gbasereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/greenplum_stream.json b/docs/example/greenplum_stream.json new file mode 100644 index 0000000000..68ec13a786 --- /dev/null +++ b/docs/example/greenplum_stream.json @@ -0,0 +1,36 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ {"name" : "id", "type": "int"}], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : ["jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database"], + "table" : ["table"] + } ], + "where": "", + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "greenplumreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/hbase_stream.json b/docs/example/hbase_stream.json new file mode 100644 index 0000000000..3662beb674 --- /dev/null +++ b/docs/example/hbase_stream.json @@ -0,0 +1,65 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "hbasereader", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase" + }, + "table": "sb5", + "encodig": "utf-8", + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ], + "range": { + "startRowkey": "", + "endRowkey": "", + "isBinaryRowkey": false + } + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} \ No newline at end of file diff --git a/docs/example/kingbase_stream.json b/docs/example/kingbase_stream.json index db34cbfdfd..9566ba92c9 100644 --- a/docs/example/kingbase_stream.json +++ b/docs/example/kingbase_stream.json @@ -4,12 +4,12 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": ["*"], "customSql": "", diff --git a/docs/example/mysql_stream.json b/docs/example/mysql_stream.json new file mode 100644 index 0000000000..a0702ed741 --- /dev/null +++ b/docs/example/mysql_stream.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/oracle_stream.json b/docs/example/oracle_stream.json new file mode 100644 index 0000000000..2ba2f47d34 --- /dev/null +++ b/docs/example/oracle_stream.json @@ -0,0 +1,53 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["TABLE"] + }], + "column": ["*"], + "customSql": "", + "where": "ID < 10000", + "splitPk": "", + "fetchSize": 1024, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "oraclereader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/phoenix5_stream.json b/docs/example/phoenix5_stream.json new file mode 100644 index 0000000000..7d1d0df7bb --- /dev/null +++ b/docs/example/phoenix5_stream.json @@ -0,0 +1,61 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint" + }, { + "name" : "user_id", + "type" : "bigint" + }, { + "name" : "name", + "type" : "varchar" + } ], + "username" : "", + "password" : "", + "connection" : [ { + "jdbcUrl" : [ "jdbc:phoenix:node01,node02,node03:2181" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "phoenixreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/polardb_stream.json b/docs/example/polardb_stream.json new file mode 100644 index 0000000000..78b69c0d5d --- /dev/null +++ b/docs/example/polardb_stream.json @@ -0,0 +1,52 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [{ + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] + }], + "column": ["*"], + "customSql": "", + "where": "id < 100", + "splitPk": "", + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "polardbreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/postgresql_stream.json b/docs/example/postgresql_stream.json new file mode 100644 index 0000000000..05278bb6d1 --- /dev/null +++ b/docs/example/postgresql_stream.json @@ -0,0 +1,48 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint" + }, { + "name" : "user_id", + "type" : "bigint" + }, { + "name" : "name", + "type" : "varchar" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:postgresql://0.0.0.1:5432/postgres" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "postgresqlreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/saphana_stream.json b/docs/example/saphana_stream.json new file mode 100644 index 0000000000..084eaf48d1 --- /dev/null +++ b/docs/example/saphana_stream.json @@ -0,0 +1,87 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": [ + "jdbc:sap://0.0.0.1:39017" + ], + "table": [ + "SYS.P_DPAPI_KEY_" + ] + } + ], + "column": [ + { + "name": "OID", + "type": "BIGINT" + }, + { + "name": "CALLER", + "type": "NVARCHAR" + }, + { + "name": "RECORD_ID", + "type": "NVARCHAR" + }, + { + "name": "KEY_ID", + "type": "INTEGER" + }, + { + "name": "KEY", + "type": "VARBINARY" + }, + { + "name": "CREATE_USER", + "type": "NVARCHAR" + }, + { + "name": "CREATE_TIME", + "type": "BIGINT" + }, + { + "name": "DATA_ENCRYPTION_ALGORITHM", + "type": "TINYINT" + }, + { + "name": "IS_CURRENT_KEY", + "type": "TINYINT" + } + ] + }, + "name": "saphanareader" + }, + + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": false + }, + "errorLimit": {}, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "trace", + "path": "", + "pattern": "" + } + } + } +} \ No newline at end of file diff --git a/docs/example/sqlserver_stream.json b/docs/example/sqlserver_stream.json new file mode 100644 index 0000000000..2e8ed1a2cc --- /dev/null +++ b/docs/example/sqlserver_stream.json @@ -0,0 +1,59 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "bigint" + }, { + "name" : "user_id", + "type" : "bigint" + }, { + "name" : "name", + "type" : "varchar" + } ], + "username" : "username", + "password" : "password", + "connection" : [ { + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], + "table" : [ "tableTest" ] + } ], + "where": "id > 1", + "splitPk": "id", + "fetchSize": 1000, + "queryTimeOut": 1000, + "customSql": "", + "requestAccumulatorInterval": 2 + }, + "name" : "sqlserverreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } \ No newline at end of file diff --git a/docs/example/stream_cassandra.json b/docs/example/stream_cassandra.json new file mode 100644 index 0000000000..80b8e3cc41 --- /dev/null +++ b/docs/example/stream_cassandra.json @@ -0,0 +1,64 @@ +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "id", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "cassandrawriterer", + "parameter": { + "host": "localhost", + "port": 9042, + "username":"", + "password":"", + "useSSL":false, + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_clickhouse.json b/docs/example/stream_clickhouse.json new file mode 100644 index 0000000000..77a65aa4f6 --- /dev/null +++ b/docs/example/stream_clickhouse.json @@ -0,0 +1,67 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name" : "name", + "type" : "string" + }, + { + "name" : "eventDate", + "type" : "date" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "clickhousewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:clickhouse://localhost:8123/database", + "table": ["test"] + }], + "username": "username", + "password": "password", + "column": ["id","user_id","name","eventDate"], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_db2.json b/docs/example/stream_db2.json new file mode 100644 index 0000000000..17370323c1 --- /dev/null +++ b/docs/example/stream_db2.json @@ -0,0 +1,56 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name" : "id", + "type" : "int", + "value": "2" + },{ + "name" : "name", + "type" : "String" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "db2writer", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:db2://localhost:50002/database", + "table": ["TEST"] + }], + "username": "username", + "password": "password", + "column": ["id","name"], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_dm.json b/docs/example/stream_dm.json new file mode 100644 index 0000000000..ec97eddb0a --- /dev/null +++ b/docs/example/stream_dm.json @@ -0,0 +1,56 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "dmwriter", + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": "jdbc:dm://localhost:5236", + "table": ["TABLE"] + } + ], + "preSql": [], + "postSql": [], + "writeMode": "insert", + "column": ["ID","AGE"] + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_gbase.json b/docs/example/stream_gbase.json new file mode 100644 index 0000000000..76269b24d3 --- /dev/null +++ b/docs/example/stream_gbase.json @@ -0,0 +1,48 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "age", + "type": "int" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "gbasewriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/database", + "table": ["tableTest"] + }], + "username": "username", + "password": "password", + "column": ["id","age"], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_greenplum.json b/docs/example/stream_greenplum.json new file mode 100644 index 0000000000..04ee214b7a --- /dev/null +++ b/docs/example/stream_greenplum.json @@ -0,0 +1,45 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "int", + "value": 1 + } + ], + "sliceRecordCount": ["100"] + }, + "name" : "streamreader" + }, + "writer": { + "name": "greenplumwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database", + "table": ["tbl_pay_log_copy"] + }], + "username": "username", + "password": "password", + "column": ["id"], + "writeMode": "insert", + "insertSqlMode": "copy", + "batchSize": 100, + "preSql": ["TRUNCATE tbl_pay_log_copy"], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_hbase.json b/docs/example/stream_hbase.json new file mode 100644 index 0000000000..6ae32d1461 --- /dev/null +++ b/docs/example/stream_hbase.json @@ -0,0 +1,104 @@ +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "string", + "type": "string", + "value": "2020-01-01 01:01:02" + }, + { + "name": "boolean", + "type": "boolean" + }, + { + "name": "long", + "type": "long" + }, + { + "name": "double", + "type": "double" + }, + { + "name": "timestamp", + "type": "timestamp" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "hbasewriter", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase" + }, + "table": "t1", + "rowkeyColumn":"$(f1:col1)", + "versionColumn":{ + "value":"1610509763849" + }, + "column": [ + { + "name": "f1:col1", + "type": "int" + }, + { + "name": "f1:col2", + "type": "string" + }, + { + "name": "f1:col3", + "type": "boolean" + }, + { + "name": "f1:col4", + "type": "long" + }, + { + "name": "f1:col5", + "type": "double" + }, + { + "name": "f1:col6", + "type": "string" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 0 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_kingbase.json b/docs/example/stream_kingbase.json index 5a34e7058b..1de3b32d8f 100644 --- a/docs/example/stream_kingbase.json +++ b/docs/example/stream_kingbase.json @@ -33,19 +33,7 @@ }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "varchar" - }], + "column": ["id","user_id","name"], "writeMode": "insert", "batchSize": 1024, "preSql": [], @@ -63,4 +51,4 @@ } } } -} \ No newline at end of file +} diff --git a/docs/example/stream_mysql.json b/docs/example/stream_mysql.json new file mode 100644 index 0000000000..1a3f768667 --- /dev/null +++ b/docs/example/stream_mysql.json @@ -0,0 +1,67 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", + "table": ["table"] + } + ], + "preSql": ["truncate table table"], + "postSql": ["update table set user_id = 1;"], + "writeMode": "insert", + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_oracle.json b/docs/example/stream_oracle.json new file mode 100644 index 0000000000..5f4aa285a0 --- /dev/null +++ b/docs/example/stream_oracle.json @@ -0,0 +1,67 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "oraclewriter", + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": "jdbc:oracle:thin:@0.0.0.1:1521:oracle", + "table": ["TABLE"] + } + ], + "preSql": ["delete from TABLE"], + "postSql": ["update TABLE set USER_ID = 1"], + "writeMode": "insert", + "column": ["ID","USER_ID","NAME"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_phoenix5.json b/docs/example/stream_phoenix5.json new file mode 100644 index 0000000000..c9b8643f90 --- /dev/null +++ b/docs/example/stream_phoenix5.json @@ -0,0 +1,73 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["1"], + "column": [ + { + "name": "id", + "type": "int", + "value": "400" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "phoenixwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:phoenix:node01,node02,node03:2181", + "table": [ + "tableTest" + ] + }], + "username": "", + "password": "", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_polardb.json b/docs/example/stream_polardb.json new file mode 100644 index 0000000000..e55e2027c3 --- /dev/null +++ b/docs/example/stream_polardb.json @@ -0,0 +1,67 @@ +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] + } + }, + "writer": { + "name": "polarwriter", + "parameter": { + "username": "username", + "password": "password", + "connection": [ + { + "jdbcUrl": "jdbc:polardb://0.0.0.1:3306/database", + "table": ["table"] + } + ], + "preSql": ["truncate table table;"], + "postSql": ["update table set user_id = 1;"], + "writeMode": "insert", + "column": ["id","user_id","name"], + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_postgresql.json b/docs/example/stream_postgresql.json new file mode 100644 index 0000000000..6e609e52bd --- /dev/null +++ b/docs/example/stream_postgresql.json @@ -0,0 +1,63 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "postgresqlwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:postgresql://0.0.0.1:5432/postgres", + "table": ["tableTest"] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [] + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_redis.json b/docs/example/stream_redis.json new file mode 100644 index 0000000000..7c09b00f27 --- /dev/null +++ b/docs/example/stream_redis.json @@ -0,0 +1,65 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "key1", + "type": "string" + }, + { + "name": "key2", + "type": "string" + }, + { + "name": "key3", + "type": "string" + }, + { + "name": "key4", + "type": "string" + } + ], + "sliceRecordCount": ["100"] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "hostPort": "ip:6379", + "type": "string", + "mode": "set", + "keyIndexes": [0,1], + "password": "123456", + "database": 0, + "timeout": 30000 + }, + "name": "rediswriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_restapi.json b/docs/example/stream_restapi.json new file mode 100644 index 0000000000..0e91831e34 --- /dev/null +++ b/docs/example/stream_restapi.json @@ -0,0 +1,59 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "data", + "type": "string" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", + "header": [], + "body": [], + "method": "post", + "params": {}, + "column": ["id","data"] + }, + "name": "restapiwriter" + } + } + ], + "setting": { + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": true, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "errorLimit": { + "record": 100 + }, + "speed": { + "bytes": 0, + "channel": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_saphana.json b/docs/example/stream_saphana.json new file mode 100644 index 0000000000..243c4277b4 --- /dev/null +++ b/docs/example/stream_saphana.json @@ -0,0 +1,67 @@ +{ + "job": { + "content": [ + { + "reader" : { + "parameter" : { + "column" : [ { + "name" : "id", + "type" : "id" + }, { + "name" : "CONTEXT", + "type" : "string" + } ], + "sliceRecordCount" : [ "100"] + }, + "name" : "streamreader" + }, + "writer": { + "name": "saphanawriter", + "parameter": { + "connection": [ + { + "jdbcUrl": "jdbc:sap://0.0.0.1:39017", + "table": ["SYS.P_ROLES_"] + } + ], + "username": "username", + "password": "password", + "column": [ + { + "name": "ROLE_ID", + "type": "BIGINT" + }, + { + "name": "CONTEXT", + "type": "NVARCHAR" + } + ], + "writeMode": "insert", + "batchSize": 1024 + } + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 1 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} \ No newline at end of file diff --git a/docs/example/stream_sqlserver.json b/docs/example/stream_sqlserver.json new file mode 100644 index 0000000000..f29512f363 --- /dev/null +++ b/docs/example/stream_sqlserver.json @@ -0,0 +1,64 @@ +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "sliceRecordCount": ["100"], + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name":"name", + "type":"string" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "name": "sqlserverwriter", + "parameter": { + "connection": [{ + "jdbcUrl": "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database", + "table": ["tableTest"] + }], + "username": "username", + "password": "password", + "column": [ + { + "name": "id", + "type": "BIGINT" + }, + { + "name": "user_id", + "type": "BIGINT" + }, + { + "name": "name", + "type": "varchar" + }], + "writeMode": "insert", + "batchSize": 1024, + "preSql": [], + "postSql": [], + "updateKey": {} + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + } + } + } +} \ No newline at end of file diff --git a/docs/images/LogMiner/LogMiner1.png b/docs/images/LogMiner/LogMiner1.png new file mode 100644 index 0000000000000000000000000000000000000000..15b25f0958104e3a0ccfe1f4f28e6fc49f2a7c26 GIT binary patch literal 41841 zcmaI7XH-*r*Y2&NA{zviEuaXo0MSj8A}x@}wg6J1vK5dj9qAB4PgD>@nh<(P6ckjN z)Bu4Xod8m$gkB|-NPrMRNZ?J}_kGUuj`581g~3=!R+;}Y=losQin?#4bMlY#e;hh= z=%k+R?T3dB9noO_|MA3e_TNWKCdfmFE*{dmt^H5H;kC)#t%GS(y7$>T$D8e&THZ;W zRA_m3{?7=RSeerf9r*Rmhn*?6CCXCopLs9iV9hJZ)azZJ5D#6KAKB~Q+NKew7$H+7 zTg`UGzQw-oOBy5dS*zaS<3sz5k^_cUWr3x;SQoeS@yC3HBBlyoZEae=wF+}d_nYZW z-GiC@--CBVIR_g<`;YI`D&pwH-NR|c3YP32%nY=By9axmQ+_76k0ut&K9*}MWgHz&aA6L4P)p5B{^Os!vYh#YCy-;SK#tDgodF~wCvr++>+Frx zE1Sh!F;zeI-wTA2D2wvQUCMR|l0`{H?xq4+SL{}dekLjSjqr-ZjXSjbgEP ztgBl^<7Z(qK2G45>3ECl^s3shxSaVkV;C!r*k3~Khhu4L{J+R21jI#AZt2&FGc72E z@gL>#T1aL)G9;ikc&dDZ?2=zc(ANo?(;IQq*YO&NF!z$R5y9T@PG5zhq8Im47i7;WTEJY+q8O2d)5`PaJ2d02bA?=wi&Qbne|8R z_zT$jblw^~Z4HhA6vkRD5{%8MgL!5>KS?{GclLg&HtbTM2UIA~&blGr4MxbII}fUl z&(K(Jz^@(uvj#X}S^`Z6D)MBA80I8qBH4ilGh3j{mSK32tK}+Xv>go3yl?nZNH?n6 zPJAoP{e5k3=&y1~kbLN}zG@Jv@(v5SzW_CoddlzdlE0}^?Pz9JK^Uo^eS87UNA6ci z+xQ1+L|+*ruQ7+LSwm6s&-CRoU`|&iWb(N^R=)YKvIm^pA)sQ>c98h)%lvp~F zt933Q5z}I>V%{@VLvR!?~~$HLQo#3 zCZ`Wr)1JKgYCU^R!gw!w3lQwB*pE9)Wl!2A;I`ggQog1G&X8Krz$gIn>Ttak8(Oj# z&@oaL%3K2P)MRQ|?~RRAS>`QiE~zic{A$>gZYY~nq<5O&eds^(!G3fW`vOH;!7mTk zEBPy#BmvT0V$UUPGiS+jiNO(8^d?~s(4T`Upeq3SH|{F?wm5>T)mDb=@=UJ)f)X~) z|7HPt=JP&hjzpSI%n1SaKR4tzt_9MY0eC`2?MAo6J8)|*D~P>9v;pKgJ^qO@+eonl zg7o1sK^-VA(DuWt!QyO218cm&9dxTCXsEylZY)s;m+fHA2{#0sx&@Ex@D6qvQ|ekC z0b+GqN937J@>vq+*N|Q2Q$A;)JGCSARU)ATeXBA8qT`mhPXKcsfUIs7`}K80_a&_t zJ<0s3I^A|(@pk+?0gBj6>Dy{=2h&R3vvzp8r}xRz;w=T{P|V)MQq`nk?I`p(GU}va zODa9nj$V7of{Ekiv+>wfu_;QB?{bu{Nrn-_k40ho03vvP2< zBmcd?T1OrbG~(eYbh{hY_j@=4l2i&NAU=A~Pq~9D#5N)zFnrU+;gA3s(R&mfstgi}KTO zbxr*Rq99f1CunlnDAN#T`nGJtAF$kmXv80h1Dmg~M)(#b^Oemo@ zV%$T}?Ah&6HY<6~1Cmn!U`h+IcRJ**L;`|VjtQj?WAT1(P2Yv~M_a+@>+B8IU6YXf zu5>3twQhFU>T=j!WNgjRdrk8-%HSdEAc@K|&2yf#Wlk6tBwuwYpW@Dy0uFye3Au>DVrQB&%uvb)5gX3g{By4>>Gg=vpGuw8eNlu7M@m7fD z0;XBN{IQma9^NzQuodGSx762VBji-r?03&ZW3psCh3MiGbue+ciC13X>#BDI?s ztlah5lfW0dG;BlY=Hn~4wXyE#6Y!5CyZKoxJDr4&aIX75th%13058Jhcgiao~6&R*K5Z7yV_q~BEGfgPdK>|mh6;ytT6>o zn9iZ%cj`S?xu5uGrOxf1s_mPiRbeMP*L2Z}wQyLTyo@b6w3MPS85epz9C-11T(7Usss95|~qvd+gh+*EiN!!Da=WV2XBqJZnFp30MlheSP zgbwUfo-atBd2TxW&-b#AC9Te6(<^GS9c4Oni*M^pYndNkQu(yM^|`j${Q>Ax=V0-e zKKzS(z+I^=OP~6-(Ad20? zUSV?=YR)VQ{Q(odT=}&^KZ)<_{1)bbSrz;a-3>ch@7Jor3lB*>n%KU6=9s`fQ&-DJac4hZ z&M0n@e~B#F8)bWyqb)+q3+8F3veAA65mn~lD;`(wJ(OXdIihzd{S4Rqz|PCDnjem; z@dmvU29vEk76IOJ317rz?yd-$w*YF`1*&0hlzIgmwx(f-XAUXZ*;j^*{rYTf!Wb&| zabEMPG|$k2H)tvi89@=<_+t(JcsXks>p~GVI|ZA=S6^JG3!UhF8PL{b6!+?pqW$45 zgTootdzIGXQZOukUm8pn&7Khr_{vOs*%kPOAj+`2uwbofnoLOITW&3qcIGcBhp(I9 zQ%gr4WFG4%62!?S0(qlai_A5=gm`e8;udhp^*2P@sUrz^aT~v6PvLLLT3U<`o*_2v zO|pW$t)`~>)znCF&>yh5C6j2ub3*rVe+qf1Jejd5#oCNu2H)MmFL_a$3l92&GVj$O z!b^~qk5WmxH|tsj-FeQO=lvL7g1?g+E5^9EQEoRsYrYMCsF~Yr;v5?-!FLdC&76~+ z?T$#lZ;#z55{pB&yd8(NNrP^J^x;HJY?Cn9Zsy!ISO995Q+c5prS!&<-eVHac*YWA z&kQbfj~o!Mybu;dVav>Y+N_w9Tj-sMwqf!>+fa>t`}g7`-#Y{3s&oOBdmS;^zJ_Ny zFjNsy>@o@d+GJ_U0Jc`WY`!ErU32!@@$GAnpG#|&lo-Qb^R{lWtn+(HVq9DIM#5eC z%Rgn#Sbo>gT(x<)^yv&gZ@S)?gxkyQnfzmI61-0iRhrq~6#QD&S{kKR4R|gjP2MBa z-r=*#xmCO*KkEA8>uQ1~Gf|pXPNQ$mW%m2y3X-PViqNZ95!q+z7DicbZ#IJZMqhoI zeEE}hLUR`u$}U&bm*8P{^l}!SP4J(<1PscY%Wq^BW%EUUDeVxTprk_ZCvyfJfgMT! zA*z?)SaMoaQkDG-*0-;E>y-(C{P;a8;Po-PE2VcL+sUb{@tYL>kYKSP$>Ub}a*e~# zs(|f@;v937+uMMsf-@6qWT9k<*X#F4qfT{XweykuDzvWwv~yjykQzcl+FRLkg6r^W zlNt69guvIW$PKu-pZM$3z0ck!*hI;0M8`qMaf@7t7U{N8vKMcm`A4@o;f;>RD(@N_ z{*#{`^I4F+Zs-{b2Jpt$5+?p$yY4+aEb+x-<7-FF<;o7+tG=9Qq`WXR`q3ue->{P% zZoJ%i!E>aIR}&s)5`dHHguip;g<`ynji@%Xqo`XEcV72+1%so_J-a0!m^HEhVAAES z7}WvlHFiA^2p;<^DhCmldpsguGxooLQrcbdRCzh~<@QN;M2x(s zR16??(l{#7@`f_XdQT%ofWs%$$`x$h_XP^t2~x9eRzBj+bZ*XMN*;noXm;1+yLjui zhwqg!GzWN9yX<>vX?E52cR<3Liwwn$-9zgx(RX-7Ja&%E}TDOY{ zE9x=Jbg$)3W{*vPmprzL(&)UiO>|O*ec+xLH#V<$w{KLlaT_$=S1U_wuIo=kb(6x3 z_me(bXdzg{`Df$t`kP_P+&k6wP31K-TQwp|*bqiMMKws8R3aEvAM4H5Pl2RB*d6$+ z$lG__uW&ZbijN^Gv#QHEt2advMl;5aWG~TTuulU+a1 z2>0Y2R(eQhyHI{WBMIn1n+;iX^SE>yi7#`#>&uzzm9PL2^TEZXS8+Z&AO zl}soECaP{r@fujE)ouKVTS0luK?jp%2`Agk#l+4Tu{x>BpfbxntQy<2i?T+cAVvv7V5F!qsGJ$N_oWHY~3#x^=R+$3?B)}n3u9H>+9K6 zNbi4_SwSf8_)B#;)%muln7VAoEKRfjxmf>Wfjppycvesx@kPYM;8K-^&d1MDqrP#V0o5RwTmy47qLEJg*U~s8`U=3Z%P`+>!VU>hqE$EtU+(3c{N;DdA6_yuJX%s}&8C6xICuRU2XZ>ge~F#k z*zZUA6|GX;^;6iw1pY52RKxx6Y>y56&Q1usdzDC6mj@riwjp9!ob&v@xuK;>!%S_N z`M&^@82jQkGL+Q#FOOtn%AD92{`jrzNWlIqU`nwbz<3YP0M3VD-1lgqTEN!w5^L6q z38a7PH&k3Cy$r4yYNY2JbA_&p8;w1(0&e{mKeG3sTMCq7|9V&bKko_g;q7gQll8?p z051#0_r65-gvmoek51`tO=Oo!XtwbbUN)GdXR^4E@7bvI=csEY1DTyKy>G(&J(yDe z;VL|(60Yc5`k1VVNR=IZ&@t4~J8DC`DUOEtKd-CP3D z`_y~@yPtJ(qY!%$G)Lia4|b<4&VHsD1x9Lm{EXnwj-H%4pvNDmg>bU0-ZM^< z>#j**1KE10#|=FMet)|JAfb4$i#Z6S?;sC$n!DJikp29AD<83vuVPk$*e+1b3O}Kx z&W1||p&S}iMhxHKbhWcih@VbXG*84wI*-r~SX$`_ORm6@u(lOA4SMKTJ3)LcNQqW9Q?-LVmTmE@%{38@Ra zq756O+=NI8Ng@SqC2=hglPJvdgM*I^dpVtT?9z>r!0%Ezb630rH=?aWa!^@tv`1i< zxd+dXFPqVv^k-wFZAyPU0)`6B2Ti)nUkd~ca@7g#`OZ1DHf2!$L&Do^XT;V6+SRea znwTr61LsAj1Ili4i1nnG$7!BZgf<`W~Zh%wCdp9e|9?dvk+(kT|PlNlW7AIZ%o&*o(tuh~?Lk|SL2EFFBx?B(Ac*kZ~Q zn4At))l^bGCWa`hd~QA#~)& zLN#L}r6K=W9sddxZXk`ln_p3vXWzC29NOhjM{IbD`F-bl2T7jNuAE=+uAQq1ZYN=|Q{AprMS}-%`dKkeY6z!)$1N@nC)*b|!K^4Rg|jqC1|~SDhhM_d zrDgFqj;P73#42_`j)VZl3-GqK|Jkl(j{oH5k~{%-T}!H4HuM}z6j9>|v;R%~hW8Uw z>+HujN^i~I6u`3y;dr9Bz`61M(+1*F!%{Ccj*pb=uatPyt+T;u6`UvT=;B#6JWi3R zk&J-mgzX2iC5DuVq?te8VcA24y}e}C8#J6UI(phibWIRLZ_&#UB)@c$`7{zY1?MmN z|4pRf)kVoO?^<%j#J1T04lr)Do^spkyfd(sKMq^CMhciG6cuv2YSYKszMgblc-x%MH15(d!J+)Rkq997uuiv(9_iOwIMA7=;JylJy6wcc$$x;>I5 zaSm>ClLv~^cAjOYiS-NY@C3Uro{?$fUIM9$%cYQXA5LU6ZLtyGHibP@9(#x0n>)L0 z-c4C2Ra*`>cSzc^H{*5z^x{(!)|11C2(A$FuPlqJgZm8`7t z)eELezw@}0+;N7@ySUWnx_9%g(4y|b`n!W1G{s0X#R1ig4e{I&m2v6yS(nW+zec~r z|CuoUB|(|jK`RfDE$AZEjBr|#U1cB=~58!qy>ujX8YeY znAOPn;M@gjGP7{u|A>b@_xRXSqNK*^AOGhNyKDL_GXAbPFv$hUL%n_u|GC1ToejH& z;8aTgH=NbZzYkW+kIh(_sO4MF%1p~lb4nwx$F~3Lw`VwchoEx++hpe$8g0-3G6Z95 z5LmT|HAmI)eOW&?dk)#tz4+;5AvpzGw@B0RMr!P|`#`3`ed&jtU}L2Fz+iFvDR#o0 z0v{q-n=ET9bvYON6}h^X&7$HVT8tx5$UZS5G8?iX2q~@H6Vw1J^H)ui?y8P4zWjo& zT&y1y0{ry@;7%WWw97%!v+Zoju(veZjln#KmI2(fciQhns5~2i2CZDNsOW^PN~iB$ z)=g1FFb2dRRw}EjoP^iR#zno^N^)bAFc^i)Drj~kzgDHn$a==_+w7o;CI^xmQJtgt z8**z^%JQKdx>Ba1!E~F}=>vK+r1*~wr*3x26}=F$HfXAXjt!mn?70q96KHv|RD~#} z?_qY6dpD;aIqciz2M#f3LOiVcg%@kA(o|#W)3oE;yaRtWoJCHXhwW~@u|k57v1CpE zB0Wf7JOjU^wXROYZ{3-1hwN_vAyg(Ole2UOd(~K`g90~Lbz7Xx8cq-V$q~TF?U~Mf z5JrDIxbA?pK20Ib>}iuY+gX^kktQd_22c_=hDs2AxvN05ROL?ndT6GRnj)S>_VnD` zpl{Vc(oy={9e6j1EjH_9MPQT>e{|kegYQpHvnJbafotMmy9GENkvM+E2DYN?aXp(O z8ii6yu4}TqlGdos!2>9C#qt#(QVKdPAsj%@+ea?zTaT>-Fh3+UB7Q;Y zd#&jTn6!F6}zt^&|JW6P(&CkUq}7MO5{Q zrqzc35c;4vcyiTa-qXkz0owU~x=r{`(cuqXw(oDnSWNB47oJ55d$4YJwEY-8FYRFG zY9RRu^0oP;QM$B4-%DC_13fyDUHcS|W7^g^R@>vLtq}e&HazjP5rTyS!bs-(M z(u(oE0TeNrSD(kJ9nI#pHSK;q&W@(l_~^VfJGR*eg`l>EQq}z)r0?s}8zY$ov)!Or ztjNvi_04Xi^gbTzvGH;K@4|*Qw%j`n6Bz`UCSG~(lZ-!5O`a(~6!!9y#EEQ09fJ#D z;}Q2#XnMKXG8I*yQA!|DR?F-v;ISZp9}0Krow9#1Tj2x>Q6>!KO%FNY%r8?=0->98 zJZ+*`DiYFxGf5~+ndIwYL_Y9bK;C*jIAAmP7@-@n9kAij6+y_g2&G9A2aRCFjNWg~ zO_ECIR?6$vxxwU%V{38wj3n!WRY6=L3UNJMP{)InM)wNnh*u3R2%Dw+d(9N&jAO8Z z(rzD6{#;`~*kFXcp_s7|qBG?pgOQE4pGn7K7@Nq7i|Is6A@wnax@(H0eB_M}QUrQd z{Z#sj503`_zM>98yMlNcx_u2`6wIzO>PcCCkC(RlR;Q%;%!jAtLDg=rt?Ci2qD8edGo9cYq`$jr|Jv^S z2~6&@UNzV`!1skm#o{gI>ga_R$9#RD%{){+lmS%`;*k5^1c2~`~}Oh{yVtH4JrQtv^<#7iet|SyEfo%gAcy5Js9bN zu{+CBnE04}aDH{+V*}$@q@;$@r>hPfE^Un%M^ctE%_vJVcy|c+T|ujIi5?odVvx5F z%ef7h*umk$$`8szz*ZscaRJXVx9wP2Fk^qZ*WUiK={?7l33}|PY)EnnWqk4hKc3zZ_QZFnHeTBi zu?!(-wqda^E^spyg4_wlcTq6`%_bP1Cc}*SVm*}SOx_g1I&dt((s+mSPzCIVDvZsF zi^_ysQ#M0Z|GMIHyVG~dtyj;_t?E%+UeGD1`|{|jfd_2kE+%&<_i!$6lcej)Yi^IR z+zabTTHYJoG&>C`esWU6zROmUY8MJx6QoEhbB$cTDaw-UkMd+VbQWqZ8eEwA>^j^} zb}3g#o3IJ^>OGPYw{SMUe|josG_oL~s{MrFKi3N>p`k zHr7g8x&|WpBB%eCg+YUvkv8WFTAz?&XB=Uj4U)$kLGy}I07eRMjGlv zw}Iz^&EvKeo_GzI7|@4_qL1bjrD77s^*C`spfkv7y*-U@n7?IgR8z**#*(c99hai0h`h7q?fb# zg`LhhygHAoLsY%D--Bn3i^XH)Y8s=aS54Xhu|8EZC7QdA>KUW~Gp+F^jKCZr?waa3 z!o;l#=Q1q@g*2b;PRx;Iekclspa5U*wm*>Ozk-0AkP+_4|VX{YLmfQq#IS z8n0{Q1H5mCFpC3FY7f=(KuzLG0v{mE!c>a5hqR_dySD5UQEWt0i@zaM)rpC{Nezoa{ zsJHUTDUwy=mQn9_w61tpMc_)!CKif;gXhwHo<&0@Q9ng~Dc3B=V0za6;BGRU%@t|5^0HBg<`egFty-`4vH4pda#oe}{CD(%Xxc^ zCx6|TBd&DlcqFpv+I$+CN`8vZWOf?&qMfYc#le}$zp za{tD@wIC*1l-*tM>-r60eUM4$P3ALO( zj|#|1uS?F2+DW`w7ssl$IB(_*>d*jqR+YJ@Y+NXx4+%{RqRE` zqM6k1HE-7Hk-1&NL}BWX=s{HGq82j5#(lXnZl$srn!cwvo-fw99JTYZ=}dOg%1!?4 zH%`HHuOaPfEyOTkdSozlE74etHKk|lM4g?>^wVg{hrE{C?U(Q!_%_7%G;(i zKB~C;$ENc$uGyan-Su%%B{2}Q*^aS!6vQQmms_eJ>Wo~&jR+K?vxv{gjF1)5xeP51 zs-;zwL;9gLgf}HBbdqbd%K3mry?pR^Zo1-My;XLa!4nHOiA8!?Kwc5ylpP}vzhg&t zMpd@EMrH<7l*dW!lTe~Rz%DdRCA?Ix%OcTwajn0%bD1rQa4j9l6%SopA@DYhLB0h{3iTje%M;Dci3MoGMy=jLD1$=iM%yWmJrkIV zS552Q(yj<*tX$lsM;DnvFcbRYDLo-km}077V83&83C(2-f6|b3e^~sx%=*%aiXvka zIGfNZu1pkTUkMM;HOpy!8`Jp$&#}niDHM*;b8oUqIpn-bHj1la>CZ)qF;sDYHxHDK zklbMVTLbrCR8GG{MjC8Ls-h_iALP}%%h#j}dIQ>&_zK#;z{>XWr%-^vCze8dVE61GW}iyKp;-Hr5frI>u6hz<#$1*F5<@f_5 zIgY{KnA+;4xY+66SRa0@iP)o68`>DCpeWvKy#p`)I*mt$CK z&76BW`DHJ|%V$HPw}!<=rXG7fH!g(o^Uh0K?{<<=^!{P~XY1B#go1hv`etfF$li^6 zq0q}hZsR;hvP7bOs!jxJVogTle7iK)&lmoFb$#S8o)36 zTeya;M4kB&O@ipJtO3UX^y4@a5Qc=E7JU7b%@$7g8}q%32D0)AcnLZYyP!39I$lk4 z)vR}HE={&)#QuiI%ARRhCV7zUOC@|UDygJ!$dn532U?4ry!nJ#;yJDJ2d~=k;aL+} z)5kss-vu`MSVZ-UMPOXXem`5ujJiI z?G&~>QEaO+1gw57U#mO|{1(6PS%-V`)d6yM=NzWA+4gp?4!iwFnHpJDh56dv$M_@< zih|F%5$6*{Pgb|HQ(*5apSk=(;z87ODodsWNmvj#K+Xvt9J_@fZ%^lM^;2ErVWo>~ zI5;`Wwr>M^Hy$yin%S6L`Zrw$`nYDo%_u)jf3V%xPEg<5utCQ~Nc#OZA z#5^=;N_lY*8Orvof)iTrR-F0UUG8eE?5FjGk3eGkOc;PUL$(kO;~1=KzvtVo@3ReZ zQHJewsW>#p3qQV)1{!_JWFEJLnoKv1SU1G&mP7W+-L1ZX=HwD(U%vd_Sc^w zU$n0BU9bb}jiyx3W-qzhg@Dsj^9&&x%t3c_*YH)uMuDheuCW}r40v!fw^yhxIn0@5 z)m(i8xYnCYQ0*d|aP0js``d2}tn%+^+v0j5so@vX%tzy_aWb@&56RID)Ze>V18HqJ zzSlL>K&R@fmX9j0oJ>;8IQI|7#LLeC2=V-<_U)mQvmN6A;`Uy{!QS`>=fAJlu=)Y4 z{^4G>lJf`{mKbUbXU=wD!=Ap3755kzY6y5qvUsJT&eGQ0uNTQ_z{bfuQCcKQ;fPcn)Fekr1mu0H$`d}NWw7Gc;KJ#E|^$)T*V6ED;0s_z`BMhS;`*4K#z18x+|(d^PJYtZ zx@_S6FUW%c3C$IDmFWuwI5x0Q9ni;|C)=0D!;WvXipCY&L$fJj&Q6)NRqs@M@&T0v z`n9sB=D6y4C8Kflf)c{Ol!)PE=WKQ=^GBfG&W}t(GUAg{up%%9v2qiGXP(WX25e5Y zU>pV0*>5M@!s7K0WfcgKUE;UQ4!U>jMXi0)vZ;Fh__71x4M8)v2f(dFsq>6`*aaz2 zOs>L~-BL*N@lB`cd`pCvtI^x6dlMahLtNsL39bgnc~yiRYS!WN=iS;rZ)o2mnY{eN1)jlK zhkPEc_^RxpRI@C_s7Cv2Y~C>820PNf*t={$UuyWKy$Fbs5tVQ({!k%mV4_7~T(x0Y zcMB+%ce#tyosq>|c=?L|se257<0uS@n5nchVllc6)EO91nFcSCI6Gfz<9|pI}Yt(PG6_{Y%ER z<@xwVLr~v&PwdxcnQ!G#m4J@M@-EDfq-F7&j^w(nW00~3nfUkd+!F2X3b==Beh!O| zf#>4Zg^|DYJ-WiL{d?=F((qp^CF};61ay(w$?YqzUZB(=gq`s`;kIoe{F^V^wZEA$qXp6MS?kK*MP!vwL)| zy~H*V0AUMv9&pMr4YB}|-mIG5a}7jTb67>ZnBGxh3F)#DN#ROhulS%Jxkglys%SRw zrxcEdflGBg>~B@|8YoyX&e0nDQRFv6aUR|UERKLQ%aN&7Qq?dA3v8Wl!NScgYv-qC zsoO~HQCTe^b?-OU%(_KcPj#o_4BFhayt#%}S&vdnK2oMPt3wCuGoUkXJJM>A>Pgo| zihV~CGyrZfyy(b)8w9bGQ=nyJ=Qvp+UnIsR~<9{>cSO>{+Tu=JkZ0 zD~eX$=je?Mu^$npI~l!+sZ%pn!yi8b1)-Z+Cd26~@3K{v3}A@?t1aK z)u+D4=9_lrI6~#rTPH-CFYDvR&|=Pw*Xl_ZdH9;$T)v2&xZ50PB$b6a+M_pnuTkp> z>hm43cEv$+%Y!gydX*e&D#=HQU9hM|yIyvX9~SI;#u_ zH(XBe)857kXqh`+|7e}rdbl3-0d*rQ{3Pbpy_;qed~eMIjE-ZW9c5`?HtL>A6yCqC zw_2_TvK0Eq+&`ehDIbwaPlz4q-#S_5>a)rXV7n$g$J}mu+DhEIxe_fMti3AHxTMZy zSCGxOqh~oeaOu6zi5`xBz#NDFX-)MVCccWd$qVG+NILHzv?v&$VK9Hi(86)zlTSuHe}L?2Lq{in0eA}V~x$z zffVs*NPS3G$t>YCvMmgtwd3TYEG6VgcKcQ!@0Lu8wst_3&=w5GlU!YObOwZqN^=IA zZVG%lNW5XP_EQE#bD0mT)uh?n$E~jnhIOLhb>? z72K@6^wZ?a8h4$n&Wg&NIhYaHw~6krAq}qw#5Fp%ytBVyo$EJ6!lb22Q0iO%$P%_S z_VDJ_fp4y+23!}=J2D>)6aib!d+=F^n5pfy?yAWZGX=Jz;y7=%Wg_XFHI7AoM#SW@ zW|GCAR3l<WtJ>Y~!E&UTO^wua;Ix0f7MaYP|*w6JYtjmeQWQaB5>XyNMLVtCJ` zEwiZdXx~!V{xaI@KppIla!HKk{SKZ#@2^}pGy5hp>~~^H>9i8@?JM&bS*H7>qciUr zFqgA!yR=vT+tokJys&BxVHd}@x{hQXEh8*6HyT|bA&iHZL#ye_c;i}Z=MQ;bF8t{k) zFWbtA@QacVm9Qf4h;IQV6TK7-{+7Dk_q6}@$O2g+sl)88fsL%Gv%&ef68pg;SKA21 z( z_kd+k*Z4kIpDS9Bx)3_|_uq2bua>xVl!Y$qg+-^kOveZWC`yFV=g3vhZx?PgA9cN8 zm^P=Kv&|ke|8dDS;M0XYxHFAre@o@OFT$n)Tq|;4b@#3-UP+4C<}rUc@cQviQdw{7 z5d^`92}_S2U97tlMxHactg!r7^x5cB(~AOIk@!q#-z$N8GL_};w}xuE;F_e-#5i_i z8Dd+S4*Lo5J=c(~BTE&MQkbYq^aM^TWWj@Cv@mQX#vc@w%}9!2ipFRA!F7Ek*48R? z6YaDzn0sYG^C=JsKYQm)D)kQNX~+T1^PtKyshaX}{MbkD^2|hXyUm8DxYsPOs_+i{ zdTes7sUB4E3wJZ_t4fI(FeX7wMQ4-eNvI;&YtDoApvK9qHBJLPJdK77nMB2_MDn`y zexl8k-$)1vJcR4(zQNz~Dqh6Bs5|Z_jd8`rx74~GuGnnpHG1~%Hxmsbja8?Wrg$7} z|8Bnzd)XLu+OsjYhoI4M`sFBrt7^2mwsh2F+CO9JxZcbk)kdc(iobL?N)z1_624V> zNtA=X^2&h>-$}`6e?FN#lX!}H6D?uXbZO(JxwQe@*kk7bDjRwen9+8gwYTb9ulCE^c`^mEt_!JZ@+(rg>yJek)efX~zoVI@ z7$jclzviW-l#49BN$_`Y4_W?d+^Q;d(ZpgwUE(}LqF>>u{W{eys97(E+8`Zfe=HYk z`lqN{CRxZngBrx$bdz1qx8|?#@0Yf#@70RtV&Cvxz!mv-#^ob#kSx~`d@>Kx+&iM2Xopn z*Km}#w*!n6o}5~z-eRG=-dF~0tVH|Tjmg!#>v|PPTi{wsPIL8-kXr%-Em`;468pRO zeUD6M3U(xX_&naPhrbbS0iT-(!hbeUb|swK33U&(Km_^=u}LG_aDFpQSYK4iY_Tri z6G8qby-jMT)#dE^K^9`ZU+mkStGysy!V0#Y4c9K*A{_E{Qv+!zKHScdxMm{=BABG| ziL|IcFbdp09hY5|VyIGlwY*tKp++`F$eB0FTqqHQsz)SI7q5)Z&=XPFjMbxxjZfam z+9w{ZutMJPLWGra7FdmgXLK4@GJlE?c-H=vEGk*kj2Bx zXMN{P&3Uek|5#N;N))^@4CV%hRv9N_Zfb6%3naO=7kDH%L|y?+bj}B3e+(!rw(y!D z-`omK{@S9t-Kq+>9@*?#NBR*zcd5Y;H<-#~ULyG6^?F(=)El3CG7<;tz#a8{$_8K6 zrxi6UEOj|+YS@AaMv5pqxtT7s|Hr{GJ8=r>5UwJ69hL^5)-QC_<$%}MUO(Ep8)VVv zC@jhKUoU`qL(BXWa62JnYi*+cvp_IP;1=osWhn~ohrjG^>U43@Y-z>8$46-z7&+ADOe|Vo-Fkb zIq?{oF|?(JUVY8&9;)it_O778L(*)a>`I!!wwp;0ayAc>c}jHN2u>&veDmF2?ZN02 zEpMaj>_i%9qve=pP(aYC@pbb)T^QxV6|+G{lMhlyOwJTjEMDm!CF=@H6( z<NKAHtmZtD=t=0_uP(>N~R zsSeB5e+#pF0}XRY?N+zef4MvAakCfHmwd9-lsPSsxIUcRs`Nx9EqHg@v{XN{CWXu{ z{zQC>zu~@SDR!vjjaf*10`>iX_G#WuZ4QjigA;N_Z6dv2_&;5<=#Zl&i2$#F&UdZA zOQ=|lv~`J_bjdlYTA4|r1E0mAc?D|{*7vd*@d6*fASEc2@2;T1-RpI{+8a6G@xNB} z{jIvqmycGaH&Y*U3rqLhu#>Rz*~ZeAXb2vb+YxgB|XrKXF< zGW}-WC2^;d_{4e+H}h32JI3bI(lyw(@Y$(a!Q1ujW)=9455756yjxHz z>+K2&ci!9Ya<%03<45CL$60rJE-GDZT=~*`Hf}GHdg7+y#9sY)HXq3*Ww6AL!9SfD^jqdW%+ATHC z;u%%vchkv2m+_mL(Qt72eb5U6?Y>B6qBex6=wTvoB!1GK`d!fEW1<@H?SALVvZ9Ns z)S%9q2cILvFs`kuoS)&^O`R+j0ZpKv3Xe;w3LPat!A=mc$-P8Luudt8&9K*cpg>h=8RtY z*`hujeFt&pN|Y~uK+iY(Uv;)Rc1=P&h_t0A(?W2^53h#ygKbp=xp6U1h|K*URD2n7T zg{ho!$k~>VXLq-VEhlVNSMY^$#Wt~RoS9k&qbJ5bbJ0TwAxu>!qR5)?}e46QS$jg zz1(asH_~r6B&6s9`K?4`^itsR4<*_`*56FKj)0kVVW>n^4f;(*Yj*$ocO8jWapB+gEb09dQ!uv#o_aEO?RLc#nQ z4h!^)36^RR)OJn#1<~0$(~EA1Zj&;1%1d>t4y8u3n^;xynbLQFaa;M!(lpDCDU509 ziq4}D34IAa&_G`vaL^}=^KcZPxGoeq<$m3qEcMXU4H5+~{KU&sIe(XbA@lHx1@8~f zA>PQKEj^{i*jG|rZ~9Koao5ZC(w7Nw;@CXEg%A~!T`myH7ypho7#NZ7^$ge!^cJW2;RcE{8=}yQ9Q_QGIT!R zJSDnf`|aBNw8}Q}eK{*K!&R$Dn9~}JvuS?r@yD>3VLulJQI>Jl?ib!R*!tMf{PHmI z=KM)_@jC>Dv`m>CD~1^zf)oe@m}FO?Y&;MxdMikCW6z8PAUua z)EKShS6ktkM)OiH^rO5lN5hsm1-6B1VHa?u8|*TN9p^plzk;2h*eZ@y4PvF460t1D z5?Zm%p^i7z>&!UGO=qjKSHykuaBVh;(XRi&MsQNXJ0{b*W$?6cSt9969SrfZQ!txG z(Ypu&gbHCd91TIMr}pPfz68QYs~cqV&a;1@T~BA0Kb_B)6{l{axPW)EB@{%TMilkI(cXR|r;}7%F zkAw7j{Y;tah)gdDkU-NW=HXF!bM?bhPL?{Ee^xN90QWBv?(5oIqjd@pEhn&hZ(Kc@F4m%v@9A90|p_)m^i)e@Kw@cQt zV60Upi&Wc~DELh*wwMfT`DNH*w z`z2mEnN0i6@jYeHf0pb}+wfTZNFTgt8pAu8go;)&Pm~o)J2dGpmz})AO3E(^3c{_nnp7IdIrJki3#~^3o~PL3#XUIjfJpActpUUsvQMiYBG z1YK}IB8q=1ln=Q31)2mcl9!t`3I&F%YW?-%Fz`vL@TM(@H!>_Q532jV@|YhIzF>~& z>cABih&HSQ;cpdD5_mS~v*lb?UVWY3j<3{TuC^^MM5H3#>nThms;@YCQNRb5KO@87 zzYv4Q0nYJBY1FrPK@=bu{}a)jc2EYt)|*g~Q~=tSTeQM19>i+ytbQIwxyn1$5acGy z#V5r}M9~cgx3YuR-uMGU7%_9NPgquiCkTnh=l^aHull1tLCEl=w*2>KtJy1FG~iVe zNYI|}L2xs7rXLwYxpO;Bw|5T+2e*eP!IAB&uHJK^6MI+TvbsSt3eHU1fokdlC-XtB z1)MJpRAvr0*Eoyn0-_%`v@3nxV8c*XGMtR-1D8`zlNiDEkdM2(3*1Z`Nn_dcc9$%A zT!>_l=|n!=I&Mcx=dkOz8CON}>nGAKrh53Q5W7E;_R0D%Mwyl%9rS5zuwli*azq%A z{09%DUdtUDL70fEZ|Y=EUXlse1btuj4M1=H4%rfLtLHlx7}@P-Q;%L{PKK`zv61jmR#SLHu^WmWrfmF~))svbHJ&UjGzV0tliYT030|rqAk|lnyo)g+n)k z23ZQeAl$1KIGL|A%^`49bQ|NCA_Zu5{G z0Nb|t?a=%_{0x%iZ1l!se=yXX`%52MeCR#zFo;i-#}?&SkC*0T%+;Mkr_W9Z13dt2 z+RBUVRpHKtyYRUQHgp(wulO8t_n3Q0G9s_YD{yL}if4Bw)H$zKH&1m;S9YktuM3=`^`bQU-3iGcJji4)s33>=^OQ3^wP3r%y#HX$DnS z>3NB{w@ItR%Ub-rc6ZkoE#rLkjb_Z>$dHi9u+s`WRXy1#=*rVz0{9J+655B5ZSb2* zGi7M6qm<)gLmD+Tj9Rxr%QASN^lYZbY^S2S=pXN4)*#jB`~`H8JR?50)}z*J({LVW zf(qB~uw(7;z9u$d9L;r3llqNIQ(r_(6ZGW)=vrjfR0?o7`1X>i!y3Aq+#1F$;|rsW z3#t=8G8@$zoea%Zgln%pGP}Pla8vl_&S@}(n1b4Sh2WS8ozFJdhidD1JBA;$z&NT{ zhWsaPr39C`*j-=CPd{qRUT2036>Ytp+s;eBztsfjz&Ex_CBsB>k-1S35GCVy4glrp z#)3X(V4DTd$l;9PZk~H?*wh3Z~Z>1J2e1iS8v?0O85oFK@_P|V(ftc zFZ)PYp6y>#13me&jdz0?EX%ES+_wSa8oAlqv=h#p5x`VQmg#KGWVm= zxWc%#mpoesCXcH8KMZJB>2OR8AX4C?pMMie*oE%DN+NC#eB|pcw zAO*Z!byq8Ur2p(ln>!=k_ho)U(wmJxu}_-b6k>cx5(F@BZ>ezH}0?TWF-IU(d(m zDE*M&k0!z@huIl_i13>x-{+J@OLmDs@)(Evn}@ODy{^uF^66Uj-O(HZwC<~QhrhYF z3y5WJxJca(>h$KnZFfp*1ds96C2r*g;WMMyNf{bJjG+~n4^ffVx$Ba}0d_1GGvZ!_ z9glcg8Tks-yil16IZR(he_T=5qccYfDVJE)d0fO^$70trU954LpzkJ34~$F2e$6*p z-6^7ViqajJYx-EXW1%|K9;U;SJM4tSxh8aDju9B+l|KP#6JN8lo3{QjqbfEBlk4Q# z2HOnvCpl^Wk8c;T!rGe>#m)piaLO14UW(LHK}G0Kmfbwlr?c|9P)G=6R_<*xCw0@m z?(V7O*gG^abi~-a8a?3<-NfX8E$A(}Y`DTLrNUDbh`Kpx`FEaK!pM=MM!%x*W2%@@ zF}bDHxKZ{Vv(umwe`?PD1zoj)ugS|!^K-;OKJ^!PF$vylh*g7}qX-8|()pC(wNet_ z%rL-_J$c5^EJ4Z=u^DsW(Dm7U{qTYi`P_7J%I<9xK-kAjgA{M3%XLi~HgVVUX5EiZ zqsK0}f>iO!xw%J$p^W)2k!TaE9d$J~ObM-~;q+KY(fEy;u-1j z#`Sh$ct7^kW<-ui@Eg6{bl&B+@7a)%r(pbSa_hLSWJTfNxF=TUlV%4z zItMfcmf6cEKE!8@3!z*iX6QYq`1F0^BDN zk4g!+t)5r!$ziR}`={cmH+P;SP*&Fvb|q~njR3QevMDsw^tPE-_(i9G)5n>h4b6hx z`Jfj=)X#YG`eDNQlT5CmezNzze?1>puJ|p^_J)EjU9eiMfm;fd8X;KN2#}Ph)3u*3 z|B&ez!5IKMYl--JAPAhG8Lg04w@2fzgHV5BN0=U>oj?a?Ink_78h5bsQKMMzE4*V= zAJfR`^-t{~GTetbWmZv)u$k+sEelG5LWG+6#Cy6ug#kH1_#XGF3r0u0A8;@AE=iWpl@@%XJ+ZVtD`yy21 zPzog&ak=iZZ|Oi7M%M{m=lcT<5?VRfJ04?GNtg6rI z6Mu-8&}K71%71;bP2%U%9AaUqG_A6|=ZDt(;o&H9Q-`2^)L&?6Hf^1vJ4+J$AqdHu2cZeM2^|J?*= z&+`VcQ_opP4bvnmHn^$B6Y&JJGbGv=^Hi#5util-EYHXwjoVO3qRGt4OtHC`W4 zpUjgj4|BDT#m@WVK4r>KSI9a7h>J5)%c7Ao{F=%<({@e4_V&$Q6Bua@d#{r@G?z$G z`MgQ)giUELZZ%A%ZVhPhY7a3_jBk!#3?$qF)MFHOQ*Gk9>Vy(&$Q}8 zQ_mQBuFv9;!Uu=A*85ATj#Lva8Cz4GfR$ifot&ToFn3`;bh^yhreWYw<#SzSx`mJayr9$ zpQJd#b+ZaysX(%MSx^E2gaypUUtyC7&05z4Vse@|1TTdr?8Hjek(K<%pARn4PeInE zOIJ2(oCod0?L#C7HZBlFQ0FV_&V4y$o{uuJ} z8~m`@Vet#RwrT@qz6^DMLm4ADDVsIB9Sp_G#TphDSSay=O9s4+2^DY8NXQ`Uq6Djq zW|y91&L!kKDF7@6q{`tR`Z$qJSwLhk-^9*!bKLX)oFE_JdPnz%Xt}9ye-82e9jj0d zJ>JQGkOgl>SZACxuFEL{aA|+O`MUMJH-H&>))WAJ^JQ7tQTwW8TwDH_iQ`c$(Ql$i zX+d-sTde=zn76$%IZzGU@Ul;j8vyfJJ!izu!oPG19Vb>+klZ7DCw1zJt6sRE+qO-a zvS_-3T|>#D$--Qv`vTc1vF}1ZbLr5U1|Q3m(#Pz6PSe`Zr^_cOw=@4xZjfJBJxwQd zu!PyyeOJn2hDR}6cyqZ|J5k0$Cw++mVMXCjhB6x0d?EfAHgN&sKRSd5O&IRxb(bCA zELXD=eL-D|_rM{3s3k`O`AT~c=3}{|yZU-u$NBWT?c!hg1jYkUePH#9`%sEvwylDN zG&jt3VC>5UPhS2=N2Q0KsFChI*wMo2iTh-+*7dh+UDCOvE}2+qTxz|IS9a1{Y(Kc- zTOtuxGSJEER`(z5`!GsYOa6=ay*aHa-N|WG&cg&0s+XFzYJN^$<5*!r`o(;cfW-;^ z&=q?HKo)PEf5C_x{FBcluNLcp>f=>P`;+#>QYeWNJ=hQCXi~*yGu6 zztbX^=Nr3{fbs>!eQjRA2ZcVek8rMmMezHl%l1k`3KQur+;jdD^fzxV=6wInYWKXlM3(tFZ@k;9h)>~Xgx13>#*MOgMR`73m>5x| zp9A?ZVd>|&PS02Tw9vsQVIID>d{W!Lah)LiK5^0^vh>5ih?qK=7Q1MnZsfBgExgmc zMl>VOZjN6n@vNrk^a!g>-q>=DS4N%iBV+tu+@a32yiOI&OuvNKO$*hZnV&s3&a{yt zz%@Te`u30`BHy8kVn~H6XGUzoqb&5}FZ8!QjH2bee`42V99&MMb#SO#9PEf6mnBV; zI8-8YLDeG<8{fYN2YoOL!7c5k$X%alnYPQFvW4`GG=Ok#^!X65obgwD?_9m2X8tAM zQE#-I!db!UVrrh8rNfx2p{0FY-q${`@XvRBGEBbrNGJQ3wXa7mD_R~7X@5j-%YG$} zztXCR{Bogrnn|&+V`2?Z$A;*asI`$Fp}(3R3bZ*zd0EfAXsB230JqKFCJ)#&-p>D~WZrsZjrp zBsa|oJNXJ1t;*n%PHjAfGcd?7exdtHw60~UgHDUB-A9Q|(;SPpc(+E2a!h5=YyeN) z#Rk)M>nd&P-OL&z$GEZQA>^CYrVo)}z47|!w4)Pf)9bR$;@;$9)OiMtpDVEH8QGyJ zdrs0m{`}+bsC2703u7-Y8NgyeRk!19&r5mXY%dK}Zn#8BTfyRX?fdi`9`>zxAI8>` zGLJ%CwJbsf(pjgIW8)B7%bzDA?Z)PBVQ<}v1SS`u1pZhxEL<(D)GW`=JLXn$Be5^> zMD&5oBRgn`Z=TjHS)#Vf*f^rX5_W}Ml6Hnr?z>ll%k~Y3<9GTcRJy&4<`rgnwD9q4?2jF&x;XGMLgmoqeO$m4CWyq^I zwyKgns~hf+_DWBduNGtE0jVY3mP=OL2kE$Q8*^GB3WX+R6HpoJD>jR?1x+Jxtv!5` zH;bUdro^jN%_vDq)I=$n<8p&_g`NV_NsZqLCaOEP%q$kbz8ikdD`Ny(M2tu zwlp(^@9g;ztZ0nbv~w6wk%mN_bBtCwX1YqkzI%GeGH!=_D$k003AecbRLuv z)jCj?Zr*=@M^|UW+ZXdhK#0D~r^|+3-;wS`MZ242pq>iZYTo*P9159-=T9IqI*yb# ze*+(-CV!<4_~@lra0#>6bXVdRxo|JHZ+b4s~&-aP-vO5My==8Tf79?i?Vmr zqJZipNaO|984P9M2aC|1y_Ma#98kb*7n6O+{l@>B1&HyYk_wHWxAmT;lW`gAJ?HXf z;2}MJ0X?L+q#IUDl1`u_<#V1vz z(h&8A)^50~!@<`%m9TPdb@hR{f5TM_!b@noL2x=3#=W@3)~P=Y!W= zC!OS#ph@~BzVK(*w0&*SO}atf73JqWNj*v6fvYO8)!!Ho@GENUcbFTt;?D1?@zGp` z#4u0kKi$&t_#suk&-(k!zJ14;@AIaDCd$~15Hmx?VE!Gh-Zs+=`4%H+o=10zqXu08 z&yF=_40R8I7Kez}Qh8f_vipyI0c6a#`gVI>7%w?4f}=flZNFdz^6j9S0s~f05C+Q$ zMiAN%8LjAF8qcTk@)%vfkh8*BSoq#SdtxoEA5LG9-lpKoX=9;iyV*wjs@S5!}nKADtiOfWG@>gjr~T1g_(@=Wn>OFcx?=AGbSXfaOE2&{*P0Yb>Kto z`Mqc`A@2vBqnBJIpW*QA;|Vwzo8QvW;qN$;gOo^)5!sK9E5lU-Rcq5Y=&4B0b5}9gbni<0{{euiWVVWOcNAAr~%3Liw88$}jBU zE9Lq!Hj}OZ^#s*^qn!Q@x0bZ8jw71kj68X?|K~>9Nl?Cl;ix+3Z8$17>4+`ymzfxs zYLsrYdS#b2)|2|_9tyQ#Ml=545k||6^oTaoF%m|j7Sz?zzrOYQ-7-WU4&G}StkQ?~ zxA;ATiwv;s`J}<21v?kc&(sow_z5(#sdC3Jc|*IXW7oPHJ)r9w0%ESi63|F-a9x75 z7kaIkjj_6a7v$3c=>$%|EhkcE4wibyK=B7!V&e-s^Y4&7V*{3Zyk&+iq57C zEfbuoxLiK7BYX>Q?yo;RUa!63BO#i80DQ0u6-2I=-a8b9)>GGV88v%9wvD0b3q!J0 zW4!rv)kuF4;k4T zAAfXe(rMHPOmoD%5*CGUw9HxPgmnBZEQ9 zt@aGyn<2;yXKD4m?>!}D(u#?r+X&+Nc+5q?01&0KQ5uLjEfM0Y0$pk!3SWRH8?_9} zD#p;gWDTUubk+vV%Sk4Cgd)KZ-(o;H<77xb;VL6G8!7u>#$RIpH_IyXO)P1TJ3d5T zt&x7#$!Dh^`FTIIoA^C_Q6C&Y>tnmMR=)0tGB~5+tHbBc00%09ZjL9FmpN8S07BMi zx7kpQ%eoeg@p5k3P|cb$boa5}_AKR)a%ie{rq*|akfVTpyAt_*?(5rOBN%$#<&B43 zt7t^Yj{H`^OBVxsA}Yg#0kl$Zm))?cNLMQsQ5#$#3`qZ(3W& z7z$?=I(K`<{W0jb?EX=+GJUMWBR=6M)X3fI`;(Kt%2jnT6K={judP$Ygi`ix`R%*} zAbQ4Pb`Pt9Nl*XnKM3x$rMOgCi1L%6+At;ZQd~QDZE4_xGVbY8VnbSQmQEfZqI7~y zXl-d{%tJA_a0PPts8=jb5!hE096k5tW{B#HSjs=ty9pW}nVBi}m`Yozm@yisGm{jp z(bJQV%DDrkc4IfWM`tvHz2`LU3gh?nl4ReMVckUp+G2F3l=Hjn!{zMr{7YB&{>dUj zqBB8{T#5C05Y_b(){_Ir1^QD)!*-4g|{RI zs1&z9!Rff$Df0hHeSB!&q4Ubl`}jzGhTgYZVW!Bd!sC35f2Tl7xp=tT)%VUygu|tu zAr^$+3^DEFk|m`i2Hi3=X_iC_$tD&EY)-C4}qZg)D|6*%uJXJ@+z=$CEo%|;te z{#(V_1BtY4?PI!Slp-3YP9I#mhxzRJ3wtR_(CChZ_ zt>0x2XUhTYEu==FTf5~Pe@96c@?{~--uzanb+qlEl}MbwTj?H9niul0Bop1x zN3g`%f|$iQI=aCfZSQo3RI>^J4FBW(nGgf`nq5<`+{nl2f_cItn3(+hsfLNpgK*@r z=Kc?NJvO8cTSI!g!^8VxLi9oK&rSa`l8xEvCGSTIBqL%eQDcOUPTL2vLI=k#8`E#x zRev`k`%iH**_?mR%grKknh6tn%KV`)6F2{-w+U|}FGkdSdLMf|)-0;|fgusGgBu(> zxVa1JFG2JHz%aG~(~8$quSeW`ef=u@g2v}WS=Gi3^0q`|$GJBT>HQ&S9J>=I@z{c- zaveA3OP^g~h5%@0h$dL2ey48iaqHQ8yFzcf6H-6n8P@GaK0uYoKl(;CHbGxCCU(|s zwqnPoiVtM6B)2aGzq0oGTCn0ezenq>)J)%$;X$)c301N-6&iy%sW+Y1T)2_Ss#scf zsZF~Tx=}IzSVjdKr~M${mAZt#=I6^%&E|@%0)r&<@=$jdinsii!3J^G&y=(9Z?}eU zNfSf6rc(;`ei*}KmWAqd5^4z1}lom@+GUj*|KIVxvuQ zmi9pFo^Sy6n{@QI13P%_B{@){izGv{Fg~v}<{JMNRdgg&3USJ9QOkZ9>0i|!xBPo1 zR;#koiofH-RIATy3CD;`p&sP;#MJ|$y)dY;0x934_OM~rx+^)XIi^O~3PGE3`}dHA zU!5mYCRl0z65Y)@)jZ0&>n(wC)fmnOcF45o&E{yZ@WNg8)PX(Z4DTYMgm1>U7*KP+ zh{jpWDAAj(M2gQ&Hst5GcM|%M4S%QVG>NlgNDD~hC)WEGXKzeZR2BaLNPQ%~dh(}t zV6!nPxZ>U0umiHu8OsCzx`b8;1pM=Inset!T$-zTPrauh*jhGqk7tm#VR zHwQ=!iaX;<1=cV(TH7U#ttIebd-dk0vkjoxCxx)f2U%4DD;y7BQCMUAS=c>cJ_8fF zhIv*Z)vkleEcJN^9FKL@f-D8|%Mx9w1SV$xFT3W0z1lH+e?rP0> zsmG?7!pZ`)Qt4lp{tPPE^%BEn?VTO{GI(t8(>nA#GC|`8<-eD*AzyRonEUhYBaux& zIMQSMk061E2MlKSX$j+%KB9aIbw!_|S4IGY_<7HG_3@<(t?Qj>3_dx@8XGv?jrRP> zH^^#52dQ5Q6|H4eab|(81b{;AB?Rg{11lr09LwvZw9TdFKr6`!a7X|mjbrBL-tHRf-0 ze(2En)_<0Q*^%K?+bzrHn;iKzTO*Wy@Xmb*2WxiuR>x=>VFNt}s4%Ak_55Z}en0<~ zeIWH*UGBvI+{(dA;TLR)(|&R-rC9TRoQIcR)u-u*2J1=jiNzQ|LwUir>rXqQU|?`{ zcdOo>nn?OeMgh5MvkB`lSSdnHWLLA;txIi{;fOkQFUx+;+@b+u|74b#k6s9~|6bwz z3>`kW$feFA%;oR;&4@*6Yf1fGtlIdy-{%{9guryyXGkCOXG5o7m|=*zgWa^&!Y_O; za7d`<__{H2a=_QIRrHgwxmr$sk7x!Y+DV%c=3_Rzx8(wJRJitz6B?jj??rQ-)HzGr zuCUUUFv$L9@YdOIQ$iJv9OdZ1WVJ>Pbi@Tar|F~O1Ow}}v#l^#6Q#F5dN`%54-+_% zqw!C&rp-q5zPIMPhf1ha4*rkf7dBOh=G-<9l%K%HL6Qo)$HwM_o8j;D7=(uKAhFf# zd5F_4{F(V4?fJ{vYfl#!wCfI~4_J%d&MDy7q`FNfPol3;olntiD2gKnG zMpUmx-mR22XtP^nTwh3hq$i0)Q+QtX&+kOQsPSyI&)uu@Y|uSpMCHTF$DPte?u0)Z zvvsAHj8sSmqUpWO&TIbEaVOtfdHtfdm*uyt7>VLMT>KW5yG_~j6HJF^-RWPlKP0=J z!*P7VQt8taEsZ4>hx2|vA9En^V?G&z$wk~g$}+o4QU z)#Ih=nE zqgmZcXzYvZ+tF2ec1)Sgtpbe%NMsvw@XWnak8lXjv-E3#hjyqrfwNU z34z@39i?ZWIraj_GJ6^ULyeWcz3sww1$8KitgX5e_aMSClY@y#Trx&H(_iBFHU?#k zb{jeU@y`w|iitX{8Q$NzGRipTwBS~A8(7E;JWfc$)la$(vXqMoZ#1Y;bLCHiE0PR? zcUgwwLcWJ50VM;lAl7WQ>_m|7^WEBSfzOV8O2qg3f@8UE&(u>#vr7qQTG=paheKSzEx4a6V4d+`jPj%oZtjQ`()FN|S8$ zo)^Yw?ku=@8Cq|aFZwPHu~nYU$uFg>T#>nI0=gerXb|kUoa&8@;wq4=^?`6waD!<_ zkV?3^zB?{u`hb#2A`aFqcCX{mii4Qg2aR;pt}nr*szO|V;t`-o_aycE?11PkMSTu3 z1wJP|Gx$MI*`ohEbD}Os#sMXlEdrU5z}K{}h&4nFK{RrGtO3M{b-m1l>hs@xxRXsk9@~3^eaUaKtU2^KIPm1g&b{BBypQce zx#0RbPQFZsl-4)rKm|%?N@W2Kos_0;Lu~Nf=rOH##>%^i3k`wk)xD@SS4lk$ns@~2 zi=>GMjAb<>F?$bu8?DrVZFu;=5rbNm^nN?@Z8Pb3I^BoVDq#ikpGO_b$DY@dB6h|+ zzvHx}s})Nb%1DaamsgB=GLD+50?D=>e_Hwn%jre#9RDz2b52tUzE7V|I(h^>-Ro4z zRi2GvRugTnk?Ct88b(q2;BUTsWL}0-KKB#m3@+T|j&JZ8w8A)6y=_!TNE3KkY>}SEMX0~w)S1N`LhpeF-vK_?+m$l5 zaj^WE{o%hE>=+B!M<*4%5*wYkC&|A2g5Sg8T&+p1>#3?}c;!*uiKy%)+hO`q`T>n% z6;NCrycPwljW+d_Uza|)L*;YPG3WVoQjgl3JB}3<riSL9%AP1C{wM`u=tW)ZUQ)9~S5?0!Tq<3d3F*AU2&{in|p=;s~ zcy+>+10s>oSdT(Y$5G_`noAGmL~&P9JyM~R6%8`yt7l4cnqeak{M1PgQf~kor|Jlq z0-~j19B;Ea-VffG*Mazz6J3Q;iw}2gg>;lO&zLielaK!4_^XZ&lBq_?KpQnf2PRqg znL0PG%A0=5W5*SP&1V{=Xmea4{^}*;Ed=WL;fEHye=Za_@GZ;8|X?$&MMyg|~` zDWrTVi_l)t;oA51)h1QMJt3)|At&4Bq{?#(?IZl<>^!tk#FCsyTgiXF|89)k@H@>* zroM(Wvu3^}%=Vr^v@Ki#n>m*8e4VK5H2Bh#>Op&$y6^N1X}oGL`IOO?-x!F~_rX(z z-x6&!ytb3l@A#HT4vAXs<}SQnsb%C;d_#9bNA6xd?oOy2>O#WB`uC1c#0MUg<*7`W zg+Z^7f2a~_Q6<}*6+MxSXyDW$^`$Rg`BpSf_TnL*^jACt&4^7uSSykyZ0R7C88(j2 zhuZyVaZ-+qPmMn%dX8JD(v3Gy-pwPg?Zn`f5C`H&|EpP8F3X(`>%1UIKaz!U3?zn^ znr(WT4M6l@nE38UUO{*Hx$c=wdORmDrN}1k1zl~Nvk|T9ty*#L8>x@w>S|i}Zxm^T z3x9x^EB_!QQw$xO-Ra>w?X{p&oPVg^u6`Wv`|l<51>78X#r-%%t@>W4gJaA2u{YG| zMmd$B43EcOXBiOgy~G@yXAp9z6S++dfnO-do$! zl6%R+=o3|F$Q}Oji&+YCHABzXn%iG-9WL)Y%W8eTK{zW#hHbsuV;a6i=e6>^lfq~` zsaMXCC^GLTyg;}!Ifo#y6SUE==YCu`FAy20|M=pyX#LUeC&3%w@vWbnh3iC&ZJ5Y^ z>bb2k?@cc1A@qG!zHZ}n0J!flRM2{eOTcK_IwVGL}+m8HzfOJ zmuX=k@9oGsy|2@eA{Ns*F@5YK>!GBoJJmy@%Jq_8V(<^px_7}Xw>Swtle* z9Yi$>8L835Ysr1Ei!xSP{->|am{ot*GZ-Cg7H_S)!>H87-}sfmcwS8_&8vOMR$X@* zR?K3txkruH+xya5VxBqD5p53h_hWH`)GzYAT-B_Gr#=wStk23ht}cN#$^-Lx5MWv} zmwRyO&_b@Bz&ZcF?A<$j?+WNWy%DPINK<-5;R$BVLOJ-**mGfN@o;aZF`Dmy>FSZP zvJ&L*WcM}hbh`3czIHim4{81DOWjEoS|fqq9=zsM5{|lp?5w(Q2pHIZ$Y`9(hksk2 z&VJ8V9KiK=vZ*k*wfW3q$M4h&V3wCix%NKSDW%%MPz5{BDKQ5L8LsUbXVXQ%WGRD?2m9m^t-B949Dl;UwdcBZ@0?s-2kCb z={(eRCc3Fb&3@Sh56U)CI{=ha8TKw$h_jl{%75q;lY`!dUp2E&czDN;;BHnC>%Wdk z7&CSM5OHX7^@s`olyr_mY=NtvrtyAhxJKzj#r-_Pd#}b?3ISe@18 zgb`n;lv!MKZmx7Tzg&@YDt@MZHH;5R#k}~LX5>?u?fu26zZ>IT9?GE=A1rclG>?^w z$#!u|ma87_=pQXKP#Bv2WFY!|V0JV(IX#ixhsBJM^Z_bjZ#-_*tULqXRuW#zb+!0u zfrv@#W|8VB65wV`{&CJ@)XlzcoVq&uCR@Dv^%yTQ1U2`oFO(he!wLZMB=tDQuK~ZQ zUPYHA2x`DZo33mU3>MuJHYtF*&C;PlzFPOQwCe#&S3#)8!XCEFuOJ(&Yy3DyySDk( zxU&d#@OxG}!B&Jm-zwy$u7)WQa*rz>$X1AW{=gcw#A!?edi$e3vfuZ-C`$}c+g$&7 z-mV-N#4&wjGrX%<$MT5w9KwJOC!4UU(LcB%9fpSpg2I=hQdG)jl&U{q#X5!Gg|Yv< zcM(}AcS$t2*cXnvHh{ zi!O#A?I&vD6MK2@!(pMs|B6ny+Wx~MQ=S2Cy_vuTa?4Q6IWh~bq)X0+%|%s)qa zl}f*N^|t?(Ibl@3NlAUn?xvLEfyi=Q#M3>~X+N%)eNdrKQP0Y(`7{On`DN_C_g^K2 z$UR()>(f7c1mM1T_h>7e+s<9eO=$!KBA;ER^d%7-~HbAfz_x)#MEn1L)h5WA}u*xPLN z<>CL$0@S%D$A9P)l~KO$xzcWOz;gkya(|5RgTi6m++Rj!o?=xv5_|}JgEB-ZUGqyn z(~xu*JHdH(Q(AGVp;lZX4$|U%=8qR{Q5TAkNT_ww2@#*;U7|Uu8t|2%2tB4ht4aSQT=$LV zJjd|7Jwz=BTf4wV{8@#mw2@GIER-;TWj8^0!uI!k11EpH5c?th6+YZU@u%_x=3vE4 z0%aGSQf#d1&Sl-^OBieh9mzkC@+Y!rI^^b~(Q8lne|n%6<@DjWi)}&O&601&fGeZf z9ETXY#de2-E&V)YLByOEGHc?;4~0M?y6knl3eBN z8#+aqRJnH1sCKc+@I{vW9ej+(f6UTS1l(&krcGzozLq%mcjxK?Q|_ea3Dbkdp1$kr z-ipdBWJ+HVKl`_zh_jP-o51UYZA2?iSJPWUZ`<%VzeI_2Eu2oO^W%B7ahbqZE-Zi* ze;l!=8=FbF62y81PAmzt$ybZDm9(BfrT3%K1&i-F1z>+jyihqB@nhkkY-N4Z7P)>i zwCJz#Y52xYR+)2h8)lOj2Hq2Y?KkWnAE${cr&Va<*KudmJmKqyJn|NIg2Y~3cRO8{ zT&x>3(eMb$*l_{*iWUcs2_L>Y5Aj=bwhBFkXqS0(`csy66Si76kJA&T5#(^Lad+TW zSIIqK$hSA`XV4e_KpuQ#jdxZ5v)LhyKpRH7+wK^EudWL}6JG0%v#u9PzffA039OQv zdU6gU+Nm-GSZe}~)qkH}G5IKeSny`=pS@)X1(KD}ZX9?{y1dzpWzB2AP%A_&OGkN0 zFfv}q$hz{KXNN1&e{rtCFZ)6iPk{2{P{DVqxV;X2WtQ%tI3vFL#s`+c0cV17em<{@ z&!E%|!-4_7sPbOndF(2#$gi#!Zpcq!lJ98tdFs9VMmdV{10`$_PGSOOOae{Mc z|Neb)*^teaV>o-)sRu5UO|#y%NvoIgO$4cR!_>=0kAbcnn(9)<(_m6u1|ORykhso~ zUiZ{5oKREd`|&Y@MGaWPBxEr~0Rrpp16|Zu&*N|$XrR;4ko(F?PX-K0)&C&Ld{p^Xim7;8t&yrjAeWy`>Y%a4q{$INUO9S_|ch@^Nf3+*(3l8?x_CQ#U3 zp{DDqi{I|IOV!2w>tBPvBlr9bRU1Mt*Ari(U=M?!`iIe1N;Qq+VgS!F@eY8V+Y3YJ z-&;T1TAZswk)$HyhR1Gc=X_eSueZ`J{Um|$Nf!!<_vA~ng$8|wJy=dND(;y#Opd3a zr)%f_UsK;6&-C~BFNL%sd1twdN?F3zMU*jATAE9xNG>B5<*u1wn^37YZ%a!Oa}Dou z%l&>?A{${tGxuxmY;)UOe|vu(-#>om&+~Yk^VjQf&UrnryA?J&r)*I34BOzA;S!^J zA)cVG5_+hEIt9mCw={fm)0TIncLIlkrvB(o@`j$6#ZykZ7^!p{@jQ8m^x`pDxlwsXg1TJ!J*qz^$VISc4#RdlI z$p*sNl|T260K4CsmaQjEUqwX1lg!1u>A{+11bc$k*Dccz$9q2xiZqV3-PkSxfiCbO z3L{dyF%$dyeuG6*=zS;tK@r5nHheG4pF~IS@F~Ja-_J0OD2VYFCol9Ljgt9Gi`}J& z5M)$yl1tmy$S9u;g#&q%i`1I6+gZDS<14f+V+4$t>R|1a1i}1~TWaN8GT5+Ox}SC} z!(t;1yL{*8kP)utmCvU6_M3v9CMm^F8%J$7@_5(-VlJiRKz3&Vs#p-d+xT+h=)hro zR{zpaj0J@Vt2PGLSBQ7bG|1h689TPEKJvh8WSEXl7%p>@`8+uSX@76+h4H<@Uek5; z2elcnJ9}6~!xqItR#Lz4hG)Sj?z}tq}Dc}e1 zo!YAjTF7~^?wVa(IP7%Q2MlcQbNvISAe%^!=j zNb5521c0s={+NdWJr@-J$85N@(SbD?;l`HOn0+eYz88+O&Nar3DgMhjbn8NTR6!9( zMf|Z<(U=y>fUaTKMSH!j{<=PR-{|z&;fXs$bXL7sdJ%_xP$edb$>C+sYe5esV`f4Q ze7)vhG+Cl*cNX=ZZgTBq$lCStEt+F}5jgRy-MLDzd6`N40ESvX3_?fQ^@VK3MexfH zAqxnZGdG9nXYw@re2?4Xnd|QFe})$5+hZowE`q2WV}7k%D0C3lKsNl$C~^t1UW_yc zyP>T+Tp2136$>eD33%W}G0JoBU3|N%mfEf{crmy*jt+ELQY zQQA4GJAj?3V-;K6=q-k=>ZOOE`mNeM|F%6hntSS-bWZQw6-L9Q-p?sj*suz4)BvUC zodaPk$hB25tlaBY9vsjk+3pP`4RI_<;@`H#lT4VJEqy|cX_yBc1 zkk<-TsdBeG2HM(iLqiU21*yB)VB)eh5;de>Nq=X3puPT(%QV)84RuM>&$v2p>iZXY zw2PcKz#i@Qd#Wh)VHxv7>6MfHov;?`WaF*|+fS2(_c0`k3#~qSL-Ywjk(j)Q-uyhyI^iBYd+vzIM$1 zYR1h;&zP5OWpx!|)+dtnX>XvlY#DfeT%f_2`4aDxn=@%#&IZ*Yg{H+?tp8ReaG%>L z56^xfx_d;@ErU3oH;EW(&ru(JM$N63m{N2=a&S7pnuY(gz{^gu>Y!V{f+tU;vN6FK zR>XpTgjv^wjq0_Bis@oG9rQ2xQBk|f`9%PDgvqwW@$Cd3T?renpF_s`>d{wn!c;>G zl!y|;OznTE;&%QjRGj1I_?U^jMI0?(KO~&A@*NSOYinHL`8VX4dC{nbosu5Qwk+TQ z?DNvhp%hO}iF2zCVHRR(iG6z-WxF+-qKvj9{u!4HB}qfJm{V>Q=JBXx&zsT;GvqA% zIXzAdFQ|$}wqP-b8`V4Fks1VuH#Z`)I2ME}(+px;UwK{N!R3?X&Dsq;+5A1Tw43nl5w1Gkp0jPYpTUa+!5Pg+F3V2(dEa#^v*$#_mr!s3F$Yn%;B zeXi}NbV@S6+$=~yx3p!yg7lI9w54F&l(%Pdlr(F#-GK$U_%*Fx25b+PNt#Hb)?h70 zpZ)<18?g*xXlYVne#ML1q+o*Io5V-cKV%nW61of0g+*4|Z*1x+m{fj;$;S0p}E%T8V5d-e&W_}sK-y{R3p z1mzjNpIaJ+r!HVx1m-FIvkbyxqYIYrCdEcbSw>lsW$YsYya1$j0h}n4hJQj^@|0Y0 zV1$jn11@%_*kc&`fHr+IIqP*VkWxcEGQ$QH`Z=<7H5iQR$Lkc^#;K6@n`DjYHbpn$ zWkgm-bXQ95G+DAL>F@c=qHZzMiXlV`ZQ6sOIEf_>=v4c+Yr}oT$z>L_yTi^HoxC}!9kFc}+GS-qs@fmlIDAu>lPRi@(i?hc^S++|Gs1@#bOuMvR(K{CZOeozJ zk6%gAxjrQul5!*P$5sABv*NB~v5#}zvm=GUP2>XWpZ_KM_n+Snv&_pSgB|tW0bWU4 z-}s$1Sun?=_DCc@QSq(2ws**BDfewmhRyFF!05hX?GF#1N;)+B!a2KzeW;!2n-x>C z?A!^v@?eO4xc%+)k;4h8dLch-&rtfq#7WCgr6QAGmmCQe3K81mXFLP1!D^A9Q2MGVs2#yzR`vgln8Q^zy8ue4@o7Q_anf88vlnPfSER z-+ohX{q(R}Ihm~G-)_cuwezQD?QzF%o66wJp`R}rCv~M8l!kT8vXZ;31%2<8>d!i5 zTntS@kM-^vw1PuaXnx70;=6hU2;amM6>c(4;=pVM@#GiC)*q#J7|3=O3mO&QE-~`q z^zsWh(p#yp33nSDsh;IF`F^M&GJi$A;8$1X{0~V?lHWwl&w`%wl!>HKC{^ zg0{;X={&u6-hVABI3@cnf&_73vX2?EX9iYGf!kuc3!AM#Ar$AJGR@j|C|EXA?8nL9 zyOBb;|Hmu}Dyl@b4@Kx)8S#1N_is|_4+rn4NElS&@3dlB*E`cHH{=eI6UA~Gd$Bxc zBvxmXIWyseom`o{4G)<}Y_QJn5?Xc&jWVMD!FMhlZMkW|D?VWWup#pKlgp7SrW!u} zbznnSku#V(p+T0)Qi)dkxWmKkOP4xrMYya%8cKST; zido*8Z>qdJ2-T5X`X<-)rpB|eoTi$+e7)E{?+NAy_#_SXvag=Ka)N3t!DaZZP0ddKx}Et{0sZ>({^@Stp(A2L4MQi_>D#5s@|pU$CFZfF=T2sG0gb4YLMV`C4(CH z@)}tajr|@?k*csN;|XQN+1X`K6)5ouHK*}HCiz)MvbU6i)wBa!p)H>t5%-gvFAM#T zND}JjqBozL7D}@Ax$g0ce0pSVSReafz#wSA>F&4Ak0aXycgA&NZ)&Wcq2}L2tS$X` zeAmBI$v_#@6U6!W4HQ%-8|xny+_k@TdJXwU z%OPVY8l~EDEV!8&cx)hvE^m2XH=K?|>2&@xbQE$0 zo2GxY2Ph^s`!0_gS0Eh+w%qhUE{H*UMW`GXi80`MPXsV4a zAl;Lqg4w)|jYDs^ldc{M%_#N=aF6JNH9&7tc#@+w9(_h zLT5N12cFFK-8ATqJT$@5WtRFEtVC^&PNuXPrcA>-Elh9py^-1K)zgU$TBkm8I`5CK zcw_ZEq8%#w>1LvmQ9m1JeqnLzY*ZP-X0S;^mpvk*ghsB&c?FJ)8k66_p2GAPnTSdZ ze`Kt4zo;1od+wcA_@G$Vb?-eF)%%P`6v-vKqD|M}SLCESRpG{O5Po#C?k%*a+}q1% ziMuW)F`=T8Yg@Ou!Vd3E^R?fI;|)?}l3AK7qRBf7-ebRSh;|Pxk6My4THFx4#mu<- z-rQ4efmhzIcHP?Z0UmZ^M69o=RfT4oFq$se2m0%tqz?Ni{FusvN->L&RBryU;fG2B zEeiLv+uXr;&FT;HOAZBO9|8Br)Ez~*?q1Oo#-iK4QzTuxt#T=vn2j6Ha*OHn3A9wd zyR!hIa5fY}S1#Ve5n@wTB&i7^*Ppu|oDSK@LmlmPUOpqUu+#I<=~~tZ`_QYpEW@>- zWD!M@gW;;56``-o`JD z82;VqoV4MWRru6QC}+Duq4zYo+!!}a?1l=zG0 zuHiC)b7~T@k*2AzP2#OL>q)6v6MSx%+HJdEi@6VP4hrqMHH{Hj7YS7nSyASqARR{6 zsZ(A&9cR-NG_t(Rl&q5PaN=@(&bgU{67q~+OzQVLOP+p(T@KFLn9*mF`tKYn8zNPL z|9Y+btM6D|_2cSmlg5C|moMwT+r22cA2<|#L~o$5=+5)A^{}He05t$X=tjH9WMybv zju*2B)fV>*mhE0*8yvo!moyPJ>cL8F`gs*>bmrJd{9+wBwpMW8w`sR8N?Oy=@oTx! zji~r!OuR&V^VKg)N4wSSl0AFZu3CNbW~{cfg`Yk1M|b;zL5{{IEl}PD+rhUQKCHiy z`4SIvoeAB?ec}eFK2EKWE#a8dN2Io1mc3o1Ys-Foa6fbA{^O*lQ-8#<9@*YHoG%$w z2M)gnT;{GM?(yGe<{k676e{0bH1ZX01l**PdtiF}_hBInsFAhL0+=M-4cvt3Dw9$b z!ED;1@|=7DeRK4z0-o9vlDS(9f`r=d;B*KLSBFlz+&wQ@0`hrOysa^P{!2xU)p_zk zy*-v>v!fs4#m$rC_Q;anbVuc@y~Xa;7cX-9wDo?;+rq!4=+NNj>Zo|cv3WYNWCH5* zgp;Jr*bse91jL&(e0JMok~1*jDVmL-WHKITKaB1y9DdaJO3h1;{diBw@`RXUn3>_3 z`@;Hme&e?{aDDfX%37*M69J#zb3pZ8%!O?*!m#S?}orCr6g2BITUEgbqMUn9;L zMGl@`<0K$tn93oi!5gW1pEaf-j^9qco{u5)%UHtyg>!i%7?Gs6qK;(&nU!TX!Q(%s z?Di0N5!tz%`MZGxW05o|JUVtNYUfdHz!@A;oZXy2{A|e8ICGO>6Wux++E)`4c7%ds zGd}Y82S9|0UR#+?wjT}1<_U{{7j&AL*`R3TxKy5BZhrHvDlwgvwd{FO&>pP+KD~Nj z*RYRHr^d1Fda{gzV7^q=H9rle$gl=tei?74-b)annAaI0%M#D7dM8!9xdS933;fuT za$JUkUKL;`B)ZV1g6^wc)LX@cDEPAO&@zV6Y$2=Ws7nNv(cB+c-WxOa}^(id8ZEN3q zTRxavi#Vww z)bOC5ebm>NH-^H)SthJS$cv78k5K>8G?DhP&dsCjDejS)sy_Aastw_?NY zTAw#4STDqJkQ2tKC>*!&FZV<(LC(88=cg@W43XkFgBM9J9+A}SR<+mWt|nXII;BAZ zDq6xq5r|*Ask1XLX>ACX-50W@+9Kf)9!WS=9NbaT@cY;_r69b=s{DCdJKHuUW?PW2 z^Vf6*&AHgJWmQX8l!u!?^IfRt-uW4$e63Kn{<#=w=j*Te13n<0r2UiaVTcuGG)EQ> zyCg782r@~TZq>cKQ=-^v8WQi7w&mt7agPp`KHYbu`+hsM-nH46!hM*uJ>UKH3kqgDSBouK(5gBbSmH zjir_DMn z1`$LP_gc!yj*lZlTP7?42R{uNxThIUwfBMo^p9^(79Y~gGu!-^WF-7^5~~w(;zPgc zME4sPI=m*hI-hq=)H}ejR)LdF(1rtRn3gKROKu|waK(WClJnaozcazm+Zk4@jXe0H zpZgtPt~8Mukp)qX)-D>Hz&McFl5gJmIb#q`l}j4bB2*|okBe79T`F-gc0;8WR(k|g zs-O87)}eDdWqc0KW!;z^7-wTmYJd?osG}vzc409Mkp=zEGqI_lCX76)`L1Bz#RvX! z++*3otF!m^?_OcpLDlGUa53o8_^F})*iAVnd9ZsMbPJs{0UrNM>F6r+tQon8ntnXB z@wBzW$!={i%Gl?~`JkZTG$CLC#Oj+ps?ON!MlCxYq<{K0x9+uxM2XS!I`2Y`q_yT{ zwLC0l4Q(nL4b#A=)lz)tmDyfbVD9}~H7Jna>DSy`Tg@TY*9=OMvtSk^C+l*_qeB_R z<-^2Htc176z>k)8HFs@}zzdCY?s2Gt1-Km!lI*sK6%iML>Fnxfa+g2is;igg3WiP} zc^za-hL*lbePOZsI07M|CBF3p5gAkJ28@|+8aJ^lT9CrAs2NCM-&dLv2u_ODpIu&! zSu1w$hNjPOMN-N*nOz_Q*uu$a7-R{#{}a1cv#iMfpF4TS3OU7L?Tcw*DeIR zRsrzX-cqL9!aHt;pVQB;dpYl%r_50-HNhp+Q=5Pl|4M{Hw(n<#&gNfGkrPpTIQC#~+&7YL}P3b&?MaT{_ zZi3si%nHk`*gRxzu5a%@gF35sETE29Y+7Mr_CUtj>h9Te^TErNTpF-ha)a=r^;NTE zm3#42cn3bVI=pB1*m??WN3)atcjDVm_Y^qhi_`DRKPF+O^d;5(B>+}GnWi5xeW42f^~{>b@uGF~>I?lt%U;XS~8AgnmUf5!cH2QB}g zffA}{FI)jn16bPxvFLE2k|_FzWtsLz<=X~E0D3L7?E_r#gQ0-_9qERuAVUmK8J^)h z3Rz(%EpjHwsF2gTp|4n3+!th&&>^y!2_?+zRpgW-U?dqJ?3jE z0VW;8%9}TRYWs{=Au%HA1f-QhfrA5KKg8B)L8k74fNoS4*0Y*xs94kg%20k98HuvB z8#Z@a7n%d6gok>Gbj33!&RpcuMC+f#&QI=RcE^6H8gP?4PyJ7h{v&;L{ za(J%voNKH9DtVrGV80R-72l>dk?9hm9aW#LTQwWdk?npKTu{{n$R4;4viL-2B{ONJ z61XvoT85gcK8;I8z zhYs7<$x*lNiin7AnO--(C%C>IHpqD5q${KO9#U}C+?$&eXa5ilyFw{?kFa!{iKnIn z7ki+SwB3^1fiotd7G*TU(HT-+U|DQA21%}&|Dvfc56*A;60A-Vm-5*dA(!%Tz_X|; znd2SJxDgG&S!DUjm@Xw6A!M`KHInDaP>{;N6CXTv{t1HwuA^H9^My+5S*cyUk=a;C zN0z5Jvfxlu|4xE*nd2zOM~^_$Tg`$BJY4kM`;IoB8&_$*v8OWfx>0`!^9^*pD}lT<8nX%ew2FgAJ%;~T`H8B>JTzU z*vC+tlQHM3%^lz3>E6kwBwARIy@KP_>`US(d(0FP zth&J_gxK8I)%y9+YzR3*%K4=Gg8-V{zG+OX4%~mL{r;#@}0A`Bzg<&{uxJx3xomFfKbAUouMd9Ne)A z!W-jQ36>Ci4Ykw|G_6tpvv`u`cxLcr=z^q5?)(dSV4NSvPe7$;C3@C9v-@=5? zr)P7q(M_?jNpy0dfH`p^MM}^kA4-f;b_`x#=cgmxcywrJfNk{{qFkprlp0Ek|3CDp z7h%)U&bQ5l^cPSASHp6o|L3){QK{Hy2we@+=&aOh3Jv&Mo3E7#q>IA_7tQ}9Juv=lmA;v+H5#FZ6DSiQ11W#ZO89k``>2k*=5gAftN{3z7XMu hU->^Pl`3Imcdy`qfq?~+ASNOlrY08G^RGIE|3C9(FB<>= literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner10.png b/docs/images/LogMiner/LogMiner10.png new file mode 100644 index 0000000000000000000000000000000000000000..d2c7615ca15659df0b50fed75cee077b1897553e GIT binary patch literal 31812 zcmZs?2~^Tq`2SzIRc?(=k-1Ql%QTv%mZl=)w3xY2S*f{}Yq{o%3Yc1%V`@o?rb4C3 zlw0mQN}7iIk_&FAxZw&23W)f_=KK4e|NordIUbI{XXD=c-1|JQ_xr}*v@(@EeB$uF zefuQMu3x#mZ{Pl6!T(8?25@9*#4D?tqm=+yR;`*m)xf@DSw~+ zbo0!o>mt{GD?fNxcDwgDF8tDcT8B8$bjmqKnI)4Dy7K z>%>=_M(TE&e^S9UkrCY)YZ+9|8=&CNfLFZBYLt3UiP~M?Mcl@e!+I45ah=)@5sfzK z4Fj`V{8oKB?~Bs-hGl&q>x}ulX%y2G%3U~*35Fi`uLf?H0FC-&*S@^q;J2PnbUT6u z&?GTyJ-PbbtzAADMVIY)o5|hO%fq-)ij|%v19_0$h25;dWcEkg>FPEctj_cgi_Fw5 zCe3BewUi?V4Oq$yw05pX71g8fk~hA6*MBs-WvJ>Ht_AS(S92718pY)UyR9L?w^qd% zM-|^ovPC0aiE3Cv09`DZ16VsQ?NA#- zkL4VFHdf`&P={udawC_d8~x3Fv+5&v%96Ak32uYUb2g=Pui-}^x+V-0v$;DCbD(V3 z5gxhx71o*bD6MyD!ucYt??<2Zvh{G7z17wuE(6k-On3YZGr+@e`-N-_Z0nT!8agGs zto1VTPlVz}hydh*CvzzhKYE<}QyW39IxA08L^&Hw5WmDFpeKBAx+kg|pAZ{AL$V2v zq=6a6e(maEQxxgre%}VXsMtV`_4Z;0^D_XsWR=vw#~qEy=U^W+ZU;a^;e(S z(>h0N5l@u1A3`oJ^LCoR_oT({(<;=DAU?dgwDBbt)6}Go>GY6yC9UDBXY#bbg7YiX-Ye3W9!t##-758JV@_ovQnG_ZaoKiB;5mjX!~HZQ ziTK=}1`=I=aydkDb*pT>b>X|`*R|MQg`Y4_M=@21*2{IAYn@9TSZ~m%AFJgu2RF1a z#xy8}1agiT^L`$(aUK*yMR9>({*sLdXx==OvwpF)%wL}TZFNirgK6Kc%zDv=hA^$1 zOJx=Ps~hlkP%$S{LfE=Bi#_lragll}S$}-CaNN>9Brq5UN$0!NQpl76H2n2=3HTo@ zrk!+CjjTO4kh$Zs13O`i7qK~D1a+NSN%C_MHTW>zceQt_hse%p1MX}`Q?u|Eh((cC z@z}ZGVEV^1VJ2(g^d7yR_}Ql5Qe*1$fVMl00EOhO=6?Z=l~$E(Rb4s;=8xK9Ker`# z&Y%XnXTOP1ppfJKN2H<_(D+$;a0|e9si3%Tlj+J=k=OcR#iGhjGNEwfbMw$jgZt+& zqbW(e`1SOpGqO<^zlIJSt8UC!wwoGRjpVza?(}2_+kFBcd^3|(p18`Jz;dD#$ z7o}HgScU$GfVJr?bKkwkjjZktDJsjh<_dRkXHQ9o|Clc3FBMnPA7r^fv5T?J{5fYZ zVPZ}*j1&;ouH%^TQVUbFKq%h`X&{^qb>APc%Mwb$%WUZ?cxySdRrdi?TX&L73T#PMmV zuo0S%@zSkiLtd*tuQk3Xr3=U-0DTsIrU&JAT;QoUGStVa&x`gS;`eLq^oRP6^Eay? zkJkP9^ZrUs>~8=ir(Nza5VKj1Q(Oxvj?C`;QLr~IfI{2i3sC+nRBD%7yc@yZ>&E82 zu4qO#^5&YW@T|h-y|4WMt&;w*=4KpZhuRF;-GBhW^mwQb+?m^tt`r`+%Jtvf^tbQ* zbIQ>!@f{w_{2)ESYwz)gQ{=@$MU*y`Rm-Dxx1z|j_FXz~_X(Tb2Eq4<0C_S25gS#8 zirg3l+s58#vEAaG(PBmLE~m%8b}E%_?geBwiCOuSeFqp`5+kYS`)fjbJR#epuK5*N zJdeM=%b!-&uOS|iT-$pjeTd2o^Kri0CBC>b``fg-GpBv1GR}DAfjy+RII$DUP|8zZm)z^QF=Da;RT&ewP zYwxGWOUoE2N(BaA9C?3vW=%n)uZ-ZS?^zUG9JR9?*8`u|wZ}Tb>EAPX-h#6%tB@(Z zCv1ScDKY8M=4%vO=&&^X%Ptkmrw&<#mVXbw6&f22?fG)NX;Spk^UTM zs>3$iytu%cSVwLh^2bkyiD@1;rkX)VXD6NjeojnKYUqjn^&Bnj-qmi$0KOpVsyH|M zMf+5)7_t&`^>?n|bb*A$wpOg2V|2g4lgZAe$KJJ<7H&W?-{1y(db^~Iy0bGi<4P{S z@Y_Giz6rI;*iE@r4G(qw<4KKKv~JSK1xi$xw^Ny;>;dFLf~M(t}Z;pb)>>$ z=+8s%!fS3ltJ|z7Zs@8V3Fem*R{AQo1mT_R$>|vCaM~=jl)^^nDc4Cyv76Pvghs*a zG9hzEUxM3SF_b=^AZ&=%8nA@V7B|vlmGRL_MiH;>?=?VA)5c zk#I-0ej}%=>V1peeK93MMTU*&V6oP7Cb8wU|7)IDGEd?LzSH z4l3HjTNAJPQ1T+6ruN(S zhM8tLe8vCRTE7sUA!)Xm77n_MIx=#uuI$PXlTui~-8&``IdAU-&suE3A5?R_=y>X(VivO^I4~Ye6)R zs@)E=y0jpSRsN$|{9{<-_C(L&YK{xf>BdEJxJtnnL4Eo+2XDD*Q4V=`apA$ zyqv3Z)Vq+HGXax7%$(A8-zpOCktyY=R{CaiFTjLMn&;bgzuv{a=obOJ=lSbQz@&#Y zsWgzWw8gX1( z%3HmI-J_%8hdAi^h-yKDPf)GG4C@7p{1`?%Jg*hd0ph`i$}l0NS#|5UmHC9H*N!pk z5C7a+G~VF$#(~R4oC!C`ObM~%_bcN)Qz}!_K|-faU0mQ=?33N+A6k~_Tj{}ghwRJm zQCk~fqa2L%jb#UpAzTU`CkOqV+-z`^nJo{GdDPLt?3~lb&*+E#I3dPcIvLe>72wN) zpWFVqR-=6VUM88{s_!=}v5m>LLx!MCu(Nlu>ox~UO`+~Y^M~~Qe=;S973G5&1Br>X znexM*@l(a<$D6C{%tlUTVwHP{9xeSt&9%;^SC7j4ee3!X{eovq7oHUsRqWiZU7k!# z$W%rSK*nxo9s}uC-gAbN5l<=)2z5VAQ*bC3Ej>Av5l7H`DwkVEapbXUT_k8VJ};}p zookHr+^D!cIyxd{O5;s?(iZM+Ja3U+z?gi1mJe<&X z>iGHXms*RfhK9(U;i#Pg{>$!RCZd?Aj#Y8>4Mx#ELlH{_BoEMvhnB*>yTTxXEMoIR ztzCBLLD1f6-wy z_#Qid_9#WSL&k@b)=lc6rd$+(uj)x@gtbXWlq^W^(xj_0m0X7lN=pXVO{PPMw<1i; zLtEc%B&#(9$lyKQ;AxAFrhZR5*^%=XwxgErVs=g+s#aBK{Xi;^7mj~!sq4N&$-8Vs zYB{YvB+4BL;xB3&jg9GNE?-r3ZK)aCp1WV>Q>Zl(JUPMab>3DVRx3LjiO5HWjyvwq zjhS)#?**RwSN`S6clt>V7t{U>{@O&!R#wtkJJ=BY@7l-TB$R*W1?|JhPA2@m_M>Z` z!$HkUCSOt4rJ_Orvjq#fhV`G+qgzvsX1Qpc>NaY51cY|H@ac;63kl=Iu;KH)in>bX zcMqhUO!qfo_y{)+&$0#}I|E}IqKH}0)3ksrA?)GBT_0hb<3;@aemcmFF%^^^a?uk5 zpp$iq>R=`Xn-O>wFIGXkV;Gy;hWowsdD9K-Q^&v2_4B&L4LiKkVLrURF8wkGyg_jc zlh0aX_TQ>A1c0{#iuRK1%~{a2S?fq&v5ae}No7}FzdKyM4xZjoAd_11zR+)eaUe`; zxHXczJ$W`fL&YPE`>w%jDFtA(LRh#Aa>5Sp`hjVYXTEn?`or^NSLSmz4%tMRe}fO> znBhmk_HFOcC|ZrdOAM6R&0s!AZH(Jzd~g4~UB|p#&8xwuYIJqkU+hcY2uEc0I)1v; ztAFD_&ZVuSuzJ_JrxG4QVwWwSlODx7L%*hK=x+Hz$T7n&cRxh&0r6j zf-Xi6mZka3$vbcIU&~%vE+e(cq`276;7m`RcRe!lljKqCc`Cr~=2WL}XZ^fE4|=Cl zou@SPVuu|R)>GArC4H?09C{T!>k(OT7GxAtQpuLO(O@%!qVrx_>01Y|ese0T-&?pd z9TxgES`u3nHvHUPskydZ_=fQ;;ROU;ZY-GOMDX3Od%X*d<8J=Tpq+*$WAuVZveZ!AX*27v# zUN3{3U>f$Hybww6%QJEc8)|6yS#X{4gkY=m+s(75tN#SCKgEIpF=)g3hwBeFUB*wr zs$)Z=IY&Y#MN+Z>62^n#S2*p^u&;=8jo$n1Z;u$q5!8MhDga;Ul0Ia4@g($q z)teyRv@}_>KE@&X_Ec`Ot`s0|A4Y5PTVR;R>t$VYb&oe`Js$G-;n`1xDj^Fw3yPyB zpUk!4cYhiQ%MiWseI_0+Vrvs=c8!I=Yf zxi>Gq4?2S7meAFv8LL~!^)0=Ll+U%fYj9gVJ)wZLa_im9R6}QkhDP+et2aVps#++% zmvuE=aaGF3;*{h{JrCiao%en|b7h%ItLRHVrr&sVHaW-^{g^kjRGf+$3||JVH>E7 z{rFE;eb1@}OOgKkT%rhW1b~7pi}2OAxig)*e*&uN%m5hirfE&96t9yfejY$4v_5F6 zyQ!M(a6aCW^ggfcd0+hUfzQYjEodYt=Iy5&5sp^`IButL^?)pVVEvDyv!%tITD;&x z`#49h=yn8u_-KCH?Z;!!TH3~`#_=b$my0JbMzFND;Zd)=TE;~f0WY1n0 zGl>GVi%?uCJq8)YH1j*HYK>X4rVFW>=ATDLYnpIU*t0vmBQO0`{mgJ1*^0cgXzzq8 zW5;uw8Dah(-bFL)r-~uWPqU}uSI3e)$&jU#SK9jMLeEAzDct)E+G1EVDv>}v?I?A= zOs%$mLA1*fXXlE4U-N`9TCewUSz4RW@%O0q*hgQk@1{FQ-02K~o7-a(<+l_52zP=h zc?FR7>13JM{!dBoK%$j#^D!AFW4cM2qB`t+!255W>JvD^Y%KMI{U7JNX%5#7Fu`33 z0cknwr#NiYvkKfvQT*)q{*6x&=fC0w)M@h3GNXMosxvN#Xc;v#w)`hGt~d!oPMz-3 zRDe;ipn$T;<4fa%Yi>BnY6JeF8JLXo54J?4sz~V`i~n=H+*Newcuz4ETB(|az*;S2 zVg`a02IQnxesp8>=zSxhcaqhxU&QEF$C_b`#tpsgtoqta1@l*tb-}i^Z^xCbJHA+h z<;9DHLq8=&hyQl|&xK2VUrk->_a~`_SL=FH4@iOyc-En5YCVq>9qRgRWbO7-)p$(#Xzd0DgV48{V6e;bvc0H=#(KnuuFu{+k#D{BQ&q# zYXbuVI3VB7$~5?oDNE=%N9z|w%vODtdYqW2oNuvit@f>Of2|#keKt@rvd*T(v}fww zg~OX&2Qud}=B|rUyBmX>ZKUSy>j!h1)lL1+oFL;Tmgw%@L(@wr?x60ijxr}MaWh8U z&hfkJl!cgn@0O_Kys@toOehjkO>c93ngdnr+pJ6)q)5)(!sy}FJ_{lHH(9)QwFB~n z5fQZkBT z3ABV`*;$%(qmIKu9#`8w(s*_$?hM-ZoUu!N8I<=|5cSMBl}+WF>cL@1&8+%c-C0Dz~Wvp*on;e98`RIi~!4rX7`P}Qj&TVHsg`e z=~Ps?>OS2ZO<#S$?Uz(-3uVr3@}FVz_Xo^c$;*>Qmv2M$Mr&4_m7t|`5mGwhBM5Q3 zm2xuZ9fo-pc2tB#FC7w@Z1V(7+)*p+DTDl%pzZ69En@L6WIMq(45*oTk|HkXB%?+Xw)h4J3YWPK_o)TSv z{DWIiaF&@h#@FiDEP!+duybB6n17{559Hu0DQ)<;xjZ$=XYGMusoV}@cE&zHrFh+2 zqI173Y07)wx94J#3_G$sR0^4f>ugz{OCCgB%~$A@%9ahYfQHiVCP{0C`>y<9y8i86 zl4tu}tA(G93(0C3>W7JzBs(xV>CtYw#JG<)$fVZOP7&h3CP7dTLtqjo8ca{cts%xX zD?hHgmkQe83seyWHJF_J*lYvtQpF?@bFxf-PEBmPY7U zdsYXpbNVQxZKH7Y;UebQYMH@Qu;>XN(Y(Nr7aj(NJOyTChA51I0<%6qomfGN8xO1h z2qblR=n!ZF($$MJ>c60AiNCGm0!{UVQ`@ag;R`XGvN&=g#*KJK%A+pg9Q@{PPFHA4 zU0e`+#kVzclKH95Fv9(7AVc?NT!$aaCs$o@8k!HCfaT-DxG2r?0j3H-3mt5(U~{ zTboV@*-Rg|&5k@8x^`pURkfBI?&!VuRe(xT^k9a~L(d|&ij=_{xQV%cu z??i72ht<}m!S3l0n;_)c<9m=|q?4;lVl;4G`)kT@E#>~+n+t-uvU)TzIzPJ^(iwK3 zqbdYfyLVS_i7f2@`>^GrdEWS70n%xwe(yj?r>l!faY$Rme`g7vtU_MM#wac-eO)-s zf7vSJzk6JS{J#$ycI@KR@#K~2^u)0JHj{@9f7xI5G^a&sGyj~S) z1YE31qjX}t?=T9#h@7?;EvXvimBf zE5h7E2yp_+wR;i1kx2&eIvRN$p>V3Xv*Gy6v8oZx&g3W_MP6!$yt_k|-WAY#IGc%s zAlQ2l^Xz+b@a*EHT>--Hu>=_G-<9iuwN4AZz)GcbSnS(_?$W>DX$wOavZ1TS_>V4g zha2%!-l8cag3-ZE(5&?u7;E}d?rsz_2Z%ViJk_(OANam5reAMa6u?^>Dd6@3l%C|AJsy}k28r!CuJY)DNGhSX?wMc%+{`4H8c53uy{^u%9`_-qNB77HBrxG%zx7chpg zg87*QEwGu4NQ5p%zkzBg4@O3))HNjjR{TuIofdJ={WI;m7LvYMe?v!itgZ*KRz{8Z zuI^MqI$x(;hsNb(4DZ2wOQ2#>umb8-Rnnl?|IA-^L82bF`$!mL1S{X!7$$f5Z|;gG zq|VrW$^B4=kB5dpnK74Mj~K42{G<=^aFH=5UUfmk-i3F`h6|u%lsQNMu#Ky7=>mxH zxGcf5J-_TOGrBuEw`e5SZ-dy?gl6{knUcSs%@gbmKeyFSx zF%R66H1tNt1Owpg8J>Wu3~wF-DxUp(^ry-z{rc%^)h;*wuNOd4&D@Vvv&ZyRFnhX- zSWlO-od541+fc?@Vd?JAp;Zg6DK_y3l%T=H1phS2=1;MmznEz@spLE{tupfTZF?p9 zOH-e+8~%6n%(FKC(yIR786DI1v}y9gh5(CodV+&wXj^J6j1N`(DWcf{j7|9W`q`WB z(Ff-{^RGnrB%G3&6B0bOuN`9;xjL+uP5t3_=xfVy=C>F6yEiQ8Vhv}0Ahmofz=nIG zfG@-26^W@6pMl9ZW$fW3v#hO1bn_={fC7fIw}^>jCnCL998b43fe&Wd*|p~D_Igfj zPzLu3B!ke`T4} z>b#r}2&6R|3uGp;OnjB7%6=f-P+5%}7Eb^0>}J!JD0X%l#_4dzQ@vZ~Q$zffBGx?4 zKuh85W*}1M^v7m#fQJ)I;g>u~Vs5^_;y(W!k3P$u2_F)mcA!$d$&WB(=G7bbWF`bk zoxcKRdwjvqFf+VwIfS>pg z^94w7nj2V9o@13Qh~i0SUrUyq*K+S2=s33M0retty2JEr`{{@bvvzRmqKhbIn% zG;kDs3Es0;%TgBZx88T15`ze?64mwIu}SkiAlLKZK|s%=@{FG|6CK@Lrt3`4T~+Ic z;+T9%&I#!Z8MpG4&uwD*9dwQAl#%~g;g#B&Bz~4T97wO0D_28e&Vf+={XcR6c*f78 zn%tiNplvwA-P^x$A(Y+TUT<(V&`6mDx%yMNK7e$=Q9E|1UBJXq3yJm6D>@TKq_H9eCdzq0S z|EN!nHkZ9kuzgw#g0{A#3ad6SYG%cGuz@#D=NyhW2hW{}dMCx+(e`xeE8TC^KuW+D zx5;C`b9Oz~0{Z`2XHhJ}&IhEaB{MK-e&u#aS{Y_z8}rRAZ4lPV6I7l;HYWU;w3U$$ z4yQ8irWOV2Qw4SDaY&WHxc~`37XA!0oW;&dqMFZL;A=LJ3VW7{xndH40)NV9X*H*s z2%3Xl5-Fp4J%ncoSsY7ttPXKdK?GcTBxGu8bSk^k>YCai^6ctIzo#Em#>~6}K6z_7 z8w;|1NlkmIH&KhvjJ4LZt9DbJSmsdQNA|xh6JHz4a5xgx#?;LY;3cHt&o~>T`>@sXUY273x!XsK9%ca87}h;8cXj&>30xw)ks9$jz!577|tgaHj~nWHp`qFfD^l)Sr-LHQ_kaQ z?2R+y1?oV`=ZPSi8sroSZl=oBo*2->!%_@HQD#vg^&~65e$^tRxc+hl#NnLNSLCGx zrB~2wfx@mq$z%{MF?QFAyXaP#-aN-qY+^LRncC59wo|oEhJA!ybT8AUZtFNk?!b~r z=?mIM(=pb&VY>VubN>9^jO@>0sl_p~bKeRmo?<<+Mq*dP46IcZGIpBQgu_xO*_D6o zC4Ow8#GeLV_y6`Rb5lNmMrzBngqgCYe#Sa0T9^h-=TnGCgrJsNY#U9`=Lff93t_6{ zMOhfw0&Sj{o;uI(^(Qa}KtCQ;8{5GL74&|BHsJn2fv2(+= zF)p7RGrhHN0Uodk8E+XFtZ7ECHFUN_RZ)@~9(gXk3%$qbw5{&v{_SbBe?wVZD*Kse zGFRe+ELL}7|6Mf{CSapG6q=78&c=v(zzGc#qIdLA#o)0w-CnHA=j8hcBnoF^xthZUEa91*cKiM5(3uW59umW(Kbu}z&V5bJl^YT#+dj<6C-#Fx# zROU+2Ldd3&xAm#{@J}kGx?##nCw2t}gqhIsSyg?~d*>GykyJGDk-V^eZC|2PAMK+G zWtIy87=mGt&=w(2=_~fy&vSIccRT+Md$_kahN@Gx2L(m|OB+3beL~*PyoUeY z0o@@$cRszsZN9fDE{vR@E@gR%XM|hPyka+;jF#4CP4t+D>ZYmGdah7q*ISNUGXGL? zoYckO*IGU?v&^H4M0YjCrI-WZf!XrR_KKs5WFPe`@O{d9=SAKFxhP(MUzYiTfLFqg zOUu06AdT#&M|Uqv2l6iDIqHv2|H%<))`ERfDdci3Jhw$&O8WS|6t6r~h&BAO=N*B3 z7^4k6XdY#+^8A9Fs=erMwuCxDGqIrdMKX7&N=mPz!r+_(%y3izjWqfJ4KE$!{6)0b zrk3p2azd7VKzI=!iy7!S9BF5dRoi4pu)hjCE41z~e2}`QcYnp8)MmgUwO7h;q#x_T zA50!~(MQycjwz(S!hcZb^Q^?OsUGTNPkZ7wQz0WYss~!LocrD1pJ~H@kzTG<>A~HB z$wE+Y|6G=rO+-%F9PGzsrP6GhO`H@@Ft5};DF+YOOg-1(zsfJV;OlK6C)A3nZ8Tx; zk`o@b!?BS)&)a{ugIwOG-?pgh?(~ggiM(uqb{|`6^jDY-taT-v)UP=#sld&I5%8c#d*vZq&%)~ON&ituItuN!WNT-*laC{db%dy8| zaS8-E;R|wY7Rtc!Ks}PX+8$A2Yw>Jrr_OrGTRmNhWUB zqHe`xh8?4S2nn9KuQu6n6XOxl*MH= zxCFD@O0~n6>f#e1iq=u(E)1B50kS^FeSS(xhnojL| zYy(FXDInVC_wGUNVY{l!Pk~ASQvTHgvs092d5Gf1_`kq`-=wlvo)Imw>ACVy9cvr5 zNDsJ;>=~tCGoRaOLi_h|?bhaEhT^gHk(sC$5W)JA&I%VhKc`MfgrqwOcsgcW zNwu!6hg-v3nNus);nu)LGMEh@!|@ghXEr__S4T7M_Z=HfM`=}E$UMpO7WF{KVyHEV7-rnnu7Yu&})m<$E2cazd$j} zt7FRjHBbZT(8uY_^K(hTEwMmLV?}zCr}}ZptkR`Oc_OYcqMmxAC)adr zgQ1Ox9BqMe+DV_moYXBAJBCQHmU2ccr6Xz70TNN2=z(Ar@f7p8i0bw-{6d0f zGM3dofS$d?CR!UPv4V(ekI-#cr-0FSbAp^@UDcE>e8y8IQm?u`IGv~ zP>Z&fu0TT^e2%EUdKDV_&C5lH^A=pr{`fgZjB*vrlTB{D7CJMwp-{FTpw#Kd4!$lNv4qdL5!(Bj+9NZ|FkPQ7bB-ft7M%oBd38Q?YvBReGaJ3`&Qp=MsJe2C zQ@(LsG})DVv4eWhDTAVg-Jm$I%re)v=V|QFn{2lMdxXDhyCNln*-m(*PV#cuDOpF+P&OG!LDUQ$bzk$kJLM4Y zRn*@L!U&~B>nL|nDt%?yy0|pxx!R-|^jLucA_?U+u!b*KYsyS&CpJSHmgp<2=Md3h zc3pX~Cw36Z+Tp{mK(9^QGOh`?dfT^PFxDZsLNX)On9op2Z*X0GSvi$)e8+g}`-+RZ z8b7_1hKlOZpHp)#4Z8U}p_r?k9Y8;`9-cg?w(I0+_(LV3k%5U~oWU9K0|s%Racwx* zV$!wRjj&>sVI~a(+8RuhDQPOA?it*#ja^JKc0?o_C2Bdka8>=%1_b4tXZUCaoES6R zyt-_S4P>5lD5WdqL-}k~<$?XlQN?O}A}(u<-Fz1+^~~r+vR-)dbc#rsk-Mrf9PTm~ z+{4vIRvSyplgv~RkvwfDZW7(n5w;nsX3TK~P2X^$TXF8MZN-}MN5Ntd#TR!U@?;QixF4a3y+=_z?bG6 z;Lc(*3d788-eZ`)=tF7^>wh-kz^u>m+2ebh*HWp`@%O_c@6SH}MUU%rm7@y19lZw< zcIg5Do44@4u({Rz1#r&+4Gr+=ZN_MAaAG|n$15t_KKVc)_7@vwRkRH1XO0&CV$GAA zx0Mr+%l2e$OIU41f>tto_O!mC5?QH_-jBuy8!dj_5FJ~bLI@y2vor6zvrp)r4cy1b zXBIngTSDZVOQ=rn|mB`Q@~*z1eJvxh827y}iel*aF^LyPB`|V45Q+-eK&$%!bWCH^NIz zC--n#GfQBhP4r7piYzu5r4$*Hd>&>f@>ajA**0%{AF|jy?0UL3q888ouH}bBP6+`= z7K5wAPVBMh9pxvfpiSR`N${&Z$CVV2Pm9x+-!oNO!QcKqYJKhpTHsK!mY2q^c8PB7 z^$krx?H`U&R`zar$noR$FCg6qU7hfhz0dFyU6Hi~M*Elzu?F{i&$lc*2T z$!cND3y7`qKJ@F!owYE3;vW-CE5$MkvV-5{vJYp+c&g^f2;r`eDHUgm++2I`@B{Rj z=qkI#1c9j^m66^c+y!HpGIj+tG_rcn)k=S@o1kmnpEjBkrBd_@_ErikuMgOQkx;G@ zT8(a4ahES{n3@zcA;xFu@c;!LwMR(Vd;IpJH@C4}{q-;5N93GVNncsbXJT{$6(89ySvzAg~KAj5`89qkb;>tbuVU_R~tRJ9yw z+{fEX^IMca;5v)U=PhFMo=3pjFHL`Gutyu!Oa$o_!MKQvo65O~C2 z=*r95?mTYR=nm3uyib>x)d_}KDW7Q&G~`6+Z-A+@#e*fV!;=SSg4h~L6b0}FQV021 z7PX>9wA6Lp6bdIv3`=yb&85W1WjPis;KOB(06Xhx>CYc0$jOfHx#a!zK+dRYmD@(< zxJ;kW35n|(9l3-L-e?M!LMzHeEwL`lT_C%Bkex%B+z33i8=AT;RBBGoAdi{SL5Vl!3cUe&e%7SZ_{`~b+(t4u<)R&e(o$( z#ukng(%DvqsNCSHs1YSF5(xkB4*N>L<3(1_*`A-jL>f>^>Bd`(h|Sb5!BE^sBFo@| zO!E6hxui>HuBQbYF2zh=zvgsDuzLajRm4Hd?B^N2q)PgIDGSk+_v?GA!Lh}&M^hr3 zZRJzWTwR^U5ml6g6dpc-a(7wb-* zUZmaUJc_;CFY6}aQu<5w$rxDYb+7yAkWOpu%S`Q3F$it2)JF*`)VNd*rsAyT?A8rZ z<&H*a7^k4l{qzsXx6h{GiUClMKXMlKO0BAlcMFa%iuC=WpOcfSo%*_3{&IXl#PH)A zgEJ+~r;IPRkg=W()zj9ZJAcy4rVZZX-&`LqogU=RfLJFU!?mir=@(!USqMfXYfPiS z>bKv;H`DSW=0z5sAJEnAfU$`gCTvx+hWCXU$IXRf3g0htl3Z2+G#QNNMG_%jW1snJ zYiY-{?;18UCJ9P!uiwy^{$a^qG$AL@n|06CJB_?{L#$V9i8nm^k|k;QBi9vd-h3Io z85tj&73-&~5zp9jb6NR(V8J_dTiC`x*JhudPV*>qJ;>;M-DT&F7oN$hTz|r#mY{j} z2^LwO`RQles}1ix&8$vdkY_lwiHk3$3}?}s>^y#Fb}1?AMq7WQkJh*>xeK|Q2YV>Q zlmmV{%zi&-*i-bDbElLx{z@~nHziyUrgaSm>9jZqJn~koVw=F-yd~?TlN4-rn`nQ3 zqan@HY;?t4VzbL(=8XcV?mws3kSN)m#UsjK^?6gI3f|(J1j5b3<4p1{SpHJfNV8Jq z*{TSmuX85mDNGLA_V$KG->V+~?;Fik4l*fK;+!49goRvgUUpb8RO@)*;X3*`arD8V zpO#kv43c+)Ja#vJ)M)F?pmO!MxviHvL_QUCLjC>u{%I z0bA>Zmx|lPvt)kHdRkzC)|WN5XZ!L0k*z8Z+-VniXqY2{%MOq&{eUT? zZZP-vfa7zGNK3L-_OyX}Az%3kQI%)#@Vn&g6luaru9l+gOg7JJ*%tB38v&plzqmcy zz0dB_GjX6F`IN|_OK_mb-ReaVST$@$J>5pyDm#_1$hr6kg>N&>WKf^xXF-+(d2n;I z_3*5jb3NTMyqc{Fma9&R&(tY%ra00p8Vm8Zh?3SO5gpOPF)ByZ^c>Xe8ulsqNhgbR zea5+0&h{RjqDL#zzso*Lb~b2aK<*!rCY=FFjROaeIst9%)=;FyIP*aoCHrFFAg<7r z|2(0Q`?C)By==<8hqHtl+*9|@wl&G&Je!QtUS&-Qit1c(%(PkoXbw022o@;9a0$dw zvtT;gp@OPDug*|j<+8h>>T0?)H68I(3y$oi9E7m0neRSBNh62U2Mju_k*PheHvT7} zJ}Ho>u%2B_e;xk<=*$UWEKDG)4|uWMI`JPxGT%&R)PBTWHZ~661X-x!i9+b^sh8rm}14JP^&Rc^{^yad$pJ$@`T{zQQ?w39O6{`L9nS( zcDJr^yKYP$8gh+r$MMqdZaWDbTlew1r5mx*D2n(b?S=k0^6TPY2&5v{W{Ty30cvvb zXPjygUoAX$c`=#H810w$&TUPk4!1Wv2Xv`(@MEP<{0wPj(-#84?DTYKutCbzErbAp z!QFd%P9lcCS8l$x9u}<*&TZ58f-}Za;hv~s_$g?z|C~pP+Pb|_DJP@&r1Q!*LY@t6 z=V!kGVf}8D&i@XT@#E?H?ICvoTceTaZ76JX9rbpT|Yh+h;#eJ(Fgx#mHNA=X5 zSHoWBDOP+p?F{|gMxmVr7!4%DT`zx}O-N4RkgGy`DM#01)Sv#B)ai=ekPx{xCTS)r z;`H@We%~5R2O_Zq?gf-$HSB?{`ENDGUaQ$3XwwO>w4w0b%Adk`ZjFB z74xNkr0?SIR-x*~7a#9IF=dQ{?&biJG{yf)E`#sY>7ly*n zF3S7tjj!So3LCVh8A6cl(8f4(OqL;pm65$kDic4X#O;zZ~?1eY}M5}kRf5zXq&HFn6n23d!X8+#Uc|Y^$AQ{f8 z?|fx$uMs;Wo$}EwtJ?((bGp{5gz_lIgoSUq9|QEncw&17f%yvD z*{s~zg;%f&6P=1VR0i5kB3mX_rJc#>=sB$-3a*j`AqL`BfUy;pF<+dg7z2KE&Y1@p z(JW6dZfCvnP1sFxq?3oNtKnjQqlXiz$3xHOT>+%RQs$F5lU(k0tp1(zzjc|PL%_ov z2Ui!6k+wNa^n;5!44@El+ufvp_oGK!#m~hW%jk0boy8V&?0aw0&C+wRk*&Iz5)nuL z(|M5Vc_$mR^0&9rp86e1dZm1+_{xY!McN;`Q__4F^MiWwoAqW-djJtMZ`49Cj?@}f zbXmUP;7rD_lNf|E7<9hu&-ft;X@A{fc)x$jFq{-KnNdGq^P%}RE;lT5Y}{j`%rkBP z9mHw$S^ziZN8Q^k(578B5Y-I1@m@V$&xrag_36)spDubi4guvs5jq%GGo+Iw6tDb@ z>^*VdNuw=w_rWByLoCo*)z~F^Ob1I!ZK2Tq7V_GPjJG(qm4*dquQ2`0dd1~ERIIV3 zZvEtU_T90?h)BL&a$9G<+)*HfT4;_Bz3w3mpAynniVsy)4McjyR}C*sD5R&BTnHI9 z#Q>Lr5Jo%67%g%BO{b@7uI|~`5TroL)@*IEIBr-(ZjZz6zxl`FTrBe5nG&;uGD32? z6465)zqu)jIMDV=w`1v%cPFiu57am7IIATW+2yHa`L`&mM?1?1>}*N_G`$>hJXcvyK$v?iRGInZ47Hp9e`3uhWGoGF?s2jT`%5mA41|Gu9;`G?2j1=n@H&g(pm<9UTY zH-v(JHRbvY3 z`s>HBYX%WA9QJG;<1!-_F2=Fz92}}rHsJp2@~mQTdCrtC83l);t^7o*eCp_D3@QIJ zfMZ$z1gqz%fUAE^H!e!+9FILtRHcojya5AAHeE!Y8b!Fx?KCnwSAKx%Ocng zhUeP6Cq7x_MtP^rs7_sw{lF2G_G@z{xCAuwF&fQ};P@b|jXPBVzlCp8!_nSVM6CNqOP{PEiRr$3j@;_hAU8UFBj#>K9J!S4Du)<`Mbt4B5gGb!hm6!_AyUE}>8>tcp$ zm`8oqx;xXM?)M331Ytb!JhLy@78FkEu0H$>GT5L=_IMmskpv{r;p7`f?y$N>hjYE5 zZSh}^thd5vv9AFu-)oElT} z4F>CDUy5YKvzb?7QUj+VDGX3$)XWKv6Hd^YrF}`Z?H!Cdh zdHCjsfCbKQUE&X=Q>J_ZFCA$Vj4#Hh#2&#m{tp;^kF)Na|19an%t#E8x7e7+d=DA2 zJ2=-a2J<@6Iy&vFRDbO@JdG+R5gFm4C$&lK3z7pIp=_u>biAV;!LWF!(~KIsLCSl? z#~;Nx3T|!zNK#EjY>wT!pE%`NsvD*jo*^<{V0nn-%wnasxUXoihKZfd&0RQ4@}OPw z^j2YPS=W8TLso{ad<2b1nyCzr#W8d?X6|< zb>8Z0zHr&o$%rPG6dS#^@Pz}m>ve*VvoE#o8@pC>`QYU@8MU32Q@V(#)zn~PZ~1Si zk1AQK+}6LWBqBZ@A3skpCy2=07+MJQtG8mD%M_N_{6mh3Q4PNm=8|o7wD<9nLBLtW zuEM0MWWZRzW<)<0I<}cJKD+=>#g{&dDchgg#b*)6Glei5^ix4Q&%-rm8n0OMH66Se zR5&i1UBGTnHtIL4fsc09*>fKsWQOE4PHdRXGAM|*X9TPat>OkmRLBuYz~2FK>?Pvi zw*6x{{yJ~p#JTSQq;f}jBc^m$Fl1+Rivzn+x%xG_^U*E549($g>{_DdsNCffgUk4b z%W`O?{wE){a?T#$ZD^LF63ggw;tibjKDlGnQmIFtd#?#Adwx~d#B&e&^eBzf`mR>lY+S3eQ}QCbGUedSezwzuv8C1<_QK3MVlSDi8YOQ7#fIT ztNY8mLZlGBR(GE4OT0tmdHi2%rq7_V4EgV-BA<=n3I zZDhBKWzK9_w~{xTEevG2XOr=^+kR%8)C`YD2LR-?RuGuF`I*2CbSlkia z6Z#Oim^7*NQ*Oi5xZ-a5>D~s07m*MmXD*XhO6HRyRb2ZfagzR7Z_ju8CWFVO`M*jb z(|=BThzgfR&9H5^4JvQ=Kk)q1r)VcCcFSw45c8YMIL|tYS#3qnVl)L_UCEgWkCbA?Vj{VI1k^lFYWZZ^solR!p*foBK&1_KI*=i&%TOUIEFntbOlZS z6>zhRt$ts@)TXh0!g#iQ?20rOPbnv+-Us3Wmsk@z*Gpjc%Oz}B3UEUG0`Zk{WT@3^ zD@@*oYGhZhm?FpHKNrb+{}$a2TkVJ@-phSCyj6&t>2Ym9r#Jt8D`@37)$JVl$AIgi zN1J_<>Ro(`kazi>Y=2gQt?5W^Zq1&Qg*qrA&Qu|`?j9ZHvm*%@-vU+y1V|$ z+Agv-0#O~vx}zj$PqMx=d*}G*dREw}c_^l?{!O)(tx-4*`tL=WU)jau&QE{-LkLzs*Vu z=&ZYg6pv_8M^^s7`nkt8o*ut*9NJl3+Q2P$?!@z;m%(bkQ^O;k#Y{5v%$HI5M(*mQ zP2&LbHvda4qCPF^4$Huwf7JyAU3n0ar(g6Rb?j$87&B)6iohmZ(z(*MA22hZ!uPy} zR5Jbb_Ah8has%Cb$CxI*@uDQR}X1mI9PVplh(4)e$= z#CoN1kY188nj1-Sj-6A=D958 zlzGRIeqOVIti$=ahGSrkM+pMm0=}MlR9e6jN_88wh#B++uPl{evn&@tQ*Jv)K z8!wq8o<;g}%hnJj?DK@?!Tjlp(}4V&<~A_&!xU5zFf?n~0Ui6fOYqTTQLH1GYHQ>B zcF0KSR?v6O|Kc0jCD(l2%>Y-)X5}b6@TTz)M3S?Je1+4sDVywUe9}-FVvmYs@KZBF zx8W0>&R_=HTg(ZjqZso-KDDCQq9${U8b#>OeM%^9(mh~()|x6q=3~sOu*MPFNVZkjc4+Xc|KPncSLoiYv;?^8 z{NAP7%uuRM7UTV_eynG)=0vKDFI55O)BQhe-hbg1vX|g-PagT`CntB*&1J;HLz|Te zUv5N^-#dabn2Chag)|0B31Xzu&;KPtkWwU~4^?gUFVUNH7br~or0mfWuGoA*hasCG z1a91X1^UhTmHk)Mi)2ygoGq)8>YyD)u5t#bMUl%jZT81|f=Aj#{@c$u8^z(~@2qqA z%zrc8ux!*!JZK7m(N&nP?Sg-YqxM~H9vz}X5cQf9&(8i7=|DwH-T&9R!vSf^5VV z(uQL?@0MmZZv{ut@2*X^Ijz}#KU4YN&_O>T8se(UBX4S8SaCgNv{~T8kT{3NAyOR1 zp_7RgLYP&CQ2Z_GboW``|B`HguUC1KDCVw}PQG0>R^H5V(9juJSZU46#|Dkm9A2vR zlr1KNHqv5idMSMk5sf{Hxqmb)j$FP2OYZG5X8qq>i5x0uKR(OzQT()ysC0HE`U#y= zF_Y8vyP}o^ASaQY7mHaCe?0WAXC9+AFlQ)awNE*L&J3enviBXIOt;4A1-gO`Is@ii z8}3Ii?}|^)Pp)Jmx~RMgIjhja#|D|>r?MNU+zOVYYwiPEkVdLjXi6ODjq87J^Y1eq z&6&=3^Ii<-adVl4C`9sdS1zXtTv;v=I$J3!nM-d28hW{Kt&Xkum~U!imCTmpx=R>7 zsZJq+F)6P~g&63#+|^pguo46-{D8W;W+5})rvxnr|FJp1Bzmq<2drZ#hQEIKZkL9A zUp0YPGsZ~g)IOH!Fsa_7L;Lz>AQgSKR$q(m6=tys*JA)8c4@ekRDMCmj$b&aRv|csY zV|Gy1ZpL>2aEvE${gnw{@OM zzF)!qbHV8r73`yKaRY4_*^^|@#Dscd3)t3;fDIxU4^B}yq+c%a@DcDDoSRsAos^bn zU>(AIKSwmx4p|_`5K&4m%C7mc4d&`X<9s{{;k=y29<Y{cRB;W3OW;WtUcGgi zD2D8IzG*wHW(jK;v}&?hVt+K_514*wu>IM3a=uY*eH(f79knH9a)kfTLbX>A07>9O zyCOAL*m6TmF>sT8#Hu z`OouEk9~g&F;vpKw$vh1y`8#2A0C>^(u^3xol-q-hxntg&t$SgL{q;^^{X5G68|z{ za+WR<a7ExbqTM6d~GS__c+U)B!q^Q;a02At_ZY%u27Ejgk=GYXVCnX!_VJ2CB4F-O5yXJV zHc;v>m8m19Lt)x)R^`o>3GLOELvD`RCUbL4fwzmz(s3{Kp=N8X4Z){(N0R85QwY3< zEc50X=e}17;S`#yJ6Ql*H@OEf^#(JPNVEBE)L%9Dz^xI+JuGn_m~@EKxq8H4{q=AR zDVl#PHLTOQupu9B5l8=OoF*mT-Jnr(S^h;##4S^2h zeG}S<4nal}!Kd(UpWxaQ@Zzw?cdHb?kdW)mz`H}k7fFmUi|UbxLCX_822t= z{H$Qp%!w|x`tlDuTPgVCsPeb*#0U3CG!LMqxgxRt_hF&n#+#;k-{3Ye!@Ztt?oe&r zU1QUet_8iwp#-1hmo=sOQ$eXoTH1R`e86-3_nH@z9OQp~8|vx)DlKZEenH&=SH@sD z#UpW=`=TF^V`2OTdV79?b++Y--VH%NPW~ptl80ROhW99@AMv zLn~mttgZywSQA~-aNN*Kf47!6>AO=q%m-&HneZHRk_;UEu3-t@b}x_Hg%^wcojeSV zywVR=C$cIeFB3>Zr!bhr@fYo(|7gYg3zLaDTcIriSN`!Lhvy6hT6LcLuJQB4S>cuk zCo8ZyMXe|+&Ycc1M{XUWO3$O79{oABn|WNw!>V*)nsN8*Gy?b4inNCJKx^90-P2=B zJ*o(oa^?aMt9W0>7J@*P!g4}hU;1>G+p9;ZUtkTIoj#{$*O~kAAv*j+eQ`OJw*_<`l!Wjm8LLSv&6#XfFaE0&?m8sguf&O@E~Lprc?J^R8aW+>9|% z_b`=j%Vr4YKO!Bm)z(nbn@;5i)4T8db>*zcRRHaudqdZ=g`$CCkbfgmxyVHV5ESeB z7W#2j5^~Kc$qLzXwvJR91ero-xA&!|T_FtP{~`dhEXl>r(v8)t6K5k@1TIIF`P5W< zP|oWXgQR|LC5|}YNY+j1>2CU4Cx7Z6o%%<4?ab`}dt)ovR#5+XjF~zYF@ZaWG)tIs z4Bu1;px#rGumS?^cH-YHn<^3^)QFj+@0y(_;8fK`+ zlZ9J;K`x-)>4tHbIXI$~nFhz7ARGf1l+X>&A&2k8P6D5LiE&q%+j5G1pJpqRp3(O& zlIv}wAXur zj|#$_ho$iTqL+JVJzpAVlV5H+uJR?p-54qf?f~()i;a6d)96tP?Y+?Rr!#=GaT3kt zAV*R+d&ZD>s-fn@-mjYHloM7``}1d1#_s)(H0jy-myxPcTc5uqsi9u;+x4|eM-gm^ z=9!G$oI1J6+x+EURjteLU2k5xf?aN?d#vl@ZEUr*nVMCU zXD-ayG&8i)YfwOasw7<@+{nbg3N|!PXu?}YZvW@R zm|;t#JQ9GWV+-rH$)`fNMWe^jcUvCQXgy}-H9o7T{e z2AVasg}(0}$kFm)WnPbG^)UrX3@Brh_H&(XO+G6 zS^luFDrYz$?%kq3y7Je6HoAGucjwsUpTmmkjxCmTv@d`Rv`H2XCrNxs=+p}A1Xyo7 zHV|5sY(SI;)7@+@26cTRy5!dBqgqGf5X|W%%i!uAeM$mUZ>G&Mm-%!M_pK9{-barP zw+}Op)wDbeu$1Urx^JyU%~!NZ!q1RKmzI-8WIO~t|F${*6V5btmbpDF@SEr3Jc?k0 ziS}Jq`qXBrU+et7U?9sve_&X_i>Vp9S)?vaClu%h{t&4lL#7vu{lbrMn8N@I{ohjn zWQxoHnqaye&>`|DSAX^oUjg6j4q|zU$~{y7cKL&0&gJ+nXEPo`|F+p|CCXLM{(Lfb zs<`ux@o}7y4mH))s8wtGGQd$Hz&AYCjq(Lger4Ffh3SK{l`ht8a73WX)hEn9pK}i~ z?PpatZd)@`u(=U|4dJj|x*~w>e3^8!7=NGRqEAYSgkd7`d&9iA$vSzZJH-HtN12%U z!Ed`BU1hX|q?!@4(xk5(Y}8Rd_S=4QKKKP^kDzG^cyCcXne$;wTv?Zxv=Z4yBe3Pg zxMgGmF4ww+vOk<@?MrR7JfwH(t&p#5xyp-b>JDD_YIV{2zI(lP##7zi%nx`vsCaFD zZle1YQG}LC?ep_GUaXX{w(Q7c2M^mVryhd+p*AqTs@}?D$0W#q?+EyU|8APZ(c;W| zexIM&*I{)Y`mUz5CdqW7wNvvh-duNQs9oH*;q%hO*S!~?o8PNoEhYpBZe zu&8Gv?^>X{oqEyGh8klS98aeW&4$Zi?e*Sq2B<9uyjwZ5hPw2IsI(YB3L|t@g7Z|G znMS|N%10<$b>AZ>3h!;4H;=F5?+^LlUG8-Boo0?|KK5lj?H0L$9OWTz`yMJ0vpA+r z-1UxDMwyxi%lP~Lhq{0v}p`m{1e`%hABth&3B|AY{!vkMZFcpj{%d zzHM9x-}&l15zWpps=Msc*tu=g&fnZN>!&0)>Q!0jxGG`y22y<(9|%6Ny_T5tgeoI& z)psqJyf1^D?BqM~6A5u{vzxZs*7;zUZj_ewp7su(nDCwsc(=gkur(onRQi0#zCtu9$pxw5p|wYMkfuM;X{!0Ya4x}Nogc9vE3*s%O(G;bYDV-$=1p;X|1W|4{Z{fCQtSWJEA0jhQQ05q@tE&~H zIjbPeud3gOPCX=J+s?I2Z1~e_eCC#ZV3ZAMCU!pulZeJjA6}}MJKr5 z!KY-a@G<_3wedbP9IzVs-4noP{7u#x>->{X=GYvDKQ=q52xwJr>}DvrhyIlG5duZb z;*zSf4o2X9fdLE5`wp$-tmerLYx+%@8v=jeGTW0Z*K?sVmbgi8T6H!0Hr+f3d%Q66 z=$+potpt|6*~!B3CT6dJ+5Z{_;T9`yEi(VQlQxhya-j3N8MtDvWlt`a$J+|~nujZ1g+T%CycML28g^=&%8oozUm z31XvNvKO?by7lNo;Lvvjn}|E~2gly?HY${E9B;!!fVPwpC+VS^@z|{z_8|C(lmXh1q2eNCz4Y%@#23?9T zFl?V72I8TWspM*_w1(9mHw42b z6(w5b&CBx7)chX(N?t-=kjNbx-kC?G38|Et%L6@;?=jTqDuZ*DH_%smDp+06!8LJ_ z?4XRbXl8=9gye~EI>dMK3?i}x71F2&)!8m#2aatC;p--JIdHnj0 znL4QDL377S@`K)EV>iY0d)?j-EK$UhRldrZcc&w6I1#rl2Cm5|%Ki|Ww_^1VBmThuH9(DekcF-LY zzabJH2)(i3YCJ<{}^s{iC`ZvU}9$= zZr%=&TA$&u?6#b86KikKXpK!9{Ym$Dn^q<#rmmy`I-eoboT>OcwnlzrmglazGDo^4 z_`|o)GW)YG;3t~zuQr@ga17s>4}BaEc4F8UXMeR?v217LEJv=D`LCXCh$Kr3$G*29 zwLy8>g|E>1Vw%yST|^3**2S7o0BM_PNk(XXcF-g(+$54bqq>9Cy}Q$yGJO>nCVCSo z1(zZ#o-VqqG632B8kAH*72=L6TQm0k?-6aCjLI+fUu3s*N@L-5vhquI;9LI5O8tye z`Lb`#Zg7?g+@Yp?{@s+REYb?en^%5#cAh&}TOFh?8s#t~@Rq?1uP+!{VT%%I+1iLG zH~094pgb-Aj6)5l@-_s>Gvt;1%yme!-u{=M=8{<0rl|Bjs^Kv2W@HB8_@SsBF|CKr zLYe+ixQ~H%Z|N{l?DBuP&Po-RJzivy3nBx&SLXgy&* zlH9LW8L?@5Q-3?q>kfRn#HKPK-iPNqJDPQkU1XZiTy5~WoVN}eyPWY@^|Vp=M0qr7 zeJM0GGlBbQZGTpjb6D@>m(nQZ$F*Vr<=%Tsn_nQw7?5W_m#esNlAk&V-0vPVzrYt5 z*X(~ag88W1FQbOR?ZYu-@l!OH4B3O)p$s;zLwB2ubXt-!I*8hrcl z!SKC{C;)4Jwus~s7tz;H@wE2jr03=Ey)YA5gHCjMx=pqW&XGKtfO4_Y=8r4$mhKV6 zjnJ@JLfUHsQ%m!AtUM*e#wRRPz$c}$*_S~O*b-UBOn_S2FC7?}rRJ2^{3g$HaUrqZ z-e?#kvu?|wtDQ@xYHNGU z4HkND)+GW#TFRoSMZW@Ow*~vu&9t>Hj;~98D7%CAKFxqd+MW**dnOOJKPT3gRA_omuvs@A3xzN+L^t5>(XPIs3VoXm=hC*Ee_B6l9%jTFY-c zp6BM~e))FsE#()^s&Q3hvMPF0xth}kasaYjuXgS}t1T7c6#yh#OvW`e^ zKnLG6YI4yJY8zVV%wo-lYgRNO+#p!NJJ4LqS?i8_Mb4S3$zGtoC76$9*sn32(HdqC ztyn^s#lG1|rb?jc<_-9SWg;_C+y!TWpC>aJk8_Fcgy{Tc+=e?bOP-L|HU_k2LU#V_alnb#joPs9PuWlLd(K| zp!jTlJ$yhE<#uFsx|t5_56ppp8c@Q5zt1;!DK+U+-dNeFwNwG7TT^@v;?8~rwum(a zZN*Iv1^I>iEYw2y9zZ&q2GS*`wh4?-d6}bEAc>yChIe)3O|I-V&RfR$DNxzFuqO<* zB0O>`SCXYpDl*|4y7XzKC6ZrC&A zp)*1j!YY%#)GLN_=GFcmrN6345xh~h`p`hKLRKlC>R~_;%m6ccML`S}Cc(CmpUx8MpDfyF(Pr5vU| z(8@D9Jra+{d+D$v`FvZzFiq6ayiEZ3p?v-Js)zxNK=)!9Lq8sAH`rNeMpk?Sm$dr2T{2niAd)F5Lv}<~34VEGd+%yc^z%KC%C|@Kz)jivk}L%izVqk0T(H} z0p$>e;re!oAd+Ks%5>QGkd`Ij=2qBZPUs0@78)e zeQooe)6808W!|jPS#ZzFh%4;IZWw~2UHVFRCF_0vtiQ$KU6*MqVjK6vYDYJ`-Y*n1-gMso_R*#enbU^9!UDe-mbFI`SyXrV z>is%1~jwtl}2z-;4~ z)o~)rJL`M0>IST8g}zZnj*Qo_?b_4YR^ z?ps*7r`-VU%z<=`$ojbu!nIA3yK(1J!&Id1&KyJN4bpfSb-50Cdu`Soa|W7~G9bhm z9_;yS>R3fgoBZUC>$@V@KI-M{B;+4aJ7q+&JjY^Fp+_ohLV5-7(~kZQ$$EubpF5_#b33D){OSX#kJ4{?}}jFydf)RAd_XRnu%J zY1H(2WJFM#v&eZp&5s^U)#Ka_c6)WWzVX0W2rsrX<2!l1Ew{dA#`Pp!B01vc(*0W~ zkq<^QeUHj1u_&It+@h!@xEIelXLv(I!R(MFPr4L>dZHQS%#r>@s1-$_WxeBfOnZek z!S=+mjyDCutN<1~TEQ&J&S$xh=zben^ufXIw9MN5ny2TJuvWLy-H_KN1%Z;^HCJM{ zzisG!2z{wQzJG@~BzSYulWk=lo0RL72FJEvB;XV?@7!Px7C);a0s1LRDGs72OHyC` z%)0)EvU!2q)7Yj%=8}@`KKlhDiT;))h5PCYj{#F2N(ESvg653KIQ$D}PD;laod}6Y z&8q#LGzm_aY5qzHCL#^-h4> zJdeA{A;soJn2nGpgNYDmV3< zuB^pQwdwWnR$0@oBtt|wt8M*g2eVH&JAA~`BYU}l4?-~idp0pxQrG@R#%;XZykrL@ z<$ZE2(2a311m*Sdho z%s3o)#LNoaw!QkjY3mvcgwWgluVh8DOGV-LJe8D8q^&e9J6|rkBNTVl_|c#*KsmW} zyHpf`!p~qVt~zig%epGY3HPfD5|z0PIDR{5Y_^|qX|ofdPtEoy=lL;H z?4cm-os6hc+?ULx`Jw}KW|5J)>JE?9K?+zKchUHMd6mU$-i%^8s~}5YVJ{61;yq6!s=~E_ymijac*) zcj3SUYn!NtLt^#Ed_MU)U@#=Y}Vs*%ZcQPyk42IE~eF$P^|wOE?&f5`kPHOvnkiDdJ4O z6z2;~dP=)%wV=YAxl`@?sd1qCt^1q7-aQ$6J>KAqg_-?@beJ^4`$ogn8zqPDWcQnF z>o(E5UbmUqNpR?%jp64NKKzz->&tlIU)J7t#x?|)dyeM){+$6aeP^KE$xLIJI?C03 z^k#wr*$)#+tqsC{wZq4)uorYYxF_;tV&*1g7TzY0Lypi*hZBvq#ho_vlB7(S1f*plLhtO z{yyJm`7^ObwrW)16tmaQWryZ489E9`^WR%HITcu^D<`|Q7<)JhI^21SDPvwfSnd)n zwtY@)n3>}td^m%%ky!CC|1ETDE)E>2=zH_2=BG(Gk)FXr8;R^+dM?nyOj#3v$vIBi z{&Mo1rZUfABZVYgeFMp$e)?6|$usnZ2l-rDGY`{#D_=X~yS%iNHlx1@xTTq#S6c6J z*0W*kT~By5Q^AH!N%7E=8d$3MBF6}OVSMts9i`VKx*j&3_4If#do=XD0Omla)-)r> z?O@9SG-@<39uA#+KnH1ruT)T61d%<#%pTc;ZIW`gXUE&_i4QE##fL^m%cWRptxJ9R z)wksW+!Qh#x>G$f%v&JIjNUQUM7)%Fbnc<|WKX|xj)mD;Pb4JO`%UE;75&35bb9+G zb6}EpG2isja6eELl*AyFaE~9gt?oL-Cgo{~ZSmVT#y2h0_y_O*+4%v`=bb{QAA{SP z=K19dgrqU-S*ZH>Nzu2?i)Kmt>Mc{=VoL*)QsJbQdudUzhixK;`l#4KxtYB19nO!t zEECtz=o`suFrF!n64OpGa*+~F(roW zEbBsAnWkgIkL#w@j)@wDS3lgO{<>t_4dH2qi+KJ>KjJLR~wgG>L%+A~c%{$%yFNqYRaFs%7ttnC@hYK|($MS3i*aSi-Om3CbT1!qZ4! zPI|lAu&=CH%PiTso`^Ul7Yh2RQA|&0@2zf!U~T=pwlzO#ipMtQkkC$Ka3f$U57F5S zNr)zL;G43_T=(i=wXq|Tx(Nq(u7QnA%5&y z{}0u3mwgOM+CBs9a|dR)xmgnDxZeic>a6FG*F|vfCK_nFn1#g*INKt9^p9;9?i7&;K zQtWrE@S815q)+&;nf|RD&%;k(6vcut2uZ5wqXMnyJVt5j>!7nx(HfPz)v*TaaIR^=L#~>p3?WcjBItB!n`^ddV zk&GF2s0L$x=^_T05i7$gW{$lhVNYT?&oNaQ^1nHwq5) z*WD3pl559kDCin63%~4aHA&!|)>9yq9XJn45SJg1t4uc|*~{i_bWQUwezE4M`f>q{ z8Qxc%L|!4|D+aT3-M?;Brr-nXDlqnedfN^Bfv6si$Osd0v}L2<=7(0n+S@q6(b@-7hq~K& z@31N@^pA?ub2}~UDT+34U6cdEYDQjedsE_SpP0ixZ-Tk38c3fLwSLn|xL^U_#hPeT z)ySacq22p7G(vyMS2sIr@OS-L6a^syUlBKy46Ag+E34R(pSgP~WXUkl`~VokvBJ@% zfpjl%cUU5lzTC38w>ia#$NRbr#aeZ$KhbxCz;bF>DC5 literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner11.png b/docs/images/LogMiner/LogMiner11.png new file mode 100644 index 0000000000000000000000000000000000000000..50ba3ab06ff244e71f3350aa488281015f6a7fe1 GIT binary patch literal 45832 zcmag_2UL^U+6D~ISl}Q6GoT{UEQE-t2qH*HGL8y{lISpsNC{Gm2uOzj2~kiH5u!vy z`cMQJdJ&Kkq{e`NfRxZ7^qPc(gph>)$((oIbLRd2^?hrxSOnP5F8AK|zV7R~o~Y{x zQ>lGN_Wkg~4^r@JSAP59hh0m+&zb)c1HQ{|EX4ls!_gn$S1$dI`Eg~8vo3(&iIeJe42>loE)e8yaurfO`+jv}B@h$M<_y>Gi@j!TeE1adfCKuw1}Ff?z{NU-vjyv0 z_l)Y>ZA>qllwc4wEJ_wLsuP?Bn>hHmuN zsvf;vAqBm}=UAGWq)k?xaQ!v1qP(U*)5U1pOQDJP2g|V~PL8dgUpzjzTT`qn>+n#+ zSMz-#RO=((qoR>E%9d~>P~rF)hU%gK9@cx*hiT8kzL zUCfPLYKIUuzenfn@oHKe+Q{K}w7X<CXF&fDkr_1TnjE3`0q}k-izr<0JvD7{Cf^5n3(c$<;TB5V;!+80k z^BwXX@|t4*UU%=bmd|^4&purT;TyKc*6Gvo4P3fDena0j4`RfWY@ka<=0e$dk%J9F zcI3SAyg_90=#09e!>(W3`qG80W-or5my%`Cmvo6Ma50R8nEl6QO>>7g#6u7)lHWIX z{|)HQ&ZgH+AeTQ6!*iJl4OmQH^0V0MOt zrPlQ4s^)%mX0t{&(m11StD>%^@H-ql(&b6_c;ZU`%}=`-i?poKdJ0+$*LqFyI7U9 z)#;_8R+hd!Ixju3)fI%0{fOeXqKvUjV)`gOo!bJ|yR&LLI(-Nn^`(l$E^M9 z9nCe!da}o%{)#ISl_^`#ybY<5P1NurWYNwBed?&p;C|1A25`xEKAG5}Nxe^RO-}72 zIU8n1uWwd02&ObqTkTVqWJ11C?m4OlFO=Irpn+d!G%>CrZ{#_XjKfG8?_AV7O~FgY|1uS*n7CDGso zQd*hd;O)2+XA0Jz-rsk@^Kmqc@5Y*9G;Dq>hG1t-s2VUf-b6j@Rmb;c@Al{w6VF#k z-^`XGwMVCGc9?dU{_nOzv_6vO-2@i;%xiA<4!^q@m=oSZA|*l4UljPUnV81PJ`)xH z4zlWoG{UaBql}Cw`O#_4>V(_UN@t za@Ko|-!FZrP7MoNb3xv&neMwanBeg%YGM*HrH-n;S(MQA_r@wXe9_B!a#yogK)$VK zRJSPermBgF1(3KCg0sMjsoGvp|iXyn%{cbANGOXgiej=d zPiT#2TPJ(jb#N2#TM5l%n2^y~toIAz7L1i4lWHfQB={5IRk^n>jN7bAVxO=J`xBNM z58dm-4p)XMYu{R7&xGo*I}aX(7xz8Dl|I&xs0WTx7-q0`JYZ6M@Sej)(x3+gRxb*h zjOz9fxsvf{Ta>CJs=wiBk8&`1NXPyuy9C~6acGG>T0(rMO`qgb`)qF@r*;CjQ67_l z6yuKP{?fq9Z>U+sP|O|5s3~@{oeu-c&q%8mkA6G-(%KUxL0ar!v z;obFL08UwjewLpQZdO&PB{hTr4-DFnQdpUm zlPI$Mabm8|4dtwYP`VZZ+De6vD3wsvcNJ?I4Oi8UtyZdW5kWP5V%=!IWfenavj7co zFK3Ik?SPCR(^U2hwem)@^JO_1dltP_o9`d5wAp<%~i@=QFHoaYr464+E8wI5vO!mao$QQiVokoBjyM}dqugHBumK5V%vY`H~-g|MG- z3!RKx&mL-fE)5nIOAI?lkm?7vKUEI@_6bk=NtoTd;UJawOAav< zyu^qDiQx=}a!ytw#XVH$sO39@!`OvANe~ZzvOO&2Ex8$5;&l3=@=`ITRZ}lT>i25C zU2U&Y+=piJ`u08-^R}MG#o8-%mP#l^f+>vGqZpHKKK&&}o4RrZy?cJ2Jcbl=FMfG% z)qf!>+ToMI*F)6s2wdunnYwXMb+dA%+AjjgdT{rx1iQnSsB4r{VU%ZD-x@*XKSbqL zwjh|vX_K-i>nI=g&n4Vf(?%FAa^C++@umFHXNIBNdS7E&rcYEDyrGs#K_v7!2Et=K zPO4>6;zr>qpScnQQa>#4_Xe8sdiB;!qf8jx#QL{%71zN0Zm@!^VdXb4yngONad6yM z1g~#treu&Kks_AWH!Y2xCi~QG*vKU;)EgiLC+`d;|`F0ZH z&RHMQiBJ%ZpS(tU**BAQ45jhg?hJ&R5Sd_EX8a;#^jO83=BXu)ZAb!BSE3x!CPI8QJi4z#P@2mbX}OBkm%sZO{k2g1AKUNirl#=$;Sm z5zeB7vk}`Zh-?rz&NZk6EgSt%{kT$f!nrt#+YvJQ>X{DYXYYw)yu<@0VaTCnP(mV_R0)A{eWltuwyHO^{;hc$9@@{eNd z^GhF>QR6-+I_tzmsw-1J%hUN^to5Wkz3sGK=!d*VUbAyA%K+*fUanl?T1WZvV;SsZ z&ctEH3FV@eYh6^=(-2a&rV5VNbk@IJg#Jd@sx>bZ*dFrc#n@TRs^y!k&MWErWq3uq zr_7=FC9r@()2B73V{r4oj^zG8nKqx0x+cAqMyCwZvTSEtyAx)!X0!fxb;(YX;Xg8b zgFLgcuZ_r$r|{v2KBh{W`P*r~`W`a6H^}lz*`&{dVga{k3iYGPit`+wyN@v zF0l2(qH3hxxu124edya#vTO_OYfN9+vpAg;uTx5-fBJUoTCo@fVur%>I#0Y~gq zpELz}n{g@4{brkId!VbhKn{S!F=-Y$UnTuw_mgOSB+c*%im@S!I+c)sMIXh4G@)sy z2BXn`mr(%bj?}*1v~2iv&pbi6K&T{arv6ZS>*(FrAA%XR=rcX9|7G;)%_#h$TAWye z+zn1O&VOApI__uqpM4gb1koL%YPVbC(y5Gha4U+*k9#AfM`>wAa%9w2Jc=s(1AbhM z;?H{?xf%xv(KjXh;a5YL+p@9?H5JX2`Dhlu~%*{R! zll{62K>Sc!`?4L=>m;kC+ZV=4VRWp&82sfR|V=-pB2@gmYdMc8P(GIG?%; z>W)_{@eaGwiy875vZMY~mZcq4WHjT{2fo01bz-208R0Uf)q7YbVDMqtYY1jBTE3`; zh>fbK){z;?Ob7KBH7-QHLl9c!vKz z&8Tl{ze8;n``c_o)jho%X_qx`#C`3_cr$rTSpH)n>W91FBLto>7u*AXBAh`AU6wTs zllOTCzM7g*O+mgXT6wb{UQK82#Rq#loP+Bpr#v&CQHw|51xav0`7RjezgNz4+)2)F=(P{-!Z zQR8`%;A6APYwU>EQ!ic^Ezq4h?EQsRZg4SWS#RgNU)i3U{dqO{7pCL(2O0%eNq|l$ zw%!CUQPjo9`xUSrEUxdVoj;uOwT||!TdDZNxQ#NE7cZdHHhGJE?@862(t`ypdlmQ# z4}PXP6?7)a1*GS=h~GgCy+`llni18vt%CzMT({h|-2UF$T^Wa~9nYCvPnoXyb=utH zd`8@9%+~o)O0iK(Sk3jgC5P3O>t&z6qkI2I9i9A`l|EaLP+it~;#d+~8(X8vZY_TH zX61Wi9x-<73h`m?O56vD$b3sZe87?P0{Ta)d;#f+l~}Y`L0HPObUyk|uxW5Xm<70I zlA4o_7|MOg%8Y^DcI7i3N1tFdP8Y`HgjCN^f3Kn0ha&YyLLDwT-?I?-nsc3YL2n11 zikNaW3bTGumVvxdvFv5+NIg(rdH7+L_7*^Zv`dg36fY}+lN%@V>lUyM!rMX-i}M!| zvY{uY_5BO9rJ$b`}Cg<|A)kJ zef~K9L-E+`f9lHrjfLMiv4KBQA|iZUhmJX9LO0+4HhZM_0^0h2Shv3>t_j~?@R$_0 zpJnDfX!_*u8vgHK0-rNUvh4p1<$vj}dB^<)mkh+kT~`CUi^tZ0=Q{kO@$ApLHE*Dz zc>iz13SRm0_Up$!)PJu8ko|$rvTR^Ze^G;f4f5~)FN!-mO0^qYJn3E)8nzlZh$&>; z`R~SO35Rb-@yx#eV}}23oD!i;w~9;`E~byCxhV~K3s=M>=eyB=YI5)=SI`Jj`#`i3k||6}o0T0UPDcT?}|H^-o-?Z{32 zx4K-g(`cOf0H$IwS=Z_wE1r5tisT(Xbt||>`yU78e}*Nlef_umlz<3prV$Y>!AvW9 zEH(lDMxMz;nv@aZLmhO}1eCBRsw}a!c!60irs&&`d1T!d=b)?aeJ#f^#zw8&!Is(WNH&`G3M*hX?>FDe$3uek3#p`{x(-lPW!*sek_{H%#d7t;q zXx=zl9J=Oy`M+wTTl@Jlt)KT&Qu5ErZiq^{k)x>fkiikz2NK&f6;8F3dY*{6!*QGB zN#3uNvKmC_aw-gAO+KVR!Sx z-aanXla!Boe^6iZ;lGpl|72Iqqhe>d)9;^DxoaTmLEUjE@QQWqaDSzXgReBFvZ~(G z{1s>rQ{1CtS~WWxuF$tq++#hU-_9i?!Z`0mRsttT%X}sLkKh zej1^rV(l?%y7LtcEgv(jvU&AkY#i!K!;4xRCl;^{&`w%Ytv5v=VdX@R-(M`;VuF11u;OP-2E&)2v8BK)V4~2(=rhF;_TMjY!PA_ zzc0A+vSn;>%N?~n+)Pa8&z#W!xR)KarO;@c5qoA455%{kr?&l9u{$BP`)Et%%E_>U z!TGjh4`|yWPTX5QO(i+p2%y@<{a0G76JOpeiYoiRL-I}o&cg1w3cKEIAhFyv&kp#t zY`JR-ATKitg_-j8_szfCyZS7xy4W7qT6|kU!w)s9R5BU>US`}I(cS|9dk$5Wji9aj zCe(V+5kM)-Oz@Hj4El~s)K&ETVEI&w7&q8252G%8|Er*p0nMLpM7Et8-1Con{I67~ z<@CpdqIJz$Z&Xe8P+mYmP=QCt*3vXX-JLPGflywb=s@3v0qYFYL$BVvk z`{mgH&%bJWoG@n(oa49tKoTm~{4O8}EKU`hrEF7C)=Qp~2= zx4{`BE%gbW%4doaiuQ4C^9Bn<&4Waofy4wH9CN#6;sa z(O?Ok$fn0x3EnXNl*|M@ZdSxFD>kSND+l-kl)4cTL0S(e4oy{C+L%}9+1AN(*DdNY zX6GRx#=IWFcu^}WUrYF}sIaI{On|0^Rav$u&BSd>*s`-Om27!$-P^kN-vH{iwzqWa zqrN4v?EKTR4;tF1qm$NNY|G~2m7WGYeGdTpWyvgAYy?59?}!$|nsf0g{_}1X@1s^DNRxY2y|@r-uc2E^C?h{i zdheJaoOEraXvnVgAD}5kc6Wz%?!|i`E6ogt|9g#9A9r>HEdjvg|NcQV_O{PQ-uQW6 zFZz-?(p!&^{>PC8Ql82Ff-5+n@&Bl6|BI9WVX*P=?Fe6K>Gt8)G{xGX3)1dN|7fhK z^ZyC6W?zhcIy1IB^A9)#4C-OaX8`VGlK$^`05nylmyadIXNOexZdqjtC!aHV9&yN5 z15Vk2u+>XsZ=x7CE2vtk^nRkaqN6Fwl@c^B?Bf>F-?dMsj2h-=7$ zzSk?PQ;m=wOS=*`6x+GlUtQ}TBVADUK#J=#$V-cVT9~Yw%i?Fq2`L_yK_E;;`w}A~ zI*`zT?^$zg6^AYpA2P+#D;9^AV`YqaaI1RX-h3K7FeIYclC%`R7#-GK3`w79wN4b` zjK*30MORd{lHk_LmTWVAc~R=xWc=cv_0#L&_y^ru@%(bmF}tw*6xNd8elVmO&AeG` zf%4tt4{sQlz!=s`hV$<%LcyJZvDV>(bJYt;uN>u2_BJ@?RhW;Xm4ji<+CpO!lH}Oa zc;KU|;BPO0CBLBW>DghMt`MDYH)|xymEKNG=WbSfY$oSJ$P~(3=ElSO0j&DKM&J zVNyM8VDVS^)zfH*LSsdKZm|w#b_IzI9Uh*d^^@wR2_7=rvDCv3lr)GXeIcilmj$^? zRN-6T5Z3&jL3K^x{an5c+3%$m5YS|1?+yI)jqs=G*v2k3edTsHlrh=SicJ%Cm>ncpu?Hio@&oW3P^XQ<^Shr13&~OfP3J7KX z%X<@PzRe(*if1)7trDTobE}QG(vG=j;De1rWx@wM*4-x6#|j1`C6ZZ_ePXNd}R0G|+IPSHBj+PP@O{S)6|v&|s+ zl$8a>nrkBBl! zp5p#hv+w+fBR9?#f+_3$IRBKaNYb&bn4;M-f~Jn>;y!ZD+d-|;g)fVBMzuq#=?jXQ zyqvDNnKuSoadLS7_Jvc#WJvsC`YM;ge)U?K?egBMTT&ILODT|2op{k8>HG-K92@W6 zI)^0{cobw=&Cw(->$?$6C%%80UT>=ICS&6pn0G-*-aXctRGK=9G~V|JsZb7eLqn1@ zgM%B7V`ALxXx=o%39k1>@&w;o1X)8L`b66*J@h3edjv>76R!otZ+ta?4n8${h4dvq zoc&A91?ce3=E8>o`7v_?fZAOc73jB`36m(^?PVD(@;y z$1Y17;_ABsiu_&+=M_P=Ls4(Bu7R(z)~Qh{(w2t(R5y#V*UH$|VlL(2*ZYgKAds=r zun3gd<`wR6jhA}p`cO~9xKwB$(+E?UO;8dJ6EtlWmSq%%T?Qud#2dIyMn}3=B!>^; zVTlZ|UW+%wYc$#2iFeBSQ@b9jvGwuCUo|#cXuw(zs0*hE0|%iGiZr(J17^wf!qU!^ zdsZN$82eSHqe%nB14{l0WM0+uIupO6S2lC_Me=8zFV5}An|A*ZTW zZ5yyUe1f2Mc!M&4ELo+;11b$A<9n#@4oC95hrLNn7|h51gl;R@x$IK0WtP{M3=XWTj}vS z`~;#S7|Lg2c8*xB|GY798wLotBjb&&SGjWyr43`N535?keN3&to}|>MtxvFiS9fon z_{0z%lvAPo7Vdz!6Z{_WO{>n*yYXz>Vw*?+X@&q#XfEcr7kkVW3x(f_|wYoJjr7I>?Q4!T8c&y~m04MCH^gP*^ZJ3~Eu zlFq_&B9&w?dPB#040ZWeBZ-d|Xyj<=7-A&%CDAbT{>vD( z6^DW9+3jVX^o`HwujPHLzSmRuG;~x`? zyJtS5B^5MmB);C|Y6Fs9VFG1T&Og;2jLT4wO(%UVtl<9gIwTmfEF*tNqq!%p3yI5>tDyX6A_C)kVLj5?M zGD6Ee9r)CXKZ!>5v{)tLmj_9Z*|lYcsGmN`fp;b{bChTGqbmfGtl+!$)RI-V1KB~9 zcR%jDYc?4Fy{PzpIn2)0x5-oyI`K$6nJr=d))#8s#-$}-`;2MCw(Qbf$)v&9;+oCu zMD9Vnyx=3jpqSh3ceMrOtWPfFx%r*K;GUCw+xBm52O4Td6M_HfD#iXr0Rlky*x#(L zzV*Aoa?f6`l?R%H)`{$7!yIL+w)o1hug6lL%J2Nt3!WZ&DBVknMHf4_6I432Yy7+6 z&&UzbFxSne1bxgZCS0{Im2{sWX5O`8tFy1t83(`R%A@2vNW6XN{9~B90>f3#rl;Ew zOK*2RXG~Tx22hMt4n&Xv!ChRO%!$P*KIGkNowzE0OUSgqBk{}tVNr^(z#*ziyV{th zrC8puIUef@mJ5 zFHf{ZBfApV#oDQ~0tY*dXz_|xq;8L!JdrX@n9&5|>GwIl;nq#$Ea)PDP{J4q-I^TQ zH*SG%-75oU!zJ=mir9y%_hInK`RqhCrppB^^tV$QYguO_M%~zX=;1u5PbhWjTWi(4 zf#v+H6xw#v(w-`DOL>g?lJdsSC!mZ-2poeFI~+u4{x$_>P)1d{>KR30Iosx9{-nkX+*)3-!H4iL%-i**&QxZ#9yq<5M` zZ3XYF1}e~Utnn=3P#Q&EKxLX1dr0r8)zz7XjOdII7I#THz}E`a+26%|ZSfr4^3(!c zqJ58wJMwl&4{@)&gV7cxI%m_>KFxAM%RH4`9M(@d9dO!)ETh)_bAz3?PL)iZ!Ov&C zE=w93=J(kMZW|yEbPYg#lSUWK0(F`ON!2LtUgvGY zJHyblan(lr_qeV+EycLdq;X0XDM4=M)15WBW@>sz-@WA~2Ww%L{^&~^QmUtaC_a54 zt!&xaFo%SL40q&v;EX%pM)xVg3~}kK;m{%N8j4@hcfywQrt36E4_N~#(hKu%Uy`#H zSj=ie$d0*`UqW9B53js?Sue9GxY*`r(-^z>@ZqyQf}@p};Ih8wEtm-`n8T)|kKW&8 zg2TEL{TWJ{3`+?4{#lLFsnqX=DMjX|)^&L^%H=RcdfF|RHEV>4d89_vA*GYu2d$%Cf98^JWjXo7Ws}O)ju0GpJ*}zw83g(jrxdazcM6p<50FFi4iJ8hZnvOaq=t+3N$c-v!GcQ&O^0IHh#9VWM~ zZ3q4Ob{K28h|DJkxpgcOm#hZg#~TSgC20Ftf9F+IxqZ}1(%LyJ@X46&I8HyCH?`Lp zB&=8VRyWm+N9RWy8&gu*E|2Vg`@~SuTr|vCe-~>YL3yBUq|PBrb&S{JnTr+RJLlMf zXt&-pPn_1q;fabfsb_LRd2M^J104VP-)whUTBzLVvdYk<(QWr>CP#ZmUbB^*(h-{2 zC-h#tAvV&TNC^t=NB?#m-)&;8msO~RZVrNeTNqg(@;uhbZqr?rcz+AkX(KTs%klA? z`p#U&WAys__@8&am#%lT*bfBoI6UT9>sH36uQ}7o1fIiOD-|*?eJ2pKp*s$=?BdN_ zRunwh`M|FK%F@i=Icd1b%q{qL*ZGvHCHc7u*d-Eo3UTns-c~|>`I-HGyShVu9QpOe z_Vmg~Uw<+06Iq(S2SG*v<_$>LLgx5>kb`8zf{n}Ny^>6S2B{tS|2q&QwLwY^Dg|Kt3) zvyaMJg|F)Fg|1s1HytevW5up)eNi#q{mBTYvq9&+*r1h zFzXG)*>w4OiR#t08Bnp%8lk-C+4-(o1Kp3mm>`f~Ql6%2Ub{EHr*&5OuwRq1?f&pm z_krBh+$0&&{d^ND`UtZWFBT(y%h0-l`J*he~2O zr-udJ;vcjODGx$r%4}v0Z_*|w$XEg2hy6Isxu(py?Lo<+d|kN=*~OA)Gtm)UrJYuh zfQHoje^03QB(soEtLAO@Q=L#MH$RT|BpOmaytn&{) z_yM$g>dln;x*MC@)Gge5xg1r|lzPh{MVWdV7S9GL=Bx;s^n9#caGTCvK{#iCF+S~S zsh>!0Lr<6AT}t2FyZ+Q_+fJ+Xs^D~3*r?vd2}R?++@;XuO>G@dsYov^^h8@i7Z2uY zKhE~=%OY%((hG$K#b{U#U&Ebux2d~7WOHr%5dYm6y2+APPoSrs@$&C0a&Ep-jt1=DBvO;^8NWj!NhtCYPGAwQU1Wm~@n&8z^UPZDmU92{# ztgYce(&kjsVBMSy=+sQM6|SNB_8zXZ+Kw&Vl6T)t;x$(00(R$-i7bxw9OxWzj=9u4du-6qQJE&QW1JdmaIb-( z*ic$1A#^JXt5h+;-`Nt`*hiFhM2bYSNK9BoJs@faq_oK(67Y}17||#!W=axOE!e)m zrP6aH1#Zg|iT+4sr|lbrknfGy%4fWUuJa6sTNm{fOmU3$@d1X&L<+6r^%q0=fGdBK za_3!r4^-s59w!c%Z$1*e0syT8YJ8aH2%9YuO7(_az+_`ice8c<(oLj?r1eDkiSc%T zCpcnjhlvbTHD&@r@`!pZR;^4p4XQ^-7XZSE-z2KXHQJme$|tbU#s}H zwVN4V>;Y$jVOx~uAks!^v&e`|l(=&}{(!`&!k2nSf|T?)JUXY+mpQ(O2pT>#u){0f zF%*d|5SyPeC$yNEWHE?bR&;?IiQ z#xO6zr7SOsxe2RLlfzy3oWC6D@@S#Cg-#RyLR>ORnup^@Dn?(v>jFvKE zpbP4%r(PUb5#QMNaXxibLls{aKQ#gPY*nBDSDwc5QQZmU)yFz>A(l0pvw#&_*enuE zJnb5Q38uc>Q5E#4dTy?`E|UQ*;!|^n!)6(=)O>E8-?i>hN-L%{GHtjb-pTmdmj^`1 zY`)@oKXrv3L@*gE19QD9XORe*%JG2i0DkR?)=m+RU#RMIV`XeoOuzaXKJHUO))|BH zM3Kr@)mrrA1Z0uWAX^tAc&;~|r#XE8fvK$mX)~=zuO{vOB2f(|+(?H#mvr{+upRf! zYMw7u|I1T;=sXtG)0RHku5sQN=YQLsekXK+&KDV}qxQ4P*UORpo;nnfXjnouPt*Qv z+fW8d8wj67-`OcM7#?n48o1^cwFaD2D_dAgY((1qQ_C{8aS8>po|TBh+?M{LI*R2( zU+xeYTDdrH1)oPlHxlI_lq>k1?FoGJ4bu6EvRM(s0e5&cYLnTKBYJ5e5-_nhCtKc+ zE3S~XZSXu$K`efG_S1G{Clu9wLnwN;!6l|FYI1qti%XneL7H7P%S7MYUq$@Dw0#qF zo5;toQ1yYX0t)P7{5+GVziP*p^ea})(mHFrDwq_9Rf87wYpI=;NBQ}HtGrp{OkY*A zde!Bq`B~x4^*^)R;%={~BP)6|=KEZplbbe09V%MtG{1&;wx{8AAOY%^D0lD%*Yg;$v4V0xOriDxrXWJ#k2}+T#U6!CnXL#z@#xYM?wt{88n!jW-*V zqN6~k$_gzivzIG+qsV|KHNx|8)lVTd;{fkNdR{!vd|TyFdg8~@R|d0=GDbP0ZQ$A_ z_wH8r&AqS`Bt~tB99N+bu|PyY@NN@R(m_MK$-S?;SQ+^$51G@PKcb2%b?W2G@jZ=} z4)nBz#aI)A)faloeI&k7fMtj;7+2kAmaL@}SW{+H+l#6NBrRf%%bY2fmmuX$sZ`ji zZP>1j<{murSmE2{GZm}XtUp6 z?gf%iSitt4yPLZ`aT&Ex7xEh;oIKQ?aWkh%7Dw2!R~A%&dF?X>$^$_R2Jrd|&UUw# zU-C%Gmt)<>qOXiE=3i)@OHFafpbnyUShEKKzcB^dAmS21!nktiji{Z9UX$k!6~#)? ztD^Ug>Qb3)=eHe{i%4m1g3H}qZb4X=Y}#L1-0#D>j{$(&+_!S$i(u zPh?;SwKiV3Bs1(nlshZ3BLzm)1V8EC{!$B{1{iTpD7X^f6c`9)kL1ma4EIfqkXe8V z;&>{J8Gljd1DXW75wp34ZF3VaUGv+mM|Iut$}Vu{k? zC}N!602pAmGYBVoZ0%D0^^fiIkbFdVbw*A0p+8jWc@g|pZPZ)m)bLQsMf*R`@DJgJJKQ@{TivneEQo2=%T|lgz3u;<2=6?> zyBN1d3X?`h_9?RIX2~YYDVpj#pkSkl=cSTi{MQDZjYi^XTC1LFnBXj zVHtqr1#w zln9+w!SVR zR}t*?mi0Pj@Ptpi9TqU|jHn0baKHQtzGR~1HeD-v87WyB{;2Y4ms5+FZ{1N!$)nqj zr)v%!%f-jQNoIqI?bvRy=zPK_Fa#g^nR;SMtzjN$GkECyq2OrEP;OrowZeITf-Evr zeQqRhjI0zM(Zph%%H5PegDxAMa2g=Gq@Ed9qfQ?gA3CC{^=-O1;UrX|Qg}^``io35 zcDN;s*?Q6tJ8dz2iZtob)bg`}h+9WsO%hrTq>{PO%fTG0Oj->gc<%RFZN~_*`oL}G zW9V~z232y%*oBq?=0S*~2G7V#)9vOjq$eOhkq4dwLR4F_6mSCT_3`H6?EXPcfwt$p8cw{E1Ac`Y4? zoc_Z9CZkiJ&}L}3Tx$6YYE!i^V-4S%3>qCUr5tG3NIsn69{$cOQVNvpnVqA$gm*{t zmUq|Xc5Vx=KzMq(15Yr0RtlaqU-ou9qo;N)YdUbJPue&#qHi*$x6e2|L$#Qc%$y4C zwShl$CZTqwyAQ2$%Ot-VA$~n&V(9G{iA*muQYeQkFB;y;@zdIEoKjt)@?3TOK=K{; z2!rc!h|1kkrEXas9)${iCr)aM%LNS_W@rfKYMfSOgPf=-cdelnKIIPZ4~yDCo^}w= zl&r!Xb_Q@2abSmQDZ06}PESpL^@uG)q6!6iOEbyt(uWH&#>>YUPzb#Xp*})qCIsyZ z-MUV?5+*!mo~+DwD(tuyrq;Zj-q7-@`JnMaH=C`MUd-HQ)~odetaW|u$fH*GH1B=7 zmlkz%KC^w9zgB3eaoHVB8X-D}{7t za;1$?mVmFfpg!tux_jx`t&lf-S8b-l6l$( zk=9C$&b)mYd_z}y>;@95gl~kKHEx+tU&|$dteZZE`CI8JIE5?omFlm2c^0MB8=0;Z z-8^aL-xE^Wwdb`o;Yx+tx!dnC=hqO?FIt^0CQ}H)ie9m&DY7W2wSE}pUThcsZ!dsa zJ=WRbK}1$U#d2s=0+jl7qp;Hs*J_2@ibUnwZ^b%dsv@mZZue`K!&r=n}iKb=YlPla*I)R?X_AX9HO_E3XnPHk!E`$a6k3 zL^GY1nxWn0>dTS})Z3JG%w%Dni;djSJ^Ltx>I*tR{PPmT0`M8uDYmC)q^TNfz49@D6kv)d$JmCS(uR9Xik6 z9eZ}U#Nbuet|7`y*#C6UDq`L5Sjdw(N_`8H>_>}eL=}z4FFtqahoWeqX zpT6g`jHC{#ds(@M9;#T0_Fj9GhOBdC2u#x_lm~O)N}-miqYZ+6Q$@S&9~iKa#T>yh9Jh>SRywCU4|+ zTlvBQ)Iw1qcl(q*H%;f?d)hTeL-gR^vl?lyTl!E=l5JkCHN=xz1C-p@vq${UZK^3P zSM8z$JdOQJnscIdlHDEX_lMlauhsOG4%SAF{0k#t?vfJA%wPY|GAMDZ-rKD5lzCD) z^7P}N_D&N$gDdCoEq|DFTxdS02SLmYIa$6cnMCJbXum0MIN5q}d!aEw zI!e0sNC*FkxL2jM2{;&(tEP&(7~oy$|7usler}p0RlyEl5TQ5kR2XE+`k0HTqqjxw z6rAU~S{}O0Tel)D#NA8*NgAj4HV9pvSs&4=^h7Z9An~@@mv8VN3YRhSyY*+Ms*1{} zhc7&wi@TEU6 zzM1%!ujb!>5tDpDc1Z>!&b)EioN-j`2rUh5jL298+4dH6syeU`pFxoh2dKo;QQC&9 zZVgYrb=c`RZa9grU3LF?;px)eT&+Wm7|Z*T(FsKRnyX_PwR_*-Q~7;pAM@PWw}@E7 z+4J|@Du7!suu+g_ZMQ6l~9Mbc{U!Om_M&) zizk<6tnImu;@KES+tn>O-1A3H9Cklf*5~<>eb>S2xcE3*j>T`az7JX2l1%l9J4!o$ zi`pTrxL0Qm5~iAJcDU>k{@e3nDq-dN3v+)={r``vckyTXkN>|bMG`(yNe)wCNg?EH zO9yh=>LaHdrkr!knc0MpoXW`A}bU3wVmHCayj7~`~bU*JKe!&P3Wnqd0$c-erv*C zRFHTVqvfd3rdgT%Wwg?N6tEkrrT?Nl<^E0uk+kr2fdIyQdis}EdS12`tk@D)c7A=~3FTN>m0RQHn z8hzsxWgw@TRJf`!oU}W;mYfw)9l?AQi7*P5tr6iEip~VSO^f)&Cvml;f1Twu;@YxEI%do@+p>p{gwmNSiS4 zx8tEQBK0qu@0*&LCSN}n`s@h1I+8R1*?C~WDcsVwWhq4&DO-^C^2*V(CuArxmwakF zZ-@KNdl)ax&p8wTE}AdK&pXZc@hQ@T0wG&}lQ_XXh-Yl0BXJy>G!N?RaO7oiZ$#|C z7*O1WDrnPEnowXOBJvhBJ0#LlBz|C}ET0pLXP;S1csLrI6qOkYq6O!4n*T{TCSCc_ z8-=WPN3MKkjpUPdoVZ#OU^&v*iz?Rt$3e<~70D z;>`sW>q-xf^Pe8{JXKIeT(Z!_XEFyr;8#^N8J9I>zZ3*AHm`9$Y3rn-MQI4*XZMZYq? z6WrXx#RK^p?3o{t>V?|&sK7EZQb@)y^ zFZ#-9_X5A4(2bMCvX-vgP33&RlI-l1t>=KQan0?kQTx|Y8)KaMxLvmdOMmCd-3A&p z?EB;ZttX6V#;_AY|2=VE;L*76L*U0*4mXe7>z?`s_k}&eF2p~Z3d1{s?j(HzX^_r+ z)uLX+3O>5>;JlCxhry9)bhJ84ZqN@ME6FnT=+Low)znaiF|}jl+b8eL)Yz=%wT~A+ z#(vJN`temu;X_4&M9rVrr#VH19PwX;(^2}^CvSJ~5l zMMf1~siDM{@~xA+TFgtX^6-0pjI~jy5oK#W;DDAqE=2Meg~nBvtzJi)rpefLw8s;3 zxk881DRyUkH)BL5ql-2y9eL;Q`uB-2V9+Y=bCZQzV!Nulhxa#)^S#gAcl$*0 zppsYlnsN=ZJl5s!vj1|;XFv2lvK9Ris}_usFjR*%rmu3xPJZ%N1bvfzb>oAVu*I;3Y% z`SwZ)Lu7fXxp@70Q3{E4GzwT2e&w%jH7YCY;%% zeE&5U`>g35%%Dj8%os)F5HD;esprb=Vi*nQRRO*(v;V00{T|R7#(!q)OMY}rn1@fe z7Z&+Z>rzE~n7EKVHfO~R1Ov~Fvkc8%eR;08b?pMP-ay7ff5D`HnKggr+aE93il5=c zva~Hes=;}`eWB51*ZM7}^L9VmzJyRbsaS{ONum1FNG_?GZA@3Au!)Su0J;AYJwvte zF^s+YWB2y&aPH|UgJ@AZdB-TTs~a5iqU)f(@lE~V@c9&Rb?!L(WW+zPgM;MDG8$bImuEXW9mhDg zh1D<+v{tdvQ7C$T;KszWB!FqW#nAI8QORgQ$u82rSH-<>H#m7@n2mi1yS7={=lCoJ z9DGL{u5~#4#piy zfYW4t^es%AetI*wgZ^NJm5_7?3pT&K^m^ZR&2_n;a`8{~Zh{)P*Gc`XFFTDxt#MBL zFUyU3lqx2xahpM&^aUV2s zQIxsK)F||{!84eZffm7|lvE(36Im0~=QR+w<&fAy)b(NidP_u-X&}xbqeSJL-P8Qd zk0I+vw)wT^beoc?LkkaqM9VvHh|vi_lnSc2mBC4-KKai2tJ(tWM-fa1HJ4pL_+aYq zr;0?ucomAju`VZiSB{}z_Td8)_NX?~5AQZe(>N~Vn5UA>BBNLH+G8CwpWF|OaMJs& z5u1GYT&XQx*dF~8<&;mC*b|LT#uM{5m+giY8nOV>XMNw6-8D_$$CT7CTAP%x;p|X} zDeu~phV>~Z<7O3z!uX=H^oqZd3M~5VpZ*cDI4pmsG$ASu8+AcL#j;dh$obz(vP2t* z(ThZZ)r~JHjxzW*OEZ9q#=YeB$}l!q!e-5Q#f`KptHK=c3|&2Fq@>X>M$<~jE0kKU zf3CUFB3(bKXz!tkGk+MAr3b|Q;<|FmGi^hRY-pLN5h1M6JJ%bpW$Z$zLsmts0b= z@}Ub<%3WVg=Scf-uaw{_DGG#22vOGJ$u`XdsN2!o#vi#`i$JKdVkx_0tFVJhi*SR2 zvGdZR)L8wt5F_TNR;1+dybt!8=Q8b&v6g9xyhFU)PCn57MbV94k>CARw=oy^=B#o- zlSE6;rOc;N3+_}QWEPX-{B9dBd+CaS75lMzT>ZNzhI~;qcOgS2hJf-R@Y|NajYEYE zl_c#l*esE3``EVyXUyL}&8E*Nf zE!N83W|Z5t-9EIusJwdLOnZ;ZxoO&Yp7MEo=|10FFW}`>3&1|!k(ad|%zjnnIvnj=d*r6OVxH6e?Lx~#? za<}aNv;;36N)>U9)byB5+Fo2g6#ejF;@-pk55IIkIhxnSl-3e_XX!{sqdJay&?X|V z1C?*_OrQHGzQOo}IP!m>Q2;=YI>h^l+3PkK?qn3VTg z0Q+~gjZ`;Y(K2~H46`X5#*ezDy$4Ms;9m1*6tHvVq-t|Q`ldY&Iv&&F3oY7>6scbK z+MuvJbm+;AV*|ioLS#+fb1))Wd#&&?Xh|C^D5}(mK^+6K1f`VA8sidr@f@kjcxl^V zYehbyNK5*wXdNC<6C)H_`W|S94Gt6Yau-8>iS1E!|?f6s#~drc}#0;AX{OWyQKA>EY2n$7h$F8 zgM+(A4-uH3;V1K0; zrki_$Mg%r(_Zf^Ne#7MFyavlw)YxU_i1QeJw^>Z6IO5PlGIFj>I|b!9nh^@54ShEo z3Pzx#6Y3H2mg`IHik;`|XClS$ITe9uz0omcB)gOScxoR3KGBd*Cv!$$afAh{!ITj; zfx3`m=D)|c56e)z?EEZCnf)M3P*jl%;R`5!&y=5c{8Jv(8;?p>esGW(#fBkK`|-%~-v%L3l}sjS!+*&xFACZ1 zq~|ph0iT9~6_y?zej$@yt7<}uE&SJZ?K99y&T~NDi7JBri&Ese5y&dL)T6|FZL^@SkBPPI>?M&bM4j^_J+Kh>Yoknn~ zjkq1vd;XWcYR(i2QI;u{N^0u_I7q_}kW=bhwaN~qVh-vuJUJ=&iYj@A20aN&dAty&bbyv$D(RNJCX7 zkDDekTSOvb(KGOdDi;J%StexIIWZS4PeIkKuoNbfa3XW*cj~B_@=4SfQ~@zd5s&g1 z#dYT8v+-{wnBg+_$K!t_NDipVk_FANN_s$+lNkB6J;^rRO(Mo0E!~oV-gtLA$=WYK zn<`SJ;Onl00&yR@3~qVv9oZ*(K#pQKjhXqyre}w)OB4O$Zv0$fIUvkpQhak-GD34W zXp3}#2%G>hwcIJs6NL|!<)C-=-OF)M;~gN#z^Bxm9lc=~<<$C1*3oA5GCx$vtJ>D{ zz#hA4Bf5aET4<7!tF>6CU-|n&)zL@h3Z-#0jGDf(sbx7_9wn248*Y?ddRVilg?AP6 z59uByF*+shZEx$W#K@8%+!Uo_?Beir+~S6VmSZ`-I<6w;Y12x=#(^kM%&^RFC}$*t zt}w%3*|%FtVelOxP9(M<)qZJFk(FQDM-*4IlDyp#Bz_ybwN&+l)*d@EMn-9|M*EYb z3}$!7p5GkMAhFig&hU`y@HCgR@iT6FC{^S;oKxi?rjztFa0VJju?07wC!uCp49bf0 zHPtQ?8Jk>c)5V#wF{`OX(pcOJ};6l1Lug$mPB`724}R{9xOC_omH-70gj&)yy`z%m5Ca z#ZSV(q6n3tEicsE{tzwM*k#*FM?!KTMTXLAYQAMH57a)wB-XnoAAR{E9zn%FXgbR~ zl?qOod(eS9uUOC%+;EwMl*jN&=+k7%(ejEi;uty95j9mCZc?$EQ!{_ufG!sF?C0WZ zR2EZ=#5Hqk&RBjHo5GqQcl!IYDA$5Rp#h z)DD?>CnF>C5Vw|B5;J_0-75;c^kvyz!|(IcMsbu+c>Z7X!e6y7 zlJb)IQN&MgP{y+Vb`(-PQr$_`w8=3o&E`F@SLnTdU}L{bf*(gZml*s>Pf-|NL22s% z>W-#A_&o~*rJ(w5W;W5JM2x8;D`i!0u?cQE{rF~J?=RT!N;jYz@ZSi0b&vUME2p5} z@Er8D9Nq~*4ZJCSQq{2Gz|*5%xo6unS^`KuMG_eOy8?y#gwkmt`jGL4A+pg~ zZcVV}SX$(8_SXW`NlECsjt8xO+9oI23_h{=%;+5#?RRWPToj`97SZ+JaA3_MF^)A3 zys6^49wh=9YBSB*Mz}ml^~I%|qx!W2s!(96TOiPg4*a8iyQ`jHb{wBpy@zT* zJ~a;5daZYkFvO$}oy8D-#ED}hKtjLP-sbp$-3D#%EGm2>=Ju~C>J830<Hem?$Ia;yr^PnKRUfWB+7>ITPj#5JrEcnl<`>FQa>T9a?i?|Wbt z2h!$)-?*ZvS`*1X4|Oj3?X<3tAJvr#(IzmFj4*WCi?0!32PC=>{DohM^imLmrj;*F zH`OZ2Z+X`iHU9W%Bx`xiLao70djMtrp5e)v)fv^Y?g~C<^xaUtB>tVj{;_Clm)}Lb z$&XZe(stRuy(^P%VIv)ra*^p7GfF=?fl`n@9xn7bIl+J6$E}krdF9urb8^wzFwFq|ey@ zn?!GmRiNPU=@C-D{kc<9Fa7KE=^$tl-&&&N9@(^7=)yMbH$0~``i+NtMbz|^rMiGo zO6@7WeERqa?IB%LAKHtfUTsz1Fs%|)|5r5Eb*e)baO?(4f(mZ|MBBQu5H^=9~^= z^48PUyyKFVK&uHU%jVFh>es9Dw zmC-%;HV1TJVr_4@tV-0dB`Q`3w+J=Fe+uE7e|H>Ixy3&Inywr8Ta=H(8y_rwn&NoN z!{_=MW`FhfuV;J(i*+Bim9HQ;T{)1XiTOx2^k*B*ReVtY@*m!jQJ8tp*wM45-c(lW zlVJ6kQ``-!lP{S8>SqLr$^(sF9-Hpk*^0!!h{{v5M|G9qeQv$&7fW*W*ooSbOn zVzhxCD^f0bsD{*~!2bYV`8TRt-hZ?OtF(CV~XuGZ_gLq zFWG)r4{wLmrtA8MXcpE>H{4B$)}U^C%WK>hOoU{>4HZOU9{YHsZZQw&HC2a%+iRp| z;#rL5=U!{4r=#h5{tOv!>c6d`DqrQin*~R=$#GhkH~O%_x16o|&SsF!p;xnEmtbt7vz$MX@{$1vbDCW2`yf_XrViB1BnX9X%Qy z!@F5b#BE$r1Fc=@K80n855_HOQxIXi_$ixMiKuq)D3VmB`JOowW5NU zj1_*%P7aVn43N!jp6hdWFp3FVbaA&VQX(mhV*Blxq2x@Qv(VHb<=!jL9aC`ES83T% zQk9rR4hU~wDm#83wSuKmM5_w{8i2?t2=! za+8h=`z|M!b*Eo*lo`m4ymmhS({Igj@lRfB8aucHo0KEdX`TH9Lc_itM8EK`L`SH&@9(dPoTWp(XMnJOx1|4R2-!G@oe(dPL;Mp6)ML#r=a0s5`__lSFc5-(&+Y?3fdsGL;qd)Ms&35cwt777 zYT1w&dV4%`R4q#|<-Vl7M*GGQ+sF2?@tVBIj!=cVm7O78KNl?-LnyQn3!q+tU=X}29sOu29q69%s#y||mByb6UAqI#nLC0xUU}iq1g7Z z`OrIFlW&PG<6}C2SYJ;%`rM(;V{S~xxAQRu`0h@Mh4+r8%w3^mx9eqV1>v2>Zmr7R zBg*S5b)=6qO`B{A$eiVO;}uurBwVlb^h*cC{+#qxyK}bktkBp^0lfH!DAgK2`$QoU zKC(&S&BiGr!lKHM!D0$17E^w)#8AENXIr78H=iigJFZ`;Fq zwU9K}X5V{jP4ye`?MBf>knBZm4h?>T{`H*3vW7efJia2>;y1ifhz}y{5KR@jDg%XA zM42FRosO``sAu=&xEJg0^a347r4JMk&JSMP{uyNw`Xr>ILv;{X*(}(*SW$9zK@DwC z{Nr%lY$#+kT*^?0V5Yo!=%&6SH|IPbfVNMYwI|s-FK^}Mh)-)84URokZG7jqdtXk0 zoICjkzkkbTCT|v;GwnsRJuDC4M;G5_^i^--C1zX$qPRl^iiiP%ou2o6QL>|dSie{lbl zsOC-_y+a$i2BlKv#(9E-gG3+jlW`@XO341J)s^dM3+8TIX&FS}i`tV;0|qVo9k)y# zyMHf29vx1|jDw01W{1st{!EJuRhONdIY`Woqi28Bw?IF>yN3PiEb0I{M^CjAyZ(uv z;27W^JoiGTuy!8zF(c>+(@0LgJsqy(#KyIKNoY zqPE~olD_#UqnH0rmAY65$$}T~N7JAMz0G>de?Bs&Z%m?9G^xdB`Orw}+QNMgm6%e9 z$Xm`0@?Nto*Ez){(EG9`{b|b}9p^b4jirev*#d2sgpo9JyR$~`QB}O_z~&F7!MZO2 z$(L4Hyt--$r4Irgki!JDXge3o*B4m*8;eKU_ip6Htvfc!D-Ucl9AQ~93G7CR<=SV$ zA%2D_WEBT#H|ZcQc~0G}(ruROWk?-uV!6`cdRYknw97?aifB_|p>LxCjQ!{4BC4}lAZ&Oy=rNmY?U;0D7aQzg4S!0RK(86Mjwa}MD4%8Jc+htbo&Rs}!n_|H`@ zOyD`gmfG1LW|b;3HfOG0H9%Yq|M6RVwfhX=;awKhyf&snMzR}+?@yh_X4~#Ru1N^x z$ozyenk`NieHnXyeEFBw>#hQ^mXBixdZrkzXL3}rRMNu(qmAd9As{n5_F9>FYgrW? z|Ev+UypMW2Yj?&=B)2{#kgR$3TBB6c{w_=qRMnt)-|U$6Fwl0+OxtE+Z zoOrEVU}dMtReO9>vynLnDfAT3k;hI_q?sJx)DNcxPUt(K%Ehvh*nCay+e-NMk*cz5 zl>8g%b5C~ofH&A)M}?k+TZz2L|>v^4mcZPhekdLnAmALK&S1378jc= z7U2ygCEB*%cemWmj9nw!YVI5`!*|3TJ{ViM-(AT_WnTZ($0Jb5DL5v~d0pmre~3e; z<>NkJ*zHTfKon}nPZ>{;0x62yx*o245 zeYIw@qEU3z5gGY)cg06FKSGYSP-jjexToRa@6>g3$EWiA$I z0dIBVK5 z`xLqhS${`owOt^_7B<@d6K=IUXO&bwIr()h46i)ZERrD|&o=jKSCXGZ)B=~xB*5oG}u|KeECa^+0+_Ux9>G z;-MD17qCE&?C^$N1xi=PqI{6?MmN@;pT^x4(WpaT*B48_gk84!AFLIHHU0tA6kl{efp)n zrNEQ%Y?vqdwI1_mcTH;@OH2~Uj>n3ogAN5&`I0XzBuJ0=18c8A5(fopZGguW**W#P zj#PiU!{JUO@pWILREnuHZqzf@Ux%GyBMe(x{Fzr?O1N(PABl}0Cbm6(E5-7Zi^6v2GQ`=1;X7$8fCqmxIQ?9R?OfqtIotU=o`d~vzSa^* zOXB&q=bW$g*J863Vql|c#AiG`G9yi+4~bbQp_ywi%3>!&dZ-vTuD~Y^?NDf<1F5!X z*Ez0u;xeB?4Q7}Frnpguue)tqZ=}{^Sh?~YDHf@7^XPw|)J&#o0;?>;hRv_2WP)Ik zbiExmm0X}A>x~Hq8gA#^m;V1fFL6vZBE!u>;34b4Hw>n3b|59WA=Z4?v`GDiK^)Xq z0U~ERB%f;0ENjrOkcE2&2p6aSstJ)Ig>0mSmOOUbD$r&@HyA{*J|YUNiCqxr%e3lyo_Gong) zQH~>k*HLbyvT5BMFTKN&Fi_zJ?bF(M*&&qLL@aB7*SNsM zw#pJV`;Xy1WUIyd^u0V?F#a_5Ka$71<^KvBGZ=dnmpa>l_>0~ARsM0Z9OMKu^(Lgc zck*b3sI%*P*J9OplBKi8O1BsFuP3_;gQ;d@=w~qZTn0PLl#me0G}qOA{bfgs38=31>_~HC9i~wo7ALD3UsXK%?GBk?h7>h7Yb}Juy-*zGIg;QyZ)0g{Ke_x>q<5sFa2ZnqWmK*)FaBk zGtVYgDRR7fHnjN6!+29`e~1#!qHU*z@Du>8wkZt2F#;WSgg5V-!1bt73D!re^CoX4 zTcH!!K8tC=Kkv~KrB4*pj1*h&kR@whT)#(=8@&(wJnz1OL_@T^cag?|iFe<*Ns$a6 zVr(kC?2|IhF!R4B#2!%H%5+J9MTWJo@2LlWiBuDx--@^&Hx9!VIpCU~fHFw-`EeP2 z=VnU5=2mRvW4BFDm%J4W5Uqsm5@LGxcPo74tuvaz_u!D(`sKqwk(Cz3t-~?o#+VrC zsZkPN@9GV?&L+yOp@hf^A)aASr5j`23{KH*N**Vb_A6B?fpbNgjjSl#&prNkO#tJKg6&qY@&nM@Pj6dK2tEP^J&0S$RiC(Oem6X+peb z)Sbt(p72s}VP?DXLDm2Jaug$Gx6d;CQZeix+V|b|2(BpmL96Na?x4d1nh=-kBSE`y zPvxvd1Chb=tB;RT{HF{%`mAKCS^8Y%z!@JpDoR(61acO*cm^_>W}I2Czj@&HDS3Ay z1+S%VEe4$$u-)=IUh(7i7|334&bDFEyQDRGrXT~D;=jrJe&w}d;G6BDDRQ^2jD)=N z^}5EiFsVYCobB;F+C8LY-*rz=WhDW=QOnbRe+;scy?epOhJAd-3OD-QwoPOqB6?JB zW#i-&qwvA1(Ssnn&%(3%mx^%UE+CA;zGoulo%Rt6BFymjCbeon(R_X7|2x=FKrKSf_d~-+t-(=N-6$ z7@~4E7eKw?d&<%SfFG7bcUK2Ev{dQZoDGXr^GBmQ(qy*}xg(m-(0F_PCI?(*IJtcF zUK8x4$q18Ap1_S=xX;;W0Bf!y?fek!%<}0(C0m{CzG2c}O=+60v!AV?4xDy1n06j( zH;W3tMwJptrBkL+AxPv`i+6Kd=#jFWd~XM7mDQ-C>CgGnaZ7m%r}*#ZGj>s+Wle3< zQU}sbmN*K*xxF@S5e7M>9XM+Rmk+&nlaa-wD0CXZzN3rXkh=U98D4Zc3GNg(krBDh z_1Z~t@Rs4uVp@Uyll_$=;?-JpqeM14K1`@8=%zE_CHJ2%(7xat_O&78WaUKV#D5|G z(>aDJ9^2-<{bqOF0Rj6ouFnSxu)zEEx_{`f13yJZ4$&<=w?l_2m$QW*wI8?rpc@)G z@rjZ#U6|!r6K6@LlT}kEIfi&Q4zr6ADJ(@Gr?bW*DW=Gf$Wf0gep*+5W+XU-UPo5u z)mqhnvo4i7$ZJ5{O&KqYc^#E^T)&aGw!Sg86-U%kO3`Z1Z$4a$2p!L39>&az>n$|V zc@ex0zI3?52d#JGi%$X60p%)7g1gROnO|l_EIT|y5C%)?p#+LGTqpt;!~##xAbu<<#g`zVq=4Q>A%z0r4UaF4>8JXsk_z&H2b8h$a1=?nV)Ap>$lFf zQF5IU?VuQWWf5imuWc9yW!s{fu8kZ+hnhf@$-_8^3ogO9Zhxx#H1B$zBzkfR?`zxY z8ta%jh?tSHiJ1&unx4}+HS6=!@=*bU5-XqA{eMDldVY2A(Ler`J^ac=14BirXcgU; zj$xl%noby-j+Vp${y%nRXj3!F78eaJT(Y=iwqBr}?Qk=I5f0z*B;T9DmVyDc{46Ys%KiR$^;C(~QB=~p?JfHNqtw`Q}R&ZyRm_MWx~nzkXk zUs;=(f{t-aLwtl*01d{@@_x)(ohqq>@5LP#0ot|oa@>TqH@&*qC9#VA*&KRF@c(MY&h`Q`w;bX79U}s~B zq%18MU6zhg+UW}7U#(Z68~=ekv|1Z<^QLAqqCh?e_W_waO+#XwSA8gd#TqC9q5iVx zrq9rcD_Hy*yl$o{r84Ee0D60X(p+$6^53s1`W*dAWJGpwEoV z+e7TQc_E1n(k!9zhTW}G1oQ#x&SVxMD>tgg_O6-Yf#*li^7g;d#48r3iI zO)y$KdCQWZ{-Sh)D0}u*=dgJWpti+&|3q1=YaMPr!$S{g-T`ciuim{m+^<)AGyXyV zBYsUnZVmVB_7SOj0Z^C9lKx9$FdQ=_45~VGp}v3nmlOis-G{Aalj1)D&V+68T~! zDa*`nKBM9CM5%eevYFqcCpE^l5qPf}jtMt#C2XxMD5kH07SR}6Mq1S6n}fq$@BoX}O+z6;JF$FYqYz zNgiK|9=dlp+)~M_f)>C|J6l$^k*3bbleD6|y8C@dNynT&`*$Yr*k7esmBX(i;&&~8 zZhHr9?#X77!#TdobB@$UlikakLAkpV{GtSVJkq(`BFz=ODwWgzsj8v9@|wTGP0lm> z8tw-jfrymM3}YCP`w)YA`&ZpxAsNFXPtgX+x@VyY_XV4)j!w9*dl%d~A_k@T2#Nz>KAS@xyJ$W;;0M4r>aX;soHe z6sV87JYuaGT_0v_6nS2&p*Ma9H?S6hTn%9}JV*9gVA3A!`X?NURAA{j2zHRLxnM<# zH%b)lkMTnd>8uP2Kk+@#+M=4)U8n@zizt+Yw~<#)d+IET6ZbkQQ=%6NVj31Gf$F@6 z9)vJBbedTkb4@z0jOrVsmAy-8^T&ik{c52(es*?CX_7{xZ@7U>5e|-<6ws%N9VZtv zmoQcM>dQ39_l+ky{kfA6aGL)ttAycnV+q3iG@%F?W^#0|R~cinXMC9#1d`jOPNy2Q zWVe`@etdfvrd$@lLOHEpcc$4}8crkk!_~9TlnvcGAAati!%+toXG86)Kj%f{?#939 zqXiK1`p(7fOkX^x?z8ia8|j|7_mYm#n0Ak0sl;m+{|PBsU*l+J`FUO{U|mnPBE+E} z`qIY>4)${kk8>~sXTi1W*veCjjmT0rA)P7V$J9?^gnQeaMUct6t8d9M_DA;lY=E6# zGgqQ4`!#Oh?FhY>VnYZo-m4WM&Rz)*k0Pu?+^Lf z%aeXH*d{n{LtX=^;?YsCQ~06T{?KO5f0?#%9{ZnP`DX?N?*GH-X2@K2)A7ux2&10p zd)h$9tYec%eRI*rQr#BDcSW-ouAF;vHtTE?3mt)_#xY76yY_Rekno^LS&tFl{2b<^ zpmOI8Dk|VkOUcKeh6(HNK3v4QVDmMLHs+Tmm8{Im#_I)aFP*<{3@{&$WaKxC%ah8V z_;K>o+s9Hd;E~GJ%Oi!c{ze(2D)}Q6H82ZuOKCyV;WW5Ub)M!GnU#*U5cm>p4WCA&yTcDHY zlTv{J$i@+Msr3+khmG-VE9&uP+?4o#wZ!VPpqFN^tfdbWe*b=Nz_5AK=Ek2f#2Drb z<}9j-aT*)@oavI-Y;AV^VQitvT5{tW7qXz~@m-&#LD9y_^VSGVvSevzzO1nXH*rKw z?`2j-9`EHiTxYXc0ri&?Fe`9`tby<;=f$Mj_7aXuNvsdVD}OzT|9aVhmBJ$8^oYE8Co zJ^)J9;-?9zJ6U5&)4Hl|vaFE9Ub^-!$F+P37!ES}F773c4v&q@$7b&8`*Mma?-F|- z8_J)=m$O$CUi@Dz047PUeobgYqiCnhxYk!f;p@88Q6_%P&U+Q6b-Aj6{{2@xY&(6C z&WyZb9Ua^@`8%s@+s`<&_~Yo*uvo2a zS!fddHOW(ocfc+rRGM^ms|6PMM87gDk1cvh$&3J)>_4N2rw!Qe(Bs5Nk2W6A%lejS zq=VnY@&9CJCu;Ocr_{=01*WO(w7k-t%U23VT5OXsQbeK_@5GMvQf$L?akDw$*D*i)$NrqMmXP-v@}BP-)S=fnzV_q}X%)Hm`Z^S*?x6g# z^WeXK%UVJw)T3k+4!i=juf}TA2MJi`!#oE=p6dJ1|J~qhW;2kgB6pZ%wqs%zdU7Xc|aGMTH8s#*~35rji zU5pSnm=e`qG8@Y*;DyxI0+F#bJ8C@n_3Fm^4EVy;=E1Va16kkZKr*XI&p5J#_U^D5 zvF|_0pvz8R4B=l~f)B{P^jBVmH>>3>v|hl)f&PwTc>LUTWqsXREWR_f>GtOv8J1%) zrfYdQGJRCwiB>0Son&Ac0+n6DT@=<36;I*&W!@U=7+ZAYs2}L*cHz;EogsxD?x^?M z7cT?(q2A;ewrmzTa>9~M+-jcdo@lDQ2tv3>=Bw1?cVh|sAST%bxS>^~&?#jr>^Rs@ z%B0@}Yce0QFontChWe6a7(+v8>tcmyBal}I=Vmyg;|N&`SiP;oHodLWmziK&+jFFq zW9^J(VOs7cT4`F&rT1G~gAf_Y3r+csF2V>G%xx|$N67%(@*(ll%=Tt~p97trdiqFg z(z^|3JPT_w#(8^}FRH2?`HM9v`OSv{BQqHNHdm0F^FN>^bmv?cD3V=EJx4>1{m+Lm zI+4iFqieUbPyU}?e+IOQuyQlwm6aI9^uC84kv|R_w+8B+-M64B^Dv$e$w8Jan> zF)Qb<$vEFrJ20!oF1YvohwJ))JmQr@fhpTE)o6_NKDLvSY~Ojm(~ zc@U*3ZJX?gkPj^x_TPbuRJVlkksx8JvlKwQc2WL#pd==asH%^c<%TGi722XPJlab~ z;|Y6i+ni_;Te=?!rv{@G%uovyk?&))mUI9IRQ97F*g=r~@c|tu7JF){CU-ph*WO)v z1PyAzL2+&8!Yrhr-r1$d{zHhy_`-&`V4uWTc)?{rLn1unonQ=2?l`a3L0Q-^M>oHuw6XLq^ zKv4&u(A78Op;}rtki~kL^}H8N<(0rxAb(03I;CtqN5aDQRLf*85GLa_N@SD_OxyiI z6B&Y<{F0@fGoi}IP1<8iK%|J4vkvMGwAA(WZ&6%D& z^W_Y6$UjJI_@-kqIEfl)wQ;M{dmo9FTB2VyK?D_^My2>;%$vE)*y~DE7ONF+I=9<& zvC#eA3Q&h#&bu7*rReCs?86OcUf&s6oFF}ThYeWDRtLu zW9DL?ohQIcHU2dk2Mg92o3M_<6?!56hnQ)@gx)}(N#z5Rb-?a8e&nm90E+^0Aw?K! z*I@B;>(q32^XnC`oI_NTPpo<%4hVxE`G!MZUk4;oxl%wU?Lg3nTe7tRiG&~)7;Ooc zJQ54xnf8V{!R_Jy)5wYdIUk;#dOC%TPxC$2-3j6{m1BGizDJb(&A)xl7yRl+G2qV) zIXd34-}t;W-P{W7bJWX80 zk*dUe;AONa@yIcu0DPMa74uMwg9Asj`LQ=~vBS-00 z1RuNDa}RbW=~fwmvV1UK;%{>o>I%IQs+OkY5XVoqBK3I1xMHp5Rohhm?TYi^QwJ*3 zkSkVT(vOYw4NQxEs#bd`(RyldR|B~5i2Kp4ol%XKt6MoE|4(1v9?xX||4+>MP`4t< zp@R(z<(#vel2dnLv`l%!7#iD7xD1(<+p#8<+^Wr4x$hIb{#y`>_dCKs)$B0 znFtDJB*lKAW{iSncFlds#~GkjvMYR0Wyy(xt5xw$+=KsseydExX3q7Gr>{QBFB%$D zGKKRxr3Y^dZS6L-Mg%^Ohx&>znx^IlzDiC#4j?35^cX)dYEKOK5>v~C7{H57?&Rzt%yfZLm8O%24#6CM<=Z3ib z*`GSxFN;{R$D=)3*X-D>t26pj z;zRb&{t%O>IcKP&*vg=ZJ0>l4b62(oz1iHEOO({)&c}Hz+iBO7j0wzf>TQs%%Xj)# z&Q3XqzCTdD;x>v-u^4TF<}DVk6=1@a`p2Kzl`DqSz6M^Urdo4`=*j=7y!bHaPf9io zt;^s6?E1`;PF^Qe7;)~m3}P0lJw~LRbtfS$afx=3jjiYp!I5-x|HxU}#1hyV2<6YJ zy|;gqO-n77ip`=A(T=iwC4Bv%4V{%bXZkb{rZs3-q~ccWJ&&5Wdi8&RZO#5?kzw6w zF66O^{a@qXg1R#J=%_2i`o1R7eh}uqE3+3;VZ}@mdVFGi4Rvrf6GMQ4kGlkpsd;I= zX+YQiCNCnst?%|IFwR=Su#>?EPU-)s2&9;6x54OBf)APMfiG>+&^No?t&DCq(DABS z8MkQlG|{v`j4O)BBQ^HdbkbEx`k-^5n&^|LE7RDHEw-36)-Jd0qv*v#!DiM$DR2F~ z6;HkJVW_^+O<`t>hOhArExxcV3XcNdEuKH8?6rGAnZC0YE)IcrcJqrY)PG-kG?EGs z+)rO0H!S!1_;>c2?bXm@+%ZZmTl#ZC_G437mKN8mI^>F3Q!WJ%hMd9yvgBdI10Qp` zD~IJ5b_u{Rem7sedGQNop7-MJb2pRpeQ)sie7`{k;2I)W(Y{}pc&W5syxG<`0R1p={X%ilQr{7ikj7WI3_&bF1T>-04fTvLF}I|PT%TuUUo`y$Qijy< zo~_l>sT1_vt=R$K>Zyy}pn{sb@XP7$w?SFKg4`*&J&N6?j*)u;PyHgSqP{7tu?Bhb z+R>Q+{)7FlA23PYJGsa#*bGGG#YwJ;MDD*@-JZ0sG*eKuOCVPr_xHGWDqiEIC0boO zr}X?=FPca7x{h^iy)R&DXV9rGVT-d8BL2(P2IW;!POW4~%`v;=pGSF++IB-YZ{tk8 zAK@+^E5r5m23UKyS$u86*6M#Jun_*xj@U6DFQu6SH%M4m_+n^U;^^U;(3ENR!@*L4 zg_;Kg8G0dh>FkdneYyw5;tG9`J~7RD*FmvMnfq!HQfwQ~4*m9^dr-*`FZgt?tzUxM zR8edQHq&2%%mE;t=X+t-J6-biWMTK=A~Rh2ip;bKgS(LOGZw-2chKY$h~2-?+-WA zCfzY(h(0u&MI7DPSqd~>eAZW#io8vUn|dnaoKtxXn09akDWAAp8fMaXEj)kxj?}eM zDf3MHIe)h&tSWnr>^y{D+gtP8!^NdZxstPhMe0%ZXFuGkZV);MG;e#j8L-sFxk&9` z-p1Gn*4yyn17>Nfe5{q-5~}3Tlf~3!)0hp?prnCp;QoOAjT33EsYIGdJHgW~)nOGj zmX>|-N5x9Kle6ZMO6Fi8_eWZeX~&l$_rK~DlTG3k;nO!jl21Dn3*pti)8ObEVIcxs zXAYU+*YsI=V)5@w&aLl!xL|PoMct)ah<7BMO|PpCbmG2`q2d!TZ`JcQONLJB@Dk82+8#44G5~_dG0%>QLbMIc9Oi5 zNTFtb>pZ-4El-)jBsE;ch16=E7AQIZM|LcQ$?i#iNoJ^g`qb+Hp+iG9b(&WzS1XT( zW;g}uh;3xwcB`qFH6TS@q@2<|lRV95#G%cGMwURfsA4d{2MGq5y_lcNplXn+a&v%< zBz(HbP9PU4a#r*be+W*gQ0&F@qgb?mOd7H=itO?#)r$Yi{L_rE>%k76Y*JD8_%Qej zJ9o2o*zc`8`V0B|^;i|kcC772)+xGs6o(zrX83N#w?|+V{!&y~7|74kJL41_i`Wts zn*iXfBH~7PG@G4aZr*UVB}v#k*D$r_1_qw|=Nx!xhrQ443}9p@xd8maPA3}8%oiJa z6sDaGpY+Ln5AAd}VI+8n3nL>B8a7go*ms)GG-oPd?XAdV|X$o(_6 z;YaW{h;AspY(lQ3q2yNu{z9U5KylZj)>F3HHVF|3eP1Y=lw&*kbdUsmi+;a@#v``5 zGd{7kbnDC*M(plr$4+R?Z#Jn=Q%pS#o)x=Wkm87o#2`}5Sj|6x?VX71*cQ&9heR=U zbx-gT$LW~Qqm!%B801Z*Gg8I9yd?H@Rhe|!fSOFZ7(0JaNs@^B*wn-6%3IF_6z>Fv zIE(idoqrF(`SV{VJJaxDeLs0y_??agEezz%-qXdlXk3tr@*rb$2P>H z;4`=y(3FuO#&2fvd|#FY3%e~LSHi%s;d?HBSAVY>LCYun`_}A|hsLw4$d{!N<-JeSQgBVRyBu`WZha*)$kM7&q`!AjMs%?5Rx%7=8KSJGqAjVr5tjGujH zp|GaVqXvQ}??>)oZKe#SrNh6>FABDR-;`PsGWjdAYTZ|QYEj0|bq70ruEw}HwpMk2 z0f;hwx?jwodG?*(zIYY$n9*K)YVpB&^x>zWi}-6XG}v`(4d1uw$PC8^KzPMs9s-I< zG5~7TL6QU;elv5s?yyriD!2jfx#fwst!YKx9mMS~HQ#E!9nD3fD{Qq8LYRhFh;~&8 zrbF~NCh}nSej7LFv=WQx1kW(Pk{2rXTn*rK6W@1$28I9pAV9rxhQlo8jg;3cfS1D2 z5 zD&DX8A>M_4O+wxwI?%V~+x1A{Lh$x;^FtHYOtG~n_kYz!f@7Rvl;sy3JmBjR^BfEg z7Drchj$@{5uexIs{Y!?qv`g~1$xVpbdtQ#wOgPp~@SW%8FFtoMclN8%3ZV@C+gp!@ z5;0wPGeO4_KcpIUer4(lwP^O|*_11l3VwLG6Y$cs+yU>963xPVucR zeaZz&iC=goV9AOfxm?X`#(vN(l3^cf-k;VrF{wSvxO(ec7YHgmNRm)~203$-N4L}gk zVdWfG;G^g3zuY!CBhE9LcRr7T=&r5L zIh~K$(7Ed@yl@*^_c`XGSC(eWyor`|yCrei*{(Oc5QOP#iem+_HEBLY89h!`#sdPV zTksXq$+Vh%pmSOD{5%M!kR?GKh2SMg@b>L4>vuTXulVK>&ua&S6IB>6#sip1wqzMc z`c~mDKJD>cwIRLKY;62Jo)oF}*~CViq*?y4WoR2Cc`O`CA-hcNhTL{O7QTHQ#p_F* z@XxAKXP+mvF$?M*sk^|}LUY71im++dokp3n2a8vKlu#3Qf|lslwhJMuAzIEXuch!T?r-$5KOdE|8r?qw8^A2g@Lcrq zyi&(?FQ>#wA7pP6drATM3VFq)L3D!8q&Xo=DR0Q2As#V9q<;}OCRjnf0QLWNKRZ7r zz{8;ZWx$@hXSUyEW6*J?@)s+WcD4$rcg$tMs(&<$!}rqSP7y`ls8;BCID z6$3c#9#XBSartcHAus^1VWE0F*XI7Z`E0aoSKVPl)j`OqAl6sriE1XF z5RYbIr_l+4)}TfUx||@bY$e(*Dm7u!diQkz;)WGsptBnP>{pTZi0V+tg9D{dFKt0> zZ%S2>g1F!r;><=yi8~GbVMs#gO89DJjE$?NEI?=P50zn%NykWnL8juz0z-rxB_CE~ zRwxKr9h6C=?lU5=2OoL$F0;BpG`Fd`qCnT(*Wv9tfSe&=<2Xb7efMSB$jQ>KpGHe}qbw;yKT_joj!^PeVg5zs>Uah`Ta9Ugo>t*=5NwF|Rl^ zEjh^pPIPJW>yJ|qf6xL8%SJviF0-r#j}Mf+5uh47iW}_Mj>;D3pAVBnRoH5`cXgv4 zkF&f~3xicZ83|7@frLj_Cioi&$*@XZcM0TB?C`kXUWNN!q}Z}#DO^aRZN8QIF2^is zXdA3n*0RnVtEe$jU)6eu5^>rzWOlh{G=19p3StUyMQP5cppRI%?!EW>NCjQg+a;t; z3SzOf>zQ>>QwYVHkqqcld=RR6an5CW{%_BQk6NOr2R){)u05Is@?WeS1{I)xnoc-) zblwkGTFq=IDueZ3^u}kcd`jJmdLL(=H9dUJX@fR1?C5ewkL%|qH=S%3IukRb{%HNV zmdyCS8Yi|*q$66r?PvSPHkrcQuxxX6nOIaR5NlOpfmz{*rns?LtGKcC6>Ggir&{MT zhFfX21(bnQrgFFAEm&OMe*CMMXKI{lsOPu^(_H00c)bIn7Ft2P5oxQtCoQ<@0}aHN zO&1ry0yZ3~{67wTIC!qNJmsK{SxVG%+O^|Z=nKg;Z*>cMBPb%FGild6Q+dLEx5#xz zs)fLYO{l9RF|#=U02yx-0JG|FKORxdp9pBV2gE*d&CT=X2dGKA(FQgonH zgU-MOMOqr72Yf?*@7J$>Uj2Nuf>+&blQ-<+3|+j+@{=<(h6eyzEuwgMP&uipGAi~z zD+WDjU%sUZP|(lJ6PhZ}pok50XKE7#Jo(Z!@FwO}NV~G`&Zi6FIlLxp`MI#sJl@(b z8}IzP(icMtCU)fFB}k`y_ZtW2;vdMPtX*gWRt>7oy6<+r^fbt2dEi-l+tkwCJQfNj zROMa8`mwc56`|6K_71x~T#^dc{T60`&WjGh0#0RUh0L-FM-O+8T-!AV-SNuacNlFU zy=6nx*^tq^Ob;*5cd*^*WSz&JJD;Y~6V$_t8E8(b^<+nf^aTy#3FqU?2p>E)Vgn!k zT(EW~cnU-WGPffTEQTrCr9~8~PeVcl|9Exv(cpjg40jcj5QR zzHs4cyF<#)C6%$+^sdbXR{W9%~5*Gj?0OW-}=IG z9qtCipdr7LuMQ0$@itSpIb0oRK)44=fke-ah_LCCwb^xIPWUz^i%SZJcg|g{EUnyV zDl1ZNc&FH0Bi^DEQK`)smb9eBbmcnVRk88YU~-JH6>%9VsIrZDu+HvG{{zr&wRXDg z1De`I4E4XN*dyJzxK}T-m=)WOZxc?)87%GQXo|~;fvrFQG6M4;(6@ac`~dJfuiC*YeZ+h=q1$7>;@xL-vA`KfXkB9q1UU zl#{vnG-FHdwZegW+q1;rYTU4V%e=*Dz4RqM&fc&j<;Mfxv3$-|=*Y5i&4P({z{83# zkaJkRU_H)iyNj~;$7@yxr4G7@=0s&BxNQuy8AEgya^pO|b(^BrY)B5{H)5Z^cL%%f z!)YrU62<>An@m9oA;L|-oj=K6XiDGY6Q%tV`Rt0EV3NGf!!t}zPw~41b!Jwu^X~9n zOH;voLzPsjbp(U@BlsCFX2vtTUs~(V<%vtKLjxVI-`QGeIM={aZ~EKOuaq(4DLcL7 zUVi|*tH_XION^=X9UR~oy8H2h`hGz%j9cXJr8Ob|c-v&Ox(D~Z;iAZUCC~w$Nlt7A zN?#CvfcR$Fyd1lte^4BAc2{;zj<5uni$hHAo%Mw+^i2+%TzF=hu^v-2*b{vx#KZ^^-=HN5xqNsvu14hbcRP?mhgZ^>-gX9%&+D%?#~X- zaj8V!E$-c@<5Rm1E&ykqodH>!1ZaZ95ga@N$VEAE-HE4z3j;mK$XV1(nn~xfbnU6H z^Gy4Pw4wufAlbUN`5Jugg$De0@ye2=M4^?Z&vL4$wzI8in3h?`-^kvSAjY|@BKd(g zLRMk3-CERV2?(VQ$+M8fSLw?IO{^nrBhwr_YMH)k?@0)w`Pj?k z%yz!@GE`J8Hcg}NiJt8TA{$sFIhZJE2(>DR3@|APn0#)+trJ%Y~M5p{N{R~WUpt}s_AUgio^Q+3J&As{TCQ! z8kkmf`}I92sw{L|ta6m>t;=p6PNXSKfMcT0*4Vc;ty5c_daSS7TBc3Qq1A$!w%0c| z3{PSM_KiGC`&IRMqFtCAON>Ye!g5pnDj}&kDkFG8d?s29Y}|S7|n2wenlqi^cbrgAG3G z-+ZIP_R%anMY!L=vi7B=Ix2T`p}f8EpChKEJhrtyY1va{WdzO^TY7N)z#F+Je$kKsUqEbDifc4@ zWguv5Ib`plBJ))<3C+>2RK+ii41QG4-r^`BAKe;IfmP1zPci9iWw2fqrFL2GFIEk{ zQTCXTmvjnE66~!QY&WYN3V%cTckQx;^xi@`7C4;V*q85n^l5aLLsoD8vP^B)T4LC- zzgl@9ek>|cYR3r`=XaB~u@# zmV|7^dGhFO*Sc=UGreI{Rz)QY!|Is9V}sWVAH%_PPkmiVwYM3 zoDoWVOnm(B2wyhH$)mxaFP0)23~Co*fa*dHGbjr+ZtLu zVYW!y0eFkUZREWPJDqZHBGwe@+7XGmLcoNI_ zU#G;=_PNG}MQE7|7byEK41EEp90s5fjyU@p}E<>QRv-_19Bf0=Y5O zr?_=881{$nj|FwbKXJR$E==yXau$8KAmR7RCY|j+2)itBq+;s~_N={2LEKW!Ud`UW zi?0LdO{K9r<4P=N()bIH1;zv*=|{mV7npUDhkt^L8A|us-?p(NcI(S#Q<10B?fzc< zruK%YlOUaY?iL6ufV{2EAM4(zjn(Mre+mNJ^p9k@`E zG|S22+B0ju?1$ObG>|s7HGp`#DsXFexPDq*{KqYY5ckNSQBJ$j<_{-Wg0hA&A=pao z$bJq-KfhB>NM|3APvV2GJ;~Byj6Z!VXe4N@j`Xi13_cwdzU&e4 zem%{rAVzmdOo-kH{UJ1+b2s+b=KQjwBwt?>E)sfsqY_b;P=kAS5|x$N=JjPW^XEj^ zLJVxR=4FCaGkGGMP@*ezXN&erfL}~@+vNOsWht!8r$hYG5hC)BR-28=8iPU(j_}2_ zG);H5$wjGh9Wtv3aaVT6>GuFuZB`Cd{9Xn-l801DJq~|1P*@q3KP8&Ctfx6_i|*WZ zpHda}I#J?Dc^*m%xe_Nbj9@A)>QB2VJE;KFXJTwj+h1pYZ3!!Aws#iNmNvXuIPsAm z1{fFzUK57TG$oY@lt+DiYI59+BNM>Cs22ElK_YqBVSnCU)g!AOjpSeK7e0fmjAGMu z!J>7oGBur3z6&wefbQlFAV>O!uzZeZooB5HX_ITQx z?j(_^egjtXjH}oE8KdSxJNFuIov-we?t$f@Luy1#yQjz0%X+n(T*cFgO|_u}?WwkU zYx3gjTVXVheecb)y?F^F(u<2I3@OON2If4F%(Soh{vXEY5?z097PWrve*X8mD=Me& zoZ#SpplO_~kWIPrFmqVy%u~h7=dpt7C2nW5skJY;JvVz~FH!s4zUX^bH!JmL%k4;k zv@sk+|BW<*sHO`4C18i00QR@bUZ){O9vBU5&(oRNYHXPkU)KOCX39ll;PmXWAV?~m z!X<2B)lZ1m3cTdIfD#`R#5A*C?bH7A6F?%lvr04$4qY_VDn6dys?$KXI>0*}lQ2lN z9i-pocm=DJN0!+?D{1i}zsisBH7Ixm_fgDgmDd|6xW6%!;J@H|_;>Brl8{FY#y;+K zss)XbTzas?x>!~MAA8jb-j5-p--0MUU-A|4xlECCdSL!`ajJ7km*d--^+RW7(;?L| zwu#$YD0|7s`d2dc=hs&UqzGJJK*LC64K|G>#aaaWnpx50_5pdF~Ov)j-0-*%@r^GpL4Hgjt|DzzI3zeVS@M4*xV?UnB|f2m11 zdXPPmumTA16w@HZZ5D<{E)SY_$Xg&k8@L~SKf!))Ca^ZWA3bgku00RTSN_Z^=4ak+XuHC>E zH5@|STcQ(Nr;LfM>j8z3c(xH$<_0wQl7N)X%Ek%*K9!Ge%N5Djl$%{hqeSuf+64f z`K-v9Ol*t0Og9F|Y{w+uZ}WN&yeL(fb*6dxO_K=Ibt2%*z78ut=DKeKS*!^x0Zn8> z5+JVJ0SiOc1fv%{aMU?ymJVU}Z6N%E(_WI;)4sLAKiBmBd|Vb(Vs9gmGSDH>UIXK# zTVm4>w3BVHAB&s1&Gm%78Sy=jJ$O-*DR_tl6 zR;YvfbqLh+4o9&uwGN*TdRRuqUXx7P94z`x8D=`9zXrOyA~!Sg(xh_N2cc+AFpA88K$p7+zH2OK-=YnXo99pBjDxy_cN{H!5gev8XB0 zx7)Gs>gTV(EwSl13W33 ztnMSG&skjRPnl{GGURW@^^OP!qRPYxVE%NO>_3n1nvUYXhmJ6fhkK9TWIyi#NGbbc zdzEF#Jpyi9KG8syYW#yiqiu3-qDz%2#oh7Pe#vuBo0`ir{heK7UqALF2;>Q!p>I8J zGb%o{+13BfL!ycDS;{_s@syy#F zFbKC0`8b?{*QoAGQwI^J9XH3?rW0vOa)RmQfpS*9X`8%0fZ3G~I=~rfk*k!?mlzXA zOSiE}&Z+^b&^EUyT@+4S=r4rm&|SpI7I)?#iB%0_nGq_$bBil%Fb2JM-$-QpIsqG!(G|m|hL^1Dou7Ve3)%c(9Tuv@3@D&c(bl>R^Stz(@3sN` zo#yLivZVgo&l3oi_4*~Do^pAVLih9H=*Lz0!Fvj+&G26@hN^Ulo&ybIJzmZ zy_{!WMT@pYsdCDPsLbT@>g{@JHu&SXtX({8g6hm54F=VWNZPq?kcsY3O`ydpEVmI}1z&Tlt9cG81Hm)Kk#X;B;C$9ei5i)o-QA{GJDATt|oR zcbRP!7c@|R2JD7(bY$X6Bf2u{;<6Zcoo-O+h8(Z% zZC2yp)w%m!lX`m+lm8v!|2gt4$uaT6H$e;x-wN(O*%AL9lL>J?*lQCQ4$f}*e^1BU z#X)adOBNfG4G#*h^sH@+3_AS3&N@27rgd`)N&hbj{D15eBeONy__-B0b@KU+eufqW>56U#TMi literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner12.png b/docs/images/LogMiner/LogMiner12.png new file mode 100644 index 0000000000000000000000000000000000000000..8faedb9e0492831a8b2174d5e851c50e7b5113b8 GIT binary patch literal 30107 zcmXuKXIxX;^F0hApj0*V7Hg0uRR~>0K&0tKiFCL~QCjE$LT@S<)QAWKg3?ra@4dG` zK%|F+5a}f$K!5;$?)Q0qFV6X#SNoi^XU)uBGi%;HG12E_6JldvVBj=(^za`B2F8=K z=g12zXZLIFld%j8R~ZZ*-hT#TT%X%73AA|LN+?Uv zd&quZ_&zIDI4FueaP*mZb1AKml}4#?eQqvrY@jpMDPs%v7B8gO<92Q4cNzWDeY$+P zw`v$etkyls)>?I5u-fjVO^B!$B|63_b!d-$zUF9o!ja#`Vl~w{aFxhk)Nk04U>-Dz zMMB_|j}w4q$|`8HPgSph-gn~=iAn6pptWW5&JF^G>o9bfsU;~x#d?h&y)E7{M!U@< zYy>0em-wvM|51>u5N-YF7AtEbkO4L12MtK-Y-1$~J=)V(TsH|ilfCbChj7Co+FNSS z9;|4~o?ApyXe93IL@@md zI)`OnN^f3CN8l)5YDAhQLYgMZ8n?5r-DIW7(xPO+!DHz-KWZ89s4PBM-_BxtT=zwz z!gd=-U&(|moBR~G`&9pQoBI?>Ejc|b(LdeuIt^Hs^()Y)jhk5{l?!s4+foPEmDSu- zGg?pQPmkxhH_SE7RToe3+$VUf*YO`O(dpMF&$ct@3#za1M`2t0Ve$Gw6!$(9M?s-m zFsUv-Q2fkB#Z24=qOjcpyHL5qM0=L~)M5BvZ20MZI1tqvx<404l|7}%vN~c8B16tr z*vf99CA5GKhfnFl+1kfr+M1ue5x^4yFidXJUf=|HN(O4@sS3+3fn=j?R0Tgd-d#`x z!s8m~B03HvKmcXxo#_kAvQ^8jRniGj`-F(6C3m?c8&9Rc`%*;gRSB-hYpOAXVR&g0 ztn&uKyS+#(*OZ+QezFvPf>q2UC+f6>ZcK!2$lJkGTU>9Ny>ciYs!xU-VmzJBY{_<`OaaGUic=%2?mc!AB6T78ofvyJ;c9 zFwVZl{&O;S58I!ckA!1n=K&dYmduK;&MUr(8xE4-&MT$t#4PVcs2r3pG#_m?B+Y2K zXp-WQg%z6l996gK6}j>bjc|QZNr(?b&G{Y8B+?J~AN47GC)4rbZH_?7afS9#MaSZ| zMtcwfcI+!;T^mxX9$~{8APVbUYb~Momxwg=gf#Y)d7peuI`|rGRx_l{>8fWDO~-7} zG1&jCDQa%hMjt=>IV_sqhZb3e-Zt;kCIhs|3_{T4&>fZV9qfOW2b%v_*3+qjnb;4N zDEjc(?LN2jmmc^JgTSDtWTe&#l1t(I&Sa0)-f~U&QB6`aeV~QD%pXqX=k}S(rTsQN z{!LU1pGpax!UB)6KvDDl@Pm}_pvj*Rrd$9qsWm}7`W*Qby?Vsnvdtd<+}r;oahR6KR2)#1PWf+N(oHIq!4mL6M=@V$?J9d- zKYNBSqd&b_vUap~ezB&#suIn^&!z{TiPaB$ zS098voBK2n^Omb|_^5H12?XkIIi6`bUiL2unfuo4+f^(bJbyelABy*&*k1`A*4!8A z|2$?9`pd{B?7yhFeCCGSi1la2Q&dr~I{UT{o&4yJ<}By&{|~P$-f*rc0)0pObSK)e z)P~-xuSKz^=HDlM#cju_h^RBwim>REkOf}23~?0u>D6o-wona|jCfkDs=R%GlxsjA z_sNdSApTjg1F@Mtl&W}-{0Ct>E@>gtpnfwh_;xT+cpe6`Bcz0_q#(c{R3kC)9%`T> zbGIChX(chiQIbXDmo@zF#=B+)YKJ_w~6y?nFo$&RXP=SSt7-{y>e`}=E^MjndaX#P+b5;r@$Yz(iL)G;s-P#IxAuksBM(txS~l z0}a($6-A2=05Z2@nB2DkDz@#fTHAbVMOf6Z5pG>Ww9~&{Cx6R0ka-4)yu}$8l(5xI zpHSw9s9QbPhhy#?t5(~O>_8vU!e@IzXT1dlO#Y}}>KpzTsd)ct!??C!mAruewUT7_ zN>ets{09ZW6mL3A#k+p3xPj|n6(9r1Is@XVT48@%s=#uOYwCsNvH#+f2L7{l$8t)1 zK20Tzs3JyFz0+P&{fl$a`>#kK<=I}g_F;DX9xZ~K7Qu?|z5#2vVtfap+6rF^3|snU z5@4M}C{g-e72iOZ_qU&jjDOEHDK`89@(R(@G39-T`hA8rZJ@d&V*9_v5uC~zUZ+-~ z?K;DCwMAi0R^7@sO6W}5+e}H|YnY_$i((lf-F%`_PBUxU+o1dr#5ve75=~#!QPsbx z`O4=p6pS;_&Ri^&dH*+CZ1;yFd}dKLWNJVR|ETrwUE~eYNRA4K2>epO{#ug zT(sFg^OJ!%1efjl31Gz>3{^(~aD|56c05^x#Fk*_yQYGyq4p9_^xi9Uz$9j3F95vo zZD=v{@GLn;nq%(L@vE}0ypxC`8-M5C;ClhmmLNf#cE_nSW>$EBR%^U) z(+bVSg517%F?D8B7DE zK0kbSFK{=C3oMbfR(y&v4wi&zj~^N*1UB-6j$gB{$!BQr;<9SOP*f$k0PmrNH0%j2 z>6GT^f||wMTSA2`>W3`~XyMwZ*RyN>4wwy_r_o?wkW?5~!9MG2Q>Or4_cGt|#aGwQ zr9@Xqj*E#@j8xxMGrrpTQRZMF-H4>FAiv@h!`N*S_W?PfjH*>iZQfjK-u$p61K-G= z<%3sxd`pKzJ4szGoKmvx=}o-h9`|ujW67B~?vf9v|Ke{rIO-atAb|9e--o^<8o4Z) zU`NG2%j9}q9p6y){~;oo>jin3&6md`WWYtbNv@lAGQxd4(zaZEcf(ycamNueu%~^t z7g76HW>>$q`YK`eRBe_jF&J$EK#LHOlKnt@mg89IugGojfp5<@3(g_>>!AJ%G*Zxg zV;hANY&Uv6w*&BfZE&6|KIONhuZEncNo|wqNFW_{t3vb~W?or2I6!qIdW~5LiMA(J zU z_@*)WpnPhz=1885MQFc(Zh%5ZSVKqTMdhHz)&=6Tf;CJ;-|;C3wwUY8^z{FkesU#3 zB}cKFqqL{GS*1QSujgrH1}=TUJOWj1bt*7{KAu8rf+y~*W8a9hs1y}RdW>5dSdz3h zy_3`%3rV>e2nBt+4)Fnl6%G~EqP;mqypz0FB(bm0*JoQbvbpGV(ez}|u}3oqR(2`V z7e@Jd8x~nIJW@zgEw$a#nMim$_w=4vf$tLDR<2U<>@Jo;cYk^*!lGB1_OZOI16`rhBsJ*2c!*~_%>qX zrm59*28>PwhD+JsvVAf5I*6=eC#e>kLF701x;`eKYaeQZIc(v_I7MAf-a3aMD{m@LuH>II;Nq7zhM2@^Psw2DVfFEVgm z6LOs*ze!OGYU3^fJ*voT&cb(oV&t@%TaLDxSz^-ylcczJ_3~2|jV6u0RI;iX z4{9Brg~eJikV-&A?@uq$@edO(i-J3As()}%)>`b154WG>+g|VD4hLjW9|#$3IXB{& zWx?Pzit0)8_g2|VkTpp4xTPlNXFs<( z0Zs(n*t}K6eN@V=uAC-#%pvRe`j&@(-z$qcW`HQ+wd(|R{Zi!wg;wMM!^NTFY{Cj& zP+%h{Z#gUgVrPZJ^k%CMLT4|~Q6KBHYGnA=OY7oD!gj?vyJ&WhxDK$bSmo`T$! zB+&@tDQVeEj?uCE4{vBdDZk?B3uA;Bh=7*qmowWZB2vxE*G7MJnvTNcmaN& z7iR?tr$qI&jkjVyWRm+kV#EIlDkZ-lJT|TSW#$*V^VZqHsZVhyVfd0MxGiD8@Jqj8 z^ng6bidt>|NnDMb3I@npd>*DRJTUcx1aE3X2MM;!R>!u@6-@Fgw*q(H;Bt74;K{p% zL1terR*%(I6jB&737t8&)x8HTy1fq%^9#OoQ_uxX|%Mt42B0 zHk+IE3I)$o0w1;5`BVOidX{zs`AnNr?|6+S?3;tV6DG8z7o?;S7``Ec78v-!hE+Y~E~+tkoMoXu#+3(6 zBNY3e17^@tGHD_7Lm@7V_57cfT_=-fKa%)vxL*`MOFBkZ=Y=46rGLRHSO)FqB9cUl zQ5@(>?g<<%F{;%?f$Ts=NdX6kj#OT#=(c+W{_Z`E5B z8bsHEH&P)N9((s;zs2ov$qcHr6*(vWUz0;AoRMj*Ai8oPS7X*bcY@yU9CpT}A@%t0 z6$_}(!1wqGzk~c(LZ_qggt%MSHZnj?QCf>YO6G2X5pwE!O7i6myqsIs22?TBeYO;? zR-!woxt3x-`rftI>tmL5t%1%-Op=}hC00sWY%mRx>ElFZ_xz#{T>tFf_4G~&{UXpN zXiaed|0T2X`%m|7qUx(C1B~m%gc832w9+0;eQ&JF1L@zj^@(S{E(FfHapdat!Yt;q zRIaHin_JANZRzUww0@4vev)nl#kJx2u!~c35rhvR_Z2G1okt%<#={QUV(scFsv}Y} zZ2vLz)9s6B)1JQ`u`pxY@|VnwODz_-t8)t~JHJeoKI`g(cRJYhEj3d6xtA)m)+)GU zdk-v`+!d-`f2D#r53m=Uxl2m0P}3;%^@INJVN@T_eeuC&Es0Fe*p^%!!lB<7@7QMe z2Z)w6*-yq2;M~pv6gl>F}M-DrLyhonkZ+eaX<9Mkvs0DC9&AAwm5!e>p(0_nEjVOZ2Q0ZOOd z0}pg0tF&NxYX>RITZCODA!;m_K+hfj(VFgWDO#bDQ^%r*{OzZU;OG{q*4~l;%EXP$?1hZOjGXbN=-i42C=SglowE92@fm${F6-(7lGE5 zf0TkLMwN3d_f1sNqEuFbXJ&8#wSEJ!cF1PRD?3mqb>RkewAgL5t^7EgNB7%0%Nw`XHgS$|g9m2C9LLWPJU=4u1$7?BA}FBNUnq&$9Nt9sKx_P{ z3>^DLADk(X;`B>xy5UkP_DnnLcV{?G^LfO?^exB=5XkGqk0A(h;%-w=eWH#!m#! zW=;i$VpyN_H0%x)z4k0*p9-b%VCN4P573P=9*;W(YWs@Ad(0Fd?LJY-4NV)nCE>>e z{zi%1f>&*|w4U)Y&{ZC>P&rSSBBOUxk?G*zvrse4wvkYAU96p|&`a0Wddm22YvM&) zk&2^rGUHOUnnv$0glA*L%#*ocz(Vdu@4~8`pSYNQ-@P;q0x_i_tXq*aJ^ivxOwMfU zZ&{EX5@X;^6;*K+KPD(@oEndST$Qqn^R@$wi{0FhQ9qKor>EOJqB#;nw&zUVWUj?0 zy9LgC2vdG&fA>l}I-H8^2wcbRSl zWR<0Ht)8M_^-l!VioaG9m;0TG(_g}&+Zvv3l_X>Cxmkmhse(o5bLHc}XKz^49mN3| z#rS}MY*97GoN(2fDI>HUR%7L!DiXBsZM-*^ovlVi&@?m3u!Qbo-CzNpvbfv)XYWFI zT=B0ze`)HkTYDyzyQgj=>@vNF%_AJFddIo$d%UK%tQdQ#^|rDI4j0j|wHJ=!F(=j) z%FB}DJV^`CPds{J%nNe2zZZqP!hF6W^iZVv{@*01=c8!HwV8X{Ckz^pn)*+BE2iOx zWPADB9N?uFCx_PE>Y1(vrA&?TKz_so5UmXBDdfi zvP?lZmY#d^mt4J@R}t~Se?a*X{!AQ+Z7ZyLlz#n=dy04gY42Tojze-1Zxy4*xvtW* zELWB2LnU1Dx9H}9*RxwCvI2|%F+Xr%(pzTS<^sgIOV3szSipOG`uEF6Glu&}vADdw zfBlN|1Tdf^!K&#ZIo18`G{N{&m1Sz*i#z}*1cbS>@!j`$YP5xKnN7dPgtm4)BW%?S z9lXBoQ}~{f@ax7&ut9zy-S~;j1yCZmNmm%GHeMHFc+YvuZe#e91kbCG?D zka__*Q}x?S48V@O2w%505u|@YWy^4jg)HYi3UWt+t=gw>X9bvevL{`IrCrA64d0Q0 z7W# zrrt1Nk-7nI(s&ym`J-5{TU>L+Al~ov?u(!yr8JZp9FTRA^Clk1N^oReyk+1+b#sED zn(Li!3s|b`e;a; zANjhzt8B#!saua_kMR3vnx=ar<57>WRjl7b6tVonhrhqEa%G$*x08%^x(LfYe_yVL z3y?S7jG9Dwj{r4{c}8}SB~&F_KN0kj=2lRAIpsamUMU)KkXu#lTJ|!Y!T6QZByU4* z7~biAw24pAH-(9;?0hahNm%M5*viv6X`bKun(H|xb(0QAQGW+nW0qb97EPZOMUowv z*UlU7dwH**Y@B7rXMQl-4?(N3SD z#Kg>Lv|qyTq}s-y-%r(1JRo%2t6MTQnUOfYR~}cB3Y%=){9oR^l*!ABs=E~^x}jCN()pWFjw0S@!{nHRlTZxm zvbt~wrZy85VV176Si!1#xV_LLIOYHtUsNJgsYf(7x<2)}E25D2;^WQr2mbP6Q`guTw!6QUNzM!WF==vi_zn=I?;7;TPtU(@ z@d-#b;^SF#4!_fW@;pVeQO~i_V?*sdQ~A*1F?4Z zciO8cf^LYha9V)PnYygXSF<*k-yr3KhS~ZO_{A_2;*G!ygc*kC{Ce>sssSDjS8_Hq zw4!5tGkLXb@#8a(JK5sOD%U=G2RQ%Os?2z66Z-p30&mQC%${czL$_qt%dBK$s|45d zXJ4PdD)|Lh^Y5ZI%6dvZdeg#P+!8C<_PbS=c1Ba0sae?~+BWhlcQbu$yJI~3^euOT zxQZ)3_!`?y@~XXde<3>N7(AyOMx*&w)A9zem7H4_T4B;9mwb(9a5qySgOmh^t)ncfUL_87CBTZq)|M{5`oNr59jXrN&v-?j`S0G0XtR^k53( zKL~w|jNGSm%ue3*C_*W|Hv68SxQqSF-VI3i^#k#fpoN#D&S>2pWpwqc*zbdw*kD&j zpQ>9f$=8Q2yLla8)nt^i$a3-&pN}VFki_WPixC0i%Az65+$_)Q-hR0Fsk*OsVQEK( z1Y%wt<#UQP&*njUdZ<4H1ai#i_7~E$ToHb}A4S zYZd%$u(NIIz$S3cAEwfJ#JH_0SI;sWy`>SDRQv(+$#K)Y?K#WDIi|2JMv0Dz24*S; zDxUtpA8mC^X|<4srqKjL+)bR;*#r}-DEOPfc5q7VI= z?JTqMJGsOJ#hU+>3QR*h<&cyH88i2+JPdK6bcIE2G~g8eBNwu#X#h znH%{cw7T71fKAGE!=A6$R^*qD`nBpKfs1GDOGn(VE@?4iz?E*D5LaK)Y8Z~UMezwu ztEm%-eX?{o`Yz*J1+QCj`_-xU*uRGmtB=I9u9kYGgwE2_Vy&iGmnaA3C7FLWJkI^i zxUjMNMqgv$9+Qm!A2F+qswEqyY`N+q)k|twNxKE{S_1lh#fo9Uq9{>S_49tHVvm&v zYVl$jr2_Ycm4o1m!YO8~$p+Es^*kA%hU&VN+b##~{LR~P5-eV$!+8;2q&f3&s!^1x z9U{edS3&y7$;B`vB6{G&m6bwmm#go7s}Rtoe)lEbTBaN%qE@BTNFa6~%GfCdR{GcY zSSS@z>q1roo*yRq?doEg)$mmA83P=Kwh=u0>wnqSmUxJ$TbKySU>A--#iSToUQ>xE z>L&S~i0m}0s{fxBfJdnMa4@YXP4R^1&%1~Y=ku-jYx4=WocC!k_l%3GvT~mxX^IMN zxuq=s%wX(|uAm6fX1Ki@)36smF+NC6JqAyv$!eiGuyb_xRpr#LR*`tq5TUOucMnlF zEH^Y@`t1Xu)9(slsTtXV#n}wq4^mTZa!xw`bs_)t>+>ilyy<*wI2Zc9tv8jL)%M{d zC-+7dc-2q=*7!Bq``$3KBz}4YTQY4J=6~6T4fIrSbZl{Mz+@t~;ExtHO6#B;G?geS zWnAy%lXe0#NXGT%J9al;42C@@Rg^I)Q5*RCJL>Xuq>}RjQ>IDU*Y=&>0p&7nzH{AI z>Imh)b(Ul50~tZzR;a2GdkoF;P3umnH~zg#Jy&zaec~ASuKWc^UG3XM?{gDO6KwF% zr{4W{HoX_$6f-~GeJ}>TiLC#j(y%bc=3%VmIP+axZFS(w?J>S}p~+^x2gK{^cJ#e% zq4yqoR`E{@Q4a4i6jW*-vwAtZ2c-M-+Yt$pfF!+Ztagpi+-O$TWH^%Rg=}SPfji|U zx65SRS%HFZLGT>n6QGET+(B$Kt!?Yk5#ru`6JGwAWlg?Sum&@e?QeNO7AM}v$s+5q z`N%#&OsF4B=|_4s%Q>;$x1kzCW63=wjhY%QTsFDYk6n6m?AFHG{Sb&=M|HUFbq>83 zjrGj3b1~lSFQX;dOQ3)A(FFeKr#sg@6xx|*ZWKw=k})v}UJhzvCJsKVE+XEG??vfR zoOvHa`bRS5x{@I&FXP;#s~ZH0rdd&adUjS>w`3364DCK2C!C7elF3@bo3~q^DP+{- znSEB{L(qP06N;d@$@>p=rp)Pujj-HxsSFtCMlgXV4p7n#HD#+=g$fld^F{6ci%OAJ z4W980P#qi;E)*vtlAuLBgXxVRnFD~SMVzj6kSPq{(jCq1$z4Kg)n0lTu&VEUTpT`@ zyz+1~+CE>bm%WBEDrHMT4##9{yM;Nm_xZLP9|D}aWJKZqnU$?DN%nwto)QjsazC5P zNg_lHxY9pxLqDkRxf-1!-mI}=&lWw`5#J5d+U&v5mhf|{e*I=)1o@ArkZK;I6TZPT zRxOl;d1fj7=OoD(tYIh)fx3Z7(EyxCq80UfNQHw>VP#R8$CRpNnauL_%sVR#>|XtX zr;B&2SGrAu$nDVA6YU1VM$p)P$crGX3<`=gp~fQr4$c!H!@bKBhlVe-Q}T!V)5%pD zvlpdGTo#JSRL`!AS7aE5_Si633mHduHL8m$DLgC9mCEez)v|1qWs=t_&AWMKBjoTx z0;EVc2BlXQ%u{Q{E!$G;>Yp^%x^xq-0na2ViFDdCF&RS6wG75ox$$c>m_fE6W>KAp z{F7KBXU69}%mz3y4+|~(a?6fd9iY7+>bU~_<>p~=@QU(`58+q-Ea0EoQSFPK!!4n4 zFE_G(s(0-Uub-xBA9)U$Zn2q0amv>fMiV93_GIw zc^wJ6hgcm8uJYL2=vQ~MAMDn8g+J393`p6434a>#18ARHA+$C&NUek;qKQ$&ovXr+ zkkiuT`(>E+PS!^AyW&{i3)@PCLQ?h^6G$tT#0)H1?%OvCgGO@~m(r(=!)lQM&U7-@8E|NHoUm#LhwuZ*l98f*}{ zpzGB>OzWH>PMGHJYc&v;5eFB=Vog`vP(>O}irOfaulFnOJ`u7`=KVf#5AmhZLW?Ak zNwgRvO8_&L*=m-fI6)eI{QxxG>bZC^UG!Q{E(s$u5DL&Z@RXprA4}Yhj)4$o^H~($#)6}nJiBl1!yLaF&6QKoM z)r?Azaf*4pTV^ECsFAJBq~nsc|Hr+&6FOX65c_14oKfs$*~FZP%SoLj=!WE(zRPZk zMbnd`CA``bqEj`ZrA+Gv!0`6pGadckvzx>@(YrqjO0I#d5mvatB)f9S1Y9CK#bXu{G3(Gg=2seS_NUingPP!e@5(e<&uie>+I{*z9rI~?LPR|(&u z672+Lm5g8|xZ;|X75aKI|L{d%s9Z8@xcLPYRhM+5MwB14LORUKY3}thJ0sX$)2ejy zvAsZOvw%B1N;Yvxy2L%M?Vx({W`7`ddMwASU00ko8d56{F6qtbja6tK%2!;E91#`S ze3vzLSgvo^Krc{NcW#k1ILkL>peg^V5xV9o#3R*pb*GVC5B{WC-&IKBES z5af$Xo{VstEF=mlZltuoy}C-qsbqV>^^}Ek?k`3O&VLeKEFR*>WBkH|=HeR^;qaXG z4T_-`wTT#m3Ct6%pX_WKiK=8t9mT2dTS9XuA_6i_0DX;mk|C?zg%X$p+=M;*VqQJw z35uNRpVp{gguT1~O5~fa@aCoGqlPta4QLL=xB>KyYUpR;y;D+IhKxB zkcG={&px;-TG%q`|31$PJ9mYYr%|9)^QR6`ZP&cuc(2|Hm?)e-;yah85Y1@^;8J)| zcJ%H2@El!NYU!`gV+bbgeFMVWrk+ng6e)!6_P)o#`@&~Xy|V2n^}>nPPhW}4vx(;| zdVhIxh+2Bq_boV4Cq?%m%VMI+nL=wTR}O4m5(Qen z2Bc>mNm{=bPGtW!d70$eVDZToVsc8FX)<$`79@hsFSCj-(u}W+Y6m4-wF#?t3)t+T zKY-enOJx_uv#pM2AHDAB;2q!XK*2uO#6zr-_D=+lD@e4V1ywG#Ek0GI&Y*D43Dr&d z4dMLWivpKeDjI$YGBEKLYa4mL=G59Zlp-eDTzhrNH*rp2DU4CUkMnlVrlnPRmM&ib zjEN@{0&6C}_tF?4$E%S{L#D^LN4c>3?gb5}QJ@I9qGkk_S<+3BXYb zqfXB?Ukm~|@;c-SaAtcO>|;`QCIkU7zFbu^4HC3eFjPdGJj?^0$saq7TMks8#>p|w zY?-cn3UGH~EeJ6D^OvGOzM<)Q3p1OQhQyUd*83Y-{0k_(#Fe<9NLZN!FA>Ths#8_}$`= z$2?BewJW!=`VzV_XMbt6Ix^!r*Nmyxs`#Pqd}4xMnF7Fk|91)+VR^sehQ|`CYz}u@ z%3i)UF~?EO5m}tN&EiNg;G&teB(j#S*qx;_@5NZBLqlaiNP|sVsmkgHBU*)20BM?p zD#rw`lM4!ZLgo{j`|9$jET%2h>nBUjO`P_MK8TJlFW#>>p~K9vS+z@W6jcNR)V`^t%r`RVXFG&IaAc z>{7SMgzwYF|B#x(Pg=q)`&8s&Fp~Lq}x9qcRi6;(MB)FYGl#UwbEN zqx9+hm}1bfz`Z&JeWt{RE$tln!BOfPSgz_lkB`XV{QDjQ`$zUPa|#FIIk*m6;_b-ObdTQgIk{Ee!TcN(3qCjcBe9p1+rAAQ z^>#}jN55=gSnmF|pJu(yhK>ro45?X~fE4K#Zjj56d*RsqB70}FjeVua`|lh;-OzRV zU(>+G9vzo;&${w7t|&5wy&gB|691PReE4sDr2T3)LE+O#;!5l8a8>z z2Q5_~2pH#hb%L(ATsR@tl2+faKOk|upj)AO?+AUD=;gGGLM;eGomYg$Bq#4JGv6(E zsdrU;EW&{Yf6(Qj$Jp!(XR4oL31TTV(ChB9ubInBB>tsgOD#*yJ^<2Yl0fHO!TWI& ziL*hrjlbL!_`1V0$=O)#(v7G``!SHi9H^lQQdsXN|9L?XI!Ru`s1z zE1$Q8g8RteSpF2f*w}Vtr$+pf5Q#+dqQ*6d#+h{m#3rU=jX>V*ksgqU@uO{pgX_bn z-kTktLn%1o{@TRz*X#&v&S>Y|rhEu!3dBI^=MTR2`@D^)Q1rrXgNAb36>EN_+^U%Z|F<8>SZtr#Mi%WKXoxZIAgtE)QvD@5~tB{ ziH~{p@gJC}2J}DWaTS(Ps+yyGT(DXdOi$4{0S1=- zm$#-kO#m1C%Y*mNeH3NY`0Pz}rhqtCJI~HmB76dR9YMETmb+9~SVRkuI~;I1T>Cyt z#q-2$j`AM`dvD{4!aSUkqrDod@|D+|B)tT!-ha@U@m}aJqV>)Hsc$?G%&s(u4ckSW zcPW}1*VISw*H3>On`qhoq(9TvF~Rlb8P*0suB?Ci4{+{sP0`&31#c)OXHcgS^Qvpv$52L!!Jf3XjI(9 zYQJEj^4?#L>>#`>jB{tma6u+sq-nmS_#?=8W=G^}0Q4Z>*W%3m8%zi58fm>b$k0ef z!oLm#lD*?tgH9Wd`RUoq56@tF@%Uec!S1s9zfbbp@)vEdDB`Et{wL{nrwRsIxuO&; zRl{f8%~zKNOPf{(F2FiaIxbH*JehD@OmI5Ko_PT-&x-|LxHzD$pukN;Nj=_$9JreF1nj4N%&Xzb-Ltu}p|sv6cX|EgUf?Qd_{4(hMTobzS7tmr6Q_A@2>&$_2C;lQ;`clR#cYU&)tu=BBy5wb!h44M#EGwt2U88O&1V1 zo9-o|$?2L`Lumbp;SClmnFiWcxrvPkp*|&XQp1U@re@LU-nK|+$2Fl^)%JGPH%g02 zs|)zTJ03FeWAr>;!}4{TofM>C*#`Gk0tjtabNISMtdp33f(Je;FAQ%|Sa2Fwv5p`dw-FNqb6 zCo${+Uv#~=d$NQ9KMQboDz=bS8V=D|lyYdIEYWo0$=+S6wS&vT1T4U~dI z)Ufm4RUIJUl~o2vaO~9gx+R;_JWBcg2$M?AaD|&@a3t z8Z<%44%x2{1nhUuF`DH_bt{rA-siU@~Hv04C-kX%4Hxsa1SuOvP z%pIVHF;GVpT@q^$$Vlc->yuwGwS-P6fGHikdxaWNAGIBvPd>}ylG&yBe}y>N*GylB zk@{@GgalI>ZN9p+=p^O}hE04jt%Vt|Zb0a~oU~-HO`7tZvj-z9z4^LbLa6LK`cIK>@j zk@w4=Tk>2^em777aAdT7tW{8jhE{AcLEhUfX$9Vq>Ofw;FwyAB3>a#sxYeX`^J@|w zi#O87l?e}JYbh@Ew4oSUxB3)Yw#q;P1auyVpsZeUCsi4gV!!(Hco77x{aO=3+TZ4t z3&uAvo7wsdKbb|+7uv+lq?6NzV}uV zB{(xizsb+U>&xipK%bifLO8ZlnmM|viO4(qIlEEt-TuP@>Y46Qc_LS4HT>I2p=rJt zDo4EyB33kxA1Zs^&{^Uar)WviuPCMCUBbK_Om{DuUQn5G(ZXfE1puKHMj5Sr1cHermmDrk z#!isnm2>WS^CcNO01dUIkxUg#9}YM?5yQCCcSdTTe_fTKTA z!FlJ_QBo11YRh3!XC2Jk9{z-%>;n6-w8RmTbuqJva^uLluccBS3%4ek)e!l! zkUCk=cbOrKp{YgA`1A1i60K=Q(LkvPY^IcLCBbcN(bqKQ$ZAy_>W7K-TYL!gpm!c? zgI)xDv_0^fjSX1btCJGCBg2#w2R|hK)xJ+tJmT#jIFg3-^J>wVbt63J` zH6;Hz^R9^Ip_-W|CkEFn;PYEnR`U~acCafr-6aU?w*dJMvQezT*u5d28s0BfmFAvZyT8$l={X3GFK9EOlv5lvx$i{_6>Bq z%6hN^i{EO&G^}s3n78y7Xo<%{d)1B}X7m!HZYoJrdiA&{D}mc%nNo@l54|=5m1gH6 z^7K!$#Gg$_2-6q&_*AYvm!-klCUDd%TtXoIeh|=S6b;)>S*{&!-}t;b#+^~=vT+qy zv@&U)9q05~ROFsn(qxN$DQ!n|?2@f>SOyBb9Upe5;f;9ve!;SF(zb z%n>%cgAzw}_Zz<6k?NRe(db>V^x1H`e*Jx%D38aooiL5I6v1AeCKt(=&T+?RVVU$Q zT%COtOL8=C>pmOTi4ado_Mqa?d$p+b|K`rTH3`PEhtITsj#ZMDC70wHGM?i)`2)+$ z_*=*SoXO`Pb8dhG3OG!ovgR#9_*rhzbhg9Nf`?bgjDx)5fw<{J$y)ld8$?~xH6|K3 z_Y9(LL*nn2P4zN#RgUv+zlrjIaTPOquuBW@`Z5<3$QWM|;eX;Is*@UJ<~(X&<+TZM>D+Ll64$JNlj+ zEY|Q&h*f*_dhy!on+d9LIDxOPX%>TVV8(txt4%v`gpbHrA3m{guqw z_D{?@U!Kbsn>MnNJ4awIDy&SlkkZlB>%Pg%tP@bPL2xX69rEty;JQ_~5RE0qr#oSg z#c-obuC8uKr<;Rc$bLJUTE!eR^XWOOReLR3Z zftM8qF77)UT?k*v+$P?lLM!o0+@0<-3vPpv^4(y3H4HLWHNEuYeGP4>bgj_%$zpb& z=K>aJXq=Yg0jvPpgY^&%ebMAp#l$d=+_3dGZ}0m%A=fKw?}#?SYXUxv$Mjr2cQ{4Z z{@%p?Wg~j1&#WSM#wAcs zl5gZxTzmgAYX!F7A`G=jnJJN0HR9|hdxKCn=|UOEr~)&Opm7P9;8lzn_;!YK*7#+? zPv$tGv1&!f-`E&M1JSOleZHKfRMatg9FO^cmvc`P=_$tfesSOSwx^&x(&Wv4yAtE$ z-X^u_UOev2@$twDCCI;=_S8f_^OD14a9~zngn;KeqC3Q+!^XWZx@2sAv5GT{(_Iea zw%a1iLu!Zziru)e@idOb7R8{-@lWW&FY|*w>~PG8fAvorBj;%ZX`A!H0q6pV<*=}# zp7yXrdAj2$<4^P)A0}?M%KKeSnX|}P&B+aSgz@^n)JLQh^A8n^>tDc*1FBg`{ zb2@BKMe?!Hgu-D^dR3%>sw&t7ywELKPfZw-cOJjS`VJkRh^&D@xU&;jzZRc!VJCe| zKM}goolX8_f@t|HO!TW)JcJ2EFRXv6mizWY443*urP@U71<}{!-^sFWDHE%Q1^=f7 z5J1fT8`Dv)^UYWjRqKON0}?ZtNb}GQW%V z-hT<`lyZ4=M@~%nTT9u(E!j-`q*=gu8x5mo4POGIm^&=i^%b!G<7R}8VP+U}HZa%t z$@UZA#?QG*bl6x4NVWKJJTlBAsB22U@|RmV>LyV-{Klj+d%u~zXR_a!XMW>Gyg?VO{bk~dN10+cVQ;>dA%|_9 zyHKlQ@O>#Cf9K%WJ`C^k8`HmF%V1I7Ki)l)wKnSoGBy(w`o>@_mWb|kU?&Kdcge=JS0Z++h&v3t?K zIQ3FAV|eA3F%r1{NJAiec&Z(@LMgt!Z&H`*f?cl;)y{Nku6DdQ@vcSjhoJu0+7&Y? zrK&PZK~<_JqWk9#X2iYT#-4YO+&V1ovy*<^!Bth_HgVUuC2jJY`<=~i@#P$~?xw0M zM=IT214%!SnW3Bt189uZkOLx{q_jVo&8WZis`g9KY2rgO#mBI!EVB#y3xph7Qf^aM znNIMR8QYweF~PIMcOhoYMV@0I7A)uW(_^^g~K&b9QD+U;bA$Am|q z=$uo2o>oZem@~*jx&^z@*vvzgTbjkH(_3g)?9OT+H2J0epMM&&m-8fUJw0vw6wzEPp zxNv*(m9*O?-;ke7gY=eGcOz`lnU^|#kk+Izor6{%&S7-cm@icDb~QyvwY=%O{JWuzDI7C-Q|9%H&m%kX z@g9;BP96gbZqhgWmU;igwOT&7#H)wS>bua+Q=vmAKo13bt~fwt`x08SH&mZC_HUyh z!a5H93+NWM5A3e#84{FK31dcwS1Wt*TDWa3jD@d0PwOk^6P`y7P)ly(t{*wqL%9Im zSMX!0-`>_Lz9p`GW=vOwT#hJ5T3EQ~x8zF{6Zo6IoNv+zRZp^Oeg7vK!GCR!S-x(Z z6s#H%`r=^@DyiXsf*STO!)jUli@ya1jmmee`GZew(8ixf84H~kVD*HE?8?1G>m@=O34m=L$1%r79t@W@?p{sDgmc0+ zFDQ%8;1^TX#EYk$*);MsLJOj(KRct)@v{7v6av}^q6RbAen=|^q+WtzX3{;E{DQyZ z5%>4>N!<3~k0;L<6LLmP(j1s?ZT^1quEn9~h9-~D%}VS}=VJ%N$8TOBVO9Y@IFkX` zFP;%?W2o!koTRxJQk^1=lTYY7c|3~I5e?9b|KuHA|$K!-m7Nh-pk zB()}AWPg${i!Fg7`n}N#>O}j3$L1_qi=y2=mU`RqzF4j20J7*#c(qfZ58gIfo+ns* za3Nww?&lYybR}nT&n*`+lYE8~Qs~nNrVVN5HTbL2b=uihC5aRq8Gz7$=lB}hx3)bn zm@?#Z8x}-(ctj8@jZl}JD=!~i0bb5)y<@kf-1|Pm52<|xu_<_L{laeHdXc-ozZ6%+ zxb?bj(?cGA?aAIG_KC2EXK#(9C>MQkZD}r;)<@i`8%59CQ#pk1Y2`^UHAfT=%YFXL zldGI8B%^~r*Wv!|dO3#%--+z?8vE~%`aby>mfsMO86FOGRR6sgw~7VA&7?Qx`r7+~LF-HYkv7uy`pN`H5nS@fdTyiQhb?>qqyE0gE0s%4gKt32r&fti)Cl_B zteL-sbA&im*Ja9+R)6N4$??F(7Lj@H(bRML062WEMNI2B*`d8!>wWpx#l-G=zNE-L zYS_ujS}OKEE5PgJcfjWJHKRh~Ma=rwl7pk<05A2ehPZ&Kt89a(Lq#XVZnnA5VOLi#z?}d(9%N?(H4Dm6!Z! zq|}FB7y|1ZY)IJ>+{lnwYN>yWE_>G658|yW_4Z89Xm8a-=i_eWkq>2_hAv^rt$x$#N-Jm3y{21e=iQdKF{_X&+9?-1IHBVk%Ek`}W znpx1gS5ambN7Z2cyXW2pYKlorPSj6P1e5F)v~%7B$WGQ>sMgxmCKIi^qHFCrvvblb z*AmQ9pbBlf+E$LK*xM(`!o6`cRmHKEV;}yHwxw&9cAYY6V*W3U1+UXrkBI-p zhA9Pbq+V7O{~jHpcX#b>CjLE~i1|?VNo8?Tk?dtd=ZMg>N`cRq3h(PPwIxD&<9k z<5Od5Va?DTwq~Ot}J|hVtTR;6U?wqC2@b&94t8Er``)Fz0_uz1rw6+S0 z#gijLL5mB4vL9jzag(oR1y6fuqk>_VvK86L_&as6Ic7a2BzVAEYw4mxCr|hU`1)?>G1G!?zZ6>2#ohe+KF%d`_Gg?{+5(irW#A56`Em3xm z#!ldr!xC7LRa@W)>3Ey_&YTkpjno^be8trk+m)019yUv$^8G7ivt-ss(9VPan z*0CkkG-Y6C*3oh$#?eNDye7gRS@5H%j2?lCvi2(!+qX@dnpStz^w;| z(O}*ARDE&h7&M2cI?vt}#xH^|4dO{Chw6}YEw8A_euA9+Olu4D_@J^E=^QcZptwHf zeq-9)Q!On(d3~0}%f%Zv>(%f4InDnPqJ*K;Y5gqBY>{0=*{*f^l7AVpJGT)-92Vwux+p-Ez|l`XzWt&4j4$N;sF1Lj9#T)w^T13;l}IyLbbhlGm!; z7I$9{V1_!`o7xk07PvCCYKlJ+EmfOLR$#>x$38IzRy~-R{ZM}(J0}w#R7OKR+`apj zcNXqcPp^&wG&iWst0u-D1|^e(9}VBO@c1esa7s2ydf`hi6sK;f(d8HX=(r_0;Cu|Oz)Vvl#SGoXISFP`dxY4zJq}D> zQdfALjaVl+C0CvsD2$k_Q6rKC6qkR^ggMJ?9u66{QF-+IS1>OQlT^%I!P$j~_gfV!A<$nQ;y}? z1+_mu05W9M&-&+VP;ParEwKR=up{_sJJz`ip`(VG0Wpjdgbt4c!lAW*bn<3R{)<}A zjd@jSzbk!(=UrZ4XyWq#@(-jRt3beZaJMROzv<%2{0BY%_BGc5VGGi)rton&o&|K4 zD~c! z-1tu=yB)XhKey0p2qKUtczgMoP(ejFD;EpH zJ2^p!$Lq-wYZzPD@uAFeKtj&+Gw4LJ-dP~zia#dOnbmus>lYxvlIMj-8JS_R4h%gF zj+Ge^5$0FVc?q2K3wiMAV0-?hd`#W3 z5px|Np8R2x#qp^2&I*m`Rt3iA1p(R}Y|z^%#o=Jbys5!kxxRWfCj;JRd-VO3uUr}` zd<&Q%LIWwy3y0&}R_F;ZVKPud-@dRj|5P!3&HEcral^cR%546QZm9K;{=nN|%xE*U z=a|RlZZ-NmFyAzTLvlr9!=zEzg}5#ud;YaxT{rw-@tUo0%v$)DjG%q!n7vBK)5P4< zHP#}HtgI|atUc)I={+6aub%`QfOQu^>2dQ`)pRkaO8pZeajzne3c}@#Ds0(LvMzWc zdooK>nHIGs^+DOb{Es0ES?pMJN8&VjA|Szwf4#ET8wpxW(R+D#rj{32Oj&8!k5tr- z`&v5BZ5TRp;G)gS!q8&rQy9qNiQ74Q;o`vuK}PU}RlKGl!qJQGH`C*(qZ65H zT*2BQ1pg|VaLw&=^w0Fq(az<@Ru}7U+&fYVU}tk6q8xebk>^iZrM4mGa~AnBXiS2t z;uwlrSY3|qiIh*Nm?ngbcG4AfE7f~3D%zDp1^8ka8R6q#T))f;<+`v@10gE)JG3tT zN>zP2GZIB!n%}6=&fvd9KCHOC=hW>^_!l8u;Mczebh(l#RNx2;)_k$-$Oe3jT0Z3h z^A`fRsdn9MFMyS#Ji10l%eGI>I~n(y4`+Q@%NJWoqXtM8K;($Vj^vQPhqlqj2s1hd z*-AQ-;V;f3LffJitHYAQqU-oe_xUHiFh~tbA-7?vjbrEB+y*8SO52kNWV{RW0{U>X z)+1CuDwit<1V$~`@WA-3uHvv&(kVC$S^pCEV_ZPRXj+s^%Bc^hM8V+I&+dGD>0Ir2 zx#Z`<{ZqErm*9M?#!FSVqcVm5Dfc)KjKSz~B@8n9DztWlSHxzzzbii!f4jpK^!wM9 zt`7|y8dEv_$=K~!AaTS}Rron%zIvgnJ-4evh;OBBAZGppK-vV}EuI3-QZ=ajU;$r1 z$ZTeR3}wUSwdC+$fS*m>FXgIj_@NyDR3zb{r@!1VZ%p9!owNe-VZmrPtd}k70EJ`k zyOE-p9JI%v$iho281V-Mq!*Po-n73KYN*0vZu9ytR_Bk+FyWDzisW;y(+F*%C$KX1 zlSJJ(dr>#H8&h^>UVEXsW8F#N^>3-F`oF58Q2FcxgUC)h{|}*suM^D@7>0juB9qy- zCgX?NYscqq1|IcV1aMCqo({AYs~7te;~DwW6oK=9y9Sm|$AWK_vA5iGlQc9-6YZ8a zir}-WztnWU`kmcJaNdJ#dyV~{25LplOQ|*lD6%yy=vFy)xyRq_9zf{zQ$ttpB@-*K znHxcv#W09~dPZkJ&+-X5uh({QP!uSFwTCi!O7kTf^9Q>IZ2C#g;GdYMG$fn+_ zDh$d%m5&Ru0{Fi5mqfk!4Xjn3cFSubn4UGVq=oKmceZ?q4LBzGD zZj<^%{;GHmKr%s}DubAdYrjLbrRhx0tDmPC?p`i0;Sw$Fpa2yXT>sU-DG@wxPukB2 z5e!da`YGp(UDh1Fr;e=3L=^JT8v4KQSj*Ts9YQTCBfaKZa!>e zhRQXm2j$|HiI<~ELG(w2uQ*Pn8MIYMJxeUQpcJ%Mk3v=4)sHOHg%ql47_bTBAleN8j%VVIX%xKVJK5gz@u1XqRW-pTn32rx2XxfiO5Tx*om(>@Z|yNr}evxHc^9e-_pX2`AQb{VNw zw_DxWs^y50?#%{so60WbI6C?U9ICHz#&$i^cuQ8x!!0Dp2mc1YFxleoUl;2)Ns$Oj z-wTPa26482o4fd&bgp>;iT-tC^DTX>^ByAf(@Q|6FhGg)Gyz?x>Z1kRR}&zIh5}qB zjOwU*#_SiBpS+h+v&5&Ci2-pk4#K@}m>pwbxt=e&F&$`&#@^;%9@wR~e68Cpv0EMO zj}@E>3^emcn6Z!Ht=xZjf^OK~I0xkk))6DIBi}t9m zEskE9gKY1oE>ZMmkuLk6jG*FrOKp}YvKTuk`Mo-}f+_t?S3K(p^;Q#fY?QN>qcDfd zZr!ka(7Q`UV5o`2S}l9B{x$;|Ur>JkkEmk}nNL@dWvcDGf(YFRDQT}`5+F^`Pl^v% z!=o%o8RhgU`}qxp+~54yhcUZNXLFt8-YQW|8Cm-3A+Av;o(?tQ!_Am4SB!7;v_I&Ia~Vml%n?c3 zQkzHDpd7r>t}kg#gaD0idGcb>T{>&_^a@Lt81zV@6%9nB8C7(x+WF!4fs%I7Kk?3pcTi zE|F@23Y+`WMCy|LI&^7(P0Et4(at9>Keu_VfiQN11nv~Jz;ov~)c>6kAnNNqU$jQH zJIw~GiC>Ywu~ceB!N(PQ&-b!cH*8!T@HAvNMIGq+oA8=O`fN#?pJ%8vU_wAnLx07l zlS=6wSUmr+?Q6LRo|VX?|Ngm>D$0L$1Ejn#TJw~_g72?@Em}e{u&td`ByG^JN?#u# zu3tm?=%u*Cg~=?ZeTuwC*cGV5NkqKQ0{e-f&q%u8FAKM<7l9dJbQ+O7fU@B)3Yi#C zPIW?YSDH%P;I%0F>|pvb=jT9>0Yp4Jdwp(gC)O6(L7gxe%lexoI^=$i`M z))?8(#1=7jSM2&zw}cRrI(|l`(5k5NPpQ%2;?8BG@J_Eb&e%Z*saAUSSevAy_&iMv z=E}BML`CyAqaWMj7WZ7*%scBgrPG~N4iDhmRk4EyH-?xA*ITZb0^n#}mQ1x;vjJ9E zoZL`nb@W48({#ZtX1lgec8YO_dPyddg3(e~v_+FNya_~fiU-o_NnAdT(|6_6MPMac zyC@SDjoJ?11-V|~47WA>w;Y)5e4IrM*B%RWeHv8)sE&#u?^gRIJkA44DJ?Ol41+=XQ)}F445l=fQ`JieNZs6U< z&Qfhx5(r82NJFL;zAZ2-IfjhSayo1QcetwBtx4B%(>OWpgColrAOI(KL zJ{U7|DUtBjP9a(gdg~iWGjl&(t`IYvhL2MHDx)}irYyk-MZVnZ1{2wq00=sf+K72k zk@b;@b`6`~YW#Zmx%``VMt(N-14{c3Q8T!MmmWc(gU_uye$*8YgbQyg46fGM_S-%f z#d89fG=kgHS!~)uQXO0sMOqUzT!-#yG+{)dqfDpitI1c+ik=@+CU6(EW+b==2tuXG zMl@9Vy{}QgTSW(uvW!4L1|?zA2-6>gk~6B!PH68|Xruq(K_iskyWi1#m!)fz*~i(sw+AHT zT&8ilW}sn;l?IHvn*9sAFV<(;wZC|_nhJX6%N>}|+>;<TMA{*S=k#`aVo}{2A=%j0zFH|zkTb5UwL=etu8@sYI?h1X8*1_wPQ(+jr(}&npoyL?m$J`6(>a+RZV;F+FUhBWP4)qV>%fqiW$g zDfHbZ&dJk8CyApI^(;GE)bFbg5gP4%Ki*Cqxa^8PVZnW`7Q$m+bxkhbm<;}(NE zem+i#1JPCWYZ!L%zDRzWQ2A#fk{5Aaz5ASC1_K{3zV$Fs$t}p8cF=m{nm>-b&Azgy zFL7e<5IZ-U&9>=q%eZ>U^`(2~plIp|6q?q=_z}0LcU7^&-Yra6oqRH0x&c0AlS~?Y z?3Z!WXD6^)hnYyE|}xl(9x zx^A63bYT<_J42fYvKm?VNy!B=%bLEI9B_w$D$GW3t+&{Ui9LKql>0O+xZUfu6Cicw z5d>8JDv$(g5pgLisKS|hx@y1n^~E0fOjKleOmq3FSeUBpKx(U(LbK_V&XSJwMHv+S zrYXb}x9(_Zuw>OQEtv!#EZ#uys)3TvS3k2Kxhj~Yb540%raNc6O6Sr_my&ns!y8SR zz0fi<|Md*-a<`|eCNYas&r(du#e7QB-ia1e)hux*G&c4f{>zksnc^$tEr(dfFIz(; z!AlgydPY)aY)HhQu0E&a;24_W5V{MRe` zNG`uk1oUJ9+tU%o9yo%`fVGe^Le)tMkJuUg6rDXhilyuH@2#`5-~D?}%Jt=T&n4!% z5QBY9-P$*KK3~8Yb=Vg-RhrO}4aos%G(lJ=p~ArBKY12Z7YkD6jAwno76fMN(c(#g z-QkPcdLzCWCQk{qidM4jqV2>K5zX#CLDizUr*)V2L99A?^0EK+WREX+))>6KRaHG1KO9_p&9+F1y(?gbQEFA~LiGZ} z=r2&y7Hp*+GyHpjz?UY|YyHrTKAf+4+?!U1A>xnyR)5F(Z?-Z{Wmlg%y}2|1Z{+x} zrx<`}gG1r?l(;X!Md2hB@i1$*#{xS-L9tElUWiXlC&bMz z;-;*i7IjF-3qbvWOr?5zsIzv3wR}#m@J!e-H3vJWEX`I)iI(MPYLkz*iqRpmksof$ ziF^o1EOPd|GDQ3f*PeTFjy&GXW;o%to6H>WR;QM;X_!4u3OkxaF6!8JkvdJJwv_p7}XZRMSxhfQr= zxzA(G2G2~ynV%fvV4_5(Ot2@(!_T;H0f}_&|h|~5q%vOVlvEO6(gt-*%vwp zOVH(U5xaF;S@IV;R+dqKbtCMD0BWxIbz@KW`GtAc41LKgL9{&>;y%lvaRark^iJFY zbltyR!qcqhhv3j&#SQ=WFXxK`0ndN&CRH~q-H%lF*xv3`1!A0*Qr-HW|^lQ zuxC;_=$Mg$vU)Z>?|A2GHTZo9LW1xh*DeFVkmjU~oIVSalb?7P{}5nEZ{+=QZtLhUHW&=K0-y~?fS;dyGD#S?QULqj)TMc12;ZV`O}hAk zw;{DZ=?p(#e!E|_wXQeu`HPHIm{M5%cHM?$S~1>&rYXZ`{z2xvB_;g<7%flavGqpf z4+8?blBY_8 z=PoPX5W5lfJumy=yNgBV?r^sJWBkuG@Zu!b+9H3^^kL13!cA8LwivYx9+d-~F3)%1 z)Hpg7y{KJw0b7hMbiE{7nTwAz$H-9Sxrm}(e)aI8`vsP-(%%kFPE_xQj|%r5QSP+* z4?V)?p?i^u`O<|yJlZr^qYrsYmFG&-mdg zLEr+5u5+N&$T5aDnD$=HL6}A&0N3}C?IuXy-@`vxTwKsMA`kP26tvT;&7+M}yqm6z z9d!Fy(ElI?2FDtDa`ZREp^t+K?(^M9bb|wk4M1&NgX$FT!>Pa*n0_s=6qRhg8&J;Y z7d1Zo4tE?BxtqV+(Ie2UTRWze;7c?JJQA|ij&QKR3i6N20NV0RK#M)EI0I5-OJ5|y zPwS^6a9!@R!sU!la{2`(JcZ!)hg}$I9QmQdz%d zkm2{kuU8?Feoz;_VDf%c_wU$v{(9R%b@tqF+gCo7rB7Q2wnnB! z;>uWwt5^GiXz%}`nV?`8MYPa#{pvwy*6Hwpy-}@F3Ow5L%ix}pzqbk3pCOhyQ}VLY z*RIntr`SPZy+-%n1rG<9iZI|adz|z`nV2$Ct&g;R{fWvW__EDz(^rF;zDOErr0Dui zsUHBb$;e;n8G*qn>EcU4Yvq9YqGKnG_UG6R&jK~9b}vfj!5sB;jvMbQ18^3EGQC$^ zL?5-RJaOkZ$D396IGN5_`e)cwE>4h>WX~l8vC@LY|A)y$S zoOsJ4-TS8cM6>G;#U8t2NQEA*(i~iB(G?W9zY_JvTAiTUr#^l3@c8?28BK@PiemkC zIY-~N-za4*r;j#cp)YU@sGTv^VEAI`dqltPept* zJE)kW9ua47vSFQHeI)|io!J%i>Quc_y{zlO9fL?M(+r+G)rQTR_k{FKHVIemjB3Mo z!^H|pNMpaFbCXL{-SP`%fHDC7WYmarL{+F{qEE62ITLLGP8LyJ#OnFx>-0y~sQyI7 z@$7;-GeCc%nn35!2X56=vZZ6&(rY*G#JTy&ieA3SHe zPUrZp<>wh+f4!rCW9!sDBG0a9-XqF9Ve&V#q-wxhYc&9AnmclTQXtye?xCIY^w%;{ zZrJ@-ydM49F>BNbvyBq&d-R`!-{I-#-F)9@;}lKP)Oho2v{R9P{=~BYV|EW<>Ym$= zBbl9`XWPB8?e9ZbTAf>Xh> z@QsgoiY_RO?Gpg2Qgvw6*VRj9cS}ieOWtn~C2GpbR7*M&6^8YA$)VMBHt%j*(i=Yx zy3Upft01=K90+h_vva^?D={%~bY51`T5{>BUGJYF2~>HW48G_bB6E~FvHDJbmGOw zsoD9QW_BkvdKg|22dg~k@$&SPSMujm=07*6DKcq@0c0+U-d-{_g2`uB=p=H4%G*@i6b9i|qBO-`fEH~+4RgC2IW_MK z-hY&yrWb#RQ(jhH{_k#n4$hNmc^z1iXy-UFIXSr&siI9?Sy^#`v2eF$W@LbvTWeSQ z&XpapO6#pHb{K#_`7k~WyC#{C6^2IxpPK9HeClgzyj4_H{pxFL;l4gTK8t?dWE9e> zAPdygxbwf)cxQQwi4v1}dHGmXV`qMqr*+p|kV|*p*~LYqU^~`}cHvhH!7S%T_q@Ek zkaS)*GF|p_hi?UqytvsX8H6C@SubMG8^^0-6q>FoCagmyoPpB+-S-nddMH`h7&zNO zdqg@AQ9u1gPYyt7!_?W?Sy{6|QV9bC!%iR}M+2)BiJ4Yo^Z7*Rg_zm?H( zTPg{1eA?A7$cQf#ipWO_BX(Du;O|)|CN6R4!AtAU`W1FDWkRQNo{PLnDE-D z|L@rTgXi5Ak#iu6|Ds7}3#Zk0L}1j7dbP!U->%|{2@~4)C-O{Vq)AeBfyMuLRhbs| zkHxv{t}{sE#H<1E-G2rziVapiQ$9P`qy7AUg8}LioyQ4y?9UcY>70yndey+cQKp)9 z_GU!0+Uft=_o6WE!SZtR4SoD?xw^l_xs(9kde9O z&M=p`ZZkIbpU?N4-##=m%UfISNOarBFC0)96t7r<1=N4}2O7@p#)=U>%; z$9fXqwDyc9{c^;8#|It-`Y-|oU%bLqyUCtJO9trrjezHb*HKBDQQ>8C66k&=jt`hS z{A-+)@7d-D5Jb)o6sm^-y?udR^xUMRNJH81)ih>jkz$n_F}nIo{?TpcBL7WjSl-EC z3f@Xw$l8`c>r4Dx*&p?l2l@iz9eP+JwdPe`{nwk_x|b_2VK4%HNJgV*v!Ylg35HsL znb|X2M6-0aewsr9zQL#W8mIS2D^2SwW+=L0IGtqIG;e1HC9*?_{`0R6C7Na=%(Au$ zV42z$C2V;Al6g!YyhZJ8kLr$799V0NR&Ea>1OpD`SB$ZAWAnoWpTh<9>#cl23w@9^ zcXOTKR*5M%Eds!dhz=pt-A*`68UmI#l=0YYJ=y>;YC1CaSFndGGrN1(qrC*eP6i?P zPWW%oRG;W2<5pvHcb7g%<37~EaQH!|-qEHWdVfyuaIO=3FpiZPlh4Za|9JG@s}qJB z7Orta5}vZD2Ykp=Z=_GwH;&h5HgtqGbVNYYgq8Q5v^OZArGlQ6KkEmzj77U2{1cBO z!iTKGhAy0fH-|H$!x4XuHjWNAI&~UabtE(9ouoHY60pn|tXA_8$^4K+{4vdJ*OX>U zZlAk#bTEB%Ki84X?a4P9TA`hNT55m4LZf;E7{835|sE6N@78Y((?D>dXwI#M>w#p8WP0(ze$ zBqXql!iger5sL-I$cBEWFLSnFgv(5kYW2FQy~#FewF9;4$iA^#v=Q>7i*$M#u=^FD zz4Kpu1_JrB1a<>jT=iOa2J{XFS~*nr;pEHH2Z@cjn*!sRhu-Y<{EuHq)Qm<%maR-q zA504NY^_Znu9-K@f|_Pac<0ZekMKBHld){^qh@xghv$)zxGu8ME7bcE;_4>Yb8qwpNG^A-aAG@pa zsaI*}W^e=Uc{%gebffw~TU-)Ae-w^4pZWHQUXs=ti4pkHP^IQJcnwj5;9Tej$cUpJWmx<0_)jeUGP&;)f1X(lOiikxY-6++Je zgwcfbs6w*gjG*u!T5|YKvaE6#+1T8EZ)`XVN#PEs$eA_#Qtb|+mH%Lv5=(Iyt5 zjEcXM3JGdGcCS9V%4|E!KG-nexnZcodRXmv^U-LtfA)X)L7jN(u*AOp50n2uBtUR4tgx$h- zG2y$UnBzSvNSNVV_-_wu1y3`=rY*)`(9KkC!z1M*nzC&8HY_~QcRGxb*|2O`tQ}^s zf7yX{IXYY64&uvEq-X;%JE7Tt8}}w_Stk1cU%Zcxn8u)%G19-?Wt!>l@Fi!8Eb=J3BD32sRA5;T~iVSSw3peMSNBHI2ctK^kOpVh zq|i?+aj>wc;bG+8VMzo*7`1k3NM!2Msh2a>Z{no{0w_hdDgFM*yxl+Io-jktC3MnUcBZH9xxo5p1ET{feu36-J2%wRy`+j;1KS5oNvpxf$@?W6l->uH< zg~Bk^!Q0T|q!Ae5M-08@q|nFNO??kBKt3;mwHYA)zR%C7(X1*?g?JIYfMNm0boh~e z-Y%j``;e;hrQa0r$z@AA{AaUi`!S1mWYO58Jh+>SSa;Zb{H-ee5&a2S{x78xbY?22 zhf!<8(xl3S`%?1B&s(jCII1dZj;Kyl&W6czD`_{oRO^3c*AFyz%pi>Y_2&Prdu-&o zz{l3Eou3(@<82xBYtAalHNg=l?DeVtZG6FeDdc0e92Nn+sq)fZUddznpALf$PWduU z*0Zo_Laag0VNb|Z?)$sCpGg5qu8{u*R;{A;XerNGwwqSIx#NR^^bPow*7BdpTA|csyg*%&1%Q93QbSp>b7;{%%lI zc&U?e+XV*>gpaofgx+5lhsRz}3E6hL>}t7K_N`oM@9*^f-)L9^ljqSuX_x3N>Z7h^ z6E@w=ZMKX;WL#Pk+xM5wx3i=Ph;}uBh+8b^ID0VIDvvEoiesy#Ytwh1Y!DuhPVnU% z-&q~{H74*L(FpPV}4B^4!}ss+6eNKFcJIMJiFwCIbJap}h>k3FVd5 zLA)j<9D#&gh-MY-pFQx5*L)6R)%FReNxsAUx^mrXdxyT^m{+TR%8|;7&>0?cEib<5 zd=KTVYvP}+%?!7`Nl08Z8URckYS#8{&@Xk640OuW>Mx%DU`@|bdve&5>LAaYSeT zS_Y&>alg}NX!;<=p8l;h=hIlwtj|0~0H2?;uK;p*gvFT)CJ1bt=@z1V7N2a0_r|LS zpBh>0xu+f)a$9OB<<&-54P(Ze9isu4X+7=x5qyiEgbGN*<+7oN(uRiKC;Mgb|Lqic z+)AKIC%i_{68FW*j|?c=bsmbH%19WLD?JFGCeZ(Ru;^!o=rXXoD8YofIqR|XqIO+Ea+@l@HCvE0lL-j5077o+Gwelz_ zo@#^T@=LENB%7m!1@uPVN5NFczxlG_EKrDCkieJ|lh`MmXk|;)0kz2-b6C{h3SE>? zhe*9;e)C!zbVI}DXUB%Z4*uWu+I~yDnt55X9r_Bi7U$80H=eN+`K!QJg|QyFzXqa& z^;eEJf8n1vSs^Lek=&YSW_V?~QoGfyxvRJW?^`kR8#c?8?mq8PH%ruUDPRccNu{Y@ zY1L!?HeG*pS=I)}6UOv_K6ckq6V!te$rly%X7UB_K#J<0J>wk3kior zIUf{-mxD`qiZ`ssPsnN`a4mlqCX11{mUMp_sKHnDTe=pllh^uDlXd|Emy;QgxDDWKqAoJqI5uI`e%+u@LSAa;anofnA`13iLS~RNs>mU*Ds7uh! zcql?QGN>4dzb8%bkIOsTgk*GnG!POP`JIpK@21fz6V%gwZhkNBDc>*9Md1@Z#X0u% z$vjCc4(X;DCEHWUDDz@$>kmSChvy|Ni9dPr=?psrNLlwBI^VPnl*Z(Apw^@Jzx_1P z?vh2up?Q+6HJ&0k#`|F{c>;YVVofu}?y9m4NSDx%gN|8OfLfmWop}ir@w;+B@$i~l zIHl{l^%Q>20&34JvMVKLd(u7_3SZQdnyscqO7wi^gLZwc#~X1|C|%YfUaH?`63tz; z`s$L+i;%uRej7?ZBFPM~KYMipH(_qGL+oef=I=RnP7JK9-74cd&|TBjVRi6&)@24n z54oPJ7Lpa4kON7|f%{k0CYS_AJ6rYKnYv`3{do(Z?gS#&SZfl*oj&}XUmCr9E==wT zX-+WwUjS`lb2wB|=^!+xY^o&?@y^^=a(6~eq2t$$p0}>ho4*Rq#)-a@BLCG~B%+k5LxF2SHaGNk6 zNegO<8o0Qa%-9OdY ztD$gQKa9-4HzyIOFcU5RchZ>~r=Sz3iaPbAJJEFy|LL0xfO$oVosFp@{a&-{J1bx=0(n>>DG*rIs+_^$3S7OU5cQAs zywr&Yj%jbQ?q7Q;{uS$ebsK-BBq>@a_Fpr_J&kqQx$LZA*GpONsEZj+s=a2T6pHQB z$>bs}l4XbF?OmpG=jcQd#(17w7f`H_N(Qdo5So0H)v#o3Pb^C6AW3-zJOIL1qRjVy zo9ix}Hltyn)Uko|u)xW&78~`ie$2RXa`~dW=<-C^_WfTb&`^E>hpCx+NOKerTA8!w z97(-7^IL6#GAo5NtyDcWFHnFU zxlXy!ZL(#0U_}RLI6bA}*b%O3Htx*Jf5wjdp{w(eV-rK>E0g@2l3nibvVX$8%t*wZ-pPX@o+7kPy7oK9Ce<(ap%GvyU+1$R@@lxWvi(mh)cGK(fwtZQ>-4oSp8q3ga?vsI&Wf{2NE&>Y)|t|F3k)3aBXhvbSLDqNl%tN`f8wE zSnAB92)+ymfi>u&NKQZv_XV!^ti9mH46V79lfL8nC{{A^MR@@10pD^D6qbU}@);+d z{jTd1G;vuP^^vVZ4(_b_xC4IfC7yG{-(@21WE()xFoReAz;Mh@+wAxKjI7B%Twi+U zdG6Q<1}z!85AV*$JyZFYLLNUz>+y}M^XrkI;2n1RlcoJNvpH`Qx`tNz;!*UkN>+7o zq4O0Pm?DAYWnEv|uZ={hli?B8b}#HRA}!0cKN^a_o@(BHk_6t&h`Oy7p3B=K?rw4P zGTK;H6!&! ztS^m}Hvc`G94M?yY!8ZiF_&OI2Qc-8+*R>5?d=4%+W4I1}+?Zmj`a-71_7cn&oB zK{{?!zSWB>^TPuznk)9u!K);sigTgHTtqg2f%>tLUWHV5b&dGeb z>5Avelln!&5@8%v$$_urSw+-U~|ua2hJq9f8%c>O!ojhf4oCpqprwLu^s zz|i){@4KHbVDTKs5)I)N8=&fcf{==lX6kLHmK)VOm|OKMj#3{kH4{3^OY^Oq8uEr;aEctRveB#viPig5U?UH zL!^v$9)#FF_0WqyD}Ai3OJ!tX{oDZoWv|_D087%dzxGv20>M?6*UJ7xT^`_wTCdt_ z);(%v&kFRdl^rMj)JTgsKaypyxY(kEiA^=(&GWQ&;7w%va`j8$_){kd)09ni%9=*0 zsLm~>L5@P`=1834r>Fb!LX;gTW?xD1Au$=!8W3z2A8o0FRQ{q742;%O0n&u&*E$!u zO`H=>>oF`GK$N`=?9urc^BJtk#R|bDN?}i|k59oId!9Fm5(*UbukcIJb>i=~;gIPF z+f91j3OwxllDC;Jt9i9|o#MB*Gkayrqh^I0>XwWRC0Ng~6Fh~L z*^Bh$E8Cny|C9qF`qMW^SIyW@2^x5R{Co5$ee|50G-8w>l+>n~ zAXpA3{9Sr>4-wa+f8vfa6ISysCWJV5H8M$G!0jK|#wlOa5T}pRX9uc0m%wR1!j659 zVc|E2ELrRH*+Rx|#GAceY)BSurVc>8WX8UT)(n|9n%EP60Hl??iX8SDF;%HL%7@E& zkiL%t5;bGS61DWq!RI`2yWQK*FIOx#OgkvaTA*Dvw_MzXo;p>;tQ|{gh3`iT0g`RX zp^nL`#kX0S@)gr{%k-U;_f(n`Me5>}0nHj~iESOs(r+Skh=oRku%n_u;>^>osT`$1 zk5DOru-H%V&lh5=AA->ah7FVwc`MiS^v7_JkJqd41}1pTC1zq|&}^9xAb^Fb9z(lH zIPTqhs*A$FfQxEk(GtVMC(6);!!^M*1?=1I5*n?q3KbYF`1kATIf1Ow%FBs;k0R8@ zvcRn{#L4ONr)+RUK7mM6>{!L?^(hlQ6!4TIn}D=4FNMF%YLHFP2Chotlvd{wOG#D5 zT%^d#Oc?2OCHg#p>*g~aD?CyW=xbgb6@C=;pq~CM``xSa(;{_7ryIv^prZnYUMsLZ zu0KJ{+Dkw1;{aY}rfvCEAT&Zd2IAhKO2%=gXJ;l~EiX2*sC#|&yD>)Cwhu9xBt>mR zgpQj%AMeUoZr!~v|A!oev!3S5nv#1ekv0FuV>J?Ijdf>-b7(0SamE)>XZLwfZ?G&D z-nzknobC7RUGVl(64qRLP*2Qmy@C%PHd;w;_U4KK-4wI+1+Z)4X`Srv7hJ{prVEMpy3< z3^}!xZwTKJYPOj*(n}HX$v?aEAetL%_n7_- zE}mMF~90rP1q+^*H+iv6XRg#}D_n|6o!~%I{ar zeUg-JueqvcP4xWBS1|fX_=>}`7tKAU@}plFx}V)&=YI&Zd-xDAPolx3gKW3XZ8ptq zUrePfODcy1M2IIW->ZFCD_m~KOKXn|e380jgSfInK*7gQngX4o1y6d$?-lWXuukBg zyDznKO_};!Ii#m5b9Z&8ORZ?#)V5WCppKydN&eB`$Q5i}cS_yxr| zn6)RN`VWZe=|wl##OFw29J~mwFQbq-bqh;i^9j6v3r75xctvvWDu-AB@T}DbIHziq z74bBQ|0KLT-0am1Y#*q=^`)-n&(#`Vd3vE&|AEbO;@C4aICZ(U1Qz0YNWUk$Az-7p zXhT{2*ekXa_F(otC!|#;F9y}eg$P-Qi#Ia%a@*rqj5gA&fg`cMl=I__gra|*6yR0_ zKp>`E4Qm4SSoj<8q$0Q2aj!EG_d3(XqZ|+$5NL69Es#S*!*MUw3-K;Cj8Hh|mN@6` z>mgYJaQZ9z4gc6GlW3aN7t-KteI-mmq<2$?O?}66C?z63e$Zyo+;b)0LL0Og%H$Lc ziv`y^F8#vphfu*HLL|8uOhdC7+O6XCT6lfx@+}{sZMxb_whT5`sr_OD&pUnQ$w+KULq3+j80@EEMzvS>DD>G{BCz?#l~0Xg{?>%6j7C4e6aG zFbcjUvq7Hk$i{XaA=%XtZeHAjsCNoWx}R9|=63I^*U4;2fA`>tmdCB_ey2kdt^Zw+ z0${z=f9L`j1Vz;3+Nmbh_WDpyaAEY(ChjU~o&>Dd{HQ^>=>pikAwf!yfd>}DXKVY- zCOTn%J2X+pkySqyFvRLo;|W>zri-e3HBe_h;+9|4)vjZk#dw%H=;)oS-iGYjs}AEw zhhkoH9L4Zl>bSi5`tjc-nidWMTW4IG#RJrQlavWHt?Ul4O!1@NKdJJ5=nt;#WKW~I)I@!M03N+Vku_=B>(pVIbpC(}*+U_9$&mpLVL;^a*(n7|?NCEQNHWXgM7V5|@Am!LJ-jh4F)%#2{MjatCvDOW;XbQ+r zdED2e+)goOB6w#0{fd`C@l=L&ol^{LIGA zMw*W#s=0SrXL%DHI}3D??)$Q)^QO0|@W%w5@6tM4hgBUs*|ErvV4SNW8YwRWs?nS$pN>x|uGvp%oO{0=TDM$=??;7C48C>4%s@m=)&WRcpZ-hQ%I7hSv&A!SvnX;P5!U@0UYMo{zIP_$cJHl zSi=HaFTg6y?yET#=1JbjW>sC6WJsRC_w^*XEOHrYN}N{OB}p0Q+?O^h`;4WTW8E^5 z8u7ugN~qkvfOzOg_wiuslE?BH0cp2ci8vZ?BIH3El+UvK zBGcl(TaAn1twz|?H|(yj+i@7e@9|A*0;@LUFU~kNdbch#EKRDDJ9^j>YiP%-L~v!M z1K&C(z?h!BPj4J5s8=cCR2~Lu}kC0lzUdtY^k4V*w zsT1)gx_JfbbB!2lg7@^Z-2&EUfn8+YQ$5^)T*XP5bKI>|A%Uopp(OpwHj7JMemet6 zEKb)$s^&kQiE7aAi&y1v3}&CY|BByj@t1_nL7&Oa|0YPKQ3s{n=2jeH`;{DaKe;neUeG=r0QgvsS`&6JXjwLiv1e?_!PT zbb5muyH`i>kNmVk_j3ND45>)_vHYA zh2yS)GTi3czk61pJsXv0l(bSZ8KA!(rr%FEUOjQ{!~=CM_vpPjW04dVClS-Ep*-Cw z*UhDF{9d7)cG+XL*A9DqUFO1xIL!$UcHm8?+7mQW3lj30k zdE6=aRYq}x-wmIDfyzB?7=Zo{zY@f(Q-5|CN67rlrDDDOYkkW{U2E@OnRDwsnjvXT z`#s!W%!=!9%aEAA7h)BCPESQ|#;^%cURU z93LiJvPaJ(_>}OlaPggs9e;W1R6+OO_9(FRlY7*1mYQP=kGEp4UBIk|f!Ae$j_qoF zg?l!TgxkujzJb7VbVl&zvtacfzoGEC50M=C1_ zkmh{bXTBIh_VLY|954?3v@+XE9@y4C%=qPRua;uY%nEnYf%LCkcu8 z#7|E1vb|$Kf$2>fTK7ks=Y0miF{)C$Y^b8y$FXnV{lveD>686k#7axFMRQlxMxD;R zVD^gZYhblBJoW6`f-ksyW4V!6LGi_Pvra3$cur+K7P1A1wIQ&>;b5^!LmW@7koju9 z@I6O$o-I$Kk``#*R8{Ingg%BRoL;NIw~XKVJB<$UpQYXkVG2n~?Xx{7VbtK}IID;+ zSinz~e@up4kVC9}ff6eNv_eD5rId~B#G<6vdJU(^9iz#Mqv%eJ(2_1E?mvi1l~cbr zWvXU&`1FJ?Mn6G(;BMELglT4VZk{H~Y%%mzkqZk~vv_{-fNj?_D!rz)XhJgk5OhcD zwUsa7KRnHq(O&Cev~bqPH&u{5lf*M6oEn5!us6t${f~wshCjK~kHi(nQq-XDDlx?3 zl@F<5jd4)^Dt5{0HR9awb$&Y656kSuD!NcJC#u}P+>=n-%jN}ByD<>@tc?M3|5^PBa3vM9XPy@R0*e$d>YZs_;orYagaj7`h_P#-q*nccHY z(|Qa(#9w4|V?W<`*XT*PG;W*j3%~PRk}%6U@Gb<0wy9looV~ z#lG`Nm|jnC*WKg9l>Yd@9~|E*)??jV_;_mmcAph}DQkBSBi$}MU5Za(92Xq(WnGPW zhsAp5IP7gy-!?3@G5w0FEI3awvOu_FW30u(Xd8@jcI70 zm)*bS<;$wL$<^_y{NxUfvzeD2N7m0IHe|R0y|>b$n*F>@b4rH{1nrIluDYk&9(4Dj z>EC6gs@C~YbvEvK+~A=n@Xa$Hcs{&Lqq@~|?U_W4XJ%s-Ck_~Pq3-n*)ALJEhmuxw z+=C-QkyDLNy(^t6ze*ubXHV3#1C8m0v4ts~`gxd;zG;~gATM=6m+Gk@)9JbU>EC5Wy3K<_4{2J)32uajwZPFN z>Xp|AB{vVkvJS*^A}gfUO9|)`?Kq zxM!lG-exL6mnoq9q^vRU&FY`jrO?#BJ=QS^J1MY*AiDbG?0_>)L`=xq%7QwDhY=r9qDzbdkKX$`kvpht9;JW$}Z$so4R&UXl(opLA zYRBCc(I!fE!ob%8UXG?J6!nZ)C(u})Y1;=9@?1UnP&wjSO&#&_T5Jd+ixBz-c>Py2 zt?FpV&ntg9g*_YNcvmB2d`vQ7*I-&CL@3&-hI+A;qgm}1jqJ%LbAIlrZzR_8ZK(fP zo{?~hW>DWJe(Uj-R&i428aVd}D{>a&P!V(ZyERmBR~le9Kl8P3T@n&Hd%kj7A-deV zIBIQp-jiLZbT#>a9DdN#L|+Ns?m*3yh5HBgoAgtk`QfQ5?G0hxl2hSF2hoOxD>Ju9 zqeJD-7lNb_#E4ew)1E!+S)A-Tu%LZ5NxlT%@**CxF-ea9B2 zl%4n>IbQR_VeN&bcnO89C&Mp6N)1Wo5!W80gAh<*LO-I&eZQFskC%3+pl15pQ7Ct1 z54)|Et}!8K3m+|uWV%3jA;vevDh`^~X}t40RW)*)m0Gp05Hnn=9`?(wf|_W+C%exB zB7Ciyb^rW8d<O?PN-jTOjt7m()d3V^oxoIxkpv(>GU}w+LaFzd|VvC zEIT|7UBH9MZj8Bmgh#$~^{?oVS@cPy)re$yc5^ z^prxgy%P-Gy%a>k#w&D(lOiE+ZWy8)+~5;^B)`$H=bY4n-l0F% zFh%2=91a=~I5#}=#_yAQlihZZbeQ%f0MrJ~utS#bY%-_4!C;Gj-P1}w^zx$0xq`hl zyiIU`a%Ob`)qb>}UT`8>8zO0MeJ04NitES4{fdl zthdFk=c-cuZ7x{dSr$vzO|z3jH*^)tH3gNeQ#*n{dQahbISonL>*~pRa00adXmRxV z{C{6c3Q)j~ZQgd5wQstZvKL%f*6J|BSVk3YuHh|xthatic_%A}H1PiJJ7^pO?&9Y$rY1~V-*25? zr+>w|)RWInuMb4nCp~%C;pTHWQ}vH&;MN*yXmjSzMJK$z&JI-4-mYdR%hF8s`+QEc z&I~oBu29=B6y$!@d1%fG@#Jcngcb8LKQR*w27nbHCW zgC0dnloS`i=*9OsE8>J@$b7fHN|!aN+82O*p!?Ynv+6`M8|E3`90t85po+1he=1_# zLA3VH?>b$;SsvKfl&!j*%T7Y4(=yn-MK04N2;=}U?)+3NHEAB^dZ>^I#3G(ltPy5) zy7Ep^*2y)?M$y6=!JQzi9o1#I8sGSZVVAi^QqfaL@$r~4n_szChWvL8tJWiS7XuZr zpvv8u_8DL;hTruutd8l;OJ~2N&80DNdx_A3>$srMgtn??YKYZ$w92~gUPc|`qSbWn z!q!}3(_hm=JVAWK16`oCcd*4fJ1`nzJP|Z*rsMH=Wy8~Y`FM1DX>yIvY5U3(A0*`x3{uQKI7fQ}(@YV~k`pm=%4tm-~b7{QEWV zz20CaT7V7rTeGF`(s`Hh!xeQiqz=v`({%4H>7#tU5LsF)|AY zqu&}WM|CELc<7$?;X2&76n}NfDt7gz{B>}R%NIabYy3aW*#qvUE%AMk(W3E=NYT(h zM~gYR(99YbwMoK-EzV~^YG%2f3G8;xbqJ_=i&XDfrg-hGCLjlIBOY_WywuPM?@tTh z-=UTrnhu|;Www4*2z|e8WsxAv&CwTi67yCgqSi9Y`hC$GGj~0Xr_6IVOZZXs?ooeO z{gJVos{TPS6GXz?4}LX-OkQ^!WzA+aMEAz)8kxsUMBF|modFe0)ZTvmmr zZe`Y^HeP9#2vT~;dc%ny`kac@=vSwj^50iuY69c z)okeLv3Ch(R#dvbeRxAZKWeAHbIxpWfM;5~f<>nbF z=X~c`ts8x{Es~9_7khQsUK|?TNll1L6u)C;;qg%?t&jZJ%+z=AN7whKc=&Sqz!(Ld zv8S__T865h?;bKfb8MqT6KeK$|nm_S84vI;Nm5?jPyb zyG59M^yTGJ_Gts|HF>TpenZ|%^k7Vg6}J}Z;qAEHS|(Ot{yn$e-{Eq-D&xg51>0ox zMzG9K?a6jq(g%kqCXS`;+ING}wqJAVd2yAwUCrW(x~%OccW%l#o%(Py>g(d~K2?-q z4zHy1x5B=FH?T%7Is2;Kl`C&NDc#r;S8Wh%N>EhQjW?_t!+Bia6@l%)8bHChxZJ8* zib^kMc)~gMB5Ypjj;efJdiDu>u9H!>iyJwfdCe<3(5mVdtZH){z^A?1-1r5CPa^#X zxR>7~Yj~ad$ZkgP7jm3Tce4Z-qR%Px_%cp3XRlvb%6h_9{m<+Pl3ZWH&tG#ieYw%`tYRaLC^G>r3IBk!qtrvHkMWpHK2-kAA~G z-xMrk4jw}E<6cMk2K)oypZg_gZXtPnaWQOU5V;q z;>tNaTHS|NGS$M`)v{i=UFUyvZ}>z~I!^5E2cC(xD^HjV(w9E8%{Dk|n^i_#|IsK%1KfPz&Hm}1$% zcDwWI(o>^fJHe=UZlBGY~ZJ8eVxP;HG}kzmF%&HjC10uLdUdzWwm}s!i_D zeam~4UK=aQHBXlZ$HTOI3h~e?x0F)Y3UcZ`_b=v30-DIhm!s#1h()zJ3Ayy`uOfn>lf=NK7bv`UsK;$qbA0T z?DFlUc|8XnETZwWYE>E{)&SJCVh4(h>DgPkg0aqhn*WIfp8_uK4tR@nq)ocWcn&`| z2at8*lRiXQDIY6ST1rmdP>LN zvJs&VmhQG(-BFPf{od)_kZwXuKeC7>-RWz`lsda~1Lo~O)jBVO0$%0xJcxPCoax+f zFQ;=Dn}eR!Us#T`szV*HdnZ8x6gNtNkVj&~{0|Djb18cCI|Wr$D=mV*b6%`Z2|ib- zm=#jit|XQp*Vvoxc*vLBs9g^a(bTcLch>u9_fp*ceZenbvJQ&G9hy!MWOZkFxWg)P zo^Vl3!4AcZOavo&_t(vb|5(1(806199K2g&a9W-7!=YK8FO{;&B_+e1&knQCzIOGx zVct+Bm&VFF>z{f7Gu9<{;?Dok`ul}5h#u*hDW@{BER<++ig?GaQ&tD8mJ}46=cHpJ z?Z6&;e5>q3RKEr)wX?V24mG*D(abv8@bpQXMqHH_NCEr+XZbSOP_2uUpBz~IjGwz9 zsc`f;`@5*p!g#B3o{W=IukP55v}>Nq+svu2`ZsK`Fz8}NEEoY<1bI10Kqs|mT&$47 zD@$Lc^5YBJrtIDpY5xv22iS6d(|cq&F8vR%g>GQc>Ud`elq_s}32ppwFw-ghi2C&} z@WTaN$aX0^X(}#>YN#jrxZc?@(mXJs$m&hI<8CWP^8okxb`uX=g@Qk~xjIn=0^BJU zQ82al99r1>ESp*7HF3_TG3U3M*8h6}Vs4nLJBt}F|6J2GAX<<@C2s=`&tzSV4cfhH z%9c1!4(~ErW_0p$ZAO3 zM3Zc^5EDkSASXoCzoeP2jd$Ik0AUYk00qjS;&v@?qdNZ_?6DE$L5hMYsHp zk~dz+Wf+I%`nO;C=*D4p+Ux&1I`42czyFP^S*299)Ye7SYEfHK)lw^It0-!;b_r?* z5v#3FYgHpO2wJPQ+O<$>*@gP?hrb$vH9LhQKV)c2yrc9%BR|xYx=0I!Z7e zwD1W|Vch>A+^VCwwTvH*K0H!2fP70sl?3h{Zg*6M{}$hK#Hu-mKpuGv4XSPyp<-3= zXU4#BRT*>ioe8#i{3-2JpT5%x=(jK>&_0+GmVBMpt*eRXe z@t*+^?jn&x5|mO1&hMWd<$ZxSyxQAWuI_v$W!iz=wqXxR#k}@q-i;8S#t=6}MC&a- zPh7`?JCRl54u)f|>m;*+d9V8~i(GfCN@nM3M%cEddS-()dTi!QMqGkUU7N584Tj0X z3{GyW>0`$CkPm)wez_|d=5|N-xue2eU#I7St*c2DD#10BA}05JI^($@MQQJzHi~LP z&yjDfIK*(_e=UtVO>7%ES47?r5_+L2XhCL@;Q49;O(ycb;G~0T)b{k3=h)P$liJ;VpdJTjE$$Xw??ascYNuH(`|R7;T8cAxOU*tKBm(8?tU{RmODe!V$dl zymwj^U&@%l4h+Rqt3FXrcMe5J3cW4#ABc&xNpz@jH@Sz2uaJzKa?YAv_Zf6>20B3b z-p#poeGjpE+~Qf!2LL)fl48&kTh<93oxOKv`H`7$vyifT*Nc9%<5_`t^TM`#7I5JN3_vUEP;4GS}zgWpEvb+ob)YM zh12?@WmQ2UD6{NBA*T-G#%*OQzRp&YMH?);cTB&ykL&lC_%u$@i>*UY#+Tm2$f5iN zpB>V&5|i<>A(WkU=e5+h)<`b;Z^5TCFO+E3>tsx=d!8v>7{Z~arFOM&czocC2DEOLjWx>t2_mv(ptBlP5o{H-%{D)H zLVtH7L553vcF3v3jIDDB@!EWW?%Fl*xbGZXC#&3X(O{lmqn9S$9^w>t#$>EiHhK4E zEieLx?kT4!^9&GW7VHHcE%gTGxQO%C#BqFT;VPw|`@^H-;8eMA{Cn~i<1St*}3{_@mSvJy(oILbWqTAq@V5xc#d8=ok@aI=3tHtMA2 zUQtud(J%FH>@$xpaPyvD`)Fh3Ho?WFQ)t|#<_}`yWv(@UduA+|UA)fHr&?F4DY#&b zL@GGU@T9te09b0)gaIPYJtsbZJed0a)%75Z_Uw5{ft3&{DoFHgA&W=Q(>!7MhOYw* zU8;B&Hl}80Yp5P@VX{XvP`FHB^9co}Q8$X}pB|s}=&BRZ3K$ncFjfDMr4+uf$1)>Z z=l|olR6O{zP$c%;&Q;B$PPMm9;!xKGEP;+0xe=D;`=4&heAxuRn5}Z5^#NDR65ka} zpMfN|e~;uZZdC!YYM{|+4Mk%!ZlQ|XIz%5~dp(aOKFlYEFU@Ez7W-u4;2IFpn_N?I z-m|b5yS%}icB=fNjMmrO3s(cPzK)J@%^JE6m&l^7Dy2AdPocKg1O&Zy^+Q6@oV3vQEw8bOsM?Bdd( zHsCJMu_Q#f%j1kqU}ozIa}J*5Iz{iSm$t1mj}0@1p?@=W$RPAg#Lk2^_grdj%(7Yf z%JG*P+OuA%3D%xwhfrRXYbT1$ofCDE_LT6AhIm$Yq#AagWC8$8fo)9s_6iDReM7#9 zVxeDS^%kIDHQuwqB&LzD_8c1?Exm;I6yKy(s7ti0Anz58i~bNTO2x#RM=sC(`DP4X z>^zGsDNmfI*4#~d?(v|5@1*E#AKcKKM9NJWNC(VZb(>qELuv@5Y`nqu#p{b7E_WKGUOGhOBUBoc^>?{AQ}U z@lkz(t>e^Z^_TMbpELBn+dQP#{*4~4p4!m0UT5q}1?3przZ_e6!R?ckz>$4yK4%6L zWP=Bf{ys9;0K9UoT%VV{81T0hSwgRH@4iGYvw@k?J6*R(WMiJqJ4tP?*i-mLe{HGkx; z{yXqSw(u+!c-!ZnIUvvziJdni;111}#khJbm!yJaC+33rdSzNiLQWE5`)IP$$&A%^ z)i%2NbhCV!vOCQZsAo)>UtM1ZWGo{AaQ8w*)S7 zti|*b*W&Q( zOv(VfA>isIPGrEtzpqSv0R}$dFH+hwLiW&PrzGNzC>q0cvq0C}hGKrv_iCSpVpN0|?T1ba-o{Gh}@c z^|@>J3lX$Wlgn`SvwRJ)?vJ0DG6-R#?8m8m(eTj{@ibq2N8K!*uIUQRYYsRUy4E$e zW8<{dQ{!_ZcdGZkSw`~DQq5_C1;EjLL9+3^;s&v_VG=7j$SOofC^NFs?T8+O<>_(ef z^u|q_q`4O}!|b|E;XYp(qBagYy)-AbtMg1cU@K&Xh`U8Mh&=}T)ERvZ-cL@1p7IA` zeT3x$|Ne7Q5Bwed_-yb!nhor>jwQRqB#9_RpW3kMl)U>-#6;C+4>yZS^ z`xvg8_Z^=+A0f^bkC}Js6FEo4v~lvfbxTR{2*6&^y1hO*=%9L-cthISUleBCCvCe~ zvp$elj&%0B^O3*jgGd&4Q*L2eMQ07EB}3B*9@mWYpDT7WiNe_vguAk|qGw2ri+JNE zxBaOG$@&%Ia@oiF;n2qTT)M~b3KzABc&~d^xQ)obHFLy%50J^b)#TD}udv1WQ(gZF{*~oqXYHj<=6LeS8o%n_N1fW4!|MJFDGiy;NAL_my2a=lHF=LW1pYU zA87w%Q=_gzeIKQpqzj`cPbbg5kfZ-O;YinS%Ri%;?$0Jh#iTpW)q z&Tq!$!y{KJ|B}>y_dDQjO0H3!zX-9cj|xVa=()TzkIE=(KKV4FTH9T>QHNd=+S&QT zx8{>|xjGVp>I1f`7I%DBLKj*`S^a2To3!)-eP$x{GizXzE)<#JA|kxG`rilJMMz>( zZ$1cbVdK`jGDl1fj!2vBy4FU+q znLt~XiBGkFl(LI}X6~4pOp4~#c%I#Sx=qAoK$@-6izwDGj$6GvJF-93-=}QS0FSWO z)oW_A_|bQ-^jb(+*D*`1aoJ8hMY6ZkFD)}a(bfxl@Z&jGpH?kS7;w-}T9O$<<`?c> z9URDhnyOkc{yXuYLYa)^kNANUoybgPziA^8b_T+0V;N7ZHKYoT`@O9;boJZ)nSg9v(7Edw@OVbA!1{{1!}Z(48o=l=jJeHIk! z)a<^7=9vA}wh!j1-;Hve*Usu-t!lod$iS%Z0Jlk8ezs?n5Q75sC%~loAcF?dd4AbfhnRZTf@-@ zI01EQaM6oJarfA+C7TsJ+rR!$`g&o9g$k{=-ssn20ZNYa?M;=CmJox_9`HNA(A2J# zg)dr2xaYT$2wF791Ik(`+n*wHx2Dd#q-XRKoC55Cw?E{j(jVDh{*;UsLdf}}roJSWhcW3(eIyga6B5} zDl4%|rnSOvT+>o!&6f@~?=Cm5Kp(8SZ{$Udfz?E$4dXy|zj)MyxCV&(m{}ih)(Rm0 zap`1%{xefifb*C^k6II>=2H7P?Intj&wwXQ_QV z1T*+#*s#T5Rs0q1bH)=xHi#Ls^i{d=rI}s&bIC%| z$#~^$m1{wP$Jduv+^o7Jq|6%0PHg{K*N(F6M-&HZi*TRo_pj8{qJAg@9iZUj9odEO zjAm6OvPn8;{rPp=@9ntKre`(}*|G|E@}&Q?N7fC-c@gv}HUH5GC1x8qAF!V4YE^a2 zNqH^>;!X0jaZj4wUU3`>99en88X>KaFnPG$bFi)Zb$(KHB>lbX9cia~p=P8*O| z!;qA->%$2rQc=+@jJJQpx8U&a1C@U3f!9%tzn#~rfj%ybpG~h|-B*$jkgG|eE^7Px zLIisdOM`-xWEsce8JFyTbc;jy9oPeNL}F<<0|v?8vSU?Hdlb zORy^EQx~kc2=(KN)`cBs{(q#7ndlCDiCGtPl-O^+8gCRnY|o_Kr2liMQrQHgi}*-! z9nrU1QIromC=?2mh@n5^T{nG7F5;%6cFjv7dh6pn7ngV8^>K(Rk%-?MH_QYWdDvPl z*yC7>?aaP@a!a+a`+a*AAZoV34cMC{E56p$%p)C9D=*4kqG!?5#*%cR^BC?vd-ahe zQvN@D&lXkKMf5>oC zn=Z^;`(~LXXFG3!B7irFGyEH;msMrN&ndQDD?gacuN^>d?m!3qPz* z^RnLVxP_MHh*`Xxqn!GMmvkzaID5k3e!0K#q-Eg46&Y(o7HB;${UzW~Q?zh(Hu=7( ze0)#L{K?*vzACVM)_Of?>F~eC*b-l%sb|yT+mEiXxqJ<3lKk!kZL~{@BoM2;iuoG9 zxBljMQ9Z`#EiYif5~z&Z9D6}3>$kA>hEFT(Uaa!$gP?+&I`QwjtM4>6IBnm4cou2k zyeQwxBX*8FcE+0|a%=g1k9$wiK*zS+*&B_^0Y=H+dnKTIvuqX&sw-r(CiA2xR`ADA zpT}407cUmwXnas2GKl#igY}fQWuO5V1bhc`{WYm9D_bHVS&hQKel&ph-XXSLG^{Fo z)dPp*Hundf$h)9k1)j(*v(F+YLhgMgIGJ@E*i5|kWdT*|Dg8h6?!b+gjfHG}Y-^>? ziG94ZARoH_12|-EJwg6`*aFb9&uxZy2KlRARI^)6I<`IWOj;(IP3rOlxk(W_9r$se4R2Fn0n-%5>M-Rt0MkuXM~ z-IA#Lt#MU7H;ZTFP7T;Hrr>gA=QXhHo`PX^3&BpHczaT2BP_8Mhkv__%0Y7L{X>n3 zvMilf3rH@0+&g$Tjp@NG?ovm8=54+H8*HJV7iX&>)g}7nGb4Chr2THLyp3?Z>3)&> z9=TM;@?49*i~yZ_IT`%%C(mmn`wl)(!jnTQ4LWvvmQ^4O*rX;k zHYi_b>L%&kd-_QvEJF1)-R762?R4t)bmz!Ux?7&JRZsd8tMGyNCPu%!Ra3q2I1jIe zKM)P8W)ECS8RdFdD#O^mxXv_(lUM6=^J?O6c1^#bW=4CcM40;{HI48{3DE;&58mX| z@?uw(Gn<5Am850z!^u2{>AZ|Xin-{L&y+U6U6Xx zo-u2EOuqb$LF?3~L)am?amT^H5pphgW@0X-=1Ck_ud815Y=(M&(Fjzt*6&>G=r!ZQ z1X-B2VAE1u^#cTl3Y*KLeyg*(@EfY=>?{s71-w~szL4CSt5bJ3!w$?fm0=-}jXVOv$Aq4nk&Sm}3Cj{V-|nvFsG2T2o)BLj^0%gvG^Ujp#~Q@;#CzN0fB z;|NzuyQ;h*3i1mVD(#HR@ht*)^%s{f9+M2CI^8|ba+_Syy9}#AqdBS)Dp2|va7MDh zj%ZkK)`+QMu@)iOyIH~)h)oA={PO#=Q-&)Q_TI!=N)*&Vg}13whu;Q9s5$KVZYEim zvr#nh|MS8ZiMjgSXg%c6Dde!9L6I#_Q}V%p;3aoIDE<@h{epCI-#*f89ngeN))h0} z9r%5xP_qtN8g#}$s$QQ*b$`B@!A12Rq){%*HY`W#cS&b(DMB00~3_0HP;{+jV zm_aMJhFZO6EV%z!D5EQl@CT5ccUaO@v|BRqFQjeS%)GLAI2@Yp^E`c^U|{nOknNx? zRyWK*SLz^b@xwn8wFA=ARt`P&E(9B3Yx!@q|6eDDk1Fry72ogz;flt6G7gSu(sOrb&P1ZV8eNd{Nza#_xSj!1OkXQ?TIH<*unVqA;{m|BT>x& zq1rrUYQJ;nK)%vL!=;B@X3l>y%Jz(2(R1(yt%A+<}U{}OHXX%T!XL{GcOVN0oM#?frJFjHj12hf! z&k&we+_2bq)Bx6{W8s%0B+~Tdn|C(aQfp_WJ!nKf7KE;?C0f#hCNbsMljpXL@@ptk z-w2u-5cjTQs;@cJ^4Mj~;r|939*n=K>Kcbn_zAepF7KRZ!cj6Wxa{|~c9s`BJ<;sY z0j{GO%sg0w?RafM@Sv2>DvqUl6-6*t&76v!GaQyL zMYK22TvKw|K+Klh#_c)GmO+*5tDtStodPo!jRBj6HoMYXdormCl8}7@mC$-@IeeS~ z5=q=wD)y{12jFwFz>FF@%45=?I_vg9DetD^U9BeG)wD^KiFpM|!-{VccywZUyb>RK z6*_axN7b(cjBC+2)qrxjNMB?%hc~hXz;ZnG50e9WoVe`2Dq~IHH6i+P zwpArUPgSu8laxDR8Q+Php|NC?+3zpsap84!*!I_@Gb(vmtI+u;*pF_JGXmV*^U54wv{Xw@k4aU~&x6@c|A?C~2 z=(!lb2?}-fHjrL{FIexdWg{d3Wc}Pi&z8oPQI**(nGX{QjRQ8R;@*OaJAg)PNQyo0 zd26n0uY=3AgAD$32dEmN`nhwIvGb*AzV1K$T>V}Vmat9sk%~^*3-hCIOVj(ybe|7S zR2&-!){L_k#9y-?1Z^;olrP(UR8Qa^?f4}V=qm#Z912D1 zTMxM!W^_7<5Q}$x|C=Fkj}WCpc8~yKb}*Gd@&}qISuNnAgl6of^hQD;mqYiE;U8@o zHUI)cSyx_*PLpmn&%Fe64a1HkWJl3W-we+XZfXN)PVAIAwS)K|lIp} z3TKdKIRin$wlh@q146jC*`aWIl21yR!0Zv{hiMX6l6^+s@?-!~h;HbeG~=#~Q#3wf zu6dWgY#uifJuVFJwZn)oMwSfB3g4akpk6C?4s%QA&yIJ5W&F*#`CmPB--bM-HIOen zYt1;5Y7!6?;S!4N=JPtrZmnKFYBPLh${m&d>YVy^cOY%ND_&XH_h%Ys{*L4XIZ)KF z1ycX=50hb0T5;R!xOzWz8;EwZKnK3t-|KG%-2bkC0FAz!oO96j*_ZJXVn6M5ZaQ2w zBA&K~l+Z~L9q+oXxN7)L&lT`idnMRE@BG1=&w%@`RXrlyM~S~3ce0wy&_jKBZob%w zYyCB#s_1mikY5+ak`OZl+k)=xx*>%zvlO z1+w=q4_-)1p79}+0zZAk{t45d5b!@33nTTIyIhRTgrz9hE9m?l9!ZTq>kIW@)_oS^ zgF=4EQrSez?$*V%*3WM8;p3Di_u8T&`TCdlza=-2F5O{Hm$>Y;EK!!5>{C0`lklVS zm{@?jwDg4!yjJkK#eDs8rSMWd_}~Yduw)tgclU?SXeNT^dIDF5W`=88R?@tIyZvSn zmxBDJE=UYuG;XGYmW|TVy4K<8qk58SIo4ys8iStxubd85-e0fxPY6KC{FZOnmOY@q z1W6tU+T@32c8pd)c@7eThgKMyg!QkRB|c9 zO^=s(+ZdQ1U`&s~c`3!Yv21kp> zL+;-A>(MC9VrsiZJhzRJ64{RUV22 zv&N?bg#Br`X*7awB7c97w(DF6hq^HOYGPQVXXqql!$U&Z%P6t`l|Dz4$VGzyZCfE} zZ0jupu`4aduWyJR=vWuU9j;tMQEV=m#AUR+NF9HV7d?KJtsKUIotapE;^}vPtlYaj zz)kw9^;$T;Xbzncy5#i^taQ-bsch*z6g%mA7dkS7z$P-4%v8niJQCx!+KUJP-a8jB zCb~LI3YXc1qS7QLlof*SQ4BytS$HUB-yT-E=XKAkm(sy@P zTl->mPOFO47pXalDfNE0qfbNw@viF~Q?h?FX6i@sf+uVWs~d^;auG}PB2aNAhQ$li zem7EQ;gXtf@~L#>Hk({K=Er-lSxBpsl5mx{=i8R+q~*J|8<9&-rIUBgyuz!Z*ED%A zNaqR*tLC{$Ur9H;x-W5uswQ==m-1}iw`J3~A8t7;1qsoQ#;|thCMdbdI^P$)c2&WwCe&|HIZ!QgYD98|`LD~;k$ zRSunWbb3{R>@o;}- z7H^t|zez`D?zgQXXRVd~k&}1BGrh3C79n)$!(YYQc_*flNArlHdSpR8n#Epc#`TPe z*=SWB(3GqRoov-7d2r}Kh@T?oNUHU$IZI#Depoc~SE-8MD;yOToFqS;3@+(HSvaz1 zn_V8bk8x$0_Eed_jt)e#h8o5xSY-+~gk2RmON(I+2d+eIOm8_XNL;0KeQvk)GOyWl zxYya$S3ZkzAMvhB)3~OmmunhCuS)f%H(qiuv_vgo7<2R(h_5hX`z=hX`+cD&KWs1X zRWBnsB03SKjhaj^auAyv>j26#WdQ(I6Jj`9~s) zGep8RsB2wm-ReZB&h6AyYzp7h9ev=ZF`T&`#s}K;CxTW4PhN*}*x1qOHWA3L;ve}%mzixL?#Q(TDQ%mtFFfi^r*K8q z#e2&IX(f!J0ukuQ;8js*es8q1OT(>S5ciP@>-;8)JheTb8eHb^w5r~@p1yk$Va;66 zQR88;2 z1#r_L%3MNx*1nKOOuC*l{^jbZs%NtP(}`cps=-zfo&8p5E5|t@?)?(FTu!ueYMRiP z3rn8A&@4O5tj5cG&;hU@1y)-wDQ*m0`=crvaUq0R^ayFCUGuLPQ0-9VEdLu1wo?(v zFL|MDiDFove%z|n7^KlumtbGz_K%;tPlH8d&-eF7D-8}Aiv3E{XY+>n&rW|U{{&-f z?-|9}38b^5uBrx$w`ObG8L~k)Wq}(*t#MKMamsIP$D<(}y4!`OrRXr|Q|sj%io@T> zdfUG2*9Gj3IrB(HO)vRz4I>r>t+2o2G1MM{#==Cvt#XYkxaMSD0WmJRQYlMM~gD}>lan# z5Y6gtCg%#AXR4SQEH+T_dfs*N)_!dhA3WxwOavjW{=Z%azvG~jYY2b1&6(}lPKJ1oz>YbmktaEu6ad;#8ki(ePa~bFm|p;krJY&q#`?QAwjh67U8 zBTCaO(Vbb~Qs2qDY!#*xRwE|yW;2`7a2ktA(;TxrdO0J%*yb`7>6c31i+g zi;t=wY9|&+sbRiJ#g6=@c}Wz*cCoY}wF4Y*obsIulZDc3d5!sr^EeF+-bL7BW0}16 z9;@ltIjn_*tu{tibmzkum;jTr!CJsnoxzIZx(S<@syokq?v>s_-pPMi;Hu|6kMeG9 z@|p5)2Eyygh}TFYO|&RP#DMB1a3 zg(N^hoe9lzQGI#!nkQ%jxQkF5?2X&?k0beCA^zz%_ifh(4^b)4drLBYZVcbKCSe8 zpa5c{Gj}=BG{7s>bI!@)s6y*;{RX)py@eNiAkTAg43P$zFon88;P5>;JxSSjFdFkB z2B_~-q1EXoJKQOev&1V z@gWs9c6|B!>#wUc@}K0;&%LdO0~$w))B{i3?NGu_6Z52#e3QV=7}7H-gx}q{JatXG zT@PXSZuzg&Be4{IFRJFm&+>y+wn_O&nj-$uOd4M!kTUag?JbrK<0^4NHJ09~s zV@hcQ&?k}e_Q?Y~FiM_$8MuxA{E(p+EuFqaA?~g1tyw@?S}nsAqrJrci?wcI-$7T* zJ_wb-8%36jo`~*{=yodNcf=YQcsyD=DUTz_IvWw->84GGgW(#HDPn(b@P7eaZ?R&b zNAERiABoSfK^}`4aW#4Qf-*knNozsAoLVux~St}6xp9q~UV)#9# zv5BZ62P2VGkIs(VJ^v=wzcBNrZY<@;#&$^jdJ)q;7vj&H);6^vNENOD z4|WoQdZ$I1X7V*0%e|OhP?E~Cgb=kFT>Qt{kaDwaDuF-Qd&RG5O0-kb8VP2=_UPgJZ{|P;3nFj)@z3QFST7?nQxtM<*y1Kvq<@^9`5F!%O15g&y`oPt&dP}_S2ZVuih^feN;H1&|oql zqjRWIuHY?$Q`_!&k5c~-$15wYo<7hmP|DlhKm2C4CT;11q8{>YK#xHAyfiv#fmuoN zpW6KxwqPD1?<%FkkboB6dT3lhA$!uIThpcl+QrQWZz zm_Qd~{8F^K;Q=n6)~6J1QyIUsT;*=yQCAoF^RRG$&wYh6O({3hU(LiajNr=en<_dKa;5b=`vSrc!W=yNT9xMHRzSsvdzHlz294< z%_j%l(S6P?Nmr4mg?hJ7y`A#| z>?^-|`sYfS`pNz@RHmicvVvhd*jyEJtprtQ%Anm7!xcN*xs!BIuJdd~Pn!O-M{Udm zV4Yc>IeY!TS0UezdT#{8Z^Z;uyr_&bzimD{NWALO{bEx!%6h;0x|M{ITYx9zR(G{a z5ySajRzZ3#r$4j461@G*$5y8{`frYsL!Lo&jm;n1PiK$?ze1tj7UetQGwlz7mG_?A zsr3idg^M>bD%pPT5&ABm>8b(yx@K3(+mSuc8D;k-3jY=Z@BLvk9dDR8AX^IRc*X{Q+d4w?t(Al^ zmRdk}@nUt~hsaZ2?A#0mFg8OPqMHNGNKpT;TqG~yU`eHIE-WrAutl|e-<9_5)ly!yD&QaqH)M#j`C;S$5xx4wf( z?8ej0uiAVX7c&Q+@48g?X6(1xb};uj&F(0;2%Tg*=9Yp8^YLzbgP#LP`w>l2tug4{ zY1cgIr;;;JAlTn;-FAZ_u1<3f`3R(9{k$kT8e4+@nz))ujK!KyRhz|TL}%c7XUqFH zHo~C^`d5#|HNPZn*RBa&qLU9TA`$clJuSt1uOW5)I``cBF{OpZDql^P*Abazh8qTqZDtc+2sauD}5@HV6M0-S9c&Zln7fGo|@%s;}hn z8kg6>vyg7h)BWi5W3F){j7D?s0Y|`WO$Z$33!}e-*a1E`eNxABivcE2#EQ4Jd&?c| zx^NLuyCsa@1B#P0dw$`_3ib~WPNYQJaSg8u{XmgVg$i9h4kPYXo_bjHq72)vPGA35cug^6fwAe_2XpmtOQu9tMe`>0TC7I<_}yQkwCg}CS4Gu9 zo7LD&;Xuc?xApDfcUrJ#`-VSk#{e>mOVDayM9)g~FAw(AvJ`hh3JC1dye)pKvOn}9 z^=46M2<8Kxv5D(U(C1AU!KXerkmeUzy=2r)W%g2+xi0j0ko?Ny$T&967}L~H4)0s{VEPs>2Anp#xTov&(`|ZJ zYYM<_-wa9=+cUOHtzLNs3_0-KItT23)ACS4J+C9-$MmQtm1I-En|kWr5?vEOIzm~) z?AVdQsrX5R;}#&*1gN~~m~6_Uyp?e1YcaM7NWHlJ@#5KUv{BrfSZE806p>pPH_wo{ zt&!>vI0}ysh#Pv|v?W}?y%q$B$~nBHc)%pKyT3WMIVN2hE)SB~<9Qc}LX&%7PR>$c zRUcB%RlKe{GFrJsA}6qBrPle0Qk?=A-&Qo}BRCHbJhwFJbK|HuDY3PyM!DtEK?^X9 zEwa_P$*Hb!rsnd!u2MQE&H9w`Q;PxVCq}MVvHkkenfTK&vXK$W>Ax|iLk%z-dL{1?1$wV3_c zb3EDxKKFB?+#gOOMLU(ckkbzgt(=szl&tO#(B(_G+Rd;7I*_&X&rn zV{^WZ@c6^iz16+dkC0g@hQNC(4w1IO4}&wM^dy_#cS?BVDymP@OyU&G!|k`EyUdPq z@c$}Y%caKYaE{zm%}UH3!Q4jDnOx)O+%=*ct+lHK^Tw>*R6VG;pMp~yn6|e{>17)O zEcBka`FdU}DgKF|lU!rnb8jf0B~xe>fB+Gr69l|5?mnl+P3Eo& zSMM9szTDp`DB7FnG}|_W`J0@+K$MiLQ)g9Dk3{v!IhLT}o6E-%UY=oCNJ;SE5JG#BjKR7zWn1`#yU7yvb93AhOP#s;XZk~ShVN8p?KrQN*Cq=d z4BH9Aec^B8-^kV?pB3d3$+RJ?yUKn|kZkDe))D;3|7hp0ve(p)%i}shS7CWCeDSRF z86vIOsSByO<-V)EK+V8S*1Qw10eAyTB?%^Pv@;kT&=tRK5bC(+bl|3Jq^^Jv5=|pJ zV4etZaR&S|*_T7i)RNk6iW#ex8Cqx}KtW3?KryV4(R#Qo>`1U_^rYA$f`vZowyddO zQ^OSx3>63kOu2B+bLv*uuMvyi(7$>%&^jYFh4;FY3i`AU2lw;023SEts*qW>s&J98 z5VzG-@C?rc^tM=R*s3IJlTfwV%JJcDjmRiVsBhD4|A>*-JDSMd$h)519`R=n_c literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner14.png b/docs/images/LogMiner/LogMiner14.png new file mode 100644 index 0000000000000000000000000000000000000000..ff7501cda650b328bc67050a248822437d7f4ccd GIT binary patch literal 33540 zcmZU)XH=8R*Z+MyC>%vVMM0&+0z^bW1?eOzDk4HWqC})brHPQxdy0yPNK=Y{0iwVm zD!un0(xjKrTYyj!dO`w8NWwpyyFB-AJ#Sdo0%oq6xn}P@^Zo1{e$U)k?7+zbfBf-> zn91!M_y73g&rrd$-(SLlpM(PHoj?9K^@qugYYzhUEKh9ZV@w~edsd%0oY8ykM)=Fa zXB10ri>CJMuQfXNo-z5+E>rrSb5mz)wL5IT=nX@?m{qvZ8u%H9hQ2f znA4GyY^4#5>0*ca*KJQ2*R6}zd91UWVB99L9n(67X^W2|D;!I&_8`xD)-u~@I4$6;pTs&za@&p_5nB-JM3v{3i# zmckqu%zn>z&ZWA{JWZw%r}1$>!DZV7sI6qoU~w6IP6eF>GRP)b;OIYqoJ_fy4S;mk7Gpmhg2)&R%OQJg@O+*U-CH7_!G=+0K17|A~r^;s_X0UptiWWP8ZvL zsO>^^7oMsf%#N??4e=&|Tfqjjd;{P#6oj7Y#8Jm)Z{iH%{5;#)!#cS^i$cikm;gMO zSjQsb&4*^gB0$sU^<5O-TS#zn7B(0n-REUUFVyR@x~M{&Ol4q*I!aQTO-|OS@fM$M zt#b)hB`5n?a;z4!qaOk|iz>C#Gf)^Rx@;zHp9%yPCmQBFBi`Cr-O&ET`?xkW7_-K! zZTUH;AUwH|wq^62m?m1dFKAF=U(;!hNo&h#wy97{nlsaJX4Gx{fjE87+UO)>U+);} z*GDs;4vm(U9cAIPh~=l6wzx1xOGiyGjE5pS_`6*8ad29Kmhbs)U?m3_$o!7?x~RK; zNDT=R0p@2O20hlJF=W%t2yOU!9|sj3$1GT)0za#(INo((HkJQ%VTW-EUD2#2n>2aK zCTPaguL+`;wy0l|;&vnW&#D%#ybxIC(#1OE6qyE_$f28ro7HB>+&Dv*UR;oRF8zgT zE=0drRTA7z>&ZaghLtGe4}#fH_N)?UJl>jVHy(ubugb>fW3TXM!H-Zgv0o$PFV<$? ztqGe2!c`Yw7kTUqwO41P#d8h<3@03vF~Pu zb=c@p_;or{u_lPYO{%amSF`!RU6y2)c9TyN<0hWKlY;Q$FU|@!O8CrwZ1h3@4Ydbhn+D)q2U_f!xzziV=?MF=A$ETNzX}V-X{)O)GUw(AC_u zK$KmPqoyI3En&X%78ecUBoKWY8px`U5urHIG%1Y5n#-@?ycwM+sLVpBV#4fAEs$h?m-s%FKsLDP?F|+>}-`QDFvK3Wv?|LsZZ5AreUMbLWp?C~LptVH8$Y5eSK#i!vS7u7t(_cq{=7Q^G{K$M2A9coJ&LQ} zF~%^Ajk}Ta8ZmY|oFPChcTOD~L@@$XmF^M4tmM>1RhLoMNSs{KD5d!-cFi*;Sk{)^ z1_hNF8+)IAw7roaPG=PPZdH>|#EIk0Eon4NyY*yI>Cy{m^=P47jnBK08P&?gh4*#K zse9S(7hLN=6;w0#pz)YldG3^mJNzP4o<}hT@TdYC3PsMw1q^nzfHn(_A;F`>va+7< z`ArhFLxjyz?MhBZ%1`eGnMscZ81)Wx)U9^2G~Y02Alt6Al()MAp4=fnP$%64K1SPa zu4wuB@9u89v>vm=iWw6gKI3)`Gu$BW$AIlHVa7ZBwH-8LcNb7d{1s9(pDFk?xzc3&c)q!+wB(v2u_M{i)QpWU7m=8$%HB9G`*L1`qS1NUSKhWY9QS-k$;9*#KkJ*GJcPx0*4I-QgU)@&U6uiLad(s=4g! zI!<=3I-^Aby&jUWhFS~8ny+-_FWGHX8N&phLmNiPb8Fb15&>6Cj|}0Ki%*n1%>q8! zR!W)UEFZ#Dj6kPZS#rz4=fP8ZB!-Xd;ZxOlnahElu6D&tGt|t7pIiCJITd3PyUV|f zfSM8lR7B~@rta<;?Gt`r=647@NTJPVR`ikn3z_kUd0q*f<<5d>xA-YaBVWZ$>3#UN zL5_fNXXYb{X5X6cx95kHT3{6~#>(q>u6)mdvo*K#%Z!DnU)1$|&Y6B9YeNWv?c>hc zeE~K%AI;1*9<{(yD>7+`wQ*C%Ao@r0@InTG)2vPK^r>C4b>AS{1Ga~?qMI!+$enEx z_%aT?<&W74NDIig`l)V%T@2nX&V^8sf>>5gRrz|we`?JKuMZCZztE5%Fmv7E9BLtd zb}KTft=BI=S)EI^M6He0x^EpHwx7}wp3jdN`wRX=%NzT-Y2?{ZiO8vsEJvXZzOe>hrTuXp6f5#*lrNRIYh$KdN5y zE3bxc_D^t3K7+zxksVU63h=BAw!TSox$8-bugws{!#uICTT#Lv8- ziFKETT@MKRhLe3Gdx|`SnY{i^I?N;tpf|B-da^WCHQXTCh-d4S+$A_v4gF87wEy{Q z;N|L1&z}1=7<9525p7n*oLkL6PB=R@P1FnFX^brBEMUD~{Vb z7(IEWA3;BI)8uR5C&K#tD{C7)S){@np4@|cDwExb@GG{j?kVVreT|{F~XUuM|6zk>QM5vfSFR))DN!lp9y? zC)w5LE?TzCKaYE2`sb6g#t@j-NDjg%#J**oQ<{&0Z;QXj(XtjU#RPUZ8NFVrR`U(- zN?k)P*vl=9#s}})j7_X@*cN^-t`mM3hH_QhLo+*3&Hf}IS>;%GBJpVKP~PZU|G|vy z)#Ww*>RK7d%ak%Gf?GQ#2+lEpO5SPx?MEooZZjfM=s^t#KikEEt9TU_)XdL@1S@XRj{EF7ZQ73&s zJtIUhRI%3s(TsX_4`Wi*A*m?=?$jm}nk*{K`3PN{EXhGx8 z)3#NLt`D-6$3Qf+Je}8G{X3{k&i0$rWzpF|az2FmQ+1SvH4_kX`*S@gqz5i)qZ2LPi}A5+L5&wZ`D65CO2)#ohO1^O|* z+k``Tds9*_#ebPtLvM}6KD6(NIsvY|?$lE5=SrHj4k*B0H+kw)4mnar(1`zF;t;7i zCVmk;dmne_LX^rj0Va7C>GARq6mfIH4<$H|j2_WNy{}svdtbXOg2yR_#(IUvLQb&A zpG3iSn^9x&+CYUtr0>~46SDdQ{#tkv$m87yith3CfjuE94Xvl zw3)-eNpT0s7=^#B?HK9O@WLUW5(@ks!j!B=_G-s&4@R7*19^UE)GKkY4@<0fDgcs# zYgTr@4VOG1Whe7eto_W|!_RMe{zP0VKRoj>lC_RRNTF?AEY6*veePE(K%7@yrG4Ly z?hrc=h{edQ>Z4? zw*y&e4#<$X%}C5d(X2>i0{A4G5ri_e15`fc@dU?^PJC-HWx$yS42myH2aD*F!{jmpCs(>yamqz@h&L$JB1e$GK_LeIJD0| z!kB%cT1?>UZulj6USOX40Z2q0Qh$HZ_o(@NzdO;2d<3qSH5I0W^tu{<@MJY2lG-FP zb0r%27Q1pi@xj)e$7jxuyv30(oW_rdoyTr?3+ojRJo-q&)kc5IRvn zVMxvGM{PS3Hx7tTs`hdBsq2;_;eRnkbQJ#5iq%b;CBlUg&Q*Rd2p>BoPyY}m>wYzi zn_T6lMfAsvHmKgn7p&QZLYKH!98zZ}Yg{{&uX(%7S%&+&S)~D#6a&;>gufsCk*XN_ zoJCi(VkT>v#&h4bn0?*bu9|q`x0BkXJH}eXMBISs>#Wr0vDd>x2t=ser8}qGmU$O% z#Wt0V)w!`PF2??ax&9V{kfbv7dR1;9n{QnHnj#A*N(Fy;Lrm(}dvCi5hy68+SL1z$ zBP6KtNAKv=pdY|iSo?|yj8qvcqOoLPJka-UievLHkQunC2JF6%#*5Emz zP_A3xXTERWvs%{u$>apk*ReQ_R~G(PB}ECMosfVP!QtsyvxgokQJJP-j7IUsWFtwZ zL(~crQ=dQCaGKeqJCq+_i6b#aianiOpN+G&U-mZMq8RU6EEbE4$>C!GRJ=Oo#ZY`kw~8)Lv3C)Bz3w;j z&Y+HueZXz>qE|I#@LjE2Mz*P!!u^T7k)MiHoEB1E!-B7&eEuA9N$YqQJYV74r5N+_ zOP~Je?>d!64Xg4CmxG(Pa2+*m@@4TgeABVMtOYiNTXX4=uEk1MQRcCaRu+%QxXL_&Bi9=zdD=VKezvRNDBPDd+6w5RvviI z%vdGDp?7?^L+n^rT$+_%gMOW7L>S-*rga4gznwpnVW&(_1|WF-D5GzH(rVauj#EBF zxUZ97twXYSD};wlyM{p{mMPBKg@~%ixUt6uv8p&x*5*LWnaHTwI>4>?qa2!~T(iZm ztmXCIB{CwOeWtY+d}NrT2Vg?xkg4G>TH_nyu@61>1J&67U|3X(gJe=}QCW%#zcIOT z{z@fZ3NuR9HRZ0CXI><^n%=u!L0EK*S)13>B~QJawftKKX<46$)=WU)4nB{|^y1(< zVs2}%M=6g}bM27r*(=o_Y78PWi4ZK=Y`!WPLBVHFnUvbt~&I9 z-_q-$w?o-yWWw-fxHQBKg@cGyu1@`}bT=|$qdTwjyV#2rZaCDA=WteEi+n^V$jx9m zsH8!JpRjT5%wF}wWy!R|%MusI8Yx3>0fOtQwRP!G6a;$$S2OB>yC7pXRLPt%Hu(Z; zADvn-=C}qAaop*Bxcb#&E4hB0GxrceZx;q!nw8iX(EZ@td2@ zE;aNmc)H)Bw07piao6I#QH17jHEi8b8}-f7bbH%cvq{ktH%6ku1F!fl!iED37Gf9p z=}U#~oW^IG<|b{D7VSSCd!4`3f=nO#x)1QXZh6kt7qx2n86EbvwH>mlaBuzDJ$=eC zBVN?6=6M?$@Z*3K@2bdzeA&j5P0w*^H&-H-h`v>EudW6&h*Zy?ckTs1L9Ny?>O}pi~9E&Nf0QCR*Q&yr1w12(XS{1G#4vE zbJFL}#)*Q*WZ{eV%PNBuL^r~Y$El<0uHxYOj3iysaL!CvKBS#oWC)5jw;m_ZhW>tnuCA)5w%xEr~GYqW}J_+ero;f55VhE$I(}3v*Gp1!LgIgr!ebglWq0iHLSG9 z5U%>S@5=L!yYK%Ras%_ig}yu&H`H3|zDXF_jasI%LMRIzX?i8a`AMHj`Sfl1+B=(j zc3il0}1jEl|go<`KoEw=c$v2>}52_!IEc?|OW{INDiKYyd zAm*!je=NVf`|D4R`7w9BzScya`kSz^!;k=GKgwpX z5OB9{{X=)m_w2H_qOmju3vQMQfdUfp%vdzLf%*_FxVTOJ@5Sv)o%ZuRh+>zogY2bW z1}9wkNVSBv&nFVHz-)3<3Z z8r;gM(bZ)K`%>@QFtnL1b3ebWmw4LbFaKCM#z-EE_!Zt+7^OocOB$dQ2C9fozJk&}L!IwI6Vppe@W#DvmiP}8S3KjaR zSb~BDbkKN&wd)O zd((2q(+!p)%f6NO7~{l4i+__LPV|)0@%fuARe^H^^h1ASFB0Vzhf`u^jOhGR$WRTT?8H@feTzgF+Vwkh9dm*n|ze17+%9SBXl`N+n zGeZl6YKyibG|j37cXu;{<)k~ws!sqd6{mFcpAd)Ds5HXxR zasgR3WwB;?K^~kxjny#Z{Z8tHb!wUxaP0!H_!sBAZFW?`&b?ZNE5QS_7eodmFc+@@ z`ysLsWuw*ix8dcB+9Aj1`Ju;$^tOMe;)HC&%~X4u}k z7it)gVWx|uV*SJu;@rtr%o{LTb{?xw(y%6I4w)^V8*~VbCFSWhW=2zIZkZ7N9T>cLr4cF6B9}_ubR>AZ)VJdOXPX2f50TCV~$PXv^@q(vTcP|hBWlaA*fEz9q07JnIA;rKUu9h> z|M?$}v}aEs2f8gYMa{xNfXf%@2g%NHY7_#eYqP!Zn5{^B_l;KX-Tv;XZ$@W9JjM>>vGPx2JX|Dy>Q za;bnMczFoV=f73fe=n{74o^E23^~DFT|&={|3_aN$Aq@crqP#&Az}Z0$Nk^gpe+Ij zMJu*z=-9FUXoABZA9g>m6<3ee`_H}v)58A3;SZv>e8HaoXUPQ1%6=~xwFmgbGQQ#e zPJVY{b;<^?kcPuxPjH^1opdAEC-AfJ!`I_74HJITV zK28YkLS6m`Q;xbzl5GBh@`lhWLuyAedhy-4ds^>~x;R!@CNmnaM!gLpL~FypAWFKv zC5v(DkE;fCWzfH7soAnNOtZ@N%nQo1cg1aK@1X`kza-N)&Tw+&mkKPpz0JSs6@^z|qcxxDje%*#XHgX?>3c+xC$$%YJ2GEddA|3Rv zN=qq6FO^5jZx6S~8w#Z4iL~v>ErRiDB>3RLq+q=FEP(x+d;{Xl-Ie#VHrk*B zdZ_SnZw#6dx*i~q?PT1wRt0JSpq#vg+1ZNWa|DtipY}DtMwwbYcGi>SC7xB84niD2NJ%Ged*soh3AE1|0frK zZXlpR!-EHc0L*X3a<&_vhiZEC64Js9W=|Vqw#zZWjFbg~1iY;xv(S$>9L&$QnLFPCS{M@F%3Mb_UiH-Vl_q(#%@ZmMx)i<5CZ{TQN6%=%a9dh7 zQkO&I8DHR&b^-tbP80F3AYbbY@<=jveY&;4Oe*t{eF_ zZ@_Kk6-}zqE1$Ew1DZ9;n7<5&^2njeE?+HhNpZ+i)Kj&@-|6k@lu{6AS7;uUu|<;$ z?P}-=`36BNuCVG{Bd?PjhK8Ix^GTVS1#8v6F4+5}g}l&q?HbD6^plRTvj%HdFwYPm zxF&2v0D-sS?q1Q2;cplVuopC%FAys@)8MaSs6-+3`Gx5eTes(Qh}8%B}1o z8Lg_Seeq^kF;v>?5IdRu$<`#!V2hB$so!*~Gmpav>!Ir7v*4}DZaDm+jxa|A%&PC* zO__*-e3&N*_7VobTdLzP_3qLzv(?8#RfDvbuq+|EM#`yWo(Fp(fWO?$U#1k}c+0&Q z-VjC(#UbJ*=RXAc@3JhcDUQ>RFJs$_*JC07VjKRt#Eu6R)6})A_`|!%>q0%Fb9L2UlMV2~--Hy!@Fu(?DxtK~gC`8J| zkFvAtSaRi%ja$nC9!D}KWbe1ZYE6|z=CAntYfb|eS z&B(P~d{n^v+po`d;$$`oxmShg!kaa&(XZilwQm@1HYF!@4mR>O1&~k*J74?eFIdg| z8}{_!mbqh$B;cHwnp13H^gCu^9e1v!Vp;yRL+&0o?s^z_e12|M@K(KMioenr^a?@1 z8Vz0wGu5M^rbS^oY0f^SdABu(th;vc8F)2fcJB6QyMGmoE=;yj8O! zSg8W)4+vF--LYe|0?MzzaPSyC#Hp6?Up!U|7-V;2`VQTpPgdA>6prx5lFfa8)d^T$ zt*T|Ls+@bFB3K5GEX}rs$N4{l)=i7K2Pgz;9M`VujVg=EnvKNxCS~?KXgmVFf%_wZ5vwaMiQs zdF5AkHT1GQU~E1S%?SasLRL!L#GWIrgzc=Z@exFOPrqtoe~#(7iN;=|`3%90uPnKL zXJQH^5Oj6CzKD^KC;_diQ_LM83jH#9PA%!N&R6&-<*c1xAwB6cs}G*eIeWx`J9tQ2 z2#Ki|wJVtB&qDYP);lcY`CZ6cHhXX8^JLx5b{)x~pStA`W9OPDl;hwx(@W_Xl3R5p zvLjzTt2>@!CktACB{6ReZp@^0;@?K5nM#yrP}a3E6Fw)1<0ni2+&|sZ8(2r zZW5HWr1=~@r7h7}yz9wEHUsFL%7L#GF5nunhw_l2M)|3|_#7{YPx<@N`N5+r;{Fv4 zS{T@_09uuY;VrqtEru)&#H(1v39tcTYHs_%J5`s(?;KEc&bv3DODVmWawEcC>kqvD z+{jBQm2x3rZ1l+^xGIZ+`mGnhD(DU2p@gDrr_JG)ahi`C6+;&%#xHLwwmU$)$e%m1mrBF^+)Rp)@y;?!O@P$x6f7sncM*ZTcik_&DyQv+d-+%&{c*7SBa;)I$&Ws8@-Z~ zxea5Jn`QPBBMGy*h26)^gu8CtaNe+?JLU$p=ugrtwEwg)c9ltCLV`D1KO2R~90=N> ztM7PO2j|$cXjU;fkip9p0)o%9w$_I((t^?hTur^`QrpyEMa=SBWe_+;szwq(Ga17cEOg}1YA=WEkH{%ji`kwV7a z+Ou%1wB8f^`b8!@$0j(u!$W8v7C^cJq^c)>-sO#{*|#Cfvhk@CjKQoICmN@xs@ zfqUHlwAc4T$*rUzJMuU)&wm1R^HO52s)L)_eD+ZL{47rpBE(2$PhIb7355HLMch^( z_lWt%@VNha$tmnofI_5JI5)Gq6X$SLjaNT-wWMwE@a2H18pUbv{o~%a?eSdIjWKZk zSGRX44Qf`;=uLMx7Nn~~bU_VDX&)X0?q8VAu4zE=bYPcyY;I$GxAP%|ldNWK`H5vG z?Xj6kmgbP`YDe5`$1(2~*M~YjxW8{7ocL1Nxv|-kwo^4!yZT6tZPb%e9Gyh!RZK*T zC_do^2y!#ZSJ;C>nxMy$<9D^TYQD$+Oc46gT~4A|#tLfrbN54$=xB@tGQL}S`1dB|dIKE_W=$NQ&>`2IWlqbztv z08WppiKa*t6o?E{69u5lz2^1w49$1u)7L(9JoUCJav%idUI_Q;+k7$TLV7%%RX@I* zAS#RUkeXO$5Xp^*+KY*_r1mznj&oFjrB$_4B)d?qP6?40&IUw_2i0o%t^2AxHR`qG z|ExaN^J(x9pt|@giBd?Vo^_OwBm|5U&!-2!5IuLID~7se#~(@7*r|)UNwY*+`u~>1 zp&eczMizpxFI%|Um;}SwPU+ynz-L`aB5DyGVpX$JGvnHo&QxQQue>p2>sw}<{92fU zkB)w&2!tO1yx`7*g54x*-^q{2^kn}hvXU~!~biX<{$DfreU=qx53Go=Y_tY#E z8R`z~m9TO{%A)UkhaB@yvnMX*Uw!3xmlu+b-m}reUEMT)ZJ(B0>>k)|UDnjGf~}nW;PUig8mS>UU&=Z^GFLI_ z#jo(;WbFOMM~<}H`F-UBe zo~g6=t*(unH^&4ve*T%rb)S4GACRA@7{gA^qgHzrxh6}9H%PiiTZ=48RLz|Ng?NYj z0OTlA59Xo!0+T_Cp}ue5jj*z~`xsIx)kSmJH&56;`C9D~|MPsY@AZ*|b9f|I_RIsEo? zv2==7j(4pTd1*I049=t8b2aR}6-Juf9w;5No;w%Tu)T3XZv^KveHCC~zA(-?ao0y^_I>}1PT=D4 z)T=O`)ahWwJxcA;5428`MAF$766O+GhrP6vNupDpH(!>dtHK%4Ljc zPs$bP>6$yM(AGf!hdb2~^dwq8RNA%Eo`2h5x5GN!bo!KaPYbi(r zB$sS8v8fUB4UvX*SqZ>l%u4kpY`e5_c56>_dWbEZ_p;ypNQYv6Os8ngVyC)0y^F(n zbpN*#?mO%69D_G2+)|~E%!^1rc32&tX>QGGMj3}XdqJCKnUEsulDD7LC*z}R3DQ8d zA&~X!^={C*{;99g1r(E%3ooS1PUJ_|)r%ntu5`3M0tOtLDSZbmlDbyzEn9s0QIA53 z`EQWGf%@*p#pnm4#!36n*iSR3zh6D}xP;(_O zwGAk*@WvxCFWm0f39%1HxXZXbM%<_&=(=3&D{a^){%eKDDJJHt8i>D)Wl?l@0i zD^`BAF?n~P#(TNrN_dR!`4bxD)nW|`9_MJqdxmW&#ovg|AAPc)$Wgpwd@`^NilOr^ z=&Ixo7diGN^Cee0v~ShgPGx8xoUd3*&b1yC#oTi4^5@(UE|#(Nx}f|(V!_yd@CZqn zJuL=sGv~C%*yiAC6TVgt%?-*dodI7X$rb^7GrPB<1Gv4`uHTDF!=v36Na7g3`OY1P z{pmr2Vl$BIMQz?s@0mGq@re28h%?@ueD%K@-OYV2Uamt)IpyJTvNe^u%(Mrejy_JD z6|z%|d#JtBnsjz>ulKm#leFY*O74Evto+0aCI8sY@NRcpU&6-kaIWtjS8H^^%9NYD z^;v~CawWOWosM&iFwtMm?y5lV?>`CcUn4zlO&okYd*!}}bq~kx5bkBcy$qL%`c3!; z+{@~R6qy?l26B{u)OcSzQv2|8r?(fB8_4~LYd+RL<#7rgK0R{}Sn45a{r6;xpkJwp5HQM-2>4=n_Q(xRW zj4?m{*%uwzL`tDo4c4r7-%Z-+43-JAsDc5o!6;0K+MZZ$!qKj#-q(60ucR@z?Yyu^ z-|(1IfbrW?AA78<$Dh$jS^d};hDd2P{jO7~MetDBW9>WK8m+CMNOi8qBW8052u%@1 zmo%60hx>=&QE!ISpE!okGt2@pn;;$1rJj^y4R)n>);mOL9c4w`a*f}yH@wZ3KMTd) z3)evDcT5 z@~Q1soRTZoU!Mm~N0eTbH8UT? z8Q_r@4i#2>ND&TH`+VhM>S8hS+vHTHe%iq14kb0q(wf zGh)&yWwd^%3TA1vq_pg8>eP9dzEZc{ z=4r(&pTYo#K`~2VY|mSTKC4K2i=BWo!c2}8R6-$1@v{>2|x*X!3M)6FmgK#XI#H{$5{ zq;7ABIS-*fe&HqW=01)qFwp&WQF07X=`+V>xwQrBM#$9Hy?F9ABEhUhe9L!9AR5t( z{XLQAge97sxGQX*O!vI~iEl~Nj0pA4#PAs5JQi0kLcP?t3+lDVxNF;{0@E6=DeEek z|5p9*L5sGp+z_dgL&R{L2tFaC8%Er~cl#s&2OR8L&2o$j+W3@Vz6H_A_2 zE)g4g@=(}twn#FvtO510RzSlybSt{1&9~Yz_a)o-ddx*RbG(mqkLD>8-`>_r zt$!E|*gYo-RO^{U{+2@9-n&HB0Z^B)>AOmW?!4h$>5n{M- z-Wf>nkuQ>#NUA+$gZe`ngzDtAYA|hIIATKe9bL&f2}S{4UY~%5hEpc2k>NPjZkY(y zB9A_couhi!Ot~1iR?~1dyZh@#AGs_FYuARknV(<;010%rZj-rydZQ}K6i)vY>c08@ z!25x1o%DWG?DA}KQ34RPsIwoUHu7(eWJ86-=vM7^U~5XVky9U}xRyRA5pYtBmmc{v zXd`NMq|9Y0(@XEC16eG+lOCV!rNS=>Dzju0mowezvV9uV)|bF(YEn>aGB=^VjV0ci zs)YI6;Sw~G%qU)LO`Q617FocQAi2~4OM_?dT$8!w*R9PmH?VWc8grIh7p7Y-9Y111 zn$^v8?u*+4`9-|MyqnynV#`xNwch1iFqTMb=fBXejGJ;PGtcPt_ndL(81p|wjs7K{ zUYwdnxmIG8H{lVs-mCs_Si-F9pLnAyQtsk|6?&)A!X^BFH-C*%o?sD+KWzY$q8^s% zJqjGSDFq*G8)8_B5KH^i%u$?pu?i#J5jh7}W@Qli0YnX16bEvlL6Yw3Fa@~SzHg*n zvX&E{Et^93V4noZ4b-=4(AoqZ%AmiS+N?3>q-$O8mQyrP-xkJ5H{-DrCl{YoS&lY+ zht%Q|pcAxT&^$3gb&|Y%j#!+f?mJ^$^u@;&i?sT!8`vQ8*a+=+`IjGA|AM4asQZF{ zb#wZ^*R+sA-PumyL8wQ9j0n>zjmZiAC#?IH{cXR|q_T9mzAJNNtd=;n)vsjMiFZhC zo1MI}C}`ZLv9wSmi5ifWchJYY9{H_~v}lQkwF^AeEOh)a&VYTW+iH;}+}fWNuKz$> zB(E&JzS|4wiUq>B#p(_FpzQU;12%ELQpBAyn^cTLC@Suk>y5;s%NGBR0Uuv`6Q|W~ zLAnr^iYzu@cF`G%29;?Z^>OR?UJ3S|#I-QwZNPETV{`Y5$yT^I3hH` zFXA7csTg5=&Ejn(!jnx)zK=1EX=TT}JLWu~%g!QL+3}Jt6LD5)gg3w;Nwt6@{2T_W z>(7xHk{kI#22t|vLXHY9J8k^5jw2A&v;Zx5bVnGYsWbP`_%dxDjcm&%%Ap3G0|zwV zxQ7d~SdLR5)4uI4#wn0h!7DFwlE=UVk$ zR6upFo9&1qvzXPnb;Z`Zdp!V!E6aCDM54o+{Oo8oW-&NvN`s5V>2PCM; zHDMIC$2*CgMioF`A-&Of>xcay)HZCfSuQP)k&6slhMln?9ZfSeNtI5CRtCg^_ha7q zYw2-LTlXp%I4m4Ov_0s$N7}A|ELY;k?f7a(HdA)~3(21W6k>Y<(`%v%*QVX@V$?t0i#jIAXA^To)aY4 zeyACzz=ZZ&tbrJ#B1yc#5_M0;&2Gz!c8uL#fyqy*iX};JvZGC^&&_f51&SH>JytJ1 z8$dtHIX&u9QT*Fcd-4en#Ox-du^h7UOrX(t`W*>e6eW!N&M)~I=8sQ>Dc`>iK=Jy8 zDYH|{kwykoTuJfospQKE4f%TX;3lB|Uun(3M^NVgr!Zaq1HRM6l;{ z#y{S;8t z`NFRZhzxPas3gHZ7f=MX#;>;3yDWW(c3Dv1m;`L%`95ZKQ-j0TV1cpwBzmE8aWJ+5 zfNTVVr`?>tw_PNiVlbkUxXRVbBwr|rV}#*A|EwuIX0gmeyxm@?@V}oOlPssp78TN4 zypN>u+?qtq>-vX=li^8T3A??~gdud#K0K_=C+HVME)4la1-X)Txwkm*WXJr?kI5X% z>fT=quz+Qde6r$ZjO8hE}%Fb3Wt@l@W>s2pkvTPtU1Xo?Bqp&vhoc$Uq>BS^8O-^?)o~ zwp&eH*j>Gi;-C*HUNuwiO|>60tUx#L-lcz|Mos;GJ=|)G5g#)0qd&+~|KWU=?czLV zwJp$&-hpkW2NH-;F=`hLbmlfRf@_7f>Z<~^Ay2n9?xlP zkpaP`oGUrY*$xu*19+=$ses#lrNJk`lMhA4f9MO!_)1HV6+M|lm??e!b;0Jhn`rXt7JRojM0Xvdh(?*RT3=;h^TaN=$Hb z0n=GA>ylwosJ5^ML8au~@wVd8duz@#3pLwTDD0_^2Z%Kj3R|Md7bVM%p60l4Vhu9^ zh(*)xMl*IBF{Tw8#romwOg|4H7%`PGpXmMmQv6mUKj!HmbVvd|*MsG(n|2H;XPD+@ zrbCU3K{Ai!fim3}Ut2HH=By1Q)8*1z;?$`Jz)KeL=v?9@PYY*9M!;)9HjJdUW05|) zXNVIIHVte8%;0?HB4&a?Kn>r}2;dm)E~g}07IInAoHLt|&p7}E^lAIn=Q3HAF!H`$ z@m{hZk0}(#6lB^+V}gE~NPvCMg>`qGwy7dp`G(agcrfh;u26~SZ8G!veM0 zdjap<^bmr%II&23Q{6w!jnlkrxg*jK+pOaZtXO>1!7GVA=zS zzaCSX6S1)j>INI?xdp_eQTz9_HJwl-?{;p$}4qw-bFyut3ux9_DjJ%=9`wUC-f z80!{hHJZ88wm>>o0?G-0S5?}=JxfX&&RAAik&bd)+Q(~SEw62sd7MH+wuh|{2dIx% z3igC)*#oV;QMyp>h|CcG(a>G?`zv?u(hc=jtnn*#3y*gw8cICX=4N~5^!?l|BS|5% z!n&C?m;rMwjhlUJXSKC1kRFpO09#A)J4-+$PyZa!;oq$8S^?QF8ox2*ygmZ=J+ev{)>=LE2oy!Gzme49woyyd+FG$) zY!iRAaK}%L$`Ag;%2t0-7v#u0t2?iab2UA*TEdeixh%nWiJi@71M>Klj~IvpW5*9R zstWY$j9rXstX)0QSyohKEJspBct&JkN!gi0%#xh#C<%jbD~cK zBgi^=k@0GA`NccTNwc@yUF0iD_0;q1J7pxX9YCAd6=xX~O&tSD>ApAK8OrtdT8zaf zg<2r!^>91Cb>>eKJJs7B4lKL6k7RtA697;H)aORZN4_PQgyuh=om#AR>wVQ6wwo3q zlPa!L!?|N!O54qWxCAmZKf62&r~pOnPi4oNmHk^v2M;~lUe4+pNa{~4jU#y=*UvSC zLM6&uNRa0FzV~4Kl$4IWL3^`k*XJh}c4pN1&w?XwlHDTJOt`&s4rXDj3zm& zk23QyyxcBnzutO7J{mm)ra9R}Wn5_ph|utSYLlBknEy;1AnzN)k;h0ci%I+L3@k7+ z*m!U!d!FDWVH4ojVN~>`GT`|Ryym1emldfj*L)_;DVZJipivJ)JCQ_zhvjT-H(nc&A4IR}6!L2%~? z%38UCd)_B}s!;&l6$Ux+|EhZTcqae&|G%Onr%I8WDk2G?oJ~5AuvOkjjycWwe3&&9 zAvxv{8e1YuPBG`hoUP6Id_K%^bBJMV!~E9g`}y5|-=CNNxNR?&UDx$`K4168{n~Z5 z8z91&)kNhUwa%uiX97$e!SI-BDa)=BR`)AJ04gLSke?Su!L{ zq1n7#@Z5Fv7N4d%ntO$vtq1A{kOIv*W)=AvAdF`h zGiq^d>h6XNvmPA;br1*&9r6&vI(CAgbQ27b zio&dCap;B&t2*?VK&xS~h^fLAU6u32P`?myYq;aliaf(n-E~mUYY)55L3P@~qpe$h zD9nP$8pAqh7Dq3S4L|Z(pPwERO5zwDGX^osoV0s7TKwXz zJiSAyLKf7^Nd(i4vq^Evnvx`BJ+7n-Z56btF2xv7K9IEtvHaC%c%5bNY?P+gU{Iv5 zTfjpN)yma#$Qda?cl>|A5~y&V26>U^bRRysXT1nu(2y(%py$Q!mehmuL7LLX6JH+H z7i^d%gvJlL*y`TCtvGpE2J&ZA&!|{LLp=Ei zGUSThZc8rdCY>@x_=(KFheIVnwM4kuK;t&z2Y`|r+HR;BdrVI%eqWqtIdxTaM&Cxq!dVlHjydowWPOl zU8Fica62~!lP`88y=k@z9&XhMa8yMO@bbW_Re=Q{%F^QhymI#2fwKGLxF_W}Yub64 zsd7|3zANAf$4F$p>p)5Y=vB~!d1ut>ctQ6tXhHKad@rY@I{zT)3uqN-EGIgj#ki#9 z6c^=FPcmC0B%5lrDY_H#MGP=dsNDUrwS`AL5iGvnfImgLvWdgt@Z|qa40iK>Cx#Z5mJy+v^@MDsQ>YSgheF8^Zdzm|zL;{T!HPp2KMz{{ zr7@Z2R_)hjF05%0A};u$WSWeSKq7D)X&Czgtf&UN=#gzJGnjTU(#MC}mfP{C_#3s# zFN2TrdY*+;T=s+wqksyuOMrBXyGOjYQOcUqmcs7Lme4p{X>0#myI`?eFp;L-Ns3ec z3RO{(MhRwya;6G>+yK@xS6q{o{X3P!U3uGzcB!DV~W zhe9m79gbg;d5Z16#@jC#Ar~%b3cboO*u|6LP=aguS{c`_GFA&pNM}rs`v2(Yi>e@D z=X53DUT+UAs~dN56D3|0Upo`g5hM|-FoB$2M3W_Vzz^ZS_CdfTk zkVZ2sIuH*OEI3+RIN9rBSE1_v?S~|?tO6eaveL9r`?Zj7kU&rAG*S9~+7#-)Yg_`Y zwqa^ne$`>3dXK57!`sS1DT9jSL6lz+bcp%pB>>WIBXx=YM&juWN+wJOk%wTya088} zN@_;U?Xp`Q-JAV>8r9>htZ8GH#-N66RoHFk)m3LpW1MWD+_Vn}IWQp$DrZvTd5%Y* z^Rv(yaE5_~5kR-kk!?u6U($jC4Bl;1_ZR2=2cP4YIfO*K2EaiRpxH$bKStsrlxl2B z@jJaEY``Z2Js(`D8N5X{sEIb6VPd|r9-Nn4+XXZUc|FkuoD#}!mi&y7hy6{A?WwTQ@o+Omvwk_N>6t@7y2@&}<3PpTfL&QExMQRI41i zrl4x$eHTze<;`SnuX)E~14IK%yKytvyApF=Uoj7ySsYwFkOcR&NzE52ftE5!=136V zBA6QVS}Rvs^)UHwBkyCD96|w?uSYXj5$3;*GXPXWB&=%u73++^3jJZEk26>0Cojlk z*igqu7Po)Uud7goMjqA9DZtpWSt1?dwfu|Kew8E| z)P_feiM>t~DbpmyiFBbV5=Kgm=Rr~$8;Qyw&gI;p#z}YWnLaY~f0nF}Eru;|D z!+T0_-6+V)6-U9)+P-8S33Mf!rYKK;61=vxXw7oFzW_9T7!ecSsBpN*lOUOKIjup; zR5cw|NlY*I6AFW%3^ghqscKTNctOqdIDfbA$|6)E-UPa!-^&d`45l&b`ORedak=+X zB?zCEi510dOS@+5Ir7l;PWI)5#evp@#V{~ghGtJhDCG>^?<5uT6s(w2I)(Ng@%Ff+ zG%7HvnSdqne{k_+Wf8&a zMup% zWfwUyVdLP+#w5VxYgNID(5Mygi~9OqS8vKYHjO!=YHv^vU)B8c`f%H^(5VXx`}HfD zRq3FPW5y+{GKL&ib?p1B!=kxhN^QY>R&$P5quZSEWDIG8DFu! zaf_(EO1=Tz-hVuJcNG2m{YTb8hi?t**IzeM*^xZoR%DruqmdDkkp?ZKn351DI8D@n zi>%H5(wvnpLb@+iOs z#ZcTIzsvvkpY~<#iM)t)K#OT(m(B6AY}8>Vnt>!R>8Pxt_2bP3CY?T|mY$>Zp6Nm2 zm^@ZEWU5gb$uL@{^~^7g3$7)ScY`(xi+K2z3>!Z*SX7XVW&hrmjuMEWtZ!%L6#~a6&w>STNygw`L;O4HX z9K)ZRpZ^1x8QdexK93y2$`7 zq5zb9!;Z$zaJ&6~^>fDo(~=O#QB1c3zG`L>-B94jBR_*x1@-gKB9{Pf|KY~n6$})( z?^LJfalw>D3M3=Jc9TVv+SI?#bV9jfXvg2|>DGwlh~p|3k#ruvM0Ee0Je$@|Mz5R0 z@$Ar)B52Fp^wUO^=K7mS*jC8^fO)cc`u;of`pKD&QLz;$)+7ca!S~p8|6kCf6j%!f z;fv(r5^DV)OcEsu+?k>tY-PZl*gn%n1c6sMDYcGdGD_67cHak0XPSOt1t)WB_^CW! z;bC%@81FSCWH9u_h!#8rGXffUy75LjtAZwe>&=p)@I=nxztZ_n?XyUPI#W#~*!|V{ z$e}X259)`s z6Tj3ijfWCq>&AYE66$^idm5lPtz)Zdz#^{$`v^CkZ8vVp1>yKOMy|`t{H5A#-a~Z} zCUz0~_28V)e<*j-;74pIH7c|^vD>*(f%Xsig%?)KEzcF{PKcD)8yzwGITOp_Wv##v zEWd#%rIWL`BtfQ6?YU03)Nxv$?a7tUmsf>2-nW-eDoAPwGrsdTIH^0h6lyc!YZ|<$ z8Mw)(nQ+YZvsGiC37N*6DVov=x zy*yM%TUsWw%+tWlL0Rl{vSC#0HE}Vg?KsVZcdrk48YRq8B`#bVU6fns$6# z1^C9_$S6PAI({#fy#a`?NQ+f^$D4*!uXl=>o3ZAvKX>>)EHW>-qUVYH+MHy^|6uMh zTE2!bwe{#HF-oLKD5}u92>GL~o0@p+a?Wu~yrZnjG}`WO-(W3y zJFOAcs@_yK3xf>hh*S`0g0hx#79Uf0c#E(_ximU6ykNV-gJ`^14f*q0w<{D-*Z)BO zc$YV!>STe82GhiKT;BOyVbgyqKVlDV!z?s3nEFz-%!8^c6ccxn@gP)lk!S2|sYW|mpHOe#)fyh5b zu|DUVV|LlAq%8>N;FhIg9fOZ_$ziixjgQ2Hr+3GFh12sIlJWn{D^&sd_v*6R*0v-i zZT?4bR>@#`tTrb`ZmoTdxmUxcHv8utEg-33Cw?>UH*U1)%agTxdNI**1;%MXb;CYO zuUYJIz|&T|L?M(T2TKEAZvt#l%>3Q%ZIgxBy~-+g$*6y_m&qope~Pug>En9-4^=pF z%96{zEig-Q2fy$eLGWGXwY30_gjl|x{P4O9#l8J6SW^`+D0ba9%gsB$#ZXQYf>V<* z|A`VkwfCr`R~_m;Nz}WtqaecjJ#ed?@!m8i~V_ZsULB)4i=U4D3JB-()g`YE@CKx zG&*FaIxu~b*SKL!1hQw*U(xe#DH8dOMMg_#o4OtiSTY~&qn-M`KyjHoq|n)9g|*j` zxp2_Ak+#U)e?q%-Wp%%XbqBdgKSHmL*5X24ZnwnRUQ4awhTeBs5*;P%KH-<5bEu@Vs3o?FL|F$F=u0T(!`{zE96|bD@K5;z~ufaDuFi?>nuB z*Jo=_Apva03cg(VMdVCYlDFy;D{lKZH%16Erlg6{*k+GDs=Xf$phZx+rZQX&DtlHR z9j0B)oW1lCuN|{&@Bo$o{{2??&VfHVjW;uG?(<-&+Ry)kV!JoNTu)e|Ua>mts14oF z9?3`csELg|G}f#GmWN%>&@<(OF|~RaWiMKfz== zJf0Nl{?Rm;$%Ivl%i9GhSnojRhH&=lrt;H zyXOL;@08N7I~mT0B2PF)UGd@wn?*E7hM4ekj66}%vst+52=jcEwc&i6ORZyW(+Z?w`!(%(uu&Q{5O)aQS;?1_99&BA{>jzjfud8=I*Isbj^gymNi z2vxlK&40FMdS!1(m0QCP_Kbg5a3o|t(SSy*tG9CV(SnOrq@>F8qU}8O#{5-$f^-r% zt_K^{yJ!KbX)-c3#}umNF{C0nt6g_MlI55g-g;Y~KB{`^yKCQ7f^qcw{(K_eJ(OKs zz{|apWvAEwMQlc#X8C^2{*xf_v1fr>ovxj`N^d=lChg)z4^M&T$}}4-uWttQyCb_F zK9>YZ&fd%e3SdqqT{_3-P@CfZNh$NpTl(Vd2Gd?Hw`5&g?3c;&B{<@3sdB0Q&HtX1 z=$)y3GM6Twt^3>j=>gD=5r?CTr(2^v_qLyyv+gtKdZ8IH%Vji|?5d_u{(bwll;j%I zq`RISZQs-R7yK^G+SZRs`$2N%B(iHjK#l!$$>sN>nq)|Yo2d#aVGFa@lqE z`^V!IR%C5>?etHX?wr*IlX+e(oB;wBJ3GY;dEkQ=hSqK|+BUEK5v9Y?v`$HQ*X0sO z`^Mg-*+Y|F$02&^bWAOS{P|93VfTnubfq{#i*^Zg9uYf=?Jy2oeNd=XWGBnAHO4mf zJ8xO+7B}6I0|_lD>K{bL=Ju(?2q%ztf?tzOk?lMpErxU}z$DZulaeaWewE`Oq2`v( z(JMTF)NStB)E060-TT_?q^5ipMqh!OR9Ui(TflnK4653OG%hI48%@2ctNk_~rO$_1 zc$D0xdS`eBmI;tXi-88%Jf1D79q~Qje!?$W#0Eo)+|yIrHLuW1Ps!`N8I|>^2OCcJ@nMI{571649mfNeiasU)zk@`(P9vph%u_pnsyV!qIdO7d@ zxtUC*z%F&v(Xz_dI9#TWVhIB1dJ$F?M~Y^GX2!Y+qY`$$>|jKdDj|>=G<4f2J?U}) zi+NxI7msiXev|=J6GzZlfwNzCE;hr;zBJ$DjScxi>HS(%bXbLONgK7m>;pxgf=lm` zJ**1XX8x#@Z)16Opb^X8xFn@AgIzf5wX%_2ZFs$YR0qSwRRHyqGCud9j&qfX`ks(|$_jij7|OCX!S9 zYwn1DNtvLAS{cVm>Gi$r2DMt8(Mu<>+XCfV0Y+oeGH@biJhRlT(_NP3n$?;l8oivj`D0l9f7ZL2Jp3!`w4ga%<H8?)95t6F~_3k;G_7Kus81?pBey!@*f6zBKD^~K;xJ}(DT zoPtYEn>luDQ^i>oy|ADlM_~snd{mHsGVYuNG4%E23kvqj> z`2B}@pMDBj(CUQB5uXk0+pOyIFq_pr{c%Sm`-;Dd5;6JGM3>vG@e{&N#}ju4uIr)yj>S$sE<`Hj?7kmNV=lyZ zDMu=yS9PNs_6(*|!PcbEs+eH&PuME6y%SZfNj^)mqJLasDr+jb&uFPVA6?N_yBWdV zOht<}D6NRX*xq}o*Hc}skA&}`)-`3-uktK#tdvB5^gr4-)lt32@-m7~nYh=a@2c_X zWS_>x9lwH|w9ur7r8y!Y_~A1?wtAL8o1^W>@`k!S&@90MNZ3fN4wA!Yj$A$57CIUP zU-oLS+<5+F26kW_Lnz*I{oMtTy(IKxF#qCp;l)W_j!b*3tL-RO?ZNl?HmB^@{94NH zYQ%q=!C$;?o4?~cA$X=$`mA=&%maJTOvyL?0%~=9U>1Tzb4da{0PFt4Ff#Pw13|vV zy5h@I4clAfx^|NrPFvy?M}3bEc6j|-iT60=&X37@2iz#lH~WuE*3wF9v@xv9@E2fp zl4CmzAIOEH@uVkAdhW=%h;v!*|78J?d=E}8Jf0UU^f1}lBO;H+59X3O9(b=A ze|p6`spK1d?v}P9TU!IAc|!m`KcqJf3xKSP`3Xjk@h$khpP#H(SP8!6ape7cTk4!s z79mQ!6}At>9{TzA=Gj-YBi!qkZ&~3+GI?Q@5>(c)ii)ok9*kSvkgo>i|bdq_wNXzAamjMegs3?I!)+B)3&TubS!eXo1b-?=b!L?lYz zwcJy_$dKWDGBgi#*q0Z*1Jw0pkS1r z#(b~go3$Kyqe>)^g9k%0rwUt@`9E*GiltwOO^0IM+b3DX=^-2J=gDf zeL1#Wgv1+1LY3t{U;pij`MNfOn0bvi?=o`@XYMzp&swRLL_MQ_EucgnKAbEHCyrd$ zn}Z#nTk%@lh_0>5c1sUDG_)=)*!*)?qRCfw70WK^)T5^9GtjD++;=Khk!;4v`tEca zYgPKI^Yu3>yvFbR8GcZ5lbCu*AXiOVZ4CQ8*gVaeJftU6so*ZW=`G!)1)4ob^sVdg zxEGDkc0IwF`@WwW21|3J-%sUjRC26hB-RVqKuUs%5_024%L(runro9B~2D-z$nW05qJ7(=3{Bet=_&qX$=JZHk!sCc$EBGUzQp5)XbK54=>?ob#h~uYBCUM0gn})8F{ewlagvx`ltl%;_oVknUGP+vxW@T&F^?df#ExJQZC>g zcTxlWYRRO9Z}uaO%%f45;@`c)4su=KO&E=GEAQE_Sf5FhE43^7Rm6Z`#ylJPtfwz-+wCps9U-(L9a?T)a2mIWna& z_Xmi;Q_L_WFcM3$p6FAL`Ef&2s$UP3(XPI+zJ!+h_}1F;d|JDie356%<+sfh%qE2E z_tfHV5JS}CW@!S2?aJ2rt3arXaXsh&ZP9X;A0YI0ymylXhF!zB5h8r`H#a^B;&$(C z{z}fz{JZO@_@@!lT10PQ+QsBPMs+8TG*Y*yj072wr^EKmr$`^4W*BoH8AlXbFtPqV zTsesM%Ts2ol6b6-av5=$R&+@?9lEzUQMDSU>pgI#Kw09T3OhPUDN!9h?Pz)uZFjqA9(M%_>t;E2&6Edr!kx^HZAJcGTh5 zYI_OUaU15+WsSwXq%dJBk6L;?7`L=mu5C(hf}kvx^ss3byZOpO7{B>VZR9rY@7DKr zaIj$Jk@7*{7tlgnWe0xftv7xgr6uTj?`VHx@=`9P^tIm^g%iCu8e zYrK%h#riINRex95>O61S6YF~(+TIgaitdKabG2YOOjQ|!*Jf~yBd9ym<88{Fw7Wqm zron?FL%}aq1=qgvhwhSiUQ@4GU=~|*e{3qe$1o`a{|B0qC^1P@w{4VN1g=##yg=xK z>jk|t7DwqE#FXx;LHiZk$4VKy?@|ZRBLozJI`G3PsD&ADFcglWpU|vH-eofS0}ZX6 z0oq5$MUA7>Um}_0fU{M5x`)iT{jc!U{snfOgV2p>{do5Z?so+B(~d1?ubdIfjyiwQ zFOfKyhjp`gW@=4h*!Tnv2MrZZ_y^RKrlc*X#BZqJUYj*vMr=GaT`8sBK}~uYYTzXe zl24?szfk&JJDfpn@z%o*7r+M0tdvlDtDZ|g!r{9}vQs9l|6b>(-m5Qcwb$8IvE*jXT03}PSNxGoKztWa%5H|SY!n^^6J z+%X&;jD)y%-@T&ZDt8F3J%mT7Mx6c03${?x$+|n>U#*=W-wb5A>gJwuMzCHX_N568 zSMBT+_M+FwjCxzH@p+I#1m(H~p^xz)0g=H!J%w3(Qs45Nc{~KK0&(_F1bd0w{7*s|M*? z`C87OMHyaf=C3e!MNA}8_~SF=tSp!2f5;Is0`}UBp!4ZL75>rj6RbI~N*T?yt}?R{ zSP&X`cgZz}x#P%u&(^Css*$)Bp@aE2>#e2Gzhl6ljIz7rFG%jkzIVX@s?Db~S!m~b&ak_dDep^TBNBN#8FjfsrNVJ;ft)TRKA{o)KN(XLdJ%H(EN|)o>~GIma`xyZk|ab+(zzx z1ie4k`ysWecYDs);aX6pg0V;@>JZ1FRBf+%6)zY31xdduR8NrIY)k!%D=B;~O!n%6 zIZzyyV_H>rnj&-5wk4+L3XD|NEb&HwogZrn5O-(wYT|PMlHCZS)*REj`{9CC=a2B{ zJg-#i27CeUR0e|LSYHXA={VNJl(Z*w6hE))(5)rcKaSa66|ds`n_tWOw>04U$#rKf z$D?r|S${XvKP-y|u4i-~u1duex6Fmrq=XbUHe(<}-;ME)(yUW8OLzhPbmoU z*gX1+U}sYcVoZ#+C3;~^d5O@+AFAvX^#wF^6xpe1zFG^ws^6Qg`57Pbef;3bfW%2y zlk`zvWOadJc{pHe^$CB-uT5^*jp{Hg>Oi(DcGC+QsC!%lNO?OK5S3n{(5if(xIngR zoxC#mcJ3f*QBHt0t+T-S7{i7)C4}?53aQ#N0iYhjA6~s zZZ7N5F9L8?kNJ+QnWX(?--U;c-GRHxwCl->$s(WSYx5XwGpzytqgZX|k6Syj^jK^z-=@hA}g#OLHnyfM0JNE=%T9(Ns#2$2)ycMFf@*4E4 zb``7-w*c-rpI3c~xwjLMQTM$qI7s(}*wBMIPmb8P=qc1n{58{B$OUuE+xIG_=?gCG z5ZG|kdo@X<9Z6cr`m*5EIBDW*5Eq0p{o*vZV8CYhbZ1jGPe|Zf+NZuBDMVI|o}2Yv z)u^te{Aufsj}+IdfqW70>m57^8ufk3?q?}jQ)oyDTU0elot>O<<9(X$S(P|~pDsVN zZ^ANaS~$7{Qe>dMs_a2tTYTZ?Nv-YJMNp4Y4nM0Ww2KRiVP(rCP>d-%`9(S1ZQS9g zb?DJI<&gcyJ-A)vS=dfu+IsDAtjHl5D;m0Hd1%nn``nU8muCh)+zGe^_MOtx2_3up zTXswbe8w!)(ox^DxjLNlNVcoeDP?mW;tm;;U7EJUkFU)%VJv{iW;8D%giWHiM2T?M z>|2di)1TIyEvLzbABc5UD_y(5wo*;9!Ou9wZa@o%vmFO`ECy#O;V3}OeD;Q#1@GS;yw5*b*LdDgcA%IUd7Zsuf5LHr^SgH2-Ny>V zl$7)K^+j8Lz^J>!pF|#6uC8ypaN7C_UfNn!yyexjMy?Z!J-J2?*NdX!a$^I$WG#&w z{kUH`d0{F3I$azbyXIo1oP$BVrm;aB&P&H1lK>TwnOr|=k&`o278U0x_eBh!T$huMqOOE&QbzW!B5aOWgxl8>ZlVlle|t8fGJ zxu=tsU|r(XD{47Yo0BQNxr?P@veAat>sLzIb|ZOlwCFT@q=&)L)8lhTJn304Bq84Z zANc%21+WzqSo4+3HlkuycW)|8Qkb6>O_j8M6ap*$haeRv z;jbydnVg|e7wXp3t95+%pbECJTS8oonvLgRj75AHoX!5)mFdN5eI+z^K~(pOaK~c| zBb_H!FEn`D^lE&zk9r{S(MsTK@P48)*{Up*!BWHpD{)u)awpxOf4~!DuF~o(Qf6Zc zMpiY4tB%M(RqQi0>>f+t3cc?YFW*=DJM_G!5hk^HYAr#))LmjHZSKHSxrTWl(b28s z`4s$u%GatL(bpfJwz@Yz)Y-WBtl$x2ebv&%clrH_TpJHcw7<~Oha2&?Pnv(nb#y+3 z*V#KmOdyzC<5jtT+k3_<_Cbo}Jr=Cg<97i(Q~vy$ zWj7w(Chrr>r*^Q=Stu6IckEOmv*GiM{HX^mPsD4x?kCFpH={*t{_vKJ~UahS)*KjE8Oc zrLgf8B@Gp96GWo5Bh~tWWNfHTD=v;A#%)@x=gjH&w5BvaS>9osU;k=OY@Vwa5CW{>mI4?GLSr#8XYi#Fa>s zRCms`w64Iib}861#%_Y)+^ud*(p#mbp42C~?Kd?{2G{j~2AlJJlAAm497L~-()PLg+ zV299D5O6eCQP9fg%3CO3!FyG1Ja;*2yQM@fE2S%-OO)v1&1mJ(Vggbn+{>3$j`lc? zMbVZO%HA=XJ7pS8`-`Zc0Z5YJPQLd({bNvf{5VMKX!n}jau@J~pt_*;;R!pF8Q%^b zOFm^L5w9TFjaQBE`^xAu4~T+QwbF!W;j<*xtkNsKzs8QTWT2FU1mZS~wT%5=-W^gm3QyEq;A{j9)V*h1A5r~8q`W$6YprC#x$CT}^~2Y0g8ev?j=#0I z-gIelJ7_uBsG0>7Y6zi+bW|b)JpDgmuf<=;G^%dXm-ZEUsd>tMAhGwFGswCH`({IT zJzz%-AhAwA`NggyXkpG+4r+KVQ;Q~*P`UL>>>}0?V(`)Y#eLQJ_poE#4>3N?4yNyZ zdwU6lgz@wwRHu%Ha1-=QNnF+=kQwPvoTy!|gx6~o*TsbQpgrQ5;IC?(67PJ=GYsE_ z1HE#`PIzOneqRP{OdCvJ6WQ4MKlpB&s2h!@DwcEJtGj49EL~I>I82Oz`3-n(yh>-T z{K8j#KB|^V@cO-;hO?^wJ>8j4XWp|#-5#oa>IRy{cyl~a zso#t;Nw9q4KgM?W9-IFVcDR&gEiw~4#Af%cXnb=55i~Dx3qqcZW?6}|T=?T68Y^tM z@jD3kbaz@W#`IiDZ{^US>25cyPB{*Z$uOb|K2!N~`Sl<6tC=DduX;nmxvYL>Z7add zCLJ}B-LJoW;BnYHT1_d>%%p$$sY}Wih`&=S{5HqEx5}6F zC%3F@S>MNUbBnwdgi(G;i=SN?9iDijQDJ9X5TdIWzh9v4Nb{`cPSwg?WAcu|_ejxg6xYO}v$snQT<~K4%zrQum|Gwf$fme8)+dQv@5c zXps&>URR+bvzO-jy9XkMb7tc!o)y52{zy4|GlV#uUeLT1Y^Oh_vibD8S)JP{z;AZn z?6Y9lx6Mp2X`;NT^M$>j3N%w=!w=@hAHG4Ps=(Y*Rp8&;111(@LBuj@p)eu$rci32 zZMLPx!us0aR1Kv$vor?iUWryA$E-;o@b^~8AZ2c_EuVCE>79RNw#VTP@F{;VlE*kJ z**&Mv(Yt$RlIrhfv8@>7&bTSJaV}X4>R^>4DRhStt0-z4Z(I`Sb*@BS2lqAbGtN;I zWe6qJ#idNZ9YzziI9_bKyvXR@2HY@@22E5n>mVt0dkwc+`F^rUH0o%iJB1AQ$?~1L z5oYw~R=}Npmn1TKk9wF1^v8uR!W#{@_NziV!xilDVt2!u!R3&ZRRM8lgXZT6#*yqT z44x>&EmtEUZpz%SDet?(_>ojox$s|8J>NrKb4lrzM7RBV9JCQ5y`N$VZC$&9 z+wZ_ig$A(fhT^R`ZgcR}y~7VGa{%dfr8_IgPyi7l46XByhpEJ5MifU9h2iV(6N7(W zjH+V4y^vAQA0|)gOV!`z;m)!;d2-T@;IwjrvD>&XC?T9IJQeVJjICN|a1Zz+>)~#s z;@3rg7^l{Rj2ih2=L|9D^o@Fn|7Z;g3Rx}mA}&b*VbM-7YMk{bX4y$9HuT4E^D*F_ zl=T%iEOxuEy=RR#idnbE`Ez>(4Wn)DzEKxY0JvG+CB|9D!7XjQHl9X;BP$vST4jex z_1D00yfu1}aX(6x<&xs~lwgtZKK%}EG(kR|fw<%QP7;s&hIC%sYmxTAa?1?3CD)Jd z`7R~2V!`t(O`8=L-Jw3`Xhw_U^Cf#5Jvrm*!~Hp3zFY{HQhY$^!w2lDvV&J%*q`al zQvX<$BhgJVO87l?6K&WjC4NPimzR|Xb>7H;Rrm)K)lpNc2vI-lb#=%I_uwh^51uo( zx$jFEyI{Ss^h&3?I?*JGT;+2j$DMNUi(h1uc)<&SNtJ0ehergc3a+cZgV?a0usHWP z=*j>Q4ebMMOjw@iluC(`8)zOuu%9p_rl1Ual!O8V!u5ae8d;I>$}r)DAWzuxx5VDX z1PghmMLGC`zqp@rbPAHQOpH^kCXX%PTauo_RDPJLe(uKO*Iomf)(GgGRpRBWt^Y!| zRfO#m$9ZBDmzw#$;~rj|j)yK5*d@6{y??V};s4{m>ZS1uTZmHnQY(Rsn6kKK(gW-= zebyI&v1&Xi$wdDyuUw4fy|iH;)hgM>783|ruWpZV3!AQ-laA>bcnHi^BG@o@XFnC# zDY&yd5UGvTJ;a+ZZ%AZf8%tzzQ9?kfO7SoYyy26hv)u7oBdzVq`!#m%;3an|PiQfY zqdxwaoE4i$k!lS{CsoU%o8&QkXV)W5WnxpI0hcCEa?kpbxO%5jG87!D2F~=41Mfx< zX#dgHQd5B|^&Vk7)aFWJUBq1krMOH(@=~T=-HN0;UK7kH$W;f#%g1s(>MuT_BVeG* z!I@J%4akzXWTmk@_2ymHy#g?PbLfy1*1=)l3^&%Q z^x~cqwZ=H#guB@eiPWk@*a-cQ|7oS;&__@tol8~6h-q-T9-)3>{&vXSOG(H}1~CtW zLgWsD_KX2VC1yTblcS+0K@6ekF*C_gw^I%9T1Q*S%@vMe)rh%qVcw5&yQn*Z`n4V2 zvAzA*vaF0zSr2vlcCleGmNqHPgIcw1f1T5~`u~=*SK#_9K5ahz_{cf?CnNX$#^R-~ zTWusUcy91fWln;K(#hF!a0{m{c8%{_1K92R?~j#E`+yuk(;IS^USRY+>bjqu?e>m{ zk*I$gnkrUgnILSv7p)opWiDdXi(#Rs=>$C_l{5=6m3;=_rU0pX|zgWQYukp#&dwsxfN{gjDZh76eb~Nhv zc$$fwLxTd1e2su}36^(QbyF;*Sa9bJBL$!8eDr=;$o>!|rRq6WdX6gFD8cGABF}q5 z`-Voq>RrPO?1^&?1*eeH_*iYqnn8GxjIPX_s)rZv(^!cU&E}^CuiV8Kw%GwJ&I~-x zwq8c6cy8gU%<*GELzH_L-lcUcc%2n^eeZ|Hhf5puhrfrlIiSsO z*2QkileA2bd&aF?itD!S44^Z71^+;Om?<~YWBdu`PO~gpOj{q|-FDi~!g#d|6@_p- zi7Aa0e`FY+yt0n_AqGq~`i*NSNh7fe>I9Ow8f-no-1*PNA&Ni!viQT7pMwFYT@CDt zi+6i?^uQIEt+C1H%qw&_|1dtmhh(Mrc?2PBd^dh>as2b35{485$YzO6pvFc+INF_7 zKH^t?YAXGOoSrF`eHFm`R<_vQodjP0nwy%jye{cjFSTs-?-RQ^x}fs`gY_X7We6F& zNC#fWj^<`F!HE7>v>$gh@Gj;@+Z>?GC4S)47HdhpO8P+BDy~4tt+&t`@m#&%;+7Wm zA*_<+wwp%N@JOK`7dk8MSdnjZ{)rtghKH;Px&DgM)}WtOEP$PBIjAADDK79@Gj&^} zIgS#U(5=Y2^IUG9N)n|L5YGFKu)QVY`57`KH8PT{GTEAWMXaZDdALfU(r3|MzjMW* zH`EJ3RSFJX_^n}yn0a`nj3Yi|;m{MMUBTQCj**;o!lO7vX-*Dg=-{GH8A}^?xmG-+K(Ze<2i7sa6h!@ z!}R(a&_Z?s7}aoWqC?j?)8|&&&qu~5pZtll ze}@=6UVl_MC*RO3%X)leN^=n#^Dube7(u@xH*ZK2EZ(pXcqKqhr>R&+wW z=WtvGFW#Q?I{nsRF8I$0^`@E&e9C)#>4Q4;{6Q-21i!;iY({D!4g4bRTC-8^W8Rn+ zmDvfYuAd;g`wN4AV)hNvFMp6e+Y%S>K43jy9g*}te`S6=eEnVRlA#8ArmGPPfn{j) z_ipzBkX~R~o~fkinVHgmGk_UT^9X8jA*2?pmm5Q!Z{6Q#bGaIhSXsP0u(2Jn@gchR z@NLyea?bPFUI12*HaFjYa}51AZ#1%)vtCi4y29wsQ%#4yi+`NE9IeP)vl(;6gZrV$ z#|^6wfCz>m+seyOUx;k#wB5{2?B7QkpLo(%O)3c!gQ@QWZ?jYdoMX2F5@Nx>9W%B$ zXEGX39Rg27vRdl`xCi&wwFyJV^9d3A9)h)l>+8&?fn7(Ip*5^#)I%#n)s+i5VPTy0 zr%_&P>%3>&vcA;?gqiR&$C>{#JS14wE){Q08&F4@E!hGcZ2ou5ZTx9ci(vl9s9NFo z|24Ld{^v~3j+`&#{QnK3>d6sL@FWy#i?#j#UL!!^w|YdK!q_kY-}HZv(390>q)qwf SvuP)oUwTh~kBcALg#AC#S1vjL literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner15.png b/docs/images/LogMiner/LogMiner15.png new file mode 100644 index 0000000000000000000000000000000000000000..d2c7615ca15659df0b50fed75cee077b1897553e GIT binary patch literal 31812 zcmZs?2~^Tq`2SzIRc?(=k-1Ql%QTv%mZl=)w3xY2S*f{}Yq{o%3Yc1%V`@o?rb4C3 zlw0mQN}7iIk_&FAxZw&23W)f_=KK4e|NordIUbI{XXD=c-1|JQ_xr}*v@(@EeB$uF zefuQMu3x#mZ{Pl6!T(8?25@9*#4D?tqm=+yR;`*m)xf@DSw~+ zbo0!o>mt{GD?fNxcDwgDF8tDcT8B8$bjmqKnI)4Dy7K z>%>=_M(TE&e^S9UkrCY)YZ+9|8=&CNfLFZBYLt3UiP~M?Mcl@e!+I45ah=)@5sfzK z4Fj`V{8oKB?~Bs-hGl&q>x}ulX%y2G%3U~*35Fi`uLf?H0FC-&*S@^q;J2PnbUT6u z&?GTyJ-PbbtzAADMVIY)o5|hO%fq-)ij|%v19_0$h25;dWcEkg>FPEctj_cgi_Fw5 zCe3BewUi?V4Oq$yw05pX71g8fk~hA6*MBs-WvJ>Ht_AS(S92718pY)UyR9L?w^qd% zM-|^ovPC0aiE3Cv09`DZ16VsQ?NA#- zkL4VFHdf`&P={udawC_d8~x3Fv+5&v%96Ak32uYUb2g=Pui-}^x+V-0v$;DCbD(V3 z5gxhx71o*bD6MyD!ucYt??<2Zvh{G7z17wuE(6k-On3YZGr+@e`-N-_Z0nT!8agGs zto1VTPlVz}hydh*CvzzhKYE<}QyW39IxA08L^&Hw5WmDFpeKBAx+kg|pAZ{AL$V2v zq=6a6e(maEQxxgre%}VXsMtV`_4Z;0^D_XsWR=vw#~qEy=U^W+ZU;a^;e(S z(>h0N5l@u1A3`oJ^LCoR_oT({(<;=DAU?dgwDBbt)6}Go>GY6yC9UDBXY#bbg7YiX-Ye3W9!t##-758JV@_ovQnG_ZaoKiB;5mjX!~HZQ ziTK=}1`=I=aydkDb*pT>b>X|`*R|MQg`Y4_M=@21*2{IAYn@9TSZ~m%AFJgu2RF1a z#xy8}1agiT^L`$(aUK*yMR9>({*sLdXx==OvwpF)%wL}TZFNirgK6Kc%zDv=hA^$1 zOJx=Ps~hlkP%$S{LfE=Bi#_lragll}S$}-CaNN>9Brq5UN$0!NQpl76H2n2=3HTo@ zrk!+CjjTO4kh$Zs13O`i7qK~D1a+NSN%C_MHTW>zceQt_hse%p1MX}`Q?u|Eh((cC z@z}ZGVEV^1VJ2(g^d7yR_}Ql5Qe*1$fVMl00EOhO=6?Z=l~$E(Rb4s;=8xK9Ker`# z&Y%XnXTOP1ppfJKN2H<_(D+$;a0|e9si3%Tlj+J=k=OcR#iGhjGNEwfbMw$jgZt+& zqbW(e`1SOpGqO<^zlIJSt8UC!wwoGRjpVza?(}2_+kFBcd^3|(p18`Jz;dD#$ z7o}HgScU$GfVJr?bKkwkjjZktDJsjh<_dRkXHQ9o|Clc3FBMnPA7r^fv5T?J{5fYZ zVPZ}*j1&;ouH%^TQVUbFKq%h`X&{^qb>APc%Mwb$%WUZ?cxySdRrdi?TX&L73T#PMmV zuo0S%@zSkiLtd*tuQk3Xr3=U-0DTsIrU&JAT;QoUGStVa&x`gS;`eLq^oRP6^Eay? zkJkP9^ZrUs>~8=ir(Nza5VKj1Q(Oxvj?C`;QLr~IfI{2i3sC+nRBD%7yc@yZ>&E82 zu4qO#^5&YW@T|h-y|4WMt&;w*=4KpZhuRF;-GBhW^mwQb+?m^tt`r`+%Jtvf^tbQ* zbIQ>!@f{w_{2)ESYwz)gQ{=@$MU*y`Rm-Dxx1z|j_FXz~_X(Tb2Eq4<0C_S25gS#8 zirg3l+s58#vEAaG(PBmLE~m%8b}E%_?geBwiCOuSeFqp`5+kYS`)fjbJR#epuK5*N zJdeM=%b!-&uOS|iT-$pjeTd2o^Kri0CBC>b``fg-GpBv1GR}DAfjy+RII$DUP|8zZm)z^QF=Da;RT&ewP zYwxGWOUoE2N(BaA9C?3vW=%n)uZ-ZS?^zUG9JR9?*8`u|wZ}Tb>EAPX-h#6%tB@(Z zCv1ScDKY8M=4%vO=&&^X%Ptkmrw&<#mVXbw6&f22?fG)NX;Spk^UTM zs>3$iytu%cSVwLh^2bkyiD@1;rkX)VXD6NjeojnKYUqjn^&Bnj-qmi$0KOpVsyH|M zMf+5)7_t&`^>?n|bb*A$wpOg2V|2g4lgZAe$KJJ<7H&W?-{1y(db^~Iy0bGi<4P{S z@Y_Giz6rI;*iE@r4G(qw<4KKKv~JSK1xi$xw^Ny;>;dFLf~M(t}Z;pb)>>$ z=+8s%!fS3ltJ|z7Zs@8V3Fem*R{AQo1mT_R$>|vCaM~=jl)^^nDc4Cyv76Pvghs*a zG9hzEUxM3SF_b=^AZ&=%8nA@V7B|vlmGRL_MiH;>?=?VA)5c zk#I-0ej}%=>V1peeK93MMTU*&V6oP7Cb8wU|7)IDGEd?LzSH z4l3HjTNAJPQ1T+6ruN(S zhM8tLe8vCRTE7sUA!)Xm77n_MIx=#uuI$PXlTui~-8&``IdAU-&suE3A5?R_=y>X(VivO^I4~Ye6)R zs@)E=y0jpSRsN$|{9{<-_C(L&YK{xf>BdEJxJtnnL4Eo+2XDD*Q4V=`apA$ zyqv3Z)Vq+HGXax7%$(A8-zpOCktyY=R{CaiFTjLMn&;bgzuv{a=obOJ=lSbQz@&#Y zsWgzWw8gX1( z%3HmI-J_%8hdAi^h-yKDPf)GG4C@7p{1`?%Jg*hd0ph`i$}l0NS#|5UmHC9H*N!pk z5C7a+G~VF$#(~R4oC!C`ObM~%_bcN)Qz}!_K|-faU0mQ=?33N+A6k~_Tj{}ghwRJm zQCk~fqa2L%jb#UpAzTU`CkOqV+-z`^nJo{GdDPLt?3~lb&*+E#I3dPcIvLe>72wN) zpWFVqR-=6VUM88{s_!=}v5m>LLx!MCu(Nlu>ox~UO`+~Y^M~~Qe=;S973G5&1Br>X znexM*@l(a<$D6C{%tlUTVwHP{9xeSt&9%;^SC7j4ee3!X{eovq7oHUsRqWiZU7k!# z$W%rSK*nxo9s}uC-gAbN5l<=)2z5VAQ*bC3Ej>Av5l7H`DwkVEapbXUT_k8VJ};}p zookHr+^D!cIyxd{O5;s?(iZM+Ja3U+z?gi1mJe<&X z>iGHXms*RfhK9(U;i#Pg{>$!RCZd?Aj#Y8>4Mx#ELlH{_BoEMvhnB*>yTTxXEMoIR ztzCBLLD1f6-wy z_#Qid_9#WSL&k@b)=lc6rd$+(uj)x@gtbXWlq^W^(xj_0m0X7lN=pXVO{PPMw<1i; zLtEc%B&#(9$lyKQ;AxAFrhZR5*^%=XwxgErVs=g+s#aBK{Xi;^7mj~!sq4N&$-8Vs zYB{YvB+4BL;xB3&jg9GNE?-r3ZK)aCp1WV>Q>Zl(JUPMab>3DVRx3LjiO5HWjyvwq zjhS)#?**RwSN`S6clt>V7t{U>{@O&!R#wtkJJ=BY@7l-TB$R*W1?|JhPA2@m_M>Z` z!$HkUCSOt4rJ_Orvjq#fhV`G+qgzvsX1Qpc>NaY51cY|H@ac;63kl=Iu;KH)in>bX zcMqhUO!qfo_y{)+&$0#}I|E}IqKH}0)3ksrA?)GBT_0hb<3;@aemcmFF%^^^a?uk5 zpp$iq>R=`Xn-O>wFIGXkV;Gy;hWowsdD9K-Q^&v2_4B&L4LiKkVLrURF8wkGyg_jc zlh0aX_TQ>A1c0{#iuRK1%~{a2S?fq&v5ae}No7}FzdKyM4xZjoAd_11zR+)eaUe`; zxHXczJ$W`fL&YPE`>w%jDFtA(LRh#Aa>5Sp`hjVYXTEn?`or^NSLSmz4%tMRe}fO> znBhmk_HFOcC|ZrdOAM6R&0s!AZH(Jzd~g4~UB|p#&8xwuYIJqkU+hcY2uEc0I)1v; ztAFD_&ZVuSuzJ_JrxG4QVwWwSlODx7L%*hK=x+Hz$T7n&cRxh&0r6j zf-Xi6mZka3$vbcIU&~%vE+e(cq`276;7m`RcRe!lljKqCc`Cr~=2WL}XZ^fE4|=Cl zou@SPVuu|R)>GArC4H?09C{T!>k(OT7GxAtQpuLO(O@%!qVrx_>01Y|ese0T-&?pd z9TxgES`u3nHvHUPskydZ_=fQ;;ROU;ZY-GOMDX3Od%X*d<8J=Tpq+*$WAuVZveZ!AX*27v# zUN3{3U>f$Hybww6%QJEc8)|6yS#X{4gkY=m+s(75tN#SCKgEIpF=)g3hwBeFUB*wr zs$)Z=IY&Y#MN+Z>62^n#S2*p^u&;=8jo$n1Z;u$q5!8MhDga;Ul0Ia4@g($q z)teyRv@}_>KE@&X_Ec`Ot`s0|A4Y5PTVR;R>t$VYb&oe`Js$G-;n`1xDj^Fw3yPyB zpUk!4cYhiQ%MiWseI_0+Vrvs=c8!I=Yf zxi>Gq4?2S7meAFv8LL~!^)0=Ll+U%fYj9gVJ)wZLa_im9R6}QkhDP+et2aVps#++% zmvuE=aaGF3;*{h{JrCiao%en|b7h%ItLRHVrr&sVHaW-^{g^kjRGf+$3||JVH>E7 z{rFE;eb1@}OOgKkT%rhW1b~7pi}2OAxig)*e*&uN%m5hirfE&96t9yfejY$4v_5F6 zyQ!M(a6aCW^ggfcd0+hUfzQYjEodYt=Iy5&5sp^`IButL^?)pVVEvDyv!%tITD;&x z`#49h=yn8u_-KCH?Z;!!TH3~`#_=b$my0JbMzFND;Zd)=TE;~f0WY1n0 zGl>GVi%?uCJq8)YH1j*HYK>X4rVFW>=ATDLYnpIU*t0vmBQO0`{mgJ1*^0cgXzzq8 zW5;uw8Dah(-bFL)r-~uWPqU}uSI3e)$&jU#SK9jMLeEAzDct)E+G1EVDv>}v?I?A= zOs%$mLA1*fXXlE4U-N`9TCewUSz4RW@%O0q*hgQk@1{FQ-02K~o7-a(<+l_52zP=h zc?FR7>13JM{!dBoK%$j#^D!AFW4cM2qB`t+!255W>JvD^Y%KMI{U7JNX%5#7Fu`33 z0cknwr#NiYvkKfvQT*)q{*6x&=fC0w)M@h3GNXMosxvN#Xc;v#w)`hGt~d!oPMz-3 zRDe;ipn$T;<4fa%Yi>BnY6JeF8JLXo54J?4sz~V`i~n=H+*Newcuz4ETB(|az*;S2 zVg`a02IQnxesp8>=zSxhcaqhxU&QEF$C_b`#tpsgtoqta1@l*tb-}i^Z^xCbJHA+h z<;9DHLq8=&hyQl|&xK2VUrk->_a~`_SL=FH4@iOyc-En5YCVq>9qRgRWbO7-)p$(#Xzd0DgV48{V6e;bvc0H=#(KnuuFu{+k#D{BQ&q# zYXbuVI3VB7$~5?oDNE=%N9z|w%vODtdYqW2oNuvit@f>Of2|#keKt@rvd*T(v}fww zg~OX&2Qud}=B|rUyBmX>ZKUSy>j!h1)lL1+oFL;Tmgw%@L(@wr?x60ijxr}MaWh8U z&hfkJl!cgn@0O_Kys@toOehjkO>c93ngdnr+pJ6)q)5)(!sy}FJ_{lHH(9)QwFB~n z5fQZkBT z3ABV`*;$%(qmIKu9#`8w(s*_$?hM-ZoUu!N8I<=|5cSMBl}+WF>cL@1&8+%c-C0Dz~Wvp*on;e98`RIi~!4rX7`P}Qj&TVHsg`e z=~Ps?>OS2ZO<#S$?Uz(-3uVr3@}FVz_Xo^c$;*>Qmv2M$Mr&4_m7t|`5mGwhBM5Q3 zm2xuZ9fo-pc2tB#FC7w@Z1V(7+)*p+DTDl%pzZ69En@L6WIMq(45*oTk|HkXB%?+Xw)h4J3YWPK_o)TSv z{DWIiaF&@h#@FiDEP!+duybB6n17{559Hu0DQ)<;xjZ$=XYGMusoV}@cE&zHrFh+2 zqI173Y07)wx94J#3_G$sR0^4f>ugz{OCCgB%~$A@%9ahYfQHiVCP{0C`>y<9y8i86 zl4tu}tA(G93(0C3>W7JzBs(xV>CtYw#JG<)$fVZOP7&h3CP7dTLtqjo8ca{cts%xX zD?hHgmkQe83seyWHJF_J*lYvtQpF?@bFxf-PEBmPY7U zdsYXpbNVQxZKH7Y;UebQYMH@Qu;>XN(Y(Nr7aj(NJOyTChA51I0<%6qomfGN8xO1h z2qblR=n!ZF($$MJ>c60AiNCGm0!{UVQ`@ag;R`XGvN&=g#*KJK%A+pg9Q@{PPFHA4 zU0e`+#kVzclKH95Fv9(7AVc?NT!$aaCs$o@8k!HCfaT-DxG2r?0j3H-3mt5(U~{ zTboV@*-Rg|&5k@8x^`pURkfBI?&!VuRe(xT^k9a~L(d|&ij=_{xQV%cu z??i72ht<}m!S3l0n;_)c<9m=|q?4;lVl;4G`)kT@E#>~+n+t-uvU)TzIzPJ^(iwK3 zqbdYfyLVS_i7f2@`>^GrdEWS70n%xwe(yj?r>l!faY$Rme`g7vtU_MM#wac-eO)-s zf7vSJzk6JS{J#$ycI@KR@#K~2^u)0JHj{@9f7xI5G^a&sGyj~S) z1YE31qjX}t?=T9#h@7?;EvXvimBf zE5h7E2yp_+wR;i1kx2&eIvRN$p>V3Xv*Gy6v8oZx&g3W_MP6!$yt_k|-WAY#IGc%s zAlQ2l^Xz+b@a*EHT>--Hu>=_G-<9iuwN4AZz)GcbSnS(_?$W>DX$wOavZ1TS_>V4g zha2%!-l8cag3-ZE(5&?u7;E}d?rsz_2Z%ViJk_(OANam5reAMa6u?^>Dd6@3l%C|AJsy}k28r!CuJY)DNGhSX?wMc%+{`4H8c53uy{^u%9`_-qNB77HBrxG%zx7chpg zg87*QEwGu4NQ5p%zkzBg4@O3))HNjjR{TuIofdJ={WI;m7LvYMe?v!itgZ*KRz{8Z zuI^MqI$x(;hsNb(4DZ2wOQ2#>umb8-Rnnl?|IA-^L82bF`$!mL1S{X!7$$f5Z|;gG zq|VrW$^B4=kB5dpnK74Mj~K42{G<=^aFH=5UUfmk-i3F`h6|u%lsQNMu#Ky7=>mxH zxGcf5J-_TOGrBuEw`e5SZ-dy?gl6{knUcSs%@gbmKeyFSx zF%R66H1tNt1Owpg8J>Wu3~wF-DxUp(^ry-z{rc%^)h;*wuNOd4&D@Vvv&ZyRFnhX- zSWlO-od541+fc?@Vd?JAp;Zg6DK_y3l%T=H1phS2=1;MmznEz@spLE{tupfTZF?p9 zOH-e+8~%6n%(FKC(yIR786DI1v}y9gh5(CodV+&wXj^J6j1N`(DWcf{j7|9W`q`WB z(Ff-{^RGnrB%G3&6B0bOuN`9;xjL+uP5t3_=xfVy=C>F6yEiQ8Vhv}0Ahmofz=nIG zfG@-26^W@6pMl9ZW$fW3v#hO1bn_={fC7fIw}^>jCnCL998b43fe&Wd*|p~D_Igfj zPzLu3B!ke`T4} z>b#r}2&6R|3uGp;OnjB7%6=f-P+5%}7Eb^0>}J!JD0X%l#_4dzQ@vZ~Q$zffBGx?4 zKuh85W*}1M^v7m#fQJ)I;g>u~Vs5^_;y(W!k3P$u2_F)mcA!$d$&WB(=G7bbWF`bk zoxcKRdwjvqFf+VwIfS>pg z^94w7nj2V9o@13Qh~i0SUrUyq*K+S2=s33M0retty2JEr`{{@bvvzRmqKhbIn% zG;kDs3Es0;%TgBZx88T15`ze?64mwIu}SkiAlLKZK|s%=@{FG|6CK@Lrt3`4T~+Ic z;+T9%&I#!Z8MpG4&uwD*9dwQAl#%~g;g#B&Bz~4T97wO0D_28e&Vf+={XcR6c*f78 zn%tiNplvwA-P^x$A(Y+TUT<(V&`6mDx%yMNK7e$=Q9E|1UBJXq3yJm6D>@TKq_H9eCdzq0S z|EN!nHkZ9kuzgw#g0{A#3ad6SYG%cGuz@#D=NyhW2hW{}dMCx+(e`xeE8TC^KuW+D zx5;C`b9Oz~0{Z`2XHhJ}&IhEaB{MK-e&u#aS{Y_z8}rRAZ4lPV6I7l;HYWU;w3U$$ z4yQ8irWOV2Qw4SDaY&WHxc~`37XA!0oW;&dqMFZL;A=LJ3VW7{xndH40)NV9X*H*s z2%3Xl5-Fp4J%ncoSsY7ttPXKdK?GcTBxGu8bSk^k>YCai^6ctIzo#Em#>~6}K6z_7 z8w;|1NlkmIH&KhvjJ4LZt9DbJSmsdQNA|xh6JHz4a5xgx#?;LY;3cHt&o~>T`>@sXUY273x!XsK9%ca87}h;8cXj&>30xw)ks9$jz!577|tgaHj~nWHp`qFfD^l)Sr-LHQ_kaQ z?2R+y1?oV`=ZPSi8sroSZl=oBo*2->!%_@HQD#vg^&~65e$^tRxc+hl#NnLNSLCGx zrB~2wfx@mq$z%{MF?QFAyXaP#-aN-qY+^LRncC59wo|oEhJA!ybT8AUZtFNk?!b~r z=?mIM(=pb&VY>VubN>9^jO@>0sl_p~bKeRmo?<<+Mq*dP46IcZGIpBQgu_xO*_D6o zC4Ow8#GeLV_y6`Rb5lNmMrzBngqgCYe#Sa0T9^h-=TnGCgrJsNY#U9`=Lff93t_6{ zMOhfw0&Sj{o;uI(^(Qa}KtCQ;8{5GL74&|BHsJn2fv2(+= zF)p7RGrhHN0Uodk8E+XFtZ7ECHFUN_RZ)@~9(gXk3%$qbw5{&v{_SbBe?wVZD*Kse zGFRe+ELL}7|6Mf{CSapG6q=78&c=v(zzGc#qIdLA#o)0w-CnHA=j8hcBnoF^xthZUEa91*cKiM5(3uW59umW(Kbu}z&V5bJl^YT#+dj<6C-#Fx# zROU+2Ldd3&xAm#{@J}kGx?##nCw2t}gqhIsSyg?~d*>GykyJGDk-V^eZC|2PAMK+G zWtIy87=mGt&=w(2=_~fy&vSIccRT+Md$_kahN@Gx2L(m|OB+3beL~*PyoUeY z0o@@$cRszsZN9fDE{vR@E@gR%XM|hPyka+;jF#4CP4t+D>ZYmGdah7q*ISNUGXGL? zoYckO*IGU?v&^H4M0YjCrI-WZf!XrR_KKs5WFPe`@O{d9=SAKFxhP(MUzYiTfLFqg zOUu06AdT#&M|Uqv2l6iDIqHv2|H%<))`ERfDdci3Jhw$&O8WS|6t6r~h&BAO=N*B3 z7^4k6XdY#+^8A9Fs=erMwuCxDGqIrdMKX7&N=mPz!r+_(%y3izjWqfJ4KE$!{6)0b zrk3p2azd7VKzI=!iy7!S9BF5dRoi4pu)hjCE41z~e2}`QcYnp8)MmgUwO7h;q#x_T zA50!~(MQycjwz(S!hcZb^Q^?OsUGTNPkZ7wQz0WYss~!LocrD1pJ~H@kzTG<>A~HB z$wE+Y|6G=rO+-%F9PGzsrP6GhO`H@@Ft5};DF+YOOg-1(zsfJV;OlK6C)A3nZ8Tx; zk`o@b!?BS)&)a{ugIwOG-?pgh?(~ggiM(uqb{|`6^jDY-taT-v)UP=#sld&I5%8c#d*vZq&%)~ON&ituItuN!WNT-*laC{db%dy8| zaS8-E;R|wY7Rtc!Ks}PX+8$A2Yw>Jrr_OrGTRmNhWUB zqHe`xh8?4S2nn9KuQu6n6XOxl*MH= zxCFD@O0~n6>f#e1iq=u(E)1B50kS^FeSS(xhnojL| zYy(FXDInVC_wGUNVY{l!Pk~ASQvTHgvs092d5Gf1_`kq`-=wlvo)Imw>ACVy9cvr5 zNDsJ;>=~tCGoRaOLi_h|?bhaEhT^gHk(sC$5W)JA&I%VhKc`MfgrqwOcsgcW zNwu!6hg-v3nNus);nu)LGMEh@!|@ghXEr__S4T7M_Z=HfM`=}E$UMpO7WF{KVyHEV7-rnnu7Yu&})m<$E2cazd$j} zt7FRjHBbZT(8uY_^K(hTEwMmLV?}zCr}}ZptkR`Oc_OYcqMmxAC)adr zgQ1Ox9BqMe+DV_moYXBAJBCQHmU2ccr6Xz70TNN2=z(Ar@f7p8i0bw-{6d0f zGM3dofS$d?CR!UPv4V(ekI-#cr-0FSbAp^@UDcE>e8y8IQm?u`IGv~ zP>Z&fu0TT^e2%EUdKDV_&C5lH^A=pr{`fgZjB*vrlTB{D7CJMwp-{FTpw#Kd4!$lNv4qdL5!(Bj+9NZ|FkPQ7bB-ft7M%oBd38Q?YvBReGaJ3`&Qp=MsJe2C zQ@(LsG})DVv4eWhDTAVg-Jm$I%re)v=V|QFn{2lMdxXDhyCNln*-m(*PV#cuDOpF+P&OG!LDUQ$bzk$kJLM4Y zRn*@L!U&~B>nL|nDt%?yy0|pxx!R-|^jLucA_?U+u!b*KYsyS&CpJSHmgp<2=Md3h zc3pX~Cw36Z+Tp{mK(9^QGOh`?dfT^PFxDZsLNX)On9op2Z*X0GSvi$)e8+g}`-+RZ z8b7_1hKlOZpHp)#4Z8U}p_r?k9Y8;`9-cg?w(I0+_(LV3k%5U~oWU9K0|s%Racwx* zV$!wRjj&>sVI~a(+8RuhDQPOA?it*#ja^JKc0?o_C2Bdka8>=%1_b4tXZUCaoES6R zyt-_S4P>5lD5WdqL-}k~<$?XlQN?O}A}(u<-Fz1+^~~r+vR-)dbc#rsk-Mrf9PTm~ z+{4vIRvSyplgv~RkvwfDZW7(n5w;nsX3TK~P2X^$TXF8MZN-}MN5Ntd#TR!U@?;QixF4a3y+=_z?bG6 z;Lc(*3d788-eZ`)=tF7^>wh-kz^u>m+2ebh*HWp`@%O_c@6SH}MUU%rm7@y19lZw< zcIg5Do44@4u({Rz1#r&+4Gr+=ZN_MAaAG|n$15t_KKVc)_7@vwRkRH1XO0&CV$GAA zx0Mr+%l2e$OIU41f>tto_O!mC5?QH_-jBuy8!dj_5FJ~bLI@y2vor6zvrp)r4cy1b zXBIngTSDZVOQ=rn|mB`Q@~*z1eJvxh827y}iel*aF^LyPB`|V45Q+-eK&$%!bWCH^NIz zC--n#GfQBhP4r7piYzu5r4$*Hd>&>f@>ajA**0%{AF|jy?0UL3q888ouH}bBP6+`= z7K5wAPVBMh9pxvfpiSR`N${&Z$CVV2Pm9x+-!oNO!QcKqYJKhpTHsK!mY2q^c8PB7 z^$krx?H`U&R`zar$noR$FCg6qU7hfhz0dFyU6Hi~M*Elzu?F{i&$lc*2T z$!cND3y7`qKJ@F!owYE3;vW-CE5$MkvV-5{vJYp+c&g^f2;r`eDHUgm++2I`@B{Rj z=qkI#1c9j^m66^c+y!HpGIj+tG_rcn)k=S@o1kmnpEjBkrBd_@_ErikuMgOQkx;G@ zT8(a4ahES{n3@zcA;xFu@c;!LwMR(Vd;IpJH@C4}{q-;5N93GVNncsbXJT{$6(89ySvzAg~KAj5`89qkb;>tbuVU_R~tRJ9yw z+{fEX^IMca;5v)U=PhFMo=3pjFHL`Gutyu!Oa$o_!MKQvo65O~C2 z=*r95?mTYR=nm3uyib>x)d_}KDW7Q&G~`6+Z-A+@#e*fV!;=SSg4h~L6b0}FQV021 z7PX>9wA6Lp6bdIv3`=yb&85W1WjPis;KOB(06Xhx>CYc0$jOfHx#a!zK+dRYmD@(< zxJ;kW35n|(9l3-L-e?M!LMzHeEwL`lT_C%Bkex%B+z33i8=AT;RBBGoAdi{SL5Vl!3cUe&e%7SZ_{`~b+(t4u<)R&e(o$( z#ukng(%DvqsNCSHs1YSF5(xkB4*N>L<3(1_*`A-jL>f>^>Bd`(h|Sb5!BE^sBFo@| zO!E6hxui>HuBQbYF2zh=zvgsDuzLajRm4Hd?B^N2q)PgIDGSk+_v?GA!Lh}&M^hr3 zZRJzWTwR^U5ml6g6dpc-a(7wb-* zUZmaUJc_;CFY6}aQu<5w$rxDYb+7yAkWOpu%S`Q3F$it2)JF*`)VNd*rsAyT?A8rZ z<&H*a7^k4l{qzsXx6h{GiUClMKXMlKO0BAlcMFa%iuC=WpOcfSo%*_3{&IXl#PH)A zgEJ+~r;IPRkg=W()zj9ZJAcy4rVZZX-&`LqogU=RfLJFU!?mir=@(!USqMfXYfPiS z>bKv;H`DSW=0z5sAJEnAfU$`gCTvx+hWCXU$IXRf3g0htl3Z2+G#QNNMG_%jW1snJ zYiY-{?;18UCJ9P!uiwy^{$a^qG$AL@n|06CJB_?{L#$V9i8nm^k|k;QBi9vd-h3Io z85tj&73-&~5zp9jb6NR(V8J_dTiC`x*JhudPV*>qJ;>;M-DT&F7oN$hTz|r#mY{j} z2^LwO`RQles}1ix&8$vdkY_lwiHk3$3}?}s>^y#Fb}1?AMq7WQkJh*>xeK|Q2YV>Q zlmmV{%zi&-*i-bDbElLx{z@~nHziyUrgaSm>9jZqJn~koVw=F-yd~?TlN4-rn`nQ3 zqan@HY;?t4VzbL(=8XcV?mws3kSN)m#UsjK^?6gI3f|(J1j5b3<4p1{SpHJfNV8Jq z*{TSmuX85mDNGLA_V$KG->V+~?;Fik4l*fK;+!49goRvgUUpb8RO@)*;X3*`arD8V zpO#kv43c+)Ja#vJ)M)F?pmO!MxviHvL_QUCLjC>u{%I z0bA>Zmx|lPvt)kHdRkzC)|WN5XZ!L0k*z8Z+-VniXqY2{%MOq&{eUT? zZZP-vfa7zGNK3L-_OyX}Az%3kQI%)#@Vn&g6luaru9l+gOg7JJ*%tB38v&plzqmcy zz0dB_GjX6F`IN|_OK_mb-ReaVST$@$J>5pyDm#_1$hr6kg>N&>WKf^xXF-+(d2n;I z_3*5jb3NTMyqc{Fma9&R&(tY%ra00p8Vm8Zh?3SO5gpOPF)ByZ^c>Xe8ulsqNhgbR zea5+0&h{RjqDL#zzso*Lb~b2aK<*!rCY=FFjROaeIst9%)=;FyIP*aoCHrFFAg<7r z|2(0Q`?C)By==<8hqHtl+*9|@wl&G&Je!QtUS&-Qit1c(%(PkoXbw022o@;9a0$dw zvtT;gp@OPDug*|j<+8h>>T0?)H68I(3y$oi9E7m0neRSBNh62U2Mju_k*PheHvT7} zJ}Ho>u%2B_e;xk<=*$UWEKDG)4|uWMI`JPxGT%&R)PBTWHZ~661X-x!i9+b^sh8rm}14JP^&Rc^{^yad$pJ$@`T{zQQ?w39O6{`L9nS( zcDJr^yKYP$8gh+r$MMqdZaWDbTlew1r5mx*D2n(b?S=k0^6TPY2&5v{W{Ty30cvvb zXPjygUoAX$c`=#H810w$&TUPk4!1Wv2Xv`(@MEP<{0wPj(-#84?DTYKutCbzErbAp z!QFd%P9lcCS8l$x9u}<*&TZ58f-}Za;hv~s_$g?z|C~pP+Pb|_DJP@&r1Q!*LY@t6 z=V!kGVf}8D&i@XT@#E?H?ICvoTceTaZ76JX9rbpT|Yh+h;#eJ(Fgx#mHNA=X5 zSHoWBDOP+p?F{|gMxmVr7!4%DT`zx}O-N4RkgGy`DM#01)Sv#B)ai=ekPx{xCTS)r z;`H@We%~5R2O_Zq?gf-$HSB?{`ENDGUaQ$3XwwO>w4w0b%Adk`ZjFB z74xNkr0?SIR-x*~7a#9IF=dQ{?&biJG{yf)E`#sY>7ly*n zF3S7tjj!So3LCVh8A6cl(8f4(OqL;pm65$kDic4X#O;zZ~?1eY}M5}kRf5zXq&HFn6n23d!X8+#Uc|Y^$AQ{f8 z?|fx$uMs;Wo$}EwtJ?((bGp{5gz_lIgoSUq9|QEncw&17f%yvD z*{s~zg;%f&6P=1VR0i5kB3mX_rJc#>=sB$-3a*j`AqL`BfUy;pF<+dg7z2KE&Y1@p z(JW6dZfCvnP1sFxq?3oNtKnjQqlXiz$3xHOT>+%RQs$F5lU(k0tp1(zzjc|PL%_ov z2Ui!6k+wNa^n;5!44@El+ufvp_oGK!#m~hW%jk0boy8V&?0aw0&C+wRk*&Iz5)nuL z(|M5Vc_$mR^0&9rp86e1dZm1+_{xY!McN;`Q__4F^MiWwoAqW-djJtMZ`49Cj?@}f zbXmUP;7rD_lNf|E7<9hu&-ft;X@A{fc)x$jFq{-KnNdGq^P%}RE;lT5Y}{j`%rkBP z9mHw$S^ziZN8Q^k(578B5Y-I1@m@V$&xrag_36)spDubi4guvs5jq%GGo+Iw6tDb@ z>^*VdNuw=w_rWByLoCo*)z~F^Ob1I!ZK2Tq7V_GPjJG(qm4*dquQ2`0dd1~ERIIV3 zZvEtU_T90?h)BL&a$9G<+)*HfT4;_Bz3w3mpAynniVsy)4McjyR}C*sD5R&BTnHI9 z#Q>Lr5Jo%67%g%BO{b@7uI|~`5TroL)@*IEIBr-(ZjZz6zxl`FTrBe5nG&;uGD32? z6465)zqu)jIMDV=w`1v%cPFiu57am7IIATW+2yHa`L`&mM?1?1>}*N_G`$>hJXcvyK$v?iRGInZ47Hp9e`3uhWGoGF?s2jT`%5mA41|Gu9;`G?2j1=n@H&g(pm<9UTY zH-v(JHRbvY3 z`s>HBYX%WA9QJG;<1!-_F2=Fz92}}rHsJp2@~mQTdCrtC83l);t^7o*eCp_D3@QIJ zfMZ$z1gqz%fUAE^H!e!+9FILtRHcojya5AAHeE!Y8b!Fx?KCnwSAKx%Ocng zhUeP6Cq7x_MtP^rs7_sw{lF2G_G@z{xCAuwF&fQ};P@b|jXPBVzlCp8!_nSVM6CNqOP{PEiRr$3j@;_hAU8UFBj#>K9J!S4Du)<`Mbt4B5gGb!hm6!_AyUE}>8>tcp$ zm`8oqx;xXM?)M331Ytb!JhLy@78FkEu0H$>GT5L=_IMmskpv{r;p7`f?y$N>hjYE5 zZSh}^thd5vv9AFu-)oElT} z4F>CDUy5YKvzb?7QUj+VDGX3$)XWKv6Hd^YrF}`Z?H!Cdh zdHCjsfCbKQUE&X=Q>J_ZFCA$Vj4#Hh#2&#m{tp;^kF)Na|19an%t#E8x7e7+d=DA2 zJ2=-a2J<@6Iy&vFRDbO@JdG+R5gFm4C$&lK3z7pIp=_u>biAV;!LWF!(~KIsLCSl? z#~;Nx3T|!zNK#EjY>wT!pE%`NsvD*jo*^<{V0nn-%wnasxUXoihKZfd&0RQ4@}OPw z^j2YPS=W8TLso{ad<2b1nyCzr#W8d?X6|< zb>8Z0zHr&o$%rPG6dS#^@Pz}m>ve*VvoE#o8@pC>`QYU@8MU32Q@V(#)zn~PZ~1Si zk1AQK+}6LWBqBZ@A3skpCy2=07+MJQtG8mD%M_N_{6mh3Q4PNm=8|o7wD<9nLBLtW zuEM0MWWZRzW<)<0I<}cJKD+=>#g{&dDchgg#b*)6Glei5^ix4Q&%-rm8n0OMH66Se zR5&i1UBGTnHtIL4fsc09*>fKsWQOE4PHdRXGAM|*X9TPat>OkmRLBuYz~2FK>?Pvi zw*6x{{yJ~p#JTSQq;f}jBc^m$Fl1+Rivzn+x%xG_^U*E549($g>{_DdsNCffgUk4b z%W`O?{wE){a?T#$ZD^LF63ggw;tibjKDlGnQmIFtd#?#Adwx~d#B&e&^eBzf`mR>lY+S3eQ}QCbGUedSezwzuv8C1<_QK3MVlSDi8YOQ7#fIT ztNY8mLZlGBR(GE4OT0tmdHi2%rq7_V4EgV-BA<=n3I zZDhBKWzK9_w~{xTEevG2XOr=^+kR%8)C`YD2LR-?RuGuF`I*2CbSlkia z6Z#Oim^7*NQ*Oi5xZ-a5>D~s07m*MmXD*XhO6HRyRb2ZfagzR7Z_ju8CWFVO`M*jb z(|=BThzgfR&9H5^4JvQ=Kk)q1r)VcCcFSw45c8YMIL|tYS#3qnVl)L_UCEgWkCbA?Vj{VI1k^lFYWZZ^solR!p*foBK&1_KI*=i&%TOUIEFntbOlZS z6>zhRt$ts@)TXh0!g#iQ?20rOPbnv+-Us3Wmsk@z*Gpjc%Oz}B3UEUG0`Zk{WT@3^ zD@@*oYGhZhm?FpHKNrb+{}$a2TkVJ@-phSCyj6&t>2Ym9r#Jt8D`@37)$JVl$AIgi zN1J_<>Ro(`kazi>Y=2gQt?5W^Zq1&Qg*qrA&Qu|`?j9ZHvm*%@-vU+y1V|$ z+Agv-0#O~vx}zj$PqMx=d*}G*dREw}c_^l?{!O)(tx-4*`tL=WU)jau&QE{-LkLzs*Vu z=&ZYg6pv_8M^^s7`nkt8o*ut*9NJl3+Q2P$?!@z;m%(bkQ^O;k#Y{5v%$HI5M(*mQ zP2&LbHvda4qCPF^4$Huwf7JyAU3n0ar(g6Rb?j$87&B)6iohmZ(z(*MA22hZ!uPy} zR5Jbb_Ah8has%Cb$CxI*@uDQR}X1mI9PVplh(4)e$= z#CoN1kY188nj1-Sj-6A=D958 zlzGRIeqOVIti$=ahGSrkM+pMm0=}MlR9e6jN_88wh#B++uPl{evn&@tQ*Jv)K z8!wq8o<;g}%hnJj?DK@?!Tjlp(}4V&<~A_&!xU5zFf?n~0Ui6fOYqTTQLH1GYHQ>B zcF0KSR?v6O|Kc0jCD(l2%>Y-)X5}b6@TTz)M3S?Je1+4sDVywUe9}-FVvmYs@KZBF zx8W0>&R_=HTg(ZjqZso-KDDCQq9${U8b#>OeM%^9(mh~()|x6q=3~sOu*MPFNVZkjc4+Xc|KPncSLoiYv;?^8 z{NAP7%uuRM7UTV_eynG)=0vKDFI55O)BQhe-hbg1vX|g-PagT`CntB*&1J;HLz|Te zUv5N^-#dabn2Chag)|0B31Xzu&;KPtkWwU~4^?gUFVUNH7br~or0mfWuGoA*hasCG z1a91X1^UhTmHk)Mi)2ygoGq)8>YyD)u5t#bMUl%jZT81|f=Aj#{@c$u8^z(~@2qqA z%zrc8ux!*!JZK7m(N&nP?Sg-YqxM~H9vz}X5cQf9&(8i7=|DwH-T&9R!vSf^5VV z(uQL?@0MmZZv{ut@2*X^Ijz}#KU4YN&_O>T8se(UBX4S8SaCgNv{~T8kT{3NAyOR1 zp_7RgLYP&CQ2Z_GboW``|B`HguUC1KDCVw}PQG0>R^H5V(9juJSZU46#|Dkm9A2vR zlr1KNHqv5idMSMk5sf{Hxqmb)j$FP2OYZG5X8qq>i5x0uKR(OzQT()ysC0HE`U#y= zF_Y8vyP}o^ASaQY7mHaCe?0WAXC9+AFlQ)awNE*L&J3enviBXIOt;4A1-gO`Is@ii z8}3Ii?}|^)Pp)Jmx~RMgIjhja#|D|>r?MNU+zOVYYwiPEkVdLjXi6ODjq87J^Y1eq z&6&=3^Ii<-adVl4C`9sdS1zXtTv;v=I$J3!nM-d28hW{Kt&Xkum~U!imCTmpx=R>7 zsZJq+F)6P~g&63#+|^pguo46-{D8W;W+5})rvxnr|FJp1Bzmq<2drZ#hQEIKZkL9A zUp0YPGsZ~g)IOH!Fsa_7L;Lz>AQgSKR$q(m6=tys*JA)8c4@ekRDMCmj$b&aRv|csY zV|Gy1ZpL>2aEvE${gnw{@OM zzF)!qbHV8r73`yKaRY4_*^^|@#Dscd3)t3;fDIxU4^B}yq+c%a@DcDDoSRsAos^bn zU>(AIKSwmx4p|_`5K&4m%C7mc4d&`X<9s{{;k=y29<Y{cRB;W3OW;WtUcGgi zD2D8IzG*wHW(jK;v}&?hVt+K_514*wu>IM3a=uY*eH(f79knH9a)kfTLbX>A07>9O zyCOAL*m6TmF>sT8#Hu z`OouEk9~g&F;vpKw$vh1y`8#2A0C>^(u^3xol-q-hxntg&t$SgL{q;^^{X5G68|z{ za+WR<a7ExbqTM6d~GS__c+U)B!q^Q;a02At_ZY%u27Ejgk=GYXVCnX!_VJ2CB4F-O5yXJV zHc;v>m8m19Lt)x)R^`o>3GLOELvD`RCUbL4fwzmz(s3{Kp=N8X4Z){(N0R85QwY3< zEc50X=e}17;S`#yJ6Ql*H@OEf^#(JPNVEBE)L%9Dz^xI+JuGn_m~@EKxq8H4{q=AR zDVl#PHLTOQupu9B5l8=OoF*mT-Jnr(S^h;##4S^2h zeG}S<4nal}!Kd(UpWxaQ@Zzw?cdHb?kdW)mz`H}k7fFmUi|UbxLCX_822t= z{H$Qp%!w|x`tlDuTPgVCsPeb*#0U3CG!LMqxgxRt_hF&n#+#;k-{3Ye!@Ztt?oe&r zU1QUet_8iwp#-1hmo=sOQ$eXoTH1R`e86-3_nH@z9OQp~8|vx)DlKZEenH&=SH@sD z#UpW=`=TF^V`2OTdV79?b++Y--VH%NPW~ptl80ROhW99@AMv zLn~mttgZywSQA~-aNN*Kf47!6>AO=q%m-&HneZHRk_;UEu3-t@b}x_Hg%^wcojeSV zywVR=C$cIeFB3>Zr!bhr@fYo(|7gYg3zLaDTcIriSN`!Lhvy6hT6LcLuJQB4S>cuk zCo8ZyMXe|+&Ycc1M{XUWO3$O79{oABn|WNw!>V*)nsN8*Gy?b4inNCJKx^90-P2=B zJ*o(oa^?aMt9W0>7J@*P!g4}hU;1>G+p9;ZUtkTIoj#{$*O~kAAv*j+eQ`OJw*_<`l!Wjm8LLSv&6#XfFaE0&?m8sguf&O@E~Lprc?J^R8aW+>9|% z_b`=j%Vr4YKO!Bm)z(nbn@;5i)4T8db>*zcRRHaudqdZ=g`$CCkbfgmxyVHV5ESeB z7W#2j5^~Kc$qLzXwvJR91ero-xA&!|T_FtP{~`dhEXl>r(v8)t6K5k@1TIIF`P5W< zP|oWXgQR|LC5|}YNY+j1>2CU4Cx7Z6o%%<4?ab`}dt)ovR#5+XjF~zYF@ZaWG)tIs z4Bu1;px#rGumS?^cH-YHn<^3^)QFj+@0y(_;8fK`+ zlZ9J;K`x-)>4tHbIXI$~nFhz7ARGf1l+X>&A&2k8P6D5LiE&q%+j5G1pJpqRp3(O& zlIv}wAXur zj|#$_ho$iTqL+JVJzpAVlV5H+uJR?p-54qf?f~()i;a6d)96tP?Y+?Rr!#=GaT3kt zAV*R+d&ZD>s-fn@-mjYHloM7``}1d1#_s)(H0jy-myxPcTc5uqsi9u;+x4|eM-gm^ z=9!G$oI1J6+x+EURjteLU2k5xf?aN?d#vl@ZEUr*nVMCU zXD-ayG&8i)YfwOasw7<@+{nbg3N|!PXu?}YZvW@R zm|;t#JQ9GWV+-rH$)`fNMWe^jcUvCQXgy}-H9o7T{e z2AVasg}(0}$kFm)WnPbG^)UrX3@Brh_H&(XO+G6 zS^luFDrYz$?%kq3y7Je6HoAGucjwsUpTmmkjxCmTv@d`Rv`H2XCrNxs=+p}A1Xyo7 zHV|5sY(SI;)7@+@26cTRy5!dBqgqGf5X|W%%i!uAeM$mUZ>G&Mm-%!M_pK9{-barP zw+}Op)wDbeu$1Urx^JyU%~!NZ!q1RKmzI-8WIO~t|F${*6V5btmbpDF@SEr3Jc?k0 ziS}Jq`qXBrU+et7U?9sve_&X_i>Vp9S)?vaClu%h{t&4lL#7vu{lbrMn8N@I{ohjn zWQxoHnqaye&>`|DSAX^oUjg6j4q|zU$~{y7cKL&0&gJ+nXEPo`|F+p|CCXLM{(Lfb zs<`ux@o}7y4mH))s8wtGGQd$Hz&AYCjq(Lger4Ffh3SK{l`ht8a73WX)hEn9pK}i~ z?PpatZd)@`u(=U|4dJj|x*~w>e3^8!7=NGRqEAYSgkd7`d&9iA$vSzZJH-HtN12%U z!Ed`BU1hX|q?!@4(xk5(Y}8Rd_S=4QKKKP^kDzG^cyCcXne$;wTv?Zxv=Z4yBe3Pg zxMgGmF4ww+vOk<@?MrR7JfwH(t&p#5xyp-b>JDD_YIV{2zI(lP##7zi%nx`vsCaFD zZle1YQG}LC?ep_GUaXX{w(Q7c2M^mVryhd+p*AqTs@}?D$0W#q?+EyU|8APZ(c;W| zexIM&*I{)Y`mUz5CdqW7wNvvh-duNQs9oH*;q%hO*S!~?o8PNoEhYpBZe zu&8Gv?^>X{oqEyGh8klS98aeW&4$Zi?e*Sq2B<9uyjwZ5hPw2IsI(YB3L|t@g7Z|G znMS|N%10<$b>AZ>3h!;4H;=F5?+^LlUG8-Boo0?|KK5lj?H0L$9OWTz`yMJ0vpA+r z-1UxDMwyxi%lP~Lhq{0v}p`m{1e`%hABth&3B|AY{!vkMZFcpj{%d zzHM9x-}&l15zWpps=Msc*tu=g&fnZN>!&0)>Q!0jxGG`y22y<(9|%6Ny_T5tgeoI& z)psqJyf1^D?BqM~6A5u{vzxZs*7;zUZj_ewp7su(nDCwsc(=gkur(onRQi0#zCtu9$pxw5p|wYMkfuM;X{!0Ya4x}Nogc9vE3*s%O(G;bYDV-$=1p;X|1W|4{Z{fCQtSWJEA0jhQQ05q@tE&~H zIjbPeud3gOPCX=J+s?I2Z1~e_eCC#ZV3ZAMCU!pulZeJjA6}}MJKr5 z!KY-a@G<_3wedbP9IzVs-4noP{7u#x>->{X=GYvDKQ=q52xwJr>}DvrhyIlG5duZb z;*zSf4o2X9fdLE5`wp$-tmerLYx+%@8v=jeGTW0Z*K?sVmbgi8T6H!0Hr+f3d%Q66 z=$+potpt|6*~!B3CT6dJ+5Z{_;T9`yEi(VQlQxhya-j3N8MtDvWlt`a$J+|~nujZ1g+T%CycML28g^=&%8oozUm z31XvNvKO?by7lNo;Lvvjn}|E~2gly?HY${E9B;!!fVPwpC+VS^@z|{z_8|C(lmXh1q2eNCz4Y%@#23?9T zFl?V72I8TWspM*_w1(9mHw42b z6(w5b&CBx7)chX(N?t-=kjNbx-kC?G38|Et%L6@;?=jTqDuZ*DH_%smDp+06!8LJ_ z?4XRbXl8=9gye~EI>dMK3?i}x71F2&)!8m#2aatC;p--JIdHnj0 znL4QDL377S@`K)EV>iY0d)?j-EK$UhRldrZcc&w6I1#rl2Cm5|%Ki|Ww_^1VBmThuH9(DekcF-LY zzabJH2)(i3YCJ<{}^s{iC`ZvU}9$= zZr%=&TA$&u?6#b86KikKXpK!9{Ym$Dn^q<#rmmy`I-eoboT>OcwnlzrmglazGDo^4 z_`|o)GW)YG;3t~zuQr@ga17s>4}BaEc4F8UXMeR?v217LEJv=D`LCXCh$Kr3$G*29 zwLy8>g|E>1Vw%yST|^3**2S7o0BM_PNk(XXcF-g(+$54bqq>9Cy}Q$yGJO>nCVCSo z1(zZ#o-VqqG632B8kAH*72=L6TQm0k?-6aCjLI+fUu3s*N@L-5vhquI;9LI5O8tye z`Lb`#Zg7?g+@Yp?{@s+REYb?en^%5#cAh&}TOFh?8s#t~@Rq?1uP+!{VT%%I+1iLG zH~094pgb-Aj6)5l@-_s>Gvt;1%yme!-u{=M=8{<0rl|Bjs^Kv2W@HB8_@SsBF|CKr zLYe+ixQ~H%Z|N{l?DBuP&Po-RJzivy3nBx&SLXgy&* zlH9LW8L?@5Q-3?q>kfRn#HKPK-iPNqJDPQkU1XZiTy5~WoVN}eyPWY@^|Vp=M0qr7 zeJM0GGlBbQZGTpjb6D@>m(nQZ$F*Vr<=%Tsn_nQw7?5W_m#esNlAk&V-0vPVzrYt5 z*X(~ag88W1FQbOR?ZYu-@l!OH4B3O)p$s;zLwB2ubXt-!I*8hrcl z!SKC{C;)4Jwus~s7tz;H@wE2jr03=Ey)YA5gHCjMx=pqW&XGKtfO4_Y=8r4$mhKV6 zjnJ@JLfUHsQ%m!AtUM*e#wRRPz$c}$*_S~O*b-UBOn_S2FC7?}rRJ2^{3g$HaUrqZ z-e?#kvu?|wtDQ@xYHNGU z4HkND)+GW#TFRoSMZW@Ow*~vu&9t>Hj;~98D7%CAKFxqd+MW**dnOOJKPT3gRA_omuvs@A3xzN+L^t5>(XPIs3VoXm=hC*Ee_B6l9%jTFY-c zp6BM~e))FsE#()^s&Q3hvMPF0xth}kasaYjuXgS}t1T7c6#yh#OvW`e^ zKnLG6YI4yJY8zVV%wo-lYgRNO+#p!NJJ4LqS?i8_Mb4S3$zGtoC76$9*sn32(HdqC ztyn^s#lG1|rb?jc<_-9SWg;_C+y!TWpC>aJk8_Fcgy{Tc+=e?bOP-L|HU_k2LU#V_alnb#joPs9PuWlLd(K| zp!jTlJ$yhE<#uFsx|t5_56ppp8c@Q5zt1;!DK+U+-dNeFwNwG7TT^@v;?8~rwum(a zZN*Iv1^I>iEYw2y9zZ&q2GS*`wh4?-d6}bEAc>yChIe)3O|I-V&RfR$DNxzFuqO<* zB0O>`SCXYpDl*|4y7XzKC6ZrC&A zp)*1j!YY%#)GLN_=GFcmrN6345xh~h`p`hKLRKlC>R~_;%m6ccML`S}Cc(CmpUx8MpDfyF(Pr5vU| z(8@D9Jra+{d+D$v`FvZzFiq6ayiEZ3p?v-Js)zxNK=)!9Lq8sAH`rNeMpk?Sm$dr2T{2niAd)F5Lv}<~34VEGd+%yc^z%KC%C|@Kz)jivk}L%izVqk0T(H} z0p$>e;re!oAd+Ks%5>QGkd`Ij=2qBZPUs0@78)e zeQooe)6808W!|jPS#ZzFh%4;IZWw~2UHVFRCF_0vtiQ$KU6*MqVjK6vYDYJ`-Y*n1-gMso_R*#enbU^9!UDe-mbFI`SyXrV z>is%1~jwtl}2z-;4~ z)o~)rJL`M0>IST8g}zZnj*Qo_?b_4YR^ z?ps*7r`-VU%z<=`$ojbu!nIA3yK(1J!&Id1&KyJN4bpfSb-50Cdu`Soa|W7~G9bhm z9_;yS>R3fgoBZUC>$@V@KI-M{B;+4aJ7q+&JjY^Fp+_ohLV5-7(~kZQ$$EubpF5_#b33D){OSX#kJ4{?}}jFydf)RAd_XRnu%J zY1H(2WJFM#v&eZp&5s^U)#Ka_c6)WWzVX0W2rsrX<2!l1Ew{dA#`Pp!B01vc(*0W~ zkq<^QeUHj1u_&It+@h!@xEIelXLv(I!R(MFPr4L>dZHQS%#r>@s1-$_WxeBfOnZek z!S=+mjyDCutN<1~TEQ&J&S$xh=zben^ufXIw9MN5ny2TJuvWLy-H_KN1%Z;^HCJM{ zzisG!2z{wQzJG@~BzSYulWk=lo0RL72FJEvB;XV?@7!Px7C);a0s1LRDGs72OHyC` z%)0)EvU!2q)7Yj%=8}@`KKlhDiT;))h5PCYj{#F2N(ESvg653KIQ$D}PD;laod}6Y z&8q#LGzm_aY5qzHCL#^-h4> zJdeA{A;soJn2nGpgNYDmV3< zuB^pQwdwWnR$0@oBtt|wt8M*g2eVH&JAA~`BYU}l4?-~idp0pxQrG@R#%;XZykrL@ z<$ZE2(2a311m*Sdho z%s3o)#LNoaw!QkjY3mvcgwWgluVh8DOGV-LJe8D8q^&e9J6|rkBNTVl_|c#*KsmW} zyHpf`!p~qVt~zig%epGY3HPfD5|z0PIDR{5Y_^|qX|ofdPtEoy=lL;H z?4cm-os6hc+?ULx`Jw}KW|5J)>JE?9K?+zKchUHMd6mU$-i%^8s~}5YVJ{61;yq6!s=~E_ymijac*) zcj3SUYn!NtLt^#Ed_MU)U@#=Y}Vs*%ZcQPyk42IE~eF$P^|wOE?&f5`kPHOvnkiDdJ4O z6z2;~dP=)%wV=YAxl`@?sd1qCt^1q7-aQ$6J>KAqg_-?@beJ^4`$ogn8zqPDWcQnF z>o(E5UbmUqNpR?%jp64NKKzz->&tlIU)J7t#x?|)dyeM){+$6aeP^KE$xLIJI?C03 z^k#wr*$)#+tqsC{wZq4)uorYYxF_;tV&*1g7TzY0Lypi*hZBvq#ho_vlB7(S1f*plLhtO z{yyJm`7^ObwrW)16tmaQWryZ489E9`^WR%HITcu^D<`|Q7<)JhI^21SDPvwfSnd)n zwtY@)n3>}td^m%%ky!CC|1ETDE)E>2=zH_2=BG(Gk)FXr8;R^+dM?nyOj#3v$vIBi z{&Mo1rZUfABZVYgeFMp$e)?6|$usnZ2l-rDGY`{#D_=X~yS%iNHlx1@xTTq#S6c6J z*0W*kT~By5Q^AH!N%7E=8d$3MBF6}OVSMts9i`VKx*j&3_4If#do=XD0Omla)-)r> z?O@9SG-@<39uA#+KnH1ruT)T61d%<#%pTc;ZIW`gXUE&_i4QE##fL^m%cWRptxJ9R z)wksW+!Qh#x>G$f%v&JIjNUQUM7)%Fbnc<|WKX|xj)mD;Pb4JO`%UE;75&35bb9+G zb6}EpG2isja6eELl*AyFaE~9gt?oL-Cgo{~ZSmVT#y2h0_y_O*+4%v`=bb{QAA{SP z=K19dgrqU-S*ZH>Nzu2?i)Kmt>Mc{=VoL*)QsJbQdudUzhixK;`l#4KxtYB19nO!t zEECtz=o`suFrF!n64OpGa*+~F(roW zEbBsAnWkgIkL#w@j)@wDS3lgO{<>t_4dH2qi+KJ>KjJLR~wgG>L%+A~c%{$%yFNqYRaFs%7ttnC@hYK|($MS3i*aSi-Om3CbT1!qZ4! zPI|lAu&=CH%PiTso`^Ul7Yh2RQA|&0@2zf!U~T=pwlzO#ipMtQkkC$Ka3f$U57F5S zNr)zL;G43_T=(i=wXq|Tx(Nq(u7QnA%5&y z{}0u3mwgOM+CBs9a|dR)xmgnDxZeic>a6FG*F|vfCK_nFn1#g*INKt9^p9;9?i7&;K zQtWrE@S815q)+&;nf|RD&%;k(6vcut2uZ5wqXMnyJVt5j>!7nx(HfPz)v*TaaIR^=L#~>p3?WcjBItB!n`^ddV zk&GF2s0L$x=^_T05i7$gW{$lhVNYT?&oNaQ^1nHwq5) z*WD3pl559kDCin63%~4aHA&!|)>9yq9XJn45SJg1t4uc|*~{i_bWQUwezE4M`f>q{ z8Qxc%L|!4|D+aT3-M?;Brr-nXDlqnedfN^Bfv6si$Osd0v}L2<=7(0n+S@q6(b@-7hq~K& z@31N@^pA?ub2}~UDT+34U6cdEYDQjedsE_SpP0ixZ-Tk38c3fLwSLn|xL^U_#hPeT z)ySacq22p7G(vyMS2sIr@OS-L6a^syUlBKy46Ag+E34R(pSgP~WXUkl`~VokvBJ@% zfpjl%cUU5lzTC38w>ia#$NRbr#aeZ$KhbxCz;bF>DC5 literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner16.png b/docs/images/LogMiner/LogMiner16.png new file mode 100644 index 0000000000000000000000000000000000000000..2026ed18aeb9d875f491cae3886fc4d7f5002179 GIT binary patch literal 48114 zcmaI7cT|&U*FCH<$OuZDK|rNMM9U*jDpfskRnw;2#}&8qEt01UPoMY`wsCP}jS%U2iHRsU_|wlno!qiL z@t5Cj9G=>^KVs9NUr&5^zTrMs{}ETm{Z%5@=ftaP3i2mnC~J_KQTX?0;(Bsy3L-bQ}prJj3y%0{Ck#HPx)#&A+kK*oxz-hh$n~MGX%xJ_2O-3l^}bwdH3}o^-eQv z>ddTW3)e_Em8TWNmb?zC@u}1QbM5(c%y8Y?Sq|60&C0JlP9=RbVyty!6y^T>X}E!XwIo6w*!Wv&G^(oFFfJX3a%t5CDl3Ie&(a&IaKs ztrb=^tK6m<`6fk)FUtpP_5F^*8gXJe4t({AoBrmy9Vf_=_Af4dI`uAYuO>4DNKQgSWau;iw{+zlt#E_Kox+%BkZnc6MMpQ}-6DENATC#8lVbW(pZKTC zEz(Z$U&5(VhBI=W3ks+|iHyiKhX~R=S^op!Ff@gJtpTSk{={)nwFjPVLaB>Ga?I+GNYAt8NeSb9f{k z9&QGtOOJ6C<7s^* zDj^k<*cw)WXezZS;a{p_v+<7{e;I>3~UxAl=yy-qHSL;C}jhNt;>cWu*I~ z5g6rm*37!0W$55BiuX!f>9tgY7^@ZEB6|;>Vx6v0bZR*#<@57=Y<*X{K?m1;G&Enr zS0{?oHmQ*Vp}0g;nbs0!BwH?4dC~miY~!?v!gq&&V88HE44jHK87lpP<#h*bXkYul zuM${yXhY*&MZg`}WCzN8i@yL5=bVh3z&IVXa=|PO_FfKKsQd^6=?}t{9j$rJ!mqz9 z8=h+j_%u7D>BJp3bWb-OPImWtRn?u;z<&-2fC!b>dV>Gi>abUG>JG`jT~|{K@;lK> zav~}cy_uI9PuFHC^QQJ2Np{DHH3_V*l%hS4Lq5G8(P$;1VLM_Qx%CY@)~0o zr#lH(3tZQ!l`Xt%8A5LG7){9&R<(>+j_o}gC0g*5 zFw5}P4Cah!1D{#oYrPPfzbKPS}sUqnoSdBV6fCue0(iGTv|6!fPnr@>nHIVL`{BSTFHcM1cvio&rASp%dr&fqVc^^s-d5IiFZ z4-wH5VuvTShg!A_&YZ@G@!*xb1iaO>E+}$*1;>2{(h7#PXk_nrPpN7*R-R!kMK!FO zG|UX~^e(Gn8+b_#RrK1vl;AQNYAp)2#>|?vnq?;YT6wiVIGLnjY7~)!HHB!4=4^o3 zqD9{jFqriiY9$;diH0RHJhPy>fTwz3uBc~B*hA~~6tihZLZZ7(a@#c|{sMg{F9FPb z<7&@)E}^Qwl1pE4Nqk=|kONv(dO=MQagGG>98XR5^ooSk@ThwbOtS77*aMPNgVI#LE(7)k z)^g0ej+GK(g*PjETA3NwVG^!!3wBhjv}^9UD<5AJ(LUzmsPoFChTI%Gwmu3n%o|V1 z#wPwQ;=YGuOATW}r8}{r%W@hT&@idBJTNoK!AXR)+A-16<=1*&m!nCs*+vR8t3zYr z)ppln|Iahjta`yW0{HR0gq`8S56<=T%)1I!*Cv~6d~aXeKiSpW0THu$DinUIW1k&5 zr7|X4+Q2G_fFPmn8HAz)WY;$%19Oq8=}aS^&<62B1CfqgjvcF2B}P?uyPIhE@oj8k z?$1RyXjjb*rfO%cP988NIHuE{$>A4o*)74Uj;}!^97sdg)9d>(>aPh(5SPUVR6;^0 zd@WM zZ2NlHVf+vVs|tkd+!naCX|4nIrFFl@ z#{lH_vQLg;dO~D%*5^M$Jl;|~`K#gs9^+{?)E}Ni_4BKiUkkFVsW!DDnpa207J znY(aa$d`_amgFg%h&;CPy^^>R$_VgWZzU|aFUPn(rF(zv`c^x-QqOmUG`Dv69;YNCME3+JOV|dIFy4an5TlFD zM%tFicb)4dR5d(K9MVCtL8!1QiD=?lSLAR@{)Ce?tp=MOf`F(<{Vli`IhM$ED zpEG(Qm}^*zZK!OU^D(U*oN1W_uT=*v(W9O%y&In0)4m71WDL;pPI|B4VBR0s6+bn` z$zH2#k@>r0?a?u)WF z5j;Q}0Qm*iI3tB`n_S>Vg1$5)cYNIo4%^{rGUiNVzi{0+Mc%3rCSZL*kujqY8aRh~ z*!YF#kK&upZ=8PjtOo!emjkgT67Ji*SvXCO!Yg`!o<~5Wb~s9W+?sQciJ004Dr&Eu0Bmjl(0So!wU#kn%GiK#ZlfL?TxvKZNZik% z)7oI!>%+6bli7gtfzm)bHgx0lBj@uma+}hhCSrogPL+6;>H3Px=DgFDzX@=jd3s^OL!m)oeGU z1|DFbJaNEu@V8m;6I}az(+MQTK;Adr8kKzEv-Qy(>M`WWKm<7ui#}jbf;!<_?BP|HAGav!1@5Rq;JF|-6BGHo zp01hTxQm|;9Sel>b_APnHf@`2RcnAmlr-t-1eU=10Tr16>n$G?4{Kzp{AF@3EMV`` zdewg8+bcoMiP;_C)nX9l)^VX@-XTT3yI8Oud&fQ&yXD~#HOJ28b@<%|#S`(K@=u(g z#i22eDY?C=ki*?ji&(S2_FTOC$VR|YBTS{95Z>r{=;TxE?OFUyx1{+TEzxFg&hM9m z2S+7+JNPw>59$xJTBY0?FM_q)u!=2wgn9ZPX ze09lUPT-wr2fionA5+2^f8V8I_{I58y@}T7_Cp?%oTDo4S%XI@Q#)|e8CkPQD%Y(X zv$}5s&LFb)v7tFmWQzp_x{>x3WX)fYq9P0 z@u%SAUql7MoNhTqYmP<)ES((9$(!p%i5(#G9MVoJ-s=Q)APis|;i52i_@QGNpRqBg zICd_dN76@)W9ybdb?KpdJx47bagYqdzoTAn*S#c*plkQNT1qmhoq^QEdx{DmnByP5 z+p!h8KdY*mgkiAT;B3EALq?;!^zt|2hzRUde0pgHF5=1$b*es>#qt{dL~SQT*aEQ{ zfYB#&RQ7{8-+T&w@2|YAmphlEgu)#^2ph_xsg`j6O-?< z#_Fj!i>bX0JD|e(w>GrLCeRl>A!cEUn5+*IXQz$%{!y5!Byp+zWP*ai>R{Nh0Bzx`&CywV}d&j?ZAF2u5<8}C9Z)n3S#xbs5Z8_IAsyYgfUG@rXw)>9XF;&;xu`W#vg4f?|grdq4&L*7PI!YQS zE_{8tDnP>~JOfHBU-<)Etx$4Q`p8+Pi!lZ5`fN$u~j~ zPdIPKpSp0%Dlz8Fz6R1KhycqhA@mU7oB0!a+eOyPXn5nkVBYVBQO#M}u!R!OSF>Tg z-fy9$=z%ojn^vY+qrOZfPF!fxz-m`d?l2zpYhvng=|2xS?gJ4_T;Wbx&oG87dwyH9 zdNh7!A&m`hNc3zPnmqvLnEzZ?eCae4H9#h%aRs7zZEfCsp>5sbC;9Vks^e?bB!tNr z+~S$CnZ+7QKt{c|~W`TmD9Wu(01O}2ZdPLpj@ zCYsLrwUj;{;s*ZVXd}8)nWBU}WnvJ4!QTHeZCuzhxRy)=TnZHSE(?3RW}^JJzT0LF zgzpmyC=JNik)@%pj{8J!c4)_^6w130#gwf01qtYl>pUFSE@we}>|moQh30gW9a`Mz5qo z4-0}Rm;^@{OZRKP@g3ZEH2^>%?3(!GqXaAdRIH19-ar*|T@%k^4%*i7^nSy(`y>%L z(?E~5TA^L=$=hot?e%ob5#@L6Hbk~yg-=A(YE9OYhBJC zWS@A_*}(9U>zB^IuAciCt*g(}RbzfE&)zRNiZ=0#in{-9{L)tD7?7YFxK95N$cJ7> zN!CyeEM|I|M{O)3D;tI=dfmB}xEpt)<>BHTt6gT1t*2?GN-FaACLX3p6A;N@B1ybm zSET3_zX_$PKthQ*&>N~G|1YLrZ@3R5JT~F~T1hxP^LyXr#W;^soFtMXOz&v&2=KM; z?)n*LCcB$pGZjbq8oGGsLFZ~{E8uwi;hKbPCfJvknKN;uEjuC>`>=q%g*Y0saEz(5 zaN_h(20Y-ERaTNR=qcO~jGQpGp-1%>jelFK868p#0w|099E$ME@GqEOZvS=nEgf?9>6(`)&%NUyUxZO&O_=ik3H)M|5CBq z=T(c|865=wGK!kJ4|yJlU_asbY)CG$j%MGS@X(xXE!s0!%KglmxZ^O$ooPwFbbPz^*7Vd+8X0Jf=@@_hm?*FmWlm->yve3C zZrta^4rf-`%t49ueSQlOqfZcz;$Sk8hH2OTL>R~Yu$^@w{) zLYT{-S4Bsl3=t}tFb}=mbbe3Z$Dm$XWA%xio^!GNw_2j&Egi;MJ^NP;MX*>s2CuGnz!fXm zj*to4+sX3tB*1Yf&^~d_r4XS5JZNBk7JusuHHx}d5z3<7>SJ?`?&U>ZLn5AzSjKMwLNgs zykhU!7D{gU?j(f*EvSi)_U~!z$4~2r?8C~UV!4sMEpYVaa z7X)w>+n#!b5lFj&xVr8H%Rl8ns361{SAZIKeYH5HyBk04@ooITIJ<(fI(! z;Z{P~3jm^Fb?4uiVyv?8uK7oWC55fSME;U|=Ra2RHyGX4d9QwaL}~OtcjkY*gY-e~ zjxLM7ymyJ2?CYBY7$|Gx<9kGJ%@H3yVNMUVeM*#G(O_#ap~=lLf0mr%vpC6Vnv zY|Gy-RNa_r^k6PQBmNG7{~D3=yk^x!Krm{)=7!ZB2B(Zf{@0JO%5E>sKdRTQtNah2 z@XyaLEEI~q#?>Ci?T8#%N6-E9{eK=p|1(S6+}~#PugMmDH4lBSJuLourwukQog)ta zuOFox)#31K2Cy|y&HwuLhi(36eWI>_br);aKz;w`h4I9vaFEL2M0L5P>WH%aL`pk>61pZr1vrQ3e+aiAB~ z;Fy~GPNYM)-YumshOdaYHvWSAI0 ziWCMx6J4xiq+a#Pa#(t;W;QF?S99qOAr`x%+_awV0PxOcJ^*Fy3ztEAEn~tN(=k72 z^d~1}Qz9z`3M)dcx1?kI0s> z89!KoFaV0zVmh4lv3_7AWIPiP8%s*~*P<5nn}?R8(O23QDGlP47AY;#(~{mGoZdE% znlZweA3W*%R)*hLz@^BgC4AlqM`b2X%MfPFXW6J)MwTKg(dcXOde&=ov(^BKXiS3O z%{w>5z}oaYZij7x%lH-Y3Y9{PY#ABfY&yWB-+n6JH9BKbIj9dlyZ*_USTcx^vI+1T zWLsv;ZY$B)kTd-q1eCt!HV}UOU$&tat^8TVLw;FCdq|;T=EK50Rb5i#6!tCOqg*v7 zP0F-5CO(kjt5i^YaWlEA=yY_Woc2f6HxW{}3^)*)*bB0z+a`n{7F>U@G8p}!Fw;8P zHdAeRf>_Ef%@?9dF(ZpiMhI%1d0)Nmn%-4>8dZkKnnI-M+CteCTH#UgH!0e=O#w2! zdYbqQ#1_Ai*p4n&6K4hgI*JlKgphgH@CFT*-4gJX8oaYxrj*G3)vArd|*1a1)1mt<12NnIkP zt^_>~Pr5c?9MC4v0L1R5BJ^b9Rosmy(cBzhJ^kkfC{OZ=S$~w8H2JUnmD<{e>Bci{ z-tupoxhgnb>uL|9+het)r9s%;ph;xL8)zc=>8NO>twc)w(jC`(^Fu=<W0nitS;Mj8+54#$EB2vR0+&66T0j{mU_##Sd(* ztkEA63BVbMv7hvwAZC+*y|ddRgr5k)PY;zE)a#@+{C~~2E&TOKb)k&gh+Dp{ik5=r z>ZWsFxiZpIgzmhL%|o{A!r(jk4y$M>El86xR|_;UPRxy*SnrJ@hU}JF|Wo!;M3= zH>|~8Z&3fo2NpG(hq4R|4(cEdy>3)}65zpkHRR6_!e2XGZD@UcZZ+Xn;w7B<@G^33 zNRes-W@K~!E0upu4Nr|_P5S#|@T@$dty!!o^?Q%Mi`<{`$Cu`2)eSB1N{v1+D zXI8BY<{hUC8NU%KZD?b`_<>0c58<4Q zmaQP|fg5N_lu6hjYwcL8fA&T8Fn;Kj+^qPIxce(Il5=%eb9go;{L%>3FOg20%DUN)`!8L;ev}R}Oog0j=%tUFWUI|L@>lRAT;7 zEnd<2Og{9d9av;BO-f>pu6Eo|bRFwH<=b>N>EI#n z9JHj<23$fDY3@b|H)g4L#2=jl40Q5UH#hv$N6J_!y$P;;-BH;L?&>7uFTPtM1gxG! z{mY@syhN3m!XcRI>++^6Pl_01#ZO*5Z#**Ij~94TBd9{}>LjEP^?B9MjcqxM3;XBR z_V<$J+bd^TesQ$3yL-j2C`j}e6aQg3C?|nJaXfqCDyClXt8V(@^Oht9)G@xsV(gf} z(kd!@?SJUZ|7qo*rmx|Zd%7D+GalgJ#LOe-R14AC`q_EvMfZ=xL#YlO+t9WyS67kY%l{|%J6CI(MKvn6C0^?~Pjg{K?+)${P#|9k)4fJkGF^AlK_0DJ{ff7~ z<1nggbcewRpN0Pqp84PF)3e#eGb(jzsR0+RZ1=exO}rn*)c|a#DF+6bu4XXi9EIkeiSK!J;R1|wL<)TyPjc=1;oBc0V5LD zPYA-T=<&7t7d*pMX1b+~8t2ZTO;*NFb(5|MVL5S56|0AwAc8}XJJ4(uwrF3JrT9(1xWaQ zlXKiIrDOf?HdDGt@ z!z-MvS3qed!YgI4M!k#T9>d3!;tYr3DO}KGHoP9N$*id}*_Fwx3?|tn@4_?6h{L{{^(tQCapc z^Tsnmy=QnWgzZpvK2~s|q!e4^Wmd**k9reIYM2DzCx*-0!`cO9{5|2Xgoc?gXwGy? znO5pn;dARk>zkLe2T2lMoF0_tg*;=aw8UR)nI0>y!sK{yc8C?;7$@Kdhd9f_(N=08 zL#v0Tzz$|dXwRweE2!p~sA=1=L|jSvq^&dTsC9s4w)cX9RFxMEzG{&x9g|cgNTxdZ zOlKnC0g~gILc@lVJ8E@@zeA85-;}N^Na2&RaI_?&g#J~N8xh7XXu_3w#l>~&kMhZ0 z0?yUxoC7#$N5ptj!~9)n(dr`F7R7Grs+`9i4|0wvzK#+O$pE)Q|V5C>8#9$)>1QZ!a??^AvOf>=QVJYFpT zv5Ve%63$-9f@vzR(L3j!)REVr68nm+{oD0pinp3TSoh2ObgXy(>67 zKHJ1bv=?NUn4&?$hWx-*m1Ok?mJe}OkWh@=7NZi2=W;B2acF*cev3L48U?l&<)_do z?;Nzc6hJ_?Iwwl&om2ZEe4XG5O7U4tg|oH^2q6fy2fKxSGrfatS3x#=g^|;#Xal#H z?_hYGWHOcdEs zjFsms?*vfaks^iCQzYErtAUu92c@TYHxopf7zXb?p5+tXPve4?POcBi1pvvF zRBXjFrFM+W)*>3?0Yw%*s33w@N;VR~@T!@#-i#Oi;Tfbo=^owx6gVCzji{ojlBf2( za>%sK5xy-!%aeJ0G|5K;>Ihc?LGtnHSy7!KWJmB9~@+-@KpPY+BVslQKhMnv^%D=XwcFK6FIBC1+|{SqRIenyQ}7HTlIQX$eS+ zIh};;27xRKue`lm4<1uiYSUpXRr(Y{SP>;B5($VX;Y8%%(mZDP<_@k-Uc$Gv$m1ahL|u;!X6a>k<1$ zTzF59Ske6sOV7NuqzJ~3l4jtct^++|vg>U~^$l_NsqiBPHI#80^Lc>ukVbf_Rdx>v zDAb#yuPhf*%C#Q^9NE(R#{K)knAAK7-ea9hKwmo_J(ML{Z#aRBcgj6g_{0J9%5xrnE>A$6f@RWfu$!9)kE z43`R5)|H{KW)(+2;e^p6$0c7^@Kh>c^Eo9Ma2b_|*MNr~coU_v@1UA; zexvfRER%4s;lWDxD$9T#Q>~PE19$H5YDJ~>u8uNv$`3J!#oWz(?kVZ1H{wQNlPFoj z-m?V`?`qz)fX?Q=3y;9>L-cInM~u^yEF&J-h-v;-QuWYJ5V#rgHA0%0S_?;)MWZH{ zV>s@Xp3EhF)O0r_;C{j>j$7~xo6wmNkw7{^?|^hm*q2pdEu4Uw`78Qc#pQ(nOjJO9 zWIF$=$y!*BahWl#-$NpO;o$crkYRA4=j5=aN56>T_AIEkC*Pv!g<~W`$&dOPkrKH6 zv!s7x)SLB?Rr=Ra&N91N5{Yu2i)6>@W-oeh56HFcx?K}nDOH+;N(G&KCQ3|_mw~W6@wG_HBHev^%}>MRRBPlm@}AZQd!;8<-(E*yQ0Q~P3fMLn(yVxQb8lb zREzt%x{Us|#bpJ(ku0AG8e6`d7Ms;M5!Gb=cqZT&lc!;dh&T;k@7mTNznG4srOX6?iXXZce) zY9t`c8T_J!{o3sbZpk=e?XRqZkn%NQt(kUu%0VQ+SxsOEe>m??*CCXCBTB+!5eWSI3!eTV+Ga|q``}iB0+kMZ@;D~1d*&)45Y&$xobm9fJ6yAs~;yG6dLX%O`SMF#^TZD)r6!6+r z-uLH9rCnNDlhfu-SV>#8G|HxR=7P)+7&9I{K^#*kBrAJ>3#uaaCxHU18hcj10#Vz8DnKC??7g)s=$6WfekGvi_Ch_ZAN^e9bf)OVY({Vu zQ_mg@7X{wsn+)-HN~3X}FBJzX8SFVL(D96HdJMAUJ0sA}vAv{s({jUK(?2JY^raz2 zt8GJY1wNZ6Whbqp$xlND3vt7mo>kl#Uqq+}eSiAs3)AeW3D=-AfVUl`&4CXHxWi_Q z@4b-`Y`x%lvs@sG=M|H4b031%^v$P?xzg3k?@Lp2W9%=TU_ntU_*Az1znbsM$Yto7 zc9`DrM{iNzKlGv()&dokrv<^Cke_tWJ*q`qfs;)Q?N%JikAQ>kQdVHkNJ@g8a*Nt{gnT3j)B_0Burd9!w-j<*g) z0qlTfeiVz~{Mn^vX9BLcwv0pIZ!E2>0FmB*0h`)xOQmfFE4W&Aa4DF*(W&23Q3%DZ zcFoqZ;{Lw33TUjzIFoa;`CB@7YQg^~F)u)%iiKM=YjO0fBkaZI*;8@S+X(ZxhSS+v zWaM|l0`WT(JAq?!j7rZEOlBhA%Q4^#FYe-N20!Y;R^O~wNfQx)%FV_KQy|HQ5H^(f zn3B!0@^?BLyNWMls+g|$5!RNZN!zGR$INTkX39>r$qCf%WPrVt0khyo6qI=hyWD@a zUp5@|^6D_XnH%CqZs6$zmzTB&lN!Rks(Mct9!U)dbQYU~x;B#yqf@6<#NL-<< zDJRD!FAumd9Wv<}RIUN>u`9zZlT8|EEx4^fsVXq!?G;F3<=z}_xyH~r(GdBNb_kqn z&*e2y2W=a~{#(#nHOkQn_))F{L?`%DfQ=Gl+L7bWo}mAcE~XaYdLn*u)*`13N(s}W zBOScvHVk%;;13MvV{br3u07*gP19mPPKU|^-+h+8UAZBj9@-(PcFw|UjH%seV$GsMSvF4q7c|#anKs@Xv+tbe&?UX;MS- z6`xvseP6ua$LCIWD@r2<%n4_x@2CPAZM*b2$(?hzGLn6bVH?uhZ_4k#2LhLr$I%Oc z_Q?KOv&&#nDF3Nv-{olDVnIp7ejE{g<+|ZJFKVNWq;*>9>VL;beriCL%&FiJRM-U6 z6S0~bQkr4|8TDHs8%K}NeFl`>FzvWpTU=Q47HwlUHifZbO>+$ESwErE)RCz{(_}z8 zkgpo_B7xL<$j*jM?@m`$J7tlZ(vDdw$BD;3nAXxiQv`i<(q=dAV{&$6ZlI7hUut@X zWTGvC<$Qxurtak!LB1r8Xu1XoVVB+ea^8oq9kO|Be#R*5epiPQq@&{RwRj1K&yn%| zMq25LpRc9bzwaA2Bj;nJ5DL>c!+b+s`1Zbup5U5N^L6lV}Xh%DKE8~slKNvg&YHqdP{bgOA}qC;qk9otDKmX>Q@HU7yJ5{ou)?fc!i z+K|zT0e=km7M%>z=<=3BQXhxRKfoX?dgt-`T*u4aq|KIeHqc#>1)+kKQLJet`{WO z4vo~=pT4Mw?k3FzNXpC(Kxlyk!I$2s184`4&>R=Y<&ocyNhMTAW1T7@na2m2Xu|D+ zRaRr0hNlH`=Q#uZwVG1nI~{TfsZs22$`cqtGae5H2ed5u%4NlTkK~pz{Q`YVGY4@D zdPgfdU|WKx|D3loe~^?8|J^bdM0`=fhC)i@oGq;fl_si>e{bM2Lc7Tl#J(*NruSBN zAFO%-Le@D<#gx8rKQcA!U z1yXP^CHH*Wa;m@Vb`=h8Dpl2&g(&T%ljF}Pz0|J+eKd4Lu%6;9Ue$(Pi@AIbSKQJ8 zNKakVNiZ4e;{>eIADIFh&C>_56|`qcMUf!wl#xL4w~XC3;sfcBu7{qSJ=Nqr)w*Z^ z8ZvUCO!#W@VHzkSo7e9NA;Mi)UCv}A;F;^j zyK!vgosvGrX{j%1|d8ELWZvfkst zRKu9tZO|xT(>D*PZo^|T0|6i0OiqVOr*y=eE9h#iZQp93LC+F-OEpb2R-}U_agg$G z;g8K(a5R{OmK+)-R=NE0+6xpSue^B3Zpyb0@=0w$ew-?o=C0XOHM~N`D+@Hn;@M+^t6rqDCA^7^=o}Ed zPCy5MJSQ>iIa)c{@;2QJxT9Ow*)jtvpy?#{ItH8;YnEDjsX~Ze0t#y~b2S%BX7Vi% znH9~@XfqK{ib#Hc%5*ZRj$j&^SD7P4?dP?16na;aFRk`l20go%X0kFB#dboKbwP&c z803Q%Y151Tp=?!^s)^gLhFeLu?X}ReWhR4CNZhd=-__EQKAgNQh8HpAB?7ADLSUbD zfWG$Sw9fse*N@jso%Z_d%+`Zk^-6+dFy^EopNgrXjz7<2M3A4>k3Yv*_+$goxG?f% zi_4;vg~Qdh7@?8pz$cF1@rXdtP-sA!Ox89YCG59?`l^D!#C^;=Tclh#hqK@D&NQ$4+$3VzOsO(Rs`V zrJqaZ0Po8<@<55Pj+@0w0I9giq2X=3TtKCtSVnWw(j>ezQw06jn%}Dl2TcP zX;60*!`EtZ*AVLPA5oDSlfIDk!JhW7>Lh7V9x70xT?fvlir1e?7SlaXShX$wpxHeo zE1m#xLd+^2s~a9D>}qvJ#nZQ@NiL+td{v8+wV)sg8$z)YL+0&ts1L;X7-PDuu5&e{ z_(y@LcO5U4u3OZm3$7bHiQxvGg&M%Cw?hTY8l4cv9uKRYgxCSmVbImM61r4Npk73X z`IfKzpUN*O%a8cYbj1PkJrXXHP*)gx6_s|h*M91lgp_pA((9%UB*;V`HeThQzN;HV z`$E@c({E{!@}faOkC1AgQC8Ho28jrO;@Y4hq$MLBp?-(k^sHCnXsGQOse6EKH<%^q z^^44xQb7e%CXKmPpVQ#h{Sq}8?2sL>A}dC1kx3lF!yuwPqLm)ZN~1^-8ggElzg#fg z---u8?z83~veMHk_33QU>5->yt7%>FO8uFdZYK9te8(yu7R%$7(KfOD6T0esGuAuB z>?*PGOD&J%cxS!szqYSn8u%F3H*}OUYAIZB%v<%pS^(UT+KROBc4T%~v+FKrvmQ$a z+xx2y{P81`+w}XAa?ka0#@Hk3xMadWJn;Y^DT^~jIDv^=yHC8|lYW=@4!KDGyR0n( zhC)n3kLi8(WYe2-LRTAVTJG_WMWb=Y<$qo`!34kMeyRV2_hJ{o+pm5e{KF^J7a3yB z>Yz>ymYV7IilE`Fi(e*u)3jh-7rq=+6rvVoQXJ4K$2=oYWNs7K3)ZfZL3toalzVXJ zg-F#%zT=XND4tO=N0d#t3MUnkei@}+BhvbM)2K{up{u=QQluPJ)$+KWTNLvhY0{59 z5?a9*$xfN;#%D_~$2gRTf~s%pkUU9WRxJOIdO6`+TA=Q*s+p5--ORUBH;T6ckODFe zER9x<;h@N$6`qLr>a3_@7jn+d2-Yw}Ax-_5W zaj#UiHfPm@sJ>`d$5rphkvg2D1ep|j>;|59;b&i<)dD-@I)vW^4UU<+m~|5(O=fa* z563o!HgtOe5Iw9b?Dirvp;W6;n!*|rGsYZe#In+65*IOl)2=SLY4;rRm?UiDn8nL- zsSxj3WcWEx&bmxhQhG$^`|n<*OZN3sJRfX@K7Wl(4+ECr@kS`$;N2WKZRXvef+fmS zsh6lS_dkMuo^v(~lfUSq1eTu8knrDL!l&}<_qj&UJn!_Y9+3=Q;g=Xk3)in!D?3{8 z;h%lk`JziY7O-eH$?k4t==Z#&q{~}4ve6Aqy#2U+zkJf^h*QSZOrz>?B|lpuIvVA$ z6Yd1AHu;){SoBJeS*mN;+3xSpt(sahmoex<3FKx5! z#&??NmgXP*|MZ?2yXDkydwR$zoBcfK7e>U*Pd-MdFLPg@re;H3cvSP$B(82YK<~+s zRjED*qzvRWDpb6^iI`Wn1YP5H<773EV>gEVJ;G6+kzQLJQ8O@Xk&C8kMvBcZD@oEi zbr5GWVZ3}N)S8um;aCKn#vR4sQ8ea=RM$eiG4Za_>QQBMGGwP#fqQc213Ql~p9OCa zmCJG&5^-ey(^a$6TVwe{4)M3nJZ)|9elNGuJtxz0ds3a&xQ*3ZjX1C@;6<_n?X&%j zvbi1V(CxTwwyc|)G7AAHDpXI6+TaYA!4ANfpf5GF6LvJr;VSmG*$(aj_r9iG3tdTS zo|9j37I-3Ta${}h_?_KtC`10Z0;H;Zq?UEhab(z)huvf0Uf!!K)ET?NkO|C&FaS{E zcL1P==Zu1q?=8zpcvt=% z-67Swo)aIl@cT3e&Ujb~WHPiu6KS<2FU4W4cpZe9aDg}fDCVEphn3L@6;9#i{;0u04j&8N4gD(L1IH%t!1^ldR4g0BpAp;5ta<<8R@_tQ~B+N!ZlFcX0JW4kO$?H9}#j*|xaaa4a zpxsid_D=D|MsX#)*~yT2rN~ze3MpY%dzW~RtC{%_B!eS&aNxqfNG4vki!1yacg0XY zL5|D0Is?3DY2;C8r&EmgqWW0grnRn0IhQaH`RpLaxUNl<2OrRLBt)*9mFjYPoFu@@ zlk9SIw2kX0BPk8#sta>lw(`mxy_^V)c(NQVT?*R6YmBO!T18KD3VV1=+F=Xt^g+0g z)FhFiWAVQh;ZY%Bu@l(yss@J6J2?6AieZ_ZNj`ub{s&RtU*tc3cIcQ1R>(qCH0jZ&+ag)zIao#c3U}xO8)0TC_4v*8y zl@Sr9n6TR5rR2EnRXkk$P#uYgm#mH+YA2z}2^trvc7XiFyU|z5edbB{?w5kl4#du4 z&dcFSsu&G7=Qic6j_c7ER}X8P*mqAP%W6p21Mj|Da(BE-uDH`$5d0m>>%`>#`nQD# zWT3*ohudkb*rU`1idE?NyV#|Kl`$`T8ygzp6wkcNfm*EFAXuJakLUTtm2DtMkA6i| zM=U&KAOw2=oar1Z#@>}rIeK;+&T)5@5bMU-&a=}se(ZiZ)9k!pYMt~$& zN95gwx_f0emJZ)6{p-)UxN>sR{b}=R#WXoDyV4%8@jSKtz){a8MpN#}dwV2LaCnay zeD;CF^!wgdKKF67khbn*U@xZ0aw!w@rweDuPq+cTYgcfa1A%6M|G3s>aMBgEdx;}? zTk2_FdeWv}&5@=QhD;b6w-ArCFcwp18(^Etwr5r#+$Gyc=)84%>DJ<23I0j zK}6H_F%R89ZrM&d8aXFv=Jf zVf5^LuazWuoZ#!(kNL;C{gmWBWfT5jNa+{QP7UDL`yJ6j^;6sqy-@h!J*yd!wP5ix zqG7vSEuGnGjT{`@+(C$vud|E`mo$3W-fB?)sMU)6n{w=J!aYcTesl_Pg-P*4*kk=7 z{|U)CZ~w5R$bL*UexlB~vi)HwIC4+TQrwL+3*8h)}d0eC%A7y>L^i%D-V5EATfR7_`zUeSgQ)2)qEE?}k$tlVjVN^7lkm3RKitKdGyN%}R8XqXY( zw(l|pbUo)Dn!X>*zJImSOqtZ|E?Ysn*qM+!ZxPX5AP@0NoT1aYg#LSM!=Xi6l)sVJ zO2>zS&=oGj(lmUybn)W+Kc(6EWm)$3uF!5--feV|KTDl+_1t1+ZEN8Wr|m$9bT8m6 z+)bkY^PK`>=P8CpiQ(k72d4)!7Ll``6}uU#Mf#)4gq$eui6%bI`L`Z$N8<|RvApmu}^C3>)n~U zbTmiOY;`PTKQarFY*am1&5z_hG9?y{;mC62KGTiyL_5w3;yO=XUKqK> zk18cAj)6|8i42bf)srS6`8{?y>`h2HgcxDZyP zK-IEU8UN{tpy}&2{RoyfF*~nsE+!Qtm*!3j=O(x20h>bNS+2L~GM<0|Ksq)MtFGJ~ zPq{bMW>r7!dRF)hJn~NxJ9T>@@_KvOy$o$pTq9B6=-2me$QxC=R{o|K_{IR>J8`kM zUf74IJ83*YC>=$fzkp%m zt8IgAgLhe&2|4}AKrT6&IQw%ss$z(p3g$BcEF+pu>mo&H)~Xy{Fzr4rnOGA8TjKQn z<;O_yK7e(#d9@Gt%Y8@oN5p5aNA{rV@8WEox%h8q8pGF8&ey@g%P#(VsKVXm$cu)B z1XozZ@l6ecSy^~XXfLHHS3J+#3(=m>vg@jaWx5Ae%-3H3P$_lgBBEm=zQ z7cz~+Ue9dWL+XmBo3PwxVqfk@Jq{Cns@#BO=A|rDlxiGi=?3=+g3^0`fAe`(s&Ns# zX^t0Ki@;Wj8Stt6*Yquva4j%*fu;YqESwah61+^H$mzV_vG_hD5tI(C4yGNC5#wwB z%;%vGGsMa4IAlN8Hc3*B(q1tiED1Ts zi)5mpCAKKbOgZsYJ(0SY(KU0XdgmrA3aug^b(9e-CBIE$4J&52Xf(Ub+pa$kh4{nm zk*cn|kf;c&_X3YHo?c@tju_X?#>EytkRZg$BZY2vZ&Gx>58uhFjCuaz+!+7@P!1%- z@inx)Rv7N54xIKDE1r6wg7{0Gh<&=rWfrF^H=}T%;1Z!{VEjf!-8N43`3yt+pwl}~ zBHdt7sUqcSV}A}(yDn3~w|4O>87SmnDvUF7T%3(s{!g5^xBqGTZpZ!)VZm6kI4{Vg zw~{2hvU>xb0?x;Fn?ajK#S%|xdH2|yRf^WLDZgIIQ8s)IKNceAOHei0bG8fJh;4ZM zAwtcLtXzW=hMFvAN#dG9RX!t6|!XX4rgOuJDFEY(51 zg7)0tPb0?kSc|!-&(4r~-fJp-|J_Hbo~gd^d(;ud{VFx?tz0vBznm*b($2he2WkNb z{bI)~R@VS4qExYKJZdob1uwC~nX8pAXr{2uAS%hqSS zc@n?vsoe zO}rMN9!U?uaQV%$AlbP#w9)#Hi2J3*&XHZuNvK&8Ah=w@zidUO`yn9rJJiguro z7uOV-pgT1s1n4OQEGCJYonMY`@--egwPkmcskT3lx?f}J zUHjFxvjLqJAr)3RFoW~n!I5SCc#vz01Dx@kb&d7+IKfNZCK3qw04AFzy4uUzAX)Mo zPO(D2U4G11WG%;QbK`_X;-8i-QN6PNtGLR1KU6pit(y#%;Yv<0k#x!3fAldC`nIY2 zL#emX3+i*W2sWAGXQJM&fsnjRT;Ofr*3#Uj3J=34ZOIIf;^0?z3=Bkh*&e6(hqHEg zSL4<~UED|FZBFB7CtN;?Hi7#_njsha`7rFkz*CAMs2rS6F6XCy{DdC&2MSiHRWTDK z5hFQj%A}f4+H!C<*I?d=CSL? zg@qFE5UW`IVI~zpI7m`(uC97;#UHr17WA~_Od~k+kF4u4$Q)T897|ou(4|@2Gzf5F z$z+{d9nBCj>QAX?SKTzjddnHQkuU2YUv1%MOhzm-4oJ4aCFo6arwEu0HN_Z z8kcjI(OW z@so=Hv$0>nv^l*L5`XPafe!QV)Y?uwp4pYH(BG8R3QM(*QCQg}0z>v}luf(K91`0y z_{M@%4r-7=!sPOYql|8iZ!!WNzWZbtRF~<+VMlD2RVap9tNC zrIp}ETu6hDK_BMgv+#51dwOtlahH#HoV4K?Sgv087SpB-U!jI~(^^&47xx5p?n44g zqNsjgMPcZ^ehf)fKYLXWicYFm%_4LD^<^qXFC71(d2wWwvDl*2zv7ilnmyZH`5#Z! z`1%*cEREtt6qaY!yDKNG3S%{D8kxG(ujY0f%x?IS~E@Xr!i;^jbPry;DQW zxW6rvk8b^~tQveP{}g{+HEcz@E>-}*f*WS3Ta-Ij)2sXVA;%??*{ca$TpIN7F0z^* zDQY*(9FnfC8UE{G$NzKx*xSL`T zb6mV-t8E$|um5N(ZT96+1j{4+P%jn?A!>_bOq3 zR6JHVBQ0hR{8*sw#9z;{5Lz<4-cOMg&m}45){OVRKRCxY5t&y*RWKz~-T7l%-iF2X z7>#}1S`SUQ(sJrFoB4_JfG=fn#r?l%zJT=$heGkZ?4a(thc~;Ge!{r+Vbu-4yI50{c$N9;=NEnl_?txx{|)2YVPs0;W(m-QjeU}FNh z`x93xPw@rI>sa>~Re6SsAIRrLyZ?>f$*?t^7WosKNBfG%Mo;$(25e8_YVQAT?=XEb zjq9*0=qQkgl4qkVXK0R$>>uvev;(}A11dmjT}|X4)cP+s_77mGW>`JF_NzV-?*9RQ zZ*P0tl=rl$A6q^YNzIq>h$9IG2m9Lvr$=^7+{;V#dbIMS+22JZxDunF_;!7AYeDtx zl2719#jSJVO%1}2U%0OccCOQe&)bW4W`AceyVFIRIFavXzhzupNvVGHVkIF%V-0Cw zoYB@#jH@C!H?3WW9ms0Ot=QL+{o=?6o)bhxtfU~}hlRD)P8UFl@E&HVwUdoZ;i;K8 ziSS=vNvV5u8UMT)9!x~4!(*8rd&mJ+H6+K5?P)`y``S7#YQa*Vgwguq2?=xm%=guH z3nv_i8cEw?QhLTV14V5`3pUvw?Pir6otaqb;w)zmfM0aIMTl8>VP&Jx+O3)#>Ek=QDyoOyRDX;7O6AHHn4j1`khVAkE~QmidB9OITF1#ij9?=VpcaG&>o4mf&gTh!K; z{s_scNyyF?+o1S7ve3|xhL2@#-0bdendTU4) z{O5@6?&2AR*r#rHgRd=fy?gp6sEN~&KEDpn8AQoz+(B+P+m4=nFM=PsyUXHV|5WnW z6Z2b4$vF0DP)WrJ2BAo}usjHI}^`PE>#Q8o@VCUW@f z_!bwa=g_S`Cg1;Z5otgp7!%~_bDwbZ<{sT^8l*%8AxHp-ryOY!DaAc}fQ6OUM|~!X zmH!akPsspMw#N($=JKylhbs9*>K}Zu(|V1viF6FVhk&3GTyGsNJF4sd6V5sLPm`*# ze^*4OlB8y(V)B?Slx+^E3 z6weHKnstvnsk#Q+ox?S#BFh4Sbjf{l#x=JLX*^pRu=N^Jm``}cJyBY6hZ!M427W!P zbocw5jhV9;w&I!M?|aeUlto}r^GUO!ADftm9OIEXT>`=sF4d8H2UlVdzWIPW>G8Yx zu5=i$)FVfcHyt_rmQ$l#< zWByilB^I{DF&aV!T=58VM9SZlQ4*7Y~sC$`%^#$(cgbmP~3g*eaJIy)xMb&^$w zs9qH3G}|$?8zmZTo=h5?V2tkN*%gMi0~)bIZ01%hPAMvKezcbRf_7_nrnSTnhRf6s z#j(E)bV2s2a!@a$jCgH)x_Nfc2T>;4U@x22(+zU68HGY0S}9;&E?Pw;^B%zMTqrbQ zF!E37UMB_aU$cBlSL84>fW0Tu*&tpJj?9)0eZ$% zBPU9Evq3Iy>pEJW@4sG%OR=$%YFcmwiK`LbTD}fE&+Zla@S)UWDb3}at%&#nvQ*AL z4hGH=^`2n5T96dp1Ygm=sv(-_s=Fhffq{;lcHu`MIl8QjszDY@WTAkEliKxeGY@;>3HmcrclZpm|1y|~ z6{}ro0D$wHJ2wR!PB=J}K3jb}Uv3=q&1*zFbT4EC*1b#eT2LqCAJKOAdE^PN1HFvw z&Tp`X9?iFAj@gnJ9Ux)@!m|+H+~E-NI2y83aA!dj(g)|>wRBF1o9@=CMMI+FxU+yu zG4g@H%!-TQG!rYh@0f3$b6aDJ^}OkdhrfG={o=l=vam~$HooINVCy-_+|hU;iCC1B z@1}-PEN~WIWT4hb_>C8tg4J_z$+K_V2mDx zOuo82!!T2qHZJICx{lJjR2|=3SX^?$drMHkWz2#)#IS)voVM=y}5tP5UYI9w$5A&q`@7N3jD8?&%6xF@I|?U zI)$F6Vu>06-AI*lTATX=Fefo)q}gEu-S$$hRRtnp{-1n^w(G`F2gt?|!BliKGFNw9 z{~TXYG$b9;;~I=1Cy{1+ZXqS5Yg#IHm>0lv3(w&ucLIyc9*L*(-_QaHdeLhQeB)KA zXx_&rh>2C`D=);&Ob*)m+Bq+o`V2rlu?U&%n55&!{rF;?}Da`BIc;QBG+eB-mo~53y9?zQ*+&Ss> zrEI)y3~{=D-xME)p=TcyAn}uxBkXw)?3zqxZN^z_lXx$MLjawAP|M^x_L-jy z3HG#7!F=@9F)!y&n-qRBRjf+&{~}3Pfz)E0YH`8LN@p&%_F40GpbNAG7jvLeD<$k{ z62L?PKCF*9d3T5J8CynmMdpRYscoa2!g;#Cr#%(|kx)sS>~5pTB29spN25Rqm3w)J z1S=^tDeos==lpl^Nd{FCp?q=AYLp!0u(j5z;t-3(=YfKym_s>5XVjSYqKXa(*AooEWL^A_HpV|--u;))W4?FT zY#l+6a+t(C>trm(am`!|m4T}N7sgWa^eud}loC4gAQ8{qI_$vg@f5NBTmH1RyYPh1 zT0^NnI@r)>$nG{v|q!O{G7YF02~9z{sy zzSm-Iq;Lbbai^?}V==z#9U**I=5it-^pu=r<}(g#vAO?D%hL{%KxH<`KDXz#VTLED z1jMbxcmK%+Dk^=UX>38QV`2c$w6?K)!666B3+<|DqtVu1%+K(}lFj3+Nl;&8*1B@t zfmdJ>mKS}+!=Hp+#rxH^hq%hn!fNieG1vWIiZE{or-HoC4Sn(BoPT>X5!#@ujMtqm zXRpU+v&PxGd&TQDM8B(EHHJ5M3u>DK7Ylhn4Gzp^p&O=H7+YxI5<^a>p7Ja!Ur5h@ z?*Y#a*WUkSFSvgg4EK_!ZWP+|qdj*{ncF__dgq10rSw8Md(8dJ)|by5I6*ElmpQX; z*{7<*S5w<52x+ySa7oXeyJ%h7@YBFN+4p(?)i`(@ToT@|+viWD`sGnebj?CMi0q`6 zTP5*#3+Kt%h>v*b2T-Ml!~;6l_RV7WNjah}=K-dhPYig@I(}zSkqQ}KIahGNEPHO3 z`TqMn9{zE)Wzc}&G6ek+J-e!_%;BQ6uQ7FpxEMAUy=6C=NT^uMV+?rXdXg|mOttDx z3Qm;)4d3@8KAAv~qNKOi4oA#yDXi$5FM2R_If0NXpxcGPDVm_hNIj$ii8HX2Vgxn* zGk>(iyt$l?IMrh^S&AMn3Kdx+awj1+JehvVhjjl05X+SV-UT$W4U6A@pzJfeWOlg! z_ToKBSVasRM8J?$bg-=HgICH|Kb00Y-;{3E+O_s9c2aPXrRV(g`={h=mYA(Izpj%`*4Qqn#-N+uqo!dK zC<*O)X!eoG-`Q-robcUV9w;}l8aF3!pp5ABe&o3aQVO~F9{zY}_n2U+Sx&-1@9-9= z%8$Cwx7j6~vx0nV03dVVC> zOQoGwSJ4zj*`)9kTh~^6ZZFgVW06sL;^7DXefi1C3P;gLOGUZ+!I_cv8N39xB*%8? z-jam#lUnm5J4b$n2BC$dI4)~qAhee433{2%SA-yFt@AsC4l0*BwQK@zpTNNbp`AmZ zk1zk@P)@5+uN+2yK9|#0aA}A0eIEo%*~9!!VMXpRaaH^q`w!*;hiK-(4^7rA&v`wG z8rM8LHZmDhV&RvK^+65<(HdG1@a=UTN`MkG{WJc%-)UYf7RHoHK6%U~E9%c71LnTt z785I}neeLFL@CDCJ)1(7oH~Y`CFX=m>HN#-pdI4!SaI{~*>T3&^s)-lsl@PY{#fCk zvDpgW?a8;cP&$#r#(8~QTKpMiD8}3Nu>a=lgL&P`+GlW9wu#cGvkO$G{k<;_6YMIq zEb`2`vn`W3GP8$mm#uM>Qj7W93uEP&;KM46LJgSy_J94Y4o`joqRzQq-7oa9PB|ddaAIWXh|)Z?w{t? ze#b)Ldyf!-$c&za5eluNNig1}n)-B`Vv4$wY5Fzay1F^F628OPCSb+(Q(t*Yn&~Zf z@7D(2DcOb^b(HjSXPqP;(dC&w#k)GpqW2w&n)zC($l9=$K29Z+nk}txBcG0(k^?!j z@^ulsQb3=O(Ee9x#7Sn>qBW~}`Vt)%{hC_rc#OFoQw`Po=3F_}2RCl6ao}u!XP8*~ z8Y;sV90ipL*6AKDa?^B?YT|#@rFX$`xBqi+TiJ|ShBe4w4iujE&^^=gcH}#8Rx{^& zrzOtstjR1^Q}HU;-g#^0HH~^6zvStrMcKOFBUX^RT6E>%N|R2#nz(Ufcb_YvC$E0; z#hk(mRFlwbw;qRC!?<>~8j9mxXCHW0OSkx_$1hECt#zN<-<)e=alLnAyN#~_`#^EK zXfVT9KZHAEb0u<@PC=CUxwt*{KiG}>n~L8DzOV%n;!gu;7A~bHmeI*i1*myc!Q8o> z^)e8bJO@?s8&$iLwcxd)1+*@fz|fb{VH zEQZN!Go4qcm?^#tHjQ!9mT|i0+?u9+uY~PyB8HP<=9?59KQW@zA{MCt)!wbTDm5sR z*-5Q%*iX*Ah)3#n;i^!rXwHd{Q#LZrgHfcq|Hd#d(Z7HZ>+LMYyY2YSP)*xbIz0LI zjR?&Bq3t`iy}+*QR(!(7#>1D9mAT19<%WLbk%77w%L1T4!brA-CXSVH&M*o56lb?q%^NBnm->PFS9Oy0&_kNsT zAI8$W2!e=IIyBOl^)fUilqUXE#s z?jH3!_{%;tKOB|Z=g)iNPkr0T;t~)}p3g4m)F=x2UL2Id=(2Y4lD)Z%-7#D>AmrPS zpz}B|OEQ?fzNxeVmx;`c5AC}WpDV!I28y-0dhad0kWjlfKG~mx8nuu_U!cb`oQVB`b_MN zIOYuCmd%Yz!^|%N^FPr#sI!)krP)!j{Z4Up$76S&E=}>ooz?+FRl&2y(?6fjujzSX z#J05`hZOTnEIj3}F-xDx#itIpqss)mcanP4&zk}N?kkSUe$;gDo2ww<2xZN^RxU@2 zapJtbG+o_xNIOk&SbZOy96(`Shn>Tv-B~UoIvlU<3~p1nIr3E-2Q+^UAIP?#dUx}# z{p~k0C3l>!Ke07Z1XH}j?p@5wwZ4}Sn)pS3P&30X>Yg(gHk~azksCzie~psy_CN?u5gaLae$7m&e;{pJswjN4y<fDogIW(69BslZE79wcn~46ylKRpzgQ_HMRzrZa+KKku;M4>AvA+eYx*8TxPG+7k&=* zM*7NV>w*=0)B8UNON_hj@F}>gUTEw_Z1yDexcOJP_!@6jP5?_n=Gji0GQpb*u19?Z zuB;B(LT;vo;q70d45LC<`$tk8_RX6_7UTzf-d|TXHdxZY=Tlixb=UmYaXz z8GNRUN@~KDCnv01hj$hN-GgamTWx<=`{Xa5IM_5Z0Z)6+66^a!jNL!5aY(r!}*7 zW@1xK^Ir7$K;6Hw7X;iRb)~GWi;H`m>f+d3>gFfki8`eAKFmRs@34~FZ{K8Cw;9xa zvWGpq#(xrJg*cn>jI<71@GLIZj+8Ck(G}OPkVH7=RaSKDjS3rLTI6iDZe>Rg7SYrb zYpIPwlg3exU#gn7^nFFmXx|9g`e)}Qe+SxHv_(ER=`e{x2W8KZUXjIFljs3C(}}Gz zy@G8@-;NB~`kRxzv7@v!$LEn!<6FVf71ts-So+o5no><(hS`!yD_b=^mH=H;_tVK+04*woOlK{jCC2Pcca=+6@)TWa?B7n?c7H%VO?j>c#sa-PNΠ_ zjk@6H+0vjIRBIe9$F*O-HH=CeG&2!{kjtJwuyTv!!TyU^`ZE20=w1%fUaLs>DZi)p zPHG>%H>_)LrUNCx>1Iv&qIDS$l<4aU9e?-Iwf{f=FOiE8Ex z70%Z((=5CRp2sSa-bVLY5io{4WenGB2;+9;giGoF-pf(Aa^~7yx0kn6Xs+9S;N+gC zzjuuW;YFzi6(K+8e~Yu2LQ)MY`hQwN7ti5uwds8*#rDv*1`LnQosSclwq+64B&3a67F%WV>QKQH_UZW=krlM^2kAxS>++ zbD#BdlzL`-fwu9PQR{I(PMfa$ z%t2JSz|s%lj~R09$2lsSyX*<~<^=bb9Hla$TX}t|L_P+q|B-F69mq!&KS+M%kGUJi zly0+glUuwQciK_^1F=%yRvJF{N0{#!GOUWjMrJgiHR#-}s?#;;>a^}!&m=SS=2m!w zVKUW|ljrxlZhpLBPcN9QK28QuklK70CVk(D%!%K1m4-!Up_L;291HQPU4PIkk{&)P zkZtcSBSBDn=Kg}619SJ|Bl}dJUr>H{`nH3dyfP)xnyrrOO~3zd;ULq9j7b4tusUplF*~dKX+go;WvlS-`2DiLXI?cU2250=2j?M~- zIm!Df6n?#w*ORRdRTKfY&ausvSp8(mRi2I@c|H(c=i- z8%wLe!32Re-{20nGdORw{wyXg_eS-s%Ni?IHN4i0>RodHowucI?e!FF%P{AQ)b!;3 zoJ)Fc`v|KcjU6f?_@hxH*$5fE)i#LoJK{e>LzlO}RARTZODy>OA1X}f>{};SUu9Yn z(7;ea^_!g=Av;9^`il4R>#N$mWvT^j>ZQ-(JQ)Yh=_)^mvq4KqngLRO0U~@trkMYB zMr%-y1IwP97X&T6(*)PfXnO%-D_nA0;f&QWyOc`lG#Gos_OIHZ?2{ED(*?U`0=payOSQl%hn$- ziacqPy)qDycIodEzKh!|PlUAx1!|><GiVPJ}XkMw0TyFtA(wU{@dn=rmLxt?dcA6~0E^E_TT3g4y} zjAxQ@5C8dgAPfMaZrr?<-pvJ4z&j1)Iwuz=7-2X}%id*C_v;UKYaRXZT197uP*UT5 z4laXve&!;iow^fN1tqdrXOra5VPd56%Q;!>`XAAjwflQL`T8rUgtCe<)$Qk(g|P4y`Jl7g=q-Q_i8OpS^E zn5UvZUEkA62{5HN;x_pfostz9Ak|#7OodnJA(eD=_eFVsHQ!EKvGaUnzQD2l6KD{y zg3<`AoJpIOhANcfsq~FxMpb!lx>i~SV-ZKX4ePFS1v~0KSuRqW0b{&O#)AiiK)iy_ z^bhlmL3`AuI<5p=53rdT^^`Cv z6U?v~F@Hq`g%fk!yhFR#Nw3xwxW3pMB&av{7vY|Z1We)r04tcwPrTBNM zWe$6}nFl{3x6EDs#B*bp$KL5(pi{GsP;Qf^re%LGVCICJfR_p@eU%|E73~hJAp~jj z+CA2+?x|{O{}bS1+^Os4%-EG^;mqgH&sWV=!l7fAjbe1)3&}~&igc^P%ivNFnYYEc zLxvw%jl4jo6+Ne`8Uen5_Q2)xyebn;KOnsr4)O}%lV9t8pXJpJ|CBde5N}?`W&T`f z0oU#qKhr-6miaz&EPa6ND+OPaq_FMjrYd~Uh@ zV@Peigj``tuLioT1zr9VFXmlI`&b&gHMe0`rS#jvVe(rcHulJ)Nysa&`<;1Et0OaE z%OY17=R#+*2{_!?-M4pr4?n6*`Gwds3&ywSbF_^nZdzkRo4zwuMtlDEXNG@_hrVbj zv{QN3!}9OrsjF$7&7>#Xb^tEulDWPBjJ1y3^N|S2T^_lIm~Y$6+%WcAztbq zKc;@E5);TW5pjfGadQd+#+sEmd)Nlm1T%_ciUmwjOv>%NI&hhVQRB0=XMhz~hmNCg zz&3>+Xqa z`wR1OPDfM;DJtX^Cjls?apa*Zm@X)se@^B+He7%Vh9BqN%JAV=tBs_>JLpB8wz2=G zTYJLCCMf*ynb+Re?2h5wYp;>ntn@=mPoz;3JazoLFF#xra%xF@*&fGOYw9;G+nY51 zr+c2;NCPqg=$E1e&$lSLy<-i^#MBRc()!&X%e$vVyGpjV4^j1@gOh)K(_c0{m}fm{ zaT50pc%JSf#Iq|fl_jV07nR0zb#+yk9N+qNj;v=1FznmeFubs|kQ(XLnuc4yJ(BPQ zFe#q)JhxeZWzOP=o0a_oc4tvRFaO_)1)rd%xA%JK_4vzRazEpMRN{NzF`Z=fVyOel z65G)0P0}wt-+xmpX~OYcEbt|!Sw(LA6Ay97K5uj%`ECBm?zd)Va(aM|4Ye|guc#8& zZ8$HDeW4}2Z${6Zovm6C*JL#*X~XUeKvsH6M9cX-q~w&kUYF>qtM6I%H+UsqM;#Sb z#k`P1fx3TQLBT}|b1vpq$}Gzpq-;v`P5`NP-?)@%tpH7k&5w>_wkfe!G-MLT(PaB` zYcpuyiL$T0@r^H}8{_Y}%tx!w-XCWQJZ98=Ca9P^;&1rCID2ztxQtcK|FioGM9M(K zOD$Pt7ZI*5&+N9VkbR9Lj`t|EsNt~r-tr@_(0vWS@q%um8gCY`Q$sjk{O3YlL4f4J zid}K$l|NFySt*%+sOfuQ8meJMZ`4D*jmXLHd*ZlR$M3;oY zCMVl29bP5rx@758(XaBo!u)bW@@k9nOZ=ZOB{ZGX5Q;3h@;h)g(>u`pV zb(ClW5R*@ebD&C>E{S8(De~SH$u42JTeJ09liAH?W%$NDQcc^%N_GXC5nnpZmcRqk z3ah$qMg;K4pm+13r4~L)KgHQJ;JWNOR_p%y#W9K6rC6I-H;nd2<&yHeoPvCnBGMtz zn_cgrIVLmQ<07997A=0qz^+kdCAZ)>w(sBYy}rxdc+wuZbA_|M4GO}zo8w~=cb?7- zkC`Rcz4>1ofq$2`P6>O`GVVFJjV|)$-Mmy?*VnECAxoqdHEWSiw(XLw`&mW%13`ip&>_xRa1UdyH_i(n)1LtjHQ}y1iTX2;H$F)=DgZpOEF<9e;g6ML@>~+=p zwvtxJ$86aiJwL2SXV<+DN)NzsD z>&|vZ_03hsuAQl~)Q1g)jA~QyhMZni1XCwjyn~058bfRLI4W<%q6Lb@4~0}0x{FKdsFF zGW?&SNfKdV&cd3J>iud?78}UauUuqY4=E*VKR%-9r{a7gYKRjunRNa<-%sXZ;nYdy zxym`~4GH*JEF8D$(L}XlJt~MB4Px*gemJO!GP`!iDFc=poR`ZWe^)gbdcy>!+WT{9 zP*5SnifTxp#4&0@B!t|}bKn|>&9$%g!BM32Tt;z;1?0fiFP&NET#HL-Qu~4={NcLX zl1pWyEOHnoJ0(a&->@@9_7j`qGSAe52pMiv-nD8in*YBSpt6)-kKdl^?aZS5(#{%~ zc$N8HJZXU2ya>9&DGw^x`Oj}61mw&ZoglqY*W0=Q`eA3Mo5kkcISlM zz<}<^*Hr3}73y7q8WeB!dBny24+-2w>Vw&i^UnDq$4F;)LQdjX@z}N8|`x!vb?8Q+PpG&T)sKC6I znNoR0#hw`>Y}@W}$kvB9HKYgM)fTV!erokR9N5%q)?g`E+YpIx2&g@*VITKyPpS-T zr75AT8x~q}5=DI0_w&6ZUGB%#JHb8`1M-Gu@hE9SE+EEk#&vF{;p#Y7*BvrHOFl&A z!1lZ($C5K!Ek)KpLfA=;A&SaF{R;=rfo&1Jb+T65RJIe_%><6`5v5dfvSGBBKa>Gn z*zaDW#%BCV<90cnPUodZWlV0>)<2+Y_EV3x^HefNyi+-tAm9{?#`aE`lcxX6 zY|)peq`(6x4!Y(CT^puO{`wHW#PV;R8UW%7@ny5XvP+-4!!%!ZzC-N6UQK1|xh6x@ z{?V^ovE+Qdgx`<2BJ_@2wDTjw#m;>;G#uMQ#FHlbv}4E&vRu8B1&25W0i+m&T*%dDJSvC6rz6%hMl;BhTHD6| zg02J>pVOWZ%ij@Dw|eMsYhNCat<~EL#c)gB8sB6Dw&V1c{ z3`cuGR9kAHD^k8@t>>8~^N(#mXYlkHB0eOGb;vNzSrFXh$R*{&e%+qWrzDSAbqX~% zX0svw&_|-PfI7t6o+5PVtT`!|@HT3`1F$>ECeBIzO$y}*I@AOC6i~|UA6tQrtn}rE zgr4o!{6!SvTH&AbQs|}s+sMw!PWfu4dHtE$I}ts83ODkMas+z)uzM!SaI#OtTpEtm z=FLK$k2GL13*8%D%d`p?(`TsvIMYHDOV=%9ESvlCeUBgi*j`?X7i;toH*%b(mS`>j z?xFQ7{mDIw(^U^!mS&feFzDH?WRVHc*4{6&P1~&fwke3!uGQADY-Q)4QlLylgGdn8 z9Mt%|l$#>qC)ZBEUyTy9k@fM16!n!t*#RWR7E77FQM!N*72g6*hLrzLW;+fKie#gx z3@;?>3KLa0Z=wde(}R0>9$SVBH;{L~u zc1YLIu5&E=Y_E&BQiR>{C-yMwN!}Lv;*O<`K=vVGb++o zsMe?|aKhFKCVSnF&LzIJaD~?LYJyel%38wn6DtZ-0#YW0Od;3u`2Mj&ogEoGY~+Wz zw*UKyPbpH3U^bsL(i4{(o`zih(C&ye#;@!G&IDJiGrQr{TQux-F~KszApxGEfKqya z%e2}b6-a?6lq1Ve?yLLs*w0hE-;Mi|ZaDyQeN#$l zZ__fripcnHmBBVEOCK8l3EHnUaSw9nxSq$6Qi~RiS2Xnrt!he(`9&-0N+T3SGSYl5 z2b0;}Qg-nFudH*AXL=9(zdPktIVI%MDdaA59mY~5Or^+O#SkLRb-m}ux_c%^V-|1i z;63KsK>A6^H3+aVA`#P6jXXTYWHb6;I6|Q+YEOz6l#9z^Sok4}!oTfaQnz=bWY@dy z%%!ltA7HWYLe@k$ZQz$NOk$nv@n=t?^I7;uF5{l`2d%z{w{a`%D*kSnnufk=IAq(M z^aJvTl%j4eUW}LYnMz9zwf$7Xe!lX5hp4C~xl; z#An)skFg&6kV^@*(_Ohsvf6tec`l$HXFpYB=(!#7mqd?+eB&|}y&B;*3U+9U_odFa6T@Bulrruob(7o^dMnif@#Aw)XsVlAORu_SH=Z47I@y%~->4u5T z9e+E@rMhf)T8T4zJuPAguc{87n1qYV6grnpjZwbn^Y)R<0J)X{U+vNn%%VOECXt{K zh*eE99N4iM%M=X27_<}$NYPocedG*dN$Tz{-a6Q?Iz4xh^l3te@@VX3$~O^LZ}l+ikBBl! z{WfIrpPBvfOW_}4e@WA4&qfM6rKv~?h^}R!VJPu3Fuz}5Y?||2i$Y$r`?seGgv8an z2+|Ofv|!+!p;DITJ?WsmkI+1DpJGqfMn<@TQDI$IvNTs#;fiWm0 zMa^og5Vivnf+urBB4_>hr<9Dup4!BU0rZ z;-ww%MTIs{MsGJNC%Zi~r9XX0XSVI?Q@IQ0?y+57$vY(G;%tMR8DfMka#5o91$)dY zkhE+=R`+P{HxTpcst6KDdR$aB>@3D4jc$6{VL#faAH_eX*fY9V0Kp`P-j7#R2B0uX zX1Al1rRcCq>II6ZoMxF*8Xg5C|Z6zHoB~9;bvFJ+k>c<_u6|#kw6y|SF z9HDoZb&)9-oZ9B~Tj%rG$){$lHU&>9eesV!zIH#M4sZ9lZt9P$A+61vZw6YcYV8uu<^6Sii*&!~Ih5^OLcJ9mhXCQZv;vev6E&j8#_J^48;hAaK>S1tEEJ z{grTz22R%EH|b}uOY2t2++I{j#2qZV=n!{`Y~lj zirKr0hZ6?Q#MGGpUY-BcJ>3X$;)PH$P^IEfd##XCQ>>Un|3ib4f_sLyBPJUnC-1sF z*n$^VjO6cZUG=xVR~%0A{QhkRMt9qYktE1o8m#MRW5X7hqrY&ru*>c*oft3E=QG+i z?SAYHe?1X@$IYl$b!6qq97d@{sTm?VJ;TVI(PQ?qAH#J%oN-YTuC7<@;QZ)4MUVLZSb&nac?tF?RdboqxW1jr~!8W`1ZpP+PlM za1*XB;Uu65Sye32dYgT1Xj~9TiSZ$uyA0+}c0~-$gGF2!3KqZ!;9_Blw4km>TJWPx z?F_&K&cUN6L^h<(9@Kgl(wrjiM2l5YiwGR+xSY2_7Cs^iOC8FG-ms0orMs>@4NFCD zQ?{sLR&+e|!$VrU@Oy9Dpt+AMAA2U{l|1%z<>Erzd_(Nb?ls`8w?+ic^F4tM_TB!I z7wn>cNg3+|_0wyKsN=h*TB{-!mEzIz{uM}r{pH<-jFV?B@zXzWFRqJrcxo7L;}cK& z%Y7I46u5TC8%5=fcZLX+SNMo(7Gl_8m)`7Ojh6d8*>*VDolt}NUb>;2>x@&8b_#os zL~9PYH0vJ@Y>uzqM|J7Stt|+)bkN69MaJvXEEhD$fcN=8L*`H zan*9#p7H%%KiGqk!JNCL-8>* zXK6X0e2VT!sOJUeUG?poJ9^(wmh}tio8#ZrLwrQIQYhy)0Pn?QF>jZK%|#`R=q5%m za(}Bho$GAk{W`tY81IS8YsxC*AnbR3wdMd}6^5Wxx>JeKB$qhP1RV(qi~0k5OY|Hy z<#F%cG2Qw@UF?2WzML}6YKXJ&KlsLag6EB6{wRsvEC@QWdgi5Px}$qMHiFelr|at# zW=>ne!bW(YU;D_YkW;7~TIN{@e3KiL-!k$%R|wm~1LP8;f`(auxZhjgV!w}QGTyh% zn8jZB!*_}5mjk}HrOsUi1@`0ux$Z$^McjaM7gGr#5l`rw-MpPKF0aO`*0`r5QUngK?;KXTdu-f5 zuTueNL;PN*XB^FUz8Lc6PU2{<enHH<{bvH;3ehq;>xefF=4G}+yKiP`CmdL( z=2}b`Q}d`~CcCW7em9k03tK0fJlvT|NZud8q!1d54IGmJYl024hlgPe#K#3|oYob> zKYTKH-htXWhHjj6Am`FQt1%NMK|*w%ephCp zD=_^spY~E(gQ>a{r-p>U^--zgz6RqntUi)+!TIpt`!i94XK!+S1MQ2s3j~*$XrE|D zouM*wtOjnlPK)bf+*x{xXJUI#G&lsA81sr{s}*+=2&mlak3E+9Q{Ejp%+d&5T}b!Y zMujGZe?8XlmwP6$eeIN_lNVbl$UPH z_xa+vWZ_&`ymc&Hic}eQq_7leGq_H|>shiX<8)KkbqlW5>ogv%F z;{s~fw0iACUiyQshI9PN(SgF%U?bp=p4jj@K#%y2!fi?IDqh@-B}UuiX{d!qm}HWj zXZX)#`(1lVxzS2n7bD~>4J)-3Eq2Ak8tvK#p7SnO`JuKA?l!~_4Zjdtq9dtDX#@97 zKaT?khwN}M->^9NTTAv*&x=M^H^RQ~M3x*HNMPP<%$6oHN?K`2NO;H7^8D@3zw-23 z^MkmW_Q6qfn*C(^reqEGEX8L1YZ=gCro#D$j{Bt$wYi67`&XUE3kF!8&zhO_dXgU| zl5ERtZ!wjKb^?$q9`>(?T|chK{h345BNm8^t?e+S#t(3wndY=J){*YPIq?#xUj0%7 z9d=^P?RaDVX#QW|MB5W@1uRaI1>`6~LzC?{+>^ zQHmnvOLok~{xioOWZkuu4mv4toc8`-j%Wd*9|)CL@_3Pmr$3Cq7gP)^Q)JJJuPLgJ z4M2xx`%5SnI@l5Qm&PFYO)pec)ZvO5(4Jx;k7ZV9)H2P;YS&+CXOs&^E}6u)!7pcO zq&5u%xGzMt(5boZ$jVS;S}*@zojhoJH`(5Zbr!R=+mm(R{-Fuj{&}V^ZW+!#r<@?= zj0y}@(`>y$4w-2?p!J%aDEJ@xmW0++7e!^iB} zv-_F-fc7o!%3!YH-ZQRYd)i`~qdbNogP*zyg-cYW74A{thS1vc_Uqm-L&x8LT23_w4&vxGRwX4Emt0z%3|M&CmF~ zbWQ|7(nuUzKKt4=6Fe@Eq6xB)>7nDCT-aV#1#46JdD_*m)@G6pFZbDLd$N{@XH1uFlnAQ_>gtJ{KH8`&QM0spHHwvqbHU;yU^_ zPi=_fG41X9K_vzmgsMf_a>*LS7$uk@-+HWn`IA-kR~`&@-JX52xH)|&FP7isjo{X zy^xMP7y^|_oOM)sq<+DI%`MuZJ}8{$YKLwsbT@1EnrD8^hx3el!YBtHIKM*+UeF=G z_E9-^7s+j#E*ctkQj-W6yXq*OzY#f9H%*j>1)X&7Up%|2D&xFe(>rln?MmzgAQuM{ zl5H#Mz2-?h4U3mgiX6Qk&)GkEEVQsiU{fs88f{TMu(SHo4pE<0TXSII(r)1b*iCWK zr^3OFY^)S#iYa2bH~u|0w@1V;-#EK4%u7<55co96XuR&HBO5DT#Xn6am!gZ$Ev`TD zyKW2US}iwBj;X?4x9rZY_K~|7BWCy$#;55jQ zG~oM|q#B021&D=akp*;@z1T)=_XGnGFpDJg-a(E>_32gd60UKQd7}2_gIRg?wYpK^ z{vc83SPMTdLq%sIrk8D?ax@s~s3`+4Z+N^W2MVnvU68EOCW}55CN4j7$vk?-?=PsW z-xr!<62GIwcUiw8vWUQUZV#-{^UX2uwQq$~8c{9(q|9HI`Ik-Pdd}RbE+i|0G&wG@ zKMT*$t6CqCep9ypkf)S%hfVgxuo1~Tj(k%_wvqB2C;!Nm1M5%n<#9PA&AQ|R^l@DU zR=*x!B%Qg)`KG=H5@b6q0N@)FlFk#|bmc9xS##X~&O-_}H>Jn=0;-uhtk%s502m#5 z%fe-G9iWyLp^%Aqdv)fv0-!iA#<$Z`&bi5dyltmQq^W&Q`{6l8{aD zK5yqkFTvReG6@2jzME`1p@S5RqQ0`<^|%han|pwT&SyX_65`|yE(cF%xvy~SXWp3V zJ0PDfVS>L1IMzGV56%pL8u+jHn6Rtq_fu;yT2Zk=9aU|0xXU&-K;k~dnVVvh_sS#zKh4OE!l)Tt_}(9hC+^V2n1PpuflkQh-_I$OZC;b`8(6_^z9Q z^7-^GRQ^q%gldOEalIMbS)AY?^vfbe^u&SEn`Vof{-!@HVU6C|C_`|Sn)rl?QCWyu z&4ihYx_@rv&M)bwVQKK`7B8M@i|Bg7@1_cbdc-m@?wq6RBrG`MJUdC%EgR5O^*Zj*6purasX-GHWR{(U1Tiw`aR-r(d_S;JKg0 zKm2sl3W^hU=(31o6LAg==gt8UKM@ygrVb3^i=4&#zFAE#pIx1mvI@g|<2e_U-_@Q5 zLJA_#erZbl?=Q0{G*sSHSE>-=QPJRT4uB!%M9{hjJ{vm#0D7I@K>$0^oati_3taR2 z$rptR_U!ucOzO}|Rb$*!k3$T=SCPc^88#}ks@1rklZ2KECJ^v84ea(tBXK+tU?-%m z!}fwc?Cr*bB<)@yu$+Nje&!lzpI>8EV6QGg%%zd3<(Ptcvq;?TwEI0TI!^TmWUTg@ z?nJVxvw_g=NtIY-@N*3qbY&?MySA7u5LB8saZE)F;00Q-i^(QM?R@59EsNj&qw2!b z6I{1Y1muUz79#>z>%m5Fu!aBNv63n40NmR%0b4eH3yY|V5iP7mz)&Gz0v}5)W1T4` zK;0G#giX7uM*P>y&Iw+6OLw|u7qnSZ`Iu5y|AW68F)y+1e}qrkQUj5tMvGid!CzVl z+jIEus*BhB2vL#8p(wP%ctJ0ydR62Ir;&c)irKD+BMlz>e%>Fq^ML19;pHZ-sfE6s zvMKFm(o!ioK?dzNN~#-dvA3Ht_@Qa`LUEEU0>^ zjkt3y&6$!#=P(b4+S5@o6Z<%8DTxBX23oJ(8NVR3$LdQ6mRYbp)ZQi+S<7*2SIOv` zo#SvCsCLW8rkPCl-fe~}>^~~-&D5T=f&dG6386i*=z&no$s~HJtkI$+g9YB(4f!{N z5ug-FUN>puott>K6;t913LT68g`eyKkUIA8-MllWRdsn^;pCkWw77(}@bW#Jt9|We zEJ@ig?Bo20H_)%wBrIKPkehZch%1x@&8JAC$s*<2-KLLEH44ycLOOoQ)}C~lyXB#K z6*9a=&RL^KaKvx9B9UTk%kWL){UGl=q7_NQy)Lftwg^<@l#h)0`sJGb6PqXs-JQP` zFXvB-h1piQ_hD`0S_3I7?sEEBt9-7AuX0E?jK}l29Bc=%tgn!e2GMs+bblY)uA<6T(r>2NqmX&Y6 zOJL2@xn!d!5zo#wy1{4>fy)D$u>@xBIkA$Oe$ zruAel7N%KYM@Dm_Uh&c;EPYpxCJ%RuwqvAx4Oe5uuadIjg&R8E|DoKUkjZF1R;=tT z%2eR31`j7)H9hwHe1hN06c{^lOIEaQ7zfg?Zrmzm25hktq6C zyM0qp&=F0^0J8~{+Cisv6*dJjMpQl0FpIYlLsxa~W64XGguh9==4oEPVzhWY! zX)~_);T*YFDcdji_38~XhJr|?L72@?2V2QOOG%rf1#8t8pd*ElqT(M406j1H)?b^M zkXoA^OY1PAUns*KI|*&*XU!1PyzAm#E9dt$CnV|<0mkE_C4OHIKKuGRN0ACB|D1h> z`8GQJxjv!hd6J>QBx+RpXy=LS@R>V1d9|}}=E33X>Ad&(hF2xC&5J8S$6xF`7j@AM zu<_TB%ULW0W7mcg>O0u;oOjxCG{i5(;**0Eob}GSa$48O3+AlhR zS`N`FPiv~(_qQWx5K*zz^PtIZdR5*B4%`nJa279G_qGYI;jMKR{Vnrw4j1q|Ap~#p z^J`CV=*|u9$*BiJ<^@)^k(RA~b$j_LXyg6&MPL;h+POZp$C8(oQZ{b*Gt2>N$Ob~KN}VyoORdA*B^JlK zhMf~TAtXErojz*aL4x2s*F>Fy@@8jkC*`LsF|rOdo}@w;J&kwPqAJ*SgIe~Q z-n9jZm02=E=gqAHXzTQ;eUJYhawUVFY!2!^6`?y(trDu;LI z&dkE>Ijh0xLU#|NW|24JMMp#hR2~E1Vh24uEhz7zScSG ztPYH%@NF%17TCy#!6}bI)QFgK{W%?0roi?}`!kQHO&0QqSDR+1>VNosntbrb0^m5u z83M!$xPXfU7qS&<+KM~;c$#@^F_E69HCGldUXT~iT1s9_Rph*G_A9II>qe>^GDmQ2 zhA|K_4LDmMsu>c%8Lo>;N#s)GDGC~NT)QHz>cH!xD~Oj$OVcM`Y49pX>={lQb?LbK4d1j35yk3J<$no46FB*~tRSKDysTJ|;2V*gK4|?$g`3&n zea!C#xiW){H~IOW$;qnCn_QcHOoIKBpi^%A%j2=d^LF}g{OiFQW$_xcW+zGrwoeFS zlFRt%t8;+MI22+XdmKtGwIP6vLqIqb;H(`KodH(U)wAP%AIYEAm~guXhF zrE#sK*RoF+b9b&6N6pR0WTQm3mX@e#-dnPH3L#d#-M^@*gvj-$yGz+S1tW;M`-d8S z859!Q#-BN`C!b4i4VL&52V(gtGpRB9X(#yj*m9FEwy!Dg@!aipTn+#<@sJ?Lk=dND zJM7wh)pPc8zLvl47z%uj*RQA$dGlWD(x>hxXs-gh$EJyrUgFJjj`t%(9NwQ-7d@-F zR9awMp7OEh+OGv!KsNL*$@9$<555tT`B|#U_UDki*HDGHht}NNknh^`+*+4z05QI&Y&Yt_`b6?|M`R?wXNfe*YqY&!&tBH*B8?#EB#LMA?1 z;rSu8{9!hzKn6(-MwKnD`$wlq-!l|*%%5h5(1gBdNnN@CY*yaXS=0AO%?p06^rEuw z<(-Jzu2q_}5wSbM5iZP}SH(XlVSCOltI08y*oB9SWuA<>oY(kdJ2+RJy#9};q@@yj z-Cjd#2V8+Vy=c=TBmsTDeY`LN^&TuzZOT`SDnAm$AEG zX@2j?YG{axw|X>U*#-^dar4Ax7XpT%Z(mBo#JITMFJv53Y-A(qzdY6^Hoe!cI5Z`& z0lNZd<}qo6ksn@=@*Z2rq}bbLokL>Cbbi;p&NSb!;NoteZII|yho(nAU#@Y`VO~1y zC=@qmNxpr+BOJHh=-uPxvx0lEWw8#63-@%C+W!*Z``eQo9oNh>y;Y%mj!JKGP{#|McOYy5Ku!jWxM7A_W$!aR#sTuF7oGZ0IIQ&qBK%#NWe{!)qo>Kd+Ti0b zI@(N{G0_jY+u$xM@kUdZBL8^knJBde>HqDQ5_e9}IOSwM%_^w&2ROQ~c)L_o((h6a zJ-PQcy^XAH$YH~Nu`Q^rL|#%drtkLx2M(3`jdzpK4%LFiR>0=sd3Znx6UBHdX3 zBO8v(Hg>3yjW1k(*C6E3w9=;VTiNR8xz!f((ms^s686=n)B0&zO2^TW^l}4ArAzG9 zwX>4C49{?$Dh|hHrp1Y$o+>d3%%TS7ML7O={F!O~ocv-jqL*F9;Y5L$^V$)uS4}$D zVE4zUA5|qtPe97aU#$}wSK>X6-0AsnQ0}gx4(4tx~ ze5rIFOC~Fa0@y zF-CaWSfN6dHu~uO1v}%(_2U#A1k(WH>)~R9V!z8`Vct;yFX*?^d;{ zU#xF?^?e&UADt|}|HpE&eoVKhFSBxu+Q$W-*tU52Pvu+KcaswXs-SN|&8#A?w$MFv(9~X*UT#wb^fPLYpxs|8nwf`qMxHbe}!x^zLB*YWKlE)=Zd1y6CHH zp#^vMCeUqZ_Kh!|#pEJ3CyB;6ZsLe4&BhcKTHfJXbnb8=GlC*0uRF5g`xmucl*=N8 zbD%wF4#QjJ_Ulh!;indd2rs)_A4!Jbp(>WR^>FP${oB{DB5OoZh01C5@bfgz{3^wC zw$B?sE_lgD(&RqaINp2zwjU!ouhZqzzq*hXym_tlwp6X<*U0#pUx;nm;)Squsn8go zIJwe51&b~tL6Tl;Ik@-vF4=jd~03LCa7 z6%~M<$I$=&9PG1MXfADMUR2wTY^hA28W8=I(o9N9YSSY3? z)&}{^dW#BT?>bztK3V_|%e*W2YXQ=%1rO~r5X)Hl6elcb2z3yAfk@Bfd{ayMK1$Ut z8TJ^xF{neq+n!KibvMMkX;C%Q!xrKPqVxJF4#(b)we;K~bhOSmh z7%j)jR5u5rqXie9juMiPuyAkVd7HJBsy2j{g*i&}1xF$b4L;bSg*m&{Ch$0>&Y!-w zq954RfimqqbJ*F+aT^o^#h>!>-d}cTdROeS$FS6R4=C%88^s@YS@P+J1=WtC1TgJ5 zNF0jqP;q>d5;Y-hbWUud)8cwFajtj_`_u!vPFgvVxxgG+^x@rek8u`JGpsTfc5Pqr zxZWyblm}a`8a1>#Q{iE_x)UZ1arAZ#GMha77&*VqQMT@9F*?K!*3acej~;z{$TO`B zW1NPTktchLv2ne&1_9j7F5(MBto8U*B!{QmS%`5Wq}`{eDc5wcj}^kN^v$LONKCJ! z{HGiL7x0b;UNZKozI-;LBL?8b7dl&x2F!b%*!CWly{W~ykWysS!Oa%{trF1g|B!jj zTd*o93YL7#KOeyPC{8C^^L=I!G}q_OA|2E3`h!O!xH{&DBgZYmbMWB1aD6Od0zMew{UTTx!D7i?K6;O65|n-f33wig{m zLPb+EZUH6-gfPj=s@3^s(LDn}KbMOD&F&dm^+MmVuI$=IgGFog_c!jwybEU8{O_}? z-_u9pYvtknafIMxQ|tA%qm?!(Of-|gZ|x^^Jc}|H0v%3a&r-1qiF_cbH*<6M7dq-F zzif>m16&g`BE`yMau?tov&Ft_KGyUFK4y;#^4cOZI|8%HTD2KPDwm$%YB1HxP2#kb zuX!ZjxKm#L#eMmuT#lH{nls``y$`<{b}0R738vMmDwsNlOL89aVtQMVE2`++Vy&+l z<@a3c?;Xv#jQVrfVo0!oedh_^+Tb9yCgJ8BcQx05s>b5+KGe7yiCbLl_3EJfs1lr# z;$JE}U+b#xTgPB1p?YUyfKN8wGRJ^f7p9wep6$Okvuu= z-(*!zoFX$XD&?P`o=r`)uTlu;R3kt4v2qh~kJfs7JZ{K;@UoJ-3^GNn@f55EC4?1h z_dRL(-r2~c*zL3mP&K_ zDf_V*jTg+(1hjF6l%LY|ep^@W@#`#RFbd@w?9k~Jhw=@@<}AE$jcPbQB4#+;?T;lYVTaQ*A94P+xJXk8#VCE!kIH zih?uiek43$ZnWs0LL0*K1Lf9&D7b`mOEkLMsSd|NSRIV$G&kElqaa?X)0Z z&kk0^vz)WChQm8An=04KS~j`U2(VztLG7n{$MacBv82LCXl@rT&SAwe4kf?m?Hb-y z5w=rolm)A*rJH4|kbOBjY%p(^FZaq3=5Jv1^oQ$4FE|q`vIvGJDDpxA6d&&sP=3x5ML*X(@4Q$epT|S<6{rkcOp|@8lH~uK{;caMRl+>i9Q88R-3&;lDriGK_6#>zXi-ZVlGvv$ENW0tr6%`= z-Zx(cA}@E@tV3K8>8t-5oLIS6n>Eg-(e~1GK@LT?=nRJ9Nc+jx^|Z8N_7d7&dnLZz z+{nYcspv3`w>ZawqR_^ikN0F9PoP?CTpDGVL&W|Fn%v`}O_u$X^NYS;qMEL^fAT>3 z$WK5RG1;-nOo#K`YPm9evm51?hBHfgynI}maD1hKs&v@Lx{Ag8&UQ7%1~|R}T7V2o zQKd-9w$$TlGo64agIvA{@nOE_}#yT-~X9Ajz~9d>x48k!oax zJ8DyeD{0gep~r&mqk)>9c7ezd!-X?Ti}M`y(z%|^(HhQvhREjPq2{(7yQ+mWVoAg1Q9hp7R88oj`brC6K*M7+^l zXWxW`bi$dN?pTTih`36z{%y|oZ}aws#Bw6IAtI>PzU9iz5jC;IpGvd6-JCPrX3%Cz zW;L+9F*7QaHD1L;bXHzN`@1}Tm;3A~l+nv=C=0Pl&J17WB^7B)TgG1d5FVPeuA3T&C^p?mG$MuGkk`>$ zEY2AM*8(Irx;hogU_0-^a7E?`tr7I$ya3#rGgcJ5oGS&NjFkWJtZ<%FejR~TB+O9b zEugd-S@AtDBt=a)9Wot8fjRCevm|CdGmPkdqxylc{p z_o|U)KG16oCVp+C|HX~`_pTq}(v4bi>ueFcl u4TjU;89LGZn3;3`-CFR_4{YmnYCukoU>=|XYcczXU9IYFcv;>?gR%1hp@^0 zJC8UxI5&^SPyxQ<>qK@)BnQWN4wE}ZkD;7vQw*F|&66ftzBcDS|9rk|bn4+v&AvOf zoF>+sg6BhojFNjpH`{Ls)W7Zx6~6j<`u3*b>9D>i0E#(n==rCH(GZBK(+9?{=>ocT zX}Wm#)~zN-_K>3t@^C5aka`YBI~T(*3!R0c!%9X%N}@cQm6f0DX5jWRP@nY!fv@J3 zet9lxZz^tNfWW>1M7IV%3Q3=3GvY*U*!67Oy(@!$BHKufR1AG%1)wM-!pob{DLC1! z2Dr!0d%oM+TeyQ-+`-rV%b<;Sb)+o#mCV4ega9E$N%9@c1}Q4viqUsJ>K9IvX_hC% z7u>k3H41U6ijeZ)4m+p{3uV4>WWEtnOo__02nfu15-=BD-@Qucexs29s@8p)XxTuA z##n_Yi)b0nr58iZt62n$Ut|&gcx>o{9<&0IIPlC|vGIjMw+hBLFq=>msuStQdeCA6 zvjFs&My1{xG0bwZ4XBVd54mqo(eN07Yg#H~?cW1rG0|5t_X=CQFlffN7G$IT&J?t6-se<{MBCf={_?nE}{XYylyuREzwIddllkldlKQB! z0w4z1Y8PnNHTypUltLcOBg)RN9hHnz&VTY;@hn*N`OT)B^^qDQ{Wbj)755LOHQ z-yOB3MHC2UM#wQETCBo$tmJaGvX<<0U>4MN5Ry(7YNQH@0+~@jD6?GcpuEKrvp*Oo zZUAs({c?nVA>Um3W+yqiVn+nK>+k5IdoHvw{+?bi+@jKa7n#cZ2a`SQ$ri_>9Y?w8 zS>fr~=XVnoCex3~L=VbZFsycrbk1%iR%Y*)?f4saPDNebuhb~xd1SMjY~leH$fm@E z?WKn4vbKOnTjb57!%a~PGZw@As+H(g&@OzhD!on0qKrP(0s^FkYSB88~z3dqn+$%|z_OV+zBNodlnO-Ro@!7uT6VNx@_j=QzVR!n9C?1RQ-l{j;-5rI@ z#zUI$d!VCjkfTnc^cps7A1fEt1Y{2YWssW;dhf8c~0@ z$*zBi5ITfm7%+)IN7t_0+-?Pf@rCT;Iiq$&5Th^EoqL@8-+WL(G&i4JHwmq9 zRXA*yHqpqQKwSad+kb3KRkF%iNAxh@Js%v{o~G5~QYaHKRKWe=oSg4TPZj(^9oG>* zCYrco_=_)>5U?8a9LIJ&Vp0zpj}BrK4JjtZG0YGgBP6D0FRMp)?$tZT{d~s|Chlk- z=bSq$jbX33%gOA-Ic$FxGFEx}s#|w3>w7dtq$aV+y<-fFv^h65DUuy{YXZ?tc$^N` z<3%l{!rN1&57%c7*QwDy%;>h-Uy6uHE8}Z=uf=+NTjGMLH-=g%-}w+!)qwVQZT~Gr z?qJRmuxM$qBDL|wTvo<@1Dium0jEMsYON!Vjv;C(V*RJ$xJgB6r~6ckDR?{IYr8Fs z)u!epayb6SNf`lW1c>s~39o7S%0rrVS9%WDdK`E-B9@duFj5}{xmtx>&C3E})1nw3 zGl!V3>*;8m0UunLSaSjjcGMj|_x5BWI|jKNBO*t?Am?{7h~!sB*AOtSLZ5i37CP1; z62=G)3!WU`{lf}vWQ5A?Sd1`^2l@ooq8iX!{^%!&_4AeT$h&pK^b$gKWs4?vM2neW zb3tk3w+t1H`TCG)UYz@(0GjNL6uR^?pV?Ya(%>MyB)(G!^kwSIvQD@4M*jk zvTs=-+C*}WSWAw}CRIJG&K`rg`h`nsHOr_qm8qn9Y@OK5{~*el8(*utRd|hm-#BdB z*ci(sEE#St_X?fWH3-OZXFO6qUj!k{PG~J@PJs*p33QR5=ygOAHw$_GRgMzBY8!B%WmMqx+$Rwbv@IH$Pz-qw{f+>QT zK?jAH{ldJHDD&Cs{+WwQ{{@!X3dHZ@N$6sd9Kx63xP8|#V6Y{T*Px7Ri67C+F9SF_ zhCm%j^$WKouG-ZZNgu!m{r>K*^gLbVmHSdN!m2 zn64lI-F==Ji^2ziL~Xki1i|!*X}YDG>kg(o7#b6-!`A>hs04*R*3Y=s*CmfyEhIlAhDHe;=U<2#NO~{wVwQAC_~rMx*RQ8_>+JUzV#P18d1ZR6`h*4} zUqFuyDu;M2v}V~rh~_17`U)!iGyYe+~p2iafu<4Li;-o z%83iDy8vl1#eg4;?ZO87cY*}wY|l0Jy9702LplC`AqC+EPdiu_{iRz&NhW&vpYY>K z51pt38MZRyE5b|^a`{UE{VsWv}MUL#1`HPAGu2d)a z*@@U(Z1@Y$>fD0EoQZ~|(1s;qik|d-2vW{M{R_=uv5tWchH&6If()vIIX+{fkcdAT z7+A9V+`?%dfUlRa5VI#_xI%U1Tk&ojNbes1v}wY=Ai>mz3GM<<`dtN=qrQEZfLS;uCTQ$x$%%S9kHpB>~@P`0V=olq?$&)tHHT^u@Lf z$uEgN&doiv6+h9-)elErxhyaZZBxWTTzB4C5$4XC6=yV3G8-sIkQ-P_isof{@!!1i zYWLo3bseL7o71)lX4^zUq<(!4oTc%w*Yx4e$EVbQ1gv}2%3PA-wM8)8@VL6w%=z_F zF@j$j39rmH`5BBD36_~C?b+xZAKqD;A*r)$W|X**@T5$YLX&7UPOmPI&Xn2L@xu{` zAk)DThA`pm@Wcf4;ADq}Uy`tIaErVT0MRX%o*zXI-`S5wR__E&N!cw@a}b zFn=08)VoXwcm1_eqm|=w;%q=-jKr|cVa`}yHk>f{Mb*&9O1meszU*FsL=iPx*+#2r(50##sm0JI1 zcW}S%<@|=}d?Cb~ud;r>2-hA1;2M$JPgTWldWPBd`kGy9U>{czU)AbEbWzgf?Iz%I zt8AX9>jn1a($fsY4C8B~;0dYa5ByS<+_&#-n!Ye*7Yb9mKjiTWbpD?>i*jj zEkUKXtNe@msX0I9zj+3Z4j6lR`NY*KvAEygw-dZU|`hZ@{-fp3R#|aH1RPO+-LWjFY25 z+h^27rrclY!nPZ?^c($uzER`oZ$zsbdW}3qpE*gO#ZO-}x>>xjDijK42K+$dHh3@i z2Zr6cd7le_=%F26mthmRC<~~cc&G@%xCzMFq7(5Pn`{6a?WQp-OHTCBeT8i=W5 zmUo)-?9=vn617>h_48@lnSmxFhKA0@Hxx=kf$Qwb|6fIxjkr(i>cqG+*M6UP;2dEE ze)-0EXrU&5-A4`Z1kAnU8@05CPEGBGR~Aj0$fLCNIKLX`@FU+j8su-?n%7^ph&7$* zkF{+mf28hC^#$&{xO)}=Kr1IHHdb|`L3RuphG12Fpvt7Zf2oKVmluW3su~QOp44y` z<5-^qjP-9PmL9a@8ZedADG;1t53Ms(7@INpbTzA88@cs?ds{ibH4bcd+BT;U9EcGDZ7U(FjR-ayBWrtUp#NryVPpKgCO*4SJJ=3=QGV9!aCF zv#S_0V%>aRnL^XL#H{`CQ$+ueeC25ZMn|`cR0mR>bRyqz)O*!pGvM2lXZs2-%??}! z12z=x7l-mgDTDchO4d&_22Sn_-CKUp_Za7tMQIPeZ^5p!Xmso9AbUZ3@3(Q?8;uU~o6M~KC&p0pUH^ALCNkSb3JTV#>RLUY zy`uhbb=HY|pSZ#-@zas)(oBKgv#tH50ej&mO*TIiQvfcOxVG~vBu^k6?4LLAfGe6Wl7>T`;IZz1cvN3??tBMi zvTZOu^f4Ss5kMGHxH2<$`pCBy6LL8Y4BPYleAT?sR}nD!4omQ)p9H1DYuiC#>4XW6WMJfax?WGv#@bJho63Z8L0lC2 z34r_Vz3Uw$5P&#`RY_wJVl+Aw65#Nb^hIKGq1l5Gk|Mmzr_UmbG=Hf?Ci%ykM7KXv z$|J}Z_+{ekW3WSQ{%t>H+ts(K=)G6;2tI!a*k-(kHm>c|ly+9C-F^(-`_f*J)HruXu1f{8*J=jsGnaPVP*kbu z4@h4WJL7FI32KZB+p$2UTQm??t2P+hC)>Px2pZvfh27ofS7trpB~P8(_5}k-3I<4G zlJvnEUo zSN;TCZ3(7V&JPCCh2*^EcxDDPq^VVBiz}ki9klRn7?ABVxh9j( ziy0L6rczC#2DBX&F>5w6HrUwA;#mDX1DT?v*G-DCBtIBu<`Wy5K{G~)I}?L%huh$;xNVyU(rd+$V5P)6W4 zvz38yA6Bc60foQ6Du6z-<*tb|+zN8~z4o_H{DBtu+R9v$%xB{HT7|9?12m@wN-KBi z!Ha@todo&|M9zY(3x6pO3gD1GOD|n^Pp8K}Z7@>6u$JdPt$h7YQQ6E+(Sqi;%J z3i&6M>eM}~)Rpss_k16;pu3G^0Z=_sCaewogjr!S|r$c-(YI0Mr+v)=5BK8)BCHcpvu+>hi3-!c= z4m4k)kbzBpMSM^2#Xwif#urg{tFZ1miI7aG%Nk{Z)8O~Whh(R}u(`U>pMB{Ad#kg0 zHrP~*FE;f$o^M!_S0d4iJ&DOD&|XQDw;wA&Z4ESk}RjcfCp zB!oF@JKhI*F(X!}6=(BM=e>Zr6?B7qSFUlEdWAcaK^uKAFSh!-U})(j)cU(t$Co&twmv}jY`4`Ufm?;H4a2JL)*=x)tSu%FqG%yeNhL<(`Zx#s}%?Gpg*83@bh}0=J&_M}t6b6!uy53zT3FE%4$P^T8xvq0IJZ^x0tN3R< zL@Ig#E5j9k^I5q20@v52#qLo)iBU45SrOj+hA6ZUKjEbQ{_NS7mAdD|39lK_)RGsM zFt=u!`k659t*BGw>W}FAF_n$c*I!0ud%Jwv47jcVGkTzZmM2jjt?jFE=wgpDZ=ssQG`* z&+GiuFO6n7_vBE>bddhW-1K@=`HoHsk$Qp(UN2;%PDbl3WxgQbGa5>*L(} z(_$vG>4(OjhjIwd)Yk6}3W16F;$EX)!gCDGfEw%1#w!HnrG7lx@d~ks?wife0?vt= z$&NldI(O>)*hOhAFRo{kJ7J;7TQa?Q8XvT@*rDeS9|1ZX|0ctZV(da{&B7BdL5JE$ zD}Hn2)$L2QQQV)s-+U)(w>OR@#Ma|O54SL3(v9%c@4v$eQ=Jq0=FEd3U*vvUG7|l? zyxEKI9sjB}51{_*vmHEeM6PmVd>1Zm((~8jR=<5i!1nv~%f#NTGkkx^R(I2duQ{K- z$iquLS-WH!_)UF8uR|Pu_Hs+W8QqX>PEf=Jo1CGuWqkayl#jCA{cfFZn@&jASgg_e zPCPEzmMsL%j_pcg|CJQcBaYn1dC&(?rJkWDG6FW>5CelYZ0b;CD7R2zP?JV|=7kaO zc95){gdUmJCZ=m8jwNxBRba^3{}@G7k{4&|7YW(AXHwvt{jt~o?&*EE z#E;+Gg~ZcMf*NN9f4BPEPE!6Ok>pOIA&$~MKCFnYzH;DT?Ao41JNjGA>`z0v3{7p26xNsFoW&G~H zXeY%?`qXAM&07X3Ermx5ayu-u%>SKINv6(Q@%RJo>o&enpEKlL3|={+PU70W3U~e- zc5)6^mu@X^4k#BdHkw!)cA^74xf4F9|HD1>O%$NVxA@lFHU4h{`6R~2)bz_OF3gje z&-UIZh7=)u4b)Eor7vK$?I3DDYOxhzkrTI^lV6J?2UiP zagkB}ID61ohE48Ny~CVU;9PN?BRix++@gxJPjXutzLZ;n4g8vo5_K3GHsrq(P4w)~ z!|fYa?Utk6f^Ua*17F+NgnJl=G(-CDCB~jKEcy0(#KDabkcs%y5Gz2SBvZ$5nG<|o z!=ukf=v>N;&tejVE9tvTQfBA=cy-b_QQtA1@wLX-?bk$!2*LY{~Pf zp6{HhPCK_U=qwar>egYC%YdPb7j8l@VDp_LXE{bazlt}mmcfNF7TgFGn$&5>Nyd@l zL4jhup8Vy%(-= ze1>=sWr4O&Rw#^|Z6hl-F4NDcLU`E;?0u^@yt(6nl|63&1HYkL8?Z|-n^KOk92?9x z@^HUXmo_OdP2c~;b3dEcr%q_!j6HWrN2)U`BDZwiOJaB$JhWZLzE>)#Z~XjjZh{?j z-h=CdH*FIM+T3bt=nzvtOxIm!MTO#(n$mu*`Z#}Is@TtjG_1L*=_k_GDC+v*&Y!)D zXFI@Sss~=$Ux%)pokXjpx7N(s=R)G&n}fu_@G;NrZM>&|VwwRkzmzrldfp=xbF`p1 zSl|9y9pbEFL!q0(IgXxNRhZmm{joL@@-~nR4ia5^Ilj;Es%=j=@6Fj%wYw&hZ7&+A zVL|bkTljvnoM{%)C%Wc+090qJ0)9kVGiF1h&<+z?>{H|;vehUcV47S zZ(XID}z7wydZ9kJJw^N$B z-M(xLuDoK_4c;p7}mb`S` zLwvq@Q-WC1Q}`HIb7BAEB|ypclzRm}tuMUhToLP>+_T^~3spbEf8JpY#BIx)ez#-3gQXIT{5LtZ!N zPqo8Te5PzQ=g38##ern8pxc7I*nt)L-;v+SFp>IR9CODtMyD!D6xwSsJ(Vm180c zj$l$wL4uu+*F{F>oG0r`JRUup9p}6zudVy9Tn__}yj$b?HQo2CskG(9C z**z<~tshX{x(g>LycMu6&4EqSt6zAt^j$pDJqZ#aQF%^nP05zixyY-F$3XvIZ@b6| znG+bmTCj|$*XRmZSAR*v0H~E%9NA9ES0m!SyM&H$U8)Icq+^#Jr+6TA~@N1^eS!@`cieHuJcq?1TIX&GB`9DrHIz8B6xsY)!jHO<ZfUE>95?$h)xnt5a+`XSRpiX>?6|a_XN*=JR zNj6`otV{(Imi*q)DryjMU&+h~4UyyDmEl_*vx@v%iYm^l|BB+C1fT}Rn4KR}4s`;y z=H+^rxj53YcmKV4W})vy8Y%Bt9rH)Ogj&)ilaAI?0#O7x}MB!Mo4JW(}i&W>CQE2`?X#T@>2P?pjNY4By^{+%$JcMU=g ztb&u&jUTBd?UR3sEPQ)Sh2c?C-<2V_Y$7qQ9rwk!wuyUwg4yJP zwjfN$TG?5xwB<*R?N5y==hu45^P`Z3AQV4)5Nr;D8FR;ioV za}i9&=ac{pV#@b#WS+TW+tshg+V;idXyD+@E&Qt|v87-`?3YJtc@3)o)_wzBe43s8Wz3?S%C*{{DH+^x=$8F0*rR-R+hBe7)r=bc zVd&|6lGf#Orv}2wmCbdDr(@!*SOr;hhPwID%H82_akZMWXnU-8u&Fz}_&MTA0nn>w zY?|I|qxa#7VSnR6zi+LYc@L~{IIN`NKwJ@E>p{q0RCEa9z&`=JVzBmfTgGtoF;@ia zO8r1XmS07ii}^HB;ZUVf>>#re+Aj~YY|Q}z>kxK~SP500gm}LamD|~^p>@%j-&0Hd zHrfzTVdxv&f&1vvYQ{F3yCzZEukx!?vbou1`5EviIU)ee3wqRdKB8ip233YYVTf)O3BTdS}tnc zUaP8;ti~q%4mO7^wp``Kz!(D!BjwCa&8&VKt%+yN$Ip*Z=kY*7*&m3%coyFkU_wRt z)lt_34Spd)2@xsD&Q8hq*`L?mqG6z2Fy)K0p^)(yRt;bk3L`#Nj|ygvr$d&$7CIvDs~~vHgTq(TP8Z zB}19wcT84ORR+LD{k+?v&NN^5YaDFmah^N1sBcDeDf{G%ZIHn2vhCAr%j@q2cXq^3 zKV%szNW5vtd$c!w9Oht|8jD52>q=(8vVDVXBJWRIBXDx4egAq?kn$TTq`Em0j#;U* zF(QDvlYJmFKD4Q4ON*m$Pl)t^gY}A(ON^AR>G`^J#GfoKcSW<&^*@+@^tLX|BxdxNlcaBnwe%*Fjo(M|3TNjp|o_&l5R1kLvX21g3ZqQrS( zmrux!e5$wAaq+t-lK$3xM}>FkV!sCA0BupT=znTzz!g99x+8dyJk0nqzzKFo*vYax z9jm5`MEW8Q?oE;Z4zg)hep0n?Ea+NV>Rpjv^55e1>a=(lt2#q|_$jEbc?cEk|1qWO zehEQFy&v`9ioAcn@-U*tzvo`S|Nc{fmRj~@|8JHAQGAyds!vp^{OUq^UoX)xX zHAMMbscN#mNFV5qj$7Mny_JF4d&--G>h2!%oNdoU@FQM3dXz3LTg3?CtKvfc1e&jU z=0JwmR^||+@hP@amoriRQ;}t7*4f9#8EMQl(z~a+*_7)?Xa9^o#@?-&-N&D+-~DQ? zqhegy-*P^sdut%63O)C!e(=p8$~ zg+0uU2AR=?Hb4;1Sn8I@pLDVXx8{YPnF(8tt>G=!b4DWR3H-K)r2g9urwSAsiP4Q4 zs|_3f>b&@U4V(}Ny!Q#@&H~xgt><9Z0UHNzUhU#*NPgRodMbDxNA&Lv@tG0dk@x4* zZeI9W6AW|6O47F6F}_0RN{09grtyVDAAOdnU#|XE1yQ@^zs(0x4ASMd4);sO%qlzS zzWZRe#OxoyAhI?+!w>2biQYG*I{Av`Z;`)Y1$hiYeD7_C%Uk>3K?XlaEFYtd-)S1; zi_{NEO6(57RC_km7GSlmOa7Hi8PjcB&Z_S;`iAkVJEH&&y1A^>yCP=60}MRrUbx29a2=6=Y~5(+Wz|p@~AXm35l&wWOLPKl~N-Yq=NNR%)A{C@c%Dtlf1A3@E#< z9aZ5zrhr!9JcOnSOSP=sqq-d}&_!E0^`lcaNe!(1fzk_dXaw=P?ysmu^)m_1toS&p z+5I6aNx9#Nl`q!~sO6ytiIsbr! zm_gMHj>Ktk8NwRhq@jaC)q+-#;g*)Sp7n{+*oz1AShtC^^JTz!XA*5Au95Yfaz<|b zPQJ1s@A|fO`}vV~_1~|%uJ3J?Pbl6A6PS*?IPcy2wR(p2Y6DUs{j>gVxpSUTt8~H z4_Za*2`ziH7uUx3%Wq+7^W_av?KuNi-rNYh<-;}KCcq{uvY^3@=;l{@MG`T$wbvgc z65L~GpES}0?_bmVf<0f@IDFGi*F9gjIEBL1E%?fSk|HK=Wjp37z~}g|twq$V*V=Fl za$n9vSvjzY8!(ZkEM4h!$6mK5zznxNHjqi)-nXg z@ceI{F*MD~N)$)s@znQISf} zeBMP@u5owIs#MRqEZ2eETT~4d!vCgz|L+@_*V5e`@pFcbZjM2bYXaYU@LlT)WWm-H^OvE! zIV$sPS)|X5AmqvgEjMA03(}ojvsnI)>->EI0CY4Go#bA5^c)5p%CzMr*$1 zQR7ArNE8d7sIcY!F?(_&hvwfI$uX~b(`}MZ%K-wq!5yr2&H9G6FB>W=O8m==x^Xm% z40YEIPvrV|^c;%~10@-YCUF`Ney51MD4rY=n<*VDbSlvkc5&^S=muLUN6u0e*{(2) z2#fK5^t#*EF!FiY)$6`cw?kfiy*tj2@+MEn!+u~xEgsr3)aBBQCH)s=vJWe2&&S0x z->*8ZOBd~W-^~_rSo;*b$(#E_?D-o(71oMoGT9gC{xu&+x zo(pIgamd21w)0nX`uluPJ z;cg}IBWJX^t6pkv?R387mbiSbDL#{1yGhusCz!XKud?I@cXP)}zKq;>_eQE1;XrGG zqpItkT+s`o3|agew9Su@Kb~c^W?CO_qrfrXKap~5BJEVM@B;&0g^D$PIXLy7n;$&r zH?5K+Tyt!sCaolS4j0lc!cMo{$zA$QDU_BYPE_cK{Zk8KqeF|bh@Zz%k@wvD+PGam$e*Nd%*Z7_vzNJubva0$_O#xTnA>35o7vt0hKpr^Y%MC z5>MyA&m8?cG!$7z$kdK5s(-&$U}nAiGiCn!#Kzv(idc6TRA|CBQ!;O`sp83vXXjea z=;Ud?N87MZ9P+(Aulj!1PVXtvv`L;n2eNT)Op45lfJ4Yq*1|VK(x>LhVhRNGn3lHh zqOh-#9hD-d zh(yb>s|hG{aAl)(kaC_57ny5Dskm>ENl!*Kt zWy~ktvo|%8@+j%dt`R?C19s7205*5qJxLl7W2J8}lwW)7JNmZ})6S~w7yrC$_WGw& zt6Z|^Ip_4*h^6N}k5FG`Goa;C#P`C2hb#`+g;C_+QAi%|*4az0*GXy3UtE^fFGR2@LePe!OW_zC3>Mb5$2M%P?$u_rKVVR-I5c;aG#uPfJ9#x>Jo2a@Ytcz_(F_-1-S8~kp>glkBv^vizt_JnvN2pJ% zxNY?U>*Ce)s<&yk%zC2paMGv%Fj%)5zT=0R(j64vR5!FhlQZpfEG_gNPuDYqx6=?w?)sitj7^<7fSnC-VI zkj|#PXubx!!#zVIZoTP%Hof{}inU2$*=X%W1~>l6Qxf)(aVJ`ZPy>|gVZ4s8&Vv`F znHE4vbHB+{T-Wgy%$t#~42MKfKWfAHGs{O)yiQ3eGfcw;y#Uyun4=b^6Lo4+Ff}lyK&dBmNcvh zAl9W?bSGUz?5dQV|C_87a2tsZ*YK@!t{Y!0K2Mho~}=`#6^r~<|@MO15mUWBBnzFE4`8-!9F zUr{!9fc87(d)mMsxbIzEgvba^U;mf_{Iy(8M*}8*SW-*<$UL6mmqoFyYe=+4ps;(h zw>HYEe6_N0^9j7=>QpoENyc z1~5Ai57#MEY<4ov0Pf+ebT-RF?M2lx*&He6V{uuryEDnS`$R`{lJ3q&(XgwS9qS$A zREy=L4BCbGJ8G7qQ2NilIY4CsEw^A|(5Wj7pEa6Xi>&3h{Aj>I#K%n1s?pGBtT%@rOq(hpMTJz2ydS<$zB)RhE0rth}@`~&xbePu=Bk>P3;~vokVmkIrt0-jL{EX%YDC5M4qET$~Y^PAOjbq;mTt_}tX`?@5nt?e`-%&u=|`aL=PR z%DyCbdGeDDhlZj9aZ&bzin*Xh(mSgNLjxB5s*~kCm=2ctD|kiROfNeOeylksMyBEj z-}xFWZq2@yK+vbGv|wfgL1X^L-M3muTaWDumHma}=p=_Etqk8<6#jseZ+YCjWPUbl zS@H1a*cC_0@WFPgdw)k;sHr0y>_GHZhAf&de=~~&N0~f)7Zf&C`L2sJEFcyn3552U z5w4D^G-ey*1fJbbKbC$x!ETs(&WyXBd&Vxmv;^K6o%>hr!MxQ|n*fUbW>nYT_6+g{ zWsmPq$NoJ>H~uk2Ryqn$M?v=UWd{8#Yp*^VVRdFIxhe&ut1Pqs!hLqS|Aq~v*YMEN zt+v#oycv}<=}RQd>jpkbaUdrj_r00QEfQNw8rs`5SWM%Jxd63H`T8hl6Q!bfNE%$2r_75# zjrcubT$ZmRlN@y4VEcKmR0Ftay2CHMNOqL-JBUi{`FhExlFTb=_F$b!{rYeG(}|Hg z^}B)40-PqhA2BpgRk$@++b&m+f6&n4<+iuJK+DkkQdGt0z`=>SGBfZ4GZ3bU6L`0R zKDxO%DZ6PUwPNXGckpKnCa`luirSw4JzaIWd-t{sMsZ)7Ueklv`1Q$XZM-lPp}%g& z$7smg{bho;%*a}^d;owOZ_cf&ymK)&Ry5{q*^*)IQ(WiI+7sq4?4;<+P?j=-mM(W{ z&kRX{{Cvm;0T z9ayPP8F*6nxh<2B=o%zP7}VYyi`Sbo4ntnfdSXl^!Fy1xi_)hUWUF9zuimM<(a8=2 zBjfDA=C7b!gjPqiKH4&@OcL==zRyl{YF5C5LrDHA9%dHcbSy$@wUobP{S?3|3AiHA z7+Tn2B+yWQ$8=hSU-@g4z6-Gm!)Fe){G+)sYYRo&Yum4m)%)}dx31D7Bp{koso$bz zr1g5;hTZ@4or5-rFI}7u#Cqt>&g&y*Ao?^5sqXT0s|_g|OtCq$bEeQipY1HSFcuCf z*PF(#LH^nPwM^YA-;) zTdGZFoGsJ0A|!m+rjGw^%tYNt^bq>wQF2ohU^{@A zm$y0ngBRJ2INx}5;p@?DQNw_>yhGJAR>vcL`rvVQORsU$UH4Ca0MI>=g-Gt<9_&l^ zz{{HlVf!ue^0I4cSzfN|wu@ASdV>jAe^O4sVBeQ}?}t-{@*7-%0fT_VXuc8eeYC6y z=k3k2KD+jYrUQ&}Ey7~?F|2=`d-Bw^%R&JA_4bM6A)S1|;t!s87_TQ@#g|tfj36_(Xaf7mKYJU4|F(0N#n(w*C1oAKKa~_(+X3?BAl4O)k2!k9f z8n)rUBZ>YKmx@~2O=0B~Hy8~@R-(#kfr09|=ilPKN5r7T8%JTwG<54g z?7P+vx09RN%RUEoa~twHaOMJc$4c(=*{y%`CgoL$Jt&RUNXv6DPhJ>_lNGuD7nC~S z5D5q&po>rk7ICk!r9rw&v9ZdV8#Lc^c<&`0!0e;4M}ZenYAEox>>>M_P|bW&uf9Qi zmXb4FM0RE)-zn%pyVk#&AGWzw7(s?}13g}5gwLj;g_f$n?ifF@QWQJI=E9VcCt#7D zr;EMnn{CM>4Z%-8j`2sL(+5zUXP!K5$?yQR;@tw$(to7KL_1xEl#t!In%;IrO1_T1 za!Th?`>R+dh0_-2E5r*pKRVl*V6O*+ybOTbXT46(;~aD{UJ__ad=v-*K>=}jEde*% zYp&d|mbAd;>3(Q$EY7P~$I61Gd|CG+qW-n83dHl~$+`u(4Q=1|@nPvB!TNbt?%X*4 zX^p=tK(0UZ4<+N*)pjbood3FMY*Bps*=&YyTGy(w?Gac^_z_(}onHsveUj%!F!kf5WZ*+w}?b5Q&DbAcgG z>kMbXDp{7wa*7{m64VM_SoH?`O>wTyt8?7#k@{P-j~`uG)=GSv3oWwRN>!mxY9NSR zZvwA;SpAiX9aIeMT;e(?u**cKd8OYYiAuL+QcXy7JC~Y#IIF zX06?Htih@5^{?54H)4sl*TnwD{*_#gQ+qqwn)~&{4gV(_egOrXCG!wbN$L2{@;4;8 z^whnA$6n6$^6Ck)xihJT^;^SAKLsC+t}!t>=^kIM284!QGc_@qMb z`%eRbITigBFKU;baU=R1YsCm8Cdt`uShMZ&mZ`y20SC(T1&!Zu);fFggv|c``V4s{ z{`nl!#?}2;!nzASl%BWDe||esG3_8K1Mh3#8T~oyS)UPm*~zNqOk$IXXzH;L&L*v$ z-w?uG7<1l2VTrupm+Ipc|AUQB6jX`3ir8mE5kf1D*H$?xoyXD>u7k%O1UBK?!{=+MeR0_3PhV zgX;$H+3w2g7jHz5HE;;=%QQCU7#y9cTrh|4DdS&!d{?l9TmEyS?UarI6FkivXtTav z8c7(%|DP5h#DWw(rgBJcEpqbdkQD8K^1g#VqknWYyP};!{7p9g4Kbap|An59$)d-T0v-7w*paPVIE=$4#g!VvvVZx>SaX-F!4_B4-tV?f3P{+?v|%m3zcC zZQHjzf(?F6+)Gk24t)ws5;r(>ntO9MXa7S)qh6-91tvlv)5M4I{G{ZWC%kTm;*EO! zPjY*jZ1ewD)Om+BwFUiJL_nJ80YSRQLZmAoHGmyyiGl=?E?o!-JwWIn0t#yAC3GUa zcL)RoB=pdGD4~}m^y=k&&v);A=KuAqwfC&G*PfZ*JJ(b(e##LYv;s(<+Y&PB8YagY zM}6Kr4UgNsfFd|wi8NH<(u1pgpUT?(GoP`*eF5lyTLu&88^W!Sd)=o zDE{_CGOzz*m6ShDU_g)|gA>PMzqW;;LgM%DR+TU~ro&1YU3HX$?!C=*k2}M;o*B>m zlZHWP#g6xLh^j>z*G1vZE7sA*AuD?OeFbxFzufcNVB>`l+0c0_&P{XY%q<~33bSb{ zaeAUq!c)w^WMjY%p+;#bqswhNs+C7izVmtP5Mgrw|?hjM~A()vQ zkmwLJ+Y7M1EbN~YBR3rEoaOaQFz+`fb;+m95{gPzi{$cMf6X`(SN4V&#&AeGTs%$7 zH&01rx6ROSe4xM~Ul4N3%enK%ntraYvD2z~OI#BK6dPgOO>88Db5+(d8n~e)YhOxF z%5aZ^>c1R6KWSRiE3QM>MX?P_A_iw23y0YCkEG0S}UEn<69q9=9#a9S54C?*s zip$U)VYYXjtN9pnt$RX>9fuE^@!^3f?&#?fS)HRr1;7cLIg%QPPB zmNHKr&wgqfdt_eF2b7>0937OE_oGQET=&3z_8D`EU_c`0_5NDhZ#)|NU5bZol-;lr z3NdSscWOOnx_o;5uO(7;yoW7&N5qC6Hc8;Cq4ckPHM)q{CMX4)PM0{OT9WHeOq01KzIxf&0`pFoesfao7)Zx^Wu+ z`oqiC!Q$O@WmX`;7#qM+0&Dqc+(fuI5gy~Q9@QX(=Kcx619}Pc9 zu6{ZI`DcPBFQD);#j)SDuf(AoFc{9OY6zy}mKF$g0`jGhxBgv_c@KTbeSN3W_bKkv zC0K&!CxrO(9hRanSq5@3*5MQ%g@5NdExl3p0ye74Gr2|9Mj@s1(2n+C$ zT!jj+O=yI4P*CRk;ZZb!MaH+h5?u$%kF^;5JA9Shp=sc;=+bd%*h25E(c)vT{Wz6Z ztzo6h=-R}0^8;DZNwIZ8QRA}+y^*!tl*d->(G)X>dW>!#XnL??Eo`Slu2x|j0dR3l z3Mw^KX$L)%5xUp@g-&hP5*J6MbC@a`jLw~o<^|xFH4~*x;LBomUTba zsNZ9-x@9}u=Ab&6NlQ7Ykf0cgs=mCKR_K^C-79WP&jiC(JPnAJdUPEqeMr$u zewAtakahVWz>McM4{p*uLJwBLqfZ>6fjH{m9w%_i8J*_%Qb%jMAF!Gz&d;S{6|=!@ z-5cB?2F|BdiUtNyu_w%jRsZCNItLZne3)z3{Y1`h@>;rr!*$7fw3 zixArPRzS0^oL~3sC7181`ex#UAA;YxeopKJh)}0MU0AKc|0mZnbz6|Gki?4l38TTAal=E=V;_F9i8gb$dV4 zB|yOKih`|?3(qm996V87Un}|iN)}^;BGBzGBn#$5ua?`pWpe>wU*jL9f!$@-e_ZC} z>bI^1@O?M>GaQNs-7@HJLB!f?3w6@56y2H}sbb;~_?`Eh3_gwz$ zR7jyRxnWJ}NOp*YCsB`%;lz&ZCk9XwtRgI@P8FyADVJDX9}2=gNMQQ!f9|zk9CkW7OAa*r{_Tq=19<_f}!I z%wckyo_elL;nxBD42D%i4I(MQ7)}QR7HV~Fda@cfh}pEg1~fEmkUUR~Hpg=^88T&* zv>AZfS6HkiEf8T>3U7b>sH1E>T}`DI8@JA$DX@96=wykoyaM4y^LzG^444uM1L;1v zk%h4C4{vijL7z%AuJ$!Mh);e<3K(fFw=jzcRS^<5p%TC=-q%Ja7hD&H5_|HL6qV>* z$x)X144a&+{LW%n`hM#Pu7aw8*+p3tX>9j~P46#lt%|4wj62atzKpL(FTxiMq=QK! zZn5l{QulX1Eqkg?{NcKGJbW((%JhIwC20ueJ^;*kG$;3YQg%^h{0@dOm90cc=(STU zC;f&IKWIfcQtTDnzclAQPi8KVT+OPC$w;x@iE*#H-~x$1seOiqD}5W|@#y18{%XLfWS)gT~j z?CfXSrfwr#t|X5?P6J^@6FSDk`rmX?VxNACK4fNwZtAd+EPJuqv#!MAKm9YuJuU6Z zf?SbY=N09Be^%74HRWk{n%Pw0V<-V@PXUk8_ccPj&$}lbK$i-sA`YKvK?x#sMO1o? zNnJXas5MDgkpJ%JYkCyDxxaOwh#J};ce$x^>59x!GHykyVllR5^_$jlEzfW0`XFGs z=O?RTUpni@SD(yqwQD+D(jGL|#apFYW&W@PE{>^`SEA8EEMI%d?|H!$S1M`FhNtC9 zDh;5wA5Jj7)sGwZ+cR@DbNxP#0wQl!J^`ABOQ)N?;1lFC1n!02jOe_ava9Fz!;8y5EU}BhJo;$F z^a9^Me-}m*B*%w+kdk#d$LW>I`O)Y0pXX9l*8H1H->traOw3Ln-5NKZ%Nk_zfQ7!C z@=uaXaz*keV~VIkr*pX$Py^!%n{7NfVM4yV z-O&5(|KQh9?OYlweG}*bYca<;k3&YPJh={2%`Msbl6QL-E7GP0fIoqE(rQZFLc?!m zZ7z&M;Ko&Fb7?_rpQ~hM2nc$m5Z$9iF7ZSI{R6%yd;VM!4V5bRPwSngDyO;nr{o*S zhHptS1I-D}Ij1W&lXR1VCxU-QvBQ#qQO427S3!v5Va=i{{$-0F_&sw^yZ!40C&McJ zda>d!%dNNuLfJPL*+CBv*oC4TM_gjQ`H>90UThEi+@9Ar+6|O%dT51_Lz8$zw`_c- z>ItkpzqN}GgH`9)KYP@`SAcF261PrmGn8U-EGi5UW;m2yz25ds05(RAOOPso;>K3g zMMnWlNaFa0QJIqp*(fg0PBzY7rsC6B0%Yb3?-EFprd9)W)9}BK6XB9GO$bW9t2~vaW!;e_Fdb`!w z9qo3v1{=$413e^D|W#hTxDRWObs)Ta%);5Uy`0 zWj&r-^^AS`x9QH7F`^a5aECv8r|jf9>#3_qVZljm>j!#31A)9?d?s>{nsP7Nh}AVz zY?S#yjL&+*6b+GJjV)v6JhM7nS)gG{jd7f5O(}!S;OFWgnTd7Ub?b}zCaVdK{M9<0DZ@KRN%%l5fqj|3yD4)RJ$HXTd6$TCr?h= zMQ9yO1M)cJg7+f+T#rd|pjYd|-(?Znf?di_u;`BTtD?mKQQToV@!kVG6DMG#jX zd798`O78B4!(<5J@o_()_wE1cY?1~7D$2X^*FRtuHEdKoErsMhSPgru#eF4bQ?da3 zk`n-wd|kX?IrqFzC25d%3;+))LmJ> zpd@YSs|?V^)g7|rQUl$9e}0L1m27-S@V2_h-&mpa&xNN}=4t&1{`#Cy`PKeLnW@LA z*E;ni1d(kXert(}FXc$+jQpZbtvDSZ9xPx+0q-}kd9+*zdMMiNP7Irif|5{v?T1kh zx-g+9i@{AZ#bkuPK7FiXOrH$etTR$YVBArjl2n0OuH32WX}rn5pc^c#;~AJ86QkqK z<9Z~bZ#5ralfvmR@1F}lU66Kz$aZ16^_I7aQSC1CISautV6f||ae@rO({Aqv1pQUf zYra!0)t7QL)#x9bWs8_gJet=O4Vz zgSmAd*XTm64<<(ie}X&Md%I!NumQbLa_f)SVlA(fF0Dj~RoxRJSA)CytD3_>VFWer z@9%uEDFSg$BgV}%1;CBgAZc~m3?FA&Ikx10s&leNPnnOt~62Kaf_%=J0d&WEypbAD*| zF!!V~vJxzF0b)W1^~s#i=~q#uu2mznk8KAfPs4Og#uckFB~xn5$P4_DG7Y-&mOba7 zVuWi`TfLOjPhHib-RU?-O1>C&i!^#d7D;U&+)b$qRoX#yh%fV-*Z4|ZcPVZ<8R|Mn zL_1|AYak{oxGp!m+V*_c64P-verWae&fOCp3m#b@8DAF#ChIsG&7iI24V^RfIH&P) z$&XV`K?8v&$nFUKiI!!3N|eet+{b$}p*#AqCj*<%Hx=yMZ&f}rJ>Pp^%Yn{yyjLvshTUG%cDSXHGQ4t-|zPN+kbNY?(nR4)MLiL57Y5fw@EXl)NA@SHX zhqQoFrUQ?R_{hYl<1{kGSz*Vyx%vo-TdE2bx1g)&`P;jV&4WY(YZTfFZ<+y*(-R_qW%UKH&DX#Wp-3f&9=HUZB&jE@E#%c)C=TfY*4n}EB4VL?|RpYub?NJ(BV^4lvN8-;r|q>Q$ell3Kos%Js6*8eH)yj zlT|=XU`jp{yEiQY{;NYGwWdG!Tpbct&Q$-alPV|g`ZCr13CK!EACy15-@Iv)H| z&h08^lxwVen`qq{Co=on#nD3tZf{_yXOT=qWUw}s5yEJTPssJDR%Qd3Q3cGZ$X;&t zwr8W&hLV8|fGoIb+Bnx+r;o)%UHSDN1f}KzVgr>JlX?AU@m@;sUHE~;KQ@G8tta0T zd#9PE%(ld}cU_3{ys*&3TjWEAq#7#U6Of|G1+~&kga03kjDHdJ7k6%46(PpMD@1AA z1%$Web!8jHu!BA(xjirqXR9SIYq$>I2cEW~q9DE|tI_B~FuF4{^t_TOMU}1 z-~m0ql{=7%O7P!v4`!xVt)<=@4xZ(VQ1~W&Mpp;Up|GfI<>ZM4-BN`WKFd3AJT#)D zFsVmN&S*`Cenn(kUUi@R(j|B54>O^f$wZG^Upt@Cnwr5Y(lYn7IJ(zT%%nZ#8=5&& zc5I;PCnkMPX_iWNhTSU{)=pa|)#b2*yU>rQ$aU8mT8@SajCjF=WDZ%j_>%f3kzjR) zlR;V#$X@ZITl~|~y4RHfx?lb{cbn|$Y%X8<7o%v%4yVZFWGTmCyBbb=PIR# z%(H*m;4YEkm60L}t04C)`hZjo^h#L)!0gZ=O7Y6(?Oze>-;a$Qq_Pg(QXQw2szy@R zh6W#%I#|#oF(ZFEJ38gp$L84k%V z$LE>)z>vRe4tDia_OOg@iN{+<_iu|a2R8cj*7(S)sEw{>(F-hhKk}~9TUH24viPT; zbqt=OFNW}_f=B+?fg1LFO|jtkO&4ORP< z;Cq?|an8#NOxVK%x%7!2EI?(*^)DNrYqn3#uMH~n2$+7~Hon5n?WI5wQ89EYXd>K@ zM2UD6Mb_*fv}I1OJWZ+8UjA;cyP4fsS$|AWHw6Irb3iL~9&3qS8E)<+*qh{C03TP9 zIkUj9C9jYeNTy}>Ueq_H&+Z|ALbtoUc0q?9i#G_$)2B~g`+Q(ySC<1C!8?Vzf^0zwdE_CtSMEOG7D_*XUuFE0n&l?K5mjVtB(?U&2$(10}{958%h>J+J;8`JJ zpvJ*_uu(m@8Lx1V-ZXtF;Kdt*qlrlLHvI!Imzy&qJp@UOL8V_4x@+dHTA?p>1$=PX z#5IsvdBf!Uqjhw*4p20%n)ado40)H9J4K3aHvP=Z0oTZXYdPNDUIz00vJ;MXd*kIy z<<^z>U1BY^f(t7s{a`fK_1U&(zOU5LqdDiQMwL&YBpibv5EOoe6RD^+Nh7#bncz}7 zP~-8UP@RK&E#1hwOqD^&e~z`u{W5#W(roUA&Prcxi~e8t8(QCY^BI}r>*C|b#q+Ss z$$8Rez(lE1bC}Q_6G*vqX6l9!u2W**g-5U3q}wFz;kPQ9pi-L|dOikdC7|R3|KgYI zI55hyz|)uZc{ecvvrw|Y<3ph z#uRhySeg6s2PF~R97W>W#<{^hWu@j0rr9bPU)#CrolM7ZhQ`Hm)_lv&Ab{^-nUx_0 zadm&9qyCn*zy>7#()Uc|P8&G_b;};?Lo-vX)mbJEzQ%osC?KN0t|mt%A{fEJeb95;SyM=8XamNbk9` zPL&Mn&^cdBW}uROHkncq!>Gy-5_&UNTwgvJV6(CFsN5^=Zy@t;L_33IzSks)uedoz zTH93Gui=B|Z1`P@{ZqI5stJ%XzB1Kb;{E1UyY*Y>=7Cv<9AN&4J77owt#kar$1ZMy zbIR&|Zzt;Qr1Pn7@hV!vm0;rj_~Ig)H=*U9qR0W9FBC(YZ7H8-vKLBO27}Il#eVDjc@K*hc0G5E@dZPp{o~>K*{b^t_Os= z?|1Sexn_3D>@%ja-llp6{}ZWp*|PS4??Yojd!uXi-%4QvcNDR|142P^4{GqvP7D^s z*i1_!G7G2Hh8!8O8`0A!fOfG9XkBvQz2R=E13b!p7scrAcFpv|dGdPox_i$PCt9Hd za1D0Rwb0sGmZ!No`SN`X-;`L=VA-$&gY{&Ta%)cjvww2ttoO(-J{gsk2%?h+MT zNzM6vbj%VPKTU)#4ImQUoRJ56*Y2m)V-|+gRe1LVUMjwC?4bR2z<%W+`!)ITJfMKX z@V(aF=EW#?Xq;uTVIsO69e1+AFmf%-<5HfXlInD-p|QKqzS&t#DGiOAN+z`5Gv`n3 zxDe@=f)}VO+|CUMaXNPrDQP7&7tqgT?o?IX%pyvsG!Q#zTnOlkijw~49X582r#U}X zmu3K;k{=)qJ9o9%V4Q|AKS|5<2*#hoq_W=m-<1JRRz1t1Z~Mj@r}-Mip}4m4F}{Rl z-@*n8TV$x0Gxf45`z6u$k_Mjl(v41-f|cn@s1P|fTok)Q;Pve|>~3~PVb{3HM2p%N3=wBL4Z@ULo&k!?#KI5Kg^UI<9pxn~(!7l3L88 z90sq9%coVR+%1cljn+&pxw&bs-(w+#YEW<*@USS4alUc0Drr8q`Q)p;y*I_yw`&iL#Fwo1y=MKJO>Hq~y=>pxs*^6yoEVd+Lo z((K$w7(c zpW!yK@oQMII;&@Sq>$s!wlA>Fq$A-OU>c7BuP3hecbO3LS`a%ErT62qd}h$HZ}!;x|Xx3jdGy z1tpGa30eoXqps8meCRf-mA*Vq6PLdbte0vKTSQ5;t034~E4-J#Kp;|AR%^?ta3QuM z#wgPZEhr~Y0BtkEDSZg$fgfDX+BC1Ie0kbhjA_YYxO9nA@4p@a$^=Fuzx?M?t*a(^ zsoK||=3oR;n`1oA8+A#P5(bD;xpg$4QBU0UeMMT+xsV}xr1rwKzpOl4zR87kvV2_Z zkPqbV6V~oKz6dH!lFtLqVSJ#J^##Vs9dGg!lqfOjf3uWpgeRHaK4?3dT}Ops(+VVE zQoaB|$}5Bsa>1Smg|1H6S_1*xpzDdmpmkG;_KH^XNk&I-)ToatB1)ooPUF%?8UUwFRcWQTyoSWcDmfo-~PA}~|@8v}o#FJm@IBJ0 z$uFW9dZrmj))-QW+K0j>7$RNH;=qlqq2V))Ss&qJ+b+#wCIQ`xpduEfMmC$qG*$TS zZw$IOZrb*d8dK0;C$PtTvR}Ez!e^E15W$jk5jJGo77y}QC>d|Vzc{C+Yaq;Xw6(xk zG=tM@O|{#UwgZ4`@YV`r3Z}yzIl6xhni5o(6*ucrs(UuiTzbwaVvfG`)3=BHrBf|L zdVYZTj_9k9kpcEO;E2x&%OzO(d1#FqwN>2I3FFI)<9q7r<3AV$D8?F9Maf8H0 zuBxmqW=5&4iit`pw*eGZ0KJP`MvCjD1TD{j(d`&I57iy^;;J_)PB{V;!$CM?VxuVf zE1dG6sReVIJDT@*h;}9`-=PJs*%?dklB$x8xTos7jP9jGZ1Fx)oAMpA#@j>{?wEd))7Oy4CsvG+`jS+Csaofj-&&<&e{vSVBcHjCl#p zXboEB#1y+FoDv?z%S>yCwn`#5<-3A{=-6aeyF#Vi#8d~#fUg^jaofIBR!-;vyD7J_ zsEeryPm>O5&`~i!ED;ZYZl*a<6J+p4(#`J#dKRTOYn*on3Oo-`z7h1BS!&XhDVXsn zZal-gz-+j*;?j(d@Y&F2&hU2Wr%~gr{ll?IQn)*%yPl}bn;#&@V0lz2hS-hK?9!KY z5@nNh(vLqOQzo2$;awhNcPhsSseMqH+8zH%hoQzujO)(c1aK4Z^9-C(2512?K>H|6 zboOxjsTh&A2U1z*)Go z=YxFY(t@=4KuGFWxTxxA^h|!y*7|cVeK~TN;HlAi)%xC-sIkd;`l=A9OJBoiuQZwE zP3m=pV4TfcEO>9}AU7bQW8?{gn#-^mnxy**xnTTXH2^M!j9=Y^67d&l$~EeEeBVdl zLX0AtvN4kw3YM=VKX<>?=lG(O16aq#22{b-rCQm~oCUv>KGH{iS6yUP;T{8~nk zWkLFE2SviF?^ib_&2+6xlp7~vVb$M0!AwKsZ+BGBTyvz>8UxoC2(}`#gWjX6-jgac zlOy;^xrxG0&Uq?0N~rj(%Q1H3Uoq^-rmar%|G23OjnEx|2>8tkDTVZS)EW8(YaHaR5AmNyqgNYWD(#7xhD+2ER)!4) zOlovW;CVKi3D*d=tzjKmDJ74a@QI4CV1cP|i5Nh>Yau(rYPA;QF&oe$>wf9R$$_V^ zACcEPb(xXK4i>a#_fDNUQyHIhg$-u1%G0}{OJa3p-te2uRs1r$rm`v*|An<*;Wcxt zf>}VivmI>%N0e<#*AI`>^xl)p*Hk(`tP&IyLG6N=pK<3D)ff;r*n477;@YM^Sk^cX zEESWT3FIf-Rm`pXYWYd3(W95*bUHr_EnpOLX#&L{Y3wqfJj>+o|4@s@^9mxQWrcy5}Eq zp1a?Pk)j>A9qruD`g-FF0P zRer#NmatTmZw*gS0eRkSeah7dugDg>EB^|G1yv_5M=y}7qp&ZR5Xk~dE4qN^#vNn% zN=D9`FI@*3 zgVD1z>t3kP_W2vIs!Ntqwtbkhk}F!*%QBl+2o5f zWL$`@s7%}JGdis8M@fgO-9dqSKNzdwNy?82*Fx@_`Wy+Q%bl#-+EMBXHfG- z&6t7Qw!yEabLsI6@ZTPxpmW!bQAq=e>o(?orS9p}_GlxNR@U~k^ydEJqo0(#ZEc+x zo9Et5Qx z_)QPt1>G6>-D~JpA!aSIuzw|?yi9G@bL}wgErSe&qvB7-^O8lQ*p}@w4G5R2ReS@T zY=DCDoVV!{UMAUH*;0zdjXi)Vv9;@yGv3r~eMX~wQO&s0imU@nMp}mkpk4drgcKoT zi~9#fxfqq_&(#_>G&mZj4V_SS_R%Z>BaO3J8UJEz(l_tywuVD?l9hQ>Db20iM?%*j z+bf3IkIl<9)01Z`i53j6(#X9p-Cik8I`+-_U?9AjXA%C5E4=%?z9%={LRC9Fa1CY3 zjVEX2)vN3^+R*z4Ri3oNY}_pUdvRYC40541xG)Ro_YdZz@7y;5>mBTu{|MjRabXli zEE~0YWQRiaK(kCeZRXz@t@oJwAoc#JYmn5yYB}@HuW>}}*b}pXWCE1X#nk|12YsLk zafIa3aYoTLT=Z->GR_u+99FxQ(E+`VyEtYB3Rj1EW-BX#7~`6B7_*M=B>shNN4o9( zO1Q+5%KL9d#vMA18;rsn5B{M9QQlFoh`P>meYa&mND{} z7_E8-(38fv<`x8@(w(LE@9OYhGcf|DJtL2F;U7D99X3E+P>fxV6Ln4+`Psyz` z3sUUcb0L%U+px=@1;y6Z;9%K9i8oH>tcOBRaX0G@uWUxsV0?@PevVoZnw)u6x}S^$ zqilYcalx3z2M@@dwd77Sqb z(;Y3NL8Cf$f$oOwYO0&_dPX#sI{bEu6+{mYUr2&s6$bc?Om^E{$Q%4ysu zu9aEid9HbQ{(T&Lx4L%k$RIMqI>c6gRY;D5x4K3)fu+`11M6P)D!cSqqgtG8zqk2r zhL-lHeyx-OBYB*`<#eQ=sxR_^v4*bE)mLZHX%%?#t28cVeV#Vn*+##EX%B`}$`gLd z?;b<)p=`R<7t5!-KZR+VWHVm^ABClpb(hHdDQhz6G+^~+LuG7ho1hJkaCu)V;fdc} zK&;(-Os5EU#RUVyV4Bo{A@#IUtyjIXce5K@Q2zvSFfxbEyOqC7c|>KGKWUop8F5q z_4%>#Hap<+ebYsyht-8cx;7z%2_u`HYNh0uIl8n4VGpYoz>M7I{D<*a=FNVFsDAdo zekqT=-}h7JEo$YC{5y{2`NUv{2J^<-!m0YWE1yJ6$gwg+{ha;w!GrBt!HX6{0!(Nz zr_w+xgUoMM{_t+tVhBy&4*S-T__9>YH2IvN6k*F9PF*sHI2VE(+Nx^UJp(2o4Y5W{z5SN!rIzR0SJc9%hRj1CPPvG;^%%YGyXI-qho#BFjulUBpatmhNUU(SEuk52 zPANeOtGm5oPTC>96a<5u_-e(8D!X z)|EchzevyH-PKah-rwo8N;)$>5&MZUw&gOjpJ;iwu$6wXvT^nN$1$a0(y1}M=}(tMG6JhsfTeUu_?@iUF4w<_>8*b4b-MCCe@6`uvLdr zM@QTnQQvGws-cL#LEJnYYO%81LUNXYqxsL1=3h!po=_LhJvg>I#>La)~e#AZ@5HV zTBwBt1*G%BRq!IU}dpy~N+%PCf_#*Dd@3&|cp$oHN{2O*C_=5Bu0oe|siS=#dj zp=N%+M5czO0it+!pPHT!O? znOj^4NpUgR7Ct8C>DrzVcrGd>0+Z}2FWREWXQ_4Jdw`TM4DjDWH2fVHK9{_j+Nz(x zy6yKcX;*Fb@Uf@UXN4w7)2&oV%|X!|VbW&Sy=VPT5})jzn<|<=qK6;-_VGwjU^iCz?0uxqaYQ^6P8Buk!*yF1DJm7@m9F7&fjXe8#3FzmT z+(vel(_!QH2{X)dh2rzsqvta=XKk$pM{BXBGrv6}8SZWvwunb_YSl24(!IBBrr7)1 zw_#779|*Hk&Rurd_GQl{Ot+#WHM_D&-&pNsxq3hf41a|0m0NJe2_o&5ubCcmk8TiI zC$fEtwB&o;x@D7hqJKd6;rigiEhg2_;@z-y**1Z#Bkgi63YpH&P~It8wMp%ig?1Zh ztw51RZdc|XtawNa%B2p*Fsq&BI-1+roHnNJz(kb>WzXTX6MTaG3Ns*qwso_G1uALo zu&ko-<3w45(wm^$EBjmXNRB~=+QbajNj3>`)y35YY zPH_0zOS6DBN3M-eFyLT(EjTgP;xG0{fbV%r>U zbNArv^cQ6lLiko?b6ISP^Z%QX*&lyad&ioL_Za6C4WlUvx?nBU_jvV90o3++H@=qm z!i$N!;qkUzN*ova)SJEi&B-VTHf3{qmcRRRkhwi{`?ur&j^$69F8nC0&QxVg5;jR1 ztEu$+dv*|HYYZDdnj#8G{Ds!bet&!orlG`D=hwbzZIe=eXKoie z5tp1TJZkBD z@T1JP{Q#>6Rp)N*t&hg(7n|@$nRa0>d}SdLw&i8NUI^oou&8n=Vk!7voBEelPz5PJ zwPY~%-(F|>?RSrPXKFhB9xbPK3HqHF(Mco#5-}d2EpPVqcr}3-$oymIr`Dyy}kpD496pdP9 zmAN=w0)Yb*(Qw)lU2vclCBQ#&JbJNDfAkB(j&QN`u)RM`g+*rhu+|jw;{U%uBgdMa z+CRd5xmvnaH`E2JM3OnZR)sxDI(5OJyy++~8h0}JagzVN;O@jrL&L&0zNAGUuDn>V}YMSDs8 MCFn)bGqb?|1H9AZ0ssI2 literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner18.png b/docs/images/LogMiner/LogMiner18.png new file mode 100644 index 0000000000000000000000000000000000000000..0dccc2e9160363eed40984ecd49e4c9ace76ce92 GIT binary patch literal 31839 zcmXt9doo6WE!SZtR4SoD?xw^l_xs(9kde9O z&M=p`ZZkIbpU?N4-##=m%UfISNOarBFC0)96t7r<1=N4}2O7@p#)=U>%; z$9fXqwDyc9{c^;8#|It-`Y-|oU%bLqyUCtJO9trrjezHb*HKBDQQ>8C66k&=jt`hS z{A-+)@7d-D5Jb)o6sm^-y?udR^xUMRNJH81)ih>jkz$n_F}nIo{?TpcBL7WjSl-EC z3f@Xw$l8`c>r4Dx*&p?l2l@iz9eP+JwdPe`{nwk_x|b_2VK4%HNJgV*v!Ylg35HsL znb|X2M6-0aewsr9zQL#W8mIS2D^2SwW+=L0IGtqIG;e1HC9*?_{`0R6C7Na=%(Au$ zV42z$C2V;Al6g!YyhZJ8kLr$799V0NR&Ea>1OpD`SB$ZAWAnoWpTh<9>#cl23w@9^ zcXOTKR*5M%Eds!dhz=pt-A*`68UmI#l=0YYJ=y>;YC1CaSFndGGrN1(qrC*eP6i?P zPWW%oRG;W2<5pvHcb7g%<37~EaQH!|-qEHWdVfyuaIO=3FpiZPlh4Za|9JG@s}qJB z7Orta5}vZD2Ykp=Z=_GwH;&h5HgtqGbVNYYgq8Q5v^OZArGlQ6KkEmzj77U2{1cBO z!iTKGhAy0fH-|H$!x4XuHjWNAI&~UabtE(9ouoHY60pn|tXA_8$^4K+{4vdJ*OX>U zZlAk#bTEB%Ki84X?a4P9TA`hNT55m4LZf;E7{835|sE6N@78Y((?D>dXwI#M>w#p8WP0(ze$ zBqXql!iger5sL-I$cBEWFLSnFgv(5kYW2FQy~#FewF9;4$iA^#v=Q>7i*$M#u=^FD zz4Kpu1_JrB1a<>jT=iOa2J{XFS~*nr;pEHH2Z@cjn*!sRhu-Y<{EuHq)Qm<%maR-q zA504NY^_Znu9-K@f|_Pac<0ZekMKBHld){^qh@xghv$)zxGu8ME7bcE;_4>Yb8qwpNG^A-aAG@pa zsaI*}W^e=Uc{%gebffw~TU-)Ae-w^4pZWHQUXs=ti4pkHP^IQJcnwj5;9Tej$cUpJWmx<0_)jeUGP&;)f1X(lOiikxY-6++Je zgwcfbs6w*gjG*u!T5|YKvaE6#+1T8EZ)`XVN#PEs$eA_#Qtb|+mH%Lv5=(Iyt5 zjEcXM3JGdGcCS9V%4|E!KG-nexnZcodRXmv^U-LtfA)X)L7jN(u*AOp50n2uBtUR4tgx$h- zG2y$UnBzSvNSNVV_-_wu1y3`=rY*)`(9KkC!z1M*nzC&8HY_~QcRGxb*|2O`tQ}^s zf7yX{IXYY64&uvEq-X;%JE7Tt8}}w_Stk1cU%Zcxn8u)%G19-?Wt!>l@Fi!8Eb=J3BD32sRA5;T~iVSSw3peMSNBHI2ctK^kOpVh zq|i?+aj>wc;bG+8VMzo*7`1k3NM!2Msh2a>Z{no{0w_hdDgFM*yxl+Io-jktC3MnUcBZH9xxo5p1ET{feu36-J2%wRy`+j;1KS5oNvpxf$@?W6l->uH< zg~Bk^!Q0T|q!Ae5M-08@q|nFNO??kBKt3;mwHYA)zR%C7(X1*?g?JIYfMNm0boh~e z-Y%j``;e;hrQa0r$z@AA{AaUi`!S1mWYO58Jh+>SSa;Zb{H-ee5&a2S{x78xbY?22 zhf!<8(xl3S`%?1B&s(jCII1dZj;Kyl&W6czD`_{oRO^3c*AFyz%pi>Y_2&Prdu-&o zz{l3Eou3(@<82xBYtAalHNg=l?DeVtZG6FeDdc0e92Nn+sq)fZUddznpALf$PWduU z*0Zo_Laag0VNb|Z?)$sCpGg5qu8{u*R;{A;XerNGwwqSIx#NR^^bPow*7BdpTA|csyg*%&1%Q93QbSp>b7;{%%lI zc&U?e+XV*>gpaofgx+5lhsRz}3E6hL>}t7K_N`oM@9*^f-)L9^ljqSuX_x3N>Z7h^ z6E@w=ZMKX;WL#Pk+xM5wx3i=Ph;}uBh+8b^ID0VIDvvEoiesy#Ytwh1Y!DuhPVnU% z-&q~{H74*L(FpPV}4B^4!}ss+6eNKFcJIMJiFwCIbJap}h>k3FVd5 zLA)j<9D#&gh-MY-pFQx5*L)6R)%FReNxsAUx^mrXdxyT^m{+TR%8|;7&>0?cEib<5 zd=KTVYvP}+%?!7`Nl08Z8URckYS#8{&@Xk640OuW>Mx%DU`@|bdve&5>LAaYSeT zS_Y&>alg}NX!;<=p8l;h=hIlwtj|0~0H2?;uK;p*gvFT)CJ1bt=@z1V7N2a0_r|LS zpBh>0xu+f)a$9OB<<&-54P(Ze9isu4X+7=x5qyiEgbGN*<+7oN(uRiKC;Mgb|Lqic z+)AKIC%i_{68FW*j|?c=bsmbH%19WLD?JFGCeZ(Ru;^!o=rXXoD8YofIqR|XqIO+Ea+@l@HCvE0lL-j5077o+Gwelz_ zo@#^T@=LENB%7m!1@uPVN5NFczxlG_EKrDCkieJ|lh`MmXk|;)0kz2-b6C{h3SE>? zhe*9;e)C!zbVI}DXUB%Z4*uWu+I~yDnt55X9r_Bi7U$80H=eN+`K!QJg|QyFzXqa& z^;eEJf8n1vSs^Lek=&YSW_V?~QoGfyxvRJW?^`kR8#c?8?mq8PH%ruUDPRccNu{Y@ zY1L!?HeG*pS=I)}6UOv_K6ckq6V!te$rly%X7UB_K#J<0J>wk3kior zIUf{-mxD`qiZ`ssPsnN`a4mlqCX11{mUMp_sKHnDTe=pllh^uDlXd|Emy;QgxDDWKqAoJqI5uI`e%+u@LSAa;anofnA`13iLS~RNs>mU*Ds7uh! zcql?QGN>4dzb8%bkIOsTgk*GnG!POP`JIpK@21fz6V%gwZhkNBDc>*9Md1@Z#X0u% z$vjCc4(X;DCEHWUDDz@$>kmSChvy|Ni9dPr=?psrNLlwBI^VPnl*Z(Apw^@Jzx_1P z?vh2up?Q+6HJ&0k#`|F{c>;YVVofu}?y9m4NSDx%gN|8OfLfmWop}ir@w;+B@$i~l zIHl{l^%Q>20&34JvMVKLd(u7_3SZQdnyscqO7wi^gLZwc#~X1|C|%YfUaH?`63tz; z`s$L+i;%uRej7?ZBFPM~KYMipH(_qGL+oef=I=RnP7JK9-74cd&|TBjVRi6&)@24n z54oPJ7Lpa4kON7|f%{k0CYS_AJ6rYKnYv`3{do(Z?gS#&SZfl*oj&}XUmCr9E==wT zX-+WwUjS`lb2wB|=^!+xY^o&?@y^^=a(6~eq2t$$p0}>ho4*Rq#)-a@BLCG~B%+k5LxF2SHaGNk6 zNegO<8o0Qa%-9OdY ztD$gQKa9-4HzyIOFcU5RchZ>~r=Sz3iaPbAJJEFy|LL0xfO$oVosFp@{a&-{J1bx=0(n>>DG*rIs+_^$3S7OU5cQAs zywr&Yj%jbQ?q7Q;{uS$ebsK-BBq>@a_Fpr_J&kqQx$LZA*GpONsEZj+s=a2T6pHQB z$>bs}l4XbF?OmpG=jcQd#(17w7f`H_N(Qdo5So0H)v#o3Pb^C6AW3-zJOIL1qRjVy zo9ix}Hltyn)Uko|u)xW&78~`ie$2RXa`~dW=<-C^_WfTb&`^E>hpCx+NOKerTA8!w z97(-7^IL6#GAo5NtyDcWFHnFU zxlXy!ZL(#0U_}RLI6bA}*b%O3Htx*Jf5wjdp{w(eV-rK>E0g@2l3nibvVX$8%t*wZ-pPX@o+7kPy7oK9Ce<(ap%GvyU+1$R@@lxWvi(mh)cGK(fwtZQ>-4oSp8q3ga?vsI&Wf{2NE&>Y)|t|F3k)3aBXhvbSLDqNl%tN`f8wE zSnAB92)+ymfi>u&NKQZv_XV!^ti9mH46V79lfL8nC{{A^MR@@10pD^D6qbU}@);+d z{jTd1G;vuP^^vVZ4(_b_xC4IfC7yG{-(@21WE()xFoReAz;Mh@+wAxKjI7B%Twi+U zdG6Q<1}z!85AV*$JyZFYLLNUz>+y}M^XrkI;2n1RlcoJNvpH`Qx`tNz;!*UkN>+7o zq4O0Pm?DAYWnEv|uZ={hli?B8b}#HRA}!0cKN^a_o@(BHk_6t&h`Oy7p3B=K?rw4P zGTK;H6!&! ztS^m}Hvc`G94M?yY!8ZiF_&OI2Qc-8+*R>5?d=4%+W4I1}+?Zmj`a-71_7cn&oB zK{{?!zSWB>^TPuznk)9u!K);sigTgHTtqg2f%>tLUWHV5b&dGeb z>5Avelln!&5@8%v$$_urSw+-U~|ua2hJq9f8%c>O!ojhf4oCpqprwLu^s zz|i){@4KHbVDTKs5)I)N8=&fcf{==lX6kLHmK)VOm|OKMj#3{kH4{3^OY^Oq8uEr;aEctRveB#viPig5U?UH zL!^v$9)#FF_0WqyD}Ai3OJ!tX{oDZoWv|_D087%dzxGv20>M?6*UJ7xT^`_wTCdt_ z);(%v&kFRdl^rMj)JTgsKaypyxY(kEiA^=(&GWQ&;7w%va`j8$_){kd)09ni%9=*0 zsLm~>L5@P`=1834r>Fb!LX;gTW?xD1Au$=!8W3z2A8o0FRQ{q742;%O0n&u&*E$!u zO`H=>>oF`GK$N`=?9urc^BJtk#R|bDN?}i|k59oId!9Fm5(*UbukcIJb>i=~;gIPF z+f91j3OwxllDC;Jt9i9|o#MB*Gkayrqh^I0>XwWRC0Ng~6Fh~L z*^Bh$E8Cny|C9qF`qMW^SIyW@2^x5R{Co5$ee|50G-8w>l+>n~ zAXpA3{9Sr>4-wa+f8vfa6ISysCWJV5H8M$G!0jK|#wlOa5T}pRX9uc0m%wR1!j659 zVc|E2ELrRH*+Rx|#GAceY)BSurVc>8WX8UT)(n|9n%EP60Hl??iX8SDF;%HL%7@E& zkiL%t5;bGS61DWq!RI`2yWQK*FIOx#OgkvaTA*Dvw_MzXo;p>;tQ|{gh3`iT0g`RX zp^nL`#kX0S@)gr{%k-U;_f(n`Me5>}0nHj~iESOs(r+Skh=oRku%n_u;>^>osT`$1 zk5DOru-H%V&lh5=AA->ah7FVwc`MiS^v7_JkJqd41}1pTC1zq|&}^9xAb^Fb9z(lH zIPTqhs*A$FfQxEk(GtVMC(6);!!^M*1?=1I5*n?q3KbYF`1kATIf1Ow%FBs;k0R8@ zvcRn{#L4ONr)+RUK7mM6>{!L?^(hlQ6!4TIn}D=4FNMF%YLHFP2Chotlvd{wOG#D5 zT%^d#Oc?2OCHg#p>*g~aD?CyW=xbgb6@C=;pq~CM``xSa(;{_7ryIv^prZnYUMsLZ zu0KJ{+Dkw1;{aY}rfvCEAT&Zd2IAhKO2%=gXJ;l~EiX2*sC#|&yD>)Cwhu9xBt>mR zgpQj%AMeUoZr!~v|A!oev!3S5nv#1ekv0FuV>J?Ijdf>-b7(0SamE)>XZLwfZ?G&D z-nzknobC7RUGVl(64qRLP*2Qmy@C%PHd;w;_U4KK-4wI+1+Z)4X`Srv7hJ{prVEMpy3< z3^}!xZwTKJYPOj*(n}HX$v?aEAetL%_n7_- zE}mMF~90rP1q+^*H+iv6XRg#}D_n|6o!~%I{ar zeUg-JueqvcP4xWBS1|fX_=>}`7tKAU@}plFx}V)&=YI&Zd-xDAPolx3gKW3XZ8ptq zUrePfODcy1M2IIW->ZFCD_m~KOKXn|e380jgSfInK*7gQngX4o1y6d$?-lWXuukBg zyDznKO_};!Ii#m5b9Z&8ORZ?#)V5WCppKydN&eB`$Q5i}cS_yxr| zn6)RN`VWZe=|wl##OFw29J~mwFQbq-bqh;i^9j6v3r75xctvvWDu-AB@T}DbIHziq z74bBQ|0KLT-0am1Y#*q=^`)-n&(#`Vd3vE&|AEbO;@C4aICZ(U1Qz0YNWUk$Az-7p zXhT{2*ekXa_F(otC!|#;F9y}eg$P-Qi#Ia%a@*rqj5gA&fg`cMl=I__gra|*6yR0_ zKp>`E4Qm4SSoj<8q$0Q2aj!EG_d3(XqZ|+$5NL69Es#S*!*MUw3-K;Cj8Hh|mN@6` z>mgYJaQZ9z4gc6GlW3aN7t-KteI-mmq<2$?O?}66C?z63e$Zyo+;b)0LL0Og%H$Lc ziv`y^F8#vphfu*HLL|8uOhdC7+O6XCT6lfx@+}{sZMxb_whT5`sr_OD&pUnQ$w+KULq3+j80@EEMzvS>DD>G{BCz?#l~0Xg{?>%6j7C4e6aG zFbcjUvq7Hk$i{XaA=%XtZeHAjsCNoWx}R9|=63I^*U4;2fA`>tmdCB_ey2kdt^Zw+ z0${z=f9L`j1Vz;3+Nmbh_WDpyaAEY(ChjU~o&>Dd{HQ^>=>pikAwf!yfd>}DXKVY- zCOTn%J2X+pkySqyFvRLo;|W>zri-e3HBe_h;+9|4)vjZk#dw%H=;)oS-iGYjs}AEw zhhkoH9L4Zl>bSi5`tjc-nidWMTW4IG#RJrQlavWHt?Ul4O!1@NKdJJ5=nt;#WKW~I)I@!M03N+Vku_=B>(pVIbpC(}*+U_9$&mpLVL;^a*(n7|?NCEQNHWXgM7V5|@Am!LJ-jh4F)%#2{MjatCvDOW;XbQ+r zdED2e+)goOB6w#0{fd`C@l=L&ol^{LIGA zMw*W#s=0SrXL%DHI}3D??)$Q)^QO0|@W%w5@6tM4hgBUs*|ErvV4SNW8YwRWs?nS$pN>x|uGvp%oO{0=TDM$=??;7C48C>4%s@m=)&WRcpZ-hQ%I7hSv&A!SvnX;P5!U@0UYMo{zIP_$cJHl zSi=HaFTg6y?yET#=1JbjW>sC6WJsRC_w^*XEOHrYN}N{OB}p0Q+?O^h`;4WTW8E^5 z8u7ugN~qkvfOzOg_wiuslE?BH0cp2ci8vZ?BIH3El+UvK zBGcl(TaAn1twz|?H|(yj+i@7e@9|A*0;@LUFU~kNdbch#EKRDDJ9^j>YiP%-L~v!M z1K&C(z?h!BPj4J5s8=cCR2~Lu}kC0lzUdtY^k4V*w zsT1)gx_JfbbB!2lg7@^Z-2&EUfn8+YQ$5^)T*XP5bKI>|A%Uopp(OpwHj7JMemet6 zEKb)$s^&kQiE7aAi&y1v3}&CY|BByj@t1_nL7&Oa|0YPKQ3s{n=2jeH`;{DaKe;neUeG=r0QgvsS`&6JXjwLiv1e?_!PT zbb5muyH`i>kNmVk_j3ND45>)_vHYA zh2yS)GTi3czk61pJsXv0l(bSZ8KA!(rr%FEUOjQ{!~=CM_vpPjW04dVClS-Ep*-Cw z*UhDF{9d7)cG+XL*A9DqUFO1xIL!$UcHm8?+7mQW3lj30k zdE6=aRYq}x-wmIDfyzB?7=Zo{zY@f(Q-5|CN67rlrDDDOYkkW{U2E@OnRDwsnjvXT z`#s!W%!=!9%aEAA7h)BCPESQ|#;^%cURU z93LiJvPaJ(_>}OlaPggs9e;W1R6+OO_9(FRlY7*1mYQP=kGEp4UBIk|f!Ae$j_qoF zg?l!TgxkujzJb7VbVl&zvtacfzoGEC50M=C1_ zkmh{bXTBIh_VLY|954?3v@+XE9@y4C%=qPRua;uY%nEnYf%LCkcu8 z#7|E1vb|$Kf$2>fTK7ks=Y0miF{)C$Y^b8y$FXnV{lveD>686k#7axFMRQlxMxD;R zVD^gZYhblBJoW6`f-ksyW4V!6LGi_Pvra3$cur+K7P1A1wIQ&>;b5^!LmW@7koju9 z@I6O$o-I$Kk``#*R8{Ingg%BRoL;NIw~XKVJB<$UpQYXkVG2n~?Xx{7VbtK}IID;+ zSinz~e@up4kVC9}ff6eNv_eD5rId~B#G<6vdJU(^9iz#Mqv%eJ(2_1E?mvi1l~cbr zWvXU&`1FJ?Mn6G(;BMELglT4VZk{H~Y%%mzkqZk~vv_{-fNj?_D!rz)XhJgk5OhcD zwUsa7KRnHq(O&Cev~bqPH&u{5lf*M6oEn5!us6t${f~wshCjK~kHi(nQq-XDDlx?3 zl@F<5jd4)^Dt5{0HR9awb$&Y656kSuD!NcJC#u}P+>=n-%jN}ByD<>@tc?M3|5^PBa3vM9XPy@R0*e$d>YZs_;orYagaj7`h_P#-q*nccHY z(|Qa(#9w4|V?W<`*XT*PG;W*j3%~PRk}%6U@Gb<0wy9looV~ z#lG`Nm|jnC*WKg9l>Yd@9~|E*)??jV_;_mmcAph}DQkBSBi$}MU5Za(92Xq(WnGPW zhsAp5IP7gy-!?3@G5w0FEI3awvOu_FW30u(Xd8@jcI70 zm)*bS<;$wL$<^_y{NxUfvzeD2N7m0IHe|R0y|>b$n*F>@b4rH{1nrIluDYk&9(4Dj z>EC6gs@C~YbvEvK+~A=n@Xa$Hcs{&Lqq@~|?U_W4XJ%s-Ck_~Pq3-n*)ALJEhmuxw z+=C-QkyDLNy(^t6ze*ubXHV3#1C8m0v4ts~`gxd;zG;~gATM=6m+Gk@)9JbU>EC5Wy3K<_4{2J)32uajwZPFN z>Xp|AB{vVkvJS*^A}gfUO9|)`?Kq zxM!lG-exL6mnoq9q^vRU&FY`jrO?#BJ=QS^J1MY*AiDbG?0_>)L`=xq%7QwDhY=r9qDzbdkKX$`kvpht9;JW$}Z$so4R&UXl(opLA zYRBCc(I!fE!ob%8UXG?J6!nZ)C(u})Y1;=9@?1UnP&wjSO&#&_T5Jd+ixBz-c>Py2 zt?FpV&ntg9g*_YNcvmB2d`vQ7*I-&CL@3&-hI+A;qgm}1jqJ%LbAIlrZzR_8ZK(fP zo{?~hW>DWJe(Uj-R&i428aVd}D{>a&P!V(ZyERmBR~le9Kl8P3T@n&Hd%kj7A-deV zIBIQp-jiLZbT#>a9DdN#L|+Ns?m*3yh5HBgoAgtk`QfQ5?G0hxl2hSF2hoOxD>Ju9 zqeJD-7lNb_#E4ew)1E!+S)A-Tu%LZ5NxlT%@**CxF-ea9B2 zl%4n>IbQR_VeN&bcnO89C&Mp6N)1Wo5!W80gAh<*LO-I&eZQFskC%3+pl15pQ7Ct1 z54)|Et}!8K3m+|uWV%3jA;vevDh`^~X}t40RW)*)m0Gp05Hnn=9`?(wf|_W+C%exB zB7Ciyb^rW8d<O?PN-jTOjt7m()d3V^oxoIxkpv(>GU}w+LaFzd|VvC zEIT|7UBH9MZj8Bmgh#$~^{?oVS@cPy)re$yc5^ z^prxgy%P-Gy%a>k#w&D(lOiE+ZWy8)+~5;^B)`$H=bY4n-l0F% zFh%2=91a=~I5#}=#_yAQlihZZbeQ%f0MrJ~utS#bY%-_4!C;Gj-P1}w^zx$0xq`hl zyiIU`a%Ob`)qb>}UT`8>8zO0MeJ04NitES4{fdl zthdFk=c-cuZ7x{dSr$vzO|z3jH*^)tH3gNeQ#*n{dQahbISonL>*~pRa00adXmRxV z{C{6c3Q)j~ZQgd5wQstZvKL%f*6J|BSVk3YuHh|xthatic_%A}H1PiJJ7^pO?&9Y$rY1~V-*25? zr+>w|)RWInuMb4nCp~%C;pTHWQ}vH&;MN*yXmjSzMJK$z&JI-4-mYdR%hF8s`+QEc z&I~oBu29=B6y$!@d1%fG@#Jcngcb8LKQR*w27nbHCW zgC0dnloS`i=*9OsE8>J@$b7fHN|!aN+82O*p!?Ynv+6`M8|E3`90t85po+1he=1_# zLA3VH?>b$;SsvKfl&!j*%T7Y4(=yn-MK04N2;=}U?)+3NHEAB^dZ>^I#3G(ltPy5) zy7Ep^*2y)?M$y6=!JQzi9o1#I8sGSZVVAi^QqfaL@$r~4n_szChWvL8tJWiS7XuZr zpvv8u_8DL;hTruutd8l;OJ~2N&80DNdx_A3>$srMgtn??YKYZ$w92~gUPc|`qSbWn z!q!}3(_hm=JVAWK16`oCcd*4fJ1`nzJP|Z*rsMH=Wy8~Y`FM1DX>yIvY5U3(A0*`x3{uQKI7fQ}(@YV~k`pm=%4tm-~b7{QEWV zz20CaT7V7rTeGF`(s`Hh!xeQiqz=v`({%4H>7#tU5LsF)|AY zqu&}WM|CELc<7$?;X2&76n}NfDt7gz{B>}R%NIabYy3aW*#qvUE%AMk(W3E=NYT(h zM~gYR(99YbwMoK-EzV~^YG%2f3G8;xbqJ_=i&XDfrg-hGCLjlIBOY_WywuPM?@tTh z-=UTrnhu|;Www4*2z|e8WsxAv&CwTi67yCgqSi9Y`hC$GGj~0Xr_6IVOZZXs?ooeO z{gJVos{TPS6GXz?4}LX-OkQ^!WzA+aMEAz)8kxsUMBF|modFe0)ZTvmmr zZe`Y^HeP9#2vT~;dc%ny`kac@=vSwj^50iuY69c z)okeLv3Ch(R#dvbeRxAZKWeAHbIxpWfM;5~f<>nbF z=X~c`ts8x{Es~9_7khQsUK|?TNll1L6u)C;;qg%?t&jZJ%+z=AN7whKc=&Sqz!(Ld zv8S__T865h?;bKfb8MqT6KeK$|nm_S84vI;Nm5?jPyb zyG59M^yTGJ_Gts|HF>TpenZ|%^k7Vg6}J}Z;qAEHS|(Ot{yn$e-{Eq-D&xg51>0ox zMzG9K?a6jq(g%kqCXS`;+ING}wqJAVd2yAwUCrW(x~%OccW%l#o%(Py>g(d~K2?-q z4zHy1x5B=FH?T%7Is2;Kl`C&NDc#r;S8Wh%N>EhQjW?_t!+Bia6@l%)8bHChxZJ8* zib^kMc)~gMB5Ypjj;efJdiDu>u9H!>iyJwfdCe<3(5mVdtZH){z^A?1-1r5CPa^#X zxR>7~Yj~ad$ZkgP7jm3Tce4Z-qR%Px_%cp3XRlvb%6h_9{m<+Pl3ZWH&tG#ieYw%`tYRaLC^G>r3IBk!qtrvHkMWpHK2-kAA~G z-xMrk4jw}E<6cMk2K)oypZg_gZXtPnaWQOU5V;q z;>tNaTHS|NGS$M`)v{i=UFUyvZ}>z~I!^5E2cC(xD^HjV(w9E8%{Dk|n^i_#|IsK%1KfPz&Hm}1$% zcDwWI(o>^fJHe=UZlBGY~ZJ8eVxP;HG}kzmF%&HjC10uLdUdzWwm}s!i_D zeam~4UK=aQHBXlZ$HTOI3h~e?x0F)Y3UcZ`_b=v30-DIhm!s#1h()zJ3Ayy`uOfn>lf=NK7bv`UsK;$qbA0T z?DFlUc|8XnETZwWYE>E{)&SJCVh4(h>DgPkg0aqhn*WIfp8_uK4tR@nq)ocWcn&`| z2at8*lRiXQDIY6ST1rmdP>LN zvJs&VmhQG(-BFPf{od)_kZwXuKeC7>-RWz`lsda~1Lo~O)jBVO0$%0xJcxPCoax+f zFQ;=Dn}eR!Us#T`szV*HdnZ8x6gNtNkVj&~{0|Djb18cCI|Wr$D=mV*b6%`Z2|ib- zm=#jit|XQp*Vvoxc*vLBs9g^a(bTcLch>u9_fp*ceZenbvJQ&G9hy!MWOZkFxWg)P zo^Vl3!4AcZOavo&_t(vb|5(1(806199K2g&a9W-7!=YK8FO{;&B_+e1&knQCzIOGx zVct+Bm&VFF>z{f7Gu9<{;?Dok`ul}5h#u*hDW@{BER<++ig?GaQ&tD8mJ}46=cHpJ z?Z6&;e5>q3RKEr)wX?V24mG*D(abv8@bpQXMqHH_NCEr+XZbSOP_2uUpBz~IjGwz9 zsc`f;`@5*p!g#B3o{W=IukP55v}>Nq+svu2`ZsK`Fz8}NEEoY<1bI10Kqs|mT&$47 zD@$Lc^5YBJrtIDpY5xv22iS6d(|cq&F8vR%g>GQc>Ud`elq_s}32ppwFw-ghi2C&} z@WTaN$aX0^X(}#>YN#jrxZc?@(mXJs$m&hI<8CWP^8okxb`uX=g@Qk~xjIn=0^BJU zQ82al99r1>ESp*7HF3_TG3U3M*8h6}Vs4nLJBt}F|6J2GAX<<@C2s=`&tzSV4cfhH z%9c1!4(~ErW_0p$ZAO3 zM3Zc^5EDkSASXoCzoeP2jd$Ik0AUYk00qjS;&v@?qdNZ_?6DE$L5hMYsHp zk~dz+Wf+I%`nO;C=*D4p+Ux&1I`42czyFP^S*299)Ye7SYEfHK)lw^It0-!;b_r?* z5v#3FYgHpO2wJPQ+O<$>*@gP?hrb$vH9LhQKV)c2yrc9%BR|xYx=0I!Z7e zwD1W|Vch>A+^VCwwTvH*K0H!2fP70sl?3h{Zg*6M{}$hK#Hu-mKpuGv4XSPyp<-3= zXU4#BRT*>ioe8#i{3-2JpT5%x=(jK>&_0+GmVBMpt*eRXe z@t*+^?jn&x5|mO1&hMWd<$ZxSyxQAWuI_v$W!iz=wqXxR#k}@q-i;8S#t=6}MC&a- zPh7`?JCRl54u)f|>m;*+d9V8~i(GfCN@nM3M%cEddS-()dTi!QMqGkUU7N584Tj0X z3{GyW>0`$CkPm)wez_|d=5|N-xue2eU#I7St*c2DD#10BA}05JI^($@MQQJzHi~LP z&yjDfIK*(_e=UtVO>7%ES47?r5_+L2XhCL@;Q49;O(ycb;G~0T)b{k3=h)P$liJ;VpdJTjE$$Xw??ascYNuH(`|R7;T8cAxOU*tKBm(8?tU{RmODe!V$dl zymwj^U&@%l4h+Rqt3FXrcMe5J3cW4#ABc&xNpz@jH@Sz2uaJzKa?YAv_Zf6>20B3b z-p#poeGjpE+~Qf!2LL)fl48&kTh<93oxOKv`H`7$vyifT*Nc9%<5_`t^TM`#7I5JN3_vUEP;4GS}zgWpEvb+ob)YM zh12?@WmQ2UD6{NBA*T-G#%*OQzRp&YMH?);cTB&ykL&lC_%u$@i>*UY#+Tm2$f5iN zpB>V&5|i<>A(WkU=e5+h)<`b;Z^5TCFO+E3>tsx=d!8v>7{Z~arFOM&czocC2DEOLjWx>t2_mv(ptBlP5o{H-%{D)H zLVtH7L553vcF3v3jIDDB@!EWW?%Fl*xbGZXC#&3X(O{lmqn9S$9^w>t#$>EiHhK4E zEieLx?kT4!^9&GW7VHHcE%gTGxQO%C#BqFT;VPw|`@^H-;8eMA{Cn~i<1St*}3{_@mSvJy(oILbWqTAq@V5xc#d8=ok@aI=3tHtMA2 zUQtud(J%FH>@$xpaPyvD`)Fh3Ho?WFQ)t|#<_}`yWv(@UduA+|UA)fHr&?F4DY#&b zL@GGU@T9te09b0)gaIPYJtsbZJed0a)%75Z_Uw5{ft3&{DoFHgA&W=Q(>!7MhOYw* zU8;B&Hl}80Yp5P@VX{XvP`FHB^9co}Q8$X}pB|s}=&BRZ3K$ncFjfDMr4+uf$1)>Z z=l|olR6O{zP$c%;&Q;B$PPMm9;!xKGEP;+0xe=D;`=4&heAxuRn5}Z5^#NDR65ka} zpMfN|e~;uZZdC!YYM{|+4Mk%!ZlQ|XIz%5~dp(aOKFlYEFU@Ez7W-u4;2IFpn_N?I z-m|b5yS%}icB=fNjMmrO3s(cPzK)J@%^JE6m&l^7Dy2AdPocKg1O&Zy^+Q6@oV3vQEw8bOsM?Bdd( zHsCJMu_Q#f%j1kqU}ozIa}J*5Iz{iSm$t1mj}0@1p?@=W$RPAg#Lk2^_grdj%(7Yf z%JG*P+OuA%3D%xwhfrRXYbT1$ofCDE_LT6AhIm$Yq#AagWC8$8fo)9s_6iDReM7#9 zVxeDS^%kIDHQuwqB&LzD_8c1?Exm;I6yKy(s7ti0Anz58i~bNTO2x#RM=sC(`DP4X z>^zGsDNmfI*4#~d?(v|5@1*E#AKcKKM9NJWNC(VZb(>qELuv@5Y`nqu#p{b7E_WKGUOGhOBUBoc^>?{AQ}U z@lkz(t>e^Z^_TMbpELBn+dQP#{*4~4p4!m0UT5q}1?3przZ_e6!R?ckz>$4yK4%6L zWP=Bf{ys9;0K9UoT%VV{81T0hSwgRH@4iGYvw@k?J6*R(WMiJqJ4tP?*i-mLe{HGkx; z{yXqSw(u+!c-!ZnIUvvziJdni;111}#khJbm!yJaC+33rdSzNiLQWE5`)IP$$&A%^ z)i%2NbhCV!vOCQZsAo)>UtM1ZWGo{AaQ8w*)S7 zti|*b*W&Q( zOv(VfA>isIPGrEtzpqSv0R}$dFH+hwLiW&PrzGNzC>q0cvq0C}hGKrv_iCSpVpN0|?T1ba-o{Gh}@c z^|@>J3lX$Wlgn`SvwRJ)?vJ0DG6-R#?8m8m(eTj{@ibq2N8K!*uIUQRYYsRUy4E$e zW8<{dQ{!_ZcdGZkSw`~DQq5_C1;EjLL9+3^;s&v_VG=7j$SOofC^NFs?T8+O<>_(ef z^u|q_q`4O}!|b|E;XYp(qBagYy)-AbtMg1cU@K&Xh`U8Mh&=}T)ERvZ-cL@1p7IA` zeT3x$|Ne7Q5Bwed_-yb!nhor>jwQRqB#9_RpW3kMl)U>-#6;C+4>yZS^ z`xvg8_Z^=+A0f^bkC}Js6FEo4v~lvfbxTR{2*6&^y1hO*=%9L-cthISUleBCCvCe~ zvp$elj&%0B^O3*jgGd&4Q*L2eMQ07EB}3B*9@mWYpDT7WiNe_vguAk|qGw2ri+JNE zxBaOG$@&%Ia@oiF;n2qTT)M~b3KzABc&~d^xQ)obHFLy%50J^b)#TD}udv1WQ(gZF{*~oqXYHj<=6LeS8o%n_N1fW4!|MJFDGiy;NAL_my2a=lHF=LW1pYU zA87w%Q=_gzeIKQpqzj`cPbbg5kfZ-O;YinS%Ri%;?$0Jh#iTpW)q z&Tq!$!y{KJ|B}>y_dDQjO0H3!zX-9cj|xVa=()TzkIE=(KKV4FTH9T>QHNd=+S&QT zx8{>|xjGVp>I1f`7I%DBLKj*`S^a2To3!)-eP$x{GizXzE)<#JA|kxG`rilJMMz>( zZ$1cbVdK`jGDl1fj!2vBy4FU+q znLt~XiBGkFl(LI}X6~4pOp4~#c%I#Sx=qAoK$@-6izwDGj$6GvJF-93-=}QS0FSWO z)oW_A_|bQ-^jb(+*D*`1aoJ8hMY6ZkFD)}a(bfxl@Z&jGpH?kS7;w-}T9O$<<`?c> z9URDhnyOkc{yXuYLYa)^kNANUoybgPziA^8b_T+0V;N7ZHKYoT`@O9;boJZ)nSg9v(7Edw@OVbA!1{{1!}Z(48o=l=jJeHIk! z)a<^7=9vA}wh!j1-;Hve*Usu-t!lod$iS%Z0Jlk8ezs?n5Q75sC%~loAcF?dd4AbfhnRZTf@-@ zI01EQaM6oJarfA+C7TsJ+rR!$`g&o9g$k{=-ssn20ZNYa?M;=CmJox_9`HNA(A2J# zg)dr2xaYT$2wF791Ik(`+n*wHx2Dd#q-XRKoC55Cw?E{j(jVDh{*;UsLdf}}roJSWhcW3(eIyga6B5} zDl4%|rnSOvT+>o!&6f@~?=Cm5Kp(8SZ{$Udfz?E$4dXy|zj)MyxCV&(m{}ih)(Rm0 zap`1%{xefifb*C^k6II>=2H7P?Intj&wwXQ_QV z1T*+#*s#T5Rs0q1bH)=xHi#Ls^i{d=rI}s&bIC%| z$#~^$m1{wP$Jduv+^o7Jq|6%0PHg{K*N(F6M-&HZi*TRo_pj8{qJAg@9iZUj9odEO zjAm6OvPn8;{rPp=@9ntKre`(}*|G|E@}&Q?N7fC-c@gv}HUH5GC1x8qAF!V4YE^a2 zNqH^>;!X0jaZj4wUU3`>99en88X>KaFnPG$bFi)Zb$(KHB>lbX9cia~p=P8*O| z!;qA->%$2rQc=+@jJJQpx8U&a1C@U3f!9%tzn#~rfj%ybpG~h|-B*$jkgG|eE^7Px zLIisdOM`-xWEsce8JFyTbc;jy9oPeNL}F<<0|v?8vSU?Hdlb zORy^EQx~kc2=(KN)`cBs{(q#7ndlCDiCGtPl-O^+8gCRnY|o_Kr2liMQrQHgi}*-! z9nrU1QIromC=?2mh@n5^T{nG7F5;%6cFjv7dh6pn7ngV8^>K(Rk%-?MH_QYWdDvPl z*yC7>?aaP@a!a+a`+a*AAZoV34cMC{E56p$%p)C9D=*4kqG!?5#*%cR^BC?vd-ahe zQvN@D&lXkKMf5>oC zn=Z^;`(~LXXFG3!B7irFGyEH;msMrN&ndQDD?gacuN^>d?m!3qPz* z^RnLVxP_MHh*`Xxqn!GMmvkzaID5k3e!0K#q-Eg46&Y(o7HB;${UzW~Q?zh(Hu=7( ze0)#L{K?*vzACVM)_Of?>F~eC*b-l%sb|yT+mEiXxqJ<3lKk!kZL~{@BoM2;iuoG9 zxBljMQ9Z`#EiYif5~z&Z9D6}3>$kA>hEFT(Uaa!$gP?+&I`QwjtM4>6IBnm4cou2k zyeQwxBX*8FcE+0|a%=g1k9$wiK*zS+*&B_^0Y=H+dnKTIvuqX&sw-r(CiA2xR`ADA zpT}407cUmwXnas2GKl#igY}fQWuO5V1bhc`{WYm9D_bHVS&hQKel&ph-XXSLG^{Fo z)dPp*Hundf$h)9k1)j(*v(F+YLhgMgIGJ@E*i5|kWdT*|Dg8h6?!b+gjfHG}Y-^>? ziG94ZARoH_12|-EJwg6`*aFb9&uxZy2KlRARI^)6I<`IWOj;(IP3rOlxk(W_9r$se4R2Fn0n-%5>M-Rt0MkuXM~ z-IA#Lt#MU7H;ZTFP7T;Hrr>gA=QXhHo`PX^3&BpHczaT2BP_8Mhkv__%0Y7L{X>n3 zvMilf3rH@0+&g$Tjp@NG?ovm8=54+H8*HJV7iX&>)g}7nGb4Chr2THLyp3?Z>3)&> z9=TM;@?49*i~yZ_IT`%%C(mmn`wl)(!jnTQ4LWvvmQ^4O*rX;k zHYi_b>L%&kd-_QvEJF1)-R762?R4t)bmz!Ux?7&JRZsd8tMGyNCPu%!Ra3q2I1jIe zKM)P8W)ECS8RdFdD#O^mxXv_(lUM6=^J?O6c1^#bW=4CcM40;{HI48{3DE;&58mX| z@?uw(Gn<5Am850z!^u2{>AZ|Xin-{L&y+U6U6Xx zo-u2EOuqb$LF?3~L)am?amT^H5pphgW@0X-=1Ck_ud815Y=(M&(Fjzt*6&>G=r!ZQ z1X-B2VAE1u^#cTl3Y*KLeyg*(@EfY=>?{s71-w~szL4CSt5bJ3!w$?fm0=-}jXVOv$Aq4nk&Sm}3Cj{V-|nvFsG2T2o)BLj^0%gvG^Ujp#~Q@;#CzN0fB z;|NzuyQ;h*3i1mVD(#HR@ht*)^%s{f9+M2CI^8|ba+_Syy9}#AqdBS)Dp2|va7MDh zj%ZkK)`+QMu@)iOyIH~)h)oA={PO#=Q-&)Q_TI!=N)*&Vg}13whu;Q9s5$KVZYEim zvr#nh|MS8ZiMjgSXg%c6Dde!9L6I#_Q}V%p;3aoIDE<@h{epCI-#*f89ngeN))h0} z9r%5xP_qtN8g#}$s$QQ*b$`B@!A12Rq){%*HY`W#cS&b(DMB00~3_0HP;{+jV zm_aMJhFZO6EV%z!D5EQl@CT5ccUaO@v|BRqFQjeS%)GLAI2@Yp^E`c^U|{nOknNx? zRyWK*SLz^b@xwn8wFA=ARt`P&E(9B3Yx!@q|6eDDk1Fry72ogz;flt6G7gSu(sOrb&P1ZV8eNd{Nza#_xSj!1OkXQ?TIH<*unVqA;{m|BT>x& zq1rrUYQJ;nK)%vL!=;B@X3l>y%Jz(2(R1(yt%A+<}U{}OHXX%T!XL{GcOVN0oM#?frJFjHj12hf! z&k&we+_2bq)Bx6{W8s%0B+~Tdn|C(aQfp_WJ!nKf7KE;?C0f#hCNbsMljpXL@@ptk z-w2u-5cjTQs;@cJ^4Mj~;r|939*n=K>Kcbn_zAepF7KRZ!cj6Wxa{|~c9s`BJ<;sY z0j{GO%sg0w?RafM@Sv2>DvqUl6-6*t&76v!GaQyL zMYK22TvKw|K+Klh#_c)GmO+*5tDtStodPo!jRBj6HoMYXdormCl8}7@mC$-@IeeS~ z5=q=wD)y{12jFwFz>FF@%45=?I_vg9DetD^U9BeG)wD^KiFpM|!-{VccywZUyb>RK z6*_axN7b(cjBC+2)qrxjNMB?%hc~hXz;ZnG50e9WoVe`2Dq~IHH6i+P zwpArUPgSu8laxDR8Q+Php|NC?+3zpsap84!*!I_@Gb(vmtI+u;*pF_JGXmV*^U54wv{Xw@k4aU~&x6@c|A?C~2 z=(!lb2?}-fHjrL{FIexdWg{d3Wc}Pi&z8oPQI**(nGX{QjRQ8R;@*OaJAg)PNQyo0 zd26n0uY=3AgAD$32dEmN`nhwIvGb*AzV1K$T>V}Vmat9sk%~^*3-hCIOVj(ybe|7S zR2&-!){L_k#9y-?1Z^;olrP(UR8Qa^?f4}V=qm#Z912D1 zTMxM!W^_7<5Q}$x|C=Fkj}WCpc8~yKb}*Gd@&}qISuNnAgl6of^hQD;mqYiE;U8@o zHUI)cSyx_*PLpmn&%Fe64a1HkWJl3W-we+XZfXN)PVAIAwS)K|lIp} z3TKdKIRin$wlh@q146jC*`aWIl21yR!0Zv{hiMX6l6^+s@?-!~h;HbeG~=#~Q#3wf zu6dWgY#uifJuVFJwZn)oMwSfB3g4akpk6C?4s%QA&yIJ5W&F*#`CmPB--bM-HIOen zYt1;5Y7!6?;S!4N=JPtrZmnKFYBPLh${m&d>YVy^cOY%ND_&XH_h%Ys{*L4XIZ)KF z1ycX=50hb0T5;R!xOzWz8;EwZKnK3t-|KG%-2bkC0FAz!oO96j*_ZJXVn6M5ZaQ2w zBA&K~l+Z~L9q+oXxN7)L&lT`idnMRE@BG1=&w%@`RXrlyM~S~3ce0wy&_jKBZob%w zYyCB#s_1mikY5+ak`OZl+k)=xx*>%zvlO z1+w=q4_-)1p79}+0zZAk{t45d5b!@33nTTIyIhRTgrz9hE9m?l9!ZTq>kIW@)_oS^ zgF=4EQrSez?$*V%*3WM8;p3Di_u8T&`TCdlza=-2F5O{Hm$>Y;EK!!5>{C0`lklVS zm{@?jwDg4!yjJkK#eDs8rSMWd_}~Yduw)tgclU?SXeNT^dIDF5W`=88R?@tIyZvSn zmxBDJE=UYuG;XGYmW|TVy4K<8qk58SIo4ys8iStxubd85-e0fxPY6KC{FZOnmOY@q z1W6tU+T@32c8pd)c@7eThgKMyg!QkRB|c9 zO^=s(+ZdQ1U`&s~c`3!Yv21kp> zL+;-A>(MC9VrsiZJhzRJ64{RUV22 zv&N?bg#Br`X*7awB7c97w(DF6hq^HOYGPQVXXqql!$U&Z%P6t`l|Dz4$VGzyZCfE} zZ0jupu`4aduWyJR=vWuU9j;tMQEV=m#AUR+NF9HV7d?KJtsKUIotapE;^}vPtlYaj zz)kw9^;$T;Xbzncy5#i^taQ-bsch*z6g%mA7dkS7z$P-4%v8niJQCx!+KUJP-a8jB zCb~LI3YXc1qS7QLlof*SQ4BytS$HUB-yT-E=XKAkm(sy@P zTl->mPOFO47pXalDfNE0qfbNw@viF~Q?h?FX6i@sf+uVWs~d^;auG}PB2aNAhQ$li zem7EQ;gXtf@~L#>Hk({K=Er-lSxBpsl5mx{=i8R+q~*J|8<9&-rIUBgyuz!Z*ED%A zNaqR*tLC{$Ur9H;x-W5uswQ==m-1}iw`J3~A8t7;1qsoQ#;|thCMdbdI^P$)c2&WwCe&|HIZ!QgYD98|`LD~;k$ zRSunWbb3{R>@o;}- z7H^t|zez`D?zgQXXRVd~k&}1BGrh3C79n)$!(YYQc_*flNArlHdSpR8n#Epc#`TPe z*=SWB(3GqRoov-7d2r}Kh@T?oNUHU$IZI#Depoc~SE-8MD;yOToFqS;3@+(HSvaz1 zn_V8bk8x$0_Eed_jt)e#h8o5xSY-+~gk2RmON(I+2d+eIOm8_XNL;0KeQvk)GOyWl zxYya$S3ZkzAMvhB)3~OmmunhCuS)f%H(qiuv_vgo7<2R(h_5hX`z=hX`+cD&KWs1X zRWBnsB03SKjhaj^auAyv>j26#WdQ(I6Jj`9~s) zGep8RsB2wm-ReZB&h6AyYzp7h9ev=ZF`T&`#s}K;CxTW4PhN*}*x1qOHWA3L;ve}%mzixL?#Q(TDQ%mtFFfi^r*K8q z#e2&IX(f!J0ukuQ;8js*es8q1OT(>S5ciP@>-;8)JheTb8eHb^w5r~@p1yk$Va;66 zQR88;2 z1#r_L%3MNx*1nKOOuC*l{^jbZs%NtP(}`cps=-zfo&8p5E5|t@?)?(FTu!ueYMRiP z3rn8A&@4O5tj5cG&;hU@1y)-wDQ*m0`=crvaUq0R^ayFCUGuLPQ0-9VEdLu1wo?(v zFL|MDiDFove%z|n7^KlumtbGz_K%;tPlH8d&-eF7D-8}Aiv3E{XY+>n&rW|U{{&-f z?-|9}38b^5uBrx$w`ObG8L~k)Wq}(*t#MKMamsIP$D<(}y4!`OrRXr|Q|sj%io@T> zdfUG2*9Gj3IrB(HO)vRz4I>r>t+2o2G1MM{#==Cvt#XYkxaMSD0WmJRQYlMM~gD}>lan# z5Y6gtCg%#AXR4SQEH+T_dfs*N)_!dhA3WxwOavjW{=Z%azvG~jYY2b1&6(}lPKJ1oz>YbmktaEu6ad;#8ki(ePa~bFm|p;krJY&q#`?QAwjh67U8 zBTCaO(Vbb~Qs2qDY!#*xRwE|yW;2`7a2ktA(;TxrdO0J%*yb`7>6c31i+g zi;t=wY9|&+sbRiJ#g6=@c}Wz*cCoY}wF4Y*obsIulZDc3d5!sr^EeF+-bL7BW0}16 z9;@ltIjn_*tu{tibmzkum;jTr!CJsnoxzIZx(S<@syokq?v>s_-pPMi;Hu|6kMeG9 z@|p5)2Eyygh}TFYO|&RP#DMB1a3 zg(N^hoe9lzQGI#!nkQ%jxQkF5?2X&?k0beCA^zz%_ifh(4^b)4drLBYZVcbKCSe8 zpa5c{Gj}=BG{7s>bI!@)s6y*;{RX)py@eNiAkTAg43P$zFon88;P5>;JxSSjFdFkB z2B_~-q1EXoJKQOev&1V z@gWs9c6|B!>#wUc@}K0;&%LdO0~$w))B{i3?NGu_6Z52#e3QV=7}7H-gx}q{JatXG zT@PXSZuzg&Be4{IFRJFm&+>y+wn_O&nj-$uOd4M!kTUag?JbrK<0^4NHJ09~s zV@hcQ&?k}e_Q?Y~FiM_$8MuxA{E(p+EuFqaA?~g1tyw@?S}nsAqrJrci?wcI-$7T* zJ_wb-8%36jo`~*{=yodNcf=YQcsyD=DUTz_IvWw->84GGgW(#HDPn(b@P7eaZ?R&b zNAERiABoSfK^}`4aW#4Qf-*knNozsAoLVux~St}6xp9q~UV)#9# zv5BZ62P2VGkIs(VJ^v=wzcBNrZY<@;#&$^jdJ)q;7vj&H);6^vNENOD z4|WoQdZ$I1X7V*0%e|OhP?E~Cgb=kFT>Qt{kaDwaDuF-Qd&RG5O0-kb8VP2=_UPgJZ{|P;3nFj)@z3QFST7?nQxtM<*y1Kvq<@^9`5F!%O15g&y`oPt&dP}_S2ZVuih^feN;H1&|oql zqjRWIuHY?$Q`_!&k5c~-$15wYo<7hmP|DlhKm2C4CT;11q8{>YK#xHAyfiv#fmuoN zpW6KxwqPD1?<%FkkboB6dT3lhA$!uIThpcl+QrQWZz zm_Qd~{8F^K;Q=n6)~6J1QyIUsT;*=yQCAoF^RRG$&wYh6O({3hU(LiajNr=en<_dKa;5b=`vSrc!W=yNT9xMHRzSsvdzHlz294< z%_j%l(S6P?Nmr4mg?hJ7y`A#| z>?^-|`sYfS`pNz@RHmicvVvhd*jyEJtprtQ%Anm7!xcN*xs!BIuJdd~Pn!O-M{Udm zV4Yc>IeY!TS0UezdT#{8Z^Z;uyr_&bzimD{NWALO{bEx!%6h;0x|M{ITYx9zR(G{a z5ySajRzZ3#r$4j461@G*$5y8{`frYsL!Lo&jm;n1PiK$?ze1tj7UetQGwlz7mG_?A zsr3idg^M>bD%pPT5&ABm>8b(yx@K3(+mSuc8D;k-3jY=Z@BLvk9dDR8AX^IRc*X{Q+d4w?t(Al^ zmRdk}@nUt~hsaZ2?A#0mFg8OPqMHNGNKpT;TqG~yU`eHIE-WrAutl|e-<9_5)ly!yD&QaqH)M#j`C;S$5xx4wf( z?8ej0uiAVX7c&Q+@48g?X6(1xb};uj&F(0;2%Tg*=9Yp8^YLzbgP#LP`w>l2tug4{ zY1cgIr;;;JAlTn;-FAZ_u1<3f`3R(9{k$kT8e4+@nz))ujK!KyRhz|TL}%c7XUqFH zHo~C^`d5#|HNPZn*RBa&qLU9TA`$clJuSt1uOW5)I``cBF{OpZDql^P*Abazh8qTqZDtc+2sauD}5@HV6M0-S9c&Zln7fGo|@%s;}hn z8kg6>vyg7h)BWi5W3F){j7D?s0Y|`WO$Z$33!}e-*a1E`eNxABivcE2#EQ4Jd&?c| zx^NLuyCsa@1B#P0dw$`_3ib~WPNYQJaSg8u{XmgVg$i9h4kPYXo_bjHq72)vPGA35cug^6fwAe_2XpmtOQu9tMe`>0TC7I<_}yQkwCg}CS4Gu9 zo7LD&;Xuc?xApDfcUrJ#`-VSk#{e>mOVDayM9)g~FAw(AvJ`hh3JC1dye)pKvOn}9 z^=46M2<8Kxv5D(U(C1AU!KXerkmeUzy=2r)W%g2+xi0j0ko?Ny$T&967}L~H4)0s{VEPs>2Anp#xTov&(`|ZJ zYYM<_-wa9=+cUOHtzLNs3_0-KItT23)ACS4J+C9-$MmQtm1I-En|kWr5?vEOIzm~) z?AVdQsrX5R;}#&*1gN~~m~6_Uyp?e1YcaM7NWHlJ@#5KUv{BrfSZE806p>pPH_wo{ zt&!>vI0}ysh#Pv|v?W}?y%q$B$~nBHc)%pKyT3WMIVN2hE)SB~<9Qc}LX&%7PR>$c zRUcB%RlKe{GFrJsA}6qBrPle0Qk?=A-&Qo}BRCHbJhwFJbK|HuDY3PyM!DtEK?^X9 zEwa_P$*Hb!rsnd!u2MQE&H9w`Q;PxVCq}MVvHkkenfTK&vXK$W>Ax|iLk%z-dL{1?1$wV3_c zb3EDxKKFB?+#gOOMLU(ckkbzgt(=szl&tO#(B(_G+Rd;7I*_&X&rn zV{^WZ@c6^iz16+dkC0g@hQNC(4w1IO4}&wM^dy_#cS?BVDymP@OyU&G!|k`EyUdPq z@c$}Y%caKYaE{zm%}UH3!Q4jDnOx)O+%=*ct+lHK^Tw>*R6VG;pMp~yn6|e{>17)O zEcBka`FdU}DgKF|lU!rnb8jf0B~xe>fB+Gr69l|5?mnl+P3Eo& zSMM9szTDp`DB7FnG}|_W`J0@+K$MiLQ)g9Dk3{v!IhLT}o6E-%UY=oCNJ;SE5JG#BjKR7zWn1`#yU7yvb93AhOP#s;XZk~ShVN8p?KrQN*Cq=d z4BH9Aec^B8-^kV?pB3d3$+RJ?yUKn|kZkDe))D;3|7hp0ve(p)%i}shS7CWCeDSRF z86vIOsSByO<-V)EK+V8S*1Qw10eAyTB?%^Pv@;kT&=tRK5bC(+bl|3Jq^^Jv5=|pJ zV4etZaR&S|*_T7i)RNk6iW#ex8Cqx}KtW3?KryV4(R#Qo>`1U_^rYA$f`vZowyddO zQ^OSx3>63kOu2B+bLv*uuMvyi(7$>%&^jYFh4;FY3i`AU2lw;023SEts*qW>s&J98 z5VzG-@C?rc^tM=R*s3IJlTfwV%JJcDjmRiVsBhD4|A>*-JDSMd$h)519`R=n_c literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner19.png b/docs/images/LogMiner/LogMiner19.png new file mode 100644 index 0000000000000000000000000000000000000000..bd9003e0ef65ae657f262356bff4c3303f559e49 GIT binary patch literal 65043 zcmZ^K2UJtp+IB(&M1;)P=mdumbfgGKhk&A@gAfH3loAvWDWM7yNJ5ZNRH~vB0|Z36 zG$~R;95kU9>7fKn2$5bwNofCJ=Fa!s``z`gy%r~nwa(e+?EUU{zfalEuUVLi9FRN! z0)a%XT)tol0`XOWK;WnQ1%Us=*(`g5Kysif7tY-X0?($pb~=BpsMKB1)w+T=mU|3( zxo^Mtg+te!|G1K%mUNIW-oHa3Nm`mb$V|~lN`dNz>+U{isA(b5)?Ys|(l|4CMq{S2 zKmSg3{;hRVZB5PkFWwE?Yd#D0>!Q1>$X!jPk4?YKecYz+Zqp&w%Wtl{+-}%i-@RW( z=ttON&oA|Dw(cZZ&`w84+8&5i8zf%psr2p{h*jTCn%dp!+&wFkx00Q9WV;j-m0`rV zo1=Vol0BFxi43)z+1HlLNj%5YByn&Zp6FCWpdkE#|(n+n?Y-g0P-!v{Iq zZ+h>lSECCuVTwb9BYw1&)xxwJMwZ-bbE|iautOG`o($?%2Jd$nw59WqmLj@`w}EJw zJ{KPg+hirwly|G8)huqR?SJ3ZY}P+sq#tlgSZV(x$Aex=-96h)_M-q7DQld(PnUQaozA^ZC7jDNR;{ z?l0OamR1WukYq~=b{x{%S1c9BpP@JE~jg7KS$Jz)$uN#E3!~6qffmnE}z3b2#@s? ze%71#E(b|{U#Ry@T1;;lm1FqvwUi>4T>$AD)522Yf;X1c6;VvHichMVTD@ixq;D!% zLPSj$b3)jGhe+*huiVLA3dXIWOi=4POagIvwjlFre$3fbW*%WBVb0q&lqZ$>rL!s- z%A;7C>S9$kNktl$SShH7>=~;U#mQS)3sDi~YgWayT?}@>uGj8{$X?sZ6o7AxhffsM z&c7Xb=64UbRz?&_MB=!MoAh0r&gN`cMEx4A;0Kyvd-uuFr-u;+z8j6OoiizcG`B|g zf=12t}PQ|w6igh;@{y`ofgo4yD={(O-~xj>5daGv3ErNOtHj zqO>ZVVsx`dzx3IoC*Kh01g{#mD&BNK#P-^7G1p+ZNq%_irwHB8JFl%Q+~97iWo_zZ z?fg4cek2~D*727|BPsGL*9GA#v9Tsl!-ucJtK%94q&&|k)pNd~Fdt3~fCZ(`C{^@Y z#%pQ{ZowXQoof`)9yw!&S^A11JqPcWG>S^$wr7yg8DN~gcul1~lC{v8_#@lZg7u)| z%JVB0w%+xtU#RyYmHU?|VwG*}(*?z2{i_ryZ)x1-FkRGCCZMc;h@+% zzJOb=O=x`UQKV~|g3dEqd3!0<+b=V)`Axu7z~|uKjJ`jUe|u17`L;)41eJa^Bx54aHj-0a3BB# zC};NlQMd6uvN4B<(=%8f@i$)x-CYlrVjIW6X2Dc!>8iGZSGgfnD#(ydNQZ+pS==^_ z&VvgLyE}_t#gJz~;mYLcl};mXZeE16J@$;iavPy~X<{#88i9b+O@?eu%nZ2_Q#S^I zaeiv;fX@0FyxgoP7)W^z*&o8`HYLE}a3El_8|}VBYUV13wTzeqZGYb5-@`7mTy|(W z)ILnrUqx(Dy481M-w;HB+rWy9gFa4Tr+3q9>xY z#~$E1^%v@zN6sKNYL{D1E6$wqQG0Z12W-|W13SsqD4I;LEt(AMz9^7_?Tf-7%2Q}Y+bL>BmlFv?uL9PG?ZFxAY00icZB*1(B56lc zTT}Zc10j=`lX&QZ36q2A z(W5t5F{7sBZXjH`J3KyG@qo^>@Kmv4Z+?(1}Ac_zKDNJF_biVQe5R(mDpA-tD4N^CVcCj1w zEHj6o7c1$w%^}aWqtj1B6LxTyiVSS z>fGvF$y`Yrwg+AY;q7j4sim|1mMpS1DW=oxqG+BpdMU~EI>3t%L+n9()%l3Rm@Xni#39HtfDQQ%K zx=WROwyWO!eI9fc|C4?l8%)cSAiu9+~z>l-KC0YU}xy_I+LQwQD8*x*Y_G*%lI@F>;Vzk zI8P)ruO;CiQ6!^+8rUyoX! zy(S_a_6gX6EbuDYlLT)P`rhel-^8XAn-BZEP<9nxDgy#aTdG5wFKTVjDfW46o0ZQW zMuC)PR8g;Glv85mpd4gy8>ocb$xgf5>)xP5gCPm#evrO1FYaD}D)oz9>FHL%gA}mK zRrbg5)*=H8#s^H3(66m-({?;2(`lOt*>JhqCP)@ZUjd#kBIXcuw>5esPb{!lCKlbT ztsV9XxxE>ntp~a_{amBri?i#?q*;NFQiPqj;!=tvYLqKSi8*V0tfV92=$3DhWN1ov zR_E6UHL`+JR%|CnxYN!}m=EkV&44t_vhKBF3|4!s4(kSgkY9b*Ue^!9W!ty$t2-3T ze$$g>C5^fUqF!2=$X0)Ty0RV>F5}M}{1GWHju`}vw=h;a)b>%nM!+~jtFE3My0tV8 zxCqm8+_+-|OPmNl6hMuTW|%}L)|D5paqi?+=T+zJt(aVQyuD4J+cKrS-Hnc2N9mW3 zlvH(>W~zJie+Vp_P-rabg+L9ucr?EI)nV28%_1!tRbw{EcEi_5_wFkumOcLhj$71J z3I1|U(c69dJE%x!;lP>Wvg2VgH&;P9O$M7yFAvf{=KjZvTzx9c4(z`WF?y0E47+Bm z!3C#@uP4}g=`KC5=jNz-Hu=+je$v|~hyz`P8gj0+LdX~3vn^oDCTPjH-sK)KconE> z5TwHeHI9^KHtFb?HP`Sx1i53Hj*G5N=q)zNx&N{v*mZ^q^2Q4VkcS>nn(IIcmW zLP%Y$VV1DZuh2)Ynx*iOsvt22V?W>cLD|ygH!GhWJ3o5R9JYEd<%9mrB@Ef1GV170 zP`2rM<**lldD+hITMoCGD(L@tUeW07FM@s(SM!{{DQ5{N^)5$wGW-DJD&EWhzF#A!NCPI@<)j=J zFYM%6x0oS)+?Oi0>rnS;aRNV}YTy~&8JwHQC=48}D-g*4t0`!b+lf+=2fd#jsJ^?z zIBqPjE$`Jo8+AOVvs)mblz)&sMKL96?Uuypm5)|86OI{Cz~YXODqls@;rC-BbrW}m zyCw^+c`kuB$WM!VfA|YnW2sk&TG}pNbrgJ#=<6~nyQ(aa>m>hd%q7wOEz}fPy zU2zqBEo04S7tvH;fTdqT_30^LH=n|s)^v3OjUnVg(h*zyiFCn52pQDX<2uzYt$W{C z2BcGZLKb#e{ICaE!jAOk{fIbVVShP>7orCZ4&C=^lad{OBc4Y5nx;vC)&9xpa~(*^ zJd1zWpX|*S$ADl1t-Hj~cg z8T%J4H3u~jC5<~aAz0@Qp#`B{s6oeejmZbmBjnZybw5W^`bK}~f)$-tg5(eb?)YE# zzy4Q}>*@n(JK|*Kz62d!5)gvP8r4l?Qw9b&LrdQ&MeyJe>H6xsP@Q{^+6WJ(C85T? zKe7@l%kaq1?SReS`jE^5$m*q31KQY;Rr9owrsl)dwZ-$(YdkjS3}}gWi%K#&a!=8c z93GYOthM9B_*17ti#_l#ks3_BaMB@ZaVLe}r9ra0av;2`U_xXC19O@zDTC30mQz^= zuN@peA8XJT#>WHc;wC#H_K`(vLp~hWOUq;mny}icnj?yH!Bz85SCq*BG8y0`!2+MO z|BR#&G+ebNmgE&#&urLgQ_{%0scj;p!182-v5Ij#9G*L1z;g6FS=%S{tXuGpvc_m6 zmXPUj0JggbcHdY`f~2QGqtHk2{>60`oNz&*G~<-0GrGhibv%w z0c#Ii@3nEHJ1Fjpb?(TSpMw3Dko#v>!jF@W*7~rwCsjk;YCTgzE=CY!#+zVS>5U?X z@C0pn?`K;JO4>I;d}O{w0d|vg+m(<_cCKR3ngV<1_rTRSU()MQ<3MDymEmkNzYG(v z%4E47y!Wx^tIgI+iH{t}#=aEDVuhJevXSd0@Hlt(BQ5Is6frbsybh8=iiWoT(^pdZ zRhW$uZ_;#WPYXX$nA{w32!1A)Vd0x5Z+gkjv*`)iWlTbQrlQ9}-~67W{>uA)OGIcL zY%yAN_uF~974^h-yB}Kr!T$MSFC`tRLikpZKZL$N6|UkxTe|txnv7X(<03#RM!Dw` zNA-;N*EVd)Jz)sVe&<3CKLkC&tTtP3;RSJ?e@Yk!u5SD$VZ54k( zd=419CO(s{l2~?<^eWkUCZZ2aNVt@OqX_6c+>T;QG$>U>JOn+QJ^%7>4M+z!{s)wT zPfL&0X?oUp+a>S@S4tIRkd7!ZGa!RkSYDHZgTb=8B`SOeF_0Te)UOhk!bEEZ%{H|H zKb*i&i&dtSlZAKm%=De{w~QqM&W)wfT?&VeQz^UZE`qODa%pC_6*~6Gvg%Sn#votBOKXKLM1I`!z09HtBokVf9)Bm{P~ZCE$sQnGe`pW3uwDkrk8=xz8OU9;DQ+(9MZZ{GnQk$WnouC3;! ze$9)CGyDKC{;k$vBpr;e_NTUAh93!exJVuP-TRg$HaI+!h?>0SS#_N6>ejh~JJtJ? z$3Xl!OtQt#y6IGl6C@C5uw>?`sP=ul!S3j^YLo8#xJ{Dq))~>+sK${DC=|wq9OQO( zOA8x)`?Fz6)!)tuHD#QBUb-Xb;|hD^N<}AW?V1W%%vHYc%`c2+eAQrs5BwW=HHz?} zDN&;uFV2dqb~a6t-`6Cr;nb@-bAiBs%l~jK`etyz%1}rmf{@sON=&}8(QCzr@OI%r()EL z7ePOIpTnmrcI*_tvwI9u5dC<~b?-mpY!TBRpMpmAZOR^<2tP>*Dl8pqi!ByoC1xrW zX{=$AE_A|@B*Pm7SEv}0DA)K)gEFT&lge9^4Zn)6T(vFta^FLjdu`VeGUBrd)JRuB zcSq`~y6+}wiv?6=_*%rKo+CY#t-X!b=%Yxf!cit=sCN=pX*Z<&9JQl0;?Ut1yOr&q zNu>8p?IyAh+d2rr#ip{luDtk;v)LRMTYVm77JFu%dABi(M0#}~Mp}xu+!e3dCl>tS z)UNeR_kNeb;~FGgXRm$})_Hv=DOmae@ji-ORCr3JJnayop|VGT;cT%e1G!qIOI0Z9$YX(9Mgxw@o$0G{yb`*;tXq0405mZIKRw$l(ja zxH0E%Pm4cs^#(&FeG)}C%bP3X7oy=#Q9phZB;FlZ;V9GXqQmU;(d>NHo7tp?lJu$` z<5$b&iQ1Rixg@+nj+XWbN63Zu92{S8_3_L5g3I~K zJFc|wM>ic(%HK4(w%jV+oUuT#UJi|t8#RdR53&5$u#pW0jKL$ny+f>BxbTP-Rqr7a#KEsQ$1F^IlP`Q>X7N2>#QJ@P)l@&O68*;MzW>N&5F;|C#kZGM^nV zMPlP$mAkaj-)!oi-%S07G`u0AbV2hP=+W^zvwU^@LI2e6d*giUspLt#E$9=x`_{i0 z#=q%kB>+Q%J{<@S`;Gl%!pbeWG!XgEwB!riFAC$U;`b3A|Nq{@yP&_KNPA0sOZ+GK zKgkd{Fn)}zjJN$4JpUV{ml*efKmb9ru$_p|fESDzJ&E86E?>oCXohrBIq{5Vg|#4U zg?+kH$FLK})7)(fP@;@9ZEh7Hz4`tbl)tY=-z9?pKKN@x%Q`hJ)_X2kCLq381-J5ZW?p70{@7?61WlVcJl(fBvvT%QH_Z3A>OthE zUh&<@@Su8y8O_`OdfhX3AhOrlPlQgQghl%*RG+AJy^l^+=dMw6>xes?oOr2ZgL4H~ zOl;Wg$!p9ZZEwGB7yz4Z7B94_-cil|XB7Uv!hA*cD5rQCyJNp!Qv_W-^UTxhb2{#u zYbBM*=f)Yl_{5^@dV-&yhW8}ah>;&(=Y|T1+<6vR9jK#NoSfeSw0~`>fC>VPUHwFi z{(RDG9m~SwvB{&G6GCt+3tOL)bo*P1HoBx>tc#_mXC`6X8m{+4am7;6{~Xd%(kO?F9eTRxF!=LZn-ho8nH;MLqe*8thTeR~q1jcou5Kf=mF zH*4wHY5{UPf9WK^(gx2IYD)+7UT-X=%IZIl@XUFI{~M!wzgY2-L7Q`* zB7m_j2J1XB9B>Gkn*f+l*$#l777BfG^p^No=gT>yb|2;hK&Su`+>! z8C+-WV?M&(1NM(l1`ok~?0Zm~z=5@`l};E5&=Dq{h&d#&(*WTn-mb*n<;~5v4c;W~ z{Z|pX;Ov69srd6n(Bn4?{yn<;L9RZ)=g&8bDvtl;!|OJMi2$KKwoi%A2aFT5^EpQ_ zIGKJV{nVK;|4zdQKrtziQM7T(dLzkpju0bGgTzNOzRHq?ybB5|-Hns0hEbnJOFH43 zXLs>Kg0eyBKOCtRr;U%v#2U|jqT@u~^Cc03zCX{7oc&W+)e{b?CU}0l61fS81m$%b z>-)bKRkPlrJ}|Gq^!@+nzv=(?LjLFP86vQt2X;Rn{o?>{5FBt$_ZEoEfA-H1VDKXL z2sf=W&+2P)W~2C&R*yFwH7+Xa&_fBX_gW!PTcJNIbY9{xWdPl2ItP7u`pI0)a_U_^K7=zJv4FX&FCXh@u{ zPRe1Z>Jg6yVs(;x{rkLt_V@lsK5q>H+2Nd7(DEeEBOiV&$P9FPeDQmvsu%fH<^fJ| z)%O>7w^s*t6A7nG#pCGcsZ>~P7Du7Hh zDIf_T{(H+e;Zt`fNR1}{iD$Y)upCJ8N1JQy59>x^Oze=E1Qb#EuqWDYrW)FYYP6UR;DBu zpPhfK3ohDyXB?N6a3>#Z6QjJ}i@$Ji`4h6Ze(jDvArg*HvcLWaV}3-6V;J7rAPf2p zTq>31n{^aieupUq_RiJ@?KO1LzLxMMC3*L}MPY$zgoX%JF5`{fWl zC)PJGA@qu)Vq$|L!v-ve269&FoPcN-D-=ES`RF)b%X2XQxZhX;T)!R91Om5yTPIrj zxlq%krxx~_|5GphokUlzi!3&(8lNPqK2qW)LUycDN}Ve=nW7dzjt(Us)0cWXVKeE{$?g z#|~X<{wiV16S@SB2VEflMwPS+n#)>_>@aQV6PeMC<+V1FJ5-JtQ^B%j8* z(Qj#U^9UGOzH`LWFr^H5uhtUE1EQ%q+|p|fYx#D`&?Py-ARv9PrBlmb^|e`uzEfK? zZh9~wJCw%E^yW?LI&7YbPjo6ZRM1e+YwCL=*|6Y|gV?m>R9sYC^~`j#eSxj~A!SSI zwOK3J+1a@h=esd|>17EpMSR)w=Z)#|#%ZMIwok`!KvSJK24KI3O zwg(qbb7tMvVhy}YUQ#l{-(KewGjqI_{Y9fw{btMC2Pt8HPJJxATFc2$LiJi*qTBRr zy)9TSuSmT?TrOX5l1JE20&uat+TWn7x6h#(%lI`w>08vBKsR~hi?$73X>mjE8^Mc6 z`-hgDT6&dPgZFKb%?p!j!1JR=uU57+9G<-52xlJ`?9rD$SNuA;>d27IaC2gQ&-!Y# zT{65PBUWis*Y}5seV!IC!9D)0ixQ=rzgvJ_FRJdehN8wT_l6O5yAbwH%~o_csoYw7 z+f|C7rXOG_pt>#5xby(>B*$RxRZdzUuR~L_0S?`?#2S~O4DxtbqdC9=B*km`NUY9= z24vCLw)y7YUaKi~MC$QcaK+(Ds~_jYUnTMuH3WHi9!luxVQS*qG%%6NrQPs3K)OTD zSq?0bt;EuF$pb3r9}xWjKl6!@j0G3zQ(L8#OYf&S+nj={L|T_=!giY`HurSjCAuZ* zRd$rFuS0LQ^;(FGb6(;c?w|xzpV~Vkfxu$Vhu8RNhY*Pu)sH|E1 zmbmmP5nf_NU6PVY#!WOM^a_Db!0~Xro7o5yTfP;?dJFG;OC2){I@{~JMY)uX z3|fx2@Q^~}!*UIB=b}&_7A~Nj=YR@CAKj>RPBz+OI2{X$zX5&w{8R+(l4^WAFEC;_+qtmbTUgSZeEWX5 zbcpy()eMBi+QFifSKbxJRAO4f>&&=TMo&M&DhoPW_6_ zPQ{g5L-W&aG_d3~acdP+RWW$fIa?%U2KBO_I0il^KQK?d5z0z*I*V|*HW{OlhgzLA zLFK5@nm?hUdfO{5!%+iz!^yRNxu}O44ir1_na(ABqc*TVTY7Tj9RwydX~|i({4_PR zFhw(no07~g9tpaF${e0bP{*E&_Z}9{)7BE5?2y0$&)6Ft^SY>^&MV%obWFQEL(25; z*IF$&Xd6W7&ba|B%V8(o9|K9;9oIBo5s$Dk)1({N!5$8biI2Ei?E#*fl67!siC-@q z28{qH!uz{;PNZ_BO~}^#9gjlI4?rQ@s5kRL<3rtYOC!yr*zf}&0Evu~_dtT+!|~IM z(t3mbM?)@&s7g-vR|^^0?bb}}%ZYGO>HCAufRLQB*xe_l$23V}-j`hjY$^4}pt2WZ zMRow-FnZ~we2?t!L{rgtMKM%BQ{x~rR+Spoc~Q*%p$*`ik%A4c0-lQR;UnUWhs>1J znq_1#CQFSr&(0CGR{ls0UM!hZ}L){^i+yI8e9QFH1YyxT-DIY6zI_h?I?~({Q$ERm2L- zwXoVKeI|s@V;wshTCjsl+Egum)E~e4Bsd!Dn4!?Hy1KKu0NpFDJooQy&a!(TAP}&w znf_%{!T^Y)QIUxGRs`T7AN(#bS^!78^!0+6)I%s>0GU{#!eH>N#l~!tsD|x1?>)N| zO(ifY7AS%GDYA4+2Hz{t*dPW#e#$Cp-SSY3G~j?z)-*W;Mt^?NDU6Jv9{iy*i_EoAN zcP!Mgy*@UmlQ^*H-A9Lp?5+kJbeTgVUFX16bm1l!!9^tzK(%<%H*JXig~sr)3LPgt5>meA6r%8^*-(7LLTNSj~&j1 zIg?`YcUbgwqyB^dqNc~KT4G5@9m2D6=sqeHU0+tv7-Hx#zZtV|icZ}GrdA8(fR<6C zw^2iLwgh-q`ZzXuQkhi&sP2)Px?e?rA?*$Vd+28h< zUI?k)=axx~9ec@W%4TR0Hg}`nYv;Tk})jjCkGl z<_JGe4t=wWUar>s$e}M$uOX&zp|NwYvb!e&on5HD?Vi(slRia7ddJGt_4a(KiSbps z1XB&*K>8wlr4M7#?4K0#9;dT`8ET^ouNyHty!vR@68!c0BmyN0bcih1H^k5UdWI2eB z4||pWaa#C=j(vj>+4UI4Bt9yq&4$MDZN8FR!}I@neE61_yDfGuqs7w|iEQDea1xJf z_GT_%zOh<#dv8gU){PG(+_xZ|JKoc2BSjb!Skyt@$^6?zlJHYT>qMmd=fwM-?nc8owFpAdz0Xv~eOi;5@#Y)vqFVyxCSAMIwWGC!aBv@w`g5qSDBSH+Z_eUM&8@ziEqiq?9F40kw zPvBdxo8@sVLchmVxC-WI!Zcz33w(rci80nm8D=k$IbS|2tr{n3$8a`o+mh`%Hh0J-{4(^qx@487FK__ zc9hSp!-MJdve<`AGwMC2qWYxUY3QB*OSBsIk^$m&0952?9>@B1QZ!$$DLYJKF8%Si zgElrB*ZE2H1bhx|isfJ!=j(f)u!3G0^=lP<BR@6`vCq1sVizfs z>uj$DBK+fP4+7(rw&IKEK#GObXUuX`sya5^)OIl~u(HBhdvoYk!-IPC&HvN~)Rn%+ zwxvYL>U6JLD#k)Nyy+&yf%G0N?qdU{T?>c5y?D0IdTiLyh$X)oeZpIE4x5EsE>Zo8 zsQi&R1%3JYv%Dbl%A3|sIBJuYZvEe*gV(R(gM)hqf6ah&-B3Jt3rgzkhSn@+cXJef z>UYRj#)g_dv!})Ui1`sU^g#cTmC=gvqc$VG`ZQd;QM@cqi2Ync5$kj$`g}2zZ}nX6 z|3F{IiM+6VRiJnKd>`NbY}0b+p^Fr3TDT;*WP09(PqTqCaO<~psaEq{%Yf2$yQy16 z4hSl~oU5KtYS(^F0-hS77Rn^73=ZeCm1Zr#r=|g9`Y+<>FHz@#N4n9w@e82bpdIa4 z?|Oiria1?Vb#D+jjUkm}LdiUgFxut>W;FKHl^+c{hyK>otVlFS@%dBeur6p0u=1hA z5Q>(>phdKw&>TRPyXXMUy9kXAolL|0W54eL46J8t!`IGpf-=2Z^7=pC$G=Spf{QQ) z8g>HW_bA&&fYfF3a#Xu9_{6M%pqvuCwkn{`b4u?5iIpQvtn)f*GH zyA52ErhYF5%I9|NbkQ*}B&Hj$5xf}mt2~BWj^e=ylGrpW3u4N%mBmIVBF$Emm=ct* z1Te7!%uvAnZ#y=4w=;ftkHsGfG;4n(i3RNEwpRmdf%+HtsOq&1z)>RO7`Y1&{wPO_ zW+|T+DNEZE(9j2Hom+FSAzw-rAE{8!O9#BlHk|t})nMm3WXpVT?h}2?SkgH9IrxuJ z%u0J>1PrG$3wX(+siKoF0TFz*yvL$&7dS(pZ5+AtTfql)zz{5Az6(1d6KxSM!$}LzJJVY-PlSe2Z3q; zm-sX@Wl@~$(q_kOpI7$(2n1hefCEjeP`ms;=P?%?cmjQ+I6ba;VvxMYJ5B%vi|4qo ztm8GAj#qrVDv)avxO(=^a@T$y=sO^C)PZdRox#r#hO0N?j|-0Y8SwE4^sQ{}ZKr6> zw@}b~T=(Sp^*sq{kRUw%X4K_juoQ20e6Jigk7;i?T|5$q250ip4x8g6RnUA7+9O7- zDI}_F$W@HrRq)tTDX+mGj3b$1!H|2z6e6|jK8%i-ePyQN2J+5&bIZLp4`dq{1I9^9*|_IrN&4(N;>Q|zLe>@oI9LJ5{8s5P zz1gN;7*VSFUs8~hN-DLV2f;o=lP;dTd(?G+xlQ?UrvGeX-*82j?? z9N976UrMGzZPDW#`52vhl?o&{Dh{)POSu@8SW@UgBaejX67XvVVY zF>)XWa4o0~xwD)o^6N)JGelvqlzDaLwri^92Ou`grvQAqEw(81%q3@t|8NPP!%}Ot zwV$IwIiOFGfa4+%&`tf5xRNFMK})TEiXd#?;O+a=q!DnFw6*araE!(aw%@@0v%yD# z@UtmXAGXhhu$9mABuUf7U_hNKul}W2dn_okU72xQp+>efJvvl%`$(a7*?&eJkU`{L zWbW5Jj#m$KG7L3?oe{RF2hMH0RiTQ2QZapZwT@3Nlm z@kbiAnhuUPFA3w7{ULJJu`;gAR;9A547FPzEl^_UEx8=}?#_B4k(;dX+TNAJ)jklA z?m$s@MNI;WrsBU|o%~A1JnS(3MZ*U^&!vC{;{#m+RhxT`{XjSvJNsSz? zC9eZVqRzhj?3$Atin$i5@B(REZls_n{ zg(=UWRF-OYTco_uEXQw_uVKwCcCtCymXc(oZi2ASU5YnK@L!rX|L{K>HE@hEqX=xA zXQXZh`Wh}U_=b7;PXO|cb#(9r4?TDObL(X?C`w*P`a0xy+D>BK~H{p#(giPQAIA{Xo9|Ias z+?c(yr^vMrj3~k3;>9)0hk#C}g-lUq(SKLhF%l;*;BZmicEtpuyua?ujq00;kMZSGztB}hWLU1n9b+;Ku8IK`!_4F ze9AkIh2tnML9<(>Nu*MS4!AbIN!KskP;X=3KEE4b{D*vcSO?GG#p}z_z)7+@UW2}M zVRmk{J#7sqbp(umGuqEkZ6}$7OCU<*6L2x3la0lpk(BHEcGi!Li;BA{aWgdneY#D( zia3DpcQqk0U0lMy5QZsTx!MMwQgbM_+72%JNB;V&ieUr(d(#-fsIU`gOnaJRA@Ns` zVj`~zWfo7ZwmO$zxz^pPHLk)qwKAKJ4Tut~h!&l5u!hkrGV4=yn+Kq3piTgq+d;B6 z4!_?VxKF$9C1FimX0O+9CiASz$Ljt)*X0WU+wJT)pHFh!KI_L?8;0q>*Q=%T=N@j- z?Pf<0HJc!j6q9V}14wdG&6hK>eyFI_vvb^q+kq*rH5(J+YpmBKp3a7BPpe(U%WUsQ zof5)+{udR8dH!w5Ruwbth5r#w#o%4!Ei!-5N9C@TE9akDJU%_VIGXq6k{qQ$fo<~B z1EW<6ZKDgJs@<99O}Ep?u71~ObZl{fU&Ef6#yGSgtkW*y4lkcG(ST!D$_iyL!!6^J zZ@2}7W%o`v^8et6yCOapj)xv}`I$3*W&G{dzPsgC3ZSd9q)D@ASL9n#cQcOnM{zV} z(=8lI1rg1#{^FVSvAxMx?u+IjLhf8$Jxu;n{wcb^$<7Uy$rGLsuHKZp@gIwN0JQs# zyxV;sA-$6S2L7X`;$^GEE4^Ci>6#U{(6{idodVuI?2lTcj{hG-->ANsPIH(MkM9Ak zUu!ejm36Pf@S>>$(TR8QKC*;R+Lra8KKZ{Z{M~VN3H-^h8;F;}l=RB@V(7_na!!mIhQuifO0wN1I-nDIn-7CS2jGd$12O8j<(apZrHmOM}#^r z_9Y+le`~nELQA=3Vp<}} z@_q=d<`-=^YPAR1GD$F9b1OJ)UeEmeN%WPGGJ^5mtCD(e-KjgmHv~;b5QpX&)7ox4 z%TP&~p0Cy&7kNAVs2$fg@&_cIRSHdXG5D8Xdq9xM&vjoayyhRC`DPu1qe((`%B!ZG zWhrs$0;`8ZEPW`-+x zzr1)Hs(?Xzg)An*U!SY!7DP{5*Ut_I8VpX_uw$cdBPl(xzNuEIiGs*NzYRA6`oU?a zei11ay7KbY_BPfcJWIx}!UArGi?lR&iZ*^NDcA@qfGkmC<2) z{Bx*ytDT;!CF|7G3-0-ByXm?b0p9PnG?Hi*+hgWVcBm(Nh8k3Tf7!CrApM8-=9f8_ zMfHU0yNSND9tfhaEvmPZgmQ9>cj`8cc~VJ=%7HQTh#PYOKTew(p){bVNr_Jjj7ITq zZVRC_y-UmM&)!w*D?HTD-dN;hkkBr&ZV~WGc9t?~>f8$`svRJ@;&Bh07(tfWt6ge= zyeHnos>)Signkg~X{z3p3RADEsNU9pbtqUl-PNDC#Bs8lI!tgFAgSjjuF;b}h6|JX zz%i(J&owvK@4qH&!%s0q#fjx}GXtNGe;RG2Y~>TGX&qJd4s`8=$}4Z~I!U1`ts2%* zNGvY~sV_E0(vlENgZIW~!#!RG56!P`zPl-U!gPDOv2LMPe;W2`v2)??a7H}VeZa6^=Nd1l7Xl{a&rcmuDqU6Y&R#funNS$!hw zk>*6hw6YGe3&H^u+z>jvtdb-KyEXW{b|lp)x2vMIcveUew!`r+D~(9!r&O(*p}QE^T^x@ak+||9C7=ym zE)pQq8&Jq`LF!^uJiZ29u_}W5Vb~G4mS8F7&CnkDZj$QMtBNM>1=dI&q+PRQL6W&O zql)V55yVbP5Gt{gH$&+Qo3NE>UXAFY9l;L59C~DKCqHNJ&m#%WAYhdAVwtn3H_61M z@+ZMAqP&;)J+X39j2swkt-`izI@VWhs-uDtfMbTbwuH-KMm|{BpH;l+?z1UzR#<+u ztAfaLdZLKfwVZqHilke52J3SsGZImAoa7_PiCRteacaZ8Wy(p{8(@O)Hxe-0Vu$|q zQndv2OdnHw@8p{nL0A^HadN z6hj-0y_+=>KH9Btibqnq!d;C5+@>a@vj@5rxAGgfGuxf(mi;Oojx5 zOYx-FQU%is#C4OYzUp|rh?cSi%-LmeuFX;$T))k~tOfvN_1ETL4 zw{p_qVSN)A_C-N#gQj7`J}fAK<-{?1;9m=o{>Givj_~R(Z77F56aLN-6=iNfV|P>1 zxW9^GRJLE1q5pj3Id~Te)%6zGv&2b-(SGP1L`D6$>0IS9wGvyWqD&+K)q-}7AWYjl17_S#Z%G zchwJeMgXp??l|a<$47~|IJh4nzhEB(8H~U!>i}pmc)C*3?FaCT93JnLKOv;kqphnM z#{h_+T3{hWLJ;~Sq)99zQ&^K?D(f`p`l^QE{zCf;@|DgC)d1(>mBgkW0x29!jwN@0 zr_gf@H?#fIpB*Ok)x)_%5Y-WF8_sU$?9b+E)*cD-_!P`6vF%OHW+rB^-^4+FO|GLc zOxC$y7=~l=b&n8ET?Y=d-KN?14^q}yd999mO zzTf#~orYl|cl=&OU@%ovce2zz=`K7wM##E^y=0sB2iy_$n?rC<%ZG$5x(=D69R2Nt zBnWE1Dh>X1ZF{uY;s2hXynm9UpX}^xEq5rS5D zx~TPP;Gy8MczMpZ-w!B_qFuYFF!*N8@mEB@s>2!En{N-!7r=%<~-`|n3-30u7V8*Bqw zv6g4RT|fyG{J-Pfz^e_!WWdI-P5kV~&RKlR&e^KLjKPe*x~@t8um3Nj0Q;1NQFGo{ z;J@uMyCyZo8@KR2Ihr{C3uz)D_$%L{=1&*fKy~XM^jPX@C9hgqZt-|>gk^7Uz{+&) zV^-9I-iJaRw(bndwY>0zSGM-0BJ97dz37RJt-PwPW3|!^fZ0&WY7Z5JG(MDb6>XF` z9#`YM*}$0u5wIYa4`iH;LTJZ?|DKRg_|T3Y+>CY z0+7=MGU+g@N!yK4xOcGaXM9$7pMaXoPQMeY^Q1S4I+t(Zd0A)oc_1^8OR{U@Q(XgK zvCRSg-Nh8g;IL&=DmmZmjo zNG^yNEZpe^**iaD>yAt!Md(}Jqn0P7hv)g&0p7Ur27pZEEtk#-4#1NLs_;JpgDL=s zJd$DKBFVUm51(VqEFCyQo5(IbMc^6)!U)v@3wU=wf_1^_2F9hg;|zoDO+qGzfV#7; zbTU~rfErCcb_4KMIthrZS9{v$KB11&cV0dJtqHw3B)SgZerlk>FePDzFnYa5dx@mb zb(DA$m247e5TJVAZtA(=fqe;}(@&HMv!AW}f!&OI=Dz8ypTggDrlNsiuZDH^{@aw6#H5Sz^mkc;C4rUKtYcj@woZOCJHLZ*$?`EkVF*j(cL z)l|gpFi9HC0mwia-0z^1SpU>u3g0q&>>yaP{ylSgtFJYp2JA(ltu^Vf5)?1K06}YM z=MIp2muYqQZvsHJR;Dlh2&j{ze}YQ=TG$iV6I73K+OdbuOj0A+qDg#A}Z-Eyby z79!SjL}LW$bZZC6Fo7YknLUKUqC3kZ=Li(aS#gp{;<;`A+DU4BN$RmB2TlDaNn+g= zF3;d8U!*KxkQxaXG2{?v&wC(+(-16?*>J=vLL3Fm0_EP~Xm7R9KKI#(RIB4p3cB)X zz#SG!oHP7~A}G7v>;(SiW%-y>B9@SeL&7m4Jl*neLgevL0;SUel)+19FhO>cfKqD# zMYGN+sIt}^EcGEtW5$Ca^T7rnR4E@u&~kgz+`$|`h|ElH;C=s(X<$A1HN*>$p?h6q zH%OsCFWy%+XX5^2!^!i4)sg!e>TPO;@QttUpe@OUQX@+nqO#RVUW)c)Sg4$rlJm>~NRU#Q z!95TW!=VQ!lCF6`zT|BYlBoiH*@q%4iE1*Frq4liLvQGUyO4hj#60I?^OM69&#qWhEG`v#1A=?+2wpL}BIv#M2FfKYuf=!jM2ZfAP8Q8}l--*+arupBaq; zc2YprR7HZp9f+sIB&Lnu#@0x%KMb8|FMv&Iy$L96rq{8&$h$SAKvJF5@#%;_2{nT~ zpRIM8;Qk>Tjy>ZIR@070v+`^r*|#96BHdoE=R|jO#kww+po1?+IDu31-S%a07cHyT zXD}?tt*$PZV`G8zbYIrV3PW|6yFyf`DFoL*#r0D!#Y zrFrua*wu-gSH4&JRs%_1c?PA|QqZ(xYFlE?yB2=S)M4g+ z9|OBM#P7&W5pNPAPMJ}Yjm>*u34Aj%|;k&+R#?>k-+6@%u$Jt zi~J7F87z{yQ~QN|61`1u9YMoYpD^!!a40m0-YRHnD3>61YCQ(dFDsY$FiCVoDq>#9 zWL--GA}IAORDo4~Jx{yjGOelRb59Uh=D;lFX@#!4A1(q54z!{8Q4RXU%k^LKGT}-C)1R3p;i3 zlw`W^+aBSwtfkYvLN=@zSY0-{)<5OV<<8~)L)aD&)rXyuabSpoi8A*a9;uy|BO|8G zXbH=MRJtE`h9#_%tiQ~PDJ&!sK2nUXr;}q*g@|uCR@WA|Q>YJ>2AffDL8_`ly zrTa3IUL;p1Kd|b`3rz;HH`s3Mh*vl7JCVn-i#M2xF}3vslD?Z0R{Aqz&bM%$Xn&@@ zOYD?UzTk+Je(TKFX>!M;NFMfUa!!p|r@rmhg~E7=@YJWQbH2i~i5avkwT3N86|yQm zp0yn0=#EhXkjzJ|Rh-f#gxcYA4Eeks{V6P;{hFEmC~-aBU+(DBYA}gmHZ}SK%?XN1 z!nW0yhGMQ&j`tC7JURT&tUC7wySXI$C#$Y~1^?{-z!{As6VEzZuPWa-v~k2tY^Ye| zO;`77KG(#Fj7SH<5Xu92z2C-COYrj%X?MYUTgVOH*OeO0M|wWKnR4=Fvxj_=IDn|q z%e+4ER3?axLC!&g{Ly<56xO)W5Z2y7$0U`%axSv`bna#!X#|782Hao93Z*s=`2U&u zut`jVblQs2M}Ij#`20sC$%nDpB)$UZZMLYb?v=@@#G`}T%7khDHYChJ~wvd^vV5q>Xp<~up9#qG2(q2(Y( z7MY)o8tfBsipwGYIc^tNK;)aiGKPLDp>e)w*1&M7wcW67;C2kkQlU>@7QeaUts)^k zi+*->;C!W(8NSEK`fJ(3MX5u)a!h@68FUJ#_S#KNBCZ?{x$o(mm5=N(QI4nKTwloG z{*o+J%BSGZxb(mt^|RG~lfin1yW2f2sdi@#hZkQmC+9JM$7i@IF01~B9xAZ#(knSS z6e9AghbM5)Lbz_TSFzcrg@mR7SH~vaX^A88et}Ikewg73kGhNNoPo*wb0(kw@NCwWFqhIG0X$E zpqv;SvD)uj=+P_efi9MbQNYI!4%R)b-3GeAB}_RD>-3k=GS^;V6m&j8Rsv%H`fl98 zB=F?Ah=WA35#OugLLKoY_@w*#K>2{Hd9A?GBCHf5oJmeo8H@PJ?z0`gyRhdN@Oyd( zp(0fl5J9X@rdYi`At9u%nVBSq$-|8e!g&S^@G)7eRb^kYxrGov%2KWDqhia-dRohn zuDInNv9U-11PoaR-VWpQGinu>8)E#Hz*b_h0?%X?`x)H&BC=1_1 z=E$urYujttYyH(w-8A@<(F2qi%P|k5t(VWmgWc%8NwP}m>?)Z7)9%znl&$L+GH!DY zHXg@TgG7ck`^U-_k1cnHs2_h?=ApqIF2Ic^Y!{RhBoWNah59g!2^!Kl-T?*#ZZTsm zD-XAW{@r_jvy=={aTN5OvHo6$aOLEcow>4*d|5Y+464c#Ko?$*eQhDfchYOxHrEdu zl2{CBHEnZwA&Zl@>JG7S=$nsQWmKQwYcJbO>G~WAwg1TV&}A+5Bxwc?Yt9v`{`>es z_+8PNI36NKnNH3@9NCW?Be~$GdyD=-3;OcF5AOY z{UbWn3BMzbRuK;Wl5gd1e1cwb$+j6u+J0$pa0@p-rB;BiWk2Qz3*^b~3uK~-Y%}0{ zoGpzwX4*b7Och_6Dr@>WGDCJCqtMd?F03Ipg8Tl^VCf3L5ft3;OOl!hwze-g z-2Yvm6FFp=M5CmlV8PtzrHXM~pC>i$by6|#N^p(W>@J^J-gXI|Em&n-J>Hpa${dQH z%bK5O6cK`7DP&hJjI3(I18CY{r5ASCIz$h`h7A_BkH3{9@2&^OA0e>>C0Et&H#S?e zzV6zaySELAzG@i-^7sKv!BGoTsu8%sCjg@1ia>DZH|3^Hxvv-(D*mw7is%e@MC1I0 zd;&Wk*{De)w@nBbZuQK*8qf->w;>q#Pmn|Y8|z3A5e$=%K9J6gk$w(Z<1%2Zo#ImV zW1q>*D}GRtDk*%rZ%VXT&j7=UJfFTD&yW7lA>K4|Cp0+7xhZB%_oji5E;`lT(0V<< zmj=E}`GIDTbYkXy=sqv+c57*tAweKYtYEF;HAOhy(kpKXAGm_GLYfMho_(o zvrwP(QCE9(3mrbYtka7(4cXpdQ2j9+;uw3kBDdHi!G!70u<96Q42JtSy zv?|jG5T1#tS1B_uyCCXQBCm)Fo}K``yG$a@+-0{dBh~e8mR1o>XK7@xM5B_h% zih%s0rf+w-PKE8*2_|`Ny+MSFJYQ|~b}m>2?&1;j^={F~boG?EGweEc?|MyTfjz5@ z|Ame9@2|6co;To5a*t^WKKL*n!X4D=8w*EEpBbE!Ylsw%RygC1|MBXD0sE6}_kK$p zBZ+>Q2Jc9`KdMwj{m@)l=Xkqk`Rja=If+4Ct8bW`bDKsiOw-2Ky1Osd!=1DcfmrN5 z=pTwpnZBnpl5NBMx_W5&pe<;SS?6~nZLsJ%!Nl(#B4~D5LoF@B%HfQvkNDvu!R-k1 zteP+p4R^;rQ+!YP!RHRAC1V``jJSVRomiXZ+IvFg82WjAJxI{%uxuy7&UkojtM@s| zAvo0|s=cqvgIyExb^5;EN=aCXtks^~VQ+u8^FYCWRcA~VUY?H$$-&I~w2vrm_C8{& zLF!4)K6!h%&1rx;e4L8S_GLRx@z+nC*U~pn#n;Mhv1DW?uJ)@=#3$kbl^F%wOe03m z^mI{`QC+4iq(eLY!me6;m=>u(MD zBhi~rNREPt!g>NtGNu55p- zoY*Y{QCD#EhN~PLO2h{(U|aavsAVoB1Yb~KcKp2F`HXdSAfioO#-!=KOWA-o?s>|A zR98Qf1+TgskfhX)k*6kPBxL-%(JA^V9zB%doH#(p&Q)F2SWD=(y3*U`5ZXH%(Esg)__i@I7f` zCS|vl*9Nckcl#!^Mr>~RFWBj-M{M4vBK`R^?N zC$AiD>2!{3@vpAki%2)ECKue?;nBKKfmLVUELvUN5m7pGA7lBjPfnJue%9HXd4h3{ zV-?Xebk%VN(7-DQaU1NU!iN+nVy|C(Jvv@vm$09P{BK$S;j{_Po8E;)PH$WwUL>ks zJYE;S?xeNKI^^Z6e>GH1!EalcuFP17D3zY6NDy-E!(sNNZp`T8!O^1ShN^bi#>-S; zU^Us_e;B{<8TVtT>i`PWF7T^{%4HxwrnGdWB~!1*RpWO=RBO^=5&)l7jIH~_ zF@Xo(CoW^dgb1_|6!&(IlvqX{xvv|;*kTW>CyuQFRxjcwI;3k4<7jn;jXCuF8<#_n zAFCgKPvNzU*aEvWjH&$re}2ESbPW9|rf0B7F-cq}e z9t4wBS61-f?_l!i0tqLB;B!k7S>bosCuX{}&P&XMmt8&kJz30|*us72zi9&m{ipJ! zU*J|a!Jf%if+z+Wf(~#Ulr5T65B%afC|0GE>*UxwHozNtSs$27f0lI1G82^anR@`F zhK09tzx)ZW1}NYyuV%RJPC1b5syXqy+~Ri^^h%DDD~GIxc^*x#gSx0TR;dfVfj+;J2N|aUr+r|@&5_{zY@uh0gzvkNEyNd5rljgyK|LD{#?MB`X0ZQf-4+h zW15zPz~-OrLUn(pp1cGeNt-H2DIt>FThd94$P~K*?qj4gIlKFtpfeoq;KY$Abkr>% zet5j*IpM_#oAG9iq8ztvVI6`@zw~Dv#1jr1K`qt-9G4*-EPuxGgKHUGnFNbMRn-GT z0m0nRWnl4%v3I<)m@e69{<`asNiC zv@3z4=)aO?q`U_*iOjr$^xDCd6|E+ucdN~abW8U^h6Srvdfg4*0**-xLJ>zs zO9@2^m$`&Ic6x(Uc*T~z2XOYXnqxd$>dLK>#2~hoYmh>6R)-%=veWti`&T^yfgsf1 zh5+$9XnF)v<2|;#QLv-Jb&x;=lc;MCYsQ)*fJ&EajBv*RkYIl8~`qV&P4TqoMNIV<~rP&2_46{avnjSTCEaw4p#ZJtZN2CFyT(le%N&)G&^Bxg@il+E_H(dYL2cXzE_m!izH=y_W zJ&jw$AE-YB(5IVB1VyEq)gE-_m`eS!OLgn6Wk8C*JwigOdpavqO1ev0QY;E029){< z66GFsm~4a@)3ZX^L3THaZ&n<|-^=8%)4@eAxKCPf>Gfg3zrZf^Yh(6KjBc6KUK4KE zpV@OTg!=I9FvoljYT$55)QsvD*=3RtZ?rtzFzCA%m7vjOs2|60>W6JkzumF6HZDz& z1a;_hgR>r5N6)YPVpuTkG|l(?<@jb``h8_7CI5XzSq6aFWSo=@(XG*AQ8KdzH#Qzm z|K$EGD5jibynjgtKktgZ?#;_kY|Xv_L|P)dwZ-Z~3I;&EiY?xWqG_^FqLH@U1>IGxl9 zq3;*!+>G)|^aJHQB9Usqg!DPoNu8e+KhL* z4k|gL@Ky{QZ)YNuLoH{R$JhOu=^AV?n)w?M z!2KVhbnB6w@TXBkAn@E)%(DGlq)Tr z&iE)!DrjZ=$6_8LFfdppYsGd|0eWy7A_PL8E=1PWary<+IJ#^lPDqEIr(;GOXQj9< zt%ppui7%V-Zv3bQm4M#(g&bF+3jdv@Z|}knlY9)Nd|WtC(ZRgddrb*oHD=zZK1Qh_ zC?pEsIh`nSI909Cl!0aUzstErHYiPYbjbs-e+3E<*8A@*XFn4K88Q>4`v;i2g+G|< z6E?tgwMNHBQJ3-d#|fNsBm&8m3PF>S4}xX1M+lW!64Z6{a6v#8dH2HmqW9A1=d0zc zkd5}ZkM0h}#?4%)BpUwqggnuUL5AF+$d^H9l8xV(JO{D)cDrqD=d%pheS#-9wWn#> zpRj5(=`UNH1QZ-#CSOYla8{K%s5hA#ILxi-zLZq^z>z3Kveb@STq~Bt{`KwOb|{7`0UD`q?N#GiHPQ33NBNhr z`~_tiHO_8~Wml_W*M96DG4WSYo$pW$Kaer*^3}N~G8J-#YYN&8&7U!eh>*Aa!21p> zJF-6+N%fJx{YEZ-*L0Yi$4Y}qYMrt3aCC!W)Ifx(aips2N}%G^g!9nERTg(PI}E2n z*OC1P;*UA2aiTP{z{UU3W`rxRJfF?|1+(!;%$X6DZzZs{{?^?A3&P_1Yfq@Be@`?c zKoie-Ur^^FZD}|krH?UZNPtl9m+r)_;#&E+-uUu>0m7)B?bT)zAw2NL?JvY~B0%5( znXt$!p}bxmg;xLR;_HtU=3Xb>7$F1(62|S&e6-;AqSTI7J>R~X{3?^oeVG+c+@l1P8r8d1ehdi_v_J7fEuC&kzI z?OM>39#zlc!e=nJF}J~TPoQqw=+}k2g2#NT_e*V_B?UgLMW_pr* zSCMfu$Y5uhJ{t5uL}VpmTRUcr473XumV?w*o>mqVR$JGy)w2EbrfM}Z7O`@uA8Q)T zW^(J5QhE0RZUoBokm<+gWauU`;-s=>^@7L8Sf=PtDo&zIPqd#LKFR2|UMf{dKBm># zX^JS`<%x;vf%9)Z&|DdErFdm*q^u(&`AiVnhG7b`9B6kIu&_i%Z+xrPZS=&H2+1M} z5B0_Vx2Ms5P1mh7gIEH)q37n@IlUUCi?8MWEYv2?#2 zRNIu1SKMQe9gNz*hM??t+#){3Rw~PuUDR+l4K?S_pc{UzIZ-3G`kEhsWKN98cH*K- zj*54i`v?`j-g;HmBsx$R&&A-$H!ZZf@TJ4K?fTHpN%(KIV$y}Pf|FGq87z*CO%N`Q zPP!4PZqSDDzu|YIIrE9ZOzGm?!d>R}XD!@xBj)@X^KwoZ%sTCDn7eY&jR+@!T)AW9 zCt3FPiW3vN0h|1m80gM{xZAIR(Ov1?ZyBl3GL7Otk~&PVlS1}iryCwRzzf^@U88f4 z<4xV4R$^B+3q6@@>1*l#S~*3{M)i+1!RFnMc+QUbcA#Z;W^~6Pyg15x9X3k4m7a{tLR;g8=`65(lA* zNuQ0SpO}(TpvDT_a$aQ$wc5HVhYW1wfo+IaYnsdKr%me`+PZ?%GJnW2++TM^lgYxo zYnwo>zH|$d+11KpWuB=G@MX2}wjb5jr(uz6t2|zCeVBZB&B@-j%oZb@++Q2f8IPJO zdW(DfI&i?kKOY{FkAMEwCO@QI#+TD^{*n-o5>>Czto>HgQs(1|Tt5&$;^5l@31#Ay z^=HW>rUzQjV5phqD^V^Us#MQAkw27hOvP}E zP(+irB_&KYUvTf00L`Ai%#}{|7sfwo0?Ch6hwALXJCD|*>U zQqi=MFsvgg9}f(~;^_3{5KDsX%|Doqt@kv<#vP7Hd^mVVMwq*lGkjN}gn&WP9B&6r z6YhN+cQuZw@A+PHeGq$%apQ7e=TMMDl0{Ob>jnS3+IMG@9Ni?IFm@l-YgSU*QR_+Gu3hD-R1P^*l{yI&C8u0 zAT#lHLPnaYRJGDn@EZE8G#3cRPS<1C?cbtlpUWQKN=f8$D(q5Zxlp#R@ljd)LUDK* z_5qU=e>CA#&EfdxAiP?YZUyJTi)W*e$)<0IGkMi!o+a5!*|_#4l6fm~bt%}8E;dO28N}Fq0_#nY5 zvktu&F=$UE;5w9~=WJA{m?G~-*D88g319{ZbnR$t(-;sH(d4gASUSd zd%MOsk?drZLYWE*cIE=mTo_@C4I-0lo9KiG&i^zT;UW7>CJ~7kh!NP{l5+F-rPdLU zm3nfZQMkc5Kj-ysE17b5`mJc}ao*ItkgbxCFhSx8PnK431T8kr$G151Eb7CE3$gi^ zFyqp`7Gv((MV;POG^X*UWp~%z!0g35g>U)(LDC}_+j~e^k6~ojz05vnp2Cz1(00n9 z+-rcBo!vke8`mAiifj(EqNNa=3Vx%bZtHq2_~e2zi!ji`YDCteaNBynGFES_a9223 zeu_JMXt}G&-G=h2-1i0-JFY9_QKie8>6Yx8{C7{P3YU+``F_AYuoE+7yl|Qrl_gyUAY@Z^I40zK~zlV(v}yMrijfj)-?_QsO%-_0{Klkh9>Yghpg|6$ve`}kNbOxJ%4y8! zYRLh0xNy^>bej9i3P15&gGd$L>TUS#sDXhNdc<8O^PKp{o17J`#F0a5IrjH0w?1|N z8JpZa!!nj1Jh3jr3Em5V?MY8Q$c$Vz(wxJ_61@`3h|pyQpj5DXMUFW~?j}t6sCQeCaH&-e*<`6Qlb~@q&j6|E&qyb)fS{+%9wA z5`x~n60=boUvB$FO@te}Ipmwqg+E~tB*o&eroRyT`(;#kBhS~ zSDA+IOzYpM=eRRN#5*iyI?Yvak7}gs4~}}vul|;<9A~vZI!c2sm~5jGC~aF5tX6m^ zZzRt1I*?Moj-%%lODAmczcwSW(1`v3K+0 zLoz1Y&E){`reP8@lyQX^`hw2g(O7PK+B%nt;(2T#(cmrY<0{M9dn(cIvI^Tzy#8}o zzxSGrHrMNi|4lAI%-N}&)!V#H4-Eq7WbU?*n9Z=hkERk#_wC?#40?Z=(MtvtK_X9E z*E#6|Fqf6f_SYj(RFi~hs9_9@9O$F>RM(R~PMRKvr$k{Q<8-I?>|JqzFA+Q=Q}Q$R zR1AowQF?0Q@tPQ!Z!!LTSko30tELM^U9hQF2tx4QnMIyu=Ud0nf|+8TrpvDcny0Q6 z&z1XXAxw zp&wCE_LEO#c4)(*XBL%}nB*vsR^$QZ=;U}RyMO{K%9#v}O8B6qxz~@ z8yu;$d6INI`* zVFP|WSj7xMcP>YwmNW)MSFf{sf`L6_Rp(oi+7jBHGq&JHD>3Qv-GAD%X8~0e@Ta8@8@ku3DH^1TzGmIT_QJ( zvM_*rtDR{)F?V}!abs!A+32n8{PsK>&#&#b!|Z0iz1h-m#;>GeN?UzSI8J4A|4Zgs zp+#T21D`p-o(gi`jh1d!Hwvu7<{t<0BLew~2dwdfn#CH9Mw;=i0Xn!DI(y3?i6%i4 zbdF`dvY>sy@d6*WTz9%s3+x@H&lGGwJ(`89IgwSxOPSUu*}Si;7!C`XG_0Oh>#h9< zYfHI6_0)&i=LxmulXSLtT8839%Evi)<{V`)J6t2Li1aJ#TKQbOrc}G+|`0T^8b7~BUw_qA&oT;->#9Xc>^0b!*VzYx&NeW zkto(*r@c69#z({ELgX!?o~%=y?fw2H&-twDSd#1Xu<1UO_WbY%Rk`9BRnyS3_UChD zGB}MWo@1W2?SD$d9B>H>Lf$ zp{u^W75LPYrM1tWWl0L~U@Zr^1jZ?#vG3Hy_Wgz~r_BS8oI=bEegesMK^aAQmnNHm zoXa08VIY=8ElxAYR4~=eJS6@3%Y>`TnS94`^js=7=3vM)>4xX8z%q+3Zd~VWh^D)l z8be8PW$z?R3&BhCb}%mHzV;!_w%|Z2G&m^pvo9XKUE{;R*71yGa1Ma#dQHg^CIaxf z_PVLkq4c5je|pG7Zz7xdHa=-ho~wUr0nTwI-s7_K<7lOLVHmIQxihhX_KNy&0zg1Q)|TQnTKk=)k{=v8w)OSrX2a* z<%w}|6GajRgTo6rafKc`xysC~Ch9Pz_^lUHc`v~fw{jQGGJUF-9g9@OpG4cG!+4gz z`#IiRb5-AWq#332uwT)J=II9pg|~ganzDaScd@(5%+zvx{tWdI_sGt}2+KP0ngw5! zo6jWIq-E>W}Nsc;2p-6M4{^aS3xR(4R*NVy;mAKzqg{eEC4mr?fjnq$xb>$t(oz?NP)XAjIY!kblM1F zXb^>_in0;0onY&(V8{RPCo=~srx8CP?FY8lbdJamX005X-Mrrg5|?&L;``>JIGv#b zm@Cly?FXa5t)IEx6ekoH->Z9} za<>o1mg?v4f2(aEv1ih}I$mYHgzxM_xHj|a3;$AQTk2a&2|sKW@(>1Tqk5E5+bA%- zBJ^;sv~s{7|GLn4ag2`fjr#su_NHk{GUDZwBoOKc_Xu!b2?|N+D3{$`V&NW|U0S=pp;8I&CS0E-TnA1~DEuhX=s*?|qsPRe_t2myg3N7I@-#z`X#?5|!F=|(G z=R&upB(!8_rAUr}fzZAzTnM&1-0d`O1^qiM^|&iF`dq?B6sQTKK`O_f3Z;YY6=UW61hu49hQSq4w(1Sw9CF5cQJ2HOfMfy zj|B+v)rQuF{#D3}6ki&m6dSr1F%(8YRL3KA$0Hb&#NfHg*6|0tycjdTn|vlTCW4V0 zBT4V68lsa;T2LDAK6oC=M~rBqHls7TdvuLy`L102MDT^slUmqMVUDo!Im1d0r3#ZP z>hzH+_U=r%Rd@?#rn@hech5mNy^#6ct}on{+x;@y7jS(+Dsw6BYi<*XsIBC+8%Al% zvLCd^{;1~bt9!yE;c-EiFe&HT%l1Z#Ux5M_r))nEM(IcyDeey6N*#@~WbR?8=08|H zIpMJWo9O}UZ=^5>T_S_bO&@m-4UhB6uXWKE7@LM+S3tXRy;ly`S!T?vcvYqkrShI+7|Z8*P8#RB{=h0 z*$2w-uLXs17m3Mj5X(BsNddKoAkl&CXnLfO*fIB=f1hF7$;=Y}@{7YNRMlb6a{qGw zKkUzrA*4Z3={l2Q(jC_ei2w|Bre(ZB7T-I6q;33xW$G<}HiFB7`KO-TAITi`dhkX8 zpG0I4rWyEmL+2zD${CE`=2Vl9O2#X7m0aR&@*-iQG)1rE0Amdp_@dy~m-ce+Sk?oPa* zJjr|Z>wbUS0^QCP|A&WS|ruiH>t>a;Rd29}_|y*L#ZV#ob_DokRSUYtGhW3Qa38<( z`7kEJ4UdQlPa=KX`W%0vZ*uPmrmxufL?0#A{L4Kb){KU-I+f6GMC?KjOpAq1o)^b& zL`ArjO$F5n9{Lvl^Xzme$Y8OPWY;hcC@8riPfRoNPzMiLy8YIu58@)LuAi#pYs}@k zXIc)5Yigt(s;W3x)RxU3=Wg^tbui?8SAVtGxZjff8&5QxjOVJL{z~@rpE~vDSL#pC zQr&8>;y6i#7@4l53>e7ZQ}!khfBmC8bHW$B3eCO5Y^|q8|JhswXh>AwBu>H_tp0rr z;Gt&T{6ShyRGSGIo9HdCu~|fPAEx}x$g=$Z{|}i{vZ2+J=M(?$k60%REv^E_*P)lY zr$^nVHD=v3>tZg5op6tx>tgKPKF=yE-K`>dbr4PHnqZX&PIF~3JDsbaJI^0rAJ#z; zj?z1lO3ytVo)jK0KGfdnQ%2$9_DpNqPagfYrqYBT-CWOLKQ<$zM$+HCez%>=D&P@Q zA=JS_ic)5+Q1*4QHI5GsN6dns=I9J(7Q@(UAw2#$9k+{9&Pv@;d)5b_kD5jnKt@A!8PzP}&sJS@!(0frD-2{==h zn2k_LZC;qwP%+YF0^KIqm%3-p8>u_})xJHcwV8!Jt&ALea;w$&uGT^IzdN zahNNnPf(dz`YseHT2P0}mE<3o-y}583|oOVP|nm1k#bY(XopwQuqa2poMb}a(SER{ z(F3WnoKRN^k7ylwg#lij*=4OE$_lrj?^A1OwmlVpt4##XkqoCB_HsTaM8|BS_G)eL z7-d-~p_5Kg1EVB?E5YgV=?#TZ%o?Cl`p9M1y%#}baNIbEZdC{B%DwPm@DpdftzFjZ za~t1EIisyOGrke}ma38g3}aiSt$qs*OO1rE>&o-@bvr|x8=1#%Cfj#OJUnY)GzMV& zOdxi3s{+A8LmQE{TOOb@l0YmduqVi#9(1d-pYFG04b{wva>o#GXxF{!2^m)4CJ(A4 z2tnc2_~wd%=Q<%S+Cnhd{RW!QWsuR$GCep~yM|aZ9@n1sJRb67wdjy#CY+J7!aTM! z6jG_nbO(F(3APO*-o|mhtpAU+HxGxZ|KtB#Bt;=*Nnr?;WioF2mVGNK*_SB{#UMmk zhKfS=J?oT2vd-9LX<^J*v#%jEj5RxB`@YZpx$pba{rO$j_xHQ5-(OrVj&tUm^M1dU z$Lsk_ML2=P#s)1h+l}~Ng5@c60$m_v`pBcBH40fw?yi^DrTGCM>ssC3T4CcF#2Z(U zlO122OJb{;R__8_km4j4utA|f*S`vai<94hncto1&YOu&!}eHA7Sy)?+B<8tQFT{S zNp5FnIk7GkY~+z5T9Xdqai9~-7UV~qv0xNCEyz8(J+hI7+?_GzYfG%2CztSE^K3!}h@M3WowGK|y|<2AuKDy`KApjK&Hra#o? zp3#f4d!oFfL9$#UTrhV9Ph5xk=N(u~Hq0T~(#Sb@jIanvC8L`+U(yfQ>KmOQ?SE$U zy8@YWyk)!_KmT*ow?%zUD@#H}wv>6GKCZM2QXdkfM%!XU-KPTbI;$aB?v8}{-Vq?D zz&0@7Z$#~cMS%IBM?`KQTE%yA`yhj@p76U>*Q_2Vm}l6>&UP0jPA zj~~x~Vo4gH4?~m#QO3pBpSJc}j)wO$0WeKLD-lzPZX#4gT{RbL>CxZY;VF$bX&&Rv z#^WFbnN@sZ4LWKcGB{c_Mzob(E0bx6xMkN%FTr#>FbN9;Pi!_JfnT}7GPw#iNErBD zEC3pv_J8recHSxAqSdTZV@C(K@g;oK-Eh%Nv3_LkjJ)yy{TdW62YFTYc`#T-I zi`frox)Tl~pr@pUYbQ{h%t>?oL5?DlBv$c z-iHfGsXcJgy_%;=W$FaJu&8=cy2v3!Hoh#$f9EA5Oya1|<7f|`JIs%~$;q#v#>-e1 zH%?$mS<%nQe=OJT0m)2g=q%)BiOcS1JD%B>c(82cohof&b=W-QrP$lWb{sQ#DhRQn zC2YBV)Z({L!JLNHmUVwZY0pMEDvEbNWdI0d`#zvT1Ske93Fp<0KRpqm;<J*fsj;f=mP>EWZYe5>IOnOEZi=NkxJW}`9QZ3k2b&z@Em+PUmvw@LIW z97EBi_nG6}pj!aQ)dw-8xl;cMKQD&`V{C^9>0WX8j{T_nLP%K2Jj>_~S*o!r*}Uec zM1shnPigPCOlEz)?-%`3H@|V^i56+h?j}>FarO9_fO`YZz@U;bjNt;p6Q~c1;-3+x z0a080O%}XLS$C zAZhZ#;cTY*yP{>^S!nvcVyLNK#ztV(u;P%>$N7zViMv033uHdAK0SEh{+{?FI%mC` zR}b7?*>8k*9NehMz!>8}Ey@ITWLpTCc(d0m*D>p5*o3vwc$x76XPh*qup1%@z)W zh00rUhin?9Sj0DvoWHk ziY_IQb93MvN0$ps)eZGvt$ECEoCV7sDB2i-`mDf2Xxw>~>t!VFS!1hmj*A#`#3L)e zi#yJ?{)zNU1C8Ev>-w`Yf3y`~^3lWy?>qFA3jt%^-}LNc_HM-}PnSmfWb5{SbUcw- zaYtWdJR)qSS@F*DI#*~2W6~WA{%5e%N5sfnk^xs$vAS}Y74HwLqTJQ76oro=x|HL` zAkh5A@9nP8&rihftVEd#hy(})gj}Q0E3akNG4LPCLcTC2oLN$7n-lE88k}~ZWs6or zqjZ_SpA#o|ie@q_t+Dtihzgktez34!zQdUj9L3kJa-aG8@c z7RHAZQR_7*5wVNRcb6%-u9BpAM*LVtGePCAmjQi~J?e97C|68=>!Cp5cil+w9BQ-{ zqb2{uSx!SikHBWbA#@h9CL$*&vq3beTRh!O(P+NIwcBV89{U{(lr|naMZWd()Zw&l z@zI$8z16tre9Te`t|Y#vmrCVp)|jhrtUf;g^10u;mLjx=BQo4x9(r~}!zq0c z7F2TCTLaYFT=;U2-osPI=vk>cua8Z}LpgmOk!TS5|7&Xn<;cGFsrU!aFTN{bkJp+h zwCTvl3~)u&e!zMY$e~X9x#(Pl_gt^>*q%MJZ99c96!OEKQ`rF7#V=W37>%!MG2XGe zX1aEYv67et_jF8IKvbe6-?9xwYbPyvnb%3@Vtcx6K$rS|p9+!VGDoEXFQHgrPN-mG zDlv1^MCJ8u31V5!a%NDh#1DUwXTemnLlD&sQmjD#Q^#~K=bGs`=Ow7Fg=*| zC6Q}04^@br_VR&|A57r5&U(Qmslzt0xaV5ZB|bWA--Q8`ZqOur`7t?rQ-m8S|t85y!D6RO#-4I zZvy7d(cZ6vnv8?FN9WtHVn(o>Vu`vwidI87!{+YX+E<-e_Ra_kor#>n+nwQlWX^$U zPV7cQIig;IYFOBmGv)Vw18kMA@d7VddxnCC9})Pqd@YV>HBgu5$E&<$JKDHU?C zVvDUIDWZtJq2v`PV=4-9wekybVpm3v{pMtBs!8$s=vLdK7ZDykN_B9n)36s&H`a@3DY~3F zr^PwPr$Y6ocv4afE``Y)mT_AcfYkU*3^tmj6%f|ex~V(TjV5#Z>n{6c!K(X*8Rd%} z!=9IxXRX%APb%%NC}}FdN42U=b485_{)N{IumAHhB-|MjRN5l-XWY~<$rdG^ zQXv`z4>;hx5KrjXYAjaY&i*_-9-}U?M+OT{l#r=eAlFUCr|P#!M1*%o02RtM=J?!# z<Ijf-xH~WIpQ%t5tv`E-zA0O( zAmngIi^4IIRQfgt#s*(Xb_I%o;jX3*g(rFi&6|(bXBrR?Ic=a3UU;@JZpB;Dl< zmS=(AcunXo9dZLkI*cj;E1ECTg|m$ty>~@brkpGzL0h`WsYS>&x+;qQbJM4$9p0FS zJ3CnI`bUG!I)yWUDt6XpAxp-h<+EElf{ZeS)rnzjMiAAgpdJ-^DUdyO(2W7v!(<)QQNGF>s+& zX(ztBQib|20=RdlHC5q=N~Jm51rD3ye}520;Ri1?Uph`yqkAqD3)3S_!p?(X-PTRY zK>BSrKQoQxaV=~M3h~k}E1^LZy|WrN40E01tyrBq)>9U=0o47t>N`em*~DLIXvE*j zif>2RSgY6-8}ihzK~rE$?1{=}0(R?tXF^5PbhG--MQOVyD96jJRUv62g}<=fA|$v! zA8_SR4MyppQu51dvVyB6NM(%m@!aOTn=u_8=Rp+oHOT$xz%dyKwhgtiZj&AJuI^d+ z?s?ZTBhpxMAw%`6Ewzk+Tyu5bgU6N%JTW0ryX}c@qpaK$a!}&47%(GyV(wM>5~TB4 z;xsTOUbU8Pj7-w}p8=T7@iWFGb^7CHD-ZJLR-aP`D7JZ__#1St)rTT@f_{KAT3YZ4I|zJcudc837HI;J57rj(|3@PbBx z$q~jY$*QXqNUk|FQJd!to#3;e1EFbo4EF0}yYHuRJ(i0xgF4>iwdB5PyS%qSxlz|W zp^-BjD}rQQN{j!5sGTYE{e=S#UMDPPFYZB8g8@v1IG(gY!-LA1F=yf`)g6|jINUTxMs84ve>{8s_@!XxWt@nROBE%HxmgDfQF55j zEgVdUR9N$d$eF=mJ@-`3y({Q_n*ZKb!0XOp5!IctOOGo;=QmrcO!PU%ZTYue^L1&{ zR;pPsN{>CkX^_Tbqkn+b=h^KWq@GHLA#R}n$K0T8)OF@5^nE5x(OMQZxvfLiq0hW3 zrGoRN^TIL34`V*7r*r%M`W)?4rsrFfSaB=4E)PRa-UE5O-=^}ULOdledy&GF1c^nL*BU;RuJyW7UKm3%GcrHZk(rCAVNY zK96-P?>o9LSSy<(Y3g4TLclT;XJDwW7H3U8{*CtL0QVA$YA|csbf}DLR=ejxLH>Ju z13S&mc3fP8ok3%EjH0Hyi14}Xr1RP>I|ef;_7Ve4iFD@jB^2|C)MsNbX|-*0^i9N}ro^SHd=j**9p^ri1 z{p7RwVoPSuK>DZZ=qS-Z%>)AtrW~)Xl#vo1{WoxpGBEVUGt)u+Hr#sm*h;_F2}cex*dEWNOb z9?$iKtVogBM$ZV#=2u5&eM7XuYUy}l$PWG=UV-}N1_QnhjOL!;HJNIT$P@GOd152D zabao+AB&57=#F%EQ?0FF=Sy_;DQrM7I#W|!gg(yPEp7=qi7bk(jCs^h__n)$8TG~0 zt%EBcbTe$IeVRIX5zm~7YDsspR zGWHjTw#bf$FyP6-Me-(r2c}+K@o}2{GMF)D|5(j9fyq@xfW&i6$43IS;vm1`&95}z zJa$krusy(ZJ&~N_@7nb1rGb3e1)fE{>tm5lN~9tBmvJGj0Vu6Y&zuWIthFUGWBrqi z9p8-7pR&E&`aRWFzTt-!VkKiyS8ZnrX704>cJ0+G5{4{}hba+xUZNOVGIY)VT=IF5meo?W*Fw1L@;$x^a(<+u91%8Z?V(X9b9`|GYxA-@4~O95*?nOMM&y&u>^PBffuy%y zvR;P{vW13P>_EirgWoe;L-t?d?(0Ff-mBT_A($d$C|A?vQ~&78ZOcuI%HKSGEcAOE zN91>6gEO_1a&O8^AfTux@uJ!bXJ6iXTIp^J@a3kh{%Vnk)to+=F8%5mLj1UaU3F8Y z5`M6>&2N=AMdn=^!jodmDMV_Ye^K8y|ITtMt+qZp%`2uDJ~Uzt9|}e{tu?Sfw0OZk z-NEpu@7S+C*!6W`qL_r3O5TA+1h?=S; zk?YTYZ0i4EW6Ho$sMtfY6TPqLXdf0%A-DUnhV9kjT>AuTKAR>7q)!17^>O51#AjYG z;9%h1JSAbiw=-^PUz3m(gnQ%-=I<_@v5u8Q2o5<21va;D6wK)w_AO2=PZtU4m7OX? zOf=J36Q=E|32bs;1e1T)fzJbFizqJ{h_vtppVO|P=67|1VdG}e`TE{NKA!X6OHs<^ z-+K=^dA?1QdFSG6pp@iwExM3@-_ZG;Y+4w9#43M!%T53P$sSJ8rVd>W;QYdT2q9$@ z@)k$T?$#`y>BE^}fh9IfxFjlh{tq(?RgaN{^?P=2{09*D{hezn2%y@d{8d0W#L@-vj{Hx@#QE51C8!uwxuBgIO3)yhZDFNv@u+T0unX;q*wuWmQSZj6~2s z!>oDv6ew{6;RC@G8Fi(f7J};t#(AOuc{=mDz;bk2qOG;uni-z*GF@zKQ%6qLu&l{e z9%twyK^Vk(H#wLisZ(OzdzG+eADtjbShEf_6qz#O`-1FRZY6h{hJ!Rh3xm8Qi zwU>?8_{#J0s&a(hVq+HMUxd9wrW9=1p_AGdW*G~AtA$C@ANC^VojU0wHM&u|5iVsa z=FH&QJ>_H@?J?JVRB7|`(n~8i&jUZ_Nh!&TlB%z#y=N0_no`Tp`V$m9+U#J)%rvnwk-ax<2W=asE{YbWubmz5cM~Wy z?t0lJW)sogWTNBVVmNgNp1(bQF+a-BCu&X3Sc)6rXX!`t>84WrKxcJ$nTk{~~@C({=6Ln|zLn`k#SH}-zA1InVaw3G0 z)D>wGvnGIWni`RP)Dv>hD@TRjKIJS6@4WD!d~o90Rh|aVZ^}ncoJuiS`x$rC5meFK ztc+8|;0Z+%Y9DLuq{E}ha}!{cHZWg)4GrUYGp zuaw>zH3ymqcMXQE%FoFl)mlb5XI8x$aO@UP5@b?`4$v;6CRIo+jiUVNB!5FWPSVbB z4KoP`?SAy5c3~r~T|P1(cY%x4N;o#q}QR{ ze94IH5b>2O3%UX2DV3o1ku9d_XeX+JuRKLBM^({jB{fU=!lJ(ml!MNFgmjdZ(}1C? z3`%ADgv;Zv_k1|b%Z_zdj5}iZx9Z2!gFmzJ6xwFl3!4zxnBhwg^y}@<9o(RmbIA|g zuP;^{Rp{k5`t40cpQ6PdO+?&!8O7rWwQjPrv3}mNKqdZYbo3E5VZd%$gKXT62c+iv z?zT5s=*)9mu4ywSS#!UMJ*EJf2yC{;NwSkssv35;@?gonJU4?Yq%xdz65O~1Rmwo> z6e*lpY#ZLp#1{igM&$U~<*SUheEq4;KDpA1-=rh)ai6!DS?vs(?A*|?Ldx%~>bEH< z`vRkwnGOwE%w1*|-^UFFEuMoO*R7Ronsr<=x$jEqZ>Zhhrsbq772bg6x#+5W+yMF| zyh;oiRk)iw7&Z@r+`Qj56@Uv_C8mfR598(5idC#)uT~gLQv8E{|UbAo`(UPa)^! zXLs@^eM)z=-qh6ej{iAwYh#rp*2fL+u z58%e%(aJorH3f&RCV*N!WIe;FCXneMaj;V}kk`>gWR&ZMEST@5ecJT(nlF*nyYgwC z8<%%RD1+TlgTpZk--1Hh4LMN%2__jGHmB#s))bAa6Xt!^SZbQjzUqSjP6Csmxr}q= z&jd;jXpqYyY3fr#^?CzcG8H%K+jqI_-1Ym5odSieO&5=G!Ya&xHkUOJCz8g%xsKN|y2z@K_HJAq+5HRpKw zwK&~EGIq7opSA>aT|{6UG1f-EPOt&sbtQ6LZ zW0DMZRa}9@QN-FKecSkAxY>*IpYy90j(zvmM<_vMk&oe zM@v!W(xHRx6gG)HV5DP{$7oJLV%xc98EbyOXU7ivIq3B=E_rWwb~)%}9+kYp5qrID z!$~Lkf@cu3dLXzWy$*PatD=B_*uU%MJbVtqc$!Z>Qm=o9Kh zuj~wAz^?wbuUHOSyI^ehTeXFghFJQVtt>oH1@zrCeBK_ulo?iVG?w z@|>k_q33w?BWEo8fDgxmV}Wm6JF7qem(!7+LF_%D_no>moDd3|HOjyI9`KATLEUNpQxY;`!s zAOpskdDRRPI@^D6Bpy6{Scxcr7vRD~OvjkU8)^jZ2G$OQuGD49twg<1yXa*2-uoZv- z!p~D1;-#(jseO@{S^{J%(_m(cA@x<|i4v&&X7tH;Xye+*g|B0ZAjx#*wY3e-`dS>})9!~)bMpa0Q97fd%Zb86H zK*i$&P0#1*2@Y&Zip0Juk>lf9W#6Rn!PPJMTo#ul;Y#0q4xFr1y)b_!zYL6ZQ75DB z)3CX+n2-My^m)+0Z}(1Ez%69Qho?1>&StgYJ{Q>87Kqh&L-E`m&lx&vL{IN4q|fUC zyy4T-X44Aa6&uHAd=0+$L(z*@L<&Ah1DUo5Pyq3!fytzaxEpiW@l@rAq zGQvLC`1Y`$f^%Hx3_UUqcf0T6K+>bLMqeJ&q+}nTUYio$;yepGz;jyP;j6u6bjm)}kC3o699QcQxqJ34 zZbsbklYY3IADpBU$*|v=zBlR2Bgyr8BEtczJ z{1LhsRa`G7=!UC|X%DAhx07cDUJz08I0xf#)9r9|CDf{;c}>~dL`77ZJ9WyI+2*~5@7One-=gT5>ChYS8| zc+g%E`oGBMn9S3s0u?8^r)X|$(`)+w_?u`0OZ1&m_IuBUXv&W{|F#WDSvsn}fTOBO7IlPJbOT zkM;k~l-3Noc#8CyKKWp~s^`wdH?ZXp!0f z{j63U>IO<|9)au#`t{Q|D*kExOSn~@ak9tE4Vzp=kJYMoIUao~Pzj@u03R+g2= zTeeg2I7rLGe?rL9A<297lIQH?NZ#5nIzDd^kd9 zd)u&vbv~{hTN^*KH1KoZ_3B3XQo|_9rW7~$7my07YD~kB+@P|a>MxZ$U1e8G(~p2q zc?3C>BQz1DFj-tSh4By$lySW%q+PJvtmPJdx8P%oXDN^^`t;i~_FC+hKRamX@Kme` zI2E@_c0n{grL+(;pWzRhlgR*-WV{ACQIovp9~-%RuKkHJ0&gkjsyTdf7DPpMf$Nih zSO%(z6lk`lXys4(@>`JNADGp*+~^ua7jF9* zTlJlK{s@Q_w7F)P2lrbFW%KsHRPFh*e@px9)be|7H3ut(sKYJ@qM`tkcTW01lLaVg zTm+&J6ay@^dwK|hKS9u)sJZ}U%&T9y_a0E8k)|}@xn^@N1PE04%Mtc}1u2Cs1G9?O ziyJ_Gl+U!A{EN2>YLvbZ<_PSo_5;5i`8cey-v^=?e|2P*Y5)C_{^tdsq63+eu|U^# zRVuJmOQb+*sSg(_r$nht7Br{ttDQ#4kf1Rn)u4qGKpAP1<3%`(0#_hvQF5l%1P{o( zMV>gi9Fc>Nw&p6hB%jsuprYRe?QM|6MgZEIF5+Ycrq|T|M5i|xrHoYEwNG^=@Z^m9 z04RxV=$vTn<~S@D_ylcmFGEe77mPFsqR$+6Sg@P*U#xsh>DT_f0p4Sf54!Rz7Q17c z7Su8Vf+0OlFwiQRo4pfDi*2w1bKiKCiUc9L8I8hKXQ-ulMa&iE_;} z9fcx%B4YeL{qt*!z+Y2(5)J>R2&C9G&jqhQLw89*PG_{rO$O8m4G?0+fZuy*Tx|`q z=)FI_gEQq*6*}o2k5jKNWzkJ+hcRs=MiGAhT+bY)n*?O694o)6(GX))E&Arwbta?w zW$^c_JPA;fk?3m${J8iy%}KCB)gWv*iNCz<0MFmWEqb;4XwW7o_}m}RO?b5j)K$fI zxE_2u7%;~BkuE=<5fIF?!AlUt%af02d}!DK^sta=sdKSN=mx zOqL#uF_s=ZhN03b55hy4P3=@$u0O!?@f3{qSB;F0{Z{P0I%#XFN{%Du=Gm>cQ_Jvx zYKy3aIDMNAUJyIh`QiK4G?l0wJkezwQIRJ&%=NoEViSFnZg z7D;;9B)-c3#}lb}<;fWN)fLT0m97I3cR~+k1V!;i1>2ULzJ2I@#0}ou^^XpHnCdeO zuyDPLR~4O*Jngk6w$GyP)|+OnLnX3I7E|>bPVgwPGWCNT9?t@TI#vgoK4r}Ux5u-Y zhD!VEbG1K3TsYQ%{;w}FxWSXbZ^n~h+4_U)qg|Op(|d@{ir;SMft(U`SdwmLJI-8d zJl@!<(u60Qi|3iwJxGk-+*VqZ7P1|SK7snmdL)qDwY%-oiQtSfg(@z(+-$b~jmyO| z7mFYfN2bB@nt3Na+T}7Po=R)EnfGrsB#q4N`qt?(dlP6g*WCfqF%A>o+ic~s(Tr9A z>d0>ynKh$ShxyEZrVL9~{&q@WOE~xC33e~`bG}sDh>7NX*eiSvDB^7pqiV?#g&b;2 zP<`SdsDLn7Q;4FB!LW1^bui;~xIAAP(NWa4N7Zi5Uia`akV7Q$V!_t(LGLyb+aYJU zLn^tWTo$#P@8Ov|vlU{JbDt&JoV6;JL0!v|$<>_S)~}|(#Fy(vaeH5NgK1ypY&-1E zM)?Nezu`}r1;gYyi%F3SOarUDwDwKBvFp`MgkHXyb8G_Jj>FHjWJ5j;5&LnipQA1C zwTRMa2fc1T(mAx8%z!VCi!MF$^g3fCzhtKA3u9qA$oE@3cnSs#nI>Y3HS>I?HTjn=Ptm>Da*2wyI_aA5xz!;<@u1fY^&t-gezuA}f1M!~U z2zEcvWyi44yGKRpLXzK%VKy~sqcPj9fcubqUsCy7xxT_(kMyL(rL}<%R7?Mp2dnE| zud@9YZA5zkqAm@a!ITm7`bFl8?jGFgr7kSO8x}>;mqStcxXtee^&26hb1=Br+LB!C zYAfnP&1OC(dJ!M;`>GcHbm{;|80SBqz!_wXNd1y|aCqCClCY6qvgkFs^50;N8;btw z*V&T(`4P0&I35Q-=KSV({M&B0=kDk|AB;EVuh`~}?k5oFLDAW7ulYTOyh~90b+z!) zRhMIF2%o58V4?{MOaK4>k4bgLq_{*_#f)ji{VDZhD=`IZm(>I(a7o1; zj&=q2{Q^XgCShXW9#zuMt}g@_$cE4W&zgUrYt$s5#pA-w6m5r zl(<`Z?P^C>JQt!pnoW95O=mr-*V})RH!jRiw-1HDCN2%M?&`~I7)}I#C)PX9N6vd; z_reTQns;JC_o_38Hn)YG@mZ$XUUD1n3%nMP^SO>(oD0`$l$LOyT@aR?#@?w%@}JAT zh6d^;c^Ik3ZohgKAKIdUZvxA@t1y*TP{@vgfCn>o8Plt_!Fer=!PCPA#PTNO~G2r8!e1w~?i zj#1Q1^20d_DQh3B`k-9~nt8Bc-T*@A9lhlMz_<>ihPO)~SJFzrMcNI5El$%P9d$S0 zJ7`*ZdHI_goczH zBN*$-@3VvdY)p=k4qJlJ&gu!e&P0k-?bN@J?&j0)wSek-9gT`eLJ#R9a>rJoK38e4 z{$#abqGe+3<^$wFR-sa&qF9np%Alw>dPTQvX=C^^)@LsLOI17f#;L5h&jp=8Lf!m2 zB-CR)T;sB%P!5I_zE>?mCW94;Yc8_>8;a%2Ne6@;jve+SekEl2?MdwIQDvM(Rt9#_ zyZdWOg8s7$x$MImz9+l)>$XOx)3>T0j7n|nyG$%{Z}0a_l{eS4);vN=g?1;vwc0_@ z;8)g`(!vf&`Da5zGJnRoH}|&M{nMe4QXh&i|D4w8g66nqz^u(W6gIOf{fslng2Wgg zi8*m9m@Gn`@RIDW0(*4UM)j1!pW!i##d3KDBr#apfhte~4D#Dy&n~o`fI{`C9@c&}Q)dh(ogfJ>nVp`SPgu*ZtF9i`Zn<2%#Xm!7QZxoVXwkgrOW=ko134 zd-Brf^vah8Lxa;mwmWtw)M_<@u6uw5V@FN6%#Snr-5rAXJ+;5j#@F*KS=1tVgT3LvgL%tCcPq8nZ}H8O?u zsRLl=^@X(dgBvo~9}f<2kCaeW+A-@KG23bFia-Bz^QOdyF~q60;w^A zY@Nr3dfonyE6f%Z8=_#)Zg{^gyv~1aABp|3knc0^M|SZ^S0Azc63RdtLwtHqA1yQe zIv@+m7G!}Ldo}t<%W_0TU1M}D{e3T-L?j(H$8#dGXU4Ax zN+Q^n(ch#r79%YIUb$=c&(Ehx=&bMGQPYHxHgi-H3@^^!kXxvKD0miL!9gI?noJ5zIDdfbW|DK?^ zs=v+FM|^raVIR4l=*GR0gq(PCclSzK`w_7(PaWO&_a*oCf`m+u&Tgak_fL!rPgp=R z4x9XsW3`=H$M&W*mClYv9ZXi{KZTGgQ1TJD{ISh1B#t9gVV7cKpnwNDz45u!Nurvl zH$|T#tpxOE{Y^&^+vwY-Ys4O^h3Nd{l&yDggiwC@(q;46ko#6fX`7+kC{f#t^+vrO zQ^|rt!cA0nVrf@l*luO`MjdPijMxJhTDlG(PE{(Y_>`>Z1X!ylz82$x<(TH z$uP~VgdEMX8F-5Rj$!3Ac?CXC7)9<&&Um9*<8$m=<0uBAACLAJ#TVjxtY(6pW|rPq zie>vNvZhrE*Nol8$cER|?bRg}8uZG|-V5i8Mc7o~_1PT#92o z3Tnr*+XOK?1igp-k!khup5eK@;#^zxCR40in<3~ybSBrp&@46vC>dd>?g+KHeX^S$(|W3*h_qD4Ac5piEk0y zl40-8SDM*5x%-o;O9oT1ik=^=tRq@CwdGu+a@WE%6+LH$0mHCzE2~vwCOTYUH>Asi zIw83J|K)#v*q?!W6Xxp-d$*I9Hm@b^f%Z=CRXMTSqdD!(c}@%M7D|3IzMX;z9~#Sq zBv~8>B?`>iA5Yq3w}~uMXU=un@vRXF>3j9}g?E=&d^)h@EXbbl@J(x^f+Qjxu{+sW zo)=cL8OAkWhRhOLm=?;tW}S;k@}F?|Zfnxp;B8-nFIeM>()Cf@%*A(v%x?~5E<gqVYJG>*u#mu(I!jYYwb&$vrwFmdpO;wSE`;>=H8x> zQqftkXFQX#(J1hpx{>#Nb9;9GGu{CAT8m$m5+3cdnI?* z_Rwy-|3S;INHN({9iyUiF$eVsWK>!hPWEYk4r1m$JT`oncsnK9_-y2yJRz(>W6<5a zbTqkMu@Zdm+uH0U+z~pgGeL>;)w(+v!jGD}#1~sjY}pns$lLIBemnDyE5+8Ja5fK) zUovRyGTx1eY^x5dM%c_WquWp{*gdnz^C)Uhm)_{G{U8U@h#)!O6~+;YEphZk_wJPj z*X5b-$V!ecZr)aPnD1n#m7Tav^mX6CQY%cRlGwd$ zxTg`Gv(k(pnFbDJ(5d~AAho#fGX`I>^J@m;59$RoJylD@%!06MY4-CjJG(ZD`!fn! zH6w4wU^dN7mu-CgDmUCyqK}D|km`-%r6dz`=glI&$FCaCjNV$3Ldwl=okd-j%6a(m z8OyPYeT?2t5#z!(hu$lr4ArmfpduF4??odIy(n)}dC0 zb>&DJNU;ua;yKg2l1fwd#d)WiHebxaQ#3br=Z_)_wF|ZXsj>H71ew!$<}+?Dv_$xB zPzkIdFsH`(~H|$~r8( ziggJa6ULsqU+FiOncAE+s$UI;@jqI;^?$F6VEIjAAC&lyF!Tq^GN{l8UNP(p{B~c^ zojpld0AXAlPBdT56`__Y1xBA=3wxKJXN%a42d$_0&9G3xb+R9CVc4~Kb0fz41vWj- zXpzwC$enJF41@WY5Wl9%IwG925O29JBXsH_kh?+pfi%1f-!;Y+6EjSABji zJ_2#RvdO^lEjeHt#=4leZf0Ram;8vyr%-x4buO<@48I0a1x|S zu|hTNb7j>jiZ+ngPjG;xZCoL!^P{IjB1Q1f_ye1w_+q>c&TrOy^la7s8ggGk$8$AA zY8KT0jIZx(f8h~w=7Q{8f)8GThK)7nl7wn>ZUgE5ISM5k%Y~Tpffobk8Z&(Ht<>+M z+z$WhOzp894xR=1-#UzQcpOw8JeFHxl2*zL1*kxJc>ZCAM!(+yehFgZk>^w#t$MFx zooI-&prZe=C}bzOd2f4BiJHfsmd9OF|K1NO>~BFb4#e1Iiw|?vqbC0*I{%A!+Z(YaeQ|VRt2lv`>D%ZWOx!>hwxlBfwBiX^zK!J*`P+u=iI|yRRLSw2w1s z`m)NUEr{;sY zAzNmWUjflRS9Q>m^@bD$d2}&6oS^a75{`Cde{jvD?emGJs%T~dnx1E%`vd3TAV8)Q zvQTAo6jD5(8wO1uso)QJ?=d)#JGYNxb1^x=qu_Cq(kh@roql361`p*@gtCK5i~i&L z<&Vx>IIP4k+2c9z3}b!48x_)pclY;(*2m`;iO{ns*m@cr|d= zG|ZPIKkRpSyI&u5lzUZRImXm>7rS)#XZp&5yW&Kic<*q#S-Yl^EFG_+@6FJv^{GHiPE63s#g5yEPM!Y6lgm=2aq%mTO1Ur!tdwqHfPXIGjy>%rpd+ z%^g98iL)W4ZA9icFh%;OEc#Fwaw2&Cq<=g=?S7qSw>xkyrE4$Ec>P2!`$gK9Ppx17 z`GueT7maaeg8<}>5%5PkFu@D3P}YH2F_nPT%4dQm2L4D{c1v7AhQ~eX_PD~ldi(sr zX&!&3uZw;g9|L|xxBD1D@LU5R(*dFxMup8u!{Z-U!x=nRd2*RbfEnW5e$r=!?9?_DJ1;Tv?HHxP33{0qS5F=G zB1_WxF6-Kn!P&cI91rP%lXz;}Ll2dM;D&2sPUR(k=%7*T@XOEMIMtCYJSWDjPFQBM zftL6&yRUm}iX{DRy?z@a)p!WK^+CQZlrEnKTfFDI{XB)Wh5ZQL*W+^6EO!O=8$S|` z7F%swZU1!#fLuM3UfoWE#?_39cLxwc$Kt|Vy^r0r(POB)@jARqmxwPv{1|!pLegma z^IuC-p24tk#jAcRp;oD@Zir+WV--zgF0%zu2;pV9c zb608Dan>xSQCn)`^vcjD>I1pM7t-OG(fbeu=uupk*NiET@;&A}D9@}p54c`u@SyMLBqOC7c$v< zaEIS_^E|+u0zP@Wh?Wbyp7Ma~0iGSbG--UA48NJL{-Dx6p5+Et+ylsrgA%}6aWv5< zVCxHO`KfQ*dpj@kOZ#GnJ9$Fv^7BIAuat=pTht-s__HCgd6NW3@5}_u293FGAHi4v zMb(K!wqFx|_A@Iy#}+Fa+C=C#@uG9SUagAXU8dqOd_wR`h=W~Nw#sFNG%6dyQSx^I+c*}6BiQG*GK*L&)|dG&EO8o4_0W{&I|TVuANR6 z%vR2%JZ0P1UG}M%vrk|W0Y4Xo~0_2AF~H=3S9E2#?h8s{QfjEAiKx@r&W^9ffDj&&TNXV?JJRo=HE!CZv6Fw^OeCzysZSGq~QnsUd#y z8d>p1qgkb~`?aZdrcXZjn&rXg!I6JGh z)`@eRGU572%;jzJt^d5pf1dRx@h4B$=nB|i$IegKb3TpzhUee)Ya~n zwh{jR_q8vZ`S2|B@P+$f?t5sS>H$#gC#y+?EO@_;89s~MTL?usomUFKGk?iR|9m^{M*i9nG+Gs?yhY@(G?L|6w?~w#W^5cysYw)3fp4FjFRU6&7|NcIBfp};iM}38g zu7XSw+9_(+pJm&+ySEIpuj#-8SL7vHx2;S0&c}A)yz)>A{N5PxAlFSwF3|c{O;gmG zDEj%T(vNUyO4<=!4@{GU#FcEQVp8tA-&WkxB|R!x8=KL+e_(5MX4~B-S|}DFb-2w; ztc~wEp^$blcgI{~vu3ZQ@|~|-+v1CN7mBx~7VWLRu4=XmO`H?T&;H{P`KEn~FfJ6f zB8|046I$v^tR@o{*tr{-HPCeSVg@k5-E=z!4iK{gD*MYRM_D6Wyi; zNu{edr+gFo^If>6zyt&T9yO+Kxc~gbgh|qROFVV;I%eTbX_sCq&U8ptYqk2EqqwO1 zy=L~^60!1L2d{ZUAA+pjeLW9Cdnqov&~rCjlRQ$%WlKF<_`TBORuwC%x2YoRW?{V@ zWP8BkCR}Q&zmcxos#>klSoI(zB%yAnw z>~~$=PfySD`{VcD&-JI5eO=#O`|k7kyiX?~L5=nFrh9GX{T|j)o_)P7<}+%HmROrK zM63frWs;0eHL%Nqvu+Q`%55hp@^PFF~V3-R2FM4hx%Yblfd zvamw}b28RA^&FRqYo}QwNVuXbJA#eDOa~Olt<}cmXPJgfj;q=Z(-{^z@CFxYlwB&b zlI^E6#lGYMnTj>BVI4IkRb@}%zAiLtU12vj^xOn)Lap!|=E(nhjY)MUol)z~ITxMH zmM9{Q(?nq_#|7PQc&*WTr_mWbomR%VluXV`eoxmYA`6E5d+E`pxX<0d>%VZxaIy7d zosB@06I}}b*(3k_hU3Ky^>_J)4d$KCxr>s<5H&iMH1!sIKSx8^m)ot(?;vT2utW+Ju=gxkkI1}tubnGOKWQ#X0EpU>!W7Dt@)2L#1VNP zd#!t3po(V;;}Gd6jn?Y!Z10}#%S5tGnkdwhcQSe6v*v0L0g$U$Sg{-n;;NkqgZsX* zOj6;5sB6#$OKj@3bazZ#?7!SxGB_pK*hGXH0S+7ElwQ8F(jO-Z9mRM_2iaPN7n2vq z)M<`(Z*#~j!XsVgQ+FaWGR~iw+EI*eO(5jIzFLiPp9-hDCj@!UETV|jZw<5>lxLk= z(tiyp7~XV%^ct)lUzLQisrSr-WS;W07}_zD)#VV6zyq;Nuj;3SrF5)G8aY1qfk!Oc zs$AsTyd;L!_W|6aYt?y(g(m6ALHsxQSu0u<_w_q`Ys=eMi~BwBoEz1mA++Wo)k5`^ zO$M^XW@9t?X0dmmjIVcDE=Djo_72ra7;;x%s3t${PuEox{#gArLhAODUB(Z9V!R%R zps&&d01oj;Y2U_CAhZh_Hw&h(9@g&LcknM=0Dg!7rYh8L?`I35foL=LwKKG2P26`S z3FMF3%Ob3v8zm_~j=cA!xI*&c*QJkaV{dNge4~7v^>l2kLd1rlG@2UUyQCr8v;h!t zq|xlp7EoT%xy__zzxWk012-RV78c|f2bAD)P5Rz>4i#Vgt){==3=$*D{s>2)xKWy$ zZND-yI?rQlyD%zijt}Ajt&0$YY6VRohm0Hgo;vDc!*!TXvS8;$!EuS$yX| zK=&lW?B?|sx5LomN1wkhIHYUaQFv}~XvgzfC1x5Y z^iTl;wipFP_EOVJ*05ZPU|q0F&_L#H3&f)*ph{o zDl=U%fnZr6EYR?g$^RxHy%?e?;BE@G+YO6V`kOHYHTA>XbwP5-g0)!ztRq=LLfWfy z7Qnl#tbq>G%)VE0KhgY0-_EYRXsBt^>qve<6#j??n#(qBqlVD&0DxA9 zbn6s4=VmW2ga8Axu0v%I0e$j;VAx$f$eMDT8t5;G;;};uXLAnEeiAYk*=US~YBB4i zrTEiBFYCmi4PDv*OD(eN?rGLIst`0o%wxOY=JM8oo)S79PvlPDS^7Gm?m(KWmu&-s zsH4H|c?ozNVzj7$2b__}&^?Pd%q-kR7vOJ0RZdqeeriwB@;!5RIKJvcF?0O&?y}TU zrTc)Pl+|G{Pi(L3yAP+A4~3)&g^Z59a4(RAp8EL2VjFaH4J3#WaooNgInDiU>h`@H5LSf zKnZMX#VY5m%?tJkU_Mh{vD=t~0>cg65G#Q|WvhHaP_uRZvhTM*yUBh7AJXB(X5o3D zO|1Boh`16`HMnEjHkk;i*#$O)Bl{8H?x)ydh5g^!0BCC7StIx2Y z$C2aqrYg%8;mBUh@>c`WTA$Ds*w*&KFaV^7SfnLOj~|0@+SPInqci??e2_ntniP#h zAQYbmPZf_f@6%z9UxEd-H|D3y7I}Yp?|0A)$x+KO45~Fn+D=V%Zj93W+J$h-C(@;< z?@jr4Z8tnmZ-lCB>4X<1Q`lKM9!yc8n)V{)0cP8Ddp923+R(uc3o6U4R|WhUfs}yc zlO9M4ScIH;A-ca-XSmPv2@5C8Xneb+2cx~_TQbGNb2K%M04AtF+XSu)!kEA9 zUsU#99I1J*)e^ovi2Bc4w6!A8<~NmvWD5jY*#ZXEhTJo<;2OA;>#9^4AD^o8y7VYD z0e*2uZlUTzT`iK)o(h?k(8UCht~R*IBQcsvX1o-@-Js*H$o=W$poW43INnm8)-XM$ zjj>0~zaW0N?ZJI>%~kdr&u#U^U*^eUU8(z0VZKgy#}up$9nSom5~$K(VR#^jrC-kzpvpjFl@_#;#sN}Nzxs^a>RuaDC4R;ENv$Jo~>3vDL`Asl5Ke@LarjS zl}eKZhwNtEZoV-5KQ}-i}ZJ->rS2-c5}b6K$$7L zA7V40ZAdBu3StpH-NKWl-Wt53)nQ6OWN6->cJad5jFlY0RI}lb4(T5BHG20~8YwNI ztB0mNABo9YvK3>`_!70)+^2yh4JEi#R7w)F`WQ0^@y=xn5xIWbDk+HHZiiU-q7B$6 zRexkib5o1^eHp&mg>_Z)&C+rBRD$Mgb~?PAp60*Y;(vcW3C+sD0i|IT^h3@l9d~K@ z-~T0tG?~h9e^uzKUd=sVf)`3A30w8-G(MggyQ@;uvu_XnwCH)M5$V*W8PEHzuWh5g zNHk2Fne{K4eGuPXjlv}m)JP~#YPL74-5NANl_fC6wP;tEv_XyI z4J^xZt?o{;8!gUrC9>~+)mV*o zdvdio1T&X}60}SpXX8ky&97rlV6#}FZ)|Z_b#(XX6=RvPK;&dY z2K6qXU3bNTRN3w5B5)3DDZQGgfuUm@2b0+bq?I{{)XMt3VP0t=xiLotlg0?}X!d`)zfJcPlXT`O)D%1zXH(%MPJkims`&R*uWOMc70z zuze@ggm^I(8C$NM=FlvPyYX^nO|@835G}*46>TYxdBeab_q7xXPDL>)*^DLWRVUVX z6Edm=fpS05!U*nZS?W(v{D8YDcK%uYg@0D7P5M+Dv@_XBG(`axlWuUC-BipiJCfLz zVm2H|u(iV}U@XN3q|+`MOTin`yi=*(1#bz~JqC%8w4hCTw2jq_^g!B5cP5@&b}Glc zE*pA7hdf#7%;|8EH5pH?e~HV;OR7#hjVu~67E>mKvxe}T`PJ=PGfa)DAJ6b# z5&^BiHq)qJfT^rmW&9((8=xYw9t$`L*MlRrdeAP%AvtARX-Cb5Ax^+^{}k|MV(#Fv3w*3B!Yv5D*x z`SrvceDKUo(w*;it5`bzUe1GzZ&&BRdF;|-Sg-@S6)_p3{L(?bh62njSBNiotP~oT z^3BuR3gmp-1p`zWo9}L6b3oO75=$@pp82*0xHId>id6y5OZ%nFUVSaDgJJucZwCjm zH2JjGVfDMz^Utq#1rAsK&pW2)6BzNqX|GFEJyIV>Cvo=J!q3@TM>F-O*?k+&;>pKJ_R2ka(*ZhYuUe&4xCnCaE{W zk-i~vTw?GC=D4rNR;-a(t0jba9&tnWU$rATVy=Nyf3yuw zWDQiv6l=`oRv%ow#XCzW@s9X$g4UYGY~M3~Kx44zg3{4LqIqD*PN>Z2Zg`7sivV~K zWyVDpM?!$^L!<)(C~`!?_Q4p0gMGt3aooZ5RAnUGNT%Wn za4t4iw?SmE3|pV`+k$L2$Rr%=)>8zQo~wfRhO(rPl~%fKS#SVTB}noi5M_GT;V|tmSYSbH8E$05B%W=85>DIP6&jc?@rDbfDsl-hHqbKlIDx zFA5HQ+B?NB(1lO02cn06uR!`7d9t8VF{sKLmo=Ta9*#xJ-C6TK`7vv^a%I;`Gv8OJ z{CML%e`OK|4pa^&^VoIomy`)*Mao8V>?RURC-!mM1ah2m!%WpA4S=j3`FhYvzc_E#L&yCa~%7A{U-4s%nHZz1sej zA)av5Z!m7}%P^;#(R@AfWRjRm1L#$6NC%#LImf|Q4t7S|{jV-VKn>EAuxbdP-Hwy( zHRqvY2Vd!v;tjkOY8ODUTxE+K6IR@ECk7CCf8@0oOCzfB#8((Mf{5Mn$}YW-_AqVg z5nX0)BN5h=AOI8>L7(s`X|rLQpZ39|ox1M;wM4wn9H?*V)n$)AdRnSM7H{y%1d{0m zMds^89k?0fArLV7WB=T55)YpMpq1aroWR7rrUOTz-?@Moc*RK{0f@~bo*b}K9p!G@ zBMh&q?B0Ik!RHe>`#(JZJiG`uuTuV~cU7bw_WdmE<>i(5q6<6IM4Hs`3mz6T#jJY5 zGuGJ71-4R{`PIkUu1FUzg*jn-uZnEM5B~Ct!=sBBtGcXYTb`~uTZ3_@W2Q_`3nQY0@&bM+*;nbL81q>T_2HD1vC*Vf3IzEuw zr%{FO#sVcbBO@ikBa=~zk9WX?())1*yECpHQP;pO5tAD{C5`_Bb1 zg&~t~zd)2mRXh)zB=er~QpCbqja8wiSj4qulfAvuo!Y_2N40zt=|aD8uSg{hjx@kh z6KLA41`-c%Ej9~jnm8Us9lV8a5Ka#=Jw?t6MeG!R)s&n8bmy=Guj)xu za@I%K5&c?KScOu&hHwEnsmqAF_KPTWPt)}AoR)a)#&YyQ>v{5ooM_e2>s6I9VZ?*h zzCO|QyTq-}co31x%J)r^nF9*GYiT71jqx%M)!T%W-}E9rUZUmyqCFVKcUljVgh9`I zgzeV*Q?9P(etK)N@$%crolms}8O@9kblM)LL>O9sjlAn2KhqANG82T1z{CdGt-%6& zk=1KaL&b?Ndj941+GQEtyd9GCaIR)4)>}HK4}Z`h?&uKlT&VysrJ<}p`~*XuDTl6v z%N0#X^DlF0+Rt!x|5Yg;+OKzoEbe=0Ng$YmEIsSG7jB{d=4cVo%~$G|obVV#%cH^3 z*>Kt5JBPF=Q&du>oW|ha551Jm&=oaf#+Gw@3xwW{SR)Y?Xo1@ zm}piPwy~znRgjnp%uQ05*zIPZAtvwnxqI~8NzQ9oME7|ba`hD?QQY5;H#vA@*4WsN z*M4)xroykZ;+>^kDq<4NJEH_qrmc6nQvYZU3T_)|_8`vP-5?H--5aI*pD>bxczR|s zrbX=x_eb&AFi&qD!tQqG5m79Fw|1WXiMJkbcz9FIbeAGxr!i9DTx-JAVnXm4Cc!pQ zOtP|5Mk}%g+DP z%@{~5n%vbi`F-Qdtw~p_leyy;p<@YE9R(_WRDIT68H%V;aFLWQ^K&3COLvSYReH7H z`Ho?Qgoanm5*#CU$JHGd`k6-M?bLx8Q@5 z7WO59GYQ8_zQ)ZrZM_pq)?)P(P0qa>qrCk0mkv;c)JDK-b<&66~{iBD8TMJ50f$=o&oB(o7Tu?V(G|o+$pB>?iG# zK`-v_B~q1N-u5O2nx}oJnV#$&Vp+I3wY+n|jXhh-`l4lv1^9x4TGZ*ncV5o_*NHK* zPggtis>_o|Y8YBHL3&J=(E=Z!D^JUr%1~B%$?F_*k{?iD=6PRD^jW#`w<_%pYO~4J zHfyIUw(U83{DBwLy_*_AqNGC8s_&U+d**LUd(-pOGWsotIW1cSW`Oq(CZiZpe%E=p z@#j$mRNN26tf8FWm*Vdoy_tjb`^|}PU|Egd8;|6@H9H}91-b%2W#2B@@Xp^py>h+c zIX%g6_CyrMBV8es)omDM8-jb@#D?Ea$BatD-g?>w2S^rP#jUfvqzFKenx!N z(|(dk5=M3#|6BA$oT=5R+!8tc=)$oJKBD`LpZMmFOD3Zn(@(Y}ox6{LmVVJxTxSto zF;xj$oNI_l+uPpr?J3qKJsOb!eYHCaf|E?sd0~|cRrRAQCgxT&sD|7x2v1>O3t4IL z4-T+=P%f2Y_rLAM3DgMKfvk=weYGb(s?)7`+v&AG$TRUT1=T6K{%hV5l2${5swt^^ zW+n+|VPK5gq!-3&y{%SNo;`Ef*mUKEJc?bK?f%(%ijB(iX0;pYx*uytFz4|u1a@g0 zqP4%Lv*04Byw5J}hPH&EENaT7Z@RL$puYEWH@d%idgwnTZ|HVYpY_Dlw8V?MeG^v{ zl-CUO6i(8Ma|Bo4lthdxx*vQiYuD6XQZ=1Si7A#m&u_tC@0f@t&3;BVw2;$h8H1FV z&PW6bS9Y@dWVeep^Vl~;#-?KJ)KW7g#l@D!oz%uCiL|*OCnb|PE#aBm5=v*z*qTsP zLuBdv&DGq0iVpjndZ6CSvC%h5e5jz`1ea25Pk7=)M4^Bxd~9%To!#2|wAo0?AJ=#( z-?>_0+K~u<`|ByJmC_`h(^SRQ*PQ{tS?&}*en%RA!kT=wu!WeqPYO0RSKt2`5s4lj zuMZx=;_`r7$bH;$glE}I_GgoG<_`r0cQE>&;Y$DOZ1RKq03&$hSM%SF{+_r`IO4|W zn0As$W?Kus^%>Qqb}`w*(eC+xmhrp+7qzT+{Ilx^}xzjK%WOE1{k%oE`pHPKo#` zC3bB_Vx6r-60LgTg^y=m{udo(JP2HDj-c|ehNQko=11I1p@#9lvia9!8uHQ%_bdP- z3{+v;2juE*;{%>-J!z&#pL%h&&Mj_B{?Eem+@nxX*}d`Sg^X7=04Ng;X5eV|JTt5W zV5-+BU?}{Z_GuI_eljj-0$#YLil2HJ*)V1gSZoc}U(@(b@*hmmpbPA$cd^G6d+PD`SKf`HcbGR(-Woe6YPI+@iaBD{}; z`HdBZ-Rk{=iLo&CH_=WqNbYm#yhvh|#as0$DJgzb>rk7N**uV0G$!mc?EZDgN6zvj zC6835v$ZyMl24FLH1ZzL66%@BXlDWtBZM=(G|L1z8y7r3E5*nM@$k+eXr$J%N?a3+4QsREc*P!F&Yt3cna!{0;4*8uYX>&4Nhzv`fZ%RTx^G^aC@ zZnKEv`juS?`=E|s?pgh>hSKmYwTvj@qsuz@hQ#JO`!xc52MFl>sVP9KE%sETU} zrCSP}n~5ipJC4xFO1*Cuw7`b?*``x^&jZ-+{L3~!i9lw1cL4cEb`7p?0FoOgGm*{C z%W?TJw)kiMdw%ckD?a?FH_*F3KUo0b=9a|qk1Bs@9Qy0QQ!~Q1*EteD{;HN+rzh+M zUO&di;1oG+?+(SvUF_^%?(3>}*O)bb;m_m#J@6%jg^CV>*N?UtWN4i`1gyGm>e{X; znTaG6f9k~LAFo!ldFGEWEAaYx8}+r>pMXWTSLbf~V%9vc(CNEM!P@=v`=ZzOf9BN63s8IJ$?su+L(I{{ulHVoWm zkKchs54WWOa`1Omn@E)h3m;bh{rm6xZ{lO(^FMs2|KHm$wtEX2|7L&L^{i{yZNQK5 MCDV%q7wp6S50M40!vFvP literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner2.png b/docs/images/LogMiner/LogMiner2.png new file mode 100644 index 0000000000000000000000000000000000000000..253f4bcc3e7c9e04cd1144981d2074f0c79418aa GIT binary patch literal 29283 zcmXt9c{o&k*sfQRy^?*a6w1DoeN>WUnMz3Z$mq2V27^JeC5$rJnIVL%*<%=EU$XDJ znIStfV=%_x^IqTg{d2DK*K@A(ocp<-`+3g&i+*mb$Ic?aa_-zYcKxT1U!FVnkHFcz z+a<=c>&p2q-*e}#pVNQ*@D=p@=G+0IRNu6jc zR{WIN>BTGg(1dHRo?qZ*yZ+?WElEl7KWddn5^;O}atps#_M3M8%$EB(?E|zm$L8{Z z>kt~`V3*>(+>#bO-BuwBaVFZ0W|)g4lZuP ztM&@n^G2=zyWH$Ler|qi&UaZhh=*3W_{B59cxj9=qSi91a+O&^3-lUhd-5JOWR_%s z!5Zf*kjvm(B_cHeleFe>rv5@sD<9b(Lr#~<)iKnQ?NY{a|>8csct1O@so`u-eai$tT z`CcgNB$IKP(3?4zx}eBL=2GSzVczxgimXx-;hEPuCDK|D#DO%Gje7!gZ>Ki% z;|U0)R--th+RuwMAI+RjimvP+SXvByO`YWqF|gG}*3u@|9X1#&MO6;~U$DG9`J>89 z?fDD|xUzFYk(RGdbUE_3M=UM{Cu}Er<3A5j^eLcj14b(tD4=64xZxe@$HF(;(-e8i zuuI7KhOg>xJUzahRC&1`rYp?7iyXaWs1kL&nG?9vcu+4V zDMI?BClqwx7jiJvYkNY@5v7KTs?sLok0%|%;!hfyHX7hf^j-{XC)e-$)fJfg%!}tf zq|to9-m2&+NmLXPURB0sOK-P@QB*>9RrEx66+|nGio}Q3zHH@aNEJu>x99Kb1uyA| z9M52mXNE8*8yJz}Bh2a15CTcbo$Eg@?4C?F-EN{Bzy=OBDOs%x$8!r?O|#jpZ8FCr z+Q%b$m|YD_Wv@mIyjmYk!E#bRaMr>7{4MMF8%j7`gX}gKcq=%qOxN!K{de9F~@eK|#r zCObufT2)}JrE^VZ?{%Uc1e$W!-PW(lh=`{Z#H-`M$Mk_ydMW4frO}hM8^|0NV2d=`m5nf)br&A^tHRYG#C+AG=JOqWc1 z&XU8$s3X+~SmXkA5KQvj0<|;l(ffDk{W+(Hs8h87%t0W=p|zI$6o4e9tG6koD#e=P zK3mpK-4Zndz%1oSNR1vJsemtDjs>j zTg&2;aeeNVqh4!H3szMCLovWmvbv`p(}NZcf*Qt(38}=PHr55C0N5xj-EfV}c}nEe z2(mevo)8J@UxD@iGE}vtM%dQPAWx^uj;C`b(0}hWbnnQW;N{?eT_x>ZYU9^3+Rrle z3U7zEu%nrUrgZdR!kXlV98%-LQKMc6EkC4wX6|qbbvz|pqR-wBI!pPEBYl47c>ZhW zPmxBlsu%5e;q*91Pn$NRF}qj>ZI^=k!I}XF-r9bh1k_m)<4?$Pr(S+m5q<%e2FPEI z&!CNIsAm=xt)G7HzgJ`%M5zrqs%@yD{YD-A7HcpeuKqjmH?a)1T)H*CUsEf5yc&P9 zIwVTRif*nB-$q6~f|7f!Yof?-yQ7rT@c(k~lcZ?Qe9 zww)0cY-ZSP`Y`qbf_PPAuE!>Bq9-e^g-GyYTimY@MRvzFL12dC^EdpbN07dh{{XA! z`}X8y=k%m&DyVLr3-U7(fsX>K8 zfr26`-~O}Qd@D>-LsZ;|BZ*lnQ+u-&;3{$IKM*+2fni?ojAiB(kAietzzM2_76pd%$&3$34&ayZlfJ z6tHXpc&Fm2fL}CAyvLTp+`Hubv7_@y&gO`mzee#H*XVg$?%>bEayH$+a_k@7_YCvAJIXBaP+fSiNCiD= zN^t}mtG?>)KvWJJb}lB zdqI@5pK&rOcr|LfCtFrEb{VsDTW)G?&mVQ*-^pR;f7G)6yvx6>K{$>)$h|^5?ncqO zONGU?y&6Y*^uq*C*xJ=D@BJc=dXnbqn@`o)(0?rttP^A> z;J_80xGZ-{mfI2km#bh*F*4)he|Vd3HwqUi^5-HSp$CpO7sctnW#F~Uc0Etk`k6Z~ zYRzLK&T4We$huztG4%}dYYp8g?rKuLvfB+iO>?=%Wu02vo((zS9*0RDYDsh9JG#13 zHgdNab;A6-n$D^zBtMy1vk_*|^6(=okL+)12|Uu8J0NH(IcSPwA>6$|`;Ep^ia7@b zoHrJA;Fy4*>)g=VWXtw`_05K8R6p$~f`i6?RI<}+myNB}A5C~kRyetYR*SOf}-o9k=8=YgHpJQFV2ubtwA_P5MU~l5@{lC8D zm&<9zJ^VOpnE(CG%M>~~|GJwOajaYU?|eKs_haM>l^1ET62|m@{|wNI2h;;_thRL; zVJhAJ$=K2hX@#v#`j492{1k^Y|0={!xn-x%P6u7j?+!6eGJ?j#mc;x5PDX*pqhaDN z)uBSsudM?^&S!Q$3up?z6rV#5m(a9VQJ84~f~XmC8jrQlxUw=Zs?D8If`jADP7Snk zQ3*g$f^UjsnX3$wi^?j)Z5a39kvTW2mPYzG!n)JFBjah_OU+-O`=MxssCsWmm8Vw| zNf6_yM<@`Re1F3qyg9wtXI~=gWU=UZladppsl7D<*jl|P2gPpJ87!1bU9q88X-O#5 zf%gU1?s((fNE&aSm%5~v^w+8{{8vDw%KGyCvHnGLAiWJZYt%LW;>*EL^TvOZPqH{^ zS=MQu2UiJYr?fKBd?OBV0`yEcI3$NwzP;%+@lVs%KY!qp&$x@3#YDt!8b>60a}E3# zWz%f=KNe_I0y`qw_0x8Y+E{JLjfS6hM(=4A(~M??&wp8-JGY}`2q0qqx->bhd@J5z z>GB4J^E#{93MDtKeg$3KJ9+8{z18r=y&cG^2K0S`-nb$g0FUyT{PhXtz{Fwim!&$d zaF}Cq|J^8D=&LjSw;Y?E@Oy@>*3eLm-CSH%&I=916jumjJr+T^B1*b4vAm%(*ZRWp z&mc#FLt)Rg0h!s@x&IT--Wv?&zB{j>Bd7lFxyW#s)a(RgOWx( zzp9(@$i4G|Y$B=ZI=|JTe7}CLtZe;7gopDgjO8l_adgizbq-8O_vB_;i*#6o_TJfZ zhYJh(7>nAn6t-^9K98q;Qm2e@M74M(W@baU1N=UXI32W=ru$iSUFtlNggo!JJEF|l ztvG?*3vAuCs;WoEnM4?aJ}BJ(m3l5f??dZi@oYZa$oCPKyilao>g($cvdU`d5G}m8 zG|n5jo8i!TgYKi)_e&CmcHD5rHWHn`ueXo!Hm>S6bkAOSqT0yz?{KFWG~4gN88rqw z=cju?Vcmn<9|;%b5!V9{HL#*hh(AGa0cvGdxBll=-wS8EFB1)X;`7c@QhJ73-iw0) zvjz#su@`iX2%_-slyFiO(9Z(w+f|?i6VUWaKwqP!V-C|BXs^nA8LH9=89HP$DDC4B z`WqrEh-ez(o79z`+>{gY2|-9<)sk>-S?f+M=6nT;4;anyg$)Iwfm|GiGy#4{NG$q(8;^o@iv1iI%h6PQZsl+qYmAU&uPnifYzM-kdCw{x= ziIfSr4rfx=uSwv#?};-EfU?u{McLdtZ({YF|L$Ws%qwA`M9 zZh=U%!BKUA@6H!|%jInWO+n zu}>E60(S38ygn`y6G!9CKXtlp@OQSBR8vqpo45Q}IR*c$n+ux(?J7&g>*K}g>CEOQ z(0PugB3bCT`Hw$6Qe}0XZ(z&2&+|suPx*oKqx_r2KG!l8--U2=KON^05s|Q*n{}Kk z&fLj8FFtU6u-hmWkeH!&QQ)DC=8E0rct2dz`Frn)UpXRW=eyIg>tN9C6wkwat?oA@ ze63taON&k!Oi9?r=aR%E0Bsr-w6;oION9|I!)EbKuRPhD-qr{%I`{nIr$fuCj)uGM zJhnvp!flByj`~cjDi+l)bxN2zYoP`eU!BEvDG^KR;)(VKnpOzNi!o1**h4wH>peB2 zlZqB*Ipz@_^zn*B*$`19|5_Q2+p%c>Tv5d)DpVu z@V2NyX>9IZ7_4A-Oil0;E>*fGJ<&S2Je5bK?z>A?-*Zg|Ggbc!t)kn17-m$WjWYOA zUJ{!8Yij=62>tC`?Q6jK&)PUg3DNoYoYUMPHDiC$1KN3mmW$9sQQne|8_3^~#5>vm za!i>$-ANK~m=tyYgv%gFnKLvv9(7;NYHo1 z_9DfS<<}!P?NNK5`teZZsTPY!!IE?;!8B8I^xZTS<=(K9rv7uJ@BFwOD%0B+npo*e zA?K9*L$Fmx4@=2Whl=YEeyXeKwAbQQ3@jHfiXM z={>bOafAX$35W_)3Wyvnn@SFg2L8^&a%hp@Q4rERta}S zap^e;*E3DidW@+blGU1SZIp8Fx|if4$~$?8%Q^x&&9o8B9%%;s+%!&Knv>-Rh#e$& z+xiSuRv7c+MKG+yJEtYzR-QO_LqMFVkP_{%21M7&{;^nLJ@#yi=d}oB>K$Ng7YqSq zYsDQTh0yAz*H<%ratXh`PfLW(h2q*DAmXrP-GQ1-31)eA)bG~wsD_M5HMrp-kuQyM9ltF=9) zA#FMGHlQ^@=UMqp8}5U#$4dUU^lTKJUR#}P77g$I3-c#kjdzc9#VMf<|2SwZ<@E{x zQH~uFTH(8?6=6A^sn$N*rF`pp9-R5BA=LmW>(L zqWGETqUwTrt$8opYa#7*2EVM7-*FG#nQA_={Y?h&%ZJ~*j*Hw>UHFWM>v@zvayFOp zp>Vx`o^(<-rx65;>tKi#iI80%=G}B|POlo~y1$EWjHtDLE{hn^vCg0V7CUX?EaY`} zbNV-lJ2ofV1LA2NpqspV9zehevrO?J8uSylz-5M-L)YXk%uMhuPb5oZ&b%enxP0LN z6DBww*%K-P9&$<3xRU^j)D9_rC2c|FLvPu;m<6Rp!VjmdEO+UD#p9RSYy$w5Cu#x% zAt$>BfAz{>Bw!g?d_g8SNXui-^DnKgqJjSTMXL71`dyHU)=xv@4=z<|*JDF-7%T%! zt7OI|FeiHrIh-A9Mjinp+v;TZp3(Q#pzWWohw!I#_VKtRAJ-6rdj+t!YiXX);8~}c za^syyBY$7(SwM6rx|d;|AbLujlSwT#J*e9e6VpkGDYBsrCvO8*AiA|P^4J9XFHYqG zutab6v9PhpaDqenc;Y;Of3GgRC2^IUKtRcq-UnTp1+V*#|Mq#)*o^Y5u<9!jnf*|R zoMrhJr0ZX@n1E~Is!)qt8Z30d;8Gq79cNW)5Nz-9*Yqse-aYyH(WY^O$!$5ZG6Ez5 zp4dx_dG6t|Ff2F8kW5Zpr2{`9h0X;Ti6%r&7?pQ?ZE`NcD7M#9vIn=ymzFLYHav>d zv-SH%IDJT~P{>0eZrFy5o?_Pu1{%(rH+gnA{dy7HQ^2{gyB39Xca65a!WYG(p~l-G zkRnBH)!kqUcSW-PZI&6PAHHei2X21M$#+)eo8l%9TRZEu_UmG@>OPgyi%c{{FGlX zSrv>0i8p5JViS249^m*O2;FEEQS<`aW6w%QFA)&3@57xRz6 zswz%Lo&WnMT1rXcgY>CdXsg1v{YqtjPEBHSKObH%B+}AaGAzK&bQvY)sNlLTCDCpn z7tAI{qfEE-zK?YhCH-mGL@; zamVk_AGc|_<`^p|+QHVaNM#h}-;6qW&(`JON7w0k2O&>I`=JTK8X6CQH5WxsJz5rW z$|~O-lTw1xXqH4(<4BicRvAT2e20Ebm*}UsQyvqtgdAy87tAW_8?W?^?QWQ2W$mz< zz|> zQkY*{lTU?#?ax3n(b@Ey2sFF)1Nm0V%gOS8okG12*MwDVr5jhb-rye2(%Bw<~RGsLS^s)2j7s2b*}fc!b0s>`1@P z+yIKJ2DQdj&^`xWFY)MFnN%?bZg4rKHp_@FR;W0xe9>DNVBQNvukbBt&W4;GH=syE ztzmEgG1db?LZIkN!d7az$KS5lP&-nhwCq_0)1i}b7k5M-1czpA`Wg_+yA+$Igngr} ze>Jp@g9p2^{3`AySFH@i2UALsYT8?G!Iz{LiCuSIs;nowL(4%FMRAh798R*KYR)>K zhV>mOFK+eDsq#@W(h~Z+bN_cO>fvZ_mQnvVMLwU!;5hg1E9Z}N3iORYW1$-Nofi)j zB}MGiy~A46a>qXF&n;p*vz(Vn%qSvr1J>!@l@d2oRYn1%_=9;P!ZVXXfw%CGKW?6f`PUM3xD_e2i!Xd8;$6ufdE|$op*<8&wUQu zd@$I0xcSXGm^^k{EtWs?xG*v72VhhEy}^^IQ)ks6$LY+cobr3^H%KVn!;I1%FN~;<{1)LbqdZrs5e7e3s0<- z8y#@6CF?vs9!Lf`W+{kJv}$xT=O4j8lR$>Gd=I%w2ERs={}2Eg*XQr0LP@u^C2rZw zi~Xt`X<6?K-t6?<@5Nu>+%frrdk=G5*fduDQ5oULEzvr}tgeyiYd=I|=`6hxB%M3S zP2v$-6;eTTyAS3NlAtqN*xKZWfjeTIMo6akp#AvRt*Q0Jq@xcU0~#JPrP%SFy7Mc* zQnruLdig7%GehNb$OA)uD5E}u>cttJ>ddaA8WxooXM`+1_rUQIgy#7D9avG5u|ekH zN;zXh_h?(!C5soi4Rd@u>96?*Q<03ff`x9(6fjnK%~Vk2znd9=ha0g1(kH`OY*L^i z&E-XTweRIDZ2eO&;EZT7cFQ`F+5G?o{Ln2ORrca4MSLf==bXQiwS}s%uP<%JeJ3)j zvgZ*fZ;6#_tN*fi?@yp$mpLcgt7?cKa!@xX%N*8a50N!oS*;9m+CwEhe18l^`*| zQ2>xa7F&okaFI1^zJO|*kDijE%eVnn9RB5xG*pEI_Ko?~0A$V0n~hsHD{gm0{c=WA zj9=BM36r=h@_U?_S`_3!VTi@9HfYVDV*eQT+})x~eMsYj^@6s47=g>TEQ4c2+zLvk zvA{oBx?cN~OzyShRmv~$!?)wbZu&_%b{$-eKwu>DXW;gzL9<;&b&YAy# z%_e|LRgB?p-rYnyd@@q+%74EOeCeBxeAgVkZsKE@KrN+D!{mhX*sYXnfQd0-->qsc z6CV18ValIa_jKc8D!@m6RP-PZXm?(6eTsA5`A&-pNi<(nvM!h(tQZKWVS zqvgd&t7gu zU>|;9<&m+*V;~8UoQabh_AZiN}R8RDvOz-dX*gi%hBG)n)aZ5-|GRS zLWeMQ=`i~}hs(R$YPX<#x6DYfpT<1(q#0kJDyMzQzdzi`)8WdMu`J?bXueKq zAh|Qv1%O_UC6lLh@w1cc70cF##Scn`ixWnfGPQ(6y}BO)wRZx*nnss03f%MvIQtLd z-MXj|6>sVu_i^^GzpADS^5c8layAvrn-RZUl0Ahrl&oieqB6_~d;Z5?N`_F~j8P!8 z9xG3i*noPd4S_siZtqg9p7+`BItL!3q-zqi@S|TqusqJr z1yxc%S7)TP!r#p|J*;Jy2OMKsL98-Qb6vR9B?D2l>K*t$p1zYZz7b)=f!BRKR`X@5 zqIJKJA+wKo?_-U>=`NH^W7eHmMMY{BZ@6|At%NbwME5^#1FMdAN!?d1PQ$V@3NJjH zA1{;>5x7P)95-uMSJLEQ2Cy{C@4=%q8GCb`A9Q)R3PMf*NY!mrc-XVh9yUjTz7QLS4e>z;bVDq^u+-}3VXPM$3dZA?TIVT z+ZX%`X=WTV6_5Hq(7_27_4QTcx0~P6cONshS_ykq48CSoD}Bie8FT{Ud!H5o{WCNp z^@H{mB{hv%qY6|Gx~qS#7u>NqYY?$U4z)EdVd<< zm!i6a!kw0QxK%?Sb*+}G(>>#4nj=y(2vH2V@;4j-`ZylHwy3~gd2wk+LtWuSLmk`u zj{$#$$D?}Y?tD2L2v}Nu=V~Duu0K!ztFp7?T^9@T>3Cuo)J~9qV@S_D8WhPtIXknd zmv9wQ@F&E4IgR8kL|Jg-piKPMFG;Rile?%=9WTyMFg1p&vMsAVin7~EP*iG5T2n9)_ zCT?c_Bb%Es=Jlv+#>jS7H5>RHd~zu@LPv#5Bg}5XuID3{h{%ws@mi0W4FzGGsWHHQ z|NWvjfOhpZg9r*TE^e0X9l_JjV`JlxBR3}6=HlU|ezXui(6mY&aCg+OQsb%wHlvi4 z$^aIsyWj6v@&4mHY%nW)1#%_ciEM43$GUMe#$?@Nd04zg`OC%!+{E;|i09Ux1&fP%!hYDSS*1Q>J-{mVkZV`G#<819dqp&Z_imPGG_#b5anQqh zz-zRoJ^Ba7oy~K#pY~hMj@skwLBz-rji|`&SMHBpAro^t_X&UO{e8Q;#j#_+(9=8&-#jaKK64*i3OJ zd1xz4kI7@)wyqnYO&A^;Hl%a6Z;ax7C; z65UCCfoplPti##XGOW+@)yzbEE^hRsrCTpxf0^4>-G<~$ zzau#aosRuap>tmo5_4^a+yhQ zOc|JL@h*zD@pAEbvw*(%6`{x2#m~v}5)Nc3k>IsL9eLbm^LdtJWJRys<7E5jPJu{E zNo^$7_!;m7M^GSC6Yd5Y*lrvo7s~$Y?Pj$gnL*HP7zlpOFEMwv8{r#xL}~BVkDtPP zH%bDxI0;$P!<9TbBu=S}I_6eLYOawUwELrpW*T+dU;s@Gy$$N&v=LEj&nGAooPPjt zz~3ZxMhUdO<;mHM^(*McbO}6B@ksr3`6K1_p+4##jW!J^Bmz64t7?xkqW+2JkXB8@ zdlMOT7(k*s)Hy>wpcg*#51YXn(Uhf7Nn^;IT5+d>ClBNw7VdwN)-wHhA^B%wxeIf^vN~sG~RD zgVI7xG7rdGh!R9O6H}l{+8HfD{zLh_7T}h-mR{|SfogU^L!alo^;CF}&4dOwo2aYr z=vVNEq`ae@dg}3>{hX$n*V4a z=5I2`__jE~Y=f7rcf~X7V9D1RB+BbUee)CI92OVg zWZMT_$W8A59yh^bvw)I&6f_zc0U1t47jGgj#s?~(wJ$JwxS7izC{w?2HJ&IOn)d5! z_+N|vaXGWM8FqIEo(Aodt54qwF+p_z(LpbF5I!nPj0SS{BytTAzqeIlB>O7e%A2pg zf823sD_x+7+b683Jkqt7BLji|CpEhD`Zpe@>*L_eW&t{}#mI@C3krrdfp((6Lv6q` z>4o{D$l3Uu77L$dkDj|op*19xB`kt>Vd+D%iv8F-&2U_aW)RDUB@|U?Btm>9Q7iaz z?#9&}Dof6lBxE_Yt+qa#V8HKp1Cq}SQbNn1pNAbk++blk=r0N z%dzRsa=6EIL*4HspEp(l-b>5fFmDG56oXoVg7s@y4=Z$9C;)@g^g7zs642=oAE$j_ z#%C!MM+Uvt%seBwBR8L4W9>))*zKlL3cD<4&iUWx$Ie|)dafm!Y)yTwv>#79D&tRY5g@hp1;}pTzhFOQKNYl?km*@?ET3r z-dL++=iyLYjU$8dmQ)E5p)x#n&!{0zw%_Z@{4S4YF6Ublvozjl*bLP@ z-Z7T#`Gi#iZY1&dy#nSTnKZc6X4>SxKCE_!?qxcyDW2D6Tl7S=U#|y%YhCIS6WA(< zI6fbtDSMf{On`cQsn#c zXwJ;hs+u3ypwe4PrOc)=JF33v%$7H2J)a0DmG5_I_eSYkGa08tq!e! z)x(}kQ}OR`Jhi%uGOV?JL+3Ma{Qad&5jlRkiuIK5m}IZzs`O0c7H#+~7joF^r(Tbo z?L%e!b?1!;11g@)V3eJb64}R=)}}2>cHb*l?Yz9&x9DO&_50SRj(FK!IdnSO`;vl3 zcc&+sd_l2vhPI_`TcHv@dRW{20{ww|9ZL#Ar4=0`oYNV5N7-M;uC=x*+?cqxEdG3E zH-mR4ftvroJbFdCnLw%YwM3qte>%f5j_b0|<$KO0$uGV=Hc89A#?V=CN5f$%#CjYa zBf+Pm+}rc;ZN_m(XtrsMyRI&Q^D~=h*k9Deo<4sQ5OoNF$bs{{DKdb6b(vh2(iR@O zBN11+{`p0pq~cQlB)@S5-Qxc7Q3T{PEvw;lesL(|wyG3cWypXelNx!9lfN5lqBRu7 zbyI6M;dcb@+dGVGz*at(fns9}xM!<6oTJ4vU_AB=uPDtQ$fC4K*R~v3_ z8fa677gQUBymmhF7(QcKD^?BGINkKuy`{RP?g)y$q`!PweO#BW!-o`A&xl z<*BYtiPqKv`Riu4#pMo#_%+pT562o}j8|^Gw$I8p2r&ttaCo%g5guHT#6ccmmxif3 zN{5R>>eH-nT~Qtp1DBT{x0?gd&`+Hqd#zFXEJ$1+8$hjyBd zwl!oK=7o(MUmv5HXB!085ei0f;JxGjOg0il*4$uKA;G;F*-yM1U6lEY8N|B{>MP(v z6q#iO&iGmxKjW>iVOR6@ZZir!Sq%_i(d>o6&c zbtTQ=?TdfrIT@zD6E~ks3S0P%aqMXmVsh~RdL3s3`|@o;5`ZcW&GDBX7b;5CW z-DPZ+P=zDPxGZnEwSZ=Dct=icb!j%6y43qHe)w21y=h6F>N+uCE0g2m6)Tsb%J6Z% zrw(psIo9T^8;8ndVT@5I_wx8IkuRE~wqHKy!94(#XB6M?J)b78Ca}?))!ldb`;8Fe zCh(c4omNX({=)0lappO2Ht%{C2N#ll7Z0l1a{iN;sHqqM4;}oH>ixv{Q2hMiv4O5g z*!u}Rp?F76W!1}D&KGKXa?=AYnoRK9d1;nD^~=>sqLJk(Mc&IRrsQ=omrU0+R_Ak1 zLWakyzY4P->!ve$fBRX6rnbv9xm&HGg84nn-N}O}x?>;TVt5LAWq`J%adr9CYHdff z=hjM)FR|;Lm=I@8E92jSp_JP61osx5@O&=ci7SzmB5_N-E`*@~=?QO^h-HnF?YX;BycYG{s zcKPSrY%@Rj3mr-pStk$)8m+|EuSp)&9p$w2+)OrOrXW%-h${rai7LOO=?vzne zfl4rNv_3nX60a#!bLQgAJTOpY=@AG~6;j&4f{B|;gZaW&A;Drs9z~Df$%&;UAjHFf z+r6I?*p@jXJNK=S>IHLEl}&cDwHsLeYHE1{pN*SKv8Qq_2wB&ZKd{>C%E0s+u*&yD zwC2mA>S~w}eDi}ayYfltXO;SlzdXF%8l$IZ(6ZLm{g&@7tzU1l0vmoqux_;}e+)tLKwO)$;ijQL%2rHT@^S(N(_E!x# zvllWE_w4aXeM1k@b3*eg{PjnE=4U23GN)V3s70gDJ?+1mCw@aT`Cq+z*Zgs>`#qxv znfHJrnp`t;E{QE5VBqn|?4uendkK)hBXz}oA;u8fOxTizPuhj5JhzP#{O72$PgUuO zx7C-$bu3ha?L^!*d(Z8RL#~<5^B{Yh0q*IQqo>oh^k-HJX?hYgru4zyjLIEvWp(vb z;vGxvz3;WWSy-I^sf4;Zn6Ybdvc+@u#+(Mf^CX|f)ec=ca6QE4pXx9FZi;KX(fH_C z#q}xK?|!U(ja_?kB-b&I4gI9CdPuGIYpBb2(5qq__I;5C7V+zG-yCqz)?D|iDwi8G z-jRFjz%PjbLoqg0flSz3OEK@8e3uZV$;xV%#icv%ip$TGQ3=bdZAo|E?D(io+e->X zL%lPFY|-+UDT~*!Mx2p7Jzbyw^nQ(vhSk3*QAlFriE3f7sdu<2jQOvX9akP6d!m=4%eaZ?{yEM5WPBINllXlo#Xh@3|B8Tp$vQVfUncCHp}?b8l~vN79L-wgZ|L5ifvy}8}RS{W&bQmzA4tsa|b{j z4tI_GhJ?5`@Ma2g!-^KxwM3}?@vT=3K@(7gX|Ss3x@XhVpG{*J7{%9j5jv_Nyp1_M zAyl;Nsse6=gEcVx8ln}`Vf0p3ql>76-mGjo zAO{0Q=ayg+IXn3ZJ=QqmZp&+AQ3YH&KEUpG#I)^lx%6vEw5t+VEjx_PzG-JXK@=`# zxf8JJEmEH`db)J@wl}6Q?p+NiSj!Bs~9#Kq)m3XBr8y z&^dFul8Wc?=Bj(qA^z01BJ?oW2~F46u;=16_F|Qg7HLT=MMHTR7g{s`b)uu42h|U^ za}pn z_)#nb-vJ_D+zm%vvGSBKT1Jv%2qRqaloVCG5(dp|kk2e-BT^6HDr@2-nKx0w#6$*$ z9}O%Cy0Xe4ucU0L%(xmCbJ@V$j-qWf27+rky5scJuYE^RhsjRCisG)rxgcm>YrY(P zktl#s0{DN{6J3x&LxTcb5Uf+;6WXYCAbzH{lyrFROOY=IIAbO9vD7p1~JP@bcMQoPniv`l|$!2EgBRG__ry-JZ z0c&>ctq~??OE5Q4X(3!}GA3ezHl$k=GbOQh|45hAGJQXd_7J>A7MG(e;!1q3W6)4> zC96X&o+diJC5WW2a%ePt58#s^p@mv%^I(;87hRbCn=#|GkcBLdeNx`S=$;*^Onk<= zl2w)+7F9c}tF((q-5orKQXOgsz7D2xNd&cUsz8+&@k$|ovH1&scy}xgZ_7dLHS-A7 zoMyrF*0W_C7SCiBX#CofI(BdJnQZ|3$a_(%`F$(8`r!N?!q(CRf^H)FVy*}<0U5mKYvuAfhsAHqee~|qlq3&0$ejrEs`ar*psLw?-fD* zPD1XGh)S4}c3nACxrHC43e6q)rbTK^CI5?ZqZE?2eXm(!iY z+lvHMrA`c6<%EOQoIwFSY_e8|%?8Q(Tm$^2hieL+`(r_5kj2ogTKe#EqAW(9+EYef zjX*Ll@OCn3Hnl*bIPdSq98_qlNg0uM$VA!`aDMoV){(bMqDe$4chh|~Iyi^>Ck&-Q zb9MXMStc*wr$IHFpIOHNGwfQ_FHIK$sqAzpliUTADi=}+HTxTVSER+%-QP!ft-A;! z4-J-F(eoj9-Ie=2EP^VjR}te^14(64{;=9kE&?%k3%q25n@7=zO)T3R`YYr5uH4kO5-PPuNnvTi4jp6xBps8#ulEgQqk%HwxM+r9jCzAXdIFy=uK0Z z%Iw1Th5DG*i@wm$M;^~KjalZl!uscU0QP>NC)Gf?h?90BpNEKHBimnK~L1EOs&l2Xe&Cmt_xa3*us^EA(C#_20|MacpD z>osykhYn#GrGR@s!s>J58$J#m9+>PsJJWZ8D_hcDx>HdEA zlncjK?>@5<8+&d#pP|{D242@BfrpZcKH;2~sNYa*+1Tx)L!mfucsp*dm>7)^sxvVs zs=KOpdf-EU>*4=9Tc8k3mlDgcBc+^eG!Gwfga_Vj;r?L)9p09dpZsBe7xP?%<`~`S zM`kmdTsCBdJm^EOk+hul^v*u!3`;!Qo*YZjMq&A|#f(SM! z(C}YfvX>x2R2UEO?q!)O6CEDxiE(ic^U>RB*_BoLw_zNyieHwqqXvTn_Z<2^2|iuh zFy=Uj%<5L~^A}Y=@Gp=Gpl$slthtT$)VrKYTIwpSbbWkqd5CkTXQrAsmyQVTd93G` zycmPDa-E&A?P=w+ezrB1 zs0@)Ur4RvqQDu_!1*NqdAm?7e4#ef2)mB<8?M(3%eJh*YHhGEK_V_jUl7rVE0ms1y zBYlbw0GndKNh_uBiRX`0jO+jCU;Hg!gud%+Y-Z;YkZHI$N_-TDe_3jW*!jkdQuo6x znG2OA+ETch*3%pmmht>HV9hq`SIf^R(Ojp_tNhX@d8`D%xb(|&&-qTIlk1rJ{&408 zMEeo>`&C7_%#odA+l>4xvd7K1{QN338A~wbgHyt}feX;)1>|l?*z^L1&$*IJ z8w6q4;$cn_yRk&WWpUObcW^Q!wH#ajS4i&NuY|s)wlERunS=M1ugqN(w}&?WH;;t% zfiCx{_7N6gPTre$EsNgKx9keM_wH`d_Sa@}b;y&i>%*jyGr4_2Rruysn?1ii>16c$ zyotZSB~Vrgyuej|aonf~zi}oc;;l=WM4*e*_z`gByXk6QhIsK*+4qFG2pzcDVS+ey7=1z`C+G}3mo~c`Q+-$5>!^VPd$`5I@l|2v{5Gw);^z{K( z-&9UZtu&9Z~TjW0g#tmrkR~d{2tE?D?6KniCv2F}Zu4YRavv(1w=efpO^_Q7u~ z+~+IHM5_P5{wt;Y=E=A0a`jL`N2?xy{I!#wI5nP%Qp>(4Pcj5V85}|kJM@1qmUg_* z>h)I2aY>V6m{W@+yY7*y{O8uOAD+TP=d1gBa-`WWgk5TcxfPb?R@)|fe{~4_>Q3fU zlp0%46MA>=vYNyCEXz3~A6+1&RQsS(L$wAzP=POwt0879y{7qt7Eq}If;Zb z2yx8^aMyW`zIM?I(@BVReYoZ*!ZI_I1+1Fl<%O+nbeLmGJM)WmbDNUHmx9H!r+xsFiWM;D;d7MVXlfKlb znmi%VUTAikisHf$tma*j=Ht_VLy}}>)6lbxhIK1UpT0cPFCRGLP(`iKvxa(v1Yp=z zRb&L2Czl~o=USrpt1OWL7Fa4|T^Ony&e6y4x4r0pDmoW`rvERFSBi4Uw-$35k}{=S zlVPmLrNSz5Np888+c3;sxh2}lTr=dJ+%F;Kex3Ws{W_P}a@m+Lx8HvM!RPaNJl>yk z-skmtJ4lNm3M=m7`eRpPR-d9Ok+OV8K*>qc_U7xUVQsEhUMx^vYC)07olVrviZ=aY3scbV5pm; z)3*RBliHIYhAW=VWy2Vo#)odVkN<0hKymGqCFT0llCFaTTSEk;0?$XwM#81avWwc0 zMxSt(UB)*nNbbU`osUQo%`4~Uv($c;1$2BU#+_b=%n1_3>7|Upe=)Zh!UZ>`CS&+p zU%y|_ZEH0xTGuHjd0cB(F-8yYS`-Ycm;(9rN3!*0NF!f`ezwf!sYR#9q< z&^;cJfAK|zB@}Hnu&()U`(t=qV3LbdfI!JwNxO|ctTrEJ?myTg)|uI^!M#IzCbE;W zWHxS6Sw2T7Ib^W1U4rs2UzpHb*I_;Qj-%9@<$(2&gm!FJ2A7a|PMwczrOnAs3tX_& ztoy0RGW{k8bf=SSZR}}cG=Z47cW+Kb$muvCXvO{Xiuw)T)7n|9^SBSn?*_jODIx#O z-;!}N=#(dXM{;eDX2PS_7c9Nb8gw^#f=Ty68>wda3ScJ{A?vXT*WEFROk7q-LKDy_B~CG^Cy22cEl#% zHVgIE8brO16ns@Lczr&;x#=o4^bE$?uP+t5M^Jp`XwcvHJ1f0<2hIAGVhp+yu)pVo zrH43Nl`|sSbv{Xv>38I^@cXBsbn(0z=L5$7tE~_Culm8=AcyqQGm7Z9V(ZR!-)B7N zdV}Kk0u4k;5!4~`qxP_GGpgR}x9`W8WAkY;%R;v*2qJ-X1&-rfTe=i_Cbl!1VE-Wn z?|*vxzs>pjtTwIjHwH{8*Gfj{lUM(zD9mwphOYDbErDpqKV^>&X)>e&aP5;$;HMeA zn!Qp>%A;xThN$vHcH;O1^2+H*sSyWlNoGm^St_ra+a6KXB(F45P}@f;5y|%18!~Ws z1O4Q#&z%=8jtZEHU@oSfPvgw z<1StggE4+1#j~BE+}eV}tr`u-0|~iqcFTrk`}NeXs%;MbvlCIe#mWjcna?LL%%_fg zJ32=^`&{hG0_Z2gumje&|P&FV)iCb6OQom57aR8(hd zm8zaP5ZEow_e(6o26_nd6L+m*h8^c#@m_FaJjh3ou&s&w;d8P-jgNXqq z1AyLzW&lj6qgBjK_-QY+U402yU(#|E!U zG-_T^uIfkY?S<2p90%v5V*fq$;k8)?#unXCl3JkM6NSBIV_4Ri(bD`fpTJ8o|PBI;Pc+GsWv+=D-YcL4v{W+r6M4dUs!7hMCEg6gAL#S zi0Ud3mo;*$SwxD9-ODvnN40Wsn4LukqUNa`Dz&vB*=IU*4EluZ%Xgeg?guFcJ_x*7 za{6Qc^{4wvFpIR01H5UNtc=;e4nIHLs}B9vmz!mMDKi}+_dt=BIyNULE^^HbK13MmW=KJ9*vN z_<`xy&x3G_wRlFk7-*CJ#U31&<$lH<_xW%8=w9-bJLkK$?O3M19PcUY=c?2S%S0>r zY1=gGl%PX0v2I?oV*r{CW{8?~AtE8w+QGqMU7m)|%9~aG8U+)cyuBU+W z3YwKpiiwmo9J58PvF5@GxHt2@E%rY0v8~_wx{L)Eo$#|wK2}afk5sPQmi)T@%LA!- zL+S6*e!~J!$+#Z13g*&Tpey`_XGAKr*=RW-uZ1Q9G>A)t5>Jgp`9FKy&_CfXLUuegFQST4j*0~0(z?%x2p{o9S+I^M3FUkj+oz!i zl;^3m%p(1P5U?Pwb5;c~-~AB|ZBr7UT+=X7Dg?#P8JX{K0>GT9@tnz=Z)#=LC*!kD zYvn#*h5IgGW7L&R&LLh=HCCFnXCP_sS>}4uZB6dzRKHDHe>f1MD_2GB7tCRVSK)RB zOys`Cyv5#$G=!DaMox`g!S=sZ5uBe9KFT2|m;!<=@j*8+S=O>c#i=uzhXkTASr1%i zv*1`Hp_eJ1zZbfR<+ktqM&n7dQ9G(to8D08NFW%$GF1ZiZ{_6O_7ecpD~GRQiq}3W zS82$*h&VK(UWa;&ViU8TDVy$*taE0J!Aqrz-jKE5*?>SOnjMs5guyxlWSMGgHmyra z;wMjYZ%4}YYeoe4G9tXAeW*hqe(y7`-eonV;M$gX9DAoZh|3r4yd%^vV|LDd_sNu# z@lHs>AyUFZD1)q%J5zu-aBOOkHmP+c#Y?Jae3LIsB&|}7b`_Z7Me3ZsI@dMl_oVO? zE6&SPYqQ5y@}DzWHeZzZjkvQlv#GJ=MwC9zh+Mp&Jn@+6N$P4QlKf0xXfJ;W^<7%b zS^Vt8v)TK*j!T@ey6bgDNTTK)hD`vf+s_oQs8&`=@_0Y2>Jgv#va8%rbLYPa#Zt9$ zdtCjq6ITZ`f>}T4qPC<lzx#0srYsQ+|l}jT65CTW{$FPc^z|EBnX0m0SG=5oIR3 z?_t~r7fWy!Z|nZg?5X$IdHdh3NVyCzkXVsk7xL~|eYD?G{f@q+t8-SP2{A5#?@p6% zKx*9v_DA0}m6(Oj zTh=g7PCZ@jmwS;M`)tKF1>zT+naXnXNTj;y8(Ut7t*%t0L>i$9s9Ous4i%!9pGO3a zge{S#Ap*q6i5vZqi#9u%x-{>AFbvIs(@y2(v(N&O$yR#pn_6S^wlmyy*294qXfJHLHZt zV(d9k?PwG) ze3x8+SN{5x#p!8iXXxm+YUS7gz8aYvw9}ai`R1GXqJ>nwmyY2vIxN?GmuwN@E>I+| zv~z}?1aY_FKBU{!bR}Cdy@Ay#G<6o~elK-H+~2H%oQLo3CO#b%w?poA*`Q)AyUIND zUk3W_UsRp((|DQY*Qs4E!3|P49lU3SFLL_zYe)j_z>Yw$G zpobpK9PxECQdm~-lZGvU53G*Lvj5rs3414Bj^uFQ7#JWrUM_(B=Jf{f`v>Ztls#!G zbDc(!$Fm(c|8jfp-@fiQAgQg9cJ5<9AW%B`^U{@$Y9vrBu+fCTj=f+UFK`h3x(f-SpiWWW`>beQiZC6lStg%sO17 z{F_uAv&s2~sw0u@+Er|Y9=AANm7*tq#|K~z5mVK!WxRa>waqsx$n3`hrEtu2=O4Ig-bK>;X7WuJ z@Z_u4&lI=CxtFnN>+;*1l)b`v{*oa>eJ+Jp?8UW z_ytFY^ppC=cuXuV*N$8Sl_v}E(8QhVS*`z>27 z=Tr=j5$ua%MHacm%REMk6&HJ^TK<08O#Q*l9AERk6`@VWNiFk8&zPKt)jKURjHra+ z{?HNI0-b^mqj;mZZK3GZaLIw>L<@p@Tu7dfLI#hN?}F`Akp}n4)45dYURM6v*59e+ zL}Q58l}OmOerySz)WURoZocqVjxes*!xOk0Wxwz_yjIxrki{qL2puIeHf=YFV)wJfcJ z;2WO*K*eWo{TRRXTyx}n#ruNa>L|yo6zVET7xk&MW#L&68Og$yf^@QgNJ1?f{oiMt zmQ;O{@}Z|dm&67yM*3O-EK=D3xld{g?v0D?({wE8%}+X|RmWtY)}bT_bP24z-_7Ck=UqmIL!g?llYr}vif^xSOm4v4 z<}iS+K&{3I+IjdK zF%4}u9G@nvZ86K!Dt5@9UW#N-z|9+Awkl6P!f5}9{fBjT8yY=-#M~-;d`{BBB&V56 zT-as5Mc|eiDUr8W=)YdTz#eY`tTiwIeEe2oFf$p#lzs9n@5;YgtiU_XF3E4_ZXC?Z z2X~N$d`HX#ENTE?6YH(V!1tYvd%aIOzAD;9C0a7$@>4A;f1Q7`hL46X4)(sM5Acy^02{$6dIRo_I?+#kjSX;Qk%OKj` z0R8r6w|_2JkEMo*;Gn%wgud54t zJ?=dC4}(mi6AP7l>1u~4JwH@VPsyE7Ak*_sG~|{n*$Y?M@7>F0vE0stVGs#=zJ9q< zq&Q>01!tPmqE|t5krb1!WG+(kHY|_S^TGC^ywvdh4bOW&#t9sSbWU1a|2#03OIV75 zFg_-sCS>u5%nQsDoem`<*jv9?kg2L~!hdcxNt%bH?Uj^gg&Q2PN}oDbmQ!z z=6=8fVx?8xcSp?bg2Ra5QH`^01s(sO%|2;gY9UP33SMA}!pu|~@9|JK_9K28g zq-sR~>4xW@8@td}x>;+Wa=858*CMm5b^U`DiW}9n6O=#26qXpRj)9r;Y9#tZhTF%3 z%8w6d36((CfQ3Kdf(~8<%~5ck4~YYJa%f4c5Rf%ePiwgyhN@jy{^SbS=;aV;T%bMf z?+pdO_h~JS_|*k~rL?B+1p?S{U1#Z_qdHhc2k@<&*gqV#4-dgG7W)9ZZ8 z^ryTM&mS8^YzfEmSXh@EsfKPcNsxp*l-DH}Mx4IUx`9w3IZ9eM4OIhhpzstk1y`i0 zJ6Ind`l8sDr-Ng5$@thejBIGazq;^RE#0JUvn_AK+(^@BFS6TLczN8`6|g(XCWREK z`GGr-(A5(50qqHo7r6^yp2_mpT~qWZ^Aa4yD&WFxI(#a6 z0J=X^jM@29^TaLNfz1qjXWy26SWJyr1bxqubwPWT7WmVxeB{MW<<^Zc_DP{LQsk)3 zZl|FeC4?twsJeT!5TI*mU`9{;wCVi&LRY&Yu6bi&n$dxkj zHlV%`cwnA1GTYp0zRW5Ojf0Wo^&QBj(qtLmmZ;*xmME~AZ@iB%>7|r%jahEVgXzXt z7er&P?`jPJKZxxb1$W5qjvDXMM`BV3!gW8GS1t0YM07liCmFtLxber4&K;Kwh6Qmp zrkpW9%c@4R$7`L|FtZ>nB{x@1x}|$!K?|%`vBHz!jB7z)AI;%t{OO<(PM+P zK$6}YOD`AUn})0LrXtJ02(iE+Kw9c2&?jmEe7PuQnP(^iYJDgUoyMvoiTY3({AhX0 z1u5YpHDSrd?(haEvRi364(4^rtS4E@Ad)VjtMB!b8@<~NtKDWV2exfQ>RnuIBeLUT zVh6qEW-`gHN5@yk&vLGi;lYdzBc-i?3YVW;X{6WVhq#Vg9~W6V2b!sf{dh4B#h)3~ zDNPULqP$P{|IUA$ro6D&p@ic*aw~6{Y4Y*6Cn?4w^?6DdMP9aGgl|jgs=3 z%)}tnelRCW1u*2w>y^DQiO`mz1dql)dJ_1HBg7NDRuX#9oqVWpmliw19T$6gpnGJe z4vA<}w}eHd9L~HSyA&gM^6Cvf!8iU_u2OjMeINQLjQ7yrs=G|DB!-dd=d-?3=#*Ya z=4M3HE|k5cVkzl1l^oQLK5o7%Zs#{(R&Wq$pA&k}W z9#32qX*jtxD1<&qK_s=!lyP@v=^_rqT(CanVRWO5fV|*)ZI)e8qCm^BEYDsDKzxVN z;~xFQLzkl0LY=7*&}ZPT47H!#@LQcM_oLY;d*~$;5B6Q^y3jNhR$%^9BvNyxX2@sb z5EKiCyJTG)^maQFX@4B|L@24Pva#Ts(ig{C^U*RP)gzmCVm~aK-6Sm=4UIuH$Y;Th zwsrcsShOdyRW7Ss_i0(zl@J8mv<&w>>12F0Kec>aW3IF(%k3NKd~8p~=c*pf z0Sm;=;;o|+5phzV=nbG+o>m|wc&~s3eeM^|7{YYbq*sfzh`OGsItb0%4ITVFy6pLi zY>n$O5I%%tBh`m1azp0`r46slKG!q#+>t(H?M8N5N}xwmY+Z^tN=7YKF2;;ax(Sr%#61sv>OSx-pgwiS$B zjz~(hu-zvCYhwcA(}fY?N&en`3a6#L{R85^(W?W!x4lhW3j z1@JYhCUsT#pPg;HIqcf?mklqj z^8|~|taau4)Pygr9A!UQ{(hTm_iCKzQJ6j3s^WCO%+(Au7FFLHpPa`rnnr}qM0~FJ zeA)Xd-{SEgKTej_VKDFA>(8_tsl~1)KzW6>`&{J;?RC@o$E5qX zH=u*Yy0C0#B@2qTwA_zbYa0>-c%MAT4rrE zPcyuEJR)xS_+S&Jc18y%4T^olDu8IT@OC);5}U@LTRglQPO}(pb~qoICa!bnAUq-d zx{h-mQO`j#XX^(h)xDsz9Gw=C0R*fo?+bm*bD*OgD&M8D$kbix?*7%x5_Ffg6oxP) z-^MIzN=uU$NSnVX6^V&NrL8981{<)ur8=P?mEvS|{Ir{&oOC|sRVmV~v#;*%mVfu| zF`Gl^;x)HB+ID6n+OJ`2KaS|UYDO19CP?saJW2m=Eun67uBvvNvJlHG(OmXDX{4$b zuG$&EWybVRby+Q=PNO-CHu`X;Q7GrHo_u^GX$=I6xWzKkV=F%3zwygq*CR&=5FupA z+Pe{A!(VV;|8&~=^{&WH*)!Ft8@jU}lP(sV=1kvM5a-!sL4@FYyQ0eLUCsDPsJi9i zR+e&S4J^E_>#<=V!iaTt6tKBF{q~N%oX10(=L(`8Ew%IyS*O*4rC~mIssG9|Jp8PO zqrW%U%RH-X4daA3_&WP|7E;SK7JA~*;gx`zM2nH?L>Z-6aeg}{1~3Odzgobuo`s*> zxve};>ba&L#wdFtTI|*vTXDl~E93@5rTtk6MA=$oIg`_Bz#W>)76#KW?L!sh>;mlm{X|(hnF9 z{dd&m>mi?Qa|=G}gI@TpW-(CaqAz7nox05W{~ZD80hu)IM~M9;gsuNdPEaV^Pj7aG zC;jLV4m-0a(?F@CxLS<*STI~Ys-wLBvLt@zw%wES6Tgmy>$f}WLh8?gg}HE?@jRtG z@PMpDG%Gqj*wS~LmV{q3K?R?(!1tv4>G2b|OAtBS7v~b{%X4jN)KS86P{|Yed9?k) z&uFG0c8K4tw1v79+KvI?nSc^SsP&G#wG%GJ7M|Lrb5US@R!WQ7vWi{7Hwb-uF`IGs zCBvW1mU2an&zFjqx6XVbh1;rEi8Ii_c;Edk_!7066?LN9+OBa@pc<K_jma2i&okX^zs~1KuojM*di{8aj%dzecVjdqjz+$Zs^cx-d$ehn)aM z5q1LYF@Ijo{5{W~9<%n<^?3^3p6wQ_7L}l%IFC)}r8cd!$&e(A@X3}?NOH5$q-j4g zrKXwPnDJ9s{Lb`VbO{+%Y+a)$P+U%A6SL{gsJ7vnT0z$YbUXQU6vOX=37E0ADE$w? zWFF(xLYW#X;W}M(;%IF=aDdUVWMCl!X=nBnrTP9eYDN+HG+DUtSY4LCcnVX#W$B(A+2`Ok+!!fue6i@NtdtiqoluV8b zVcrhoJ-3o*RPb%36`tHDq~A6PvNO!R3yon~!BeK=phT4!HaX$HOC2FH(BZu+z+zSf zH88(0lGHBA%rG{G6F>BDv*A&`Sl0!8lsjFu;?TgNCUObHyvZM+bosGt(;9y1hAVsR z4jrL9-O-qu$lexY>KoeXaLHk^qI-9`3{4@`nfeHeCQRjxExkDAo!x6w{9ERoq0ZO*>|8+B^wD>}PN#^d|FS zz-sNSrq=So-Kp-7ZUy&(#+%ao@-ey8yN8eJ+DktD#Zq2Zjl>v_SyAQ;PMBivqr+-^ z#m`nFxS6VI;lU~9HU|kqEB6T77P;F1Rv!`WQd4(qZj^mXtSL&uZ7UVBupI@*%G53; zfN!JkG5MSOq>a{mi}^Uq^F~3XO0*JRLa3Q<>>Z&X<^HW3!3$y4TNIp-9*r3d1(LUh za>}nHG3AExHR89c&+mn)_@vbU!A#U~y{$-Hq_g zKzwF8H1>-tS=KH>-kSDJqf$wmbdDPcHQ%M_{%gZ9v&4b}4*AiP@=}J;iCWe_-x7G? z%x6l%Q&bwY|C&wfud;g+6sLlcNM^sRd~@G8>xuRW__}s$4#__K zlL=2Y6Un+HE-xhR-2DTu4oNeu+It9Sq~8*%XNus5h~+#u(ki~}a4QqgUzJ4V*M<#O zK<5b=OuKjP+SbKzVjOJ$+*FIx;=KLe*HwfV&#Jm#MyuRS-up4pSjJo*rHnY^XXg6< zXAWOu;dcX6n2G9DHa}0Ixvy6gbmq}~RK5$!v;t}shQ34xtJkO!i#~X3dg#3BjOkYx zD2}PJnb#UZ)fD!)U>wbNoD*|kku5ao|Fl&Q)AufvSbpu<9hQR4FI?TbZh)ITa^}#t zlPJWKlgidNbB}B#7un~+Un4y=#8ubQkUh*MvOE9KwFVjgGx#Gi^rmX8q_7)JnDN~b z=ISYy#ghwv(xbiUV7>R!x0cHwDDg-0cB``2!oSCgAw#X7k{wlxLwwAuYpRUxcSc}N zR76?)NwY+>p!88uReE8*q}`af>%Z&T^*%+44G8MLOkt#R&yEh-Z?N6Go zvi6hrNQ>{7@P;YRZLe>iey5kB$fhM7#b(cs=&qK>Z8HmsbHZjbE?&DbYb?fYhn_FW zPt@*`6fQ{0WVv>>{v0&AXSJmSo?ZSd(opbG&#&sGlHhShvr<>cFRLg|>vCCU=l#*C zNUvAi2oQ6j??ggZ7bk++!`cjKVmkKA#4e8ocg~&d%$J!^Iwmb4t{i4v0|nS?JZ9t2 zZg@1t-VeKP-=Mr;C6QIWe-csSVuwQ2%st>=P7WD~&V#k-2aUMzymY)aDn%g{fAsX& z0jF~Yv)vebmgt8xSJ7@dk(mDu?~jrjY_12OTLH7Dp80P23Ir?~HBv zO2+l+^`EpXXZkk@X9;m2(ZsNm#>vbd_QH zo&+1HPGX6Hb@ejgXCLV7>z>S0LY5HIU&J49P_o$-Xtw^YfF-6(5 zo@V|iz8jUzlqX#xjLSR}4`%JOO~_D8R&IH!gwT_VdDbYD4hY5@aZwcK{@k1h2(Q{`JCLts(JxlV{P@GiOm-P|Ex5?W9hq zJzL_6TOobN%l<5%d+gd~UbX$jh~}!_sK|Y|;nvse%3nUbfes z!u8IBg!|tGU^SMziLH^d-jbm3F5|q}p61RvC0j=T)CsVBxie916>@&^I{y-JYTz%m zR6aC!F^=+FiMU4=2=X>t>-r48_zOLolR3SJ*&Rh(n}|LO7p(%%(W>Og>-<`*Kw#bO^+m*3^@waoKZY`&$y65i4M`yLA5 z5bzc1a+Gq_p*>;cLsQt2^X#+8M+2>czCJ9U)`L%`{?m4N?QnRQ=1*DI$+2OrS8>9L{g!W90cwl<1LJxT$gmYsz>*H~P{!x7ZueIG!Kp?+CM>5QdMo}YT>9|j z-^TJi^mW!IDr+WZfWg^&^QS-`uI{~V>hu!6_#j^*uU)Z2dhk=TfQ^P&Q>#XnubM)aiGOP2jVIeI{(Of_bRxV zm{_&iRp+y*PbYd>2DtO+?S@qhz3Q?ly2A4X)9NeZv+d+c!lv4g#>3ZEQzAdt;}Sd> z%0nMU6B6AfPx5+3zXsk%_G}&u=fS1_B6k9RH5fY-R0zIefyJ1me*orI|I*bQ|8UsI z8j5wdJH^Ka??QaCe#plMZ~o*4aB5vhf~<^eevRlffQ7cnC^X0^a~3VW^WN2RlN-q1J5i4TIz0D@ArJE!-e@%g3izs|4{!f?beB}qnyMJE! z>R&{GSbTXwkqsf;kEn4I#tO8L>9V*h9{0>v&|}PT4ZBsKFJ%by!1AyT>nkArQx|rT*|+o=5uLAIC0Ds~SG{ zjtuArP8f@rN}#K{iwR;7StzS`@9~)Z`0{@;X&4M9FYChoC6Dv=4{fw8(C`6*sBA3u zDt{dDdT{9;Non~vY>KK{^rY&FvNyDKJQjNz`0T#r*+^_NRdF_j56ulRvfwvT)Iw`P zuyfBu5cmhN!jR`4(ht9M!!LQwClT3Hqc#GL5BKAs0xq3WvEDQC#0zpYy{(4<;qlHW zLOlZ0OcW0l( zF_K$EA1x;2f%qEjVL?j?G&?cXZXD89E?iT*=EvL$Ra*^2p-L}ae+wiqc-F%$y`_yT z3rZTjxhk8Jz6GD5U+*WCl6&NU;^%CTxf!obnwGsDY7mmH5KG_eEaA>9BW0$E&dw-M zWL{n#FtqD)_vuv^p!T^fsDN~D|5Fd~%zvub~6NMikJIaC@k`fIhq)> z%kDBTJS%ik|MYiE19kCRGE3mM8_MEvmOwerC5TX8wZe0i{i?%lu3CjTjdwrSGFk&G zBkDwi)mK&UBS-0fgS={#5C4wc7mU%v_h|@6R73qJIbocMCyvyI2L1wllz0PXp3>=G zYY}Xbt^b?6{@%Gw&pFS-LU3cl5(l$j#V+G;)YgJx-3SU1_%BZyp9SnqiV@3e&UMk( z79c4GZOH}iW=`|jv>L-l>+0%;&92f|Jl_l#oE$xs+gM71{#F8^ZnTy?jIIC^Umyud zh=mt|&fR>k2pI@`cG$|0h^pMsm-B{;EwVu9TjPl8g$u@v&@4jK-?vwfwB>@UYBN2Eu#9)Vt!T2yu(ML#7bod9@zOmfl0f zA<^3AO!+CI^DMyL*+Wwrx;W2jyvfeH?ik@8_BS2bj9GVd@!kKEdW`Wa37)N{t8Bgq zG_xo^7@44~iI3h-AxuN;Rrd@#!i>2~{Lc0Mle$v8W6t7+XO+9Zt%dt*Q;P0*Bq(KK^V?}m@y*HlPNs@y8vQ9< z^hkH_Ck3@Qa`cOLZh5-{wxw}-eHK$MQpf7e^~#qE7hKDPM`>pv`+d7YE(MI1K))o) zA31F?>rSQ(6^r`q=4A zWW^F7V1H{l&cVvsHVJ=W!R=>#Cgns3JvVD1>yo?XrTsr|t1a{bKQ)bVVP57!}EvEW&#p+h?{|EQP B9{K

v0fZs^}f@hwko54syYpkp#VUjO~M zpy0jhZ|+t#5e%oU%E2|B4+t;{L`#abu7weg`OOG(6u6CnMpTiva=NrWe zv#Jhr(F>O9zdjVS=T}%zvTw9|h0*(#xesT!yc#)!cyll+oOBf3C6p+pC|zymYA4v> zxGedq3_+A2e?pW$`|m^j9|x7-SsVNEAu)`zGYX%>PB9<$Tj_iJ=K>^OiR}_WdqfJB z+qWOsHW;=J+=6o3_gMhV@_0G!J_oVCbZo*I+zSDKb%VQbks#7OcWOT<{zU!B6@s{8 zx?IL?26i)dhtr9QEbm*RKF6&tf-^+`8o-3slEM&M3CITJ24X4{UXuVW3ig!%_K6f= zTX7GekDcKWa01c9f<53#PhJ0W?J?0PTj^%Jt%S$iQk{|N5_pQt*lpb3-glpQ0L`^W z;d6<&UiDnzG1e+uAkOv?WJ0eHy&HdG85uH%ge|)xs&J?F6EXrzin-M5v*b9j5~dCdYBcap-hR)R<5;||LGd1JH|3e;YhjK z?o`5378gc-ehXRzDcZEe5ICql@5|NkU`%t1(zm+cO*5bc`k8rm4Sea5fQdNK!4IiR{0H4HNM!tOFa`wXczx9ewCOSp?A zKwHL3a~TpNHbBO9{Vh%gNDmcu)wrwiBgWk~yVWdX{aq2arVq19na6VH`>G#jY}e1r zhRkS$%sk7Jb|#%PmxE{SnZ_vVtp-i)xT6UxZXP=I%v=?{-LaCru@}Gq!bUm(9gENG8*R7yQ^`Y9@=3l^r zr*Fmk41(aO771}CYQ+T+gkolc8it{BSK>yU7wCmZy``*@*$xb6Ff*i@5#td&owNwh zeh1Ant6drN<%@&JIC9?P%ff@k2F3fZ+zpNNRaCnUusE>C#a)|dJZTTw`6+g~;Uu

%Klb!+wkMGRx74 zmx*+?JYCT>jvRSW*smF)zDw69^_spyLeZvDjY zc;ghrJ!~yK$1=G|!e%#TfAI08B9#O-Pf6uabT7KVY>z}DJ zY2mi|BHfZnX_wn4*Yk4CKqahKTn10y*n0mm^wOE5N7b#eO(BGhhD_~|(eMn%hga#~ zs8+l|ZNju!hHjwS;4MDMn62cn@KeWjUVk&xhP=&Ja+u!u*$(j!ikdqwqZ{lR9yOXM zEK;trCcPOKa$#@)f6nQaVcY>IwvramFSo^*{s>+}YFVV4?(j_422~|$4TGvP8&{&w z{i!)%RmyGCAe0Fbcp_Z<666u>^K(aj`#>LmA!Nt_ligYDcWo{Vn+f5W5Gw_LUn8VE zejl+mnKqiS9h|74<)DAODEaL<{BD-8p7S^YTcrLj6_4lq((Ibn3deUu_ldl9$ zN_R0bB(~ltyD=9OJ*M+zF(3b6^}ld;@~K2+Me&s78;mt#x7^iA+2FT23YtWy%$N zwwRYj|I|e^7q9Z(G^;{Av#L2+pT$4dQiA66_@#MJNeV*N?ZVuydA+qvtfoYd!WY!0eo~HP_9dJiyDWy)bV$ z3G?fu<9p-RTt{Zh)}d$Ph9wJBc*|pydjAiB2pw6aC@t9{4o%IcI9>?GXAA)=O`V$R z%vC)S_Dw^}ZapEFp%Q~*o>4^kIa~m9OF%GWIw!EtU$R~0TzbJ>*>%Lod- ztjRFIy+SqfiDtMHZ^u{Ru0(vzP!5Uj>z4dZF-6t#`c`8~ThL;zGV3c;2~Y*O&Hbt; zWWW#E(Mr~{yPr}2kgMBBY3E-!yL9b`WRBR+&v#pW3Olv#Jz>*xJbh9IX>cGFte3Lj zbS}NVKsYr}T-TQNfURabJY!~?;{GL9Ev>x3IAqZ*6CF7-cMk8v&-s7${y9KfCf)~o z#AWpD%sZjyAH03M)Rc3v6{v&IAN!Yku-RqseI*5%`q#d=32ZZ$?kZH}0o&bKJlX7mLcxiMu80j zK?_9r#0h`7MsNwniS5>j@vH*c&n3R}^<S!P_& zsZUBKkOe#MWu2_&5KFm}*E7kZV1P=f5UU;(EHp-+&Kz9TRNf&AWG(JY#+r^DzU?{x zbj;`9DhY?>nDQ{)5cQD39w(;rQ)rsq!is4HQamJRuY#Cs;*VY`_tb_dCWn;e09NwL z&5uKafU?GkuJ+*s?~2a#!!->dtm?7zqamGi<3WWG1sj;x!R9g17NO!x3)Bn7elMj2 z@lQ`TvvqF0$`0X~#QlFO9MfD4&~dh*3#u8Sy~s{~kq^gw&i!%NHXZRg{}?o>-pq7i z+Iv7m*En5JgdSokFeT(s+frU;9g05iRwoSW*qYK$Y1 zqU~;E^z1nk^Ml%TNM+5K(D1gx*jV49BK54L?9L9;O2W`XeBD>8$tUK)umS{qaL0YK zXEBv0BilDL4y{>J? zz7;N|jMUFb=|5Xm@a_#MvcE-5QYmjp)J!*&KeBruovSN+i&l0*!*1l;92U2cy}Vc5 zuIHZ0K6N49=4>42cVMS-xoY#)&z+lC^W!J?V#0OHzp}0=d^142mqhgpynWDL@G8-k zHYaep{6~m?0kGr+g24D_t@99ljqs*#J+!>dFJ|dL7}`(B0>3_TZ;+NkN&2>64K+SB zghmKGY%-Y9(DPh5G5fKncvrSQEKF6_JGMbhSKs`Q1iAZ?phUURX1IV<;$993AGWg* zNXdBIW zxW5TIp3!tfoqAJclRLn0?p;%psPa`j&QT)TZW~pt&p26x&3u)^*(lcWnl|#waNAxh z;aN{DXY4;o>HoAjuzS6%Avv$Bg3rWM)F+=4j*S~lN`8cJf(h%f4RzSeXdnn5no3J% zcZfdor4wIeJ&hE0PyDmLO?vrY2U}j8-K+}r%L7@=QTL$9p?1UcQ4f$%k2ca_fO;h^ zu)9q(BT7J1x%n8cTiOmTl(;Lxu##bCUwv- z(3W$vzm_>*4EN2GXy_qSFb<3em9sx}>Jy`gN1W*$0c2ju2Uz%GwSP2?cExe9V;M&F zSUK7ll{7eZf^?@(W@;Ez0OhR~V0F)%`Xyn#Z@gPb*J`}8?3^@ZuIYmQIdUse zD;t0-_p4R~t7x|1BXYIIkC7v&@8av=82b8fNyw%{<-Z`HOrzt2wSEDS@uU6dsRaW` zRXyXF+1GDm%0C>KxWSGbxpMVqh_kx7s+zign#0xicJXnp0O3x^q44pbn=B$HxyaJ9 zi=Z^pVuWNL#&2hjt%iLdz(QC-sLw4{v_NmOHQ@Ij)x};m>X%)4!9cnaMYBmuezS5| z1}vkhgSfb84SJF^`isZ6>K8;z!5y@oUbd1IFzNE%qmh#gwFlsI|^BESfH-Zc_)c2PDV`6YRNA0>DOYX6`w|Dda$O1nJoR{kZ+KJj8x za*QV#QgH<2p?Ynt$_Y7FF{l)rT0O5*?Y#T%kM29t0(@PmT7jPBzc-6OGHg7R;$T{g z`m6F%hcWwm(fi*d5%rtHle`^-63L#>&3w39*L(W+iwMMyQIX{*?k3SR?5Ox8=4190 z6Tgk($Up_X zn;{HVK_%Oab{7=~{KNWL1x8J`=3B)wZM(J(iKk`XYUQg7o_HKl8{p9~il;q2O`EiK zL>+$C_*pa(W4b7NUIr~PJvO*LH!J3pXa4$cb1OMgneJpqDPJSteoOgCZ}+{x^-8}m zW!SwdGuh>kflQL$BN8(-R2-|Sw^yWM({`gqjNZd!7EsNCD>@$Nyp2Ny5Bl(I5x?F) z{h!ynyvobc>|?!JsL1rDx!X!ROIxT;6l$tHaOY>vp=*J{I=7(dYAFlhhWsJw5Q@>x za@&IWC^MO1p`AiI@f`6~o zGI%z9ruWV2ppi!4bMw8zc<5X`Q%?V1%vQ{i*^T&xZa9AEm|0)z03BC>6zRvpPH8W-_Tr43k$W z{Mu%|K@uAKob_Jo>wsOkb?A&5c(+0qNc=wU3UAE}lb2B=_HdZdYlW$7Bj}T9kY2rA z?qc!69}0V9*l1_GIa^P>9(D4_ws%1$dX)`7iy~gA^5rq#Q*n&N>|w($p=Ply{|5cnJ4a6!V}^8fu^0K{1Ux`q?YXQ{Q%5;8UqV(-rMCPp4s zJBuOr&-3dTH znSZCN-sBmd4Un zHrnXqaaOm!e;n_ms)3<<3FTs(x6*&ny%6rxzEn&|Z@-DKZhxK3rlUvJo`;x8Sm9Mn z-bN)wS|!-sufQt3fAl!a5dUomiu&nL0$*4-=c4UrorG>LCD!bV%57WxdG`ynqyg=` z3B}m5JIyl|W%T7?S#Z{3 z!%~P3!%h@ww-<>Nc5*vIS0C^|YaU1&U!-}j=a_eA&4+r+^sJ2qp0Vh{cazDUSlS=Q zY0|Y{%@CpwJ2{5rYeG)zbozK zm&-5l%>wr60JzL24vi%c84lELzI_%@eK<_@vP-aEt~mCzHGVlwKFJZv*EA*BlDx?f z+}M#3pP~#%fGsJ1_MrotEj;O9fiOa&VHP7(#}9U!QPSWP6rOQT3eo&G_CMabt7i>g zKEPJK6BkV9`8^?4Q#)PSDa@$za1g#nRSpCl z{KP;#Ipu+7fGUacy;)Q_@d(jsS~a4zy~K;GohB7W5sJ7FGvFlFW zVE0nrzehm7*O$AtT}t}svUNj4zxUXyV7|I6yz3LQ{*6PA^DZQf;AkkvdyW76mQ-Id zvGg*NTk7C?YoYloJC#Cn>3Y*U&4l$9_MGt}Yy|!9YMd#sC3_BIPLQfzln~ z$PE}U1jfeb4c_s6p5Jr5*BjT_U%T$>+`G?xcH&c~Xtuwq`K_u39=5&cs~*uXZ~dh{ zJj=Vp*>`c<_Ke%8cT{i-UB?*GSF-1aC;=KTY*BUjU~7X{GkPN@?0lUbr)xX`zZFEIpET|u`bJM5mfbv^HTbqMv? z?X#QuC?4YK#Fz!n3iSOs$z++aFVcTz3*#h1d=>RY_CLLs)xcbm>DD}a`VRXICkWWZ z!tIe0EJM&H8dh2N#U(T5J~{@Wc1SfYR*1GBTUd}1mzfkkQU`3r50mEL2#0X7n^xIa z6@fB)f!)JF?;wQ9yjZ5~cSZ@XI47KWds^^evf;=J=4{&|>vwa1AE1chWak$SAxL3+ ztTiv?e&)`NJnzBEOriThHXQJ!^94_vn~;(DOm6yXq9r5UcI2z_J8l38e~B zt=|?z8&*B4qGZNEaiG%qvcc@};yfUhCDh#KAQjNe&pO#cjn9~yF4WQ%4mpGMC>C#R z``@0tlyJ|&Vcb^A>|%=A8>aMiiL0P(sj$(aI_6Ni>}>mJ7639^Sex~}8Quo>EQF^; zhNa7SeIoMElS0#7q@eyep96H1{Mz0w)vYn+i%yptvVwe2+4>YxOZ5hL(r2iy`WGhY z9sM(+DVu2Txcorf$#j@WLa?IXnOVO77uug@52|d>4e50f?i7gu)gV2KPR#g=gTffp z#R{0<@vQ$uO!B^UK6L=8Ti|*v%1s(*W@ur~thBxOF{3_hwJ1;UN5Psv^dxRaVTK7~PD@25YXp~tx;v$a5BLl4OhqAONd^_LCm;Vzj4(Hz3A_vo?$C9849f7mW80%Q_xyey9pDrjwZ(b2TkS{qbOdYms4Ki!_isbz|6wb zvER$%O|*w*9hrH!=&u`wO(H!!({qu>=1mb5reE!KY7ET<#(t2jns_{ei^_Ea{oABxtm^#I zr=w^x1BTv822=@t=DhF!ohvT{i-Q))U40VAD}4p`imJ6+Mh|1gIxW-IJ52E4=-5q+ zbF5Gii?~d_Csi;2Spu9WG+SpzSf&{urns9Mosfpd>keR?1RHFF5~~;PCLqq*{Jo|% zVc~uz{eMP~I_U>lu>;7};R9>;mmtz}OFuc9fsNUg-oI5tOzm-;o3Peas4}Oawjl5YKCPbk z-#h7A1a+rlxvc9}q}B;_!hx#PM{zak!)S90(?H^NsaE$-EGmK7itVe?HzZoR?}?Ma z@kU)g<24EWfN{n)CyM;8mW?(nm5B_>sR0@9`+l-a>=yPRCNI?AvQ7>5$}r_uYFlmU z`|q{F(6la|q*hIrqTHS7Bwa4j4Yg+{TFnV(eA5z4lFV(!{dA*Dh@q$gLdhcX&m=Sd zG?rJm81Dt9FlcYU%2H$kJ}{-fC>=*E`qV(OdG*{l9l${`P=fx72#FPSdFXY|{6iPse5Tk$4{Naed} zGSB$#RnLNg%;(UjiD4Q|f{tLNa#=DJrbdrhtA_MQSZZGhQKo7V)YLtytQ`(LaTCn> zlEEuW)l*AWA;g5O8jxHO(Zef;hz`QERpu%#33!g((eztxG zzE0sA+<$A51xJ?g6PIe%(Are9#sa&lHvnLpDR*~sp(ws3V2^}xEwY_)vU#PuUrgGCNCmqrJ}6#s8TJ&iiZYR`jb)+h?|?*79;o5R%p? zQIUUI{LkGF$g@?v(gvNa=E&T#xT-qN5ORC?P}H8XCU<2TIW%T>A)mZ~B>2=NpeyGM zAfTi3(hZ%gBxa~%N$M$84PNg_wV%>TyxGWydKuSc_G@bjGewaep1m9fT!Bt z`VTZ!6pzgv=jGePYs{c>Rqhp4%Sj6_`?(MLGgx-}d^F(*o7`@psufx&LNZjX;(9GoC?n)p&!fW91rF==NV|C znVz06#xr-9OF1>LE)9t2p@TaJy~aOq3%8?m83IPi266jT?x#+7$&6!-p_>jA+J(3J zx_N}e-E7t7UL&k-2pPqFj>z1P)Ek=18fhs*FCse!*eg!9%S6*AF@Sz|P#&Pk5GO7h5I*NE^uffvmqdz#lmM=eKQKx@bK^`rmj(c;z$^=B9%^6+13 zR|fh1yfWixi@MhBAh``PhJ7DA=q(64oNgWCCZ7>Qe==0z0o$UFP0P$RK7U+hVAM5R z2&USZR3*5yx_MjSptkEdrU(srrHk}xONy*`6YX-O3=y3v64Y~IP zjqH65sbDqZ$eyXJy_Hqes;@pIi7 z#!=IsNXvdiXRz7pd1_)4R%10>!=fzst*fScz{vr(vkcs?YtVP%hig;Jrn%lf?KRhN zYD%GD-Ara9=0OHx@aY8(!6_I1kJ_?2AGz-iL=A)v)G?HN`D^d(qOrGsPM(}V%V&?d zhq-Xq^)YyO&n^^KlAFkB;h4bVvrCui;)U&PYd%qC(FY`+?0fUh9wVm3qxdDFANY1= z${tZy;nvXBqTo;xhdzg7qR~!p`~lL^wr<>_80>^Jx$a@AsZ`c=)a|TMp=7k^$(I^_{e*jbf#BA=V0(pT_L!UbKUJ znkTN1X=SV+j0k9LGnLw_2ma5KFb<#ytUY3m+1 zYUFe};w)Wk$>~QXFdDFMVo%Ei_@eO8P24_mVrV*b>px)4AQh__8CQUJ7y5$FoN;v= z3~KTS5j4oK48{sFxE`jJjElUXijiu+k zT;Hlc%56_rEYWU>{9`k6PAV@H?P6g&q9$d zj1oyd&hpX*$D{#81okwoY7rns^&ucv;S=Kax4C)lJ>Td`toJUX-u0Pf-mDSK#v<3nBf%}>xx1qQ?yh!V ztLJk|Tnz05>;v_A*dDS78L*N*+MxDm6RlR8q!h&L?s~uV{Nq(xnpXa7^y2$Lx5W#n2ncNd`?aT4S{ZmUejV@mmMQ90tL7(#_6na15-qe&7Q9 z=uDMnb9&x+G*mw3FXr|%vkpa}gZ^!GHCLjUQSolmiGkZKJxG6U>!MhvrYmwhB@8j& zrPD;Rd`HUOJQz;Uqk?aVRGKctn?ArHo z?VDZKC0mCl16(J^JG(DH5s}R$nMcch;f&|m)0<=L56l7bSmvFXYeSaut7zjp(+M}r&8 z`Pkay%lN0#u3jz zyHp~FKDcpvdt~7V74|TiDe2G=Ba|aH+hVGHz{f=lI6VAMBje$*{D}|I}}?jVYrq>$JjYYwzex7o2 zx-Y9#7WA1}zw3}_E&<7~I#zJOilUbr>(6{6FNih?QEu9IFFfFf7%% zn8YLB=J?#BHG$~SPSfVUat;p{jp9y$#+ zU8YW%o4h}FdzeUIme_`dc<7Mv0e#g2?Cf!M>PpdWsR6mz^neMB zkwnsRF(=u5FR%}jzWAgyEx{c=OYB~@46frkJ@{>zZkEcXJ>x`9c@{G5@1*p2+gCzo zd-=A`Ox2Xj;tc1M8AQTX+ny*Jo}#o9W_Kk(8F9K~{?rDYmfzfygI%*Y%BQ}X0_dz+ zzxR+*Q??md{hi=jBrIgN!Vzl3F8fD2@0^m51DI-B@cmQ1HyP{6Fl{jTQ>ApCG6v9Tw7UFS zppyTOs~wpD^*!vv?yC_@U9oFaD^>$rMT9lQUk|05|0m$0`ZuVxsgp*o23!LRX(fV8 zj|t$paB*G|LXsO|6NXZA1kamC$hT>dQ8R z^jpS@TOIbPM_^kLNb2XzlCrl5X|ZB*x7IcS4608#|y4 zot{+U7}os7NEvgtzbscVi9l!b-(J9zShgB7ISEA}qaNgac)(47w+H#AVUt-A2Ycj- zR40q-e0r(~qrOfE*T$#!tHtKLZSF>$?}M>Ysiwlj-sFsKiTSYbKu#W$sy@P>C4B#O zkz}44A>G2^6q(zgD`ypK%B&5)|Nh!w=Ek5|D5qfI(2LV<9h)rSu1O52(%JF znz6$Y7RaehZv+;!QQ}2>!>xTDXH5(yOGroH%=7#Y$3R~N+uZjRzj6TPp_dGTpFTaF zFcZcAVkf7}d8^!AuTZ@dX{Nx<`Tnn77|TT22BKhoFFd^QcN^gWAd$;n4TZM@Jb0W3$U1sg6L)m&(l>|Ekq?O^ME=0k=kbsfEHPV4Bfo*lc%(zHrCO%9 zs#k|Gz3O4dTz0s2eodRp5tN5>s7GYatMxc4gSi{8vQ*LnhPMrG*D{;sE@~FuR(~py zqUMfTIh@uAJin601G$|@Xg9LYUu!$Dzv!^UILvdNyy(Os+EfK$OoYUG0WN_tT4-91 zKVbasnt_=O~>=o2h z=%rZL`_#*NffgF>6DHgu zavw(?G6A%ZjNRbTUddY|Ll&l%3E}4HEiQp7*L*ZiM6Q8fYNJf3LNEp%xP?sn#qv817$Umesb_d|fIDUV-JoZ@z{$rj=PCf!NXse$jWOk^4MaFNE7fz_dX~bJjF93&Xx`Eu-`bGrI1j}hRm6)p z90QL4PKz~JL|5a(FVD&#m`sU;x~uomeR zBjKUWE0d}1)Y^ES-?=SRU98;d&$metvy@Amv4Yo7k#l^x=^w#U!uWkOX>ZZqbD$6B4Y%t zi4x^^4T&ig?PjP&*WOt_kfQCmB9=-PY` zquKG{8~8{d+Hv`fLXG>L(fH^m6q803h^JFIF_+@I3E73aTis1J z=ol~47VEgj-f@`sq+w^meY=xbjiQO)Xuj7UFtqGx@dS<8Airn(z|SU#(4{qP-(Irx zXNVr!C&3vJiqR@etnub6uNZBIYzumR90?_6XYRF|MC>|d%x&)mGk$O3Wq>;!^tm++ z2+OQyGae#hgVa;G*s_>t`deIM24@+bD)**nh#OzsFdjL}T9cqeNcWiJtvh*j7M+fp z2tsau7V8@ttSP>dSgrgy(R)XBB>xh+e#$&Q%EL_gc*5B1Sg3Xd^RwZ*LR}i$=D1eU z6YZv!@)K(c!V@j6Qk5^`tET-Jjm{f&w?IcdqwQ-_&hrRKb;?b=0CW=@R=A+z8a{Y& z1=)ULCUKtGR#u4>-yaGWU1M%`Rfg6$b|GV!L!m&8P_Ts9aJGhAD%CFw8hkOZHZp!s z3obQs{x&A_1-*p#=Xh1Oq&k-eyOvaCr~`T;VskAY67woTF$}T%Qsr%>k_@AB2+(#g z855Rt&Y=5CyrNo`b`F+!v`kH~)r=MXpzFrt8e)n1GmQ1=?{{w6PG09cSjOnY3Jc>N zwJ`B1CP*Fo^E<4<<3tD9m;bYgK2;H@YE~EG?BaC@C-CS#ozCe%KmEPDv8R_b9?N2= zyVQUk1gZ51LdXmNBnOm}%bikc0RenhZerLGVbN{{|Fpci=&T-$;6j|=6-t!3JC3Nl zdW@-JM_mf3-8rCphTrY!|8j4z0R43R(UfszfuSnE8tt)CngGZ4AB>n4TGg4$w?|-)*{>iKr+i+Hai1 zo?%XLFdb1Ee;4Vc9Q&K60okE$GyUgNkr~s6B99G z078d=eHpxS*#%3b?#Tq z!(~a*OfSRmw4FLKE!5bSW=2LNiq+iVU`B0@5>=a>Yh5X(g&(QOO>hk*e0I_qm^&5G zf z(Cej{%h943hurMCMWf(o0%5>*9@pHne+LzGa4ZBwddfL(PN&y8{Z-{ zWxW2rM)~N`0HP-?s%z-9lkuN8G3Po=kqDVVi{jNg#lu#&H1}24 zs6r~wP17Pmz2$5IJ+bk0JRzlZVnIi?5ULKQicw2%G$jv}i|!n~#Zo0QH&78}Iij#b zOk}NeHO>%Le`XXyam-+^llJYH50rfQ&H8ed?psx%o|QL92vP0;Gh7(6OE76X;A60@ z>YVyT%hC~3tf8UO|I6Ibgef%Z15(Mn>62cS1P2}!2_iT~$CnEp4Gyr$`8V&6iR&qx z;P8cd$w)aE2AW}DqZLX`#uSgvgMHw>vfFjLEvq^QH8s^G&?=`I)*Wwf(T#VVQPm5S zZI^l|Z#+K;iIx7L(dkK39#ryV zr6^NeN_f&Fdva0&!=PhxAvh_2U|Q8LrFxZ_$!>SFBbOQ@8mIY}aJN+<+aGN*Yfa^i z{D(xe!cXV0I)iNpF!=%TVB&d&wSyRx}=icGH5G~H_2M(88Qj! z-$&>OTLko*<$#5aieBr`Gk_@?{^?xhY;_d*BMGbz&IEU&&(J%Hhh zQB#t`p(Fyd0gfG~PwIseU%hS`L(HJo^Cxxs&sDUaI|4y8vZ&_Q7aH{lBq(T?AjM{{ z!E}xlR(3SD?N1dd(OmNWjvfdHCYFqDxhbX^s*U20B^p4BLu9JQ*|)e>!3wsOyt#oNrm+(^eq&Wv&<-r+HspAEy^G-Ec8b-=(Bh5oRRTFC~`%8~-s>`$LV zG@>DfscAB@?6iVM1)XP~c-=6ysFYUlk-Qq_Nu@XO0Zjf-P$S2;WPMRI&qcX=t%K_K z4r_^Vct{nK&BNDm)tQ;XII9a0tozDZ;#=A}v5!Nk)RMTJe|t=)lnxFlp7fS8A64PV zSxMeFC6jZwe2s|U7B!6_X>&?P zTPK*Ww*N%+m^2Zfw&#X=-cHTh^uPV+7$eB*n#A+e@C_zbmF(-4+ujd9B31K0&10Ki z`yB4l{d%-);o4wbUPgWWhzf0Ev6Lh#E7j#|Q+M*0_wgRfG>h7)R*$`4n5*%$nN{Ryem`*UDay$;pF3P2d#Pw1Rt{Y>Ru?%f)qg2x8CIL79<&QBWm{2+ zj@3-a7T()Mt)NR~6*utrEt@wuiZV}YF##?KsOVnzn$BK%MjCia6iS~@ZWFOzG5)9q z)ryFSW*FA~2XsRAy+c&r0)1}VTjP+=q*+cd3JKg3|Gm1&DSh=GFCP?u?$bYCp0xjD8!aiod4XdfNX0>rAvd?TO`E$KL;HPVznNhpCwJ1@(Rn= z-Larx^;kXkVnbnv>Us&9^*48rYjx!qu%eUm0bH)~b2%7F0h6~8q(3(y7*jdMHD`bC^p~QQAAT%v6=Q+Jr-XIKpMTc^YZ=AQ4UsyN>c0!Bq z6jOG!rh9|{>wFEoRWv57bX$`AD4IJj>Wv}gA5|qs3gOJuP|S8z3OHS)AAYMw!WCzL z)2{Fuj*T|CIMu9c_8DKL-7rw-?Y!GqZQQy!W!dPj2#uX=e%2@B>OZRh!SAipNHh3|sLJI&k!>UudO zLK0UNN658~i?1@N;s45XjKXyHn0z~HYox%(fgc-}nY$RZh#L0?R`yI^`Wa89e>5}Q ztWUeqv-D7NLV$DI^c$^`{zP|7Tr1aOpO^<7dN_-!9D2RtU2mOMeE)p zNDN!|nuDJ~-??ZtK$)3D-EY&woKkt!)C}=n(unL|S+~Hdv0Y(OL;aYpQ4@P?iJ-j1 z0`aeX2GJu^hbgNz%Z`~uSH5p{pkeqZk4F3;gyW0XsY6(BlISG+t7l;5yLTC+V_seK zr{j4=^Y+Qb_6rx8JzvY%Y00$c&lP$9oXm3!ZH&oC+VuaiVQ#k|d3q(s1vHvGIPzId z?*0NTJ+2r0Wh*-oruTcLLi_6+P^GI)Ds_8T0WFl3StcBBNq-%{Ofw`eBAobPoE=gQ zSxX~q{miWo7%JWO&PvU6pe&QVu~xp5j!KLF;A2=v@FWC?eN^nlsz<#|s-yGLFWJ@K z<<{tvHLE0o8<$3^{IRc^18a1X!b!cOZd)E{!{g)iY$2v7@q9=t*#XsxY7ODia4fwk znkwQr%dgWFEAs58eKXik(biq&e6n|<3_dypMulY01!NEsGMpQ4~_ZmGi$Y%&WcnbaW z?eGy^A!Ks+FHyM0+8Vvab9D5Pyql@t*(9hMGKj}`(HeP~Xg`C9aU7Itj+h(POuBu% z;JAD%BQ7qy`-gkcg*Y1-SPEX~DftuoP7Te%4z)#>XMNd@*4sa<>ALxt7Il>EOnR>+ z)ORH-T(>w>&;T+RjM|#K8#%xcUSb6a2&U#vI?ZJE{9craK{PwnI`JYan=G zp-F+2z-Wrt@TD~zeZAcErmEE={n>ch?3{5av$eTHjQ%! zS0IIoy1K2E_<$Wh_t~#QVP7jcoE>Xu%L{+?3Q!YMoxq4EaS4l~rHnzN@0UFE?y^{= zxheA-i;UEVrWP*aBs)77v4{7P97O2*HNUZyoJ!udInG&Kq@;b89vMH4+ZYj`TjBnz zIM@&ala`#WV0(v*WwPIwRrt*pFU=J`{5@JqIT_*Q2mnW$Jx_1;hhVVACPJY z!9CD%2(( z2jMnsx;esA@T>EuK4emoG{5`$ho&Ek$4>~!O6=Y_NK&wVW3w)0Btp&ck7j$3o7b=R z&|R~@m-#21P0wB`%OomX=NPFi0>SDzByAMan9-vQLu0+_*wQVwQ+u=e%DT#mr%cn( zp&t3Wam0a1Yh@(;_OF3tg_8sc(cc}38~|-7PhpnMM^|??K-XKn+IrI*;5=O;t8}dA zue>`7W3PFJtZo$Q^NV+7JX8tq%KGto?gUtN-3@;H&TV9(EFy&8a(GOsRB7RzaX@Bj zU5F%bkgDedvSA^tG%pP70fYYQ&o*=-Pr@Zm(+Fb$=lJA06OJ~|Gpvxd_e|*^c;t4K zh}jNodbkW(S(n2rCRH_-H1DEFAB>+E4*P8o*c^F!qHM`@RYv7es|=U!WH|}^mXNYkulHy^sR*T}JbIK@%w^;!Tmq4ONau zdBCUmhsRN{;OOC!;P1GX6P8TE)$GP*Ng~@zulZM;?(#iznDka0b|jMcP(-bwo&KyF zoqQR*Uh|(S{o~#!SeU-~z4)57>4-+kzVDdeuJPfOp1u>e0cdHXq<|(p07a(=0z}VnMbsLssvjR((thz z7HsHmKE%33k67))kZ=El2}=3EOml)6N@*IC1UlcYcMMP3AyudXsB z`}DV#fr?|Sl?T!e>LdiKwvq{Oo;4hsQVRxtEm8zZDesyT)~LVY&5v7rcPja9Xg#Y2 zdoMNLR;gvx+RY!JUt=gdTB^TbRplHAC%LEIJG>`n_f#%^YhJR*jWifW$}4W%>Q|Bc zlHyXyucBrT^2lvfeV3c@t<7WG_kR2Moz=6z33^6?XCT&yi8Nd}WI6lrxyDRuU?t@4M${9ICZ z1@q_aHt!(XHe)BF1;%K1uJHq&ohw%EOOEbwyq)^#w&sqJ6g1p5hh*f3n-ojf;uAOA z<<%ip2$d|Me13DE9t<>(Eh;P>hwDc|8T%<#RzeM?LRkV(dL9vaSs?ol#`B|GZ47Ey zrt^y>lH&;S;p>RvN^M*p7w@~_qcB+Wk^~g3I=&S0)|HrBliIB0DDVya5~#Jaou*}C@944P^7NgzQ+bTGtq+@<-RshPGH)l_U8{81(xm89RkyDBILz6NdZ7QAF0a z)(YzD8lh#e7Ozqu=Bca4GxF~(HcQ0m$n~NUqdu{9G+(7zBBiiP)mgr`ER_|=9A)_> zgkXHU4I-sAMrAx`HCbTiVz&iHN0){Y`zBsjDhWuvPZF&ZyudzE?=e9sxftE z{yC!2!*2(VNk(3*P@WgCU)D7Wf)$E93WOZ*ZGx`dg}2|Z$z>_o^f^B37}TzD0aT#g z!m3tW?>(KW;WVQfI1&X-IGDy=E1 zv~LLw{FDw)x|wNw5IVoGhy1T5;KoI9{)Ox969v&MkBgPIuq+gs{%WXV)X-sOvteh# zsK8B6%-ayvc%B_`TI0?%p5$#3&C27eyGBgDCG--j3Us2_s5Hi#D_n$ro7-mR+#aRG z!zEbmkn%;tC3ddHyIeFlwRmBp`gjgAnyla+fxlOsb|QsVJh5WFw`$%p$+K2{ zHGku?Kk@OcDsvT`J&|~BZmYt6{<%MfrF~#w5c8YoyVb@&>{z>RFDlr5<_qii)+bcF z6y%trdJ`5*ZVtqBx9(3yPe#A|9lZnf>YXR>U|bq6+0NM!rFpjnR8LU!=WJT#475;IiBxnvNdCfJ8C=`3_(DIx9OYs z^sJ}>tpM-~j_o@v=Lv|YbK76q)U~jKhuM+eHpi&174+gI<(e0DzB(Ig%IGZ%=;O{S z%_p?E&GVRvn+?8)8Z12c9HQL$c5IZeJlht>lT6EX{P$6eH)iFTHR-vjnLWG--T9u* z3L&dZ)9>r<-^Wx1)H+;VvAeQz`{f%h6P?c7wV5S2D}RWmUHy1T?OGI!60?rDMVrU{ znShx5=Xe|SzjXqds_hQG1Krn!ZWDIvFN#zgEW(9mZ6Cj2MxL+{P-(ta zvXGQMIWrke{iwi|j#yREh20FPqIBNnGx=vyQ^Th3l*7?|lz63Tbz{RSH6jv9;4G5GCN@J3&CL z;qhbo6rP@B-UdczQ(wR8Z=m_>zIAU~>I~{JbpJC>&@!32HSds`YsL~W1N7S7yDRB; zc%uq;2Ht;1qnF(|^%Eh`(hr>wSAQt;?+LK$ok;J*x z1-$?9^W*1d|9>Q|-A|-HmeZGJv9B(&;H)p6mD3V>me4ZYQ={m;qGj*2=f{4hM*sE? zmQc&>6do%cD+{KxvSrA`L*gOr{RREkaN-Jb)bmNFxTZnc((tQBA;*5lgq%R?cW34Q zxP+3^W@@>&Tk$`&@R0tiQf~ulr4}e31Ju6JVMJ?MJ)j;?Y5nhw{_jgF5C4K(9Plp! o(Gh3&{(s!(|Nm1gKxjY*NR`k?4ZxtDZB29Uj_K`+e;i}}7n3&`7ytkO literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner21.png b/docs/images/LogMiner/LogMiner21.png new file mode 100644 index 0000000000000000000000000000000000000000..e394b1cac006607dfe337aa52d44cfbc39098d9b GIT binary patch literal 217935 zcmd?QXHb(}+xIISrKu=Y2nvdYCLo;vBA}w-vLaFy5Rj_004Y)wl`c}0s)!L$x^xId z=@P1RlF&j=A&`WGWG7ztGtYi!-u+>J+Vehp7L%ET!k#nzxS0GZ!%S;A2sBFED&sqJy(HL(D?Pp7~>+2rkJ)N%rYoim;yGn{ec5e%^UJ z(_7F*k z;-rmN%;ibR1(c%3BrG5mE5$5YOl3U)=#4ga5)gZcJuzC#d6xbU#zB8~#E{7g6PgXn zWPzuXfg2&2HzHW59r#zcTzei}M&h2_bdQH-p=NZ&*g`ET&Rp)toW)V3F z+v%Zd`~v=%CW9u~6vBl~fZi+T8Mk`JsOj*td#+TXoplm6?|ySf4dd1dpgjro|H)T> zC*y~rSmDD`K>7q~))=Rcvg78le;3{0)ND^o?5;@b-)_6kjODp8<~{ZuJj@(s#zI_* z0gmd^$7g#=GB)ovcEfjB#{uK3R%1e;FDYc!1haIkw0iD3D{Bw^->!v(ld zlDURG?OUtd$EHG`gv~z;RU^?>pT$c4(3d;u6&LF5w`vG_yp?IF{}~sN{B1%T8Jtls-_W%6d_Wy39+phqOQwH`&|2Sgq4pORhUANzFocE@LZHz}I z?E~1x==b5tfj*9TeQ4s-Wt*3T45H=R|Gwji zsCTZ?(u3`dyulf19B))X{WfUFc0V$+?u%^q)1(6Q4*Ek)|NDNnzfq4v$>gwoTPfD@ zF2}e1pZ9TU?y<`+BTb{mWOt2++W~d&cGv@`0d?7JAxCD-2&~^7X#LbyNzQ=NGBW6o z22^P|AJ>)K!$PYV-6=jk5mvoH5hVYQ29sRyEL%898U7Se@oU420xpjL>Q;M&jb16! z%5j|_M;ao;`b^tW;Qsou9XXW``oi;3D z=XS2?kXyr5Y0IX0#g~{(%C)4L!L=WUOo~N2bX097wuf@SwzeL_)x*^_HJpnJEA_Sh zz3&yCVS&$L4jE!q`JPXX%Sp=BP(Ce%(`H`AZMn(Z+AV*AP%g1WXK^LXM)BV?P_D!g zLupz~(-tG1l=AD9O%-1F-XD}ITovy3GLKfWhH_G%M&^Ib$qLWdB9H3n`wo{oEjkm7 z&OiQeX=%@}$xn%9C5;!3(+c!`5{)H;QM~V|L4w=bAMV#y!a*k zf&iJ3G5TTI(L{3Ozjt~KP^$*;G#i||XrN}`N4_!;CN#7~+7vYIDu<^=|F(BQA9_0D zWF#cz7M+66F=O388_tFCA6`1KlSpXj-dZ9xNMFuO?Oc0r)c*n&c!6x^T-p3tjD6Ju zfZBIF`z5}r_7u(s8e(i8RDRb^y;7eP=rjxL`^~g3Gd=hr{7Rl-^yW&q)#vp-rkhlB z^k}!-nW+Wi&dL=Y-53APtAm!=yM(O{&+JP1N5iKOjtCO^U`Vf-Wa@Td=p>VPlysEM1I5^*lx#Z7W5-rV=oe9Y}4sb4( zi}$S$36wTEl~C7}0}KeHx1-$xZ*8 zY*qWys~bx9CQFT{4khDD%z0Rc0K`KLHQ~iIlh7p(*wVzl=n^j5FamP77+Ft%Wz~db zJ%4n=r;52+%6y!{+nqO>IjsK1$okLbcENSJ@PT+}#pZ&h={YEo> z&)7HqvR~f}MbcGW5hIx^Ixl3bjp3uV(9wyxiCS+b!bxY+$=-Vj>E(<{b}A=%Pj4`# z-ee^e=tcRC-u>QtqUuTBx7-Nitll0Lz{ED_f}?fEo5tgs#y8Q-DR@J~#u#j4rU6NB zz~?f_2=)3%#BgQq_BZ!_B4>()_Bvqm$T~q<*Ty@eC2nqUMdTAv^Yo>2>-Q$q8Exw8 zzy))l8|A4wL{bB(!PGF%(Zt#Mjc}QPlXw818Jc zRp)hn9T#|LASn2v&dL<~Q+ODARUf4k(wsFb1hN6U9{~4*TIS1>pz9ZJ_2>jFc$}*Z zeUa5aSv9qlV}>i5&uOpmFwS#A*~(uYSdlUk9^j9On4XvK#Z2bkMSP0Z(P&(2XkObi zVZ^e^=-DV*cIB5!NE}#$0Ne3^hfw^PzmXGrI2I!>RGmdqM_yV_D>xeO!INPo?H^5C zYwTZBk;^yRm%~pa=pD)CY;*Kp1Dt|SH)GsL3LnvYmalHWY{t`@g;X_V7P^lI$3|y7 zyJ7&cGM>4C5Md9tZ(!T7E3GBWHJeyv8ikX9bo_%d*&8-#qT^Y6NcDLJe6N%hOyN91 zUrE4q^ml}09UzR>JsD%Po~Siip*N!FDz2sH175l&am#fOKa#z}ipt>O$IeRT#`2e^v;EbNGuBW^Ew%YWV(uS*5IP zq!Sfm)qHb{i=0cJ#;F;2C~HP+*3`XhB=G^ep6O;`u+lbXcE!R6qSNaHWqcmXo9on9 zm>Z?N6T5)=(C~|3h}<$%ub7M}@*5wolm%@mgk$JkRzT7GUnTp5)=cQ44Sz8ljC)zpdpk56UBMlvz&A~oGOJA-H9NErk z%|j$+to7DdE3_(%#ncbL47j;(Px<=!dP36jI?y5AKa)~TC zZEV2*k9Kf4Ocn#%#KHWemqnCLNu%ja=m?#FXBRTHGlh=4Tt`ylkkknUX0S6QSV*n{ z<6G7?)^g?{OWcGg-n+5Tmc(36Qj31O|A5`+N~e-aPy~Ch*IfgO=^%2sF2kTz%gdLz zrRMeY+`0F640O!1Z5=QxtyS3Z4~26e$HP`KU@JZlSScmE)P0ayl($>tF|Qkr-U5~*)uyLYB0;^rT3{)to&o&v*%cA%FMT?wA= zV_Q?&B35tZ)sH*mTguD$EZ9L)g-mu0p)P}^+3HZNBn+#m%d>S;efy@h2A{lRw~Jo~ z@Mds6jCy$LU0zWGfYAzYSWjJd+qy;>i)b3N=hm6H{ccz}N)J>}rkt~&tG*5KsGSJq zKfHM?80tY_dy2(f5G*^KT#|y z*yp$DtHh@~k)r4=sh=9uX%>^>j2Y0{*UIG_vv(*7{8Ljw<{K}dnH_U>eeyLtkA~?c zw`+B!6r0_Wx0CTfv^$lw|KIljfy)&QbF5k}qiVfK(#Ykb{N|cYNDYuN=ffua#!82( zimJZhJKPZ#NgO|x|NMg8fDePkHrpNd-rrk^X={Bv6C{3J@uShNr)_0`TSvL(VTqo541(E#y?L&uERs< z`XWXqOEo;hJ)rj(tJ~K0qyT%VP3rQ?+P;`o9teoL&^D>KS8?C`o5oX_s=*vvEcql&__`UxiG#L~AhzJTn_IZ6n= zZ@BL_!7Z@D_K(&?3y=^g<^a|+Gp+IYZH39oDm0NHwA4jBa;}05V_ktut|2;QT`ZYW zqD({5?w6!VXqz&m&CVr?H6WLrFAifIZ(4>Iqu2F#_Se9S(W@|{?FJb@Gs!dxG!}Yb zU2yq}oH*oNM%cT;cGiH{-hg}AKm`R+`%J1H9Tb?8%`}yLNtJJLe8X_Ka=x0y9HADNdOP`)TZrjvL4gooL<7yP|)VxvgnOrZ56lS@n{D>e z(w{taa|<3MKFc*y=6Ty&Q+tu^N=z@p0-w797ViLOyD#(0Dyi=csW)NTPsY;3GH>X# z3L$uS)jn>FDC;k~Dd#YK;xY^Z=+{G(^;LI>*X{-o&5-_=sQ*-t@u)C;nfY?YI%cUp zAf+z?XnzglO{8C`=Cf|Gt_0n2I5ifoSD8VRrR7+*DAr-?yFdFGol|yIb{ctAA;xGC zb6CX=vy%)|P~;fnCN#`aR*u?z(V6s>XA8L9js65tc5cm>Ey&9cEk>_``49W;g*T7x zfmcH>03mF1JL;h^ct$opXo;U_uu;m|Ep_HUA=gZIoLzKG@1Gva40hr-$f+LLM>?!d zJZz2Fgz8pt(&MMJjeGfAd?H4l2=wufuCHg4wEoGwCv_PUohsu3iRo%N@$z_^pG)A0 z50QQQ9?nI*5WD~jx!twsXq75eW2AY`Dq?n+pL3r4n==*DOy_Z5Y~w_QE-ZHV33a>h z9#&CGC{Sf2lqXO+9=g_|1L6z0_e%sb#m(J4tBEdLvVg^z+47o$KT-R6NG+9>_@h@` z8!bNl`Y{uVUzvZUTe}SQgQb8lEd~Ct`37zb=2V11`x8pT5$M)yF(yXLX-U5Mw+l1> zn5qF*h+jpvx-}~dQT57CUVq=4bl6-PcPzQ>*wz8>8fZS#q_xaA1 z8;v6g3f#Lo_89G&8@d5GBe(N_n3qYco)68Ctc!BPJm389-TP7*7N|j9M!oPSo${|3 zuo6!Usxp>6qLc!FlQOdg^ODMr&lL9bp}H(247$yGL(gPC-M-4>OlVRqbvDB8x4G9` z9+`{xC2xr61$0|%8utEvq?H_iE5aMbIolYkcOpuSlJj{gGRKa~1_Un~JYYTT{s?7g zB!MQmoti!CuJXsdz*(8>4)(gt2bra~^C*IA<|F6b+DwO2^dSS`?d+71@6TAu;3e5a z&>IN9{tYSxoJ6^YF-Y0hzi$q$iFHrb)?Z2YP)L(`EBuDPZskkH{FmWG_V~MMwdD{@ zBVSti5cs}AYwav}W8#ci+|3$`?=H{8_U&fGb-=UsF9~$`ld*c$ejirG>y2ujG$b*g z3s$fNQqRXRi3BvTj8YDum-jN-q&8e5A&9z5arZmt6X-da+BQU@g))F4+eVJI5+qQ5GD=qj?K-M)VDlq3>a_`Dl$SXNiJfQlTAY^E_b`C&282^ z*R&t7T^)TfIb8dN60UqK%V^=&qbG^!zYJj9(krLZl1iS5N{^QN-^5Pn3aQLN=^rq^ zUzYY08PCM1&n!Os{1%N;SdahQk}!hY2+B$=k8W_`H@Ome2IiJ};CB9Z=HO=`2f(bh z)hF+#O+@zTX)ZA8P|zD^XlqKwp5OM{P4;5BZa3A`qpt7y$-^7i?lIA>v($X7Rr2yd z)tv*8T-O(-p0zx=c2@Xw^Z3-h50ztHvOcPR(ZoSgtKQud2oI%KV_Zq7o#YV z@;`uRI*7q8QVBepb)N$8XS(ALlEPF9~b-Em1g_lJ0-a@|FRx zH{$zP`1i#-`{u(nug)yV>70(IFx#W~l=W2%@Zn<(YO*re7JshC{R(-BL^846ZECEJ zv3EqI3ZfyC!^w`FU|D0BSWui}0Cma^zkOg4Q*j2Wis@Dhjz}9(MHTb7OoQZG0Na&Y zeu;|<9_X?Pa0vI2!~_XIT1={158t!5NoG9gPyci_$71GR1wa(Jvcr`_buIv0LrcGX z6u2ub-~4W0t^@70N0>y-ii_V1fb{S7p_%DEOHDS#=>lqN|SsSnV- z;+ne9((aglJa+tZ*~AzvadcY;g0_BYESj=BpK+i7ifC{O>J9x zUKc~upDH+0s&8pE+r)IC%zPZqZp&`(hscT#YDk*SkC<|YFAuaEb)A!NJB$g4$-6$m z1*hFKi_vj98!9sld9FI7>xji*b!9;IbBiO&j3Be&Gosq#H)bGPy6`Q{vBN%x95i&r z6-NM2 z4}TGzYuHpt{QiEZ(%?P2s24Kx6$ack_+7*Ccu~72$`jET1r+9UK`6&1aoxj8lMqh53QkYXr0z(*`aE9h(cxNc2^I}2i zzIl%_3(C3XxpNZwuU!lu?zh$8G1lxrnQn1ToxRV7l{^iWy+jQ*bT@SWt*XjS3B*Uv ze{}U$Fgu8rdc1zAfEHP2h9Zga9rpQ0aGeeYga@d;+-SCE7@Qf<#pYACJz_RGzU~$OyhTUGvRst`ivh+EO zb%7^7N%c*tV}Tj}mg)9o7OA*+{l*LxW>yhjDku0zP=hM@j!$w38Qu{$^EUN^tTD-Y z1`=E1$91>}F8!K)J_E_4YDkW9!%x_`!bW{_8ar1A`OzG~&J}XL|ANSc@H8bl#`?7! zdVY6P>YZSsmhf;#PhX+zU zG_>yNZF-mzK3F1=WNTy)ws315%|)}l7BCQvUJ;1oCFw3pyR-} z-)io3InZTYR#MKMhkdACEjd4O!p3R-^rUwrui->TC_-@ZRn5KXXFt|_AFQBhA7a&{ z6M8}NlPe)ZH{|s{Er}8(vo+M~IJGcLYP$pPcaTuF%w$yfR1#eL zBZWd-#I{Z-pY8JZvy>0>YXod%cB%+Hse+oDC0|Uol!ojY=G@;m&#ByKW?ak@={mq$ zap}KUfF%$tusy|+28!#^)m7xF2cM>>=!wfAM0w>gdcPm`du??!CYA30lD`;ypN8pk zoDdU%U2L}B138EHo3?#-M%UDv*zA>b#Bz*7QMbr5dE~>|%iwIOqp*_uNr_q7jyADw8pzU3cny~v|+Ge@UJ}>{t=K#DqMZ&tFWKA^kC!OXQNz?NW~Ek9$q(EE z1<<}1D5 z1@4Bf=0Kika}xH_<;Wx2UG0{@PZOWtq!zA}I%t?XM(h^}-rMhVR3p!m;U6jg{T9!DYDj%+s9@JsfU-K@0TmeWp+MHjCB21^ z*EgNEj^gR4JLrvN<3;T`W)7Wn^mx52Lt)EMU5gJ)<@7rqg-C{=1+3>lqDxJBf*@pMcr> zJKE62WeZl(qL80wxLFfCD7WY&SV+9O1MBSY^iZeT%IeP9_&R{}fEpO{PGrDctc*I_ zqnlGXC9E_(84Nu~Cys19hEjTX_8Ux{zwd;Y-S;_l=*(KSNWh6d{D%ig1|bMx;e(0f z$W*ioM)$q}U5s8PW zP*|+FrRJ3x&;`AsXX#H_$a+z*0ddrTScR{`|64}b)B|`L`Snf7TrTqzL+D2mZ5?&) zo^uvbBB!)TBuN051b&PA{G;6w0nM*_aXa9{Zy~D*Jliv@Z?dHcwLreOJ7D0SMIT^aB*Mt@}*Zr4L0JDj!fNIf7| z335>OG5s2~x8tL6{_?BZwQ>5#GIc%KVNdYFwl%9HdfzlXsphd*Qrr{|Oe& zK$rMqe}hHH^ynQ|ccH7U<S`DVrlV*3V=Ej3Y-9%Yr zj?1M~qk!qbIGuOuiped0enG^oyu*yKm?iNcH_~JGNN$oFpwutZ^_}t=8=FA+r>9JS zPHX$DDM8|z{$V;Q96%(vZ;?QZfR_tc)H`?FrgzuOvn9<(ihjaqIJIl zX5yKfcwof#k>7rS=2_R5f>xgM0|~r`TfSP=U!LKM<2})}QSfT?gUC4zH2FHZDxy`$ zschM={Xn*uZ>rpe&bwvnexDT4n#WgBtN!dTjRtPt-7YE``;N29kh#1(#XEe%&)Di+ zu)$i;ztYyQzz#J3xn-PaQNfOcJiSbJ5-$<6@Z(u<#3>K=Shvp+T>FZq+J@);L``=d z6PxjI8!mW$K3C#9%&?v{-U$o|8*M?BC#p$7YPv!%@5mO$%g;>Qde$v)XiriN!JpmY za%<7Y)l9wVMo{gYMX5a>VWoxWny<6BO7;7%1I9PmdT)Rbz&cChL4Y2w+um0wev=d+69e2jVaXZWLCck&j8mvp25 z)m*1FQ{QBLXuU3tYxyh3e_!B{kR_L6Jh4x-_zr&+sz6wW+27RTi!Xjx0` z^kcQjBoOY`T1ajzkglB~_fDOdPdJc@`ed_P{=5Nim5i=*;l2mX^}pTZceIXrkIAl# z1y0xBVT7+fscSak+0RzH1?K-tlx9mrj%;nd+TsKG(SD^W9={t`{Xz}V_94t+yM3(- zNBrWj>rXw09^Vj_&$Trld_vmiPwOs#lT;jrcSTt zMB{MVlQ4=vzrrqX1`uHJkOVRc#{ zaUkAfS<4A#f+40h^%n#qfyZ=&I@{`SKIe|Q9dsaMe^2TT)q0Ci%;`E8>|uXqrf{Zp zMtFCoPjksTh^=GS5t`mX!+K!BOhfi8lfaItofD@^OKyOs*n^|LRZ?4O5BgxARPr9~ z@_zDuT6Lh;@U8$xm4UTipgSHsalhw}p9c7+m>-(|pi8n?N2PsoyWH$~nje`IbmVlG zhTL~po-xZvL%`>Dg>rG2eAJ{ey&J2ba0|Sa8V{k?|{mMWSO=4~C)%SXpPi+25 zPV6ioVTngVMxr%w&i+VsC8N&+7>H{gT*x8y>1ab;8pJ$@*G~rAQuv?%av#o}+R*pC z;2V5yw}lOLJZELEU;nSGkRyfY&n%wvJ+E@b-Bdn2T{HK|YgJp4CsYD3P?94kJr%Uk z_^d7Gm&7g9#3n*48~-1IqvAj3BFB;KUYqwyy4_qkiC<$lfzNa9uc@&qh||td{PFX#f_bC zt&#S(Pj)127kMUiCx3XO79EeGu7vJxu_2ny|NU+gloC7A zuK_ht_EJHL@hgzvl|MzeiANrexpV*ie#u;;+_?bp(2#roDT@7oZ~n0w z_6L_!3H*TvDUJL=s88!dAN-+zqd@<6JLmzxGx7nVqT2#hxQGe@nIgod9f0{zp#zV5 zhlpYPNBm0avwi*ljNPHNII=}Q#5$-ckNQ)2Gk0&@1J5pJGF?7wFDRRYQAQBJ;IMT0 zdYeV4FzX=R&B${8EvXy+hs{3f{i>I$ztXK#peDCqO+2ga7 zd#D}cXi|x!-!6KcB}!}A-W|Q$Xx`|-Vl%7XVL?p+HT5jJwFdQ*$pkXO1WIYd8T(VF zallYY6~fq;5~FPNVE+lDl`A&)Gk>l(lEcrPHZl^O!4Z#beGF_`IX-)fWydNg&7nY= zV3M@`N)?0(joxB~BUjiVaaV>)IB4YIdhu8fj@R^nKDDsw%$rNjy1=?!X zXZG#rb|P;=HT8}J>t?HaU`hcA^idf%m*rO%Wn zMHX*28`{uK^ppRwFj})i-3Ogq7weBb`vjnKKjR&US*fVMz>~4!jTBT+i=z_K0yP@p$I{pWb5M ztG;+CiYoRVf}$2pp!v+XciqFVU+2Y0A^Fc zO3fbvI8?BRifGhdDmaz3&WR&c0bvpLxVV;r=&Wlo`sgBA&X7Z<994W1*p%{z zi(FtaD3IBh%i|Gm0$o36s))E{esbIrazv_0QsU4RGDSxlY4e*ViD;Ht9t!NhCCUFEK=$ zk=PH+BZda7oS#Dk5?ZOoz~Y8OHg#xvg@qN2Ko3yt@8a=u4>)_SG$=+kFB-6A)GJn_ zJ@M9q;!>Cqj185yA=5TaS2nsBj{yWy<)W`_4&6ZXzJU+b1>RT4`v_%rAgifT)PRay zG%Zk@D21J~bMn%DEP%CcmtH(j+DPp}j~$mPB7wp3E4;c(${7`ep481@6&o43GquaH z;_FoV*~ZU&oKbxb6S7|^K59fdDJ*ww+GwmOykz)TudLD^&y8VutmULD6WDJ7Q?{>g zl9-9>3TP^uC^#Y){;54Ix;IRG&z#{OME`V7Hu*>Rr&}R3I+8`V_*JgbG-S*mS)W7B zYD>4FS&mG(v-#f>4I{NU)@p zunPQ^Sq5{^#xiKA+z8p5nplNtShPO3l?}CekNGI11JO_fF&0=e=)IhU_z zD;WEQ#+J1HGNf*K_Q-lAmQC9GKF+fWNK4wKWcw&WgBYTx3`l>#Zvh~3kCp3sMh9aV z6?ET)Gtk@|NbS{X+3M-FMnO#=#omdAT^`rPg}9of1mMPHoQ9u zMroc7UBC~jb-y<%$WebR8}d9AS{Yh(juNpD=Mg7|QiRoZ4h6Innun=&^#HuOWw(r# z?P-dAVFn+N&Uj+7v%TC#ZZ>heCRmTk$D=$k#MVZ#AZZ2Ssygz)`%l>ws(v#@V))eW zy&e!SoI5M!Zk@Pbe=xS$Dy-s8h|>YJFsr7hJDXp9Mm{vIAM>d#q}@kyR%v;&-EkpQ z`%Q*wxTVM2qwCFbR#ORO-TM%X1&>asK`VQbFySi6Cd3OUS{XXluaVR{d69wLz>NxE zJ~W=ICC!yKHcnnFQ;avNIu~7Fl$`>qPi$;eo)Tp1)NjK+afL33i%NxRx+8iD3q%Kn zn}gWLkzsDmpi9(}OfL#k3=0je(_B5OS+X#>X=b1t#Hio8mDg6rL6b}mzom(AuGrkF zcbc?K$XM@2=FK_F=Y-Fml{}`?tyU;mSeb^lz;DKyi>mk?>NF^b!jnEtm>>A888o+2XJK5Mf)^A|DnC}@9kXTF zOtQ*dL>^pwt|POOfO28XqGVUBlqcpce9ooIHKMzPLm~3mdAU8eH;LNOz5SR)i%=Dm zWQP$-F=?T3R-5TfC4OwD9&3bX)>#=xi-7LaE_{q=MCGsRghoFm={G z8*M*DHrjy`-liq$ksL^mJFPnuJT$h0VbSUUqiLA36=YY=(JPF;alWbLr^nB&W7o9L zCLz(=_O7$dFgmh8YuGE2GoV7#evvU!Ys;&bdeS9!&nIX; zlbyO$5l=Nv(u|aSU?o4HUN?UOMxJX>7XvD85Zcd)pz)aP6l#3MM`v1jg{KGZMl7i% zo@|vJbnL+*)?`je%%Q9``bokfJ#`+_3buq-J;%8WTos_<29s>bZ@dn=sNq=_C76p#0~;m`#F1DwS`5xpl#GZoBDIkx!=`XOzd8Yb_au-x+wv#MwHv$T zxw?A>hn+gd`YLxj#Huie@tIxWwU9@VAI06El#3*W)PLz4WX!XZPpt@C>3ob}(v8GZ zmMA!LE2I7u-gAo7Z*2v&%sy{2VF?NAUX*T(V4W4)SdNFZ3`a8N`I2&b+Rl}tF<5s5 z3HwYgEPC58(`2{6#K2XzC)N06Y_e;sXQhG9-((`R8BflSPK19nfqhGBM-bC;Vhxu{f))yDP{D`;NVPLUcz8N zTBqWDtsDuxQEr;+Mj`p&4CI1C8J!vSjU~HhV&Ey|Vesm6f~|y@MnEUTotmKCy5a0f z=VIgWeLqSmf`%k+T&S)?6}hm4`eGQr5V!foTx0OQ;BS+EeVu?x09%u548G^;p^S=I zjRr19hp^#q^&0x3GjmbvP_$|Rh`=ELLrbZzOKT%IEa+<;n@6i7gI`li#Elh0+IZdv zw{W@+8zLq$pc6AJMjmU6_jL{6tf3pgVWkx`6bi$II<4YjVau@+kqh`Sl*E+@%cJ_23&+&Q|Kv{k|(U7Z7*_C}qdcKaf`1&m6uIaX`` zI0b#&Qw-wO7#nx@-{fdU9uy8SXWioN z5WodONSYbp1X#2jG%aU_&AfGPFgsaPK!Yv#x$Ss4okyPV%&}Q|a=Dj8mURRF9)tY; zm4R(y1Kapb4yJ0Z|Ni5dfGt1#j-Qa~)7r0D;Lj)&yV+9Rs7pCNQ@yoF?ro22MgZjj zuZ_uiJJ5%6zjXV)HS!{h&9I3%au*Y0Zy_ae^4KTJ*{05j#?H+&r+M7mgY{JRcGf^LpYXO zPv?Mqk}J-ZM&a;oHqgB(f+mQBatL}!=@sRH%kuT{Ay@st=xs;zkH4)G7P?GAeJQ=( zdS==!02?U=!E6HAy$(Y!HbZI}HfFJMvk&ZI+Pww`9s&T2>y7_*B5+Vs zA>{6B6r&F{yRERn?`tz!ttoHA@~=q}mSI}~~| z=;MuWqNHmwaa*QTW(kD{g0=xK)Z`8zsAd!NCqaHY{0W zLaYhIE6~$BVpA;)1@6#F2bvi|Ip{PWx^Wzw0m5usa16IGf)Wa+xI29=g(4{t$X{xt zaok^cjjU)wQbg^pE>udB_vaDz8$xr#v|$SoKNaQ6m(v^y!AZWx?>r1tHQ1xfKXU41 zvWDiU{sp_g%GFF08(^puEe81`+n@VaeA?1_7wsIYJLD9>KlO}*@od#~6{7VZz#^Y21)F5+)bb6Pv5y*k+qBBgq_u850k==oJAS?vz(XR-E>EJ_8nT4M>YD`RV* zT+rwlkJ^r$oNSlhDJv|wt}$47&#%?@BR~Xj&RAH^NsD<^G)XmMX8neL?d&ZHm$uSZ zs?rm}B0?1K<@w@0D7#T(CrUs1hP4{D zU(Yyyz&Du1?@!>5Gsd=Dkgf^TcOt5x{FPtvmVb2p0Q>_lle|92Bm!8(U!r(22v0tt zy^UVb4w<_nx4tC@FQCYwiE={5uq`__2Q9>8w^TUiU{eoGzYM)mr_87_m5|!IOkb

;u3qj6R? zUospMTNxvasCB=C6z+N%krgon&AV;Gh&@hRw#$m%lFlPaTbErp{hIJ!EC81;X*ucY zS|wHn&}ogP?6Pr8^^dmD3$uS)X?QQK0p8>8h$((&A6WjgYpwSyEeBky&f}B;hfUUu zT1O(U!q+=tJ}X8W_v==FwrmTR^!Giyi}1>Dpj|%qdGWdbKWD|RRb9IoT~olxkH|bU}9S0ZUQaWaqv}hqi6HMITkCy%2vah z$08_NcVC76RKpPb(|%?B4hUiJ)@a8E?G`pZY8cO5*AQkM3^aAklyO>SZuu-o$>MdU7;%O$BMsQhL76~ zH3a=69&@0sNXZrPO`o>WJ9T`aJTt14P3W=z3fZA0tJZGa@I;Q2;*2ZG^F8mo@d@l0 z3bV(Fz`df@LS{nEYK!a+`mOZ+@4M5V`vn*D#q?de`mXzZoKFF4@Xe5wVVUXYVhfku zb5>dE5KiBi<{;11++7!BXKW3CS z+mW0W()c#Z!SJN1e;gW0YmLEOiW=+ha!pw!3xyNr#F+(vz2!o`Dlfy$`XbP(C?rDjQ@*XjO^yDgL)-CH6#KTut~~8Y8b};l zc2r6}ICGP1IqBq;p9TD&>80Eg|D*c`;Gg?IYA)a7@o030?vO~0!3~{KugJCXeR)f+ z1^#R1)4Lw?vEoi3DLvIi-t5~fvJjFWB(%9)uxsLb4O#JnrKk(j`9b(gh|O?v1U0DO zBzNk&o|>bYP5M^)T1G zRfcJlkf=fpY#Fi?MrllIm2{nh=Bs~zYBKV8T9 zXi#mMRm59;U|PzVMsUc952608kJ?kxafy5piHTJnwG!`6ZAd7UP;c87_H=jzY=Uid zCQpP_w0DgKHb%88jw>6lT=7Vy69y7J!dly}r>7+1N~j3|L=bat_Y43@O#FOXp|!=| zY$Oc={wST3!aN;|3p!&WvsiG$334_1kdeld!Bf#c2#>j!$uYlPUP(z6as)>ufoQlw z6l6(D5@NMvXepWqS!TDvQSMQp9Ck2D{<^Y~ie8QsS4J+aPUq|QU7ZeCQ+%+;;a7lwrQGzQ_SJdc4@F5&&mNy39xt}2=vK4ibi9NQP62fG zIQ%|p)H;?Yj=tHCM*z2rgyf;ug0Qt|hA(;(S!?V* zH;Y6vq1{{B^8jxh;a16}$`0+7A`=`BPGqH4!m672`ed61WmTC9o){S-Z^pI$!95ce zj-}$>!>6>YQmZIcmZY5N;`5}3r_`qn+1(1CP;weQlqYdbiYK{M<(eOTYXi;P5P};% z=T--|zCsB$u#O4m(pq6>$IUGx(dB&?52&LJzSx+Wz>_59M6|FEje zf3k8l15VCRL(ZGLLi=F|TTXY*^?&xKJ;uTonv7chIKCwwdBB#bHk!1? z%6=?6^=3?@?Oa);^06YFPa*~@rj|4}oy}!?!m#og>$_(643=5jHHs$GL8q9;jimA$ zXpe82Or_{2H(1wREc8CQNlI}>P98-roPmG-H=~A?UGtpK5sGn33TZm;#G-78D)45F?__GzrdmM@k5V`el^`C zX>pBiyC2neKNj{~8uXPM5WQCs_1#<)MISg-PEB<05do1@eXfx#vYK4~=A9nm4W7ee z53@iLbsV{*8IuVaZm?FG1le{HuZ=g^sD_(31yn>YGjWi9oQmN?9}9SAc;|fo!ZWQ+ z&p~C{K!>c{E;vc%+m{Bp`4XSxz7X+IEIV(<sIyUsF6%?D>Bm1gdl&xj{0(jf7?e|C!UMBwPyE8VNiLU!c3iA#{wJ1J2C>?KnQ*tVv(dAwdvb@cY5T2a* z=-qk^ItKnw+%GpOc`b-&2yOji%6%+oI;6**P|OoH(d+?enLo!ENrQ#3um{pNodal@ z`^$4!YBa$14Zx4_HBS3RFR&6J>?`WHBP027GTud0zoBdH>HTLWMFM7*AR{UJoA;N7 zL>IYebzN02LvDW@rLQ- z)^P4lRUusbc2MNnB%r;c`kV9EpXwbrahSD&=JlH|R59rs&iriLFt=aK1ThkTSn}GnkbO*-ltN6sUGty6(6-WUIi>&OQR8_Nka7{x)cJ9Vep|uwq^a!hJI18U z{}lS{RRS7#4+psf=NssXYv;H{Uy@s7`@q`5;aiVBdcFB9t{qmTknB$xzT=Tp@~i6A zZ7yyhIN0EBD*co-yP?jx{2C~&AfcrDtdWpvRadMnBn>Mx`|xew#{J0efjd4#E5EpH zw@bqxr(6oIy;iU1R}(i*nd7%2%X8Syis3W}rpu-6y&RtqX6S}=Eo`{dwRIxhXrWDa zmM|m&qz|`0Pg3k3ylU%Cejub1iMmS;f0Y4#b@CtAB98$L zO+gUp#n1yPMFoPLUX-fz9!i3Mh)C}sB}g$UO7A5QdO%7*S`ZSN5NZf5^b?-_?y=AN zKVyGAV}61GA!N-p*DUw%x~wrHu|*6CZ_g`_-xMI2N#4iwftc7Ov?{;E;G53__T=)%yIO|FSd&4I z#+CwP$88Ot_OTR;TItf zw)!TPkwsXz1aBQg4{2ula&n2yZn>2~cTg)ZVW1!Smb=AuOyP>IMQ zY{qdb(b#%|*3t6)lM0+ro`nlm{K_+*t(r}g>rsdK$3(Z|;*xF%`e6(w?q&ciNC|3l zSp~vW-|Ya8EEz-RMd$`R(9IDl`9IVDeigS{`uFV>4m)>#{i00hUD&sBh1~!wrTiXT0R@ef2eX zO|w~QQIcR(w70te2v@y~b9@~o`v781MNpLv5xPmr{Ouk?((m%6JV zu^*}%juzQi=>qZ=-&lX-KX3<{7lKXi8%9Y5%sN?nD_r3qU;VStP(aH}`O3iw>%tpc zH}#H0wcQ)$^xGGqi#6>i+#+Zm308XM(OJ>!WSVO8YxjtMbn;(L8L#djbc8+B&HD z^e-T5WY_dtB>;*Yg^&% z9%&pwNUgV*6Z77pj;%tXQ--NcAnz%{jWD zvGHf!YKiPuZ+5Psn+N3}Gk2pTR|n~Pi~HG7&h~SuQfh^`z>(?FEk(53~%a-#hVP;%dXeK3_^rR4h@T?^x4 z!+JcgC?)EqA z9%{%y71@^+V~Bs*ac|*0EMf5ya5@@09LO#Yt6T@Y+O7e1DkD6JkdjQe@cQSRow zr{58|PBsSh3YKs047;$0r0s5eO!IZ(#o~De=;bFKYrMXYYeas|fi?U>p>D+g=S1${ z+TPlw`z_tiMSrx7Y-S8DZ6}*KE*VSe_}I@0zPw~Fn``4z{>wkuHo59VqWUH8Ld-uf7v_HG6S7omv1t<9WLCa5Bb*?D}}g*jFJDe;VbNHGR&p zEIHaYkFAHC7Z;1}E@*6==lWqy=eZ}_(IWv#F`OKbLzd zdS|a_xEDWOwC-)|TGvpfEPxHFjksa-Tunznsf;S|dLp-lvo&?0RqZ4}7@)O2pg~{T zVd*NmU+fJyR{;DYdz0loeEFdn<5Tyr3nSt1XBvfBQ-d#AS=sNrBMh!&GJeF#X)Z|qAPHS zOImhF8Gay0ZzuzI$cOGt7MUO1IsWXQT;%Jz6&bWN7zM{Q!jpsk26|V?J{8Q7oG3U^ zf>RejXO%4DHJY$t>RmA?J2EkeEo05|;*;jOo1McD;S@}MmDWCA$HgpmaTVOioW_ve zKh%C%P1zhGoV)NmqL{RjyO8E^oyhZZZ_&GFy3S{0U(aB-+l@&{O$E4mApr0XJZ}3K z6e)j^dQ99)WXgu?K3~w4Pxlsk4$tXVRo(umXHDVT|X}7S7Y@~5Ax0$%w4^Ctsy1} z7WLBK#;iM*;@e0qnC8wHPl2yFCR&m7T7I?Ps9l(;)>y7&9-z?Na*2gGG3EE?Ozu<* z6|fTlu&4{Q`>J(a7$)Nyqgy1vTQYEJXxTa0=T!Mqz6GT=GRJpV%F8E<{Yi;^d_u{R z*Z4G1uz3n+o>ZI=cP{Ecp7=e>eEBKN=c;Y;?V$R)RmyjyWXQW!8fkU<0v14eOL>jZ zH{3f+VUpl#k2Gt*cQb$6-`uuQk0B+GiDnenh0v6x(vcHcODZBjGhVE5>F?nPlE8Xp z{88y}9xlaY``Y~KF<~j3V^s0Vc-*uF?cjW}w9KTiTK(FuZ)WIJDIdq#Usj|lL60Yq zhAfQn&mVoXphtbDBMWCfWwnG|GH(-*43fC2KRzSO1RH4}heC**5%XqH=Q=G$&hD)DrVoK&XI_XEm*?mzG_qF4E zMpb;hv8+t$X%Bcm&COcCRWtdpY^d1!{9@mdrM$+HwoLto)=ggi6F>?;4^V8IFFu)w zC1?Xco@UWJHxGSV?^3EY(Cwo9FllSG1Wm+J5l7I&P2aV)ekKVw}8X8rsrJE1ZTrejXOJ?H$UjAy}}mx z{lH%0v5wdrVcaK~*(Rx?)_w(EW{RGa)!V&ZVkV+M?i!!L%Fisd=k6nRaMXg!#V~fm6;H(`MU#BkM>DSoCzw$fAxQ&h_@vqufkEn>5b`?KR;EqF)&k&bb6d8Fh>!E!+q@ z_?vG=%bY*emiC=H;8Ve@o`6f+{6J{((Q->WkgYzXqsFL>dO&xcCC}zgbcAP(+IH5e z=cdpCrtQ6>(-g+KK;$11WUajAD89U|i|E!Kw9|I)xk?zBimIu%0I5i524tO+1SDtgN@u-8J z1>UV0vsp&u`QIGDm!+XH^(`<&dLXQ`j5GS2;FVoNv2X{@_r-h|GQo$OA0FY$iRW|; zf_$OwO`89vt#SjuFe^L1tMXj*@>X*X!{sVYO$t6=h>D@^CuCko}utM4{kDGT32|j* z*ZtUF8pJxU@aWll-NwQxT4u)`8W3qZ%E686Wy7XxSG>Vc*CODDqq4orqty>b-B!m~ ztK$&PM^A#iYgWGYkEDs44Qd~!7o?>f`MMCNi#-;E2FR_ODs%UKeTGYe-hYPN zC=lUN10(~0#_ez}Jy+YPL~-2bk*+r}UGY&vo};mMDW8V?ot_-$1L|?3r5+gNU0Pdk zNO|*q_}Y^3cx_8PnY+0Ps4|-&InJ4L8hw#nZ2=zkToiB<9wAD)_kP*yWi>fMgZ!9v zxfo_iUPw^n7joXQqwVBB>IfYTl{6LNhtg=f9}R90@+Rry)>4oa!=lT~0!hXV=1)|$ zLN!-IYAobx#P{=JQklh6=3)cfZHdZtU1|dhTimcb@UQ-gdV3~TxR20Y{@XF>pc?!Hv^+Z@7Pfk+$vTfN&5E>RDavY}T(}YCm_I?2Mdde@ZjH%W z6DmLXj$nWab5rw{@o5&6?8v|%EET&GngzL(JJ#?Ub(_u*5A3r|(_up3QUn3YGt7U( zk?RYuyC^=YA?>j}+gHemU5$O?-e(qVAUSP^%2CU==ia0n9};~ZY0~xW z&paTO!D1t3#KY|V)r)z$I(D&szz?hTh2o}5iFz3v@EMCmiEwfJ+n3}Fh)4xlB|!#9 zYtUdiuRG?|H8-eGm$KnoPRpTqK<;F)u(S2K@+fIw1C=nC_I49=$6>>_ZqWU~$Wx^wFlo3{$MuUg$T9cq}*#`4koI?c~{8J%96$fT^X*C_Tmy z`k(+e|81tqwHJxaewzy>7cgAs+zZ`;SXy)0My(CHq}*w*QCRX~jO1ed2~$hhxE~gP z`k>KG?fQCR{B({as@rKMG*&H2^k`p8MSSIw%$z)q*HmgD^d>2QO^mYjgyvchI?p4= z)rS9I-8N1nJt)0tm2u6={dd^8unvXHahIz;%M-2;uA8h)M^6ZkCPVdfX@toY{sS{1 zy^9qB8ESzk-SopAW6ns4W3{+|xcY+A^f51RUk)w?-owCTb^P_DN+rhcKsn?N95&}T zXL(~#pspLht_0)HTop))dv7FAsa!o|xzJ!S@408@eopcZubPhP+oy_0vBx_O-cL(# zoN7Tj6UlxvKZ~>EcuK4j?_@MOI;oN*F3l%IZWueOe!Sq8%!N`ztIA!`BP28r`z$jV zC|WF66p|@PK}zloiDha&bk+I8gdu!|g$alVl@E|Y7XJ1T@SEzo7qFY%a>>a$h{L&hhYc&v6dkK%EPt2C z*p$l&Y18$W!8pX;XKxJJC=7aCE|IY{P}y?UJ~9-m#mo5Wh^&fy@_p0E8m(O$Yig4H z6JnTwsR&Pkc&VX?>egIxd{tcR7BiE#6~9tK;d~lATsOEzV!!vc^{VcWv0IxBD)UP! zASbdaf0fawtUc@GHrv?YsK_oL6&mfy)4qxtZeUgmDgni}U6-H$P?UWE()QhokVjx+ zIvI1idr9tgA~DW(q780L1O_G-6`o{Za<^N}l%SAt!Ybe~ z$PZcWC(;=yfk9#daj|3KB`3e@$^jy*WY?Ok>0FzTcZ8Ir$-FBT4gJLw>~Qh7slS`| zRgcHNbc3^kIHd3j7@3!8ak?d-ug%5F50{MQ5vwW_LX-IVU8&U46v~6Vu7gc~Ek)XgOH`dUQ z?6Jz0wU;zt;%r2yb@hHJ8|<)=R-X2~#*PFN z($s`F{QuVOyiBpv^H%k_Nx10ZZeztc{ht|o|G(9aufA`|$;xy3X^lUR_&brOwfIfm zud=+OxTDmt0swhZ0E0Gb@;Sh~a)Dn2J^}gxOJoeuQP$*> znxaS9pgng~jH4tA9`H_vCx5I!=SpV$3=Idg0QbRzcA> z^^IUne(;zgc+lrMS{U!d;}-JD;d6jT*3k7h38;@)O>e052|&y@Xrg=Nnuz{< zXEWxZ+;Aj^a~=IxNFR>;< z8Q!ulyGfYA|Gd~hcj;aHs4g+y9j{JkuUVwMX(M>Yq;gh%D85U_6=k3Ib^2)XA$Lwt z^_Z%}D+3h^%tOV1O^@SecW$6*z}NA6*Hk+G;m3zX;KxrxD+0g36qH51ps-Rsv$sa@ z-ENMez;AM*yo5n?bF5Dp+o=ehGM_yE8pA-|CkYb$7xAu$EJrH%LTl_5(BL={)Dk-} zfqez<*JTYYpu4(krmjRIw`?7B$@>W;^um8hUNHlH=p(`y@_-W5uSWe$a?AV(`m0w=Jf*dvSDc$X|^{c3BRAWRU8m ztqQ?q{dQq*Ut|Eq3=~^C7u=l7{~P1@n`7Jbb9G*v^5|&?-uNruirEo&y5k!T+>L;q zUy{i_8PY_>9Xd)vS#HT`Zs7$>(SkuQq9XP;V-;~e`O`+v7=C%I^?@#|(Ram_PLc)_CYOs0)+s3y-{0oj_Qbj%MxV$~kK4}LkAJfq3K#O`^Jiif6jQ4Q{ zgor#Bxg!!FvY|5V`-+%fv5mhp-zI;jx~06LVC%hGDfV#}U5>>=5~C37KME^H*s!9c zYWK1ackekkD?UIy;0dr*WTtaVoX$=D33OLU)RR6iZXb-3^d9+GuArORfnWQd^dj!s1|7)NfxvTA+)8YBrgKDZ*{c^H*>_4 zh3$1ug#UsMPkPvKE02d8;9_`Sm!hZWpuPg6&zW<_-W{S>@P%Fs}SULJI=m2N5pNAmtR*9BG^Q=j`*>(HoNenlaiC)w21G{AV+! zwnK<@+$Ue>kaiv1ZT0AJQWaf82>r{|?UVL8A@ zQ1Ac-~}G;c3n1k9?ne1Fig% z38v={$Yy`!FmK0&*Y2*_k3z9&kjo;IM1`7dFKcOy{p8@};UZlAmE2YA%T~nnx;*dm zv@Y;ahb?D6zTBMwLH3~0JT`_W9eKYkF&G+gJC-(CovnW}xBOl`MyO{_+^0?PZ>NA1 z;-Z34>V}lo*06MB<$m4;N)^aQdG6-K{+vQ`QBlPnEU2%^Geqs zHX;_E(JE~&`U(qjyYIF@yaanqPN1_mR#CkwecYL8$v~~xZl2rwPuPk5-+Bt4`zqE+ z)~toc=&-jw0bm`wmnC6NoP_h$envj;Rhr7S-W8c&{2cojx%!)m`eHlnEbgyLV)wnj z)_F^DW-MV8JGpb$(7~mpoFJGsw>)2db9LD&*2H+Za&9>GNp=I~=NOItFJee>@QwLH zmSVzq=ebbB4Gr;u*qqIa(Xt-&-0EDx4UQspBI%5CQb}e96P#@|XdcuUUjW-V1k^;h zA|%`wWC%fX>e`Ha3rW2ISn_pgXep0FXTZ#{sY7ipntjuhZyHD-@5&3L?~5?)F!;QH zp+|R@v@%Sqo6{o7)8`le^&`OE;i+UoIX54@(V-n%I(}tRMGYzgR*mJ|UB>bFy}__fWTTNl{8+VPoqq zw}EQ^YL_1|^F$ zuhJ>6QeGvicf726IF?1~EHVPp_Y#)5yCK0YIG*$ig|?$7=x6~Ueq#UP>yX~cbEg86 zEI9JEL#L+Kg|%eE<$-P8CKi^GlgblAGfvx!-b$mVXb!_nuj>yN5ZR zE%tBtX6sm_2syzn*`+|9`hYsi`Tei1Z}7-FKqOT}%qLc*UgYt+3oqWh8tbwORW7Lj zdPrO}fMDagz87+!(8AJ$_g)5O#t@>O&Qu=y{DmUPsGIGvvN z4QPM5tQ5Rs1otj*j8kE(7*OOOff0mq#Mw4*{H_;KTzY_!S-McH} z0xI@NPmnT%;gsm*DM$q#m!RhJ)Bw%7BbmZ2n`Ij>exS0Skxd z=z1Mh3S87mX`!u33F4>XH*D1Mq5mL@(s~DZF45)(PJ-WB2WLI!a`%EEg3z8U__;$w z1;aocz>ju+3DEzbhe+I=xuF+SZl%_xkf}AcK?dAyW2S4)=HKk2yVHFf=cq$x4mZs~ zWm>MvG4>1zSwiD#aom=cfkTjTQE2T%As+i!*ZI{G@VfpjQ7*O0a%~DWDYQr}OIF&9 ziOWqz3kgazn28(vx^cfto=@UKYq;XU6!!FG4RPiKeMBMK{7G}Fm$ zrnsz~&dLsL9lRC%e2SB2-09p^XbpU!13pgJ6!9bd37L z_fsi1IjYO&6PM{Qy3(RKIvFi-l_@daGXqZl9+?uW7ys5!rOEicbS_qF^eca6xUlP>n+U9)jY@`eCQ}If&?Mnmv51GUo$HH z?QN5uIQGSOtu4ki=O>vT^Aoj2_aE;0zf=7iS32jVU;v#!a&X4TEK&G=;Gx^9>IZzM zWOPH%UY0-#6No}KeWX=W8RXr$7>;oXZef&mvO++`q7aLmqBv?u%6%b@)YHnWqg=br z87%N(;=cS#_fFdCd|L&JQ$ zXw~Gc9hQWmfE_g)ucVQR9k>4`uvdNK(7e>z#O7@Xv=-pDg#gS<8PwIllT^lhvmniAuA>sssbqjL6wIVr@{OP z*!aBU&p#W`IPW17YDl@~a?Ct){{}fU((Tx}IV`DlL$NkTFF(%SGg>YX;yNG{z)eGexyKxVeUabC3EthwoLdH{T*SI;t%c5Me9%-i8Ocd(UlCDy4!(6`G&YWC}niFIMlh@fQ;?gl|C^| z#&_SH?{J~k!;?i+{dp67T3b06p14op`8|DK&&Vq>9mUceZkyaY=38AugQUNyEVHHs zM~*ZPz1eE94=KkBd8~pFiyJ70ok_Z()s^AxcA?sXsV2`ocXUS=4EO|zZMr>zJpC

_w2G@wd(th;P{&%Z^v6@rgmWYB$>feI>QMtzuU-Ann!)EW5>AWQ#{k59H zMtFf+)EAqBwHa6GU1j?yv-AWnsrefm4X@p>NXxu|H#t~etEw459ou6nmySvp#wlv5H5jJilRs{}TKT)6p_8=X z%_k&J*2SgdwkOllpU{TsbHCmAs=Ox<#g7&ro_hK={ak{rEM?zV_@93}#<<0N#3#yG zSm9;0bw<+s@$8JwKeabrV$|%iqe8uf27a%NA4`gDN%lG6Y*Q-+zz#)CO|Gvq+&GS; z-YgzC;0m}{c9P)PbGZkR71A3-2(L?3zUu3I`Imd%ZU%c#hlxt z)>`y4r*J;O4^o5O7Ms>qY7QPY)E^X~GWWREW8y6cok+F19`u_zftZpBz)v$X2={?Q z^ESfAJI_gGeX_l-ir6r30WkirRtqyG;1n;JUikIE)29Z+l@!>M135g{{@!#r`3k%y z+v{Bz?BhjKXmGrS21~Dl2Tm|Ea4W^bm(}0g9yf0cEM=$ZO7kBE;bSd*78W;aZ&xDy z=B$@rWY#YiHQZjRwl)8+KW7Z6JvJeHaTb=z4BlMC3xYVMnvXNQUpGpW{$HB=WK|39 ze(l=s+K}`A=(qo;9}~WqS-S}T@lKKo{r`T@|NA>N7OX#F-c7pVrTYGRWcYu7MU@(i z>a-dt60g_(U*1ru#Q*{c+VPxEw4cjimIR@YEQY9moo2mM9ud{~1HaXfa0u0{QM(e09UArdaV+Pv z9f|y-l&#X~w(GV)ri4z7jrdA*IUFqU+Ydeb0t0(|;jciA=S#toMPYkbcAY>UF8FBQ z!7wFES=>NbWKB%*h|tn5UO(YsWp(%qtq;bs)f_q|=7YQmkfR5I)!N8i$FCgCYV`{< zqn4HpYCH?J&!uJ*OB{uDl_}20R5X$FHrK&SA4qdV-0-AA?e&JoA4E!`I#?c2UxcXt zRb8=U7{C>rI?!t6(o36GrHM)_*);LlcU|ioLiNL-fhRGK*RbPl>l=I9ZBf2up97V7 zyz?%{SQp`kKkDqDBdpYq&OY%!Ek#~WElAi*Sy8CMEvB0068OUZ>QE+{tN%VtF0eY% zCKnw}fWx?FT2^OTep#PiuPpd^?n0;fANL-^w&sL`rYyzKz>1f|5ue960@6a=aV1V& z(tWr%kRRP!xRhGtP=ec|D9(YexV{6xhzyFyJ^911>c=H&ho@%{9}sDqZLeGz7s#Jh z=wqUGjJ95LyWnuGswq$<3Bs>u4H2bn6LU{n6Apyk@Vgkh4=ae2#g$(Frco57~!UnKV2?scE(KDW;<7%WJrrkK89{x*+%ONXX0!MLXpeIX;`rHENdDH{_lz z!c$j(qF?-ty4K4SvACW2x%SndE5*r&~M$!fW>zi4W3O?7p zg;PY&LBqczqWd4(nr@xDAzqBqvubOv4ON5Ag6A`53R^2J6`BjNjn=zo@w|w$P@~e( znQDxA6>OcO5PjEMXJl7%yjO10t@2OUIes`b=Z*J=uG$C$=G$62lLF0Z{nY=;#!Va` z4PR`B2dMN&zdCi6d%3gEMTMbpXQGm1v;Vcxku0cb=}fo6KsXbqsg15pKwz7?1wsvP ziXI85zC}=GEIVq^hJ@(J?!=l!ioI?J$E)~zH9ooc2^_|V*G)-#nKzYE{c9;nVon3?Mcz#9rV}J zv++L6AbB~lE>H%0D~|s#UNobX2sT9{QrlK8rJ;7rns-RGUFxoMx^z~1vz=#FIWDwr~nFM}Cq&U1;?RGHp5H#M5 z6wpNm?9b1EdA2JJ>e257ocGAkqzB4Zs-EX~7e$TE%g%Jm+TC#*4HppbAX}a8SQ$b? zvn$)uN-FvEA0ok*ZTf5*1sxC48|zR~PD@7`|A9S}x+JI=rUW zY=``EJgfEuZ^;-nbL5QR@5kE9wN@zKMV|2oz|6lD?{0K3WU^1(*Z4zyEpTI}Q)4Hl zKp`ENzzI=E2|i;Glc;jQ<%lzn&4C6-3Gj1Zg*9>i%+D8bX1-ExoKr>3VQgQ5zs79_ z5aoJ7Q)6=Q zoFaUV^pOf#h%ltQGaPR*H7{MEx|ue3!IUr!3hB(B6me#^Qz342QgU|ZpDIsD+JZN! zLNU^r*l&^&Ff|67YNKXPnfT#Uv7$E}TYsO#X}jT3yGWbGx@TJ7#{8L4YwtP`hT$^w z_c5SIv=f>JsJ6HndGMS(MDvhWFz3t@1Erj4yxJxk`zDs6av!{(%=bQJpfGyK`J;c$iyOrT>wvzFEEFQ3Vp_`uX> z)5*NnL(f)bd}kN+MU4ce32f5@5{_zl%v_r$OX5Av{$YBGC@}IyolR6oXLN!HYEFD? zqH{B|5tGRaK5ElAB4sx3XP)Xd&;JYVXro5N9R#{}S*Sv}2vz)ke}o@w%x{WvgIl$e zZezam_3kwR^0kOFzzKdk@a)>GnRJe|rpzsb8Wl$anqTUHf4wmD?|BMYa?^V6Qm>Jk zpB9Y^g!~IbiR1|LKl`Zrz8vu!RW8WZSM-0WMg1eB9u^aplFH!vn)M$m5{#ULXzX}C zr>hS#@;|eq)a%b{Y5qRtHZST0kc3mA<5q`NR+HQ5 z9(=`Gmi6&;jougfzf>znK^=Ib%Koap%8mJbsb2A_K&!>S%cr$o+S%|G>WrNwnZa1O zb9y#CeR`+ZPra-$;F~pnYyj)cBp$BoV)c`KSAtpoIKJCTvD#0WY~P~t9}fFl8jZQ? zrGnL>kbrGK&d3?+1)5WKxSBy#d?%JW$Vf6`T#2U^rD_?g3P!&c$uX z4vDJva}jFu*Cy?g%95v7(O?mA5y@g|I^Ip~HHQ>JHHkP=YEW=Svv3KZ5?q>((6pw( z6Y=FO`)BwIu1nbDx!2eWx>hFX)%O?10L7R5lfE^M=O0sV_!9QB>rxZLAiKgxC1F?= zAhseM)aI^C-W~7!*zs-reH??D|Ct^%yIYz2n`nwIOHAEa7F!*AMLo(m$-26VQoV$b z6Y5~p*z?k;nfUOtWpZ2)5eOBjCz{7DQg1Rt7gn!>a)h=@P(TaSAb8j$$xle=o50#$ zhLr*qNu(hTYLec~?@T5b#g_8}v$B>Ru8gB#Bd7i}P;)8ZwRSnLAK#8br758?@M=eS z8Zeb43es;rgW2GM&c}W=g>C)sT7W39?O!42WT{++v;m*M5rN4~W}enh${a> zEwuB`q|G6Y4w5z!lP+w%1HjSYfu=c-sz6BMqWbildi8f4{&)%xOWHic!O%55>i0cL zcRk~Ga?DouS&QMguUEcJWsrIqaz*s6kwaFbc!v@2K;HF3!kH?yao-Vsi@V5$aoc%z z>{aY0YnO;@?W^Ill&8zwvkT5yb1SqZ>?pQn_m7GY%RINBFseoB^neT3@yq*`;(E3> zJ>IK5U6Nm4E%UQCO)tCcMDN>TZXeLY?z?{#1M(GBuo=^q{e4^xn;%}3|NPJPgQ@0^ ze`n8k#QriMsSia2J{0Ew8+C9o0DKA@cb_+im)LFjToK{dT(WX4^nLm6+57@eOUxQK zv@{Z%T-dSpXa~VDJ`&9=(zQu{WlgI#CheQ3Wl6MBlepJ=@dsyI2sU-IR!MD#(UBMY zIe%<_xAk%!W8}wyeWIr(C{C6ARHioygg|J6;Ym~DLG6QkF6dUo0x-d$b(DKP-dQ+Z zfmD%sCz!1Alh9CfAyL@S`_zrZJ?d*_g48_F zv}hz-XR3AsSZhG+t#=guY!2jhe3!%5|6mw_LXT6*WYP8@`R-{XKUkY!=u8Hrc{B}@N=O&o^0{R`9{E&U2F8=X$=PSg(r=)=551dj+ zgBT0D%-2$)Kh=QIHHqUl_&*?CMY8FZ^}YAA;gEn`ZCLNPX%ihZUrKaZRo4E;(q>dl zLi84ik-Nn914?PO^1ZL@}vHDh9RYlt)UXI64Y{i{aYWt-1VO73FVKTy#PZUIe zMS#Q3R4#geZme78SEa}&EY*h=6ID1pNfM61r60KA-cFKH3*;G>E8`Uea;=2(&Xq=Xyp>IZI{!5Qp`3vrl z&MRFM6N0Esx%1`4BeV^9Gs{+5bZ_aY#)pLiFIVBLdjU?>VqrOybarmEnM^OOVjFFc zs(_U4*rLrV14Q+=t)On*ULerZV?rcv^RL|#8$}+Q_lQ8y@AnRC3&b0{A7wjJYC}b# zNcxXHpxvOqACdd^D#BKk^Rqd_uAP}l@SLLAAHxrmYGSN+a@;_})VY7Kf0XEQq9Pbn ziB8aMECL5u?L4=t!sdAmjM-8z5_O#?L2XoOoMd^zqemi|Nm^V^K)TSw3&tXMrwPgs zv9GEj){B)joq$`3njebIHEV{CMyVFONiDoBp1sUsB{aWfjZzh9*5zXGW^KrnC!LT$ zr|w2x59tXb>y-w;FTYS7G1ygrKm zt#3bO1HE{MWwa(hBnO+@+g@=~$iY`1w7VN$B`9}=X-K0m$Aasjs)i7a z1EbJIHsUeW665l@RTO;2^V&)}5>XBucBJc`)CL+fl`SBBR}KL=ki#OJmc^ygOuJfu14Ms{efzWGn~UT)wGLv0kkl$(8yqYYf@>qJbJe{XwsfnS5Geg0R8B0>Iyq&s(~dHmVuu6foGr8|?91X5uf_;#fCGZ1`vrdRTa zXft$`J*e3#Z`y(G(zt)5?mXHE_TX1l**yqzpY6^B(1=+m#lx$p}Z{mC(>ZKe!ENV%Z zUj8}#rj2SaKAw*8u1$(#yS;7jC&tJlXT6ExCxT$xyi}Vu-u>$CpK6&)@*TW3qh6QP zZEEQa@gyM|i&e5gjbg^^J40ovekPuV!v-%OzGgAYSC_rS4B{8J0@O;eD&RlO{oc6z z!T;~`n2exu54n|?Ysi82OLNxz3!;kVUb(Lf)T*?ZVmk8z2Gni(cyeSq`S;Z{2CEVT z+4q<&73MK4{Yc-=jJgP=-;Cs=Rc0&T_Frm+51mFSMrayfRwarw54w!(qnXXjZuyji zMK6XX@|dxp{`=;6>k;x|f&-aBe-M&R`^`5=kQJQt%!rGpRPw&Q{Vpc1&R^;LD6O+D zhTs%c)4_1CtS+PWVG#7W{L15UiYDs?Q%7;uTHEz&Pc>K|moo_% z+J)c?o$qPV>b(G7q6BLCY)Ai^{_|i-#hwXa?iqmDbU)sv&T*KV+||4**mwf$9Cn%> z&>6C4isD>^<0aLJ5mt?o#>B$@SDhF-jO{ZP>BG{lAA4aBD5|bEYm_wOqWQZ)SMAFO z?Kv`;-CGidJ86_H0O`vX^S^T%0XXWwIba4qw!XIHLfa5e$9~#UBp~+@#jHFXTc)>+ zp$_9)l5QssWtxm#gPZU3`wA@aGb0SKtKNyYxA(Y@kCilwYEE$z(#?XWM;?0c%neh) zxyZ5P5CPTdZ~fPWDV}l(K-ZI^Sjii(^*4VkeT7~MRYwUteLs!+HIS_TUjOPu%U`pT zabe8eQAV06=MQBT3JWoAZaEW>3yu{)L2;#RB|rVXv>BoQw&`9uPEC` z-wo4W%6vpNIVfNX2-0fa6|haz23~==whlyJ{+x8t-EVg)qlM~s8n`y|n66+s5twg=Z55;&Lil9otH$PL?W)tfS=F|W9k||QJ8O2* z>HD?~7vMzYLC6~ceZD>waZ`u3;nB8oWn=v8C+h~wD@eG*+n+lLJmLun$6t=j+6b>J zvLz?_o@ZEg0)xxW-^mAa_G142)*V2QfQMW2SFUV7Cj*2C2d-J*iQD@xP!jGHl7mW< zho7Kme3L#&Lb0aoCaAL@CoSfVcQnwZl8Xsml=`9CW||IIBEV;2=Qrb?WDUWwFqa1m~2GBi?2iJEbKwJ!7Y z$bRb9e?1t^LC(1uFx=O*zCqIu6p51IU~+q`&LhkPOVUO|%(%Kj%d%f2@p73(NpM_b zWr_lE-3V$7Vdgh*%NjY91w9!2y)K6xv!~gZvCFrf>NR?vl1N3#7C4->9FHgiAb|xa6kWrQY5sxf_F+e`!1Ob2?P1 zA#9K2k(I`~I%yxl#;gSOj}H&ac}g>%&dPFX8J#0i1UK&M2Lm(i2L^`M^rgZ)$@Dn_ zH$E~Qk{FJAra>AlWi=ffW|>C^IzV=4n$>z2-ae}43xHjO3-Fe36n~{*Xr#5(4>ipm zG_V&mpu;8AET){08`~T&t1+bOEi7NWH5jq5+dK}@;D*xX?``3_{B;8=6!MS#30{ZyJ9B3nPPa&Af| z!e;rv?DUG+^rsaeP?kc4XppbHyey_5#-g3GS}roBd@i}n;QSi4TC?K+;qE=7n)SN2|OX70J?_cNukK>N--g(O+49O^yYpjxqn zv8Fu9kL2q84)X76%M_x z4kv5*gVJwpiyK2erFxPIBco->DZ02Dwit&uku#fGi+JlWR!N}vWT=`DmHIF(`kJ+l zY1s`c_Asut0Jg5yh3%|nd~Xhi%@#XildyQkv;EUViyI+hhHHIOsH|U?FRO9GHe|Wz zE*Xp!ZAi%@ZPM7ZE`wIs!H(UQdz%b744coZNX?gy*84x#4C#1#nZM*U>GIv!ojGM+ zC+~GLX7IGwogf)=v;oHcBiUAnuAZU}jN-YNe$K+DXKhZ(r|T}RUB79HgH4w5 zl8RDX7|Pj#MV^a1eX_-9jN|KyG3U9?^bglc%m>>GRU%Kg*%x3#ObE0YK(gcohHg09 z#KdLy$2RGS3(qJZz?PxJkgC9`l{Eh4L;gNv)AafD2-94agwf2A#BaSLvOfilx!=H0 zQ&|LVk=gnwjc}#FDxLa>>=q;M7aK#En@ZHX=M?HAbV1b(AK8xf#ekDR9|FY81%Aui zOZtK9WDaK;qc)_&;db}Cr8aDW0R&Y{RN&dJfZOFGN+oh-a{2^j_}mqpwQ=T`3|KR5 z2?c>*Ykc5JcSX^$ycf!1jr_*M<7{qQn zPxtxYgX8TXbz;}&R6oLb;Hh{R-EFg&EB0F8$9BEuB_Say#{24Z+YHgsAXlyiDF^F8 z8|Yz__+fKpuUUF@N3=Zt2dfH}2vZnFP50!;S^2Qmr&kY0BO=&`+gIf71nldIESW){ zX=%viEb$ev<>|_4ty>rky}^d?`}A=sDie08Tmv> ziOR#eTrVdQA7(-#k7G(S6j6Rp|I7)W^7rk$XJ@eL?y?Nip?qYVn0p&u68&;3|6Apy z7`aC?CHJh!v*CY){fya7x(BtdXNYI8h9AR&A6fPX?l}I(Tb&wq_?1-T&zPS#bG*Ltrn5SfmXz#+qU1<%2I>klePE9 zrc=gRt{ho!{(gxD3|3jnV9Gr0g^G&;F-pT{Kk6OMHow%YshMU8C;?R0cb#VuPwxYc zS$20Yr;)DKa!f4YvTRGEqFVZAR7Fkp#aU{46rm#Xt@5UaJ?p}%6N^9uwNf|t1yLsb znrHKb>iBWw`X(3goO4?M{&CPX>;z2eS6W8~5Ss3INEs=MY@uuHGnjsTq{$5oMRpN; z7OQR?{7{)RLvf=pc|Dp?4207CDu0_5x>TMlQZWB6PgVe92*=`Ji z4mx@fQ4G`ec@Nx@R;x|EgP^aZ6HM0?ZHWpz_(YZp7?O!AjJ&f1RB?C;*@=#TY&t#A zeX+vmq8(Q*l?UaO`g7xZSJg2YPIbSDYQ~zll#%6l&=^C;= z;+;@Uz(G{FrDQNMkzK@XJkW91`_i$^o5Gf=3B9|7UowLW@0Bo7t8R6_FK*4pQYIJu zk=xv_Tzhj^B6M#tdf=4#4%O+00z&^;nwo-+H)zP7+qo=#qcxlZ7Qb6kR9NF(Nyc&+ zd!ln$Gl|7Bmupp_E~j80ew*ZH<*sDdxII5#FlsLfoSrID_QWqd=LX4}Y3pw@0hd%1 zni*DOX6zN&z=zN8p}XQI)L%gYc(69QNn31{pj%t|1qwwr51KNEgllCxmIhzC_?j38 z30q)Ak#{u+7=LfKW+?I)t*qSBk@UM~?2hauv75Lq&8Syrhq(dqfr;&Cj_;+B(Gp%Z zq|f{l)mI*juUHp@spOM?{KW*WUs_vU&k*XkLdu1-D;!06!lp$8E?E7oXX zi}clwh0XFAuO}kL+h94gQ^`nvn9O2zTWtN_6+iq0^Govc2dei~CcIq2?)eu#R7Bm< z@m&D=btk8GKs*oS-XB2BWT%DJ>_oR6_O?3axJb>cy(Q%^ul}9hf8)PQXc{0I_NRs*YF0dq|BU^Xx0FM`sBK6 zZn_2I8(()tNh`~U>V{HF4c*4l8z#o5uJ@gbwJCpiEk~h|0q4D#50#gt;|xcjLpUM1 z8_Ljo5{HVjfVOFX4wg%;&%}w%lSub%sR>h7K=VYtd0MdnZCevLz93mNn2Tb1fK9DW zjvjNW9~ajb(?c=5^ZOA1lZOY<+>K7tK0?&=aA;-l4r_w5q_3QO|4|OUo%>!t)QapD`$rS&QKsZWydEfJp*MxkF}ZkO`9*bp{u ziz?uA@JEBE7Z}eW8>Dncg!KZ>6~2$;LbWlcsY(Pk=}$+tCQ#MD z5vs(n`E=0m!tb1xO87`=7OZ843)p^HLTlkyCzuw4M00th@>;{+fGHGJA6ib+fMZB~ z;~h?-e4EA*T>H&2>Q9a~?#_ou{jYPdNN8B>>&U3<`)G!ftb1y>p!7sX2a)|=*;SwB3Zpy(?7aw z^wahXZrrC#kUPNnmRon_siHSBQy6@lT%F&80uB(D4*H%;0o!+FG40--Qb)Rq{f@xe zQw0K!UJ`5%07{V`dQlJGXZzoZYjG}TvG)|W8th@9C9~v$`Zv5e#pKC1ukFq>RZu1e z!v4^M5L~49FC=3tP{TRoE0eH8`4nEw}08t>(RS>Ng7EaPuB2D*wvT={i={n>6T!ytCs#pBxW=mUr%gvfi79G6veLKCcS#wj< zd=m?%UP4!f7m7g>rSE*)KVHz?o!Jz>E?1G}N^=Pw|61P(wAq;B$Um~Om#@uWGB-+l z1sUab-|uYsbM)+5?U3gthGFl@k>%&K_{IEGoxKO%UE$p&r&|biPc`d{A2}C;fiR=g z)?j<sJ(2z+_}{E_xr^|HC`L&xR7fvyAoOB}wlk9yU7yL? z@Z@!WG<=HDr@`^u3X}1+h?@*6lW!v62KiCwbX=!n< zmHaj*E^%j1Wh~19`NYJ=npg?-tP||hOQ_c8B@Q0A`hNOT&MNWt8Irk?6_w^I_W(dv zytP!uxXG?2TY-P%BXs+gz6L}thNRZ^*v5-n+!gFnl7hm(cjwIfj|t;pC=ra{jHhp( z@wALYKo@wwK-R?bYDUD7>Cd=6Rl%+Xd2R2lMJw{U1uzLN99$W9dA4*m?ct>4@9jc< zSobG*hPF=Qm3!U|(;Q&X0%hYh=?9D>fRK)O+uwRL&TUwi4Hk?U$olOf{3WZAixs7Q ztcA~I4-cG)n|z46UtgIj^7u)`Mnyj}%aFoi5nUZ=0$L(9^h+zoYx?`k-aHGt@97e$41Kmk`7hOI*O zxub@<)}@2V)#re}5{{Dy6@49BSHd`A0z2u&E)8$h6LJOz)$Mi?Xyt4G<4fCF5u0Vn z97(OL7Kvp|#PtRFYj)4moZXLqTgE0Uxw{O(jf=mz`F|+be#K#1*S)f^F7lHtpyxA_ zv1e)-oYCm)o8L0t-)*^ybkKYj0s8l3j}a|7BGVrOvA3rb_OVKygmi}`A1a*FP832H z7Q-QL-Hmixrx#Tx@#I8P*pJ;84Twnq?|2=HX0(!KUgyq#yjX7~ zM$&K+}dM3t(CNUX=)%cygmQ|T%)2!Fs!Unr-K4)ug=P$R-981S!#ih+< zrYT_Vj-P&SPAkIkhn&Up!L!%dp!)KJ&r!X=Bkz(beilfv%wBrVEirZ2QK@yC4SuQ% zl%^ak?!7PgO9ZC!bsjfzgIP&=uwB__LZDGCY!q^P@LP6J`4o|&cxk5m%Z4|%q}L)V zq501UV&WVDvOB(?X)E=ZzEAx=7UE#NZsRDWf#se;mK0m}I#fh34q!NT;6l%PpD=ey z&POl;SzY0ev;(?52tj0AEDVL72Gp7|(q)`i2nXL-i66RlH;f;Cv|kx| zgUhqo?5{doI`=weXCDe(DxtHu3LfuEvJ7w!F#uk(xkwmnn$6aidbPHld$hyHC%9J} z%}#i*F*11ZxSY5spKZ>+z~Njo^ze(gzqWF35Y30Fe982dV@|#`H~PiV!ZdcgZshI- z-R7@U!-6rbYk?9;en|;YQd^W7*z4BlorSl>t^YJepI!6yCERtR8`D%AozVttL#4PH z>w_IuLPVL>!v$@4xUxN;t?^x!7QcW3{q*b$W#(+YFZ1ko}@qmW!8W+_nl>P6u-1C;iYm z>Z{ze_YS2F(wf%*HA|@yKIOk&!;2j&d9>BwNQ@lJgpk#l(2 z$#p#5bAw*FWbcr{`=$Lo7io*7Htr$*G-~L2CIi7XEc(}8$}2UtR?oYAmBT0wN#x{S z=~1?%B9SQ^CU$%92j&2t0B>`TyyKZeE80WUd2gqPg+WT*5q^uHrEAHvaJq@laj=_p zdvy~ZisGh)Ij}hlL7SNCpCSr%EUx90RE3y>ZZ^ms< zK%KY+*~3MkcTIPv%dL-EKGeC9n{pBx#fr6YBd^olo?qc03@+3f7U>D*JS(}gKkm?! z6~0f3=aW$BiNR7#f~FEvm-cnA$CoN0UCrNI_g)lZMi+*W-y}CK=A7SR8lIdSy89jT zb7sqt4!QAazBCw~Lm2EWRZsJGvlm2s7MduM?@##=D;`a5l^pBegGD`or0rbOsCgsr z;N^Ey2@^8(Z6fnBLmdRZr+0KJiMk)bpKe07w0Ha3lIWcr?dk1aPfK!H*{L?Jcw~=- zY^^*=hIYurT-)tPW^(?_v{BsZd^BHqH|m*n8*{pw%gRNM==s*?RWl{N=(p4Vx}nN`W!pLuw_@b`x=)SD z9`plAwuQnpHrX(e=1+=&Y0X;T;%-3Wpwwdi#=rK!UpGE9cmWZCFKUTkwZ^@@o5&U) zD0BA~&jqv4)CepW8~t;QvTiTd=3#LM4F3nx{_BZ2{o9Pa$U9`7e(_$$7icM$#rT%F zm$FVGCdyyB{rj^7%X^kbEOCEib6|#jbdm^V;daN&1wE!VH+(aWZ;O#axKHZpD3Zp1ZZpIRzQNyxxDWQ10||B+2%ZsdXcQPB=%2xEAwwW`Z2Weec+NFwr1)# zXVNz_?k6HQx!(w`?1dAlu-D(wopPtF;~!hhg1Ge;Qp7&uBolsn^$IuW>AS$ULs(x& zD>TBUMjmZsSt})Nf&GW6A8Ps=2nS^u*2{QPtXfv8gaqR>+_lYePdDsgiQ2zma^&#lXWX!@ck*(^K;KWU=Jo6Gyb!YBrpeh-MJ>1J@Mc(jl zCub>|ec!a-dF*}WBjp_^smUYrUTT-%%q%)x9+J{x>RNDKo`@YSAEI8a)w94A<#ZaV zj^sF(UyHO)Kt5|6zEFC@DJyoYJFzGi7T`4)aSj_9TOs zm2IY}^{Fi?pu|4xnYN7J-gRunL+8zUw{|ShKOGUs&{-AM$Gc&|3>~}4=P}3SaIvx8 zGleKWSnBQ>T?*+#-$-rk%{PD?;T>emKCG@SW)}n<|Ky~t={%jEt11^G09I*(Yes+O z^Bn}6CQ=Gj{-p-W`75J46F;84fBU=Cqg=mP5%Jm3_xRLPH!d@a`l#@*FMb+#E>(Mc z?}C!?R7l!ZS%$5<-1B(C2rt`C1dD`xB5Y16YQMGaZrv3QWt>Q)b#Wr`qF~kgIpqdi zh^Dto$G*+|!)yW5+ob}=y=yv-4>iU_l#qHj>Gw|#JVNp$@=?PtIw7&g3AJ3;g;RIUjWAAXsIma-I6(Ec;mswyqm*>ls@ zqT!K7&5}on@xyo?vqRha_M2{>ZBiR@m3+}LHR66)}T#yIbz)1HOr>+l8T zK;W5%+1}KPFu8*Z0^55Ix5{7f6uP@wz1NMV8qgP?f>$`PXN%UkF3xYE=c72vmzEh&58WRWudFYnIbPWe%6S`6?^*r$2S!U10?u`bBbHo22n#_O}u#>-^?g zRBx^hr|#NxP^-!6A~(WUT7)p8JR;Kg%$&Q*h`W7v^Nz3EDFF1v=v$s?W1<;wl&^Mk zlR6Tsc#wTh8@{69F`diC)73FMtgA&1m9=G{RbP8G!K*rQXJo7|=nsy~e&Y#EGCrGr z2f_(|BR8ui;))aXX{^L$m;PJI)Dr15@zC-G-LKtLZ|(Wr^m0eZ(|_(Tm0FInWtKJs zD=>JL`H{5!ZkVN^n4eVjVCnopU$PMA?YS~!E2ah-0L}O$e)1EGGqPj-1eSThU&;y& z+Wd7{l2}M67YNxgMyNoBL?qSV!6}*#_+rA7U@!ONACYGDAjlqltoE$KqcKr~oS!;V zuz;KuuD~R4m(3Tmk>VX#irH~HW8!}F1J)} zOQdZ?d!kYoWAt;k3854IkXvkgd~TgQeu+Q{^P!SSt*0$87g}RG=U)1|AkD^;!-OkFd2FARE|C+7X{%%F zj@m;a>C#Y#&OVCUEgVem9j+lohYAx^L6{m+P8OoZ^zdkXgU=(Rt%#R1!Bps zL8e>itb>R`{=;F~3p7_+S(*XL!d7R;EZIfL6Cio1rKuHUk+gGU5l}X@{ydY9CI&i0 zzTarJ|I<7kK z6`9cpSWzC0mmG|UwNpyKRbky>Lz#)+YE^d5=5ln|=)R6G=Qu47U3rL|(7x=1 z@E^oCy6?@XbPksi5`^l8n{YB?W!JR4dTwGpV-lxUpOfF3Q)^fhx5tSZdHe)d;`(^v zio3y)%wjkf-23`K!3wg{dcJl3q$uZV_%#1PWy*jL6kQy0SK`pIXIU@PCW7GTOD)fd zUEe$8jsRgd82*9sJ>D65NgSf8rRlzSwCMd4x<%k7pLUtU|A@_z3^zIGMT%{kW*vUjXzm*FVQo^7XqB1*iiqAVe|cJgJ<6{G>siuwqSW&j zX#0tUf`6sHPu~8U6WmIV7RqVhlDufR8AOFm3HX>F7^%cOWZII5n2L9H8OEieJ z+scbfsxXQPx;<(?;5s{8`#mAlO{KF3Q#IAB!mCkxsILKLfD6jW3-9uU>0P^qqU+q;MAunb;i1|$_GdeJFG4LB@cKRUB;qyL8(WIo%lG(0cI5yfZZG}mjS4k znqC)rW8Ife+5II*8?oW=Y2+o=J#37alT%?Cb^}|d@B1$ex=V;CE#~*8u6(P@_TIO; zoxY9wyRv>km&PLhOk8hNrmF3wGFRsKWbM)z5DtgY8zW5?2?R%1f4~3BM#_OA?2s0e z=u$ZJ@Xa%f$~@lVF*FC4rg2OgeZjo!NNIn*{O=(1Jt~EVw`+L5sl*=nuJX)!TpIP= zA~>l;uc<8)vVzPHPEPJ4c(^&No8{lLzqx=N2>k!xP!?r}pMF^*AKyQ8<@J+|cji6@ zPRT##IX(^$^7+Z#r$6u*+UoJgjD?`-v7w-!Ui5!_j_ek05+|!$E&1i6&KVWbdqmCW z*PdC8oYXOI>)^?7yO?6FWIy)I!Gh6;*0JGqbl!J7{Eo z%Opb;eFa1U8^tdh?p?iI@c(=&kB1GQalz+WV;=5`dwI?36$R{N zhNl1DTtffzI*Tpndk}NyM`EViC4v!X#dcG3S$ei}gJWG<39REOfzdTizG0E%f=r&0 zcgyq`LiN^Y@0FmMYS{@9Txp$i=s&g{$+N(5Tcl3vf++-BCZX3LHbIhT@|aegFWJlX zdcR@2_u7XqoeQjx+y(yDnIPFITh~cSmFua7LmkaNql_A+%wEqoNBsJKb?UkVwPvFM zRfTU+gSN8ge@%Z|txLF?+{JU8w|#88DmY$p965315w(~+1TI}QtPZS$Lut1MM7!Y< zwE3K@1lkSdKlwtmTYqu~Y2Q(ic57j@#d0#!|DQL$0X~0gq3^ZzPyMlz2X(E_OpKxN zr=-{OEdEbuEGI3>zE=ZTSLITZDk%;^xcZrcr9M}?!wYkUzCX-0_D;5qfD*Wiw&4Fa zKHK|lwB~rGDS@#x%iMCjGI6W=YrxLV_=H8`kuSwR@ZeyRR5;u)U)k7r^Kg^2OWoO> zpRyq3THjk;Cp0y_I7pv3I{LarXqwm@nZSkz{oYY-JUo21hz)O~^;H2J ze_cHpEU#Au5OmU4@aWmd$zTSbX#bw{RifrqoD5d@=vaw*($|+_fH)cKe`6xh0PKAb zr{wH}Rhrjcvb4RKYxzL{W3tOPv)?{J#(~`H4yi52C+66KTKMOfr+m}Vwev{Q{NTVq zZ@;w_t2K0-J0Q76xa2>qvpf*_r|>9*?UOXmjgp{ImN7N2iZda8e3n@NMZ;t=4OUl(HylQ?X5y{E zOrpsHkC%X~{H=2#eWxK@H5M_R{%+Zy<~O$gmk$3rFe-@p-boAd=?dTaGyKpnY;Nb4 zLsCp9A^E?L{hz+1y7IW9R7yzF1I z{7$Cu-RD9-qh}xeQk+)xFoKnzTw&VoUaX*qNT!&>9*yz@x%_BimkSkyo6; zC5t(vY5aGmstEyWpA9PL^3vHII>$6b194MqtrtPkAH!p%O_Tk3{0{s)!y;G=J$pj? zluc2;HNdHwEHTi26?`mniu>&v-B=rgLLEVNg$~PvPxdMYLnZ~Bb$)fF(f$Tt{=gXBaeoU8d)X_GHmw-gcdG6peM>wdv*5 z1va9aysk0$xwb32O;v}gh==EJtggg*LyIaCDa@If8i8g%`^9gB-Tlfp5u&c0V-_;Bx4+vk&h-k@ehp$=%9g8t`$n6IT zjtsHPpu0TdWoA{Dy;o3j9Shex$!|DQk<+wrS;tsb2!!egI65QpK7p7MWhb}hghkU_$Hh`xy?#92qIhQ4Ri|L$R7d5BioLpRX`k(GY|16#uX zgnB$Kn&hifpsT!lhEo}Ln;&UUC<03(V9e8#LfsS(JBXaD-{`|?h z)3u4!*!rFYG7o6g!Mz))XB(qY4_u(sZopwTUfC2m+!LOYs+>r>(j^))N3LHP!gRe{ zdt3nx-by&Kxb&h-kEUfkb5cM~ImE(dEomy(AZO)~j;+#` zMb)Bi!Q{;FY!J0A2t`S9*-L5y98Ch8%eq@+YCIVRo*g$CFl*?eNN+$$#LyX->zI?? zz{LQJ1of~dHgHQ_*Z=4gc;8c1_n`ld!u;%1=HHzI1Z*vezS0FtclH4!(g}WlN+EL6 z{h(n}OMFV5d-6U!N@M3Vi}yHLk!;YW zEU>}9ENeD>ld!z7*xa)FGI-D-FS4*=|G~kD){g(^gc1C*U}uJ|Eosect4U^L@_oUAY)k^q z7JO@MDk~NX*)>};dr~yS!v0~ylP*huK(Z2euaA-1}1wcQu24pWnMY@k@4W z)A2~uhrsixwc?;-l2SEaXIh_%bqb5EP}m~%s)LL0!3~fb>672TZ?gY$e8Pb+5rhgZ zXc(l|m)(e*w-S`|!Y6VfdszcWi^nHA?r<6&CI}heV7~g>eAPxMh-}KnM(iz*6y8 z6f$}M8Qqk&=9zB(9aRibSg|D5@Dt-QzSX#ywvIxcG-26^psk1p4b zFC&QO#{B3RWfo_1r@yHQ3A%KoB33CnpGGw#|EA#{R5|YCKnP49Be0M zC@0pl3p-9D2_4f^{3JFolu@6a$3^~VxV>=Y$@-%&&|__op8?SWWOrH_9l znm1hkklRceJvRMv{rX74*?CH|{fK}4fPY%^P1L}@cC}ASfVC=hTWl(*W4ek;L)e5> za@=ds=7frdFY|4kXgo*c0$v#)UxgV8Lk4+e0^c%0XmO0)`t|x~Tk6lCw|fqrzBsG8 zj{pnQj@8UbmG6o{>!>#zHi)&C$;G^arq(N00KVZ26v4ZeKkQO!zH|*4sN# z=t2Tr$zpSvl?Aq8Cmxs1W(X}#e)crX;mihO+)DO`)P^BU{SbHdhf|$X7fv%h*cEY< z+;x}qLB{SPW6_#P6$@v|l<)Slg>HlvA6N?RcZWNp``7o<*M}U!JXm)^y|{dAH*Az_ zsofcRXvwOj-@dm}W=_1gWA)nb@Cs#+U-efF2GsTf`W63Ddorix?^-bE68@ToMnKtj z+V4Oo8WiiDSB{JGfA^u^B)^(v{XS@YOI<58=T|+dBUQrUXx1W-baz)>xw{V1H+hbZ^>1au9cy5<^ZJnHvo^d4@H?xHVO755kESvVlw zvMcv=TznUQxR7N+`hI!a00{WBN>pd%r~^=zyG*2jQVkDPxELaWLJ@k+Op36(zSzpe zsFpyK?%;?mQz`w{{B#)aF) zm~TT9*EqHI`6L?kk5UPHsnMVSQ;7W_cTm9L3C#t3#9l}_yhy7e4m7VDJSwLFdH^&t zB3X+Umkjir^|tVMg5vq;AJRP<^{+&F#?oCPaQv0}c7`@IO;)l=El z!$iYh^nm%Mio>-!l6$}^Lx_c8O_))3ugz)JzOB?23oHd3W3BrP?>I;C zS+`a3#ENz7i5*;efExEOqXstxwOo{It&(!`xD23GnWOHeo{(BXPdW4&x}Kk8 zyk1*FzjdIWudWb0J*iyuA;D4RvCur|um^-FQ||wM=P^U1^H!NHX1IU*mTGB)&3N57?CHLYH)2iy23rnRcrb_tc7|+V~ za>wUCUVnzAZUwVJOj6zzv-*DBWgek7XbhmimdLLa_fkJKLV#G2C^MlUM8MX0w_*{c{Nt!FYox7Obo~w?$xbQKrfM4ve zz1VlV~19B!A0>5br=4tHDk-JN7nL81%YbR?zPm@kj?hzcxTwTqibO@&Y2 zts3}U7`gVAuam|35K5xX9}eJ7k|{&^!_TFhX$Vck(-P6*jB7t$5ApJq+X{BQdhhJ+ zKPjAV8Z0|gz$!M7rEL6lrYm-j!y4K66=*)P0kB@1 zx!qP4Bpz|p6GEf-D6C)ux0zCBe(Ae!#9zVu<_@}DUfX>Ym^wL~DA*dpqGCk+#-T%T z=NTpwn3vF^f|EaB8XLIh;kk_^5R&+xgVBd+5iymeF+G@UIR)o@qf_&J4#C%C{nNdC zn-Ev5?|o63?A4rJ9!G9Ib_p1ashjbi5du|{12C1o7<2=n)pI7S_3Ju*lU2Hy^^*Id zKh2|%Re-3X(s8<(nlYZm#rHUY%zv?Z_Zf7PIT#P^=u_5sdax2UY9qqv738B8yU?X< zrfRWwJt915VxINInK`-~-5r0nJVwi-0iI~E(ruMlJ*aIdlai;I&do(G_h-6-=%;cb zhqQSDcms%o5*DOy(7m$d2SBwMnul5mXKtN4$InKbc|btAe~5;`2DjgPZ)JAcx9^=W zElm^!@L7QNT-`xN^T64zY?uNRPGnLt2_gUN1;FFYhZDpvmra4q0mj0h@e3CYYkbyYmxk7yvfpn?%?wab9{_|hJNNS=mP^_+R!=WipK=O zj=eCZxQ3|#hY*}bZML&nC-cB8gANC;9^^a4v7uv^;(~JO#Gdfe5J>Lh#)H~M|LKWI zrS-krHLY3=v7g=S;9nk$AYJ7nY)m5rs1}&wdDROFI^KLhQnUCfM5g(Eq(7#h=Z=1h z?bc7VsdsFg!_b_w0}5Vm3Lu+WbSzV6_;Q2-f^lUw^6oLt=FD83f96LGN)Z19``% z2N^EjegXuh;wsC= zUmO2?==C|xt}RyJKpowMdJxTn2@`U%-Ss9dPT{m=C%8>)kicYq<&*{ZIjbCn`mBhy z`M|Y#XV34!+m|=9%(pq}m7k84hPxnecAzq0AiMX%vI<(C>$yt|jNRC?n+fIu#7FZ0 z{U-gam%|koPqor_N?A`R`7h6JBwYZZ7STG~bt3$z5J~ys={GgRP}EEQJr>g~fs4!& z_ag(x_lxr)f*V$z=fn6o+?V}bbn!ReyBe+-4*Et;ua;D>FH2m(d?tZrJa4XY@;t%*wGAr>Z@aVkqWajUy zBR*TdO}TL(eW)C5`=k2!1k{Q6(I(q)vc!Na$RSH>0Cn-6&&245gego~PoKmt$2Up~ zb`<+*6JA>Ms*PuxWvo{PLyJfdKzNSV-wyRdB68f0P6(;s9Sy9hRV>8&6u_Wf^DAC5 zy_Gl6^DVa0#3t1KIlnTPky3z{GThoiHy`cNo=Pex`I1Ai0%8}o(1UZ)sR~6yJ10;{ z*|FwHcnqj|GS}y7Fx7RE82P%&YGaj;A$bBgvHvb`pIy9YsH>hFBhi>u*(?FcDO+Tb zlkih-+6;w7&8{#pTTqtfn$xt?_4eB<`)ydjx`16|GYdzFpP+G_;k&_G=Ooim#XS#I zJn=0&-RBFp-dAPl!XM#!j;Zef#TWN&UJE5fgiUjNSZk4h+y&;N{=GW|Nh$Xyrx?0GzDKUwp+fe@l^`>~n^W!|mn~ z@{LA3@BJ7ndL=2{I!MlDlqLj0*{OX%sRn~S=n&-9oX{`ft=OvX%%0N67qqB$S_-)U z=2Um$ACxNfa-^@U;6hFNI#fZ_!WwlFnXCtlS^-(5xDyy3?XF56#PRGdF)7^}wrR5q zzU)#0J}qjJdOBW2d6G%7ivAnxSIti>G5Er~h=NzDxIw!xH9EQ5W+G8bR67PgL1u21 zDYm&8x7bFeQr3^b+IA(^I}yRbZl2m^HC?QV3dmj#vpNC!2W(1;Mjv}dGJi5+QrE=U z@!`p^m!9|9=+Cl`;Vs?X*a2h#=P;)Q&5=z^ZO$aTjWk<${S0@4<;rKYR$BPGa7Bd~ zD=jXgx*qAFGD-#;a8fdy0G%AvIF7?JbXwi=FY*Zpt6&M-VO}pWypsHii}&`}Zh-W- z!&gjM0=_*Ne*0aBc!0#m@v9RquEEx^@GOGyyAlpF?cttE$6(M%HFzz z6V6@4#$RL9udwkAuz5$iy9?ueYoSW)azCcKRzpYu5?3}A%Y6wR_POi&zA+^P3_$r6 z1$fW!?6qkGZE3h{!|$^M=sH)eDGphCFu=S-gcqIe&$XSx6KNvICcT(eH3=$7EhsiN z7ACy!>z`VfvD`<0Xie`BlFjK7V3YL^=)1?ui-)aFfe{rP7mf^5OCDN-p@ODJho&nz zkzz|_=#jaYhO>F!tIM$VVIqn9&CLL%(S?pfN=Bf^)N*{Yo)UsN?grnYR-o%v78=G< zh8y~B1w`tpBRu1Hcwo*sV#D3Jg_8z55P&4cY{?Aou!7UmC|LNC#iLwj)E_k5eDeq) z#jb(9)W&@*p|)LJ771b91jr5J z_8Z=hHD3f4Z=EGh6@~c+FnCW-83>ptl7o|>?l^l;naN#WolM%pBMpTE_m-Qbi{)P$ zWZ_zh(Ng&3;dzaTeYB zkicZEraV-3rxjkX=}6G78a8dc>F43Lu181w-YHPzaOZPRq-+EEiAz(lo3VDV6Z8(z zsca!rK)Iy4JEbHH7CWQE?2+!7`CvnPFZ0!F9)%f?n`i1O+~wA2ahAU?nH-_62p}u{ zh0h13r!EZkV;+8}8tkI^Y#%6?t-Q*2KG}RRY84T&!Sf|wG;dUKpJg&*yX1n^uJS)v zy)tv%1gHOTmwPC?-EH|s4*uNxNyD>#mPT^JHbqrDRI>E*i(5VCU12pd++gklW#0D1 z)pRx4K<*+zcFX$Tn`g5M&7QliFbv>XYo=%Ain!>;vwb~a=G0W^DeDY71I&u||6uFh z!$zc^)<*<^Rj|mA=sU(M~2<5O4!^~lo%Gp9rb4n^P=i?kkVZ_WK z<~VFQv(0Jd^qu$T`u_3z{yy&O+8^6}xpwcq@7L*mJ|E9tN?827oT+~cA)u9V)t!6B zGlQ>`e>r3|cv4hTQ5DaTQbfOuFFK&*=9zsxuqS(Fs|5uH|MF)EAVNk>g!lbi)HPT? zSF}dsNz-t2?f=Uob)cDz|G@T%J#IJ(*l)R!n0=8R7Q9g> zHs(dG_U8TK@TOYqTA=WB$%)Tj&HvgKuRQGWBASBgd;}a-MpUOlorp(f%U}i7)z{lT zA~v4B6f0V=%x=z0(y_Gg90;^-g+xm)hsaL8zALTgA+b8=5~?>Ngr!eY;@?A6z*4W; zm1l@1(zZXPcjcuWACyDcVmeis{bnH8m-&*cYUu4@3UxC@T`WqV|Q0HQ1Wft$69WkEd5jF zy9OEm)JGewDeDZTM&)zavUbhjmVbR{bMN(;@1Opya7#-k4R9moOSA7JSYticT8UTlM^|qf8yEy{6(gwN@V;z z+y1^aT3GA%@43rYHEn^5g#n>^n?(UE3xZ!(D!t)*<@fvNJ(DZWdwv6_|NPrKC3$q@ zewLw76r{o;TIos0c(LjTTB#Ut_=F+Ki_4iS!MK2fM>+VOxL?w$TOlqIE_6yPV=uP> zCFpl9OZpO5PB^bLvM_H{Y~N$lNkB&FRkkqREAfMA)$!vk5Hgpd)FvHiIA`}Wcecf@ zHY^ff^ow_&F;(%7Zh1|EOCt@IW5|nzM_KaLbdtD~!lkvt8NKCUt{_oSWEplu94&@& zOM?eT&d%wNmLPSo`4P73;!9t_y-$6kF1Yv=>1w@v;^Ulf^TBfYz`6Gp^5=+86zXh$ zPn)UJ{LaNmlRWNU3dUZHpmewA&BU_qm4I}c%a8%#GJ6S*(B`t2-`;2VJC=xM z{E`_>hrbooOK%CFpB?cOQ1?FpJ)wA>T^?JUP~f$m8T9_GKX*!FC$|pmWxsjJ4b9+G z`%Y2kTN5xFNUUIaj8!c;=i(_|!tA+iCm6IhfidNQ2Xx4aJlwc1nOxxgDEj#8jn$ z4ak07-BSn)_p4rQx<7A3RCA!(>3-u;C6=+>5%W>HY;0S#$FrB*kkKaxOS2M$h1cCQ zb#O`ErsgQf5j=4;&0PT=u=zPSOD7)p(9R*p%p_~2M8cMwzj{nAVM&z6@s;c2ggfj%)d*U3)|w;h!UPNj?lZ2D=!LK9KP1=SM}xBWC4rWruGmQlN=q9EW9C z+!o3ZdIA>6c0c|YvK?Daytv(aB`ilaZVB335mqYtoPNwS(+}QrPFfjD#yd6t;sd)d zXI?zK|YB25LGU{r8I{w7_`g@RI=X^H}863kNJ)xXlT)bU?q!VFWS1{G|HNQDbC z?So7AI=Kjca22QipzWE%Z|-?VNR#gB1?e?h?ywukDR3ZJRyv(H8ugZ^K<%lalo#sQ zxACLYkVj!v*^&H5wEPlWBU2RD!LIAWA3YXqj&lg^G@nh$e7>R3LFrV3j)TFydB*R3 zU`x5<>=L}@8&X#5GYQKp>o+)SdvB=`8so&n`q4}-mvuQw+u_*gh#>Z8QTy7+mwSNZ z0;{jCHREm%?2CnkNNrCUjG3tnV)gwatQ=gvlC%$RBRCJ8?&Muj$nBFO+>l^z_L8 z83{M9C8ztSuynYkeH8jZl>%y_@uIh9s<$N!xjk>}wf=h>9D6ZkvL0u11X(m^YGvr; z*~Swm({>IR(7as~eX&54pCKJ!0vSyz3#i`eTn7ey0|gFRdEPcKVb_isyBmH>rOzCC zL|N)$dLqe;k3$kTN||cnZ%%JiPUb$i8@D!D@9LKQqx+nVbOP4 z@~$d@tAY8mp0izzU5!>BJhL_1NL>8UtvN4mNu}Xy_|uGYcQd3SJ4$WCeEhj71LW|l z#B%7|jq6A*c$6#30u{g|tmVb^#;}c>c;q-JoEsVaOLx0hMT5LJPQkB~XPP1-@c#mr zO$a|T){NskXw8>qKBI(`q8wS9$Tyv3Jg5D0EuQ1>z{pIcKwi;6GTCe#c54|uT3L1c znnWa0Tsz_ZnQ%cbZ3=_6%w1Mb>{h1V0#=!nn<;Df^z~h{h^Ag1+mUJ5kyHR)h=Vk^$v%L zqo+tTSle#428`h<0 zxP?0s!AxlP0_v!J0U*eBV|Xr%vH;~mfRrRU%tCAE4>myMi-C#1kyUWy3!;5P^P7|y ze89K?XBrQ$j=)?(r04pU{~XWg`pXiJ(E=HHixE(|0uZIwY*kV%y+rF+{#vK@bw49F z!6tLD5NaV-P|Pg7rol2#GRH4|d^agce7khBi_27O9pi(t-K(t>@+k?|qS#SVY16h@ zZd;E=^PbjyQVnW@q!TB(Uni!5&KDY(tK7n8Wb&xe223i>t54i5@zVawQng*q77*^_ zL#Ft*S{;3R)ZzvgVB=q_jUU|tC6}z*7c6B+N^l2guDSl+S6MHG{~HTntIN9As?DlA z@9EfdPEu>6?B3o{GR`_yKKT378b1r(b)6SLI5!+I%*iM7@Y@-FyKOj^^sAiceYm2d z9-uJPjL2iO!fc_24wh@#@cJ(-Ifwh_#K{$^0!;P&-vEvC0$v%rF5j;$`}bzeqXXx{ zSCd$wjN>QZizgw=chmiMERX9MI9cgKyM{2^C3=CVRTalMJ61Zvb%NhfKPj%!Zc^>- zTtDZ$)4J&P-H=n=6Vx2y!|9*MZc!$a#;-iX-Sm6XFAdWe%L~bqrU&}bvLdkUl0UX6 zXA83Yqdy4Ng;RHvbYq=dCCAR8sv6Zm6c0@Tc=ngGb3o^K-70|2h~Axt=7{ z`5RgPGT~={^3+|yqvx8)Wa?2(LxD3Wx5@Cgd_6!Od3Sk(jPh_CSPSUc{;svc9O>9Y zlsoAW&b{8zfw#-JH1kvFhbXTqVsZ-dc*Qj5gi8scs|GE;43N5RNYRkhn@?I02TZ)QRsq2sxPX`vv z3{x)8sntBJ`J^a0*Wr}%kNr1ku)1`vdv&}}MoD~Rm&wJ%x zh`xg^zjF(SPe0?&t;x*r4dn)}GdYO2#b(LB zJ~j4KAw-T?!R?(Nv`=W?BDA@Em1Wo+HLr9x1ii|g>@teNaz?TeR|%P?OGW?6h2yh> zD11vkTiVKQ?SL3`InPLhOT_7gSdeTX{IX5$Jhj@0`)WqdiO+XeM|d>x`Fb-VjRfyw z+!tF>jf1bW7j_*tE@+UgCH4$u9;)0LSEl+ALls=NSYk3G75?7k&;H&s{^!(eC8%bmn2z@9(y;EC+;kA}rUF(sCE z)_b-3+86z9Gf0_7&L3IP5ull@={&FXmm7ImgI?VFiar(R)fl9=0U-ieOYval3@|=S zOh2~3hla*~_v|)^(4qRvP_>oLV-Wv>wEWV3hc+2nk&emP7loG1*Abp_g%s_#w<$ON zEJr+@xR4Eq-oH5fB3h-X&%1d+p_9y@vNF3*dqQmY9zM_bJF_m~dv_1_ugGN<-w0(b zx_FlIMz;B#NLDkjj@xAY@tUy)l*pE8-FK%!A1$|g2Sf9{U~W6(NJNUE!Icp|J|HzR zCUEyj)3(8vPI@Yq{Nw*K8uPg&bs6Z+;q+wg-HV9uQUkL6vIVmHaU|ia#3r`8BJx*ca*af#V#{&r?@-Lj0l@FN?NWjgVP=&ubg`56mMF~ z!J``#buY4AZ^^vP2_H8(aD_xK{c7BmE8tMUtxiex2r?D;;aV27;*bhQ*{PN6%t^;N zDI%1{;IkbQop`y*u=cK4$GWc7CN(A)AnoF3IMfn{7UN{rCrNm)bC?|BNnX<1iakh- z*ZlH@TC?~5uK~fS9jNZO6P-pP!ZOYS%1PSctq2NH?7J0Q@iy@qvvqXRsW&1|a8Hcw zefzk~ef1Uvc&7fIw@@P>;Kfh7+&UNA0UJuIBo!%tSMeyUb54E#DL4L~*WAyU!ur;+ zUS|w5r_Z|RM}=-+Fy&@0em-4SZO^Hv3#i5ais9A8JuKn3h$zj=$W#jA@n2U8v9P;t ze#_P0zcmRmN%XJGi;$!|;q$}Y)HZuuaH$sWyC7e#^+p7IDNb*lw$gu+VO!%Ge3=!g z_SNB}!JW8~OBx({)RAoCYmYj)`R1Or@LM%TW#Lj{p~JY@dkp-vKfPgZ>xeda;h2M8 z8UJ?ki+X41b#58E`!*h*(R1>{su2BoJzgsz zXMd;EERZH0=?^s8ID6l8*JoLIkK3hV)x`R?VY=_gt>8-?&1mS3cq^q#7)@*sK>W?I zcib=oGhy_?PW)f7u%D3nd{?aj4DJy&9w+iT_tyi3cxYYdv&qA0N*g#t_u#YT^Lk}9 z^@0n>F&6q$rraVMNWI8D(~4IiX3vES2qfpe&GBtu31d4yf(pwKdE4|pwmVwy@$aU4 z^4Iofj7*qwr@R>)+d~u+qu$geu9=fIJ_66y^z|GJ%mKfUjmm+U(bovAQI3@J>M)>s z@22(@XlqI$hwv~g2(nd%d(KrTl<}O)RYVTzEPwy*!6xx1URa_1*JCqHCDAjns2N^6 zy;-;Er+i~S@7G3^SKm%S>Tn!v1hLSH4Q)46!o_j5`&@fk{QialrCHRBGUK^R>E~u<8Uv=n6P@?4)dW*K^ z0W+8S(xSKWK0%i>$o<7`S{IG%j_@~a%ck!wtnF5uO)o3tc^!17p64>WK5HqwabTt3 z*Y|LT(QtyE7eW3cV^m3A-*(=$n%69K!tGP@=@75g;BUUu|zKyBDdW6)83>pFkDW-H8Q+bY!C|t)tF`Bz5CD zUDqh8xIFE&ksP?+x(hHqgO*c0nrpFOx=VJ$jC|P7DN4iv6*|{_D>}k3cA5a$un$@9 zuV|mY%Ixy0VQM?P;nS#L%)f4G{!q8k>dYvM^f!D^4KS7yfWwJo@eTc`kX2|>t1K|Y z;^$P(e4>_6IHcffM`AHN%&cy|U~EM%Z3;E?CQAp~JU#RoU{Q9IwUF-oF;aZH(k5KE zPV>%dQ}q$~xuxs+Ky}(J@6D5!pIG_Kq`<+j;f?nGPp^}!qS5^#ZKt#JehM`D^GvRM zMf|OI6A#=N8W9Oywq5x7(LgE4&!%Z4n*o{_K)Px~T@{^!rIF?0e5>|Aqk zHjM<}hqMC2i{Yv`S`)3Km3NNl&U0cPQvbcjCTPoJ+8NzA+*0$>cyw?-rfE0Q^6YBo zU|i`V$oi<9-g)Z>4Yu)XzxvfLbeS@-mRSP;rr4gsBx)3boBYu z1v522P|@RGO0biW64!&v?6S!KWLSa&Q&n<+c@J^8*G(dQ`CvTJ!L-EZ4S@hLE06 zg#0M1cJ!P2J7y`s=ahlqTZ$8;-uWCEulzhFZf0O^W*K&+5%y`IOsIA^@7G6_JHCD$ zUWU)ZUeHVhbZVvBc?lD9_e2(=i}1fDU<$t4@;8gchMCQ;*nt!^ye|xeQhW@B1-xpm zUU0gDR~Gv)sE|f4H!!l=!(0D_ozs5k*J;$W39LkYpP07{tqw=`(DIj7<#+=XgB0WQ z&c8E|agi6Jj@qi;xey-)(h^&eS7rK9WMZWu0$L%s9uH4rhxpRXbIp`bwKIbwSqNj- zYGeLXl9;~dICl~2vv*JekKn|@Qe8NAMFkrhOMyG%|IDvu!Yd2G zcYL9sr#dOWv-jQ?KuM%iIGoDR9nBw-8HhohE^k;^grAbG*nr#VOb06F+W-sKtpu*a zV7W8>;-2CTp1km&F#3$0^B*x17+u#F+hO!~Uorw-F?crgyY4(Fb}Fy1acU}GH&w3n z&Y$j-_;zmX`u@ToLp%K=Agfy@Wxa9HloSjkF|%MBFkD1q$Y*wT5=bNWbIkLS1?Ou2w9-e73tD4c*_ zk#GP;k&lA2Wigvynie9(E_zIhMW4?~kE66xQR7V=PeB18VwPBpM?o@1L# z_-WXPXb|mVkwHt^!1Kbbg-wXadZ_xOQ7r3jmO9-#VuVp?w>*S>2Z#VVJ->Racss_a z6yan)V6`}T_wsJSzRP~=OFAZy#EeuWqS9mkl^0TS=)utDqZT`rXSDJNceKQH?7-r7BdhTqzw>x3Z_Mk)nTfl!k9W zc}{55OJzYYY7e1Uc^~=hjH^;ihP1U&7LuYGqTaq;PoI9cbM0k&EH$C^iJ3&uT!pOr zW64m_JAO-Z{rn=LD`-;lN`;}2REXjetmF@hZF7Q%@OJae>@>t1+nZdJkXyT&CHk7z*@c|LT83no^ri+cLa|tjLL=SED)8e!rmAr z>-ozn?GFF+9J(#}#I)?&C~OKCZy?NZfePh5LH4`!x_gZF*@>IKgXC<2_E|5FGyXD; zABlML#z{<*KJcZ`l*w~KaTln+Y%}U=4&&qod%;WuM6ewqBB5nR+g@}1@dQGo*D=*g z8N92*+ClP9NfGub_KNaq1jNZMmuK2c`3tqMswQ+y>=4cLJmvugL0uS)KPQDd- zqqW}uxlfM?tW-ijXPMimcnP;kn~dODcs|nn6M;xFs;=`GT|?6<{B5r}14 z89V2Dn@&l0-Q(w-X2;r0S%oeS^n8YIoT0YUpV?Jik5eMcH5u?d6p(q!eM;b4MA_0W zd3d*nz=(|A+-xv1gXsqmKLIEak%9s4_4OteMsI@OQin!a z)7riv79_E?Va~8TxTS|6Ntx0Hbl$Eok9{iR%Zda@YKHUhff)UkPUpvWLR8f1k`RTv zJtNCyqh+J5--I%BtLX1*?2KM=V%Vk`*1{aauR{U{gW5Clmz$cp8y&I&^E7b&~)VJ?r#KN6B|GHfY=pP62%lF{F zf-QGLyytHLqI?;WzljlqXXyFqhxs@e-9QiWs3ljU)@Kn?pNNB+ic>3>D^6HB4L?7g z+uifl_?Yd~sdXNRVsdZU4EZ{0g>}?kTmtX>kCXbF9?N9zrV1M^q;KWzn^vB9XAAPJ z%+YyOdP&r`kfSHhV;Xas+(49bt=~rRN3`a>*8@0tWo&z+O;mD{q?2}tV9)DzgIm3Q z^1~fPSr^nV7Ak!e@C@(qe*rO{00e;`1)`?z?H@-^*UI`#@KktL;Bq`x(qf|Cc=V&AKc#?0N8p zLW$c));OPDlrnvdEqKXkKV&i))k~6!<8;3rX-Sun*t#cH*v}(jYHqmOd(WHZXznAG zL2hNBZe_94{?p7j818D|D$Ltcv4Q+t(hZ2I$|2DIp(P3k!_;+2QIk)N;GdZ-m0hPi zs=hfYoAZEq47scE^Cm{**N|r-hC<7=ja`4*sv~$kj6*{#ph@FAV}<6b%YL1}`mm-j za}FOvr?SO0SpB*olaqh0KTaf9e82O1YnPwhc0tHY8KG!7Px|y#xN?5hya@c=N5B;? z_wtdN&7`LaZenx^0lsD%>|C9b1Ez=IgSa|CWQp9R1O&Nmb^6FIu& z-;#S@i*|e5Mlcz9G06dp9gnlpAL|`(!EJoPX>+xif-kw1({GWUfd_E7@F?C|*V=$P zfk<5q)kM4JM3=JeWXgAC)cFwF{ZKIwtgbo2T=}8|&4v$6CH4zFuw`5{?Q;IqjYRRu zXEqO_DZv3iS=cH)e$(+*oI6ls{?dkHrm4F2MN?7lSMK7rBnp@7!s(FRejaJ4Gf_q< z2&>R8X{kC4c3dYt(tD_`L1hQc=T*J`5SHw-JyOsA-au$#%09Uz6*_@`_K$ZNh3}w3 zsH{%65ac3eP05b?D2S%_Id*_Rx@CDfG`fp|HkMNse93JM+FY+=6RK-Ye~B{{a3J`F zN$xpOYl*ojdE1=&aEVhccU9MI)Up@9N!nN4k6h@P)$)6p(KJ)#ma9wsB~lEpt~%fA zN_-UB0q#Uv@&)>-PW@}~x**OZ1?`j=?vQANP%W1w;ckc{!t<91Ddh8WE#mQ@N6wGA z*1vCcQpyE_#Z{5!=p`8u4d(mS~wmArJ){F#v%&~u;X<);it ztQFdoD&<6AX#UD5-kR%-Qv+gtD~U*}+>3RD<lyT$9c&IV@pS({ZZuD}Q4hLzpiHW^A<3 zb~U)&dcg$p!Bw?9X+VfK2F8;WefL%T1jXrvz^Dg023JFTjVyKz@R>=?Y`y^sY@V8T zIF@3etG($7Z1u|HIafouZ1!^CE{za67dEj*={)u|N!>M|e z%-oV0=g2|)(Xzx^|2KTP_!-;4p!O(x_G1*#c$tz$%0Q8za2kzl!E)+NOzCl=)B-V1 zC`qYC9%C}HwBO2AP2-ieCAEdiQtgKxH7CVlt`1j=EPqJY90b)O`9u5enryDPeJNxe ziF0JqlUusblnw-Ke2NF9IH7<{5uS4OR=e|7%#j`|tg$^oEZhr2=7B#VtsM{Wyc4(! z0l{wv+qCI#u#-5UR(eu4wmQ)IV9e*p+X&yQt|!}3b4&?Vf zrl>0*&sGxC-9A%RvCpY*yJ&^yIAn&jsrlA$9L1bmOO^w`N+yC$_!J#J;@d~HgPc&I z2afx@^x4|9P32p{)hgbRy8E@B;deVjb+ns4) zQIMsg;h{{hjASNUJ9Uxt(M8n%Chx^ZFw@C$oz=Ulg1<>zu_}J~45HsfSsNzQfoKC3 z7q_QQwcY{uz>piU;t#3)U@`^A!}Xwc*4bdPe0S+6pO;z!p$h*XP?i2A#bQlil zEa5SHbIST-V^ej>*2P2zb7m~^(Jl+i;jP;7Y5~0E%(ZwFJK1JnI!dz>)bpkngSM59 zRZH(Jy#>_v1EdLeUC{KqghBDtE-T@U4D1vVWt!!Ywl`gf)iUhThSyL=X7VvUep6wl zLpz6gq{{)a!)*O4QW{={!-!0glt2yBNg0&%NU4G7)~J_@#AzfEP$+{#m9}+E@34c(=-5SuGsm4Gn zR`EK0;k zM}KV1Y09k^J$cs`?abPzAF>d;J9MQiobpF(+Rg8v%;;pmX|1V znN)?XVnHwL{GzSI#EOTlj1H6keL^u}&q0@p|eG7=`W8~9~Fm`%r zv3Iq9eqi@L6doiG1R9%~u#DDJ<4$iZ9N4)sG$vaC9Qa zv)epTctvw~yMWGqCv~;IGcW4FdNW~bDa_RQ0ZyCHp=`*ZgGtpb0zBUhX|dpwGz8eK1*Dl02>4bTdI zg-jCR_0S+*FL14)%lf_&S7K5=LS$HYVex25hi%IhayBTmZ({|lOmK@h&!soV9dJqL z!Ra{XOCU!#d67WfuEJt+fwvBRj<-Ono1zS{Wtq>PTW$kBEDPU?byOw3va(mX5zxtQ z^hX3VsObC&;w>9iPM(G1W%QJseh@LS)712sYv<7_26Kvo_lI4r_M;J=hWLEM)qa1^m!A|4 z{8-nS;_qhS^V&a24{qdiKuzA$$ z;!3Oe)Nm`YK#%i96CIc_?LCy{cB1;IGtw<#JRjcqNr;-0uVYK$NyjqF%D0@4iOV3^ zwh9pDLT%kZ>*oCnaj!vdvA<1p59JLXa~*85{;{->!JyXm+t~&|l6fT$=J2KhpmplP zJ;|&8Hx{4}a7uRY#ssG$8*T$0nY%Q z_51k^foM;Rj;A%r1GTfCCtTsZT+1>j+;ou8@f-_=rsO*%atJ=!z(6SpMf-9>Rtq65 z&4i{)NV{5Pu zozWTt(aBZlekQ6u{u!D#KOXaJeag}n7i6VQfJR`j9H;eaKa;1=ds`J?$w~FZ#1?|Q z(-`U*wv7K_;0k)?3e^nZu1Z_e*>3agyC9-Co(N1^Iw}2aw=)Q{kFra@F?7A%SKzY; zA4p<4C8on}+(M5rCHIs-Rcd62pk zu8D7wO>X_;<#i?^aT9qQh|VNoMsAX@M}{NuLrU3FnD??o@z#PuofH0AzlU60Q!%7@ z{(+bkVGdZZwItB!6Nt|H2fGF9%(JU#P3e?9_{}Av9{f5kP7+>bp`kjK^hSMI{CDX| z{SxO+1hLgdZ-kN9>I;K9ENPEM!dIP$Vdhn4n6f@w;hrMJ{q0lROa(jk^DoWOu&Sz6 zzDmnw70B1+prMsT$zYv5jto1A+7~nEshr53`=hWB*^EAz`QlJ#;?)T{F_d}{$Tv)G z=n^S#U3F?%%kZ2=A^d=mTW)y*0pne~g+f2npst=_fRO2_1CR3f7yDNX{jigmV$6&G zp%VYsS8|Rk$BWRdV{$s{3;ymE|J86itmF3oZcsIW9|kQXi~6JgQ_A+R&iVhl>8vZ3 zaPo^ZXgQo#`u~0Se|OI4=43yMRO?r6`5&+Oe|r^?7WgrF6E-(M?Q{NLzqJ$lUGkFH z!|>PT>98w@^)LSS4K`=%6f#2VE$q)H{tu7-zrB(ap+lVf2W$EVzR*=n$i!=3kLjj^ zohHz2HUhQ381c4jhk7g7?_vXlqJ7?UGgGC&%AG0tOwR^!DP&qS3w9@f>?MA1a(VHi zrDmUv)|zg;LOa2Cb*F-1HUoCa*tk01J*%A`f`Z}SKdb6g9-<PDK@?<$1E@7GP4 zu8jDc7a;HLfUB9FXb-_WjUHCJhE#tR}=eVm2Z1!)Z@SyK0LbV&I8CnkV z?V7Apj;KkvUO{Zs=HaRip2-mBC66w*W7xr+DQN@Z!%E#eEELa3p4||6MDZ zLpDe=+2D8FgVlB??@J!4vEc{=3-ym7tpjEq>1OGHVU4^Nk*t?%pO@t!SKpfdt1xxX^4Z|}b*~fXo?y}!Y;a~=C9YvSqP93=kVitkTeoSE zQER>HuCC%z*r%N@>BtH;_3g`a!2!&HG8q?VuP}EmdM}tOulhk(yJGRWQ=2sQ(K2_i zX5JA01!>IF-OeUOU1Mt!`k9~C8=F8{N+fH13SXgU&_^y1H! z3}5#vC4n)5Snp>2W{6)hk1ytKNTFU`%9_2(xD_kQyxln9@SffkV=pQ^2CMzOAy?r3 zkG6Cg|Ls*@m7gc6uE}Qg*!s*bs0vq=-?Q~f{mZm;wGAE5g+K_OH70~~rBT!|a2+`L z6t;B^MEa7!=(uH52NNmY^-E|P+Nuk>p{<1q4cs!n_2=R$ywO@B@9g-f{{iBIsb6z* zz4Vy1-|914wKF>d?5mU9GCH#~8Atq9q|oI>#2(wfl9s}retXet1gmxnTX>qR=G8Yo zsVqHSZQmN94?Ojk$sjY%zsPa*WzY|?B>oUta zbz@(m?SE~R(_w^mY-%c378nkG;PZWoXXv`HhpED<`0S_XzKaK_KIf<(BK* zv+0)Byk+|Dpr&J)zBoBP%|dhRmmfR3d%A0!>PdNQD@K{sZB_2~!uX#UB3ufGOrZBPVcUUY@h=l33JxoA8< z00ZZJ{@o9_esBt`vmNQz85#(35?T3F6idtqgm(`BWkjFpf&|9m#J+lu#g!$vZx&vh zvD_%#aX9$47Ual2Fj!xCvje}}Z{r2OSl>R`P%t!&J4t2PuSRs!Fz++T-m#;M%{o9`m5204aW!2nD3IOtD}%=h=ofM zpLG=$l|g}6h5F8ayP>rDOUf*ompHJN!Qtj-8Q7`qnpRwH^}DWfHSw$_ihjw*SV6A= zQn}~wrai(|Oe2z~Vr@Q@){Ah3X!1=K_Kj^BmDMg5*yQ^j^Xkjp%8bq`n8TF0gu8@C zR`aafI+MeH)LMP}a_Zh7WN)zbFcp@~eTf`E3*Ycphwk_5@AbE)gpg9C8<$5Lms{{) zzCcjn6j!%3Jw)?rlL90N{uGfCJz_PDuYWtZ< zAf8jmU}x}#jxQAMcJ-OHUe&COgZH?=u4Z}AzGQT+EsFXd2*zRE$of=LD|#`+dz5V- z=?}xXLz8v<{!@rN1w&dy(NMcwXFlfP|3@%?5LiE2m-}jz-U`5M$-GLp0S0};zvJlt zR4z(7zcT%tXr(VNJ{e$LwmeU}&t^9c{O$JXu^%7ZJRD(&ND7Q48F0SvadNO^Ol*2i zZqft^G{MM8w(8V0yxd7o`s=LpJZr-?YlAkzM!3XLdGfb|hf_uLPver3t%{*K3Ry0? zd{?`Yl0>#<**}Pg)s~-C0PK)@8VI=^Sg{G07jo%$gg(lU4_>|UzL>#Y&YKOUT@|u% zWHKgbLX*(Ktkr}4)us+Ulm`)AX0> z&|Af+9=}em%|jDVgusj@z+i{gkAR)D-PhcI;zU^4B3e3ZQD%86HETPyIy}{K`k#Xl zP0beTzad|O124saOBkUHV5rYbAv0DFhG9O7bn?D*9NlChI#AJ(Ae7dyZs_E4f)rKy0{RP1&Ft6YPtTG`Z4I-rqF zr>q$U$tE=}(L>l2StOf#sMgfk^3~s(N=4773WI2zASkS#ZIlR!ER_ab1lfh}w0ZBg zwc0gO?PMnAVkYKjX90|};vr`wk@yFFP#(PvS+(A0b@GmZDf#7&O{~Hu7VSTjYBs2N zc7*jes&OvLE5s}e@A7!@oRN;sVt_Y8-n(q@Uu~VPa(1=u%*pLcktQ}rREpaIO1JDC z{xX*S%Nda1P1n%PR&8vnwiIHgIy893u*%7t$U1ahgyP0GP=`0GK71|`F*n7w*Z?3h z&QeO$Eoy)byNFM+^7Q6yFPCe5^Ta8|{pLhrb~`cHn16Pof*wHrbICYTV)G-y0sB(` z1Z;!t?papP!h2?QuoG;IO>8W7lQJWLVf&*^x^OZdg8VizrHXkBLpv53xBV%O6anx{ z3&O4Gxp(gT@!tDWPrxKaE{D35SSz%^YdY=G_H1hQ|qWIEs1@Cys>lx)MQF$C{i7U?5@6`4tm9Z z7fAS-#D_izXx%=%9t4spzM*Zd@U|&7+_hbWNS`c!`RD!t7_iMiu&aY1f2RtKr3nsSXHUY>5ct4b8%8}Cen*s5Gum9Y1@i94TH=4L|Bu5tPT zC#nWk%(;;T0$NOLPCJ$*zXe|VR4y_K1lhabrO|%Vxd8ji^i_vPk~d}irZ!|0?OozR zcW0_O#;*TNvYZ|dW<&%YVkealjij!uI#`yDE-*^PMwNaG&B#5!%nAoE!#&T@E@2p# zTs-T}FMeLtw6i#^Ysy}|=riL&b^i4Ya$l8~7c(Y5>UebXx%R@NZT5xgdJxL8tP}5) z3S|C*?EdoPeEp|ir@P++g`0)N&JOQ}^XK%-QOv$9%p5vd>JscygutW3Cmq)p51}V@ zFH);HM~<^mzx&tOK=drhO_zf34d#{Jn@{xhJ}T*ykF)1UYu<+8#|xAyu9ZN? z#735^05JJdhbc3CGMAoHUL4;ju)yC5w|9t7NrHSjC9|C^+d{SXu~SpyW?t$cszb-L(zg1l))HU5tH zNV^nzSL#XB)?DBLnN6>DbJz`lcby8P=xI=0Q&9^7TUf(XYHqS&4}F1G{rZ6|F*ZnU zw62lhgs(IljTTAIIT6UwWX zCCaVeue1go_TLWfscPWMTAG(!W#ICCHTuXn-1ecQ<>?f1C}T#wxvXJRg0_=0?Wg1j z6gh2Olwo|a56;ZhT$CH>n>#q96xB*gqdpV|hvpCcP6@DWFqYj`Mi%P8)-$6o&wp#M zyJ34=&Ap7Q8u-u6F!s9l#hFr9>{4BNHtFB*~AD`SCm#_YX%~VF+hCer*8nF~}c9N}M2A>L>vSN?6Ag{W_$<0rvA1#eG z$X4^r9m1*JWv#)#Y~a6;VaK$4`|mGUUwdV|S{qi4RQhxl(%=$F+06y9a*G-*-dV;k z(`-FLI}6~Qf44~Ev67mtE=GY1wzZ-h|M^@AyN6`3X8~B+Zb_RzAzJKM1c%4PlC!tp zh@C>gMFTfazB^VBRJj%@ufNIeZnY&r{=Q3Df;5XU2fSw1ix-C|VKoWuG4jBZE6eGY z`V(!4JVuQhs7bHrE#OMSh1tX{DO>j$i2YtxC)DRIQ(HSdu*kMuT~kUvFMYnkkBFtjw2lPpdg0IqZCkWBV26%kX?ab`=bmH`>)|dhGSWH^QThJ?3G% zqi~QOc)WI@;Be2{ZTK!v!rIxaHO;JYTtfc-&SAr2H_6i_93u%9IBi7?(86h40qx0T zvS*KnD@bFVhtqqxX1`lf8*G)-)Fbv2R{-AE3ixT}o^X#`hjYi^b$DMywm7EBKtxmzvKK*%y4{UX|DyHss8oL_X35(Anb0UwE4 z)dn!Wd#0(eBF1D%4rPZ<4y_VdRu zBTQc~V%pJ!*Z2R1w3t9h*opwrlgr#7tn}hQwW6bRjz4AKh8ctlg+TRuW9xuX4^Zc` zY@8!0zPV=On)`a7tRo!($8i5T?n2lgucyMnVJ$d;iq5h~PJ#PT5RQcZoJhZ@XDT=6 z_2|--)n6-4RoBRODKzvc=-x~gXtRNpKnKMSUBcxE5ml^Gol-O%dg7-spZbK@h%=p^ zRDl!~gbw7?rHSioM{phvD*}izw4K9NQwgFgpg5pU-dVcdXbn;J)UE$2sygT*9wc81 zTE7nZKXjdER8w8^x9LTiic+L13O8Mfv_Mc4kgg(~08#{`cLE_85E1EyDkTCc(yR22 z5JC&Rm(V+e5L#$&yr1X&@?Yy&^YyHC&N^%M%azy+!_xwl1s>Jh<|} z68AtB8nfg3Bd5huKw~6>4pf(&tDAA>ZfL=4vwH%mn*|QniU)L*N>>^%3JY!hQoQU< zzyw|BJOu-cM_?QqP@5cBtFgPtK1-<7-W#kXnNpJTc;5uw=+l^5^ll z4tu1gD7@=zHGQ9wka{9#fd=cY^=+^}p2;rq?biN=@o4)OHY~Bxy5!Wf>cq%dkW+Q9 z#eX8cTuPl0x1uc%#Sh#hU29O@fCD$I-+z^C_W;bf5D{rlS2je2c8q^@JR?u$*<#l5 zqtv5f26?8Yn97+Fr!q^q+v?@focty4y5bg`UR@OL5sJ-jaW=Y2)=bb5IY@6_#-h)N ztxwCY8|fo|xT1J=JayPr*N77tk&o@ZynFwfnX~y7MCr`J{GwDZ83KT9>M1!e4^}w5 z8=AV$n&~#j@ZLXSX8Pr$xG7XG5DT$cuV&*!$rzhD8)F7+y5AyU7uag!f9bq*jUsnO z^+SVYez!k9<`-C2QP0$f5NaP1QnY$Ae~z`nj%jk&eM~+izIM+MX$INxsC1 zItoyq`;txP&pR_FiEQxfmc34UIlde^tm<2>ks$Lx&RE`Lw4!R+&-m@^;rSXSg!2&Gph{m6Z-<~vY+(O zvyjftDJiYs!|C%MEwoP{Z$lokT>7q_0=z4j6me$^u;P<|<1_M;(d5Leyq$v`;^w~D zvz4wvVc+Eak+lTo^<$Uu><09scd`}S+uLOTDB-uHY=G@Z^M%?og$KDKsN1By&?WLH zWSv^2+&*w0I2;^xMB^xcbu?BK}Z(M*9`wQwa&_1qH7;YC)bYH>HxnMc=(W zhJV(Sld}58#BXGGKA+9v8L^OZUP?J?Jt9PQgdM#l{Zcfrghu5m)m_9e1uv9rtiTCQ*-P?@B^);XF_N}C9mg-*A`osj!(p!IyVX~RuWXp4XDs3HoDJw z_B#E&DQ-3(5qqH}QOdqL>>D;S{>05It_XqTo5nW{{q3^Y!E9JCdNaffuw0`lFV2_{ zS~21G+t_=v<$KbwtmA57tA=RGN}7m{J?*@8>mSqy`Y3z&rM#@sR4zJh5#@icAV@v~y;XjvLxV8*-CvER2NzzFE;n1o7)=kew z#r|;l_H&wk6+?D_;8Xq2#Ur+o6FUo}^5ueT%+GM8;jm&J#(MyHk$)2Vh!X>O$m>wK zlK6?Pi(7j8x2Eq|UqyCJw*9#svos1iNf{W9@83`n-{WTxCBv|7?=*dtqvK844!0L5 zd-5XiNd@IKwyL`&s&RzQG-v*SsSL^j)KTS)vzAbOz`bxT`|@|~8|rQ@wDbHciLGU9 zKbk?C4jQ7X1NoVwk$!%FosH341~O|g{L-7j%uM$Jv#K^CR8ky52U_vHKp6s5kg*bF zVXBPglBFie1d80oo^cr{UOl*SQPd@Cp7BsbJ2Ta9(hL9Nl&&;Ha{Ixj<$IyRJUKw^ zUzl3Z9OVtv9EFoG8y(OavQ?rx(fF?0Yr7tdCxout1ougy{B|k7CQ?ON)a?u zIC&k`AG2N0-<#wi=iEv{xb2^SP|URFu+_H<2|lMuvn4`LdMkh(ZfDQhMw$3nNHmk2 zNz}NBWD+~b@2ZS>E9IkjoIz&BvOAHFb+ow@g7CH>uU7QN&^*JCTexe?dkT|prJi5Z ztaVyl5F3Lzl<$tG8mjV{Hkg5ylJ>W56)2CXw@diJa)JeWs9zUnY)QGo4vGoy{p6y@ zgFIhg#G!)>#Sm|bg1%UV{eSdo_2Rm!l%u~ZDi54XHRtwfNUqrWrIut@D92k3KS%uF zID1xct5{wGNUX+FkbFQlnK$BEF#P`k8~9dq|MBHNpY5_d+=`N!!Qt% zvD)X-TZS5_r!wUGKF)Dm=ws0-t`2E`!{NOz@yocTqNG)IWcrV36QwJ;B#=vAmQ5fn zr^VZrnocyA%p6}{YE)vot|DMo7E5UkcF0#gw9OibeLq>WjVBAxweudFjS-lx$zZY4 zYNe{D6a&QI-L2sX-Eq`+cWuy%32S3xR zT*vix8PG;Sao393t{5}3B<=Emeqe>MwBTd)G-m1UkGAc}MFq^2m5x4bG5BZUT$Dew z&o>nPk{2dsU14wcho5z$t_5VN?**BK^vq4cfB0Ry534Y|Wz?`lpE_c?%1@l1Zv z>b7rP3LX|UhfwmL*uZU07YwV3FwS^=z>x2nAomJc!zPMs;x-R@&|UJA;wg=!d`v-5 zHquR?LJ?2Vv0alD4{=bC?=WOJ7)*}kpKE&gGwpIsoqI*6#`{){`#}(H;Lh6zPkX=}(lv|S31igyqUUFx<> z-NM>+)}0uzApF3|ev?)dN{5H%@Y5I2Z28xoZ9klX()QlX@IGx=EcJP*y@}MAC*aY#Jds{ac_0E zZuCj~Qy0ghS}Ygsqxj*u`ZfkNC_<(;W42qXy;<{)E~$b-@HTNQz{fjR>;B`rztw%{ zl7P#rK;2I@+Y^H(frZMuV4|c?`wwX93&}06`F7&D->0!02Qi|YZkzoaoui>xC{p^rAg@_};ZdGL-`|d*>;Y$Wq4+)8S0MyfJ zx5`20`wVKcWiHBJ1l^gH@|dsTSze5B*J0?AD)`bSpZKU&V;q_Vr5)95{@qcxI6DDF zi<%vgtJExQgcx68^f{TXOEt;-IT4Wj%Tto0{j_ZAc!bZ?9V`2hBM`2I*sAp)0TNv0 zIyr($Jo6-%6jEIb|4_T)TWE^gWHaMhXMb=y+c>udtn6o-J#qHBZiILF^gF%l& zEiiLbe)%7Lg}F71gZBP67gpqPk$l$B25+m!+jL)LWBypFDn4r!yM>+Z8$?2?bt|v+ zOE{m1%MiO>jqhiLkE`Ka^)2og{$R?R2_6H3Y3S~O>UC@yiEWg{dCT=($pNtun9x5+*2=?0jiQ4c`-Scy;2X$3+?+vm1cL%Q*T5IJ> zo1<9V02uEw-pBJRc#1P$dh<-)C`E;w#u6jkZ>iamsT`wkY6!mihRs2*p#I$-!h!!B z)YEw50gMYjg9S|pGi`47d+DMJEcAKgShQGPPJPsSiB0ni&8m1zUtgNVdu{%%`w|GM zZrd8Zms;puQTlb~36`-Kt(o8c>SW>e!xw6Lt+njWKOyk>$pNw(!;BlQLqu5*{KXTB z`u$gzC`S3F5N|wh1?Ii5S;si)K5-s{oog;&Ab#Er`605HgPRS?&+$I5xck8MCjHp` zba4+m4z%x06sp;`uZb5OlF3cH{`y77wQ};i7450H?q$o%x!rn6$<~fxm#p&?q~cq; zFPl#8c#XVS6}b&K1X6%;LYQm>y5{mFl-YQ;tgKDyPj{&~IC9tyjG1HW+7MNb3(7pX z?LLMU*E5@VZ>Wr0s|B$)|D+#Wqps9Pge6iVg1UYM3BW7Y{0qHJFq|yEGWMAX-yf$w zetYUaB|T6Ty`)R`dbQfU(oukQtNjeP`a1T=rTw0w^kMv5pFk^Cwv#luT&WL}$dMaW z52<>ezFr_FnxA6{i_@GPtgH zMa1goEBuYn`g=rCgYF zMK;E1-3rx0E^+2}rTlRYgbveC^_X$pBpg(_0D^m8!RGgRY`K4VmSTc20|W>ijdB%e zR=pNJ`|^rPI3PZU*4y_v@-)NT4tUYhvn;P946pm{EJ(IjNmV)iw;u&{kBvD=(?IU!|z?{?EwSE8uY|ZmFbrLXQO2 z_q9dv0Q?DBWd7o%<)@~OYQlh_8Dx=34fDCt6dx8^-61=l)Ax0GjJvf|Jb>R51zR;J zu{w=g<`1unSGcf#viYn27!O1GIf{qS4YpP*48;uY=L8H3l#-fae9MGZz$ksfuHwGh_v^xz=lwqoL;@0WMN6%ygdy{gZl8N zGZj@qn!jd!S4(nEqIj;p4|RXu{ie={a{PnbZ)rz)Gl}P!`ywy+yY{$>ohWb0NEIp{ zw~$RaB*UL|Dx^^!3>~$~|NdTr{syD0X%aQYe70~g`|9>&x<2se2|C1!kO%y6^)dEs z>Cn2?MU4#SQsOb@{3CJH=F%XUmr&fdnVQ7lCE2ePv|YElmdTF= zW--P8k>z0w|4Lr&h9l#@@_ugmy6o;+%vpkpr!C3Y-iXt0A^XH!`74!i^d4Z=l{7;S z89Tp4K%1pS<`?xZOj5p(>g8$F+S55HKRzLf2NcT9zLD(d8aq($d(x+IcmLS7YAd*; z&&FSW-o?0#tG}`Q>Ub{EVD5gv?ztrmu7!MpMYw$XD1VD(ZIB)695jH&bkeRzv*AkEMSOY6xS)w(?D<`b8p&0FCkObX@U@8OhH4TQ3ze8!V& z0Dx}V$@aCVV2OyINA&0nQTWzt%_~fhXEu}B`Zb45dzK>IA16(qEAJo7S9bBm(e?Rf z%f98$<*!&W%L&}giJ%Ip8Z6;%_d09`p3k4>yebF9{}9CwZddltJVBJ=S--!reDZZ! z;zqK)k!T}-=GtXn>186rN`2S16d$QweP|~+;uma5{ZJ>U<Uru+_FkyU~ZehglM8`%o zmvfV8)y#h)&M`((&s0Wi$7kg#Jw_7wrydR;j4pZHSex*eJatGsVD3~#`r13;9+nLc zp}6bf%^aP{2jQy{z{#bOC>Q6qimHD75^!0tVcoxHz3#n#+xXgRT@ofyX0La05G*+? z&x58&#MM_PB}V1t7cuf0x)UnB**(QpatUEJWb?AC@&+)9dcT7}Gyjv6-mlswf6iVC zpALGbr$t>DTbxo+FjqE-it;Ec3{ICp1A7)=HTP z67(MGV-!72G{M$>2^Hmxa3#4$jb0Db)r}Oay3OFnE{C$k?9T?tt@m~?8cSnIhl7P5 zzD%Acp+_$#iLs!AEIC0pf+YR$Odz`u2IqHv#P0&5rpR7{Rw z3&5dh#1*!=n{JuANTrHv<@RpyW+W3l$-45A$jC->Wz@NS(hHf#UqS+Cw0k`-(wl;0 z%}sFl$xpM*%d}KMq~b0y;)~*2Jl#iF0xA3KxtZcns~~@QX&SR&_blOdU;$!EBpr*D z@G!J&O7z_b#KG^Hq8=dVZISR;4Q|+7*hi>#d{g3r2GtYiL!h!9v;Vz%!W*X*hx$gp zI~r-E$zo|OO!(1dcs@sIbmA$qXZ?xr%Ed#ECuzEU@^Z%n|Yp$2F0J6M43l>H*F zi6z>WoTEV!8ZJVZ=`~6UmfT(6>)i1-j42*2g0L!_Ti8vve#K-4K_DkEfS2(w%SUmQOf7lJ@}!4yrPSX4y>II$=U$tLA~& z#^$hm$%np1lU2if4)M%;mp-|QlD;M*gk`JC6xK70QqJc38BE-#`)AXN*N<@f@*x|l zo6B_fTLj}u?*iXz@zC9r5hwXFyYT>Po&APdl4F1gVUtUwnpfYzZly@EPo{ERyZ@SL zV<1oqw-lM|!{727|=DH4!i-B>LfC7-!FDfW1c)OG-!0_<0Je+u;!oNT1=Tmhf1e-Bva;Pn_-9|;fp@6 zmFe>@8mBW{0RCwpQxH_;dexP8j*h*}p&@U(0M{sOMg!d^xz7CEVP-MFmxz)ZY=xKK zS=y$mxg_wdNG5K;`Y<+I!?h;F02--+_zE;k1)RKM_HQLA$Lb%Dp)I%gyxqf)&5pyi zW%@O4%>45qx`fQDF;etS1wF#{mjKLY-1+Q6PO!ezB#6l9a8b)hqL#f1se&`(cb8ey zHpIyyAwMNF0E5sP(*Qe3&cG6QfQcK)=KV@oiiSyq(g)CeD{Y$4}I!LJ_BTRtahI=tDRL3Fr}%cpO!=0;#n$GWeh0XVrGWR_yO4|T^j6Oj{=@%JYFB_Pd6qlX~GMu_i~~9%z<;8Eg>It z#TvSK%l=#T@A!ON9D?<4G`PLp5{bl zc}o(L+I3Ckw`bnHVndm&MKY>uf;y?v}uvRUDw|U9C)Wc<^n9* z&#d2LLcFDazxMrO)t92gEz8L$kPu%k;1Y*wWrQAwG>>tETERc-N$Pk4qmv+r(DX=}3w}2p8qrKqdbxm=AU`J+he1bN7|>pu+mYGUlcSRU6^kcG{$ft_@mDKN zQDf9PKO$$*d4zZISx9-zdLoY!^J6;sN_vXrYF>NU`Nm;TQ-mEk)1BomLwxntW(H{A zN%nf-^yBB2^-jf9#q3>He4&MuBxZ}#F{A)n#3fP+{4+?TAH`4tl4qv$E%W-=O!>m) z6X(T5b`A`*tOKMYr-K|xutnBQ_u*qQt>QtUR0hn~PNkkF&JI+6vf;_oO3iSU+#{#e z?`Jv2XtS9wi%>e<_y&TtwuiKZz;#9VCtUEd9SNC6Fmi^ye`s6xdga3W=~S zBIuoIfMzh&r`RW$NKUVtcJZX0Aswp4js-Z)5{~BsZ&6QyM*~$`^@NP00&On6S#7qJ z!D!GZT>l?k_5p_C3~qQqMDolKc<`n2w?I-@_kr+eJOm3mXW&YIk#d?iFQjT@)g3`` zz9F2V#u^j`b><_Hx8qIrQ-@ZBLc7&j%tMe@+FaSKJQ(Ea2`0IDV0S|u_4dWXD$BwlhSs7l`s zY6hEe4Y;$_#W;r|M;}f*4BZg*K6I*qh;T;I1{~%jphQ403Pa=Z#?r?@EN;N;`L03V z=X1r)olw9JJhK86M`k{kYt)XCKTZ!4Y@*_Uo7}Ks z#__w(?U2dRJjkI{V5=V$TUpmRWiZyIw-vww*~_Ni`Nb3wqCc!_a+S!QRFS~ktP&BN z3`s1U-8)Tscb2jB)KvE&L~d2frqSrIkLHtm-Cp_0z0 z-&I##E-4RW5{KD4$^o-z|2|&Mru(|FJL^NNWDuX?LUsMO+@w;z_1cyfz#%zL zU~qlNF*g%novdgwnw!AhoUq%fnbXQsVFaZDHc!UV>KeT^!YgLeOX2x`O23) z7F&7>Ss%Km?QX=|wx|jC+}wbX?I3Cz<4ArkjaN7mFN~y5lTEoUHBdHR-Me?aThHN` zkl9kJ7u(VgeI^6ez3+Hsbr=o#urE0OYFgaq%ai%4)LVVs*;|c@5X?6o-Kx}!JLV^^ zps~)UfPe)pnWbPw9QZWE0dejAdkI&$*>1z(ajvwIA9M{T9rpj#+#p5V`G~{_S~1FmG&7)$#^yk(jW6r(&Z)tutYjD2+03X z(mUzN)LG<`oo8}IwE?MI4pW1^sNWH{n~DY0bc-zoB|@Yp>PgxAbp{%*?G?0*SX973 zpPk)r-w6yF$$NY)qvChKwc-YidPT_W93@&rn)8MpcjTSOG_a=i(4?|nUE+o?b~F;i z<5~Cn26$!u8Bgo{&H+`VjKJ+9y0v8mqpuA7K|Tu=LcVf7bR=cTT3o-(b~>yFX=!0k zgC-3wjucOYhWR0zd=J;9evyfB;3AZn(f2CRyziI=UbxlM=pg=~NujlMlyR~_Ukos1)~zdqp?#+cJaj<#5?=vbPt`r$S4ZwSj~w+(r#@Xu z?RbyYWt9*ZSV0io{j`ShzbRYvMSTfqF}LJk1@b}O^x>2T`mp&_%%R^+A9;AFh0q$r+f=!ifuQB|*;eJ2)^=4L-*E_y+=YlSiI+c^?`{6fnQJw~OrQ2` z_CT!@%>8#y)nezw6yep98wo+H=rtu5>5>px$HFx`n3u`p zz?tKc$=yI;hh-peOj&n9F`;Lo@GPrvZOG;Czuq+_qEsx?kRFq z5fL_+lu+~srNaiqYgpo%@4}!|W=xGK1wM_WRclzPHip3#iIY+DJF)MM^~J+UO?*@L zHkh^S4#cire_s8*G`Vw9AGTPDmPqYRQ;b>Z!vTWSWG`}i{WSM}V#`l62IyoP`J-{` zjpIfyvgWJe2s+ft-!|ZhF^XH&J41w|S50V+VpHYeIn>J4#F_b@y|j~P`knTj$zaNt ztF>3lH|BJ{JG&c9x`oVtkJoz_zkTq{`^xfYx81BFyPN0oEckp(PDp-@_Yg1T*7lQ>?8lYn9h6a~$kfzh4PnFf1~HN&>Qu)D%V zj&h9+W>&vNTbP#8aCh3V+{fdNO$Fx7jF3g_yB_68s5f2unxg{6+br--yLX94(U@q0 zMfeuWI?>q7tVU&T-Yn!KJp@o|keQO5$;I&xjDKToNbGOtYvs7;dK6)lIJz&T+_vQ4 zh|P9$Kj2gZcbD_hY`?;FF;&kQnRW|b&5Yx&l>i_9hD=>ZPjJ30p64BF1^6!@#fla6 zmZ~Q_@hY-43mI$CiAM6-&AzE)NjCDxs?WuPMq;B?z6ehQ*=e=u90Wicx2L`Wov4Hs zFR%Y_>t_vchKM-fBblC!U1P0%H|e>nqad}Amx8`I&; zL9Yc>T_f8EM@hKy6*9tiOg1?><$>_Wc3EYLZ}2b6h3=A!nX_>h$1X zo%_H3J`DyW(U#_2v}CKLTeojDARr%~#M_41M=yYkzN0 zHSqe)?6}NhKBzZ?LhoE^CafX(2eSjWZ#1qgJ?V5j%Ez}vALcc* z#B8y)DJynL_6i)l+MKw*!*ljpA+N-4cjMN{GK_sI4fnO;$st&J-E_n7B!Gv|8lPL( zR<&x4e{viEIm*P zcVn2#lIoY*|E|AzufBn5M?%~7m%pHyZj41@kha!^v`I@o+5my{?nJx4YqLpjc|EyF z!g7w)jZxd|#Iu~4Z;YI2l#3-D?T|K2Ig9B14jh$jiX)6Vd=v6xJa1B$A#7B_JB&()!Q0ZpXi{KD<@$dWpe#8H@7PWuh=J0bbs-Cj$f0&FC9X|SlPDx+gpINg1WgLBf-HnIuG@Wx?^oKOJCUDJ4lG)cUOXoaSC&+;AdGBGJ z9TWKp^K0Y9Ab`;tf2HX{g0jDWfdySF!(1Wf>^fWBpbg9JP7x3XkJK$Q!RuSRd5dE5 zg@hf6%rLyvt6EQKAkee(C&o_8J~XH{tuhASC$7| zk`Zde=|73S>(pZ2TVBq@TOnl0ig;mdtCc)zgy%6nzrm9UDA)ap8!8AlsxjSb8lx6_ zh%;rWO)!9#1%>ID{J0HmEy9oJw6hP>WMBe)El*=Tgtlv3hrC85YT^TLLd3UNj0-6j zjarF!XPZ`m7xykfTbO)3kaaw6vmK^Yt0;(9Tm$0U0Hu$3+SM)-0vR(#D1*;Af$pKq z69js3ygpBcaC)G-4e++Cf0q8%tFeRsc#rYpD0dg)Uw@cL7k5DUi^ee2s>2UqCC8KT z+2$s9!0p^Qqmz!g*cFtQzgW`T#}Ba#k^}c+8T0gXWgbQjik70Jp1VF*Vr`3ziT9xA zNu7L9^(mUniPzU)Vnp;2&jC$_qvucP)73KC)OVKp#ZfxeQl}7K?jEiSfsIIAkX5*t z@$nK7qb6Hul;CSx;@(Cs=lNm${uF$fqvz#Hqy_Utk&$$gN8i^+TchD_p59OEam|;6 zt+jJWiV*!nRHGtuk13U`uRG73U1WL8)zc-<&7Udj&F#Tz?IE*q)0{;X3m z)(4@QJMKrPT}~1li&vL9uFVd_ zEX3ViIoQ_&;S1yvEihkli`xn~l!eJT{K4Bf{XzJHbd-of>&whrXMwsGDfT~DrIPKgO<=F(!7N zK%;m}knV?I34F!#h!|WL_SgB}Vh6(VX&#vjZ;zOVEmUs=+XM_l>Im-~T8ma9WF0UW zO&Olp2!O$ukU)x}fS!qemZcpHqD5~JruGVh6XDt7vVgHG$9fDDK7{*!Fhu_++I6xR z9}=COle*sHAwzn@7?^vB%GVUU3D1%JnOe89N?RBI0NkCd75o%qWMY%6V9a6Laz9r4 zwz%DrqMS|r2+S#}A`5}K3C@NqD#WhjRlQ{8T){^bXUTV3ZhP@_w3;~*vg$|D70XC7 z3L4@Knr;RD;o&oLfonk8_;Q)<8R0bT&6DapfO((=DrH4@?EZmvatnp5&#)) zrb3z2$w(`$@-d87m%j+i$RyVtvIYG@-~E)?b-j!4>D1lItv`x(i676poy=L&h0Oe0 z+W`S#{SZ$>LRN(Oa-l+4bwgV~4E^o`1^VDV_?{UypA+uzPg)Yj5Ue6>=fA=oT?QFm z7mDQyVXZBI9xNQ!xxzY3d4W{1^Zjd1zcc4Uqf4muz{OqJ^F+_5mPO+RitKFU&>L&S z>vtGKi_JFK+VUxM2m;b8ykbz?GGgACzy_Bi`H5px0%I3eo5qgM{u5)$C}k$pRM^Hb z$>km|4$MQ1$*eIe2b6QyjHrJk&=np~4!4NjFToe|($3Lkc?7=C3k8kZ%*fapRK2(m zN_)|Ek@}z= zzyVQ~b3J9?m!dvPfQ`wbA506#gLd?3WhklinEKwmo+)A#!df>P{7Obxz5Vuub!d0} z9Cbg1?74o@TDQJQhaRXq+bh#4XuXp=o`L=(fm)qAAoaa~5j7Cfb$eoDbf)nZ`Pb@h?IR7+%kx z_WOOTZrpKcM_iq+v2gOezV+Na^eSBW{Ehz_U|t@AgCm?r^XRq<6(K82?-l6$?jFW- zxp+diZFtORrgq3WNz8Vbvw(+rdJJyCXVYmr`cyGHmvJ?% zrF->bBlIBF9sVLOD?h639XA~}wu9F)WX=6%eTt$AYj2HI4~^U#3vDfcp$pDonUC4=^Ab0K#&D|{tzSv547A$Gt zJ7#O{hU-z2=}$kB&Rt0arvBcmpTF2$s6bq72b2fzdLf)Vj6mp@z|$CuHds*H;dZCUXsimitLj0l9-oJGkn;CDJb*S_G`~N%;hG;${q@&{4%Yq!-vd<#gQ}GUvy$B0-jLf6m*WN z3G2uOlpYipvM(!R^*huzqK_N@jhbQFwSH&D+N4%k(6{fPlJkPzeg5k5I9#-P#?yZY zo+-@$2_=F(dMsU`{>$;|{U{Tcn;ALeq=@|}d+Bs8znL-}16KWzg01paB5zdWjLM+qPl zx2xx&qKw-O2mS5)NS=8X?J}L;`s+a(T9)t5{pN#?-_-9^arjO-<}PG01+C9~w8gCU z4F2Btnejj}2&B8K!1=HSxg|k6PVIc$`=1u>GJ(6W7;0sI#>)wtST6hn927&&lM<;u zYN_I8s*5~sSX0tTZakg~tQpSOm7Y1sJm%y+n?N$kw0%EGR|L}S{e{=1X-~T~2W_Dj z*O$1DYZhdiPy0A$j9JjiC#_6(<`@Cn1J+;9C1Sz7*uPIl&$9`XEfu*1IaQQ3wXPo5 zjW>R@S3aA3I!mINrfL~^Hr{-B*q#}R9hI zpn+%7CcneWMAANwUY>lkpQ2_+bB%@KbJFzI`D8}h`Q#33Hzk_v;S<9jNdasE`wyC% zQjA?gpM18kF@|g)-gHlF8lrX0B6Hz$PbaO1=Ruz=Pn09}X7a)a8zvJJWyu7=kWU*^ zAV65Z5;sH6^C(!Ss0;=9{L18mBzCh-f5+Iang5o@8voV_6nhWR6}%5HyUkaZd0*3L zVnd7HUxkT7vKZ)@{9DLPNzmV&8Nr=8C*783Q{#-`oSt3S1*TwssJOR-Fiw2o|j+M8#k zY5MZ~281@QJ%41?^C%@`)z=5@Xy{SJn^>hIhe)f2nV!>ljrp|s^HsHbcpE7?{WMN1 z)EgDS)f&G>Tw+OjN8M;3NAp&3y$lpPVR5Z_+4((!R;2a|Tq@GW&6{stB_-6ZgtZA9 z7EN`zZsLBR>^*xxCrG!0Ev+R*e&qEhv_ENI)s*pQ9q;*gyS_CG&cJr;P^OB_4m`;q zuB_2WL z-sg$U#E|8R#ZHW?flqj^&&XTryIKd2?hf7!kN)sAMt(JgfA4e+G5=ZEB8+1`beg#Q z)A~CtfR{@=SN=+AF&+f3>K1F!>%rQYCrQvE#QJyy?P6R;t_^_>|Jx0C z7)xnr%SRfduBmfViuBGJnM1}usVCRo&Rx^U3r&u&B&(K;G$C)h*bBar=Qb$8P^*L1 zYmB&48+2HniAukc{pfN#Of69uu&|IP1`(B{zb>x^+UoT8r$Ps)$Ob zR`ld223{wfwFvb*ig1;=>Ad*d;HOY^9abAGTF_vF2zaefx$-FNXSq=B+WK|>x&^M9 zuc0qxJKq?su$V!%KJmq*IK^TM@w2(21D?(R^k#wqhx+BR&DYF=#*hNVyv6LR!;;`e zhReAYe$^69Es(oDck@g2T~nSr-JZ%3Y7VMnIkCVxowp4xAA9y1-`MHF(?PzIKV0rN zEHJpG4Bw6(h+rJ{EKiw8-A=i0;X7&9chxu;G&P(#=AS|@Uk-Bo(!Q*lQN1Al<#VPg znBRaH5KB0m%JsfPX3n*x6{gN)+dtv7KusP+N%cFx2a}udKiY{I3%>9-TVp8QxHa+j^a+X*Ev6EZ>7W!4kCG)|LSuW}-lXf+0l3JYx(N{V(zeOPpo>>b;EP&Ko z!9*gFV*{ovwAy9rp+G$5LF0yw+dJQ|t-dTYf21N;-gk%sECZH1wNa#&&5?P zYeAI59F8j}uFKDAlRzC+ed%+yzCvG}>uptd``&qSmc0Q7xbUQP5fh1_j|Q@@HZl^81pt2>&XXw=b5~93-Cuz;?tFU?NXEL+YZ+GlTO+GXu$s? z>%GI-jQ{=r+EuHi4H~00tM+K@C@rcLT3VxKjiN$qK~NM`yD@54DOGLly|++AQCnii z4uXjFP0#0?-*v9v`6qwm%DC@4?>q0;cs?I5hb(U+Wf7bqm9#9wD&po7BbE7^yu_Mj z@`ilwCQ&dyKbi{16MNKxR1dI1DLCE*(MbVMvqH`owFP^FB?3y*CV`ETa80uF(tcj& z_ylz`*y(piv_*`tMD;6vkA}`W^pXmW6H@Fj%fs^2dzN?^iL#;qpT~^!En43$*8VG= z`SdD;T<308qo|wlW?ME=c z*e3E@1&w*?cH?9EJVJEp0@zHDXQXZ!xpFE!S``CkNtLl>#RHKr-mIHXImJ}MiWOe` zN*Z}!unI;b`!SpMiSsey6q$kG*hj}<*$mCmV%bQ#Q7t5ea&#&V(2Ppfrma1g^CVmb zw-ibB+3uiqnNy+nOifJhn`B1nId-nh-)JOpWxh<6ajP@PQ5s6tvwIF`4uNEiu-Ogh zynn3h-$tWW#H=qTis^k3c#Aj>w%aCS_!lBS$(~? zF}Kv)UMX>TK7m)CP4!favbcFJ^85L(EpVsC zTH+xJ;?|SBwCY~M(IWx`x!fvsWGy4+HML6h2<#(%vKVSpiIG%U4wYlO~&^y8M#*x=IWAZ}r;;<`>2N{I_7<`r#4t(I}yw_$pA zh2YxTA*DhNA)x;Kf-(hJyLk;=4*(mvoCpvl%$;ITD+(fVG<6BF$$B>5j{M>a4}E5Q zOk@lh>_)W?Cvs;oeregm8^{;11@*b$Oy1$0MViq(q#V}}dveNqGx*`)iUjZAYQXL| z(XWd>Aj4O&3r`NJ1Akc+(+yo51V}REqdjsUhVB}{mjk_R6fbI?6<@qiv87A5$u-nl zXVvdCR>H}z{7QbkP#*hGn42}=pazCm1d7MK7rfW}Wozl3LJnl6ptoKvO58C9ImA4i zAt&**M7u;K-ifgKpETL&zhaG>pQ!7o#gO-vi@j~W%)RS-H&cVA{Mib#dEL<#DiG~; zg?`YD8qSb>GI!K0j!uWaq;6TbQSylaYu(t!`pz?H>>*p6rYV2}=g+WDj*{!g(v}iC zY&V4)IOl!Orp|Ym4^jpiE_F{ z)BhR(TDJv0ob9mZRdH_kR;XrC5nXbI*k9$StuNzg4z8{(S17ODI~@4DrG1&7U)k?a zyw>)EqrdYfTfCEOxnd#~w|KVg+cHniIL(v=9{|dgnDzAu=61J{6gfKzvtNF9RWEtG zq1A;y3_^1I)Fcqs^n<51x8gmbs}U1;vy6NYTVKJ2unCS*`c{Vnrt>zvL<$L|f4O2t?7A=|!BbaNtvpf< zgBUBeypiI$Wg2Mr%fo9?o2%=}VL~@(tQpW4H`+6}B!^l|ZRSmG7nLI82rq`$1K)J` z=wK;3QkxWI=ocGEg46r4>3@XR1{{d$L>dVXIzYU%uC?=^JX$Ud!prot7p2zr-lgb4 zSe)Is_M0m)6Kz@~7uIw=ruWb0Ms;izC!c5dcL;VB5(fgwDddX$&o8I9l{rz1ge#o* zJ#J}Ee8sg4k8kTP0&Q(2-Hz}+?Cw1ot&&pjzDtr@K+FNRoWB=K{6r#;n6(%DuBd$FavJWZW42H<;{n+_gfUJ{E;@YLALM*|z&ggZu=8qIaNt zpBMcaO*f|3#NI%)rU=*A#S@@dyJ3xBc1t_2HxEwUKgm9BHu)I@-6`<87!fah#Lzju z=sWjmV;oU^7)8iknrNj!Dzxiwz`CiexUM}Ii{@W@vQP%mETh-`N?R z)1Sx++l3{QEjE3Xvf&%DAA1?}H(8_edpSO3x_)20JGWvEq8bPmsGQ*Q5HwDeW^(cA zjJF6Icl~>&-t&U3-nQ;dV9S3J+xP!fR*jKK+=uo|+Vw4%vl6Jqk+&#@Iv+P(+GbtS zS6a(A`a7+@_di+yheV;VTMm@4k9N{&5K0AFX798Z(}(`b4pI~_w@a`E`*sofZ?tdm zMl67z&^rRXp%A=Kd>2O}m%HqvqMFiH25yf!ZH9k%hbsVX%;epgD4QCtYzHdKkBe@7 z6C(a2qG4ZS+35!uS>xJ;5G?bf)H2RoT`#Ba6E$AWXdVXpZH zwC;$7HXc(u@P!rhSF|+=A$>evmv{7-~??60Er)>Qyn}Ok^K8hdA=Wm zvbx@Y`9DBf-{vIWwJMF`-_&LXnnJtu#tMW|yvj_zx8%oW=WPR?di9e>li_!7H!xkV z_R_U1PkdXz@s9aiCp#29RJHHU)zG;zZQBAWj{GC zQ+Mg9aaJp%&krPW2bHm?3@zLagsSM2QDEAKXkFb#8zMI*q*UiMX%Ak2hStsT(6{sU z`wA}K^nMRB1DfZPRX~5533&eaUAnV)%u0SJ)BTJ9zzL}J3&YD?ZK3l!5);35+l3p^ z>>a&*V7ENmuT=6w>w&K+g+0E<|7cr9E>P8qy~GXtB^1%?;YI}Ik|rm*gwI}>p`)^5 zEV7$IzOun z4JBrfUhvFv^TvZeCA*Kg)H_&&_6--+`}p+%=Oj9-i{qi-x$l1pV{FM7+KV|Gjjk84 znNWu~R(LNdrMcMZJ<`T%eR;i$C2;d!{cMrlVt=oTiwWwb-O;2khn;+<9jV*b7JXxr zQeWj2n`dY*Hzyx|V+_@KUNppb9|8igu)y$TE=y32Benl@^<>3L)j_PRpQsg6by~2S z$|KY=+8tb0fIE#A!T#oAtT z@rX?!F}}o2K9W)xXQGW3@P7&Q=hu=zeL-n_*sA13Gy+DOjP_i9qGi80U<>*c$RhZJmB*<4x3_8x26djhF9bJmdAa#Li2-A1}39sd5_eevo8q1k$SiwQc#XAij=|K@II7 zDD%kLgVd&a(TRD-Xk~2Bq(7;^qOWN26`N+_&zW~4AV`F<~r?sh*sr+ZtPX3U)Z$^vskj(^zjBkv{Rd} z2$M#Tje55D8|8k$RCs|44*#!=6z9KwePZbw>7j4AsQi4)EWf-;S%Sr4SRWtq5$1dky#dGDF)%>t9&5F<9XN zFQ#sAXKZxq$Z!f*?Dp@>7{Ba>0Gm$Lx#3C*7O#GzLBHy2pTf)+ScthjaZVYJb7IJK zB(X8dC_4NAJ;7I!D$`>jm9rzpcrDeM&G|Hrk!$gj+#!U}WgV2MAJ4iI>$?K(*3YoB zbPSO$2xZqtunM&4Fy4wLHF7*F84^h2$9V$w4D7zJs7PiS090F#Gvg#|9)8&L&gTok7bzhcajZrKdEwgacnb!Jt;Zp9Lbd8_}qnQZ@c`$O-iSf{nx z{kfBneqDs^Q((F^rSODkF zivDqa{>91H8+EAzskrK>Y^9__JjFPEV(rKr6|zmdqvG~`t8$LZNHJ=0FuGs53Eh8d zMWJ~X*Lw|UcBu?WA@ScXiigrYjzd8cu6{Fudkl+rd4xcG*R(`=>vozGhcj zAMEV;sIMT+2Yu*89R~K>R~6?SD5TDfRpY3IzS7v7w12fp6yvS<7Pp1WyQXr+ClYi+ z4WeiX>Yo^}n~ks}T@6BHY!>DOk+>w64d5@O3dPBAhV}&wl`vS?pMqpaGE}8?w45>lw)8zWWmkf%VGG9*Zl1w+X@ z%kPOz$`cPtgfK2K+_0R^9ipp1v2j8L5)vgti#_%%&IeicCN+p*n(+{u;?Wm#A{&0> zqLo-e(2GEnRzvfb;9XT9-lk|kP1-yQve3x|{)H*tx4A8cEM)nlkGI{AN^ebbSf;z$ zR@cSfC5pc{tyu!Vk?sD>~PhBEBn zFK1oJgTAC>S6%+t1p!3c<-u^8zC(bF%^pjrt?h4(zK6PIxC~ofb+zX2KMhJ|x-DED z*v~u9$4LYzb$!XyZxHMCY6Q)P8>hU78+PejQXcgvTs6TJw%eEtQKQqvS0jbv~ew(QPD$2G|0*&SzBhI_0Li7C?ryxa}L_XZDM-S~)eR(a)69>9#)Yds{ZaYX=+)FLy^*Jvnm zDmZh7t`Zc%t0RJeqr`O@IyEELb31it93Z_ds^kOz>BK~G{@@eN=r?6%R$k=>Rzh8J=YQ6-KAl0r4#w2gxyl&d4A{uvi`^ZG&F z=iK+;aFAeL1(UD;9eG|5`y-2P!Cle)kG)Zn_2ixEpW5knc`NPYqg(Gj3%pI!z4FF% zMwAWT^y-;qJpB6;w}XNgP4v$wZ>ylDVK8QzV3>&br`Gga3=?t1G}^BGx7bGd_&t;; zLU=5N@Q!sdrLFPiC>k58ZX%?yroNBnC|w4-1P3WZ=_x8fbG4g$vq?*UwBi;Z}xWorFp!jSI5ItvJF?r&k=8zLyglf47@la1^3#VEztBqdF!?rG=s@0Iq0h zb4k&KHDAJIu?8}c@XH#VIPK=$9zJ_|=jZjh9=XvdQIMUOHG0v|Cha;LW{86A^Pd{7 z;y}$HR-v!0f#G(14Qeh4@p{oGOYATRZ5_PbmHtA**XQ}_6_=lma^#fGkvK=5S zbWcIhI>g;{l)+qRpWdfW{8qBrow3hc2;`%X>CihmS=jWbI8@F~P7|5;KS3V16Sxs! zOkBpm6&E6E1l+zjxPRa=H3umm{#dIElma;#MOegZeSDI&t8w24pUH$hpX(Nh*29*U z8R$EH+H8}SqOM*@Ob+tOs|YW!y6~Y!w>r3m!EPntrPT%U)bA?Q^vU3iEUmdB?HQ0W zqst0)DAfi9A949-2R?JlY0PDHBQ$pRoOpzzt0poKAeBs|-Mm+{L2Q9Q`gxWdNHUN* zdAzze!q6qUPqq}bw&qFQxIZfgUoJ!HUz$*CF<_C8&iB_4l7oo#rVarSQ`YMJIbw~n zuWb13rzAmy>QPnYmzx6G8NWc@=;}uw5PrbKfpA?)J{A>*V3&k&YVKp|X|vP6)I;{q zG++!zy^Mp2y33%c*MhL8$2NgsH|u(>cKDCv()DzgD(;+VHqr+rvw){Z24G)1ecxn| z|SOqhDm)KkY2Y(e#;1q45=al8xM_keRI2nTCvbn@b3- z4=v;|iMx+ZE9PjYxgq=8o2Q;%Tw~~bxx5tf!$W&_vUO?N(@rCjsl4b(3bw2Qa}WC@ zt+J5W+^dU?$z#m$_53vjqJ!4DkX)0&5_hEF;ToXzM%QgzjY6O`iSUp?P?VfjaEL+=@VN(Gl$spI*Xm z8#xL|m=@&CewcG^n*O!MrpKYHa@3Zb90tPL`cDN}{9~?9HWl$b0K4=)99T%!JGY3B z3jFBrk((+{xc4pZiBXc~t|1^id-Sc+eNTw(s86QH`h zmqo0OFV7b3^`j@h&D1(Ju&B>u=U7c`>=62V>!yOtxW}pJj{w9m583_SlaZ2WX7zNn zR%m)pM1)kQJro+tYz@8qE?BSO&T0kG{+Vswpz@rb`%S$$N3W)11 zc*fJp#`fH0#mY7N_Vw%fbMU-8bKSOzLqG4`YlYIdqMUHLdOR^)vwGFPEmAueSV_zW z%k5dc>5L;8sU4uI9-NEPU~a4?3)a=B51elrVF2rfc2_svobo@xok*Lw;F3bFt36;h z|9DcF2hj(D2{k(0Q*Ni{G3P3JxG%5N+vIz%WV=9<;6l!b4;D#vr#=y}5AESAEjq1V zBm0{K*o9;7fi+uYy%m=i#Ko0BDS7-iIfVmBS5NYJwoXm5IBuk434$$DZSj=688KVv z9I`UVIfQr<(^u$Cp82IOdrY2JEA+Zfe&~;HIXc4GeB>zR>w!F(V;TuBiLdKSMDP0n zUxM9bgtL;Mk3z4p@A8Gs(1rv(yQq>rMe8BPRt4a8yP>^AyX`_~riabn#s?o_e|Jm> zwxF&mYIGDa2Igdn$Vx;dgE6{k6G;z^b~{({kDpt&%ewvdn*a3rU#IP=8?tdywyfg5 zo)lYcD+ibEXjAx4wi^@!0wO(TO@jKRY{X$Ul?`ezQbW6miA$(aq@y2rCuF4i(n|>h zTGl$DBVQW9r^cyk&U94=kI11DW}xUo??MYxQD1Zs^!X{1E2Q5*p26~D6anofnPRx^Eo%_jx|6)61~%ite+@Bq0)vr#v7 zImXtPRvdWqqd!LFjj%RAR7EkVL6_VpDm;`u{&h%;Tx_Ck?!93Ss=6vwB!x2yvTh3W z{qW`dXv2qbUD7+TP3Wf8s;p^Jz67!a z5is{Idrugsf2!54?1cxOB#_<&NPPr%>&^9&R4J?P_q=ebbufl~pUlrLP^Zi896Gqx zoO<5ssz)&K+NCLk%xutQ@6$eRxLXl*-E1m`l?s1Do6py_dYi72m;kvNs3=me*WWh3 z) zYlYEw)pDMdM-$EaXY)#UEP&UuEml}IHG+0;2ew=@k!N18H>c!1TOJlH{hu4dE`+H% z@z$d(&p|~O>07Buf+nV7oYZ1~%}WgM_0xpdu15fKA*-~d&#w&bT%CIMi=g5_nM5xR zr}$$>&kOqud4$AfIMOf8y$zchcHic4lCBM>f>(`*R3lORvBE8VRg5tzJW?VuD5 zsBjcc*DrX@9?v^F0gVo+q_vqE;X9kMB?&Uc7)?sy`S)g>a@C@hV0Cth6 z`D~b7vr*=4tH8JgX`mRhX|<78IlFV)bX0NvBiVF1;q~V*_l7Ap{XTXT=izu-pM7HB2HBGU-oTN!RsW$23M|*$Ux^F0FQ0w4cmhP!D+&)a|UGfS+CF{g? zaha%O8uh|%DVlBhsps=qI0yl0c|Y_ex{x#>>aP3i zH&DUnMh?+x3ibL|^Mg45H$p!B5qB}AI@~G;b4DML+fj*j*N)HxE%eIuEWT8Q))Ewq zs#Hko_?m)y=&yNg`HY`Y;-)d8sc*KyWV>;mE8dxMwP%d0PA>y_`$9Ubi4fh%<(`3A zE7>$rQ`YFGjZkXlTX}2j%4T}HYKBL=0W|e4%Ck;TWk28e-gwq*W?clpMt(fUdQ|3= ztdDmqN9{Dkv>~i?rK%kmsgV$EY4H}dx7Fu)jB)Olvk2Cj;;aA;Fnp2T=0dd zNG+(ci~V3NniR>@i{Oha0e>+12&bmAau@Q{a0g@`n=Oc1dL^7xGG}uv%f1sJxacKZ zgZSvg2*%g3tJu0PgnM|rmn2o!y|&X0SW-mz&;3W0m?ZBN}Q684}A4tv|pJQV;&$W*9p`SvuNDxozEHch=jt&P~@S0KLK_eo{6M)T+O zODnQTQKU1PBJ6BKPXQH_EjpRckC!IRq>$3afFE|9m}PN4N);{!MWy9Ch|PTF%#c2; z^%q^ehBc@rGlOtt@@fxYQfJR6Mnaj9cZNcID7{_d*HLg8)vu@@m)dSNg|}7^_49+i}%~R zMW!&1E%ERgoy$sg8a-eBu2Az1SI>Pnk^6jY&W-!Q>h$R)o|fEV`lmr&dMA&)`r~Oy zTVde>Z2W`;vt(bWV7v>OF(#n7Z~(_OIiqdqE3aq(q4h9g!ZL7kXAUU9@Z4uqja#}ao{=QWd7wjj(TGn=T4d;X7e6;O z-ei)lSt0}bQ+&EB&4BNu#|yWUlojMi?mD)-e?Eej+lpf5O^UGaz7Pe3Ckx{R|gT?X#df z_x8h9_FA-6oT;%@G;TyZ4yi zGQ@!OjLO*Jz%MT{0LSKbPa}CP^{AuSHZR9#r`clJRFTG6YQ@)l>FUU0QJs-5=DdA#$~DcW!?&yMCD{(Njr1db%Oa5RA%y zd^M6k0r-hJpm8sPCzBroa3dCJ4@oQajG>Y+k1lO|D^h=e9ctBT?)}u((QbH2`^kOn zaV!q_Qlyr)_B6dkXZI7ciM`Jsrs_0Ap3_I1OL1E}WxrV=S}?L-=mU+O&@@5%8YnS& zLMTkkthnaw2d-QPHyMhdhod%aZ;gT>TygRjb_819E%fbZhtxws&rO?L-@{3Vn!|rP zj1V4xI^`2)ePdy0Gn*j?qc2a(j};edS}qO~@@qUEigai%$SSUHF*){$Kdo#4 zef%@HZp#l-Sfd_9z0DtuOpyo9VOP8&g1!VoAHw^&6w2XjxpPa8{npVw&!;28qtrnV zF_{TTpfDEuoU<_vZ6#(RhG=T4cI<0c%3tZixyS0Fu}aCuC(jtBrF!~1Vm?DH$O%gV-M_+TII;wqzz9HpL0uJpc|y#h1eF& z)OlMtV__pPhg|Cn8rQV4!(gX2pIF5)V`t4`!UnoE?Z(dgM&dx`>)j$|6|rzCZ47Py z5HWtB8uxq`FZYYclsR#e7gY1Ul>Wxq7emB5=59uxcg6biYmoq#r_|X@9cF`to%$YH zyegB+OGy3f>9_BHsWm^k_D)1OvY%441wx(uJJD?Bv#7FQF8i_JKRe-9lUrwLn4VR%yTK!?-#V1L zktHAYgKk|DQ`5z~gt-w!!0}cjE$c8~APPC) zq_89x{F^1Cy;x)jv<>BX$8PT4XY*zUG^b6nibJfb8doj~Yq)WX2+5 zkvpBKjM`&4MBies_me|;-YT3c6Dvq3UoC!<;1*+I;dN7zarhh3$}Km8t9eMCEO;g0 zU-O0;0e0G=d92q&%j7Lu^MAZZQgg~!S5Bry_hBk{XFAk;T9x;Vh`rZ+5?@$u80cv= zlC_22s9{R>Pv~C=LoH!8Ul2!OFZl0`9BIkg)0^T5L*ARYS$h-D{rYAyZaFoOblz!3 zhU*LQ=H9d7nYEQ9(qQsK{zosBcT$F+~p|3t^@!tt>3hjhE6 zYYYrV(I}$EQnK;sgqC^PD~H9)&G4&SQU=hcw&8ndek(Uh!;RL0xkS4Q({3REFO_y= zYou%TZrmnx%j+62)(}etRO)7^_JW!!KN=R6<1@=%ql%E6e0p7bn8h5Ly*j$Axb?#J zXQi?1de1rWE`$(^3wSZ=E=r~I*YMtBMe?~JyztYYKegL5%^m? zlZfeiSUNCC#CL}VaP~XC%XCqrD}>!dMxZRZ7OB8le?!PV_jb)~ajm}>=%@3oaGum~ z*#EW?QZv%1!Xmc_q(oWu@3j5YgeA8(_qiCh6y@$_QJWt)l81q;i|R6^%x9^nP$JD0 zZ(?;-wuF>z3c$4?KO5gbi~jy$&`@Ock-z zg%8kG6j<>`XXi>@4$F42GkOc}Y5e%Vv+T0LvKj?<{od9H9lICf;lN^yrBz^`AI;KHdP^)>q#r*D(9a`WiC<^v!*Oa>60VQiJbA5t$*twAPJFoyFqdnk zmN@YH<(->Y-uf_Q;>v(|ZmP=i4I>N3uVDLm&go)(iw4dr@Ac*RXY(Gs7xZU+@=}Xr z!FAIgW_^6F@x}vT?_^^vnkg%3nKNj~oA-1y(skKt-a|4qRDJ~sx|Ap3mYwIzU%FlQ zw=mEE^WW-ajuze%c1-le0ur1MbREN(r4lxvnu)#~^wXEuAG!Si7&{Y&yc>^Z7WXbH zIof!-x=xw1rxIytrytnI|75S%(2o{$I3Sye?OJJj;zvCAkD=c>{U3=Wv~}5}PV-6D zs6N~Jp?UTt(b83I7H*2cqKMIv+Y~2?UPNLs69jm;Gf zyge{v>tW@tc&l1fIBcZC9ZLx5ccQ4nVBH!GgR3}bHy$SHU(FA%wBMpzJ3RK$3bHQn?yrhh~B6Nzgk8^*y<8n_92wO(@{0bA=~av>g9-j1GK zIt9ZSTv~`4m5(vIF%teF#YaUfa|P)=-DAvzpGrLiNDqHw;+shHH+ync^#2DlSU)iN z$L5hq90YdjrpUkZ5N+e%%X=ZbZOUQyk+0>-5$x*kwZ9;E&)oN0&tgZX^h>;U- zx!tQ0u?#vV;h`HoW#x3^8Qw>^(R%AM8~RnQWeC?2e3^q+Or}$$Q(Y}*fWY79Gc!m3 zmkMG1;N`zBFDDM#eey6>$aj5kxAD2ikUi4c;AkkxSk-d~P&Ke-wska= ze^F5I#D)DRNaW@Bb~8mX%gy)&U#c*xySI;n7ws4{Pd&BkstGH>&kJ? zy&St7Ts8ulQ8s&S>f#wN;tV}nZ*a<+dcEBvHSBczOujakEfe~13DQLBlUF(C-1|;O z8dRr!wZ%KM9gzMbeE#2$O!oh3ic+S7?-ZkH`&`~27jahuKaNK{HU&VcwsX$FTc$2o zvyuhNm?mDP)MuA8)BFL?C($&mP6>IpWdvo!w!UOn+Fwp~#cIe7StsRklTcI@MJq2@ z{Qs_Y;i$(^311f^Ba9=r{tu(#)6l;&zPjQ)Z>_O8m1F?uTMRcQaBSF>=hQ=VhPwJ* zZW(z0I{bE&TOw8~vT?j_W^G_L8k^;Qman;CFb!ekILgtf6iWWtY;P3*fDk3V&t*20 zs{I|iWC||&@&B2j|M$s`mA78_p2gNM_SgNLfseQ?PGv-@Ss*6eb_Ii-DqT&KucvQs z?C%lD+s?+{)4blYd7@VLV9T^xbE)(mr0wFDnKC2tS90mN2BF1uOS~h0!z%pZF-|eI zTVqp=$ycyX`GP0dxII4Qao!0AdeI<$Si&2b`ey&b_?FuA`Ix0>AZ81*)$#{i#o<4D z0m{4}d*ba~^M9{S%qGfb^6S5ebiJ5vIfG?9`qh#vRiw&pR-JH|QyKLcTGlAk2D@@9 zjA}BlXBZCt3Wt0dImVoyT;%(h6_wAO*IXR7h@T(DzDwRaL9d+$pB!COj=foGItw@{ ziLiovnUrTG6Hz{hr3)z4HpB4EwS9esUyu5ZUHAlt{0{tbG^Wt;F=4MQ-`|R zi^u0j`b~+Y=Gr)-=>F^K>9T3}be110WD-ku72=+7dGb)@N$fyyx_MLM(d0JOES=Fb zo&V_uxOxiwcg>e~16*1UCev1ob{}f^&W(A`xndxg=E+AUDCFhn-Dq7q_OObNrNE4f z$LoKbTu5hSHLqYDOlwrC=ib573Fm+Pyo1Rf;Q_`TUA4#{*LF#2o3$RaFTYiIpNo1j z2Id~}ba>Ds!;&yOG^+oKoXW7ud%PINS!HD(z5F2cV1Jd`aB)SDS_9lD%?)tM%I=WwPl4Ghv ze#7jAtMWiQAi61~=6L3;Vo5!)k*vz2wDa*(;)_%o|I(B`D7@g`wtw*4-EUT0Cd+2 zmmST$w_=%Gx71uaOt&T9bcZZEE-RL6_BeTnSE_hKfE$yuynfSP@DZ zUqg_*jjO%eib}nz!n7gu=D1>g&U8O!J%VE~h)c|@pTEIF z%JCb2Mt$#%#$GLe8+?(CoBs*nJu~i|eT{CJzNB%#93elzi2 z`R~+7`5-J`1C1U}oAz1>xm^k*S7v?h zHO{ffi1OZVjAC52@q0F^(=jpuZypvY( zezim?Xa0HeGDyp)l!j9#y-lWlRLWL63RIz1S*8?Fz;g<)CbdO$SDL@oeR}U!ZkWE1 z=Lr6FvhXc!76cmNC-l7n_xpQ%T&^-hP}u=a%SRDmPh3&7-+ZL)^PGKlbrHqv_|;`r zw77D$!|JgGR4^yp>eca$x<5YaxW|V?0e>&IB7H@bqxEAWP|xe#oEPz~P(a|D;I#0T z84~V;9`r_f_GGuGkmTl9l`~ycZj~NznFWEq$w|?iU1=;HO|Q3^pM6&KYU;K()>aw5 zjk=Uh`k=pEAl#xaWFL;>ae zoqXOs$@3nowh*VI@t;nG*D@cyzvyspn=eN2s@g|k@MFb`J@aWnpSe`h9lz;kd%*j8 zuYuPo8Q8P1A@!>_~EcH2f{uVcssc;=gWLMOPq(I zW$s#2ixHvHxdbUU78*G%r(^5UbMnXS}t@__fTUBFOxkx!Z#i zS~eB7yKG%mHC>5BfI{Q&;L;Uvv)%Jmve>HfMw&VHE>o#v(PD>lW$O9h#rRm4)o6#d zu_@u<18!wI4R%g;8!Gs)WMfIyEvr?}-6~uZj8qc9&QaGK>5z?jQeD=hAzM!au7B85 z$)MmZQ9sN!m^rfnHQ9=Xtw*wF*tO#=ud)7S2c z6mA=xJC6f7@cM4q;c7p5nxDMY!_qzJ@y*lS&1D#**=lbI1K8N_Mh5J(vW7s^Z!bU@ExfrqUFvIL_3yB`5 zJoe0md(AUoZCQqf;~|9kqhQ?3>qsJfS{t?1pbsf8U|hrQJ{8XdNU`3Daw-wv5Y;1ndyv9}I2N$IB!?$i4y2|Q80C= ziklOqIX59_pDe^FLF&i>nnQ1J=cVqA=3Wa*wQzsYpiKoO&|4|OYnQ@~Qd=xlTPs2? zS(+|fuIn^EO@5)Y`P1qy6`fWbcVG#{Al0WlyrHT(KG(5g#-(m;0Xtxw1n$mK%dBkj!<={=! z&&%+re?zyp-EVcAKHnvIQZnxw+*mn3!cSH*unl}A-A5#5lW;X!j1} zXGRx3b4jQz6>hf)8+q5ix&r$%3&d^q%|m!SfOv%w5$&hryY7XLDL=$^O0)D_6#rkXsW5D)Q2ke_QCt% zul9dd^D-;%QFyZL9bbQ9L0M`Y?V8R101L?I@YVdwc-iV=#Y1dz6M&2`GA<&$*u=naszwDI7SEU9QRSYf4 z(56{3hgQ$u95`Aq|Ks_w+^nHqwWK-zpkTEvl6&klHr~#LS#SDdRxcr-dW?lnej~-0_cp)&G+WZ^0}|bcDqR;@q}-)W+tXPCmyMtp5!^OrPfkiT|CG zL6Br~czOKW`6DfSW<4j~Y-~l&`d|X!z3wbPFsV$xnqEY8LK^X9ymm8>R~h#2dEU52 zjC6;{5d3iFwZ^95^jyTKdgNS574lP6&)f-9PnO#PzKQoVfA_I&wCZlAPl(gG*P?ib z;*S4?+k#F!up0Af{8jN!Uyj434b{jVwRxQm&6bmqpBZ<@OI(Bnf7SUwl*`(Lpl)RJ zi!#?q0$YSj&2-VE-uX9ewIJfLWYDY-ZptNg;Cc@JN5KB-xijW;(2YP9&s{y^6ZtCI zrJA>TE~s#RURd=taZNIKHV#m8Xuit6l+PX<*pY#mvxy9nXvQ32`6GHRRt@RS-dRkF z?Ox*5PPzyhIOyPXX!eD|gR4oUx zs8m+%te8P%H!F5#%`$>l8`M5kgs}-gDYE0yXO#}Bu-YLCIu6-YQq)EZuKG9!hg~#5}9-Odmhgp=X#ir$<>|-hOdw4J4dHf@&_-lMR9Wmv#N=0 z#)DUGlYheiB0i}wYicUz%-n66VVC8Us11D+mZBSp+4-c!ChirdLQ?-}u9``Fv6Ph( z9Tv$IL3H;-%Pj`T11}h1mNBFWc7gO{$o|*T2YENc0qaS(5s?g!o25ntqnpN zjN(1SBO~Io0xYKydYi`^m|w1Yz%ZQ0=r9ZPn4U%1y%zJVhp|VEaLvVBwmNX-RI;*` z-*Qr}jiG5VE|^%lYIy!jwXt>LZVKzIkTh)v((fv~IeuzPZd{v@JWAz%AdOv^ACu^lP z<6iM|8aVZ`g$rEPVc)!MZ}P(_DPRaBK|?GjtW2vzH=R@D}S zqO|sm9edWMb|Ur)F+)f^`906`%io`Wa$e_k&V66!y5HBq#LyqBc&29301ms`Xmr3x zQ7`#0A#f@l;My1}Uk)OC7h3b@(A#V+3JF-&4kxc5FGeTdBM9J%!BLmGqp@usvW0I~ z;fz*W1BTdI6JaTY>vMGB!t5Vc&bEzRImF7(y2BSXg9R!*vsW}?@#&k+E>=$1z5rH+IiDx#z3ZQ!!yDMP#EVaXt#M7aLy<4->-%ppJdz_l?WYd)6hXr1R(c| za2N9*D}Nmv1M#5k+WyR97f2;pyq&_pJL>ep&IN~Hajv(iUF>rV3kNOcu4=;|H4`=M z#Qy1R*o6|qxwu%oN*W?2ZKI+^G6oQK6g zo$@y|rg~RA=k*$Po>2b-1X0TCI*c1&JE0M{vcw;4O1Tk8w*}(ak&|CUbQur%6Aae^ z%M+J0N=y&2#ManqU1L%PX?rYuC;%sh*M-Q1*y4z!X{jCXyq5pS?Z9a8PJK5B!fvQD zEOBH~-XB)>?ibr?2Rm;v=Urmpf?M!ZJN6VJ*Peg{o7)gO)+adqz!6JG2uhbwePLg|a$iCvr}rLfT<<-sice>jH`Ocd zftU|$KK#a^ded*&Gs42tgKf76)m+&rkiXlppNw8K@^Jl>!D1-k+Y$Fyk)3V1Zhb=< zJnVubeobawl=_f7rZpjA$*xbCfLCr|mrRI3(;!I?AwhNf&!8o=+)246I_|D3lQrXs zh_J}aPyizp)ng5 z%rY_NC)n81I>E|;nISK}x|r#+#dQyk?XS$xQBIK;?g%HsWA`q*<;~t6Id}KkwSEp3 zCrvc3SO|@6HbwDO;JKBzC;Bjv$qWJOFZAc2xNqgCD(&Q;?dk;r#%X3g?JoA0*fTY% zesDK6yEUU$c74t!wrTdtVPa{3xf!$Vjj3jD^A>8XP@`#uq<@Q4H*4VeSRN#Zh(d$M zFkKh3d^>ZS0b~RgH9*5I9;xcGq1*T97Sd|5$aKkTYw?0)kk^pHT1jQ_BNLeC1f?!B^;L zU)b-Q5-P-M_`RC1P>4Rs0H)V>#Eo#7b!sr2?NA`oN$`oE^(aRfA)H1B#y78n=# z={>#X9E>tQ^!5qv)B^5QnN`x%N(8R(K;m()IQ=2DlN2JPMsfMSoc9e?V{}yUMJ`zU z;%uUw-@DGyY)ZiGP^z5{+)xIQ^WS*|c8G_tpV6?Xd~z0gK<->ko6^?LPgx(-hK$>2 zwDyLiYgqz$xa^vDq6fm${bBQn!OuKQoaDV9I86y)j_&s9sJ80f?~lb;?svpOJOSjt z1EAvhW^AXN4jzZEAt`ORZq-v$V36y`E=7`;>Jw=HH|r$7UA8pXUOhQ4S=6*4@ECpa z<5f8Pc)k=jcVI!}Z$2&F9hlz4@4`&XuBPOoSBkv>HjR%tcifXUJh&Xc`<2cJdV^NC*)X?nu6 z;mmsb4m^OHH!H{W_v_x<`cqIq=MlYxF93A9(-XP3>4}Z8tYiC$%TFTy$u^53Ln?pR z;A9?yUs%N3>r#^%UMNJ0zCua{?Ej9c)MHTCs5#j2W8n-~I}0Au)H8;Yd9eJ=1#UDT-1bb6Qq} zdNE|e_+--(dp;Za1<>YJNe}=D$5zn z4jS7ZeouWLFaz+S^prP+mlF(;vtLsp*gn!0>*y?SaVZ1J;QM1pm&icv`c}K2w>5vQ z5HupCXeaYkmZqaoj3n7c(-o;lGgK1~7p;f3x)AY^Qsw5}<#vv8ed@6xo=f?6T!%m5 znGy9M3DK~;|81zh7WydUUS$o+$1<)HG~$pH1snA46vxR#VrnzRG5cn;b7#%rgL_Z) z`BbSx|8Nz*vqx+$vaKEbGZ^yA6EzVU*aERmBA~h{sBA;(pUsoZ4XR}8*gkWNp*LOT z^IabCK+tOQ9V|c}d7_$*xd$)W=-S)${5}05Awq(YM(mO%h@t_HA}T{kz~uvU%5754 zN1D6x`EhYkM1p*Wh#N;hW>Z`;`b;P358!65p{aQ-#&8-Aj^l^HRo|J(eq5@EW128@`USl2UQx>7l?0 zC)<072+;ZKZgD3gn6HT4KO}@isO2nZ*x#v;(Q!_?@4wMEo^mrdpve^S9gSLrUWT_lk9> z9Xl>pTgA4PLeIYS1k!Ff5G~JQVh+cb^_MiD20}bAN14YG-&*}w&xcsdAWitP1MvnH zG-W_@K;VyR)4?{eit)_C4_mEE^7ujRddM0FoKWYB)srwpnn0Jg#o?$)4Wln8_9vpb zU3^b8V81oi|7R<{dD46~LP4(l^l#vJ4nwAdua;G_nB>XAhP;`RFz?Y9t67gOfQ!T9 zpHVrVlHCahrZHIFrdsad&vh;l z05_l>sIG*MsICwm=d?)2(~J=4d;9su=jr9@VqJG6{5i6jbrr!XtF3Ghc+0Q%eGsR} z`+hr>;7CV-lb$-fV!74_<=yik6qmK(T;-H7YeYnT=J6bdAM!jQ9|ij^)kXg$o|c31 zsS5x)P33Uxu-y}J&D@A{!{4?`pYE#EU{$F|4ASVgtu^GZJFs$HGSl^S&Rz=eLk;SzOA}*W~7Ta^MkJ6)!8gd%Sy$9AXeT{rfU$pu6GU zz})kc`w4deS<^vJTBYj-+2W|!;w<}nVy;TIk9sV;j##$H*BZ>H@>R&=A9owJCpTji z_9^W?NYXapw|-!W?lQISrTI5fSH7pdHQG;+Yh0~%xpRyrs<%UDcSpss(&)t~6E+i) zyuFCM{M=l#6lk5=WAb@Rx6YM6X5}>mEi{MM$R~R4@ukW8*)@}n@WmM~YEiz;HXb3- zCG=N=r;F<`qs!&5PYK!}3=|1ITuer3>eDmPgqt5y_q=BN$8doO5n^+<3ibNrZa!o3cZa?GU4DeRi%k8e&_2UaYNZN}7fPU>= z30L*CKGv~y|2|~)gh^CE-)B>_osF^vV4+twU&efT?_?jJz}@lzI>sjkB4o9v4O;Rw zo#8bD)6M>7U#_V>V{xR4b=&L^cj`83^mmrRg>Rk1FoocQuA`aVf*b+a#zUKniGX*k z8RzfO=e0`$QZTXeO5}wvm8=Dpn>z!c#B&=Baslva+$<(QuQGP`HlN1vz+&N`|Gbjl z*isU7xA=+~uYb>t_WOa&jk>JVB@Tk9yG@dJrnkL1!KG=HY#AWpDHDBgQ&0wkaA=GB za>g%BFqPz}n(4C%+Jc1AJ!%xVSmyLNEU71vsEkBqM}-Z8SjX;xy+iz2x9Z*oJ&kJf za)FZR=}>WitD|38918EbD&dLoLN(pdwSm2b?f;0=UpVLhWX#39+>9qp0fvC+-sc#A z%Bh1cCxOEaa}DwQbe@JuP_-pV;bU++Dmk$BL;~UtTg*WYbI92>ZhGMrUB3<&6ZhvX zu$tAO$#nJDT*>KPgjl>#yt>>{j)>=lsvi+2u5WP2g1uOR^Lsp#3cB~;;eYp5FMZ-L z_uvpNG>p9d?@vEg38!e`kBbD0f*d^dmY+7(yjd%*!IUs7ONXJBzbV6h(X1D4xzy&F z9ho{NlqhTygzBvVe4ief!xR5|!~`AL^7+7Z{y<>&n!@daguYTW0{svJzE zO?&#)P5+>Qc6w3vO@WiG^TfGH_&K}ZzvUCnt;XB~MGH5AbIqyoiq5nbDmH5GcQOJr zcVdpXJ%Bt}wz!Yn0M;Cba}E5NtzxHYD*#bpzdSxc@L5D{{Ji&1K;@3QIj~;Tu=n`bJHS zk1>_gevw+9@tTKMdpYkYa%5P2uk9B{*Ncp?e&gew%TdZ_1(JQ9(MZWRP#wzCc(vtZut zk?JcDF6Ca)JYBco*poX-iP^F18g_B^j-EM&bI+T57d%hRbxOid0w=xLY%)_45aWnk z9`KYXsp;TSkKeKd*Swm=(Log2Gol%P`#x}Tl$hX+^w^Uk>bVfqlu9+N#|n>NYdZt`OxI^kK4@S;3X#yIU4h}xHPF(?`M<-y_Pmv0_&^& zNDqg8OHGImuM4wYYM76zTr-EgP?TBj7W360D~^v4@w>cl$e4G=MGfc1cIF#xhnywJ4Uq>l`pRQeJC3}4jQm5yL54yecJS%~_#8llX*No3fN6|H` zqTkYTzs*vXOR~>BQs(1zmwSV1pRUG6)t1bzXCRWwFV?%2%gs)I7mmwWUfVZ_4R}hd zL7@w4F6ZF-8T@lbn;ap970RsQl{V8I>>sZEYnA7{O3xd7vpq_li_bm@&{f;o7g&|z zD%AopF?4x}f%Lz}%nvjK{N@oNjE@aIvT-SkM;bgt)=MYsy_|=D!}OcMnR{D*@?lj! zZ|DrI%bq3=-DRizq)|S)f^88hy%3pUHdB5cV0N%PCf+Kh1AhY&e6Q2O{Ws_ag+Op= znrHYNxxj6Ql##>q25W39Kt}^G8pNV#O!*7f{D zhp;@7^I0<=RSpvmU+w%M(%CoDyJL9dieH%Szcf1MXcGQ2B*N66A zE3QfA; z!ZEGgUKWiO6PGjW7aITbSPJkL>NbV_E*%}%C|qcBN$=2)u~(QQoJkDD0P#xy*hD2V zF$;5-kjx6Iw2&7<+RpWdu_hK~=i`93N{c%t)vAolvb#jlZpV7<$wQf{*AVY z`J3)Do)3WQA?oz}ccjbO_!tu&Ys#fLMH^Q&A0N^5t9?B07F6R_GR^qm-A8g6R7BsZ zdr{Y$0gLw(lSr_P`na1X^_m-D?5I0abT0jsg*i$rI5wy=>(g~hofnl#EHOjBxn*;+ zquuC{FA%K^%JFJ@3)A;4E3)ZlUJK;k2rkw+vC68N zp9d~J`7&61WhN$CelzXZ>cLcE>wk2+35G*lC;`$xf6B~KZ=!eC5aq~z@k^;ak+reH zGJv?y-e2BY_wfxwa+L)9bZg>xT}XgnZ^m$cI$JGf`%c-YZgXyGL5mB))^jO(EV0u8 znB9AY5-(LIqxi%56NR9y{A+e;3He8^RjA=@%ZS`&YyjTDOi77**9XOCA-m92j4{4H z+66^r8K_FS$;NpJxUUln2S{Z4aXl&cl0?(?x4mU0@{eHpi$lQ% zX~GScqBKVrF=^;CT%ok*4Y;PWKIzn9#lAydtZY+HkATTFLAGh{!RmGcz>h4X{1pQg zBnhn8L|luEqM-=nwd60IE?)n2M@CiwW!$piE6{NM_JbXkrQzHi zdso&k7k%T)Lr`o3*Oqpo(-z9IZlc>dZhPbgvmLovH{YzcHiJ7gB!6Hf@gjJCcmMI>b%fL>Y@G6_A zLGKnG2k7L;+31Es%*49oQewHzoiyl-5323S9Wm_&7R3vh8&79l*_7ACXb!bWNdcOx zTVb_;pwW{^>5 zZ3+QPk5Lmo*VzKcm$q8o`eu!;S>zu^sq(V%GVTNK{(xJ3e0otGFAs0FN6hjSp(_l> z5s-wvaF3BIgEzGK&7n5DLarNFx0y55IT|*4ZZ2y}DeiZ;7}}SQy6Rp?hU)!xyLL`t zGx--ot5s~m$Ym9EE?uyyXOVxd`x8RN-AI4U4gSsB9u1hOdD3OdtScB!yA-X?SQ;S` zJgWB5(vnvbcTMzNp+yuZUX&(G!NHXz8|z;4-|t8sW7I8>?o?22RrUWo<6_ZY%!gJU;xDFW74jxJLH|I)=|?XYpkoy+P2cEcQdWZLD-Q zPleuBW%xD+kmYX+BrX+5PLI`y(6TW=Bf`Q+mEcbB@lh7aL{^px-F&zKI!WKT#x`pi7ABsHx8l($ZBn&V3?XIBIDS z#jV9P6~x$ln}1fVY3X7_-~aO*{dxz`>n*YlM*Sq>UM5U zgu3{xHEc0oM&6g{ePt@`*7rMqGU#h8e2%?dTXofKp4%?%ZVYb#$7!3))$DyR?l+eG%)7s@yKhJefcfKj+SMl)A;v9jA)Ek2l3u5;eu?+)f zTNCtaj0zo(w9t&Kz8jChdj$R(9^|0`yEiKUwmAIEk7Sfyl)eVjux;eeli;BlMfWHw zi`-1VP3Vm1O0`VyU@Cp>_DD?r^O)aOb&Rd6>Og*cAk_fe_@Dq&1K)^ z2UbtC)-m&bA8hT-t^_H*2rob?AO6>z11|LArRkDnP+Bdt9$nITtz~K@Xt%~;hA#E^ zUS4TM2otZo6ejLX`t24KzdYO?@Xo7ZX>{=M`S3-h5Uv2pYbN*d@t*-jU7QCIGMPP> z#ZJ!+TkmNJUVNVAFDgoZRbwi*`yqR%R)k^S?%at=bq0JFeq6{Xo)f#6PvClG{J|#- zd{)oWpCYA5!^V8tLfDp5x3xCKKe;Yx+!++X!7jUJd}Blt_fzAJ0)bJ~+Kt(O=HGe? z0sw8;IUf0%7o}{8$ojhB*Jl6UNBjOlb;gqM7fQ8#-{dXdtI&0;S!#d194GYv2|El4 zU*z(H394w_yq6rL|99&9Dp}%tX%keUNZ%d5wuK8GaBgbQv!~+AP)E47M=) zZ{(q#qHBWGu#9r?X3nhjvTH;Z0FxY^tdZ9c=oHQr11AJsm%_S)Q+nP$9M7vVpJneMwh zXPgJ|2;B26eK0kHb>>&r7P5#**Uf>P&tP@==Xjm>za;WWiP)cHR#>aoXRwQOE?;7< zZ5z_xsn+L~K1~!ksNdnXH7)*pv?Kn0T4(CYJtIaLsejXrTEAKP~*mw*L$JjxLzN zDc&sJ)jhpg9Xk9}dwBwIo+&rgwk7!t`LRfYVbag0_z!{>c&660uj@u%V@_+ouN1D- z_FLk1$qb9sXq{)TsU6ZpR;DO1uKRPHyLALbO*L~7kb2!cZIp>%df#e6`dhUMCYXlb zr`$>hdnJ~C1gx9X`>kf>`WMwTg$E3=lHL~DSyp*LLk?f`Y!ub;QD4ot1v9dDT$fP3 zxe_owMsk;6sTrJDwmUg_&!UnNwz7RVEf04ZdtCF-XLyhV?VUpJ6ie)XM^}TD-_Lh2 z;6H!p6*iyWV`=hiJlsCN%W402i(h1K{g|LaYbm!nHok1|_?uGTZ@Wv@S{j;jAf)rAp6kZr1j*S}uf?m2 z9i>%JKm$`fEmEta2M5!3bGcy*W+eachEizB1S-(?{6${m#tR#N16yJ&{O4hH?YfuJ zoJ{_sTIA^*Wg>st)k$o2^XeVGn$a>t)Fnic4Tc?%jISSw}vxWDm?; zHR3VIYb6nNxb&RCw|QmA+q1CfGs6aZ!~dA0I#qquFZee?@~LfM-t_4$QqoMO1|_?c zO`0(L|8#<^IfE7v2s4LS5!YJ-OV+<0(sf3v!E{1^u-ppcWKpkhf~dzVK2Tt-VgJw8 zCZ9uCTB2pxEnb5P-fIaDgRc!19lp1@0Tz2iPxp-y6PRG|xxYB^ePIthhZa{T-s@`y zl=3?9<9CC;h5enELl60D6RIy?<)ar2{z(5U_@^11dzx2lv#6v7Tv*u4$aQiZ)V9Js z0Lb+G%$HzOMn@CvT!^H2Jj|lu$E@Fh1wz}nY?*asdy_9~UA1F1tD@s%KIOvGn|T{g z?P>EV`YzbP-YbgP4P_9=EjL0e!A@oOUW{<*u;gw=*@;v#Z|Zz&@yIbfYjwFjbQVA4 zhl}j9R5+Mi`)BF+C0d=ik$S|(U$vWn^{f8&Ho7n$JyYs z)6Fcrsh$t_Y=t_jo zmjg0Cu-*gG5VDsYOU!xS{druCz^|5Brw_GfgS z+iu_4yK{YR=A}STEI+juSJ_2z8u`W3C=EWm&G+g4wI=AvLbRu65EBCpCQvE7v9>pe zvCMO&>V-4p7XKD79fErS?duRtc=4cH%0FExu6*GNQ-JR~IDY(Qcrx#GQaB>#5+3$g zwYlF;`4SJh7p2gsjxcB4w0x6YI`eo1<6j7I3}*_zVP`sn$;}lJ8F1!N>c$lfR`ohs zk`m;eH+Vd=~-F%=@}nrmdr#1&+ZV_GM_(X)+4R z4&Z~6t^l4)f6+G;e6Qe>kP{fM75k1(L}+mAE^u9_l0=l9<{K68rC)h7OGSY!E4$lt zBg)fvULplcGkbk`Bhu+dMvxdg?B-DOt!FM;-Y>n+b`#ae=L7*t0jY-Kc6xhqyzu9s z+2Zp`4S-V5pmO~Stfif~D=u$>ZCzjQDrwu8!B?KFo}mXi`m2;aQF5P^4w?71dya&v z=(^!mMj-S}mZ(B?I;OL*-8BZre2@MXu)#Zi_b1g{eL4wbV1{@qlb~}BC|}JX#ZNi4 zBMQ$MKAC+2|F0Kd^v2-xlky?eK|Wb#?Q}(gc(qFSxdu|edLbJ?BQ_T&DebAn4({vg zV{Z?ULdsrqsa`8Gy%y`%`8|6**YJ>Hwqd8!e15yM81~X2E!c`H-dL}AP^zjZOX=!g zxq$f3UDw|+3u%emQMgmInezRWM)0iP60<4|gOV-*QdsHRpS3bQZ!;O2P{Gf=;yoFX zp`OA=OAqWqAo95&n~C)W2xSUYb-kp(JKUOL%Ph^2VV;#4W$D?s>qHO3HuZ97M*53r zGHZ5~-xfQ}0qC)uY~PX`8ETEN%+`NTO~=NqyDkZxhz8q+RX|^x#L%w^wrq6)%ouc) zxi8l+%P40oxExJIQ0Viy`sp9z!zb4#Vk|!HQ7Q~OtC6y^j=Eh|l10s{OpemmG9@z?r@bp0EU9ixh_*}kF8HyEwfL)xDbYJGh7gkk+f}p;^xYP*ilUASo zK+0~;%NQMwb}Y)J@kJ+u@CyZ9vv76uKCWBy=A`Tp`AmTeTiAy}o&(kYHU=9AKX>HO zDrnf(N#lN9j_UiMML-i>X#`KBaqo(o)(eq_F?w?^6F`Ok0E_%&2vxOvH~7i&a3=02 za`ef@5J}o6Y)JS<^yqM~9K^v!UvYQltTgb|D|8Ox4V#T&Y5J@7k>9$b)X@rh_+lel zMILx*p!PhyRafp%eej{|vKYJsoD9j>|#E^GNn4OK_wX6h@jv zRFx_LJ6JMEvfa$-GyRn&dCU2T3@IU9@HY+Cz@K8Fur9Y;0gss93>G+3*s#*=vDMKi zwspyx??F}fK|OZLTB_PtEWXiP&;YmJ#Iqn@%9-}jYB%KArJb{Gu&Ly@i$XGcA5Pyh zTvk;=ANEo!PFO0a5WO+W;d_{Iggr@e6IkIbi|^5sR(f%fD<5r2DjC+(9Bc=ZR~swz zr~E~*tk9Yi%^eS||C(`Ov6Zpnr*qBVB9KdkMLpd4e9S}xZHKDI@AVFi0|1Im6R$El zwtzdTAAEEbOB7e&;-;!D2M=hH8YhU`_4Q}Xd#wo@-5kL6(CU8{sO@)03dF|9&DWoq zJhdt7DLe9OA6}OtM{_JntCrff{22UK5Y<3D)$uCMyGb1=nQ`j@9v)D&L|bO_8p(m6LbBzta@81SX@yIgWb(54l1 zoGht#1H8(FE{nsssusBy2t5KyEgTFc>skWZR}_rVPkGkDE>8%HtgyUq9V)PWCR0`U z)tm59wP^k-l4|1V4z{K@SExevCI&aR84@b~6p^{EEYz9!?Hu%Tl?esyORw8p8Z2Zn zzLhBDFa#Wq`c}l>0@fm-cYaSwQt9l4@%6t3RMzflNdK3-zx>t#cvHGga#{^}QHmuD zJlUxFaZVPBeW=~mg>bfDH{P&UTsx2amYnLpDAf4a`TUE%IoI{>4A{#(=@{A#3zOj* zUWXZdpKN!_&+K9xCr)i|2oyJyBW}nB?`1bh}`G3C;{0q6W1kgF4n{>O` zxAP^sCapx5AbW21=&bays_fxN{dNMu7yApTmK>#l(o-dsFGI*^K90W7eIJ{U~UR02SeJ zy>0C_i`Q^cx`_SdSPh-@UKiNF?=ZcMouXf7X?mO4%2T7Q-LpEfFIDIEh=at%YWPO7 zVqNV?MgP`^>w2;_>xmD6lie*9ZQ;gCUoM;*tk0qis)@cPw@7VzZU5W0(+tovqP*vG% zO{|%YbNESkrol97U?VyTm)T2-jRlo}(7yA!sPV!er)&G$BB;Tpdqcah$zN95SJuAF z)9Zu1s(o#A0|#N-r&PmlL4Yn2>wX4)g{{1(3A`Az@pG=kGhBnZZU6}zY@V3`s0}}i zSq;*CTN=owr${aug&XhhW#NT*FQCG%6ua_vuHMt)l7@g2tCcGMqb;oDsD=md$DyAq z;VH#1&<9G491Adl8g*TfOQMtpdXPo|y!Ue2+ZzS}kz_l7(%TVT`Qay?`G^6PZ{Xjc zWD6yKR&VSvZyyUh@Ow~r7<1X>xqhf{$2%#9>H!&ck8DJDa*)vZe4a_dzFkr7hg+pC zdrIa1{QijO-U^IrM7WS&5q~5%RYdu+7jPNLQ#Ow8aB&^i_}f)7V~o0PYB<~?ZEC$ zhXuF73(VOzc#whh`bEyNipPJAJ1j1LB6{H)e*N<_jGC9jZOI zB?t51dywREXNi5TW~&L3qx(5a<~#S)SSI}eI6c7Do2-lC2)sI-Wafe;Bej)lfCt+L zU2e<_yY)O%3SLE)UxScOgLCjFGcKsuZ3C|;<*B%=0A@JE@(V}Xit?5GlciR+K>x#9 ze3cN0*JpgF9@R)T*JQs_RqVk~<%5X;08>9aSTdp$OAt6)&X2US#xbapUL;O0M7#KX z!69F?1TpX#H%LM;M?&*m2q%M~C?VNjk4K7k=%L}te<~x=6ItVhc59HnU8dE(H`||{ z*`IlVk2gO92wal-0h$8b!{-pAyK+eQj|ncGUs zV})X)y5BppSybixcIEC@8oLHtl@H_6mg!0siRsGivCxmORYlgT4wgSOhIVHM3u*L! z2bUK8vX&Ok0DaePA}~hfE5j2%%QQG}g{+%stsiT0ggwyy%3gLvhu^#_Rtk~a4k?{q zh|Y^&&r642uHlm~ZhwlcmY;irl*<)3>6blc6Y)6{N{wB zqs3(C8GP@{AB`yOqY7VgBWICe#dE!0kIhB%_w{Zoc121c?WbwVP*c~aDbZz~wV6xN zPqAX0=*`bVg`hOKd!?**J~*t{CAk%)E9LS(6~o6NcyE3itWuvbw;1vi|6r-4tvNd( zcGF`67~o3$L7eypQ(s|2Z*WSNZtKUM`BVqHuTKiJw7O_1YyVPEKnHf{D`BBT->6CJ zD#cWeO==%R$Qped$K-anwT3>QaDOl|$s518cT-0e7cG&JB6+?J=;kQ4{$c^Adi-C9 z7cg#&o1mTBcaB_OuQI6D?ZzecKs(sNi=7@i-po#*@@R-w09nd2=Z5)mh`3P0=1P)Q zjAG~ZuX2rt?rLl62d95%v}plX_^hL(|KYMJg_$E72M)}|8Yjb&cQ4}cI)PhA>u>&X zl`Povm`$SJ&QaaB{K`HbA`#dfBek?j0iS=)+Y;Jlv`Po1m@u|z6uRb{gFZO?kGo%6 za*Z^;c6{LuhsNoB0l**DtEW>mk}Cp-8ASM&`k6`LZiwkhDE^1p)&D};o{3xuv@ z^yX1su*GQ_ixTibwE3o0d1oOD^ivVK712P0hCjy{$$6dY*Bo4O9A*h-KKJfxx#le_ z=_}(y*_>o8!XFt|a;qoG*_|KMN{fgZu?KdkSPh`YQ1FWwub+=;dXHgRW>I%o>Eo5d zZBy-Sf1Fb?pxi(@=rlQpj!8NG89gfEK~ALl9h_}Q+M^bl{KhEBZ#>NTn`s^R`ZIp3 z_F47lLXcy$^U)KF;dx)gkge$po{3b!)v*x$_r3D9NDCl$Iz9^{kMVRQhetjoHk}MR zM`pGTThCuAXs+4h{xa;_gVP>gSh2PgBhcTpfuIA9?>It5uqQonmMw7sn9$_$8ql7| ztCNSAyDQ)CVwL1xr(U0cZU)Yq1%Ho->y>Y`KgEY8bFgs{1h5{v@8h1lu>Ag>+ekpG zcBp`xd8E0o5^2V8-4sHjp(5>2UOf3SWrB(YHHwpI6$D2)9q!cjhkyQJ_nd2d5=MC+ z8a{Td{G$A|D>k43&aiBr*iK%mZ⋘cpP&ho7@}e zxhwrO*o74%q>(PcO=#5!Cv_J1{PXzts7_8)#^DFxl+aS+E=FU~y9cU%A+CW@_pagp zfoI!p+=AIGuH~m290EHS#ZfgrYYpQaX4cFc(@gO9Wl@8t z8*6G75j|7UAC2ad<#}^AbXdaDCY`sR4~xu9TX%jd=80WM@pL&X8F*=|dvk2{sm<^g z@61BHj{oBI`X!`QxZqNB>(cN8b|ZG=jPd3xc8_x%fy+D>*vgfE6Sl@7iu?(^_&}=5 zE>RuvLkzCz$J84da5svz>}dS#a1i^SPOAX-cx?LdP_6PBiqph=oJD0hMEx`&zj($R zdOBg%u6QWH(WbxCgVeYL_cc~IybX+N11kx~Xh?52FB44?40Q#5qKn#oGtJkp$jv5J z>SE8wiUau$1TKd2v0F=^pn`ytS0=9C;_in%Uc#FDlh#z`?HK92(&fT$ zLvLk{6YQ%wa#en<@_eR^^A2y%A94-@A&%1KK~8TR1Dbq*n|E#j_aZ0eIXBLGMX_afCh0vUy${@$y!OZVD&C$c9WqfOe__7Hzy z{u7%{I2e1GypFUO3yNoM7Zb=A=CdyU0vh&Sd^?QY0K<_Pf0#cUjx#`KiaA;=PJSjQ zWf}+^_6IvFvCJ(bWJZu!fq@0-expF9Wo(Lo{H82ceg+nd8LWjg9IB{v}=Y}|Mi;N123-&_5O)DH>dz1 zsCM4V1eNZ5>1J>8fpkp4n=CNuuvHG>nk?wjyXwHhW;(QAz@0qz4D=~?b~sC zfUC7`A2ajG$Gq711wJ0M-G?ErAM~0G96u4b&qjNiTg^pgYk3k9X~haSBvpoQv8-p| z)g~X-x{E9%l0@5mEI(O(UsTceYq6`IzYFV#4SXLAX12mUk6FX^w$eXs?Dme~n6;E~ zP3=khFufQ9-E+nR4t*KhryyrdCPq$d+Bv>9)Tf=XyY^ouOBNMx4l|Wpes|Vu*E#S| zyiTgt_x2TVLaPj*#LQGT0CKOwiL2x8g#r_<2r3ut%>tQBt3dSj`&CI z|5Dsi389(oGL2(H3h@ewM|OBeyC3GRB#>fsPOq@g(ImR@^T9vw#s<=ZJL{t^@U`I| z3`b*w)Hgj_l#w7H3kdz_YGt6{6%vN5_($3KC47q#udZW0z@^zVU|5Jr$bFugKTd>+8BO z)OP+N@UB^AZxSfqrSfQ<#prLD@Or6Afn3GO<#$VdF+BM^L3lRod^_+Cs zkaktj(@lv^?Q78;e04!BdEXb5O|@}DsPV6hYOBNHuP&knmrrt;T8uFgzPR^tY46*2*kMNKaG?} zAh}y>pYUbyHp9be@@*LZZJCt6kvvQ4YbZ zbN1K+!!u8P)6>*>92Qt{tDNte+uZ`B5uft+!MIO$U-_=-2X<99pqe2|N_E&vZT5L{ z6aD|}=yQ!$$7Q_ahVOJR>(IaVcSlH`RgIi-f|WoTEoY%pVYs^^w|op?hVS0^NqvdLyK8;r<8lq^l~hCepTMY-IlGPZpSrJcF<)+Y z;=grT&vWs?>#@~BiQm?fq>F=|?(si-0o+dPzD8d?Uiie&p7f?qiV$4LEZFx)>I`>! zG&gsYX}Z2fOl{avY=uh)gzHbpu??GIg3ZF8zS{qElI~KsN{$YGIKfo9Q1-^*N-kt! zL{Fr}GS$gcuw3cq{KD}gad5wyP(YBVY>*qre-bGQ1@S`Nsn-{IN5lCUp8OhmpCRbT zE4mC#s}OD%F*Ox6VDL9VZlW#VpY4B$>oyI;da_m5(|pxh|;OYCYeOxhPZSfqd5DVG0evDyV=tPLgdbhiq*9`pvdWfFaKZ6sJlu>(Tw|%K0>Xrm`rF-YMKcHT0xPc~6JZ*dBF(190!_Xbtn8 zzHB%f=Sf|CzU-9G)EmbWtNYZ+XfQM1hg>65V8d07-9 zEplWVYdJ_>&IYI$z|^|3I0ddlxSXArg@r;pz=)XQ# z89wMUX=;YvISKimah3V0VE9+xHAkkXN`s8uqRM-@XZq!cN7iag62mD<*Y>(HNIh$* z-WetzP62xYCgE&_J;HSojnq!U80xTbxP%e=|X zL;FE|7CBEUO7)+|hsRVjba2>)!7cNZF+CFeWY>-4|KZp)d7t-J1{jjXS91H;0#B~V z`!7~y5Pvp(rUgd=d=fNJ=dM1i5;14W(2S zRW+-bw6s(WY3a{AwJ1smT7saVs8ULdl?pYrHP$?fc`9O_io{ScN6cex@_XLrS@*f? zzJI;%TK6t1YaQ`D`+U#YpS{nv&))k;8G;rEjNY8IRLS@`oVQVW^vNH>(;coopt20e zcvg3U8t9WBw05epi$)76d}2oAGGF`Sq#4AMR3^Y1)tlXU&TNRB5P^!-FS!$)AG^3M zabV>$EZvdxdxcea+O5AEF>h=TNb=n{OnT1$_Ep4|jWVoj9%Q+U)(CjssJI+I8 zT}f&&*AYiYH=U-Ld1kwR2@Vb6M6JZhFzRmh2C#&&Tu4-pVeskh8`PW! zACja2KR40K@zzuGc+R#lHs`oFtMgZK%Wiit%U#JV8GQfa`4x{V)ntfuKjOhJKR&UQ zYDT`3dry1oubuP3h2%vszVoTPp4EBbxk-lq&Y@)ZrD)*Sto%|)*%kRkRfQYv&k!82 zKa}`CpOvh_jH@gRK##5@a>l)kH~k0aIS#rRrz|WFc(1iqCm_Ru5P6VpZke$_q6j5e z@#nV@FS(Yl-fY3#xlCyuchY5dzVn-sB22U_iQy88ZQ1$cNlAliFPv!U;Jync{Lt1P z{0C{F5v+U}CwSK6m8&h5DI5)Oe7WP{z(ji`J0p@&j9wT16{8v&%65EiJK^ap_+ zqGIud>Y|bA4g;?}oyx~e1mSVfc9JElc?_#{uzK@fJJld$qF7N)q48_i^`hwu4gG2% zv4}vQ$u5C!#-EguOX5%o{$h`fWAft&$CwoP*2Hi|u`7yKNC?lEA*#8Ge zyW8`#kaU2Mha*iUxV-TGFa}QRk4a;to)Ow1xw_ce+)tJzd-Fig3}CmvR*`a6+K zyJky3hi%g00WDwOz>r}Zi6Xe8BGqeg8$nKtC`HrGms@!r(LI|CE`J)l(63s-r{{Rf zYC@QRau^TSrz!79N;g;d>Q;;XgjoUsvbD8ub565OfYUnbC~xAI$Un2Au=Pgp>*udJZ(?@{=~!`tK-fTH!y>m8 z5fkfi)3?<>EPUqj-w}($q0MI2Q%Y?h9;@uyC2xZgWHxst?!1B&$;P+rIOf%f9p`|c z8J$qtwf~F-ILKR4Iu&H)BC%4U8Xqt)aGO46AV}|1+-j!23MqkSOrt+c&bb|K+q{nJ zg}%H%p6`ogi}{vScZnk;sU%ai?OEb8w;=M^DFbVpPDJp)9n5W`CuV%k$(I$+{26p- zvc}Rqm(=D9cCJ9<%Sa5NDrbRUAFKc=vejtsUN7f253VG% z`D_v$Z{F)jA}$w^xwcGaE~5srSb;=A!Dig{nwztZuWvKRgPG^BH5~mzf$IlS{l2h_ zSrnZZv&o0^yVEUSgu~KW^>l+rX(-yN#ckoi8dFu z^EgEIV5eEc)n)uG!muS&>f_&9fB+?R{varHj^~Hp^mAL?{pz2XJ+UeoPXnyNc#n zPs`MfYZMogI8+PYd%wc1*cHj=(CW_-bZn!2*qWhJbEzP&Ci}6`i{fhvH(8-Wd#1Ixobm;9b)-P<#*ML{>u}jo%E1D9yvnDP<*S*P(rY`#`>SK(5W?-|nBw=+lT6zuCfH#FP&$gOQ)*Hy{auwiAeZpM^- zLrHOmj?3wwXI2YXjlFYTFay4lpv;I=*v&>NF%TSJSv_Nky3=iKTlal<`r|n{Pw%>? z;T0Za+OjE219UW?LSfDH`(22tbf+Xm*tRvoPcj?^nMIRw8sjuY_(DGQkI{Vm zt7VK&32M1)ZkStf-R?qI`|xF(9U){k;bQ-?jgzMsC$BBue}&ojh*{0hlM<6A#3Kx~ zJzYm6TtMNXHWA>sBBWl`^gFtm zu=ZY*zmD*KcisskiWBK(dIWC0xnx*c?blU1wyY&k( zLjsFYkS6-Q$3Z=yJN%9sgrBNAoE>c_pl#~tG~uXOVQ1^|EI^$ZPRRs$#tk!N9&UKS zY!Ur_)&0PT`_mj7da13s0cPYTT~*o+FqnlJt41!aMk*|Fgs8Z{aXT!IZKb9S|N94O z<6-Qo;bLbC1iRXh27gnZIi1{ix~7LvQRvZvzS>b(0JOs4XimSH!x+7US?1Ymnn+xx zkTy3BpcFcG9|t&E^W@S`9plz|FEN0yYJ{|7oRCHaq!vtNrLuxUOt z%RMrL9m+c{05~LMi$@_vy{!>4L*54|;8JmR~`RuFhUsd(MDT~<-)?5BT+V;oE<7}*t z`>E^2a7OMY*-+|hNZ7bl`TPOzzNqw09aTq8dMB1S6LYisgY68{s0}c6rR5HW1|6HE zMf<4@k)|{u$h8yayg2QQMOM{TgJnGNPNp%c}aqIG=#Z7}ZLbsoZjb|5-`Ud{1 zLGdSId#zaK^-cR&`T{8Qto+x@r{9iQjZOP+*3+A#ddNEao=`ESY0nKBGHEhMsm{b3 zM<&6FEze9uz8+T}y9JbgN->!7++tk^eeFQPhH1sdbXEWXITFlf<3>{?|F~#3%Uoyl;DX+)+6Dnb0S3Bf zr?Z}0WR=$JHs*t*+}}y<>p}f$L~>9f+58p+TM=F5UpFK52vp{9qzb02emxcURBl>O zlt35>f2^f_=BzXYRkBRdLt4v z3#Fops_UpwWLFGmp(2s0M_*Ck3#RB%DtZej+AfufF~pDpiV87~qLckUUSOa7X`dJ$ zj`C$DwSVy+YAYe+MkSaS?2=LUFWd!^bt6)#U9xfJw>PQiHoNfGa88&i6QWfd-T!it zBe*)giaM~x#llchA61n8FC#_2-B;hSRc!WIzuRsOUvqY}x|%qaaBw7xccjjjQYTx~ znJTEi-`i{C{431o{BpAX6>s)9#*}}jOG`+JqpoVd-eY0=FVr<*Om}?p+6UM)X3ffQ zMtyi_aB9i-Uye|q@|(I2R51O^5-Pf=h^KBs|MG+iNa}hzQ?Wv&Mgn`aenp z)qYkCK$XcaS(zIC`7;x?TGUuki)Sd3hu;080$ip4xE5r3dyO)qp>Tgl3U!Lq5B2@2 z5<#6IRm!O1vO|#}>IB^=pABjUz!Et}24Y3oX(}bC4>O$f%-`?nQSa|*X*fY<{QE0f z-o588>oN9BT=H0z^~8n`a73&VFI8O&IjO8$+g2q8`SKE4wkv*}z^UNS&i zga2>21H%JrK7hIPp073E-hG{3+e>;Q!=wgcJJpe1S!84jTc?xScY91{&9B*M~`hDdLaI##;F;XTONjjnp%#LkOIs%epQC+VPVK1mL$Dz=mH~nm3i5RQ` ziMPgF(dlu@dq@LzyuFJvjlsJ|Y&kN4DS82%eDFSkwSaj373uNu5{R5~_v({Wuu*sU zAyR}g8ck-~z+jvw$rs4KfM(S^I;o0h7cL0pK7pu1#s`OT$`6bxdQ*2o2Pff9&g7m= z4U4&@p$JI`2x&93pv$G$Oq7wDI)PZiGJofr5VKuzqO)^dWDd7fg)MQ_{x0?d#znM^ zW*0wJv^(esW??ynf&zt%!$sUFckJX12m1hP8n(jAi>H3VQz^j8E>Tl+QiopNKt& zeHs1!w)pvnW{@iEpzxjqu|qazGeK1#v&z-XF-jKp`)aY5wzz$o75T&y8B{61P{#3iCXZ~QJ^dC zJh+?QyT_y-Vr5YUIgMA_sG9${WMb{>fNvU3kelaZY|&4M^$0*L1LU|m9Rw@vaU}o zcL0}K2I$ZnMG{u)M?8BceDd95CA?i_qbifYIX1W!WP8!5zjfNPM||Gf$=Uf?rrJ(; z->{%V79f!D`b8E>s+dh!T<&moG($gJ@(5Qk<>vn`M2={#m?7k>(KmepsI50dV-7k% zXVM0vB5aZ}#q3W6hah~r2dg$@Ww#|93m@_)_A%ny@7Li5wd9D>+E#6 zjV4}ZaK!PsZ%gIbsv8)VPu(zU=5}~8 zq#6^a?K?j=)_AtN;RrgOpgZvTE#2{E_Gq-S#-*@22h)`o-Fyh618uFqho5P`wK*dt z)t>IpHr5`kYAbazbh z(Y3{-I#T(QvU^uDVNJiia13CAE4d{WF7E09X;`2M@mgR6!!-6A_Ukuk!mMyiO%Qy9 z3%TP1i{)Vm>*=!{gs9CytPCPnwHfo+lJvJx>SXIAOrb8rf6b16>MOLFF+oTX`^Awl zZCd<9zP?Qs4box%%YA~qoGjj#Xd)`8H`7`!cT3#C;tUmdv^MceOzh$3K%4y;hYS8TS#pc>kCp?;-=i9H^;O>WMRc^KO_;m33uO-_ob3(H=M83W;+%NR0&m#@- z=jE3?NjUBT2O1KvgCrrjEZ~{HytMyN>dbKcqPY9A_l)n7G<;dOen7ZBzg?l> z7pZX_;-2S-SDD$0j^@)nZDRQvp*s}`Iw7{y_R3G?`|p*at&|-;-5VuT{PFDZe3@-Y z(4X$YRWGGY)$bO#ucsXi=X!(v29Y%s{jofqlj=5U6?nL4r~T0rHZ=(CQ(AgMXthyx zT}&|9s2G14x#81zY?Ztb_kK62^|_BT)P--@*PvtH68aeYa<5X)J27PR4o zUQHvPERAsy3EH3<3yPkqbr0N5wL8Ng{Vom(YxDWq?{}Ltlt44Qc1KJ$iKx56;HjLI zF7BfY)`0KbYIB!kZqK+DH!&>UX8EK6GX!TSf7gu!&mI%QL6+-$+y-2x+vL<_jP@`-8049Vqoe>x628 zK`jz1+B)8f34(t*-J#qtVJWkr*@W~wZKt;jFVT>@ zI3QL~{^)K=*RqpD3|Cw#R`zBYC;m_vgxE>1A7WwwNv>(#tU6(Sn&WY#eO!hDC zF4?GA^$x%8Jx&+z;_{T4vpEvIubnA9L=z>0c5|Naplmv+D3Xz9 zl=}S(?>xU1=?6Gqg@&u8G*LpTKwbJ;NDQiK*r?_o4`Ne%N5Sj|BeJ&I%Ck@t7_kXJ ziLXh&i*iQeINb+%l5sy9b!5}jw^RDpD9XU*AOG88jZD+IX@0L4pK0$J+d#4Q&1$oz z#=sdQSlw4o7ES;Mq-~>?-YlVm0`VVyA7%oDCrHkA4D#^FhkyK53ryG#?kH;WTOPEZ z)6VoH%^MLt2CEnJr-R4NTXVlP3PZL9IpEJ`_8U0g{KElO)UZBi=HwtRSiBwiXO$a_ z!Jp6E90WE&a{rjS6#P@X#m7H9Ap1p$vqUDMq0FYatmq$bA8%E>%c7&+8Z#^Y(ZS7y zWZJQX+3bcnB^_cY-S32wv@MWaP9MIgEBpw*NYO*ji0kL#z}!Bz2w6)y03S2(p@=qv zi3Oy5(D_I zhP9kArAJd@JA)%m1Dr-28w7i)Wj;l#Bb0LWVg%*(G>4HP*d^;M{z=4n&iWj9dmDyf zRyKDfjK|IN5JNBPoS~wc_a5)~A67WKBDKm@({%^!SYLU8Z%nD&?U3aQkl2(uv8dz%wqyU~r2UY+St(nvdnNceU_wuFMLd*vZ!uejUa( z_Nt&kC9e!enj`Dpb2jlDUidhiQD8?B7SF3gC^fE025v>hwI$WBIV5DkyFWUua`k#OJ!<~g7+fenbA2u4rh6>A0_YNBO%8RZKCHv94>H4R!2Xm^6T;WXF;;s1$ zI^GNxQN@qT3Y2+@&16m9Q5O6ljbtQpHV{JrxOXZ$eF!V5&zsf#0dCxC!f0WKCcAJA z=DZ$>+WhW3k;${2T+lzDhf^E1o9gWl-8HHrf9-xce5YOp|47$pZp|q@k1KTLiV~yb zdq;GD_x{xESw$`3TVxH@qWC(^W!#V-H}qEe{QAZ~1>!nfynpK=pTpjyOVO77E^E+% zX1U6YOxDl_QOw=(yd%xL@vPcve(6O{k+mKQfw_b;PO}R|o%}FZn=kBd3MB(WcXaqF z+R#x^^H?C9S0-oZ{kSO~RB%$Mwjttc;;V-}b7biNqrQY1sUy=^6h`HThtr zmo2lzzsIxWzyT-2LjA4D>^0$3SHrSo$~|>QF?;RoRcnE`OEjCCL1h1REo|X|v&o}E zVu!t@QeX4h7v$gadkp@QB5#dmye8!Ysz-#qTQH$X_!m$KjvT&b)gGpbsp#`1Yyf}U zYwfq&e&!O+KP;S>@)xA}TzFKTEil$)0o-x70xtl7`4@m?0Zj0Ku~v)O%u<47%~&D# zQQ}PY{#y5I2g})kr38UlUUw2g8|1l>@~)DJ&VCm1d!Qrg-VRW)T{i$nF*@wqR%a)w z6Ff<`I~YIl?RcEa7 zU57E0uWGHPNPtQ;5^QdJORr(L<;Lp8KdMZWvunyt)7!qz#fJPybUXy_>n;62#MaiEg&sx;m_NL$tTQ`THJKQ4--+z@ghNlOJO$6M9qePy@}i?6R~ko z5w-POg}wcqp&i`3>-2>cC8^z6vnkRDfmxHrqmetgGkwGh63+H0gwAI9)b!#Z#p-(Y zR!*tnW|Ho;+~B>Iyt=t7RlHiT?SvwDHsMkc%W#_uclU(!=b8-u`boCa9jGm1_S7p~ z?J^+i&g069PHY(G&z`m|T@f!k&Y2-pl1xm8BL`6X8Ae)sDi4X*z80J)a-NfL9KXPUt=DD>NMfyyfbm*N;CMLtw0 ztAN`7GZp~<*lAIECqmcS1NRfO#2eVwl@@quCMyj}Z!EwU$N3SIFjG)5 z`SVTfQ4LZOW-(RD7LNW8_s#G5a{-Igh#i>4qh-)y2PSt|Ps8e2dPUmA>(^HYFQ)rU z^A>l0$i6G96&9lm8c;dZA;9vj)UNq_rWicbS|3LZM#`; z!BSe2pBw#tVX4BEP7mMQFzglvN@Et1+%)W8+F~;n^f~g+KE#j=zH$W-?0@3{Eh$^Q z$UoMqTgJ6@e6&Sam0{Ic@qRiJGd?%55(b4@8EaBpQhd_!psu7iwky)DT~s|oWGmz9 z1{glaULG)kc|~jMk7m@Hl*uSLuoH4u54v&BDd21yy;)}gDG6DET6|*RPw00DHBD7= zTASx5!wQ7Cc&{HN>G~g$DcgP}gH2wtliYGwLXG`DtYGF*<7Zp^1;)|^%FsJya;61B zcBGo&Sm|#lAMPoHVjW%TRwn8y;KXB<&bo9ft&;JRH^Z30XCn4t7k#*VSC*XI4o5^_0J^RWG-m})y}yBv5z*QTnFO| zE|BT5D~Yh)w5v!eOU#ik2pHYp9irN+SFca9|3sfH{Bw>o8J5%MwB7KXEG|K=uYGN( zuf4#w33L@(oYVjfs0-s(1~(XVtTRfggZuZ);n}N1xkF^pX^|R1##4K20&(uuq^Q)c zjbHSO)r&Lh6;p-8vLc48X$r4m6|Izr-J6u70oL_wzJYr0B|`rWcX+(C-t{P<9~Ktg8IDBk zYm>;mduCW|rBXc@rnTR@PB~^;HSgmv3W=aarJE70aer zY!<3lY4)%kI0TrG84zevXq41RO7s)9*{M9i4YjO=Q__`OUpk78rz}oVHVWOPHXZ9& zE0cN7`qih-Nn2mtXk}sSPgq(0%Y$V4fm-t3Y%{FE&p2jS=q3 z^~KvWw?v+pJx)ngmF3`menok4{Z;EVR~g1qQoB>Uuw>Pt`?q97(>t0Zl5mB_;lTVm zv~hLiPHGgw`JBg(+Np{%t~^_q{8o1Bz|Ek8ER+rtT=*_Z)b?8kht=D#A={sKxwebKj`pL#)ii^gxq#@pbZdS>h=$xNoTE; z>8{!8=~f<>@O&9p#GLy;!ZV|uIQ&GyuZfj4Kf2ldB?`7H(73{Hj%S#Hw3Q|Ns+&yT zKrn$U%_1JB1_KyiQMaHs=Mm0K)hAtfR=YxP<}#$Fa`23-sv`9XKAxQ8q-4xH^mRta zmmTLM<*=A0N+J0zVUtaEU}Qx^ts;sT zvMv24VjtDxbv$eToaZ+CL35dAcWzX{zB9;x-lUs8{HhP>+iZDR>V@Wnvo9xI^9d1U zF1jLS%}mI7OBhp8s=#z2QX6jje(xlSBbVtcBtf?k5!oURjZ1{XB)~H!iD`^pyj_z| z8_U_-X8vhtcr00wQkyd~@srF-RrlG>O`^!N#pqJjnM9j@%2Uqif+ zlAf;8A$2^Zh8#0ug;n3WJ??lt7j9I2Be%p;1iJ}tUPR0M z*Twi}eTn7{%)Hea07p7i-V4ARbdiG1ALC@Myb)d&WpC3RsI~hL42jIYdh3nUQU0)w zO|b2c;mX0u3Hq|a1ePt(h{kz2edVGSo~g2SqKS_&)^`C|DZ8vDY6Hw_v_)Z(!u zj&&Ug#~A0mye-DWU^<}%;3%p*dKVG{jch!ai*)e`fmy6iib33OglV3#htEzLSe3C` zyJ+Txz}j&mz{2=nY&>W11wnq%v^(?O>l6phKOB}8kt}=0kGB2^z>fQ^&Od(FUkH5j zE?eK5a$YPW!CW=t{5w=#Zr)+iW)h%IM0~6aj+VqX${~TCuAHH|0c$Jw*=M9KU`jsf z%NupzM)qJdn4r zm~WS)Dnh4@{9X;eyR5D858S$eVw#0DW(XX13LQReM8)0^4>;dgUOq;NvRT`ZW~>}~ zJgvB5NdTzqzKaoj%;$Fl4eC`AN6}Ur&Q{D~Hrs=gDON>I97s->S#_Nu*E{M@?iq9w!ug=%-ROqsP~wv94MAQY{#iH2=fB#%z9c`b zhNqX{3+EaXV_Rb0JvFJ#8Wv|LmFr`M4>v~f?0g^}MeE3fKLV<{y!hCnYcNpZ))f-8 zQfQ;BD6`Jx*Ww>Qk_BI}Ppxa@ z<%FKMmRiiRs?m)6)%`E05*vU|PFsUnuyy>7E~4!Mvfcm5z~qdZOv?Vyq5|XZpU;f5 zT8)b9+A3yJ(<><#@z}$jeK-CxiDCb1VoA*#2dLInezg72y_-bJyu;1zPM$g4{2Tw}TuavGEs!@S;&!UY4>Mm0% z(jeQN6MC}r{3})1h7~-eW<~9S9|LdhJ1ZbBLs;A=6%V+t5hp*U$dxb_=~zkZ8jARG zD>(H%>@2-5LHwz0JD(|zTIm(NJOhYPPh#9Z(YaXX46YS5)KaagZj~I1y)+Kt2?1NL ztvz+@<1$&{Jz05D^j*+yXq}s^m#haIh}15Wf9|7d$1MCQZJ@|i%qKJ_o^trsSfqjB z41cr|2S1c_UA|!iomZ)j<*?)pG85H`BoFEt?_PadwhNNGn&AcFHN!Gx?@@J?-o8z@c^^53{;m%;#Ta1N)%x+)Rg`BBAwe_DdWqIFM?0HLiKFBuN zXC_<)KCLVE7lL0r)l`Jyg5v06Ax!jdXb$DQ2kERWq|Elw(WOi`0(6uDOoOrjC7SwH z=G?hr7|5>Wn|JWFM~Y3kfcHl^9$%2p_^K?AuAEB7oH?T$V@lK}f{(b&*9P^ttIWC5 zMw%28rZrHp_?DLtX+<`U`QMj7)NSA+j#@zA#uts>k3OWiP3+(3>14H%~0?=IJ^&=f43vECIv;Gs9KTz(fzBUPPjsnd@&C~wduN20+7|z zmm|k*1uq=q^!q$p(Rz+Y7zK922*hjupOG0+^6L(BRhKM%vyo_=95+X}fwED+O@LEu z|0;*Ag;fk4GkiDMhEpW~8@)NvHT*Yo2`6X)4c){YN)|=XzzZ@>Kuj#{K$*|Vw^vI) zAJzGlp?}7QhrEkUaBF^`?PahM>BI<{zmhteQ(@p`OQ~jF)Q`*XXpN>^kxLl0#(64+ zY6*P(?kWayG7&zcq&nGO$@gyNPhm(KLrslS~; z7?0HFcc%E<9HcsL#QxLAi()8TKVT4HuV2s|pIiL;m!)#WelYfEPSRYF%`$^-J!SX< zlb$c;Z8=>EW0p+HrQ7pxTDo$YtDm@0cy`la;|!~CHlOoyNKYzqlv>QJ|n;Zk%Nq9 z?&tl(5D27U23<58`(*R0D~~0DbA@LTUDq!;M%oAAA3s*ZIqWnQNZ5aD=}M3<1X-CB zc{tIL?Dro>=`SC>Gtt4K<)75(WRCEw_Du*i6&2mF18DEi!<_qZ9PP`=K4VCZne(sa zmcmg_SH6R*wO5`?k(E%Ku~IyKUBbC^g`IRoTFDk;uZk#cSc*lZf^eYOa{U zvU-#Lrtrp@5fwG9y3kUfIW5?5Ms*{VD0t;CkfdYREkO5H0LPh7ZhpU|nON^(|a zEW+J~JuL&AwxcqY3E?pyOT6#hNW>WU5B)e&pU{ldp@K&9r~(v)|zH+2}qqB*-QS)@y&Tw$#eKY%hiuz?M zC5L|?+o9j1QWs}qQH^1?R1v~BvtEc7p?lj*e3oiKV=Nr~9&-`Qb!+mr3R{*};b(EN z>@LYS7Q17A?XOKke*!ulqg0$cpQ7Lg&fhX;@EQL+@3$6+ebvk?FL<%Q<5v1*-ZIlr zW){4zM%`Em3 z0chfcO-jf_HtE7ix3vQ^V( zGV1ycxLc^;W=-MF`=eN;UH2E=E|qNBX;zF4mNX4wDkg+{A|ykZfzR}f%SxVsj;sM; z=twd=y5got11)~!w(}cQ*P}j_55V?Z3}e6NXC9@C`duM=sN-)jHHLjbkQe)*Vh*d| zv4kkxufSX4%OQdSyw{q^EW9`RWA!~}y94Kjg0N!KNht)`qWvroc8eFYLni|1&T14( z-Tp!dG{FsNdp9oYl0+J0-cBE^anG|&i>vwQ7M)~tczL@!NNfCoe{KJpE^9vZV^PP+ zsfFUd@w9{SQ6thGdD(`luswfg_(`K&!(HqLq)dlh#!b{$Y=R^loM0Rd67)m-#(g#S*BuxyX=)J+6qCwd8(`@kEGxdO$b{WmX)-*rB9XLbTy1-ig3# z$%97D(?-BFWGs37Au~V*2CFB61mieb7n@}>H+x>d|&9U#;>Oj9!T9^!)QEw#7Of< z=c?-?*EL$c2d;XTEQ;?lKcR_av$#b+e=@qdecZ5Fj@}wtLLCRDwf#09OPRdqzqc0Q zQrJ%;h(_W0YGa|c2a}CCYf1YM!g<)t!(x@lcQfC^LK+@L1>T>sVC2z-=#9PPVNNv* zSJm(wh&h@wb8N}spO$*OFMoG3GF%EG&t(@aaV(&|IMY6f z{bs1~Q(x2)G+plCyY(t{cF(w&S%zI#1h z#O7%-DBIN%bXqcGVMmqvL8`58N|@zz`e|4DX0f86*N$)QNEWGE`iW=)Xm=|$#hI+# z7ewj@-CoAdzTGEmbDl(1R9%e^oD=irxqn(lUK!Y%V&WK>TwAvi;E|B@j=X?MwJsMi zYXx5Ka$??}wTa)3=vG~3QzMNArrK7yIgkAPVfRK@(s}!{1^rreh|c1Il=lR9$5AC~ znik8UeigL0_%y5lMsS^E9Zz$M!6oF21-uQsz{!H9e!Zx2bW?u!$0ijvN(HoxL>%IY=_1Fv@jf1nDB1Uni0usPx&RtB@(Pk?(Dxh zUI$<%zxdOaO8=L;!T8V*$`UT=iOb_dwLDo|AjWn6aWB+qIJg!$8VP?0;4G()+N+ti zH&8e`jW!A_@|7@msbgg`{l%n3E|#v0~(UmKiM;#*+^6Lv^9`AaZ+;xn(;yqGE!lDBz#prPZ>hEL8CX@T66Dm z&{2}{CPR3olW;z){Aq=6LD9(pXF#p56dkW}$+O~fXwaWmvN~zLLfmRcqu$^~5h>@m zf2`0sxCh1nMqvmO**-Kp$B)VWq~@BHf76#g^~;+!GCx+Q9CFXf+11LLs^wpIdEJM>6nW2?-7c>Fk-8%OG=4uI+td381Y(#_ZNAQ; zYO$#WmTNtVnI62 z@ZMbZlFM|NcdlChc=L5CVL06JyfXTTny_-hufX;8{u^A z;@sgn3!4>}coCZ!Of~E8D)mKFupE$;FL2Vg%>v~qJMHTS*+_f#mJB<2eXYImzor2v zQAzeG2PYBz&v4ZbZVfvQ0)+^A$0y-H8nB%~_1T*CXbHX4WUEaHC+L;w*I%*gBDGcT zd;;5-G=gR9=p^OmEA_`esm{)Egh@X!agE%RsvH5As%_1=cMVMaBNgnY>{W|6Y5y)m z$^`K7S2~u79k#CYUVpQQbDaF9DHz!E>Ml31^SFN$BQ|`0KCa+PM>2SFOm4TtRA%Eki7^ zB-3K0sQdjxxh(d?0|ABEuC}0xUSX^TFUb8)c_cH(XQw!Dfcv8XFRQC?fJ9epA<8q= z_U?xMI|Ad3eC-ia63R<yxtB-$zCA!M|b+H*V0>eP(_cex73ub|m!b z0e?*a?#s8XsT{E{Kro^0#YooMPF2;qgHP{)pNVfi-)8{aA>@0SK85M_NDvZ$2+t;J2C6%Hm{k%Hd^!xe-UhHQY- z<5yLFU6uvzly>Bpy=#$oRV>|q^WjxS*}*RWBf;l$CS?0~^hd|!Lk8tv$uCVN@v`rC zvSxa-&n&u6LwF86*V_` z-BntRgx+z&Z^-Kx18+>+JoR;XT(Q!6!lE=`T?xQG8nRIHbjs9}rfE2&<5%8dq~pJ$ zAKnk7!^%}K5wTA16NQBagpep6$_uj`;b<&7mpR#O2V!;K4JMS9nb6Lan1j4 z2(Yi|S5PJ3@Y43EpIvN_{eF8oo$h?{W+$i_&;RN2{Db>Q*5^z2OZq2<5f6r5@Szux zeeLPWmGT8I@|AeyFB~wc7d+{M!>cCed}@9!d1S^|j z8=rffc->VsU{&CR#znqx$dFy!gUnCkN1ph0a<2IbGRzU5`I52RyP|rYSn>5l zUPajYs5AqrmaKlfpL}3`0Xg*IslaKce_lF2@z;ZNce$<+@hgDq6PI|$`bR8oXU+cr zF}+`-w{pM1t5`kyAxS?=p#$3G6ldZl`?XLQ#A7DkIeNa1_ZDXBRZ+>D{Vxx9{5KX2 zNKB0T%vQEwjQTIB3~h*g$Z7hElDO<%*E-iTS7D+NSljJd`VZz9dBifa8jpf29P0bdAlPyu^FRmKNe)`d_Dr^fMy%k)sn6Vb?U9r<*3ts@k zK**+48v^OKKEwcUnxXw=rR+iYn=W*=r5DYeH?Kh4HH&y>b1J(2GzR75!$V`a)1q{v zvEq{c-)^GEhb%Q%#$p&K-3|Aju>goqCElaj@blyL20**yuOYjM(+?u{GU+H~`wKif zrxXXT9zyEyO-YV)Gc_H${#m)xqY5thF?C(K>H#fj6OJ;!eD_>Si+)eRc^0_|5wk|x zl9NkPpHN5a{LJF}Wm3Z`!xsdIGvq@`iXTLAGfh3r;lhuV9 zp?8I4tv(Nl58KW?OI^4=7c6KYaMqTvNrgYxz$=hX+)THc}$<{g-($y6S!*)c1wSbdBxTKf6(>bK~3+$ zyRe9Why)b{sllUyN)<5lL_om;L=OrmU8+I|od6+`CL&dXf>I)apoiXj6A1)q3B4wv zS4kj1fCS!n&b{xQx%0cTGvB|ylVm>G&+fDPZ0SgVoG*7Twz@3iqP-syT60?0?FZw+ z4Q0%2%SD^Rnwn?td`=~K&_JidePS+xT`M0BakF@Pr6ZBP9+`wgWX2i|^dO}2#!8=e zbp*ER!%~zaWwAdD#EYMW@lI{-yKNX=v=Ec!;}Ij#pSP?#-}WEXjR*eB>z(fZ>(?Mm zEEMINV7;I0=kWufA|@_!%o{QC&hq$blyTrDK3=>yG0JB8w(wX#dPDt`=4N}UrMJaE zU$1)|ePFap1`DkI5_}kad0%O2(GTn^T;RFX`omZ()`1oIp=2(upsI)BY8w=43we9> z-fTG04Q@0y3WS6esj-K}OMZ1~^8)tPXxt9en zj}u{AU5C#qE4hFUHoJ7lEIw178rrNc)r8mq3oG-d)2OYoEi zmbYqMvgvmyo;}LhE^zWPytqxMTiPu(+|f*!wRqL%{!1f_P_^%)x)vg~_KCFh>q2EQ zgW;-~8e(c7gAn%t#BDknOM6|V4sL5P(62EBYlQo)WlX+TG`J|)xy&vnI~hc-Zy5aT zIn7^GI(w$o(y2tMY_a6!iQKBbMMEE=#)ns8u;(+~7F+Y_v92|pc!O;DPWrI>#oCR< z0UIXf!fx~2{gFF!U^g$R`AzJiflDV3ByB8gVb(MRbUuEA+?Npu>%pp%`%t@M>q>X0 z)H4tp9-)ZT0)BkFObyWucGsE7JpQFs0NzQyYq%y`w0zLY24wDBsJS;+vnxXVJ;rGN$KNNAc~ID&(!o9`7oJu8ECbNiNChQ z;x3FED>V8jpLGgYA|ukL?=nA=DJS_1o7TmQz!)UQod@PJqA=Q&P&o2x>RLu|X98Yy z@{eUT;xF!YpBzaPPX4gtJh_E-*iTyvD_}c=3uzjmz>@iICi|jM!~(-4nid#N)@LX* z+W(MV`RC%y8%ShwTE*?vX!w$AlI=`|WP;s5_jZ>r!qM1&X-V*!fPcFs=O4*crTz^# zqi#tG)WUCFbWt&CmzmVNo@*8|M|tYoU99N|K)+f#FS9Dg10;2};U7q-$B({hjxOvj ztxendS#T1dN%&Ns9Z9wM)o3$1Ci%YNVmKgKJe1KcSsHh#QV&_mDMB?h|Zae6puVOdl??M&&n@PE~u@4=SphO z8yjVOkv$8>$>T=Z+>lszy(Cl5 zkQNTiX?P_z(FN5+7 zKWBJ9)@H7-VT{4JVORd;t;&ZYBi5{pbxO=bC&8k6bxFqXt7+Hh!S;Y5Xod=ZBghy< zi2YDq*(P(|tG@Ye-5D6>7Gz9)g_fuDKC5IqxpYsd zmHNhIG`nm%39EWvX8c6BWMDHFUyd?Duf%sPmG#D?G_!0db9nvA>yT~AJs)g7!wUD_ zRn6vD(>8pDv@UUL|3Tn9H0B>N>l#QDjG@*Ef(72~TnyhJqE7bbUZC8vY4p#9!1xsQ z@4^Ko`FH8(yPwSw%s@+>y_$i0u~1qkB)x8GU3epo3nFTvwe+%Qa5uYCY5_zDPnVrf0D)TME zBZ$6sQo-6Q!Kqm_+OY51-(at7LISCiPkVxa@1>cZ1R8mjzvvflU%X`- zqD~Mn*JgF)3rJ6GxOTRy)MCt4{p)WOv&HisCo_719TgmJ)_o$ipYy0A^s?2Dqy?k* z-A8hjR_;L!EuX}n&L~~V_Q@bTQVZ<5ZDkTL)LPcEQM5H-^m=NR*SS0Ve*CoK>jdWP z?d=NVGO%PDAr&G2MK;E!*ew>NQ~`^8Ya4j&KFGYpx?tPCuGj?8^5(vw!#w_GM(2z6 zt|(zeFW5H)hMhCe_5Ln-`iZ!W=2bJBpvZLP0RdAu?H&$sHVO;4hV@eFe9@W@1vzbT zh!5jg!B^q!lgB*~eU{!3`mAWqks%iu@I&v0y)mWntLbU@2jiF`<~`u=dv1}@{>Z+6 zPOe@ZV;i--kA*g|+XL%HGno>CYK{2Q_1YHnW^-)FH%q@W9;$xql6jgKS-&({q^*H#H0N@|9Vi%`ZfHcN(gsj9mjFdBg=PeQ8%4Y4=QM z5pnCWGg`m0XKt(zQrB(<&r{mqZCKyt%sE~Y@enVL2D@wHw4vs_ftOx5>D_$y8~fck zVlk#YNGpe1lW%h6K2b!qkX#l^RGK<>9x`5yN=|xg6WgA2Lqul1!hPtjAr|S@tR|9| ztl03QcVq0k`-dWRt)_qf%*3KU;C3m++e2f3y~31H5%ctjnSh#R%e5KiwS)uTgU)MF z!mCETdrJ>w+`WI+oi7si^!=nNy>3S~-le8^63$_#pG;Aa#Q~FOmuWDMfe<01>LHP>&9aeRI zr@Q5y42tUfwDA;@P7Dun5`<1++y^hCHD9`Q-LjN4@}vPsa1}-m-LEKklCtkbvKJ1! zswmxh&uu6fER9A?UsV*ZfA=H=7H8wMK_1{+Ov+GdiO!X?)7GP1*!*;I%gd>KI=X86 zhE19K59x(_W&TBojYzvrVdy8?Cogt`OgyEhbacJKkY&V!R)`0PkUvD3K9vY`i}HmP zNVuV+dR zSXj(4&09@zC5LrWLbc|}(&gnd=d-WxMrf^2LTR^D((d}?g1a@ic>FBIhV9(xOUg-uc)%zMBU~So1B!yunx!xHw z^T&aif9Kd9!HD-8d>!2=U+8S+kyVgL_tUf-I!pP8Q3Qjd51X>Xfl9 zH_I600XEI;$6klo4-W#f={2$zm5x*q0I^5pS*7fil(4qEMCHMbZrqpi~3!Ir! z--q3A`1eZwv9bTRQ17X-U$TiFQ|tfF+x|~aH52~T@CdaY)cD{3A^1Ni{{NQxGk;z( z|FYff`(J?jKRn5Crkwjyl(;fd<-mVB_CGy~ZvA=b{kIU6Nzsx2hahGKn!$GvGpkkd zTYngV|5;=S&Fm9nk`DLLz{cMzjlU^wP1|n0prdIERt(7D!l5>E{gBlp;AuC<<&>fJ zGtmnFDfWFbT0i!1_;Ambr({{!%lp1Lz_k*dQ}o;%8J6xV8+?wIq@UW^yZyFd!!cjy zeX=>w!%KY!x9327v(P}#IHUL^9-l0s^8q+U)4h+9*J|hV=)JDdOR@PU=68}}nuF8e zq-gKnX_Za){CIbGghDXg<>SsH^(rZMloa zjE?{pVg@}!Ij((7*fd~P^K$#O&FeL0W22L^UMJznF#h_J{Kb?E_YKiHlS5|!)u?zWW$%?13wb!y^ED6;#$#vR5s{BS#l&lXT|`yN03 zbRoc0vRf@e$thL&D*PQnKXFr3B!?+m-)OPF@M+sP<#3mMdZF&MAN^#TlHX&MyBghJ zc7b~LK4z3=q>^PObe8a&7U&86%@TFeHjK0#bg8__%aR#%M9q0QN{nD5z)TX)?X(rx zJ^gWU_|-;L;>lcw(8M{RxaazAtnbWO>srRyp6}iW9lH*dP2&6XFJ3T>r^!DC43TZj zA3w3rRp(qrjoLRY*z*WaV*)D|&Y~TB_bTp}e%4~mUad`DT-_@OP5#OO3W`gYEOSFt zb3^^#{87unZCBV3*zRz9D9;dqo@OB0$XZ?U?~y z4HpUPf}lqb``<82-yG!VM1v}p6AH}bRPJRuFm7*NeXh;b?KazzE5q1mdyjon6sZ-7 z?&0noG!AmSlB7_`{M$+^zAf3acMz)KEZS4CX1SE*b8;tbb+Dj3Lkmb+0uEK?S+1C; zn0lM&7&Y7hW(B_KQD7T?0Kem5yK7$EWg~o|>vH&D1!?cz7`t@Fx|l zqjMoO5yAIEw55B#mpaS|!gFL+-G2?RBCyY^1zm`>+(3DC9m0oR$>CbNH-k`-YY?Gu z`7U3l+-U@PGmQhA@xE7nqp&)m+ik~P>&eKjreZhB0xp=2xnPMeGPYTu`cM9$0;Pg& z+GjsC8N3Q^SFE3HHms$2^V=%rB7kBp1c+JHMyl`EUor6aUSAtF|Fbs#vMU;eX|^vG ze#+eo+SK{g`Z4BnXY+dnz|P~J^7%PdL?54nYqPZ<76vlGpD>up(N(tTKRAP>3)*MY!{Zh_k1vphZ|7v)>*)|_hKRSb!VU07VN{bSzOQ$#zewD)!OmeWmQIa5uH zf`+i(J%myzO>R@xuUPzKPBrM9`GN9W&3p4wlaP18^9h9dM-VS%kdvuQRbL9*2o4X zzp*?xk3>L4Fcq|@-?J;Y&|2hEI%I@Rsf2Q|#Y8cS;8Qq29G25V{O`T6B9p=CWJP_}~(y+Cf$aQx` zrW=dY3DGi1818tvGaEHu!O`^Cf7eIz4t$t;GxrF5ZDzTrEiYc$7_osDmk6wXL3$%N z;nPkt#xac3Wlfv5EvqSMZ(0h18iaPvc3U^zpq>@ATgWTO`UOsS51i3}?VfFLKy*%D z$N+m2wIeNeOSbfUTa2Enz>GR~Zo%USC!nz;%?$H2Hgx>sW~Q(H)sG(-~ zMBptA=ge&1nl=$4eOWR?85_eygNr(@ztoVeYV{r1tn0hTUscFot>{BGFqjWlU^X7C zG#wna+|PIq$~>}$l@?L`Y9aj`0Ux*muHh15!IXQI)Yk7- zR((Lva#(!tvaPc=Y=(3&jt)*SJ^W$I;8VOMHDkN)hF*J6XycmSctjJ^2cFpvIAk9( z)e`j!9>{&FWksq-I~uktuuLmfyOUq>=(M_ee!MfTWb>%>Yc1I@0+3c z;1?QqrFMy?0L0-8nwcOxqg2mp8Z^|BCA^k4*Yf*6wt;NMU!N3yOwX&BZTIHc1vLBM z`Z>$Sxz?s#wZr-1D_Sy4TUH)=i&7U-wZ2WAK}{qCa|ks|Uz5u}(VkXZTcQ22?-QR& zF#)TtV%@;NvdYlgR5SRNnE;Tk3rsq$C>?#*?fr!yd<&)bRKl1L^PZqp@gP^-koA{C zQ}i-ar@Za2vvG?CicH79ZLOW`USQ}DR_-DF=B7S)Rv!>#CS*cymA4g#(B$reoCM9m zV=5UNpSJ=IdPo{0LkX-B$TnBZ_d~qw9&q<9FtB*C3bj%NmOGXdJgpK4tx>YjC|>c9-N?akYFbU! zsb8sUTxqqeZ?n{{pB!(PZ0+6EfGl_xrx-07H{i=AvHNV@=Ex}8Jz%3x;B3kDGw(R! z=#PQ7%vXt+;1gIQ2_cAt9YjL7amdd4!6XpH5ICd= zG)k27^^j-6ic=qE|9uq^&yCu>L;z zxyqQ1-AZn!`zI1>lEexv>^{9t1+04(^YaYK)@y5vEhp?(CuosdLiaJ~4JcULEDvkH zJ=k=(b9m6#w7#3Mz8i*ZJQ!@+GjF1DF2TRPo#;GVWhzyHEoie-EdG< zf{_r2qMe|k))8`*Hd4{{l38*>hJQyCGoD1#l4RVj>@g$FSx?PT5}0ox8$afAVNokz zj~xDtYNp7be?S?$MGL{@-S(lUG8kAU2JM{~V;i!mhTi1-&4X;_(MFCYhm7+1oi~pw z@Px4XAuP(mArpH@4QcG%(dgLYM3V)Oj6MNYU+eu;yT5KaJGolqQN>6*`zTWQVJIAx zs^~newuIoIQGf>&9u$5zi>;j~Mg)`*5w6gGU%Rcn)0y?Z+*aQo?nG7B4kUpweaP1j zWbg4B;p5?dwaK}F54=`&bNUE+k8MB{#RGeugowR%5bT=GI}n7>1O;ODA-gh%mj=Sa zh4+-s|qOn3S?r6bL4B>E+`MnGj55YToN3sFs{|S{t4ru z-iE(nf3JP`*}%^|w#gtNb)L>12WesgyiG=vLcap}cK*FEF?c^>-dbk_<1?^f_#ePp z`qq1MvwY^+#*L<(or5WR9}gcX$oMf~(%t(XyWh37{(In&YgeQABqi)CUA_hbggjKK z+a2hh1!PZX(e2)B;r!ChEc7YIU5rAO7-}u4$X!<_$$Uo(y|Au5QYWVF>9!8s(?>7b zX@S<=JI1?QaXw#zuO0Rt?m{4PY3{CseGUkjM;zH6uxCPAYTi#cAV7l77gf}Ommg)TdedeAs`!ZDbgnfn*3xwE zGRjkq(a{~C@nCWL9CJ^~rhS^ey3JP$-*$%kyJA|8iD@|j48@f~FGFfXThS|URzdhG z0NXsKR^p@K#^^nmIWXX?f3WC_q-{%Sj7{*mje${f88R`2nxX`Bk_k93nMG_ zh4{Sc$wZDxwK`ZKWR!4TBE3ywxvdCN$C14sb1?FahUz8Q5mZ_VRrYgB=`8eCDhF z-?ad?5b>1H8eU(@*a;6y)#g_@4VxE`V!;jpxLmKRFl0=eMih6P2!o2+?mZZd@*1ty z;{=ET4kA6XY4d#U(=DPv<~anC$|@GvFBXO$$bcu<>Ug-_0-tqbm^vr|H3Y0?Wai5l ze5KL5aBZEjgFx&~Adg0_4Vg7X+7A{u(A)3WW~+dJPG57vNx(zr{p{V7JUGZA59>20 zV0YsQwS#PjlUIUuy4*mM>stRy7HIIZG4RoOjofD@hdz)UAAz1dHteuPB_;@N9s}lU zN4-$_Rb%P69mm+>|sO(=$|8h?5X z!4=NaU=;WdUsbK+lTBG|b)}Yyn{v<(v!Q&A(RXUpIL0;${(Rne8r!MOoJxl!mpAH^5g z$Y3|1K5(XNw*Z+fQ5(@aN2zy?0{J?|?k0fpIwg*dC=tuJ7MD$tIJ;IO0S{r;AJigX z{&cjCQwerPxIDRW3IF9hlzvEoFr#?(EAHMfV0YkK@BWOGTdcjD*htgw)f=@`3uP|t z1h5K8e$Al^FXQ<&>b=<9Bz2zu-7Gy2%10&9Q&IgCd8rKC*HppT^6SNSgchwP1$Hk$ zW*;oz!qwh*x*JWP`p}ZzXQ>XR0JA}jQ%g7ROxo_Hk)Ob33z@;sfx{E#BjT(PQltj6Y=~S})12Z%nJEJHr zC$@?EW@&;*f9G_@QWrlO3b>*opNCgx!FMI+=O4xe{ zS#Aj}3%~7f+-BxgQFvBd(^?g)Qn(S{l3z2xaSfZU2uIdFOf2%j+}j^~&W9B=u9A&l zMDHRT4kxG9CIM)MgF_Go&mlQeZck`nw&F@va~e|)xnw*Z3#St? z;}<(ATgEy+?5y0d2+X|V-tDHv=H6#D4_H_R8~J>O^W0ZCUnSAUB(fD(;Bm!6++$}= zzt^hm>gQw72l&hualT`evwwrOF06k)Y@wJ=Y2P)7lH~I9*dRPk{qD%A zUqmsN>`mBBNzfR{;LO+aYoFzJW^?nyu2PoI0#@;%s-&6E-mg}D(n;j=D_|F8FdDD8 zdzr_w2Slm){+T_6%E7cI*_8QD$s(DE=sk$z^?4msp1Xd*T~IDvS&+RfpcN3SPHb4q-gmu{%jugI z+mAK=fYKWl5YEZsY@xmC4w%nF$(^KQyLp~U-(Y!g>XMc|+7K^?EV{)2Qe|TRERUI_ zG&1ZA%wy36vL{`_4^w{AKxAp0niyu?HKY&BecLnK5V@rNBeUwZuM3Qt0;LfSJnhk?tf6MnQN6~Xg%`b;}xOl|(a>qqMf$l!2v74_DQ32PSRon+s z-IXt)sW2NBcguuPPz)4AN_eHe&8TfG%!o6#z9k$VeumWos%H%?tOkUHpZvfd?o{W) zbmH^?pWnf~sd_TZU5pUTBeWIy-`VHTRu&c13t}`2Fq=7{YRbooRer0znCQCp(mq%y zqmYb~natY}2?Ix(H~80r;Xb_{L*rUZzL4M%gCLsx_23+}M*Rma`NzlH#oy%`AG6GX zmXk%w2z<7yiESJ=L43F^X;jDQbkj(GL}k=eALhC*a=%qvb`?SlJU)kr>I zOLEoJ8{qlpO;5;D#hl@;pL#ps9*VQWv)wMZhArcnDQ&yo(or`~9zMJSv5Q@pUl~)M zJK^%bYHr&9O`fKhwD&xG;_CfIDEnycoY(8-Xm4cNm$t0)`9PnUzJ_!`?=P-CV`(ZD z*AXU?5@JL<)*?W3S;LnqeS(Kenm=@AKHst)^tSooe3XZGhsJnXsWE>oC`Y9l>{enZ z>FfTYLM5=pP=EDlx+b3wV(i$kuAXv`O2NdjYur5ToI}qZ-Z5AbnsfYYDOi*uM+7Bt zVczmz*D0I>bvY8uG-=H|E7VAZii-`ypgj(U5+!|%2S#r9Ya4ws=U9An?}a2($+tC2 z{o;c>L+#O+N`k=Toh;s7?ia-J*JcZS%^YVfpnGR!;bGEI`WJ)BZwim_*L@bAo4R-0 z^ib%cYmJGB{1#|@RDPG5W2*eT|D+>V-WpeZq%D;X)uQV{UD9uQ$7Srd)jE-=Zq*+^ zj{sUz;$9u-MM?a)?50ZH4F9rhedDnS0Bh4__co_@A@xi#rY8dOJ|oCoKN_olbInKI z`gl)3=2#)*;1_yVk!N&z^u5`Z{Ni*ft7GSTSnKiniEDjm31^%vYrHyZ>E(mh*pZ_G zg=(#l)J|ZGgeviR?WlXD>1qL{amIYZ*tO(z?V&7su1Ne&K=Xr%@hdEgx;0co2PKvWyDtm>=roLEn@n(0hJBCobTM^!d?G6~=z70e z!msaq$L@WPBEh9h=|sU#sLvSKup#X<&-%?F^%U&Pz_6fA-H)X{9CwyKj~IIUr!5U) zSFTIaJS&%vB${qL!+yF^Nl~-;Dq%ppUzPVRGNXK)qThI1Z1nxca~Z_Qk&@iyt@L{6 zZSm>V;v3W?QCD$|G-n@F|4k7~ABWES7!Ubrt@h}k%5@kd7~VCT{1bWnJP+f&WdM2Z zrv29^>$*}JVUb8w#i33ZQS%`IiZ=ANQiizx||kC=Ym^E@A^ z`Hw@!K;v*!S&o#9WChSh?c7s=7my~CUI4owQgGpcI@5ml{}8Zjs_eCwNhAEZwSb`d zZBp6Rc5hBDS7Ub(zr4)z5_i*!vyKh;xiooS_fqv+@k3&_;)UbkNS}H~D(=bT`hG~$ z`psWhHoe4av#enobJK+MOhSO}&a@Oa5a#ETA13$xg|(=2cb1cquit;G?8PHAwdbn? z(1GVstEyGr2F5s+7V8=gJGY!u_pS{k-+H1Ht&W#*;ZdmQR;&x&Kh}=xh&zhGrk;_0 zG&hxV$YrDh23w`Wkp7$P>hk-TE9?s z%=H*nq7lCNC!3`Rg74vAqRN)V!c;;L$X$pRymB=WXj^-Mr|vJs@t?`{o58`QnS}W) zFS7ZJ7!OW(H)I9>p-(!m@;Ifq1@J^%ls@xd)e@L9gR@e0;|nR1snk&`vKt7F03^6~ zPNxF00u%tCpgBr;Nv=TC+rE^uGwek}sIu#H*-?PTNce53eLvZ6FFhEY%|E^ZL5dc| zrQZWz3xanS<5-JOKKoP`F(+2R$wKArM1>!abK!Xj;*vuL?!uC(7iH1*X?(a6lzC6G z#~e%^Dhzlq87d6d{=CHBFMuMk4`*WH znNq0-75D?9B8$k`^N)ZJhL#Pjt!OI}E5mu0iYMCcn_9-46U$;y`2hSAi=NH1#bcQo zVhz+jX`6dsUSnZS*hNDf zKssuCyjOZ`r(OID*WW$jv9}x5=7l_6Z$s0gBm4bB0=Y@k-iztVP?_bQL$CY~_9d&O zMbxgS+nGm4&SLy6bipy5SFv10{-em0J2|mJ`-R0VWtl?Ml3kV1Ksk3A9R}mmDS4B@ zZng3$fThtGGUXJJhxMZD?rW56dp~Z{`Myr=BBu402{*jw(`rQ&z~~;rkf$Da?8Su@ z&n!*)gx*URA7S~e!pmds6-NXNjYtg3pI;QO$jE|0ZW~+S(WYb1(oJl4YKJ8}_p;)d zW24f4f4plA{AjVHXhSV07VL zhc1|i58;ZTR-lZJx9?nbX6ddmsPc#9N~`3n zjAymQHCd?utcbVoU8s?A1bC7d^);lM7khVoq}8cn~i6`2V|_J4EN10Y@+Ao8Se53uO-gS;e%{TwP0$$0jl1+g!4pYR8A|ziO`w6 z?|#AxrZ)ux(95Hs-%Q*c!fkwh%e3*IqBiOp56j(t1Bj z=2!Qr#}2R1w)-); zQl#*!9C*Zx(}kaJO>7p6;(ZiSI-YT?cmE|rvaDca!8mX=TE~tjJM0*IP+4Ca$@Fp{ z5^Q>qsHiF39}~@O?20CpDV^E!JE~rZx1T;BsZl__ggYOmpTsw|eZ#k>+WL(-&A3i` z&q!Y=YabUq+V_k)=fr6cx9%s z$e}J(_a^#Y7lrF(`VT;C>9=itU)rlJwS>Ksf2*clO}P`y2VC1bwnn6V5fpPNUQIZ7 zmpRfRTLWW=31a|HJ!GlRVb1S0Vhg(g4!gCZ<{p>74+~F;3XMeGRt+r2#8nqe7dRjO zHUs|HGJ7QDw(*AMEql-QGt}JI_2P^YhY{@bilP$kv`E&{gT#~W#{d%Miu?h@&|%#u zelBq+{ilcY+q;&;i|1z|PRtlC?aoj+ zL3s1nm$|^3jboZ9Z$TN5enU3dB_a@3pB8YrX zS_u3HCX_86&58;^8bED2bIS4oRcJsOB&aUwI@7OvR9ryB2xr}UIhp_HQ)wq=dVPC% z&w@djv0!=_?Ze-e?kD59wzH;M3{)j9xx$D4q70hFAcO2&T2g1X5*p7XjYqfvSq)36 zjZ2}DpAZBNh7dnN4b}Jirg2R#f;{@owZB&Ch=Y5d$Oz5~t_XXC$K;a z*pHdsB!AE8ehrQ_8E2Nq@MQ*e!7dh^1ysS@XXwpGe>g$ zvzg=!?S|8T7k1+~bo#$4Ir;Dp%G{eRedxFUEohiLeq=*N{uJhWWqOR3(aomoCI&N_ zH68Q*X99ILB&+i2#v9};9b^L7S4k!!H}*G~H%|Dua%)x{YT|luC#vuCh^XZ@gVwkl z&f{R;J2c+Rt>V;Xm>M$nSsWGB;yUi>xXFVn+KRjWg$3Y^m_1Xcdqmi}F~&v*zz@4q z4sXB<#Ub84ChZxEVF908Nm|F`5y4B95nij@$9LVbx=&?{nyM3$1uARjt57@7hTMd) zUyO0?euz(Mc*7yOKx6rCG+5mKYw#AkskD2Yp>cyemq5P#Qp2hXRc2p#r8?8jsJCQ^ zoZMZ<;s%ebTDTXI6(yR@;xkHcwpA!fOrNrXgsPU&0$&%4(sJFbnW__af!>U$KX$Xd z)-1!QqODpc#{V;H@(O&-LHx@v9w>wyAbw2^%bwR2%7b0YvG)G-tNd3`hrOHzhNH23 zORCoK6minMf3~9kzXWRDACrF}7x&yN)(RJyjq+n=O4`_?nSxZSg2~ajV5#pBC zHMX#GtYgyIN{kL5xO5RH1QUtur?uTbO zNat3by2TcKxk1ww=<^OU|v@* zXax827I1qF8;8OjVhaEKAQ<9j!^@FDT57-#2*ia4yQW)RFK_TcV`ZI(%6 zmir@T&qwLmF;RMw`GwJ2)w>Vb$Bx} z_EJXu$T5d--NwZ>k};QB-ozy(9Kn2;|5B=V=tM@F1ex4CgU}I>Tr}z|AK%&|(LBO6bWG~xT$8nBx zvx{Wrr@O%v_zMfLnq&8t-dKw zyvG%qJXqz(@cJ1lR~+vg;4s#0<-6X@gN!tsKTmcic}x9B)sWu<%7cL>MgJ>HrclJZ zb7`nyvVbaB>o^|RUMKdBnW>2gyPhCvdvV~Equ=7vkua#2e!<=}C-I={-#TMYgFgjD zi8Gft7aU96S_XIQHMC=%h}rcpN5Z-*HfnB$wG*C7c@=}6u|BDSm#vd4$gqo_HvJOG z{Ke~YJ}GU|7sa&19GWOnvnl&I6LAqDMvl_&g}U%iFyEb+C&TSCrxjeL;DAJ1sVaP! z!lwxGWv#TSd#RV)<*07YsMclr$1#SGAKwaX>FYuU^>0>qyu8PZKH6#CxtXhWbaRJC zj{Dtu-)pZ@`E?)eQX&+S2rpUu0(kb$eOw^Up|fk1(uKFK|Kj1YVmpc&_jZ3 zLn?c>F%sv3#OZST@Q%M$v#RfV*5o;AyoG-o$735C)uukW&=Q%-g(^0xujEje^fkr# zIWe^XA4V<&A#43#kD6~k>Jq}pT4!1J<$yOAtF#6v`Z8hTXA#}uA_#ysJ`+N0%Z`z( zXMWUF-O<&1I!5z(Za0>tHVY6ga1hOlQwvl?n*o=dB51RbXDs}+o_`V;0onF_I$p2{ zZR%%)uX7zC!I9hhAPTi&v7#`>V2kxC=0xCe#r!^!?vbs*Aj^LN8urj9$lLlwys|!_9ediC4;97rjWVB|G zlw}g~O4BL?XYuW8JnArafw20a-;FwcWWu;+wQEp$vS2`)+NVP7G<$Z(;@bvtTy_w< zYx)s{RQ}28-Q+Q&rjVWVldhk9-LExpRq-Bg=RSkobte8LBEfTlI-L+-4(Bwk-Q0h{ z7Z~I>u7Af2Zu;KRqW3W0>qrm+K?@I%IQICrKUavMV&IDHo}-Jvc{DK%?uzSbL*R5=?nz|&<%6F^ECR=*S0r6^&g?}?pU-wMsWthr5q$0 zg_PAGZ@4UK&CZv_wt5_Cxjn2NV9_vj>UCx6cWQpwyK@<>#s|5FpKmO@sAuy5iiiD3 z?q0;aV)@$%mCr?S)QMZ2(D3M;2ogKI{v+Qx8rwJ7T|yd)@pwPQo<2XK_~Vki^t2+b zlC~`UqoLtD1)>WlhQnMmAI9imB3eT`+omc%dJc4p9ZSpOQUiNxOgc9A8XQp^wtk;` zg|180?NxAv5u#F`AHlI-A?DqE6QSS}L$0K-( z{3f$107RSpY{-%-61ky;nk(1vAy@)4p#L;G^ybpbdPZD{l7`0)yqYI z`o8rhJ>SHaLc#D@ixDGBZ?2~Ge@6RHPTfrW^$gG5%6+`B7l<_N#4XnJMw6HI)ftYM zPPg7qe{Mn$Xv+PVEsuB{uUGG^KyIt-Obwj1l+bnH_KGWeyXN{awNGWni(O>u9kjaV z37f71tB+1QlJH2_#yex3V7Gi|Y8pM^E5%>%9wboKhJ2s7D3u{p$1*&&Mn?pDXupub zcA9kjtkTTBl#qG^CYLoXe?(ItK^*XOv@f+`G*w~`DEVpm7;{7v1^P_C7axXH?ow_% zr|5ubIN>UqOgA>EQg9}@@}S=c*L_BFoUECc@3bx3bo51#KdF4eWq^1WYjX1A6~uf9 zzAwXsNiW_ONpPu-YM_=keF(Sf64Qx%C{OWsy`SUb)X&fGKoAg--U%)A9*Xo9AXF(y2#`?Uct7{^FTC%UclT4y`R$&~?wo68&Y8JJ z?a)y?^W@IE!vTn51a-yZs(BBqh|>hh^DcR#$GcNyCM%|-U43PMh(Rqqi1KS5n(I+yxN993$5#z7=Krm9We`wt`8 z#3g#HVy3O)^Uv|pYBE7Gne93teSsQQCZlclP#^{1s)b9!HO_pPg9uvd-omlIyphN3 zk8fnxxe2*htYFJ9d9>z)ou_N>SD&)Iux`&Hi`pbjd!B4>@D8`K<4w_yzCzzt_@^MeP zGhv;K;3Kz;GX9{Z%_Wd8D&-C%S|l~RgR^%qUac?7M4pZ*J(c)lUs=1cHH@=SMvS~& z`$|<>=ug~EImcPdnjd@uZ?q&1gi+OMRmu5xbHLT543ZQM?|e_^vAUN9w(_3&>4dp2 zj_8gP2<^0z()&{hsD0UZ!^U>zN$qweIE01%P@;dN8mFwmL$d{nre|xgYM<4*1(09( z$S2hmO#gM|ow%gE5Rmjdn!D(9KVJe7Z6j*gBPPn@t^bzxi z!ADYXgH>l=5hu~I6ZWyHZe@M=$x8i)cPkDt-C&LraP@K#hyO$*nuF19%4fgQ6@TR{ zHbl%lwY}zPT~~siWqX|ngvf2(b;Zo;bWJgO4Ep=vrl~oJAVo~#Ar?BJ)r*R?wRvQBINmcRja{(e$NM2?xA_%V}GXZ$RU11 zEov~2>v6wK*izzrAbH)Vuff9K{FGhM1%*(xX~Wv6-3g;8x0K3Hf827ohwr{E4dkIZ%VOgrV_p1&aMr0WfG}e@GnnsCoC4Wpqyy(W@kNTtH*0IMm#+80Xm#KXThMpfm{TQtC_%ps^~)@pYur zR}T87TD`)=l*YV*;^+YMfZHX5{(q7A!`NKyGp~=L0G3RxR5B-H)>GbZ8^3)$$qPZ) zMMDC8CgGlqJF^4fj&`?X&Zq6v-DuH3fM#4{pg4HTw!$InrLin)R(`gqCy(ZF)z$5m zcO7wa+vzS$`9lm>Y|p8RE*ul8n^YbmD!#+P_m6w&7ULeh7nhNR5}3_+N@_$@%FWkL zHl$4)$Xmk>QgGOVX+l~ALo|%d+vkR`7*T;aYP3;;53s`b|)GHl|;sA%HT#Br*#^h>d0I?(d8(x zqD_V#_Y<9T%;5TqQ=ECW`$;H=<3XBqO4n(nW@ClLKD}JV#dW#0e&^R65vSzUAKEEg z|7dXaJ;X|7w6>+zKuejB2dIoII zTg*4S8CE>Vs<6^uGE>RS?BUH^*)V0BPUY|S$YiK)B1Dg=G=3Su?lfLWpm$tlfD2$9 zcXQk!wo8wJORO$kg}BE&0D1gzALTAzXieEG0L8*K!Dfq{B#g_FurZW7b&d+-IRyQn z3|KPUaddDpUGZp@q*|F1yxBr5)*{ThU(!Fn4-0#&_+stluom}eDGhe+>4%~|TdiK5 z7E|Swo0?)@NcWv`YXi3rIRoo3Zi0(8&0)drKI6E~5>Z6bEz@J;S?T>QKKU$mPZ5Kr zy7MonJn1_d00 zA;-zKTax?n^$Ji?Im3cdVO?18baPs!N7V?3ehs++Pxz4XKZ#?;x&Pn{{agC6ON7%y znIldc21paZBk*I(_{jGggAVLG1DJHjErA*;34V}{J;!4g4?jx|g%1)A%D$d5=QJ0o z1mml!fEJHs8(RB=zr{7K zvl`?(_j%LlXRTV(;DzTu_zQM!VzvyaQ#pfS`0~o&YZsJ% z5meyNSAFG?Urecn8&{Dca8b0*!0VUEzYNEn^Z<8S!HT?;jh=!^2fNMm!W7p8n&RFS zefA-Y%-Dz=i<&qq$x`gC;)-P>x`Zem*X%W^N^oA1A?*>!i5QPvht3OPB$>ld`N+hf znqG-BAyy(f^osZxY*N$gJr8?#*Ijx1eHRZ*n0Eec=7Gd_eKfj(HH>f{twx{1y;n8= zjojAxPh{$K0(@Z;&Y;)oH+y=kO)bnlW?uAk4{LE~C9HUZzD*_YpFMNHf7{&J zi_tUe4Kb~l(>US7aq~xc9ctFUR}4rgG0m#q-MEAF#+Jw&Gt3=fP4)MfupBaQU= zK*|;Nw1Vc9zwTGEsC-XV!@C*n+0-0A(QO@T3oI!BGo1>eEZycc7E|%JVmBY0BT0{4 zb|%)`<7(etPSa~}0G~VjE`Mr&nie{bgl%n)GE%}3`27OpUcI=*$j zMl{gz6a;C|2}jL1TNnLy2plyU!|IXZl8Sv+VfFeX(Njz7k*lD26du07?gOFctnBYw z(@(drVOX0Hd6&zzTScPr9%Kc=Y!B}0Oc1qZeEPK;({!b}-78d!ptx%sr8-y~BsnP z^HTejYhe&2qu5WqdQGQ~PhkhFgF4na)ED_E)t^NzuCZLw=oR&nG{w&KWVe6><4P^@ z0NSR4k6P6Dcpb2UaXSDi@uFSWWWGwHaWtYRmVseTe#8Hb6eTob(_cqVnbt(kdtQNi z93BOu7)qp!t+3c6N2`tjvgOXvF%T#)($c!}JXyk!zX_AIiZSqXjb$W8G7UPevXZk| znDQ%@4ZOA3P2c77BOXs=b`R6&H6FTqrdVB*s;d?rS1V!_2rN2flJr|&MuX4*FF0JBnjZ!Y#P<4aU)Pl(K6UFsc^pi3^nRc_yfw(z z-XX3^Yd#?wW!k*1KaT+f;lEcJtz~1T$cah+PSJl!2yGsAR6R;VQYPJ0{?pb)8TiWp zW7)K-ck|cRQyq9R(io%+$IA7N-h7Y;Dp)HdDSo1BDpqzlEUZ~!i7Sli+xU&Q`i84- zpz-EYSiR~suCgjDmrq+U@U?N%exi8O!5j$j`V}46zV~wABI&THZtti_Zk(}b4u;ay z&P>YPk$KB9r_<5Pm+4e;Dk`7v{C?~OO7J&LfoXHOa;%Pa!1X~L`ClyBhDrSGd#TM~ zGR$Eh{?qkq4(Zs}Ui2t;k@cH zLN|L9ojGvm_rGoVg3;E;k~YycS<}0Mr|Z%Z;FZEg(D6tTV@ZW2?n?x>Jj&3Uj>j0R z+jN?EbZe;G%^Ihu8+XX?Ka!|j{11Mgqc`suIo{Sz-Q2d3O?|xfTL!gR<({9yiT`nX zNPr*C={{n>)jZDsH~3w$x;CkG{+>}|Z0E*rrdN$v6Ov~Au+e_EkFgFj*XaaGFHuXz zMs^0fu0zu4?j`;4;8Awr0*v@wGZ-=5)bx)`(e_NRJVaIH9amSlqB@b`{*H8=QnDD6 z{$B`?>1+^VjE*$Mcylbvi{I>Tj*hQL!7_TxlH_1n(c!S1@hD5g!7pz=G2m8duVsFb z*@}4ww_gLf(yU$sek*pf&<%>+52jsbE!MGKCuX&!tyVuy`vMyHk3v}enn}9tMUI`x zd_kGAz5S*M-?g^IVLSh5{iA{k$3S625;q(BiNm}CsW*j_lxhIp9KsV`H8o6|t}?Ch zt;Vj${tLTlUa|iVUG~6Q4(M&szGj5}e0jyzgfw@b%ZIP#1@3To0gZ7uf6Xk-x$Tl7e)Rvjfd55 zY5`8tD|HHW|DAKuKDvkt0HrV5*x zkSvJ>eQhQIJuiwHrA;#48;Ky{%T4|N6VK&LCRpLRPrO(^Kicj0dY!RSzs_tuN9iA1 z+px}%ez8j~-%+-+=lj?da^!#Na&go+adZq&?{@sKA7o1Afg}m|o9-em3QFSgYR|Lu zD7!tt{h28zH5Y|{EjNoV*2h~#;i!uB)$?%63l}kt0PPuMNOvhhPIDs$vYK+smk|}| z9(kU3to}EX@ZdOQYi~*BgV0FJ8%6thSo5=s*JldyprL;qpkn{glnt|AEfMP-_5V74 zcxoTtJkvNh!e&VS>&U9wRX58CDm>@TWLp2%0h>_R)F23tFSffc8P3Q#HqJLI&`da>wOhgC@~AznGD=nN?Ab4YF_78R6F_BpwU?R z9mkqFcmcgoRIQyvJD<6nWmSeaIBtlFtos!C)9pC8N?kZztUIFFIsE7bWEN)6kBC9h zP9|Z99hZ|9cS+Y6dIjn=&b^(BnhXDb?I+>RLA~C%JIK=FEdSSClhDCQPUiS)1+sZk*Pg@gS(Wi2VI#n3wr z;)0w$UOXnh|Jtch1v35=AtKfUrPIk@m;9Nc1n0@K_o(JV;e4;W(8#ss%;~~=h}yY4 z&)*z{TR106gumk!bB2`1$=@f-TBotd`t{eja?<3o2vES)W%sJmLPU5_j103``wHj4 z+C^QF^}2_@*8If9#DxrpwB(xY^kYdchBZzXPuOHRL1t$1kJI_=IcmC9G(FvVa%xH@ z)xELyY!9dCw4;BV;bfvy`(G&>ShM->rT>w{^cpXLfT5a_LVxAj77W6NcPZ6vLaYGT z&2Tewxx;zY)1f(hfn%}H=B;cCC}RdEPK>d zi#J-@Tjf}Iu{Qzr*o|>nt!-QEmugH}C?;*p{a+kd827iUqjT|j-LvOs^3NZV7X&YB zTAvrS|6@pK_K|gc$vDvxd%EUwzQ%&>Lqq@3U_@P9Egy|OWY49Pnt_B&5Wcu;fRAUe zT8jiN#znl~qT5$9C@ox%MgxS=LRqP|EPGI^%B>c)+^-FUT)+S(TIqGq*6X(_n=ESd zH;gd9<)`|MjI$pCEzY;S?4$&L`udAT%{ zTqdkpqG6P%TT{ap$LAn6N;%W}QU<*FP$4wZY=^0xg2!dk>ZvmLcytrO#%Zna*XMN{ zGxz2bxi+0x2&1gk7sXr5v`> zKxEm+3?9i%WLHP)WK{j=6$b_1UQdR4yyp$fzzIs2J`hmI-!BYeg4gizW~rzGci|5is?dh=Sho}dQJ2>N5XkNeK`ld zJiUto-ZQ3tnCL9L2{ck-awOvSzcWg#>nJqCq&TQjpzBtxr7HS06negALdBW^ZeNYq zn)`b031;p{leGY`H<*Rp?om5sX|EB_eb}+*K!}Q5eFC3BqDGjdJAT9GSCL@S6&;yV zkLwImK0j%xA*()QlE%^AtfBr%hB}{M68dbV84d5xjj|!=vBi$4#cj`2Z>_VFVn^nz z{YCwX=_Q0(ZO67{{NGHlnsnb+yo%DNTC9s`F9}OQC7&&NuH0+rw9CFyipw41W#z#9 zq{{%wS3FmGfcjQPRS5MQ9?YY(kplg4ie;Ubvi zO>zabSw^?UH0*2g;;;v;MGPgf6blc|&4<%>LU`k1CBFGw9o<4Nu?09;E8>dj$FsiJ z&Mtpy2521+M7m*ktIHsT&4_~E60G?5YR#`q+3D@ePL(B)Kqjag^gRD&{Qt%Rw3AtN zq7I9A^%{bu_Ll~ZeU{W(dcU5UD~x}N(sjC~?yR6l1K89#OCuaErgM@KH}8DZfS8M%yVh*tj}y};N> znNt(*=8KxZp}!Glu-!uSawO)%}%4vrzQM-$X<-pH|z235%C|#Ia^)q1Z(i%)>*MJ#b zR*2SN7U}mC&v9|ICZ5;nJ%{BC@wnH3Q~ri(Ww%~RTA*)HKUzuy>9tPlm9Pd9F!FWkyK3MstzRlERF6FRP37Lf&Tb_UiZ5 zX|7C0Crg#t5O`s=?wxwy9)bepW&mc^yUn@LxxWF*{)(>E#T##G^GH`ODB2}o zudetL)AP3D7~m^;nSII#=;5vm8@e9JUlfRa>Mx#Ynd#R-8CXXOZuE#WmG#qGGkIb2 zK`qc{+8R@Pu4zMDHUTs5nCD9My*4=4h$Zv{_C8po2QGdgv$Tq}#6`{XN2DubHsj<^Fhy`(r#r`juTG5YcfT3e}S?%2gzZ}=7z*%E~$ z+igF>QL~fdRj82Z2;ryz5{KEL@dY{aakcZI$o8O_2B{310+qnGPvJOo+sn`nMaM|+JSi33e_756utHxD*pk| z?%$@^P?p~d_~crFs+@XwLLb%h#t|I17FQRf4#5ZFFvyKrj4~zUjB_kt!($-tr993{ z&A03lvcTJ`f!%To<+8^cBx{Q3qvgsoN@yI?(4xX~!?2PjT1-)+b)=CrWJ`+f6UC+;q3}^~{0*}zS=G83Q9K85dV3p^aYc zUZa$hD@>;X9d} z^c|m=fu%(bbkM3?37!hKSalkY7QZVn;$Q z{{@oYC#`1Zg!QD8GQs;-?RG;8f&+?-=HGjM8_Tdch;e~h(%4)!rl>%#nVkDCo-ABB z+|p|~RyDsxHF?`8sGBh(J2Lr_!OO!&hL%Zh!WjL$cEFZSkxtg}pCZKEs!(^-lL2a3 z!^01@Oum!Nht+xoa`T(IF8;H#$znC{lnaL(OKYB9z_Ly;TU7-LJ6pTnEhjhL@f;j| zXAH25-+ZChF#0;hbAj~8q?&CC zu47BnR(5QecM`F1pi91kJ2RVu17ZX&NQBca^qU@5s2u*A>qR=5_(cAix*aDMgeG?s z+|eb6E^d1l6|n2BVKlKz{F{b?u8ni;x<-t9H}Avqz{9qsGZO3zOD5~GlLlyQ&tE#l zLw*yLLZT-|i^mfd4%lgqAkW?TWf6|C3p!ir_ALt*#MBn}s2(*6Net5hH}=@wyC4)O z26$%fXV?~+c|wu2rmhC^HWAe5XCIK;mRt1;I$1|RIzIs(X}DT?@w zm#)zcO``%2$KRH?C5>yC;uEfTeME6%Pt#XU=Jx`NicAlFML9eA)%Tk?A7ld*>_*~@ z$3xyUjc9|LNS}`;8`mC3nc)jh7B9}D0?m6;GdAl?z5o#GdjCF0%6SBIv(;wnmD$N* zZA*amhpunR$nWfM2!KfAo9RicUq4 zHB3w=ictCRIrtc@+)NfYul~?dV$OWRhz00)vaaiKaw==7BM1Tui|L>~ZcVYt3M*`h zl$*UEgfNvba@KtMcv&Z`@NyY4MyXf{$Vv(|gH4B~wf5fNxMT9yaB?QN*ikC>OQ6Kg zZ{x#?6ccK91ah8++DVpmKw*UqrBCfv>Nj2_NxdewbYYfU*{d(Lt<57=Ud=G&%xqM| zQ;6`JS3leciMSZB1-r9FH!t5VmA<#FECh3>w4$YPRDzH^`Xv`O+gzr*KZZGJK5NC8 zO{+D)oV;{mT@I;TEF8c0p#xidwpC+`&8r^3z%b(!8lE(Ik8jU2x2-@%GgsKBrR1Dc zYN1ul=Le<2v=rNdHu|@)juuwetrMJsCq8n=9uz(t-@G$4h>f@#;z;C{ce)lmjF5XJ0I4wT;E;s!Hc#=cB>O~jIMr&LfQzu|y#Y^?CAP%83$vs;(T{L7zl?Pi+q0}eb4nO;jZ-HT4&mhY{+T3q(Wm}{rLcag#>2v5`}LVxuia_t zWXTef@LUu5tygh|7V1?id-G4LU{zSOow933(m%M?NCy@XIWwnRdo@hm46vV+A+)p` zJ04>rGvL&?Qt$n~HgWlFm!JM`{Gg+JTw4oaL z0Q2M44ax||yWfNL9zA?4ouG$yW|=Dadg}FNf>NLI&r#D}KtFoty&hubW*ri#6B`)f zJ`iOlamBxkIzP04OGK4oU3U!_bqb7KU3u00F8i~0l|sL`{)BxRG<*6^)t?`{GRsB3 z`;<^}et0Z@87NmwkJ$SJQihce+ut?dCK9DSf20KlgzD@PA{e^SRmbW=VqJ^-4Rc8k zn4l>XLMN=6Al^(X{n{|-X-}7pd4ds`8?R=7v1UZM=7gEm1hW(awYG%LhFLwoGEP1d zlvlg9L*0P=jOxj9w0a)5@f?X%D)sjI;=P&+-=>zOFdw}v#cD6H(Pe|t7FxARvJv)Z zhpnqBU(=RhAAKdX=aKL&n>G5BtYeH8#>prS^jeqhmeFOtnR)QrRV;iv05rHI--ZA_Ao zR_H3fnSs|z&&L86Xpy^@IWMJ#-!Bltee0EfKIO@jR(Gyn#NKlu`PF}@s|SIXWp!SY zbEIBpwy0!DZxG9S;M zI8Kz;K~g;r?z7tOX0mlAyb6B19EPW_m=5{v8gy~8SSsWiLMza?vHa)LVJOJY!wVMX zh|49*s@s=$%(r{8wc-bVP$)Jh1-|xiM%l%;)4-V(u&-IhYpY}hT%zA@JoGLXOx@@Y zLisOL_9&^A_2qd<_Bv0BHoTUGs`LSfp|;iwo64$A!Kaoc018`Ah#s>o9;SOdqygU;_)d2xv^fy2%Nd}*5T|voPd;9y?7@DQJPQxta z;PQ=Z`o|w858n&Ve2lqaJ(D`6hp0`xdpLr8S^>o)K~f}m#?AC5#OBW|7RG7?u}MhQ zVO4=s$IpL8CP&s|eMj$#)@8n?zk3_Mg>SZxbngAL0^8!aI%&)Q{vab=g^%&???yPw zmcJgO=^rePTWR_Mhi7Bq)OYi!`7*lvDRu4T)Gw#GG4UudHeRy%WFX?Mj->eotnEYO z@-#pB%*`pUfuvn{t)@edp|Y+BW@E*o$kV-gsd&+J0g7=MekJTSlGRGzEElG&Rq zzf=(BE%MsUil1+x5Bru`D6?;6&%oC$Ohbszx91+$!KL+4oNhSKP!bEAbyhy0F>zm z@TyOUN>t|Nmt?4vF$w2zB*rO$iIE>wzE&c~y}@6^5XxJ*ZY3rn+9OKwaV1|)Aps!I z(W?^CZw7 zBl}Ku6!6IE$b8$n%^R^yIzR$-471!+hQRc~-gB&|+!{T;X4H2;ZRK@J!=H-Ey*E!i zgN3A-%qcCa;NccL_%Ly{n$MO;#fz#vNA!@*<(lUjaUM0|onBbz>ozYT55We$c9-b+ zKF(?~ZMhO&{B&w=NKzg~B;;}!d9-232HpC-c00N5$c$dU_$Xtm@0(0A)*FxZ zf1iCXf@Cm^F=IxOhL}H1eWc5khfJUgG8`T6J^p@2(f9)$H<+|yNj7!zf+`-~Y4M?1 z+~Ep>(GmVxV!|!fWn)h-*KZ~|PmjwrY*8ypy6-EyeIvALK=yoM3i=c4k$Gx0pwh+@ zvS*|Lm-VF@8fVl99rO}cct=kcmw4>q_gT@~@T)O2(`7C9)C_z<*sdVuHjFaH*F5Efc>hGLAKC^N-+g=ftNfM}!Nb?ExK4&OTIU@zBhbdA3fy(l0m^atdp zR_hfmDZL{fm?>x>{lL8a&UCBc>2-hjizga$D*Zc^{2(aK?|eV21wGyc6PH6;3)6xK zk4sef*?th!*A<*Z{^=Pnuxk&?O)3$+r&k?AQ(YS9-f+PLUWSR1%G?|x2C1F~@5eL; z3qSD_exfl_`F=+&o9-u9-tA!e47y#c8czn#lPXwv>Ch8%sgYsd)E~jhW3d@3)McTP ze$9jaWG|tfLb)JRdWM+lu~El;wxK3$E&EZHj_ID-8l40#dF%J@Yb>psj0>0GLfX(} zrj)%EC_u{wfiNHL9D7s)zX#)lQ>4%1V!e%Zt4Bs&1wY{MNxAK!_6BFW7)5s0qMCLF zCVH2YfQa~wgb0(ljU3zC@KeVE( zr)~(BS>=S;!X;0pZvTk-M8gDxJ;iQ1KE)uEz#(g7vuHohwAa$j@=lv7@a+!1$e&g# z7FpyU_`VhvO?`FK<*AEU5V@|9LoWFDaiv<6iIt~ykmRTKQcn6(p&dgQ{5PR{e~9cb zbX{hUC9V_)V9WW@goi2d38gWKa z8fHj<<=T2GLk_K_@MSBIV~)nQr7W*GoCeBj9J0|@`b(#3T6MePlIVoDXC7Dl^boV} zI8A*9`x9-suZ-L#5*x)_@sT^rpz3`PHO^_ze9R0gde?lP*>K}K{fY{t;5r7`$sz>r zedsJv4k}Q&k@?<@y__BXb}C%g*mvx)wYU8Tb%UK9hf z>=X1sFC`?He7o??FBKIDs1B0+ih?-arw`2c-=DQFwJ&7~WBhz$SGoD?oz{2v%b2le z`Y9BNyqS0^LD$gyV24%{^p*8pfTJcmZ8YDzitCLbfs|g9cVORu?{o|48|c7Sj9fE4 zxq;iI!4}6S}hr4R_8wqq7pSE4K>V`c(MmvVq{*hxRs&9rRBg?(jP_3KK ze|YI3Qxm5nA4g_qW%DSi{TX(LB`tV{zUHDdbfg71K}S=Zupj_5O#4uqx{euuP0~&Q zitgNOcORqR5;ICIll};S8^Ke1MNhuM&l3Lr%#Ag)0x?dHd{6lkdmjonhNmkD>K`I& zXYFnW6I{@Yk@@h1`gGsltEcO9Hg7KWaeXVI`hmnY6wStk+NSnG zaA>WTph@}iHP!&JbmBuby)+~TIiLynbCut+s5#z62_V$aOTWMvir^u2*W2I6%^2hIP=x~0e^fi@Id0A*MInnS$$VPH5!`)S-gA$@kV&+*#9Ur^D5{_pdd&PaldN<8 zGV_SK3T8Eph%JFw9+%osP6D) zVfcv;rPLuHIsXkK^K6a)^i<~+M#dd0>f92dNjGqF5hBU6ddRGD_9GDDk)JkB#pfF~IK@kz89Ca)OSW-z*$g7{*%+Z7sNr zr_Zi>P&M)(Zq~Z&nkr}6^$>5l9&Atg>C6wN$AmX)Php%``F+KMUR!~eJ-mCmekrAb z`WRl?ixpeD9QtsY z9@G)>iK=m=U$J@wNVI6G`4Xp24X^sde6#0UQ(`ZRc9j!oY=hR!`1RsOf@0dzvBKTM zimQ7jX|wJG1(g2``Xw>=A~f5{AY0rSml5Zn0ioLZTNNL#sJLDWfxK7l`vEoe|Inb> z_$SV9y?>@kzX;B{f^+wXa$1Oy>ba!)`)AA~^YSMipH;B-a5wYOYCZCncjJn+{5Z$} z!ty<&COz|*7dIFlTHz7OuS#MRnW_4v?Lr=8KeDg zza)*36aVRBC>)t$4lHAnU}yc-^AMD3nHyTH)Ew+=aW9zvNb48!$EEkRnqQfSKd8af zsb0*q-|$Omm`wpqi!D1e%?9-X^#w9vPzy+Tc<#wMosb(@4VNdZvN+ODT-npD`#ZPY zvE{>QU%(RRTrpE$P#Rm#ldC+Lqe5xHbRSFqi@tTbtM!fTa@ri`M{Z<3!y7L?tP z4)z46%%VfJ0$2>1W;V06z0Cc~yi{@N+81Qs5qXV&M{ z8BoF>IhkroiJrJp^;{isC}7?Znd!yX;}U#NF~bYY*^5!z&|71*mfHYEJAM|Dnd-U^ zJE!)sOPJTnC#*5mHXjtT_*T1AJX8j7`s~fl0|=nk|S<;pzH5$8P5oI8Cs|IMeg-vu4XsEF4Bf1AE+f;ieZBeNB+xZMlS z=&dukHd$_3w>6iO+UW6?aVt~XjtTC3Q@p8PN;(G6>`F>W?PNYRf`2tYe2U;X8U8Nj zh0SF$R{Y;sfPwBMw$HjLcA+YiafJQdu{#pudSOQR)&uoxY+Op=jEu4Tfy~L(l%C(| zO2)*tMsfZ`W+~FQNR_PTOZFo|q@@Hd!8+#GQ#Z||(tm;uu1-GViiNtZmRc~MNSNEw zdUk*+)oQRqSslLEWf4O=!`AyJUbHmmjdZ~(Uc3S4UknR3q3p?$4nRV1?kk)V!`#1# zxgRSFRIbx&9E6 z*wV-9t$BTG-Q|<<2ybSLO|jgd{wNrl9o3Q;kQ-8B+c?DzQOj7B`E&=>OYd6Z+GA}V zS8Eb*sH}KW#&y!#E`AbE>C_pjj(Pq}9&Oh-HY$)Zykhu>l=-r4nd!Y>`B%)}RH*V;|p*W0R!ZZxXO z;gO*~C05cDMBX=gXv=|zseMUi{aDkR1g*JCzSt`j`=W+W?HZYNT>f&YDu%aeqpGe0 zFm>NQZ%miZ7kgWL6B*QTGwSy8byM(xk{OpAT$%Ae6M&cSjRRfqR^~QhdoWzWpjd(H ztkE^Y30kmnmHT0DchqHOcA?$ke#=xwlcuZRc{|5KFEwQ(7cwm6@LLDTt=Z{QeGKDS zbW&=)C(gChowcXS;`KMFP|YJ?=9%qbxjtw?wy-s&hmT={2m2(B)2e2ws|J#y&+006 zP2TWz`N(C6^ybo^PtrES3!z*Je;ho&IaO-yk;rUiFm*WYl_aNSnl~*78XDK81Q0Ix z^2ebd0TZwN1>omV^YtF1U~O|kpv5ykKC`1p41Dtt=Qv~Qa(E-RS|{o&*a7WL%7;Cl zx!vL^8>7fCP92&%V8OxZRN%J9Y}^TOJvHRGl&i(An1Q(uGO#F4<(-gCTj>DGR2kIj zM`dXk`hSU=iU9a|o#9>rnd^IQ2td!iO(_H2p2ea5Bpa1=En3tDzU6U@E^={?Qn4Gp z5ofg@9sA*3IxlgUgAUwD*@)B*non($Yn4#cN=gYIbnJYRBsyMNqf%f316qNR@w=FKujH-&v7v(s#8 zrX{wFb!UGZ!cQsHd-ECYbZ}rhQn<|%Q)=I`8%uh9>$=QQpFt-w{mE$dy4 zx^8Qf0rz#TVzh?vB*I|yf%8W2JFU;id_Q!hFZQB(5{d4BEWCv zD9$E4W*2n5UD?{I5@u?p3c6To39Ogr&U4j7ljIcDn>T#B^^CI@>z+GM1fP4<+_?;x z-_C1dwTse#P-L&>Q?kLoiGAyk;aQ+sULDh#!@<-tSn#L$RvCMJ9*PX%JNvb5o*-k&fg zdb_tJ0j))6ZR*}AT;}G*URJ)~A&Pk>ng{4PBZ|Pc%+%{BpN?(Hp8uq`%bLlk*>TtQ z?VFnkL$CsCZZTn2z6*YaTSTSa8DAr?bv<2}7rfRM!VmdMtB@{iUd2})H!x$kq9QgP zyZo0BDt+IXywyY)r56sMfJd41+J0AFTr+X(aQMNbz#-v!u!Qg9Sw&VJZG*?o_W#=V z+xIgiFme)}K2_?WDmVKfDDtBF$=Y=7Txbz|{Bm8eb)Me!$4_p$Ua_iT~R* z@BjZ_6jkfx<<*#>ElsGcYLB9dBHAiy$JUx9cI={R1fjHuT2-}2Q9E|52vw_yRhvky znBRE4zn}B{{XXZs&*LxWI493>J+JF=J+Axx_CQf2hke8(pHfyP(4g7`CNjiNnCa!& z8-4vuwAzGBN@lNj0{$MBsZ__b*&FtGQ5ThWa8UN1=hN)LkR zT!Vw-*zSRURp`c-28_hHsDhn5rGn#H_RzBc*#%8yU{(wQf*$hM^xBoesIxNCUxD3@ zK3si_P82%PYR=%7ak^986x@Y!4GMrR*d_M;rdc7U<$Jg#F1!1;O7psmzhg|ZT2h*E z9+{JG^iWyjPc0)XNC^(c%$V`k55@Nq^aF?@R99Zsu7yaO!5`?G5S~T+vu}Nif{7i; ze~>T4E^YApK zQuhRC+R9;Lea$wRjgB`E%WGy1nG%P$#PgAgElksJQ303%425xgf zWty5UdwSkV7s?Fv5LZ?bKU zM;si*tw%`|A@+cT8gO>S9Wz1yBS7GKb?;k%yiE+eH9I+|mnpggl9*&L@R-J9KNSE^ zreH#i*k!H#l<=7P-R5fGvW(1|Bf6e%bfz8Pe`I>fv}Hdxx}60gDk zw9IrG6XPuB3^{TI7QXyuCNb)F_sB!1YuYBeUd?B#z!PgDPF6mZi;c3$nKLrR0ER`P*khBij+skd@=q1UHN(8wG$&MfZG>;P#E`j8%)ev`w?1!uW&+g_mn+WUZ%|BVx7$3T^`Kyy44QWU z`r*HuwyM~n@@3jIvVQGrc469;bUEEyI$`wQP;AF2nd67hDOt?*smkYK*I^P9qP=$9@ymc zn@?eKa4L6T&YZ;qu%DztcZn!BwZ2$U2J?%D9@@y!Pv*eA{#{JB8-b{_FT@Pgyg^E& zQ-hBf%S{*YB3TRBCXx`jrI{s5X>&V*pGI$zN9r2b(4p?l+4K!sb;rh8kU5!I46|k? zPBlk%41shv`@#6gJWGP--w!jYKk;J=cvjVES|!vFTW-OH!4dJy$7|8G@&lrFG|{5$ z_D8v0L0hB*R!v$97tV19$6MN)7ITElwqWC!%x@4N8BK3lei@u45A*#^Jg zBa|m`w%E-mw(qNWlk1OwpXT`)Cp!#l$AE4DM%gfHb*WB3VZ{-foH-wXvV;$q_t1>xS@K2far!4k|TuqBC ziXS25e;XV#salHg=mL7?=+H<_6!P+TaX$Pmc?6@5W#)Q3hU|Yyx%KvZ>bktkt55DY zqzYOjn`ss|2Em)i76o_R4Q{vG>s?tzTB62yErj3a=yf|KiUk8|(8UI%F<-yU__TUX zp2?!6 z9ZMMg6W7Wv%)>6zyCW)^6QTb)6b_X}>NDKtMd6ZibM$+9eup;gWv;n&T&iTbzx-Pw zkVu@Gxp(aq@lG3k0_8^_sJJr0PFmpZ(WG~^8N)bKujL9oWEiG^vVckr@48gkd*qM) zRWd{A+hfzCdV}f__Ie#1`5cR_bIUU$-!xsz$Ipi|W8(Ve8pGV~3#4M0g8Z5;kI)+E z4$YSL-&=Pl1@ogRDu(oX@ZX0F3Jszk^HjYkZ~#l=NbSBSG)6ryAG~aXi0$=0WRhpk z|M(@B&q|3S#k53=1_{}>TVSS%-@J;=|M{?eUADoGt9KNaPu|1JZk7haT#V*1Z)hx6 zjaEQcXH&!0U>bY9x8@kf%L|dyZs1@d<6nfDD|j=XxqJP#Uh)xjZ=W}}Qz2^6awHM{ z#H>Nw`;~MhdA~$vC2{g;hyxCDb)vc*UW2Oy(*AJqb&{57(j!L%jOZjw=Q2&1QV; zFwPIs^l*fMS~n~szBf@Rf}*@-xRTG=_-*E1H=B8p<5qUrddgr1xk#3vm$~^Q#L4un zkd?xdLZ9BPg;#Yp$v%skWJT^9{gyfNn^gW;9#BmWpsVIR3!tDetz)(QK%_=UCKPAdkL`TT{z{TL)?hUO*WTr`Weq-#89Qq8 zNR$15~TC6sl zSlf6rTgruae2tnj#ivCh;Nq}OTe3zox{o%qM1p5AO>~Q0Ou3gWid(?6_DovT~jauXuu={$sTD!T97%&D1>pcwop? z=kPEiDtgIWXEmwyegKFcwy&)g-2UF9*DjYV+HH3$PB~qt(-YtJiPuZf-R?X5u1jyd zE);08*EX)E<{gXDzjh>V^?ZEqnOj}f7RKh87$RsbAsYLT;m40}7;*D#mGn17nm*EK zG%u6oIuf$?^kRDE?1`mj-h8!0z%!|dj9FA_q4UnV)mi$)0hRHXoCU!>RNLKu>#G?o zmoJH+p(VaUHnjiivvPZDyzHt?Z61) zmYXC;%~$%Z8OOhgy*iHI!XwZzfT@^OeCm^=x4UZW&b-KPlV7>5+O3hMdsS3q`6oEm z4?s;GX?|11U34S@k-qY+3c3SfxoHHfPCed7xweJg;q(7tNC33;qJIO#cQFpDRRy)s zk)3-;c+bPh+a;29$KAAU^nbxBZobz`IdORtzgvC8Emp<(W;9kYi{_@j&l-#wE+M6o zRwZN#ZCsv@cF4Yl$zy8(tieC#4jReyjj5O=;gTF0a*IUwkb5==z=z$!jL551e^@KS zi?o#sY&nwL%tE1X!Ru?sT7-s=t1iH z-cipwBkcGQp1bzOim?%jP@6v@oxcl~cGIdje@%7~J9CV9zEk$Cg4auj7FS!*FKW-F<#JPrRfMC5&k}ttSp;Suyt6wA= z+uc2fd{-y(pMTGY^C%@EBV0c|Ux+AFtcN{fH3u86+6?Ra8ZSeCBqd~-e0L%;MT*wRW0Scl> zPiOMqGfpH1PgRC+eQv2yk_~=a8N=El!t{?-?|b1TuK|Ekc3Wa;W5Bn(GDXtn=OP{d zsU-OdKjbWKw$F!%ZhM+W9`MrJHE(L8r7cd&uD9M%;7X(`lOBY)Xt~pa2@~n{7t2$#vE&MZ`u;ln6JuB*l4<@{Z;ohaBVR#40wY*O-rUb!6m= z!21i6?J*izd#touC9%`XCfv5}KH4g=rkHNg;{ux;_!))O6$Bd!)pg$|FE2ee$gILf zAGYd{ROh4psLr=GIk(FQMUu{VA%0D>PZXd=P&H<_qhj)_B^m?{hEMN|h34H;F)p{c zZn=BNqMF$>8E1*2=DG8Ep6Ljc7({TFER~qvIhE)TZ`fqNA{jQsVh^#jTq| zy(@L_F$)D;w{F9RSPkp$i&peFj;MV%qj>(#^>JGanl{e$yg`16o{ka}U6pa?=z{>A8;vZb9mTg#o7alM zl0FGca1qkqu*pdRV8E{?lx zEG|!tHEQ#&knfWhjBo-XQ(s5uYA%izh1K&`@6kmh7#6-mP}Jugi`HP`%F*Dnvj*9q zuYBTrUHWIM1CxFDIefY(v>-T#hq+LA_#v0Fe^cX@&j;}cK4jQ;bQJ|y z?3}pcoRg+;w6jdAuFHIs<9LEaZ<>gAiHnnXn1k?XQni*){DI27gC0ID0V|=_Hh}JGpLH6I!0z_Q9@xz~9FMR-FB##64omJySfn01l zgl7;pzo{fu(UW4ZHzHeuq4D8Zh%BMDu{tR-7hpeZMC%r7V?!@zK2qtnRu~kc00T3B z@6>TkAMzVc6RXGY&F$2+**XU_cB#2?FihdA!sKrvHDm<+Uhc10SnL98Bd|=6mvTzh zNwm5WjKbOfWGt#eT9PgVMj?c!t$WBofiH7gky)JAdtf)7Br-_nmvTJkn>c8FCHDNG-xb5brQyRD4`uWPN>xRU5 zzQwOA1`46&`a&RnD~W2qe}J)H2+p?B+P(SB@3`##rXSwnIJ1i^%&rR+q(R#=ON_VQ zB&{aH@i_0Qx97?Q8pY&=Y29@sOX5Et64R(!_=E18Q)t42-|2o4do&v1rWDL2j+(=p zcLM<6=QDveT= zDT-qjOf2#ve6i?U=iG&vcpksC-q_%^c-rH2le0VSgps8OU~}yD>Zf@c-U}zNdO$_a zD6L&2ub$*m9wyvKarkvkY@H;le$8;3f>QqMe8S6myQiRUiMChlB#lwpH573dV_7`V zx(3a_#I8mR?_`abH{X3fq>q_-ju))j!n0rBK|W_IOQgw>)?*L0H?x4injL*oltM?Y zC^c-xysz3^k_#Z1^9}VPPeD})xuc9c92V<|43jsrWOTZ~@VqUj7EG?jU1U4_two#G znw2sM|A)3!u$?A>TnwmU19qeZfUCQhW9tDo=aFXL4-8N8(;NkZQD{3M63PP3?J5~I zSl-n51Bv7XO-@lL*8G}p(^o!PeapB6he9!9GH*wi>p}gOD*YJ zi-_Kd6WK(o3vm=IMRfqk=Rv9c>fi~NB_K+NSGF%4a1t+{7?Fk7oc8i_vRC&z3P)sqF)hT9e+_vuqjSPC0$Kor*u zq08ZzpLc3X=_7RhAy+G^06+ZUfA%EXRM9VQsa|2NDQWtcvd1Sz{dAcUs#=a0()TF* z)Fv>uWrBi=0sjOTOSdl?-ZXr~vqNp}&&NR4t}g=MGq`=1-L`ktS%Ev=x|n~0vL1&i znBvmfyRuvMAUuMN>`j3XhrMP5(N9e5#WD)#Gscq8G%6Mt5lQ>VL{KjIMo3jRFM>ffNmf{$TFIZdR$R_4*ueR$c2Mp)8WJ^jBLN&Wr+YBE$_&r0a#|A>? z(xDOdVlakhn^mS)JsW6Sx<29NcJ&fUNA`+;H5yQE9lWLUdAuc_BeL&S*{=O=?-m#5 z5|V9ZZx?ksvI)voS}JIDBuyHuN2!CQI*J%X&*$4{c`x5EGCOm*KO#ChE61m|CaKh| zbuiq$s^d-u?8mJs;)DPYbB8%N7^lFkeA|jgnK%sj4THGvK+)fq8w2$r9&Vm*GW0sq zm}wFmsr-p?{(L~}B!`8fkeJF$zEZWO1UpaWsUrO4aj2?352xK99u;E1`|zX&G=HmO zJ=&i}NW~9U++!CXedy5^Xhfs@JzS#)Xbx+Rpc`+LJR0$f!>)EcGkR?f3yH4zDFdbQ$KAS?FMD> zuR!>GJOOhf3mU6F-(Nn6`6Dc5iqS5&qcslRYwpNlH5vZGFo;@4>;-23-}2Gs3~lE` z&`}MAEyci4l!X|U&OrPhm!OW#FQ2t5iTiDUp``bK%+bhSZs;iqVH{U1&hfbQZS-D{4ee~%a8bzBt7q#Ziuw;xUGWt&8mBEqUKP9qN}K9{>ov5?}@fk z%B2%A=Sh~35czRX4QqsQt3={CY;OJF1ju-MQUYE(7k`F;o#>K$+^R<(k%L?kyL1}= zE3!^nqKa3)$cfoSe+K-$Y4c3d6vgNXWjpkR&lRIphuX436V8*TIkvC6dtG0mH(_8gt$+jb;IqI^k%})t!6|D_kSLe3t zBdHn}L~g#g`N!g0m#VkJ`~X-k;i>71Co2lO?-0*V9`r<(H%-O*5(Ti)!irgc!QZ&S z?#@=4H9iBd`BdG48@mqNQ>cu02A`U=H2O$vNo`48>ccN<692iEi7aOJ>;~rJ9sr}$ z42TWD`j-f{nxf`M*+&ZN*kadTLu?S=fw-06rElwY$GfBjyf1)(r5Mw{w)%Z=LBKk{)9|0$3~VL=L(6>7{Q? z57c05u>bG*UXB&~1$OxY-3e83YmM7ruZf8e`baZUMpP;eEyE}L_|X~dVv)xK4ttd`-5Vo75g2h`UO=O8>8yK1<7qJy&O@&8SUm zBWxHW`(9H}=tj(YAMx!f0lU$Pa$HO0{R2#9sihq}b8~CoEq9vT9V4I#JIB1)%e)&s zUcqH<($3k-u7)jcv8xzffAk@W`IB?qY@q)}O+xw3WL$6_;{+(EPDhi=A;RfEdGux2 z|Beo8iba#dmMkqo`TXf4HLB%jR7*7N%TI?NOq}>ot@7oR^f}=nX`m~q@tz|ug?alS zZ^^Lw65D_T?-?)BY)=zwu0_H>Srh&i@AnnGU!WWbi_1ZZdQU898zCWZ#E|^pdaIP| zOMkws#f?8-^yL$dQ|mHP7~f@}d9Ai2muG_uLZ%$29RI%`gY2`QPp?w9pGz1t;A{&0uYrH#9QHlYqrG&xXtJj5zeS4~RFr?SQ5%f@-!I_*{moiN z{adY_uFkE{VU+T}#fcmF*D7{ESyv0GKmV72)C1D#>A=4lvOenlulxFc_qo}4`D-m1 zu)i|_>Hj5U=S|6%o$0aR^GaGo{J%uS4nWq0p6=po?xMMr5OTI*l}5ZcF267x!Rx4> zXJ3%A4J-PZ+=+Y3ttjy7Nz}@sbxPot1i+*%?H3BVE@@_vfMdv~u;+GNYEUEW-ozlWHm}lTQj0IJ*V(3G_I2(p=7G1399x4# zd`0TX1AI$*nU!tILgs)I1)y zt~Nk+R%SL$R>Sh;+RIejX$wn^L(?p(LU!;I!-4*2#165Sc2=Ws`_7_!$jGfCeQpyg=+pQZE_nf8=cR9Qcp5^1Bt>9b=Evhbh?o-e3E%J0il_^k>4-gB6R8h54Q znX|jD(t?Q&k-TbPvXh#Ic!Y&^H!wMV9;FPdd~nV`GXTbYo$&{7wR7}8x)^fTl*2y+ zPwY$oJ<~UzRtKp*WqC9kLheE2QW2jW=M)}O-h@KOy~dt2DCM$H?O?)&!O69af79XJo;-b>XG*fW7y{$Mx6R6gtDfj5u zmXzK)`NyFmW7##Tak^>UI)A#lYF~bWafX0)$R44}spCQghH0;`I9iDA>}k1SXO_`( zaX+;Vq1oeWyk%kVV`NB)J}K{fvb2dsU*Wa8Q7uLE@X<_QcN7Jc{?IJS_zCT{xGxy(sO4V^Sy3ge+$3(( zxi@sTtc9%#=PkYmrl_AEm2Dlih+^e~>dTe{&dd%T{jJ1-Lh^+sxy7>D^3>aL*c!l6 z{h1uH+zw)guCc{?ILQAyk5yz*lU)szzo|#VQy(m!kEFHK{0AgMCaoa&`0AnbD3y~* ztdW>MM#pH!uSY=|Dm2|Yv2;uR20E$EB$3gfe2u*q7eY2Fe-9u-7FvuYr`z2DzhwH& zz?7c@n+FukN~S9hFTobYX)SvN#bh=JxJNZI52D$OpsF}{cGiDT!h=(OR(1(SiA9lx zSY|S(ZYd&4IwJp^ovx@@DJ`%dUNn>a=re>J7ROeUrj;S0mCPhyGnqU6EwjTG>$>K$ zqOhI}7Ym&1H(J{LmCEctm+*){MlWeL;K|I1xijD(oS~_|v!==Ef{Z&cbAqL;Yp^`r^))WAwn&&HtrJly+~C)xZ_={npHKhj!^!33ohH>Z!U7J) zA*&9xyi&UUIHiT0)Pt6+req!?{o5{QT}XzdUW?4cx5MQ8~M0&g5BfZT-EeOP()BDM_mh`o9A= zmY0z^lGp`I>-k2jTTS^?*EK3gaBP9o**tT|3y*Mc&s&4w`o1nk(#4+s1%Kd6<9U9o zN{v9P!d)``z-OKXB4Is4)3+Q?LUjmPyqN%BRh-Oezw5YT0gOJ>7bhfE>FjF=LoD&U2;PJO68uZ#V$vkt6;vlO&Ixde7)BV zvID$7-^+TrFLpr`>nX}Oy_=>CPn82gX;VZRu&0BwaAywBo zcgKWI+H+!MZ(3ej`sd!Z?!M)f2hd^YXt(XjU(Xm74_- zUeCV_o8n9URDU?f2kAd)#G}+&kN4G?_LYZZUQ|ohEZKW%43I|qqvskUH)zQmDtey4 zOfovb;yCq^@IpWI#kk6WkG^v=M*kcZYxc`#zvZymono1uFW8Ti|HE$iC)tVcFKoxg zR@qD3Sl;vp|LC_e@6z}NyH;nblPR6^dcO?wqi}79wHKg+;*;$SGCwHx3z|3X7pt#n zDD|U|Kg!2H$=HgWf~>dAmk%k;arZ~{5kH`g@Zxl&yJo+ToVXOzo8u{U%>RnLoX|w+ zZ8nUe*~W;hkpVP~W^KcSWM;{xKUbr!+^2O< z9qD&%`YxG4cWPsUx;3S-4V^b)QOCTmshU~@8&j|#R2QW&oXP%FyZZx#oOx!}3*TgF zU?!=lg}E>^p{PU92baj6sAc*-Nrr@9^B?|V=l>7v2UqG6(FS5J6$jHMf|Yk;iu*0L zJ^Mx1K>~q8YK;_2_fWfW1&aZr8M8IRle*o5L?4^$D-Q@hxn}+mxY>eB3~(TvP|fRzdPn6yA9Qvd zq}r$Np0%^JR50{0Yz|9)0MIvOR4A{=VJ9B|*ZO5Bp4C00(xy_E>fQfQsO*g{ZpmLy z-s}@GZ@*W$W%7zhO7Bs0h$y6``gUp0rf(u5vo`JM%kd&m_=ee>d;+89G*9#df%10} z0=|?hX1b&%^nHlzmM>Q(r=eoO9L+TGHXzyB+pK1|uKa1rp39jt(m>O(xgsp`pISX1 zMOa%_hHjL3M5eOdD)Uf6j&+8@PpO6Fn$5mcLsxRm_ES&* zs#V${S5Gm+R;Q-jcz1J;6DsV(c}4O~QEFyaXKlIAi^)2XnLu^Zc(&}RGbCy+zbhEu z>)6H{QX?$-eZ4MO?u7rMgZG>eDW;HAXW+9N+Z|wL_O|?t z`PfWF>Bja8UQFB-e~}kOtVqG*pg$RNr;Y3$+zx;2O`!+?KI9wY%ix$ezYgDT_gqWZ zf14=J)%<0`6=6F7{w$z?(S}r2t!{p=y$cCL=x=Ru%^h`*)V-yf0?iT>7kyV0=14m> z#$R8JLc3Th4dQUL^}D_{2-NPIX!lB`vrciTPBy8|1yOKcZ%&h+usv%K_xajOc(O&c zp7bpC2vF7)!z`umu*FcJaJK_ zyRgGOfnRjzbMEz7<2s<>*>@Z(Hr50-0wyhI^VIA1BCz_x`RwHWeeGQ!AE#v#amls& zCRL$Q3y-{HJYUG-qkYs*6wxex3PhSGG;YeR9KKB$+b5BS>O90mdyg0ub}B(xnzs=7 zI5zeOh2>}9xXq!EEoUpUAX2}TB}#=%+!6sZtX{D^tN$0dBN3koQ2Bw7bu_ljGC9T2 zc>MX0baDKrnv9U?dT&!e{Uo5{5_{FW*^9&4Ve#K7_@)4~L*(^R6t2gQ*{u4@o+WjA zT{PoPnyP<5x?=D_b+*OS8w!zZgK~mJ-o@mgOK5jZ9>&@uheR9b&3guJqMM0;T?2A?i{-t7ahJW582 ztdH4;y4{Lv{ixWqsv$O2Veh@SiwJ@5f+Kt+N0|pNv#b8HPw7!*Sh@CFjscJ7;rqo_ z7E96`;%I2~9@S11Z+v8%bjVjK!h#g3jBKBE?*&^aRYi~L+WGX*uPyDBl>E>ZIv7h3 zzFALX!|qyM|D8={9}P&Lmn3hUZ= z(wXfy03+8m@SVf)Gw z2m56WF8w*~uj;Jn#0Z69*pIG1VdfyZr3Nx@r!|acM)IrU&f?AXK)7aL904brC^hAQ z<1Y@2)}C=N_Dpl5sUB~78s$NHtiTU|j*Rn9T^U-M%O88t$6s6vC>>dKP5pS2sJiuP z#&NiwOnhvqvF}lDTxwl6EH_=MC&N@M4wAdm&W2(CSfLt^8_DbAH~ew!FT=)bGU3h0 zjYu+L<#seCKkQS}D(B`qnf1YNCwOkP#njS_-Q*2@ET}u}`+z-QYOhxr+MCDI`J$0P zT)lPw8W?dx#e3BDnt&oFbYebq%)GF%a=xQ=N)+|qY4BwqA|rI3w)IMhQ9newal*R) zeV?LP%}sEIZ-FiS2aimnl-zy20{=~P#o6-D z$1Y2cR6~7~$RjjRU+YuxKzPiN>V=K{ma0Enz-f{>q~l1JdC;RRiWaQ!rqx8geN|@+ zaUg5CaAbsxWtsCLCU@jAkZP=|5EXfueB~rX5f5=dZ_iN%wZ`kb$o{@=_Ggxg^J52H z|19&x*Sc0}-7-@Z0n~B{wVjHx2L;sSu;58*roz zKHV*nXp0YPbwrOpm>rtD<_s;Ss_1v{_?`ZlqS>|~+>M^jqy3&ojTcxecdeFwMc}I{ zk1U*=1&xcU)-IM91p>c&N9aha{_5N^(dMy7LkNm(5 zy0dv*6FfKza@nYG(c;aP=C=}G0;vSSETI@P@Bpd|zw6tCP?`k$x26AOvL+*F=X{`w zWpaU4@ddO{@wd{A)-i|0rc-NQ==5aU4#Qb*d+No5-HjVCQ@&sBV+7)wsO!)jlXaZ_IE zJ*U@!=We;XOJEg$bx3_MtE!?$Qe=||f2T_0UW?(Y6%&i2psU7y7=-dgwXE#>%X!^` z7&N}~YPYcAv1qo#rw=N3cVwt_>m#h*o(>LSZt5i{{F-NqMFoeB-cga2s6P5w0T z4n||jq4Nd(w@;sx`g1ifs_^W&~;=Z(-W=9X^aFiGE_yi#L1-_fT5bZPDMSB+H>be>7%GR&_k0^v_;#xV8 z*!s8&yi0Ep%%idknQkJd=?!150Gj*uab#WWEm!gGo5VP?Oo%COd8h<{ zhml%OX57C4jZ+_$@E_hvc6@4;@!6OjIe*nxd>735Zw`_Ucu{w``)q?Tz-eqLn<)ar z$mlsWzP}o%_fd>ejHljLX!JJHcz;PyJacQGsdEjl0OI@<v82)U1ycysK*M4sgv3(|RbFp4P%;Ss|DNnA8=pU{oWmXoz|y6__h{+N)y$5l3mf z%hh>H)imVug25bKaf5j`Fb61!Q7l?o1VU4G5(Z`biBE);2sW7MWT3>a$)DN|cHb_t`vN^vF2JRm-b zxjVd9TR&i=pzNskxWG-%V-*pyiz;24b0035j(5SN&2-xA4~l-9oNCli-^*NTyT|=H zlh@=gI~^>H!D?zpYAM=WW303rB{;j`^7{{GIk< zD&mExa^9WVm-;p=$K6rND^[g}@d-?W3SD|)ia)IZ|-wB=QNOn~Wv1WlLC256`2 zuavIN2YuRgT)HfV|4-S)OzZ9UD=!ZXhkZ_K`=JA6l~AX;lAA8X=?A_1g^_P)ELy_U zbv|gnhg0daKA8!{B>kim4L)qWI7_22j?f6Jy|SbEsy7sc7F1+qW{sh-0zs(Pmc}_I zZdK89i-;SQC`dpli|^G)>RIjB!6e&dX4mWr$3usqLvMN#l5V(UJr8KRRfs{w{EdR# zsK|nY1f_tT7)jmMO8mi&+oDd(rj2H`y}`U-Yi8h6j5pkkG+)hi5j8NO^|{ zYd;{{LnNba=8VgoJFkrYM+?w~Schjp+F~sMPkQHX^M*kA{KsOgzVwW-bzVa%!fN%yanZh8sWT#w3SeNM%!h?Qy{n^X~3)E2UpSika%n&z(=iQ`u5lkS?o2T|WU;H81xK z&5}lD!Zjt)c@aRlh8tX!y`oTHfdi{JXn^nw{V9pTrZS|?scO{SFW;)+4FW;!Bt1-H z;Z&&93>_f0#p*3U_V9@@17v^q zr#XFPkEsvO8QPhZ7E7Q2^iRmMN?OzSv7h_O=|`-s|7Kdc^ezrz9kq!!#ic9$h&jV( zXf8x4svfwFjq_7C!6}~p_01rkPXaBdb-f<+JKQniQ%a`};fLI{r7{qw{7v-aLyBxs zS&#u`BWBGnsq3oA3RFYh6g@G?Z>5lcnW*anO(tZg>T9x6Q`t*RKfM#7LGLdgrF^5U zkcozo^)b-8coMh&ooWVFkNGk>GLMmok-YZVw_$oK#l7msC!-IBKJxpe^-V3*SMP{ZwRDP)g}^Px!mzKZ~z7(nhb@`NTEEqG%}PU(s`v!N61&-m|CuzQoid zoITixb_mf{RTp`k0Nc`BTWTD3@m%hVpx2YWm0^3j7)hQ4yrdS~U-#dPU_4@P*ai`O zOcr8zE(nV-yDIT#C}4>9M#`0}kzxw~YccUS6Af0P;omX^m**k(qy;;fFYNHeTT|6H zOd5ypTYrw#7jGCD3nb7sCWii8)tud(mr3wUzPQ{j-sbKB;cJ4#88>@ftlRR-Yde^8 zopCWi-;`?E5I-V-#h?w<7X^#DEPGuV;}^ zng*q7d~1?1S6;RL03p|zJ_yH;bgcIfF44f+fPi=(VOhV^BN`qkepv`g&@e`=h_X5o zD?h{puyK2bzMtd$l8{Hu1l*#J(}*T^x_fuJWBSUox+T)t^wEEOvv23i{2~vOSoYXL zPZ=(PeM|f7aRQ64^|_1{tm-;r6S_De51|}@12O@V4Zoh{taNorS6k5!2vQ8{O9l!ZsPQ4)fUD5F##GarI^3 zDi`l_%U^o#J~s4=_r*68hoDTL^G;jxsnKPancypg`Qs>tuk=EvpjyH9%Bp{P=y7sQ zfTY+j-1KNAf-g+`1!YyAf4aYVpeM}T*vB~ACLP*E5BI|fPUP|36p~r1J1M+Zxv`nN zqsAqoI(Ov;B+Q)lVH89KKTd7c0x5NGTPESeWaVjK$+WW*+M)3X$%yAdEAlshDb(Te zKm~nyIiRV(M|;|5geMk2|08BG9OKIN1f?AS%MC;6bFJF4HOHQA^35^L_6niU-P2c$ z$<0_A8^gVCtDOt5eRN~hg|EtQ$8~@;oAU0=wX<-u8Z48nDnNpub$J(m7wZc2kh1Fo znSefqn+(77j?tr-BwZidY|&Wo`sCF-A%#Hqo}km3(2~;uM==Y)=&%I-3nW?d9Vo=Z zHWG|e&Yrli!Tq5nSWMyB-@-)@{HJUx?b-| zg|kVj>f7DsfGg?+j|rIxTij* zv*{fHdI<%wqxUgiEH)hRudo1~qjI_$AK}(6!9sp>oc+DJ{++Aj=Ia%H70p=d&U3uZ zzB=d#KI90>bDI+9bz@`MTA%Uq>`BvIC4}wAT1?=U@%>y{Po%-TooB40jy*gvAI%IJ`o7Y* zvXEu6uh?AN_^QD4+bhG(SpaM1=MsyzL5S;m%N~6?TB1RuJ>JvA@#^-c+r;MUx1IwI zmulb6XpgShGUNM~*=bdZFJgXn7l)0ty~2YS6_3>@9p$sKRapyPb@?$lFbv54%Dkn> zB=ZYRwON-Xj>7;W#77 zc^0dzoQ5YMK14l5P^sLNC{5$Oah91S4XpDmI|JTGmtHU8KRO!^)piV{*y3VhRAXbj z(J@z-eHuVYpH<35dow(J-_#p0!akJ13R~8CsK+&!QF=@ZIDCY1s!l^KMJwN6*26@r z$u%lodO45YR}@s1u8_%;@1CJ|B3S7to0cl-bC`6?Mh%+Y%`0xGQF6izU2Z*-+m-ge z_cY^lrh1FqYOM@te|nGG&eBvw(uEjG;t>IBS)6Jvk(YAAdADi__!R-~CDBu6KtvL)>K ztG#7Prc?3hv|?K_{{2#c^wvdr7Qee#jT@2f%aEF>_9CX|(=QRG{^Bi+%VJ21{tTB; zJk9#_(OXN#hhgkS!%;O$1%%(Jab!g>EheOv4lFgn8Q>%+bWYv#Nv~TZ_@7}uutJn$ zSlA$7wJK4$3ow^DNx7IJ_~ML4Wi00E8bSD4wNh;BN@$vL7}K#2J^LCUck(_7d+NY} zl{)mlPk>7$x{iqw@B1v1tYhak>uW=$UlekE<2|dsIY%jD=KfT83e!=x&qWv(#bXd$ zUdztk9JllaWG1vi9dA`ize1~?Th+R6o>t1gI)>(e(xNsbT{6B1ccUaTS z@~?=1NK<*=&-$Kw48!54V=z>p zjQKHRNt7#p&~|JL?+Rr7op*wv^Rn+Q=qBpDMwM4UCiC9o>>QDwPYgjE^cpf1H*wqO zoP>9r1*}#|fQ#&R46SjorWNt0<2-DMvJ892A09cEf!nLj%Bo&%3Pc5SH`wzab#(0a zVCfQJx*48vkB0en745e)lXZR~VSAL`IGGkDkgNZ6JVR~@1S#p0(k1duWHbh}zYtht zHE}Q*$bG|0MXCBboZ(}v5H?bX$1j-ytsxb*=${k5#+y>dIoka~ILx@+L(FM~Dm zv}Y`4aHKeJ{`=f?8geXlP%tTJNa7Oc-d2HU2f-z;^UFbPlsx$2y9y4`+*eZ*>pRTc zp^W~!pN;ZoRPz8O?8ob+qlG=J=J{@J*9GxN z@*;1nM}4j)$;Os_s+y)4-dR6|*9Xd95eqeu`2}u~i10JGKGCo%`cQh$NqD4}pM4*v z>xg^|{iXxRRq*1EmE69eiH+DFoqX!old#~bi4%L%JSm9D7o>tU8%;R(Cl6}!)bJ7l zAl8c*3!Px{}@bV~SVD zB_;tYMUpd!ro_6;#CsOP4&Z1{{v69DaLU#S#&S~97Hi+v^AP%!)6N0)8cUj-gds5SV+xZh6GT)%NI^^Hvnf4Vjn|<+GbZ z&uwp(^VjD`@%W2IiOC1wP(n%63EE*5gs$|@Bb8Sso|oG>G6jW6&72y}aA6}gTMR}C z1o0?65sK-^FR+*Em2*jP%J&d?B*Gn{BErT^?^#8DLY#dn)gn87G*aPz@N|UHF3r3( zu$)PjxBG6%B14=<^}5J!Tg!ki9PEo@bLWWD{F(XsZ3JS*EGg^lpbMvLuI}&!vu$yV zf#1g}?gfDsl}bVfFX39^*tKh#B<2?TM-eK6q+$1(+}1@nS`VrBYNmap0L#2~zBlnP zq_`Q!I(Hane{T<6z=xjE+jlF6gqW@X9xSN2nA=C$-FCz#52jv#HZCM&hwgr7E*2dsv$7a;&MKh}Pz=0dS>4bZN5?Z>4^`6DZ~XQS3T z9Fs{}ORaxv{d8_JW_FPta#pT@*N%NV2TN91NNWIG?BpuqQW)A~Win~#_#y!T*AD;K zkeG`ZQ%4qiinSw3Lvxr(nC~Z-hyKUXXcl2(MP^%sM2 zaI}@aO2?>PB*{?eT~;Ks#p@!yLMv_VM?@0)EpmmqaD>G6$j=1(V5;i#HJkH}E5_=- zez+tCMdu=fbH|g6Hl_Uy6?3nCkWBECu)}7rM0SXDoq>NIjy;bKMO=rb%d7g=78tBQ zJ7eQ^$|G-PNsrG!;ro5?#a@dEg0i+j0Q{^fpWmDfIhj$?42LVaroVpMKpi?Vwgluf zoA+C|olQ|>$OeZcbgBpY!;f+2!(_AVuhr%1Hch3Oy4}YPa;PcZ5TA3oP;J1!d2a31 z#?Kf9wGOqq8xY6vy%yt>(#?ZXV;**}w0YVaYYU~PjQGsT5wR8t7Bx1)!PWd8qhk@v zYO8=!;u3BCF7l@zQI(ZD(WYDfc$%o~na(iSK%1;LISmRW6tEd7ABizF3T-~(lvmTQ z(=8PA>fgGrRokCHbG=KP*Hd)%@WH%ho+qXDG&Rd5VLh4{ZnlPr$Y)abUKhoys#S5p z9Yw95g$u|4StW(oEo$z)7eah5thkDhiI~swWE#>@$`V;GGUZy(0EEAU5`J#IgTK}H z`=wb}SOtjL4_|&DtO1?C-7y7Rats0EQUzM<#i5aVPB%&H{Fwo-4A${#P4=5#1PMi4 zPfv_CDlCA8?1^EnrleoKos5fiuP5un?I{~`u-V8{i}1}A+UB)& zA+G$eivJ16CuWb~QxWqabgP6}&Ynb;QWf(hWxE|ecw_gCf%lfm#ucw`Epd*+J$PRq zWk^CMcb$^R62Xx0&bK?E0mP#7TV~JitXGREO17QXIi-tpVeKzT;XI}xHI#EmO}o8h zRv+AR>>|3Op`%93HCihv0ES=7wXbJxw%-(B^@p-7Ec+N}FHP2SlgChtO6rE-3W-Dq;or(WNzflT&? z31$(@G`HJfQJ)#Nrtw?*!m62oi;@OGP!`0QKF^eqji`i~JJ>VyhDggy2yKFe?)&5J z?dK`)2i8F00e4piY(ZIc4PY3sYSJ))+Zn#YK>iA`y1+)lm)OZ$4ci}OVfR7bJpxtP$`goJ#Tu^vuY zHSHDKfMx8U(#&AnqUvw`21}v>6m13S`DLJdkSD?$@&YXH#TExm%?hh9s|;|FYuWu4 z&D-3k3GQ>h5c4&~w3Ufr$K~Ex8EdMm>-KV=^|GstkwenlZ;Q3tNv8FmX_LI+lOEX3 zjuMt)k-pzziD?>#qW~h>)4-UP;DakL_M>E_CC~IJt`6EHPq|#Cjq`?K5#h~)cK0F;AfCC1FXiGbtL4J-W@SRT0%$2eS1>>7*rzM1{hQd2`xTXOp#bn|a2= zq8J#PmsE7-JqZ)pzG60ow=?^kU)YT#oat?4**YnsiJ=a^CR%(#3gIZgb8s^RQ3gmd zFzWO@tv~bLB+TC5c40J*_{{>bzIv4fs1P1^d8^)CLhLs7?lSlr7| z8Df;+>v^iTfnV4A9jaVUs@Q1-6nAw4G0J?GIU1yfQh`Zypn|w7n5&sRRQM}bUw@kX z;mvzC|H@FijWDlEKooA|mKh)W#<1)yZV#W%0If7OAqwy7SLAy|`=U4G{qk%%rjg2O z-c$gCfx)~)iC!KnHy_8(LgZ+>SsplpiGd_dN$L`5HA*?gmzfZ&@672AC*yuwSq7=N z`ajz<&+_wF7a_QeWm^(;z>jaqgpI;ATJ1uJJ@#CQ)`o8xY2Qn>1AgJTbI>!%HTM=RKX2D7rcfKP3!w?#Exd?Zu zaCX1m?|40*szCWPsjO=!hBuUUCQSF{yF(X@!wO8$tiltPZfVx3#>#mA;GYuth;%J6 zwphw|la*O9|C$dzPS#m-Hlj!3c)P&ERSeKkt?4eR-Yy;XvNWU+ zXXCb4aZULwNg7lF%&$tSY6CrG^?8tE&mJ*wL>RlwYeR$FHB>K)LEJl@y-?``5waj- z9^*_lKhItll)YQkpNZ-@VB4oA-*1`31tt~;$nu!rK6*%&o81yEsO@Mg2QPooZ6jH3xfM|xvl_Jvz@Eru1|h#jB-Q))7@ z3tydTZ*gk3JyF%@cT6>5s*j>TkFKL!f2_BTSBd#mz*)kfb~a$#|%o7UeH?+ z8E!ot9?k#gl#P-00-4K8pOkGGf!r!l@EjxQ;$ruz6W=+@jjgqjY@S(jtS?&#GY>4NiPnc5r`RAyM93f89p8>bWe!NZgjcV$-@43 zzFc)kZUHyz$T#rJZ))tCqH?$Om`Gc^4)+fNA@VH?BT$uH3YI+9{Z-lynN3(9$dq&z zBj25~#aFz4XZ<}_rQ)RqFxLKT;@5I_)oD=MwmZ0vNs&>8!P=&$6j~50<%k4b0=)j{ zTaH6ZPNc6aoi4VzS#F}^79Tz?IP;NlSzaV3DCmO|_!5c57%V;1=aL^|(>(9DxfFMB zOI_IT%ldgk>$Irm30X5-p;h?`5lGP>b-pQE(X(h|(op@@chAB!z~8k}GEaFY^s==y zwr$AyH;FJTWcBr+tDq!gKF;D(o}9|?2I{f+ncrEZl)~>91Vhd*>ho*`c|s;I2a)Q( zQ%`=M(Sh_34YH&jKB}c=Dm|$dCzwRKvLwjd!z@K(K`2<4F&(hk8SA709~uSY_3F3J z)Qt+vcblNx)8fzpS^;GihE{!YWYJMr>}5-TyNXD5Hv?sZXvxWZCyjzr*qE2rQv516 zi;i9O56&xCw5vivx~pARY={#RsmioJ(m>Z5X-ZSq3{}6?zNXhasqHmhU4&hKd^TYA zkj)Yi=0z0>ax;s#es~g(s3RwnO^z+*3c{3=&&W`hBx~ylWni)wv7pA-P69FCkz7$5 zlOCRh5d)r}x%@79NnZ~LIhSjM9i#`rCOx*D)`W#KmxfM~YQ{v1UmxV{!_LoQ=q^m# zn_h5c;V@AKV(RR^>cBz{~gZj{cAa|I5h> z8}FUAQ?MH;GlYPq0uyKuH@ym7z*Wx_4oI+{$VQreCSwJ2DNbSQn!eDz>&{7ES(tl9 z;Ck9FI4&rZAc)_A5TjVfTCRJeFAH60SW<=tN!LL-?y4Pq3)T@hk+Zq}cxp)2P$}rP z-!x`j&)jS2I$6`y`67>@cV&9usRYa_3)pp*Yq;3qmHPC?qTX)24O9h6nkX$=@p2S0 zI`#s8CK#ryD9REH#=W{rx3eTygJk9iMW$}UUKej?hPiDp)gWKcei{4zJn& z*eJwR)n}kqjrowiMH#>Zr50tMUtuB04^`97u?Xwk4YXuJM(EDR4wT>11%sLBF6^yg z)&y&pc;n@=1P_Ub%4-NcX=TZ^V6}ouysHB!BQywiM;IGHFYKs=UTtl-?0GEpImBl= zgT|aq%DHMR*i5?68K~}0$I!Zuyq_^^?$lSS;0jufDzlu|&d5zdTK8r`qcJ`m>k|?( zUf*O+J`1K@318urZUo%;?pgNr43GN|9`Q33Y<0&NEde2Kf54{*OE(ey-;WIIGrQp9 zc*0!zrfz3LRM>{9&VL0b>ilIu9+*1(-mWQs-Tgytbi$!q5|^BHMZMY~+G`>pyo8qn zkxFB2-p@7(XX292XVhtu+kkhKtum%)h97CVm^cP|HEVm%p3yBOISyoOty>pdM_=P= z>s%6hGk}*&6m~RwhuK=vnQM`l6H&`1&jiu^El)V6-!X)D?X0q~DqjUERI3c!KgSV0 z1IO(%smBiDbGMWrMm{RgA(i-5EwH8c9;JQRSsAYqKhi!BIz*OBlGCfryk{;?j?7p? z8qt;mV|gqg9rTCl-;Vq3M#r`bKu9L_RYw7yNgBN|w4KigRr1U4&Xs5U4;DcDvtNZD z<0FaLl|3pQ<5mA*tM8gx2zR<*he1|hn)O5o+H0f06f!y;2X3R}K7AJ#QM*h+Ml3uo z^+JqJd<*p}Vs~cc9+VSej&t;zV?RnKE$?jtjpl-8ocmn!pDOm0mM}3@8*Z#+z8_lQ zP8FMc#>O4sCaGIp;+l~W;WfcTal?}>Iek*KSvbxjb;7TcyVV7~=6eHyg3W&DgnSE0 zwZjxz@rY^sYM>tYu?*I)PGWX~I>i6Xe=##NZ+Q<$7Y#6Ba zKeRA<;T57Skg)AFH$XnAQ5_0O&o&& zE-Pe`QsgFauf8;^_n(!4;xERTSfZVHn!Ui`q~*VjX&d0#W|@e^KMaz@@gfxIUZMXM zas%QWOsQ#^ICh+kdBcgKV#w8BDeZ4>67{%K3TiELa%t^7a@id`a3YkS$^^s&y^8fp z0Zym-9=r!%HQvJg)!Mw}OZvB0`2$01=Rd36zrQ9+DvMi6jpK`5mG18ZsfdDO7QgDoB{W$y_>^@pT;r;8m^zX-gs&v6-9 zlp|jo@c;!C*?%sd!G{8#$m0@=NNKLjwTzd&{MT8VnsMBz>|p z5mxzCcHSo3;V-sAu#Pv;Hf7JU)Hh^*;$FR+kINGJ2jMiaVf71sH@%U+Q40GRJS|?F z3&ofm>giQkeN2Yv@Mt&vRuks`Ar4<+d!0oD^LQ;eQ7LbE(nX5V&ZZU136x$^l4i z0v!oWyV6R^XVX6D{vg-4+L2G>q)ndSw#~%<9c$< z&Mac|Z_wAT4ceBRQyV7~_7upM!GhFhI2%e#P6OCsa-9;)9lJ9z005`x1{rD4b1y|A z_+6Tf;501#u^B#-I)fRVKTDD5N4^uw-X@(Z;!WCKx@si|b3v9?J7c^HvIf+Kl*~sR zvz4^D1|-Mc1rEYZpc9sASc%{vVwwzugggJKdBiciy2rMFNg>GF?^TCCP>>&|G`b-A zd3%AXrEh618xQ{t&6H~+yy{?CJQj33!l|lbSz179g@)86?C*i5728+yfi{9CUMYc- zbV!i&xO2O>)V~dl(HE7UT|6h|4G=o@6TaR`9fpk~5aUeOb@79m_0{%VY9XSjO}tIE z7g2k-8pq91X;eSbgZ~Ig^Iy25*mA}O#73ps&(Wy6$wW4H%n}NEb`W{|>xKy6u{$1qWQPc0SPe{I6O6c=v;T zkF-l)ZubAalJ0nu)GY5c8h^$G@OQw|`QQ&eQKs~`)_;A8ryT!yY)oC&$Jyvo=GM>5 zKjU(yo6fChy%(x3_H9Pri%ws_uNVsT|f>a34x_e!*5-{mi9L5WNyVMK5LwnjsZTRn<$-K41vT{x+>E8(nIn zuei*9s!^8^9`fBCMh6{f%tfI}l&;^wU&u&M?hA5%wbhyZ-^0=Io z((>2Ck!YQjqC?iPFi5fGOa_k>nJ-bWbKti!4b9a)%GIo=j>_WH3h_!Np&%=e>&sfc zMTY$I%rY`UX9BrPt0!jj#acq;n=sXP5l%xm^AIKE;DEkNp_bEvzm4Fh6*HrHa>hpR zNE7k+HE?}3W^`+b@5t7AuW?N1QvQAOifU)ZzPdlD)9PNdR2yqPr1f>Abo~-{zi%!P zg~7kFz7b|%j!QznLMkffzcF&LZA^3=BL6m9yB0>_C~8I*OLllR^9KPq$}83+bEF10 zhGh&P>$}tFE>=!@%}QhQk$j8LSG#;H=;q)nbxJA}wwa6!gF-k$n+8uaoqWgB%$rLX za<8HVrP;iuKbLzJX_g27=8fkBJkQ0!!E;-J$2<{a9-HG%8!B5yM$4K*M%nSNGU&$1 zmpQ6{No(<5oiZ*`32ItCGQ=6whqU9cnf;R;Dx4UkGRj^;^5(!1rM`a=3mxqoT z)C2b|B7S=c;YcaP^*(5zm zT6av;cT8lKc&_oFNU!=Opyyv4FPH@{#p%n?RGN~3Z@i)rKljZ-$~y)WY_h2!OCtGb zvg~NGf0LHmw)J8CMNaOS&?2p$W(xbGp1q?W%2e77!J$6FHLf>(k6=Z=gJ1Loa&sJk z_^27Uz|H`fxv=epc{6Luw&Z$F+kTGB97Sr5k{X~yRvEqa$NCJr*`8k6;PpIbqSS9s ztxI`tbiz}f!s8sVs2ILzLd|gtD7vZLwHQH5Jh{*ymQWNodFvET^Gp0ngDkVEw*)us zWT)+TY*f}LD>~j&UM+*?>-MlO1Ua;&7&fYy2J1eU@8m1nroX-a+$uN!pm#bAU;91> zZfgMj_MvkthnB~G7SH(S(3V5Phc<(Q%l1ObisAf5$pr&uKBdZc{d^DnAxNIM)pLOX zr$_a=7{sK&cUwu(>&n99UG^f0kd7;!RKi`~lcAG+b#*_IM8t9#`?AaN5?)Qv=hfo( zFuy|6UCO({J^Kq8Hn8(TLJxas6}qsu3%<^f9(g_T6(~ z)WXmNmh6*xby{41;DON`ks=V@<*DS%%mq4-P1!ufua1AU^16V@G9-9go`?k>#hT`J zOT&h2$9MLq!gGt1l8Bu$=qip&vEdc+DOAT4#oqB&j(Ekw$C&Dsuef|>&bN8~ZV4>n zWb@>xKk#H@?ugR5dDuGl<;Ac;ItjVV-!^h`)ONBL7asn-?BJ2{`pt%dG`IPOifH6$ z7^&k*=n!soLXD~yx-b+W(@5-7W#C%YM#-O=z#jaC?!iKm*5rZohy{OD&E z{Xqjb%94@dpNAU<1lv%q*!b*Q&;1Uy3g4GGS!Vf+IUbETn730up_h62z}XqggIS;! zFiDS#V>NJziX9on2F^W!v&OA6#`WTGzv}V%z2n)rLsF9y zALm;%t(^%Ym;!FG-5>U(zVNK4{iMsV%;sMBW?-r@Uj)VY7?+W_pPAS|`*Y4*!%If3 zB)aJUK`<8?R!X_=tsGBdLa9ibZA3VJnO^+oNJf3j>39G^uNtuxalE{8)QkzFO126^ z4gm!Po3XAi7t4Ld@OG|xSUU@>z2YT5&^^C>ci#Nekwe>BhE+I$ix!1_@>6YgQB4|n z@D>=dpIx?}&FHs~*A>`)SZ#b*-Omz1VgbTP2Y=Ya<~0%Z?e7I58inuJT%n;bzQNc) z7`Ds!s2fA&mI3Z31B0m_CnYQq>}>=dS~TP1hP1D8x%C;^VD!rAz>xg`I)JU8C7i%w z-FmWjavW!rvbju8ffvO9%W;aB#O@=rf6OFPc~ZXQWv^}JK~zC5Cp-_hQxN`bzWtb} zepDPMcGFork|SK@*}malj^U6_nHK_<@GpXH*Ow6mNW1l5PQQ*=xc2oeD9puazp z9zo&~7MgJi`{5)MlBwiu)QEp6Gft4;$@GfevOG4djD#x^4f)BDh{1OfpYt?n9&_tW zR;>iKR)X%iLCx)H(*(V}qS#Z^Z9Cax)8jrHW#@c3GHB)(OIW>P2QH^Ele#5bwZsGV zbupP8>9!)Bv;(P(*ccNql+hf?V{YG9G(^b|#yD#gDDm~jDf$yyovNqfOjM?~j?@pG zPS!FXep4uhb>wHDoKX%bz_Px#9)`U`2O8YrUfYa{w4hf_b)g~YLd`VNl8)Sy;E{Z} zr%Q$w$za+{+hL&0+}>gMoZVB0k&*#h$B+Q=u)c+TF1k>|f`|5CLx^1)>k1m*Fv@S) zumOzTo;x{=IJv>AZBah#r15??xJala=r4VY1Hy^&670#wx-A#jhkySyGErN0d*DU> z&2@3JK7F(cc%uL=jcVJ@I3B?d(Z+#?c6R>n`VrdIPFDJ=qZG?Y`te6Bko;*Ev)aKH zucOmuTOfNu`dlloF$O(&_SvdfezrQ$(dv~5a`(D$_&z>W2ZSsH9u-7v-w%Xsa2>r> z-_8sw=R@9Li?=!1u{nA8^B=x{$T%T(nB#s%4LJ*7crEdoN6L*T2kGzMgmcYx?)};M zCa+>-R2SNiG)K&xB~mbS2@kpE;m)vu4V%Uat+N7o#{W?($-5q#yB-+Db8;lz5C+4_Pht zQT9-AF;LmCvd<#U=gZ_C@wsKwlKVaWZZN!6ESw?{PLj~^u14TlWTZd);MrPp4ek~@ zjr$%kD=x3L@+QiuyUc?aGpQ~SE3jbXW0mu!j{8W@YJ6Y7{uPbM=Hb9t%z#=KE3-AF zo~MX?M<1WGdCb;EVoN-riK{Oc3o3ryQ`EwZk@6!Z#}F zZATVvUeE~fL(9(Pd~e_Sc59bYD)gBJtdzAjzTPCqs4RHb+BmQi1t{+=LkFQFtWNbx+o%)va; z1ieiH8_1&PomeEe1Xb|otRcj&MlzFL3+t`txrP8ye)JC4oqSNW7#l%Bl8n1ArV+oy!p5B%5;3LMT>=*m1yd2B@km2o@ zk(rWsF5sMf_aY2qiPz)2A;75aVnDMk!25E`G2T>twk8`43is;4L7(QkoA%j`M#|Jv zO zMr*GsQ4cd@j~c>V{rV`Y$QpS={kL*`l_gqA*m){oH5TW&pf|)EowJ{&XHJ=Zw+P3T z$5M%^`#Ri3AIfkhjS|+*Uk{ygPQC2`sD80RQ@Pd2+=;ODn`Q{`^lTY3I+3~3SZSgr zwCoH?bn;2SW60FK=Dw{Kal7+~^Q}#35lGM1;)%J6+VN$r9D{VWYdiWNh?nQwC#xT0 zfp$0NE{pd@VD^GB1)ecH%`o=?rC->idSD<)EF%eGtthv9h>V~ZLzhjPb{aL=c*y6B zCy3%!J);XObF&{hNRZ!6Qm0-Z1?il1vWfW05Y$>IcXYau8yfO&Pj1zpfYJQGa8(q9 zr9YWp$vcI)?y=Kne$_7y#e*$TUlaFlsQ+9D0zeW{yK;;Y78aI@z5&o+>aIy|DVQyX}txu)CsyRu0Y0 zr_kuiDvnd_IK_sd&)LutXey$}*Mi-X-lRzDWPQ4GGGcR}pSWofsjtxZNz_5)tZgxGYyy~#{_#38i^(FcDIU9q3TKz%4Tf)l)CcAFuqtO}zo5wS4hvQbd zd-Iu+SHTuO&#|Ei380j=yxCz%E}P8&{!hc9Fr4y^t7+WVkBY_5R(tBM9`{G=%NRR+ z(VVg`HblMC(is2%EFI-*&LN%ZzYHKf*;yIJMXHm;<<9jpCn_q zyLO3jQiZ^#CkbwVRjdbKDx>ss$syNaIhSfcO@5(d8bVDa<+-VDD4v-}^o6eYa!#!( z3p-yGnAjJ^T1cfjXG|_apy%Q+_fOn-1hXc}XKasU)GGY_WrR>2r6HudF%U1SRgn78 zM}#po##p!84*$YkNu01)xW*L06VdbH9A3OAG(0b=YV0z*btsFHQb-=Sd5iJ846jZy z$CrJf*6&S?lf_X1j4ob&#z0EUYQYZngbGyO?g)teWh{&_RG*X)MDR^!Fy%!1ZJJd{ zP);h;cd-d$P^vWa!cZCYRoPjZi)F6!R3G-|xwWJ=k(^k?+M-;QQb~K@!at_o{qW!b ziHNUd`!{vM24o)>k4RdI^W;1@`PKK~V(Y-=$X2Fu{Q1SM-PO#n;JriR6QcSN!0QiG zY`;?Gu(IG}*y%CXq@n-b1m$htsED|T8u-x?#C%Y4PIOf#<_j14#2 zLgv`tks*IL0-pUhvTkGekFk@qC83~OHR&djV<-1yVYs=#C8;mj=QXOlzMe zHLQF2nLz?42RGC9ya2?zk^Ew7WAk`}r8yn|C6G@gzUat{XU+!61~surlo`Dil*Vqf zM=|`ig4}2XISGa+;Wqcjjv8$O37N37{sADN7MTq8zSuV09c0(G?Soq$OtkKgtiNtDg>%Rb)}mk~~?w7tzEy$`a(z0ANlsu&md(WAj(@rKd3B|8Ku zMW-p2TheN{SROXX1h-4kNw7ug3(>=OrK}$yJZMVKw!18Z=dJ{Nyqnv72k9Y2x>~38 zSbq4^#x%Tmj{aUcIm5}`_EC}wH^nl4^IE_F@6C=Z(c6W0o2b1lFxc4o{;U0rdKPN> zqjfg`G5>@hp6C8F)=;7#LjJs9zYaF8H-`3-6*9Ya@3neE>QD<47@HHj?G3S%#&!b@ zYp<2NBRcIXscA!Zu-!Rs#0PxDH&&5>4H-W}Irm3`D!M+1$*aMO^?k~_R=cXsg1FY> z1%OcZbeCCdU{QmUoU<7$r#X6#?j}FaVG!!%w2zqlyekg-ZW{KTJR|`?xOP=p)m(V` z7CB=u)9I^ zTJ3sbpgM+OUDi@>Mq98GLeJSYA19T)2c~06rv+9>qEuGoZq_D4x%$5x-tVG#>_wvr zYT$F&r=Uc^(bwy`-d@b3(Gb;pBTDj&T1Xzw&Hgr8e$hGUlIf#BaGv|U( z(Q4*soX9Rpan7jrUxtcH6e#2luoi z+S0=fJVN%)dmf0&9idqMLUa6-dhyQF#RyGszqtG7-3hTDi7)YFJWD~VzoOIN4WTfB zL+}hz5-Fh+u)dbZ!+xqVnHhBagSE^)PcfI8n)_+nFksmz;J{mj38?ab%Pl5h^~V{w zAgF!TMlwlL#VobU=(2LQ<=aHA(3;e)OI( z-JZi-hmIN4_X4oNgSVkLI^Gt%g4^CGy|T z{0|o3US&xfU$1AerO;6_-3I3jUe*UU?K$Jxf#v=VvSz!A2_y4-PuRZeH9AK4DZ9&A zfD4dHlb~!64GMaJmWgDWCo^srHv@L5e^=r$LN|Ss-ii%JwQ~{XH*e!iye`FWuzw7D znJiQy(KG|Afwc06b+l*BqNJ}n3;arfJo6z$)dj}lMKVMAS1OxO@}uZ) zS;$K@9K|u`ya-OlCUq3lgW=M&sY|gv50~X46U(?A{M_DZ|wQCS*B!3OFBv2V7Jt7DRR^uQ*1c zMVKFt_??ylEDd)vKSVifzW4s&rF~nU9%^I5Cz4ugRWN9kuUii*RS+3)$MWyQI0IkBm*9e`D>PDvGBMX{; zd4;=Y8xfzV92SJiw) zi^bhCx5~)24q+krWJxY5ge;|Z_MLgK<+trVj6E+t_oR{ z__>l=YL)eU$0&S%)@2=JLiOu>V5QNITJ{y zG?pL{3ZY}SAvPQa0@(o$QRsEoA!%rTceb%3(#=vJ#zb`_uJdW#0ZOka)Q+WgXeFaa zsNAz2*4XOQw~(5Ce77~AqYbbfHX?O#Q6^6xv<|_r{De{D`E79oS2mD)3TSb05=}+t9X7urV;&srZC2%C`^l2zfL1@0j|(a*2yZmAxGW8x|J?fnf34 zivttp&Hz0_u!^bD@+MwgYqls89CErHL@=KgF$!ih#_HakMkS_w~@lhg%bH1iO&dL z!7BulQdliruwRi}zm@N6@bo>xUa@H)k)>v}2M%BT*6*!sL%AvLaEz;KnqjW+(2a^1Zi8Z6`zyKrcp8_=Fp+EuzLeboaO6 zZMmy`4C+%Dps*xR4e3Zjxryh78)R3@Z7<$OeL|=8%UN;Vb;L$t*k^ za5W!ZzjG!wI-N)g+$kf0SjCNWWt`R3yH*9+~Wq_C{l`=)F7$Mz-B7A=Zq4}@B*D6-v_;ETq1 zBGgNy9cPFotFwx{jv)?#(G0Cx3{f<|CGv<-ti4xaG4#$c>dtE2w>TBmI`<%Ihqu{v zlH}zoCxwsJxxAWepT?!V5SMXYBJ^xX0^iN{+F}m#Gs`@=oC~T;{`=SSYP&&OHz;oT z?a7LVO?SGwexD^#FS%bm9K({;={>Vm5kU&YR8zYL#g*0K^$N;o>W?)T5)_^>SX+Dt zhYet11M|PD1EK8d%P)b!TZ|h_Irh$cGBtc+xiV)H2cn-~9M(z(zF6LtcRgIvO!O4R zw030n>7A+1y}GTK0mFWZgAC0Z30X@ng2?{H^+M7HAcLJ6qb*?7* z%Iy=*bD({S*7-waN=0O_ufw^6%hbAuj5y67rx*Ia#Y>!GHp3?y3BzrLUc|;YPptpY zTeVkm%}CIUWBk$5FI}Egu-DqKz17#C#aSP)MKVP%ZAl8v!g@MI9)ETB=QR&vz(6s z^$`w#YWnE<_Q*je(D8gB#Nx`@TLm*8ZD8ZcDpx^W(>JvRgs$6vY%SE0X?SO-PKY^k zl+a=|&*NDXDdmGm_o&oh#ISmn{>5VyTjBSUght#$^@J(_x_ z(`fS3@8f!>RvnY)j=xEHWZbMJ>AK#n{)OA3VM}%EzdCc+a+fO{-jf`ltj77g_f+3e;o_RHP2n=3R~AywMz_pnI#o%HwKFyEj-;wCb_xgyIV+} z2f4Qjl$l@z=@qQmHSTPusfOq!?SirIgu^jr z(Of|;MMx;eYpx$Z@R`doCWn_pp)Vw9i9Wn6DY>6YgDKoC;!d(||4Ks`{3l-rb>E0g zk1`Z+Vkh4jwL%6HkX1?NIiGgbOZrg-wn;K1tN0% z&$rrPI3iQK@-WulKl@0;nQ-IfpJr$$^hwBAc-xra+5sJyDYNDK#@kEZQ`Xu-cRf3! zYkgAS;p4#BRcy}s;;8g!@{LImps4y(#@svTQC`+y*QUh>SJyC>3VFzk zPzg#EZq5|CCWkT44F6K%*WnghdnJW0!>e}Fs&@ZR0K@UJUz(tUZthv^gVms&6JN@>C#qv7Rl4$psCdw{rYm+fJ)8j ziocW@3{I1pvxZq&KZ4bwAMADM`}VEvUfoz>D*euJe=({T|J-*KJ|H&^w-Oj#NO*C zF`l18EaaSZyx2#Ea2wo3G51)ydOID&4uK0944gNK$m|TV&ILD`WxYq_AKg{L_sNKW zTMkFQ%r+t|9WKm7X_qMZsWbRnp+EWKucwKY`uc@6=E5Wy)}8426zut@%47R(5JL#; z)nZr7cN}lr7x>7b7au96>0Zi+FyL5|L7j;+P^73MYE!yCpx>|sLsAxI@o#UYP4^Y? z>ZMbP?+}}`8z7;H-=BIf$Lj-Os>&S9l%H;Y(Noq@p7~p65?6tx*7QP#vb*Wyx<<6o;ufOL^{p!`gB?2)v4rm z)qe;oGK~>c(D<;>)c1vL9!U(jCv)$dZL(|YauDt5moH0Arz>qri(_Dom^>6h6> z>EnJqChQzH4X+Waf!0%0fS;J=a%#buUj|$8$7@u)ZvCl1(J%Y?I*;)b7yA<*PbXOm z%#)Kj%!ohuQIeku2V~Cb$WkA;f{*QwKEb453xk1l)^yGxPda!{mo3&}SbZiPQ=4uN zXofS|MjA|8=zNlXrdxm%i4^MD8GfgU1haIV#OekSG95n$`7@sc8$>GfCwz;|H)D>| zHtHwb8Me``DfyxMDeTnEo-{-Fbd#&5V`3@1SUJwJ3cS9|!REW>uQxoDb(LP((=K=WZ?HA{|=?BXTnqvFb7(}JutBau#1HgeTZt+LJ)UdBUh2vfXX z4dxnm89n3%n3XH;F>!f^Mb+jfzt47v&Ehx7wZYBD{tuF<0739(Z(C>WXyI#7D2L8p z?1lG>5)D7-bWD9oHGWpAJQr*Kl0I6uz+H#E?(W?)%jxx9iQ0$?*wgi_a5`1I--@6D z0kH7Kj;si$%;D8k>iI7Vp=@djo|e$uW_`O}I_4wA?B?7ezt4(oiAR)mE!z1nPJZ-3 zX`UBF{1+*%KAv~wx(J$){QY;RY@FCb74s~J7Rb=~!OUZZ$R`{@1oNWtA|9lmOzgzn zyb1@g7SN*f={<>4)+pCZEk(=YR-?n#%qX8*rKWDIt=+^CKl_Hm*8}q9UX;~z>y68L zYXF+=@lneqZ;}Woeb}!#63oTjxQR$IEFF_JH9Vj31Nze8p%r2p$GJuLIf&wrqL_n8|gL8N}fF3dy5{F||k18Yp? zY%IUMcq4G`?f#&jjy6H8UMN>uV}MXYm*hL8A@u~N9+7cneWOC&Td;#EV?fTWRLQIc z^Ipam&fzKLz@2(aK0>I|Yq_a-uuDLk!Y~g$$q=l1(%}R|K71ZFNA8rSi-@=jc${4P zg1Vxbd`VPNS;^?+4t2PjqE#Nbe$VC)qSmW^PxUl04XmAk&&G!VIt7DQ ziyDycz-h~fSbs(06SoxqM#e?>hUdu=>QIO~0+*nndTc2C@NSi&p6{G_?7in9ULCzl z?T^x0JnMLrl_O6tS*E{cn7C$YZJt=*1eRSMQkj%a5`GvkCw1b4ZETjy=AZi>i9i2% zhe;e@#=F(1*V^vQZR9N@^BPolLS2-U-29LaCqy#%H^J1MnTHW@_KJQ~`Dd@<*^S#q zaDAUpyD}|(H3k=iLc+kQK}~-C>=f>S8$Ml{YEIC=Tz^a$lqqm|GME)rgwiXpQ#vo1 zn5)#bc)e1+eFs1`d-HV6T~w}5CX{*y`BSfCm6(&xcTV#oBdfp%S<;8G1>D$u=7k(Saqgjh?U7 zSM`q{hr2$YgC?fAli-gI?y!Lzw|N=6=6=3{?_y1|tA8r-V@fkK^q*3odG}39G9T0= zxk7j!GlhrEHNXS~Et|DJ%^quf4J(BkxwxM*4oJrfQY;?Aw$k}*QVE}JkBmU839j$V zkGhSAPcv~i+*<&Zuz6-r#pSxp&s?rvye(jk@?$fA$dgZbwnsB{O^V)FH2*64spHts z`jmmQF-lmj-pPzbIgjB53eP^~zLQ?>=ppVqL;nHP4<30z3yo!lYS?7D8T48P5^4_$ zf@S;Wcm@31UEIhUvQi^xgcJKI$=L?#YrCa~vC1jDU4-V!w)1}9WT8?FG6LVN7lrMd z?>)W`HLbY-Y_)7$&SqRuwUT8mFKYH$W*_A`k@*;!so9a#GQze-nr+zp+wZ+fk3I$+ ztA2v@O`M1Cn>g3zcE?*+Rzh45Y3|WmwWC*^n*VHKej)!bM5uQXH3W`kNR;qS^@-+E zp*QR;*Rb<`%FkH`=HI-KqTCESZw6)eMk-#ScT5ZIHRQqL5Zrw3lPq(cLYG38CQSKx z)7JvOKz>AcgQGEERhFj%pouT`nr69S;c2p>s}9NEW6i7e-5WW@&T@Sk$m3%5UbNs| zP30u5>Ui)6lrfj&fi`w=Mga@D0%=HMWmQOGkUJTFRv?<|x~LC0n*ryL{1%^`urQJl z28z*B7oa_ivdfTQVr#EZv9QV93kg@wtGzQ87De!+$m$33tJXc$in{+ zvQ@l##_EfRte|)!{sGOB&IGxn|_| z$fQm&|5F{gaklVh{oYl6iHR3-U78HFxGhB>yq`|W%vt;EL=_~LozbbO0PYS7G&*hy z^DW?z?Dklum!!^$#%J*L|BO7{tc#s$oXK>;M{YTY!oyi7wg$n2OXd`Pi_C;W4!y?M zwO?-6Y{iXLJ`?Kcbcm{_%6@Do3(y^<>R`W6cqFz|bxw&t*GXKsakjE)p>mdfaz@KN z3E+r4LP$CEiDrC09KBlGmow52ct%WZ_CvK&1yLVHn6 zku}RFQrn)^z9(N#R7~X+_^0L(2pC6Zc#L|5hbo!dFZ?Al3!1Ri7P-}Q@!zOD?tcY!Ao$RbbEu7O+6T@Me)LJJU*MIh1C?BGQ;;41(`r*NLK%V|IDV^lH?P z7554eibO|IO@xJtu9mf`<=vS4A{8PV+hh4xqC80A{43WHgF1gcZ zU4}KedH>wmgZm3k4Cto3G%KRiXNg!P)g3O@BManeX4t5iqS$k+fm1n_q0q-&(nsB2wt zHjHu@c)teqOsLc!*pFg{i@!3n6S-}y4|Rj9j`7tjUiAXqNUP9Q-PJ<~nv)>2<2dG+ z`n(mWbF2%UaauYjwRt*(n6j#)w}CVPfSn6UBlOjn9K-BgNnXO2$Y|=2#2`yDMQO`y zob(+U)&)$iX)}(vx9fdw7Tv-I_xCZ3!);TGAWXQd3W;qeJEm7{S*|oKWCU1}6!;W` zHz}d4c7;wj>g@G$;EQD2IS#@vrLIDiY_U#1mrR*T4?i=pL>BMv-VRGqX(8%~rB8=s z=qX82R1Y|SNP+aYha|Gd%@@O?(16*bx8(Q5Y)l;s7CnV$6Mir%ti4RvR6OBA!A;h? zJJw~n_Q%8;SEm|wPno9sHj4|(!T$cgtUUN@To77c@40}rlQL)cJfIVaXN=Neicj3a z`lqYBUM@!$1BXQ-Upji45QAc(%M`ixH}xy3$yY%6%ddE;pRECCZgkdj6vdRIH zJF@gDeMv~+ULLdYyh!E2mOjZqc>#w;YTLIXJ@G*|r*+o*AF0pPAY?nI{6 z`d}=%k$&SvvBtzX%z3e-xx~T(jAT8v36eybWrLVc>#$*^)2?&YIOr>Gh1rQ1@hBc= z3y(z_`OYqxzuFJ=a8++M!Rz>Db#$MM*O*9^!o*0VB6atj=$IS2MASy7b(speq7RTL zFz6_jO~sT#W}Mcxbox`47XFZjyR}J0Lv1xB%JSjYCjR#S$bKP^>_Bk-dSp|c zX<@9e=tp)={lmzX6rT?!)#DlTHOuXy0vs<#=H}?V`qgGPwqFngd2EJP-#QQ8#Z4RS ziW<)9$O1Oq=ug6T|h@eS+BX{s6rtJ`PN67roX-SlEN#kf7E{$#AN!lgaSPmNOp9QKcsy~5Ne6&KaM?g8NUqezIxaY>|ub@hB$$`gpi7_!NJ#tlZ zU-fax*rt0aA$g-s2J-#yv3|aN?vDh#7h=cz6P9w$*L$drRlXuT?BQ0M?4N*uc($Rc zRO5{7*Vrc?o-JoFN|5JLWtKiv=NO$habZ&Tw#LJY8OYO|t8*b8)jb$7BkzK+v8|Fw zhu5M;NQ@Bnf{D`8B~GkJfPDXgUkPV-%zCJ}_4tti{5@wu-*)mmcV$Ua^=p547fO1U zrVXl>{-G;M=Ejzs;*BB`V4Lb=6`28Xsms^NGvpn<36C0kZ$eHq;0n4>mOlN?cvVx> ze~D$aXFoMf`wVN6$FDGyj3@u#UK=ZN>J7&|EW2|`wJ8J8?9%-$+(#MZGm;Fb@D8CV z-oq@p4cwwRUq1EY+#QJg{z)?psxzm+obEd54ci};hoN%MbRqq7k=^@zhMVGZ|4tlr z^0ZNPEZ76lZE<;CS>Z1}r(XDwu6MsXd_5%1QDJE5@LuHeoo4osrRu{OOy0qaP4f?h z4w!1l)kLq=)wlaGOqe%!;r+ThAvgtHr9?;7?ifS&9eBl6 zLbwSOOP;!I^l|v2*gP4kLpQZ5>3v9ACPx>@%_^1Q%gyz2+YEoL<;M(J5R$B_rKOSs2zJl$jB$NlW9A2Jdpw?5_h=^6WDIin8IWz0)xe*lN)M8K?apf4S5X_RJJ2e+$PfWhq zL+4M%#HKEuCFY8{;e1cJ#~b-5YPJk+$AfygK`}P<0~_@(lma)SvCcO}7(p@pzK=V# zyZENm5-&^;IxiH=#!kLlZ;7(!4`Wv;gpoV#I=XNogl7f2rv9oel6kXjvP;$ z+f0HO``+V*EyAU~WcQxK6g%b(eXsu1{kZglIo5`nB>4e}X<_ZEwzvuzQ~2}bd-ReH zW6|^j`E|NR=rZ^lLkhbrwJqVyF`nc4Y?JR{qXv7_B3$x=+EV)T^f`NTy%^b~eujfw zwECn0g&hHO^*&A=!RO?hyKvAhM+tn5axorlPc|({9mvhM?qRPBez{-^Mtl@X?I9Z2FPnD;C73F|tj#dI zoW3BPws^V48Q91qJMe3eo?>u6LvUnR()y5a&fhfRqd+!;! zfu?qAbY9!p9Z0Xi;rm^l3=!k~8HY&RtC$1wLlei%pfQ!yb$}Tf---BRujj21=Xy7rVawgEtUb6`DqOy8*eTgKLaF*c!gfgG7#{RNVLKCb$i z?D)gokpEH64l-(LPQ@`SQ8{jbaty9+98XYn&fUUR-TX8V=7MbMT0!3w_F5XYQ&?!p za2)i6Z%Q`E`<+@*I3Cm`+w)N{h-gvRP9(%o%B?n7v|ZZ!eWr489TJtkzu?-}YTMj% z^AQx@TlZdL(xB!&`pGTQeU{WCHQpcdIZ+=tGLlVs#=_Lb4BgtA-e6@=cROy4-EqEl zs9T75G?t7^U|S=6d6_|e%NaFO>!7a?lAl3RUaC5yHqea!FA`4BuiXwAZ4)u^2&HWg zbm|D)4!EoBU)G_*{ekBKtmj2J%!McsY*vPoG{|u46z|&8rOnf*a5hP9$Pb5C_QzQY z8@X?{1@y5FcjP9C@5EK(7D6}B(O>P$2>ZU`(e^&q(wDNYM%LNEL2KS1!SW^zMcE+u3xjOxShNk$uC z?R~ZtxbJlve!1X^ZTe)1F6$3jQ#fwoNc4}!l5y`QkIbPNv9M_SHCBS&68T1OaTwor!Iu32EW3iQ6ZakBY33_W zb{nV-KN!*n(d*3(QI&PIq>Nhul)|H_=k(YY)?U+7?WnbYF`Cuh_IC;_V1?#V8f0Hy zSW9x9tv_}>n)+MsD!#Xm@DiEn4OUfnM(Q&4pD2#)JA+PBJ+Bz%7S&t2)o^zo4HSs5 zfBCCnF8>rM@WOPztrjTqpx^u2_?GnnP>p|e`=hi9+QBGJYkKH-h&J(@o~;~sGMQ*c z9ZgXi{?<=gtZuA94{GF~F11kFcqQXB$quKfxM*BZH=)u$; z?faf5=kX{|{FoC#;=p_8a12>Uv!r=hANbVPp0GS$#bGJqR=-LtROYw{-5wnVMKW?P z-m2@*NCe|G?j5w|Qzv&8(Tn_@iT1{}et^pU= zS=P;3$>-X*bKNj4pN9IS-^gz}o?O3GNE_;Q9)@MqKK{Gy#2KG9(S~CHh2hsiw1m>y zyJ1tLe82)V7l1BznLW~+6Q|kd(d_d>25HMQSze^cK@AfP_-?&9VCaz8uosX5ESdUE zgazJ(Zl>o$axWXbGG>pLYR~V z_%1N9ec2C*{pavqHKH{z`rk>6Ej_j_wl5$Gy-Kw9+HATLfSRU%d)=ww9Z5c+BI?O* zjcn0xg|zkzK-m;QF7DlK{-1UHFHOBOp*Jj4i3ZvBbX7LIl4>Ljzp1h6vv?0b0UCo{@HDq{Neb1&3;=Xww#QH3!L+ca$RD}Mp@U9A4hKJE&P9wAhQ&( zvG0mj#x_eLTg5X#yFihy@WM@Tm8y<6vmWC9VrX1_d*u#6Kr|Y-2*u>P`#e+qA*k>2 zs>3rKGrSWpUp=<<|95}rK?MfUg*uK5z` z2@k-I#$W#NB#N4C|I^am(*AeW3Zsv)|7CSdjHnlO|H1Taod28gTj5Z3m+J@DmJz>6 zO~`)wPi8=lzqRpSS~pi-fDd%*1B$m>@0q6)aXewXJm`1QQ3?=^fTVx(h<@8hGv2BJ z1}bkSkixclzzlJ9W7A~Iq%v>pciAPrVX3oWVk|@*rAQP8tXoP-qA0^w(9N#GhPwG` zIVJ;cHQZV!}4{Y^6 z)e=TOg_b2<16huTg}P4No48v}wf+y+4=MJmGyJlj7)AbHEBdSag-JJsv@;oiCfmssu1!RBb2@yp+{edP`?RWBHJ!KgV(ZxSm$cWSMuYMZ@(t{s0|2 iRZAaiMX23cpli*#9rlP>qT&Sb(!FDNyFk+>_-3c1p3GPmCcL^@Rp&JP9?hU~;xL^Kr&;N8E z_P$T6s;jHkx4!9P%n_liD1-i<_&p2^47!}Gq$&&yax4rCyw*FUw_gP1dVnx6lrVCV zV(K2S$DQuU`hPrIPJE8WhJWmQcFO7)ClPe}I>-?kAZotbxc0bS+H#ZWM6^FF5l6V%5VtfpJK*bX-ZElV z_?Ksiodafi=kF9AjCuvelU&?nIdexUsv?ABilt(0zzn3%ICW7w*yL}KAZUt*;+CgYD|yF?F|$majI zM+_jML}L1s+|LoUjib1!>Z~QYgZnVu4-B;~IAfMGWztX>G6T`f2myWrV;Nnu*g3@y zE9_yhxdX71`-L+6t0JZc)N^z=5^K?9lChP5nOIHl*;h%;)@0MtOMb(!rR8N?dLnF8_}F5 zIzSV)Eh*@5N9z6{O^&uIN|ujye+X%6gwK9A@7I=dy_z`Wr_^XgxHbOZAc<{`f3PPx zDmm)E@xlP2yWHXb@k0wIA3g2IOcA)m?klp+M6|afmMjc=XUGhhfbf#AO1>nHT!B>6 z*S~i@=l&yRH2w-3S|z~f_r?~izkkm$(O;zo89Xe2Hjto&Kr zi=G_O7cNkS(rc4J;;k#frw|E8-=Q6j%IFCPJGk-u6%yJcA{P0UEXp2D`)RH zqk;GRG&{7_=%?~jYe*WI5(e@c9~I+|ReJUnDxL6_H_XR*?x0`fRo29}yo%atY%m#= zhxn}rZT?X_9-zF2lPfM`oe8lh_U?C5ku_p6R`{+}KhjyVbsMpMX6>mHH(y#;^EIvp z9HEa)&PLAm@4Utc>BW*`a!*2=l;W}P2K_C6ZJ-(}4ml+DxaiB`hncj7x9UremP^c* z6VzPY@8AE=6+0`4)prXcTB1D@R%{@3UJwqc_dIO zwGhkPkdm)7svFa4(xoOP3pAZom1VDP#*?Z)_T@lk3;F2h(L?MpB2z+a%*9rctB8SL z54H6qSI1~hEx({PhW!p0=4;kO;Dd3$-Uk zp@#=0*;v537*G>^P01s^Rvsn(&6DzLoNB72yJNqS)D<;6B44eSD2K4DRa~kHgeh4>BmfEm5T$9fQ+p2Bx;GS%yO}By+iJjOG1wIM1 z)E4n(d-F~4;8rq}4ByE1g)MZ#%)Q^M)I^Dtk~=#m)*KF1#12ent!GhB|fn0n}Zboak?eL8*)Go5U~6rgx)ITPM;3fWdVV>Wf! zc`WQIv}GOt;g`hU+j4iZ2yx}p@GCT4x;gjKX!iVAgQwVMT56gfu_#FH@8UxxL*HO% zu-Br!#j(X<{Lh`vT>)X42W=B|>_jYTScw4Q@4mRF39_S_cT0Z-ixkWq@rZ-h@Bb45 ze94KLRJeTz5lQ$TW6h}KjMr}Rw1MH5lth+?v{Vg1-hjhkIis~v2_7444v;L+7MnAv zc9jJ;safiBdUC80tGa$EpjAICTvvQ{gle}K`?|i&O?EXgoE6OV*Rc`Tj9Ij456nIz z%Z#Yyj9;NII;udO11eZP-z>=tmW^)v1Qi3Iu%WLpimHhdDLN}pk4(2o#D72L zuIl?$gHUj4AuT8Lbb-=-C(ql}gcR8DWj#UE#d$KD6qg2toFx&m;@~dP zge92Lb>AD%bX7=it)0-7)WSJX!(QOZW=+P-@{vtZ23e+FT5)Hh>GaPFdQODyJR;f z*7TV@i-o1(-RHtf2Fg>*_&i%{Waa1Ygd1hK;5_xa_&~e24j-=WHEm_&$YE8+O0e7Z z6?hE3RHA{FV&OQqQ6I&ve(iS{s*=4OzDuIS7|b!ivA}nUqwaDI^Yivh<1&a`c+`Dw zgs46&j0k26H#dLlvkU^D)9~0bH|_;zA$+GNtmGgM;)(T$pCZEHTf4@sp0XL3D{Svx z&;(IeSjBn#S{y~Dwm1JMy%7IhyjbM;=UVwh2aiVq3WQMPkbyDBBgKnf7eo+C&%^3J zL~X16`YjHadnYG*AV)fN;z}~ zKc5)!_XiZH7jQLEmh#8~woL_fj_G}%teg@^Ei$?8ODJ@@jn+yNKQtYn`pZ9Yx)+hp zizrqNd1yGSp8s7Q5H;MI|95iubyRo;Bjkj|r3O243U0|x3{o8eKa*N zYbOn+b?}Bt#jhX52A=XOrCfY=nFiJ*gL;PH+n~d=#4h#qEI`c_8ST=ycoxu(Dzx?8 zS``lI`~1(=Uzt&YAXQ?Fw_M}R3cn3tGe8S1xO8d#7o_~-!`#IuYEXeyg6FM+wQgC2 zut6mh8_-19bj~izG)%Uh@^TJPrKXd!sa>i>3KX_UmZb57N~vP(|X%Trr?NBtV`OD1s@`WNd@< zbui@XN_ax;HEKAKRGL8)7C2Q6f5<|6jPe6so+oRe-XShA1wf^LXyl9NJxwI?j@j(b&geY zXn%aBtYaB=e+~T&ydZvW4oHxpOPn)a=GRl=jO+B%`XI>in#j zK~9-T*KUD&3lIBZIy{reLs6LJS9H-5OusDE;VrVI@WD2TMOUy{z1jbAJlCRm4q;0eIu@;ge{hTaNqTZCy+ld)HC>v46FbLS0Z?HJ!`+j7O2$Q&-Zy`dy1-tu)B zJswy_*RAzDvj6&c1(V}#eb0FM>b{nF*N)R2D`X*L@h^HK8A2ncQRrO}0>%&%NNqY! zE?j(yo{t955;=u1RDnmEApt8*TV$KRZwGCfK1u^Eg8VS1C?k4e^W;R(r7q>~0uhGh zLs)2fr4m_>OS3}FlI|)9p2Jyb-u-D%3kv=r)mQp=gh=uP_v3K-UBUym5c`$Os`m?j zS)F2#vC93vmwIMQcxH9{9B~mD0sk`^mq2O0*eiXb&W3dLle<;&3;x(4Z-P5CT$f1J z0Jc6U_%xZ48CWkzN-s5?UPO~3tU2nn??r>wDOSW$=*JLi*MtG&E64npvZUBL`+BGT z`C1j4wnL`i35ltaB}R>~vdB;DXi6+lce?#PSZDrdaEuITG@ijw`LwEXWNhs!UX*xV zcqD6*I4lr#P!{aK5XeX-UmOSa4XtnG^#l$5F|hX8;m(rj%zvEz-6>owSC%1g(yj^o z#!RY_TFAxhD>H~km&uH>r%XEI6{?l1x}J}KH^?@9W*pg4?kjx*nPtJ{$A76~{%^^}-?~)5DoyJ$9dUU%gf`J^+Wewg{a-6YqkNSIJ zp)9Ii`0-Fm`Iil-eC96eNpfM2x8F%j2unF(ATnFBO)IR4geSsu+56pMlZa5)o}sam z&tFr2riGY=KTekaqT~9Cq(w?0k>d!xYv@&B1;K<388696G$c_6eiqkQIFbaUJS+}$ z)-Kdo>^=Z#S=m;s%M_8EK>bMMHdhWt(^93zP>s3P%M4s0_xSxFl5b_Bu5!r`x(L0K zdDFZF7C`On=cT-yb~kt+XwVD8gqrbq%jb_no%0S+&raatN;x!r3LIY#4;hn&tWuTn zEazul%QEk63f(%e|H<%6a~->7za4%JqFs2R$!XnP+3$jewZ0yD5xSr$W|wU#Ch{52 z_PE#tHvun zIt&U2Le_q?8q6T>TRL00w9tBPdHk?Hl~Lk+nL0uHHBX-N^hEN|v}UOTBZ1vurRQo! zPrXo+WMc-}MG%o(6N+oy$T`KWUx318fa_XMB4bU2!EdLq#64|7MPWsY*Ocv+Iceaw z#HO5`@3!(mgpcHOQeFx97PokPl2s}Bk_&mt`j}<~etmk7#hz1d{{l3fz+kWo+;a*K z6R~X9ezR2~cp69zDzua`G-I+=#3Uxotl4(D&xG>ny;xaSwaHxx-6U^1+y;W7`5~A* zcD*AtY=fksWWA^^a*P1o{Y-qEEOQ1SPBC z*=Ii7EnC;=gVLESsu|+QzM+Q4cQm3-RuQZ4@LQ<)uM21meH_Q^3d(M%UtG&Kv9-h> zu`q~3{m_cVlu}N2Z_(7_Mxv}dF<@sW0(sCrx%1E@YPY!K6B_9~9SW*7Es1!|^1{RI z4i8LzSM0p14p=S^Jmd*a=1AH6=}dy{6eZi5EUFmcyv)HH^fFY%rwO`IJ=+c^xEU_q zsTYq}Fs*fe^2~2lTkBx>QvD0@H_&yhwx!O5mcv}SCEg7f?S6_}GNDZg`f{X*5vmzb zINw*4xGWAArkWXS!_a(iSBB=ih?Dk{S;2)BU_Y6eQ8GPi{5>Oa!noTM1PPVfPRo=U zEeu0Wq!zAJ=C_K1u-ZjQ+E;rh6{Mx&jNH&Ag2{j9ucD2psnwWy1yCN3$*!_&|k53F9FH8PfyCfb8+iVHGtt2Z4#~iQU=1Ke)g=KRpTxAEx;PB{lW)%fSZ}k@Imx3<(;;L*j9Bo4AA zTHnoq<#u^2D*L3Iah_V}ZW3@rl+M~N8M`49yQPf9JsZo)rK~(7@>7P()MEL=d&-=V>p_>z=iu-xLJ+c zILHwCi}jv+t<$5X7JG zh1ZD>J1(ZuB}y|;XnkG3czk-KPgi_YirRRef>3%uo!{m)J%>yjkfycuRij9@|U1;t@PG4^i;+QB#EMM0)>$d9t5vbU_&3r?e zSz?tyn~*F3wLMDxHnhbCHG7P8s+7MTRU4wW3?P%|9smi5E;(qUj zTR|N<+)<$Y!ES2cagpSZ)n{uwUvImZ2aF^jTR1KUZL*ZLO)*~c^!v|txYqgth9IIZ zlI+#rT+R#+jpcDtteUx)(Dm80s%3R_a?LL5<}nwlgIFK<7s4=k!Vz4M+Vn%Eg)157 zPJ;_FrhC*+_2!~v4^Hle!XF)1=HeoKKl_n$(4emsAmfSilZ%XByxzQ6%gatDDJxL{ zJP4EPL-^9vflkx%zL2z(6(Qfc^`P9VscgDVQ%vpr(4hLPNCDY&=26 zEiWd5qg(5>GBs&xZRDAFW!s!UuRq>aT#vnzF7bSdUhj#o9P%`uManKGf~hs40Ml}; zfjak(zllr1PBgWnX!SZMKcrp&D`#h6CuC9`JlBI($*-{H*SZXZhEhwf%x2j$o0%B$ z!iY&Ah9eC18GyCvntezv`(#yC2+zTzRm3X}67zQ6^ZU-y zBnx6HT?Ej)Z@hQb)hvZ`%H$O_qvSX4h!L8OLiD6rmEcpZmg~PBiWapSQWbzT;%*=)e04k zsrT!wSb^%!Ea%dMsG2R-Z!k76v^QxvB)oXz3j_sX=IQE~VqpdVCq%`HGV$OBPGfM+ z!^qfwr*GUT|Eb6hQO1E}z#tki^+Hius~BwWTvV+69b^v3e`ebL#x{pp~K zZmex1h`RPcl)F!|#*SM;31Ri<`+s)<8hdz7r^B zlP@HHoY;okZZO;!V((7L2f*f_aHofr;3lXq$19>q#aj_?y$o@LQ znlv@((cuCsj2^b}n5`v4F5?URvDO=zAncaNGLG?{r(Efar%nv=_HO(i&3nd#Qe0Ng z?3X%3uN)Ta+ZYZ2&(m$22|257@KXevG0{)cL)vl=x*b6aPY*? zx+go@RU&NcW75~=CBkUh8Xb(8v}b?8?5Vu1^w{=8tP#$G&Z&IJX&>pEasR=wI}jzc z?2?x)15D|q6hUO7RVWy%FZ=No9$*0vrzRG4Sa9F@~k zkrKuPsNhdCKx+F5cGue>BU2dWb(8^bY?terc=$F3X(WB~uoNf%Od)Tw-1KqzdL}{D zSF1BZpS$%rf&7!QV8aD#MBw|3ug7e8i5P+<}Hd%N%5vrlXf$S>*{H6^ z`Gy7fb7K#8hLKK_Nq2q1_rhkQL$;jW0+gD9?LaBVXgbI!0BVmPnHgHiyY+xd-uqNn zWBL+o-g>#0BwCC zfEaDn23w)`{g0|w>pNkFsB6|@?4(l<(zrOCSkIR3@0UruTlplGg_eJrJ?vj>g!t|F z+oZb#E{Poi5*hkxZS$6;%B0CdRV6{>k;r}@F{1{$!g4qqqQ3dMJ+*s0<(qr>9NQAQ zej$j?ll!g8M8bL$lC=C99_skoruKcu0{eC*TuUxZhwS6U76V!t=p@N9$$}j_sL)3?Ax|}bkbIXKTR5!TQOadeXL$){9O{qT+?Y# zD+=|vuRvF>uHE2Un}{7-u-qU3Uo+V0g&y!y=INA5RXH|#Z z0b7ShGLc~+OQ z$_`ag*yzjo$@4X9K03^v5JP(+PyZgd!+!&YbYH!$oHNXuttO}tmqYS_PIHwPi(7_8|NFr=w`rc+zEN7j&rvKO6YU8c4%QyI3U6;Ge zIWfG%x$I`9;JD_4YKCk4=*MYI?&T(t!GXGu?ADa)r2cgbDOJQ05N0_EuT9a0vM&y( z>h0K}xiYFzg^V`}qa8b8i!WAqH;PfFD_mr{maU7c2S(m9XvmgKT3-J61h{w}CA1Lz z@59eI$wDQi0?y(Pm1V8Bf%#Io8Ymr?2{hB!DtS;)9};lMMKIMYiqU9c@mf{&v;{FT z+hCM?2ud4rV3O>@kd(k*%5+8YwznAF_bJt#4n2F+c=OIrh(v2#-WaV1vGqmp^tus? z9%!tv^+xh;hx}B~>bg_wV7EjZl@!J;g;ifqFKSA%f~K z*9B&cvcD>%?xx&2c)mqx`AZPg?sxHC3?#C8oLne)Y+uAZ&inhmSgd-@d1!gz;I^ZI zKhUrrVxw90AL|A$dioiuSH18Aa19fY+4AMkvH7$aEK;9XWZNSl9vvHmz* z45h*LGzKnm&WP55p4DL#)j)#+vu+0;N=#7wMv=@U8XMJp}Q%m#aFY6!M z{X;b8^TE{fpDIi@s`#1}593M)UfxxxMWxL|r=^IkPEZ~eQ}SC|xnWdjVD`W3cglR( zTVz*+r<1Au3Y99zb%NN!$r%jgOIi}sT^DHBQmw?4mFzT4VY>7BQIHH;IvQCJ+X`e{ zKw6{NPr5EAU0;=}&RC3JlLHs$(|e)b83}U_s9Q2{FFSst{E!%~GQ(=c=Zu3Cgw`$7 zfvlWQ+O7LThGMQslbxg{UhnJGg3Z(n4ZX%9+nQm6-iyl>{dJPre?OTW<1xjwmh-(nYc5PcBRXxNJ%f?`%sb`oi>kB6HR} z+PrB6q=h5K4PTJg^=i@2MwcQ|guO5NqOZ>Z)stvUOL{OVxr&3K#!EX2TD;rrpR&-a z(z;P6ZmXmQ(@H1#55I`_bj>v3oQ$(Zvh(j@de9(VEficH}wfzlcY6 z?FK4mc(<#|SI(!zjqa=TwfO$9?km!$=LTC+rsY3ZN*^HHb_=fh7hnUfiErxJpLMk6 z6k6@>%zPH%tIpuov%7I7#&u(swsWK=?!MGo48eBr`cllhkqsiP=3vl?hszm|*0tcBQ( z+6V0L^eIkcXcZsZf3tf1+Of(pF_-s3@R-h}y2jLiy9 zU#hbCac?(#gm;YCk-WyNKu@&IaWAlw{8+jc1hqz2#lQd9s1)zz6VgdHXN6>M1wxA z9;@YLlQRWvsn+O@Rp1E6&S>Ig#B}$xUGQKisnojsl_~=v@d@fnYs5>BeinCvTeKkl znHqA;10|&iqT9>8SpCg(oR$F&OUmH5%=t{24omGv1JDe}o%N&89n>^-4+~Y$54v#J zYZK4v&q^Y&QvYlK)MSLU+sh-AOH(9XyZo_9Ng2F0tF1rz?6P<}&noT)lGTy|W>Ev| zVL|o4#6tV8@5s9jIjYj(t8(Bwewoq#?6${csYFa&j#M{8uvmKltRJigwRpR24e_6e zP1)q#BTlvOD2EyyT(MG?@G+9#TH?Te$@^uErV~bYWHQk@dA-x*HwqUrL(% z7?vWjsKy#(Q@q0>*@Gk5g^P>vCb{47JkQny#<=Ne`U|E`EC*`nKh~M7lk53UNl@3x zPtP_U#z&xoM{szch3nTpqJ2?+ZR$;Vr+ZNt8n_FtXE)(gw%-_EKl!#EOgkCpT9+b> zF^lr3u~(6D(V$M0%cv4ZJ6{1h=HC8?Ptn)GSVL9vhw_n#!5)y94tV@BHjQ%PKX&|V z^7`u6&z5lhvFyg@g8e%Ux9LS1?}hz>eU-i=)|#ndFh!#%cVxlwZ45zduV8~a6fD)# z{>^;;j{EctNEaL4o7THbWs7T1b%C>>IsKhT%KJFNW8g!`M|Wy%9aj1yYu4G zqTpIDJ*FSUBOPt6%d8{rwau#fqx&MiFO23WVW@xmlgV#IRH&P9=)NI%m&4NNOKG%g z!-~^o2)~vE3VgI?*^(yL?~Q@_OeEu1b$AK|E{;DXI%U!TryBC-YqTO4R1&7A-yOANw2tJQj)w(OWLB_ADwS?=?Lm_SZJFA;)rKjfXQFkA zJ$M|_x3PiirV~Ugqa#y8 z;%xVNfLt{Gv?mlps!xH7=t@fnJDj62#tW`Z&O&o(i7w(oDEyff=PKMYob#L}_=i^3 z9z&xUbB?t)qA8DSle#LcfyBYxZg5(PMAC%Uf?f*K6mMq#52 zpIVbmD*6*y1wd(>%me6-tkx_K2pd$~0oW&MWD+Ys()4Y@*d zp7-?dnzpIZ_JCo=KOLwYIKsLt!)#%@8$H-F?(M8GeHPkW)AnY6NIl(PjE(qlUZVJ1 zlQfydrPXPucq(Rp~>ffwt)9Zb*e2sMQS2<>*= zFMfw?^u=ncbgZ~ig*MHi-j^L9qRn(gVh-L6&TBJvEIWNVY~+At;lT?5_y9>hyV#V zbW^c7d@YuIHC`L1s>B#5qi{D?nE2Ge`HB-w!A+2m90bdVKGzkKyz+fD!1!)%|Bu!$ zBV+?#XPih^k?$ESqHd~=I42&S0~jhGuchNT*+U7e77|H0$027)Xd2{VnLn(~=EYv# z%ZLN2Nqj+x-ai6X$LYSc2m;LrO~oS&MfMi*BP+Im??m zT`c8?7*~>n#R*~eURZqof{GV)&R{94*!ZGC;uGH zc%nG-0uJkU7mgD~8p}_116U{rwI&3m3u&J!UAKBZ7rwsx>%ANJPn0FB#irH&u;Y28 zc3F=|!^6;MKaMOnP5JQ5LgdSlD+BH*_GlrOnUKod5bcF*ba{$^xKdgtmA|%WzPiM= z7ERkCHIH=7F^6GeGddSZqwckL4>K3mdz2k~b_iT6VzV~E+sRHjS3efH4!?d)Pw$VK zVUrlx$IJ#Xd9#LVTXQXLNbMDG^!_}ca~nHn|1&^GXwOOQMN_uwq^Hz(bs7Im4P*%_ySu-=%m68iqpt2*P4zHM_iUS->jM6h=q z$A{DAc9vzd^Vrp&@F%Z_us47l~WF`9ntA9a_wlht-L7HntWtx8_}E1pPc|g{H<2 z=@f#xDISX$mm-K56ri)oFNLQFKh{<~O7Q6Iz|M2#9e;Oq zs29Ip<^qx-!}ZHVk7!br9S7WnQZ)ihs1c{XY;rxvLod8ujp5TG0|L1S7*LA zb(M&a;0GbxCdHI$J+1y*Li_8ePdTq7D}u|f2jma~cB6twzeVUUKF59{fotu2zW*y2N zHPoD=$2!QGz{&4v;^IK!xweTjYEPgh*!Bw*G!^TbI8XZn@mM$bs`7 zcR9}+J=|J>{aQfFEn9XL1I?K*UZIy48QO8Ta50rk?dD44@psF3G8(s{idfxS3K9fo zVgWNJ2X^EXcOFQhH9Hm~EO|oYiN0`jtdAiO_zCGTBZdzWpt8nM#Yokp2h^pi#I>%3 zhG+LX-p2*jmBN}n!sE;=a?TB=5+OB@yI#C-;ptlPElwSn%2>Y7wBRyXUL&}U+)g@r z3nMA2*li1gegq+7%PKWO^BK$QHa9O=U zWFI8YE0!KFB==URjk9Env0anmvxU&wkrGqzi`!;~9Xr^KQRhDh1TXA51kf~zezfhs z?-y1&_Ftt{+6>a}dA!8fQem3Q9yN*I7IlrHc9P$&_qq}5_~^trJKE) zx{4lkZh!ls%J-O(yw&^-T8IkA3qO)6H(z3qXtK#`=Z1>Gjg5)c>s2k+Vn4$8)MM68 z(*j4iaQ9|k(ljAlYXnkIZJ}T>ALXH2A$7r!sG8RInIg}Uzqgwx>;T)zj(@|ag2-87 z`&|Ve9MHN&xamBfJf8k>Am?-akAhAJCSrSLLNPqGoC%SbG5H?(Lrh%%yZ`JZw`!tf z=*9(WBd>sT;#OD%P#I@n5jobS%ozJw@Q z>ZEzT8cBALJ_y=PW>ZT8UkrU7M*REfXPmW`ozR7tBW+rcQ~)D77_7@8CO0V?eLt=7 z-CWG3IGou{r14QoVFiZN1TlvcO#^ZzL^fmwe)ip8)EJM<0-BX(E_UF-TZgn|FIZDa zt&!ux3<07GF#v!Ut64*E2M-njgMS3;TLAFNfjN`d`g$mP0?dR9+=$vX(8lb8jd{d0 zZL#EdOhu<<+Mfp(FZR1ll)_>(aKcQ+f5XwOsRl*3^O!C_gE_W-=BIh=-FsZ^Vy6lH z1?hd83K1uzl}z8ld8JC?i+xjmRGR&fB<3;ph*!WQ<7}^cgbA+IJ-_qsLwg@?^QlUn z4A>G*c+GpmFqSRV!CRM+7;1E^f%*u!D0Khu-E0pV#YY`EC;kV}<^Wj=<*OI>l@5ar zcB&et;Bxd23;*zzJ-fmBQw8zeFU4eN@=NiF{0KUN>PljVC{5gU|< zN;ZS_j1;Dx6Bt6>R;+r(M4>wLfZjQE0Ui568Uoh7wqIejCR>W~tquuTgNZ++S1-0k zPCSf<>8i9^QQ5`};gzfdZDvvVKm7Xi3aGcl|8_D4gCQF)K zU5uT3Zf;v-rR~sK}1N-kceIsOaVQ zr}d2vL_rMFf!H8X$!}C@M7G^7Am~TjL82+8dn+dvuh82_)0xlE{sn`6KT>?mk6Hfr z-0Z5{;-n+9nUCtHA{`&%DE@gac@unvaK53r-^4YD7_L9(@|6Trr5sj^ty)V$tU|4E zIk+Bp7x~|C(n^o}(3$h~*)5N*ZQ`sNAH++)C$i%#?ST>AoS5rEb049~=!=$Ss?)cX zbgA%~t-rZONBc7&e14w8uJ1<6WJw2)^J|l(8%IP$Z;LkubIRqu=ITff{-!dH)9XeX z`fJqs32h=M@olx6UZsAbSL&~!k+&tQo6_a^=raK&BVHcEwJaebP&Q8$Zh=8egT!dR zy7pRUmU!+BP*$rVG2iS_$76HX3kWWy%3z{8jCZUSlN{-BB+1ih+NVgqs9}V^excX~ z4?SPJM8oBzdEmh{5GSUHjoOJe>BljZM%$FatXT&X^Yqr2Pp}W2ItnSN7ia(muRw5# z!wjD-C;@}+tB-l{1G5e{OFs;7iYw&EXD%eKxyt0(Q*=`GL6nyHT2k?RemfP2S_{5f z$r2?H4l=b_OiAaob+vk3PT4ziDL8(MlDs?5ge$i{_$=A?u$mv%qkgpW^i7jYZg4B7 zI7wpK0D3X?KIs;vi|oVFt=|)JH3X;qknS6yBBBUgp!bW!yS&*f&cWOqs(#DL&%TvD z1YOgSG9L^my>M%Cx|Nc>O0La8^16&gINg3F1nfO|RdTycZ%}3iCn( zwRP1JZErPh5w@#Pt21-X+1XSy$wXM{Ky(@Vo-$_dTANcps-N`h z?Sy`l-@`L-x6dl=JS-lFRVp6;{31zE%4XIoc^R)zkyIoO{F!Okhjci#il`iC+LnXD zG-ziAaY_&>L8dBuZQk00ob`dsYkgnE68H>mr|1jl>rM%izcR7oo<+3nE zsxIYoVT_nn0vFMB!?)pQ&j@=a>^o(E*;45?pf3M$Z#Fh}C}!$;+C|#`BWm$kV!-+D zF2MgQq02&pmw9>3#TgvLCHjxDo%Ch0z@9v)gQa3kz{-Lg+~i<#!dB2T%QwIAjdIhb7A+Q+zyLrE{HU}=w zMxRu~Ylt10)N$6~=C8*LmC_+uYdSr(FdOZ7AwVv-au%JeCJruV+;^N__M=LzSE~Tu zB!CvbE4%Y9QEj75kq>H)gLn&&FEpkoy`PdftU-tbec&~$I*zW^E-?ZxX+^p68Ky){ za<(k&D}KB?wG~fS?AxY^mGUcM7d}A;Ed!$$0Iqs^o6lk=xOJ`*u8fNVrCrq*RhD?B zaW`nJ;4Ax(M*ldav4s(W%lQ4eb3cEXgqy=?lx8g4;XoX1DRIW}rrG%yZkcaZB_HojhcE^GM^c*ISeZ@#%0b6&B`ezIFy z_j8Xubz(hms>LPf!7K{Wpe9M@MZTU@D^X-^nBI-uHG5+VO#?toKOA5~Jx)4;-rU_3 z+FGvEU}KgLlO1Lr=f@QLI(LdGC9ciw)$g2x{~_&Wcm43NnRX>~Yl<5?*e-KLo%ngrT4RJ~+Taa9{z#n`nL1 z-q&yy51=oUaMC_itjh3&dBdbfEKmE=R(8c*QN=ftI_5}0WCWB4j6fHP#I4@kKLTo zElz}tVw60Z@m4Qb=bEII`m+0?w}{J4DcWc-nwTZm4KFl>bc)0^ehsdlxj_F}&FAiS zx;T5cz`adTRRi{gFl1F)%Z4cEkAvy@pVt8WY^G*KO!kW#P07x;MF>t8?vN^MiJ)uW z*~1O7&U<|{wO8UFpACBlladyBXZlfZTL~?E;IF57RV;ll8QVqFWh5Y13FkIg*GJgj z8YW|48IJs;S_TmKxHQ1?)~>C#`=X7N8=^r^tg}`d(nAjzpk-}_iuW#{Q;T7%_!n4a zY3V>lq#K24jz9H;k%yvY=y+21EPAAm?={zN;_l3xaVtv}HhW1{AbbNSNheNkEZL)i zmM7JBZeF&;miN~k+`B>|~klrxfvKfDA<1g`7GXm)8;@D}+7KjwLOQrQqZL(!{p(S6u)_MdJ` z6Mm9MniY{P-LcQoe*jO*QM{q2z84ZCd-o4s= z4xz0aDdlH4B*a)g%>Lgr)#%u?)xza?_KkS=karz34e%5!u&L4hI7Fs|PacA*E(X(B z2%3UiP45OPr0D!U7-T$gyb*V(HKVVhq}}D*Vog!hu=2vGGKO_9wr!XubKg4&-8X!w zQjm!;xJuFf4`Xi`*47qv3m?2tq-dd7TZ+3olv3Q?B{)TcJH?ATgyQb*?xeUCf?II+ z;Cy-C@7}-X++RCSe(Wb%d(A!P9AnI}c$WgDs&wdv&ibu;ey4?<5A3_MUxd<5xh~6j zw40*MkE3c%^8ad7BI|kn3k7|cZs*^(NB?aln;o!gW{lTLWF6NW2 zBt}w$&Hq{lR3%U4`6Jc)KQ%9K3kRHJicoZRl0QG=ed$Kfk?5UEqudBTn(XC#^&`q^ zuN5_Xx!3WySnrPR=fvk*01}C0X-5}3+-r3>F6(%lro4b|uE^|LI>-agzSXMzTB80E zsQV`+JP?ATw36Hxfc@6Hc>m6$E(~%;99+no8J+4Nf;PFv_b$?bbu(r3Fvp6+L=)pC zD-=k@WSg!VYoG=nO&0?vWZ!~InjDPh3yYyA>D@R&Uf6D93;-QSBVV;JlgjgJJ?|Sk2HiHH7c~gWZe`;iVm4H>AZCsc zhTJ#b%%a$h-;o5f>FCgXL{GZOx_GJAOn&!KIb45zkgWViCHJgoHou(s zd#sl-CotpoNV)n@l7%6_A&h!K4B_pW*FWv=!l9N%9w;*XL8;{oz+yoZkRger1C?_G zluZ~F@&vPG>U54|X!dz_us1p7j=c7}9?jkG0<%iMj;2c3nT4YDUZKNjPgBTyZ&~~ARfFtV-FF^2S5igtkOlYi zv4R^9&t7ls3R#V-Hb}f`VHr}3`mDZ1iQPO1VFZ;<*0=M&u-+- zhx#(L+Z^8-c6d`|8MTPr*!h{ixCwd8m3x?OvUE6tQS~2>_@!&L8ucQ6PcKVYxlyD{ z+N4X2xHvB1O!N)8H9T_U1bW%qzBgh0;kV-yVEX2w6s|O< zNVW_zCf*=(Mom-RpC|3PG+g^`L`7PMC(%s)!J@<3qwc}@fUm^wP=gWvf_E%d_!9t`_Y~K7O6IWEg`AJ?#0UAvpidJl-sjtZ&Qs} z2+!j>`wsXu^hQ`@Ua1-yDyP&pbgEl+_h9|M3T5gmhY`>V1x z7X)tDY_qwM(=9OvC#=I513GV;@Y!~>M8B@d_SXu3IpK&pwo2Y|wMBZJ6-KtPA2%H> zK)$`ML53-B%$w$)tS>~&aSj<@YS4Z%V#~*yKENDl3qa8#3)85-@^Bb>W4tuI;CzI; z!0q{kO#@s+>eC@~u7Z3UF)!+ z&C;vEj$MZlP4L51lLrdSYX^qKhyBdzUd}d+1Obw5TpSLzMjripC`Ct2 z+>t{cbGKozGHG-VadZ?9(R#UIu-2|aKj~Zda*@%b(dzYD@f#mane}_Yiy`6m^QD!{ zGW}b*NFKn5>T*D#$avu>GP-_8lj$zHrrm(g z%Chon-M}mTkg)YtNwlV% z3J#$umj7_ok<_8J1-j&G%TfSVq=~HQH{p5H#keEVX1meI^Wx~;Kt)<8M#rIXTKC`t zwvU)^kotc9>*f4_{o|JQz{O8wyu=v#?x+?u%}&j?RWY*+>_WVI{^!TX^%qkaj4IVh zxChJop{`WVZX8nMt8wXg#XWuBO!ez6j1>WrJ<6!yEa|LF|Y_AozP)aH1WXUcr$6mnf| z)fPHq65`*(>~MM1@o<}fU5sVapKoPw)|gG`c$m$b-}Muwib3S(-tzP&%qe1{Uv3a6 zb4hMk=Hq*u|HePx-d3wIn}=O&jd7koz=zk4g!eXwb+k_r?9!76 zuRnxbUuBR=nE$EAPVRg=i18fkZK%4=dt4^_ zrJk}r7XrNZR~k}-RSYG(JZNSTtx3$+V%<#~U_BdU0h2ON%V|R^P zHm}=bQV{-4^Tl>2y zm+m01(Sw$oj>n`Gix2m()N%2Vs(>v%bHZM9=Fn6vYv`5#tz1W!Pa%WK_)!;N6g$Ih zw^drubM7s_bxK@a-oRsk^ZVzEgZJR4OnLRZ$41`gs&-F4FSM z!#z#1HRGWS))|-1Qyu>_L3?YF_}oxK_dcDpvz>70+YvD3%Td#q`jIdLDCkA=n-ZOl ztS*Lf=8nGhnMO3dnPur?XLO8aupLpE3_1usd~Q&aTCvgENt*2nYtaHrJCKEjoDips zejHN$YxvLVcXZ`2;Y=F^|12?Ryw6OlJ9{Iyd-v}pB zyr#A}?Aw|~<`jX665lU@|4`K-gzg9vi{TYh8V!V4Mh_|ZyC6!EyCcqQY7giX@iz24 zOYm;>46@51t1STq$4S%7ORzA$h2imsg}|P0m)($-7tcU=z$?WS-`D)96~q~PT>9#g zdCT7mbM2eOQoWEiKpIo7f%*)Hx<5bRk4j$Bk+PS7WHH0d*mnn;AuZujd@;>R@XS1I z#vRqubw~My=-w02>fBksJV&j5Ko%n`(zKf3M%gWoVuUGdd}fi%pE{*8K|nI)&o0o@ zE0w`{{Q2stK#4tFOH@i2w4B2~{UA%GR^okZu-e`=Upc>bUlZnLL-P}Fv}4VScV+Fw zS;&KIMJcJFsw|k-1*zXmkr@Q zSG#g(*8X{e>Bij~!}bdUeV=<|uXWqY{Nk7`m^Jm*Nqs{Gf|7n`u~>=`jgGk5w3El zsSXy84d1LW#T$v;zgFSVQ=wS7?H+^dIjEP;DR~6?BJB_W2iSpRUM@ay5wUcoAjjo* z?o{_Ss_IMJ1t%(27m}vhQ_-Nwms%8UH{6uL?H?#q{Sgc{cDe-h(Q8R2{(0OdT<@IQ z+4?9$G^^jrG*2^1yv_NjxM(2FPvA`0Xg8ulLvBi7<%c``QC&3FB^LZE2cL=qWk7pm zp_J;R;nR-G=hK*P<$@p3n+>QnlmBrZW&2yJS~WN4j%S~WH;Ud|HDPWx6^bi{Ciy`A z%k$*NGePt4+8DDw&>=gXIYmOi5>_^B0nI96mLXzBTwm`fzg+Mi- zD~rM!(}yJC@HCd_6nfz+jRzOB9rB;NT6V}1rrmrj^cU8@?PSYzEz%z{`;wodTfcvc zGpQ+2{Uc8%m5`c{w3Ac+D}?0c_ZuhK-!#4$u)GhJviiv%-SX^yWDxXFfGGwYSnyt1 zV(IZZeH7}$YTHERHXf3i6i8N>66Jv_A0&^Cup!L!M0BB8>CbLdX)+>n7piw{)Qywc z#AT)ydA~bj=zTp;$ITiAP+jGx`LDcJD4GZN#QI_<_=!|gQjK*~Cbxr?tTUuO|AC7> zL=#5k_v$;G&v@txW@+#rH9&?~G0Uiv``_n@BR}$>hTNgkHD^RE{h0;2oyZD_xT0jG zUGb!>)Z@N?+Q*q%5xD*dlfVU14OCZkNE+;oFT0n^AdYG}^h~neV{vhZ!{1@mpRI|E zQLtPUuy;2rPSe+%H{JM($R4deg`rTFd&YKMk8n^&)7Q1yE+Yv&{GAbdczSwx5fdiP zGBeox_Mi1SD;nH{PkrFxg;jd}E0z;bUTh#8(y*kCqeZ=x^sYQ`D^+H5Q;b&knJ1Y} z^FTJSt|}rWry}_Gu6dGKAy>h+S-08p7LSc=uqvY3LeYsLwFUj=mLEE>I3g9Im57Ck_EBdJb39+->0GT&_e=O;*SpqCkFDU*qI?-$PW8p^+*K)4T zsx)7I60yZrnv4oH!ki7Rb%Zh;ZlF=%w1Ur$UXV(+al7ir3~w{36_T?(PfIUheHr6m zDu3JMn(?4G(-?2!X-O@S?&R|Nj~3^Cs^}Vq>>JN9+d^Ck<$zP0~U3z>QF~zvi|8kqT|FQ{F%cyuapQOyI%8ficT|%SO`wU_2RhymPxJcLB zxIZAhxMOC#=v#3=pIJ?>-R{NmIL?Bo{E3lTp%c5ZVe1#ju*FJM0{yc4$I#D)`o(dU zVD3u0{z`gL-Ym6e@hg)}rL3Me8rUG4&+W5)dSA;h+(CvV@X=D~gru2y;EJ^~oY29c zcc`)_5A6jn8cLhdl0T&1#b`?N-zDxxOrN6v8TG*hKfZVOMdIEc1ac|LJ1-IB@g#;U zN*A@6H$1pv;w=AVG-sbpW5UmHBqhfEURXJ6NMv2z>`nGHaU>Q+16YFd~wI z#PV6}o>=+YZIOY2EMJuovYZ}g$k;Z!Ig2?5HOXL|jJ3Ijy6T+SR`Rpe2`X2+9yN<= z3W{5PLy0VQyPa{u?-_x^YO+EjDr?q!s;Qaq=+PCS7XkqC}oS zQ>3EbFI-L)?-|acu<8jQuJM`ibt4~$|}LYbX$Cjg%Q9Nn`Km13!tas zX?NKRi(W%^VcPJcUm8^=Tk6yg(`nEaSArfaM!+i?c zUXw%}j<2Xfev^70cUh^xmsYK^fi8*$DnRNxUT5MoH{go9Qh)1vL*&)b@&<>IF|m5N zY3*gR+`KC9nUT`2?P*W!sP_Ff$C6|+SJQ{flK%R|pJs8i`zK~{y@noc0|tmm)E&xH-9{D%#c zbYc}=PfO%waAZlY{ySH5_BqHerht2P@b_xz%w$p~lYgdD8`b79yTk`-NxeQi$xw~C zdYzMX=Ia#Sm~CnalF3ZrB)uoMFHfp}1sCb*NHQJUrVd@HWOPpJit8gA_P*}HR}8 zM}P+=|IX;u@#BrD%xFT@@b-R1?|MQzM3?ELPm7aT+?OR=$2i9IHOvt;+VC)>*fC!`K-F0BtO8{$BR{wULbAufU0Qo!j5t^#nYR zfMtunoN3GOAG6eom{3;XoaDiyiZz!0LT8_Si+q}K`{gw4hHO{SkHFbdBrv?Y8_P82 z5n&~VoW4DxMI#h9VGnUkz7>}Br|&%SN8Ufxq~N7tm(O-u?>L=|5A|>&rqo%to<*;2 z!CyjZWi#O1yeqjj%i0;aU~_@2BW+Y*TeIt}FV9=HG=bup=5%~oR6oo|L6Lj;A&vja zg#WXn_#9Cmd`f;npK%P=zlQT^eq+aTY&Gm9MHg868UHJ?fE7PY*07ZT%-w*hkvzWx zqhng6<5}^|(pX(T3M9O9rD6 z;VzCTm?bF;)WB;fnfW8nNt@}G%w!RmT&jWLKtrpLZoO0D!;ar8L_0qE&7HF>iRY2( z3WUjftiaw(HU780e{F~$geg2jd(CnA;nJUC^%BA66=M@>U&vinaD74_X?n;~C*ZmUyooa=qu_u<5CZYWl z0@yC>%&nJQAcuOV!9L!!3}b;irjE8)e#|0;fTBOeglL zom$@Fn?nW!iu!NFX(bhIqP&m(FDH+}hUI=hvchO8zdVBG!^aa%g1ruiiy>8YXcB(r zRqv_)Iy8PA=lEgRW7q@zzknNfRwMpDB@_bOUltcE>!Er*^h`%BZ(X)%SJuEadi7fo zJ0(*qTnR%YLZ8M(BTtQ*oC+v)L4e+qCrj6iYW~H0^yNd+1dDgQriO`jvw(V?T5@#} z$Q;M4IQMpkk4K(Y_ME}`NR^BOcie?lt?vY0cuOnmwtc$uO<7W0G)+FA(@&w8DBotl zpQ_T)99AAL(yo#aw)Hi-k>|Z@NI1qnC8S1){J|0%6y*YgG@ z6cZCB+}UdQ3X~YOoi7*-amJA8kTwY=)Ze%=A5PVC8jmHB-lr-!x_`#Wx7*a4IJOV zAYBveT@($@AQzaQZf6*>g)rDrqM zu`Z&gcT9UJ+%~5K+`ccm+nGd8q&G?aYW7V%eBz~mR`RcTy4VbT??@A%ASD2IzW18FIukFlb6771=#rU- zHzB`X_MHj8hoSL9q1|IuH-i${P18xcpP%8)`aMcrT;%NOdzoI z>3f$6V?KP!hvr(H3c6EIq@6}13qUrtCF(%nffK1QNeKG~(UhUxN1=Hu{rq6J?l-Y( zyxOXnmyknp%NIpJcIoHgM{w~m^Vh4qGaG^_)W--Z#B_Qm>H_*QjRVVwyu08PGlX;Jdux8ck#9xVqc-7&Io) zA*Dak7zg|ho(2K1anuhlp6nu^ruZXo4=WCrkDj}#Ade2e@vid~RIVT>b={ z6M8MVw$mx3r0<}^eH0-L7FCO?ZMJs~AK){`mIKsx+oLyWoa*IW(^GpkV24~8Q*;+u z*^Q{X`g1?h`!CoS#hOQ~zRc_+k@s6&igBTTT9W)yoDRO~)$*elo}!akezhguXD3_n zdZci0@tH0_SZf|PUx(UOL7rZW_s+{bUl=-1&2!pyh_TBBlZ}fB1x~QJ`1%#pT~U=MjKYB4fEEWX)vo=9&!L;CMSB5d`*Nb1?_kzeo%H}@Yb!U z7iH4VN3`4LBbfi|^BHfm*a6;i2TY_j_T~9|xJb&U-5JKX=Mr1a9n9G(`3#=a-7hE@ z7AYsYHqQP=)sPqVrFBpCM&f9RB-LousLt?P-$9^5U+4XE3Be^Axcaq?yLw%!`?5V= z$lfn(*+%xD(=csDo!+eLJ#~2gjmVprC(Tz8$AYhzN?ZM0t~P6-!ifo^Jj`<+q5R{<&4hF5tkE2Fjln zxNhDU()nIf!|JjDo@^%H4+b1-E#UtL@28Xw=ly&W&VwAWZU-*9wm_Y5^vHFDEvQ83 zsu*iq%hEeO$7X~$1$v7AhskJMwUe^$+}F>chBn6Yh3Q}{1dhyZH9A~2!Wh>bXd2Ux zd>)|9)xTEdDEu+d_mr}ad}C9)oiSmAV1;m|y0h7CRLZCH3(u1vd`;bF2HhZx1yT{B z?=ZY<@v`9V)t{waVeUO#g1rbW#iZ%M`!~f^9|lLQ_RoF9*2(QTSy}#Q!F5EaT`XqHz7j|g{0cf90Rk~Sq(i^+38Dd+D z{@iV66VhWYI_h)!w%{gm*lItYvuwwpQlUDiC|;Q4$x5t<-Gtxcrq36%+TK!=sT^+A zzw4`uBxmB^3BDh7tlS|EX8#wG`+r)6yC)K>2jv6c)2Gi?HIb%kab!psW)V<}(nAMI zLU418E&1QAo|@5`$_QQgNY(nhZObJ9Lz90v#gC+1mB9H8Z2=XW?cS#zsK=s(0<%1a zDjBEP=z!@6yRe6@*H4p?LHt=t z4y(`MrhfDKBCn!6*=zhGwe@`g_GNuXqTN3)(}}yko$6qk@%xoD>G}wEE@15?*IDQF583$*0vh?hu0Ds?qaHVylVV#z;u&Vnv-+uTi- z0lLdkBfidL^x^n^2p1n?wd1s^pA3>G&5i7+W6C$YQ*-pucN~!skQu9;O zRcR;g+*#Vw^G4o;-$V23uaOf-oYUctRgjUmobuqGnyC0RE8n$L8PQ^7E|kkuOQvA% zR-*r20bO|Xzx>=k3bTPfc?{?-u{OXU-1L_3WgfSZ?;13aLxhIj9Xy}&OI!K-3hSqX z+bK*~HR!Zg|buwN`{W3Fm&cc(Fxw}E%7)vo3=sa5joFhAP z_Gx8F1go^Qzlzw|Uwt(cRe$3@P`~lD%ljkve9^Z#jbRPeNFROA0C&SBeGM|bN+i=t z8k?jVB&badIknJ8vkt;nTZmk%y1iw$7w@wZUn!^B)$+>LA%LSsp8NKSm7_QJoR6gC zmZ_sokI`$c`8OJrW^0imFvhLBM19)(2}^(0Kg6TNZ~wIc%4R|O93%RT2)Ch#OGC6> zGo^wTNf-#h9+Jk>=G-4#qcfEe>d&Ocp~7dtRj zKN9Qizy^26twczC;z#-JBH0?+%0SuqdM3eE%b6^>l`&ZI4<%wxZ85HZ&*uIY;XsOX zl6e3!x9gaYy@sYp$xUr?4aT8hJYCE{?e4`@5UJ?B_|w75Cy}X(0=}4K601+tRXEJ& zxsM{pAKx0_Z18LFNg_2(qxbIzTDay(uzpG|Ew_B1EoDuMmWgh1`yJcIDho|8QT!Xs zn4gJy{bz2YI7AR2uqqAGuu7A>t~kp41Iz(4ufLkK#$Tjf{9AH?S8f5ZraAO6uK$-U z%KvCRD|UQ>Eo!f46f)^4GVc69s$ro$0L(U9f+6ytMVy^wwh;jJHD#cm41byB2FAOg z(ii^|=}6+~;YXgL7edYx=M=%#{~;ur*;0`^B0(kaJFfoze;3^4<1^yI+pg9RZn1lZ|2JvOTKO;L6V?76|C=$W#^O zXp;3{_a%n1tUqo=7xA$1F4(l?jo_J*CQ1(3Zl0(4ASo@gBU7TAgI9xAXtB29YWZg` z@|7oFP004uQq^vfu~T4>mSi9EXWXRe!_*#i$C1(=CsMXX1`fCPKKbNXbJ(L*;_fy- zWbU!GHIaDN(Ncq4{IUycc}3o7P~t8aTpeDWq!aREE6`txaRU*Hq;@PAS{ zB|za{6Wcpf5hc{`WRZL?(g-wLclnf(=-qGfwZ${24LcFZsu96V{mtc$l^Cj}WG>}_ z1uDwuX0jBs%)NDyHH(Z(;j*m$S-g*_)$wla?V4Ck z8Ufa+@$@cbSrszrr6)qyaZcCy)stY~q6Da$Okp(rod%8rhGutYl3qx>wbXB4ljsh- zk*AI{uJ*vW|^;UOd(xK(%G&N^ zx4rc5QWhh7_==uMiF6ue$d>*{o^l%@9t(%Qpy}eGb0Uq8^y2(rYZS8W4DPWDvW8Yxh$Khk^{g^;Lc~HCd_r zigTz!V$)tigP=^N_Qk&z5x2WF1u%zPK6K1A*+s2QlCJD1$W1v1Hm25{FG?P3r%lEeWh7 zl3x=y)VRTb&}9d)G%UbZS1NyyG+NUkNp~HyZ3CIS%(f9gIJZqaKEk-BEhAR7sg+ID z8tQ}<@^7j-1c0#-Dr!34+T&B9`VClm6c*`x0PNrulE*~z!)nqAP8s`ry|kgSMNCKx zpw(UZ5bDj1iE}8ON@oT-W$brbGA@v-S?y(xaiuXoG5ISuXVX())qM0z4{IP3`%5Rw zmlef*G2R4u3N&)Kp+u4_+$Sat&8X~mX9>MJeKh?q?(UctHOYihXif|fO@IA~mHCeq z`u=fuy897yg5&TGUYu;$q>p2<4ux<<-#L8@r^=)*>!E=SD#6tgf3*lNUE4`Q!VbZ@ zQK;&2HVovudJE>*A$ru)^DKzQWGen=Hl~d7{KceB=#(J-no$~#hy~$ob^6XHYAT|e zL(0n@FTEuSbS{@zbLW^%3e>)?u$Et*%h#53JRYx}`qV!8uT~Xi#6Cgw|0CqP znBduaEmp|4n=g5w_Djqor9Q)j)WyEKdIHE-EKw1K*^gYD!wx<)4$du9B-1?|@(*z( z)7u|7ufbwgBsT;w;aR*k{l-Dk5bmJQR@U3 z4b=9t8<(!2hPHBJj=F4tR_MqK$Rfp^^MTR};ii$Mj!9Zz$Kr?vqcDq9V|Kg6^xwX( z>^zeW{(F?s-bT8R>XNK#nd(68YOZ+>tSBxg24Aw`pI0^vuJXOMVXlo1x1i?@V&BW> zGvNuO`&RlG`cPA^e^}Rl?(WVd)a73u$dhvhtd!j|6rPxU`0=z*oMwFl@8st1H++{n z)*p0io!nb&&AFo}$V7;~O{>2wdRZ^C< ztTL^z9xnei?+rS}?_}Mt7MAPUDeQ`2SPc){4MgpZzl?eTjnl-B?WB(L$AXR**%aXW zw`br*wcQ-QH3>cjTQFNU5|CAvj1l3E9g^!Q+@)A;7LKkxva|_6lG+t;>`R6~w@EFw z{?cDvWa{m1Tpze^B$0Z@3GZ3*wM~W0`xMI}r28o(M~Q$gHd;7KM^z_R6vvoKj!O0f z*aEG$Y)QA604I>Y?7!1^qNGvSo+23DK1&$!o@QG^lQP9~MKB<%0{Dp*6X0w9x9sNg zh$}*NJTx$-o|{B3uMj6O^zH@&byzKff$*DFL1K$8`J*E3+8&lqmW^8l$FGB9l;lBz zlMhlWu>S5r;k_rS-As2r`xl4zsm~r<99bx%#n+YPm-q4vv>9a#-C68XTiYxCCcFGn zrqgM!ztARM1Xr&59gKTDr8ah*$@SOTyEfm)pMwT0t!)((7WJ&UUt90YKhW|MwetMO zgZYR_MvQcgqw=%l|43T!NAjmg>SlJe(j8Y}h-bo%7zD)yGbv$| zi9NYeU^G_~_Jibi&R5cWo@|v~73(n$JF(&imiXt(#fD;&BjYW0b2KYWKah5-*Mkoj z;Rbdi3N4NkLBHk{zrB^yTxP=8`+2$>cO6mog-LFpsPnTn#&R9Vf`w`)InJqE`B|F& zu>(=nPz61M?nGM^{0_DNZo=-bDpfa`nWWlb57AXFwdyYuZ$A1V*k+*i$eXeCzV5YQ z6q$Y0i}#(g4WH0@$dRFbsI|@7;275ORQ;{+O4Fsmm_aLwb82^xwVW2T>@h3Qxlf09 zxBitavD^IoZjPUi(8`URz8DL=8H+437#s#b4>bt1)-cxPwo+8M$xsv|)FxA-LYu7q z${We{%!An+K^*S@4oTeGD3kCXqO|w_$Z%t`1F&4E7l>8=bFl|H|^W&EzUd+WhxW3 zBFn~Mc?Lt=MUayArYR@(yW~AOiW$nb>m@%OH8G+lvaH8?p_6Koa+Rw^y9)jJbXsrg zf+drJ4@0t|@K(LL^AHAAID`S*`Zk`+CPMJZw0YWo#RG7e#WaM`%5K^8Id zIfZPq{@-CFT#*uFWo=pq0DRq>F{#z3>84-Yk555i;&@;hy19rf2U|30N1GwOU?#rzP zR7M`K{q&Ya4w?iAKrR)W&vspqz4@TE-rvC8&TezBxl-%%mZ+ol z!&ORUPet_E=0kHc;8u5o_liPlg%k^7{}uAhV=-`P{+BHl6%xj#5Lt0xPerLn0Z?|N z2B~d1I(XbKU#Fli0WZJJNOI*(j` zMN;{8g%hLMZqRda=Lg*r#N! zdXObZCjI`qf@)igAU^rY_)Jq6XZhqS({i|Yo$J%-ffn2I33DFf zMy)!hI&qmBpdy@o8pBg{BW%+XU&?#K+JDJtmC`Z)gt)Mtg6>VdhBY#cisxrgsO-ti z?J6R1BkY&@nSJ;LSM#mVCW351%(^LRcb4dnxlOaX$KhZ*v$!3SQ_p$MUtoWvit z<1sJkeuKB_YN^gOC#|*low?=}SWThoILpP28=VK09v+7zvOM4p7Vg~dT4V&t=ayR{ z#aLhoC|3tu5M2QRex9G$?^&|`)TBrmV827x|VX>hjmV7z(GX2M$JFD>&pi5JXcByzF?yMwk*PJq3Sg|UkoAE(6lVzaa ztsz(<%CY_zd#UZid7zN0CaPo;RuLqi!7z-dA)i`p$dNYIMO&+|wiT;L+?AG++O|@F zNw$;K_?4-4Y&+jNHHSokOoccIowaf>D!QHhpiyK0Ohw!!H zj7sa8Cm+g#sJ}dYR$X+%bX@{!F9e7$4?)W;*ZL~U1410S!a!2E3jsv!p#XanFf#SP zr3(Gg-+mLBR1zpqlXlD?8LRpnysD?Pv%(5Imu8Ga-@43*2R~;fw96k0=ESIHs~ozkh$~T%QL}+j zXYkV{YbyS!ivPA4NtG?~fm&Qja`ZF)Ji97R+=N^1+)plQAgLn&#CT5cMJW{RvZqAzj={5N*jtLF@AHEKUr}LmC=sW3nYXys)C*H- z?Lk=E4MC<_ZdDK1o%fyV(XSW|n^)r&K6om%>JXC{h6(8}ORRCk&Ca}0vo?FqYop2V z$!*oC;e~x2GMJC|pThW8a135ToiB1sT?fD+GRZ6+v_l)Jm~`Pt(cO-8-VJyLN#En1 zg6w(_GeM8E96jv0?0s(x#`KWJo{wmvUWbtjI;Lx!42VyO&QE-3{o3}?oAv|Qs)zClsgyN&Cd>qRGrdRoFUzS7S0VMwjz#9Iq z7U1CH+i}w^Ac5`^4_PKX^p`#<_HcCHr$+qq$wBs(%f&fR$A=bYe#fN6uQq(B;vFi) zntT*e=I!hSQ!N(Xd3iFm9+jxyh8oeu(D&*bHz(1gwMadOSb90$-qbM~ zGMhpmP;7madsFVnJHa&61TWOK%rMe~NV0^c*d)@gI8JPRbmQq9&NcK{%qi-z32<^m z4?xnEDDlmCGng7AGU$eDp{yyKdjiMOaQh1Nn>bYNkCpC@?`yo1sN?jaIpBax=sH5b zjrda4XKQ6VEn^frl{Bl{n??p{9Svebq_$b*bxtquDF1~f7>puu_GzVI^o>U>huD$V zn?68E%WFg+@XZIY4}?%+_1V|Z+O^lP>D|{z;F^C9lI&rFqrmtm7su)8*-lC_K8+wY` z*E!o>cePj2Q3obyXDK#Cx)05aAqB7*|M#jg94ZwZJ!-BSa4ABK*1%pNvjvNfQ$$>p zIvC$8NZ6(ynp`l82HUA%#A@QD_vs|tz-$iP=Xe(<%55_9-eJo!pl3sL*=c}phQP0^ zigmGKhH~Fb?>!=YcH*|a2M4vh)SX&B;)s(Nkr@3e#bSI8j~pX-i-ayen5+@+dy&rQBIs)O#`V#u`@eY0>(!Fi zL(YZ<9TWw)k z-~myQkk^;hO6fbG_NwUhC`Y;i_n1-X_6@s#lvUS&Arjw?Q5DtfuW?AY}d%kz4Ql zk)!;rO%?QyBj!Zs3GSF~Cuwt~-+N1aUUabP?=Xo4d8;M!FMaktv({n0^MfrLw^EMd zMutNLh2_Ab1dW!<+`aR}gUELdq>(N>hg9jelBOODF;t^i`Yw;1ExQ+mcjHhw;Dssc z=I`bvXc1PALUl(NDTOBqMED7>%GZOw%F6@2>f!2bi}7w?z)3gIr@?;?=hpb;oFJ}_ zv+THDcDBxvQ?0<_SHq6+^DcAe)}6urj|6E#>tsGlolV395w!w|0jAq|=FfIJ8^|3* zo*B16l$*U$Sv@w45aXRt_w)Is8;{!^2%F8zXAo1ookeafxY9@>Y8XI^Ha_vS9?|SA z_lo0{ddK$(OV#)xU`Ln_HLXUub!)<4>x@V^dWuS9uc#5pd4@$oKVwE;CQ^6Tm$biz3$n~y z{0-)KxgJIIOMCSm6Y+mT(Mc1!=MWBUO?O#=*}UD0k^Rgitg)+eb!1h`AgzZWnf8hX zd*uj0qjlK|9aRiVWuN|CZA?cih|HbTY0AF2{poJBTF&FT4*O|0qxM(*5*!9pq&2y< z8pvQVb0s&0zD5!9YSg%v1%PR@tMIzyS?Y~AynQ*vDg>DTMqe}V)dG1M+OWQye8#YN zJ2cQ8u*3Vl*0(HVusN%=3{jb^I0Ji*aka_$M*^aR5I6;&$E7H`lgWh_NltDviBT=D*MA+5`J# z*Wt~2CaB|3c;#sUkJ?VvjUqfJ(7|HK5jtuEgp6diz*^_5Tm`3oXvAYPNC%^kCE9^Y zyVJu_%H6hZll6f2*WMbF@Znm_vb;G5aBi9kvA-RTyxpkswG^;yq!Vej zfS^?U(M)$p+(PB=OT#7K%|=K=#4~H}UMl?u#z%i1 z9Q{nQ`FT+82DrS0u^pWVgSzc;EV>&$7>Ks~Lp9!ZC?UvH0{!Jg&MR_{xLdUpSUSoB z*bQB6xhLPiAq~i4?rGp%>s{d%WHX)Pk@xqewWwSUC(ZJ3`}t^uf07+;fMLMvCG1EM z$@VP;#o~7p=BQ1{+CXSx5V={cMZX|Cyo9M`7t{Ih!17UbU^@hhro5RF#gv^4XSNkG z`!s>K=}o77ZvUK}1bF$n^ZspG;JlAEx4BqVn^2eWe3^FS(4dZfwnh1U0Ns>&Uyiq^ z=YCV&f74`(M6`DE&utCb)vYjy^VQz*7=PRYd zt3!vc0i&ZD=A8j%wZWyGE$GDL<~UxP_l&NC_tnw%lqB$dQB?1Ju}fdQu*ST1fkANM zC?MnF)%dx9Z+@%_nrm(O={&4F;Tnu;K)jix%>9ZbZ0V1`Y_W5G+JIs4A?L%glV!cTydm`LTAh+(PUJE*hC1H(ja5Wk@tp0^)-#3Dspz8;DANpcSPD?d)xM ze8RB6boMhw(n{~C4nx3a5)!?zHPg-$nvMS6q!+EnHH5Td^b+s@j2hYN`}H5=iI~9w zMK}aNrY(=|&xPm_zo7eK`Cj77<{&YwQhD-T9M*w?MnQJ>Ic$A@7``+rxpcZYby&h8 zVyO8tP?`J*qzU7>~z9(#!IN?C~QK@~9)$FyO!)_`FQC0U%;*8+`<}h4g z8#lnI5xmn>3xulY93e#65=!m1O}Xy4$Qv}_UqKMGCjI}BVT@!e`DB;GCbTaJW32{P zB=cIkd3frCQIp*#e+e-ng&2aB~`cYemv?Dra8-=Ia`~QE0y;VTeaTE6| zAYIZRr6AH>OAAOziFC&jBHi5$OR02 z0~a&@-^_f!^O^U@a(K9HkNz$_T^reE%wth`XX3{+3teT*8FpHKB|d6V$I@FJk7_cr zq#Ra3L>KK2vJ)0wO(-{)w&X!7(P5r!+Gc&r#BtpnIAqTgaID$W6in52wDf&_wUOQ* z`NQy=Fw-^I_HOrAJ^+*Pqh0^-<^b_|NF0o z2H*}+DJIqel6M}rWtcg|yCr^7vWo2I7q zstB%Em?(AGT=;%+RNZ;kSt$CEjRknau3+AYd2!(CdA;oxJ3$1WyEUzkw)1gjeT)I zVIQLEd{DrI-&R|@-`1{5p?*S%5Ebro~$r7yR_s$2<3}B7u-KKB##&p&^eyUZolCha)T} zk@u4OVKZno6KX~RHOKaT{3FyFwnw-Uws@VzvbD#Ou+s`}xWn(biMhyb3}mt__Ys_G9q_DU!=K--V-b;4L2HaZP*5c&bT3j^CNPZlOjw>9gp)wT+8hNeJ z^QhB;qwzbln5KL7{tLJJMAD(4myF)YHU)ntM(od~UWtXFEO`Fq%FMRkW_AOixGeIu zeeF`zs^uz(EQciHudAK5Nk5^*4DobL^}T5&IQ&xX!EmwicG8{Y`c{p3Y4YSp&*xmd zHzbqA=0Dozj=bn8GXNItmBv*!LllsA(!zrIwJ+5q+#HaoFVjGp;>>bOqr`Wn;-Aan zTIjcFvL0{_#a#41_%FI{(qx(8bJwug+3C9hFAYL>bF$v>_qfoU!qn?u@aS1%2hh6W z7@GMIudjABTQW?`8F`q!fe~!}8LzSxS7+;Q7ydkabgW?v_1bbezdDZpbJZr(jYKu` z6TfFrDUm}g&XD_V8a;N?+NTVvT|R<$6CUwl(v&9}2=LlsQJh9^1erBsF3o0`GIPX5 z`>TeF-_$*;x{!M9pKpY@Y3b1Was1n_ZrvKMyS|@JX^^JEB$=q30A|(n9xjy5)$w)n za=-`U$GTx;=1&~^J5Q0%_EJQT{s6HZKeGK9mEST1kesl%MK|KcfUa1T^8DFFIN(?$ z%~oNBKPurdTi1Z?&{(Xnx0X@r%>tSh^{*dNt*cPe&l@ABDTmD|_r_@^rDO5UxMNZv z8+tRX$Z@TrT@R4OFsLW_1H@7o{G=mWf@8<)!S*mkf=4Mlajq!bm#BSW)i8R zDv(h=Y|o_f=YZC4$XUar7gyI1F=*E=gEvtPo&+piy07OHo|mPiy~Y+ozsrzHxqH<%F;GI>9EX7gJQN=%$HI{d}ew*4~-`an@u% zsUiOjMI?o`qElPV=q7D{><|G24v`@Ru2gZP2XFoB{<@^SBHJQ>H95&@XD9%x++MPw zdCeD!jehNH1`iKs$$zK0PrUzW(Tt}#QMo<=>vvK4`4emHak`B6!xcTdO;1s^_GNK7 zn(AV6Ml)yJl)Y0n!FAzyO;=;mZe29&5ta(ftkmk%$j>CZIN;;(xZ6Ez_4iwKWi!*3 zJ=Vxj5WB||5szzO+>VfwsT)4MWDFAU*$sXLm4=3Uc9fEjx$%XW)=mg5l!rb#XNNMW z&BakQl)J{47n)k{b0$)=ZJxx7(XC_(a##em?RK1SkQ5BB~JPgmbaKlL2lzpRyzow9Rz^X?s$K1 z_&;saYHlpq^S-MYzaP97 zki6IS-}%k3zOM#vtYjZNb=IFMOxTS!)IXEh5q;LqZB?}U^p+v$NZdS%T4dnt7TXlE zuTQa9AjOnbw4U7agLP_sE)*-*{KHv?MCRGePecG6J01If?V8Z$Pifj`9(VY5)?USZ zdRyc8n`yWk+FE7Pr3^`~pGZ>BAD zg1kgUKH)PXc_Pi!Tvold*Xp{*j`#_o} zdrBUFLU$2U*Z_-Wx{KMdAJcpSeEA$8vXPMBR;M`KCDVbdv;&k#VZ z>Hi(g=6(&l`DAslI-u0PHrM?KMvI-6&^~$(enN38o0b&CX1M!>*WpZaAp34_+jB+z z$8}8d%%tA}ajX;HXM9I&p*C~}c2XY4JrL^^^BJg6i!&AL9cJuPyOyur9^Oj>f}J=i zoMP@D>mr7@AY1+8?3>X}d)5dOtKZ{{9sZZSAAjnlLI0$UH{d#xaJDCYv!;s6TJ^Y? zQx|j@B7p{Z@hao!7(2XnCQdEsdoo#7abI3@5Pli9ToV*Hf_$T{pQTrCGY(jD8X#C1 z9BxGH7B3ebctBf@$9+s6!Y@o z+GqFMZf8~Z!e?3{L0?+p>_MyiY-6j}!pnHk(f0Te*^PpLrBPb?)$Bp&$HJIhkN(!M z{yKxGm^ZwbZKzMR%~f69TK5NcQ+xS{ANQ+4Lj-@%YS25c=?S0WrHr$!HADDavaADO z1V(M1z2u>Wb;}jeR=%6|9gU>R+}l%?USHickXJvMAIWVu5_5}JziuNB{Mstx|BBML zam{8&di_e756%f?B4qN{JJ#Mp6~^44R00{F^T~xNO4MG%&JMf0VD;bz!ttN364=?G zxyyG|N;+*Wu31IoHnS*qKI{?- zN?X|~NPuxvEh`Og0w4)3;kJCQ^HohQFq=_yw7(SZg-GbAhQ56#f2VDm`^pFJAPM(S`r&7}F@S0d zH83D%MxaMo|4H?&56Zz3)V+8NzS}VsyuOuFN!5=5HkG%Bw-k*m6-RvMwglh(=oau< zriNXAv5c_Eb+WS#5@j&Rk6&G10tKl1!Phrx-ueMZJerBvIR-&z;~v4_<8whLDgmdT zd}e}=_^+b65-WXbBuRc=Y?X-{Snoc!n%Xw+K(O;~jXD%U{6a5STu}=X9oIDV@V27f zX4n!Z)wE2FGmETdDv}p5(3pjxy9Wu$UlAi%7oq=oxO-e*|1j zHw_iZaM#?_oQd#jbfqhGUKH)pkk<|>R3KKN%mvi=@{n6flWyeSQr6uL)+2iz&}$`y zp>*xC_gt>G*Q*`y>xJxATY7umwP-a$v`$?eK`8%~_UO!h5iDp;5od7Y4)?0Jxj}c4 zR|`801iZE|7NYX_Y{r*4ycHf2p`-PP4V3x?%8(!Rpu)nyzAtq7_jPSG_7hN5+nuTp zgD$|L&7M_bC$lW5H?dSP&QYoJ?fwU`-skznRW~l;uFEklQV}B)i<}+{_|@;Me&A&b zvBJC$DUX&a8HZA3kRVUkl`V&B=E7aga)Y^&BELl=3I3eQD1QaQnlJ6ep{~P}E?5L! zvj?kYTx5ON0Y5m^N0jO-CvLY}D&~@WtS8D_+wx@+1AZ|Nt;Z9$rSZad~yfp*o@Zhz~(1Csf11J>^dYkGQdx$V(Vrmyuc-IA5YQ_>P|Hs9+1lYFG`ntI?7 zgOW6UG9mLb|CMS>`*JG9#{x?p3)zquxtdJzIV@$TJ#GLb5-M^17U241$c@!;F>UIy zguCKJdNtQp4#^+a?ntI;j8&&^u52DQ+x(5*o=j2>&ru^^zgSB}Bvxug6VrBz+1PT_ zRAFNY3a3V!Vl$5I7ss?1D*c*@N;xgHjL30f(ZYanv)Bm+_9gIjN9fHk-3=)`{X=Gn zg^Of%5MpQTmHQbAPWdmr*g%Bv3otV7ZhG-sf~PtlSdSr&f$s6U!MqYcwjl}PUDa^( zT%*0j#>{Ikl+dq!Ev8dl3_ittM;<$;U^|~CoaD|~N{Mr-z4aD7D5FPMv|{<)AGN!K z$zd8#S8KQ9!B1gl-9Fj9AEkC50y#(hd*2b2N;R!sAY(k~@5MESVEO|vHYtz&BS;$F z^V0%JhVO~2>J;wht4;!TJFc*cT@t7zG1R=5LO0U|Y+l`FGxgcU-+xvMc6FmHjk*lf z2tCCY?E~7&r6b+KPVskaTaQaTlS99GW4hi=62O{+rI=7_LQ9<%{*8D6)zL3LZMxSd zN4QOcK|5$?$Fc!T}-Hh2L$khHapSZJ<}(Z@X_~_4KsK zyd&(V7OdTwri_0rTB2GUp17RK^2HH6p|>ev*Wrsr*UtR?-h#p8`M~6KZ$u}z-&VTe zB8P)3S4I@QSQ_3tTDXI?q_L>%$m~5`*yyjxPj?f$vZ!=C2uMPkd)BD_-##9M`*-b- z*puqtZpteqQkm($c!zlzEHO9!6c`=;drtV@U#=B zuJ1DQCFB&YEmQsW&85@aEVSaUPoA&Ms;d<;@#6GcbVsa>HZC8jZ*~FdygJ}GWbF1h zQA}k%y`MX+9z#(XQFiR}?XnPaNAAxW=f@rEi159zi)jdM0`sQ6uvCm?Ig9JLI=S!( zW!iu7maAgO^Bt-LS#EUMJ8LA?o#^5KW-pG|_Y1Q)a(lMmD(!J-uO$W(@A=3UUEF1>c4>n-AHsk;R}5D64AWW?_&G1)^juj2Yb;8KFcyS0vIHo| z^t`=iVt=l?)~Cb{-aise^zQ-p{!tbdcqaOCIixDK%aQC?Fyre^w=z;@_}6S#t#x@a zuZZ^nYirnv{RQ+(u1G3e9-iM?;JRG;=no%4f{7T1!`#OyY(y4&m?Y#;s`$U=69 zh-m-YHekIy4IrY!!oW-kzK(cskoj)OJNn90>D3U|85;0c$=wD-G`V!_%oYvdjlL9l zjyH#Gl*5zac>pNAVS^eqVKp0^fai;PpUm+O`Vj~~SyJj4QcW$)lCQaaw+amnfX=>8 z$`+N!c}t|m;b|Y}Elc74KS|i)wQ{dQN=pc!dU#S9L*JGqUCmoC=L}B<^2OV?e+9^m zih30eml~(SM`Pqhe42}fC%p>6#9pD{^fc0mbNvA^&o_J$i^fppA?NSdBm>4dO$NU$ zW5CY9@1Uk#*o2|CZRFVmzJ%6gk%f9BD)#JF+sWqwEq49IT~(Gw322py;0v`hs2UMMNi!At%uoi&%~m#T+=^v56~NFoN4Qu8 zM*9k2IKL18hU1Iv({2CYt1LI*L(>{+%Zc9Fgc#j%6e{isDs>T*nCU0G*`D7@Jn*^S zP3*~5gbVW;=sw~)qR(xOnj&aHRqM&jGWYoi)a9Y7 z6dN1N6^7HW;XfJc67L|LEsudO^*%M0-uhC8P)-se_`NKINlxjM;j?f2k7{wNpH&w| zZGkeG^#Qo*26ZEet@I7v3z8ROC+CMs+`l&)nPr4a0#5R|C1-fW<4QO5bY>wYk2IND zvX~Qle~?E(!WKPS`6)NlB!%T%kTWoCUl{4iuV4sZGB@DgCW-kp`Zrba*S-?0M7@K?eui*v z{9CRB!K2Nc#Cq2n`gY#yP_YY*R|Q;wCH&hTo0HL(hty_`FVtziHUoxYutI-DR51Qu zF8~&&p1I zrvg+0tc&|AQm}y=@gNpT2_h+mQS|d^DF>%o_mcLt7zksm;45~1o0e5V$fu!pr*|)& z>lTLdYe=h0#P0EKByc_=d&ujg3E~XMzQ3dJSXHkU47&D(2HIiwJ^RPvD4tI6iPp4Y zGk*7JzBdFDYSKbQ%D0-pZNHZKHaiA*!x1YA;t~}I*eB31Z(^DYYo(MKeEQUY+0&lh zpoeI2UlHv5lBq>tl$(Waf6P3N+d=jlQrDHP&QgU-7p$`bME^l)aY@(MWin8*{CB!ERacB@ zkDBZ(-QP##)o#a{x-xq+oQ!^vyO#I>axzTWWBH{;qHfUlJsMuy9d2^s|El1LZ)W3Y zz%UWL_WjZFOUPMGz!fP$d(`aoZEs{X9xKN{ao+Ew&D{y3b|3aOtWQ7RF@zWae&!P36l%#-1cIpie4g)CuX;O|uu@zgIdkTd1k#e{f z10?=HM_O>Ey(HZmy(5e(aQy3wwUPcX7@p=%NSk%7%hM~ILOe(Be|*_ItC6e#{>eg@ z{{+Gi_XXNNG*ljjV`}fvV5^MbXAvN@J^0jH3i1-@A+KmE5vBgyC_n9*XgxOGI&`Z0gP7snZ1i=r2ElSac34PF?auLh{L&N@bP-NfZINwbc>DnRDm=h@SF{8 z(@HmnSX)qt_w_uT%XAw88UEg?Xkbp+2p^oL)f0TZF}g>14`=~kK=e`|ANzkm{N%nN zqtK>MDv}Y6^2G1zknOU)EGtAJZ^uiUqx1i-ZsH%*l~f}MIt;FWnYz*^n5 zO}|l)5^X3iv@vYG%=O^1MT*wud|}Nt>{FeG7~XcbX^jL`Dq`l&TQfGyT6xz3W$4et z^^5JD6Kdf(Jm-<0IB$eu3?kJ1%pHz+-G0^`-&?tTK#R+sZw>bMH6CY zp3LmfeilFaxP?IoaU}2$CC0BN;Ql!HuWAfn?5{&zy;!e{SkG`01E0N9wfC$#1UM=Q z^f2@UQ*d?@J5Zk9t{)BTAXJt!QDU^~H3}%@RkN4B?`{rCcuhS*$_U1bpJ5@ z7_kuwcWVsEjItw48EK`N_0U*!XEtjt-fr4I;0r|aydJ15x><*JySq>G93ds-uQy)@ zy>I+7CgJfk;@jKpw`C<${l~%|a`z+^rcM-A@6>z5L{l|&A=?X%>?PBpn?6^?&gEn# z9{v^t2Aiz=yZRa(BM)xXctgLhKEL4jmLZENu0fl$&(0X-$PRH|aJ(;eT01puFzbS= z1g>A*2L+by>0W*bp!#3Ki8sOmrE!VwJts*vA4}c>sJE8BSJ#2O-^Wtvr=M~IYDCBu zrQ4qh|Hky=-vIHiNiwQrLEw1(@vP@|!F6IJS#np@_I&71^g1l1xjrYcP#$B1C=aDz zEzKADk0!g>mj+?ni@nixi%|-b^^PMB{WW& z3pewWp@R@W&wLVVlw{Z6C8C=Jlu9=%b5Vb^5MqoGGFt4e0TGOWgLGq$ceY~5EW&Pi z*|rf+PixiD%xpvJTPj*U8uEtjj`%`vrdZv`tlAfjlIS?6`D*HS?rOCH3$j)=$+t6; zj2RKn;(_oTfwi1RYQL0Md~{ZJp%=R?QtyD*<$Qv=bhoTt(ewvkoFjtXB#fb(w3(8# zDf7*KiF4-n;Ons%V3+yw-0FM!7*%DTx%*sSj%eY~JiZljqVA`jQ+!rkF8e8(Qn!5@ z{g2FeE1Y>AxefWDukJ8PqgYaeA1NpLq^P+aGLT6*mP(?+&kIKcS5$*AaG?F8;JF7H zvFUNENd{73K4#oiCo5+W44F>MQS7i}2R=Ee?#6ce#w2F*7o#9Uip6B@tv~aR z;Dr5tg?gkRheTMaGp%A8Hw#eZltZ#vWtEQwk1HNTsg)dtQ(f2AczZ+Hwnn+VZ#ot2 zh%^4J0RJZ4{Zmwmn+H?BoD-{^-sKmG|B-4D>Hjy=)EII($WoL5Os^TrYw{2LWnOQ@ zojb^YuDRQ-e<#*vcl!(Tbv~8@_A_%(4e+q3+fYDiD6{aQ^FaO&`qKeZ zdcmzD^#v4yUE~3+b}C7Je6g>WHg?G+{;!swckn9}y`amneSX)eV1DnZhZ*XxZpTe| z|LC1vp8Ir3GU{R5&;1b0%u*Kk;g*aP0{;tpqSfO|P}YB)l1NZ=wj$NxKbSfzePRM~ zz30~IxO6qUoih}05dF#;e2E_4d#s88 z(q1Dpgp4ovpPJdf(^Pk4l=k+o813!(iN2(#F+)~Xa*z!YnFnB03Ot>_!HzdL$WGvF zmB2|=;(9Oz{$yJEik@l#I<-Zd=eGrsyQK<6%?tYXSi-sWmCcVRv4!n#!w!;8f_k%=Tzx5@b8`f z{>6`_CUuAsX=?I%4;E^t-jX)4s?XtNYEhjoL45F$^GO)9*w?ut*RZ~_M9LT&Lpej4 zZY?^x{wi^*Rr3f&v9HDX0!dM8Jw6qC*1-22!H>wxi?|r_Z)>l~AN))2a9-y=l4inW zpC3B>Iqj}|Of=Xn+C-6T%kzKG!lDogi^A1@By=uXjzSCf8vVg~0A= z?m>-4(rj^zUYIonRNE?FMy(Dw9~rL-?~Qu`)QL_syc+#~tNCxIxE24UijbH_b0g4~q`K_S>7AX*R@TOybm(QRBJy5c8Tg=oj~d-)FQ%1j z(7^KTi)E@Oio5N$9Dpx=Z3yU!E0_zZ$z#|YnNZoaoxX_Zre!1?Ars*`q)8lcGVit05l5<&UzA7K*ROMVrP0$ zGRQGUJV*S02xl*_6azo10A#toJy;e#J{HQ3IpIA|?{g!Y;i4Va& zNBT`2jxkSEVg2@e*8q(?kdS5JfY&)#f@{QoZ}(9Rz?23+I|Gc>;Zlo7U{FRL^0b=U z{b6Jg?8D`rkJ(d^!qgmZR7ax{&1-hCnJVbE{~Bu6iLGnsI{L5GR1mnJ3=IvU3hnR| zX1Y6G1)P7TpRwCXza$hW;p*xk7_~J;DfE#pK5iOA1Ds@b>()~otJ^l~GUx@pj_siq zZPZtM7Rivz+M$f*6BVJVO62|N7dkmIQGHt%a4pcgj^;7@WaJMGjrDjFadw-Bcwe_Y zH^Qu)(T^tx_o{X^C1}GWnVHu()mZzE;YS32rN8}<<9CR23H$R@#a`Lq6G*+);P!EL zJ3)4vGkaO>QgS7FE{M?Y7Vtv*#BL`R1>AOgp`l)kb(W_Up0kpnIzCTy-%LIg1vMtc zXF$Gr?@{==vD&t(I#Z+I5aHx|hOoTnk6l5b?gQY) zluvvwCOJN_*LUR-*N1~FhqI;uq)y@|n76^4Or$5bJLk3C5$vI7Xs@TeW$l+*mP{n_ z$j!naWT0Tt@6t|1p#N$rv!LTRWnZQ)~v(@^Gd_rfjI6qRiv@&+O>0=#594~R^$ez1A&d!lc79@Jy$*LIJ1 zyM8MZTDmA#uw7=V0A-Bx287%1@*{7|IcsusM)GvER3=b7ATx%>qbD|_Z#5bo=m@7# z>du8Oz?~zUG7FE-iHWH}kU?L_;JaMfEXJtLEb7LaLTWM1B#b#_4B?a~WuX{{<#|TF z8)~RjPD_YVc`#ieV^s`kLIaJ=;9GB8_7}U_(T+Q_r{sWskEa$zzmhcPWHu3k18XXn z!YWvP%rUid+eRW+6CJk)Ue9Pf{9U8JZoJJ}RbJtJ;c&5ecn?G#e3f<1*#};UAc6n_ zew1yK(nN>5!^nf4IRTen{X=T6wr==94e=U!N`EE)PBX;DO&yJ1o|HBm{3a;O3l`Hl zh0h}YmOMQySjUFZfS&$?I-B&Ke=)u+!5{dyBSaZXeo`q z&j9@7blOAyVgFfZO5L!e&@~dUaxxP0g~saP#9(U>Et;K7w)(p-zn1RvW-WU--9v3J z^2tYfbca&$Vbe<0E45X`XgFe$v^#a!jt*s?{~~~OI}=jZt;Alt)sHBmf1>#Lz@3Y0 zDSC~hYTW{W(0>Ua?Fa#2hE)vjFj%as3EptpspS8s)$er1G!Hqm?P7%5Rlc*@7lQ^N zc%fl8Y-aw)$A6Jr@lr|)8%F>iT54>F#qf~qpHw+QalSizra)lS%2l6|>FFujTYNSC z9Co_Yj5-r-hc_-7z;kv?JubdV#4nT8G?a9O3H#DzS{kyTukeG zkuYNi$YpoDsJ<`^KlHqNWT}VH-yx6MgY%Ue>|2rC9m;FWUz66=TuEH_t4|X#kN;_$ zQLi!NG^GUNm^F3i7q<>=y1jQw(N@C}u>K(qjZTAnH z5xPgR*iUu4amd~l-ZC4K?Rz9art2J!rF}hlM}*DihxU{@lb0d)xGW61B4w(1rzWa@ zMhE>;Mp@DBLN{uQWM<8NYzgrcfcfBZ=qqSKMXn>vvnzFDs{ff>UYJB*$UeFYwHQ9O z$PT5T)+PE(_m5hr;{uE!Er;Gxl<{K8oP3_j=A||4=1`OrAZ{Km+!xFMt9bI7(a09! zhNf_pOt`T!U9ed1S=(vM_X7AN`M`>XMbU=#g)vztve^8S0PK2x03kmwfJQFYhpw*w z?r=p*-~e?OkFdHu4hh9MBfl4wt(mGT&dzS{O-DWo|;+$}F&icbQ|cu`+LisoqpjjW6f zV&fhc>aP5!6Po;AWUv2o#0PgO91n8)%q6yaU6Mkne0l5pHsH?~syjMaPK;Jc0b0J` z5mLOzjabwlZeqLuw=GqgD5M&KmnvcrX8_Po5-S*;;dTrCuFW`MwSbodFtvT_Or>a) zrbKqQ=Q)~(k;T~TN?iFC;7lHfvOf>RYV8<0QqQ2P1&{gH?|yAKjUoMZz@b#mjxMO&k|M z6kG}ck?zwqgzEytk1#djWLosIeF^ERY3jtuGG&;`WH8F4FgTk@)vAf5p2vx#&eeiB zTQby_S_$#6rhsY^I&fdQeP{m2-wA-j&nKJPhY0ZgMh8g-vizlTy2QuGkO8%7kVabs zQ)-IKB;inZ_c9zF(aFi)#;fl!C)Lw~p)Os?lMN9a`tNT)i0lRt^()6A z42v4v^Pk^^$nRp-6R}oobCidB_hAY{S*&TE`I)LV=t^*K?GlACFjU75jDZ`Q$D-Mc zt8L88Vs#LLaFIseB9J&Gf9nbkt2QFhBL{L}Qh}fRN zS7UBWCLKw)n%+UZ=)(-QAeNr+NMpLn3J_CjV{NZ$Cwk0K5Pvx$ZuRhMGP~_#fhPvD z-##xO7t>Tvd3*|tbZV$2kZz70JC1Rjw=&VB)gn}0inE$w8HZLQDSQzW^czqEeQjMDmdh{0;xGhvc z>LY>>Kdj_D_AGAKAyW+Z-|!#>(~1y}Q7HIdlRzX7v>0>9k%jKxcc~*qmXDQMJJE*9 z?S{%iRzxDxhJ=C^5DN;ra*QuFpZ>Da*``nZ&WMuO6-g)P=y<8{I!`6{imE@kR1U-3 z0t@0V;loJltI0-cdx!Ze^eO8&3*$~Yfx4=SODPx`9{``-0jZ#cthb>(CQCw+W8c}U zO7a>vd&Iyv8r@Ha##n(F*4VtmJ!Of@OcW$mE#l!u05XG1|Los@?0xEA7R+5ynI@ef{dH`I^^m*cSJ#A)k{*Rf0@j8jX@ z%q8eu>MOL?((!Q8$dSHKI~l@j>MhHMZFmo~!25ezA1s|P0?4QOP9^h$kjaW0w+D;$5M!FP z=z0r(RxQ~>g8YkQ`qK=rx&g;MZSX#g1~|vE|9N>q5cU+{kqez$ukr&)l-AV%dnJdh z93kr&0QeoZ`=xl@XeA|xKQ~%4FQRv*jUMb5^3(vMaQW@1Q}G1{gHxNbmrC+YdVC$NPVT4M2 z2-4z#qMOym{Y>irD64;7?uex$A*>m7;Hj@<(V&22tGz_kfb;|Ava|`rwxB%Wx|#!U z^gQV_ic0EqhTF*ld1cR%S^XlTwm7c0RL8s%ciHFDl9G|U{>5-vWMblvA6|l6U(;I> ztM)|qv(m@(XDQHxuy?1Pyhd7dDTYd;TvbNKU$j*6!0vR8$}71_R657l#NMV7qJ4p> zsUE=#$511<26awFWOSZkZ;fwTO19e7Yc%nAW1_?M*uTBmd(j~6Y(k(}^c4)#CX_5e zfw-}vboc~do+Xd(+lDj8k6u{Vilur5#j}6Y*!r}6Ng=gB@fUw(;U6@%Oa7{Svf_w< zryO&2p#Y-R9<8UR@9{gq8DA!pWKFB$DUjO~2O`ygR5<`3?u1Hjdumy}*H$olR40lQ z_k_<-lGYY9D$>;s{;u9QiBPmKBiII-61?;x)xV8Ec*B6j^A~|tAXyJj%%%{zFQZz0g5FZ}pwUOUR8RpHV_#1h z&>A0q)LM9JfHS=D98_oTEPFz=Z$>X8&uH$wA1Au=Q7cG^mV~Byen#<1SeN)t*7mNg zi&{h3x*Us5))drt*{q7~Eu*VZH)vbhYA*lya=7#VdI4Z=#YtK=S&kSkwAON;Lm4%_ zM40+*SVHf6&rea4S?F`FazUm(FY?G(o9N?(sRsAorXR*UD+tJdOkMD%f+PP&H6JWQ z0T*Zrv3p~N;Nn30S8h+HV$KZ;>4nwD^*_knO4Jv5&%aF_p$z|$CWNzj zOJQZ?vmjr1Mt22M;BjW5)=$UFCfOXiNIZqbdOr1|;^bPK1(FW9-}*q>bfZtGw^oR= z?d5pNNi!$1FP_mYM(*uClhoWXO11Nf$CfCCvGBCxY`6^it-8Zr6QL0r`U3L__9 zO5yE7ox{F`NJmktr!KF0#JXx<2j;gOh>CfJydX440kNyXZ4B02&c~@n2IJJz+@@=U}^P324%sNNx`AwEm^~&Tz;MgN3HE$ z%iZ?;ybTiy`#9b+(evM_ykXMln0`Z+a>eqN0SexK|L@QurP~XG84B5u7)_V`<(NJ7 zRiC|1d#}oHKGLl&zV1!8qcA9pcr`wc%Ddag+aYK+<50eFc{MtfQFVPf4Wfs@CZQx5tw*)Nh>R z>~gy&C$c1TbM(#mWq!D*|F)S*1g*_Gjng?c0+%?!1f0HmR?2p4{^fUQXq!C#7HCqx z9?3PX<1!<0EMjHMyQD>*xfXuxUOnXN)BximQYRdg*7~vsspn45aW!>S9zbj--q7uR zj_lbvoPL`nW2jI^S; zl+)sf4y7zVLj3NEw{HoSC@F+Pm98GR(mOyRBVl~5?gRodo+dg9;+E4+?~f_w=WkCB zx2~8jE||03HDuwT)X0XEMpypN-XT)byJBdi;Ae4J-G+au`Jm%}U{UUOZAR9jxz@JxxJeX zJ9)sK1Vdt>=8-L)%ua=1wV0IE!)!z<#oJAdp|S@{13c&He7l zPI9-Gj4eRr69aFHU!PdPAcxav#41&X0X%$kuTv_`k~Y6tE9`uone)}TD4Wm2&w=&! zlI4^m{h629O(BA(~&`8C_pJ332tEiLtupEcWy>o)%52Rc>)A1eNG zil|qw4A>1QZ8kjDuG>c}$SoE*eC*4>%04=~z3~awI~VsZK%T#K=dkT&GK0i9;Nl%U z>nn(H*xOoB8tLrgKbDv_D+XV;z%b3&3Fp4&g*C8;oIWVDL2xJpxFqE#Ro2H}I7zrO z_@*_R_9g&m>Vj?dP6rV8XPtKc)zyj)hfOQ{;`7HzjR+ZAJ)$_KjQ5X4>QSHHePhsg zCS$Z44p|UvzE7ZDS!X_-A`CcHq}X9%VC+{^fXY6cKUm2=U}5Ll)RAj^{rKUjR2p)kj2fVXG7TqC~am8YT4|3Gej5;#?K( z3yGHu6non}X-Nnm!MqzqB)?wWrd~@(mCQBV(;1`VLwh#2NbdOe*N#FwRKUcIaG>2z z=utc}xU^V9rpT?~3bkE301B!HLl(A9PnPuMk!zR0OD3EV|})XBe*1h2YA zO6&y^5sbX!DGTy`|7Z8wqvp9;HIAG6ch!&=a3|ym8qmr-Tux1X5uBbgHQKq-?1(hw z=h{S)Zs3HKrQ|{S!tx%1gJ_a3rek|I8m1So^KXJfQE=uiOP_Uh{3_z)Hf`zry)P8`NTkVQL3OXf4e^F(<*1A&5AUrZ_eR@7i?6`rs@%G;S+T&#ZXIk^P| zeo?|x5M}ub(|JOLj?Fzkw12=db9ms3E{tZ``Cy>;W`UGV3l@g>i`|Mr#mPsrT=AE)NSy_#fp z%-qYD%F%E#i4Agj9SI`1wFSO3^ABGDZbT|C_hyZ;&Co}2Z3$TuRMGK2oVr$RPCmKQFHIC$f{? zl!M-oHIE-oOv{s+d>3u)i(yMA$Lb>MUd{#Ae~^7ntB?=}xAVrm5Drx*T*SJ?l}AV_ zSk8FY=hztA-&ae*J%l?oJY+eAIzrc#naG!R6-!krgv&Af_Gyk-$^4T@4P6P3{Wk`V zSOwUmpqR_%@thj!*!LoBuU{O46vv&owv6~?X9^bZ*TeMwbohFDy~L*ZIFnP)cx@v6 zdml&gqL{ntn-oJ&;a{!SG~?;ojUC#_8NN3Qdt!QSLIrmkI2;utRJQSV=b>dE$jsK^ z-X-!dx2^07?arhCEul}d1T@a>&b`HOxTK-bRN( zF?U{gn1#*6Am!!CRhJp5^q2*eDEbN&w8p=G)R0$_!sLAOfy47is6{aqELbpba{*IH z2>ie7<*~$v8QVIRNu>rsh663w|mCWaSq6>hJdJReZKCdy)F5qyMRN&k^ zcj^Lha+Lq|M?mM&6m$0C5Hm7-c9I#LIs)+E^Pbe;VBXV^XAUQ~UOos(%+o%m<@D7z zqXAcNTtWyX8E$G8M7pCAZs^spw|?4JK*t>*5>fifU}qKjWo@%l7piiiD#WCu{-kFB7S4 z2)Po?JgUvcO_p{rv3-lb$Rsuf4&vQHI~x?-R_r#&mM(5*s3t>Q8KW%UN^g!j#SrrS zD*;6xSlC~+KfoN@u}2-e8Y6tgopy8V2M%?WoPvCO>SwQ-2jPAId;G@meL_udT+oIq zv4vDm-G$H9x)+b-1ar;*1Ox}4p?4c1ipU{I_uW@jCO+GJa?!ZW?y?Ng>C)w!zZi9j zVHB*I;+D!1I-E0~*kAAq5M-p(iYs4K(e7Aj=X*jTRQ7n}2n%u;9k`gUMK@AY>_>Yp zd)N;$Ziy2{hir1=`uQ2V{vXcXIwM^fe5qw>`c8M1|!2j*y zYE;>GW=cxcQw9UhJd&K3<5tnDG$HY0SSCfkAEZJP(N#U=gOU()cN7k;X z$@tuTKq1CL%?CN5Z7g1Qx={3P>}z(jT~Ypr%~}1ubfeK%Opd7Q2 zb%DB&$+Gd{mE0I1q?#J22&)Qg>BzFJxk>ri+B z5^+ws2(5Rn89TMprB7*6HyrVo1eX|&#dmE`iE4ivQveXH(JZ zzW(VONv)JfEvez)v%r3Q7VbLz?v>w~wm>}Ge*DzUT~mW@&J@5@lw`#J%f>Yu1IzSr z+o$R=pz{Rj3PdS--bR8Rk~mmlz0XVcRk=nM=~H11y9aW5a@wY=4@Ii&{>%zka^43L zZF;@6Yy&URI^M`Nep*!K?C1&qz7sp~^S`bEcV7Bx#1Uqv9?||TL^YynuId=?pfbXf z(P1TN^`TifItIE->_*?g#nxf_`mlU5IU5w!j>_Iy*{jB5yx6}~^GdVBYwp%QxPBLq z<80KJ(p|ifKr*=2h?EDm2Gw-4-ne;?sjDyM+}~~!A3UkbozQ3t!*zNRWRYAAGeBkB z2@Fh?M?1_S0XmZODmiIS5>9DyEU%PQ41x<-=L!jp zdsOAb7bw|bekIuXBlfy;c@8R&$W$2)4;a=epp#%vG+8fV0B>IP4i>tqPdrPo<{Gog z8@`?C4@b?aFJp0vrm5J)7Pe2N?SB%gZ(^^>+4Rj70$PaL_SJJa={5d-zPvg=zsE+3 z+Kj)w6NuQ~UUSc_7=o6U41P;`05`N zp&vUuFm)XDrTsU5P8->4YY&)g$-n3B**O;skYMXqV|8bZ&aXApG+Z)ojGNpDUKEwl zVr4aH5x~=$gVp^xuvHk=_ZjMCX*+plXSapte#ynRVKaU5x8TU(^iHq}-{W*Zs6AJ| z!8u6Y-UzLewbg={M5-$L5PNI1E>CV{@s26rRlv%(0BN(L0Y7XM5vUOi8BOd;bZ|*a ze%SOiOn_i_8mm4}pe%6G=NKW=)+Of9Vm?5Ph6{Q4xW~(A?>HepZp3=e7bNem*5ERd;$7hBc=Km%AO$$-`E9c^(Er_ z^T%3k+h6~Tw63@)!O@u&9y*txf4r0f$sV6GP`JwXj{aDjcClVzNpC(=EP;H8n@6lsQojf0>*0nyuxt6U(_J}7pu`J;Tb=+(BgZU z8sI^NpW+QlI%s6^wcAV=sR7&J;o&Yi3GzXB;xncBsfUa?-X*zlu-0n+!5YT8J6+uw z%|buHQ#1jIN?KX2Uu?7O>o@Nb63}DUA@*ss%LHEiq1oLMR zrPDVy*2z2PGOztM!5~|p;J#<;Qpdi4z-j}E-|5!-@JXNf{Svppji=V0JimLNm)fT@ z0u8sasj&{9huZU-nXYQ8wPPPT)kputfKtSZ=i0b*AQ-4<#3HOl$ssi4L%j0i^{L-Q zPB_rXs|hzuQ@G8veW2Xvfa6(jUgZ+#UAZY21Hw*(dn#|TuB7^L^YHsd*nj5Te%!ti zTWvo5r1H9*LF~3aOGmcfx5NoBg$DLCgY&rBMj6ZbOR26xo*K`^gn^>}y zViI4u8ATeDjajB4p@qP=;vz5MvCAiQYG3MI4B@@m?6bn?oo*?^xoc`Cxht^agNZ&T zAs)%gvCPN#l7q*(sl>cEA%Fkj{Qph7s^x!%9Ftk-FF0pZ$3A(nNjjm(o%8xf(BT7qX?;M2hWnzPP?uaB-~v89WHG zBL)YSWL}HZ^1ETwVJ*6%>vot8#wDoHfmOQEW zbD_tsH1Eo`ZJ;vxtGBNiwihlM1PhlP2Ao$IB^O+O)~_xasy8i$FEFsv{eaJ2QiRg)c7%=TtV#6Eu>^wQMgl2@NMGdP)Up6n$hU-og?u8F(_LxcF0fK zY`rVn)%C(vlOM8FV}k!w7`ZIpU_?u2mCgyyH_-eoDNjE`C`0J~<&aymqmUC~|Ko&+ zJ|t?={0n@DM(mMC^sE)E0Ks503d0dWBf+yTQfWcNJAqh{WE42;kU{JpOolaQbBN<8 zX^6|aBzKQ)>o3gKLi~)VTa}b zKbTekj7u2nsp>w8cHqoUpZ~Bi?!UK3g@}EIm?VgHQWD2GKSLoS_e3*f){muAGCx6f zEs}^(41qg6F**X|^+8mpyh*5%I2OuYwHt^E#3~;m3XIIVTU;^WN+dYiK4{#~Xakmc zbX1f@e6>(QM(Iz=J-Iq?ye(0xmqRU@B&=KnFXpVS;-PK=X7{{lu0QY;ua)}d9ICmO zj)>GlMtTD6X~em5+Z!$|Rn@!lvxiHuLRPaM?!YD5xPVik)jH1s72$_#r`yrhi?-DL zKS-bd6W8c}j`-#ei8@{rO#xPgsJNih#EUxyk=LXRgHK^sGIBL`XuLiUSaH$v~^{ZG+Lm6lQcZ+M*dkJdbXZN&1_r3yF zalEw@ zq6ofWv=AH{1fQk>IxM{X8>#Sf6U?_L0DjS~DEL}1A3K1bd(Xha$toBG9f*9|5@{CX zn#^WOYBH)zap{Yn^2)&czxTnZu5t2hDK3oO0%4L^Q3D563 zpzN2M^$|c$EV^IH@&cez&b#+DMiZWQlEg&+13W1HUkb8?_6^dII>!c1ROcWTWP}>1 zA_r?Gd47i?>D)h4_0A`|eHTUhQ-nW!mSX!{Dt9aF^d9Xxlv-$13Du_K9aA~g*<8{e z2+;%0!VmkmG4}SdqVjm2JK`rJ=yeq`P5_aJ3IRt;{8!B=t2-*Q1@x(1lSdv1*Hwi- zxOiTc@jxcQScbWh97F3XdA7UVfeYh-WvBb+aGX68IiG8M(H&2Ok3A9Q(^2V*(!Xl@ z;1!}fNI4&vlhp~#)?s(l$#&GSe$=84uu&aie>D*{kp6v%|MLdY0Jy2gmcKP~3e4sOvG3C18M?(uoSDy^vX+u5Q8WvwjylCk=^>Kjr1X zNv3SBbHUq-03Q~JnX{aeWYm|TEhl%_1L)!Cc#_!kgw#0_#2ApQOJJ-!Lj9->IkP8{ zvsz`u6#_%R<3H@7l;h0nlQ_klw8%l7uGnRTlskd%xU|D_>=@)+A&WoGVCCUp_BhYr zhq03&KkPfW8ZHNv1Acjt8@BQC{bFa*$t#p8gb|d?^*>yIm*LX1Fe2=W+nv3oZs?^t zB>qdqGOCf89K&D&*S7#;w?A(GJrLlp(O_IZ>yO&+EIKbm*M!dS+3rP*^~29#+kXHx z=|DMGq5^s4)kkad8L%H<5ym9DwN$23X%t6N?XmN-XcixgGIRC*<|0Si% zpW8KEh8P9Ahj^sC-um^^6&VFwATRpsW&q33$u?VQNN+6BU}zC|J(P2GFk;uI|P`I}Z8pf;z6;qtD>v z)WSLZ1lMLu5=mE3XdLdcM{GZ{5zKS+K7M&jT>}@O41mX)q;zkDvBq&@4DJhCKTIFZ%JnGGT|d5`@<9rAGoah#_mh4E0|UuoA;%%irRgSA>PK6 z@e|$I+31-Bi*57}vl+A?_N+D_vu)V!2$7BNJdts@+{r0F?@A$_Jvw9L1xyW1uk&)g z+U|C~w>82h>oyp;6IA>55RE)gHLHpjie~xs&OlSYR3l{mcwW*ST&avO8n6xN)h|7UsV=YRk7)jlefdJR8Kw=D zR2{h5k^h83tx4uvFi>5-tjYIpDwORGy9@o(py`K`l#R1eu&^@BMrmmc*;-VP+w#9X zf4$FUCJJa4+upZ7a~SDyfm7Do{KV$dm8|wf5ve)Dgoiy_ugA9Dr?4n&L5zt0Mt$eE zrPA@>{TkJ+5`BwK*M@k!8pU^f=9gJS_RruZr*Io?3IV%t#r^hrbibO|trkAKU{w_G zM#~#U?o&K4?Uw$2U0Gmj7~iN%Go|SFAZWr8hfG>h`|Zlt^n9{D9d01lmC4Q8Q>!C@Q`;z$ux`t@QCMrHE)T` zHkNHxlipyHbQzmIA>^)rnLiV+TP3$M*OLpNV%;A|<4VIw!}yyfv)>UwT!GX!e=^*m zWaetP>C}1mjBdZLD=vjXiR6Jp9^-c|dH&p~JFRwH$TxHAqDlX1%(XeYik~Wu!VPTLv1V$U0-J_O41@xJ`SQ zsfE7vX?W~1wc-`IoC|h4KpJVx@V-rD=N}kDqo&fU5 z%`=UQU!r&|onFxHEpbXNSk$W^)Q2EYLr?&Q(VEpxnekdki(#8yZH`yFMm(9a?)4BD zmb;8$hmy(DM}fOJ`~BmK*Azk=7$JX>g}@h~xbB0ZY2220ZA{6e^}ZOlc-$=V99BQ`K^WZB?Xwf<&EuAzGQh zyx9?-OHN+9;C}snz19ET)qmX_@Jomqnr04Rs*Ec>;Upqji5k=1wSz|P?P zF2?#mnEJ(V(`E`EyQJG;lQu@#iH6HKJCv_P zA9qh=coS^wyzC-rUlc7{Zv$Vu_XF%l9p2SUFKT9dzr(V5dHJy=*@BD8j&&BVo zX%($;{}*IL*9RgW!A#gBA}iL|hQ8xb~Y_t~hwA_Or@FgYVI z&07$^RmNAj$lrdbKeLs0#44962+-lr{bArkBIfkB_y%HMgCSAOXa1aed=aVhw_nSR8w$w;=7&mm}cw>oOnAl~o zL#ObH=Ib^^Tl_jTuTb6Br4imfHNefat(~}YLzGP>%47GW7D8(uLE1gBem2pat)?#8 zK;PM{X;Qh@o3r=NJ1H{Hzh@bQ-N&dm0&S33>}a%@g5qm>SYX%ew?YDl zF3^HJVC+}X`|C~`6KsA9Uz*{x?{r`J!qjW_#nni(mIUIUn_fB@`=Int&D>{gMq1+K zw`uN9AY!+m6!H5iEsEUwW(9rWOM_Wt_}k;`Y>u2*VY}=g-m+4A!?6vcP&XWb_oH!B za?%cAan9UA2`hl>EIb`Q`-ZSEudEg4tY@6lvzM5ZAj&|+K;>V>>J|b8iG~iJ$E~y{ zn2E`Zp=o?9b_2*V^&{huhhTHPn7R;9O1=S>ib@D%~hUBKDI zQkMr6GGFcex14UF+ajCVXtL4ol66PqY3J>$g9$;enz*>1V!xQ>J-!| zcr>wo_8H1E+8QV76!cc|YPd$Fh?K7!X;wYLZOpCA|ElNdArE8kAVE~LHZ$>^%n&-Z zL++rr=&LM5ScL9nQ0tXQ+d3W}J10!5n%{{%wT5Wl$y|W-J*6(0B;w6_kZ0|CXNFDU zBoHyxKonUOGj);9;5=4P3SWo!<8MSqUl2c{7K%?5bQe3&-vv7Zb0WM(qFI6c1W~C2 zalTyQN^k_S0^AYC$&q>RDsOW8Fo5)IStyb(%W8YzOT$I~gr=~E2K46Z6Q7i3tlB#H z@MT>pGy({Fb?IV7Qk=IF_m3<>=|9#(x6*W8{^SFXmgQkBfD*)B-n-Ypj+RSab=bQI zD|Wb5dh~-PHt~2CSr%FTS&AT#_(lbV6lJY`6rkCJfCbX^1XPc*`n+{4QU8U|y#s30 zA-X$l_nY{Ou&efjhqJ#&?;dCm^iQVvuZUqB1y)id(TsSCe7Dwly#KO*u--(=7pas7 zc0vbbf+XTMt{TtmQgL?JlUV6{Ke>z90H<(AucW#~DHl+|k|9UMdVYxr;h<2no{j@s zzk_yOLfIXm_w6vDqwl`ny(1Pu6-*H~@cXi4b3N(D3Z#lw*!pE@qndt-v?(oUpS(3K zFOGtoFtTp3NGm4>a8lNB>r4F`mYDadfVZtG)ZO`Nuxj@cX8I()>+dBO6#qnLB@Uo+ z5A<@m1SeB%tOI@^Nxg3H;kN>P(X+?Vsf%AuL1?bV0f*n5B94+RAbpQHhFv7FEJk8{ zM!5#62CD!3FpiVJ(hs{#R)h(&+Gr8A&mZX$EShBD@LLiz>qY`J#lxuTrJj=Q>{g5- z0Q1ssF$*qTZUp}pcJ=yOLd>MDU?4MbgEJ&lBSXbbWa7#!aLlmFKOo9)w;N=>&P?jZ zeMq>dNKoj_#aQdOb8UUJl^@3aKItn@iOZ?`%v+!bPV8#xKV2p~Mss<}+6>04(Z1}@ zg?01?3Cd@pjgrZ1IEb^VkQX)=EEhRf3c6Rs2%~tn8jkqN3<{<`ySSfX`1`L$ct4lo z=5~}s!@d4_FZ?oN;caV-CD!?%T5INjQa&GHPS)eBS-J;`^p zNje^k)7h>jnkUJcoCqM4@4UJ+E!%bufC8ll#HTPiNNbadW!^spTL1N(DsZrWWrX*M zT&4RB5Odqfx-8L6vR>xv z-Nm5SnZ$nyB_i4--6`U8y}k1?2;eo^u-l2~a#*_n-b^EUn0Mm3sv!8Zz!Y^t#m+R`c%%p;d{LWz_D#ugPV@Ew&#QJhpG6ghDh?-VY{mk z6fL@(pR3C~(sq~@0oZ~g1UQM_*?fxAU5xHgYc_Z;i>iS@?LasR(w;*WCp?Z^o)(Hh zFt`d@G}pNBpSA5DisMX;oEUf1sV%cn5f{_OEs$23$dIJQ+}2nXn@vW&k|1qbuL7AM z%*}+JnXs$ZiN^KsxPw~;#DVBS^Lxw5il$P*2y@kQ(I!iN&Gg}lwGQBb-EJh?4b-&;2pw0gO4+FeXIPo%8p2b9sfb}4M=I! zH3KC|1m!rgwb*hx?6>KQVBZ5j$|`XKblM6t(5&!;Zp=24?TZh<1bS2iKLtLZ2VNnD zR~$8EdG<1LlV?~YdJDhszfSG_d*)0R3AJ2BevX%Pr8x3%-GMr4M|u9qmg1N_w{Wu!LI!B}=3)n49SR5f4-j!I*ht|( zQ(Kr<{c*YUBNq!f<|AZ=Y}EA^uo!8Nf+O{PU-L@peTXB@fs{Bwd4ky%g=88|;v{l$ z`Lh3QEOadNZx3dA2Gqn4$c&xo^PKP(_kQyIL12at-RhkJysW z))a?LiGtzQN_!&wBfWcKZVzs4&TW4;kZw^S3?g46z+p^ZBEYYehYV`JW1IskzkId^ zD5mz#e1bkSmC}LU&mnZxB4}Qf)@`}y_OKCllS1iPuK*9J(uIf+Y&Ga^$r4SiFAd$C zZ+w`jp**gNw8X755`$|oZo+3<-`alYaySWCl%-ce(XByQW|HzyoX+8N&)OksG11f1 z(Za|y3^{5UwDa2>S0cuw-ZdTOVwOVj_0}Kz2&q+QVQNGMQWi2kgCHqAk%h`ppA{TVqe>I5|`THbaiw5byh5X#I8jh{oCodf64v#_INUY- zaiH&}*P`E||L;DU1OlRo;{wsD8~Y|VKTAwtZC{ITJC#Oxif+Cs-T84Jk67sba)4j7 zWX$-DrzbC9VUEaB{Drby3zuw36Eeo-Y%syG9rJGemUgXSpYz##6X*VJwYpDk!wbi} zBEGYSP`3jZn$s7x`D=JR&rj&A`&8tIpHqPkLNnhr` z5yz$@p17@%l{=~Kh+T=OWA`mKN~{&TXW4AKV9E*9QKN%0a7<~DMAVFE?sOZHFz_wH zo7A`A4B8HiOC1#s?w+r6)V?U{ILE;SVej7~wFGGliqc-{)Kx%Y;@#g*Tf7I(H@&NE zfb92@-gb!T-vu-WP z(wKB%8=Wc?{aF?=hk!i?m737nd4#c*mD!<05+kD?uIdvWYpI(G++n|MF_JlgL6mtb z`$T9hxe7Kk-+c3FFRvYYyvuYj+>mf7et)V&UFc|z2TYqOkxkWm5JQ0l^<~KITbOqB z6gVa;>%L4xJAY@tz}IX<$B^FW$|KrEiS-9`g~c+03|Nhy)};e>tOD$2{G5pnG=?7h zQy_N-j`G@!J9c{gLom}zI*8Jv)3)(9mmQKaBF+xbYLUE)vdYnE7WvEOV6L=~Gne_PZns8HqXqa>8iPEeQ1(++ybfDj$|AqZjJ}q877y8??;Hap7 z-o5dcc;}oc9aAswYhXJ##;xb3!%th$oZpOyNXK$D4(H1y@9U4E@ZC;u&~9*qoV~EE5vBo zl|p1SPjkH9jK4JPK$wj$H@OkrDfNUSaaQMHMeJG-5x!3W=H2$Kn3+TcGd}ceFE0M{ zc5xyUd+1j=X|=^v=nBhT53x+>o6byZ0)KJw_l`5uW$R^`gCK8f{;U5G?_e03Sj@{* zPiW2CY!8{o^3*(R^wk8L-oHo|5=zdov(E=L*j{|0)Qbh8WKdT}vkv%av-Z7;=vdl+AGjg-ouVUePLBq|#s}6wp$B zJyk99;8S1dQRjOmaeX{E6?Y>fS6{9psvV~_kCiZBrli?HOz5v2E*LXhf>6;~U$AA(|`)epwPjk&ruk zmF@0T&mFuLhvA&3SK>%BDyv^UbmY2*98}VH9POygVMPyQe`c{w`PzJtJ~^YBDwZwN z$r6+<2Aif9=;0iws5;|vT_^nZEnDsO0HCON6z~H&5Bh{LRU30ZJhIQcIUBEtwu-&w zD(+#3rs^t2puf{`KNwqS5g_ouVIx?cH1``qi}M z(dOs{XSd+3%OK~mSu3YJJ)UXOi7y`kO?j>c0)$&Wx51N<8ik9*Q22{9A-^#-5PMmr?d(6$+-o{wkd9?X z;YDr+Z`SApr$t6xm{7(BxqW`lsR2kPC-GIt>xVciQT{2Y0>X{O2IZlrwy_PpeEhr8 zg>#_$GB0G8PD`eJ83~c~{EQ1K^D>T(=d(Jh$i_h~j)X36V1wSSg_*Dg4Aq8~47fUN z@e=pu*p1aTh&}~GS#ZCdz4bn3wh8w?<+8%Vm^!jM2W|PNTm%t%7w8xmDG&0dc5%=T z6Y4zT+b_#iPg%V#*}xEUadpZ~0Xj3$!zSU(RU!d(@w>e6#o` z&Rz8Ue){ZLCI92H7sEm>vZSNisEKE#Key0gc9G2)6(ZDMXN6l}erYg*Cdm7+dwBls z2f=|S3ifbW-Vu4iN9udn#gnd<6IAmGnorJR-0B^u^F`+tkJYRao(gJ^*sBJ1y(s28 z_e^Sco8E{JPO&hVKo2vpXj&+98A7MBw$bxw^@U-*nXDey7kjHccnJ2!>P8F-=V(ZpHVaroQQ6H2wvy%(F42on@0 za1RsZ)}6wx0-z7RsHC-5j&icvUM28ujt@N@NMx)XcS)ZVEWROxpw*_R8R}cXlW*Y_t zGJL>xU7{ZTzG-GPT^WyRNOV8K7xbV{f5rE6K*%6h2C)CgO|xx~r@i6;N#BWd5L4Kh z+g`BYN3VhTck<-)ijF8gHL=;+FfXW^35`z9_9ow$A7ubRKKWD9FLW4gI@e!T)V`nZ zG(xM&?Ec9xg_-pcQShJ-gibb=o2;M$^Cr4x%6B-Jzo_rVl9+cfjuf`qr*f3vFMZ{x zWy%HR#Zx;^T)=NMxe|yrnsQVV>Hbs?T>2j_Ks;MnD%j5z`;H!AEiZ1X?;cY)r}f>G zG~RZ`c7(U%-uq7ePV_0z)WGZ9vbV4sFXarV^;`QZ;r?vK_cUP0#!}o!0h8BOTC;WzkE+AD<{3V$thfR;G{7QrI zB;WX@ZIw@u=>&VKyLi%VE0quZ1ECHFW_x*QR$j#qF?wgqjklS3{eyCU1!BRW8x?3~ zf&sj+w@eX4$kgxw`vLB9=NwM^u_&{&EkWh)dQ$c+&j=&fY>?d6F7-Eihn%(--7P^0 z9#q>Vk_@F1wWjs8wP;E9io0ecINIiXAV#*beQF0P$*o?!;Dh;BIW!VOc z>X0rzyK>|zu=Zia#9hdrPzqy{^bbHVDem_734>Z&+e@6nxMclUF+<+MZSV=jlS9yl zZWmXB0{1eW@NX1!mkGcyRh;9c({K6sw6cE)lk1rV!SPz+T5Ob7U&(cGMs7{+2?ICo zea!xLhv^}06TSJQvH-s{ISY*PGO`Wc6Fyk11hfVG{j+=Uc)!b(VOsjbT9$)y&Jp?W zC-fehA$d~8C^rqR>Vb?!H#?tbL=m&Iyk3#dg;9@N4q+~x2X{(O*_|HPPfX5W+3H>D zqW7cU5xk(v=OgVK^V3-l9N{>JBv9d^A!$+%tVgfu*6!ZJ=d|Txmlp}UP-qx4LS`a# zUH+7be|BZAe^=H)nvOceBk5%lAh++S?G7WohFtlay>)gR?84=#qLjQ0ALrVMK; zPvP&c?4PFGQH}V-bhdqWea#_fH6_0f#Mwyk-R7Y`tG!614hQ%8@}tLx1+w|5&al+0 zIrg*bOJFYioP$GcP(7^yKGK+x2jbi@B!0JMftR23CD?=t*a^Gu- z<gQH>4!n8VjN8ODXvz`g)}!NCv2Z;f2U+bvXj$Q)QF@ToeiQ58Zk&0^1kpr zKheOs{>YnY$@h3XQBHb@-3Ik_HD};+CeaP2tX9c+j-=T{)xBD7DvCmZ=zRwY(lSjl zYSI+vY`!s(ZT=Iw@$0pTwH|X(D@G z24#=t*O^1Z9xg->AiM0a|MEIrl}XlBM=&|Z73@))8Au)B>3il@x*2pKPD=R7ev8MV zYSA?t{AF)3UjXKNHQEH56dP}TnwHe<0=bhax!o5DmiX8k**e4tCh3OnE-(&(4hrtKhdHY?!pSyTGLkiHtC7I@TB&$ zpwC5GZ!!@eHRwPD*~eaG4dc7JA-B(moHtLi2UJe%xyZPGAkzuN5bd~@^bdG2eRdvt zxcBf!AdO)gri!8QW^VhCK#K=1n0xRhZ$f{|&8o=ePn0<9)|J-0^w!iVOh>IplOz= z8W}q|!kQApg;inu@wzYK3>Kik3`)d-2~&}_)0u$-5-V-3#X!6C$-yVF+R4&18MT(M z#R>>>>k=&?wFr|fNV!dEu+%Y*0ZKB7q{3@zzGs?1O zeusriw*9ZL0Mjdeu_;wzwYG?U4g^!`;qaK|8(o2WzHdSfWR2pqV-_lI<;c?6-Q-4X z-cHw$j>fu)ff972vO(~4j!)~ZV4j9Ypq_BiAZnAq+$3^f_RR|R3~Gcy#M3183L4_b zGiiF>Ib*eUwKgPx#W#mM^d*6g8xE403SIYu)TF8bW3cBK^Ntb?5QG-hy@4sYR`d*^ zKht@_AmRkJE&Bd%p7?0{U~i>b}tS0@c=i@^9P2) z9O);}DxhJZ)_~Qb#$lea*2&w9GXZ6!1S!Z<;8Dl7(RwHBV8I zy?C%YVCR_-w)xMW5+PND6IxeCyyQ(7l*y4H%0|;ILKQcn9sU6#B4>av_2OQDWwqT- z$cJopSObDs#UFF}sKZ_)I&22d5zLo==DOY`RGmOiA>(ZOCYeUg<{##_z%sZ$OhJbU zOYDU!P5-zt4TGze4BL-_eH{3R0H@b{x1p`Eigq2}d-^nfDwKwlhV)mR4hwd$xCr;) ziQ1XK5VbOR=ngSvcomKswZk?Kk@nX=%WBF=poE(_T*9*2=R0ICW%jrMBF# z6ZtXEP@TKV{82yq`#6i)lm6z+@84xACkR`alY9NNnFp+*SbJNdaCgq)Ci`aVhP*}F z;inKVyeUW(auc3@?SrkBkz9gYIY^aRhXx?t{CEyi_-iM60ySGN)<;)~t`DkosoNue5B2Ms8bm0ZIBLIf%nnjI@Af? zk`kPDUR5Y*~T>IN);gu!tvU=bWhNDcEnAw(Bn56E;U?P@=miVQab^ z9sXBBm*{B{?CGp_+xQ})5g7JF<7IL6_6i5L05=vIIF%}idmwQjf?NoMn}SfksF@*z ziQj^|6YAEYb@~z;Kg2KfM^;Hj&hW9Fkaa5vEAMO*AtX#c9&|&~O;)!&SE6RKnV>f#c7>p0Udgh5uLW?#x$XFT zrauWz&&P};h8H1sFIWN;-Ue(H8R{sZb%R@9_Pxme0mNCMLigPQ-SW7VqiAt987WAaZB{B-e z)6R**lbakuqT1hH9+5mrV8N`5$%`?LE-S*@y1X|PP9*G~a$|B%Bkp^fo4qe(*dA++ z(C?(IW}3T{D2rrjdQbaF@L8Tzfji1wq3>sLJdU|?_Iw?uor#4W=x1?c($`2G1u0H5 z^4tXsWt-??WNKH^Pi>f=+BkoGf?;TjZ`703s-P%aOyna>grG4@^l5=X$$2$g`wu(q zzuPVUB@9egMMgd_>_W(uqT)=I(c}PVc}V5hPe(_b1#PtoR3qeQIyy+CD^#)2H?`z1 zt;Bw^)33F6lK8}(gTw=B_C}B|gZus}{i>(0X5)BVw&APZQ$=F>w$-{W z5*T(hVd1IzdBbk~^umrv-BzV=X(`^;iYOMv7OlB(ZX^1W6$)Yw2Q?1~g0igS7*&Cl zA461Yrj%rg@2d63RX0;V14W7gI#ugfyBMk5pk{{}?QYACVf)qJKX1%ziPdQ$8^1xl zOfKn4@6vy=)O*f5CAC`R)joT3e#rjot!L{M>EZ2iHOFc*mB`!nZPqDagG1J7r%TbI zrJ6qFvHTBIg{mg{7QUO6s|ym`PPCs@%3ThP0dD6a3vXuv>JD^ycXMeRT`lzLhSMDt zYH0WccX(Z^)*=pQ8+KVkPI{{6f(lU;1I}WCxwj=(yu+R_Yz0 zUpUnuItpt)P8xTBl7@9~;1cQ5`VXbB!ety-@j}`j;TFW&&ou+V7t9fIS$Wo znux_x!S`1B!wxcVi>ZqDWZwu`t31>etZIoLUGz z4a~{rnu#rAiP_`~tja&jLw7uFjF8$lBQ)`&_EdXWD?{{k(Q4Wyt}(tll-#o0J3X`s z&gL?88(Zf7OXbZ(lnea;Fo#D>s3lFA^VwR>R3ONsq~U#CV)%|YGi_h%+zKnEM|9kd z7(&#l8b;_^@c}j*WW2wt8_~~+(=M8qT&a)I#oW=(zc<9@JJ93Gj*^aCLE?x_I1x-Xl3o1!gnDG@~9KNGaKdK%dc3X?BvwJ{dmqK|D7Yq9t+O$lZ{ zMIRFk-q^FvoPVxWPM~tkUc#O=i|T~vr%{dngy)y#=81@7x_JIyo`{Xb*t3Pvs%g~ARq64?vczu<nAO<}9M*?}S2XOnkKr`xJm_XNrtjv1jQKZyVFABDg*@yo2Gn(ti0WcA zK|6lLujcm3UswYA2CRNAd~3{Y##(F?Bas8C;*F}lie#THvhg%iDVb|x6dPVt=&O;e zhbE%~JLHcLF@0eXs6xUqWcs6{EesN<3Q{aw+S8In3OHafQNvyi)3H(pZ_r%MmYchb z9y0#2nL;@=swwZKU#~wT`@Uz$q|YB7Ja5d999Ezz7{Vx<%OZBpw^n*#fapm9=2`hi@S8fLsaHuvrh1swLOy}HfPwTlE9LL67%faA zIy}2KgxF26nm;6Iy?&WnoGhSU*+u)B$+u+{;Ad^w5~)Gre?B^s_M*g}B<2+k=0qe( zf4;DO9mXGjnS@=NlVoDswrx8T+qP}n zwmGq#Ofs>Z$rC$IY#W<*Yqz$(xAsqWRagJ$uDZL=ed(NAZE}kAf^(k3GpDZqmfD%X z>Pf*=fH69umK&Jy<8vHy7+@t+CqSVPoU_WB1SUeiJWhZQVs9=={CJb6`)-GY(?+&f z?NL=Wojll05(Anh^=zqG*3X4}M_-EX|FD{&5&=Mu&=@6lPz(_VdNk;pclgU>>EvsD zD!5a-vK1_1ks>9&ixa4;>LA?O>Go&X$)#FT!K?h^buU;9ZAz!DFUAF(FPqLRYacJp zan~OQ%k7aKSeqI_bV1`h9T?XlZ>l`J!f$g9YoiPfXlDo-2)n#V@#S}6qr7Qy6><6- zCW*cD2paL-dDz^maA@o6J25R_`ftWu7cxd}b%Be2Va%UsxJonz>@Fl)8}UtA<1*=L zM{qYG>NupIU55L4?^pyfWp>weVK_}yo~1d$uFO!My(q#XSV>n3w1X%mXGWWo#620l z(TpVK-b)g=r@tw&esq4q_6P>BGzqpr5C>}NXRCVD><)^xUGZO)j}ZQ#y>Ryp~!ov#xYngJ5X&OB#E&yU?Mu} zGH4s@g!L!|HOl7cER-E$0zsLh)I-o7^IyD2-$}oarPQx*>IM3EFvMvURLk;Lezq;F z-#wZ>)>mO{+a|wM2AH+*3AMSlC}bOG(at(ry@a@yyc-kW)VqovZcn=5hW-)F7JaYG z&F`_=3ycXTtfXb3`|65cMF;9wKu^nkwB-Os32dz&iZxBjIeTe#!8@u$Tof`U+1EnF zM-+{Dk7rlCUK4AdeJs0Q{dY=*LBv&GGYF~c>${5_F(3~k*3pb|%<;=bQq1vT!%lyW z;xlXFs2}K}5skj(_X#6DQ~gI1(Sj{AVf#_)dTREBSP~$uhd-*p>zansXx{b^{&o+l zm7Iih29gGgR5GuT^njWd1T!gr+F00&Zl}e}(a9Luup;H|ijVe6QZ-j z!9ld76Z>qk+9^wGq{|sX0DE>9b{?(=Rc`n4SoxjjFBD=q#$M5Vm}jQ)tq5N#lrf?p zo>na7p3IG^u~y>u$@89s4?`bpn zlN}m+3FvZ74A!t#Yjmu#uzV#+{c;jJk5o=|60>9y>vy{tdMR(%==o6V6se71Th1ab z`U-i-a9)UY{0N|YpWAXY*RZ)2p1u<(c>rkG9#2PFT!T%DqxHHX&6QAf11|m*O3R0g z5~D%1@`o}SPS-A{2Xum|N(d=`lINQ2n&NUu8s?0Krbf3Z#{YCH%&owm9ur=K#jXL|1N~I zucED$t7zpkvQ$HmkyJ%n*EzC<*?SAWiJR?yE2!#Py7wtSi4F4^=@WO`>EcH+q|KA- zu;fZ88mp`g@IDUh^==|V9k`}ITh9R>MeFTY`J*|+Tn3r96!8WHqN|B9|7vo?>1!WM z4vnH3529cSFL%TcdAINz&b!|hePB3f87`jC7X*GkYKA*;)NmWb9I_4m(&KmU;N5gv zugZF)vj=4ND_K=Zm!h>pgi47p=bwT&Dpv|`=J;m*I$~>l=%$;Ewd2zL}P58W>NQ@9DA>GcRyM4D965bKIR>w zIprz?Hb327&w{(}?@;wA=~6yfN)Hp$%r-tB>i53CBKYsG2>lhC^5M^bt~!WxdCmrL}K~L`s?WQlzC=`qu`$aW2P|&$Ao^b zw3eRlKywN{E}XQv4DEMvgh)PmVf}k*X`~0fV!+J-VOmd!^lc6 zGB&WVN1R=`VDnq_98Acl4XBv5g0;Qres}AP;dVKuYq^VUTRTEK+OW#yHo|e{<~)~E z-(9Z+L+*u_fxD@eiygQAx8t3Mh~6)-1n7F&iK3?&UPr=L#>X6QLIU5s>rGY@ujspf zKFt9eJ8qB(@By=_klGqZSDP0Vt{%l3tHkNWG)c!tk(~ea+b6CAU z1)wJ{LVA%0-io{Dol0jo8f#)Nk!0H2upbt?geSA&G0j*V4I(CiUGG5pOvSfCad_lJ-D6a?c zw9qJTgh;|Ejb(0pzqebMdXeQO--XPbAwRkxq}=3aZ*}EcFXtzGBgKSDde*oxd&VB7 zQeSxHq)p(($1Zr1J*UpNjv^VRV%VuoPsq!TU>*)*>{KKp9Q&bZ&4DRu_ktCCCVbhYA-s)x0JxlUJSd86?Pdd^r|j;12Vy)HoyQKVNe}oP#b7% z440RS74>4Iz38JCAFgq5cUvfS+u90s*W=B-FtbZzljHRq(CPbW z*50=DC+moOhmH>%DtfnnR<(|9cW?jOu_q4)wm;mku}1c9t|7L_Nc{NsZ^k5jX#)E} z0fjk@Qm2fbI-FR!{LJpWV;;d6dj`h?DX^-b2l0wvNMlTB?I0iHCJuy9ApCbs%+N|K zA7vnvJfY#7_$+^ccfL^E&+Jzwn;^n}&1&xlJg6-Koi{J8)4U^s zDDpunAV4!D0JdLh$esU&Ss&7*R9SY>B(9prGvDNc-=L0gB$Hf;D{Xcs-oY2VJ>U)@spnH+ zTS$AF$iy`1pRyDT=fQqHUr;82C?Jx=d_)Xp)}`jat7UXL<5qC(4dJOk@M+tIQ>swI z1yTo(lks~Oy--lv-955B!!h-bk3Tde=W zqqGMu%=Gl$(+=;IqnW~$5Dz8()gouKOXv~IR>Elh)1Y2XyUQvInB=U`^Iroyd*OEG zD7jK-UHj!5`|$z6-z?9{w&*>$dcj@aFxT+bmoHf}a5NeA+R_>Mn_M0ttt944){Uo> z(V9vBzg~bn+6KWUJO%NLy_;_v0E$6qM9R_b{7|Q7Zv3y@1V~iOU5hNWXAkbM`c)iz zy0_V7W&aW#I1yAWPkGNPjYAOJl3kQQ!~?P7P4}f z#Yz$8=(!pixN_))lL2S-!G=+z4jb_%7+k|Njy?v5Ks7DOA>(o$kahu(_D%x%ls&X> z>Z%Q-bi~`U7T%E;ch{98>gvrpZ{Xb)Eb2<$1NOXen#z@QjIBVc&;YF@Bc*Id{9f9~%nQ-T`4q}Cw4n&{g(jV;N(cki-WbOW6b1m1|X^)Sp{ z@sED57^6jj7}`v-oplHs-bs#F9aR)Dz&I@JwBdiq}`V7EJ#TNYxWj5^s|s1$QUSb|}r_m!qMR{|>kOVL(%C9h-ZRNcucKL|7QgbhN{T zNReBkCJb%Mmit=5V>$5n8944x)Qr7#4B?q4R6}uQ++>Y$u(B4V?T4A&y2E21K}Rqq zC!DygZGvkZS6kRm+uMb_bn4bHy3~JGcQtHVmPN&8*o~ zU~ZX|d#QCxfkf&wPgV_JE0(}ujt-*TBT89h>ExRExjv;jYt1e|B$S+qCcY6%WIu$& ziPG=&0B7u~#~)GA@KcVclss_>MGDJFtY$?uI)|C;0Sko(Uk0O*%zxo!Z%>z+o%S+< zE+OIqtJoZ=H|v($mYmsm82KX4VMsa!OrYtVs5`SWDH+^Ev*R`cPkToPr`6j?uhsXalh8=>)`g}798Sk zg`Ol1a=zn4Ij*0Y+tThkJt^Klt>`Hxq%{$2g2*i$YtfXoQMl>j>iy4hp5HZkgBbmR zw9o%I1rp$*hP6SJNJETRBaCRnkZGf-IvJNqrA}05f?Bx`$Q$+r!e6g8JroJX_5Y3x zcSeyrr{x2pn#o8c5Rjt>={C$!HTwBcF)4fBRsP;8keEZjK^pVih`5O|{2ga!gT4ac zHSz+V6@oxc!HP9p!N%Y#MTfRhLrGzaw9+PQB`Kzh+@^|p;4e}G#N@%P08X-G3|FE} zhN6kB#fR^Zu4QH+8wn4IThhzP7YQ3>XUQ8eb19`~>pLQ9Dm2y@Gx>xtuB^oF(MZfu zki9vMHE9Bx1Oy~L$3uc5oi^YG5$1SlBS}%{P6S&WTJqL}tT99UKy8+O&Fe4xDcmdx zawcL}UePfF(hFIK z+;-L9L|70yLw_qr*hKPmUvP`M;lhkI@AJgLZlw4PUzeE-gkM6RSm&sbkIrVZ;kNME zyef`BbGHKi`cVn`*{p-jBKPkN*nNNHzB9y518;9fHaG zHOJdASDn(J50*e3RGzAB0SoiMUg5q}@=rfP+k-TE+RbwrQZUHc0`|0IU9rMJq_r6= z?A2WN#e#6C>+AMP@vKg=vsD65bej*IIPvF`JPC|mfo6q!+d#eDX=hLVklOM)`PN&J zLNT+%m8`c$9qP7h+9Ad+e%LKmdd_j6tO*5OFR5k00d-^U>-`Qvf?bkZH%X7a7NIR=cMDMoXc&{B8 zIxfgoR8@0f9~z1%caFg;*?}ut^zCK&Bjc6L&4xcM8`Mi0-vhI?bNUXAAv6B4Bhx<5 zszF|*u=#T3^;0fTZ=}iqag|{N6Hz)iIq8OW%Z-nKt4S_5?_;Z;#LgV~arto{H_bPm zw|IQXT##?1N5D>WnT7yUC`8H(;MNAsyJFmq!8BQqxy3cN!EjGif`aO#r$b{XZ5}Ac z3Bg))T=AaBJgrow9a#-RvNg ztnXIR=&jlabLm?G2vgBTY;QrZa*whPu+hw5!a9>(x%S12axxuR6E zvX!Xp#130Be@?A<7dN1;2&lp=c;()tr*flEDrs|&W-(0y{$KBc z{}a;-5{3vCIQfPork3khu=C8t2J2^C5Wv9Q=d}&G1^rkPlnl!TYCNvO9B7E~fqso> zJ+Tl2HY&x7&MJM5;ztOCSO4y};srd-Ww=`^!<{$5yVEbE#&(2BCU9N{yO8bGpANoY zHIHQ7Go@5+<7x-m&jYP*X2Ka!>y=xW9d3g<;n^rKXl9@51=UU(NMkYn$>0whE8- z;tkChZwvS(h0DI>-Va94{ivhs;}pi?In9`V$7D2{tx~^CmGDHTuty@B@dihnf0vTE zjtj}ko1{sKU_yIH@7ylI{j86azN`h4F+&kBQHL4yhM-NYL7IC7wINUv9iZWlsYn}+ zB#;~;DvA;NR%T?gn8~J$#2%2ve!a6dM_O|a!GSZ2il^?KCtKE#C2Ay(&9$CGsdqBh zFcpvSaPI@)P&(02tP!Ybqbl0)um$7NUbx5>L-Rcn6Yn&QlBIG*q-PC>q)Z}jfu%@J z#aK>z9a7j5Q%OiFO=Fc_W6lU#{0VBxpjMm-kto9dreba*EB4Dc8E(mKv`nYa)U9Zf z8l-#;7=B(E4Gu)w6IFrl6(RwT^`(BRq(mxq8^#QcknJfWRCZH@0y?~qlb0HhuA0m- zn5k(de8RFZiVX*_g=2@*=urV7Y3Rm$X(nrtMAajSYARCWR=->5k|pXWhb)p5jmM~z zHg0NpohI?ONASO%?Kb?6juJ`d)r_svm>#G83Bqcgb*@Tj&Hl{oj?$wc&y&Xo01c_cM z@c84Zb^HFW3c)%|R}*sWVn&JTcDS_-Z5974%_2e+HX#2}q_G*b^pXRu65hd9yhvW3 zfdPZW&CS(g?((sR@>}ksVs9%EscSl0S0wEEIeh327-h)gA(XSNir0f^mlrX1{G3>^ z?})42(ehtWs z$)lZO#`w+!pr}}F<8)e=KKGK?4kDms^D0^|YEUw(Ovu2rn^NU5{j z{)AktE$;My*;T{Ub;h}%voQ6!Drs)FeqD-fWz1+n{uH<2Q-27Z*PUE+oap#|c*;y$ z<@We^Zd90CiLS=_FLdp3PD>!;^$Vwh+#X%SQw*bKhwF*P?Cjy%ucMJfhi-i>!(~pG zz@!)K$n|H8r%AA}nbd#{Z#_hyt;*>`UU985@s%>M6vOO97 z2@B^?%Er;dP0Qi{TuM;89M4i#ok{%*&7*dmX~qMMyQXPZw%%d&s>7df(UEFylAK){ z1l+p)#23i(X1#v-BahY7cZ4oFjiyP_rq&}!jW6HAC1R71zMpq2F)G@J^6P$>iryJBTffFeAU!hBCowA@2$>wxG1Z+f7~%M4#1X6wiq zh~#!q!L8m?!Gr&oPOyob+dJisCWItC-92MqW$zd2hnCyISL@EZ!S!9WIPmNJ<*ULn zS^SjwR7eq46#Rr^nWat**eB@x+3U6A_xBNCVxW_W?|DAUZ?WU?`A@+MzEyGQ7Y$|1 zsGfCFZ;un(_=tIN@edkT77o;1%oD=vfsTGYcNKb499;nQ!#bC9m^{od^gb-=FLa+& zbX`8m=t3n^GvYr?`MRB9#>C<_O9~2WA|C1rK6T+U=R_m>)>!%#q$Q~!CSx3tr`lj} zO;C&tMf%jq;7DJXI~|2VK+C!~PhpHro-wvqa+OmOjZ2CoWbq(?5ADeS6b7AFhKWJ` z5rCTu@toT;+IGPb9319Aexr>ic*}dEjtarzXE~|#N)ma6!tTvz$5Oo{Av$^uV)JtJ zYV{xq67%`so|_>?Ar2ETQkw}j6}2ohjL+RAZJL0|Q!Rqcyy+|vySEZqqGM1%NtukBrwm++C*NBVYChx1C5CmchwZJ6_f|m_ z#J+L4ym$w37a4#{s_-gSLfTpIg0O*<_d!=KWKFtmnv7)<*n?+7fpg2_8m5=LwxmW(K%?y|0s#E>oVG z^O`pbdt^6sz;PLKSd;R|CRxXR+RTI7R3`f&@-&8Z)A+2OGaEf-1D!;gx*&qB(wz*^?vUSnwRHe}dIk7j$pmiGL7X%RXjF(u{r_$f;(7#5ADe9tKKgsg(@|ON}*D3E#r`{S>S?o6{Ixr_voKbx%_u zgZ{japoN%8x9{N~zIT{0+8Xr>8_xzmAOIS#+7fJ>iaL%sYhlJ`T$VL#1mU2HxR6AJ zK$4F$+p{Onvnb6|xF)V^j6+v}kF>BwCTL%?EiW`s-D(TDA?Hd}u-Qy$F^)LdTFFsW zcxzA7SZjPVO`?S1UmX7_|LLMvSBflU z`PF8nEBKCI8|R_a6u~Q%&KqTFdj98EhIhrR2eqc20am5zz%45=#@ycAyeHWF6qYhP z|ImK+B3HI&WtgJ*&xe2a_w0_j)UGKr?J11yiL~%0yG`v}j|W=v`J!NhkTZ$2?J5Ph z;&mIP!n?Pc;PJiPKP(vg!9m8}5O2ro3yP?gm%D!~+3$tT5ADN>c7j!A_=<8yom)D` z0X^TdDsztm5WPO@{2-qfzi$?84Wnm2hQ@>6_aSI@sq(iBhHtq zuE6||j+6SpkYFnA%&X;XbAs-0M4r)`6IiFLZoBUd_{zu?e9})F{cP)e-DdMQ*z9#r znwHUnnOnqBPCX0EMSOE8%>hXYKD*5qxsiuMNSyd~F8eO-Hk zxX{;~0SC((gpNw3wv6Nc2bV&fpcWVNuOUTwbw9avUA~4b+tUQ|X2fX-*C#UF2MTi( znJ%6}5vz2@C+;N4l`Mjld2I;4A5`#;%ySHRr(K?tk*j9PQAhKT`L1$w$U7s*Zpr!) zw|=){7ZPtFAfY7u${FyCACL1TD3A3OqB3%ITYB<_x9C?8@@lhURb~8ldW@2Q_v+^c zUHo&F7Un^#YqZ6w3LjoIP_iEje~Sv-Bej7yelw+yfZ8ZNl{s`GGuVtSh$UVKj3EHq zLa4dWJ%U||uF&%b24bKwJO(FnBquW77wg23S~Xsjd2 zYV*P`*O`}Bk79?VlRlHL&0a)4e{R9vm+Z9Oud`ee8W*js$^Ty+Y{gL&;hWCZF1#@d z0tWnJeo5cyDCVl8s{0BkP6IsjVk?OQmVsn;cgqbGv?nm;{aZAo`61w#+6V*w?v&Y^!3tpQ!wu z7Vh}`-`MgyN-{5^BsLYaMcBb)6^al7yj`< zsMJ8{>9PL_L{6ZdBHf(5ep@%cdsZlOe!@>Mj9X(zZ(a{tFc~)-Cc)Pg)e1AwyKbW8oeV2auOd5LJ!fj~gWMxCTLL^m{>bLk2+C9yCe4r<588 zFmep|)eX?imkn#|@h>CcQxpg3#Fhc&GI8j!k^;lH8{9EWUAtCCt2V(pgo{VfkkGbl2 zgX;CbX@io-ZIqgRf(!^G{I(9N&)v8`jt)g4DQd6*E55@uGrNB>iFTH zB8KgnACXk);!dyotSXs2HqeZ6r=%+Xa`DiJtd7aZIr7LsC_p1eQ)uGV5?gx6toNL% zZY1hpi}}*zDuH5Jyw?u!&U#3F-0~_b=_4r9QPpCcdq~7gnyDSdpy?C@;4Nt+9fWNY z1!^VA<<<%V zF>lxIUpBtRT*MaWVM1xe9X=DHZQ*%ZP-odv7+^aHGUcS(2~383#7QyC(lwkYQO1SJp7c8D(0b{xDY{H4afa;IHGKea^Ofiz5#U`SL}_p z!>q2ys4@&q2D1r3A9fN|iH&jQUSE;>tsnTzrToIco{=Sj8y1d%LGNSGZ>)q8gX8(s z&nkQ2CljPLtyCf!mq2wpBKJ;ByKHxK4}Rh8@ko*@0Ex(VPsbhfU^UKmsebAbq{FRL zdbHo;Nq3@*$3H2^$P#r1PAs zsnb8rUpU6S61A}UykZzbKlQ6};}N1=vIzA{NX ziLSmx7e!Uisy>OTU3qyI6B1Lq*4Mg!8DBv3%h;b;eI^ z0(P{12)A{6CBxq-G`lZpaop=0AK|TV9I?q*_(hT%mn<1nqlFivxmDOI|D7PiFtob! zX?WY@dPm45RvL;v{TE1WK8(a(f_OIn0$~DseC=8L%ugVT0DBl2gQH|y^=`mfRw$_@ z6vMKF!fXV$WC#T}f>t*|hs0q#gvCKCW2+;z(+V#+7qW=G_j#kxZ1W24F55eH2yWqd zQfUS7u3{F4mqyQH!qj@2@PC{Mj{9q&_3sV&uz^5DC}Hqy2;X%FMyCP1cE!&h$3)Mz zz$3nvRiW8$-fBy~BYY{RUIS?biOC2CuosIv_u{7NCH9R5&k<>9_f9*I^}btYO&^D- zSG^~b+Vr8<4Wn~Vx0X@Gl5=WOD{dN-+6|+jR@;^|{i{>M9yK<*C|tT2!fG>$wS4SB zsD~}zju}2u-&sUqp>A!Wwp8TU<{1x$pxz&CsAPO;7L36^VCXf%Ix(*N2N*}M7fr=Kos z-KhM@ad-8&26)%O^VW5OZPiER=}pjX$JH5=v>Ow%%hO+F(D_*$0Uzntybr}YfHBmx ze(pha{F#6q0BikD?y_n!wBKg(@QQZbDn%Luu&zkQsozIi$KJ2)M_MdFTq?d(1wvsv zLBwhfw{2bx6Wr`T#lUzWEh*AtB)}&$$CqE`%-li8<20-=f*<5Zj%Bd(uaSW6u9f49=LFkjZ8=Jhr;x zaub0}W1YaY$_w4Bi)wA!84nU3@LI(TZljtKx?nSW;l2x_`F7U(sI%CsQut@KhC264 z!jZ1ZBnaaroFF+%-Ti{UbjCxnw_D85pX}j9UCc{2Bc*0j-C1;hV_R77k7#oVL(Ftj zx<>L;%pVOy7fNpKcFeWbg5p0a3~2aLF7ZOzX3M%rZoF6txBu4*aO0V#KzpS0!Wjjt zcq$N7T>K{m3lyZCe469W92X@W5Th&UKUwTm1Q) z>S&g_*~{y<9zf$sgXDJNQ(*H(()Wi+Tlm_v4y+R*GOl$+wOj`O_o7dvpDS;23ipo| zaEK!uZ!}qSqMD#*j3&8nV#K}DDQ#OReXIBfgBvXoLvKbp)XylLF^^=`$5#HK(*u_W z$zrtZtloXmtrtgnLf>w+W@N@#%5Be(?*)3+{wMw*d+{vtRZF!Or7;oZpq;A(EhlTf zT~ipynz3_Fo+)qy|D9o`cS}wRr^zx<(us-I))h-tcK*rF`H}gc(`IAhg&bcHoR538 zK<8l=FJ?!#)Y<7?%1}QU$vdif@J{T;x_h{;)oV7YF)MgOE}Qx6;ue?QDVx3~g(f&e1s;=;GL!I|P3$&-yg-u3 zWw`29Vf4ZI2v~6DWz3pKpokj--`#;*@qRh~fxV?&)&NkF$QBoSD9G?kV`v>OVD+l? zM-iug(iC!##i1MG7e2p))ywt?s&b;o?5^7=D|FHmrF}3P&(Gl7ls6MM&_H|U3zB;A z9)RZ;`wdbKkQenQtF45nOR(sm&)m^h$b`hV_cv+0?w+1-Ee|oX=cV0L=iq)gZ$mf? zJfVrls^1Ac@yTTiNC1vpIf{J|;w^nZNUDXUUrRu%>TI4AYsn;0E)onjX;z6ktTz5{T7@@aT=BQX9f>8zOKTAaNSr zWIBS)bo){A6Dy+bo{hcUZ-V3kok)W^R^$=YbTliYB2eYxh|&pu@!h#8ZWQ0H9N>^H zXFF&sXq=aSb;kDwjJn7luxePh{7*9RKP79YpdEr{rvo&e_CJTOiQ#n+$K1*I^30wj zS@#-cQ!-OxZ43uiPDNRR*=JXLTL1O_tt^w372a^O6oS&+D~hx<)EULo+rh;<6sIaQ?BECGcEXUssCjZ}&5Q}fTIUls^x9q+AE|n}laFj_^FdYaYN`j|OsOr?Hw(KV`ehtlRtHxEsQZwiXrXjMeBi^Ts zReNk={*ysxIC%9)9edhXhb~$^$R|BAWi1fdC9i@Te*$y^%UKy6poGq>jJtSVcJN}J zf$GClSkq$@ERl9%_F&npzZaS08O{dEi!FkAVaA8Gwdjp`i&Gn0hze8*`pO;skULRQle_P>tV#|3o zyz5?&^|2M;$WfaPaH}|MIyAV=a|`MRgwp{+gO9<8V+j~}dg1i|#fNL>NqqX_cIkL_ zv=j-#iy^;out9k~nSobABL?UFc5eSRh#a@e=_btH$u)Ei2l3iWV7e+Emkgs8_bOP4 z0}TN8JPPpP4RQVcjXUX`^MTFuHKXT_D6Scr&;PVJruYZT9qdPsnN8qN8X&D5BVsf0 zKnuysH6vCnzSA2!jg^hKc&|9chO}|(=HU(q}5IM&89=TG}TE{I$2#{$nab&k>=m=mX%2gIO4`O?3dq3feIlglG z8qIeL?2gs@n03w#ulgAmJ1_TFW%egZ&~#I{0%P|QIU^t@f8dnf(r1cuBPZZl` zKy_8ZI8N{AO@G2+h;gsim~at z*roJRHkAO*#{H#Z1G_Egc$Gioi+&lSW^;M&Q-Q`Y-t&revw88XhMUtbbZktJD)7Cb zbLrpCNV{!iJYhPf*qkmEdbVl(4fwmClFpl&6K>i$<`4Y?^BQ}$)w0T@F_Xc)`Yg`3 zN3}9d)4anwU)SR2nE50>%{p7sMK1&TPj61tSu1OY#(=t6j}-d2@;R!Pp_0_$+sDm_ zMcrKPgbAz}=tyO)p=2`}qFy~SM%pm<%ADIdTq)BvDOlUtg#Gk&|B;NN-FtC4D?YC! zlGXs@-dbVI3va4Z3Hk)>H{YDWgO?+t#q7VQIJ6BMT4)EI>DV;Z z_DY46F4>^Gj4n!eBiHHkHJc4`(-CZXeoJ5P-Y@STvl<5347hkU?ALCNH6ZVT!9dxb zFh58I$JclA8U4V^{I0Pyo{Gs>BHqNtca5#5?{Vb))xO+cDV=BaoZvRS5Yeay?)0QO zGcM#-G>J9$@&1|{ass^hB8T2ECplEV9Vjx+)Tgmel$+hZbnMIPHiYxM<>({yRC;1w z^*CyUg|*Z&hA;JR6C^7R!*kiV(GGM;%4&>&+8RXxww^O*0N4&ztne?avTb>^g(_muOWvZv4;C&pG`v zLPoBx!%lqgT6Blifoe5ws?B_sCPpknw5w!@N6s6$^z->vBd4C6{6;8$a; zCMzq%m5d?c7ZutHaAqUFu-Lh=)BTy9%1Ckb<)$7Op^RYq7SIqk6A#)sOP@b#A2tLV z}BTmWa=5*b<&>EkHW1*N~IOyb-8S&?sC5fNA&Q=r(Gyf7i$*_xqz@w$_# z@q3}_vhW=rbx}8dh%s)35!Faig62sfpGHq=-5=s zjpt&}$TTDi9;U{wP1dUsCFGzvF27SViUk-)<1&TQYl7N8_+~H$4qpyl#_!yRI(dR} z@eAknAW#7lhd%+)$TQAjcDu?ndn*B2LSK9uGii>@R0HHx#SZtyP7hVOwhRSSd{UZ zl!2IIw&IjO1lTmGf7==Ha+KpIOh+tZ5bsou?%i8*HybIT@-l<#W%@F}2~;ZswaF4V z429?-@EKtB>v0Q4W2X0pju3$EkPV%>Pz~ril+qUEjAPy4cU~?h#_94p6Bd5BD_mDh zXGes<&DHA8vzY=T7tg2&89w)AwxuUM!?Re}^K6f3LW6*=?5}>e5C)r1n?beXl90J! zx%W9*^)VKIeOm%fi3s!p&hJie{ZCO0?)z45}iR@Jar0K`R=)gUs| z&JB2j4p@gR^F}-p$V=Bkv2*9N5$bCNv>u@F)PS(J`xO)SycZj<8t;iAc@^Qd)ZP6` zunflx0iRd6jZc<+4saxyL=VenPmzRM^{Ja|yVG6a*Lqh)cPxWZ-=|)>PuJr6z`9JN zJ_4F3N!@BAqS^#%TKXc(93}5PRT;U;A|iDmZI$WbDDYY>H9;co=uQawi1UABarU z@q~MfzLC~vriw-D2yJ6;&qbXi^~Bx6tuM$!@k;jwqh)qn`$YNlQ$?;8Jf01WBKE9q zV_GdQVwF=O_iP#uwG^9pqh57;?+wB6Y(V3mjafCXt*=F=i+aw+!a9`(jli3}rJJ;G ztHB}_Wl)erXyYjt!5BM6tCK|D5gDUiP0jkEhm>3^U-+M7+iVoA+gS0Snl^vhb4#G* zK-@n*mSkmbs_7dRUy?XcKv=p;%~yQ7n(Z@!q}(>J;RFUj>${FUSaYn_o3(a@D}Yf5z+`!`+M20z_5F5D zvN(<5LsM{(;rb<07@ymVV1?%Ta@H8ZqB+1lwTqW7MWDkLL7v@@CU1;D;S|csC91X2 z57(oJ5x^wFlu3%hfD)4)B_=IaLQZnSk|YLJWC~XFDj{-0e2Q0ifYy#Q+d_3_HSMph zg0?IZ@1sBC;vE0=S~mRe-JmvU!m9|>koG@5i~nZx{`V!{+W$bMVy;hAv$FJI4Uo~V zy`A?ah2U&w5nCeG&LWyO=6Qf{znn!;xYJ8@mLi_4MqBtyZv10zno<@@sVQRA z`?RMtY9xH8uvSJi54GNy@`M?7Qe8Q(&5`8ESB3&X)!v>d*l2lb<~tBC51V#S+Ft@g z7p{I`VJ)>>(_TrjkSMytH%A%N!c17#ugzavS&-}VP^2&r_*+w@8vMS|6o;%@t42#w z8E0u?=EI;O%x65T`zeTWjEZSfxTsN*dVB~+Rdwb>gcPA3UO{T|??bi;nH;Gm=L36cZmmc`O(~7KsGCp%{c-=nBT&78LsCeC!j=ZL z>9D|vBLt*Tl&(zN*IVX0`^pBwIR+<)BwhizT;EpUWahAqJZj6~ zb#}7nxe=cLB0>Uqif9D?{=hl(^#Zg)vmJ$PrhCwV3PPjxIC1{lgRJi>+{59n@ACjg zX|ev(trf?8zQxUX0`p>4C9)&Ft8t?zbk0JkoRvtrckm;RZyS<+HDOmX# z-UwD>3G5P$csps?;N|>oi+0pn$&l>BP@ESiJGSrA&n?4H?RpRs5wKaFdbL$Q9b<+m z0bcirl)wcA00D(D0xI(`G%hP(H3EqV{aRavs~edyJlQv-G^Unw*lybntBE?%#()Q^^xbmqp%gR)m4_ok2>~pPCm!yuabu z*R7e!5^_f8EvPpDQpl2L5s54xqb4Q%lK+taXDYv~iCeoWT zCx=a9B)h^~>afEvpV;ZK}4OX&8t@9Lb~<7yBMHEeV-sIsY-wQZx+D zWI_^*F>L{dx*|FLdM>fTGso7naCaK4ZwZKss6h*D*trKo=43w7nM+$RRQ8uHVR&`y zo~u&o?DVM{xM57N7GRX7OOviZja3JCxGV{e$6NK{yTL;2mPxsS%iy#}@x^q0eu{tSUWQ~_MYj0i6 z2>*S|U_XQvp@omNR%FHc_UKZ+gx`=gU%=#IW|Q%D3bV_W;z%UX#gimu%#uzkkWxc6 z%$lw=dQe(0R?Ip2&Vdak)4R34woShErYH+crUE z4qhJ?APF_D=%seeuqF%5<{ zqiETGUh7kgh3lwi)Al-;B-l|RNwf|HRPcoVFok*uXjbY;GWK_q6Q*Y|eUpeKKo>(xSEt&xAPr{3;ifkXh6* zX^(IJ@`xifp~rGLSG6`j4!hKD|G!|M9Ki#f-Jk!l@g7kO1j3>!U`vr~>5$U=i$jOn zmWM!Sx_3lGiZZtLb#jh!z!Bv4h6Q}{g8EEcD4Lw6CC9!kOA2oJL6P{JFP>?7Rbb$!^c$~4dF!GT`l15twK!ZmDkwdtufzjy}h1t3B%(k zVggN7&5B*C@pkq`3~XP?k%fcQs4DNt!qUn{pB7)L7|kbNAE4fEsV{4vQtuz0pFY7Q zjaX**PZmpAHfBx#mc_Pfn@VIF@oX1coK;JL-TmzdkmWj-1i221jS}4%jKV7O-f0d!Z=H-orZ=~JR zu@Nw|nzp_*g_GBN>drj(m)E@Kn9mencq)qY=TQ9UYm4bxvW6YFa{5K(w^+jUetd~T ziz)R=xbZ#BdR|l5aKePySmec`!KEdU zqFeX(BXa6!i^1Zog?6pv4$#V&Ik$6%>yPQrf_uN=0~&6{#Iu{i31V(wCL7ot>(B4Y zax1J!=S=7rtpfGhX-!YIF{*Mv5jM4B>af~`bd<<)7Y9l-72IZ~RJVf3pw~v@CQC7S zT#tzZvAB=v8ym~u-crrWo>+(FyGL)_!U)Cg!zL5*FocBjL!3DA_u(eO#d6a2i(fcd zqaiE5{;+xysRzK(oF4q~^Ay(mB5EDB))t!1kNsdn#kx~I$NM2<7M|=LTkp-p0=$aS z1~_YsZ_dO`hG(dj?v}9#6@wgRCK|5bb8Oywykruc;jPNm7W1$?SpONL`iX!^>=HLX zKQu>(rt`ppOsp}QVY13(tD#c+YTIhe|Hf|*bSa)7_Q0kDqVnkuHn1<~WPIU>zgHHg z3;c82=Cb_zE)FnAdTP8^y8*2xZ>l#PF%1lmxys_}~4 zQtzOr3A*2WT3b*yKdc(??K0u?0SSUnLzn=_-*c;gvp#yYzT`P4KiQDDX=l3gAD=4& z7xGpgxtuyd(Qzw;^=C1>$IAgpiYr7@B4_O1Jrv$0Qq4O-S7lxHy^1yGlSjsPsB{4q z)Asz1BHc9@U?FcjLK?Bj2|p&d`J9LU)4v%xtn63hxWlTcFkcT6_~r9rK{9LK^M7gC zM+p?^d{*Uc;>nr1fF0n5>*+QF*4GW+Gq+;O)bRH}Wu>(Kc-@--dWkj&+PXK=t?5*F z@Tre{@pb?(@3=gfoFT9BuKkl-kTe=NBS~9A0Cu^vRJH#1_Z*ey`RU~(3x880ahZWN z6WuS~jX?~MZ@9*hN1Y{%LtMWxSIUJVgAr}sfz8i~!A{`R^!i@WejONQW`u_c>OO!$ zb(&qMjs#A2VD2m04KA#90uB}j?Rg)cRg}jK9C}xwYiDfBoOU7;7HZks2|qHk!}3T< zTyj_FY+}wy+15F1MEY;9_~R(?Mg@k5%~A7&kJAtDI4sM4IM@~bt>T&HULTMnnK`7qC^rp_*I zimGU?HGq05oMU^$AwMI0l}q<%^s2vpm9hfrXq1h!>QCb?>@{y;6+Jm@nisO`$a{3s zc+6uo`o1rPy7rNV$qpIv`yQ1txFtf(?VlF1cLp>f8Mr@l>Dcq}%}1iOvn|w3X$+L2 zVVs3lv&90^GoEv^7)W z4Q)N&OMd(-@@s&2Y@2xWz#2C|f5mr-r)Tly*Ngm61aGut)cerGh}UuibjdVd1bL&a zn=-U(G+OWbd8eIu!|$^{t-jFSv+Fj9PpH*Xu_O4F=@U5eJ`M7q+*CV$u+TL+1jBA} zqH)7Oil4)>611M4j1p`t^glw>{#PLwa7$Sh2*DGq`S0Utd3!Xko%sbbtGNr?qL-L^ z95KtDZYzu4=q18dNNj01!-nxcs^i98!pi8{)mJS^z(1h9zA=_$P<`VQwvm~@Q$EC+ zZLbA*Dm*Vv878a5(c!pxMUS7xIM+?@cK!S8PbC68 zOAby#Z3s$RZOLZt#AF`&&}ym5jvpHEx8g}y@RWY-Uld!mJU=WjxJ6tbHq9_=EJL1r zE*R!RTdT%jzNAK89^ACakn!uf^+N6-*`A_g={p!lmAI3zVKQhkb+?d3WQiZac6L>* zD%5%Z%Y3|TN|wgCJ7R#idyCM_MFi+|((+c;bWf9})1Ok$GS9;7Oo)H1t9u@ce31)^ zsYkAF>b@aFrno%xFPz50$1MF~nbV5c78cFx0_qvaa|xyS`!lL(@Wo@n~<% znE$R{l@j@BI%z||OG3~$ma=grRwYoxAK*{~=Ua2A{UX<{JQ>neY{iH3W4j6b1(lVN z`H^ZS>9hY~0kYky(i`J^7@yG7#6oAtOXh6`h>5m%q=zV^FU@a1@prf;Rd=Q>ww$Be z?d4dWtg72*dS*JrC^u~G5>YJwi)PsTvdfLo*Y?E~-TO|{-kRp%W<2_7-_QAN-ehNY z1%O;fjFz7*@Y>DfB)CDr?@;XcMlk$O$eYpo^ag8jd1%)tqO($``KN%B(j!GJZ)|X( zDsNJ?Jwe;WUE!)P%;4d8-Dr9SM(6M(w&D0huDtwbZ{r}EMbM#M#2L^R8v9M$%Z6#? zegfz?txERj%D3V(S#L0CJR!>$FS6GwJ>%g5?T<(ft7l$R=^qToFnpVtNFjT8vHHh z3@eZRzJuP3NYghfdj`hxv@$0nO**dSV32rY7du<`1d(q; zT+z}_*%NICOei!di4x`jn`kIM6I>y;1#kB*2>hGnqDd2}m_s(xAWtfu-g2mRWYyjA zIkQ0-bzUxfz6i{OO@^2eaG?B6CN0W=^^nXtSmJN+o&5>6`xEgdQ-ubQk__vpocDZ@ zm@|ppllhM6ra#Gm?VVQl#ou)cnUd;<(^mOq{&OpMD{{!@)aTBncsu1N*7ls`uNo)b zni@W<_(nx8YpqKY5{^;px&60<+7B@8Tdb|A(`h=Xf2d-+Dk+`d@=Cke5hsE$?*{^C zHmQmg?^c{B`n1D)coCCkJWR!!tk%rIgy0M;$oOePKM4|zqhsPI^P?DQPMhRDhmMO7 zP5VI{%t+%vdrOA^8pGY>Jpr^b~(`M)q z#7QpfDqVs~OUrnH%A^bt?K(Q%F>*fek;%r^4$K76n~zK394BpgLiy- zokE#^Ly|)xH+Cll+CH?a%TgNgPc08}ubu7VOGDF~1rM3){#qHtS3=pv^c}?dl~%)y z+Q7%W3$7h#-R>m6q32@<=VO2E!Tyl{{lgaHhpG2cW|C0@5wD^WJ|x*QW!p0)*)nB2 zG9}y4rQ6YIThpo7(`nmk>6I-i6)nmZEq?pUmD;lER9R92+P+h-(;kFp=?!rIagB!? zdB?+9LGoIAwLAg;x48U2gJ?)mzVlk5___Q202fXqa)_QK>>F106C!A)mi-;B_5-8w zLt69U6DMIL=&}DHwdhxg#XNJ~1WH zqVsSim;$6=jh{*GM)#I?NXD}ej^^GIo9atF5IfFYpO}y!5#kkkR(l}GOE)ktTWz1i z6P`2EwkTlp38!RzC_>#|-VPzvc$dW@b5r>2B|*A%%u;}86NoTAg~1Al53Es2VW?C~ zCZv2x)>zNnR5Es~5i<^L>ECV^SNSZ(l7(0Qd2PTQ}nSH)b3J-(uGqt@yUnR4hpB-PH@KIK-1EsgmdYr7^9 z#Kfa~SA(&~M_2EFTSh9MgnEZ%hMSw7lJ%K~y)E*xD9W-6Drccy(bY#$gjH{*wAY2ES^_&VLxlFd;7%^(`;bYQVC z7ISLGF=9 z8nGZdq;EWMgtNqhpk5)%wp^or1L9Kvn7--+1*A<{(wNNrTx>f3;G}F20w&RfI9(qg zYapQffCq=~%@t^s8?kFgy$(wM4DMK{?kDOj<{9|MMS3#_veIu6VZMTo>im-uBmCt^ zp)@b25t{la54NXg9sv{)d?T^4F&xSs#QSG2*`#nRfm5PH4)F7ugQNQa0Y2EHL6VxYET{IKiM ze!p`l^&1zWHNl0dAYkQA1U-OQFcoH~>$*JPs69K>YGM=wf$8sD2?;viiPF|Ajz!Oj z2mv%hnxLFLr+(71yV5MqY#wt{vocsr0r16`Y{9}WnnBgzu8!^!2cMhl)Sy zw{~7$N-noP&9@5zlra9HyEGpQ_T$i2?>5Yd2>PpqY@$0#=rv;v`==hoeqy%xrlU2J)_l|Zn9RpO zJ3C9TY(YO=Kf^apXiezHMAWTA`DWxP=ss{OJ=sw|dDN&jFye<2Pa61_9mR0bn;8&g z<$x2%!jI&FbJo~W4vBDYtj?YLG6z>h>Oe5%v(q1~vrE%5Db2o{LoU~?m}O9^WsTJO z9RXArAEj?9l%Y7kLQsJrVKT(BW{Iql1JAx%`CiC-#RzN~eIoI5fY?9!ij(#mxM zz@8-d95HTd2b2!)QwFx5s|+RNXP=d1AC>%iE6>03t?ke5YcN$?6koiC-d~d!iLOlf zt~kI!?vdA>EORKx>$w1C^y;Z1%*`{~sP#eg!1Xd|rB|UgL!g6E{5oCxjHhl*x9`gu zUv^Drw0h$PH2n#WkO~AQ-#c?z{IEjmYXV0@C`uTo!72ImYRnoN!gs~#&O+JPq8AV1 z5E+>@(AIlQqJhFyYQib%=SdgXxZYm{4OBbwsX0`MaqzU!8OHK?N@VE5W!~5^M=+)L zQjKilT3m#~y!x;@5_!6;=}@j?FfiX19~|+zn(BL%BSlnb)4)+yRFOzz(Mpu%a+Kvt z-+()_09$$X_`H8>KA-n~iRQ!+QHqV=@^BL5SR{B?LDCZS*FB#_MYwLfH$>}6>jDEO5 zv2MEb$sjw;Io-m~m6jPg6w)PJgsP-g@`d|)b%8MbZML*@KBsau**PdZsG)QtReclK z;ZHsso=<_!s9+*i5{MhYt1vX%5B`1o(fMI7dlEHMHTCUEJA^TtgYHWjX~IfipHU2* zcG#$v{8+BZ$m1E%?Fqnt^VWDG#_5;Bjx*f8C7aRF0Thw)o2bEI?_1MV&i3AfPjW#c zzjh46cK@ynH>7;mUe`4_F2;ig)*2SN`bM9O$bFsbHtvz(sFc?P1_fWsG*>isSN{s7n!8^A-13Yxu{gCao* z7J;Dzf(74gMHljk=2VDjLdEqJ|4}hoSQ|r*CGCw|qcdtbzmRj4E0{KHgbg_K?D9dL zA8^4dFZ5=71rOd;BI1FLOa5evepz2DVht`69lkKe-w9PRzST7TiRx>t`X4ECf2$~u z*N^D)69_`l*r~iqIOca$IJqc zSB1G{fc6zr$Ka6yrPbzZ^*J2^Ff;UdW{(J*_KoVnVKROb{3EVV_t{W_MN>;YHgswpP4$j^B>JZxSLzkwp z)}dv1M9a}2V)kz3!(0ILLZWsMc|WmIhgFsCi+xPP#O)I`Ow`fW)C$#+fZ>yMwuSzf zjsfe%OqxLFQ`>c(BK(fCF^(*IF+4$Xii-p8CUkA-1b(Z*9A)4W*h{YOfZO89MRA1S zD7n*MMx<&b2~z3UgoqHN&gZQC$X_K+TvLV>vRD&wJu6i)_09Ie`)iRx97~D}V24AMh?^TXMxnnG z-ocv7&*&u%TM#^8B%2!2bNv$L?@awea}MKh0i@|7`iual+ognSo7I`EsXwdvtg!fz zQLUBi5EX@g`y?&oyB&+(eR;>rsi6L5AnCV0#k8VdSD41pE%RsQf9@&WbNU(6FWH-K zpCV{?{{cDqDDWN*qtY~rJ{vTp_|8yZ#Y^Y{w<=$36Qu@xZ~7-?Q1Ln6g??$<8C4W( zNtz1+{Q;Mz<1{)|1-pDBg?HPFvK;!Mojyv=NDR>4>=OP+s`pkw!th=MtY3Z7755jC zU8m_QRkIrT?kiI*v20I?C;b(q{CN*JPupsgcMDhsj4_heo-4fe^@d`FuFEI^>>p6(E}X4>on0Z^nAdp=a@lF&3gUbADB*w2T<%Kr zp`Go8)CDt{@;;G(z1A)B@n*uj6DkU%=q#MpWv3r-eO$`fP4~Mr>4%w#z`-TvHY(bv zo-&XVV#B6k4u#^7%-qdtsn}A>`?ZDKE2Iq)lB3M&KcX$OYhG%A#L836?u!S87@R@Q z^k(an%ewgDfl)v(dOR_n`6K0BigqOVr*mzL0Ib;)dJXDtZHxV<7&s&dy(An=K{9|_ ziVWW~(Mkx+EOOT;3d(?6=^2R2#&*f*afbXV^7H~vknHA9nTPL8{{zv}BhSCta?$c5 zHv>6#`rMS(0huJ%$zn}XPKn1Q18QLp{b9c|F|V(4IoToxWU{AtDi1hE@Y+Uy>}_eC z>KEE6L+xp@aSUnH%Hm7X9DIoi-XEfV=6PG0_F8k^yiN;^DbPk6%5IKR=*mbn4~oji z^lS}L019i{b6q+*kWy9D?z%x5k5R+ikHQzUPc#u%5}T1vl=1V1eOsBduG*diMu&U} z+ZcNRTY=YkOHOdYysWN1LWt**CDn*`X;8Bt2bpxAXr9dZXTRXQ{zFt=*YnWnmajI{ zJ>FGM-fU0wWC3{X5+QwQ0|&#KSJo?SXmA@ht{9f=LXPx>^nsuc>na%Q&KT=5bf+@J zr;E=;FB0!w7oIceK4;W>&ZHMvq8rI1og~xrk{I{UzVzdiy$H}=ojm8GEh+r~@%l$+ZA&kWcfxmBWqmd1H-r&P#Zq?o2UAUMM>q>t(9Wh59L7>WOAdgxn$5hOA$C?7Ch5 zzJM_aSplDd%EC}v+HJktmW?L6ZmFnBrKCpsL*wFGjk~wrhuOjd9gU&3K!9C&<;}&! z-yC~`pP}rmH~-X5OO|cE&`z6O&AqHEPJcw%3+*on**g0>CH#Vy|AbYP1(`U~S-J&l z@Om^Hq|o`c-0?1F4XTizs~?s19#bm~ky)MkIRPEEu$tI}z=fWm*n!>dJ@-uyOgP|{ z?l{*isCqQPWkZ?C4av^QJbW;T;B+yb!s5T8E!d=)%$o720}SIDX24P}eMtXlozawZ zuN?kL%-^D{2o@fAJ~8{Z&aTIXA2ozgaNwkN!l$;|{O7in8}N3-YJuWkKJ37GrM^SR zK_ZV$Y;0Le++y`kJQjT@;tG$#bDtO$FFkjxgPq4N3a&l#CX`$p`_=t2+{X@>Mc47P z%ILv9DJ?SuYMn^SXWp8o)T}ngZ}S5)z&UVszTPNGr+w zyQv!HVFx2?Rk_V%r2`R5IuQ)}-PHAJyDfyjohVS#2HtsCIpDJ84a$zT8xWEy!Sv#w zjNnE7pnA+o=GM3kLK}2Uan@O%a?#oR$5W?nZsi{$R`5#Ja_w4NB*ho*Leu3FT@m0{ zcR#6zbo+acoa3Y^929%u_7~6RUudUrcV@ZB(lnBECG~A-p$pr5buLS<486Q@8^y`z-t$U6&b_!WN)Xo%88@M`js;qdFdZLD3@@i-y*cIOrq>?l zLOTCkw>=h5gz}r-W^xJdKJ;;iz8Hz|mMII*jC^HfkS?XcD?*-YI@4jsmUEqIy0{Q& z8#r0H2xHV@chA>VnAj3@|FRAMq_y2Dr=9*O&X3S-Rt%cLRwjM#lPPptWuYu_Y~UJ% zbLy*BiTpP=hr~d_tI*@)XwEb+z!$65n0KU60T(-u51S@)P+Y(6Dq}P|-K-nC+6eoH;$NXY5(dIRPfWX>Q z-cHuwD_YEV4LKu|H|Ie0Y%!wtTk3pmN0Z5?kVF65V-dyt)e*@AasM}h{_b`fpyVux_$+|KTOZa+Ki+ZB>&D8cVYqh;!I@+%B*1?N&m9j zxBPilO)Umc1;$uxFy&?RMjQ!P2f`S z-8ro$Bz5UK-OOYyeTD^;egyh;#)@Id`o6~1V!x0j5l{+L*NO*`v(?BLVD}l?<{R!rQUwAKgDd! z$FArjm<^BFm&A(gK~Mdx1Y^1>a`CD6>J;)7M~lL5dH6=1&l7J=cj`q~BZKQy^~%T| zGsp`+-;1@!i>k*9r{`X&ci6FacuGob&HAa|;3J?|ys(^5W;&eYz%xnb!u5=jxscDu za}GzQpk$CIo*ev|_2xBee4*j^3`kRqLc#q0QSXFWce?b(ZMU;+*}uV6pTXn*U-JH^ z3L+i#bVFejsYuym<9{7+wp1=JOhJB=pL(Ru z)bo7I9!IBGOkQSNh_oOx$vW&9I|`aKdxBZ->RB)M=%{0UX@J$BM-%80>x}c^S;K&$ z!&o0*(3YIql4I0)H$lUqFyh4uG-sz(8mjuXo`=oWybK(=mI{e`gIH{NU_RrxqsSVU zSNu0}{ba{26>i+@<#o70-T)ZJ5A&j}CC8BJ&aJc*M@-0(jgVy|a*#Z-?f0bNg5(ha=Fo@}oS(blV8+={fUvtG`5 z=nrCEFZ?}+cf=g$1_W7ff+QP9Kf8C!rGURwHxuSa2kMB;iSsJi1uYn@f~{)bnv9P$ zF8XQ?Jave>zLk{J5IlPQu$PKupNXGE-n)ZCt`4b@q-{{$<=b*))E4t$712ZqS8{=u zTsw8b1r$fVE!o_neKDef{^EM%y`?eA7mFdhT=z$1*Xeb_VxfwF+!hzIcJf>c@b+)f z$A3RxeBqr-%-@%kbZO0&iUW{6?)f#=-A=UMEYF=lc{i)HS(A zhi7p%h^2>RmUUeqO{M(u#@KC#gt=D{Wg6XZh`^ZD%qqUU{vGLW+U)AXfOODJK*!MlcEVSGnwvg_g5HmK;ai0ta0$yzXKixl_l<8 zAibdhrq}Rt7fPU%0>xK8iVnWTDnio|bi3T;e&{0<5HO&5Wy0Fmr!kB7{G?oSeZ<`{ zacJJ;damMbE`t5VwQlRWRlaT#%NUF4!rwYiwBu}_cni)>$+$k5#)Yrd;HU}ph#oTy zT)GdPrwwxrmiL-XH4njjum#g>7lS`|l0Op0Nwyb=t?C?3bv$nJ_TUkFh8mAf z)RD+i+H6F15TySdOkOlT#^HGRcGn`XLFMpW${R(sWHPA{VUgfg{50poW1Ysy5dm=( zp5csXFDz<&h_+~SjABj^{y4oG15X%FxqjkTN!|nBC>vhOO0!ABK+@WPvY_-ZngJzw zauMyudr8!vwAJQY2_T%XV9KLX&LB9 zQGJ{_KHc!ncNKf<-6^JAjn6x+@@d+wdt$qOnf4SaO+y~U%ARVrhr4ham=Itu$$LH| zg6KU0KhkIOoP}}qPJLh^zG#`mO!xB}lNVs~tRqH)ml?a39>;LCw{BM?=Fw)uBupZl1}K@;m~d~;x!v$F*8D=y@UJP{A_4NoO)H80CUNLFE8s&C+n+%(w1irf zojc0Ax3y@XOOSmGdVKTAQZ*WwT(yaymsyF_J0;Dflir~mZZ`RBU-UWkZJ8gI;QA9B zdZ^RSB$9FRF7bdN1=-Ynu^)7l@IISTu}w5wvh5M}%??lg14K7SyAqv9KKB)LGXvC`SH2EI_<`r_=}p_fsrbizcG>lcp@I)ij-T z9JH1Dm>;Q8UwtRZ-{(?DiZ02E&dDpteIL;_ zZ8h9(1=82A6*U+XH5AHL=Iwg2h>bvmQs!P8px9N4m8o9A-Y}QYB3f1@730Sb{Bd&Bi;YSNof^TUFHPTF<=j zrQ+X&yM{;jYzwlN{QJ`ozKg})THxi-^%ke{kEF!7R>I5cx2R2#He(4z z3`aY2Mm~48;Td8LZFkpiiCN6QznqL^OrlbjqAx@}=Z5+_f2_hU6zX^+rwkRoRD7Fv zBe%N4+LeC7GRgg=YUTn684IRFh1;az<_7BnR=jsAvDK}kGErCVny^n~Mjh7qirlq4 zhxY`tcX9_T?`~d_s(#f`Y5nJ!Y+BTi3$s{}z{4#U!IqMpctR>=2ft1ZblmP^q zpqxL|L(6G|b#4bP>$y)f96sqELBI18Z7V_~o#(^69>9mG&=H?MUEDHh)gn?$I984RC=*4jQLy<75(LN@y6pSU?7QH%Zx~{PU_@^`vF&|{eB1IPc&X{ zr?-Y?8J>B_q?1up_cQ6%Wf4oCpI5ZXs;rhQk9B}fK*0*y8)I+uGZ9LYp0H%2dAa#r z>#c`bbmL>hA{P=f3re#)H%#SqGN8!jWy)7821?bUMTR!#zJxn#gYI5TicKZxq=UqH zg-dbLio5BW$IXtC8tnCd#tsuHN}utnQc=gROT1{GiJzo|2;3^}E|uSN? zSg^>aa+vKFG*)Fc2PGJ;jE}&bOgz1~vF%^D%@q)3q$IZrX)DGxP!>)GO=v|-SXtaT zq0l{k_T|aL1G{rUq#a^|@8*pM8%a5&)@ut0*bj4IyV!Es=1vB99)4-!6q#rnpjEfJ zR)aLDlX9z&R`i^pESZg;9-QGh@{^adA-w+pEm@Fx<1=0S2SsHU(K^r2nY@?()g#=ZMoX6`~_3xx7(W-`@;(t%c%Rhv>R87+zS>Q>e!o?q4sgVURsEy&I$aW-dK6*bf9$r| zOb&3m`67O>(RZlh1o8M@e%V%zcfgHV>74GjItL>`=}fB?^pnTV%=?YT&y`3k+Dtzz zXSS-Dbu_p#feJBWU0}s>kqRtW0~B)w?cQ!@=IO6DFdyB9F7@VOP(tn&4`B4u?B)Xr z;ta-n6!pa(C0(L0*RSKz8fTTTQhD@G9v{yIGMtJ%w@=s%;Ec3Ri(~O(K1*$Goz&m;1jk%g1Y zLi4FJ=&11r?qYR&sJ;Le0YzvE6% zZH}j8sFAo8J3d>uI9|g4{u5QRqZOj&(SEHfVO;N~-T@)*ak@SUNKyHj$af^JIjj~{ zA1fq=*D+3VU#5&o>TCjRjB+U6AYPQ+EADs?@hPb(h$Sw??XE--eY9jAyzs&jExn{8b!MBaiZiOUv86l^ zf4wA5@=uRM7Rt}Sl%vw^#I!6F<|3NgQs%TX_|DkBZE>hcOeDpjuRP}0>uw>Y>7Cco z2G(QHjew4_q$0d=`?YGcNpxx}ytSfwKl+9Z8bM84?hgQieYzFFc;%*J3s?;&Da32? z$xJ4=w>3srl-+Na-L%{T%gZW@(Ps0aIRk^Bj)k=0o+wB9gz5NbL;dNW3$ht`i#Uaj zfF3uxp3X#&DGQ)Ws6^A*83J?&>B^=0j*pIg`UM&s=ca@5^hpRW$I}p zMudC7FW_1H{Tc$oP5dX+fdwUSc` zm2JBI=EHI1=FdSOU|QQ3z}AJX8SqN)Po$Vi_vxewY_AMPS_a$YBrlanU{&cR{QZqdtHWsyvFn@NnVI>4XupxGv zqDCQ$MiMa|{%@4O?aFi!*Z4*>Lg#Z0M4%~MXNTHLNY693b84;rQcWkVbyWD2f12H> zX8Zi^5pqHZR1*{Qh*NtJQG>*}edD#qdhF+4iKOg!63CH!^sDoy|BgL*i#UWFJ^nT4 zQXKeN3F`5_69W^h_wAl^^KlsY{CUC-&_WQ*PcA^`4xf(r(>7}D8+uy)ol47}-dDHX ztqI?iY;bNsNhd!bDnLf6IEPLcw?h9;(`r3;g|KVggyR2(;xBog=RDjC6)E93WY)B# zW-FgaP%MN@ifb!}1T0KcS4a67cpBveoC{%3GOeFsfC*P&XY-I=x7r^yMB_E+Y}LBZYjEA9^kNa%oBHgSPuFNrO0zwuV>svpKkG;{w&C8CHYm9+ zVK8yz`LjR6j!nhHtGOMibn1}oj0&skWO36tflc-YncrCUhdl-~{2#It-Uk@_DDo=SHoB9^Tl`29ZXw{>g7+IP1)o7)nG1$+N(*QwCw< zChv;m8*KKt2|0xN5HsTac2Vjl7iv_EYLLo7i@1ya1hTpYBzk*VHfqnKSR8}(=RlFp znK1)M?q0XTV%yy(W$}A-##&TtO$wzJAAj?!h;hkowe7XEn|e2zs@ztL1~CQ2ZYd3W ziazwdsi+r?E1^p^y=J$>0??(F{dXd%IhMotPR5^(xbLr{gIEHazi2O`;aeue7(39> ziuok3J%ZupPvFp#dS^<@$eDiUgf|HcgBYF44L&_AP3(e$4YvvdTU?vxnRN?77n!3S z0%!E?sXaA-I%zqHb0i*4|!}N6Aj8^-8!-7(EJGS1x*jA;ElC^YE%`4S76GzV#otBQgFjlfU`(ax{gSuRz zjm5s`r)J>dT7{q8qvdXl-cc%M5apGl(0tT{-JEC%GV287_UADV;MIga?(R>=m_`(! zh(Rmv^UDB61iR=+NdDBi^hV|LzH=>M8pDVrv7xCaQpbZse3lXymLlmiATY?4sC>vrkWe%(rTLn~a9UL2vEO_y`@E{jUp`}5aUao%94QD*-A zKXhS=;sW>Y(5b-;?EF|0ggbL^`1Yf^!C2 z{icGsEAM1=OI+4AFp62Cu5;2?B9$6vugEMR}OSTA1r_gSnAEW1l%rLdelRkt5 zqboNIkiAA9DZ)HH{YGCn*CC%Hxo7B5B+otzjj`WorZt(M5%P=DS0@76<4nbtna94* zc3@JSrvUTd(o(y9OQbWP_=Z&buaO0T$IM&5*jS}MiaSyaS?VP z;%o9mW0a0cS-bGQIjV&kk2g|J8BD%^g%EbAt`P zN5)KB##CE|T$<`=X3|-!$5WEL6C$Fm2cK@lUH z2sx4ew^8~(uJM1p)6JE^?mR;3(c6_x&y`Khr<=5O>C*}J=d10|~J$4WxSIkdRXmwM))&+`%QE7bTii!QBd*EM+7W*8_ z+HSR^B}a=A4JkgS6w>vxv7r3l+O1i!9TA6qql6Pv0ClWS(K|2o+i@?p-PUCCgjS~rmpNIZrn26Kyp@|45yo~{uFAg^SAxdBe zXJ)i7{e2L0Qb1CqHRNvjXI<*e0j|8puUBUtMMV&tK#S8iC1P#Kx)E5Ci;+`yAB+_m zZ0=P(#)Jp%Gukh73-%v-HEF^JP@oGnba;iFg=}vEWpH1eTK$VsaHBmoih4P0jL3u|Ez)8ge3>*vuP90rRUmeRau>B>@a1Z%b>~L zYI{=o?fCY9w4o&`#~E<5zm-ox$l8dF{4Mcf*{9TxCOKQp#hV zu?tQa9@U%I)7)cLgU6TK)$%Xv884^`XO|w*I~;BMLxd>0?bt7GuKer`T%+eW{i4B% zf8>&0e89fSMbVJYL5F~7v28TbGv*nZyqF0m$- z0+%X&WtjBnQqMCdYYHi@}Fi69O5+Az@;tJMv;8aqdII`?$&VQ+czXFPR-%x>7!M? zE&_S%VYh~G>X^0gDTA1Gt&sPoW-XIJ!&`mok;+QkEg}wKR~E5hi>o6C$ExT=6Rs0etW~PQBJmJ z;Ub>lq;9PHRrG-Lx1_H-PGCbp9beeel+s6u=n!coy7~#D9CA)wc-FH-vnZ=&S@M;1 z9tTF~%cWg;wg3nAsN;0B^OiW-=BMls>4}elTAxP?nK;j%BC@ov_YhV72+4KrhWKl$ zCYQrXH@WdI6r z5JN0c)`XR8QONTy6w+&e+Qa}J-Wkwq9Sa_)wE-IhlN;quFmSk@_Uah=xV=sMNxn3Q zE2Yj}gkMc9amphwJjO=Qngz zQkAiH(b2y4s_~?CE$U|Yl`cO9<)UIPKE{U~)L|{!W@;oYLL|G&;Q7R0$aoD$YnQllSU(fZ03M~J6v;ccru4M7+ z$BpjNtfmUPyw>Le8&9lBfO5_YpRSQ3BTEt7^SOBMS`s?dJyMLx;wB!?4z-J-7;TeJ zz^@7DOQD^`dq!!Kw^sMRC|h!8^VG=IZs~7}9}PL9u<(wF!;1&&6$~taHf+-^{n2d! z1SCJY{~WquzI$l$ zixb@0?9T4a&g`G}f9A}YyytnY`?{kK&$a|wdS$W#{t8e#wm!`sEE@;Kp$8^eLm6Vw zT>}#m=v@RyF-R1>7mhVj$%-PN;ix&dO-)L$7Lc*lnJmX5t^zAF{(+h6>!gk&K9wrHMuHB!iD9Ore&IkJrnuJ|tm^J0>Xgj8Z^&bY_f*iSVL? z@FJy;#Zw=5s8DG2P-s_CECfH5S*$Aa&z0f&4yzd5ij9}+Fn{XOG%LAc`P0U<{|P?o zMB!{$f%HRZns=r8*lrwO7xnR|%&mz&T2nMl+ivP4#nt2Zi1c3tGszv^FS4(b3CzCr8Eth7LNYO~ReEvNY^W;nFy@da&D%8m)>MZd zUH>oOz;f2c=e#j|ym*0!62u|v8oFqOai=i58LJ7>NuA9!mtrVR`ca8&4CwZ4lMTdf z41KUwC<<#SosXD@{`xPHKFB?FtTlNearUng_SdV_pUiQF068n)u~I)dOK_DpO1d-- zqZc-#2ZdM^vO;5&vkUXUEa3~f$Vd5}aLM&w+>s`7fB(}k9ZA8~k2xjOG_KmX+Cko^ z8%fn^Ow0NjMMuBr%(x^S@I96HFLEk-iHP31BucP z?1_?1jpJ`P3$1CKiv)jT@LC_E*wgm&(V6PKd>rNe{Zn;dT}t6if|ZL=l-5x+-F!Gq zf0R+imG{-09KOPd-lzS-n4r0I&npj%SN)7xWTGbaB@Ej*WcDLo&~I0QlpUP~$1!`h zD~$2>HtLFsu4JCjBCV2jPNI~)r5*9V*HzEKRuv3z zF#V75KmA|RWLo-rKggMmsdTI9brp&A48h6A@9Sy{#(W1$aboF*@2fWNPOmdGs1#S* zI_EThmnEtim00G-qkO6PViMg$g(!o-+e9|KSx2TvZtdHUHSwZLs?SFl^2ZXsXxsns z4HPx}8cBiNu~TXm3EA$w`qdcZ_@W~pw7A}@&|)+4ZIkij$g1-lDH7|pe)G5`PqN)s z1JW!|wzX|h#a&nW6JrX)JZrpgCyP#{qI{w5opJ7+naSR&__V1kir;8Z-_T(uaWujq z=^Mun7*R>V4B#ByXWhO}T99>#d=A-saN>ptTCuEF_oPk_*`y~OZ6?FnyX42e1hkEGN78a|J&u7hxRP=TKIkIKif5o$Z$0I*(E!L>kT)*W!RUo z5Pp8~M*a|f&B$>lS(KE_a{zP*Ly($)d zy^Na|Sl9Wub$3$fs=58;(I&0%-tye?gv8tLNz30u^t6C+a=?!8yvK;661`WjMp?)R z5Uk^o=%A3)>~N&}R2Ya}2$o1Fk38;3F%8VWB3O(YZU_HYQkTt>cHiIuWe_!J{7_93 z{pV!wZ~u=-=}6P|aPJZ)jP8T}yT{8(359gr1`b>BzX}=?j7yt$-nQGIu_(5S{&GxH zPfb9)2FDs7ZrWz#hAjw-9dPpoXJ_4UHN5a>+kjN#ePaV75E!}-49p;*T*7{k37vKP zn?Ta8V6rsr`m81CdboF5G%^BStLVo#{Oj@R30=_(bS=onaAZ1IN5Be|)+&qQ_t!`e zy1@nSuID}xQAawcPjBeTxPB+4P$XhtdjQwf)Oi)EqYf1FVk3rOYwN(U&nNC6IO8sG}9@qN2{qnM^S~3oeg7i|Gqa!TG5VZR4xy;fH~z;3EH7hS8t~y zcwu}ojB+9caTO&P3>i!sHzBc%)qljLhW8@6@b>%PI(|(con;hyJB_S?w#oBg@pzi| z_FT0yG7!y8Ta`3pnFr7$(C}=KCCp2bLFhEKIzXq8*lhXdkkG+2GO{$_)-uanSe>7soTpykL*KZJ#KM$tK~)8o3gfJ?>TUyMI)!TxPD^ zp8l6)O`knRE$e!z35mm?htdP9bmBrl-Kv`Q(uigfV6U!fADC= zmZ=IWcArt)qbtq5SD!)W({-oSX_5|bPbrbS&pG?C+qx32h%%-^P6<{0z8}W+b%vWM zm%A#NUl;!)@?$!bYB^Q;!h97w-^?4x@J*v%L^XV@-Q|4! zpfGmIbCgM08UTxEpyKwlO}lbsGy>IeHEN55W6*k5hRQQiKb2&xU?IBQTWdtx%7~Mq z;qPoPJd9I-0e}Zb1O)N2V~7 z7}Z71vks_q)O_>)@>-8GyrRf}$5|Ne;j^`YH3fz1wrz;f2c!}*#Ba%u5tEG(s;vD| zEz5E(^r-tyKiJ~lKg1utf8Wx9W9Xzi>usrhQNq>R%jRxSxo%KG2NoifSqG=jhn>P$paV3p4tqK8$T~#*y6La=g7AL8sL!VbWtEgMA%zC($k{dti2;F832I0xe zf*e6VwYQ@EAFROE{mH{lfh5^$*7Dg{EelpzglLy;<>_?HJd+%n&@vnT+?ypvh$-~j z+uDP~T~@s+n(4}CXa9a@bmOme`^j&}{tQN-k&8yim5Ap`-vsZf*RsP40mgAXAhR@9 z#eh~TM#e>W_;t+JQLfZlwC_2!TUCxgGTv^6w3SxW6T<8Mn=eQ#866XWWt1%(9g=cM$GIg{rB; z-XHRG#Z4JDpNO~FT)taw1*nTxJPZ?bk6Fp2$+fCadZ<-7mC!6mHulQBilk%n6s^W& z&~7{Z`(|K2C*JFdx?M(#{b;}37Ju*A;( zqkA+EqB@qD#datw?Ts)~vxgs1-JFl;n->QzEAjf_tV|Lj+K+sd*>-TG(!pOjHJVjx zTHmAAWtjUejqTLTr(SqWp?OSgK&T`RKZ!GH2~JT$W#3S4nwP6s3+Y>y>tOEx9UOON z-h8&#e8rkXV~{1w?7{T7r$S#kqCVc5%-n{4<@^j9vS+V;C@AVuyf8Rvg*^S*;1-z>?Lr7UyD497sq74YJO!HlCC6u zv1eXBdIZ9IsqfElm+BqGCY1=R%L^}6g*+5Vb0vqTr@M%We5d*Lm@BjGXif2LjP|i{ zk878~!k59)B9U_JGXj|cNgH8e#^K()SjXjzg1(f9~w*8_m z&sDc+6;D(0Ok+-168#a^^kYRH&^4Up%SXOJ5NzXn2O$~i@mf;`F2`0klJ+s*l?cjN zn#1p7`g4!J2euoW-ilBgJcZpUwiIw$v7{rR9=6bS@r0U z_fPI%O^ixd?ldL6Nm?GM5YZKIe&eu^h~5Q)w3g zs*UXEE=&vXOrGlD+J$f7B5@|GYjHqtNHzbBOXvX_io}QAT`|CJ-kkk~7`O~krsKVC zESkH1jOJHEUr zG3r@Ih#`(Y-WpGgHdLllNB+(K#+?6kpUb}b8h-p0I$sh4L(Sv0Ccon>Y;6BC^fh?; znC7HAc>bnvs^=a%=G7MrV%g|Ivi*ScksW``F>NV)RD~FbF{#WqVdNJfHGzM5Fze;* z_T($7@0;cETDZ#@f%lEk-Z^eAOuz6j6JVrv+Xomxu~aq+UvVlQSC5#n8?i`iJSeUG zJa5Q%21nsWmRpX|JQ2Ez_f03w;+RxndF35XUrPSD_i)jnTR=f>X zP$}L&qLIdhs_~X#OYtj5{dVJ*ms8Cvy*tbokkz460_`HN1&WJ&0i6h%AH~#V>#XD$ zhFKPD_IpEbZZ*cf$?k?C9<4kbhDh-qzG&xylZk?pS~z9mEJIYV0BkM?e~*pCkp)txaEDlugx7 zGoc+8dDM<|@P85(hR$m&2DM2f&{2xKo<|W!TTM{MI!Ges*?lQIR7MA~x~p3Mqe!%6^mB48({*s4O#{GO09gxg%2<-Ry#UmvPK6J8-F>d@K?&HV%e}L)_cX?hH6;O^Dy=HtaZ6YuW^W{}lWRh^i#F~Sej>K_1{@%sTC z5E9N8Bfy-kqJ*g}ig%R*qWHqg=>GaM@6vl4*mjZ=-Jal}kCuEDnzAY z2%9VkNB)2}WKR-J2K|GW+?6&wsV~1G;M%I(%S;KqG?H^CA!W5_%l+PrZb+tfse4R*yQ&Jl#{?;;(} zX_JXPw2QU$xW2z16(T=j%I$qZxzzqgk(u(5B&H9;@spI#h&6wcnEg$8vEJJv;;e!y^mdA$+q@3Zl#79hVOKsGdDd8>pg;8Q;l%iWIC7v6ue zUB_uJv*cSL;_>hCxJU7c8gexuWsMK=EI&D^juN=tip3zcZ`AG2vug)a&S`98=)XS( zY!l)PnmVE(b<%vXurHacxY4<)^t?jQ;B{xTbSavacdH6<*(PcCnOpnH%uZoY#+M54 z=LVHH#}a>Fa70_Iwkjl)nVdv%hD zB~%@+sXM09_jt8KS9?qYvC*z}#q-D!uWlse<44<)Wm{9}6V=hybd*cx{Kod4tdm+@ z;Ez31D>Q0%)B7|BAj>4$=<|*7JhZ||mm+mESJrznI))beZ z+P7Sr-&sdy=0#Cu5A`5D^$ZgQ4 zy*N5vMLed64b={OF*L zp5O-M(9`L63}tR{RObp6S#VY=ckLiL-go^=khGSCwSIYV za`NUBcFzH~+}*4KM4)P1A^%0m+kVmNdgz$|GUYm`!v%2D1>bv9>Nb9}Dd*q{73#q% zsB&|&;+ATg{x86Xq;kz0Q7;J<+&NRYN2`*dk8r#872OTMEp5zRxpV8OBGTRsSUZSu zQ~36$L{_S|YClGtHhgF^r`C>2d7$gnuo zZpUE9n{ffZb8DKVQc**$e_6taIoG~D0eUIN&$;hiajh-o@$Oc@wN%0pHuZNlV_|}W zc|}a8H(F_x-x+^e5;=&d@{c}JsWtVlDU|%q48HOsk(1~Rsq-awdK-j9*m#^~FM?m& zUlkjBUrKo;+Z3~mR(8Uj7F_H&{RVT+Y1K2By;J1RG_LX*VXl7_ySVj1ljlbf%oK-z zKNLSeCGkD;m>aB`uLjoR2drccc}$#P#l+1!e2+d}8?>T^?s+O1=evt__3bPpNPVNuH;82@-}1EW%l)f0Cu}@swe;VE`?=H;`T9U{d+$h%vHz$Hg#7S2 zaCsxBAVDGN`tntM`dM+=0RN>7uJ2!9q!OiO&n1&lhZBZ!96|S{Ttc~)NbZ;|t(n5* zOiRFN0ZO%{gCs%I4^pB!!k(vK;(V%*nbbjj?GgP|@xke#r^oV@Cc+Hi&Wr{MTrPAs;TsyT0fIyf6Tb&S5`@DV44{rY6p{t#Z}hKqY@zyACz@w z%#pNTjo)m)!Bmsq9>NmWXTDio?sKqNNM{elZhAz?+6k}z>f>?6D|#X}ae!y#y_Km# zN!D|q(>n6h7ODHR2Xhg>$G*m~ay%wxLW!w5+(pe6xnuNf-W}}qT|zwKFrjJs7b821 zswZS+=W|ZsTE1fsYc83nbPSx656-Lwp5)#nWFkwL`P1}^f(Bd1HqK^r_WoCh% z%0fTXN7NgN%6m@^SPxb5X5pB`K;|i6)O;7Uiw4t9nH<8#m1pmi-NdcyXP>8L zMb7U*(fjVkh0=yBb?@Wx4%>-g^!1uTw^YM8)BT@GpR!J%Ih6B9FC9JbG6<3<;0^HF>fU8*nh4Y{5`eN82-IveD%n(TdZV0S`_`0-Y)_|e ztKQ!joM2qgAuAoM1|Jum!U$_CKvuV6)`0sJ99$2ChL|y1P4tv}c9=Bol5b4e52{`S z)-Shb57J+`7R-NgjnHa46YkHx{)%&d+b;HF5|8Fy-eiNpX#ZQzx?`8Vt~vSq*nH*kH7*B zI{^aLkOr^)WiA3pqwB36XM#w$8jIvC)fkWa)lB&+R)cm{m%5REwnm+-pfRmB*>`Tu zy)g7ufdH(;{0M=#jm2f&P7X=zVp)P=(ccj&d}qD>iAlWnR4M<<1j5HOZvXoHO0{v~ z*W+e6LPUk#<#$M8cbIp7F8;5q+vXo_SQhDZ7Dv&R2?8QDy?_-O;g4i$O4TnPIigg- z5WiY0uVOdLKY3V?$3FsGuM&pceiw$f8nt@28fPB$&%MMQFUPq8*NC*HjSr5yKhYsG2J6AS2Ow{Vp>}NWSqSQy8t)^gGNY*;6MW{!tCWNPeo~4e z{zp}))0J4{o=pG)zS^nhnc{PTrppXB?e^&l%Uow8<(eC7rT&v^0)Vwldunz{J*PWbq*0Y{ z(}gcOML&%BQVKAl!8ZER1d{)05Jn*Me~xAju4EsJWa|H&Osq^H6l&I{&Jtm7mA-O| z`2x+K@=zU3q|79(&FUH(q(^j*@wZ*5_w@GiuMTcBTY!~X&||8Rt;lI^5sX|CN|tF5 z+EU^erWfIDnQEku)lt_*fDF>VC1~k&iMZ+GeW5#4ZOpws(~JlqWAB}pD`C%TNL9mo zH%MtAmn!4eyqff zs^>zu2nX;Iw?3Ca!x_E*yY(KreV|0Z<|_i_d>WQ=_5HzD?alss&t$NYLUMLWl8ePQ z1u26}x)I3O(2=XVmbN6qyxW&)v(~&J#<$z?7o9Dkf}^t?S7>aXmFfqH0wR~13w84?hLQV|?ghGw-7j%gs?*hN>S4^T<*M_f z>^1%-U3d*JeMF47I#WudZ(S_E?&&cEzc0aJn|~>iSc59H&y*zP2_up&c5V0Wx?^sF zIX5Z7H^Jfq!X!_Z_oU69sWB<$qn*zhPjEJdh?gD3NO`Wcq6R%aGFGJ%?Hd12waVcN z@8~fj^5z6b%dZHEt&%>==YZJt$zo69q*aSt{IV|F?462z)2NV2WfAp!58|a8D$dy0e3+l$q@8X?gXT*Qz7vORnWB8Q0jO{d*&RL-7T7 z@C0#eBFo+4756$O&USs$ViNR%rM3zAkkeXdkB?%}EkpZ4QlQLJogX76O5b$(B*yd^dj0MGHWK-Sl}EHY(RrElPW z&e=Ymm@@Gmlk#{Bhf%I^GJxVnHWg4P+C(~GD(Ub)@e~?qRPy6zcCoKz^V4M|KnpR+ zRJ}UC6h*c}!DOTY9RR_Yuj!X0wPTaBF$9Q?5()3Ul@XG+^To z$mP!z?Q^%w{k5+cQw|na3z=zbXtV~_swUq>7vy;LElx*y{8_=2Cp6(GRAhx93|G9P zIiLTw;%A09=ZDj=>Lj`k zl~*bGVm_#t2E0iy3zW(A?(<6b4Vn-i5ZtB7hOH9O@GR-L5{fg)c#!G{g=`7(FqW4%1RAe_kOfnnYlT0kKnC~7> z0Xv+9oY=&sy5st`)=LYoUPOpHolX_W3b+dw)V&(rpx{%zYYMj7WeGP0CZwPs+m}p} z&qNI?z%`yD)jPJwEF4!d{D6^{vE0u0CV@3Es4lMv6vHnga4_-KN~hVCuhAQ;eJMl< ze%Vp@iIPn$AXVa|*J8CP%zM(so|P@*m$%8%ubBCExC^HD;gE&RtGynGsh3_O@@dwZ z^57k<5pLzRhw=5_Suc&bQ1zGa{pDAWhbc-;5n@xO3wvoDkH$ArRrZ%T89noBN6^`U9>2t{s80ZlxtX-rE}AT-kv}m2kA+1NiaS z@8LzC|7{ywW6+nqdfu6&dtI=*zqGj(=hSG|c;v|_ z2TE&YjdxmtwrZV@1(^s~c2R$IX%RwpoR;aQBwUS&1_4}$l>uoGv8NYQd=}#Y2d~Yc zU9Q6I^fME$kCaZ9WH!Nw{$*k5xg?mZGfxoCvDU_UPv|f3qZgx8WHg4EY=UaIh$1n;6urcuh z$LGuOu?f0oW{@xc4-0?<#bNPGKI;v#2*=X4w_NDToPU~qXn9`0m&O0+PNvxF!k?nZ zb@YUhif8(K(;y=xYmWGWUwB{|+bFUfd-5g5YK#<~MX%QklMZA1|$U(5D<3G#B@zxBd`#Ppdh+#eX6ug1Q5Ya!^pNP%$L$NkSt zXHX|Jflca}Qi5z8*!@>)1*G%7pMT*OZZJMq9%g+U#eY0MRzqwD zsp#o+?-b@h+Eu@FE&yLHpNMwfCp|z`>dX2&+qxFtwc(v&tUM~^wY1YSswQngYFc2H zE#TiSuou3sZ6EuPl}x7ids?}IQ}MB1 zz92Tpt#^LFqn8@>#?t7%YnkNE&5gUf21oL%u77NrnOm}x=8`t6P=UQ?ye)ppi%Mp0 z)|dOMl6y5HDEbylVFE)4?k$6QsUcqm27lr zKT4*Ni3|b4q28&ClHJ|ubwp*GP09H!FL*rSwnkeKG#S~gn*udkf z28h`fdQwWODc1zHaz;r2v+|=(!6h_w6LjLpH}S!gOgY_J1x;h}?1vmPHxlCls?8f| zD!q#x^(1XyE`aufeywL&$B=tzBt6^4olf>xAUT&IE51(tWcx~%6Hn6$f3S#0=4E0J zxl;#n!=R~I+Z>q76In$lUKqd0S!P_yt?So8nSWbYN(y#Tgd6ZQsbRYJ?%+1G2N~h78#$3)Oq))nE$hW?ee>fu~shd`ipbIOlurSl!;mug~1{6J9&N zb1t8#?cX#-NqHhu_@y(I`@Is0RtmuU$#?mH7us-8Ohp1HSf4D9RM5qVohjY-nYRU$ zcVaekap*CI*-T+>+Xm%$NOO_r+GdgeQKOIl&J;^>t1~#;u=p)OV~D;)nm)3J`-DE* zBL$DGedgyF=aL$jpi4oQNO?AkJh7ZpO_%*;!9N;kuiLEg1 zsd^tW)sm*PUo^14&Cwyf;S|X+H*Zn|&VFy1mt);5%?VAVJSia8Mf^EYq&71fpcHc} zv5?t$?6uCM`C9nLOYB$1sP$yTz>!u)oynY@Sn48 z$jvk|&M?P(Q)vC=nUTT}9}>j2@TvlrcZd7*rkV6^Ku4*$$Am$rQxM#~E&pFSYtFIT z+R{R%$^oHC9ZEsV05j&Ms6$vMZ*O2(M*X4Qqgh?9)8XF-&w{jK)JXQjftjE3K9r@Z zlCew4lSU)`W>i$JAC}$wBDheMPL1dPi15YJ6*~Pu>u~kq-k<+;)p?FEt@3K>t|jZ1Fh)S>z=&NQ6e`8EsQ!ig(3LADZ!|DgQIELsDFp3u zW!lEF2EH2M8iAL*__OxaL36o4!woUbnVKau%Ajvz0V_n$|YQ$weB3^EG)33$oP)^|vS{SLntkhF*2N6d7-Fuk!q>e>g=c_yLP zPeG=3$t=NFkma5??xgi+G}dl{+@(5k-C}V3#NmL$-SzB?2(bCZjhFRr;?%{+aU1sK zMXV*9*5Qt2cWd=OWQTl*t&{hLLy8&h7#v2KNF^_RCW@V6uUOc7|yPVGG0brsGn_ z$+1|2)0~{A1OMP&cX<(kiGXz}tBJPjIAt?SeY%*Fr^ zijkg{;pXXObs19e>aUXd3sSMLcJ08yx>hUim+j6&!gmqIH{~D}&md#wM)-NqF*}ms zGARnaVo*%G_FGE({zImJbq+=%OS;__7e=k{*sA^=wEx4s8BXWrFDwb!DA6Cqz(f$& z%E9GMdpQx;C95a%Yhrt5K+o(qMp3a#|<< z()6Ea@>kUaCZjH&eMAPuW2|Exqk!FCdfLD!i$dOBV4AZpeJ4^1zN04R^o~WJ!=ccz z8`0s(MW6PG|J8!Ft-S!Nu9lOJZh|&x_Y<7RTdaOc2Z}7?1W<|?m!7siiF%4Ke1v#abT(0JaMN#bg@4e(TF%XmbBEiA!Kd83L%RJcX z{u#97mEuGqO}(o4+XgK)?C=I{HsHN(oUA<`F);#K4a~u7(7pBhqov31dxZmg#D{LD z!*=QZ-D{gt@^^=c*g@D zcqY9knJT}+Axr=%4vVVTzX@m<`X<_aJbIDk?Bp*k&tQ3Nh$zdcm`Zh< zlV#k&6FDjOhM8xvVPQGR3+>SDuPa&kRNq0#Q9B9WYX;!p+6UqfFEgbHF@rLRT4I+$ z5%buM`jKHnQHvM!^7uoSL3dGPC4xq+u zZ6{%lWD>?3`{2F{tW&g1_ws3=k5}xWHV+Z$?X~wb z>NlN`V!nCEA|#fTXp4yS|IqHeR>Oz6(!5-TJpAz4&| z=HsKv1&u5N7!E^#8U>w?+}Cd33*uySzA85}=K*x7hhm+BZ@;4=le|7o>kR=VQ)z8x z$2?TW6KOKY6^hnC(cU@FYe+^{tKLhF7Wtb~HwK|Ht~O;>&B7Tq>~sbY*5+I>jESP+tan|Lz@sH^(G08qYHHvnezfC7jUL7&Id{$fr~f-8 zd;1GA3zG6l0hMftGzJJD?^62o%br>AbjQzhaoAL`BeaLJ^_A0edzZji^}WU z3%p9kYZ{86+A^gt{t^U>mNqjxD2U4k% zN#(jkF-#Fj%>SL$W{tN6t|qXi5m8VzvpCI}NyV^%N0GA%z%^#l>A%{H5|rWzLNLnLEa#{EZ1TrihbJ z$_`mu0{xMK;gN5+j-BWDuATTW=$)KBXqq&OfdHL*R}7&V}EH5v833J^=Wi zpfmeD(I5_=Ej#Cto``MLdRQ%YzM(nwHWY%%auWv}N*0|)iaHwvgsm6p)t!7p`7$m5 zn`M~G-FE?Zb)tXj7LZq(9yFyBI?)2Al@qjqE+^A|ii81VXn|ZX^v@DTW3GnHolvd+ zQ0Y<|wspgFq}Th5OZQMRR%coh-IBZ6UkffHMwHC;owr1hBW=I{b#t zGlr8R|9GMj-AeB^yAa>G)kPNo*mJ<`$Ca=^YM`$@F0>Yt7C5>IguVTVCA=CgssMd4z_2WEw2ZQ)`6|Z z;a4PqR#P&JiFSuQqlWG42{W4II+MOhq&=Je7-f(Y5(s3ts>PYUPtk3h7^PR)T zsq<8{%}KQ1j_ved0GaGj7uxO%>yhJSZM|0o8=vr*x(Z(ENGSH)GPKnT^w-^jmpiZM zpO^CY@;zi$INvQCd$yJL103eVQ5*(v*?(W3{4{QOadOUT5L&mcL(@*`O_EXpR_Hvf zYLT0-{*?fBZWB5=3Jio{iY0t^Q$69_zIi74%O81Wm)Pib=5pugQ%FjT(FBR!GZd=Z>odwq1WR4vfnmks({ujr zvA+U|R(+71TohOh?=!lH;iMQ=#~GQJo(&y+u(L&$Sv)E`<7@%tYhL<}-){x@$}69W z-UR0W;wI%HVrylF&CP;<2N~eFn?X7v^OT#I5$rp{)Jv1dNTg8-cdui_-LJ}v(bESn zxW2Bw-=BtpRng9r6;sU5@#lT>RS>?99n_DsuHcz7Vt~6O*E;DZ(ILd8R%nd0j!@m; zI~j+b&xpNQ_WE#~BbQUB5^#6gZF;Tn{+7VC>|C3kw{cro6!w0qDD~~(?6N(MUMeUP z1x_)XNRc!fj>B7_cN-3-wm_R8e~_j?Mb6Q>M+oX^h>l5R*@UvwrwJ)8GAQ0V{ILJ2 z0TN3nEye*yxa+?`BKH|Iq?B?DD;GQ6wKoz+mM9sf3J@{UX(jKSC*M83Ag=5qxuFCC+`CNSYwi7e zWytRN=3r|3;3wtJRK46}rk~@=Ujd$6S(Ub2#6j;Oasv|1vuB=>>zF?ZXN@fW3-Ot#Dw^nhh1MVp%i3baA;uXx-)AV&{T3I&G2G9v0ZgYrAlmG{ZLl8%bRx-B(jJNGm zT%e6kaGkNuSHr$=8NEIs!iwzWH>K#g(g8WzCdHMfQp-88}-yn7IrO`PzyW(NKGW1!rQ*A z%=N%Pg6ip%(U}annV%CJ-08bU<+(~o|AVS??9MFOwsmaVNyWCEN-DN(8x`BOZQH0= zZ=6)DitRUcPWHX+w0-ZF`3L4|tMxhh=ua15>U^h@e z1{KSX?ReusKXb2|b$ZIK!iIY#eQY_4M5@kY4X1~pEQ*tAd4Ob)GJVvW!}=M7nH36K zNlG{_&KRBQId^HES{#QBX0HxS#(#Bl*C8|!Esswn>%00rRc8Ptqx*)(Kn&U5hOL?3 zInC(~fxc15&YDJGO6;VT)rA$ZA40S7W*k~&N9wf+*2KMa3SgMF>H#FW?pBW)nMcYS znJ>&#=B9ERxnn!k!vJ_4E7(TKfPPefgU7fJjRJ11Y%{ONocW%`Wu(8!I_t{ioFq2E09I zARkg`C%^JRm1N@NkQQTf8E!H)3!}Pm)cZ^Wp5}Ek8yBvTF72<5!c4_q7sR~d*rQdv z0qiIghkj&F6-mr-)%{N0^oY_+-qBe@GY>p_^^3!w%Q>rWYeIXtrS4N(-T$B3{LOd)gvD9$zbtz8 z-clL4E%JDbElNRA7={g*Fu`c1{6|l}>4qjnLmNk`hq^uVBPB|1(E|E!ie0(xDft$6 zkYZ`FVr8_VC-d5o2>ETSsj}#}%x>wXY}uTM;9n^op+H?fdR!n3M&^`=o(KN=-!5D} zq0aq|@*CX?#e&9wxhRp;U65e(L1)@4A>E!`>|ifGPF( z$$}!MHAt$SH$p)~uo*jR0)Ksc=cgP)!A>spt$m1I!}WB}IE<0HnE8C4eQ~G6x$N7a zt3jNNj70HIu4}-&(*lV8Y!?bkwQq+HIgDV`6fkP+zM`%u6jGypXo~>Da-7iAD zGWUB6n#b$2nC233srMmv~i?GN2o^k)mo zL_GNI`F|i&<_BD=*MK7P=Hsf&1+Cc)UBc~r(;_xRSA7I2l>QBIe}ZyjwKa<5YGg0W z-n!yzb+}h*~6d=Zo9YWq=Qt$*zX30UTs92=7exVq?ZE_kP| zcqv5f;5I6j&Tw`*F`753_Eci~v~u}?K5cK8-=jNwO#fy;UB*FKK>DX2d|U~3uuS-* z3HeUrUIVVBJ<)LN%65M$dVN7>C2sk2FdI%;{YK=>u{!HDCC`^28E+p^PnkyEbdrU5 zI#KpU;KJeH`NK}qD0DtyytxLj@-Fj~*m-`%Lj8sA!nTj9_IBm)(eO^Y;TiD^kNKrG zBbEMltnn#d51##W;K&BHj%+>7zSnfbh{eeH(@q-?u&Wz^XmLu8YjcVt-QX8Ry2eWV zw9OEj8*S!q%&(iyk!?J7+IW*@p%NS)u)T>7sgkH~6f^zGfE~K0^^CH8E`O+LqZVlN z`~|5T*Qqpiqb^^jv_<>g&v3B#Z{QiF`V5)opgDdeeM=GI3qTWMen8 zMmXvuqVoRln9eLdVvnHmJiP&51 zVd_uOB~j;jj&`=o$QOP#2TZmj)%FG6ck8BG`XqdD`UQD*z8KjLNzyeeWzu^3(1V}` z-cSabJJ0Z>-w6U%oN+yPdrP%$-j45!w2aZH8QhV>jDIvg_L%@djbYYXZcmfosaRf zm;9vfviJ>PxtWfC`be|=0_I_%xP5Qw8&dFPF^+$7(y6-_F+%`DXBLsEk{l>#r`BE> zmtaWt$3NlGRdH5cUft4aeWk|5;7znL0HnAKwhoEI-=b-)&p5fEU8nQxb^Jo$W3iPjxewyjb;)JHTOEcVdDC3Tlp+<}8tA71@Q640f4PWT{J3-)3j7UL9StgK9qu9ECLL^ zB_r9Cr@fu)yFJw2_T2^9_NnI0>2u5 zS(fq?cSoh}kWy;>hqQNZzz5UY*4lBt^$c7BL!&C>>y=N6hq`YZUu2pD_2cgn(tcM!ep zqR}yzrJug;!BfzzshL~371uLBTGpSE7!~xf?MOg&J$6xc2L&Kt-{o}@U$0qvZfgUi znmsU|4nuZHw^qKg&|eK4C!QUmSiA<^K(DnK&BL4Hus@K}ehu;y&fX}}1LXdK=|v8N zbU!@%AbRjqPRtVdHR&vT z`|J=fal5wt`rhX(^r^Ig)@KYi5kq1Nx2R-`G@f=obKe(j& zA}+?O?*P?4N#P&#Ic=2kVt?VL$DrHWis7${Ypy}=I8ri?1-T?pL- z3Fg9Qtr-tJ^bF*2-kj%G!xip7xQiE>$1zR4O8OC^6Jyh4G>PwSJb1@XOoOBuClQTe zaXHzN_i&3wer)c*-v$-NwSqFmAKW1*q>M$sidfMR&njdG6Vsxg_Fif9Z*lGEc2dp#H^Py6!Q5 zc%XOwG9?8?lb#)^9+qThNm*3GD$Y!)xhG;1;2MZL2mgwS~jMKMCISGrs+_zTi^;(Z;4g*QlpCtcUjAf$i z#cbC(fFQ4qlSLq{V+#?iB3dykBac*)ggX`6O9FohwQS9JG-`6@6Gk_ECP;gy|3fIr z*_=If3MVR4X+lFc3oP~hPuDSm3MaBM@w3D%nvK&rPXD?6tVk38iS*(th_wJ9g?}eixna0dGC#OX(spnaFgB zWF+75Wt6E2LBRQ2u0=By)581P&PW4hu+ z+{&O(r<;mJFY1G-sN^9pUtj z9xs$3W>kWa%07b5@91!(vzh6G3x-qn$X@N3GLq!gZ$0W9nwE|MUc%}Gkjo*{&% z;!M3YA_Akrh>s0%elxu%Kh>>!2DLHzRPe>iVUt&)cn*e1`pafRrfo(qvF9d5Y8L{f zG-nu%xcUD6vCH@=f98XgbA{t)64}rx*^fbG8HdJ7fs5N0*>3)+G)~mldWxVS-z69L z%>z6Kb6tdZxJTdNGcKW2jb_vgS7M`1P#?wUv{osA`;9b5e?rE1{{uR#pp?lleI}%88~%&sS06OW5GmlR_Wqx4wY5e{K>< z9sE*Z{GuVu!%z`!qZgqjnA<&Q@NG-$>jRu(8?0!wbChEE=m$xr&{RG51$n{J> zW-LlccrhR{DW$pg2*cRi*-Vnf9of$8r#jz2XUy zeFgrRn7NX_*{fR<4C#J=_jfO7j?6psZ@_kqlxQe)a6=vkHh7MBjUPNCC@>ei!CL%~ zto9DC0jfIbbfzI1l!&d>2>CStWNy#j4uH|3a=u`4l2*qlxnSmU;Dn#pF&)O>q2y>Ed^AIa0Moyw0dgW~GbNmFEo0ryJ%M?-vNY z2pv?rcon;zVR;Aw+pR6n2e-?_&A8L>FQ+*UxbEqWV+>tdg0#U#HQmofe zk8J3-Me3X|5;qkOH?mP)bd3a#LX{X|c2I{Q4Y3?qsFD`^vX3loA4-&;7Dl; zci2na*#63}WEj2+mV$p%?Z0f23~P7u=7`>Ge(sSbsPbDBl@;(>q_PHV7VvRMpYEQu+BHVN~*S=sMtL?m>7(ro?K)N5>+ z|B|zDoh@C&tOGxA`BB+2!kd4ae=|@u4`dT&;}78m+ma?|091n4{+StcOdhx{#o~nPLL%B5I%bMEUbS%v!A?v$5Y#v z@l@lxc3gFc*q6v4f=@?AlouVEkvnrENjk}}%f9~stIh8eH3Q>hfssxZ$fG!nL2nG3 z_T81I2@ZyZ-SCZ}^c^Ih(a0PQB+XNVb#IAbQy)a7+l^E0!X1eXF_weamxDl;gGh=E znFoipfI^){RElQL8Ez7lq+ZIwQsx0}*KtO@<1l{$ALhWfKHWh&1U5c_9zvgQLT~SY z?T_u(cCffRHHDSmPUL^3@ehq7EiNbp|H3BV7PQV)WJ$Vo%HNJCnckA2(40AbP%|Qz zWH2c&{iU9zbdilPONsaCR!;Ru7f&NfU{NCyuVarX+f=lV)X+p3Z=%jLapMI{$p&iu zC3oPN@=NpF)W|a*Rw{6fO%J>OT4F;!ygtV}7PJ5GI>} zqb9b)vn0dBqehHTAV<@PzVls@GPLP^z}j4oq(90r3HTQ+aFnsM>bDm42%9g8RO)D^duGb8TiM3dRkV9x*2;ixcJ`kfCwu4cj#_4;fb4s5QskIh3kRtbk4vwt9*Z$A9V^sKCjKh&*M<>({dsYyeD5EJNsX-)o~ z{kdUxE#h!vCfD0*GYuQBL~9Rg+C01gRxFIG=I4(4k-1w@65V8LT2>y$uKQg$t*)NA zvTa)#b^-c-hi8}9un@IFWSk+g{LrtzqT8M4Hq`}IILKppE}QISJbHQv!W~YE+ z{kb|uL>HZN+{|iiB2GI!CJ7!}x%)mqv?KiQmn{NM2PI}Pg_@C}4@`0kno2Gxr?$=7 zZbyjKIDO@X<=|oDfy>aMr#F9&p&I){G^yuhOC!MkP(YMe9)HFne|CzsFPvhR1fI;O zj1sSdZhKts3Wm+DwdQH5Nv+yUZ*CCI$kh9Spapl;;dCf_%WrC0tfQ?rQ{1X#2GK;9 zd^ScgfkNM?+I!7p#c9~5G3n9Zx*4e2)}Naxj)i;aQ&Mdf|8LZ`JR1=Da>dLtv}+W` zG5k9`zEThy`Hf%JWnHW0PX?E3-!5W&^06o+BeI$*x>AWg`R+#5{37Xf9`O<)fy`Yh zwX15r$f3S=&zEHF5rN;A`wG)%`;1P{+dp{K`y*ior2ii8`_2X5yNi#{Ny6{G{YKk( z(7rZ3$Li0o#J4-IE+tvs@n8O5FJGPLanD;)pawYBmCj_9P0S_hV+Y}6p;$&6)JIVb z7n@hZf=03dQN zDN~K95&1#3qX++ljXF*^rJV3=FHVrFG}PKj`;VAk$+MwS0j#xfX>b8%x7mMf)M!lg z`lCeHj1X{`ArRnNYEY%l3PhzSCf!Wm%=5vM6@=Na#pCrry>6?XV8?29*;Ci%em zQ}V^zMZ#NI4RN1F;9UjGK9C)pw4V0Wvlabz*|P!DOk0zrr2-%(!sT%?Akw~b0gQ%) zj614tbp84I{nH6?SqVxnu;*artqR}N?)3zzz?mju)?OcjJym?x0(jVGtzw@b2Ekg!w;)Do2);g>9u?ELeE!<7d9vE)=_>J-|;b9swtR4xVR=K;#MsdZ))&^7+f%X=d%ku|_Sz*P=XcIh2X z9k>6xtl2u4Ly|};Cbh<^ztUOuFjYOaZ$c*@X`^q_pqjE%t@u_$_?h@tgTFo(P8QqB z!-_Wn=afA>Dp{%Dw>FClJZL_|NHNB=xJ=?!Rj*HHRy-%q^Nuu2IWW?$73 zYF_^(N=LP54`UcA;$#TOwDx;G_L7p6t=A1}8I$(6QM-)Imj#Rv3%P&(%J~fD9utt6 z$qy*=k9MM)GBzqOaM?H2_wpsCn(-NGR575Fx)75mSM{bR4s-fi~sW2{458S7P;r$7^wB-9)dHxjhhVK8K3n}Db?tefP=SAsTm41@M zOFNAiLr~w_`3}Vj)|G0P@G0Vdc>u6c8PU+FW?imZ0`8>WbL(-`wkj9HqduXJ zzr$b?QMkMb`Hi`NHA`dK(SdqhpFS1WlU91fTPshy6+pDO45r9+=DCGyvRy4yK(WPEq)OP z)cE#*1%v*&!v-j4%^LM*fix*g+I9MTN zOT;q<@;63Nl6gy(`mZ=SjSSsiRl3ICbif#T(MSt1y+ZGCjkVW#S#+j08ii%C_ND~| zd!Ki@&fAsS@~57@?lM0D^v-xQF(#E_A&ag%EkZYz7EKZz>mSMA zjWO8FHW0hF)ISeVb+O>Sd)Crhm6)1MaO+^wY0Ngpq(0%ZdpD(2OWf~r*-d$=O(l{N z%Dw^1%B74&rJN6cB=9P;Oys7RPkGF{0#Zq@v$KXT>o8>OIBMXZwNPpubISvr1`4O# zJ$X$3B%GdxblKN04caz6n#m214!w|*8B;`c{tb~vk4%^Nw%3lF!c$|j($xfGZHhdd zQ#nV&MW;CR$K)46b6B4a3r2<~<%QsXq7uzgW@JY;bDo&wm!=`dd}!e*uy$?Av#=$l zoxmJrZ2DMsD6+gjAB7ijaO~siDp>9)O0TI?LMHQ{vELM_4{g-Jui9NsW@EAG)cSg? z0qOMF48|CZ8na$q7M)ofHr^Vh*U~lYd_DechI37~c30M$80Zlr;Xx|4->I_7X*3Bp z>1Wl6)0#nNdRHM_BPqWiNq3?yT2wz>SUN!j&)4dfV``qdxeR)-0AjyJf4q4T{LY*G z0r*9*@Rg>3i0!hznExaT&TZ6{u@GiZ{Upe>?5yk#`-gyoKVcfp1}CnbJL{((LFolE z?>t4G+X;_-Z-Pdyco}#1<~hP$gQ!W`)lW{XFCorQgsC_xTwMI0jC^r#1Wu25)|!M% z9o>rL2G$h11mg|QBv%byn#2uFOWXNxNw^O==#}}BEZZ}OjjU|oPMi>=wl19>0Nu&( z<yC}}2v-AQt-dv!+Bvk9Z2B<)#fSch*dpKHL#-s)Lj1LbP;Z;}*XM?d zfrF3N{=)y88t&n7wC9=An|w_KG*Zt>^<332F?v}?{M0^udD|TQ0OZ%Zn14^L%D>(M z7xE5Lf4a0fz8!~NSsQ7Qqg2<>0Rxgrvn18N*mAz|z5lB3lO?$gQrf6)B`BNLgVkMD zDKeqO-3Bu&Gohspr9S0@j49Jk9tE zO*&m}_oG;_sw2d71jOa~nfRbUD}tb`hd36)Le{8Bl$WBH_Ry_#(vws;A)yRa`>T`! zy}In@WHNG~M&4@#;G5tYN`!w>jM(;i2yD)P}Qwbk>EZ(-)& zTt&^U{y6}Tvwe*bm=rRfotRzg@*~#RbmT}2U{f)~@EdGjU-REh zQ5T+yJ;_lW`^YcxxH==Z;8gGeD# zpG~n{w?BL?`{#E(-5;{L;XI&ThksO~TVJnxuk=;9UAC@h!%ej|Zvh33Sm9w-p@Lu| z&{I@M7|x3)cG}`EqtFY?Uu?UE=KPPMZzNG#25s1T$m zAZnK(c9dzz)oIH-PJA$Xxo`(d-;^dGlKZl$`m#es7Hi}AsV-Pgn11F!vk|owg09Q` zH~R+ec{-z|-6V-cvaC(u47)$GaqQE0YGMyUy2YmFM8;Nwg~9g%9VGL?B4)!wpmqP7 zkhQO#3*J->JI@Am9|e2cX8f43NdZpUgn7bXRJ$xPYMfQzZ2L9^1Rkr86hxV%Nx-^) zj(#4$@MV2+VXJ6{s$C;cCDPD=sJ^5m9ZKFLC6{-XqvDWRPF+inrauWn5+qkjfnqQm6QN;N7-c8G;Klh>IIweIw<{mxz36d)^D8apnD#6S) z%71|gNMv45kld#JIEBZ|HoqV|gon)>0qaU$k`Mc!W^FYxJtrewkuU2DP+5XJsD>yT z^Poca!)pO3?$10C_0>Gww)!v-B73Y7F|oM8bd((M=8IM~1!#Msy0Z32^C{BL6iX^* zpk1l5S1?3C&<$j$rh2Bj+Z3`!2EJ39$6(D!ra1Wx_x~E8W(0ZzG2<~c-d-E1Ob0}b z=<*PCfC~YT^&W3GaVC79%@$;z3fnZ#xvt<_PPY9HoA@_v=j7d|+^7z7#jCBb_1x_{ zzn(bX=T>5`!)rabulff{u)P;xH&dAQ&ojXUE6W@Bs*I>pggFz0$t#AiWyw(L`?2c> z39AQ}%&|8f@NCCvbG%>$8Ib$b1B8@9gp__NR>L%N7jdDMN;o-#gnMHk|Hv#<0HU;P z!t9MWkxTE-MeP@P2p@thFzTym8&A#7_E@WmzD0M3RgRStY z(9brY0yG_qwGbfUH*?wDKw=H13I7)y+~Xf_p}PT8<&I;5ZCC|MrJ!tL>lnxE-&mIG zP*E^y5F4qK8io#o)UCxfcM>lOsqY}{5IX8_-X-G4yEpV_59MpYxnlO|Sh(3Y9E?8} z!0NVJ3nTviDiod!6=+F?kG90{o+z9>Mv41pd+v#m=#K2Gf{Z<(D%a8AQVBEUp7sYT z;-ZI8b~}cHLII-c-8n5)=QpyX;l8z|N1zdxw)l>E& zMFoTL)(IfpN91DR-)ef(PSJ1F!*_ers`5$Fyj@+P87WL)YydIxYTK^WapiNOJD;EZ zc7$l$G`HzS81`EiFrgNI+xcPFLfQ-@SM=Ic_ABfA6 zlkRELzTC$6bK`cPPHcC>iNlzMql#b@9|E)=&UzS^?#9h;=9p_s%*_nioibUx^fcOz z0)MdLmM~f;)J=q#Bkq_1*i5DZWdwmPo-`J2^~$MY)x8WOsLIih3WJ($p?K%1+ZC@_ z{rab@X!sQX$T_f&2>hI-;XlXCaXe^fxRalyjRTC1_mD9n)t$M`DoLh`eGWdG(AUfP zyAV3nd4FThaG%`HBEpvI&NDSrSzIJGYb5LmRy7$IjsO!i@&E1uz=LP9<}I;g;>c-W z|5BE9h#i~EJ9EKw&^@i!M@Xf@5V-%{o558+@|BTfD1Jz9N}J-Elg@hADxx^5?SNuw zzw(AdwKa8DPx%DfeM7M%o8vXakDM-a9)&63UYcW$VVwxuQW|x|&tCl%E(Wu?Z^^h-@P^5R?2L{sCjqj%Czn+ed zb@1nP>~9dzE|l#qqN5U67J`Kl#-H7|ju4g$B=23gH%vpUc4}sYewmReiUjd8*39Hh zw0E=wwE!Vj-7r zl+YND{uMy%kwt4Z26|0e z3xZM2pG6->BaK^Y;NOkdpPeX{vauR5s!xrV&Wr#OHzxkSM2F)H z0IG%>oPi5}*6(Ms;QD8KxUaMR_>MU~?9XJ18ni&wJf`}F`f+J>l6p@?;d0mRVnR8se&(Tv#p>Zti;NP=`rO^ zDWYE`HTKacJ-0gLV80P_v;;nErnm@UAU6Wz@?O|)%*u{Q=B|qcYbI-cMa!gq_u12Q z=%mJ{?x%xWQun^R6<=csoKXS%cykZc>l|!Q6>hTT+UQPF^{>50pHvAb;7#L2647IA z*5cw&+dT0s8Ys#C!u1(Ceu{ZF>pScHh(pEx(L%5vQPL1fL zj1Wq6{%UZMeo?_LSF*hdQp)+&HO5O5##q|WdO=bJe!rs@&=^I`;Yruu!#wRyf6N89 zsc^{DeiTprCGAhJ@oZAM4g1;m#(xJHTGNNH){D5TC)JT`r%|I&u&8LKetbZEKG%x` zw*(2w5>@JyGC@`$i6>gh9_eg_G%|=Z!d!RpQyW9GEU~Qbp`W@*&gQ4Dzy&8J8+SMG^jHL{?{m>U+PnRt=%1_Nq3(UDHuJSfSyG^$v6Z$hpy zV5JR}k072Z(Kr-Db-4iq^9|R_b{y*jbx2~c6+(5;QW)IurP}4{!M}04rhFfDqupw; z8$EwS^jLB#y8ZV%HzHk+YB#-3bL9?bmIX@Iz#4S_ z$Q#ZR!?mx4Xjc#5OAO`8`$du`$~=)0E~Fj8q#q_xKa^TCRJ{U0j%4K;@<2H_jzuai zR5VIrd}=h@L>6H#fifFQl87Jn-Q`Xz6eH^=rd-7yCFY6|xgOzBO=)8ysn)sUq)dK- z>z)aBsW>KUAS~Wpi*2OpE=_83n_+UTIrHX4w6uNstgvKJ$YEi`exy`6WkSvo<+9JQ zdDXP@Yn!HnH(#W;7u?IHd{d~>Bc(R}#}bKQX9&EB(I)XA^=>{q;`d^g5w_WTxrqb-F!KF5pnl!gNs0&b=l{B3#R}Lk#zD;R1>y zne8P_Z>&sEh$j(u_e-%DD^hMwo#Em=2%qci^gMVm`j2I_atzJ^oewlQI(ehN_~Q(Q z(B-j-BEmwQx>c$Rlruz&I?=s+!H%isI{@Y*$&Yp(>#dNpJe+&D0P_pj&KoVVV94S~ zadW)sb@><#^P9DdI6hxaAF`}|9&sV87|@(V>%Htx9HQSHK(pGw-9gfsW%v$8;}KsL zKB76%XVu<_pS~F4cDh%0_tJS&7)~l?|t!Kutf0> zA3D7OS#wpK6AKE_ReU~B_ww3juQ*|U(W`2R_4)@-CyV(|e0V`0YBg~-cS^X!1=C%P zM&6i0AIy9>?#9_(P-OAi)%^Qisadz~)721ng7T=ZHqpTH?c9P)8sK_B;wTyMStir*=V9g(wDKA!rkS1B19XN=ZU zPBrj=mFph=GT`W|i^T`ee#?QJ*ycrQc-UNN8uR_PsIJAt7+i0 ziH~VB1No_k|Esqn%79cr@OU`CnY|;#)CHl^oRHCIn-PZUA1B@L-79xf$@P9^puR##y zmyTgwJBS;*$k4lv(7TV;Fe3;Z!6F@Ce$T}}Jcc+u$n+W(f-|;yNe3>%xeh|>^&}ja z>1~aJ%1phqjUjG4e1cYqIz;_O4k%F8(w5D&Y~vf4Xf|Z5_L)VRnKuhbbd88}OmP7x z*RtmIdcu6}8B;1v{^P^H`w5)}v7HAnm2B;Td@uMOazSVV+4$&LST#f&B!X^2-uHN> zeC4n>2Grb#X{VHoFwu2mI!%TA=b#-jST|DOsm7!W5`}3pw``l&(+!(JS{nsN*wgNI zQaItkxs8WDy|2IV4VQpG@2kN;@28<%m%;JJkX5WaohCQJD3NMZ%ToSkg?DcnooPXH zJpoi`AEUN^unv4ktL9W93PwuR3_{Jh&b^^8vXPFgb*TrDrV(2C=3|P{JgS^9mAhAF<8LhJypnJ}2JFGhwc1WI&qbqj0CYm|NFICX;3-WK#Xy zqQ*DI&nSeq%(LgCTZ4Be-#W*+F&&jYj|GqX>>9F39DE6IU!UBhB_@FzTP>_atN<}< z-IjNH_CQCgzNAxigj4rmKoo@>O2&r`>mS&ZWTz=RDl(r3mobPTwH{dR#6~*$eY3J! zSb4VViM!~}aq&p~g)<)U(;=4T$ayU0K_z0R~v7c3g{l_%6s z7pZFWdv@|b7fa%vn&k5{q+l!3eiyPz)NS@coqZcz65YKd^#62SrQUNeP27<(N7-3F zR3tei3ezRNiuzgJ#w`OxcLEj5=KZkBb z-pdcg-qX#%zb7K%@1tF?OnmJ-5?gj2q?_Sy`|_KqaC>R8U@O>7&(@sr3gpeJm~uwc zC50FfWVo{Z=nO>~_+ssXA^JU`g0-;O>ezk84&6Z@ACbirA!HO{q!fwyNEB6jau*65 zsLXgUrP(m$Z>Umq4cvK!@AnY*7wk!$TI6>wlFH((!rKp4rctzqQ43?~mVK;&KAyT* z6cy=~`jd7e;fHU@m4lAY%LM*J=Z_x7PYygx3^1#HuSrx{cXQhJy7>bkEI|q(Roo}P z_tUQ~pT!GfEKQWUUB;Oi9F;(@YdPIvaD1FdB91%826|D2?jdg|dP04iq;5~Z7r?A1 zd?SQ4GSk3JB-_WvN?m99s}518%9}byjRxR-b#8h>i1XinkS@~iLFUTpKlPxi!S<)u={~+;wo-f*U2;mLsR)%T&p@4_dq(twZeDeSdidZ%e zU3UtIMp{cr)08^J5)PqKqZR+7*UP9J1Z3+hd`Z5SmOsCw+D|Mi-p;FJ8wC_Ry1_JU>C? zzIJg?=K=HWAw^DIsiTP{K9M9;gT$;6%~FU`{!loX)#kPHj^R{1olHubeSzi{VsRUM z={%0SOqrwSSXj^2r3ba)z~u`&0(EJm4~qdcBDfRGsEmvB@s7emZq@R9I?jkR>G^4f zbhsB}JKdPS2}lEg13u}$bb5*F+Ip#FQr@k4PlwILC>k6~8B4!Z8#e;|H~ zjMWM$HOQ>&^Kw! zA{C@5Cq?D>R22sH+*HzTlMasR_$d87<-jfr5UCEke2!YScu;Dawsd<`7`B5H^my}> z8av$-EFzf=FpqWjPM0tMAuuK7CC@%pu$m=N#m$H=bGVFD0*6k$NWI+=5%VlMzfcpX zI2WqeA8ObS?+~$*l+-{`oKlcPWsG{WfR6yzoK7zfrk1i-D$63mU@#XhOcD_w1mRCY zZb&yE4Ez)OmM);$$t0ox<{{7s@>-`98%Uy zDU)T^S|EI+AqKwuJt55hq#=(LpPu_!5N*?C<(H#lF>)QX~%pbJZK@xZHek&&YA`Jo5Do< zAbq@>0z_^n9zpGpVd_rus4`$`khxU01KoYI85;Vk6aVdru)h}^c@Y0^$@CTTfK?E{ z6c!zXVvaC`11*>>CE?COCINM`x_=`9`NG|S1PPgHG#J%Gwm%o7!~^54v@Qx#M#v7b zfVD1Xm)REe(N%0OT-G5ukUqvPkwQa%V^bn^7b4I*Rft-Ds7?{(GXU&W&95EoaU*H| zewi{j4Pg9*9wM0ai43Xwcb~`^-{qTl;?5D9`i;|I0S>#pr(W;I8mYxV4pP!iAE)`+tD2N)8QDcH~vfl%3fdT zC;C1Op_+G#m*g%}3x${DYYNueOn5w<=-L+ayEAsdK)gF6Hh--TKKhrv*6Ti`g}Y zVGg;5N==T4Sa~hBi=ep99*{fXLtH{^$&#q`|03GIIj)+R+|MMTqCnlC|3|i-bvsj* z&fHfII^VWI3OJ|N2pCO0?&5U|U%6p5terSKD51?a3F1gGezMvlg2V~o4({J65{(IR&3RiDA%7#IP&BbAtJOVY}veAdVl?`-vu z-G{n+)}2$QoiFz0iaE1CdH6Sn|0ltgexiscDC+Q?887q)d$ec;R=5&%Y=B@pE0VV0 zb@ix5?#g2R-rcR(TZ-}H0%-|Pys8W8__DmwtyL^xL5xEWZ&;))VmUzNxnw22`oe9~ zs&&sP8&aq>KWaq}AK2v*?BUb9w;G{wnSGPxi@w$1F!}?4_5Iij5`C28Hr=MKF=mZ6 zaLI|$AjEeZbA!`A{}*Av`v*9bfd5op)bS&K^cbEeCzZjFRYjaLDG+O#42_Nyg>Dd^ zZV-KTh;Vi&k$NceB3#sAAkVTNwR(uqrk_|%*7OyP%9w%NSk4kixfcnw=LOzqiq29T zD%lq5XatftNV4~WYT3y`?vl*x+|1l`$J_)zc95PxLriI}{TjMEugTTEocwUZZ(sFbS3t8an`8C%Y@N0KX{h}gx{~P* z?={+1Zl{A2dDOG0H9*z`F1b>~XWd}fWQIh)Rmah_E=0^u`Q0Mh^OZyawj$MXCNP?& z_u_UJaT4mi@EqfPes^REh!y6{mw3e1KK`pUWuY`g;2PN6T6$*}j! z{ts}4)h?hgf5fCTg{*n?7&|2@4xU6wU!2}Jb)BoujWB(bw=$29tc+2C;^=QrFpCn4 z2EM<8YlRSXza0Pzb;vs_SLolfD^!O+ZM$amgijFr9n={BTOW{*5v_xXg^FYCj^*Pa z{5JawsiROQS*CS3Y*G%a#F|-SN|iwhR3W62Jr8E0q2mYoFgQ`mLn@ZE%%1{WVTH1U z?H&Bj^;?Vti>|WEP>R!tM&x{9%AO#d)3rWsK z!0t$PoW5?h+)rYhAQKOw@nxhO>`9~ig9v{54Jr19x!vQiS!uVg` zqq(&d;1X4^6Tidd2zViLuObl!tV<7PxTXp8L@25c#m=CD?N60Ld`s;-e{I4nFhN2d*$xZX(n?+R+4HZCbWt=FL8SK zbhLlFZ?!KX&)nkDP`t^#Tdp`eg*!seoQmA~Ti2%lTe0EChtFz$B1MdylW!4>w@|`N zc&+97MBk+rlL#y;RES$SZQr^JHB~BoO-2W-z@H||M%~^~igCV(i9^rbT+IM&+DBt% z191VU>X5@C-VZ8cg@6qFFD)7c$lGl&qe%3VA&urrS&8lCL|rm7!l2QQ#7~1578E`zce`OWxI}zox#<@sgzIC@`gqU<93E zDGfC+f=u#@D-6Q^2Xtn>3Y2f^INGuj%tj{tL#LR3rwFNaq+s6riYy{o%|x7);v3xD zDT1a5wfq{BCX3Ij#7U%b;|vE7tWdr@;@fe}(*&?~OK?R=E(^$EE&qaPv?mPIHK{@k zkY6&dvo(MV!e&VHRY+;DUtuw>3rvMoc_R7On3QOy1y1FZDAa`zN2;@z)$h zeP$_;LJ`(^tT3&g4c78?OUxu*$kg0Zoo{skNm0{xDCLZHk^V~u{OShbVW99b{U*oP zZ+GzBmT00rl78tdmbox7XV%qv=pSjVOMOzn;_2(ilwtv`cWaTaSSP)w_ouj1KHk4x zw-SV~1uQ9b3mk7geoNPy+M?NIFQkDaTw&pNn;1(R;t>hYzNo=C1YPxn?+fC_y}x8kE0}HU z5T9H=A(>@FF@xUl%QJlsu#tR~gZBI3<2b-`rdUBoBi?$oBidC)h-chV!vS=yhH1}9 zaH+@NVHYcqiRa8QDHWmDWKmgdAgz_n@V-!08e{lfo>0-zXXe1K>tTl_OGue5v#?gtf*mOb-yeR-KtjIy*y1 zU-A9qYr3boIfm-$StLgHrS&R$3lwo`@TE$@9lzU*S4`u?I`NZ3(4eJ15NB2aRmAX6 z_^h`>TkYRbR&AUP)-8)rmRn^2;~XV(S3((dApw;!L@j4h|V{OX^-Isr-=CoLN4 zq6brFWJD9f!?aj0xD)1r`Bi4%Y0H-<_?s&Vl39LJIKY?weP~bu!(`>aXS->YkyDHAe|CUaPr?)xK5qT8Z72w3-K|7UG!>P1BOPx*F30QSY!@ic>2I!BEJdtnYa zlBgOQJb;6r#zwGd%&dCdeZJ_r9&EJlbUKLxJw1*ngOhTA{qux*X+3uw)zh*gSei1x z@r$(Yds%GJci8-=Dk?(T5Si7o3zq|n7597{x{hn z(1xrK@kRbkkGsT6;X&Dl%2R}wc^A0#BT-nV6}~OE=RXfc<;atd??TK`H++WCmppwg zAx8%7i(3G8?hJYs_B;nccp=$TJbDo}=6-U+55Oa?@V2IanMKRC1!&s>p@BcAgvNK(Sng=u4|wK|NGfeC*Qm+M%(JFY46&RF+ES@iW`mL5b1+vWw&ST*QB*PhSa z+QtgUIhK#u$1N=gKu0-m{az=dUkjJ&*jqZmHI1Bc=B^a|N1yl9SOF)>+p=+3PP)yI z^$^06eKjHm%t9TV5LJD=s4P&F!jP)Z((wRk6UIaJ8SdrZb}W87&oMlDl~RRID}IIq z502{D68S=LKYV5J(nk6j$LL5O64*lqSKC4T-_o}nKrAN%{d#1kw7y@lzdYu68}>gwSV zgK`eNY0Ghx5mfBKOo8SNG?@R`#8k}=_h#<25UHNVQ9NywCRO+@ABEAhY|KS@<6yto zb%ZPd@aNRT*{0A}A?^^VPD&$;kPF2UjORkb)s*o~eJ~E@L+ho_=68T52L~W$K-yGr zwXpx9QVpYozgUI&Qq&pt&j8#^Bcn!?x{e&iHeU&DC(xmfoKgIzi{}pNE71f7gxcxj z^NB%=IChuYBl>&9XKIv0-=hO63BPH= z?k!woTt|^4mn}iIJ)kbojVmnw1*T)iWqUt`wT0%7nUx(*|FVF9-!Ol%DnlHNKIB{a zKMNo3Uk;nLxyaQkO%O!oW(Ea7flUj8L09Fx!$ak6@Iz5OGL|&Ys)O}nB8G`SPf*DV ziQiG(7VRXR2yUaX5m(5Hy6Ooh4uthJ`v);)d33&7Hn!U@>S|>+XS<2uJVC--*VW-W zLS}V_if%Eck;EDIWPap`voubiove`E=JTy%je3rXGb3b zw(s72j81(vUhVw+4ekX}WNUcezai8rh~eJo!wYOQp=VWdKHjB%I4BgMgV!$V9KWN4 zMy2S<#FtHm({s`vmKUMSqFtBnd5rdzP3NH>;6)aZGvKG17qx?+nsoWiQrHwE-|7T| zaFzsb5Ln`ic<7_a+B2u}gXQ6RyJEnOp6ejk| z@B}SCuqqI=X(&JiU*k*9qFQQLO$ax19%<93T>tdm3NWxK?c}^KKnzK5ge1h{XTnTApGVB3%$}l8KD$v!Na<@(s z-Wq%N&``{p*Cg1;P3pnE^pyFEbR;a(JrI7hi7Cw`yuZPkh&{uZm>8|lV4K)`9J<>pyX8w4vUBJzB>g)x=uld{!Nnx%AtD%X z0ra=AkcBhz2@xNj5ONJ23R8TT3e#t`qzV{&Ga+4~Z`Y&zn~=moKUDF>BFEnl3$meT zte@83wzqa5>%0Y5{l~q%APJ?pn*tbF-SqdTE*95TToH6Q$;w{Wg#NvU!FH2|i!VB) zU*?YGD>_^}9J=dNG2NOuY*xT1s(_&-3>wUGcV2Sn*Fja zeTfC=^W0Kluegjw zstLXi-`@rGS`_m=cht9|Iwx@bR_KqNU}48J-l$X7#Gx`+wySF?Q8~($s1u-`Wao;& zujzpAA8`B}#XZDJ_=|dKI!&@z48Hw4<)m(Wwi3Rs2dyvbX^QKpEMUZddxCD|v4ocW zntl{_h<8uLmX%s}6B2qqp$zi`8lZ{!E^=~MdE*Z<+jZDe`Z}>^ZdsUDr<+FL* zm6#Tba(>^nRyH;-xxw#_{rIwXw6$dVj*D+Q=AL7nX0`wu*M5(;oS^c;!fV%HC(i$D z6`BUQXE5FM)%Ej!QFczTm63A5Y=nGvx8e1fChW!vQfhM$#}>cL4buNXDwf4K^cOvVcivDstafWn-5sm<0Z9!fhi|EIxj) z=ZW+_Ahz51xpD8YT6>j`G8z~PdG^m<@oPdZouD%1qa=ILD7Ti`Y z)5ar=bi6LK&MjY-=Yp-^l|+bZAaB3Ob0_gQ4AKV) zy?$sNxM*9d`vjMtJ84P#jV)mGTuEx@MQ~nABCclH+N1q%1ulMc{cI*%x%lbEZ0KU& z%f42!sojBlk+A={?L-8zTJRva1Et!N4?;UR{7__9f8sIZ&h=*eO#fmv`JNsZVJU}6 zJZQ0ng=%P4(Z-w`#~2I|5~@zJQmN)jl=22)(f0@(oOOQ{jU9E>JW`pBd%p$Dbqyc~O5nAgq%q11Sbr|F&CJ!zmD6X^cDbVOmPv%6|_Q+G! z`9{->y9+7cm^cpAJ>igY&RmG~06+uizpY#@=^#)>5w{3r^)$vBikt``-Vao^6qiP@ zfZunwa-?N;xuLKdUcHK*#R)nCsbaiO9ouZyev+!8%rX$JZBi!^|IT!k;is=HoQj*Y z&=*L&TwaVw*MKJsdlxu&y4FL`DCM=|*>eIx+r69IvFNueS&lx!lmmp%g^SyV1IK)X zXR#%~s`%cR2@2KlmN^;Y3V%3VSWCwI4$|aRHdVNCe8TCGMBVIuMMrwSm1aa}?rkt= z?lG?A-6|>9+XXvH@Lv4=`qFpYf2s%1Z!XuuS+a#`FnHf1`!eLz%MO=$VZIt8`ZrMj zmdBAB-A6dHPkYoRi!Em(aY)N@n(6GL;1L<2O*l17dpG!37W$3sp+~HVb!Atr+5Yu| zuV?j>ey4VIGjwsF^!T-D-Rb`jx$pRke;v`La7D((GwF?hVLE6VApG>3Qnrrkz!hyln!7KM%4edIfkUus7AtarmhBAwrZB3LIV zkSc>a1dcqi7EN@sZd+FqqHYR+3{N2*+x6BK)hl0H?Y6S(lw`n zDfym7o#-4{=yYi}Ek#*mrR+!97nlC)1nkNSnj)c)A6H5+W^zuQbvT8UaAG)eMYzYj!ehw39$X5HF!ukb(2D`NNFx5{h*A zTZ2OllW8a~B!Jc)e@ZpkijZR_WsxqR%oLA#Q$eTjMxAy=z%i6HKfri*T%OTpSK7Ww zvdc1P8Q9;%<7;18E4FyxL}q5dfb4+poV5EZv@F1~+#b<7D&C;N$NG`-{+#9_UFDX} zcxb}_j>H{JpcA58K{R&befP7{S(>$G!2X~(x_K|<_ZQnt5Ny;wNBW!trFoq_3R6OR zd6#p4G~MDP^uyayq50a=R>ZS0#z$vbXFCaxSvdU}jHrj(<<#^(J~@X(pj=lrG#;qH zXg(nEEwBl}5#^1k%V3zt6&98dt#n(JX$8r$Vf^KR5X%o=?@>I~hasHxnc@w6EGwq8* z%A}-8Rw^WDVI&6>rZ()qvB;V>%lP667*4sjH7_pT{?#tOWH%rGIwpw%XzRLKTsZkB z76Tk|mEBB3<6m}*F7>Gp4|GdeBGi_HM|fDGKao$7-dvQLZ)bVgd^_3LKJ@25x#7a1 zm&8!O0FJraNA zZaEo>caTqaJv6~XcvnHDoHl?z;yELpyna`l#22Ty`2*SIrn+S2JE=f%LrJ;@`3mmO z(WsT>0D<(Pghc|xn=oKn?dOQm@^(42zX)r)K(H^KN&$Y7gs+qS^v7bPx z=Q7g1noV+GQ;BU+NI8q-;yl9Ts$wKObRGGtKqa}S?}}{(#`a-J z>W*Q<$^rh^;$0ws&09Zw-T;ky!TFIK^sm)&u;ocx`C;mr!0O8VOq7(Q)G&Fht{EM_ z9v4dZGjd5lT!lzX|EFWq6f-tLAi{A7WTToknWaOo9|hE5j598s>I3`@;cn!ah&yUw z!0RV$^Ya-$?75}jzk3n|F&y-8`!O5d{6I;&CS8$zFgO?A!t#ZNLu+1W{G8t7*Zd`H z+u;L@p@$EQ({yW#6U};^!@%g7`emXq)*I`8090IWb-&fLScrZoOGgsxX5p|%N`WAH zXdMG`y$n*_g=Az6EolwiaZNvSP5(%Z^r*8W^F?1-f=>hq97_Lx`Q1h#>IflIFI>I4 z*bu{?DHY%NNo0zjLlrA5!hQ{5(!6ItBGq5E&28ZRKoEvlCX(oWu_{s2))@Y)Li>GIE&-y{C3nRQrLPeDG`;pJ8l<1UFlf?{(MJm+sMq!V@MUXIo9_ z%P*PDJS#8n;@{jC8>c)CUNC&cW;!5#mj3DYhV7g&I#9gQl<$azXx}VWXRUf&P0;-I z#ECndbA$FJlmNt0W^@M95nQ{;+$yXJCRSWgr0pvg8P}BWHYU6mhJ$`F(ZETonY5wQ z_=*^+wG8Y#U`IoMsR{kjBT#|Z(`Vwm_p24<+uBMn;j6(zs5h?P* zZP;%&Mt?0-%>>S>@2Z9CWB+Nau}lu{~xqRoWn*eM1(>$9TweM=x(Yhn6oBzID{AeMt3 z*SAy9=c7K4E5VZ)=Am7%&KA076@ZM^^(FX079@Ok@3zJ0jpc_|w_T^En`8*Nr8Spt z%l(IB`oYSU7;uKN_3w>gBI4*FawF{Z%us})W^=j}O&d|7g(|K0G5%e%n+3||g|H)| z`!w(cfVI$aFF_X-gTaH_us-yry=L=?mw&G{0I9Nt>luNDTDB&kSLqCUlYNd3L`GNN zos<7X>Wdn@gVDs*Li}5j%Ip*v$j%OYVNutJt7j+4p|HpjBoq7toTi#VV)+Lj zo!)jwQo|oky*bu!`@dUMY!ITrekWs^wQ0=84Pqfb+`cW&D0&g?TlP!!mY#p9t2$9= zVUSOIDd^~VW;w4V%d7w_9Oo`PoBqFC&VLD=Hw{kXiCzXoRvZ|mQw%dZCn8VPU)cou zIgL-9*S-#wP^g~Yvv6n{^FXx1UpEdu0Y-_qG6f4c5r{ybdZ{vvKEg7V8AD*`#7nrWC{ht4kbsC&6f4CYasJmqXOusi|+tszrM|T1Q>LSE5 z=CYtK@%>BTYKi1551HoRj=_OXvvN0B3K%o=vm3=_6xc#MF~ zKC8mm3y!&%Pc#dq6sOI+^F!2MR?4&JmXPUFG_NfzFG)zIf;n6Z6C1uxnHco(dFvo< zrOL^BD*#{ySo}g|l#t-3&9wwb`Z}e}S%g2D>MOZe}m#3VR~+G4>adw~BNz zUo^Gv)=1$HKi;!|L2=k0H8j-~-CYxT=NgQUOwM(`R3yc zhLEXOiFn8DrMvSyzuDyY`&X=si2Hz5_Qj31-7A3{qq@ZYWvNKXlVko5?zbS5U4gLk zX0@M3=W~PmQR7Kjsel+5fv?tyHp*04QL_Wz6Iy%tlIVX`G2PkN8Y}#^vU|)n9sWvb zR@Zds;H}gUel8RYTAzv8aL*0!Afw-=T|jEv6MBEoEB95u=wDd(1uEMT@BAy|kQ4qj zG1)ydNiE@|691Y*(5b1*piL@S^>vT>m@ClK4YgE^1!)7da4(k$zrPej8y74q@rLju z_jr-Vo5N2hUD3zBBdN+*Uq&(IXBdUEDJ9D|=!O|$gCn-*6g<@u1b>QV9Pfkf4TR6M zt3tO~p>ezZ;*|ck&t+D)f*CEcG8*%d7vSSuMrx~(imUf85w$ZjeZD_Uay?MU~yoav9G*JJU zES<9y7e=8GYvlMmlEqhzKh8qMGm1Xnt|4qj?0zQ4J=b5E&*JX3o+C~{Le5+{;fv1x zRN*DxNZ!^+y9>3LFmJG`!-Q`?-xYH`L>HU=0rcy*!oiOt@U}XJ?vwHS&?+y{GPm2aGGDn^BF$msU|PTv z+JAyMW<}3wa5AXU7~xeIs^^NQ8S6lDJ%z%ABVxaTpTzLVV1rdog%XSIBD@tD83Oca z6bRz<{aEx=AftCaRo?OyYg-m-xtbffN!m}qCS!4#Z~YM&p5o>2LXbYLW`hgahBNjs z6-guKjn&{Zi&DN$`j@)pxF7y#p?czU|9DqVFd_H8q7UiSSRO^bEuzGI?X@5Lh%O=Y zXO1GoXxl4#nERiRRf~2!k6C_?{uAI&?|V}r^xG9gbh`2T$o|k;%a1?cSwxrc?yX)K zNtyiew>HGq41p3Ow03en>N(4xg?}-a{*?5h?$o?(=d=e)q#ZQ%FBtwz%%^iQ(8`;p zW!vnZ&i6NCMyQun0S8!UUv-^)fH#Pp-28c_M1q;>y4)hd6Y z+$*aK8gIh&nwjgT+>5`*lovtw-&gUYwyzImvr>o$Fu79U#o7*PAbSe%jbwzFED=4j z{x&cr$2P0PHxO+{W=EO%uKi*|aomU}O!K392F(GLvgqK+X*o}%#u+j>4YMt1flCR2 zBV%ZoO>GC{o7-)hkthY|zGdgt2l{Mw@AuZZ{+4WYs^<&*>Zn3E1nGVtb={t87*eD5TT-zlWj^aRj3a}}xnSSeVX zCrt4fdp4BiB>C{SGFvOwX@2{nQWnw3Tlh9-koWl7a8R?eetqw9ekE7e?Ti@t{QP=@ zSR8jqL-6ZJg9`=`{#zBKRVDXpK3@!1Q9Y{JW?vDp)#$=6fOHwlTN94I+>9}8?8-uK z>xfQ-bn?cSm7(~a2k%y!%ge;y3Litkj&$h7>Gb}&h}hB)&5%~uR#eGWbl+CQU?}22 z^iN}Yc`mXLQM{lSoVomP`_ufImjS5#R+o#~Gxwxi-LM$*Gx^)bmgfSKUFC+(&qr@7 zPc`lN=52phas5ShPs|PfqO9asOsK={TegMAo7LX=OKxZS^AA71dy~$;1wasvs)@_; z5Du-g9N69jHIY@}V{)`!(70CoL$N>5iG`($(>DR7C!VtmW(<=<^(E_(a;bx<9c>5D zh_zIjJv?SbSbQ|Xnc^P@4)K)xlE`Abr|CwwoxWLY`hdzT0QpH@kv412>fNLC5bSuF zN@uN*?&u|$X))EhXiuoy+3`y{zzl9ww5P?wclOPa^l_~s^GSJbiN(qj8v8o-3N~01 zAG0b>hK`99tb&Be=be~6$KH>wcS6;wJ^4hKwrbAL9EmxlTJLv$m)o1P6nD2<&;VWQ zLycJjald8e*j4`sTS_{?p}XTP%-aRbUPR!1*Rw^ofCRMAf#qou@W zVvuHHNVslE>TZ$h8VTwe{gRI`%#YlDlPa(hx$Th#s!~e*6=QH48G2*AjGAn`w$PcB zf|n{pDV)~br>naa+JS1iiOoD!J5I*?iMPGbxx~ub!C{vGyAKv@hQ(apED5P z@I65Mc6B>1BQe_tiE(Kq`7-r!RoJKeiu`Tt$fuZm<2q_mmcnV*6MUsRXSVCW}X_5B-r+ri3DWOZzx&xz7p;s~9b;Xh1pr2NRF z{Ls^T(pcn@7fY3?Rt*KmepN8Zn%5a5&Y2{Q^b)b5n|)e~tH;sw z;Yk=Q36}L4rtId2gY{WC?=kFD{A`(fd%v~6PSFMt!4In-y(C2wjZnt1ym2C<-)xS> zf{?F>Z+XmPZIX|oJXAW+ykn~U>{$k&39EiwY*WB1=Krz)SQIlsi%#Ex@=Ak^$z*Kf zqz@qkgJ;eNG=U%F0udUFQ3GWN3@g!7wF1@4yYSh&-!WB6Rw_JJs-ejndZnb#h>cv| zyd>2Gbm7%!t*KHRk-dK*z&C0L#+-FC^Ao>1-oNlU&+g>@Nem5J@VLDfH-t9EJuf)>bflKw z<6pN+Iy_RC6cU zlIn%9szQKWL7mP!+bfzY}nKghM9n9h;Dmq{SWxu@qVBiXRJOq=jC#HX5A>5 zAY%@Q_w;TdpR#i&E0jXx_w@OX>7=6|4;ij&FokO`x1SAlA8 zfogw&YIlL^fKp*NZDDx*w$R;T^ET@~ok-r}#g;wNwG;}P7k-o;ehDq}Vmh0tSY~fq zd_RCIu2uo<7J?Aji1OQ3f}@OPNOK^+QL~io3i;A^l5e$S(AlOo6l@M#m<~6wpx!dF z+Xuf!zx`MVW%t`V8IW)nc5HC$zpaV^4S*4y?(O@r0FFNY$lu{hSRXz}x3&)(V9$AL zyB59J1)d>E`j?Gj9)P?%NOi!&!2P}65Q1z93;4ypJ?4V3Z!y!deBO1x)(%pM+RQA9 zWDvvCSD$*?evihgFs@|z318b7q0!~Y-(e(8y=qx81{DA+rfAR*FCEfap^s1 zh=iR>`hv#WVa4wp`rqu)33t#KG{U;~6`I5_Ks>(KIN@7w#i*Y>B@rOER~gL8lG`T6Cl!HxfH^>u5je0wtlTD`sbyS~KR zWHACiz5f1aBU^0I9s@bJYB6{7*6t_=ajiTWu0ZuypdNb&h|T%#sC364XpcAf@-+Lx zn15?yN2h*N)S`7bz1*5z4^VLYk#f24;@SSng!E=P(P@aBPL_ZfH^n>n+{42-rTg-V3P_$TXqSm7nT&2FZ)$aezVE4VL~``m^NF6WcA&`j5eka?@g z!SB^Oj=`^`HkI_*IKPkaK7J&b2%N6Q9WOv;yDg>A7^Yl?2<=0ga)EAad(N;9T3SY(yd)7BPlf*Ct+<;6f1IToO|-QHQPMOMRa=0kL&N?Q4+5DjR$=9A4B|b03LP`T*kx z){0XAQ$Fi{zA5n%(Jv9vY8-Jvd17m3suQZb=WjZONCy_Hi|RYkK->HTAw@jah;l%Y zwXAqZj8O@sjFH(W?NPT(4Vc`p+Trl(8R7ST5rz?|SwrRW_#GT?Mi2Kh zMwix))u6V9c?G!F6=8VCsu`&b;dK=1oC!B=L*RIg#tfS>l#izk ze_FG7P!3kDnC@M{SpU2^+o1>{Io?tvZC2(Rn@>+yxd*Nm$yNg$=W|y6)no^Ha-vJ; zk7#`pWVOPh6?<1pJz5m(ogt*BBKthcZE~C-dC6VA<{p~4!bdN_zJTKXX{an4=}Mod zVRB{`3FRHGVg3N1KWqMObajROGbAMvuYND(k zSqi_3pT)7>u5IC3{qPo?ZqW9ZyZ5qq^3OC+iF@qU%^Af+I??W4kENdh5oA~`zSWMG z?NW3?tGsQ3&vV;8PQ4Fy^l-EFdNIb}0T`!_m#7>=_?Uw2(!RFI^RH;$_M;?cb?Hi5 zt?5Paw;0?b67os&2`U*d5xWo2>x+qJHGu`g^Tt0PwTvlHpu%5=tZvrRz4!*P)iW5c3gXsbaEkB zopQTi1i@+ebn|A_eLZK=bwVcaK6V#${y=n2N)8dlHOn%b(JkW#R(>d4>#}xpPN29o z|8#mTe7)_{S`$e6B1CD!VgxB+Aqd13R{sTLpz=a@CmZ8;>molX6qV2gIHXI7hR zwQj{Y(SoOUe9vEXrg3hq<8!#j!CNf3f9AkVJQMADIh3<>VCrd4qZLS7>}bii>=mY7 z5v`?(BGwpiXqujIkAfb8QtEairjaTS#l-2RlJ-`itD<`b$ycI6{??>IK3h9}y_QUy z!cpRG^b(I0kmBMdK7su`qhQ5!i6F#wF+Zw(HPW2mf&^vf`B1REynLbe} zy$QDjuGYIAT>3m2IiA0mCR_Ceo&6o^JIz4Uf%?ebFo80&h8M#E{;83S5N z6jLhE2qxDC!3<7TK*Op>C!^-nH{kuDU>^+;?dQpg&UJtA9ec|01{B!j^kU&V38M&m zp!UnrGkPfjxnAcDy-+YaU-p(Gy3@Dm-h=YQf7Pl2k6VOn+n@C=HJ!7YI-?#}cs2M; zmaHRGLyq(H6R{O#yB9}T&cI=&G}hj9kYGbz7{!MJM$abSN-#E?b6&%|J67X*p#5lv zgo739ZnQCy@R2F4Eb1?ObWme}t>@JqdGGNG{OPgYPqH8?A<3l1S_=LbpiLP|$ELzCP%!IEi2GV|Cji=Sg> zY{0Z`n|-_;0m?~qQI?z!`=E_j!mG%PgB7}|$$@2UvAR2u$%u6mm=me47--hdrOHBs zxb`n?)MnM(mpF^kJhSuT7c;>DF=xQo?u-+W3KyxpnVY{JaM8&g!d+ zIyg>fQ{30b6YpKjFQ>1Qvc8ct8$cxGU+%2d{{_Da&F;S5xB6FU-5JbBxuCmX&UmX8 znhFth|3LkPYR!`H*HR!cZ~$mUERfTS)1?+Leq&|jzuuT~KY9K7cD#}*M)vT4)z;~c z`2=pb9m6Vo`z|`#U)AP)Uj*OWxm;^8=m=j=F|lsmTvZCpwI9~x`HXua9+jVk z=C)fus(&{76+alrmbTh$b=B*mnZBccDh$rgPS zwXo=zQ&izghsfAD^NMF%UCeXY%40rf1{bAM3|^Jo5n#Cn+_JLSBSk>?$jcknpCHL? zwYj5!N}Fi1W+rn_RQaq>iv6e;95j=s`fSMvdee;J1 zJCeBMk1g%&iIf-Hj?b(CY^47C618G&NU!*JLsHFal6>=7x+}fAwytwGI3`zo{Lu>f z=H0Q^>CbfMJ-T80>eR%tvB8_{?P0-HAZhQPlkGJ`Tu8GDSnHM;Q3b5sfizuniWe+~ zQ6Cx-_B%#C@`t%iI6c3-4vjuIn!q;;^RhXNJ4cEuE}T01nFNK|3O{HsY*t!^Rjv0& zc;q8N6dGh84TtG$@-b}krxb(pEbA%N&e=sSsppbdYQaSh+2>{0gzJl85#ZKJU~? z1v`;iP|Qht{%(jcD)03fCEgv)=1QFa0Wv-Vh?`nEI0xyy23<{`iB^rVMOV{>NKVy! z*;r$*Gmo^Y4){K-vANgxHk7;E^?h78TO7e^#M!vSXCo2IaFtWPu}9{0bSq|pSGraaa+rCR8q&PZ-&NSKJWfQXa%5vh{hUN@cG}&(h6lXQ##Oz_#B z0MREHlw76FGg@4V}5<&rP!J5{+&QyYtWN3tRoJulM3_JA(lf?$bSv8KZU4+?S|Lg~TV$ zKc{^}6(=2DPB-2*+iRRE&YyMLoK{@Pp>~bpiLbxVL*^RL_ysM~O?czT*{KquAZ~@; z?YpPMG;(4gp{K@uJ>PR={k<%5{*`kUKSh__`c#KK!oVx0nQAQ#LioD)dV%xQ zHqR~JAx|!VsdaQCHpZbV>l4oT0;K=*A)JLulQR67hEw_Z?z0wfd_uimo-tD=ZNQ~X zQrg8t*-C9?PQyzWekZbr4!^0P=8pq!lt2F1g@W=NVQ=Xv>wR%St?l%T$~C9`nC1v8fO^d*>OB zkSHII>JSKCa(OlU|HYDoaDJyNZn4 zk6X(hZC)TV?i396+kr3c6l~}v3OmwhD-V)x(ZbnW3w$hM;)xgh{}c`X*>e!`PxBHM zovD_BcgooW`{|5o88j5GG^0@ZjlXfGSLp2E$0gvXsX}&>RH_E>>`**;E#;KcIB*9} zLpbT_oTM4FWaiNSEMxQ1OSvWT$R5P~Wj6i`GvM%}GqD)n;Jh0Mv>pvM^MH;%?`1p> zymUBB-oM_PHg{e<){pR_@8qnf_FZbuCxb9qGjmlOa-m&IRz683huYR)c8$T7bu{pL`W{PR6#f`CB8KI$rckURUUMm0;kwaIvRn+>btbq< zbuTh@g0wQcPwXe6@Q#5=yc3VcS)-<(7L$uVc%(O(M0{lT(aSU8UQ^1m(9rUa7R?pT zO-mU5{Q(M%r%P4|UGpV|Q3>(ir0ri`RZ&Mi++{#lo$cb% zjJ@V3zKvR6KecqoWslSuJLaWtV-LAfa^?%y26~Lf+cRCf{2f_`!^-MOm`@w;By)y* zh&%4axpAm4XTi2AWbF^lp>DqjKY?`1TKN@w)rb|{K+AVdP~4o!dXCwr!&4q73b>>7;Ju5Y zRfM>`W|uMBBzK9Dqb)eFx9Bkm#-!qCa=u=Uo%;XzDyZWEcPHbe{to)HOU zlt}YmL3hF)zPhFPE0txLlRR|88TzmUHlwW=_1LheIw;_VVDBr(UUkJPzEl`)MOF36 zT7>ErO?cuaz8p7=`)sfPEk<`LJ}W?Vbz`=}dqsZ~{@NZl=}6ibJ1?OFRj5FaH1{=B zV`P@`R>-rYqDdcGs=`R6T2zCnPNx3}`%sJb``&))SaqnK(t(8Ng?2JijAP{YI~Yby z?nytiUa116xmtv5opb)}@m{JiO}{=dY<@Pmq&E=LV|CVcAc{ct({c?g#w|n?yDd$h zptJ`ndF*;J-nHo8USI3iq448_m>o{9 zZ=^o+Cr*z`_p6((*1j*~KGBsvdstP~ahvT7zcv|@ zZv7~OX?SY)ym+n>M}le$ZWx(!1qDURkBAT{d>u{QRgrYp^1B#D!TuvepCc=dBkiI& z5Vh>m?)TcbW9nLI!+M9QVbue@xsuTQ%`ai1rO3^r zYXK^|v*LKmU4u$=Qr>%tq4cu6>Jqzg_M#i7x<6rcu}s2visE_h(n&Fhi*i+G^Q7+L z4=3dsBFiw3Rs+OU43?VfUPf-k51+2!RRSUzl)g^kP3a=VUK=K76()H64_j{)6$ck| zi6$gKkU(&U1b25$f(D1iHMmRT(pa$I?u`a_cXt|SLU0<_;1;B@9==&??tj;sd9H_Z zUh3>qyLRnzN3YIAQbS_YS|Qi8ip2&+ef?)O1-=)MH{3{-!p%#_e0$p?n#`%T#9t!^{++%UxEzcfBi*sX8+#S$aV?k z0Bz`0m%ZAcGf7uCC1CVUN`4$;U8`eLpQ8p;p-2(FEuv3e8zMTqxwPC@Wn)74uVC7Rv&xy%TNrhPmZ7iVeFnTxB zTOHyOKzU=ZCh^3dqws8StYCPc&=Vsm6zjj_-<9Hi|B#!3!6Nt|moXe$Z!SHfbMaYk zZEZbw_bD#R-`B6>^2&H|NTF<`GQmwXi=w=W-BL8>%<86bk5#5I_clK5oe}FyFt5Val!tMVJ6(rpDiY+=3yp3v>lMXVg$`TDeLV3Raf=InTj9QBH1mPFu<6e^{m6kkOY35v70vYP>FC&%nZB_9< z8(fwC{gMgM3#1*wUpfehOi`^P(*V;Gw0s+~Q*$RPd{wvb`6|mM@i(5!+31QKIr|Oj ztFsPItmjQ}9S_9#6R<87w7dAS7)I~-=BRRL4I=#1JfB!&Z^w3ISNCu%yma;DbV)O@ zH0(sFOVRM-DK4Dy47?2WbjiBQ7an3um=Z~%ml0KJ>cX2l_}ze~3^)RD0pV%Wzu;>- z<88(idq

3_~Jyk2Z}TcLsf8;s_Yvo*ff$cH0ITHPbG*jWVcBDNRf+cSvsXqd&4O zCv*Yc)eXF-^A(^)_qyN^P=$0ItN=Xm!N(O`ZVhs?$7|7UzHoIHX;}goB(L{11p(1?A+{uzD<_SkWcU?mXe+ zmZ4?8$RV23HECy`tC!hdXUk)tR=jhu#K71XdDk$j3GKynhsH0V)t-Se`om15|r=^J}4={|bZ;I&84 znFzW>jpD`8GVyKsz)HFx$kHd)GepVgg?CuGa9A;3hQuA2tsExbRSWftK{<&8`Pv;Z zV1|P(o&_BZ2NESNlIDeE;Dhp&znZNq8f{CO@N%PVD*E9zZ*vjsJ(<>_CNIQ4)|oF! zdZMyJFiCr=5hl1Wg0njzpd!}Am~YB#i!rbC@z-vhH9Xz_&5J)JJ)*V123#79*TmUH z(mu~hMYPdcpTW^AS)A3-$u2Rxi-!n30;O;?>w$Brw(;7k46;HV6$B@le0bdZap~CC zGm29u)KxM&#Uvnx@K4pZqH8G=GfjTcw4d$xr@z(-g|jFd_6sU!pG&cQp5HYTpIp23 zZ|_XCf_~4ol6WzxvBGM0s?Bav-WY;95bN2a!%6|R3h@GK$dd!;>e!_XiZ1?$KqRJ-IbP z&jD**MD+m`Ca?(AOT)I@{`|0DIsXCq|7ZbhU2E5ZmL(Ru`#k2`L7Zf$%RO3I!Pi)` z&wV%Vn;YhLgO>z?)$W)@jH$y&|vVW9#=oUMF zTIf^W8F6P&vYeGlq+_x#nTm||o;S42V{7=mg(3qkw2-zHdgx<=Hvkac4SL-@!oe`D ziiP!;rzl|j$)#fVp@OH?p~>gzB-rA%Wrd>!VbGbi$64TkE>94u*@pSTbgqL-5?tS3 zR;T_Lp^Jz(Oa_G2*rotJ^IMQBfaii=S%?Pu5w!xS;EY+IC>`;hq-c( zs9wXBUXwNYGP&$Ti}4yoh5stYCN#I&7WXpe1h;Ri5Wp4@A3OjoV);w%F zRmguM$4X5SeDiKVTit>~HH#}C^eIaS!(8#E@(hX53fPB;7fnG?IkI@=OfO)lL_rlY z#;%OX#A8&g;Ny^*frurS=pjW@6u$FzdBa+{Q(gAnVaRKC?i98(9Li#O*2q zC>9c_VxPymDvEmR*q~wKy}+9lr=qDlHzA8j%Lg6~h*ghWju4T)KM_@YAB(bl_&ii^ zoZ{|@p?R02^UQ4X_p$lagk%}R&5#hILc!gPzomS}>ZK6@LuHCh?l1kl^ZAx6Z>%WA zs%^Gnd$Ei&4KfEvl_k3Poxg}!3$F2^n+G1rh#EFw>7e+?0zLfS4*jzO2u~$##H+qG zCOk^tOMVF&pdAK}d!)tQ)2x19EM(uUS5WNmEwS(3Xxug)5zD&P%$w zSASTh&Do7CEyiK5c}!^~3l=|2>0ju^8s0ziW`rCezQk$O*+r zAFDG9*(h$wOK;(3C9}}e4Hc^LjLp3t=FL`9vMMjnw_z{e%d_UEn=B~RO;!w#JW&H8 z;WJ3pN!lt)twv0ryq&T6zWg4O<4qJYFJW(ME_ay!jo*wF>Frg@Ht-sI=2+6DwzH9| zH7B+zctzrW{}vlBx^H~lin>ycj&68l=es43@^^53$A|YioL95RG7_A8kkHLq5cU3s*iLG6e^#1X` zD{Kt}>w?eXdcelk$c~=m-IZGK-+u|K(KM^0$~(Dcxf0`(Y>~xzx)ZW8*^TWPz}BHQ zkBerSNj)sluD`6Dlbl?Oi%MQvj#ZA3kY>`VWEVP-XwH*bsc;5tD{S)*2RS0LUW zm-1dz9mJ%P%zr86A0mrE;9ph-(jLD$I~0_tio5Wu8ElFehLL{4SQT1R<-6-2CH71k zfoE_*cMX$T=-sRT(R(*G2aydhJDtDL^APcq`1X^xXF88?#Cj4%A)yad>Sk&D;_99; zPE*FKoO=vlrtek2JDj365#fjmv-vJysg4{qv`kI4T74dMB68PO=A5m+dwPklk_=fX zBLlTM)+%7DJ$XYm9izRl9XZn=K??ReBozY^iZPd}-ZW_*hL)M{*TjSdyOOe6=_{{t z;QTLY%DcyNMgd>6G&qHkCQ#k7C&c3%OqVQ!-9&H!E&Jpz$~WxbIOzL>q?m42b_-xf zKl{THh1?g3aEj*G5$%W;{dFqGRp9Yct}s#6mX4cXZUjqi3(Ep_N!3$R)1NiS_0yaX z7^bzX-{vpqa8gs1a-&s!Rs-E!Y5p%bU}G2HP}SJt;uqkKwcq?-kV;wq!Orf#e0ivF z6>Rx0_~g92Y^=|6Qf07ioITLy@Lw=|X8w6u&&Fx@{`rs`>hNDs0^EGFFfZ@RvT&=t z@9e1mUvRUrsi*nB(_auL+!gj;@c+6RnL%0rcNvedVDR@ZNR%aSVHW3JAEbWa5$F>)bBz6L}Jq z)`JRa1;XIA#49;n02_t>l9mQHq{J-ozr8* zaXYeP3wNbhN3H0|Tz$R?o|9Pg_f{>QAT}YDb*L=O_6fgTQhp%re0ne@uBRA;tta1g z;+ANq_Z%I>PX)1ENQH({KuzRX4+#&r9(JFCL*WOvq$#Cld; z@YrVwiC!BN4Gd23k1VEb^>4Z_jpE)l(@d!_-Cm2=qM;6~E;Ie)(B5rD|Mr~-T35Q0 z9yx8Rz|ugd^jkN)jq3Pe322}d6Gv~|D6ay1zXUNwP--Y1kRz9*Qn*iV){6d}>1m?m z_^?1uw`9YD4tX`|`K@#SE9^{O)S3i)>RKmf>fYrMiQKAfD}vRHPl=!<2MI#b%KXYE z*S+(YHUZ)~?h>$0dYRm2FaZq^o@XB4H>MrrA-UjHxgEQMnmUiSPJITMkQ3E&*&UTh z`F^6P$xw9H^Zz*S>A_+CXNNB0D*JT&^Z@f++O7oe)8-SDcC9F9u`try6WREjk? zt-2SzYGFkkF~IoB37aS0)0(j1%(*K7>&E{OrpqRw^YrFyYENOxoZ@!>6^V12((i|} zZh!XPlWyCWAg!^HC;d2LHcCb}m-QvQ4_djc< z_Y9jMV!R23jskUw^!e+3@68d8hntkg--#%ChAytTsldw@%PjBuI~U5T z?r2#I*{kpVAsrNH9Q;1urN3${^J*2shZ?V4?L!X6aAMcHvu9%0CiDdy_f{PA?;b8b zv6m?Ve;I92HjGZq0x6Rh2SQ^WWo>8>d;$0fzFTg8iEHj02-W0FQOsj?V@8RBq$n5P z#6YOpDS-YiKW87SV}i{;#*R8^oJ`!K#G{6h zUh1t_O1aG^U_q=hZ{Sic+r3xJJvUS?S+=KM-jVE<7I^^Rqbb*u(u^x3;o@V-w;j>Q zPR}I!<@2q*vYpGF>8&-9Ajvm~k*`-s|*Z$v#c4`fO^#cs`XuP@49r%yf zKmvFCX4HofGte8*k+$}b(kffWr6Ts1+d&F!RC%?VU_zNry2l`sJ-$Wg^ueOo zI@NZT+L}*IDK{f;9Mv{y#mr%My^5*3U{0vbu{ipBkysqNnK+mR9a-?k>((oRj4w-s zFADqZH$vliY$2H$Fa1wbB_m;N79cWgVme1GYW7GXFHhawCo}yvy6M(cmWw{U96VdT zH3_bX6n&$9PtllwNE#!H;Z`VShn+i=#c`j|q>mI?JPX#BrH};9;5dhN_GbVHlsOo| zYSk`Ub-gebGI;N7z*drEz>qe-(@uvZ?9Kw1oK-1$R%O3EiAMt#(b~DBAZt@unxIq4 zB2lydDXrxdgxoc7gn?DEU*4H-u=UR+XGTE?%9t#~qrDwn(0{Ot8GTarj4P79DbaK_WXsPW^fFcC&83c>CA$E4&M*0;O?ed`j7IX}s5IolW;j39v zipUsxNAioc2rhsAPCm^atpj5~n}rPuwxsO|bYEM7HC4uo{Dg1Sq-zikmR~EoWCn@6 zZ6ANa$yT@2IkvPWw|rBFJy+0ke-C?C34c*(u@(_)*Gt0F%!=bD?@rKYEiby5bDT)) zXHOgUy)$3g4vrlovP)V4uTNBIh1SsZvvauU)j>yOhg+>|LprXxjiXqh*^V;}tWYME zOT&iVY?&N8u<)6NyN?!)`$uP*xZh2J!$2W1*I-r(mvXi8WvXfDRm*;xyM1K%k#u#W;o7vKUK$YOlIA|yo0c*_MZT+S92doUvy${xIqbG`S+?&4SOdDda(bd^$8lui6 zT^i0d$`&yURf%=nT?M$rjDvZ!Hmd;MYr4d{Tfd$YA7sMd+B`c30jV}K1NPFhdP?PA zFc@+MuJ`IBO~@6zN}_?sA-He6H@bSX>CQ9GLi(&Awo}>9H+}W8rfp7EO^RP4(3?|sjz zSUPc{V)dm>eelp4j|-XXq3a%yZ9oQQrX2LI!0C_A)K+u*kj3o+|<&D+^Od)iP=3Wz3%m?xl4)0 zE`NAW{00>1)O;dBoJPw3)M6}h@Ahd2_mwULRX2Je!GI7`xB}+19hy{YujV+yUQ)a0-q> zmulR|FSFjXL<~F;3b0%8xlSx2*M#=kysfsz&RM*3El5X9s9SEQq(#qY+zo;9+dFq! zM`*og#VB>ec02_g^D20jR%d$yJ)%+7i#~s!gMLkdGbO z-WN*ueZWXAmxPQsBQ%?`UnW-#wU-WNII zC(82J!hEZZG&A}rqB7bzodRXjt%N_~+)7HVm5Rm#7Nt(c=l!-{(K0})d6R<^d zzkP9&3Wjs$G}6hAwRnQ6Q$%`QAFIyKlsd`F4^*y8FVEa5Cmqs@`C2usp=6;xM^cj& zZO6aeK2nIoSroq06rV_#Z_QPnnm1?WrqsJd7|EwJmy>Vlu6AZBl)^u6HkpZ>Vjh>zp$5t$Zlc=_{x*@AR2{rf_AZOkYqGtOoXoSNGVF##l&LnP$ z-i2rpuplT9fbS{eeOhy}5k5q*l1a6#|BD9X>Gs1ZF0Iw39>N9?dUm%J5lBNMI%y#U z9GM+A_3;QCCwE)xIzx|=(||FTxQ~M{|3Y9u3d)P-QuR(}FP9x6h&__+so( z8v~=!D?W>#PK;!2dfq|{$pX#}HW&*jaa;G73axd{ye*3MTRD&P{`vNMYs(+j*|C9| zHh9KtRFh4BffaGJt-U(6Zr#OUz9KeIRej9(EO~C(g14mDr~1^|;Oq=)Y1h=z_ifHD zA+jaSg}qNigl~*;y(TgV8MI~L-=8I}nl2E*R7&08@FJX zP`R8~W3E!h*j6@{KvmP*KOB-kjz^rg$@K6P?*6%;F<&2r3HZ#NPZHk0Kx{kV^*MRb zGG*r&&HcGd=>gY^uTf^v!d5B-@QG$}0Vj@&nXzxDHCR_Cd82d=QRCEE`m9M*8A*?S z7JFM(P7LfZFF0V&iMdMl+a+^cj_>i;L?nVqo2})*77hIZ$GEq1C9SKBzyzahDZc(W zq6|-88<73$=cx?Ry~8>s(e(A<$-1Eo`Ji#;BvGC0oh_|!OYV_~Oeg~etQAJGw+~Uc zvv9|d7>zp*?#Gv4#=lcz8tEzmQs$8EU6i3YHkm=u_WtU8YM0AySuQO*oCh(}Ifh4B zPlFDvKi#$n>L}RTWZ)gUa^BDQ&{y?L&DV_F6YEgD`5%4h`MxO1{kriF_h?0%=)G%; zM28z=s}Cagu7cPkU#MwB&3o(QHVZL-T5JefgpI0*!_gK|V|{gk3Xgxe@s_Z?u~Od; z&A9hV02x!I>T(^ZK(C(>-o}XMbR)k<@nH9!1)%ARJTVb<=+GT;d^>ja$sq4^Vq-YU zI`(mv@LQD)Z->(2N+=$y=|j|CD0!Hd8pZ&V(Vz{hiOkhe;6>)=tK-gtX;vTZ7)sgX zN%Wz6T1(Y0?(+CeqI@b!L8h$=PZjs`Ocw2aG$4}M2J_(Y%6W2OZNCrLg8P9`_3krM(m!?u(2*7GQum4@ z1c~yeA1KdV$cY-`bUr;1csMsIGSv0TWk~x&$vbIcge4ALsrse)V``}kX;QHu@F&PU z!d545>)VRCNmP{l=b--jD_LO~J~B&d>^`){OmHN36nxDRwDJ!yboYG<2pmRYR^H|_r(BjMLalH{ z(ayP6j;d|@tcl@`Sog=A$(8BUE9wlKYhT#hnR(81o$83MBeKTm44 z3RSu)nVvVv^rmGwh~o@*jx?(CuNkx{s#fQvR^X4jNAhJE%zYsqG~)|bn=>hOoWwR=E+u5zE^=a|iDNtQ)^HM&gql&o&SyO!Nv>yL# zIM$=se!+*d;InD|c!OV1Gl6q^#f3%5a1SaymEAwNt?fVcmJKnu?g03zVc0kf`IU$K zip`#P?ipUFD>z?xrI2|AK(T4WF+~tGhMIMCjFzT;uoZ>_j4XDsIleH|-}ac|k=Y7$&+nTIn&i0V#!eY-6^&S;n99T0ICU4& zs=G_xb+N#AQ8F)aueQ}_W9DiVc4=e()5cZlY^JKz_tK|k7HER3%vQQenmbGC><{Nb z*>Xl&z$gOkxZ`b5rMltkZfMH4-hDkbsQu)&w=3}&wcbapj;O{;N+$qo^|`&cz|3aT zw_s_J9l`N^Nu!{nE~S;tSYUX?)gU^0Dy!QqEjswsp!&8gjH@8l=XEU7UjX1A55 zS8t`ux7n>lb1TS}$0|QCJ^n3w5CbqUP4&@?X>f#RZY^}#9%#!pm8S5^z$OFK#5FMD zVZZI;mk4|078n%XSMkh9;k4c)k6ov6)<5PId#*FuCYqxb@WP@>&oE{oty8RqQ6sII6*K&=!9;DQ12%bl+}?LSf=TikFetflV{-WqV%&8v5Rat@1I+i1=4f^N-A@s z{`b(s{7CQrAQgEA*Y6Q*o(&x*9)rTB>=qiHqQ*ZN!BZ4K$CIsY*?4>|+vN`11+b!lq$)d|g+MtMx4 zi!OglS0o~hPNBK^#Pm1s`o7NH}o-(n$djw|EkN+Aqmt}h^!ZumS`lUPBd-m80$ zz(m<~J)WW32i*j1Q>V!7j7j@7!e;K$3Ok zg5VO+3#=dA;V(XAy%!&0I;jf~LFm1z+u@Ewa6yS0+92B3{KExCV$H~QnWc%qM+i5M zpc8@G*O<;x^$skH9{DA*k`2YghRCu(kPui3ZKTWxU`_qHO4gc>s_HaPmzmBl*%qB0 z!^=R=GMXbdPU6qvs%OtTX9FXYSdjyR>2spm{viuhjhwS_U?5;aClL8t##iaRP#tuARv_JWG50sp|B#oUv>P zh0#`k`SRMIGmBx4S=qiulXDB}!$~R#6-uAuo??iEza5hWOO!w<-t8R>Lt+x~)U&ZU zf(W*AYJEQ$iI#O`qhBu}vX8`n8?RI1v#4a2PC7>JI7@S04B+g)PW#=D+-aSuoeBZj zyJ4&|+3(aHqU5MIUrO18ge%aj7?bZs{1NQI%a}f-&COn!u!?vRbZM> zqo+8>I($S(6E<19%W2uVqaH7qSBjc^{Z3k?^boCl`tuyw>?FN=MTD(DsmOZ`f&5dx zKZPhtwh5fP`b@z}7!AbzNPk57?9szo*T{1O;?CH&+7wz(l|`Kday%88d9o%wwLk%# z>HGXgegS(ecT~>eSkg@+|9od@UdQeWX->nf#y1Kw_!%aC{*nYUFA|SY9zJtrQss{R zA1y%gFT>d_q1b86XRV5Jc=Z^(0i{OK^w?TJGeDd4)*xx$FJ0*8{i$fRh{BB1)k11t z8Y%mJh@swT^}n`~AqVC$R=v*N%3WtYVET9xsQ`)vG}aHPW>Oy=eZ4}vRjl&7%_c>_ zBdc+Fg!;ly6OOv+`FB}7U>n(hY-nZYe&s`mL9Py{$jN3@%zb#@M^{Yh6xWCHE7DWu!{vZ$dXv$f6W$#!k z*!36vzPA!I25${^-ujj8N}&j%{X{>~o~^<3bxW{CEUrbsl-=4LZq55bS@Tob3c$a2 zjG$(T2=z>F7Zi?##-QxCkbz2%T-F+)-V|v)c6@NJKYLduSt`Kio6VD`22}u0c zw2{pETQm#z?izW%FQ!&&l%5%75nnKgEZBhUd`T~MJTX1s^yY5kPKF(A_-;fu=~hu- zF~x1>NLK+3U99{>D7kRbhA$~V&F#5JXrOJzmGzD*X!ATMb7-}Eyrm#eMfm|N2HQ(H z5wcvdq4q;r7r~JLvCTk5Yi?-P=1(uNIJPhn2Hf8#-STD8PD(?*#hG%)5OCm-HIUo+ zf;S28{@G_ob?`wm9Zeq$Efu3$W*J(=XYamL^|u^~8)JF88i|gKl@BwJvUO9ocs0O= z`Sihs^KG@Y!7OBi#x_G>GFmYHjo_T|4y#qHcDRGlk~6%O3qsK}3F;yM>QP^GcUZ?@ zl?%Q`?fU1t?BZKS6FAyF9>DCK`_Ai8aSw9uYxWe-hvD`7h}sF8uerK+H~OF2N+RVy z&>BMXG>3fWwJKhB{6R8?NWm1jziKo3lk1;*!yUUzo6huRYa?3Ugdi4e+e^OC$Kvw7 zU(xRa=lFkU)$u&_54yfCcypR7zY_K3KLAYzvcA-AS`t^1Uc2EJ_xC~2IGo9k5*3qqN=oj)rneQu z8>yss@~AayeY4b<3%|a<>!_1~PL$L+xQ5lLrfnxbC%h`@`dGwCGE{6R3uKcw$_8r| zXbIGgb473~6ilerz!Zh(=vPejWET8ZT#`Q9m?euQx62EuGA_^I3#O)b4ShDH+Go_G zn=Mx`SRFeU>CW=(+!STy$umtE%YtKWG~D;XHec}|?i}zbQaY>DF6@}_noM_qAPh-Y z&I+#7KVLcco#%w?-LV?PV~QCx4*BHo@il9v1QFXvalRvcu9TiGE8&S++!3MKtP2W< z&zoHS4TAlAkx1_z2m!@|LQnvHg!?Z!=>_%P>SF1?9$(d})NY5AKUoZs@!F!w~S6G}7tM9W6KjEgT0 zCwWKkiS|#8o2{5B0&6n1pYMs|2>377(aG+dhGIkw*Cu9mt3)+dhLehvMAgJI*Z29) zB`#s`xPA{x=Ch5GWkF9NWJk0Y@gcFn@_B3P%+8O(F<5qBl17%%*hgk-V(A#rl4i?2jK7XLH$ zzfJGj_M->m-&1LPqGS!ZBvX&ox@}5(YnSAQOI!@rC?ie95qcO*KdtCHfQe?DtKN;y zXl#0UX7uF7b3#R%^(AIoDB^QoLid)fezwT%AkXrc-sTIl@(>WeOIOW!8m5If34y<`u!f6cg33*`ZHVm z-6IPaluEHtCn*U(b~{&t_u^=^JKqkR4SO;->R1S0A$Mm?T{X;8i#Y~`EHX4j(^|A_ z5O5S*#|`qxHxejzAMV1x05bTQkQk%$RIRtD1qL`2umZTn{I}iC4I9EYEy*Um;}j;S zC2UD#S{7u6Z3_k}&$Gc(9Qr=3-9=IS3z94Zz^JOmc8cU|k4TSDCsLFikN1v;#}?*w zs0$8s`nGf>`0_{_Iz;2x#^&X3n(I!QE8s?vUFM1+4My5z+Y(M@psJoGaLw*#HdI?7 z)$t%btjDR{&jb>Wf1|e>at{#u%ZP6-`>ZcNbBcq(rEFpSa=86QWp=x2>X(>YoVe}R z@Dn|R#Qp-p?=f;}zD$Bluo}>NvA?=4_&s?H$^{w&;$^#6^`JXJxkss++1hsJYa@T(C$W`)8Lw0L zlGo|uH1YFe5<}bWaCqB=RZILVFWqjXdA8UEV3Mi$G=TEw?PkzSId7vEZi!-cYsx3v z!oM~hAw7?OKb^ui0gor`BX{#d+ITpD6DmoyZjCw#`qk~Z5B|cLA_32Dd+vwWBAb{J z`+$ZzV6?9}qXGhnwYWM<^!I1EQ<*{!I-+(wfZ-|n)VL&Pyg_gX(x2dGj90gtFoZ<+ z*((6QEvb|uHWQNE(x-ZvbVcE??2B_e!hW3|ukE9d5Q4K71w$!THTs?1PZ*Cls<)TJM^a$or~Lsju8&fMhn+lA`Sc-rB(R5 zgjHY0t+z(a^tl#~{(v{}5+J20#IG=@h*#%UkSaMbkB=doKQe0LeXPiUN97$n)PQ|? z810~VmZd_SL2t7oN5ty>IO6cYAjIw=Y&3>FC7AjkKO|fYxq=94BgaW(BY$=2VI+Ki z$$UBJU7|^#DzvNoN9J|KV%VNWJOUZBdC+)~5PrZP0%xd{FkwdJp363uPHmr!*n4-r z61ypo)D$WDu*E|DCN>-W^EO4ti@zTO!&}_>UCPROjn-!7+WW{e!zqQD&iv2Y!Qm>v zV_I@i%%10`lX8mwAB6F`>61%8IJ5N z0%H@Ef9p$Gcb?6%Qx>jwNFGBR8=nn!il)pw-9Gr51}8IIgzdCdLUP7$20vn9lJw{I zxs#VtP2Bv|*2xtLcVFVD`*cm`AvI;;x$SKqiEWucX(sb+KS3)uYgrytp`Q0khc=7n zrxhEuO_tmWJ!Qi;8MZ{VB)OG*@spYO3p6XSej9zRC^IIST82@b!WqJ{MkVI$gi*qu zb`}M2%K{k|u3ujvtfh7-JP>OcjP~i+d{5KERbn6PVj;{K@<^K-<|(U@!!P_Mmu<=n z&9za67de=$ptrP8LfE&oWx13J{ff&-rL?RBHh?3wKJNO$Ym(dBwlh(H#szf0&Qm;h z?CHMbss&(o8*t_hc-DBMCDLz@EFA}`%hsqPsG{{h8W?LE81)_Cq^+qUBsY#Uk;&VK zF`>Qr!;w(cYMup?yIb9nr}^EP)9XKcq}3Bq|H$8I5;+IPh40C;gaL|=n>PvCN6V2S z9fyh))0QJNGwZr2^(Le0PCOFn;_Qcp8JE4NGn`^SI(a1Bz~sig^iHXbHO-v$fndw) zhopxMkgLFf{q0S%Mf7*PSvox zzUj1r{7wl5H(|?WE1Q=xM)#apZ|%t^L)*Se%%NJTZ{G!T)}uBBOtdm*bq+I(hnm95 zfc=CV-X+=usiSAc!^;Igt{iWig7Z9kzqhQcjqz-$(;z101i17pGxMS{M(lKqO~Cs} z1b>D)c3GrkvjMHPy=0ABe#$Zz#f9)`-?*n*eb|;B(hfi1T#sp-Cs-gkSs*c4A{kVM z3Zn>fGK@aL1bBa{)V}Bv=(GfX#p%+aU7wBBY=1l6jz9j)94ml1Wm{t1yrq}@R~cKV z$8bo+yu=UiVnM?dDQ>BY9^DhR84yz`X0WSYyclgr9A+0?_}#c5nONN)HNgc#s0Bkp z0uy#bt;w9k)>7uNx7$#!(ygPmpp92^?84(yW(a>}wQA{7Jy;cLkWXVBx9~&Fw$IU4 zChhS;3a0{f;h>){d@H>_`SjC4N?oRph6FFdPO3^_G&MYt$;&#lp46%f^6u?#%}|LH z%9L?Ewcbl*zWJk}D9*{6%?INlb`QK7{IWA=v?hExLcLMgB^eH4e{&QVyYMCHk$z|| zkFiIVNh+xzHn5I5uP!<%mBIe7`IR8MM$q<)zMM=J;X5*|*J*cA{r`gzP`B#_FXg`k zvDgYhSjjE?3898fcln(f_L*;cNzSde75}%6IQ9)s7ysc`5|mKxH&FrgukH?D(0?WK zh~;bS%ZF8O`%=`Vpp^jGXNAqd5)s7u>O$Oe&!{U3#;U;Tqg7L@-^O2)SXXt25#?-+ zs8y#EvBE$zUBM|xf18|>W*E&Y7iTE*&+`K^NpYWv=d1hlG!U!XR>wfw-k{l?)x(qjBWyYH zuDKP-GWd21MR!VE{5vp5h+q7i;*xot2xK)Ifzz~2JBfdx_(Mnh+wwg8;5J|?%m#Id z{{TPhKKA@B8}AK&z1Q#_U#FKqRPJ(LJ8(po2hPYt{}^2R>A99EvG~hk1kWIuVz5Yv z!6(@YF|a1!rp{$cTuXUO#jJ73jJ-$v!uvT(A*&{_Ke3)h zFY9H3?Fbt=1Ou8T`|RRT=cd-lf<39I22YQ#AnR11$EBy5X3vk$%*Io(h$XnbLr*8< zR{IvC*K$sqMf#Za9Il8M8WC(Yvbx6u$$ntC;8ezKk{mq2h_E4wpV_vy!DiqA>-%57 zjaw7QMm_UG8lOz~jf&U6PHR4kmtcyj0^mORL?bJAseRp((@4@^_g=3uffd}IOE(i& z`~BY0D7zu-$lqDahDE`UU2!yCP)?#8&Js|fX&mB5)!4R-ur)4!s-dK*(cI`FZR!1` zJ9Jq2oEtmT3ojJhjvlG2HSZ9j(PYCW;I2l^=|MkN@2z`fa+%R-{B)cY9WnPcOXZS+ z>O_acvk3dpUpKv-uFmd57*M2rq2SfFI!tv~F>`E1JB`zK4T(;ys<2>{zo~~a*#hIV zNhEoqY;c?zBc64ACfYL1%f~=MR=IB)PzraSxMSLxL<)5YH|<3C)k%j?JFsCq6~jVK z_<#o%>t@&OZ0MiGUC3{`{C#^)tZhYkvHXaoOoFZS^R}}&c z-8ebs-<#Mq#`=|gI*#@abx7d2(t2b$#w|NoOj{&*LV{86Us$?vzSwq}Yd?~&DL#_v zCn6N!gjDE!wLM&eH_ksT4rKQ9Oe_hw71?3YHcaO_`WYe+N`Xvo_Nl_Z^0dy|-~L=+ z=7$W&Cy(VN4{bCK#5YPx{-ICC4P2yUT^mA^^d^>V;O=SbOLhPG-hQl8h1*y)+Vp}^ znT(EGCbV)%YT1_FLLjM5e^j@=cDfvy~B=_WzS-$5#g(v;N%&Z&e<>ZRj9vKg6-VC zlVuYtw;=8O86w;&;@&h(FxQQx2--Cc*~XIPkXsl2#@7^I&0B$R*sma7dx!jY?D-E> z)od7>{0Ad!Zv?e^kx2+LImflBNLQ;V3}DT0k~T-2$%HBE+Wl?q>93(31s}?Y~0H)M5iWVyHbg35Bnpby|@jP zv%dyyCbGADV0urZ)%mkh!t|Aw!+Ec8u%)e1Txo zX2(EOT`Ww7F?uN{I(*6`g9e03((K!Ur2r&V-*QBdqbaHvFT5)ZcYj4MWbSl>YeauU zR|fIjCMJAEI>Y6awP$D_p9IT4@4f(4wLYJG<bu`NTYNHb_CRRp)n87CsqK)){9nt>(SLJc#)7l}jl&AUBpc=pe?-YhwI zgWt+R7>9ubg5pvp^-l^g?DiS#FOD;eBqv~=hGLq@JZ*+nPz@bXuVr$%5T3vBo zqS6sJ!?vTRG8@;>;la{U5{A?tbfgP-1IYg}Mt@11>&_lP)hTEr zy#cVdlGCNf5%`v^E!$-%3y}vvJ$TL%_$sr#eiA%&q>EVKwA&?|14339ffBDore5VC znx3#spK^h*d8cM>s5=)1NAHFj?-gYCRzQS}JRUk=;s_hwI~z zF|DJ#AU;9d&Ud*`;%?crd35K0l zu##ND@;cOZIdY;1xoBuIE+$Bgd~eu{a=Jnc_@$`@LZTPs{-l^jvzolF?83)kP?(+m zDZJGo8q#PKURK0p(VqS^s7I(GHrRjA;RReKBMwuB$j6@tpA6h~E#Ig-m!rSesOv-Z zCLiywSW##yAd(u^ex|JbTn5n?e?S*}a9icc5tBv1#-jLHQZ)*Qx(q`CWpif&EU$}C z$it%*^zIQz+ET%HGUYY2drIm>)!Gpr)`2%IB*1dugqZAb~#)T3`kM zm$Q^d1YZePpDxAxifsqJv4 zmIqy_#SCd}wB+=ZXe;RxwL%*W)t(21Izrg_Nww7A;w4?mgJytJX_tbZTYL$(EIUu! z@Pa$9)KT=KDh)N(P5W<)CP=Ro)=(79UdL4A3?of2)+)UcXpwzl!)*>iRMy_@Ea$eB zh>GMwAD`Ct<8OP25}(AodUEw(h==NIpV)-{)P_sxo${)$9PKkz_#Z02vivW$-ZH3; zuVu7gAlF{{~=2vTgolaM;K66|+^B-SO@hTd;Ssgt#)hSEjTayVACak1;Q9 zh76aswf)_#TuoH`u6}Je8vY^9b+Y?RviqLf=~qtO2r>i`_fqUtbd4uRA=AGMdLIQd zDc02>0!6mc3{l~5qVI32UJ`#n)OjQLu@ftsgc_Tr!ofJorbv+^8K=%qFk@a~(pnXP z{7JcbX3tly&TJprw%t7gZYCO)aO@7&f4=Ik3DO`5McZJywF#JkFd;nqWE5LeF8+ZtGm9N z^h|gpnngW%-TTV?^<2rr`9=TXCKIqRTIv7B_=1A?g6DoN7<8WU8`h>7yM3*^tS=Zy zH0+dpWtJ#G6+e^6;Hrv~DjqYQDsfjF*iHT3_ZS~>QWj-uxdo^kqUSI6(!0CZ^55${ z9TQWlUoq=`FpLwvd4U|{!ZAU^Mo7a>L+wZ}eNx9qyn8r$_|0&2OmUrFR)Ff=b?hhF zm~f<7BZLZK{5Ll@5?l4cyy*X6;dg)5eVyqb+ykC&*A_lPyLnEHo8zSF;4>DKACJbs z-mk?Z2aEqy3$O|C0*L>06mha-v*MJM<(1x*D{O?(o}fsbil2scmBVy)mX#E+kiur< zG_m|D2;XOq6IAXjQE9S=X#XOhYiPyD)1VDkE<{K9xu zYl$CaAy#-aVR^OX_S$ARgZfsDap(OOMi-Q20@BhFeE!TDWhmb#jlt6=e)}2zJEfg~ znQN&0kXURo-GEoiRo&|2o0~^`cxK^gA5+ba&vZ>%B8%nmosOC2m`y_uM!!0F^5s#=?(ClHuFi^cKp!}~{qgF- zUtuj``f#Z9MoOyA@_)1dyU(DXw}fF4g>~s<#?}MKa*cpY7j+0H@OvX*SK@0VpgOUz zHxhY(RYr18IwUDx{F#A2X@Th14sl&ZUPmS2`e*WZ^6H_@*uPp9H`*Lt5lQ%z=Ec(Z zqDb}pf_&W;Wq8@){xIQrr>s!+b9QQrpVFbvhh_;FvLw=`&P-!cSBT>R-rmn% zk3+iSA2;^u)&pbr zA`-fw+rph(+=S(vgAjGMziNaa6$-|u$gF2(L^Uzm#BK>@mk*pOAYDq6+$9U^O^pQ8#HY9jAo5C@0xnp`nQJwP_6@E&W`n?{b zo0bUuldsG@!E_VDYpTL5X_mLYY9?Veb+|9e-v&&7>{F!!7-XB&-&1zXyHvJ<**dc@ zy4bqqLy>ixb1e`E?a}+0x4-r;vX~-X9)<(#1uaSmmG6}+G@2zwRNMD(ax!@hYop9j ze4pm25z^SO9VxEMMdg2m#=GxTGQsN27*wxCTclh?*!jH8TsU7|j?8Yl+}#2C>E$GB z5ud~`En4a0I@~{>lFBYz3~}gpSyKU-gwyQ8U6#`yDF`tP&0J22UB|9xLDXwu>T&6A#CE<9PPoIq?^MPj)A%NB z3`9{E8Eg`AfRCez(ksCpv)AY;FhEEfnE7H=5;b8xUA$J5*3?D6Wy)F%`_OGQTA9XBI`uy<3@4eE!XWd#!m~EJ^_iB z>P7@Y2_O>19ntdiaSxMaJPi~^ZJs#gz{<7!D&5Hwb)-xAQtW!F_nzKYN~Uc+<=y7N zlQ!h~A49k1iFyfmBW|ba5<*@QP}4niy*1+c1<`s%e|x+Ns3Zi5&4Oyr-N0BYBm9B8 z4a(yTR;wr|LJw0p%(}MY!Joitu{QB~0Qgljix(1So6P(eb(( zRQJ0z>ip^>l6O=f|MG?_Wd8YiIU~_}vEE(S=6~=0%{gz)rhS@kq-wpuyfcT6+0!0L zI;ij;yoApu>O;KOBmUHl5x5hrYkf0G9iEj~l?pc+>0Qa`#6 z7KyUA*^8?;(e4{mh~EbifjgBChmt5Tm9%P@M-y#8@82Wr8tpFp#!c}1tC66fIgCcZ z(zx0s#3d9PI{Ld4JbFq(%_~k4O|#`jV`AcY0o3ni)W)jeu9|Qu%m(V6yTRM4r|Qm` z6s8Bst0U)UKA(zM|O)qBF#FL#L3;x#a|V+B~H ze&>=O7l&Ibb9^}6V3O|#OYiQa0YiQI`sJJcPzw&5wf*Cmz{^mQU7Wj7hYGOnHQ(g{HBzAiBah}LKBrFw=j;$@%`AxN?1%s$O-uqr%03) zs=e{{T(kCVpQ4}Zgu-E)s2(QN!nfFD;PHd%65q|&G1`mmylx>;o!!oCyn`14T&#R? z267XNuL@%Ydvb+TAxHM7KXDZsK5;6On;vsLM$ZWvgZ)xO4Z>VHMX898Igd;&^#&am z>sY+1f_*xWc{hIJo(x1qs8xLL%%oBoMz3|5!`{i#{7Y%P=cqzrzd(~IlQ4F?>mm`Q zhUi{F%sL%TQjRvYoXhkssW=EYFPyvh1}=d|t60QjEDrz4Usnnv*-=r|o@gtj)KCD@ zoyIh;1L{I|BD<@xhj2bo*-L_AQ|ZM+O}S$RV*N6gQEe^kNSSi#G~aMeU1i=vI^-F3 zDw4DIOHWAa3B$E1xX(?8+G=*zcTFdjT5%d0Vk;Z}6pa?%3X!+Rhmn>{MxovH4Im&R zxvTgLHsbF}g{Mx@+GT~NM;jGJnql;_sp6JCK9zDjwMZ`1mGZiYHC2HDyOf|!tS?| zn}5+FPJ*$Sy1MOaVvoE&)fIi8HYbUbFrzh^B{0Bj8W)YhT4*3MDnDuhfBetn{?SZ; z)i=I&W$>TqU}iG3l9H$`&5C$rNAjc3t5)Q$|JVaA;OycJFX#0AL?}Quc*nm1_j?f z3Zmw)Aq4Rq1*GtoxjlrrBBp%&$e)w>`}!lGv*LeJjyfp+ z%|c#UO$IWM(4@sdfxh=;^pVeH#M4FkO6y(C;{A^OOoVv+stq%#SPuiZg{`{lc7LVm-&Voi5E*l}5PSLwYU;n3 z8u&b^r&Mc$CO^SxciRafrdFD)4dAz^>|G3l{dxJ@Kef-vZCFX*wknlG=~{31O68g@ zA0wYGxr9xuK3tCeqJG`0dU@QjYp(Yk;YeZ9pX@{x7J;%>D{lroi?bk(2j&K{p$Nb% zgfR95K;bH+^?}jwRZG9A)Ok~AYQUGxZc1%$$=gEQbPGo}?jrCcB5N>J$CtnGmgA+k zcce>9*&5#jsg7s2q}34T^`fHRV~CIQS#c`J#wPwNnA_V4`XKo@6<8Ag(ChD)Es%1) zD?zrFLz}N+l?8!&M~&mapCS)$VCW7#&OFzghVn$L3w_S~E>3&1ks}tbhHQW#AmYe_ z)uMMc)AQ-7==RMHr~BpI_x13^gKaNwG5vkX$Nu$*G#npD0Yh}a?dLm~eTs)lIFw_w z@D{24e#&AIt0T+)?AEmjM6{tLwRo9J6!Kq{<&hT{bxAa!H35BnfpH(2p2dEDPGlc% z@u#0X*Y1gW!DTzyhLwL_9h2|!7x91EdHX2=y&1%uL17W*?F+Gg(^$PnDYDR82Xsh$O_Pz zAC7mUcW*D3b@ENNz99{vEP2 z+V1(jJwM)MiQWCY(7EP1uYbFKN8tkL%7QiJ`R9KjWDnV+G{dB>Ym-@!VZ7wcqpry( z9{0Bvw8;gGBfcSCGUcj?h>4~3x<~kI96ZbU5B{P*(dO}HEFd9~l8l^=i$$Z?5=YLK zUnxvF-}SZwn4abunk^JCK7kH-fqSJBLLLz3^S2{BqRMc|%J_;@nq;{I7PGi~SjNTi z!^~XseP2XRu+r?4&Mfll-a$e7^A-h8)avM1Y^BL{Hml?Ezw+>sAnh#bY=yXM)wxV^ z*tpoI{`zs?C34R54N}+iHKp5Y5*$vQ(JPLDmk;%uhse|(WfghONPNwm5|+QQ1n7ny z!y>3~U3z6QVYp=}+@Qa||(-`Cw0(Fb!aVc$1?A;^dwBGf$wnQ70`xk757I zmM4^u`Ffx-G5u;wrFiY-OFfC!&^0Y;jg3$Rfp3~6sdjIJx8dFs94_0yoziGJ7ahA9EctT1_ab_KqO+|ji z#)C=ari?m_?yL`%BvBnx4qI9K_kua}X2mik(h;ngvpi^1(8O`G;eYvH2#6-Z-PFv^ zD$2!%TuOyO8X4ZL1PcNjj4c@@!cbv}A{*+WnxP$;y8OW5>Gi*zUX~B`>%7jsfkRzliJNFT2XzqM%44}p90#S zmsjYtC%=U%*1Em;;V|n@VNS`boU#-NY&RB|1r2ia+H7w~G(T@HthVV6jR zfeGg28ugw}vMCi>6A61Z+8rjuoQ98Z6q^k4l*TH174CztE@O7DTOfeOpIT|PYQ1Ld z3DP#o+y}Zr6zX8Aiaif#PzDxVspnTmrj2$5J&*n=b-@7vlfAae2<4(BAT0DvureN9 zK%tokv)i=V<)h9=s9;p@sbcn6?+~jCnch_VA#4ZnJK!|TCy+TDafzfm?hN_hTI0~i zT^z2S+F&ZfEKG9;S-t{-@St}I>vum0Nxkqnf|xGxLnOyir}7+ojXKEDvxcZFn4|=9 zD09X!W$?vkcxoFSvAlz3QwUHwKahrqyQE~yHLnqPU;v~M|eep71_8g0od7Llz5$fD}cMd7P z=zih;IVL!z@4JC|?|Wc*A!4u~P-bv{t=%YmvCew9_|klG>N1?=aqkl1b8l>-vVO4+ ziquJzW`sNVF$!ntyV<_V7^&yn_26(IxavgXe(l#dG4HPSqU&q;j=F+&d$4{c^it1x zz&{)FunOsKucWq2JGFUZQD?`wQsjdR}bC{pD4$89*9oloo@FdI1v*O->0CLLJ1Re^Zb*^WVQ{`~RS zvkx`a$Iw;Qt-(@$?4fR|v9+FaD9`@g0gZ_<94=s+IYH&VU^@1AUKF5^`%QgCx?dW1 zB1*0K3_VY*@Zb=8jhnzNfG1X(N8y84*BFjxSH)ZA15(N^$g70w50pt6pJCRRV9B@H zagP-f8A(7$coYN&o}FHnR{YcU-cDMtG7J+&j;WY_*JM+fiOWzSXAS>2Z*hjP$q8V_ zG&&pDsYt7do#2eAX}!SHrOh(hFs){<>kRFvvA*HQun1|e1B(l`UW|$iwr#K@&_MRx zaAG%mGe+Q+IQBjhKF8EKUBfRZ@{4W~BeKod1vSvI^UCYRs>plPnBBT-C~fv;)v_Ax zxtLW-n%E1wSx{PmzAYlWzCLd{%_`Y%!)><5*wQF&Oq8CmR-86nF1EZxINAD0n!pj# z5?S^a?o?!YCN-MI*4hz_d){J8#YVBYBniNHbBi%c1_&{oy`!Dd4Eek6JB!+sos^UvqenJ!Z#6z?Lk4{`w=!%c3UjA>V1thSaE~E z|JPfA(JG(Tx_{EVZ|JhX6IJHRhQ(}>@GU@Cbx&bjCT@YN&P^ic>FuT>*C*-@Gv;#t z^U^g^2X%F#(RDv-`~uQ(J$piq=+JQD)KQ_cwn}kKI@1r5wUM30hx*g)&W>f=S3h3) zU+S0}7A_Dc9z~&HxdNOIXMGAHW?nSNqf`XsI`HwJ4u&YQ5Qq}*p&`N5$A`0U5^DA_ zZ(Bj6X-?#@UK`mQo3nL!rP|TSHVxRrE53+>Z2r7~xMX1*dHwj+C z&|?c-rFb9py?^hlMpCaah-iKV4Ncm$G zrde;CcT(}x{ZX2c-#GwXTRvTPd)5E$l3A7EvqMvn5@TqROvHtBDUuy%(&_03Khe97 z;E~X`f9qq8cg-9Q&;7d`V;|1zT5f{XO-NP65d}c`sn-dK@2Gg!_x6Q&$LZ)cyR_@di*I3hAFI}xJ(f>O|g$};nlq&!4sz02sfkkx{S0e*iZX3 z754tD9ks@Gf<~iab96iK0fkt&_j>*$Y13@uaNfze!q(wjsHZ$q zqsC{#A|@-~Ei}gI^m7)>(7mt9Rtq+Cw2M9b1ha9f?vnrRfK}W#Xp&EV8XF{_P8l4@ z0zR(O^p)Qd_*=)jmX|j96;>q3F8a$Ae$&lY$J3!z&z*GMWsYqs);ne@3vP`200l#OlX?V zgUxuG4Ydsov8CcBMS^`w7&e6!m17o%$NLp~&WYjh+DIhO?NtXys5O|kq(1^4F75SE zk8738$|CGwW;cqd?X{8@ZC=bX&mITh8ij0Xyho+Q2jpc{(jH3leops%kk?Q|dhT`~ z64{ReVl(TKmDzBuD1-fAR_R={XO*=0@6j@8dm>~)u}q&!+i5u8yYM?iya>xdvU(_N zcUS3Vv)W*bsK12NM`l+TXRmKRp67`GhPZz8l_to|3H~&=jv^$+o z(Rt*5XZl}3u`?=EpivhF(^Iu+^B+T%zj)ldwmk9mku{ybFXPUTNqq842(kSYh5h)= zQX#t`U4>%=e0M^v+p19oc{HuScMCvPy}w>rGM5MLN)!X02;^QK>a0)MH533nK~g!? zxSZ329@;{c(L{>haS_iq`!l9%Y-f}=(z+6XZ*us)4{Af}y$1^coS<11 z^hVd%<2#;!x zPkfRmnji+}6%wjnzw?=i7ZRh&_Vd23_T4S|LbTw>@dzBye2sxd_!&ku$1gnCS9e)x zkBafxo&BaZb%baC3z+md>}smndSI8Gq^v=)MZ5 zaNDq>aKQDnnR*W>@}_{Y^f>j)#NB5F&kvUKJ96QDheg_Gxl?e}v_<;tiX)lkNy~F{ zH`Wy?EjZ@M_+q!zp)bST-Sv~d{ef5nnP$rW4H;=qVJTc2D--r9hUeV(k-jlDY8@`N z&s@{sx2C=N%*<5O89=!Cbm@ktpS4v?8-;F4tT{SZZn55F6aAh@Xf#Ing@ZO^jdO zRyvYF!+Fc2R0toDco?Ro<_}VH&xr%m{vnjE&GZa6Qs<$HD=$r8?zOW}IqFeTz{~jH&TSVhjARTe)~m2(uwGs0OE>^0;5Wo)K!X-{Xy0l7iDi zOH}Dn!U<#u8n}23yy4_>mZui{w1k}i9`8rS1X+?8k{E)@y9*a0!y#87Ep9u~?w9S( z@s0;%o*0qdz7M4GnN!w~7aU;bea+^Ntx(^(x!FmHW^Y(p zJTG70ObvRRUGOXjI8CqDb8>4vU!hYJfteCh=esNA4k}&4Fl!4ZzlVq9X&`6R8j4F} zW$(U1Pt8_T^(fP>F#T*#UI$8^M@RikCxA)9*=Fa#Jx%)bv*$iEG50i|@tt16p_(cO z)Q?iQd$&B+dN0NMryV6dNP>NRBGEsq_gE%Wd(%H*c?TDRGjU)HUQCjxzBUtEiNx|P z33EKVCQV)pYF&V0(#Ry4&dnf95|GzX+AkRsmP}&{!ZSgk*#^~^D2zYfV8D7-Wprg* zDNX(KmuM#kUP9PcjX?hxA1!S)EO%M*;K_LgsIT093(A5Uzc zju=~z^UTToK}nm#k9Ulg)aa@r`*rKs(0}}qRQ0O+bQMrE$$`HcuI7P9IXj+~3F*z8 z$}u}CM3%>Y6Ecv4vjR~kJGI=;Y_dVpCn}D-Gm%wMUB(H(y~Q{7XXq5_^5H2hauRO-6YHxv#JbOA^7K=$nyxh-pa+CAg0 zUFuzCFuqTMX~|Z*=6b+UlS#{2OOgE`TMqkyL`;srq*Kh*?$XsL=`4#!f%k-UukTw5 zaS84kD`W!|4QiDco^CqojF)n}zBxAQN7N2GX0dPI$zqH?#uqI?{Q4-(b8xuja~j{B zHjRAFNYFd}?cJ2oY0yYiHeWD&VhN|yvHZ}3lKEO;=^7AXN_zA9u|fY3V&X~pFJeOc zjLKM0c0e{^o@vdKF*0YlcsIfDShoVIGK`QdrXPG}gO9+a=p{pN)*DnI5RslI^suN< zk?k(a&K!yg`JxR_$Tn2cXBXvJ9!T4qG?jzMEGUWeFhiVB$z7m4=P4VCL+UTPnwQx2 z=n!-Cr^1Xk$*QQaD0wx8O=;WHoK6UB3H&3t-$I{eNzV&e}dR(s25+n8?BFDUVk{Cc-TV$~lx4b`mm;}|En`$ErIy@EG=ERBuH)n?!FAy{~&nc7A@?+qZj(28nWoT z{kyk92Km30{j@ftD+0+;N)5dNJ%dn>q!oFgUSP06GA|yr}&=;}nx56#dSdGf; zE9ieK!z$irjgu4<8rAh^g6-7Q0@IWpB}%`XfdhFOR~?o&<~OQRBQo z{wz5~ZUK@O3>*N;PN6r~2X#mioQbxS^6YOk2PY@SUXMmiAd$M$Td{us&OZR_N4YE? z&fe`GuYjpk<20&l-+Cp24S9~mhpamI1+^qoS(uOHrbPSkIx-E4aTtV``0|2#8GCAZ zponmFq!`H50!-Wg4J@3sCvgN}Ex+E!HW$n>BG~PoA{Z1US{iyvK>rDi%{7wW#DeWX zKs!!ay=17Y&}!elJxKL=%fw5OvhhX@QSV{pndJo%NX2BU9e6t10`Qw*>t}u|ANH}_ z|G}T|IjOT*ZioEMpp41po7_Bajnp*ozq{?Yo;hJ;D+IvkwPi=FGk&X7Jyt8jg?Yuv z6x6P_%4DEe@!!P?-e;i(G26{&;S4%XzS!YWUx_90Q$fWHUW+398M@0g=8er^(}o7Q zi>3|ehp9CNTo1i?5*=Xw^#8)glg(kD7k-zyspf=U9?IZY;jKT)x$G+7_^Mc8Ld)Nr zES850*p$yKagy=KGJFG{_*{5XUNsUGNvGueh;raU0@E<|{;mWL@$9AP%h&#ks1e}| z`sn7DsS_6BU6#qAoWYE?3R$4cS{EQA!}a`uBi}7`43A&nTR39cndMWS>6QgCSP~Z= zmgpglS0bcMeuXiSi6@b%F_GTgxRn^Il1I4(m7^^lzj?`*8Etsm|tJh+V)N2r#e=5!bh#f@e&Pz5Q+I}!~Cn&97VELnH-^m!*T!IepoW zI;zZ~rs`R?Ud8Z)H^(=tyfS!SlbrpLf8DrEe>d+jGg@KYal zA%isb4JJYQ$ec5yoPhMfh431)K4|;yGjy@wxD}8bkJAY@YMoyndy3-b$eF)qTBNLI zo#!i$hQj*P^^rF|qz6%Jn#aJ)xicX*htr<_t`&=&75#P-QT1Sf^Dkc-`QVqj5Git@ zU+S|P7K69V_;0sN1Z!W(+m)fK*ab((gvGzJ%QxWlNAT7khBs?*-=W=DR#Dr1h+7ZP z8Ift2R2EuU&RhH_#%Oblj*|MynJ6{16X-!KeC6@tt>F0R;x03JrsQ_A_}L6nf^|>-ml0{Q93D zrpvXHnq0pHRrOAr5{REob=h6jgYPOU$5HU|XQsr}%FiO}1K}IJe_vLYX59lPB4e&b zZB!3a;TcU@^$-(zoC|#hCSeV0tmawD2xVX_*f6;daudb(d1!`6+P#goF!xrxJAphljc>R`g44vS?(Ez zv#jmhgV9}`w)4?5}r7hp!F>A<)-g1WFqd(g)iOt z$|p*2(U$1MslSkj{Wqr1y`glPZ+l;ilgH>HV@0HLXkTm z_Vv@)(ATe4)l%?>^bBD5e2t7abc$WcL(8XDVO^Q-ir6HL9_IGQNeMa*&jnq^B#LF) z2vg;iu{#+F_`Tn^_4d7rb2WaW%1x~N6AAaRVa0aIkK`ZzHTU@S z8%6e`d466^h%x8abY&Rff!elIe;Fse(SYt0K8@WgO)OA`ZZ#IDE%9J>N{+u5y!5AC zooZYt{TNzSJF6~2K68DjHF7=TvwjCy_|BhGQ)yWMkW$(wEl{*D@^8xNcg~ zhIgyf-9x`2Gn4Qak=!bn?OdskDWxu>o|`I~sr}x+j7Cw0DKINEr*x1-_T44?Qr`SH zN!@LxlyzCcGK0N-CRdHUq*CFmy1YH}$Yno~Bf?i>KA$heGt{BE?3_BrriN2p%Rc9K zu3z!6bM0oz4q6{qx)oV^vfX;{(MBbhCZS1AWcFJxp7e?}h2DJb>bk(QHV1zhG-U=Y z`utZ~Q?)`0wSn(*C19bvl632z6|)$p3L%eFIzyrm$ga`l1wY!eH%bsBlp;1Z1VWn& z3yCoe?B4O*f|&SkK`heo_m<^j%Hm|kf2!;{ap)^CsP}LohD8G@RAc#)(iHvm$^*01 zf1f2K%`U1EJ=`U4Ual6Ws@)HpI@D}=@m-|=A1~RqjC=Y6LZMMVk$e$C^2Gb!3q6}W zU%Z~a#qzy}^d{Q?h&qF>!#?i$W<#d<5;k?w<9lJbRgDbkJa8+B^}Rc(EyEeS(hN7n z=K?tA7xFD6sW&2J@aDIQ;iam9=u0H&V>T!nXR}Y)^mnwg*J6kYDG>)sJw%eaCD3#u|E=p;iUU{f0i9Gwcq4C-A z9|Psju;|JV?5gd4e5oY>s>al+I02t1otPDTtS{him ztjeS`SJPt8%9JF24*oe=C=Gkvaa)afXp0TLpu#lFlv#pTwqMWx+ELLRv#zRC=iRHE z;$A^jH5jg?%y3HP#~Z#~4UQ(i;bVKiXvfvx{V3t3u5TaTkP^feD^$mLHGk)$YatwX zV^KA*%d;D`UMZ^L_7a4k=1c~nDBtCnf-mudqGj_%y1eeG0C(ze41mb zOKozPSDAJa<{kU@s8o*7)++K#{excN_(omu3~DtV5;BRA#W$2 zQ6}1Zau(|4z3d1ALT}hrH^M&Z{lwD)V03Rws;TlFSu^C#lD2`L83+2DpbvJj}|qGlx|CAz>xbX;8B}Q!X-R$tGpAC$-8Q;LQ}=hK6dH>(2Fx{wTgtc_)%{2K zl7hopu8b57joE}N!nv8C@w8C{C2{2IZP5-vd)}7=J)^fiVsTCz>I#VK7Z zdvj$jIAAR-kcDvoMNH+|w?^U-dwcDgZ`q-A)g=@55=;#f@|cRoXfqcyMGoUKh71ws z1~83<$0_gXxEveI=%A2n^NT>Gq{vWls=`WZ?5nscdkVe3Dj=gnZU$8Cj|8umx(l-X zwLn%ExWL!-&-DN*lljsSGSU9LwNXd8nL@)g3W9~&G1IwC@Ig);RJT%bdYXq-X`ObC zLXE*gJ>^2I5=HYhaEp|0`$e$p@2lXX<~t>baZGD}MQDDFl(P(T;pA&!nR-|497j5R zoU@IzAjh&i4I0wGDA$&m8HL7#fZqfbmC$5Bw(k zz;6_o%H|&=PN(RgkHg2ShH64eD>-XZ*(gvgQR_{pq>EU^w3KD2{3I1qj$}lstD-kJ zVO^A@AP|lB6j&7Ks{HvNf(YQIgWew+>z5a%*>6AK47auah5qiUMsuia?|>!SbTH!~ z*_putD;c~rxp=*dYdIhA(n|?SzAH6T5CGP;cgYeNc3qcH#sojol}={-BOXakqA4>~ zCS5dwr?RPl6mchMG`ubI6R&o*ek4tuy73_0fnbdy&`{AsmA17Co3$Hm`8@1o62Fn zqC#WfstZ=%R>$?1Q-LM@^;m8tmC>W4ubXYIvkE;A?u4NzQ+7}8J?6LsJ*Svf9s|uf zZ|Iv#`%?2R1uF`1=q3t!3evV#O5Q`%_CKMet}VXm*sKbsX46tGWd+)lCh3p(5Jzsp zVt0xULfFDiyv~hr909DDRzr>tGJe&G`y1Z}&-3G<`#pG4d;Qd|Q~u<3K4}>{$?|2p zeoSt+cAGK_zPw^l-uzSMkpF1clclzxKkb!Sh^(Mu5jJI6rdF?jBudU`U<#sZQ`Sy2 zZV9FWoV10U8yAd%Z5lBC%r13l#YqpnrF~!lxIrzTJ6YqhZ6%`PlMqd;?;`|Za)`=V zEqTP=D<-V6JneMhVuZ%1Qv`lKswmqVY zsm<>c%F&ySm-t_lCk9a#E|0flh^+a?-cvZ`@$OUVrmI}34s;esd-kt#XZRsp7tLp6 z@#EXP+&+>j+fyhSrFW=JuC@Lfzzt91q3C)4-d2@2sYc8h)1WgBlQuL6cSWXq(3xCJ zo$DkA5wnz#J-Qrq*s&i>Rj%$S2h1{B@(2e-6>6?adqxErCMDI0U=lKB1Z^$FT?J8h z@~0ctYf%u3xeRfQs&tcU4-y(q-DVltuK1;md7&Oaxjx}r8;pNjRv~u(9qyt37{JBR z9p1riHYMdg_06#C9KHTm5Vol8u!KytM8@OK740TwKG4iEVGSkh>0b$*SjN{So8 zZqa<|9&yem-gUkw>HsqFid*FThPT~%*15O#RK66}y^ztG>i+QscK4ap<9Aj}e;ihC zC51Kg#%eP%kXtIICCe>(6~ov&hAh9&lQaFILDJ9+i+g{9$09wLFWW{9gg#qFH@8+A zX4*e|Jxm$vv=@N}N}b=Fob)w|a4W1J7+?D)VNtwZt}*Xi5dq~<&Q`~Lda6v6ds2*5 zM3%DZ&do7RzxUNjZ`2ECu(m9d5;!su@vWO@=kZLN`KOpgWO9>wHAh~w-fxP$U@pb> znL^f|YP+2j;@)72H9cf8A`g;hwd{wZo2}hUX)W%7h{hmmghCf>1bY(RM5!^H@OC#U z7mqkcxu7O{-U58Ez4Ab{MA)wLt=vI)VVO*j%RoLX%oT&ArtcPQ{9Ku zF-jDlU1QRNrnF}zTPq_j`(X63^5?M2X}XO2<{Nv z-QC?1TpM={?k+(a@1Px=#+?97u+TWY{Ac#vnK}FJe(6)EKUUqkzgv&oC%A~@=vS;~ zPym!iPD{8H+IrwOB4$XIR46e01Ke+RA4u$C0ZdKQPf|~=Q;PnHM^&Yrxe#w3u2uD` zM-!u^Q5zQ|jC86lsrpT^KFn-PjpDSF%2+}^$)hJ(JLj{i$D7ki67!=W%U*6@G)Q`4 zP8^oPeQdV7b?EatQ2Fb~{Gp63p?F1Nh(W>^u{cwd6cjXz)BRSb0^d};YyQ*)Cn}!^ zkIuL1BCE|rn?ZhJh=+x?YLE6&qM|z1-0>Qk@jW7*;LqsbNK#Ts5}?9L%wB7-Vg`$T zcWjv~pV2f(Q%F#xGEbYEAA5mH@r;E3U4$1o7ITS*S-7MY=+nYusI;wI*2G!C{o&g% z)o1QZ%OSAM=(M;+?^!DSVxhfa(|W%Ic?8^48v zf)ApT{Ao>&h5&@N)NDC%1T73rC~p;~Ai-<*!vyMSthq}vChGA>QR*oD5Q!%pPi-@z zDdpd$0=AEM@G7&!UjO!9NxOU z)$_DdzFV@Wb5V1}9<>ZicZG4H{yo<404l*}L94fj2H@t#1ChPx*xjGX`T|Nzu6)0X zRB+InI`=%6Snhs;PK0D4UjxD&KmQ=1j!Li$owOIia~F|Y1YUhSg}X|TQEt;zyFFly zXO_xcIc0k%6vP8s9biPegFNn1)DCj7s}e!S6j*CC4BqLdCDF*;sqAOe+Yy*=jT)3_ z>di;<^u^2xgl&z@=C9N)D{&_HgsjG51w7CuGf=*n9wbJ`7=(m%t5pxnAd|OAX2*)O z_xC{P?~H>A3Dblx1jT}U2;KQh=oH?!2^?gSoAO2A*PG8n_nhp7hlH=iDAK0P92u!{ zF@S%iFl^IFpJwg}R=V%7ElrBt!*9e(PsCr!b0YEuGHk*k3?nf?l2rzg2xSEhGKGpJ zjkz3J$q5g#sW$agLWyBd^3|Z^mA7*8JB`?&tOa;~tiWII1kBed3WLTx-3ji=`+Wao zl|>b)R?zpWg!a7YWaEa#tI2hEuMsZC)~T>D>I01GAK+Vn+9lDvDHML46j zmrjRfqZVilMxt@*oI9Q{I3b-02RL4;QTL>c=@sx4Ei2q8_jP z=Pm_0AF1A3qBoaGvHZZ{ciweTO@T)7V$S!!+aQd2Lcff$JePYZ+I_6;M8lB#-yx9y zqnlqCfBv&4CAxd}V0O%$E=LkL(#7xo&5#+JU*;)zIq{g?yo;f7#6b&J%_B4!;nYc+ z+ok_IxL=_1V}B$=#*?BjyJEe83>gX~#gQ%kN9;uu#cFGt8iD!6`Z{ThV5Yb|gYvc( z>FUhWq%=pXeMg)JuDC7zYjXwrgUoajoraIkRl=EA$bq~$E6el?u z5dk)J%C9#Z-@PItTvFoj^@Xo-U>yU2*FHE(<>~d#G^BKNOs2JAdUUcYvsJhr3!j); z{P&=x&_8$ii=N9VpGIRUCu0tns(w%YazGP-Uw5u>o>K|q8(aLVJ-Z``Hf(=`ygYD; zv0ciq*6fQ=cY2qQKnW`mItciHCyT^w0A9CUY6)%7J1;aJncqXwQxCIA&>Gq~E=u}K z2~fLu_{EQ#6A%c~aRF~tKr8pEaRtXI3#sDvCVPb~OIwBkYZ4^^wC2N&F?-3x8j#-0 z>?m!M=_QyNq+^@=P*q=kDz3(WHjU9*ki?_pt^i4Lz&gx3=-(Kbj}M8(^js3sslo&| z!b2CrTboob0q~?0T}$T;PmiG*_e-SUdzt3b5u`unQ}Q^cm86#NA?~`ImQ3Y!ZJpod znlT54zoDOarqvh>e!LX5m5!j2OY~?kHs1K9U}5x6Hvn3ljNMXG*i}KUdUti{tD(OW zY6D`u+emb~PAlh-F(*LP@*wx%X=rv(emJRdT}~$+>))#Ddig9I9x56Q|G!m4Q%^hR zvzJebvSPd`6JEI!$|U1o`G!@T{2o8A39DYJp2~U+QFS4>I2`j;H}q$gw?o5qCjStyL{%bj%Gc4UeSRAV3{T!Bnwy?7!lq6whOVz& zs$O>N{ExeTGP>|n-_?w4KKaZwQ;{pf^EX}zDmR!lgM4qxLCxqUTt>VN_0?79DY7FQALUr3cAdV?jThurg^>eb;%gH zy0X=NH59Bijk^s2Jf9f7IXYvL$%;tOO79&m_m)Wm%iz0XhURoty;h`bzVAkgsM_$W*9mLB=Ec%9 z|Hs9j(@6oBlV&4);v=GRijEfTIBD(oZWZeKN|Pf$ypkf@lgGQ~_kLf{j$R(P(gI{S z*MntnLFY9={**=5d5pCKqhkx`TWudEFJTZ zx#z84zxGQ$-t;%)Y--%PLY9wA^R?14_CweJ_~CzFa!!sGMuzLEqE9nQ3?9TNqZ5_e zJ$|l)-eq?$7m4_GEx`9rwK7|iPds`{#Q5SZ*je$HHS!SR1bjYuqXr+B_rF+9KP3L~ zK~wFwN-WSgITI=uSZZxKCu$E>XQPKcV9(aX>@lIz$EPv7T|xc5(W3!{T4nuJsc1do z9gMIQUu<{8l*}_*2YG7;&QFQBPH7Ud$vvrfsBzj~e-AmNrV1q3tx}|pGx2QDm9HsM zlBn6n(qE5Q>hGFo`Te3J^^G0n3H!k@_=VPh=Su;;ZE{A5;x2}V!rSA|2j3Y6V^oAX;xl~yN8^>2{eo%DUT_1-ltIGMP!G6qNADcu$BlkVMw8V1~4p6va1sPzA6 z$)OGZ)ru+3{%0$0{!c%4uaJNMCI)&iK^Ba;nFs-tAE!j*07dHIThZd;-_shY?7M#z z|3t0p8I_gcIqjk=AK>!z7U&Xs#iVLJ{!}aMD#IO8?MD|jjkdv4AQU|GW!ZZg#diPQ z;f815iO?hPX1boz`T+9N*>~&PA>zI_{-t%p(tdCEumvbv=FPmzc7{T){C#*S01Z6h4Zd!Lti;mZVRZF=Fs8Vrc>~UH3 ztY)j~p$?}uwjjI;bYt0yI)Kqn1m=|iQEm8mgWzw-CqqL3=T%p198YHcg4NtE2JrJV zy`uY5qy`CMJ;!rP=N!9l50(7hI8WDCG~x>A(LnEHU4FZ|KH6i%PwuxPtch;x6!{AZ zMCjFD0TzE$VJw2vhf9`Ip^kX7dPJJwv^c9-R030ttMEPPf4DY4qK;nw>^@R@z4_)G zyNTjCWM*u&x^x+L7Q((qLvVBF4XS1I^nmMV#%ikuhB-ZA)5)~j%7#Z_A~(7@;%O`~ zTgEshRFE@d4Odt|srnY=qvc*gPjQ=D?ZYT>^AgW8&zdST01)*H-o$wqYsU& z&Y@sJQLiu|mm(ahvQtIQUiV!DSR1+dblA;E8?M-Rp--6fG>we__~Man77`FE%BuUB z-$@AJv}RWT$L?@1HhN{=J)f79IKu)`SvN#|^1v{98H}^C6_6RsYMc;g4?|lNCuds} z$5qj2_)y)%q^NUxX`_<}Pw(?Id#uy^`C>!mo)%$^qw-mAFd%B{mT&DSbTAVJ&gN@= z-G)p!ia-@Wt1gEN&4_5ny|lwqqC2t^+3l4M9!HF438OK1TR+&<+1=kpCB`{e3 zH2Vr8P{&as zEDkz^QLoH8%<1G#XhpV!FTdke4)Vf6Tr)e(PwRbnipZ-0G60z8B_v&Py>z&I^W<|& zI_h=rN6Hn1KSTlf0CJ(+>FJEk2+y&{g83E-c#riXmpP4?pj1d6<13{3Y)C?n&l*U; z%?z9Nwo_%z)g1i+R)rF60PZ()S|z6i6cL+{Jvvp6+my5!Dzh>eHwKln1XQj;wl zs~xR*bO!fC;e?Q+?;+I$kkw8zWR+x08lh>WIFm`tDt< zWU}7->3R@b9PQ7lM5U4Dr=O{HR_>RW8Y@$awU$%(j(&H|^a7#!sGa8509Ehx2_{3Q zj50h#?3C+YeAKg;_P&_iYrjaV4RijLav6mFwLJk_pgtz`G}jxQ zx6QZNKfDVLs*)oyeJ!6?zR)sB02Z?c^xxI{IMr@sziq^r9S~!w^l_oSR|6i`t@9CB z6adrbl2L~%Zn~;7wkQ$;X%Eu%R2+4_pY@wz)1 zLo%2-oc7Yr(D;ZX5$*@v{7ThOI;TB3Fe;KVLURsfGC zA1N)Kzsc(#KCEqEM#t}9+<%1Hf0r@OYMqX}H=NfJqbdk8JUMf@zilg@tq^p~MP>8N zYG}Jl4%8AnsP8lL*9Hz{?iF@IfNzMlR1o zIpe}p-Pb8cF0EuuLeyY^a|(|b1W545(*>77GZWq3jmHK2v%<4q)9_hFmZI0(EMmn;YVB-~=Vk4Y zr{R8omfbRs&$XGEuoqZaonT1`ki#e%s<#mjk`bSEiqot5^~7nIY_zvge=)UvTxR($ zR<>4Q)S+qe<1Ck#XpbTI^@edYii$>gF)3=r$WbSdvR<6LOM`5{Y9cEW=AEhy-!ev5<>C}8^RqgqSltoa%#R;M3@%VZftS4HyZE=w``>O>3{ZJTu-^vBI%>jtVL~Ut}_s?=a=)NFz>XCmcEU z2sHu{%!VpSK0v>SGJ~KxOh>G(uY}wAVXC54?3iZRE}cL83oqxknIk_Uf$!!1v)au9 zFrD)mv=kcZSp6bN5}zt+Mz=6Rcpt(b-LSL79QN-entX3%oIAho4Z*%iuRXhc>!(VJ zrI!!ZIZvJ-iLZWuDkoAoZ5ehVy9X!3U{D`ySzgs2gmNlY*eFL1yw!3flrzRrH^jbGp7))D z7@j9a{KmUB?#+2AjY%4}?yW0zT9YMY_*8nhJ@&$lBzzrzPqh9L&SgD6bgD6bXgi3* z^HFE~anOfcX@CW?xOBFn+4^asuWi^xLO)Zu^OV6H*YvV6Ka902;G9)LsiB%kl^7|4 zH)l{+8TpNk5#XqlGvR_T?V;c5B1+I$yt<3$sAXC$|DwzrYlKkRQRvrTa8q6(h{=3S zP10KUm;j%tK9ampOh}`Ss|&ezzZ%1k7Z#^D)WHGsVS;(ECYE+4@h*;9BDmNT_~RRn zIMP1U`SI%1kg;m3vFOWi6jEaHBh&Cma)!ID?{u^cZxN$+(y@e@Yck&N=>Uo|DEIN@ z1>0A2sH5hkxtQzs+kRL#!UGT34-l(-^EbW!{r4)5|2a?h|JOWCP~G_dFojqkF1I{a zD%vE7Y(_gnW?2cH4~lnht4#q_qBSObWjw9-c2Fp@@0x9LOMiIi6`Fe5@J1!)M&>qE z>*xk*;HqW+EKRm=>II)p^RH zAwM)O{yO(fA$nqP+O7!Mz|Bdy3k{|dekkP#1+f8N7F}cQQ`Kv*V*fnt3@*QpjZ}e$ zLmipSQ=eZD*imyvYB5x7Mo$d|77*5i0IfA=mRAs}6fIh0`d7>$xsA6(y#Qc;UYoyO zP;dJuKQwa&Ht*#6oWDV8+=jL_v+1)Ubc-^ylkCxoJYe250!D;qvx>g@8t zC3o6~&aony2C-LqS|CbpRQThU|AI>){rk9vYv3m5O-6%4|D+vYP?|1z z;lvQu8blDKqoc@1`%&)cf?k}$Y%2_WgTtZn&x)5_j3e&d8NHxDPkbLMLAEGVs*qfy z01IQ<{J5%sM9(d!9tF@;<4g)M0yG`dn>`{CBDeGlLj?Y|QAIA#oYSOuX;rjsvC-Q| zbB4>vq885d`AH8bkd3sc5r?2*-TXqpR*Ptc1y4CrzXOb&5rJ_+S^V@T=tiT{YsCA-( zz{Bf!Lc2UxsF?0ufe4BeSrE3=FYU_To@%{gHLLZzX<*f=Kv-;RRsh|SeAH9>$mPN- zY4vHB!JGgwwJG3l^Qe#sA-}}vn<}xR1$~7hK zYy)4i`+Hdih)Ik4^6WUj=dJO;Woy>Y$4WRL!+Bt0fje+hlFb;aojYI5H&?GXVPO*055kDco`+s>3bAo)~z=mpL=C}I^wWO=`3*b0=NK( zKLk}2jq_^lhh~oKOH1w$+`Dwz z4&}M-^B-~o-mrWjK!cL5-$e?2ikLzAQm%108%{?J5%R&Qf>mzQG~K5hn-kRVlxH}C z?AXtL4q@Szx&C|0;`Ejq5<_J2-Cxy4YGiRneJf&r8p=)8t4DYYv108i)W-P(py%6YVD#C$DQhhc zU=F+_i@(e}mg@6vgy(^4lXS`dza=7upfBX)Cs?3mILAM{32K1o?g}*kei&;>nV{J< zhV7qUO<#burmSwFZTVAMCRiy;hWK5C~Cy;oB(vFgw{>#XU|DL=2zpIDljw=X<7r*VsIN-cSh}s5QIz z)5Bjb8e>+?v28x6Z&V_g6k$<7Wfcqw>H5WzZdZL<+>a&r*fu~V)wP7TQ%)w2HzY7` zBde`mHd!(479sQ;N{teMm#s$kFHUs?`P(;Gnt1M?mC9qw(aN`#M?LgX0yYaE;~ju_ zmcmC)cR70KNV@1nQr@{`VBQLc@4Vpo+k#fGr=JPGobBLFwVTM~J}Ge5;O4tiTf><*D|HK%_pOHMOGgv4~Gx%e8Y^M#JGeuiDS08&~9%^=A0eziN5A#jj47` zg_ai#vzog3sqzO_48Q2RNiP*(tgWgx`5dr?T>c7w&t)#@^C6_t=s zU6D~%+5@kLZQUbgd|=gC9F9`9)XgK2)cRNH-Ph*+)7jTtF?ttrMz6-oE{mXj;#}8R-<9< z6z?2p!0or)u+`#ws1H-liv6rBz)_i}cI0zCp3h8T@W!&RNa8h4@RD4S{{HgYBV^Sl zGx%%+>e%Zhi8$#s7nzDZP>3;CotTVV5gkQu@qms zy3w*#(Oep2u|NdfjsL1$de{a-QT>fJDrrwmpy@ybcb41JJzMr$>y?T9Vw2J4AQNr! zIb$loo(*qiRY*aJ21;cP|OL4odzRX%jYzKy&rZ{Y;36CBA3Av&l|AA)b0JNT@q%5$A;;e!2INrBj@< zd66yi!Px&yU|K*YbVSvlH7~YDC88BAVsWIxVfQtGG>6)HUx84@B1*t&0aI4?-xG$@W^guG z)p*y4UlDD|yZoOY>{DCg6Q(FVU!FxL`4)+$ukJKJt`#ik8JwTtK@ zxmpPyZvL3EwS-L4;uyoq6Lbxq`U%yJs}gAaK_&3>5Jt$8=M`^>-3&*tj*>$2$0-N> zutP+~)t9kZbOrPhe|ZkGsr^&%tPz8zdAH_h)gQ&MSL3F&Mq0h6)+ukbPJpFlwkB?;MglqTMpX#Z8!N5rlvZNnP3h}}q2lgbPO~RV&N5%M9w z(#|Gd*E)@p@v*GUjNt{|`HhM#)k7v?noE(gDa(meXpQi}Mxf|gShL+FkxbO%+1iDE zCl+7$2#%1Ns8HAqGwgCOt{@@E8w(W;;qpi{*j2m_eK2qUA=|t4;En z6JrA|;7+f+QcYo(fs_1fh&-etd0=8YM<@#qVL3f?wS1faue9uI^05gGfF@zhW-z9g zz5eJb-1b%+8oNiN7>h;h(e}w#y_{aB2fI>M19ct!A{WqDb_EqvsqAjUlD*mGfNV3= zmVnxXJetI}+-dkznz9f}L0{p`In?I3rF~)y;(9ymx*_|jmp(TR9W~iBZH`W91oZrP z5Wip3c$VoSU66`s!!(aPq7f5SFalrfa$9Q8+$@y`LVns|kh52UkfzQ z-+DyA`o^gGUfzZ$BoSIW1%CIH^?QzLC+|IDD>t%MN$JKHdMBMQ(O_CFNwmum%B_5IWu;^kHCmP`xiG zt^tCtw)CD&bH02Ld|M`OThoLxJ)x+N+ndV&^m`Au7w@?dTm!-2_qI67(;jQq#uT<5 zM|P1urF>7@y}kV^EhBe-g5n*RQQnG{3(UmGc)ly8Q`MY|MEa}OA$!1dT7r}rWZo{S z&P|ZR8_G#PxER zjmP@FiKiJ*2=r?=(0u@w87EO%RV}E_P*b1k&UU*I8gG=}P70mHZ)rd06jtjTG5bt+ zD{o@wUGy##<9@$5tt_-WA1nI>I6SSNTEU%A)HZBQ5NP`T3`Lkq=SZs&vwB}~215Y3tDo{u^H2lB~nUPt&d^F|}b zW*0(d0r^z9n#oxGc zf6*x8B`>prhxj+tWtiOUL=BM?KIv=QgyF z@4wZgJlS*p21UcLLH%V!Ky^z4>}8nl)8jrcc1QwECwNj}qS)9K;o;K_zck7JR1@kv z-4-q2E$^E6i^dPjj5CpZ+fC$U~e5YXhnx>kE7h9M9tILM>Pf3^%(vM!fkBED#E?qT>C^?H{S3~nwyvr2U;o4=pX(tLi4@hm!pV3a6 zu+1oU|BZ^-@jMZt!{t6Yp~uTC|Lf|e?Tv)BFG*$DPgEQ}VBdx}*jVIZ{~c;b5XWM8 z&0-+6PI?vTXjZ24#|uM*AeZcxgrc=pu?K6*b(hMjwpaBphdO2R9%D<2&8A;KhQ`0Y z*`c{RYDv0~zKIh(z%6?jvVuX4YtqSbx?Ht+S-L)RSG(w0Lug;#W9Tyl#xqP8^=|x& zg=#`E*;l5WZze58f7eXTeSmw-=g+~*tXK8R7c-$FCH!BhYm5$GKns-cPvJa;Ix(&ELM|?Z;(P`MoAApbPOou&?_T4#wqnTGDT? z+b`u%0`GtYMQqGD{q(Vu>l-*}2oQV^zbFjVSm`;BMXVyzDwRJ=#-eeM`|18P{w)2M zJ?vJ%+*oc>@z5nrV4rcJH-#SrTr?J1 z-2Gwf6$%joe!h|(FOZazAj`^`)TD z&*ba!ZL}zZ`KuGGt&Iz6kG9T36PyFlgBfdfUS*+AFZU_GVEe%as$5~#p`w33zlLT) zzD)EbaCka?P2>C{r%K@`$Ry6wDTT&T?0#3GU~McdbwrlKVQ1*c>-<)2>Zap*mwMa zI5@ikJ3K#*5Z%wIrMk{9j|i_;uGb&`cs^aF*lH6$ty~B7^$l!;%KCmK7t(=&neJmVMx8N~^2Br2c`vBr&j*8v4-|IoZTJpnuL_k{Bo%RJiv7Dzf}V9lE`V z7z)~s4gu`}gA1d{1LMr8K_36^1ZPZ{s{3s^*^UN5ru&=Gg%X~53I=oPeCGQPw_b0! z+`PTbYO@6JPv_2a-8@^{pDmZN60FCAl#k-xCzffje^O`uv|3C^YbJAPh^43lDrZ^! zcBZSi{0)!ZYz#TqJ%f^kusxNQL<1g=oqm&^%T-1eESs%p+~P;ggdo)g3;e3rSw(rw8Jm;OXBKAQVM}KiZkq-{YtbNW__eM=6WC6|g zurPp%GjEn?mNe7mIs$LmAK7@TOBXnPI1sfd-eh%uV1f9c7GVv!zDHhbPL*6|IKazV zIu{HPt^??kR@O+I!+^(^Ct|>$XPS5VFw^d+7Sk>t2Y3lCU&)C8^{dAXhEf!S&fHXS=R5q}=iL=HZ6r+z@!XYGN2F zSYeXzORS-OYtaCy0{-(siS+fA5ku_b{X3erPt^loI>X}HY?hCW+sD6YiZV1n9T4YUjrd5`jtHp9l4;Rwi=_2kFupIi&2 z*0qmyRq*Vc8Qbq_)l* zaq)1~l^&A3P|ISW&T;C(t7+2#tB#KRBIdXMlPV0SgoEwh!b1JdJ(gwqkfBFDJ3>*A zPaHk`f{L-{6ZclJ)+}dh4KBO79gRAJ^~ZSPC@TVC$jcMYjjuBCMX1N$b`d!_{${I% z;TTB~hh=jD>k=Itg;={}0~<$erCX7j|Y*% z|90$=|HU%9+m;2USB${9+4~ar7^!pwddegSr z4?Q>WMg?>2&sIW0|B=Y`{~VzQb>BTLye_X1kZZJ1YGDH#{Tylxg=61}Kd@kkz(iy_ z%7il4ciPr*0_r*%(cKmqQJ`Y&m%i)#E*%}AQx6)U_SJO^$h=`CFRc-{*pe&?zvQvF zUY&vw8Z_~eU$KlG^ciuqS(KDOG)<;(>O`Z$BCBE?BMUDuMVO1CxP?n&?3ox1JX(><}83N z;GVtfX5lfHNU^W1qI~r@Vj8z!3pdmt#<%Muzh}Sc)|PIg>2ZMl2@y=}wi4&$;f8EY zcex98!*3d@Qcu3%dbk2-UV07?4FF#fpPnOKvW}d0cFr9!mWb)9%RzfBC&-MKV!LQ} zpAQ>$idxW3#X|*M@1GRoSLb-b0wj0NWs?l)s;j(dsSVO^x?$F%%`Gok?0Y3p)|9%` zW41U;>Sd#r;+aqWdh>D|aL(Hv;$U!l#AQ3qpo>>vX!lJS-JJlVYI{8G z>Cw5@tm0WG-V_VoSxM3W_hOF=x7Oz_;rk?(K>4HVWkff>^(6o2L5V*;AM1Y?y&|;? znsRLnWu67OZ;GvJ1ITKDu&N`6WICJ=4bB$MFDhrNw|-!t5qBe*iRg zl*sbxxWBRaAzsOz;gL?etg(8}%s{(zj8ZlAYk90s!oY*zOf~`f1>2@?LRopWB*O0N zMBA(7vWimi1St}3(tcU=!JWmS*+8SKmnu3u)t6aOmb=Qy>Jf=^+_-%NU%xK$o#>>Z z&Z323aGy>}s&|QWzCeh2y@TjlM+wtb`%r96@7m8a8f*+Rn$&$sZKs^jyT>rm$)1&_3x-g)+`j*Li)-Lh#c5;uO34eK z;_|r-c*kamo%HsVMzcbJN#8%7sa(YeW=&o@nj$7_f+5q!!|;jgM&vRZwk8*|*N=wUt%H~svRy?gPmPlN{Du5@)w2ip|-R|>UQl*!imbBq@I{g0FP z(=c^~9-S_^7+!Do?=0Xz(=8~>HRQRG`ABg~Nh!{WEn7ABRL+yflcPSd~zt68`*}eE#^%Y4Q`L)2Hp$}f549+cdq;gqgs=X*Hpf)F1 z;41+v*u8mn4hQ@Qhw!{C*9F{Qy;LvBfccXrZt0JgYFMn5SX%0dc#pr5Hd?8NN~QIZ z+i$f&1K>TgHw~RCt`L4x>P9vb;}`hAK-PIdIkN0l{_K1BJReFcwjZ_n{+@rs}Xe5O5Z(;G&7)h!h!)&k( zA~gWz`$m~9Y>nfEo{zOG=FO4#R1)p3zhrv(YE^I@Qd1b#*dg-#%!|1q1gZQ73p+X( zR5n^EZH~BvSp4x$Yrg4ckANoYTS_0VkcpXRIlWGM#K}!#uv*BA(t3@ohEJl#V604g zez-A73?`b+536>E{rwedAT9Pj7$7jS#FTfUwW+N?CNVF!QcYv04Z|Fj#79JeWJEwc zBz^Rc)elHfzB5b)M?g29_8P@yH)`*Hr53TAW8`MkXm{hKA22a^PF8bFD(qWCXi{RU zhex(&P6ki${Om(f0Zi}J*91QIPEgfR0V#7QFyIw zsD7K4va`1zU7n%(y6{C^PoT{i7@|5QDS;PO)fPh~h5R98>!@c5a=RImkdP2lH&LSq z#yIl)>lW3wY!X=Hz2qCboa%%|w?aq4RGRsX^PLwG3(mj|Vs7`{Sdm0@;(h5d4u>!# z`RKiP9&Z6S@w&1`?7mQL&28x%r6;1gIE|2Mf`jtW54Fg?zm%~=d9XqC%NxBS-OCuR z&2L3hn790x)}9ck8O`!VtIEG2MyMBxPfH}t#)v>6HTCDIZqTrj9W5f#6J!5*o@oTR z|E%vWt#2Qay^s|3<`?>n^wWr!_rqo=DZmfxuNhS5M}Z-19g-g-(Q$>`QqlKMBg>BY z9gimhhqTY%Px@Z8Hk6NE%Jz@;|BjSI{6xy&Rs8$=|90UbjfhI7_XZ^Q#q!gQ&c_*i z^lr-sXWW`U@T>9-<@C1#K;CbjrF;Axx)pEqZPxY3v+EEl0_mC@LkK7YH?YPCCEi2c z8!cUDf=FLx94`z8&2;%fIxJhl{qx@&jlx~~%Y>}_w@D{$ zo)V3I!lr~esdr>YZ>~892)}cE7)KIHsh5AFg-{cqA<5W{R{rPh(({eX*}f`niG8%) znpo}zSvOxNTRz2?d5>8)mbNgmRnDAbD=(ABmyZL1K2YQq^34+jYuVzK?>fphPI%f$ z1A%x2Z5iRWYQ}I~(9E|&yiOWtimG7IE;8|#8kF}MjlM6_&+sOwAU^$M(M_yAd%IdY7gCYG8D6k|rp9OX0S6&-!;b1r3xOI|SwMe5Yvc1QS@-P6 zR4d|eGr({19|Y(-0N>xl(4knq6j?+0`y$YGjkDJSv72nr5*X7D$LcTL$76(EtICy6 zgt$Uq{|p>^Q*2Hr@qW)S(rrK_Vc(K+g$5twbS@D*Gme%0A7%7^7nW^5`e%{4H-IFB zuet?}vR^;A`8mFc9upjYZpW{&c*;VLH&lT)@*djWixQo&l3X8(yo_lTD{XXTTFMXV zllj&s&V*bSAt2ll>kme8Rxp+$Yq)-^R^Pj16MEr5OtNh%Q7~C^S8{L(2wt+G5!TO{ z8vIt>iq~Y*gGM}UwDr%C(^R?p1m`eO@ThtVyz?(w!!x})IS*@aT-KyBW8gDk+D8hW z1nG*;eBQ?Q_p64B)()q123W7QjjtW!Z42j##R)!bQK}j;k&MBbDBxyd1`r?Hol1Hd z9g{s_VqJuRTTc_I(#EGJiLmYssSRu5T>qLA$IhKHnCLv*F^Y9TBrJ(ZE~Ju#w%$+v z_f_>7|2ZocjCk&2h%K>lPlIph{NKe(+qLQ2(*n2`-1Wi!!CmhqpIjLda4n?Qml;G*5dBdiu&CGni+5t*y_VEzq|HirODW-G>-FE%>HH7Q$wU$vhm{@!x z3}nVwk*V5g)f&b~Wlrci7>7J`9-lBgh@Lttw(bXT8^B%GG)KY-^4UXEAe{CQsxCZ= zB>rp@IArp7l-g^a6N;c2=t`;Ri$y1lL{sI!1T=gmgnC7JH6_ukOJMGot4>{CFs|f@ znL4ULa1ANLB@qiFg_@d*8gD|}aFtE^<3;gKo!zG^VfseDI+I<0y!%i8`ugX~e_af; z>c2K7tv8@}g06DO`B|Hf%SbhE04Upc7G2S6ihYqTpGp1s&IrD2a|XCRxHF#ukN56x ziBtdPFmmXpB~Q`tL%t336ki7EFMrMN@%ly%DWjuu6f_CkYtPZhy&);xLBHFGwe>(3 zSNbi*l?5NC;3F>Bs(X}g8u$i&sxxz&p}-$&;VR2utS7`5I1~ZBL6;Rn3Fe$R!TC3q zf4%URH?kw|9mqXe-L zuGbXm4Ch@uSzC{~@S>d@NP*W-*Shms4`F?DX!tS6{GR#FUjwc}dStWm9irTG|19#% zca(j)e)qEa%>T?6%=3J-F5LLi{9G}u0t~M>Vsj)>s&RNnqV`@Ovz=${of57cn$^co z8~Iv`pnBi~su?~?!?Tj%ZO&Z|SUVcJJ&Vx2`F_^S^^(MUm-S9?FB{BauAY`MHAQ|* zH}t}nZ&$IU3-x;oGASYy=`bI)6hJK|f(|Qo?CY`TQtn)f(ltW?kk3ZM1G@2QHbM%I1O-@qkJ#^z`<@A_EuQG~A_seP+GVv0NPIfPD3 zGmZ~WNb`Rc)Bma}i1)zuj}LA^2{>o}Iqbgt=6vSEss01>*Qb}4`{oLua@$@)BH<7U z+3cG|^yVsvzAS#ALAMk70q%~S4>;)8o!&bYEY0i=CL&~6inqmf%Wt-^j`I0aEJ5eC zg8Q}eFV4crfsp(d|26!gBE^Scxb;*M4@>$8A&%Wop?w4xSOjZP^uq81_ zE?;hcB;9xCir3JkNxTIJ9l{UPn7;RAs2n%Y6`Y^K@g)9FX|v0%o3|{^BGb)kI^r%` z__i5RA;1qf@Unkk`e!-9O9Qpem%?^9Kc7qO0H@|mXArrn5Eeyd`yaw498`Y%I!k@S z@`_-SHIpxIE0Xw%2O){D`MTxK2~Hxnb~N^`fO*`;=pxJIU{%@&qqYZ4FF8xEsI@;< zz5}t16bUd)8RD2dtW>WEfkKM74$LS-w_V{`g>&h^rtlfJ_dVh!E8(^wK1DbBrX-<0 zk5$R~l&!Yi=vFMW11yNSmvY=X&~Je*(VBXB-fS4zEQRFYyMao>145&JrA@Q{?|B6E zWyhBIXZ`x<8J-f<-6D#HzRn`LGpxT!yh07Gs1aoR;U`PR5Xmn9`}>S_q5;Uzpw2&d zE_?Cl9kK0beI=Z~KfC@dP)>h%;$x;Dog>J7OBXDT=r;8w3J<@P89)cff&j|8aV{+6 zPKbJ*SU~kH1gamo@J}Pn^+nGB%oXco!>TxKzc^qHc10;GR5IiqP#a4;`ieMtUs#mnqP?KE{P=s!QctP3LtfbB!E@<(elB$4E*YwG zbNsI;vKe~w`(IE~p~&g)v=2(95L=aXuaa%fv)Z=;VoPJY+y^}1oyDX4``(}#$X0D9mk=a$Kx9=pT{xgtte*X?t~p_)v55zh?QB zV7g)bc{_qfN%a3QauSi6`B3<)(y$PhI%bp3??|pa#hkHiP50;_keK#Wm_DO202r+_ z{=1hkt@0g;NZT7gBfbLyNp9AnA%=j_8*_RA_V`PnHwZpPPBr#j3bu0M!}X@8_q=_^ zv~NkFsLO=($gF$J_4OoRxSn3$+xKBVMEP!M?vWq#)Ka9|SgSS@>UkGVxE>Oxx*#>& zh+Nu9$isujzzLA>#d-SX(uQr!|c3ASSV=*Vz>B zU?fSaFDJUuGYn>TC?V`=N&Jv;Ioy^Koyi1 zl=IywJqvE>pL^9kJHG3FB(ZVzZTsqlG%*#VWC|3^qs?Zh%1VH2x%gh+Ja_wMK|rFg zVD1`2N@al`NfUG}e=mZm7P7cr_J>2ChA#f0Jsgbss8DU-b6H|X@vNrzA&^&Zk&HUI z_~QMRaC;o^*;gWx8o!@gt|f|jvuebbodOmrc3RT-|7a|&|6K;hnZHZ>#0gQM&VLe4 zgQMN}N1CkpaA`a|*&!`e^X~L)?X`k%di*^14SBu~wZHxEK>I&;W40mYtIb<9u-5#5bis|slbfZhO0>XM zKMzD!^CINUrZ|IqeQW&rlAq+z`77Q1;8pjig+gXqjre zaU*wCC2v%Pu6VD`Tud=PU$oqfc0psua^V{~w7{`5-$nsp>!~7y8!W!u{rvxB0dD3e zA887uo>Dr@#JNBMJ)6eO2p*ea>f2qkipH1=;bGjJcUdtXEDZH=hR<9e%jnJ%e=wOB zH^Dg%7Ei;%5Q_(&C)U+|EY6JIgKwtjB6vS<;sOMp-en^`^9xW89NlbP^Dbov(hhwa z`Jeh~suGnGn&KW~^l|!%IjbL7<|F*ER90k6l)qJQ<87^F9c{}-^oi<1Q(?PGf#Ux! z_j&mx&xsx;A(2lwrc`x6kA%v5VU;cvK>@AY=sI4g0sT}_69th4do`$u<7?7n{I zm&$R+1dePz4yzJ6(F6I3WRh}oWpGpS5%laaGMz<~(4R+oEA?s1gKAZ$8?b6m8@ z-jmVCzf@hmE0CZf^M}1;DBkt6+!a^W=KznaYOT%y zcwxYEJvV!Y%X$2yDPHu;p7dVp|GKJ9m1aWF|C5Vqvs)r(zg$!(f`&@T`e=fNg5Oo0 z7uUz`gJ3w^)B;vrzDLss#wXAAIbKg2Iqs>1(U6V=vbAXyfLNRgXubd&}9rl016jJ+*h`Z zL2|8v(7#1lDD_kOCoeM;$E4Py{R^NGym`Yz^fIvO@Cp`e!aKYQKl%(5iR{Z<&x3l9 z9L%t;{C1mcWs6J_ubabwduxb~yKF?H?ON4#J#tA*U0SRAo!aQfihc_cP}!W{06zHl z@bErU>ej9+g5kFyvVwo56$AA1Cns>jzM!*Nz)!)4ZDu zKHS!x(+MLSSMKz_T2=KlfL3L~Q9jtbMt}4w)fN7H$K-3fur+^npLVshNGl#jI@y~3 z0Hu)U&4n)Ky&oKepOXK>yM0ueiS0zePb(7MSATm{AF49)2NSOzS4zl?wn{2QGi5~xhO}1Q#DvLM;@*e3M@v6#lWpF9& zd!P^b0BPFpyKG^vbTYzrt8NHSMI|;KzaNi2PdG*1lo~_tH=4oCArCL#K~n`aSWQBD z{zAKbk#Jp;Yb}0o=YjotdIQ+14B|TRyL^VNnAZk1KkeMec4^-ifox4e$>rT1w*qLo zor{CU?TxBpAJ-frR~&1dYE65&;*7;0{yT`<1B0iBE%R-0t8QSkEi2|cX3qk^L>lBa zak*X@Ennwk@mnm87o62fyYXGj^Gd?)asO(vMSG=fZmsId_=E$Qr8X{txKAPS^q#l@ ztb3CV;bz_nyt=bk;70^r-t})xJXiVkESx>p(0DCCB2UV$AI~YE4}l(j_{ORaJ>Bb9 zK=@OtCrkmp7v%dmrryPN90qmODrELu;F z`y=mx1n;$(Mm7t``dgQi|LQNq8^DME)p*MKPv}1+!2oKX)oHDc-sA6jQpjvNHnq^~ipbsXR-=UCt#OS*J~` z0~R>y?%3*=20v!pw#C=;3H-Pt=Rs)lKt^5{UTc=~izZu!rbN1oCEVG>rzM>wrRQ3Y z$JtqH36Y0I+COP!0oApPklE19WEjcO$|5QVUOMvfiTnk-Vr#yA{=sgsEFBF=_ z5n^7Eq8KbV&ok|j*2goB`fuVGMey65nj8kV(Hd87jR8CjQgaa0yMD1da@C04-B0NG zw`h>!N{Z56iZmskG1z6?sc@@B4f{iNB=B+La@^*g@na}pUkc9y6q31-EG9wvKLz$H zYC#1|J1I`uy6A|5{4HN0XHi3BU>wR|)d`6eb=SIYo3F~*-A+&5n?_Cx!~g4+b3WMX zLm^-fhE-vossRlDl4d_m4$f8AGM`k20J!#h>=ps)L^{ovCJQCjLwXu2KY7t4N~ z0lgfti}l*Vsfq(bT&+7weY8?fhhMF@2yI+G&Y!ry$uMu*!m)SNxb}V0N~P0v$!^>C zX4X-%BGq7uoS`W^m7Fr)w-U%wIQ?O&#c5gK@Z0hdbe%R8xV{qCcK*aaQhjB9RCSpl z`4j2APu47+5Py^xq1*%3prUK{`hh<_x2f!~A6BE7EhvdBKIBRk+sanhPOd16<=(Pb ze7jSrqbD6FGQFHiqBRR3kae9iq13vDbUpA$()u9?>ayd*mKNZB){9Zj-Q(Db%CGzCI!wEx(Q>oBdb2_)pv9DpARC0yDWI)C$@_eym$PiQrE)*#-HEb;DP2(EWa zZ1lyD?tTK0qqu1ZtU(BTqT9CKd@cq~p(qHnxdF+fz>Os@o{DJM*> z;D3zv8z%5f|3O?|s@$LK(rJLdLR6To2}hgRr^Nf?7>9^AoIkV_h3rG7DPy(+0|D~!2i)7A@BBt-LU$Xdv>Ehgmiz;aZ+I>H z51c1lF+8W1jrOZ+^@VSM`*rJ)jRN+Ql?d7=2cYg67a0@#OaQzIPa|ZEOot9fMtu zwoaIHye>hoTDQyeZ7+tnZN$HeG%yGF+!vHHhbV$iG0Ilw$n1VLyvH9)2>)VJK;k}g zeXx6Hsh~ZXc~z4StTmS6{ZK@amXbj=_wPn-TCHBlCV%^2;XQ+p&A6N=An4?lZ)v$V zm${-YjA?(s$jFjANywpg^n2K>-saSw}^Z)d+t-mIaa`M1P}0WY|Xf-4HvsJ zmJm?;SyldiIj(f$6SQIG+;>2-MvhMt#GRIr4(dJqFxN!S1obsJzQaPAMH^py zVIxYzx?imuGP5TI+ifLy4vKv5Vci`L&l$sc&}s!*q3_))9A?7qW3nD{ONSnFg}-)n zd8;cPyYVx{Eho;javVgJUk*I>gLw5=wvB7v0i|U{z{pZ3^1~8}N38JZS&~>7?>RyP z4960E^`ymi^S3CU zENSfu4=Uv={pO^2gJk7HQ)8g}k9=zkQ8tXT8A&oJmx`EI|9E~o%Zx~ zOY8RNcq8b}p!?lJoOvd17^}V!q#m}Bbf$cN15Kk zIQe-Y`aKaoK^hNVpUpurN5xA~-n~|FU!S)9zLBZQdWk%}^!U4Ww=I9Mpzh!P4N&}7 zY#D7}8`crjU<{Of7yXIkf0ZB3-T37vUlf?zQo7oE%zn~l{h-obH=kn);2(E-`>b~a z_8n; zq!Ch^KRGkEs61hnG&BCON5E*f6P^SO3`K z8+{#l0{kz)fZv^ox#ucX8WMeTrjIj z-q!6YxEk?RnPkEYR5$S%AijMcec(Fbv6?oO=(u=>QEd}pQqQo5bH`m2#-3ROfQHTtWk>=fqm^twM%x0E8ln~fuO-m`|UDv@|fpZ z%AigaaE8%6gP)1k??Y`*P;`11waR{)go+&IxDcVjj5X?@!qxv4lXj_5Z!lBR$M2(* zS3X_s49?iXp6a=fT*vW9cTM@iYe}>$0fGM^}sQsroTb9j3 zQoxV?oL8$>*>E&MzAK@JT?^ujXZLkI@|6tOAe7poilH} z&nyjb{$TrG=Xo>;Z48b_g|P!LM2-H@9)-AN2J@cGlLKdH@FdIFz$uhDb=e2W5+{`7 zkI=g|_g7qb9^J=R{0#65h=d2#n5e#GHqDcNF~>d<^m@;q=Yna2Rw!_ZVBKjiyZip4 zlEJOzxs$UV1YSQ8J~CVny2{LcxED37WiO?TgcjIO2Gv0*da|4!Cfw^$1JfuDmY4?4 zXsXi#ED7Vg8qrR_1NbIX(FJBH(L(|ih%-%E^+2= z1eP)=R&V;#l2Z{ySNoQ7MjtwGn%wVHgm?h1t?4 z#nGtVbl?o58T(F3I)G#!5}PTX-xBd%O?J|$;EiDUJxOOnxSh_BZ)3!{30?}K6RFTs zKoaVr9;e4LWSRWeqj`Xd3i1hTU--ASATz~<$4Hu?LH)bFGre)}s+`UzKF)$Kr@Xs` z$1BU?jqTVUe1r_0feO# z`rOekuOv`r(E_1=IXR0RtIM=BlcLMtld7wHVF5pGd;+&@>5}+&agF;NGW6S`L?vsa zz*GX>)sr{NVpfg{+nsgB1cB`f4)kn%s%zHXO6Hy0eWn|H@lAwi++|U0R%5Qe;wpSw z3#a>Vm4W21I2xKr4?W={muT(To}2l#TwaD}aOxz_+7av-gg3d#-%S2t%S)CS zTJ)Mtf%E{)A4Zdj5Z*sSPo9k~@5F2N6JM^4&Xf=Ci1Ga@7ZySr65gpT&-juSMCX2$ zAE&1D+^yZs!w7Wu-%s49mHJLiFu$~(aJO<*)q&s@+&VmW`tkPTV#>J4=QNLS$9neEBw>w|_i<+Y~55tw+AEe#oUu26Fi)ChMcU|>^%D02m&9HZ& zqfFHnxIx9&Nf-o`V&eyt4@0AGy0;&~*3)jvdM5iIj<|uG~w<-j*hFrgKe3@?WNNR++ zvvMtq;g&L#(}Z`C340Zu7!vsjB$~)#%o43ySrG<&tL(m73vDHhW7y30(Vo6JmZWn0 z`IJ-oN+xd}(P}%WXtI4RbR>~dx4|5ogNd(Ji~C0DE!xgH5WYL)|GXGV>_~rjGG;B3 z`*~A8{*XSVj_xx)Ayd~MP{Whx?-_C7C9`YXaXkbjV`X2>yM2Y<))nx=ZucsY}8Dg=D?n%+C<|H2m>dOvaF7I4$6 z+bZXhN7)8_MNabHMd;?CTTWQtS@2oI7A5B?N>8LZ7N z+wLR%J#$Yqc+EVDw_%)~_^LSf4sm8w>!m2as5|_FfZ5b;r;(wa!Q}B3>WQ;Ow~q<= zFhdATtBed{jGmV!T!+*MAbYc_?HNW2A##3-vf4#A{~=vEvG&IL7d2!WPcBN1>bXNu zcVLUAGelS^RGhXxVkSF`yOG3JqL^!x>F=4;v5D#Zin5G*?r+}9fd-_xz~)$%#UxBm zk)t9W6}7zI^echXT=&uXN^NQ^H1Bn><&>$bYkASMv(kl@{^hmy|G#-kjv8P0i3iw(wgw zAAD8fjb4AN=gk|W!QUDVTO|{U{|K{4dzeWTM3lc!IC>rCYIaJqLdmRd=^J=Odb+G= zUS+PL)G>3Ej?b-^W$b-7YEHSUbE(fR9$Gmji-){s*T-GGh!VMs!#=I#Rs4I4}oVV@c zD}52fX2Jva;L0pIinIK_vPad(=;n;qNM9@90FDq$AP+W^pZJ!KAr2qj`{&ipWi_D^ zgm7t;@SoIr@*R_yZODFc@5Y^>7vZlg`pt2$W%fs!!8V_?2FFIpq;8AfM95Gxhh)r_ zpe8w=jl7vdvyh)n9W$jqy0SiPWx%=SHy*Mkb}rH4zcWW|i z@@=>b8v1B;4Yss40>@3R4&t^Xhw0H4lBzG8f@HBF$6;IWHep6rPNcFxW>WxP?SjMf zF|}t!ef0Za_->ZkFGU+~$4dtSLx}9)N#VYL8Y7*;LW#6wO~f}4@amMl(iX3#II{F0 zKcy)d{r0`0&&xwl`)4K#BcG;OqwOgFQQ?uU&7Gm;&tIa+E_y#TBjh|S)?X?t3MNHO zvNklmEPDCS#FqGKdz1?5O9Q~CQD-LmZd3s9fQWpOu{o@Zanz89@ey)B|kX_vRYMgcMwiKv1Tp>%+3?G8rFU2r6c?1(B z8>|8gMCXB2{WQ3Yl;IW*rkjS{6pQ1q`BJlBmZRyW#f@e+TL#{GwIhjTJnQ;rS4soG z4Q|^EPFek6hxkLdmU3STa+%v57umLnV-RI+eX-X+Za%MwJxBDu||n74Ythk0~$!+efATTr9!nIS|#Nrxvr9ZVT&h!GEDu){!qnRaEcU z%lx)KohOtncve5m#f~a(HP5PnZ8(ps9UZo~`>Qy79NA2nqY772KRv)g`1Knm!Ri=k zVT;|l&65}P*$0$)3G|#)zG$C3X(f}HS*>AY{Sgj{Xe8rM!8viP}A_bbkZI4H(&>iMkS4$=&YsF6o1$98QW zQI4)Y*62M>DaybPR6N363}TnnAA^QVvkm68kT=Wy!T4#8f(fSFb)m!Z!!q>uTB@D> zI_ZPiVYM>5v{>o(cRrPt!o3l!%fz%jlVnEx()-48tDqm>_#>2e+`7@No5L5Jqd0ZJ z8Nw;M!A%$erpv9v0nNjx@jmu=SduC;2M|Vd{)|rdzP+4wm`%pu4|(BR6Yh}~nojgPBoIQTWlZ#TYb=IR`qeiW!Vnl$g9ad( zYcQG%+dgEkUStqC8=*E8_nSvH^;+D_^ux*6>Q1L*Bxi^_f-|ca7cRJ;nUBnP=T%yN zT;)IkBic_QYA<`lrs~QN=E(@nJST-Ony}G{LWwbc)IXt}dK)J+jkw6$1&fSY>g?ra zmmSe!hh={FH#^k=19qL=dFepwI@djR_*P`Vrzq{B+is8Tz1fc|TOLv#dRM;4?M9@{ z;;Y)We(Q)KKA$YrQuMA zG_0!rpqhFx59kmAS2gK~N%&=Aw;h>}Tnhg*P3JrY|n^q$MM=thj3UpG@&_%9-~3 zQ!K0K=%qLt)v(!G)$utLWR=xYajj5{>^-%}vU18p?PJY<(~by6*qf4yN5PS|~AbcKwz&Dz^-Br7^` zT{`d%lhvq5B_oqLrjnC{AaX@RRod2Wzm$8Eq8Q@@z*k z78XB=*}Nid+0!;5wow|t6_^1H>wS|8m>%@~T;CnB+y{060GxVsw_=E<3X7v;=k_owyK}wBschFX|^ydyEF^{WG50h6G0fp|W7c z;4g%r$gdWoqMNrwx6B&7Z=D%FSHiRzH3BVHDLAF9RQv7{5Au(7oIeo&PjGO0cfP4Z z7SyfOBJB&dbas8jOlj)E=Qj914;B^nUmT~P8V4W?0zruvvdhGwm8}gPE`C*F8@$%Nx*n(GniEvhSih*rsMyx0P47H z5`-eG@7=yRu;Yz5#xmK&mb!9k(B(l9-&QjC$;j)=+Mb0`QI0hliddyFC{7K-Jo&a& zU)FQJ@6rxK3q1pJ1AQD8H>3fg!PMXT%!S=Bu;Q1p3=#nXlMYxx-K^xv6bkew2{kin zs6ZnU3;b z8`+z(&P!tpz`3}z*9kHAymM@OG(6w3jJRJ}U4=1B8X*}NEO^j)l|Gk?Hm@NUe# zOR$IzzKq(L*if}SNmGjsQ@G|Nq{nRn6*@k0o*I6tCC(?OWtdwzRv^j zOoXLAXT^dNLQI2;IDW3jMLrBP(6Mv^M70z~ON`(+iJXvf zBe3Lj?d8F1<(hbD7)X8l3?4(1IHxs^rU55k<-im_7b<2ZdP{N(gz}DVM)Kl@!3Xf# zW@rE@7Z|>0B!^q2`=cfwOz5vBCuiC9l^6x73KX4nkc<`>tUB8qh4NIIrfj0t>*Ll< zSn>f&Cl*eh#VjjYs#}g(G?YQL8x_wMR2N=TZ%avHLaAL`7lx%jVkhJ8&1Dqi5R&B# zQ1A||!%EquN|OQ1-MjdLqk%?>K4>qdc)v0*GHuDRGwms)F<#dn878y< zGWfJ^kPAPDeeoSww~|9{Ro7i}N18?c($^!FUC5VRpp5C%8p5(J`DVst)ZALFtOU?0 zq+%g8>mmoPXqGT$SPW8#q+Y`o=tcE(rNebYnOqKB*ztQE0!y(Txq@%FRww5;>?|b+ z1>ib&r_V;XIP^hM9=PD==bhlE(u?PR=Bez#LvC278*)j0gsYt?hx@d%qo5~2tG4_gh`L5+MP|B+ zVnO}v;y~8Ip>-xI#X~n*&!$9dZuFIhUAZ5h0(9b{ndYNzof6#MPS76DxTdb!!_TLB z3AcVca{|p}6ktVRI~EsQG{QxHktBJU*f17&5h%K2_GiHNQ)zjlu`YXi$LE_GPy)4` zh@%d^{aao~R}Hd>F*B>D&C1{-lmS0S)9YFS&P_8{WP4MEUfyt5zN5eN+p0SrI-f*p zaaOvCY&%ac;y|s&_Z^?zYO42JO6&+a%l{bdXI8fv z1u4Vjf)>?n>Uhg62~URtGiJZ+%0d&t!mG-f8e_k%>U8DX!@t{&`WxL~Q$eTACtVkQ zT@yfz_rD44OWCnrJetF~uIg$6Mh*({^C7yCuc!s#vch*DoAT+X@qoYddHSV*Ph3!x0Bz31?F)3_Sd z=x>Pe?i!ow##GrxeuiZB=51pPHm=`RYw`GZr^}fv;LInLP-10i&DqC%7GWAqSrO}EWi;)J2UOB6%x&`0^b zkTP}#j~V3^&SEA{l-kCPffdf{zEpmuCK4Xlw0LztQ)ZDk7THZ=0n(GdHg2iN?j7&q zDA{@xH2Yl)*SpWxL<9uWF4zm-3t@Pt1EkM%f%_g3(KeE;XIQ2|z2`b3^kar@oLZLz2mlYSHZ`)vXpM5J!w9DgYd z?`Z~1wJYoL|+Zv;@9qadP;ip2lVFhclLsb2aQ{~OqU;rcJg8ia(J+gt4g`UKzE zplKn2UoZ|NFeGsEQI6CGrOCZHUaW$z75|41t4E_woPthDrd{mK-)M1$p}l&mnh$`t zKDf_C7n0$-|YT0PFXf{wX}~2Gq%gd z|4g{8nL&E7_)~8A1~4vn+If20ZWW}BywXI88!45xeTRbelWMUJMg~Sx#fo@6EqeUC z-2#LXi5b_vt2S32Qo`Vw1z{**=Qz-38H7^*(QV|yr;3}1C(;-o`uC3abT@z{E+unp zPl<|I>1lY`bR`W$e&P*1*{pj$9=}~B&5PFz@(dz%@x?T|0hl>yleGkjO08~4?$Da- z2jUnwQLZoGR}<-CSTId1@XPx3B@b6Rp%HbkqJ|$$U(PD+rQN5=fOI4>fC9o0-$A9t zkyo6v9iF=UC?!hL%GAu8N>5DQN->i)yQ7t4xa_|HWKKzDM*T4Q4}MQQ70!)*{c53iIu){u_Q!-=UA& z{eyDL6xHqUthvuW+u+$9NIO3c18v+s&IY5k+P6?gAJCvu9o&1?_yqWiXxH`JFI5w- z)*%9)AmeusA6LPVf7MVuF@*cYbKUtJ1nD(;w7_|0FtNhhJ!@W>LFM-_$5Y-6kvZZ8 z`eNb?2XjDBI{)*iKL}4Q46o70_FXIVJUD zaaC7jdtmvbXf}}~`+*#A94>N)JmhboS%#GXa*$fPF8z=Oq!bUwWg zPGP4j+xI0`dNZ&`LzW9~nklJ8Tb>KIVf{i4JJkPCH|_os_QI{Tlz)gZSXgLT#c(KN z4Q@MfHx(KyE>YpOY@GWO%T-9ykSQz^!K0DX)cWxT z&2jYdxN{86ST)moDn9~#X}{TFza*R72shy{NLv0IJM#8AAl$@xP9~?HDuEGqz0ptP zOEurogD}Bgj4`DWn(OJ;5kBN8!7#l~9mcY(8c&W(B%P^|aw}5IxN31%zI@hrj*W$b zRq(ptJ|pc=JYW_m`b?1>o*d+%KKLkSN5w12@|pQ1p*n3{6wKKo(Svm?VmG&iMoC$^ zp{?sUe9{g@yV;D;rmLP2)xgg4xu<$R?yZf`4;{|r3-1riT(aN{iI|X^rL;d~3roV< zKf8O)j7e4sLDkS9{=gq&5okSWXjOJ*xNNUi8F|8I-2l>|`EgC^$b}w?P0^?cskPc1 zO;63LXyW2Jq+Jfq=_jFpu~L}7^tnb-L%Jsx3s0Q=fmseht4%opCo07V>hp~Ae-BAW zG_mp^H7-gAfEM}fm%Z1ef6V~FhGEVst9rQ}h*9;0=*y*%**Nn47 z8ya(->8$P9$N3`Jily(Yg{4JJYaB6pzQ4n|=J{eoCzWgTQI_Sy0Y`g6>22v61;GvB zrNG&;z@vaaUY~K`^+o&1+KKGQga~U>mJ;J;$_Z&=sId7?iJ=beuL;lim}^pa(A&N; z#JyYlNx?=4`zDW+b>j|mqENW+aDH35ZOf>V%57kC*!R}R1rx41>j{SsA81=Xyd6SP zTwd~QjlB*&LA^I}4e&czG8up4_+hx=zxrJ-Ix*juIAZgkI1=*z#SwI82W)d^X*)b# zfEd0PqZmgBApkLQsjLImGOFK8PKO6`=>9<4+09=fj$n{7te6p|i!W0_5=MFRW>$E( zn`X9==#bv6L&RdT!UbQ(Sc=Lh(Y-%HLDsqU>>&U&xSU??H3a+kc4q*Uc`e0 zc`%Mt>7X2n)aw*V%d(Xjb6n?%kCRZ;xR>OuwilRoL`QX~I4LMBfkp4xlh7#mgQ%Uc zCGJI5=Bq@Y_|2|8Bp*Z|z#wUp4vBbj2sT%KAi1=D?Gps)T91CdAc%|BhyZ591>HMs z@l=9T!JFcLuPq(qxPE_Zr$mL{k#%tOb>Dn)Xa(W@^dhe*=4#pbhv+xR=9`Mzn;b>T zWx%eevZ!G@{k%q3FXCMHukWlD>B)T#+g_Y98V2nA98l!;i}GOS&%5$Ip1KmZPm(h+ z4oK*cZ#CEDvX5RlubKt?btca6VrE)jBNOp5Hvu7K1TYWxz|9vPNR+ajhUxr~|NO&8 z+rt;mXgpx_W>`*2p8grc{Q4RHPAL_4_CunhLwesKmxPxaO>3kaI? zKUn|uhnIJO>3$>L3h}X4#DX-Z^LMHloNciC;t$COkRZ35A}F_g=nDdwwB0f_Z*Lnk zCGk8ceq|F8bp?KU*Eotl;NMdpiU@5P>`SVJ=qUTr4{&x< zCn)t0hzM)~HIcwt1@l&vNfX3m<}wEACbzU(iJ+~7^YXhs;@>Yg3sd-JJSlhU5#J>V z;|^s?^}gerg5A3<)*3O`W|_R!T9L}W25J9cA$~l1#nAX#8|`1^`-4Q`<@1G^t4hLP z?%e{4#jiN&@I;RBjZp{2q70?kR^y~MeFKU3BD+6`^p3DxoI;C*WL7^MM^wla%t+^*arn)SDpwlA9B?V8!5xe)D;WIf`*F%f5uzB+* z?ex*fW=@a5aRp^b=~A|1Z#26lqtJBa3XxTYe2%^bfIoYpZzrdcY3MFA)mVEQ-rQ$T z*lel1`wEiqW>ZVebKYBdUs7YI`ex;_#p>fM)5UEBL`(`wQR$*P^-fK(l{sU*8a6RE z9NQ<$u%Az&sdh3y}1Ms zS^$++mkS*F5r~yqcseNbPc)AoN;8_)7&jJFNNANm>L;?@TaD)-?d$y^;{SM}ydYqY z+?2Cgj#E1yCY#y|aj|DL$+law2g>v5N(+4`nr$n&cCHxji{bAy#Z+GTR0J0{$S?MXhBMV}TLE&z&Fexq?H2O%A6Ppo{a}Jk#lE``-^(HO#|NJBpRB3nr zjvnK&0mEtFCQ58^kLJU|QLN`s>avS*l=j+_@M9Ng556=8QX`XNbBMFM>{#Hf=(lz4 z_Gan_Ku`_|@y%UrvM>>f zHGf%&ZTZeBv8;#NT!Nc=;_>kL5l)YP9B+To6BA=$RryYbRVwufuVH1%JgBRvhO+sY zjJvf6cXfteXDqy>#Jn`&+KWig55?E|nEG=L`)8KuA1CZ0DC_sQY=kSGlw$is&Zs?G zxZj&YM%HERjQbZsIItVWrW$OAKsr)%o^tO4RrB7sYM1haeY5p@7=2^^IvVY|3`QH^4Rk?nG=GdFQ}C8N=dFE z41!cgQr|nvlwtH&;VHpI;lntPOGi{nh{IaU$@g9sB5b|ub3tUlDZzL-{M~Ze zlJ)Z=-YCB-CDUI6h4S4C$Alj`gZTi=BR87>+- zOIHzPka3Urw)rBEIRxyR-FS{G`THRK!A7QcLwiAxVQ+7dZZMAM+~UclZetWbfAB|) zMW}#?kaM?dP-!vAa6`pgIkAk*{u@Mypy_VZ)^!HxiGJPld9XeS`LI21WNl!(|7P=w zD@erXxzX<|B26s#`TQL742R2)OyXIZV5_C>{^V5NtsNDR;6;e0{~1foYGok41Xgl# z@{ZW5Diq}2oPhr42HWIZ0HAl?7a8+GTi8l-qe8W?&WFO|!mwW(pg7lGEUcU=PVF^! zK`t6P`gbIY0Vg*)@A25UKPzxE@bPhcJ?!~h`sS~4T{kJ{^kgGk;OSK4^p{frDyM2M zZNQCB)RSU9ES9fl5388Q8exgMD#Cq=)q4C6Z{(VhP+^TGU&La~s?>QSemTH`l?9f) zXE&<0NWAk4U?NmEipC#<4G0_kX2k}kHf@d%;cydrcUpU12>V)_Qi+=6NXJsAggF+M zfhKtJHfA9PeZ^-Or8HbLn3hB=jh$LD_+d^Z0?FP>>F5nTr6WbH(;JS&}{FY+qrX_ZqC zsZ$T!up9OjZ35jO>%EwVoYf*p=vFWH>Zq$ku8+U#L)z2FPBWlH z&ZAb~&T1B@48eU626r-ZyPfpD8V@9@3)*wqvJN^8e1xWnMzwWcuHAN|A(#RWQzST) z{(f7P1Uu$$0czZ+uYSbHDrYG=&p9|Q5M5)VBsx}@(1pxLK;|Yyb8S;nLiB;U4=uGy z)}?Sfjtb&&y#_o`QD2ihcPH|Km=)WmYccK=6S9$$vd$o$$k6cS#@`l+)2iF0v>rUC z$1UR_Lr4%0n+vABx^jBfck3Tee+buRtiV0WSCq+&rjGnRm?;ZP$SD*2^;cW<9%Qoh{x-47blKdZb2B;dUyd~Vae`5QJIj#2X zQjoVBmW@%O&|*gO#dQN;6}F-iS~;4P5;8eYA$Vi?`UetAM?5*24M{nmDZ*pb;S=tI zqPeVzODc__8Gk#JJ!|J=jze2AyDGvGwd5yj8NQ&jUm;x_zkPh@+K9>v+Lrd-#H0P~ z)O#QF7jATg?|Y0kwT@88#-LzKS|LNaTJwlB9PEN0up!Co`s?ZXnVl znmKy)`xSmOJIBWIJp4P}8W9ckw&H7ksQ_@Ji4ZQ8nx_yrtFOvhx8AWPTuF=-T#cMv z221r_srH+=U**tTZHy_Q)Z1+qj>;igUBEIGlm$#);Uf6qt7444!@sA|G71a9KhGcHgfAqJz$914#*bwxqouR3Dul+q75XPJ-4| z>0P)Kn8@3l0Fl$4LS0WoS6SD1$o+z53uMcBRxq#7@)f-SM$?~(qGY_IN6=)(PfV{r zEZvYfcq>RcM3eixlrPD)-k1YwLGTA+^08(=+a%eu$B08Gl%AJJ_q&X|YRdYVN0#q^ z+^?U%aZ@EGl$37IsDI-5Tlakl=ttgNLfc=wx_jmBYYD__|K$W8-C+%jC1iSQ(!JAP zXd(a)H$W#Ys3Xh}zH+ut`V0uX0|$}bfQ|#&;oxU1(Cy0w`$xn$OzPBN(I1?AlR=H_ zlA+5w!GC7B@i2HhDB&y7dpTdn)dvGT|AE~$XoEOa#d@&W(?ch$7xb;<-X%zLhm%51 z*EXJC2|eQ&-|hd+iOjuIMFC=0WXWR|#v zfD>gWAYbEj3a{=SLh65I(ChS@wiaF~iVU9Np6!q6 za89R4Xa5y@^tUiFxvwHw_o~j{LQHezt-4AX|J@RO+u7^n$4d z-nrdgiuG9^qtp0pV*CH=59*VrdGzzYTr=)t0NuoSZw%%Qt!z9#|7TGYNiB%IBG*Sb zgO(`7Y~hpD7h#lAhW01qLuC5}%iS-AI}J#{z-rb*tl>sZJ+JzoQzoc6TT1rq1jEkv zdIagvv#*vq2iY4k(X8*}t7Ru(>6~#DRoGUE4WF#50Rh+);F)i^m6>})d;S)3$om9W zYPypO_3WFXQeFsED_kf_qv@Ms?Np9_y9@~*F)Nxueiwtz>LY<{tmy1&=PrpKS$JEc z!@)!trS+4PHjVh}NqVr@&Mx5>SH|aE|55~daOH%)k@Sx3)UNK7@sv;x_$&hr*Z~?5 z%q!dJOk6Oq^u9$zoYg^jTG!7JJNH(Q7F&-1Zk5j$5;yDzzqRZMo8CDnJVV5vbuZ%V zajydDH_zA+YpTDBuIx9E5PLtf8!jMy+Rx;-S&$%IloShU3o&VcBLI-HgL7wZJfW8z z7t*{`t*dyKvCP~c97Gx24@nU25l`ZwmbUaH#coi+ngL;34!(%GaGp1U9)oM#atMcC_hx2r$errhO z|Ist*cjFz@7M&{gn;;0Bb@~%CM1BL?|8Gd$7Kg+I{@nW^9{b7VQ#RrklEt4SZVl7_ zuc7MZmftfO9!jNpME3cJqNb_zARbx*!yfdlE=R@}F!Kf>OsmE&NTKr}cSxU5@d&D0zGPK+D zbi`5N=u@oiehs=X-iV9axUm#k8;9AreW+VNf91mMqYh~=~Wk(8_G0~Cl#_SHLev*O8DY&YO)w-)i;U)5g= z*5Abym*uk*#ifQrqER!$a&BtSo7{uHa{a)`{~uofZPBvu{OzB+ZNE_|Q&U(tP>E+K zn`X{hlJz9JHf4eDLC{~2A>u`J3Hlsyw1yr~Sj`XOAeLjSZ1lGqGu((jnwqY^~E4^PS?9XDP1ozMS%lT^($jKCi5s(wy_Dr%2A9 zI@M=n3Ptie40yiY+gng7iTO9qCPQQScm7W=Q=+fy{Gb(h*~54NZ3QGwb&>n-taENl z;s#Q=1WyPz8@ro?Z%*0+o6+on(1(%U-UzIt%u;OM&qilG(za~Y`7|EYyx_8NxrS_~)fbp@m+@wrUmF&1Zg?v|I%yB*?ws;F z0kP{+$c7DtD@UwTeVlAHoemT`%kHAU?K)J2&Y;2HCy87zh+Lj=ZastU^vo>hhIU{U z@0=}2)({yNBlwoXD4po*Yc|q+y5GaE_NfQyaRW)IJK&i21L5!p)m0c_;N|O@9Xy-# zO@SVRmH5vjIKRU3ZLg8;b+%2>^SDdVXEO@axxx*W>K<$XMwoCv+w^* zq&iws@rD!5wl^~;!`5AiPfg=ZrDV~L(-TWt7UGPb@SgfJK)*uXg_u0vw9-?)zs{C> zRu?9dOXyr&8Dj0Zy1DOl0a8)>E&#UZ&mS2BPeK_lMgl@ryYcV)eZ(+$WDMORX6#y6 zd))|5#so_asRBO-q2m8VV`eEP$*bmGepVu(bOYkkBF=noKJ|{^Rz|JP8J|`LA0!SP zOh)}q(t+vQ}q@?5J1t*3tJs*37*V5is3p{KlvS)T&qaG$n_p&20b zr+>mUI8(S{*Fu38g`h!M;6CPkBO08H+V{Tk{-EDv0PP%(WPDjvkY0};;#J2`-if}o z99}ps>yTO~EH`~vc2VZxStORJu14~pZj)UV!|wU0&ssaO>^6s<9Zs9eO)5xFI`#c# z8|gFk;m;Kr-n^fpE8-43(jHtX4=$yib%|yP0@F%mb(uJ?#N44&h>{kiBOS%*$kTZ| zb0e(?5C78}%R@q6M9L5{O?6T!TNT(r!mBLD&hyI zFR{1C$hqqN4FUIltL2BqREOke#l1nAIMqJDA-7RPLL{wyLJ3_z=j#gf>=)%Sk8z7E_7F(x2ivIUz4KL# zm%j$dqWia*<=lSkX_bT*q*Cdmly|BS#ZqLNIx)Y7IAYKNU}K8v1xxH{V;>5IQG<)` z0x1h^{t|*l9PEN&Jow~S4sZR(KABG}-M$Z0@1X8X=VJ@M3N{K}Pi`GV^8%Xf! z%9fh^IaMQVMbq+x8F{W&l<=H=mh3tbUJsK!M|?TT!y-WnhM!W)6oZ>UM%Nu2#xK>i zf9TKev#*bToFh{O!(b(AH*I13e{?#-Y9JE-7~&exItBhng02w*Gye$V!bR6_dZ~-} z>SvV5ZI`sJhaC!p?C$xo>|#;W z+)I4M*c&$k4B-i2UfA4pqW zhw)^q=pih{sbRM=@TJO7+6Cmq``1Qz>q-ym$=0e4xuQ9BCa6qFPEEMZIGYE3?RT1y zF8^`T&Ydz!0D>t}zgF@u!cQ+Q@EC;aV>R@Dsdmdbt|uS$)vwy3w1>#Xu$v^Tuv)W6x`7A*Ckk~^{ndr1Y17yoMfh3_&!6|tBY(%Q{1)i{ z%ppUVO7fI!8zMKsSvY8#Sdx)sBAfxdF?=+^X4ywhe5fj>lq0t=Iq{EcHswYAb0IJv zm?!_M&ZT>sWi5LY^^b*nR21d)83z+(O$Ygu_JdonGIDV~;UTU|B z&+qZXyw7zm7CU>Q0HC|L`-=cTl0eU%z@IycWs_6hWs?M@Q%=scKQyKYY{xeZd#)G` z>3_0I58^F6@5ScN6{jsfuZAA0B$oz|MKhjDzY2TVBbFF+5qTlwGg;Sa)F=nz=DOix z64o=&U(t=Celk2`*x_qko+e1Ex)zVCx2-_=Q_#t!&?$(0ghoO<$J-Rpg}*&ybYD;0 zjRHT;05BgfSlk5@CUtK++{eKRdeMSmE`#enBurU}udbKF5mw#U4qcH<9buhifH2(W z`F&`{UGN*~&T~ayNO(a%+-J&UE9mZtTErWob8#2|ihA8?yx>h62sYru#f0V@@^`*B z$0ww-HH~~=Pm+Np~4XJvcP|8c_0VkxJ0#)oH@aWU&H_up0!G7DG zNL=94w)w(z;=}eGP`9CyYwfcaWW?JH_tw-34hT4Z{-~98761Z*VcSa!Vl3139q#(& z?piwg2y&?Odt65(LlJi4bYamng(G?RdS7_ta42ovHAkkE@4FVB33zz-PqG{a{95-< z)&PIC6c#G9j)Twk_3r$8?(SfeVPLXsHPw1N79T6a=VTU6-ENUtL6^^&G zty*TfIBGb|AHvEg=LkA%^Umg$h_TVZ;HReaqJsrIB{oZ+l?MOb4@^A z!H7>UgqU3jBuML@!?{r|54aKiczC*HG@kHA> z5V6MMeU0(N7{nm5me3XH;t}tu3bi)ec|g7%xqpMHd&+IuU-TCA8gwG_mT}gT`f~-p z=%mq+G`~6mJ-MDVH7Q{spj5+6fJo$Vi=)+Z2 zGe{V3{?S?9@_1L1P_&O#Rb}TNW6#EoY|qY`taU^69+s%`g5KO>%)J3@i_}$a$cT^e zAgzM*3-7>H&0AHG#7?eARyDcMWX&95wdz<)Qzq{7uSLsTpNh#^LzK{y!DDhm?bQU6 zzvaHzTmnc4f#>c`rb1s-h1`OW<()pE2Ytxq?B|)r9CMBS+cK>AO#}Jbe0CCT@nM1? z@t-8Ph*bpeWRVZNev|KiL77ht=l%gHw+h-u9DKq<*y(IvlalP`RrljIJ0dbC*7oYa zOJZiba^HQn$qq*+zr({#_)voTr#53N4ZA`PxUNaoiV&|Y#BK3bKIp^yb64{r+fPAn zt&37jMIsmEDyEo5)}-{GKa$!Ys`C|N*?1&9&@Yqwj2WBIH>FQG|oxX19loAV< zL9bf!KEZ}Toc0D^+~3UMSG5qPc&-{U6P>zlA}fjZ0`2Uq`%4IOPoN!2+e`WbwL&U0 z(4xKOX!8!b{_ThF-SgD_#5C0Lnjw@KaEA5CT!<5Ji4=`}Y%Q)yDXx+~hV`aZUZG0__ zePx*YYM7VzrLyH`$nchp$XnSB$jIE;AEHK&r2GU?X7x#w9S3K_?9hIzpJ+wCd5B2H zoWT?FHJmx2e;hAUcNz)|K{*tl;g@C7k{A~kjt3+xG!bNx&m960he;bZD9jfzpYiJa`L;#LfkS$J=Gh%~9c3o2HbGE`U*bTtFg(e8BVLZ$B4i+2u`Jb_$0GEid8v zZ$VhojU#-Hz7q(+1&L$SB{U?xo7mev#L(Fv)K`7Q=bOBJSUe4>xD5iP{nSm{S`90H zvDi*=izFn&e@JoEH*mKwbOl8}>T|__tv2uUeY=9Y%&33B5W*3$Gp72^aW}otpuVf40z#abvcUuLHC%~&0*TQXCtrJzj!>!wJ{(8z)s)CoP(vx8Cng-bo5JqcLc@r- z7IZ-yB_O>_Hd@9xWg$B!)Ug6voC}QFe|3CJ0M@(!&TXGC2VV2PG_h{OB`!E^-6BJy`1p)*ip2*_o`MEB?*Z&Dp-)Or2^5aY;uzzh zi+xL2O%0?3Zd|>WTwEKaH17sZq4?LY>U^?w@$Vu@bt$KdeRMNty?bN6YWrC@C%dgHbTT~}li(0a{L0iS{iUj*wS2Jx;jm)^-LI@_FVBeret}<`A_heI*gFAsLPLXgyhyTB1}%OxP^cN z+*FYk=)oo!TOvfHNFBStHbkmOxBpL4>poun+7e~m7sH1qiZiVjKhZ3+R4QCj&aU|N ziOK4R+GWAY0LjT&>9Fs?;2WRj3OJHKI;iHQQvIf2CB8CBhg~gpOo8cNc`d|ttZv+Z zleNxw<(l`Jt!?s82e};nFTx7#gkdC#vBr2`ihwGK+~iNm8**fT^@{9L9KPFne`iKj zv1<-93nqDs4|66?zXCR;JIHG5s&5#LeJ}9`S=3^6FWK0P!AJ;-r(`O4v1KHFkf7Y| zKzY7t;~rF1z#YpMyVUed#ph~*uRQMm==9iWn*Pele2^AY8~zPD-#0hm)NxstN`&`Q zL%!~F6=^Ay4wEZvYal?sKF!!`380mr3mS&neT*Ud7+Hv*Hle~s@r75Bt8`sJO8US= z1ODP*pA2clIRf~=NpAPWfjHOyXL#}uawW$-K}&?Y^*WP5UZo>_#)bbVw?N`Zg+CC`P>BU=f!jvOw##i%-8Fx zeE@YQIVqG(3@v<))WpNoOS^=}^HMIZD7x+78+gMEz8oQ|=^%z?rE;bOgaq%8nrpIQ08^<$}(4`Qxn?mo|wSRs^{mMe0N)gM2 zOL#$`@?Irlo0~_;08xHi7X@huvi5_10gNh7wF z?fan3v@!0(3di4LN&|6E$thf;$p@V4C`IECaPNjlf9&*U?p+RK}z}UcT*j{ zgdKcXtX$c3E)?95Hw6Ae_+#Bk<_q*63@r#HO}IIVw4~xU54LF8)~B?YXD;3qExY2q zxP9ot{dz#II^a#c8(JnuuqvAj4>n(ZSCq_yRjg?Q`V=fU&ZM6_ENI;Y?b~x@yL%TK zzRd((m#yCqR$s4|dg>><(kGeFuTrlUiTJX~6S;|}G5(e`ql7!E&EbReq+6;GSYtOR zC1N{xYgw=aI`AWwMb87V5BqfypGVGd$+brJg$^ktM_~xkp|bMr)P=*(OoRE*5)li+ zOs92ohP4N>%<{D0l_&hA9rwXZchpiSUH>rQPz*ze)d%U`c#FZ=SiRb{%j+9bH3re&DJH5jn5c+6w=D~{tj z)g(OmRFej#5MM~vc>8_2iZ-lDo~F)_1);bLK+ z!{&Zv#y?;PPHOemgGnNIv%v_wagJu-$J6_8qV*@PsA^>1+F09QzRg@bIuvxQR+S#k zf?Qh;l*0I`8i@NQx=w!u7%|~S0e0zcGZ~JZy7=4HFCOdF>&9cH5fNMqM7Zb zNuOjbO{$f!cs~ul5%*)Rj>2!6(B0NLI~1(9HpvWBQ^5j$*$G|Ib@`_|XkBex>6b(Q zFx5#Om>)Y zfye2{+=$lXK@wpOgKHMtb%0(*9an_Voe7~m%``B@;z90nqMEtD& z9$`VU84y)(J^$XD&&YbJf!9{gO(_7S!-6C#0GY#p*&7(o*58j{w-xR0+7W(45v+sp zjWKM6y5CG-T0~ySBsQ=%?!=YfTa;Y%YY3zua$&rPCRI8;t)FIuaV-H{w#^xBPLNu( zuZ1_7rh~3#Lt7{wZ3yh?2sU8C2f`?aAfykGZWLedQcTr8sht9$n%PV_Y0P@}+R#6I7Vvi@+#%bD`8t{lGO0nnmcF!u+W* zx&-(ZeX5(_s5n(C=2Nx;Z-Gg#dJqBgdH4rW+>~2@k_a%j0x+7J^!u|6W~ymw36v3& zFixXaxSd!=q+eI~_$N}LPt)`8n5vTIs64G7f69RX)Gr{vdwQ2pk`n- zCiAQa)*WP!1Ogum-vkzGj)5C6+;U2he@MDzzb1N)xKww;lHg!i^E31?N%jA$Z;m}S z_^DpO7x?zZI*um5brqLl|N0Hs$0i_{_u>w?9mYX9f7xj@NBf>E=KV02!1l6S5ehxC zSHk{W84>if3TOT`_1uRqkL}TxvNq{47MCh>%x6y6uAn7Bz~ml}UNgqAtL^$j%<$X2 z;FgqMoC3VCenO1Y%a5381a)9G&G;#SAyuox>%1l<$Y?+N|_5$H)=1YVyd+kj0flc}z5 z*dJS3MV(p;*o(7v^-58uPq4LgIp*K{=`rtCJKEcAt=qT!quLd} z($W_2vKDd0k8GoFuuGNMISSJ*N6LNECm17x&cIraB>8%KZ$Y<`>|X7_k9*F{@51Iz zG6P~;rhIrFnyuM9qEl>$?ACp4k^O(-jg2F8CWpNT{(x5!d)B|=hwMLEp(ERvSzP|8 zl|5%aA1bT69%eP0?e^dQWLB3MJidufy&*b>*wi&Iy+_aVo+ zz;d_A`B&Gvf(Q?V4pRyLqWPn%5|H7Swu*=^r;Erf3S@E>obqY`eEgcFLjuU?h!I|@ zyTEeTr?8k9e-rq!6{y}*s32i7&kgN@u^(ShP60H!HhFob{55yb;t0aE7pr+>K}MZ1 zZ@#IzZC(4}P>J@hC-yh78!1=lX0DKW*P4i6ZufD$zqNhu(shRp3rH;I+gMBbcnK{j zuUhHK!w2N=?_^;*Le?e(8i{FXnp9O;h8o`2ii?{F8TTxd$!2l~(84;C+z%F$o&tL^ zcx`H@J$63j_?+ygp;P~!QV{*E%Rd|OenhTy?1>=#x?lAv6dQ1&4%*#OB^C;P6j|~1 zZm6qC2%+jPlTNd|&6@z4$=K{PO9!jzgmf}}7=C(%?ZQGxcgNcmfoOs+gC^|+X8Uc| zsgF1eE@Yha6r7vMjQJL)QPh94ncall_K-S-anu@r%h%WKir{%XP*O=gAr1Nnm4{r~ znvrUNvMT3Cip!=d7F`xQe!QL!h590&lQZjuo^YjH9+<%Hn#+iukoE_oNXlJYX;)oi z_hz=ipVyC+qsQJF{bsjKMmBaUXIR4jJk{P|rBJ)F&Xjn`w@ZHfqLVA< zEu>loDSDYAe{hz;&DkrkT-nlK%j$|80cO8uW6=V3Xu%l?Iv+f`!(AU6(0xUb-M!2o zth(R>Sm{%%V;vn9Rr$cw4%ODHy*9lro)b*%iB#BqcR2 zA0M@qmt|HW*y9D(Ew^}^*1hXDB$g?8_|gf?s^f20Y}mU*TLlj|Z_8)Q1B=*9=ri^7#;H2KUMB^6C=5C1gBoJw&qQw1nSW1?cL`Nzauu$Foo; zF}@qHKz1I4zzQgUih~fM+waF7Oy|3uK|b5q$G&;oVQvn17FPY>}me?4-SNBmAvVf_y6{uuQIJeg+^CFge#kF-&ZW4 zAuEj+=kmGF!m%2bOxbL+bTOfp9#o&Re-7gnGm+rKc^JSHN^eIieJWlOJpO$-wDvbJ z^|ug~OUXMOmZ(-P`43#FwFNF^9?9{?%6ma=`u|L#w4ujGwZyzPI)>!2R@h*R)J^<4 zF8%6|1GztP`E0O|4uFf_lS7&(-T(b}K(FH302nm@q zhqyD|6fjZA&;1g})s6)exi}Styoq4n)WKB{6(=Z#d>1+Xd#2n#Bobf z>RX!3rxvY4#xcPn^UJuX_?88B%+cV!=lsvI>hE_2#bI0n-4_h11kfb(CzbhA8L@<5{=L4uT)R8k4S@qN8bzM5Aw6C>s4Tl_D))m>5HNS6x5S;L`YP`Yl8 zggOufwTih6{*(LDznp206WPG6WXH6MJz-wDlsWS^nuV5lH?9)gadA&LFPY=!bCu}+ zVoD9qYsho?oo?Z3kYNCszh zND>yh4HWs=`sUw%8*1F}wNQ)Xl;$QD2689VC?@OO?lZvj<*hf4z-me34Mu>Du_-u( z`cOAGW->}K5N?ONhUqUfy$b$&XNli)GC!m!l-%20VbN9mjJHCj%9^zl<|^fvj=`(3 z;|>V+4?X$_SqRRrWo%)YFOs`fDC;_`{W%Z>FcHxQ|3O(es5vdItWN4y&REm}hW4|X z+05ysbFL0yotF`^T}s#^VI&K4Ek(=IQy!YjGt=U%uk7 zepHDfg4>QE6>zy@$NO*pF(d(SB9R%5_beXsEKo4OJv(~Nc zl;zuUO!3-f!#UV%tO3DnyD~oAWIK#!z12 zFg56jz2MZWd=9f$68#~$^di8JyJ@cHo1Bg4Kx={5T;sv#HSF#aXA#AP6jlm-|Gcxu zF$((wE0brGb)Y@f;XycZaoYV4JCQ;zwc^+&CpHt&`O(m9as}hM&Knj=_Kbh+oh#12 z?^`FA`o90ceV3rky5~JWzkU#21WMca9Kq>1?}HMXS`T#a7N&11?SZxw_06!qj6n1fN$k?A^>V2h4a?n= zv)zmn=bOB*DFbhT{r*L6&NW-d7af>aM@C=GoWr4G_h?Pjb`z*Gve>J;%5G7nMorv#RV9D( z${@w}uCngt-he%tc-{4R-R*!#_2L1esqhpQt_= z#GZO2xd&3^W}~x04!3$698T;yd)HgyPF&igR5dsa$Z?>r9PqwfezzmkrKq@2I@odC zyU~h`D>Pzv-!trH)N_|}i`uHt>LAZ%!(y95J~Hd-;@|FqI4&U}oY7O;GFPb41O(8- z?JJ?>w_xHgD-lMX0tID_4>tb#oE0~Bx~!h9`Nejl7)Tr%p=UyU!F=Zu9wm#-2Rx!q z3%cE=8RiBMD=+vaL|VkvxZ1l>mu1(g3pmB3a1%4%BZ;DwovCdzUVUcV z`E%G*cgDP*Q?@K5xR1IP=Tg_&obi0Rur%kJCV9$KyH1GdqaJkrOn{uA^rQ@@+y#9K zcj;ypOzCO2%Y|b5~=`?g`F5;5KYEI%^esMRPjCgg%2#tyGUuIt=^|D?oi`d%S^9 zM@kabT2Yjg#DfV<<^<<@o{2RrtW|oerKv`SP-MSI)D{UvzJs6_UrL^uyttZ%i@EdH zJ|9oz1K*8~Pd|zl5dDp|NctN(fTPy;w>V|LO(`&qjg;?JD|787>MaHvh)5)>kEovS(;gw$wr? zg!L0IcQSR*ze!aeOdg-dFUvB7Nouif_yN<^P-VJ8lcwf!Ah85$Puui$hu`AO-03SBHR-lF}J$S_d{HXmm%L) zWR*DTx^j*SB2810dPZnlRQ=*H%tpXb$<)+wy!*+`I=&;czmm_)0o^rdb|YjPVc|zrl{#2N#?)5T(chZ#T?)r=Yzgm z>AGlJ{<&lfqyN=PW+M_sPdZ49%gY7uX$(GPh7)cqW)Pp|iB&o7iG!@K+|rH-g2KL%l|1{H1Y< zv;K)aUe*j#A00}q%&a;~&j{51Ias=WjPdjS@Wk)-ApaNFlPQ=J?-3kAHWVT(ccV0b$&T&ZF<=cU!vA zgdVo7CZ7Vc3Z_g7@0OLFQxJ_`U)B5)^Izm&yc2!^jLrIS*Y2y`|Jk_=l&pv;e`3hH zW+wW}Zxiw>XS&B6+gP;XNpj5}uOP(gJz|Aru}mfrp>H*vNbBOjmR}x5qSe*v4}~IU z$u-(tOrm$c>Qk=RCGdOKM;&SWisj?`@H~A8YvVIzz)v|8dRVVmvW3$X)sb~ok=2mQ z=M3Y*;+JfD3EP5Z{JG!SJ+4SH-}=JS2(AL|rV2e?C2suvXTKV&D%NCA1&(d=nYozX*4Oy{pUBK~HU*uXp9=g3megL*+dYUl6uwcV~x{ViJ&T1O~zF@m>FP_orLcXdOj^+Dm(4pNYvuu_Sh@>pTe-*YlwkJlQ1FlR6zpPXLuO$b*E zu*-iGmHCNP;7ckqHvL3M;CE)T7dIxP38bfQ?Kt5P?h%%L&h=7)H>J7kdo{E|@z8S2 zfNO*wL2EzxCxbCvpA7a2hHF@-`PGu46apGu50l{Qb@6UwOTxN(*amlE-m63jVdoY3 z`oTh4;l9Y}km*bO4I_&WWbDLMu@_IC^)VqY;I7{K8Py+$QcIblJIM7Ebv%FJXSPa2 zJsZvMVOiB2Zk^E1A_AJ)A7xVw0WQP_EvMQZ`XL&2du_hh6|y3~`314f){eHohNF^6 z@X!Lzo3$5gQCXEb?|$w1s+I9}%~18)l^zZ2mCxR@I2JCK-}2v~Zu>1|R7 zBw2#tD(GlG4$kdLF@gX_A`6frQeRx(q`QGp2OLQc)}=dSG#{U|D|&j~PS1FKO^8GS zx%A<7en!unJ)om{bx{-5kH=+H*&VHJ!!!k6f(sTqEEt9FPPC5C|Mlb?8TA0&EKmh+ zze#7`0BgH>Z@jHF1g^3`=`HC%xD$B~DPZW6KAm#SCgaL5qw3Vqwi|Q#5xYmy+#G@f z*{#M;LQ{MI#5BZk;!BMcy{b>j`JGwP<4FCgNt!hGSyS^*H?*9;a?k@j zgdugOD;a@Pa9iDNpFKp5G*<^KcKy!DYfSR^g{Z=R zU2l1&e<|VYPKVRrCCuM18$A7ueMhIa6OwM(AMYIO9L5`GW^$$aX`$Nc?1Y~vush!A z1}wPhnUFXXS7opoIYp|oKB-q2=-%%gSi?1y-Ysu4;fM0%%?=m~Yj+y2wzHm3{n)}! zQCxf*i{whAs+RzGSJvy!x@7%rOlCOUeAz>pLf12#-fMa$INu54J@?6la1KyzakI+t znXHCZe6Q#6Ml42IXNhpVyddqZM>sBDwD#(Zt(MNp{z`4Z$x!i7LSQIQi2E@ak zYBQjc&oc&4P;bZJ+I?<}a6Ftr^XXY0^$}wjGY(Ks?vpr4T?w&(R22}KGPk!D1_^jI zYF`*VbKLS5?CN5Avh@rMJ%|!SMs3YL#qyos`enf;==ht?M|ak`sFwsx&iY+{KGPYH zc{lXaZC{R({rxOGVx`pDjZdM+8tON`k0v3%tQZ#M^ z1dV;tT7(>!-T_YP^4Qi?dd{?rhk1f|hE#F}sVf1N@G8vQipa^*eZBdfEI*HByZp{C zOEz!s*{@oi_OwMN7b%BKE``sy2k@Hi?R@seQ?Ob7l=DtXychK}so zI_obP=I8b?JeTGl6E&5XKL%Z1&~*^sFgIuVnbc_BWPBQ2*L&u}%}Ay()RgAZRT>pY zX&nkKC6n?$s%-%6vXPT@*cEiAdEKme*0jm#tXSRh8JV7TQDI0Hznib&a-lD`mzbpO z60zjTyMk$Aea^UPsE|Ht^_!`65*c<(i)@HH{b!X;_q@zNJiPX z6I|S^Qjv$H52ZXD!O6dTiiVa(-1wCP5@BGG2qNlOW&F5Y1ME)tms98%Z!jDNH_o^L zmjFIK3r!ci`R{N65kPa^E`08V0#w3QLXXfAUvaVEje8KMD_ev9CY}gXu*Ng9%lEBI z1H~AxlAkYSP1?ko*zzwIMm+Ku2pgj%&fbJ9qetB?&o^@CI#5eS?aOL5VxsiX1PeVw z{8&5VK?P{E<)KXefIf{eOZ2^((P90@+g~CJQmXxY9pp8a6J?6%A6ZpC363mM zMNC!Kv`|te?yHpRl`!etR%N(xv9o&5P>2QRK2k*!`#PVX0TF9}8CSsjoKRLXHvfcF zbBegT!x(B_(-;JbyfrjZ;j`dRYaj3uu9kD~1#jl2FutdOfR_(g@wSB4O%i_2`t<~i z!H@Ex>}6BN!*zk|W}8-vLw&K&M>&4>Z%Q(9-fk~C5zyvPDco|X)%3{XGSt&CzuvO@ z_UG6r>WA7CMk4x>cH^$e%OdYx3`OW5rW<^5Ep!a(aI@`>1bHb-bkvU>dX z4WG?{?ksp{NOAy4aweRigw7(`Z%A#YGYMQ_Y~iw|_t~L#uD&Bfw!oj*h9{wsDTcJB z=r*odA@N3hjL_aDP?kn7xoRJ+<(g>ZxU9|2_IbE(~`a|~Ws|5@&h@5g{hp`~_NO`!HldI)UE zkenr%^$mY00G7EI(@+fJE574q(ba2A;4m|5g^$agvkw&f9-v*H-4v95l(UEEgUyn2 zoTBs!*tOQZE`4hC#_#{Q)Hm)GA(aXf=kI1gU6OOS+=vM)COS%BVm+x46JUZQ`@&++ z4ihIFD*8&fhFTJu-I?z7v}Py165A@ncQJ*XCqVJCn|UJ0Pw5YL(2`o-_|mWSNE{Qo z+y*}0zaJQ{JXUJDktewPnG~4PcLsIVHx@@g!Fz$m-W$0!#&8#Ll zkHh6AFkJ%=KaU$PR$1aly0Jp&kEe#HZ&$_cM){6Y7uuhKlQjb`5BG(F9z_HqzXVYl z`#tv?T!BR92B56_AWro};Sb}R>eo`zG6^&(+UiUGf`du>wZ9?mM<)!s?QD*gySp!# zt`t4HoFpuxa?0ed(^xN`2e0?!oUG4%(evXgbfsV=*iJw{FW)1MspN_=$-7971<8)x zQ_5npIwwJP9R!w|K|d_j>SIAv7*{il`0egDpBEwq1M!ctgI$K))P{tG_2d%Y(=2$J z>7+~qr?k*{|I5`r-XH1^Z41*Ayv{WUZfdvu*7!HpSZBL zW4q1AVCATZMncem*}A*CG~+VMeTP>Tfww+tnS(0#SSktK7U z*6@kkb4h<6EyToywu#l|djnNHs5;+mTLt9=TW=NEnDQE7^m@bCjDe0UM#|G`o{_55 z1(UN=0(1X-BX538&GJ!qtkghf8ciDZDsgb;`|dDX@TVyj*`YR*TT~`Qe%asH<_e92 zRl`!AW;`bVV==!RK1~_)gO1xR8M;>SU4P(TGpR0q@$Y%;+T^tE&vnKTFfQ#pqfoxm zF4Boih1bqqT-{$}EP^g<249Z#{KNODf^v|?{`5{8SsH{-8=4(Ko)g1o@8<&ZNb&cR ziKz$H=0Tfv6!9$nsrk#sDxZcQ1Cn82F|bj+VMB_ZZ=Xuj(_4-F@?CLB&b8_wN8_bG z+nl0bUVRjK9XG~VcXH4n9(5=k#~k(klh#VLGGDkbZMqN&iX$ejYU@C|NI5wzTOJR? zH<Vi&M4ERw7*p#^wlRa-%97c|6V}r35}#3 zr1&J=Ob^&I75Am_vqtO?9xU?F@sJ|m%&z?^?7&=D!L_8#uhKtO+;3tIDZ-?vj_k+2 z7%A8@6OdeW+T&%)|6g2vV{}{rn0A~=V_TCnXxy-|ZA@(2w#~*?W2Zr5n~l-fwlmr6 ze&3$mv;Xe=z31F{-}j-68)ce~J%1a8kkXU|#}t>mgfVH>+QaTrwzLuV^hmGiDge)I z{etKX3`MjPOFU(kPnFxNcY{RzCk#4|M}Czfv-8$p`FWo0RU6L26Di8GRpyqN=4i;U z4Zj%KC?Qd72EX*7dLi2;gaX-@9$bR$|0ld^8A3Gf<#^VF}-M z$(`Xu?B?THVpH|goYlO)@jCrIjumMVL$)ZYjze?>wp9fo8x)kUS93@dwUk3n5;yh{}$ z9RDjezS!#$5B96aV-ljz-4Jh;WQwwE(o5}(K%DOF)a;y7j$Cv+y4$AA49#F3y)x6a zAoy%s3x#8St0#${N?cQ>p2L2af-N|`ljU zZ4Z{zkp+671_pzMP{Ad6F5%6OS1rbR=V!|U|IQ%cVPD=Vb8<>P1?E~p;{G-(C?%XG;_P-2( zPqkUlE!Q)j>lZ2N&AH(HfBO6!1svWsak;t{3h0x%Yb>DDxkKp^sj!KLQ)?9}VoPh< zWO#M%2|ymqevKgx7R*vhG^+wDQ@vitTH)oFzblY26HotE8)4+Koawj(2FrcUyDhxG z3?vciJ26zM?yWW*vVLu`!Lc+3G67#(5<$wYU<-u$9>C(%iPCVi!a_u$c`{Pd+BZrZ zLZLp6R9L2(!z(DRSs48et=INk|G(CV9}S)kYv%X0csNl?*Gu$`-MV?$G7LcfX~~7C z`SSu}{Gs{)!b{&Pd~s3Abt8)cE^)4@8~Y_114+E_rr7Zm_0%Uka}6&9`M2~Rdc2>} zoJ$eO1);1=#{qd9&KnE{M^v_zizb4*inM(ls!r{|ZfmNUDwu!p@pNe7lun_OG~Ej6 zzI_c?85V{=Jz|rzH}wnObbb5i<0vgHbW7q=PBqql!$A5%H5?m83RuE|BW!8_JbyGi zqv4Fd!eE#_0Zlc)-n4GK12vw9KXgzDt>X>x7oxVyv}utlvBs;EzgO=>pDCDVK+|q= zark&@*`8e3wvIg&*jG%y>>@PvCf~&rf}2RWwBY$BudcOk>!}O5qKZ5)Xiatj^ge zT{tfJ-;pnQIetb?b1)yCr+m^4@=JeZ&g?s7*Cuk(R~2~^+2T}HZH0aBdS%TNwEBXk zyG=dzRbVEJWGjAFUISmpT#(IKY2<;WKXLG2++?DDl@0 zZF-|pcH*fJG%R;|4U^kR%w#4lQ5ZwjWFE_76|{lHl%=~HELGX-y#+&rQHgK8&dxIv zUxG%Jd6g^)Xi6!0!@|SfBkM)t78Iy6fUFn;bfxsYpBd3Ed5ev?r96CkK09pHGh6208fW*JWO-Vk=HeKLToZm;A~SOne_4B`cD|_958Pz_z~N+6&nn$< zrgysSII;l{{*NA+OIgoTS$3(F#?fu_}*b z!#1@XPHxnY4BRr}NgIgHS?P_LPwBNr)frCf{wenfGYw|7h#EFT@a9IL{b2)HL#>cg zBWMl#CXJG=R1p@t4VK6|s5jTA%&Y;~_~(&tjO<(axs&it4Q>zuSUs1I>NV;^lC0*1 zm0XD@ZRxO1HM{($fdMo#8i1*h2|qx}jVJANr4gn{S!I0fiR0;;mm#>kU$|B0`pOSWfSB{A*)Vd9 z=URkKcxO__<+LRQxO5POG=$!%&OSCGUT66=g{hvonlJHN$MzT4*0;Xw_MhRZk>NZT z)H1h;q&Ld>?N|osbzfIl4kNTq7unASYD~A88ysNImxDa_G(Gq5IGpaFZj3pcCh3i8 z*s`eIohptAyiem;+g>1VV(H;T!8zF$D7o2alb9T|%r6eNYWzOF1kk^I;C|h-u8j=c zB-bCG775q`~N%@&#TC93Kc-d(rH$7*JoX`uDA3cu(laZyHdOEBh|`4Y9l@rw0H zDt$xUq9*ZE8Z=jA`}7(@LTV?!aCWQQ)aTs$`fwOBdJ0i{)7j{A1wFvtF+O*$($wh}kD5D%Q+8MQQwe!mcQSGy9KL{`(b)o2Y2 zpbPkiAz)2t*BJ2D^KW^=djxsqkM?5%sgnwsC;B*Wnn-kBcVcLi51&2eMJ3~!6aG8% zi@-=M@<5jH?9BmA979;Y+|^2+S;FNgg+Qi6vsyrRe!5(vU$G3OYg+DpJ^Y7cmmLLl zQW4&EZwCx=ZLON}IfOw4|wE>f%%A^93MVGk5>u0UilV?M$jT2B4 z3yka>`iIZ%)E=VI`yFU#rpPU-~XDG|>TE^jRqH(Bx!V z3Fk|eNp<9OOZL2Fqf;T_YcmUe%$$ucrf6e>8f| zV#`X!Y^b;vSKb?x^9i+dWpWr+ZpD;R+r|EKxoaRp%6IQ8?KwysZ`=Zu{xy(BGlUcS zZ4hW_gtZo%qViCF5K|<_`Lwt4Bc$mYQaNkCX~=!A>LAH>Vxs+buMtE_V+sTdv7^ZE zI^t0~TOMu`UHLxcOVd2kh7R;?5X09YM$J@j{m*xSgB4w?OUvVDmUNe8;8h)G=>`4M ztoTC0J^}_RJUYFtLJ7u*Gu~@_8ClZql>&hRC5aBqzcL*|9 zr=~C%P)0b3SFuLBNQV2HCJOJGL-D@Y?%(6+&<9ITy3r^6ys_a|YbvRT5}i|D){F-~ z3EUP-vy;mJ%{YnWZm3*wWy+v_4bH|x(7U_dg8xmY9o;?WeO}pd#Lg5h(}H2diNZHA zyYGTzx)Gc7L!{GXilffaWr_?RUbs1OX;`}&Q`GF*ia$%#{#;Aof9hVwYJ;;?W6MGgK5dN^ztt@L$3UkB!>Gpu|C$!n#@X~ z>Tu$wQb~_+<8;&Xz6iwRq=)#TgZs;5)yo37^anSdFtt!d<%K@Hhs|NsTgHmmTw6r% zh*yaMF^R2Mob5%CWW2}Q6^d#iJDl>dai3c$LakOE*KEW9v?Xz{|5@jwyY77k)0=xo zcju)yE5fo>X_~K_eJQEyALC#-gg^*Gj3mJ%Dd7OhfY|za$4jy1h4#HL!dCT_6Q>r( z+6Ky3=cQYlm7fziJk}0o|EhRqQMI5nwY~E$pZQ;TU!U2C%332fRzE8We6eAB)eS0t z8c6SY(y&c^P~mj04nzfkGvPH}R~EZZpVbqAtI~q9CFmT=HqR5#-+#iP!I*GC;2;Yg z@d4GC_6x%1v10bPn!}51^a+R(cjB>{%$c|iMytKICXAtUKF43|fylg(Mu!bD^l&lP zdh;zraH~}nK#QI<34Jh>4#mOe`Gt?f2koe~?w2MDG;6eam8n5gOkGyXWrzV*P~#XL zZ$Y5=mqHTrCMWYvPq>Ur0~;=6Juw!2p0MTBgIWKk zc5EW$V8OYu&;=n<)b8~B9j~IHsI-}qsv#BhpJO(sND|vgY#ZmAbwh^FV%(ZM;vt%_zHiIOGpX8&Vpe5jy^l0O8M(J0$xwhat}G#(=TXHz|M4#wA^aZr`dPlgs->9 zRyjLxP++ak;PrUbuqu0DV5PPyqE6bAA*!#|;Iu)7>(4XFgUz|vb}@d(hw2|JvDHeA z*IBna4&={BZ9LoYwd;%1JQ3Vs5tT7C*1TPM3#ew=80Z(olr;c7^i?5FkTnP{VA zJ)w;H*pj{cW0>@7%-d}n#`O5xBUyJoC8oE3R1(WIUK(-WaHi%_jrOa&B+W34vXe{1 zvV(On{Lt4@Nr%g;Ij<+1?XYCiqatz)euW7`bG4PmB3+b$Bi02kQ5Q&8SiW6j!2HI# zo980&0^8oZC$M&ny5)>uC1r$>Py^^LD}{Tewao z_Lhhs?aOlDi|KY$bwj=^f&*-TaBtIZjG-IiJ{Z#loc=Van26U+)S?# zi3Q-ILVIgu!r`y4w_E9oKh0Zwv^>enMv*k!T~0gTdHm5(NWBwpK0X@T?>51o#2*>B$J5|Y{u)DLRK zcDx|XkbI>G!5!FEOeUeIIFY|my{brfApSQi^pJT(z{;+K*KMh{WP$T6u4d@LF$OOv z1UtltE2wOEDDlSc4uovzzKf9XchmvI9nH1L+Q} zw5=`cJ7Yv`3M(Q3*GQsC2XX`HhpzVtv9pQSH$4c)1@xl>j(hQ5HTRZDEH*H*H9QGR zvn~zI|M`WO60d0CX@USNS}=)%EU6!EJYL_z9J=iA#y4sM@Hn<2VbD}f-@x}jZ0fxl zz!(SbjqJQK{36)hR}2@7NPyi4GD9r#;P}&vq3+Qb+jIi$#GC$j|K0 z1MworX}*no?-+^lBoDQr_i<*JT0$vmvBQ%w=jf>|8&U3p?|$b4@FPlChs`MYqP~Li z0?ui}@WuoIROH@&BjWMnr*riKW$qsxOZK^O{F+m5Lx(dx9m+}4FTT@%%&NDFMlHTT z(m&h4t{*gxun*OJ=fYApgpHbMe#4Vz-yGaNH&3Lc4IC$k?l^iR_eON|*r(93fP1jz zg<#2dcxW8I>#N)+H)u`7Bth&Z@3F7A8nFwxxAB;`bmnlI&=0{JuPrEj;eS|s5Jp)B zNbBt_Zw@!se?YQh4EB+#v)~tWgULbD6@g@7f*I{U+Hj-Xp&ppp0QC19cZ9^f;YrN7z4~PfQ>f&YfH0hdM5HpUgyn~APDonNZG|^!jY@7qjbGe+b}g$xX`uY| zS5=Xm66H7 zl7q)SaRSg|X_Yl1TE%cx46p_l)ErmLhM(8n5cyg;MPP9Mmb3038j)^MdYyOP-Apeq z`o2_fFrE&&bEZK8A5dV)F$YwBCJ0gdOi0#ucCi z$c^(NjMA2>U^2MOygjle>AQ%1CmIy?wPnUJm$4JUk1rn-=uc80sJojJ_qk$1%EPBDp;4T%6d=Q{a{t4SS3Flh$BKCh<_=2}X36?#!BM^OWgO>F|8vgqRjyA)m zi!=SXo>0aFl(~aRwm|xKIORAk7G=0XmZW(Nn8`xjj)eX)sg-Tc&!21Vycd=vF7Y9T z1krjQkbC;~jhAw8m;x;az#_f4Bqe$#{8aTl4!J@0WLA2sB{$eY;4p!H0?CFBk~P;X z+BPop1=Y-7fd?0l@4{a|fmk7f8&IVLC`0gdV{MBjjlxndZT5|3W-Ygs6SH^{sCq&TsG{>aW_|c? zpjvC3&&;`?5*}U}3`)@CigM+c3_8A^@Elc>4%D3JNeUzEX`rJn9Pveo=l1VBX3l8y z#N@7$Pxy8R>7(x7DjjHXe}@iV^oa&ngdf%Vh19gYTG)3*?#3?7eC}~5ocaS@&2t*v zCUm)ZS15<=O1W8585ZK?^u|!Abo!ES82kR~k*Uwab;fF_%?zhK&0lktn~G7x)%6YU zif}=sj zCZjT#u9aU;E%&breI&nYxX13-n%`jDm@bCFuxZOy*P`dhRhqvCJvWUVCEYM0N7(4W zU4hf)H_74R5a!hYtxdP&7A&$K+k|Dd8$q4n3i*{Eyxgx-+Zx`S zFVi{$e{Rz>>!t74+U~94;w;Z2Jc`yLyP@J}39}o$f9@6}5O-=0xVF|e!+OW(4s1~5 z&}6-dG;^s&1{9WF9cJ7vpS)Zo7`w_1#Evi7vqW^bF6q5lF5+glOYETmozUt-F3@c% zE%tX0I>V=~gHatTN>R^s|8cGSi^k01v$#AFxm=M^0FbXe5N*Fu3nM|UZZ?5ogW(=b zR1_d?03!xCbzh>e`qb^XK?}Gw8#~juz?m6y5@PCMD#00YjO^B66^1423AR7>4yDm zM_mGODI@-L4pr%*%VUBK3?4MK8uxwFuLS@Mqb?d0d#?VeFFcvt3Fa?})dzi*a9rw~ z+V}m|;#)~?SA#(#vpO@p&ldXb#5u_BGayGiqY+_26d-46C`+XI_G8(T$q>Pv8o#K1 zKicB7b%+8I+3=W9^L`LMLHI+{bS;sbiZR zMEf@s3N!>Vg-Maz{jlk)eqy)&qPA^zU$j1s!W!Bt5YHxe%n7^gmFT~!W|Ru34s?Na zbX2D%o4qp4nNEBnO;7xs*roc)V}NL{upow}6gSm@IHk?bcnfIT=AF-lbG`Sk=E7AZ zh7X9bW3>^D&yr}BIb@DK-l67#;M0^DQGz`@({{ksT*DuxE5lRgE<{R~q_FsJ@aEao zq0-Rgr)_H@^n!XosQqv5=Zn~udV>e2)`@zDwLcD0dN33~HkPoAOQzm7FOcHD+; zfVaBN@MvoL@`@V~x{$c!1KHAuaNK#=o`L~5p0l9a4vjdLvwpA92__jr=S8FSHxa+z zbj9hBMOU=1+%P%f7oTtC+l^Z+yo&EqKGz`F7Xp$``wkqAKWMwE`}yDEhYrO))8=62 zpL+lT`Ci$ku>e`BHL+L2X9^i=Gl#`!;8 zM_k<$_c~u8+;99F!H@ddpIw_QXV(e$z~(}nBN75SVYWBkIQ4ZS!jl|qz6e4UJTw$4 z21!`jf#T7ppg{NGdb7#5+N+j3eQY}S0-|qw zuAI$`r#*|GBj!(W{HsiqQbE;$THf#8WNN1g1HI0)>a-vL z1vg}r)|dWlh(Q|IKQU!{SU0NgEkFMHwf{%51LjK9T&NiIT zXe|=;>p%GK9XS_1P;pFbBCBKtB_s+`5{D9v3o6N=C9@Lg8`5DaVR#|5hQwKac`rP1~qht`oB=c55ZLaTf5VNoUh zJGfNx8>@rCuIa_}+G5^Tj1^SeDERUwTPQ~f%!ML zao{i|7>JuZ+`~efg!ud{!;*6p3r19DxdOC4qMc;}EfA>CFfG#nNPA|&W4mbgs36F~ zJ6T&Caiv&-C!8du;k!}|KpY0-HXI#_beZC8@%{q&k+gUe{PXm zAA2NtuyG&32iM)K4&u_MN`v=hJ8z_FjvsfNn($npSY^B+QJQiLv!~ME$6RgK+-++Y zhSrk^>Gh6)uRB-ba%N`3P37sYx6*KnQ)~i#K>~|H3NhxDvJ6TP+SyT^*h>x_+uuIF zvC(g`{uO97#eoie0Ss~jhxUui4(s{?A=r5wA{4F$F>q+fD(ukbS$5m&a9eK#TROxJ z4aYe0&Fg~RyU5v=z9X9E@72Lvf1MEMa9ggP`$0th@wc@avQeWx3D~u{WaB?1lw<_a zfjYCB6*8O!<39SF0lQkmsid=s>0PTOol8 z(OAu=KM)76g+S}*W#!AsN>LEP5FLw*GHIven z!XGvnq21Bj1lH`9MzWe z0GgV9&jdc9L$g`Bnxi$pb*QOqNb9~fbm(f`F8C`*ZPv)GATwq#So4`zI$Xz-; z@K?8$8T&#)-33O+M>Ooyk$NRNpXChTYBJ+k>g9ht!h_(8kVAGOmbrV4z&%N*>eeG#vj;^G*{R25RtD1yFihFf3CLDJHoxXHbS&P`+D0 zxJ#x53rI|H{GFhGnZ0Q{F#4iRrnYh{?ZBs34X2%~>~@`+QhIfoH)WWwO>s3vweLu) zOmXksyk5R}doTES@XEE}flZxdSJ)2bpW=X|wzl6^ycMI@oge7`PJU4)kGX?H?@@}) z*Ay=p^*glD3O%X>P4*YMEEY3a)t6tAnR5lBKgr4mXLbf7>b{!{@FxT+mV(PTJ#_qw ztrjxguJkoOgGJgiPI9fXtqvQXWgK67DL(UqIu$hA?sBZQ$%!T$9U>~GA>*Xf`c5T! zp7bGxU1W{d)OA>8tF|DG`RTkzeFKQ#B2zf3l0gE0RD6LGiMCPD@bM^sky_x25#cD`QK1)VDv)eer97vLEJb400ueXq4h z+=&Xwcy^la8blIS<_{6I(p!_vfsz0cqR9IUbwN!A;v#507aP0_9ggOBB zRwSwa&S2$i&NLz*uze1wqq@DVhOcwn}! zH&ed0tczWXcl?Qr=O4U_kBQc?LW1g7S?V}Dvaonh8eHDkA$2fHIOHSzPyH)sq6k3( zks@m9v+9s%|CZsUX11vvc#^{NF$8JOLM>j2ko`8`B&kv2Vu==}G%} zrm;ZEE=^yQ^J?{a=Uc;&LBLxwsa%xSTDT-x;2-A14k;AtFT|G;9GFa+n_mhK~)5*_$v#mhAvk+B7xNps+73(tkGeLa-5>1_T8G z&N!}u59GiXvLic~dtOnfFBrK}9sb@Yc7&yMM9f12#0hK%(;2 zNEL0;{Aw_lFnUg7W8>N7w&tsUPr}b9hZE%~yh|&Okd#}3t*Vbkg(h(HLDkN#vzWj< z*I>flt#7s2Gz)~HYoqltdC_Ag0E z6&A)js+ZHSmCj#|lR|DLNW0W@4R?YR_Q|+U z%0Jp;a~C1I?Agjc@-f99Ee-UK~z7DpK z;c!uEf1Ug@W+VhBL~(q#IW%{_DO{CM{rhWE0OXU;vni3Te=IR(jU5yw7Pu2!+j&9t z=x=>sE28X;o+Mqd4`Wgd;b=_YVhdGuL^<$_mgc52q89hQu+$0SRhtQDf&;{$Gx^SV zPgKmsq4LXz_)<1~^m*!BZmHf%jAD3AwCNCST-Hn-MT%A&$OWzY323!LE;qF05ZmX{ z#2w$JFARw`e=XD?C^YMjp`M*x8txpKgmmDI$Vi7^7jOO>T7^||sSsd1^-7h7>69p4 zpV@jiA;zCuRWX`Dm8t;Y_XUn`GXrb)XQ1S?W&L$N7l}4|x{3HZF<;)SI-BMf&E|AU z{Fu~0o9aA2IA-o*65aOyyWNr6qsBlD7xi3!GzXHvPDxZ(kowc`b}1?1Y%hPn;ZMiw zFS&n!YkOGA(WKOlIg293RUGTyzSE(!Eae0lC6i&v2upsRJ0;oGp&!b_Ad6(BmZv!{ z$>N>puVh+M11Y$5rEeI)qVR-Xc*%%grMKcT6EKuC60ebLT-A{ax(y?+hfQ{g7UvT7 zfRw>=j)iYj+A9xf0(~#>ZbsLKt{M$xIMKKuT z{WpK{5GMv(D7G;T--_9O-W36Bm?v(WIheZ{h|j<;D4-V6f#tSMr)awP539<#VS>i4{(?-vS#+Fgk69`CTs zOMXZTL3KrE+$(_AOjs*3q-@H{`nN>j4Xjo>f@?$JsN!^81EDoky2{jg+{V!9CcR^p zUsHKW#Cngp>aOYKIsOejtlE_=!iZK;3b%!!nGR7=3(f%O>GxweSD~**u!uHYV0E+X z3z|*hr=n-XwY5T7j(WsIyAGn5T!fhQF9Jkue;`K<8O-v)Zf?zSxIo z(Y)~gh=!q+__V^(xsVYzD<#y=eec*&jiyLB>pIC+Ais?QmlJ89%+B$%?FfeoDr6+1 zSv9`O1Pi6u{=7wJ+eIss8W|P#xTgu;sQ7_`;LBYy>>o>UT3b|Wm>wRoIe%yTFKJSP zznjGRpvB<6wAv!m9fHrcWw!pAAa$xzBqVv2WlTbbf>FLdO~78%{U5Xo$Hq{fCN@B9 z_iBKlwUEOc;Y6IAGR0;x+92bn!3ZYN_;8RgjFICE#6sQh_H(4baEAb>(C!%WOm8h- zMeWmp!#*Mp-N_KVtnCfQB=4D~h8~m}_I5dM|4iy`ID4iw@nM9S%1~~c@4Ihat?}SV z0iw4Y;JKU%u}G^wNr!VqxEAwymM3hpCf!zj=c7+oi*}%gwg*Bg#HdP&F;6%qs8!To z%krbe#Ha+96fypa;jIu*7IJ>DVav0HECFj(3US^jefq1AFWuz22HZ~s9K7du+j=DJ2g@*C4ZHlJ?c^5mY7 zDB%HpX67cv*ic~#W>y*SxpPUQgLY-H4C^?wCfLP_kvs=(c&?%!laJx{_FUie8*DZ@eBqnV*JBr}*Jj)H>3-X8 zhF0@*_a4+f7~HCF8XVw`=bG^TJUMJ%6i5^sSno(Nd(-raq^#fh z9>AJ=>?(1G6yZXl5|+;gMsTPi2xTsUZFPEQ+;nHu`_m6f6`cam;ZMoWp8?_GXi0Me zUTt|E zS+t@OG?B=;QG#^C>QckZ1`IP;md&tA?5_Q@#IeHggGIe(-KyrgU(Fx8`L_}a<87Gt zLy+UQgS8*Y@gC&p~`cj>w-kWhfl^KP-mVYqfh!5lp z)Q_EV&H5n5W}yfr?IJ~(Y8qcv-P8B^A?9^;;D`c_%R1tb5jpzl5JuRYx4-|J@%?I{ z{}{k(^$5xf+Cn6^G3ovF_5mpSqxfby@#@LiHd0@_&u{kb2zgV*Yl8nd89s}%7<*S@ z1@*`I=yTgKz>0ukBsV3LsuROHHVpa!FtR!x(o0(h{B}f8KK3@zcEE`U%}!wcpRKr< zKrE19yR@dz`kV)Jigl)D##!tuIy1!x@@Mgj5qm0g3al&JCmWN>__rqPg(V&C=72o;5EXH6e`n5c-=`0)pWI2P5RV;}Co zr#Y!Hkd~TT-KoD6c5BIpEVORC_>ns(qPVvfcB|1RAxMl%(G!>0*1f*kh)MKo6t9et z;%w0YITnJ@mvrXf=L8+85D^X?Wf&4|!Cai63#OsS*d@n?Yn-gTs zb8|GLqo=zR?A96Pkg1bO8hyzgW55p(qthKP4cO;Q*A<;w+s;V+)DeJ=?f10@G_7f3 zPBb!{&B<9%YX^}N%UNgP_G8_J_ned=9{boEq%SCOi+pCw3&&E`@QZu@CF!g4Kwj@z z3rAs#AB65Fx@z*U+HLv1>G7(5CCT4fO}}R`Er?L;(E3KuW>stCzwajriO;w)&2ed$ zGJ?$iv=6uuM)&xNO-=#Tor~CV5+;GUHD}Qcw-paS3u3X+pN;L3-r72l!J~jrto}(S zi07F}_(uz;9U4#=RQK5*6IsMU!yiJ3CoZ3O_X+(ao3n5ioiczsahqP3 zl>s0JrGGv$IbD>@k^(Kd7mM2PB1D(zu`u7CGhDtBd)S{I|W_l2;9pVxzfw zUn)*=|J5){IZ=9VIIl6%z=#Cf@px7TdPRFFE5o=Sj$R_elfhn@FsgK6(dxB4J(r!~ z39(25oC$LX#1f)4!zPTSxIRb(n2?;k=%j~?pY>$58zd%T^^$yz+pU6$m_NlS0nL9H zLkg+!+2X{0GEdN8@qJm5y<*K3RwBU?Fi-ogd2at1{Sp7Zq-W*AAiiYu|0(1DhX4u; zee%ZevqZM@6)&3sBSk&OG!8FM;!mxL;C6`8w}b~&8#VBgukY$54ARe?gyb+G*1%-$ zfA~b4>jO)%rQ~5UT-raYrX=MO6iW@k?P^E1ZmMF4{MACsQx0LN29yR8C5D$nbVn8) z!Tn98h6q2&6AM|jMxttjcg1|J-H6+);ePCT9G`hfDmA=}g|*g~o(c@ciH&H#w^Y*>E*qVxt*r(M7fY>^*757Q*36r}aEyp3`((PSU&i$fIY z8>)ZxiicHOH{q?wQG1LijETPF1Y~;|vC2t{#X8l-a53t1_4W+^3zR@I8xe;JGHP^n zVbcs@>e6`;WobFhGL|3u25D+W5SG&yYG2F3EIMX9DDCi2TBJo)#~jE`Sn>1}2p_!M zu}e6`Nfm9YY=TiIN7iC8;JG#HevkwW=7#}(3#7U80(3}yr;%S7x@!{~W1w0v{S6=} z7r5u*f4Mcs1DfuL7;F|`MwBVRm1wxi;)LD(z@Du8D_Cco5Rs22Bgp)-Za_IXOmAQ| z7I$Xt4nki&Cloz5b_o3Kb7Td@cKB?S)`wgl%!Px*oOvfjdIYDn@= zYU(eZ%jx!0J!hnLqWbsDZ71+`ZoOurMwU;7IOZF_pjaJDj$QB6c}2Y^k!2&gd!c3d z_L*O8Ex%VBo*8F>lqaTNEih@cS8OwUz{TH*y_a%CB>2uhWFn$|e;|6$NFO2$8UubA z=gV@|nu;*e2-C=0YlVccKsRny8(3cy$t}&+U67&f2VLWZctC;cVq;!-58iC}rtAs( zdXdfRjO;M!_$hw7Pxw#{;u0B&e{P0s>jQ-uDG5d!#Lv~r>7xZ6aKJfV+i2YdH)-s7e zlc?Wce{*_NO?2?`>$Knn8JluTDpT>hn2nr5#c4!#ppjBZcF{bw0Uy3+B=+z>SJc^A zk9h21qYkt{6M_O4$N^FG7%pUJMDoFLwC{)-*9Bhz>+S)@=7PFCUksEKn4g0l&bzC- z^Y8SP{3SFpFqAQvF~l?)yr6@JlKFgI`b?YoIf~?Ue|`}lP%vJ(N%4@3`UsDo=`dXa zTuDQ(v(c%5YB!(MKd-LlGSG6xYGKR>hD4}r06DGQ(Ar2dzr-v`l z!>A-sUomrc^Q>X~G-xF&LOWkkAW*{tCHj|D|Ao})hC!IFO30AM`;g4M3ZLfvrA+>& z7t;tF(fEcGyEsk$JmZ^FHTlf{WbgIxaW*}o$0`e2i{xsM8+$3a~HTLr4fr6n##Z>6mRhC8lC zNJhX$s#Cn9aTLByJ!hy(z@-R-DhyG@xa}fR2Ipiqxuv?jXly_P)LUkR4g*9QZ67fU z`a{ar$dGocDy&b_1x%&ctqs$V)gPOCnT!tf{`i2iAR-P!@1cUCWA3&r`$8uxlZ|?{Sh7>X{<*3q`2p#Z* zG!!(VmbZP2zu}WzRT49=z1#w3rgZ8*NBwsS6I4_ zW3^}>DD$`tzim^4_@Zs%OdNsXaVq2y6BsvkdSu-mKS>w|+=(YAi;9zL-VXzsT#eFI zI%d9Ho@8msh~f8}A; z$@&@#mRMj@?+C}>I&6|_!ZP1GGBYUaDw~Wt(_nn6dk$Vd160FTn_lyaqqbOW+z8P3 zg=viN0rVw@5K%}4mq96kiHssWQ51LzfemTH*qwo}r1RSjgKBygFpan8hW$59O`pPjID#e8pfVgFo&s0Yi6RlcDcr%E3+9wPG>T1Y z#J+RZa!IC_q8C4fMD_!6%Su>Ac2P_Z<;ApQA~DqxBJf|RbmFX@C>t+!D$B-kchT(S zqCcYihnxY+N{qyBsO1+EtS~*(t=3v6yms6?@kH>K$J_0<&!x_#GGUD`yd<;5FhhEc4jPP@j%-AkTJVA-McK+Xt) zoX@I#HKKEgo!_(c^Sc|8(>O7a{Q@T9`$Q6iLyNem`b;TL?lc;tL{))Ae{WU!Re|S2 zfUPD+`y=$9moL-$p~8;TcRa-~%||Pv87%LoK7&dGm=;xNSJtbow+88fyK{%N{bW6Q z)8^inWVBJ`9}P137MAJr{Ta`1MA$7zSOh{?8A5w&-zhZ%6Y2N;!XwK>p?t?S4DVWJ zn+X3)3dnu)%=q70fdAR?!03-XyMDG-Q zAevp7Jz5%4xzEL?bBA7gkA$(iPcS`|K6Lmu*&&ac1?|44Xs1d7QzwiVWs@a39ph)s z^%4-pK&;AnI5)m$7l|#4{!WTE8+9dScXvg9t8>$Lq@u5zDBrP}aNi=<1lrmR>X1p- zxb7ZIcD1T45KJQ1A2FK{5km>5j{U>#v2T7L~-C{h8ai-{ST(T zDypq6+!l9tNpY9r?oM$i?(Xg$+@(Nqr$~!C{c(4vKyV3GNO9Mj`*7~LFY75I$xg;z zdwz3Gp+)Ns&WWdqGmOv~XTCK?-N=6MuGDG~O1N9)5%4#O+F&MSn7 zG?P>Y3&T-NcxupTEU`tgt4qgUqG8oaq|QFWLh5>xqOQ@5P%FR3$Dx0yy>9#v9 zsx|$U-CCN%V)gi~)}Jx|)`!yBj!+br1TXFQ3Y?DZ;Xl-I^6OK7{3N;lt`0A1aCxdEIltDT#gpM{()A|k^f}6zEkhM0z_7NKj<7$Iyx;g*Ev=)7n@$Re zZvSAFC2|W&{0g@vd_3sI>#SJ`YiKz&EPdgf1NwoWz9fZ3`zhMf#VPQ%q zU&_)Aag<1g<`Rm2Z&5RF^?q~MlVf`gKo2~Pg=PjKE8DJks#*3cQ33qe_8d|*F`45n zq3)1~@5{thunacj*^L#PxFjgN{NN{zX+iO)#OEJi2Tps9b9Q`^qTY8$@{UGTUcMq+ zF|0|PN`OeS$F{x+a^&|z!|$k{u)+XxHa?#sN40@g2WAwLIePHAHBC|;sIA@BKrh6{tk1&%YrHyf-3n;UsDf%4qGmRd)$1fiR`knUe zGM`Cc#-B+H4VeXsAyjH@H`?O62ajn2b~HK3G;afVCS2^_OsU+|p;X zrc*r^K@*9n_m!7N$dBmDb<+bwt3A9M^WW%AUak_yDn<2usmqkSQgf^^kO|2vg5AU0 z7s226;wZf0I(%V>E&Tlz$lq9BsAbC@Z=9z5OLmssNtKU2M^nY_X?Blmd47{guo?+F z;27-HA$GSFYwYVq|Gbra;WXD+*t&{BO@)Gjf{Rjyww5o?>&Mbbc2U~%jAXcSAs0q4 znI7Xle8d4%lu{&?bW|7Oh=zBnFYEF_={oDAMpo4g3;YB4%|}VcVZiQYyqtDp| z_n$4yS|Go)Ax=UKPLOVBV#=wJptc8EAZK)#t$FQ!>=XlEHO7LS zNs7&`)`QLEq0!w*9lUVfxz#)4;pHz}>-hgJ$wIB;t7h0#wKHw87f+^EyPeX zaZ3-0%q5t8YwGtERf$_QkV2QEl(7am5K9s89&b|h$~po`s%j`pgQeiRbty*4B*GN1 zC9kaA{r?fZ2aD?$v*dym?J+X(Eh>zx{~X)g}0C z18XW2^{reA_qxf)fKg2kb}X+!>U+i|GJRCHg(5X29W}AV%)~~h{ET$F>TVFsfFib%0hkaAN&UcG;)Z1%z#;FRjmygU5!TgG(_q6GZb9@f$|g?ru8INkdTBeT)<5 zjZz5`7-a%aR12q>F`Dp#udMG}e71FKgcxYJU)xvoQn>PQJO0d({~xNU5K04!r@E9F z!JX8fk7{4Ez@)%zCh;OM?2EViIgKw+DxVVWn~)__VW9^_|y+ww>$A^?;E^~GBo+lJTvE7 za%p-^ZX|@F-rOQx|H_kt(i$b8dG7qTbgq@ofa4t_$B6ZO5juePE#VjoOc049R7=P z1VGKdFBN8N_DWXLOVKtpb29JKkUH>bI|G!OVRB5Rh?=v&9n~5fHU5uYcXXB~Ay0Z* zafph$q)5`>fONVzD|WfTzY*etNcnak6c-366N7CQa)|Q&@-~Em`>>PyAy)L36bgJ6 zck=!xzv}wG_zj`uH2-dgzgqCL9}6C@`lm_j{bP;C%zx9$%zZL z&@wCxAAM4AcjbXP0r_{~aOL^{ge zrq!CYI;=6b%|VZdm>h+I~*h<|$o0srGTQ`6k~sOT@>(oixTU7atjPYZ2H9n#*C> z<~h^O(FCjJ&iA2WI}w8sQGb#N75X5l4U2WkixZ#vL)^==olumFKb{zkbOx}h@Y125 zSTH-#a5Xb93Ql~HYJ5os;m_rE*-XStPxBG$FfBm~0P8?s$#;Q#bNf znum|dN!l*8AC6RaA2$`lIsZ(>Dw?r~o8h`@CW%^rzbe%h! z;gMe$(@;-$m0TgfqM48vkcrA17_EXNB{mZ|V?)_i(&0ezN#{XBN1X-&Kp_DAD)FBp2VDx!W=b z!FI}a-FwEQ8&_(;(eA^%!2&yP(0c#b9$>tvMS@*u=ZlXmftE{mo*>wj`)NZr+P)eX z8#*U>Shmcti@*AhiSjt^lDdsk5_RFF#L{$$HwcIJb|p9FpW*4Vum{E!6lt4b*VHmb zsBi_*gwC?HpD(%}IDk4pvkQsYwg0|XD5M}-H@tTI*|}cM7aU9LS7X&l%B!3)KC?#M z0<{xz8{ls+GX-L3Zy@X!{pX$=PyT?6w9x+Mgd}cLOoWUG)S zeVX-6iHZYL`HM|KOq#d01Z*xrEu#H}P%3K&Mc8*PiT4Q(oHSLC$E14Yo+n&#ol(&y zr7uKiI>D;bF0DBi(wtfZIizNUol4BAw`}8P&i=wxM}i6w`;*cYi6z+ETlVz#6gZq5 zu5DZ%M;xy&1(Xo>$CX|_5Yf(q4zxeo@%DK;Fuc>v3<6pS{Vk_BXQJ(N9PMpOD zaFLj3NbP^|$<&$O|Dvc%Q05@h`^1LR-D-eO_@=x$DVC@i-#Hz?0a2yaeSebESv{5$&E2aq1Em_Ed6^|T09~A z(oQW683#}Rev{pk0mG=b>wocPueT55I?V=Yr+C-qvDaK5|5Q)vx@UZ6>`ud>!7o-5 z9PJiM3NP;VGwj|vVLCY`GY_0Mkx!S^|6Q_%#!-_41AVw=^(3V2Xko{uaML0RbRaFB zyJfZ3%y1L=!wZ<%9=>2&NVQM|fEL}V%xW(odN+&S-cfw8SMZp9qsT$grZ1-cl)Avx zKWe2S7RO`693heFKQyXXJd`v;J#QIq>3OjyP|quza6WQVRVBHSWHghe{LRCjo>z&# z>U4g9MrdYFl%B(Jw#2x$LT(U;m(ZN)KL5@UsW{Sr;D&uvOakodD zKRuU@bAwKy*q@(*P|!XPM?Tr-2f>822|zu|3`B3C_;hBlMg(V;1WIaX>}C{FdWR+j zPJrP8Y@$J@Lz)eLBv@LMzaIv%RI} zz2z|~w&$zI$89KftUU`OdR+xGpXp!c(~e=6^!dV|7yG`|6y6oz4QTnZaTv#-m(>qk z5|(<*N)H6(^oU-(R1silKDyX6d}+LFT>USmwAiYO6modLBYEs#Ut==ZY^2 z&1QXichvmta*a=e9?Mm%X$_PDjrL@1^zW?bY|&bG^lOJ99nfZMvV>dU)`Z{CGJw!|RlLp&L2n z7(U;SY%T-o-*Dpt+&{8KLoW@kqiLh$pjCfTw>i^Ddj?uZUfoWLzvDmbi$ZaBcwwq^ z;m}_i6xuGBzXwuE2RvG6`%@0tC8}B44Fy29{Sk+dXERH>Mch=b7|P3*Hy0^msHiF>Q*nc7nlDmkANLlD+g^Q$X=YeQZ!qn|BVmX z=XIiwk-P25F0oIOwZmEeJ5|EG9vvEgp&VI3+}Ryo+z|W^;O_-7oaoC$AWRSa^8S`{ zu+zDK819ePZKc##}L~Ajo=v#M?A6TfIR!q}UG{L_ypL6j)iSl6{a!D|UW)bv5m<9U{tUty9#Er%&KaF}hJ((fRp$07(PHJBg+BSD6Jf3n4y) zok^@?ZCT#PWJ3I;e>uuBUAXEDaw?^@E1A!y3?=>OxRUU24MxbJ82FFWAy~Mh@4(CP zNJtK#uR(%>SHaS`2*~-D|)sifaRNDbh_)te3`Izp`z`MDkuFfMsu!*z1b5XoB0b!&=#9PyqClanb&eE z0u1VU=Dv6B4r`n86BO6{Tkx2*@Gu?s$9yo%4+=;O0Hy&t`b6k44I|$u8mt0~ZS;li z(E)D1QHgZ#8#(C{&=!9$K~>mn5Iul?jm!kBk~ zX`?kU7`ri|d*g}RxK3321ggnWq z820Os2tkp1iWuCgVI#He8G^$KFAXh&C{(JEsghyk5gs-hU%4;4Qneu=j97KAq_KIf zyHkXAO_taEyef|fQ>GgUgkWFuy{J19QTOJ?Be`{rzbSv=v#i{}^Z=+8JSf{ldI zXVD4Ol(HM{?1-X&J2SfxI3(xn42n!hP#9)qXW0#5W~(E5QdD;icA~uJGwm;J%Wi5o zgwAT>i9ehgP&3}a4hrtxPd)%cAH;^fcI}iIJLsferWAy?aa26Kpd-*|bx2xng^kjs zqnw?{OHI0bJH%k`K+13T=>9O;uOry{Oomn48-@rbOPhBrA=lyD_o4aX zqP4$OE33^BnfFbdEz|=&?bn z|6#oP473xB`okrTE?>FDD3dt#&0lR7u>q3(BBbVKXzi0cwp+XG%L$&HX& zZ6|x*z|+fSB&`J~lV*xoU@9$D6Ag%n#cwkBe#Da_5X!u7dsu7@t4hSgjQ+PVg;~?( zu#?7F>vOL>T0jBtcHO3B#B5e#wu-UnoC7TPOCt1MZ>F#61tm;b(ugY^PFAZj?<(|g z)TQ>t9z6zwGCDm$I4PeO9$HgPz`*e~^5nSPh^RS9$G8y&1`zeO{K@6<3yV2Qt>{+^ zIHeM!`CI%s{gKW4Iw+;DD61N8Ep)bYD^_ib-LpvW4xdc7fq3oFz>C;&<5=Y2@%y*q zf})3&*ul<9@1KuNW6E41M{t|=CJmvT{)Ec9^`rB+4HnPS0#_3&4d?X|x8@fD_6H4l zAGae(<2Q_n`rI5dFy9FnMf4ZIpkA*Io~#bn{nNU9(CeD7^P@F7fABT)N=P@Mlf-t+ zDseYs36H^uEq7w>4bk|c$6HtM4ZorI6G02n=fPJ$iT-eW#w(tJwyWu<7N&-OpH0Ip zS}=68@J<3W?LdCFM#e68cE@=$JLz{X43~$qTb>uLd;D2^ZqmG4asF~+vJxs1H??O3?o1VD^lczpRx37!syF!e64FeV>NNA`|*{H-=s5dP3a`5-Qs z6Ae-3vpOGyIDNL9ZNuov#pe`0kXSEX3TO;(0~ z0h338eJm)0f3I+XyvhH-PXB!ptH2m7jdSa2r9+?FRcxEQjpCxG9rcEheJo6hgVs@Q z4?V#AsrvFNVo7k`6S=rf9F@Dudegd5?OU0U;u(!yvmO@X$i>E>nmudA+&W5~=7qSmK`w5RlN~N zs~kDwxq6PEdE4{=lTg%HUE_htFJ_jTE$E!}56;g1Uc^|n|3$o=eco@W{NZ2ng72O- z1K@};s%rHEfCJ|;?ZyU1rv!gPM0Jy1NtS~8BAMTaESB zMF^ZnvhFL4yMfz6TN;5g>YyEWq9C%*R##CCob{_O{$7tP!Ott25F=mQu@&Qil+hpl z*9+U1IpY49*n>18=eQ2O^mtO!_+yjtMwkD%cio3DgF7Jv+r__06UTy+u;uH=Vy#xq z6&tdRzydJ9Y1zrsB!h;!{w{oh{hlVuy-2+di?z!J4@XjjuVRKktf!DAf>=IT2#}`f;oE58vhAJ3Fm z9!tlHhl||rrdNIMV`Ie>OU?y35*DOz@&utWGD*D+i{EtVYnAIqocNv7eQsRegUJ0M zf5WdKTByxjX8cb8U(%U$;PU~>WS5CccY>oTRHp_`mE(&&|KqV zn|hNqM7Nu-0x{CG0Od-){p?$u_ef{Qz2du<_aS@m9nnp;D#^omSI*Tm8gJ?jCV%%o zx8G4cXB^NnJhy~lp0#$a7QmIhp33P*FHE9B_SWzKc0;Mfv|i5DCffABout$wiqYQ6 zM9_cl82t6?v5T9u=|HaHXPYNIIj46W>Le%r3$73HWxHP98LkhJ$9uk7k|)F%@8a;! zj>R9}KMGxEK8RGDeg9Z5I+wA?7@s9Iz3mM4E)%<3-_3#gPHA-2c0M~j4C<}yEk*B` z@1!Tfv0Rw=!1ff8FhBV*rM=5NJzTH625lcO+*0xz-ltAGd~G_<5q3wtXX^>eQ0?EY zAii2*J>&Ylk<^#=VUqsZpn8>{wUMQpYVbohA^RpagWqc6qz-z=9}9M$#B2f$se%Oq z$+v@ic*v@PqoXrc_9qx)lr|UmnSObXl9{GVM*#61QN~~xYe@;A6d9`8cpbLghB;A=ek6InesQTdv(#!7O!CCD&rie z@vIWD!gIo(&V8ib3r+zc4dkG#Uzzu#i zv^Uy|xB>NKdgpiW5^Q;>tIc4YVV7pTLGn^@t6>fR5~XM95&XVPap{H7^w_!hymQ?IvlnRJZ`^Gxv?lQDTyG+E=+fVsGfrV8B> z!?XD9L)yl(NFJya#wG1iWZfUPD!+a|ze3L=c#R!m6hzumRTtqPQo9kQ8Ef50!Fn8$ z@RmPo^>I2iSRm2;jD^L!Gu*qp>h8X!6WsaCY5GYVlPs|iRmtSIv|<8bc2e+2ORVwo zj=|Y+J+_YW)6g-w+CF#$S*|m1p(I(b7s8NI-ympS>}GvC{3}0F1k76#}1PM%OYm}(ZyK&VaL^5Z2vEMocpP8LOqY4 zPj<{d-EhsH`R@gcNAfdP$1l>Zlq$7YRxp=L4O6z9X00wdi#h#XJuL?3&r}7@q6npt z8oqF@RO7jM0J)>}zc5G0BQdv#m_C7x0mgxvy064H50}?YpdDeLbBpPyJT2I0j>-ut|E5lCsybwZI0_}N zfr*Fs{p;Cgbx^~c_k$Ycfe^f}$~Ib23=elV(mpH-26yR0|LbvOOJ`HXkvOX1enAsrm|^p>eg z+4f8#VmkYM_?WMQkmc|{HM1*~&3n98pL3=$QI@ zm~%i+6n;Ph7u`HRW7#&Csy^rUud_QA z#9wVq<1>69iYrT9-y}uHz`4B~`w$(^$U?qW7Yx*+$xdHn{jMkFVLazR=4jp@IS#>X zXC=W^E>Lty4o70A1nk-XGb%|O&n@U|i51-4>J7(YIpcLrU* z+~}{hVnGpa=gY2M&wUR0s>N^+nEgtB1`h&d18~MlxQ%H9qw42rgp)oF7SJ z^)|l?1$$q)0L*-mkylMeyB(%*Q~uKMsT%CMYYZ%Q=vmBFW(b0Ui5T^^h8GS6Dsr7a z5}+HECt`8^MmX${nWRUpMtM2w+u-O?!_^uiW24I!b@1(; zp1xqP&~|7OZfsU)DtM-fD`@&G&i*q(+4t31@Sz=%`2~ylaZF7Pmk(-0LcAm$jQV$q zR?~Sm5r~?_O*$B~1sVA`5nhqbJf$ z&v+<)G*NK-vW0Ws*I6)`v}GHQa$7?soV0fIS&p&E4vgvSDvm{OPgUJj3Zfv0PKYiQ z?VN)H*#8&a;{+Tf|8BSf$HadTDd5pJ&KC-IV(V1tXiSO9`exPR*xH8ZJEn6@keh;QQ@^cv{y$2QM2N6 zsWcNRtu+b}1mLjj@Yje;o(_wTX7okl+ka@gpha@Q6@R(l3%@$CSoZvs{pEO`-FW$t zq1^Ws#M3tv`N(;A)Y@P#T3_Dc!9A!Iy6i7E7GKd=cqKYq+=Uqs(P)KYu%wh$6L2M@ zHO7Tauq;{o6Ro&62&pUYbNrYGuk#=!y^(N~S{fBi8bDr50{KPHY;d5GH3yn^{Tq{? z;?bH7LmiX9@9mSXfQh$gIy@4ASeT?}999BxLi)f-?~VKKL$Y!GV!l0i(vPwrz{fW{ z{86!xPV};o%TXGX*nR!`BYB9a51E3l*;rcUw2#~F@adU37@mI9&s8wmA>*iDA=RRJ zq>cvRM6cFYwx21&?24-1=F)mi5@HmLtFQvMp=UB!lqCY{AOo$!KwP@RqcgH(5@k!G zm@Nv2GwwLRmz_?aj(wUGe$%n|7!jaED7mY}PM)$pXC3NZBc}RrY9pE4coldknjQ2q zVE{4m`9j7uUQ)4oy^lO| zd1~18T~TeyzEt@qs9O+>Wd<01tf^{g!%|5yh?rAi;9a)Jjy0zv*8s@@;+y2S`WLf!l^`{*dfWC*p zej#SK(Gf7xs>an4IAk+j+-%q#y6nt6NR+Q$V_watFKmbI%U3Y6u)KOTbGjio`Do&U ztX;PE&}(`FU#b48QPiULUif75^pNh94!kh%BXXNI#$c&*|G*azL2vQ9n{JTvIexqr zX$FGD9>dK$Amk0uKLouQs9YoVZKZ2S_bhF`tCa4U32L|K3G1>}p)~VQz7Gb5d|b3{ ze^7GSk0UfTsqwLVoA0Ee3?Mn%*+y^9Sljit9>GkTm6-Q5Aj`dPH|}3)Q{W1m2K>~` z<;6aoF4ZC|iTVweD3F_00S_n?52v}QP4&k9H+JMAQa+J(_3z+X?Og_D4a!>Xrw=f| zD47{Ci(Za{1@HfLT!)N)Wc?Ev7BR@^vBZRSa2ij{YG~cXod(cp@ik6s4eQ{yb)u_= z(ER4p8K^KIg%sQfzUx~}1msP(+;QjBQH(8^Wy-vEk_}1})wEJph19NFm53qz@A7`f z9bbHR}K1K;=2=(pK$#Kz_BhbkWEgT}-8g^J_#Tym`s^Vc@e zZ@2O7*=QgKxb*nZmPi^7wr4k{--V3{d*g6Nxmo5B^iqBJEuTv{<{4C zf>sI@z(!_!Pj>Rku|Be_h+;EkNQnwca9A6(EE zh%fR$N!+NXu05!KI`tQm~_JNS-)K|MeEV%E4DUEBCSMB{EO zHWz1H-Nwh^HNt(M5{&7q6K3E=FKrcP@PRDsseh46H0z{1 zD}&fyr6`};lP{oL&!k*l_?Oh8!Q4cNsQ{c-Y8W-nDvKGS|ISI?dQYb4)4S-kBulfK zGCyZ~^{k-zw)ihL|NW}HDghH%aq19zrFNN>giR#*$C*%HzbRAc`(NxIA43^U-sir~ zMnPz1wPN9)oKh_xVwONR6Npb6ucR}JjZ|{V&7b#|Iqz6h06i%Ap1i}M;p!2mZb z$AJ#ae!t}Y?p581NWp^HfpMLIgk`ItvJlYPD=ohOAWfRg;LSzLKUnwc=XFje^Rv6S zGapXvD)SK`#hq4U$6_eaA|PDyuwFBmw03h9;Or_KgKg_J8n=DZaClD6=Z$~qRY=7HUp;$8VCW30r9@~crl$#^ zWt#H0xsi8n|HcL|^%`sbvZc7bl8WdRjr1NRf|fk7(5o6otWvvCwtgZFSO!8Cz4V)y zG)cxutkN%l~J!=Yhj!|H1k7rtYy@2GyatVrD_`-}+R3d4KG(_# zdcTf!^(|FON@g{IUAocL?*gTcgvBD{xVJy2wK111U5LbH&X8X%(cArVw4!P;I5@y2 zzItC|AQ8382pvwb2KP(@(>B_tNSe3ocmqHPIoT zB&$iE6SA<%R9{K;)rlKA0@}0RhDV_NLvr^$GWX0goin@tuqcFluzoO3N7*(%mEg&@ zv%(24W3k{iPvO!xDr37e&+pSid5zwaMKG7<7pla!Eqd0A`U>SdeyOG!1VOu{l(C=E zR9l$z9+>L#@7MrR2e^hY59fQU93fAt#lINdf$EIPBe8kA6AeKzdI^ldOm=&rFW8sE z7YZPzu;S}}O$iLXc)QNNuo17kCYK><=N*5HVi&;}4*sIydiEXglxi%e-8x)U8Ell6 z>2H+4Txx$p#2;q)G3xn#yYr(N(Bl=s`-UeX!qRp*XY?}?-Pq{?=w<_V|b&fOlR@{ zJj4E!YE@&%23Jxsmx5@xj3J7WXhPXf;(98ZF!oxf%@_`G;W)k;!;+KGsALp4O?+(c zf6;YLSQZbJ7map9%#ignP|F^tuZw6@A$i|`9=W>JS(Z#rlR0d6 zC3cZMb$2fcf9-m*y1*wXuNz(2I+BF%ACjDZBb2>pYxDeneINusEuaQB{mluBO;8N+ zlMWB0jELtX7?Pc@v04E|gF1t;v07i_;AbQ({lLu7fhjyy7np3(pc1zFN25Ge!x}-u z`9;ItA1C22iHNgs@ACnt_>LzIK7*=)XSB3yW610;0Y;!^DFGwD%Xc__|{np_{V6&!ddhH6X)9{8Jct&)8>Mu zKc2N}7jByslkW?c%f6_y%X&O~e(ym2(GO71IXPtOk$ok&vF`4@YcIY_!rQ{6{* z>SFeYcbCxnNW*rps0%dqJQ2CPAkt9P2M+RT(a4_9aZ}3`-F<6t?7x}V%f;OV6p$u? zy(;nw5%evB5ZY_4k<;)N~jW{C?%3 zQYeZ~y`0!sN`=EW-!gUs>OKDnv(&X(Je=Y)_CXVxYQ8!;1ZujX>s@~o>s|K-xT%>% zxzk0J{0kquR7`rb5|&TS3s={@7~J+@YzkTy0Ij}$x~~^YKI-V$TdR5=`thXUOf>Q6 zm$U^`2TQnI10L@MPv0o5~$#ZGfcn`e$JQO zbS1uc(OOx)eMd3VfY29PN|Fe&I@H zhrBo0ybx!^TSe0CqV=SiDbfdG=(P|+%zW^>R*j{Xg3NxnHKZ+?SAa+-`k0nqr>;5R zQZv($Pn#TnYO_uX&IDusxgUW(huV&SdX0W>LXf9NM(Bem2^wl0rB_s(>{6pHf|2a>Y0_Y-ZZnu10G04|->GdP=l zU)}{aCqC#dG?YhLgsb(aEWF1UlVh8H9P;||139L5x7d%H{AT1F(m^fv0N=V7T#?m} zxCSW8T$i7^A8lbp_fdY~!K)SieG4<7i_tdKsXt}e&b=fa4W0P%X+7i7@{PHbZ7aIB zN1nEt$InB%%u<(mFi`%UJvhZl7poy9H_EE20FdxRZl-=uMm5vEdBMH2*Bkqi@+~jJ zI4+COw_4gWx|CKP4El9(alZB0q%TrJ{2>B@nad-Wgm&_mOjh>qFF>A7f_|q6+|}oC z1AdB>cQcR}cl6nkDxi1^^CBsPh@KffAxr*GZB*vZ*cC^gt256yUv&c_G`)Jsy+UOxXgX9~ zDA(}LVX?j$O(NtW>`3X~O4vF+l<-yUMg5Z^WSuciakMz|7eek>LK+c2Z|d?r&54wd zQ}4L@_cKMD-!+e4t?q?G!rigSEcvd9xpa!f`UWFyJ)Zt6n=O1$uM&xcexvLTP7n`f$%_z$u~{QMAV+A_ zGkJI?PRi?Q@uuO2lL6(GnzF_byFs))Ln>9s*PBd~QEoWE%Od~D7dE_v38geRI*sgP ziLk|SD}Uf0#=$7T&I2UdR7aflZmleZv|qGE^$NN{CyY1u1Lqu{S@k=Oc?Is& zcPR^4c2jH~-Po!?tf6t@PAU6;qAqbc4ZM)>1Q1aKcME`2q~zVE6{osWG&>8oXRbrI z@0f>767)B^OcqZER!0S0gSLfNr_OZ0X3j4yYl*gTFE{tjH{{%0tclBT{!2#GZ?tFL z4h}+%=@Eu)Goi?0Ae^O5t4#fRv>}8X8jJXrqsWi1KeHDt)nNWS9i78%B_4Q3&LS5` zWDqcen<0K^EOYMi{*}f^b%okqdtg=!hHn9pdfRVyPFqiQ+EJDS9>H#{ApIwvdH{1l zWha%hwv?)$+<3{Uv{nYGV+l<9)N89lTd6Zh zw-I}WMo8{ih^xAc;_rNLS^}!zCSu5)H`Ef{_)k28Uadb5ayY`en&2qKqQ`iAq$CzK z2%%--Bkaz5mhkY97pk9S8n_pDE;*J-jjNX3R2yt;;SPOxfA!6oH2$#EwTt2=-E{Ml zPFsdG?ZMxSr}2&>F3DK=BI-Q>f|Pe@+P`QNV3b7ulM4SAsq7(@jEJ+6xWIfb^4Jo( z*+4Xuy@(S>jxtIhdTox~E$ojHbN?ys)R^y9ACFo`PU=rGjY!y_;D9o*=<}Pqf!vAp zlVCYEPLKEt#~TUn1Il0~nT$(*Nne|L5hI)bCrh%Sz#r`SUQi zG;J*El963(R+MPMVP3f~Cl25$7Pwaf_tgVYP6^lYs_{|vA^6fF3GmJmgFz8jFLt(< z_~Kq{kc5>>bODq|WD;vis z8^10CVGLSTGttErpzKS~eA-odSzy*eSk}TtW%B=dB-Q=rfgU5k0_BwT3|FTNYAa3G(Ysl=Eb<7i-CJV`2TcJNi}jK`lmJwy zG0nvvno6}c^3D{z$GMqDF5ZO?Hvxske#38B$gW>;InE^olfL0eYh#Om=WhC9uP(|F zagyZrQW0?5rJ3Rg`!n4WZ(H#&r7)fR%ct;3)-QB0K5?RLol;2gS`HH4{Gd9QSD1>G z{u-Xjv7AU#hl3aoM?fcB7Pm^lE~|ezFzWk{z5HibHqPNn*>O7zbVI&{LMf8dO? zQ`YN~B-jp~VSjl|cTuZ+E8bsawSwsW!0QYD8IiLeUPe4<=NHEDPae641QB5XpNxPf zp$OY&h&2+~LjNQDC+n}6dVLLTo$qic>muNtE8-NO;;0!Hfxk{4%$Q@kkY)4%=CoYl zYa>BLo5|&Kw;YL>15wUu{x3sVh^zt4NF_CYLl)I{l_@XI!oMjwTBx+TtjP|+YWO8! z5m?<`o(B}hM>cka!GrfwfQ1Nqt^ky*Uicc7fn4F^B=Q4+FE!yz~O@MMlmMfoq)>%LGhr}v}wEg2^qAGRoU5W`VL zj0rrCDQy7BTV_C=wgze*>9FT?pHfp0dgb_*s=P!yC-iMPeqTX_ zLt(I9Ve@C^CbzAwjHxq~H2_3k_t1whpiPwCVOYVLB>XMjY5wyVn+*3dq3utKIbX_n zO_?Lhesn8M&G(fC_`yU*dC&DjlG;f|HjrK1xMG~#!lR~BJmO_Lp`O?DU%;4-L7XFn!gnk4m$KnI119O1`$a7GrW+7UvA&MWp^G0}wkbx!) z5n2H>zdR98*p6y;xt8J>n9Woe6cF(ygh5w^vSk$-$x6)J(Y1iVU$sd_o4(c=@K@hq z7S_zwzcZ^-frC(%_o_5aUll~n_$i8eUf8jC%|_obi8*hh9rm&RsWalTDASr(;&yaN zCTkC<{Xx&?%~4Y}&iFS)ylM{oJjC}uT7Xe@blr9}hhsy~FGmEPFHy~3m~xZ+x8=2C zXGp(%mzvX~HD5?9@Ladi@Z!5%H+V!<%KlEBqolN#o=l}%iC)Imm|*6u2AX{>8qE1o z<;)VT$j$0i)I04w+n8||H?PN^tB_+__a7N7@X3w7I8q`lD$k}VDLu+&KOQi9grmwc z3YbsFuq5&lDk{i#<-=k%W4#e5%TA~&o<`4ys1+$tloBKYZBVL|4|XilwrTA_i1O7d zxnDhq5^73^ol58gADBJV0z4un3gCfxKsL^s;PoTQaQ%p4mzz`!E!cpkeCyWjn^)HC zCt})hr%c+Pm55O`T)ES<8JLs~3WIrG(QH(KIE&dJ72UAfZR=h`Q6B@rRQgZt;<>wd zI!LSod@X2soJmIX1rwKtHJLsT&UzvK*`0C^8lCuW@*3GHIi9i;2**Ns0cojr7Jn=h zGD=Np&ajR@lTKZ4d^bpJs*IBNX^BGI>e3!$9TmHJb+5 zjfC5U)q|9zuru)WY7-rQ;bG+0iI_XhGiGi?s0_o8xkUW=pk4j$%SE~XVC4nV?I&K` zxjsSMIV`eNJLHtHZ+don|Am6*?fPYJ$$3dbkk!(6AOhZKMGyhB9;9TsVru*aze`z+ zU#O7qnwR)61dVkLQkFuwju)iUoor45Yx%yow=!gUN8?| zpcY!jgQ8RL5oez@O>f7Fmz1^dAVE5YQIoBpJC2E&G{e#4TrC8nS?_51P0 zcor7kJ6sDUjqi_~<;5_vSTw@uapgTuA`0*0a#I7Ji6%SMwZaj)&KAC0DguD%OAQSk z&W7zcl}oz}^qZ8zUV-3TFC-u!16tper7AnJMq2VvvzY~8&T9#lKJGA%e!&sp-R8!} zuW{*A4(~78rqBJ7o<NE6B<6w4D%kqX@HZq5 z?mP@d?e7SJm}qYGr89}h2WIv}6)Tmu?lQ?#O|(ocuy;K>>Ci3az3?sc_RXy~8~If5 zu@N`|u;0ii2cv#9k;PXF@7o2s_nF2fy9462a5kuJfE7sFah;_#FfxmrQrAYwxTUw_ zEBGK2H~Jp8;RN-XtrS|=t#;Htf@tv_S;l89ro^ojSlG;W1;{|RV+;61Y}jvD!|)jy zrZ51MDIy&EMJwbHx{JzgChoHc&=XudyQvoZaP+UxL%BD>^GJO=k2+wrJ)hX5H;Qj? zL=%7M_oECxHPHB@~X+c9JO287e zP@(*MW;}mOZe@0s^;CBj3Fo%ZkTPw;hDwCCR;eq6hgsHEon|Ll#XMaRc);h>w~bz1 zuPG^klYPn&_hvo68z=ErV_BAEB}}gR6r1s8Az2se4xD{>Mw!M4K|Y#qwD(#K>v0dQ z1;#+1gYlGd2B;M1($3hr_`X=boi-1y(iznszDk};{=KD(F=>_>@@NP-CoWhr%*uYU z239Lp!KnYp9(zf;&APok-wm0y#x$(Ow-8RjzD4~^DkRR2hbNLuLJ-^Jp8-zekmcQF zn!hL#vf?a=0%UpRrzQoAeeQC#k>*--9eOuhA2Tr0gzjWjgCY>*~%IUHxSe$VASCH7B-7I)j$Tlr* zj4F{=%reg00k0JYi1<0_?e}{(B1Mz-B^zs&8QIcQTC*E6lRf!#5c?Fi*4P$W6z}ku zW8CvNWOHS)!c*YiM$s6tqw572&sk-o!}qd|k%y1979u4u4amraGI5ijsLmZ{A`mIj zVV}M+h8G?F1Sts$`dLe;parv?K_0>V=}uBNa;FK0nNj5;mYQWXC^Le8EL%l8uF*Ep zs-#@m*VH<)?{TO6{pyR?i(Q%sUBC29?iP-1nGUR|Nffq02%GV+=dVk-ZBa)dbGqjd z{eCnj=;~muIlxS}7L`!6OaF~kS|fpoZM`71AXJs#A`U+fSsUVKTWvVSSJQl;hmpi) z18ad&EKfp98>qC@0{Ini9IA5d#y-sRF~)p;sqqqge|+~RMhR$7D6!5FLD3kNU7z|$ zq4*Z3;mQ@EvR4XYpPcjmLh&9Ep7me1P@SzsR8V}xJT8OZ) zMO~Ab$=UwB$E5!6&xm>716t%l@BGSn>h z)ys*WHUXX_)@X#A)6xSh7BsI+tPjTF3o+0p+VOpQDQS4H==I_y90-f8Q7J{~OY71q zrcDS@(Fzhfob&u%voQTaGG(el8@LH!$#WtStq2_kw7x?vq9?2jnU{xUy z)NIdeBrl|}gEZ{On$<~gOh}^dth?KD(@j-Kg`OaU!A7k~i2f=!QDB@v^Q=JDcqC48 z%j;)axX-U)m=u&;sL-Hc?vg@_4=epxySop}hZ3#riBFf2&dRVCAMv10 zW93KoMtmyUN_Ft7-qOFO6ogT`{fO}#tt@peX%!{<1e{Tr>O;ciq)afC9szerf-gXu zji~BR&Rfj~!uH$`Sa}RvAxBZO2RFa`t|{K3Ok}XzgkG+XL0!nLL5zVsvpbEbZjUcqnnNJjOc*7fgv`_a9;H-YXIn!g*1_}+$Vs%|M3Spom!zRMDS zC_%Dx@xwPs#Kz@|{$Z&6Mef|RF1Be#y-G)D31y4_jb-NtEL4@0H;2R3@s_M?lU5eh zwHz=9{!`qD- z)gpq)!+$6Xs>Iri#L_6>+6e>vl3mf>NsdJD`$KGz8fJvA=WfOZ?+cVj+mUBdnd!o& zY|{_5&yi!`&zq9;iVsk)d{nO?9I)!zZijVS)$r03W#W#Hx;Irmv}Jvh;?Qs9$g~u} zq;108OF5Aum2?|+a$|W{PyG}CPhsWa)nekd9(9o>>3SzS8*y85n-@G36_7R*mKc(F z^|LSNiNG`@)BFRrBHJ5*-5D^9m%1=2+&TeNG(MwGT5?a!Yt$!_)#OddFd_<3yrtyb z7vgS5?49@60oph-1(yp77z0&N!*D{ABd`II$2fP1#81)j<@WFt(lqcG6k_zTp$*7` z1((MuojCM(hVC@>_b;ALq?C6k3yPjmgMxr1TBgJY990<&a>#&_+z#JoWbACe8(Sb# zYY@{69Mj&$EbpBaX?QcSaaw3%$c|`oALwfIi9W(i?JG=rf36i-KZ9JjwR34XEY@p@ zwP|3qf>_L$I>X)3Z&JQ=0gawWWw2j;!ke~lChU(f=I`jvgQkuNb>0^ua`(=!jo^Qe zBzHocgcai35=ft4 z@sW(bj0Zh(R^0zAZwO*b52MCY=?5to4;ZSG7@Rm3|v8`mtLE@ZtIp z`B$fnIln!^QPoxSwBQ`xYsams_d zCWD@#!ag^LkrP^91x0V&m8tHSU>RWkc;wm|=EF$yll2yL_507fI8sIYA+}Em`_GrV zbE|C*%9Ynlg&XrWxKF43?(4K3zU5E4p`QF0t-Wgvmw>I1iA!kC@`DnToicBOa~`VQ zh|4+S${6+hp0L6(i>7QbVV_M@KSNGWnWU3gxF7`B%TC1UK1a)R;>LJkr3-Yn3Gtw7 z(6^Zjn<__xX)}MvIvlB#Y=Pajx-Pbsx;q6eJIOHDToLDZ(3Q zM6uX)A~;3k4Kiuhr^mt9DLHP>gl>mN6vXcGXRA|-<9348T#9UH6CJR#*38$&d860O z*uUS{wKdIW2GKuuAktI}!9>o%*Ra~uqv%IgkPv6SR-P}a?LnjTEG*KaZ^f?)ogl|r z^lAft8iPgc={oUvwOIXj)eS}UnG2LS6`M!HeV&BJl)b*)Po%!}oE=4ua36BYh`M3g z=KcPH!H=s=ojFdYobAHGTiA z+93}814(!sn`M_KOzLx-?nMTh&KIdOw4OMO9>h3%0gFLw{3IunzoK$6d5s5ls*eRX z+~8vLOhF9<`^~o%Xe+IIq3@C7mA@LYv;8!8ARW+_BPN7@Ly}P@Vpo^Ej$(&^2t7dKz<2Q+Aa@#205Niqs zu4%QWyL19h#ais&=NXGdwNm;MVeaF%+f9I7IRQ=r2l zt7w>Chx}(`;`d7`0zPj`i>d5=7aKS(u~E~2^{zm-9T9c6CBu3|kOemA(-8f#rF{Nc z-tjFz)M;lDsyU=-460#JM;mU4mXbw%F!*$Yqp$#+OdFoyYanMiFThu4Ev-JriPxi+ zu>F{iLB%)2V`6UeJucFl%`_p^0EXQt<$a4HDP#F($C}UYWa=GdyaH&JQNgiEs5lK# z3W{dSqhF5=>*`-l*WaAI@pXamJ(K7Mqb<8w9Tj$1Z@?n;G;OwIu_?45sMzp>NFSCCyH#0R5K_eP(1F>{o0H*2LIyAfDFjKw^1l zX~imv@lW&4_@dxy=cc@7kIJ|gYYbK?y{Je&H?Mn?RTU>T;Jw#3BEgC3( z4a?fEplWhIUH%nvG>`AMK)nIKnn)aOOC0<&$~@e8UJ9O#y9RAL{$6H$t7>FBDOv;H zbAZBkuqqIzmA8{u9#=T`Vz7=-f=dK>eozMzTX^x@PDQ~MJ_iF|VU3dEZTu$ZOwN50 zu>4s+7=H9^lK-|YMgP(lVYUw`I4~(Dm4T9bEr2t*8Pxtyf!K=h>ew7oGx0#zsO`3o zu%(04kvedd{{pxeV>U&OU)pYHrfuYuz~l)4G|~9==k0Z875rei4#_smy#qS zKkkY~r(aO`qYqq`AwsEQeHB;ER|%U^@YWJkI;ad3L* z!oQh~Y!T3b1>clVgfb4;!gah-1ORzR<5|unAdz73!z|HClzHN%7Wza8(DdY2{033p(B@7_ltfs~v^D8)${Y38l;gzIL< zBb^pHMi~44Y!rwWVV9y*xHw z<_=|*a(CtZ(<~2xy7L}8{Z+^7V>d062qBIyj`k)Q))h7L8zUPjZ@%PPxuib|-cFU|==5i;W;!$O$AUV^ghH)x;)$qy3uho+!2j${^B2Tl| zFjV>{-{V6lle*v`2(TSK%((ZYaqi8%TodZ^Hz|G&L-?r{u6B-^Y~2pp&r|zOWuxq9NeEFyPK`rngI(_?hS

rwc4lzlq3+06IBT=;da=%TgGei_i(RYD@xAXW70kYrw-Fvp+f%f5WHEH=ZA*mPtx zI_w^G@GA9!^b_A=vFa34{G}POwZLPVZQ)`i3ps)q3A$vmH-f?J4$WV2ZSz2IEeXuR%@O zCnuxXaCBpzhsQ61<)6FhTqkL@)0M3n;@V>A$*UU}B#h}{tx@6&sl&c`99SFbUmuiD ze;LfR9f+?-ET#iTn3h~;(~th}L@r>%){UKPOrudhXiBlsAD#e zT7>(djNh~v&;UxQq2g#^iQ?gBWxNC1h3^4hXJl>Li_h3hYcz`L)4Ms9M+I*fJ&}LU z0V}H*O*U$z`8O$|HQCY838B9WnHQD2E{vH+O=+kdQWqWfBDpL;ukq!lw4uEKJfiLP z%Ob$c(*P!KH&keeZ8d7~{ItKNqut_9E8=yKeQ0Q4wQfx1tO=?_CW&=gin6P$@6#vd zt{=3dFdE~U^N@m9*wT3^LdUR3WnjiDkr!E)4JsOI-RVSrlzF>4A)zo5^hER4(j+oV z(N>&{3B8TSIRd@F(c)ZYHyHsi~>(Egr$=kh7zApX~M~^YGsCdfGl~sCnz@Ep7&$n(3T2 z_%^n2W2I#RnyMGx;_ivrB0KeV@5&$z#|b37U4dE8)7$}*ROg*?{cQi1ym+4#!!G@X ze+P*tcJKp>ltl3JH=Tf}Wo-^|RpU|JJZ_`HYP?9zf8V+M-@uJqla& zvDUO`0P}L`ePN>14*&O;!dQQNlD%%YzHxqQ|NcXoIw@iKRV9J%E#HpStUeN#uUn1& z!R%JdHcG*>Tu2!3S9_kM%fnOX%q()Yj5AwJ_kWnzlJ#e{EBu-iBsr-d*jhJEqF|xz zUo-D1f~HauSm^PEVR`0j9y-_W)>q34I1TA7Z?K6*-nYgKU}7}!HkqIAlz~h#k5x^4-td|Chavhc{V>MMa0N3Z?9MW%;TiQ7jh0L5iL*%}1UtSLZw4N}(U(9vA zz(>?W{mg7?y#BDBrFYYgDE^!xcwW`0N$HvR zc7SLq2xig)%|&Zb*iE$iXoKgn+;e&$Z~sLMRk!?M_`NL`&Ll9lgNYKIApd!0ta##M zHcx9;32&JWdEwCq^)lClchWQO1o_%IGZU@YL`X7TZ0SugjL0}iU52dRMPHiZlTk#! zsg}R2xne20kk8at;?nZhDA5GR$i@PMtw48B8C-pMq7hdMl~G3 z<{x6Tm@MoE-9q8SdLNZutDrS*<3098rkp(RinT*C{Gfq=p+EB-=5rJp{%)Wz+#@P* zR?*1{b+PUr;Nq|O+nI=6!gtW?QfJ6+Xj_@og<{)gMn%F8n-#g(*jTV^0!~!n3pjVs zEjmRi;U_W5;PWujo6dn%ap>zxhkoN& z@p!qS+gQ;l4lt><2{4^+(MCyn!e{K_WFCAzen;HQq^tS5wYP8O=5SSQM$g@l5d!fT z>S^##-Ib3rWYZ-|LrKtyC=g?4f*FrB}9ZHG!eCrbcf`{q-AI z1-kg0%PXh8w9<#K49Ws=coJ)2S2KFg77sOv)jRXMv@{^{2ZD3m1O}Q^C0B_DB5lm) zmb62XRw{^nK%8eOERQ*Q_E4#Fah?HC?VvoL)M)Jm%`;3RzvemG_7uq2j*7W<{QmrWYwr4 zD#7w34I;VyO@qhTl3m(Qn$+no54h$w1l9L6?NOR@(zC}>bv~tS zM@$bm&UL6I88FM7FPdPd>dn&5(@%q!B|zSFv?|72cDMa zv|Qb(RK4zV(*mJ1U!W*F{>ig*A_&UfJKg)Uc0^#;R#vwA7nI{CgBt!QNmg81e6>!A zJCFdBKrhxP()@ma3Esdg^u;QB171#13eRw-lESXF#u>a7}uG0iB@Laqv{ z4y_7ORosOZePb!Xt1zIf=w4C0`0>K(UN`_2Z2wi+HfxO*J@>)w=}qIuDRk42O#{K( zll`(v#u<|Q{6kkJeNR|My6Yj_@_;TY`c>B(uyS+JiYhTSVkC@(xhNNBBT~Fu5Yx(& zRjTiw?lKv{EcGAKHig|?BGYnHqs{H6j&pC;J&>W3^7yT8N2%=`Ykk=-ngEVm9=?Nt z(v?wiYm_L2cyvAj?!~rG+aR7JH5NN39!AH54_cey_#C-v@w5cP9K9EDQhPFIEVFSy zT#y?> zwO2LP9EIxu`aBvkXRseu}G^mQ$tJgE=fg4iZ?F z@dVe@y5U%M#jK4}(fLtR#?IH*VAqWpN(%m-y$4eeiwwV~?=b$4(EtWPjZhsLxWhx7 z*{@uXceeJ2^@Z)@hjQ*IA5|?>{3Sqz8YNLy6U)|-HZ2X~6_HXXJ!6B(?*KirEvX%a zQPUq^0YxOa1kw(9SA{*o*?CQ?gXXz6$3K}-Eus^^sTnZAz3d|EUInz*TBw#d?AXzW zokH;H)lXz`s})_$!So~w)Iw%T&!->=$Zs##du!It$;c<_&Ri34-LAcvJjq(HyJS_u zDPQ=_j`(_Sy|!DCDN}BTpV$`3RKn%nvIeO|fh)J$)Q8-XnQA zrKRI)rkaRZRL}3DTS0~NWhZbj_H+Jxi znX%`;#z@heF4{@?^QR|FRumLB<+)Af*yWkqRT{jP*?m^LOq_)SlDt+0<;sf6Z|x~Y zO{sf`U;c9ScKix;{(IC1@N;ZX20I{MHv{(SSg);!R;-7$oXwf~^w&IxDqR6~|Hk0t zGKIA%>Evu3gM2ge@MC@CxtzEeZ_nM=nVca1e6IB|4-v@g0&PNe>Y`XH=gj`WF-i7|x z*5&Rmz$Szoq1&KL#Y_zTNf%<0DME5F$m=W*QFz>a z^rHEW?`utnF&U#<~nx!UB83~u7T)e^$`PEhpCLU$}X@fVSwuh_2wIacX{*yC3^oeAH8Qb;_)|1(a^$F#ux)=fO7PI`I`cjTvgqW|EyF8KPY z6~rl-V(yDxvDd`Xv3;3~l9-1sX)KclWR-R=s9-`QjXp&|pZ&TDknqRj8R{t7Jz^K8 z=ge`RWveLdZeqq&tMMQ)EdX+f`hfCW4R6p+M=fJuQldAdGmHeEJ zvrhS4VTa8H{)?JV)-UFZ3<JCFn{rwj1Vv-Z2_y%QtV9HjzAiWkS-5qe_fED9YIO~2E$t{yF3wZzAN zVBX}B=kcWH`7Pj?f;AhM)75&KbVSM)lYb>6mb%5w;PXnIkazwR1H!{66MQ_b#PU5Xrroqhc-PO7u`;ZqydlK*8 zhtV2wV;g#}m_2~x8x&KBPhg$VdlwQ5HWq7yTvCC4Snwi& z@-@qaKt-)uLcfP(|_h zUEY_y%H|V;F)mN_-QS@n#^fcOnm8b27TjccVH@by9w>x-IokUIipSVgBdT7wnO2M;sRKV_`7asJzaFkgTd^ z@)BaMNX};q{*4fwzbDjkz=rwAA2^id`S93j#w@=#?7Ru zVxMzB{!{A%$KZ?*)Xlt~OzNJRQHVUr=ELZN6LG#l95YV*1w(QJr~Qv_`|R+RF*i7- z@tR{esnL<)fT91qiEyF+4j?YdRi$Lv(LRza^dg-rGe3Mk0tzUJb?XktoRTafnveG0 zcAr?qVTB3mX$$Eh0@&$zB6j4fNKMz)BxDFUt#CS0UQ2=Fx+K{>R_SJQ1zSN^Jd+a{_MIc6{8>pmh7AGh_Kc_Jh#&u!lT zH}cW`us*<|xCqCiXyBO8XxBT&F7e1y=XK5fA0bA9(5vH2s)9zMaVinefcNM+S0|P@ zmCNJ6N#rvce=lEOCnrO zSU{60n^~4#zaKRBB#WFF~63Y4U@zDljrLf#&6A+WBCOo-L z?RqS=s(5W>lBv=XEN;97+3;0i={9BYgz zT$NoqDui3!GB?p)*MaWSp1y&?Ez3-WxWq*DPCwYpyu|l~#nDW*6;X6>KSDYu5%H5I znDpOG3Z9{X3?eg!qrgGrO}IOP38-mXL|-rEgNq0D2M2D~1|FUN_+MUX`6LgxtN2Wt z)h7;IU2#pA3*A3I_dmCMqq-0BCnGZ2_ZAO&gm4p`q*8rdxySt3-u^%l(AzDBw9dHI z+uarb;^O2tej*VOL^QRB?i{yvw}dQ#?9T1%0nu)D*7zL$flIca5^tl*?urbqL!v{X zf6G6(D9jiOZ^zF(w1*hK>(w$K*zprZ-cR5PzTp3Mw4X45UC83)L^i#qKBRC=t$jOi znsv@i{kL857l14Yz$xpL%Iny>$8vo~-b<>Y7CCYr%ALSoM6c_nKG-W~h}u25P3zq^ zX8=+#VcLf4Nbc6!^sWuqa6g`DOj_o}m=VK1UZ6#t8b^|1k0K6zft&boOuftc{Z-Jd zQ;aE#MLn3GzU)1a-CiB1?hNu}LDv)URul-u1kh5lKe_8qe$BO9v0PuieKsrx&Gr4K zh`$Pae+EdRAtv694S1POSt-OsTiA%T6VxSOvio6j!(j3>S9u^QP|)T*C>ke#e4i&k zF96Qy)SZ=G{*Rf9IG~s6@z5<{#mBuM&SZ$=@2SaupmT z{BoQI#qs``xSFBJ%c0Mo$QeaocW-(sh^0}K07`h)GjuKxaZTRaPg+p9!~Y+={tLDR zvq?s8Byf^t1qpZjP!O-sY7gfJDVdj_#NsB@27GoxvheQ z{)Bkkjy*Dmgn3^WnwpAsJ=4Jfn*097!#aylU^8XaIJk5Bz@Wy_m%PGfPbDvEvq*h& z)Jj*#X#Hsut4_(nn=|r?wgk--CWZsN%D)?BEHfbb#9cjZm6yQokG#vq2(k1QYIRQrz;8W;~SNw0)mu!n@ z$n>alV1a$;toC+t>VuXn znuiUoR_h;kCiYwtatfAZ$IXnk5}s4+UPe^VWEuM_DS*gMEFs?qv?VmKY-c8xVv-Kt zLW_Q{2Jng-Rk;8^{iA}A;f|ADg^yrIKho+_&@?>kNy-3#hm`F}2S&dO5*IjRPO%a% z(lrHDPsa?D*7#NCkRYL@$*-AF@^_t<6y6LxXWj70f}PV|0zDpN8_=4+|2?;pRcXZ# zi9k^}4sWKP)v!0Q1%j>iF~EBl#XeeP5}+aa$nIPVX1=a>aj3g!Mb`-I9M-mJ?v~9? z-HfYaF90dr$9xU!{iyMM|0B&GqL(GF=eEpG?yE``va3)wx`TeyPnm9o&mD#*lqD$> zHcvh6k5J3@tPhr2|3Z|16N^{30~ex)sB$EjUPhXfLtr5Q2Q$StWgXn}obVGJb93Zr zWTRS2SDxZk65ST7x55d212{b{1w+yXa;T92-S?C(AzfTl&bJl&!jJV85BbkM)TCM> zQ;$Xu=Ld&?L$pJ*zt>A1m6(Y-yD^WNpXt^WS7rmf!;kcQ^fu@ICYS0DQ#%bY4ZU}L z`baPVGi%tg@60b}c($*TDWtE)+$YXp($Isosdm!Ga^{Jc68K`+lu@1E`s?i~Joex@ zCNk)E(rwuQRP$--^l3T^TD(C!}PpLfxLPWL)-#VEza*qNy|alrtZ=pqX;hz0O}uq-NbW+LiL+B>(XK;XScn zJc+vH1@B}iTGY+}idz@c=;^)KN45U#_mp81A9yCV?;()AU)O0%n!POrxYOBA$HRZ{ z9??S7_HIB>iKmiR2MOWh@(T>mt(R(fLod?Gzk$O?JmUi}>VI8fKOU&Ro07E6?MfZx z6S{*UQNAymG3XprQV!o_!|%4K9)|phmYnJlc(3SM8-thZUE<&MZM)t-;F$BIf`q>m zX>xs0V)YA9CGf~M9^~)mBz#6OkB6n}*k|P)B3$n4j~BfhO3}U}V?s}R>>`aa4zet$ zoKo8`mvUxpC-{q~M!=wQDzQiWa_89FTYqq(zxW*vL0#_J-!)v)e0lQYrqbgG>>>QV z4~l;teqprilX14 zYLP%DGU%6kH>OB~UI3n%vY>6UXe0MLQK-?(^Sk<*&sPWmi^=0Z_I18JpW^2VGkwPp z9Baib&@)6@4LJ;406S3q5kRm+9kKX8iujoAPcg{wgeo1h2WScQ@eUAKXWUN@#`_EC zw*72MSEnf^<6I(ulOCsO38KHa3jG`Qem$fu{o-P$oG;{64D;$*UbB4sl5-3o3ilik zH2eAkEv-OC&@>TdwY}Zx{R!irzvud!%l{>L`jl{xfEr|w5}EU+IxTs^tLAG)RWkQu zw|Aj%nTy)f-O;<_{)E^kmDY*KT3>eZmNVkrNe#lFYk%B#w{psOAMYeUa(1ujE=-f* zqdK+=v&D*1z|7D5BXI!-X;0P*R*K9MeUvdv@Zy?&GM>-B$o(8~EGcIN%VZ5=MhO;n z%2&JXFv^g7v93{(&S^)n{qmV@f5-==FG#m$DqZ(n{v;9m6CLbrlpEsW%Z^L`3`6*K z_dvgG(2D_E?F#{J&?n; z{$6Q%x^0c`9GZXv>Js2S*I)wxG9GUcJjdMGpj6Aemr^5?YHKUf4=N2N|Dcv2B1!JD zw5*_M0JW$q)4%jS0*sN~*XXkz>2*%PxkWVHm+f;gSyYUC)|*_YY) z;I+z6)vD(LmJHlm&o;z56Zh-Q3Z9xE7BTDn@cCx@g_YgY4ceVJ0NMTj|M~V-?-pm% z6_YR-;_gj}Q)ad|aG$z-(k$xNm@zH?6Qb~9GzL}rXOsT287pF6ZAR@#a1~VK2bp4| z)n8ks3v3?Dp)_Sc)*`NMvX}SvV-`)bYFp~Tk5;4UD(g;$6m|vMJh%s9tH5`>HhxT z#hi$HqRGsN5dVz<7@C~@D3TZcgRorj?^v5+ozouT5b`>cY@-p)$1m+Cpo-J_oXoxe zGPcFWiD#7)=@1Q$R;5Y^@T$pVq**_&gE{4PjjL7H<~f#uj)gvJM3 z7#U99c6<9#QEcgxz48u}6L=X^ku>R%J770p|P zvbum5^sYqI-9~)%`2jzGg`*=3@~ZjtBf(&G!M)*eCwjB1ya(L{-Fug!&M4t?Buc4v z5#;SK`LLgQ!T74$@?-Hh%lqoQb#CPjf;eWA4KI7Z<1}xkVRq$8_5%&F;^nA)m4VAo z76=qsRBVv#U8Nn$lhTT#=?_)*m{o49mVo}mk1Rg=eF^jRTiO$tj}9vi(0A=ZeR1(- z(S%b{Xm**594U+C%)~qR^`V=iD#Y#t?{aEpZv)OgPY0yeY_-fxW2{WxocLMhb&jb8x6!It8GpG5YA1jme9NC4TbXUtVzL-8@bEg_z7 zrwUiEtIp3-y%)4N|eI9mZUVm+B^*?vh-?rBFrMbB&<7gDv zLdAiV1;HT|s%KErE>`a5Ec;}Y%NM+<4oY)lJ2mTjF7fPHYr|Ez#8!8CNsz#J(jl_SFs zH@f(ONo$8$+`)$z$J~v`zH+Hc?1JSo6ER$1xFOl>!7%bPKXaqxmv@VH_j^Khc&qP( zf-*Q!OkC{*^m#9NqN<_JtP=dDYYWy#HAy!l8cE(hG#cbMq>qk-EPdA@lV6a5v?_aui zGhV9s8em`hR~}%!WvK9l@YV!U3-QLBDc&iG!dJ z38wQi{!6=F?zkx9Cx2;8XfcbR;Hh-i+7|t^cyT=JL>nS(Jb&1FDCu<^F zv<4%^G3bg0wL3|OPM$tInX4gnoHp(>uxf4i+9W=50JeKY8N>oFCoC9+Eo9a$gqj@- z2n5U-wVHqq`leXz%V8PUoQI3*SI4Cv{P6ggG#N!Zy#v?;vSwTuDDZotA5?s6XTnjh z6)|k*qIbN_W9Cbfu4|&knot|xA^kY2eQ;7}H}PcQd}@n&FzvjgG8N=6B3=%G$}92) zU*K#5phvDc;yFKA5<@9*uC@uu{d0bk(DF8Sf^=KT7(O&qMCUQ(pK^nh+UX>agd~FP&qYAh}g; zCw}aa*S#1l+eyl3pEQ!c&#_x5dU3xcnPWB-P{-o>X`n}hyUs{re~>>Z zLR?>kW!#+$KYT>KFZhbkE?m{!awuecO*Z3*k$HrYScwQYppG|c>c+PD1k3s}++%Lo zT2rSF$pNihE9|vrvlnaai{Xky?jQ4eCp;G-m!}w_dNluf}o)R1r#{Y%I3hv5(k_CPOd( zN1+BmHc|uiM^(qQ1JmRBwv3U*cNGGUd?}Y@mThB*-bDpgm7je{LdegLV%J`%8Hbpe zKxX?L@_KtChR{H+^gZ-=9|}_&MrBvTw*HaKXRj095I8%szBtd5Eug{3d{f)q_uOW= zG(@gkx>d#YexJ2*@CpW=y)IdD$h-8)2sv$F;-hk*1@xWIL0=@rRHM0(7jAwqj)lJL z>M>BYgZ%AX!%0{%B&f!1GDea@Rb5Lcx$omotQM3vSa8>Z?G=b#eGLp~zn7!y^=t#1 zb5@l8i!%K;=DyB{D z^v?i~yDh6MpG$v2epBZVMrH;{CXj>l1-{r0%I(rKI#>~?)M>lyKhsSb|H7u+2RHGd zws6pAB{YjyOInzy!6y}e9 zalo}&6i^0pa>KYdPzOF#3QBuU;<_%(_u268Hn$@!m{th}O}(IWCr zjx6)u&oLMH;jl)HGT7p-UL z8t)c(O>&go*_9TC`9v@E(636MO$jR)AKXGi}`#0#2DmH zK4Z?>*fZXfq)rsg5ijN=gce9A5;aD_=6YK0cQFQ$>8Nmbx#qO;N^cFcD}+;{gyvT- z-k`cU@r~1dOJDVJ-YURMdj8(a^aeiKZFa+e@I52s1H&Z*KJzJlc1e$S^4*#uf~{%O zUF-^ih9Fe;%RJ<4Sjeg}_bcNS3c?&lIBN>|)eZG)Sa42gaMNw1it1#(1VElneJqUp zv@MY{mA$A3*S7lSP0nQ_0VJvg%C%aEDqq3V5*$jR_R4gwmfzO1Lp03%8Qa|_uPz5E zp?#ST%Y7VdCW538tm&N!zAvZA9etUNx2{fbYHFZ|_J4By-4Y@I*_w9z*D%(^r zG{NND^RQXig_R~Pe^w74zUMdUCbeHWHtvmx+P7;e&aR4G3Z8WW!m}?{^OQnO+&bu7 zXEcL0KG?T%WkS!5zaaGkpdXPD4F99L`R}#P-+t*2qS!{Nf3S^C+t~n7op54@Ut!*` z-yTc}H?=Z;nGgjM#bnB&$!0Xkw)-+)%1e<_KU7Y z3Qx_%jZV|nQ6KOV@ebrG6qf}j3TRk;z4x=Y3GFsFMg;$eej|9bmj8iB8Z{+LN>01y zW^qhj!XcMlt&3>OPYS0eM}A=%lyO%E+y9K~V`kL9agl`~qpYbeDZmL<65Z5d8B%j@ zw7l}DYNZ`H3?{M&+Q3HCLgRCGq09W>r5D^hEJ8!+D)b@#ul!cW3jN&{pdt`meHu&v6#^6I0CuQ>4~%r;2O38!jX6;RbusUx zG19Iw!Yc$?B_@K%rLrC(4@_xSN6KO^5$zi<-o8xcQlk;X`hy><%%DHx=fYGv@rJ+p=yq)a`3AnI1>o|2Of*PSJ zL{8r=idb(-{fYi;lJ|E#V1F8dZ>Wdt_v}^#^*i$F znEyst|4VGzIU0syo-|Q~RVex2h?raEtPaho7$Bt4sorUQqkfg~G&iAUUXu}?p@Z{M zZ^t`qahMq4e`Dsm{7E(9_6T)tZyIkgm+rb;F3&IAdMxhtsHdLO?`RS9RVBWYCgOKk zZt4&2l3@anUYIsKk(;AiDMNu9rvqmsUGP+o0pADEc+OWCQl`iEeOfe=#bW;=0fYqd zlBDP{CK0TQ!WkN#Djp-e_Yg|Qmk2zNC4carja#{FC2r_Cs%;&X94@s~a4Ltx4JqAmG|6;F2asnF>+@f?rAk7I z9<|Kvs3%IDO7M!2X8!-$;9qV^h``|`%77cqz}|iIREn(BSix(Qz%JbKPtNp7Z)j^m znVAki?3>srSUR9X0nGP|Herv79R2}ksq6EV19jtT~oiU)Iav$U87uJu3_?e`)!q)9^C7%HU* zxHq}bCZV31_{|&u9o*F?BJ_0BZIz)+<_rC12K}($#^`8%UW_!#qaT%ch&Fgn}7jSM)i2OU&CCOJp~E}?o~&9!?f z(sB~++=Bfq0Or{n{8-jfyPYw@f`~*Z6znT6M&m}+sb#)LSI|xuxu&;~bRB!_7b5}m z!n*aD>M_9PR~g}x<_7Kvq&SS!B$hYwzeLD?&MbcqS%wlAC`s<17r%Chf6Pv`v7oh*%cSngb?#a1a0ZaZfB z<*@j*nQGK9s1+;f2~v6^4Y+Y5zC04d*A8A&-wk?8eZ%j|cz!>iZhFLC3)@XFsortf zcy}>x5fQg^P1br|z#qaOqU-Cz?))-oT@$}(?2sMUaQ3jG#N-}gMqAl-83^0=PG9s` z2JVf@lLQE?QmL5ZH_GDhLV6;kFxVk47itf?_n-3xE{MopwQnpFq{-qeHLnsP+M!W5 zK3gb2hPBjePs~5Jg6fa6=0Q|OoNsK&CE@Tfb}bl0U(Q)TqRiY}k);G9T@dku&ilsU z)f&0Q*o9z!KJSMuKiHoN{;sl7IgQyQFC#Szp$;hz`W6t0H{$iL%BSU{orF6Ax4L-n zL?$G+#U0yz(6ZoU8o&S6&N^0bRO8>#@F3Igcf!?Q|4QhJ4MoZW7$<^p_P7{P`Wp`; z{HHG03Un7ZwO1-Ij3X}}TKBT6dbmWyBlq?6?CloH zpvt}?9O;U}RY+c-Digmz@DrclAH91w4a-b?eiB?@Cy zswmVb-}tKhS4f3Gw*GGnZ(cIk&2@apbUpzcCk07LCX*unY7ZI|97!gxml|30M(qfVv!@3B&Ro`;*}B#j59tKY+?RmFnS+`w5 zbrTnXtC%v0v%%`WBT33~M(ROZba`BFq4Vu1Jn4~}-%Wtt62pE}v##H9CZXfM zNXq|IUWexApphn~Js!cv`t0|z;tPFa6R`Jeg-_SvA&jGO@%jN6)Q;FA)N>IU<$n!_ z5lA`r;AK{|;yZVQ@q@RFSpl-AG{CNaMG`q1-)$;;rEv2BCt@!|=!%b}lF7#im#mUI zF4qo@UJaju%aOHt!5zqVce42vk?yDRJ#;;Fck?qVa+D^Br5Bkl#)jruo{3y~XE9&XiJG-q8 zkg_0;w@lL8lM%ug`&i*ud zj}G@J$lvthWnyiTb5t2I*WoH{<=u7T(c>EFNvQ806X${bc`XCOeLKifq9$kN<^Np+ zqRc${D7e&vjVh+!*5wlqx2FQj>(+p0^b>M87ZTs#N6RB8QId%f)(#Kd3C?Y250*=n z`@(IyMR^iLl>0NMj=*kO9y!JIsl%<&KrsTnJ6zxvq%2R0vOb4WGrlOzDb>Ws%cj=C z7}YE#IT!mht4D0-D{2;)Got=&Pq=|0Y%>yRpNVxENLM>G;9GLzak)OP(VtCCcRA2r zaTzB!gDvCGb=bh%(D2N&;n|_on^tDqu%gwWw0_vhHs0+91fOrtUkIWTNV|@BCKA4+ z*mC7`m~Ypapf=-xq4?O6?ON~lh85ejd7@X1N%g3|WT`JXfUIzr%zGQFb@-D1GD%#9 z=whFD`Mj(Yb{tl_+(eWdF1*}pB4{ncFx?LCmr0{O+N30w4O|2mtK+#Wb(ecFk2D4o zF9q!=B7#a1OfG&dw?P+5F_G@QhrbE3_%3S6;`7-~iwoTK#!rmT;$juXA}FVS?c^&i4L$ZzSW>%g+{&!D=42H~NOsq*~wE5oZdXOHX z#SXpEe0}u`lzSON3QS~>eWGi6qtp?Nd)Ombd831q6BHg5e2*IPb*@HtSS`1=^3Zzg9HCDcv(QI; z@hWg7n`(b?Z?)Qt3Vdl!lmp(w<&)caayhgC`2f8?ss&D?MjQ?8Kb|%VlI8RLDFMvq zW4qnsh&CA`9J=fraz@lZ69c>xRvOG0W$!bzTHWB#mF*KS&cl8GQcq9|8wkEg-0pw% z7I<%D?K5llWHilXi&wPsjX)<7$~EC)PV(}FDNbA_YWCEl368M0MGJgyyYI+{h1T#UGT}3 z{kE5?l%191w+NE<&OY9>(Ulc5S;m%D++7GJ-xF-_1T=%h`Ifr=hm;ZwAh;jRR=xjv06UhGPXpVF~NfZP}`tnayd5%?Z#QWRythJ zo#j8FFmGbuq^Ap70qjH%BLO$9PwAI@hZuZmI)g|sMuWQES~YWK0oSl_#hV?!1-J&3EnOaW92yrW4sBj!a^0v??FH$4}D< zBFfTh?xkDvbmAn0@JlZ-04x)B4D;ym@k#OWVn0Q zJE6L~BEel1@+gDQ0|Vf?gO2ZAnfEFYRF=Wbe~$okA7>us?Nn46PNlojTO#R3qi-c4 zI^%#7S%qC86!%R$5|Gh7>tVP26B;PGc-uT}2~V`^!7b8jQ*7fpjX*!xW}=F|0E~I!Z3-;Z2w{Ogx*&F`5%j%!t)ZeXF zu_!Yq#mXfvv#KixO4f`9a^;my`&;k6I7x4WVTkW{9n-;X{hMpj{<18vU@ksqn2vD@>2ETbHrcJ%I(zS60-CCebSsR{g^1 zxrfF}VejVTIk!6#ZI@vTm#*aI6N;Sn?}Q-of(yhf87Q^UYDUy|izoJoLJ{e(c_y5v zRFwb6C6wyal9iWo-&iAq1NhKSWna%}52Ey@?}=earf2dNBH~4aeiO%G$%uU7`2y zvEPCx&l)XXw~0oS?h|Couba^uG{_5c2_V>Z?PNCj9#StMwi=cBBrNhL%h%n4AkI+a>p^2j%epIx{O zo%kD;*JdzzYV(Q6msWE-mdpZ^QmsXSD422qs9210Z?SxM^p9AK zB8n6zLrLpadd&{8SG7ObaBRTvhM--8?cPD652PXM96<>BzUN2~A=tfmN5%TkQg3f1YutxpTV1Zb?tMFi7v%H+&lX zxr8}jTgD*2vl0=FymdnN-ls@w{=MAY=fG9$@{f0L(Vc;TM?-u3vl`f9f8mpLcUE`* zJ%k2gK70P3maf1}BH0dEUlyZ4qYBMHOQR-B4=B$a5UUH(#g>BSZU76&vimFaI zC;yJ+iZ&m$uE)opjCG7=G|;8vpsl_4H`O`+(`0ziS;l>2hzC=G+40^zmi!DvY+qQj z(dqz3QrXHw%zxmV5B;_oXn)7_mlBJi<{;HOBX#td zLx$j*>yE%!*=sso0fk7uY_ke5dyRtdO*GckQF8 zYRth-w0XK&fK$Ju<0M?#6VFxEjZfc!H9e#dZnrp@R0J1%ebPk_zf}dLe|dx*X$>}C zB8h}&>PReIGrV}Sq_Twn-`dIki39BPfBzqbVu#KR6!H{&1>i&e`MD+RnGc;;K{ zCq4HHl)eGFBSueC0HVO>juv%c2UcPWLwhYlrFem~$3kwUbTB8a&IB(TL;OTW`iHNh zzU=gF*shx4)X#xExZV^tj2%|g##KbD_;6+Ng72a3hzQ0t$haEECkf?T)|f%c-R4@A zEbqH!vg$YxNx^qK-jYt@xx;MUL&R*mKvqnx?N}t`sU?mXVEF5rD)+L1op_M|}Nx!9Hqo#!=VTQnU+7}Z-SFuX(`j1QR( zh0}Kq2hj^moOzy`yhz$0&I)%3)K(so4WIF z?=7GZZv>KkVP29P*kUxZ_5E9~HLokzsOcBK=pIgc(jKFW&I3YjH7z5TBw;rz{}V|o zeD?CLlfuU{^Q3dNxC>(FjTP?90rL2Ar(L3vrcqtC#MZ3m{loHEYnrw{9Ha8qWrquP z{XMD>*;Q>7gFQ2N#-;SJKaI({J_v!n`i1`lL{Lp@e&;Aptvq%v6XS#KR)C;HlDur`=wvS8h=oS3?ozjZ(FPXX>I)%$Y*)L7PYxDjFHx{Op3w3 zdWf;p0;Q%AkjNh{C1Wry-II@niIe$}9)ye@b>NY?&yz-T9b-P>L2~x_XHyRnX?_7G z2Yym%bCA8vg}M_YmGh+DH{O?AOkkrbAMmZ?G~ttIA)OR40Rjs6m;uEuSgb(D=H17HaU)_&xMb^T!o3Q(Xv*bw~6B0+2!ElRmPcW!rW zQ_~Ya~VR;DSAVuQ* zV3}c@)O$g$EPgEduSOlikh6 zhLQ3te5D=DWy(T-LIa*xq3_5vcNSueovtg!bP4VR<1qQ`f;a9?eq8MCwztfH%i10o zf5O-HhNqQ0%&5yu%nu5txMf_c??@V5O58Vsbw*ibGmnpsP~YfegMsD?zg|Zlm`)l; zj=v$vnY`tng3o{H5LF@=1P6LTE}bJN=P2*Ed~rVJ*QsxdL=4u~o9@>SoT+4PLeF;J z;Dh;&HCsQ?cfi5Lwq^XQprG&V8wWcT4{77;DSI!EA{BnFP~EpnoABGou~c?_AfK_q zXrF(%tXkH0LUZ3#A?^M33oKn>Gg|$yQnQz<6O*EhSmq5mIZt)p z#*e)v^{0FV{ywPe>jo@+lZ{U%{@PLe)0|m^IgvU=cPz^$s5D~x$dNMAIAkCrav&r8 z9joTEiY+5mH|MAfTl4oaURKYa5&DZpPl9K5F};&H@?eHB0!C+A<&tCC)cU(+H3)&I zf^+4(aTCdrv{x}(iu5cd%&h)pLWb&O#jiEcDH-PznGLlXpo&MWWZ!i0atodf1Hu^C zfY$^&y#n*n{I8uMUm!EijbxqHsSHFsC63hdwu&kDvLHDNPraZVXqqG+^hDb!HHDYw ztp;{lxq11#_E4Vpz)q`f)0G>+S}mbk%4B1;V=$J#X=Hz76LegYH!EFp{SVf;pJ^y* zQIJ$+#XZ5EMf1BskYXQI!gv8>$!6cu>4K zNu;3$M)LTJvwF&csvqvVW#)qd{ct36NR2y~<1?b2qa*sb;5i z$u1U2%VS;^1fkx}QQI|)ZMkjqyZ_PZIJA8w{F%o5aoJ6F^9E78{H2Un&5g^hjZ?jg zWll<>fiI+oH!g=eK#Y*FM^I0!()gzKsoQjy%?9e)UmXbM5QGoC2sjQxHFPvsZ`m!l z&!rX%u7_?&ROK9B+wFXp6WfE5_)nD{(Zeg>^5CS_4<6raKmnY`kv7MOIt^Ld^^=R` zNEGh7H07^)l}XZ&vZN<>HtHiFj7YO4mCd=zN^Enc|B|r8qB{YQJqRz)M%Y;m1b;cx zZXI&#{^thyzXugypzy~wCp=P2(X5)L_@9}v+Y3682VPlE19Fg>aVs~}bFl~ZzuWaG z!u$uT2gh{k?4$__$RVMtwUET$cpA~?!*(I_b#_@1t9b11R(r_>{JIYmq*i=#a}Ehr zWs$E4v!%m`GF52i1;Ty{=foTf=j2V@2J-N(+0|q_KL{7x{eFNmJe0NTYy7D6*}D+S zJ3SR7%=Q(E9kGp<$P`#jw6qq8`NB+8q=@BAEN=aSsqce$3|?9MXDiEIL6D>6yDt{q z#ZrlaKB^T;3+OdA-#SJ*Hg=qE(xnnXs6N+yOhv?JV?_o?*#|8jGV6WJ{%rJqGd_e5 zA#$Iie}OcQWXO>SuB1JyCTn$4WJ6wRSVf*C_2iMGixZ6xn_gSL?t*W-g1#Fh z`Ygwy=77hbK6*`}iKt(2`xlidaou?L1xDj&j@=@kQBazR>|Xz|-~F?W5K^d{#nk+i zz;;y<(c}ipeew3g&n|%2)obT-k$9NR`$%5PVG__t7JQ(Du*r`g38#{6qZZ+QD zQLqhcSaL^wNeAk;aveK?M6C1g{~oMe_}%jvHnU@DyaC9vLp7 zYt~CC#%54Ann1(6rE}xU7CO55nVIL|ugEx3YcSt1XME2h#Se*moSRjua=>tq6;L9E zZuK0oZ=EhTWhhlHupz!w?=DgxSlna;E zU?;g?PdWAji#61h)mVgV1aWU%ymPp$CcHH?u8X_NV~p1|<+0@=7A6f(Q|tNQBd_7v zwKtGUaS-xJxl@%rkM*5E30glh)ywpO&dptlgFRhO=@M$kDENkal8;_(#2a=-MlrZt2!|K z{z&&>^75g6OD+rpByWRpf?qc|vq5_mzuhe&o&kAb{EHi@?xAi=*yspIl)-qmk!XVP7%Ae9Wqe+;IO? zr5~o))D{sX8H=v1Y^Z9Aj)CBi~WjN=dq zz@%?PFCwD%V8($~e|h(;GS2?p>uRhLKec+b{$*eMpU17gV2Z|x?{00Ujsb40_gMm3 z<*8aAQJ5g!5U#`C)ZGWBB$f{YOFZ11!E|^N%*A5o0aSODZAZ;_K3|}$k5hLH?ou0a zv!(B6+O%$DBVrK`p7UevK>-1Ca!J=S?duo6u3pC*AKA-cTQo9l{F!EV<9-AUX~Jx7 zOruHX8IT^Jo*G zl>;B)JY!xOI*IdulOI}hVwy$39Y9P2)|z%y56|9scaf3{wk-Ebt&q{OBL$lKyOal9 zR2U9IGu6A**dBk^;pn(AJ0!>Rjp)v9WlBh-#yRV@R?4sn*u`Y`D(sEGypn*TEp`mC zHuI<5{gGp@2-fVKIbq7IuKsXcF~!BEg7LpCYudw0RrX(z(p{jAICl*rX$<5EBm3*r z-i@T=*~dAffIwL)jUhYd0pcswP3Y;9cz1ekc(uFf3EH027>GeeL~gD&F#nuZ&^S>YfX!KIzIK`KY8%jWN)@dr&P&n>FRiHp<0XJU zXiS+WEA5PI)3C55TKw}%1f8aN^j zx+o_(QZnUse#xHWwks1VVExwKJj|T?1$6CC#sL;k+$u5^=qwe=ITdjjQO&CU-2R*( zx&mmHvvR3)$0UXAHQ^3!>=TIX8Mp(tjb62=kEw5BfNh?swnp`$f9*5rH_NVtd#!--E7a2K2v0Aq!oAIppo*iFv)SK80CslV)BItl#$WvGG9Sbm=FTJqnRa%Zpucp2E^Lt+@D@+{ zo!}>M5S{%WH`^Acw1cY_0JgftjpX%$v|q>g-JQ(jd6DG9PWHtp62SVw+2;B9hURS+#K+|w8kdqonF zrrx6_=#7_NZNKiI5V&tAgxk`R=i}9`V*M?_MUn+K@p=s0zL_HxX2oa$E8W^H zto$CB`sKz85}EW+ekydUX6dbuYhq2QFO9>jO}_8g%$XM+P;%Dpyr#fS*pX(`vCm|V z6j}J$oyVEg`1{G*MwjZ@Y3z|%zH`pl(A|v~Fz%cA#kA^aX7D$SU+tv10r{5+eS)zG zro!qG-M@YltE}1W?IpW9H`i>1u?g_u`3QyzjR$8wTM^7-8|N?)=&Qz4jAJ$fWLMjX z-COT5(T#F|wdmCs9*Uvv7F*jc2L|73AFeMO>-g&iyghb}_uY>&zEG$y7S?A2w>_A{ zvD#KYIQ^tpPEUNmrQdnijji)hSWQ}|;)M3kYi&J4R6*ikl97dss?jdYujNf zwva`FKUHq@vRbaFEVx7mv=E~^X@{M7CFNbZ<{3=lmo@zYnaVBZUfdvM?st3ckuz`w z3%&tE=cA_ldc7+==h&Za#Z1gWWz}y4mo7!!ay87=gUZkEV;geN|HR&Wn!eiqqdT`7 zJ)6oU9abFd_>*GAt!^n(rsGkgR4$I7n3mH+UQk!t${4ek3{;qz=y z`k|wgrM4r(9DmbEFqhc>769Gk{&wu}>C0&02XEjd=Fr;k@=NRaYKtQ55CS6*)%W^T7P`Rxr-^{~%#1bwq!BBCH)uP}_Q zH;z8zn|D=6`R@W{0ftcIbhj*Cf?ptBG0O#;6H6v@wwOGBs6gYVkzxH7rzqg_3-Py` zOqLbqc;hp~4JlPdl2SxfsZiAir-sSz1ubIi)LI5jSjg=@F*w_NPez9+Ir<>QObwIP zQWINm%0@zGQF#iFj#ZBNE$+_qsxLRM;l9X|q&|y%4eI)kPvv*?lfJH98d?~JA>b!& zV2v@=+~L=!N{{X~5k{rm zrg?poa+gTIj@j!h4rMa8i|0qmVkZ{xNpyv6d^}=L7mH9N5T(A#NK*JoE@{hPJyo1Y zHKo7q#XTN34$}C@qtiFEli;7c5$whE4G#yJ_%Lw&snc*uTlUN4gGA?%lck%Ky$s$k zxDr=4t0(JR5oE@e5r-qXfxwVDFvncdJN_PSz5zs*ofc*?mc^6RmIWOzcU zry{v)tZ_9FUcAvox~LwO0D0nUYjI?j(_=pJ(jx@h=~yJGe+9*qCs0VuM-IRIYB$oPU^2iqkhccmu{5?cAF*1vSboyWZ!xj=7#0I8D-%w2IBtf6Z}0iN_=^$lP$a|WDt{=6OvZ`Wuz4=CG6 z>UmRrUnNbw(B@2s2Eq)eNK~I_4zTj{Wd89!5@SZWFwd)Z88z{1OLjlD>w{O8Rml*CTMow{aq+b2QUrI={-t$S3C2P8ZlreK!O}Q6u zHvE^IBR+jjx!#%V@2LOf*tY73=5|(>g32A~xqY!@* zoLVySYiUNv$1xt2arz3AWy42ecvA3lq}cBPqso-`YSW5Lr^a&ej^-1wHnCfuL)ysR z>~!;%$(LVpYr9wJxy7rOVB0knC$cF#rW!i^_30#y_G7LI zpw;AVO{^~?K|xJ2sz4`wIwn54HBlA#`{lu-@v5PTl`Krl8(Dh($4=^yT~^8K|L=_!TD))dkD~_{e0c9vr7yh)qr~V1#kDg zj8UZx(X!z3z{x93t?yOf1!U`a)@_CCRqLJ$IDguDFW?NMA-h=85O=TMI=eT&;!@{! zld-JR(5Qz2(~C1e*TD0bPZB(rjLkvX8#77+WJjj#M`gb0*^7VnrogJ`vp}r_#kg{Mo6MN7!zS~c z7rweQmIjZIYr|!h%TvJqwb+XVf7xB_v$=kj5={e)J5X4dDr`s)1PCbW21kE z{F~J|uZD3f5v!tCp{>UauW+aDQLj-)y_L2?{=s-agi!W}ns>LFcjw!FtN!sHc5$0~ zjavJ>4U9BO>N$<`B!4)s$Ne=K`^kzuSv6JJBi;&I@IRa8$PLbA3JMS6nDDayhF!&) zORlqQ3|9NzZ6mY%qkyi9f9o_pgp;B3mp0j0Tu6GZ;rHT-Czc7Tf*gthY)-gI6-2mp zwh2yn^S#MniFhfET&FbUx+taLjiZ8u2$2jgTzDrcJCz4$a3Ptk_Kp^%O`?oIyfHjI=$ zsW0&;U-r338py8Med_X#0G?$P1aJ5b?EHAd&>PVo88ty8U(GNB3{?7_*%ea7+fn?$+hd zi>>F$W|ZyWJcMiyuHXMKFOfe#v8`QuV<4!}IqN%DICY^n$$>rT@+g;}DCvp^hZ6ZoF+J^lNfHeY(v8FogCtVTV4z?@R= zK4w;ewn*bLh+qa%x&(dj^F;EG23>Dxbg~qy^;;I=LJQ*3fWKHXSYFKFA{62O&u$3U^W;6PzZHS-z(bvP7Hv`JJ!->pU{mgBR zo~lyHTY&S`a)V*tBCF@Z0-uel(+7CUpQ8zx!GArEI4EKkb%Sg*tMK&WW>)ZDA4N)`)VW($^09$XxOg-(C_PJ0>-U5~15 zZ2HkKCkfo_N9vog;7uZeway4SOx!kFXL#sJn$?s_RfY`(1_Rcpe2nO49G>(uAra`K zX=r!-NR(R+#^@7@o3w0nL@54ody@7~r6z~`Y5uzFe0gh=?r$G3^72GGJ*VQ8Nc&Xq zu=pb2MCWGx33oixtn_Y4Ip+K<#)Fg0r`XmyVPclg;7DB}3a(pKYR=%dlR|?PtSV*yvhNPv>mdn(Rbt6Z zXceb?g4W2n1>N2}$!umzv?KnlY^tz$Ah(ii+cCo@u@$n_^EG4kRU$dYy~*mRhw#1U zS1AsC6C_6+|9BfYCV8NT^v|eBF#B}IZD)1b5w|#AmuTLC5JOkJti2l1S}ZZIr=>nO z#|>k=q!Vc46ZBkx&XViu^Vr*GJh`SnUzY6%Y&XEvz0@;`w**?r=%=Hd#FtybfeYA( zs<4{;E|(~dYpVFDd0M%x2r6=(xk?MiQF|a+SSy6(POHUzcv$~7&hGmIvyO9PCkRsq ztIk=%Fh98~!m>Pa=wJ?w)rxNErX<GgjEaA<)pw{;)<$k7& za_E(c(%5|JlC1N6OJ)t#6$|Rp;8o#$rut8Cs~d!_3cavD{~Uc>%qZf$_Gxy|ieUa- z6G@|Rb8~O^W%WXQ(yw37&aKSM(QKN`q}f(Gy<;!g$;Ze%k%O=u<^oUJh)vJMRH>E- z;nbibBq%~UShYWSgRH844I;YrnN`|mSSQ|yP`2@jIB|-qVyoTHF_si<7l^BNjm#{A?oiXrEBq=?CeU@sEN4|qO7QOp>^Lc{n6XeeV#1h=uVjeCReL_E3&z#)W8Vc+&QmhIZ&X#*}5q!uB z$yScLBr}z&CI-0{b~I=kZ4S8F5Af7st_ZSs6f6s>V-A7t+tN78CD2CH$vARWWJ2aN z`qcjof$}$6W!a?y4J%Qv)G?*hxS&+Oq*UX8#EgJGx|*hxK`?zlHr<~gFM!c;>Z;ge zBhOLwm6{wA_=p~J@==f#b)6zHTlUAiMAL#0^sbqw3lN(>=#tMw(=jl$F-L1`L7d+) zYU8ssfQ8RCHu6M?Hp5dr>BLnjO3A?MFVPhk{KrXM5zJ`DJqmmr_{-&@UYi(_gG665 z(w^#<3A$8c+BGobx@JWlUHj2+G#Et4 zQ@iJ*O(pMDc<0aTMjCN5A7cL!k&SvQdv|boz7dSCQPy2%$PtBo{PXcbXd)S!bGT;a z)9UBfMdGSx1W&@VGQ&T}FN919YZjp7ZS%0 z4U&K9P=UmKq&8zPohQ7Hn$CJ)^~676>^EkYldN->V#Alt><}@YP|>_CLtfQnulTsc zL`R#)>UmLU7l9LdCo600mScfTt@o1X71&TM`AQ=)*J)we2St&4TKsMqb1LI^^tcNZ z@+Nf)xhygqF?e{G@usitmSUpv2e~eX=>r zGl*wYo2n_fJAm)ITcm^DO&0juOT}=C;B=wo_s?TZ-@kB>J+1ZTUugWwqf!mX+Gm`Zb@eR~DIK8L1u-0w zD{nGn1FwL`QN5bp zP@if2^O7{sV7%hw1YidQ=X=nrAQy?!TsIA=xwhgTEsL!P-q32i7_$8|^j*+tdja4N zlv5;OLQiDqdn@JLxxX(fc$G`D^GYJScNBamb0nh}6}n>Wdo{~&xzgAXN#^xcId5hX ztx<}bT?D8tZ;BqcC6>B8^X8)SjSm5km+htI+atbB1_q$c6WFYIdWoyO*+G>mr`t}j z?Ih+SQ9Olz8zPoR%TNJ=5}?0q7;24`s6kFYB=-s|9cs)R_fCls*^`7+O1op6HkYOB zxLH><4VFloh3(G^AG#Ub(Okd4FEGaTe^dPaO| zkf#Ef$ZtPmT(wgNT3OGL4d74K##NuRh43A|4gzZMEg4Ncer)*?T{Cc4KJZ|uqnBtY zGsXhELwx%u^C2eI?S%(nG8g3^9JSM&$+-Ja5qz*XDxBm|?(Uwp6lrk}Uc9A9 zDWwFr0xcRKK#FT|D8;=%fffzLp*SHe4#6FQyK5jH&wTrxdH2kI_N>XD%=)ut%{uSv zy3XT1Z*t;5BgV`F2bCbxmDvjVHx9|+`Wf-MI2JNGg;(o2tB$K_r$u7vuw>H5963Wz zDk0HQP9RUh^%uok8&G=OVpyt-YJHHqT_g z6TLl+yAjs6OaU{O)c^K+|MxzA1D^c|E$!nyY>iq)fN)N|SC^+w!__w2g7Icp%=yv; z!bTe5Ao`3APAy)HOxLkiM!hYkRdS`Qt<=G`BsM3GC;q$*qu1HcVwtd}_C_Cgrw7bA zH=b@}xm~WkO82`vZ@f)40e24vQq;)u1wGFfcRbB>^8IWAsVAk`{Wnpc5{uM{9Ae+&E5OEnOao)C; z&fm;VC!4i;J71#8yMuCwAuU6L>2M+nw7W-zm}06)?Vs&GN5$l!dFJnKvyOVcae=N! zy3;*2&9086+Q|iy)I@T_7H7EF+Le?iaqLJOaSS;VXviuJ@@-mX zbLe&EzUpjMrU!LeN%F1!6^Xvrvs5JRgJ=2|IX1j*t{1?u>BW!@yCR6E^pz>)Di<6g zlC<*KVq(U}!w>iw2AFY#LlO%kf=8+M7Mf`FU7HNFxF5~i`{f*v4QR3C-7XWtRyzqr zngiRXQ?ag>`Gi*b`h~(5lciBEpw(_a zf2=vmAo|T9VfOgR_SwLAy+7l|@c*tS|0z0xSl;|!HTjRMT*Y#*`5r(01Cpu@3l_4t zQYjAvD@qChBTFJEY3I1DG9m04?is&wja<{uziyY7 zQybsF9I^Xy$;((?P&(1chrH0o5~|UO>=kd986d8Nbdv}Vh)*%ZW=4oQ)=54e@niLQ zNwnWt>f3cZ(A?`psQ;#Bl7FG6*aTs~h#|?c2lPy4AYlHBB3k$Brpeh@YLj-A6Dy9J ztG}3st&n@1)8d=FsoMT*=cTxlVuDv3K4rFN35P2AGo&pp%eb3<1iK7?j%MlG>h-o( zs9z_E-}P!tSa);Fufq0U!Ga%~MYf*~HsOOj8D(}k4gHhlN!7PHiOfD?D^=c#--qWTZ1 z=dKN+r?fS!hyR2Mr$@K+&i2SeD;d@Hf9TsUjZ-MD^1>ajxLn~o#9(v$u@alqT8cX- zHSg@b_Jw`E4{*wDpe+01Wqso5F(S9=mr-yn5Z6u+3R@g5WE#sc3p4WQf$B?frJetj z-wNi0d=HmAE9De_Un(r&bKNd+uO*}d)&=(+7tWcCr`eG>Td0hV8czj>|8|Wz7HYKJ zGMHqWw4&k{YrVY^tv!~eQ<>bPSWcH%xx$=N9`-o5n}@LmdNBSOQ5hBNEN@z!&g``dbY*rVYN^eQ8N5XW zKI0XZAt9seKmSW~afs&Z`}H)N!EZta5&S8BuyW{qvpumhvEyfR4x|>3t<0*^P0uy4 z7qKTXd6~RCk7*w@kl{#*gZ69MzJRx9A8NE>HlzzeEhh0BOoyx<5MSsT^}9Y+l!n!%}uq*u&Mc4}Jc>ezw^^`kXh%GD%B zzZX9$i;-!h3q8h$WHQ0-qU7=^@Q`Qv$7|ngq3-|QZf%q8@mtwzDmyo4wQLpahk>tX za%m>xcd33=+#R@#3d*F+D0G$Ooa_m?S9r%|tRizAxcc5gb9he$UMJcL+K#AZdP-fpNc?6 z54z%-ZygO=pb<90V#Y3tQsGuCmpYsrs`O9!eDzq4kg2hzzs(4|e=NbqFQ<13dd^zf zC_}V)$|LIzWMDV}8%iETt3N{PmotvKYmP%>st}X;C};pT%%2+u>65uy!DCV)5@PF- z5=4tg4$LOlM+EKH1QV82gzPhZ#}E7b*qt7?$%k4U8MH1-ROe9*Jy9>uZe`tNY!A|4 zl#AMUq4wjcpbl)rf-ll|$j&Tcq1NghKen_H>nw#zqRTX~V*zH>|7H3ROJ^z=FJ^oU zx7>{nrLk3MX!@~@@8Z%?8yvWBjQ2(yhmhd04_y6Up;*Rpa^>K+NJW&A<^DT9>%i{g zWjupvCH(hPk&Q;dq)8%`=`hE~dk*aDr7_m<5J$>Y(^Dk$Y?2*n*DGAVBpWMndC-;Q z;ca=|wJ0Q7SF_yc4`F|r9PSj!a(F(I z;b1CiCyXWA7|<6$%(BZ!t4g`{rqZh$e4|&xZN0MQ32OOBDar4P{d$9631m~TCl~9% zriW}EPo|{=A?F?3-3fSnpu%#xJ$X|?sKa+%v{O~^D9*MoCnm9q$dPsf@W416?`u~| zrX~rZieZ;#Bo%AQ%6MB<3rnFmjscebCPQb5pFLM(h3;;V%OVX1{h*428bt{}lBf%PVE#P&Nz}- z5*w_0S(Bp`ncQpc+VD}zY*dc>R_z6|lI4m5!%-co=N=tD7dOEZPa3IZ=u}r_n49+L zJMo7WP@pvD+b2k?{+prADaYxi&te@wssc&??J=;{UML=*fD_9`7|#aZLF9$yYWAZb zUc?`rifUlL`|ofG(PS z-)wEsJ<8ItmCaE~+=jKRd+9_=1nLXRnA`!oPix3ci4L~?F7!V!nZ;V*c_pTf+%^TI3cp&%t(F7Z8Df+k=JQ3i0uEBSA*{n4{b%{v0|hJipcWsAcpJLH zLIez59ieA!hWbErt|e9OhO!(b)&)PDbc(e-;5CMaf_PwX9jYNEk_oleo_}7kq!XO}8b3*;(@N0tHkkR{ zcj-2)H%f)raUtl-+tRA6#kF^%50=pPxs2`NZBZ_M z{%)7_5ws|}ILtAuC#*D}j<{+pXoWy{I$OH_q$#i3NS`I_%L9Z4Kf$#uAJkQ zW+XD7ltT+`+KyB-*-X8V``hg975u`ou- zaC4oWcI6J;{nnH#*@`Px%T|kGz~DYY6*H5L>u z?>u=Oba3lFrv4^)#SKRlKcH120Q($qk7U-l%lnLuF(}@1@nA#h&APjuGz`Y^EW_6a zD_=evtZfsBD0*3w{W4Of5r?J0@@Xe5L$hPws&%>%t0AmXM^fO?W5>I(e7-S*l5brT zj>Y=GevjiAz&({>^C}7Ys}GEfPmx&R(=jfc3+eaK?QTK09$- zhGr&(g}@@dY!<^NBb78Y?LKVVhkN8;@5R@_zMGys&YboNnE|bmL7QoN#8Ne%uoLi^>@}Sy72@F?#(i4E0;}oC2L?G3ncG)` zAKXrb03vDH-wa=*gH8M*NqVHkY+>f_p)Ra&1S;d!w}hiFE(qKbwjY^0J_U7w+HlsHZt*m zz_rJ8(gdxo!U)gDTUsQWHHqtu2rJLtH&OO0-anRmK%_EpZ(RY4n0r##=CXOLH6BnQ zw#xrNOjx$%O%&G(?YzE_9>1#Gm993=pDSOg9t?KC=29o%vf(G>o_*(^H)mwKh89RT zi5Vt%V$w?Z=C&-bIOYGRl3L7tXx6TDLs4h4Y;0xs?KPdpb5Qg!TEpxfd{d40{1=^% zKI&`LTE5h4@N?TRouK1Me5l{f>uDe(NYs&jKr;agf#yhEH1wm{P5gPUpFRpx`kmfDRIoB8FNVc4smOtT|wldj}e z(vL$dsp<_PHpSHz45wTk9rYHnPN;wld-=W~#bS_9_yo z@)r%x$^2&H9rSKIS5ENZiud?zwXFnQeMVJLdAdPP8Apz7(Pw`lR3^cMF@mlQ z1c*-^o&;)ve%@5U2+1&TD^@O`!<`>{Hzp1TWHaTGtS3cKeb!4qt#B!vYXqh;G`-hf z%j8<)Nmh#ZT<527(B{DGrNi0@C;{|T7%n~QwEzh1QX{qfWl*j)U+OHv<}91RCOt)O zoxAGGz~#-6X3N?ymUV+r<5tqx*^hz_j7Q{gJHKBpu>{65ck<{L*Km8xh%HYjV_!R0 z5hyazq2Bz;#_umBk=MtlwUYHWytr%tZyJ(KYdf@sbN~ry;}UF?HwvM9shdY<*(gv0 zSn7GvG59Q#3E7bo*&+Sjr_mPAqpO?FT^coyPtgjRCfKVjK8e|=YP(Jt3(GTJ^0<^o;} z@f%14Vu<20eA0DHq_U{%?1*2-RO@VQhk?Ur3TQaQ&OM5%3_DDk6`BB6WlM9MFKiVT zT3TDTbW8NMw&8q2do;_Z%asefJw&oa`oMu7YG!7WB}FJuv3@*pN564H*nD^;#m8sF zm7fCy!@8_oDM=0k(y#)$K52rPi)3_ql+r|YMLh-sMn*NUhNeu#@FgABXugU$*Ma#o zVDdRK)fR0YYy(c32CP%jf4x!$0zi^l?Nz%zngVZZ{%7VJFVKFZPfG-s7a^ zZC}gwfQa{hyzL4eWEosnr740C5Sq^lAm0I`!(9E=t+9ItK{yjof;bR?`4cY4#t$UX zH)ip-tv?}lRNTH+#Xci>T7^|rjF*Bflo|;D`sJ&|ZRDH7?NG%^(z{BI)t#FXP$Py|5gt$yW`Z6##2#0KKll1?e)P|s z1Hb=a0seE3t35d{7%c@^II&7f*$GM73+azYs7U?{d3vN&n6VX>1+7Ch@#sY$ zG3%1RK&1?CDrGGFXWT2EX7}RRhs&`4<=%@<7%45tR3xkj2X*`*vO_X zHgUwbn#EiA+lyu%G=cpX)qm!%ZCXi!rlyqNeCj|VY&!0F1CHvJY({@bKtJ|z%ZYl@ z+L<m^ACXUwJ!LFW-2k`}S+1_Y_^|>OMVqKWD|9Yy*)V!vsw9ku%t7n9$h^nk zg9(c+YCMn?6F%&vheB@N%YiSA=Z|9^`>oJgM7T~^`j#)4gqZ*9{<*O-Zi=1WidHp+ z+^00|+EZGuII|v&lsh+Kay=i~E)r9ZnTn4fT1&CPnkAn3(wpy5rf)(vC64!pF?5$Y z#_`wf{$xwHX%?$XZ6~j_?#`9vfVNzz+D5_n4e}uXvkO)&A_Y9exz+fM9?KTJXKCnI z6G+;Ya&k~EjA!HFK+s=>a$hwnQ|Z5sx;b3+sws9hs|i^xpczGM$EY;xa83q8m;y^^ z9)0j%?3%e+y{+mwvrq`)N}5`ZVwQ!S#cPB7`dQFB7Kh5n&j2<>x;hV|8X$n~yFuSk z4z~*iZpzI^?;JCs>uV$EG z#N?9v@73i4S*-XR+HeewL@2_yOof2N^I(ftK#Oq~cS4P;gEi+fe%Ylw~C2E=* z_Ye|B{|m9J?%zewfKzTZs1=X-Y&Cn9VP(hotA>K?e_aC?U35oc6bah}Mh&T05-Bu6 z=TlRraf4ZtR-2^5s~w&4P}|`csMTf-bcd1>)mIfjA5hFH={Q~@_GCADrjzdPCv1>M z6i^U7uDFuVb-9x95Ifg*UDE z?7#|;4ml-66n@p}-Z@Zd<9j^oH_D?Bh1+7@b8jZJItq`;N4Dndf63@u4$g0a`oJB^ zEt*&y-&b@7hem`+Ur%oly{|Hgn<#r9DOqlMj5k?f7FEI~OZ%T0q3 z98fGNG@21=cjxoxs_XQsTcv-QmBnt+P1YRhLwa$z7{_%fsQ%NjtCcM$aAt(f$-IeV z_kCqo>4YQfaO5!=F7WwKJ3*OY1s%eo4*kjrR72CxFhG4?LW~K0M-cn?Fhg3w;`Ss> zSVnpCT-Y3HSN2nJkK0IJ8o#DcAGn~_y2a2T;05_RKKJ0Sf@pN)$i-6Bo@K!|6>guc z06CbYqc71qbUn^vc|&yYg6ALYl!=RXOll9;yN!+$20_^ z!n6!O^0cTri%U_5ru==BCQ#6_r2OlFgQ}uut2z9Qu~`P4Tf||aqG(-x@A6gvir8i} zL^f{knpb)4TJ(Ni{gn`)+fi#YkL(=#{2*1bUIPk|5fP~>WiK1w{UqcKm)Ub*_9=>W zUr^RHQ&|ea?>`0Ci%_@UoP3b;zvR6K;zC_ycF6&V)C^8&00qqRMwp|~LqY)^tos+z zU|>{eST;+v5XJO2kWD|}H`}f~j)@(PtQ|W7DMn#Ehwh#h*?m+T==jWQfTgpyNdr7M z(eOm^jjEL&2bUp&vJE2drA1gjh~t8u)RLOyO3PgHm%AZxv8 zqlQSU_mFqExXl(pdfs|M?TlsIiV-;;U#eFs(_mLSo&}V{WZ2dj(y^Nx?46|0Q21>v;6l1ag>H z{XA>^=N2xQ?{2HA=m9XN-U7fgTY~FO)vGNh9TK@)*v`6~*Q^Xlo}@I21z=gUojv!v zEOk5%uQJ1FHCMHePsnD%VA->7t{OfN)W}P=&eNJ8cCm=+IFuqTG=OMYh92aF;P%_X z%?mcFC@`fQH5tqY^%vUZ#_(|O_tgLD5c7_++zxc=IjxCbZOYdZVP2m3)d8@_(IuCU zcrA3T;MClb!65gyyEN_11kdX6oOiHIoqM0Kokb1t!}CA9O2qM*lO^KaauIXKjjz{g z^l%`6zGXH#@FI5AqI6yFBMa~QXc!K8n=D7l2?9K5XlHY3)^Wkic4_65?LoD>$=-#m zS+h^jo%!M#a4pKDpdXh7-k0It0d$RJNm|aCTK13dN^fZVqDqJ>_bfS)tfhq zbGcm+iWi??8)q3kfhv6W@c<(5-fbqBjFzE>TR5kVr$Z#zxZg_C*3 zNOnllfU@p~H=5oyfuw8zAFDL1{z|NgSHJ>wFb@B5=S`!!J(*G}SRzVUSFy5vMSV}v zkEyV*@C1$c5{yD%vmyTe70#Wz=U4FRjKKfS;F(VQN@Qku_(Hkg%jTLk0mZ8hrbD>l zA>5b{Zdl!`=S1vn1>nsTyZhA_Fx=ov$D)_;p0`qmB^Tu0)&RkD#B&^wpkBA)^ z1&n!bWc(mZ-M$pj$2Me*n>GpZPj&?;6CzP;U2X$8Vx(Nfk9F-Wdy&(10b;^_rX2yf zx57bp73cZXO@MXq9*|Z8fY>-Mv$8X)lKCQ>7%U+uc$lF7$@DhM(aq^y{G9%!kjmFB zb%8|u8VJ)brju=o>5Ql7^go0Gwluu9t0SUrB8f?J)pU4Bhcutx)83Yp8h~5#Me;bo z!+794l)>i@p4wjQ*Lk_h3T%wA@&W)uTB8J7|r<3HZ{2j+0m+ajm+Lzt6Dj!1`Zq)A9 zXm&QFuZ}To_>RRKo%P`z8~C@!Yq1jPx3_BPZYUh5?8_(hIKIamE*NFQoBL<@D=Q7>=3wROrPcFM%&&%Q|p8cP0E1{`+$ z+=hid+3Go{+B!Pjw1QU%S?!)t%ZVKlO%5!?5tg`iB`@)JkMDBymgc9yY!9sDBjA`&(M1OPg8?4;Ux!4u}w*U4*4nmE|nxnOXNs^ zTszjZFzNx(xQ{#9!A~Y=)?*QVBjk_8>Y^B;?LmoZ%clp2l>SsSD_wiMP{#$i^)JwI zr3w=^Ww*A4B2VPQ54$a2wi)lP~ z@7Vk~)hxvZYQIV!FKwxa>_u3lNAhsD#tw=FX`cqCoO;{L3`{icHpGaWH&9wF#I{0R zxQ_6rkB!iGU6`;9h!COEK2DM*U$md<~Kg zWVh2bj=U{V3?@INuwvzr9N_5e_O9r;UA%pqymdy-{_dP)L#LTBVQ?|l*1HPP*Cb@w zUF$y`F#KTYdE;FUgW?g474YAm0z1C_;;V*y6mx0!#~$#B^c#tD|EkV34&HCq?Zow% zT+Wy7Kb{_+0Qp|lOp7|x?H&*?XfFi4g{xUq0K)Ak;F>x=B84Q#l6zYoh~qYJrxeWL z#-%(@B-S}5;&|!BDmJk=(cKQ3#;7RwY^t=7PM@YIrmkF$kL+e6Z<$TcW`lp-c$+U}i`H=D2bLDv@^L(A@ylURKPcI| zB6za#rE$|R|>HClxy#E03*fTLV`^G@i8rUQ{CpxMHLFQ$F|CPTkjVp z{1wsS*3+>+>iAs*#Oa(m?kaW$d0}Y&KrDqK{WLVjx0PD51}Lr9O~73;KreB z;}w7;T6PfYbvPp?QjZKZ^57q)inn{9m_ff>geU6Yd3Ql9r|8R+ zfVoMSh9E@bq%c$gcN45f9p|jD&yp~wNp;Xe+U8)u8p1x2MD(u|`x-GWD;M74Bkp7c zLAK_s&3Mb+iWsX2(vhssOyszCE&xsXVh2GILieA6RJx1xl)Ls6Hr?NF)5Z25ll{45 zqm!e>@9$<*yfc$i*iV=S-aeY7XpMC>yCH=b^8HP?d_%#mMWh(ga)0;ODaV(N)xI_I z+0<~*!btuDZ#6Wy+(j8c&&S;!Vs;utu7A1mZP@JD4x1ycjAn2g?8O;1TfEwqnkNB`DRK_x)KQ? z!p#3Kb}giAMYVJqw@U2$g97IG zrh%tKl7cf4pOLB7T}`39pShCxNf`q*587RZ6y!uOYNkHd!?tL`J7f&Tlo?ZH4#jt$ zkgK%=pkcAtr#SQX4`#XNid%Ws9UH^E1fhMNO?Sollb^;{eI;$7^p}5P**$^=RS3c7 zlrAsw->X;8lD2$Pc_U>xE2A7Jhd%MGhKvz7<#;gZ4lRTUmv?|kj>=fjN`Ru$nCx!H zk%&p&0zawilbD{5SeaWZB z6RL1I)=TtRz+NK&B<(F!yy&AyFq}cuYWIDOwQ(;mFvvjPyr+hHAIy z!JPC5hPbP#Y^AL9pm?(3<N>3#S*$ zF3+nGc%Dm zBgCzI+#4d1H^a@D5j-=O&;}tek#cpJ00rEg)+p5G#&*7Xiud9nFiP6(dkLerP7`e* zFn)B!ifkc(CmX~PU{8a+F!PV;j2~%?&;`c-g1d85$g?x*Kz35W>CXm;+F{b7+x6>`>GIMjmRaW1`q)tLi!qYhsOGYYas6^L0??533Tlks1 z5k>Pwt9xT=3V(kp(g23TrS~H}G~EL$5tn%5>07#Ib|36C2e2QVxbFIAUHEw0`8)pn zweZ_t;rrhww?8nSS=a*(Pv!nWv%y-8T!^FFyrJScudnswhUph`Gg^1oe$Bf+(ng%7 z&ESHYCrj}dT3^O`q=9VLWSCRDH9$V}o|8!*7DAl&3hNFS z$oY&T?KFvd%0K>lG-}Gm#h&EisQbQ<|3=_Ks7=dPMUhY{cijSWZtG_Rs4NePk@>a7 z$sgcLT9Ty>tg-rfxnvr~msJ2{?V=@3JX7o#(ruR3mwGXGLn^0t?;N71*jXF#^7%`n zHLDGb4=wv{BKWRQYUA;I$3g6CFJ1dkO0i40u}ql@auym_$7g)>LAh~^WCaq^_+F{^ z@)6-=_KIo0uU=Jp6 z{Ia|!lwg|yXQ&KXR9gmc72&X@-GbpcS8oe!OlW>bV^iTK- zS07mEk>BA!Q?O=9N@@!8Eqe_Xs0F$gm#WzjGbt{$8>+LALec+zJnPYh|A^KcAR1L=FgX=WQ7#}ufMy%VYa1*|2$pYu+e22dpES2~S zmJ=Vy{CpY}c`C@8Jhb!`8H{kS`WK>ma%B7v*t;eNzs^>iQCsLs(TeOnUgB)_5(wMl zw$@qcVL1J(Vp%m4JGH$G(eUN4zC6=>#40rtl^~hKU;9@4Qf6yIoW#$mQd&mL87CK)0n3{ z3ZdaR?->h>FlDhCBy^xxz?i74_E(ihftxcgv5^>CGTQbmE{PpQS^)?Re{DaU3P&7s z?*CD2Xr|Hr3jrB96N|98<;RYZpnPl<-2eSY2ZoVFxTAUnm^zxGj&PfP5GsDQD_Iku zTUz@E+r6?aVEtEqag?|52g6FmO_f(Sk&euL{t3tMJ|Sr{v5v}2p|y6>#+^#A{x^?h zA-e=6wUHPh~Ab zJo-~Kf7OUmZ}n^f`tET4P>d};bW6=Ps&&Z_zO)Yujn9>NcR&iC9cj(ZRkA_e;nIO1 zO>qVux`aZt=U3eiDFNlXMRnI?NOMejNq^gL;!PEBbRN+;Ya%`{*AG0r^J^>Dm-!`C zbsb_U8ZCq>Q5Ayc5u}M}vfh;F zWkFJ!zTTIF2b)KIG1({|^f=xMIu0y!jbV9u&{<@7rYWSBQL1PhMp~3$rn+ zPllK$^S!84(Tp|v7ZDEIuuo-Xl0;~&)Y6BcGXol}=KebFH%+-IaJ@@UDE3Jif&G5#=ke;e0fK{F;xETl#${TlX&gD zkb#;1ml7r_QyCU&ITp^*n1Gw&&*u6EzBsG$^odAqEkKR(Yx&O?aonHJj_uRYqskjY z=KVs8P*U9{(9h*=Lxv}BsJl{*3VPo@RY`!^S`n7)SO%g#DOc0oLzyIsRyf(Zlk2(Gdj|I1Di~BfE<5nk&cO!eU#w=+ zBQEAUF)+VrS|C9@embq7%?c5N$V4DTyxLQkoxqb1f~s{!ymgsC?-jC8GPmnqTg?xK zzv(^S1w+I&oZh!xj@&s_{Ic%9Ip@zr>$Rb#-AgoPK5&1idz{_g<%Y)KAxh!gIq;Py zw3+abm*A)j($Q#=593}PcI&IxH8jl^7xUk1VQ*w~y7AS1lNxpG&mc=8i){&Rd}l_- zA9dzU=LPk1)MV^unXFxehWckaIHJxO&6PS-Csuo0J(q<9;m*}G?sSDDSI_3B@VQrq zthbrkt|mScFs+S;*Fal2qO}?RyeGuACx$cd0kpEy6UsQ4KG@q7?xn`hIDckxsiW}% zt_2ip-+V=uG=KPW(g9ZFB-0kT`&N}m|T~dkAQ!!HM2fZs%QTB6< z%F$nN78b?2YyI613R(}F_~Ig&M1^3H&jedNrXIz2a)bSsL!Q4wxs`pp^@+S(UjRNs} zLk0~-#0>H367)1R=~48VKk7pYN@nfs7nQcfAE>Nadz@qzJCj&+&DjP(DU zdsZbz2L--YgCEDhvsT@dTM7U&Tl~c>WoAR*58o*hKu}8>|Gp;+qqXpav!;Qf}hZqV#kw%UH-qM`G0BkRp4mQf9`V% zviacFfE|ya=)5)yLCFg7KGD=iLu0gmw@mSLSU!KD)vveV`z@4D(5=}gOx{34&y(I_ zGNxgGhrL;o`;lyXVH8ytKJz$zuY+z);I+2S_{DFVeXDPY?&BZQUo>UCsKk@cUTXym zC+iJ3pNI12&V&h3CnDmDIp6(UTCK5d_&7J+U3@MZg+Tr8o;KMi5ig_}?G;0LPe8^m zN6mecG7bg4SxTpm5ry|bZ>2}wz4F zyn^e|Q2+Tv?k%b$dc>3`0MzhCkzYu$aj{Ik`6ifLuF=b7UwC;kN*c3O-lXk)_bAV= z*0|TNMU)Fli~W1&&aWDcQ!;t(UV6ekXP0NW`P(Jaj>B^I52sV!R9}5l4rGGd{m+d| zjC~b6&ts*M7oPR@RT-t|Ws$LTD9UpxUBZ@L(#?o?5zq!ce$))e+Y%iLJ6^YLq@TJ42@nq%i1KN*vRVWRP7AYCWj3_J4VSdJ9)u34Db7df%mDchr z1q-im8*-_c*8ChxEkxGoON~lhL*_DK;kIu`16=P7!&i@}Q$^+y)78T_A9BS5bCbv$ zrY!=qKLRx$8HFfr_OepG;b_YHG%Qe5)0|LWjBTfD#I8Vwgzg#ne5=eH86DJVijPZI=}iz&iIAevHEoMl3kMC zbQh;z6NW)!HXb=TI&^HKAw9DIkD8=~E6!4X(yI%qP=BDYpj9hi3&aij_w z^B#8Y_owOY>)>#+&SgrCDLyhqh02|Fw1#PST?@Qu!x~|bQ^UV{%?4=zMC2=MwQ%LE zQ%BG*{dD~szY1H>>Tyv&Ga&uS-y8#PrH{91@BY{9>ByAWwn06~Z_JM|Q#C@64sNXi zpBOJBPyLLQZirSR{EYB3*IlhM0HXGAr{1(Uln<(I6;b?um&*T3kN;Cl zFT3jF5&$JTs{bR*{Hxj+!OrU`Un^P*zZgaW!%x(Uu^P%&7Me;isY%C*@J+rda4E}0 zxhp4I%&V8Ie}7)lbbVswOedlmc^&!gN)P$Gc~gXw^mO-es!6%Id4l3@ zZ=M0M{cpUa$H}VC?wzPyWmKLY(GjMZ7uOuf+?KXKqJ5QLq8vuUvHB|IH<(%9JhOkB zTqXPGw*A2m`Y=(K5Qr?1NZW;Z8R8pp-C;gJ*3@oUkKDon^~NF3?%!dr zRA0wiFPs7i4PQ7qEBZRdQ|WtleWKUx4d%1uNn`78QOtg=ApjBY?D|C>DNXwKml`U$ zC2w`Pi?w!w4AxMY^+Vwc*oY}%i&`h_AHl=2$n#p-P_`?|vvnY^mc`(bM)Nij?%CI> z>kj@$KD%=>W47>xW3Vs&<4XGETWhN>JUjV@WRE!TpZYv$z`_DrcQFIg*R)IAEC12P zV8$lrNO5p+J+xMmvbu1CDiIuQ`J8)UxiUSZN;w~*^^D6_DVvyF_cNYu6QSW4&;Q}; zt)k-SqHa+jxI4jv1h?ScxN8XR4#C}N+$BMRdvJGm3l71ZhT!fr(tY{vz31VKanAoz zV^`H%)gE)LJ=a`wQPP|B?`3dg=n(5ZhT3NY#LZ(2+B@;bu+wKBGqGJ~<1Ht%_Sa@z z(X%pS;4W0j42*mJFm=*td?#d%txTRRlYrwTZFMwEyB;?-5@r`~(6XSi zY`_7J`dJ9Oe7zgk3JwE@b6Fi@5{#kw2Ez>ShFCF+c9PFIOkQD&7zcjtbSt9M??G$* zv>nZC1}Q3m`>N9E9`k&r6*jLUg$wk!h>Po1S-VkL?BkQcVPiW8_1d-aElA9&Zph)X2jE~I^hh|Ni0akr}y^g5CBDD7q!e5oLGUcB4-C9b@ z5bJ^3YLAUfxoRl7)&w6X^5~HVhlSx2h=t0+qpqkpoC*;_l7Lie zoVKImPUW=T_65r$5()ee=JFTJY*sr=R;cAt!@J>$X07VE;UbLY>@Z^JjWzcbsix2m znNVN7N;)ju&fpvGA2tVOQ&?YeOs!hI+{9N3iV~~E@A$Be%DjaLR=9a^`iqc?xZTby1^rZ0B$Eo(hgeP=ZYcbns7+;lARyev{+u2kBM z=WfA1ECHNNwF(OH*Nwk>$iCCN5(@0d=&Z_Ey(}XjKitmn91Kv4{@u_W%~>Y1-*=MGtFd zY-vv?jOutKdr{%ATNm4dq0qqfJDJR5iKA|KH&6k-RT_=58lPLdKGS#mb@6?r`2o4! z8~Ob(G!}lX$Uwi5NAb!?8U6H;vxdI-LrZmg6Kr#<-k|{=k#e?-)Gpy_e`RTL#I)V%dO5RNM)b37j;uio9#UcJT<6s3Q9teg zlLgJKwBB!3H)M(>_7+S3m4;WM~0k*}NXqD1lP-VwrD}+ZOdwEh*(3C$A zPj%bZh^Wx!COIA_HIW;)hZI`NOHrm8PVR&JaM+Ahr^A^KrN$U6LRDS77dg%`y6trv zlcgkvZMJ5Z0Q#o(CxGm{3m6Ps8rh5>ykL=0JygY>T z__F}LRUt`nKTyAMycF+eSCQ{{J5ZvPRR7x9&SR8W3J;sBmL6H1;IhR2hVf ziH*oI;Lx43nlH`#`g5*-iqs2v1z!6V9y0_M%t*zhT&iQ%b`M97Fx!`}g_oRqa~0wV z2PMVsPQ);L2Y|levSvyzdXy`FCZ{TsRr8?4*;M5IVG;+m^H?I3QY+Ej!UM=p8pAJBH_$cieCNC;gfy#n#GGzGN~gb6P|&5Tu7GF>4p0pKB`Mk3 zF_k8vqw zD=oG(F@quBIScg9%8F1e|25ey*aSY^wU!$It^ukzcNsV?~K0iD%1MrQXu#kB&tl(3xdX!A~Yn92>tuK*Mpg4X&SQA(!RKiXtHR z$$bba4JwP-jF>Pj?yOWkP)Xdlp!&2iY_?5T8MjP`d6quKwz(9u8nbm#))@${zG%nerQ4h;usbDtu;^M(eeu}S{4LUV8nnm+pD! zsAAy`ivqg_{TIbI@o4H(+7+(zlb|5&$8hlW!N0#?H1MBu(q$a-QJlG27Qqe&t8dA= zC)!6EQhBmUMaWqfRHls7tb_z{5JdTTJ~_Q!`Npk_FA=_(~v4#~1z zZ`wLmWzI-INtQ;qOhnR$MVS^?9HVrMNmcC)Vx6rA=v(awwCd@`2W8)#FsX}7kOFe3 zHj`wtYB}sqT&Z0^+z+4`cxUJMFF3lCIqrS5h z(e91%EXlj*!VVMgk_qtDD9H&n57slxl||7LPFu$}J;Tf!>nU2lIVA8HnLf)ZUZdov z;6;Uk>$TO8%#KcBUj-)4@*3l{cSu0 z@|Swe&X0dixUb4q@j6Y&`S9!skE(qIG*6%nE-Z*7pE4|~C+mqjf)Y$WVKV{+8jW<6 zv_06*<3o-8ApPFibrkf!7Ey+&Q<$+_ZSDouvSu2KtmxH6VrrGz+5>K0zlliG8mQ25 zh}q$?&+*D_VOrVz%j#DbJ0aZ0i5hI6)3Iq@ z&yb;mDw0c$od}Y*Y5PjmMIdfK+zNj$@QCMiCnR-0Kz_7}o?AwpbI@aft}Pr=%OYkI zmzN@#JELt_*bOG>{C+!Q43c6e$LsrfT@XIPacrFCTH>3|)f~!_*&;E$t?B7?n|FV$ z*M226zcGZM!;Ruh|Lk8~#!DI#^n)-qY^1Pv#i_9{ir=m>RZaaTe1y*rdr8)Yam=0d zWhbh#GjPqk6ggYd+3tpXpYG)J7q6oDqj!Q18(qHg zBQo1b*lP+4T1s;6Z2D(mF_U4+Rtpk1jP_}VpAE+OY7EV7Y@#NJ*!Nlq@Yr1QGss-^ zTgL_#7&UHcj7=U7lsN_U`L!)69fOblG~>1iAV-nF+M)cM<1IWeqhP0;$+Vnk^V~yy zqj|&KSgO&LuPalNu!E&RsOok-#~sjSd5$KCB@bu{y|bYo^MmheDL}Pu7!;p9PvX!F za>604PLK6Iet{0{CIYOE7i_z(2PUF=jGH;I5<;zzDjHK<+Bp$r5s!=Va+!%Nmm6W8 z=0d2ydX}EK6JoENx;u9?8Va@?cPpz6`Rixj1RjO+nuYI9bs`w6!xkp<@6^pT6ub*Y z${u-D6WWPtH~W)5qgmQ}T`n>+VA31Krv5shL;RQ|1{QkUq)#L!F${C`P}m`FSc%3@ zb3J%M`A8O$Ya~IvX7K7KE^4Odjvkv}-&__HW;m2cB>xsYT8@e21UXoCh0+Xu@b9s1 z4I7JzD*08nLQOtOU1M;jankpVcCU2!1qCO#2JMf%2W}Jp^=dZy!`;9Y2a%vqaf$S| z=B{NcA)Mq7%0njWRb?eWsm6F}vuD|HC1^B~4Ae>3dj`1E;|HNx9e{?G?Pp0T!0QRu=ywQM{4!|K`MdwV?lxml$Tx?bUW>F$q61y<1Uc zKmqw{BGb`G?GYFDpnBFi9)5EJ_*JN4qkKlO#VH*7*e@4mQ397nGMuNf$F6~FWoQw) zx7vH8IL(9M_WJ=HW&0zwuU-dfxFjD4m_FRQHd*cwnq6^jy7V%^sEnvS5?!|S6*rhR z>yYesB}GDYB*Eob!NyBnT-~%x?myYbH_}lp!+oXuPn*rH={L`2_#)ATvU7X7Ti|1* z=>_!pCHEZFlA6yXs`<3^c>xXS8j3(^(%FT|%}{usyqel+dT@eIe~Szgr* zh-))S;49ym30LDnxtk5}zq@f2v_s<81WbmEr7{r18gAMQd^zjM%RyuRyM(;1m6wvJ zZAL{rOs3>J%ehMt_n8QoqGSR$XOitKAwGqOQfzLKf|PM-cvI#a|Fg=D-t5R@Nz}LawNwjXY`)*7U<& zs`s(T!+kk^cGElgv)hS_y9P|izl zv_@(`c1`E_q`ceJw-hx!qmZN}Ie4ksOMx{#$NJ7|{sQM1F(&F(?PM3uaek}bwiFGF zf%i;0u6)A9TG1t~9of?T@B1N1*FSh~s&!?~IYf;ml9^yczXWHJb_|}L;}-lmyW*m# z(=<%A#J?LXyTA`=LjwL)<|yHi@yjMhQ<>Xx&XSBjj6xWXI#QR&RN&0~E#Vgy&CXiT zcuZ1#s`|CnqR8L}RY!B`?=-QI1igb#WXn-L7uNIHY#mGepo3W2MspM!b=-2)0TAs^ z=a3&Ipk*od<_Ny(y_u3LS`U9>97%GS<&m!-2;rvk2lu5%jZSBxerGb)RNAW31}t6W zlKtC3f<^`B*W)C1+sgBOu_;FIYy9wh!=q4=($5>wp(JC+61&Y$n;6}b*-ck=Q^4s3 z5AQ})U(O)9KrAQ@nZ83*E+~JKEvMqL7cAK8eJ#+%i$>K+PHgFb8!$|F%!U!$V-|aq zq^B7s zA4Hvw*rO^XrNQc+2~kS>3VDIem?~!pC0TleGE$jvHS~;ddEXiLBdfupN+BJRc(U@q ziif7%XYOugz>B8?&G*(ZH{|pfHHDmPfz8i*RenweG&apeS+UXa4Y=%e$#rSCO#;qY zlipIQwTpCqh2iFc|LQpRb6PE$Zq2`*wddejF`91tuhHlAniOj;AfF9+NX7Q#k8@>GK$YB8l2GDwC>cW zuQd3WQy^nu+*Z3h?z!>n)*a#bU?Dmh(d>oB3{Tu0g@U#osrNcOi`zv~fb~i2LQA z7Bw%uXl!bugTtqe9|&zNYigHm5djF&Hp9d4taQixgf+HvKyo$~YO{JA4t?+ZD)f4w zOa%Yxl0x}dm{~IpcZMav^Q+iR2sKLdf7QE}Pf4mnX7B~F$pps1#CePTXn@!HI7A}s z&b=KXury9M!VnAHZsCNEJ^OP-wF-Wvq=e_D!JjtZD^kgp7gj7~E63sWSGqtx*`XT@ z{9BvVShAM%3)=XPJ5Lv)VkDGZ@$JO_H-SY^L2$1j#>f9S3dBBX+`Ip!1>o;FE&Ki< z8`>OtTl=_6*H`b@0BX#b{maxZg*_KsX^Vo}@A{zKGH>XX({jM>b067DPf8-GQvKuS zU)_D9zt=A)Bb?rNq7u5b@P4H9lzZP6k#iPXb)S3*1fcL{!DQusy5k-pcrRAK;nfbD zgk{5>*4nOk0QpeX{qAD-5RH%QMSX;*9x2Fq5gUUGi|cJ!ljY3>t&_;ht;92Ep64yQ ztbP0pYYBJZ_Gj<4T9Q|8&iFA;fct`Bni_A#aTpis#c!h4s4EQ3AL_k>!KsJ*8Xh^U zr-0!c4TdqFtAlR~N=Y`pEDHRQ6rG9Q#{t6spwT&)k>gU4B=WV)BT?jop)eg&uSXKKHuc^bK~m)5tMGGl2%=T zHktYu_(L`AeF$H~guRn+6d}o5cQ`9vJu`YIW>k(5Fa3Vtt4i%yh&?xUaf%p$`|-|Jxc-C^cPSX?W7pFke> zH;YfG)kBKi3|d0PTFm5dRx!z>AFVF@DF2k%8wP2^vXXmtsFLkJ@7MAJpJ`_TPicct$Hd}Nb;71+63A;hVLqFgAZgD0CLnkwYbp_+X{dD^bU^M{i zEy^L^d>u9TyhMG2qf_SP`@R=aXXOLSBB>DDxBPN|J`XQ#vE)yRF)sQnZT4DA)q3L4 zd>b_ivzcGYHoTp2t18HHZ21lasX?nuhx3eug_r!H@6#&n+9nA@roZ}o(z8_>3XTPo z%)zrr|8?oIHBWXYi-qi$j&3H7emDPl#!r#|c=CA_L{=vRVSktE{o%qAjk6-e*~J_v z8ruCJgyUBqwgvZudd2KpvF&@#gEAR9ZRP`2aL|JDM5A*18Mpo`!3%bNN-Km84K_9& zVz+#ADuY%pHB z?)9(V#Tf9y>nOpWHtab!;3t>a6K&*<)iJACfR`r{Rhyrpq>iJS2ti^~k$3T%*6GQ< zhxaS#W!U@^7^@=&N;zaqWGCeK2pB>CA1LSlj}R$={r?$fME;(MguUKC=fGCukB2}P z)KbkqdSR6dQXHg1X_$C&L&&4a7G`^8*qp=lTyjUchJnT~d`buc5@ICKpq zjLdul1j1EBDXLUHaj6gZ#x?ya4BXekbnNN7o?5ou^bXx4Wpin}MyJnmn&mu(AwPAR zYc5ZWcAD-_Q~T_XH|w*jYUSp+iEMb^&6539nkBc7$ZKa@5IGLdWt!M5SoX>PUSN6x z`J%fm>2ukBU*|aP%3|Il4Qyg3qCKcI*_x^RYyg<|&uYJ9rT;CfE;Z`77Mb)g(fh&)+og-c zPBFkP_?s){%snC8P08O>)VQj52<$N10wKY`^ZK$+ub2UG{mjbM$*(8S2l`d}&RMdn z0uTMdcGcUR#m1r;F1VA8b;PWvG^cqtO1e*|Q!gDXP9`SO?*>pkrv(%r1>gV9NW#e5 zw+$ge;D{{8d%;yv4aYlV^R`2v9Zc9sf?0UXD17`Wmy;lof{uV}aFU47xh62^BEs}Q z$w=}OsT3nxkqyfDt7^UV7V)(GvAOCX74;nLMOcVX<*u^SzlZs}yqOGNP#?L~QcANc zz~bv5f<$PXUrCan zZM6*|0MfN)9T|}*b zx>OIm*80~XUnAgofP`e2(S?(i_wN60={_&LLw!!kqm$u%p)lt7kt`(9*w3SzU-CDzGo$QQfintykz$#)NreSeT8 z`TU;Vm;q_AR&a0>YgIS0chkNs(oYKHXDdDO*ha?#-2vxJc@Eptxl>uI3o8tMo?Z_2 zR!c&_9|^|Ivr4n})?5E>*fPIh6xYFQ zQWL`HVjGUSZ3B%%9{PjV?;h{#tnN{b-u7oQxWONIogBs__qU9WNO$>WUeW5+rQgvp zGuBzCI@?BwiH>zvCL>}@jV^=H^Yp+3KnjW|)j zG&QGzly|4%YKPAU8gvd};l6w3w3UwX@N<3M`Hc;!iGz6mr-!VtbHYLup}<)1j9`MS zH;ZBLQI%1Pw`m}#Z|c;wci_CcpeDier>v;(q!-;rq@W(Z*ZEM4eHhy0jSIvKNvPGy z%EAH$hc94n{PQYfcfbT)CNFn`%Ppd){|m-9MVv2{;Za8!EkaT0Ebn4J7NeEAny(Uv z{ki&NLf$aI`s5(zu61$-xCh0}XvZUUTryed6km0!!dQJ6$Hc+0Xp4pX_8rpKA0q*!a#q9>A4^)lC zxs&_zVz8I-XblBPMOVkmf1PGm^-hg-mYOoIdDg*Q?a6~#f~(De?9AIWR&&fe*|ro1YMTUBDu!`-Ajz@k z*+P_ewuPl*31fC{V=#NHGWz6;L_6+=C0&dgA)IZzoHJQo&3R$vg&^x`i+rg&gVA`p zcTyhU2GzRK$=T{0hUj$Xbq%M-fNLm~sm%Z*=Tz3FGV)>3xd&H}*U0R;%d!jwLpo7| z;+fEo$bUFG!pM`y16#htGNsljjtD3y#pw&cGuBbCi$8+x>E9@?ME+Lx)WAC+PwHa$ znUr+2BZIYx&nVfj3y#E7bF-xR1hJH_ul@vfjn~+88p$oXl&8zmF=isio9T6^iPP%! zJHok#wvAKAP&U>tY{GyM#yO{|f14w=2&zkwm6NgTc(i4aC?RVc71M zV`Q&HjNR!7x#97Q*8y}vyV_YjuVk>s@=mPZ^?O4|vVbduex>uDaOC&t`pRp!!j1%X zf={3NzB9@3>e012Rd)^)M37Eq_dM4eYFtngt4|BMThh4V2b&h}W}A!CRy7WB#syG@4dUYC_xB@#gU2WO*)?bTW`oGq zOB)9p6|ijD1F7GSYC@*1Y>fGaRu$gA{XobuvxPw?J271@e$6>PDUoAoGl*Hx&T-C= zN9^S}Q1AqaUl$;zKSh9fezb#d{W=Bz@AX5W`76)_LVv;rwS`c?RuxgaH^oyJ%1s`8 zvZ$Ar-A{-Z@n5KOjz>NGZj8n5eM>8OdVn9}bw-40#?(@xWRQ+9B1 z*jYb4oUh~Rpgv7r!T9lUW3#aQFl;S^>NFF&nT%d^4Z7R#Z`U(ZCx%b>Sa58E*ZKN3 z9_euESBmvDWu)K5z%|=^fsZ81ezkKm*dM6dbaR1_=kN(W)nX)YMRjP5nq1F=9YVOp zByW>?c*Eu6)y3wG3mo(z?rv@Zdq8%BCNw|tpNGA%ru_IWt*uKwA-?>|vDXTCT0_NBx&OhF7V>^37F&&k~?8&f#SLg4I{aDS=Ls*EFmuBzr*z6oo29DW*>(bKcWp(qH?y9%!B>-4Nd`L7i34Dg*PBtdf}A_5 zkizO_Lo|*HJ0cRe6p2ELOCRS-(AEQ&E0)5Vv=Hyb8t?`(7KW6R=%Li`R1#a>*{o); z)_kTbESVzAB>@EYpDy1ni*iccckptkQy-id! z@l7F}ADI`p6$s3Hz~&?vKa4m65$0^}hUUzkZyz8D?{&nzeE**b@#=S|ekJOw53PH` zRc>Qs2_Kp)ijI+!X(VqQjGQN+yKKWJ2Np6bqu7&gJd1NXL zDQT^u*%qYDDZuc}O;g`-iQ@g=U!sEM)WeovbxLV{i>y;F2p z6nu6}2_dd{IXu)S&xn-WB51%0eC+{o2E4s^W941vTV2d^!Li8s5T>WAuvk@J;R?Z=FaNLt~ zqc^q}nF!uAcD1?HP2llxah|{_naU2QFWRj@E+`{zsTCOSpE|d9T%{AQW zMaeXF-JjqmoQY0Z3#o9}KBI5g0=LdHP(h1QDem7M7-n*tg1Va>$n3fX=6KkP+ML23vibP z!@`6NvYk|hv|pls6DSl8O2_viTBo_K7bSM%`sV69=odH7dObN&0hyX&2wn$BO+C4?EDUx zc{}_3YCh=QSm>o|qkMN4Aty7^V0VtA_n2+A{>uUW?vu)9Xf1UNJ01e37X`OUQ1UlXRFIhm+7`@bC6uf^5SzJ^-V1 zMbj^R*%+86#ug{bA3txp@GV_zGBI=R>NoY1D@l(#yofUD~2V{x2lJ;BXp z-UYMCz0OIID7kco`jgp~ImMlK1f?)U*w~(bu;5oC*``Z^WLBrmrf*CUsfueE2>(T2 zr5<=_i|HURb&WwU(C~(gb}%!XHOoNQJ(|Jq%Pq!BV!bBR2yQA!F4A{$Gnv46aOmkP zm?~Xq@kCJrioG1d)pWw9m8OsZ1Jg)^2C!T7ZB-?HX|>z#U2u zOa3+T-F-u8`LFk8jo(vj1(qvjvh_mc`mG7M6KN)LeY>z9@l`h0voCD+HjN=9n|cyu zvr2!=#JvP>YPJX9QgG=nAs}TzLA~2PJ($0NE*GJ=`Iz?)&dx6?Mu7)N4d$y(Y)Ovi zDXS4i7F4EgIq9jDakHtW9k0q(yJUGGU#x9ASB z5#{qcbb;5{pBfr-aByB&o+E~P5QpyX>TTZysY|GFsh?j+AM335N?r||ux>3_WLM5i zHg|_zv-;7D3p4D?ir$qqPSXI#{5<;61S+=d0R-tG)Ttr(6oQ|-zo>;h(Q11inBGg8 zAAgM!apB*R{Wz?v|4v>Z0wrNEIKxTKXbYO9WBy(sFPOH>u20v0{bLqe@?96>Y+R!> zur4M}RX~m7kpqRHF*uD`44ZM>FFwrMJ9}?cx{tNE$+hG2e$e0UXD{qOSJ|b`+_0A= zcr5s%)ZvWp5=or5Pilb*(awEhhl-s#4Wj#3xwtH8aKC|Mf5$@ba1>#`$bJD)M{bBh zMV}HcBcX>~>`$j)G82PZm)id!mH&^MuY#R2kdpFO{pYN476W1@O!d(kI2aIC#otCE zSlxeA+4bwFQ%Y|IL3t{MU*{K3Xq98Z#)*P!3X-cD5YgpC9&SvM*y~&m{c(`_w!6 z3!E-Fkm-F!EMmWy2|O-H z4f~h#N(1&M`$+~eDt1&-&<|=BvIg3$VN=@RK6{c|7ys$0vx*w++Rnyjp=~}VBZxCi zC8yJt+hzMQ zh?phA!05$vC@jJw<_I7eME0MjUN`DN}S&CdoHqiaAtA?vDMilM4U~fKcx4wb!-tn z>{6hD@XuF~tv13hmghV&kJ@2#f&WOart+Rm{a=%}6K*HC?&3w<-w+>4$hluJ(e-7W z?9E2E#R0A+Cv4mrkq3MQ!z6KI*ch#mEh0Y}Tf2Mc4|%l*F*}Mxl}5$#E<4wcaNL(2 z(m#ASYqi>tu=_h$<0Mz+JmI z_FC0wS8czAj$7^+*;xS(i$ornGcYRAA#NnYS!)}|ZOsE$9Q9+9(jEZ)@}J2pF_`ufNE%3=GJ zQha+;i>jCp&6U?Vk*Zjc^2b%TX?>i0GveuotdI7oa{*mcLA%QUU_##Omg*kY_8X%= zxKWW&a+_s<(SC4uwYRu1n_bx7?t^*(T;k526>PVJxR_zJm~$&!E|)#wk*%I&rL!HBj?jJ? z-L8~Q1P?a0nU!-#`~2qiXvg9ZmEss6RFZPo7*eCU=j4Addc_5J-h0Y*dWL6{#Y1ng zo6W@^YSdn5Eb5d_JjFe#cM#qRYwV=ACQXh6KA0Ov_db6Bq5z&n@x^wtRTq0e~6Z%wrxk4Y|FSNtk#qQ zvBQ^!RHW)g&7mDu0>Lb6wNw49zsB$u$2j&PR5l&V6y4K~$^(*EEiD%FVs(v?2UgGC1}nQ1S@B)t+{gTcdwtH zoH(1@>uJwV0gMXkXMZs_as)pf7%)5aGZ1lq)ddD-~U=b5s)ypHSoOay#zLa|}l zxy}nznHGg~LKkPobw}(=+vQcksBvFmq5Y$xF~HYeNQ`$H6!@g^I(H}<34HVGO;ios zqc=HQ0-Y@s0LS#Bm^9r{RJ0#C7&LK&SA!-x@F5P9TfSkn>>>kBZwN%dCmI~cSYp6u z(QEsdK#-!zFHxAR+(p&zb!eSqvXhyfVi)Dve%}u{e@uw6eP~Dj_kUdU5mlpOtI=IZ z?zH6x?Tx_XWWpu9pg2$NO+>&a%$JSqmUW<$xt&X=J_`B-0nAo(eTA__=xHJljL>$a zC;hZs@H|aFToBeuU+>yD^404v7hE>eQf?P5U!SI}Ty(wpKzqw?pmkpqchX@LD!)b? z&wuzU=J(x{-La73Tn_ojXjXQVYYtU;VH2~XfGUVmoKeKFh=_ak#=S`f_T)|&{qzXl zZ?=%_MDjx>Wzl)Zu4a{=V8qYhgze$nZE5<0)kanu^i?}7({?Jv5;P$8`>1vt;xuIa zOX`!ZQ1wSt<7Q~}Qy%cycnA6f1`O4{E%$~yy}50pOq~`IaGtpmWjdA3P6*HQevg)| zCeFmg)p5gUiX?HbWruoPe>+EGD351>^+M2T#7J+%Ztjy}UK-NK*lH$+i+3Tq&yO$B zjy$+5IBl7|M_(wy^psHsDbAc0aoLDroebnTE=aGR=ARzB5^{m3Pk2io&wMjdEJ+#+@u0CZB`>- z23aGmp(HjV%I<@h=rj0E=EdL(tdhmvLQf|%;sjNr| zE#!Abc$0aV-ROjZ?l?Z`9(|>w_RY+)c|!acv+Dd(w;?+Dr_V@DmIeg$QiinBeDLKL zTj!ng@H=W#Xqo!acZ!K#in2+Zf7)*2gW2Io`N1*is&)kKYDpgd)`GF4r^>KxUVU|&~|mS65iolcNs?RGU> z_7rRT8h3gt@BW!S+#g}!>c($tCrt_Q#|eU?aO&qp&dI-sh{dewiQ6IiY-mw(mzpvi zN1DtW%%86%EI;l(kLcqPW<`|DmR{>wrth`xIB56(SR6dB$j+#mOHM6_LP<(v;uz{| z{xowJuZ&W;!Je!lPF+*i<5-MZ`t@gHg7Q*lNA(SXB1i0ar*7$&$wrss0!7Jm81Mld z8C>l!?m!MnT@5PDtSr|W?$eRF4fH8{0@U4dhrg$Owjw4IBiMNnoljvzr}tk?skF1qS*;*70sWdHD+$VL2RZSa%xHi z+kQA5?9gv2n6Uhz4WD?8eS^P}?PWLGd)lDfdklPoh8bC`^Zl!G$aPY3dYbV+ zSq}z)Y4(pH?STH?*QxD3dH}?Qd-dU@1?GaH)~OpI%XAeDz21mGPo^HewjsK;l>!vm zE&Id=;2nm(6luJ&Z$rO$>pS&VoSW1>>F{#4k*+((uGbb!K{&GZ?_Xrb*0UaU|L;)y zj>5HH5kKZ6sGnsDdDypd+ceW?XnG7+bzkO1%7( zm`*K)CmMz4Cgl5`yZQjH;__i@1P`7RK_FCx-7&P!rMpJFcJVV*P0X6DsmF*B1BZV{ z!amp(_4eWL&G^TU{kOkFy|1+ZqvY^63Hbq6-U{jWJ%B8<65Fv`zVef!1HMcNMY0lH z0+Sa}Dv8zsH|_ynRqVw8Yi4wr)*+smn12C7zK*R{%SUE5i-t{1|8%Abm3-~ad$xW` z?R~Dj!UCRPT{8p0Ue{T@Ft2Od5c1c{-aGxLyub@>jw~`{<)!hY}+Lo8cS34)OVeBd-CF-5HQ7ByW4@68a1X9Vs$Cs#mG&`G!9WMd}47 z_fbM%1NXGwhc`V(LXtKPxk>_Yk7+hDzB>_^0zj)v1oqmBN8jw6`C`2vo^T)+%NILd z0IvSa*CztVd~I$YdrEz)rc9%{q`4cO_)m(ZTjj!bCI3K6>O-gLZ39GC;y;$3(t`g_ z3nM;axvRu#R78EAv-7jUzvYsZi`e!lAOK?@BVmCy@7zYg{LrtR(i;9EXG5Ew+}nQ+1nFDhZ6+a{Tdq*ZkTR@fLjK6`)?Z^~o81E(P7~8r4C5=Blt58A4pXvklc;la! z;FWzhP2l}jS9PrAw54!y@`1*=QtrglfA}IJ@NtE%374<_n~9U=EPIN{FP~RnoDu@zIsZ?9($jF1suvXBwnoi4(<-Q4<7$^Oy<>Zm9wT9;pr)!Yn~hZ zL%IdnL)0%z-E~_s6OHU|B`83l4Z7{7&Rf5Hgf9J5uQMz+o5WLD%g@RKqbZj4k5ZXG zUYr$PKYbaA2v!Yv-TP>NDYi}BW)PxTUU*$h*%$eUbwzIcc+b(=rlCn*TsS?Y`^1{N z@LO@_mi5k)X@%8EVtq^U(uKHm7>Jhl)?)p12ms$drBc3<_)l*1o7fSJsNmgy7r)WJ z!!1Q>`E%Rhzw^Sk4k;97xe*2DjGy`~LYMce^Y26a*_QYmfDb*|{uBT0+d|>}o3uoz z-=Dtbp7HycfWx&#u9zrs$qz!|OWz#s`j+h9`tRAR*{Hi#B>TTN2hV)Vy~GE( zq^AhA?GV}A=;&e__0uW#5lO~Tf(Od_#?3n1CoD-yD0us6 z{i!_<%o2EP4G|3l)xKdp4MWc^_naWjGrL@XCyl4~Xou@i8EM3g52GF&$sE*V7bFPu z^ve4uJnnM`8Hbs%KFBx`DNXdu5X)(hj59;flV1Je+aAU3!E7n;qRs3y$Z($jDE20s zb)=VO)f|uxD1|DV^`46QeS%g?5QGk3e)V%TVzX5I+*lWHWJ|Qkn%mhOALyYPh~<@K zBeI1{Or~XbB)m0{-w+%4Zb~-D+cJ(Dk;(boF!W;|z0m=eo6RZTcr(1c9qn+8uIJi5 zPBlC6O8F44jZ}X&@4s{F(muu=)9=pIFe)JQ{8Av#DE=*PUbSbQf&tnNnYf4E>x>`v zp0$XHLX1wqfMwu+4~^e#c87ha|GV)+IE5~o^QQG0KwQCoLFio>w*Ydy1uKZ5O)(-< zGh#~!Pw zIJWq(Pl+LpRyn*~Wjfi7fn=-B=#~mmCx(V_!kC!vqy_0jiecCU}!2-P6viS*j+YKIp9m(w8otatP#caJziF^wpw3_2b za;dDzxbS~#eziuv@oM=NBv>0yUV!<@u`Oi@Ut|v>926?vH`@12;3Q<#I!cXPtJ9Wr(AR0F&P)91Zkyykk^PQ4s~U8@g~ zNmw#^6!zx@nydd0dG8g~WYl$yQU#><-lZ!=kls;HIsyVpM~X`Cgc6E?NL6}o0!r^a z1Vo6`(4+{`A)y9Hs0qo>`~Kg5dCs{z8oYJsLdc9~|M>mXjR z8fqxiOZrNu5|YY(K9T#>K&$?W>$kG#qvcM$oip=0CEirGV9L*LSeQCHyJHAh+EEc5efRdG*z>$n|uF!d%P{>T-5 z_MfxrGK%Zwwln*L3aj>lMr{T5Oa8GTUOcn5E$;W>wgFR597y0nln?w>yhX!PVFY(b zpGOa&^nU5l2e)94)@a$k8ss)$GL;ewA3~Qo4LNtR&XpoIH+}!Z>!V*=@+2vKCH^W_ zUt{5|fVhba5yTA{oVK=-v$HV#x!0l5#KXS*OAM*e zzt#MxX%e@5nBP-Wr{#5B*yFZx5!O{3fL(iqwSmr;c3>uLWIRLrtmhMP=MxZ?S9)+p zIe}^IU9Xl#{2FWZhF0R1PO|dGsEt5YrquXSl?m4&gQeaP$^yBsiz~elH+4o7m*18v zt43S2!#l#8d3sbeqr&C#-%>e0jpNxqC)KJ>ekI4yH`K(dcJ5vmUXjaJp$jbq5rGjB z$U||iwf~FNDB`@taP_YL*=3kGTZf9B_=iGVM<#HgB>?^i?8ltETqKd*cK2kX;DaSa zRMuKo;;;uad+ap~=drVP?#>AqL##a8YKM(E*CBE3X9z$$6=)~uxXfo6xzjv<4SaJQ zr1Bil_5X->YKTaf%^_5QjW}d9DDCJX^K+sP?4qa|O`BzJ?9k(@8lHEuyo1?eh4v#d5n?@{J8>pCa&1+V#y;qIu4cY*O7lzz`c?)5GW**jq_iG`7av zi~(Wt4{yKV3b12+iGJ8b_utycFsu!b^1%YQ3tN$L{j(M5ds}>f`|~&r3cSWZKH)b6 z-l|m`ePC&`W$kSJI6k@%cPeE65`@{&BQ*7b&PYc2f5An}Lsg zX-xh4_g^msXfq;Iz(B6^(b1ZyQjS^g6n740n_K`RKy|7o3dT8`So(QB(10phHt54< zX#t>m=8Zz=OZ^I0Mff=l-4p6oP5oWz)mc{v4D5kwE5 zuEJ^MT`b?zAC<&=`c5_xq;MbmxC0acDRHGi-e!P4 zkJU5!4dspmuGIN7QnCGrzWy`V2CBiwvX7I#_2AyhFzU=w)uV_i(pwd=}b@_aPldo&DgvNcqbL zpp#ZnMW6ywssi^S-*yCaMg_=}?ItGpWwd_GFdd4)53V5S^`#x&%`UU}7in^*N7_cx z(#p>T9sGRu+YU%kyXiU~x)dDIQg8LHZnSt7JTm(bV>tw^y={SBHriyI(6X_Wm~* z(Lt|k>^se4K|_AHYB+Pv<_otBv30+=@*_r3cy{3Q(qHYMs@Ppma?op=Nuw=CR%SOu z@eJneH^sXdqncp#oN-0LlI))tT>za2c%3W}dnCb5BL*6dQM z1MU?klC+*h%LdOsM(WvcMa-Wa5OnUGj}L4}++*$J!WG4p*6?q#LMrZ$3q7Mq?2#m2 z_Wh>a&>Eu=C2K%DsA-kdZR?0Q!Il_@q+#iAMFOFK3y9Xamtiqi*TFK7dPgl87{Tzp z@PL>sIU<{~w%?AJiTb|47bAhh3<$c?V?;0Y+IB~;Bs+0RH+0Ia!J_E-^T41QJg*i! zTh_9|%j<&coMMCztTqp~z(GRY3g#ZgS!3<$Ffa`C8Yg=0B@XPu?3`Wh9~OsUcD5wA zf!9}Wf^Kb45HV{UbWhFwR&L?0(gCKQ1Hqk?phj|!o)piz1gt0h?N6%lt%?WNf@ zJcgRqnF$}1UuihhzVev}$ToBB`+1(N(^jY=Qxo7nO6fmp@M1!jK6F{1axQ4!%^gu{ zvZUqL6KkVdlmGqj-S>qja~%h6W(U4Ih@Do8e}f+H6mX%pIN85#`&^3=^J@Q7xca#( zzwK?S4s|mRYI@U9syau+aZJPpD8--M)AUpHP4Q8ZnYwRqs=V9vi>QIcQ;|}kkaJvD zIW|VNwGltrLcypjHAU?yBw0$>-L*6j*_7DTbUo3EKu)W7J zQnJSar@WbZZ{ROR=^v!HSR>PV44PVISDGAJV zymR{C*s>7RK3zTkD+Q7D+AW*S?Nwk?ht0FPzv^`}L$+;k!-2Ht?*(#kR@i^N)mk?LRiJ?%d(bqxe$P9 z#SrFqbvGcxT&q>9XMU%)SC2GdtDRJVCs?;^lI9= zH@l)X;7okx^?7LFk$)JDYqRR$z?Qh_7GOZm-Zb$yc4YDTuvU#;~z7r<%oU?5@^|-L2d7Bwu=H>4F zpgf6#^J#luq9Fey9z_k7LcULv>%*>%5LcEIBS{Xg%`n{rulnW78S*v}O3Jmax26~u`?4Jo zqpd!@v{!^GiDYbY4N*?S>+hrDJWI$*#-{!(MapM_>D#ALKDa>wTFa?OjXPpg|3<|I zN%^9H=XB=c(J`FHGprh`<0W)yRC-(!HV(bK!nQrEb0FD1!}TLlNiKksxB^_m z`DKv!UFfiJU45}uis_v^FzUj<&WcCR6z>W)@JithO8Wj*Fr z@mp)(;0W=5i>}HM+#m#Q$_g_Hi(Z1`8FG}<_qfFoy`#+Oiz=$o^6Mt^b}wN)t98FUvPQViR0fRS8pL5Z$VcULpQKmx)=_rT3i#x z!3vCLUvJGWmhb^dQVJ%~06oLZHQ`1iR*!0?6 z^a1GN?7NyOv*aSRIL%Eq>21|XwAkXbDaEGVl8L5-lZ0X{ci~{F_Z)m}ff z(Sp%{!t(`c@pLiZnoz|xbfJv%K>*jVt35itsVw&{kHpDoU>hKZgb+Ix4t|Qt#xm06 zsXJPemY8HoNcUY~`^GYagWXMgUrr?=0L=da$$kG0pw#AUg?n%WOp(n#?awQ1A>*6f zQXoz-g}M-PH5lRk#JDJU=l=J5pP15A@h=I%FN4$YN4xRDkb0I|VwTgU^1Mf^&Jyw) zvE%14mNtcO)%M|PW$G5SRk7OxQy@6*Vj-ZATOQZDe_=ZB#=NHV=sdHR+qR!vVq+u| zZ1IiR-BAEIe()6L&41^a9Mm~X!jh6=pKR!mJh?rJ>xII;jbOr35#al1bYcbW&MkQd z<#!vTl9lB@208}cVZYgTS1v*TCc(~rcLI<&F-u@?*3y3v!Lc1gxE&4(v5Q{Xz7bkb z>%v-T;l2mZ%k@SUu9s$lcmwmqdrvbTtio;_^RCuX;K8BzN~+o0aSiuyVV&5Xe*O=D zd#+(Al6~qNHhffjg-El0hfv?B4HU%%oOF7dVH(bGdc`qt#gVDay`w^pg=3=j2(WJd zP?CQywkbl;NG#BF$1eHbp%YYZ_ya?+qq1t8(y|W!MI-ZTw`&!DPR?-XQ;GeAO9_oF z?WZ)^ksfmRDIwxM%%b{F@c%K)6%t1KQZ;);at-`rgq;T7aw+y@7C7m9?TS-?DF?kY zE|;eWikil~mr#MS1DL#qNm*ItqEPocr*4HhJwV}tPP+HP9`0M;h}DGLLG^xEi-$xz zV%N{+@9)6s3&A5>VLP}TlDDo{)S{4e;^G@dH$sB_QBvmr;R5tYASw|RM>%$0GeyN( zb)$yRZ??i}d-)j>a{K*&G`ofK9YjWZbqk<ktP!p+cj zlsK5OEqsvq#`8VjoGVOBy;Atth4hk2G#wyg$u+PS2KZI3WrL84FDe2Ee4wktd zA{;It-8Yvmh1{>>4{v~mIM2VM{8_e5i6N|_xe1q^Gq9Cr(MPa+oZZ7qEO-_$F=ldZ z`An%0VLPtZ@DDBIReO~-7cA?9#A6Gk`W+?V_q)Z|Be23F7pHV zBbY}bEBp4+AOgA^fW#hcZM)!HYqXdYCC1P+H({5~JClg2Fss5#!P}vTH<(-BzVW?U zaa=o0?p^vY-^9>b&EDFpi?QN~>r=klKmML$m%fVII=YDq%-7Nqm&fh4wg;k_m(I=- z`|aj3*@%gQ#_FSQ6~|&X)hh4QCJrRMmP3tzw(+gvi3>XyZ*Ret^NL_b(4Si+7LJy>UaO&MdsVGj4KDF9~z?^7yzJ!-ohMppQg?YAu!T_aj268-D|5}?X}h|)rd zDJ%Dm>YlQ~zcGH@?{b&wtN&Gx*Mb+S3eOzO(H5uSMF^LkS>NV1t3Pi*O4(Hk$>=kO z+Y9JHsNL(DvmF#H%sQL7<_fSDdN5-hhUjax9oGp7*fb^DB0oH|TJ)fDPL8E@=hXI3 z{2>8%(?e{^Nd1&paC9qg;gZM-#f2swSzcZOP~29t{EJ&o3hZgF96B4U7`XnXKQhV5 z6~}Pl#*KC3_LPibKIUiPbgXhNtE5bCN&A!gTbWtmjbf(!TGBblb`x#&ZW1Zo7xq7O zm1`0#ra^v;a;&Xlc`e*%OxVp&20%`^B zzBK1Z*F5B*{~$NdXC2Zl5oLgNijlpcp;Wy){j2!6IQzU3VGLsJab)BgpsI`p* zl6ndz_z}|=N?%SNOGqa^DjLIS^{Jhm;sB|@KR_g6N@-~Vondz;<$EM#Iw zA{*@gu+DD+VyB8_I)5?0)`cB8hbjVu!v7oJN2>jpkg)&6`XAz5xFd+vfj~&LAwR#rw}Aw&6oy-V?oyvNT0M)EC?9cE~h>#J(WPw8ZF*kia)q8%J!hw#FMSYA;w6 zjqX>UNnrw&-q%Q{tB{8k^T-_h=q=hauPbGzk#`NTY1Q1h9uJhJ#T-9?TG;~ZGD|hz zjd?-SZa&n4gDoz9@#k`t7k&(SsF!`weB^;YK~_NDD@Xi5$lee;OKuk8AcmE0mA@Ww zMGaRj$S$kr3HpaD#BKbY`o}BNP&HXW^>->p5EW^=b!ZoT);=YP5!X+Bk*?U3`g%aU z5XdQdgHje+uhRUKu49ER4%lU#A5ARe+Ns7K#>~|ehL_w#_0nQPGQ=);mvyh5Bw4-?OoU-!3!~MFWW`>%&#vRj!Q>=zhlHL2Soc zY%*!9C?x*MAg_yMClmK%LC^CwLIP(Jk-MHhiIv=ZBRtUZ{B5jWr|(m<_KNy*A6D%s zzXQ1y7yit|jFprzp1rVD5AmBV z&sR1n>ZDo@E_5R)a+2?+!AlSD5rT*&z}!b?Gg6u-R!(2cOb!8ju?j=p0MUikTgMK; za6VeUV1|)^mw|9c?(t>K*Ft(CN3Q1F#|-9y??>6^HQE+j?1b({LC@5dyK?Sme)lC? zxDCG@yT|Kdzy&j|~jiBFioPu?AReuet?sAmSg_O#MdJo>C ze}~lf2CU5@HIg|;n`a&gHM&6H2=1SM)qdGmo;~KX;OI*h2HXQ|dc)**oaMWI*`HCw zjF=|OJt=5TCYdp8TP$Isy0>MD5@K$YzsS+quS2n(D;zup^fvSj8rd6BJKy;y6d4)z zHBQ2)%b>}#vda?6@>$&{Y6rhAY6?%^LzF{CRC#bD+o3mk;Hn+6kiAU^@)}5Hof|1?q}q<%M}&aIilo-O10S5*gYF!@pBE~bzwmd+*jN8^c`I^IB8#z1 ze|EhQmN)6V7a$Itqt+G7=qniGclHXu$&fhv>MYDD0~RgJn{#l8Fxh(ZEQWo2t|#N4 zWVa@Rsun|EG7o?~0wY^+r8{ZQo}_STd|4oxFRYk-YIHN76jq5@)t?KUQF{!jP?YQv z0*Ft8I%)@2rqmX?<)_E|^*h^xc8`jXyoIjJ$cpa5bIXG4S=F8pkcL`Q=u6Juu8;fZ zI9029%(6dD8rT?w$Y)DINo5m#$HU%ZjoWqJ6H-8!bE>+qb_Hv5h@FedFk?V+bF;3k z=bYMJI{T5`y4rbjvzoaTQsG!LneYPidpp`DV?5+@-=?49@$P^Aca|+xC?=Wd!hy<+ z0D8A=l38=sP~umaGkw|DMmKgNH0SSJ&_>&u$ws*!>dPT3PjH+u$l~!>vf_8~HCm8urbkibm0M2k%sYkZIe5`YR`2uOi7juuVV>^CE?U4K|W)--#t~q})B7GrN*dP7i z%bLpBr=v}|#gAe79&FkV6|=G4bl3EMXHC;&J$PnP%k*9xa?fXmLsZ||j#A-j6rN42 zVdi<(eh@xt3Ypw!4&NXvOAC(93r)Hu|I#3-il46bQkFG!$jpysSsM3hTV8HuA5i&>%^ABw|N5L)VUWZeOI+*+N<*9@V*{6da? z*Qbm*ZX}jyv}4teF3aSYE-C|(HvTX&56VkA$Q7!k}Zi(YF zNdi}&Ew%BQ#5bLyDv>(yj9&G_CL_1<2RD(opHN##nFNC z&o;a3$o+8Vpx#@F#g)s=kjL8Zq!*`oG1Fx2mlTI>@ny^3kztc!QibMoHx}Ky{a&8& zFLs@tT)LEAYpz<%;ZosWw!}qR&9%7@U&x+Lfv$+!Dr1l01V%5NufbD@mVf_Ked0>D zTGSNcrIccxK^Uq*Son@Q@88eHwKz_SX@9_!HGQ~6yeB)nn@Q0@LZQ4f zHgN8xy1k_Oa?sRiaNs*`q8Q3lJRU95%jpXbnvC}(;rKuD9PIFf9~t+v;P;Dbyn@m; zrWgGt`W&$Es$U8rv_q#n&q*HVnt16uC^uD<^PSkdRmZtCnyuS``VmxpS#GKsJr&OU z$WBMvv*g*-cA8f!9p`+<67sJ#|Pq^$!_^*>%rE$uKl7+}HMmA4s zxiGo`^x-7ec8P!oph#g6@+wG0M_rpBh)o0f9Ap$}9AfVDe@hOBnP!()UbLCmyA8p6BZ3ln+{@W z|LG<|e-HA?W$m?h1I;~Cg{pb=F|Q7tmY+-|u-12`7}h$ahO52qz{S779q_3UVfMle z?eD%!{PsPc$vKnNoqCcXHDa!oTcW3S0ywgtbm~H#^*t$(7?mYVzxM%W4yvp}<+K`c zCHuNGL3@Q35strFF5CKo{80lr|4GSpea4^9lrlOVG6htCzU0a^&2Ve2F%L(3>Ze@( zo}DM|EEytDx~^tT`u62GhlT0Dd^d>AT)Aww6Q@B(a@=oFg{&vVvgBJN;ecJhuslEc zTzOQ1(h9}g)lm=ILv4I-Mw-b$8Id5`70p4z647tDU6M7#9CbnB$XC zQ(uYzUg$GDs5QptqSF>vjiTq^E90HOZ0S*uwjTm%2u}%IZ*$!&d`q_oBA%*mQ$&P` ze(`2midIr03lu+#BIljIqB%a)4-AQu1}qmju%uxH;*|B`$RC_4)TWX9WzQf2gZW@| z@t6}v;};&HJe+E(ub&IOq)z2t{F)kOmyw@+`sSSe9ktU8_#@*Pf3^o~a@Le$i78ro zL7i=f#|cNzvdg!gcLu>4$-R6ydN~#`MglvpBoxO8`m)sY?a<0vqu6dpyq^Zb_w=VM zLzHI^thQfWt8#^7l){>#U5(cu(ZQSwT~(a8N?n-X8peWZScvyb$Q;!>iz|KlxsYE;y14Rsl;l= z9hcSPwMjmM8E^s}e-q;xVd5g+@hQdCF2zzv+)d1u@&amam@>^>+`k6=93emGtvu)* zh_?rxP8es;U>7&^YwCmy-QnXt{ri!5pQTcNC^B6;<;zDnQ;cDL9LAbq zXJn1NOg(2}?LC!^&(f1suXS#2_oT*ewYu(qYH6j?>fQ!3i-B@e>{3YPleZ$zMgmPH zb+F4holvQZBL0hFe$v$_c2HfJ!S5@RB1$59z9Oi0#36T?oJwC)e}T9xx!PAZM$81( z;_Fp_q#7${9wo>%QaK#_EZjVKTtCUHPRheOvDmgFj)|g#!U?HGYWcv)ACz}thL%S}(>h@=20@K)(~QyU z9>G!?%Oc}TEH@dnC&Bl2;%Q=`&n{CjRiDB=%QjZLKW;pIHubh=@!o!H$U}kg zJ%w@Bs6{`H@@OiMu%hMRr}@TfBsTSf3830g`fV^0u(uf7@`G}WEMULv_1-L`O%sf&M;bV zQ@>aSx@}qv(p^nQkVI%b3@a8^((qt%?)w1JD=fwj3c1dCd4s!AQjZjUqHPImvUs~{ zf%h2FFsId5cU@6mqSNkN38WEp35-|P}s zR^8u*J~30Je2Y=ObX58H(p(3o&mx~aICJ8bJj|+~VaRrCx-4CKDM=%|{{mDXSFL)D zETAryV10gSoT8>UecT>*1{zCNM@(E{e;)Pd?Ad2(0IO(Ep55b+5U%OHwZ?So86&Y4 z_YSA-(@1N#F^btgL*mTWJyse$M)3ooyo`F6T0nm))4)d{%(fMb+jt9J#ZZx z7FrO)6a04gqfx(4AzV#ow&tt9>GS|WT&m^1DU9!Q~C99v&%Um6$+gDt_9<_mwZ|%W5Q6;SfmcYagrtnpK$X37&7-oduXuh~@UtDmepLFCIk4ygRFJ}}BD*UH; zRdP?znjs&@lr-2mfz&oo^#z}4A5Q;7E{<)^GuRF#IAQq>6tU z&%afL{!uo&61>!~%R{7ZnC&mBAgJf1QMkJU1X0FLImMa7@^R&(Mq|-=S?!*i;70kI zQYz0~_;uTZCmGCrUZsz0;aR_708AB35md$;<#b#m;9VPNs6@oJGss7rJo}6)qW58h z_sL~-#$+r;xHoq$T0U7zu*lCh$_>Yf3Xl8YaBclbSyM7pl;?4S%mXrQsZ*8cm%2Fd z*N+_TeR&UF!Z)jos11P!-p45K%# z#t}h#7cvwl5;xrx>3kor+blB3H;|n2ExrRimZHqm>SG4+ES$wK_<_=>)`gZg4=(nu zv%0t?_5tz67U!o9rFT}$vt0S=i(K1cWp+oU9 zUznTmq&EZ!#9wmh0#h#}h#!1Y7K!ljF?^)%9c7c8?_X+v%Aa`N`RGT4g>$9ydKn!M z3EJ^VPhw^*ug00#4msg}5N7_7jn8b|75w`jVr4GPUGt>MXd1c4>6@GM#mN#&G?2tD zXvw5P1rMm~?IV7hU?_D$tueKjNVO41@=_MRuLw%~^n6ll+vsi9me%m^eN|IM7p#7A z_}Q1sIX+&wx8Iwalhxw;`IaXPSc%>_^WO`ICP)$S8jLjIF{C-ND}fnGD)xTdNj|Mx zI|q3Sy`TPao&$_;<{mMtGgAYMkZ0lMLgK+D#h)e_L6Ebh zKtP!{uWm^3Bt(+AGsWJBwDKi)HL(e?E_Xsl?Qa8u_hN+s_azjW{y}2f`+}cxe!8qx zQ^4QRVOqO@mWDxdNHxth z4ai?}1&{DK&d6mH+h?^$|j{`IHu){Z>$S|MHZ%+2cOZrVAbbCzP)w{rY5n&-XOcZtfDAn$&Vt=C^Qlv#NYn_RR_f5mFK@^ICs&g&HR)0H21 zDl1Qahs3OYA#x?7(~*^GxM8u! z^YeL+-W=qEHqKxJIs50VBZ8x~7i(I(YCE*GD5vL$Y5TZ_>+aO89_%JCY2?ms4H!+U zB1}B~O?9_&lKyEtqCZ~*ohz%TlHW=Dhb;B<4{lX%v!@RhFS!iqlfas7_WTZ}k6dVFudILO%k&@<;}V!Jav5rRj{uRTNQ{`Mpq1{fZO@ zKYLrBCknM5jSWkk;a99AfqlWd<|pu5`Nn_@{piO5QN+$y+K1FD>MmRP~$t|)6h zo<=Uru`mDV%pDs@I=EROxkQ=&yEj!)zfg&XS`7D8`V9@?U&?9Is|9xCoSZB=6G{4? z6xFwuHb+3dHWF|l0NWOJ8z6-8Minr^%SMy0J ztE;CQ(*@Qb^v%^Ss&wV9o5tzqt)ix@Dc_s2djy&v4_;jwV_^vJ2nHsf-gGHQ0O9y< zCU{93nPJieIb|mC&M|bkQ=iB^0~SkxMU6xyh*Fg3R6lBez+0Z4?qH`A?>Y#S^roaB zVcUO2Sf@pPZlc#4oR3YxOJ^Ij>LJ{*+Iy^DA6KMf^d?S2@T4Qz&{|ulzKoy8UJu_D z#S&=lW~VTiQ{p%ELX&!y++2r$K11%EwT{xNR=x$9#=kBq(3tUDoxPE|tF(+g8lPe! zRvKlWV}K%i4xyAO=ad;_Q*2MLm57ukoj%Bw)S^6ZjH_JMbhR%#Vsq1G`^(SyZ&4lE zvQA+`@sK^-6>ktxy3y+JRJL1oz$kk45pkd{=~`cKi7Y?BQFiJpV1Z~{K)xutj{ts; z6Q3dU;R1h2O)1xhMXrPI`uZ=H=_%WM$SDC&bheG|7tJDrxtjBMMAw9XdF5u7kN%cOUs9 z4t46iBb%5%3md4goT!+>?=js~1NUv*QtnCbt0%INdC9b{sJ^BG-Sohd{AEN2ZK>P0 z2Ilmy>J!_`ONWz5rfC#amOMyMZU`kNfH;&*&{AK)L$Lo=7WJJ%rM8-qC+R+q1B~@= zSjo?}27Vj?o8Nc`7u+W-dlja$&3958QI**GzVw11ytrRLV^i9S;am27W(sGMl%Drh z*(J>60U`+LVuO>b3UUd|$yPK$Lt;R$6?ZO}tnnMS=gVEk@Z4gAM7Eqn{JG>mLb7K= zioRbm`Xzu>=dL-T1RFnuo-nJ8Fq7r#su$RbS^K{@n?`2iS?No3pQ!gf>gv=w`NTDj zl40^TP;Oyf{;~fAhM2g^jf??o!cZYAJlCtXNOH$kC!@EQY8v{0p!k-mg(9t5b`|Mp z5#|@2>pr#p-+twP_+qpNiOhp_O4;|hJvgs~Wj?%%1HD$z5U>ya>kua~g)$N`=05n2 zh8Mj6y&O8&99Z!)DZu4ot*#QSq;Egs&%iWUREj%IVpb(lbIagg zJc|&XoQLq95`p23*s{a)ooGFp|M8oE&P2WJ5yyQurOUWi>n@7o3K3oM z-&a92shiK4boQibd-<8bA2L_h9V=(5tR>5C-h~oUO8nQyhk3lD3Qc*RShO4<{)vAdP}(Qdh+D#fj&4o6BOf z|0Mvfq>4`J@7^Q{DO0vq5M*8;-u3;7-j=#Hp!PU@y|LL8>|fM`D3b=Mc)#enW=JGrkaL#*(La{q8r^r{bDcs^va*7K0!}TZ#rQz^I3JeMAR0%> zM0=yV-c(a0goqWM`z%#MJkqp&@T2M&BeZj*FStz1%YP^)tFxAU@!}2YrXK+H6|OLZ)Za0b2g5 z=adjZtzbd&oA6W~F33FXVVW!{I@r-Z}~g|4KDm{WUBE z%_yiqj@w*tzf*aOEr&>f**iBfWI`C$FL6`AXdvOb91tCJ8iZZOQE36co)q`<%JE78 zCZw$6<}37R-N;i1gw?fhG-@Y0a;vq4p(NqugO^cO(M*w>;la(uh`Xdb)s5{ z)XC=WdYf>XP2#Z$jraUCsdgr%seisaRRz5uOG&L_IfP2)9O=n?ULDZ)=LOjb}3r12VuooqcLq&;|+f$)HwJ8NdFZ)6)Xvo&$$Uw&Ha8QceXA=bcSz>P*iv zn~2eNQxKJ0Srx47%)Mx0U7s~K2FcKX-Uq>L@X@U?{HK~sC9;)5ND=`el$(sUaQol* zjX3T&ZdK;UOk}jXC%Gz-97ed{s+{y^>htaU65nj3HE6i_FN*F@t*JWvN+nO~4WbRL z-SRu7dS&wFWuwKp5<~!GipoMBhnO$z=UrtW1XG0+tEMzUgqo8?+NUP z0~?y^$F+K*+FQO#YmYDKaOgA&Iqle}>hU#8_IBc-m}jW@_oLjB5`dy#49f0g1ZpPm zM>xU;J*{nmtU5Y%`r`Mp{4sV4X7fGcN&1- zFXBaw#XGHL^~iGN+(|wqv-VCBMTHPF0OR`QHCOT-Scbnp6MXH&>Cq9DvcTj$jbf1v zB8@H%uaf9zfj$*1sl@x9@{x~;Co|IhxzQWI(&NLrdt*{WPuUlVxI!i+p}_`F;~j-Y zkJG<@ek$ht(#o`KDZ`gOYA3fcsUEJf)$BjQuIf ze2pIxgqnhZGyK^lQXxV`rJ@a7$7CLf{S_Z+1)+PV)jz4kjBE*Nh~gN3k6(`16{-C@ zjrTS5oSmQIJa5|SXo3YzaZ}oN5uzq|<6J&QscI~$X}G^+68!X#dtO~9o}B_?JA)h4 zzlApL9OAg3J{O%P8ZjGG(*Y{2AbVn*u*80WO~9@vNu`mOB|)=+0`T&78_eaej%|Sk z3reALR(FU7M7HWmNTSz?_CxM~Vhf3##H|WM4guR6Tf(?E7F>|h_&Bf9nFb9cyLU8G z^ah>-Fe%H~A?RD31nQ<@x?n6l+vn`p#=RpY$*dF!F&9@?iuF z!8*xb?Gi>giD#^Qt!Mtm#Js!eYoB9=+#$y_Ge=wI<(O&p<#A4y+ATpfam-st2 zT5xFHw?ejy8Ybt?e)%dOXkC~)>c?g5S>!8ww&me7GnF~7Ra5d$*24Dir+}X{hra2O z%i%cV-U>$Hz?42<POFGYh2VME~J)4*Ekj)*C=c3`qtfGDm}iQ>5O=Yg6~gRC~Y!Q+wmPV ztJj{1D85g}N9U6_X7I*sa0HtsxmbTw%=oY`0BujuM^d!G@upSrXL2J;Q5P>DnRur^ z#%BnhEb1t$40VK|t4{Y2I1~IoWNgM%T~}k0yQr||prGclwJ=_IUezTgOYgd8QLYx4 zbt-g5&%wW~O$Q(#i_~AwVSi*ssuO*BIv{`@E5P^rtR( zgj+ZfG`Rs16oG13Uj}aqH6egRfxW2?VLYz%@zc^tn?Y~+4ZLtP0GT+($zOVIidY)K z3mxx6;FEiky!Wb{SV)|5i0~e~{wV@*(8bBK-#(wtM)Eolvjw9b-wzS|cHe45s`xCG z%M!J+Ng&3|IV3g?zPp*LaaUB3N2q+O^ z)iT$?A2>3z@TOA9j)fU-d)Wf}v=kgR5oTCRyV2Kd7X{=?s7=S`@GnM7W14mof5PmT zR#^$8^$*mI^5*FC*b`rRRa|2xyTw?|xd63^zjr>mT5=6$&}@n(b)eV>5WixIuV0qz z;#?6R@Rsmw-89-_h1*l!HQaUA3GYpppG8oB#s13YEI&MVUp8Nk$I^rp!K8+Zkt&f= zlg)3(6!0hIU9Bcr31ufW(Dk>yHeXND+@XrDg`tOS@7o>*1et?Qf4HiJB05KqmtRgN z(0o0Hg>TL~967~Y%+WWS(FF_RS-zKfh97#!w@=$Z_NhRG<%0Q4?q$PG<5u_mgaa{4 zTxUd6u+ETLhTE&t{Ler)rJt5C*cSJ(f|(U5q}h8$zaN%ne;S_~_bm0bbom9Xw9t_B zh;pWDvt-xh^1C$WzaeXW=r$(_zvbWIk81D4C)hh~BR0==pFTuS)DbFqJ z`QLmkMyJ%TKLaZc?cRg=BSYFw5p9KbLv*^MAU~z*)6I5PyMOH)%6tB48h~W9guq2Z zAa1K~;HPG_cn+Ue(U8)h+OH72DhWJS34D?iE=XcS-;eue58CVL{-j&x9Zf=|i_Aoo zh`d+BLK!A)G|@im@XO%qwvG}ds|1x1e~a5^f4J+Pn9bkmdwia`ClGgyQ?xpu6s1~X zwG7l{DO*{98-2)s%PW>hFieA6*#%>jCT)&|Ftwjm=DLTme8D>HQUW<$REd~IlbE*! z7xbgpr%nG0r$AW0ht=PxD#2$EekS55HY)RfvkwRyQ#nF;1M_-o5#%#P!4p^A zi&91*KyAi)D&W}YE&DCs&-?UcDrN6t(UZg&^j1!FhhcX|(oizWzEmCj8Rbn(s|MK{ z?Ho)plxd6O)JqW&uZRB1!Ay7gju2H(QhDg9lqx_X>sIzG=I|KHBm`t-F&Y$`b}W6c3w9U$9u`$42$Ztu&nDQ-u}79QV^#$m zbNs4(sUsfed)2{0Wi_sUyuNcyDtzv$U)7j*3&(jrcRy5fs^d9L&gFu}eBKv(1=U_G zK8QOGBFVHAORy-2f9G{`E|R6~81%PhQEfu415!B1wYR`>Zogl3-|lsEypnZZScePk zarrg=)H>p}8L#C>2btslvKZvo*D7A?_2YUQbRGjP9(o-({*}q$Hq>?C_&RWW9XP%Y z982QWu0cZYhFYyZ{{G0v_~5x&c<+gsc>D3081~e340-A^y!hPP=n@U0!V01y;z8KZ z;V*da5;X2Of{M9FU^h-_>j;XC7>Z0?dGVTxSxZ@JvM{x11>&I~^pJ+I7KJ}bUt{_e z7XV!G>m`Gk0AneF;?`P!AfvegVOKno)la!^&;4@=+Pfuqxb z?Tz1J&g7*ijd)RJd*Dm>5R8{15-k?iDSe2&Q45}E3Kh`=Jh~yiw*)i^s)wU~5FqC( znWpqGCYUIG(3HxKmxG|YFj6%Qnnot5^89tdI`J)z5g05J`Cv^ig07V#5~&dISb#tA z3Z_T2@_kl*&;bFd$qc5KGUbU0kkhZWEFQy6F%OE<72>5Ehy)N1T1fe-@zOo-;iFgQ zVDjLVnDE+H82S8EJkaN9#3KeGwpTo`Ro|=Pfk5=P_J5i5e<;=!k!Vq;_Os!MWKa=J z!lPT_4eZkuBbMzVhqPb0v|m`~=h^-o6aRVbFMxNvi-5#ktO$y%fPni_5J6w#D1E6Z z15LSg7C6 zqsS;lC>ek+>J`sq0>|t}H8wbR)%+zZy(At(iJ_y!pf@>-7RdB2g~y~9chOb0zv@4K zoO6I4$I-$=*p+%5^FgLlBI&Z9JCaduqV318@&HOnd@NnH9&;Bh z!=%Ppc<}K*30spAu51qhJ_`USfBvcV5OAvi)XMui$(jWWb)CZAuq*NIu+bPdaRx?> zo*;}-f?1M$XOhg$6|c<{`!^Yxc0ttnfq5c z|0LOKRl`cv3*(7i#%c}7fo)}`IcD-`p1(xEVgklz0USAT8Xu2ufW@S2tRD=ckFbAP zlp#`d1^H;sFU~!(pXmP=H+u=Qg%Wx4J4>-%6`)68uY(24b|CPj0N%8`3ah3d>q{We zq+g&xfLJk!u15<%7CTtcaVwj9pNnX~&c6uOaXA!K-B9zU*%+|nmx+5O$fy(He z$Odm0mQz_W0&m`dFP(*+WdTyPB!cOqe5L2tsCFQI&SWo{*KSco_aAxJvi+D;4yBni zR=&ezst`P2yU|luymZOl7C=$~_2QK$a+9tMvjXCoU}n2B;VSzU_eUu(se%@*LI2vP z@Xiwr_~5y@82+c}_~Y$=M%2h5PIkXt4KvgYZ#+H*Bc5x*@IN)flrb;9#Ierv7x`*^1Sy54|w=luDW^g>QpW}t`*UnZRcm3(=)LzlF@;%^;=|IvoF{$N{-Q$msIN<&!@&crD9dF3ENV} zuSvO9wHC-4DeE22L4btU%DJNoG=z~aZ2dg!cn>6_x5AE*c}A&J#Uu?2cwR`Pi!Y*D zrwO9)Iyhd`__=(&b=T|P`ua2L`p?7x9s7n0xT@swkM=0G!xgN5ar(v<*1wMUhbkUQ z#pIXbAC7sEG%e`K6nUfd-O%z zPQpy2h0XiZ@!QX}>u+r@6-WK_cXy5-Yh*8S!DDy-icpP(#W3j_Y}&r3egE3A`|xG^ zSIsrWA3TpL%KGvC`D^Fr+DH2rvOKRYeo*bG3I?xj|7HAp#rxm!GqV5j-U%Fw)G__+ zlU2C$fyc0H^#)-byWsK3b8UF(FK@x7HHMi}>~?v!Q0?aAdP*El5a+%>K7*wzH_GP; zK%Z*^j-R=Jm){r+BhB|yJ%uVMoWmJJlLU;Ek1O}HNGhQ~JSqMmutqBd<>?!@sVqlO zHY-3f|L?{USM$m^k3}y8{nS`h83*t>xc<^}IZP!Gk&mudY7hz4 zKH*eSs~7Y* z_Z7u3Cb;R%rcqkypdy!0aSMyn(n@OEk-{E=!1YR|n+hoe{ZsU1R-V3;4d&jUI zbrxc=NRR!-btsQeq6d750U|5z&!nW7Op3)QKQ5lvY~yMK()9=>dP3t~DKQEGPoA?w zp(-kcYLs8E6i}KOne%WCgb&yD}IcXa}PuljG7?SJm|@j0RMb#eXU{c^q$xR+v0hqBblW85~C z3W{5N$N54paTXHFy5LIn78K7qzF(cEIPpS`c~BNBN5qWbg?oo!cO$TO8nCVDBu2kG8DvHKtpI#l2ytI` ztekWlJEj49W}L;`Y0J>nEI|pqd!v5wf-TjRC$VN5sEqc-_G!nld-{27nei=FEZ&BD zo_-J&P6fJ#ZiFXVBw#k#u=F=Bi<5C|qC8I7;SMimwoxEfA)tC78iJ3$vu3ekrC9_NHa+I2Wl$&7`o8>4;l)@Vg!3tL5&Z@^T_x&~4IqNKT&b)|iQ_kYR zY+%!ro7$ft#qR@V19NVDZGFj1V6OK8>PWB)` zN$hAydF`4-@WqS&OWL1+KHHz{?P7}u8ODAPr04Z1>1V}6ret|hqDVZ7OKkc%lZCE4 ztqI^$^1H$epw#xEh+{F<72Z$c#=`sD=$G@DG<_bv zT(uF4majuRRV@;Q1XlP!FGoT0Sv^RpYmkRLc^EBJ6U)PyTUqQ@6uZ~pCS3UH{;u#I z_kq@BEVci?@SrPko~qI&1A(%Gl%uAnF+Gp1Y&|A6&BNjq>#=azI+&E(P1Oo8EKHn4 z4Xku`CFLt#cvT`*9JR7|{Fzs^p3uBL1QL6)WJB}PS-2XTqgXS zynxnh4<1uz%FVc zM@V0vxdbd)wpQ3I1df?3a?c~*A;|`g*#d~vK7RQ-eqZPDUl_ALdXMrbyc{(Cr_ae} z+<8*?Zk}7D;H$)Mntg7itMU3Tx1mI#Duwe0WL?{)RQa*uJx(dfmoWa?P3J5-U z^aSiQuV1l#2pY3Ufnz6;?~dA@190k`dN1W8+2^mm@vh7v0o=t?@^Z%Uua7^rCC4A1 z&D6Yi*M&~6KWe|uZR!53SU$p_m$7{n^N;M^f}A&HsVPsdnORAIYi3%;!{<|G$h;?D zeEJeFsc8=Qw|KG=rlS(Rx?L}Rg7oC0KcUP1Q1{6>$mFui@*qfAC-33?tE4m643$J@ zf|3a(jv*$(g6XUrof*fR=NyxKk4jMT^XmTuHF+)`9Ge1;3rn8tBZ7?tKY0!{=9$Lu)$^aD06I*5B8eoBDfAQ@5w{7kZ|D&fy$ z;3uHu)JWnq7aN3;FVDc__f}xc$Qg)LRU(+lAk2ie#cSpGSBcdtJ%21)3!b0;tS(qj z$wns3RWg+0n816blJ73CwnXYwB$bsv9YIL4#d$3v3E9rel<#pWMH= z?ns{(V&5x`fAvEA8&A|+)A?g8D!PXFf#4%OWVt4A99(()cWwJ0_e-h|G47^n4)-wH zV33tQ@U922Wyc!v7BD0>M71Xau0M%jH|y0Vj3eJzr)S|sCY9U`d9wbm2>xH~kvm-M4z zqjrZOgc^%`dWwSv1)b8rj88Z+O$SD9dNy z%P6*c%eO1pzKQ#~CXJ!7)eWv66vt3kt z%$S1#*;S=Y2CC;GAHbL54onT!QrR78?)7p9OU-iJ#fd>i#>< z@EIQX2~Vi_MMErf}@k?*xQj08N56TS!p?(3il)Q0;0M$GP0lQK&2rngpc})=w zFWvJ#_L3bk1=uqk_+s)Vgu*(!!62d`2an(OceKs~c25TOH2_~uTaOCUhjN?J!(Nmb zr6@AHqKnm4JYpky8gs|5!}?iAuy)QtESUc_JWfEs+#uzxP2Q^)rIzw1F0z8c%;a^G zky&9;2D=Cq1(|Jn8IxUEY6MZ_l%rUHM2|2z#dBDX3R9JS%JiJ|#4Au{m7>%xR-V!Z zS;>9`A~6Is8@{M6>`=BBzmM%#LV(W*3gcV+Xz5F9&~v&}+K&BFVwIswqF6knE8=DF zI3*~FcR?iNAm?>8H^$?FVe1dIH;<&R|RPS!|ef9Fs?Vflx52 z`p@nv-q2Uv|EBV`E~S63?Lm=XbBE{W{zSE4~(zs_uI_ zzqY@h3Bu0zRD(Df)BG$Ys3}p!s&P9nOt2mMj>`3)X@kh}k}WAfNlGzQG$Stwt0<+4 z$x<=1sI2Ra{YSo+gcTVN1j|mGxg@fC{8<3xHlShp447%k%BhD)6|B49Yv*g^&k7`S zK{;t_Nr;fK;eLjaW?3Bl_B7621it&>q5#>vU-!P4G#E8ztR!!lz@4*rML{Oml^mnr zmFh#@r%umF0f>G)PCBlGzi0pPmDj1-M?PEd#?(#9HL&um!YpIaLAo063>zh&@VN^>8-Sz7 zPQWIE!%89UWboHF-);w%FI;TH!6V-bGlNM&0aqwJnxHhjDpIi1RoJoXfPCiUnM*iw z>?A5{`zY2>x?W^b6`Q?d3|-0iQ|rYQwBKdphwF};z`5G_$IsC_oQdp+rI^MdjthHY&SAs0_hZjWVQ;M!JT)7xm^Hw30x7hRZL=f z5ocxWxy)FRQY-*nd>_ACfnRFJz>-%8SaC8_)dKF*&E?E~MZ8I<8yY$c0KsJLSAFsXSZ z=Z_>=#MBc}`no2o5KQL;xMo6>|Cc#h@L6R0alPaBGZ9LrhwNSL{lxZU!a5@WJpp7U zQ&qyRu&*(}-VVAm*-T00jAA}lvy=K7K2n zzhek?%sh=!L zONJi%OV&q%Yr?`(;IYhmNt)B^nRA5xk8G#Q<{#HjdasJFwPO0UpP%!B60pCJFZ%Vr zUEBJt;<2k=|G0j#@QKPNT(=39ko6h0vtSIjW8aZZ@lQ&WQZA2w=;cnvzZOd&W~OoU z=n0%Z4*`#*0I3iIG4A1>$b&^m=gwcm%YPZ%o*3u+>Nwnes<7|5>$dyq?rWE|{jbK4 z?&k{UB5*9e+E$fVju43Kyni{FYV6v3xPAXR`t2#%zgUDvkeRA8e7)NEmmudgo!{(O zXFA%B?f5h8ciH`OZd}X$>jJP>yZ>>|6y~%;5VsyyhRP;XSg3@N$fKrvpLXzA;;9vD zq~MBUR56dq9Ig|oYBHY*2y(ufNdmk%)YSKr&od5Z;nvbM>*YCwMW7)g3nNKO8LCgz zBjEHyFhLNh5T~)2#_5GXqOY(L8NbPzP5{a24Zr-JRsxKD-T3_SIQdupFQ5?jbhWN= zonb7=0x~jW2_CaRhG3Kvr`Ir*E|da8AW4~Bwa2p%NO$Uyw6mzRZ8$c8z}pb91{9PO zxJEK=$RL`s#P6N+m<(ewT&YAsrYga6-e;UDQS1j>dA$b<%wHDP@EU1hL8T4ecPO2J zVLPbQQ|KAZp_i6JCYqF(kzjQ^(H)s+4Y~y~sM8H(>=5Ek454Hip5i)dBkpbeR zOa`>(@Kb{ZN*z5%mzxRe63+BxJAUIN|r(e7;Jh5zO-Um9Szxq#e^%B{EJ5nM48!7WbKT zQYb(>0WC!jWftP8VgT-wR1HZ+ZbK?EP;5yEj8eLp#R6m``%~Rema9flRR-Q{3ejW= zQ3CD+meuTJ@~23dQ-eD#Dg)b+5V8eemy{+s6H_j#qX6zaD4&$~tt}K6?H8uD92383XNk zjO&c!Umh@SLtO`suLH-|f#d(lz;R)nV3N&A!?v@?Cu{NKgD>H!`(MK|_q>Z2?iz^= zlaFH8^zX2A`YQMd0>{eWBj`njsO3S0MR2b~0LZRR8OW%lFDgB2LlFgV1vLxdumvq_ zAQGmOu99EodrO=m0n@rV#ke_9fg+pWtzQ7bAmwu-G58}E0uc*A%|uAk5sql^>YB)4 zla(4Y!YJ2D;nM>6>q8%6&twH0w@$r?&2tapfjb_C8M0wjWH5Ee3T$lv_Duy2HUNt! ztrnJXnNcD@wMQhQHSzx~qi1Z$M$BK0XYP9wuRcBm{(vE1c1g4ZLCS8^>)7xMh}P99 z7iP6b*M)T&juzyjV?ObY6_&7OAfTm$QCnpDP-c;J+eHBMV!J|scgh6&^&mo#FxUpc zi1MJ-v@jxC5L$%HWHMYS1+EUZTNj?+qW3dD6NeTy5!Q71o=_wtugcFhB1~Qah3q?D zqzl4MF)E^65clQq$xB~gd*cPP&N_z`Q+MIjN8UlwQ-!>@8~*W^CaiDz9$T8uVB@s! z@YI7Z%XbtzMYyv5C*!#c1dPdIC4erUH-qr#aglnaq&Jzpex2UC%1fJLAfP2t9*-)J z++v}EAH|jj|C#Mi0GsX4Vg|~N)9ae!mW+9C!S|ce>!FB_fE*D7BVmLhWQtR+Js|Sd zEPkl4G!#b^Kiwn23k72#4|rL<||f z^jh|qUU*qB5DTM%veGdP-k^o3w-SH4e=uHo>^&L36_mq{l_NlgG#SiR5sIB+bWN0^ z*d|aqL~HSHe*$3K0zuZ?lcgoip*Y@-rRtTv}C(?1_> zg>6SdkL(NjG)h7-Ne{>jOeR##BzEpQEPyE`=lDAUYf*~_lQv9J^1;V__~Cb`^Zye7 zDc6$gbUi$e@|#o-Q7U1iKBuavD5*XN_SC$ zj`z{J_mHp!$#`IbDduG5(2SJG81ml9I3xgMr^v|5V#e%+ICDWINDm$R0dKxL0%ocb z)!qA`apqh}oHC($=Ge_gH>RQN3bK6}3e(K)T^;33_4C!#3 z{p7Ly>Q88=O<{~ai8k^s_oSEx!34|%Y0~d z`>*3~?zmwB%8(-uD?zWgEGA5vh6|Sz8)@o{x$=4SfP4l0FMY4tV7UC{{&em!N`8ggY$LxsE=^|qUxJVZIFI??!`YN zZl_?Sa#;A~QW-lJFSQAbo8QNW9lszNxP zkU2m9i)FI@aV)G{w?&v#lqTkNk(n$qr7WNkkG-U@otPvh_?L+F77ty@N%P?El$R|* zTQZ3iaLmM=Fmtj=Fu}|J6JTriCZ?QntTz&RA32!tWt)+SOx7fUQnGz1mFUkDBq+(m zGrPeuDci?nb$9u`gx(i+jI!F@BzY$6CXvh|;G$$}2_z9n4Uv&y)FP$#$Mg3O#cNND zhM{*C&^T!1U^)b#s}W0)Ra%L#Qw@WOY^^7%qJxn2-GMLuu^T&PoX5sF-yjj~g%qW; z!~Kz~X9Lj?)XTKlCy7PXqQ%0`@dq!sc1u z=u~a{fVRcyA@PH=V|zCSc!WV8P@qpme1_YYAhD>#dBFSnq;&aCexdN@{Zr zP;*5o1gQOwOk|E9dOWi4$=3Vl@kf7TjsbW4aWCNb<9Olu2^^ELb*1NLa#}!Zxd#U! z@t>@eF-PBIbw0{bcNZ3VBGwB$M0%4+ekMF^%4$~27$$hYc!u7Od>=t~%29JJsPRw9 z^*&N8qI@*ToFWLFRvwqBOyT}}$othC>d|p-%7_xYuxbVLXaD_{1l!NQr*r({?w{P( z-Tn2K*Ke+C*S7vqDMA(UiL`!-48E*NireYS;-5?*{>ecz(uMekb{k|NE3l5B#KFVg z3ov}**(qHlX%dg|fKfR1M-X4ZY)0%Pk+Y{~a$IiV+I>2HV6mE}$+%_o0p*HtF z_h+@1a~xhv{K`3D5)ialltOz2#-uC+QjB!nOw0cD_Io4n-mni4=YB>OBb8^k=aGea z4e_s;s4kqp>MGBF*}iwh@dNW1s%T5acUQjuUhDoRZAdj8vHYO5Nu^l5Fvd2LISd$d z7k)T_VJu%)>wlvSGOsr&I#g%_^`;`n1Ui&}&1%{r84l}vO^*v2E?D4sXMcq&nj zhl53Uu&fOC`@Og&=tEx5B5Dp0*=7YA_ktDckIHBz`b2dM3~CtU(^0P_kdLSFkUxO8 zO1t3hm<=u212uXY_m>B7o7X^fxDp97jkF!Zqh(%ZIpebWRHGVmUmE^Tjy-+N>s zW1u5ncU`Lb7Xtvdp%`#p2M+)H1BY&f6kmVK!g0O-Rbue}FXR8W^!YEnhXH_8s3tR@ z7cUvH^f?V_26SHrt0r&7uI3~7s&N_olsG1*EsJzx_^ksRJVK%Y?*ih z8=LoH%-~PplqFGFR*x^nv|#JBHf;UuJoZgFk1r>#M39WtI9b6y_zBvD@(BBL2n90G z!&QiQv#lpDWCSLa5Qr^`?pD@QOIMLe8ft{t|7xuq@kkKSavNj*){H&#E@4N* zX?*n7BWcd~Lzb5@(YL=mlplhNWMOF#C@hC$61mgY#437aTltv_wLD-*y9?Bu= zuSUpIjgX#3DgCXj0RAi5-(P5dk6DCBq!hkbDZ0i>P-gk%9>SUftvn6K-yKG2H>5qi zk@56^Q=UiEV<8Za;bz-|E>0Or?Fv-HH2A{~bgc@yw;EP?ccjY(;L8toVRr+tar#jh z6_s#2)o@B`gpC~XR6>hd@aq-uk+~iBA`poq=u1HhRU+t5Ll0FV>d7J*>L%le{azZW zfZqu|4!=Ofc4Vk)0uD?{)#3nQIJ z*mPj$s-c;RB|`=S83<;gN@RD#F|O9?9ii^6!IBpN}0ELwK%$H_AnB{@pjV%`J6B(ivT|0~z57trm(h4c96 zAD=509X;0wtfrXs$|7PoNakt~HBvCi3`jEhPe3~lGg%2O?!Zc^2Uk7OO?-s;H!G3H zkz+rIC*QZn&j^ssYjoolw!;Vi_^gn)y@a_7zLL+9=^=oY&BS>cF*~L1F`)ooemC!r z?H4gr}jxGD&SIxw}X_Ei&Nzzq+ryBz!4r z&T4T@3J5p`av1#1aA{w~kXc&*0hH{YN^*0Yt2oM?cXpQTujKJ4Cro!-Vgz6c&3Zo@gI)99MCIi4NH!d1*KHuSNM?D+OJ~G3R+2?+$$* zXU|>0nX~6{>hxKW8YNg9iCc(S2^=``4LFy-Irc61@4%sNg>fv**+Sx(@dbMf(5cNczv7)Y)=_CsTyJPbNq1~;e-#!!MZ2 zA(ZAg?jf?zTmuO7Wg~r&(fc4}_JO7M77yHX{C4rOC9^3&D*&#yF&0|XAS^zUHHaA9 z1Z>R*Z$WkNPE`5s#gZ}mv8%BSTc#gJRd673{@YO%ydUY{Ksb>;h{t=%`2%DvU&;AX z>iia^Ijw4FnLNr92J~E|Fq^Y>e`Lb9ARoD1?zcK}mjJg_+FeL%0}(TNAe^lhhBePC ziT*?eW}-WidOzgDx1%a>7b>+okkN02qxD59)?cxz!u{aby+Q9s@!B-%1w1C;`}zZ; zr6605mI&H_}0?m>;`0WA4+A6lBvVavxSacB~-=<{s|I(cY0N^PcK zrfOiu>ya^ULo#%$>cdEXWVQat#co3)G5|)Tw~|QDR3c=hLC-@1u6`#e{gR0HK~5Wp z%E0Z&YlDz8Zj~{Y*83sLdx-XhBQl&hL^CyV{%EH1s?Hy95(sAKZ=HcRZ6lVhhN*W$ zCek0(+U>}OZV~o*PQL}IU|;05fpSfp4{@Uo27Q_-!g8RcOw@ZgiPc% zBm#X=sSiTZ*B{lfyWr@3g(Vmz(_Q5G74XHMIUYLJgkR4q5cG_?29D1}@&k3p|s zhbjh)6${?E+{+zxe`H)T4rF`q)!o0$G~*S;PFC@6PWIK`9{;L1Hfj)vzMAtZrY%8N zUbmA|>H>_Pd2ep~%=m#nQ~H_h;l>bdyTl0$%Kl6F`YYdmWp6Eve;LDOM&b|6NJ#v8 z@SEemvj4ekJq5gWddfS|;O#to6BzmdO1mp!>&w?LU8B6+yV4KYbCH zy=a*lQ&bOP3}A5#5Gd+{NU{&Sh612dMhf*>1@4Ia9(QQ}7jB9CKd3hG9i5Shm7G=X#!GF^d9hWt8;mIoW3sZfdR(yBqA$?2Gk2lJS zF*&1QTBRQ!-u&;F*rf<_BO1m={1_P0QDG5e?k|cW(Rd%&v3}?qs>H*kK@5$>F+3W= z{Xq}9hddbNy8+w%{|h5)s1Q+w`#c(^+)#v1LUnjhyA}1(Ji14`__U}9-`pHPW2!qG z-dCm?kxUMLDU{;a=?zowk2qzc+1?oz@7W01d1Q?`4DNCxb_MiTw*-$wb)>?ksGZd4IXqY%#Y;DL#d}?Thrz%9Hw?b<2E13&1#fow-}r0M4S1#G zCcIu!hQHoWj3Hf%F!+Xl!&^l+h97hEi#!(T*@>CxxFok*ld_JeL4G(^=JTDgRn{fj3 z4}1>cE1!wvK9ej=QwQ&<+#t2Hu~*t zBNz*yLJy!krlZ&n;^ss- zN(>)-(Fls8UU;$^ilZKp(T#hOxUcRhj32xh^FQ8+l@pF(#b<}G;KPj=@uyEl!ZzWasu$heK4areEQZ*%p0`=lU|#FLCMGQ_OlZ)ZS)E(X==gzaa%BX z=yKfN>k0S+8X|fa8a<1X9^7o)gmN-iLk28wE$+%Zi-x~1#TTQuVeR;XSUq+hW)540 zp)Y)fpyq&Jaj{hakLee%xgzd^t~q$|-nXz| zdoXkOG7Np;6NJkw#G?+v(J(?*1^n?6Sl&Fo{_F^LFFc3Ejq4H8ZFG&7qaslbzfq2m z9>@K6KaVxDcA{nOeoXkFQTo5DUU60ZA0*HkEkig-f9Z1Qp(Mr*o{fbcuECVo=U|}o zB>w))WHgRmgQZQouyFi#OdY-ggL*y+uP*?v=KU{if3l@RaY}=iz?=7>C{hlB#*v~* z+?#(H6aMlgmWRF@LLrK@jw@W(6v}Hss+u%zDC1GUtz@1F`y5&Fo(4Y#6uap z`^;F({$MSdN36oU57%PB=yjO(?w6SP!Ey}y+Zcr7QTWJ~k9$!eb2^L?vZd2Ol<8gs zgJIad4DPFa5}&`d2=hPLg{9+-xe@T%JXj>7m48UY0y z$F!>kj+uBf)3qR&K}IUU;i#Fym`^8Q)7G6hbnFDa{oyRWIer>@4}XVM8@A!OSO2B} zl7e(I*-K2C&7SuKHg0Lb#?9O0ICbW{fKTTxv|;P^o!GRs6&p71!1~Rt*sx_MW-VMU zGKpczq7_)n1k4Bk-}mT~Sh!>*TK65o&i#jR;OKW)zoiA!XD>iqk3Nz-jf$*YH$*K0 z-)Y2>8NBxPyI8kvJHFnu1+TyT9^M`{3X7Mm!I5vzVE^IcShu+a<+je5zrWU#W*^5>xzfH2`nAD|I z%EFrAUHwM3hP1{-GVB8^{$dpl9XW}EM^0elmR5|P+#p~cfdWd@@qwP_CrEeX*l_`l=|#(a zX$Ov(pe_K%WISKOMa*6J1>(Xkpj2Z{!00cRt-|K*t(gDCQuOR|3&wo>Io7V<2Ff(< zJ#Y+PuicC%o_S7yM1DpkzA4+7gJ!W9AT0pdhyNIlja&EN&^M>>&39+9y>&m{d2a+h z8#h6Ko#w}h>t#zeq#cdg$P52k5WYq7u&FY!zOUv z?%H<%iA)}kKJkoV{ak9prp=pi=FAx!I`TdAg3Kr9AerwY|1lA(Hnd>fmR7l5&XuYd(;w8NM++<7~xdh9a_hMSBVca~%Er!819em}nc{19dj-H4B0nTu4Y zKlHfrptH%su`&oIGvc3?HwU58dpAZr+Kj~??Z)zP2eEp>QEZ(2J(i5#joHK3lT$E%#722v^ZOex_v02cy}tp?A8x|XH>bhM^i{sl?02ra zRvi*{UnHWpV*I;{uwc|Ci9HvM+=-U)7i9l#`TR7#9JL)^jNFMu@9)L@4_eSXW(`IS zZIbgd5q>4-=X|Fxay(uQGfvifC89a5J(Wlqy^-|v#S^^-W76N3V&SKZ`;K7wxC5B~ z!FG&%u^Aa-5W+@{^nEy91v5jIS`8BMei--nMVL2oE2h4-5_hFv!myX8V8*Abv1IyA zEckp2rjA&KyZSs2JyHjgzJf*%#K@2}d*h7<$Kc?!Htd^n0ZZQBi5;`gV8N#?s1H1f zr@D>6*5w}ohg-31+FmT0upN!>ufpJmCg8QZ#$x)Y^=KNq8ZZB87-9ycI`fERDJk6( z$;cp72mXl1>xN>|-&SJ&n5|ejX)jh!-H+z?SL6Mc8;~;lOHx`(*CI&vGz+^jbx2tK z@xr|$W&X{7ZwCfHFbRKqY&<3oS%!tTh z+JyH^s+!Pi7kw9+0FI^$vGL;x6HO)gPo|d8;1?W=l zKaD>=`7D;KT8AyId+`0~^VoCn=uhr{^dmMi1U~bkWW)HCOw$klF%Fx#|9$&Id;GfZ z$oKeqZR^sl59>e@Emt*VBefaj|IqW@j z4C}Y-z?7z0vd^;cN)>or{r=1SkN((-Ij8nrBgF!&JRW-d8BnQ*#Q{IJ|FQjK?G@%Y z+1m<`Q8`%x_*D2wYTrv+b{|yU*o-$sv7{r>-3}ZVn7fQAIOZOC@>vzT@LF~q zka0rg7AhVw2GI$gi<~ii*1N$CO*D( zh>~INq>zrs(I?=?vwjcW>Usm-==wYS_2&NBb z8KxH9gl~&W@mV?pe?Eues%iu?%F8yK=!XD(lamBcdmuo7E|ozvoj}0MBdvGCsIn4# z_Wo1~Sm0*?W!;GYk>FFqDrZvpwk&0kON=IWljv1L4W+g&c zVui3Q8o+$Z!pG?x2E=VCR!hcg+)?Jo=u850GddQ;G_0}{SgDm^b@T>&X_lhdiJ;Na zF)LwVt{KImL<|i{6CYJo;Zdswc~2Jo0v7HG2avbo@H!6q2ZDI}#v8FXreT4l;i9DN~A_b3ThwBWVOBrLN_{wJu77$Q*hSKh|1K~j+T!G=jD5jDF<>6VJS~%36 z&!QL>-1FJP^+WCi92eNr5f?n>J*8-8qwX(~sssNf0P`>4R_dAdIwRSQv#!5(?34eh zwrIyguLH-w+_>!xbsadq4jf+xj`2SoI93&-^hwK#^e+8{dC-S9>dWBkiMz44@f?;< zSr085f;Sq5N4HRFQYzSs@>meTNDQ8+hH_JbC!{0puf#h~ev0)Ik6}miC2VbO!;Tq1 zYa`Gy^$hmUJB!&LFG0T^14UBV6V*{3iNG6)Bks#0U(yRxUYd{HlTKpa>j@%%mSVpkK;(sU6YJU0Wa^S?*S ztRr|L|0*UtGZ(Eh&SGuD5191wDjb}C35zGJ63<|N!V7QA53Sh7KW{tb^!-xT}0F9#qenwN}^?`uqqI=yokhn&|(DKe6Y%@uwneS*fsGC_Dw#6 zxqsh|P1D=3ZpKM$nRgPK8c(3L`2sdf--{O>d=Z*225(45Y0L*-G6;`90>|4OZ{GbG zwv9WFJq^H)hI804<$`#|?VJv2&*q7nkm=SB74eYx7W<6~gkxcto-~I2aV!o@xroCP zfu@(%Vr$cB?3{Z6TboXzW$J0{pMDXoO~)~P_*aN}@+c1*D5u1=QHrEijkOaFV(aW9 zm^1Y&R77=jNd(X}S&pto5h|Q8Jcfa1o_-#GedZ-Ra^J(!|K)o43j2Sl^nW-KL_jM+ zI93i#3nJmE#rn^WVb{d7*f;4MX1}!^TbhB5GtXhe%;VVDZ~`sU&tXkND_(izW$59M zw0}76`A=^Djne*<%r1%gP?oH~&DPDRhy-Eys`2i_pJUtSXRxmk*fIGO_EVa1D$qI} zXqj>e%cmW{-FH5Xa6AHUxEQf;5>Caf_~O(3*gpLdw$od>;W!RXy@>A`fSnW0V%Mzm z*g5Si_BLEV>x`4wwBQh?PMC#&5roh3$T;Tso%Gs5?3#88J13pQ{%Pm2bJ8gsYzB5U zpT*2imLM3?P@ny!@FG%K%PZ06zF+f&e9CijcWWMgzeueuls#Jx~dcQu6lVC3W!k{9Y!L z_+49e9+IRfeO8%-B0GYKug|B>zz-MHd(K``?{9y-2ps+HIG%s`&x$$dbc2~<;*fry zNqjhVtoZVs1F&e>3OUcYOX@y&ULL2;U&P9F>jc>4LmU|=1YY+X_*VRZd7TQ{Rj<4U zxsNvBJnyA&{hN0lLAby=B9oDf<73B8%KaZW{GDP(Br6dnphOvOCyV!nj}*2hO#qiJ zUyE4b`pKjwyJGYwN=lm7`PI^uxI|g!OUg4)+7ZC%^B3{;y7j_D3dIRP)yVami42Y% zS5lA)I3|Oq;Fn5|&O}uw#_t#{sqQ%Eet_#jSM$FxdZE?lxo^>LeGEAd|VIZ9NoUb7CD+Ae{kV_Nf^ ztLy)7PpJN8H0U8kTa22jBhhdrfdcujh%>Nr_RaPb;qnb{+Wzr5`EC! z56C(By|HAqc#0cLZrCX-Sh})(4)F8tXHTBHggFZq%l(|WZ~>E>W{IbxFvXLV82rw# z_HjS0ak`qv03Lem3H3Sl=eY}5zI+8PU24POZ+?&p<2aKDM_z80Uxrt-8 z1?T3C<^a#Tdc#&>>T{hWLo682O0h!R=^_4tnqCiG?}2RSPOO`F0y`(E z{jTZFO;|tcI9AU(hK=)nz`Dlov2Dg#tZDofuRJ^kSz{0!tG9S;>e(7ZW3|GN9R5rL z){Z}h*5-5A)c8I2&TB(!(>d&Ex`gdhPT`AjEx4urF(l)?5TmqXq8^E4Po%T~sPjIE zxr29L?_{8LDzI(F1#E9VkL^uov8(X{_BLHW>+Iur>*)`WFnfu@g_fkpVh<#J1JL;A z71-YZ?4_N_bQWy^Et4-|?@XZilMT?K^$1hiHd~9BSuLJ?Idc#)q1*ArCp*yEcuMx^ zmMQ14e==}v5^!K5B$jNQd;xnI2REL>);Y&9b?nl@`3byT`S}f_78$b-BGJ6Ct_ULM zwc9Z2*+#4z_YHQ$Z;N8&3OBY13fN6PGn zH|`&eJyXwOcf%Qc^88$EpLq&fXP&`h)kE>wv&+#s_Y#&*Ie_NZ*5KfTHmn$T5RP#> zB9t37YGK8CAsf8|Bc5-@rfFxeaq>ykZ`0ebV_F+pnpFJz)zp1>{Qkco7VQDk?u(e! zQ^sG)Kd62D?VSS5dS|+eO3%>86fglsbEx+4)9h+&V;5Rd8IK>mBznYTMzw9&QGNaRkWm(dKf<(KgTiB zF}|{`?EnBE07*naRDMpKS3k2Y_8mTk+wQzi;&!$FG6p7V`pWiS773DZ&i#+i5$prT zF9eQ%Y5xlq_CHbE5W6oH)iEwo{=Fh~D+(q##8WIJuyMAnP44H|iF43RvfdRC)OoJ` z7&um}W$wo;*koU_Js8*NW>yqqB=$?yAr?y_Yo!op;Uk}AoD`~c4;mk_ap7GHm);HH zyP+mFfBXn~SDLWWN%(DB6c;+5|4M%*cuN^+-Z#C2spKcclPqXpTt_t|%2=~FG|^2- z=O$?Nk%sLgp=a{Yv(>2A4Lng=hS|QZ*r9d7(4_VMvG*QMc9qw*@B1HKygv6Fj134$ zop;WAueEmGyK_(`M-z<$w!tfKj$kEq%u(dxCowZ1sl9CM6;Hz88d!oG|V(({Q0%h1B6I))LBEC1hQC`!~w z5RQt9OH6t3hC|{}TuhOFEb+h?WqaXI6c9*+C<$cpobxAq=^x3F+#ho?^EQs=jOJwC zNX}UA;AHL{oVI7N-*ziK?*E|QIf5RSl@_mq*E2`5ObfH!7v(Q`Asz{&nCrFDL0i~Tuwu*026O)9|rl$ZHQ7Yma< z79}<2BqGd8C77R%QyC4i+UI7c-NCM4gqCochkXGYi3on(V4^R`2NlIED$rOK^s&S3 zr`JA$e*6E)w!lc1Ckz@44=ckiT7yoSjUe+Zdm%n_^#yVZ5S*8R8=1vrz!TF)n>mlF(6P`<^1c#Icv^VptgW1BO=LcNT;y-^&h zpiAsPL`7vRDW6Dd7c1{;v4RMOCgRsC%)M8@Q>7jw1qouwV*|Sh2@Z#TFtrFRE$6sE?T+6Cjs@aPfjJQjC5n6qRco zaT7Qm>q+pZ(LPp=a?qcVN2}k(La4Z12+=F6N3*%V;-crSw4h*S~2v{t623Uo6Ktoqt~UMU*>wmnDs@7w*UHoQWeD}T>#zSuc$KT$0!yR=KIKBxS-vo~T z%LB(MDIZPYj~iGt8@aj_@oIJYbb{6*wl6xwftssqUivlOfE9bdj@1`XfTegD+kzH6 z0T))kgFIpW+T;AJ;zjntyzVeBJwBDLI%sdW$g5AzV0+C$dYcDX{QPRRO*=$e-34kt-9_{JdpJ;Y zjrEngaQM7r__N3fIf*nB%?tlx!Qo`1`N9e6QPhZtZ+8a)@q2UPgKUvM9sVmvpaGbWK19Z2XVa=k= z_#FlgJxFHI^vHENqdZvfJbP-+t3>7QrsJ%v+RXfq>#3ajC0nWv(O!R*-7S}?p0y39 zJB%fiqZqGtzf%FnpZ~Us1JxHfR)+!@w>I}Hc52l}Ygjd7FI_bkX|Fs__lk?W`SKj{ zv?!So2crX-N|O7V1*h2CaDugUTk!^CWa>UfY2rtmO;*Ip9pdR5cHs@!@CTe~{4L*k z{4c2SciRlynjKp(2dB?TFuQ<_pYNxC=>P}o&e2wXinTR|nD_B&7Jjl)6%lmRU!kY< z9P5^RMZg&_rJ6&z|C9UwlInk#D~=`Pz#7XVE10FcydN!jp0?`K<~&_?iY<-bux#!c z)-Bx0zSfH-m|S<6^^1E*CCacybIA+Zuse0;&1hs@Q#%{#y4YOP&)&KVbS%9{*O%v5 zS+j-JHD9r{sg31Jx3Ii*8?)vu#pd^tE(a^*8%6@h(Izmr_ zB+dtEUAP&$*Nr1$Co`CVRkJET?L40yhaSL@7v{lD_&&bhMmeV?#T&pMAl~ zxm(y>bCCVbgY2$8LDOen;kAW_`akD8_WxOP{=0lbz_G`C0sGEU)grZM`m_mCFPbjn zqfcl36&$?=C(mABS9>ShckSiu;8iXPcnX}kc%4ZPJg$IA#TE{y*|DdK6K5`P`rIJL zPoDj&U#ftOC(m={{1uL!9^k~e%N#y6$cF8E5zx`2i9EIV%N6prcNcw6mXkj!IDZY zU%#%NRT8+Du0UJ+J~nUJLwC=S;h#%FbH-e`|1s$CiXoGSGm%6nEZ zIYegcufVam#tCpN$x?Zt74KTh;;vb(KY{r|uvGqK#W?|@1K z%lJw1R>n>e#IL+HS&g0auOz#VoETv9wl+F@4sr4FP*QrRxS+0Or5b;?CdshxKK}6n z0jx}4LSf5)*ZH?``(8EwC4njJOVaM%ho3||xn+(^AB$Z1s10~7A8QNN1uL{bC(!iXCimp z-o1}KoqZ-4Kh&?nhEj~0Xb~PsfGe5mLSB8x?0>m`SwC|9maq5F*4eKT;AaOearv4# zzn0Xs;1$qYs`HH$?@s>21dFf1CsRM=%=rtd4;osQvA4TV{hKh6ckkZA)vMQ#IWCEJ zPe_u+g=mQ~sv5tf|L9pxUbxDMa~C;zZg9BI_I4joss(41FvK`+aPidyKv%DA2U4`68Q|`dRw(8;h;V9J~Zf5PQ zZuTxc&!J}MY`j48%{LZ}9o3g;Yq~)7j4dRLdoYA~W0Vp$#*uQ~&sVb#)4TW@hpIuwZ$m>b z3#Wd`;!jqq@oTRUFn56U3;PIrCJ~h4fp7&8|9JlL@`WhB|^d&p$ zkJDX$k)EbYtY6Sa+<(7n-y?nZJ=!mgKZH-t9!Y4&%85EB@%sI}+SXa|U z^=Ip8p7k|*8!jm8k)Eon^tD{&lQ(M#rOL5K;{@VGME&D=cHCR+tvpL#^%eFsoMmNY zJ2NJ?QaODi+Z&G4(L6xMigQ$dwoZA7%RZ!7QlS#k-UpfWyOkWMhQrm++j5Pqtw(5> zvyFuxtl^8#I#mH`XZ0nzmtW@Hmlvo+v`5K~n@QT?r1rm*v@OA-3+O5PmoV4U{Q0f7 z7#sr0WDh@d>>Qi7wbRja1ljw}UJ%Gw_ONS8u|w^D+V|Z5#O7hhn_bRNiGPlrI`{4I z&rtkk#y_1ZIj-b6W$!bR96V;q6Q3KruJ$~iE)wb@PcKr*rn1HQOuziQ~SK?ARf2CN?BeA8NW#SbrvaNc8KsdqT>V|JQ|H{?*Ct#0%7&tan zBgFI9^amAHj(nw_;txkSa_W@2e zoH{4hi~UOp9tETdI95o3*jX6)N=7%JB`DJatoYLeE`40W;N&o;KP_k5XTRaGdnOTy ziOok;on1c!a8R+M@|Z3mIK-0nt0F_wUPhS4hIk-H%JD=*9U-4UEKMRTz_w0WB&oG9 z6YUP38$FU1TL$g!Y(CWu^0flRoGb`OC(!Y=zLR>pK*cDY%YbsGm9KVgG=6dzeeQXk4= zb=b-ueQuKeC==X1{*Zk;UwCe#&-o)(xo_jmoDsZ`Ig%GMM)T?&nY=l|%5&K+?zQ_! z`6YfHOW24~60mWv$H%i4jk&k|l>M$dSrG~FbFV>(JIWYWm{0B;!71CFtoJ#1!582u zpNF5htvu%Q^0-guL0^RX{4pN!hk3^7=AFzOmX6G3jmw}>FXT~=h9{XMlqw?Wk5OhM zxm(kC(`I4mt^b|9?o4)iM^lrG@VrmwgWMddNBldptvUQQ&(7naF!x6j+~YHN+7@7` zQ)8D~0-Q>* zgSkd;fV`U6Er|t?E($jyqc3(*;wdkzTX_!^1tFOS=Fe{Y{KoI8pUXR|$Vi8a(8c=1 z5ZU>Bd=Xi{!q_+WG_e8D^a8~a*EADAmg~i{*&QpQFqq)UJP(cTAbmbRt*IElDoha3 z!Z=0ZI#Nnh-mM}robd>*xU6|onMc#c4FT}dmRaZ_$|Dj>hysrCa8?Qdf+-c5wpK}S_f#acINXb1J1mk|iT(yO)*n%EB?ihM*IXe~{<6zBo zHdSxK9m-Mwp8!>MVaSHEa0D}PhjPhNKCgZPj#8>7Z>3KXZMA1uHg^@NxJJO6gYLBR z`+NVW*si-557OQQ&;0ClEV`2{k^0q~_#IJNr*Edc`Z#^f7pR)H5+hoG%Wq(}gn99a zDeS5`#s21NwAG$u>RSsG+u0Ea;iOU}%FJdFZ$iONmioS-=HcUeOd~?Kg!R>{((I; z7wBoaNcW24{Pvj_$n<5C7qu$ps(8{?PuZ-P$-5d(F=yIR1*~QJN8!=D+*kTEUoSaM zXEp4pzQU6azCgB?MV67NWQYCM7@s^lpYEEI^f#Pk*UEl=_q&$}=uw<@ov3vzi(g-- znBcn`FH&2%m5iXDQBm<|&Lw0mXV3hL>}$Tl=DI!D^K^3jLGm;Qu9$_;dv-;?7I!phu11wvxo^WIw?m!%` zH_St2zhh_R89M6+Xm38u@1FUS>VHcl@Bf$n|M~qd4talPD(0{5EMUpoYuLB!D!Y~* zrFHR_BujMc!3mPif`-GRfY;kT8t zY+eVumtSPV>JBWr4Oi5P)tiMo8X!jtk?jj$4~9(H^l&yg-fUbxA9lAJpI;+jO_9i* zq!Q@68_u(I&X$%`0|wDVq4M0_)45M2q$B}1ZT1rN;vxw&S1^Ggj7jm`6JNP#n$Dj6 z!$7U+kz9Z)7*n#;k~H$@;zwM7c?h110Hg?X4z08%I%qpR1vf4LW7vWT>wGY~apwZ%(GJdR?fOsOQJ^l*xE=PoOD zvdBt{1f%k7ikF$vQbwA|Q$97()9cq+y=JRo%K61FN!S{ClzB^*sDzs&Rj*u!kEYHd zEZ}}HMLbzfebX1}{x4p=#=awG72{5l)xv%Z#S$DpdsaRBvD4>FU{Sr8iVV5P0v8Q4 zj<0Zq)-TtZ1qab$$rB~7Q*oq0X?ETyPy8WZ&U$++`oLD{llxOqF$uP z%xdz#2aeZo+OC*9{~S1$gtdU~#j1Eh62R-e+NI{c^wX}ry;MxN2l3>V_4LOVUsL@r z3@Di!fBf@Xf7kyall*mi7iR{qnEl_@O@4_aeNy;Cao+!Mnqq^Q{ePXRhUIt!IF5^C zZGsOznW3Ib3MFQIK8xM$d(C;U?*L~mn4ZUT=PjVCt4C4s4<9>+S1TYCFGt4E8B8kK zQ28!lnn?m#*3*st+SA!@78oSTaD~K|R#;mCm=~y|_utMxdH$~61B{(;ubO{BJ;^h_ zdLHrazVUr;PoAon#?ptFFA2~VEFpqzxd7% zT7UgVPUF@^uG926Ha#ZOXt6Xkjmw7jzIltQ&3Pb+qgWY1@jDBbsKk^o4)mBLh|5X4 z?`7M(LmX*<19jJw%}06BLnLD36*KnNlm0?`^%=Tr2H3aa3a>mf9W7X)KsWJfPKNHK zdh%xaTdvVtf0-4t_D~#ogtT@a1-=J);jWL_yZAi4b(d5T;nNSiuDlBa@d}K{I3m`G zto*p0jwKi9Yq-q9DJvAa%M(tj`xdX>=FfJqcg028z8v747v~Wc_K7ei6OvRcMGKap z8)Gmc6Nm@yqtyKb?KP*^-#SR+ye&BNLM10C$vt~Ciai~{mLJ2GPT`8A2}ddjYva(p z<7t|^lN~K5*uM6F@}&0ZMd-2dxFu1TD#EAbtM+C7eV6t4DzubiH zF6uwrN?+3zx@rbh|CPlbQ~g>Pc$nuWO`&7yAf0s=*}MD#Pe1UQ^12nU!U&J0!1EAW z=N#ckJshlq*6F(_N-O?5Xe@w!cm7i)^NXB;c1 z?x4Tn5}nl-X!`Uk62WmK!V}P375wi057@*Y+FD_FsJbeS+fq9Z#w8 zO?&U+i;upcuWFD3bysMfwuPej-57xqV*YWw`SeV7&p*k@R_LuAWXfwbSR=)_3rn%< zaSCJiu=wre^fa8NyZ$tbr>sz8E6o02a6FGx{F$#79%E1KIog+<;}?&-s?M!YQe>`+ zm8bHIXPel!^eX$Rud%1~1TX!53SnU*`^Qn}c#xXcH_^W2G=0kkSy}y!I)82e$08*w z$wn!Nk-y&nk^gxDQi^9}KM=JBQ>J=XSC@)kRKZ(Y_t5!Sg4-M8{SQA=`@ZadCgZsI zhwXna{YBlc?A;Q-i6X*{_)R?NW&f3Z_@Bi;9$}eEY!Z>8F_G|0(|7Qg+F#F{zifKk z3z#8n&PWokfJVQ2fivd@OiKjd#n&gRvs26JH;aZ%_0Rf!+o9+te8@%ygl@0*jSsk2YcKUEABl>)eYLebQ>o`0fH zBIlnfS{(+Cg>77p2^~(mwSumcIF0 zQek0iiqyMVbn(yE_P6Jf&FhBlSK3t|x~!*s`~rN*Ga6+C#Y!Pujw>pjrv)kw70`I1 z!^?dxKMy+H{4zI-zhvCb7mjS&ToxAjLOkv;c+e5#A&ZTB?KbZ6xEbTsnPd+$Guy}E z3@c}I?%;^^zj54pE2k`@Id8p_W0s$=%R7o{!^W=zL1F;`^CsdqCJ_x3Q|yf}E?z`I zB%l0P5o5w>Ua{q{+C7?85g*edaUK_tHBvx4znEw|%98<&S8pB38ckjk5cp;@hoj=&Dd^qYBmgf8i)s7549W{cc+y9L| z+pTQ(XYs+E|HdrWC}#Rb^Rz9Kv3@^~ShM)c=pV6ByNzzw|IShSEga4sMPJS+dYzf{ zTkfFGI)aTMAD<@*ndFV*59U)C3h;>ScK$HxcBb4qg1UeI5qoVTSmk%{7psHE^4vU> z=V98d|Hd)QBgaZ>C`nDM zoQK_pvIlWSiV4TYVu_BylOD^{4hxI!_`g{1w^1KW@Y`4g5nmC%%d*ol@^%(#8V?7e zBn5EPQWW|_JZ{aQ(P5*<>Ec}@f;W+3tjEVY8Ck3j1=Z*Gz8IRwjz`N7wz)_#M|JV{ zE+wHC^K#}$b~$ckm6pLrC1FZ^K2?YpjE*A`s=yZ>gC$vkEmepoDXI{KM2rd|s_?N; ztrzjiz99vz6bOV(Kjhgf`=x-mMp&5S`NYF%5=NScfS>aIA$B`MpP=#`H-9caQy%d0 zJ3~KLU~p6vwu&(H0wO8~%r}|TV!I&9UeQ9OC=iYm6ETX2=tV@s#mN8ba@H!~SlW$K z64H1tYS3r|m|B?R!7@>k$RA>3mkQ7-Itp1mw})@4 z&e7F+k@3Y3;Er3c7$b3ORzj8pi(gwqZv%8RT;(r+{1i*jiY1(bfL9S;K5OUhrn~7p zdz;R1SH)vw2C~W0a&YNEG;1+U?{BBS?kYX?15AGHbFzbZSo|*Bc@bWJ@-w>XFVeQ` zI3K(`i}!vxkNu6XzYZi9kG(BXX7xOSVDQ^1_gx6#o~9O zIV$*S$tgPOPt)Gi&95H+BQ`zEXuk!E&q5$C&BB-0D>g_^(`hEZ@-a3&k6b+uM<|HT zl4kkz9qe0im0h*RsYpG5J>-H$C`$En#Ah^)rn`M2Xg_&*%~H~u&s zN5oBzo=cX?ir-SoSBp>3-FS-5)-HbW*zc62)+k>tZo`e?C}HXAUn$m+$Zb!4^;6aV zIeHetdY1m*y5uCC^`~fW>iUQMzv$)l^wwOWxABMf{|7I8jx%6aj9-y_4p@s=HM^bm z`ZIJaJH@1Nk7A8G$cT)@63)is^t0gITH0HVvS;}*KA*aTJk3U?^sSLWo|b`2vtqY+ zNO;DxVrDyS%>%6aVlO$mjm*GE99j-Jz8vyGZmhar$;Ar;ew6PHHGf4;Jln7-`Ej2s zN;+>MdumS7*?fV9#cLF^-Ws*4c5Nb2u1n#AgFKI2G0s=d=%Bso0^QB#_&JjnvceJ^QdGXNMkS7W8m{o}OP~Fq{{P^WkFW-Fl!Ac-zW-dR+LS{xgJVs84~Vf)RPIS(U}L^Bc?A|oA&Q!)O2p8Vx=Joxx8NtTLNph!`f37k+| zJj0|?H%%;6LQnr;1%jR(yr$TQ8_y8mOROgZW^gAj^|+?p_M%mmxX^XF88?8Mm%st~|r zU`X<*OjT)5pS3_G4i#{`YON_VC@=l0N;}Qb(*uf?BMD>iQ`E#~PUHs#T!6X7xP!)U)U!trd!^YrTOmTX*a-6PMSnQPZ@-WbVYr3`sVA zxACuQU8Clo%t4vY|N8tBuvF%stP7LvD>Bg{bt%jodG>+J*SP!s2l0h;k_ANsLIy@8 zL1Xg@1}|M#Yh+<{E8sNesLrMA$pB*k7 z_}NpxQVDk%0|Abuc%ya2TGf9d{k)*6QGKte8j!CTK-~w94bPvSJ@spq;0(u$(S%VG zNwaOo9tJN7Ukw)5wkXC}C?SOxNv2GjY07h7hsvsIX3w3k+8VraP5u1h^Vb91LN+x zhd?+&JYNz|5%frcy2fRkA25^W^Ov*?t-rGGwf;^IUMH9si%T;l@YLQHO)HP%%G##k zaa>f@KqOIYO5(@LaQIT@dK3^@WZo59uZXn&KE7IbkiDX~(sYI2Kk+`nL=iSUqWqYX zo_nZyV+;K?SLtuKs{C{H=vdRkFj7XuJ&9GbI@w!$o}SiA+*kfvLc;DfQbeL-Nqg>N z>Fb*msNB(To)2DKj7IG8(*nUQD^JTj0FVMJXn*w-+ZLEt- zwXn2fg*X%WxbsclHUa4bl`!6hF%Cpt#cMmN=c|GrJJVHh3V=Ar@Iq)wQ^x%(H zC~rnjBtkS&!UwO+Wy!~@nLqhU?y2}ShEYz?TS{@{XQXl;W$}w!=&BlIfAav--l$am zysn58EftaU-mAQ@J0yOozsU1XyiX`uj8zMxMazgg?qca{8#!2ydWV?yTEkHLDZW?x zuUDykTUbH?qnN_r18kn#PjB5tdYdou@VMs)#PV^4V}#@76a^llb@DDHB`xv$v{xFG zT&7dlmWeWA{)sG~v5NyMuF~0XmI=wH@N30{<73hN<0$n!LDfrJINS_vOV9J+pBI~K z%b3XYmXhzkkEy?^Q)^3D)pc_=;g1zy7vJG{DPj9K#@U}?Z{;~UYtOT(rB{KL0vyZv z^RENPH_ieREHf1FDFrwwx(*y1-oGDv`dI~XX_EAgmMY2FtzWls>FPDa?4hcz@q6xn zSJb{PY+tunZ44E^iQKnRZjb_wcp-iJkA6%1Q%X<(Q$4M$5(M-WspCV(O=Xxf=LY%U zqiIZ;_8IR@nacE;b7@_%np0;ksKPo?UD?~UU&VaVx6-bh!Cm{0bACux`QgVU{!6MD zV{U;Z_;=F{QMJ1+Z#3>e+7;OC{=N*NU4Y8svxp^ zpIO8p66BfyHd-kGUE-)4xfuELMrVu?hvK z`GaYt2qM6f{Js&2Q>q2{d1;s_zj=_?e{mma!%&j^N+~2RG3SkS{I~b;um4@1F&Hj2 zS%-43v3%U|d?mLl&k~ZoTY$Uh7+eyo#tMk(5yrRz{K_8SQ;$ZS%|TP%C|dJwWw(1I zz3vRQxbvvA&|tMN+ndWf5gSix0q${xn4go!nVb>q^<`0;@K7C(&|)On;`7nw z$lx12hsKzVr~Do?HP6Qq@fGl>TjRy73|`92+&f;lH7C+A$$-7yjn19PHeC^I=TW}=Poul}*EteOvGI{5=pD;5!gK4?h zJe%uaj5n^vBQH_PSig@?H76~)mo+{YTRk?~JbCn6M{(3Tl6H3QVeJNP(f zBp>Bu@m`LVw{jdz&dX(L=1*BX>fczN`R}X?I;o2%c+#t52&*(PMio$a^%BydG%t+0 zl`XDYXo|RbyW(!rp>hSBcmiKcShPjBqT}!f%XnebDB7*JupyMeC&gjL`2A{K z`GSSSwGv#h3P$ITBPT6P-8i}tCniOr%BH1U#Q=dwxhjNI;Hm77!n&2U85M8ip>-)h zr+AC&!n`%r4n)#f*ttsTTCP!rl_iQvEWa0&bs_gE@6SVPT`_vao=RliHSuVdby%dl zlf`RT09#i?;@)C>da?OCx~wyKH!&rxMGjr9H9gAskj7IUFZYIGB&DduD92%x;RxsB zjz;lBQ;I$9OjKZ(Jc+yuYjO1qrWCjA1*U?MiUHK~n|{xMM49PH95bc$W$%fG3l;b) z*Er<;L7l&HZ?bRQkgS*c6dMnDZgVdaPh=@-5rqUTRDfR-wI0(u`tSF_e|-FtDpb3zm*91A9dhxN8JRDZvw|Rf#d)3 zz_BEa#iPa(ODe$Bs^uyMqSxa`vlOs(@nHq>e%Y{->_{%74I5d(5P6yto0f?!B!H=S z9p_=!>}c+MHZJa?XT>!(FFlOcn__e{i=XPZk)h`i@D}pUGmGez#8v%SW=>v=J!r)d z%~OB+a>Hz|+Nb_){eo`%-Y8kIQQTqtM0v6X94Tfz->AUsp6Wq9eB*O+qd8bZ0$Rs; z^|4RrtUJrDmVHcq##c`tXs5;V~v+sSNRp$zHD+5*<{4B z$c|dc4A~eJ&SO+K7mqhgB6AFfng%#ff1aATtz<=W$uxxF>c*})u;~`;VH@_47f*Hx z>*pS)qv0&yG1`fYpjel-1_j`^1=aZLU)9hFycEwos z=T)#tNoAkn>&9>Jc{N6dtYpTr811_Q-IZp_)3x+fT~j`Kv))^*`rj(^=YMAZ*IZEj zKkJ<(I0G*7f^M9l7(quFn<{(hS#g2QRXzAT1>6y|Gb&*vJ88wCxfowQflsH*1_)NPCi%UaDL;ER$NUCyCpm*}iJP3`>U_RklvulXuH4QHA4!J?u5w|uYu zpY`4XlW`n!s~5e04jjvijR~$PM#XT#Lu7`<`?d^ENTdWySh;$ON-Bz{Xj@wkufP7T zO1uf9N%`}Lq;OP#$`n%Rs`fu&U&P9BY558m7TG~hFiPi8mA^1nOcr8+Vs^-jq<8~M z@=X)aDO89amp}7~6^vC09KV(%APM_uz6l&fQY!g1W7Yx%dWrX5%koWVQ2{SZ$zm;$ zKMWiTc+~pET2<*QFI!Sk9@Nar-OlcPrpF#ocF$u3lI4Vwr3xSjC5kWvqzFZn=bt3L z_i(wfaL*!1&@XP&6(oz@JSCR`Ct}030gU8M)7Oi3=#wt`l$+aJRG=poB zwYvsu*Y6-0n}A2YM|stzn7MGF0>0$_n^&$;er#a@IYrV?%SZZUZ_mEr-x+)NQ)ZtU zNrKTB2aX+O;OZ5QpE*xRBs9aqz^G6_mBhN!XDw9YE5LEfs51$%P9@s}t4=M() zR9XugTgEYxETp!vWmt;4uH_3g{^Ha8ea63O<%Xg8Ctj7`Xa0%IrARyq<5wgWQ=B|^ zQ9a+`(Bc`*5AEP z6O4~h;~=1d+`|pVv9ux5=9Se=Y8`umBJ(HGq*7ob`-aI-5MMR{VWPgf*}V9WvPIb2 zIzU95h&>g@TPEODf<*8>rajwC@6zk^*TSMHq8L!Bd>QrRSYqD0*|>PAZ5a9wVkr+5n%wUynbYk1HX`+I*6Md#QYVGl%NTV!>6P_wvjm zZxV~%O*lM(aBw_f?>GWl5$;4Bf4TqxjeW@yJc$BAi8OjNiZ`YyPq>&di9*|x^fsY9 zmTMP$g*`6v)F~1o6CE0Z{7%3aW3J;sF)95nii3|(V10~rGxpQlaE+ZSkEr4qkJ@L( z6VOW$nQ2)w(!MwK-P)G|A!*+*N=?RiqExMeWaxfnXVTF;sAQSL`Z%mfgPcSFZz@Sa zLh0Y3X( z3;SxJul5?V-)qJnO{?U5%zGEFKQWh%hAZr7KFtTORPjjaPxLOm#DOK(IJop0O*6JA zz~qbh9UQ2G^$U*>HtxfwCGZs{@um{EwG>8p93k&GBH@Wju_qC{mwd-#lv{tpXTMmj zJfhoLPVngmH3&P#la!=uiIUFVvgELuf8y;d^Dj4Q;3`hxh$NM#_KfG2(_4L=&Lx+b z`{ueI)c$9a>yV^*P?E7msRG_NFE~I?^JUh}>s9?{O9~@Bf-9C%{rmYJmUBSnQ_Xc| zzrLKPHioDGaFXQq-OKixQ?%DyV#}Ok6nGv%#?T0kA?}(;S?=R}bYCs|m%zSynEl=| z1^he1DYYgGLl5!suWJ<>ysh~>pMG42v~MjmuqR?f^@)@@pJer?-R!P8!`G`0lPtLl zNmk1qd;>U^z5ZYF7YxguJx&#E$lfO)+0nD-mB;SUQ|HzGEk$Dj-h}ibBC%4c>y~lp zionL#Sz6!pz4kvT8dCe;%8j^(d{%?;3KhQz_$d;^?m*=4;Padq>5us&Gm%(f%60ifpL>3nED> z#dmsA)MHG`42fmDT2ukAHzc0hdk(5|Ru$?gDb{keZ+X9G{41>6AJV=6=1Rfh2HP)O z#PM^c+RM?CgDQTI_)UtgM1EP+$7au~8b1G)uh}{r|4AJ5UC-ayVq+6$-{E75Nq%9_1des_C;x}Qu?ga+ zH8`Y%BT6#+51&!jpHtTtVaS+=3vk9o0YfZhiU`JK4TjM~Wk!lwhTf9`@e-WjG;UEg zicY{MzUU&4KKvf_{gUDm|A`#6#P$Mcd17)u`ASAt04Y&ZQcTg0_4V zP7XT$J4+HCijw)nlj9hpkL9;^jTyPwwCHwf(jFcTd2uI8m9<2XJH$I~8!NMZ%%+T= zvN0=@4cU2Y%(bw2^uN(*`9J8l-%7tNi|yHWu)%T%Us~>9rF8^rGH+w^h+F8g{g__& zkJxFsowb=3R%B$eEN2vJb8csK-W^Q!=JK%NLyIP{$4VLR^)M&qW_3hovolDiHII{y zk(|lBgX39u(rw9QN5ID#pOa?I!;GS0UMro*8(B7LM`o~53sIwoc+MZ<{#ZVLa=59@ z%VKrVMs3W=A0j?Vl4&DD7+;iMo!LKTTQHALQxPV31B%hB8HFU|oikQ~B`H8{KIxFb7=M6? zz5r#RIEhFpk+4{G$Xph=X>%#(ep{DXDA>Z z%18XW<#&AHG#)8POBCab7h`Bqii2TNp&~-sB!c042t>xI0(c{05YdxFLS=*u@pLXl zyurr?0+i}Ep1*{!QKUfKzkhxKw54b@A3c;J7D|x{CMgQWDGWr3D6jV+0DK5Emg_~f zUa_r>0`u98d}3OfL?}%vn5Hn8q9~LmE=pta?k1oAg+aQT&-VS!m;bN(Z5R)|2^@d> z01bE4P2l(@aC{Ru{x1(4OVyuzyzw|WhJ~DP7FInQuh)xVN%Pg>qe}XDb^RX2jLp=9 z{p(ch$2>h3t7b77y5f%MfGGkO{> zD*vxpA1zf9yjl7vkd7O{lo`%mFW%fsnG|pd7>!Nl1`GxoKc~h*N*TI46 zYiz9CgDd1#63iLNTtQ-zak?eLW z058jziOsO1+e_Fub3c8{&#|p#7j8#Tfr=R-BOJ+5_w(@+i1+UZ@i%-%uMtR1YJ-Nu?}JNSBf8wYDI&|POt%dAabcX#6=h(Vv8=BR?;q~GOx^U|btieniMz&(uiZrz|=)vu`DKOLK6UKF+ z@(0}8a*j0%+i*JLj1IdPB|g3p8!oHP8;?z=r*VL;rgO}hvJ^+iL8kaL8-9Y0@vNJ3 z97#H_Z`iGv@0OqgOCp<`@JIr-FjJnaWPd$$HwbWEg)LxZbYL`jhKGQCENf<-plihy zHn()(a%f7zJ4-;-NCtkJ&Xgw?a-b3R)?VPV4;Bygf5!Lf|2ZGkE4Hk#)ARgp{MLMS z)*Pm*?j&0m?Zt0PkgEq6Ww;m-v|-h43WT=@^00*L*!>Q?eits?MRqV3TO?1(e+Qi& zQh5bzn9)O54Qy-Ok4JNxe%7HpvW2Cq+f?5P;KlDQRh z)}5qZ7{?30Q0(~Jh@I?!o1B0XtKlYBv#aOuTGDK-I!ssd88$B2h0CEU|Ld$&uG$;C z4nvJ!N6k5Ung*Er;nE+}|DR8;!V?PVd`q{8~2d}*T9-4q70&2x2;Ub`&SvjqV zB(ov%dubwxlHn?UVUP&u>XT$kxCoB`xdK)S=%IKW!*`3ai(DjaL$2s>(S< zFp^;I;zh$in@BlpQ30`q1tF~5a&-?&>RZ|JO&gncc2hCwDfM$D$(JhN$m!DxIFtJj z;8;DgjIm*Y&!2w2P$hP+a-Eh{>+p+AvG}8k%(#)_(1{DmAMfl%6l2*CS+78hXjGPIhrJq`R;*V0 zo9uCt=oP8dV`m1`{5&^snU0=5+Pk{x*|(oP9bN40?xm~$ukx#_H|_p`>#vJ3_dJDL zD@0eUZCS&TI8RX5yh7clFuy8mn$5jH{9J|Yq>4>MYIBV8NfXdXSYHYEeQce3Q~_68 z>JDP)<8Y=@SYt883`=?0d00I7bmtA4?+%BTw z3FIXVoW*gxsRA_j7~c5V96DSv9+x#c!|Yji+Z5^-WYVp|YoFhRJyL=I7%KMKHv3E>@t}QIX&Tirr7pRezbD zx~nwI+h&54DJh01Kns?l2P8pPfm4`jsS^AG0DAAD*#0={Ki{ve-LdQ#dZ+?DIR>Ym zRv@j~FT&;Dvwe|5EFhqX8iQJj>RX4`X=E{rLm`D z3OG*c_cHzWO?1^>rmy)rQ~q274ytop7<%H5D00=x_f^+ubGY5l@yCX*RY9Eer4&C3^X&Dv zKT+}Pjdn$1`sA65>~8NhgoO+xaKOHnKlT z(UOX(l~nZ@{9%a+%wo`BG)*89qx--?6<5ml$y$)TTX{SSz%Rv1D!!66B}J0LSeAlG zS%*?QnWX2yQFXnHMNW5iWsQpB~cMRUdT{y#1*d&$=<8T|L zxWuAHB=(g8(YNP}zb$#M7=}YY(G9OyD=v)@MgS% z*@1j&bd6?BK-5gy?RV0mJNP&eq}m9u$!=r2$I4p6!E3>Q0-XgoKJ55+79_(8oVLX# zq3fkQXbbROZXR`!080xD9te4G#8WCREpjE}f<9_NQLFyu5Dx*=BCKI$80k#DlbbCg#&v`qWy>=R+`P9V1Cg?dpY}5b;+G#N+1w=fDjVW z=r}GIjBVWH-feZsy&D_jf_ss><=!NBdw$1STQcP)xxu_&ULJ#S#<7t0*`NKX>#Vuv zoOs^j#fyO`ep~3nd-wkYlkfe1@TL2I;zal-SYgU|O|W>po(TmpKN`aBNC-nL3sNeJ ze$gm~UrgaR zK28=YdJ#&c1Rf8Ic*E_-$h>@17j(xI&;6JX%)|ShUU)U=$3udMl$=47)(TleaA6I3 z9p{iyRXiN^<27F)h8E;uOkp0zdJ8Z*=)v2Cd3e?1!!teszjJ%=wx=?chfpOZV5>z8@{-qbscGXGzaN9W9=s4ykd%uMv#StEl;EL=guz}P z{^a-Lxo`+C`U>%eyBmh(-;0Tb-7&%E#Ru*ne&dhg(XfJ?l;m?D%qlt=j`^ryX<#&p z*F8P)eqm31>~>*-+l@(XH$L@w@rS~EJP;EhQ?V9VU6@2hs)TBm;E`|?ANX8&%j?5$ zJYoDf&yQihKUhM6a^o0tJLdk~F#5VqX#*?#27 zE`;nT%iRXlAOeQKjL2B93}4iq$F7Dpte?9MvS7leDDYc;gfs&FL=1jv4EEM^U|-X9 zOdM4Qj~Rl;cClx$63k-DoO3wT(2k8WHbV{R?62!0PhC9(QAl9a?`C0FT{{je1%|yd z838qdpc(|Rr2q8v80>8TwzZtYrz0jHEQ#={B0O3E0nNuhi$V_TXPjW^>a{cWAsQ1= zpt+Hk@-TQVhyuc50D>GuETqEpmNN)@xS<^_Ggcy?kj!s@?;BP`2$G1P6@*7Ggce9+ z%jBbIU3doTnm0ibR752SF3ks*6@(H_WAxKAaD2fH9H_s7(Zi=9PtQkwswZ5A8&bFw zTV`G1q|xbFz>x(vu(!D#yPK|IPyH<%ZoG`v`A2b};R3cdUc$sr7s9J(a2tLEG!HZ( zjgc?S#i9Ckv^KV3%wNYlQpkpYLXtIBBM!)`2AE|^^2dHN1h-zu{jY{H*gWe5+M2H5 zi|VZ~{Yivm5k5_ZTb2+tB8bcR@JI6z)?x_7{g5O-qSRk;5o)j;>$z~?99GTT1Syt- zOVi-C1nz$+sABNbqp`2%0`@mu#O5ackozd(;d3{_#>H?@ zMjn>hkC^%W`hRRA+?is!HH) zAqovYdTK6s5&?weFr;`KKG_FX$^$Cqh>9TyaUY^e5R#z5@>O70-DzeBH_uoJpX`U% z2*IbvK{Dnn6|aX;7#AQ3l~_OTG!862ht)GyLWx-Lt13v^n%1uxmhY_Jy^Yr}{gcMK z?Ef2}{vS1LJ_1S%K2>D7Q%XcR9$^H?IKFwzfJ-{eVv(xb?cN}UCnbB-L=_ZDoEan& zOm(!U zc+e6kM&M#s5_svFFCbb~kcm;PlgDxn=i0CE^~XxyJXEK zmYO5UNMgMb_&Q|782(#gnyy{{rQ^{>bDmj2=<8k%}m0%&o=M+vEX$8}l0$@z@Y}sF)cHAL+E6#&^-uwcK`2z;+hMQF4MR`cIy{ zz->5ux*cJI7_WUH7^N_>{rEY$;$)Sp?X%oxmu~@k4xIwYp%T-Ge9`ZGwo$_Elr)<^ zoBHnjr5g-*b25YrT*#yQ>x}tZXj#6JKd*}=7T1zE-*KJ6ZyGn6yN+=R%;-zv;vY&Abkn6L>66)E4tr|dG%bBL5X$RKUms4Sp`L_L_+Te zNvHj|97U1Gv1Q^p9B8_Mb@d0Ksa1$4i@+YLawW2n$MNB>=3{T|b?j@ohUvqX!xH<$ z(E2d5G8=ys8|R+H{^sjgH{&pDZ6G4)3?kVKGpdsDhw$OEbFsgMK;dhc_3?7{s*4(F z#FA^OQ3z?H#v>g_mY#s}<{y3BPZ`o&_B^wD=Z=zdIe_WQsxEn4Lvd8HBAf&f?^3 z;NmRc_*`J;f{Qq`TtdM0*jK({gijdU#C;!5 z+X!b85EDh*Ulb=1V<|>sL*O4t$tqO5QpI~YfuXeD$BC`Q{_O-B(!U9;szOwy^QI5f z1Oa4JJm;`IwPX>_bU1}Whuh9Lj9ZKRR7)JY3Nwf4b85H&%E?USh>DnsreLHJWzs> z&P{rE*s7=7uk!N|W0n+F6rI@C6-d~9p=ia9Vha@_-MEHjt2a2nD+BNjfJ*!DwAqay zX6dzCj-)c}!5p&?>-jdYXWt=w_~B?SY9r+s8pD`Qg=-WGlNU1iM>^^>^iJ24Z}Of4 zr}_6J>rUgyiV;-wN9Q8RJu^5=@e--zWVvvX;$M2PwFWvsYCbK}T~^L(Z{z*UYD#TJ%#~NA12XX@aq6Q}V^UxOf3Ff5IFw_AECk8;*Gk7!{ z!v}6JYHbU%(g{4SNvu>tMZYD{9DXI{@Sa_QKc_43dZrx1ECI9Rd$B0u$C6M2qvHzx zRAS;csRZ6gl;HJL6<*I&;Cg zsHiA~rrJoU6|nRNkW%{MWw!^jd;Y)pOmSg_6vjs1eGGgbj{F4Mr2DW^%Ex$LPy8w@ zfSA$my7I6;+#NgP-O!Zq;qj;lRqulrf-%g?y9Wzx86V|}&@YyRtoA`X*`Lp6DMvEL zCS=n@nJ8m&elMJJ^}@Vl8fK;vLS`TmViAV+>V{*Xd(dFU@Mx->y=E!4E|+t7#x0}H zTYw|c`!U5(kkzRuwjX~#p~7a(PVg9#QdLUL;c-F6$oucXKKDI19Ld9ms0TCi?!)+A z_hV+keQ3=42{yWaioKryz<&S#!k)-~V2d0?V@$^%3c`5S8$nG(!ro8>L#-4SSoIU* z7?R%|Yh)R#MFX$NN%rgwr+V{wtr=y|%nFuir*-|u{Cw>3_P_=)gmLK%`ilmyCxPr4 zv<#l}hf&@Aer$ES&{mj_^MPJC5%?KSgnoj9;h$oi;>IX3j6qTYQLQ(kW?xuTbU|^x zmBav{5bt?^gk`>;q0Q3`=REi0l*A3@gKf(>qk8r{LW3&bfuq+bAut*foxbiTm;3sJF z{yQ%Ceuk5Qo;VTvId+LZLaiRa8&U$j#T<001jTv=kA#Dm?d^r*!F)7WDjrFt5l&|i zPN(sQ{6d`a_^>y1QXT{{M&l{-5G#C=X|Ry>QXf6Bk^>(7zvJ)d)(hBm^Y`BRv2fATOQ4xi2(Q|WrH|iz zay$++0DJ3!@x$xjjs@V;N%~n}pR|p0InUVw?i!lj*d1?#;ZM&M!Vf=^VAxJT}^#}?LH@rq6LUI&RIET-tpTO~X zz`7axAVgveJm%T?aH$1|%OS+%AbZ!+{f$6h>~8MBf#&O&KWPa9MwtKJZwBFt`IyD) zRy`=xT`;4i*id~G+Z#?|&BCn^qdFq8h(gVaf`lKEki;j?OvB-t8_b6JboguroeT6H z2pR!Mfg)_3)4?8SM{2I2`mOmG^U+j%I%YOTkDP}w!>3~EC(|);aL)*;S|OUu7+O~P-sLQ;8+t7R){H1tZ6^IdI)YY0FUfOED?bk z&f$wWC$Yc&5979IU;CU5!^UZs`5Z{gJwkR6bl<0pdysaW#O59Dp3NkqGwZ zMDmfAo2Z?%jODF~jT}xy;8MH{A{VGWkYBPE?S~Ds&oR*a`P?rc3l^{6d^L~PuPmr| z|EY=CM~tHeV8YN^?ti!G`fmMScZ2(X%<%c_(d?0dSF$K0A0Y}vjK2M(XW#g6L^Aj`ll zaH;(&o_XPQhw*C>2u+NSV)o=CIc54Hz3<=&C+T(rP%P)O$>UhdkT)JxvO8=`@@FGv z!DCPV3a2lg$K{*X__|Z)FJb-0?O3sLBNi=Qhn1^7$Cj--7&yL-n`m0PoI%t(z_Ixi za7^Irw}Im>pwKzbzGD6mbAu|RyV_68Oag$q+E1RVdXoB&{F!syegZ%!ai}CnYLYo(BP z;sD3Q3}@h&l3@%SJAR)IOQSdRWFJoOkyLlv`3vZ{c?)MQUIxkh5(sWkH8Qc5jbe5}C(P-~EHS<*Avy0(nh z#l>m7m*|>truEz3IPY?KTaONW)er*PI4RCuiR$O zb!KaGK?^Z-OYoIP_e}An9T^y@5}a%QiZQ-)`6?`5xdttZmSXX;Ramle4HmCh zgT^Jxv1rvg3>jH{2ROc~^+&)F`=*g%LK!EtslO#H$p8-(%uo`)e&KTH$znv5B+P7Y zW)~B9%s_=s>?LA35W(_1AmbYA(4Tl4`3i$Q7d4p1CWV6f;~-_aj^a->Z(6yPiK-HmPp!Ot%NBKfI{GF zGJ|+Fi&!EHOX-KA(BoKD-HP1}m+?j8N$BwboJBri^)fsvAnkN7;I@EOsx!JlEBXrq~e1x_SpsoFn2d}VEl)TTs%d< zaaMc)?>fTH*1_KS83>DYt#^o2!y{ock7R1a<9`;+3w-Q0iN zum(gGDqJFPstm>Ywd)~k8A`oy&dj=~;m}yWk z6YZsRHj#S6>;?6m`!_JBzR6*@-u3=B(E)Bd{VD%${B!H?#XnR4M8%4Z%>^k&96foC z+jsWD6(o31cM6}#Lz>2z6)1&yGxj)4#bbkiiNg|dC z(aan~BZZR}o%6r#%%x77lX6IEjv3qJtNP)`Us*LQU;PDCv+53TteFHHpXmgSuW|hL zpMm4){QEB8n9siifn(q0{G*})nggZGkR)$k20568Ps%z}j9I}dHF3>`l`X>YvuAPX z#x0y_zkVmCq(Y9Um1J*VQtXhZ2$tE(R7^<m#6%4%;Ig;sDMOa3{iQnjZ0D-qO&fg1sae!kgzF?4?Y=c0yikN-ssf>81&UP*&-pxPj0@Nr3*)U+62V*viX z$a77*be>7SlkjZR1H*Mkt$R zN#B&1!$X3F7XxuT=?&qDfFJJ`{up(x|Bj6jAJ#-{49oN41z!Q43l`wzkQ>kYym;2_ z$MgOGejo5+Ni={1AveaWCQ76Xp7zW5uxA0@54iEmmzwZ^m zWN$C5Qu0urk}<|epw`_3N5l7Fz3jzQT|tv&V3nbuI&0x&HH2#4y*L{98IA;gh7Gb0 zzgA3U!af&_qoLq_)R_w2EhgBZl2NFH`Z^03|zuuzMk!Bo+dl+b8LP@B{+Kb1yJ zPD5SNhs{b1tziRGeIj1+1TcrjJ?_H?Nel5r3IpO%3@x}HE7drbD>7b}4L--BNh+|V zBA+5II11Z+pcqy7bFW_5@BJzF=csHN{lo+_&{Jv(4-0;b_H@G+1@~ZY%#HPO2usW; z>U04sl_0i8z1S;yu|Dd?@OTXU)HDn^i-b`QOHH9lisO%-pJ7?(Kd@8iiM>$)pQ{S$ z6mvYia4OJ^neo4%g6z)kyYKYug^t1=xZvuAtpPXI#XVSV_?cB-pU|;RjbV4h zgVtyv)?56}C*cg(u*pdWzazYzrd0G$2+oOW$ET?^%3 z1vu@J%AxnJXZ-g2zw-S5_OpK1pH;n*SpZQZ1(y+l%Lu`v1R={RtU#Lm&UO)ST(cXo zXb=K1Riyxeih!UbAY?`1wt{f!K6s@NB!4-!&b`c1)Ej5-LOhT_L0o_@X{d?z2Q2xEj$BD?~S0Xqd@V%7x%)BZ9IeOVG|IGi|pm<)_w5l9)#r>$S3)WX~%GK z9`M&PP}Z zK`{LY#Jtc$CD=0WELt0{VdbQ4FqI5QX1Op^h;Di>crC{#*A#Y)0ole(j)A(OkU6t*@v(~jOj3fN)#bEhKLz~H|BvH%wgla zQ)pds9qVgb5s?k#Wj*MYaxw53itBhV`y}c{tihc3S7YegpTe!U;Wd2-C?aCsQf!=i z9y{vWvA$+MqHzn|wS2g;dGMzSp~P&w|Kt?xn++VR1EvmZK}ZR*46;`Zzz7s!)1-sg z*K`)^8h1bn+lWdk+`6B=rRis493QSBkHf1NKXfj_ssOK9$m7e`S0Bf|rt?_Wv<-5^ zLR6CB)`D}sBMG>#b%6WvL+5?3{tqW~_^8Mr5r$v(K@R7zZcZC^Hn(GK%^pNl z6?wTl++*E~!fcq?+sSASW?mUGo`)fMl8DMt_|zy0)HvK`ge9mWkucIi8q3D*VyXCT zi`pPYX#J6_a|~`d2x9sNl?ZzayApnMvwNUW&u6A_T%-9bAR4wI7WBrpx(-g(H%wWL zkP?PZE@W?QujYZv48WHNp)l%$60F3QIUT%yH_hFHnBPKvEDV3z55HLeL(bu?C#GQc z0-&`CnDS9mXa5)6RsWA2Hkaq2SBXNl9Tp#X7CSPD4j4yZqG+U<5y@sr>f@GVMQd1d2)ivXsEfK_j0aGlPlM zQ*r&q4cxqa6N^@?b(jV=CG1FE&;gDK_#)u$+raUE-rvi({VFB!zSe%#%(7HHX?Iv`) zlL2X!FlbEPPg7^i!{zIaZ*AS8W$f)N5YV3@R%{k0&UP?k^J2#>_7x;1N=!|$j3gz7 zM~$0=D>rU&@^HmxTOGCzRXSVb325PL=kvFnInUDJ7~colPIBP@&99Qm{>(Ze z;F`c=8cSkn)9;9JJiDfm!E9y*n<;!a@+%q3h09j6mpOsm&Kh+XzqBu$JbRwU|H!Ee z>=8=^DO6}dX?*J@+!c3CXYYH0eBPMwyr`Wq(O{5`N zDeiw-R|?6EvSPtMyZ(p;>Li%7RBx8NjHp=#4rslTr{o92KEAV;efGtd%;=^z(ww1v zgXRq-fC)s6=F$v~k<|1T)n}Lu{8_^xs7fUwRt93K6eLBR6CcO>PflaCnVl_{FyWIX zX7I{c=QC5vL(G^Zsp$=Kk0D_UfIn$KC`p2tvuW)i4Efaz_IEwj2u%NQiIYT65qOt_ z#9nc|kxo1SO{#=J76df!j8gQid={U+Q;%&m7qENYRUB-%jWKW6GH4;>Y`C)uKMQ%v znnqs+R1K*Q($PUI-AoK;0*-0CY0Vjl-bm^XA}v4S$ZcCiESF4P>nZI4}kA;sY4dSW;S*S zqIMBtiBjl#6|xK*pT@B{+9BX3i%k46Y?yow`x~xfQ|%F? zn>^mjQ6fHxx_39Dtp+$S=N2Zs*#ucEg_-INNiO5Ui!bM$!~VMKSTSxtip|Hks3BaO zKprjb&@rvvLK=jK@^68gce-3G6=5#xZPH97z7oEZ0rX!sF*yCeZmtY~6haI%T_W!|`fP zlFI5Tq2WEXicsxNz+TE?vI~QhXu16({~o zBbBM-So-Vn=eHThnwbWjo76X?+(Gf+H-Td+-17LQhu&!( zkz9RDD`gcL+Sf@Yo8m1qmBG0V2M|AWg5pZ@cCK=Y)KW!UG)re{oC^_%?N5c6RG3xF z*Q=I|3s)Q^AiCaUPjzSBlUFgtVzlbsgA!RmKh2NQ*w2wMMMv?6 zq`A!^r{M4(-pR*tp`24_;p`30=X^c?NaGSt6)`iiLRRpq+k+)? z9GhbSyki-NCNk(FYRouZsOwmwtN4>)qD)pWHrN9vLO;TcoP=010ap>Rm`yw?1TdnY zCzh)!78(gWEh`A*a`0wK*=u${NXDx!AAU!Yz25sVq1%7Mm#%-qUcrs^f`m!=UVPvx zz?+_*<89xM@Sf`)yjRc*zYYs{$nVG8m>-A3c^GTjNJzza*%iiePac}B5Z)^hQKrTa z)iO|0eUMXf_)ESE8(iJ7UGid~Dd6K&5>JH{jCU7ehp#*4m?;d3mf-zZ8cX9cni6pg z5B>}-;h*EE&;zIaJ+NQ!<9#d3Rst{j!dT_WXKC!$b7e?L2|l-D$x_6uG~#v!aoV%w z5>$kXF*4tU{n752or$B$(vglU_+?zjr`>*n6aJrKcRYZJ?f`}scE@1Xzu~X0|ACQ( z{{xHsKf$hO0hTE-yzBDd-9k5p78c^Y{BHP@KOYZj1}o2$Ye~Em^rI$YVVx1jGChR) zWEAt$CZ?y;s7@IenUe8Jst5zq1JOU3!Y>s!W(#2)4clmrTllrN5Hl1H)~k7VH?478 z`#>y;AqDqhDS^;N1izORSXvJ8WC=6;xkxWrgkZ8CWTPMc+^Yv#egB5f)B=1|ltQ1l z4O=e9!$KT`y+6h3z<=Oa5ySo-0A0R1U95G0cnmurJmfr=s^_j&7j8 z)*mso5{X1HUUGTS(z84E`aReck7A-};)$YS^v-4Qa8koy(T}xJ7uuq3wEFTfG9n?R z)2blz(&DwaVqzL83m8>M8nT}67*;28x)!$+}0XXlP4_zC@ zzY1{P^;~!Rt{N3EO=1P);4xzGS`oPA5X(0!UK49)AHcS{TUb}u3Qg4D4-xRFBP!~M z#T5j_5L{9|#B2!um$AFLkN#vWq3=bwhf&!!LyqjQgPYvl#VqBaXHJ+vi=t zs1K&XF9n!Q9MMFGzBCq&*@nYSH?hB_9n%LdN8DQlIam%eP=R0N{*2u-FXKc5u&<^a zBi@+=pC+(WwGxr=?vo>NvwIej8pS~HM71%NEeWA<)c?3 zpM8oW;V2@42vM>S3l>51J%HVdu48*$JDO&GhA^>~6C%Q~AYyU^!DtX+Lx5imLv$D8 zi)rVvXW?b6Z6U_1iKtA#vKwy8136;gqvxt|pza#>G~C4aAvFlbB8VrV2uopTp%N^w zZe@^f^V~K_{sbas5FXWqfF>Xs*C7S6P=zW8!4ibx8ncJ}Mgc;qA6lpsAHCSftn0lE z?HE0H1_DYPZe3vJbx;i;DtjTw0R#yAR|JIQAVR7aLD35^xO6WHiSZpt z;Nxc}<8bXwmb;$#*IMrX*mv&#N~iy03HYQ4{F}k2q>#cFh zYJ?+UsJh6k<_Hyns4hg3eh7*mhMzpTPdSX9&h;y4S$N~bzz;(5rZN8Y#W>VR5{j2F zdhnFH>;D?3{|UHf09=nEZl)X-1WE2CNj8swsWjw73d24bhgW|8HvfN0-VtD?rAgi{ z14?qLl+YuV1tqRXK2A+=l2X(Xh^r>d6oIk{h^mg3CCeE2qvX<>4O^M1L(;n>OGTB; z^R|roM&y@9NDPXjrt8 zf!D8q<20tsn8m**;G5b{edI6?9N?H3$5au`NjHlCBLZC=nQX;QqxJZ4T)BN4?Kf`X zy`du@>EsvYFsQ9$F*bj>i$DL;mCKmh(87|*#DIeU&rkigYVGHsWGu0kDDh7~0=-yfFyRzx5QxR& zM;@;>&7U*4LX4;@H=tN)UPl)7zn+4T$YMcVGp^pe#jmDCYoM6DnQ`n$p{B8U*B+jK zla+#*y!{JCuVXNaD4RCd47H; zX`5wcx~L|R%#ggO&HX0I2G2M0O5eTjFeEhrTGN`9g+_^YRze^~Isw}$96RYqWwOU7 zC$1@}OW-wuGDY8%%3Qt@1RB#iB*qo7DM+G|fND}mkW}&jOF>jk*x3^9OCwbbGfncU zX-+&-F$T>62RQ!j>yNxaV>-!UI;sQI2HFEF`#t1%90Lrd5QssHerhMlUD6)SvdxJc zLN)=r8DwIQV$+1PIMM=q*>VPhdcVpbtf}>bEk1yf@Gmj#kE^h|<}!9PU&QE-=0iv( z5zA$GPe_RmV#S1g%*G@S*R1de5_Ug`S}`oMACjSmQ2qPG?BTk5K?lYSsbc^Y!Pzh8s)V@dygD~yQ6%O!Na|?4uuVL1eKVvZCSILw@7kh&M zHbOc0lvSemLh1`P>M&=au8Ls~9|q4$9m9*Bv5T8IOSZ(v8`1&ke9%Zz>Yx-`mB5_tk& zPB_h=eAeZ|P}& zT?$+2&wv*BBS-8~W(Q~D4`J)HvkV-snso?L(qS)JQWa(oS%Sm0SJ7H?6_ejw3SH@o zKq`R{G0W@>($RjH^1FpBbA4bwFnQQA$KN$cg&V~viamiZsxP3m?j|8lNF#j?Nd8YcX;$zjP7<2y;x zuw^jRKG1YJ?>l8?sSt>ZL1<4Y=5<8hj|uc5;Ene1;h&7fD{uUnL1c=ts9;V{R)Iic z0=WnzbN25nHtpE=wf%cF@8483BHljePo z3ZN){puLqC$7JbnzWoMBZGzfP{=Rg^cNM6UR5by1YN|H`IoB!i?8K@EPka?WNruhO zZKcW`<&;Db7Sy$Lo_{w$9`^*ksw7=(l8i91yQzT9<``5@<~V+}(m4=#C}{@Aug>`g ztlqGNuNA4Visg&Rdp!#~UCL*zqT3J^^G*TK_Pqx=rjBX}s8nb}%uln3$G}n1aN=J& zBNOQov>s~XSmN?Ed@cTU3J3^%Ca{^>P9Wsn+@E3z_7%3B_)j#GTsTHCH^mQog@ z;x0Ou>D*n=(Bi~zx19LXv8r+61I^6f_{nqp`>zAXJa(Pu|2Ki-wd{>eGT-0r{G);j z`p!V0agNo8$iji*;4aXYnAc=IL*G4q{oCK->TRc(;>+C!5m#x933xB!>**Zj``b=2 z2FqB}ys$DzWQy?8Yp-!J9{C$@-F1L1X-FlA>{F@=h&`i1NUHN zRz)nCf;Usn`^;k@0mBP>VY#Mav6jHIq6&X14R?;_X%>T`3Pwp9T5<_2v59Tk3&%ov z*d7aEv8H0KWnf0e#MD$2v(rH=&L~)HnHW)4hW<(%lOlOI8vH3HWDOXJ4Bj9BIRB?u zm5AcavLuQP6A?X&pj`niQHGZ#6V0g%sxukW5sH3i#z1vn&lFeaHoR!cEJy4Q6d7Frsf&6dD6QV{JdqM2d@ z?F_a-VPc>c-Vy?MS2Hle? z2?0%37)!Jewn`rCj}>A~TtKavMvY;hB_X3pkD?}J;P(}kh^I=RX=x0U6L>k1#cy>J zFD48OQlfY|62))bA$;O3D2iwR03ZNKL_t(?qq?vcejQbz+a#5pgqjNDxs-`d?J~Tr zn|LaekJ)lQzLdJ-gOrL$B8NVbjQ3pKvDl2VpYt1%21_kNELi~|NyUn#5ba9%Q+-(q z{q@58uru`USf_e0I+sJGXhYXZ@Pf~S7WaQ(m(UH{wGci@n5eKj_c$$yEJ&=Ym0hUe{aB#7~wu3*pgikODCLvE~! zcE^d}eQ5J_$JC-M7ndql1_L8e%<|{sT%a4i6ug*Zr|_6b()Ps&TPfr;5kvglus`^7 zw0Z8w&Tx?1S(Hq&wNpaugC_!UjBx)Ho1;C^8Y#p_DIH=u%^u}_A|@sW3UOGt7Y&w# zC+Pc#Q32g3!5<6m#Zi`~?uNCZilJ5(Wo8C(y8=zSs*P|Y+6YfPv;73@jOL)wv zfGQ=YK`=pcI(BA-#e_$B@Z`q^o79;pq zzOD-o`T;op8vyDL!08XbY1iWKS}6Z2z-d>2+VzF5>p8#v{;xd$zy6$e^KX_F(n}zx zGH_`EJca{eMPefKPy#Dw?#8a>TiCMjBwl#pHE6L6$Rn7*OY(D#M@7UmAA(|c2(dh* zgc2+rwa=;OUvL$pe^&#^UxtX^fF849xvH>!avS^m9$tPPuf6mJe5MzLdLF!@3tB|O zC$Emf!TG1rHt!k^%)5!E;rlT8jn$a_<_2t@(2m1%!JogcK;^cf_)tpW6nSSKi1?XjX5R_%;v0{k+3>2XRlD873 zcM!JMw_|_fRWwdmjd(zT64ao$RhTgYnxZgkHKt0?Jr&qAPNL0N=X6BxiG za5*M>7z8A?YETJ5&;#(x`G{%(XrUxpMs330nkzWebPMC&ng=CnK@k&>f=Sq(epozm zH+$UeZ)(T;Z;ggKC7`F-6K*vhQb^#IGDn`6O zQq#cxS}<@N(p~VGJrGiT&h-;cqLmm+4ZFDSqOt&&UI16Zjd)bYhtE%BMs{m0F!8UA zciI2fxc`T|I~fsEhgmXc@Z!2O6*7NYgEh{quZ?!4w3)4J+0UM&1KhJK;FX zMDA%khw;Owu;(`1yiQjV3gBU4cxpgg) zMko?NzzndId%jNVP=+2Y;`ysaN+ElDBjI@z+iEYNwdoobPX3Jh#1KrV9t}z~4%rI9 z6S)_X>Ve@)W5uTjaA5vz9QfM$jRs7}VFM{&KdhN>1P2TrS!Rh~A-?9I1 zf%<>^Q1Un?f9^0uV$c!mffIDah-pbY|C`sD!O?LOpx3gsTUa8Nl2&m7w36f@R?10B zM@}(|ZF>*n)P*aY0Q}VpFGIJ}P|O4@l8ZDW z`wkq#wOb@q4YYilaop)~%!$D}z$?j6{(az>&Nl*$|5p234|g&~0KD;+cO4*<{M7UW zjEut?VYZH0%pT1~CWkG%cjF3q@UgdPEi+9-H3chG#^B$+gGX@frX!p9*N;AB;FTFT z%y=Z=YPQp7ciR6Ia2)Ss96OTR^#2{^BC(rjyvgHs-R8{O)7#*4ok(l0k z669S+QrOf-Zw(&7j8QF366Ow5l9;vxT66y{UCCo687T&KDG`3S?c5y(>A1;rIALdI zt02f0v_uNK_U&gzHC<1BjKokS(3053#pV5Q_F{(vjNiKP4aV`^jQ_&rYrfa~lPm&j z9pISQN(B0Sx$6KYiV1uiH*p##kO}-%4Jrc2V%^4VXul5bk9x)SF_7(DzV zX1o%xzHY;2W-T#Dns5L`S|0=u^NW(|$;wWNOY+sT@a5k9JpXA8%&BSM@rz0ZGoNHF zfdfaJIZ6Hd!O$^Ktv>A0O^I_WTZReKXL8X1tvO2If4*fGNEu+^k~JXtP?8}hwif+2 zb6zdads_Q{7dWOhN`Bsyb#4={zi$G^-(~%gbUnRFm~l;Qq;*8V zG4&G_AJAH&v3TqK;ao&QeWcq29uOej$$BE@m0iXj##<(wXJD2*aHkAef=uiY@yRoh2EY?hJ#gpZKVBcF? z8Hi%zag@iO!t}o^$Kl4?%s8Dsa+#CB%h(8|Eht7Aa_S>U2m_E5AA}uwkUfrfHC)2s z=G$l*MC?^vn)9E^QtiiaCAO!WG+zs`eI}v zk1zn7QyxQU^p{vO`5=SYI~QNX>n{#>gF=B*RXIKXkuSHLlW;shL%=QA;W zkJbVE7u>{z;R~5bOj6xhaS+~rdfGRDtSyqyY12#`T!(PS9=k8;VU-J1( zTYXXeW-SiQzk*W@z|`L^MX~f4w~yDim_>Q~5v=}X56(7%eVQi^S?Vz8cIaEgHMBuu$KKKXS64iMwF{uaiKXyp6L>ASrD&_$4w#Vj=)Q;Sig zJjRUQBjnFK9XL4mHX1+Nis^5z#JqPmVC&>_IN1ocTG?5D5hLH3$Att0`ctwuB|nIk zv0K^CxV8QohQBhC<-^JQI30Ny<>9B%Fmx++)?CDvx|8_${aMcV5lCfJAT18UsMl(l z@xQ0xGR6;aJgC{vmo-vnL}a7?V*3Df4{(zV;XXOPtWrfqvMziuH~7O%wF3l3;Z z=Xcls{eAYoX>%Lds)0beyNiDuz>uVuV_FJSP(@5vI=jwZy3U|4F?hRxV`gM0Nfx(^ zf#aCrcm)&SPVtMGEymV8d+x+9GZ)l2=buc)i{$g1WZM)v`{?!lu#pVf6L9auvPIZ+ z;5bfQyo&wDPU6XDU*!0eKw~SDfnr)%wrcg);@2&mv7y5hjw*CcP`pINO$j7YIaaNp z=b=4(&!HnwY%0(sz}&Hbq4O&w*<27t1$NZVxRT(4v7LJkayz@?U+Nz!=!~ifE^x>79%$flS?u`I0nVk5Vg z;+bWuHbT-%9D5d0o1wjz;*V4%RJ+_^wwsQ+jdSK(SigDGH?EJXMKII-xL8CrOJR{r zx{>ArM8T|JP_tNa<2B`f;^g}lE|19I)aU_NG~!j*b`e5W6+(7D#FCEX55?hklRx63 zM=Auh%D4cw%X^eQkM|Lil!8cUqLd}GHS+uJ^gpJ8HL^dDk{LW4mhnLkA6DopmKY)Y zCaED}r_fuG@rQyQXtX3OGUIqtlhG#@$0T1Kd(6(x+I)TnlKo*Qm3TZTVW=w)OLY~C z^aP%dD+nhu@Dvq8N~Z9TAYyD>!&*Itt&#h2B<8{{pBJkX33GA@%qp=lyU4`MA_LRY zDjF?=nahLA%2BF|7#+F~ZNY!X^lTJqGmdu)euk6Yd$Gb+@l-N{gppwezn=ikOcl!2 z4Bkjr;OUY9NZF-$P!jQ4;r*B%@uMad$M38n2xb{Pxk`w38qWt+Ozsv#OZFj*vC8oO zv-h5FR#fY@`0qcsAAa|od+t6v7?9N6E9bnbR;|1`N1_RH!~hDqw+Tg%AcCMoQ4tUX z8oFt^>27k4l0%b0qLM*`ai8&4uLeEtbGCTD+D|!@d)gRQj`Xxc+Z%FM$L~V)r-IRHBm-% zCc40pEQOnDD_S?$l2wuv*2jDgHoE_em5KusQ-vrC#1PQZc+=Pk`;C9WUbhJ~(FC4R zis6X%ge_hLZzKhGg7yH#c+FtOTEjiq>C3~SXaIe}g@`E+;&F2XQ#yCVHlGa}JXQ>e z#u3S{LatT_ccu*HL=mXy($j0lC%(?u<>`#m=1!QADT1n%KnoY(5qki$40$+X%f&)9 zgcl0SH_gQ#%D(9e>C74BRd zb9BU{bO^qbCQ2&=j;C02am@J_)I~jbRMcz=C86Lo)1T4e|2J$5wa3(gI37?VQg1_| zD?IW0#nzz2W5sOeJve9l7c_;OcsgEyR3L~hAqPIS(!;qEmPA5$GtTGAcFiD|C_zjs zL|G_<$1Fk2wmEUoZbDrmiHB6O~dM~RZ7DHS`dzO0(RO*iSmiQ z57F&>T0vt1$*qjz+nbaAv1FG!-+iZ@vT*^fRd*Q2zW|T_nSrza?!e)l`zMn_&v5*E z3&B5g|NndU{Ez-!Ui`Ee0_lkO@ER0?I3D=?L8QDHVF2u|`3ZaKE@H#%7R>r^DW(rv zh8Y8vW7d01F?IMnyw>LpxP4Xx!#4OWAq?p~3HugYLQCxxk$0Z<;RXzPaXLo4G80QC z9zaXY4eY8ri51l=kjf-r){HPj^I(hHg`Mj+`myAzRoJ`W1okZa9^2|qV|)E6>{)OI zd*@un;^BvIaNZ4Uo%StK z2)f~SrLkt(F&vo#ESXc;8vrx!^Lk*Pp?dp=oZ)%2wM#Hu#-keDLCE9H_s91JyraSM>$V7`g<5o}Pq{ z`^-Z9sNL8x>nH52JCC&scA>}pPa-$$fgx&zGiis%<;So-W3j*HERI$GgadOfqI$>* zjCg(uM)#SCWfRG!z5+@wH`gpjGC!^Z-;#U~fM%)0y3fDEp81!sx@I2&jwIZEKTN6# z#+VUqmx>Qx{0c4efZg+e@gL5DgGm7qK3k7|{p2&)Gye)!*Y1@h0Wyw_iV4Q34IXy{ z!(W_?mYVD0u{~i(ZEO3Re^dLnG@O<8fAE2)VD!;r*a?T~fX^Moz}_FCrS^O5`-Xms z=P+ydQVe zsY91x#?bYc^7dLBny*W{Z~FQSJ|EP8uisyW+3&Bxl%cgUe*GZ_ynYjcHU%F%KT*an z<Bm2!l^SBlqr~|gmJ0{~dlT1kaTUFCn zNQ0~gN>2yL{?$@wT)I*MNc89>kjUeS=lki*$Cw^)(%Elh8y86MZs-1^68O7#4cK?& zxG)q34jGP7pN_+dHS2KU>a{kh<*}2eNf6LSHlX;s=}b%lGz{7_FIkDpKk7j~dfqNy zvq5}f7q3`@`bEp|;m4EO7&m;O8a{eFF16N`a*Yd`m*cfJ1`2RI>XXUXzWY!cD0!7q z)IVb!lVOm-7gJ|Ru#Ny(P5n~c&*?t|b{Oa>60cbXG3jxZtv_FwS$|3#E?T-u5B%Mr z=imjbTE7{Kmaj(B(v_%dT7nUuOx9oW80d**pr-OMa@=@nXM)F<80cED4wJt81`{TK zjh%ZBO8|?&_t{IAQQGxE@hlFCRCR0pr+w2_|5~ZP4jkVEjhV>T3e-o7#4As)ds^7# zpHBW97q4G$Ye&j?kDELdQ@);q!^h8n>}dvFzdL;f#Z?c8$D1%cv;t^}LYz2#PX2%K z>NTnB%5|GCAmO=2YVbKZ< z9QrZFel`ue_vwiQdf*$FH^1rkJ^!XO4U$fF10C*!}AK1N^O(`T<> z#_aicXW%e=I^jz!S-D=u27%)X*ML5+zbAor2CmtE$v7S|;-*CLfy2kqykr%cm#jt; zkEN^8D90)pn~h7?qO!*mtrE|adCteE@sp$s2Aj`c`ANph#4o1di)k~lyXBCyEkSJp z@Ri*kLnvAy(vAdIy-Hky+ItQi5zv_5AqT-r9@l|Q-)_fG*Kgp*pRVDXxdgtcGC+^L zBu0$>R6m~{$62ZBHsDy7(&W5y{LERY$I+8Ni1&9eLZ((ex^(Y_x`riaY+jD0Ma$6G zycA7~mbWpEPn|iB`o<+#vV4tl<}tn$G?62F@b|JogBa1Cuf1WhU4hb zy*J#61l;MQ_|R(pE*Sp8R2- zkzeqz)SLTV%3_nT*+hGXIajdnn`_wg)hT@TW<3TxJ^@3XpMrTGt;hE2AF#XbJa#pm z!{EP820`91<-XNY^eB1>QwJ@^*B>m$jA6?$bJz-eHDEEwPNkB~#%ZT8?^hBTq?%fm$Kn*l_6U5fgEzY4|P$1!v83hbGC1ub>guzK1t%=u_NW(-+@ZwB)E z^%(ThXL|W$jo(iBLmSbr?{H`_U0$0Wx6fi$#-m zW6y%i*i>@@D{5O%Ui^d(AX0K(EkmK_F>L(&xB&C(<{d*K&=aa!25&qAS2B%gs2fJU zG*dwL-SaME;ztVvI1Z`={qM)XXFtd8`PZ;>!6ke-tWj7yoLk(&5~A06g|KDUe|;SL z=Kzal?SYF77)s-YD=_G#$!Mv*isRM5!D-hqW55dZ>oo#h;?HBibK|ji;vPv#*gf?s zT4w)*DI*CSm&o-&wFLRz$FTN`lh`x=GS<}`Mm%)CF5jBUhclIg8tRHseXFr&-jCQ* zcNOD5s=dwff0XjS9N8%S){~*Vda#A*i(H8Od8nLbROTqW( z;gFr!SMwuwE;xs=L#q)C)8n~VJgC)hB?k7Lh@B1Ru&3q{HqSbVaeteKcY2M-r!UXL zyg{3=>zhkB*!UAR)E&dqkGv@?9j>PWJO}Q_=+|apSKS3Labx`O8d1ccq`Z%EwqzVD zl{(|N6*#8MEtMxY_IYp~ru08;1vuXmbmabvpFEy;{uTXPUQb&Q0!Jd_%=tSdiFbPr zeg}faoV&S?$tH4f-Q2WgE3RGt30JRO#q65;-#P!i*Sdb*?*8X9NxTTt@0S2^q7Vna zJ0ts_3s(U(UIah=!wHa386=`7x`mRI>F557795m`X0V&g-Ga9L)5RM<;lj0RlKeJ7 z*1uWU)pA7IgzMkwbC*!rqnEH&gIvF&l!1<8>z;#cd)Br4@X^>QlK2%#@j24cA|nZATA~0e)@+jf7R%%QciEbaXk4}$%`4Vn!J-x5{v)gO z^V`Z_)mr{qjGn;qS%2>T6Bzf!)V96fg=>`aUW-XnW?;hS)3CGUkhI?w?msW<`_C+J z%s$JyA3bqa&iUaY6;G&;qDzc3R>r4EJJD zh6z$``GptcUN2p{E(v`vzxtMVI;)8;@Ti6GN|ILrRF+M@?PxjDS_<7gxfoaWNBlI# zkISFNv2X0dnEcKY2t}gs$135|y2Bl&VgUh!+XRkVC8-Gl`=|uPc`UB`7|Xs)D-`Bq zgi`G)0c5TLWY?F0?*sHwWzyo)7C$DnH(+TvfO#<=dTSvBqH$COe0amq9I-Z6H-$PRa7dgu1ol-CZYSVwm!ERDJ_p)i3r z5=nf~p#x5v%~%vkMFAhSTV(#_@0@jK|zT zJmZR^QZ0r{D-t<$CQK9rV|d6HL^rn&U40=u<8a}x?e4`Ys};vBR?KwR@rc`w9$p6? z4!Ds~Ba)n05{~0pivzW83wFD6F+7@vC(<3=+!+k-WX578D2(I&Aw||POQsy=Od;Iy zQc-R3vz@g<^v`RL4X%5!LUmwbI)w_x;vxwQ%432-2W%lmD@e0_}WIiY0 z%oMG_x<%p|=c*5nuH|BQ2RG$}(;K5Wk_@aey7Zk!= zSb|`x2>lFsSZlZs8+{g3r!_nk(cshy;f(i$ruIO&H-Q&z7OeCb@x8UZczcJVx@ACD zw~A?&T%5Gug$B)yN2%aL^_ye@uN(h@1MWXzO)wAR3Nt8ld*MzO!>n-(RwAerqR<<} z7tT&NYW)N1G?y^h%l!(TaJewcVZd?o-Pmo<#T=UnALiVR0l9y|z>e)P^qyP{ZPyMX z+y4QJJNyBMbMMBIh#&nlO}v+tNI5>pwc(^W2OENRj4V!|FzkgZoq{J-234yP>y|9( zORx-?qf6TM^uHFz&YqXO?$^$_^S3*o@ITTA?f}QX0E=&Jw0~ycOeT^4#=zm7`@aJm z-#XU+@t<2+_^}j}OccSwAj~ln42lW1uv?_6ReJ<4RJ?)BU!Ors%_SUbx`usouZoB8 zv01=(Gl4y|=P>!>x$p;T*WUaah7++?rrh^8{e_y3<%bGsxD~dr4L)xGUyiKB-i6n(qgn^p4%7hqX5GM{8enJjMQmHV z51)+r5N_21n_>ZhJIYW8JP8cx_a(MAp2D|Hhw<@;pTQpD_v?&fe!nYZ5m{>ii-Q3; zf*R~fNW7vQ9yc_140A^{i=6iU>KoYo%}+R1M}`S-unyS00NBuc6z>li4vWtRi$4Tw z#Dm;m9-MwJg0>`T#;nA?xmSc)L1s$JoEy?^JLg=+j=Hm0w|Ey`eyxwxnf}z)um?83 zimTKIk(6+v_f4Tip$e8#br4{f+o-?cbue zzv6^d@xdDPNb-Th>j2rj)gv3RzxpEf&%TU9H9yL6VD1&{ulWgk7hJ=pMMp5|lX0+y za$(cj!5qznDe6RStM|0u??ue1;+3uguw&*$?4EN8Ej8zGX#RCjqVeb~;KW?ut3k66 z_66bh2gMKi@uF9;ciu(pul)(TYp!9}+#A?SKfu{!zW^=Oz+OJf*}%bXfbH`xmVTZK8HSGFr?LS{{ z|4%=h38(6Sm5KogvJUjXyAosIq!0~@mbC>e37DL|a04&(A0R%Y^m1abj?6b63|!HV z@14OT1X#%e%@X*$t_Lw0Jj({%4jes>Oldb^%8@O{05?G%eos*RkqgoPZ|@3la`B4J z5azY~4jyC=ENEISpqCyvrPN>o-|pTgfjI3?qT-p7=4BO_(Q_8r17%C_OTD>tvvjYhDRwFa<4BrH>ts1to>G8+S-uv;Cd*mwwxt48S(6U%XnbX={IupZL$c zZ7;Wi^z1u4vi&Ce$?xs`%3Cu2$k>Wzbcy-3oAvTJMtCmSxBTS!{2unnk3Jp;@s5m? zLQPglKPT&mOsM?QO3?rL%(*Lixi^4=M^2(<{z6pGoe#2qc@TJDfRcfCCQN)ddR$u_ z8y2l>^BB(Vi*$fIjpJwZ3k+-@I_v)CzDmO?Z?Ctg2q{YWdhJ-uCLjox8JQ}mCuju zM;UOI+puJf9&ncNr%RnOQ2txSpD_Ppl#r+VzwT=)00=#tl>~Y`_&B~lM^Rb*-m7by zgwaMRRr*hA(F!QRD)@t?B12r@eq30U`)c&$sI@bWf^6M=^*@3@fl_x8kq$0uU%oU7s^OV;h7>Kg(?@1OUhBs1=;J&SLfPM~JS8k8g- zMJjYZlA-%WQux`@_dto?1M`_!b_KiVUc{jV*Kv5B&T!@T@?KhI|0r_9I~QEQ=UvDM8x*5LSP65Zf30fK~GkAQtQt{t>G`JA)$&ft|G%G3g^+F~LtS&`1eV zfrqeS+Cgk<_yMcx_8}JTF8zZZ$F5`wYN!e$`pv}ly7So9a1Ikbs{LK%PyA@UcuTud zNm!y0_>+Z*Cb~$upMSgn%j@@G-O@u?yYMg;Puq>J-(QX|Ut5F&3$DpNc+`7eOTs{a zwi~GeQNK`wRhT|z33e_xFA0=XirF*!GWN~B2KF1-|2Lh%m=Uwu0ATJ9n8c9?b;roR zO~&2nM9xnM?k}hwK)V8+b*#j6#v_xaIOk;N zt1}mV#7q6(7O;>Y8JR(xzalA0Uo-iId$=_6OS__R$!eTAcU3@R0;LC!9!G6m1LjoE zlezo)4bI;UzjOYhRPgWF|Ev(tVvj=5HU%|Vj6)}M#xcF-={e86C9O~h9CM$?dD0Wf zhmST7BGJwLPdQv60*Fk0OSJBvuD1ceXS3^H>;9AL>WI-3q^*Q;O`C#5p{Sy~J?O)> z{pnEQFk9~)aAyd*to|)7!UWgZ7`CHrb%FfyR zw(V(aJFazqZ~rOmH?CilIHg>BG+*}DY^xtGTt`S$F|z9~=eZ(`ojhId&o=N!3o2`yQG`+Gfxmcx3oCF`_m{kQN2c`aGe+$&WG zNJ6U^%DqcJW1>wkCQS7Q9(f#l4(bXdyngkDEpojk^JOXMZA&ExN?mJeIReqH@Mz^I zR+4z!nS0HlJ0ddvBjN7QV%>iiaLoCHHV{mnkT#4L!^63r z(&CZ~Q3!?mn^xdhiRkAOH1@>G;7D{qFj9fOMh_Nr&cTY157SanJQfNG@Ymhv#T$nE zL`JzOYR90E7Y{i6_{w0#Ns|RLIA_N4k(1vIj#w8wX7^!2$9u6z^b{f%~`)ACz{RQ(Zf59w!2MqPQ@R$-sX}k>SNC_Uc=V83( zFIXQmqQzvy&QAAWv)_ad(sB5brEtW%K~;MosP;fCK;~?Ftg-(8IAr+)PFOnOqUAoE zwBCjKm=pa9BIuEhp=&sdF-8MgoGyHs?1He;1I}1?u~K5&JneL0j=dvVTz6x9Mu8)i zhA~qDPl^c`70{G!P?W9+sO6F@K>M3VoOTQ|b;96w_h4wJT#W404&yug0qYFyams4K zB1^XxWRxE)*Ni` zyD>VMK^Lm$sU`5G$u{=Fp5*&xH}OIqZMhd){T^R^6xn4_+L0_ zX^-|Jv{l?pgl}+~=3q-ue9wnEcB=a0fX4PX^x|;P{SIb#}mH2g)6( z>O$lgJ?Log{HBE~+m8S5f9hjGd=9k;JTWgU8i8Z7SDkQ%U2u7v@H--S`hnLm`-{a` zR=*Xin_94b(GhHFI)d#>j$y;%T^RexWZ3);*fsj#+TjU@MH2YqArr8weit@0euwRg zPh)faVQg(WiiYWH@xt@{VOPj(_O z6WG*z5bGE1#g^tn*xq;)8<*|G=<#C(yr#sk&l5xAjE&g1bRQZUSHc|#NSVg46-Lz| zvd9Da55|VYd$4ZFK8zVP0giwfrmzVH)ed9W0vBa_JrRt0=L@W#zaLu{9KoK(quASY z2-_AMz_K~pG2opKL3v}dO3+ome|p7w{VE3jZ45RpX~FuXJ1}D8C(=G<)ge;VJcp9j z0@f-H0jg~wvw)uTLAFIaaQNH^dr}zv_EfB|If$K2r?8{p2=+7|#>V;ku%vniUV8a0 zSQIxLst4|%7dEm+qb67*HaG$v1UxZ({N5K>Ti1fk3y)!E^Y_@Y;5fE59YMp_%h3CU zmtc!nU{MHo+l4i52|3~QM$kBWHP$cNiA7DT;PUx}C2ppVal`_r*N=hyKf}p; z3r3Hgqyxuflq**K`dJ&Xe(7#3YF+`i-zRKbQ;4i`J6t{=-t9ja8yD@x`o;VIFSNe} zmaths?L5r^d(;Jo-vPywz$b4{#ip8r*tPH^b}l%A9d!q=tLZov&D?^w-W(27$N^i# z0Bh(z*fldud`1x`%wa3+0W;igt4MYCdwM8pr?1A!#_d?Q_yE>5AHmk8$FQMkH$EOQ z5jMXUwtyD_HHt@jJcpf&4`Wk(3%*@+5SyA0VDq9w*u3y&|J|^-1=|)M!OrIIuzB%b z8NXJaU7iK~zU@9A)Nm3X4*UeG8+Ob1-P(K-n-(0xmc}Dk`1LwG|Lm*!_+@_zyQJ-J zr~Nu-b1Ju+SxWZJa-umtd2!MFt381RfFDK|yQ zI|9c9PxCQk2K8}W z&4BlZWB;KOxN!LfF8z2jh{{0D{DlOk%B4m1{$-Tw*s#W)atBbz5Wc~X6ql+C@Wkb zQj6~m9)Xqv-%0(+wq?DuLBx3kPh+LR`XzwMKpa^g!DtHA3z~7};xz%PSbqT?ZvZWa zPm07M<)1@jH>8K}tu%&BX__m_0P z=q%%jLBt~`&tlN>#F6h#3($4u+<3C)*%5~c?dCE)yj>(?l_x#^@ z{w7~+>OtFEk}R@8X!f034}Ry9&%Y)t8A@flLrIA5WF(F1x@IXq%RI}1$omhU#GoM~ zb!JnH!4a}vScf8z^+W(zJeVU%37+6<1km#Fa{ssG?@aW__WA7hWyehR zfBZj_L+TcE4E9EENfOA#<3P z!Hnt!5-{g^JhEfp@X2$SGP6cN8s5+CjKA!AfG~{+esFAx$1&SSpF09fq=*$@Ufm)A z`m>VN_07xCkKL2|kH66OW>0!Bwo9eGu>Wo{W{>9KepIv)I~j z8XIblVPnI081=z)#Nu59l%(G(uL{Ud*d1GX+Dpw^G{>z zf>UUou?>A*94y|%l*9`u6}m@ul%Q=DlA#Cj&a>mO{LB41%d`G6cFsG8{S8;Kv+gW5 z*B!^yiH&&RkryR!6;uG8Kw-ahnc#S=JD%+E7AWDodHyMEZ90Q*n@(cW!sGb1{*-`` zTN+PcL*r2azRBqPcF{>p7+(WVr~s7Orbn_Tk`}*PE!YK#!2NjRg-tMypfupjG} z9>bdEL)fzTBsMi1!?=&;faQ~gbvxy6Z90vKqiYce>EMNuptoIx0KQ`f&%xRShp@Tv zIJP(afGu^Wv2Ee^STtiN`t}|o&s)4vwGsr#5{nifs&vDmZ+1#z;F5(q5uu!5gaKlL z0U2Sde$el8Y-v7)^-V|d>F7B!_XZ=y0$lvH&nT>4d;;s19L2cNv!N2y7iK2=Rhh72 zqv3KanYRO*7oNhR+1n8dRSDZUl&C^T>4I*>FJt!TRoGK|7JKIZfZYvL9Xf-J3r?YW z{3g8h%tWkTcnlkt9>T{XrwJG;_16kp>%RjV7aaxbPcLZ#ssxq^d@JD!410YFHZ>l{ z#zo&@;zx6Dv;3n{{+N$uK#5dH`*_kBxRWVqCn{+8LWNMYa&+zX9J-bEM!xq^l-i%c z7q2yA@7ybrP%&=sTqs%vJh2Syu{a2Dhht@kD?Kn~z-+9WdqhBdGM9JP(HHzImd)OS zK?5fsTu>s(7W!BwAXI{=(iOwsnu5*E$FZ^b7(V&v8=2d6#xd7d_I1iba?H~kxe&ex zRaSIwV}f5)d-4hfnG?^8bi@AYxdk*5`Px9&VQx9YyRWuacEICiqDJIf?4}g*bERiX`B1{=VG0 z58QX;GzJbCEqf(l>vGQU~V~bQr6*Y!W-qn`90jfuGzE=&%NAF_IOf1 z0Z0q+Lf(;Hwi>+&Sb}iwafZx4v*=BtVQSO=61MC+{S^#8v=MES&9QzKR z!1*gTz&$jt%O2c+{GRgpJ@t#%$ew|$W9~o01RW!3%&Kj~50`J;+<&%~+j96s>;97< zbFmKI%i2{WZF=;?dAaxRFWk`kH4{RAw*G$Iw#L^ruhy@>?e&igVqUMjUhOW0bfibD9Z!e|zUW960)& zTu-+0*-Jm-$>&~{|I;tl7cPN6&V&<=;X+AX@+7)}AZMu(LT|^Puu=XS*Wm>UgI8{Shq= z>CC|ldk#k3_b(WG@BhF?Zw~gk%$S#F$6NO~(BJI9GY%sXK@H(l4?LS|!)AvGC*4MD z?|e7D^;j@J7Dn$_9L`h;ER;}=bU`p$2~|zuai0^@l3pxStk@8?W1q)>BaY74;4x!? z7C?0>jGD9`>pdND)YB2OQyL!1Z%rzQR-rr;MqkH$SP?ekl(jQ<+fAsCYUrCNf29Ui;LkS_~Ius9gSgu)WMTvmpM zTtW2Bb77v_fz5s+hA{C)>57LODU9l1KvTqvxd}U7iul1b%$6yFgWkRbKGiZPN*828 zl^D?8h#i*BSQ&O;Tz(8)eE}4PllaJJz;R1Qtn*s&aWal_uL^&p3`|rDMe^ZJRKgi6 zM-R6XA6fp0E$V&P;rTO0WMVSclR4dJ>wqmGI~qa(yy}ZVRVs9qt!M`BLwv@CPt0m5)c=3gVGs1QNwiVhNN7!g$GS!(xX4$ISO&W-0?MREk)n9QQlJm}bty zF-LpU!~%FInUOt6AeO>Aojc-`-H1&tAI52=c)*`Pn3DF%d;}v2Xpu5h1j;bFb1n`$ z?nQmdho=)UbaAVA&J@N1hZkoo9kJJ*izSK=_304p26pU#?_GCeslOvWD2bsY7(j$;m0Bw87}m-~F@%ZxdW@i0_5<1Y zx+FDcJ=6Kw3K-K1`3`V=2MEsgliU5wz6W#vdq=AJ7a;N-;P?)3oE=!%fpP~p{_hSP z>x&Ni0fXhCL_~{U^j9`&HOi<4(KC5|} z6~+jGK&voYJpm^kee^NB-tSfP@B1RkN($lf&_~$~bHWQ#&=!1hO9{dqa=`C(2ymRpEySGnSAtB~J@c+()O*uFuU>P+081nX&aeT#fCcVQF6?Rt zSk*jO6g%u87hFMl`nm+qNlkfHz=+Gh~tbBb%21 zr$sfxtmVNL$$>-3fn71eqFP|ma$$^il5@Pi5L^L*scaVmj9MOYV|mDl8DLf|;xFt9 zxaAyI$O>0DPk?WuLP=()w7WB8htKCl(BsA94?cjmUhRzmuRo6mx>X?%&|nDokrNFd zSMdp>nv73^!p?vfUatb5H!RF$XV5Lrkf5nqCBRI%YP*2RY!_Rk6YSbuFl#yTTyr%C z44MsAO3L^Zczg;TdaxH>@Ao49+V5#}tty7kAAmXNf>Cq895lg8f8VePM$I6SecnJ>aT(km8|)!7>~RZ>A)^i)x3;TYo3>T9&(CQwK>`Way{ENXoEw^lY@V=Dg>$dyzTJ&?C|*=a0l&h zGjT!M-;UqZ{{Dcpe;)5s&5`@f(cH+5TBRMB%;5GZ@Oh$mu*XyQ+beJ4t$u$+kBS~T z13lz}IqrlZm@BM%2R)9%ZV157u@^B2L!L71mQVmD#S25ku8+Tn8yshbr~!HOH;#Cb z7jemWq}R94=Z06>#|~%6D&E7+kbdvdo=P5E;T(8FIdF#z(!P9Nc~O&m?=Q#iBab{P zs#R7_vJwjQ}X&FQ|U1Qmbnkn{KgO>eRYW${=(fye?>_Iwm? z4;mq54;(fcJs){S7%luC!ENzUOX!*UT%UifJpPUU(?M4q;L*?3>!$~9^}63~{q=j# zCKBnAiNeAZnZJCz*zYZj8uulJj+%(>4?iRIWN?`G;fu5~ep)@ve%W4Z4+f92%xGaa z%Dw)w{#=iA=F_cZ^XERV55UN=Utsv?&+y>my`@}fyEqw~l$d1Dt_)-x6F4SYm_X+b z7cNRN1wE4gHefJ5n>q`Bdv_@E%X-4EWF#;yK~MV8#mR=_%WeTQ_9-PLp3XfVeg>n) zO~E^ZM?p)MOHfda6(Sr-Bg#OhnlB!ExBL8+#LZ`SThFhxf9QdGwqN$p`xyH#s=JOjl9``@|bgP$e0aONavmTMH0eO84<%>6PKuto8(@$5IfF`p+i=|QD&@8~i zPuGEO=G6=1jN^mlGoV1vzqY^vL9y1cE$#b@ahC0$?0bK&|NXK*-uk=VxAnc06C+;K z001BWNkln5Nn*_o684y)y@ zOic#+8T2ccdJtG8B+1TbYipLpbTy!q;A{O!e$(WB%g@%d$P78$M#?hA<6 z3JgmEPPi*nrBbdZlbiRa-$%YMT;stWhz7g=tbO!xSE#p>(pB0@ABTFM=JO4!6;RYt zku25171GYhKzC#U-Gs$MATCJeEfc_$VstNi0)3x(2ZP@H7_ammh^`gSA{OZeCBy`c zeDQW=ZPw@8_k$$m|cl-|CHKzGD~UB&A+5$q!6M=05P8|6o$ zWr&2jK#f*P`AmM0ejH7<0*jOYFNJt)`yWHG^(o9Av`(bH$^QOi;2e48tUDReJZ|1c zSCI;T{-J?*|CP@$xZme^yvJV=)4ISFOTbpB!k&mqKho!qVrh5w-*})0;=%6HPub7N z80YsAEao$oJw;4s1@iwg?j;zjGXgp8`20BU7nk?MxJlD6c-R<`&?PfBlB@tdopq0M zCY|VOv5)Ivv{1k?*^+G?4_~bK8`f8T!Be{RA=<$dEtUk(0 z$Fsro?0Wm_>vTApmi3xeC_D&kO47Kjdz{m3KwH8n0esrTy!XL}VrM~ic=kPqtYbA%DU$Ep3sGr_ zswcbl9d1)4%3eQk2>s|O=dHgdab6awcHic35y>ibK9daso?8a1WyN5Dc6&(6gl)lZx}CF z>}a&*;)?lhoVDC1GQ8XD_h7U6f8v1ek2vJ|0}fj5MvJKvcG){%X`nM+FDigZEy3fS z5Ehs^;Ha&Wc-F4*dhl7M6g@o&#G_U4alMR{z@5?=x&^)n2E+>Rsn3mh-aKp!7_h}> zz}Jc!@22B;E0e;9F%`>g9dO=y7v^eVbT8@ww^k)g*tn9w!#*2kI6C6AvlG7cI`LJi z1doK7uv-CZIv=)t1@>49flxWhLrL^W$I!dD1dnJbjLp3dhh6P3BN0S-G>T%s1<$+h z!dGfXtX4W>hOaZ;H{6ZI0UP#oz6(bU?QqC=AC8*a;XCU+IAXg8d(D4Bi}ByF&3ZSc zXbwD67(-l95#^XwijfQz;*~%Ib7Kmc)gTrHyjZI_u{P8ZyFK^eu)PDedz@Gr($Ex+ zqCS?uxROG=RFuSn9w(kN889bk!G>UG3`#@A6gD-#?@eH2ys?gL7yy-N&@erz%FkZ|xW3k(e{q}n?wV)7c zxE%3F1s--Qm~J=Xkf*aqi}y$r!6V+{g?PVn9*&#xu+}j zM+`};s0=8=qz)-nh=jW$tW`=o z>+=QU95?4;&L@m-=jDBx96b zRf&W!9jy^N%wY$tDnU~E?pg)ZB=AT^q)7lsYwQ0AF5Beq1e6In638-Yl$_3kUCj}e zuifv0ogTk_GAS*v2F)-kHjtso@?2pXYyq;2{Q?5JLmu&>4LXVuG#4T6s6@h4g+%AB zc%g70)=W8y9rG?=$HKGt>&t_M(QA(Az$L*&N7w?JYJ^$K6+qh}9>^}Z11

c`szh z5>&H?9k3__Ypt-VMz}&|_ySgV0~Yz&rVxDb(zwI}+ zzhBzlOoo1>Gfb)t2GuX%t}&|j7kS1Z4@^E6Yyl<+7^F|x12*xrW`A;M^yjw27IMoo zB#WDXKI_jmAd{XTtyyux6yaFobL7}0yE|9fJ1@ed4ziVXkX-T6^c*rfz; z0n}ufw6*@iUIa1?>BYwTrc@;BDA)a>z4XAG9-!92hnwsq0o~i$Kf6ZXEH@kEW7|>w zkM*FGCjTCi7fLc~SuXnu!NNj$DL!=cw8&$gy?6zYL`J-_8Enu2dtH)~?a%WW=w?8R zY!S+4krknlaitc*A5KF{mdkUEq$}VIBqX7L!9fPpnXyiW$nSandjH9`Kl>^JxWA$Q zwFUhCW4jUjBXeG7u(bO73LwconjH`9x4cIJL@bxz!@ig8^X&gR5UYP?$Ad1b&hzva zdwneZY5=+wNMJq4$mLiLlci4CR5EAC=3*I?e2jCv6pBP>BuS8i0q-;dYFq$_XM6X- zkz=QHsZ#($J{l{*N*y?69hd;XKxQ@&uD7i&lg^*%)+X@87#e7KJSa|rPoCVX>^ti&&j@nWf^^X`PrBG?AXW#y$NElzjB__pC9Kk z9^uycOc=+%XZ*GGpKKj&z9VM)4;k!1ErYLS&TC^mQ>y#W@$cbDBw9wKX#EFy10 zXHNJCsHQ06S%yHENgZUz7sH?6xaIS%fInFRTQUwCW%|D2#h+Ls6QZgWI3+Wa&n(+F3WfE;^Y}N`oxrl9R*Li|K{?(7 z>r2o$Ac1qee`JAUg2x1y$(Uh3B3R7wdC#{~J^^ilyDXdE>uv=u`Mm^6bw+7{w7(Xu zLXga4g|et+@QYdm19EKp0s%a|l#8XM0Q)t+gZD0q0PG)3m?@L?A#0Uwz_BLM#~ep7 z`b`pyE@>@W2d6kku>IMt1dh4B>GMm8v@^kP*2~NJJy8H(Dj%MB8s1n2fkY8P2_0mk z#3U09RL=K_VgccKf8hw9k4aPa5NLVkwr+kj* zh*l+iQpPrc6qR6d6;j~`k?=f#0`H?J^FE7wawOJFIfAWozsJtTvv{-5XlZ}8 zyWTh0fAnM=evhhFAmA@SBwU4<+6`)`62Xw(C)rN^q~4G8IjQw|D3t>06a44%kv>3H z8lNrID7fb!(^%xIIj`w|)8~KAn`9j8V30nS70R5=xtj@BTzmEX78B+uRh?-C=8DB< zlluZ+I4vME=M8$j1|xB3$qb(D{W1<8J=vz%G4!Kxt$QH$uYY|0)8|Ca18j@mv3|Dg zpR(hebFV({>Yx0bb<38+Pkk)tZO(m3D{w61ME8l;0a5N%^aMM7AE@sSx!2&l#c@Z_ zF`S^{0s*H|*^B6FAT0}0Z7k^QzE$=E7u9j?FnnCE(`@7cL#>3?It@=Nlf-q-)*`p@Ti^U&`}?+4sJ z{o49R^$)Ipth-O){G`9HQ1NBhsPSzqVBpkb~1SjuDbJ*N9`tp+KD|JJtnuYTs7!M%>`cUs#}-dAK_#I;X9 zgc;rTbCIx;JzNJO1t{@H@k!2I*l+w-%ufW+tDp*jcolrnD%@{X@nL5x8Y5xMk9yHx zaib*Q!B|@kj@mn8YAOwXv<&9t15lzp@u(|=;l}$=ANONcG>8|1VcDy@;+615E1)Vx zs8Z5+FRJ02m>)A@3Z^F0Xh@{7BH+iipb=ZVcVn4q#++0{Sj$r~8lEYwfHzZ#D!UV3 z+H$bd)(NNWd03#v&^Ofu-8~6(cPc0hDDa1ZaKejLReuiI%{T?h1P(A6iUBcYiL%cm6k=Ft@{Q zw+&wg!|35w;7(P-nk<4ZnL;p;FNv7Za6a-w86=c^6a+MUVr-8ij(ad89z~f}hWkB1 zylH5MIf?^IBW_G_cE(7{y;!C>vCV75QojXrqJC7zG|Z32Q4@=xAsNEAt~?yG=b$zk zz;pR2Bti*9BNd2H-6T{XOy1|i5%f_acqI_RJ3cE$I@@89uf6yNSBJdl@7M60KZ4#$ z6c0o*D2)}M$```(7BlAgtk~)`VOUHPuk-tZ0eoP*7xkJGbukZK2?e3@eN!tF>E?i1 z23M>L%*h^bM@sQ#PA>M_I$?Flj!%m-DDZ|5)e7-OZVnE+^U&foVt&MrC%j&G5+yLj zyTTQ{AGT;WXt7fCr%ZbLKVq*Z7mF1;`bSkH(naXua^O>I2LZu%sGTt(;75-j_gv+$ zC5mB7bP?dXufdLa?fy3oIsS;f-haoW5)EECiJC=F zwIV#=^kBNFJ&xM$K}}RauT&-cajFRvV?g`%IBmWUTl^l3PL!Z)Fb!Y46b>@g6HEdw zK!rb!$%Z`ff?lEq(JxhqvPcS)-~Q5V#R+ z;F$NKuMb=kZXW;D{B-C4w>Pi2*^qXBY`afc8(yKoK9hC zlCkOYQ)=3SyoevUk$?baS;nkQ&4pdb(^;@;7}iigfH7Od1xL&Q3jxNk1%{B$YIbS# zxOKwiu_NgAAnh(e)1+enbZ;^=SOWg>j zk&$UqT~dxo(HYY8wIu+``yil3Fpq!Z{|UB|B9fEs~BGK_g1nXLlo2@9J{VS=EP z(e}a^4#6A_!xEw|Z~$h-s{?wJ9VP=isVk>MR zouy0`Fx!s+vo-94UDaSEsIEBRR^4!iTyS{IaCyyeD}Jd5L1$qQw=%%Vvb8G&g$Wqa zM_LDt>2)liu|nx;$`cdBw!=#BIjHGvBJ6F0NJF!&tnrY}c2#p>(af+#9CDwQkOkI| zQ`n-mpaNSk(smC5+_r+0mY5U9h!aND4XfsaD@wrJ4GWpez94w5IUJDx8(W#|M)r@0 z4Q7$kb_=k|cGV9ur^)6v!9s83U_@BgJYTL?-NJe&$S+KCwspuS;G{i7P~Rr?r}Q-w z9LN~{?d@;T+rRZY*rw7S_!-I5kJfg##9T0G2AH(=utYk;9x+N^vxd#Eil?z#7~b^a zCexc9;{>|N49|``N6;ld%lHd8g$++;J^LsV0L-9q45?J_BtGw#YdeC5;Fq7PNZIGMK3Yh?H9-BZi(~-Y~&h zf`?=V5M0(}2L*6saFk5dTLX`+KkGr%TV7UfeQ~+sHUh-l^@ab28V_Ela37r1c`nLhM zzx!`^$p*;ZERqW{4C;oj%D=iJjGaqrDe6;`dB=9)RKRJAJC-kT&Sp+U@OBOvB( zLu0FL216SF!O#eTAd092QOux?qSA=D+hd=H*U!FS0CNU&O2g`8jVa=xD4`>qV#RdWWxq4s=H9HGH0coe^_k(&0_V zCj&tk;ISZchjWJg$N&X8%;_k8=#gLJr;q&(kN)cS`03Ao1Lh7U`$lp${s-`jU;hD% zr#vUMKaS%_`69=v_2Az#;5Y+sIeui+vAsufZc?YHIyz|g;N4d9{E0hyPC`&Yaz8>P zO3*TAF;}?&-}&Ib;s>Ak1Ma->pAfivWsPP2;z}H$(dh~}$hDFnq#vo2umbJs5aM_c z2q3sc0JPQZ3$R)Z+ahHxlKNzrN@@Y-x12>I>7(36&-K@V@FZNq@$+O#i0pZzxwwrmODG^8z z)^v5DE}CBQ0w+k?)6dX_E2P9$Oh(qO~BWkL*1K2 zNq`ca#~dF5=gp{tR>*(@GJgr&^@ZWgfFSl2$Hx=UxDSIY`3;s|qYSs-mmI$Yw0&m< z9qU<`H!r~FuX+IYee^Nhb=A-C&5!&V-~8CG@tsfpPUNP)|Ha?otG7Ia*_r1^y$;5> z;W)Zo1%Ekb+;9ngu%f{Y1i*vERr}QSpKIDMa7N~mfW$+a;4>98CL^9}4f`frzlQ6U z?gyINSI#R897uEh^86(8jtpZ0(o~+{|590Jg3CXC^k49kpZ^Yz{Ni8m$S;0_U;a)d zgeiH?eDS~h&A(xB@3Ysfzel$IWvLcM_fFjhIL{8_{I+`jVf)z+Ys=T)hyA1dGkU)OT>56>T7o99xdRvD!G-=g$mlYws$gE4d;6r_$pR-HZ~uS1{qHw__z&6t_-@f~nOm9fCJNgA^%d`w z0IMJT_))y~gCCQ;%}im#z{Q5s(cl0H7Fdw)n&CV==EjW}oH+p1i70FS&`*DXEC1md z$@#1-O9tX(Dmyy=>d7MV(JZP#U(|LSX9gW_24`1Kz&$Uis=F=gL0j8p-wBNCuKJfbT<0i{N-Y%r-roKb6Nn zmQKb))#GvXY>2QokEyvcP;BqTS+yAN+)=<698V;nUu@PfZPxMb%69y-FliIG#sZ4| zsj&PdoLQ;kZzoQ|tw9wxMJ-%pnv%P>5-g+Q??WZnD*$pFb+McT*pnnUJ)FTyD{Z`U z%*CCCg}cp3ywj`T*|Cqa{1E5a0hT)qERAtyt%|o#Y{T~(Is7QM4Oh2kFzqhjjJk)* zrY7*xu!Wh>l(8;2OK5iX!Z#eedB=7R!x!8p&a0MF1(U@Tz@>sx0K@6S zaZiywVx|?~J>w_gvEm81E=;iKosQ?061;QkF5GGdxH(Ah&fISNU2!`;lNk7H+s3Q> z7EbGTaYlO{f9Wsc%w!&C#cjN6auPq7n!;zp7%%9}qhrm&wf0I7TF8I^w*!uIi2+?< zA5p7?XICe1wKV8 zNo5=Eu(#sT)^WJ0lE=%dJXccyKuYd;Dv4vksHJFY+O;@iJw=F z$B&vP;=0y$T+k@NcKfI&OA<6bZME^7vWYk4Ch?*1lkxIO4Q;1~sm?5ly$&wR?Z6|2 zow&zp;SI?wW}0o(-8od_IXSa8DY?$~m+V^iKC<8XvhM#!x?XMGe{}2oQ?=i+eSQ-- zKETA{fioNU)sET(PB($$r$CP1)s5u0o% z7t97qwjrRfuwt2Wl{tc~vaoc?%B`CP)LY7$&A=i8v6O;l4qdW&h0$x(FxjY~=opee zH)l;DM;U2n5*31fl(`P77;hC(^G%c*6*OuM*u@UM{@F+Iotu7#AKvos_|~WX4L`UQ zxc`=4;hSInF}`@)SMl=8E=A2Gi_vILy{JSP1das(n>g0*Dj)`jUQ635g(Ul0dQm({_DTQd+>}6cj8H#3X2K*HU1!f}+Db(+c(yaNg}UvBN20r#p!W0)Td2 zq@Js-DOBv5)PVphWsRjgk#$ZPyJV)4ed`pZkJwKN99IOC<=@F_C1_o4*^(zZlQ)*S z*;P!KWsxtR^v02+Br(BmN=RFjqvrGKC|DNCEe92|F09(Bl?Uav*&h?e1PTmD@S0$* z;hd!b1tq&FwpJkHm~3mOF5{)(G1=`puF4ixcC&!oY>QhEu$~NE4N9ns%rnP((rtkG zn#s=Peai5zDyS`NdNRfNeFC6ln&%amHd_)baajFt<{|x0V3y2$yN+_p6t;K8&Vd&} zT-JTO$#R$jdJ+@%Zjq;EkV2WPVv9Mdn_wRl7&u_nr7lI0RHytn0pzk=1c`-(ZWT~B z^Ky}0PX=*bnA!xSSti?B7n_o?_jidDd5#Qt22I#4C!p5D0y+j z`H+x3MZj3ui*&wFzK|E4isD!lhJ>Vz4^X$ME*HE6X zc#%26D|7y-d?Oux%t59=C7BWpIs;v0%PF9w;IZyy1SOTJL9m<8r*m5`GQ%_%kT?>> z>Bbj<^>x?t-s;|Xkh&`WOg!@iO| zW(3GkN3}9lG>SMYe|_0m`2^!T;&i10k$uXsr7bXsfR5Ta^D+{s7bf|P>Dz0m}~f7|LuR{eIK|+z;JO2uK_+NDXv!*)lR^; zNhy1|PXPtBzk|I!6j)h>Y+|@jQI%gTKWCpZz`Vy!D5$Y_$XCm?cB9nJ8FA@Q+Mnev%EwvIw*igjB{a z*{NhA>w3<$o(y9?M}RL;JkY=!GF2&2UF&wm8O5>Sos5(kk&rgp++PL)mP#N@ zs30~$Qw3ag4PzhZ+Q|N(V~cD}O8s^PfF(0DV?!$tkWr|jjhOw%ASMPa&BC(hg~jUn%Q7eU z906M+%8rx0OyD(EKwaiCZGdHIPJ7n55qG3)$x+RANWT2sy@N zCO0Fp%G;>4y9x|<`Y_wm@*dI%6~L}_rcvw6ph^aAI1A5TK;V<*q)fS{tj(NfS&#w2 z{jNNpZ6OHW3i@!H$o4$)zUH%TV2q5mab}$%3rMJmMryO;u9iZHa zP;V=foPfXMC~(ZV!*Ssp&E~QM60D|(;F#(EGUg@ckX$mJtOIQ&3nJ3~&46P{@RH@o z{Z$oJ2>x^J(Ezyyb0!k#khC8R%;Wr!`LZ z=P&+zz)RaB|C9Zi-x;az>K)r{!*}d}a2^xkJk2d{zaN420+T&+vP2 z5}vbp9_@+k+0o&gKhynM*siqsv=1_tY^p?{)}W*QCV8+I44Y!z_JYxm*n@3{!Kz3cUuotZ^x`ZScHMR>tL z*qcGJAoAK-dynpu{i#8&67(toQ+odAn$lqKLo$n6%+wOol8%TuR$YG@U9XSxOEtV@ z=QiBgn81Bb4p+>CsKqfRXI4-N=W$Nnzz0s=jW1d@Ziqv?$S^Q(H1Y1zc05u&9-j&8 zIK^}@Ye(4EuyN7EF1&a5@%WNc#piqrFECo7Trrg_NIu)PJA=J-7tbyiabbB3FDi}U z;;C`G{=^gUfn$%yx2r`wRG-96xfAi0ozK7(yN|)!3fu5Pql(3P6R#NCi7)Lw5x<-| z9*^da!v~TOfz!v?#VX#P+li0#61=oKKrdK8DL4g%?q0OQ1eb?4ZnMg`$7$eB-@(<* zGX6fd1E1Y`93Cs=aGP!7bqh;qTa+0-4Ykf5IDwCI%DeHNvE%U7dI8_5R`JtX9zQa2 z_<&o(ywQ+xY`DuXy;+25s;6eK57l4^fgRx;lPBR9rDx*WxPw7(7G5%L;g*w6#vPuE zk950u!{j!+ZQ@vb(y!s;?HFe^Y}BH8l$moo;Mr*wR@}qIyLaI0`JK4Qck$wAfWFC~ zxfR$E+2PF7JBNH{8RMO0lzMyM1s$BxGVpPyj_)+4@V+oaVt0{P5%#rgESDQNt?6T- zN?GJKULFtd)|nZcJGTI%GmTT4RlITHB;4j(xY-NwGAoqSIh=0{n#3cfL{V_m6MI=O<3W&l?!;RVk zv{&O5_|6<&F*b$!a@%pgRm3NHWF@<(w--^2_eno5))KsMERQc&r|^r)c6?;Iiw4gK zVS;_lB0iDdia+GH;ZEDf1>Umc*yefpbz>*ucco+TbvKVU^vNO)VB2%3#49Lu=EXLb z8dZFBY%BhCY6rd)m^i;3!;KOceuVQ&WqiI>!}psL_`b0d*Hw1n;@obWm&@V&dMSBMG%@~F|heo)P1`9GM zZf)>+whpb`r`GN_JTqlicK_z*+Wl*@KOVn7)K{WG^iK!I@u*E~N>v}=5IfpIJaA{f zuNulGaJ~tgKLvpE%&w38Q(vH(0vzuF8OKFy3T5WK6~=K3MavK{Qe<(3k(+{}%pGe_ zrHtG(AFD(5Bju56s8L?IRhJyTWwU{*+d|19kX1s-%?lH@VvVC}PNL+Nu+z*VA2c!9 zs-osus9P55wGcO4e>ZNp`m4D4x_fZzC%%R2KJ-;wf9>b+ws-vPG+m*b|t9i=Y&~1LEzCbAbFMvs*;f^ zEMR4(mIUBaW+?OPDxjx1otbl&%xqzbT1A!W7SO#4_66@H8<^lMnV|&g$a-d9ms^(H zBe|5NZY7K{pLATu&bWx};W%~$l*BHe=rf0J0R=OYVOEflZ6cFXWTI0>rvRsHM+y=P zkWFSRg9HdP62xTxv>K?l>S(km2dz1wnIpOC)KGDY!jvVTU2+Skc_q~B0&-?f`jfzQ z*)&iy%V?N6G+HGzjE2ZcH{1#;WDA>QO;=O4Z4O0Bg{R}d@gjImDQ#iUuGX)tz^~LZ zuOK*qQnJM59%WDyH0Rh#o?zP$5SsT8_-FeRJl6J-flJ_6+2&-PAASFmbgfD z+a~Q1mNS|0WNFt#c3T+r>??mKru-cO9up`hLw?e2V!~5tamlBAi1}M_3{<=($}WQ% z7%VW3nlp*I1P`d>_|f*i1Zj|QY_(E(PXYxCDXUnW5@eWh4<{qTj+CLnmj)}D6#$uk zr{j$bG+~8OKACJJmpORV0hj&G`{<}qb_ZDtx(*(E{p8tU6x3Y*voeiqHYHf;6jDbJ z%h6{NI9Apy87Bld*%mS;cyFD+S&*@g`EsPd9i1=&x}^@?CfUHsfdzCv zdX!=9iVZO`o6HrChk#9F9y`i#qtjUaPibJv<+gSF=o}Zex-hS1aJ1u3IbCi4fyR!1 zTR$%A?T`2H83(sxVK1eQWnsmzFKC~%eM)_c!FtPONR)9E|R z`N+1p?Rhl)E}A}pCXOR*d;wOtEOUk+J{>}wtJztdcKxG&{v95C_%S^A@Xzs+$9{|N z{P3r^|NB3}=f8X(&V3$pHFGX2Xr$oFkg-INj(|AlF2{Sw4Ay>@wjT1mVST%^e$U#k z&k@jCY+L(31KNl7uk&^P@-skw?eh*~3mm`f@7emcs+^EAb4MNGM>hX-josgVvi9rx z#yW6(Z0063V{zo$2_08mRQRzt8Uud_l%{!N{~(49Gt zDTC;^1d$nF!+<8PTV%&hi&BK=Eg^LG2*|9CyS_N}IS#VVu>Db60Y}bff<&Q8Z0met z-}DrSWq)#Br!3-nn$KC~vQ>7Oa^kj2`9!Y8$_yqWxYUVJ=qWH-k0P| z^uObQA3ug4{Nyn_{MfJY?H~Ra-+b^P-2RpOaL$D<728p%XLB1Z*NJSrv-)S_evti7n3KAmatw9c z$Y3OcJ4=hgAUF-85NYa|>nK^|Ok<$+GNXAp18IPOrVwH7YQyV*Ij$wu4pVu!GXhZ4 zPO4T%f*HxE=TsphiUi zu76G0r?NI5`KRukOx-})TduWKpwM(7v{3@bH35#P9@2yDcMuYgb9$I-CAfUoI6k}m zc-(Ci@u|qdiz6FW?4#M4LE`oC;_)1=-*FP|u$uT}yMq@sZ0xIBcyE3uepWdSU-Wk4 zd4=O}!PHh)NxLyqq_;ctt1Dsu|;S-e-9xCK;ea*nRB@-_y*6{j?-T3SAlktW| z38z+FWk|PIguPn{B2X#jtXc(syYmFx>Hw%fSHHXXblAZQ><}#ml(m_@B7x;c2$=6X zSds5vvU>vG$RCHh-5jp!1~{c+!!u{$`W-B~HC$!o@Nju6e%suE@8yogeYq{TV|**_ zojeZTp85-XuYNM_4jOp->^yqhzoHdUVyyael56N=i}OX?*_gtkB3f||mye%_hl|JJPCJjk>2Y-4i#z|8(t-Re#&lM5=mpoHv2vLpp#oQZwtYRYTbX zjyHkhr!{b_n`JXtL?c*2D+y7uccDs&;}*ftmM}Rrw<>eqYQ9>6$Rfo{<{{aJ1neBm ztxCX?z!&qilEF$ACE1T9GA~8em_Q-HW9D=%pk$_Crmr9_nTHjpj;VS{*qhashpJ(q z+N_{ZZlGbdQFB_zo4df=z09jybL%J*C}z%HrzXry=CCDz%iOS%)0eq_ZL&4V$kjZ< z1kj4Y2ql~m#$6L_X=kg3h*Nq%7h+GIde4w>Umo3aWhnl)jx7d`%;Y+HhknzNVBt(des zvflIB*OJ><8PWROvXrAadX)js`q4(oj%NQ0FrUgi(?$pyOTJ*6`I?)^lV$8p=$KjB z#*OrU#n*mhpRw<>{49qxnZH%(tD;#!-c(>a?~=Vs5Swgs%{xu|D+^P3YG_AfsXNM? z*4)O*pqAX)!XRgx7$~8(tFq*IVQLc`CKFrdAIE)4g9JD?O+(5ThP}vUr?!2N`CF0s zN?CGcZx>`f7f=z~*;xOx-|}RETZS-z2=F!Oh$iz`*d)`aIgEZKt5hZGDpprmqs)I; z_n4zr9XDjgHps3dJBBZ`bjYR`f2TuEopQ6{tfAwX4j*+CX3itNz$imUUQXmCg{+0q zYpwJxojzoK2pH`vE1JNj=B-P?t(0++0c0$j%xNQ7K$G&^sgq33C*nk-Lyc=RflCFA z>CmSmh9DH{oaN%%SpE5|?4>PpmeGlrISi*^3X`Fu&b!pn#JXGIlCU0zY`x5hP3L#& z02jt|nj2K)UNxs(#!lep3}LU$qe0+Tn5kNonlmHvXe|aGbQWR8eEFngUQcp;lC?;u2Oa+k2$S7Ohb0*R1W^b)^icPEQWk+? zg3P>UBjc~Wz%zP*k>J>aUWJPK!dX99J_#QM}9K6L+vdK{-C2> zK%j`ZQwcgJuoK!rg1XZdAcIa*GU*f~VIL0dZ&f0g^M`>OBj6nSO6OU|3e^AxaaLqZ zEeUF%6PWcHz1H@vIA22rsvdJ`<0*p$BuO;n5jTNZDkq_GKyF8!Dcp)y05Lv+nnpoV2@naVon&sp1#w0zb%oS3q)m4u&;@mISE~ zEa!YA&^ME=YdVI^4J-i{WUR1HQfY6h5($GfVjgBPU1+znu*h5%5MJ{g8tgA6uyb6$KKo$3Rmd`$^Gv7V;w<9RuNK{Up`vPs# zjTVK`?niWJFG&7w*Ibf-2(}@L_rkEIMJC*GIhSX_{MQ8IX@g{q(>^%IB`Ae<#6Ty% z3xd=Pf-sVlK}>d^uk3P;GrvoFW*f602QrMs)8Nmu674+Omcd^wI#Vj<7*xkKvlZHV}Yj{UWYe45|xZ(SH0`)^OId>C*0W zjU29f*_wEG>mON|8$16HD5w2upR-=WYwbFydsS9nt{rMiy8r3^qq5Hoo*J@dCAftu zG6KBw?4kjU1drK%uI&s6;@OoK_rmP_%DQAf^QFS-|_iW?BTxO^xK;HC7h8UKr=e{`O`|zpVGe)IHqR+){(X*pt0P;I<4RTSpRJQ zo0T)}LH9q-TLFJ{tz*7#vWA(OgEH1UZwWh`L5@KxO&!g{YEw?OhgQshDvswgteAlr zT*nyLqUluRjKdu6>=#O$b6uv&O)`%v&$%&EBQW1{4^@Keon`sO9h^~X;cr_FeA2h^d7ms`2e;WCz7#fbhrI*eF}L9dl@swbr-GZK5VyM)?krE@ z?ph8vYLkHyWtf%6OxZ$0@0{H(GK*TojjnVu6y>AQAs$6bvwzFx`W%AMPAX>JTJ zo7|1dCU)V{$qBrAYz!Yg{=eYC-0`^8_we#$30=1%Y*)ugFk26CYPEs03*&g+R1TNq z%6RpTU3k~_Ex0eY4ZoT^5#OHLiR-5(@Q>vQ+*sL#e>6+Drkmh-G1J}5!SkmvW4f4b zx;WK}aA|%LS8n@nxU<#3Ep7)FnIX(DL6vDYc;*RuaAa?shv(1WCHXvVYvyrdC%_x0 zXE9q3VA(vwEWtP0xUgm7Jjgs<1f@U5tT+xh|C8uxH6S-39e z5p$=LalL?AxQuSSgNuv=9}3#I)wA#ww~D*`GH&rI_*CNH9|9Bat>$pwQfY+w50GSh)GS18wOQvp)d1JktfKz1Ul4&Y{v4B?!0+RVF61W7&C<$!r zK-tZq>Jd~eW3pA2?-d-%H}n6yQS*u@w(_VF05+yja%`&BUzw+vAg_X>3LF#E%Yb9b3lr=mJG#+qp<&ceGYMEK zo0WiW$+S>u8N#9_^HvzclruJ4B2PRSY)8)DB{_rz9H#8FO|~Z)uL=%|M6^gzE2vHo zbIKpr+`<-F&18R8QdwmMbp^;~E?%-&O)^+Zs5Xlz@gj4V($Hkn+LI`|6DSJtVvJV%*#zO!KNsFu8fplC60I2Ma z%u%O*Pr)-fipf|NryW1Zq#^?#1Ix55ag>iT6<6hy)S*N$SAmsT)NJNnBPgx;m+3GO zHf)%(Q&Yzj>mlXw<%i&dHzzMwYZ=Ew5R2=1=D^yYahx%ySa*&1A3DGkh*4(Ph|^CT zf9vbd10~z64mEWyszXut6F#deQRUe<&=E?2MjT-Tv{K-Q&PWA}6*%Va;*1o~PIH8^ zpXn?dGLChO4^=+PWgU2FT?y7M3Q%9Q3EB-LkRa$SiLALx78Ds#WZDV9y$Cy5ffX;I z;m@k@>HwP^j9=RSXim=z%p&lpz%dy+s}2eQRMI*4IPVoQKiQs09jCNS zK2w3gQ2^-RZ<99bQ|8(8U=HcqPLs|cL zQ5@D4i}~pm(V(OznVfX45d?84GdzPv(h&wfbD!6zJD41_L}HgYb$xdQuDL7#GS^?q z5mH*Z+17E>zMK|`R+;aM`q;f5}*O95U7xl&MRAJy;YcYQ`uga7@71bI2m5x&Y_8 zu44ko5O@~Gf2Lrdpe~gNWIss3ZmtjfKAGySvjWSfeBmPM!K?%ow46R%dk%rM49c3D z!4jIuJj!I^#%)x?7$q;%wlZkMqXal5I$1A*-IUCqlXei0uOzgu3^<;ZvW1~7<&zD4 zTx0?&b0m_sT)U%kYP-J)&^yAq;g5wB+5**-|`!T#OgRwd(b9uO) zu^wx$tRCEl53v5ly8aQIYi(%%9L?{$z7E6lmx9Kl{i3qF+>`Xu@sqw=2Yzx*Qt&rB z>!@;rg2nnhzq=&B^7`{D&y1@0(wF^F`&cEabx+WG=)TJJNY63~;*Kh94Es~Zmu(q6 zpRN6#*g|&Z9X$u{H%=QXe-t>@@n0{UO%Rvo&mJrWO>m9l*^L2I;R5OhIe(4<$7|0k zdd`(~YCqt3cn&_?{jcHmg+c7L7t(XdtjrHex|6BTIqJmAXgISXOU^pemI!*(f}ViK zdhQ|fSkJc{Z>F>1-o(9>bBTcn4A$hkQG-E~F&VVesMs;wgzq#AqV0mfwHr>uV{UZ6 z59TmtI)YZKhnVMLGX`_gO2ApNfT}1q>~HsL=i$4)#laRTo08hFo4AFpiI@%OD7KJQlXz1ntsZ_EFIpO%luHGKzr`qPN4 z9v168oL6yh$>bQ`z2kV?K6X4FEFOm+myg4}Rv9-XUEJ)t_;Pa!-!-@4A?sM&QQn46 z6-)T!)?@JX!g2UkeHXr9SMmB;+Gc=W=;Oj#5nr$yczFC|+}mp4b&Up=91qQQ7op!4 zw(W~@MZ9U}$+%_fF?h7N9k<(MTvi^#wZ)V0VDn_WqZ=Xau3%p+z&pl^_VS`?$H?#-|e>?~5$F zs=t6e(LMyu4BUYGQ-Vei$~-foeW+P0uUgzp<1Kz47k3x%teFKo$M*5E zu}OS2H;&&Sv(c{!jAMy%WuPWyNK-wODz+xZ+rMa&&F$)=FwqZbF+<^x`&rJ4&FCN@X9@> zV|wsx7{Ol234L0vi1*Cfc;nm*o;z~}+6<-++A#eNEK2^fADtOo=!N(|C%{_=D|k+P z8rtqO>>yT^7JnISr;GFB1aFTz_=FkYv%Zhd^aETu?clE$BAhqdN4w26t&4KlLp9zD zCpaDFRBT++_3`)p0iHW^8bSuv#4#FC0Mqwj2Az>}$8ep=fMb21;u@&`Q-8wfzBc^M z;jUSm_igrBW&^8u*d}nizfthux?e|a0*6lt;Ba+;-S0p@w1ds@f9l$QHm{HNQ(lk= zXe1uW))=Y;A*~5v6K23GS;Z;~OpvQ2U@)bga{~MloK=}%f`RnT~!A1#h?H#VvUXR2B(b8)Oxms90pb zD!|Bk5FDlRzY9pr{MEv+R@SJF z839*bERWykrEL%iVfilmDG#zv1;jN9D74C`I(1a+DyCX&3mM1E^-DmLpmQ0_%{<{6 z0&tdsB1&wZRnqZiUx-|DMSx@44C}yt7S^!j2c{G=`;`5{eq-KgGRV~iSr_K}<@YH6 ztc+u2eJe<=EOE^<%{Gx;oPzn3OShGk&h~Gp|H;Bl`%m&=QwrKGNnnNKXr>Jd8(aG} zZ&ffQ%?iwQLHMElGT%7+ zwJ39iFdPuZ~T9P70!l zbARZ}<2|F7yrg6iu4_ICzUsxY=>X(K<_M*ShmKC&gEPhm>nF?txt}0K${Y~~-sm1R z+`F|7!}78^v7U6gDUh0d9!>rAIqJaU&zVy>%WbE*O__6+7xNrawvvD&ao-_BXWItpC=2W;Vb2$+IF^?Ud~m*>B!Y2XN|`O+hFEl5F3qlU2YVI-L39t{@ou zMQle{@;Yt;7^m}tfD$F;$q;Dxl)Y1S6~}}5$~8x$I>#HS#H{4QB_opT8v-9>4tg^Z z48VMwUYio{WQq@DEVQ0F7FypSgOk52_^pm{oi931R?p6>wx|w@)xRI&OwH%&v-w=E zLCW|fh@$}GqUM^-*n^xSBMevG=elGCXn$tUTl=Z~Njo22Ixi&1fqgDu51s3*&(X|3 z?c<>h4f`=$<0#|K%kS+$*`$;(nNQPlvA%SWSDiM@&J1c^N95cqQH)|IK$Ffh%4`M- z<}x=c>n^Owq=$Mya4YRg$|N(#ZrnjFVUF64+$U-RX;25}+$eZLrx#h50w@#UNbfUU zvSA6Z%&WQpg9pgWN?Do&l*#0z+%4~uD;p~TU$h@+`^tJIqf@1ym1WCu;F#*gHe{?} z{nUQtm^Yc(S@_E0 zmiyh5-Qo3wxoUL!Xm$I-dSRJlr8?{{cM+DmoZ7C=eX3%}Ixg!2gLsr_O#5Yh6o6uX z5vXQ4n%A0ZXg0RGo@YQ6>#D3i*3D)9V{ZvoKwz0cCmQJDgmZ9xohK&Qxn$f$v#5qW zktS!bLMvf@YC1qu2D;?Q)-mI=SRV!{vAlYu{jc>@hOg}|99{WQSAr$XIGuys_a%LS z?nxY1rZ(U{!F8AWKKFkOco|4;V~71D3Ls4?N!Uv^>A(Z*;<&M4?>=6C`C$EEJ0wkQ36#~*oQ*}4>^l* zUg)_-6;}wFGB~d~r?LD(9&!=^Dtg_;)RPIL@qZcCme0r?fA#^-Sz)QPv#J4Xzh*rTNyg z^@@GBy8ov@EB8mvH{PRrnx5-qPN(ak?71UnW*uLx!)SfjZq1#pXN|S}$NTmCM?2$X zGKi1BD$m-SZww%lGi2KLqjsJue(3mfy&1LhEWk|H2L{fl@&nflj{TwpqDTOY%k#$^ zqV8T)TV1&El1OEf{mc65`CseGG1ReS>J+X88sNq=DBlZszwE2D0W#2|IRSSqcUFR$ z3|HC3TnlM?Zm4Uq1oF7^aDzn&pkmwWJijpw2${{2iCYTDJs;B+c(x~#&~=C4aXq5?2m^jPu-y=IR)}X+d|Xlq z@o7KASAz!L76n)~W1L&I@ZOe(&)5!bYm{-HIgW1xW4OBO;*yyq^lK58j4s|$s^G!e z1a9*zyw+>uV%x{7Z42+N)^S74#Eq7Zt2;9|$MIl9%nd&S)9a&S#@J{0cvi#1=@ke2 z>Jj$TW1Krx#cL+VaiiPBtxg>;b8L8C8|N2H+*l^KYvIGd&vq>QgV)3@Rvlkyl<|pGlv~Ybl z!MSlHL12B$#VeZzKGv+@_Ua_=snzk7W`L{X1TU#h;+nXG&pW&E`T>Kk2UxBfc=N=` zxYsoB{X!LY+a^BUc5q`9;8xeiO|gm3bjtWvD~Cr5dE9RKxIFA5v0WIQNb*g;w%)+U zLJK#wo4CcT<96G^7kvk}1r~1b>-bcsg^%|fe6Zu-ogD{n=(xBz3~{a#W5$i)y8+H> zv~ZOd;j^ZVJE}$8RU5~htufr>TX=tW0p~db2_mCP5!dEAl{J!{q}cP~4!R8sr!`zG z8WB3~44Nk8+loMpj8=p&m__3BF=KdGHY~L57*yA4`U~*Gd2}of3q}odR)8MgHO#i; zLboHHn^n1i?`s`1#Gbl=rB;lNH3!#W-q)T0#})2*(Hvs0jm5f)zpVRszTx3K!^A13 zfxg{>OC^z5^O-l|0qP;w&t(kEHcqKEu-ppJs?>`W5OnVbwki1`SbboyE`%aWgL%wX|ZeFvIGI$9z5@b>^S){6k zu}bjoiMteBRWOr4+bDQUFi@qK6&$Agahbrh$Xlm!#ufq9jFC(*k^mi5!*O=hw(&2wpvXaVCX`7Qa7 z83;0|OlFbqW^e>m0hj}v^WcdeyKMd{Q$H{HuepAibRJXckcS=SP!z{lCaaf$ESZyV zxUU`k+V}K+Ms^-h;FykoIw0vRJlK^wl7*eBj-o&3zL%HNgSM4@Fl_J8L9LF`%whZI z-2SZY3RWG6v8ZylLok$%RRy<(ti}<>?#N4i_UzHm1NArmf3)S(c|H34C+lYdG6F8R zn!{F@n=T!;?5}LyqCM$-WD@%JvM@WP{j1LJr}ujFWR2g`wOqBwp+vIUY6g1k|EVC7 z-cKegzyIX+oiZLpHh4`Sl>+|x1_y57$b~xiokjqO!E(T4AoH`kr!kS-JM!Yi*vM7~F$@N>6HOQu<^336s z^QXQi+gBK5pgGRDw;aL#BOvgR@BjZ<_TQuJUjoLGmzMym0PYHSNbn2;Xc*KYiV#$$ zpn8D@8c^MU3JM}2{WxcyL)Goi-Ts)~z+=!%FfBRd8Jxp_3dfs8U~8Wl@d66TH1eGe ztT@7q>EQg5jkh$q_(*pV@9y_;UJxKQEIha3;A71gU-m8B9ZceOe*&NCn0RHokAY7` zp*aM>G+xs(ahp-WyO#EV&o5F8_@ZinceVH7o&D2rmQ4^hM7iBTF;zE->;YVt z^;8v#R(l!?brY8q8u++Bzzy*dUJxYk6Bo}b=J2_C6W8`;@mCA`uqRwV$EE_xq696T zY8ZHT9 zY(GS;GmSyhz~7XoaI0Ivy>=0w^Sbyz{A^qvui#m&2Hwy!@eczNXZx1qre8Elcy(d)fudIMY*ckulF4EA<9=yn-Un4lT8MKyr$PGqQCeqVAi|7ER( zYg`ZCa11<98N>Z%4!0x?yl2qHMYBto<=ks6!FBh^oHW9&0M|7?O4r{bJ%{MNZ+TQF z*&~wFlf7w_6AeHkIL&o_so~?|sS<9>7x2i`6h2iQ!v*#f`d%BBeRF3U~g zb6cN*Zx>I-y+Ivs3<3dkoBkd&{C%><#BKo3_tA=C2_P}OJrbOv>%&IRAsgEV54A>Z z{%^BypQ<)6dLG>|S;MYeQ-g^m?bX_#g{N8{Yy!u0{2k`uEi;s-3UIjk;x zs*V3f7c*rCG0#Rf%}dNY#LS;NDJ)KrGPcNSWIkT9HZ_kfbFWfHmw@XK)DtAeO3KV`o%k8hSISeUx}-~R2_Iuo4D>YBDkn6}yd>EEP%Xv!=`>)Vn2zV^G?w(NiFKlR=WY*yRmoXqAW zgCN)sl)om}Py1!C1R2(3opTIz+?gAClo#7#wE*Zvw|p_DcyJ!|N|SPs!P0 zm7SCV$pY>V@87uN{mE@Vo#e_kITTRNm-<6pPi`AFb_P%;BHKqt+A8x%9LXt*iHs|< z3sZ)nf<<~izqblB9PSx31L{|8i%y^QiT=IvdHq|r2#$rKkq=c#Ii>3=d+Kp zd@2sCNp->Jj6Ul?bkCby|yQ7*V@m~ z^TXP9AGlsedjFBz^?}Ph&~wN}XUiP)n?@z(f)48l_ z7nIJX6dDv^XxHy`{E0bmHV?aUB!(79_3ylWO zPx_ectOywHTPl-WY!4875l%BKoacmij^D+AN;pmrRjNybJ*>1Gyef+E3dhBTrY%g- zYS2d^nnfj9LaDnC`FI&EvyWM;gMsfMacelkbMXRy8q+}!r5+Uu=Fn*^!1Lx%Whx5K zL&tM*j^pF`ekZce;%}Q~yNE zNW2hdc@13F9pKV%FZQ+O5l&zr<^TX707*naR5}aj+VhwZW^9ON+r*09!YkblUNOCl z(|U98Lo(jy;I}+1H>x-<@Nu3O;aPQC*xdV!5Hm)A*zJgvdaW}+fmbpsAZ?Q=7O3bT zfg5eiHyr$x6XVTpfGe7Hyxwc!#j%4kx)HkVHf*N@$C*RuufTN}+%bz*JPR|{GYaLa z$!zCmlL{ZJ?;?8cRNGgX>v}kkTBMiZFTr+KP>*I&>}ZM)uJ=s_DFiEMTV0%6ZQzsT z5`I<9;|IoWe4@D(Z_97TYj;oLRXca!jXRFRXUC7j4~kpx4X1=_+5w&y^1KVO9TqVUuh z|Ix8MvY+*U&pcp_U>S{Q26?N6DFQxT0aK3V`V~Nmd3#ePBbkbFIs6-?f1e!2Vg{ZH zQ&{Svz*)wC=BNHo0dC4_Gbb!RGr&6o0GXqf033f;z%2tv35sR8fGHE5<^-1eGeA`( zh}VLJ>%qSNyuTAXB=a;U>{A8;WB?@lkFw-srxua(iYXJD{n=2qwbcT1DraM{`T1mw zGl68Lt=g~V{8p)R$#tE|wm;GKt3W&JOxf^F#_^G@-TR;GHot## zeX<&Omw}n#M)eVhHe-Ujwt{~l+(JJS2t zm-om2o$oP+Qh?yPMsR2S`j&l9--C|6UfFZ9I;?-*pSt{w+4O$n`E=X=gek4-F@djI zG=pk9KrJR4P;-=v@(uIs2Aa>9>o;?jS7WaI>&JUz<8h>Rc(nTu!C`)vigj8=g(sWTqP!pcON{Net8X;QAhdAcPYpl7pPeL1c`3&YUO=5GX4r15}c} zFlWhxZFmv9ppCZQLr9eY{}dPz?~7n`7+5e5+Z-TpC^tVqIqsrCRV=#;+iOc%7S{=X z0Gqj`g9XVm?FR`QFGkC0gYxE$Xa$w_3i8PuN(}A@mSDL9*nSVqI6*1$U_}v#D8(TfQ4A;S!4Fus1(@~{40kU~N>|I7<;d1yf~x!;0~{iI4zAY|Ca@)A z)k7m#L?zsZM!0~;aPi7Y6?aso@K4R1_(|h9{H*v4JUVp@eqDMd9xMI@eo}lUzT4b| z8xs$&>CR#%m`6U`i$Z%Js!U@QQ5rlDn=W_SD7Tphcp8l~&|)KN$mVx9wgzwZ-_y7M zGCYj?@FsA4h?l>tqn;|j+3JhWA=*5;Fl@H}sRf)q-N%2_wvXm#%_+7h4o{U|E~4P& zF~z*K%$X`o+-jO^cECU6b5e!T&*%UpIGoL1BZnEm9{1;vm{JWg5qQqH`A-3xrgnN%3eE@lKWKv=T+A26Wjic zg0&oD<|XE(d2|&h7O;2~FfL$Hfa4-2Mb0}78jyaW-1nbXoBwmvcNFkVf!|fu@}c@u z8OPFw6fCA46M!u`Qz-FDbAC%eghdv4b9f2RPrIg#=TVeA(#lAGBKu_%IDYyb){kbN z`cr;K&Je<+UdOaP%roVmvTlFsXJ^1InM!Mc;ZYgdENWk!GrUh&z#_xE9*{XQyMD6E z-)R3Tr)|dO5|Fto_tJsC(K>8?|Hy1^v%iM@CGW!`$N%`u z(u{Sg@6dFlXWtn<@5$*~u1lj=Rv)fSqrmOxT6LiNv+o`0=jgNebN2lws{E|Hqx*bX z?|-Ynv4X+1cv>9#)r7z>*K?I`Ho^{CVMmm?xRz(&;YNYOp{*R<*d6u$6YKsXU?2DM zCu#rRX!|$*ea=^c=*qMvASvwBbnF}4&!aAya%o_g+>?UG8y){4aQx?Lf2vZ${$oFC zdvpy{R`vqOE+vDX((Ys+lC9rJriCF~X8=PmE5LF&o<}WNLL**5Fa1G0V>;re9Ft_Tyi8T3$VcTw)OQS8PjwqpSwY2(aa%{%wq3clM%57AtKf zA$UCvn_w?<$_jgxjL`+Of<3T3E)4c?4y}|qY6r6-`^{&` z`JB&*mQYWYg|SSzbl+NlVa=eGEJ&W&ir+^unZ{Im8nuLDF%2u36G`tT>+3ML`LZyz z$+)E~v*yg!yydNE8WwZlMhp&Rj_zet!#&7{E0_uvQI6;@WP3Q43#dgaLOwBr6*$3) zjENVnzzJ7ia_qx-VFQz?O7NMvhg|~e?f@2<*<^x~k?Jj?>MWz?oq~F>4^@9j?7~Q< zVYYiPq69|Jk$PGFA`FL8@vH~e&e47M@z>G}IA$<`@65vrdT1me8i@}h4mGgTUzA+E zym!S8aj9kDy`hOucMNGxs(3=pz|pE-YCO31C4WHkDAN72m;4sgy(Hz^-_j-MQm8}f)}0?`(0fx zHW|kwj^+cO*^Yco-+azypBGyru0ix7WM?!*o>9Awz~$27s!3<(xR+m_v~O z@<#gl$u55*bszmMos;Y%0>^rFQin6yL@E)=`;CB1VRf94$b8DPvcZ1frhkN-G*cwk$6vdjNdw(Y6&Y_;PN#%Bf` z#~E;}z@b6L81rh1S_Hu|uH}>qeWL8?(0zW?`%l;xKq?Cad!qOM1J9j9xAl+t9hGBe zc`DVN0mlRZ*Y4xl*eg?;lFS4^|9HT$1|zHnF8rC;AA^UoT(xE}GLNExFfD5lW#1Rk z3YLXgOYoKQwx!M-3cWc2gI%6E-B~of0cznQD)C-aqSI0L&xC284&OROn5AUeTHy== zXIj8M*Cj}~M{?fMew*GjN>La2={_cAdKmBbQ0nMBwWJKqOHCk?bC+Y!^BBQv`CGIH zwRi!Qge-Id&1ou;XjbI5n?CD1je3Xb10DEwAE86(A|9+jo%mPW#&$Pl#*j*Uy4$L@0BMMQE64>rELURGWw-2r0|7Y*Z|0S#HYv14UKJOEi?pt%!sXFsK z)qJb&+|49{5ra{qNrWUMF={X|5m93_8qHu-2*}WkqQ2xgL`^hN29XE?afk*KMdtNB z>$~gpy|m4x@4ekLE%n3s)V)<_*!z3-*?XV8zH2SshjoO>3jCDwgB(^sko+JVe*><+ zF0#*#w*uQ=7XMzdi=A{CmVm~TK^}`Yv?LlN1Gw=J0X?zP6#=Nt;2_-K0*+XF@^ZvPe?0<`P!Tz(|P0;EG(E2{?G!^e|0{)#I+v5~^kR$P9 ztj7snllSnZ?g+0RE#kbTWsI`{qG$*&I{-aB6P@e~0rxu`GYCY}*O=^d%HNB(@w$r7 zFR@k4@$9GP=X(Ce#1~Vnx4!?(8OPgz$3>v#!0{Y7-dEP~(vp5kFI>%mE%9{SWMkI-s$yn(^nN8^%sLXZQ%qw87`LDah|E|}y19dLpraJE`k*V{u;RO!i;=N7SfNb*Vb5V{wDj+{JGeE_i}wM zbgO4zXN-g6^V~3QT(lz-g&@LM;!}X^icL&r<`il{ei`&O<~vvc9?u!ao9*Iw-IMuw zb`sNqHO%Knv6G$~Kvw&*9q*Lsb7Jq!*8YZn9|;)qyUp0J-`V>A-v2(?*FWC%Ir4l* z)=&MvTkv$Z$L;+-ZPtq(oAk64kK>rY23d5JlclUKlYz)Y9jHbBczDg_r@khqrkYm)&$N!T({_Z(`;q)2v9rkBRu3D*xC~FDjpDt^sBIQ02 zKWutp|Dpu`X4^WJvYo2!?=WLCTtpNsicdJfH_E%WbF%gcSdtk`)-3_hRRqyGym(#Y zkqIbkWEiJYxY;uNXo4^p3q!XPljXV&KR6wccRE669TxqKGfF^@k$D58-caoyPoWWv zOIL-(N?@{;kI?9kQSVP+tjl0@wz%-V1aEn?E2N)U*$ z34(A4GajLtjnJgTaNdO*_K*S~KYikV_P_!HUB_YVq?Yh+Vl$kJ2U z6$Uqx8pvv;JordFk)5EA%o!qg7m*Q=mfwdkQ-abfFr!822^sZW7)gqDnhJZ{>rN4o z)f}!P2-o4cQ~3T^7`j&0gO@FeBscpG%5!^(Vj`O%=LTdFFTzgwelUU=Gmeyj?n6&h zLJpI0oTvvo8;g`Vy>ppp;CRbOyfq{~V={)FC|U7()=U(aF4EFWh_It2v?I3F6~!Ko=`}t;Kj>rRc9A#(7!K>V3b(f* zWp7dLJD$SJr_kdZdX`I4seBi#A@mL)^wtps>V4pu<=R>QF+X-5uX!8TeH$q7P~sqS z;CP!Cq9Rap;CK!k&w=ApA2{FZph1ZiKc;_3fVR^_%c()5WV558=~!qwwrnj&7bdRe zR?+rK0-AQ*3R>=rvG_a#r-hd>uitTM=s0yLzvi^jayn{rH35;^ZVhd>BAe#cKtP!c zV_{^97qSBHC{?YP$7CBT-{We~LMd!wA!?!&(d$l<@c2Q3k%>_m^TDN$>1o?dt|)$zxti~31f@< zi^tdQuYd9uuGPX57bHV(1Aee70CB-w;zAkgOMrpQY=Z2J7ylLs!x{6q*pK_pw|NXs zej8Bz!Lx(Ms@jO&`YG1!&Y8Cajn%Pg zTsT?niQ^o{yUk>&WdRG+oQN@;jgN=DTW9~M-sLK)~qkt@!FMUWLsz%C%5 z8%&S{Q^XM&tMu8OzzIi4-9FMFN06j2a>_C zC#97s6U_DR)E^*s$H)kVx)XTGiZECSzS@+kPrBk=oQDLF2dMT3X!Mn;j2{k>dIYj3 zXk|kILh0R1Ah?xw(a3seWCOHgg1Pi;9>a>p(8+AgMzFF0{GJ0y2u7U`fp<>7||YU}QP$EEhj$FPS0?R}qIBu=7>adzAei!3|YgwaJj?96KJP z9gksTOA?o!vk1mG5Otg(+TF3-FQc~vJD& ze!$pU1b)LVoT#f588Wh{SwF@?N&8#k4?h4I>9(-asbHie;dwln%SP5kgMP-n9IZTu z9_EO`vG^p@-iDFLEqnhk&KNuqpCJ4iW@RM_4hX=s>7eqtEQ4e95!wYkmaZm0y6C}vC_k%UW z?ivzr1(DDGMD-oT^R&CV?7sh*UvD0}7jqjZ@X$GM{2VVrMWE)u@f1i(QsVZ1Vnb*Ga2UwG^d3&L0SP=l~nZe3Ov!@)oV&QbX!Bm zW?AUn+7ZCBCSI-``L6A>(6QClb~Lpg*~A_hm}J$~K?Y~Tqg=H*Pm?jA|8H1FDXL&0 zE~6BePzfu-@+3Q1or?hT^I!kH#+r4L__;+~SaR|o=H}dm0N6Vn{{btoBI~WQHovNPt|@Uc|x9r zmYla7skU2+7f^}HA~{bFZ+YHN*7G|Dj$a@^QL!L=v;|zgTo>DsFId|hcjUe=?Em@F zUTn|)62oC~7x9f8e_qpnLVt!W(PXeHOWXCWT3T#&~Rljcn$E&l& zY07(ZKeb3V2aZqe=jX(-%?#q-KfUc2b?kM_ymkKF(HF7J`oE6VeW{mo`skMOS- z=Aim+@AvRzU;oH>?Ck$y?BnON$D*j?Ge5bYPgdd4*u!eAsbI7}^W5D3*^YbmGCcDf z$74Tw`zj!CX=c$t<{vlJm#b$%NwO<;=mees%`X(-cv;wMJ0E`tj1c%y8|%57@&DwF zzk6#71#qm!Zv`ClzA^@ESDfRhq_#qh<(ch7wHeACtNnM|7Ej&wr{b@GZ%wyELV>G*qQf5jd^LxrdchZ4?t|3A4a3bt!FHv5; zJcmW*YPtx!JAsui!b(T*f+0e805_ojZ4X{PMj$@KQ*?5A0kd6=(TRJ|djpsm<)$+@ zX&>|;mSl!tBCPIaZwx0{LhP)-^Os59eB&C;fNXG0Uj4u6$6~j9cKXiH@PvHtPdI0Igoqmt))gA+#QSfcpruDLj9Kn7}Q;8bg=opoU)5u7-LAEyYyo_Jkzj1-?|`ZyanJmTv_-xs2Se}Qvp+O5x?6d1Ag__7p zR|tSQ9cY#g&ERIB)LPv!ALal7A+_59DZ-*(s7 z?zipFy9pdi+l!juXT_;XY+FGaTBxgM(QTn()u4rK)VwSVc1{W1shbP>NoPdRjrrDM6&?9UV$-zrtzaI2zj zz`o3hS71@jHDQ)9vyr)YJyp_h!i?o4SG|nxweOoB&scKtV*(=>!A+-VyFK{%vMhFZ z8Nct#q6j6&cuyEJwI9DHXt%e8v*LX8CkwhO;#0=+GdsVyKfEr>WCX)!l!3n&Wh(aF z>8C83Z)Lz9W&X^vuuQs74>7f%%x%nlGgN705Xd89CO83!z*TlDMX0nAuRg z*z~+F3r==-j84!4CG;$JEDJ$KG=SS(f=lK{%r?WtvI$(u?{NW*3q2`4)fVA!Q7an3 z;UXDXxfvH|cYl3fr14JPi!XuUAY6szEF#R;q)mtnA%S#Rgq*w)wq|f0UV}%qpc$i*b^1x5A+??d)g>Kbt6d{u%2|>YGra zb_MoHZa>9%M~_X8dwxpgh$ZpdWSda)9*$>hCyTIVeL9(pKKp$EGj{&W9{LJ#|@4q z&7iGGcO6!DMVOirw{VCgT0|0Xn;;>x-y0zm@OOkLAxn7#GoPSAFg95dMzZZGrl-*z zqLIeX20b+Mp6n-JrcZEx30iN2g?@s1m#kJLHBWGln9H=jT!TQ*-3#epzchwClDi&kMLt7ykd=yX@m;%B@JGhG(2Rp!3wBFM-M zy(w~k4P9>yvAu+tQpADk=g45~B$R6A^+v*^?ggueI98E$D;s672dMW3&_tG6v75DS zU%+KQVtFa?JP?U_z6Xmmct(cyGJ4jADBDo(xt$T1ox+G$VMPZKcpFIF73qgtU7lkk z@nQQTaB35T6{Vq7&}*5@STeO4>vl$fbtL8HdMklYDlL#fPJmvLOS7SPLsKT4jPq8` z1Onby1@`heW3mO0bHWvv@v6u=6FjB#co;B&bP=6wQ6$Ov$^Uu3B7YGvH*bpAQwjl* zdjOGpP$bCt+!cu+$Nmhe_f{kALC+I7Nmr!H6R%*2RRtz%k~)@f?KDSNztVd1M!aB-c4Q0QDvTLCp#At@@3^+V~K;e;q)6u}G z0{Cn=Z2>i_j)q#KVSiM?X>kcJ3l>m{YG{OQXugG(ZK3TrXgCJyeg_MF9s8py7Q%|u zm+Z|tJ&fHpc&^m=q3ZwuAOJ~3K~$>K6p*g!mr?Qep%(5(Gby9umC=j{%2}w}4(g7s z>OW)f9$EI|@&D24Uz~Tl-->;X_IWsdu8F$kpy@es|EpmO<*7wkEQW06NycO{VZ@Hb+l@dd!D9J6TZBIY zJFS3BhedGFP4l|K)^Jm@0o2Atb;BdLyaK};$znW#rnX1c1f}$ba$R0uSQNXu{@xQ8 zjz|2-$lzScppP3D2!+W#QgI|8FNzTovj={M%(I3Dfhc$|ZMBKtyq zvR`EX$n&yqWZzo!*+qX^e9zxJ0)MgE2z>JS>t=KuP8a+ zXbFXv;qj=)?#`jcFD9!V5p4zrBoqM#wQ@3Oyjms`_Gr#srJw?h-7&CmZEj`F%WCm+_qYTM0p@%>=QN z5eN3!789g67BQhInZWebd_;|bBj`~dc22)z0!yloXTlHnE$VzS%D(lQJ0E{WGX8L^ z`ehh@x5i>;e=A^1o`G~EGP`Ep2REm;2m*slsw99GrfUK&vz?HsZBl|a+uL(^s<%I3 zUS=C;xg%lya%^EgOD|$47{Fw*5&=xVFMR_00p05=Msd`GE0!K(_~8^0?N5?bSY)J9 z&Ur9GE6dPKQt=s&Qi6|)6-)nQdJda8!8ZkJ62zk$c}F36Y&8yl3eDY20eTO1w0c>WwL-@B*~_$=nzcK*5ujeyMcgz-XemK%=Hxn z?gUA+f;3x$>2vP9C@kQ1JVH2JQ{$0GuiO>b{uVOV%)W#GHi2z|+I&6<4zr!`I@~xO zv(A*uru;J#KZ0;9AUPS}TBexDWYg-LZ>JoumxWbprb9HlJ*m6LWSH4})REY+Ei0dQ z0?LXFtGwRX-qp1zWvvv3h&%9pkgY2!I+0SJ5RZ7WYi9}H{Ef*L1o8OGg`B4|w6X~t9m;)+E~)g?-X6Z{vTJ)9u&$+lN2LwZI~y@ysW7m06r zdNa{sXZ_WD|9QOTZQyvdfx~#{95~*-Q5_D{95|i>$K^S2{9*@=WwMj@(Tqb>15L4S z!xBo~KGfq1%Km_zd4CSbVg|LnVvU@!j3k7g0tka!>XUDexcHmUa)fCtp zwy@7%z(Tr!{n=?)$o8R}lu#D`R~=QG>`EJDTSF_=W#3X-LM1Mv>Q_+@35>T;_BE88 z7AiCE*M?hz5o;)SUWTSyg%*&J&CNp1by0IqP2jjO1G?*;frjs+H87VNh>9+(l zuG>wt0|R8fliAMXh`QZE%WI<@u`So2#cg4}SA!uzwLh8?!0s0c>8(U@tPPv_jxvfV*(EZhsmy? zl$#m#U?&4O=@4c}_BVkB1x8r2>}vuK&wY*U>sm17xadUCgNv~Qka=GIO~8ZTM{ymN zll$_2F8UYWpNLP+m&uy)2y_O0SmEIC-;|akxN;({`+S{~i{o50C-c|MxgbRLFXgD! zB05<(T+B1dsYiN%DZjX7gxLmME)scUGb?FF?&lFy zpx;~2lj}RXyuKbP&*kSUUPoSs3vrbGGrP*?PBEyt&_{W~vBXta+bLNdeHcj(es7|F z;{xj`&R9P(9`paV@9R!PeAtGlhQPkE=syV{Z|zTc-J%aIz%Kqyz_S33sS>~iW6889 zJ2d3t^+@%1{y?64N<<1sN}0uURlwR4QKw@+r(zrvG*&7H1ab+Y6Z{(qJZi@Y? zKH0Bx;eT)Ak78=?<@(27*XO@&J=gfhwr7Nk2zqPM-Utp$tl4&g9>SEKrS$Vu<0zA^ zIDUKi3g{Qf&y!hy_TL1Mw2+JWGlua@##|CG#ACEC{d+6Qpl3a3aSpRPKqv0P6yMJ0 z|GpG4+w1t_az3|*@&9CxzeOB&`^ohW`p{}g7be+&NsLw;!Jq_kq}l{ohCxQ95YCmS zGIsOcJN+R#VRujM_f&6x1dbh1QQ7GSm!^#uT>w=L(HUT*|D z>B9Chk!+6Kp|Fx|Z-kD|-;~-B0aVVLf_x0$?;+sapDIbg1VOq1?(cI>9;tpu-d{vP z=tvKBzS}6kUl{fR91qa!j#2F`q8{f`9{L-L40JR?lr4(1bL5SYhKq3BT;{NCveT1Q zNIf}zk`0?5gcon9HbOyJ)S@(OPC(Yy%kV;+s)SnJQZ+NB#c#k ze9ott$TEA+BA{wV1R$3MfTdh_6pY|AsWW0c1lfZ(@Ejc8X!lbMysVCX+dFSCyA^usmZH zD~2(FVgkklmxT>3lFy1EOr|rzV}o&vHsFNx7^c@ay@1K)B}mS;pn!A&>SQ95QLUwG z(9#X)Nilxx6gXD<^IG&bHsXV*=R6;;&*TRt3iV-RJ^AEwN(MBg&IxvNBjcE5c7lVj zgEL@-XF`t;qMfYGKyfB5ED6(_;5d&bs7%H&0cp8^v&1PTuI#oCn;$oi-MkI#t_>W< zL+8Np_5nN`s5x*v2ae~!G3LOr@-Z~BR9J}gSk=fbb}Q&al)qg-JE@@-oQ4_!#qc!j z4;D~P+9<^~YLSv^-VP)>F~2F`jlfwcsG}M;P!7tdMP*ci{iuZmJ@=vF*HH1AD3L+z zcN8ETP-3|(Y|MsVMa`{>{BJdEqU<+>QQZg?XxFxNbW8)LWg(0rv~(Snj)tc1pcW9Q zdv4(E#MFO#u*5%aNuKY3v(Rx&=!OB)G~szJ+NO?*&STrC#U?5z zP-1+$`!NTOx1XmyuP*}3dKREy2*d9o>`sLd;HCrkIVG$IVrRe%crpeQdq9DN$D^DC ze>WwvmBcR>^9cYHfCv|x`9HtscLl5N_yBK5XRQ`4M}`vtQESG)a>SEu z5jN!l(-o0+w7dx%GB$!G#5n=vC6J{|z={lIJ6nV&vgnHGOwf9#*XKD-UhEd3rvtR| z9A=N;I|1`0SOm!fN?Fo3kG^(XxFgG>3oBEgI~UXfU&&vd%<|8#eRA8+@rpk|;X;*x z{VDrL_L)Tj4zI)KPJq7IzC6ayR|IfnA3zyi_U&PJ2_6As`arTDV81_Gm?zN6u|hG8 z6}$I%)nRw%*-U&Aw)Bj7t=OYu0lmp;V;{~>j$cdwVj>VXCjRVg{3+!4sn$_|)=a)4 z2zhMR-(9_rXS?0Y_%{&HxMk}1427`J4I5zr!AGNi45@(LC9qd${uXChr;+mQ8O_lK~$Oq$v$U~xR}+wmBG z`1~)*cz-I6zk6*nMlyz$5HRk_JvF;M7+DuiS3SS9!tjC>xUSlS8A+x&$MNm}`d)!! zGQLmY_QzyEH&~KqMwq8$QKn1cpWBLtXl9FOQ0jQJhIYOPjUXgtjJ*W@3=n2gsuT2) z%=%<9WhsF#^TkX_B^%)o9_RhxNPuI`P5opImOqraINwbq$wkz13h&y9lFnxGM@A5p zY}uha>jYQ{Bon-(r?Ac>znsr9=e5 zVn@JfwI5>}h%9qoVoH#h=PwxCN>*L%%fw>-PGH*)2xe~r#wtN{DrHbCX)>hM9D?sn zWG)jpB`9si^pZXRGg=q6u>xi{0b}k{fYmHBfp9XOjZ}FP3z)kH&{dUn1?_Yh^)A`g zV~Hcn*vW>%Am(?hM-V8I9r$di7(4oSTsBerB`pZb2btLY( z$cYosCvdEFb9Azvw8g#lVe{jU@0dD|=Zhzv#TpOa3tn38U@letES>4$K+S>UIdD7& zj$ib^F(*J|l(|_?JbUY%7CN>8P4B>VT}CR|mBQ*o=vXe=k&S%>oFnD&y!(0_ z9{_mg*WUy#>6u&cD++Xv7KCBmjH<%4Zkr8QhJkiNgXj3rEmPu03Gn?QXWp3I{{lch zS@k(JuS>~rw$Fwz+36py3)7gOamz8#@on*zZaF%tW)n511wFA)wre62PVeKUtAXdC zjQoOE70=_QucPBQ(A&Ohx77${Erf69Xc;zYo{kdzktuoo9E{@=@%tP&eql_`xhPOD z%+zAK^1QdWAQo|0V#7xL{iV#azTEK+KJQ7i=(sA_ZfP z-|;sBBmCUk^_5iNkzUQkJXt)@(h&kOV#7X`*3QDUH+}~H{m>)$?}vUafW+Ree{$Oa z<@`iGE~0dFADuXd&c!ZDQb)@O?KMR04GgSvkn3k6HV+{5*VKK<2g0_coTE*a2LY>e zS%729n;oC5fxdMFQtePN8hO9X*@72==Dl2BCnMrn-gTaj@3|0Y42S3>eUam~tqGFg zAbRQ95+^TOR&tj!CWqIZiZ^uRtRuA!A_~@}{Bz0bllvSzA8qACU4zdZ0pX%QWZ%hW zjMt0$YtnDB|15rI|H|k3nYr*iP>E0?x2ck!xR6dpwMdQbY5dtLk-fX?KX)BZ zB&@sDxmdn6MMqI`;XzWalC{Au9+?>>kdZ}}&9A=?qbp`HK;%J+CG_o`^8Co2X7 zQY1-=Y8jNIb(ayl8;I;RNtg;f`hBiIw^d?>$XaJZNpLcg9`sNtqnxbr?{)l1yaO`+ z^8W2+{6E>_?_T3hP{{60(MbkqM=G&P4{})k0BN=X&t69Ap8;QALqC26LT>|(yNEEG zqT{FV^1VHqr)v8v7?K2w1DDL)sW5Ok4!QXfI`l72mtb{Q(To<+N~dV06Y;&Z@IqDdO(vL`!|J6XNlsR2llCej zD##EZ9*bXd;FA?Sgdxcz6J+TcvTPBTzUMu743FcU|NaiHzx7sl0h3NgFd~&asAVaf z?pWr-^v))GRlK4p(M_f`C8kq?-kc{ZcB@NpSpa5b6Jau`mi3$MN1&KKz?9@BW0nlp z&|g9rOd;%K`HhJ&OX4R@7QJ}(MpL;q6J`jArrsEtH$p@v^^EOIW-ZyyyuL7WeacHu zW}d$+?`F9fcY?WUKFfDA#dIc6%+29O>F1=YHeMrIgBcTSrKC5({XWdJ3rjY7SS#sg zDnf{oPcncT_rzP+<-5lo=Kln8*@tR*PXJvf$q@%#nKNh^?}GyFLw}0YTSV@T#FLqI zsrUP6_PUY~%6^3M)s(R=>cMh*Oqz8MB6SWTcGm=eZs%Rp`WY&NE?T)Nzl!zV`jWlG zVSem9_AkaZP^|OJf#c`sL=OjQ4jj*c<2i8rq6dy$`fxdY`RTsWFK+qY(t_|Qd zCvciW*jff#_tEKS=r|3ugF5P78TGI(OkD*6&45F{DIivT3mA3SPuC+8ve=FUtXim3 zs<$}4@C=^)Uw~JX##QpTYX3@D!#;08z&6iK;I>l4ts$IB0=p5xvH~<6Lwt!vidt{L zaU(?SK9br9PAfp$(NOoQDElRp!V=0+6=dRSrVF!`A=B6J+vmO=7rydO;A=zJRxBR0 z<)kbuXI0lDfm8m!b!&q1(5imL_rDl}qMrPeSRFn_9b0po!Z_VLcM)%O&f=J@=i$C8 zw#9X4_rPz8l(FX5z^&m{QFHj&Ld&wy)_fSX6hUKxq_qO8)P<+_VA=tyx`tM$qv9}e zfahg78k_fTOWmK*K4#ymIH>)LZV|XwdIZ|y67oGPGTzKBw;kU-n zJF!TtSJMU-_#7XOb7QlO%JaL~PKt6f_A35c+inqO)y9i`x89>?jsL9OH62S{0G)UR z78fD72pkg>;zICLB%;GX-wFEL?xiiNc8>t5Ug z1#TRDU*#`+^xuyE%Pm1$KUft%IPvDq`>?u{y4-W9 z5V~suNH9K>1U0=8Vt*Y&=Uj|BuS8-WlxxZY@|J5KdCi^vzghNY)@i5bIkx@PHJ%;( zF503khrf&VBiu7l1M4 zHGML&{1ib(dEp+sIENc$u!9tSIsnH6*)JnFt2~5HZcJ4FO$k|!6=eDrps`JVQlAM> zeZ2YY@4{Q(@g4!b`8@&fn0-)0Z`-|$KT9${*~|4`2=UvRIJL9!Z+v(xW;^2i;2oE~ zU&hEMo`!rr@V$rdwm-QHw#*%t&~X(zxoDRsv;Jh9`SB{kU=6OmgmHK-M)oVv*Uv`Z zI2&F2ETrz~hzPK{E3n-OJW9_7BZS!!f@888IC}rLS?{g=C-;4>{Z-$={*yq2 z$oP|)XUh1?dEjowf2wJ0)?>3zQ{xZkW+!_5*<1fFUaIMe^5><0sW(8Jte~47fZLv8 z6r78`c`ioI`N%rwAhXUw<{yObP7siVxX1Y?U=^FuN;Rnn2s zoj~tSpiw?K9HSerA+-3p4%?f;OjcobSJ9#jbZ;3^M38WVR(}Z{`u|2$Ga}P>f^NEo zw7G~y|6FADc^EhspxZtVx%qNQ=v53ixr6(@|R4XQ!sen}igSIeSmCQ1g z1;|)s+zG12{sd{Tgvj53WvrtgpO2ne31nLC!}KQ}Kx=$r#Hn1a?UHd(`AU*3QG ze2R9|6XlK|UWM)V@s4-B8;|2jJcdW5|NHKDAH*eZXaAQY?2pj$Ls6n};x*Xx{FUdO zY~X?^tv>neurAzKUiu01IpzG504-U!?Q9VQI0+=vCz^n+l7v?8FTAhC@jk&``VkAj zt^iRpA=_G=(+dfvtBtYbHO14HV{2SUH1m^eV)e;$^Z3H|nDbbLF{n@WW&Do!ft%O? z%siUq-AWkZ$}^dLnPP&IIXzU~#-ReRSzr2EOIZ_^afsNbx3lVhiaGht+p&_uwnUn{ z3qR^14iuQHWy>&QC7sPWv;0gXri{4iFQ+2yZKtV9P79Z2ENjK)RubP#m|Yjpoo$nV zvz6v@F3NN>F<|Fq-t1@R$IfFhZv(q;1Bda@IpcWyARZ3X95|i>$8+HLMGG7ki+0RR zOx-nP&N?Q^Ir!6yK8#B*`Zz9q)0MdF!fSEag;(QUzw-(F=|%5{-;QDF2HHjqO{*-P zzb#)eP8ImH6-+uYe{Thj2~bsoI$CK*Sd@A>!Rg^^@Kw86tMIN1{}!)0;|!MxFkr^+?@4xy(c>l#W;kx(zJO1Gv{}=Cl^Cxh@ITyj`M9?e?M(oT$TY)(FvXbujUvy;P|AzIPwLUUR{(ibAkbq?h-D2&xi20|8N!l;fibV(aW#K$FDqu5B<$2 z@awcB>+PR| zQZjG?(;Fg)ml0>{u$?i&XjMKF`iEJ40g;tQGMRbhOdhy>O z_0B}-ts{xn;W-mw^tpj*KPT({?SA~ZM#i6ayy@>{oF_=MtMUINkH5Q*`wV2IcWkhV zAl?8OuD(4*=B=Y^9Yjw*3%|bhHoWJet8vZyzKY8)`yxL0$2Z_juljT3&OwCE61>^{ z-#Kvg%hCQQjU0(jbRUiW7!}G7=Tmf(HT1PHR*f}`noAgkXP|8@!s#6p1~TW0fyV^J zCA1PIfUcpNu8Z_E$ElvZjP=F=TsV0PKJZ)D;l|7E#MSTk20rqZ&*H*gy%f1evO@wp z&INoXKuN$<0mlSG6`;vEd=w01qZhI$O(dM(Pv!X~a2z{}NX!FR&3+S?{_Zuh=JgjB zejI;w(ck047u|#p{O0HI=NBGA?4KouF?n(suX^oo;Y(lp zDjs>_5j^?S(|GzR;Nc%XhPPkxP9zzVP&qa+Ir55|Fl7&k@UjbTKzD21I-$NAG zN+6X?Ta~CuFqS|w8P)VKZgUPU>|`>5mC6R0yOdLIi4?a2ohe5h@;m}SFA4Z8ss&66 z7SCbEtpJLr@*U-&)%=wEjD=Av2^YeYR>_ma?*xN+eadLF4hsAxBUx<%ltxS9)htRw zRQ^c25Km=FU&r)#rX=(fsk?;8Qxe!lvIH%gqMfPpkX4;}WK*mDcjwQa8Yfsz#)QK0 zy*>Z{AOJ~3K~z9A;~O$Q0Tbeu#jDs3*ASBRo=#913{dY4L@t{Q@W|Vcef>BSuW7~f zUX=JM@9BZWO3Rj@<@7fulRAeNccosbeI^2bt~j*&KKxk6`uTqI7$3b29L7WE!12)o zR5YB!ftmxybKrOm9M6H{U>UKshQ50cuRHr;?DPUC# zRnI~>P?EXgt?V|?cG@D*OJIwv-#YzX`M0m38qj}Oc@Q_fCOS?XO|OQMQvn%^o;ARm z&wDq%bM+&*=h{c{&0Btg55Mbjn6?Sc($F?6`0XJ+cg0=!_6?8Y-a}90|6O%EI*uj4 zaXl!Z9#&BE>Zsc-*nWs{d^)~<%lB}{jX%b>KJzHP^y&Y^t6%>HDN8k}pz2qo?kp=k zW~)B87Rpf#`xANwmrxDMs0C!#k|Eka%d4X9*H9LzV8)4COX5`{^VDN~TWI){A#Or* zI?xywVV2h9n$@r-WgxIg#wpL)2o})rD{>7UFXs#zVBFifUMuYc&{O-lMRF zvRi^4nXvT~UTKQguly-)eBbT3amS&D@X!aPy=Pft84=T7E&VILBll;v#_P^5}T7BJpT)aXY|Hn5bZj!M8b zwYT`Ia#60FUe|3^AA-J&YfzGFv8|GQD`f~8s08&{EE==>&_vlA z1V)bc{?SM9@%J3U-~9E*QDsb{s@$u(Q$xq;K&PBF8TC#B4a&8%t+@-R+htgugHd!k zZoB0n+;->@-1Vs^@UNe}53hRd>(Gqa*zcE6k4k9y3$y3fM%`tSg)w`68CNpO6*HV| zxaP6FE0%KAH>BBq&Y{l9bN}MpjMv?~zIwSA&~gS`7RPSZiS<&OVg{-6ZkC(b&MIyy zW<}Y_uHweB@^}?PmQxsKESoiCzVMq@t6C?g?-kQj1<;?0%Hbj=rfN^GQlW|edVD~<8xp620s6< zx5FYBAZ?;(5A^1xG}<#+C!298+R1j;SM5Z_MBQ61CQt&6a%gVc#pRzoB!DsF@YvJ9 zM=!q~PS_XrT|i0HNM-SJ(L{NA(JPDoS4v`ysvcgbWa8X#6>hjD?afG)w<62W*eO;4 z{lX|;r_Q0i=K>BT)%wCPQb}Ko-)xh1tJ)TS+s3e`3Z;7U^fq56aMW za>~>Vg{?%%!8|&MxBS`%@$GA$z+KlqhC4s?GyL6Uw;&G=BJtM|+XpaqUWLzp{65@$ z{bRWM(9iLit8P=*-qIF$Us)&02eM9N$&d}kcB1aNVsfz##r@@e{G0VQ(}9$`*^klA z^BId{*6+>Qf{IUHo%ir@o7Z@*>$4oZKCfla`;QB^GoL#?e|*;Y8_UJ`%RLc>9DV+% z8Zc4bk;1}I_oe8^RsGfCDHjCA&n{K%n$m|{Je9h}6FE1Z7hXs1H(4RG=R%c_3?s6Q zHosS~QjF&!hCJWq_KW>=am?mEYJZlGtQ~XKkE#7O&&A)5+_s*Jec#q&5BKA;_`5lN zY#uY~Tjf5ooQjcT3Tu|qnFEmwCUEMdBV2jtR($?nZ&UrJ`1~rX0eX~^naczbk9`>1 zO_0M5Gko`l4@p1A{{HbT+oMkdKcDr1U%Ty_xbWgPix;9QThSjWCX(EPS({YjG{;~1 zFETc&+|vIS<0@4}X6!^2gQJh5n{i;jt&~WlZG}p5n{}eMFqvy%cM+?nze4IzjAg9P z@4w}pV$VRCeEJfr@nU}kXY_kp=470kF(!PM@CZQdru~sA z&-NEmQDUadB<*HN*r>uDm+@E488}B}?0K)Ogz~?vzj_ytdm`hB>Q9S4rIcG5?OKG>8Po+)n7?tnhwOdw*v zq*Ga;KgGGPejV=o)_r*DY2b+`fS)}42!8ea3lO9|aO^Ss6}U{8O>?+$AC&!8tYY=P z;EQZF*}aRB*ejmK5m~D1B8jZ#sUrc0$vhTmYhefTIaD6N0-*Dqn$pxEW1!f&eD7l$ zl=nY*r9`gOF4WXU2v1itmA}3*W<(oHzk~UFM<6 zKU$>3y>*!b%UG~wPO-CbY;(+l|57gq z&mJ&la21t59Hdc;^nc;D zTYiAMuKyqW*ELV$zdi|k@!H!2q^%MRBM|23&>RPvT}RtJ4O-Af$#T$eTr>^EiqsNa zB%K?UhML<#Eu@_9esui(&_W&6AjC`D4qg%v46VVi8!)U6bk{}QF;KFas5>Uwu8*4C z5%1PY+`)cdL)-75-7dq9+o-q%qzR&hc+2_k#&@rL9N#+x+;PJr_{`N`MyFke>)7H` z{JZD=Iqtanr?~Gr;M>;%pS|)n)QpY*s?D$r0&;rbplNr|whdUV7@zv+SMc@ge~3G7 z{V8s`?n^KSKFVPOWv`B=$OShAFkbLkClfWVgHl|_uOu%+Dc_GOeS@7k+Mx%6?xtHt z*;zm-YvW~Z0~N20YS0po+A_h>z(turpjQ?d?T&83^*uE8I@&=~0NHX>!F~Z{H3jCo zB{ag9py@9_^IB*H21-r~wLr;m8=;1lSw%Cb38R{Tr{-HIc}?t#Yj~-@U%*7SIlx-| z_4v1s{0Mh{@(FzF#)om|t>42}uKyPP?Z$iY?L&{^{(l7Sx#3Ct{be`8bR1NJ)6fVz zsOcfvW{9@lM%}KUmDf=7_Mv5$#6!9qwNMUfXa;q3d>3seLB$}@s{quBqoL**ARxIv zZ{vUG3wUX=AN8<`s?G9Jc3guV)v)01M>%PsL=Zfv$T_Mm!CMPVVAz+m@xRj&7P5+f z=~eN&_EEQ+(EKt=P6^dS$3CaIr}$Sx4RyDTc3_GJal^Gyb9w)&XvO=`^2;!M9d(23 zZXG4NfqL3OHEyC3)lu4J8FrB zebGMD;|dyP4VGa+H!QSF0>>IUk&3$!@cF1gbLw!~8E*cYuj7sz9>bkCKaN9Jeje7q zLM>^a!zEz=aesPX zdEG?;nT)s#CnNJ}44?AMTr`TA@x2O@08+w?alZI*A#*5;#9pERW0zj75&fmOc&J_u z1A?1eY^6k5MAp#~{EUk)%pM;gp#Pc3V)j8#V0sA$`LZy*ST-`h6llBxJ>Y`(61;>9 zAwvX-^3=5{nzMMwW8D9}@5^HE<4-?*m}N`WPyr&RZVFr?0yW@n}sJym=oC`qa@6l~UE=6k)W2Bv?b-UdPmUHNNn{@8G^`p2k;h zykE-GNe3WU;smQm!ZYDJ>vDZRBzt{~onGJOa}X`VOH_N{0y@1a+d&sbe;{p&jAqJY z3s@cxA?!jaJD)8s9Ip!7s@+XsWg`Rilk3yFnb$XZJv7r$`pP^$T^7=V zY#Dk)5WX+vkHZbA3$N2o)P3Qip>M4tidSL9J!!K|dI8F!@)Wi+gcq=WOE9t}w5W7I ze`tE_QeHS@Qos~pOgV7H+Gu78D4ojoKp>y->hM0LJ^9-vOE6xO*a-udpckK!SrU@^ z+yn)mIo=fS*Zf^2E%8&l29qV2vS>V=CBP7bRQtIZ@5dPIA8pE&li`vt3%JI`3Y(xa zpNCBOwzL29nOqU{5E3l+x>8p%S>-;-v%e_d6_q0BDb6v^7vFc~LC>)#q=amc;+_Y- zFMZd~p9Dhb=exob4szN7^yFDE=x0dQZ90-fnFoIGBRo1w79!9}wl24y&&B{ME&L^;y13p*Xk#=egA;InSgUozW(6)uAo6$CDm!8&x0 z^3nyPSpndbmKG^wj*a5cJO#%#KPAvM1^aP6*96Y_3{akw3~I`me(T#0$auzbfAu}Q5V;X;o_|cn_1X)PHNbtJ1iocY9Km#>I#ZDZ)7wAeyx5X*DwMyt{`Eq8!~Y6u*EhEtiBv;TK{6Fi^&zd^uN zo@@iM$+I8?VBaMBM>Ge!640>{F5@w8Npa zKb5Su2)>8#eWnv+@CKaUs`oBCn?Pq0C2dN=B^l>syl-z{V!Rrk`{2F!&b3eD%d`2P zPT$vL8NRoQEa3caMdD8|cPH_`^F|f_)Hnm4!};V?#s$ukIZtS%DO$Y*W=w041JcGh z2j-m0wU>~j8!$P?2$y#mfA#!J;>}D#TE=`n6HK0=e{#N#-r||K{VtU}qx!$E!HrVs zmjhb4Fqtdw!lzU>J&B_gP{qLGyQVuu8Z9FWhDd}F%lEW(Sn)Y9;&plNk^7L2pfh<> zJddZ)69Szp;f4{r&Y#z6H+~#dyr)^;CJo9B09WN<$ z)Vt!AB*vYd^HkY?4jfBn?Xy2LbKrOm92XCB(NQoXY~L>d`(zaLXfL)q>_VP;r~6 z#+EQc^?=}910|yh!*gI*KJ2iIa>qu;vC*-bBDYJ>u9};u=DM&c8&*x2lT`x2VGWI_ zj*e@g-E!f389J7Ox@DkDP}j4dTRJNGj7{ol&@CT-{OZ5J_pf>y_x$720+#*j&Hstl z{p#!BX$hjnGXCN1pT>7?dP)`qzH+cl;iF;NXgVfLJw(tP z;16GODK37^J7GE@7J~gKhh;PZ0+mg)!lsnJ>S^e>Hd@UN?111mWrQ`DmI>QVptWN7 zRtVo}Lk~5))TK{!2O4FA3AFY!EcgV`b+jWL4YQ1n*@0ySFf+{s^}1q8E?wmlu5u-@g1&+bW$$4t=HQkbR(O|PI6H?a_uu-`uob;>jQC22dR!}uF8!T@ErE$nC4FyWao z+N}tV?Za|g(4!XiIVCiA9sf2|{OizM12wY_Jtm`B1KU9}s-y0nhPKs&rTg%$9FCDf zHxks^jxgFAK?zmAjEdJ1(6!}TXgaol@18bBRz4kH`S1g{|C3MR-#`5Ts*Z(bY)Kn6 zEDw6q72w`T3;}qnRs}6vN6T=~G!2QJWkm4nL;T?dm*EdCxD=M{qa3_c{E*3zCNsR@ zl|TTu6uGF-8#(Bp9kx)h%P;~19Rk#Df~J$A(XpgWQVRQJb{W;Awx{^lBjqLCa4oes zx&=5EX3Q@D#~jPJ&Af?{Av#=~=Hfi%w+;iUi!cLb;t!w)DfBQy(BndVUt}to6&w?+ z^v5cDo7uu#fc3g?!=A87xd6|FXpy(|C>f!#1+mN6MzD`-0eZB}H_+s~MV zPank`Txd@FD=^$Hd;%G6SFS^VKo-IY`XnnNStejcL;#me&>sAR(!Odx0Sjl|zZ}Ep zt>=+3LD^Ay4O8|vBLG0j(HxZMl>PEGm_At$s|d4$Xxl@?{k56bV-F4&rN!Hlfb|%* zm*KyD@IzVXedwnTmc{fKV263jAS5y zG9|zHtdmWXF!8`%0~aM5ekSai)Hx_jD&Jf|=AD7SqBQsfohX&Qs9>A#==uSfk^{7| z1mvE4m$d-iw9XYNN9G~+mdPPQEZ(OExC8D}%-iuwb;D~==6 zS6Hjz0BLUtEt}5~pT9Ap-YN_?N0@R97>Flg)SV#gGO492Nl9e(a~z?!CdV)<7{Op- z5uXEs&1}nL?^pfjV^6`4dx-Nf3@?>_nTieyj83CUT_R zbr2w;8UxGZ6il0ZB>g|@pAI(Q5FDe_B$Kh&?~;W{K!-kp=?I3Gh_5pHRn{r)Q;npM z^i&Kljvq|!VjRd8R*DoHZz6bC9%MZ&!xf2W#bvMv_GIotQ<_#?o~9@0&l$V_vJp)io#DqxhfUuwy zL!Ycc#ds{#LKLG-z2`DkhL`WHC*@`@51lA{@wV)*L(nf z@@pT(H~#)7c;G7Fi`U#M$x>9Aurk({$r~zxYd7PMNf56)0TYy3-7fTKh$vnM$1DEE z{(^ld`zp7yiZnPAVYC4zc@V6l6+|AN0n1sEWCOPQos2*CT@7&jx%Kk@knvXw^Bs=A zYW#Qd0gO16V_iuGJ6%~b#~koWDrd+#~60p*=@PRa>H4z|Gw=YUh$ zJ}%qX#+hJ~a}c1g6%jcH34u_~h-jh^D1rFhHs%s`ooZF575JTVUaQN8-LPTrHP;$z zt~ur!ee~W12zDtjF=(Ahs(UtW^8|td;@LFzqfzpfQai{*=Sul+S^uR`O!i+gcVw^S zw@l%OdHFPy^wPO=n4^Vf`SG)SX63$1$T*!P9~9A6)=z8E03GE+eD?M-rGz7jBre%k zOT&ae8^1K*S<~^k@^IN?|27MkeF6dhBn4QC>WW#KBm)98Kl=PrjTDl{r1>y84};V~ zLpiFU$Zk)^@0^6sGKDnnGx#leM8t2{E)BF97(OaxUCb^zUG07kn9Qk=BfPf&Q=xwj^E{q$oC+pvR2JYSC&38XgK)f_BNfVDW!-|T5Lm0-s@mPdK8^~2_a=;xR)n+E! zWg}v-5D*_|PaclQ1kApmYEV|b#Y&YxdpDMbI>ja*W%bHBrKu#yzmJ|{AQh3QHU=Z= zquAqKX~LR>%{NIYQ%N4S`{jCrSOQ^8vPYE$Gr8sm4EFydrtDv;wEVqwUB2`8{_X#p zzkjeV)W-R1g5Cr2&7+K^UEjYy{-Ye{@o)aK=fLCR|1;0A2Y8S)BR&R>|7t;f44nS{ za`9lI{IdXus<{3bIRAGtSAV_G{WH)1zrNOo{eLNZNyoa)AI2a+qs5BF;>BqU<4?@w zje|0`Ztm`RE?pXBJ1WYzNHj^KR zK7uJR4YMH|t3C~%J%u=Zf|A{h^TsRYt}s@OJ_}>4V=;u2h;clD#jiu3q{pjI#*>(W zGa-cA62V|_VlaC!+58yYE|OdZlI#Y|CMWY>Si+?uXwAP%&Gwr#?zzT>9}BTQkwPT# zSq^RJqOI&Ur`BBIY;>5t#U~Va7vm8Yv;l+7gUOtV*`7jz$%)Mpz@8Yyt53yiN=0YY z5f^xZc<)#Y9svpyN%V@8uLgQslFWpX!UPh-1dPeJNq)RB5kd*MxOAE5O(BwO9t? zB%ZX7CB_wl&XY)-0NPF)HnRs`qFi%2_M{2eblJEa8N?V8iT8~o);|W3{1ypwy)_nV zl97-;gjW~BZHnNG^ASl*A()trCviO1xBw2L1CvdF=dtJ`iNpoQlIR|T$rgt#$wW{W z#w#DQ#NS79LNflCRMj|Oi}hnkFk`aliMPdJb{QWj{)W-`$J*mbvgj~-48+-&&FBE?eIhYbB;7*u^ z=gDatUEag_VrbZTP2JN(y&s1q1E(nyyFLY@%|?v#NfLc>4-+xj#D_df&0)RSht=%I zq4VI?2k@Jv`9c`aPY~}Ki_v9K%^wLa6Y&8P|JxZuj58LKE0H8iykgv&O%5yxUL3}3 z%=!t~3<3lsqqi7I^2ajP^Dm?EANxr0*LiiS`69t#)jId3urT^EuDz%U3@2m?qJaistxGF^57O_EjNSzdnKVF7>x>i>P=3|!u1eBy;DEZ49|2TRdf zirwkRze{JnfUqLPC~K+|R+SW)cut8&lz__u)`{e&KNoj652G^~Z$#K+A@y5tBvS#< z7DrHd|H0S_k5cuwUHM>tnqze7nY{?3?ojh>G~ z*i!yv?0$`jY<5JjI!D>ms^}ujC}BLLsOuLWF!7G+I(JcHE8PdXD}YOwmtFx+B<`t- z?I8Kf5tc%>$bgBo1V}64X2P0^k7LAiS0}R zp2_&lQ`B#gT{G|`O(e}aTOBLQ{v2WFG{sD_h11Xl{OAKgys6_A*z9*tRzRPCeIe@% zQVg?6)y*Nx_5u!jj$%AnQ#8hcC7ev6&x<3Jp}Zf%-l_Pk6G^tuAgr6t(Y0skF1}0i z?pvx@=ygmY+5I#j=PbOADXQ7ykBn~%rYn}PFsKDIl)|Afhy-jEP|0J-BdD8B#6D9o zQ-z%>3={zrRooPVF&leO7>+6UEtARCze=q!QL>w4)b z8Rp3LE_GhQLY4Cu2}EI{Nf9;dpN`j-N6;~aWXDYWx~W9Wv+x-N*q)?{lvZICMKlJa zu-g5GDP&pZl5Cno*!C=GmgmUQzd&;0EKHew+A0=BWk>nPVQCc*IS_CX$EY3B(J47R*xX zLdcU#sxyzEVFJnK8KhgEC8D2Bnr#+7vj8kpFbXqMNh)Sz4ke>c6ZL}>?fCX7glyAE zcFZDTo2AWr(@R85v$S#j<2AN%Kw}<0?7sQI_Z9PTl<_OgOtL1q1G0bg{kwHa7__qP zI>j2mn}RoxhFjJ;rIeA4BRF0Guij7=W``e#H>@!;RRfZ+*92gdrk`AO-{d$dzX?NG zfP#=HU&wl@{Z1NRoZbN4=P#-Cb9e-7PCrg>1iP^3M?H$AG0GX#)^71S_J&e9cj=1O zICN(aZ&;Y48i1)9lRRObf9ZAl?hJ5Wn8^3x&kGi73|yDU)k`Br3SN;Q_Gk(iqBvpk zh7_~h9hRo5RMnv68D+~$la0hvH3BL2n&h_>eBm_glC%90yt3B0q);x$OjdDF>%BCZ zxzjYjS(HS=0$N6t%7shhMBRS;VObM`TK*6D;^&ZNnoX+tX~O2I1YDC;o;KU1*+ka=y#M0$e}Y>7C2zYqZgWilbdGy=M(Nj5!4s$~wz)|n)`r($zuVvb~@R|-0hF#mko{1ZuE8Q-0j z!#JlGizrDr($xMWVwpxrKZBs|X;O`I$S}_#%{-G7*HmRGVveMvlZLxMF0Sx+^&TK= zvox-H-C3l1r;}`$rS?PFNw1R{_Y#@bm(;#8Ak1B9m<)zV^!afHvs6Q3#4{bAb&}d| z$-e8z+8(-#M`+pIr|tio6G`^WB;uTbG_*!*ho2{hBHLC*EWG&|<4c4M^;7h^i z7hmKw;!+}vO$#djU1`?yc%(5}*#DVA)K)riIH!W!eFcKdlcOGemV+DNQh#~4p6G2&G#Ub-SXY}b=y(>z@T zV6=Kn*z|VXx-@2GzDxd3``Ekg3`YyjvVU7QU^r^ zF#owFT-XksMNqn=S-k-6Ti-xr+-zP=Ur1~D5FO=x?EkTup6DREijOeXJC^uBobv0n z>LYyh&IZcX)=|8&n#z@D*uA`wiY2Eg{`oLJe7l4M?^xnoPY`b%t0b78vc{tKnv~zH zKPgPnS3B6X=mgup-otF)Tm1CKT6V2!;OO>C9NyAF>CgL^l{*)c)q~DzLGLs%))m8; zK#XE++ihN4aUq^heT#x0_fxU*6ekKhIJ~KjeJkr(_I??T#7uO`BREO1XX71l#M<=O z?P<(=;U`MgoTg|+4S#xeA@iSKO2rQ~oLqmIed{|Z-B8PC?=Hci5282e(fi_v5&3YJ z313VGjcb3Uv-B3nE1EG{9K^U|iSx%0XCH^fU}fGjAJe$Ahx%P-$zNNj zW5a0hVlieiCFgzef7;LC)%DcopXbQt3sfwt=ll0I;xhR#*vAp;9LJN61axjAW|NU% zVua!ayV<#_(ojjZ=(ko5e~n zA%()Pc2l+ZBwH8kV~+1_mb|u(y(=3yy6qwd^E;G3(#+gfv6utsE%rYYfAJlT5zk+5 ztcs7>ZCA&&>(ZH;{5l0c9cKTUR%*7M|E8x$}bL(?wN{1Ji;yGFgoiu3_=wHH`H7F&RDB%9Ui@~Lm? z;A-z3I?rEX*TKVl{`Gf!{p~_pThDU(bPG#YZNcPCSAdg%sR9`9K5&$}rgrunsnLqG z!rB>SHp)x;7vCajkcl%h&p; zKHWxsQKiOcRt4R3Yyl}4WGFV4fX(0hxP+$mZcd(Q;+r2A^X;oO%(QTc9a8``OB?qc=E0(=6fi$u0aS~@j`w6LIe9XQ6x`W8+% zck$Ajf6_YNooQI?0WMs=rV1Rl2M4h@0*LITFfRp=l_Gw$yqpp7zPt-7)^Ah5u~`cL z!d~#CDbT>}PvfOm=dt&2HO-ypxpHz84Qxy;`?BoZVcL7Z*cO6pq z>~wt#r|KHH-9Mzp8Wsx7R>!Gf(lls;kDi2hTj-URJ<4+qMUm6PJldkTg zfQ?eP70}|b_fI5boI(D#M=4lR zL;lhuOpCn3!uPgN^vfX*6gG3Xpp)p&CwY0wXE+U+cpMtTRx##0$@oKgxb5SW1meuR zw<%nFjH4^hadOiYPHeqM{wXOo6{EHb>^vS7r9|!-U#EtXPM0RA8e*{^-1;Fy|$S> zE85ukOA|YmpJCBA+Z4N33WXxct)!wwX45tSf8ta&eYcnLWwlf;JI($zZ8UDU#kE3c z+R{hWs(LDx)lt5rj$NyoDP47nC5yKyxnq(4e5CPR+P>E;MKO0Oq2M$E)~ST7(|I=U zLyDJFb8uA`)!Qy|NMgUdnq{Ar60yuup2H$DZtw(fgd(`yDWn=^Q8vGxy+5`S{koc0 z!}D1=uY$cR+c~u50{hl<5dE>5=O%oDO8{tpo|3y3iNT;fk8fVw$a&d+6hpN}pjgcXU(`^wsExuOPV++W3s$|m zL*0u*+q&4dsgt55wal6LF+u+%1%8Npv(cXXP~+!}jxUWUBFAiUq~P^UP=#4(SP1B+ z@$I}#RIRS($d+ynZ|5Je<}u%+73>@=blKw zIEfcCKcaMT4SQF$aeU)NYBpcu;JP+`{?j%EO4#k1%%|5kQ58h>z94STcs_b%F}v0` zQufO!K6_ym3trnu>G!qNZNAR2jpx|2sflm@v;ntKVoQ5q-3n6sY#@T zL+swWUjfG=Wxc7WQh{lT{U*}GN@_dI+}G!EP|m;S5*Kdt(Q~zz(=DCk7nS4orz!wl zDN_hD_5nLtsb8dEw0l^&ZWB%IUG!YK!qr=M`1R%;dM;n*Xl(-@eg3TmK1xB{ok}n* z53^!7r?Pb27EZNvP+i|dV_OIH&8;*vw^85JPFdw%)l?=xmneyd<-lL(pEFGz{H^@+ zd;EUSKRfqT|Do}XF3ne5n#QDYN!hRQkLBLqK9s!Qsz6fu;{hr!W2;;c>!ZpP>mWBtTTfpEv z1+oZOF2J$HpN%n)gFcX}d@V(WS#quBlPU7lA}ySb&mF;~*!&tx`5%4f{mbtOL(nSr zC5pkHqjHejB>>?RQgyRw%|fVKbnz(`ytavtX8+8-O-+=oKEa>nEmZl( z?R&KRv!}RTOvoyMyd};r+>;H!v)%q{*6sAZ-HSEd#OtC!8(T(S5iuTb| zev7GTZ)&ppkqiu;u*xsb|5E)&+ly|Y@31IJrS#TtgHDa%=d?%e{Q#^!(<_haeYK&k40T( z5}~9iyq^69#Xp^3&-!+b$-DE09`>(jW$C*G$ovweIZ<0s3|aAocc$PAjK}N9Az+%w zGimP;U0BWj6Lh%j;b|hIKHKaeVdwD|I<$1oBbo7zqE?oYnv!vS<5HyFIMpv>F!5~ zzsoJQE?Rw217>AC&(Q2x6fi9IE2+Hk&V%*u9^Dr%Ytr8$RV~dg!a7!pCDLq@tIoq4 zNFm@%;;oE-;owI$&VHLrI z($EwPD;D-&uBVUx`|sD`AIIi#Y#z^n|9{Pa2Ydg=!13Pz5FZ1F|GdEAW8nP%Zs7cn zCL&`Xoup7QNg+20UL6UZ1d^OdSZ#Kk`XDE^cGFpMp97`M#JP-&bte!fKD(|sbRsWo zOTuKgVD#unaE!rfwBs3*$-*~^sV}@sQ{e#ZBKuT%ht`Tabe0cue#boyZLH<#+?iM{ zCQMEvCh;sb8E_4)!Qut$ zHIG?`0jti=f)|$2vkfJ+{L7!JXsozRbIEPq&REKt_iAV>yGKLOWh%d_rn~44d&-XU zub!uf4<=x=T5-mul0W|t?PYgqD;c4?5YBGXe6G&!fWo!A6-!sVnuV#W_lexI1%uay zco7F;GHKdyjh=!LItqr_@%8u>!MB!QH&0T=rEO9sTlrN4a zy>TR%t@x6%`2NMUwCCSZGEZGq&|KP2dwCxn#Ur#A!j9D~WM@3B{Bup7r$}->h1q4n znUu~quWX~e5}Km-Dg3IAmgoRoQB59GuBmO;5RGM5sr==D^3qN4#q)&cN#Z>^ys@dA z+j@)c=rs;(uf}L}qP&MadgV)Pc33eP%zX60r+oa;pLzB5xmX-=m_6gr*~eftcyNr( zVLD5P=_d3osV24cO!e0HjHNsgI7(}tUL zY`sVG_F;DZ*hE)(KW!Dm)Ro?)wy>YJsu60US9s_7AJCbD#CshWJaz?mhR5d7wEi~T z1<+bBLe+N-T-Z58Q^{@W%7&;f9j0~1J&qUt%DXQv#9#_&@%P6*RQ#((~5Cmy3G4udm6$*Jo-I^ygxSd1Rr zV{-ZF&0-pghG;InO;<%PZDqG;jNYNU0>xA~wxy2AY2r(28;!s7q2hmRH2y}DU-=~` zIL)JsV*!q}u4gGCD!D9S9D9Y)CenbyHh$o%q_J#-$>opaV0DI+e6-6I=E#v+hKDsT z#(}%Pd-V$ArJ=EvVWCRhhr*Iws@NjFufo=~y8Ya^J)k@uulEhAqPYM9;%BRMZl|(n z`8owC%CRbUA5qLDDMBl6MX$!PTDp1z@_X^MlFz_h7`penPjGWb4^KZgS4mNdCzeP} zir28iA7SOXjjG5h=e_^%F?B8vd?uw(FUOOyYfjfIR+7dzmg0LF>+_41r=@(St@{!J z;7T#tqgXZRcmf$b|MFXkDJI{!c6*rA-07+_-Y1|i5r;MfyL zVcDwHzX8X)57j8(Rw-8~*=&(eO=0cE0&Wfb9(VC59u%+{)b(7wF~EBte6GB`6kA%! z6sK_R(oJ=30!#~|ZTRkehVPE3@ucW1=Pdr4!=rwijh&ZA$Jacg-NA<&UorOr0)FKv zKvELInqJzA?o(fM zo1z8PD))8l>Z5VTH7y604s&Mv6~3Iij8w;TCBf_SPf$|7Vb?U04KrEz;Wp|juF)PH ze)0?PbFp-h7tnY4Z>UvUs*>GJ_d}oioTue3`>Poa5qVlv+Vs`2d|e zhG>cQ(^+(vbEVK&-piloEyEj}fWaG5j1ghu3M+N%g2S{_+~G{gO`41E&>6j}R7ct? z1}R>994S&u!8hohqCh^EBbTscCVA%96~m;xY>2kf0a}ahaA`Z-*aTNa8Z$aXN6{b` zE1L>WId%Io1E}0i{CBcOv!axi~6}x zD*J&E&J^6>{BCG18BhS??6i+?IdZUuH90=FJ(X>S}TU=s=UX^;;Ve{%2GV8DT+1X^-Lfrp!?i0Du~wr03ZNKL_t(7bZv#M^7}0N zw33F3n>0sz`5<=@n?5>@cNYjQvI=4eh(S6Fksi&>9 zpZe14)bG4W=vg&>*gIV@d_@}CVaXw4ok^PR zIilazsMt&Fx+;fhjozkH^4Hcr3O+tYSM)A2FBi|xSKj5;pvb+a;r8U=H&5f6w>D8% zc!So`VY+4iQ#eFN^e*kuep*X!vU|;GW{-c5fMc>6U%aiQ=|e#4g>MzmRdH91U-VVA z=1p06M=_t;i-+i_9HJ?DOBJxC$w2nd0w6ugURnwdD*uR#vP+8B(f|?U(9s%Xy?U_T zJy^GU@7!bOo>!{?R-`3BL%L+ghrmI(cFYR6~ zUb(K;;9LDeO0r)RFytd1$yS#iZ!khpv{W5a(rD%WW138MKxFnsDmlcujoZ0BbYHuE zS+hq!Dv#ZV&a;pTRD|ivZ!d}V)W->@K4l0 zV(~2%MHR0+qnbjC$d`M(8iQVYZhx=wX#f9@6)1%jJ7JbwFRGOf>Z zYV%dj7L91Yv0_7~DxlNt%|R3%#FtgTZ0!~W)5rqqfEfbGtypqsObv#3s=_Lxj zI!0UZfLb%fQ@68xkh2x{Xv)8(*3b6B0R?XU@^w_@A7M^DO8(hXP_O0R(8J|l0a+#g zO1z9=v2;nrZ<$DL(#tfjAEawLv=!Z@VnMyi`%Teans(e$zR{gK?^08Ii4W)ggvXIR zy8esO#((Ad->24pQNR$T7h5O|mn}!xYgDhjL|5SmZH0p@dcQz5#kmFCQir4OV0_I74_xfsDqgRpZZryp0hG;*Hg||6ZewVW)ci6YCjTyNg zVR7b+&OcENc=-9}c8R6Pc#TCGmfVv?*f@nBUfn=jQJ>0l&Aa+JQ+kVrqCU=*-=`-! z%)vEnJd^Vtc87Rri_CVm`mNtNS*e;V`k;W*#aC%6>!W?g0G(BXw3XkXz48tX(d#sn z-sJshi%7B0QqtrR^Gqh0-sI?-b7=d&L6v_W?EkJ5K}X3D?L|Xs|JNaTxT>F`{KG2# z)`yM1_xGj^*}q8cm;FbYX%4w@uTlEVDfN8F{-dXSMB4-G?xV4^m%WSIXxl!d>Jh&z zC{pn^1;r}mA>!|O@ce7Qu~uVGQw=2YJ|_*(vYrYEEAOSEp42})Ji7kfR87tzx9@JVBeCl8NjGEUDF9G)o%AnbJ~;}eyTQ3VIBc}G)v zkoRw4A}h-f*97H#YzSmy3Z^RS1ZkvsnDzGYcmJ`x^f*qBVFQ8-~Zcz^9RrPqyA|YNigxCObrq5N+Qk?M}jR$fuJ6JnCdOv zbVuP(w3S4MMftHNI1MB^5)|m9vnMH;=y+!wdVdmzI4AQaeMjB4+q9OTK-22&byTj} zL)F^Dv{m%dUNA&s(G4nAAI4&Ik>HFc&hZ3#dn}H`ARkTr`8S}mz3LW+qNgZcyMvOI zyJ)ZMqb>hF7o*T#c$*(TUroHrfX<=Au5<9++*JzbZ7;pgm#=N4w&*I|6(g*hcZ@?z zdufgiuyg4Ns=lqJv*0#+OKXYo>lh=9TU#O?tB<9ht>eJflhl+pbE@zxwHq&}m!6jL zJ481hK<_pykTAiVNRmrOj5D4jccSuK_B@qF^)DAxp})Q4E@#SqW%rh93RdlB+p@!) zEV@j0`LKGSJhZ(Xt1+Nr9_xuE-lfN3N#VtePifkIi>~6kwCwDsVC^wBt=z$erBTl8 zxWegzJ2X`dvUT|ZOcoc3o><1%{*^?l4tsnGKfhHEdhB?XSO2vA&pk^P&GQoM2(+m;`oy7&?umBTbfZ*yd8 zH#S2^i@*0F|S}LKj_$H?c8%YZ!E4gZ;JyA(;Z(CEuiJf&Ejn-3J+)c-}J}&0N+42Dn6gG0A zqJyKwEgUOpWq-7e<*W0F^P73n_9RAEEN+XNU;ezF!`m7-QP|3v!X8d-?54A?ov#bs`tIS_bJ=4AA?972Qn~=uWlrb%h!8V z!TEaMfMOuKLn7~-jL8{N9)T;@Z_&Wqkr8$uJfYY!0vt;LU%=a!UY|#A|FHUwD)M(< z;CRg$j@Q(4>(1TZirF=F4GOH5f`JqcMB-XJCO2;{Qs?=g=q<(h(@m`$I95Yzm*)9< zPk_@AShuNA$umpgU6{_vnR)c~YtqW{`?ucxSb-5z(ARu3gA|wVRKT$m;ZM}HD33cS zKvx21n@Vxx~3&Z!kCt_{uHl>A6C3dY)qND49}MgwqWjT)BRmYd7!E+c%)DQNAmm zE7$v!zvi#k?{KYmh)dV|Ia1p?I=*!4Kic>L0Gr$b+UF|vfJl8hydkz07c<0tZVwN$ zW6wcVxb}wgFxXQS<5&v*#y|=>zkn85h)iEf%p4ll_H#B0ZDn_9F6-mimJZ5So}hBw zNlq7CLBOl7(g6;wYbMz_6{9_@Kxn`C>n4ongDDHCE0m($EjlaiQn&F2JAZDba%BT8 z75y|9-cd!sJ!@KtSf5tlt;Lb13Yh`pRDOJUyYisFB5P>0j{}A6Y+rGh;*}><{%I=d zqptiaHG9vKnezJy_9S`q2vIi zK|M(an*9ZVr^R zvSwY`BaWY`#eoa)ZXrOg8eoaNU*|)Y$ zF=S1_5Vl~5pf`t%#5tT^-pj=j=qT%_vGh9oH?|U8agylj)0~dpP{sVF;(nzd;;~Lu z%`8&D4Vk9$)2rL)+&)5U`5<4twSiO7E1atwVdMMzIJW8nZKZ?kT++xqeDcgMbe0d%yj_~@`l&9t$gYhIis95)b(6-jUK&eoa&&7q;lNCi z0?F7y=>%PQq{YwX^r~B&D}&~;5gID`*uS}%(zPcjTyu_89&`K2d{W<+S3Vzr{OhBX8w!o)Ew<8yRY)ap)H+MtvbVwU+TEH3);5y zD=?C#gn=mnVP0$>=53+|azMkMcEj6fLW!@aJQkD!s&+(qC!X zd4u8=$MM=H;t**^0e;;XL@YB|_oqE{lnko^cGJ#VoY{Akotqliv|t|xH+HI`_k$w# z!hhSQv3s8F-Qw^Ww>GyLp=%y|)<{9996Y0H1P9@h6t`_3F(5E?>LDv6HRJFII}h zQh4?SQ@MQQ27|*R3=G}F=JF|)u{)50-5tX24&o1`a`5m`bzI3Y+e&t-yeBpp!AK^r zz4_j6c}}kXO79&`HMh~w(ao*l5taW0+!ofg0OgXq6!0rC45hJZ{bu^_-v2E}UAlII zy2fT|>zlZ8^R~(%0)$E~89(J2+|qOFiOTo3{tm9Fp^CYU)&AtH?!*(uvqSNF#xqpEL&y z3~N&2*KXb6;PD!28(O$CJfdQI_r6?DFDAQJa-xf$ylB{-rNbR-MJ@Jjl?3;+yt2G7r#45v`j8~fb zyaH6nZ<0slyZ;z)A-PK*2qEjQuxB*@I}5jUJZY9^IkEl%-NpA1scQAC{{`b%v1Pqk zkC=b@RQ?HDr{eYJAi$)6&=K3yeDd^4T8oF&J5XDBzgnlOR@YL#xSI1-BXn*X;(Qc3 z%SKo-zgXoTVLm=u{%KJ8S7d%4CI1U!UN7?LflNZqiKNHPrfyX)=gXj_Y?#LKTO8Qb zO4*ufidP@!WbqX`D@GKkcXVqfVf&QP^*`gkc>Nz%Am)#s<}0~f@%;5TCy-)zhHZa7 z%B512x9y2F7kGEZ_XN#Taoff##Y+4GZC+2Dlbp`GYut| zX^vi@W%n&gHk_tl*Bl{#>0xS%FViFz8l~4LU3FZ6iF&UOlP65b zKNWYvWZs_i6Q>Gps69;6&Rz~h+la0?!rpCXsIR(8bJ;E03Wqqev5$$7*VTKQ!R}B!Vc}A2G{Gwo#fhQpApJp3zYhYOA8bd`^+P z7BA#fEXpcC-Yv55Iiyi$ykZ^8aYZ3aQ-k==2YCMD{n+F2AIInM9C$nj9)1oyzylvM zj{gRL{1`a?y&&*lqC5tU9|Olv#hCclai;&PJLY5qC>u{la)d;?jwFX3o!yMdXve2Z z;Z*(wdWzv_aT_|jl>~=gV=UTr7#%ti?Mdi-X2!Z=dCEEtr#X%MuaD4DGE7T(KhdST z$PT7p(Z%C)xOsc(pK01QKt~BQl;2^_thY(@ClKTPpBVjPaq0t9{oKIWvJu*&Hz`}Q z2VcNPqG=pfi=L04|CXlwA;Pw6lVzg(-BvN}f+E~A^To?p({LV0N&;)^#|QL^NO zV(T1U`YTP_p`&V$AKqHQ;pI(=(Ybs3DPkOY;=HlwJ>s()gTrpZW=q0iNx-GIlVZr? zWVDl(y*DY|x{r995xv_%g4wR(5$_pCyz@x}{7W%Rpmz0D#rUi%ze(xlLxdwK=q*<4 zwlJ?u{FsLPD|8eO(7k(zw_o}QlQ)rg`xta)1AhGkHh*@Mmf~SLcMh@qhhmH#Kd~YU zY>Q{|gy*O!y+~tuAI(*L%zkUW@{=UloIGVuBF?Et z?=WE$=CL<{aU#*|v+|TDiTGe5F|M)1nqu(lvnZHfO>1<7CUyUJk{d|Dq)Wu*4Djmo z`J5K^P^sLbn>_dQtI8uc#`QliS;vs9OXuXe3yKwVs_G)qO_ijE{TR(U%w`uaPI{j+ zg;!`P>{CqyZ@=~tF}|mWb^lK+wpdaP+0?ALL|geVr*>SXEPp4d5ie#-A`ZKYmnXkV z-L^|~MDNhEYmj*_{^=3pe_O?W&eN|D?|lM;ZyZLw9&cPW+rO%%RTb@f*}i5UDVcdB z*)5ozPF|St0ky?f=qm1`r{oIX{M+}8^^YYc_!RNZr!e~CNVJb5QLn>ePG$G%TDr<_ zbG*C-quoxNNn;2boI1>26EPM8u?{=&?nDf}C(+xU#Np6mw?bbxrP4THx)g2h0*J&8n*jxo*{p0dW_G-Z+h)d^Zl@6uY)N7>3MGW`KuRwFK} zn|Eg}pfO*%wV`F_9iDmmRTclZM~wg5FMjeHaIC%f{-eOLdaFe~52>_z49Z3mfU~J&MbAAyT8N@F_(>Ux+I=`xNLaf7-oa z#pDqgNCAa~=^`@GQqYypj=hI5yTy-oJmPmK9+O9F>Qr%Dit2k09>MJoVRd*F^8k}T zY`^B!H|BBq+D-NELnHTj_v0^6jN@RI0)EBYa^0qU1-Q!bg=w{D=?dI|h>}kh##i}{ zy{f1z#baTZNzq-3w8A(N@5&?9XVmd;4cuej;S&nX5Eh(#oW2m9=Pv$se%~xyhDD^z z{5hB$sfv*-=O<(L4Gb!<*&+VO{v0J`C+GLwPYc!f0?zM0Qm16u1bCHVu=p~bJl*nJ zd_-DW#(m(~{HIU9;MeQD8kjKxUw*S#6`AE8$aQ$bnaYDxfLn*hPg_?H!n~FuxX8EJ zTtOuV>j`OInIc(k59BDu%cG3{)i(<@Hu2KmQPrz%*W8RD(6!WAbIzY#c zTfFzi0?f$~B^N1imST0uS0@!X*;d(4+0Un#>VKP{bvl06RNjAK5jFY0a&8AoN#d

G6|AbFB1JSW?prt+6F3V3WPy2X}7`|-FY;PFquVofJ#pGNt*TAB{s;Kbf;KK*>L zDjZuQsY>eGVb4}xwDLXW5vzZO9Q~^_mh{oSXOQUT;~1S`yvgGg%OlYjAR)zvAtZj> za{tAvQkXcoggjH(v8E1TDpyxrz~`BW$bgE!t;?08d=ZV_kQ!gSmPP9PA0Jz z5#_ZeKA19o*fx`G-yET(@-{6yZ&R`KG*i9rkZzkp*gBIBo>@fQ_UoMA1@*-@cs)iF+%U8HgMO$yc=R&f(=Y4P5CbLIk$Z8=BJj{Ay1{@$w#F@-afN1(@< zMV9e-YF1yTIeM4cie4(V)G;CR1#Gr7gc<(QgfBU@`8FMecj>Mg;N6!OJlgnG+W3E( zyF>wxM)7?QO zR4le^{H|$)EwkAC-9GBeuF$sUChLAGC*aA$Zp*;yoWQKyd7LOcN6VgmYRj+j^7Efz z@P{#nQ;?<(@l{;$W+lRClIN#-*LmKVw*ZfK0x6+c_>7ZTzhJjw8=WnJJ)4`Af`vr^ ziK1lj`@7=bF2u;c%lEX8G_I=rBhtZM@o7xu%$ZiTKFYdP*VINhO(gt-YW@D|n;#hv zDzXS(j6hN8F7^HqOq-~fuma2;t2w29KXM=TA3BCNl&rjdWet7#jdxUIldR9unDgPM zU#a!mF8g>_2w4|}`Rw$C6*IWI=c2L{7`lrBg&(Y+0@x~slE_>7bMXW<#;ZtVU;g#F zTAy#<8Bk0qS&OYIcZjcUDsQ~~f!5G7+6468CtoViQoNnTL;2v58ZCeH-=nd$o2&^_ zuq#!AWIp@)d!@vo&i^jF|EJH@x-T(&;L&`f`m|bi<@ydDIiZT~!dUf%#KSU#sC7tg z9k>VUw?tL$6Yy1XxX~F>${pU6EUe;d>vd%kbfyq=h6%WXcswEXJz4L?qf@+r|MqkGj};Mx zaV#K|H7pXd8W1AVv0ldnC0ks(bS`QS;B6bG}sj zO0i3j`1veT*}0-cDOQLoN7?ckGQ+d*+j9t+C$eDf8k)ECajqDp1h(k&!ataQ>a@nK zWbrzGl>D25C8PmRUWeqL=crkImFDOWwG}t0$Ui-rf6_q#UL}w0a!thU%u(0>{NxX*DLJb=f2+$bGxvqh z)cg~l--nrhBJVA7=}u2BZtp}K!aR0OAk{LHg0J?|S<**G#ckFv+=<DCu-n)dUuGqrftcQxf z&KFeikBhjJ)Uv@FQA#)|rf1l*s8uy8HkICD$BKHg?JwfDh`;eZ@j37S z4}1(9|BZ$GF>w5QG2p>Oc?=vs296&C$J)Xx8(xdYN1{cC-eE>(wPH57@$1s5-F%L7 z#rHW{+>Fj{B;J`wlEa|9jrC5Wl8F}JHO7;qz)hDijRUJ%>8$LhuJk<5=DmQ)8c(9* zDfF%c>~Z1&Tti1Ow3OcE+fUaJXV;Msh*f^6R$U7F*I%Hute>Xx^GwZoj##US1Wz1# zdkns$GC=nVOHFfSKVN^if}cKEM@MC^lFn?3 z_EQ@@$3*um4y|ZbFAoO_&X8bBB+(bcSo^=A_a+i+j#o_Lcw0PHvlCB33QfCyrD@l7 zwr|))qQgp@)rQd(B++3a&N&91e=IsjEbh2ePOiH~bLkML_FUz~x$h9`iXqk;LxSCa zD?Wvy&kk@dD)NneEc|k{V!Rr>@mK^nj?3Zj%1f$nP*;4JoX~7^7B>ktD@LaaOMHk` zpOkQ}WSDbBLwx_yDkZf%&Lv*CIsyq1R=rk0ckuvi6}Kr`zYn9!PP{b%lii3}=fD%6 z&YAp6T#VkQuB-!-#YBut$5T!#j`%!UJ8ESL001BWNklPQ5J<}6$np-lEBZNI*3GjMpT%sAQJ`qN zM~@*PmHY*#)&1`*8RXkPFXKt^+YOGz?1;w`m%@p)muV>Pr)J+JUV8aWlAL2m6e(<{ z341~+TR%UdObI*>{;2Uw#8&LnnBiB+-$G zBO$`}FAmaEJVHlNKa1zD{X_Bp;iF|Fy5be^Z#9`n)=y&p^0O-Tr+0KQbNchdx|0~| zPhhNDM}pqYx*wyoRbHU0;yjy|7c$OeV2nFP&B0jL6C`!5#vKS#eX33aE`ezIZp9w8ioc#O8Fw&+i&r&Z_tLdn zxW#{Lv^Xr_u`tV)ui5k)Fj}$ukOEA^^HyXtU4c~E&t9are@GR#r_Fd8UqHP00(g93 z^=GuSQWf5%kiC9u6l)+0iz>9HDu2th8}ffUenV@g@|ko81Uv{T_Ll&tzh1qiipzI~ zP=#y7pz@~R^r!O4XWuB9R4J}sx^h!_QhEas1th%q%Im7&E^K3AZwVN#UO>EAO7=`7 zAJS@ zs62L?*rq5yW07?ZdZ&;Td5)*2&SUbVR|%y~R;(|x zKN+Jptk~yDcGI1SQ&>#48DtnJtY%UN($zX20Tk4P{23& ztOA7n0=2nz9TskV<@cdgETj!+Mea` znzM?jTU&aWm#2JAz%~_^6udk+O3Hfc*M~LP#L_{2_#$61rbMpM>&_uXH=C2|Z>sC9 zKYW>2U;b3hdnu5){CR{-(h*Gp@D;#FY{37fE!g;buw#{Px>@^6}wrNK%-+a8D zB_9>hQ8q+Z`CVn1ai;VdGlTDObX|`EFAo&ADSuvJycsynN;($>|fK$+1*3b6l~JSP)U!-okvIb;nS@+le5sfLInL&$k4yQnpgHHiFS#5`G#6V zR#X*KT{(o^(~)yHwehNAm7R#5|3kpB%ypH20vXC9`2Vo?9?Wr_SGMT;5wC8&sykD6 zd~GX`MCjaoI-J}&ArcH?kSK|gEK72dv&SA^%gW3WGnfMig2<61C}xmi&d4B0kN_iy zOn~dv-Up<8>v}vxs9yPv%~Ex>K%o0O=R4oo`#XE>wHC$8wyWJ7zX>3F`v>ns2~tB+ z6f*e(2ckH2_7Y3*Qv@t3t!3t=TZyr)0L96U&KnpW8ROTiIj?ZT6E!_i^N^$mQC_qA zNq*Q;SjIk;ls|}h>sBJ1vmVFKmiFt$v5bP*%vvzXIl{}<44Gh$YX4NWDv*e^s)o=_ zQq^O?@W?2Ze7O=!SFFXNFV|qz`faG&cZAu>q~t;1V^jNeXptNc@JZjT_+x18xXE(a zl;7Td_kHML8k3r;!VtAE_8mHA0HY&gC@QID#S9Wh*fwCmk8RIxkpQB?TmhF}1PWXt~ zO?!q=?zmSy*$$9eNEQS!#4Kn6z&Zb1y}_R&5cV(SA7*<~ybDZxd?l2_z_S)656UoF z+Pj|ES{Unxz?B%UBbvY3y@x=GBa}jS?5YBNo~%6d5P?Z?bavh4&(k@RpEk3_$)7b4 zhaO5`&%Q%E|Fr%JODexn{^R%^h%BkAY)u_>y&WT))ZJ;lU8N*@9St zw7()5SGqrltw%n_%!t#onDwS=NoZ;U8pTE}3Pq3deoP2v!ycN(UdX(EA$i!pQ2Y9i z{*8cNIhqZZ8Zo?{N$OQkLYMOpbH9R9J8rV_(_`S6n7Q6yl5-$8YXsxpDF1x%!A{WL zlRbtbSqO?R;lP$AkUaI3iaxwC^Eb?dl)@S8M;{knMcv{H{OsCMIf~^U7k$J0)5!T( z2u3E9|D{+i1IILvM*evXCpP!+^V``&lz;vJDMbGDLCE18#Jq2y?344jMv5LK1Anvl ze`-AbpD68uC}%?oXF?RLN<->w(Dm8Bp$TnuBe+n0AG0IxgXEtHSPpA*5OKVW&41X7rc#m~e}sj< z+s5n9p@p7m{ZT%p^+)%MTciC}9>U6OWIA8P(flS{C$?=VjZgcBnGngh+dmfp*UQ-S zhkCRY5235#5f=Y;E1Y@+QYeRk)ePsGXxMO#f!A|ocks@<{{^={4j~qSBN*a6=K2M- z+}KUlBimN(g*z655RP&j@{ngd-Mi!azkOS-prvXU$EvR~m`O2K>6yrQ#&H}}FZU~1(3Bj$N0@0%mIMTlPay6-`^+~7y`IhCvpIaW2EN%Ec#H==1CIYf z5Pt?7|0%gWUMSCi<7dF}GvJsW?nx>x${pV>(F2>$4ZBZ*NAN?nByhUm8rq9T&`@!a z<*BW@1!&nBS!h5(}*M}p;XCR3Qq{u2vK|9RiB=}u% zZ1`0@T8c>ubQFs}+k{lz46{URNfm#XzY-ng!1<~>So%pm623fKsOiAPnj1J@eG}zt z_90-2QQZ9Q6avt#xwu$+3m2+xqhw1xtUeD+ zJ}>ODA7)Wt;LxH?Vb5p9mc)?_H*mRj2n~DMSr&Lw-~~(yJ&!ca484hGikUI#>P2#Pry^=q!6qw*2X z?e0KERtC~lJJK`}l6e}=Zn}%My#v@?-T<@P2b1oGNw&jE66U%CDZa@JG^Y98n5+v( z5BM;}XM@A*LC}(gGn+fmR&xstWhdZqIbqYLU{c_Dyr8E-@FwxeyG3Xzd4$f2QLJB5 ziZs>5fGA6aTXN8_@eVH3j9`CVE9{cYjMkLEB=**p>`8p_UMa4U_jK7KtXWWublC}$ zLSU;MnmG>*n|g7+ZVX@5wZbM#Nb$dbl;HC)X=X_F7`}L~7@frc`Tnk6xNCg=Cr@nt z|5ya8Ap0l|WFl_ayRaJpsa_FNp<(2CBoF??Ouid4}KQ`mx3Z3=_;Hj+s9m>>}I zUoye0r!jNbq`Q!&+hOuMU~{O@%rkJgwx7X};!XQt@ws3T-Ehh}%s%7VrmGH^WiMn~ z7ETphM@#i>94k2uA6y|CGX7qWfPl!x_&hO>ptHFv*>_L z6OE2->Y@)$=Z9)4#u}h)0zQq3&$;j@Bnds0JFb;*E(3TRg$Ep0%q4OL#q1boqjH46Cg?j4+3YE zU8}o-VShxAZb)^DssRuaP#y}$p=conjEQwbpa_BJ>o;s-@z1fbQS3d?z|xlF3C83@gHW++OZ3a7fa14B`J^~wq8lVP+T(}hjbRd~GVj7W0q8N-rlQWRxegh}+Z=k7i7$@s` zp!o8bk>w@T2`!3{?^P`SNe%m+UoP*%x|Nl1L_!cE5$^WS7T(6$4fk-Z8fYl&KvyI-mhI-xJHrw9WW(cl<7aiavikv=YWlEe?M3`{{&K`z z^APsVK}?$q*_#8B7Nm0(spzQ@+m58_h`k$S5LRFob=>eBcFxD;sy>$8F5i5dW$*+l z;)jXVM+Hn;8%6<5WB?_bnZ-e24i2nuVK3iPHC?=Zw79|S<7-9hUHfL^3j|2TGT_i6 zQ1o0xov-5J&R(=s53?M0Sf0bjM(0B;p|F287XPFWZRHQy!*Ko5Iyi`R7EC~ubCGnt zg=3p<<3e384(z$iGU_DdP4i32EFs@ZSozZ;bd-*8{#n1c8Z7gvXECGszo)^Xt2S%?tK%;gZ%NHC5zT-^JGV4&g?z;cLBV;U1P?W$$t| z8-8^rXkC7_u9NproG)(o4l}EWJnjiFl>%|Nl@RA2`kAZ(2y`Xjk3EhF#H9Q};8-AzJ9i&& zW7Ey9FzzkP@8f>Kx9|7ir@#C?#{-so z_9wXDeWT|Af9`JY2s2J8ABk#=8MOqguU)_O32?dZ;Bk%%RDh@BBIS^;4xKdop~nnQ z;txJrz-k?oLx|BWQ!b}-p?iKjCR2W*bD@SFng_}el-meEC#huieSSpWcbyx6$kUnr zp5i{`53*XIYs{7jT67$IkD>Kyw~;UZo&3WM3FH3S6B(aaVZ_w;QgekKW2x@P`-fZi z2gaqy6Ks<~4O0Xz)Bb}%aN57rANbl}yy*lsWbzuie)B%h|AT=?2IH3+%Ge{=u&c=5 zuoqx`dJ(Oza8x(JXC#1ZgK!>N+1 zXs+%>!;W^&SGGtH_Cy#$Fv*N$BYqB{qiPsme!OEm|71@n|Lot{#QEPt;NS%Ezd&-f z;Y|2KWHpe3g!@%AZ0^FDs{1&+_Zr7jXT%SCRO9#`^3KDmAMYK1ejCAtMYVsU_}|3w z|7`6os%Q(OL4qGgdy+KF$Bea!|;&fpTe)jSL zDB?7Df>|6t!^#X~xL?Pqo!w}x7{>8!J;-o;4{^^c2n(|ialMRe$2(Z|y>eVH0=f!; z?Vs24`lIHviL5`838(!Fu~)<5EMz%e$N7S8T&)k>zgZ26rM!2i=Y#+~D`Xk>kM|WE*wD)Je6sdW`#%O)St-DR@H34^&4N0v zgR5b>b-YGw7ncUzilM0oZ&HGD=Ivp!LH-8`D@W$+=r_b{n)ys7*@dsn`(Hi%8ooV?0A5dx?vpIc^VFT08?Zcle83& z9IoorvHJZ|bd)_rOT_~${&X$U{AQ#}CWvkoAH1=e{Uy)W+`^L2mm}bdVEw}Fs4F;x z-GxW+*>9G@Z;RvTmJarAK3I4eZm$zIB^A~{8f+m4OEag)c34yaJ|Tc`dJY=Pd(gSN zALSeOKv3MUD0T+Q(|rzDBsWZo1?j$YNahTj*m4(*HN!YmaRnB)50m_9m=sI}fpe-Y zfz|I6;ac$!+A1I5izPeof?q(Yo(ivMhMJbZnSy?_6Qj1Q7xgR7;;VI+@YTjv?9Ok& zp8QLwUwsNUDhF|`=m848EJeCzMT$Qa7Rd_L7RSmT6rrPHgr%HUE!@f=zge-sBH3a0 ziSUY1eD=px*p^?8ZCiIDm>EPG0p_xcLGq4W1Grpz6ZItxaQPIx=ruF2Y!a;?7Peb= z!Q`_d#X~@-of)H6(Fv!=2Y-4tn#fzJ{3ec9or7BtkuI4r$)AEL0Sg?CFg|>9GZ$m7 zRt#ao(lQ3<2~@X>R%n(i9NTycmud!asJ0PKQ9`<&id6p;m=rr?dlFy1SB#F5QT7R4 zy|4%-*^6|=42RDS-8>b?Hr>Oe+7TSAZQ}DKpfx2h879@rYkK*6yUo_FAiz%K|Ox6fMmSIwy5L73eUMC!G z4?L0rm*|7p=RlfbMT%-gy6S?wk)5M@rO=7d10HjtD+}Eg#p)htN`X2OGaE9iM;8Q_cU? z5@Y`J7wiCerl*Snq#yyjS@4qo5=oa)5s_HFO zar|^6a%R59ARYtU{tPJoB(@ipKlzU1r_MlBHORV-mX51W{{7eg_MySvQDTs(<6k1s zIfm;!ceq&oV0f5;2AXsBKvoRq-10SRpLiS}IC>TY`cl!IKwV}xDFMuT?M>9w?#1yF zXVKVl4b5#`xYX2v=C*6-=h$e zGq@|TBmg^Ld2-+s6p0r4gHmFXa95YgoUHBw7582K2t586JbIKtF26R-7+;w>q48<&ODyL=Y$jAW4+-0RTrTKm zM$f_REqtED5OX9#a3q2deK}b0-gdMXkKtw|uw&H`I6@h4Q{h&gilpmp9N%;o7i;@* zu(lBjc~}Oc2CFPU!0}A1{8=H|ONY>2K8kfqD&Y>3)MF;Ivp@dcX0%s~;#_SHzF4pw zN%eJXS@;$9Zo9%hyNiFj4GHh7IJV;&TB}D!kTY1{)Q#B5LNQcBj9Qp;=cJfwc{3h41Zm4 z1EM>J8y-wCh3Alf^9sik0*&eK$aj`J7v&IvdITC1a7ForVpBE*B{80V z!jQw$;0xr!C6iQof*W`4_8C6*UAKB6NF>7>hZZ4FH^Qt@lD#JxW&)3Qmeq|nEfI4q z!oVnbyOW1}|IjcRTibD=u?3B7S8=ha4VRkR(bRs;$RA_C-a{wgq4-Nm2rRRm0Xam^ zLkW(b%~!iQhEYDGyuskCZh+!$0?fjBMk5Tdo}>_~Q*_@pq^b$nCV-OSA2rX=tN)=9 ziyKa!$6N3In7w%^KQQx%n7nF)Z3W0$pyS$Y&P94C8+s^*i{&)Niq#vRD1PkSf6M@_ z{n;!R?NwshM0?_SmIGaxZ5`jYLvQh=K*|rl>eP<5$JqKkz-~47QQaDe(DdL zV4qAnv#ZM$+%QCF^;LxZQvmR#|fk9h*Oz0s-%A)q+;S5*;8l2HE9Mmio%4XSXkC6PU^FKTG6KN1oDQBeS<1xK&8C}`Q1}Yqr4iL247@4%z8G`19`AT z=fM_!4X(&Ecp`>Pizk!?&7X;c62nZdAM@>A{5ai$-%Xy3&tLd9c33B4uYD5E36pWv zH3_$^-$j4=zhE@=Ux2j#i3chF6PG-8Y|}N&^LSww6{MxW8xM>=|(S3^@Kr2F|~6q9PVs zI0A1-V<0MBcEGIo;P9ysoe>-_YT<|2J-ZH%OB7FmHRNDmYO-X4UA4h2TVeH?*fTZa zn2t-^?xLe~5GQuF!eh;VSqZ5KM!`JsF}Y zfwM(JXsdpRV>>QF6ynT;out1AOW?bZoif(`q6D2~{b(-f!WW;fMtZ=EG}#QF$Bz%* zT*Fe;7i+t*Y{?f8y%HorhsPFJ*klt@r59n=>D>$~ zu{C8IM4N&{S}s~kdeB+%0Ck)9!RAg!n(qaqN|WFW_+Xbc*mWNqngA(16Ak%2EYEbJ zss%1VM~dpi6awl*FLX;b)_lJh*GdM_TzwBqmlfdokOxzOsciBrYbol( z^@@k=#oV-O6jw-IvVIhe^|#Qn`xY)$Uc-hpJCLGV@nXn?6x{)@D-%n9T*_>wOV#)B z<$_Hxix#9x>9A@pq)Seu_#80#ZAkS`#$@RQm{l{(ek=c8o)ST8!ELmb_uz2ZNeEt< znXBZXOu(y4al`3!!RwLW_W0lx1!g)sMLX<*ml@HQclF^y=`9>8IS02h0Ea4LvTQ+$ zVu9w(!tdv<#EsfVXshhSx~0WPQ+!C%EU-&f=#~t8z3~S7>mIGT2)A2?Rk1Q#m>BMg zEd$HnD?)qO0Gg}sVAY}mm=%$a%_-X8H)rF-#@lGD?#Gep%Md&YOtJ;(ngO&DaKGZc zLR>BFM{Cs`EL*yLg7e>F%>U|Lc+qD^TEGIEYJ)2+iIauHxV-B=&KF;R-=;wjJ+P}X zQanCbWC7`(6j=NYn8O~VdQAr7ovx8$#}r~;Ya*PUAT&!h4z3|tx_+G8+XQRShZj^6 zOd%Ii6YKC6Ow$~1J8-^KOn`>5Sg1+(xSnB>Vw zmn?8HNJ1PRy|NNFs`_!Q z_BJ*wE8_W2Cy(W)nE#IP`CqeWGsm4&uLE*8gA34JmAclAA~yku)LE^Bhz*r*Fy-Oz8irxgiJoYQo5-cfOR9^n%n2B*r64va8r_;E^ zIwn76^2W=9EF}@qrXtJs8hgYymk;B3#dQXbJ>e*9ew~>V5%+v7c(n{IJHc|*TNds^ zK%2#Kx+Jw5ch1Mf!ag)t4&lI#W(1{K%z(590_>+pg}hJZZ(+9nwMt;q;yRZ2Betrn zWFw#v5IhS@ep`t16?bv9;t{i3t``C~$QQBfAsXxYv1juIeDLEn4Ac>uRg=W@E(7zy>tw9>&`u5w`A!aX2wH-V%;y!{!%J@N8$BnfMl9Kd3y^pk}lNV#`48G*q1oyc?BWw%ZP|C zvs7Zt{W6Yj>p)}q5bDb=L5yd@LH?am9ukh%vErQyG;SD1&rYDNaEzM}E|&MAxpok3 zH4mAo+q`QC?X_dbUs?_Vh6&tOH9Gz`aCXyemOA~q>MC?;1{d}`L6ZB=L|mAMbw8}e zwOxn+vtXs3*oeT?dj< zU-ygMhE#UZC{}%10wETH5DP#KCJ+&4;Y{frv=##`+W}hREj15uxuTDyzOU8-S1W;`jZ=P4C#K3OMWWy#?jPqlVe!#5P@|RQ>nn6!A&;=8Z%otki(!Y zfr{iIe7)PSdgvb*!m70!v1EBZmaN!-FV<|uyYK&w^AD+WFcV6nadJU2n|>;oYf;?3 zH^9f=b7z1Xk-RdUR~|E{=x-J-S;e`7tO|&6On|Er%7aUcLMIgwDT2b{TJ#S5DTmU0 za2^>k-kVylW6{#p3`R5CSRtU%U=gzdf=)^!Sqz4g?=OMI^!cXN8w{|z3Fs%8%^>BT zTqv;_oPXTJ-U?8zGxB+3NB3XNKg?41=P-CYk?~1Ah4vo=Fwp+OtA=r}e*^^d-ykcA zP=Y<>Y41V}GI}%@3zn|pbD{Iucktf_j%gjT0tw5hlB{SZ6n_TtHxw{%O!>QR-@ysx zKepxw5lBp+?_URwiE*s@$#$ZPkDK;YWHk{?&SapNz+o|Lw$n=Nz6uEo`=={T7kv+hKInjG(EqkDrMzR`j8{hQ=O2Tk#;af3X*GBpbdcDLQ2%BF({x;_GOx9>kH& z9f*qac>f@T<1lLhc<6aUOtRvSOg1G70nWM1>g75dTaOq`crnkbIJ~aof7}02jwV@Y zlE0>Tr7`Ix_@nsq`KKmBY96Nh;xEmA$F4D+|1F>F<>p33Gd!M?&Npy+^KG=Ps?c8a2-hosHUFz(eEu_@YW`{8K=V&~28s(tBT^=N9Q$bP z1aepzixnwoj}f5w*Et^lM&MK92qk7i(&EtlVW@J16^lqggYG9{4ZHl=Mw1LRw}hrb z3Q<$jGzijEqp>NdK#qi2StRMv@tR4%?_Nm5y6--Z-7kCx=hMG~4*R#!Vf!asbNqMQ zaQzc{9RCA%EZ@TY)NiAI(zh`*`M+V<_J5<<{y$K!d-1;Pg-ekzMGwIo$^^w@RgUw% zMhF=m@YGmK>O1yG5(23W{1wv9v)?_NpJ!{}*&6u&>l%0l9RHQz{pXwOClmFIag1k- zV*xLu3jd`O|37-;z&8!Z)J5nef1`j5lS}}e18J%ND=||YK{V`aL3`B*4p*LoLvz5S z5wL23Np-*;@F3lr&fdN@-2$5~z-x-*O6dbMm*2;}%_pHsS(qY;cwSG(Wd9U+ohlZ* zwFcL!2XV2q8=F>_umo|6Itf-e4XQJVGrM}xTrtGJv&*7`q<5c}zXMBP3fwk7R{ydZ zEhR(ftQ*9lk5(c*K)|ROUT*+@d}}RtGhf-=jg?=nfXgGm=@nrX>~O1Icmx8DlQ_EN zD*Ln^ENp>Okl^$?ks`l{R6Pxo6%$hZR!sJp;Bf~LNS}_T$_Hqv96^ll$&z7Qt{TI|svd0qxCEOQ6(WB@0m}j}U%VM#EY8QOWm~Xz zMIqjM|5q?6E~Mx-q-rksoS9hm;}R}vH&)%n%Fi~yr8r=fY_N$QSVakTO~+(nriN2t z37OzjEwITJxD+pfrfgin*HUvUPm^zdND%|G`gw3A^To zSxbdYONZpjMnhpAOJyA|JORxma$^L6_vs!dNDkiPbHnN;d20y{#SMpOhD9Qea1T-x zFVbWMPEUx}(xLpbxK?%_Cu=XlEZUK(nE8H5^`*n9Ibn0#Ab171bvM#AlCicSMRo9c zp!Mq(0*F{%LQ~~^v{v_`d{Y(NS~~1{8tfYRdA|s!oC>REhQlwxotnhu;sLZ(^`U-q z17s}d>p&$4`I)~BiMJ~7#GXy4;(@L!J}C8Wj?c| z$)k~qyi~k@3>*_MP+7Yl9y!eZmE@<$g-wPH439lx5Ta(!VJ@H(i<%{ui7iK9y+4lj>yN>PLAd20^a!y@4If7G zfL^+C1qL6EJ^_wNvem1R%ySa6=gr5hdxrlqNuA!l+mAyHXR!b9Y3$zDfSSDr(Qxt% zi0w3nQB>~T&x~VY5fK<2ODShkxydWo$^z;VEkvZdqIgBgf9 z=dge3mC6A$RCdBAW-{B;84Iw)WyJeBmc3oYQq@GQde^&IG-X%IE?xtt2}#C>(f zFR}a&#i&_x8eea}jw>a^#2rLqNk21@FBIRyo7umGCKB+M3#T52D-to-#<2)&dJsWv z783UP3|@BD0yP`YLer_yA_*-#6$(kqE4lFai0PgSC6Ws_0d7(bBEDB}WK$>mF`lmK zW}i6*tu(s6Bz^i2Fpdd0&UC(mlj|R#v0?8!ClQnhINr(y&C9j-uzJ}p2BUR#79=r`$M$=sAzOSMr+0Rv zz3LHai_XHT`yoa$5z*)1wdikfe%pOqCq`~DaCX-n>{@dK>z7tz%a?nwZNWZNFFwn2 zN=74;hNrwIFom~j{UgAQi-5+KqC;`W7ep!XqvJte` zKEkq(cCpl{!ykl4jv(lnfs@5uXssMYW6>~nd~^U?7uRC#!V;`oR*7|sYp`zdZmeEh zjy0=F@QZ)@jKN@%5+|SUus9bhe_p{Jhs3&G`)MWnONvr9v)>5tA<)#v2QXpa}iid@57_Oy}pO|{ttg;9fsk zH`vO-ADhachg;8rgJilXhmpT7v&#RZc+u3+{Ura01OSAyd5sW=OZkUb>da^(U(fK< zk59n!lSVo6){SJ~!O$pQuhy;}=y8%xCxF&aRv^~#(iQ7?|3T-o|M01C#&MR>V5UWI z?dDzn{r!PaX7(D-$c7&!NkgywcYxyw<-cI^KLn0_bPsSt!fn0=_xeYnlDw>zgquJk zl6enO?lqYH#GZF)q+*cEexD&N48Ib9t{W_H0CP?Vbof}XiJ zUqMp4gZO&qO^%=RJ8C?WLP>;uv#{ou6=*4Yh>lv|%TJ0Xlz&d`yvg}z|F-4{=Kl=N z|32FD>QQbOBgK(Eohz@yr=R6lDr3x39n-9k7^jWY^?6Xnq?K^ z@jvvpiT_aohBG*SOJuzvWxy{@M?{?i#ghjukjH>GF-k*$myq*PqW=f9?Qz+*0FYCZd5E2s&q?p{NtBdq#1txEI?$*^dpMRAR%z z3T$1u8><$TVePVNWBk=M_}R}sf;SK|lwc_ScxK^5Ne>#!hjF^#4(NIj8{Zd9@GBUe zj-d7uJW?hEZpfu(p={oNK0bb8(aKUTBA%CVa9tZZOM%ms-Rxy-?Ej2dMyz5Z-crLM z0kK)kKGwx)2ud&U{OiGKe{ud>D~$QyTX2yjsu{!$<|68yhf@XJXx%-G!^N!#s*AOR2o(-6w9Ax{{QU$r+t81F?`--Qicd*LQtvkXe#Fh zFFn5ns3FU+0J_&dWaLv~X|qa0)Buq69B9EDmWa0NnQ#X4IImN~s$YvEti%xW#gOfZ zVW!86d9F!#*YYBMoiZ54MT(hR2$NU_6EZ;(p<6G#m|1&zR{~gyY z|BPGaZ{dOIc?_k!fYH={!m#809m>u z`FhZrbA}V}P@}+K!5*IdZsNJ=+1x#wyJu_Q|Jyb21P}cOVI2SYdU_^RO%IBHC3!sq z&YyWaKfYm}1d6|t#)-%0HPLrs%8<8`5K!4$)udaIBHLi|c_BJO_{^cI@y?xCUd9PCyJsfvKKa5~cbFG3Og z`1Fl@G*=AbY}Gxi&96X8z>VjW$*>3P5M2@M-`tGG%0XPLxP!Pl4OYntd%y;p5M4GwQQtnPF;Jst#1 zQ*m*}eKgh#U~kzuqzf)g)}Kd8=(}))(jht&%oX23<)_C{wfF><{9!fBzBEjdCt-?W zgVULTqh)t+zWy%uSDk>v>EY{V3#7rSra*Cp@$nlQaHVnxXKLK;y3oQ3F6;q=jZbHeF!!*4Nw-c{31jl ziQ~l&aHXaX$BR!xuo9@A3cKIL9?`N_g*R1!?#zJ88iva&vv;^fcEBQwNRvdQ`6M{p z0jQ2l9Nusi*URqV!tN&6-A=ej##(j4E}P+Gd2BmeUShqwkgBCI`<&M86u%28nj0=} z461o1F4qolL8`LgAZ(%)CT}Xwx6PLZn?&BoDMu0LjsjgCD+UHuXMtVYt8T8yQd2{{sHS2Hfx^T%Q~hyl-d4;7Rx0+F*M7wk*v$52 zaNO`>%Uma%#xWgAhD4v7-Obem{d+bPehF`8@q3`)YU`wshR&B04d! zND7>o=fnUsz*z*#OZ|%aR>V6!r?P4!sXHbVb83xH*W~kr8XRjBbrRpKh z)!xF!6?KsGJP6?=$V<+zzr;-C^Ly^1p}GT-U^Gz>OM_%FGql&S;O#BARB;z)>$$zNHrQ#c{XF8ssZw~fvY-M)QxvIO6y;I@Wr-S6F8E_`4an)d*kX)ZF@bu%m$UilZ z#U6#^J*f#ZaB|m8W>r(+Q1u#xM^7ToOzK$R6@2pcW)Nt9zV-o;hY?p_#iAeXz^xjfapN#He_qSulSi|n&O*}pJ!TxYl=b0s`Bj9xbNJsy|1>^_ zgzz%fQDa%@ATC!AV)Ym0C?Tbh3Yafs!#=BG7iEq{o{$^oqZq=wH+S7ty{rtw-O=JOLT zZ(j0qnLBbc>HYbG@5`YAGxIfH#ZS5P7OwM-#k`s zpx8_2D(N9~-?_^%yl?0sBx1s|gtEcRCf2?2qWG7^Ol|tvc&>_r8jGkfPUDgChF1=; zQiC4Kg+h5)Phi#BP26-eGBSw6$4@fDlxzq1K4x$}&K~zNF|_pvC|CL;Nz9u2I(8J5 zqkrfT$7LF$u(X!*7Bx3{Gy)@uA#O0mDPPd%wLt6%yu*%n&nDJT`$cuW- zxK#CH#xdowiRM3vfP?>laU2LIp$DTF7%?nBT046n`4gO{gitOVp&STW1iC-SfU=;I zb;UHOfovot2`{-*@UonaX|fd|*~fc9IYcbD05=xW+MVc&8G)oULJf=5xDYa)hh;gN zrK(RC+{V>iz=`5+X3y%OnNW2jmTLa#aLd%FF_HY!&-rI#ekI(IAZ$S$Vl*47Zx+5P zXvg`=ew;182a2EMB^r)TV-M7bdp1g!97ki-FfP>$WAVq^IsOv>_cZzE%y|C!s<81J z<$sqNf)X)+Kt-8`r2S1+TxqX(h*PCESY?A0DQKTYjZ1L`j;qj8Hh{}Dy;!rf=5HGR zU4CQ#O?wz>E_&ycKVad{w_^Ff7UTW7pEF~X_Jp+G6qQT_d^0gC@MB!8xX%h8drL2K z^9^}@`}J&OYOmvX@l9N;9KxZkU5L8pA>f}1HJZhZalUX41T~ZWr0sqMYRvGAr3Qwl zT7Sf_^-qUKB6<4^D0&t&ISWy0JS@3^3l;r1xTO;r(i;fsQ{fM1!L5ZMg|iscbOqzE z`gO>OB&1LR?m!g8lnsjWaJIY;O{I@;xo8L>%gfMJirqgE#2dpM|-%SmOKX~m! zEdJ$otoUsS&p#=~{I&V7FTKF~P&=uJgyRT^vv8v52Cmfh;b>_)e6Ac2JKG(Lv0_hL ze}x;@NV(-wbuZQ~EH{90k_vyC`~S)O8}><5f2IapYHE4RI3^_yQj(y#*FzNlMlkZ| z5n4ODSgM+0hbNc=S2zp4P&V(0C6%lX$g`Ows%O9-Od#s>V!F?bR|F^K+uV51?7^?6 zJdb6Q{~y$({xhzmKZo}8=W)&QB5s-|;g)$4ddx52rs+9cw>*cd*5}Y+e;!wzFW{=< z1^)S(={ej;e;&8ezk@E*w{cDpP@0*6m%U!332vlnVVDUl_C%n2;*g~T1KL6`6ZA3w z+fC{i8EZpPm0d zwp91y`+B19qE2j`Joo}(*c3ba7ZV6eAfx7t;qX8SwxCtLTb69a^Pa~Nyi*^buo0bJbOhgFM9;PT0^OH*L?eFu^~ zfCHPa;Zpe^P8796b%i0Q4y1ZgAm|dD4jqyu8=mweR8KY>b_EWv8y3|JladbA5yh$> z6r;0j46T&|Sn>H5SX3LIn@KY99Ge6O?5YzszXfK6{B5nwmL|rroSuP`o4awTq#s|G z-+&@yFiX}Z60_CI9>z=N?_iI`OLcddG450ar29=sBTrptHjWnD#p#ke_`19iA#Wx; z9u*$d1E<#kB{c(^e|7-v6+lb<0G58b9#+{2la>aXY-Nw&uQ%R7b9FC{SDl7eP+*hD zQ`!oLWQX4x!^(GeaO1&+x?W}s)8|a;6rNwzl8di5-a}JO9~x@T!tIe^mR-EAiFK`6 zqFDLv&hg_P#Og(*#{A1qIsa+n^Y4)0l3Z};DRAj2kZf@r+uDmOyM}SG~06_k{wRf3#(-KD4YFLc%H02A53nK;jd_mBQSY7+G`);VsRg8wx5K_ z>4u;wV9?tqK=3$V(MY=737?RMBb$40rSu*ySGPel>kvel8PP7q31vzG+kbW#O(kP! zJTQ#UKVHq(?P=$q7-yI22C-_%4p>QQTk*i_r!IQKGmt%=$lotSg`FTiU9u9t_}wR5 z-eiVS|IirnX1omQ7*`1VB+yb0qOx``7j7SpJ;e6HGA`Ws)Fkv! zCbUojP3_m2)!RGx5Rw*#PYE&eiM}VAB(`S6000;$C%~70W5W-S0D;Zh43A?1jw|Z+ zLyMA3WeiR^1|>3;{W$5I2_Phv1hYAaiAxgBfoujg+)@Poa3(r$^ne)0y#oVWw4h>p zW9tp>Y$usUDtfz#*-akKVif*p7P@ZT!Qj|K^od>sLG z1~Z~GJr7~a92EZMAexK&(EimRmVdkx63J|avY|$&LKbtOJ7*&4d!5-As*(p^nB>e8 z?2nuw%tztB9m0)rpt*1eC2Nj?SVP44bd!u=i2T~Jm_4q{Ki z1-Qi|d(f#QuNlfn+iE!6aOIctYb7C{7a1B$4e3L=jb9LBrNgv{Vn_T=^Zu-18XBRQ*$#{hi@{16zLi z6*_9iaB0tdEc$#qGj_b8I20ud8N!>O{ByZ_07t7^nSI0_e#}anfrw`wR{XRWO_fym z9>CgVW$;na)t?DXnuE{Z-hmrMKx<_mHhfVADhPXHIdDZ1u*+eDv>C`0=Hcx2TWH)d zjDx#cdA`W!k9?;$FW7^ox&fRoyN3-Q?q-iZIyZMP&NBTmc^=mOYYi7;dn$mf3+vzw zh9Lwg{>?#D_#RGf?Z%bLJNUY~8M^cmeELgVh!pjyhzYZ?_QyqNF7LyI+5xOtTK$d2 z-?FgYkZvW$oSx)!sL4Nz*4hzfwQu`iKVr7m_?57|i63Tsj1xbckbRk8~obc?fLLgO+u_% zvue&+bIclj^xm;*P9Y@nU}R<|fkFwe$RSP=lh?6RjOXO8%*;`Zx#b`Lk0cnq#4wIw zN#>agGB9q^3)VeSH`Az+zUnF6iXN*F4!HYJit z*OK*IeB`6g#4vMN4aA|PO_K+{-`#^?Y>IZSH;x%kbj?;NmXp^6pb~qN7{`~d-$3`{ zC%ApD3m#&UleexE%YM!d0@(rsVI_Z4HHkr2W-XKQ0?E&N18BQ+8TTGO#=VCKof2`!o+@Ig|{m7=}a9{CTPW z>;eJh8h}rLb8Gt*jz7d$4#uP~cueE$4U=4GEF3c3bAra6eUA2tz3x_sm7T(S57Fa8 z`(@24;_k!8kg0G>iQzp0#W=STgKgPGJa?Wvd4$}8QjX7TbKxiXe;+!oUgfxT>p{0R zCaj8~`EAnYR{prL7kJj@Co!6zHU806JDi&R@|{xB?R5TcTTIB^uKSF~>fPGnt2&^rPycQPbtAQaCa zNJ<(M4;=$xd(QZ?VAf^^^Jsl1ziglTbrh{Uji#c9Xf1kzjlJt9NkGS$-o~d1GjVFm zO|~K;U^!!c)~|~{+0DNy{!^g^`I3_YgX$lKpy5rNTzd`YiXPxl&N+mX;SkhRxX4z; zn}S%|2(0=v4^-H6rtB7$FV62>{}b^~S^p*Ss7>I#3jv$c$E-$u;T_cG+(O=hqX;=h zz~xL}a8XfG5i7lkRQpFbvx9OF-Po1a$e^E-{9lE52F4Gry@=L);N<4pT*ySvi9u0V zHO8wBLdg9FLiXVZyN5z_YV(iYW1eUJk-ChR{MAV*L5xMB{5JM)xQKIwKy6kxLbeec zlPr?R1x!BAV1!(6A>bIvwhnGL6>BEL?g_KCMeQGdeCsF!d>_ZPI9UqhWU5+@`b_A3Tdc&*4*-B=bE#@M(7s742j1`#Gl z048?`HZi0X%<{W40op0B(q4&*z({qew`hmnN1t~MesSK5zI!nTUd(}i#vFKxhyI3v z$IO?}Nt?X7;e#EHkUqAX?F1Ho$33K=GnF_GoG@ zxQ5l=XThzHgWK$d*QO$||1i{L+(q-YCpc5qiE*Dyfzcif0;P<0J0zzcv&Jn#UEy^! z=k%bx@G*8TIgJfpR3Lv+E$Y|ZN6YpHXe)Y*y3)ItIdui}GO;xsaM%Nw@$O1oA>g?1 zB33QRgh>?9UzPD%fP99dV0O5n>QiuP+a)yT-9uI0G4@A}@DmQNe8bXh>Q=nqB-vgYDhPmz;W1n)0sHt) z|70obRxg~+5Cq!*sF81=Xl?_`jh-pIfCW>R!zx&y_w;8*ug4NV#-}@QJ`e0OwRr9p z^i%YB&G!nT1Fu4-yauxr4Vx4Ns}upf6pcv5h<;);bZ#TO_9T=pK8@yr$2eEogUoLW z;5QC{Y>b1?IuOBEMqtl^^Qg_cjgxyWV9xC2FuCmv%<4U6NY();p5KJl!XC8kyoY5I zw?Nb-z-977wuKNAIT9yV-9z*ChiEFjj`5#NV!&6YM8YImpcvC|Xx%L|P{nZZ3E2C9 zUylwZNe_=Hh^3!wMPvSB)RaENviY0&ehf-~K3}5#Uv~#}rJdMYdIDCL$g=8D-e~TB z#TdlWkF@?jQvxiX`|J9D{bz)g;DGW)tZFzg{>^l)lv8a8B;dHA;{2M-Ei6y zp!6Ss{VT4q4|hw^9W0-_mi?P`o=BJ#9Sl+=->2OzKrqFieA#LCCaurAjgMZRgn(-x zBy$*oJpiZRgU;!KPIf>iN5SrohQ(omUy6s@;D^@~huG*eq(#1omhGKrF91r`oQB;R z2a7!fqe+F}^ucLy!7N%4DO=$*C!=UV1J31l<6O}dWPZN|ZbJxeiweo&Lvr*eoLX`h z%~`;y@|*bLvq`=EAMuR+-vjRd`B~7rjG*;HU`gs;Z!sl~6ZW5un}BPb_n-O6Qo81B z3ny+#u5@|kdOn~2kCLdw=-qd;7I(UVF1o%QMZMRb0FhZ_p{KwxfxP);)o`hrZ!A}` z2gvt|q&2nVD#^v!NUkxE#DAw4KrBx3&>{v>VOb^L3rS1U9n&>!*-^v{8;PXUh+#)S z1A!hY9Wgk6S!1c)x_=Mk*-WzG)IY?;o4s%ux_X}Q{Xgh>h|j+G26ma)b}8@%6HriG z&Pi{@{K_C_zc}Fuap#dMnf}?Bc?&(Hp^=@YD&~;N%o8?;77|3dvnPS8cqtB8@ z^u)>2pazM(Ooafw>@Z4t{i^nNapH;mm5Cuh>q0IA9-+IQJ+dMSqA>t2Gba+Ok{t=BvdF z%DMwFP~-%}JKn;f^_Ov`><;#pHn4{?B|}N-S@5JG=o*P7W3zF#@B!*d9$?kNe7Jov z3`BYfIDUT{u4I7|<7*e@bHR!^5Cdy$3@0{KITZ=EH&CA!dQNy~7c7yobY^FQB&M zG7gs3LzV}@;Y)`tKvKLZ2swse#h9&VD7b@~vU@E1Ozo@kAUs?9PeAQ&nqS3CECSsq zu^g|wiN#;;Kx6KG2DfP+vS(QPqh*V?iz#LlDE*9h1PUBuWrd0^ctWgDmwjjVX!EYl_C$?E{QI{$c{I-F^5| z{5g6?^UkHB9W@E$l}pb(fuqD2C7^Zn+RfVf_C%A2K3-FY?YRXg-dTlfx9{@*J$yu~ z8n89HQ1f&p@K_?yG7cwewD?7m%qv!JYFu?5y01zrg8z3Lf|qfR4|h}6*h087E^q*PKVKP`HHsY-S6h2 zCt@g5L6XCh1Shemd;^#@N4|?TDkx$b2x4l7AgTe>G&Jz}6u*d3O-yWoB&aDSsWE8n zxS#>Y=)%&>)f#JwSWF55pK+*bxrkf$ALCrd6}K{O2k8?umsOOJ0yM3>=ek1j&R_Z_{zE{Sus#%JSh9_bAyMn!fSaD0V<^;=yJA(EUkGXk~Z0{Yy429D{vpEv&f zqaMcx4j+H|OjMGuwNC?lZQen!`G&*eO+<{MK=x1p5&4=15vQ0jIrM+w!BQU{EcD>U ziXkYP`7!)T04`4|ME_9O6^g@@bp3UbsigQoa=ZkBSxF(ln~JdeE$q*1$F+Q*Hs?0R zygv&ebtHV^FevVHc*UXoJI5d8S>sO^uk+u2xdLu+07P#pGhTW99JdXPd7Ws>dBX2; zyB43v`Y-mOaB3aStbc%WJ9^Mo*n{S>`o;GhCAH5p<1>sUED8%_Cl`TmzJ$?0AH zHIU!Itt4_FFZz=fh7mgCNtng>Bjfn7{MKplY?^E)`4 zd4*Z{t-0OEn41mq;3n^4V#|_jJMEXKNYX0}Mrr10?kBRY829FU43Iz2_J8sa-v3ej zbV~`m&muPW@Zi{|{eOD_g;XLHqy6M@Ecge&Dv+X63cvRq zIMTQNJwfH3{UFN#Dx?rd>CInNX34!2nZH@;rv%TnHB>~5P-PaNA`RZrp>@W!F$&b`2yu-B@@T=L&D4 zwcsAA7S?m(yRP6C=FcXnY9ox26K+ckX1u!!7xJEH!o?KEm>o7jl84 z8!lqYoLyKqqZ(^}*o~D7bMfWZli^Tg7|bsC%&%kLx@)K_>cXL-b8tERFvuq8Jy8sh zil!juf4UV7`8_yW46K-&52MQgqpX94e3~r^ei*%)f!WsL9-PX$g%y)aFeZHgroX=t zJAOEVhWrO;Bq_*}7K|GHE==U#OCG<14T>cW3qRkAv&Hv0`MqTBW*7t;qLfG&1CfXj zUV%-uvTU-^MH0bIM2iL(6%)*&4KAw>(>}|<*|K}6%kM&c-b3WhJb}3%?!dgi?m+eO zW;Es9M?>W`9Nc*lLxv86Sv0{Q8(2!*WsboQpKL^P=|j}#cA_q)6I-Vp#Qe{;VczIX zEWb%Ftc^SGVsHLw1j1qH@n@(>mp#@C`Q9=Q80JRkJ|Y~ z)c-}ls{h{_@gDS&0cM{8R@nlF!H?-5tw(b)Nqs#)UH)Bcox2M&KU1w0CJY>LsNb?T1p<{z?KWxG3OvQ&)AJE zGxuQK59OFJ<_Fj$7c6cw91c5TJtMF)yV#CKXNA?!li7WeLHY+<1K8M zUX3kt_F(*jtj=UOnhR zZecllTREgSjWOm8Vg8Z~PRu`iqIoG+?B0*X8JSp?xfbUxXh~UOp|)PQsIg>zDHwIlboa{PGct%J6Iw%b}}>Q1ZEv;tTmFHrUbL>i-+Qm!^pQkKxa3xYoW2A zyLvEa_`AK#Zjvh{z?8fKNs2p>S?ZT>+(s9BA@?xwOg_uy)rU~qc%JX8^U)K&zI}&} zb34q|Bj%M7!s2C_PyHl2yB>nPoF`76j<}>jn6qFR8e7_VOddYbz~|D+J-ta^VjIV^ zOsGV@*1lL~ob5k+ie*M=9IxH%#QsAku(N6(Ds~-Uug|%QGP#g|=F9W7znl9@qGYv) zz{>&5>QY0*r162qp2m@wQYUJfnb|?IClA_Ij7S@;$z29hnE9qS24nu%?Px4|z<$K! zNt{3H7#4l911rWAVE2k9G?E8-X&3fwZ$eVSC|GdQMZd*XWb;UyrLWd73pA7jn*Qe@3Ph#fNz zB7gb`6ihnB>j(Mnp4@aDTW9aZwmEyTW%e%oxTqLkel-jJKsrQM3jDqy>~HLm(hv}b zVcYzD?048w_z1_hUPbQgBiJ%+54O(TgLU)D@$I-}z3rzzSNj|0S77qkDKENenj!DBR*ccXeoBN7rv!R`<7+*5^t zNVL6!!|N|;Rs%)n*=vtLTV_;vQ=p1NF#oG9{N0U(50SaBfIWsusy^r%fw}K*;X2Y~!^Iq6-^Ny`ayt7T`&~mZW%MdesFRFhZNV+9 zo4ylcN6y0_(XB~nCpKnJ^ z_8qhoJ;AA*8(29x9~0hOh&i8c#P+!da3-&lJ&frdJ{&$-^M$0s138TuUQ5R1Flcfivzb6P*$N&p9s zoZ_{=Z~Z)Q_$05N)q4)1de33ZU7XpgR6%io*0T`KU%J6<-0XY=ZzxFvqyyUeM4rAR z2QAY&$$%ZHFC>G&Wm2o4HFMS4jauxbYkUOk+R^Imq#6-lA2%s{M-HN3d+g7I^&R?-^k5c*W|q9IJVZA3i}@ z^*&^---;gXdVxk6ildNqZ z4R-%vX7J0BkLB0Bp=7v4FA}{DtbN;vM~emAUmikZ#+z6=?hW{S#DFKjVi3eYTJJo3 zUf`I{i-*fg{YT0*36Q+W?Bg1+4oC5l(>Ry=m=&i^POj6HTREXexe)hTLwn6#~uKkFk4U zD_V0%QK}0|XKv;AL-P606Myz@I>+&s3bCFy{!jRNE>txgiX$D#&JS^DLkH@}YN)t_ zrJf}(DG8)N^bA12ISh-(ZbL&sCmM?HBXc3eFOmuVXRZGOA=BC~CL*8=<@iy$>I}~2 zKR|226P($88yhB7VA+@)%=_ym6wW`6hQfPjD(*&o$vu2OekH4AIDD~On5Wvb_*IwH ziI$v4Z11u1`y!6v^Tuw*j@kRsQq+aBxpz@td)SaAD!{AF{Rn_mSGwvpoMt`1#MATZ-`$=k%WZ>BD8wkmt4pCN-rkb%fwm&f152(Mx-KhV8 z+xkD@q9Fno46opd=|6Ba>LuKYdKp)Y|AF(iKcK_Nmo; z{6~Pp7x(|uL-}I-wM`2pc=|=z@k*q<_r>$4KKl85`K|wM_s78!3c(uVL4>S_-m7P3 zronE3B!n?6>0=zqYeQ4{eT_wu_XHR6fc7omWO9A!4XjwY0Zyj`r%iz1kU%Wt?A2wc ztGI^x0!n}bjl=>bdFkAHsIR_(L*=!YIPM!*B?ENce&{b4U{?IlyPdFm9B?aMEMK+> zl_f`TxT+S%E01GuPBqqiw-sw9l=DkPYw-ijnVx}2pALG_2GJagX&+{wp@2MkZ{f$8 zxp3S4Fe^6nQ~II5+8;L61((@}LtC5Cnn%g^qi{Nm5L_-ebt*m_G!gY>chNw}{oEd$ z&j+)|h(UIq{+-*y%&WFspe4TtCn~RC(TWYQITeUjAN;04s9Jvx^%ak?zu+>Q<^b%X z2^Kk;y=5KtAf|k=p8cr^99Xkp3o|y&ik>}uU2Y$2hA_5e?8B*|PBa%0^B8E_4)$bj zEbPST;tteQpTo)pYhe|vh?M&AJ-S>zNY-S`9KR07cV5D&T^F%z?s_;KA}oR#MwwXA zdX1qiiZGJju|RB7l7TkD;t#pXr=4V2F}q@0G1D}-F+4NDr)iNSKl#U9qD}y zR@nx(#Sfn@4f{8=;7s9l94Tmp+Z<-ru*K^_l-mr8OTl;Jm*7nCJ$^A>Ie!zPl|M5m zEINbm7zbk4mS&tRxr%-HXCac*x8Q_DGQuS4V0XIl{rE+wExU*MA|P`?c5nado~!?D zZX5KTNEkc@m?bA{PA{Cs804%x$X>y1bpKgE>kjZ-qPcmhv>hjRH(~LDl??XNn2^l4 zQP#sK>S2-0Fj}pk`x%w?1rFz*M`L*x&g4Ac{yditoXY`Pa)I@8bK!K_;3m-5sN%gr zU*c@lEtV3a`AL$WZQ0-oaq?^3(19w$S`y=n6Z!Pw?~Ow+HWgQIK6sjZ zrUWTTP14VYkAah?YM8M@tS+A^Bo({?c`p8Rh%#(XO@c9_S}sS6#~m3>-F!LDoM1V!C%?yPaul z<$LD-?g6@=kSI9t=y4BAw6?ajg9vv7-WQZsYefJg??$q9#GLZSGrP_kPR6HWzrmdc zjhPen~a=yNf`1FSSj~J@R!9vdnqfkT-Gbo|g2cHn(RN zaWdQ&jDzF}GK)m^4TZxW1A9P0(4CGWKVD%e{?psLSn{x;@ETfj?sDv?DY=IOyDsBk zWg9YPZh`7aW$$7F-pDVN7?8w_$Xj_Bbp?0OR`3LESpb2y9c1s2cOQ)voj6q7hVQ=4 z;DoG64MB`aW#*FKJp$kTbu~`s-a%_2(2}J+|HMFP%z1>y;)gg{coX%z@1tzX86>Lj z!RH==pfH@1rk@V}0nHU%49pR$vyD_gsBJ1d;TT9vjvln-5+mg?%T2CcQV3N^hb*SD zpDnRD35X=t^5|i+a4P#c%We`wl~fPtymN(*K`a#V$ffoJ!t2k~er?%p{J69fe(7}v zs_A>YwxP()Jc2XjchOMXg@(L`3|h7n^|1F}UDbUYEV+R1CamJ35qluS9=CM8iPraU zXhSJ{+?@*np;-2WTni z!q!>45O57)W~$k%!WBb`1q0x-4?#`V4Yclff_+pJ&jO=EurK1#pq9vYd#ZYbcVfqi}R{E6$bO#nGHL1|c+W z-$Ag6@$h+OT!neykb46D_$9(VW|b71PLLP%GLXIdPAY29J9%cC6aV9-y>G zYA>dF{D_%LErkzJTX+k#J8xq3{9itjSPG9`_>zIj?_S zi5fUUiBmZaF-gNvzWXSy-tK-H2U;&&MOoDzY}%If6u8UE+sSK)s3gK0NM^f(zZX9U zbf$HX3!(rNl~?mxN$Wp(D-yGmWKn4ywM#)}s*WBv33t1C*YGF3F|Y@`9(Oh%{$JHT z#j=WB#~DbVIO+=}vlsR8nzMYbeb;^Op;oNbH};*~z_L}uW{hQTYHiNOu)p>BD|Z-J zr|~1-YVy3Mn8s4%{&-|$ZsPu+_*qtUs5kzRRPwXOKWd*;OUNIT<_GXk^#HpMY7BA8L(uc+^e4j=N`;mDr2_;&#KG$cA?WoX&ZS57lLoPJMBy#+s{Y*~ALEgWeFYMubN!xbdLV;E+b*kVh zj#pel#-dza`$+(SKz_gKoeCG8YyOpWVddf?c%`B6x`)6g{HOU>+|BA=O*s!i@1isx zj~89X@ro;0xuAdx)~H}XBFi%G0N#I+#Yk4>KAbAPj;7)+wB$X)x$MUrf1C2VIR4g@ zU&qQh1$_Sw0`F+;B#C+@iB)Gv#Uw^~6CVtjg2M%uP+v|()c4U++{1YjQX*-|dV(M4 zmT>>ul@Ok%|B3iuiuJ)1@WCdrzYsgga;KmPo;^*;Y~JMST5Wu8_*DpS#sRK>p)A_ci>))P( z$GJ#|fO0NK^d-Z{HZ=*bQsHnY9g>m?RThvgy^JGo25?V_M5q2`+=zU zy$D^T8>VOx#%K|Gy@V)}j7Y132%87dP9Kc!0Icp1++r+5AqJ8d!%9|mUmVQg1ejtI zdF>`|YC7B=vMC^SffU#S$*h7yATR;Hly7A@b?^WCJy!B0vWL6a%Uq_Jdp!Rb;GsPkG2XQz75Ygk2QCJe2{`WC`_VP@?*D#4sq(j;`+aeL z|2=d5FUI|+aqq*!Xz%(;lnt*%JNbP?D+H!|)K8!Nt@Yar;P}6gs{Z+a_f3@F3UK%W zIRCF1*T3cQ|M|T=&+lBdM=wvo5cHxMW{(jTkDftJlfw>43gX>2#$e0JLL4t|L`}sx zoGNQUU3mwZc3;Ke>KZK0SO$yR0-M_ci^~X`+X}Zs#{8+PaHQxQ8Y}MMbp9=zEx(7_ z>MJPNR*8@P`U%WV9jprZzeXY2WrjiUz^J&PQ}pQP)WM|4Fu5fR9Q-Jy`t1dIr4vJ9X@(`^^{AaJ;$>$9LCZ$=p?N2qFxk5eA=$kN(zZ$g&?5 zS^H66-HwXVDwsSG4BA;;0^S}q2D?hn;N-4LsHwh=6J-}sQ$a^NYAV}sy7D|uSGMDH z#YLQ~x_|@w8n9^1T3AFUY&Hw*dN+0z9K+$tW|S8kgCKiglKR8wAz5f0EP{+l6Bgk_ zWgAYGHDK|)MKF0Hp_ltZ=hiVhR#bg38)Yn>u@MJzTTowo3k@Z=aVoC^H9ODaKGp298#@;8@pS2kWvm4M*aucoPcTrb#31=$LV_)e}O!#UN zY?2dt!34cvfXQQm*=yqQG|L89XZM)rg{24&E<-AoT_ZY z;k~t3NYAfeg-xWx0=r;^(CuNOl+{0V08RoP0=RX-krQ{_<-|d_UqOYUd;1@Tn$FQWA*cjn}+Q zsXCC7Zj{V-$;7rz#M>Wyj{2qxxc~5{1oFME$2fWV4AyPXq(lkK%*re07fnj~IHVZ% zYtv+DRF(KxgC?H&4xn`K%y5`@_i%#!lS()Q}UF2dUJ}(ar2%g zb4tK>UlO?O;&p7=p2vR3Zt@nRq+u^`Oh4Cb%H_m3CAD*kcR?U&>c4aJ0mmdo%%0B* zeP0*^9200v^3i?3F}1%iV;#T9OQB@VqmX1sNhXsL)^d=~yK?ismgwkZY}5S{*nRy@ z7wVcjuwi>4>{1AxPzv`M$*n2G*yZnv#gI4N!=6LOnd#hj&y*nVe$<0QM^0ewf+Y-G z(D#;B9$?n9iz>*y@tS93FGx<6`@}x_Vcs${v|hrEP7Qb`K!E%z>HMOyJuLS|jGO0a zzpV10##r}hz`X`sQ&NE#XR+`GlX#9$lAYS4gBZ^glLXn%a(g6UOhr#NRYIIJoLT); zY;j`8J^b*&Ivn2KfR4%sXw1Kd!zGuo=RiB=&EAAW|8R)pu^LE#%^QZBuECRtKximL z>j3;Xrxb^GT;SwgZPsnn<=sJD`7M-fJ&7?N&43^$!xcE2LLh+8pAOZXj!7Rc zMe*vBs4c#QnxY#xntu`XRd;Z*;tG~8&gSQt?)&-LKaY&%J0OS&P>3-jCqY#Q!Y2*J z;%QrPtoQ;NcHYJ5{OjCyP1SW2Z9j%jKbi)oNJ*+#SOWo;%nqurGvjsF_IlJ*Uc>$( z@;Dw0mrVPkME)#!IetGmgOmBSr8lvBVKy9rFiZIbq#>C4#Y&u{^|tC7GUpaBP`$MtXUp%RBEJS!CB(^R0#1FNAxKa91e=yt;&|}|)bG5_{YGPV zH1{HMmmgvuz{4f&sNH`9^A~M|Sq<{_$NGmOXdi;=ZM8VH^AZa}WN82ZAOJ~3K~(k? zHZqHfeW!yd%)0dk2I0Gj892G~63&!f!^(v@&)NQYZh!IoZR~GJmH|Q_9!`H4?ob#u z)dzU>%fvCzgs5hhWT5xUET8 zHKzcFa@uhw|2CS7yIC&wbny)wENaD!NvjbI3}Y1ryGjh7BzToHX1L9sydFm?J8-o6 z0+!CpX7&?-!y21~yvRwpp)Ywz^86ZbLCFG%STY`x{MCwSzWDlk-0sqVN-h30@>)&O z#{SsB3=$IqmkyUV5uR`w18L;yy8n<_$8_y|v7oi%I%dpX0=FDy3j^}_ zCGeOT->e)F#)2iQ(AsgCl^$r#?JKsT`={cu1IKIe?#G{NKqV<`sN_o=k1xi5hX=jj zG_}`vUSHe)`6qo}#lDl=cO;un<3@3W3S;noZ>!YId2as zr)X*x*@b1?e_g#mdEXppI)53{=Pc1a7oeiE1lYs?e6gvZB7pN(I=PQ%{FI=k#?n_% zMq{%)bJNqgRNNc?NNQe-pBi}lU&p`U9RFlu=mt_4v|%}Ig=C(?P}C5<`F1MKU%0|^ zj?`Q3Jp?}ZbUg2ODAtoNH31D)h4v!k?@iB4JfEAQnDMUjU(x(S0FM^RXwQ5HnbST; zK=nZIh8U=$Js^9-{)*%ttqEi^p%uL(GILJxq=E{0JklV<_(tNBccx(bs$Do%+Ky9Y zmvO4(D(cH_qq*un4wbcH+0t!K-w)T7hS#>I08Y~ecGH7eW_5q=@>O^BGxag1jYEe{F`Vl?&MH&?hvbH-c!;FCQ+eOB@`nN(&Ax;)*>})f(2cs>JE$qXj{QZ= znDR{qe4%vilV3akR;!~YWO?J@@ul!yOmHP3>>GivMlV3_`h#4|exmd;j^uZ6@nc=t4XjwSgZtm+ z*OWm1(fR8c2gv3`6medQ})-fA=lU zU%td~j}%KfdHov)H}y4n8OtQ?J%HPk0>rfS^E-eb#v|5i!CTV5;OdtHam)4!Zb!a^ z>$+EPGx{&Mt@{sL(ET1A_P?Or_Ger${}~rdFQMJ=2V6G3ge#_3&|r8OWv~7YOaJ(< z82j)4f;WHvJEXq!GLrwIL;Nddc={V*(-~keJJ8=O;5Dm=ehvw*xfHx6s^~BJp_jt2 zh_P@8alPwX61=_vyx()tn(t5I{TE53lk_%?|5L!1`ZJgWS11`aUe}17+^f#g%Lpb< zcY1Fo@VJkSOz(kxjAI9Z$i3`i`a9(}dA;OzDy!$|VrvxEH*p z&k^`c-pnMguP8~}{`2HbU!3#X1=L=Q?SG%K?ZX3UZFm73|H4MPucKZ7r@sxx=?mca z1#tXp0-fX;09j0d>Pv*lVTHjR4Wksneye)74hEMICSsy`TrfHHFbjIvy=G=_8buu< zo&A`}YEYsWl(Ys+h;~Q9B3d9yG6cJVQA0k!H>0QG>(SpKIc^|af(tf&dUY_n zqv2E}L|NT1$__+I5zu)o&`ECS+yYE42^OmZvcm&aKL8u2?ng7Tn7goO>P8q92TVRI z^lm-$t|(YoHaQx0)dHPB_hUjo8_59MV3D0L+AVNPBFt7ZbaEtK@x6rpfqsba=nz5d zU{ywx-42sWfL)T9NlU=E#chO1Fu^Dg$Zch&q)xOWQZO?pY?dP+dd;v{qajEx*hD*Y zLL?&PzaYZ%DhvT5bb^I_mK`=HB!?Yt#R8oY0i7BNoi_sg-LG(e+7%0I4htlgjCY5B zfGOi=V9HlBFe3d;2%-#C{F~UeXnuf+dVdzZ5Ftl0_?~_epe2EaIs(SkNc3~R41+fcdUq6DvIB0X8*dGJ8{dqXjIYN`LP~rpTy6n&*#W(% zXE2}o-Jt5B7y7{_N5brQ1y*N2SZE$eCfHObA_WT~1uOIfZYp-ZHmB17vur{?#Q;5z zzYbQ>0EeK5-t;OQk{M>v%uMA-(F%i4K(t_IX=6 zr}OFljcPOuLNwpM)9%FZK_fA7^dx*cZVFx>JOWOao1ZJA#|)EVWshV6woQrwk?z;{ zxv&uPUx?ZTOLjr$ru9T(AXD;@&n(HFCb4I*ClH6k z)O0LewH`C(WiT^938itOoRUr2OCbSnl%yZX0462u9b%B#zoXv!6pJ%{#L^XO@%~>& zBNUs2AA5o0&aQ40mhNWYm#ULJ!Bl>MRfFUi7G~y;;*Z6cag$hPdiH|l81(umX73P- z$0qu@U8=IzSZkC3k7ut$V){6hAY7gxMvtG2Wtr=-U}+})`sp~nFNfggHpr`&Hmw96 z(@#pi{v1T6d!fJoy@P-PAwa&z8W=(hVrrYDNw=jVV1eg<8Pe4D(5K_{G;t+kOM4+`V z7Aff?F=O5mWUSqYALcE_8}EDwi7NZOVc6X&Gt}t*wDbB{OIkc9$!JR6Q@>C$oB&;B zO%Z#FlB|k$UDQ|qU$jq>-E{4JAbww>otVeOWG*V({ggRK?73TaAEIjaVF;A;@TEZv z6YD#OIQMWIS$!2vdBD*fw~*+37h%UBjC*rB7Ja=1qd%MtU))HRbM}j=oM01!Nw9cA z8i?r)gZ?d&Pn?*_-fSO_oQCORSL55ymLWaiV+JD?-#`W%i7jhZ6$mjgn$)`#X3smX zM9IrkyghU*CXZc+8Q)}J=65SG=Hn?CkUWYdlSMfmjz9>}0U4IK0BoT!`#XB$2Es1J zax$`yO+Y`1nIe#r>LXPv~!sxENSN0hnYTsGx%q z?f$?JB*wmlaiiy8`nRj`)#vl@dirPZDg)sb5;Q+SH3hCpK$>^=f>L!cxtjiV~2GeeHnt>~BHR#F)?WCDENbTHWHkApK916MGP8GGb0DcF;lIjuT}FbiD~(==v}suc;)82((T zf*8kCG^G8E=XFhE;1Ylwh9{JS_|#!ovT6fn%v%O3W+8x;)_Ddy)kL1-#JZ$)%%NzW zvIKw;W7Q$~;a2?Y(X04FtfaHEdKZIf?VJ=LP192Gh)wfu@ zaxG>p%)m#VPhbYARq*k(F+lAfz`!*XfY@b{@r~hioz~XA^^=r1XdNRNW=SPK${<9A zJejr6&^k(==k-+8z*G9Y@85qf{!k*9)&uf#B&N0YJ*3{CUB94)@!7aZ9KXpfVqp4P z8Y_!fKq4{yDG{C8%luA+Cy>I-Yr0lqUVl7hJjd_MbzAV`#vS-}`dlOp8o_-_iWC$V z2=Ju1*%vRo;Q=taRQ?R9XwbF%bcEPH*R2M$pWc|Kpov1vS0TjgXBr0r)5)Ltm*Nk} zWBf|g&nJ9`C9Bq9=7Qx&9r~swMczAZTKv?C6lkpb z#*oH=`h%{Ge(#e{rM&?aMbP)t9)|vp=I!&xzgX@+H#5HCp~j@(v#}G=*wVog-vs&- zXng(FeP)u=o{0+F2sozsD*6Y*3Y@6sgChL;`vF8$ay{G+>H%yFp z+K0q4lV4Kfct1pcV~_h#EL{Eo#~+s(PZK&scmu`>;Wpy(!8T-pDM}&SQb0z909VlU~gZFP5xN??_2*h z#-Ym}14qaYM@$gWvI1K$26pnBC6GIu4!0{A@4xj8rcKPiSEFYkA$BCQgh?SrC;MQc zJ-#m;3$Y0JhQjMg$A_%rv|=)e=2-$<0kR7F zb@R{Z4Z%&tYd+fl#KWbA8OWo3j$caS{m&2MS7FwKOuRE}9DJTsP3a*JhC3JwF_-`= zd5imFxqy+1jwt@po{f}f-XHuerjE(Pj4yx0Tj^ghgPiti;J&*Nds`s`X^k{{|?v8f53Iazu~6w-*74V-*Lh8XEd4r9cQiog7c<- z$F=DHz*RlTR=4Y|3v(&{{nZ^ zOR!kXFxaeo=o}8{oB|Aj7iKZQdkN}ax0J|o?t+-aKSe1Sl1!}WSV$^?{z14x^bF8C z7{hB8?KQ1x94vuESc!p5?}=(E1P^&6lXXfeGlFSvLB(j=9;5etkkw&GhMHurN&cEZ zVDhuq0At!;(|SVyuuRv`TL4Y(!^ANDiHS_#rxmryo_Hu8^3x_;lVo^g`VRWeUyczk z{{Kb)yqE*O%{kDAhtk^f0yzGKg|M%qUI520faBkSRP%p#;QZGHG?CbHQUW}ISeOJe zqTT%wDZavBm|oGrq*|eqEifo%^mh|Lri0#ZKz}6?1RCl6M(DgcM9NWk+4&lxJVr#w z2H1jbMA~#PyDYG~>~Oej5F`SxOwhSB$z(bwN{WVFGD7dN!7h7Xv>Rb{L^0qfTEg&{ z2SSM+f{y?;DLD}#Q#($WJx&JG z%~Tj6^xgln^?#(t!u=m5n_(0z%xX9JO^8tXBf{GcQN*GaUV+7H1OxgmGfZ|fj7|&m zE)#exye33AsejxsxUJ9&M(D(7L`%dBCqP;cBe9TW6O4+PJ)X&foIrj>pWBpL9*l)m`L(kG_Z`gNuqhI>n#?b`K#lDp65t85{7^YuZj_f@|qZY z)+-kDw?x4qIbkO$Y{3ChcEK)M`1z&lV9?lO0rkJ`Is4y8AirRSMOGNhq$IjXK3N1v zP;#CaRdNW5KM^(|tRpy)kgg0R~ZhA$kG! zVOvfCC+>;ulUq~`*`Efxq_LeH5_v<$K@E}gRv1bkff-Z;@KW-f63KK>LY9*Dl=K$W zRF-HX5S1$TN%okMwtY!PN}zZMsFFhPg;SyUlOPj_L_U!oty)?RBr|J{uATO1#Guny zK-&Ln`! z(o6slCH^H!U0s(*$B|bD(2_@2q zJtLCjrxN6Tpaebv=>(>zF$6M^hxGv1r5I*u5Mz(Vkv%kJ0`aI1k|76DxIc(Rr}ZgG z$tE%IM<6)?HovxgVm`6gGlR!ke^WwTldX;iv8vok7@M;SnLS4gFq%6Zmv1w0Zj(uJ zlYn?1VjQpI*xGBjPyrmuzK)P{I22bL!k#n)r6F+1DX%$E#BOyWeElNhamBuKF- zAOOwa@j^}@CKeR{BtxVmhcXa8buc7_Sl+R)t0avbW`ABeHW4-@z)8IShrKudvMjsm zJN-xWYNLTtWeu5?nGtWE=R3SPrp(C5j2t8+3Is;T*kHHYjm2QoU~Jq512zH-mSjK( zY-oWb5Rx!hHpXZGNrpmJX`tp&Dow^1>)Y$I_dW4mL`GIlYH3N1AMU#G;@x-8x%=#M z_PP7JzkA0-`7?I(8D~^hTMS)mM*v|vSt{GPCY!qkCIXzc*EZA~uc^y;TiS2}DX9M% zf&;dl5lvmQ$r*K~n`+NC>{|dt0@bsz*v zPp0OuZGf?WpuZ+=#Ffc)81_4&+d z?-?UJW-gsKhDp1R!}^@nrKQd0bX~Q{RPEWC`fJoF2W%(Z-nL*0!$QVsmriyKm?I4_ z?4D@joT}+W_3>EMVXF2FV0%k2K^iG6I_mnKHJ0&U4gFkGhg!}Aby#)))}Hgm?x1~G zJ2%(zwH{?D=$K*V1p3x_UZ%nUdZ`Mcwn3DjbJ9084c~ zGQfSX;VGJ9f-Ry8f;mpA%p|^KX&;d#GHXqd(YiW&B@Ubc6M#=ZetVyQwaa>neX&3s zQaPVBr!W6!U51@(*Ta&_YR@P0^R}k;CtC(C0alLJ0odp253xaO`j&6|n|kEQr}XE4 z>FMwP`A_|8^#Htw0IwMsZ8nTPu-?ObUNbf+AZf5a1n$6k#^&HmB~^AZ-?jC#ov@yr zvpQY$6C0DTmSS5{#tD6bT}4&}Y}Gkq%=29BjCIT3orJcL$<=me?|(y{kF{k>!}&!6 zu309#&KL`qo=y!t6-7(t<`v`fGvbqzkTVJu6T~aMGCrC4CwIk11 zU>3I5`e#gQ00_WOXUG(Dez*xP1*_VnQ@ z`@G1gz`KTDL^J>Qz4FDJqrvd=&&$skV{TJ%D@BiQRKYwDPPyN&tz4eFw zmIXI=NQDd77S9w>Q`^?%TX$*#9Jg67NfQk~L4XNC3xIkv>>TXhsoIotnvB(6n;Xbg z86~QXQ*~zZL+wwrdFF=gPj|W{vURZcp&t8SaNgFtdGz&~cfMZh?QhU_|5aM6eWU*R z&JXDKe);3ZqJ;hVn(z2=vp@aW_AT3=+p+)H4Lha)MrXbL`Q*F>67sn{-%@*%a#%Ap zW;1oyw#@!mzw``WtghT-eWRRv~kw-4~FO@y`%)&+1*h>T48H{-L3`fYf+|V zenx}MZQFkmz=r+pSZRw(nvK3r1j-GgOU9-qNT4;JTYw<@)v!iJln)xMtDVja0PT~m zeohvW4RzTsj@MLACu)wTqQ8{*Xpd*NB>zSx|72~~(t~67lhLlaF=YnUHQm0V>R@Vn zS+nnh^P0!6&}RFU`swd}s~-Hd|E@=VLwfD^zuD%0XL|OgMab_95)Hfuaquon-?r6Z&tk#1zIV7?KkCu%!x$c&k96THM+f{phCS~Q zaQwUxG(k{D!0{1qd;}aH)i`E7uo_siiK>Yam@h3 z)uf`;w65H+rIom1VDU+q%Co*sMRlEyo64sx72=wzY2BE=rS6JK-BYSWd8-Fo8rB8) zwvtwLYEV=;ZL8YrsN1P&)UIh>8R~am_g=l{4Ug*mum7yx^SbBso?m!Y5B%m|>Vv=a zdANN%M z3RAU^0Q@=lSBqL|#$A;=4b@;2cZ+JK{igEmx=Mqd3Q=GA zxTiwgRuQ&$0<*NBGGp8CTKl=Up#oqgK>^We)u$zEzY4I}%cMJEl^2VgA&{r2K0pe=-7~5A`_q^q~MTql=1$7c{_0I5}r*He(=LN>Nxw&ag_?&Kpomtvj5) zFowdWeQ%$GV{y2y;T%A8(-^P+`WOGLrw{v|(yzV!-RjZiWLGem0W8h26K`m=e!)^Q zv#Ioy)UdO*vYLK7O4lygZw)*howv^pfbs&oWgwli*Kn>=0}Viwmsfy?b87U~EyXW= z#(4`<3wpH)XT}GCU>~`9} zkU7n%nFxD2W78TK1Z&V!cmf=uN3Z@{e^2lJgAZEm(uW^^&h(`ROMbGUXu7VA?kn`Z zH$J65dCQ;cKfd`YwV^$l122x5XcyDjDw{Vx0H07#!%riz$xFwn>*^PZK{~0o<4AM zTZ1`(gv-|2f*Cd2Qe!qZMpb_ewkM9+n`}e{O=UDv@9dV!qg1{1O&iz70N`pzDfda3r*m_3 z;~ZxG^#H)H@891@NrSf#7_PWlYSzX@s5oVDOqvYl9I8K0AQ@EySZh+MYOi5`f#G|#k*;dvwQY&`%i0^Cp81ZLW`0m`4_*P>Dtc4w9}Ouze6Y>u5N+XaGO+0X;F`cnV8# z!`B6 zo2+-1>XCpQ(n0%t8Lh$C+`o3BZydIMCeEw_kR{-RGmdM5u#gwQIVNL*v|zhZXLf>2 zSd;hLRKCCZ0K6o?=c3t9fLCl8dd%8MkOtr-_7A&aOto}d>FkmRQ>1{I7u*J64<@Wf z3;W{%Jf4oy@8=BHc*eTDqYlqL+*TU^n{hO!Ho;8Tgfr?=cbhfw(7y9LUMm;dgS?jQ z5%y`J59iE22OF`lKeyLDI!MNnXtb+lf77xQz+?{@H;^4NUm0lfv4@}1&%FK@ty~7B zJ3I(t!`8D_3~*1|7d03Y5VfVrcug%>zT-{Z-}vA3k=GRT%#UsAV{iCB^?k4Y2Gyc- zY7Q@&-V*f2xvL5{6R}pYCNJ9`)@Rn7>)D^?x-)hO*6Q`QwHDv6?e42|w)Cxf&l?`odw=E$ec<(< z)O&vRalQ8!p3q0%`R97zU7yw;zTuw|2(9PMOO|w7UH4kLERr|}D zYU?UNWPrpl!2t=e&Dhwr4SPRmuCLjC1)vdm5&RP}aS#jzc>Lv9|H)D@8C_I&;QM;) zC30kc!Vztqwct7cLO@Xt7`i-PeKfcCj{e}B0siP?bvW@o$of>BEtu9b1J-OGgqHk>NjK4$=Z9UYyoi~rl6Fk-KH*_g4E)v*jkl+Bn-Z69NH zZM>N++1&PE(aiS* zgW3iC%O7}){`761)q`*Q@A{b^dW*>)9lV+O-`?7==Ri*EOMLzisC-60{4ZW7%33_| zvB&gNuYXe(M7DU3!tQn=g1%V)UZ!?RI`<1!-@QHCRd2SV`>QY32fwYX$Nu@Y9{9s|8|JzOv8nbJu;QM^M>)Q{_9<>Pv!ey<)czeLYePUz{B;6x^mO%w zdaU?DJy}1lXKE+)boo9#R6V8#+b`Bb-LKUn%@^t8WzH-5ueBAur*ukhTRo+pJ#|Vy zaB4+=W2K1XyNuieUrC_y7{h*lZcIm<%S}$=ret8ndYdLs|wE zG9OVZJWH8w!>?U786 zbl}Uc1Hn(oUigS{{5%U_5Y!QHd;}aH0mnzcu{nZq!p=s@4T>tl@*L-sALg|Z6$GQQ z2v`?2jrmzkIKxb?3eXR*tU4rZa$Tjgt3uM#$zDUN{3dN!?b+hsw6R00(Mc8iCshKF zj%r#N)Ri06R9dTRB`T^CHPr4^)Nd9wEJynAyB^VlZ~vS=^49;U2Y>4edf*+O*ZY3+ zX}#}vp3=X2=lk@UzyJ5t>{nIq)l}+Mybpj`0D=?0XBn0+0B~Gab5K*4w8w4OoDCJy zhAMH>(r`Bhq%f{q3TOb=dQ`WXz4c*P<#<)4>?j*Ux~$RwKo$_QX{mOtR`9T)GGJeH zO8NM-ifP{1pVhRlY7(hP+UT^Mv3&C?#d%d>AV;y)?#=hBo^pFsR(VuZZg5Iv7|XCs z2SsC@!m0*DTpd@nI;-nsR8>A|D^Dusm>ShEZp)VDxEz&L9kx}70Zjpp`^LB}Qs0<# z-Y}^LHPxfOs!>mcepA&^M}YdJ6c+QUDzM7qRaIb6XAE(G#*=ATCz7gG`z$ zDuDm9woWGnQFBu5hiW&A{Qr!e>lG$^V#EZdx{44bCgJkLrGhA?xqq|@n&N=ZfSr04aL z+f`Njbya#*HBwTk7u6V6RE}~g$2pZESnx%aV7CJ-4;m&T0Bv-lV9f2lPW78w88i&w zh5=n3tf-!>sMx(%`QCl5<8f81FwLWu$-bI&Z%Y29m=x*gH0*QcXv zyrlwe^@s14fu-#9>+J%2bjzJKAoSpv$1hu9FSUbc0tW(Q;j~%PSW09Qgg6r zKxhDtX}>v`n**Ku0623G>d*r6;V_P25hm`eH&87Dje`@|94VPY|5`5@IK&6@jOm-z zXEavTa6|v#2Y+PmIUj!V)Aj-l07?qZ@y4#Pi>$8K_=17)1_;YNCH0DQo$`6bmOQb(Rs7tmqtf^cNEhMQ_w4bHQcTAXJ@K0a13H!s?~=!4%E z&?Xy$4E%5q1VCoX7r=q7(riR{@TI-uC zug}ck%pQC`eW^yI6&_wTU60nzsWzIbyS2H;h;2>QjA053ss+m$=jPZ`KT^|cm|js7 zF<$@@Ut_u7 zCiOGzr^Aak)jn-=015&2pm)@%1DLOo{*V;TGuMx9a!$kP6%9y{m%{9U@vx=F1ZUGZ z^&_8`IK2U!tJAS6a#T!Cqva4 z0QxU`x~`YdzBt|+voJ)+x~|>_O>tRqrdqDJ@EEF z*B`z8llsHoeOkZ!_W!Ey{qEOV+HM@aq;E8){CwLS-qeJKeQl}usar=%_{oj|4(1eI zpR0LhuIglDtR{d6)<_(<=q_hIlN!~=)c}q)bNJ&B2b>Hn7}jt=L)Il6psc?RoI1-K zkc#x3tOa2m_SF1jNC0GI-6G|zr>O=!###B9zwnyGpZ<$4*g8z60&LFb|Hl{fxxe^J z{nWpDgW|~XAz{pm#W-M43 zd)QRkW1iqJryNB)B0cu5*--jUUy&=e((SG@o{iNDpa)xee!=Vj_7=c;Fx^!&zo=$% z#@S1g^Ja_q%$g96z$mr(&YVBw2<xGb{2<$bjoV<;^u-v`1R0P;0LC?TEMs2-9((F^)<^;Z98eDbL#~YJ?b*K_ z4c9NLF<7?%9QF*CUhysZp6~ysY7Hiqv82%(nVx%4o2Ldv{szoja11rRH#MG}Q#0LB zcf6zfqtp7xk6zKA{_J7n*#bjIut0Ze2U z!448YMvDEd);DXr@@jqLE&t7GvHsaxzo3u3?SJUQzxjE6@Hd~)hkoa2z58wdRsY~M zueCbNfP&~~=N9Y_w)*z$Zx%?yUWrT)*d^@G%u84R?)BEUoxM#EC+jq9ZuaDSPJJC7 zV#Bc^U#j&VTR}OA=%Ob5mufzIxs`cfU1g62FiZ&!fI)%?02ZwdD?y(GGqFD;JJ0Z< z#>q=HOfPxalKHl}>szXfM`~}ZSw@pKwue;C1kJIB1Av~NQGde$^V`)w)_xeFWE^P` zC^lMCn`{*XhVibEGQY^;#G1dWcDilg2mId_P zj{LDF?9aLczFOevS@nZgC?S8e`N z)BRTFKQ_~Xf!Q+j~>O?@{Owd*SYISH0=WY){%7 zuc?84OwQ_Jr=majhp*CmfBNt0?LYm0>3hNBt3_<-rP-rVPH4QL=4?&%Ib%$=0Dl+QtFJuv_Y|1P2Z3WxV|`A|$pzKP zG?oRu2A1^o?8VXdUw*LfXbg_V;ESaL0S`R_j<2(c4uU!Yj*o!jBjET5IA$Zo3vY8e zHfC%tDLT`0ST;szB__3VNA*F=(jeD5Wdmz_Nk?t!1$Qb|hqc(Ls2X)tN%~fEG1r4V z*;W|PnDdOjU^2TT}kSy z#w}H%o`Jn}SjtgV4S?I8gX_hptlGF{zbTShx!+SG8mQ2$svhy|1=WUCRnodD{iZ4{ z(scJ!kDFR)=ZrB+>gg%~Zsfok{V8;dDh|p9!U8UmE_^kv7qnA|P zu-d63oHmVZTtM+a)b+uAh*$q)Mnco5wmk-fj0tKoTFaT&@H(3>_n;msk9mZn**2f*? zyScsjTkhXyDcD<+EKnqJ2KX(>f9UcjfPh-pwB5&91F$iK<#<8K^rEG1BqASYGoTn? zD^6x}XiPoDZ;azO1CMb?!Vtt6j*}TDDGq&{mdT74_D!o-Ni_JMeDqQME$9?WA;HHz%%hmW(f>5zG7ubq;bnH1IM6~;cW&2F}R*un-RCSGeW zu&=^704O-&0f=Cmaz=(Q!HhLP%{WW(xvq`z%k|5zeYbw|mp-I_^QPZ3r(0)6DmNG$ z&I+oK?sD!aNRwzs!~PYG;>(t<&YWM_E>`l^C$126MZgkMw?$pb#L*fVQz0T08fpJ{6ATApA2 z^;XmAWKhm4rRQ|M2R8>7cpuh9EzEqJD2im0CqLV81%DfZ*MVpJ&S zmiCHwp_1G{2w!7N>A0=JKtY>90|4R8D~ zPM>m#W+b$(x>UIAhhnicM~oI{uLQRBJdRA1Z-t)kr0`D6;&u@W!~zg8Ip;ia;go@Z z)jd15@I9AreYTf6k-{dnYs0veSY1QJu5s9iG8?CUliX)A6j+gF-rH;5%u5@(uJ6vt zKFIl&AMDEH_Y!TM7+H-ga&UGoF*5;tvZ=+tO}xQ==|8$WcKXE1gVSukgqs}*`a$?A zpY$rpc7B<&U7pJhuNnU8@`IT%KDFcj&JLFnHaahaLQUQv{qq$8mW{`JXN{p-I4X!r zBpgFF3k|fQhh}<$UQ)^`=aT&7%SeZj$P+%QU0ctY*GNDpA3|rxcI)T#bIf~;VMnDK zRi-MHlT@~K$xLx63YIi_r;Z~V9Nd1y;du0J^|Jl9avrTChQOxad4C&un#>KgkQY}j zg?lGYcN)iL{SAE48OIZV$XII#e3r|GdYY0J@Y7NEL7dmKt~e1#E=HLY_d40>XDJ}jP`X0QTs?|I^uG$tmXM|>$5Yad^p~C9Jvr*5 zYpPMZs=<6)vdc+5@A~v5L5;+snPs$9K%(tb5`H5!|1G^u^HYiwhf*uFPP4>q zYIN8!DsIOwrQG+C@j}F!Ew?Qm7@ySN6#?|0)DD<@g?25q8b3c^@(CfyL&ro=E0Zqw z5Vo_);?!hN3I+$?I`pKwI-2F^iEaZgtrDd^ob6>$3yC-Pn1gFQxoWVv3$7RTYRplSBT4kX@Yu_(a0fZBP@M>3 zZMJ{#a0}zXRN9YAziNhAKRkth*>E7}Ur=)1Iznz)i8SAGcgV;%O-n=`(^ozx2e zEQgQKMIv!b@dei6mjm;l$>eO6>z#a38lfqNS#VM`5d#gW3|%r=C?uRz@X4MS(Elx1 zh2sxUd`a%Wj;Vp?da8HI%FD0=pe=z5^m+DZ_%VMul@H+SUb6aCT4t7$$&ozldFo0Y zHVG-Urv9PGx?zcaZBYwT!s{Q$RLI&ZyedX2-wTE5RIEJKKmL2DVy;+_aHzAD@Fjg= z+;t-%-HK{fU!r?UXT3HbLfrThLo_6ctLV4u>fYAbYMT3$Es}{Dk6a4+;oFd1G?y<_ z60lx86LaXh5*gHdEE^O=q`UZrwThWyK8wjdndRG;&|w!Nv-2S81(LMMw?f4!u)s>f;%VbLVDOD(UG?&_CFaCB=&aff(qQ%sJ z`4^b)et*2aR$yGYV(HOuARm{(u+yYW$FwWg!NOy8A&O@BfbCu2-Ggb`AN{#cB%glD z`Flj0;L(HOwu04DRlf9p)^5*Ky0$y=f6tf;uoD&SI}ty)Y1TA5PE3Gap{en>!E<~h zt02zaU`3;sy)j$A!GqBbDA)4Js72Nek1UeVn#b{KPOGbI5|RH3{9Sfk5tD!=WNRG*(LQ7ri?#+#!r+i0a9bMxv`U`dD)kv4 zF!tTQ?`JM`+fB;9z{!1VViSO$+8&|?H+n6t1oCKR7h_P=q$*qgvX!zYYl5PUh z$iKcZ6TeYP{7W|DIZF~BwLtYk0lV}15SP*kQO=@F!c9ZqM3fj8jXn_RZ@3fq(j9>n zCjJLvW`vmH@#Z_u9p1EE@|ACBFPj;L&GfvOH_g94_5^6Hc?&qp1*m!bD&P2ZTw^({ zb8{Te{Oe*IQcC#frD{~fkC$z&!idj572^TFGF*<#U6^Ks>pmG1QdA)b7OySYe37>1 zYFirlVs9(_n&JIz?}nmh4HeH&&NcjEe&3-cvd*O!9(q5mwt3EpiFKoig7ZTmXZFWz zi?iwuC}a)>v;6?|FQsWCNM>y zq3i1*jlVkc%X8~;+6jP98FQ}c8G%k|;D#cVc@a)iD<1i;5z}$jNx~V(f9aDg;+lu0 zr86C2q}xGH`9SU~5aq|chsuMv&D`9VYK4yG^Q;?GBTx-Thj zO`n%9vR9ew4gZ#ongk$xu=?T8!0bL;F8Pw?Dh@xZ$7+1@6d1g&E@|(kJiD7{E8NsY zFUip>Gi->^uUPT1Sq5DFo48C;sPVqMIMV&mP3PBl@Mjy!r2-ZdjoTt(r7H~w6dj|F zhjNQBXa5ro07aOc?kYmrJUZtShs)2`$EPd20GH_x;qiff;~Yg$>G{^>_X zeXKPrhR%KTGTm=oV1D>Q?3T5$z!boY`@Iw+UNj4mTd7zmQN#!Q^jde;bvSBborjk8 zy&xd-F!s%IUn8*DzPQMlAzD0^i#sQIEa;!;D-ce(hVO_TG_1?rIgt_}HP7Xip-k(_ zTyPbT*Z*Ko$qRw#n;iF8n)5zV32Cf9LYq-LMvwLBm4hTdpq=%}K_f5J6rK3XCmG{k zL^1$bC1IK`mKTYM>bJOK%dS`H)0Fo;NYvLPeED$}$fTLl-DVT{>q!miN@lH{`UcNm z^=OB9gNd;aiidh`W-DWW0!lc3q9>jp#bP7lX4ugmA))2L{;c>TE2qJQ8gIB z-lE~FlXv(9@Zm*aob_11Av4dvz~Tg^2kyPQU#Kk9sQbFPwA)LPg~|JINL=(o$K078 zszsG?!+M)Pa^&QPMP5mi;!wHw_P{yL3lPQW=WS%j?wuehkoQ&7dJ&X5ZM!`{7gBFxo?!#ngK4Ym*VS+>$;01Xsi?V0>H( zaTG*v|HeXjfDT)o{6=jqUQZ{Ck;%F;Q5K7}s^n5*FBd6rqkV|)W$1`;^0x5n!KB*A zR|)j}>S4qP9K{dPuHRNlq>1%_ayR<5gzZ$tZ~sR=Ktv!z-*o+IPMgjcfgk3C9uqtV zi|cz$hXBf}adB2i<2Ol-LPyBr*ffBqlcubVh513tX@xs{c+Q&g z^$Nd={Kl6T^&Wth2)<6{F+Qvs7F_W(dCH)tw)YG;t?M7V(vX>*;qBYDZ=>M*0ALy% z&3g?&x9Geze#10G;3&bhwEzh@G@$)9rNxmKPs9%RCYC5m?vF^#ln_^qT>~Y}YuDZ~ z1x&sS5X-ga9;&p8ZLD>%2jO1`1S_} z4gKjemJrEVnv5L0FyP6@+3*iGDetk&TAv|Pqlt`)$=9Q)5g@)QU4DZPbSXSlYGpoC zd}Iy2DU@yP%%wZDv_R7Y3A2$K+s~xjq5AN}hXETvkX(W1Q7;hXF>zL<)^P^7ypT@q zeU@F_A-LH;%gJoP?Io(k=GtjX-vQOTVHe@g^r1l|_J6kJ`OsuyP6)%2GqriWIR0}! z-QIEgZ_FhOBZiY<;T`>QN4Q`JD&CrXH{H61ALzz7{HfO?9*7!&z64dBFP~JS z@TANC-kjgW2&=Vh@AJ>U(wZLRMc}H~KK-*|5>*HIA2~dw^2|2;e2tX2GkN%`F9#qs zyA+2p;?fa)z?eI=aI!ApAa<91yB+s0DE_88WMN@8p!!_nNL~bf*n= z`o8cPNhBmSFR&UN3T@Y@z7|{WKlKhldTHl~&wY1_k-&iS+197`vFl0SDn|=TBu0ED z**2#CT^z|ybX1I+O!z8uFuUU3c7Dhfr zE6A+2*r%<1bx-f1U!^A3T`&+KsSSkyf(wUtSqL|$i$$A^TIG8FEI z5ILmSuFU-la*_jA(qv>!55nM}XW%k>TC?{#g@(^s^Ik6#Q$m>966duS@{ zek%q3RoT9*pnA~|PNKW};Lraqv$HE6ejJ_^ESruo+zKc&7UuWzOPu^h$1j`6+VSo1 z&&12fus>^G(y3oKLpDo{x0URkT=YB_UtMFA0k7T@9SQA@|AX+EOdTYu^0OnLX5AA8 z4r@{OFXU)rhTFRgi4*#dNV6_S=`N&_5|)zW&D#JLN0(YYW_5Q0l&%B3KhcIPvz8%fRY*$>fg}#^aZ5R-3C|%u_-~pa`r#Dwm#0U@_!mG>mQLcy1v( zQZ=((g2``}wvp}F`V{uz=LsEmK8~m{-7)9oiO_UepjDS7 z!GJ%-R+%?ry%}pKJM6~5Kf-D|*UN@BahR0AOs znXB!9EQ~G?HJbHFYJy`+81G-IlC|;DvKkN;ges%gU~a3~oNz8E@GoHbrtvTd)eu;_ z4@!MTI*O+W(oo~5L~c$WVBJqgufs#_{sB9>j*&9E+3_=B^Cj%Mz z*6viEa(WT4Kxg!2T)41Qb-~Qa9&3e#y4+(u)$3!Xef+$BxV4O!D$%z%bC9w|b1gu{ zM!6*UpYmW6l6r<&*8Q`4W21_~1dJat`=iB=XG*he7v*%I=foahTwEH6g8q3tECdBo3P3;k! z8M%Q{JLNd-rudV$m@^}nvM=wtHq^gGuLNVbS(5K}aeF8jGasFws6i(~<>H4`ThjKt zPH2?7y_0E^*8|n>wm2DEj30jG?3J_>FY+u_@HP&-*Sf9Ag5WsieRfaX9?*DPu9%o~ za0K^RoF=65eUHaR(Ld(?CHdL=d6t#Zajbv@x8)l1!_`EH`|Wz046``3VVEBy6T(Mt zQh;2U?C146n!?lSL7t{) zy^myFz)1TvLq;}iLRJlH>*mV*DguTx7ga0Z1zhWlfYpUs^4YMj5Iw#_aSPCpKQGhv zU6?snT}Ns0;EWuj%VY{SU%-es-&uh z+VhXFa^mp`R;|aWnmOITulSX#ki`}W=X6d!>o!+osi4MC=&4LpmG=v{oNgb~l3>zON@QA`&v&=3HJM zX_Q;;v_#g8*Fd{C-{PJR10DA;PkQ7B=Og?tWdK>l@5KBOo8n_w zz6*<}UPp=&g=JkzT@8#dz5WxJ9^;nenU>)Ebr1eAsr_9CFkCEyNlq$?pe&_hqSY}o}>g(HQ}cx^E$qR#-h^< z<7+k+;qLDwubSC9xuN`RSOW4W@>(mw7UVU4(?`iETw`iSpmyE;kRPBYb9JXQI-_bB z2|M3k-1pa`q?zph-64W0dbH!Id}~7(i`68?x~{WY&&FFPc4#XT>_qzTQ>EQp-GL+O zqR$DX=S^GV{0X|b3KtqlJXvh!Ji5}}UgH4F5yJ!z6{tGI{Z&~T3~s4l27v^Z(} z+nBdse2zZB(1I&JtldF!LM`(ixrx{w8ZtK8vXspGcID7?#Y4qfwLnuiVB_=5`~?u! z!0W<0-kTAynHL8kRnal*7j-tmKxH@gSq7GLa#q9Qos$nzC5TcAM%Q1KLv~;+pHA_h_{$NhK0eTrD=` z@ZLD@y~TK^qAMEn1L*(poQFqKT45}#(t~HJ3)t&a*0a|i3}HZCZSSpISUVR%zD9LM zdg$tPT?9Rr(kRUN=yAuLGjWhB<)|pDuAZK3@|}LKS3{xL4i)`*+PB7W-=>V4aE5{E z&2Kj6^&8|XA6ZHp3jj>r)ywC?X==V*OrolH#*e>ocB~X7@wkH?7xQI?Wzv7}x1bj_ zFE>%|E;}m&S{ao_Gq&6sNeTtXVaY!Whs}82s7sP9HcX5*Ad)ZDj>tD4iwx@nG!H;_ z7#7p?pQ`kh^n;gdN;zG>8TLv0zTn)wQ(GtJJAc#Qfn}>wIn@kdXtTn zJ#k|vyA|*=EuxNrohj8Cd2hokx=SE-4)=+mxBH0RDPvoxPYQz8jHV zoSM>+l>cw6*V#@7(!uA%)6#X~wpxZL{?{Qj{6y{j9a74EDfXH5CJQeJddij)_?@XY zbj|B6urM0yG7F#o$BXqB{s0TEmM@4|^RbAYV}>M(xl2^2t(P&Qf1hlwnVx!7^4Lw& zBwB7=9$t3$QAh*hW|CiIAJ1mZND=eP9eLt8)$KsdDQj1#bc0riWagVbOiNE^ zJ2|vb$IwQ(Cr1>gyQeCDaKr}yE!9wkl6kW@Ua@6Z-nIn5FEe51a(6MBrV)@a%acAG zc*SxJ#*Dgcb)cH6Is>LPR+7}l{o+=eTH9T4AZr#}0)FuRuoO!S@}Os&qLQ|JkD1V1 z%WQxkbxe1)hq@A0bYOr)J-ogMkfX(juA@kGFNIQd z8rY;}JKPmMj5$}~T14&nb-8=7Kh7ytg<1`l?fpGrsUT_PFby&u1Z|u?tyXIoKeLrf zU9(7@?L@J?yX19`se!MmQaRP-B$&Eqxjs&;WLomyt|iP$t3WX)`F^T(OBX={okM$1 zITR$7N-@XbE=eN(GCe61JllA>-yi%Iu*h5n?2hi+tt#+h5M}Z6B#ubrMVj}S0u)vN zW(s*QM|hvVm&lROxR&p9&zL&1%lj`#zgKb7bKym%DKv3I^trW;Ii?4#6~eGqi)uBH z!@Js}6sk7o=e_VYfGhtNBa9v=f1JJ*W_HDfxlA8eik?!9kbPK|u#h6LP1dFKHA-a^ zI|F{sijWC%oEb|nUbrd?8JzKI`^^zdPHxKS@K})K3Gz!o%m0LXw2Ug^aM!OzGJl{Z zfCP88!;X&B+DX;a1R}8m_AM@_vZ)TvY&+^{fog$eI8k~!T z9EOsuZ}Q{@GwkQ!zMOtmA$dX2yx&6Da_#)5Leq$$)x-1|PC?I0&Lix+fcp*jN66b% zSj!9MS>C<6ASf!Pn>Xxy8(PR{s>^Zk%4iMWHAR7pBUOGFUaPNFWXD#!bXB{wJ;#~4 zuE)ittImjRjd!Q4Pyb>dI(4(HZ_H!`)dZ!ePV{dKy*9vAn~&^1c|G~YNGL|c<(AId zrDdtMyC!1|CiLXq2t&_Q7Gid-CDlObKs!_SpC&E7^X- zqq4GfpH?M+T)FAf#I}k0X8m*RDm3rn@;PC!0y%t3^$4l+djub|nxw#Hws3pLaH}m%Fe2^8*N~+sms+uJYfeVlo~1~C z>!A}mzXG<29J~vFk`-d3jr{Ch-{06d!&-c7)J<#*6m}+N_wQiH*d` zYEl$+$=fS9UX`zZ?K!AD$gjgKIGJO!J=oyFDM$mn^%bsr;Tf!sM=VvIa$aJ@6 zfPizcew;6BNGXR%sGsR@HgJB>zUuI_3U6y7;E7nd`>+Lk4a45r^F}C_MlU1O=ZGKV zqyZUMQ5MRDp!<~>7`<}%wzWO47%#4ujCq=+d)cLuF8egjLpFP5$9rp0uf#j(D=&4J zn&XD18F#zSQ=lW2p^7+C%L;4?7Y%Wa7*|rzos;^j^L2&N1P+~U8j?i^%w^~KG;MR8 zz|gAJF?7C6i~nZOsPpODI+R6aRL^wB7GO?1>+?6M+q50g0*-L5Xt=*3JuGxS)UuX( zKdbqrd^Q2%2~YE8JmyNRQTXL`^n*`9(W~OuAdRwr$5kPb&$~X8SQ{rDyl3f&1EZN! zetN_*VEmgmO9%V#@CpuS-BtBO91#>_H=8P`km`)(X&DsB4A0h22M^a#gCfU=rqPFx zpZ?2=3diDrY#M-UB3RNMtnTc|o7OJ2ys5JUAkPO=s%}{FQv3v!ap4*te6Fum##WS2?BRFKTOCjDW?&++-O?>j0 z_APCC8_{wpg-=$b)@M8i_arYd$qZq5GPehUe0Dtpu7c%Mt=qGk1ag|H8$AID5ogWW zv*aoer+6qE`*M%R9Ox_xC*kXR9;|Q-uVBgDi_0M`JGu@(Qe%d0C8J1wLWdhwL;{)Q z1zT}FJ)=ZY7S>Nzw{iQmyQHwdq@k3pSv~^E77Z^FYg{FtuOQVb3gx98qpg+zdy1Ww ztx{2Y8dU0js9e?fSV2;|Z!bm~Q@Cnx;pZ}>U#sfp6??+G__bi`Q?x5`m1 z`r&n}#0J4Lhc!|C{-f)kFPiKln8QaC+ zmS$76tQSL>{}dX}HFR121fLubS5Q?g9#|HrF3Ywq5-N9scB1F4h^(k?uO5Cy z4@oCJ&sTj7n=y2D|6w%NDIZ04tkt8rMyiP4p3 z7F!Y7KVFpe1g~?jts2O#@qX1flG=1@Fc=HSF|lhd-JFRR-f}X`$&m##X(B#5DDo>L zQf*CBjv{)i5~kTeAyaDpQ0t8M6Da~Gd>b7p&p^K@!21ywgJ{hgv3+Ip|7xHI7egwf z3ByM0J`nNf?44G~D3;e^3KseG9}n$aeb4%ADQFg*<);_XF!OHv`*u8C!}sZAM3*yP zdh1;HWgI9ixpx_@*%Q4VM>ZBe8rwB*p^v?7^s;z-Ko;N}_WJ7VHz1YUv31mW(0*xPT}sTs&OzQMb?%qvM=fw4u&h#EWP8;fR1JGa!PkHCrxxa3w5 zQ`~~%{3cs5q|RB13;$()OM$g#&R+3bS+NHVSouxHuJHT!1BMgYCZ@Yp5Si%%Czm7} z%Z&7s3Kho%rdCh0BT#{>mgZG*#$!L;VbF>GTvh3?eGB^t)f+eU&^>%qKfvt~n0JTo zGa(Bzz;CfWTq@4bZoZO{!Te6QqQr6S|nhqAP{xZ1ni=nNy_CG`BXp{{(0A^FRT}$gafb!8? zB%U3Gdgn+)wpE?cucB`>+Qsv7Z)%mzkHE9#l_6_XH0qQK^eJw`W;hTbpRm5#8tm!m z`O!N?(eDU6>?O04f@_E2iL#REZw2hq7qlrFyr5xu2=6e$cU6p^T|Kv2$kB|f%$~dv zXvYcUH5eaCm0ulEu)|_stU-jbDm7=jW@??7?QR{4a)Ek&@z~ZiRFE^}XsB{xKiY># z6Jv66!o$^G3RT?!Ki6x03rse0w`ys|VM_=DX@tk{GT@1S@+cC^0?t(!jMPSvrsfz_ zdNp`W2f`jULs#Wz+as8mvmJ@Uq7(%1;eb+X-UIHx)85=LC6Cl+G?~8C$2|%Ux>7ks zoL4(&aDgZ`y9@ux71ACMR0L!QRqOd$-Oa%OZj;TJZgXafrtJSEu^a$Sq_TH@v*_q% zzEMcGFd!5He@PBs;oeneoOvg=SCDRy`CD@Rk$GR`l=}a&053qvp7|k8wu(j9rBz1Q zQsmLU$bXeE8aP^!cig^cgI~V}q4enaJ9z5P%Fd5NKJ0lSNwCDoi0ce%e(XtrniKPz zrgjvX8#{lI_9v|)lhmN5({oAj%n-MVcJ#gq3(Z%s&D&Aj3e1{7cnS^9+La9mY*T5B zq7VE~TgewQv4#!BJhix(Q8c31Ey6x&kV}nw(MocW`_dut{K5qww}^;q!Z_73Y0+rG zTtz6CkJzVeLo9X(2lMAKqpI#=_W$4n&-2Y{%d>3$A9Ul#RZ>dFYB&~AHIu4*v>FCP zs6B3-<2z#}nPmh2!YP%0BLi4>7ic;-hmj5`_(j9kD$l;+9?kFBdf(PA--_SiS!P>_ zR*j^goHTIq0V&JvXfWwt`yAX; z&3Oyr+ImjdT{BPSL!o3TUDiNKS93EeE29?W5fKbu_qlEHwYn}PwYPmilsE~{!l8l# zF>p{nM@)}fm;MJk*Q)P5Jq|3ccvIapEyN7JLr!$Lcd|U;-tFT6E{gmi>F6+_sdkE_ z*EJt63QXF%uhck2sZz6CIG_7;8lv`2CuHDQoxp5w*`-29#GjPwQ%DyF#h$ zHV-0F=J)4_em9z)_TPsFJOkhF@?%N@BiorxLO=3~wW*`aL(9)Xa8%?SK(@=tm--`WXoH2f*rNoL_t5JqcxIrbGmjt|m1qJAj z!Z$0{fm!3^_pVP9zQ%7=k6lQhL6*8E30np$u*0-}R^;HbRMnIggB%g=ai^GoErWd6 zq5X-qieZ}xpDQ;ux~1?itgaZh6F_v`P7bXwpGdfEd>GPXK!!BkYHKzfp?)h7pPW^i zjhE%q{>m&rnU-bX7pf6i&+wg!s8hKXH zXMS@PxV3vmB&@pkjS>><{eJ?XX-bO(7K;lrz=bJbf%X46UCB@$h1I7YWm1OB-1tqI z7q1=NEB?ClX+Oj?x9WkdU`0uKCbEkil}tY!Hl?5ZPEPV!*2nchyN;)T2Rm!gg?j!~ zR~A{GiQt!!S6}O%yv+@s14lV~m>PS0H3+Tyaq4Asa))2QMr0$mWmvdAsQs{u@RzYa zUMf*cOt#*hsc(==LGz`p6E}pbS!TUIzNl3HH(>_GxcYA5&*qsDfKC65Uh=koJ@CsN zCjS@T?iIePc~hSh_nk*^95beO5l015;wg*lw9;12mr-x#>n?M2*XZ`$SfrF;z>EgE zUHJN7TNjH>yVA5R=kG5D7;`PRyJQODr2sz9g|dkQoAM?{^%hrUzmR4M5@M6ogh=^q zU!=lmI-Jn?cv2_-q+vkT+gCIQo>E$W@`9Q62 z9?++nGm-y=ps>Hp8d1iJ^%baTw&*sB zr4!1(mejUDgl*IJo^<+Pa=1dz@D6k8F%hI7YtUPEu=QLD&l&}~Tse!IbvrMi;FjNu z0z?B+|L)4smsZEkKa8opzI$Bt{C%dw>hF2G8m0Ff_BFkhv~HFqQrzvmdo)RPvSEQi z9)$H+f6+Fl0AE!|H2N?189{dS2=O!j5Z~GP2FLLqcMpEkC-JdO@TF%?fM_#zpZW&Q=TP7np9hVq_k`*PvdSu60Co%uF0e#gd!_sjkS^Sit-OKIa+(O&dBS>Nh z2kV4&WTqRUIT$Avi$P12l%_`#?V%EHc$VUDY`jw_eQvGs(9w8!|fLZ|Z zai2Q!*tRv87YjJuEZoq}vv)ywd%ho42O#X&1Chx)k&aIJy*zmFM+McRxOXcoW zgykW6J)&v2s|6a6@THRxsIDwi-q~%U=aa?ES(~w@yd3&}1r<5%=XTbSU-9j__Yz|e zl~w0W&Xf!q4`V@rwxZA=so8+Qjc9;lfM^WhLNct;bwD{fN4Vo2<^C$~!-3Zmr+L;5nYdA=G zNM1HnD1zbT7?WS2Kn`Em!KVVJTXDTEEUTMO)+V3JP5dX&J)d1MVNRaZFiA+>8DA2C z<_Wn%OPz#!58ZY=R7@HzbXN%z5tHEy`ehqjJ61-uyv2r_B_^At)_ia4W%a(_DxJ|b zDlZf^db1+(*|;>M90U1--Ex^2NSBLgMj4Efp1h*|E~sIy%$~C z3j6;)G?_(`=WKZfzTv8|H1IK$0u z;UhbUNi)!>C1kQiWwP07viY>fK-Gla>KlH&Sb4p4EWvQ2*I%4O7A6HvcjAD1s4r4N zwtBRZ^+ubpKTf5nSwitd0-n?EG6{0TMK;_JESb4(1{Y=WM=cdsQFx5VZDrIaD`$H7 z)-F_uQ4dt1G--2O97M|Z4B2br2_ZkD%(q3jYU|8bY|bMm-755fjiBAR`n=4n7k4bT zm<$O|OWsw6HAHBChE6UkNJ_?0#&o<#`Zt*cA}WZ5zoxuGDz}z8>jfLaUWO1vuzIY$ z31Rh`@FIvi|4Bzq%v$Y{d*-a1hg=UP6xw~p4>_K7xU|?i^;KKvuzzKFNt0rkk*_d3 zQ_5(`x7n7>Ip6Mrfo6Xeh;7@-g8R1>9kMqd@ownZfRn?|3?aLFmA-aHkYc`hQ^7nM zCCw1`p76RMp2;AUNmw5+chawPY)h=4FYx^=Wd(c`;&W$x_io~M zfpHoxI6QQ@gD?uV*TYNL4-p2A%BGcTsyFQIh8m^LEse!}&X_^lU>z&hq%{dmcgigA zKD{v0n{VWg8=*~XB7-;=HkCuUHc#7~rkzn;3%1nrf(*38c};(g(i-oFYS9b*?rpBQ zn%~;cTaz|llL)DYZ*8Bv9*7YD#*;Dw6dwF{isSpsqbea+)ygi;hfnZ&L;i~x(jS_S zja-dSxHd#^3J@22n(Q>Z0?4HNyc+v2srVkkj|d7UHYNL+#aOlbdI)w5}y`a3+H znCK8cD=G;K3oSKsL`R)^Kk}*Eb&CN@@4*}fvpNJt>tuF{8ix!OUKMQ zhLx0t>oVSf!=1mh&xF;!S=gy{q$nayQXSgL(nC z9f=D^_A7RaXm<9`VWVHdVvT#*M{d|K6|&Y`E=CmZeX30~?6_n#f(b>IgSBsqvW}0( zM&j`|$?qH6xliFX>|+UwqN|*I6vo{;nZ)l$qNL$vj}XlT226FrXk7g(2rT}c9g_lh zG7Y1QLsm1+HO17DTIo3+_l0t)ua6|))3&purn;s*Lw!FQwwee8Q&>Iuw*D#iQw1yr|OZYx@C0 zgav^g(Hy;Uw@aATGCv*n(PCLYN zzlB`lK5mpkRzz=q^anAZMP%>3h!WK67>Fe3`7-_j!ADN#>Aj6m6^SnH%WEr05_&0DzARGS z5k5yRX1^lWrzSD5baKAn1WySM3U15CrMWj8V%HIB4HqeX7>DJU)3-SHj|n-{1(&N% zuhjb zk?zF<`BsP+dlf$9*(hC-hL4!&ZT{Kmy(mtNH`fj1}Lv`MF6 zD|?ti?ul>g&GC>8&vZg+d=skSITdr|y&|N|PDdTjEi4hMJC`?Psgdd4Lf-=vBRnl`*3i1Ll$*I%XV2xwFKx`WRCpt;@+s;UBe#&~&uPn0fS!#|*z(&)) zl!}Q`f$r^ujg2oP5^H@DdEEFykuW~2(0S_Va`RB)PboR<*C*>FwQ^LJ#c0Fo0D09M znv(t*1F8OJFZzclgDtXZFE40g)!VVLc#K#&j{Wi*can_ZjJO#R*M4I6ODs ze)pJ0@a2&XR&5eC7@L68c;(kt|TbDJd}@m?zrN6Ote+@c$OWVV8!0$ z<%sIU#rTu9{_c}3St~5?Ml46pqZL*tdy_{Q(K&3W0|HB^YFXBnmCvsH_w&v3PsE6>nWVO<_ zr(Dd%t)|lXKa%e5{}49|!Kek2!ul6qlgDy}kFAsCCb~4DA1`E9tjV%xnLH_fxGDH0 z$fvHn^S_BFCKFsIqkPsFoA)ogn@c}TWX=UNmjBid3nU8#uuJl~wZHIu@^h1psYpv& zDeryj_xE)b_I2OyWky^0=cKaz@HM$EfnNe6M_76?tbu;-o>BMAwoH_j*W71)w*Nzy zC3wi3M)t@b?YwVb3zI7LMw^?@b=tp*=+}$tw9%s2W=KOvm@8yrY0A3h!N%?E<{g_9pV51tEN!S^Hq>G|SpZ@nyPgM-OnST!YhPCSFVd1=1wtt-1{YZ36lb)w7Xo`=k&13`-v z!c)Kh;SmG^vf-M0|LrNb`g}b3eVwHZb^V<@RU`WL!5c~Il&yWswzA~=#Rg)ulyf`6 z5YAq!R0#+FcQ#V~qtBA=B2MNa9OrjD&*R}Vp@ZdOgQr*Cl$wZ7%`bRVy{>yw)yVux zWK5x+#phNM|6Lj3x`sA>=IsgW?OBlsy}R8u13yCv2WGCQYJq4U9cF&>Oui#lj?dR& zsIGLBYk33XnC!dZ#93L@c;h+Zv6bGDwOjGcr@Yt zwcdzZAQYI=P7b~0wTu4ajq}Q+5HaP;?}_KsgM&pMzslnF%!=tVo6B|?Jm;M#7POE8 zA1G&wbVWm;bXSHP4VtqhfjJV+<~!He^<2bOG38?s7(2t@MmMKWP28F>I-u|eV=kK3 z?Y1SV-b!PHNj+r^mdhNdwJV(Y5P#s8d-ZIoMLD6`bH;N77#c`%8ozQ|^ua7A8qh1K zmN>M5tXcjPv{zweR%2K56$5J*a(0#c#c@zLssGH-cS*S7w&D$K?b|4v$5mxf~IEEEo| z0>XIr0rqr;#=%xQx*Y$#Vm*{~Ql{`;&veL+{g`_%9!!?i3HW-iWt<^hvW@Hb58^V* zuFkoWG{WXK$tzU3F+SMiKus_IX?K#5=RTZa@J;V)>e4NBxq{dZv!Viq4$)hN!M$N@ zwK2NHV2sopY~K8j<3@8)X8+hLhmGVKd##NyYE?aTj2q^;J_~`J0vbETV%2O zq!&AY{T=a-cNrv8B;0!#1(lEt~szz^Q^s=^_!w61c0To%N?UfsCWVa+i$*ibJs zW#nGv@GjYBHfiDa4LcX=mdjCq)t~DX78o*y?>oio2ZwF4G1Ux@a+0Gh@ zuPM<5iSKzjq1n4Vr1%)&@1doOHWHCK_dKv({`)B=XS&*Fb}2m@71Oo_Lb()GvR#>@ zlztogH0f=j_F8NYBCyfwZ4eq?H9qTkkDr~&l*vp}GbpYqNS!tv5Rfv(-kX&P$+%x8 zAhY~!w2x-ZXbl1aXZ234*?3-R*+1TbPN#*+fL^rMC9-Hb9)o}Faq3{dsoY4)>>*qH z9{^rJp}!?JcJndU9~!c_j@#eC4h{Sh^nGPFg#-r1TOHAGR{nQ>4B#>R6Zmt#y;<3w z`{gA2O$GK+v7x2Hbt(IXJh!&X1=Zp^YGzw%;q&AZrTM@r)VtB1>fIeRy6cKtv33_P z=r!F%eY9~&e|7$``o!Gh^pUy8>d%)RuMaIgN)I)@P9Lm)qu$qeik`D@UiC&(=j)Mb zjlPzeeM`2VYo?lOaV5IkjP06lXPRrIx>QSaX(`iOeW1(rLYHa-U0lp`alWmE#iqLT zhE|$O`sVJEo}bqBGkH^Q%NzQQbY6eZKCeGrdXhdo|2Tc@@?-SS+++37!sGPN{Nwb| zOOMt^=DtB6n*VxzaPd+4VD0Pd`q0wX=_3n|(#Pf>t%nvKtB)-{Q6E{lQ-8K}r~Y{U ziF*Hq$LsCq=kybgtLeTwTYA=omUb7r>UT4>@{txtt6CauTIDfz0yZ#59`MTg!gpUv z;P^&5`1a>V>)zYza(i9A(lik4c5wU2*(XUvZ4X(2W2&2_%-z zJx+{)$R$u}js?MLcV?-C0r0h!kOUvIt5Xmd=hckoGRp@?z(FntejWS;EWV;nzO5E5 z42zKk`v}IQ7HFg$BY+xC4z09sLe?x$>j7b^aNJ0r&2-|q!Ep$|LcZr9qXUfqyu1g1 zb^tOM{a)q8Gb3|)#_C#7gEqMUzaAX+zz{9TwmsY3GSAB_YpZO*!S{BOwzfDDGXSrF zgYV$Hdu0=jodw&i>W9z5`9g`%!2gj#P zP{;s!-vx#HiKyB711Vuxex-Fwg z#$iE3KnnJZ^NP3t{mT|!fbY;dv`1ze2@kw74oifvbE0^L-`m#10{`%&U%nQhO({+g z008R(*x(mqfp(*OsHi`3I?hLy0MX@KOZI%g-g7SLj#@cMAA1L$!I?{}dwHf;d3Ker zfzyfcczL!JvsfQ>fptrei2LgTaA)l!&R8F#R9$fU#TFe|f6R+pKq z;cw>R&nN>fQ+nq8{_!wI80YoV>^C&!{18AnJ1fqBIpi< zjx5+|`q63DgXWyU>A`OghVnfsY2ZU|8(HzMKac17KKq*Zy!i~?L)N<;WOFiJcaXu` zlM}>t-~u3p@A$o-5nmq*)SEosDFd_UPxi>=2H!`9CRW|Y?}NN_`s2GslO-Qg-3P$a zEC5`whkX15;{1Ez-YarlIvd2^R7w!r&WePuNJ1*yzUxCP1n<$A@SE?6*?RCq&*_&y zAN$RG0bE=ccxL%5U3g%awPN1`?>ar9H$s(_oyA#J@ZUOs0UsQ71>nb@$9)1#J3T>b zvsWeW{QvNy@3-5FzV_^m{jRLOcDV;F`~QR83(Xjvl-1!*n@&4HR&rLSKdT(u%shgf z$e0ZXv&uB@S`c&{dkbyA$2LFH#hj<^G9Z9Y&?R(Xv_X=Y$p&a0J?j7ipg!G0;7LFZ zzb|uQ9{!skCm1u&c0NIlAio1WQ03{*cNrakYiJm`RLYsswq`xUg)CwHtnzMJB~j|G z@_SZMI&BXR-vqljj0suo_5pG^fRFHm@os5DF%ILI@?2Zf6E;2{1OK1y(}A%bz5mR^ ze`^3!>1TPx@GPwd`tkMm{ohXqQ`^DhAOAZj$6X(pu9%UL@KoS8=P~wezwC!5r*vJ? zEaP_mgBH-6Zr7Q7rh3=(s@wRk$5>x<$F9+yKLfya#bi$dzoLgO^Zl9f$PCXwU+nvo z9?HG9@td5aBGCba@EUl_z|I+7@z@zMrM#@$iO&W5l+T@JXU1cE4FAmNpc1rqy%g+C zmrsF4kD80?Y`(`n`xsb%<`U@2=+wYCx8I#NOP(-&UGl-y=Wsh?x_?HG&kQK!y%jqj7=O5%Pwe}&F!p0j z+;}+ZKws! z6jv;6(2VY+>jn;yRSD1CV8yk0qqwKb;7xUE_%*HUYs#dcpyom@-ZftI?tYTex5cLr98Jl`7V zagZyvuBG{T#g{K?`Qmw9J^y6=AB&fDU#q5{ z>Lq%^FxM{+`g&(S)4P&L?{8n$pSGT)Kd%FBKTaQ=e~dn1)yI$4XD)t&K7IaC`pm_z z*QYK#N)KQBdOdvU8}u;$pZ`XEeC{#&lZ89=-@0|ZuhY^ynhm|A-q5$y8|t;YnjZ|c zFmgMYy`s_>el>kfaZDL^biRO|Pkuo&>=87My^1eghrWoj&=#E}RpN>0zKoa)_Vdte zAmjz~;{IS+s8hyWkpJ*=y_jfmG%~xlA7iik(KL4Si^cBXb+=uhA?!K$n%`Eu?(^b* zd^b%QFE-1DC2KYNS4-W?_g3TyATCoCN7&p&B-Uqsa=J?$Fw$gu(%8m!@-J&I-cB|nkC?a3wXlemUkPQ+Hr6y|HnT>va68}rm9cA&J(z41Hg-@pT(m2=RB(v|MOwxEyS$vlT zw^2OSu4j~+v4s(aLLd_vh2Wr-kJlB=VZ#A~!=0BKy)u4R{Qf<6938Z-Duq#mK3|@J zLd6+(!->pSGuF&Wg4b9Jzg!`r6syky$mY-G*?xuJ?0>Cz*Y_Gjsd7f#!Kxgo!}Q?a z_p#3gIm6%Oe9G~Ku~e_D&rxd^)+yX47;rv+Z-^T(jY7z)8< ze&^0&wGT!YeJak@&FD#}KO9@b^GbkKLDl^wHAl-;`$KTf;`U;@eu7m4Tf0v*Y^4pBR(ztPe6ZuYs zd|%ysSQXB8$mWHHqxnFj;lM}jzZHkKL41O)iMSG-l=N|;9l{`tPJHzi13 z@~zP|>u162lGmqv?(+)r1OA534G_twm@g~FEqHjuypHKiao@A-zX8}*Rd7=J1^A>| z(H&NkXUG7PYo08EJod4%2Yz8RLq}9t`I&?V<HZ2}1CIe*O^36_>r-38_ub_#f!3SyUwLLAa*@`p z$U)>Xpj;dMQsnlEymz^W?1OgzY|#lM3oI+`jZL4z=g4|!65WklaA2Kv-BLT-woVx6 z-1cxqEzaiaUss$EXkJsCZ>iNI=;L$(&9u{X_3|s;i6iw+8Qo-8?c{Dx`Wp1q8RPtK zn%+E_{$DNwt7oI1v(TRZ7I_GLmHMIt{{om5k_fBUSpbqiP#LH%_nY7m`#6i~irqw*6PfF`_pd{dot z+mZmI!IpIfLKf6UYw8TVvLCwQv&L{_iJ*Xj@B#KZ@&VaK1s@lINj@Xt1)3y?&+q9# za@^;Wh{N;SMYVp$Z>25m@|fV3gLq;6O*Wvv2>24%A_2mZLXX}*PgJddq3L=~lRmt% zn0w9cM;2c-e}@DfJ`cbLa+7=b9y;%Pq8|I@`|kVi;E&0G1057t*RcPL9a(}Ls^re$ zc?bWlS|gWpCch2jKJD|7cQenk&H^)C84dty@C;*iK+gAjO0x#M-HQJCzu&{=ZoXwj z|F)LQr@Ut8?e;{d1TR6xGX|c6edue?S_J*E-``{3kCrv=FPZE=;3Lj{#v1#@{ezCV z+3WG{8~g#}7x%5?j`OYCW7s9ZJ}#55${;pl@_F6rc{iH>LHT|3eSBSyn6K?$C4Z|m zuI|%jPb_J9OUZCY^>{@Ia%%t|9&t}TJ7jMJIUj6^gRd?>PX7FAoU{9V@b4S%|IvP! z?<;YE{r3;|LC?6n;rHIP#K#_d00+U)15t5B4Fg2hRZo`{kG9nvZkxVOM!Sjz+iC!0 z5>zkN6c4%QidyN!64v9v)oB-$7VFJU2Lje0uLvIDWgH4$A`bzq_**yRawZRq9<{p2-{hos~ z_>c4bYym>BH(EW`en(5)Ej2qUid!9BYhBO-ojHB7c2N(X`+9w1{!w~({%iGDm%dhi zec=iE$oUI;>q?>ByrHF5UGvSJ7TSes-9q(fIK3M2NL+kY>u`;<)E@fPDzw-tbh(r2 zV${`zPFEM(i7vEKoo}XkN+Z#e>#X9xlv{b)bSN~#NR~I_^j?PH;l!N70oIKE%cLgkDzm>V zA;pt72LRC7hK+k^u%}x89<_T{OFNBD3x(Nf*lhh|P2;X7$^$F{o|(Ug4P*XNQGqHJQ?e5b)|E#|QP~4@(Vpny#gyySi6l2wf@cx#P_`1>mW9C|V?8OB|#l(CRv*!(ncYlHB z@>|Aj9kmkot4TqswW`GRdty=C}+rlO2w8#_s@p0iIBLD`g3T8qQZ{Xu83x?kWAH%iEc0~iRPac~aFl5z)&aS*ie zz|GA6%Qdl#tTKz(jZ@aCy24y$KfBR9{Bxe+mw`Su|62ja)o;RYeSYq6Psd={UtUGw zKxC-MbVu6OF=X498MW2@_RqZn%&GJR{Ml4gY%3X2O}e2*MitSTYU#+54HyH#PS(3I z9NHWSh62!GSYddOsfQuvz%lf->_&+hS z0L^@F-064v0jMK8n)@;y=HS4NB{(n^tBj=0_o{=&WtF}g=By_G0-2>d7EG|f!qib> z?}8!jG~xTo_^LpH2OY!nsj!%oyiw^|r_tlT5Bx>|yH;RSlV!VXphwd0EA0&(6tP4T zvarcU_3Cm!Q>(Ny2gKz%>){v91?y6c-#{AzaQ1!vANYE9ytBOGpW7Ix8Tt;cIl$&} z701l?CiGi^$Kkn7 z7pz$b;6lT+bSK#Fm15AK(Ivr~DGkl8;qm|W`Mdm_1}Uwg)!WU&ukZ;DG0$}RM;?X! zE(7M%_s7_Ovlo5;L$K7=EhJmqWc?gi-*JAjjuglo2LKF|Jjj3Wq21T{kHCWOtMfN| zO9ed+368q$y@B@x16)tWoY$2O)yTLPRr(uB@(pEF@+Y2kykCAD+<#;R*J1h99t_O^ zga6V0CJ!xu?CXPWW$nov24wMyWB8_gMySX(AhcXl<3awu0sqace~@*vJaLl04HyEP zfDih}Z)6g*QYIV_Y#abqjZ6IiUC#ih_Wi(f2VbFMXwDsI-}_nl=T0=Tz-2gd_BHT% zaX;=6Km^7WKo3B92XC>ZZpwco%oz{v=RfF(=ULJdKvw3o1*p~<4Ao-ZJl~Rj=n6v@ z2lT_(0#WP_JO^JhH!2SR4K2`-6VMo0HFA`!D$cK{o^Gf!TvI%rD9V_K=5vA*y+Ua|R3|~V-lJ}QRrTIb-4Q`xKyTOo=L^v3B7#jzl%+QE#sXB;Nwv3=Co7FtQnitWmnkq(v~zjLnf4f zmd$Gx%<;ez{K;Bb<;t{0I(!>=mo=KTXI)kXJFRV%zdmNxjw%I$UyO;MJNIezsQlhh zmTan*Y$%D>l?<+0;y^Pc*>0q6vFcR_=tTojp=&xSaBg&H74CE7-qZhCFwA8Z{scA+ zIvu*)|9o1*6I{Fb^Xmqf!qb7bs`W#6u-|6GmHG@B$G(L1v#N8dkL>(x4-5tU>%W^_ zPxcaiE`1IIGF%7pEPr0uGsfihn)5vWuk@nJRqtZt)v)}pq9xYLWimYKdp`x1%W7%t zS$Jo^j}q)R?&q`@UMs!hW3@5!ohkU|^XJ*7k4z`r{Q1GA4tBuoTALkG(xQ(68bP0U z5+L?F>)_vGU!4A{iW6T;1A)rQbJoJX2cMPtGJs*>)onhvVUWE(?}|Q5m+h-;srJS9 z_xSH!hBAlBr^I&(Yrg;WIQPBLZ)bRV=2?~QVXcBp3D3Fly~4PBKb)S~JJuI|cAL`m zgW1WYjGNM=eaDkXt2MjX^BB`X{d(~I&Br|p@=n)#-#x?cd9B7mvI~h$OI?mu+Oi}k zx<9qzWhKbIY0^cxhsNWx_X>X>953&K-xU5AzFUpq=;v>!880gyINcv-4TC(|zYfqI zHb;2AzxuvnH@lq)53nZT9=6ZMo6yXxA2jPPA)lRpx{Q0Pu%Fl41W*~QC>m|3l}%KO=#0fNWZP;b>uOt7 zB*6ZL;(XIU=w{0Pt(#rpU9&dTEJzAxZr+K6Bd`F@kST6=t(CxBsAvrSBb9gVcT{UX zyLbQi{;q)IKr`^|SnUkEYF$yZqgH!GX+6@j=NI+Hr8)i2&YV7W{_FJVOJAdhFa2|U z{``~l@h30pf!;)eZlt+(s)bfxi|v5{$Bk&Da5dwxn(;)9XskxW>qPbL*e*b1zxt|m zQq|h2YTT!pYpyZS#pXcgS_9qD%5_I4*PX3Q7n+F{mfC7vt}D4v)9Boyw(dBmXPkeE z{^i1)ejthT(qXPwPYS(d)Yq@{BE7d;*B`Xz^nuzreQ4oH`U}A0xku@txv$a3=l{9> zYW|<;lk;DrzkbTs>NDpbuismo*Bge*x@WPWZZ^6#FTSEUx!V%`T1DStpZHbqA)TY3K`J|2 zIdRf$_2VrKVv>a?S{N6`$GsT4413A#6|;qtbpZtPe1JoEIZr0)S<(jn3A~L=a6M0_ zDv|-Qx3~|sU#mw2=%%ua>Q`^kPwW>PfbSL&pe^(CE+OVG*zespKMVg*2ohoo$Tj2@ z>&e;Pfbz_%mvHVkwx}oWGcN27w||)1>G$ufzr}Ya@safdIK~gfu0j3*+(KV~r#uHc zhB(*a&?|f?LZF)xen0zRwoT8AZHV_lAM9OZJ|(>FaS%(iwc;3;&#aT%VZPVcLC{n` zS+NV5h@C$_gmxT&&r%YOS8RXj^67Rs`oYfuw_^k|r?aIYb8vqiyTI&F>uQG|&9!Zv zLd{msuq{2di|$};+`jk3P-vDl;CcLxn4J6Sl}t1LsEoH+l0tdZBy(gOtS7cKv;m$# z(uUcHeE{E~T1BBWn`jt$aw`5zqrWSTJz))@Iec&80@x80vvHp%Tx#g?7Zw#2ZvQe) z!!62A0$$PW{xbW8pB!SiXQiXB#P4qd#}zC@)m9l3Zp#TY}cn|?W9WvsfC4@JD3m4)caA1zqH1O}g4+@Jk}@3W?xq~x=7X__*z{A+u%0~C|dB9`*3j|0GcZYD(s+Z?q32@RBWdA zsFU4QW?5rAd5d`H(*g{}&pe0chWqpC2C(ym8@m4NvpZfGZtO912w&62sa^Ed9i&R~ zOlgnvoVvtfTr

A-wx=6i5W z!u&}Z0Qdsz3m5MzUNCwz9^f;0;*9%md+z+;JbuLb0eW$4f`JBry;OF>Kdko|_|N9( z$1yN3(@7>TJoyBAZUH=xCQ6s7@LN`spa6jif^bx3F#Z$EFCV|n!-9hjj(T#4%UR~- zfy1h0n9B+O9I(;;f2Q>><=1KTV@M1!Ip_P~c1fvs0gQ`WtzuB_7kkT^eW~SNkP`>w zqsbyNu>EtBtuyp(+2m9{pOXKaUI2}$u;!=-&jPRX4*?^0?hRN8`oIE7BszIg7l3ii zT6%4YOmQ4=^qRH7PS@1o7?H`27UG_|#X$XHS-q63dgMo8^1EfN_;$5K^=YwT$B=Z2 ziQ=MAlxON>sj_0rj-+TMyGqAc}1J#+MW9r}KmxDfUR{b`lg z#n|={K4iWQt}|A*bHa1>?>}SqifvxG4|{jh{kQp5x-sm#kF%n`iWYsHDq7`zpugik z-{(;VpRKZFKi#|h*-vXn(SK!c?B73l;AZsi_p1EMYJNw}89#OZSf=j}(1q#p6X>E4 zPnm*`C-|*DE9i~jYdW9YXdH+ASb)Fad%B#s(Ql9X9DVKqP`4MIHywyDnzr?sUAw@? ze8=tYqwah7vxD;t^9`>zpLe+5q4^(Uy#wGD4K|eyw+*oESX=xVzc<7Mj&Z**mgBzO z^!erby3Nj7+^t=P>GQ*L_XEdnE0GkvDuChej2OAw zTT^ecXBEQu8`!19Z{opAuAi)~s{7gk~1Ovbk6?QB`?bVcoS zRjp)YdXezJ1(4k7juf>9N?L{DMxm%NRHrdit5K-lEHvNBtpfRcGttFnq`5|0OSOiY z3rp(E)l^&PXugr_Vk6VJT3Z)tOKP?jwVpKetRmJA7qMPHO!RAMOTXJ)(jT`j>I02C z^(T$T=}(%E(T5iwV-?AtzHp~Ldf~i&FV6Jpd|A(E#Ok&(HR7S#@mQVin&R%Z+VP$m zW+xMGEa~kGJdYkhPvN^+2ROQ680hC+R!D{$#H;GZJ4%wf40OZhr{a+cUhFMe{6hDO zqh)cHIy?plIqqP8xt(XAmvyJ2n1QO!6%CS+ig?9YO+&$#U4N zOT&rH58#`wFBH-sp98QCV7oXjtOA-io)yHH4egywSV!;9=pei$g6DK93S+@0uAbYE zu9hvbITDkioZRAvaerX++)K-jY;>V0lccQ@yX>Gxr!KE#U(r*}*Yu=wORi%q`5gJ+ z72bCA7sXFj!B6MIWvkGJhoK?#>e*@OANC%%f#X{M9u8Cl!^sV87~zpnaXK{wZ$h}Y z=kG?AGx#HG|IaF{Jb)5F7tdfPs6pVto$KIGz&~LfgVJ+{+|QlytYwUz05b5m8&)XK zaD@N^3SdQ4mGry!C{3=K)8y4MWUP9tVr#+2|9WID(Ykt~`jDW?xCFFZa35zKro0lI z#3SVf^9|Qgf_(q~rh$Xfd3VRkzZ2HhL74Enqh61C&dok!9}ZqO`~Hpo2QPT0x&f`z zjP;|&c(czA-rvWDF--8GmEuTl*!i|pCUrP>h`@3>Rx@+37K4>6F$@KQY2Lc}9zPSY zPi4Zo0ftmn(Xs^t68gdL?UrYXPJ=gAUE|E$6a0V@x z>r3Dc(6|JSZBMOA5QClyK&q)QxG~J#DfOT(h8Jx(agh1k1FTLTRCv*fi`Jk#8^aKz zuB_}U0o?uI@zH@yTLZv=&?*}M*+2mT0A87E^i0)Rdat^iCC$E2DLv>L@I{s@PBs~K zjNSR(+szn|7Qm14oZ%1vG|;IGoF0e&j`|L*nrQoi!N~tTz)bcf6-b;*MBtje^r~yd z$$GKYWactQA3NENK7Q830^1)iS>8-g}!Ms z1h3rc#v4gXm6sL0>U8vUj1~ z)9yd(&)HL~Jq{lE#~|J--j~+wN~5g$I*d?o@w6 zUpB_FGSf2C&%}Q&1KZi2+8Hgk_SEXrY8YAS0gGU_p$n(-jNmP;(pkTJ!&-PP4ZPi% z0mu2c1apzUB}PhEoNiW zCj^E8pWDe5HM>_8(HgISmR!%oz3~b4KQh2&;BjT`m*;y_G&!(>-gLkY5X=ENZ&4iH zBi9^c4zGOAY-7N;!fjjZTLO#N4-PP*i)ItuT)u`q<~t4qdNL+_9e}~`oUm5Nwy+M^ z0no2!@Gn~e5zh&k`5|c2F2-8yk2FUxDPLALSW#j&26ToFWNdzZHTFCKBjkq%!AkJf zIto=?hoDKHyH^mPn`x(kS_Mq-^rhEdmIuSO0c&vrN>!6GK??SRSt48cIO%B zf7$kp7J}?w1s?CGf7?Tgcd&P?pa0fgdA21CgaEkHAJ4FLnx+4n#7qK@-1>DExP`ZF zb{%CGu`Z0wuUY@`M)z``hUa?19sJ$wdtX1F_iV7%ZIz(s_K#)F{LWTNW$eWL zP?H0t-)n2=_u{_$e|zx#>~n7P_rc#B{rsuFZFb1n_kX&kGwbQL4LrggR_o|CCbH1! zr=l6gPNLat5+Q#2vU*e+_m8!H(sE5_%_CRVlT zt--ITjf^BF01$;vvua_I$URQLm?A2pvsE>^LnTz}(Xpi9dCOLvOgxkGemitWp;HP~ z${cqQ_@PWt$2YQsxWQG`23J)l>6n(~eL#k>vdk0X_?`PV2lwtD^Lyt{60H~VSlz)` zNwltZn=V8X)z9H zOq>Qkqr-a6TxI$x`6eqlka`OE5GI;Z6)KSB3idWxRYSk#Xu z9sOKC)oc5i-rUdhYiURCYcJ{j?S|f;=XzbQ(EZ~LjpiC!?Dljy&b1g1)rgkW?yjp7 z(*~WyxLvD~3~`5>^ZkwKfH(#59|{QCgML{6L3OEjYoMak0A;#~*g2_mS|kB^(~d*y zMb|Wlb}XJ%8v-Vy_dJ;)#5YKGfOa|7h$Gap_?n$1OI0x)dGHmXYg%UGn~~^e$$pv3 z_j=*UdD!{zx65{_K6y{4ChoJWeoEZM&l#p=G!@Y32KSBKR$^YnWcck;5zgzq@%r3q zUEy~J3poZ6`w&^cac=mnOVrXmO>@`zkQ%4j&4N-~q4P2Sdc1 zin9Gva6C=fTX4*STqsloG8s?($j?0B_ZUqUxU|+DTYeUTWjN7}!vkvs76>ZveRno1 zUcc3POzEXm1UDA=@J#7Q+iTz<9I(m>KJ2vn|7>h!Kq~w_`@8CI?sxq0yK_7{uJCsl z-%0M}j^pec_`4e0QGXwAywy6pya6aClQqg$)f;cB9u1TZ$sqRi#K_`o3=DA#>;rHn z_nL77od6Itdv~dmT{ELUWPmYVYvXC>8UZLe*b>@{^4?pkIQZzG2#ztqO+RC_*(ccU zXKawoX3wAQJ;99%d^Dq~kD=~?QJ!x`dg^Cq0<6pi)676D!A`$#xj$AROoiBubws#oI84l82cJ>IJ2nf8g zZ_L+&AR$=GIn87=`h_ELtMMORNA8P}8t8`k%*@*WX9pBL5bhOPI8``m&_S5fXDw#sod28_pa=Nobo-CqgO51xY){RyMK3^#^J$Q)&i@r~&Hte@ z=YRC2jnBZ=lk*>E4EsD?)>=^5IwtrTQpVH$tLheeY7chR=FBG?W&^l!3TZK1f`!)6 zed9;&StdK}x0&-ny;rV78BDaGqJ7Vkb_{qr1OEj%aF8x;{od(~tn8!r zk01lGlKsIkbqCt%#{&*7$2Lp(cYyznDgVf(zB&Ev*P(82m5!;k+X_LRlk%U-Nv|-z zV(o?7y}SI(Yy*zj$Ju9>ay-bh5;!)V#_lT5YNJh@1p_Q&p0>oCwRX4Pe{$6v<;(qn zS|fM(0luwzt55R5o))u-7Ds&p&6Xx(JBN0j^N?4+S>67<>ZDIq(toPAtfsS!Wo&!< zuaoG10tZg7r_(>Trmy4Ox3Y8m`;|TIcCiC^0oZlWILI|#38c7QSuvXyP%&KCyGFm* z)Ml@GQU`K6Ty9_ceFG>s`|7}>>o4fR$4k%&yU2AO$!$D`WHP?bSzZR*4Rh5;JvEmL zEsaO2)A3+LaB-}(x2dduO|71vapoOjb}f)YC4>bt(H~QLo>t_Xee1w7a)aOzK{l%z zIpy;^w7WL&ku|O=W*vNY;N9k3t{;`Z*n{EnAZoSOVgG{79zf06HRbQHu5N#>dzD0r zJ+`c+aju0)Pm7bj1;$8pX{*V z!jDP{Nfhwee)@MBIJkZ%mkaK{aPOO6N83kEXS05{kMrjD;#uK-$9?|6@%ituV@QGw zz;Trnc2J)AS%qdFnT-_egd5qv#~as_&t?G6qvsyxTk)#zRTxLG*Y=MGdkO$78*M2b z08UOUX_+?4fMX;yA8*eOJ|{fy;OqYT-SGLuUGR6U?PeuU;K!rqfAs!29aR52xSk;{ z<2+Q+jq8&e0LOGvo9Y;jMqv#BjY;~RnBPv)x5+*OVz;eIm^Q}QXj6@NpmsV?1^||k zWCj4~m8<9<62>fPjLPA3#m-0Wjd!e~H5&p@lSH(te6*=%oU7d{{G3m#WL^`M>D(*$IJ=t`!hk9O3=YyAj|L9X-dFBD>&Td?yu3uB z*l<+|08lbfJ)Rg?x{%~rOxmiqE^8-h>Dj$PcLOwc8hYH)Wqsq~C7o+^b-9sQW$+Sb z9LFQ89B)x=3~)S^GyXiPy=Si`EtW}a1r&?d6h&)_JF9j@-Bq=_%XSv>LbuRdG|+rp z*tO8jwb1G5VmH(IxTnkAOiQg)^?IVlQmmz=uIB3ZZ&nTR@dTuP1&V} zuFPfnw%S7<}gu|57bXq?T8I*4B{1L=U6gWAsc1c2Mdzv3 z#+tYnb^)=?C8|m}>(?sfcdZjpBL^h(Gh6%J>)!ghJnNeZ)1`u{daCPp1ijOKniy9A zl&Q|e#>c<2cnY8+@h^)vQ7uhdYWUv$X~1gY8;iqryJ)?Qy$!fa+if}=8DH74Vf^n# z^4aO;R{Vv%NcAce&Lj{{EWQ{z z=#+62Fwz5uZRpdrB$xrZG@ zr89d9eJmDRnj8CZZ*I>6(qeBCZ^zf&)lzWM459=<-UR(D= zaD?EG6A#_=7?dT@G8PAfSy$}G5HEy|Ilj&+(95>qVc)D3dY{g9eKkgz>Z9EJEZST5 zqU)C6S{vnx@*UyA$-Ev0O(1u;8{T)hv)uNc)blA z-v*9vO1O_?;M4;lH)q^O^8C*(pd2{HVF?virld?Edix5_p7k?hPT<;DVr!(pc_IC$Z^ z{>pQG-1|R2+8C=GSO5+Wp7P4)`<>Q1GW0?JnHS%o5gF%Kc z2kv2)ol~)@E)F^XYrdx5XhV%CS2kSJzx)0la;MacSuz>@tT3;@B*T(3c?l4^1IQVS z0A-#L4dCG|Q=knDC)z7Q2L$y{{Gl?H)+$^UY__aBz_IcIkhY3rD(XTOCI+;F(B@!I z&+KF!a89XmbK@e=a>TPaLl`HP3K**x;+#pw64r|VZJBVv?>TqF0NYSCJL>@Zeoh^H z3w&04?`z_`JWZgK^N9Sip!bx2 z56|yb^Jg7BX&{Wh1hvZX-FW}q;c(!Ss?*cbughtd(cHu7Is}|u-b24m8zv)-x1e7S z+T3c6H~atL`8!RyOswRH^I_PxQ0?OUa<=_<{`Wob>}unI!2d==A(&TItIqJh11@Lb zKdOF_3&<9yHRCnYX%3E2WnAk4jG*&2?X1~uv94sYqI$M`Ci#WDvLqVQB_Xiryb*x8 zN}kW?TE+^{1n77M{=1dVx-sprUg!mYbCO=*pPBue>PBdYbuHzW0sNtQnrv_Qk3;}y z>G<-`_@w;p(ernkbRYkkoCtEqpM_kuz=PRHC+EMAEMt|!)_EWRs$t$ieg&JQOoB2! zkDjA4!s*+AcGeYe(7O;2$RQ}$7>{*f)Y8IMR~J?nwY0gQ`r3lpqe7kZswH%Fiybva zspco=HNXB8EnT^+`Sm3&uB5t@4AtsiS2}!}=}B~UMD}-|^Oms(PpAI_joz3}PA-#A zaxd&+0Ku3pDCpL*+X?rW%1TQDn$T9lx%17?7|*yP@A z1Dl8a!IleS4$lmB2CsGxJ4<%G1m3EIOl%^PcNQQS8u(fP$Az_sqvCie8>`vhuvV{h zs30KR;Y`1QpSgz}fc-`Ge*liL!7MN}fWA17jI+qF!$_`M(b9;371a*3#)CeO|NK_@ z{DjI0TXWXHlZ{wA07D0N9dr-tM=QRxylmfC!`W+Q|9lU9-#lTq{M$NGz$b36;=h<5 zlTyt_Adbr2!m5Yyhv09X#f4vloOl1opM947HyW53Kk|gNHaajG^vXUPe`(tz&yD}g z9y%NS9r?T&ddIh79bL{qe;#OME!^%4z9IAvy#QqI-#`C7a?;n(_RXr3s^8A~U#z+N zv;OKl;CnSo2halV1Mi&f)#;#m{>|P8J`MWSE2sdq!^OCvw>j z@2YG6{lb0RruK?sznA~se;0n^Yl%O5xO&ysteQ`?hT$1Ee`Va^zP!2}bfe!KUO(2r zbpZS5w7<`{Wi5Nl*0R>YPp_l|w6(5R8S=+vbv1T>PIVt&>wV8LI|AD&JU6_$Joeur z=WqV}X3slpop%4LF;{Cgl~q<0P|2){HmdoB{q|R%Cp?W`i;M!$>SSZ{*Snn8I_7L< zl8#sOJ@@^vbrDGhB(ZI1u)L>6ROs8D^J}@i^_R7f zE^9Gc)un8x%e{eW^+eHJqJ?^r>=8LHO)19T4)ahaNJ17Gr%!&=;muv zA8cyA*wo@+(*R`767ME!>c*?;Myu*jk)EtrfRgO8yinKjo#(WA?xI!~>grsW zS7WK8Coi;hAsOmioa=Jh*J6^ZnT*sp9+R4FE2^@{{9PsLWhen$mp* zoA|ncz6&G3vK4EKPDLbD#n?-9S;S^CU&3_Xa@m>Dt7-AexHtA5ve=GWv+gc_rYqHy zRxxQYEeEk^gKVvkIrr8Qd`ds0)WHsX!Pkt5#Df(BwvibY=MA#jBKV-biNVji15!rPr|HG%@w;@JEtP5Kadk`5LfZsGlhpnkW5+$~B4qdXw9uq^R zVY5N6y>vzW)alkb1)1-(Ejn{TPsAY;VkJ; zD8PQOugr_Fnt#xvIDv!&?`DYoZ@xd($lTYuf0CF@cPR&_(WwS<(_Iq#*XHeZg~gOe zipBr43Sd7nk9D#{2J0voV*dQb{r9-%;5#(T_}q5zGwT6x7l)1=uwKLhNQh%y=?D<@ z3#*_ep_DnddR~~cRBY?=a$yzLb9Z)h?v6-N-;0wnOE4kPi1py|=X=F{IEVWmp6+f$ zk0<^7HgJ3!INldLI7mQ`WZ={c^snOmKe*Q?e~;ipS$HN_+X7VtI(F2H_w34&T~pND z8=-H?#SD?^Ofw9KkTKPvT7}G<5%$;{fvld z6{=NY&KlQYJzW8N&|)?auwT)5pf3Oiv)>DU@Be;qlrc8Lwa=lo@z0LO|DN)R0g!I! zgf-ardJ9K^^)W+t`h3=_dY*#}RltG(beelMY%40Xwfj`P`EBpg$3OL1eduGK^ma(J z?XlpR?;i#PXGnNuxwrJ=GYkPUGL2RUmQXdkZAKW?EVLg2$g#|_;lOCgDq#SQll_%0 z0BJy$zqDB))66Q@Z7x0!3@)RmvLeTlAIyPf%@`lSf?so<99(u|lKW${b6Er-^7;jYjKT8Y8`FpyijCnw2 zB$XnZuLB?yG9n8PCK_<;YwLjle=f9vp*YJAeqZE*j~{t-a2|Ze*2*fo08|24Oy#=+ zCm5@pxRTGv%`q zlyKUw_}|yb&#}UZIyL{91H>Md_X-IM1~9m8Y>iU2CaLPfR5gIM;;K596^k?Iz>Djx zAdjkw!duF(GN9nV4LlI`+xgt(xd+;))WCtVwJ>Ln>}anTJT3nrZ%#@xv)`xl0$}=$ z_s_wFAlIsWbs6sZ7kWRP{Bz)WmVb{&zl_7>e4y*o@gI5;c^T{%>=_3PJa`(MZVOb(!YPH*|ZkL&e+_)`7vKk{@fEJtb$x7A2CwbULf zpKPg-kdY3(onHUD4R}0RGfOvLs=oUJ#09Ux!^sph^CUP2kG*P`G1z@d0MVa?U4lID%k-eh(g41N`w+Z$z>|!Ax4{W2 zI<1)g>#i$L?^4#iOL2QkJ%Cbz^cmkH;N^*W+{dem%Ot%S+rY|#Z(U}#efrXJx{UUszJ!SE(AIDNA*J7uKy+79e&y1h7C(!5Xd3dbM1wYB@sXR-L zb({VU0RM!~w?qb>Thfu~%frtO&kgGZXq+XX5}QD{+wsnq!S~_S?UV4k={lFdaiDwt zU)jm~<-zRwSN93Yr09YD-~aNy5B9U`iJ*V*t341uJNMbY589c2AG7mIfM9mq2mP?A z{67$2uk0`GAA+=%9gLlg&hYqvk8%I^_?^#Xzn^@(H4XkRT^_ zG*jt8I$KdGn-sK-rsZ*{_FY%g+KCo* z$6NFL-}8OPZs7A^ClQz@V2;h#8?GttkJQZyZSUP<{`)6B{RMsSFFvWbzoAy*Jjz+P zz2T;QuvA+RdusmtvFMXk1g@s&)i*C+}~B2ZCk}1@)euh?I7saD~dV(Dq~zJm7%K! zx?t}xU$3-3?Yf<2t-o}?!Cob16^EftKO`u){Zpa>T9u6s1^+kIQDAeRjVxCPb<~@E>7$(?PS-~1z=29t8Zum@)l3R?I2w)C(xqH;U0coTM)N)Vx$_GZoVEUS>w08yA(IQVWWQU@?CDw@T-W^2 z<2b|ameC@y&d^Z=`<+Tp>-1x5=Sk7%xqiH*BH~(C+AFN0*^a;%xjgQr%gW<5P0#)< zJ9cnyVrTg0%(DmmCw|tg{ml_GeOh9Vv^e6TZLpsk8z}P}11)%V5e5aB-zUIuI=K~JYIf@QjW^-90AxSkPOVi=cJKNCt3!eo0p=X^c9XJjN zYKQmfG<$pN-vs?&i795Q6GyJ%$HCq~J~MV?6Z10L%aTl~+NN-7)8rCjj`_d?vu?6Y`JadNjh={;)pt~qer<1A;Y(|tT){+tuc=Pp-Szs=IV z<81F;Ypp&=@H2fr%!%g(yClSS@#E14B=}It4^Ym1S*K6{MnyL32`_N;8WsBW{xvmv_vrFyTNhTG&gP$#>cSKIO0#>F#n(0H?x`PN zu?l12ho+|tJa#*Y7@6ebj!R^fT53fB@$Z;b=xnbWKJ39u=K?ly~p4z!*-2DRMirfxuT4 zbm5tnDPH(_lqg67BSrhE8a1D8*Rb_84cqtHeJ9Ot(sJWzrr-!7kqN~44g75Yv?)u1 z8&~W;!}dKIH=kw~_ZW2UHt?B%9pG9O;Mre+;SLe=o4s5KFjqczrwt&I_oah)$T8%; zID;r4Z{y*uJVM|e;~>zPKMP~V4Vn-D;Paql_%6;4L1zcA-G~j&JI~?bJja7#e>0ynXFFGtLYUsw&E? zU(PSi@)iBUFTP6;f98w&qNLCNo%GR<|FyLMF(aaE`2uiZZBUk*nqQmK(sEsmQKH7M zuNo~dm)lxct*JK7JiE9IPLfGSz>Ic5i>sHluu{_^-x-W7JCXzioJ(4yEG&05zqY7_ zwaaRZJ9efDElpT&f{EP2vkbjzgFR#X1|*eLp9EUGrOZ&w-7?lVzi&{e zHB3~Ww6w_lRywMUG6$Khx`;7g6gu!j5HJM9tm@t}0LcOj|KGGVrp*f3pblP>P*GMg zP<1m*)E+g}E>a5sF+Z|JF}_j)rTi`*+QnHn7Az>UB7N@gxo1zZE`E6+lyO2c6-@_P zIsU7)Ag5vxpnpRWtD9O1XZtPD6f` zl_l(x?Vpd|fe>#WxT_gLP7R{@Q zhCYUq?!WU1^CUrnjNGxGEoDh3$n*egLXSAGoP}M=vXDrEPK9=icTTwemQn3G=s0w8 zqwjb?yQH78TFYf<*uU9|LYL=My;b{&E+3k_K8^e{yXnyUXX%$`B7=h7U_D)aqIX?? zoSy%dOvia3iM^!eMt>UX&+CJzJcuj<^CKUgD%Bp zcF>Uu6ll&ty2>t{eeZS_G-$wA*}j?IxIT5gVm4n%D^5ct{Tki44&weicOU3!0HM>k z*?3bxprrdL{Uh7FdKr1(Np1vtJrI>gS2b?kqqW8}^!#VMT;KbQmuoZoR>duVgf+D| z+j>Nm1UeKw;*}02D`x;r*2Zmg>{dDp(UzI2T=*8-f{sQfdp_U~&yVvhW$})Iv970B z+sa;M{=UvmUsE|=@}rOS=>0pmcW&?Ox0Qnr6!yLsUsch$Tlg!y0#uuBwJMGc^JPY@ zdo{tAX?>IO?sco0VINMk|E1k!`)~Bd@3Da^;MnZQ86Y>j_o`o)Nj7fi&nm%BLx0}s zhsqz@N1F$g)m8O<@GE$Axm!M)3LL7xlO3bqKqIrh8nWDeSK6e`SHZ_ae+Bx7jst%L zU#9edroK)od)fObe+FCb^fp}OFWvlJz7DK0{;FT*16rlc!D?O)s=BO(o_#LWvs_1` zr@d1K&n{)c)Zb%$xJb^ZY*n{i{QCiTMh{^Nn-1)AzCA55y~-2Y!tD}l6Hg|>Ua$&B zg62o~$~Ts~VVpD12YjSkNa+uU--bCm4LBWGB2KBx+-{mpjw zi>26*i?HyQ)98IrADNMcCN+lQ1zHZJ`+d!P( zIyV2w{1}VpIZva@pzB@&-I63RRIBJ|VLZ^K2|z0}zhyvS9=mS=_=~+yk_2|vBKCN` zX0|=C4&OiIi^r@0DFG|#o(4GE>CrXM&rzlbgZr~*fO2$1VB7$N@Q_tiV|Sb}{sYkG zz%{L>ui84A-Bl*Q!r!bbM;a2t^LWp=j8o%3|HCDI9o9!4@AUO#&0R0JE$8~3j#yUk zGJrq4Lt594spF_GUt`*x!z=6u@{9NsJ{h1Xe|vJ5cgDhQfHuOukhsZnU1qb#?D@Jn z+1-lbf1z&jG(VD$Vw+K{xSy+^0Qx#@Fec8=2IwV;7@yDMKI>+el1LCIcbR`@F#)qf zu=6>CnHU7!{=&SV2lVBNCuGKVsh#bqo)eSwSSETDn#5-D;{tvESR<-`xfgR`?nOLS zo^bv)w8!`9Y{7A0bQq%oo%5raKWjsUv3EEko^O0vc<~AT3>V!DtwWgs`K~7^-kBeh z5Wl8{J0hLCBUU%RW@E!fv{)$>)&UT*YpcN@!NX+OMXK?{rtD+Sv&WM>?w@rv~cGIrDMS&j5k_(*cnCv zxb=)E){5_#Gl+pwc;+$R&&In}$q)^;RgZapRb4Zx!dXch`qpRtpL)%k-lA7L_y#@a z`|eklAbxoUE3NyuPeS{p3{Y-eIH(xqoa@80eO#RLgF@xm%Qgnilu3$p{m_rSOs{>@ z+x640c+lPxB=Z2ms#+;$(a}EA6=o;jwENm;DJ?`c+~_AzV41!CYI6ky$_ZRxScLy? zC`PfTWDe6x`f545hs^clV8!C&+<4o9o1V;~ee>d+Q(+|pg&y`dj`<9F(Z z9{6{9?n|Dd`5~<_>w58n_v!hs{?~fZEACgVN1GHJl~pCfD zu72m5Ui858_54?Tx4z@PXR4F;lok`kR9~flGwa5OJd?}Eu&$)PVHe}davu-Qc+~Hk z0n<&dE6(p$BVAW#xMCTG?UkBZD;LxpHOxWbOfQ_aUhlf%_?o)$veGEyL(AL_^R*Fi{JQ6-T$U% z>jxfqw(fuPcj-kBe3yRUXYbb{ZF$C=3k@A_8NEjNM9HX7x1Xz>6iNxeM6@!ZDr(#2 z!FqMmWetXI9Czc9(%zaf+6mDbv0tdeIVwHQ=(=JV^hy7!IuQWMp3?N1Vlp^;*A&t9 zfR;V+itmLbpH#NS@z=t2k|&?^lnh6w+kYIoeqTMRf1_Oe;fC5V2@9M9vu0~xU=T)b zO2Bm0=r^Lu5GSKosGj%KqH>XTv7;S32QI^biAU<;xTPfL;MDZgS#4Ca zi3AV`$mE3@>zDM_-~TcF!T)-#{`+U&s$c(;U$Yi;i^;ZE9$SLW>GwZMhf4Z|1?6Z1 zh^*(_QXD|bPL?gQw4%^A-Ch%HCKkZ&m9c#Nkv9^@xN&;zZWShx=O_Sm5B|Kqc8El%Y zh?z`NuhC1r;-e((zE8USIvux&=mg1~iN8<~18^sgodQNl0<}C6L3qL}ZPh zh5zn*mM0HD2ZOQtBiaOxlo3ooUuD~N7GO8u6IUBx2boSrJM=~%7CmRcOaOrhXwU|6 zr1mheB%YVN^LcuoUaepM!W;A(pMQ_u@;~38SH1V=^n*Y1Vs(cAcvQHI)fuIF#XEmW z5B|)oGvQ2+NAUaj|j@l|^7!@s1vzVo`;MW(1XG`XDQSJh(w&!GQ1XFOy6bhH;} z(EZM+4BQY1r;4f>69D&h#%3nS!kY8{cIuTv7T_TWSL`Yy$k;`vZ7VJ~f};eAW&kf* zdCuCa^*-ksdYe^~PZ>ICUD<$u6ao1SduB7nZo>w3yO5x8M0JVTyvsJW9=a}FQ8ODV z9RS4c8R)s%e7ZL0zfJFX*$4EeZ~B7%^etc1e|gRSP!V4hXIwTo_qZ6Ui|v$;RR?Su zF-HPlJIXq{%DdMz%u8KlhFL+3)#fz4SS6*0(?H`z#=ceaOA@=5^iw%vb8CpZ81p$>;ro ze(17Jjca$X~LH)p8!v2m`HHadV3&!&PAe)Dzo zd--#lDIn;o>M2PvB%r$$s1VB%{(_8|bAk{~5jNb+6ZJU;ie(;gQkkX1(~| zzf4ttM4M*b*5&x32|s!1Edj|J${7!{%&JEehXK zF$$mJ-`csM%+~?QsL$V?6v1#pP# z*WCe~6jqfESx@9MzO?)6JQtvq{f~!UCB{0i=J8-xwJv8l?+H5WmVlVWMN8iC%1_rb z=m*BndP8?Sm-qzh&htnNvJNxZNKsnoSKj?!_1Q1D|4udWhd=gdz50Q-iFVVpjjl(u zn#N{a*0((KyYw5s^?Ul8FaEt%4wESUiO+mi-}i$rRz7lDkoLt65CK4}SzIF>Zdyh7 z)4%;+>V3ca-}N_N)XeYym%jgn|CgP6?EW0!^Nx}podfXeHtY=RW;RwbCK-Tzg}0Gw zyLPMs-;W1=oRl5$G2Ax3JF5-wmHa&b36;LJ(VCh=D*Dzmh^{D(ud1HzsSclHBhB@P zxWVuI$r_= zubRJ=akj7lSIcVkMp_(>G`F&;oUwA9~Gt+=Wh zM-C74t{J$@nac}CZDrY(%@=(`0!9;A-6I)j zO_Sc5`srBRbZmNxR@sd{=W<`Q?=F(KnoeX@*w{f+AE`_P!=}iwL1+h$UfeZZ1wX~< zKm&aKjPsS3HOSUA|HPUuU1+I>Z?L?sr4hVByxj{+jMs^a;oB!W<|ko0pN0NULW5t5 z??P)vZ~b-M299Co+raS+6@4HS2=9>woK6M$SLyzrRhYQJ7laAzV>VPI*R|ID7QN+V z@7Eu^^Aq~r|NJri>+iTMG5!CNExdQfJT71 zwWYPzH|xdEe5HQFqSsp2m)ZkeKqZA<`^H}sPAJ*1 zZ+iQ?&3Q)=nj$Y9HglLU{!p~(q;v!UhH>(6hSx0!#r{PW`~egmTSnxE9{RN1`%|C& zdo`kg+9}!JoE0-xe|*K9EOWxkakdEp4K~HAXYa16H`=kF0|6q=6-san%|NN%bEc37 zm)s!_fhL@E_$nI_bY8ZrH>MRc!5ab}n+7mraMmM0U)I+hz_{2nup8hwqaq|5=?!mw zyZ%E9wo@28^}LI)E9k9%;omSQzyltgyfVL4FJ(0(}JD0l9m93tTq4PgOm> zuFmkP7W+N5R$F?>gFm2Gzx&_o_3wF&YJ>d$XYW0{tSIxf(LdpSYu&r-kkTI_JzdLvzk|&pk)i+N%+I@7h)GTW`I;-;)qeS0kKcfzTI8y*DfpKvV-7^Q+8C z=ti|P;!^2G#5;=$f=4QIjG~sDhw0WMmL%xlHOjlQ26a8J$wu^%a8RM){#OTL>-h!v z_)H7dpKZY>m)fxE%w&AhJ`3}AE)!ig$qRv#!@1EPraj18pypbV5x-2R4l6WJi^SH5*3m+k2nv* ziL`)T0uI@E#524HIe)!#Cy^wY$iX!0WpkW}H6WF4K+HB#?iAs*=BKdX{2Z)3_aTN% ze_Qq)R}UO}MI2QjG1Y`-=7cQ7@k^#Seq_0l3`BB5LDsVm&DH%I+&ocFw)9 zM&+o-k`1J)r5WV@(#FP9GqCB*By2uC9b1lX!mplr6-LFa2&&#VcW3-L4>pRl;%zG*m(RcVO&NC8ei%?`H)pWzdr#=EzSHr7MfWY8G?HmnMVY& zW&P9LZmoYBi(uVELgh8|GTr1}QL7P2l7Y^jIT#r}KRrhaf^!74xqh-4)UEsPoOl1d zHGQZwedC(9H%X9q#Px}v&ulcZ!ubqIH^HZHU&D31R>&^|u??Gmjo0*G6JD(1^af#g zvtXu!N3amU^c^FxwqpWTpPz?8ZSSHqmqIAxB4Fx>+B)Kf1JkaT#&5Z}CawqR$Wbc+mh>i<3RgWZ#a?L7S88hy`IDWjI9G3u}Ic_CF z%zfrQC{ZJOCEaeovkwf%u36`BxDD7n{Q@QoTP)P^L0%&^U&pIWdW@IfB(f}K4iUT!M4WDiykhT$U9r&%ZxHJV*BzY7NrfP(Z zVF89c^@i+6&1wO8yz9a_L%@i0C39u|b?5j&<_!vg#;a!(f}kYaa$m_>p%VyG9fF)W zxxNuP5oIdnniELo^`(-NpsALY5*B$I?%|t zdjJ4yIlfPRa$m=22O}qArtsL7D}aRnOh3>&`0M++&NW^wAeCuEC`mq@!?K@vjP zgtcmA4dqV)hN9pU*CKC^+T6MTtJiPGs&(5COOhPt%4ZSe<8z5)#OI}!ENeJH1pm&o1MM9^`(p}0_ zX#vgkhigJs$em%04hymNu!~sT?LDf&et1VZ7hGvF2yHxbwaTv2FK0 z0gyR=j-5Dz+wS@m2!oe-=NoofWIJ=Z%vACeZqau;{p`yh7g{Y>T_-+%n%Iox{3 z1D)i}I$3ii2iB|?r6qxGlD7FAX2x&GI6Bw(CWeI(uJq3g9c0v=L0E<-7Cy`xO&qNgKsU7bS{~@`Qyz zJcUR+g>c+JRCN&MOV4=6kbWP*Q<5(@H#oN41&&GJ$_T;UQETCZG*qieq~sZ0g@9X$ zNQOk8DkRlv7+n8pmz^$1E;I?ktTpLi5TSZj?rHAbGKgzw&$%v5fSF*V*JGq`CoFr$ zhR*diFJwUW;Aq`Wi)o&hkKnRV>17ggf8nxs%X+C=NWBW3a#*X0iq#WCFush+Qy#xa`MibdpCXuukXj+9f8W)PDg`EkEChk5E2`C zPZi63340h+{L(I(0BLnZM`Kt+eMCo&eMqXrxwf122&D-cS0F%fR@!s9Bw6_N+YO?CZHI)I5Gz9 z96~~<^9XV51Mpcbdlz|bknB|up#k?Byzfmd4J{gjZF8R2pdp^c?d5UQC>r86BWNF>c|mvHxG!YiOPm-<_Tt`y`ydAi%@LBjrBhzfur=pn@>y@kaz3Mlh`un1h##25}(fd8k^g` z#;(>gIMN1uGW{s-==Zcd)Y#bK2F}pyk&pfoZ{0TyJ0_pTftelH*>)Bi=N`i+bH2p( zxhL^?Gw|g^VEgQoxa;m`gv~7vQ8uG#-=%s@124nKvjE|yr)z_FWiucTC68^(O*1#p z0?MY?u*eDsN_#5;RMDh`m}+5i%Pdee%VMT=?h*kE1-Rq`*76=|ET9NBOQVbodp#>) zjtAa)EMhhjlRXNHFH)GxT53%}kE0!NwEXozJX=5XWIWUtZ2ou@2Rb1+%R zC9*I~D$NLtao*NRQ;I-yNU}lLRAxcw-6K?#=;$+pzBPG;Fxsj1@VsHQO3nWX%q}wv&AztA0~vW0FJ= zgKC8e1Y$N+t4;unK+1&T)*-4lAx!IkvtF7qWK+_nh?y^nS%)$$fna5+DD_vLZ^5ee znRt8Vkj|p1YXg0Cb3qyp1UNKW;}VRmlWZ=sQ&p!)7}I{uL7dMWs_fWU5Cuym!bx7U zn$Gd}uHV&;zs`G@Afn2~8rh?j2&xVePC**m$t=OFYQ*gt#O-Q?*wnCc@JG{f{gS+% za{rO2b2ay$r|gXo{3Y0G)d~S8O4ovPEh26cO49|D=EIn={XJ|t*M=?U=3~jOc*f@mSwDT>ohA%x0ZI%UrWg z)=pkWg0+gaOkn8!C-!&8P!Gc`9#Y9Z|Usl7vu+ z81v_>0s?GG+r1H1SbWzap?E|vUK?St3u&QFnAyA!m^sUsh+rXzMTaM0)wvI_;p}SU z?z$U3tvBN8twIs2*56& zw%&*suNeUn0x3p=>@Bb6`7ht|8s<;eJnHUGHbg>>jR2ciPR?619FN>YK#U|Xu4^PU z3V=+qo+#ZiAZae)R3K*Ok#IXRyE+@FY^1ZvuUlSYkKia3t_0l(dKGC&L@HbaT^j+z zMlxF^&u3ZR1$g0pz$4x8C&3{;`*>dvBo(V@%NC~++vp=oV>%qr#fAG{u@p{L!qxg? z&bUvoeZdLrn12dOXYPb)HOO;=--G)kezroJI8hAcpi_k#tb-dEfQ93>VCzT6v2Oln zq8QO>jjdL~(Cc6(8(?d_!Q%_Ds~g9UyWVr%+nZz&B+1g@wV(piBdzfKSr2espdz|F zX`oEk5Up_Gb4-M@t^lzTnUkXHf(P=^VS;WQU3CS=leMH-(SS0=5Q}a)^w5cf1ShWJ zFZ^m04$cA&PX&&)0*fZ@@*H~ftSFP8yK^Kyp9vgn1$MVyz(>=!3y>cn=uRtF+f%TJ zGMs8T(<2l%#Y7GB(2b%%6OJ0>~;|a|oJtMV?TRXViL1cseaO)d;G# ztdELYkmGX>cIWufY)W<3#km1L7W!^GQAfG6s!+Y*?QT7$U5v)$i9~2 z#r+w-1AlT3(`vJ8{JQEn-lGA2Z##=}Glf`zmYtR;-Q_$i&hLy@%hD-B6slDMQWDJM z`*|$Rf2!Pg9p!sm)B52lKB@GKXaqN@EcS>=UA2?cP0T2i7hQ{}TH!j)`B^RM(GV5q zRN?9c0Z0YtBj9Rr4iv!WF6%Il9KqU_HEXS?RjF7FC{$EarR>$M1n1~HvH!p^v|j>F zp9Ks8c2srhbT%Sr?2)YaJmx-vb&{f4ICS_ZE?(+D2Ot3Q#SV|~OVF4#FM(tN%jYk& z@5JN zJckLUX6jJJ=!8ziYcSyv8B=tv`KQ0|+&>&NHClLv&Z__q*?V7$?tMlrg8$c--Xa0#FLrMFJ=HTAe);x|4MY zQ@Agt<5<4B0lW5penmOFZsS(TI_CW2{mJv?cyev#^NsVH*EK-}o{>SJwh>!*?Y%O7 zYc_1{bc*qwr-^g}B2*Ex24v(48~auz{OoHHASO9U<~F}8)oS&K*-Z$u<|koX;$g0? zX6f!;;XmSiR8CWg$9u`kul2GV^5Qsf zZh7HW)p@m9g?|BwsucTA4D#W!;fR^)=-20SORyJ-i+RAQubq|Ruz0sgMhQyn2U;}4o8W=Ng0x1 z%iL{SV{UC8wwgm$%ORVzkyQ-jlPMHb6`4dFHXR+5e&T*+aKA<3h}a9eFjN(3MTMgn zNGlf1qzz5;UKu5gs&ERqcnVI+LM~~dF6p2$YM~H!QLW{XS8NoL1`0|NRZ0>SNezmT zMj2JC7Ok?qxyAd7u|xsKL1`v~;tZ9~okXop^5frrExq@Vl*~REfpdu{q z-*U4WB}OgEnI}!6lvRmP+Csq5M6}@@^V;gjzx2JY^ojo!aNGqVv!Bd)&EKO~grrOM z;;6u-696L~Ox=Z^GCifQWqJ;AdKzg8g-im$c~m6~{Hi>H=i@q_Rt)ry83M2-RnJ+$ zi%qZ)H|Va#xs!vf)WOo~Wp2etnz2aqrCq&)x_Aip25&}hr39KD6>D)bd7DIns1`rR zMk?Xp37>+|JxlQ$B?{eE;B(XPXQ~jC^|V^{yCJTr1Y5OA>9c8Kb)AQn%E8jIGJkr< zBe*4Af~;mErZwS~(h!D}{RcovRA=es6x zU8s-dQ5iGfstVkc4ogkJRxD%{7a27TSGVBAE##sZR7KKAT-OU)k22&_Ho}&T;#?(s zbmd_$N=SYL=(Cb7?XwG#T^FOk9~koMjyx5 zQz3xzu;I-f6Pdt+M$pqMo~V8$&__m?2TK1L;8^T8Sp4KbRV@n_uR!ehd3MPj$V%2MvmvRdBYTiPb(_UGl~D!6)QtRObAk40 zqS7|W)GuKF!Q)aWoj7v|35`l77jZR>aKb>!sT8nE@|i?cND!YkT2um)(amGhs;yEA zZ7IS!!D5{a8a7sF?M%C6sw}0zrxkTxY`O_Vcbz}kpIzTyW#eD$r)l-+N+CIvF7}2OkFnx{t6#r4SuXV)rMskr(@B9g#zxVY}iOss@AhVsrF(1@iAD@J{JqV zSb=C|S{gQ*Rs*t|%d<&5G2scUxZH|WmnPxexkJ20dxC1I21IDJ>SPddP54|DK37LD z>!8d`A)Ij#unZ`<3=$4icT`CfkaX+dOXQ$s`XZ2|@}AanStt%0^)(3Uc?4|-!3q_@ zdLN#f@*vh+o`uyNvoUzuV6l8Rr~po}dCo?9HKIBw0iLObVs>GWZ*A#Ef0T@yUf5So_UKD9MFUmeCN*+6ZLQh*x;o zWk#w2N`gv3TDdepvFi~?Wf4fb@Y^N=83R5l&)gJ(w584Dg~?2|9j)fBWesAc2{+ih zwpQ;99(v_bJov{)ardD6are->u;BOv0W4O2u^2%oFF>$H@LQu&E9*7jXr-KFPGUWT zfZ)`_pUQfLv71IPYr^kn^8NjW4qEC*gc;2>hhSF&qAJ0Q8kv6tT$xK0acx9%Hps-%X>`(BZQTgdNTw6I5Ms{?ew8iv5sE&~$)U`hQWm9x4tFgFUt5I*n|DsInq@)RzpuUB9W{`f;ov+BNBE4%C#(lb{4+0 ziAaSl^PjoqA;&~KlS9bLB1U44*cx5^_>+krr^O{1+7=b`RY;f(2(iKDxRR&db^rSu z3&n}B=f~QFmL8uyX3Mkf8lsPVZ6%{C|h@+&g z821mk6^{wHBqjs zVlk@N)zC-`xW@H^*++CqvFv{05I`kP%waKZAe2cV;2H?$v+y}-gmMl-SqtT65)qf+ zcTPYBe*Z**O838g{R^cCZqP2h8VcPB^m@?Vp6d_S?;siGnJh{z4be&)fvkpLMu$J0 zLL!@&V^dvDLPz)RznPu)-%vye4ACl9EU~%Apeu<7lt(ied5)DiNkj`7_?hLMCL_BJ zI?p@hH945~$0C>w59U@$4i!NeSAoA!j+Py7VZ-J3vF_v|l)KCX?+abKMdpOx(4-NO zw(6jh1#i{EXJ^o>GKrTz{0)|0cn=#+&&QkNhl@2nl?0mPYSoEbi}(CzM8Mzv{Qs!) z`LEesR(aR<9QwEG)8F=a5w44t0bXY+d8z*8oYXlUjB!yKVXA$Q*83u*Hp*OK!;*k7 z8~0?e2MkX+%;%1l@tWyWqjO#vYJ*rlTY7IKX+g;uE_ z;=`k0CVGovP?}|F+SO-8i9j_aU);y1lXNWlC2GR=VCDFI*fsAIK507wH{2g?qCZm6 z1|gPOk=`&84RU+|GOyzJY}nDVIFxb_vK(Wv<9S-jnI=`t-@SI z0wQ>WOYH+KUJo@YHy_V zelU_vFtjF_Qxa8@uJf*gMa2}aAsh2ofBaY`kElb%wx{M6(gBrIiAtpOMp&sna{W!c zx73QIe%SgfNk`HrZgZPtA>>(;e|wCKJN+EvAO%V-Rpn$_vkJ}3q>Ud zE8FCOekAJ>xD~5mDnLEJk?IyYe6W5ZSumPl{ZtT@85P6?g9(~a$w{zJ6nClv=DLbQ z6BVGM0PCt_sidZA*CQ)a;mrDrIk7RtLfR={&%VPR$lDI+BzCK;hrGBAl6QG6rFN%1 zGV4#%#TQ>4m-l>o@-&`)?ss_h`B(7NGrz}cZ@z;mEpzbY@lzm}+n(EsO!pXFdiCOot-sN{>%_`ufMev(e+XSo%FkGrfNtDidAa|!#C9fSp zJ_3XM9FlLuHj~j4qFN>JNFp|$>1l~4GD#{i_~#7%ICvNa4;w8tIcs96^|=mk?5POv z1dSuA4J%7G7Ltu!?Ahn3ug_foQYP(W83W_RFtH|Pzu?-AUj@yj+Pt%WK)X#Di%ta~ z)qfJ5nn)!2tjl`14b*+nJ4AYS%P(Wh4dd-gY%lP z29U@IQOeUUx&nqzs0)%;GhSbfYb-?~Nvde;5lvEkUxBcmmAwH$W7c#Y`JHO?G!mvIy(zIb zXAOJ}IUPk6WI z&oTPun=$;Rn=$yN8!@!U4S1*M1`O}<3kWSBj zdSY;|9vF7>jTm|Tbr^O1br^orbr@1~J%*P2979WfiovDVh{88)+X5bY*~A?4?mR!c>VHOMG{_p-mG|srlhfIOB#?~k&BAUD5k5~k;}!^# z1}&;tyA;RQHjaPk9IoIie+@V$&?@d+d=}RrNQb+09+qmOF07&<8b@s+f~rIqmGKbj zVo@|j}kJ$qu7kwQz_!f*Ux^bM-03|pv(xu}flsEtNY6APd^7D9bQ zMSaLcCSHxC>Zyqtp+)j$Qq}S6XaF+;Juoib3y(*OP#x5uEBz5O`p7dfqgeQ5Fo|(R z0qh9GFy1jxnN1?2{v}RP? zpU=QZSHS1?M!DG=5xQeptfwmxG%IAiijXMJ-lw=DrSM?szhGhXW=ykGJZ)DZM`C*_ zk4B%0my1Gpv-l>wQSwh1TKZEA>vbJQ-gE=T++2ird-cH3UcbOwy>7seUN_;bp4Z`p zfBGNz%|HI%c&4-`>SAF8T^A9OKGfbqZs7cnNu;KYknMu`+_d2TOLONx```blIR0-8 z99~)2eqizbuMC|2cR&AskHv{hQCg?48ANuCs|>{L*=MnH_8DxSbqb3<*eMMoGTGT^ z38a}x=_!L*q;q}|bk^-!Xo(u+5(Dw>v$L>s=4nv`&wGC}Y;^z(s|jkd9&Y>=v<=#X zgHwUsZKp9|#5~E-q{W4v=!0zH9t?kCo-p%wHJ`=W37^4*2sp2a#>Y*|_Sa3kRX#yqdT3>*`%?RyKUhLDf(zr&FjF;Gat4rDpj&sHEvRAi$P0Peg+%$mSpv?ArMY7Poe@U zm4V5$0Io-mpT@b1!0B_Bq0w4Iw>t}AHU+W>DHf7xT2---_LRf?-ed~%C#_b6T`UZB zDqyP+QkWsi#tK=*Z1_21>%?XQ&tI&jXlEt1IaDf>QSCstSpd?G)rRTP%GnU+HH$80 za?nmqGOd_f#UhqgUOKa{EErB6M~;1ag>C%!Z(ab^+f?+jC@`|LC}Xx~BNU?vs!4@m z0pXm9XC^<4C6~rw-Ngl%y0!%+b{PT{WmtJ~9@bzUmL6U%V@jqr8}*9O8!^2OzCt%=hGTgW% zQJM>&w5kl{wL$dE`4MeOp)?mpun;E)PjT0+$!VKyN|R;W{nfK9L>g)%3C zKwd+zV4ye~L2+d{dez*3=chl6RhMUC@!1xQAA$*j&SWaO@DZ%7P*7eO zLUEx4B^AZ+Rr(RBO`_aU1k4V(c`$#5DhglHMTE9U1rxn8H>0%fdfZg=cW6B@6zeWc z!}61}P~N*7B{k*fnJq?fz6Ab45Mk307OSTB5rrDz-Q*p*Zaa{uQziP(%OOvtm&^!c~ zC&Z?c!8~YBy{OYJmrYe#N^4ZFS0QL;Q0}G>u1TS+5JGW9nSB4!ybqCT6~(ETG+fD~ zq&=#l)Hh=I;D7sX-iA|@;#worRMShI29)o2OcB==F*{PSll_$oO6Lgr>mnj)0e6eCcPLfEcFj6@+S-r4vTOX8~?e*&1ET`AXm-Uk7-5@CWjRT_${V)!#b zgdGRr6fK#Wkjge8U?}i8Whl#+qAVXm(6o?95V)t(m~8MW5HJqr)9@FPD9-ues|mta z?MF$j9Pzpo$}B~i0Y1$U(DdKl8-Gh9qO0-1h7&VdsXD1gz;F@D8Ypr6vR;-|1!Vp9 z*ClYXQ-*L|0%f_NN8-s?V$sM9bB*A*=Gj^OP3vF5TYE|L(e>VlE47HIRfyWv@Y^o@ zSp$KpBuXm-=#lG%lB!Z!M?*CU6qzA}b7>*Au*r9A_unwF!zO^u!j{(DlH-@Am9L2s zCxr6K68LIL<^CzLqKG;K2W#cN(bIjw=8!0kS&%bdl7J=~e`yoJOai63Vzllaj171f z8_vu_{+Incl~A1IxGH((u^G=yE?vz_qdS^&(6hpaw^lxlH5h?S7v|!ncU}{FG09%1 zx{UE8Y!Y6>^Z!~E>)o4s-_1`^*06~{5RdBmcm)i#31+Mh?C1bE;ejyy{g8>?3M750i7f92l1OUB3CKwFmOR~T;#Q=C{ZXB|TaKR!H^NGIfUTuBiHiV16Ot{s9tZ%Y z_YoJNWU>y5QVTUv4@>C_JKi4zV%c1#(oC|xXg8}>LepzuYK_S0x4^7tl^#a-u0KBT>Ti)>r9&;Z?@RNTg36^yP z!A4gbfOND!a`9ivIB32`Wc6F2@qEnQqHdr=QaJ61dkyD8B3T7X?;{z{Zgc?L@Ibhs zTab<21}8QUxfH*16Lh5>7Rg?!r$}bDE^DXX-9HM4+JFNSfj#eC#Ln5LF?w(tTz!DN zf5o`Zu)p~-cD{cOpUt?81>3MV-bTC`E@yG8A6c3Pz> zUJc57$Wi;t`0?7?p#exo`Xj5|0z+x^B4CmpXeGAOB=>SoYP9NXgdT2`>*XE$Hn`E- zV1)<3jP!*Tu9xvh#%g7}Iqp z`A|r}C2@!ZBCffj%I#Dp(-DINTw04V#zSgJTP%DDZW5Gb{l`pK+U*Kp%9@VWize+x z*+4dM;PWG*96o#QBGR)=`QYl@G*RQ`pgv->PxSU5CtmjKob@pGIED*e!-Sp|B{4rELP9mw`C>`$Whl!!Ms-)37R!!ANW|3? z__8vdqMD9g93viA>VDqyLcq>QZA_vxuM2a1Rn5F|{mz_ihvVj<6MRz)XqGE=F^SkD zzlq|N>*!S#G07Rg9p8L{*W*L!~ma zNZB{=dIvlg%cY8&AbK9b7;`R}1tV9@MyQd227%5)$w|1*sRWMDhfWxpdX~E1xrK{!#=Jmh4vuGzV!>-D#E0 z_j5k5e#ucyuS-Lp0{MR%yXW5%&vo9@-_1?Qnmon+{cU3ShGMkWD&Q@TX@H%g^S%vY}hzmvkg44y<;p>v0;oFkG$H|iaiQ}a|!Qs*y zaip{qUzC*LM9+W5m&N}RV+~ELdzk|pH5eI17cL7=mHq;oi~j*%7X1UhEd41C`F?`W z!av2q;LmW__j4Qy-hc!C8*!-YCLArg35UvW!hygqaNKu2PM2JVugZUpeSx3ibN}Dt zQ2F0sum7i59`1?f)Cjb68WFu7x^fE=YCowH#Fm+REtAS^&jq7<_l!5+Bu1+t(E`s1()1a8fZ?Y7&NO!w|9-=a?-nR^2=PpN`{S`iv|1LML#{`N9|HA7AWa z^80driz<~hJniG@;F5Mw9n)}kQ4kN6`tX3S826R;#67;Ac%Zx&9xg4ygXLv-xU3Yf z-tcowx$f`rajXPu6F!XWaRZ+2Rf-2n!?>#~hI`BWc(Al59xl5X5104EgJos7t2B!0 zpbJHz=%4`>_a9~kmS$jZID&mXA3jg`FgMy0&qQL#DOCuOU~bl+cSyr4Hx=XKo~1Zd z?85>rhFjEfWR)l?l4<0^25Ms#@^KZ88be-DP#sT+8a_p`DC0HMYM99?)W&R#O+>II zawA4rrYPB8y!j{CUiyDwx*NhhRtA5#?j~&f`Sn;e~xd8ZotW+p7^@-MjS5t84d(~jy-{&;y~ml zI1v2@><;}5A18fy*;1fdDFiv!nB}cC$#abaoVdk(4KWZ$_FaEz?)+!}`yYX0-o!uf z@c$7w{qEt|WhL`p#j5$r!v5cM{I7L!l8h0u^YjcNWCW*br8)JxyGKc;+3wlrFn#b^ z81W`i>iHakVz#IoRN6x51%%QbTb=4TC)p4Az-?Id{(kIfJ%_yu&*Rl!kAl!w`{&?3!9GnE~o^u8tjGixzFgA49*hmKlV)3}$*faM$cFa48-#uu8mC~14vr8wpTzNSo zdMewLQ-Kkq$4W-e(PLlZ*taLJWyeko9X=A{#!te<%bs0V%j|^+sTmkK<}=aek5*w0 z!is_WAASN$SFOd~0|)Wt*C%lB=rQcv^Ep~(do)??Z><0S|l@xMIQ@j<^U6z)15qqk9i;4cJIZPUwh&_3TMfflH868FB{3s;IUsncg13A$G$_J>YA))g4MJI zr^Pm{VUtwdk)2#E_Kj@(KR)(eEV}eQR-SLegcTDJsSLxPFUIndbFk=QE0!Kvf~et$ zZEVfY#ExO){E?Wt`8~Y7`~@iY7vbq?58&PPgRtOe3+C*dg6W@(N53cfAm}P$ z%^pwI3Fx5asu6G$_$$ictGyXde{er0emW8hj<#aqu^E`KZ5ST@;1>L*RV^yXGB=2!b6>`^oo`~o_E+)2&Oc)5g-KY|F%_%Picy}YPvx$*XAGw4|eVgAvH`1sNXSn^q$m$zfp ziH#kV@sjhURw5S9!*J@w&MKO9Q0hcclrKfiZ}Rf}r|laq%4L54X%%SA!SWN%?9lZDHAUyou?YR4`TQGIY2WZ(k z70(QLNzUK1-1^&!k?&t0$p%)dB*V8m+#IQJ2COo zp_q4M1{NNhg(O$eLER*4v)jsEn_fZ!B8anXb4rQQk+E-9RCIx|47#E9RHHe z@qY!=_Ko~*{5Oxopzn=;*!1Qv8y(D|;{E^5f_E`(>o|;D^D^{@%kjIm$K?9WKQayT zKAVQgYsR3@<9$$W$A$TNP50mI(MZ{3-Pgg-yvZsB&wua$CT@BQledq=#2xQq+}2T; zv~3h7Z5)L+nuj2ergESXDO!yyUL-@*VrG`pIsPLtbI+vC@h`#iGrPvW6$?22>)!Xq zKOIHHG7%FCvTB6WRYE)&v+y0v*z^I$EPM-rOcF7t34W~_e%jn-BY0xSotVCP45n@x zhgZkECCz$fHj?d3MR=JT6Oc6bXd0u}55knaL$Toac(ffFhe;d9;E&CJM6i&M`|sM< z4=+DbGMCsC&FFY&&_kH9Z2~559g5rEuEl+$`(yZ;KVZ(0NtkzNDw;RHhZn{_i{gx5 ztj@w*|FV6qa{UX-`WK&(({%aj3swu8N1Q3}2m`K+lLOa`Z1kT+lyu0%ESp3yQ z%=>%-{@DC7d|bEL&^7x=ZaY;faoYLIy}(RLT5nd%zA5O~D6fcP>h@t+i!oSzdLC|j z_A#g~0U|0$s^HhTe$~TbR%Rj#lh1M6Mp;b)?`(JhYcUj`w$H;;ue~VX4po~9?VGL6 z#$Y#||4TZb|B`7fd;fp^{9$9tdv)0=dbQ8htxw-ytN;9;Jrxb@j2)|=IP;8oV-6Nh z*oLXEuf!eJ3wZz8k1+eKO;|VW2v$zpk4bMY!9xRIh8b>xX*D52f}cYrW;N1gZy51L zWMTtxPu267_11bU8@&ggHXp;fNr$oU{cU*TH#1gZGOr&)TZ5+t4uFm*d_(&m$e}gA}a- zX=g{PdbJ)e+&>cg+uCt(8nEWgBiKK;9SbLJMlN+UjGaS=KxD) z>_V91pP~Cjt*B&;SQB#b+hqJ(-&l)f@9xE>smHK#+=v!IuF(Qy-3IUBh45A zDirl zx^m$faW%)Mf<4YmJSrL)|DiPqF`+Vf2VQ@4g3P~@v%+7KYx4bh0m;=m6Yacowa1t;5pMd$FncIMz%!hz0L$!;oj1 zg-ns9RepkwDU}E@*ITKAne2_{9vO}~V>hC0bCs#qBk1hn*ojjD@|`$)K|s_LRjGOf0tr{ts0777 z{AkG)C#jZMA4vu&NzF+*TF~y^u<^j-Ph!dHHQ4?6K^*<&1P*@rwS506t#gE!%`8)$ z>mC9BwC8k1S1aZ)68If6b^^BT+$-1at8Y%=n^R|S@QY(u`SC_P{o>2wazUpq+EkOw z64hK7ZY9QiFd3V+?!flld$Duxe))_o+xKGAw!N4!cd-DHRCm)FoOLgOP$6LJE|#tS z1e>?*!L}WHuzkmFoH~6LmoI;J{dVr+zwMS6uk*rX>$-XT%Fg3g2-s@o8WCcKITgHm zRtN?UJp2TfuUda){EvL~4Yuz&fXS_G-uTm2p9BP2rP6(g;9W9}CCk@i*WQEj-e2B% zFPd9t;*-r=@$Knz`26TsSo_Il-1pGq-WV_sn#xW-%f+S48waX|AARb#SL*j&pB;i? zxdPVH6(B-oE3L9gG^c|TK_s!kuLoU_qAFd_X!%bPeI5@!{xr^acsbL14}AqALuD}u z8&om!v*_00S+6q&ggqPz5!jER%i05vJuPbE(-(mq`wt`KkPu%$km^|i<%R<*WlA1q zgsRL`1+s}Sp2^;X4Zlg?(3^Q|9aM{94;0|)isWr7P9K!V33PJ~b?^S~&YJP>`5po9 z2xdrbZ!IFCfaPAu16`=9<#UlWbui6*RZp=`f?{DiZ}b_Hv9_ zpKx!;o}g#p%b%4Jfx$JC>upzGj>rB^jZ6UC1DfUc=;|h9Mo&r2Kl8ImSmfslc_y6} zmjYd>g|1YKoi52CB=vHh(upad(~*qs0<>GE$~-UqBB{TBpm+S&#){W9oT`AO<7EDfMv!1fNg3otTjWJp~W%P&0@A;VTBsU>O=`v zByYm(bO;j*4kl!Cm|>*xS)>Qf`~MeO96As7MZoMU3Ts+KN7FW*>GccjDgGHgi}b=O zErK~}0y7N@ZB`l|rc``nMX@;16Z?a`aH6CLpC-#N*(pOyCW$t~!VJU0EXTl7Er`vr zp4gu#$ERuxzcUrYDIPGYper75Msg|bqfKw+H|Jb?n-Y z^X~`{@)|N%*PkC0UhUC*LVID$vYZpO2rQaln4qA^17n8a7QMUZgiWsv0J#2{fxAOjiT)YEgoCKN-i9j83C#l)=noSCqkqp22+~6XOB~)<;9w6~7sa zl~TMC3?LiR5OAw8&>z9XpZyeH6yJ=qH~llVm0yRkr9Z*DMb~3OaT&%Im0(+u%{DR3u0XR{i79Ct6CDNPGX^HQF6N|@SQ0P7mS`D135W2ip(1Xo;=+|w z`$Ewg!1xf(eZ^$IIVbCCcl&ll`!~*iHQ@aF z3qg1O%!jyy$9HBIbCXG1>M<|Q!|vJVv2R{G-g{}j*tIdw&F7>L$ePlm*CZ2>Y#`=E z(H^S;nZ!U;`R>8yiC^Qu%u86`d{`P0{*(=W6>Vc3X_me7>W8W!oOfQ4Z*|O10%h+Y&a}o_~BOUVM5a9(&|f7%8&<`9S&( zIOgV%Drs(RyEds^4TA47R;m$J^02bKg?*i3eyV0;$;x$CiY9`%U9U6eFGvA?xx*_a zXU<(D4O^;3XrH6nw0zEE%=pPTcH;D(e1>F1F)!*8BzNhq`yUqT=9F85XwsGJU&Aio zz@e|iD(u9`vsknCV=4MCb$H)_tXsamJ$nf+zV=op>(66xr`#$qpXtcgXQjwGcIu)i zGg(})$l{BC9`W8Jv5cbC_PoXGB-_Z$H6TKTv0Fe?x3GAXXEEKCX~pxuaQQOMU%ZI9 zAI|skuR4HfGZuJEEHXFQ@Q}hkBTDOc-Wx9s2-;LHU%OF`O^}={V}iztsE!H({<7&n zWp^}XiQ@D2L4C0j&G`7zBD}xoeFSrUl({{y^qaX@c6kow?fVd6U4xRg5VupE$F4%4 zst6yQdKb&uXJXzL%LMo^QZ7s?QH%`AGGRP6;SnsS;`s7344*edGB0U)NIT|GZ5Z

WdSw?!s&=*s?&B;#5Ax zGHLiS5%kRU!1L3dz=CfkVs*z9th_uHt2@SFb^8QtXrG0T&whm2+oz)P{z^o%W+!mi zJ{LT2?^e6GriOE=vsaSsbeXPcKthz)HdJZ<6UWC~@T2S{8Gd0uFaI~Cy zlq)gBD;4yx{{fdH#tynD#)F#?y#aE4VTH|6=IMN3aH?u?FL@?7}3hyy(r} zr5DCx)RtdkS^Ffcyf6=Mwp=T4TL5NmFjemVuIsn# z;zTUJI2r5EigoR+_~iUttovpm9vbw3XXUOJ#M=7m?!SrN{kOFZam^6mfmy(TTp30# z{xvpSoPhP1fRz^~VnzEjth?Nb4Hsr$_SR-$+LA@D>daCn0GC0)G`cnZ5tnOdUYrhs z1RFCxEdIO&n=j1A$}d~t&xOQxi-kO&yMcTeemm|?tUJfK)P^y0N5LQwNb#n!1$lpzehih9u_>2#0X@m?*Y~=%m zAfEf+VXVKA{$;Lm{cD!> zFP1T-5zX%q%bMtAmZH3-1TVHeiG|-zz{-w^SknHkjOCh+@mO_kDmI*(k7ZxZ#lYwK z$upyS_upji{+s@`G;<{Tk_xg^0YSY21bl;4F}%|HBtGt#iVc{6H5Vpf#aC^}|B^YM z4)k;%QHrubql#LG1u>&FyxdM9@Yp$&E*3)tHB zE#^%A1Szx8BTFzIf|lr7B~nTs*nwNHX6ixgo^uY{rk}!r`M};;7v=n^vOUxS9G(g6 zZfVD!whP!g|68=oSPC9Le6`0{=_2DDpN==QrcRrEkWTi473hay&$Qyx>Bq6JtsVPk zUBdnum*o8S&%G$|JIg2R$L;yw!d3`=64@rQ}b7H%>AvG zuzTux9G(LizwNV6WAp59@#MfEFhWhRjV2@`Rj?vW$c1mg>M5Uz(wX1?qrto7`tNRS z$KE*?vAOjd>|1gH8{59Zvv&;>8**mjvIf$VjriM-e?pG0MHsI_oAMk?tsYLI4>G|4 znELuM?3sHOduE=)XR|M3@AQi}FdNv>d=^ulTMd4nJ##K%#2;o#OhPc8@#eqM7pA{2 zM*en|5FPf+I*+|G+GYF>v~^%_+XWfFMPqj0m)WObDSd>vO5#!`JrHKNFJ6CQ61LAh zg`G3cqU{fxW&R$R0U1Z$13PA%mH9Vk$R^ma{<40sKKyg{_h0)v_7kXlq&;(&(-f7l zBn+SL=#cvFO6_=|9TzTLmip-uiN_tljJ8D{ag{`80=HBdr}JR#^v$WWQg@!e)LHvo z@-l``U+TcF&ky6admlhpH4#!RSQRy*fZ6-`VX)3*2JxlKUQRKAv-_vJPZDdHO;J_8VBHaQ@F!cg zWs@h`=*lh!QCm3zn>uR z=AvT)m9cy;3DcUyIJww&@GG3X;9Wn0?YxHk@9epAS887V4RfP;E?ajWl+Wn?@jbir zh)Tt?7gc4tm3V)`RD6BXQ+1!a*df<}$K&f{`z7o<@+Izm=rKe!Tgc#Ej8qOzt_GhU zJ|;Z|s>0WA+;rs{^1U1{j>FlDz-w;~^{zSNYUmV0@_6TbCIMpPm$X(>zM8Eg4k&+$a7l1z2ot`T*-VrIZGqt*1WQ|$;PynPd$>Ll8U z&v51)Gm}_|{8YgcJgyOiI#uRpIcEWUbM`VgPf2i(q&$}@C1t_v1dbJ}305kLdZi4l z_k__gIF61%ew-cVVB_$+(L1LjoOI!%YeUAn|GNvE{Q+^-2}%4~Ism*Kj>zv}}1{&e5;{od)HJD2YR$DS~Q z@Ao2v*rTK1gMC82)D)fJ7#~-O2xBJf3WRNf*`7-I`%2}m^ZTw(_x&7X0@5`SQ^W`` z*}vEXRy`FmZT|l^d++@n<(209o`2xpd)8fdPEYD?XG{`GrBdbaRt{1rBaw_T&N#bq z&hB&O4`@&AvG`KNzV{Y0?{7Es)kLyjxP zV@4}g)_9JZZ>QOv%7-qE&j$Y^wF&=~3SS~0hC^hE8ag_Pm@A#f6D_=HP32Wr5~IBq zT+uMLV1z7}hq>{$a4hL}ES2ai&wZEz4=RG1DL4{|HhF3I5H^mun?{$Ncfv95cV(0B z7)`!w6k{A|JZLxbrgJ#W=6Jdi6ItXpkr^`*NRN>2A5FS%3?n@`Je%TR#)x=Uq^7dg z>Eac?ijoU)I&BOsiA{Rys;Z*W% z95DVpT`7OZ@#MefRMKsnNE*(bR0ExM6D>{?tK6xyxQ(>BjI4DUsMSpTB~#+YGDr!D zfW(JAAn^p*cx*Won6sIhVq&xL4i4G}vBo@zr`;y3kuXmil4u`vE2oq0;Je-_h&qxkZce`2TU7V3j3{KeyBp4rM8o5shnOdJw%70x2ZV=ieV|NbUFFlXW@Rkji@)1U|<}6&sh9{41ysKL4S;N*GL{P`IvnB z9n_@SXfwHa&+Eq?@?#FmI5|p>ND*?DIDjfQQgQhki`{!8D(U9NkDuOi|6l$5x5n__ zfaBkQo{C`jcv7OdHuC%dbW|+P+P|HSmZtSR9<0M^*QR7Za}OJo#|O5`Ncvt zq^y8#ku*}AK|P;IlT0WX$g@=wLfO4}bsH;vVJlYEVx^OgT{Q@94DteYO0JYFLIqTn z!ZpMPpMIghSt+ul*gAIlBAr`yvSV+T%G!~?36LuVWKl((%H5EhBv&9Eo5#R(4I2F9jU>{+MDwiKUZRe@Xt3JCMe)GZ)qC$Io3>#nlZv>l^p}>;=oz=Sxmi zeN&sB_oY~$QS6=|AAI`xzc~Jr=PuK^Wd}R=b#b}(8kc+eRbxV$6UCMFy49K#l5!l< zLKrVKbZ>Q8-1TW>8Zjmm6Xp(})9Plg9s`BJQ+5|A{U&{L4TAGhHuzb@J z7O$7A#yUE?i)g=G!HOeQ7(-6HVp-@aP%PTySPGM7j$tiDH1{v2bD*A;UDXtJE~I$t zBGz?R(Z({?vV`_)6?|SX1(PqLig3x1P0ld!*wlMiccp^1D-~?&Sxw8al~isor}1Dd z9hX+n*;~!JtL2pMTYz)4S7p5_o1d_is`bUR9%Sr;V%3{3KUiM<_4x8{|1PSbKwuy#wRb)pSL#@K zsEK9kma({XKAl$@SbKFD?N^IfbyPq|D^6uuDwegGBxe|S;_FfB9&H&|%=+#c)^ycV z+_{+L8|Kp4Ur7rKSi@3Q_g3*m#dY9ZuEW1F2mh_tgw5-ahE*Vklt7fE+!Qk3bW(dp zh!1sY|JNO_rfTy7R_-oj^QCGw^_9}rQ$ppDrGzGkLL(2KWCZ&I7}FBTeS!@3QS+~Q{+2aY;0(o7?xN(rn!HYCmVH6np+?pnZDh@{ zDpsB-qH{pN>;+V8TcR>e9icouUp$zFEi8G@F{ScMNROxf`-N;2qRQ1pbX;jIju0d}Zv#MucbEe@7j#rHA_>5%IUNKXD?i)7J$hr%S)bFfdVbdH+*DYrK z@hUc5Dr3WyS{84ZhaqCS{`}k=@kBsQS3sgRVwfTt@j`kjd5YGaWpq+VM^81IF11j! zX%P$7&ZKPTd^Yse(s@nBhHBoQ|E{|KoSu9v!EBNR1|7$auYyp_C8QLUR|7M`3sj+XPw+0a+X#_pA@J+_+a9hEd5 zs9|IG3fiv}v%a^S%KZ!R6v&!uG*0(TK7Wd#r{_Q#HZ;xNuI9=KJ1CpnhPt1SX470SM8 zL-Hsc(vX#0dQp9Zbp?>ml~2*)7Mg2!(^>aD?Mu7ZU)Iav64+PWPg}`h)|Vflqr8i@ ziUX{!-NUT;CHnTm5j6(>?CtyY?Tgcl?2o9`obFu0rU|^5GnH-2E^wf_pPkhg=_o%; z#oP{R7j5FZnllJ^yr;aI`Wf4ljkVjAhc75O$oYg*C$N6;QTEpe(AvYdm1kJBbT<|A zI#~YYYPQv#VNcZ+wwIn~{qnf&x|~%D>#E$4? zj*5%yTzP@j)BR+mR%K>Ia&+5i~E@Uel0dnwwnJ%!4h`d$K%=Wvbpdi z`|Eny-f&LYc`uvU%(5BHv=<*_U&9sl)Lf!^+6Hz0rQQ<y>0gk19EUu_mpCO7_A@<7qrG7l$-L2PX zz`?`EX=&R)$EIzZzHmj=c2@=lSifZpE{RW)dD-E`9ZF~G&V8IXdy&)UFLCPJ1qDtbGx_UafrI&EB%a~@Tkfnb150SJNF$?Ik-+yyU8(&iY|~rs^mzAv)H`jfI2^? z&t2iz$#XwAiwOvQ^4tZrFY>u3&UJJ8(g15W?D$pNcLr{(n`7!6NdMvSXT%Ep$LoLk zd=Fcx1>c{#K@T8)6eqVN`VvQ?fraSQT`VmM7H zj_u-DZ^Do96ohk2aYfF==s$i$-EY@IfmISpPh$U^2Y=Dy6cHY zdJ2-mE%#rDBPY-5eT}nM)IQ2Jm)?h1q>EZ#vY}-R^7u0GxPru-CYt_SK+lW_y;D6L zpBiK1{Fj+Hehl7N0m)h>7U@CUMBVbg%X?g&7+wL2l~b48OX64{5h6l_5l1(%LeEp6 zvFdZlI$l{l>q=xEB zZZmQ($(K?eivn5yQbLyN*F*>KK3PO zBu=F|mR^A(7bU7q)vnp3hV^?X%^u)&lbLdtiMg?K-pL=!r^!QT zO#CP6e0Dw$Wpb~@OP15i{mBO28hSU&f)19YyLi=ZWUSdtrp?6|yNkK;f8en3zp*UM zhr^qXDJ)=BIw?WvcV+NO+%PsJ-bI5)L~-&=|%Vy?it*i3X#IHkX4pylzrML6T*i?-x>sD2+oZ&sr>s zWv!vP^Hon(T&7IH7#3fFQV>fGA@iw>5khR1_z1aQ%+ePWX91b-z4GkENDgO{AZ{D} zO#C7$@un-quB_EHEteODnP?gk$9GN}^*)W~ZC*SMi8mXC$v;X(Jcv7mP28STi$N z6Um9>+Z}4`Rp&-!cZ)Jve-;E_m#81f{?4%mm}K_zqL$99z8G)11AOdove00lamWxh z-EkYo6Nj@cWjL+QL^imFv&()Lhm1ow93RI)Ln5n&-N{^20&jZkysmlpiz}!;=LJuY z`$9q>Psb9@zz|L&MeVyd3FPB-X7hm6!^bugML`F(F)yD*JUn1bV*0RK*=HO?ha;I~ zVIOm%QRZjmvM4>1WuXvDBLStxT^tQk6!h_(IgYbQf27Hi%%7q@g8nEbEsJ!QpC?ih z`1H2Fr|y>D(UO!*@vtOHfA@c|KjH7#>bQgAq+6-I{SRy#p2*zccQeUw7voGu9yGdG zl$gNbgxlF?O<MgK1G*p&((0iw9CHJU7C~yMy9b z^!wkjE#VJrvJ9uhlFaiX2JwhFfp-VpL37*)R=W)>3MTPp5#_)d=IR5F*-2e~%Gg%e?cKkm!Mt-~gvXGTbf*3ms<;mk)<#iaoka7r9qg~@;k$;@Jo?~k zID%=|gCRne34HNjooY56s_3O;W;2=qvcmKUiY-Y*Wqi0}lEr#GXH{hw>zOLDkkgsO zH|$abK$a|@G}WNAiY2!e#VN&x6g|>BR5`U$gveKj&D*!D0!51cjxD(|pWptGe{wYDpUlGt(ybNjJpUgpHv zD+&rr>9qqfe^Wm&7u|-3ef<7wN zG_zsPE=pE3Vhx256}&VqQ@xp__#-4_45sPK*L3tYu;NIovU9eD!)k#m|C^j{;N^vn z&?)N6zG7z8e5Df(+``C>5}!SSlCyJY>RZAl>Z#sYLuC8}ETLQ^OYOebN3-to5?Zb< zrLDJ-7e0Rtv50iqN8|JsV9d5LXZsXZTq~yeS`AA#7UIj0A}5B)@8^jx9-^H>R`$

kgpC6$9#C%!@7SYyM z$kU%arrXAQ%F-3eNWyVu@cq%KIP20s5%wkC=3MF}dePSmng_`82$@LnI zNgGXaRy?yeKSOK(0@}M9m|MRXhc6eqV%~eYz4N#lKX2CZ zNf_%Yr&*L+k~OS}6;O<%n2lj0@1pX;ELJj?#_v~P7M7{7g#EhpmMD-dfdU+%aaf#r zs_9|O9>T0GlUdonfHs<0)>%#@cMJ|GG<>-{^5LVbI6s%pYs=YmrG_WoctimU0^C>w zV~Ecj!J<8H(%kbk>-*~{USCNtc8_vX&>|t8{dgjsJKv`cL1V6wdD}mtd7zkNRvZU<`&mTOrcT4=f^W++jG?S%wt`DGnHGa7&&$feqS1H&C8Q- z-$V29uay#c#pPn&UosWVe=km3K3;DGOJ*E%HoVQMo+8%C_20ON(5R3Cg^hs>9{T8E z>Mn``y^N0TW*&I^X-pxpX%zM4NQ~)2s5|)~?E^E}++RlJrfSmiAH%MVRYgPo%Xu_j zT}11(MYLaB&P!AOq@F*qVf!V|pUhG6{0Ueqg{quC0p2vXu;O*bzMM~U*hO;YP&#{s ztT3C39m|vgSnhpuFskxVEn*cY)`8v_#;jzjkIZCK|8l;)T20la2K;%GbTvZ6$C#J% zSbeFC_JK-TFIMy7CvV`^#Qto&+K+_n!PK6eMoV8gD^9M$5OZJ;1eJZ9fFr5tHeO%y z3~L2AzE;Mp+8N3w)$W*p(L0{F>_nz*d6qRSqP3@ixeZGYP}OD6$M4JIo+q+sJvoo{ zJ%y~fRKvr6zS;3ZCx1CS1-O0VF}pLAVt+)&-Mn5tiM73jY`j*>rn4QqJ!OiTa|AF< zNjEd4{?D{uD^p7P<~=ny-DzmreV9GtNXklJ=H_Qu(?5@OJq^rRu~n>FD*gKEl2t=^EM&b0{OKQ~4vW=gz>1rvR7po~>UxO_$3X69vwzNF%e|rKA zCuh@v%pWB@{pp|7^Jly1=WmKCWL=WiCu<~EhRR$vyRva)Wb^2Ilaxb6$JKgT&#$0< zSF_5BcX~(R)9%6H$W^(jF3$vp&hXsZiOap65hMS=$T!?Hom$AIfjXM@ zufpJuVNS~@F_@+@uqE@!CW=MvUhIy1^?B0Vl8of1W<1GiDp=LCoQ0b{B0l?0m9?Cd zV<0XgUV+py|Eszrkggc>KWF|gUD2TCe`yw_g!BfY!!N3nsBU2>&#N{Ow_DUlIcn^W z504=woTJtklJO|xjwnTMX56~*9%*dKb0OqOu?iR9_}h8s6kxi&?mTtNwveCo1U~y% zVvdJ+W!wyQ7G3604Q#DA%bU+lQ}>V2?@@VeA@f8UKHbOe!fWiWgYr2Wh(spf5>P~o z^5R3EvZwSa2TNgZRX@)@^ob%onS>k{$Ry$zPt}x84%ZB@r~C?wrmw){OT!dR!yp-< z&U|H^{OyX9bT%I3{guMb=Kug807*naRCm6*p4BC5M`^0XRAypQG)PJ$tny;TL1=%~ zRrc0greeuDHSX9#66+#liEL6OpEnRv#~}(JLnw;dlTXYvkVTnKdw=-4(*2ijU)C|UP$nr}$=%B#mijPjK0e5SB?Af=teLx!^ys~~qN2!( z^8BM8tMhVD6v$OQJU{VMa&aeV8TVoj>UG4Uqu*qE z(FMAyuPH_Mn@`TvpP_(mo%!w~b2wcJU1bAG(cLg>7rUzaSUvkF+sk^`UVDkDFBh@q z%bk2**30^$J?eZ&hOEn-Mc6iuimB__Q-6_7)yF7Wx>~I}rC}BJkLS62KVx&@8TC1x z4aa!pxv9G9TXH3R=_<#10c0J8`_4_eCwKUUB1zphdD6pYsfxOOCNe2pF{&dG4W)5a7%x z8RNcm1zazDZWRZL`q^LG$NJ_&yzuUa7=0nNP8VYHlJ{2ARSkQJuCk_NkDg5~IfVgf z0t&HJlzLK+WvQQ)LeHO}o-cV#SsMp*rKsf7N~2m7ta^rTkgYp+(mgPsfZxsA_L7!4 z2D@9V4Kw)g)0sagT6_C@DXDBw>uY%~+~WM<_0qDwjb8NkuWV^mc30c5v=>v!&dH10cN_{Ho)j5kwbp@pY?xm@#e+d8~s!LrV6#ijcRTS+b({bshWh`~`GNa0N;UV5H97bp_{5ZvRJcP6y7VS>4vr zdA+`dP22X7UGRYJ0^-Tyt#?1?%*Ctp4hVUBfY;vo2#Y%xmjYX(xPxJipFQ)#`P*~o z1UaJ~R3Mv-k54`S7UwSOy$ON#Kb`iK%2?J3^g;kiSBiJ3-z#eBm3m!(=Aw$4>-Tr; z2jaIgI1;OzhwB!_I^cJs|03h5PDYUXQAq8(4;@$gb?GX&y)k7c?ea*^L-MvoEiQGk zH0eb-EA@ofW~bv4Yg~Vj{YQ?d_e=jl`c2|CB~dK`MA?1$syj-4G{`PcypYW zz0>cd>#Ol>pEZf1sjuS*q+t>PK_DHICN|GM_D6sF`L}cNi|zqYQTy{TdecY=g%p9; zqs7Rz26)UF;<5aRjEs%u#StTDh`WQ;nwR-nj2G@2$~#HJ`Fzknu=uwB%6jWiHoB52 z9C0_F-94BW;|)A(w6HYp_jH;5fu(6qm3eNAq^mxdAtKL84lfTMN@vn7w0K>-@69ID zA&z|!jA=RQxlYskJZMd#)Oa^tsduy3?i{?+fE~XW_CJ;Ps3`$f~xmmz1zYIZ(RXW6^xMI0FIYf~9PX z1;Eq_6hS8?Bt6`0{GL2q5-kykVDx2>te)L0SM z##2@=0pA!>17p;FyMpO>H81X9kW_zAxk>o69K4F05LU5HhJYTi5KzLRTv8@iR^{7%|a&1^BHvf7xypUp{x92PWZfQXi-96D{@d^{d~zeTu|9LXhFtm%EZc;)=L z!}vW>!j2FRN3!`aH=hrZ6Z!g%Kd>S3Hg=otU|Gb+*l?~Q-`jnou}kbmFoZi8LYz~Q z((*};jO5kfgE*M{d!?3sA()9*OUD+>W`e=Wf?-41Y8cE3!>xRubSDRshH*6E7EUGn z9Vb%$j;`eY#_6O%oQg|iTS^>F-VuCij%V_SL`tkl>@nZY7IPe}nuk>`3u_(m>@^SO zglRCXj#R$R%;dqW44grYv9@GBh`Wtedop{H2Xi>-7LKI+6UPjHq$_0*>ukwX_#7<0 z`*vDJ+{Jq5U>e-FF*oiuz8ads>^KwC?;gsGgj<-J^moim7{u2@5}7yTpZW3+|Ap3+ zzh|o>kxw*@fG-1sTLfoPm*kKd5NmFkYhqXf5zL{WB3l2V@#24@x#lmH5GlZ82yjw6(f)g^Wr(vZs`=$y#>x0*Rn)6L$hGtB(342w6HlwcTZ zG>peNl33EibSyZ@?$UlXRh?FuR+3L-^@m7vKE%S8Iyh7TyUVXI`-3txVUarXh}!Qb z)Ak@4srQj)cz`tXV|ZQnD0Z7}b0bVb8Q6tQD9v|aDQ2o1B>|5GG?qc%6UoCVc@#f;xEU^2b!OK!8{Inzy(_HB6*wVbg+mf)NfM(}80t z*#7e26qS1=YF@D;lq{pBHS5&p_x4|hM$Tw(|5jVUIu zFnR^RoHcK;t{A=sO{>?dg3YE$p&h~H4X8}18!YvQCQVl7P%?=9(NVYq8FXyk#?}5l z2IyybSv6u6DAtqGK$UDHu`Lr8U`2Jq56uPnKf71*A@l}~Sgl&*8U>_*sPChikbP0z zI@%M|vy%c5O#T2VULWpQ2B{uDCT|dnKTN7m7N~M;0)XaVP0J)HXDE%QX0pDgj+KX7 zm9@H6*vOK{63D}tX6DW1Ptw{~$lAW;%&h-XF<~{gs7Ash1%nx0a?@dhgSymX4ry3|IZxx}VQEPV2@ghLoI5-IMQN;9)*zpxgYHcF8= zeC|;yd&nvbP=5rMC!)Y)Eijgpw4qcU{0r;*m$K$u1EGgv7?ov9L|J*qWe(=OhG%G_ zjJEy;-dXs8D!gwNIKII$*0dZ#&H}uykxIFp5_J%tGlbf+^ArfR>Tomm>y}e)S*-g) zs^Ki`Ws{bpY|Y$aTa|G;#a$n=o;sS&w@~oRLwLM-cwF~lb&kTE>)@l6la>qql@l z7QL;kmuyZ^9FAc`MjU0wKBe_q8S8sH$a;LdV%Uo%rxuD~%}r$9#o3;x^t&(O_K(Bm z9E;5r!;oX(gS9Wv+`E)5tl-mSAM1n;MIMQe7#%|SVNonMv-0$6#y|TAK6f@DZM16i zn6gJOd&?`V9#}wYUnx^7re44PHwPT+E(Pg|{cQ>5F)ZUQmUq2FJ5{Vc)5e4soL5!-XHI=g;*EpFe4K$@7;Ql8iQK7-uQ_FR^x0_mW3c7gBTv zNX|&4v!{}E{fnsBR))!6fLV&|K)Pb=Tl|?yaV6PZsk!k~9-hvo{!%ucuj8rrUceF> zqtqd0uLoncfzO)XX7$w~Hc`i?OQ+z}M&r=NVGU=HoI9BMGgDdHQ%=*dwWLHmib*Z! z#_1l3Io-ue-#pHmzCv2A6*8xBDw-#X(=iH1U@VCl!zn%T9%7B&bhd%WqtetD+ou?o zu%E=tp}b%BG%eRkSkqs}J4-&&@1I|+SpN|?cDTl26?^1Ji1>`5yiq2=aWU(7{Xp9-dsXFv78~O`bd$y7A0~wft z0W6^?hLA>F))3xle1;aW{U4~~?Imwx3x}{ch5S&Uz|4;pzsLMlQ(4qHgWN}R)c9=) zik(;pV?iPxHa@N*2s-<#_@eaFU-kS=tN20@p6u=jViW4p1Q5+vc98~m6l+F^yr)O; zQQ>F2I`hvYhdju>*gctwf$#BWW7X1ddPif`Mq$cwQn>poN)Jz?;^;K$PR(TPm8EPL zsG{{k9WyHz;qZ;cCIkl|^o1i>)6!MOmeV&DhqpjqPkFO2y8LMOdnx>W4xNB11{H2|JB?IiK>f?()7bzf1tjER4C1Rh*is`#lj_$!im^!(bfP1t8B|W}e z0?yIohaP9*Jufn5!jpv3N26&n_UN{fV!bHQGs)2aDdCW+2lA4h%eAB@DkNPxU3H zzFny75#_yNn;NxFq+|XuU9DVqk(ZvFq6n+0!4Rq55Halm%HP|--jW^;)b%mrt#aIs z0v&Yp#(3w+uQ^sBmeN<4`dTqlo-bx+bq{;1uCcdjfUQ+$xhFW8*15asD(`01l5IHL z+1R5Qm?Kf;oHYIYViteatQ4|ibDz-_jjjx(PR~w#gu>U?(pBBZzQ)VU_@G!3KdoX_ z=ucM})g23D|9ja{ca9gI_(at+D`zd?3lifw#%2IpkTZK2Z{X^HcKlA+( zT&^svp(sLL2&7D4`}`in8v48H3*3`CS#PKYC0;0#nEO5?xA{QT0Q;-2G2^{5to}6B z^p&QvD4tvA98mY$p3-ja&3FN4FdK0eaA?^?OcPo5Mw`07yQ<)`SC=UuRIZQAF=^I` zESkJZy)Me)>Y`4PeLigIdX}ljQb0Iy0^3T?vA60f?X`z+i+hJO1BC#h`Upa1(Cv@q zyE`OJ_!uTvM19x7`dY^<^_T!$iUc7bnK(B2Idn`18-;lM(4=RT0?^^kAQ%~?p7qX6 z+v({S*mQux(pseqv$#ZYDhkXH`;TP1RCo&X^yaDBRcdzu%mv^R^3k#5rxkE~ z>GBn(efcF*rcLLI8DBB=tJzf5H!0VtOFdHk!R~`6a0O+omHIk>77DYkOZFe7x_+S9NT-a zOPwFF5`W^ESCm>)Kvl7B7J_&4+79}zi4FMxON**;_=NNEiTCodDn+=tLHw5EzL7CqR$2E${a#pJqwen;!~;EBRL>Wc z<5$XKMM#kO^v0Yj^KP0VL`W|7fupC9^C-t9V2i^Q#V1y$l9%ZfF~@b_Sg8Pi1di=m zK=oDjc6F)yqPq_~Vl^)Iz`>EqCRisx$UG$XkX(-#W><)upqr<1987;c>j7%|M&L$4O{2Gz2Ex%`QN{OW4yb$waI26R~m9?47g%MIhdVTmpTpS(A zx3Iwx&mwaQuMHo;Vzo#w!4o)NwqSlnelZlBun3T+<bf7vCfyks~$UMUnYi#)R~z$Jd#12hU$}P_h1R+W0p0CPb28_ zl50ROZJQ+vb_C>w(>om6o~2;>uVMaXq}$h8E?wPg`=h6ww; z*gYO>fpm<4d_2BUJUb$R`uJN|XCFaTP~$PXM~!u<%CYeJoaqh{bBD;Y`50yLGsYSr z-6hufLMn}6Ov}dL7VBtHhv$+ggp`l~*b$=k06F#`c{VROb~jOH5RU>yqnM&0%;7A8 zS`N<-OQL4*t?aPJvCfmk`(`7VCXb3&lZXXnUoF}0LM#&!r(YasB*rG4u+LAn!@<2N zMqU~*f*J9{DKr}>_t}}B8|B@+3|>w#^YLHQ5GlpPB&>zClQBNo!eYzhzUj;Q}4Ws z&A0v&=PdDTw~k<;)6ByrE2EOF++(zJkHyMpYbyCRHyO4l$-V+cM91<%{N3y~`~z!T zMqc-26LLfqVKd9><1LGiMbQZ5AsZX5LpYJ}2hJtl&hf-MIBWc8PMUwmR$CmEPCxUb zX-v%u@}b+pdqeN0(VoIaM-ubxR$g=kc*|^Jj$ttEmSJo&4`aSNmHPr-LV++2hmRbG zo2Sht=10OR!+VD{nd3==IA|QfI=hJ_As^GT)0vetgtmk`*=QX`t0$Qz%|)G-Mujs( zy~j_xmdY0E-L%_NS?Lbb7;sYI8OF+hk%pj`*A#Iq2UA!G7P5BC#;xV!7IBLwTdg64 z@G9hje|_xvUo$5DoPGKYIQ|Vd{#UK4e|i)BM*~iO1CDQQU=phZxi4fP;CGExnPsZ+ zWShW)jyKp_c!dL%J*-^14YxA~mn$2eyMTaajLO)syE7HT+bVgr{yePVk=TNHxJ~0I z|6&WfYc8^*{y1;EG8IE8LVTKrA>E6^l}o1SDRxNXs}$B19mi$P#~sMV>WvVz-AD2J z-?F>%8oTPR@R#Ri;c@1&bVf7ZmYiqLvThD7?&shmEqRq~loXWLvp}6e=6thMFD!envU+W&Dxjn=m%`iSk0?tr zDUy$$zpQ5%sb+{YX2n7(Q)L$^V0ZvltXi)MN6AQZdd1Q?psc8`_Q9#MmsN2kN@-ab zi5-^F=A*K%Nmndi>!-Y?QJq5}64*4!=gPw74yxmj^CZV8`?9p8Qg5ty({P8R;Lbw8 zV_`vyV!#v9^RtBIXwpKsqL~;yL7ZtB*uqf+Bo0vL^TBUY2{MM_A3;y`W zeod#?nVwpSIpW9c8i&m#s{8RIgc5jl>AiGP!m8dPX0M!zBj85!_|O6_lB2h==Il3g z_O4{*;Wkoz9s;gR0``1$ADf~EKCF3)4FbGgE@M{J=eT`7yzWeu5$BZVKr{_ohJ)nj zPz=$*q{iY%%}6C7&B)N4+jy>Utjg=$&|Ak>m2#Ir?Ib!wbz=M)wn>7BflIt)yJ(HaJvQg&co*o;q_`NL&h!kq*6qPm0sAvi1a^FcWNH3ea);m-bqT(gXT)Z z=O|E0yu^@+7nePzGOJs9moUF+Do&3dwZ0u`gPVH=*J`GKpegQ9r zv_Y&nyM&J3RjfGDhS9I-*`=-mm0fC#Ch%V6BW$5W<=oCF|8>BzG~5(mBsNhY*}uD3 zb#^wLJu7HFyiqA2W!&-x#$j`dm2MW}o_UO^C13Jo#dQ8M{Zn=QMX}`wX^cp_^{=kK z56zj0&prl^C`yB7US9f`QY^RheZ%bL8JNTl&n>pSBe7&zsOXaCzkzk9*J6r!)i@<~ z*kUCa7fs;#;wM+8I|AGVfKx|D)~-AlJe&&U{ubbE$G3RJCv%UU#cwfj?+yT(``y&Dv=V- zP=KVgL|!kP%-Y@(1?GKSKT|m_2x-9*N+Ts_2u-JE(B56i>Z7fg!y$EF`L+9SI7ea4 z(0F;_qpZDF$m;&3%x?S)r^k!iorNcyLt^+&R-OKe&Vg#Gj?^$bC!U0?coK6H7@j+n zA*1hP(&C9U43x8)24=39r=B0lw)*Kh_22$kU%!YIvE5OCLxA>hjKqw1-Yk2Jw%$@& zFV!%oVWDbNSnT;|qWq0|G2{$q@#fFidaZ^HCtJuFKTZLu0)838gJ?VP6`Q*&XgpMd zAty8{crm8UOMU7_( z#}H+@D616M=?meMc_)0mu`4!fz9`9Q3AA1+p|y7*lF8hDZ7Hq2g>+o2;q#_<6%g&P zkHcfjBdA5Nc!ElKoajr(7V5@OXpR{UKe6ukzRg5Ra(_wM!F%R0yY# z+LRhE3zt7n5d#11d+l$J>C&9C2Gf*HysSww91pN*-U;^AU8TL@kox@OfR7ZJYlW$A z8O^j8O4(i4&9>U}EScVfS3p~FV)2POU@RL7yV$$p9PJf5@YqD*=pohbz!?eRHRUkl znNki`!;Z>s7JjipjT`PrmU5KQjAJQ$uY+TiaH1I6zdXuY6K0T^I*}~fL}HfvNOw#m z)t06d#m;cHaysxS(pJ8*9G9^~+*#y430o&n5I>o&GB{8HH48WE^PQ|0B_c)}sdDa= z-LoRSWGNDXkWmFVZlBYoK)j6&$Mr^HB#b2(#1si)2nQ5cBTfrGr_Aj?fBQnTltfGr;zm^F$mI6ksE6GcrH+3q3qM_{h_t#|HO60_ zoyDQ*ezsMeWyU+DjE}s?mdexYs_kZ5#RVGXey7e$>o@!8s_mtvbg!~Y9O3s7pYBs? zZ_PW3fazXxtxplNK1iB2fq-R<$}i4|e}s9D)$o1s09|EQng3C>+JCW)m!@UR@c{b% z_p`0~EFs%Hstz)Xm1iV_fMqPRCfBmBw3h>Q{VbT$baUG;=~vrd{C+)tR{>^kM4kVr z^L}=f^sv3;3LRy~6i}@>a**|%-(En(FrI~PG;y%@8apbl@a4xP*n<&_UXKDU!35oYJiB)6`H1XDEY+`CjHf z+sNSx*jav=1=H*G7zbZOUB{4ToC24d7VKb0{RK9xK1w9}UL0bxs|jGNTepfUl6=Yi zrMn_DY5)Kr07*naR2-7T4mOyh*DmTcsToJAm;|ScBcjw%`S3w5_w^~ou~7xrAAkErZ=6&=63b@0vzw&e}q@xcn8fNRpYi(NNOJWS;%%#-I7)T zx&LyEzHuJZHStB2MY1TAx9>ZuJI<+$O1aj0=Bn5)D@#ZD?iD~W;|Jhaj#X@~W!_Mq zr+{KzX|4muGIrnO_C@vW$;9Q)QtNZ47UcZpt16@T^tr1#!O4?>CfUIfr{Ydy_WY&l zx`^U<#j183zLBc7lYJA#@qwc{aIE%O$c36%sAu2}7hrLQS+u-t|_yI->&`0c&_InJeS zQJhK086!6}$Rvx8C*ut~IU9$7a4ru`q}-ROl%y#kab3$MH7NPa z`8+owfsHA5u-kqI>nwLrKXfovcNr-fmc-)tK~xOCl~s5ABS+!}aUo$STTKePId|A+4^x3R`Mn3w%F1oTzb*jg@bEk}1I@s7eAxCf&+ zHTXP498Sg?-As=2@%o)!{xrzSL&+BMoKDqG5Mrr8A{-(kuRAklQVaF0Eiu*uK(^E|K~I2yD?o-T z%EPICUKwiTl|cq34>d5s;2`PekT`eg^SV(JwP+3@PnaCDnMbT@BNDih6vv6gA#@nyS(Z4M&*JXj(_yzV zb;uu?KJ1T78G0)p#2a`uHAC3P)4!Vz=oZS!#>t8CQ_JP!{Q-G$vb2%p3Gawm8kKb31w2 z7scSsBx-f>V*C&)T>|Kucs1xGH5yQ=ZLug1xN?G?RR3EQ(L!Xxs=knC*P! zi}FY$#+ckJUQHfMOZ=^DG$+yQ@>1hVr!11kqF@$f(QG!DQ#q47jIAj~s=P6lM`J9^ zi12kd!mNx@JeEEdi!TpTG*_)_r8g6@=M!+`<8kMqX_V8wG_ll>T#DRqI8uODrCClwy?JQ9 z95ipPvY2yTSBFS*Nx(X>!ZUlKc>H;UjrY+u=cvkf+fj3xdkUTt4p@Cx2l_?<#4?8$^b8Ep-Pg~~ zeFxcn;3(hi>SD*<@7cEd2zw45(}Caq0amtb#32Q|Ohf`~YaxyvKcim1aJe6+Fv`SA zSQT|y1R}Y7K4bQE#xb7{FSxVXC_GdC{?h>{C zef`&{ZCtAgM^XI>^H)H`Gv}`$!1eJ{7Z8gyVb2M$EXA!TdzO_{Uf=#TN-CxCmZdU{ zr1+PDTRungr%_hj@I%J&qQWwjr=%C2LK2W{Bi*jhrU|$&Uzy4hT^g1qVNXZ1G5aEz zykP|-n|<;F2^nQI5u|Phw5K02g&kZh*OZ@{P3u50 zoxQcpteS)E2jEKB+qrrnANFF5B~pH17A-8M^K3hIEf242tZoS{c`wpb2t;(d6=|3S zGqGe3qxs}iTDzCCwtFRux6h_<*9?jeOlR@l&scb1DvP>i(%fG{`?V_OG=8l&lEw1= zXO6)?{XdIetg7|g8{Mip7x|kp;vq3JiG~x!wDwi9=3qMxO~BCtG^H++d~?6LHyvVa z<_Y77I7rF8i&Ynwu=-LRO-Ea?MqGq_Ap-UcG)Fd;n2TrTJwkI|5vvD^nY;2Ub=_6Q zu`h?z>|~lxEuf=&74>`DF$BFzX`t-L{dpuujr_S(ta{67XC)sM-dx7#I`0aYDd?TXJyRqDGHl%uMC^<&$XXFK2Cc zHM8nxD;sCA5)_LfL*@`xUtGkh%Qe&=ZNr@I!|O{|&2h6W59dfPZxrhpx9wMInN~XE z`t|>%z~7C)L#y6~Hg*DQ9{B zYw)`3faJS{10?4TW#uIu_-{J09;@Wyc(MuDa_~4Zu%+9XJoh0*9BCO?&Yb3%SUl-? zJOw!1QH+^!R2~0{j{aI&PqvU0wBiV6sKVJT4D1XO&o6&mSvj{|sb*%)yz9?jfl~GT z{O2!AJ%5&9NC7F5HK(aDSeVjLomeCHW7tn}%)rJgjdWZsqh@Q3aujfCqm)ITOY9P* z5vDSDA|z%GqxRI-bo5lR{&X{P{q14NJd(`0D5)7n1)Q(#D`D-`VrJKWdA+%pt*k~3 znL}whK9dbss%ZOu4Yr6^6-ho%0XAnk)=UepEP9xBDp=iD&a}E|IKt^zgh(bi!y|{& zbZ#1}t}dbJQXR|pd`;2rX_V~yf+f2?X3@USS#o$Lt0||AYGyXf((j*}v^p2Xs3=DT z1QZ2#a%KW=mOiY=CLO*1FMIF(CB>QM`Thm>>^XP$%x+IlkJK%LFy*V#`;5$pwB`yb)tnQ$y^4 zT_ve>oS8&-?<&?`YGcVaGgz@_GF1mB-zE2YCh8r{Cr?+iyXGo;>aMZkll5qR$pDmmzCz;eaqL)iMr9*!sXt64 zIMM`15^-G77>0W|pFB~;fu`&1Y3^a!6pBShIhB~!J|u6Hmr;Zr_WSE3xQNzKO{OpvZ7t$grO&iy6>kf^BE4+!Tv^Q zSiA$LQkqN7*8tmI>>2|_$~I4~i;xwxU?oY%c)I5t;aD|%-g;JnRHEvWV~9G^+`pA5 zCAP}U`w7Q)Nv^vB>SW(;DkcTsOM1ugdG#5-s5?Vf-2tMqZxdIVgxLBgFx;b<^jHl? zYf*OMizltc5lSN8=L{stbw9|?MQ1qB*vIGfXV7ewDyGIBiQ|sv67`K?`g4scBX?i@ zH5N{6R|-T?TFbNc*5h+HRtI|;t}y%kCIvK$UHIx1J856Fi}znyK%V~rwk$fL-nVJx zLHv3dmRO7oBcuqZN&f>p7dr!xKj~}6?q~OC+r%<&K2j_*WSyzs@>@R;cDIyQjw60;?B{`d8vg%jn~=V z*w5s@)Tq2mM>K)gC?x6~!?tB7I9v_KRzTOZ!>pb24I3BlXZ@UgY@TtH?K94Bpr(&K zEti-xsZlB5t&xawABx$>F#E|?jy1?VxXR2A>(%w~#>Ek-0H3#*SYR~M-mal$b~j7r zttDAB&Q#3?r8#eAaSG5THId}L2BQLI<(q$xNR&h&PQTbD7piqnfMfZ00jL5-g7cTI zD1htaxr+*f6D7X5rpQZxvK3V~>nEtLZ^x^Z;R;BNCu1nixgRcbwg0+`3);Q!2;c5K z#as0zTJDAJqM1fx=H}*bz62Q1+fYKn%R-!;-YH7a4!1}nd)Z&5mi>Ka_bQ% zE+JN%0G$AULa0_?b12U76Q))5P2gAnTmgv%sFv6H%l&No{2RtT{Ck2?A-xFjE#yf7 zb)#JP;i{@%kDWY^X2{wj+0A+C6^a(~(I+!icCdVZ$A+!yzDVA%>_NqX_?-`?DphLF zb<>qgYSDAwU*(&P<6{^G#(`|bjfc!@b~#7^9wOEg)|2Om#U^}Bifwy8{E z>2(BzI3A92?3DSNuk_te*59(;OKq;!SEGR03zl-FSIo|@)3SDhB9yCjQ0)_h=pba) zgd&B@xRGAMl2vv1CF@$Oqs0M0?xzBZW#1#fxoLkcV6XZ5@%js)Ku;8^K8SpmSXBpg z0p{~@`-PY<5nm>7EXrx~C3Cu5pIk-g*5kSCK5&R@0zCJj!0=$K40|YFWu^MU5e&mX zEZZ}M3?buEtlOhvF_W?>ZdFs{i{5FpP9<)%wMt2urbRB@@z^9&!?m@ z>7Kh;^^5<^7p@FG4O*C&mCkGFR(_Y|;E`+_wL^ySeM&m3A`;;yu0_(PD#4+TRO`kc zQ#06_cBcY-yS1TodY!EIBxn!CX*R_6*ukfc436bwa5goSUG5xe4IA|_A5EH%mas-g zL}y#T%a_h<_IT6T>`&t@Ls#>XM5Ot}T3Xrk=3|XY#90whkB{HlGI%p<5c9JJ(PpvI zZPnOlb<^s!G1HaKbM~PWI&AoY(p#v&=S@WY_sLC&F-Mnmd+~s z5MH+1DRjoMdu4B4%(M1Xnl1mCZvTBW=ste$3*(EGDog2NrdaxZI*7QrR?1~$83klT zlSIP_##p?3n4LpI`h9G&4q=-&lbt~eTQmntH9IePvZ!!cDe?z-Cfm-`dj``U^iUsl z@su-#qL7=>4iC@Wm(ASFbT)cZ*b+)%gPz9LP&RAb*}UhoGBy$*;no=LPVg7En`--g zGyB!veV{p@@}+%xZB_jEyQ|PCOe%&+2J0_#y|#*VH>qE54ABL>!L0;__A2- zce5ZAXSg@6TsdA$zn4AMU(yk@@seIZ%p-MQDgIzSB`%%E(}wcieg8m}V<;OnH`}sP z*^@bxy*X)o?Xt7cDI}&W&~te#KLf`<1IIrD$N$FW&(FZ|f4?D;;z^jS zlJ6sm+=TOf7Chh1k-C2NHC$%?N6m^Y?G<2Fa_d3~WzFY_=i!P;GbW1Lh$y9QBv44P z^|ySr@Dhh=Zm_HREb+iNTv5r%h~U-@lD^SQd%B*Z)x8|8xy<6p4LA}8nT7{nBthIU zhOd@ihgCyQSFGA_Be|oc;cy`gZ}2rBJ%b(3Zc;ONwv#yu3QQb}GiX+h2el zD^W$;@ssD(zkPqHpO7>;0|_)yWcy<3_k1+*$HK9*YrATii_McT+kL^90(|AW&;8J= zib(-}MQJRIZfRo7o;Sc)z5%UkHmU0-g}o>egW-fK(&h7KFI+Kgqy{nwg}EG#7qM_z z)h*z7RbBJIbricuVe(35t}vGq)VHir=PKtcjAbd{g@LTpaiM(0*q5fUH2uY@GGLTq z_a;n6v#4IAp(obX(&Sc_kohW`TB!p>X(Fs)$rtuZ{Sh3vT6#P;JY1oFJ9 z!5)>0_`P<5UK5df^b(UYMm6pRI>Fu_r?KgOUS2 zQe{zDifp`C^8o9vO9pEv6IaYa^A0EExt~aE47OYwPtUES{puXnv4{;US0MFFBLuV}CVUdmEYFG|L1%l=4R!Mq-^N&t?c)ZZ`G%=P8i+vvZ%ROhU1C4+n-T z6`_oKu_3a@3$Pl6WCo*HOHr;#na zb$rp^#QfEN8*nV3u~qEU#r;634vW-(XC;HA7iQCXx|WUot*kq;4QFTs9?i_+6{WHo z69O0oOH~8O6Z4Z(Jcy19i`de;nzqBOIAa!~AwO|{0?n7ik#zIeva_vpV*yyefcUQ_qnBynAfkrimA<0ZtwXE z3Ykf+yR4_8c#jbha!^RD5Cu4vW?T_=$&kv+WXF}&e0H^()-URnGDCnKvBDCU0CV0g zS9waQ$uyeJ%%;1ijxA@`kVw^ZDVrVJS;}OYR+Xx zlBAaorv1_+wy=uM{wB8EsHf}de0JPe#7>s59drKZ>ZxOUe+%>5O~tYIHi;)hu{z5ET5=-eY zIxbJ6iz-^K)zL)@8~PTrm8D97EAwC1wPrrM*2LF6Z7gb8VBSZq41ZoR83kE9zF-Wk z*H+O^6CHB?G_awkmW|gM*g`w&uGO-I6>RHY!t~ag&)?|Vdj3AHo@&nbL9=mdW)8|8 zFArBRhF?o6tLBgpRK#T^V6vnYSxX3OvcAURc!l^EkUdU0j*v8_N^r&t$xa3-`C|!f zr|0u2^XWR-h%cGMAxc0|=u%v3`DW&qq2o)~cwx?1x~?x|N6%^&wJpRJ z3@aj*%%#d=Q!}4y0by=u{r4JWN};5Hi*QmE^;Xs7lKr2kB1%!suW&v_*igW+Qc?)f zODXOCHsDymUcWefXvKuJN=mXH=i3?AI8p~&>y8o*Rbmx#NFt${OA*HyK6rS!QsEwN zfaMd`6Be?U+?PNJar?u3UDL-`b$zT~b_~P!5CLNp&S(*~aFU4cem;0~xdNT{xAw8* zlMRH!W3XxoY*A5MC(r{WOn+mg%E{f+a)kqJ*V$X!j~wsFDmb|6I-PTN^XI2NK@V0c z3wJdSE4$kS9zBLj0dWZo&nSwrpHx=Mdz$;HUHYk+X)HE=hJiB@SI?a^&O}uy`!&0$ zzqE4VP9c#VR>1L>&F7V4h!C8@zH;SWBn}ULA?#=ax$ZyV`1V9Z9bZ5)Eg%+hLVztJ zVI8kBe0MiqX5-5J7`{;|b6Y@l$+HwI>PZjPa(p?Qtc7J0*DK(JtNq)>U$0^gRQHMD*})&E=1N`G)D#RJ+hFaRd8C!u~l%S z`8sE?^A)~8u~Ota z<1v+c82686_KQs%uD`+22AKLrqslZD@}(#O4ewZX)|}_NMmV;_yhn$m)@!=Xftsrv zZ@R(Jrar!DIm51v$Cy2*MxDRah^iV^?(eLpS~((i;Y~fv`k+yfdL`0E_7~w;1#XYH z`;1UZX31*Qq7`aQRo1r($TMS(ga{yOtJIHLRB99xe3N-fo)>XCQYz3;Ar5i0(sLE? zd-mdGRp*_)a9O!HiRH0UrUVk`u_6{Nt-1xAuBu&)Pihan49$?-@Q8AZQgvhhK%IC4 zdi$ZL5BhJQNWD^@$$Q#1Y*Qqdo9;~t4E<}x>>)GuNCBU^%>AOwEL2f7$|XlgABJS) z%Kwvm_!y^8U*x&~bXR)$WXeos7d>UxLOz^4n+eqgpz2krUqjJSbsgn8d^B;!t@B^E@l!Pq2OQ{DOBBb-`dZb)@_F-kf6MXBF;uGJu;khZ;p)Vx3+k9cT$fr}>UvSD zOGdUho=u!Qlb${?`iAwJwkv=|?z@2W-k^>s)vxtm=lrFs%KBTy2pGlc`itY)jJb>G zy)K`>L38_h6X=y1M^xd8_-$rGH?Hnb1O@3=+*BOP7?-_;xHE|2UI2Gd9GigoP0(1f ziGKu+eR`5`JfB0yPb=lN^h4yl1T+@1nVh%WF9kR&CA@&f=DIBX0?9ZIMH3u4eo`st zg#aME3?a*@Tg@4a^pnJdBtRJxqJmUADCVJqAGS?sag$HA<- z+3p%jRn*2fJ%A&U`Q_}f7YY?CNQ)!OdS!`&`HVT*? zFJNjs#ylfHt>3}d4m&5)bLa{Ln35aill(j;N0Q8l6*4Q7q}r>q&hF+5r=6`13-4<& zwQq5111>PpVl1&FnaQNeSborw!J^cA=*sv7`|SUfgZ6vbXUpbrb_&OH2C4kqcKhAD z?)MV+=Aj3RDR)Phn4ZqpL+{{Z=5IKb{Tq&3B@_2IoXfnEvzfnPozuZ9?iewDAy%z` z-)E=Lmi6D*5lE#mCd)D_{eY#Z~^hOoyngwxq6oXJSzyR^GFp7~48XWhZ^ ztb1ARb1~AZ;dSS#IJP3M#^afHvm*6Qj;7qhcbUUDV9(@`eHcgWgE(rrhi`KRv)JS0 zIgiHk!?Ianaq^`jozvO>#L2Y(#_^oHIOH6}r@mC?7A7eXSA|fnQZ=VV1vt*9AeiJ2 zmLM%Q7e}&E*yqY*Q`kkd=He5Fg+HbZINHMw51aB7@EVBNFuYGsZ<-L!FxQBVUI@5AvG#WlW^<=O; zWT!FeXNrH14U{s{q4CDh zL9}QAT7nUtw@1-@6*!GiIP`LYfzd==_u~~Jc+}vJ*}q_m^*^yXYU9=1Qlj2mBsTEr zw4roZeocpFV@@)ckv13MV1z)dfLPGr?IFM6h-)xynww|!Typ&escBgZ8=OIIFi+W& z2emSSA}R>!D(*r+WJ@%kthhV_akVbVe&FwQ?)@1!{uwy_894qKIKHh3Od5wyDTtE! zIJ7vq-f`5txrM{^ee7wt#Kb=@#jOJ3&CCdwAz5OQ~>xbio=)M9*1)pKh z%03R(-C)zAgG7902C+S45Yh`UEF)R*#ug6L^m4fQ3X|ViiX#@srbp4VGAg`JbFd2b z)?K4x>DPpU5=b9I(@HTcqp9#b$&_cS+1+@B&+9I*bmlsxu95RFi)Ir%79drrkwTIS zWWFS)Lkc5lHpzRXfG`y&MTBByswfK?c?vLt2NxZ)6Q`$N<)7-X!j*Z(`+qsGM zb)8I_w!pM13l^zDRhkUPPhL=k@()+87WZUM(P zf$vjie^7q=$mB}{W^Y=jcbU9em=BBb#A(W=^HEDMucl+I$4ed^q0Rrt$w z6YyDbsD-sIxm03d*3;jo&R^Mm2gPp7EId78`5jKMWM!RdZb*ZpdCdj|981Qr6tb$( z7&VpFCgZxqydRob07#QZGUoy@u~L&9{Wyg(rn)laX zjd}5hig0;EEja>fv7J9RJJU%&L2AYr1cZ#x6fih>vZNd zPi1EPG#0imV?oPe=GM;RjR_wP%scc~CcKwkJiv8ynfJqbrs##QwM@R>{Sk4O*FiLhoI&G(x@@fBCnh)0FOgab!b)umn zLg7MeNjp!?d5}%Cu#q*)UB3u_q=2AauGs21$yBynSjMN9YT0T71xnJwhReb_afXrvr1n6DJ0&oBEeo2j9j1|~O8#}O*S7aor@Tt-e|Chg~DvW-US z4pozt=foLK;0lh!ZHy&5-^uF@kFb?heAeH}#HyPa&$p*I7bk~6u#liwFNeyp7G}|L zb`INbtfA?^8nR-V%8%EKa-2S~Th7O->$nmIR>QAs?8UxXiWr+-sBDp2&(3DsjTV{@ zG^t$Okhg+ZU<_fc1V_ToGqcCBSu$5?Wd8bjYJHHLM!%6Dvv3eK-%X^uua&OTTd^ek zIKvUbdKq3{9`*tk&o33L<{Gxr$jr9sxB2{)spn7B8p2);1dGrVkRq9(1sdw1u4i;SmG0g~y3cG<8SmCWF}g8U zsivKU4qjOH0Nv{TH#4Pq5f{9^Nm**M+|8Y=Hrz4yTGm#@m<$es zD5BOtUrYe|N?h6~++tB55JvoPoY68;62oXeKbH-?4XnA?$>ep@nX`5l^H$GbZu2x| z)K6nk`%)IREMk7mJl^_X5&<=K3vt9XqK|o4cV!`Kt}bCye>-cwTg`&?3t6ycDRbKw zGO_hT=I#1~jW=po-&3Wmx^M6KTdAJEkEpv8Yk!z@`B`jR}ct|#;uiM6>zZJ&8%&&vW=N^Uua_F zlaJ$&^*36n94?~aV)S4EAyMqfcN!(wBN|!pVNBliFLbgHeZ!J7!stA)l8Rtj)w z2nz^PtZewye(Wc%|BjF}iAO4cSn{SMkIJeP$3o&uB90EBa0!t>jHF*96iVWheP2wT z7g4PyZfoEC*UbWn@DSiAWK@u!{Rm&q_<>_hu%-S8UXLjGV_5Ylp;$R_*H}J!WCbUh zK{8sGP3lkza8b{OLggf^53#$tmxImFx%dzf*I0akQf#88j^z>YkLJDKEm6Rv*eNfb zuug%rws0IrB8EK_QTv#Xx18~1FEHtyl`NgJj?UV>d|LM%dz-Iuu%VBAO;`A|?g$T# zc}Xeul!{pLx^z((2*?*ziemwe3oMVbr>UQBTY9Knx($0Ip&~$>GBykWj&s!>%@;0E z>Q9>?1i@l-??^fq?B!s6A3Il{!Y3pFaV`?;Ls`egWkxSH6@vmK+{W=GL(?ycJ*|jv zsDh+*JiDtdaiFD_^~?6Chypo|H5|ng&nNC1$HYgf`K}sH)xpwF*5i)nVKpM^S;@6O z$oBcCIndb4j)qf80V}aQ&Ug}wVG#0GGU?fB_Sf_&)$!bqnv`RJM5u(q75wFQ3piSB zf|he9G!qF8C!lO%3l-qz^B15whO?{Yk^+CbmLEnF(u9^z#9zUqMX&PBstX)g*{f91 zJ63(i(rFzmnA*m|No!dAPABaX_prbE2K(x-Gk;Q>vSOC=lku15c$gjYPjjfTpPlum ziFxi<&!n8AfZqx@ey)~7b^U7mE&i~5;P?f%b^L`B*AR#nsq5!5a*2D#v8(!m%9q~U zbcmp@TqzW-A+aUTCFUE={8!o(uq)(>S#P$ewM$fh!9WEu$9O)kI;X6(_f_?<>fJ7u zPVQjN#|_M$SkLSa8dx%M4f7{9Gjnn^&%gLCo@l;u%MjIUzV{*KKHJ8jy6eiJN>tyn zcd)Cdq9R<%(!T_c%vVMQZXultuq6BE0kyG^iUr6Lvawk!=A*{8s9BVQQt5a6hWlH%kHD1t2a_)z# zig0`RT0d=TH`39$jdh)$(Xswh+ShioW?eU{*KDAD{U$z{u@JXBLqZ(Zg)Ae#`KHx! z9{UG?V+DMM#ZfA$2(pqlt;og-FcilF0q|tLjB)DJ_w-+fic;_9PSB9?Zz`@QPn)aiM>&QNpXK@qaBOu) zk@txmw*b&jKJzDP>RZ|M^Ux!ddF?3qSNIa)1Eu5?>`?zW}U4ERgzr%Jezv`bqsR3Tz?2%Xk#v zSgfv3oIb05^Z85H)VWLDF3ND_HlpV-chL&7_5+OgxE6L&&wW@v> z;P|iWcL8mseiy~B0*+OtvFyhrqu4wj^_(b2Fp2m|lI`$v?uTn!y$;`By{?WS>T(%( zrrKEQL%DyZ($G{J%X61RizKnReYYaG&R@B%+(cx}l|F&$75Gh3y(c1jj|yc+>k$_! zrI0REZX<>kR8D9XA=!y?Gg^Wp`1h+A`|CZ(&wu~d`wRbG*F{_g#YU)f8I<{S$~<14%*tWn&@?vM?R@64 z^QKQzJvVzqdTS;@O4d#RkKN$_uVxP8i?sXr+C7xDu}tP09wvlh%nJr+^I6#M&fu6e zh1r^)0xiObY&UNY&Y&fA5WDU7utj&VC>CW>GQ#A9pEfOr{f<=jdF(XW-8`m6aKyxd zI-SOhJJ}wv(xe6Wy(g^tL}N2-EJ#nG%bCSazm*Qn$8y79PBg^)n3wKg21o7pveji_ zrZ>V!Yn)iHg5PCZSupe$d||(rz4oDO)cwqf$C+uwSmg85>B!)KYbaj^v#4@s@>)(d zF9!@3yY1}CxtAlhd)V(DN{c6(k3u1yE2_W{5reo_8fDm}-ZF}a`12WMi}9w%pe8rT z)PRFOTZb__CzbUMJMC^iE8PKB`8BHiUK+hN+C5pcy3%NJr&ATOu)uJzFc@aJE5T}e zfNqzAI**5sEDk=(%weuC$dsZ=eitbw=8hBf=JC5w0q=(rObzNRaC`VVX9(w0e#!D! z7<(Y8>b87$n0JTV#TVIkus&qv&7hz0-Z143oflKHSUl)Ato3G68}~5Q7eq+ie;%5~ zvCKiN3AuPYDGqQ#crRy+&C8_BA+&pjvMHR(^df_?dX&Em8^#w|sjSnYylyji+vaAj z-@(FIjQQebara$Z%)FQ0w14KT3o@?ds0@1@>Bqv4?@66Fb77(GyqCsL`b z)7`=01Owx+7{js0V!V*~8+K&`~`2P{DP zX*&tKgc&FHN4Xfma!Pzpuy*=UzOCy5}AR!6z2fiPC|L;atvNG&6=oc^r<4)mBpF6iahe)J=<)tyI7#>u6-DUHtR@4^kYY!Ecd;tGMcFLH-?8s5 zAQ{V2?91`xb1uK3n!El`oawU`DCYUi!qF@|iwH%E6tFE8(_-%>=OK*-Y#@?{ zFBD^`6ol7BF%6Baor-ZT3}Ufblfv}S(KD*>ywn3lWusLgFHB`&n7jNso3?y<>-g0T ztJUYt!rQdLR7xj3$!l+Ya0@uzy7L>w{#DHtWg#{IoDC@6yXR}5>hM{^0_q>msF9Eo2WflpYG1sSxTo=10Y9qr#WW62LIHKjNFfC4u% zb8Woc@(dgLmniULO7mR2;cFQdTZHsrVUrVh9@=xtG@zQq?Giy zg%=k6j`depu(7v>IqT=4Y>Tu?wUKZZr?LLb6t?#+W!;IDxC%VVhS%vIh07R0N<4)( zn;&9}D2aQT_@sV@+GyG$aWWDXURm)J8~c~A`9>Y}2dWhC`=5{Q?FZqUku{FyjJDE!3 zfoXi&w}P!#s_~8RDy5e@R)RAxMr!_jOx^Ypn{LckIk^*R-c#rQ(}81i(Q99M%&7iAyEt^j5Aj9yf^U_2`;Y;AiwetMphgjFUlFr_G zX12{!8EL-2{dlxW90gW7PR(Xp-%8p~uEbU(R>y@{yro#R5^{>Nc&%;}+gQMso)t`M zxIN(brd7AtXX`Rg`z5<+B-SD~EvFZ?U0ddw4`>_ghK9D?lvEI-1kzPEMtxDy;lE$x>a%#`^<)^;I22<;PT}vRxaEBU-NVmS3y>9bG+3+16Lj^oGewiRBNC#A%c_1!(gz8u-FJngWFf2e)mtEGtBqj1dbzk4Na{XE`2yTd2U{- zdV=nrdb+QyW^&y$HNS}cVo*2mM-s}CUx+hG^{SPcF%tqLM=Ht8wX)$tHQiS`XxhIH zYa)irUqa9~977wa6oUd33h?0Z6`_SjsqAL~E$wj|Z!Ld<&DZ9sXQOG~avTXSqE`30 z%hmgBxlUeM{(Cmj%9_4*rb-5LxD0<_%ujs&=Br~&t)GhBUy4s3gFjlSNHj7oq@gPW zD|rRuCOEECX`;LlCA&YV#=DFSAr85NQH*eexKWJXEB3)cq^d+R%Cd5En7jRTwlb5= z7h8Dt{SQbBl;DVt#U^Z_U3C?#g;^)ZLR+vV>Gt#{W!y-xKHAYi1Id}fKOqH#{)qMy&j6}7RuaC62Tbu zKt6T>6rvGaq3FQ+WsawxIu@1FO)Q^mO;^q+N!x>bG4}!o8~fSbasneXQvH92mP8or zG5c6Pd7@e=Lcgj1fdx}rRi3brF+<@pa%~Ur<>Ip(ZSG_Hs-xt&?^pY3MgEKCqI)X& z_=)8lYUo$$)1{wusJ*OM_lJtguuDV7SEM2xBy%kgEm7IF{%8S(`Qw=TQ6pPd9Odwu z>wML8k;xxaD@BYg8XL$pzUc~Mf@FqgH2Idtlr8qREmx^qx=pFnWv!Ior0l?D4G&3d z$bjP55Y=*_QUb48@-=&!uJG9!0c^w~TtGP!Xp?nY?CnL7BWhJMSNSI%U-sq^aT}71 z>r(Qa53y_cIrcVRp{wd3@xWLGQrq>Ia-;~khcn^vRUE2DB7fd_Vv)*%{;K&R zGbh#&4wj({ASI+r0p`4g#C_x0yyO&z>S5!`g9HQl#KV=ut)rOnd@cK{uW-1fkIqGV z2>B`oeqUVedt=@QnEYG=2O3Z+@Hrp1nu_CaoUmR=p6ww%UvP$lO?`acaE7pLBwnA8 zLy`(054kH8@m4a91zer~Zu8%C{J9faafL-~98>d2#5RUc7oTKb%QZf4K0~STdxG9_ zM2#)Pm8kQ6R=m~4;W{|p1T$Z6AQUVi=r2NdRg%mZ&#qPH*j@Jno0lFU;v0okiz0UK z?m{6g=P9?GkZ%NfU=$LyB&tKHV`cp1c^+iWbE`R6*T>$*E6jPXSv_|GtcgNV$e(g8 zJd!;e5sPM1IV=FEBEBlSVo?^0TGa$c&4{C7oPoFkl7kV6LMlN16yQ%OFC}I|qK3jr z4jn(M>YmHJP*Qony7y8W>hVIP7VO&knX0d({;IBTQ-oOo4&^;wEs6ln0@fWrbyj7~ z3W#a~zPaj8O@KYk%%PR&A4R@U?g{@RaBNyt3kh0k$zV9H?EHh0E30BL1hg|j@>@<( zVpV~e%io>;URhU5 z{c88>=6n&1GIP$NTf}AMm=zJ%pdWvS$}kpyR*Nb}i?Gb+{ve~rji;uei3^v^@0S|8 zzI8w```4hc>Ia%?V_(=*1_^mtUf=9LGC*K9$*Rh-S~uf9OyF301_BxnWE>wJP;9G= zWBIPQ>LbTK|;e%bYl15iNZRiOTxnib>o)eeR0t2gtZ^dLyboAr{h8 zXPUsVsKPgN?-(Eqn8b3q$0tskQCD*Ryjsj8%PEz#*oDWLId7@z5A^i*(XnysE#UYj z*emn8*gU`T#(V#_`dy7T89$;>H{_fL0OMQN|9=FI6UvP)6iq6%v5f2e$Ih6&216{& zf9%_s!12I!98fGP;6D__i0AUd)n4^`b|3ln_D1u2s_4iIl1J zCSt^h`fTJohf)zpLks$`CWfnwR#7o~1Q_@?_6~o3-`_V-^6zo~B(_iTe`U`ty}c1R z*}Ro@pHhXk8g8C;`-pfG{2@J)j?8=M47ix@^|HY0qF(pWs%x}qA)112nshr&5ho21 z4=WQoOCn)9)Bc&0mS3j#!dlFhN$Lh>S#%=hKGqRr*ji`Te{Zj*;i{W7Wfo z%op+}$PI=VZqH=7%fdmIi+OrdIa}GKp4IZniN>%bG)g>n{v|bybzV20``x_e_mN}7 z$&Qw&bzAxinpfh@gy0|Kle8h6O&iP(pOtrFeu_d-9Kj+Yz6g&xY&2S}9Je`Hl893p z*Lg8z7+rVW!#>+ky7e^PjT=N`!%0b$lAeh2fXBkpjQcofOXsL1jfs%}fn1$&Im4*S z7|b@Gi`GzxC%qwbf0Wl!Q}{OR9=`NsQymNQ$4G>PxQ~hJV>rg2oawBy-pM{^Halz@ z?|KR;aOd-vVX5p*{S^m1LzKGrrM$S>2e={?Bs_V%m~EleGKjstLG1M0%^SMuIQX!| zPIdYqzHq0o#hb>TJwC)O&K9Y}ChlMo=NKBH9Gj$9;q{l`^AxMSmDE*+=Akl>!xCEt z=h8Dcnqj5e=A|p7(H?Tp7P7D|Y+*yd!6u)FHDMPGhK;sRfbCiy2UBynFf4^_0fWEj zMO1ooG5keDLY2zF$Lkr6Com3Qa6CcT8#!|sVe_&i<1S7)?__Drr`EJ|eH5Y4X#PAX zjRQGDY1jNbqvsRz7NC1ecr???tYJf0Z8%w$>tU2vQ%*r|56b3D$`Cp<7mwyeRox^} zgO6s}XmVz;Iyb=N!VqJkUgEl+SBIsrGt18MP>iwB1hJ4tksjbF-Qe9ECyiN|9I&Tz zB>gV-iDExy@LVv&^BMOrJ^Po`#_TM}jWFKnQ%-pz4zT!J#A*NlAOJ~3K~#leIJ6Rc z+9(xQ7781@K6D7XQ--k9k0Nj6#o%A_UB+FUv)si|>mBTK{hEE& zJC&;ZSk530=iJQ!$A4wN_y6FK=f7~oeg~&>?&E|dhsI!-Cp3e&W~kaEJzh*&T->8% z&ttm5Idl=Vna%_M9&^jT_jmpb9IK7mzim-7nTP_Q{Qo24R$0RQeE$Dof%Ct0a*<|@ zBOX`E5-Fr(uKW37#X0uWT;-d_i^Lsc5j(G7xJ)t6q)-%fpC|?be#yS5z;2Y_4CZ45 z%Xs(M1&F2Hw~g1RpZpoQ?(yi}N)rB2`V zlF3h3bF>EbHe9B5)@Bt%FJPFcu@d%ijE}y=wyNWN-gJhYZKru)^b4waB~3$<9c~t& z!eW&|R)8o0L*Jh;jhA10TNOlZO$wtT1r|x+C`~Vi-&721dh(W4l&EgKVsD`f*c&Hk zBya~pbZz~VtJiuE#c|{6bqZLM+#gZe3b1?XtO?wnyC_y=5%oDKVBPXvlDnftRS_>m zo~T46-#91@TVWQ5a}{u`tjKP_{o@}o-yg|U%w1vME?-r5i*YQ#uQ23Q@hO08Aj*;B zXKn${Mdc$1NsKqAQ-8$F0W&SJ<@)APIp*P8Hr>1TA6G16$wz|Q-ANA)ZD|^x% zJ#8c(%OyJz!7BD7xe?OxQh2-ZIWt4^dM#5MCaNZrUo0Q>Vx{t3cVZr&^)93P(h4&3 zhGCDzRW6Jro+Ps*ohjXaqWk(XHeRk~Qqxo{k%Y=X%*fAX%EosTpxxEi#Jde|V~^_u z0>cUUrAd~Dt0PFMaVmfZrF=_QAd`3hyB~>F=8a&cSi|N>t78>>~V$<~+x-Yep zlbek_>A{`|l9MpVPKdQ<5n_8Q*}-0;NLjtEJ~fx_>kX_v)ktQZ8wEV+Bk%+YaYsT} zW0|<}tax%gxFh;N;eC6Jcef}pl(l8v0B|gsRSE2IKi0f-)}36zmcEs&KT(A(HwRn7 zgDoMyA%-=Yiz6z{=LF7h5w=hfSxF1+C&a3HB^xiTBE2AmoM;GJOvZqQr80vzntsdH z8)Eg{#^lXwAT{4gRiQe_R(@-8XNl; z(%n+>X~NKKUks0uQi!P$ML!5_`ST6jJ#pkVo@Bi zV(iI0Qu2o|ZQF}<^)F%5m1?Fn-pn}uX^P_?fn&+T%vY`flGUA+Z=vzzEH+$UsTB3Y z3NqEbwuN;ZQ4jW_!Mw6~EIWFe=sv%O`fY1*>lL_yBd~`G$jVKl{X4OWU&_WyRix)< zkt5`&XepUS1WQFGZ!|xw6ysZaS2Lw%Dqby*kUk1mxPHBVeJ5wV~3`iA3`sDOe@Wd0B<>*nw3$;|iQl$Zy7 zgd#|JW4f9@oVga|`BpcqZ!GNf@Ys=ZJtA>VL1 z7VTzVLoc7Vo+jk4AQX^B*9c|#98`pd0(E?QRI=U$i2iBE7p1Y}S_=6^8n{xgbT2r} z-qtH@Z}^UwW2^!n1+lLlQ?6AM9u?zvE&?k8|3^?DqBB2G+fzqdt!NLo(iV@ zsm>%G)L&=G#I*y*FTJhfFZifU&0lV*Pr@Z6T;o_f?OXOWUSogjHCDX6iNfqhDe^q3 zNE1&MzRS0(F0)^N-R5gdeY=`q*vvnU`bU!Qc!+IFjGIX#;&L`MOiDLphy3oXHEXzGJ!r+;sK{3dzB=JC-^{qI?2+K40xV zqUtBf25wm+^;7Nu`8djBPd&%^AFinSOX{bZhBmWy69Qx)suZH9&Rlffy0O**CBN8B5NPAKz z2c*slimKI&YC3kp1Rl>_>{0cqTnATBp0OgF5~nd>+b!`%Hx;uIC88{;jeNvnS*)eS zvRYHw-=bJfsFybsWo2#St@^XRxdWG2FsnRU`7Ht_MhVHj-XB6hV}D2|6p7-~Gy>r; z4K1w-c$W<1u1|NFy&t8tHr3X0P6F&o-7g@5F98u)hqVe|iq%bH~CN^L3XL5WyU zQA^S{5YSi%=SlV-`7t9ppv!l~R6jtG^%Uv*N9FNNwf9dszOsfk3UK<&{)61xqbE$o zu=F1SimYxboi&N{3FuVUH{Bv`*EO$I_eAAzYXWK~IC{dY|G&T7ujU-t-^q9uVuNH~ z&t9-xT@O)hH@0=E`dtCQYAh7t6lJ`T$B40yaHY=#-1i(fVXA~R*+bt+JR=J%P-Q#Tdk(tCN~rPtJN z7MtrYb{|yhyAY40i7$IN$txDu7$KAUg#<6X6u-u3?;z%l&7khLX)JjllQ$n9PL5H8 zEm4kLi{Ojg9^v_K*)#n7*?(KF{{P$8$s9Mz0WCZnYp`7Hh3~fpcrQJZX5CLyG{EC- ziApNwsbM*+P9H{xt}`VN=0j^1Q?ds$HEjsfhovz$Ba3-i8O%%{!j!DRyy0;0k~6@H z!FO^Z=MEOHvUy|heXMud*y(lfy5FZ9Zn7hV1oUEp z;%wy?;!u)U(97(!`?!{JFZ;4mSm1K$54@5%)3K} zvG1NcIN=<|+CU1gcxm-89-u{;`}G!f>>jG?q=4COOd z7Hc&JFZf*KdwfjIOyzvmV0PFtn5y}C$n7HL@ev8+k?+jq>C61OEc2Bn0XId+?h-)$tN!qB`Z=wR-znn{VB7%`Lol(3d2WjBETQYq?RdUti?f9 z>R@&`GuiCT;YGjH;ANyH%E*f3;)zOsT*$#vSC*)HNLLld60F3+s(c7UjAa?hE1%mhqSS(m0)V4;`9=$8uwM zH3P3vOr<@{8#X^5<`?rwFvu8(i$}67%o;L`ZE5K&)OAMZCJE^Qw4j|z&BbrCQdv}x zOP#J!n=y>m>><=g{d^p>Gim5Q(va~_)I{xkP#mMc>s9+(M?(6726`YtnWunqi^gwr zY)tvh9qb-@F9$5?bbD+p3;CH-kjv84``9()*L>?vXMG@thERa&XdX)v5mv-q?C_^? zJma7F(m#y4SPskLc2*{R)I|KO3TsTM7)fQcfS^C7utLE&aw6p_+Rqn`sYoEzXZ<@? z9sdj*{|ub|hXqdmqXXxE>*h*q=A`i@Dq;aJ0cOu%VJe5TT*A>poSqOnzWDZ54pv?BYP3ikZ`i<5?|Hu4ce}M!vs*Lw-tL(W4%kHHlXFhWnGD9b zZ39l&4F+s5#u$@x&PgDYQ2-=?awg|sa3GkR1xTN*-*bfBduyiWnx3kuJL4CpBse!*jU->jKRSsWKIQg~t`ZBO$R zcr~-|QzDaeL?v11mk@{X%~(nm)YW!2CTkX{LO&YIZCr(yJqU5*EhKaRrF zTc{J@xU-b8^M(_Z*qgYN7zQM0iAr`V3rCwJ8IQ6#brPG{hn$mhseiVU+Rh>-Z1|Mu zpo% z(?3~DWQvV`Ng7^H{XIqB&r_Mr9S`;}W#MNSY~qNJtunHs6OByB8AmPKsbMYE-8pPJ zv4$^7rZA^^8Wnd0P~M`>PsfvL7UeGM0ghjcc_`zrT>nU)^@Xng%8C^#v(9Q8gwc~p z--HMXudSo;$zB?NYF6N*+z(=RD*2<@P&89^zD0d^9<|SkS-E$G$`DrM6K@D@LM+86 zR?+cfCv8u*GrM34(eXxN6YTW%$6)!3o{}pYXr`39N98OlS^W8EA|;2M1yN(n64fN`N{Chr)na>pg-eDz)d&OOR5`bD0dl^9LG_1Ef&llT4UnC$6_aU7Qr!Q|XgG<4WE| zRXH6@*fgG)L>rNQ2eAn@`X#>1rj8jjKggx!b{Pw{E&PY>pRl@CY$69L#dAz@44>^9 zLDSP6G(Rq7X~{}OjuBP2fJ`!fD2pj~xT$Oti%L;L%l%{b5}VkMohN3~B*1LvP9|;m zkl2t(Wo>H`Ec8!&nXQMvrs+{Gjdyl4KWCu=t@I8*hLDY=WwU61Rz%&s9W1Y&fj%)B ztH*}TZN(CZBPQiPQuO1O?C)OBe#w{?;)W{_CV%~^=Wln<^S4H+9qe8ie>_NYS#R@U z@&(WYECk>2u>J5dYHt^F;94p3@)r=}bD$H2syCbg9OoW{!7X;BL3Msay(>29qTH2? z;{hpgY(Kq{#;%ytCaQp-&@l@pZ^6c79hy(2;#K6@Hm5N{xCVR z(Au*R+t-jKn$P_IFZb3sxw6E<#SKHKSUMiB)vnY57T0LB)_>eFLllhq_@K(b4H!po zXyXlKC4H*m9$V*E6Ecp%A2)(j{aDf>-=ks8CC=}C%#q5gd^vrk0-QBozgkE6O+%?# zbCi=MH#l8>i;bU`lM?kd>85uG>W7mUJDR4=mz1LRc=>HUpSTpAJAm1jgvFYQ*EpO# z8&A@jdxiR~-||lCGy>+~c=Q7a8ix@dJBqwnja(>v!o}TP<;n_a^C@ah%(IpK~Sm*t_A7vK#gphvV0cR2j&UF)X04+d5d~k%?m6X&Z>B zm8;gD;oE|4&Q?6))4wbvpdCr7d8`6xBa#p=E!Jd0 z=8;saJi_tPTb$T;hgDNch>sghvh8iW#-Ri?qttmhReq176*txS6UDM*pG!t{NHdDI zt(Q4devg*j7x9^gs{hwJ;*}+|-IU4ViFqnN_gwK~HqYG~&IY%o; z`|h!3aWPJ7y1Fl|l3VZes=p1I-=sO`Cr%eab-}la@FcD;Zrunr%&6wuJ>48DzQdMz zmDtT0YHT+N`Nfn%l5rHZYftlC;S+u=dB&E7d(?H)JL3_6SQOKu_!W@9rQ|yT<`KAT z0}+r<93>n*`~Pj}6D|~YB8p)lT>bj(cdG4go4-#zhq8_na$3+df)x{sI8*vSoudl{ zPpMmYi7gWkuwzC&CwAWBQXyO{?^257t`k@v07v7TxAd`6Cug&k2)Wl}&Go%CKHDp`6pj4>(zVmo?w)#%ayQ z)Bgw$$uNWCQWWPlGRPUY(DM?C6Az3SVJsrO`zl{!X1 zW+AwWji*u^JLB1~CGThA?SHF&63|ZS$GoBn1?q_lmeh};usm||oKhT%qITul4a%8H z0A#1As_kCd4jxD9DFOAi=a;G)S`n9BNhmU|SV9Y^=J0do+aG@3m#@B=i^c9yq-BfU zhubUe4`Hib0Z=7NR{&*kypr1Uj#vV#b0H)n8JC}fd(Xi>0r&(sm+P0HGLD7#eB}5q z^{>PlncZGBH(9-YOini!zQ6o){kd@IS~cEVyppvn-xuQex$n7l`vKow`H6Sl`v9xc zr2u2Q+l}4rqN=7w?cd{PT{N~GQ4V5qf2;bjXFpV|lOqr&XuEyN0ZfkR#F_J$tnM%w zS_tIg+~NoWt8y(xP!i4*mUycGn?Pj05GT)FQs+;ul%dju>GTQbXcnrvQ<96$SW{k|iwqzC8@ptrjbF?&>chPjV%Y&IT00qWnC+Jy6*u`}^y$|J~1zsz2TeHa_nhqq(;@lH%X zCcXADD_?nuCQURQW(_m#c9mnVljlSFGk%E`5tkl6L7RzXk-fPQ)thS(ed&nPvN=l2 zC;j6X7NsZE zvLA0n_GfH=6O;OTnEKjl%zN$sq{;DTju`$(fm_4)h`#KN?#qq7f1=D~XK+y5M#P;d zMSU>`O_|LYXI;g8%Y=aXk7} z^1MdgOBR8ZLy10Of1SYS?--W#?MFqgKXNJRRlber!!e_o&G8{7rzf(o z&!0Fs;LlXMbbJ->XQU~abp0@bzAQpM6HEI26St%PK#k4Bd+rocM8FlJ#Tus}$>qiy zNG4W#A`*dPOG2E|OcH704r2DWRIgcx_D1)=->8G%faBkQOyg>>iJ94WociPHOu^AiQT-gt^FUski?of2xkKF_(_PA*hFrDNY^ zrhmR%6$)0da#mK%QapvhV^M%f9&trwUAPDb^|KQg^U)-XlCL8SYL$JJh}I$v{18T) zhm!ItW$h(E^Zm{)O84z&)`CT>+q#`|mwME=UEQkiFDR{4psPDROEGmF-XxZ<-lU2g zv5UUMHq_+7GQani^DvasBPM)c3?z9$PxF~Z5`B5^a2sZ{q^;_S1 zkeTxrv1ZG5j-EcJJ}ZrfJNF(lddzqX0+P#fIFp|M03ZNKL_t)`9?oE%w{W=vrp4}B zwj(*qOIEL8&6aIky84p>(S?D2<=RcvsF32xAS`XksIvwX^ZeAgAJu0cJ?q)m?(olt zPrEsG>a5DZ>h6NVl6^{X9cH`>D4Z0|vg#>_#Y#+$L5}t4$%|-BZpCJH`BO1ig}wb8 zJPsFu@_b4Y>c9T#u``ogtsaKCs13AEzakGfB!|u$&E&Q3QGd6HHY%upQp%#*nS8Y6 zZ9d&KhOKAjafm{iZx&O3t_pvaD3wF#O~Q;0(LbpddAF8ufX&oB-9h7%5|-4?Wc<1} zn6h~^I}R_P<54LGAMWGe?Rv7`9io)&W_t?uPzEsp8w*Pp()hT9+9&I&zQ2V9`=>E_ zX*LsgjA7TeYiXsN+Q&IGKh0-J(LxN8$tX>qkdJW--c{Rdp@Q<;J6KdRjW=e6Fb;o- z_m`!ScV-Sv&$iIYPFgyPSifgY*gnra5R)qlgRl-|(dzMHNi?zQz-;Qfb7^{-L*RL)`YlIh{XK$<@uFZv_{6W5QU;ZZRy6jOaSmv71@GiqTLA8Z)O z>LXvU@9`SypB8hl^B}|jIz|=V(!~GuV~_wB$tsV^tkmP_XzZy(I+K-!Qe2{uwG9hs zCXd#xoz&mn%H~7gu()O#%j&*jN!3i&RW4`B!l^26-Q>?8)@NnPn(?&WFQNY7R+^s{ zv8eVdCgr}*r#s#y@B9)DcIDG{r;NsnwS)!^#O4`-!7>oNkWdl=3`iP4?&K>M`@zhKjp61i~sFKCIo-1x&3^w_Ja5U3ih}zoB9sX36BU z4@B!qCMG?Gvg_+;dRRo$)oNmccC?bu>&V98OhOZgWb&3#N-^K?w2&1Qt5i0)L2^dp zg~hCAN!c9QAD7VhcoTJZa#&J1gLjq;VoL726#lS=+NXKcbmh`SDRXmshy%YmoJ4^uru#pRA(!aTy(VPV$$@ zlTa4C(!lb1F(v4ky50>c*zUQ1R0`S{@YAbh%1p497asFxs<~YBeqx&1c(2)6$hk|XkU z&)*F7{7qasgE(2c_)^ihZFol7SycH2%j;&cpz;e=G*4$y%O}*ajq*Q@s}WZJ^Xm_1@Cok(1IEG1W#)9`Q?jo0@P7jH&X%>uaDL~$Me3ZLd=(ex~j z=0{bmDczu&rFM53)<6QWAp_%?Vrr8UzSziuega^Iow{1Vza3uwMx zMdh&uOo4bc{)^&SGGpWX8YZqDM@wfp4bKXxyt9ihizhN_@gT-+c!T8!r?LO(CY8b6 z_Mn526W{xX#t)@lb&E~0iH1txdqS zfkZpg(F;R7!OZ;fsWd#>!Lg@REZ8^`XUMHobz;vQZA!;1N@b-Glg7WOxBgpme|WvB zz=5CDWyxU+SGyOdJ*aHq%%UuHrsH#a$g=h#XVjlzMJ)H0Te-M#G>bn-C+KitwWp)E z4OWDM@VpiN{ui0U1UOM3ls{gnD#fb$`$Dl2eyjkn9dqiKGiECb-r32%1*f>O2Ttt1 zLv#5>h7bD~oq+FNX<{cUTi@A#-OS1AdrGw`kuF8kkFxmPVwQihoBB-`6)1PC;yN`& zCrL_tQ)Qy7#=kY4c-tG4tZL=_-iMqmxlh}+AK5sylCR#}#PadGDV=+SbNP?CSn!PR zN}jQFQVuR_nuMDFZb@@BPA z^vz+mF51te$@6i#vauPnu$eOOyN9T3TbF4tdGlNNz8Ef+J)?Ekca$zWM9z!`^1o>% zZ+0VJf4uqyx8JO`pR=HjX%m;@aSg;`P9o#Au1fZyG_< z_A8t(f$IG8%2ripO~UUO$<(oHI9>jb^JPysUG#vQuWOk!axod!v3&L3TB~Lq`vk%4-JBZ@7%^a`1r3ey|#eJ&uKIis7=49bL zY8GGMe92>umi@$nZ?>Uz$DC{pYs)+9Nl$`ji2u0%R%c{Hnx!R zc}F;1beA(__c&Jm6K@auOo3J6mLs;+Lflv}u~>n|C#vqSW}X1YSz%Wldpds$94q^7 zcf3+hnpE8=%G$8CuMmla=v!7*$HOj=x=~*8`#T>@R3M!IemYU4h@(eO9dWzyriY^@1n?94V5z5ME>fz|c&h7L)OKb2m#^QV{qQl$_U)&-rkQ=! zO)OltF}!Y9t{h>zpc{!O3XjRZWcdn!947F{wuJa8^MFJ;iOq9}6>Bz! z9k05=HFs4*6RS3DX8raZoci`sxE_}%r8^Hu&mN&H#0>&$%C+z(bM(wb^*8rBpYh{Q zHz=vt!#DHhv268fn%djB`{*%`o<37G@`kOsO1bLvr77}WYUW7xs2u2KD)%>1xxazx zng;c^on77hbn_1NP3=@2XyQO!JA12}S-NKH&+2jk(iNEQ2s?z;wRD8{7r1`sJ`JrM zR8-YaRoBA4nil3QS^E!df6x9VmaN{a$X-GkaQPEiw<$;UAEfsnwR2Tn6YI9^VEy); zoH%zed_A9bbMy9nQnQDvSOC2}gxwb=pZ|F6uKJr>_nu-GJLsOCh$1$)6Ii`5N4+mS zhVrTgMZC~E62f)7D9d})_Huv!ZT)`e*m+gIO9rw4?xKf z`Uvm;^)sb7Hp;!}&rpOxAxz7@2=U$S^y9F*Np=`mmik9nq=yx5uCE@>>M8GF@P

1 zf270Qm+iU%&Lq${s2L|-vh$)JCTQpxfZP;u9Oybn!kTrfHvEl^h zO~hsmFji|K*KDHEtmTj)nl{}4>P*p;Xa}&o|7&~}5yRUWCy53xNfsCL`oF^Em_P7+ zDXqx~Ww$frh><;v-r898Y)A3j(4o%kfA>NA7u)SY@T1|1(Iorx81{#I1AgOH2e@XfM!z!3k|O`UmwGA zjfs`o7*^|gvsycVRhn2l$5M@+l74^WRNOza#~;h5Avb=j3xh9}D1V6P zpa-2tNMAAr2C;aZxNSBD>x^uU?88NEUp9+)$d*LkP&zulm(TjW%C`f0abBxouiiqj z!$pzTPNO}V!#Fx!M0E8VzP98RHom?Z)wk z{uo~AkI(8*?@z3p zza72Bj@cJd4SRztLuCa9OmC3CsGj5Hw>VaIo3rIlRc6@tIdCfsEPpGYvlOgSD4Cr> zf=QWNmYm=ou=!C>)8PKIZdJ$~JAF7II3q=oaNpJy@!`Tpu{l?QacvzyxwpYT{34n5x?$01;@WLE91YETSq zixiB$R22B^OGYxV>KZ##u`JC6)nw>7h6m5W#r(t0Zf4C{q?#3aDgNwY+a(2`Sf|Dl zNXTSQbpv-EK2zJ1Z3}C1pGjRJimQgQG%CX2@$+mh$;dSaB+D^?(c#Aw zbfZbsvbB9JZQXmReV9uth15UIp_P1^pY7sk_g;?NILORJ%ftDXK0iil0EZ{6*vq@Q znyRjCR6pEC_5JNMbQaL|B%ihiJ302GoJ032>A2C%>Y`1`K1E%3QM|eX#09;?raRbo zbvI4jxk@?NN-i}|w%mH zyQpC!jjU$>v!%4Km1YXazqppx2c?{Rcz|{J0{moQwhvX7+j?g>>(%57VG0-+F+GjC z&Rw*#gSJN-IM}&~mirqy)U}I_&fTo8SgA7fq6N?wcBIQnR7l6>)-|+I&i;oRY3SZS z1M8`Jwu%F6qLu<`9=0&AFduC&Y*DP3_nzqvZ zXfLf@MM^1G`(!h1>{3ebjtAwOxP6$<=FCASR+o-}nC(MUR<(`H6ij-ZzJ7C1YuNT1Z>x9v1BQ#}t!_ooY+P?i%u_0+cJi>&SF|0m$9Z|Gm;H!@(v`% z?NgcBnp7M6FPAChq=2k#-MeV+l6z0?;azMyw}iT9`80Qzu(Wtd&m#KQSyey32gH`# z6bhoTyVY2vb-FR9de~gQj)PAsXnV4YgIzhab#I}za~;iWrAdJQdq-H2zenXRi&8>k za^MI`PN!K}2=8dw#KF!=>K^W-u`7oIPqxxR5zWtvXnR=Bkvm8Eblw8A0$O>}ROCXe zEl5mK6y@Kq;mD)ibX=)KtE_zoV{#5wjbS=e~meV2E_xfIQ^`i^ynlQs5Dq6M4|4MFRa+)tZQLP_)2;7KRelT5V7g*HLY%9?pJJ=;zD zohnvsUyR*j$7=N;N^hkCmpsCrdH?y(f6M$I29U$Rg*3!%0{n^Py<1dMi3DvLMi~DS z2S3x$JwK9L%i?KS_CBNhCj1VsvZfWKfypE1;KiGu;(Q_v8=Y8q`jQF8y}|MI_xK?n z&K5l8c>Wy)o{0kUOyL8*FNJf3ot&t+!}`T#m`&m4gJiGj#EHe4g5U85+g8f*;@xSR)9 za^d2x$6Va~1R<7Q+yzJW-eCFKo#?FzxZOh)Kn+2hpMkK^|PhQWl7epHkku#X#_1Jsoi-_yKjKj7Ppn&9g3FeP+mngj?L*@i(nPYdzdl&> z6Q@favp@f=QZ<-_3~5bOnat(u4|BHUkvhkhcA)O13;9nGYw|PY_o?4{fzxI8I8}Ow z^@~cC(o$*}hb^6W%NuI{4^~~_`Tn~s*_fKBH{PJ1VbGvFiw%XH~Oh(tgesKHy^YV;0WJ zK_VCQ@yTdh5*d_?$3Bdl#dVx5>*R+bxV8(fY=!GP;A#O}D0#@)s#~m^Ux3$@9nMI1 zr(m=s;xi9p$>cl^?YTxrgRDfl2hzSQO+KLge)o#v>>WashxhQpDHOR z*-huuE=4@vRaB-xFugsXz(a9lGFUxy96haoVF7%l=93yuzT^4t4?3lOtW|)gjF(bF ziegvlU#UmL=2yV)PbN?2?xV13^ygZzrzVu|EURh?uixBBJpg)uOP6mdh3@V9U21HW z@lnMYxy0EctUCT%;8+ynvUU)wZl_Pwrd|%8IH$mGRj>Z5`c?iIm=@AChhJvPkM4IMud2IRa+Pr%_9Qn{B2Q@`ZylV_?1e}4XCAH;H5>hkjHrk?G` zztHyOTXKdq(;oO0g{#>y8lL zUB0cz1$Q5!{;k%2qB>UhNdha@Z&yTaDdG1WXjQ;*I0`CBWf#lwi@mj|>W7RP!_5a_ z09*E_prjJFKSkv&i^G{xX-iJHvRW5=@u0f@rM|a&;u$vTZ7yBD{$jH$@TX)Mft8F|O?s<3;iL?4t=44D@5Tc9}OKY>! zuQIHiP8Z46Xv#iz(z(FS&E;7%FQ34hS((bJRkCS=qT;v9nkRgJ|0C;w-#+`|Yp8#Z z=T62y+3#3?fQST-QhkbZ%3tDiEa}yYJvI~hUK?*aJ<3tzT}=cfx<66m*R#-VW!cOB zLR<8oXm$;tFhRrG6a(urtTdYYa7WjNZ7vHL4jZdtUglcVzfj~0;jj!v>mH89E!nzR z*qzDD=<{bTNB%!tkNS_C*Y~Ew7)ztgPMg8T8J(5mnrM#cU*nwicYLS)kDN8VPNO!4 zW|NUtvypb4p5rd*z`KzYW~cX=-+cS`VX9s8^GSkIQ|+DO_oK=BJHdEp#L*JM)&5N#zeb8 z%YI8Fb=GK_j4`yBVremJ>CkCt(VN&9O6G%5G86jtp*X59b!I(PE+@02;}}06l5%qt zKg9f=^M+nDTVgn9jpML!04>%&)H?gnYK-Qj&c=yY2M6^Y3T-ZCM)jt_*o&*W|H8(g z1y`b|g46L>WbNDo0(yj8mVwD3aV@DBJ;_+TK}JVwSld668fz^3+)+%jS`|^xVjHTW zS7gm=lisrecs%9=2E_*05^Lj9R0Kch`qQp|jb>|Kn(X~KV(!B^ z-RqpuMQ~iN;eNywnC$4J$g?-*QCWL}`tO?4oNvx{%Ubk8U+HFHH z3ApJF;kNo2q;;`5vOkx!z1iS$;B+Sw5lq1BweaPDmy{CufYHRFKspnANqpk*vs|yE zd_Z67?KXCKLkub{1nrhW_>iz>B z-&grQ^7kBY>iEEHXPnl|!~xNaAJC79{a+>brQdUH>C;_?5i@$Wa__&4DAH{kf+ z*!&TOtXfF_FOqTn;)70jpp!fX$uJfxFPk-km2-+XbKn6dtL}2J;wnd~Z*p}1Ee=&& z=XCW04p&{Lq2hvS${en^Mt0%{D*sV{JjtqZ*s}<_N3m|vZVpv`PkZrYPL@65^q%`1 zDZfHh?s2AmvO;C3i9MXnmxk4ufz2h&jdX0rR9xCrR?pu>bJYb}_x#AQJ$E=)-pQH5 zhtzDnz@!hCs|+=RC!o|mVs&d)meXQ4mVo3`Ddw;<$VU^Wa<8*ny`%u^IDQVT#f!t2 zhRzaF3|C?93ENqUC943D_8{-RH-W<^FZ^7zN|AW|{$rXC9Y~hYPrS)^r zSnkPRUSV>>FaMT%{TJX^Kz=FsQi-)D;S6T0b02GQt30>?0X+-L7E^!y0IhdwXuE%a zgPrxX-m9VFObHWbPseDJMOKE&J+r$rRgQOLVk~)AH*nx-36(c?Gj_$>RQ+{ z!)C@U`k14)j&t_rqH|^=HlT8@^fm{Y1T$~VAH%MrIUKxI!?Amf z9J$@hi3dmMxPFAyg*%nnOr9BGV4EcODHJ5eZ(!c8c^tgm$f0|+9DZD@l(%h^on|iMwrdTx<^KZ`ucAaSc-0cIKd&X{#sEeyfSL8*QvD$WhI6x&8*X52G&;jVGS$ zaihs^$>;FhCJsHQq2=}-YOn5QQ~d^JY?;fk2PZjrtAph`!&Z4OsyNrVQ;3yjq*(0= z!`T_Y;PDaV(X+I8DfO4DX}MX&p~v;KKCGqVSqmi>%9y%l5e8xVx&~sFtWu|thuml# zZl!{a@>rO&V=--aTdBXZkAu(ZX?a*f`~5~rkMHKAS)Z%C8iTT`lJl2_)jyE9pq0w= zrJTCc#-VR}&YxolX3t>6Zq(p6@cGJ#blhyB<3>B{3Ub2FpIFtu!1K4JARoQt4|}pz z{`}|(!#HxciX*pcIC`U%W4Bv4a<_rDyLGfbYUc3$CXU@{K_kcJ&QRHW`gkK#)_qF*&2|o6J;A!XVhpw{oSq@d z#X*3;*np3SL?bivzozjmIq|5Twwsl7+^M7ZNFncjHW`a&1bWv%G;V2F2l09n zF`C>M9A5MRJM(ugp!Irq-#a=RXnI&f8H$y^6x<2wxi`B5VYdlSII2QoDrEe#s`1an zyj=@8c)OXy_v$(PSk8Y9ZI5f!`1j40rRx5Pbp_G-{TRir(vgBG7>|EsGMgH-kwz0f+4Ii!iki$1xxp4Con{#v3^CxPt7kU1c?bxIk?=F9qYC0>{ zyDLNGI%|R+-kdRt1J?^_epo^C{R0#pE?3t0Vk_wqm9{+|}bt@GpV-?$6v9k0fDRt3vz&NaI`(?iW`}2R8K&E#mpcTg%kK|j1vkoP1 zHrWu#IPL$Qsi(%g5t(y`l9 zS+XF9*4^K8vhY5aD!Ubt@@T~kYIdDs{Fs?xuw0@#Y%+JJ<8Wr;at>4=>a@SEpmbY1 zN2-3}Q2A9(Ro&%O^&JlHxyqq^m&sdK&)Y+%;<9Anl<~oqg3~>aF(bd?;GUZt-gleh zHFxRQbA{to*Oj{SWZ6y59=OAi%FCRpyu;DbTXfX@$g&MN=#)|~LxFgrLf4CvkS_t3 zbueFySwq>D4jT8Jq4mIbG*|sd`+>`}*L=r@mBlZ#{gYL<)%K6=zsdULWtc4jUZi4k zOO|*Rl4-nP?lziBzT;F;SiO33&s`3cU!yYjIO9K@g-)Vnd~)9;E8yE9Yd~WbmHEeM zsk=mF^%0y7QBj9&?iFzC$l&YG)^lk86%JNkX4R5Hm7gn73pPs{Q$JqDp{gsi)qKaY zr8{stv$2}euuDVI6;fHy0ow>_icWK|@-kJ$M{qjR)w(KLY3*BiM5<<510I+UhQ`bn#B*E+CoR z@=^{(J|UQiqOoTTv5G~1&px|*Toe=$HNZ!oeu03+LO7P%sr}eFG-eO3V44DHC7MTS z9w8q{POcokkXSSZ$>mMLWcM-Z?QuWXPd2xQ9eMd-;&B%gmF>eSK(*_6Zme8~WYwt5 zE7`-{$6Y^z#sb{QK74oe4%>H?he^INcgk3%l*Wk|q}eYjF_~K>+twLm*8HWMy>yvd zJ++{KmGTlGw5+NHi5XJBpIvf@6S;EpfdWSFcM=Aodw?}rD}{mMU$TVNm@VsPi3^f3 zz?Fa}Bnqqm#TC`ueE3A|v+VbC^7FaxuJYyA3lM^}UB+#xJ$$LeSv(577Xt9?h0FQ= z+O3~~)93sA@L3p4ZD>2f=&>J#$==E?It9HYz+WeSjqJC8t5R>uIg?slUh=!Vo&&&A zi^}&_*0o`B2~aAefs}Bbv--|N^fnJO<}T#I4_CQy@3H#(=ldeHxB%~R%mTK)$oA#D z>=WQx)xlvUuQ;HH)pl{ker`SNQgy%dALMUU|KP_P%$&UhquHnW7y=xNW7zZF){j3u zQ2Tl3A($Lur!MzbA{tAW7`Af54s{+xy<57sLCt?+w=UN~=0#acOJ70m1sThq*Y9GF zE}yA9&`e6kP*uN6&q6+yloXI&8BK+tvA1P9U+_R!E z)>)-rlcsW-RqZeC19mTtKmrp#|B~;o-c;^w66+y`W5Q_{>SRr{}F45 z-@fDD-!%}YnYf^jD8CS25_v1u$l_OCXP@0dvD?arW(xt6lZky^rY7<~v)5y0vCG4< zmwrcc)E_AI8Tiy;B;626wqXD({_s2Q_KRY-JD$OIFDoNo;cC?HDe?tyTL%*B8H6Yy zrN`v8`I+1MC9cK%K>?2&-3I2mEX;Dd`NrmBi6y`agGH%}59?p&n&u@M?Iz~gg3NLy zG0zpxY^R?&76+U4CQ2eCzc-eB&RC|poH(Vgpfb!8(0W2>eetSZlV}o;*-yO5$WWV^ zF=hkvye_tT9hBRmX|wm`thp~Ibp6Q*I(f?w%ktiT;7ZiXoQUm5xx>LCi-j-TPCoaz znCf!yncKlMuZy{67mKwXCV8_N5zOY(z601h;8j}9QS5Q)`AiqVo7yOf^%30c|3^9u zkt}tX`O;?P3!8)KZW~`X4a~IISzw4~u_l3~nh;;otlKKUvD8q>1g!Clj&YD5qv89g-ZWZtEOFYI?{%=yWn_s{&kDPqRTe9& z&2Cm%9IUXJS>mv=+-7D^^edc;`Oj23HGJ+15HttSdxi8Xj(t7_R(q@>fJnjU72r6) z@K_f+`wifSnAh0i^&qYsQLZHX78kR7y~MYXy(qQn8RPcgwL9@TT)ZD0!-m&iquOa@ zSHR0qvw+em%EgLHjas zn=^1YG88c@$zWrg$;`(=H{$~)R`>ZGXCnT@9;cSc@g9O!I{~wcB@unN68j3(4lD0z ztSm5^Dbq(#Vvc5~C58q~e}2?Pa>N)DR=ev>R7IJ{F?*PsltzZttJYKzUN2EWx0-W& z=5$=v3{`InvD)B?54(N+L;Z{Yd1Kyh!0~Uu@o&KKzwzOHzPJ{f!ruLL^y;VkAJRDb z_qymaxZ_pMsV3k@6Y!}ZdSPx`U8%~tNEY{!0qS-Q!eU5M6O+lGh*6lilI0-wt^!th zvWc@Ns+Tl_4b~8ep~3uh>>OrJ*uuoIi^xcN7n>~;lQWYzQ6=~XqcbOCHYTZNfix>! zo>bf_^E7}rA%n5+PvwiL%bEPq04RCfHJ2H47Aq?cXCMuOvR6vS>Pu6Zds=%yF$3juRske!Vasb_+|FCRmTBM2 zS4D|5JC(hqEgqLA4ZSUd)+tslDL4XI3XHZm5^;Lvdqf2kudJ0;Z`r}3)f*Y}@uYC` z)Rm}|K~iiRgdMK-H4&3Nq$WsF;)-I|?oYtz@{u-V1WVU#Vdnhh;X>7!jNKtDxnwkw zQ>8MZ(o|-L?7y(DJpr+R3Nr5F$t+&8k-5v(kU4y`V*DzLXvu{VfYF(t)Z5ZL&{=}Y zdQX6Fr!-`(E}W8|;qhX2Ix*SoR8=29Kx09Va|8$Fn~p+HJ?^OUDryi2$xU}4<5oN zUru7y$~k;Ib2dqDP9!>zL0o(yabnFb$16$|X;_JJ%ag8-MY3*1xn#GdsGOO|pjg_( zqYoya^~Mupk4Nvx!ss1}-Z>DXBLllL1-CN+r>M_OUYwx}G)_^126^kfiAtS4ec?ie zzdshE->WjhoITmda$WxQb^6iS3y*U+Mtc@|zgVXx zqP6?bds8sS4^|ds(rlAvg)D3Z7!w;>n{SZHyfyh#h;e&}@w<3y>O>|lUBLJ`b4hvg zT?P2)Ts~|80ZJtUR_^T}4pB+j2Vyg4V=|@UaHnHaW04<=H=bCJo5++1=2cCgzH=k3 z&$h8R{~LARzG%j?WTXhQ@z-8|P0#h0Mx{J=V)?4Eq@nXA5f^Zw^V+a_y%@!o*E<@6 zJym7J8JrdjK^NMf4{b0Wi!T9(BZyV9?>s>@zFQna6qqQAy!26T^4X%V`0MOx#EG)f zn~v2pTmh{%pD0Db_O^fT8vobd3qS9AS2h~ga4epISnWYvUKu+Qu*atnX_uVt5FhiA-Al zHRI=hM)F^VDCL(ls}*5ES$>IfC>yOgP1%NP{GwjBVedJATGt!sBtOmV#^APM47t&V z{OAJlYVLTE=TF!3{E5Z5TQXSlDJqh1t>Hh(%=e~37%Cp;Egg+M23ig*%-4d%lc z6ZvZO9L9e$m*B{ciAxwrOi=9064m}XUCHYCQ`wU*^88toFxaxOdj?^!B`QLlFvLx+ zYz*E^G=UKAVd+d?w}kO?XA&#>+M}{JsWnilI!DSD(ka=G9;Mi^${b`Jr0lHCUJpwP zX3_RAkJd*!>3C91>Gh2)KQxy)I~S_saEMcc(??8F44c}&Vau^;)I3;Cb=P((*-P!M z!z2uU8-pW5vEogxcuYQz0?XAn{;%fw|Ni_R27SaTFwz$wPV$;X@u)2MGZ}0cz#Pv% zadT+~H2sOp>*BHWqKXax4+o51rY8MbzUS z7Xeo~ACLHwFF#t!^odK!PJ9=)DGQq|yuOw7qS+d+<|>^<2oPDSF|M^G;R+67@URb< z{OMf2`f?f5zgWoo<3GorB5@W$V%;H3p=9D*9<}z5@q5rYH+jqFrl)~E4lU-}_B%yKpiF5eEV{$NAT^EmYFdq*4hOa(e$7dfeAv1Zb z0yAa3FY7ps)2Gx7HUV~RiMU+@!kMf) zT$zaKS1ELaz$Uq=qM%d?@)7E9?9NQ=Leeo04DY)u1&2Kux2Rhj321E|%z+>VpQv9` z2n9wk>4QalGkFV>-(N;n@`re2yfvp|u@1y)5f>40PLO*cRn;_Zw*bXiN(C=#%@ekWC&3fs`a5#5Q)1$B9aX$>nr@NO3?-v5ui_Y9Y;EVG9HeZSu8`!h4| z7?9YxPd=4*)y{eEil@>^-7U0;11KVjisFozMMp#>i3lhlQF4^12nvE=02Bi)83Y|2 zb-r=dcdh$5MF*j2x*esVe(bBxskryEpLE~rUTaxOa(KWTwL*frRZ(SPTuGpu{go2; zM%b18J7Z8XZcA1qdoVIi*#?~dHZzVD(~j{M<6y48d5e3guIxjLCZ)dFhE;eeCA?F- z^~j@e@|kDhGbf#jnfVR!Jo7Ky_tTr0^9zAv7Jv)b?bqNYEtH)YhaT}>eD0JpBz`~n z^m8PJ?<{PD9kE!rx3vFM%yAYjv-p|7Iops7a@~!k_*r)Xm>%yzBG*QMo%R|6lKS*5 z&Nl+<+=oT`9Pg!f$CmUjGacY^X%BD>~>TcwTNl=of{GH93e z_IDnQ2Y&YGR)GC)|L|Wp=3}3d{!C`Nn^2*wBL&}f++WJ!Q2i~D{_>Me$|`HX82`^d z0U!S8abhdN&$G~u`(YMpbC1Vj1BXR!ELe$W&@)DHwEq9%iBHG)uM?{H`KgED18-hH zke| zS5en%Sge>>UzkAMFx33*_mG-399SviGtMM#(3j)B!d|#MsoY%TcWgg2XZ4|i&O{v%uVXhM3;0X_%QVYJ-!Mj(F;hSsr!Vk<5T;Y%72zvr;w}i8@ z2@$~eb__kfd zyDcA4*+(U6qr_sZsEWwYF_)jhQPV5M8+txlg6{P3w#f>v99fNft!dm8X7N7Tl>Gzc zqV*tw9#Gbs>Qi;(DR1nx5Sd(uy2$x0)a?Xs&lGXh^avi#EW_8#3TDg%H6un>v(VBF zY0Hl0Vj&-*U-B?hG0-q7=#?@!Ze$eLou=dunExig zPvH2#jEk?2jN{4aHTZfQAfyU}*~FXl7QV38GW=?4CB6|9ad^!^7+7$97l&l0aO#@n zxXvr%f|`vDh9Pe)c6vwh@1;0IzG_P@LCt%+-#Rh?g+O}01dfNm@v9d&-Xdt)k&VkB(KrN- z2b;mBcwZI67)1gpNlg+j;{Qbd!mutfZe-$lOg;xQ$oM`gX-%Z1DlU^%eH85`%#;96 z3s#+yU{zRdg3w(+WW5=-IWH5g7LbWhm&#>I?2@UfSDCz3Y)#W{AZ7xWpr%uoiZIO~ z5ZnF!@P8~iak|pq6lF;&%(J=k7;eOb!K7;{Ne8pigiEPPvg}#$?o;+X zm82a#IUOEdZef{pzL== zQnzIHkTW?C&BJt>@R&gHJ1A9BNu*KgkbbpHN-%j@iDG1I3zfKzyiNbCIRQm=kzu#K#`s*4Hx z&4iby)E?;5CZ0_39C8&VvKCQvI>=ZFDoKojSB4(?GAA2k&#>@@#Xn2!=S6c$@gO1~ zO`o(x+OOyyAj$>~C86kpOJ8cv`&>U{9$jLbyrl6rC*x12WUpU`7j)q{eU$A8<&Yjb z9az=|*tC-H$rkRyjQX&>8AL3UFlONS8&L5nO}_v)T7+#eaaDDvB&N zLFG1doAVRrKLy%*^vtwsGVeNJ7rxzxW42T~J1I(B`^iLz6BOv99Up+PN(-f!a>{*} z#*8{Qj=$#=;&Z~U001BWNklw3^AB~AvY&-pKf z$z(7YUc!*B${NP?mSc$E6)Wy3A5UhP=@OLcLMN-DR>BqcosWBee>3j+kL&TB-(QLE z{MQXQ>*n)h&tp=)E9oFx%ixY@FUCE8xEMeD)77~B4>#h*-`J~cNq03rzLfP}fh7dPxJRX=obGp!clvTIZ!z}9wfnyYe>^$GqbDuk;!hnZ#U%x& zFMB^NXd+Jq0fGxVI9I>W``GS!bw<9cT09WqaeOs!$K}6*sr68zDh!J#+^PV^5v>vI zrtGyfdb>t_t`!0VC3P5$CdOj~_*ib}wHO3;QymnyE|m>`>hicFM$=kHl&7h$jn&0v9MTY$O}!8gV~-i%27 z>LD4Y3M^)S%f}&fp1(5-oA)kOKqn;_nY3e^Lu(H8`>mJw`#n)LC|74tsMbVgJqYLE zy2|S@3g+PHv*_q=MR)oLGz)J zpiE_}s1DPMVELgGqP^{iBkE!n$M(=0uF+6nIBPCWrmX zO;DkXHs?96$t)UUky11(Ol!s++`Dm%@{oOr6i>GWkYwSRU*$Cs7$&oJ33T+Sx-cW> zz#>5Uf+A6$_Dk58A?f=G;Kg(cpSQRf?#F z3PD=hA(%DzjJq86yUla3LjcsRiSIKD2zed^i5UxX&61*T7FUNI#gvY^(hv2h4KrcU zAlI~(tZ`*w&GI{{au7HdFg}_=-bf{}&9#tfV*}MPDxk@jqPL}$%DJ-LI1dlnPXJvO zEpzX;i@(qPs_fZWe9mIJs3*l8LCV-n#p;wG=X$A%BgjT?!?4?;vcQ5#?nkN4MDUjd zBrJY#sNf?^bjHvus#2_E&M}Pt=|fCrJm(8xn+tf~mA}azm-&dfh9wp{a~bEHe>wj1 zzZ6TG%wxvy7QqrL5fGpEm2up^AOA&R2mcrQIDp5V{59V9ug4&)HsQuC)viHXau-FD z-pWh41KBsGIQ9!CfVN5abD*i@cYlDc>2UyeDe$MMC>{;j?Azr_Gp^O(z0E5G-esTZ`fAb^Z|Nt(;Juh z33mY-r(7JrW&+=Fi?~}~iHpYf!G+}_zPWl&JUYH7t_Xb`5x4R2mCJC?TKsNg0@nu#-c-?X;^@BEM1VPsq%d_l>7po%(hvt^b(}voj$ciW z;KDRUT5Vu5BKtW(Chno&&LK2Y9FWiA^1=unFRaGtUVySci}82?`Lu_Lw2y4uMNhMF z^xCz!+RWfKKZ_5$HcY>Ve6$_~cUG);s74gnKISbK$7(rTlpn$IhKBW4DDfO+pPN>Q zBQhhnZ1gpFeEK!GB{cD|w2Na`ti|2g3EZns;5s*h_hcr}He=Yb->sph88|REjpL@r zab9)`@AV@@wKk5N%HXEe`{7<|3YP{M9PS!ucvT!foxzj25!_qJ;AGFlV#SbxAWCYx zW`KVw*!b8;7N=@O99NmbJJSfYl8Mu`0-nlHx_c5A>SY`{l|`z%u+!wP2u_%s#_`z4DaPv8sL zwK$G#oLP-)*6f9c##Z3lmW5-ZrU1tzeU(IMiFs$G4pHQ~753p-4QwdbICp#uPfo4C zH!EX!*Tga$t?T%r-@v)Ae;u9}-4hRLBe)>98pn<7jZd!H18462kGN@{e-^3l%VH02 z(k-}dicgHK#ACUAaIIU$K|uqNRfp%cQS?bXS->F~9bX$=h3^+fagR2JFBHeIu8_eY zlR12HO$L`03b@5B;kafEt^n**d*nX4j!s#}XIC%7Lz64;_}Fqhs26dZmPg%lR52*k zL8&OleGTLHppMD7hho^ndnU(mdv+PF^mTkFs^P$F5pSQG!q;V2 z$lwR16}YgH!+R<|-esqFuN&dWz{Qbv5hv%zaLLNOajRd(`Skz?mMzJT=aMe6Lf&c% z+t{-h3n+yyN@tg8wYHnCA4A}H2pqq9f#W5)DF!n;GJuD`@z(9cifhZBQ4&qE0aA%5 zvM9{@42llfL2cN2SEN#ysMO*V1wRrX(uoKj(8NAf%-x_a%oF;3u)^1-{IpJ4qd8QP z1R6bPLwW|(;S(^38_3Rtn23FplD1U#hejK*Ju9AFx1VEz7$k$B?6gtzm}qA3sj9%~_r}COf0NbsWL}xVE=fQTT3C9*iY73} z^L6l0GkS$5CL#Yf8 zTgaCEB_FK8b#_Ibm!DzfE}42Io9EqC4A8iP`+oW`9)99kJo4nPvFY*0@z}4Q#-IND zXZ(Nvt^AGuoHN$Y_u+Tf9i3y$y2z_Cgk6KFOfCcr#+u?N$w!cX=W;EO?Z7<3T3 zbwq9ruARb->%#1*5InKwQ4SXch~fs29wnuj5nT1?7jX9m*)d83w2jVPIgW!K=jO_9nh*_4W&liwj@C!a*fB?)p78I_cYnTKgq z;kZi5R8`1RlD#dJgF7ZW zuXF49cH7rWQEeyheyZ`W z!-{JHconT49B&<5vyYGoVK(9aQ zQv|VtAdKMHb$H>d6fAK3a?T=4hV94XEQ@u*s44Adb6f;(5`ftXI0w|AyYte1e2rD+ z9Df89U&{Eit%i(00*jPDFNBmj;~LpTHYCur9#(KL%I1QsLo()KN&>pVtYFaz=Yx{h zLxG80zXhKx5|5w`$7vm&-xoP8%CJ*G!sePyAIC_64X#7LlG0XG%Shq4WyE0(0f7dq zFNt>F?IU!0h+|f$x1kYiOB331geZ z8j77_+63r#ZrlvUk;!?RGNf!zT21gCGLaf^-cYI(1meVlO4Lr~Pzq*IB6!7h-|L`k zbzu>3bSUdh#&Uvenj*^rHv(_oB6PC`i*l-DEfXwr8v-V~0rw0oP!iTN<`9KU$h&aT zhU|y9FQW1Y_f4Gt?Q}uZ5(rSmEZTBd47L;F&z=FtA7!G=xGCVZ07onUAp?quXs02v z`veUApeG6`hEt~nZ&P~_g*89JK3%-2?4*KZ7p~ON1o(GE_ zoVpYeFv-DULv=4-+WJ3W*;yWohn9-nf-VY@Tu5=S`A@j-eJk+0lOt^U@>}rnLpQ(( z)1oM$XY6%aD9(aaw#9>)(|`9HYx%7;a%M~iDHRTGghXM%?X zBD8f8rY-?KuL`F&D|=RgUWvB>C7newG4yI1TGT9sXB_cfxP>cg%?B=OX#Ns_F=&%5Xsy&T1vQ8lVL}D0!M&t(Vq=jkA zfnKXh;)>@~Ccqa{eqDJn7vdPxu_ItA=SNC9YxE-*327E9DF$*OuI|+DXH50`^b5{0 zZX|2hXiM=7z2G8k9@m|D5AHhsN&NWaKjEI!euwX${~P?`^8djD=l%irU-lb(|GG!e ztR5(FhnrG4gsg9R3aUK`=S4T3RUo%BBk@epWFalppc=^Qp#WB@ctjPvh!TX`LD_3a z(oy6NncSCxSkCzb^qK5aebHi}Cg%)-r-6#W`1_12`1=DTE4Hy>ZW5$#s8}iP%X5t9 z#B#{ye5@3!f~1QdJ+KLnJo#HZ^5nDF^w^Vl?AO1>AOHM6_|u>NM*y|o{>LBCnLhw5 zVk{DD=I?U<>aj?p1218b3z?$?K9z(pSk51(0IitG zESezDNQ)vKdeoErK%VvoEXZP;E!mH7>?`0~t#umDfy`KbH(9O(4p~^mHeM=#)X4Ug zzMud_76(zgQ5!b%c*9hrxcaq^8Eb zG7DmvqhLJ9eN{Lsif6X4ua!(W*L?wvg*3!ggfWRb#t@E4EZ z)GvP(rW+t+%xVYHmxWnx)}(;TP3EQF)8m@snu;5r_}u43mizvn{Sv30aSm+uf8GPD zih$ClK^s{;mRPvLf_qA>`z#>nqY!YP+QLEQJ#ok3qj>hD9)5oM;rRUf-U7{~VpU6G z@dD><0unozHyrFYhmYHjXJ2*g6R_VV(EJ`;cRe;vd-(k5816R5@KE73xPR((ct9J+ zrnRrfGt+zFoYco$y@PkJU4tJ^F2~ObYw+FNYTP=#8n;cY#(k^bfG4K+!zED}2bnpX zo?nSy6qe(w)llXj0>_L&`z04gjTLbHbQZtJPvV?7MwK=^X;bnErI^LPi)h+499Wpb zCB@}*I|j2gi)A#P#+#zUzVPVF6FSlW+1eyn60=tk^uC|*W&5%*Wx_}5wk z`D%&F@NrmI!t*5F(@B4cEa$;{hF}VznPp-mK(<||GW;uS7*$kig>5*z;_&E({8msp*nMjsW6Klq`SG`m<18OF*DBVQ4CILt3mlV$+ zPoOd-klKZAk<~*+R2_DW306u`khR6Cc!DoXLQ%502_sQS3Nl@Vz3a>gGq>nbB9i`6 zl)`F3x0+HZo{2)_$N-`@uir)-&7oYTT-g$%mw{{4Kqak%iHr1vB)T3;L8LJ z13n&uz@L@l3^*zK#>#OX48mAZtl9K0A}C9?u&T;svcRP0BEfWdWJNKP2@E!u7^eJA zGIE)u)N3NOB&=yx&@M@Tu_Bni%^;Hjr?3zbCL9uJUw$9Im+WTwekLUtfKzIWV6Onk zD*3Ad+NA)P32RDGS!99tpjoUk>pmC!^V)gtDzTt^s2D)9Sb_;+dT}vGWN@smvn>_V zyf!%}dNd2*Mn-Z?n85=-r$7FW@>rrr6xqVR|08h9m(EtzzHAdYe|0^Bgv;jdCBvJ+ zzw*Fh)qC9N<0Hp@P6AUEZ16j_WEgMz_vU~_z~e2)Jw2LP)$aPz7nwXUeO9P-Kn6P{ zNKGS!%SGN(jCv-y$)+t6_+UbiEHWlr8#-?K*%i3=sqf$i58neVvW0PFOMon2d?wcFQmjBpO$myg4~I-NOUb!Oa%%|$Vv>)(oH7Q z#N?3Og+b4lCB+0L^2ktS@dzvVCD}$cNywyN4Mkzuavq~1g6a=syE&ZqMW*ki-}lY^ zL!CRBvNlyJDBB&-f`F_M`hSw~PT5HU0964>q;Q6S9f5I4B9mpp&oH65G*|O}vbcuK zuwYhv>pe2S>_}Lw?2;@n8SXnV z{v^RnnJF^*l(#fl%q22=;tq-~$K-lcC`Bp-HC!v3(l+!DWMZ{o5a^hdq9@I+!Y2bm z#;o$FV^J6roLmoRO+ecM-rJI}Qt69I#fl}_-XIYvbC+x*i=Iu2Wkoa&ejf`V=-tR9qbO*IEGfqdy+#>avq*{i12ta=04?*MVzd>)8gfjY^EtZ% zkBJyzVktjK&MzWS&OHOa$5PVOWQ=m|Vv?Fnb~4mB=Ex!|=)8VbO%Fc9b-W{m3p+jj zSar_gEAG1@-V2T!HK*~sgvn19n$tqXtRt+F-LJeXHz&-wSD*?7+lCeEWbu+2ssIDB z`ot5o)<7xn;Wh}0nJA`~6a%X9(?dyE)e$PG4m~kZ@@^9D~2BD zxiT4vY_nYm{#F0$3)n|F;HtF|w_fmb+;#1vxb21qVOnij;|ab~4TSqI-BzBua=&D} zG?ju0$B;iO#gGJpjI@oCFsfU~hADE@1VyrR0~R98z+!s_d{12%yj)AngrK7uV^SnZ zR-!j22@l7gRYi#`G-CmY?4!BYrD726PSo7VihAx7xF;-ezm!lJg-mshnAO$K<|Kg%KI3^ z^UCj-b18j8$w(tG=Thm2(x!FUi%~I%^BiLg?j>Yj&AF5JgT*H-(qT-(LMxZDv;_M3 zKDAzO&mdzio{{#a0z-jCqEuXp$l|PtvV!zSuZyA-7q&&QM)oFLJL9@2AFybM$tmtz zSoq04MIgAuq;$e@sER?jE-|JcLy{H%RBvEmF99-MC+`){pTAv>=}E{Uq9wL1_i>cR z7eGjPeDnWu+?2f@8Tc>d_qCnGq;Bn3$v?uL+6bPqcMt-k~LAcfvSminJ zb1DWLNdB@oc5sf0naNmIfPIb;&iTT~Z3ts>+xUt9lS%ozKLRIz=`6Kx>o>gZYefatYh_AnSX_;uZlmZYBAUY@ct=qlQAFj3A~q_YiNtY z`FyEerLdbIG{=*Qec2Br2A0@aF}+oxzmidwm}BW4vA+-?=Ap5@lb$Nx=K95JOUQs$ zzc2lTY*>-qUivP1A62;M?aS*@=Oz0*#&bpruexNT;Wa=8JUxOr{<+^~?9OxNdcLc_ zFU333*9Q9$rDP*{m!$#*UOW3?k^2jgBIn+Qg$q=g;=WhSP0yX*k~}rp?EHQ{%01>I z-^=^W`?drYtF~poVSJ@n$SU_h7PInwmKbA6PCRYR;X|MJtPlBb5j&$A4?T6)gFU! zF?`&-&l0>0*2xFYg#QQ+aSD8yxEcUcEAe=C8O}`z9Jeu* zE=as_P$rL)R_u-M7RGUxk;Czk57&#Jv4}8W?85xk98x33n+sWdy|4nVoxvw6WmGNZc3Frshf1uJ*lf3nmgeFU)04QnJc0X) zBlx~Hg-fSK@x{@#IAiT9Tt2o2cNHh_BfEez(*`z1v*>9)-c~H&Jimgw%A?riWbnO- zWw>#21Q(Bu;r6LDxPNM2+*=;Q4T*#I*XxMFDvp>K#VzCe;rrzg+!$Bzu8Id&>!4c- z@M+)14YrP-<}R z5T^$^E({%9_&&}Jc{n&XvT%0n;H)IXIf0LJ!UUhKcX3#K z7O4?RAwAVRsKmy-KiA*_)olFrh|C0UDoo-0$z^zEYF|8&U52|V-_r8)r&H#r8bHUoSrui^0Q6wcMh@%_qbd`Z{Q$t6hjI;v)b)Q-`r z*m&3E2#$U2|H1WD17}qg)0+Dz7GTKUm@&M>uSi2MGVOkLA%6UOe)dIy)R^C`lggo%eCvo4;AD73UJoJ?O{llL=f~#)09dCQrzp7-3FtvBz zcBI!?MG|)j%4ALi2`;+k8~Em3_u#5qZiPl^Uy)i<6@=V;Dn_)bc<0~4F}54ue_7B9{2y^35l5)JMs9@gOB3g2Y!VsufGk49r|1EQa z!+XE`-TS}aJzD~f!vtm;VIrs^@3!#Hd=}SF?t`D@C-Goj!!1@3Uo0)d^_e%|mzA~n zQZ27zORqqn$*QR9aHS&Dv2r}DYC%c?;){byeXT*rNt@yqA-fH;wl_l z?OEx0>$@#PMjc129mDx6_P}?wDctN>I97Mi(L>k{2|W$uqYerI?FQE)GJ4p*WZ~0h z7B|~DJd_*3GgB+^%=o@|a(q8*%8%kZj)AYl3EtD_igFjnTwuj`n_=Nxr;MMLNASn7 z*W#IxHweS_k=zPAR9=BgV;ApkE}|JOAT(xh#Ml@vpWFlAbH{O&SHQa~ws;>`ydD}s z6Q43e+@+a#I5&kSGb`}y#Ov_%)N62mc|TkoWN>^d!JAtw9%pfU8?{P`Lvt0JS}x-@ zyM!NT1l0Gzlau@5H=|>CBwNNEc7zks8WtM~N-0$?LsShN@6DBQY1zbsWev|x?T5#w zUWc#MT=*=ErPncwYDAfb1>ws3x@yK)R}8Vf=wU%~mJa^yV_pj|Tk+7X*yvPjdF0>Q z9zET`wnx9Bo{KyO#n2T?fry3X?s^oxIn)dnZ<}6)E3(Ulal9!%iCdizpQ-iHuFB62xC7ctB z9>?~upjU9r*lJw1YES$)w-+`Q{~1qa{|Qe_zaGCVP2h%n0Us;nvEC|S|B{I#)|7De zihc25eji-#lyQ90#rujTjxFSIv!w|m{2Z%@^_3zbBM`+ZDn@AaI&vv(`WiT9a$GXK7VjO;BZZ@D=b0_RIIfv=FlfgKs7Ti^d7aJ(aJw(WaZDa>Qr=XdP? zEkDNzcI!!2CEqqrdRtaEKi~JWScK2>eb4v$@VOmt7fF1G7qSA3$+%O#MyyI!pt8ug z3LAN|=NFSD^vY7z+B^Px|GW2Q(689vHut$ffBMV*LN)@u9pMMzZup7IQ~{)QG|)MvMTZF|4@ zV&^n?{eSZwZ+qRUz5eok4bEZk{=xs-d)&4y2W_$R9ZcqnA1FQ2D8-w?ZZuHvJs7nF z#n2JyY`scWXpC~Gq+!`V=*ve&#n$b=y>of7*9OOJc9Ek8Fn z*THjx-`o71&wnj{`MIt8*Nc6AM_=Fey488AwP#1aW6Spqu5<9*w%w_I@7wadgY$g8 z>)i5v|J#4_JO;;>zx902ZTtF`|3ClVoA*b5?du$T-{3qI!)Vv;_w!u?zyJB3=W}F3 zEgdYDdJY8L^4tdRQ*x{7Jxk1{7yRBA`n?^0p8I=Ywvm}f=A>Iyi4kGj(HE1faY}|) z;n62`Ny_`L1ss#nFKm2MG03TWL1iBCpQiQ55^zkF1A@l`;P^S#`&Mo9{QGQGAgv^Z z2lr@4@Bik#`wQLQ7kYlj@4@qptIhlLZLewD-`n>8m-4%R`QFXr@GoB*v>hK`y!{4! zd;8zB<##^+-!F81{QkezZ#>^Q{_lUcFq-KJZ^w#hZPIe058Lb_^47sI`mppKTx%8< zy`m{kOs1sMLXqP|n1|1I?z_ixjGv?s#a14?l05ck-9*`Em1|*~L|~bA5=<&p+n>7U%v#^UB}iw_iTLA#7|i+bQ=?W;@q^j=v>fm?{NSC*o@? z95bkXLZ9a)iSEDO{d@Ve;jj&M?p{zf+9t6eqQ%=R4la~&x;Bmrnl4VN`#33|#aCCZ z!A%(gcO`r{i9|_;>t@ZZBDI@{>>0#v6*b4i8?|v9Ik^vxS-TRS+IJPcxqL4?Hn}g( zNvd#zMdZC%%$O-oT)P6l8s8J&)^qqwy^TexuQ{}i>cDrX_%ee+JcGmv@h{T_Tsbj? zO=BZCEpVX)4yu-kcV%<et}AWB(W5v!`%z58LWuLHF>X(8L!yb$p`L#zwimd6Zo4 z>8n_%IQY2j;7hKB3u6aYrZz6C+W2zK#|NtkHdfm(tOg2HS4-E4>P=Mgu%Tq)7|+9r zk%M#VE-sBdTNork{w%wB^qu8!R9n_R%mr zyel`2ORPM;XHMh#z{K%Uj5#mCbTor}wGS)pq7XEZkJ>1?>)_ZMk(5Ksm&fte@)X`P zIf`SK?Trgo?uGj%mSa<46lc^!n6yDjsAAQ{{>1>ttewEIt5)H}@wK>YdK`CZ1>6={ zxI8s+ab)6h-@$c;gyx-k8^ih9G>$Y(_*M+pt)Xd#cyq3RkE~gN%U=Iacy#r?xZcg;!)YF0TJe9eX=E9$ z@qE0i)|2yl&twHx=BM#+aRu%!PvK(6#d(2_3!)q@40K##*|;S;fgemQ!(DCx7u6%Y zsbYw-&ve{GE>?vAE)`2%s-lpEPjxRUxNTN(+1=*QA#gkdj<+k~?PBxVtSD+nz5IA53rrHA*k1B0a^V29v9Np2wh?p3m`b9t4;NvaZ{H=kD}B zlZ=#5(|r1QDbGC0@aa(%MK=-;+)6;UF}*XnffzE5cc=UNV&}?4zDP8Rba@k+R~6>D zS*1iX!Q)seIx8t<_G1+3BTYZrSV^m?g!!v;?s&1b9lk#77sEa<>;r#`J}>}q?4SuJ zpr1FbPHNDt5QZJYbW?er()vs)6WrZ0F5201959ZRwF$lEJ184h#N=gKN(s@E642x~w;0 z*fRnibId5KlpXu6m*YJd#7};;hk_A37e6sZQUxmX{%tGZcnL^me8sqm&&#|uj6;XH zh+!Yt={`WU%W+?23@0{dHMS14Ls_R;ID2daPi4n(RanLTwe|45 z1=wZ}fz?4|k>JokKCL6NLmZMfab;G=Lpcp6H$oKZWz1RA_;PLxcbCWUi`?G$W%i%( zfc^&D80YZWB*day7rUekmDpmX%172E?VZZOoLFh-eh8aV-Btw0PvOT^cu@>D4B^Hs zBuh~7m0d=K)-YnzqxQqJDRI0e(x8KxsE2Aut0f;+7>N}dL1r?h&2$z;M_IghvZto) z%Dn6`Y+9xSAq>xj69fpO1aVYH9JUY!Z8)^)vT0>7i@euEKInp;wMDNbmLarFqJMhi zH;{x4#8C}Fn85Y}@Y?jKhDy>#Ih}=;%!#Zw0eM<4I7uCTQbXty`0v2Cda}>)>{<9u z2cM8PJ9pklG zv4}f*?G)}>z7mg5jABz}0{3ULxZQAZYPEwyn)B%SUG$4peAI5>5S*cH9thGCpd6Az&QmC zKOWl?Ppte${C0X@+?Y(^gV8818rd7ao5&a{q_7r=;FP>@Oyff3Ow zjl~p-NvB!6ugA$N;rx`wicT{r~y`gts#>xw~$#7f`+vac62W0&Yl;pI^ zU(}&5t^Zv2y^ugM3(4xjI#xyTim5I8hFx6$hxhq!bDv+X?JR}oP6w9Rg=h5<>Qz*2 zA5J|&zUHAu4kk_R0aHdXz)!Vb;^Gbp-M98fIc+{na@p@AEm zGA{9p_+rDrzqFdjMSYaw8Pr@C?=2N@j_cs#?FD$r0yM7=!)n9g+Uiz?ElIX8S&|Js z!BKXCFL*JIXtq%fBGkDY zekcV`m1q{lxQ~3=MK0+e6LpYJXHlrmAscm(4O+;DO=w|5@@Qo*5ipoV`SgR=!afS& zB68t66oYwa%)PiIZ?vFCEh$7R#SIjqCJJ#6xo{4-kWAWn0e?#@zzQifpNN%QF`}n% z6D7(~dp&6GEOd7sh8%Oyg(2FJb}t4^6rvV#aSIuK#_zxm7ElRikx$ylRvXBqRTPpM z%27iVm)cZVnngL7N1h;bIE!4|M=tK59WP4NTr~}*U!E!s$!a9sN#6e{f=LRiY z5JWgDb#ZoT;TE(C};UDsi&C5e_=e}-fg9s1K?DCD%wU5N!=fF4 z--Ty)(a1$OsNmy0zK@SLUHp4)B`))dIAf-YLs@WOhsgW9h6P0SBKoBehvhRk&CKEI zP{*fQRit4Rq1nb8I$uubr8$a$Q?hHaS#pvd ziWa3fTF4uTBxVQFPaIE!zh6$*u(Ri)BrBUJ*(oaYDWQ}lB}jtW=0hY&`21j_z)Ca9 zP7WDEJKOFrhtDza^TJs;eoqpex-($aHw8duA7Eu_nM`E2CJ8^vkQz&gQ7d5az;J#Y zw!yFu4Ew;a5B#nB0KwNX0a{6zCzD~+VM=ga_v%s{XHg0->Pqn!KesFA z%tBO$lGtt&EFw}sGQqK?{7v82fx?kI4%doZYVTdSSHsW#_2(xRU0Mkfwp8U|qPV`a zk8XhDjIjcb3cdfx*a$a|w=LuH-TB@OBySlrF;*JHMvSRcY@vK7dA;;wl_Hm=d0@-G zcjq=3p8H>aYz^D=#oJU#hIU|t9hq+iGVp`-pKJdBG*7w&|0=;O41(i6nYj%(VFJsKKxuD2 zuLlb#`|DBiW?_UZEb9wnmue*>YZT)Yxj02p_796Fd-Es-Jrt5A^3@pSG=?3gu$Z4= z@uu5><@DeTw*cQ+RK>1Qf@0*O9EZ^228^(as1jhYV&bh)h{LK84o_-0Ebd}mJcrPq zh2brtV9!B|$;PfAa>|&u102*^#C&xDHMfJnVZkhc_c|(mL)hV#KL;aN#FW=Up+UB@ zg+pdibgACwaDP~b7dH`DZ6xJ6L@t@-Rm|8Hj%dbMS4~hcB6!{+0(}wdqb}YOI5?u# z!9o54>a_(-hY7ktgm*eF4y-qzhY^e@L1240yk5n}>In`G6C_>}f!h~_0LPsbCUML0 z(HLEY!;=_C%*>-@Q{sFU6>nDZy!1AvOt$4R2i%grlTQ+P++F9!A#gkdj)%bUb`mmM z2B<^E@$+whcGt;oc;35yKbhcQ@`K}qiH`aBgGhVLV9#IQSq4kq^C*q zxv?Qk8Mg(?>%gYxkGt#RZ+CCQ;rS2yz_1St`@r9$4={18c}f9;x(JQJ^vRGkhPD#wE8U37uUzZ|((3W))u2ht1?QSsOw-cxA%6SYw zI~;$1uRc)rSY%g+Tk8vOtgK&{sH_QtNaG{rW8;o^k@GywB|w=#>fh_U|60F0h@S?r z5+8Y8JQ!E;^(|v6&QUxD`^vx8c?`d2I0wAi`hXOU`E6Oxxvq2Ukn+OZGsi*Ts)Mqqh7Du*S6iR?YhJGw%s`6?7*k<`MT`WF`zR)TVYXV)40MCutStf1 z%6=0iVdAz0Y^L9~?nhF{sJjbr^e&oK4^6F(wq8fdwKIrO2&w`|S7NH+r6{?v0CV}c zj_J6Dd{Tub%4k#$tc<;7Sex6nHe5<^FAkv;Def8^inaubOL2mfqQNOnaS2e|3&kBu zaktb18F$z7DXDzuZ2v+8_yeP8ti3)K73148dCXbayU2`qBHe%g>~ufy2)B z^HU-9i1*)vZ_M)P;y$ z0_D98fANWS-4FI){sB4rO}Gfj9h%m@*W3(;PZerPFNHAK=XR0?wMuuh7QuXlfM#-r z^Oss#B-&q5R!+;6605>2Ih7VkJ`yIe1dvX^n(;@?ZtB&cq64B>&c;u~-gnj8nDF5+ zECkpIMD4yMe=#mAh_A-WKr%pBHI_RPgt2W}@eI^Lz@Dzyr?Q6r%x+e(FWd~nr5Jb` z-ioi$qc~d9^K~x8iifO_rHB;-3zVgFXka}~-I^h6@u@-?XI2}_3wXvh!!a~QUw#?G z!$$Pl7N%y*52b8DS>2cZirh#;rk)p*CB=ZSYU1w}WeO|?l$sBAZ2O!v+*ba7tk_Th z9o3nCXR7<>F`rJ1?RcAz=lCxOML=8n^&=TGzL=)L4fxuEh(qeFT1I8z?dWVO?z*=L zRs_Kdx@Vt2H<)DouOlu|SD&zPGig{3<1>p2KYT5&V^d{mQoBL_e z;U9CpIavm`)*j)>s67^6i_ z^0zDsTJU&#_&S6MjkcnkS zfklZ;TS0k<)>s}|%zcTE|98p-dEATgmOAT#3mZuXX`E800X5Qd3);7QQJ|NEH7Mf+ zh;!V!&oPG6(n6Jaarjrt_&!E=C*4{u-_HDtV88cewB^W*a$4AF0 zja&w4;k3cGhAH0`T+@i@sOzF?7e5}VCsh!v9(1|1{7RlHXjlITo=Gx1eXiRLcs>1b zOtzm=W+^!!5iM5Q6O%JS{k6H)Wcs7n_Sqn19geh$F2L{4RnA!keQOaPOrcjXL0hHZzbK4r7T5mpgz`<_-nj` zat*c+;FIeXr#6wxL-Cik_%-v2`|j&FmjDK8v;mCWNe_JNF1^gWNYov)k*kT^*l$y0 zaanfE6~(jvUJGEqolvX?JAa|1>g3~%;hWUp9c+&>k3)3xk|rel4x(^dJox+DN4M{7 z@`+1m?+yXDDHp9FbLCiKP5UFym7j~-zx?7@r%77nIa-3fvq!ZAb5&Q;Q{sv0O#i4d zQ#0T$93KUK_r&+A#ZRTism!8rcD+$Bm@IIP3{b}D#a}u};Sk;#E6oI)pDn24j$YSj zo_@x3)6KCHGkWud8PAhmUA5kuI(b#0Q_?zaPL|9|JCeC@4=* z2`1=@tM*lZa|J%bdu-xm2)UWqo*A{uCi6+Eo=D(;L-#3bZae;!DN0i@v*h<^URGIE zvKCv9ty88SmF`m_uE@=gO499uPD=e0%q!?K(3I-;<#mfIp6+vqfbR#cE4n9VM+2Xi zudjtaN+S14Qe5sUJ$My#gkeQ@QLHlv>wxTjHDf&DeR$TYC}eDK+acBv%cFY7Wv8Q9 zo6ezT+`zVL5GL;)5YYD|8beW8=47!R7FqT6A=keFmM08loChX z3m((X+)u>()v~Wa5aRNFSXn_u=&Xn#cbKo~9EP!~uW2Xn)8}kHr|f1atK!Y^9o8X{ z?4LwS9fUx#D6FBa5Tj$om94$?_D)^CRtpV{ikBOFK|_gqG`%+Cr;WoM5=;L3GsicF zeh+={_z{NEdLCY70@wi2?URb{A`gI$26V#zdOSlHqswWmo`*?5lDD0%jf>tvy*mwhA z%i&~}i*tdKgmjjMVjhqOMmtTEpO|u`<}5vX0bVbDJQP>TY>O3ylg1g+ODi(ft|kj6 zcro$`ML7gGw?(~Tbzqbh=El{XeKDu6_@Z?9!>hIA&rBE1R!SYKBuOl!zoNZWGO~xC zu-l|O{idc$#iG&umZL!f=!YjXpDH<9NCq;SpG;{CdHFqv%!%uZkF!C5xv(K&KdXJ~ zPLACb7AQUoy4L81q|quD(JePDv#h;?pKxcG(}B?>P*m-UI3G0|D(e}Y%&PQ(8vvhJ zRpvL)Ia*L**&%x)gKm6B3R}WM(kWT^AnRvP71p2adeM}>sjO~7Sg{kWRLxQNaUg8_ z8XtekKJ#y34qHF*-@e|tSbADmqZ%SZ=G7GQkkcj{TU#bRF43k_WxRZGevm(P!Nksf zc-RXuS5Q$ZE#=RVIP%@Tn$8dGk`{-J1%t&R4tgQt=AHe7cG8$AkLh^kOijkEoTBkk z2TghA0u&hel^?%)Q-!#DubUKHny;ZPt$lW1Gg~5R9eZ@KxGIV8Zb$g8|MJ{dWXvFn zU6fxDo{ej;t4&=+81CdOt@hG%03IrK+kwn(spC+Y^fCUL?N+ih}TE; zG2y|`^pM~@?$zRYX9y(g-K(kno^mEW2Loy%1hta-@1n9f>R?;E*OB(jJT@t5bp6hI zzeN1{R+S65lF8ZibXbFU!5pr}s-F)M}CjtvsH zwsCXv+-v{lh0D%`Al$kl2sWNDSIN*gbT+TKZDuAz5~d)}+~Z7r?N$`~>hoKy-X5=L zptZ@U1KK01qvdKM6Lm?TM)&R5EtAla0M@Y#r6iGEQ;K>CBt#8o5q=~FkTxlZX*X9n zXGr@Du$l+FtXh<7U+T@_d>fvS!YR9eZ|M_Tc_lY3|0b6;fsF#ZvIt#H3K7gbUD4Z3 z65hBH&H|S@m3P z_L60bK0?l=2%miku{xENOQDDxoAV#^n6x_Ke5mexPPgd!rPHglj^g|uR>usff3@)2 z;%qM;4qUwBk4we^#Yp3qPhItf2vS(y2BNM28ZAEz>hI5~L9L(+$i8+b?ZlTZPd_N5JXO~ROxWZk@I<%hWgl>5CD{W%z0dfTAZx)u zz8e&y)s;qpRbo0ttZwPuB9cX&})ia!3lY%5dd&`U#k++m}#-_zR(vM?VUKg zHHDQ$8vu7QtPJFUZ5d*__xy5UC)y5OGqBQNqFI#)Ne6aZ38(aidyazDTEBVSlT_;C z%-b^{CM_ibS{EHF$8ClYWW%%H)d8s_3%=?`!Jx=AUy^my$SyojcUi+0f#jTRBa@P-La%2OUWsJ;TOik%4i(yH#?kU!Q;`hv6){VSM~ut zw!Eorug^(+^Re4`#*cg}1#zW1UP8O35hcaXGr7Q=KdYk>rC1=U7@Sl6bo_5;g5S)( zHDh9YLMnn_fgQ!uLhG2tnWX=!Tv}eH^G>$H$_!BTp&m8;lvH8brlB zy<@Q+FzT2dP{mMLM-qO{tGsw;wHj!{>_qpC)+j5@KDqg;W4w5Rcv_Q(q-mZq?IO8B zx4* z9EK2d6s8LlX>?RLuv>pQ0m)!lrR!M8ku8XTXiEP_kuy(%iB*hKW;`t(;P+;1@U|`4 z3?CoPEHA%&QsDepKz0XTIqrxm`p6U7`_IC|@pWJHXTwLq3GYo}_hSA1oeDcZuQRmA zhZrI%Do;N+xt0SyLUHh)Qyr(`SsgL1jya=DLYYQJSdF~KBfJANcGiCo|!lPBuf9Z38~ ztsl!;d&RNs(u6Z(HTvWQ5kS)qmJj4g+!V1FyQ{X39Ak15l-Gc|{qkm8ZZ!0~Kh}$w z_QFe7(jdXrPic%4SROZ%A;b0~`C-{jp{dtmPvk&j4ca7OF3r?L+pP`-@s7{AEp* zw*{-DgX1hvkxx7t^iwbaO&{FO0JjxD-hpPz4o8w%jK@}G0G>7&x(Pkr=tcI8%c#o% zF@1gnQsj>{Q7{!G-1IWGTD41|9xmMzPxriHR9>_wAKh4@3^w6Zl~q5LA2=)C^oH0HcH zvldQTB5AK%-Hf*6Db>)eA_Fh4Bz_>yCme*^PM_Vim3Bz|zJM`;$-Wl&uRVH3L*eQ~ z!Q4OhWaN03r(XiYK;)fmFxU<7rb>#E7JO#_7I~lmE+8%^9EzexrTUp1TnfR3?_Q%s z=CLo=W@{2LmpD#976cA|6kB9Af)JphGX6)4yRR$shACerKPC&Q4X9LO(Bfj9t|3;)=l(LZ(sG`%@2UrmfR<|_7f5RuoyQ%`xj8DSrQhnlkKT+Y}b^ue0H z1!rX-OZn?ouN|?|wmmSwN{W0_R2L*~Pbk5lB2Jnv5^*q$e=#c0=F`?VBXv2E5-ZCa zH=-2brsJKTheKzv9S+0~e3R-jnRRIcDZ0souda;p+vsfGzz#9I3czW^*E@2Dtf{se zmnmuDrc9{m^#VSF`4QDD-NH2Eu8EqUvFlbL?9To+dr$tC{8dt6u^mf9gfbYl@&IX__*bu#bm zz3ZttPs4_>0dT6nNDVLrdT`+&G~0L;;RUF|UrT#3-iz&Be?^#`_cxGs-1bHFAH+Du z&VI$a4B@-)HV%UTV7sI+0RKkzIsqSqekx`ptx-zH09!TsvAH4vh2Xsr&?tx>Kk}f3A3g|`-;Nv_1Z9-gi%G* zK-7aZeDWI$zHx(z1|$Ened~bBL{%jF{5d_Oh>LrsI`d4MHTv3>J7fQyYbPIdI%d;G z`<7g}pB@P`ZAK(IE>ce{b+>G{F<-MGDqo7FBsr}z9seEwp^SPp7VPl(z(vp3G_J-M zL9|l_{PIF*Mv|+$(6QzorK_G=?!v2r#iX+Y`R5kt4n%MkNh*{{V6Y0CCHA}_%ZUby zFur{yjF&|8%kKUDbI!fv6#+0my%NTYO??A>4g0AiBLNuAtBWiT*4~S#P8OR=#YuBz zBx~+W5#0t$HzuZEfie{bHFble;$FZxjyR8M9{M`69GXSB-EPW1#vbB*w*#Xa<9UOFN^$e&(DnI1LKd06@u8{rheC~- z6d>9JN?ALJQUQsp+=4l*8j}+vA@<}Zs8T8L*WNPo(W;R};QU zwdf-6ANeBILl!Q8N^iO0foo9CvSuWsf;fRe=DtN4(?Lh8USBPJpD(=Ced(!q!@J%O z?a>C_^F3En zZxEW42KgdCF|Ec1iS?aTebTE8u9`Htk78s}y6bu&a4!S6SE;bYZI4j)!T19~M=YxU z?`M8@4D&^mUm#lOE1u{;C&7ulgVX+oaers(a4{=8rv#}daA*8-1b=<%d|fn#b!Pup z*%-|`sVhM24rKv?2cfTx4_#|XU%@y%kc>O}4Jz;mzJO`XO8JP7f_x-#F73jzu&oqY zyvFnEs9nsQ1E7_-kZ{1d7wx@xqHoN^-f#fKH1%z?)uS}j#QwfsneMp=01j;AdpC#^ zu=a&G1hr_vs>(6Lrd4poB_gmya=fVd-Y_!Bcz9&xhQudLGJRYUTZiN$tMsJ$pjL}( zzH@oNH>=&yP0t{&?X(ZonePT%`v4wJMxtWM18DNEAP4#%O=I*U%Sm+7oYPk+^$~Nx zR7JxPGc2;b*0=0i;@7W*K7~k6a=CPX%>`{+b>gvh-*H7PYS$LNGBSX=IYoozZj&O6 z1&Q#zBsFeQ%nO7uMYXps(FoE$zgrrprKwuAuwk(F*^bhjw~yjaJXC|r336?I)Cqnh7U#aSG}oZ{Y?G4vxbuBOuUlc&j;p% z5aQ)MD`5rk<2$!+L<%B?4@NHWrTEW+M%`U8Gla~M;j>(cq%YA4$Hy;QTtR zRSD&h5t=tHzcNTj73K0*)DGcwe(jKUnBk+jR?w-c%Ri-09X~UzSID1=1D6jI%Ni~h z<7ibruuT}sJWzr0HGO_QyjN5`;?4Y`q(sGj!$NL-5tu%hJstFH-ejbGo_sMb?NJPB z`r{f1nTvU4(`qnInlullA`|{EvG+!k7E=*fkvqMW8~JE9nT_s!b^ZPcH_xREOZzQ+dz)i01?OLBnAWG^okgY1M;?p}uV}bW-6vQzi znv<9?t4Gt*)A5PQCofFEl1Zt1%|lUtJnIuw%{xh{A7=#qPyh!d_0D=g5TzmTSJDxr z&GS(ZsuI8_0_|H5t)kZi1&ueZ(=RU4bgBT5(Ysza?>cZ>VmlKbOz9L|%4b^KeOxnJ z_#ww8V}uO7@LD|Q=k?bY_#d(zMBCIHzN?)=@f~aEZW3=O_1GH~O~y}Z%>z9l zC>3AbxTWNZqq}vQ4{Ezr-}^0ttx<{8b5#Bn5hr~fW-U`TVD-z~*b%eFqOM^{p^%`a zgfgdfp(dFwkoobnX^VDKz1~2|Dc+m|6Di;L1TZn97TIHsC|KS}cy%wP-L#*afI|7H zY!IbM^jYexpX5rNQ*p=WW#X1%H7YsOAld!1^oYO$h?Xn!%3^RD-3rgHFi;5JmLc?< zv{X-nHL@jT77i>QwhuOfIvz{#^#D0JwdH_NXW4G%tp12_o7`a~Im-wJW{ql9NU9@t z(Nao8M+xO9$xm_adNgLkaNAV3a#poPabuUwXJ(vxwV1Q6RI$#~R4ZxNnX`j$2bai? z*ttM=DvEqpW}}vHd}eOh@LG1ltyE}RV5?($Lm{w>-q{R;h>T;WFCyyG$z5HQUeBaC zrO%cxZ!#sUj(KshcQ`CWwV3ecQV@dMhsR{0v3E=>jt83UQ zrs8!&?>-?unwhK>+3|YUya}nEpT6@3;wk`a=0=nTj4H+&`^2Xj1nxKmPR{LoSV%sN zLh&O?eWy;gEshiB3UGNDpO)mgW;@d3Srgz7JK8Jje~C==5(8%w0vWFG1nzwbo9{2} z3GN!wY#f)c!1&;MA&QRP98EcDgTy3Rm=>ImdZW8bClY35TF-=^eAx=4puK^Th(TE? zTqvpIrp}@8!m6PlKBrAw(3zLykF#%_&{ocEgYxJ#MVU`8$1xK_G!|POOibg=nf61m73T@GO^*tb%CXHaqOH6ztGM>svMSY#g^el-$ts(qGMu?c zL|B&!Ucx!HHkIMO3ZB2Y5R8?_5R?bIWN(Y1#e&am&qt4_tk&I5-m>sHcPu^3c@QD< zDZDY;k>ON-4p6%5@KTkHrs&|+MYw?hRQTNBc{>+>JJ9$xW_+~Hqx=;OaK^RxxM;Rr z(Wv_bCr{Ujj}0#CHzG(GpP>pa^kmW``3OIG--^yROf}&8ld5Z_dT4wgs1*V6cl$j7 z0V&o@cNJ!26Em?(J1lODd0ne&)m1IJ$I$lS z*xtL&|FnOo2Gy1QR~}RS(OigijyIY?{VkbP#{>CgZ;>RPFk3hN>wyn1@{J@3GX_X8 z+#)28*t>?Zuy53kax++QkSHf*nn9OU zTi%2IeNLz(am|$Pj;!Gy!$7f={-2(XD@H3>d<3r!z~<9m6)Ggf7C9GFN0{%~y8tfttkXUErd z?GP|2l)$qQgnnR1JjUEIQ{t0sy@1{Fq+_oC!tb}{(oH9r{`nDFC$b_}wZ^@2in6H> z8w9_`+uk@TcJg^bkXcgqu~N)~R%@Ytq5dC8zKR^R=2;;W-h(Lu`8*8~69zOCKI@D( z_=$LXGcJcgECwww4lxKk&^5cUw5WLW#&JH-sA`}5_;N#ewwqn>g|=IFq34pqP?qQR zC@3@IP?k3DAKdk?OF%J!dIXw(&}@9(P8d4NUsFt%IWDKTOVXnB_C;f#(4-EL@w9Y* zlaDk%;&4*)nj;M7WVoiae(+DAH^<6e=u(l#{0Z{^vzGnW zXFP7`<`QauWUUR%4i3CjNMrB50=Xu;0u0P>^czAm4dlmId(PRam+~-S-skDAKj;Br z+WGtR&N9TT4T1TZ8Ho*PK&fTeG%Y3-e@xmxx~D&7_!$o=l$i+6>tkCoV`q^QKP-`s$Lx0WaKUww8*QK@ zD2JceKK~}QgTTYy+ur+sR_cE%d|(M5;a)BqfL5V50=1y~cuDC;dH3;eF=bI`Q)3?& zpDL+MVFf<*fQbB(*Q|smT4(m10&uJuZSy1Bi43Yp?8fo$Sg9XlsN`uw3fH~BE#2snV5b87c>BZiUPF2Re#JG#sjflW7 zd0-*@l2-Em)g0Y&zZH#`9RvJ*>Cd!xsLQZk0^z^X?!UL>zXqL3h5F@#sTy*F##5k~ zI<`m^-MsZz2O0sZu^*}J5#%58$$?ZXjt-b%r^^trKXc&la}EMJ2JX$8DXQqnkh-h& zKiu9Qe112wMVxbw?A#FEY7$B;6j#swY2yRan9K#QhuAxN1ABx2UHB-LYiE|GS_Inb zo+q8L=&wV-nRP4?aB?Q=v*^c2eL&nPiJFJ8mMTta7wy*qp{IUms(O81o>JDZE%?b2EIo zAncg&gO~uHochk^RyA5;Qe;Y5@OiO>7wbHJ|)9(r@w3 zr2`+_@(zhL7TF)=IbrpfSje9RBV__BYuG7KC^%J zJ0Ahbe=u?wJyHjXkxGXuO2zWW)OFVN1lSONS--((wVR)c8-%);CRu650Zp$K__ zm9Mm(yGeJKTf`Z^hG7kB5k>AB+m6eXe}3;jxZPiu0A&9qYGhI~KN70`N7sqFuPp3+ z^K<&3VEvqL&@8GG85$kEo-1-A?84&0p3C!`TU<)9fUT7EP{BqsqZzSD_|Auf3l%Eb znAm^fSpG_CzI^!nXP|<}fvO^cozhRK2fXlpohdowH_hBDI*%q)P07tT(jCkcv_rDp zpdUr|t0U`3*TOtj1mCV+x*+^1y*hB2?Vj5wnS~*1^b%M1vE@dtv{cOhM8E$8RjbfI zbqNkKD0z{b@-W7zNYz2>u2Z)eREC0xOc7NuHW=J?-?VtoJFEDx@KSPz{-~j#zGiC{ zY4pFMwCDVSTaLU5z0WCO$5MC0)^b|gcUECW7HvW&X4bz~M6X>^(^S=BtB};jzg^nj zgmUC10ina9&VDM%#AM8nrS2!Gf$oyM3L)?wpu-<6@Z5lY4#^yEMz&qI&%7wDzI|G& z^JqB62^Q-g*#^>XqitNv8;)}+bIQa2L^h@G5npF}nwO9U^t*mEGxv9%??0b)*7OA+ zCnTRuXWjJ>yGZd<=?AWchu>lwOd6^*UL7w=idX?tko{0npdZ%i{mKD7pre+y(e>cl zfT*P&%Iv{_yMv8%2>jJ`_HM&?&DrOPy@~UG;P%wep}y!^z-+nJv|(g?Ay-A*oQ=yN z?Br>HA@~|Plq67`ANZg2$^B|VtCmmf_valow5ytaekgGq5go88iZtYr(x4iVi*}=h z-OFk%DlIDgPk8A+^I%%Z^ZAd75Q|D_19MuSducnJPf4ObqVqYR$uz}iNd^!n#}9d5 z9>)4bFW&^Dn4lrgBlWHhVyk_YLRDUj=O9Lm_p7%GuZz}8m9ykBeo?qAhk|uI|ot>T#{&w{fV?1Wg z*Bh51gII7ODN*jo4m`&4r~AcN?B(!zqZ>t|e(oc*#`2DJGK;pSd=C0gtk~ZVy6bmH zj>9jmU4NZ^l#eN>|N2ja%R~GJgR6XG=1|>e48hv$nQ8s*x{`QGHK6jGim%Z{AGqnZ zcQ(Fyp5up5SL`pNrxhA`I$gchfNa&!B;n~ZrF#4OUhz2I{~mY07@AMo#VQc*&OU%f z5^XyW26i~y&54r475Ili>oYDWB~TAIhT#V2U=QKeINx%O>m)9@-eS%TuVAdB%3~tk0awJvXGzAVd^;wl3(4E*MHB z`%gTYDHEmIx*iRTcwrbElGPa##HXksIPL*E1G&;$k)exirUN4?!oRAN)~Y>M)srWR z+~reUD2N|_ue3m_r|$jo&}w7JmG=7cKlN9p?~!36@F&pEeoYHVqs%Oh9f=Kl3%~q4 zNCpx+NNzU0GjEP)s=kL9jsv&USRA^z>f+sZH5^KueAIByy3Q8vb-{VO|0o(8n;@fc zrx!gS7NM-EZCmknn24$3oPq4d~t2oL<3Ystsg!wqobLQ!@d$VI$mf@gJq5pA91^j(_X$+4277FC|rN$eU>qB zsgl1tf=^La(^pPQ>+MiW3Io^b_OA;bpQaDEKn%c9vvOn9!DR_1>ZluP$}rR;u~Eri3iTn+yzkHF7tR;X z|7QR-@(ihv-ywofp&o)YExQ1iJi0(|;8+t9~lP7LSfds3u=S zjxBOcOn%N-K6u_jDX>1sPAzDQ+{{amXh}nc*q2C(niy2%hffkj?L@;+%*9^+o%);s zj+d7B!^}cX(&Gidn=toB0AgAWs2ZaETYei5yWI$0q%e!$$kWI7m7sqcgV{mS1efo1 zBYH9F4q0$zaITL#Hc&$2WMqvD!_(KO_4QejrFGoDEu89y-PhAo(Gp5w_j2kvP}lbq zFjU><9Kkjqtu)MpY9z^6bf0QhmKx{*%9*xnPl87V=nwk57xtbb++#e&2lKyNP2yLz zje*M^#4hjQjJQo0-wv2O{b+73?rqbiTxL@LgF^n)693Y_!&+e?vDrG|Da1}FLpDaGK^<5{7J)vLG-M_H>kj0ch7Fdqp&$lG3cM19n9 z)V7<0n_i9V%(ok&&CvMs3}AsF;rg7L^2TG;sR)2;Jgx!#WE>?|Q#{F!-GJ_$-%s3{ zW02O)Gfj;oyU5>B&wq}#qo9~AZh4$M#hu3+Vuc+}0{x_}oe3juSdFiv3|ah`BJ1c8 z0Ee`Oi%&LbOy_?THJ01_ZM-MC?YAkp8J!JL-P$#R<&Iy?ts+tbB*{z|o*|a&F$a7s zGh&;loQ$F(8ic6KQ;NWBA2WL+ERu?tFglOWllbG+-OY6u-6a5{b58g z^5T;Ff+I6GFUN1@j#}zs>SF$bZ~i-F1hxN*K7jY&GQ$2O@~rk-nrW0gvy~4@Voahj ztj}R%;G#LSJ~+d&Uu6Z7n|L=JhOG)YK8i>37NP?3 zK>WyosjdUiYR#+oAK_gJxK9zqCNf;%(M}-665;1#j@SVtgkM)4@%ecR-2I}S5TZAq zIAiG+$*Q(8i?p<5yc06<@@+--7iV}t+}z%-jrUgx)83Oofs`^DI>OSzLga-PWDQb_ ziwL>E?Hy~^bZ6h@)uy{#xJB?*reiSp^gS};V^%zfkA93%Kc0J0vB5+1k_5w3gS>rqSGXEzE$gH zM^Tj~$p-j&H1icPFJv?6A@}wr-$!&m*sn$1{rAl}?}{uDw}NRZR_bF=LxvdjK7=2216um^C>q z856osZ@VZR(VQJRfbPuhneCDU;@I<`s?m^d(Yw)tK}C>O=nqxN#?X`SFYf6j?ac32 z!FPA5Qq7|hi;Y!a2(nd)v>~lq;@41Kqzcs(gBpYy!0D%YKGsX{g@#tY_BWC&6pIe6 zzx;bbAwnIJnSi|mUhlD^wVR(Y89+X7^TKFbgTF_)r}0QI%}j_x5K4zXZ3tOH@;W^p zo+J^2C?Rl9vxWEfDv0A;N?ujdc?nSwi6ZpHvge2t zvnEd9c|g9NYM8u*o>;P*`HQ4q>x-t{z*muov2Z7l^*iKJ#UL%Eu2j{jZ6yhKEhem_ znyaMr)PcNy@zZIV)2iYeb1GFOL}Heqg{?|xS(R>j@}A|g zzN@{jiJ*z~wt!_S@5PWqeKQ94;kBv zzws4#;Yo8Ij$Gbjd5>Iz(zoNe+(fdtYdqZLyK?zZ)N=G?aiwlH)pixsy~ZJeJZ7uK zAIb%#Vvfo9|8dy=iJUw+6yaxTZb%w0_z#T-`FCC*!fmMjT(Xfryl?YT@LqiTqwr)d zIm7F|z47x|kQyaozu?i(LKoiV-BUyEfCG#Tq*k#R!Z5#)9J4g57sH#Uq5?|(Q#e1YZ`yh%S6!TZ%G-9Bq_47Xt#;Y7p zgw;#|$zoW`tx=8N$oD!Go);>;v2G~wwF?Gg4@`+Y^yKW-G(PaIH)4%jc~h+e>5#vB zw-ZFa){B+~BGj57|Is-QW6?xb7OKD%VC*3I@~8q(s2i-o z#5_`Q}N~*LH#ta&-Kg0M+YA2t4m24UcQ72UYN2+8X&RD2FH_hXInUaPEi zan7aHNQT87ujx5iPHfhg>B*kNFL6yeTMY8}-@{IMen|f3sj+NkR>VtEEK%zordk>O zOcr>=Ql-yPa&mC3gB zlAN*yoB3Vf-Rp%a`a#d6w@RVfj7Xk{j zF%ySC{IB-|a-59IY{&zc!fL*iVS2@ikZe#Z@%7c2kasQPt28?gJJ73G;i%-}fD}y> z$13Q!IlfI0*T-?;(XH2;h23q47xHAWMc2)WM_%M!@)E!Ie5gL+Us_a0Wo>y>B)y`% z_RvZWdeBY2i4oi&NE%&NgQfoTK6Vjp$H;oAImTS>9`6n^8SzWM*w(tJ*rUC~{qFfr z{O{mQVqnvtwD{N%!~`40%<(#6DisPAjr{=;r^k;*AIhNaw#nqXP@?v%G(r+sbxf07 zd$bL5-@g#bk)S zrBYsfX0Q7=bnQvczz0{dDbBYax55U(psyapWxFBC4qA~NGRjN@j0d&h{Ul{4D1GqS z)(-_jpRO>?nxeGY!(;t6#Gm-#I21+?*mmXi{HUNKo{o27S5o&pPvvW3ElIK$HEv6O zRdrAw=d6;xx63asHXl_bHrm|N$5l%NHfI7is$(SgeCWm7EN8py08N@2ooJBz+A&}c zxubGTrQP8D<_ozFPfc|x9T0WwlMqk@@9^e z8Lcs<^6kG=?p^=7dgF1~Yt0oW$sK>XC5SlA=6Pv9#M;B@)`dhDOM~?>IhrI;&O%xb zk@zC$zS8OlE+#Ac6J}>nJ?$L`{^h@c`v2QBPm=;YGqsU)XCqAEPr0YH1S0NLBwfUd z7FQVVLqhM~UPrK)vlznMu-_>`ARZm-fAY0D=p-C<90<)amtp~GNN4*zz&wk@H!kZ~ zzlXDE%61=^mk0;ExPQ*)hw*H=QwFQ#C05htz)tF5cwK>xW53zmgpJ`BSfg+}n^28F z_!^my+!bRfG?vF82rR4^497PtLRln;$FWbS{16%pu9xal79HbvZR4wOmF|iR`k-%^ z@{0Y_Ijy0ukVEs<&D{B-#N5dUS7Q@jsXMWJ849z>M`G7F95JO!PKzki0kYKhEm6^n zL>_g-xb(>WqX({WR*{EJ?r+3Xbrx?x-YbM*5chn1uZ2pw;SVq!mCUv66}v8M(G$N= zbLZTTBzyAYVgomvKtBxD4levlkX{(I(0Z6X=Tv_%zJ*q-G@` zT7Ij(--9Y*C5l-SO@#wT;GxJ(!r+NMQLQ6DXVFXSwMo z^0w$<(^ug(vrvySs75#0`#93pp;OZ!r*DRadt*N3eB(Sv64-WcjlLng>@D4plZh4+ z+pqYyv?NdzeFLI{tPt}PivO#mL8b#L-SC+cve9j0GhV^jpC@R=Hsuaj_72>0yeBmrrFincK?cO7!b48x~#BL0R@&XD!sMe#AY|s4J8s0dV9{-=#E~Zf#d6rSHCPr za?X%a7EbIcI{QYLE2P`CBr4mSGO&T<AO_pMl zr(#>FTb{-0bn`J<|R3?X}O zB3DHNRXm%Nl2|dlr-BgYni%5YJ7A4abV4b+ld~lnS)z`qnHfctGJAEgKrq@C9Y-lX zSl{Au&`Z(#Uh$$cRW)ODvwrt8*VviB9_5tfenT8Vvr?E5Blg{nCHC=4sSo?2<<+!a zC-e!VKiBF>ZaF(RyCGH6Nmlk2d~X%|ZOkFleGwI!yqDvM^=SV#!lKK&ftx!egF*!d zW16yV31$$muO5_oxrCOYw5r%|vP@vNaxX4<&7BiM%(#T7c27V8 zUj__}+-Ts9_Uvv{hp5iui2s82#+npreQ_ssy8ljGpz!}T(*Nh3^a#=g|Eordo%ZGP z2+_KDI|}sv$28DtbR$?kw;D>jbj-hWvG!%|YKe;o%KEIM&Rc5b9@-(dz}m%55DnB4 z8{m;RHcJ&tmMU`E|4`0gC_5AzM?f{Oo$i#$wBH&db2AS4u>Bx6WfCbF=_RSDH2Xr{ zg~=#b*7`u8GUw%UO$`15CzFz(<_jI4tvltKbH~Sqmw`Wu=kPUTsBJnJ++y@?Mf!fP z`=QqNkjzYx9;%d{0Jj<&V_vv>lq zK5@o~+Pk})a>hCHzrtN$flxF-I`zE~MtGg0?VTk-=ixmU%fx7b5fnmjC z>+EuP+=TIXcpSVAq^Uu7MSZp?qMJmr(iY;E38=5BYk!J^W5hTtt2F0^+~!%i8brg} zCSpq?9Cxb6-5e+h>6=+w(4He(-muTqFbZ_>|HZ#lxAkJ_dOE7{%DN*tg2;l4#XZjK z!@12bG4gs#QMbbE?R+fJ$knrU7bnD9V4PB)^Y^I!PkB_j z0_mckBMW7U!J2{qxR4`sJ?7TC`A;O{AY z^($RXq=NCY12hF9$P16{=?|zU1hzcn#78gfEpC&26ejHQ_NW(U?B1;Z;J9728@JaN zn>%S9NM6u0W2!W5T zz8r(Dp=-|t+AwEN#;DPpl7G8a^v|$QVj1$xPlzZr2%s&79!&!K3*GX%eSRL<(-6_}&!OK->75OJDYM4`u5s+< zPQ--{=VCJITceMpjvq_@x*|%C_+=EV;Ni=uC*sum zc&y|0IZ!O-Ksn`Zqy^e^q$eO$Me>!_&u8i#S$LKI-p#-Q9Y;fe_>}bWhfs0~y9T@= zu`Df4ET)P~CztH?=A`!Vi853}yxu_F&0UUK4^N;^U0EF^`O1Te(XV4>qnj)2 z+X30-jdZmLxVL8Py?HY6+MeMN{LtFjbx2@?&)An(t=nMr6DSyMY#dAcVqv=2<$?C; zR+2FDXA0YMWlaRw=&J!--D$~u$`%Fmbac~u9sjoF&0+cvzn8A(QJCAjZc@j8-J}X| z9&O+9R>V}mecdPjQ&b`vzKp*RB0c`q(JTU)vFfOw;U904UkMy(X8y!6{RuJG@k8ws zJ7L*n&LBT;xqOeRGHw09Tsg}6>jB)r}|)tOC<#o z(C^Lu<{G`U`KkNEqp`gn`uR)GE6$qu$woqd3e;d^`5N8pkM)-rxk?USujN1Jn>wHe z&|efrb^n_5tk<(f44N5!s=1xbQ17WcB;V4dT8pc9?zuz;sUb7-gw?M`(4EOQ6eozB zHm024f5wRZZSQm^AL)WeT8jS3mQC3OUtB7|h=74=hN-{SR+vEl6O#w-`-eyGy_fFs z5%r$&%5PBHG4wx;>ZzAE5t5O?_<@oz2_2d}40sCB}&+zcaY6AeGN z4yYAIo~{L>d!sM_U#`|-K6`Ksijr=3u(KM7&^t1HbaG*!XarmM5#)CrBN;txt1R4Z zkANQ98$tgp_uy5@CZWwCw?;Yt{#=5W6r~boX`e{cUT#bI&hk7`XtDxR@FH?9>{YY0NOBY%8A#XLlcn`K9mFzud zZq8X&(`&fY2i@q5sS95Afc_g=FKXruRpkwBm`}DUq#eQL`u;b?+R)cCZ!RT0d$8{> zZgicvUKm=`Cs4IN`?$Cd?(^2Au36!97&#tYMy2xbVU9`+hw+G4T+HFjrXiP}8o7PQ ztGEJFt%ig{i33=@opd!lg{PGbS#$BjoHxf=PJMd@hU!CIuGTX_>ND@XLUo9~`Ku8I zkEcYb)ARj6KDrkt$AR33x5wR%BG4xTJp%!!lA!8!pm|iU6ADQk-vi{N$+ z)=Dj8*0vp+=V(cao#y4=dWfn`SsnM=)!}K8kbNI(WW0wzz6^K|{H;lTlB8K>=auVD z{WL$uUYntjZnxLZND7_o4dvfWi%G3E52*jwf*--vyg@)rcAP*SwKz=S3Aa@d6Uoc} zbj2c;i^H-Nd(VwJpR*2XGu^-9iN;HN=GLdXtszGHNZrSQAZ$R>RvQUm7go%E9CiFC z3Zn(=j}IkD(#L*eEpJTCS|WYh5)5Y&4+h}`muFDU=LcHQ@|~;R&Lg?Ns&Fyjn(hk4 zR}I&@Fv6q;r&2?H?V{VDW50k_enLk0Md)hTRiW!9=I_KK_qgXXM&;X(co!SCz;)T8 zDY;%(GV@ZHD043nKB6zUr>6nLd98m*j@p{=4d?K;W{zkJ96QifHzxRU zUKn#ehw!tXxL-wkUFqRQ{voJe<0V?7t13tgpHB#WPb)m={cgcShx?W0*BCYv`hVFzwz~~ zAJJ@hxBY~7G4)!JHiVG#sp9*y%?Bfnzr9!ngCN3{cSzrS#2LdEM{1B?&gv!@_Oi2E zy_I@eqn&2t!pj5t*xjSpAsmh8Lt-@DUWPvKL-nrMInyUbqNpDj@t!RFq~7WX!mKd1 zlP*=XV?V_LOq^8~WgW#$MiM>wQ354slbvaW>_QW_ydbf&EKJTEAQ8n6ez&QeGbxIK ztxWwLOEt?l{E%eheT@1a4^sb?Sv?&|YV68Nu)94%2vyeBwo<+!i$^a(sY34K zSlfscfGDKZ(w6ucbTyLKwa}TKp?}0UO*Q#mlhm->aviZge# z?Z;qFY;{=%abnB|fh#}TEs_^iPVW3LVh#@QJ3mr2_PqGKDT23AuisV?QVUH|3D#9j z4zNX$>S=>eTq~;vXq$C5E^nmbN1D1Kse~nxzeI4SXXsjLHeg|oRMr)|Ono1GFbSHifmGIY=fko5hO3n3mICG9_1CPq6AoTR(057dU1*U4VTsn zYG`>QF~L;jQv_skAMCr}hbW*?*$hao~iz<<|wK(k;?eXB{oH1 z1$)BOdpu`V7hlFs*SPn%9}~dA4fEZ{1~$l$c^L%LHi0Q(LR}>|K1ATH5lxkLepcK? zGxo^qxg_>ISxpm)S1Rnb_t!!y_i)x(aQFYvfB$Ez7Qq1_kX?a;oLW#EoQ$O#C+(8{ z8A0%}d}>B;W$Tgcnp^(4Eun{$&EJl7-IjBG67=5nY`626giAYwrhhIKQ$w7&Z9{K^ zso9#@|LOg@Lj(uM7fmOqo#3CTKfRzdhlqx~>CEX*uh~6Y=(k@9c^^w9MY?gkL#%nt z`Tdw+QJG4DOui4Ry*!q{%M@F94GE?6x#fK~!>p~Co%Qdm1es#G z^>y8WiUX0^P4LvlqxN+FkmE3ZGzpuXJ2hWsPB`zqf0g`mAJ+2oCHyqVityygh^FiD z=56r#4$;Z25n+hbN=&$I-0hGWL@+n`$FE{=PSRh1O@McbTXq4fQw(X~MvjMqae9}( zJGRl>bW)UMmYs&eR3$wkSGg&npxfKME@5KjnV_@Di>LGaH#Xq5GMRo~c+-y>+Ct`3 z$&JhvHCMDA3P*5HZAzkoR@j1%$7mTFDhTnixB5&Vd6OGZ*B}s^^HG7MQI6G==%?Ie zANfX6TX58W#HrE6s@NYD+aI6g=nE{Hi(sw(Id= z?Dh!%q{kXxDlFkN0GR)>iDDPHQLLBBLl5&fdtJ-)cVqj2K+@wf7y7pec3!ef)9>da z86Fa{g=X{&to0tD$q9kU-fks_#9f~F;_R(ovvNQ7$bv$ig!_XI{$w5K*42u~`WQb8 z25!Dd-oo|~kGymVAV5q_q0f2fdvM!4?&|I`c4N$WhwaxKpJ@2S+jcb7n?j~MBGS13 z!j8a|-MhaT;ekTJ&?!xy8M;v-)gKh7VZBFVVRbtQZpxDYooV^ zvjN2rS03C%jK@_fgmKn4yS7D&5 zr*WUE?ui#wWY(Kn9QBxJq1GFJb@7bFboog;*BT^oW9l{?^DHGEu!Gs%m{5Bf&)iKq z5>~0T5U``wURzC$Uq#U?)m4KgRhxq2n5rebH#1pJ%Zp zKkDr5?W~ON7RA;(-Rh~qgV#({7nSPuu;n)Qd2{7_eUe?dDFK=+K>=;m>Spuz$Y6N| z$kT8(O?&$v7`E=-`Qr}eQAaC2kI9$*WbPng@tSZWU-tStvI=inwXqjGq2RNhzBb2N z?fxI|O^laOj2eMZZxNFYcf!1`dVG8np-}bqQml2A;y0|D+r7O0m9l`*XS}=?D|)ao zk4bwGKB9TRwa_-uzt)I$)YTf<;h16Z;-zx-w-rzFI*-zbgTqbrK{2B~yO-}H%SYcI za6V^YInek6^W(!#Mgl@tu1l_5b~7s3Fy8hfeXq)ng?MJa?e|I_|JW8!S7!O%jUfN- z1O@8HfA;f@smx^%t)luQK8GICgdr}BBK#1SZnRNTRd93f;WK`;nIe~76js&rcyFtz zYhTH->43GENl(n&T5RH{uzc6`IpG3e$2UP^(59 z=X7gR64V^r9WByy(!vJ8m_hPmT~xPOK|W_@$EbuJ?w zM6kviamaPnUN<{V_QOI=0gPi~B0J)CYFf-~=vUj>tgT6ROvn|#)51CXoU0$6@KGR* zuEdj}QI|D-b;uD#`{6WsQH|Ms@(Rgxc&YA zbN+Lj$)LX({{N*@{vXaK(-EiaKe!48W(=}n6nscbd>{s5d1>%fpp2rK$wl~wU69{I zFd>)Zzs1XqTWhAbl8;(asY28~iX6ai9z~Yiv~_Eay455b5N^;AI`&UvgQ*`4MEITU?;nU`59DH_M-MVYfZx-t2 zV3opTwfn7Kp$`XLeLu-NiG56-)sCPQVNZ_foBfWC(98N?8f;9Hx(@%kbV#HJd7&C=)G$1y8WeD=wAHI*6u`Zlxt3asF11QKy~E{ ziP-$!`LxP}+`K8vN@-yHGa|~SP_h)9bHT14*&T-x32xjf_z)$zxb}kcJFDoPV{8r{+5OXC(AgKgU-sm4&iAg$}eUDj#bZTL~Uyd-f+e|A<=v@!N%8>d;R2@vC_wbFp zcJ~D4cJiSa9&Q*M-l%@`JA2Qhzfrv_bfn9JCmqC9K9X}o)w`MhDcbGCa$;k>`E+2% z(6sx*vJd<|MrbqKbsd5o?1?VqOZz(f8CfFi*P}ei;1BxLaF}nMiuPkx={gd<* z7u~W^ShA@fzYFyA{mRwkH1zuO7!;9aFXw5}yw7apH_xav`N!ad!O*F2qQtTt`^{el z%lW$6w~{8WniDYzhi)YlaV~sGY$vZEe9_kjH zvBgM4+15H6>yYz)z_a#q24l;A>QlV|>Ylam7=aEzo$xnO z5mZ_l)pYRmp4LH+z-om;ow&)KlxMe)fC@6*xsJ+T_Q3Ualchl|Mnse zd}aTmuNtR2UhDQM3%1!ma$mbwA<$$x-(R}^@g29J!g#4;1LuCTb7_))6Ta<_XC84s zH_Ct-gV2DQd+ggCA^%(?s};P|MV3b}Ag(ri=PQ9qA0@CR{$~5xRs4J+gyH0GuCDIc zYQls}$P*6egpOgkw^UsWbE5kF=IjVj_rUD!RLtwTs+peDi}C`ctF_S=$f~J0=0lCD zPXCfI|+z9n> zv7}1u39XNmQ$RZJ;L|Nn8zjO@nx_?Px~^`RGwL!3sO-Goy|T50N+D6r&Uo9Wpy&fE ztTw~uC4X;4NSp7-j4oYK1Xu1!L!X03E5&2|0t{g8nl-HNpSqsElj7j+%3Zs$p z2Q!?QULs*REY+4BV95(cQ zHd6mGjjpUErrMajM|7!!ZK=e%NJb;LGUXn}(KQ?PFt93zke1^Q^K-CkXhJ7!r9Yzq z3hR_bz#-%{7}Z6?@#X0l$?m8XLT#k&UTF+weL7 zRI{A@JND5Q(0@e)aJ*u;?R5sHWIu zb0ILaw=z=3ZT8VT@Rx`1H*EYyw%(SX+twN^rD8tTWNsNG8F_elN$p^jf!5)t%oIUa!rpNF{ttXTb_#AP{?l`%G#KJ)1AQF9AL0Mbr zE!A}5@EeiWo%i;{c3LQgMmCy8CYvV4P}8ji-c@)~-ebphZ5}Z670Rxz7@dX&REN}E6m(pH}9^2LIiEVaQ?Lg@*W7eCW0hz=)^@?5dlaB`yx(VV2}OO#5lanzrf+O%CbanMlUTf{QS05OtS%BudV|xo+eBkulG=c-kvYr zed%PvTril;xo*1n-3>Dxn;Sq2xiwR|H9bnDm{_xQ4};x7OyhQ3$d_&0fP}&!#hRkv z+R>fG=mZcwz_MA+b5me8Pw%uc3mLR-?kOoLU}wXq222GJ^LA`GYYQMHP>CW2MOhQf zpvsBF3-E;}^nKyAT}Dd#L<`H#MLb5wgzVe^Ki|quf~A}l@Wk{Vy)tF!mjA|yZpK@e z4&@gb>B81=mqq;d7z}?$`gj@1CjuK#NYhLL_XwdpZJ2#QZobHig9H`;qHp1j>Fq;W z5T2UXC;pw0tSa1ttBd`vfZHn0Af5P!zOKR{8m!du^4rJ^by7}k1>0Tf%9k1pf~^1 z>dz-fF#_%Op0h>qtB{nu??H}=jPL0?hFWDlr9C|5un<;ws6ybKy>+^8!T-F3iwYM# z+o@3|r;yS+p=jOcA?thvw=FYFtYh_zAR0Bs$dRa*uppkc+GY^`DQ7Ejun0A`DUSW` zW}0aAPhkD2m_3>fBy~wQ>SuRYeillyLqsZo1xu76K2qEZbRpjLF7MGWaF>X-y?P5u zVl}!z{hMdR`c?!57ekrM2y%XA_k|7+tV;Zh`C#H*P$&@9tl}!en-=XELcqwRybp{q z{o`gm979gqg83^SN)gg=XcNocA0#dy5_SlDiSOf|277Aan3AqVrU_uV@L?o}5FNQl z4Fm*7;Wj@}Dq_hCkp0lO8q_8WX`2h#TzJ3r&1TMAG{|EE7JwB%1}w$s6oU(hpxZ6xAa6S{%x7+M%ZF&<-5% zcr#@BFz09i@tASi$U;8|(A|E#wd5t{Id+lr8`G>>TuQ`=0B#`>z|ydCr{ZjryU_Sm zgg1Lqy`T?^y7YsXw=zBz?Px%Xsx19NaNg$WddnTWBsCDnK5yE|gQ>3}?^Lv0n5#!QP-!Sz8+ zPXdA?7**#UU&5K*NT+>J3=H4GeMi_kh_SnYe|}!J5CDw&^i%J4PB&$nsojF0NNQj} zn+UE^Lc+vMCA;;7FF+ARXAY%OYB>RC-4_(Z23-lhP2;p;ZGpOZqbVOiEX{+&XxKt< zD^f!XnWX10XrGAB9_Jr4HPNf?uj!_)0(S#-{(+|Caf^HXpu7FntW`(Sng|?Nt2OR? z*>zJdPF9P*aCk9g$H=H?&)j^+JWac)css$y2na~->ETSAdTTB`0JnIt`ku!<&}(33 z&wpmO_~Q!(EUlt-+N~;@t}J-4mG)~RZq@ktGb1(Ur^c@(=4R`}X%|bX4{G~M(<_vA zCKAXV5NC97!zw3fZSeHBlq`7bv>M7^8pkW%14QJ%vu1YxJHU4Q#0$p&wLuMe{!5dla4KfvZR?VIj%0k2tyfhhtV3 zx2H2m7GA@}}LRxr~#KB^6 z+rcK)n^P+9O@_x^2vc-DkwdN|f}#>F~wVl;z{KJXSEhdQvUm`V9+a_L+S`T%> zO1Ho(z=C9&N%oX#zLSaA!^EJQs{P9c^15yH4S0>W+2ILg;IGnX5eVBdibSm}@roauTr&t7j%vGj27SytTW zF1%dW32-Kvdu=vOUU@!nzWTZ=a22%M{J096B4#Y|A1d#N+dVO=${97*17n zV(%g^a#3vkEFH~c80b96oWh2EpHfm+u<7~+5j=g>#$<=U14 zBFf(AC&A;taMAD)23JKBRyF4bAD42ZoLoKH| z6!R=181bEDzYC<18N($L@ye9& z?auBQ&U}kg8aS;m2%&n;ha^^ln9acc(+JGbYM1V!Y@Mny)M1Cv{xMix!38h-0i=#( zfz-=cPos@IYRt>mX3QT$87GxUzLjWzxhJsw%9>GUw!w0&$v^SJmVq&)Tgez!27Z-+ z`USeR$dGny536_BxMR+jK7+(m*hJ9Gf%(jXfdj*EzPG%%JRR$?C5_z|u6Vr)_B}~} zBXvEIEVZZx_(p8$c3KW{V+_;dFgftPfnz+@9mlm`fTiC<-{eeM=UqA;tP=hJ1oF}cYowDjh zJ*|Bla8Lt2I!O#GvgxU0Ie#$2n$Q*WPNEgbPskRnExTtOMi?elohT)crrUD#oe(pWzetS{cafAPVD*OMZ?&mgo z&AFLbKWp-KC0a!hwD4r7f=_k(w3@GTATcX(n|%~NaurkJO}zpP46;Fy7c-CUnDItc6rJC{ zIwV(%C=^R!+3ovjgqP2UnYt{km$B(l+H+dhQ&~26rnr$p^80(8FWt4Hz_^#0X<=mc zg;QYXnR1oi{RC^!89Ryk1T4A>^VVk94rp+sAX^ufs>mX(Qv_ze18a?0AOaxzrRD0( zD;}h35A|_5Sjn=P&)wE%tWoDi(e1 zj)aq_-?51|iE2WR#-dnm3AM#B;w=eD2IxqxdP)>is=u63^$XlcS*~LqLN7v+QrV0@U+bhuv8%(zyh2qa;6#BEzo z%m6|QyPsdT>8q>FLe|I%v)H zrXY?j)*T0r3pIu&{0ohAfp)hR22F*g6B8X3n-zwp6)Mb9$@49Z-_t2;nJbKAR#!Hu)I94)maZs6YdvtAdzL#6l&&G%joY%ey2#|VR;Lhe=tVZ30@23^T*41CTSZ#wm_9lbW+W1AxJn} z)>vAAyvcC35-hULii9)0MjJppM2DWvD&&*l;pYM1{)`STg$#8>AEs(hVH)NI>1?+p zGZi`$u&n{`U4tygM2blwETKpsI6zZ-IexYcV(x!_=xf#g&9}&cF|X2>TAjSAFXuw| z+;fp~)S!_pB=4EP$Q@&C{BELHbNCt5o)#5`YE9(EjGHYL**XveT6%pX7Ko$9*1bC# z_EJl?3Eg{#{5B3I<2&;Tbrv>d6>@Qx;Y9OeW>$9E4iDWO_<}yK^g^Wz3XUpEjjq#xHV zJ08Pn^cr#N#Ow@4k z3TH!c(=Fa#$SZATT9eO_wX9+_%e>PIHf5R%jdU!nA#o@*{(rP7Vc#xGdlez(K@3jA zRseSv3vMbFP)N*S)Vi6${fpd+AgYN(C{C`HRSdZC@TI3`QC7pWXzOQ*ZZ9u)ctJVA zjw)acaPfQHeA79kcWsI;8=GJAYQx$Ih^__KJM$6s?I4WxCnL5hA2_$15TD)w1wTZn zw@ovu@PQUlPzXUoY~>&Q4+%V!@Gz~n>U_H1y`LqlsENBy`ccAvw6R(RmA z4no+N-^;V03mFO40T2s$T#efS9ae#C1Y!*BD zGu=OuBbJHfrAMp<3t$60vTIfHdNY&ZQ@cXqbq>T->i(}nBS-0<14kbMQ07it`UiLp zH*~ZnGZ~jdM_kSHK>d;M)Fn7;<#9J-Ne!+N$lJjt$?<)$=QJS>1TmE5#6L|CDBGj> z6+o5Ky;WSsXSc@ts6~duhVK%P)~|SJEpwsy0ejlb2@|G`EF%zmNGD$Z8es}-C~*BcY}Zx<^|Q?XHGB(E3Hs%6n^lYk@-#1CFfL zpB{g82A<#KVE)&o9F~udV(d@n1I!2gC9ek$J0f1J?TMa$zLkMkdPq&bKS)J6V-Ie> zd@fbxw6{k`)azL-494O3MR}ngFl2aN8^FJ;RBt0Ez2kk%+X(Yb`7b|X}KC$fTjZ^Sw)b(ATrfz2U$c?(3 z^dvp@N3)x4nN(|4k{7YH%x71S)`{7ECALjsOp%Zw|dQ8MngpZ^j!4Ob*7etWw3TS|3K!^Qh+DTr}wJ5f#$Asa;|!^I4O_~-2d9VhHVm&A{<%;`xcFw{4iY~n(ox>2)$pWc)Ai| zVk!V@fjnqlG(Z^OA=76v$!*9H-DKVam zcridTzjh#Ao03Jq6A8%GKuiB9iJW!NCd0x zFOp?)G!M7SDPg{@U4rxzM;MCrp*drQu*y@l%(NvzKfp+VCHn4T=EHe2Yq?l-{m*~k z$O}G$*xnzdDVr*EtF*GBB@`LGXYA&z{(yIM*p^pL-w#{B_b|)M&oMIRBX#vOtUsM# zkIh{&*{Nvgn12<>rWpMIvXFf9LYX?pQ>Lk0M!)1W!$S%!Yx2q*qLi4BwikHbq0~f@3%vGd>U~b^}7X}&7;%UmRB5aUkIrpJ^Z%J zGX&l}i<|rpMm^#Pj~eKn_y9T)2^Rk(sQyWUZl1IyiL0Pp6ca!QlA~4m)N!8VlQo^B z^=ZPe7EnZFH7+TmQfpA4eK^0=98zNrRnjKvo1Dd9@qHmSo%mDhT1SnZA*%T-OjVzj zkn%H0y7P=`X)$KF!f_Gd(Gs7KQ{P+BDbJBxXIsbJ=YEpT1i2P;FpE4f2&D?q_ zb34Y3NsHR@3>g9#{dxe~u#>>T?v)$np zjl9g|rmP*xq(2aP8dgfT&j#HAQStE1X3Q}FRC<4rMK06!(4w#*?0OXvVPqADWO~o# zzAEk$03)saf$w(;V&0{UY*WG-U(<}$iBep~z6UFv4)29Z=c z;BD>9*3e6&$X#VGYql3tV3+Tnp9Dc&c1M&xeo6W2RJMj{#TRGZ6DWSYDKuK+*thk1 z15)$>K$w{*uuhUV#bT*(q&)g)C7dou3rnpC@-4FZnpI?s!_OCB=gUA7VoQZ}kZgXw zhNqfq7+bi>iSx%2Ks^LIWgvFjyXWBz5!PtM&g66DVq(VofZ10Pbq158p;^_8ApbV_ zIRNJ-qbth%K+Yp>2=z@6X&+bbNUZxMKfQ z6EUsKPui9>v{u7rXQ`nU_m=?T?C?K5Lx3MUW=?eE`=Tw4@RTc%xC3hO70GKwmQD3% zHvmaqtW@NiEh%`0xu!KXai48hGqo(ro$AIDk13f#^hH)?Hg}}{&*RNMMx&2r*kn4i z(&j%R>%IH9sX1sc7>1ZaqI=&eQhg_ljgbdEutd z0Q-lXy_BYZE%sqB^)8{9<6zoa9wh9rR*OYX-3y89@bPXge8BVOLwY|Eu?&b&Nj&#; zx)f%0c!fnZdQL?%)~jKP%J`1$QBnxcr|JbGp&~5R7{>SXm6VgY(c-rZ{-mcEq};_H z5Kd+Wf#XZ^(H0c?pSreR@)2k6Nb4k{yWBq%r|ufMiyLBhlv>J*M!{c+7eSoYPi%VK z9MMzw-6%bPAG2LfKqsy%cfn5&ox5)E|3S^)!psqsJi?)8LmS0jbPJMXVIzW!-4ARF zL4Ik^a*JT-CtEBIstytFTE?3!3nN;6euKC8@G>yb{zNxe>cThZd!G6WD zv6g&$JPg^xmUcXVxO5B5Q;9T4uj32Kw`U=owmlqK!#{R28L;#B{nSt!td>CdpvI6} zPA@{eo4;U3{L$X-Q|uRJL|m7WO<*w2M@`O&h&xHB-EEaB@T#MJ7=Ej9%ZjPSw=i_) zpDnVXeK=UR&!-fsT-(sA+Cb8$@udE;H- zrj$0UkGV+{B1GeRW7uwhPm%&|yAE1uJovEEd|~|PH571rcyR;Xi*G)kFL8zU#QX3) z^^=0_?naMQxAyxxnSQiuMTd7#vpRvINtX{lXM#yp67tNg;E+2GP30o9AhBKNnkdh^ zf3+OuUsM?5=?zHc+qDymR?#*8yh_t z?)6$+3oEUDp)i_HA$fDa*0eu$N!GXVnb%}-HPTRJjy@PMSR|;Er1|F)0Q(w+`5(33 z!>Y`-sSl61JB|t8s~RatLhqulO8^R)KvQv9Cp7{Z;-7Ps*)8U=9A04%f4tUZEpzIN zzxS?M`4f8EV7RH1nn+(tV>O?zqRdMuCXpJkKe8@IA9uDL`dmbaU-KY#V*12(QsIk@ zU+s%6EFnU^El%19$VnsepGtyu9DP`6+zMLxb%3(+>x6ivek+Jm@>Tm1YCAND=fv<@ z^hCdd!lxz(AD_#Qx>@R;DCgN}0VNAiUJ8EV$F8fdrba`!lG*{N)51ZqaUSIG3N()M zTSYO=In=6!AV8($B5uf>K#H#lw|<)xL`u&6{HVzwZLZO5QTQLgQN!Sf)Au_@jsl5n zL`x=O!XeqbPXV+LH~HESBt$(-Q{ntG4Q}aK5UgKZ$O_D=2LnwqDuxG~YR~Xp#mPhawG=xB41|*TPX0 zAzMm@VoK1CyX$~VD*X%FJam}bAsXZ4|7HPllBD>$$yn&7@oQW*^|VmRs$9%2&;D4x z8XI#VO7csB2aoxcW@F#&y?~@{@AS^=w*IRD2%>r&jMwX?{`AXM6$;N3gP?n&f~#QDJBTQ^qt{l;tEf4ZJ%cb^5}Qc5Y4)O?E`-qRQ56n(nk>$#$@7-I$+=HuH2QK z>%~xFtlJ-A62%fQ3&Ulth2L6G;gK&+YO=$@nk4zUD-i*-?daj@nE9eemQhRtl$2Wc zu4jK#@$oV+{<6OBJKrLd-nRpCfj9K*F`T22W~5wUfBG3Ju$Vc0-HDtOQsXkTiRZXV zGEvfi`PnqxK^ZL?pGf*PwcZ%0pcnt#zOmzK!{B1Jtn+ar!KTUV)lywf6DPy2h@qSs zFk7|M^d#BuuB4C-qk>_z+i}gg`G%p##6R9PZ0U^pZq=OK-KujIXVxtoXhKTZ~_ zk4cspOy(pCoMsOCrll5yQ+X##g6ne{B#Sk29a=S6smAgzD6*_u5-$utcBs2nVV z_m18vIbdfDVS|a$a2q7xAi9{D*vWvfd5m{gQ5zb_zV5TH(`fBv5PxM7 zo#w*t@xU2rH~!j!a?%kFN50bUBqpXLYRc;$<|AqPH>B}Qo?DTuy5js+=HJ4WW6R&wICp8IVGZR;+3WGLLSvMerxF zVTZ6H1NSLp`G7F%57fS8zNCYG?6ooIL>TK}DfWXf_XAsFbh3GBTQ#`+Shq@2ckjIp z=kjv+)&Ln<#!efancCf<()>qN5Dnw+jdS&ZOGimrKW8y4%4Ob~eb!yUMl($hbS#A7 z@Vyh|n@{sbUe{O@lZ{Numos)g1Y+>WmzuEGG*?h{!kQlPF*gU#Zvl;g&o=%*>w% z`MJCWOAcO>ouZK)Im0RrSVq~Z&GcUnmUG>uRt0T?ST;d@@Q4CM-pz~3v*VMBYLT<2 zWT}YN;`)2o6|?vx<>{#Xs2qHTEy_~!+!9vp&_Ih$TwT+;>3t2x7a7ZaN;I6HC3rmb zu!O)8UtIj3wgiE~-Jx)| z!rh(1-Q5azcdf+T-3oVicXuaoC|nCCk>T}P{bqXlO~3ibwQ}Xp&3kU#hcs>js?SG~Ol#>(}2*o%XIq-)ORj+2yiB3zTZAu`q0E$+&d<-mr( zw5N-vfc*!M{O0*;V>CxtK z<;%?>gfFB>GHmS!450~?q}mN(SeC0}d@g}gwZJNTu7s|t;t#}9_o;vE>|SfC{;tTV z2W3HmgUmDXdW1fHD&rJae?C!6&8!h)B+iVFL6jY$xw94N#`kEv+&Zn*eNMuuMmS>A znRTny&&sA>FE?{5cev%qv|Yv@-OTr7Jb3kOP0I@w-<^19Lm;fKOTlDTWAcmEy8fymj`c8a%7A?@pt#c3io^oi+-~HGeXShL_S$0TGT)ao#H2u* z%gT?7AViUs;o1{Q9B~}+ ze>2kG9r%?mi?aNu?N=L8TC8K8&y7J0tH{rcj1ywN3!k%I0g&_0QK#9MPKXCK^jN0D zkEaXWfDfvINEJ$-Q}rm;{==A7q5c(z%DU?t7OKpqZ`?}|ay|RJV?VQ(pC}3>SXGR0 z)~twhxLvZU@bXHGL`@>{0VH(b@rw#-zUj<=;WzKo_#7uMJxCqiFloGDgU^230@CY5 z_tA=#9;HZ+v{I#oRn#chlG@HPe>lzhJ?&=Nd`ADGgF%mILcf%JR~RVN5|#g?8p&Nv z%2xsoqLv#l6~Csw$D0vTDYPMSZMovxzfb0j;d6aY_Fnk`$|%fw^*O?QsI{|j*|guQ z@F3^_Ng&^mXjJg1c_CzwcXmaT-natncBOobF}Vlhnbtak~cDcMB=Zf zfqx-aY&WSrwvm7{H;)Ft&&M@exomeA3x-W@S&M8^m_6n{j*-a~d@#QuUeY_O@yy&L zm0G&N$-lig;`6jU&V2VYdYo*#dOU@d932!BZA*aT!?@_}y%ivK>irkm>9jLAyEg;M z6Sv+>p*OfU6F<%NS&VXG?|F!dQBEmiR-KNriC8p!(U0V9R60u_bNT0BagZW;&v?B; zUcbfyJ*mx#x2|mpa~oy>y60LE^m=eGI2!ifl$ALlX!@MruDIy2Ffon;43ZgC zKc5UU5-MlA*AXaJ^tc7Xz*~fG4M+xFUIN!>Lo$Hr1T6~laj6RZ004##ax1vR-yftP z6GvFVMYz$;@q{}Pf`mbi@y(hg98STUE3w50$(lKC{$vGRY6}AbWS)u5J|aq392=j= zb`=Q@XlX@Po>J`&y8sA-oLqu}5bKwc-GzodB^Tqg@muNR66xc^8G+JFfc|u-{$!B+ zWQhDU8x8}GI|F`G?jy%mkTR@H^0EvklvTyM4C4_bg<&LIteR~4s|2l6hS9QxigOw5 zT2uCyJvnbq_|~QA<$Qyd4~>9Z>*Dj_r4So&$w7Us%O)O7hr*a2ZG|yr3cbatvoyE; zHCRQ(6YFi(Oh07Z%&jSi87&9ks>6G*aL(qeszRt~?co7D4T7}sL7$#_*fNB_&yBFk z87=MwJtoA8M78I5@y8JaNg_86UtkE(ps{6~IQgGzrCLhbIVDu;MAY#PB&%$YhwPyb zn&()MBP+&>4?%8 zqdbZ;sE(O=V)T>cMUeL+GMHHFpQW;B@ibS7`}|M@$BzWZ!$shVjBp@OV}7l&yaMgd zJ9u&&ZCm&NwXK?u9Ur`4T~G6=ceuS*=T*J$d!Sof=l?>Z@c(2oaDNlRf=-_)NJQ=r zIau75YJre28JPV#c27}?sp)?tuTQFD5;#%{f~cQ}>M>zTjZdg_=Z|}h3_z#jZWK`z z$~{!ScweWnk0RQeM^wXCpR22{yyZES?VdxcGwwxCBNC2RJj%J7>Ps)sP?dBeR^DtS zJwQpln|Lr)AQ>wxb3kV$a-{Vb&Z}pDMKo?SN{W`8OXD<|92p46`zZEod7T905ygt2 z6O?Pz)26$(E(BadDsm{ti)Cd#<++e2$g zS@}-bw=iJ+N6b^ezqjn`wVQ+60RI17gz~>W;OwAA<9yxY-GN&x2MXPw{ejyM((JtH zVW%Uz{Sw@z(`?FHt=lA2d?Pn@tN=vD@mX;!^t5tp%TO+`T7N;6<-`N-<@WUQ!p^Nf z<}r%sy3#bM_qZ-diIZ!T5?aXK&ew?utA(XnK4VbQ`?1>i>=Oj_iRN(AXh+PH^(z~* z(x>JyB%2A27Zju_&N)YSFB{Nw?rvY(7UUmKKf{xVio3IdveX9+bHgA7^rUa1Q@@S>Qa z^jM+#iX_#Nz?r33Vw-MxSoeq3Goj`XlN;0KQ@t741BuJ4_WMo4e-`y_Lq3_ZgU^$n z(R|lFxQyeg955>Srku5N{D*AE0;^!hvAK5K>pKF&rmsCEJW6u|H!6On7-I@i3lHXNCMhyh4>9~7^ua8Qs8en?sS?l(DKu#V??`fF%9;P` zo-%EU4JI;6U-pO^xf=6Wh_rn-n_+foULUWjI@|4<+{>i%BwK6FX&ZUHPJU*^;Y`Y@ z?;$}k=dXcsDyC&RXQIi2y`GG8&Q!JSN+kW zPq|(F*_t`ql`*Q^8`e?SsySW0+bDC77t6S%JkWdI+j7$m-+r%ICw38-#wlq8%8l9y za-*`I_rH#Zd@SPBjFHN+6x~wSC+U??KrC8VkQ3qSZqtk+~g8duT z3@Q>~RF9jJT3nSo_1n4}sxesxy0IJo`1DUkZSkc=|8fVO*?&58f<3l zl`eA4u_=(U2UDBwRpvF?Pit~a@)+r~bG*MQ+~gE?AS*F!RR5H~PC=5~al6vjrk?4@ zl{yh1b3pUKMW}IQ0RFJ)v6bpxQ;?3M$MFGwLO5$ytD_sZEm}>T(I**|mulIWlx7ZF zGv-}A3x&8T&<^RPQe-MKXz8Q}{zUoy_N_>#WylyEo&hZ#w`8{ZsWgi1QWCiEC7LnI zOOV|O7g0NK^dhSRxiFI@hRLRIFT@ad#@Koe&vXp*aH|<67FU$Od}m!cvHBXB*T@SU z@}9B5)hroChENhbMGvE_=eh{sG-o~XygZkZyfkKj$(>vA?dKfv*?bgg%FH@Mj5Ot( zO#VWd(OP|JZ#mU#d9rrWP|R2`O{$Z(bCD5A)d5z1nzr@K7WeuB5uU<$iFE~A{?9`_ zA&PIQL=zzgY~e;4`E8Ca zxgo;3gEh>WQ>`OsXHvk1B(g=nq{3G6>C&Te#@U~NbIZDz{U8n_V(umunsVVbzPDm{D7 zWNM(y=#3@I+A!EJ1~_SjfxnX@jtsGg0m!z_iN)pEW-|2(vB#)-qJhN~<})k8ku07i zcb{wL)oMGbV+yzVobx`i$?x?*&yG9Y1A8Z7Q|U|2(cGg>W{hB;*hYQChW5A-je9g| z^9~!{RIeKDN@a!Ezl%8_->O1)o%)jc{FX%ytL7))?BB${KaGz6k3G@<5JNJue+eOy z|Kmjvyo~WhNJfz0CVmc)Yu1%UWmBEg8L0ajmYhf{48mf}(D#S0tOMdR=yU{KV%A)2 z$O&s_MImbOev5I27*j1+OLJq&a1aK`BL^>lHF`H5s%Vr6vK2=u3{Vd5sjJ?o5dM-u zlRu4UrqR=ZRHy*Ta#+tgPG{Q-Tsjbj4Vl;!_{puZFLP3b)tqin6(kJV#O1=JC60@O z770JYv_=>9a!H{n#$W$A9ob$UI8;Img}wnLlPqF28M!WMa@j zYYfGX<@bp~2mv^s-9yh$2lx%RZL2CIGJ}fc0S1)W9-eZSN*K>uKIuoHZU0S%j z^Blr<{Y=z%VGsBPOX^OjAv%B{eUxX+D4bH>Ybit7_I$V#0MO&?*%;w2*O!VOlA9$2CZUxNncRT zuj*yX6;XycVA?Z4w*F)sgRShhUBPY{GDVN~St2kO)gRi-eZ+T+QQ}>R7(Lxj6qj9L zw9Z-=bgAOipP#bY;-V7T1l73b?!B@=IXSy!)mWc51GV1=n_pb3KIhImj+1D@`66MK zyQo$>Nfyz=`i`iRTtp&-Q^~51Sa`O_>p#R=GxBOJra*jg4<5a_$5*~>hx9$YC;i_u zK~+=GZ?ETsk~8daF#pXiJ&Uu#KN9^u=-QhnY({<*qrg|Zdb{=MT z9rwL5DF=8sP2i!U2tnM99)Qjhvz~@>Yt3jt2_GR?s5Y+0{UIHG^qw>N+G(g5J9GKU zMl6?)ch;1?f7>p2b9dKc;4_l@Ij#L)tCaum@X=dCO!t-tm;wHN!9(%G8eykB+}~k9 zr*USGqG;E14|Q4YAC{a+Z>ATO3%Dp3zlJ}uH{3`z7OSO62&2@IBYuSAIVRjLw~3Z= z168Mo>=N4Ov5GD6V-^wZC2PL!m;5h~)S~+Gu z^e{z~$+?#=j7^vHBV~qDNV7x);lj!C7AykXbBipB!K}qKC7LdvpqQNy>{;RtA^o>J z$|nubYiC|_@56>$c4N{2;b0>`(R z;%aJppbh=3P8VxVvmF0N0&vHNUg2WrsryT$Do@Q>oJ$4MnGYZxKN8Jef1tTggUL9% zpmPduI1T*l0gl84m=k&52BgwIS>&n4gbc?$fQde3&k5{2WP_#-?p;6f+T%)jxjwT; zWB;2jMBn$e`0pnRV(pH;-gBM*a}~a7x@LuqTx}B~(Pe*rLe2KfCR;dlw>Snm-fZ1_ z26fK%Ks_y9Riz!^*Z)B_i^p}5(;mxI%uiN#m0X{mH+A> zn&|7Udu9bd4)0r7Fh62f&f_*2EB%BXG|_|0LpdV`D3FsJ=_g2~oWr{lf!u@P{Uuq= zaaOv@tTx0LtwIkU`X`rsL7!}n0X4upT7PVzBbzuxI1PqJ4C`LqJ4B0Px%$87Jmmjp zV2~Y@$MwbXqxtIQqljvLL4&tOv{;5wmSv8T-qRGSsi@HP+tU71lBdekK#r}p(k!{} zLODQPP_{kDMc-n1hLUuqJadMZYUHt8lYT6RHo#1^&85=3+h}7)&%4t^-otkGfHHme zYim~yTw8tc*l#v#x1V-Vxtyoav#xL{=sl>SYSfl(_sBU=&3hWU%eDu8ltc-NKsJh- zo~J}ucAFZ_5U1LJrn;0mJ562f-_NcZm!VxgKOS2_o?Dm9K=&4tY<1mEp8)jWEg5uV zhJHvB^hjy;RB3JvQO6Us=!fBW^C9C^OI`FB`Movolf7S;XR{W^cxf(MJwkT8hM??< z>Z9k3=m$Yf2XpV*a~A&!o+oXW@3(#xcBglD0-rew+*dP^!YH}-dR6DO$G`bB0u*g~ zzZYVx=Fye-fgFb0u8;k8bVm!lijR5k?l5L|T<||UuG4QlZ#@5_Jbk7kWYt>3$r8P;cm;(Bo;$5Pfe8x+JS5XJWuxxW9gB)Cn@@jSUO4ZG9|dmu5VQIGw&u0bphD;e^=?Yi%>*s1 z3Kb;n@W-QT#Ny#So6UNnYC^0P95B^W{u=5iwB&p~I)xgQc|iZ#$#E-5Q$Lu#~9 zTpRt~Di4=0m6AKrQ{k9GlGBeWs-$dt3uR+lZ!H%R*8?JPh6DE} zO*V&fPNj<1(r@bt*tURK(K;+b9v^bp3ZFSH`w!bbKAa!=tBvD04_Vlmz^V&~d+Xt; zuA}fuT(tzf?UJmWXdHpMgkW9ideb!|^c%5goKoydf?14=Ql7f3jBUG%tKVMwbuSC( z4nDucm_v;2UW+*0nM!$c$A$0<4cyj!vcs0`gw}o7e!3!cVrEu0>?NG`W9=_@jf5L3 z#|kG~`YBk*NztQ-Bm`e9muIb_FCUehj~>SE|1wbgLlC;w9!nHmcaFxr-Yf~sTdxS-HFP1EI$!an8SU^th=y}9U_V~I zsy`((jqD#!yE^aNT*Z58^y8QvB)4fYtCa%<9`7N2lLVo2v zQ><^t(|VURda@n1owr6+iR5Q!rR(x>f-enOP|;l>T5{IqadY+zrCSH6b$qhUrDI`lGZM6|3(Oeh zduhAp?|Etc9+!J3_N3+B+HWnpv$_p>*zWQ0`O1?%*>_JLzusE}8%m2?9aRDZ8ibQFRTQ$e^pgS%zk$Tnwt-DMP&D^(+3!uUI)9Cu`cB`RzR<%uX ztVHi}HlDzk^vGfI8f=t(Qaa9@d}wmWWV=6J(+ z-Oi3RSg>{AB~!*_2_ox}17>E{2{%jno%GlJ%(!i>wsXwfgHrwZsMeimbYpm+n}i;s z{L@r?r@Zoddc6>36F3h8gN=TBx$W;h8bJW=c> zjNMvC?y?h0IY;NriVale2J21&NT~O2#S6+Zq%_HKzlW18x`_ScARt( zNxktW>u@rF-ThmFJ=sExD8oqh7snf(6J0ZgLQhR>J^vX>`?EX%^IIfS#Dbbw2H@8j z-|%o1l)`nVgQX4-Lx{cP;^@5nW|cNIDc^3SEzhv-Vx(*wYLY~lpSeEUK0foZl}ImN z^DN3s@J{5Ld^qWDgs|&ryst8q)`su8qjkF@Jgs}QYcH|;AWSPZKrD_-jM|uI+{Api zyw{J};UW{oY>s2erNGmJ>09(7t$*Lz1C0;ZQ1LG@>=BW8XFdnsq#NUoom20G5c==| z$T@e4BH2h{!Two;OK5>z)L*B|(kGNxQUoO-_BWO69@6YM31GzH9YU}!dH>-k=HF7; z@y~BZ=%&J8xzjDc^Wk;-3K)Zg8xK%e^SVx1oVgXJOca4Wgto2QYN?k`RZ1NrgO#Qs ziLhkm>?aY1#77koM@zFB*4gGdihR;qs^b!uvyAJidxD(=P%7ydsVIv4P>gpG6O~HhFy8U#V3G- zJz!UgsW;6`zGJjy&M@eO)iA&d62hOX{j~N)`ON+01<@+I5jkoyb=OltK?=dbAt;M4 z6ALhD$PMr3E-;8y5?h+-oS!}fZ=JOl+Xv3!?==UtOaYGztQFU?e z!hA?Q6-(vo(41dbFG|3u-KpoG-_ZSApYz(RH<8mNKDIzudM;tZ&2j%b;rVf!Mm(+& zL0O0p3eSfzNO0qT_|kpH|H`(oEHx=))njhfoJg5GN)2MtYICfqzS{8Y~~ z`owRc?mB40P!u74;oDFZJa0CUIN@V)L)W+O zp(u-$f$xG69^~YK=FoF?5QmWzeDmN^l%%)sVZi0Xr|*pj(%qqmJKtUm`Yd!>g3>jD zub9y)g4(o2qOSw(zlEv-tlB0-X!+pDW9*wU0E&!dw4h5V7UgtM@wk4Zk3R~DC|>N*d#tG6d#HuoD@Qy+g^l+0y@lxyNe`Mn?^!8l z9KT3fR$_AtS`M7U_On3R1A+#o?un72_J$TsK8rsO1Y8qu&VzH3Jok2b99D@9R&Q%0 zjPs90P~(>nq+H0h>B#$Mr@iL@P(NEpljLZFC-SM3M&jXWf`vo0Hp0wrT%e?TVW*2X zIOEbYB$j@Vjpy+sU@y%ZAE7qijXh%(!Q$mmMZ3Wv!;$_QbF6Peja6zFQBoHBsfG(l z_m-}U+f@+y;M_*FUl-X74_hKH)msw(``?K(uf)oo6@RLQ6)*Xwc zn$%1~;(hWi`a8IahC+={@p1edyc8NTq+3xmge?SlX&^FR6eLy;#Kroxm}3(oOHe83fH*09* z)EFl$qNcsEO*`>-9MR|mm5@nbTgCfr;R>zMxay72=h11X)Z8RXi7th#dLka8*U{Bi zd0?;xow{OEbO-?H3_2%LGxL5=imtTtKRUT!rEpGb)VO?*4P7X<)Cxxba%p!fF>^*S z0{C-PM2Q8riDV54yag{j-WWj?2jb;kOYJD?_;*id8*~s80*&BNmk!?HP?SyW0RAoJ z1rF(>-J%~LjO9E2>u?~dy6wO;0);^8oq($@RIGYuSYEqE-^m=uY?hP9iSwHsP13l( zMy!SP7V|yh-&A4QfbA#nVH{v%sDe=3V^tI&E2Rjzg~ynedKal%u{@PQRDg;R^OwoS zLWj|bnh*;MS=3CQ*wUmlqV#?^eyrGpRlWcX>3;b~@p_M3hYJM1ajeJy9%a%#V0FGP z;holUq&|#=n9A=46s@T6$K|3L;o|t(lc+0=APaXV0kObA?6pAHH2BIF#l+m;6=qL( zq?YVbo#Hll94{w>pp6PuE?uUbn3{VWlJ%#{6pjaSCG3C_8&E0f6`R!gTfLmYYmx#g zL!Cx>o_h5KRU3XFy>L$egPi%7~G4>pZNaSxd z-q^}tj4SSygRtH7%Q5$qzv0JzmQ$fZ7>_FjUPw?&0)LBYN<*(H1z7AD~H8Px=U)-?J20!~_THE79Lt1bW4ovu#rM%O;#)cLyJ%T(WIj z3p2LFPP17h2FG9EJgjOelKT|q(n-k%;k?jV?F&| z9kOtdr^O*`QI?M#1Wmgb3mtfty!P#SqHRar^Z;xgHsOJ}vl|6Y9{f)MCr|4u##P8X zIHmAER>@5n_n3f^PhRrFLI0@3Vxq*hd3urYP(%#|4jsp3y8t(wPPjvqW&Eyi1xNL@ zaj>y)7Fmu}B4mBcc{k~?bo zc?kpp;(x^B*KFK~)PgY7UC7rI)#t|)`PON5g}#P;UY`1+ohw!71RFi$zi9SE;aJ#& zCzwc*n8*^Q|17$oJqU;spr9=#jZ-SzM+OJ)CRXsUAuKYG8XwPD!AL2i$vjxrL?-xy zU%Dg14UFXnyF*l(AQ38H=Rmey<#+#}P84&T;-GFwMi*0+v&_rz!opou(`p?LM$W$pa(Jt62;a6 zTtmKYE1Ed?*zr4{Qmnte0|l*?K0ccx!>aZ@Xa73hV!HlOECMd>G5$<2bmthde8|J) z^Nfh~YKh~Y|Bp5mNj>Ve3i8t<)LAUYjspbYNo%)XF?jTau&PWyhP$NCC!o_wP;Uj4 z=R!yMb07Ndj`0hCa~2Cqb(j#11pz`Ev(|Eo9oiHElJflcu2f1-SuI(t;L{Q2$_vMH z2@lMy4#vN>#U+)5qcL{XIOm=EaFG~XklF33*6u1l64GYX=q#fBgACOOMtxyeMCGy+ zkkn*D7~yzOM;h&*enw?ZN^UeKxI4qwX>UQ4^Jo)2$y0O*mv2cp@st&*Ipzqn8{jLI^)P4B!U;gb*14Cq-&K zn0*uIkBLMnt|``0H{#YOAMg=!^;>!u2RGcQ)K*uT73Q;63n#_*)faY@W~W1bSqHlv8d9h%sq&>a9Jj`aG< zZ}okbUPNlSY+s4=-&gnS_J!umMf4+>g3giQ>CC2|$wGCAQf(4PyFpz@Ae3+>%*=kxR6x?m&SZX7h02N%Qh z)6E>h^%uXiQ6Ywc_z#jqMp*XP^S^}}1gdUt`ZamG+~qGM0DG?UaMVNA{$NV9lf*uj zNJbQUiR2N2AAek-jWVG4sD;zveDt1?*rblPjgF;rbWtxLkii z@=N7ZE_v}H{ zTuca4@^@hQ8@D%jszK@k!&TsFBsQQS@Iz z*v~c=)PKw$pG{D)C^>p~to0b2OHWV)$a2|)&eLKrzY8#QtIsu!Za~hTVcW+CFLKN> zG-Ggj8zm#@Th~tLG{Ot6@s0wn-b5zr1Y`;%K(q}sTq=R$3Mm_&^gmbhq$cRuH2G;` zm}nfb;je0wbdwzlmEp)b7Ag@EAk7MC3GLB-K0pfqQ>_bIJZMU%5Hh&HL?j>FuC5%= zEc?eYR<+vrM+h6ZSSY-OazKiib9Ve7vUk2RZRR+gjUYLU{$r(er}PeLhzkZ7;u4=7 zk#4dwza(82EVp)Va=z9wGgZ?*4-(WUU=kqJw3M>p^sl{4jogH5de(Y?$&?N(WkQ7| zC&c`rRt`}NM=uP_Npu1`X3Ka0A(~+;mWoiZ< z(xNl60cab};1&9~8!B`6){Wgz5;Ksvkzl7%fw4A62m03musm90Zx-^JSQ!gTE~aZ69w=>fiwx_;U4EF zuALMj`HQopqRn;JrCQ3L@VtLoo|#=z+c}A7XlZ(gd}GbCg^Z z3qmB9DNo+Jg)WxB_h%W1s09UN(gx^70AU7{R6>kyDcMPmO^SmQd$^`R&LdLnVpz zQ~D9P+0hN~2i4?tXpbRE3|g=lI6%xcH7r;IVh$Fg5<34c3HOnE!CBgQ!485@Kqv3x z*bYK(?aD+2ZwV4zF3JaJ_4+{Q_}Q@c^M)Um|3zDvMSVoR)A~cu7;C=VD_qKi@JYf_ zZrI7iD)?EX(ppC@tirgZyP<-p3uY8%M$B=u91F6>JyR~3CUPPDCjB`3-^oVbZQ#Tn zFPQ-odoSr<^fI}looB`=?n9qAE9kmILgpXXOASJ4GZ+a}$MHi8GvQdwrWzJ^Ip62zH=eM`Pht@B^*G6Yd=VW%ukYTbp-Z$@_X75VMrP1)Y56U|7Ex6HQ^JT;D8bOXD5 zR7soBiGik6vMdP+$Sl6tVb@i@l8Hr>Jy|j@p^nkE)nTzh)w9)}|6c$!bR;m7PqnKM zDi($+T`e9lXAKTN@)PdeK#PNmnzGlbp~~&uIyiyw2;}}5h72)!mr)noY_Zsmw{Zyu zCu+(R7QNSRUlC{84j-DOP0^q)#p^}^SMX-UUQ`B^35mq_v^hW<>c6f`8{NkaNtEM?<(AH3=ehJHZ!G+oGB0Tzg+n~TT#CW-|a}0<#V(2dPgb;PtsWB2&nukqg>`VrX>amZ+Z0lBjP%Eu7 zRyA7i>CeFl$X6J_K6}SIcV^O=y5-Q78!35BY*+gyO1NFR)j+O}(Szv7!A&=x4cR3m~T3D~7HC7M}fT-0`GTb}zkz z?sEO+)vCCm{wVR6JZ1kUQj0;LjVJSz% z-Lih%zfAS-DRlTNBX1+}Mg#ay^C-&$?!VEZQ>UriyWMHL^BOq&8bS*i`m`A*iMxXSPSC;;A{9LH=l24a*6b8c`n|+fr8pYeI zsxOa7Y=9;|!NxZhm4qTm5uM%@EQdhoC`#?7QS+k$&=^g;UhG^Npx2adr1~n7+8+E7 zX8khbxIS`2lW$R5h#7h^!20f} zF6v!kN+LDD%|{R>NdbXrGG^avGXUd<5IJAwqJc`BOE5rnlt&-0oj<(W<`1O9~DszL2)_$bjlVFsVOo%+u?*A2)!DC=QMD)AqKABik3Wlz~bAR=!9 z{FaM>7D7Wx40;;7+2TEtEVJb-RB28Mw&fru+%N3(Cbp6@&Q zPT6CgB-}*hn)6mPW`ZE>gYZD!({g7Bxq*|WPUZ#t-;Z+NQ`O@W(XD!B*Zdm7|E$(X zN2GZ2Mj?`xj{`_5{Cp@#<7QdppAzQj#iz)AdKWr2%>U*LU^j7x3e8D5#9E;#?cmvS zlbtpOzbdIP%X|C*5m)HT#RdAZF&M&VIzJzdEe}MYy=UJM$2u-;(4i zFq-;KcMo-NX&|W?96tM;?h9!dSwluDxruaW-iQm zx${C6Fe1|}ydh;3BiMl`Y&T=)z`_85`rQp&k-nFqN&{P=HX*@To$lqpSYAGFy1Wsq> z19DBqC%t3-h%)xGPu5v++08|E1@9RkQLFpRyG{SP*JHyf{R_8W#$D=M5-e8&yI2+V zeqXLZbeSTE$n>};UW-rt^3L7Pj7@zkSr}uTj!4Zr$t0S)RKNGv+HzPzFHd6pMcpRq>ecsz5B zd_Onq#A+u7_)&oev>;F)`7jThi^-jfBv#n2q?y|pZm?lcWdL$1xp^7-zXy!^fBatM z!qqj7E`K25=Y4+aqV^>UnY`^T3%!(toxvOGC?G(>{nnIFg4MOAyx6dMfvyRr5O~*N zSB<=-#PNH0BfDiW8`ps@I!PmK39AH&>bz0%jsA~63HGxZ4#G2s7DjugRhYb|9GD?0 zR+ssG1zyaw7KT|zS<%`~va`W34M?cJUL}~8&-zgUVjne=&a4i?(fIwUgiL42WCC7` z)=ifAcgGr<`80xwP)S?vZI;>zD=`Jv7G zA3b=@MK2&tjUeE?-YdJ1=c52f=*)04q zW2d!3dN9YDsxeJRE3$McHe%&=wO2pbgI+TT3Gq?25oZ%D(?V)2WK#~`#nJ!~%hc=5 zT9d)5yODG`aU8nDg}pn?QXlyEC9EDrhXf34hR&e(2WG}s$T)S~oIwOxxzI;^YpTHR zw009D5L?H=yhpJc@-mzZx9RDl-A9)$i?#ur?mbBJlD%1n^%j9S-a zwvIjb7K)(t6w4K_$}TraoJjh_n88_@pmHcJ!Tj50@f{02$Q1EzcEIYRoqM?xtke^y zVwgBi^>-u7CUKOdKLIaXRF0r@CdMk301QyG!Y}oMd+T!aW~eA(`TL zoj5otYjq=ruSF?-P(o4ztD0bI23Gb_jEL^7X!MvuUtAaQLD-dL_R&50QJl!Yo)i8U zB!??5(Fwm8$Qnqyw-J$aI}E;z^XLm#wbewZsmG^IfODeJ$9w(TPpchSDgYVe+Ni)c zitMx%U&JklfiU4EVNJ+|+S-Zca`w;HJTcY$_^_MB^%{5MQzvh*zP8zZSoD9LmHlT@ zcXUJq+!1E0+5{blQ6=c?K96CTGC01)A;iM;MQMkm()B-qj9#}+<`BYv;p3)W_Dc`f z-}no{9dErdQwco~)%ZPg=(V5JavyLl&h5EHsJl(kk({QXVJvHXCp1SR^hc_ymX1gOalM{ZSiOv6}|0Z%Wu ztn9PgxONxu@J`_DMC0n1KaClXj%mCJOR3$RvCA|4NS$?j+S!%}+ZO(L9c=R?I*AJQ zBjg8o7R0ccXx&v!MRz5AJWoWm5fc{L8@)`ud5p|}$;#BFWiD4r=p-Or1p;G9 zSzAVlwgMr8fQihsV*IVemv|=KnVPZd(pb;=7^7|XQIgsCgPi+%=I`@ub=01|FOX83)phfNfz)F;+}TSBQaiO`jd}=X30ws@=)KT> z?|Jp?1>BHIjSnQ3*2Z_i(Jsv$F?=ND2u~ps zanITbnV9_ygllE$hw@;Q_;jdIj#LT?IO~ASkMhqFy_|ZO<5aojDa#eu zKB#;*hYjI)44Ie0f0xQ?$|~{WGVYA@;l=RO!Op$vpT(4jm>z;%0mMG@l=BeXTCWDmr6X%nk{~+0DobTFJr!+r#kCzv^{VV@(e6R9H}>$^ zko#myF-7g`+SW)&kEa*!Cx11R-zOe5Nlz5-{zy4 zWRwSg3^*dRJvZ#^UWLfsE>t?~pIR6W5P#ld)^wTEA@Whfvxtgc#YqeCx1w^ZWZtch?^riiqgi`t-o>d@vssTDFxUSVU1t>)M;K*mT!RI70>K>u-M9yLcj;ik-Q7I| z_u#?ZrSZlgxVyW%T;{HsS##H#`&tk6P~BDMKl^-VZ`SCBe4)3E7Sh9)o>x*2X5j=- z%TQk&zco0zq|Ua#J!FL4Z_}sQ{1R5H`T}}|%a6x4yAEpq{RY1pHg-gCE0hZK2d3Cr zPH6(iMbH?`QxEKqWK=49`;sZmiJz8M5n_%RnFz zEi*9)Y}-G#Q#X5UIx^`!QS7mHl3(LpUW+xKj)$0}j2z>*aDgW2dT-Q)lwY;8FeVH| zKuILx{uL)vJa;=6*M_fTPJ1j}ANbbByrvup6vR60q9b^|@D5v$w)gJ6@eZkpZ|Rx* zOtIhYft)@-v%a87Yn(DpNT5d4=<jAIx9@ z6nF=Vicy_bHTvuQ+~?=@*T;+NU6+u%2?$C|U7iB&fy~7d7ivY%@$3FZ>9JkcY=WKm z$65fdYGpJ=e0@;rz(i@bJ5z$hS6rk|3J275_T_gzU+wvoWS^`~y1<(0za3q z3AkRBGucmlquY+qqc6Tw?cRO?(S_8S0oejcC~eV-Tej_gzKPzv`9hDUJpu;|-r2^% zd$xUP*83XqSC5rz!owvIS)H3WxvKE;x~D5?55|usIKEuu+vZhWg{T^vc`c0Wu2VjP z@~QN~UIm;w!!dnwG?!}PuOwqQY;V8UP5MXdOy+&z-5;`bt9(W{{4wLbR^ry!{?+J`XX3;8{Mqo>3Ha79 zSE2R)Q&G@k1MEA4oYtA`y(h)FaCv^Jgo|KvxcQ zcf=n1z~oyGn|y?cv+rM%!wdvrX?gVt*nh$l@js&{m-G3X9tRX2`y+D0$_lQ<>{_`Q z4H#-r^1Yc`OeABWI)+=J>n(v)L^6(hklKRpL|J_S_XeZhJJKt@>)UMg4E*`%vQ^== zvjsNP#{{aYUw;~?FGt@J|4#a z2VL&HYCYY@{hCSqopv`}|Mua(UPR54)D?_Ss^Nenj_lzvAzuLpp$abpiRY5zkAJx9 z>E0P%?)fyEYzE03E~+(E8=M-A9Txq5td^FA#|28$%S@kR`mL9{^LsP#)`LIeNp`eD z>9667+BU+Tlg`$Q;fwidRHtB9m)g1@Tk1Sac`lc1hbC$cZ{B3&znep^=UXjx zZBev1q1vQVw@%mC$20ySGSif^53uXqKb{PE1`(6PsEb?nz4iAY%xM;?I9x2gj6r}` zJTd~CMLsQNbGb`@X=+$-~@X#%vD>3(bEf7}wjL099lMoVskO6TD;P6Iap9!{wJNGiG}cT?pxR;HHBr}cH(xRHMnj1yg-v9p)7HwaKYe23dxr6rmUMa`#Iv{`;$A zm{*Rxuj&jcJ*${MFz$jbIyq)jCi=DMXT(+0ZDnXga%^G-7DFWqB=z{ssJ%@YJ@u3nE)T@xD+YHt|B}Y{ z34K2SIP{4GE04*6^yJ8ec$D5fx$6BJuBzxlh(pnHt* z-e_sMjm@f`W##%NY#2TO*^{Z4G8U=4MKAGdE_Z%x;|UL)p(iH`_+4P-W(q&G#RQ|M z>Vc$A!#8%3(G7HzuuvxuEvy5s{{s{G zjM#a+G2zMvG_FdDpCckTyrXj6GflnQB4S#AW72!ZJ$v4PO7?3fj}lr7fj@RL>7L8d zs`bDx5_)yZRQXFm^e}k9eH9$8#^IqJJvw(8^e2#qwW^04VB3#vSA`u?UZ~!OqTPiS zKzQn8sV!}v$|vh8B5X-1>T;ksroYh3)$?cYtRLsfQt;hyob))+JSqi0VI zp;0+}xenPQWZHU|Z_3okzpktfCM^$Fg^P8?+Z1;xp@GLjpC7o4Fa=GKhK}=23LGhC z%Tc8Z@lW4PzT;Xp#eNH@OOyEacPl4KAhp(0V9yRZxg7pXACYOfnf$(TJ;;>!-gvWh zRjb(*pKPfeKZVs7su#(lR{iU=b_rn?8$-Bh8)mK#K!n3ei95HU1pn6^LtbPoTUwz1 zie)$l^|o@0vh`da6J;T8{A8)cM9cs)qvm!enwS0i$4%d+h~WC`;ASM970YFjbEp=1 zJ^kJh`Kyho&Q{w=QMu3h+GC#C+^$o!svl9A3>(M@8$+~I_H!eWzMKX`9kA8qMdzZZ z)>*0KD5)j>{-zY~SsS`4+@{});k4Rp@vk=(>_L9ER*kR8`jk&sKw1_q&cNu;6F=Hp zPXwbcA626kaC$SiWE5k=K-5o`dN(T(A!P7;*+X zeXsvia=0s7*lJC`T142mQQ2}~raVnT+&=UxJ!ojM<$wliG5iREAI$d}(x8E_LBadd>?( z-!90=xrpTtrpoh!?#Li93yCzsa_+ih*^#2ea$P- zX3d_<4o0*!c#88kWH4B*l{*1=PV_CGSCK*`2SYYR?WI#&-WfM$=hv|bn2@OiXDuGh zJ!n=`6@iP9(gdRbd!iWgTvqw$}7r` za2QcdOCu};#y}hnq1AEQjGKVt)6c+EXT(W?4{sq)9>l zY_G_rP8GAcXZyVpeJN8>kK3h=9S$jD?hW?$xmgNV(;jbOYUvX;KFCHc!;7{kOv_K8*rkfyZ5)EZbJ zhu>xoxc$&`Icx^5Ctn%15Gtxlfac6rvH8vriov_f@$?2q!QeBirW5g8!S6A!CrvPy z*DA2jO`7s}Y~+F4xiURt%lCuV>eLGCH570n)+v={@B0E~&v?|a3uh*`);}aFGL7Xy zoLw&=BNw<7_OAg#0ugXIi_Yq7QImz%FjkJL6@w2O0{Q8Q`%=$0wbuBtto0V}wwI7T z__^RpnO8j;11VB_PT{3b7YWEhkXU^~shrK%j5ec^8(4;To16(H(dXWG*kSiixDd(E zSd`Jvode7aVb|e5YJUrhG)p%yRHaNNLc%SzAKOY5brY8cNe@XVb3FEImOWJl>p7k z`&}^d#|1GG%vOR&smU==iH|{4`ZB zFxZ9&m4-E_O#<&cd*utE897VAx`rzegcE*|+7mWCF^Zj0(}x8hoK2}b1461Bl+dK{ z5WmCI=B*mi)R$5G)Y}!`QD3a)w+K?Lmr=;1i|ArzJ2{&}WJ6+_7Jdy80bVBRanEAv z(~icdw=Jz%#X`Wc#@WDVF}KMR_OX30su$a93oG06qJHzv9)x}64EmUCx+E}`g}w?A zt4eWZwER{7m@Ejerky{K2HUl+(GilkS?1;_?YWgIVnM>fp!3 zo5>!|c9a_-5^GbUqu_C@*|#*l$v&yKu%?q=wV$DdrC$nE4U z&jml9eiwpmNLvvR%Ncw)A^gFc#iwh?W*`rSF>>x@C~M7+TW&p`eP|p%Z$gD3SL2ro zCw8|}8m5J2R1E8<&+mfQ=*r)IdMYILway1&DWK7MqP_g}rLpzg)o0RaBlUYZP_6gy z2}gLfa5^T~({zaZASG);(E*IZhPq{JaWz`Jt95)AOgP1^CTfxmH%CQ* zQtlWd8Yyr2c{{`UvsF(hRKWSb)~&8`+X)~%od@WO+bELc z!m8k8Wui<{cO<2yUX#RiLpkb9Mqd@#C<;pzgur`(z$>wR9_!3VZ6X0Y0~;5qO<4697!1 zZT-v<@oZjDFxnf*TEE-tC3{xGK?uc4Uh4m#jL~(=C2cMWNc~_KbQh3vUzLiyDu}h1 zj4^&BdvWSS+#k~ql_0*`?q=>Hw>3l`Vd<8Fr-Sp6qQ3gT1`0A+s^^Smjzwn9B{I|{ znGPh1RmF;x6FrCUtPU1GYH`n>?MNXixEz#UWL837n*~>qCyn=vFn1123+g$Tz~ssm zq0vGpq~(@bq%jIj#8_3tC3@t%(-K;)u)~h6f&MwSC<t5o$oG(HD+gOsrdS7AeXNTx{ose`KyYS8gw$^=@mlMK?eBN<9?H!P0tJ36eKwEL z=KJM@CBd@_Ee=0c69XwW$>~b)mv7-#D)bsTMNJgFk~LZ#uxWOC6a(JV3Q5Q~*+1)} zUM^O$?w$vy16H^Vjd*iE8)r#e*mz21u&n4IpZnEo%;fUj`@pYmu1AGiqW`9`DM$W+ zJkC&k-SL~`S1l3-E}a>vp{uf4lFbrUlc-k-lrFIq$VB6w_=w%p{k|i<)4*63(1+n*8j?1}yS%{_9)yC4IA{t^;|;>5AV>7s92u z6#0t_WL9`qXKe0pA4m-oiyT*CVZI2<065iaf&yk^V54hkrY0XpOP`IH1tymKv@0%| zaT5jPe#jvv|Hk+IpD;pZZn+os9obge7uv81I2X4HHP-@napPzB`=%@V=qkJrAwAs= zCt6(&l1q~2MaU}f*{wz~S~O;T-vZT5M^@sU;rCFgxh(7q)gv`HX}NqCxz%PJ52co- zG+su19LQft?&kqc=-^SM+yxt7D)e8Hl>lJi@yZ9kWo0<5+atm^^cHi>v zCOSuSdgYWTtxf6C=!=zp{}#Ff})c{#^LbGP?oJI(#$@K zqFox1_-^R?8?ruQ0{ELg4h?OD)8y`?w|+T+s#)w86Hgd&skcaG{I6**J_8`~eBL)V z4&RLWvN>2wM20dDg1*=A)tk?p2lmBbWyh{;#@2ta02~Hv{|CzfQd<> zbZq|pjQa0V{{M=y{_JdBV!9g-7ycFj+ucJI0m7C6*XQs+p*KYT$cT?U2AvD6iR6FA zF5BXEOqCul%uHJM@Mp^})Z-nF+@I+}vJug$PxOFspBzkVm=x%4jKl+ktHKMi z!$90di$3AHuLQ>T8GBZ5b_->VBuU`!j*owDMX4{h2eQBFLIk^X zWaM6h*z43maCo0j#fRQoE?9!D0CWdFQV!8j?%NO`*JtGEV7`9I3Q$&Vu6T?o0mt$L3u9Zm+Y)q_Z9EcP&3rqbB}#?>k?Yd_DWxB5938)r$P?-WX?e1;u#``qleO2+?mq|sMS zraTx4Ghn&Vw_+jZd+9T}-bj88@QGX>H4F^a_m{nODGhdQ3s8-G{g{SIPy$e}>L??b zCMv1Uoxl9lv{zB;w+!)(9*x5ym^d!pS{l9E_|_4udY#Y4#D{R&4Rl5@H)@^@YJ9PX z9=xS6VaH>b(g>Yr#!N$Th{0RbAR8)zsp};Rnhv_ux_C0zYJFqmzd7}kt5EYa)i1|zgLRKyvlGvIF=_=q9uGck(i2-A2a3%iOF;~8qyqW3 zy04d#{cD$fIkv7`-QAY~ARRQ~7T!-~FpR1J=bsNW;6KL=-WhB8;wV&Q29W<~+kH_S z>=2_7=Tg9F8n9NMDKuNMRXIz(9pK`1G1->0op_>UFLZ)Y3@<=Hb|Y&I z8MFNugVf6ky*U&x47-+oDfwJ1@OQ8Jtpp}5SU6AESq8?!l2f|>bJ7%o@wC6h$>zjq zJ}>P+{yb_-Bwc=RJE2(0Wffh4qVQ^5el{;!e%^bi!_s>naR0rZZ~*xrE>4)mQmpPd zzVS&SfYjxffKbiB^Ot0soyc_pQYO~lqxZ?J~1UWCX(em%uM{h_!ZldWVuR}v?2I7>)G6Tyq zL}@CzIV@)taV_Fz@_)!n3s4ZCC*s!nZ?hd%nHs)F@AsD!lZ3@$j*;R|?fflT?f)T(_(Uzvwbq-YPKFWr&VK_pPFW9RRRu$bT_Xm}_ z0_>KN169Uu;-~bQp)0Oo+h5lpGPgcZTFp0D3oX90ZWjGV_`BJ4F^jhRLeu+oN!6`P zK2ej=F8bP3i%3W@*LgV9FUjd`c}%0dSF+)S@&qJy%rvl=J9y$lvZfpmuRHcYryjAu zhH7r^^Mu3Q8i!?c_*kLURW*__U*?9aewyU!bO!w;OD7>~PEo@~ELpn0&kEsLU##z= zn$3qeB1)>zp)iE*3cdvOk){;*LUqBV@t*Y z-Yhnk%dsk-U8ITr1ag4>G3e`q{XtORhp{B16%N}MQ8&&x#PtshS@Vlg0q<=F4g`jz z#GR`yoDi-*yU$H>3I}SU4>eG`M{!Me4In!5HyJVr1O?kNtF=E_L=o3k=o($cF5%-Y zlS<1bE;C>SNMioTL1L z^+Au%Ye(?fOWQ0Qu7spRm`G)nsY3Bk5?-MDSBb^LpDfF zgtT&wJjd3NS z;m5=Z0n7dj7LC_~%~T-XWw5{@8eLr2nqi>HTsDpPo_lVW1FUojYi8IG`y|Uo;)b4{ z{XE0U!-;w7`wSw`sR#E8eP35Xix)D&K&ilLVj%RJC1)01iyl>nCJem}gBFjaA6N0D zJJI}ZW6!*CYq-DaPshFE%UeK0>eY#6^K}G<@iUi&+^717*q#FH&ivVbhp`g6_vdhI z6Yk!yd;F~^-%s@%#y7NxxULBHV*1Zh%6sG|d}5b1&$<+=T4YW9H&UZ)8HaxxnJz6{)6Cx3rQY-YRgV5R=ibA`iUOJ@$RGjsjE-!F$_PVi10G=Xa#)J*I^)@N#+a(VzYciS-I#zOzm_;qp5g+YVxVa zcM^iPeY3zXYJb71^xG8B13PjR@!unu3ACZiGqsYR+TtOn-JFT`okJGZC;FIKJEDdd z4JBQsn4;4>7%RPwv{$a4mYYwb2F(>d;FO@jAiW z74Q}4;<|a7So_Ms9U<`N75)0>zTs9H{BZ;28OrJ3P@6uQ?LGb{qRM7pGV2QT46cwY z5(Or8_T!5Ci;JbzhhwYdUwF&MO#G!NF&w@-_uJ9B$;SWoSgS2o%~L(!K~g-XT4wN} zvmsihB$XZyRn5CqRYr5QwBotx&K7C+N z3NsCRolgAy^%}V8Y#-R^Y@LwBiT2p#Y(G>$r91pIxgVY?9zT@M{gsYO-51xbbUC?X z+^!}RDI#3)F=gN^5H@OUjhZX{+^_*XT{(W+8BV*;harBZIWZT7Eg?+Wv|=?%HIwY+ z50FTx^P{8pK9nEzIu;xCd8U%{vBW(BjCQBC$U3akY~*qoZnZwfQ4XZnrIq_n%1wsF zw-~%sc>`2Z3}S`&FZs>DSLl0{i{ZK&RoyrE2K^KJYf)HvbJW01feFg*{yQXIFVv*) zhCM|2kT*&ZD;FJkP2(R7;Rm?TH$vsH1I@Z;12-vaU77PjU0LVefga?xU1N5gVe}In z(Mx`M5f-@^$_x=NEJ`i1{Z}O|EGtiYVa;ya2BZ2NXjJ3vaA5s6f-}!o*ioNrs+Gs3 zu++B{a4!Gbyp;0dlYX~Dok2~lc4b6iX%~Qsad;~6Pz=A#G;olgsYF*kJTd4?>2z<- zJY*>}_>I1frS#eM6z=6x2X$hcVXH>kLr?ob=yKAu_wpzP!5|}lvqD#Z-%TG6e5UZ{ zM8>Qb^B^D@8&+6SWG-F$O z`7$b}v!w6#la4wJH0L4B4`{>i%l;RY6H`pd-9>N;$M3Po*6!XBDrxSr{ZeNYPH5_U zOl!PYcnQuJhHVw9F-MXggdE;>@!QaNB=&XT;ZTtO@x{u-Nv^OWsbaIH zI2)4j@$Xl+>xcqZd1CXqp5qQSdA;Lny}pIH@5VG;?s<`K&Ni(o!GY4I`HfQPX*Q2NdH2opD$wbGVvXrDOZ(*=(P)-mmGDAh8n(UV zZX2BcXQUN&J3<4)SFE?rBmMTyC)oD8xi%EzeQH853CROFgr)Qa9lB$jtgsRTV&idQZ z+4Id9H$mXv*J!in{;|)WonbI9#@W*@GDRrR;G0%mFf_pT$c^5uh*kh=$dQ}Sz>PKl^buDK6ed1@eZJVBQv3!s zRNAFd$yO<7gU@_FPci97^}{c7WXJ*7tcRp%cErHn3sOWNwQ8BD22>o$k!!yE^IR*Z zG^0zRTaLSpDw*kgs5OkbWnpd;(rHpJFDub9Csq;|X{P$dKjrSi^x@&~-JW<_KUIWq zU;croGRGl-Xvx&H%n;IxV>g%o#ew%Pk@eOE zRR34O>p)IN>f1J`XTmo`dfKjMHo<&5ogT1(F)t742fqj)brMRR?0?{|S4P-XX6vex6Yq^WFo<*jVBrD5s3xrd2Ej+NC1?X;C|u5vv= zrp!oKJR2L}Bi#g>r+Vg$o3iRIW6HYGz!~x4 zRl!_VY_W2ZtJVnF)r5;4R_q0!KvWYgzWsio%fThVlr1b)*hX`kHe*@@qp`dj*BeR6 zdxP)&>Ux=u&BD*#|5c&z2Adf9BfbUsZ|7bv*Or?3wA&44+I~nBtU&T5tk``{+6Wg& z7@^ufW3`L?Ev?aqWkjuN7ooGoc4GYXRwyimQ5EEFq96@uhvLY@Ie11+85}Pg1z0Kz zV`C5?4+aYO& zCw3}t(q^N6iXfBI9C^=XHEQ{TgJOMa{d_%V+x>ei$9VBhKNal2mu$NpRY>mlQL}Yo zT)~D+#1(%KvGOAm99JK9d;Pk9+jv#Gf82BJiW<*)CC;nSJHiX#7qFb|O^RIi0X6d- z@@^R*XN6Fjv@q~y@hS|(^^l%hcb#UqZH89lYy&31fzOSn{l9N>OfExDQ=5z#@obhO zB+l*n6=~KYxN}?7LuzGB!|ZbjGa2+hVXw%>%|anv0h>Y^K$ zc!2wjh}GV`DegaBZS{QiRceIg%>SMp^$hvqpSeGS`0L-C_;2<+qi?)08;+~TSeny_ zmO5B6y{`4O8jWem^qPoO&tCKC{N=%c;o1ZAg@P|v|7dG0mJOH|>Jg)w&LM$gIosJb zIooR$ImY9vti$4Mm73xa3d^entHROTbp{3-HFV8?)$Yc-SDFO-UziPz4vBNZIaoLM z*(g7lUR}|0o`6A}3Kh`eVb8@M457bS5r%@TFizawkscYYMvrs)&E~7zOwZ03XG;%+ zzRxRGTD64k7R>g$Jwmz6nj(QNf@3JWjvWsvdaxk7oVW~tb$?cdo6*o*M{1iZAD-=c z%%iRlWa_ld0fJhl%yWz%-pmyNMw*p}VEWy2Z&&r6W843-ox2j46Qnc6v*@O4sIf9B zd}B>A(0hB4Z9c8bYJUD8qs(8;wHjRjXRD2v2Mqn- znn+{Cgm|7^3CL1^Q{pLS@-5)piqt5MUO=9K5VYz(jKH8cmp+l#7A? z|LwZHkuf6onr`2YVH_Uu5$ElDDA%-a_&t8ihu1_AMYSA(wC)K6Id)p@djF>sqC|}= zKA%@Cd_7^YK38`Z@!ui0s=4}1V;=OlzT9OD*sn37CPGE#@cjPM>`ed{Y?Q+sIU#`k! zXX$b@^CxOpf0n4IgC!}-YOc&`;T8U(XI*oM=UUGKFzVy@XuXf0DtOGTF1eUjnmX(O zKd~sNSmkYcp_2jGyLUb3^AG!=wI7N747}I#KZD`u)qYNudX6p0J9aJ$E>rykt)jpY zs5*=g<+`nPSZweo3tRx5=ZNML3*CC0Yh32ZP6VN>7h5zFr{R`hVlAhS5*@BjuT9^d z?V;lMADyRv>dso4Jqz6RUBf1f}L?SBqVejpTs?0-rYG=@QBoMH0I zLMc|%z`YLj$Nm7z zsw^2ye6$u(e7$zR%B8zfRp$8Sf2EU4^S~||ZHJ$i&yFdh9TzVV{7!GC+BtI}PspR^ zy=ceoF(Cgj*Rj^)a%i;{a+#}W_I;ACm22M7bd*=KR>hTpcPq5|R++^w+v(IT0sh4W z7Rzv~ME-^BNh`BBbgjocviaBg=mAE<(Yg++JH!r~ z65#JkgH-9x@p+NSL*jVrfa`};*0?Sm6V>~^++wXGOT#|GAH26IR43UfY$Z=6I z|5AakW_o~A8f7|c6KoyDbZ%noVmZh+GQXpbNz@fuDscC^-VMo5FW%b z_Ip{Q>wUC8pRA+Ex}`!^%)e7)5 zeG9=GheWdbQV}2ZDUCbwQ2AGTwKnM_z1YH0zmgyKp3e(cWV&Z{StV%(xGiRoIEr0M zegH}3fR`$K_S1-5x7osMe;Z7A!XO7bAs>cM2u9^M%5z6pS3RqNIX;Z)j!M|w1_3%j z&8Q%J;_7of>f!O#zRqhvwds0%*aek=`~Y^WGk+6^-0_$6|ElBoYz;tk_=!LbEhLVlI* zuT}pUD{JpKUVnksb??^2mso41Sqdf{qHISqquVs%1)nAi-HbM6$70xi!_|SgxX#|z zxK4?;1|1x(hTK1;xNV(f-I(SaTc)(5*!4cP1Ve?B_Di%c0pesjRC|X#LZIRFrLcLxje)QVHoX0URI0?z=t*PX=*zm|3!J zHc1W|>k)~MxZ+5wrJCvs%w=hmKyouL%;ZUe_Kvbo66>0IHzuU62Ic*Gj16=ahvL4k z0w-@*lys2vL_3-#qj{;<1h^x0~ZQq49bD{0Fl_s1;-k!wvQ+4`o=BZjheA1 z8a)yrUi-aJ&a6KUPTPDoUqZ2WI@bB?m|<8?D>_c&jgg`{G0MbR_j8UxS9-p=lEy7m z+xd!^u@CuhX~1F&BD;!XT~?WTM755l$1CYaaA{X%C?Zg6KBir<%J{n5Ny}y98;Tt_ z=sD{<&ioo=IDg%Ufc_jw@W}Xu{#J9v(2o<4a>1W>`OwRAC;OIBD|#xES^*v^dY~7G zE|*`f$$u1dU9pmsp7lxe(G95xn=m>4cTLx+64t{fPgnJ`J2wMu5+WES9+m+ODktkw zefzf{>|zLuqXzR_+NON*%52T=OGGTh7qkjEZlNz|ZpCf`cZ|ir-|+ZxJrsFsbM?tk z*FsY>F7D0QBCx<6FSUA2&dz_N6hTqC+y~|=n*;3cUTzJ!Y+*qogr}w0cml9_0>}wW z@5pEwWY8#Kc@;|CS$T4qE41?)kZZK^X7R+edQ8pl~-`T;ll@!Wj!3RXJ zRlAl<3eP!zm(AcK$NsX&^={ryXP7I@zpH&NwNiiJ52U`^)myuhpzUfeG&u6| z|0YyW4lBhJ&*utpTtepkRhsI^5L6vSUmaC~z+WWqEN}jo#5|W|8cIwx&>$!0>F!H; zo4$TjTHU^*8T+!hYG<{=y_ib03b%dgwdEnv@2l5m*#gW{agZ!u`gvn+N8P6uv7bpzuz2gY5u-})sxJ!g3x7+(y%6v^j%-0)d9cT$QS>hhM{ zI}9#BzC+NB5y^VeGPXDrnqm9dT8i0#Tgn`aAd5Hs|n*U*~6qg!5*rE4lvUlRMy{$!ge& zS~WqJmY7))pKUHX;X9nYP%E9fy58rO6~`Ht{rLC6>ld%LQAe|~atNzpPCo;B zo8W)30Ql>f60%n2?=7(FzJACBTLnc)LM?Ak2uJ5^TQ)zuNJ9^#VCXk2v$e$0;0vE0 zI*`;7rm~#kNWYUc$t#-dLnXXyyNv{JMUZ`DH_2&C)Y~fLT(UUMS@-1ambGjSYC{>h@(idV3P+kpAGoaLb>zXoVd8niuc)*fZ#NV9B>_?zQ$tIW<=8eQZ7 z?E+5qmr;^&5{Y-PN-Y5%w*9zv9hTihc3k3U&5E9A9IHZ3z^)ZHCjr_0Nfdt--F${x z@3*~9`;Ux*l~IQ6L%Z={K;o$DR_+c{``sSNPWf7-DdT$BNXj@CyaP>$Q!X;eg!g}7 z=vqOAjPoWZJNO&O=4_wH0K2B6nQe*Cmu;wyPqW8~DJQ{viQ_*(fupdIKUAnyIY;~G zCyf97CSe-WhG<`xQXvs@^&{CfRu*v`E?R>3_raV(VZ@$;w!CvccX%La;_bKN-ekS| z2HsiMp7g32I1>d3IZcT;|A^WA9ub;*^IN(DYqN&TR)GBFOx)I`L)OS^jFacki@3r{i&nl{%U1=%AtqywH~HPXO8|!d zms84rULxSS*oTVs8y52OT+NT%{=4-sq2cg49dS|o@wX5m_jjKg}Dlh@s__yeXe{5a?oyP^Pf2|f_isvOqcV|1l zWC_8Rf!jj2o-{~W)<)CvB7c4QXVyhi8>a9TH~An`oLKfJ5yBy1>W6JWS+cac*NGXv z+1N6C0yC`Zyg2{vXpb(rcW8U7?MeG|zzSzd{DHm5hMWS%XK~igTrj^&s*w%}D$O@9 z3nCVL+9Hxt+7x{;qO#V6!U<|$RrSjE8Wz2)m#D&#<@%oY%aj)u!d{MSPk0@$yEmM* z#t5Yn>wdlLsho7q2Cp2so>;*ZG3h`EQ}Z^0%FT>Y8Bs?RG9{%!tdUl{)vpOr_6G?R zI-JJwb?gqCcsrU6xqk`i$`RI9eS(uH>Y>T=Jud*c&Z8|FVK(x@Yc>=%^3W=CHa!Pp zx)&ywf=us~F#T=;KDb$QLLRs@0`L(92yO*kmF64lh6oylCBo8T#4;@Epl|>-yC=$t z=x4@^u$Cz{TS;hFfC~k<6I=G4#C+K{EE7gTPBgGT%soO)&5}(vjo^eNN|I8w=Jd`7 z=bN($WGISlQLOqXQ};{ zDgI{{SZAz*{1x8X{cllZ;gz5GD+-Nwi}9y7L{_!{_r&|z$ZDlY`76`)ODUzV%V zGh=fxMie65cwQ>->V-|5!#q1AU2hep1Ug6?a5O-k}`|#Oc!0FB(=um7-bmum8=l?{fwGo zgat$79X|)+&J)dFf0UBl8G7oHO(bVwcetRujsv8x=k9~f4euWhqY!Px6UNw0yH%2z z=>?D*e^I(Xz5V+6%lE7>zlxm1?RF!>Ddh>^`|bgC4VkMK9~+5$cjrQ-apExe^+v42 z{mRUH9h%!n2QS&!8DQjJ9^I=XPKr7ey8UK`Tj9aMAVZ~f!LeJ3fJqff0puUVd8dt? zAPa8PL)E$FN@OOr+BqvNQTmE&SBx%p$|7`z&!bec9JQDLhg3^!I~MU{Bp(P~jobST zBw6`}=PJ_kW797mpeb=4Yiz+>Z+xB}Q7!$sDEu7@qme&`FQjDQn;!#z9&)SxrprQQ zVegidF#wKJyS0q05eLxaKm_|k#ZddHn|)DSwdh;P7y69bm?pDrU)_7LkHxsF-uxWn zr;PKvWwd84b}8=sAGg*=O+3YND$1yibF?Dp>19m}rT--6>xe&z&~GOtI(_bzLFk8z zdc~Z9(v4TW+pP@b2yJxkI9#}R!;>P~h4nJRRwLnc##vnzHYB&Ns|~=dD}XAPk7}~` zbqi%~LG;+gt80VNj<@(jcAP?tT!rL1MRfe;wl*)`(dT@I>-zaO$J12|O0eowE^sAt zk^lAHZgZ_e!S{Sz!Dx4>qMW;kh~`2jt$*FuEh$HFlvMxw zwwNZy@V#Hpbqa^4G@%?%dDHP32k=YO_Uv(!$!X}RyPvM0%m_yQR;)2d{(95Ql*{@z zMqv8z7}fX2O4wWGdyltoYgVS=(rFcm*fMDwzwwOh<%HX@1%Tc!aD!%yL~8kL=$jiMVy1!8Ym0B>%KjRH5mk=s{XD?H26-BlXqc#eQes z@@AjEz@ksrP-UX@aVcGZ5~ssggn{&kt4>>P(YuB6KG#Vf%0ZsSJjR$W3qnobUmfk`9{Y)YPeItAX!_50_6-LH z-8q@^U=sAr!n2aO0+RY<9l0rfb3^0(3xmtZut_SG^FjmB2qKQjv!!K&=%Q8wa5PvG z0(rc8oHEDcA6IVp;9`-1tlBl`PIxRqd+60%Jc|jcEZo{IszsavDTzZG|Awvw*UIf( z?SL-)G#wYb0}9y)8RZUd9+}!k^ElGiptBxztFKctZ&?3_u5*gcEDW=BY}kS+Q-~wry8z+qNb>t9w@W znwh(EdCtxG-fzEqKYJ{*cPABy>C;*>hi3<-*kG(&PP5&EWrk^-h?1Pid#loZzs~y5 za#FW8@~1j70)j;Xi4#!tVO|jokY=Y%$E;d`Vkm1K%g;>jBE$5TI@N48#E+36inoYV z@!ewW`i7m~9{w;nzq!kBWQ+%DmF!J*h*ryXEu>-3Won~9;&frEZCZuG>v(-^P9FFmKI<`V{ZG&8AnU91w^xmm>uh^9%S|O;BI=*>Ka~B7b z4oL~>1WQ`gY$C?t;G^MFNMMQZq2B&>2S3ULXUD7fOH=XBIE-a)EX*GdnTor=I(Dix zWO1IA!9<8NhmIWW9wwbaKKh2Gi| z-o0WwUM|yaNF%( zEJ@8)#?=|nlF;}Io$f^b3CBw28Qu3+7VR*ZaZ_AwgzN?DC2pMx?uJdo4zfzBlWsc( zvVi*T%a1f?L{XWaiM}pcl9daES|YK}BJxnDNb-@162YE^IvejJ+p>>mv@5AT1Mw($ zK`O}~PMwKrAell^i=DCr8Pq{CjVAfQr&uZ~&IffR>0*6fag#|GKFFAo?+{6Fn-aX> z1;Kld)|?G`u1^U4TOki5)=aoL23>cXDjTSfcFk}hmjLX#;j}$B;IiyIAzR!+H9K%xSS7btIfUH8a zP8wTDY%UE$#)TdK3uHDq7&)5h#n;xaJ|L+lA|=!_@njU&>G-8`vv8 z-8i1MuDb)cSx|D>&xLfg_ExZdDIdcjw%SI_uG{E$EJ)T2M zV&oZmloj=KhFAez1$8$E&nv%|{qW(tQrp|%!uH5CtBit7O5pl{=q$lvI+$y}gnvCPgM`#L8H`sp@s^N79^yjW22$ep2)uy1EI;KJ;-og5 z@7RKjG=BZVkqy^aW2A&Vzz091;(39VM#K{ugA7sfxgkxw;0fKk`+V8C8CmhXfqKIK zh$Ymp-Izs;{=UUb*Ke<$g>d4f#@F2k%+)h)Jz%zW>_R)Cm9?tBq4<1TqsV%o9Otcr z5mKywaZFEo!SR8USl(@{aPvehme+aU$d-rGo%;$-9bq*dt=sc(K+nA91e2&V7oyTr z6Ralp^yamUe*G4e@e7W!jSDgb^Uo#L@Oh0wR2@}FRak)fc-WzrD3A0C-i zi4C1$m-W}EH&UPb4v3E5ZB^z*XxMT?R<8gNyA>pP2eA2b_JxT0gK}$mpe!Kv)68?n zfzjd{|B@e<9M?|ztej--ElNp>GlAqcbdv(|1;RlwuPE;rsb%5v=AF!s&<{~fFy-oX zc~4A}=Bh4)YYgMIQC(mhBt4|k+K4AfbpTLntVzEnS8~eiHASF>6)JDWX&Cc{{0g#V zE+xvHa8CitVk5*!Uw?7^z2>1c^aE`hLtI7nQ@abmVZs6{r;k=9Qj}gz!>ryk_0$@m z;7qzah0Og$y;WHsz0T&0cv&}wkOFAV{dew;?c?q6fyTk;^!#8j(@kpaDEmx2K1sg5 zrmY4K@p1`K-2DJy9S&Jo`g2x9!QSeL?z^w8F+t!*b_f*{c*FRHb(K++eb!fpp#wnT9;qIyTZS}nO zuTH^W(ckM68e-l(Rygf1b`%R65E7V4?7Ga-p7G=5;fG2GX^>ij%eW+^SiboIT!O=T zHI|XsnS+Oou8e1o8rx!io;pXJ?ys^ekxHBv3xZTJc#i9Z8%J}vY;Pa#98xXo1egW- zHv6~9p^zz3&u3q-u*7Kgc5xr5dPol2(^JTh(&`>Q5!5v7?H(Dt;u5UvtBkToPICE63ReARTFbAX2x!<<~8HrJVGdHR$N#Ei6>77*T@{}qc z!gAU$wX4J&*=qsQJ!3lkE`)|-2DZf62kc9*1L~X=ES|UPmM!i6nG!C!d#-G{{!5x{ z?_Co*-8O{lNnN;xlbvPB+S3h5;eC0R#akE1czMQ6W zJ!Xac6JXX)nzzcd@9L(Ho(72w9;K=aPYY@9UBGXF_&3DXiX-~{3<$137pT%)?l=yI zqmoLP)Pvn+?601zJu}re{$dd|wXdu=2QZP%gj=5H2)BY05Ov?h=w}6XfY=wbW|Og` zf5LS@Bf{J>XBHN?F>5_!59@d3&Rq3559=`ho5JHB!YH5WWIY(Xdn$r^gU9?S@-}8Ll z?J73%i`(Wu7bpKyrFdD^07Y~^QDBaa`rII*V;<`J9Z! zAhpBe`zEA(=xI9tp0|3p@`+vs4J9th#raF}!l^QOCOeG8=a0F3X!yKJb5V04ru9ov z5%Hq(Tv$j2vstfH@U7AWi@vCr#H9Bi6i_%3pgA-)4pza7Q+L*|awGT4g%$X{hC?K| zKNbzqnt%{Rl|h#Ij}J0ZUQ`j$UJ??)vzg)KmBRtYV{T_P>;x==2nL9vwXxN)L?hpoQ{1`z?HNII$ImrQ|EhbH$b9xwDKfdiIA|XQfj+2n;7a>LOPyzUv6g*))6~epn zc19yJF4<@xungGxU!i6?*qY9DIBjeIfW8hq+TxR;z8u5TmZgZ6H-pQvE5KEarPz8a zDDFx%c27e?nd|*QN`YcYzpO8NjXp8EW=K&2W^$U*xPvgAD-`TmGg4o_-|~3Q5@0`e z%xHy;G&=q}yw~PdC@vuv8rfW#JJ|B{rQ?S4fX($s6X0p~`2xK##Gw_SDv93GMm6UJ zmHf9^t3**D3NT;|9z(6Wcc5iob}smkCbf0}`SAJTLyhV}3zi!T)Ey1nBsgk!A5V+i zz@6f>Yk(!p=?LK@PixSjwoRyKg?t^}394s^)rP&Oy6W5LL_{tr?lo1|6ZbR2TU~Ma zB#^fB`(uW4o9N37(IK&K0l6Amvaj^b8L5{RGFNVMQE}0G*N@otB{}Ctk3XNr@73fbQR;WCFwr z#9_2Bzu$XPKiegg58fDcCImA{LH8$p!#b=a^2r6T^^ax(=|_m=5qoQY7+n63HFgI> zJ>ss}_^z;CquPwoyT^*;_!udW`+vSMf43lxbb&x`+sO1`z3Y=seSwiA_lPvVXt*tX z`4YLV`HDDZgCoK?q0wbVf>CF&C?yp3^hwBw=XcQ&Xt1+eAmpj zd2?1=dH=xqoBAM$)S7q7Pn&T1OvSUQnE8giH}^MyjzF{}ZMh7^K{)G0iC&@W%FIfEbvrP{Bd%+ac%KQAkO>FNHgh7BO2sUw zuDH1Z@H)EUhfs0yC2k%4tHgHa2)z~W_ZDZAbMG1V$Utpf6(9VLziDo9-rpt9TR8*f zXUPQzvT+E>&kkg#k2sFtHM*~GokJBDKQk15Ob;?5;P+J6T~t?cU-WEqe$v16hllfe zvGo1RQBAkBoTHaXD;U-iD$GUMspRIkg;uDur=Zu(_w9p{KST`n?y=x8WTz$%h_XXZ zqX?S#GZqmMjM1pw7_ASj;PL&Emu)yoa-8&7{6W<2gTFiXsp9Y>lQ#h zAgwSh$Hm>$Ph?ysY*W-!gT%SUeOfYhlykW>4v=47qDe61j^sO(AnF{O-fEJ-si#(;(Ao(Sw&vub>XV)BKeLkTJux4~rS1}+ndn?4!Ky8CeJ zWq+}c^1nrCH8A>)F;4?$b6>)1S`(yJ*=Szckdd1KR`2kd>z@Kg280=>&qVS3l$2{_ z2-Is6#8|YHON`}@JVBz| zlqkPYXxq(Z^aBqomW?(JEa0NB7qqh~PbhBi%vf^mdHc*~1Hi%@TtRidp%-qe{<$=f|6Wd5 z$@BZmL>-MybJ)Nt{lRC5dpan-pjWXrQ%(-p5 ze;=quDvua_p7N~Nw;-Bmoae>{H5N8BIcJ{vPo=?iBoA6jDf1?Hn8p75@?8`2YP)eg zt+?UsDuaokZGxi<8GnwRL2aHVz@h|Y-Qm89U7D1gE1SJn%ppz84Urn zaenI+fB5;e!E(tl(Mtb^78PL`cfBeE#Sx&bf2Xn*;*Dca&={(%#^;QSh3+##$_!Ez9tvc7Rla#$MH; zyXO5EHrk>~Zf-XyIU6FQ&le2@r)eK-MRFje&e)251HsQH-}E6qVqHm#^9qU2g7%#a zFls;)U#!i9Z~BkR)6ob-|Ezuz;4*$ICny%@>Jn+6lEE#KzGMh;yDT^7wzk3Z@bkgy z$_G#+SPo6l7!CNP@;KmLI`hb`x+dzMEG zuh}$$z2<`C7g@S{#JVVW#Xf?!hMhvB9t1j)vV~dF=?bbZe}B0V#z)yK3r)hazLv>3 zwa|c>Q@4?@*F_AsZ86&PV{rTYZo-~&@&S{DqKMn9l_Pz3XQ0g(?N0^*!v~{+CP7Zk z3rN&8T?;LKeZ%d^OdUJQLJN`Sph*7)y{f_wI{HA%q)UaC3ls4_66qu`kXfXA|L*Tj z`6l=S&iorjccse?j?RO|u<16|Sb$>i)3ebQJ2)By)iNT0vq5;^6&+Y-b!VU`ie$PP z&%YugxS&9DsB42L977)Xuwe}iQ}_&Uk;VnbCw6PTkrb#Ey=`!@S5Bw(@5pTHFejmt>8pp9d zO}$hmc40tBQXa%bo=&?7xupA*VDa%83kNl^pmM9Yqbk%`{vc9&e21*VNVy>XkSYt2 z&1$1lWu*3A2$NG}wHO>p0wA6r?M0mMUtR!Hobh3~FUlP^4If}yk5g|1-#ct=Y*;)( z!6-idFY843kEu37yQe+2z`5am5*1xCp?hPJKI!Cg;!qs66i{GAIvb6sXQpp#zNMf{@FlcTFkV{M(~QQ~yGnjnYb! z8~2(f+(NfvqucN%H(v;)jz78|Jiibc( z3TrDw^EXG%FW!0XX16xsrE|V+Ud%#C$%(ziB~Sv4>gh{fWr0$yD^5Ehavz7z**@7& zg5?yB*kXt<`{dr}$xjW`Ay7|H(HM@k<{hnY)>or57s4O3R*MFa4X{kqq4AVZQ(^N5 zqTV`OXSe1f+K=92vxKInO-Dfet^FaziXhwgx!XH#`}!0B7+zjgV|uC9v?>y?RN?`F=#)e}mzD*T22To?lN6Wk`-^TC#SI z>M1t=qCOv+PC*-Sx^hBGA~dF=J_+vPVWUk|!x1PDIw_H30X<)303I=J$)qZGv%Phg z)NjU^>3L(XFqRsVUM8PD6Y7k*8Z)N8mWUkcyY~u|GRc*KxlYkbiyo5E01TbUY1Vt; zV=$CV?l)<+TcRgd<{JHi!pp$M^Ca-!FXEc0BQwKa#sF6IF9e%`$2A8j;l`lXjuD&S zF#Kq!A?eGv|5ID9MYMjiTnD&?@lhDZ)XQuL-gX@eU(~E+g?!*NS*u`#HO9XL4BdL( zJFcL-y{7oi{7Y7y2qP&g;q%%KZHO06?X3f@IcO}Cfqaj;_)~E6hnou|Fn@hS0 zn&TSaic=kArL?wDKSj_~Nswdn+i~j)K&HTFdf+M&6d+=!E4P5NppP>WrCSY4=&F~^ zT?AEf5Og6YR+sOGl3>r)PK|JF_6-jL*C%;j9NO@rST~`{_gAX6$7*?EnHJP@%SUCX z;qP_SQRl0TNVJzbuSImaE*XZmB)~l#?VqeaPxXWu{cHyhKV!KlwD5K)u_u$x&xXn1 zNyr#2O}3bdNf5!h!ikZ{WB^=@$JCu2fL8L%hT>Dw-1cug$K{F(Hqn*(gcj1Md*|BA zDGE!*3SYbkN6A3vgJLZDRj8?1t9bX|dNI5r<0bG+194uplo!8a4+Sb3#bo<)vY{6= z!jUODL9V`>i;KW=A&RcL6dQ%&l`V?-eArl2bDJ$y60zR)hL#&25!$Cs%bjKU+6uZe*#^A7>DW?2Sa%4Go!fbSsj(`GPLk(v-Sz z{RPE)*b+2Lx}}3#XEt~rdEiVQIJUB7hCtzbfl3}Y+3Tv|bL9jVWmiL)^#fQ(&V75u za-tjrx8k+8@}%MA+DdPie>6L>E7M8IGZuG;RI-V(XSM_Gg5?vpJ^q5bLo$Wu)`^F& zlfm7U7##+kuc&N>e`1~?oy#A>Y1F4IWKm;PiZd2GU{%DzT6xrd&9j6$LLyOe0bIat z3TY$pHLd~ImmMqbpZIV~0a(>@!Py}hpuyl0r0$yt2OQ1?waxhV;|TXJN>{a7j{O%- zeFsh=h4!=kSZA{Rq`Pov3#|E#)!5*a)56U!d78S`HRp5vcGAefe$aZ5;#_Kx3t3oK zkvkU4SQqW9{`gUsR3A??XcdU%nEP%zG+!!F9Oi1-$<-n{3cFEr!w) zFd0o&-$PyJ9KB;N=p}z6#k}@wdhcUMTV5G_58({PGpjyhF)G3u)bfCrO>h_}h;TeD zxf(5Y@Ap9C&3`Tc4S+?!!+N}-7#4MEE{+g|0wqeqkP4ZxFR_VMI&+J-|fjaI+t`hjuyPPVex>`>7{4jZdtztxbkw3lPzG5wrZmZiS#*A>D)1! z-NeVQRcO}Q^zWan1SF@k^7@x(Hk}Dph}#vKHrjue|(o1hXVT+JrM1(@U}z(ehi(QKO|wlsO&F?t;XY zjz3HNyRvIk^0o^K%v7IU!X?!_2Eo~_WU+#fu(u*DE$d%_!*=`+nUz`3+(nc)4P2=MeGjF$3 zMZ5qg8~2b3eOEGEy$(gH)R}QaVSag^A|8i;dC2I~V(?lLT8+!l3{VQcHbc9Xz3e&t zs!@UoO`pR!R&iaMni}FJu2AP9_-a~zG-To6)UTBRuCA^wKC1O^;-6fTI)z#nW;x5> z@iTG$=iy}bW?qVQqjAF|{DsW@{=Asz{DO)Q{r!hF+RFOEccSBvH8s{vEAZ9d%Po!f z-l=Gj3z>q)m7WQ81{3l=pZpKIH?yc-_z>v`KZA}|m$8N1{fzZIkrn@!xAL@JRq1(7 zjgPt;3R(#cY8G<$7i)?I-sn+!Vhg-$ROEiKr*0*n5CW|!N=MQQV}kQl{y;t%H`%IVTePwWLj@7 z7*`;{Fyk*%7A$^_mrHb2@t;DU3k|&Q{5R5+AF;+vBam@jVg8N$%!%RY?z6D*gQ$lg zOJ=NlK*x0fPUKilPJH1dkqF1?(-wFH!@s9ys(0L3oAEQsbyYeYU=*2l=R0PzIwUaf z86%sxs=g*$uF*H-J}RS8IhbEW%V_mP()n_x4MhmU7NBdvx$BZv)ACJ+_sGHllh@F%}RtrZlRWYWQmsmdnGrn3A%Vpm9-l)Lu;xOs+e_P*7u|uwg(Wlp+wu5SqZNEXZVH4MQ!H zb>0i1A7z4U38;fhISHy{dg(_acZRAZFe<~4$Rjwwfs}byU0K7;Q|w=7&BrxDpa@oSz}x>u5p`h54Fz?L-6XEoP^%Le z1P$j<9Q*PCe3C=SF3l0Aa*<|TefoTDk1<2dia*$04n&uvn2niQYcOmU83T=`zQ_F&h+m4`Gx!MQ*nMdLX0#5 zi^7(WLF7dsPp6T0bQq))l6Fa9T|JNtpYBfn?g*em$qGf@2z`tie)x4rRSxR10194W zD}pk1wCd*;0+OH-Ftj^Nl|*ERvE-}ocR6**=~fc-r;M6m?u;lF{V!24aYds3d5Gv! zj3JVU6mmxs968%2Rn1wyzdvH!=oxeUJcXoK#zuhwast=&{LhV?PAig|>Nsg9DaBEM zoGt7!LV}CswnLCz+Df~wr4Fw2ob=T?5tr^&{+-d$)IW&dJ#Bv z8d%+y8z+~~0{^{35Rj#@R@(wh#is$?AkSrQEOpYSs1M_!S3THw#*Wvi8nZiXJjf+k zChnd>%v&YF=p#c6^Aqq`S_m2<4{_U9FP4>=1#j4c5aJx~l&ytenv9Fy1|P}o?8ZNA zsv6~a;xl9C!MCtpw`}qxh7}srG#PAF9dyqMwfAIPhzk_DqOi-HaX(|pasg4(eK1+Hv!h8%jtALUaCL*7#Tc^%**5R(N2m%{y^PDG`- zm5{kJ$kY^lh=?oy#p>GdM=4jRb(_T>-za`iQk9$SHWM8UPoxPsWS!5qJsmZd|Bz>l zk<_YE<*FwfLr+KKc}(?nO__i)=-d%&h4uSo86;6ZgP%_$bUIZW4MT4fa$P<`1IORK za){@JV@5L#{$|2&oFTIsqwgw=i<(nG3uoMMT+dj~Dqz2|8j-|}Ob*GjO$lr~joiW~ zU<5szUPbaixAtb~zvu}fI%T6eZRkZmnWAc|fpEm=Tkabx^bt55wQ-CR98%5;kkvKy z#m!^h&9k@nQVX}g!O1QQY+Di{j70ZL-!2g=|D=-Lum89LhDC) z<}*!uE_yUiGq<@fU9yS4a05QX3@-HI!+yb17%s=B)S7%bb>5z_YC4Iy7t3&O-F+}P zU4}xsgmLnJN_n+#;hVwa3|N1EhpAli1uyjz$#>q8*&Id+T6KdGR;Ev$%XCWL%qwEg zYa$I6aleVNk>OTD7lN^<1lO{F(e??9+oojly3+YW2)W#Gcd=Yc(4J1TSh|yD3t}z* z(v^w2ID)S@t7y4m%q4XbuMtyBYw#jRq@C2VC=Y~pW)0w>bUDPt*+J~}i;-t+C8{-B zCWXTi%R=im)|F8VVDA2zMaU!qFQvu!zz=c-W0QX}pvu^IsH`8m!@pMAZhL2zM#7Ba z7$7(u(|YR)I~297jPbUwPjQVEr#OA$0lj9ee zP1>YAprF+$X>FiOuc!t7W@qa?JU#)g`(gmIBDD$LEk`1mQiD_L!F#QFufTLJpn-}7 zV5{%Uyy}_ef36MfHvY|lvuLr=605(=v6Yvz^*Ro0vX{#5v%nfl$Vu~L1mn98edcTf zjer?w0$0wP5hZc4`C#v8?o?VrnnBF7bvkv$j|4I+pn?}z1gOPXGbvaKlSZWD@KFgg z-|9iGwf{YO*WLIKss;u#H^DTq$EDH4@hw*gTNonI^2QB6U1xhlD$5J;L;UfdEv*KT zzr%@n=rizhen407jKyc5%_UCB?xgP6O?LQr<<3ui9kt!?Fp+daRYZalvLT z^b|g^BBPz#KIG2a%sXGCP({o6qf%5vQvW~$f@IVUzw*%;wa`-~!>o!RI5B5XD-KMl z%G*dcWMcu&cBq3qj+h3!b0(U#oRD?UAShMox36lDmGzOU_n2QSJ1aqU4%`7hGJ;j? zkA=L_KjOZ2ALUZ9bheL%6m1NMIt-Ie58Re0>Q8_a7oHUgqt!G>rA`Yn%eFR9gkakk z4p>sXH~L`OrJF0ks(@{8qQyLyXIeT56oG<=4R?jl@h>gth(gjW8q%%un>J*y;`fkJ z@H}^t&;nl#sv1o{N^0~7^~#}q^z+u$8Bv$jN;nt5L#tUGV2p-Nryb(9kR1auv7cb@ zH$I&2>2~QUc|23{h|yMl5#u`fWOfghjC!@i@QT@M8^)|6j!FZBpOG>WDr0nAsh8b_ zKZY?ni|)HpJ|w9zq4=19ztJ1Nl;ytWY>b;_DN6;>!cogMQ?t?_KUwB_`^-vgwRW002pPlqrk`~xhC^3c0&E#euZ6@S^QEJ1CKvA z;4&*VP0M-h5P18}#|`@bDG52A@NhSZZ=N(T(Y{BBqyITAMEe_!vmx{uzz-oWKz4VjPFb+ruOUMoc(*R41GjD$V+1InGZ*x~=oq&4TH6avy7?5$QQXx>vQeZ=0 z!SU{9g#Oyz6mw_0*$om`H!RJEEwsiylk2;xP+QSh3`V5q*u{>jEFWuUh3Y^O&a{Qg zUT=qVS3^`(-}cMwANUtUl=%d7OtemICQq&=i9>g19%&adBH%Vs=ll%m%(vowfdWR3 z%Y-o=D(@#Hx~TgCsSm|tnXaU~9=3N6k0s0A-JX&M@BM{bZ7sX*4&{mSR$EdG4|@mI znb58iauxxoW$^D@ccv<}?z_6a0;E!cs+%#3asfq1g_H&(QO5aig=k8JoIqhcO2E|r zEOgoX6cJRC_kaY6fH-pTK8^HcWsTvhJ&9VWZ5IaGe4)!>!*I0P5=gdTJ5GYMU<75- z0=3pBl-# zjcV3KRZU0Wtkqd>Q+DpYY4m~1%AbB}$56qy^K`syMj zA^y}Ff?c;L_Xn|$sf&W`0SU_(*M2eR99=E=tN~K8mIjIHT-TxQk@K5gh^79lxHv3i zr}~7LbIbQY1tR-n%_HzURhI$_MR~Ys*-j*P7*sin2CPe@l8$tUu53oqo>x)gm_a{i z?|~^$Icvibii&9Xl`#~h6t$C(ykXsa$+eXP`_ zkiGVV)Wb2^JMIka32m#r;>pW}I3*Oc{#)rW`U!W*7TsArUic3l#sR^xfv8H?q70%s2P!ky z84ux9YIZ;~bly>X`&xn!+u_M#2cVy~7)=uu zrPmy--)Id^A?FPo;iztO6|ZJJFmq@@5q;uE=LtX5lI)Lci6QP@OrkCuOyjxx&zq>JOAFSp z5U0;5dhFlUzP&2Wt;{4m6R7(%@FykL0St~h zuVCk8l|J}~;-3S5UW{!sU4aZvCkNqFJglPYL0W&+^+E5r}r0_(r zjPyuO4entEy_h+kDXP-ak$44pgHR6qItQ$L7SJ$DnrxTCUE9yq6;*q}_4L1UglBH| z*_E+Ix;BH{x}nZUoU(m@Yq!eRCkz)3C!_C6bxq!VBw7k$agrhgzs*_2COFdV6bIBr zqU^gOqyA~;0}iKx)@E48@ED?CYtz;6q+SUq7f+~qtl;8c;@F{5lt3cLypyrqiwQ6` z=zcVHJ^Z`guD*1=N?tl}V~W2rt#W!m;R0A|8XD+{<) zjT&i7F?^9m2J(LK3?`VCS^~FAEiL|jb231~oAQ{Ef%*-q9M8`bzL!4mHjx%ILP1{y z+2m&FVK-~9z-=xq-0VRW&Lt}NW!%fUb2VJ|QP9VP5Cg+&vQ{*yzZM$#2W2vp&Vo)= zMwE1=%bOR(v~eg=IS?W08px6wVk{@2C&uu^4suvRn}-zGq%v%u$`Z|?+aB=2^M|U_ zhmgrguRoz0%5d}aCkcGhnjOvYgeReALi@z#WHg;@$cg5EV{C#eo{upsDS;9k=Uw9n z7qh+nK3vdQ3@L_QH7ekYTBpoy*r?iE_rtd2B@PuG?}iIz3TMw=1TsuQgL8})yK2I? zu)Yu{5VCWn2&OfNBtdUX4hLg`FOLUa{V~Eq#u0hm$`avg*DT~8xL_h|oD2h=u_B#{ ziG<{WPjDiH-mp47Z{QrRoecDi2#+pk_T>y<%ZjpydC#B>QT?9J5YTjN`dF%&B>%A> znGws}v1(>%5uv{+y21p%lY}bq>i8BOys|w?Sh|BifSxIO6z(Gu_xF0e4@j{18KfZI z<_9#OqVid|rDxF3-NFH5^WoR0LuVvLb8_L&xD6YMznW25f;6i;0rn-fh2#DIEh3re z3gC?Pj?=1rM#1Xd$a|+awxMRGh0e0v^049eFmMIyw0})`Ybv_ka0D2IEyyGq$*PZ) z6YA%5`8#6T>}&*`b9!GfPiOZ8+xB$$Uo+^96jij?KfO=9`~=K0U$0PLCJ+R`ihViR zdk%W2w`U9gIm?UlQK*P56d1gFsIO z(H}6#h8*1V9bk_<vj_^n>BN{0gX%VDlL(O|koauR=Lcu(A{=!WFPy>hpD{6QhLxq) z!7^LyP86@A0c(}qjt@pHvBnK66CzD5UqaLAG1mIur?6yN2r66E7dN}&V(`Z#2~AA{ z`2J(n^`8wFM9$Pg(1aCTQf#JwjWAt~Fj-zmgKquu%xG!snRsiAn?BW&RcW*=`*9{7-^Bt zdb3TN-*@LmLsGJty{d(asZdkxCxuY9@nFFkO=tVD%u)jd`qcuXUIhBsn2V;_1`1WIlYA` zN1>JsvQ#EiY?Ui=+Lpc+3fcegxw09e@die6Yz6yo4pKR(gK8t$#DQda^yCG1W}r)< zO;3>o?b`nw`&c;AEL(KR36A0zbVy9)8>h2$=(IGfG&|-1EsOkkW!1-`lgG#{L=)JP zYko%*=-<950H#00;{9ZkSRJ(!)Dd!QIE%1K)_83+8L^6T({3E#L2%j-13xlZgKfQ9GkVmnah6+j(BU zGa4V5`{l{~(I74^GH@4Tbv+O34hR9EH*`)@h%;Z|I*H$DXg;P7q>OG7+$97$Puz!@}z z9hYfucM1X+4JYS8{8pD#$#T@}GiCGU=0HJ1_=zW?S{{*)5ti711^!f66yT9BL}@#G z{=@Lv`G<_fY>4)Th#kgr<*mt&I$K{FJV`aZ)9gvmm#c*O?|9ZPl;=b1 zcXd;{C{7df5%xSnM%sjhR%sxDh5LfVBr$#Ig-1die4_(B=^lGQz&v`7ry&7>mj}}o z0Uu@x&*s0`(%b|+@v3$Blp`B4neSJSXbFbxfDj6L@v2!OOG}#PgffhU-@i&5Qt#Iz zapZ3rJa)g@&S!lOnPhe9@UtG7-L|g}bp5z1NXDI)p8y#M{idMwj~;n1J2EFdcU0Lr zaz(e4Cmi2V}XHQ0|jDJscdIZejQ&zdC}VlZ9r8@_XS zdtHe9)jz2agubs7Z&tO}OlwiQOpz7(@y}xNFRRG&CE$5a=Rw%z;M-p3s!+3b$piSs z;dEnM%!NAcKMo3waJcQ*RM9hEUN=*H`8)SQ8DMNk3xbNELh%3X;by)PI41;RhZAVT z4Bz|-mK^996Oyy%A3_d$iZg^-OjK!I2GY3??vXY3&z&>c;H;e?w!Mu|SdZa@Vk>xU za`8p&mS%(3grIo-%;i`KE~5jFDeE;49;zs!)bYL))mabogWD1PXWql5_#+U_g^N(I zzB%xZYaNv>h<3K`JS zl%w|jctgw&?dm{%vocN_VKSj1Tksy~rkyZE4rz#s$rzXX;Xi(5=weASL+3?$v>tL^ z&??P@{gamxj6a2JCv_xqVa$zg$oLJLcAJilw;M+y4jc2Xczu7-ulJtD820;8Qua?3 zo_AL-dl&F~H>P^MQIClBlngXP|B!`Du<+eX=C}a9OaW(v&%1MX@GtJo@I5q*yJl$= zI&&_3Z(XnadMjRN2^{!+jf;c>x%kane>Kaodq$o60&G7BR9rwTVberjsh?K|sF8LNE&wWXm@dPb)(^iB6Cs;9v@Jkm@b4LS-RR);u~ zD;{VKErbJsE&Tm8RTxfmE(0@j@F%fWBB4a;6M6ha4$5*B&`MqCSr@cSyU!W756z^h zP4}F?2?(g)pChbsxiPARCEj5~AK6pGVCa@&wUWc22Q)s=9oe&ct#JML6&QL=RRAh}t$qe}#M|5C5}ovugcU88~XTM$c`JQfYvV6{H*O|FA>HtvE|<1&G!xNGD) zzFF#A>4nKCYZ#sMKQr#%f2O{^%0GA>k(UY3z|53<<(poSaozxf)JS!wOs-symXbe2 zoKUZvVgx>lJ54?guKavdM9#%P;InIQTQ?#1nMVFnCr;Kzg}v0R$3u1-$h<2MOEZv> zz7}orb@OImtXkiFw{qt!{}6{kofE-~Kpi=!>i?!#f~( zK&{eob7**Z$LEYDOC?5#X(}h@VxOa{SZuVef22x`>V>|&!d{QTJ?S^4ysY$217qqX zjtQ?>i#k`wdYQx+Lm==9K-b zDj?Wp=35QNGFb7yh1|HaT^`lPlqbSZnGc1(yO1CSG^mzMfV+ z$lwvVb#dJO7hT^NoLLlY8M9;C=r2w>wylosFSc!~W81cE+g8U;I@aV(&8t^4@6Esa z3R^p&hF%hX$u4__`KyN%eF%=Mz{#%um6X@gTUwY?&-& z3ZN_-{oE}8xP8H;@_5r<{Gv*pW#$c(vUIiD>AE(4o6&1jt_qBp3L<+vv$6$DmOcA8 z>^tU@9{&ZUw;$*`bNvbe7nj?eh1I{i2*y(NWzO>hj0aHXq}fk=0^(~$apd%;T?#z1 zZjzTbl(iS=br$a~U%&3bM{ows+wdLtifG&_zwE|#iws?F!sf6UoeGU_5D9M~UJ-D^ z>D9EUlDC@-#5Y;i+w}rRppn2ELPBG+!@9x6@~-{juzBtzdHAd#2+#PD!NsEb9$K~o z%~%gor^zP;C^-m!d6B3ZAeV|ElhIu=&;!Z$h+o?##`3elYtY!R9*q>EEM_&)%#{0p|=P_VpaD`wfA%eAkZ0Xk01q&__0G)O%B6 zx3L8W#iEz7;!G`P(m;cz7ZKffO6ZSbGz@3Q(4rHqz(WCd(2lJ#3AspBTCA49^oryQ z7mGj`2ue26N7I}y%==~qW?Y_O4z43W8z)RU4GZ#f5Vdh`nxqh8u$t>km3$#({Lljk z&K-S{conkO@SWmy)`i{pO1%!%9W_zUo*U>%!R6U?<9H0@`E||`rQwKuu#Df^^pPV@ zy%d(JC9*nMP_P{|eR@7OcN#oBz4YAVO-DcWI84T1+dtaWn=s*0;P@%_KHq`JKeHVr zeMukIk(t>qiEZvQ>J;jF&En>Es{6y^m}9uR4JR-sJM_;+K#Ux`&z{rBOg;O4O4+%6 z+7?>ezoe5)vp=w2O{!@9VrcOmWHP`FONfrM&-$}0{_!Ty6OOm#mhx$c7Z=Wvvu!cE zNmDjJFx}Qr)!rAl9%A@G#U9CpFLUT2+La*h($QoNd*m!RAVgy<6MH?1@j5QvzVhO6 z#wGTL&(i`vI{dV!-F6@@j0JJ2e=bzOe;qt~AUyLhf{cK2e8%_#S!ap-4szHyV$^C9 zxHKV5rCYUUh?B1Zq%VLxO^;}ujStwAw%q>5{GVV?vRc7-*={&1JWOa$J?(HxEf)h5 z8;?eFuR3n}CoHr37$+VY%sxgCz)5uHveZs6aJ?_I;!7ytHwGyqx)Rudthl}DZ<#gy zl;UB$rrYE)%lwh*Slc{3c98Ny`Om|J3uwh3#i@mOaC=&lH*G1x>kr2Q@erU+wnD)p z!jXE#N?eGgc+FZ6xDh6ia(;UgC3^C9C&lwRj=RbKMetAehMw{gDtpW22O8gcxcE_3 ze_*6)-c!KJoa?sXKSC}N&reofO5c)kh%AXi;6iu70sS7}Dvz z=kD%*MsnS)d$F|p3czUf?EB60IsSW-RiDRjjXwT-BvU`*Wx)fh4w|t3S>DZl23BUx z`^St|Cx&-QPv9F?Rj>Dba^B9n3fMLgJ_jLF32YV}pm^h^Lt|nv#Pf<$|L9n6*MH4+ zcbr82;HYllcCZ7F$po6w9Es87*q(>Syqj9Xtd5`_Z`L1YZGg(?@zp4`4mP#>3dTcT zBWP2&z-|eRVeix3R(fN4(*xa+oj_ni2TA3e!BzJmLyY2C+Jk5uTiyXJ6cG zJPDx1i~2K(Jlkmw4D}vWel2d?cAMS2r3#i`Ukf`8*052qb?BqxyEDF8zQ1ofj&~Ku zObNI@HF!Syl{+6EiEa*=OtfSexq$DHdJW}AoaB>5XNkEcOB=qZ$h|Yn1s+3Z{Q9@^ zDsDzE8|rsQnm=$pevvk&RcmUam3A>YH@_mX#9&n=c1N#u7-VVGHzujV$phYv>;1FD zC#U_9xpo%O;C1yPgKKK~GH85bDQv5U=aVcUw&y`&cjrj9-KbiUuQ^gRC(A3b!-5=1zn2Szb5^*_zg~ozOCs+69v#ahciB8l# z#`+G0RCyL!$K?3+F3cV1iY!tKJK|DA{z`XZIvNY-aJt|?^nJjH*@gHHkPMGI7&h8; zWH20P-pW5IfOA<$)*;?Sax{ce@s&dfE+$OvF~-UP@gI`AOH~4f7EiJCOCn8&NCQWRIC?CMfDo zJuL|ZId}ju@{@c&yY~QwCBBK&BCTio1gV95w|V;~X`%%t)*n9ksi9I8&zt8BfgKaQ z-D<2IVapx0S;x`eo7_g=AvI89)vK{{k$fRrhhl?P+YF8{YtgN=jw~LCIuRR`ozWeX z)xA>(JN*bV?}N@Yr=n*fzK=KU#P`5UVq85U&WoxHN6O6ewYXr%-(kbzinJz)0EiWod#)>U7w)3JRenNKc~OAKQ`V2 zGH~X{1jWWXhrz(-L6)B$*s2yc22~zQEur<>V5*1dd9wd)>GWxE_v`o*AEQxY7WjX$ zEB}XY(SB#m9kyQL5c^~Ah3R&|t_&i_fM_QL0>B9|BZ{crtbe~~6IJWEr z(_PRE#fH_3u^?*pAOt?wg@LT-60m(iF%`W#cwvJKT>Poh_kou zDPE^Fa}#w{5lLtwcfK3p8w(3ez6)6DoqNbovK|-@E*e%r1%{U5APT=<%xvIDTKxX3 z0N6&qDHKv~SG~zKAMEdI4$Jg65cK}N>@7Sy|7WddK$-VEP3GqFl#<%uj3hSWj_}|D z9RMfx~;ji!G5_Ts@31JpIr08UTD4^sN8~Pay`a!GvCtf>_$`nx5u}6JDUJ= z8Xd+pbz-GLKGU|^QFF19Df^9|?(JnZpg>b+t^(8(dRp&kWHx+$FkEm&m_FeTPiQ?F zT|^DR*b~o`2`u31GyASzM9694F}w17GbS6sr=BtT_kIRRiR9dBUq_zhaf+IeiGC})Eg)iqzx_nGJvZ;4Z(x8uuX z*o?;rPP@7MOCHL!Il>=I8Liv0Br=fePZyrumyV6PE2s!J7pmm(jb&DXEhA!vcTgs` zAN};Z!BxGDyuE|pgozQu&3rSsdaXG-u=)kg<2vvGL#yJpROaH<@QOlla7o|A`c*thPvXcT)bv0^EAxuk~$@Bv?pi zYV&3A*EMJk%)-#x^a4bb1!~qB+0Y=$MSvt&`W;*NVWD!|dhvE=7Vgb7T#JA81@7TM z;XWXvrU!Dhz>1zbM0Fhb?mtT-GSUUbmgvR4U3C-6)k9g2eIZ>VxcXt8)j3;W(PnQV zc`(yn&iLQhR>WSv;y^EcBbU9j71^c}74}Vk9q1Ui-1`RqGLZY7x0?I!{Sfm2%)y*p z=Emd*GM5=V1#Y|J%3Fci^o5{0P@wng6HD9=QhG&Vn4V5MNyQnFK`twDVCk}}-p(E4 z$Ln$0jh}l8+~4}E(nJ%&pVpzaJ;x|@Ym}QuE-)AGmnb5y*B@%1K4TF(1OLY38>VDt zobgn|ZmvE?$nml)jE(?kKknLW`x_PpeKO9qTAeW5^und0V5^eh4GA)7ns=u#uJu>8 zh4ZH=IljCG*l692ICayRq!njs{Mzm@BLO_8$q=omGF2OHQbiLV=zko9)mQ5D;xB(|xv0j5>?zPkvObwARWp!{JkzS;`LCW5NQ; z(dZ55mmx+VznNW;EwP}JGowQV`N$&~c>1M?(i-hpabOQa#KGP*h_D*L(C$)i?~=fm zO0OLtH08Qi)_a7^0=T=*7Vk7tn{iw5ZhJesKY(3-L=&anKqS0ODD>4LpL@O0Yh()^ zxa<=gSJB;jvn0;sv}7-%-UhwGbAv9ViWM|T-#XYu5KG4uB)$!G^~QU1!`<7<+B>^6 z!f5z_h24I{2vx%wVo(ulbWuLw=Fmq~+M;QYs!>Rb)f^{Ex(1Skwlj&|{%}Hp26Xej zOE;APBY=6<9=f-Gy$|;7ag4@Z-~S3&vHd_`1cZ2hPqe&eK{{w32HW?*w12dAfkFv& zA03}6oi-O)tQ1&~-VGA`dWF|z`1G&d#fLuT_ z1|67uKboP{!Plb_R4Bi*-K!G+Cwy0~upDSg27ri7N;GvPY#NBNX6a#cbMdW1{zN{i z;g3*DM3QFn^>@#?>f1-qGaL(H zxOsgI{2r_hvaQ`h6v=iuRH5Z!bQ+SC%4o?H4}=X-?Dr+FPkmX5b~{WUaC_-OkSNqN z7-W&)#t`8wO+5|Z7V8S52x`f>9`?gBCAq*vF8C;sTop2rMB_TeSoGLZICd5^s@bM? z>H*Jll7FZ5hP26SL}qaz7?|ilI>%ZP1*;($Oz?ejw$Z*dqYbdi;i*NKoo#-Mhp6bk9 zkEz|9L2uNWeVdRoz8OH>Z_nJyyWoE)%NU9Pb4? zyKjjxE2#?K>)Vl!h%m-QQ~g)i@O$y_ko0>8H6*a3XVbDuj`g|d7&mF`+U7z+ zb=Cbeh^+f8wOoy^hO55$A@(EN&0q~*-O$jHE5R5p{d7C!BI_gNY1WjfNSS zudV+^7FxrNPs@7`(V6VTh@TwCf|cpKAOo+Th|L@1e?~@+1X&Gu^79g73?_|qZm<5~-JUL1 zuku|2tbA^vy!fivm*u*Vyx%r|uq=>{H?-RHkNzS)L*Gs3vazYZx%vxzvtr&eq{R>| zpi~Dzmo36sWw8F|{C4i5qs{`T7lrcz^J65*|GIRD?{ae^0Le!&FnNl?DJB)pI8EbU z`}f!MKG(V6uhhrOBZGc`P5hoY8fvQNC25t^WRW8^M39$4ZEDTO$gH;*kOl~l>W^Kq z9oS~*iFdZs;Tz2OdbLpOoQF#KU|%wukNir(mD{aTL5C@D zhd74Zba^5JgKDb@jR932d)=M&J-mJoe!Cmi@=g4q0<4{HcA1C)zsur%mtkGX-l{N zA(O>~gtC3TS-;AICr6mk3>acrYTlH;B1`!Y_ak_V-D4oK9n3Y5mq@npCXEi0ED0zN zhyjte%2G<@v;}3m-mVbva3+9gy8g1}CA{KBpKGPsO@mWtCc1RIZhc}xmUHaj5%1<& zvE>dh>V5)c=5H~3A<8X+F)C%Fhl~2_{)^}63uu2G_;K^-Ni;qP>dUKaaL*fSId|up zM8Nr`$O64Uzk6nB`ybklq|192=veTLC*N?4>vxm{)5)ak-_AVNM_!y?n{0hVZwA{7 zWa%_ghgolkKW}tZaPHmMtad_wHs~g4QVkaDzjmI>`!P{5&z5V0(wFa=KM*YO)=g)# z+A<-wfZs|dSBWX5WFw1`-w#!+R3SybNI~8`fsQ;#_Ui$qk5Qy%tGVf(9M*A_mZagI zJa{Sou!(e-MhnEVjk;@~Cprt5*oYW~*ny((Mz*&`w+DsSUMcrELMe6$E;d%)dlg}T zhad^}kQ5ky!G;P;A}N-yT+#}8z%@xCIYSguwWukqb$K8>J8#^?Jman)MK> z6q{is8rQuS)($4Ob&G$U_x7_@eZLlF6DYOSKyMLU*rT$)NGPi|!z68eqD)h)ds{KM znOqJDdFmXHvUR(07`k76W@|NnUu-h~Yv&neWs6tgXUqc+mj^GS*%dT)j{_BDXA``9 zSCe>hxb5X&15L|*8%De)eIQ_LNSHm=(NSg1L@W;*CYn6kn zqr=?i0F~W@&QtifmY|Vb_4Dh!=qj*N7R8+|$g*3e?#} zl$5-*Hu=i?cktm*g zYe`_p2DEjN?Aar0Q_w^~s&rX-A<5EE&w-(S?8AMNsJ|+3;?z+j8&E=40$O!LfU3sC z(#qIkP?#!8x3u{{mb3k6iS(^MzcP#!*bT7+GQK@NvJ7ND*_?4wL!cQ>`Nr;#h~ZPo4OP6K!&uk=uMPG-Wh4@qHZS zn=%C;3;E7P;b8_OXN3yaK;)_9xv{^NWEQE1$BD>!hBLq6kc1if7bACRHsaQnvo0G; zZ8`NT&vCHBX(x?AlXXL~Y5STs^vMB?|HM{xE7WO5s^HF|@%t(7CGm188o;idBLmwg z7B`|tpE@C*5Iy18c{wB!j}=2-JWLjc9T=6WBs66u)G_MB*%l}kS2RYsb&ywRVBDul zrBmiS7OdUu{-L#+j@oJ4pWBM=B}d_mY9am@Py!H>dD=WsHPkVOHGxAPFZlkZ!ytCK-5XtNeN%^V|MCzlJu|#a?g|mGgMVqbzL=VG0f_?(1gi>c$s0^eK|NWN=UlHFjOv|-*rAdm%Yb0nzV0H z+;v(m5n+r6$v8JIh_0J&jKeZ+D8gd$0(E5_#-UuN@#>0b@we&)D$iq;hH~R^7g{aE zAk@R}2D}vI!0uzAB8ncG$6*`AMJMGSXrJbdbSn+B)&Pk$(~2>i?)ntgBJ>d{_sOq& zLXlmu3P(b4i~!|IdlU+qrApQ6bUNus%Pa7zq_Q8%$;ZW7LNd~kcN`?{Ll8MqlCohV z4-3hvkCO7j37P9WhH~KjhB}$TG86r^W<)u`hnVE&%XX&{y>F3Z6Z!1P;Zy(SjCwQG zQb+|w^TaA{z+{>sufZqTT<+s<#~gE+zFOBY!IVl@jixvdN>|)3C{ll}FExC)&B%-e zL9;0J(z$*il8x{O3KrxDNzYh{YD^o+GN+_iQ$`pq`2^)CtE~+2jQIj{MqLcq>RlYh zq>h+d$o@30Kc=e*9*5Dv{Vs!#9#@lt8HEvvX_!h?)Z(S$s$t+Vsoh@PCZuvcByU)8 ziir2$x?k5~a@cfqvl^u~D+B8ffhSWD;rJfW&bo(|$K6*~8M={swmvR3WiCh9b9qla z%+HPxC>}kM=0X$e1TRUB)?D=0%*KHBWGRVdS492|3!N`g)=TGQpr}ZD9iYt{=*OLV2jE#*_w+YyjN(kRrF#*wql80&i2qGQ@pV11Dm$y;6;C` zk3ZG+>I8hN;Ks{-U?$n?f?gR1QoApGq9(t>?Vfp*f4^^QIl?~&CC*OmaFHrd%&|#t z;_5yh*4}m_U9r2jc}7EGZ_|88fzFG%ir$n%b=1*4P%-Y>oKEv_C;Du;4RL6}F6v6T ze1#=LdBv&>#QU?F+%>PFbTk?ZupKBU9gxmq@IpYm%KoGip8?*JEmw4#)*AkL*)0Q2 z!r#fzY7K{J>$Os~R>Q@G`kXkax`^#Zy)3KjY-E2Q1*$wLglt^xK9Z9XEdPAqRb9Am z4LQRXPnc$-_oq>wqRD3A44N!*Rh2evh-l!myu9luC-`vmP0Iy-=mS*zvTMm*Ki4;| zd&SS!HD1A)+fMRELdQM=<>(!3QC;I~p+}P3s5VhvuUFZ~gN+L4yq+hkS|5rl6wm+8 z^;!2Fx?nA?>!38RBeUb07y?-0Smj)rmb?!;vK0e%^qzOZ1o5D(vHzP05qYuK_XC$> zOc;FVc1`|hECjBX3C!+Yk0?{m>{#}Xh2m|Z#DqZgv>=cMa$wBAZ2;em3MN7;no-Hx z5KY#E3OW`|6_*52z~ozV$Q;InVBi{p_-Bn~x$ zJz;$N#Q?qzffJ&kkegV$NZpSd$(v*?aZ_pusV;fsaYVtO;jlOe)&|;~jKYYu(7;@p z#;Enj4(2beqGht8DQiA`T3|}Cjw@*LRtQ%IJiYW&ukT;aUA5q=XXk;X=O<;P6IaAR zC8Dc-Y3>-qtLDH~>rtiIElO;lTy9YER6)WaadP>(sRgH*6%`Q8VM`2+rvMT+FDL7p zMS?&UEd{xSsbQ8lfkf^WOIQN6rwTM~4*E0Y)k#V^8BqM4D5;wRo)!4#Cm`f^{N9M5 z7np&j=^OZ+GWuNdZ0Qm2M!d-;$P{h<_9SsOfLO>=i3*-&i?}z4HGZ*PV!l7qf(lF) z5V*x`F@>L6Lomh$Qz_T#e$r@Bgv%5coM9=Pl_pHCBu2gE$3+tggJGZfSN|7rf{oy> z^!Um1){QXS^2Ji@Cdb(WPVrPkaWYTAhm=-*J8gl0 zkrPbg8;z&*IGIF!GZQl|#|+2PF^rpx#L(%oU!-zk%;>UJAR~Z@CXD?$X*%P$n+O<5 zYxX`?O@mpvdHUSxv3&=IP&s9EE)4%e5}GwC#4x{#$P{< zniSMu4Um#u7)*o(;*bU`KQ7@!5@cae{zxWU!E-Rm6`A(ahHGv)aiRJ(IjCtWr(vVsetFS7kHn$)5Hp!80%I#5Dx)`ur-~3G?`Q3q< zc#|ZL3pp$c7i9t^ZA79k3+E`KG@-ey-I*D?MRN-&%95Jhity6A!fKplo3G^sY?K}~ z>KEO&o<44`wpwGw5USw}35)a6p<%cY8F|{}tU+61_StDR_ch?( z2nvuS1uM_rx@L}INVU%AP-ti=FPDwb~WRj;>$QQ;tNdW#Lk2O z&c*8x^z5mKXjmSlGn9TBX8H}C%Tu~gSC}_vXu4J78v7cGK`k89qO-EA%qVNwt~bk9 zLla@P^tgE2dAXR$+(`W?*Qeei#Twhd(3h6DU{HQ`7|W9@*jFp_!SIrm9IWQK&riuxZQuSjpU`?9-Y=(`6jVY^C zDK~1;v-qO0OkE*77sAx!tO!146FIz=gg3h)@=X@Y63d-}{h3KFki_U%lEx%s@~f|o zi%^l!l8-}mqfkhz?=f?+yfhBBP_(XUx2_&^hIBGrhCOfR4;96i+evDP{mL~VS`(82 zr^@DL$-dtuXvBRx@{ky*tm?shor5O-r z$DFN*DPf6C@3aw;Hk}`nd@2S?(gpMQG%zebm{-mMQ4@-=@wo5bEqs{%=h!`T1?`K_ zH0PSZi;yON+a`X6CZB&Garg^wBzx50D-|p`wjCoo1=Z5_NN54p|3fP3_Sbiu5eq|t z=&9ANJfEfS7UcB|-zQ3={4C%9)XaT%vfL#{Lg>9+`!>W1L;i$;#49B#Pyo4 zSrh&)LabK|Rdq|b%owzd{Ic!Q^^E7q$r~uXwGp`kqv5y_&C8RtKum_DTKg=8yeJ6h zyRWTw79LX+41S{MiyQ$I!Efm*`yt#Vz^kAFjeTN#F z;hwjj+0XjCV($+uQLp;o!@iAZN-cyVja4H|t_LKY?|rH#(OnQ+Xg?OY2`v_IoW}po zj!guXj~PdNq(QT6Y%6Uu!vdEl@LD~0z_J=@`3U}^VZ+y z3>4soMfA<;3^qV3=~AXC{pgo8@E6^rrL58+AU#?_UB@8ZdUmx$;{m25p>&Gl@dN{-_Fv=4TSnxJi9JE>I5DH3XLNbVSIUoI=eCH=;i449$U0-JQXrlLF|D)N`JN1 z9EoqImVo=w$6ub^W?;DsI_cpK*5skn^~*j{vpP9krLh9|))0$M!DQV5=>$kEE~42d z7BFeZye?;DX_*Qqvlmh?A-7Q2b}a2K1WOZ$wu_GFdxYbjQnj8;H~Ffp>@M2`deUIe zypj4&8!5xt1EM~H(R!4t8T8B@X}_=_Sumb1V(-D6&QZK-st4iIQtR(3-D#qoTE* z%HH-{{^j^+d9!m+Hd^&i`$?u=FhHb(SwP~`95PwwmswCO8vEFjm({xn+iU8g@f{EK z<+B+`zfH8GR#uRn89E_I6s*pNXcwJJ+D(1jSQo;^y?o1Vc+p;8%THX!&6;*TKLNet zj#J*Ge=w+N&h{7bXk$Z(f3F!#Fp;yAO3xZ_WwpiFHYe2Rf4*L?2kJlY zF5k1Z{OfaW`9MUdQX6CKJ1>v99I?Fkt|gHMR63|FZ+W=PMYV0v46gSC=X4m z;pd`3(NL*nKxBNXrXSeS{gQ6>Rbi=OYC#s=yi^{ATC<$8NpuCeJZ-M`?ySjY{(6>M z3P!HNq+6u=DzAuCG|N{89iUBeEJBS##S4Lw?1XCgDsEq<{K)&{upOi;TvRT<+bk)x zTce(Lo#_%_Y4U?&v}E@!jF^3f>J{-;P-cD=@NFRwWYNR4MW1kcMfn{d@oxp)&d6ez z@V!~-vO`e?s$Nsg?~FBYYq66 zG3{Z87+2r7@koUt$O)m05bzbyOq?%N)h@Bke6`U&`^>uhPrD5St6kkvuOuO^-$4V` z(|=S(_TK}B`*&BbLCR>WiYSV#FV8-08Vj8Kw4vVQx-7EfB3>!9CVWU-ii}M zo?BUy^}O-G$OpG76_zh)%Oo{$1c=l4`zY6I)kQH!vVdNE3BRgyg7L??X3FgG3)QIZq3Pm51CLcaYmzI>L z05|{$vhuXLW!ncY(G-dKv`4qA4q=`cL9l)1W#`Sramts%e|iPqlxF=CU`nU+>ubU4 z=GpL2iZnC83*LGBWy&FVcf$S*y^lrxz!X~>4ti1M)}mNc>XOZsFQUq06=c1vsbb!r z6#g|Y@s|ds=q{f!#uMLIrQAt8aYQ5NM>%DNzPfe((M4t+h8?Ck;3OK1OUZleF%ehQ zCD*LxLc4xP#*P>EjL*szZpyo|4;HGEA#9EXWFh%E>SZKIo{A2U!PAj8pCGiqFVe* zr*eOUA%AF)xTObDNJg9`sktl0#B+=o?n9+*feK}|0;Ejo)5pf^#tg=JXk?s?7=pFN z8c~Ln_xNYuc(c%MPL%+*0bw6eUSSOz#g9h=oER7lpy3)RRO#W z>T5Jz#aA4FXM%EHB&%CNX)lGVN-xu!MRn-P#7{Tc5BGAFi~2U504}g7MgnRpe)zQH zWeb#9@!(3ole1SC!{56V^1C$LE@UP30X%-4^-B)j3a34i&pNd~8yCEpbDa7&n1evZ>%v?y?;w8BM| z*&LEesRO~KDeb_|_92UCk6aMSJ<3X_a^>n1IqidX|IkAc8YTxS1e~cC!9C_N`2!5- zW78~jiZ-Xn;%Oxw8A+T)>VuOA8vU#lF)Acc!Hm&1*^hA4`tHqDS9mb_TpxbODE+4v zV40j(!)!u!1@p>Or+(JP;O)hF)VWCy#*0%@#)MyqWGcAax=H@5)s5iBG-y{FCa!I9 zPdX~0u~2yQ60m6WUqbGDq&Tdr9yp$mpnR3raBQ8 zX~`!rL4V(CIG!(K=2;eIO%}#;Dxc?S{sWii#?F_#q7Jl^KI+%yLBqJ@Xt80cB9jxH zVUE0AWwj}op0027EuuQ#Kh5{gvTb#?PsM-KZj5V03*li2?LXS#-0Km`%iL*|e=E6b z+u9rI8$Eow?>)`IcOgOVd_5i~t!2UO<4vfX_h$1+ZyBSE#jE|5Un=(iYxZOJ% z)CfL@$+6BkhJqS(q;r)Ko1k}3nwSgOO6W5XypW1gjtXfS3zDJDOK|B6+wBd>sL!dRE`p{F-^IEG za2?%&c)BrN|9q-83Qne6Q%2K6?ayQ*iG_l{4rURJV~XfQa&=~7NYNBE^p^^yvm~we z0hfXNy`+}p(RY%7Sd%L+G(j0@N@ZsRb!)*`Jaut{7+Jj+5lwHS7Rn)KgQh+%Myp(b zKoxFzjJaBR`6zMoW%3>;gpWCBcAO04o(QZ*Bcw|?4cFYf;tVOqL|^knzFl?GLRgzQ z`TTf$Cg$*@E2gv|;_h$GYc@C#?vmGnEyk)@2IP19n?&oX`j(`2zBGE`BcREA@TKaY z%{p5Kiu|liDoA&LoGK*_bCa~cLeG>DJ{mm;yw-w_2lr}#6TwW(O|euQNW;_KUduQszJ9R)HX_9Pq%ZdJUKOg{icj5L!Ee<55$FdEj8kIL z{G%phl(s9d)R>Us6SpCCvtGZ-uk@gcIbmGHLHTzIZdWaV?)_d)+FD4iD5`irb=<69 z2(SPn!6`{0_yk=(0g;#p`4|qD@`NBw08i2+w!}wOUzk>)o6HujkaeVBFCNUdBB-Dd z9IN~~50%!N@?DJC3b~SnBJr<9A@Oen#j@KBm1s2|FyvVGb`ZTI)X-{%WY}DjeKZLG zn(IWM?eY<<`_qap!g)Eq7!t}l#mKuT(}?>o(K_$n>tida+$2-TE!)`o__99;`WzZS zeiG_sCZJ_P9To~O+Mp#aPz0bsz?7sZN!Z_zDI3>Y6`IPnte@mgtpKl6^+4prqKA8_4$=ff+sGH3pVw zCZu%+#W!XF-|Q3_7eY(S$L6b6(#S5xpH}JLz0_&)_{P?i;7pb=sniq|wI<$;05J)o zphE^zqSR&Z97k;`c8}=9-zKa?7QECq5p{gozbLYQ$qU%iCyCR^Yme&wLSh1lE70#C zDl0uXzuL#eF{wAV&(hySo$JRw$p6vh>Z-Y{UW{OT$3MS0`BglsE1=*VJyI=K9oJ&)Lw^2LoInmW$ofJ~r3CE+S zT?icO9z3PQ8FL!mgTUd7)eM17;T(plI&haa_3=cxmBCUZ7jMu}v`kURkP&osxa&-} z;~5iFivbrMJk%Y1BcWQdizlLbY#P1K`1hv#PN{TLNh&iYITLY@@aC)n?i6U2SQO&i z59@W8sSG&+7Q3_qyE$X&sqL>--G#}n>x-5TEB$Fx+Av;Pr-h6vY7hH>ndRLsz~XT8 zOIUft*@fk74QYIb`SauNd$y5WPe|Yw^yUB5G5QX&z|ul^K4?L=eBZA}_Hoxxk3LYQ zx_Zq#G32%(obf#%8r^vUk^irq%16AHc6XXtU=bt>d>R*eCPN{EmRRgu4(Q{so9>?4 znBwA4T4b>T==%k0Y0fa-cU=G1E#fj;;71WH8Az@-1$1qi{bV0^^v#NyUmlkJELZG{ zjbTCx+JgcCMnXQ+fdKsyx^l%&i4uW~-^O>`leJYP^Go?sZ7>PV8)7(;Rn%uph^b17 zmsenNYcaTQA?HBX8r-Jb&$uMb5&#``>*(eAlnhd6I4 zAN}o8&a_Q(WS6c@bpfV9t& zT30oBvS5Vgilna6Nb4DWEbhvaE+Shhz(5XQN?}YR%`FNj7hIgge@2T`2uzF1gisLV zQ4|rPp=NEl=t1Hu&9_m04Cmn!x_nHo#(z(cQCjkWJ3+=)aIZ!uPv0nAjNY74hBeRu zwFE`5NuUlY8(l_)RM$kogyw~nEUyip^>?3E=KgIk0;&eUU+ZD{Vv&Qa?!}gaQ2-g5 z#RwwK%oW;-H?kBU;6HZAf$c*7C+86Fm!0C6g3QXQ5~|Yi8$jAcI$iX+=y&qMuTYm= zHg5#{xm1&%-;x-R3|_|r%%&;7cJAOx3wV(1a0Nt^xlZDK)&GLj=%G{C9e0kVZ89pPd>^`oLpXvN;XrX(91h5qlzWv~W zSAdls2}JF8(R?6K&TJE1EP(^_bLYLmP2n2@NKkrA9)R%OZ$(O+N!jBrEl3nklrp95 z$OThgzQc-Zp)6!TEK>WGEGj~TFaZZWC#{zxS%gBu|>(uep%5zf~vjB=<28bv3K6#bp@`6Ubxj7$tfxqX*`B1!=J$XtJ zi}Jfo`3FCWlNhm*q@^+Nu=EPSAV+it*UeDRMmOUbACJ}F4E2LElqu3-4Tdxialr?g*Ay+J!V0vYJV$V zf$9oxfAd@g{9=7w1nY2jF|T4c1J+}xmG9naboY7(&PTV5`4}?gIt6>tE!_?VUy?2T zs@T3<>*GWG7Cxf1qwHZ;II2fBU>0sjXT=fKq}5k2zNy>UGx^9OR_^tV_1;2rVLslF ztFn1X#iHy7N#LJIZazX2;HQ2aYH0kgE*Z>Xnd&cp4cJqv1oh~1ovh0g^uRNig1_&p ze(!go6Nu(YvWHdy4zyaMvYu=P%%sdw+aKPwR?ZOqu0DSp*)nqt`rBPEqBl~dGo(ZM z&BeBj=AMb^xwz8u1yT{5uQxJvmnLibw(!?4rMRgXJ>ZPB0$ZQ@JJ(M%Ur18bEU1;| z-{xnCtb#qRivI#dJtHtZ?--1X)}F2@vhJ&63!;ai-KLU@nyFbcMU<>Wl#YMQ2yQ=O z+Et@-WQpdM{HmlsYh&clTfr$AUQ*@wpq=hH`SAwY^EOV`F;g-8!>Z*M?iE_+*)mg~Z=ym;${3+8*VNsQO?;4fWwPtLslIP1Tm zOMO*cI<4ox?EI?d>Dz02BlU~xbH9*qU(9v(|4`yX82%BTx$VDnA8?{h$^d`ca{~Up z3sE_KRP&e&qXxyO3oojpQ=^T5E(<`tbrCDu(N`&zh`m5vgv6rXH~YuPeEhMaiijmr zn%mEGjo^Oah+~$2KV$pc6d7eEKSwfENkTcNT7r9rwfN5(8NAv4BTf%I#T^LWyr8}g zf_FP}7=UKoQH3s~7ZK{0?m*2!E3JWbg>Z&DJ_C7;+9S_fecIycsHRI^x;6Uh)!aSbrk;GC>t-HCZi0ao;^*yi4{et#y@%fb zj-AJNeJx;k{rP8?@0iI|i{(i>D=U648SdWTPk$Sp--i7028V3HWADLY2lef9e$TL* z;5#6wv2~$p@5cE|r*{VYuyS5C6WmSnTv*s`I&HE6yIRABt)S)hI+HfodqeWiV8^dX@Pgh`7(1ka^inCLtefZR=`DpOrXYl)0I&e=RcoX&p z=;s9LoT&iwY`9-9HWoaZK*1pIEwDYb-#+B;=`iIL+-=7hns-&Wv%a#wCJxtjpV)>( zz?lC%$^W8MKpIyh5R%oU>fK455!BS5clO9IxqDxCqSxe+6uo;#+>tW_PwBSCygUE= zJU6riIDC`Jh#ZYFVt>@mQ|H^YdQ30UQ)Kop?1jJ*!p`Lbm%ULT{QBDPurp}nzdCJe zdU(C|U!8uaxi;z9b3goU)~N;;IE5cvfJlG53YOMD2DVX(EP#2a1vDG+{w~o;i!L03 zTt-Mv)IDlZ{}0hCsQFb8t-{Exgl7pBpY zxsZL{9$Y5;5K@RU?KrFU-v;@zsk%7N=smnLJ-xl(^o90zPq)yK{k+?tGluRt!mqc3 zXZ>Ex+WK4A0^D~8-kD;LJW>HV9Pw4@hoP(({s&&{McLE75;`M|>=8sMXxHCD_v32Qs0Bb(7DzKtb0bi;klV}Xl4-x>l9x&k0^wDtmw{W zNgBf%gggKq)GgO|8`F}3uTClcLsRr*zx-cMr4HT8@O2D9e6_vVo?BAZI&RZ#0KkPD ziumKWiNHH3w{LyQSwwi?9oK{{!LxV&9ECp6J|%kJ=|k8C-QRqWKJ@w?+*XS~jrKg< zz9f6mOc!Vh0Zrt)=>|JdGl;|h@4U%R%r(CL_Xz&$PuiD4XMQ4iyu5)swiU`YWRWfi z7DX7kSH$$(mMjQa>s&^yuf0R&s^|s!oT9o0*11vziC5#Ftd|QJ6XbrdfMhT%=U8C1y-56j5mvSbocaze|7)P% zyy95LGlN8VHHI#TpxLW%#LWaDnof?;eaN>LKbPEivTvEnePsl>uE)z8Zyi5w-6G8Y z3QzPVc44Y){@44^X(EGP5qHwrKRy7h=ptOrR$KeMb3K&62t%jvJM?wPz`BS-Z_zVM zTyKL%i2@RKjtfJEq*Sr---qGN3T49Fv!449 z`e{6&YUwX;1M9o*)}bez2PWoXxI$EO$w8O*0&vpxvjkVNbwG>iA7rpA9r{Y)`5h$Y z8b}RyufxkM65*~W(C?_<>&ZDy;xuGayRNcHHKJCo6jjiZHOXESH!p{M-T=>%{cOYt zroRcBMU$dK|7W9TLfiurPY4SFizE->>vgyK-plT~yTgD-;Mkkj?I7I)X?It8_tQ6sYR`W4VAg%cjjnQN(&Jh5 z`-o`%$POfgDwrN%FKe}*8`xEv6|t++=Qn26w0$(Fe3urN!`{ty7y80wh;hX|bozr_ zj29`6|0wa##>Li67`_OMy4=&1e-ZaJ1j;Ub#ISegVzC+Bcj+gvy>#FUf%OJH$%pP~ z*X*f>59b%AYf8!SNcfBwG0PdvNVdf*REz7d+EBywp&mF~%RfjFg9uUf-=h)*MaDgu zZ$-O9?-%q?QrGZkDX&ubUi{nHD$y_ahIswJUE#Ahbsf+iP9AeDLcQny4X@J82L&B> zb-4dmGW1;&5vGEZAtE)J#U(4nuzdk{urZXb?Op!&|J4Ef$3DLiG_{>#hCN7gM99Gb zEVEc@oBRdvJ#s&j5|h}-qu7N`mNqk-F$<`0P3MnSz=*!A$Q>55V2_pgVlYf&NfxNU zE^fobV}^-myO`=JCJw?pQ85P1DYs3;a*A6uv5HH-l09t}0t`XcJ=W=Zg2C5AzTehj z&s&IGNB#IUA?=ZQsIJh{`Nou}Fl4iaEb2GhA#EskQQW2u(@8bBSl$RS^PAk}K~c@19EEO9w{Y{mbV5zwvec z!}y+)UjUJhR&2Vp8H&$C(-cUD6^5x5fN{fg5sw2IbP#M;6o37ocUTxnL>Pk>^au?o z$|u9NrX5JN=Ev4j3@7Mz|77;FN+j0oO`$uorlzliWR*8fFvL=_2*kHd6 znE6ad3x#?5P-C$Q155Ron0R+vQ2VI}Wg!lb>QU(rvV9C*P&b644p`h?cdy%&# z32~+vg95V=GJ!ME6y1r5W|9|XQbu@Y_OR#c`mcCbU3QmB3F{Lr3l5J)P?y<}neXZs z?dyk5dm6|06>u{soUD6Ae5Ng%!I9+B3b=Y6y7c}kH_ue?F6H^RAK@T&=;TcJh%EDO zBrPY5rZp~dBEB_QC066C=WkokW>@84yXitR`RNwL*`66?TF8lCLBN(h$j;cQeqkl~ z04nfEmIOaKhyD&e`fBb5Tz_4q;nmGZ*n-}1O*zDl_3G-Vv{rrNY7#tiQ>K1@c`Z&B zBz|%4wGB7Ncb2m*RKI&pa3Oqp-VlL(Jpv!iyl_YR?-PS!atix5iTREfZh!S!V5@^h zDuS7E^jrDFw+_^kxAdRk87p7bzyjy@57*F}*1{*?dB7v6^#OEnAAAMhZMF(owtBa> z0il4l_JF$H+qb#?044+4KZze;0@UG6#hO!)}G3!M-6O0pHc?zZITAv(5b;Lt5{Ge%;!KjN}4Wg3gW~;T64s zH@$^thqzx0~cK_)g#1g{x1TMh>BX4b}V-okyx%M6>i|!}ipDvc*y~O}J;ij;R8m&{iot zAYaQH$wRGLvo@w6D%*$e3bu8!IZANehw}GymttZ@L zt<&-bMfALSM$hR%Y9Vp0ZS-<3w17r_#5|;_u+&$de7*bkt zX~qWb87K<;FV7{7#}z1Qo{dNqslyhjfQO7<-)Wpucj>zh2F9pa;rBIK(oRxi{@#}1 z{{7>EJ;Z!-ykt&aLqE2J@3Y)wV`>8pGc}fKPh@luHJ?7#@82kUzL6BR#Q5Gmg1@aD z*cQLCgV{pJPUkkFmaOKH0gzNt=5NK98fELu?VYEGIfcs)2?}K4+ayg`+BOCnk~e!z{J+)m_YPLrCwV) zFO3g))mahMS%bkx-WcCPb99OMSlD-!X)z|`H+LcPhog&ZaZna+x{8;ob;AH^M*KGBdT+zybR(hZ^y z;|}k+pF^7I`a6>MO&jh;Y~bMyz&)DaG)%N6J-LBTbD4#a4#Lrq*7S}66;|J3F&JN? zT*}cJN{gP|9L+(1D^Am#!}x~x#SWruM!a;|Suq`_Om_BZ4e^pLgwBDy_qzaVJzn>8 zIRU~%V>yDRvoKey6N@LT^}*Y6etz{Y+RvId)E|~1<&hP!I6vUAsyC#K(eVK`ytI5~ zufSve%b8O}>}0ZnRFVU8;g?iQ*E!JC#-YMKcqce6?t!Z2TjEr{wpGEA$wqwaXWPLV z%o4r*#AlTv?EVnG=>@lHqsS61?%hP6erzk9hTdHZ=rbDO|U{T+CR`rad9qTd%Z)r)^?0T3M*r1SX9 zPRvK+4}0ot6E9iGk2BTa7|Y8NySK-+!12*X>2mW1SQF0)++Z18G*jYkg|lZ?f|BbD z*cjm}QXLHW6sP_4&RWP?U?i;Y5FR1ob?}yBhl^V5e*htn7?JLb@mc}<(&hKE`pOZ!bfwMptot= zG`e9O6;#SkRS(oFuIr{XOm-xE&8VKuw#+>!uDf|kV7(5|{GWD7l@fC=cgc+fG#PkO zek@;5R_xpUkTCBU@JukrIn{|@csGVF%^+;$H%xG~*Q2KQ z@+_ht9FaM(roapL`NXGmQZd-fvG*asrg!Q++adbBtNGQ_cAgbH(?dn@Avf(1O?S|> zIC-d5XDUo6G2k2wevdPP7$k;rS#$N+e}B0F+(cd)XRKO9Qm|Y5f~}h+XXW84_Vb=T zI@%Zg^uJF8|7rvY1aA}5etflOP`!IL~iI{ zM#~;WuS7!eE=Ep%r1wDC0eZ}3BI=FdmiHp{$_7uZ7uhoF(xn?q&b!NfIbz(kC1`Hi zd+Wc`FBu-=M2e@4EL^tx3eC%rc06xr`xhl1e%G^@T zMnzZjI%Cp}=zsT9+q@JuP?-ZNWsBx^Ejj!9g-cs73B`!L^9WR-y!pbVrDB5LDHr=0 zMJx_qL_@nD+uD7*t$gW=8{XU6)YviS45?`(wYJpkDyXh+^9l_~+a;j6+74!l>D24C z7*nraZV%R$u%Gr{tf%Str%F+56-|k)9zF*YWeHEDPIAmE3FvgwV6=voevvl0A7U&v z;8tKK8I}8%Dxz0Y4GD}E<~6N!Bh5DP6ndL|YoWQKwHXAZi&9;%BZ=Qi?$D7i;*@Ok zPqmih$1WAZp=!I^8MW}h_r~s{I)4F>n!Zn^S&rjoto7%0p&kC{(15K2T8jRxZx#Xm z0K)|KLiHUy<`sx-pWKdUIxNSQAkR-=rmvkE>~t!#YQ?4r4x_R=Tb0`@oUeb@Ze{OO z1I?HaUMd^^B_1UoL>_m(-Zmf$<;7bJ9`&WN%9WTm^rSBRK$E^N$;ex+TQr}-jMlY< zJC+lkxfWllE0urhQBGgxE1?IFo(rfvq_GD6>53VOv||`YYkb7*unCGj(2LK_)CV2V&0YNJ`N zd?>C?8M?p%($*h|^bq-3N$s>*lhRXRL^}oC=fQ}&wT4zPu2zaBBZ&+frkzi7zljOj z1MsIKdbs8bVsbL%zZyzSR!X#eQ^*y!DrZMH$^4aE8cdK^gN5&DMBt4XURF0DuELHd z?;ci4g1;18+Fk9SAMP!m2r?XfC2X^I;Wm45Op=|F2-mgR#31$fLBWKC&xtUIu2WGn zhNLiCgQ+8cwbl@2Ube`{SmxdM=e)fLFQw%Cx0XwYEoXXW-86A=CrXBw!;Mlbf8FTR zXeEiqbZL%*wThe^x3qw_1{YSkHN)N~f+eH0c_WLp(@_NyHX~z%lJDZ_ZOJXQ^tMzA z9i)Gi`0~O0h^l6oc}nKo^tDDBwM__e#6qkm?n(2o#QPRtf1LX@L?A20c5Ba4pgyDJ zfN#y5su84BU!*u8&9`1WZaa9D;c@$wWU~=s-{Jjup1NJ!D$84oq{yP78@{7o{Kv4e zn)TqX|5KrJet9SFt(Zz}V&cJ9NlpBjM+dLq!BhTpq(lzJQ&Rlq2?xPyMzgrfH%0_P ztkP|q4K8HqDqwdxtqG>ntkgE7RBpxDfu*z$)Q!oxc8GGr;dZ>V{-b|rUFx`%u(Z}u zrItsmTC3ou)9~)J#$=s3AhJ1e_>#!dC)N#_LEO%hj5CYn2tO*N5nHnJkIl*R_ zvnu63|BeU_Gb4pknQ}_8DwL7nH*!|Cf1dl@%J|mCnS2>d@hVkyOjbsM^+K{uHiC+YYZb5htL};bZ0bl?^Cr|c!-uI7v&OMWw zA{~4?e55PU^d{Cqi0iMVf6M6S!G zaxq`yS=-xRuqvxF!E~S^x0o9b_=?P4fvWgLF+#=Pt7uf*g4C3LKgrI^}X(_*e4PhPe6JYWB9;Luq{^RjwTwsUu*_Wrmh6%$(!d6J$6uF;!?*{BcFt$-b@5RN!jn5%WDk*_l zWQtqT!><{c#=G;y%^@G)17PWm|8ZOQZ{M5|R1_VlDV8cC;46L})SUix zi5a$6pIR-xpy`}Ez;*6DjWowV{;K7z4w8(J{9Ms>LqD*_ud2C-Ob;8n5Q#F`J&)Ge4uR z0KXhbx=v`nG!uJDWPZ%xiQ79K+~Se>vvnb%?VM0Hyk3kOEv>{DJjeGI?^wnXo5qn=E##7!^3W*HcLPDy= z;O4=l9@SIgv{7rY(^E9z*|7PEvQkkarqhd>hV5+AifNbb(i&F7gS3va!p^F)oMv+w zIzVQ4G5JIw(J&7GWO48gKh1Pd2hTAu(z9Pe-7aZ-eg20ZJHpgyIKnL|#C3mY2OZIE z$RVnxO`@$j_UNeq!GhJGJg*iBq4^mLe*zM#n2g|WyCE$#*Whr(kBy7W8Xb0~ONDC zdsj!moX=|VYiw*+$(6JHSdw{%QQ;!^tlY_OI$XiGHJK4a(L{t+`_HE;@4ShK;+ zPRqo2%CxlKz*i28Ri^d>kX(3+TUB%B{18iAyy5WgP|!oddoQ5&j*wFFh;;N3wWK%Q z5o_5!+*QNWz>s9+7o5mN5>L38=yOb#)vq?u)T3S_NnUa)r9o@+VJWIGw8%)%r~Ft8oXbe?bu7031=FysgN)a zXKiAj<+CM+tHQ)-EdFYX(yWWQ!h@_R=*1iP*fuSF3O%h%-DW3i9Jp6=Q)Q&Yc*pUL!AzRkWuP|G0Lb^;c_OXOZfO@en)mz zJ${!GsFLpUmlKBK&OS0{lgi2s%gNJF^*1Yn1QfB?s{f=f-NvYSq%$Hr@Fm z5*~KiGAM>F*9<@#BfgAe7rFcMwb2>Jv4TOkD_cQiCnq|5zcBdEFof9*A)Ug^%m6BU ze}`xnbA;U1vISC|F+z!A3%0ck*(PNb{^AW;#5QL&&OBy7ISN+;3K4D)GRWvoUaaNV zjcffEi-CLelDJo>l1B;#u8J4M_h-j<;l|)!mvnS{32{s+zT~yfxr5TsiX8HttTa6i z-b9YS&vX`3#(3Lt{Ym8h);v}P+&TA--)J0bO_^6@Sjh6<oZ(7U!uH0Dwh@rscIHYvQ+5+C#k zoZqoJcn#36UW&dA29A5uN=YWp2g9?j{k=-SI^3T0+{g4%b4k;Add4ZA0ZUstxUM zV%qcP&Bs(3Rw*Ol`%c3?Z1a<}hcwNgz8TfA72%UO|0}qC3Dq;uwKe{hwM<{G6=uM6 zjCU`zX9N-g$jy#iS@p`$>)Zu3h8V8flztjxGs4>Oa?x3(8bC{?X47^Fb1K}yeJTK$y25xJ?j6_f& z0lb<>yOXUu5$2dalOqtTxZFWxj#>EbhV(}p(PB=l6D zV5}odDaKIB#COOKorURJc%-)Eq@(13tGtqHRFzL31BB*{{1Y6hRF^-;bd+(2$s)w7 z_sM6ZFpiEIv{9Qp0cQa$Ct5#65r!>sS%pqKN9@MgkenIn`c*3kYw_c{@O#_VQR&*` zW8r`DR;b41_w{9l{pQVs{PZzr9`%AHT0b8a1d2V|mq>(jnE?tMXcz-nPkF`2+^{%}H)EhvMPVjKI5YI7anIo3tfl zJY~h`)Zss!<$Diegt1&n{~ZGRms3sLLUZyrK&J)`Nt;Y*FnCZRjo2f5i2o{LXC}18 z>iYY$gxDZD)DeSljjt=A!;;B0SAm=|Ju577A0yf}ukWikYL+h|br}gI-lhp1c&6QI z82mz!qN1~YPH<y|Iqs=cEPHFqY-4US&*m!|Iw+RPq(Jo}J5>W|7y}Pqw;B5sbW6y)0<2voK)2>u=(iU5{=XIg&^ucVkAlnZ~FURG;c zS*qP?CfsuDdfiapAH4{9s2!&G&+(_Q4T8k_kI?sCpiE@^26()noKSvyHXNy_2c%osJi{r$_u zhok-#=NqG|>?Ypeq1CvuRY5VL;GfD3m-{5C!C372@;_1s_x;p|ei2PH5Med!uTr}N z21i&J*=KSPDiQESaM5V<#(rKzqV#h70PMwom*P59VEw&u?;L9J0#E5{qN0Ap0F;_U zE8m+pg=cOtmWl@$jP^WN;Kk={~DrgF?XVT}&XZ}|=i zg`3(mediq|)lz?p@LtJ zj-)B|%YPB34w^`~Wp!+4Do35P2#@9K#?fpgi5EgGUniUdvc%-)MKRbjjLAC4QM4Ho z#JzPa^qM5q0Vd=aNy+4fXZjEjw@Z&s{Pn_3)+9_EjoeHI+=5ERIuMdFMg1te#SkN# z;XW_*zTsAasg}-%_zM(xY~33GgZA48;xmH)tdA)^R_EJf1N%BAZbqE=D_1g}1aFA(SX>C8UDO@qS*h>!LEE zcvGR=lkXKwSVH@Q6=g>qDs~CE2uaS%EQS?F|S z$X>-2>u)O<`ln-M3yI{40mxAQOhf|D=+JQmE{E=N(XAOQ|BiZqGKy6<=88V0?uPV_ z#mk7SS?=be4qX=Iw8a9y-oh(MAi({IsvF5qX^k~Nu0Z^nUerXlXKs|^`YaM}iDk|r zD;cf!w3ZL0I>rr=w%(Y#%R2oYWjd{^#(Y(Z!o^03;7y)aeXRXou`{~S=t4%-Vi5BWVKii9b$x-?cTkY3_46_KT%<=3xJ1+)mf zE&>{7mb!5@+awhOQ<*uoJ*FI@q#BlRm*I(f(u9gW83gUGrDQmmSOUYk60{{V&gh*C z!cv;xNAqkeV0svvml+#38zn`vBvQGJxBHM!9?zMo<2iq&#o!fgQ$O<}*QgQMIG&68 zFo~0>37B%9wDgpYvC0fx`U?`Pc2qX|z$fo8IH#zM;7b8riTN$QB#=(I6B^4vt>*)9 zDR)=^#`P>+hH2PVd`Y3a5H&BoSr-a@v&PofdUcT)t|5r;T{wp7MvI{>y;)Xz|7R~n z!>oj#V85;^lHujdS}`!>|EO+wn>cb?5gxma)L8FHQ`$sJm>g3%eOUg(KRS(z|3J0B zMx82ILxNDT6J<#ifS~iTUAX?Kid84Ub6`EP^J({i!6^1C9}w|-%H@M4`$s%Csd10N zX7sh;d}vTeb=b)6+Sq8Z!BHRBTr|{Lx^I@3_W}}<#w>#hh#M1047LvO#i_?}CHoIA z8YzAOK1LQ26MTsfS`nEG=%7=_^m(FFXW%Gfn1&CD6N_SH#1MWoRM2OACLPR=w^$!+ zo~raqa2Rhd>UWm>GATXbYxA4*vTyq2x$X+{x_+=e$bW_4DgmKLVI?Z5UnNvckDG)fyb>qK(mz&Y{J zmQ?uu4s#Z=s%SAo#Dn=ABa6V9f~cW!>|$rfRY1$@vjx74H~W**7|Wlb4=3V{lnH2W z=Te;|N^A)~;`vyQ#6_(>ri@5?vWfE`{51#8<-zxARwJccN&%F2vt-I*}cU|>XC729b2PP-H|x$eW8i+NZL zy0SbOAUuKkQ@wte`c!#clA|L=Cjf;fhFO1E4Y3mTwIFEgZo}nZY5QdN2ZkEEErRP|8_Re3lI_W{}s2?1LDY-5f7)udqN4`}kigom=^pkUg{WOTh zvBty96^XAlB!^cR4zUNhQJZWTxQ&tM;|xw?o77h_ zlm^)HXdvSYX^w^L6_+hzt2dl+mAa7dnM*QP6vt`xy5VBX$S(?~PUgiL#HS7mv6Fme z@So{a#<&xs^yYiIDNdLT{9$kNbQtG}_qQd4RKzanouOw`OZ%VPA4DxVv(@;1wZZ$q{Vp>J<=b6x8yoqS3 zzySEq-_F^H>`LfdlRHx@RC69M8yfL#Ne;Kp+6<*t4>#Nl$yo?L0*2oluCj|@(en9vMS@sH3Ifw6(FnDXBgFk`c*=B^KZ zl9R88_OwGa^<%BmKg)e7V^pG8Ga=Pnk=ta^Zbe$M!axV;Bu&BhyeJ5&v5p=`Bo2v??){^+{@e{$Nv0Yb$4T)F&uo46hrw zq8UA5Epyuc*m9CrahH6L_~A`LYz0$REAN~J{=F%)7vL`k zPDk;v6Sf7(U4cmyG%M+oCv{@rW=%2#c+8O^4y(+~PwhYaNXN&8NLxWo#mC>!8Eu@2 z0`Ta6FH2Ca8#=W9^P`eN;qv!ajBcDIZxJ;BE}s}O_-fX|A1K}kRT@&I+tZKD%qNH7 z!`Wg~eenB@^pTGOmQF%l)a7JsPd)22Nd@KQhZUGyZ6h7 zX_V`=-@=_yoO7Zn!Ewk~po{M*;NV4!xHwmi5@cHuuqjl85||UpL!@6xF11h5&X}&b z+ru-=zqjBgXyes(K{v)jpVnwu&0-+;nJulai5=yPs377kPx6O;ne;&M3S`e~DF#;+ z5`r}HgSWB!Dl)-1noHQbDMmC)kIEZf&z}}p7~<-PI^J}=xhsMIO7ti_)k)&9E?sCf zXQT)D(#GB*UHtx~vtW#$jin%($xwbJEnv_i+r%RwW4lHT@avrQ|B@!To}Si%g91C3 zv?7eERDJh&sh*VBJFUgOX8?*akF!TRKIY)Vvsh0HMmSZJ72cK8rfz8goeV z4ZmiAt%U;%DIS-kbq_&R+p8OzgX^C`E+N4}qb_>+PDKSjpMm$`!ph)=1HQtzW({n* zj0%`S=x}CDxz{|8TaPI-0iR>E^P&Bt;EQ0Q{`t`x(4y<`?SJS@|Dm<9coNt6DJ{JX z;S@daBs%O4i!})iX9hgjHpaTXi`iUQ zYA|LNiSD1ue31&Dd659EN88-4(oaYOoTZwJis^=T|E&dhKpiAjRuCRx#ww?x&TJg^ zt`%-P6Ox{11L;dDHT6NSA0wTlZvEND)(z?`2*24$Qnxe~NNCvSB_OQ53qTas)=RxV z+=+E+iUyRaMid%u=r=cq`w+15zGfbZpzKBY_=O7*k*Zqupgy9*D)w&WOkJ+6<}mV! zt>10=CLxKpPIgLe8Tyqt|0MROS2)=5CrMvp=|)a(hw`Wn-WP;rw{?cnU8~HARt(y7 z#386w+u~4|->?45G~^e|{u4KLyDqf{AMWuQ=58L7!3cZO?bd7J+g9rb6gRUEd~MiU z5n)6VbDix}M9*;!T<~{~cz%{VT@8ENulgY>Z2gc55+#L?4jeL`4>t+3pc1t2DfNS! zQ{L*xBjGfJ4f99t;Z>?~i3q7<;;ez6T%h5VJfFG#R$f~T%XV6pmI+wt(lN0uG zAgF#%1q}Q%vKrU9DlUdhCGJoBJ;klRB}ss{kssqw7mmYeUJ&(uq^oh~0{cXQRTvgr zR!NYpmeIHbkN`V)Bc<_|b+f2Uj(CQY5gh7M@m3MaPU=Si(_HC<-m|$O#_vk$DwQnT z&QCDt&GGgpl<*bymxjlCUvn^$U?kBxpQ=ejXydUv$BN3V=_2W~H=8a_l-ls*b}Q22 zXX_yTUWVSr>RDAzykVTD9zkix^_n9UsvZkkys=B|lvzTKx()W$RPmaT)>cNzh$m1P zAD%Ucx3uylESd6L4AiXb3`gh^YUURCp^qFLqRkLJQ31z)mY$Fw2Kn}Jk)^+o7)B6@ z@-HQHX&Cf#=f}fqw+!M-s&ZtQS*@*vU!MpU&xbx9kf$3WbFfU0wY4CRNv+Q;(ii}a zI}bGL$!0aUwxONOfVM%8kO#5FcEOT5>Ysx_Dh+)ZVl6NDYbWmTVp`1)3aD6PCe)g6 z6B$f+a@TyX)zR@ngQvfETTkz+vGhm>hPZ3*4Blc9#g>ZX5WsJK)|4?=R~pe?eUS4= zAqS2%SfcrWmjp@99sbr5{>ohLs})0TRbRtfUUvNylJm)dwyzV7&)qm8y-&f5y3?{H zLa|i0Ykoi$!7VHaN8iU@jzhV9_59HdX6&p}Mp*p7_ZKPobU zbh0HbR_Rp?md5nRiP&n(SazyvzkpU*bE!N<;`Jqs_>$IDh-n7^8Hibar1vwMX8Lv>jA)IWn)r%OBt%C%dSYvQI2%!^&u?YFY9tGE=^D zQJA?hl0Lol19Kwr?<|qO6F`YzHK|2c`S5)-n{YCn)P~dxVuC{|FASBNuMRb(e8u>( zr5t-fUool=B{EDKdsu6Vn4>aZOXzYU1pyoixeP03hBdOkNn4i@5XIldwv>6)twgG% zZ3_^!j`H6{hNIpSYHrB1lorjYGGx3V;Q5pDo0XweL1F(Ge`?t>eIYcKhhhI6l7BH= z-qu>@^6*J(8_vhdv|OkIXduc+mTQKUzE|SOD1N_wyFz@-6DHfwU#=r?JovHh0tx^| zM;6L))yMrV{2(he`HqeN{&Igja$QYsyj-BB2?JN63z5&^0!ki81j*l0cEGY(ON#x) zDKT*zJXqMugk^|1vRh(5-dkU8tP7#YTkFIZ4>rpT&F4pRlSVmomh#+Mn)lB}&>hsSj?|5e6NnOtU? z%a_TZibyXL6d~d>9)fV%ah85SjbG<7m{XZn*0T}(eXNlUBukp|v6@O~Df6tOk}j9X zy&o&lNhAklU6cQW2<&*Pz<^CK;m*GbPn5=EDB>9n%iWG04cvfKRA|hj6fJlHz*|f? zBpL-GgT`EY;Z$-C%?$>J+Df$=M}cD@*1U1|$0KyM%&h%VFdQF4wcHIF0@yK%wp zJ$oW(!-JC`vwm#wpCMGaVJ)ii!hKn`t?)>#q5UH$km<-M3ozZHhByu_8b(*sj-)`a z%UTZ+9UavxA`5-@O@d_++K~e14Ob$fPW2~3!lp! z*MbI7Dk9}xSdp}1l%l0S7-3URL-~g(MNoR)b9eW>%_{xf=$F!Ndz~wM$5C&1uh&*Q zR-)fmns}q--3Z&9w53JFgh~oNZmZexXeY-d>70x17-i{ajl4g|1q$Xi4uqeEi_eXdSlFEKFMO-g_ax`xsVHF!cGHZs6EG zIMg%+7~xzGC9#Rp3Pi~?G|!&g|AEj-6wiVHR<46keDr%CMrIa5G(XkCa)n6sUYqIWps~8>1$ZXg#wYK9OC8KXHm&Ow0QpN2FZdT=VSpA2sv3Tau~a zBLDnpL+y^Rf57%>>f<*|Nv?6b4N;T8=OjkyLc>R%A6^Jew2Sjte{(&57;^G*GXF4) zFlY>k?9h~G${9}b<{3yXM&t8vAtAlTjqajRFap&pgV)`eefS+>@}T@#}p`hX&RnaX4@` zq&YeF9W3_K%vAsK8-0ssXXVBZaHfuLhpz-7x3L>xCv1rFO6|&O@2mB@$k>c-bF|sIQk`oG(2v)Qet2=N=3O8%E#RTWEWttFAHo-#B z;1&Dk+k5*zn{RNoQf4GpU`Ga!Noj+t0c@_2kAO4N4|v221d4=;%M8q)w|?j7%W_}6 z0wx8%fnk`=sEWB$qD z9Ug<*JC~YZKL+98)rciDQj6)n?v5*8rL5zyMXuaDLpq`-MgKpIK#MgPboO@gIj0w1 zZvgV56}H5EY^1)v7mFQI2Zlb(z$mU^zIX>`Abh-~_s(uUqQN>I4$DrkM(G4*{DSgGV>-~lFTaJ%!tg~lXw>YPXF?h)`* zcZm5x+>_@c_+&V1sW~@TN;{tXEpv5- z;SBIHJMxpg@{e$2PBYWkq?hc3_%~gA=W9ohpVh{*2Cd~L zgON{ovtHlgG=4?8mLl(V-EA1LlEfB|(tgs_ST$jODd$0Z{#F2KnyP0qc{%4@ZzI^A7pXCr%Gq3gW7v7NdSYtBt7}W2u_|y8=edYRx)^mF) z7FSc+nj=?eDBM-#e19QC5P0H}fZTH7e9m$q`GjPSVd6_21pH(7ynRw?I>%;|vD%XU zguR~I-k;-ScH3!F?fu}ITT74X5T0lfvM7k8DySpSMrMnSSkdV#+ zQJO)zLAtwp$e}|zht7dv=on^R?uUEVdjFr!TIb_g=bUFhd+*=gWXY1k#48XJV2m@T z?EXdUM(}sQhcY?E-eNTuG_m7VRZJ~o2CXEXPMEp-Q~%M7f#gqJp{VzI_?M&yZLuv` zAi*2c;KE4``F$N+_=?gg!B*^?RMD02=kzIRrkFpqU*8>qT^|Eda`%^(Qc?XOE$2OS zn@3@Rk!ct=*5k)+weWF;gS~sn2Va3Zb!A^B>oUT=bac5aJfjBQP^O6-iz1#%n1*|| z1n%+riR_KE81HdfnCnVph623>aUr-8K@JAgk)wXCh$%jWi$cN{#;6Std9WIS4(m5> zb*`5@UvX22RQqQyN{<2H<`uqvr1M?v�uJgGV73ynd4wE_cp=pTDZ!S>4kkS*~Zb z2IGXyq+)%=8SKUux9?akH00U{NJknBrUk17_(&P#yI%Lxh+4us{)IBL9I-(6V zNGoo`^P_%&OHy6~V((v5X1?FTB>6b`dRLY2-uBBx?7)SbL{7i@1(zaZ`0JvFJ0|HR zr))>OjLxAcRH6Z|GX%HhaXd!0l!%E&d|de$GY;fqT$LVCqp%qpdAdo~jUX8kk>Y#a zaeHr)H(2|pl&)C!E@g=c0oi|0`;s!n^?yOm;(tMoME@)v2|D=c+$B0#;a&U4a#5Y` zGlz7Ih734h-x$6U{FsWI;N|cqJn$tp^nG?9@r(np|CY4NjXcU+Jm}bdvQJc~@L&1> z8bP56U_*1T??D>WVGwIGT^{DLa*yUtgCLJW-Vu>@dy%^l1P1Mfh1T8{dbv)Y+~ls~ z6=eGa@g!pgX~)|7$M2T4K99vHaKNQea+qXfXFh*n)jEAwdU z`h5dlFG)+`wFiYb)4g7J;;7NZJx}`LtX|GTd7p(myOmUm&FNpeS!I^dH_!R>a%Zpq z%DMBQwgq`Mop7-7_^aibrJBlzNO|`P(0YT0rdw~$_Bk;>9%GJE<_6c>``WYva9i@s z+{dzqjDP(u(EXM7X5qrwwrcgOAB!K1)kk~I*lRlYkarVS@H9EFhV@OtWOf@j43K!e z;YA}I+>W|`_J%#SB|dDg5eM6Od~75;I`%?eh4Do%JME9f9Ovf zhs=q!=qY1n)({> zUuT+zGXuZ(Kx0&=cEDQu*&)aqeXEPlbrFcPW{0=Kgk z1N=qzzQ1!(8*Vp_F+mSYT*wzNr~Sx}wZuuUqL?-pEzcEvKZ1&YaCW~kkw)Os(isPV zrOiVhLyl{X`wG2%Td*+bxIy$A1RqlBioFI_~ zSJ`At3AbRkirXSQFZ9Son$RC%l_1GZVQVbkn!6D{4fNYPueb#^F&9NU$o(`b`j?ASKD<6}wEdFOW9(H?OPzG_vr@~Vtgoz@ZV*U3QyXyh zRGht1U2EgQ>q%o4%^d4tbu-Ncr7#W)j`iGLmWBj}JID=_DFj$5ILIwePp3Z6I>GL- zX&*KM@hq|LLw%?GtObSWEP}r?x67aGW$%;ax)1+W79;5=pepTPqsyCPl&>&6C=M zjT~%^Fd9vlvb0lBU$8`A6iII@DNe{1&Xy)AaICzA-dx7fO1L#p`2okk6FokQ9zTel zFlfJxCg<;p<4*!iKDvKwvs{YtU6Vk5^?yEQr9RT`!BCQ(8}BZyUwCGwh6TFk-RxLj zW~W947)E!(DtayKAF<*+pMq<|k1nK`gfv0-7#Fn_TVQ}s6}$Z(~v7B zr-s-?2u;GHf%L-DNv)u|W8GgR+&NK&&XAbzef725LlR@Fez@7>N4*K)nVKry3e(?qndkD^=cJS6kp>VGXE)$sBG$lXJ%l)M$u zH#!AyM=Lt%Cd$Ct&qdvSlc=%w2f4;$>)z2@zy7SIMEqn{hv9~1SieG)<#~T3IffWV z<13|AOoxcKpFZ2&oN*!|G8iZrX*Imv3Ro0uq12>Z3jS?EDg&=q{#-US8}+BjLtqaj zjcD3`EpfdmaG6WI5|(03MQVe)XH8{ibqrrxM9zzudOWBrFNwyr6tSFqDij)O3jK0w z;qnk4Rl7SCaviTlN$Z;_wipLLI&_lNE?UilR_82wHh&wpnUYqDnn49+Dv zpn>*>h&#)nHuV<0eh7Nva}GLje1J?z)OM%U-^7NKX7o#a(qdVf)Mlhi|AG3sl585S z9!4h*Ib(hVP$gTfoPFT%^FdQF&>$_8{|V=g{)))xNano*z`{}0K6c-0kY5C)$=BX# z#A8?M<8D!Yo@5U<{HivBq9T>Jz;dJu{4~UQV@2CU$PLnPw_2IvfYN5i2GCXXnDGN* z8;T=u6UGE$d6Bl@79?T#qeXPBE1vdRV?Spy*on4$MI+VfuxUZNl?2 zp76^144FH(fWE9q%*f z<>}_TPlpkofw!%-63w9t(N5XFu5nIC{*M&_v41OrPl*2?;syB+D=j_$7wB#YEi+a) zsBYfLctTFF?9UoOX37aD){X~^P^_B2qaU5cw;%N zOa3~2Lg8-k-4EK)uP4>xgD+Icwu7Gn-Trw64^;dAnscTeh<|k6>N(Ne%Kq+Gds!LF zyAKL8M9dS7`R;kBouQX%@u3F)k)cahz>5t@v{E8v4q&i0!mwDhkUq2x-`fM8Y`;xG z>L*r0M}`S)ae=T4vw4xNz$xuYb4nn8QRHm$22*Bq@)ghH{zDji7GmDp^*FhGfc>(y zu7|kq791S%sq*h2**;0IjX8=5Cilnpc}d`Ye!lm4Yj@P~UBL4yO*{;&bwu%>vYlY9 zK@K?&LzB_LLIV^v9jz$S@`zLiVmV`Rx|!-B+nbcm-(9!iWbsuEtknK_UxX*|MY>~5 zLtxn>`d=w*YHt|BA9%+F{85DOU*Po#NE-4X{XFCO-0a^_zMfcV)b6{phdYWuu*nnU zlg*qn;06GwH<(>Up~zqO>Q^hpH)0Q`zq=juEhS~#GvzouBzh|--R9Xxpk-iG|I@g| zwxsa#SwEvHSEVpRZUfGqN3Z^rs~OsZlz9kPnjF;dIqH=wNdXS4!7KauvFYp3YAR?dYvYio~*0QV<+bnOMfeL#NDw6+QE`$mfhQ6n=h}R;heIBCwnP zrBiQkFV@a_AwmPq^rowgX7uKAqqe)U|5T2>#?R&5JwYYtAG6<-C57@~W%wvfI0fey zmI9Z%hSJ3dT1N`gUu(?sR(%emwcaoZcj0tHd)!tJnpX1=`3czQY_rUID zj={Hk%HEbsYm0aHikIDR@+N9%Pw0(!?a3|QlSOafs92xAa(QYTM_WZ>G4w~;*LHmD z_UMm=5r$V20+%*Yp44VRT?`JjNV?-1mm5K%}4!wCvP5!tZuBhecr|9&ktZFAvmGHdHO3$t#>eTu& z%P`nX?zm=eIO&fdS*VumhjZD|jbC%2smr17jre`An0#Yzr=459+(0 z8bxBZ@7Zkh;r|pwjZLdBJ`URH28v*Qt*0THQFu=W{ujKo=;0#T)HNS-;#}D?-{a&6 zGKJEZhjke&SnX3C*KE_V*KUKD90Ll;vP}FGp_;7=(<|MP{eCAYxtdG!D>ld;tZyUiql*4v*j21fg~L3?L{7XZw-k- zMcc?Rl>BNoV|Kq>77(GTqyM2qbJ|_@J1iKgc#X<1usbQ@Oka-+k1n3AGc1NMF0$@H z*v6Lm=%QYT)t)@k;^A?Fs&YT4Hs5FoeCGea`GI%kk5E*&x2tIYif|fGKot;TwkX_y z+3D;&P`6KN{g{EPpavBeC42#Xi;ON2@Qj6tSix^!LVA*mwH3q+yg_@;Wi~{Bb^}xgz;qGAF{q6DR)?)ku$= z-G6HNYx)v0b_*M7v6{F#82IMci+)LK9F<&dOvL_bsucWv6abWR&%N26I|%3`+@)(` z7{A~171vgwiXV7m9GK91Pe@DjHvlpnvv{x1Eq3E&HRq5;{w%?MFAv~NGT%{4x%WG| zhl>TSId8_1B6Gz=Ve%jSGIY-dHtk_}gPZB{304@O_JL$m=MLn{273Tg-6d~ zU+M3Znpoa*>h*I!E=o53EM{Qqg#4V7o{)!RohyJNNCZtK(Zks(=KK z%h<+$zHLd-e{e%g(nI^tdy8kuR?)0-9(Y14sE<~9Gl=Ih)64dDym9mQm3$CO`-ue& z1Mwj%H*;^?bcqv2W98UEWh6Gy8LWuH-%aK1v$WPO9U<8l7>QkGihYMTB+>!YdG>Y5 zHpNtD5IP+{jkwAtOnTcLt|GWLc3(o{rN1wp*ZcF?~B{#G|%hc#hwLqgsd0o3_|MddA?@ahn6@v#k+L>+C zNzCD}@q*|#1`_N*6hRG;Vu6#8bv5cZgoxY4NVo%@v~zLf4gcD*c9z!>e<|CzmkiC6Ur?&-B*ko^JPonP(U?n%@{RFYeZ@)l_=mp&6Dmlf zuD^(F^DO>mWOeHPJtTbJC2w5rEignHOP%b|CXlsJ2i?mEz}yb2n#Ey@Pz1yfiiRAp0O4lI>6+6NANAs| zm%s*h#s;=ke0h%$KGftQ3Kxf}4Ov|y)k3uW`MaB5*e}WV9CEFNI=zzm?Crlw@DKDL zrC*uC1&YqT{XfJi}L-PN5n#Ar?*0I9x;D?N1c=EKZZFPowZ%< zx3{-rK3ALVJRI+QcnSB& z8_j(8b}m+;2qXsIxj%I4-Wt4pqf=jhbSo?b(*_mCuUti0ROPJaYGE3V!q*-{L?pGz zaag$LnRAf5``i{fe_Luwa~`gY=UBP5m!3>T67QrWy|jYoe;^*{GMStQzUj>_pu1)x zruiekOEUaDyTm(3oopqrfYRbeaak2>gZh3ThP~Dq$wzgbwR%f*-QU|?bq(o`l9f1o zGXT{}zSYieP0ixn<^S&KTk7;h`6on1L7iXUUVt@%IWSrK;Bqjjq0F0?{cMI|LpJ`7 zVGNQ@XmW_!5aD*_?s<)8bGB4UT~Y|Ue6?$){t@)a$7avkE5iP2UwaOnnl7heTV`Rs z8{X16E&?pQ!919kxJ#wM#F0>s{j77oXMA4-oEDy!8yFVy|4mq)*Wpc8?XxTETSaGU z*_OFE>`D7LH)A*eP;aN}WDk~_96~k&R3Yy8A@wxWKIO0dT)9*wWb15rY0Yk)W6WG# zsu}P|%#0oHR$E1gbN!2R`XJ4gbZ@grxgNMb_mPDp3$prGo7-U%ENTYI*GGpyXT4)mAE*`idD&5 zM9C3%2Mzve>!!(rGhalyqjF0rHPi_{U1(5Mk(5tiT9bc}!T~fzH&LM@sPk{;c>~)H zSsQ|g+nIl^6Tg#;8^4}TIm{`u`cjv+b|PLIhCTvVCc>+MVp)lCo#Z;VWlosH_&*F3 z*WM*t{UEWz+y9fCFfmy+KP9O*IOWIycFx>rjd=>^7OL(O4SgNRayZ} z!(AgQrDZer+V+8kwTL;Y)csq75;$O|uy__hmANH&Hz|UODVsWeHCQ67)S2udYYu?l z!73973_iqk_tB(5Zrd+rZ&wj&9V zHNxCqSv=D0VtmK*Ho7%?Up`t4LDKe%tOImP)oKAk!#AnSDzGnh9Yc zkayxWcxF69BNkPEw-rk%Q9V!3u6@(q?T(qZ`_^l>yd|XwZ8+y&CZoD3>fk7XCTF_E5BY@Y)C}b*I|KJaU);%|qOr+|n7#c?$Hz|4mbxuljp+ZXRmq zaM(RJwjAP+v{1e`p=Z9JN__Lr+lM(sC&@iWG?eR`-f_;TS9338$qQ;i%hH#1l|JER@}$?gs72NKvP@i1L7xwwp}n?PJsJ_-Ekk&$cV)d)I_Z zP`bb*9i7I^7Dj!lFkbH;wf6>=XM@9$i>0U=O2ul=i-~dB`BQI_lo9E=z)5Koq7B;Y zb_|%7SU~wUq8^3^?f1@adMjQ)vspBpUuGCMs`%T6&3GUC;xaqy!{^-vnVHK)MO~w( z;gq5BCw6tgakZqOgC-)pOR7c|1FHcBvf7P`l-iPLcvctz#Vd!8GcT)8SN-;px!+2X zk+ESc$Bv);p0@%L^U`K9yIPc5bePZNXRmgz6>hh&a!UN@;>tD3=3fqF+Oh)Ri?hdo z3ep71;#rZx|KN>Ma^?R)5Yz>)i+#><%tZ|2i&G#a5aFznDVi_S9-6|0C_0JgyH$8W^UPnlK1^I22KkZK>BNKhKu~ip~7gVlo29~wE ztfEbY5p#%&s;IiCghY$fojVM{qtHhuP_6f?4Ee}u3N26&qGG3z<{zBX(dLjq({lbp zO9(blf1Y3DoC%|^*!y+`A}eE)B3zOEm@5)fNrN->#w zokidYz5v8@q17Ziq`~8g9#k&Vo#df@lBTv>I^~o}`w^y#fA2O31qG!1O~}Dtc5{ zEE`k!&>_XD?ByLr#WAv$t%G1}9&UW-5LpW&ozDU**XtZTe~Oj-%6PRZ z?2$;s_N0)VnFxK8;>fkRX2GWuuzvk#%&g=+Na2O)pj^B5Gr-klI3XcbQ5wSkJm?w*fKsEE2%Y)92<#UYt z(!7)x<|3CWKL{G>@w%x0y*@;A*x_UF@8U(zO`Wxi^+)MlL}8i#s;C*1j9_dqEmh-- zpCn`{U3kCEtER68CKfl~7hnLnjzheu0a|5Z*q*5!*lJw`iH*@~WZ+3?)E{2qzyN;E zOASG7lqazeCz<+a-!71~6i_~-hifuL()%aj=P!tt4BLVd?GMbhSZlUzoyyy? zo;;54c(YGzsaA=z;RtTyB(Ot(-qco2>C?Jz@@4T!-KCpfLq1p zL4A7BI->^Lus5ZPKGEGgO`j0}pB)lG6lvvcjxUmP!gNmN?Q!q-DMO+9UfQ;mqIoV+ z)fhle#~?&X80l$1BvkpV*oK133bALe?cszK!jF$|Dh zXBSIMW2LmMOi=rwuaN$C{cl(r_C%h}sh4MxvjGiB&0mXAj#vZIcS48AmwQt`qLC7H z7vCLe;s!Id*YmRR^{U6w2roWUGv7!O82HP=A>8qkY)A{x>*$h<_r-=9EjFnPuIRCQ zQ`bwpip;URd_IYS;s|PAlwK(Fv8dF1{*ERiOpGx_Y-@yWnRa43dk9rTY&o zdDSf$+-DL5Hv2WoiJb8j67KFDPD8tQzH%#R*IxR(DDgX$Vf5C{n)NfCb7AbaHmCVb zi|xUk`mM9O+J{c^Emgynvw#=H4G1yZvtB4c>tbc!;ZrHis z{;|unVDa7R7*<|*Tr<@8#iA6-hUY^uo3m7Qjsneb(fZvzy=wgcqi1a__x>2iemTW} zEf(i-ftu?LhsBP8R49G(sex3t5NFwSg4lQ%<7v^^A^N;tN6&@0`Fr*aebMDTf^GM^ zxW}or^HIoy8hP@08ninLNY3Szi@r8vZBcxlZ$OcketNvd0ocQ?Dd)L6F)UeWz4^13N}p-`+$}qsE=B4C z%#k%fCkg7f&YJ>=Bx^ttWYK|0Uqz+*yI0j{QHvYg3=f#96c_hVHni16#d@`rdF_x&|^SNiC}^xRcN<9AQh*f)3m)Z+_R@1spIU zsdBU*Fs!(`UC!n(Cjh1CCmipGOvIa!WaAEx zuxzQkYXCQ$FK=2mEV>VIszmcoWGnI=zNuYG1+m6Ft$(mEo5j{uWXkRLB?rf{rHQ@e z?~Y~bL&;BOa$caa$J*I0VTs(Yqam;?kW_# zGwCHW%b&^Qi-S5_%a_hh8Am3^KRpy*q{^|w#c<(E7iZonwfz=DHsV0-0N2nyI`i$A zWhXuF+sP*!@QfNMarI{bonT1LygWyecR-}E4SjJ9&5<6mhh6$qR&Vn}RKt20qSe*e zT^b4 zDaQe!ON_#M7fI->MAzSO$f7kqVpnKLKM}b3=u;w3ARPw}I0%!} zJTBegE($bTQ;+(nF0}KcFE}r|Vu_p{0d#zHj2lw#>CpcGE)R{a?#72HOwr&+=j8-G zoPR*jOV^SL?MDC_&_wKkS0oQ7ygA&yhIy#bg!l--Z9!$AK+M=vsZi6NHS#B0P6LuB`Kr(PrTBwFysXe7#QJje%6Z-Dd5oZ1A+fT>wbB#uCOx?pN z+vT;l=liK%A6vgw!=J5@h^kx;mU3^bc-MVE_Nf?)F6}cCtWcMw&X;fhjjHF(?Uu9_=Oae!S95FG3 zW9u3faUil|NR~q)EX@=DL|M2r?XmUuyC=6_2cD^e-K{<`GkB#$td-7;BPKMKa7LZ6 zlFo}sl5%bM_>;`$dn!Ggcq={|u~W`_*yV)&w78D!n)ua`+qhAQzwa*P>!MP0`bL(b zRRwBs&gdl-4f6vKX)UAV8yZn>30}TFRqv*6_>wwUq8gljR5x2qkXXA1Qc?&GsZz>6 zPqPk*@)A+6#DU+Cx8m7Pg;(-qyno`l!;5dEKj_6f5utuzRhl3QYLUNE}IV;o$JQ;RfaCGb~fws}X&PctZQni{@s`xOjssj=)SVrmXsSVyQqWVbx$)Q~av)EbPJ}9sm$!wF@`sn9gkmcjbPbp;D zzW-Nx?-F~iEE)J&8STb%caB4ykZAXWS)Gs%sR0hbr9_?3Fn+_MHjYaV0}S3xvLchw z_vO<*oVQe9o!>jl?r>X#4D9>8+s?VPYyN0uB%!DP2T{nfcCWhj(o8bp?*n^@gPGRR!Jlz%ojYAq zX}^_7`TbJ3v*=6A&}fpTiQ1X!81M(&^xXHlJ#!w)P)Dq%*oPKoE`1Z1=R5B*(5~ z$SLx~?6b}D2>2v=*#WiwwNPagzpR(cy$<}h!92oJ6*RQY#V|wh>2KCMYFWDIyc;IQ z>F=k{G^%k8-hS1~OVmcn&gI+Es-VaGFi)2ow5CvW4lj&A$?jjNt1y{g6CS>nMpCcF z?`O5Pprh$aH>RH?tt^=}=a30~@8@`MHsF0)OP5*?1uT~3Q1X&%_F~ftrs1*WD~;&w z48dGhJ7G$6dN#s5dU78xu3HiHHCH#|QOrXYa2~E<=TmCD*881%P9YLv52(X>@Fa2H z=a|hT$($M$Tu!K;8&$pzeuo?U`9OJB%$%Nb6L*-Owd$7^-#=VOrBXVw#+&qNqP}$j zONQ3-wulsozlcVxB1+b;YbcU^!2VO)*H;QA5&2l%unaP^Z5t5@lk6QLC<=u zH7(r~W~JtdlW%%k(0{Y#EFU-Z2_*@;&5C?n;a8mqJuyjQ)SQ<3IER2qn$yf<>r~2KLGc z630kF2kL;}@hGq0?Ujh-&5^#tSJz;^imJ~Aez-mdyW_Kgo>_f4Ya0i@S%KLZcBn96 zBk?n-?yt0;cSlW(UYiF?eTrUBsD5vl)k7Vf{T8BJVcpkGCUUn{)yST=S^uf8(3|6%!rMk4JJ%U!V+$K@EGb;H}yH`F9S2eBDKcsFAqwYetkKSYN_V55-8 zu&0}ovv3=osUIvfY
(4k zi#B~F>M7-Djf{|uwX-O8nODIq@^IUO6MV?3yD`dLmTl=$mYFaUubb44o%eD{pNf~c*-J)O%b*XD?a zW+sN(8^F8^JI%fCWY9$*c?w}CxlnHL#5cPKjwBDuLfe0eUb3$HoexFYUtF9!vs-TW zdrP(-32jaxZVOub*zjM;UdnPiCps59mZV3X+C3M!y!{O)XAtKC!5{I|=6U$c!JFA3 zZZR12dYo!*tZ$fh6*y_pfS4khzEIX~cZ1C=c*5RK=$9Xe7xv#i0&~3OqwH;itq<7u z+Ma>XyZm|&e!j>yMZIJo9>3Jkn~~O0U#QeJuBI+PaIY+hcBC;2BVXB!<-Y&6Yudju zdtju|WRK|!cy}+Oj3)I&AC8)1-qUqU6TZ(z2+-+q*a=^2~~Sm_AlJ<#UnbTMMDdTS)F!cTs)ZXH3G) zA1`ejF+=?^)X7wJoxV^w?122*Spd$$b1=HUTaX*Ad!)vQ)M`<|A0_*v#kDRB)wjSN z)`~H5BU$06o~thqkBb#xEB`149G5b(QHW@pbx9@t2^BdIw?uq1x+LV7RsGO;Ndyhbbtrp;}$+@`cM2qbL;DtOw8x2d9U8G5lUTpG8@aJclL^q z90!P!TJWZ|E{(ALwW-UjI+$>W}^MM zz)m)#`l;T<)2vg8za>i|_$m=^n*JQP-;8{@t*NX0i?j33m6!8^`G7uNn}Joc!Vr+7 zXtO>mTBC{2+bP_!n|M4WY$YxhOadk!-%HDnp z4I3mJ*LN2Y1F3t7o_KlJ8{FFQL)H&gH@=Df-h9>XZ}KhDZMxok(m)6Qt10x+w`|zA z!QT6m=0@1LphDiUr2G&jM=tjc^VZk#_w!5zLk@aTx_;=D^;H^nIlr@lX;3^w8) zLz1JUsBmk%COQFS=|AnH3u%9uC9Q1N zucqHS_AflHB9+RF;};hc{ObDeg#$STZC$6(m@Iynbjh`h-B-3N1=oArxyKOGzUxm? zidy$I``hQ-;M2rO+#%0Fy{;d=ZZ6*PKv;FKjh5yuy^Evk^RpMPJ8r!4 zh9o`xHnSDGTW2KJ*JSZ~!`oyJ$EEk99&B8o9@UzwF^&f(&}kGuw3}zpfjJO-?vc^L zoEkO34rWTUg?B5auy3?nu0G#|Wet*>7vcEQ#`SxE4XG_A>O9&@C9ni@K!eMH9^0IM zp%B#{y|lzGhM+BOzTut54rZC)c%|#eOV?EO71fp>ZrZ`!=zk;MUyZ|!82Op1!AuFT z;bWWc0XVlwA8MX;Z}D<~;Qbt?R2hmFU>>ob-XBI^eYIu1?5cHrpxJ0~-r;oY^2eJ9 zXM6wB#xTePO?8*rG`NM0D;%p{Ie!>lT@CTwzs(L#h;-!`P#^f>c_#|kZ4(Rh_f^bl z_ZEa)Qzv(`zg|4&h@A))Cecv=d9d0Jp``r|%g}+`kW^(-xk8?;Wa}?@#oQCu@fb&j zcS!t`&P7CP>aWM+n2oa+qF#6D_lU6psMadafDEoKKL-kZkekKpF ztA)~67z3^4UzRI=e8BEg2YFj`o$J3TDlwDTux^_S-sbkDZokkA*1*yKIOwXKZHOc9 zX@Q==AI|^gJ^PJRIycnb`&;TDA>;$uOPl83FEP(~#j|V&W_RxTY$d$B9Lzi3(-d7RpL?thJ^^h}o1=6ZD`66?@Zh}RLt1qtJp z(^GExt<3ZA_$B6=oIoALRO~=8yio8PoVeX9%NXCOFup@E3stZ@=1H;me(KEKH`DJ` z^c7V}IEX~o`LOQ=yK8+zwitcHf&v|LB|$duJ>vPYIqts!FUL8vV|el4?&*`K`~C8S zAvu{K>N9pGgHI-*cDVVt_V7MAh;^#(0Qs_qBZHIXv+=mF9@3t~=3m)VD+l`}=)i}s z#a0{kYPU!51mdf_V8De7o8otU~RNDBAS;da71v!GKW_&53TF zy?9#iu!FqJ)yg(=Jfy7^PL<`DN!9%VlUlV0auj8K{hMUl93ivX^`Rn|heDRPTEDvR z*qHxu<|_J=+=DH40ChnC^qE{j%xp(zj)fxvn<+(IQ1|7T^Vu8g^L_gDkkRs zz>*R~O^}%@wuE~BllD%p25bLZc^}Glb_gV)B+^pU$hu!4|vJ zIY8d{yzL9=B&~1w<{{~eL9C@f$COAwiU2RiTcpCgMe41-Dy^c!yHIbTH3L=hg2*Bh zLA-w|FnN-``YSwjI-4&|!N>?>pr(H5adm+QcUmxdoj>NHo7?cv@YZ2`*q3(KUtA-H zyDW0LQ~Y{LH}Y5^?lZi4g%UueV)muEj*kPeGdcySZWOhJ?xte>5pYj2WTLIak-rrq z8mFa=;`=NMU8zd)IA%M)CBx3|1Tu!HN=1D%=9Y+s5*VPxm5_FU4+>05eeQn|xlnW1}CP?zkz3l}sAR-~3+d5M9&U z&H*M+MBBLRY+g$R&9$kaL9Ho>fF#z<_wf5Im1%i$fUg5>C^gNR!3`geiNbk zxc)ecI^LuJpTx?OYl0?Z`*S9B`Ia)6Hi>N2xa)jmJbG1i<_-gI^~i2xOx!RJ%~696 zEtcnj-WIdo+jiy$aHix91wqrv$340fa>oG*cO=LfcnIX62~L`t%Z z2>n~!W9x?|Hp)=eM9xkUKz=YrIlAsz|La|R)Z+M8(frnh1v~TOdfa2ou--35Lm4}N zB1y6v_#+lszx41)5kpmUG3kPRW9KnUL;JM(W{DEfbE|m1Ng^ zeCvuZ`#xJtQ#S5MtY+vo-xgD#_TBr=nJqN4$!M_+K-f33mJzA0eE`fP&^>3wql@*e^XK$gl6LoUs@41wRb*= zfjc7Y)E!iUyM&r6eS}=wG+*5KQ-IyzFjml!MZ|=TyysiZ(o>FclMlw5k_9AdeVw8f z_8-@+yC~Yj8Z!NiD7#}TG8rR%oVbSGjuwJ<6Bc7^@OR_C<3GXIxxlG|Nb9SJw&@>* z@AKbS7fkCK?e1*-QM81leAGh;x^R5cq6K6UN^7ub#Y_gt8KHi`R#p9z$dzgQo*ahX z{I$0e{_m7WF8p`K2YiD4Z^o0enjWmhy?2w3+PK`?x{NG#l){Xa>Z`t?vv~luLjf?3 zFb#|Cvtr?rRsp$z0ak(l?}+}JZ1ARe=qT;L?&PKg1H1QjXH=p^b}p2VF77m<N;QeB8M0Rj)9{YcGCQud_3ROERTx?8D)MfZDzmAtkhcb6=B`!RX&MzR@R0 zF(~b3*a)z3sJ0Ge*%(cF{?Pch=J)SiVle%{_9tLEmQ@fo6=;XRns8+GO$h{kYh##6 zG*%_S7;)l)t12EmXBKo$eT7ikxV7BxD%h{~e#sW<3Op~=+h}CgtS#){p68P4;vy_U z_#JjMgv3Q?q)U@4!0@-+_Fli?Rs}paa}tV8S=Fq5b6K>^awrl$mlHeQN?oom) zec^gqP)lKDK%Wm`Cqd8ql{B^oy>3hOR{bIWQdpPkTTD(YQBc(&>mk)irZ*m#-u?yC*mbd1&CscY=IJtkGG zHbaVb69Jjj4ms8VGLo^M~G9%>e1Gdmfh?`7t*`jpROIg?tfwUYH`##Ez5yyziRRZ|JGVIe^=(3G=cql^^ZV+2^TEiTzMQRfROJ3sZ5?|&eNI>X`$T1*u~P} zp6H8^m-@WM(D`VuWA)lpUn1S96Pvl(nNCorIcsDv4?vb9l4mGMn6$^_%T}#M;!Mk| zdtbpzukgx(Td?t&y#8`iV(M0}>XtIIsHObMQXgR&fBZtX6t9`}zp^>6&T!3zvz9*g zcn1|;WW|EEP9TyI2-8!%$h#yXp45Rpcn~~vdRO{*0Xj91^$PeU z6lAvhv3FTmZ>uL!ltOrQqbK(ao59gx9d!PBHNO3oRRpURY+b%%Z0vAloDFcBC%9Ml zH%ALhWA!K`rPYL{7$b-zk{eBvR2Y3$I}OY_&lZZb%@thl{hSTpu`|NsI}^_Vq(WQ> z27FY3mNX|xoXKBfacOVik5Ap#;x+vU`hdW~`u;#}(Egt-B$DzNxj%ZQ@slI8_A_Tz zu3ea8rn~Iy14*WL@9#Jw$+^W@o^b*fxwtfE>h^2xRou`m|9&pg=tInPG=X4cM7D~= zXP`SnQoQ~Zy&B{aJVFSwWHzL8BHp{^ArHY`b0@BSh=7>AJmRRCrNr3}J^n}CE!q4a zLpwN_L*KbSbB2A_AlG1g;mXuO*OgS zH$n~3ud}!5+1UaZHAo9){0d;c0@%ZS8LaM)m98d_^3Yl97hVVa_<~@I#Mt#VX3%f; zDk{rpKZf$Y);?rX!zg1d3}{x5WK-GBczzPkn!c0zq`{l1_P5>l2wYP4bAgZ8sEj!C zo~<&F9&s_9n5b@!Hvp zc(&xRRo96kx%@?$l;Qw);ck@%rH(kC1nc z=Z_l=a!vJIU*}C_A*w1dM~-5j1ns(EMQ6UMO~G!77S|Xn-@Jh`KX;-tX$k7A<|0*m*;&hw#G0nYv z($$&*&M(oUv%a2^n6JDoVx;sM(rb7fs)PsR3-dLD&k|zvQOG}|HUzH?EqchV_V%YY z{vu_xvH5mErXDZot)<5&`7b8m=&3{Ko}1XY%A%CR!fEL6cJK4paacL8`+<1vE4o1$ z!OC<{Z1t0{S#X@)G(a{#ilbCs^~Ho!7+FAvy6e4t_|EBsBI3a&`Fv|KVk`=BtAEM2 zViKXK5$ljtVlO|ay$Fy%_4>fv6V=Tpt*Gb?yW(hL0myrOX1m45^^aSTL;X|G&?&x6 z(Ma6;j`D4$tDIezYJxGLlrGOF&Vzvbz6nE;E^U?PBg)e`JcQSXs8WSuw6V#t?O$QI z^SnC1-&(}o!Z^&hn$7;N92UM^49};TzG4yp1wYxW4r;?H!Gj2H6-067nb9RLKm!5v z(o9iPznPQ%&Ko!z)hp8JPWa+z?bDTaHnqm_6yOu^xp7s;d_l+J+P8tRPjMo9KXK0O?l?|$Ak~pnsh25xJHthGN40&7fgkE1Wj4lYPgCtK9I5wtD zBy9253^DbZK|yr++zB40qJyc)*v+Z>Y;C+ zMS{ih?bfk%;0w*oE7slnC?KVNI!Jvd!Er?PDL8K=mfh5M-9E&KYe()4Wah#KoQA+OsM#IZBq6(q@0e3_K4m+KE!|;$8zffP_bbg z1|Q{lWgmKF^=e|u(7scLe11Q;k6tR6D*HTw^x3x;MtD4PQIYia-*r#pzOGNNj!}+6 z85?Nd;Prquew9j&cuDzgod}UTH^vbnDcxuL#oqd<`2r z5Fg{nxNsw5>Dw^#cs^iX3xB1Ad6Nti`uB0B-Jw9u06|2O*xg7TgE6>1i^h_z_b0xs zUzo-uNM4gU3N+%uK=pX$6rj5`%FX0yi9=VqQ>~#EG_D5<-4zd&4Ts#@vW*yo# zo|<3bKk9Bw*@>a!fpsRHMpKk>hi~XdgPK2dcawBme3W}{R(eVZ<#Y@ZXY3Er#eE6Z zY)|}dUsF(V|9K8=@(cdZ{zt9t4l(DQgWVJ*RwV8pr{A_B^_e~2SC{mPi6^uvtqF*! z%g@N4k$|0-jA)3EQ^lK0lpeLuyh-zFD*(GBWcJQV+B2DMdB}-VecMLRdcChuC0I&e zSPeF9r{lYbl<26N2T^F(m0Oo{`hw`{@5TvjkEHhR0ae+X-10G^r6qe0(Sjv6YxJVyX;yZtb@DZay=6EK!j22~khBYN<8n z<~RRQe_u)pWS9r=HIBw+_!P&25LZ*T5zz~Xsecet4X?fjawy;wYSJpq^k|0Qsw*;=X6Ucv_Zu+mOw9Q(A7 zyZTvspRGIhUk9;3w;M_GZK9%k84kjr`~u8+SLh#gKk+k3Du@O4o;mhj=`}J3BRLDs zmb6>|ew#Jld;p(EahYe-K8E?1b@{c`v-ov;tL;gS(JV|u>YsOVe5UE;F^)k7FUC)m zUj%uXH8yg!V2xe-xqzWh@ZYe7vhNpPYI5}4tgam&>29VL=SAm$WS;(wANjlyk-cYvq$XQ7!bJ#7OIYWn~?1*PeWp$}**n*!5czpI@>h%B3;kJZ_xB@+L_ z4JhzWb?kDKsGp|fgLEOT9VH{(9Uno;gRkzkIYo@e zW@Q+DNL!CRqv?vR&JWPM<;ht&+50Gq7si2f% zb9rDkPaDf@Ptd7kH z9CBk$nB9E8kGKyAx0!2p(5QULE15!>sONtpazrp!t8zILrnQxthEyOLp}VV-fDlD6 zIYb4Ks;LZ0W2@fYc*e1~$Cv)_Ja$LWB3tt(T$9inBdYWvmlyDUtPr}J|3UW6S-c?l z;mvcSP|r6FGdGlUmttsL@DPzou#P!T)MeyuK)_uOVnGDw*$p}wcQLeF_N|%;rfY!V zaaVDvd_w;q{ue(He`JCF8ZlyAYn3vRe90wGcECgh${{JaHszRR^RC-^Ih68@mZFA1 zQa_Hnrl%;gK4>oy1bL;JMZ#EP-#>?`3Z-?x*ejhds`_n{#^ZQyO6s8Dgv=+$Rp? z9uhbX>&s=dcuxI}1y>!}- z_;p8fhbcZsM6r+CmFNpx!p~d2+GE|ZptH!kR`fX9&!Bb z-9A!g1N{I_4^F06*fFaw8wpEgvANljII;*guK7)+mpI$Smv|H0RrQoo5e3ksNtala zynBSJ-H^bv`ddU)=AWp&heJ9H-aR}Q4zJ?G_MRR#_Jvjn?$a2K+;1#dj_CBRNu?qM zPQHvi){>no)h?#QId{AC_@4yLrWVu~_BN5V(7nS_tgo_L%Gko)US{dEbq<7#TlP&Q zLPBC|z_GC}LNhu|JQdQ!s%BV(sb@(gaRQ1-EHuI?6;_@q2-wvY0q-p^_S1fpf)D;V zkmUcecCAU7nqshe*d+-l38lW8l#I_Y24)VYl#+C$dQWXYPTrH|n zQYo_m7wgdHr4EwN0^w`+Da;(<2$6SHQPAF|z$N z!(Zf6MiXt&t>fhrq2(qmgzS#y`_I+J=iiANN?yO~tOU75Ft~=q=HrMU7HgI_O@-!F zf{X2hQwKvwi8R|$Q_af8NEJnDV|B|fk9c7VyjUCG80a!2y%yy%d|sH_+!5qVUyYZF zd#c1dFy~u9exLSR0i_B3FT>@%_ONdD`!vdCi?`v89mpica&jXBYKe8`!5^L~Pga&X zxdq|puN!wbW-cJ}r?9~H#T>dfjfzTNA~%Cr6J%+(0M9RQAL&$N$}^Mx4}h`fYclvp zIRFBqTCoPAEAqMg#h7J`LhtJt$H3Hw$Z7A7zkJFSwha=7{l!;zu!6jNNTtb3qTaT( zJ844LgU21^-`KH4+qt`6Z`iGGOzSM|l#|%fqc}VA9C>o^AOD*L zFvaPL&b@IpxO!5uvcTzR6aot%4arPcxg03jt{IqH$U6*32 zD>AmNOxl|!zq;9l(#%wmL%|XR`DiNjoPCy;$6~!_KoM7ZR2DT{<~vLRHlP1~Vfp;K z8B6mETn}Z9jQEph&iX-|R>$6JL4!{Lq+RPDmGtkb9Cw+efn8I{3jSAgM+RCc_Lq&) z!lNf`&Vk9t?*v!y>JQCR{nGdE6&c38xw8ikV|P)Ue&DP?U{#0&$+AkJ%I5=&7wENEl+9V+LRGuSgS=@uid$|n*r^Nt@(yZQRuX@x2N7U^COLN zIb|owBC(F%kYtL6$u6bbYLA8Cb4mm!`M{|`MNZsEuAnfSY|+R_4W!HvxIl;8mzJam zl@!H~A0&ypUht`V!RTTxwJnLIP&wJZQu8o@6pCg7_PSDUK}WIEE-x4|OHZYkvK$em zR)53vUNAfwfh|Ej3E)>#u89LJYo1DlaI|2mlHBj9)ECkv@P02jKW%DeqJL^unNWIR%JJKZTpNGn61zmVo-q1^iRP(K6f0UgPwnIIVZ`bg zvmEhE*$a3okDT3lTSJr-gMhRTdH?q(510{GD9${s(tms-IS^wgbN1qV`cpW;1s;W$Bz5Rtk>ELQcBNNwnQCbJ~F39Tv zA7ub%-Lc%SOg+$Ls@t#N1kh>#_x4VwzgSy@ZJ)PhvXe3W0!51k`^YL?H_`2YU7scG zlXNxQOG3xheYMp6t1_;3O10wC)dNdXuTqA!5;K~D;9liI^iyAlHu<{=#{1+>Bgm_m z73xmP5@J4@pyRQql-jUXNtk+`b*J64Ho?=}do<}%p9Q_I3NBU;OsxWh>BIAG(kpJMqskf=c?(XH!U2f3iY>31Zfk&=~9ZLo8s zwm{cJm5ZR*i*9y{8?o52N7kDHm+OQ}I%rvO5>n=#kp>81%}8*ICVq-c!NF z?>3)R5|96WfWcBPI`93@pH09rsWw7U<4c{D{#VhCNT_K}mdZq2W+jLUAMxI+Pm_i; z&3@ydOOP2wCF4*ROq_ozXjNO9RG?Q$st9ttY1{cVyy>a;COf_w zcF*PRd%d>L%g(~5%eEHJ)6V3gzHjB#3I=N)tKUlxGUjHd7TX0tBP__9<=sOSjk)@m z&m7Y67rtll_}Z|$_Sp9xb530lODs?Bo{0yoKf?vDiSB~>?F2va^6X9oKKqAUFjVvX zeu&^T@z;L?KuNWY7*qV^8oHR*jYbuu;U|{1Dsh+Huz{-IIJ4Q7Grc*U-aSF{HGX@4 zPPaHM&@+j3^!jXO7j(<}V8p>;gZ4%=b}AQLaJQSpXff-B%7P5tqb_eTsXZhO4z{Ku3dX?6&nVRCf# zTy7#F*Qy@mhM~)9)58@6aU75NjU9j`m51R9!-lI_%%R0K4||%wYQ3@Q?xTt>-ND&I z$ri}%Dm@5iOqr#nwydMIlu0@9&-!7WbQ?mUd*Q7)805x@^vtPUL?3D6|9xLQM1McGuA z!A!_vVuj&};OfT5YcDKC5WdPk{JA=b^&ZL)N33C6(=e~ep;2y3P8eaK8y8lq9O=-cGf^W~=EY5&s28U~_(EcNFb zhG!_^T~JOZ-h{XhW&Igx5@Dfu*y#pakRBfdfGHF-Pcocj2osOr^ucT^%#Nrlu+D09 zj^*eg+*b5bMu_pi3GqP8+>11u6Lv2%^?J_c?YPQDHJZHvxVpJx2c=@t>19d}O#8tq zt%JehDhcqLO0#6%a^u9{wa7vu`^MvD(r@B@?{e>JfXNml#mYo9UjhlpCzdT9Y6>mM z_7&kkjB2K_d(c$>J@i1U{#=r~?}`iNHr|FQY-a|h;<{G&rse)kntCI)kSNk@sZw*+ zEdZr0pp_`oCM?=|44vNaoNLhr$tQVW=V*1s$ENX(Zu(dv?h5SXSy%bs=Z4V^G%3=# zjn=UYudW){@dLNyd6CvpOn?~tM=V513EyZ4c5D^)&vM@O`)bsT<;jsgZIr0ntw1(B zdNb6ZIogafnD8P&UU!DaqaZIaidZz5>W}_|n_vVy4VT()xLoh)XyFjq$&qc8^#gcy zM#A5!;6;LW_IYdfu@Gb<$|AZK<_MD5<~!JBDyOpiBTNACo6$HG`Ww-CqC=akTJm^j zlTsD*vKZhf*!iwqxr-u1=;;7e=7#SGpPXg1(QMi0_2Gy~Ol!VeiH&lT%uj@~9Y074 z22-`ZpzT8+@ms=xlGl*+RNH^Bv}r<;swEn&c=N&OgNBzwWyg!`ge_$lVUUrRo8;Wl zZDE?XXz&zh>4`!xkR>%{V&~g%5gFr90z84iG=9i{WyGK-F~xz&T!AxEsY!@!lF3c- z1U{IaL^XJFY8)8ANm+39H3G&&3uBt3Jh8eyVp;!1j0c16Cb2;(`KsyiIPq19zk=U$ z(o+o$@fXU28U+tWX|YUQse-irKo;c4#O=|yK~wM$2C9U?L~f;7Uk{3NQeLA!rZW8j z!6!)&2TsPC0v2Wcus21u(n<&T9I@!uN+hZLDP63IM=s*Q2_TaJ2U8W$PX;>95x4X& zack7d|3c1tp8*9xllAUo$i!}x1Y2UK@aR~tiwGYQK<|Yl=xf_0AP$x7C+_ah1RV1k z4n=Dz%!xe%5pM&vE%945G!~`HQ;JtV+IWb>O>0_(PY)0{P!8jsctRT-x6#Urn*jPR zN5~$928sFde7k>k+I=DILBSB+M=CpfDX%ttCeU)a17x-*cXpH8aQB{jLcWA?YB!eA z4xXI6`(@OnDUnR}B^;ch(5h_%=a$nEfRDoyD=k8&LRFU9{a$s=KXLtAKLCf~TL-X7 zFH%R~yQX7&<$vmihqw@S61y(!08N&HI34Lh@dRM88I@$QmVNY1ybX_lSd{v3WjFl+?cHk7w(+vJIviZ zD?R|5n<0nQ`B~SG_>(Vt)EO{eaXArj!%-7{X2QR*R5pejceS{eyTz2s0v>M{`UN7G zbWVq~lavP&W~PCfxmyD@^*EO2mCp^AFKS<*zernpZp$o5tR~Gsr$oo;_K1L!v+UQ2 zknih=L1d&N_fRcz@n4Ce-*1qKSd2nSe_w zn@uE!THW5OPn3WwJLP;B7wlosbZ=9DFE1%#;*a8xz-U3EkUwiGe*S9*{h%^oqs0_z zCsB}>|NR>JC@X!wh=q+@uvh1SBnRJb%4p$JZUYJG%)#<S&F`=i9F9%trb%K@Gg!|E!#V@) zo3EvmXFVKqqwn)9fmiJDg9T43z8c1wP3dws!1j=Po(>qIQ)|z8cHT`$Jau7>)_e%D zigDpDMV(umyJcs5kO)oaRe<;hva6<#HRSixmVN zfL*6UvDZgZb$TE&kQ?$0yx;k2%Lb}tH;GVOqZ-I4)tsAaM$DHNf?Rhy<3DR$qlULHJWvTa-?pRLBZE>;rKP5R|7a^ zwTX=L3L(0Hl}v@%%Oo&MmlC*=khI(Bl(h2;``&yn!@u$ICZI-GjH3EZb$Df&YH)=W z*Wz^anZxZ?0gLMTW2m#-yS>(O4x0<|xHD1ixtL#iuvHn6t}C&W8Q4;~!==T6^x(^?+jY^Q8={7*XXA13n6Hk`vb6+cibDn7Qu60{sZ9$N?h(>^n z7!FM3Q$A!a1gyd!Y$4T|l-v9Ds~816Ef({_8+zIb)+&k+EgW7H3N^MsytSFhkqiRO z*`NTLwzJh(=#nmNHk;zb-i3YRcQJo=^C1>k{KgHvb726*d9F7B5#W;Ci_(VJzkHI2^Mvys|Z!$U_7bfcdShVCQH2; zHtoYU24zM|p@I2BEI)CNBoHyPDF1EoK1Vp}jz4jyJz(1nd&{`Th>E)be;*>G~h;*sZv@L3!{nN&|6Vw%`b z+1~GgRB-VS_I|zLuns(nD#GbmCekkRt1FE%(rk2O1MECFO&2#C=tpEC&P7GgmN@EC zu@PQr(n}Qw6TvwH*^U>K1a9%I2f*vKq3viy@b>#rK)-7P7d-Ip1IMJbKXNHbFxKkN>I!511j>^@Hd~)rm_FAzD(ws|%TqDP-;9!}+Sy z&KBbsUk4}UVV;(_DaFPAw}%n8M$+DX5FUtvX&1vJAR%yhIHCry#-om_4jxE)0Kklr zQdGO+QvI9Iu~G+GQi~tn!xaZEBFS^^#0B~sO-N@o!bvIuees(6MZ!@48%fKs0KDoo zaT(gBPp6pXswCD%=}v=t?eO>`w8Y(P_B#tG$e-iB*;4NY==QqTc|#Q7NuqQ|0jx^8 z-x;%0!kgm31ZQp6BdTG}8sF(m$4p*5sZiJ={D|@kHVks7n#2mJc{l0UOpi0EOcxrH z9-MIjJJL*cQ*LNFXtBU~o}UYVPcWH=a!o?vRi0INtiD=vkG>x_>~9PS9tD{)?+eP8XYX|DaAs z2a{|jd(XZ8$$U4C+kbF<>BN+{f*cL1t%lQgIMn)`nWKiVkr6YDg&=X~uyd6}bnkr%#ZA?t#t!Y+2tD?v0F zHc9{D*S+WM-r1cMnGe}}a9A(VJ3jAuo~}1z3g3B~ull<~PM`caEmpN@Z@X^y>_vF} z<0sW<^CgSE#&t8A#MN0|&jxx3t^l~Q5)~z71ysdN01$49trH?WY?mEzy3&zzcY=cwirT*{h`DUg_YAxDWp3efBsyJj7t5E24hrz0#_GJ{3wTgw9lfFiI2wgxG zfD7Zaf}2d?sM5`(Mh3+4;9gme0uuyt!h(XRDcgPXh3v&K7QXyhd4><3`IDT|jwrBqU0>=5I2D!}yZKI$S5c z;d55YpYpq%#<`VRZdJ1xJ2krfOLVz0@->YScMl3hlupUXqJ^U{C&5~x3g}Ln`*M0r zcBL19Us3Pl^%YFj)NFR)O5~2k%MyOf4*Z)`X|_D4{S6M~U9@q)S6rjLSMbvMNH!Gx zi{Cu?4S^i80C!JuD@cyW;&kiR3XdJF9 z?uXEwST}MvvrLfusjmyZ0D> zB0J_O8r}13-7ZlkBK{-KdH+JX5@5mBmgkFaIHOGD1#K(vJrwCAUsGinEJ%9fY!v%Crdv)+(5`6#P;Vn38F=ybxi_8IJo=>*^8)d}a_Z-5u)d%_72qYLkhk4! zJi1p((j%&r(eUgLARCw$?~9*K!)$z>l$u;shTEd!jf#!U)x;IADm4^2Aa*6q4A*J8 z&wB1!dldST3ahGs+J+Yz!G~{qJfWiT+5EVziAZ!Ov?Z0Yb$sdHfq=dqYphLyMsy1K zia7ZCCbul~;C$iw-@}=|x7m7$EP4kD{d&*Gn6G2tT`0f;2xDp_VmG^u$!t8IRXP;o zB6T6p56zEavRpojSozcZcI~x`;W{ppc9&1v3Vv|~yJsKTBh;jaAvYxu0t<~n8RqB5 ztI;cVOJM#3+ON7{zQT;JQwMpcHxadGejY8P{p5k=RH*BN3%MV$cLF)n7aIE)!p{bP znfd_9KWslt-qpn(;CHLlOM&KO56|`EjGw)C4!mKg1{L87LlNqY zTD47E;jiCLWsI_fFXgqBO!g;Rv|X;e1;62Y^6c0%CYz#@MGtFf(PoAqud|cU-`UN{ zzlqJQ7Ut2idb63IDQe=j%Wq2?o)_aHj3HmhG z7_FzW3wY=C5)8$?M~v@}*A=pKzks-jK(9n!+p0vnaOg;et6V6%*s zzqyG&h0v;E5RxH%UN?*mOXEqqvZ?)kXKrWU5r3QWp{Ys4OJNz`}MnHCTJHXmYc>-LX#8GK!jGk#5rnb7VCkp6o}a)jz@ z^0MWthSWTGEUE>n{=c+~um6!VpwBD8Nl`kapshDjS|MP7@vnV=_jW*os@OOzfX{!1 z$XxF>`j7!xleOb<3?pm)kxl2~Y;c}E$!v!>s^RkbMel04*XLhpuaDgkqfy|-hMCK+ z?L@X}TS~-?AnAo3y7To5o7n0?q?D)*Ko`~`bK*&-d`9QVE%rg~u?iVH=tx|Tc zTehn;mC0Q;_c5}74ZBH`#CgYc(5ZNUMzHEQ#YMo08ahMvH@aXg#O$VYUu(6AhKz`O z2dT9m{ATVTXS_THy(6T$s6+(gjf>sJ%F$0>(*u)rU9{ruxZ2|5nRshShM#dMKtF9q z!OiTr=sw5y9kf28-eX_z$`W$)ADvl&jY<;`s=x|A@(W=fEIN?LvcJfn{pRov0rFQ} zx)M!`hN2Z!ImsvMln{q9p9|7S|F6oW=dArO z=sC=!C=)ywUr zeP_1Ef0ggZ$&89B&VUX9g6F)o3q39b_ACjN6GsgFAu8I(Yi3<#^Ikd($7gigVUq4|^lgEV4_$Y?k;bndJ_yl; z{3-TgxiAP7ru0QmQO1;#BgIcpzUBJ3tb+H+yp_h{0N>KZcui*bmcPXHZ01#nkI8AH z+ARj^786`IJRVBL8{gE#`+l7%`onwKt$uHDpiAN74>$C&8=FwDSru==iCl}$Du~Fg zmgXq89F97f^ED&j>u<-#h>OQU;oJO-J0tYpEP(5U=gK3{&*Q={sXfWF$o+!R<9sc4 z8p#l{525e@U#fasoD9{4)mttP-uigM*?M}cDBoq%+E1~twMwr%!0dav;S??X{bXYS zI5gum!YT?Ywb9m&@mW6XUgu{r2btXvD3mnTx4hY`2)orlbs+h#<9o|}@t8#RyZf4aMWzLcOMk9u4|{A8XPm(V!G}xDkN?g#gF4S zn%GO$@cdQS>(oX5U~_84^CyMyz0+`FFU3kHfX?gOAN%5bBO#NRHG|$_gbmZ-C!Zx4 z2BB+s$}eAWGo#xS>BA7ltsTq-En6PIun#wzDVmqr<=5B&3@p`I`bj@WO*U8+zNv7W z+R*=g!>b(rUC22Qio@b>iP09=iob!j1{sApz1}0Qo)>GTis?c|FxB!2ngDEO;sKP0 zSfho?Nf(!8m?ZHj=JvJAHP_$Eat*B;f2NtK93|0Z-TXogla6sSEZ@Jq^t##@e)-5m zVDgw0f(YuHp_#D51NWPKD(R|yTC~llI0WJC559A&`m=R0Wg){(frM--~hnNY5 zL0btF{z9{@w!|hz9Ba++3WUGn41Bp(8z9f|zEob`%Ufc%<75x_cYxAt9V|ublowe{ zMwguMGc=u|=ii@BMIPRIU2KI<&$O9so^ZQegZoFOhlLxj580HBIIuZIaW|YL^=!?R zC|jA4ocg=SG<|Iz9zj>06@bDkH zcP{Clx&hrNLw_vQ96^KgQ*pDs2cA8i9>N(O?8!qEqqjdBK24w>G(W6{@tb!~Zt3*& z9|W%7&z4pZhA)H{4VnzGv8kYN_-zubJj{LZ@Pm{-ZyWTudkFvO@28|=j0fUXhg)@# zYU$3^$Q_>l>)^?d8}P*c4ZIwgNHTlYJ^APJG;pa^znPbL@zropW*Y* zJ3>1uW2X&c+2TY#4gwqW^l^Y!ABtTHwFR_MfQhFhS(c&= z&M{EU;hOt1sSk7x9A~>w@C;Fm_{Et&0XH#QN?&O_%FS&nGeL;d`x!RB4~F7~VReD^ z7Rx)!4c7w>KG$D`Gdx*=6iI%N%-)?&SUh>^>mE+{-ZAa=9y;V3O@hs#p%T4Th4yI) zP09wL*y96#Vh4tym~mJgQtoW4H(7>foveb@SsVvN)3-ozjaA|DT+5u#smnLk(ge&h z&321rVI-m#ZZU;M!mn*b(#w6kD}mddHU2KFl>TM!`oad{2 zmdl%7ldB!?S^ImiIWym?Fhd62su5_oqYRcul08@pj31EO0vl2DpO?*+cXa0hzhQ3* z{t3Cgs0T)4%*4XyxNZr7aA$UyTAsMlx#?YD*g6RJPOxYmoW^E|y@$VaJ$ot?!xX7G zuyHRQuO_y&TxsS>{_u$1FE!6bWB(p+{wgfi_=kzhZO3?kljnLBSm%EpZahp5=DlY6 zYyUcSr!=b?a3d7B%Mb7W3ezSzfCXwlsm^BAaEoe(W=syYelWMcjRq=laO)PGj|*R( zFcpAa2yA!i8Z{x!Xp~WNSKNw1Z;|HN%%guq6*xMCFw)$|?XRQA(;oMb zRclj1^oQf(d}a2dgh|TY*K*6ZuB5z8-$?Gb<9!@?Dkk-UndA~dul4hx-?&ijwi%i; zBsfB@HK72O6$@%tq3(p01>@03Pp8jgTfCF$}t*@zTcA zLv4lz$7!4^Agcvo-=QJHOlFm_-=We;@#$cyfIT~bM>+EijlV+nMB&VY%@YmZS--zn z?HsG-bJX#EjuH;??+ZVKE(DV$sBPj?>>B(G9Lw@Ur(_fHsdx4jxB`0n~4WtB#DU0)qzwqtyMqD ztNeMD6F>y(d!55$HOMqmY*K-INm&I$pHJGhe`BDVsYn23b{6z{`;ZhXXZW#4BA@RM z+iv_w#u(|-SP_#!8}E0mWOkGC3GvrR<9-X+P9~#F`qvW z{;cFTT zP2M0SLZXnxoWiCi zz#Q(0gg+d*^GC^a+ouGjQ6}v@Mfeo?5Hjwh4*i0%m_c-9?Cd=2&jf<6J0!-DSwVzS zei$iQs5>5%#qYD_rZ}ef=+mZ#{_=3^mg+djh%7{!RfQMV#W@7|3kQjtzmH4`e4DxT zq$IO)o%|-P04e)8vi6BT4Wfc-!r?-lrAD0lJ%z-ksh#r2 zq%0r!mzMr4YtP+G*%iUJ8^Y6j-#si6U9rO5Lz-fRm!ySLKgnpt{YPXzY`*77St zF$m189ip``5mB{u>m(<8$V0C1SYA#(ItsSFs}ERBR64<;zt&&(n{k2ch0k?>ccSpv z`Qzr123<$D246cvvm;`~fgwPu(Kwm3m0Ft?iL%f$9r|^h@H4;g4KDjra(fMCBfy0V zX9+(HhBc>*kMOyEmwkDY`j6>~og~LxSsQvSGWWcuyI87j-``LGDdRiqv%&pjTQG}42HR7Y#>R`7{K9|{*t{1;SZjY?| zbbI?>Es5}o3=lz+jv=b16zoIerfi})mf2iV&L0L_7|y6ENv=Ph`+i5h%50@h`AmJf z_toKJg#!aD>Y3?jt78>?*jY-ingEz#0FwJzM5h-`lNI;t!Rd#j_z!wl$FDmfVwu2_ z+pGI8r-jtKXX1kg8L|7OXQs4l^>moEQgEtyG&D*L*4JdnfhnUf*HbQ;n+ZHMZ0Xu; zc>zO-g#`ZXf)ur#F=hhD>_esNN%R)&hK|PkdBH_2({;mg0-C!9^ZUeEL@$4ry$Mc(e^oiCSeluvXjY<&=r)A zyWNW`_QO_y=&5PjbE`P*;e|?W_zOQZiS0TaxC6uBxDXD5ukKg-^)8hG{sRmA?}7Wj z`GIxR0Fn3^t5@MY4=wwX>pI%>L*!LGV&3a0{zDjBX#Hb|C0&A5&VVh#oDXW^>YLB{ zNmgYz6&s9`6&$1}UP!da7#L#Kxvp;O>MTg|W95~nj3kYSHG24V%@`FXNys7W%?ReD z;XYl8Cg!;sqYU&dp%w-TSE+mkI;VuCsxY=*->IGkmjo?*b z0r&iC2I@LvG&&U`6U9BJ?8^+8HuCcdXxwtSX*@63R?AY&gzhJXktq3yd|5+* zL|@P9JyW3E7ku?NGegud5B)yw5?d6kecqt1gX9~|P4C0-S5_cPmD!PwJq9hbbAD}k z5@UtpUtqJiwFv8h1s8w1a#)RVlQMHD$r2ZB>8Ss%NF$TGf`kEAImVM4G*+P^3wuFHSLaZ1=jytwPcuMoa=`tL+5sVw8GzTG&i(&^r|KX#o6oTt#Z0WX< zK&R~X-%#}AxKklIhcpM=d1Qih-ViJVG-gS1Pgoc#HltvG-_plV6s^^!dSEiG z9wfiv2zk(vnUX_pL9|b`OU{-j(vS*X5-DLL&0)@t3zy7xL6l>jZ3|PS3-u)+wuX~Q zlLbX9jT4(G3^R26JeO+0;cs-?qIVQ-LSMse0iUbm(#R#tf}yQ6HLbKIxYFw#h;vrC z^o?WR_8%CGLm#caEn`ZMGMbsC7m@j91uTESPxyhS_Gh?(88GS3Iw8WChxPH~%i7%Q z+Qw)y)J0d~7%EucJ4n?4BTEQ4L#J&JwDqzEOolkC3GtgL@vDOy^rcoDCDMyx)HB4Q zkg1}=xvx4q+y(7hQNw?aFm5|_qLbjfv-{^xYlCW&*mO#2E)rv0aU9V|;xUiDf`gcm z@N>!{K9xQ%4OJi1V*A&J6hAKJStuLeAACJyC468ifyr3J`-!au9Zv+lfc z)rYkI#nxN6MH#-`qA+xabR#gNq|%+zU4nE-mmoC^rP7^}BHb-0Fhh4JC5Q}ygrw9E zQ-|Mo_P+M{zJ0FuPk5ha-Os)5b+6TLr)u?v24{wE*uO=Y+uFpdNKYKt*5p>`C!np~ zG;OA2BPggG7_U&>h3|(>1}I|s_sXDn`}V|G0{_?t(i6Xbp1PF~rV(2>ryqR~m3Y11 z$uB=Ek9}{PUa$g$5`qL7-vk^H7wX;_*g?MW9leK3`XelFM0|(^tiX<@vtYeNMDv-KzP^EN|hTgYO^6p=;&!37Yzc+wJ6LY@6$B&_xz?(JDZOhE_RnH)O zDb93~)%yGs|E|&}IuaTtD(w|DcSq~DdJ16lghS&DIu}FL4P5OU1if=oeRp7A;`WzO z2QM$)PT5uvI@wtP%+r!ze)qypNv(?{drK@=B8BD3Ov2Mi<8O^FJEeZ6ww-cQrs~q} zy8ED7KaEX5ONI4y`{s_&J?D<(Bp{Ve{ox_?J)g^=-Kd5fTjDJTNz8XM4+ncI9b}$S zLlsrv6ye|5g!S3S%U_Y%-wvdE@moMEZ*|3mRq>N&dMpFgiMUFAs9JeF{QWX%5r!5A z;4C-0?R3W+!6cAh8hIq0A%0{ut0Qem~y78+{UkIzbjdAJ%o{Y2$)q0w=Wgf*N z>Q!#bHuuGf5Mo_5Jkc{1taO}4Q6(CDoPw#L9}bl@;%8;o-jDS1exiO~htwPL&1iED zu$=MD!nAF@7rMp0>2}d+-+G5osSo|(p0sC79;Ta3qdvK+JQ1YrkAyyw+4%-E*^jG4W- zKW-TAd2_oX`XhFnFO*pAu&`lDmdzIZ+wa-?jbWDt3A!&kBaCNJ%;n3fd1%8yLbgY*YGGJBwY4L#LyFcl?{2H5#(Y<*Vf;=m0xn*sOC%m8((+AD zia(>Ezj1=jWoQ*>rMai3@*^HM5p%z*f*rM1w5mgOD?EX@TZ`9k<)KjcnUIjHWi(Ix zLzkmKA>xxJ6FVpXOUn=&L0*=#Gij1?{y|zX#9Ck___LkBhK5^VihWw#e$B0c((a$4 z-KZ++Qf;NQwl`p0uB@E9YO{8`*izp-gn@MMcPSYslcAyu7%o#`0ykH$(2Ub4b+hS~ zc+6|~Z3RseL0^ziw(uh_8N3wg+KQZRDLdSh?2vxZ0*xWly(De=_Ozo)0tRe%ok;3( zV76Rim4oY1z_j1#ctg>b)*HfqziSA>3wLtYMXK1cHSx7eqCF&@cF-CF5tDqbCO?gd zbW=?Rgr19r8M5sZIFK`{^|O~b)muu#C`#c`{YQvttDotKXmJDO=Nwk*2_6KW8=2b1 z01MH6ptgLrG5wP1P^{j?7d;#*srQuF5_4jVhavc4yfw<~$zBo^soyVghYPKMt$HEy z*+2D4+oYpc1v%ugKRymO+&-}&sEfvj!I%7ZC3fs30PFI8(t#^E1)WX==1opW31Z_^ z-~q8^Hz%tLq!tuqi|sK{acHjN=iv6G0S_QK6%;b<}` z5gjJ65|pR4Z8jC1^AhL682k|uOhweK=78MY9C7N~v*SF^YONEBq4Nmd6APM zO_D{JBBz1t@#xMi@%OXL&SS4E3p%1~rH)pC_qBhgN|l3hF^OVqtps8!_V-?b4cvuQ zM(G!SB(wgKR{Rx;HFF3^B3wezv_#6Yy;L}AJzw-yKa$tT3?;rY6%!N@0JyAJhcJnq z`^dC}dTgP!=2s>a6K<@vk&Ykpj+2Y~M(pn?t3IeQdt_~AP&&)dnoF~ra99#3l&I;; zr{E5D60eleuw-}`ScFvz1t_nu-@H8@_A;7$i+E1=%7w=b*dAQ_&AZK>4t|ID81v(3 zRnf~`1xw!Z=Y;`2-w7{`h{~IH+Q^-#cpWZ$5Lw|^BB@gdIaR&GKkIz)1)*aX&dg8Y z-D6fRt|l>WQ~m%MPDRz9RXJsL_Ow*?uQ{mfl_S%&tMKWH+TH{Z1vE3)Pqb3%!&6+2 zB%&_+?Zndhov79ft$K-1>`&N#{4BU+XD-KF@0UeT-GfiT0*Mfz&D-_j(roZ;Ge*L3 z*qxNFxLvE)Aox+f06QDy8SP}76~{LHyAQ%yX=Jm;NqmuR-&pHk_$vg7{7e!h(VlVC zIiGCSGMXNqGbO7@`5;=R!8AHo)s-@W`GP$Pro%}Fr0jl9l7^@+zeyccIs|XPv|e8~ z4m+*!j_KtL{k~B|((II#34dP}DmC#Mz?J(65O5VslOV@@YW{lPkLi=NEctr4VgBD` z)9LNl>=aFaaoE3wF@hIhCuB@LVEjLtX_iID>;S#LYqF#P7YUS=I(t-xbdJ`k-!aLJ zgM=}2uj8D&!l?ruSR3}>g<0_lW#WS{^r3=3v4MvUqHDbU z`b10;))mpSwY)@(ju0zso(UacWq&la9=!(B=^tr=P4%6kKaDHofj@@i_c`?iZ5ShlrVn_%<3T(`>R>^BWg=*ZCIet zmEsf&QB(by` zb^OjKrml@!E%9Rc`U1rg1fFrRn-Nz{1f;yyWj~nd4KDQNT`Zh&)`@PQ$5u;AH@mjM zt4wNQIKS-}N`Qkdp_>Y2zS)UtKyyto%0lHTpEfYp;Qj05tEkEuJ{6x>I^)=4@-?w1 z3G2r`&oivCr;J=WP?IbCQ7FZ>T zc?0K|jX@m@Y1%bQX%}ia@~>)d%PH?kpT$A(BYd)Y$6T5g`7~&7;|b+ORhV>lt7^ou zx7uD4Iez9|&EQsUbal)O9)0F0P|dWK*k*=J#ar{ECP$iQYqV>>EntUE#k3h`T{mvR zH;YVTg%10S8=$ zL;bHaIF@K=q5A_RH-oPKCZ}8p76C1uSqvU3<`x&O&QmVDC1#BdPT=-_ z2tCo6=%|qx%$CIq4ZXN`Ap_z|qlG%RZi9r>-0WgZUBqN%xcMh%A2aG}FPK_$9!e4> zf5_Sc5L&ss3NoPaIzLmwxHJX~;E%ti9!*tAQNms!1PSe1&l5`LS6)di6TLPs7`dGa zX6#z0>?m9iA-2-4_Ent^J%2mX67vJ{1q^teJFB!K$@78V5R@xwi-Q~UE0zy8zF2nY z0M~pyUZu?c1FeGwc!Yij7n@`KRdvdGC6@z%^W$Wm$7U}bjFW!~p{1X1#i6ygAAt&z zU8bmL@pf+z@B^Gg9NaX)UN8*Ps{C&2$uhrFHDFWK#qc6(2hXa|!kEjEiYysmWi{AC zxpIK9Mrw!_9x~OR#L>`wJNqjn?n%Rc)H6I!hg;y(fE9<@?g~H9jEny6o&LLpz4=LV z5L~c1wE<*Y@Ya|gUBY2gQcLQOVzR(_+Wjjdgf!(YIN?v6)y;4D*X$c4 z3}0TFHIaA&bcI?ymE}w0e&-SFg01AWwsJ=SILS(Q*pAuF7ja@GZ{7G4Ine|0ZCrDY zS*Ikk$&Mu-=eSTg^{;Dv&u^FZr92Yxtc;&!=DEzmGakZn&5UO9s;6wVijI&Nvr`yW zr4pF;hd?c;;;xkj!ifrDo=OuvtnLq25^Q~39qDZqRM|y%(_+TQaWGe*MV`F% zfhOW179$I5!pz0!yy)0NprjTq8ikYvBlab5ogbTL9DvO;nUlHjx53LuiGf|JH~4Je zUM;o+i0n;ACp#d^?H%IkDfLE~wGi=ERH}m*sA*o)GJ;^el16$hvfBZCRD3JkrnNR* zuV7_sb!p4xU8L-=5tAt}=-&(ZWTGEOgJnUDvDO>DHAKkXl%m(7@)a6J;R-VR-Yg-r zsXa$%E%jZkPvDyQO4x4XtFSQgieKl2+FgY8K*a}YtS`LZyr}8_-K6E%1#i}884@H^f!eWt#tE$%0q!(-s4L9c3uV|&}jVe%f8+z`H(QFV>z+OY7?=xoY-4XY} zDg-_+hlL8G4f}KAyU#Z6se^Gni7GA%{D>nD6_+rGxroybbiGs2bLGFUcnb3eS6KaaN+^`nZsfpp_*8?+QHC&ux<7@{ z@!tgv+E$kCAWveV=1~7O*YYitoyFZ~a2E6Z(f0+nf0TYj$=FW;)*lc^TZ#!Em`^WEX_LY zGBdXw&(d05?5PTYBgFEKUnGpm?f-t8`^bN{#c#|g8;{YYDv-6kH5OWtTb zpKvL#s$HH!o~S_40;x=UHt z?p>upMtsQN`JVLU3md8$+s}%6f0!q+oGZ-QMssJULjQfS)tUv@x8G3^a%G7luXUJA zkVK4?Q)dhF#sSf1hq5wz9@vea$j6Oh2(RJ-Pn_sGd)q}T3R&{Q0w$lZ}Bd7%uw^N>`ns#b1RX?nq~sciH-< zMlvztSL9EVq0f?E(R?bp;lDIg&bSnT=f897ef|Qs-^~6eGW=&hC$g`-3Ht}+4W^jn z83xA|9Ny6gDF|Uzy%(VpZBd2RX+dYXGNuIV50{d`S16n_r!#g^BrT_#@uN19oCm*(qSLo|trEs!W8D6p&Bf*n-m*`P zKDbrVnp1;a(BU`XQ)l%tH!l_xvqdYlUbv{YTop@LbJ^?wCHmtVxIff-+8r zzsa+I?E%*rmH56IDslVpd(#sM^1z>5&I6v&5os5@_HA!MVfF~H(){faf*r`_@+5+Y zkRR^S_}G6Ygj@*uc&9+D7*;o?c}IG$*Zk|u3Ig-Qcw(SkvORZ(VQ@qCu_KxJ+i3=& z7D%p{1opOK#2IMdAKcRuGjbo2c>E@l;d7u* zZaL+D>qSFF&&v=CejO*^W(eYZ5_W?)`e{!;P$H(35DjRlwghT<1pPQy~hN>zH$$` zZrrhg9%6f{4ztOPU(|`Bax~gfH<3XZ@)9TFo zka;^)7!OI(F*~q*t;|^Lbn*Lr7;I!DP0bmH0X~ zCuhro$Q&38f)mZppBv#Z6SY7pbdc^;0pg{V{7Rp^ZCKkDym-C>Za<2asM=JZYb=`9X-9fR2^zo4{*Uswr|14$oFA8lPa3qN_s>CD#07ceM{p7HnNN&9DW1Zv?BB$ z?isnBjx9U8C=*zz-a5J%9&2{uw-$J~+b82#H#)QFf+VtF2&RbY=wa3gR_>CalfEHb7Kj++qso=?Qw! z5A4!ex}G`(5@-3H zcuPGV1UHQCPKuTw_oVK6vgbD>RK)YxOb&NV*B`@0>UQebV{akZ5VhBDwgr~QG&E=L zo4z%_kYBL*Gh)G9-;)+19HHnpdkbv?d=dN%Zk`>SQEEwqi|mMu=HsM76#Z>IkiQ4g z5rpPftNR46DC+sNupF>|^h!0F>RoN#RW|YU`gXDZgC{rsv);BgBjPLi3oN4k!h=Ev@uvM9<%6IRgG^V7x6Pf@SyyW3Pa#3&^;r)MT`1CzbkW zhns1BQ7}0!%Mi|5?7}LP+AsRIKsO1bzLzQb*1XKw)U@okTso)K%sfH_SU^)S6eK!0 zMertgpaWY67J^c5#K?^L=KRU$zCdz(oNH(!8hYa z$48W{2GNF?pnXL1Tus)L9$P7>7*K9z9OIM?y1W7ZJcqfxL9%K0g&$`JL2iDgA-W+A zE{fS6u6U>6je3q>TF09=`TAvobt2lesy-GpC|tElCp;A`9E^VaN$$ zt=^nGprK1}LWpP1VtAjx6-;NA9u)Z+AyM*Vi@vdrT`CgqIaQ)HTONq=#HUU|v*@8P zYbD~RmGV7K)I>r0Nw_(`1dUb<1v&`0tQDHdSvs8}sg&I>GC8DGo}`}x%b&y$05>PI zJ}MxN^kdwHSaQxxRz=F&M6(M*T}(QZ{Ex*t3ZQVSX`~%qZOIT01)xC|T}~vWXGla6iKAVI zSH$W`M|mLe>yP?>^SP?+tB#|~KaMwE#YDQ6De{$BNRFI;L=md>nD+FtSQ??5vEh{4F383Bb`|SRBBu zJS_ot8C3d1kE|z;j4uu4WH4tKn^~~^hV)M`s0ea^_$rkKz@l(M63IUQNId>9Rq*tI zi)G<41SSC}14^cE+x(t^^03}}mheMDyIH|yDBXbAMu^UM*-=4%Fc~*1nfA{MhNP?P z`sjGlXJMHEy2|=U&uq~Zv$OB*<5)iCnCzWUQtMTh&-@TwsQE*dDqo@jxRL6%vTS?f zE&$|k$rob(4b^bXt4u%W=*QOE+_Psz!<9t70kd++6NJ?f18WJD9XirI48ej?t2Qv9 zJtGY7M01jX*?FcDp~(kJe!eSOar5>lX)QX$RlCxcT+OA*Gycb~_NbmR{yac1n!dEy$YMzcMuG~$_q*=SS7mskidn(t#~6m(Lp_Yz_jm@3AY1e!WIfqGFzZ)s=U&{TN8Iet>iPGJXp6a$=0RbmE54c;|I*jq{nztYZqLRxZF+6WM zt~_8=V3bv(;tsv`y<%g!Ow)E=(%K;d?fi0oz3M>B53oJ5{z0;7?ud2vOqLL_A9!;nRmgV+v$I|B%2@r54J+)w?^~h+ zBo~qi9%gGThN}OCTPOd)EnB=ag}rw=4uX3E?D2&xz=bjT7?m>q(Gf`%huAUPcK~}cDROJED!J%zaim1{4d4IjV zOec126FsW~J#;vcgJa0|YU|E&hO?x>x&pjTx!@8#sMy{7$P*QY_8;N`Vn3%tA5RQM z)HDgKl!$g)**fM>N)fl39KS5o63={l^pt~25Gq-enXt-L_iWJr#o)lwaq8?q{X_~FFXV|?>z{Y4*Jz_ifuKeqG==tdR{Q-RnA#F3s7qryl>Qh1fm-7)uq~7f_FGnx(E(I)_lnfF#&$&Kj1sP&KK~>Kw zE;_y_nOi~sD8@L~lh;-@_C?%q9ZQO*k>QDI%Iv!c5ffHWkD;w@fV-|UGVtPT#R%G9 z0{)DTsQPUJJo!k>ZHJMH*f068m;6Epmv5r5$zfGJD$S6V+qfW!f;0AGq~)oSbQ0>>(456IdkS8kbDK46R%;>LY zQVN!L&P|e__T#CR*b|Ei^O3cxG;6~cBjnm*HY=3J0?Q@?HDmom~UZLoQ?;a(Bdrm_Yx`H|TE zPcLbnsn0o=v?p?224T5oc$Eg*+6M2Qf4+FOll}{K+J6?NSz?JO~QP{c`&H+Xx7ws~v=W}jxOzTo{u|7Kht;7{a2!K4oizkp^d3~9;w828Jf zJ|VE9^->S|M}hM8XS6d=0~Ds1`)X;CjF8&|K=;0rbST_nZjnTuMEN5(uME)st1oy1 zzc}IFnbu1!*e&QMs($`1(fImwp7Bg~wgXUpzn2UFXZuWoBI7W3-8&IV_UEWl5G(2| zZuO??ESIg8IP&_$M469~_0#T6g-8r)# zTzGKI>otLJ|NS0|wYw>tlNegdFgP@`{UxXLOMQ(fzRtd^=Tj^2 z1jOG$5X#L;_)!;TF`7e-x${A3^6y$xY|7XVx;TLn4<^N8E-s4q*sGhN4Y&d)Mm!}l zyjh{un2b$`ye6wu#+~KrN2uRlS8Yk2!rY-+GrbABv8pR+x5vfNY^xx;fXwp!GZu*f z8mvaB<1&F&3_GsOT(@;@r+eA9Bk23UHoMtl+n8~^!=Om>hag1QKigsoJWauB=iQx_ z=CAW2L~*Gmr;0c_-oGH@QI{f(5s;gZU6{zlhxUOb(DnVNvqJC~qxru-h~A*_TV&MP zB|AFwehYP%3IS|NE7nJ=VXhQA{oz)&b%0;dftKLuD3fI?Ex00CD-?qWicopboT!FcCIa$ zB0>dt%XLwpsweAhyzn7!Qia2yzuW@I80ueljrW~}^mA2h)AH|dVxNr(gb$QwZq1ZiB(ZH4@JN_B6_y||&jqCH?;re*O)ru2-y?0Edu9K^n7(VlkD zUTO1d@h4b#_OtQc@_Fg9+cN}jsUH>`4LR*7RY*)fJ*&YKp4Ek)U^XPJOCX2H|1l0$ za>t3XJBMl1Ke*LDGY`yF}#V?+LRZ4hbj;0 zHrG6PVG4RCweiUq8N1}GfJg9C8EB2@r5Q#FFYzw6Pp6@E8m<@+^X-c*OI0;^3N>M>%2+8v8CFovR$diFV!T84wCgJJROs_1w zmOnx!Ywfxvt9mC>qNp42^P^S0f zmAF^WgE49d%Wl{_+Eq!dO!KGGD!L!tJ7n;{qN~w;r3a7fm$(Xg7B7X0(Ouotu#s4A zi=Mo@83J*G++p{o-~X}Qj(X8wXSTFJFlGHW4rxa{-EUznVX%IfWiSXPXoD*nS199g zL5k1lU{n$NM3=n1$;Bq?iA^4@x2v^X=aO;{7YZV&cEA<*EK~VM5Z6lwh-Np5^o_6 zt^4v3#zb})6Av@|I1cz#s~k~-nh3Mrsk=nj?5me}35GoHE69s$J*3eT5Jx#lu`G1hw^qT>0k!J+Z>`q=`jxKvVs)l$1m+)gbZs?-M}T#4~i`{8$t z@kB&;pjsTn7R?{BLmDv0$%p5Wy^v{mgEyBxsV54fKJG=x@E%5gC>Fg_7@~Yqwi5T{ z)qujx=h!tfXt)Kr{`e98-D#$b)Rp2enF5kLbxPWF{^>p0#GAW*=v|q!x|Fq@ZxO`$ z?6OD@cUxmXQNN&a2D(esoJ&azC89a(cgF(UL(#>N^4Z)*; zaXM*X37!VpVPvyK@r805v`t9f|Dzehs0UseKfCKz?jJojR{9uMY`$5m8m5NKWC>f` z?Z*ITdYuhb2p>soq(zahW~$IgV9dMZkS`9CZzab@cD4(I>_B+XHsKHaX!N$BMYGlj>4htwMe95(3Rqkdmr5{#CNbJ)aUuDH$gRj>7+J@ z*=V0mC4YIQtkJbSjS&**O8P=tP%h^b!DU(CRHpz!UBLz!z>YX}X^ z)8qqOmfrL7Xr_5Bd070qS;qmm@gok5j5y(s4b^0~u8^Ye53zS4Gt(Sfd*z@Cq8~A&F2#IaWqh>k1%}=_nfbrnMiXNG#ZG>xVjzV}`xJ#7(LNKE zCa)!k)^fwIliIN~Nr0cZA3+ei-T51s6@ku>8sMKOBG=4;#2gmG$T`G>s z$VhICjH#1rX)+14B^9?$6b`Htg$Fw`O4SigSU$rLJBS^~lz)K`xPyPBAG6<)Pw};m zg@;EO%WL?jEW3nKVsX`O0AR}uJ>p7~ggbjA0n0N1+8JW>>(nQR_EajlM~4kPPSt`D zv5TI`uOAVd;GY-~dcKdoygRP>!5P41KXlWFWuT%k{%XKFG|ZJ!06KYB3b#dk*ecD| zv(H3P6=|+?i_(d>qmBEs1H3j&rh$1eCo8Gyo|e)#@p^hPi)_^d&(KTG6I}@r#ilm9 zh@4QP=Hav)lB8cSJ$jVS*>7)jeOnte#Q% z;_s+v$_|d=vqls{Jc@d;_4*w6D692CRwRn&r;A`IB`aY%drpPm;aS%2o5}N#d;Rai zMc=OE_+9z}g@P9IXEGVr`o{ejecwrxnpsu*BDbZtL(9=cR@3=4m~%6ehd&GtJZ_0k z_}4;9!R3iztacK;Y`NPT=!+7Wx9t4X-ey=h>LmEbh7-xQe@p*Wh4I_i?r^G?%cG6~ zH}q=Jrs0Nb?ye_`18e;0Nq5nZcv<1#M&fvh|U;x|?kZ4=H z7#Gb%ua0*>5~Jy=$*3txkK1!=<;Z&ZYYJhD>q%Hhf&1u=jlM89c;@YA&(@4xqBv75 z{khgB#@PMYHgW?0}h3Vc|`|Y~D;C zm7D-iysPDVzJ>B+BB5~x+a^Pa3DI2yZrY@La}bIDB!C*2vSNNIiZ6dmkco471TJoX z-A_X&h9?YFH1*x5OnW_E66Ekgn_}-=q4)C1T$;i-(=C6z*46Jyjzq8P^2)kzjzcUi zzT~*)z7^Zk_s`d3ZBISeGmgQsj;B@llrg{Jy_{DEEnozl$yke zrg%SdrAHJ1=oM1>mBJZCbuAP8i)r>;!t@#D0LG>p^Qe@d=X(|K zCy4hXjV`@JrbQ|C-I)&EDJ{g=`g8$2D}29R1`WT@)R;(PYBkc%?==wTg3`Nc_dd6& zVh@|en^~|+h&p&E6xMDd@5;Z7nEi1APE&@N4jdawls*IZ@(S^dWxBERLoFED*VEVY zJLz#lLSOwV_$<_uov2@U%ler8m%mD3TiWYyp5)nJLML6ODbpk8H;8TB5#H zSK`2hYQ}d=)0S=$!L!k7OMdAEPmXT;7B{Wf{wOWpRxeVLCdrelscgyEkyB2CtgFj$&JNgSBtQN2@;+`mMW zXAe4vta9!?e6u!SeoS=@BMaoTDbuSrLT|`F7T%E|FC13}=4VPce|4i3s!Q_FlR0#I z#ESF9nBR86U7;uA>{0P!hLLkBVTa4Y10M{$K|%DA6A}Hz5Sk2{5~Dlpzk>XY_!QxO z1G%D=@=HklCNIFudY}kV+mJTh9|cY#=f&8?X-3m1Ie2vKgDV0pDCnk4oQqw=m#;F! z?|t01`^*XvolodL;MDK8i8e8sI^{1*nvIpLwSIK(5<52Z6>x%3RaV)0xG&kYrV+nu zK<(nQ{W`9`;C?mMR&g1Z2_i(kWzG<2U%J!8=3M{V3YiKia5Z9Y825X2z`hazy=}T; zSNPPa^NOsiiY(z4mzGGKHe;f-X$;*ae9SzRdbR-n^E-j8!na+x7?ZLelQAn?6r6$< zY^`Za={hC$Z153F0b$Dxr%HYCNNhKkGJ$Mcl^1EVNiVA`-%%Fi-n_P3 zM1RegD@7C@^v_U4Hlx&ShGjIyzUKJskt{2?Ha4)9%kw{yfE~!s_Pypxf;;bi&6AxQ z{y|)#b;Vra1pZ<7E8cNHEak-5t$ii;Vp~evm4#uYdR+uNeHt!T3>iLSKXHy)ykgH@ z4$SkL{%4cwN?Ht2UtQd&b@Y}B&h#-i#6ls;#48(BjXsEHqfXCCGixCyKxb)-df})@I<%HST%+Sxfq}nV6Rb z{U4juOZpS(MW#FB8xNe$IZuh4!bWDHUa`D@-+#kmi%QIR@kQb;C-;te3_+GZt9A-j z)&TnQ}kG!>*k?7 z8G3TRAPV~p`=gr=hJ4||Lpmu2FB{1M)u=9uGDQQ3L`<4vwSK?LT=GIPi`(kD==dxVITJ>3T$L)G_d-E*cc%dJW0ywwG>d1Iw4$11*Ihm+RTOf*93c zM0hFL(tHf6?nD}ctkF|@$NU?pgwB@v*W)jzht9ns)io{|@BMCh|3uHQI%s0XJft}m zzVwEOmovpFHda8OJQw~$m*8cGQuh!%LQ+6GKw|>^*EFbY>&P!e(AsAMT zls}Z5+~zB~&6;agTLsC!^j5a@r>{Tp(Mk8~L_2qEt&5yKm$zzn5r)MsP?Zu)UBB3v8T?A?qWjrKcZg@2%C5n0yc6Bho?GCGEFF?wI?KG23paYYFf>`t8-T_&j>Sbwx5W8Bq>?=!^NIyjnVCcaQmZp$r)B0 zV4-Aop!a@NPhnFGqOvVBwx1$6j||p5uV52C(KEH_LzPRN zQ=%28ZH7NW+B*cq1*cy!gpb7muuIW_&M#J)N_RxA$Q`2H8F7}KKiUrD**MC)1v%fp zQbxEANIG}a!#XKMP+#XKS~1o~FwExK{i1!DKzs9sQa-uedOvaW3gn5Ko>^H}DeX`0 zt+uO|RrdA4Uxm21AoXorckjXjLJ%TNrVAxk_tb;oWmT=So)z+Tiu zbDlSbrpQ=E<#vBi^u+I($q;VoCiEjS&g! zMd<&NI6*Of!FLhczp$Oc8}nRf0Q6lGP$O0$Ci}X2ep9p=Gtpzj|E>kKpuytgy|=!$ zsnpU!I>(B^={j^m^pa$+8k(PV862i9j)=M`Wd!Sfp23#}is~5^wpFfCOql@XEPqLS z&waL5YiTktqqTcGzCM>UGsiov3$gzuGFA7Y^uDNwoPjsjSD4=(FGKKJR5|47FRa8u zeNmawyEw6Go425i7W=7t@z%8cGfWiec_4;@g<{_jlbZ+5yf1sAHubDgaGqXU$)B_Y z&(~sq`Rt|_k^UE=1L_U6V==@T+=&5!{;s;BSU9AdZ}7zj^ieZ!SmygI=W5Dk=3pW~ zaH)PNJTiD{TlKOm-$QH3{3owmliFBYwnt39m&S--{JkP2hXlNq(YS0!g>f9WG)%O= z$VWyHq+vK4`e3jecGX(3e?Q+Jpw+ZvhaJ%H*|KJHJ*o@k`9B+h|F9!a%Q~E8jQxwP zTX{^p>6WGuopd+!9CK45iyVi@>~!~024k1BNItbJLR)^GMR#W(LW;<9#?NGwd2xJ; zDdg)quODw2Pt@v70E;xSGkTx11tj^KKG}DWXU57*USHM>=s`^{$xo#m9Nx!E8t$s+{SDgy=C74<>gMVVN1ahib5QDx1rV~l zZ?4x*adxG9WM#2caHcDNl+ndUx)>EkZASh}(!M-$-FYm$65*bDCvD`mKhS)IT64Aw zt_Oc1=4cF*yl`XXDr6z->=52QwQ*ShqEzx{ap(TdeFYY2yabB)cA48X`SZZL;r)5v zARB0q$9+$QR^(=S01E)TKi80Dfc3-3y;%>#AZV3-D6TJL?CKmC`PXsXd2_dLciOnA zXdekWg$p{^;{_&>-cK7(H&jaP>><6sbzE;FWlcR`cd)v-bdDF-#;D^9uxT~U^lrh5 zRm;Z+WXFIXsdG2r;JTv<2m259UmoyzwEx{TtJNwPb-fFREiddHN)|UMq3-_xF3Z#; z+ehB`#bv|Fw=Mnw}-+pp_?#Dp8JS)i=4UAGRs%af9SEjMc0d3pb)L<-?R z_YJ(TM6%P{gKem8h|IF68XLE!V#BYU!vF z6#zj~(iU`2Vz3;4xDh8*{>}Pu?3S9Uwlr{f4EOlB_Uf2 zGWj;B`velc^Z(HGR$*;M(Y7ei;10!#OQEloT#alGEJHZJM z+}$O(1#;Q@o_)``-`?jX59=Y{FYBLc&N;@IBjm3b1Z%ZyS%tz!%UgTf7KtikJk?0@ z=;>P>VmKHGlp1Z4Y&3j=j0qLPpLg);0^2&0MRAUEYYZsk4Oem+fAG$6bDk-)B8~zF z2G5LU0ueR~={L7?v`~C`S}p;^Br!VjJfIh^@Xdxr#ts$soAttlj@sZYn&=1;-EmJ!boqG-Ltk6oW}aGoZ9DE`~*$uO=TrI5j_tg0+i z)jPR2s)%avD+~Q^3G9C?__?}wt^Xkvrm2<|_-y_EyQ6FEW2u+a>0o_;<*@2-QZfFVr-g56s`#eAdOWQOJqf|e zfxoyqD1XRsx?t+iqMkRK1iO@tk#;rMECV&Ube)IQI^hrOkC{kMonH{|*l#3s2l5B# zHFX9O4_hd-<;Q32^pc+0Z38KcMLiMU;v}eFNDg`a-pzBODQtA!?wR+GE|;y#w`C(| zMmeBRQDFzaE8w?Km*Yhnv8T?5B~=&)%k%!>b7oadEZKzfWgtfvH-v z_uD?+J4WAiE$1G~K3Oh0Ud?XnB`l%tNrk_|2;TDPe0OQe@s440ciXdFT^;AQ*ENa% z)^a%|KKW8w7w^jVR$Y{S+(;kqI}snncf;2U>%0R(xkti6T_DQW8)&G)iLsQ~85*fJ zviAR|>;K|?Ov+-++(5&H`E?Amqzf+9{tlb@|7EZL(MkVb3t4*TZWSNf^>xALsLu(4 zAvIkf)L72{hzMeTvV1>)T<%-8KU7!sSS5xXyzW*rAeievs7-gqGOdYypplt=9#!PZ zIzmU*Rd-h-btonz6vF0KrfYs@#}+d1y@Rj#_3KuzB~^*PEF^gsa2#ze5ACDm77ylv zUr~D}8z%E$G=bR%?KySeDUo=w%J1_+ z-4OvR4+9r<&wprg_m*C`7yRf?yMgBM zr^kxG#SzXU{WE+H#KkcG#k%fM@y+yn$FTQ$6VHyJp|_F$qiXxL-Em9EiVcFze%&46 zL5}2?_tdE)0ZCa}AMVrt*VDr>k)iGBpmcQOF?PV%182ZL+1DKF~7Nfr&pJwub@3T zWCnpdmXIO#5yUOsq_=%0`bvBs``_~he>Q2?l73?!SK~C)XYLQ&_;aDrm2|=qy1?U% zt{L#2&QIwA`%z|F{T^2wAgv`|eL%+O>pB)7@HBA2iY`e^r(?>+pTciGr|Y4unohi* z$8jHh!C%93Z}{a&Hb`$%RvDq~c4qV^aH`Q<0zZBFn~&k7?~Es-P!49Pb_uQBF0B_ifw*N_DIvgdAw(V)Oj}U_pe6yZ%Q8YL7{eKpj0fX zBVQXa;hdR~kQ@@X_ch-{{0;1njrM9EOQ#(E=9+vO0q52(8G|z$)_9Cm5$SDg(R#bO z*}Hbl=>mUF+6ynZXN#-$M1q&t(-DL6%uf2Lbh1q(yq!TyXKncSPv@)v5lP1Ca@S(M zIBUjIPx;d^x~-r*(WGzE&0JO{`z`W?$u23fVd_-nWFaBxPrh#NK=t1+fbZbu3&8U>FGzZ`SAY4!$ zLSzF+>w}VXW%+madrGOU*P=S)lPGzTC;lWL;o^f4FqZk3L6D#F!I1{jFEYr3MQ&}I>DxEW9A2)Jq#g$^G9*CIJn0Gl^m4fxZY z?5t8f9ox*2W}#OskYvKkZO2);?d(4RHq;-(Rb&l#>^p;=UE!a{lcknuA=SCRA}!v0 zrriFR>lLR!A+Gjqk^6?VK-!-J%Cs}fj6JnhE_O)^msv5Ut^YG?9h8Jg5tA;M;9Gz4 zI{;G&h7=Bu=%q&HwYtL?64U*WBNteBQ_<}H#U*E)|Pv}D+TP&u=(-oyT5Uq}OG z7Gl^WA$%>`ul1{4ttRlzI?1Bk8b!fu(+-Pka;909o09$&v=wEmEh+d7BP?Y`@XZ29 zpT99pFx>-24IH`8Y{)oRh&}W;r%p<*tX@=8*hH8;e@#v7XiqKdEsj>>SH)dCeGkZQOqAdIKzv_oLm;5+I>z& zC5M+xtp-VGj0b<3F?$X8mozk=qX^4msRX4Oih7)(_)}>C=ufK_>d8FaT`i#n45{Axvrpho zRoflQ|EXGef&3jq@zM}N%?=;d@1#13HSqoP6=2ObPS=Osh|QkUBEf2o(k_$;?N?6k zu#XxLo%Ih0d!Ey^E5VF@{n`mz(uA>m-Xlk8T}l2EUhHvZ|7#Rp8687@>Rf(D8UW>} z?`ZDX`Ka%EGu$09rohYSs45G>j9Z+p?nm*o)np@sK-W0Slp)LF)zRK<;31g*(p0w` zdu``EWOucBge7klz^$F29HS|%(p94+aw!xn#r2(7n$EUqb!6DIu21U<(^wD-mc)ib z469q^>{8~9bwK77ilT!?o+nuX5tEw20dMwEtF8BBm}-Uvz7bSKe(wzW{&Qw93`S(i zpobcgrO&?XSB7NeE`$6&01w63{8Q<CF7AxyBYL^Lj)8DKte=@1;6=IJ>wboOT~q@rTYXs8ELQKBo}%D#d~jnX1#-xJ0TZWB2aqe_R zLPV+wOB0`c+2Qk_`}oGliMCP2@!%VZGX@cUZcHv9N&@w|pmlZ`JCt(5kkP*ucg-G6 z0C{oL*wROU_I!zKmI{UNTd(03;yVlYc1sY%_51DDBqT`j^SMeb(jLHxaVWno*jlGU zA3q=dpb~mzS&_@lzmG|pIYx~2u5mD}l(Cn4j!i*8_k%&+H3@;y1Gmv^-~!J^f^%ut zOVPc;^ktxCN^Yezf%6a7-??daF%r9nR%BE+r=*aqi*1Oo7sh_sHBh! zkN0K9EvqfcZL(PI%p>j>ZYd~AY9@NA>t8hSJ6P`dJd`^avF?oH%oy87%V&hiOypNm zE)t7std_D~bReI@>xs=5SPhx?oZe}&o2Gv!b$6jER!X*a6v!x5AXYAf&q%gwl5;ME zzuVv2rYv$n-4J)+mD|Y}z{sG@gCH#2vtVCbOfy2+d7gHy!g5l9r6OJIQa{2XSaK%s zXSwMP|8fxvc+|jY{@8D|LC%u2#P54-`8xSHFXwAqI}1xBzaW~L6@|WPIR5Reh%8wm zB%@o1<~?CK^kjfCpOjw?dRQ!pkXf~tbt*?6AKxDu99R3>KF`4eb;%| z?^2F;*i^kJ1BmC!jLws;dt=;URZkN!Z;^|Q50$#3JL%YM;=@s`^o%jy8K|aQbuN=h z5zPbq&`X-!%a#WUXeO!;7uzKTS)49j?)hk4F_)jk#fJYS9@K?C!?I(fc4#1Zz*l@U zUkG|Z-C=~wkS4CvV4|Aot1|L9nshs^Es5;d(d(%jem}-P0u*)c{QjC}d%X2*KT1cya$-CUI5(w*0+o59?QX#3N4W|)W)EcyUwR#~GyRLVe{Q7J4D zvf|di05IuEFrns)vwg*9j)|e2R!pQkWl8h4$r_AQX@|pqK^LiF(QxF(aJI^%Vv3?r ze}H4-!a7~dr|2~{2zKl`_S7=zPnOkr|MYIRqw@{LnGZo5`` z@)`IZf9oK<+AK5v^FizGFWw9&4!e=#Pc@yy{IEEzAl|%e9u|e_bUCb1|Lkl$kw!r-&1&;nTV%`DK z$%@{|;>c%C?7fch+z9gpZ^amm_t`ecO7@~RyO_MZ$v0tZje`aCgFU^aE`?j#yCY#>{gnOCEnzjQ`?tVz!*^1 zUh3Ki+yaqg^jhV58t@>fa0WIVEZ1w?TQ9wBTE%lps8d%MS6BN(;J` zSMQ797PdY4AHB zz5CeE3<3C<-;E*ycVGLaGl6VPA{Rc& zFDdlUy_QUUx$f8j+#d0K2jmMJYM^TgjQkE2p@^GCSDR@J=0nP_nqJXLW zhSa>b>G=RxU84{>wZwz(Zw=y4llg;}e<1y0({nERgv1Lt#Q+eIZ}&81+Wekx<;V@P zAGQ7<(5*AZ8TknKiutdRjv`w6v($&p^Tc%hTyDC>cm<~43h~E%kuM52(_NYJ^Z=@r zS|5I|A&~*uWTlYCjwBTX*<6(GK=sK>C@?mCeKUWq^9f$ld5QCoFy_2F2gEkd0N9kO}A(3=b z9EvQYgWTDS9V zo~q34)!3K-m)bDsJFdP3&r#Wj1R%o#PO5{;P#sS9g6os4lgRD#Gl3J6ahYJ2;4r+{ zo|0%}upTsHvt;Es^L%Nr_z}1jjl1Lj%={Vr5(!+q1O*c^4fzniwCURBfGh=0c}W zGqN?^@^+rzo~%>TJ?q7}%n+>X4J)Jgc9c~y;&;oV{JYeW1$M7r7olTcf9-ZDNY*4j z^0;$fQ&^)2a^Eec_4(xmODO4=MTI6hE5o$6=`KoQ-qEMqlRioXOmX;K9GLpu+r3y&2bf-yAWXZ;%?7Jy=R}qa#!#HuTB1 z#K~GR1P_P7zSDG~GiZjraq6x@#{ZwP}`-z%7Mu)>$qU zQ7H;T1yHy#2b6x25uSgDr~V~sB*dsc?$vl^=oQm7JTN@)KPEV38t8XY0!m7=->&7P z^J@MJ3wlX(JU+A~Vrq~&4jG%4FY**K=nBEd_*vfO*lq%71i-Xs&KtRNS=D!$9JO@n zPFe;wd=e{)8RE@!BcAtJw<^zV76E>#QO!McjG1hIE3?yMcLqZo?v@JQ-$%B7isO2R z`N=9;MUvm$(noTu+cTXkw|3Pp!rIIJg4a9EQ&sQ2k~Qf}F&iTYv-T^pCYNEYR1=*# z8YS6ix;$wDMWAqcuP<#hiS&=4p2Bq6pHa#zBjf{s>C7+_Qdd5hCT`Qyfaiu6w^N&X zaMCSUB|MXUou=jl<7+&1strk(4w8H$1!g0OLQMntOBE+a z=c|q-%sFfP%YBSX4P%k+7Lnfd645YZrLHEybmGQ>VCYYMH>ToF5EXkd?T5KzMjak^ zY?tmB6oT0u*HYEJkK+I?%={XI^mWnKTS5h(8^O_lTI>tI8Y^0Y1004p>#ajc8rOn%z7;DZ~(^KZqnZ;B6QaJT{Mai54o^|fpJ?~N| zXxU(iU?8odOMh_+ep>e~&t4AYBo4OM48|X*Bwhcu^|rCp)}~YUP{BVzwAFqJ%@F@M zFr;*N`q(H?YJv6oPo@+2-whM~j1<^Rjo^oBm|d$|K~o&F7jq?uyVfuEMYcp~E-Rhf zLQ&VH{IBwh!_f5S1jff;mwXQWM&fg6gGS|LU@RjaVyK4Mn5K>~sag)NiRFF98EFSFWaLoWsLxaYqRY#|F+&6*dtVZ%T78IFqbUs#Lg&m(;of6SK@cknPCi@;|3Hv$>_kjV%G}A zCU3w2(X9P^a88m<=bYnGoXQqYy%Cpz7CSV}g6nOm(IRiH+! zWvsz)qMaiPc3kCl?-jeYBf&(jDEz)=rYA4%0o5d+78g(U!$bl40(eMp+HzX+C*|w- ztHePcF033zg+=W3M;y5>^4pV0{f)cML`SL2+2^zWSIP0;v!Os*>UUdzQk?xZ)Y!3y z|AWii@E>3lcgA-FD5a=S?(jOAn$U(7y$a>VC|bw(P02V1{5h;0ze<94H^^xk6LU6iVBGa04O zV~?4sC9#s%)G|a{9!KQylAU2PhwE}?K{1{}`Ew<%q;7><@R-g_ zauSk$h2q_qB5Lnvn4@%!hw#O<(8flqQSaGl04nILfnf&}!%4(lAM0Q-#86bWkjy9c z^kDLCdNm|=G1p{OFSFWbQNvHfUZ@6?pol)#O+x1^>=&W(rG?y{5PZDzTV3dov(&jZ zaw1z;h$(*O6WPFD!T2K8iTUfr;Vwqx5uG+Av&>KU#7LUte+?PizKU%+U?(1!M6zTM z037>bfs6s2Z&ig-aVnq6D7b<3Hg=95wbjFP#rU*1C$VOF;JG)!tOq zHaCRu{RsSC*9ZpUPUNi|!yooZL7WA_)SAI@1C`%Wn{`on-x`(%Ri40fJR=mu5{Ldw zIbt?qI!uHF*_&lcpzOrJpk}{?7}F`3EginE7b6`?Pf>=uxd{7pzKWe=v%iLBPm!W< zxrL296B9(7f8s9jGg>Kyk_hWg8MaUd6u**bc{TJkVp=4}ifBEBuF}rfX>o%|blWM3 znrJSKs{!-F&*#UAmZ2W&1o8>K*bp^gHEg0-ln%o#Rg&y4+CJEkEK6N&L?~R1n3>s6 zs@af2c?>Ta0QLGtO>WY%^dz$(lCVNTg?L)?FAv4zKlALUr&c9ixgOCs5gz07V)c$@ z6Nw4slz~=_)?3~a<$k}jn8GJMM)@4ev z{+dl;_+`ZYaJV&Ky{Lk4mhD8MSI{YdjDdMIwA>aCguhJnrV(24X+^rtQoy7^JO~B8zMYJg z;^ey5q>gODV(^9Iefs_JCN@`w&!8(tXf1rp-_mI_xdwdpaD5g(n7qE5LFihGktnyo zAw37ZPRIL5@g^i1C>7U*8}DkcO~kvs0>Bz#;$t1I#(Y17_Yr!s(bVaZdL0U)6&zbX zuKb<(JM+Jg`~Sjf{}W#`YQ%#DI@dlw(XRdN4-8<*&$&r_a~O-T3|Mi0Xc>MqGP5(V zc{TFCaonx4d7c1jwE1~Wu6-f`x-D=Pu5BcTaj(Eu4-Z*Wv)6c-@#-E5-`Tmpq@|(R zyUPw_{d5-mSNgEg&@wfe@5n7IrJ%S zDwFbpFw7p_=1T0&j!{>Plvj}0E*+j}|#)ZU=avY!XzHN0**|Pzk zr;2!VivbSpUSf%-JWMMagMhgZob_a(*VcjW^55esqRRHWS7kE7G(NPyKyV^3(fCrw zP+HnpI!P26JAlix;!o8q6&%^cmz7C87I0d=YUv0W!O%M-bq_aDNtA=Z`oUR)xDUt7 zW`|V@Yys4XBDbVUwzVeN-P*agDW?RY1k-28*ifM|82wOCxJ`X5`c zNnyk?5;vd9<{K6U>UT_z@_npI_wMO!L9o;p=gBXrGE^-bs>uXQVjl&Qjn;8~H}zQO zUQja6*#)k|qnVfzC$mJ&(68?|&>OCCTque3O6z)GXJc&k79n0hxr5Fw%3nI=sFM%! zh42lhf)mBJn8Jcg0= zc|M9etK~QDNtqhO0!JTGyw}}(V(Kv!AugD+Ws;xKMH3?5_x>#NwJZL!8b9O}a;Y(GkHdQaMFCl1T_VGX%J-B)wM!6kcw6ji?b5Mg+V5kw`|#c5_c z@tbiq(x1kgF9#Q@Q&TcOoeqSDm^;IZ42BD+1M*)fH4+=PeM53syk|#_wT|+d*rj-U zcf7{tly-ZaXAi_cTJ$E4(&;u`WCL!crGgULAfX<~x3~q#a~!ERyoc5Sb?QS@FAhOQ z-gfhCn#W`-DlZ?y@!zDO2*k&-Fh z<(pnHrz@|jZDpZZe5vygq~CVA){UV1wgJ(aXL0SNK;Xava|gt8fpj+XG8Ocgh5c`7 z*sX2AEvN5nN4`J{(2VzysJSN7dJ#santFnYJk?Ae35wGtLpHN^ZT`ttK$&GWP7h79 z{s2BMu4o;(hyU#~Da7QAkZ$XwTjqEH3XBX@7;ow-AmK;m zHd?tHg96_|2_&5*lM?P$FuHKZzv6i6A$ zt?dScY#7=t)YBsL>&T@#&jOf2UR19&kw+_NO`E&5$gX>wUBmB>cmgNMtY`yXt-_ad z{F>V+#cpYPXIIe82yb(Y5NOUJAe0ZKN18P&Kdh5S88^&fP$M`IZQz5*to>pv^<%8@ zNad^j!_|=60kxDcZ}bx z%H8qf_Tb-xKrqnwcw09TZLDUW)A@sc&Df_>bu%Y~<8x_VH$|Q=RM6{yfk-KY$}QC3 zOZc~cs+%RVFN5KmJ+_|`?jDn9O=fM`d7l?ho8Dhiwp|?kO}l}A;qVDWCyiN*om7Vm zWDAM^O6I3hn+Ui1hne68qS6-D3w%{G7u}al9S7{;kK#G2xYiN2!G0F8d1cW#6Sj9o z8zGN)L0DM41p9vpCEO3BAZ3l%>P zLYk&?Zvi>Su%o)ojIVFhI#5B1#EbBy^VCN$a9r6S?b4J?($AXnMa)TBYpDCv+L5bU zGp@aaOOAp+!{JL3ZYhgbe-rrp=tUb-%e!^GlXkv|<#%T)H0wXXa=cVe?92O9Pm*?d zW2544&H8-)@Lk2%n!@n$=*2R3LcH|itvoY@YdX;(pNQ8GY2i@>z0|f3JpYnc#c$Wu z4-Q82>Lztu`0>E1MR;>-s`^Uu+rt^V-L&4{wD;_iP`4~#w& zl8hvy%?M(x3jC=)VBgm>smF8vH>~M#ktM#8_RqTccVM(nfLH)xOB)p6ms|Zy%G-od zOI7lh`s!Lx_#*9ki7c(cTpi=Q@Q%Q)WHXIk39>6oxjd8JM}zQ(wb*1|>M3_225ndV zQ>t^}RsRRsQSU$!Ns4ylOR4!E_-lqysWe$XD6wuC3<9b7POocZ=-&WUP6!8mgf_BE z_DnCzqIe$`yzgWVWZK{pA60y=T}Np<33Fo>LY&nD!dt_o!u8z%`kivaafO)kFY)PC z=^ilN@7~yC!%`bS4qwC*xNlqs`5N|Sd1{$%pij4-q+M*L-d)D6^#i5+ur-Ygqpi=l z`d{5jaC7y%h)Ep!hKO#K)d25%VbE9wo|^^r^SDOs4Je56e6n;V?NN;^r}gYrosZNH zT3R+x$n?o2a%aM_N?K}`P1emfS_Ig*U4Ivaa-C7_s>d(#bMmFNojjdnFgl)2)4TfW)G`WRRZSNa|WD3{ZElAvM z^?FCWpf`|=N7j@I{Rk3+qiTy2G%86K zsuBp2e@Rdg`)r==3NwdpGb#I=J-B9ulGp##vL}+6T!|Yipcqr^mWHI&bNh}}SaJmo zy2EdPUP4gl0!yOsyBd-Tb61gmcshr^b*NBac77N7@bQh*{k?x$rxou@-1(LLh-bWX z>Teo1^PIZQ2X0ahN~SE>*Gli=5JGGdZfC%TGtd6mu2|Tuxe`REXMuA<2SDDB0UURx zZ%+{`K$Oi3T0yYa4PSJoVNj+`59L{O_eJHkPP+r~%dJF_l(p>WGV_jbGtj-@6qLireMxd*ECGJ$BLCt<}<&%{hWgK}lh9Czq|gjQi!4 ze_6#EVnl0gPMH0~5ib$*^5o@dW4`<^@eR#se}hB>Y_cSGtI|j^C@nP+_p>%1q90n( zqs6?WBWP4~AQD6gxU-jTaMS4E4+uDW!h+|Z%$BclZT(mi<-76o;ne_x`V3RO#+DV6 zldiHnReivstJU^Z`|WlqgLQdaL&9fflS$62SOeD5_t8wx|HTDhJ1tl#>!tpSh@t3Y zH=e2Ot%eUBosUviyxYNw6r~b7sX5qQI?uN0kUd$rY5X5+0xfj3f>-9^(?|md>V#{5 z%ye=eeoB=N1fDh5urR4HBOr1(u>5T34l`p~jt97awNxUv+@GipC>CSnrFkR)8N7cQOzHkxCU#HN#SpP~(5Hrjqn{tuA#*r9(rJ>*oJS zJ~MTcUa|OmxU6bx{+uR@!HTe0A|TN^coo`J%(>(Y9cyG5e3v<{#CK}NShK)TNjq8q%g`Uu5uUoJzR=(b_VqREVF@6Cn}vgG}qc$WBB zMh2fcHu(p#8Ei66FvYWR^p-X4CqH?x!fSAqI;=orlcVEi_i@uS>Gb^`H~a(nh+2!x zL#yNp0|$FC<5B~2x}3mVkp8wjhc{5lLk&d>@g%$px5gEWkz$-%0wXOQu4H;5ZS_ffz1|`qQj=7H17SpVXXzVrgq1xdCeuMg}2DJ&`N*Ya@ zAH>Wo`$&U0US=*jB#`k<$F0AL&#|&yHj3SwK&tPy%u$ zs&?k6c4w$Iy9!u#e@YtcPgbH){fsA@P=P=u?Pw&+EJ0>hpzb@i)HYq2F4)iUMzJl) z?P?h6y1@_n7PuB?dFoC1^J77ZkyTsUh1A3a^rvmVy>~L#S*S%}maPVr3qM%uk#%-V zJmjUI{xzy=$yYAqHxc1{%SG_S5c#~B3$`t{Dj_`B#xQY9qKEf90YRHjujU2LUkj>9 zhFxp=R{LC-*rm-!jS;JD6$s6lnmzg}dR630)u)ReYQA-X_8;Aj^^o3NS{Z|BTdN)X zUN!~t$35n3`P(a+@(e!!zmZ|E-IIHy@37&{on^EcXd@btWg|!3tqSI4p-iHb2Z$F- z%;A1O5rlpVNaP*BR!llMDuSJ}Tuypk)CHB*(YBXZHwG23_`u?tX*aJ9H1kX{g}qby z+&nZn6LaWvmm%xdo~ykZF92*c6Kq74OQJo4b}#HYvvEB<(=j-gHO#}v0Pe8ZqJ6x> zN(y$1*6{nK_1XU27>zv-e#^5#yK_lXsimJ&3;Aw%k-ZrHCj)#Mk-w{CgVU-)A<;{8 zBe-bRDp}&+kgbticR4>ubg!9VvbK!gTzKs{7C3G1J7ha|xlc#mji$Ed;;}dK8YeV)qBf zVmNsXHJLpf7NNec-?k?mXO~F#n4SsdO3A_T4w;xaz^qJqZ;*P@=jl3^F9`klv4)~! z?uT1`kl@@veLaFw^E7sSe&o6fd+k-ub@j#X*3YA4<^?2c=2j7oH@8Iw6QC#H|Ba~` zf=4bMjp%*e3!K|P+*f_4pYP@k)$+KPVt5J)_Z1&$f<#|%N%9ad$igU>)h|~&7&Tu@=B!a_ot9nwEwUvF8%!utF-fZ4I*?l#a$+Cw8Z@566( zSbyD#4lhtjYg{~tyVyb25*oM8M_RA_arrIK6=3M+#<8pVJt9m~2)J<9uJit|6e-qr zfO?=@V`RfjPa|o3L?e@xF2g!x4^NifCyGaJFXj{s2T@i#EtA@%xqR7r8b{lW{vIQz z>A4OC>A@EI#?qwRC06}ycj-nTO7~{A18~9VBFfK3HUyqvPh5YL5%4j|%l9=Z$wAYK z4mC1}6*(*K&rp&;X>-V$j2Lp{2->S-{4l|%SO|$~_J-=UT1l}C6^W5)1Kpq#E6qK#SC@V>7>-Pha(wr+O{0-J^#Ha6N z&ufopdiQ9LyHY4Dt1VtYA7LBZ=bIj?=zwb=M`jIrW^qnm%g_MtPE}OGuQ=%!d4Z_O zS_7T77?I<$1Z5^mR`<<;^xSr@j-)NEMSBMkkOqN6+X)$Lm3mGIhVEC?&g zP~=zhq1;3^KoH;AkVscZS*eIsA?ZegxoMe&vd_LZ~D=xr82#@;gw4kH*^!cI)Aqduie%mhxl45(@HPVa4r!#jP0QguuNHd0|>HgdUk^M}3 z_xR%gRx@@%1H?vLjXyeVlM9_#J>F7F2JP6kR9gUrVy|v6oQ5ooxD2sbjH6A*6Wofj z84ay&Vu61v+iz*zDS#z_o7RzS!8^6_%fo=Or~t#?}fLcGsWESL$d6eCj2a398M`ZTbv~OHKub-~&xvVjd?6l6w}xKo&Jq$TQ=e+aw={ zA&7L~K|qb?+VW`pB(lVX5AicKS!Og4@QYXmIJl?{p`M($u6HOch;lbiI9m)aY;x}Y{4xh5*>Qu-O%%gEWD!CRd+#}9|d#z+<(%i1q70!8+lA# zjuK?~fqn!^deS=FKd(euhO>>dwh{Jm^`#-4j~5O@{x2od_$+cLy6*ZNRi!1TlTEND zP+(NL`FjwwW@+7Z-@kdx?JbpL=i;K{qB+D;9V?ZY_!SoaQ#1KMn|hZF{$#GrdK%8o zSIT^|iUj|y9gv$`BggV>7NA4F<2#Zv*Zk$(2`x=;M=uu5RhDSXK<;zYvC}UfRg}EM zpSAXLhDwnBjWMws(>~lrcHu;uuFycQ*S`3)Z2ZMJ_eYtL3Fn6?0Acz|>mxODgY>JH z*~#d_s}!l1&uda45)pVJkX%!z>FX*J1JPo)b7Eg_W`b|IV!&>&|WKO>!UZ zo_fGD(hc>O&+w1jP6*k+L{L$eC%dTw&64LC`&{4&*xu@ZT-O45?Y2vH-XDhKqn?{0 zw_lPINDZi5&@MNM=iG>TYmk^T%EmYMYm7P)UAJk0A;<5u>LOaYYe=(eJDryPQ#dch zj{wFtx2R+}`T0Ag+HHdd*Pi4921IEp$+|F+0QOcB*GlA~8x1I&D1Qho`7j`;(lgd{ zrH@$WdBad8rv|C$NPg3INFYA|({t^H#ZS@(y@e&ge<^egvH70q%SkCVR$keZ>9KsY zH(b*VQGl!~gwA=#sPQ1x zXmXmgf->un4v6-?;s1G}tljGNQ~zj(*JM+iBdwDLd(W|}w-Zs;ryI*0)syw)KtT6N zJEs|iL59Y#@RjQOM&L>^@x1sE4s+*ybKjqsq6|Vv-C&5s(HT_E*^d088<%KA@W?R2 z4=z9I55#R@xiLA2ei|&6?>QHJ5nZ6%%oNv_Ub}pe3})x^+ zR2E{OZlwX{>PNcQ`hr}>>7#vZB{L`;Wf+X%%kuDt2#Cpj=}XdoAfTl#`O64?r{O2G zS5V0sq4@b@!>&9>fF<{=>T!3*!T=9&`b(YQ8Z%cy0f>3|{RL$IO(4Lwn6^JrR}8lA zUqw&cO3Lb))_3u=qHah8zFwplI6qzi!p=vI!VX&Yxm&9}zT!8ip+7%dvfl04W){Nl zlUg@kNb~Y$iP=saOCdv5T+(x+{yMz=_~7;j%$jEmrgjgeY>(y3r-5p#W%-w*1yELe z+FKQH4yYWuixk+?NadoZZpJifYgIp7lei?`ZR^CTz+H<-J(O-=3(t-9`*2m_yX)*W zwh+3u_85|U))LBr*i$bMTnbvW)6)4HmO%;IxLlbReWmlhf(&Q`f9-wyU097VFBpV_m8=Qc1hcuAslY%husI%!NQ>Uq4)Z~H4w8OZpI!{hQoLd#JzF%8FD3?8dp(WP0LRJh7@s29u=t0Z zvR2B;(Nv}T;Ooqd;tRzc%JV)bB#H%dk=FUQlrwWaSFjLRiHm--@C6tkQVJ};eJz>O zi6j$ViGXhMJ;dh1QGD^JNS6jwW$?w}EG9ltWyU|l^n;Q(=+gg;JmU!3ue|jpv?^M% z21}{|yzg{F+@oui=X%CWrN~`HPAx&M5PFdB3xH2QIDs3WF<)*b^q7ve?0s9+)iT*J z71b}RF3t^%xW7Ljk~C;inR0!;bGc`$bHh_$_$~9h%q+#x8I0!P@f%>w)${ z+cvbGSC-RHBV~;x*Pot7*i`M)`1Q}%f_Ae_y~`i37uUeymBo5*ze?MGkfl|`2&cnm z8S>^g7KVnLqz zy<6Sg{e|Lmz?NxG`$C=}$QZ8QObX;8HO{(3T+FOkQH!FeiDF9`^_|f^5#LaYa~TT? zSv;L4SSHh-35|%4zzFp`oVHRA=N@8~CTG1vc?N(Ec(f)m_rg1b8` z?``eY*6z3e?$29wZ*}*PKD{2ju6bA$VtI%lY&aNBA#}FLoka_}kv6j8pACJWYFB#% zcnW23mq{#Zes&#VB>k7$KhC~7JCJ@e&NBGJC!jb{uoeT-qAPFK&+ctDeHAR1!w)AD zraI7w`02;PW$Q~x;2Gxa$G@eGm(BkvSn55-rMvMRMg4(;?7>WN00VB|qBp!YuUFPz zgKdjm#>;=hf_zJJ%Ha}a-Z430yVGnD!!ZS%J_6M&CmYmVi2jsc&W3*7*OfxtxDElG zh_iou-f$;g=+HM*o0)nUpBo2W1d0D` zkn^ipFrf8wFEPQ~-HIUtw+V8r;k$kB-39^?b+kYtV~_VxKAiK@^a``=L7#TSW>un~ z7F6=q)4>aIYsZa?Mt??|Qh7P1&>tS~yMZlwhmge-{IbcENOxEhmf#_~>sVkol5*x_ z;#V>G#5@FvqI_85j)PR;Oefh-+xWuy;)ZxbUzOGp>n;=M7kM5Sh$ff5<*ncN8~ylF zP|6$k7yuvkXzp{DgkA5dF>%EL?_3?X*U?Hy``fg`EZj~)XKXIcMbp9wP z0lb33yb<%L4BzmLocDAx&vKN&>?mH-=eSpwmmw{+BS-I};=*dwL-0}|S0*#ump3bc z>7=71MJ(cwnB#v=TL!B>?`c-Nmt_jXQK(O1cmxrG#v4;1naW5D#WUrTLNo^$@|Gvg zH2!{b5%lQ~r+hvx$aq}FAHC70h`uz_?F#naEdLBV`VqJS%61nMe_i8|psvZt9|*dSQz9O=YUr;i?AOkeUM1`#Hcn z8A;%jJ{7I<<4b5wSDXH(&7Ks8pCj^oyhQA~S4%Fi5LUGYH;E>iX6@Z*9f)0Ia<67V zurYu8)_7;K;bvstviFR5B~W`M!0&jI^Re~GD@6h3fS-ZjZoeADb7 zfbPb-SCUfO4;>YxJSnUS_crDywv1qt2b6`%4|UhS#}zOufCDP(s*u%5X!OnegDK}p zkLzv_y7p+)i~@LG zqRTG`#){pHZ<~yvbfV8?p=(K{HdQ6`4eo5 zeW>{8R{sk#(lBv23a%ZF#ZSn?6HOU{k-hRac2{H5M?KtW(G4-3-O0I&R)C1GESJ%; zs)sd)=q6P~OdCSP0lm+$4>h!*WXDA)(zoll!|Bu-giC)~{)oq%pZRY2lM025y_Cek z&K3WTb+n?&w{EatLj2B)-(@?caip5{z@a53fCb7BHqtU+gc3u(fP%Ti&{Yjyq8k~! zvtbsiqZp9eY7TPaQ<~Vk9zBNDm?*US<3B4XdE|<1{K|?p8-g^G`xw&(rlI5K`JVY({lbPJR!+;{KMi9;H-<|HSB@{-Sq%^YPB3UiTeWlv2aAH$` z;=g0|^Lk?!={1BlEsLj`!-oqZ3sKfHYLhrq!f-|vzu_xK4;m!rz>=!;hDOvs61q?ysep0ndao1l;2L zJktZb95V{NuI$H#6^QxEW(d2ifz(ISdY?mahR@ z?G_un!+4&^ja$Oy7^<7MfuqS;EsMPQN<+825787LnHeLo3`B;&zm^wgw}D?dUvD8gXa@~XAn+odmVYwA3hiwYl828ej5sYW*{^j5e zJC|(xyu|Vq4|o9ZaVrX9{E5)I#S=>D^39QfUY6wI8J%dndrk7gj z_P9-?&C*c4!M{)kb;QcIaZ#-t@gr?vF>(?)_lYpG3Wj z12o>?y8Oi6vg>tDUq2}H_=1kKCDez~R%U*16j{C|7EvV|cpv0TNI^>p`VK2Q{t0ws zZKSsV+U47{7?v5BwUzaK{7DohoShJp_%KWe*5{)E2KV=om9J9_E0$W^jtUFuO*Q^P z9u;3sSlV@UYrChd(e>U|9uBN$|MGSd2l!o&h#3hFXgVsrZsezubr>5M2tkz{joeBP z9T*@^O1kAso@i6>$(0`yBkv8)0l99h$1aO?g4LFz=Hk#d`N9-tJbhnhjJdasx;$Ug z&s{-xl8wl#na)KwUn$A2B#Nm&RS^+>n`>TBj^4*HTF(U5lIz(sJe+|O%}8iBYV{zcPX z-EFnQYzoGvp9=9j2WN^3WsMz$YB;&=w7K1kTK%hNO^WCCVZ9K8yR9^;b&PX;B8_rL zy&&ib1HH}<1ys_)=_85-lGjdM1zRLY@HUAa;xnFhzB6WonmzOBiYYL^^1&7X&#Ww! zp$-NQ*L*q|ZuL^T_ey)njCN>%QbgWH-wHaDCXX8cfNRBF>t~*QM801;%bESlSX)%x ztsGSB8biiJ>rXrQz1K)q;^7Xy@2msn7)Xd|^c#BXSad{oF9xqR^%VD)1?^rk-iFSa zoA016iQNQo9*mjj(PUv%qXe?-!MhXTkU|=!ff55zoSpO^1BaL^e}x~{wUJeON;F`z zIrSH~f-(HD4Pq@PBo-W8E!P*~7^OpbQDEt5fwMc9bR??L{nWS+tlA9ZvQ*k&FBUB< zW}0&!0uBw*@rAj^-wIdJgC*e$C#5UVNeKcjs(kzzx{$E~zraOhaM$jijD&_b)YU9tB!Y2eU zqyB2mXAvQmgMT7ds6(kndUBqK?$4;hz%Wz8(&V1gn|pDTk%aq{Yw2q4`~Nls-*5xo zY!N^G~8aVo&=(6Kd znYq!ySUYCYK6pmER_AT!a#!DNlzTM~F_Z;%dq^|rvEan#5VeJ!fX})?cfk#l1Gi@c zUfoNAW(y(4@LrgU|K~#wcP-M8PkuGvmB0Jb^5!BE#4>z+G29~j$`lDe{k!&dfjMXP z6%LU4th9Un7vzRP_T~#bK4&h-6hTSXKfmuoL4f+4w&~W}PU0GYeei`3Gu1a{!-jr4 zgAG>fmTW?q6xWCW}apXU&_Iw|i_L zxshbqc6{t_fJR z-+=w&lfj-pJXGRA5so*_U=QdAu61x|)eR=tz%()H5MoW<(-FNYEBRPZl+zbH(>;xO#q6}uKO3k?sJT5{lg}kw{x4?Rs{xYA7?=j15 znED>}S&sxH4qP2w z#pU_*pgmx7IrYN(r0cW784<|8Nl(Mq4?JK}z9R_u=#73j8s%2qft{su+#K?qG`iG^ zp5K{kVz2cZ;eAHm6Hh1q)}?LC@?wP*&29iz)kb-8_F)yH@gGwyTE)+UV__RV5^6UV zV+>{c!TR$Fo~Eh2${C=u?y65Wr&r}1qH)|3W60X$wnFRuYZ{y9Aiw)C5e31*bi0Tx z{>B3KFVKuX(TlH3pS3u^VS!bLK%Q<`WDE>Hl^cv)B*sf z^8yT4SQSCmZSg7$I>I4q!Q5ILk)wF%f_>6i@A-B!4ggnTJ?XAVS`Om4qvikT5kjCNZE%Z(`A*su zBo?dk5zRWF$il-=3UQU~s`-aS!PuF%2R6#0WAJMDa?}b~eIeqgS!MfQPi?D9YY6ED zMK@M%2TEBkqC5}0>VzPYaQOQimzp}JdzgtxhM)OiU?hzud;a+#`9!IiT^PeHmyV;P zM5r3{upW>Q3gtal1TOr2j|K3<M@B1b5BLm!bNk3Oms=rd$}C!hjL4ncxnZy z($eT!>eU70gh0EJq$mJEMB+JbF3!0c*UHc4>lUHmPFa;Zxk0<=-q6DYxpxB-lIgf9 z8yAI%4PkVn?86~1N=@~+XcHFmEF*v^|DO$Yu0*f*$%zhNj>U+=Kc3=#%m@&kp`eXF z%{>32iegp<_ssU>DnOi$X4&neW?_8p4X0GEyocJikvG@%VLph8f=v79V2Vz^5D>h$ zgw9rcCU)GiOO|j&R?AF|Wr~2FS|jy#j)KG!-_JbRU&g;DPORlPvt-Fp)ZS=|;Ii({ z)~Ed;b;?WhoA!gDWck=@^`iInS>{_gxYcQ;I)R8t;60AjhQu8{T=nUi$ASDTHaUB9 zLBoaLBy2N1Yx(u|LTj5}sPDu2$k6Tdx+qTM(1Wq%Dx9!`iktxtu_BM|?oR@-cvJ(S z>9EG6N^#7xkp#U}l!0+8gsJSOcT_R?%JE#B7kL6bUatSopu*GzGyZ~yoML73{tH2q z5a7*C-TGLnzPBC|bb^|MznzB+nYY+aUrz7wIhKu4X2eR|fpcV<1yX5<&BN7x`DQj( znPBI9GQe}98cpLHZS;?kWkmfk3Q8b1HIncuo~mvJV+G*<&)i-B4Fg~$ULCgXR92|86jFV zhmbK>;?ur`w4jP;z%$89e>h1W3et`>>s)(O)KH4evv=6HK>06m+Ds?n6&>{DrQ7KX zi;ws9DTAbcntpZMU;qU1+2?E1&hD_(c*mj8LLG)|w+9*U#ytH-@YFifzdTcEGruuOle-TTNI0*k@d#a$ZcuRtjP4Trtg2D+ZFqbipdymS5EcmwU15x znKen7#oz;71wYWmklaP*8tS9Y@3EkT(8K}0K{2wyC>6pFPX7(-Nf=&8GH$BIHBv<| zLmmY?=Eh?FIl@-yU^#re()D~@s*be}gU{RS0rDv$bS*5q)zWh0K@HVwY0tV*Q%+p4ejd6OGInz~;d!z@9e2WoTnF~#FUgMB(i z)h_+7VQ~!?k{5hXp)W$V5wjI~4}VoxIgX^AA81maYfxaR&(uM)#_d;#!(lG0aGBmx zx%9h{3nx_H#mm<2b{eqWzv<9iC6<$sG3TLF<+q3DK{5Bb!yL3#mc!n;Scu+(N_~7V z!Z)qX4}u1m{18!y##&XXY@Kz#tMxh_MPev}x-6)2U-mP1RtR2l2B$iFFGg|2{1E!l zlX{cbX_{O3hrH2of$;58-(6b_kIdcM4Dn7dcG>q0=e?^H+V*HD&)i5)ZQ>o7>T{j%R z%|mr_Tj+15@DuV%!}%-{4GfR9zIa({X#T32O>6bKiIik3;h!Fya%=eYF%VdPD}QOu=q@cz7t`$y`bNV`Q9E_(sTH5 zpujQUZWJ>o+h+R9@Z*yE{l5+2x4(3ToAo#Pa*q*g|J;Deqpm|*Jg&cGC#tHbllN41 zem}2n0CF9ysWuj$B@aU&?(`(W3Ti5p`Ea6UeYhxngLfDz(S^IxcY_5LLHC}>Kl4Ms z^Af@`k~0ZU9k>B34;1+KY0gKD7wc-ryITJ*tnk0lL(f|_841)}=)Xb3%H}LmnI-)A ze}M)|a*D*c0f}4P%GO_LbJ(eVLk*_Si4M>_77(PjUa$rDH?8L~GHrbzyhXLXpp?+_ zsfa3D+#fZ?AG^@+&HPx*gTR){2=Bb-O!rg_MmJp)FqwDve`+hNJ2b9?}M+z+cRN`R5aMaJmj_SA-@;y;XuoF0mfg*V9E`_MRzI zp%>R%L}nN4WNp^s?Nwy+Mk{i52rsW5cn-Lw5~p2g*y}N;00h8oaG6+#Vug#eeYTVP zaN6Sa2krv8Zlcc43!g@l%cjkl-BWqchxCd@0dCOrG2c^e4w;d`# zk=wI@<`yN%CtnRrsiKpFQ74VGLnT^BWI!9Ck&f z!V!#5Ok)ETnB#UVR0B&d-WOVuX-<}{RPs}QWJ82z)`zFN7G$*jg^aYudAVjLc; zl;zO~t^APi)~rGNEx@FS0!Y73TuI0%!=sU$qBj;Vs6R#GS73e{s=Hm!%f3j_pPl> zQ-)UUQ0leT&1!f$idaB%hA=F}kW1asCQiVKE3)(e12yhLXYr!3?jTyIrP1;lm(cEw zbvU5;39F^{LrgRJNS*J6iPO_14i-nnV2-imm8#iOD{{UaDG)gGi3Y#aUkK#dmUyHs z1~B#_m-5cF%J%nIp0({)_9b#$uS_B<=k$$Fh^vxYnS9hg*`Ra|zVd$Rc{cNAVHKL) z`e_UFyt6do3W%VNJyo>f)xHWE)M$tnTt3RRGy(-tlSaDge?+}g%OR(uvjzc zbQwn(YRiC){(`ix;t=sZ&VkF-5Wj11R$8O{Ds9O193|eOoa`;pr!nH?M^t+OAM8Oc z{6&kz(_w9()8io{I>9_)pc*Myt2={_gR1(xc)%+2(H0S$p?S(kDD)#gsN(z1s+8X6 zsnsQ6j;x6Nd$ql#+a1NwdbqsX=Kkj~({>%cs5K-?9FBKG znL}=)d|ZA@h3rt>DQr*mCma<0kiYmeYCBg$jyr-72qDD#v^4?L9e7UuK0;!s&f)A~ zfO-Q=%9D5B$6j|tin2%P^Y_dI4}9mG#GU)!oF_UhTm?0$)-M4(lvioL$kx;ZV#J~o zhXErh=q`e=k*D}*uznP-oo~2kXrpci4xD6JhCK<=O|?`jqOv{WcQtrmZHwUkULfR&a5mH3v18W!;yWQ{~qhnk`FEazr_tgRN8)=){QvS$~QBmjRzva!E zOn=rR)QaZer2aDCLZ5u;?_c!)N75mO;lk-q;T;b+ZEYIy_x8N&ldn=@l-b1Rq>Nd}Sn9ZBi)Ro1~SX(@0CF{8Piv z#Y!pgcY?^-^zG`ZTI?~8TmCn7m6#m?{^SMc#gN`DToFj-3*=J;Br?b#4QULays3~c z(bRbC8`jWH37MOZApS>mrGI*vfy61gaoY|U6pe}HV|0MryA!-nE(9#*2_qa9DwHz% zIY)!rOazc09v468TRW|TLD*KK<9ie+BdSuj^j9n!~ zZ)S-d|f`7h)|6C%8wkxi0u1UH|b5Ek;}}L67@*E5in@(7Ip=|crw1Q=zEjF(tqL82&6^N1pj0p2%|QI zQ>4KUH3PwHDU4uH7;c0ZmV^V;J<(Zvak7(QthT2=Xlpi@`IIReE8sVjiwqsv&1{d*M?Y;HVH=d8 zccY4%85jLmZ85Hh+_mpoMJm7Evi+&KU0(%Wn6_LMdhlGR<2lt2tppkNJUfIGDX z`W#UjRr*ED2&zmqi`L_KWvZ(eXHMAGZC8JtJv)^Re6ajwsK#>g)G?el7j$L!M83Pw zW6xCbDk5&>liOe6d8cNmP$wQ@fA}%xLta#ca@1tXx8ftlXD1kbKk2rO(X?31=^ zC)pLL7@vj+A(-QL4CJ|I6+ntvx&~ewj;lFhGl>DoW&zGD)NR1VpEk6*xUG50(cA7S zY`*!g9RJeEacnlVXb70g8y~;dO*S0t>88nfa&7#8 zmV$04RF~4-&w|L|kRTrICRTmWJwVdHHlZVL2rZAA*~VX}PHB@|i~^7}&Fwps1L82Z zDHxr9V6L1g1pixcE_H8J z**frbQ@8K5{D#z0;4qB5T~rQF!uz%HFjr8LIr-S_MkwXjauQ)h8*^6!(i86z7F-F* zI&B^m;dL}X-_!Hj6L(o{5aK&0wNknASvG2XMFoBIkNP!9c=yqYK5EvI0iU%1VS3C7 z5bg_PZ98Z6;4-)IB7)1ruJ5T$Dk!E|gUU%?^3JvswLUpW+PIANE$i(Vl$qpy!?d2ikG!UzzZ7OPT3YCs**w3IBK zV?@u-%B%+^9ccKMy#%(!@}TBlf)!2fILlFJJbCa}ks0qiRz- zx{UmLS*J_n!&#+VnZJ!9*T`u3;{IRAiBdt=j@;n*~E#z^AkIheim z7Qs2o$wjFfsnf@@BcJ_9f-RGlw92tuZ_Ton zVI!bz4#_;zyaUpo%F%CA9>vFP&-8;3Gxb`-0i~BKD!)uhK0X&$Q+=k1e@Z072f8BG zLYejU$4d@AO2mV)rr*v8<9mJ@ICv?Z@>`pM(^ zarw5cBXO@Ldnz{w3M-s&5NgSZb?9X=EelB_NB_wEc@hP=I`k$Sif6NQBF<%Drcp^5 z05}eTYYzfCUHVZgvNB-f_3&nNZ~c=j{V%hxQIUc`gde~N&XlTLAw*=Ig~ApmoHVqL z^)FTQ3d-pT5l8y!;oYgPf;CPD@=78=OI_mBKoc73Pcvca>h)36HrxA;asnr$z4ep{ z$rt0#-Zll(8aaYgT1@m66H)@sMDJJT;ernoddT8S$A%ug?-Ylw#w)3QJ+|w0ao6Ci z{^PAwE1vejpm3i_lsB6rA$hrQ=aOQo26gLqpq&Ljw^>_Hjx&-$P~f!Cz1}A?d-HoO zeBGJ_L+BBXTIU^4M=UCRGHk8I=8o=0A9R8W79t}PzNoK98>`;^dCndWt2^}R#CV?^ z_QO+@)fnBgK#@Ms@5b%^2tC5OqsT9n+!#dywHY<|AT~b()rSDTx%E`vVFfpAGQC7n zyC3M|5t;|R`kr?#fUjzb(|XoA%zI+od5$PkSbU2BEt(*E=14~s_l_7E1JBX#Sev&S<6wi|kHULxUG zQ8nY?;;<8~zs8Ks5vkcYA;5mc^Gvw2<|;A=I_#W8Oq=Jjc7h77)DND zXFP$49Q5u;Iv>90cdKsE*V+lSJ0ehHd9imSagz5M6jnu%3LDSbXm5rg=l8?h4|? zj`|X=q#|D_j>^S<_HHHD4MKi)%!55S6I-&i{k?G<)cBtzi#KIKo(UOe;B#4|W4I=H zHb`s?iao-Oko{WZNkOcpIBG{F3}&(&JuI~3EEj9CD! z{x2_pRt1wrv7R&xC|&q29}pXXj+@13Hd{ z9~P0mx6>-3r@N#u?DH-m%s9A%&18zif=l1rb^m_En0j^hH*e!`K?l!oPrYLyaXZ36 z4_ntfmYQ;%E4GL+H+9iH*Ht^#DEW}gKnTZ_Qjd&JQD%9S4Z@tq zAs7CLmKzk!Bv$rtxA$tQ=5>(M>UOFMkck{mQ{S2!cN3|s<%iLwE zA4)JI=`b78kJ10x%wo59Z!)AnMQTit{Kw3?w6i&N=16qCH+6I<@n^47jRCRA&%qHId2)ZmRDK>$2Mq)i z-ex@fI^y3dB%Ayr4R9vI;^+)(59698ys;RyjsCtBKxn>q72{{}>9M;jYKGkaSY}1C zFiN#ubVt;imm5tG?l3GG!93RckdUZuC?0`t?VVro!&(N0uog=!417oWE3gP70r6d@ zI2jz0O|hsSCByu{3|%2^*$#qb==YoH>*NX0EFk|;Y&sKW4iIx2cyR{?X_B9wAYh$# zaS<(R31oHI>%B!Q3Z!CgAJ$+wJw4JWp9ug0)Q1&pbiPiDAki(nVOX15Zv4{A0zP9T8hugamHl1s@W_VL8%@Wjh5HuP_9dTR*k+JVwD6NB$XR`J34{WZW(iy z7^ASJ1ADYm=O$}~?yPu=||$ccOO9WWA*C^*?`LT<{+R+6U zb7-pZJHLfsKrKE50}qmK-6crX;LVNvwjQ&X;h5x+-2p^2h98UGyKlpNH6u`U8RD5? z?0>yK7~Q9gaB^5`uUge$b013{m)F8{KSmqA_bl3~Su}CScrI zYddPPFG}Ajqbec5#)8j6w^Pkwhn5Br=pEPwrxe0j22=A((YvWaqHmXuKz7)A;O$X+ z4>T}3@$<%cCRPpreS$rrHDc?S{x*ebf+Dhm_FjOpGo1b_cg5| z!svgu4KdL@Y~L>YTxK^?pgzOeN_=W49mAveh zVkG^`n1+ZHSFc*sfXln_U`thYIVK9UxidGrBq#f{SZ9^Q*zNoK3U_Gb8NB>NPsJy> zYG_8QGwX{8(z$4Oy^(Jz;7!9>`964mpS2A;d*+Vv1E#q5<-=Hvl2@n2K>kILh7`r0 z(1onH2w_lb&f7;nq8I);KEy=b66of0E3P&vCoka}S!NWf-0A*a3$J__^AiQjo<+Fr zQSRc4*0K`mc5C@Bog6|WH$+W9P66$Hje~!?p2YH(zDHC= zvAlV0?ZTt*S8?1jM0NYiXJ7HHEx&TG?rxJ+{~r?qO~5e)%x>3fmb(86hR?Nf{kQ9{ ze@zq3w;JXyuZ+r1_YVLo;y_De}+GSqYwiQgrsfJAzNv=C6U zAr|8pX`I|_*s#gb736}R+G!NuujE=#;{mhha0>0|tx6Yt5wz?542KMFQyMZnSW8^k zVuyk`C~3LH89s<`j@ zUU_lQx<15)`9jnlA?_?M_i(4D2rg$Z6E60>sSn5q>nE?51VVw0*m=(@U+W`oAAP8B zeJs@P`#;;g8W99}h{UN`1mcWbe?`&(0` zFn2>Un%dp*QoA*2)b3mefr0Yaio3==$qP#L`w4j2`fq9kQ)zM3+rTA4t6kCPZ;DzC zT+N$S3>tge*ULJq9Hc$pEZCuo#N!e4c%hn!bcea$1YLg?wt0|v5{l`Gbt{B0XxEo| ze<12=z`sKadEAB#kmyE&J+tybWxK8`4>DGyi1Pn1;o=+e6xowxj0v1!%gUl#^}WB$ zNdW6iM-GkP8nGYt?)oBMT&(W=x$TIpHCc~`XFpOi6^$5D+>}$H*5PEqHG^82lu4kw zEhqt6N)c~%`T=cELhsIlZ8^z7w^*vst&GqVSN**Wz4f?fS?lSb%+&jLHHOSN_~iL3 zi-QnFC}4F<+2<$6{;567iXYnEtuWN$M#iZ27?Lm&wEF8%R>oce#8HVq_|ZMR`^f^` zPFVG~H``)Smu>y|c#a(#xztHZ(7Qp7?oC+m{w2Oy?|d4IAi$5JH8%RM9$!3piLsB{ zJVYT*j)&Kxr!A1`8R3?_{_kbg-pzqPqL$q3u71z4W*W61=MwWJ)egoY*wBUv%lw^P>~G1L?`|W zNGAm|wiPM!;~Uc47~IK{uZb23K8dE(7?e`c z=h-I=Dl*y4LEhW$7ri6ibvqx;XF5%^BKzbsgaVui?@J&YE1C=|w4yoCsbhG0>zgWk z19dUl+nESr9!dC6wI_dT>LDmj|FK0fPD~7m3*MZp4?c3Qd3ljACa!-PIK}^eR!82K zNnYz6-n-UIcnCCazSUWq(R3Zc>Y~`+nC|>d%;yLRsq1cq^{-*bfT4z@1%AeIsVO z@5E9^Gzj;2kMn}fC?PcQ<>Ejiq6mA!&xjl;8cmg@>)^&(e9E=XLbF}ECKyY&c-*7F zsymACtobo5CC3AlDTid83TNl90lLA&R5#bdeIm3sixy(Fk$Zzs`y@rX1N4&qiuC0A zauOs^Zc9kpGe>YPj<=cJGnrqMyqbYACL=*)v`D;>M`=z=?!d_t_Rb#F?e#a$iY%f2 zP^aPtXyL7K%4EDHJAISG+2`{o)fADU)JjeLRH`==C+&^_NfkxN=eU7pFE)cjHpXLgZlv_^(HV}#~?5*Dxy%@ z8FFcz2CUz#4}q3a%76{gX4A0p3jz=plpcW%4XD&Xm(?Nf*{I>n$&v;!_Jr4qG^TtCr)8ey)pnR_dnnSec zDLsT$E;yGHIMrI0!SM!KqhEgU&2gi+v+kT(&yrlyAQ3<}l(&Y!drSF&azFR8#^~zo zcd~fef5Aj?8yo{S$*M*ior%rpiY0-qsaKy)_PBpV9FD{F?jaa#E9QC$!G%&d{o(HI z>T|p=_(#i?>bYyyDkhVV0r5Y(SR_+t;+lz3@*NqH_7;Pmwj*Z;q^c@;vmnoYwWVv<}_xuad z953Ne#3%x!qM)vWLW?>v6sk7wqthAB7`hMe*-0Pf&9_DIIT+s<(HnGf1!cYay=B?S zbIy}S4EQq>UZR6x8r>j66H&W5TE`+Rq(=y4@G|g=+8T)7J-1H7l1IREmsVE z|6)2j^h>uVR>OfUO!=xi0b2ECbFmXyGKQ}hD^A*!A@2{Lr|>UQAqtx+(a4Szja^?= z%NM;@mw7C$K>Zsl@2bSu**;n7McUqAP!W50k0tCMEjxp^Q*>+dLdZ{OctouOf$fB8 ztK`Fe$ATb$Py7vtuj);GzScK~C-EMGk4T;2nAOv9mI`&tYp##k8_nC(onYk4n^Lhr zDer!PNs9lRZy0UJQG@^f9+RIa#l50HicnffU z)7hE7I5is^3Bcy#|M;ye&zEX!glT_wFNdCaA}$_@Pw($2AlrmZ8|4Fdz|4)KVQI1v^3*oit6jH1IwzK|L@Az(l+%e0Ad^PDeLJazY3LOt+VTvp8obQkPS_d)?6p;jM&-pQ?`?XeMeWXh z40=mz8R&iO{S9{F4p5}X%wS)t38Me9Nm~8kjHd`s#_r4oA{<9lP*U_I1hv}0-}jkE z&8Q38ByzsA?L4!s0cI4@dsuLm9>^|5psQ7MaPTcFz4CHs{y$W|a#J1=B19Rr&Wha{j=61hiXFFYgP1Zo8bJbvQEuqBE z=FRgOxsJDqR^U64hq`}`w@<SCxO;F)f#Mq6y*Pnl z#Y&4yaEiM_i@R%ZTHIZVyW8P;&v?dnzVY5?%pb|#f0DiSTvM(&uek&!v)dZTkp)(| zb|O3s%Y2z;yRVTSAcYUO<68BmA79>!ifv&AO4z+&=*~(G(Ix>8F(8xG2v(mZ$o20J z8I=y*(PBnT8ZRisBa{*CHwzY?E)Zw3U&#^!*gmJ``@@+iX_`5AZX0JGH^K*EiMTf~ zWN=+n2lhCBeH=>T1vTFasA#Va+z%r+HJ!d2r)rO+S8SA8yp;(setgs~d{#{QsoPmX zV7tmSX-@Obu*{xE>LgY7Tn1MqiGO1<3%RMSlJG6fTXz^TB41kl+GzZWcVY==`vu@1 zjW&e<{IDoS652Sk(bJ(#zUuQKtCTh@VnsnJrk$Y* z7D<#&6Aan+b~byQ{N4mss6VkV{CN=TR2xkd z+3+Jgd2SqA!+B?JUFsfMB;!$Es5Ab6Cf^vONvU@ana53|TQPj5_nVn-Nsh#g^f9MV z6TUw^*{T{61F6{pPCm0Y+~*`yfva_b(Tod92CELF6%oR?Vr#RlP@g~GA&i4i-3P5e z5Rq5UXD7fIh7J$T%~)5<1glHKwm8RJ-ImlC{F9nH^4=Rf`PqP?sjUX$D=vapPx*W6P=O~ff9dLr2in<(PkLLu@#%eZ5Ke* zn|>M)giNu*$Y$du%<9wIZt3 zvhR+vt`p@r+gYR#>zEo7-eS}`% zKQ%q#m~iT1TF>FPX`YWpgUMlc-C%*S#42+y1PVN4tOAv=z;`N-K{#{0ZCWC0B} zfKN@&IBWimSZM>%#&tTXp&!hq5#1W;n*1t_ejS?KhK*VwR7AY-DC4n&=_>R(dtF!l zQPXz&JG9$5H2d_iQ1Ec}{gT`FEj6hD+TCxRw@a!nk1FU$sDAyqmH^K7hpZ%bN4#b0 zkpUq}&iaW7Y#Qi0?oS(6f3^ZfM^28Z?uX8oEBd#DY-(3)a*&!KG5iD{Mo+txV6WkB z+2w!rcBQsj_NuPa*XWJ}b9fR`C178+2G9*`9c}~v9Z~xasj+LVBoIP*gN_7~*y)sb zHd@doUy88cI=n5GIiHevt#cWmN;E+O?b4n~oIStp`!&2i{m2gNoEAjDv>-_}cN5lE z?U0nZ@Fo2`Wb((ePUr3h*`_|aFXyPXBVV&S`w?D3H0KP&IXFwjl!Hq&=> zTWtl!pEpH<@P*EpxtZ)3L|L>v4C^MoE7E_K%e(J}$1p|tnd$huM_c@SZ;bWu+}|B7$+-NH6#@`5jkd{I3$~_=0F=-G>b>4ID9+5VRCn?ZT=5*u ze+YfpMh;E_an(<7JKkEi;pk&I7Rtce>fRLYqLI9ZlsOG`*sw?^ngUVG?6h#fwP!l< zr%yEEUiLInzfym<5&g9hs5BV+%18;Tq@@H|&{U6H?+eLY59IE9d-;z)-fd6Gcs>7I zvp6Ua%=Iq*p2Nv*-9-`6Q|?kNhN=U?!gf~Mq&X_5JzWSQy+YmC2F%vW|D-r%R&^py2G!ZIJA7kbtnDo zEKui#^unJ<&5svsFcOldn(RL_N(WBaHe55sTc2!0tR6qbQ=*cy?T59WF@!p_jja z_*Anj@rr}#%zG;U&8{4u0VuM?UHhuEIjZ=BWMGC5*ET)GD%AXCy(Ih_yjyC1Bq>=h z!mB>}=)U(0EN#y>HiM~88C?P_yP~5{Sk_&qs?=SN&a!7~;FuC*cPnAO07^dnW+b|^ z0lXBz<~29GFAK(Pty-eI9A=PZTinfuwOK8j@f30?DgJ;MGYPA{B1YB^j-ho7* zHpy&lF+J|(Sa*>s_LkjyVp4mgs_k2m^>cj6$rAyZTZ_DIOac-$(Pq=vx9Q%P|qy7y9c4CervQ$}a>r5+*w1AgNo)|8!6%%?i0_G?&=)+E_*! zwQXm`^w>Z+a`b=L{bqXSECq%wUrkAfMtzHCQH(obY@={YSZC`K_1ry@Jv%|pHorem z)@}0@TlLzbBblJ@{XsMtkDi`*i-TC?@RjDMP>dTkner9XNK{SEK3!#b@z$y27MF_3 zpTRz-)M_Ja^>>4fB-@*vTxTVpQxT$6$lc9-+>zh{+0y(vfnF&=QA%m;%4oTjC zqVkI(B?JWK0 zQ1H0N=8`BvjCYUt2PfMl;qiI<=e8W904`>nw1(>=pXSidmMAWU{E|!)awrn{h;>|| zUxa-bR?t4_sGOzJZ%s=F02F}eFD^*F)>7d97d4G_2|AKbh#XinfngvDQOILZyLC~W z{aE(GZz=DTzdYN|GcM%Les{Rq4)b3te{l3pFSuE~m`lUiuNFg2#_?u+(qghvc?#C{ zm)}3S6leRz(m^S*{&~zGTf{qKra^#9Zhu(geXR~i!kOuWlnO2?bzYo=Hhr#@r+!>V z>08-?*l>!!K9M0@0TWG;G}6Me97zGyZ1vtxRIhhG zNAE?&$6ya5A*$IJl>#LG&9bbn-Q|Em$ZNn|^hH3R3fTJ((`72%_It0%ug%1+J1UBd z7yeJ<-Os8A+4uE?hVyN?f*Rc@AgxV`wG0%B6D-NZ@kYg4%P*-_;?^`A#hT zq{!1f__psGFQ}$;kdGq(vh9DhbQxU&B2gg<3M{!wWJD?u-6RAc+zd;1ARrlh1c(HQ~YN+lf#yWD%O zx}a+$)DF8yv*XpD6<7&C;5;SHl?XJW##Y~}jb}A%Ehc7ENH_H=8iDKVkvTCXnwI1M z2XMf3%bQoq=j>p<{R8QZu#mTIFvL}~LE32?bUi1YK>~LIv&b^l5YEJKJ~{*)E;Jbd zioW$}i+G(O@gg`yb+>i;L~!l)GMF7ZLIOEW(m>HsrrbJ|GFT!xkboV*B*?qf%*#T) zKt{?XsB}0A^%H*UqFPVQUEy#H7jvw18mSBy4~cQ`v%){b_>X2R!l?nucgKRRTz{x) z1e6eJHn8={=$F52p4zQdT1rl#{GydZRo@{GWsQBuc^Gw@IFW%H%4x~Gfg(qZ^i73%Pp&N8`0HNYYpIMs=I@Z4_cUQxp3;XNkJSCn@?^moa!#AS8Zi z(f#a$$vQXh2t+JG9mS{;Pu;P7ClHg&oQ_qJcnLsN!a&$qm&K-I_-E-D+RMezU92yu zY)VmEZ;Ly6$yf4ARq9WZH$9s3w-aQ@9lQ|E*9lDAFNP0u@^}3h=Kd2lnG*KUu8`x) zd$0JLQfcLD#`WAyaY^a~cugOqG&f#Hyp^Q)Tj`0T%C{?ucx?*LrYA^f!WZ; z%T*?!T6D&F^%QOt+L4u?Pa;M$mNLGj-)1sLe! z2azlULmIeqyG`L4mcB~DNmp^BNS|fZXS=PHNui)7S)$AD!HjNa+ns&K#rN9WZ)$I+ zq?m2LJ|>6~T=Rr?lrY@wfM73OU$&o=%#$1$QmikqVL!k6MI|?rXD{69U1kiivxEm7 zC_vYU8WXso7E~jcRoP`ch-KrVhf6=y>poOtG2A=c0qf0u?|!SwW9paL*!ua7Hg%yTNO(Go7#OaiRIoV$O!m$O_7~Rmm4LKeM2Av*J4}0e zC529EKTGSGD@vtpcaB!Zza2_KL@4h-mo@V=msn8xGwb!!>2b-!Kp*GcnuV&ky~5a7 zv+J1V#FhjUr2l}BbeHicq;t6ty5*uE%-aB_q;j9My?BZ%8s+H=6>fv^7}T@~D1&#xX7Zo4*Bd*3oDUN2ug_0Y37v|DRmvWKn{m5m;W2w}XQ zX=u|<1)!N*iA<%c^7_}&20xMP;xLcKK(W0&ZX186gv^6uaz$s}XjY2Ig?`%O7{h}a)IhE!CosnMVx;M0z+|&@tvv!Z+s=f>{G!5Y*vJPh zsFLvaA98kq&8TiRlv94x(0w$%_}Mf$R0zcq~9Ec1t2ziF%X!lh8}OMF-;q_O196Ux(jf zut8TN?UqpI4!$2!B9uSwT?*<^6hSablJ(BPaid5OOxlX;63fVLMf-d9Q|Y# z;YEj#l7=4}BPJp$cJ@Ri3Lg4l8HHz;2~8)HJ`h;F>Uw9TUUW&5A)9OI=k0c;Bs$`* z1%`tS+UQgbDW~wT;$z1zozCs-jSJg~4sJG-7H-d1KVu|TICyVWD_Q$P7fp;#ubjzV zGgn{J*BX8O-D3_&!B5u46X_DH?wKvgf^Y2Rj#WWp;-s?WfCvB8JD;*a-^=E;rcB|G z2$>f4iEQKV!sDZlK3gyQ;>#S*87HmHhl)%7nsc4kdwbVApW3PTWLD%1oTiStM^x9L zR1)FH{!g@&O&9c{UJFQ~uGdtMH7(BdEc<{^1OGdwj+_1|VD7iu>j6;7qx$X3?x9kn z@{-8&4&>*;!X|(u8DT$9*`eHuKSnqVpc`?B7C|nfgp0HrQW88vqG=C zqlxv|1c|^Mer&qW(zz`w7}DWcg;Ow&^t*=Ul*6?vKmm8ozerU*K<-{rncUe7mlZRS zL4_aE#)`9NAfcvI&R@T&tIB5(nSIHY)L$oMtlme#)P6*O};YfD^-o7jfwwdRr47EQm}w6uMm3uSY)7Iu4#%h7#8m{8 z9>A3cFQqE9gT)$raA4+v@>=wVhF>bGmxCzy;<#x6>J)`R*FYm7?|xn)_mEey4!tqfg5Rm#1jQ<1bx{)gcp$liWu^$NKMw%4pwO zw*Y>3&Ef|)FK0D=Ui}=tm8{9*{jvO2dXJtq190|xH6DCgFy+$caneC=-ax{tU%p}S z1|bx)ae{O~Nvk;nE(QmGY6OnQ{a!`fmiFz}cy1Dt__DFB#{)9p-{4S)Qo7x21%sIb z%&4?cX~a%n*O_F^_eIS;Uaxf_CqERo7n-V;M=+7@Q@ogRGfgiNX#dC~v6p~qX*mAJ zo!}?Rg2q3!q9JCTjGaCzZ-+?4t6RP)e2j$1^;15qiao|mSXNk;6#oB~C1iMJsUaPy7^)i;L$mg^lRSMh*pw_lsvKVg8ZKHz7j8d^ud*(lzMgW^s6)R>)SvMyr^v zEIP;GS7b+mXC7HN?9{%GG%DemP*#y$%7)`M_R?dAJ0pVSBBq#HS-lKOkha$jM@AF7 zCHD%otF_*sRj+-x#zLO^iHi4{h=swv{pyzrWeKX$jMu+TQoeg^GXe5|1j(WUki9x{ zg+uvRAgQ<8-p&m$Q=ZRF#DCWxn7Q^P;94~PT4WF*-_+&uQSrDGSrZdqtXEs zg4%mUq%lM?k+047MWw9Z*;V^qIu=kVx%;lB0=Slm3@eeB4!h8n;&I|`%K_E2Db=)AXddt~t?#Cs^W|!o z+2|6a0V=*xcYnS%EeJin@3!xbdSIITCH#rbW^R2z44{FXIgYJmIfh`|%-GN&)5x*y zI+VYeQZ}5}^u?y3^2D>nNPz#f@G=MAGpMexCqAAjP%=i)ATw(Sg_HabK`8OOQr?tij2Mzw>f{MGuHR##5!oMv*)9UWB+o z{#PXF($5I3`h7@T3$#Kx$%>G~fc=Y&LS2bt@50zys_Do{d1bL?pZ+P zDq{R&N6W0)O4sSzH{32vy|VnGU#v&8r+@Bq=uoMg-5s8vi_UYl^0xB+O{qoRlp6j` zixJ~#{^NPyR{B3tom6Y__nu#~FZ+J&E4QUy&6uCwyl}EJv0y;UF&I;7QK43r+N8Ho zj>y>ddQfJbJyXOeV+W4Ak&310ajGLtfi_-*3?H`d*(#2tVV+sNa|a_Wk|}0DpSI&} z;TU${{glf^y{`BkKR5?kQF&BqtI2*H{hU@8x>08;8en9a7DCDNoLtC6-3Guq9755n%U&s(RD9MaXZiv^Korz#Sw};OWf8wuU+oQCsyw^ z8-*l&j=i-T^wCl`EN?lFAZHBU1^SUFk}%@Ifbskujm|!Qy`0v(oW8u2W9>@tV-@g2 zNgnysY3bt{rzww!_r>>ZC}S$G9vg*2hNL+d9KA6u|2jzSNXR>e7C=ZEG(*Uy)wt;H zvxY65Y?|B`CQuukD^?GLuWyCN|8IN!-zyP8Bn@$;|Afhd%J7BZZwh{@qCI1XfR6`7 zSE-o#I~@#KwJ_G^d~z}y{hhFt?drPS!#*Te|F!kmyq#M;gC%OpQzpcr^YWS5qV5?g zmI>0(L1T!Qg-3?-rn(CLrlZr){$2}r;Jwz1b(?|{deM|Ne=LPE`LPpVG3lgnyTNSH z2%tjzq;$CRt#Z)fdX=C?7sX=snqH5ka-1WtNo$MKKKU|f?QiCq;}>5Q*xZU6`AX75 z*U4v2=d{;`M>p6~dU@%Y%`yd}tf^1c3J;pG0du$&lw#W0=GnawY1Vx222AQE;xzHx zB+;oF(DQRYthovx{s^FEW_@>g-XL@6{D%CuTF z7GhVey+NrZxqo$bw9Zn=H3%#{S>he4?|0yyc!GW>U1g4fqeQR2GM>;)td|LPUpGB*b5L<;d5&Mlh!V00e(AwhJ@)!>@Gk#iywm1o{P^vM)>dFHey+E(KX*YA)gwc+6WR?=7 zJ^9Z;=R}Bag-@1>9ZMRN)viXEynPFSJmH%x5vyOpNvGvf)1$zc()l&^D!Nm9bAuh{ z#c=qoRJsFmenP(dOFi& zv1Pmj&HJYziv8Id$DK3Po*}1GduFSLqyDOgOR4o)nykywuow#Ewqp$L*}1!Ml7P3d zMLVKPIObk`r*wy&1)YU`)d$YD*#$=8$0f)Dv;H3%_WXg#gdNl8e6y=Ux>Jt1OH%*x z&et=80zW^)e)pa41_}`}DU;i8dI{1%Z8gWbBO)w%-^NK8R6Qks!CtKZsSt-xy;EV= zcl>1%SCD?+`XKTX^L}8>+Ua)uxebExl5tyg-rPw3$_FKD{P6Hhw$=zvcb(cxZ{}*= zzbrVI+JBz%6Q~ypPz+G5bly5`S}VFyZrl^Om492qrJ~28wL;i0)r9^)$nEWjIRYD5kH(CvW!;eYe^je6yYFjlN_Y z-*!NJSG|m!-Ls-A1y@uKBs$I7mafiM?Ov}y&bOJ6#TMN{z|xg_`^x3X=u)Xb-t*zE zMGm;WukE0fMd;fzO}$xhON>_~&07&|+*U&Macy^kJ00Y--(ur7+PL^JD;ycvltz|# z5r;NtOIYv#wHUbq^t4RavRMz9YOakvi9J7gG%7Z(-T7@j-`@_bIkv^TJpOd467l7< z*f%}VL{$?Z!uGBV&rp?0nV_%WR^dsV*j7@KH*KzNyL9%SShEtjPoJHwO<(c|ZJ51n zvZ(&KRk>C9zao#b@t!V%|Ku=Zf}`M^w-_jVYb6NumSJGc@Ass#rTwvXOF-{TKmD*@ zebow`g2Mg@|CR_^HS{T1DrBM5uhkZyg$g#l{yJY8cn;84uNXkV`-B+45Le_-rn#AN zJhmoJYSYVe@ownWh)&?)t&PzG$gQv$QZEw;jLhUvP@xK$`Z0l69py+v#utZ4dNF(- z{VunR;HwC)Q_-VDg4A?5)3EGn`j_+5tHK7IA;#8kn_q04MEs5oB#hf_UzWPvbo4D& zu7=km`&>LE$wgE!=1{vm@nBC`oyAKm27clb4u0fU;=cV{#7 zRpZuy{kGo8E6s2plBd5%N7Td7;h&25k01YPF_l;Q%~vgOe|B9DbG|`Chk1vPxXG^U zhWSwP!El~7u#dD<3t$X4W!qgdUY_4Qhq~ifKTFRD;8jD#T+T^HT6>s7n%4Imx994qh|L_q1`~eyde&iR2NSt?v~Z% zh=sfyaq49FrJ0K;xU3BJk^KCNFxQc~7Vn&5V&3F&?q82Y+Idwq6sheVgw!gQ^9eL# z@X{ZaNKrpowjoftmV?-4ZJ)Jyr^8}JoG4uKEZIna|3{b4_ zsqjP-&BRzI*)JJlo^9HBng9NE>wL}lb*nw@jDFWlJXWRMh=;~H*;gJ1~ zd(m@vXLYo!)J$bK=YsKrl98F+*T%OTK~DDrLj?*0I()@NiYU@jzsux}Rv%$yG>p#> zZz&297K;Ra{%D+DL&6~7W!x;@NxOv$0}|?3-OI;ERqwU-6l(aM&mXNN=MRX^*6u)( zdeWL4yK6+MkY-oGzL59X&!KaD5O%V%rW6g;4xXy_*3X8sgIc%!x@FvT;!V)~CRW!E zEASTFCeB2W^))`#QK4SXPksZ}*aK^cRbX9* z!KRU>8q96c?;R8N8v>%@7G3t%zE&sIOD=3s3rs-XHl1+^VD}mi-cEF{BLZZ-EcX_P zE!P}zD7DfJTMQ)mX6 zt~5=)fFcgRR`MtTE;}`j{ZAS*~RY-xE)3^%FNg+qztm=L%PNcf6wHz>7#=+p&42Fd6IP}M@k7dY2S{spPD(rI zkJ)q!Xxs7^mwpbmMwOE5Af_LPy1t66Mu-{IA#!7I@A2MsGbS}})`_7eT zT6aa3)HXg|*E?puBEEk62sJ5FEe861ahO525pO(e0T|2>wNXBQdE}1u$V!`7een-S z17Pn`csu=4k%$S~Y@(f@E-KIJozJ#1CU+=5D;da;7^^@2JOB6c&|?Ph`(I?<7@%g4 z&ab4W#W^@6%qf&-WCXq=t}$&$?N1P7;g~`kSakZ3aqy@UnS11*Nnu6&HVkq0T@ZSC zuc6`15|cLt$OFDZPxenVH@R$8CK^j2!?L;Z;I2a~wT=px(Hxbqm$%n(T=q#4-g$OL20!Hz?1z5UflqjKz~ zHkk|&QiFUlR=Gqw&7VZtL6t;927?Cfy6=e*Ti=NsVG>QG%h?OL`BwD`ps65w{FWS2 zQLC}=g1{p(CI<&_u*AeBUJ6s&uIu{N2S<@a$VJ5eEz9{k7ZkOg*QuYw7|8P&?r?LC zP5%NEv|c z!X!c*eiPC1QC&QUon=~zHR5lE6;#(Nfuq}087!Nub>wh1RZk{jv>s6d<$LOB5gs-R zqH?PuGenq!J`UZgqZyXG5Ff|Be7xwvBx71cvpJf2M6ya8c9a+1fJY7I?k(*RV6Dm| zRq&^X%s}a#5e`Yv=F&sT zc0*WuQ03jbbKe ziH#QXK*ne!wj21e`cz9Z*23Z%Hu_X$IBl_F%N|l9dVaR#^H89+(ZHu*2^k<)2g~{Q zEiG)s#!6#Ke3C_1N2x_5yQcz;#6H0-g__tVGAOC(tA;#2;S4M&Rx=1Fub-sED@P4! zc2-=}#)W7#J5m1FVtAx)WZW0VyO;-dAU~bRWvkLKLDdczD+evYl<`^FdE#;$@FbU)VAGM;qlyW}F zUxdY`#sj(_!F{DQpRyS)=2fPSdk>k^=>dK582yjb%>58al40{YSiZay%uL7pXoTGnkem=T>}QXVHlXIUGUw*!c+mV;EXO)u=cR@C3g> zk1%U4(8T;g2VM4epyN!eF8{nW37cP>r!ZF=D#3SdOSuX+V@$HDCk_W6@B{m!s61yc zb1!=9;14@4OW%qiL(}^$9#6T9Oag4BFQz^4^3DtnTHNHk3(RriOx(Mon^%??R^RXm zQ=@E$NWPohA`-U=>PL<|BO76aItF)g$#SC*zB}SkaokQ_=awrALciG1i54mqLe;RaC?2cNOQSNR& zo>_;i*2r5=ygYp|p%D2usNsL^1sYMkL5l*jP1$h|}9RPqLb@>~M?3c!nAVLQY2l3J!n4*kYEs z81*w+kTF$5GdZFghBpW4N5?2HcgqqaG{#ue0q*VFZJ+PM)*Qbz#^fFyI(faA5ebNO z2+i7vNnD4M&}5Y_!cD97{KptlU@MOqaIRAJ2mI-WY>HA~E0lFy~z0!S8+j zpJ0UrfpBaV=NZV(s@(Wso`}|Wh3jI1$f@~7!zjVqrG~z21b_(;TCxg6H4}QfPch2US@LH zOiLxB(t`U)v{Ld+wPzawmqHHt%zW72>Jg8)AJNa^y7SP3X{3R-Ga!%TXFp!Wt3 zpJ#tr1lfJ;L`-)=1s-Rxpn)+%EdV_D05e2m-H&n?6M2(~=i0-M8U+OT!8tK(GX>)8 z;pIFhR(|nX?(z&n35g7l!)ol$K`FUX>3)$4b}UmAwVYex5EHy z*Y8WxpFGV{v@A0aoE!fbFVXF(NKgDhcFyY{8 zFwF?$>s&!J5P1xhP zV3Ks6oDN>4ov4PTK)8T0EG)AN&EDls1Ro&BVqi31$e~or(YXUuFXcEMtIiCv50`eH zCy$+BM$Oi*ztl(Ok4I)%O%0c7;~4k|K7y~WjtywL3A6}~kcbgc3&9?!OC2_C$OjRG zrRg*KHCn%@AGpP*Iam~0|2Y3RK+S+srET{mlf^%(zAjv~@jI4pJ~ks|Qcx;lN6ZHG zx8u1D9uI%?UVlW79t##P32&miv(TSno>Kt?cgV6cF)d6Y_}D zFu5EIrTMkgzpEe#fnFlMVa!`>JC%vZJ+_vmsuv4Qn-JOmZ^GTdoT@ZVy&B&KkP^Tx zMH(zS8x;f+$QO_Lq=a%dHg;_A5WO`{Fup2Di5*Yyw~G3N!%7$UE1W+e$b;hE9K~Li zB%VD}@wMZU9ihZ#E%UU6fA({_EykyRk+xc{F8BP;e(Lxzvbd5oP$om8MvlvSuT!J0 zvQ+fr*4jq0=mFTQpL1f!I8!3z82w%%x6j!Q|BREj$nHC zGZwwTefyY20qFOE#rsorrgdk*cBnqJYB+L-+lHWhT#)V1iwq0Whxj~90tdVS_U#IQ z5Nprh8hA+m%$&uF8|Oy2jj`lxu4A>nTCt2Eak;!Hsx3`JF*vjo(F?0eL97S`fF}a< zww^*-B7dza+@y+@&yf8_Nfc@^cn^+d^@03|5GbYt(r-yi!2}ph(~!HWk|WI{(cW0d za+2y4vZABF(Wz4*<1_J(Jq$$s>=^3J@<}aA-wqo$Wb2-RsZvr-7<3+*i6Jtb*i&he z;Rrm$>-nUkVad6a7uC^GKdATON{++7Y4&GE)QBqxB)KHGjB%@rT((4Tgd>&xA^4MX zBx;ay{LD-8HR6-LIC}Mo67?bZ*Z6jfD5{}Q^Y!oBL4Co}E`M`z?(+UNWFll4CI3*2 zpL3vLitgrQ54Kl|z~m-$l0wo4k}?FP6n)_*e#mR1!YaM12oG+;fq}~xmA$OV0=3?7 zb}plMb|cGnh&nmh4T!v)lg0L?vj^)S0C*RnP56F3SiOu%hW(KEdHJ2&95NI`7)h@H#}ZJUG1&G&BR#CUrUM`o&9wsXr{4yx5F!G+9G)~)|W z6Lw&6&s%%X@i&*i9SMp6mSQz<(6371V_X%I)q7!bSiJvXu8zFB$$_Rq9M}qW$%uZ? zwQSsO)AJz^szq)TjmRL*mn3Y*>B8)mz<-yRmu#Vq*`fErvg8Ml$0e#Lo~Pi;k1d=8 z>5}cD7G8IvoP#8bWYWe}Nhw7MZCLEH5QlDA`+R4rT=&*Q9nBu3^sHDzFWPfhJOyo) zPfv!RUdp`ue5zpzqPaqRi(jJ^_V*o$nA-HEIAM3`fKk>*azv!FDGMdvedph1KG3PoMU({6u;}z6MO9%>9pzd!^u6yjjgQwQx+DpQl$rur zB8RmIhRv~6@M0`7YXlhfhrK0n7l7R}r=rjr#iE`1#jJqK$Ng>Mw|r0ZT_#o{p0B|! zRhgE24(qq93C}3v5kxO-nSq$!Y`u-c>vw&Yv;#kuScBd0>03h#4f{90d<#zcXF=gx z8(D7^3(qU30tLpbwyJ|;K{Dv0t zEFgOXE%8BjLMiXL3F9&>O6#C;>}|rU4P214oJPDAC)~!;+68uodnR4QF3w*%Hv}`y z7-_*to~)@uamgh;5!#q7lBjdocr8#y0Lv#B~+ zxb`}^t=ueHI?0^5ggIpCU5PJEj(;(UruzSh-U2B?-x^Q07bE^tbbkuz2P$Wrw)EXQ zAg?fQPX6qwx4qW=_~(>%LO3i=BXLr;ML#VPo9vZJMbwm&+~;@4-xfF1RvNT!Ku9J; zz0$SXm`Ed&BLdtNgjxeeQ`7cY5~nw0dW{FP1X;N?~^+Rr-Ed z#wXI9rO=7$nA%`4vga&1qItS34S|;UXhzEZ0|7J&G3kT0q{df)X|RTdl|i7+$$AZy zaSUU&4=QInS{vGWxXlYSOA&6Vtzxi~R-yvXqR8&&0}FTHUhYr6AvzXp`9$4cHOiUF zWDb>g32p3ms3E;rL?I(_=gDZPpA5jprKFiaA~_Z_3?J`*wd)r#2JVC{pwKDX;^%k&iC{qacr~=%QYa9UAAwheQecNH6 z35&6U_Vk`>D{wPLI%*n0`#QE1K@+WpTQ^z5D?W-gw_wmJp4X^QBfI~ZmjV1f{)tA2 ztq(!o)|r6BS);<(7WWNftx~tO)2kR-222}kyXQ4$+b|!Y#4$yDSTt5ekyq8A!*9(~ zx*F}!Ew%mzjs06!O;k^#RsNo=*G$tcj_0LL(GwGIx6Zf7yd$_xgyA|`o<;4Q0`{noDIX!9%m_K?bFU+tN%Pi*em%H0`5q7m?t8)aP&GJ;2Vo-j?xW*Ed z?4T{Bu4`Gdl;Htyh+6MAePkaU;rT|PaPJ`iQQwyCq%PENOPt38KW-}6szkrEu85-v zcQgk}2WCW^1s}<(yIvICB~bNSr-@i!!&1-;bakm<%PvX}TgHDhj`U2R zV)62$%E@LWzW;=@$snGdLUZ-8_MVMP&gpADyTJ( zM^68Tg@160w2qv0x>-n@R|Uu{pL|g`KobAMu7ZB3Sk4N{9)Xb2;V%=y3TGHMd_)E` zAmL5Mj%`|SCx}L{p6`z;YSm6OhT zKm`yhF$V_V8F_7b0EiM&k7|EUFsl|Wa*!XPAi z#QMgqs*Eruj_kqO)ox8uO+QX4WGDkbEYvU3weg{Uf^DLD)&WFQ8X0bk_}>%7?v3T~ zPaUK41Mb4-U&Bt#)9>~`zmE^Gz+iQG)jjm&J!;6PQ0W`4b6rJ;wrZgQQqID@i+Y>B z|8ZJb5@s_L~C#beWj|=j$Y!WZO(hjGBV{;}Cw&M6zhpyY6 zgwHlv4;Gy|jPPN#-Y=7nqePWoB%&$))#e3e;=;dWq+x7X3t?|naQAS{pP_}6w#X4}rmd4F+PM9Oy2>U`&|4)X*3A@4AmMjk!cP(-jz%0!MMQ0chR+FP)$h7k zzxS^f2i(4$61dKzo)Q<7GpusJ9Z_FZQ?zp%!DfdU)-I4N&0e+3&8}Z+NDJY0t5{f^ zSR#GJ(9#GDRhsYGG?h>vN!#`3awwF}qAkg6(a30irx5tva9iQnm}b0Spty{G7!=Dy zU#id(gusJJ)p5BhCmJe?4k-=%Z8>@{_;EOkpKI}q1REAap;wiG`}xW54{IBl@)|*Z zshRu6f+MIRKtGQs^WbMq86vs&plslDP2y#3NLd{>sz%wL5`cxW?jk5*@R(JD|19Wt z_SkdnGo0o^f+nGyhEAf_dyJ{upm_?chrwn=GyMw4GQKaWH(}XfeoeA&fvXo&o_W%m z=X)zV0i~)Y^Y9tItqHcNL@H{MSn? za_<@TyB-+{*;7`SIVAO3$$+c0toUc0ss{7>fpSyGWbe~~zn5nlRWg+yhj{xz@!fEY ze=z6kx`6Lfg}vJA_FPrYq-=wBzd5HA_kOAPkv6yCK8?^7`rW*3`B|x^&PEO=g&4pF zETOhdb*MY1j*>{|m&n^PNNkieJ<-G=(kdf z(_c*G!tH0XAJh~ZfoL*&X~AL0o6g#q!~=CRxb6e*EZU66y)9+iHW*77A#GMEhd)Jxsx})i(38Z}g3B8k^UDkTW5O;;!RKpzYDO2Ru?d z+u-Hw&9yO%R`q=)^*tK3e;cgzs}I)UU3MkUP{^g-<}n*MAzdBy;UG)y(}ibz_x=-K z)da&y=1EsgfT_A_q$6}Bqt_C{V#QR3+>PAIC+VpMM0DmG52~8ERF&DKlDSIw=UC36 zG~T+ELxmQDzIqxdD;yqz0*ESa6PR_PV(=zySO+Iw|37rSWl$X9+ASJF2pULmC%9V( zVemk3clW`4aJS&@9yHkC?kd#!a z-fe4{-f&d6k)AQd8B4ojdzK@x|Lb*HsS#v)l-$yB(NLOykOMNn+JfWaje?6p`iF1ieuM`o`ZAL&ZD= zqzvMBf}5Oxo1TT4ZE&Z6{NS4doQ!TM7_+EhV|^W#W)X*nU-fE|1>ua%<`4J_5e$~f zU%M68^Qeh{-8j|Rf)M6^L}allvdi5v-UJJP|G(j_jeHmTkj@!0o|(O zWlngMYkXT#@TcLbC%Xdw9n?+$-lW&Q?(Sw+wOzbj{Qui?{vYP~`Tk(QD|x2gq_W`9?;hv4>x*Y76D) zYxh5*W5ZrxXr~>=jx1~N?`0&Gf1rOg521^^&PXOKI89;F+CK&4_dB+aq-Qf>1vbHemX<3k9p>1o_WQp-JtwmqxzH>Wb^Nf z+sYYtmlzBUZ4V@+rX@e+fZavLm~R;APyXp;_mi81&k za&EYO;`^|FU(pV)?=sqKD-5f-EWPVThbUjC8gmTqGlq{E?3uj%nBUK9CcZHCZE{YW7_L3vBj{UPh7*h-P? znq#M_hU0V#5#mpfCVcmH(*KfI?|k;E<=-8N%)*c` za(S^?C{U1oJ&xMG6^Ec|C4f2iJ)570V3p@#T+6E|h2tb}YHKlfHS!QyDC84U+8Vw< zGs0BSnmhVgWh~WWUjpFtRuZ(eyx=&<)#Djg1xcBDOCa=u&8FKZIg zg`zuexz2NHa#^bCcmbT+K5S&);@4dLeA_=j3_*p)o3bS_2jAa;3p9!@(_3;E(g)oA zo>AC5F#+%kP0-d%^>#;zVISC)ed}-I5$J&0zb9_KD+s5mo$N-@p9sC2m2y0?B27@8 zU%yNjy}u>5yW4W;-{ByB%0yx#F80?hp9+GVLeJX$49?CqWk|X1pyT$d zJLQ4kxca$v!UPj8806nz-&lU11GgdHe&UmLG@Pd>K{GrIj^eIwL$1Z(v9wzk|3*I8 zaqD+K6yv@fWD&@p_%O)yK1K; z3I(fy^HY|!X~_~+_cl5(Q-=F?-px;`1LLSDQovd$W^^plB#s83Yo%?!;xanaeo%Xh z5to5wvNHh6V`+gpM}RXr8n3YC{;UPYe8b6PB8<%th?gSGV8}BWeh3=12Je<-TIiMs z=TSCClIqhQzZ(>#S;%tt8dk&Vv6y^B=({>9xFgn>56^I=KQuMUO3CE=Z& z-nbrC2yS7=6*MV9PrE%ecLz5Cnu`i_u*dZHZ2n+pOEWk2K1(LP`FvGh9=oMQbFW&b zMcd{$xy{w1)JxEelQm+24_QDH@)(9VPW_BSF@A!U_c*h6M&6P6FxaLZ7=3UPDwIq4 zn|>nfW96jxsQ1JB&??G$(xIZ-XTyPFFsizPZNX`mdtnr&)_^D-zmmF-)p=&SB7as= z``?rz|H0duE2Gc5{pqDum0%vK1n&Rvd|WtW`A1I`UU?p>;NE3Ina5#>Z@Ql({$7-u zWQ75`G5XrMV9Q0jnSk5KOA9>c2WW1|P+Y0H@2(uB4xxZ5CE;nT&$mo(C~>&ok+koY z%0&38{Z-v2JZy|vkw442bZYZttW8Z=cHYIgw$M#z` z^>JHvwqu3vzew{|gSD@ElBXHA{`io~FWD`D6-1lyalY^UJ+znjx~_>GCqvR?KS+QR z)_#X1j(F9nMHWRh((EN8ai6g2*CS5->*pH!Jzr&48*8FFtFN}4KqFV%$bGg#4$mCO zhpjx7^eg(Mx^J8Vl8&cvXeeA&9G@R`j^OWB8qTwtvSJ*^Af zy~>M%r7*f;MUu|iBv>mdjdE-`(GNo>@{;`35Q-S*^TWjbmG?Xf-KV12oC$;jaB|#e zY$Hn}zlAfLW-b{lKewRPsK@O13nlQRzcE7k=JWT<=yxH~+|68cjTv0)?<9iN$O(de z;06etY?MpMa03*A^rURu$-`W~O6$3Z_fEJmAayZhePG8<)JgE72}vo)+-U_v(~AB# z7vLi(KhES|02+`IReNei=lvMqHO-A^+TqWG?;_t6vvyOdhJ z_u_Q-55@`pWX{ovbI!d-ob+@<)Tr_$(iD6_>w6Y@$AzX;ASUq=& z*k96gd|Cd`0gppru4kpm*;^ra;-N^tZnN z%NJwA`V*3D*GSSM40$3NFi^U$W! zzub`V?k_=<6EyVx6VgR;3_E8TiZnFp+cXVUy$WJRq$X5-uCJ+r^I`nfnALs0gEnFW z6b%#u$I{jgi($J8i8oD$X1uGG$yByBjdd5`sg|~$jP&Ke8;JaBZn)X$m>nVG`%Z$Q zsVFh2)P+t&4qNXu0&H?%L1UG&TYH#BVhAs#Y4}5DsD0*qOrG*Cjk5cvdzvI@V4{17 z$abF~AVtx(hgPuyva8t10_t`;Icjmbv1xHRDR1#Oi2}ff_{~<0kCVY#vHu?e z%r(3YWm6Jn$04%r!J;^ap51bu%`o@MKpayxIghzjPLw5nDIK zJ%jGAY-g$R?g8@wo9yVI@uz(N8~0pMXSi@8oIBk2e*DRzy|&}$0&WMVc7FjdG`XR) z4=qb&y|3WyDs$mFQNM4uHY<}MU+St4q*~eK!Dl+P4n2Bse*8&ySwjlQG>;6n2GLyL z2(-DaCF_n}-jThduaNTImA-YjhR42m?>|8Fn^WwW|7!;UAtq25`F}JASz2P#{}C+B zD5Se3ul)ijY>SAiq=K22Mfe2o-4OA(#B3kKSrF~E*R+gbw6Xj(;Yq?@BAq}ep2A}5 zF`I3u0`3^8^>00~tIe92^g`fHuJxF!ISOR=%4A&b}!!imD3yB@nA z`OqxK+VTROd@}kIxtuSfzNFpmN5U47iQqXNvZDfqUM&xM`I^E&l*rmosXNTl^95r_ z+XsFrL(VoE79{W>5?PoXQ@Q7q9*R3LiXAeZ_q!ZTG0JP>hi{@K;lqrdUC_?;&{GMd zDviLUWqVZ81-3xy9j&ayQIqC~KWIu#m!->5%1vRPR*=*xsF;coX3%SCz){aEXS@V_ z40C8TzKH7meRY3*NRyw1&$G^0dPC9g2B_1H!`f<5B7mQ3>Vhg4owi!1zi-LU@vCna zG5};lF!G^1F7*ujQ9C)AiNkhi591-; zl)#R6Ioo-*`^wFsN^bbn=b@hfbfV1oekwixBf+ zJp}NSuwQeiy$OSS+uTqDkhGQ}EmTl7@oVA6ZP^I7BU-4gJz>;bR&fLFkLZEYc0{4x z$WYldy)g9oj&Oj-|zY-uvRm6_8WzR{QpOcf=1bjEEGrs#x zVcyAU`mB6FO)a8mJ8M;>!0!%T%g}=A$e_vvE!LL~?IH@3F=r+wC*OF#(j|u515i>61y;yYWXO;<-R1jX#RB7 z)6FcNwZN-fy*PAo!#KoPe#9p-Hw;X`u~xCaOuvdkHjnrT2Hr0nNf`%{>hed`- zE4|e8ZCTL~qx)MtNi$akccc|7+ih6?H`DeD08QHgN4_52;v`X=^McBbUK;_+V7PGX zpt7RcRfk2^2%p?Kh?&rM!gxsNpOLgD?;b7k=B%#fHk6y>ykGJqV9SFv!xR_&CtWC% zEwqpvjS>gp5V~%Rq$bOga3`5SkQOR*`$3{s^GgI`mnR=4-=5wvu-Qmb?wipsqshSv zoGK}=DHB$cT|i%OGKS4ZZ{RS0k_uD6RdwA}$r1A01`Xusuf#EWl{ogV*f&1YU2V;S zPZk)nSK|7kdc#+tx*;R`i1eXkYBk3JSkSv8!-Rd5)VBL!`fi2@dvMj`F*c+)7iZB= zt&+L1TV_DX`7PcVO{_fo(2uyP3>Bf3iU>Q&%~glQ^YsC^z#~5y!T$xE|D$~j#C>HX z6-R8#vlP5)>{uCaSQwSu@SrX_`4{s)d#hGo4&Kz&cRgq9Ibo2hW9Z@(ou$~B#J_G5S9gkr=s^bl_i{V|yD3}mX|T0~lkiJi zcUYMM-lN}f;d{(=f{`?_iwE-}NC~|Ai%SD5YC^LQdKC|~o(H(#aG&S;7|~2p+4E{1V|?K9(S#O6B0T znXFc6U=UZ=?q`G1)#}^C`BEF{PY0$Zapf3z?661NtqDkdP_%6KreWl|+77tMXa8v~kfxI{7?bHh*Cie?A+-8UU{9i=Bhn*R! zq8>-+?hrgCqHhE0JKt^MdFpSXeb8B|rA=8W(MK2BqGg!TgB>LUM4odrbgA$k@Y~xk z@;r5L%o9Wg4&{`lUv0J^;c8E%aA#P6;IZ4o&+OOBgpu?drgNKh2F;o&UR4U%dB?0G z8}}A0Q*c_17XOf(tLA^nFP{W~Y=;`;RVTf9w2o?l@`>$gL1e*%n23w(D8zf-wX=R$ zbW!l|2&k~pIE)}#XW*Fws`43ZJ(C`M!t!IfsxQ(v+5^*R7i}E;fj&Jd^qGx?{TUH+ z7|wk=2Iuyg@BVyy7xGD~V0AJ?&h#Hp@}^rYP{t?e>+iz^c5!Pal8u6?$9$j)fv;a3 zg+kkfpI+l5w;uf5CY%mSVx1oK@An~V69ZSqE~k@96Lz0DoTG~uc5?Kw14caOKfSAY zQB##2!~|9A0kXMoyW9?L{u(rZ-D2?ZHPf~mkaPMT$``hgv8o|{xi6_y=Er=I%2>{0 z+kj(m7H9;gdixS$ILJw0NIwuYcI&>Jo)%r`3?l;UAG~%o+7=x9Qh1olR}mOTFG8EP z8hU8+`!DLhJJH)xZ8z3f*bynT-Z9pScHQS#)Yb^F4cN{{aqLJCTFdqE?Fc*GQj<8& zX9`D>-S&q0{s`w_;*n>9zan(DQ0o2mVsi(EGZb{JZWwwu;Crk}-rprcAR!Q#I2~l{ zYIN`Whj#}r>4Y#B2rDXwL_M4$~+}-_g^Bn{-t0zdZ&MYuzA5;^==m5AAh>$ ztQLw89MymQYJ)+RxJ&W(<$d@e4n#kUNxVaV>6!+SzJ1E$vUD6g83RF&Pp|hPje@bE zHE1g?&T}i*%@vA@i!$?5Sp6| zjPKBS%+Z$DFCbM3#D*;B`E%WS8;~kAnNM!aHnHLkOynWXoQ;+MB8NhR*PfHyS=Zt(qbcGBmRp*^gj+yJ*N?YE}i}7|0Av?*fAAM z&ejg3;`7hsUwt)EeUd&qU@xtcsx;(rbv_7v&6<_krD9fI>w>klGk;Z&R{<>SOEibb z{3g{U&CvKm%cg47`4|qys;ek!H{Wy*yCSCJj53=&pzY@w`R!SOwa)Pzr%IJcFk;k9 zkd3w_ulM%-Pl7E-?gq|+@8jBiwPGi-jY4*u=9FQOg@$L#?LShs`|+In-u*r3vLbs= zV=mL?Q?6m4PJ3LUVhUACCU>$HJ5QzAk@I&&b&(nL4w;iN;%rv4tdZq5d)GWVw*Cn4 z{k=t3DZZ{D@+tfTh%(BGTWX6*!oVw2Inef78yvgMNk~Vq^`wP+Q>zJlJNQF5tiP)B z6aD(d8wGDmRt0aP53#Un!%Z#=gU|#c?7()6;<5^+veY$jO9?953LzY0Zb*%MsDfZs zY|M+m@4OM)Ua&RDru)-Y_^2T?a|&bo6vubf-#Wm%OB15lH8)U_hPTPvAMR_$eUkXb zO(0zawU!qddD%Z=Oz6d6)ma4->zB3Bi8g8upfCc`z0gx>h;=@9m+~=G{7W$F9TyVN9nYk_w+Fex0-a^-u6Fr&KzAIh*bWLWO0ie7cy2U*3Z@?kX7v=l>Qie~nm5(hvz_d_A%Q9E#n@$&zw+lE; z^#(01Yn~U*Ra0qsp=c##ze^`#>o?eacD$@!h+MwAoMZp2^P;S!L4t7NaUlA!X`1hz zOY3F~;&|{pwKXtSO)$uIghL>ub9$$o-F%9tS9hyk(Q(kspph0i_Gax9OLHWxF-4)_ z%QuZm1Pvj~Y^M|U-HaZRkB}>AQVMsvNzX@Dz5CfCL}&}OwNBO!7hbI>>6H-9qCG~` zs8~s8_S`_W2R8l2GYy+J)K%}I=d6a~xX084VQw4r;p!Lh-fO!!tqO+6riZTgd_M+& zMqhCbq>BSp>OGCWgx(Da2Isz@#kTD2->;KlZC!hl4=!O<4NAkGZD0^u;UOt!|MY1m zTUnZ{4#b=iIBTM!?+`z2oJ{9W)sJ zh(ss+z?vNNoKXK{5_W=I$bl}OzRr*;225YkL)!-CYHsgWllQn&XvLhZbrVajxgZO9 zuj7ff=u84++%$*n=bh;SvsvS|1E^yMn;q_vV%rZeS@=)AwLs%#fJ;d8=2Nq9cnVfU zFQ;%jS?>J{(=kr8$)ZD{rRMMXHXNSX&-RmCN$-?}Ct??GMI2m!*c2nd-J`<$#QS4BDPjqQtKmeaap z9X6bVyPjvB93SKC@#F}LzWCJMof=im^khG#k5d@lZ%sM&v6BxFVO91sx>pXpx<;T# z>OFELXCg%zJU}8OEx6&yEKGT096$M9a~@g{MObCqd-@F*_Aa-%O0DacDQF&- zU%+-%H!{Kc!@J3@BW2R7s}-e}MGH>D3h)8nG=bS2c_Ulw{npKY{%1!RYxkfsk3i52 zx!;F<0mSU~M?9a`A&el{)L9=Y+?OY|bxZSp@`L;KeNd$Hq>$-}!?Mi67RIPODd<4){1?et z|0m7$$njpoA$p8LI<=Df!Ub|Zve8hzIfRJ5@vg1q{YQkXOO zzyFD;PGQr;o;^`WyD{~^Z#a_ohJzd6#t$B!J6oXXF zVgAfF+OL z5s##gN5=w&o@8#U3aJi4-vkx9Rz_T7{Wn9H*^EpkFct-pxF%YozmA)i+7oYA&H&a@ z2g9<)T3!8uYIr`c$_NXWb4)7HhPFsXZ*gw_A?lAq%g<$;nxZ3?tQ^l=?VOJaWArBz zNS@i{gCpVk5;L5ohoLbJK6H?&^UxxwR;i6DmZr*zcWiNvb0PrW?N4UIG~Q8s%@=}5 zpz=SeFFHT(1C~p3ExCEadE5;MaN%PyxU2o0%m=c1DA#7U~jn#kV z5!z#8E~Um4>ISZkkmR_qvvV1?6Wj3T-ENg;oP&yQagGTIOZF^~-eibhT>?#-ZAM>NdoSe|co|&p5hQpNS{PN~}(ZO;$Dm@(@W&0Ztj`4TiCt zm3}otp(>?!X|z%e@3?}*vkf_NR}4MVz&P7#6cqWz`!q3-urd;Wxcmkxv6tfCdA~?z ziYiKKHHqRZhz&OpEvmUs@|b~>+}m)gk=@E zHhLJAXn(KL>0$BblwFvuY4lHT>q=Vfc%seZi=FxF8pMj&KYRD96sYleWRzgT#Ag4aA{hxC=*CWEP@q*3ce6>wqf#{6`dvGedSRcM8JA`% zx=5U~hGKPTwi(KpD-*AD|q2h0#*Z-QpLI)cGGJ9!~oXAO=o3 z){q7L0C`qQdOy@CjX9+iUNQYM!nu9YTj6`gWxcGZyx<%QR#@~wn6F`w)YY{wu$_X}n`mWPi zZcds{+SS9bIZswSv&f~OzrgXqw{Z3jV_Lu?btp%9{>7ZVRRgWPhIe7#L=zhjTpmv8 zG@UeFhsGBg7E9#}JQ7 zMov*D9vvB{eIY(_yVjY*@e%_wjzWCV%wPPC|BADb&lhT)nC*F-QVj$)+8|AwK}-|* zPdlIho^?;Gi}fIY!OK4{0dvwp!+W@_-cWbZmGz@`H|ylzYFQwCOyJk|e+i|4pZ|}G z`@87C>ieFruXzK#XeK}b2V%&EX(*p8{1FNpbMw8LN=kP`xm>9GZ8EXTNNwV=rS zyWFUK;jHoe#$MHp-Awrfd#q+;?9oThUbw9w@p@^o_EM#dxGJx? zEhy>6EW7OHM@&+}-e*Z-w14(FQ{T5u+N@;`EDicrR><=`PEE2UNW*vnB$?8IEztmb z)kPz8Q4Jp-X>xy&Kh{)JN{d>MT1MrJiz94;Ru$eoNYR6(<=W)1_D;yrDS?4VV_(a= ziyc&%xdw=o)Sz;)uxKosZKUcr>eP?c7X@qauEPt?%vg-S*OQ;f*Kg67-MVCAI+7_0 zwid0({jG%SD@|TSc@={WE)g*Or*5^|?teVSpo|mwkuKfiN7eTlhQkAUQ2_lcS;-$r z-rv>+X0X_x56Cz!c3<*7nXOP?(d!lh<;I-g_u5x z4s-~cs~;Fn*H3QRswVe0O0Qe%jn^OTUmZR^U^MMGSGnQ$f8cWvH0XgUu|?7=H0i{y zG0|-BMZd7#H|``}u(~$J+w|Rg(?OFWzJz z7Fw2YcF#Yb0%Jua!JTX(c9|=>|9#5mwr&WEu1EjhT!2vriY$mDFgYTIfOGMq(SE<# zK(ZV!Tx}4hnd#3-IaoVTs=>Q&CBFIu6Iy0mbwij+y7lHM){LiZe&J*L`>j-ZFm1k_yRJunk={Q=<&3gkkP^Dh4kVohpl9rWQ9~c>|LJ zIiZ>Cot2ww_RfXo`flL90r18fnI@yFb^D#-P40Fm#JVOci3sw|A(*JwgxtrILVj)g ztNX8B|NhR`HEkAJ9ZuAXU7XGs4E8Cm4$#&}UbW2T6$RN1URX1lKghPY=p(v57uueZ z6rXH<4oTY(iLDM=%6F<-o|W3({P-cL173<)4@1i+FSlgW^CIG^M7u6OD{l3TJ}mrU zYeTVpFr`qt#q(6=_r%fuNN0J%FDrY;`_2EWl>NbO_%nnqlCrD!W#)eHg{wce5C5a{ z_eHjnmL8^^kvRTRUa!dF!hYKW;*wJ%;W&ZzFz%_-{K{v5`Wj-xEtYEQ-N*V0Dm|+G zH4@`g_Q}xwjl7e7w@hLF$zsL#+U$GNc?<)9{#dLw&%>W?O@1Efd(RNdRsZOlsZNE| zPl+@g)`1sofMQ`F#d+Y{5 zx=5y;;#;Cbk5i`w#n0~=J4=?91qV_Q*D5?gSSb$va%0nenO+Jr^ISu(iGhWe)MAyS zBYL|%S1(f6+xgS8-+SRb>lq`r{_7Kw`qgz_cMev9Zb1`kB~}L{#xw1?iA4BMccxXB zzoSM>6<*KNG@yz{;NmLZ8e$0%FJg&g!a%Kcr0t#m%iRzUf0R7_X1J~SrwPjWPZzJ| zi|GaL&r5{_z#}=n6vI@2e2*^JvSH#0M*LCi=1tLyK7o*5eb-ABFvZWet$XIbgt7lO zu)WxsY%&WSZ=ix;uGz(v;2DiIxP9elx6r4qrG=z{BFSb)8-v;JiF%dE7!HiQUPRwI zaZ7mQR=HDT?cf9@S=7;urN=(U*q7I$$Mbp_&!t(Q4r`P&B{p43{8Ym}R`jZ!+g9n+ ziLu?vr+Pw%8c;)VKumMWWfZM+ReM7aa}@({70Qmnk?Mw+j{gi_+?F0Jkp= z?#rQwCx}_$MErbg*gIq8V}hkva*EJX%9J}En45TT(uBCnJgud-2P2yzVLuPnm|fj*B1^0q z8d6&ixZw zWdL4@cXQ=VcdHrKwznKgzxvC5iB~*mBc#%cr*;MKcKJj`ckj-%Pg?vQfl8SP$un1{ zYB@i$C5VkoTrgtT;!d6IX!!=|?lYI_AZWj!If;sl&I<4Ud}5D3;n%|f?k^vOi*Gxz z5oY!lUKRK3k(M@c{n6{QL+@~U+0BUXpMKbfnY?(aC9o6!SOlOusVnzidIlc?rf^5M zQ49nr){?$o{7xrk+dXyPC~5N#uG9w-;%OFuDb-k+@b;IcTL=7GvojSRtqw^`TAxTK z$ShjT#+bEu)(2;mK6%!fIH^&XbI_ffto7LWeJO*^!vaf_<22u?m)Y*>T}_m0Qs z%8m7~+Kx27tA|}#S>@)PzN_#(`5|pzP6gDF9TwOqNJ;tXrub*xHC27ZFsTLmcdZg( z6mJgwaY9IA(`VTjpJ2Og`M9l%BEXc)<-iLtk<|8n2S9)%JAz4&WK5fm!?IV>tJ2Z! z5dfb2lo_jd4xLujb3GyZvZ8MKE7EoYuewuu>;mVj>rK7eT~^XjTnTAdYy2!LR{hqM zA2>EQ-|d{`>ePTzeJYe8M_E;lOQzMl-;Fpf=I^rLgC3;7*G06Y|BXUOdZyO6)vqn46(I0x#;gL9bufOWKe0U~ z0nYTiD{8L(t{jvO9xh=wEw0WkI$E~;IN-jIqDUb)^4FV!o?mTLjjMNdR=|GwmgR=* zhwTZkv!`D*P~P_FBXYOHcUR}JK#M2Z}E1bqbSE86Lo9_~IAg)KsV6h!Y-gK93wlJ-1y?&2t zw>}5x$J@ye-+{vvz_B+fGlEo4XUn@s)6qU(X&JaOi^w`Mr)&*52_*Jwy*Bn|U2=9> z3+YF$hRljxal1sxlF|e6 z!XoEj*)v_%{RUke|9g4-f`};&pcvki+qUI9+OLCwL+;guF}Xsy2nTnY9iPO6?7#1!dX4)WgO#x8 z9f_sOXi{ZNd7BKuv2avS1+8O*-ghp>%1UvzdFg1xLnQ zJ$@Xd2mQNrJ~pPMnHPN(gH7&zgrbX_w>6A%8c9-9HfR~>WbR2N7nSq~G-uh{#n*IvBdfOIjb8Hyy_?e1>h5Ohy zyo`6et>A$vJUnBTv9KAIz7s|0th`n30k_PbHMY?S|JNyPo`1fCxs%v`*3FVNI47`xkaX5>AstcBC5O^yVY14clV#hVG+xn#=@10wqRIymYVy;;3Thva{#P+1$w+1xg~f` zdf5M5218^Q;t1o4ssC3|eV@CR%lvsS_uGWy#4s*=MFayLQrOPZ)Mc%hHMz0#@Xt?IqM8+>758)tbpHiW% za6;Z3xm#{+G)JEFK=PDPdMz}3S~V7hcXO!LFairxQ&d`vA{DS|5O1 zQ7wz$yfWWt@C%o%k9RjsGt$<{{I(ocvQkG~;$Bs~?TJ7?_@vq-EwzDCkLpZ~;7Aoa zEmf>R-}H*lSG?uscSFXjOx=zw2quGQST~qDV&;ABx4I!}wxX9>G z^0;Q_?akdORfu-1`;g|8(zMtwDUDr{OU1_|sv zDdlvR^T^6jtcWeI?G?n>i^jIm@M|xp-u8gEzc@{(C!b)fHKAlMgIw;S5T*i-%PFYc z#%LElNNqhUR@4$NNhi`C7MwaMYy&2@O@dOZ>`h}0g4RBo`U)-eimvX;q8Pq#p;f_X zy$arySjl|(L&UJ_Qy#4DKw&ZlDi!97wpfqZ@lg5u-m8g!bR33ezx+tb-Dy#F$_JUs zt&rqIY)SsSm4&gdl&9h=e%QML>q-r_}ixHD!sFdV?XJ%PrmjoL)Y{ztk%85F{mSS zqi|}_lsiB1B9|cJ0NA=?M}G=7++-J>EWRtk;&y^ZAzKQ4>ub7A61W&I2lO#!7k_5` zvBOAN*MQ4q66JAS`J>_H$994$u7N)fSR;H7TO9Ax-hIL9)2zKt+k2}6feC6|_pxWm z26)zqum#;+?&~0$7>-CP-0~1T+)|^7K(@oUUppDG#`qi0D6I)pbtgh)hq-tRPy&8~ zysmxHi4(`MHoF%2En;YrX%!n4Rl_^yhPUQJ1a*fKaaJURKnUa)LzhX1jF3W{(mG^> zOBL{TP$lpTf7CxL$T%WA;rOgJf%*nK>?FrGK&ZV<}1ge{Rv@PxExD;cvy>DO>LMcCAFDuEw75#+= z)sAO@9BNA4?TpZ2Tuj~3YqEIif#O~9reyI6iISXUd;XWh-#>_UYmVN#v2a=t%cOde zh1n-x%4{W3AHN38#AZ=fmU%$!d}|jMh&06}T?@XmG>CFBl8-4QygZT*By+jF<^0A; zx#8~eHplM;4>yT1mi9bd`RiENp7SP;-#FKp1&59| zd1nl|=lP+{pAJx6bh6sN)($!Y>HvLoskvMwOthPt&oSbAMKyVDk?7`D;!kjS(v@UF zO#OHO6|%ZssRS)(waqt;CN>O`S9Zf#M(BdtuFV~(KLP<3pO=b!ofG}^ySY1=QG<6P z_k*gxX?DzOid2%|DR)P}IWcZw&M*=(O|@6KEcF8d(DgFb9+NfH6Yy8vopwL=2iC#W z1q0+iKrg3~-VM@L377ak+bT3@bu9SsBG?tHy)}Orb0nlQJM9ME$#eay0!??#6&&_@ zdR&|6t%n46qVDbbJn&!sY{Waj1Q}Q2S_e2on4SHHKTHZ2>^`1x&hPDtHO3vb%a}b~!P0#C zZ`;kDY|;p}ZqiMk2!=bx(QXupRslb^+6bR`R^@6FWA@NxJ5Y)m6tOs(IR>`x)_XF& ztGDNK4O#0GEHeABc@@fz;t0!yaxW&_guE*PY}Wkye5MSQ@}I-M++k@QK2jeN_$gv9 zJZHHQr?P)u7^WZ73MwdW8GjOd$GSqjbKf_P8a#X{{&Ap6$=Rzznhi6}5uS5r zbwCbnEu>E0)imb8*#RRYD+#%kyt2`UHCU>=U0Ab(>`Us-SV$g?fIPD<{i*BD4Ss({ z3UW7?xnk?PYnln(+3xJStGN3l+eu1TEoLzaEwfufVLj}SVK zU4F5%7!wnPl_%;C=Dz)iLa&0w{z2GRwc$bs(s=E5_J-WgYF%$i5)^X`#xZQzu#T5y zefNp1y@oumPZSwDd}(J2Y2_{;Er5@}V#8A1q;$4dfqRP_Su<_j@2GT>l)vW>R^V5> zI;}^TjG6>CsSl7+k7OHB?uVc;`g#b;pICUbA(PuKv!aJPaZ@-~;1cYzoAwS>e!Zc_ z8^hx13_fH{J}YVg0;tTg3D~x#d9eO2Be0VZQR%dNtX9G(hc2OMzZ^cP#I(0kEXCSE zBwinr3GY;we`v|iC3`I_)Y9`AM^6J)s1T)EFdT|1jc%fh!p#+LI?_*KWlK@ig+>1I zkEU_0rjtbHmW)=*kBD3_Tz2|gYn}$-kUhf|-ixEc+4+#9U>;1j_El|AbVlA*2z?`w zw!+po6Yr+Iow9NxCE0-* ziJHe}nPZIMP||A+^l@{4b?cqCul4U_OX=-9)B*SbRw*NFvbv+uC$Q&F_~n3898=`{ z(xI}rAv8smEt>2-l|_zV@x&U9uUib8X#=%d$Q?N-CLPDUPLo(uz#{^&lVS_u9&E81 zW}5nl6Z4N@hmr#lfN>S`P<)jeV+!4;N#!dmcCMhJx*pm1x%KY|TyGiK3{@3n_j#LK z`XWQqrmt~BsUmx@vng1%Y8BhfQmyu&N1% z5j-_7#`%{3avCb}P3_-R;|&%e0amBk@gP5dn;JnXoF@-M`&Isr$a?RZ@p^$0G@(w} znKTjNITva6jNAK@BWHp)$Mqi30W@{${(&xX%7|3^_qi{V*WEL|iU0Q~^Zx{(?*8wY zfEU3udP9rs?!(dsXlfaktsY?Q*5MJ#SP6JpqdK3!ACQKsFzn;jW3OY|{uk#Lrn{Vq+*W$FVZjmdzju6S$odfXm!;gZ(nmaCk zg>Rp|I{66Fj5w7jM?$iU5{{-U{e95rb*MV@>v3~)L?r$fU2hc>M;o<^ehHA^?ykYz zg9V4+?hb>y4Z+>r22VnO;O;W`Kp1pzcY<4RnZr4!cI|z!|LVTztFG?qdf(M+J@!4M z%A7=X?w9ft&CS9q0%I=52g5M!HJyYpygn=Aumv5NAB^!j@wPK7pzV7A)=nah>H_}M z&95Sar;Q=Kjy+H$0g+(YQ?T?(%MM$wMf5&{NE?ZRIvOkbc6S z^@ZG^((P}(eFFbyAyZrR)&LG4)~`v;|J4dklqA8U&vdkGp3G`Hag`>iAuRNa>QT}# zX4q>|lw^3yE#D^sB1+0v z{#g(cb|?*$p~lY>3p66;5!m(|NN_bB%1@=p2HQknQR*&s{u%3O4q7*?zI?49PpT>y zXd7{@!PJ0HX*#{)0$9=1BP>_1i-ZikDaFR$6`BN7xDO?6vC}RXGsT{^dEn?My{1m@ za|L>TNW^0A|1|rv$K&LyGHwW_$*$#oO5JPf!C78kdlZS60Q+`LC8zuis?o!H<9Ta= z(h;8*fosIwYH<22r=cV=3i4jKDNT29|wsFtJ%>``@E>tO8E(wx?EZZytfcyG1KV!()s zx&b7X3mnFWA(Uj)EGj51fapo7)-F+fQd?`H->4UhuC`A=`z1~^ zTD>bP1*b6CYI?G*{Zuwf436m-Gx(4T z9ZmEI1;<2YI`iEYWKM#+Tw0qp=OZ#(aA@?BQz#VN?}l{+#5$vI)W9)1;(E*c2M!NV z?s$FRq~UTBI&79#(sNR{<)$z9PDH=^4%_!6DASLX%C>?adw#n@y|bc&jc`==Du=+R zclf0{t<7YG8yY88bhT?#RQFo_QWdy^FyG}F0=cF9)qG6uEYGOle9GVy6}mu0x1ivz zQI`;TY-mRU`d(rcvzIuzt(o@MkX(nzVD~3A&FJZ|(Nf(pg@8+Nj@zgdIxF)5JmCws zZ%(t{ZtZ?^Uq4Z_uyFowI@pW7`qZ0*-H9G#8~@h&}Nd4xm9iGMDApSIhO z2xVw8fJF+-W2WDV65kRGQ7dGo`%^7~dOQ&P`iJ$Kr=H)PbZ|IFd1lfRJ+b~4=jxsB zK@kW9X4r2@c|f@+A3RYx#s+B=eC&Nqnb_CCYp--Xy<0!3ImZGut_SJ+g4}`$Kp1%*)qO}!=Bw)D3_?OJB^YGc2X(83`W-JPgf-lhSpdjqV87-nxzIM zqCI!iWd3RmI~V^^b|V)RO5xGhC{=}8svcNjW&D|}sPoj5(u{QctJrlG&`^!b5i?#~ z?+M!P;nfD;)@y@Vdk}?n|BF^TA+n^3Npl=*>orrpDGPUUmQdR1;Q)M$8{vNbJO2ym zRhe#d^&_jjQ_=TkGRpPWj;8WECJj;DIJ5zVs>gd`b}=ibm!_ga$t@x5FVo@rhT^;l z`0?`sjW++~CijJpbAE-q`*E9TUC;F4p@B(e@iN##@04FUz^6&6kWm$FsGjCgo2dk8 z>DhVGeA^r6B+l$Jhw`!xqu)XJK!C^Z=*c9n>1qFbnAOe|I^6`1k*`?z0#RRS#2zUJ z+G!8$DLC6!4Sd^~7}x=ydCC5t>)Hip_#4Aws+KtkscfInlH8d`^=ji$KcE2q^3CYA zHEyRCexXNka~>n4+6wngzrb>S28*|BoE;09RuY$qFmo32r#c-|L{$iQ9SH^c02;x3gT#^=8nYN(@hg2m$qN}VAtep@UtJ5eoVzcl5o zY2t22m92T1*)=0Izj6=B`rt_1WUKqxrC@YhRY|xf9pN3=kqb$W*Y7lW6_wUM@r?%- z^Le2w$!=S%3=RdNw+bi@248dEyS$uZTTycyC*qbP8tp#fbrj_fUk)>!&px9U6^`bI zP~XL=<5LYCnQ6hV!8_LB6Y8_Ec1iz>3!sva)rn|v3;{st)QUJ!a@$1WqgKMmmrktO z)-9_v!yS-VoYi^bf0VMV8{k)Kl*|mnNr(ha=U%^Q0@Uh4j+L zpS%~oYHm0aTG-)q^;>E&Eh28YqV54`BpHerYQUv8SLfXNBGKvIIr-J@y!Cz7zc^%E z!-vX0XT}<0?KJy5lA>3<*|d6iBx!`Q@DyIZM&MQ2a!p}x-9(Xb-LQE?{G4MMF3!pL z4KEljdf_i4Ey%BG{SEfV5zs>+=~2Rtds6C#=lp1f{dHBYZ@kJT25|Tk)p>CC?j`LI z{PBXN{7Nln;4X!Wh5rkB!6ROpeOKFI*5=&uEb;pb?e^%JYmzc9r&M3tVKhmkk~)jyA+s%_{_YKCR; zA0aP{&1T@0LotItXnyhrOUL6l(D8mQ4$^M%fchuPG+GaT4@hTLm2J_0K#OIF-;AF^ zpx@iIH>XWWu{TSvE@LSpy8Y(d0hP2=jqZB@W0*~?q{RL`t%hj%nIIttFC#xIq?%*kIF~BWk{KZy(Avo$N zw2|*NVP?@ija_V@$*aIpX@eL;K5Eg!Is@X{l`w$sU?DlkkVC?>(}2OwcZY0r2wPpb zZ)%bvG+5d`_$4)mtoTz;=aZs{M*gN|B*izHx4+=k%PWyIp|yE&nWG~Sw{oP#W~TNY zF25dXzYf_N0%{1=FaWV+nRhZFh5cm4CSOwK7RE*wbrWx+eLGOPqrUPn?>0cSMIu(o z5bdKoiMUg4-Cs^>J^lKp1I?!g=G@P6<&WPB;@>m#K#~01nga+;ZMd6dYI0^zhg&Sk5G;GGQ z93|za+^%6c2d9#21klF~4T_f{hxN!T7f;@1UyrP->HB_7(2fZ&&6G-yKc$a<x-^wRq zM3Ja|mT;W%zZ*f{lS%sUNnT^v$TemWVJ|~$#tWi37VAM-d*hQ|ElpBd-x$NiU$&>J z3Esj0Qo(GfDCD`LeMua{A${CaQYa@Ci@w3WEzi2ge~AX0gtEU5HP-bk$Y>N&*IBjQ zQ0tk%dB3Tl|7nD7olzPC6bP#qBTRswckYc&%{Bm4Gqn#c`f7J(Z#CeCA#*RBa|6GoA>7Ydm^!m}0 zxHT7f>Zt(Wp162GJ~lAUx(j@C#LKU`yXvnP0<9yuY`njE9@4Zwnw60&sbeNY-J~^^ zLk0iTa*{~SJ}>M8!eq&vSl=c~oDX3_gzYU4F=JL`56&nHqev>E!G$atS?=Zfx&iLx zo1@pnvjWZ*yKI^a|7IJ(lGl(rzW}?w3~d1Fm3}iWk1IEXWDt<)rtv#0S2}#wD_L_v zWP-h#^Q+EG#Xb@6=N=NJQ$X&UxAYi-N~b~hjZIDWjzxQ&s_Apkwe0Q69X#MIqe*hn zQSl^hL@23#>Y?nPGf37<8YEa*3#qkHh_p^(xMD`gV3PQLIZWl=v2OY6v*dBXb%~{b z+Ha5Q0H<#_lqD%4XV&(zn8|CTRtFVIOO3ZIn64dRI~W$K{Nwx9;nC7$@$D`2SkdZ9 zheJ$-n#Nzt>G!Pwgz;U$!)0o_+@`KO16g^aH!CE>Mz1g~7>>V50OU$&dE}YI5g3@zT)%H!@9^CuX>Evog@uY!ZoGa- zZtm`H*gt6Zgp94;{1MMhmeztG-tkLF7m~o5PjH#N@a9Z8GFQq2Ud3{6TVNh;DQ~~$ zd|p_vWih$yvKM@gZUYvzf|?L7d{8DhMCGZ|{;0^@^9dX5@`S;fXSRKrm94*a&o<`h zdogmGt-th0TiMzLoqN-H!H~CfiiI-Y0}#jBt*uy9x{GHfp4W#>Von?m1ui2^+V1^Q z%h;D`FuTG6@cpwPIJO?53@3`>&Q$v6N9k^>>W@G|!p8MJ;{8LOk>&iQC63(PQqac{dm30lmLwc_&$U znCkzfC$L+>$`I0oYJzIRS4RT64B!0>7Bh?yt)f}R*5DTJ9 zJZ^2VP`7s@SD8Ev!J*`1^RGDb;0um84YmT5F4~$eLfSEnA#R)~a>5dZBk4sqjnxRb z8z(e^#!2G`(l?~{Z?O#tnsIvbT1)(rDsYml^8k_`kJ&q}^f{z#mZ$RaOUvB-Ny$$+ zdz=|hMMkmTr(2#0VZW^Qla4&~Fr+svO6Ku~ z#mg&)WAJ11#&cKp+1uoy0raT@s4 zF#Op0Nx)3x-|sFUp|C&ZxDyMkeh3{^cBR68 zPn?p!eD9z=q8sZ&4AlC^GkOQ#A9C3J_?kh2fXa^faX?Vu4#8gsY@LFx-_5aI;;E;1 z;G9PAIUMPdB}1-%`SSwZtQGfFm-y3MCx*ox6|CJlZD-p_`k`mV|TOZ)y!fjXE=vn&h!z}Q3 z&p17&Q%ZGw)uvzDu6!8C!m8(rD4=P6RNab_s^ulkSPvv>MI+q*ZRu;;9#$%aa%%af z%RCLuq&(~3v_4x_%(>{R>MpvhvH7v2^VID*^}0{T3m)?IH2@|8!6 z0SD(Rkq8_V%CAN5-c`1reLrVhF4S^}Hf_uR^)&6Bcbu?l6X+dXrpLI;*qQeR#B(=$ z5w^oY(>Lu2+Cat>&2Z{~!cmQ!Oe)WNRpPN>6;Yz&n57Os(%2V|$?2!9Zl($}qi!;KyhTmTnr-zzDjmDDKVpuP&Vm^0^Xz!Uh3 zDEFri)_eC=wtPOR8>6h&2w^s+przZa!nYuNScO(&eHKkKtRt7iX{$zh`#%8YZ{#3wv4XRm-=hQI^D_t(p)}O;bs@jmY4V}Ck=mV_7mgfMC!VjBDAv&=z# z<84{}+V~*;)3*%@F-wqPA@hyynnTOEUi9TAs@`8Jf`#K?e%zVIzvc3RSk1TBgsdgC zEXFQmG-2Jo57j~9F#Gr*$~QTU!Z2o=g}x$VYmw?agCfXP49 zq>dCX*gQQYD5rGCb;p15T7cu~w^p@^>9ifVAXES}q;daVt? zj$BxcR3dOeuq-Gfml(_Ibz_|FX4Y1*Mu7Y>H1dtzj>gaF!JWX$s0TXZ8GW0aSrk{^ zJAf?B1ZT0Y(K&s)^X!igiqaCoj_Dplw!A~u?1@3EvaGpk`)pP*uj>Q?mMkZ`LpZWVGc^ z39f;!oK5KTKR02dN&zkY|1!nW2P@r}+~R`>TGjxe3Ds_Y`y+qkL+&Vk9sLtxmsd?I z*4q?d8PP)hr6t+&=_GAOsM|uBU+zoFLxtOGjOzAE9yLsFSS`?|sRau*FD7@R^fcXs zDqJ-9`7iF$!8aOJ+2210p8529AL+vRyd_~zj5(!5X_e8)AVqXr@s__O8C5P;S{uSg zs>&CeEgRGqPBHD}L~iciL5vcghv$fK~2beWZg;}U36=lZl(5;1QmT=@FAYQ)g8QwdK2m4VL zRGqxMho$iqvc+K^5SRT_+SV5;L+(6UIX7a?oR}$HzOT_;I+R!hkGGYWr4Ot8PXvjS z;a)HU`t#W<%^O73O(bmQqy7I$se+M7CO5oO8_2j%bd_1Q^GzkP?7#c*OgD-0jmxdR zN@p#@=^)43`8E8&mn|MtG1V|i46U1=cBaO;VbW-SpxxwjRC!yFgUh;JvGj%WKl@Gt zUYLTMo3~^&GO&ncZu$*gqs2zUFzkohyzw{aP+cWB$Nhwxj!8Gd`f{}v9}>OnfAKeT z)EK}{-C_AF~1WO2hfpXmyF)4;&jYsY8NV*d9WVtF0SRG70UD_2G` z;I+rIa3f~geJ48MG_h%15?-d4u~iplU2fAh_pr-s(?Asx^Y?UE;%?Z-$!R0cksUSxuhPCbDd9 z&l#r+&j@th9rW3i=bVr;EM5`ygaIn+`lrdmK&JHAivLXJ(3rj9$k|8z7WTldM*Uwe z{3|t`pox8tb=U6vM@E-{y;xpG(d1%~gczMNMx+apePi4gKB!e;(q&P~g zc^6(c49wx4`yD}Fl%f14i}dlU=fu~m@Wg@SaX{O;w7f}}<$~D{(hmZ#WE?kjmnXXQ z$2~J{-GdTvNd`vij~#v$uTwgNWHXoi|3Bgiq3u;5SgQ&WMw)q?4_IRH}i@8)9#FT=c!f z6U46uj*!3Gx-wk+7XXNeWA<~yU=JMN^D3}@m{=HTl8JJ};oy%nQM2Ghb!TSyO{syL zLalVI$Z8#(WoZ~7UMX>EuJxu6jeZ1Ca6o8Axb08bV^&h=9}p#Nw04+uX*zUdYI3bx zYC#`;dX$ae*yARQZPm_#o=_>i66`THMm5%b%U}}0SuwiU<|sP6DknUWW7;yklwd^; z^eiqjT!?KCZPF00s^MY$iJwNDI<@DPEM~L1V_X;&Oe^G2+2XEr1U=hRgJ~!OR7at< z$(;s#mW5stT_jIy)~vkJVgybB@paFG$nv0VNoj}UwyFU%j<=MhC3gwhl#>_ZvKoIh z3L@IseI-|!p`H2MserIbWsep54R3;G(dcu~y6B znQGwQlGeRfy!x-vUU1=I0|-Exl_Q&;3i*D;wEDKeDjRivOs-u-I3JA1@i4^M z872TzfE9eV1g}kpe-Zd|1!K7F{ImU{!!B4Cq4=fD5S!1AXd`aR=ma91f4`fm6rw6|CaGONOuN888YaL$I?qqK zNj-x%^GDR`d^|`+Z2f`PK3I$(c-Z_1_~+cS@Yj7V{4{YXZYphbOr>lt+A-)D)9_Hz z@Ub4{M`aO0CEKI{{)55~E2{DQ_EWXrB}ZyParEeu=%9~{sFu86$0C0hgh~=*77P!w zi_|+})p=sM-C(rRxJ>Jcup2bvu*QGSlSaq{zL^C*tCF93Z%$-yKu+fVoZTk)1AdND zMOy$Elg1K;Ca>t+mpp4Nu4Ys~PkRE!*NdX%f^i_>e`Kd$o+Mncq(9sI=xf4OStho9 z<A!EjRO%|RO;)T4Uw*0Uz>L=Kb&$S zi_lH~Z9>Lz6rxjafg_#2ipLhD)9H(p$F0aG_Quu+OY8mHn99E=`MN%>AVi0K^;4mP zy9tui@2U3|8JeEdKmS0Y+&L^gN`^QlXx@%vjR)?{nJzLvtd@W8`EyY~Byx1YCnoJe z$Ts2k#_%9qunPE>xT;8Fu!zwtBtZ;OMUhIf7Qv@0u70}6vQHRJv0^(|9|S;6UgIN$ z@=)yhC`cn9}jhyk{hhCBXjQ0IDghzS)l;}GW`Dg+0qAF6M3P%xq z{?MW^=vIr=7l2AJ4G*(v{SYdTagwXqYh`P6VMVvO=x#l3W zBY>iJII(SRG}S^L zs%yb_joBwdfok1JZACa&*?+F-8l2nUbcPHF*Shb@)`uj8edIQ^FPlT_tt^>ipjNH#5W|qUt&Y;`Fujr~R3N95Ytbx3VpwzG-HiKGP z-Nw?n_k#29VMkJLcuBUZLB!#hfYohBt=n0ZEO9aE@bD^Mk|EIdm{t=e;@aInPwSGd=fXa*2?kacaNaUz2a#!ho zT!?2a;n>vwqt=2eq)Eppwf+7Rc>!rGjWF&U1##|?jFtz!oDkjgW>P;_W7wx8dC}yi zxwsTDT*$uKo9{zlT)9%uFs~B0zgVq1F3MOW&DJ>NFn+-@Z9pAN5NeIW zU5T8ucj3i&=68J#YH>gV)U1V~m9h1w>=L7^CidYx)Zg3h0aq^%ChN_4FFOfZp?~#SWch-*s8dSLjR8aI?8 z-s05`-2)^cj+?|wYTAV=qS2e8{PwlZt&|O$(X@J2BRxM!M4FWvaq%s^8^$3w5JGAN zcfN1TI_;0I^x5jV#6m7CCwA%0{a;*w&q?>=Jn$TMQQJLC=##Zr(rD8UnsS~cmC87p za@G8c3vy+HRx+)HlfiqxNu|L^3nGiDpjCEBkHj_deEedzh8P2mb>M**O=9_d;A>{9 z4aev~cPMkvF`R0tNGPeH#-r!6J#kVpmiPmH=Pk>S`@&+n`BeGP1dVua!rs7(#PH)I z=8ItY8Sk3y|5^P0_jNSY|6c~*LM+5%LO!eWxd#EU4_?^!9ey-0v2smq` zeEc0yvRy;7lvL_PQiC%Dx|qBi=9jMXeVo=8uSDCK6sGLiiHxuZnq|| z%lifjP=~|EPeDEJh@b(KIgV#?3kWdxS6HpZ{^J_ZVaT-W-zA4vfluZv*+iu@Nqp!n zJa6Zt5`41lM-18oMdWu#mm=B@u0&syeA3|XsC}2jmZdhXN?PE`ZedKFFs{ik*s9MD z{?P|Y9-7$5rrcd`Qe}oPs(RQsKSsg{Rb4El^Xc19LM=GA zJdKhBUUtyvdlkpBPH*I;J(#sIHyw{$?!gjqH?zdU%v~lRC|4lA4wCbH1}x&;=D?YIcyfWW#aPkcASpu z`d{!6RV``Uo-aBAC5FYzKY=d6gRlAAEeNG+NjnO|HXcky@)k3P1NHXba8^o(>MXpfd#_upo?7sWec)BvV2*K*V1t?%jhQBee0$+#&+S9&WBb)51@ zhqDZryFQCY7}!!pLMDaZE1ijd=q#g(p5ujV-i)^XVA!EO`;6R)Bx5IWnqeVKl3}Hh zPkt}F-W5f%Bfov!UeI>*oSv~=L|CsX%VubW@QLzkng*$w@1_u#fN}MPxU_mL$)M85 zd^&fn?c0_ku|FI?!5@fP6LjdUY}AOtJMYxTTjbp5EjTH|<0*;`Mru()vLlOnq_|&m z42ERP8F5?eV2UaH$4af3A5LFaq@eL;xJ?VG?X>@o27r+9zjtP=i(& zaW2c&bNMxQxTo)s9;oLVQBABi`yS&Dq8QsN*H*-a7CvU$vodQ zZs%plEhO=wx>Y>v@c7QMzhkZK;3c!|YP8kuCQQ(MF5PvrA<}hwTh?_eE%JP`0_lA7 z){pZi*A-%>#!>4Qktq^57|ttNUr5q<>}Fg_p`l>|8!i&Qu%}7NDIxP=jUi4GaUU(^uHK6auy2i8)Gu_CW`XPkJ4xrPk4H5K*ZB?85pSav0VOO{+v%V z7-Se@m90DHEM&873^lArP-EYSH7ezJU)_x7^RDYa%bmPcfn1Ls_GTFNAFo5Yv!$dc z8!Xb_^hu<&b}d$3_p~>hs%l_J%S`UzK9=~YGpFrn54_p8N5*A!iYeOeKsM3!6vPMc zK&IEUBoOVl9}>PB?}c&-qHzme#;XX~KJtzj`(w#T@Q@uy+A4-I*KWnV+3ho^b%iQD0IfNs#Cbc=Vu0*I zGsnHLSmL21m2c;*Wgp~N{qJWCi`s-JFJ2CiD3+(d>Ci9!&WlY(M;!4jrv!8Ol1-Z` zYg8^$z8{=k*{p-GQ7|adfVIW^85Z2X_*A-+@5i~7btgX1G$+Cpyrc)*PiH3%+`93A zzlDB@+_?tA4QkFL3d+oqhj)(Vvb^~R3z6hHJzN6!-fCLVY5$yBSI8(?i^cHrDd=Bv zX4`9y#8ySwMsFnAYlsqIS487k06I=9freo~9s#IWbwJ0HaQiVbcu)t;w9h7fz288s zCv?;(L|))MRzMet9@HOAk3E1$PyAY7cruvaVC`Ss@4dj&vd`JpXt^>A`dXS+?2@Lyd+q; zxQ95pE0+>r*zf(8l6_6l{;2xR?Ne#3h4W$**?}nQ4XXvShp?PZr!122=s!N%s6^PL zI1yRvDfZC$e(mixbi)*L-^yr|5OHTCREPicbtv`A(tp+v>~z<=BM8hJL0Rd7Hx|V{Gb4_v4Dl zFYwnW&icg-;7~~jXP{O2W_(h`!rIR0edSIL4rcV~rbbs2u+ncKh{~2#@)A1x{kpLz zB8;p|iK&cfKA9nRl8e)OW-w51b)>NeaZnl5EF$lT%kXF3xIRkXy*o#s^UW6PPLN_hsBBz zy`Z28ob}Cmn{Cy(EF&05`ui_xlWljQzd%`|KqCG;E9*W@gxPT z1>>QXE33$wPAURk8;|0fc6OpPXsYtmdkaY{PI7(leN>;2e8$B|I{}!GC=yD_?shzt zhxbuJ6fwhJi8G0#d|hkJceKg;_Y->y4+lcZxlaQ4(4qfAV+91(eaLU2UiBU>lvrzv z3`^CLj`W&x1dP$IYE{7RC}aS@WotQOu^lCywZjwz-JidN$7LXoEwI%Z8`g1|9`{ZM zp1Bc01W5XrK9vunNj7Qd+Ski`?u5KWnev13D*@M~-R3k-uQi&P8r#lJPWEQ$F-ORr z9(H{PwqlZ$sFoxxrA(tABH>!jK-?0h&$vUH`?Bx@lT(2=n~xY}e=hSU@qB-rAi-Bs zeR>Rq`)>p?`Fu_=N$&X&qI2TtaTx2OI8B5JUJ?QTvEJ7M?^YGf0)|g^t_DWTe%=od z5?7>?AAy7BPzS1L?MtU|DMHhw;RLlNg2XET49J0}YOnV}KI+Cud^2n8PF@;smIdVV7=`#AvAxSeD&p#H7T z(2sFzkk06=mmw|8i%hND1UZplbv4$D9_U1>a-2yr*3G zkIk?Lsh}*MVb^)pb)f!GK>6{>5(@XApoE(y1R~gW+<*<$^ z&jBLx-|kKtxzT-#^C_)q?d_Rb|HGR<=`A*M8C*>?v%0DnZ+oFCvN+l|0W|=~YI1tM zoCQaXIxXRTD9d%R{-RPpb6!^+KQcF?Y;@sM7TU6YY_d4jR+3qW7wU!gP zhuwLdxj{I3tC2h$BmWIhg+7x?e4c8B=C#s!4*g?D%YCqw@-jksT9 ziMW&%X1WqsJuA)wU&YvrADXe}v1&|`qXk|1W}Q2^{a`K{0pL0y9tZe{W6`v7u_n5p z!P*w<51&>4h6F@wtYvc)OV3K7gCx;j?%ghqCo6HT!+`yr8l)?8h0As5(<)D%?iztU zhx3!LT09rA368HhBQLS%%;>$-p(agY4)^o&O0aB14TIEemPMH%mi@ruE7-mx80>*p zuv?fes>=efEEsf$UR(^#5G*Qf-#c#3!X`tvtS$VIruc2(R!6~bA)`-)vcyNK@ov)d z@pj~Ms07jZga{s$xF5&s=X17aoAvg4$SUt;Uzg`4q`h_XR>Vba)+=%_EizE#BSe+P zb&zXsSnIO2{Y?}MIUgJXzp%53Fvefd3xmpuZ$9YGmW#)ybrsw2>v(gZMCCVlG--0J zkr(RU)Al@)e)U7&JbW`_JouYF_#w^OgfB;a7TTwPZy{Wr)-5DD7dl2&5dT0`-3uEN z+%6{|yBNIk26`MCLLZN9+m6-`W0swz?tlLbup4ymPPyxZAC$ia+e|pOoo40XaHrQj z^Tpj#ckvuLb%~qH?l6obx0Iu{3idRv^HH3JAcoRP;ZBinw1 zmc!m3ka&~GRL8cf<&z<4OICFO09M`czk`2B7`I>~;9;|U)0kLav#9A}HMv!${)VqH zxds|1q~&Z!Uvnh!bIDXFiuFIplL`- zrWBOwBTSU?P#yIEpN*}p9F1z?^E)K~WpW8`P@-B{K@?b37=d$D91|=uI=EqLmMC~& z8ZYA7p^qQ)u2XiT-O`fFH1Ho=7q6>mVWA(D(@Gk-Ud8q{07sZLM1-Siv-|DU*4h`g z-t4sASB$?0kZ9B*!Z`YsfK2)WZX5hM&o=wa_a1VL^AlvTdySCI#&X;3@sAvR<3xq1 zyoWUAHr7ADu?y$6saz(%oY5i&%|!!LL|#QNt=S4bqmCUk?Z6-+7k7STTgh4+Ln_$? z2=Pn9KJP3zuaSBfHZvqqW z8->w-=!TgtJc)?NiXSUtoX6znkuCnHFmAhE2gU0hY=ok5-dCs4(r<0kGb;>F=Qbxl z?Fn`>5{^@jPT|r{zyFz&Dm3Ff%z$@bbq1-dH-8`$?FqmmWZ#I$&T<*V3pQfA9h591 zD4mW2&9Uv0#8Sa3Y0Y=B6?9U?_Ka6w{#63E)A@luOFnGI9T;ky8=+-cPJ>l%J%Ej` znhexYa^yEHz_-_@f(wJnT_iT7NZP6#VF4OM+9+-m4$(X9&EGS#ns4ZqvJ~qA5Pf#- zQmb;_0hl&;vIF)F84@mx+_rD-HgihyE@o#`fG$hAU3J#HruUY3g)bg-gd8goNMnLq zu>^USvsWrWzfp$-(LE>qAOh~nxSf0!O87BuR~?X|@X1X$H*Gv-&SPpowFZEv+KMnQ zhATtgG(&yqyrP4ch{|I%Ij}bQiqZ&P40)(*vFQ8b36yqHCNJGW!k7d`OYVNbA!J_(wbec8(|YUq4hcHz%`fFd8-eVUruQGcULMIc zuI1DG_=W;N02GblH*6pj=TOyH*j}6x4n!iX52~q}o3uIp6=dqS;We8039$503O&z; z@6*b~%2V0)5U1ew-q5kzLIzTpeFthp$;*kzydi-G^@21cT%XXrGA_!C?9A^BL$uox zjnLz{XEA{&+i5udtsOfoA-g>JGp^s}B}Q1Ynoy?eGCV8M6aGC_63Rhy(z@M;RJ$F? zkazJ+M0UA2cK(BA2!ny4o-66ehWsFtYyZfoeiP;me0=MI_8L`$&SzH4+sA)?PI@9p zxIiP$b?qzB+Bcta9a$FFo**OZNZ z?V%MBeM8)B=ixe_7E7#8flWw^8@v>r68cePM76_cN((zV49E2sFz+^-E83;s*!)Xp zfCvo3YCUVbP^V}2F=U3T&EbY=!PU@M*B69%ano>HiItE8BG-P;BG*Bsc)_ifQM9SS z$wvY{hDtl}C(EuT4}`)8OMX+s9+XLrrI>>(m!Y2}MwkEMP8+tuH}*6bz&*ga=TZ6}DjG zRKABf$_>(EUN;;jIP8qo^kBxnMd!vlqz#j`ob?|b7;BG3a3~eYLSNVJj=h_sg#cpI zr8;Lq!3!6y+N-1ex7R@Uz~sdZrTK!(=BO-Zzxi%-YKE|xL{<>xqO0Osa0mSV?$^)v z0aFis_CZ~RJa~W>_`g9$ot|GqU0p0;F0B2gm>QF^Qzf_#alRz0gimdTOQ58w~@S&k)XX|= zNK*k)bI6gk8FONqNjYsud{0h?#yb6$w2R|n$+oJ1q&Ofi&mVoQN)djUogd9RDTFd~ z3mAMM@0ZiK%3U%w9Q3F{R2yZZHT}vHY`?PlH%TgeB`U?~llqGj8G}%4gRyVFH;#9@ z)c45z3T$SaPL(RPhl1}#VOzuQ+aO#HVK82{@nYn3!O2)!3c7_ve98{pe(A1|h9HBL zJsAPJJ@nHO#wl*|6`DWS)Kl5gqla}wJqtv@Fm*q&Cl3(nJ;wVG! zMd`w|`v(+@<+q-%Rfm^|D3Xs9($Ez7gk(ocTpCtz8kgtx{}bfg2{MlRqcJhY5wXgG zE>b+aHc9u_cFz^pM%!Wdj-FRZ zBA^X+H2JUb~qUyBrHVlSXo%wf#MONA=P`jkm0=ujr(ZheoZa3`d?h||9a5Hg5 z{68~6G=VEATpcp@Dr|WIkRlm#9GW+}(~CgU>+Of<$BA;_-_t!%*Edc|6Yxht@fSkU zY8G7Wn|2O#h;+Xq&9=?duI)4k`eU>Vx2QrQb1S#bfKj3M>c^w@i%Nev;;2Eh(bV7O zpM9>fbi6xR>9MSmos(7d4W7$nbQ#tp-D6?^OF`!Fe(1oSh`?Nm|xBp{XNbgpdlK z`OD&cx^u=je2$%rMvkF9!PQ7smoB9vTQ;Mknj(x7&%+&Bkd<;5E;xXX%XOi0j!w-Y zd$%JXawHzL!H2q}0~lVv*QpRH-eD#mDovHbCp-qr~j)u5lC zc*1@T%F&FsZ{`QA$pzbH>)N(nY>5dBIulIlTV;b=G-b*j=k- zgdizSCxPwT_gf8^q|bNn0}4gDYl3$geuUm@=Pdl(zWw_D@by+fafM44Xo3U_GPuhy zI0OcQySuwPB=}%~;O;hr1a}YaPSC+6xCIS^YjC@qhrjN_J?B=xOnpzgYR~Skd-dwI zG}!g4twm-eGBcPnZKNEXx;`Tl+$554bb_Jm>~_9ArH8thNWrCzNdqIyo}*nYmvnWC7>x>?mdNe5t_EA>5WuG}u_jzwJoVsXcWj`ZoNkibZju}w-Ql+u z?0aL>hw=PHIunp(@{>iGKh0VD#3kI93;Lml&b(A|s#`S0&dG;0XBmmpm)6Ln?I_W7R{QgyMLOa=hm z9Rpe|0tF}fgp6b=6d}_!!Yeh1;Y-ID*dnGi7Y$c}uJ#&J&-!CpzbZ;^6sJ%Lrc-|K zCoO$u7_>oQB8oK8DyOu{7`FylPvS(^Y)0TudH!%A>9@whP%|LPjLyheC=QG*GJ_)0 zkbp)-nFVolEQkzA4nZahoiv$%d|Sum(W4u--N0f}k5~W0p=Vh4PskfIBL>>ff-hwCh>|}EKY8c=x9R?K>sX+)MuSBd5&Dc^oau^c za(k#HD8qVn23hKn9H=96q5auf89AWeY~D`)+rKfbxMgoCwxWr~N?hM;P5fx@rT4!w zZRfS2b8nlqc@QA}Out)X$VXch>w1Fx2JVR)_`gJ>Lgk7LLuGM|m`L%KIKv-EJ{Df0 z7RI0^f8xkpE4HqX;D)rM9|7ZD$IFeh!7| zhYDRY*?+Kn#|NJcajZ`M)UWVMx3JS34${rKj3;{||Lm&7#{;6){?kk z2Ma{>`Q_(ef@LWWCLOCy9B<6SoAX~^@vi-(hYZJZtA>abF(F(R{1eKfp8#8_mysE| z{kW|w8tv5cqnI5DEsiy-Jhh_<^j@YTz)i45(4r+7*7 z({{G1?km{nbKd_zM(0V>W8nD1n$;@{jQdYV<;3`{0A-|d_wv*LCYE~)HUBB`fyj^m zUh+lT|8W82$?+y#+0DP#3eAF3pjod^Yv6*13JY|Lyo7OIT+}E-2lz{nWu#a%+S$fBOE~UcpN?3hs0Yg zhcAYnNoIFaKCUD5;q>$|3)ih&P7UJl7oY@!Y*XBfoQARq-VD*#7yabfNI!3;9YlTq zEOJ)LGtp!9L=?0C{36?u!B_PZo?B;Phj5D0h)EMnAQ*xzYRp^+j&cy`TE^%&|09cd z|D;~Mec$?GS^A)Ill8yc8nG!5J*q;T!Wzow`_ILPGEd@v_cLLkY`MRLH9M!5My%Wh z%v||jKJe2}w8#xh-xJR{I~n17fsjgp+~l&eiB0PG+|(%rAh}TTf~|$izg!K{*SN^) zK)O^i#gtTCD64`EgU05-fHm_$>2p$;Nv-qKi8IF)gg%LyKhTrVvn|#%$K%z^VPCmo z?Uvws^-^r=u(UO9R`fUOq|v4%m)wJBDS6GdSinA1NLJIjHsXESXPP9-XR#HLnrAp4 zBOUuNWzkp5?)n_Z+CQw3<%VoKKczjH>*TUYqdur-YVa$zueOM9F(tSZM~7=6_jTZ= zkD)uXV>Hd7%{WN*7EAnm0~?7favM|02g?@~Xna?9RHP%V6_?fxEH_|gr&HvnE0-PC zxsorm1`A~k)Hq;a4&(R6WAwS1KXh3sNB1g!vuiS|0H+agy6e@2zWeYexuGfL?}sQ_ z<|TZd&jE%|WgIPB*-B3uVF}6=d5btJ=HcA@#S|{GbUQ((aE>uTb$_VVb|$L8F)ANS z@!7FJU7}znSf(2T(~~6_L|>-%(Ez|LOY}~l#-mQ5r!wdbJU_WF~IkuF=axK*CB1l zBDjcYTKNDRX3WoKIB+dDxBf$t>7zB8RHFl0;a!Lfu{7F^$w>C@KT^fDedQ1-i}2=u z!?B`D{^xn1FahMVMR0l-r=sy~IlQ!K<~L?9S!nwM8e?1z{(wSc)5V_KP?FK(I?d zlkF`+lk4qI6C|IgZB7!+xDAoNuIuK@`FfKBj32$w_s@P(hQ=QQNn~AP(QUR=4_LKR z#;oD*Fn*l8A^2FVKt0%3*1>p)r$ZVQiajdVxATrFA%iM9KR$Kb*DtVdd#DR&DWaZ6xA67H)inpgZn{(av3;etAU9r`+;Y_MQtlMz>@|8HeRsQw2CzwuRu*4i1M!q{QF;0{;ng$)-ULJCZTC| zVbNxtkvhGAE`~QWV-=1 zgv$FLhOgsl*;FWt4WZtBap-U3KSzgh)b`#Dc554N*v_>ucU*g-(u8}7{!q37IqEAa zL-)p^nkuVh+hyLgCvV@?mLEixzes<)ll}<8AkH4Kl#1p~Ad}PMt#^5kyYT3X7No=_ z<5KW`4lcqwrKOD@mMoP>_Bpf$Ui|jzA$WW20SL=!$;U&Ys*@Mv81PI%D#;Y>krF@J ziL)Z`YzsC#@qc6b_V#v^pyg$n$>Nl%Abn{BowAkeN7KB8BU?r>mI+mNS(E3BKf&Qw z3sb;gMFD5EeFlz~|IE1MU}o6uLxic>-E!q(3!>M3QS(QLZ{o|LpXU1E%|n9W+LyM| z{|*W~w{ob##M^$oKf9aPatgQ>Emh6TGwj%@K& ztG(zFzCHs1xFBzT`_227K2O)6j9{_XvduFUfqr_r&0C(UY#QH||K4ZgXw zwcetpRet8AqfD=^oduLzD;>H@nf12&2K_X?8&16|N3vD| zOG-J=mH$WCn6{tzMdY^7svFMuj*HQ%X8AqJypB$ZeX>FLooO*L5I+$T_;o!!Q57?lYqPTtpu0uy@FIN$c z_H;j$zoi-p8b#HaY%SuUmnrb=D<%ouq_(wfSStg%8}6A_f8EK76#fa`)X==#INn4B~06rAQ6l4~(-NFiKI zJKzVciT%G;v_pA+0xWcpzpnuM8?x+rs$IvuHns;+yM4I`JG%|^iw*Uqp3=868Yig5 zvt)XeX&hz-#2kN9`!BJRC*0G&DO#jcANyK)LUn}L7Um+P2b=S?9OQ#jXt!vsx(VUy zPh#4xMh=|sJX!_RU%P3NBq(lW!?r^Fdcd~A@DMlK)U5AqLzqnBToHP~-~V|NSL|c0 zsVSgdv_Gy%`~X0z)iO{&g;89zm`AyZ>FeXs_NVo#0U$ZkZ>5LZoFWB?{Vw)f_R0^D zPqiLo_8QE=gL%QF+ES}*G@B{E*73%)PhL7r=PH65x zx$x4tu*4Y}bajJODAIO*3u{M4rN#EBv@Vp+$RU0)t0q?G7E(@})~V30V(_S8UUFem zwnOJF0gd#5-TE<~+l< zd`noNhD|K|m0Czpd}|)YdI^3fPFReL1lC~Sic=uYFdF-L0hA*8lDlq1C) z>YOOsKM72opx!6}fb!^gsz`=AD3e?{GLdSy6pmC^z&> zXui7+PxIKNc58!HraFmgV=j)UzlNMPP49c$GEk@-$J9i62IF!+Gz&?vvOzvrwjGwhYHobm|fU!!#=h z!m*FqD!!G`pOeTCACc;a8U=8Zn^#Xs3%4iQAfxNYls0iaIrjlM$z{)^bMLn0TYwk zJPE|Z66(w|5;_2=7N{;>ZAre}u1HO5sR_!IJw}~5LNno@36*IWY@F+$y_1qI$I%#5 zZ#J|>#DMoreTyl!!i};LKl9x8sRE`Tb={s(e0)AA0wd@RN0gVDy$r8V+qb&@zYCw03 z&7hnpGFj_4Tx5bH%I|G3fM`uop6w&8fyYS!UlO>`Y*SzrgKgi%S^JrIRSLt#Kw!cf zL!2u?K#ky(RU^%(%jsB-Lxk$3F+Zla=XOhp_x|gY2*CWOArlx!DF^Ql0 zKz|f%`u)Sa)6h25$9A(UsGS{s#MZwvEQNVdzs(f4b}25uABwWQI0iWcnM3( z`7j*(ur|qYoOUy{rvu1043`ZHKo_@>dLU?VdMnR@D9LlqXyg_WeW2@6ZPuN9RGEB7 zZX1tbPmgBc)PRdDeAqsYDS6P0aL8bf^*SosfZuJLFRnAs7>%<)k1k9cg_gP>vEXR% zH!z{^Wv}o!znFef%!-(D?d17iZv^Q2#5ki?*Cp}Da1h+G^Tz`b*FHb&udowTL&`Og zj1XEu!cWZ9HQ}Q}iYD-7*5NO$4ZyRULj7l`7_fk|$8WwiX!STA*&(BQCPbTXp@j;@@@Uk(Lq695${x(jLRUhQmcEVNaBX$_Am~t2WkbigX&v z<3nL)G>zmoWW_i?jMP3vLNhHzSls0JZ6Dg-jGi)ueG`y*;=r@Sxq_8O@p= zgRsOVmz_6az{XT$aRd1p&(IGb^31XUb@23;wH1{s*9pgPq|vNYhpl`Sl;8Q+{i3Y? z##P;fsfiRy7n|!kW!e25gz+t{Zw$HUYYa9`wb6$fH*k9SDWMc=3c=XD9CDS$)n7oq zaix{ouJxvfRBzOEsSuX@CuRrfE4@IP5q0LNQoTZnX~N)y*1>)}G(5^5goXJB1g>4^ zeRh%(72)!$HaIrf?&YzmIM{5F?GzFXA;? zNc0ROob{gx$z~d}`Hjk$4b5w?-#?gz%11xJ6!W=b__R$M*csBc{`PmxdZO3#i|AQ@ z(_kX_Cd4&ID?x=7SJnZ+a!ybqt?$)Z4E4rZ@1TK` z@}ejj@1FUEVKF5pxCq zbEDSP^m2Hxe$}dc!STItq@6LE560GGF)pXJq$ee17Y)2guRxH9s$dMqjHijxpMRqs z?K%P7t4MO?&Sz9<1Zp>cnlf=?3zs`^3rF8_1(ucHxDqERC{p?)t0MeF=bboYrPg39 zWi_^OcB>Db2{#{8QEpbquIddtU`fNvB>K&+a9(5~pJia$W9i`L<1yDCHl1qv-K6>Q7r{)+ zRd+S$bApeSG4oaq7V>H#@Um#0x1W%zLNvbyPETu4zkojk>W*ATnoE`-b#n{)TEnib z<8msZB{KD_&a} zt)vsJ`#Hck&!`!2GI38gJ@Qp|l0k>h+;Gbsh$D;Mcpe?S965_>QtjG+3M%Vh9|F1@vA3jhVr0@CCc*OG@ za_Z`l-3b@{E=K0#jaJM1aM^CdJ@Ivdf7plM^mvTGrWlfnZU}A7zhJ{CZmEq~1of0_ zZP+PA11%tkr7$OW&(@S``9b3ZcR#_&eWcNyqAE z-&3E0XTxpr|H3=|4;OhO2N7}2x0F8qfwJa5AX_bcJU65w(r~w6nPjmEc{i?ZFWE&q z)LExar*%qU#RM9vg-4g^etcQQ_@#F&(gU9+@0*^{QDq=uK9q@NgqBiq$;?gs8*J0! z`Ii%?9mL!8B9%Kgg0C3*UFRFk2QMEZcQ)!oxju#D_qyny4?4+XA6p}p5!514aFw}T zhbCx1(Z9)4(a66f=VvP~YB5|HyWM{^jlA^#C+hf_-vks^b|1eu{^{Q+aZ*KoPYG0) zNiglGv?JKcLH)vJQpbCZpN|0&5<5#sz}9${vakfC&+)Fs?U&`e~~SR(#r9M>EYqMm^B? zPm)I^)6`8x@}a^g;<)mjB@Udq`3r@e4SH}J9BOr$-!UUD>degfa;O-?BSjZY#qa6R ztQu)2GHq%#>6XO8@_SoxjI=zz%12)O*meT{OuIhPGGr$2@2}#Hq34RLNr|u|(x$JI zC%^C=^Bj*!@g6$)20^uTi~xZM5Uv~dQ&@(|{~7Q^K`yWvkZQc$)?@9~;d8$A`k$$Wsrrs_>g4M@F^fEP`U#`l1!xmLX-E6UW1JUOc=$wG#@F|L*XUxN zV;i=7HUtEKQVy}pnWHeYO|^8u@@$l%N*Fd<%fmq#fYRCc0nXeyk-w*L+8t zS@a;8s}2@s9fO4WLAAFd+hxm@kzDiQ&^|jZRAcxNhLnyA@{eFU>M%aG zfPJ}Nn7HH&l1YYR$Ot_>e;GN8T#Ks;MjD$`LpeP2P^$@-MGHN-#&-SU-fMl69uQBJ6NSdC zK0un*G{n7?<0rj(=_5pFOe7aSpK_|9%35Sl0!@oY@GWi2*O~>%za|)NHvVZ4R9`nO z%5g?Ka2>vjhnqqV%Wgf3=Ai89u&;ezO}z=G4A$?6@OsHh4m!pd<_}&--)V7MjSePS zI1VLp4kKZSltefb;Ho1#5(=M+Pv^PV>d&i-UJA(0bKP1>u&|#Pe+xRG!^>(6!y>d^ zy6y8GZktr*1R3aOL^&I{V-S|Hs$BTZD^IZervlU(9AoWE(6!2T&9vT5Uoxs+)_v;P zPSlE*)wEDnhP^3E=HX?X4$p4vkUpRjEg9_g6wL)@SJ7K2SMgkzLaNL1=|3F=HzBs&ssow7=WE;=rl3le2RAA zNU_!di-{!fzIbq)PO(3Lh9M?blFSTOn8ey=%`wcNUXXyh1SlfaN-rfu6gVeISP<6o z7cRe}HuvSz)8v5HwWzjRbYW?WKHyfcJ8 z_VZohrLF7HN&*|4s>-fxHVPLfn))Ify>;Nq)1 zQsl^F-hIM?{6(eHy5eucG6wmJS#TA{&?GYg=TV8(5kHT_rnC9(rrM*(nTx!mY&+Ovd+&xM|e0QQc0E{MNV~# ze*o-lar+vso$@Qo^^%hHXu$;+Iw|>3J58GoW<$FPwsHu+OoxhpxEu7+G-;4-X;r-W z4YRDaX-ynC;&RlCcdo2D^JUu3M|le7??nM=Ge5uWy(73Cb0PF#=cQ0fGm-gb#UCZO zUw{j=84pjhGi>ktUOk~?nL_h($f*dm`Pe2E?aIulR^td7t3dmKXMvoTnM58R>ZRTU za|)KqE|--lN4H@mGr_R187G}Li|2T#VNh(Y(n%4ZlZx-f8~$*O22Y$-Nh2#iIpfBC zMYTN=;=phvs-{b@3Rh)PsW83vqsR7E|bX<_M^-P)7$>6*Bj_+TE zw;s(!p`k<`wTVBk1V$c0%MFiE%2={BNEcuU@O@$b{!pBdvFT%O|0ADyWD8xUtk0d5 zyCLP?3R#oN#2sIdr5cT1i%RcKyAVumc&27Q?m8U42(O^)XN3h{hVWGgvi#FI zUq~1|)UvajP~~Qbw4_nDrcESu&W=z=)+%=M_0B;b@xTeudaE0@xC@`X{RuJi@itAV zB{af`ApbJI*OsquOuHs$W%3aFS?zjztF&jG6-KHS^FvXKH{>4=SLXdZVRCqXn#rrcfc)>f6iZjc{=Cv;Pjt!hkN(F|IB}-wNDY> z+V;OThX$`;N$Bh{M}e?CIPfKQ;mUFO5l>o!oV?}ASCg`$y})0pzqad{()GfH?8TW& zy!7$#ITMGHxsOMDbkqNXmnYS!?DK1r!_57?bo3RwCmd5?gDv-WZS{U62#-#sg%lAE zwKxM9lL>f2>H>_Y?Xu8AA%W&T*JZA*j$ZkD=ok7H8hJ12yYeU2;Q-m})C@ zP9X(v3A#Y6_`@^ZA{tWYQDy>YA6p$IPLkEE1m#w_y|t!@D5&o28u+h@sHuFpW9 zUYsBIU&laZn=1(eUvyu-PM-e>U@?Na*mmxF9|h!8EX>`*yPvopxvtxt>M2d*-u}-B zdbY@&h=4fWmC}flgZZdclAeVBh|b)6gs>iTg{V?0li#ok%rl@9aEADUqAd-$+$8W& zAh?nsuJI1Wpplvy7mcQiMc?XvcF*u{^rCqyx*R0=ItZ5P!E|-I<6tn89A;Pmavv=2`PT2kj z$7Pn1f4k;aimMv^$9<~;{9io(`F20A1qojYp$O`px=}rLWD1h&k{H%bJx%UFRLKtE znv(|tJ|@WlicO~hOp<2U!zw=L_`a^lnFcqf=R8+>{JpzSJd#A%!~J+|bkhajyYv`Ns!SSm?d+hp z4p_0jqLthUNp1EWkAW*Tsx>_{c|@ZdHcai7&yeQ37rURq7%>xt)SE;9d1roR@uXt1K(WgIt^be%rhVVu(X^N(gr zIUa)_Rr4v8AE*FNMZr38Sz3>nvkasA)uE}XP%k<-?7P%(65#SjI~Xn=PUc8!`3Rk zU0YmA?JDN&+dXsequnPdJeZ$ZNwvTB=^RnZjp@{`gqO+epaR6`B?DxxzNGWTP?yS| zx6t{dnQDalCq(05O~BjxtjS{`gEVc~lF9Gv^xqtj#OiSf3ba7@8V)Eazlcry+%Z{j z-ImDmB>2VoF?_Y$iWH@TQ^JH2tWIp(4;8d%20Ap;S~Szz&Cs_;%5+EPm3e-=AHQ(^ zqx;Y{5zG#iBzkN@v4;s1aH*kTA2 z6aD7ANsz$9D%JOXPr@;A5i5g1_rw7+TwpPTw0_81`F*5i@NGgt#=Z2d@64S)>nTg) zXJzlUNpdZW3kB)dK@1HvOsR@RNgL~LLQ%DU>O!k1V9_%6Zp(h}b37V$2 z?_Hw`I;<~l(?(Y_$k7GVW&mi^4QXR7dFlh@3!B8AlqDKVnyyL$KIc(hprd-&y8LV+ z(Ro{(l3d}`M7P&Oz0nk@A+o!JB>=Dy6;r#E78-Kq=g6LW$-hdYLS@!#*UyHHMbAq?=FQcI zzfUPTu#Q)F*8`qbnuV`C@yH&M2R*iKBf@k1R%XafA6_b}Jk}?vZtoADa&tUh{#Uc| zpY!+c5j{kd@L(uU%!-383kl9-E z_oRf!gEFSDsZ)v;v1C4#MQL>#^r%(ifNSGKTk|T>ud|hY$=}*0u~*^ij@?IsD#Dh1 z3on;b!s;c4v7 zZ>8+~#cx|b=j9;{rET4#+y1|{gO7<|H8>(S#5XX^_&;?()m%`1)HW41M5(t$1ZB)h zJJQmI`~{XvPwwro7l^ck-MaXcxJ=j_!@+vBu?|wwdT61pc8!-mNxD@Sa>EX)h^#@x z`zBgwFVwNZiL+IRggd-GD;n)^qE5o%n)mF?{LwA4RlZ+{2+T_dL07O=BHKmCcQU9b zK$g`(hWtcEi}CIwxjW&_>OQ@ZH?YO|q9Mn}L}MeVA}pG!V459qtGzJ%AiFvYJULtS z=)K8!*w}DX3#~bz#f5crMJEn`D&^G}jv5+N?}wCW6?_g2=(!v_vHg#N8V+IS##cVM zqPN?NIEdW&`h!8KC(6V`R+fm6m4_?8OM4&xj6KMs2*=ISLU%}lU@asUhMmGUe=B;c z=TEgb51-x6`yXjD7cBOAk%;~0BBAM4NxwQ_RL-QN?El=1aQ0@~ggrju@^+!Rwi@nN z-EC{06a7j`!9IE?;X3NU963Npt?~4mRITcq|35PP?g~|pumFzipiS+a2cdVSysLE^G}dw%EpDgGS*|`!xpA%peWm+HEz-N0%rhI(4n2^%++hktTO+)R*ckV>UX3H zhtt@tMEX!l6w@BR&maLkMvv778NDPtwU>+h1vK8rbhE}#Qe>r(!;^!B2{1jEHf3MfE=XbtFVQs(VK2C7PamCA@*U* z6c5&U&NbC7Pd@W|p)jRJ8%Oa>hBhYGSnSsj&87}cv*H5)rscxCrH5bSH(M3dq#E#& zv5OLwj41rIi^wfzBP56GRlyVv0J}-VUr9eM*e7qMB)KToK(&#!Ki_p_RRHaFad<~L zj&*t@dsS&H=l}{<1LXn^+JGzc$b0!vM{G1301RDD@0jBVy~66Pc{1~6b`#~ntF1Us z84n?_B{r{?oOD&^Bc1FB?QZk0$ABfDNyheaNyGYpn>{M$*e|j-KGm?puG8yIEaH$* z=1gDDc!mgcX6ax@4l^k~mCvYifWYyd(yH)!&baTT${;qkCLb#cqCcRiCJ5(mS$P`nx?e;V_lf%Bc zAoC`H;3w-FSEEtdDxzFHI=wO(&{E^k4uDrf4)urqJ4jY;#`MG6A(1EHzd4g8*zR9) za-KC_aTTPuzc2`O@N)&FKY3Eh)6#w{STYhz3dYtUIp^WkXt+B4 zq039^?tl48Ii6B4B~sf4!NvP`_)|< zfEGV#Gd6L}j|59k57|e;+HNtFseD~v!e1_Bd*ZwSE{`Z8M6EqlwV4$TPV!p!0R~@9O=R^BHX)u;WrvU}=FUFuu@>h2v zcV~|GKSK*m@Z)Vw*h$PC_1-CI;$dx*mY{J@aD*P^nN7N!>6?Zs+de;x?viJ(eoS5_!Wbc3@!(y_Q;6FtX zga??AvD@FY^`OOav#?N>%YX5e5%XWeFxqM}omLKLU@^UZx6;(5odKpr8E8($0#H7OcWmeLef`h z&|+sKWeam`d*+N>w4N0R84$^kK~GNuZ0L2UcCQtz<{((;5UR6aaBC(I6PdJNmd+gl znj>f}-b-uQOBgweDUH&Wm6$KaKJVTJ+@-$C;d(2n1Z`i=Q3aVQ_YzF8S9rCYgs*RQ zMGQt`S&b`6))Q~%NCu1PXDUj+eT(*fUqs}xX=M8@fudmKaUjQ(^6>S=!K3cjdxoAw zbiqVDE=h5lKl&d>PQFZJJPr>CKxi@I&gX0>V_QTtm4BI@j{dkgKE1pVd&!=CG5rib z1^6RZ6!mefMA(lYf(%LtqPn zFJv}9Em7X3cMD_tUoGN%=fHh^vz$U`ztD_bkcN;3Vfc;h5xdC0PN!ix(!7Dkg4kO{ zML%;%ne?0}qxC)|0kHhXG4Acr*6vHz-4@axGpkAKM;@;*#{&p0@9KfWK}=Zc<3x~u znOr1_)#|)klj62K6eF?CHljl94RJ3>6CMKb(s4nv-9kSeJlLMgFEySsGOLUJ$LIO% zPg=TK@?7Oko>N7E7XO+aRPHmrv=ApP z6wym5QN!D1EVyMBl+~*v6No8c8EhuvwTWr+puNGTS14&)XJ-qxopwDET#f$l=2-Yw ziQqNxbDNyr5mRxp7ctP2l15h=geTsCBhHv51xl8Pv$af2mg2*cj57`UG%wYdBB2y` zSS`?0^6zf$VeN*rKD0$$VvOSbz{o~P-b=J)!N*KuM~jD+SHZy0uux?^V}^gEO=LJt zWCL3$nm5uf=Vh0nIo|gcs3)RSB~1zNiTxP5Eu?EFiY*huIUVw0SMt0qy3&!CH`-w_ z>M7L{$~xkV+ZKuYz!65nlDCi)@Va=wEz!;CKT9I=Zq4qOEwI4VX-U)R;mh&psb5pI zUFQwSCi7xPcs3Di)Z6@xuR`J)k!cVJF))}8!P!irwDu^Spix8Mj*cL z8@CuPTkepv$r23sP85&=cDq>we%$z7>K>K(uBodci|Zu|rzI4%2b>2fe#LXY=~;0y8l;M*^z*XlI;ZlA0$2Y}ilYzh+!$f*(w75h=5;eO5)!O+vXo*o|wg z-)cAA%<^RFwD&9UVh5!%j;6oH z)p<(Al49gy2*6^ZFSb$^>WE_g!Uor>Utg#N3mAZd;PXo1BQV z)A=ZpizoYLmtxK=E`7nUE)GkP?Cg&P*o?LVz_8FHyck_6_bYGX?#VAlmwxx`5(h87 zFpVygiLn2k=l^s1g8Irz)-)Z8-=4W!-?wD!sYQaVe|GV0D>(g!S8xHJQ6&efr+PEm zP!buF4q$(X@-1;*Ww8OIQ})X-jN@A9Q0$e*8eqz$dzdlQ?(KhEf~SW&?c^PHESW?^ z8Z!hIIK4KI=GBp>B_pY4e{A(?;EaU69oUFZ^iq{F(~t@h@-MwxxcMqr>bzkseqHee z(PZcNe0Wc7MjSM&+3tuJHa=&$T+CC*J?ip&IcU|Y3F9I~C0&#@ZH_G1kmRl{4O$#_ z8v4wcpcsQUOvx%nCVQ>Q{F{2hSN9{HoOKT$Kbhl}qhOh|5vdBRwCbojE|jqY+?{kO zA05sqGqDF~6ADQ>L>9k~*rHU!-Mq=SZ4kHU6KI0YKd-Pg(Ry`C*jCG<8vQE#>?Zz3 z34V4YU5$$3h%-0#gm!H#rDe#(_&tHdjVKD<`~{Ih_}6S2I~q!z1)fYvqDx(*op-4x zBJg{}CzGD6WT(Nwrn^N~2Ra+)%#dXx;t)>6bD2Yf4o?TPc?Z)l(wWec1vE!>n9v_` z#NNdgZ3fDc>oQJyzwHm#Tqpx$V$*(7Su&fiHqJ)jvYaom^DwtPO890RM3=@IE7)mC zjRatyg5#+Wxp*b9Fw2w2wf*zX_Em6Xr@tlqD6iC$OnsYRz<$eyL_TDNEbyLj644Ge zoxyMIgW;FJ>+w^m=1XUU`EU>Mv2M`as`WyOy@nb{x zqjw1Qi}yxn_EC!y07=`rQ&@PK{Bqv(B7Cj4RcQ~b1`x0gG{NzTgP^Idcn-V|Ttov?>U&f#^AK(%TdVuJG7Mk@=v+stI+Hj|V#{xTr^E+Gb$_{sU6 zZuJ?4GLUH?Z?!QwVtEr}Idvq28Y>N}HN;JV1G3yc)p#iu66yIZ045?*>;qeNk*hY6 zEvc(`dxAfhBzR+D?4FH0QpM`h(QN+9Dfu`8-wA!*w!ta#JGrkmN=WNhTME})5O-p4 zW|L#sJVjK_Nr$s6R9x~$?AWVPXz~-fvi`4H`hyUr7IDx1M(WK3zE1G@AO3)7pMmv1 zelLa(E;25@_whO8r5ODXW~apLx8$QwqI6%qA4A9zMv(JgW*XOWMDft~?a&CblI={~ zJ7s2m6e!T~Q6fmQ|N5xO;Yc+d`WE&mNQ-CGfnN0?`?KTFEw+wH_h`Dn&a zOnx#LLv_^mYm7jxV-%pevr*HhJ_+2Kqdh82`uEz=C}`OSFzhLObKR&&xEyRtlGRei zV9=eyOOsUklEk6O@d#}p?#DDC9*JCW_ay}IxJBxFmpbXY%U%dYk&xi90-W%^ODhLE ziIh;H!xXC}SF2=0si&E#^HFNY%Z*V|)K?8QKE$SpK%EHXdEb~sd0q=P8#BojiLqrN0`8?z941}!wdo(VdA3aR$0iYl$wB6GS8Tr24%DF=J{;HS zHMO;;PY;9u)WqIeb`-O9`Zs@RBWkoGN_mmO56lX`lKkmpK#YyId}u+SY%6+W7$vDq zhBJ4VTUeIy>zt_mP~Dm*cb+0#>K?s6nQ1W^bH%>*jR$Hum+y{RrH{*#OsWB(Nj)Of zK#Y@{n1&@L6;1<9*iQGU?%(~+?N#7G!9!sfb;ZVR?sv*sF=$n>e9oy=Pko&NONOy8 zCJ6L6FDl-$FQDg9o-t@GX z$1~B86ItmY6Z@0w`_eY~HyCHKGYI&c-LIVXxge&F`fn4%e_q~OFvGi@w;y4LW;Kyf z6fmZ@VW%Ar>Qh;N#!=o(R6ojlb-YYLK7|%b7}7x~a2MzV6Zi^FvIIgKJ6=s)VmjBn z!KQmk3>+F3BB}!VpktxnLsb%R3dXOaFR+_frN`C&*?U zrL4P!UqS|u=>kxmR*d=yguEy;y;VMSEuIj=V77GcEZhAcep-C&dRKA+Dg9y3V{b3G z;k-9P=gHd)+|>Uy&W@+V!XnNo&#*sWZs)%RLtYF3^cBAXA$gRF|0(EC-1xHZtSU1Z ziO~5ljktp{&Ky+EF#aFD&N`^Auv_=I6b(>Z3ba6RiUp@wX`x7g7Pk~Ap5X41A^|E8 z+*(S3B8B1_B)ArLQb=%j$>p4T@7#aBb7xN`lbK8=dy@U`^*+z(3LRO*m#_N3UJDb;|U zvXSF`to07@wP>&|uf2-`6nANz@ymy>Eh$d>b?Q?Vu~A3m$3{uXt9KTx$b`UOFgekQ zMBDGw7PTcfd0zJRc}*sd$#riMoD>mG4~mvamaH9=JhKjvs}zU7>p9sCCPWuL)jJBA zXGJaOFhc-Zne@VzosMDlG&gkA=dad!8te1Rzoq>3sJxZ^@Ky@U;9j+xx?*hD z6SXhny&->sgCP`k7<)5%`Lq61k4@HX08heMhp1>DG#PV>7c`oGGnY!0_iW0X^igIQ ztqzk-3r-@A@{?lSLaASCXHTWAC3$SL_HDI8y=f59&x?nxB3`o%RF*hX-cKANgv>`5 zprgf8b$l8pqVTg;W}K+?)ao$=k0@ z7X#IVg7xl<95fQRF@W2RHBiV>Rsv9Z!1Z?5 zd>l=*fgna`M2r-KDUfPd@RGdqBR?8l%ovCy<^q>IJI7w~6jg<%-VYp~Lj%sOaK2^A#1VZA~s2ARDz8({;o5Z7j!bfYz z_|!LbO6bx$ePHkP_qK;Bp$j^Azi?^^k=#9Wc;a9VGfMz>MKjS0GdTi+Z`*g%b&H$< z&ZLPAdh6zOm+>RKE1!>->)h`^+A^HN=p*t^_C%4fR`J&yAvEU-Kzgb~=J;6lVDi`` zHriv}!rMuaa1=Zn(8o6E#H?RRA|OWxc!J;gD{?b6<*&FVvm^W`o^26txfNiMcF(W# z7lXoP#&Q(27{n$)Qzb57nZpF??fo#2AhlL_M%QMjRb7CFFK=q@TO7^xSIg9 z%L-wHa!q2P&Sb*gFA93t!<~FZi7xHPHK{tjrw>5V=l@Ozo8u~>eGd!b`3nF*LP|5) ze7|Ee&)-mIVJ}|%$ydaWrUXdaU-1f|BjY;bq)Q~5aI!~Devl=sNIhwE<<21G{)!)7 ze36|JIc#kEBdR$4If8UpU^VYw*0(D}$IlE!FWy8w71|2jqy2;#BZJX0E{qOr(`z6V zFzt!QKk+16H0Uvf6^?mGjt((A2S8~n%M^*(O$dW?atjnd389sqYqy`h>`ff2TD)q; zPRjzKw8Nd_m9zJq{Z2Jay){h<*8Smb&()-npXddpsTPMHpc#~uGQ6y>3AMB2Un`@^ z%XJ0=9e>V*;sGb~wOZVjyv|xoX96#%O`94=m+%)t(D<6ngldzlWpr#ECTcS5>GCRZ z+{(|pJgED5!0(O|jguHESG(l3mqnRU-Ut!d2_s^rqK)Sl6C?vIV=~SF`U;R6Oad*n zMg0^8R>++=+$=~_oAhn>BT6$?Y=r4hPUQ`|ssL4V*kpmmD@_665Qkf(8EJl4r|!3d zF@txg#!s9Jt~Q^qje(?spWd3{cnoPh_hkK6GccNb!|mSk{LxXdfzx+&mvnh-p9H{x<>wOl}%hbu^VL`N?Jo=mJP2LMH9)8OB@9py; zj~oBH^L=^y_rbkGkN$bwF#Ti;GZgL@tqFjek``q|JJ{KgJYa3E9U5wBrAv%NmxP`*iJ#^6 zahU6U7h~@4<$js$VxRIB!AO5CZ4S82t)FWqdHUqB?9%3pPnJB5_9+nQX{m_4X?5=V z|J4?*6Ux!VufRbunS5WvKdk^`IV_kjr+gPeraLx$gRvvN1?6EZLhqD~rKzQY)Fu+^ zfzssD@^Z9Eu2>G~}+gaNTfGWK0C>$3yAt~Edj z-&p-Z<(!?ZE67zC8DJ3a)4m9 z?>;>Q1XYhAizVgsB3MAmC4hw^xR&9CSzr}fSzsyXT)Y_wY6XAg8KOHedmm7pl81j8F_G}7~!b^dtp;_!PC zXQK4Qm*iXvMz51}McW69jfxpxLOYL|$R|Iws+#S399Q6nGhV&bEDZ^K=Mzu4 zR7O>$3+BU22ULH5D3m=1k+&}PRBv3pP{`g(rWgwPUPE^+dz8}S&88+F32F;#r-uEB(yYbi=0pFt9+H0hF3jGJ@ zZXknJ1oeR6#am!3X}j}o4G!|pLYg+SksJA{HeS9V-Iy1-O$B|*-$!b1@Bvycs9!!U zK3gk(EdUL;C7kY9=ulk0!??dx4`pZI{Dx1$$BOxm(>43ttI_t`%7y>UR*_jWs{R+* zMbOA94qeYsK}(AzVoL%nC#fS1O)g!;Durh<24ez#4B(#8!Tw2F%@R83SI9q4PS)n8Of z>c`F9wjF8pK1?zI3%0kx;0_uFfhRIphXFr(#QWy!Rfiex6+WLmrTeR7xdQ~s?Tjq! zS^ed!*cG@R90AdNEiZB_4p8|P!pQVkl8V2gaObsD^(gHFw?6W&9y#rukQroC=$YtV zk;VtSRPk9Z-mE^W+ePI&UQ1Z~-?0DjP|gc~K=pOyf?V2N;NJPY7b|J9La!MpPq28@ zYVyzc(+~JcMq7+EUY7u{smuv&Wr1Y6dvdUkv2tSmRxBA*XL*-62ZK2}+CV(*#d%n6 zQV6@zoi%`gMQeQaOJ~yWPO+=S7Pto^yndU}&7$&wC#9G5RZb_|^MWzsyw+xz-FlDM zwf=68lqCJNrt$Y`n5ojAwkV+;5P2S%QYO_>7_YKlNWUQKKFJm5=Fv7rW0Yg|9DN%* z51!vWCo!gehJ&K$6rUwdnURA0nC^ zWVO%5*73Cd=WoAYcMK;1t3+Xd73Jv<5~?M zaTG4lF;%O5<|ga&Ti~-!Bl)Lb7^K@r7azG(UH)Rhg+_ms0GSZ?{aAjFT-}@_;msP2 z{OMVL#4;`TDysZ1ed+^BPYo?;tt5T7PoB?psJeWyCHeMRf1fR`NvZ>U7Od}p2mw( z5AWw~tV|9w%`AwMn#uC4eNAG>TjiZ5W3+<{llz=xnEbUTCwD4AfLQLrj9NxkB%7h{ z%zoH_=Czy06sVj{9acIXwsp_lE6IObX$p%J?l10ZmWHg@RDNz32DRyfo#$s8K##VUd0v@>?(auabn-%++<=)_MuKCFg~MU3 zP#f8$UBz9vKR|!T{|lwH>;>roO$o&$xE=14b1iqy7QRQAJ5oM1fv&1vCYHelm!g%N zx_cjW@Z(O=RK^?s*lx4*qOIm{2OG@zUTAUv6~ps44eMH_M(*cgwXPrSjIZqDOs*{*DtZJZ8D4J*RTUO*Yl_Zx82? z&M(&IyqnpKYu*$31#j~^358$;mhSl8vIY@7-H;RX?X<}>^PiW zQZ6S^kMJZN79jvHVdIU#g+`N1#Bv0+l5~(l_OzndwXGixE8VxdaGx5637;D4E3@3y zStEDro$*BY>7mWw0tC;~H>Z7#?ZqlNme8;XS4xg{2v2x8sSAF7Zabdiqdo7<>goH( zwqH)}@M3F-vBmAaT{#9P!xyi*4*#?CuIe9YE*xY*p;XvYhb3OceJ?OgGt44)=dfkF zRsX{_shbZiBZu5rqHf)8&t6OH*d+eg$N+5V-fwbHXSdd9E+F|30&cs=b3S*$0Rs;&`*hrx!0+SY^iij~a4a<`Z9=EdeMuP_DSm z8-XqX$83*(8REDJ^>{cWkFxD3d1;3o)ehsNasWN>oTXbo3hjqPp|Yv>wWz@d2l`!f z#9|6$L+cbb!h_mLi?+ox;_Qfau_K}jT9KF8_!y8R%=u0 z0kW#6)p_E@akD=jkeXG*qU!CAKw(~IAGt*A?dWrD@Pf@3Ck-`=YGS!mkL2s`Nr}5! zl?`^ZIcB0PDPIyzl_I|8igaIrL_lL^B7}$YJ4-$WP+-I7QTQCm#-$+L=cK&jcU^|_ zJNiXta0tSSHLab}afl+T0MH0tRh!GK2UXgNH^CrxrT|- zDD%HCm3UlYLoywpU(drWd#$NO(nMcUoo3V5`q4P?WAF-Gto$`}J`zxU@j(sYq?{$J z9!jF3L75T}QKa;(YOjl~%rTVS-S%nfzCY8@z69v(dZO}fhP-+9n#)q|8HcB|kjmfR zc~6<(SJuTkv_w~8=C>ClWn1ZSU;QYZ?_B&5YagXdBmQ*0ODr1j&0sZl2 z=SeHe>^UHKI9?rpWK`B_fC4nnDV+>%E}D_2YuV3 zTt%B&F?ssTJeY*Fg%t9Ooxwq4Wq938^wwI$e#fa3-p!I{pQ)zpJ}$)RG7akK_pS1I zA&?pzK<%59negpxl-(p}2^ymB1MG2IjNJ^Bs{+4cu{6fwzInHhFt89nJ}6ZXgyhqn zpGr&*42!-FE8S-jX%~}wJ2LG54!6fkkdSJ~8ApyvQiWA6{C|k@@x;l;p|d0@RF`$IPGYJilOHs)Nm1i~5_AT<-GOg6#_`-{sgAV%P(} zp>syb4W`GlVlRIPvVDk2I?Y^?Z@8@}vm~B)&H2+5k>cI?x~*d=Qs~3}9)Nm_>Rg`M zOTx_Lv6qjoeU`%0v)Pu^c37bZX|XaT&v3-l_p2nu=UE5golE+~e%Sme0LVOcIx$tmj2rjh2HGB%lE^BLeSXDD;WF9nNK(jbcRoxDyAY?M0nd4_Z2aC zdE6aBt*GsoBOse0*(4Z{J^OJ8W%Rj*z;9H(Cm{`>e_YThZLVp(UcPR8J8Qpi{_hwn z%hVObGBG`jD6w!pSU@-ym4hCL~#b8l)xqF}Bq96SF#tw0{Za z2Z{s7#QPQBR|Eb8u+dm5culBt%HYJilj4J%tK41KGWM?s9~9vNwznn;v->UCoOWrP z7FO{ztTJzus`i&S3WLkyEOf2HO$xC%)uMQkI8gNqcu3tpzt z!%Q%fLM8xaq=GCiEtK^nwUjx*n_BU&M8N}enN^c3iMZXS;t${650Kf4n(Ea5Wv`!2 zh5G0n+Tq+%UDfJI$>W~$!1DrZcu7cpB4{3rB`K4+VwLh9>Dt~c1uyQqhy9X%t_6$Q zf^~iWNp0!pE-`g)J^PGw{qyDFT*Bl_WvhG3MzXNC0ZQi`tQCqt-VWmI+2Z^-bv+H? zW*WO5cVh{Hg53C@A)6ZX_=~14juz3=vJ4<^q}Rm@xlhnoC8Q} zAJ4_f49eszuu|WK^VHY!N{jiSL!f1tD4qgDSoG~of4Kf++};E5P2XAeT6009IA(*# zMNu-T?{dFM8f(Ox-3q%{>x-7w4!ak>agfAGZwpG=<{;XBgO`KaU)9Z9oxW8<2IN#6 zn(tZ;_lWtV$kny0Q!3>bg0b8{zZxOUWeo0Kt1n&s1!wt>h|A@^wIS)cxNUs4BX{>( zM$1})#r2#0_)Dys+4NJeJobC*``EiprLi>X{F&!S*{1Z*vJA?0dCoj{UPeS_NQ*=^ zln62w6%rwnw8%*>qn?;INJZs)@F_0S2-`iHD5@Rd%5cmeOlnDG zJzg`b$F_7aLXcb7bg#wW@Q;n3FG#=gQt3JQ0!7+=c`BFO7t(;8uoqD=9#HYt(#sb7 z6cg{-bVhmrE+8v#Fexk??>}i+33Bn@w(#=t&S$uv3VmS*`1%^nP*}2=!F{2(d&1Cu zSYvXG|4ur4Ghq3!fP2EG*@=`CVbO$JBFPwMl=NY)`-z*RiX*Yuj|6yn>=s(I_pM&0 zH1%N+#s}Y4!?Me;8U>%DZii>^7>^Fl#vWRPJ(C`>H@*l7iIzjHRtuy5jayT3sE*aPi#QGm^_CC1Zr`mmRXf zr9JWyC=O8fDnk`LZIu(6MocG8kugCE?QNc)^Gn_^Qs>{ZzuF2VT1N28Oh80@h>|M0K91`>z?Na7w z7QR-&qPL5IHl0F&6!niKacGzC$lm%W#GZcWQLWzFX8@KDn(au~sky4;d$zIbkMwkk zz!0~eIlADNx+_%;y0(AIc*UH~jOG8{Q1=Y7MT%8(ATcI|hcgLd7oFT4Xx`#{@u2g| zl4aga%Dvb-MDUdsMn=u+Oc31#DGftSupu)xRmClWYC(0t3dZ~qb{{vYQ!^8sV|DP& zdP7oZ+VOJqGweqhXnf)V-pujPzlci29L{hoq1d!X{&-GgY}-Ge;v{4cz6JC~mU|B; z*}pXpH8vWZV;2;>x;&L8Wq5`@`1Vx;anN-$;%xB3L4-#)H?}wx@R6)>pj)?9bM29r zz4ocM^a9r>LE8-c#dYt&&TaP=D$8rXTexKCq5r0mEgkiSOs|aP02JspDg68i^^`#` z6Y0(GZ!15SEy9UIC!-Q_sN=gB56hXOX1^YD)fWX#$_Qg$!xH~Tv8LVE4F4fJf5#z{ zTh3deR}mmmm`GHy{XJ{yzUI*NEZ-iX&Ha8|^pP}cXa<(qeivtB4=-n?p3U1=A!7ew z#zBi0siYUje3Yhom;AlHD#wtQZWomx-{+m!GF`)qe(!li9VD{PI z>S+9MzJ^sh}TP4W%;A-m3l9AFP*6{-+ zrz0g~5othMl;ssAWL5prMG2Ww`KkTu*tpl`o`N6A%f%=s*~&$nfF?_#LS%vxX>D5T zZ(Pzsl1csuY8^>k^|fbH85Zey#p12A-%3lv9CCAc+QsWa`{KOuE|9=hH_Dh?Wb3{A zfZmt!GYoUttFtAZf5E$uAVa3x8nUM)65ne$=}hfrGg(f>y72ZvHG?qSerL$gMnim; z!vsVdtS#DJ?Bv5$8o>PtQDquoE1^7@sZ7m!-D?;lKYPy>-(x`KjhK{5`j#n+zS1xXNq=t#T+SGR?oE8LyA^jJdExMp32euDOWATpHfd`0iT6pEQDEr(6 z`<}leetF}$0w9%+ z)_U_K!SA1m#Qt`Byj;>sRdnfdB=}8zOgz_~q!! zyRv21_jk-YAxfiwTR1f7Oc@D}}rF&)GArAnC}| z=iugxuij85Zzvf_Izx8-y~*{k{kT0v3euT1-?8`aJ%>Eo^@-Xjh4%DRoVOXZBx&U< zJdb0I^YUAvwNv7xn)nN+pG=megvc_{prqf?dD#QBOwl%7;KI)26UsAzp%=jgVtMFnK{nER&}d zjdejO1H1+N6NjU~d0dbTDJIfLKrX&9{P4Q>_OT1>vAJQpSU%IT_|LSw%|tyCa>m%# z)4!#Ram*t|`E2BnQ~%ZoWOp8r(AENISO@{nhjOgCg(FdAqY6J5Y-S_#Ghgfd;)FV# zLepQb0min}sH#_~f7k1|a*ztdGg=I9P>+VYGr@2s{l9cn1jz!e!jI9#ZH` z9vAxHhI>B++Np`pwivXbcj3g|R&v``dA{`cOmPf)s=E{wAw?ax%e5xKSuSmDeDney zJfsd(9PS!Bq%6A0VOxG(Ty-D#Ny8UP_wQ05TQ!$V-Or4ZR4}M=8!iZ6N?S6@(vhjhk}2a3?-LO9F3yE3GVZCQ3aY zMj8m@sv~sBf}>oCCBJjnc6aT;+PajwIHgauL&&kVn zjl*8pEAo_c;H~7Y`zBAckwEnEZQdH%Y5M}iq$86{x&JXi?O3DCAL{2O5!h}L*tSNS zcB&_Ca;|ycJ|CqDyb8=&%*6+nF?!$rwR$?5n=Z!| zezZ~r9+Z31d%2Iu#eQkM&nOU@vv#<-R6UC5!w=9=Gjguwm4(TK?#XzumSCQj(~?;g z113dC&V?-ccVN<41sIZzhEDz&wq;k93)zik^K$k4;V+VUPL~HD|on zO!>=c?dfA--f{kc=vI0)BbK=A@O0an(8^K!@xSS|cj#01o&|OEPi&dyXXrDT2y}Qo zZ0i1A^=Xn~ry1Rxb031lun6=x$=OBK;C<4$t-HG3o*gnmpR?(P#IP#;kK{}vkB&wwY;@x^@jl zsW$%JwZViiQOTc0T_}XNAN^tJPIPZovRF>&EC+^=re81xv2(gU3CXxdyl{RYJ>}wO z27GH&tplti04{}UHg9||6Y)D%fGXOto-lNXbzEu*Oy#|L$<5iwt>PSNSD9?*R}#ji-KAfakA$P3VbMTTGMwDRgsKeWs(xU4*Hv`brl#Mdw)5_* z9P6JAFu9I7e>-H`?;*11DLe?N*Oh(G?swP~?14)e-8{meu^n zeX;&ose@S%8Mida*yKfnkQA#>Okt=zoOx4rb7IsLL>q%roo3Es=_CZU z`ksm0mo)NaGOeshBmNsV?P3fBP(F{d9}Iw3W{h8c_2iA?7*0~Pw~6g3FSuRBmMkmBfXbYyy$&0$Q=kNhi-cSw4WA{TSEm@j(_c=BtTx9nznZl73n>$ zsevnb*Wm($5`l$32(>rN8GvBglK4~*+r6#=8yn-8e8V-D^N~@fgz_HE3mq4`>7N_I zYt=Yek=MpJLKe3$gg$$6cydFJDX20sRgJ5<5rSuv$BlodCgU4KgtmX&iK}2Ckw&=Y zU)21DyH%Tq(^4KYIw;jY-1(u&4n7h!c+w@wAXXZcycoq;3h0V>$Yc_{v`lzib?_o& zV;UQ321{b)_>U_`IF|MQB$Ti4Jv{RMJ6QGRe&N_&fm7xdo2dQD){w}44z+0f`HX1G z$hSXx#b9oP< zjI_wTbwa}@E~~FM=Ay_JOKB-hRWvociH=Q~xkIl!S!JgJXkB&LjZ~kt-ruE-Wc|SR~xVrmg5islAMG-t{drppj?UVCB+rE zXBF@?yoh1j?YCbg#Nkq&t9`{TxRhDV^{X6{Gh?h_AiM*|YKO?{-l`zSS$?>ek*C$J zEbar6f25x_pV33c$_2QLyzhIasN=#=c&ycJ;uM_+(Z}$r@td{o&_aWB9m_LY{vs9} zn`%n^?@+jj%AhW1mOk~cQ}G#jltqLh%!s{+HY^|lgtrl8LKOkh(j`qZ`}4xWE?M`l z?X89535*hjG3Y!%2)%1hiUoMTAsM^eli+fp2)BEGSb7lpe87rmwYQ$tlH*+K%U@im zoFY*JH}$g^kns(T;Kio;2e$2%l?w@;yO77WJ}t&!00>jxAR)jDEmwS~YFz6}KV?my z>SjeLBu>ZqC&APwT=oe(A#IWjx@pI_2j+8A5Ppp%5rFCTl++T!v3<71mvlje>~6nF z8%;I7av(%KVAO^SIT_e-lL?;M2j69+96wsnAF;TYe0l5v^ z)!~oE_3n6;>2C55n6>%lyK-jE;&I-MtW?y_a?aQrHVJ%s`^_%nTbnIFgk2MAUgkYH<32YXi zi&kVnR&Fag8qWk`MCSqn#ypx0)bx>#H@5W~U#Opncr2Lrw4?%Vff3f};sUb@^ypG~mV) zgCSjLca~};Z2a?yvcs7&ZWA=!8i`;V>>Z8TAms%ca=zzBoM`#EEKJz4kXB5)2WSDo zq+}SN4r30qgvVvE*0}<@sDq*0F?)K9?z1*pUS*hb^a` z?X4(8e})P#Kl**LiRL|5_x82!i<3lbJ{*&oVjYp7WT$Ay&mJL|W%Z4CP!*~{G~tl5 z1q_zT!)dQizdLJF$q0Awf)=~r03)E-6ygvW@3)`gEcUlp z1HJ-Jne@@{3K4+CI)kvplnc_-vB>o*yM@<$C6Uy=^2s!?F0ve z_T?z_-o;1E5ruz3zBi#9yq6_4dtfVyJXLU7kGTCGq1=@k&2coxJ8cCr=Banr7qit!zX~I>% z;W)}wi|eEZ;K3Z!xb?6ZrI!-f}a% z%N9<+pv!2ksRA`8p72PKqTw@zydMHe422=~pZKsh=Fq&PL(8Ij(V@3k!1_h+Vak8B znv2Ba+URKg7xm)H&RYFW+y+o%2apyg`LPX7F+;SEYu*4Q=_cQ@ zk=NnAEM@wj8r*TdX=;ZL=}MzELt+c1n}f(JOHvYzOL(C3Yh+ElHYz++dE z*m!!gmvl~w6me&!WvyB+ksj#25 zOTu0SECbUmQ|Kf92j_Wv7c*79RvbY@AwIjW3(UrkSg5Cey7^ea0di( zvU@TJJyfXXDw%gwQ?pUxU0hLQj-inA7gSvK)Jy36dA}{%DZ~8QNCWsV#Wt7W{P7Bl z!#HJ%eNP~*h;^dR6L0+cO*M(qeJk~F>3LTTrtZvqtn+E3$|BKUV|L*I^7dc4RazvkT!)wQdu>bL{>xXzp4M$CTKW+=BazHr=W10g#VVG3)g zQTS@T=eKyh5-5L@A_i1Tre{Gr{rJvzCnS$KFo!YjkM2Tr8T~^T%2cC-S=}6P{Z8<% zuWj{WhVCq(0@2ylRtB!rdP*l=%bH<18+QPBekH*gdd!%ShO)(CJAS zG7$6bq?4Dr+-w~Y`Iw8o>V8{)IJ+M_e1$@f+1;!cZ+(J}eOJ7+h$mw=?CgA9ck?#O zAR#vtiy8q;231yM=jwug$tCco9xwa4-q8XEZ4Y0P>i^=CT{9{r={~UWSEOWj1T@Zu zVt^9D)^F;;#bNLbPRH-o-_sBDqV~RY7ZiV6wMMt%f7m93)%bqwg7JNpHC|t#Q+uX) zybz7ipptFWvED0qV!MimT6?%PqK(QvR|4erIb+t=1@h-^1Q{|uBShTp5t(1EAGR@S zB4L;DZLCJYM7QTS`=MU1un%KfXA~`|;6TK2kFd(5#TO6*Y7AhbePzN(8&bZq#G(P+ z$BF6>Vh>CoIn7EN!Cqmx`wvr4-2bt3c_azWclI3=4Ly-LOAs`{tlPYU_PdRZO%3p!YaG7{fH~z5Re~({Vky87^W>+Mc6~H1Y z_3QKA{#7foG}I~;HuP*sK>PQaHumqj<<2DbsBI+F!Ph63O=1uS-|L;x6-xJNuz&EG zE49YBCZ3FP*nw!oa}@~^YWByQ}L!-Yw4F8_7r!_O`}3Zd=Dxn^>-xh2(xZ8ry-YG z?UEw6vDv|2NyS;aP9E7AI|VH1i3nI z8xDfCt7$JFovw8(V_(>tvQC5v9-@A(t7j!tWR8xFv|MCRJV`X-F%(xwB_pSLwJHfc zINGkr{OB3;wdM95RmMW|Aoun-sD1`PCL*#MVE(JSTd9ee5cOm#oI!Gh=t=X{9E<&% zY})O%Ml>hgLR|X4T09SYk>;aD0BkxOZDrRS{WO9K%}ub*!ybA?r2!5OU5~9_SsPek zV9^LGc)_IVt9bH{<6wz@B~r5e>-4gt2j~s!k{&R`caj+;Y*U!NcFTk7iD1U zbQH!OW4{4AS$mLkJ75(!61qMT8V+AAX?YM){BdlGtND2NL7CmS9ZuZ>#8(6<=IQkY zxKTaoIhzP-zM9Hxakw0Q>F@{uW6H?P0rS*V1NKZUsH|9I;anQSBuF|q1L9wSUF$|A zX~0iAM$-&3_6X8%3J>~>{Jt_aRJwTj(96vfG8TA+#q&;^-#;wldEYI;GIK7UY#?9z zt&eorxY<1z)qj|?tcDR)lTvpHpssr@Y!|?a{LUC?hjm{F0aKO^J6R8JJ{h6@pW5P6kVg<{*b1#Glg+sVkq$^x~! z`DmAJ(&9)B7Tm7)OLBBV*2?;Bf?val$sC&$4@Ne6La_bL;P&%YXv(ns&f8mQaxtX) zPY@2X@!X*Szy!k)!Bt$f13;ly$-bRT=f0z7WzhRkOw zbtsgUcdOO|k}8@&nW{z;W=-wXi=4+9;#M-)+IZsx^!ItSrJtQ3*mkj#Jo)T0#YDG0 zxy*W!8t_%OI4X(#Q&7xLc~}GdUkWn!S+LW(=}P%Edll!uS%;^|W3`n^y>~-0*GSRH z%W|-G5=c5nxx*bnR!S20ME)FctY8+j_VPz(lBqqeP7ca+aG~3q99`0TO%b)O|DKgY z_jZ-aC&-NWW+%g{eKjHvP6pnxPsrnyKAx}WewBqqX$>I=g?sxZdKg&hFL>W&1U@Sv z>PGSAUgDRx9MX&D2nU}BAJ1p}AvdJBxw*=0yfMR-21Hy^@%R@}w>aL-E(>AJNqPt^ z1|4@UjKqiQEeXG?P%e958J3J`K)dORsoC@G_~=j;nTjZh?VrGK35F{^>5467awx9Y zBciS_MrwT9|MvnO5uU@&zv{R3@r$MUOJFKkaMc(grusOLg5W#pw+KofVWFYYg+jMJluN$nLT!2Etytqr_f46^?HWbswf*AWH9(v z8aOLu-R`i|pCnR2X3e(2v(Ba9h3q8IDw9%A2>O=5LH*7NMo`W)F^pC(2+a&rSaZdyGSW2MgbDnb@Bfewt$0mQ9GS7uVN`?B!c z$Rzn_jx2x9VcBzC|0I zs1`<~J77uLuoK6|RdYGmj(*9D{%J}{_ENeezKr9Bk=7Mk11X)kw%eswH#ae6;2qNC z&RGWYh->x`8Ue+q%iG(@_BxpSnBQ;a6rr-nhG?q*6x;E2lfblhXJECCOp5#V8A3~B ze{ziNT=7G5Eg!1^@Bk$n0vXtS%ZgKJOL30?`l9~cpPxE|RZM8}MI&iviw|3Z->}{! zoszbuXMj#BGR4QWeldY%bvTei^t>P$d$OM^j>x6``&!txqE~NHO4^0TiAlr?^>dcu z=WDSs534M~DdJH5YQ;;c zxRJ>T*r&`2T{O{*t~#_G_9Jr*8L5m2Q;9Z++&7_mv$61g81z8gpEt4lkR4clg+<*e*Xew4F~@Rn zASb;t{zA8>;hZi0cppnS+=PB??lIC0Jj*f=qeE(T17e&yfg7J954yJG4oA0G4!6%M z5dP;Jk=!9?>a38VC>eFV3Nd_@0yq~I&jrI@eGF$1_~AHG3uIVun! zXZLPZ0e1wqz+OA5biZ3w-eFg${zgRWVEgOz%IdoB%Fecks7TS^kgPav?B(~;`#5UX z;CY&{dal6E-);(_C%~dPS(d|ANbi2;6*6|{@M}bP-erKy6~x)`f$YcF;N{s zg>ffg+8v#M4U!~)efe@b7IOC_4Yf8<(QTCz+8vd@QNS0|;RX|tB9j`oq)6Q2 z_VAd{GZq$7?sdr_1pl<3zI3vPxAx5L+m$t9kQ}I+3jcpG_TFJlty%l9i1a4CM**dX z^cEn1(v>bEB1j8Wx)BInAcRgpR5~aipp?*yl+Z&*Kzb(>>4e_-#WOSCJ2U5e*PQ>z z#g**6pY`mu?t86!t!J*8#T4X6p5V7~jdt7$m@H0~LmCAXZBu}sI;~w187O?a^;fQp zjC_B2on{_dRUUr*x$xfWGTG#DTg~(aK5m_!MXRGDQUThXZ;Z& z_>-@u?5UVIKQzzk{%i+)AUHmMWQ$2g3ARZf3tN|5irNt~&tv#M;!8?#6#YvJu!5?l zld7vFm8#zrboRN_GC4ZVm>W3~m9}4X`ffbdgc@PMl-ioLeLxm9-bEr?y@Z5X?z$jb zzG#fj?T#8Qd{0QgR5B_wzU)UK^o#1)_rTuGqH3@4?crbVi=spbe%!08j;Y=98`FG& zG_ef7W>Y}-=z8FfFYM4Gn-@4rQEHEYGn(^<5?J8Cj;At={y0{tO za~U$Dd@SI3IkD)U?b=mby6OuvoQ2L3xAQ+MXsO*Uyq1>#%dFycbQ);ojy%tx%XiFZ zJ0upM?=1cb22Q`47(D91=0DpzUt&$uK`NyO?4{vwTBo*En_^|KVRS~jSs}r% zvF_K9GxOmWWo8w9=>d~+M=8E}$f6^EnQHt0MM< zYMP<0`cYR)a)`ve*MJb0WtyU_=j=F@mypzT{O#^DxBivo%W>Qp4!*YQbMjot1CX(D zE;)3Pe*B4Kw}9rx0YP1rWm+e5F9cbO_24QZavR=A3!DFDd-+b4+H1_E8A&&<(n1_T zZ*@+xVOltU<+KsLmkwg5Dyk>dLDmuIfC7lt;yI=|q-I8I1{>6w-Kya~gfbUZiRG5F zi^g}BxIN!c=yfWMCxzCN$<8Oy`W2itZ9xyz&KBM)HH2k$s>sC1UDU@$Gg@wMTs3Rh zhC){hIQ;{*R!XXOjJN=OLVnb>wP$mXKEdi$g)5ypzam6JpUb>0I)KD0)Nc;b;nfo_ z2UiR~X?(f(0>A(mY*Iqb!e@(W@Da%Dx4sepGtawQW*1;k=@#j^%?Z+NJZ;p%bOUU< z!7&t8Y%3j-f9r(VzidGpRK80Jg58?;PmNS_BLNsg1pI3k@IaTlw8t}Ewp^_5=kyl$ zHC#35vs*Y-BSh*5I0Wl1?t_lIf}v{{sJVfvt0?2ELx=k{o6qmptn)+7Pd-2$+vJYt zC00@nQ)*m6dzG^c&hl#zd$lB=G>b)Mj!!q=>+W#*&UX{dx%8ucu2-S_7wy3=P~Rr# zw5QxdEx?O$RE_P`t9Di(W{6TU7MtIJ-X||&uJvnts5ZB`bSqg!Km3BdypxS_+wYgB zD8yWIj&bete;PlmUL&kqlt%JX`92e+oSgcKDrKGlu9`-&4TNp=)bntGcX!DcgO zOdlIuviw{NpsNR-ZY{l@3+={RtzE^?(bAS9WNsMu(Xzo$BpNijgB~8oDeCP9_`{B05 zzQZ!>(o^&?3eRg;bj}Ztj+=~Dk{`K{W>ifbTpM(5BW<*xaQhORVCpj(R=M4PBDKx% zp*A&pwWlenz{j(_D&c-nc(mWuw8!VGe5nLjnrR?v^fQK557J^S*#R^}>66GVL}xZic2!NP8jS+6A2DzAl zs_ix~zY7|i@6wjV@7zYaJryb&;K|tFH%KCENe@vD?V;_(#*=%_!Pv83QiVJLgghp} zm>sjMooygKtJ2q&do+9cdVfwTCdCJgjM2j@l9P_ zwF$mvl6&xkD;ccMF**{N+<-b&S}#1Q2u6s;(T^F5$}<{|^+v!;10d+j&C!a0{c0C* zt5c>fn^X$DMbg=Kf+F9Gk+`d9+^lMvj{(H-ndZW$aHDg`kv=ahs>@vNu;y)t5Mmx~ zyxo+tjTDhgMZ0axT_S}VLgGKB+HT0Vv@kvZpJ1N6bj?Lc!Jf#OLSL z{R#7`60~s(T@Ehdnh?#}t&g>F;PfpB@7@bp-Ez>S36V4XOe4 zt!y<1uCjDcAi6y^Z-jt(%oO>y8X+kiOU&P1(~Lu@Rl9t$_9`%LbMGRtn`OvG>J^hf zpen7(t5&4*eHVCrx*{;?FQB?K6=E zx5cBmk391qfdg7pM&9=Z&icIX>j5PJKjIEz4<2XPX>UZ63D&dlq)n?7$_?pYfQHvg|3Uzl^Eur zdFtKx6cez!tlNQ{SfB2Z+hGpeRcnuC74R8fdf#pESu)Z_wC_%?RNY?&aUj*6%%!J@Z?7km|qH20S5Jxouvg8If->4Wn6_DL0f_6r7Ax zWSZo$R};K$?mRoW?bU&szjiJ6ZBuc+=j;*huKZq0-?>UC2fVCJgf57|Bo}Wq@NRK>vz_j4pvW$MyUT#dB6KFSM|(~#)NFE92f2|oD{!bCc0o@fMj*S1!rZuQ~%a7&3}fisqSIl*$n?|9Vp zXU%LIhXRnH;)3CmZhn(c#awkyel62isEbvEk439fQ#1od#x13}Q9VJ!x$pGQraPjS zz}E+os@zAd<5a$^RCA(J^}DZ9J*MqPFk>0D9N^A+!{t4xn@=Z>mpRntU9O;Q_W_SG z3F}V!Ui*4>QfXa5G@QVw0(1$EHB|D$5AkH=4}Okz79bRF=%mi%hS8AC#v5l z;6au*3hwNt*i3H zED)&Il2^$GI*tYEr;DezoFqoxhHh3Gd3BfYGVAcM5q%+auHQ4|VvFB&aQ@!C$b6Vj z)~iS?_(|s14rV78g|gc4qhSZH){8H^yNYl4=WldsLWW2_d4H6EZRw{7%}ZMPzYW?-&cCUinJbg7<8sNhrUr)-PF!ORhs&b;Y(iHz2{JfryW1m$ z*Xgu)SKsT4#*aF8eoUHyr^sAg(67%4zjxCWVQqQN*vtZz&j{^r?DUK9<;%4P^)PuP zX9_8q>4PY{B+s3mGAoS~-nL@uFAglw85oIZFBmP?7i->AZ?Kdr*7~)UQ}gWOLIHKG zvFd|LuWUYP%f56&4xQX+>pYYg+X2k5^g3af4LQ%?*v6N_wB1Xg57*!D( zfyQ$v$Fd>Ayuvww`jQ&f%hM3#Y7#Xnc*=I^Jz(gH#hQgEA@&j@nG#1m=%lNn9qX8s z<~gIF?8oFPfO6r*syGsc%$pm*zB|%Z=Z~y{2&Z08FRE3#cYx8+P)({vlagSzmVUTF z_JU06>5V72O4JcLR=|(a9#be!>b-H&#MJCc1K7LkW0EKW_qAu^!t%l+DXhK;WM|%k zlZvpR;9=6L^$hSS$|nOirY0DCm2s}E$+7y2!yO0qsQS^#_R_;OV;6R&=K^~1^9;E~&YfZyn1;TOL< z{EiOGZ*tzMZdqR~=UnV*&^a=4Z3?wYHTd151T7E&kMoAyeu>yF%yH?oRK=X)QG(9t z%Glqu(y;k1eg)3W(PE=p8bVtT3baH3|9cJoWA?|V7P$*MS2B!5>?inaZx-!=i(^A> zb8)liCbhI6r5Je%Gll&-Y~DTgz>Tq?UJ2-Qa13zR^eimi?S~!@rjZL3suUwlFnfL& zfcXJ#K;Tl4Bt-1~*D2)}WP#_2@vb`-=!Jqkq4t#RLh(#tr(*-DslZ^wseFUi@uQK; zGl|sK%O~w+N#_usgWiXi%+|qV&v?}ltW>s#_zHO(q>np}3jqMXqs~kGqSI%7k$CWGJ)mPB$Hzs6-LM_oA5 z1h|YH&9g@u>z*+Cz*Vv{BRU6uy?#CF(wLR^OZZAyrJH4U|4vSrK|$67D%IsTR-C6r z)0yUvTnqpyxK{mle7?IryAE>i3)P-Z+zM&w^z`n9JN>|y=#A~9Zc9z>4AG<-dvA;< zcQNSCZWK{UPPCz0jB!_u?nuJ^TAFO+ckwlc^?undf-#Qj65r!Ka(n8`^{MJM0u0?& zY$KM(9k{VjTIaj9crG<#geO&6lg!6L)TVblZHvL;4tdvhn}Ws<f`oli&FQK~+ zLjk{sIkF%HU&0wpz0hHUk0$%AGNM_SEO?z=t_PxSu>}Y%MrMuGmy#7h*Tiu?K?}NY z5$bk6uJAzj62#`o7NSVPAy+XDe@NDX1*&E~%3fWvWzc^bt=snlNgh%kYSyOiE@Z{; zleK2!cD}#{kXbPYxgO3}@4u(jbpBPR|8TLL_}9UPcIHhjPWnns;x&`cP9O~AgCu;x zt%r<(gZBPg);9$!-d~*=o)l9~b1D_baBL3JCRs!|Fu8$&Usy^lI`0p5sElZ&Sc>lj zLG+Z<$=K>j=}Y$LVFT6S#IaP|5BhE8C&Ok%$OOhRf~qMqW_;3L-iRYnjk8JLdqU*% zY<;Xy?p9UZzSz|H#7Q;wK+J83)pjAlB`s2I^!x_O2tttl!b4%}3N6MfIF--Oc67F* zIpKzu3+>lpRiv)AfST>Lkd-)6?4VKgFfyTrDd?IQ4W8V2v=+Zdtfkq#7*8(l{dMib zepUHIu|vfCmzu`bv{oq;&NE1#l5;T_>v@t;f_%eICP_OqTyaifZU@u9=66(>)!)j2 zP7iirhcFx}ELyR>o}$7Iw`jLZAxwhs5if#*5%To2pmQsnbKt`{n7haA7Cb2q0FRBN zXYb_~&Vau<;@y6^Q0wQ@B7)(Gd%(? zg-X7q@X7Gso~cEMGLB0b)lHETLiI2~6#7ZX*Vu*VgfN}*Fx zwlQXbD<66knY?ccv7Fd~-aIGWPL+r-?4P7sVXMq2(OF%%BN`&AO6s0nEu5o%mJ`rt zK=f|~p?|9mH7+x&?+cTlQxcWuVQ;SMJkg_}v<-F5(;bG$KJ^?#R(WJB-neA}oLu8w zPGC`Q+fUAC=IA-#y)O~R|1?&rhK%`1{7hX|2V9GmL8eHcHP)=H&Jtf~iF?=Gn@mDP z@$(Rpi??!qcdXdHDDH@ld%~o8^^tjqiW=9kByO-GrdsVX?l{$ z%E>;<@`;=K6x>=H6k?ytS1qdGVm}BqO-e;Lw`gXE4nN9%84*NR;Z&qiH5O=={A;;^ zO^kNiJ~uo{qa-BgGO^RRvv`QkL+h9G40@Zi!}s!Sy59)x_!m1sh4A}a^!@r=J^+Fh z2_7+Q&(fU=G-Ok2cVf~@ndzN!v~yS&NrGzcp5HkaoD{KFh#2L*60bL@O&|Kuem(WL zUd+izQp83p!Il)QT@ziW5EE2(Q`(cfrWzi%EW8EXVmV(9M!WSxbjY$Zd9b)6!MQ5M#DzaXr=2PkPYMo*RcUCQ5c zW)wRotp=Vg;a0szRvbDiNjAz6(?WP0q zJuck3*hOQxUy%pjE74CU8VV(YXtmQrzkVe{Dl) z60cMPR+e5ivnf(vPVGfwxao?`Sk<`8If_>U#+uZ)YX z>s)un*y+gB1F+ayjMLRtv2m!J_D;L@a48BjfraG3G~5&_1TRwX^D8zG72NJ zI;u9Xw+z)vq)mV9B!PE;k8gSsYA3M`&tjT*5+^W4cojt&^hD|dT|1kA`%$zzU%(B) zCzm(s?98&u?m`1Y>ux2z268|o)!^M4{=-ZXl5h0-#;=7|;DjJALqA`~$fAQ~aXq7M1F7J-wFHjdJVVRdH8@&(O& zaXyoM&(y8_+zV`_m5k!GkmSouOSU7%PuY|@*g8%~;TTEP=Y+^0$LAqtIcVt+7SUG! zhx48AwEi~Spx7VcdzJ~ZI}4;B1!1o`=B+8@mbKz5#MAuhrPW%u?rs0@lmJg!=d+96 z>ebG5w{DlBHWjI~5I-uJM);u(<%p9Viv7xjA1caozh;Fq;ZB`TqYBNm>-NT(BUKD) z!xof&A*VH{JjHP@FG?NMX?^6nyq_)eCf1m4NN81++?0%6E-%!qX%(MY8-|*14oZ6m2 zDVfG@KZzAaY@R&Gefz3SFXHF^q9x_T{kFNH>81K#cQv=cr0V2Ey${0yW9KnW$`*+n zrAaBH#CUBnUv)kvaa09G%NzGc|8e&vcWH?wjYJUk@!d^@*~1x=q5=hT5mScaO>UrzpDBKX8&8aFH9V zG1)+e?5In(ZUj}62<=lFrkKmyUG~QII_ruB)ly4wdROkf05AI}gxyq;fRa0G#&Etg z4LcdQo*fw(mqR3}GVz2=NpEsdX+|PhIR|6|@)pC<;>WZ$10gG3JHa>i0XEh4q|!Bs zk<_oa3#uxX=nTgEM_Bu_7;8p_gMepL%pNTyz)N7MHmkOmi0B8Ec732BOVx&FOZ^Zy z&8htdGGy?26q8UJ3Eg9|I1BleSn0|n)>?9;y8PdyA>IX_c+mIw+{ly9O#wDSxC^iV|Rv zpe9eG2{INen%uw1k>$E0hSXDv7!cQ_qnvOTY)=-(@Ed8jYqDa4yta}S-p1WyYDYO| z+9swRY3kfWq_pwY!ObiZ{bijeuoF9BsG!r zLq|CbP#UKeBSn*QBw^Z3zP&_pu$ER+pjkt=wVNS4^WKk44ZQIVk~G$a7H7W9tyau? zu^$V{fu#1C7QD`#A(QQ+T15)O4ONZJuopG}^I9z1dbqU;Mj5oVEk)Prc;c8gp4zp4 z{uqYwXNJzqNsOW>5mhVWn0KO9QHD$e*GLaM%0?gc%|5wsd|5BD0o*P-IdigqFk|91 z;*#rs;mPQ&`*x)=9Q(~I)cCv9`t)vh8|6t}l&_tNk#`#vBQDS*I8->KtirY>Q_YpA zFbuxSIXQmBs&V13)amn?EE{I+N=U9AFZ&?_f3Qo`{DrQnU6qeH(CKi3PaRSRl7V3Vodz$YhI7`&NN^LU0Rih}9+ z)8VU>gT&&_`uaZAl_WZn@)N&HX!-?i9&!##kH9WIEF1*_Oo1>$_ zJrlbbqtxP84AojyRKKbP5V_yDhU5OCT+^dU_|A(|^~RL_D7CMLR6Y@NVTUr|)P7w@ z%nL@}B4w(Jo{!vf@%Sgj^%10tZnb+wd>iE&uR<9peQ2BYiG^PUOrEUbJ-S>aS!)wG zUl-&$u9}9(zBzNr#}l(mb|Ekc<+u}_m;f{Q!nieO(^hd^i7us(B6P(YY-6@Fx$H9{ z!c$Y|DJqvmD5VXhiugp=M@VE{Nw364rL9VJIW4sazi<|O=`iYO`qC$0rp0-=YO7Tu zqyPOhD8Ignn(8~6du9J9*jht=Qj|OBtUc51wF!zm8mp^Z-89}8-cPu;3L0ROw+}j*j@CUAiVXv)W{_Yw188$Z2=b6;7C4&QaOy~5_#wJ6-=`0~ii$F|%{CqE}qY-sxTsmgsHyImNba;VMICKIs zXdhH9#kfm#%+0L?t~=|hejF;5eSUNS@j2RzsSht?EqA%NdP&2P8u^M_m`=wMF?hWP zvAMuRI_&I8P3AI~I7E5O8Ze57qM}5H^F@KmS4`EEmXUf<2M`+|a=Djh&EDYQIHO00=1;BNyrv7J|8c6|1Q*XpG9X5gS5}twla$== zU(aT}=|0lwnk<|w5_XwPvWTypc~u`feJIarCS|{p4f2I61GOi1;-@^@o|N%Mr2CLl z_0^4%4@{|Y7jdvM#zi{GTW~!+#&n#;+hh?5A+m@Ul8bxES6+x&VqMFb_An}mW~t)W z?dK$-aBs)-)F;c=9JT&K(3@wbw4V(_7~jvqRBre)ml%ts4)_SJ=rp@CX3mM!L5Z|~ zY5U*!b2o<7QT*1wtXEe zIwf3LliD)&9{GcQoY^bz;`0N*uT@5EbKr}x&(J`xEw46`9-$a;9uRJ^%)6aa|7otz z4!@n&t1NSfr?V->3z^QoK0Hr&@kVZPblbLmFAB3HXt~B3) zNL_yzt9)Di@X?#VCUDjA-hMk$-X%P2ditn(y0~F;HboA(9+**X9Qe5yUscjRg_xQy zCY0%V3LpJBi!aLR3F!`S!a?xbt{*NLd|{8WzkDk6jqxv3c(?K8{?GMy}TRJfZ z58_-wwlFYD*d3y~lNkWCvv@?nq=4Ltl7YTmrnx3wQvBF{kVJYEmUqx?$hp0{#NJ=5 z29}gtTsw{j#6I2%ox0B?qy~Ri@HS(0Roz9K)SUaLQwcoIavD0q(*MQ$hjR()1y$Fj z--c)BTNj?})ejQ9$@}$Lr+}7=?RSqHxd20~L$eI?(Bitu-SLku#dYgG34<*^y_31o zclrm8Mv}KK;$;?RNIN89B5jx16nw-ynbPDSNj*AA zs5->NyhS2KR3yWdje`;0p=Pk&xg}KG>?Uke_fF07BMPq{R$v45fX6!rrZyaoQs@O;J>O%W9^0*e=FJOxH_0Wx zc=@=Ec$uVcS7YGbvoPv_lcUp>yOoHu2HQe8)k7EWW^0tTkxTGqN!l1H{o*lfzIPMt zUCHgjNjipdVmkW@vz*9_*_>=Qb=vXt{5*Dg@ZPohaz@zpa=cVBBESWA%xBY>vZM%D z-ic$=yk#+d<$!6v5#D6`Dv(7{L3J*{5#TMDLn6I3I!{1Pt7jo&{X=BPfqyuD?a^Nd z)@`_OyUR6(7#-|{%R+M3Z7s`I!uA@9ht_!^nm``>snlxgW||+EM8GIy8Z#D#cscM< z+L-z9Bj>of3w&d1_cu_=-qIz|PTX!14vn{F|K7U)qf> z&HhGlKO&BUrQgTz@@z^@cb-LOMB>gDhgs{?g%V}(oJ^D6fA(fo!4`FyYq-riiwY=o zIw{;PN!S@Ucpw=>zMkOa6WO75Hy;0Im|G9ErdtxH3Mk@pCWUpc37XD`?-VWwp^0B7 zTuxwep75SCrkW!Zod*s!Sy}}VaBml51YqkMA$3)ZrA>+uze>H23+0`i(#~795Fx49 zh?--|sdM=Xbfq64Wis(3x@I6NnYHDmzU$ZK1E;M;#r~9u2uzHEGXwnjHkdUgg`=j> zwS|BS%q=Ay0v5deC!N33fd$-H%O&>c3O%*SS~S)u3xVuR@e^Q6bu zG?l@kdUJQm&2R4wtF`t`sO4( zE^&tJy>VC$6Y`n$XO^l{2$)S~ggN^;=^D0!N5Yl`tiGQuG80a6epMUsLus=5&ph!O zK@Me8ZZ7(A+T8)TrcT+Wa9esx-C7|Gx*8d7`q)*qG~3%aOEEghxND^CJuV_8(vm#! zz0y}~#SIO!=vSrBUf_mm4z9Hizf3^zIL+ zEWMYX+_53v2&!tFN{Hkc^WYkQQRfY{iHed-J2jULWsQ%xac{htlp&$k*}il(PL{D{ z6&5d?{?(QWea?!UH%+%bWIbdyMX^xgDv??#oMqg3CsXP7!7|}sH}~4du~L79j~853 zkfzIlbXT@%R}IHYRm_Mg0M&_Ob^@$P%^S357VRt9vo3WRMyPD9N0NsZs|eZf`brO& zN09?U>)2k^JujJmZO$m&;FQlqb6v)FlnN8bW{>@6mDy~L$4HTxdD=x+^|c8zKDhZ(M!p zd`L9Qt~MtU>NZXcYjI<3Ij^TmCd8Nq6_f_k;(h3o+E&R(;;s%v+tG=WvcFn^VIoI@ol|NX&{$(MKSW_ca2@uoC#*R zvv(U)K3o5ta z=ZpJU1qlBdjB{ycJ`0_V3Mp7z- zn6wf?i_lm?fJGeVCtx~F8x!L+jwOI=;CyJHZuL|qm;WduZ)ddh6)i$0JOK} zvDV>B{P92qI9W7e~!mhS9H)s`?T(#x2Wv?aDp@8NiZiIk#I zbEFQrH>szJ$q-TP%xODjYJ^>dS6=74vEcH<<+2I3@jA)vrpr!wX`A#EzgE)K!!hMz z$&14E&yHv5)={IMcp#jOd+8Sq(nFF{as8ORqZ(t>EXCp1MdtX1wBl$HndYcymu76=;(0ORGBuu!az!Hz`B2w+PDbI*3TKe zA95G2vElubPypEGRyGbPm=iDaIhs^i$^UEq_}@*1I|IV8zbxnwo)hk^If4hP!1u~8 z&h56!lCkn+(DQZ!v4-|Zo;e(!5GzXs5*5vKUg z;XzoYLDLqt46OarMXYFO0j{%vw*uDYP1W zeM~7V+?yBE7HxT)llu8`gG_!12Z$ZkvwF_8gVk#)_`D`3N`FBqKK}2k;U0HP5_dt} z-xuTysk=c(f~JYFf3sS$NspdWcCwn1Y3{lKe=U-lh?4L&ewvjy^`L>D#a7P0a-|#U zl7fA9@%8f0YxbuL2O{46v5LQ2N3nF|ci62fiB~3HDf6k zEhLd+%I$ny>(3g!v!J*J%M>#}#=9yffsDS<2Yi2Fpg&)%5PW-}`a?T zSKuh*yzgLT_L)-WDL+hri;z_ajDma0AK%oNH40}m^C9X#$4aL{cUL=ntdHG>h`U-& zI5XF>vATUv9A^|WhPI7udkoCJ3H=3@{8!2JUyZZo#!Tw_x+G}hn1^A1F~giR*`Zb$-`N|ESjrgyAxrzk77+{+SEZCd&wS?GYXp2RQ$1l{-=h%=JGq zth;Dp;bPHO_V*kQ9lKf&O%fgf^$A6DZ2r7G{~RB8^rh=T&w}Y+kgNZ&bfqn!x3}HT zR_)rDIb7m9j>?hlg$x~wpB-89+Tppc{)bI5-yf^;=E%(JB)Me|f!4MCONSZXz?w%@ zd6n0gs|b-lRXOsq<_vFsP@7jZ09|zrrbB%XPZFAiDr2$VB>PKd-|BDl*w6cYcQ3%~ zU+4H6w)cHPzW;mi^_52Rc}dW*uJ;jOmOPw>x&{S#$ZnrYEL6m`%^2sVIOVk+{QTrI zg|T0YfG5YTr%f2rxJ_HHkZ&Jdjv+C2We7+Xh&cKbroE3suY0M!$0{Hr1(70tYCh5GQb)XW9Un4;@9G&UTf`RGNs6E zm%L|PoLedV?ba~juif^4TyN)*=#L?#yKFiXrpV0SeM28z9bvPd8m{r7;MAd+u4H_< z#8O2ZFNs~ZgAqZ}F@|{6zj77SI4q%S41@dHlq?-+C$Fh4%(MSx-9Jn{NsS}Ze>kJB z&+hNz5;gMtu)VJn-ZH2G3^WXYZll~C;3d_gwWt7tV(P0S&H&y$#OQoHoCq<5w{^oz z%PL6EDwCnqLIc0aL;R*@Zm)aKPcUpv?SSil8B z;l5K3Q-|lJOx)a)EHg zpQ5H{=G+?!&U5Kz3YMT4=Au*Bb{e5n5!bDeSh80gp>)wHG%>j~b`~;l6SSri>i?%P z^}n2mvCXI4Q0``|vpoj(h@F?Q$JuE`%y<|0g~$539x`(|*0o?Iqkc2PRc^>{!7Y_y zQr$;uF_@v$6rTVUlQ-7PZ2K~-;_W!VZ`U5`)&}L2wP2P;TD|7%-=C3j>4v{_&z7OV z|C8VRlg6|1BhG1mp`Ek!)R;H(V1CHqe6&AUSU(f3|G6%4bKFW-8`{|xhGbha5H)Bs0;u2`JwE-s}ltRxKs zPECU={)Qon>qTzQLnB;EXtZsSt}Z8zgR0+Q zxxCy>fgaWvy6wHB(g3D6A38ptJv9!l{Tq+G<0giCWKyhZa|tLWo-*;x`{w(dZfT%V z%_F)PeSh|(PvPZ;p=JOg2we6dzE^IPSbqa?zCyWa`;o#f@Z6H#3l&vo7Gi95{Q!;& z?vDMMIE=RX<>B9W^Aq`SrOq5SbjnS)KY9~oo(<~VVKW9hC>o#zfL}zc&Iv_L*;f?_ zOn<^KhOV>th@_jt`;zhpJ&-wNdY~SKo;5i$&H?_k<^Kxal!y)g)7JWrNb-*lb3^|K z+qh`if&aAG%WO*Xx1iXefS1j-IfumHW!P_S10CS~U0W}h$IhNp=8qa7FxzR~+=JS% za%06VzLsd&4ualj=+J|mRf-$me1_})#zN!J=H-e0QOND5rA!vAEa ziNt6K*0L~H*}Y~JQ+Dri`t$&a;af9BIi+h1N3W^=4Z9u#nbctz#ec=@FVX2>@z0;a zA8u}7s;_wbwB&VmH8VPivJ&%~=S>*3S5Eg_e9#V=jcjPQZur+2W7ihK^Xl+a!{+F! zX=@lb0-a#$MiKOw)d=MbiWV~^bGA3Z0ZGXJLMwlGJ4ZEgbzpj?2u(;5r!g`!h|pt3 zA|%b@(dLia{qFN`iWIKYIM{NhBCwnu3wn!w@Ndp62ry5#gkpcqA*vIZ6G zn8$_|%lD*k{3i7`j4+3>c!0Za(|}#zN3@$I)SF!efPSen&;5E|lyc2HXdj1EOAY** z5@*`;NAL_^2GH(W5S$dpI0{E7y5S98-VlAr$+jnR3*k28Zh-t7D~=OU$m7DgBZChv zgND1?4$N}^dKZ{pV%Hrzl*O%CVfd;gIvMJ|Dq4r_BgG-YCm`(PET)3)`k)`k|XLUd&j+ZD){pmx(Ogr2cTNWYsK%MR1Uf2o&>`k0F!cHQrF-VHUlPd)nqn)o=9XNs`-RM6^ z_wb6pY`)-A#=o(ojV9JOva@S}FS{udgv(C{r_L&KI64h*xT_(>a4cj82ymRc@jeI! z*{E|MmEC;z#CgVP=NZ6CBSYfaU+P@`3yWCJ4cXovz{mN=hc`5EP|P0uj5v3QWLVDb zla28im3)ePz$yH>(t`+Dm7=VY9x z3z&eb9gWWYRSMuwW(eGj#37>(zb7F9(E7r$Z{ClWx;}tI&qP~aQB{QgjhM4zFZM^+ z$-PN2uKq}+mto$u$Y%=2QLoA=5L?J5oxioSCK>-JNx~t7f2@Y}{9AJDp>^0WfVE0Z z{&x^-^6M)6e6uP#`tjknvgmb9chL(O4&$i~{fYb*K98Q+PE&X3i|?t??sw0`swM*@ zMGI|dhHy{3K}_yDh1srbxWd+pV5EAkcnWkrCTz!?JFDfHebDpx5A<$MLS>yzn#+ z_9k0TEv{QpnKvtA_fkHAm){^Ove=K9<$t}oRcxsD5MOpXxmmk5+v8V6(mJ3Da&h8W z-uG>{&s3?drc#UB=DZ6=y$|GQ314mnpm-uv>3-wc4dDNG&*zIQ_t(SpC0G^o-!nMKQaZfiulVq9hROL2 zu^$7|ZT_Ik@?;E@9m3kvvjz$o{ue*U#IdC6ASG!&r2!yy?67u*zwBm*Q!F$zmY=je zmH3SYc$X!qjh`^LJAVV=*!{+6z|B(Sx5ah_M0OrUp(b}P(UAB(TD+v~Sl$C+@+~>F*3naUamJ*$3u=5woR3^tH6uRmCAFE0X9=)9y+rT2X)u&=XPjJi zY|C$+4qXg~w*5zf=^l>KX@56c;&t4HS zGIB31N}Nn}US{Rtzf|)3k1{RpM*p$!EbafACI)?x`B;*TpsFDbJd=A_dFhhUFEyjk zw;a88B?oo#+Zx^5_!?<-ATOsFAI{=^TYoC$a?ihx(_GS#gFV^;_~LDxU9)c=Ol%K$ zd-CIUljzY|&wfS{xpX6ZP8s}KUu!CGI3m|H7#}Gj5%|e zKIwF~`Mz=?CHViS`l_%tz&1-PmV-QOH0 zR|drfbEn)Sjf9ko-_YR-^!;6pUkb*ETFGjdOWp;uI#^uaNUnr z1Pu6n`^w5E>}3fa>2O2U{jG<;Bk4x{KPQ^YZ~s9dO#XvH#JbM)$<#-=+IAXLN5tMT z4I|p|`V-|ZhM%}?J(vz5nhs^z5qkv?e_W6QOtqKlqQP?_4E7Y*^<>;h=wF8rNpN1c z^)~${!gq%diydDHa|{0{^r)zAD0pq!Ztz!x*)E;i-vIU*?%w3w(_1feW+k|-Gvg-H z&%6+61~2d19tB&&`GgD08F)5q=5jN(Ti2QS7Se^U6cuc>Q2Epxh6P)c{2K@FCOFJj zMxN0HSPRzeJN%tDxtH$N5Jnx2zkAqkVh=CO8|m12Ve96gR&5>hxwSaK5|jhFmIkKZ zo8T0e*ljN#wYxeryDu_*{t<9sb_TAN`TwKnl+G!{2aSR^uD6@7Ojn15%863hRF(*y zE#DCD#^12NPGFc!MwqscN=9!dh~I^LLg}V=R#~?#d!y9V6q2T}g8Z|&d}Tl}#yx6@ zSLY0OZ9!T#g1#;ztB+^Ws@C$Kh(N>X`GR7M_E;CJr!M)0uwRBVe%q6n%4zV1k@OIb z%hMlM5#sV}JGR$&The$oY=dr(Tz=Q4pN{B7`j>~J&7%Bek>Ig13l%>e&%UX$dDQ0u@G_xy^g4H+Mcb9Ga5 z4fG%x*Y1y9I^qTZTFEYgR5V)Mgq4jzZ5YIkXf+d=f8Y}?41f5v5bZ9r9dhmXKP!G0 zi_`so;LNKP0kXlumS{b7?z@!zV*Syn3Bua1D=;>Q)p6F8xIAondm9D??%nA>;vfVZ z+_{te_sDMp^&1}F2Ki;A*B%s&YJ2a|;=6ncwq8~H85%I3tw`c4H&sj**C2n$F7>;m zLaf{dgQ_Y|g891EI}a;9!)|J|I1$%(EOU=cA>iA{;GUG(FY|a2pG?I?nl zZLXLvl={7G7un&ncOM>VPd2prV+U#R1k5fp#tlVTuKfPC=~9&4bm;Vc4>==eiS53% zC2Dx{w7Z6EK5X=J@A+x-FcFRRbUhM{+N`!4ud>v|YD*E7#rjGu;PR|g*s61wPr&=4 z+jtb~)a9RYw3dxA_tI?M0>h~A7q;NpP>lOyJNL#HE}G*6PZmGOmJ8oD)7_s{Gi+w@ z+wWHYc-Z;|ER2WizI)$!+WbDV8(LAT>Sd>^k%~txa!4tF^0Hyj8sL<2EPs ztVOYH&pt#uX(kJAMtjg#6?(Rx4sh7pyz>va|J~fX@nh5D>>2;vBG3J#>E3=dfK@@8 zta^3t?bVXBI^^wN&TiYq;GMP)nTzm0gXGlp6*QmPKYqXNjURQ_oj;`uS0?ObrQGeX zg$&wh2M`$+_@%Cf+s)U?>xTP3ulm>>Kin=is30Y~Z_HhNaIt(`?G$;k6-;a%!LQfU z_e#gL?K~GJuotYD&y7t^A-9`dhl8sDOa8B4d$ND)ICxPzi@50VZ-v!xs$qM+YfbGA|E&>!vItg7?Jl&qJ=9X8&s$`xq* z@4=jm6?T~4n*uZ5Lq6wzF*u7~?v3oxbM*3C)0O4Zdx`xMG(kB1XTAT8+r7%H^}RU5 zR*T4gK1xH+-_hpYB*@qAFm05|2Gjd#}z>iq5Fbz*Bz^`|!$vuLH1kaJi_e461 z!M|>qyKtP6GAIcK>z*AA0-tMxdjsavSEqX_+#rUY@0cUrrJ#=M7_Cx=@YVK@u*<>e5In}nO))y zzXqoE-t9@m4^*I$%G_k571;RRSHrEGEN6=2TJSDj|6EKu>?CZ&h0^k zPy+&*g$y#261{GjZ_wc{uZix$h>M0sw6UJ{`8$c2T||yM2y1l+gRI@iF+=$Bz$j@p z)~|tHrIMMG-pmOEqI3U7B10Hn&U;Y#gMJ@fv3e~ioIR(E3`;Q48GqKJBch}6(qBc==YtwxlSLfb__X^s`eS|L`e`Z zgpIqo`>vZh#RWrFhMZbik(mNH2~!MuKldk-oASAR<-GN6hiQ#IO+-k7sJC!OGpbaa zwcDcFr2n0P&qdRso(qe_&)jN)TX$&DEZxn{&5=nhoq%MN&=*AAfOryNx)UWorl7AG zC#F~k*!d>#b1PIB3)0;S4pK{25|Ghh%a(J;j#eowfyCQ|eJmtzQ- zK{#XD*vENNp>{Q*aOBYy6u`dDbR)KVU`{O#jO~x*NULAS+1vokpUBy>)X&OOZYq|? zQzXHQUUL%c6&}&og4rpOnJ!guR!#vG%_+d0Qsonl;aD03q-zT!=@>`xxJYZhTRnRe zO#TonUNj{KF=a-h$TaD=loW2XD6a|yGZk!wZWpQV6Efci?lG@+kRt5Rrw7|d^uR8pNTrr?>IG{q5 zP&^P*JaA3(P%E7^rYX_KP^362X4!>R=+-hoqhk4ll!aU$6PVUVP8ZbeuR0KB^mAg; z`L*sEhOm9Q<+M}0!?>d8!VK5gi50EiSZ8h5570EzGHopqH%sk~^JB8RmYKW^AG3}l znW=CPb*hPJO$L_pC-kjm2vkS7Vi#=?5;Ph!m4625XvP5lXMYJ!O$l{I$ks(KH|I*& zdZc$F!R?wy2aL4|H>DzJ({};D2T?&GSn2i#-il)yu*Ac*@@F~JIson%jNV+@U1zKF ze!`o7O4oXTvY^CB^GfV=#Dj4;PF}%yne;~Zni`oSndu*hD?Ozc^RA{;IPjU$_H&jF zJM5!ZJmMkoYnh0|xbTw-%w1Hi*z+*OF+-P(F9VDkd-{#v0~1`L6{6&F#!`*9_;Sa= zLBB)v61Z?|^pNEYJ{g>hNyb!B4#6*o5F%LBz*i7$jG1ZnW`>Qsqdf_$eW8BRTNS(?io0ugi4L^w)#huOOBETJyjdYY5$_po)$;7<> zSk$mb|Wh^DS6+%rA`! z7!{EZs=xe+eIcI9`jw=1(xxvF+FazK_sxTk&tvE@Bc7kH%2X(rPYTOs;de*rPb${~7n2g4?azH7a#>B`>tf4#PJC)uVQ~O%2;Z6IfiLiY1YL1O{&9`k*ccSaX3? z%YGYsz!oCU+{oi!!|xXi5Vpo)nwIzRa8;p)8Q@5G&)5>#8#c(E6`Jh5=))f#J2yy! zolmWF3pzy33%N)JiX~fudIVyLKpSMv2gC-C6dAx!0Alws>yxn(iF4_<+HPdl{&G{E za((B{m5>>wB)&5GN|R;s1G9;i4e*(gOi?`|M;;Y-wSN*wVOM;umXt z%%q&J9)L>J1)qek$ zs@qC%vWr6IS z1P>wS30g1wChp-krh7v%;d;O0OXgTl#_M|n03gh{Y1L-}JN}_m%$g$6QC28u8Aswv zrDgOfBSc|+v3`XL_vETbAi|2Ili$8r9tybkfDRto<8qnmVOt}4CA0D?Lm3?SyCl3# z=V|o|F-KIx_yrHcq7IZCIzaBq#0*WfKGy9>h$G`rhB@U6v_QcND$WcEG_NY`;De#) zM8?c3Oo{ozA70pd6jKF{ulf(~$9g=Pqx{yOd zKUBe22s{*i-^9w*m<#K7AzP$>E!?NW+VDd#R8Yz~0dMAu1(7LIKl7kqFWT}Xn=~Ma z4_1I?sbX$-RD`8oM}?}$A8PP=IZP%ZCO`x)Dz+uEk7j@!z63V8`ra<5B0f6{8WO?1 zvp^w)E_VDXGF+i6F*M3hI2kM6$c|+5?XRkmV9a;6{$`UM%OjaYtd-zcXN<11P zl|JQ^YV$O3pJ4W|4qRAE1_lV(iw_RjD&|-k4-Vhd9X5C^#tlYM@1hmG4CBCCoHj!e zB9K5x6?P25)MRh0T;n+Qn^FG2oL9tz zO+l-;YPinpuuMBglEIutO5-)7`4;OYPzTP!L>rB zr80+D+rlR>y@flt5n%cO{iU0e1DQEJm?98NJf9u*l!+eS3Wep4>0M2JQXd$2 z3QI*pp0>RY_lrA@(^cn!1BtzLVzO|9N~8b>68X$XFfSEr^drKWF5zUR*eP!|@JqZ0 zC)(;*$m!A7*=};42%;S-HeNO?E(7cyO)5a6D3o&W&U|>xm)?}VlF0mwxAyKys!V!H zpcCf5MkF9S^Jq;CP%Xu3}m?CX5^#2zc7ifTw7^i4XL887;`0E)m7tDkAiR;os;m+;Wuh z8H>{WCpOKT{Qv3f)h>Q*&^yA`kQTh!{ChElsJ7*Rz%|WljRco$EHjlvo~avnPzBdj z_#JEOp%d;8($2I4rLwhmT*V?%zVjO{{M;@v#U%~I7MbbC?gf&Fp~)^ga?~3(c9N$B zCGaGKyIU~4%aot*5Ke?P(y|P`r2H8EY~sclr1vy{4CkhlDxF!$T??wo12(!%x?Vbg|EbOww?xO+9-BG?W&_3*cUxv$%*%6+T!d=y07Y?e>mph+L zsfd}WRrYT?=Z-Fr-^wiPs?+HIJxaNQ1(26-pz`y1lDgqSeoOCmfoXx^JSLd?1_zo& z@*+^TsR1>Xk{tC$wtImME9VLFKXd{}^o%TyQfVdJQne_+xx{cD(p zEf-Vqy^FPw9ouna7|EKsFacb*e2JqBZZ`^;F&HUyB0n9S6!5t07@r(%ob6({%{q6= z|F=N^y`H7baH#cECq?Ww&3oKL&G}pOw2HPzLJsCdPNhQXUn(K=B#%8TjC!S`bH##y ztkJ&Ux07UBhAr;b70$mGE^&JV;THl9K%ql~MYrHP*>9}1_;p=3z6??6+@pRGmICmk z|9t!FP2c5;*k0eK_8;H%0=RzY_||>t#oR|eHw*$o+oz=t-SQ7`C{YL1O<0bFAQP)( z9BRI!yb8x7F2V`v607ip92)qb9?HF$lD!U6*?Ar;1?jjDr3EZ>j1W?>>J29sxvc} zyHh06z*M+ivGkZPgW{lkOm5ZA46!DXLAOeaSl=W4m%E>MFUDgy5~sA8iqM#|q9BXH zjrih~5m|^nP~+vzA+vFF-vh96L`nc_hF)Q$>?LZ37eIa-70N6u9S+U0C&`e~jSpRi zJiu70~kvhE?a$A zuD$5>;W#kX9F1ssPr@(siHp9a1m|I;WQpPg_W)AAk_tUKx7LY_8f_#kD30abe?F!} z;7ov~LI}&jrCLSaSwOt@bY9T+v{*=KxeqH>#^zBjsP0d>cqN^jXeFBjd*flU!z0@o zCcRd?VQ=wfLx!d_9!~#|f7yuKy(uxyeA}Y8GKL{B!*h&nKqAUV;Uv&W(v3TUw<5&G zl{4K*M9##QBF_R*jHU+GTO7dmQGiMY{X%!}U}!_a;-=P&BU=Dg92GJk8R*Y!4R}4Y zt{liz=^?Tr!J>@inDg}raCzY~{|xPg>1pdtaNfT$tAJBFhE`RR-&hSO^QwYlf6b1b z1T@iwF>A#Iobt!Nw_Cpq+wxR?#muXhkxvZg{zp~}BZZf{E$gr6Pm zx#XJmxVD)*s{~4#zdjCr4Tw))C|HRM@m670?%wS-z>Gr|N@zDM2z&SS!Suh~cy2jc zCus3IW?FkYXL|9S@qIbY3%hRf*m-VwZe7H5ZJj6$)gk^=ZuFO6N%0frxOC>0TI?{g zoK7%Uv!)mfp`6YC*UQxs_#ZAnEY?Gi8RmQx6O8({5lrYr2@&ndo{Fw&IXQL)QyW7A z`mUETCvu?Gp79ErHA!4M@}xA5%J6 zsiYy{$MO{AZyKu!3BxmEMf^11I_6utK?5T^4F~TXB9k(vMXdnu?PzNsTiBK(RZKU! zYiY;&91ODp3xDiQP^qg&dI3|dB(C#6%&T-8uU4;SuZo2QLXOp;Lzv0L(TK7$lHYdy zd6!1ER$MW)a0wkdr6IA&iyguQ(b2cSdDN*mggFi;h>4h*RKD*04DfHu!~2Ymz#o6c zvfMwBM|OglEf%O%dSu-dXoAEM``oewp(lATRZ=f-!ag>H=h>V@9-XVf?HhRst8~nL z`YFPFowW)$h2c>M6d}XLa6Z}8+af>TGk(8Mp=&&orogAVa=)U(3t*8`KBtXo_YcrT zN|7Z%NbYrmzv}t)(@~D9K{)#FN`xM&xSI6|tcw}q_*Kv7$c1raJL0_@cv-p(C zlBGd?q=*SnmB4RNKY%StA2Pt!)c>-K3h5UsuD0CFrqzz5L`-YaXb!l$_0D@=6qmck zd0qLV5Xt-qIdp{}ds+Y3zC8)Vw%oGF4xuK`>!?}~+~47J@VK#*O&Wxx0-o>_+UvoL;oLZQUR$;p*oVD7(!)yl<99dI358+c`pN2=O{O zlRMj*YPa{9-HI4VSOZvAmoq?P|( zPrFwNU(X8s0*7-E&_XN-*0(t#@Mkgi0VS9=vOqemxP-T>(r@$DBX<{IAu|=;?IeT< zdsrF>qeE#P4y{86gM&JoO7Os-JyeJq^0{Zv3?m-;Z?Zb&KGyJrk%&wtVrpxA^0F!p zH!Ljum>9lDCU|P*76ys~0yav8;?U76h5DSe^pX`s)LdX^shqk9+S2l8tB_c5&6Cox z5GqaYIoJf-vq&wc12%wI7rK#OjSQB@T&L0J|8 zU9N&cyq|oh5XJ;n;wsO|*`6hMESlU+{B>Cm$c`mPw^1`a9*E0omLk85YJmH+VT&wa zgOhN2+DzJ=(GI>$c8UfX<@m@EQ;aNGGqi?6smG=DC?}lm;{Z5dQgwzUa90^rB_V@Dov^m)#t5a`N@j~W;C@VY3`c> zs78DE)4ugMb@-QW0JA{h2`(B~@ky}+3yi1|jI?H~78+1laywGCN>iZfOe#5A%Qs(E zmA(`HPsVTcjaJLsDB+nkR4jXW}>FX~z+A=?ozQDhy>3IJ(+2!=D$J zPZFb*EG7yj>8ae5Cb+_0Tll|bFcwiRMwUxB)eL^v`0)N zq0lK18UB+W$Uh>1hH=4UKKv|m`Z}n}M~;Sw_IY};xJ&Xf<=#XrG9`SwY|5z$)sppQ zS8r1?S6bNniM0?Pe>z6U_10~apT_C&$@K`K^$+b{l#)zX7JG#{(bZnoGU?MVl_!2U zw0zvv+2pY+EBW~v{v2RR3{Jxsoy;SiICQE4{W&HZ2W1~S)-H`M1i6xy5N)H$9Y zuO@qQWo-9%Kw+_Ax%Q$N%Pe||3v3E||NY)dDMT>1%Iq8V9+hb}6v=9TXEwm}$_GAB z-M>smanRTI=25E%=-jMyJDJHtH%zDi)g@%Ti=H3|+aPN0fLQ0fs|Y9Q7BLCFO!tHI z5p3|#jEGp|%6eG8d=sY}X7%|EWA$yYVPj_Cfy68$+bm{4ev3@L`*L|%3v577vntm^ zqjM}^TDKQTNXI_ZzgU=HR<%cY%TtO$TZMS)Llx%T;NKsv4nsMK>w6_28j$TidOY(zu%&D)rk;>0hWQ#m8~OJ566K|LIM=b?8ghs+LH4Ftj&A5J zB}6wkVpar?jnv}q9EXd*i|hWU{e4mu>E^xl{rWHwS2~Xbzbo6P!v0fAG7DI_;vTEw z-gizXe@s8!3Q0Bk4`;J&!C2?$^*d5F$FepJtkW75xGs){>`_wMZLl>oFxtJD2~LB?C{Ef+}TUe3YPo%@+b$^_pTz9Be<5J0e4^^8n31 z#WNBA6Bzz~9d>g7VZ_r4p7#xbE;nMtdT9*sZz*U_p~&Dqg}bO z=_m0{9Vmj~CS*h|EModaIR*O4j*lI7|IRPR13|#L*oo$u3Can^jW9$AF!LijCE5)f zjupaemW4$hhr0Bc;LYL_TbE8ruL+9{A9e|qT3EM_#%4S;pN=rn-=wBe4$odDUv|Xn z053sk{Ukp1rfqc*v0)#S95qYj955^bYuXq+>>*$$sUK2^Ch9;Fr;Tu;I^(}y46a1M zz^dMV{VZ>%Ou9A{cvha9ExX){RUA4w#tmxe_rc=Syw?i!+)DO!(+vKvsf@m!4tj{U zsCxJ)`E1+xxVqH{6y`h|BlM)IJp+JpAs*5;de$gp3gY*7Vr`qaH#7-Q%y|_8NBHH4 z3kLBA&#ex8{s?)Mgh87U`=U&|hS1s@xgF~}vODX^f22H$fXNflT#z^Gy z8^bvVMYNrDFvg7RniD;+tP9oo7{+KS z3&2I(6fSF(=xMAhx)WNt9Wt}s3Gr2EWX3Jn9Rx2mJ7H7br;9yVP1$OmZF2e=Z$C4Ylvt!%+;AX2g9 zscb;QlpaSOac}2SZ}n2IziXclEUd}=nF?p2+p;06un+7$q72qlTp2gRUXWN=%CLae zsHe0CZ-)L}xR8|B zHloko=l2&;L5ju^w-0kLL3W`#1aLPk^x2LIKO2w|8xa~!?uUg&oeAuz!n3_3I*gHW zB!O_=I=WdlKT^WW9*yGGvpo?ssx8lOyI0b(r^%ya^)^4}jr0}LLldME8kwZW66wUC z^<3B7IzRk&r5YgN3p*lN&unkU4MC#)n>xWV^yDq>3$*uf;kF<-=^ZmYy!`u2W?Yyf z@-vXq2`Hs%ZK#=3^*&&?5|cI9b~XHc*2FMsm(^!%Z5P|Jg`pOiV!_jF3SP=%wy6Zsos4(9!kIi`QJxTf_m` z>$SWx;0KkWWwxJ1%H_sb7+X^rh38^lBQ#0zI)rfJilwA@R(1rrqz7_a{2U$y)ZM%N!~=!cPyNYjTtai^K4N z=>Zaz)oK{ZNln4x;W3D$UaBHv22eT87b;wCZVkoe_UxuO&xo*^$t1gu_0t?;o%wpL^&#_#x#@uLqfpARk)aV8XakxN(;{dtAl-9AY zlOGe~tnj`ER&1G)`j6>Ja~<3TgGsSsVFlr@3yd0nLf+_Bsf9iVdBe1ueL+oBN`qCx zLRR33`q_T>{g9G+yADxgbVsT5lk1Y>_d#FVwGI+Le2s#JccGUZo7Q|0JFn_xMBlXK z60`TW$A^D9Bf}-X*y4rH0tPWV{Ywu@>D>%OX{gHWR|`D>Zh_AAWY5p#Isigo-Wb<(F5s zH|q8SiQyAlcgP9H{>r26{-H_EH|bjyLmV8W(Afz%&h*z1Waz4~$%C$4llO z^kQ;KMT)^jw22V`Fo;B~)%c>OsLT6>-H@&<0NLg$#w zlv7nh@kD!(Y>M@lyAuK#4Y6D=)u$4mxe=~<0EDX7DO^PI&=l+Dju8MvH?&e@wBW$$ zP`JIK)XG$_=ZKGLRie)J*e}Z|A!g?^xG)8na7=_vAjn2Szr?k3PSeaWp-Jn}qrvSJ z3p=?w6=&$62uy9G=j5jvM1uR*6hUxR2~@oHMt*YeMMpiN8q1WOru#E{F>sA%WK%Ld zP^wO4-LNECflM?7nhfDPdk?jR^6*u}XDMV@qttoqBSpjEisUG9&pfN5feL9ra83qokKNJN+~sf1b7HJOgYu9k6N8tonyk8ufkalOO7isU z!O6RS4t=pq9TiUd*wx}PcS99#Lh2q(iniGVAt{(HEXTeT_Rt&mB(vFblPn*GhIM$U zd~R#Mxzktw0})2l3n;RFo&FVTQrOU3n>V0wBqi{6D>e3IA%+jJ&FT3^3foT7q0=fA z>sI`8L2n*f0m&kT$s=xs@X`xcl; z?U4_0KrF_h6?>_s<cB*JfV~qzOBU&G4b}31?xtIo%6P4Ttl? zcODdBA7jFEZ+~_K88l@w(e%+vb-oBI_4<1FOk)L|J9uobSOJ7(NxehvNOWwO*nuhf z#TX0ak+dcbH1PYuf@cmpYEf68-feGQE*J*VL4{USNJ9kGauzXz^ z_!llb6y63O6BIoYpf%{wpE>``_VWzpt9zb=hApo(^ZYw(o(14Wx^Tlf6q~V#MBUvANZ8JsZkHT!UEVE&Q0@pCnaJt}O$Os9EbnypN5?l1EvU(DbN8nQZ}<8Jy# zf3xI)9l19?5m_%<;%p`#WgO-BH&mNb>-2E+Ta>vj^I;ehF6N6^8v zFt#RVR=!HZcO9}c9~?iYCySk!`Z89(u~6#d`Z9(B{vEDpwEBbUz!LD=oZ#FM96_2d zAN1O_2YY3-(6>HSh&Oo=#Iy#{42W&TUD5%TdMdcytBx9fD^?;&JB;tWUJj@xTl+v+ z{nfbUxM<7jYS_CK7N{*&nV%-%Ua_tno&s&V0(WS>SwesP*^=7Cx90wWrL6?<-)~dc zA)A+0czFvF-tu)I^Jdjcb)s+-a9RaAj@Th;pd?C`C3NR zfzw~|t$-|tZAWXX z{P?|M|ZkE-eN=DQ;^5nHr;7j0 zfMRS4Vkg!7xXY5jyql@xZneM=XV7VT6=7UffZ}o0(jge;)5kzwi?U8kX@ykPh+fbV zQ55u*E*tryn%>37vU$#2;p`yRv>f?0E|BhIovVivb*e`eaGk^EKWfY*1UI5Z9yT%sA++~?4o6JWI*%b-~Y8W|gfLT}gA1gig8 z3}@!8M3@U@M&5w?)iNhw;m5{qixx9h132z&z6r5lBv3=>9Ktbz3L`kAg?4XnMJ7*Q z`jZOSzXdQLP_apvrjo=`jbtbc^Bckl!{rUTQq{?PR)hEEfEo=X4Uw%b{aY-w69`?z zj>MF8j!9I{3oF6`_&B7>(O6)*V&l4O3g-NhAi)oOP5v{FZH z#e<6SP>C}OT|QfExVH1l105i$p$4SLSQG3X1RLUV<&@~HdyaF6R%W0EKEBC9+cGe_ zav*N_r2e?4{NeNk%*5BGp6S`ff65-zmHdnLq@tAy!<`-J3tkl`A{$lE=IZ8P)}fp0 zpL5f}oZ_p0dPv5>{O(l8!`F^6Jq_P|ZDLg~%$WKKFsj97UeHU#wZ5~l1B-iLK@A@w zlHIl+j`C+3EtPN2!0+gRrV=Hu#FQQiQ1~?lYoXt5-Za!5N&v zHRZG1vZnu?^SSPyvV2j0FnogV-hsQ0B zP(JC=-;!O zRBr**GI$oh$Xx;<2N;)m_Yv+0W0`6Ko<8X04^o5m9VnP9kMf&LUbTj+%SvZNdQRng$}aKyykgg^IYRx?!lf zot_pqKEt6Nu7--JgaHf>PMAuvUGH)No|2^v1ih03$GpWriYpbQV};0h{H@Ghm0Eli zQ#@rme3gA}(?Tu)ES7M0*R#h~ngSi@J8b6aC*RF3qNvOTgAB@NBh7sH|w5#EJQpKuh_NMlCsJel=LyN>zLnb%!%p_QLt6cw>B* zE?&`)ZQM}n3KPty=fJZaU;mHo1PJOZMpv4A!1+pz&an{aT^`9$j>+?-*U9wh(@_@> zcOK5bMGqAuZiBgHf;65ex!BR}N%}$5m(%(b!IQ}xlVV7_kGve2$IPLd>u9=#mXCOG zP};Q60~LY)chSHj=CRd@1u^Er!Q9-F0(sJHRwa9}7aCIgIM5n>FfK|LWPXDD>lbt* z_Ze@tsV3Gl87(75HbCtR$B66_h?*h)x5A2@PhK zSts^nCdm3#Y!yph7BiW_BCTZD1=W3_mP% z2}mPn^%GsEKm1Y$t6+^M+e!t9SO(LbkfrFK^xR|$3=2^wj;zeXnasx& zq)5Qi^42i{Gl*>WSna=z!WsS#6BMO^-=+aC=%9KLt04s zlVP6aOa*Gck+zcrnidu=l0ZsUZOUhGlp7Vz?cbwGf>cxdQfhc@?4g5)D_;@*Pc!ae zr4ufnf?i8Z(l{yy6i4|A1rm7Uf4SThq!}F%Yn`TTu^oO&!Q<8HN_nqsA_$Wfck#bX2k2d)&TDe9c|V`6aq9 zd@XQy_+~2MQRn(-y)S&n;kCyI9k0{gK5(NVPs74sF{O0y(mYSFGLHqUk-Xe}vT@Y? z``%wM^&aACMGj8KDMAU%6JcJQa){K?@b@r3jFx-Xc&h#1%|052Ecg8E{ z;+ZW@Wcj+k2&w!Zd5?P-5;aM4M*XxPjLWccBZ=KS2)zCumM9OtN7knsi{Zs2dAw`> zgt{JNeJ}k5R9Oz;7L3T^Nj_z%WHigCI708&HkEO0zVlL?32nG3W#j$X0OvF+YXf>L z7O;A_X;*cEee2==Xt!bU-<|d+?=!MJEv$mf7xbz(+fePwC$>dSJ~J6M5p?~zH#UaU zD;lDmTt7-0Ti0#UBMyO|uP(s-&H9K_?nS6S2l?yW7X3Sf>|rd%zjcS9?gi=f1^1IN zLMdA3@54*rX*Qp4|A`>gx{k#U5=8t;bBV`|;}e{`VvG=XN*v4`B8-M4J=`2!%uUem zK(`q_`LxkD@(-2G?hy=7m@E9k1810<9a9TJEg*XWVkVpd<^wWp_$gAj9-Lm8oh9sGiB-yS7eIF8| zZWDa8J-TApp5CJpE{G!)6110|RY_M3$96&xnOxFRILtNxp!<== z7U#Vvl7}w|NhlmZFa9#2C1Jp2Cj3nJU~pnn;W82p9k62h=dAOQud>*eX16wluksj% z-Aej0UevWgbjUvXD>6vL|GqYWE=afm^5$8gAKWE3($MkA;LXSU^&3`GSU}&cG$e<< zdmV6`D^N53X)uM6$M|5|-q*7b5v|KpA{8yx3c;QlmN7R>v{IH<0OWabAjuX82D8`t z^BjL`Ew)?_B?RU zK_pHgEZ+Fi|B*ib#Rz`j=X%I+yYAIQ^DD5+9+5Pw33zZ>KxM>PoG!w8BA)#1LxdOu zJjK2AqeDMg41ny$bPeR#pl@}O41$dRQG8(FuwA6W1R7CAAeWQR1#(f=f7*2ba-7@s zdw$(}aMh>Gc-B9UOtKQycJd6M9PX#o%s?R^b(o5)}p!ty}W&2+1kW4k2p- zJ|0VVXd3_38lCRt^|5A0Ll?e7F7$@&SwHFzkKKNa)ytmaaZmS0sg7T^*ao>xNTTba zW1}Hl35cZ*J$bu_TaV^BmotkWpKE4MyGO;2W>>_G2NRC`yf5gflXSCcv%~o$;Uck+ zAQ)tuS3u2e<3qkysK;cn=V|f2r`;?Ic+Clz!c8*Jq7l&d!4>Nbd;q}-$njg6`C1Q6 zxw2osDE;ycx*GKu{kOROSn%*Tr{n3gPYB*hhZ~0{8O?0`U)qeSOedg))XwlJP!Z=X96pnAff(42` zJz@|H396iK@mm9be8XX2&_uw>vKp(EaRyiLB^c$6}C9RzVJzu_f z=*(^^Ts*rIb6AF9PN19aUE%E>AMr772-@nM(HN1#cGjUm&%pRVnPk61tkCoeQ_5Vamu1EKUorv!yl{CRJq~ygjVlxQ8-7z>6nh7`X#Wx<0g&^TI81kG3S<&$*v|BAinEh-g zgWbm)ZOt+M9*UIks6>xkRxv$Tm$CkKkB9@3VM!0#7c7Xd6*s3RWLVG_yS#T@>N-tW z>M#S0bn~KudwKBk{F})gK%Fx#g%q7|FAU8xa1yt+y%SQpqfPBg_4Ud_Oy9Dy-cY2y zs0(?=-Zr!3Gg${%ql_`2@rVp zHK~JXr%gZQS5zL7%5lx#c52>RcKf9cLXXX^6x_zEV=h>nt37LNVcTNZJ#ugVmlTLzpw0`Y$ z&S-k$GVUSHZQELm<3;?PsDz$@tm9U*%d zIcDNzX2;6Zdb3^a&wTNR*=}Zb`PG+J-5N|02X?@T_M2|Z&f*E%uC{at+_d7pFYkHU zTp0zXE14}&6-`ni3#M32XOO#^cPKctYl!o`D2+Bgn==R8_0bOq89VSAARyqE=Jd_P z6B_^d=jMr@Ub~!M!#Q7Qw#JW!-vKu0n=msZsmrYT*1$eGN<)}`v*hwHd35GjrQ4`o zb09tzyD4e`?f~EJlw_6}Og`<41figG4`&0*woJ6IE$Alkg$a@>3B`Q>fCZ3tWhHNy-;uksfnhk@@ zcml!@X_G-XeD$RkI4v7N9X<-O#A-Y87ds>rbx4{Q?sp*I^~`IViA=c)UOx$F-`E?j zes|IniVJQbu7QQHF-?U(M3LZdEgjol~}M%_iyA) zY|_>|+YOAZfp5S?HzFZ#9**B6DT3lvf#)R>WY*h>c3{Noa?4M(R%<78XNkfL%3Kn= zwG?xaCp0BcwuPzhr?KUV)RND)!~2jm<#(=lolI|rJ?nbT4XS#%@L5$j2Rhahpze!L zx?L6$(XFeoZ$iTw4h#$>QQpr}u@&7}VvV~ym+vajo{<@L+oZ2KSKr0vPM zPskiVBr-{;%BG{L30aK)Jf62e^~7{ugA#eYKvJ%0dWI-;lt5m(Rv+aq0#*NJf=0ZY z;qB0aq|PUGli6eon?qs5l@Rj1R~8_dZ3oHCiDAG;ZUH`2?P<2Wpgn2R9h50Qz4J15 zZpIS)jWd!q0QqIgaVU!-!C3kU)zA5m$WZTsz*e7UY1ER{xO^_`cM84p%6+ES`KHaJ zjfME0t5R(@K z3JF(tH3>U&(EF;z)tNYNP$yTwu+P2Zi?D67ERM$2T_^ z74#yR9~khDuhhfM_!c?`{Og02qIzw*UE{HO3)3Ur6f~*R8HyW2Qzs&8wdGxRIvwdU z>nk24fqIZ!XfR*wSHu8)s?bwbR7^&qRFs?~;~pGxfbI4vEKYT~4%I8M=TGz(Px#u# z%Tb0(BI|LUs<+!Yw}xGd3z6-oB+%(g<#MJ$5p5~Hzd=EeuwT0vT1o7rF*|6IaeA3QM-a68vuv}v$-fmZsK@oY8PFJiyL|-7D;k+JtIGi4yW)D; zt!Ym>9m^@AUz`=h0KLdbebIn_)*~VEzFa$hY$hEgL)$@MliMU-FQ+{poM!!`B)ogv zwgsmV05QAlL_LR+lW0^0m4R37Gv&I2WI`s@0k`7=d7Wi>oo3PMaYec`BW`6fi}r^! z237AnxmLJY9`6u}mTf~11f>_{wQfd928Tx9v|-H9Yh{MdETNmco8V9(#W zIC8x7YW+x8l2kZtfaVy9@QN=%p5Lu7$F%{`hWX_Qeu)J8qHj`}!1h3F4xc-oO13bk zorHhQkLs-k$!32rE$U^OEt0Fm-0Ra3K5~BikL=fJK98vkJn*S=V+eB3)%+a#$?8Ug z)KG913zoIQrsLd#_`iVXKJi{}QuT$9`OyY;LsLq0dYigZ-pn3_@yyFsfx`I6_l*bR zjnG`y1^Z7r2_1lb>jwMzZqDuBwTl_R!fW}q2(O4x4-s{% zSYO3(^creJh?%x11-0eR^J(KW=9~!>o00FS5B_#OmbQ3C4w&Hv0Y1D=q zTmWI&q})M!fQ4P4K%f_{(TWE^>jQ$8b>Rk_3Kob|y&6~%^v0Qjpso?)L0JF%;Nl!& ziR0_bV<_o|qp6xVShZsSgv^9zo?3A?cB8YOc6(XfaNM`PV7Uvta|T{Z!S6bs{zC$u zNM4HGC%1hZ3uMsLk7*a0^Ot`7_>l1B3ts_z>igEO+z8oTsVjm8_BL)@+#A8oP@`{c z7tz>V)N%(|!~*e8@37dO&G^mg#3|=nkm75vVNbmSk=1@6_^(E(5&-M7Zhi3oQ6+uw zX#rEi?w`+(vk{^2FsoXwuc}hHZFn zEo^z_G&o7mTz@$FjcsWbFL21^rEg|_*@^7Xi>a-br8f{@3; zZP_h(NY^nZ9gs__#6$YylgTYNFPxg8=g+y)fI+kB?>sn~UMHE{LjE;IArv^y*@q|n zXU~x?*Gumo)*u>h%;8BMLyc2(ysWYK)Q8IG@}fi+6PuKwY#?y<;4fv-x`1OxM5hN| zv>rT4F}7GtG>P}#a%j#=b()KLr|oDvm-zF&gw$05Cz<7`|r~$$`e3pI}4^`Tln>`6H~TjK7a|Dn}WtcAXx6@ zcB5UG$E)UF@Pe*#`AmZ|9pb9vi|DPG8Dj16lIX3#>Qj+PooSV7Rg}IF&A|8RL0Ly0 zQoD~@ikoH4a1xQcb^@;oG?I~fp^Nd?c;iX23qFeBcdLtx*OMf>D1_`_mZJN-&BFID zl)$)d8W2%Hi_AaP&$0>_XmI7ND|F&mxMdsKK0P;iZ1@7~_TiAQGx&*nPdYKp)>qZ6 zsdg&+S59@|G_Vo-K@srMoYu=|I}(M!gdUkj;oS2C2@8kQkF=<^!&1)D7h-N`Q0 zpDvM^1n2zg$tV3{>Z^0Mq`i>$>gh0JH^1zn-v5!c<%v1oneux4tqfdSkGxw6TB-#M z-la^nEx$L;--r!fcg89BR40ivul*ke5lv2)q#>xBT0SiYv{no5%MLx@-Wf$z+JHn4 z8Qi@_8;Fvh@j@>!_Y(bb;U@$nR};he!T7jZuuX_FV=Wx|1m4a`>4RzZHQ_#S2d>bvE&G$Y3ZC_ES4Uo~KPeKp{vwFAR4+Ps^Eq~a z@H`;sAq&62V#P!WvJf%$G;O+$X}r8ntz=j)1MI)?(a`!pw~m4HZo!heG_6113vtOnhJmVpsItdT(}{ZgE96^e_dGYj;y zuMe%bDZ;z`y8zfK$a4#TA@t4_ZqF;j$(07cxzlQbCkbb`%57WHr<`=ToW)oW^Ymay z+*6(LG2-OXO!SvPu}}ze@Eu<6@b3Hy!7Xp{#fVRNDhT%*iX>1>JNdk|fplA7`+FeD z`ijfj#_z%HdRlR2j~}FC8g{>EJ#ZkMrSZliSLvNP<0Z8p{KV(AJA2v={WFn|FQ0y~ zJSFM^oBX!7qTdx1pie||Ucd~(Sjqb0>7D@`ciNf!hI;Rjb3Ay~9ohE-^$D+K%f4b= zyw~$KMD>lSxCtwLl>rk%Gxd2tbnd{V{4|UK|AQ*}p_nH%6|<;ILn8@D^(Mc$2pa^x z-n!bO2&?!8P!F=u^>n%T9EnhD)#0lS5Rjr-{ed|yJmAJp#h9nO-}<+?M6BhA);8tg zbUUTQ$nY{}X_|MBo2+<+%s<4i*QIhT`H25@f?ned#;rlEVa4gcf5)-e9ZjcB=>8BU>ei1 z@I5;rOt!~EP5zPRKBoY9ALGUKu+Dm4W(_r>Sns_lrb?bzc&?%G+;nI%*N0e#5Rk_! zz6dLgm0W>9TnRdFsRne6YEk`d9^oh@53R!35=q6#At1Urr3VYvcut+Fq~5DW7-ek{ zyxZX}z%2)TI-Q}x6Ce_Wkn)i&1e_h3%C5RS@$Q33)2hx>LX*kfGVT%-ge(McH|(=| zSf^)}{qQTFm2B}kJft)nU*hTG*#!8a*#Z<$^{p4TOcXrPW_WUr4wsyDQ7|81tTB1S zI3oV|D+qg3BFZic>BRwSKHIv(JK=0^`pCK35of=T z1QQ!=B)wgGDx}&eZwKm!QF#i zI|cHVtTS**kg|1bDM38t?tWSic(&{d73ATF3)W(HTb50fsQc92KXpi|W&e;ziO@%i zNG2eTv8{jWLE@S=NGt0IjC z?>Ek+VVMMp;GMEg4%E`z04Q#fjoFCbUloPTQ|BxsLozpE=O(H_r^kAfC*9+dVENon zPf>G18;g-WG0$EWC!zyFh^aIXo zMTC>rfl)*V;cq6rr<3^ssf?R_skwUAZL!U1%IcSzgSsrP-EA+8Ihy3S-Cgh8;Y+fT zUAk9+Pl|UNEjEF{IpXMe1%Jd8LQHT@)_gYxf53AM(c)<|5`X>H-56MMPS$MQ~GyL&z0`|Oc{@3>WUc>ez5-6*F#eH)k6mNZ<1w>RgO>` zpR3COFGo{JOW5?0JhJVnkS56fT2n_b4cwn>TIsqtQ|0>FiG7TZ@zDcir(Hq!l4`k5 z=OA=;pEqGYPUQ^B%x_0K@I+Am3M*p>EB}PJ6b}k+$2_^|dle8BW|64-VlPqXxzFTH#fsXxvVA!>4M84xSy?=s3+Ef=sAys zp1u>6fetPvRVh){aM0{f520v(iEUqP{mA+%C&cs|IOVyYFhR2;k-G{dvNAT*s1&Mx zfWy{)4{y9%2fOGmwp;{WJYTL=-pyD0+&vnuU^OvBW7-5^=9Kzg8YdzbeTqt+0(YqV z4+~(g*kb;zh3%VgOyTF~Vjyo42GXRkl?D#odv5t%skaD8{WtT=xKh`+EwXDqH}+tB z&3u$=w>#lZ)$_(hL%VloMxLm--cHG@y^7$++?@+5=PDGC!koN@n*2vT0{LGan}<0! z_m!dQs+M?4_W)yYcwcN3?ju(kL!AUvGLqaKF;KReLm=w+ytH%!J{6mu8=cT zJ!TRo3HK3OU}auJwi5ckZ4(b7!i{(~Fc<2V9tO9W4Mdin<(FwDYs;dfj6oK{)A@0i1S9>&JO!&` zRG(k44{eFH_@lLWBO+OvVam84eg`=$2F(uYY*O91G5}GquaaJCStfgzK19^>pdSQA=Hw7CW}hG*_24hdx$+rF~+* zGDKr-GB~DJv%&t0vnpEh^A6Pbo=i(0RbT;0(Ic0bbDhr#1CKSDf$_uovBGy0X?ns0 z3hu7nJ(`7iZErjM}!c6j>+jxy>7B;&P$I~9UVgVmOOYG%$A z1Y*P-L#wVgEbTKf%JWiV!HRX4Otjx7Qu#%mo59>WvkvOU?>m2B_uiW`QC_d#M98}A znIOhz)pv9d+NtQ!_K(*uGV+EENnzlJH*mW+3M>3W3)-osgZ`TtbHl+5O5RqGFXTs) z#FmJHF#oRWszH87lC_1CAfKEZ&nJ#HoQ0Z@{(e!V*70l905jVOhS10d$WSh++`|zR z#->m!6XYD9=VhX)k**}2b}|y+t&JpQiT1WyNsM@OP<6np?r_y1bpa`@%r<gmUZ2O_7-MaBB<_Wiv z--D;q&sOekkLM(%zkC9Uj>D9-Ri-I*&F*88nAwjm`zzQ3hm5~lOG>agdtu~!Q}|yS z-{6!K)jurIczv(;$XKd!_H#FS<2cwiC3+gV@%mKxluTdN+f77!NP8LF)6?Zv!Ci7E zU4aKtc)fz#lH5Y zBw8!SaQKKTL<=6Z?IC$Mex;iG7(R*V>NN;R-$6$>O6R!typ^|rZ>qWPsse-3YtH(i zaMN2lioXYs`44J~ZrI&O6dS`O$IS!8rJwjfc@lrfgxjrc&QtBI&ZZ6`eUV4lDX>*? zAJC}Qs1CbV*unUWr4O(_oxq)gSr|_MN~AZOQqoW~OM~AD!Q%G`bmwXqy5mZJ5_X;f z$NrrWaLf%YU0kE}#xJoD3zgKS{^;FU-F9)*QJc6;fvxIHN$pAC6tu;n73Pj-z0uMyy_qaF!-xyTlih=6j( zf~3UIwz=OV1bg8OnbA-q!`*Rkwu4XqnM6I?lRWnu@pZ2_OM@es6gL?#awf9x&kP(E4q z)zw8tanz|XbW+U41d^xriU*c_Um6jUXmw`OsznX&t6xN)x+wT@%JlxQu3S^U=L^=@ zwW=0-=fxcWtje`qf$s2LY(Da|6Yu9P71*fhM;s-Xf41O$6So*!FJd;HEnEB!=Cs9K zm`~C$s65t^7z8nAySsKTGF>YT*kor)jzNhxFqb|y(W+Tt+UR#-tJXy(*79aYxknG8 z$jHT=pOkNyz$X!XDgh>=M{>r7CB_pN;@b4EXxt!(X}|fB9%UkdTk#SU%UF9F$$;vy zT6Y1EjwY6>Op7yj;?Rg=yXT_<3{ii2!oi(#opo_rV2I_F|9C@+6hD`d!ku%1OWQ9= z{(fCw>BkGNtpsIMmQe(BY9QlcfYAFhUVJ~xIVq_vhHBXhc08OG(P0jUPX*j}vauVb zBnb)OTx zN{;%5Qsg}Qh&GvVs1F1`MdSVN+3|Oiie$?~X~Cc> zu`Z)_86aqH@jJKG8nss{4GtqRM+w3pOx5^yhJvN$NZQ7MWOk$nqs)^p`#VQ+SGyOB z?oSdON6>6#r|uj|mq<5KL^AS}a;X-POd%@pL79KyFA@dVWBX{alomoksJ2*bzC*C_ zTH<|+5uFwMbxPdJbxfgdYm~;itd52{v6sqb*Z8wDnIrKD zzac4)MQp`~Y(ASE6>(p!kf$!L4tKcbc>TM}qlXYifG0x>cQ>V7ab8jGcjJ!87W1AL z3k_JsiPd%y;ruYQwl`OttDh{~Nwo8723Qc|&76VK3@FH6T-k9^SN5B2O)V6Fs^t(A zPR6SW;_b+Rr`vtH{;bHnaFEGkJvNK2EvVzS^5;T^-@ovb6UOBQ8$X?fNVHR@-H}Et zPNG1r>{0$S0e2lGzXZzlZM+kFth0xIgGQF-S=%P!3UGUAAqfe%CQw7al;Tptg?P}g z!5hhDi+%>!$~&tT{TD=i)yXAOPcJ6tM!6df7MNy9eI!@4HjO>0UBUS-$~z8-ja6y_ z8rgPS<6M%6kAa%vLE(zgXx-5jk01AGuY`Tl(g+pP+8!3A-Z6{gtrzO zU5&Z-^Br#(d*>aV0>0C7fEqbd$%7PQ5S5g{4!hgU(zwvEpkw4&iIT^svnY=bqPLW! zO$3m-hs0fbtuU8Lv{GRwTzQPH^PLpTqt9NR4J)c>q3UX%@PX}Ih~3;EfGp*B!xKp#jcMi0bd5c8KUMgTmJ@u8 zwnc&F&(jWQjh_ZYr_4;puQLY(sl5W0+T^nU0)~e#Zhm4GBb)c9Yk|7AN`3)d+ETXw z!dySOTl`gI{n~DSqleyUsTX1Fsu;?it;}7h(cmYb{J5~D z!KSBg|IFR6`PFsn>s$c~v0KuYcw_5p%t@g&293XV;0?K7{CMfC6an}#+R`|<`zG5k z1-6=hvM%gg?LTVmvjZ`lC!d&{KS(7KG;|(AglmBs<+PfD^GCs@z6E4V>BR4fQesWs zf2SfSE#C$w#rvl9xhTgGsVr5ia)sryVux2Qa^$ibYDZ5y-olVYZ>iE$JQEUL##1X^ z^_s(}IH>XdO%hU+JTtve$m_JutXXA9c%F$e#L`3c_&=&uWBhY!8H0Xxk8`#|dEc$( zhHN;8-dc!_9I`kK9&tbqcd5G0+t|6;QgCmk1P`$GCIoeu)~lI?E;6z%VwiQ{`@eE^ z`CM}L7px4j^@wsFgVmyR24%p^&n{QX4f#*JYW0p+n>S8N9djJ3Uc8B=%Q%!}A0a0+ z3soBuEEzibh7Zhr++-k?@xM0hpKo@hpceG%(1D7i_g@iOGZWOWW3bKp8L59B4IF;a zpmUDED$xZ!7;hZ~YzJ(Tj>rBmWN`v+b0^tan4~7fps=e3#neZ^0n`3fg*NF?mBNtw z%f-wK8|k)UUCIcV#jOi6(!B{{?G4`PK$-cc_tw8sLTmTCaeA2vqIxKCbQ-s=uIqol#m^4Rg5DmGg3MzNElD$K z=dm%QdgXk4ue=jSg3q}fHw@r;Oa;&zkRkl#wOd1=#`&2HCBB(O3khUSc`h1f(Y(e| zxb{&Qef)U2uA>L{x83);%F3@|k%IaTG7npdq-2va8Z^k%flOL90~qwO%JQb=Z+c;- z@5gIe3aSu+?iHxAQ&O^Kg4(&i<&0rhG@4pOwn!aQ?V_9S_i+Nha_y=m$nGi3bB80h~?qGi0zJ1r6&zP}< zugl>krJ{s`{yWg2u4Y{Yd@G(`i;gnigWi^#qDMk3ARMl;Io3BI|JdVpHSUcMXHQH|Wfq$@r!ZQ~{OBFW#tLjSe#wN#@f8 z#-r^Srn%wF@mjznB)zF)D4a}AZ+zO2+tqGZ+{AGrC*E$RPx+wuLY8DJ+ypEb3I1wf zehlYk(+Ud2i? z#-Kd@&cx$6tqa(a`U za$<}p9z9Kh1LJZ#J!c?mQLY)aD?I0^Sqq`gUP_b_0py+?4#v(4C*H||-+F>_;Pu{W zyfbUatL<|!%Oj$*ew+F8KRK~&Y)R2v-?yUrysKSlQ}&D9`>tL64&e)MtLe6`fzimM z5yKU*|II#Dudd1K;?aAxuk4EBTcv=r(rYkx$FctGRa*CM&7O{6m<#}$VR3BUNZN5+ zJCalBPnrFj_|KAB!LkAHTfm-Rv#lGye6>xFO%dbl?NZ4qA7fM|VEqW?2X-z3qE+NX zYJ|kT<-6ln2a+E8{B_xPS9n2enJbnvI()1}7FqXUto4grJwn32z4Z-Bq{X znZy|N7_v(p1}$=m`;7hFgPpJYzR59Y0o^>MSbf1^NoQ~NJ6y_?R?N&a`tgEX=O>*R zIDauvL@_J0ygU>2t5o#!{3!AMQ;e`9(Yc?HY)#xt-)T-aklQsr2#g$Sc9x;>u9hoo zOhj3mZu6zzaw_s--!~60-bl|^uv9q@RrY4UIJ9~{#XT9fcon+J=;_odCaJ$c+G$Q%Rv|$t;-N5-3FS3gpe+A|7&`U45^t?;_u=TI zgH`GFm2wb;7yp~P8;8Hgod&itc6>F$!E<7Yh!`o_HAUB6K*H{I01dvBCFZ}Q8D^2P zz5W904CK4CA}%UM)m``#gU{MB@z9Qce@+St6o3~ctQZ+nPILij^-XuS41Q_}?xgkA z5AFugQm6#>oD^k0KM{n0RgB{KN)j<^bqR_axCfOYiARwaz2h1l{fv_9pCuDRU-*w5 zU`{W5P6-L}eq#UExqy=1%f|!@Im4{po(nlY-eG5th{p#WnyN0SU~D&+g1E*f2iiO( zK;w3T>MDTVV=CRR*lZFT(PZ*Yn>o0VT(@(=UVrmqh*;6_;)Z{5DbZ6!GsnsonQ@=m zvyZ=C^UgksC;znvPAlnR3Em>&;O?28G+D4zpUXyZ)xH{p0FBvj0)>0r53%k#jGrk< zg-=(T3%p?!A{NW`!aa?h$i8^gNl2X7-Kp)aUR~BlhXfUadJd;;UV9&=4cmUYl5lKY zrrEW*?TI*QEf$r5EmKZ}@0;~S$zQKsLHY#+qQ&{-GliAEw~hFL>6nPOrD z%&ew-R({9-@GtqTjj!;v@$+6GbM}2!y>&Wbt0W;FBEBh7@|!o#Z<>nlxEPg>LQ7GG z75QW*!epXo=ZBdPc^t?@133cDZn!DD3{Uvgqnihq3ivFDDzvHJBAX@dQn0&zJtPll zIFQSCZ^3uq{d$rOW6;V7=9{kg;`9$;R!OsqEytERAosF&=34#3FH5Vf&1|E=S|(E= z$ZEd}S1JaR&N8Pm??sV~r9gU@1a$?Qgb$VpX%Y7-d5D37Ykjp9nSf`F9Q1*+ z9y;Ksm75#-b%Wy*bPrWGz>MSex@Mq0*Il>AzuutjX*L z8;Z?H`dLbW)CHx4bkXNcM$N((dgNfUVdE6!{s;g^Gr3V<1B%~+o_?Til?wO!XS+t> z$OvrcR*xo2&8eKMGG~CWFC$YIFRp6u16jbs8K2nA?E&z9odkG2{rdalnYbN8Gkh#x zn8ou18Mrl1THlJJaN!Ni z9!>Gcb!SUa3h0gxkE}JDK5GOvKjD-DBof{m8OhpEd~)=8T$Z_bgR?6{iYP`pF?@|Z z&r*9l_~`1%H8~kEcqwD0naUfM)DLCJHvFJ5VfU|-+x=9<#!tOly$*9&8FC^e5uw+9 zZ{x++bN@&eopo+}+{{$mUUW%iZfW;o&v2+N6!MMK=Z47+5A9cl*?^tW^^0$w(NXbWs z%Y*PMk0MKCxd%o;!1uDc5QNluH&eMi3F|X^o3{!rA@?q96nS~w@#8+m&zZzHtv?bs z`3rnQ)eKA+BJ%o%gzeNcR$`(%@ci^!)ECH+%HM0FQDY?Zug0{htFMLfm)V8wRUc*| zv53F|#K^}-cpG)1BHg|z<4YJeBil#J3NiT*MSrxk;fNp;_M+h-=2bP4R(GWkFtka$ zGp`&Bv$4Rq&$2F|f1esmU@0XQ{Z0_XiI$PZ8mDXHgOZ^4?f#1woA&%&VR;K7_H9lO zD32Bg!a&l0Nhvg@TiPm+>5Ni-%_a9>`f8?x${tXtX>CMa@(QAm7HKt??)NL(I0;ZC zyThz*4FiunQMxs76=dgHLX^>oISgvauP>;sc3lgxq~V$w>3Fr06myo78L-Igoo@65 ztevF-6l2AAPs8B!2W)BTePt33!x4xp$?m}ec81H#z`H+p89dWg(+21z4dN1d^)lSN zCvrT?q~!sA?%Nb&?&U9+&;#D^`7e5em47dm*UbIz^h(s4!!#h~OP4Sd;YIt^XnbbX zVPAl1KYE_ZpnTrcfG9u()CrjClh@qu$B7SDoCpPt)>AxnNq1d$>YxV?z6KN0HzSGJ za+?CGM7UZ;OcGhb_>n5PBT2R3oo7`mdS%T1t=s+*bm+iD?HCjVFCni3Ln5n!I;4Sn z416s`ERiXJFRx=W^(r3s=l= z+T7eC1Pc3dAG(}c4(4*ASL6auIIZ1x)Nr%AeQ>Ezo_&L0E~+wSO37&_mWEWr1-16? zojRygv${#~DqjtUYBwP%UfAj|OIK=C}PSSH^0;ct( z`0Ija**+oQRSSW`6?L4CF6aB9JGzElZeM>{KAjcL(ObyWK$gxLxJz+CR{3QPWz(z2>XD#fR~aR{+FlVXG55k81etFTYiEN z%r*gtLOmq#Zo#vi*d3vrdqBF2|9?3k9?p%Y6L0fWh&jhZ6fiBeZo(PM+e9^`WdDZ+ zSdGPvlRCo9FUQlJL1C-viAf%jm=yECQwr|N&|Jf(-fvA?R-pAAB+AA&(j%zSVIBM0?;rWr z&WM?kr>u>kzh=|bC-rpANrrto>RdMn7&_@Myu6)Kt*(pOhi^%$?0V=?&UK}jz&ErKPcv~#>zoFCvJm1E>wte6|qnLja6+Xn))F&yX2G6S-FB={Ifz|1ht zCO=?UmQm?W-wDZ^X3Ec)<}s>S`%&BP!GpTblcCm`1qEi(BIz%*B#x{|t=z=vNin;d zgG)1F@6gqv9sG}-F|u5m8sEx`E^d9kv!py+iSG8#s_*Tl-0-N8nMp_W*m{u&HY4sm z+8A@E8OEaPpXK^+I57GdORkjJ^pibNguI-w7Mt|apQxlyv6l&-G`J9LlC&5oY$ctY z^ZK~^LjYWGdluv11fEVCbI!NLy(B_R)=Cg5_8_~j%5M`KauQ8N`hWN+uAg`CI_vvA zk0pORd?h{x2i(y#UbQ00aMpVzuzbkFy)k@|rS^OzqSEnGt~XT$%EGET@H?8mZO3Q9 zGN6eN4z}amTO(DU;SW7CEj-BGy3Zy!*u{hQQ|P+ik=dMrxCApO+kz9|ZTk-Ai7> z5~LU2euhbTv6beu1-(;t(0!P42hKn*F2t=ZOkZ!ZavaoGXi0D4qmzS6t@=;LLki;~ z^uQ_YU$7EJ=(p;1T}x#B3NipNa{3CULd*WYpuxQ{;y)wre$veVh7yW|f&LZhzt%oX zuMQ-t5mP7p`g~dG?8C+oA!l}7rdZd|f}iFt4%BYJrp{RVBa6Qs2%a1{IH)b^YC-?4 zDrlz5LP9SiAvCI`HArgt&UkzB>o!T(KOff8s#Uh*$bpe}yoC4>Ak+15VlAyZ)2WK@ z48B*xotkqPkLXY@8Vbq0ZY);2v%;=Ye9du>K`Y_`XHdSksah&dvCTKqca$;c^n_eH z_l&83n@yh~wuhD+Y$?K@P@+VtDI1`5HYQ#S7_FSKK_ln+Ka?c&&HKs7K4a~gS=)<^ zJFUveYN8VZgo4cz0F|uB0r3FF$)j^crflMnFIFMULjTjl*j<9m1W^kOSr#>K*kcxZa$r(*rSs{&@4{f7$4j%vv&xu} zh)Czqhfv(m=y!xck+EYVD=m_o!4tmyqWkK|4|7r~Z4QgHY7XJl&9i>;y%uTSY8bx+ zV_6q_Q)1)y;zIu^X28N{7pRSD5Jv4yjpquuM7wDzB^*PiCVkhv4hY+TdWmE;VM*ir zm`=H|oqC)>@&Me7bo^G8*BIcAmQ7!q;ejGyqsc)MXjPXf^;dX_t4#g7FqLi4<$}G( z970x4+tpQVL( z6@*EdOv1~KBgwOwQ87HTnTE^$1aH0m<`M#WrBV>`00rWf@T~fg~S8g@uTFg<#vM$2kS47ZVognfE zrnH0J?HX%A_!3#PeRNlNtqI59m6kTIqYVwa6#BMppw+fRK0x!fi|5g@k{irnV56Gh9LFpN1FkLM;`-px(J(r&G^zVk zt3`lg>z&ly6_qpv*;7$wX1RW&a!pZp5nW}|E{PrmW%(t^5`@2{T2Vb_KepCOQf?m)=oWHIMHjCYfoVP!T_JW(~ zjdEqs^=CpQZ)MifS_snv;M?3?3LMkfFX3`B=ttZXLMdGwE~Tq}jrI!e@FW02$>j38 zvEs9Q<(W)~G-@Q9b&r^rmr>CMI3J z&ZXf`8;kXcz~@Z1tF`>CVoRPhREM9X_iy~TrSTU7(P(m!`s4I*EQ*5WMtm%B@Ao%yYTq&Yj$^j2 zY#K}QhX%%`pFt=njGH!!d*8}!+Sp|)(r-HjNGAqRAiB>Q^RJf=SuF=n^HylyFaM>&xG`YC32v?r^+M0qMkEC#1+Za*o1 zZOeZ3PvMgLF7g=W@MS}yA?}-Taq&u*tn9`YWhV&g{5bKL*gF0BjkaDwc5i7x^TtX% z&zkv1^#$RddyHusYTNhK-&h1H&N8U2sPFc;6n779#ieL)_addZyHhNAr zad&rjEADOyZ=RX^&VBCvmJgYnnUn03wSRm4*9s8mfE7SR(umMi%CN^yXPS~h3dg7B z*~Sf&&CX&x4VJ%L(P%)xMZKYWeUrb@m9LeAW5;zUUdkzbHJQ#%ObHh97f0jlWff0R zSky-s{UVp(#)R2X0?UKvV(y95>E{oJZb(2VQJ2N2QJ~-h%j?W;B>xuZdf0)}{E0Tg zzurhzUMaM%(E1-SZ0TU}8@_$MH$Hz^cJs3(zKk<|uj5lv;UU!so}|X0{8?uQxo_K& z$=Jk1W{W@Jfpbo5Nj9O`K>9$Yp&%9-KY{s9M7U0!Df?#TTb^MvBg@-VJN$dwRE^?2r`m@;7g?My<+$(jOWQj;uX{t@O*A24O0H$7(t{@z?&0)wDSgcp6l9 z?_!Ld^)5Z8jRKkhgXo81=c0%CtRO}FRYWn!8dA-3M=t|^$mjOEbQwihD9+sTzOBX6gtPI-yA#KNe=l z&>iuC?>fccuefWSrwNZVeyF`-VGU$G0qfUjCsvwv-aCI|_~bI+qpV*KpPyDalyU00 zUD6~LYGp|~u_vcTMIojzg)?~md$3K%qm`tn*L=UDvfT8k!2x(MA-6*$14mxnJ|Dud zcrH_zg;__n(VH_7A~n%#zDW+mrU>OwGnDjp6Xr9p@0F*Gu=oyRhp!rlFcOR{iv&kZ zfKDxg+Fhts#{v)F0B^b_R~(u&b5LSM%9?w8VHy47Y~FKGy?t}oyEhuL!yY8L&4n%g zfdHohJHjdf&--7DkjZte5IWjF+7;d%42@58RN7>SbLl@TNSOYd4#fR^l_|`zW7cW{ zai&Cabhbo{uCiCWKQKffqjAn!|G@+ORR%g8n9sf`rY$^}{)<=j)2sbL>NL!!a%;6` z-Q%j`e!-``a941aElZbKWXjaSi9fnix;JBjnfJpuV)wBMaiclhqRwwbtLUtTMDiPF+>WW3tn}Ng7WU`iBZg@SY;H7L zJqBEVZxEe9)WY8;bv-XAGx10Kp0bpuOq^$+PNr}r6IOfnrMJeW=cf$kXhtxlTs_yj z20=RiMo)~}5+`>OeGF#r&&A${*OlVgzM!Tf@muVLuTMLq4xt>Ln^uQTTj0P=v?U`S z|21t=1e@Ut%L~sPyH8rWZBjI+N)e@YVXkLSD-zJWD&?Ks^|KeWnrg>zD=FemNcQan+9bjz?|o8{Hw5 zi~J|jY%p>60fp~=4tXR2%TkP8TrU!l8vbt%EM^PiY!dmqCxpAkFTxI->DwozxyUYM zL@Ds>%vEV_njiIu3$*#jwFFT1X?p%eM#2azcxv(O4Rre!51?K?`*^^+&mm1S!&j52 z6L%7STyjpmbUHnjTKQ2aubCBk+d|JXm#DJlg<%wUJ^B|c*p6YKK@s!;s7HmYz^0T!Gj_`Zg8_)f*=ycG-0qA^{|lj43Krxirl4Av4ea z7d-(jYyFM4(Va$xF4i1Z%oF`a)SH94zwSe=6<9lo!ca3%$uGDZpBF}N%kDH>6Sb$( zGUSyq!MNwCY-Z8%mCU#q+V<+1>d9t@0*D{QSD)u=R8ZXjwGjYPYm+dL$zU?4j$ln& zv4CRZzD&`Mfy0GZ+x;WAa4)#?c|ecG%GScQsd9ozfWwDM$M)#lw#TvhddDGvcL2(P z^M*I}niu6@qhX{(mP=cwnb>8ZKg78!@OjmO+XRUmSob0^9kc=`h$ zUp;s@2#%j1Z|bBm`j(s7Yd$jnf=-FaR6AO?0v^Cw@>=Jz=rYmWyQIA$-@jAysk9EE zeOIZ!>wB=G&WeXA%!IzWf^Ttuv-Tt3`!Cz+|IsuM4RrLgxfU;wjSV*-V0|W0rY68? zN$!2t>un3->l=8JZ@$C9op9Mv+Y|hP441I?_-;7;)-mzk;pQv$UaD-ZCxE5kx}=+|;ZC>pt>$Y*l#&xh6V1n1zIOy*CL8L112P@0KQfZZVH zyMajh!lUWTcRqBC_;4!=@h>6r+VCJ=?WmlrSoQS}tmTex)=ZY)~^a)Ei5AuTlvIYHWP$tg5wZ1x1gG6 z`ES0)0uEFU1l2x$G4#^>Al$Up`s$w!?)|WL?~6`pHJ)e9gY^<*_}ZCL_@!>_0o&@j z<~rT5qTOf%J@Fs|t`sz-!JamN9-n8~$dQFleT3$aQf}ePy7j9%W7l3ujvr{GcoLrU zJL2baU3fvGi*B?5rAR{4ZI<+Ufy#PBSYGEddH%Ra0+kKFf~CDdO1Ojom+$Y@gr<4hEaudQBY}6Ad-; zenQ(%19>#_ILQ7@#*EwzJ(-fN!wv7F?2+$^=+BH93S}lH;7mT4N=HKQnVhkX!=3%G z!YRv_((WzKd%MHw;3+^-8PFGbY#vcyUy8?X?R&j&hKj=6m$c2%FK>^oamO0Rds7m; z-J<@ECA38uoju3nYI8w}<06=RG9eY>RZ%0#AP{MCu9>*=K!n3PJ}!Apw)ZnDK;W%K z{;Fgz*#*vX(s;4eGPt=ApC2*BqNW?NIDW)2nVwT=_axX&;>nf5Y-Bw z){ihE5=cy_)DO{btL^fq#k|utdEcNpYimPzt;P5CA=(o+M#X$c-#gk^5LoUcwR}~2 z4K~+5aohHNR|C@>ZQp5d=TC50EDX`*A%A%Pk)eVfaINTYT8CD*bgG~98|fv>9GG`T zBr2t=0h3rQ_AY;#I6`Nd3rW{xY3dx0c8Q=pm{isS8r1+CZit!2&)@%{CYEzfS{&2D3- z61BQM#x3%)p!LBQsr$47$n1U8?>yLKGsCA*4ajs|yhDBhuam?XBCLSXa&T%?Gou+E z*I}b|?VUrpEu;2*N8Pk%+somA}{h49)`0kN3qmI~5cX>UcI1 z^!bx_VJp+ZWN^eGhma8SbP>m%w${|)WVTv0>9BxAjz2E|q4z$%WlP(1Azr&eY$C$= z1*GAX<2T|=WOy0hO=YC8Mo)s8Mw?8fgId(kTP0iW&|&eHUF1rya`p~uf7fD=Redri zCdr2w(A7e-T?lG`TsEc_)3~44-I0F&QHM<8NEy6XsXCg?$p_#+N6hJ2pPKm2HO?kbhv zhG9D<5KEcq<<(W}+k9SIh&t4*5hcKfq#Q_Sbwo;K91qjs@w$6f`yU`p+X zKlXSm^g?Kj3vR2<;lU8be6-JsQ_iA97V95`>{_<9lzjO|+zw}JeuE&u&~l54-xvO24X z@Fgc(Pg|63D3 z5oRaZghMeig75jkS3ay??U#_ZvNVru6%1|S5j_N3MFw6riqeb7 zf^tp|#=<;li_&=M&yPK}`y)m&U(T%|?^;#lVpESB=7aB|0G|Jso4kPsx?uo=9ne)k zomlp8Y5tWlv+0w%(Tl|(VJA0d$dC6aZ}=BrmA%R6Djj@T8n zBHi1p$>WDGMQb|7mx+ER+8DWB5)c(`A8jo;U6R87NRZsb0E!nM05iVN`JI!!n+{7m6ggySPuH1>rJew8py?$m3ImsI4k_-PlZm>wI6zgMl=KuFtJ z`R{r`<^wg=eMP#zbUKXotC@qFybocSgPO+!vV@%F6DXO`MIOg8sJ6dn3}}+Ua8w&W zy>iYg0pi^sx%rN&54A#n+1f~1b4+hyU~@8@4x$exom@-S52uwlyUOpAFxMWHE;W!w z@TWy<4Rpu1A~6F#H8m;~q$-oYfL6k1U=G;pN)Sf8HyhpZYedQKj^_KO{YjejEPd(t zH%@{SyO&QR`FEZ1LRx3)SCx(~cK!ok%ui|c%~(9g8^`kJi&6!x@GJM8hS zmy4e?GWauxCRT-+wGgsGGzaqw_<6hIalr9i3@);q6pVvPT4zHPGW3CV0o^36fR<~w z`}Ed$@bW+;u_?VX=BsERnt~BGE;nLSEz^oTi8`Ok(Vaf|)zDuO{Bl{$bw&Kt*?=$> z%9a`6MznoBE$Go!Z2eAZ(M^VigZVR)h6e}JJK4k~{qEwK!CC&jzVs)e*>+ud?r+=I z#GsGo#x>97$TNIAQLS0JI*?U7mzI!+M`^cf8#>t2Ke1N?i{|#{-{NBEM;)<9ZvapIKK#1)5${V*@Q*nWyN52?w(z}u?Bm-v1yxB<~CCbG*NU(+)Qc4C(> z5}FRoATo(GqL;{h{dsrwAq~LP6IU26_Vo4+_IUTWRZA3OEE$hizSg%UuSc2sEA>NF z3ytIbBJoCfn*VCv`$Z2)Mvq$tZR~}1_V|aP&VN*g_$d}F6)lQkti)G0F1n;duf&}G z*-}3Ijppd(Sm$VDmX%-3qHxk$iBwVMV+Li!Ty)7Yf24O7+>(A-n}ZzSH1b|W2#p-1 zkN@&jbSy?+Yo_m}QZTBkFGVlhQ)G7jV@e6x>mORO^!c~(kJF8lmltrWrd3pks^jyQ zKrB|JAB7gH=uVjqsqR{bmr^aTh1i~X&AXfeI`10R7ehUQCtH^nOAUcz74V0*&wi~r zawm|LVT4+SsTRH?h6JDn8=%MU$J{_oYszuF4$fvTrwo78f%s&h<%jm4wYMTC`~t-u zBZh^AL9Mj5`4x(9g(m^cs2$^_@RrhoViVw}~`lr$HZYC(v9X9|He(dj(`%qI? zizuVQU%P!o5_dK0C5VNZR*q#!{i%Dfl2c>d6zD`?HnN1lRF0HxcseKr;c|Jt-R7L- zTF*|PM83U}T+xQ@WclNJy>oNlJFhu>z$>>`rp?|7G!DOc@vP{aj*_bc(&@LrZN4r5 z(d%!n_Cs*-)#2I!PoM?u=1Lf#BDSp^Cg3)4=uPvUsSA;{sx=gqa$JVA$BQ0KL7N>A zbFkMTBKLW9IyA-J5aj=?7k3r8^BI1c0PZzQ{`Q|~@%PBc34?JFKTP^1C)2s_e8my@ zy-f62s$0Y^q6VD~#hEyC#N@A>aNupd6fxu?0a3^dh*bbdEFq5Y%3B7^#`JOo#QTX$Tyx$N>V7{eq^ ziY1~Vgb_KI1I{841ZHa{pfH-_$nTR=L;pR%%hx3FxZ;V^_)i9eKI6(v}BvBMBdDETX6huWTVaUgB7qFVK zd=~hUP-fFO)Id1gjW-4H!EOK_ynSk3Ir@c+0GbC|gv-8=r8|_sA2^O1=6;m$=-U7D z_0RcRWjz+k;0*-Ng;g--sT{)Gr%_xtfz7VHfC)F1JPwyFs33bB2$%325dUF%Ale=+avHJK zzex}tsmPknaZQ>!+K7faMcHBq$OCOVIp>o3J!iMCEeyLLgTma-t)@LiM*=QKe|iqf z3d+Ce95}ov)AB6FnN1d&JqL)2`C|Fe|75Ptno6|d(tbXR_;+bL*Xxg^`=}Myod33E zDw}K}n>_sZ>w99P-?NU(No#u-VW57fX#uLdyOmT)MO$(t0t=Z`Q444|KRB}e_$CiK z(fXx$OAd1QxyTG-d;X;(TBNZxjqS_SV=FTT2K=+Rep9a4H}FqhLI0xTNA`f>S)E=W z4O%Iwj1GTN#si+RzZso2P#8`oHRXC{>Nb2MztAf5us!fkuJ5Ddb>RD>Z`9Lr{}99f zf3~wu1`iK1fsWzUi-(=l_esRSmq90UFJT3`yl|dG-vKaZy!7y;JZPniM5>>VX=%(k z5Bmkj1$_x_CpUubd-dgc8N2@lA{E^+Uj4jOXs_Kt(VlfJwS8FLLPK07y(1TZ;7kQ+KJ}_P=taB*pD?qp=@s<>PQf zBE`dDKF3Xg(klM3!)>*n-L95xVG~-^8OPGa>)m7OVXPx+eQveak5Bt@H=%se7e;j< z1rD=hliW5Pq-4~t5r5IfXKuQus`y3oB7PQ~_3DlJs?BuF3D%zC0(P1&=r+Si{yJR4 z*^MY;_}W$@$lR5YNjZz%hb30}7QhUk7xu`M@cH%Jed~3<3~)fpNLPr&TKQnHT?6vE z$yVwWGTCu~q|m_Tj0jQfg^x$lGIq+y@gv zn?t_=xPK6_v!vSZK_zyF@rP!Mie-&8QG;eZd!oyipznW{`lKj1QqR!vNCcm4npfZy zFFc?=^u{xs?KFs)hWw1f0T;c_2&YraoG=RD{W>>sXWqdxRsQRTy&-n;!2IQoM`xKB zJ>cxH-xY2AxQke1??G+eEAh)XIT2VyKcPVS zV=dmo+?=al4@=Z<1$hhg<^6ygtq{fOj;t{y3s7DaQF0b zn^*(+TpMx=ko@q4J`I~!gbeGc?&_seO2?;-f_~!s>ixkHcrn&fBTfjoFN>{qNZ~*$ zYG0AZ#5W1U31h*R_#GL8-r|D5xV;5M3*KA0Y4e_DO8!^}4gzsQaL*`^7c8a@gZP`5 zH68y-0uE)PVCdwu0Z^M%`@@yI3!LcYD_c50Jj-g3ak|4`=<(jQ%^(PfJu)N~`Hl1Z zNl=UL&g>rAYGK@tirT{($7xQinr^qK`Q-k{Ea;`{&(EN0m|)t&$T`kWm>4;v z`LPAZ;ZL?y5y?FU*EU|iEy4~>rA=6^&O{<-e~6~>!gcA^qDs9zgO&mF{4^R#e&AyN?B1Eg|GC$fYizErBCNfBGJvd)wTBbZTrktS3w{*mi>c-> zEG$LKJ`IY#RF#Sm3*r4}%!&N-t}}~e`{J`PH6dk7+&y%J+r0%%txE`*HZGd@*dMoV z-c_D2A6wA*K*GV#MYy+j9{)fmT|5HCDmB+F+HT%{P~P)9 z3U03+5ZWG@UML0{VU;Xx@bP$E7T|FYj#pcc8-k2)w^qg8Z@^kf*n`d}WQ}K1hMK}B z*=yr#Dv*VBWCd{kUPr`Y=Va=k$!L_TPxDMk zKYhvg?xy)kn)Wo`PC089-wKJlWM`n{Y5&=BZ{z~ZjzbuRV)u+mef=~ z9)zsek!F>O4y3)vfM#eklF9XGRGNkE#A?C8P=g^1l&P3MhjxV=ek$S;M3=af9FZ+< zyyEx8dTBhf_03NWjI1-a92y-B`VPwiuZ>iJu0!(Vl?IDOo?kq-j-G~?z~p`Qux3PD@*A!!7IU#8D2?l)yk-{wf-Ifr2Vf{HwRESBC_gK++Pgm1w4kT`BCXpjJdmW$&@7zK*+P~TDUf1WclN3PGgGF&dMVk zKHQ(HG+XV`gPyNI?H+du6{5|=SjxWG7N|&jQaB1mK|#=(g)!z-&K^Ar-Tn;(8U-u~ zj>Mp~=$g5B4c3_K4_iYI>bBdp)K7=t+^6Ua@f7CMSs8SN?>L^R&D<}6CQ}`-4!+ZU z`iXD&mextdgH?xb(tXVh$M3zp`R|NA02Q;tU+AS9E;!_otaY(rBA!MJVPp(%u+9|z zgwB3~NE{db6WJX{!pb?OljDyl39<92o)i}-=o_r|Bs@P!B zoP~;Xq^BU2Z!t#t8&hwsYM7a!O^gTi38tK8ocRpB2T9+xXMd_8Zo^Ez_iJNS@{d6y zGPvuIVwjTKbPoV3dFruj#PmQ0SYvU$Q|jYfSLXo(?;!UNvt;J2j+A0ddTghAH6-g) zi0Ka<{*8DWO_Gs*8G&jSk_*tilrj~qJBQ0g`&ZY~ito*q7at6S553phujto-)`(U4 zU|grHN75?FcZ8F61W60aWUhZ01D1PM<}Z^R*Zl|Znb(^>`0VX)I|j%R`-)HJ&CwvM zSN8EuZ~WA)wV1mS=m0t!zMTZqYPiH^qU!)H50}S;5pbXaK=xqXKfB~Qlga4vZyb^M z9#WRs`4@T0uN5oM@KKPx^Fhr<`7g~dZ`(84RfjZkfXEH=_^Ka@+J!&}Guh+LSIxr$ z(2%^r%@N*HU+=%nmPa>W+c7rbn;Qpv^GhNwFHU?T$`yEPWe57vR?1s<^m*VrrvEhZ ztPyraRlQOUs!`@giFMvFlwN|#)tt|IpxLVhe$Rgk=gbk-aM@Bmv~j5qCY^nL`{{yn zk@^0yv|1#{TR%AZ^1Pp73XIV-<&WpZ=7Zcc5kMf;<;yzBsLAIBi}dL`wnG=W*Q75A z^bI6UqV;fItbSjC}Kz)*_voNrvLJzz}Sn zZ%MFOCxh>!9;W-q`6iQ-@R_&Qgl|k+(GpHLu1)CtMuX2QmfEHqSX5cWPNWlK0N)Mj___~@X4C!lNPHWY+0M5&i+O}MyI$Tny?fDvOQEoq(+BfK z;cOfh|X_4~TzUPTQL} z%_i-eZ#ES)u!MSAr)Sv1Rs)0E@_h-K#{E&7IQ-z7*aOIx*!?gxJp+PK+Y-#yp-Hhr zAxo~WyZkPpry`NWF&gA+01feCdR_y6}BZ+IkWmGnNnf9CDn0>%uKV=T;Kq zKjgM)@Rt-TkuUV`L(t`!?d;#|}ejt5j1l>a74GIsXbMj7z2yLd5@uNzl)P+8MZavC`y zJ11m5+L9q7|Lx`^20V5;g8lt8YqX7MZ+Y2}jT_18P4n6FOssfs^7?g~GT&^F$Bips zHZ?^un~(WiH6-#eo}9(xnBTi|`8nlxOOWJRjpEwYE2uIpuYgTXJh7+1)ru$Tpyq|r zp5-`=qxCF<`>{;u9U1&wHlBQbZZ*u#ZqhURokq+8=)AT;TV8(I5=aLKa3I%if6+#q zXRLX7IKiU77jncb^)3ZZc>jC4s&|}%W%a(cd^_gA+6$H&H2G)mtW@b>H1_Q8zu=29 zq&rxONX^K2JnjtH6CubdItQQVUv6(A%bDIiF@xTV{(gVyYae>0^bhd`5QitAabyV6 znP=XLIL{bcPF#^sQj4K$S#ox7X$R^&pWioWD>0LmzFMz(SZ!Y2RNkN)_3@tn*#yHR z0{Ji#)pBAGl^`(`ub83skzg{BEQxx=Eyz;C0(y|Zm)HDDc4ZkrwAU`M4DG9!5PbdO zy~$uBEyxEPVDQOOR8JNd77NIY(NAv=vGxM=jQ@qQy)ng7z`hZYOPhLzgN)S2$U$jLu5{0DH}Z0!&0VKr zu2(y<@<+W?M3^JE+;;=3@XF}E9^6b?E(T35MhxJ@j(|62qR6O(X)n6^wEexFox z^-qZFZZE?P5el#u_SpAqYoSRBmb#tww~RQUw|US({m7Bljqvr2usr+$(YpGuC22_PG%uaaUfIV_O>SZ z_>q{R&$-q$9Wotp(%ohx05%hc-~%?=2FA4IJK8v5^)o)DXYEcxGkcLS;i0k0$@at8 zrS;)Uw^Do!LKff@>YM=<=gKO9=w+t><#@KuvO_nMi(R4~-aZQhMgJwb zId!5M*EOeiKPq2-y2ZkcEMnco$ z%a>wrEq;OTMW4O}LLg_oM1IKfB<+D=pLkN;HS8Rl(b%5_9{^e~orfsh^NZw+i_+$b zUE+7yT-vDQ$lR4S@}MYhh4LA4i`)r6%=ZmHz+mMIkI|4t`!|yho9oZyvwb&FrENG- z7&1vFOmlwF5Ipzf3iBVsJqpkUZGQS&jZ=ZNBWd7n0sb3cvuzDtHZL?6i->dQx2b%1 z(XYo*8~N|aiqfvnN$(i?kol+$bG_P^YnIP(7=zeNapIoDxaF)u*$|>q15Z)UICit8 zBbw3;5Kb@Gq7X%lICtcWJJO+5Z|2KM5$JYWtNnREp#6G4prUt9r_=f4<{&Gp7$*h? zC5Q}*)dw@QPES8mwHiXKuj~u3#*h`tEK*i5l8*aX!-HSoLR2`3h%yvOoA*`=b5tO3 z%3@y=0pu|yUj_RH1g;wRV!*nTW?TLCt}+)I8Obq)LDZEbGJZg-SZgoR$ha!u-~3LR z;a>M`f?r)*<~@l^6EUF@A}D;PX5JiR(@y5}1lOFbbPa!qyI#MGr@VpytR={7!N~ke zlG%wz+_k5qyo#Z`j2X9q8MlTM=kNdN$2Z$~VE~e^qdeq=Q2E83zx47AuHK8h$D=4e zP5c+}TyU#yjwX}oezx=8vEwV3|ClMT{peHqF+C3_(Gtsc)HD6Bq+W=tK1=U4<4NC3&XU=TdOcJqmzWJeHQ9uN8dPlsiFs9jT z0svV13jMeDMp1KEjV)`U?4N^5oDvLUv$y?>Y2r)Cjv-|sN3 zNjcD<=4rlI1yEjTCs8|Jk1lJ@a9)WFKYiR&OC<^_fioy0aROpR!eGODr&J1vZ3JKb zi=_75*vNcXF&bY-Ro-|I%v(n^)fDSIhAF*-b%KgLq!NxnlgFW7V@46{3y&s?2OM1H zdl{Mah?y+_kQMFKytc`Ss>#g@hlMDE6EFsKt*gI2J7Kp#m!nAZ6TZqZCPE#B0OMK$ z7t2;p4x0#63rYZ6{vvBO0{T;hMKkc%Ev7r4MvT_KJ=%n2g#Z<%3!+R4F~l4I5kp@C z<62I=Od!aooKW!+p-i@GL^31K6S2SQ`kWb^+2Kuj;B?fJZ_S#rIvvb(uPRJh-*+8) z#tFRY&CcxTK<0s#Ox~TpUhOw-+<03ZijJ=~qoIjz2A@K>5ml{z&6}tSBhnR4+~iO@ z;T$sPyQ(@d{cvt3Q*&C6Jr+(ddt2?$ps_e%AKk<`o}#pVBZ!YSwTbgApnf)Q07Hy` zf1UD!R;QebCOi?;oStztJz-KGVet4V`r|6|&DTo-aa-OfxBTZlPr6FHmM$UVC)!~& zmA@W^T#c?JivzsK7y=szcp52tG@;j8&Ewz<7=n#k@j@|wpzPB9p5Yw^wWecH|5%N;h30h6Q5%)N^5UdhrN834Em0!WQ1TpOLk?yQCg|< zu&X$p*CZdL8=_Z#$QF@m)tG68FnmPsHalUqNrRYY+eZQH9p}adPx;O~&>Xe6|Fb7O zlh}^cWuV&&gEi5o54t@3`i8moNM5mQ+Mf47=g6@d!m?=)4|58{W4+M}6M1>NS^DQa zKHN(>ZhVs^hkwTx95#q#(B+uFY{W|43%zp&VmD7(|*$`5`2T=*jR z5eP8sWp;5r$&9)4n$I^g?(1<&dUw<%p~g^P2`s`WBXPuFoD=#4gW*M>i-4bSik)u{c1bf77N1KHBGc)bgTZt~8RyHj zO~?h(dWZSrx0;#n^eo01&x|^X(_Nxx3f(Kt)s(>$uZg2z!L7lEgXD)l&)Iwf;TaGE zjcT6lZ!VnF-&`i*1N$30A{RXQ8ati4Zbf-SzaL0drj#(}iQp~#NvbR+8j_Dx zB&;;#=iN!;;mv_FWSO-s$a9SkCaaC`nYEX0aa+x3YL3D|E7_RCuX{%$EZC~(_vc+2 zU1wrQmDs?9vN1Ra&mMNLc{-w51mkMrX_E~Sdw_3&a`U ztBt={u%tU)^vq5@nC-mkUzc6GeZFj@&16M$3xV!&oJWGveY)Hu=sz^SJT{rWUP$c@ z7t|lccY&pH5-J=gc+D-y=3HeNi?i)z`p3+U=*0$ZmbMUd5eF6I7_oJ$p_C970J5kM zpnF~mv1Fm!07xi@X_Pk<4=HzVWT>O(4Z{O@ z_TZi^d!ZmWqXKVR-Z}%;0LnQJaWMZ<3fa}ue|ZZn&_n|lomc2pi$svS?j%e@%n*o` zcBx2sc5GeIYu~3t-s^%oKI9xC&SSLb8ob2*xtc`I*BVU3ob!+D>bC!*ZN8g2$KsfE zzivFU{~c$0y;invTR!Unpj_bDaWmie))0`OS@d zu||}TperxtB>lB9n^=F2O6oOtUhnUsSWep&og7Lu)MZ>{_o7K#DW(PbZ8<#4WDnZ# z;QS?_+0MT7Np+#6Z&?TohDON1*)S2>dY$V-rD;-U@uH;!2~}M?*QW5!#P76$p-W}! z!7FF z9!%S7_Ym7#-C#EbuV^+m-#=`wf`zmz0!h5YxJb9uKqv=L{EYCrc|DXFhi2(NBB;DH zaX?B1dzM)!tImSWByf1W?xjp6L~xQ1j5m=sooOa=rY%tG$y*$` zvqSH^9qyQKKGO=X(`rST_ilrgUbU6Da_VWm8PTCr<0N+AED-jb58Xr92g-@`#NunV zH0?nnWcI?R9!y6V+phXY7i_Lpv7--g;w^AuENA9tqfPlYf@My<%r_a^YRt~r-*zUE z=?C0sRwGP)MIkj&99p)y_Vw##f_E5Ljhcv*`0h<05gVD{W4`gHSG{%Mi2{OI#d3iY z;D5aU#}nd3aR}|8W6!W6g!5JxqgFJ2u}_q9V(UTS(94Vr`s=hiVWm3?M=@Xe#p%mU zhzoGzyL$aPa%Nmfu&KbvINfV6y5r z5qYWP#eNPDkW^*K0pVcYcTM@x{Uo^3D*^m_ZYgL!t5PQh?+Sw+_9!x&Jh2G<8i|zn z0sXh94||);w{#%J?FO363)$l(ely5wu1-S$rIAh`%vAMCBGKa6(5we?jl~=_E3~zB zY>A?KcS0nys}fmh>8vL#i-Y#4#zH(+VE>exO@MzXM!!@vbcNEfw->{=)@4ot*mTWF z80ZO8zo{2H77$ogY>m+>$Xlux=9+GDV%3B(Rb>=QkbmAAds&^q``bgo5+8|{1hww< zp4R+&N#(W1?_6|U%oVfHiDma1cUP6{M-DgKv1(4nc`t<;TX#-klNsbiM5o4X*IW+Q zF0FwO_zh)nPX0h|rx=au>F?vVI-;xpc7jBtJ)c8gGn9#_)HZyOPzV`ZN7MUsrJ})3 z6z*|W8ryu?GXuKK#j5o#D-A3KOOhot{)2Ab`a5HipT}z5Zk~o1vp>d}hd3cLuV{x1 z0%wtSjrw0JAVby~L>YW?v|}>GikX7WwunG~D~JYV5Tna@N7`#c0LrxeBKF@5Z;>It zmz;s`r`Bvb48FNAQv{8*Es+-!?UAXc2fl-N_bd6K0+RAi5PekPO?<&z*jSJtF3y|MttYq^m)HOho=WS+}gEH!N~ zH|0C6Ck&M-XLKt1*e90X+rIMz6^T8F!IN^!x3JlGuHj%b{Tw;%ZOLMPbA55ke1`b2 zZq^XR@?8#9{9UWHvB%1LTO(b+IZ{U0>%;aG_J8Hr;CNMY|6XR>`3xDD-y1jU9!Koz zaz)T_&bgWL=6s-?qV3CR4UO9~v12;P6cXJEL2A;SO&~IpKzDu;t36T0dIcox)pKQ- z9OSVB8IdbI;{PR#3LANcbrrD=yZ#lk9p&OhU61VTpEn zjRu{z?g1`kx0+x?OPGeM)XPE6yyA<_K>0)d&Y)OgU$s-a8u=gCQUqn9nKbt2PZ7H2 zY2zqx%@u|f$xm@G9A41S5xlaUbm7ZX_fZQzP7we_09T#C?n#(Mj2<@%u^{FB%~A6< zBk^?^G8ZJ;+VwDBZYO1|Dx^>En}V$GVX;+4UE0F2>_f@kl0fNPVs~w-*?9-=Me93P zT<#6JQcqm{3{EY#uog;v z*yeUN8IP0l#eRkY^q5~t$t=WkyV?ZBR!sai$pNK0dmAJP^LIKU%0(gTGS zcHm51MtI3=*Mt#?0D}Dq$`gbBHp7<$U3~ZU#yn$VT?Am~=&?XK96}5Y zYGb{Wo!i_so~I6Fljbr!LgEN{tbk>IPGNCXf;WIv!<)C1yO|a8Zqf&~J@aYD2=u)( zdn+BoEB)3@#_*R<5P}dTxal!sn&QKk)Pi6k;cHlWiy;nuf}Tt1V{Oub)%J(WI9 zZ2r05#Z($gfp@QL$v6}zQ6vnfNOm{nk5l)yyn+tej54W;)2bJ6hT6|g7JkZldvIF^ zr$CuM#3;Ll5>*;o8ACv@UpFy%{M16*t{G87u-tYvbAmc^L_z2h0(@L+leIrZ%q7S5 zUQ8I@{@T?6`15{Uj`ZsQwXfD!lqVbb%R(rIjYo&E(pShRmGbv(Tw1`eLE$7=lkPZ~02iShELzYTGEH^aB zxah7iUXMDx@FxGd@##bE*Ud1>T0XZ%!^}a}DQsm8WbZ=#S>+}&dX&%76iNccZC9x* zPJ`l#7ozv;>%ITJF^WND(C z6c+#}p%W)F+aWYvYaKx^wVPF^tYZbvk&s6VLI%I z5f+MrTNCaygbbPv#FL4C3)v^n5FMk6R}tb4MAd?rlbCmNlu-{EbN~fpBl9lJGujro zj!_Xe^0;Ainfa_mqUy4SzsYjDS_$}poCIIvh{Jnjy1s*dkL=}@FoT#mk}b$cn;AiD zRGAaX)D?nT_#>#a>}gJO;E!Kqr3z8tz~o{`v@XVGgYhn{B=;?FNKFm9zzcB2#%V*Z z3iK;ksEDY5Dzf7ys%+<#(e@>Q;_ zKohwrVVY)_{izdX#Y)eebt!{y8gI zSyO&<&YYP&d+(o%`xNSZWoy9ifo~T4ifzTOCA&o;TG!CBD7!fvkgdt{81fB!apjcG z>Vd#f@(g**Ii9>$c}(OV3A`Rzh2&?PR;)4RN4I3GIl?*@TBFmkA%uXOAu0B_pWTx)p+veg|iGkJDzvh6iO<&7W(jX_v<@^T9%D^x55#H{dbe2 z681CrK~3k8k?f024ei_)WZR`6xDF>KI{5YZqjeHTdIQ~0N8V{3&tG&@MaZh&?&IP* z9k=*#nY9bi1+do+L9zg`r(7-$S*$g2${BoHSwp4lqc-0Q;xb!6(7 z-&W{ErPN@n!rM5(R;fC+b+m)TGoTLzO083i5ncD>TRlL-V;!QD6p_Y3N( zmerN)?s&o5jhR@r^yKJD;G|t36l(o4>J<~;rN>I6`I^kghD%14;#43!4 zI?}eVHJUicrBMudU8VeN{8MGI^8YU65&dI=25U< z$W!)?pG@vGD;h53E{b@);r%_;q68%1`kv;b;zdyJo`*Zl? z3e@%!sQ>2&=N~~zeH#za$1T!-{CCtCrh3y#X!Az#VS0j^9TKj}vayZJCdVy&89nh1 z^qaLBrU3*CwRXpnL{KI^^YYDJhR<*k=29x1QsEY1hIxe(rk|K!G(SE4sq;fFO~vi? z&5or&YP3D{cv9TT+bj=K>-L(Qsz>{UaZ^92WLV$G)BB{&YLjVh9&fkie7ZtxEBr$1 z-qIzm+zRx+V?KKk{q$JdoCTr;4g`+tkgJ6tIS6Q0<5Y8!OBymkQ~%t>VM2=C2PuIW z32znBIt(@RZS|kO6oHL(QYD;-X1DC6u*LwP!XtV8AMosdfk|-9Tj+}Cj!7;$JIQ+r zN6>yn`SA+P@#-)g954(U56kM6NKofbWCvJVvThoZF|N`P?DTb~in+YqXCx2Gc1<3? zAZBF|BBz#CA4)Op@_Bf_HW%9BDtW8VD=hp01ewz-#*lc9ncQ)^V(0sZfsJch-n;v!fG#%Lz-WQmkC3R93L43GiZr=LZCm*MNwy&!pE)A4dHmgLu{wpeZp>w+&jsRPr{HEYmYYI-!*SD0oiIg{jc-nH= zNJWEhI*|*G=I##>SCpqTWfS%#O|o!a)DKP?evUm~j{HB0vTL7UCSorwl^6nk$PI$N z7v<9y2#a&IQE5`B6-`If&POPI3)hReDUGAr zZmYN}zFd&NeUQhKUz&>_JtFQoW4X6R=@MT!($?zo&f*)K$>KTeu zxJcg1pxnQ4bEUGxsgkNoSEtbAbl;1l~05$wA~UEK7brTrz#Q zKWD1_eL+~yMZ!qG+BnKGh*RfGpL`Sy8RlmAlUsbj?bHf$Wt&S_p|+d z{?dlO%<=`{59e12xdt8XvyK6zsN6_)f!EGL4*x|24lmR<-7(NJfUge3x}e@$O- zd41Wa#|!#FM&ih5ZP2Z>^dqGNrV<`dXO(raxZUwM&K7BN@$tBdlwlqZewz@e8UePP z@S-?KpV&CJr3za;UqL_pJWThD)L&cXO~FX3zIk@YR!ez^S&ATc%Do6izQASLaCz9@ zzJSX|YL|ODM)l&egfn*@aGiUB7X~!$Hb$gG_}=A(iMbM-k*In>8|@_$`4it0)6T$H zw=+e#6%L-Am4Qo~+I8B2iJ}uEMD_NY#afP^>^7+!`qeaL{nWJ!1UEK7mLgnUZ7vkw{BiH{SZUb|b?-2%)j)lE@0l|@r z0xXk7YyC@EDJO~Ug$+@jiatx6TK37v-!SX@#}$4tKOb@o zyxjZue;Jg{5nxyHvCwKE>C8d)(XaC9Y8GY+xI#i3qIp(TQL}$`cd_SAcC0ZE{?No1 zpC$U#4^nB^-#%Q_d&b_!pXB@sTwf*ecgNCHJOCihJ#=H;-yYxr<4o#?zJBXXS(d7v zi2P?@H*f~MyPDT{FRv4_ie^3w^I>?CuN7OQ7VAyP)O@d>@SDI)2=ILtKwJx5swV#R zvk@dchyV@CzHnU@h|dfmfc^?FSypKJ$X5~)e%T`NPKs5Yw%c#zQ&fe1CFuB-8e^6U zY0Fusit;YzCQQep%!z_xa2|baw*2KhMCu4xSS7)rEi-yMgVDv+l2>@@bNS zDBYPBpuIPq-qlVrVrBUO)&3BCcERe=d+r9Fx)9b06qb5KT}t63{g9LL-6D`tQ=zw5 z@(5l@Ouvn?>PmGq_y2;Ns`d=iu zX&i^?qO&qS#)@oBab(2bvKWEdNNA7SAG-?Hj7mjHX{V0`X1OxqY1oN0@YFInj5Tm= zff{bdvB`Yn=RFcC8(%ggaKB{cJd)xvkH1X0@v@}(n#Kurht24ynkRiK*#O4)tOZ$u ze6^+goMVTn5DB1AJo@3k93!7c>Fin`QQ73=`bOtHkIr=^933y)*4h;tZd(jp8Vq7( zenvUe#P7Qs`d+C zrg?JQJ};6=&`{27wt(R6a*1mFOtTJM&nQ8a0js*5^7`M-m9Vk`|L2J_4Y))vO3oN^ zod&4z;G@o))S22vBIzld=Z~Yp7l?Raht@}*37YH*&h_gj^gGLqX{vkMwy2G!a1a6RPdrS+%tti^(^xKNFZ)z^undVkaQ)SYFo&{D8_d2qclvgX|5lBt@xC+vZgF z0vbF~61&k)tiJzb5b%V@$=1(&?W5V_4r>d>9u%KCsyo;nMzOk3Ayt>i(q>b-gep4v zq!`o2tkrpVQM(;K9R*e#O$V~K$2<^sZ^Eudm0`&sny0Y0X(~qI5`Ieg6H+@~>3}k? z3DF0mj?|jo6-U~&g1E~gZSw^ASts8t-s?yT3&-M06NJzJfaHI?7QP5Ndd;Hm$VA$S z;E54_RlK0(+DLtJF#TdNIQD2 zYN@*ym=Pcxh=Ra-7V_ZX=I=`{vDcz&A;71$ZviHX)?JAw)#J2qdz`)Xe^`Oz$9o}L z1fG5?2U${s*uNcW&WLOYgI3VF&2C3r*XN&Qvb4YHv$S_=ZLcBs2H2&=4Lp7ReRCEI zjo8{MHlisH3Z2lvrkZ&}qOlb=zq?FdW-;uHRc--h30tM}{GvwaZ12uf-R$xg5fPd3 z-1Dx38}{w&CLxqQM&Xp=P@%q&;_qnStlv7eEPgV!z_s_!$LU*-sgsfwk1Fj#81fIn zN_;_2_W@1&X37(+~faYtKORY@7>Fd0REobCoUX3IQG0q^%w6|Ctqjc@zH|WMoS|7@ivLRiXP2> z;LEz#q|fkfr*O5@M6Cw6cfC^BYp-`6G1z|s)@#rb+So{)(De_se0^oEzfMOG3&YT%pC2e@w~GV>BX0~IBf-loXco3cm3)8Y zs#|UMS|0qt+4OoQ5+3L_%h~7!4y}=22448jH04~}Y{*WVs!gZl%Oty|dqZ2d^x|6% zl==s|fdc>^{;-3FWh$SgdpWaeip2a|Vu}w>OiuTif3_Y!{#olQg%tJBfUtfP@8izl z9+C#zPE>$3nKucpX}Jl04XyKlJce^{M)~TU9|#5vD)&BBM!t@}4;CY}d-OqsHOcPR zWKmrQ;BIe$IeduYu^-`sq@WwnYpK}s7qr@P%g?;%wSGJ4mNFn>_lbw5z}7@riqEEu z@t%^@cMay!sTsMMN?nT-Z)ZZx=4B(|NF4-G34`W-o(sv=+uAb1N)^opYh04iT^SVhdU|kn(Zzi3)Z|& zEoKyKkeuZw7Aew=1bifGErH3AH4 z^G;P;^%prd2JSKfTYCgZk^~QuAhKC1CUer4TO0(y1h28zy0pbbmWaSH&1z-5@Sl

`WEK2hvnH#HY|X~Z)9Pn1O*P+4_3_GqQmC|Jm%duNty``QOxd)k3= zjDOwisYCU9ht*V{J%`U%Md3#rt2K^phq0<0L{5v}8qD@58O31&Vh=BTq4dn-hE|cA z$zZ7xl61E%&#`?xL5Lo!+dg|NY_wtGO=Id;A1XCKj(M-TLR`Igi3}aC;<8QQuO0EJ z=?0+RJc_Bi&%fhEkdr<%au>liUfT2qpWL=9X`{e?=-bP7zKlWAvQ78+t5a_2pAf!t;TQpUOQPQuicgL@v`sgIJ+k>@7v6vMYxs(-v7{C=xBmc zC*iC`PQxR^ix$7yf|=b-<#wSQW}#KGr<}pijBRVINye*!F>9PA0e;c9jY<`0%1ca5 zsNn>oMdi(h3wjie9T(zcAC?EaB6Qyq8 zdzw_Vh=QQ{r6p+?{7CD)`M%|zl@%`S9VEp3ZsFkEZ9KxQixrNkfZ`XFfXn8yMGW#_ zs2aipUCi|nlL34pp4#PUqerC|Y`z?7kXI3SK7&|aX$TY#LCuCaffJmEa?|Zx zjCaPGhkQWqcL&+JhOuKU)WR1HZ~mIp$yA(jyCfd~;lY+Du}Uvb(Sul@lve1TkOg1) z;83Ccy#^TattkPMYrNdeK8*yPNQdyAM(KijFcatt1p|xupf+L%PLwIzN~Pg=S;JL1 zQIs&)fYQVMuf`6ymdjG@qG08T~W5_jVWD8pU*rpI&G^$yQ z+mBnrL|wzwm;1!T8v+O`aDQgW9K!T`*rD6cE<>JSt9FzO0AEc>{?^7i)>aQ+3mkyP zvON4BanXMl&fU%2_&3m^#S}sJ;9PneN?*G|LHAF`cO!zIvHWhT%KCTJ~{$Z zZmy*@)ye<`*eOmbDQ0|@t_|!;V&bKed~g;D;KHKL^%}2i~X4xQ` z&K}Y{Ts0~G!v&}pnhga=72g`MSivFS^UIoxFHq)H_5Ip!V2j+IEu4z)WFFQx@lyF# zrjqi)rnpdnQSh0zR~YM<+bnq2^nIup0`0=VF2u(m@#}>FvMz%Kp5}zbn{+O@gRzzR z+J3T$yobKkWHxx(xn?)0gyIE`JBa3G{mDWDw z-+)Od@5``6??3C|vmZY`9#{b2F?QPFaS0uP|j}SIxg2 zkA9b~RH?+7W}x8CHA!jny*{)ar!2cJQrhMLM4+9emwfMfMuotvRY+`A6#Z*Grr3P4 z+q+xC4nAk+No7xuYcY51Bg&BSe5>v{>{su+d~emY8^D z&tLG1EG2j)_GsBhR+*!!cjc)+(jFEw2763fhzZcsDiOG~*@n>Zfvr8Ic zJy7LZ<3{O)=ignRm@=r7*?F6^!VtN%j~rkWc0mG)OI9pwTvvy}&uU==q3GiRYZNT+ z3Q?N#_hf0zGx)Mk_hjDKXmd{nqd$qz7P*JT8*lb!y4c*XQ<%#o86a=^-K$;Qo(8i4 zyT{6vjR^$Sv_Z!3$$MIqWix)a~XAciIZiw*!BQ08fq^MZbs0*snmrSTVe97njHFm z$_mw}CvMaLeCz=cBU4w^`@dbS{vurYJEHYBrvOU?DkiKmxp&!lo-1TWI;_u>chGzp zyxfKI|G(C^Rphzy0AXOX9W&cmWI)yLH~&44$GpbDUiWpI-fi&+VQ9`ls;Sam9|$?i zOS9xt`5vdEUJ?tjSW}mB?K5W)z0U0=u(|!$sNJew%3vzBWyqcS3xJGz$AafKYDFmOD3HuDvnJJ^_fO9cGhk@&iM58$nwWMlS-*fy>F-lCXG z|Jj4+xBaPv{UyHHR%|WCnyNl4P9cUJLU{qU&v?bSXNXz2^i}#(2C~$Mn~tecFAmOU zJ2QQ=pJHlQePp%S5{f=e>~*AtyvBvH*4_z$1JVs=_g z@4((OGEeE!r1*=$5-XWF?cPi^EcpK^|?a$S+s zaHJQWJ>_|}u-?!Xx4LGM5w1RxVy>4vC!ED;WtRfL=4(+d6t)J3Yp!eQwbm&?*~K zoaOU1E(xAOV@lvzT{hyku_D|cO0L2&%NO`raI+^xzrEN=!<0DNj;|P2BcB*p9&u@- z6Eqaxjc9l-;z9<4*zZx_3HL?X=n2FPc)Zy#nY#G&DlaewSmTl-(Jio-$ZQ1qEEI zB#rm1a%3AYod<%fJHyZ*lswfF1G$tBIR|Co@G9NIVsk1_6G(cHC+NMO!M)N(OhWYy zZyv)2a{2qDgyC|=+WMlTRdZI+AJuWUhlA zY~t4|YswieSu&L9y2OL}xN7HAfE59g2bI2(Vd$I0BlP?G;|Orn1@;> zUQThj!zvmn(HOET=dE4g%TQnuLV1ui|3R!iw0KV>Ds2m3s-Q2hI?cFl`{w{VWP3ha4>+*bv?PvIVbkJqHF1o88d&8}3a??-zi-h%Htz_|;iM*ozk=nAY zLeZ_ke7#^aQfrp@t{$7&69qAQpVx2{prp9tAA@T~u4!lW9?=({2c^aT8oBQ(?jE+u zJNY3_l6g~|G0PUO&Y$M|D>+U1^8mH4+6{t)QI`6w4Qc;aD`~5h&|xx`{~Ox9*Tb(X z6Y=|(>Abl+k2C~4C1W=Kj|faYvs|0q=7+ysc}OXxq0ftozGRTTbX>xA-^}h-Am^Vq zbncl|@1%kKU%S7!O5;asES4kwa=&GckYCUihC+RC-TZ67EcCZIU1uNeDyi3dW&ivl zkZL?AQP2-EE$AD}fjOEXSz_M-l<&%deUi$ea#13I(dz*ZrQNdssF(CfX4Co7KyIc) zV2Iy6>Mf0XS05t81BVL_oQ#? zP+gkfECDA*{c{j3$BqNn{Rp*8%~|5gq9G&k=q$?Cr{#S6hEt&|$5K|<_S&A{0Mbd^ zBg$8)(-Eqrv^s0T;{EN^8W@HMcNva4%6{6sxPJ4J(5OKRGXUInrQR}2^iL*DI`j>m z95=aU6X!Z*8owvkpfQesa}aN2xht+mtp6%dLE`N~7=d1zM-B8Do)pDTwUm8D zGX0zs#gu$_U`Ew>`MIW)f~}?c7_i(fJJoEXN)ALPrUVFO{OLQx8>_+S&!VvX`{w3E z+P8`?H8?*~6E*B?pXbo5749ULGO0emIZ>4R7XbebHwZeaL;b<7a}$^zh>z8AFPqFW z*+H!d5^q-r3#3;eJL~gm73K@Xob&MSv0!v1!t}0B1bsQH@C3JOhQz9pcgzfow8m%j zVlfCa$R8A^v*<}7W1ZIbA|Et#Ngc(Hnnp47i|6+KwbLLHeQAmY#@-m>p5sZ^=esi1 zf32^z0^45lEDhu0apB#5hZiy>n>h|*$Ls3=gNXVCs(m#(C-bk35 zn;m30ulUx^JzLi3zFJ{ESErsd(YR<;^7eEf1r`JuG%@Lbm?Ej4S@hJ?I7_1NB6EX@ zOsfthv!)1YukyAB9$pF@y(Xw4`K`LeqQCuRLnLdOh)Y8nZ|eTXg!qDG|q97~Pk#lWTL+SY~f2a?21_w8m*wnaE&FuI6{bL8WGhu`>0OvA*} z_yT8UN-mKc%nIZ5%UafIh`Su*T?Gbq#Pd{pk831UTv@hu&oS+cYT{EosaTdGI7R6t z5AgRNWQ|Pz0%-GX?jP_@L;byFqJ)*-Lk+b=8BN?O?~lW^K4^RtTONp`23%*+lgw&XK2RQZL1N7%T7y|iq1p+>Gid3 zeN@rDE=htQecFDKHEBaAv#5O$pQ*yzS1B3UUU^U1sYarD7z2g8*u@jI8UW-0|ewL7bI!}}fJ;>h8zE!^!Ak>i@aG~gjxHDvaW#oF_G zyxJ`FXr0r$8L6yS%)F%lO33EalLAN8L1NXhhO}R=N?Sbs0*I4Y7v=lv%jAp}dEfSR zrXDeKpDZ@XFMgV8YA+Pc_w&G~zl!&d0yTFz^sIE74W&u>;cM-$NGR z6K$F72gu}foq@XIS^>>Zn{FI^!7qCLIHvG@T&5nO+8O&6;jCDr_1>GPm)pPfYaES9 zygRi0k)gIEr{IQKOqJHCbYa3Z4KhVVTsS0uIwEbEkasbLCz6POhjy6p z$(80RjU7r;q(`pmu7YpBeEYG9fub&;dsv!B=2M}E-V~w1e5#9j zAqaADs?OU%l;B8*e^l2-{PZ4Z@y&XkvW&HB}uBw1S~vX0|roe@x@Nkk|@m|&8`kv5v!`sGHhi%s=g zBpyo*|8m9w2tR8u6J=49wO!GcFs~mgT3gXp;2!$pIB;jZt;={Zw#{MRvfsm@+Ioos zw+2A5A0O;L5x2lSJZRs=Bv(x_R`=p;D8v+E0$~EWrV%$8D9J8~RcOr+>7ygTAjLc{ zZgv_J^AT~cqJd&uVC*HQ-YDaQD3m5?;M!nM2>tC)YPRxFMu{3q0q`T($QsXf*;^a0 z+m*`I!VP9w9SP`MMDrz{QsYeh(?Ur%+Qo;IWHFF4Wd|{!$1WZKpW{Z6GM=sRSHAE2 z*Q9kbHF2wRNl_FV#AdqOVxYKD_)54u9_HT=uK8pC<;S1lmP5hBj73sh^(-3!C)&Im zmQ0M>7_(E9VMxZfP6;+0QCVe(qCdMlSz5h3o(<_=&}xhzymdW-Km*y5jq4Wu^RUSs zaiT?D!%cv!B5fx&pf2eOU$|DQbrb#)eXa}`?i8e)h`u1Kk@T`4d*lgC(D{l&wbw`+ zzvtBJyw5gFsD;J`+dKbmat{&1iS{b>9BQnDB%Yd7qXiKoK{ZV!zT=Rtf|c8b^^k~f z_D46;MsQ6;6zVUk>=VV@yiY}YvDG(P?@*smZL4yHtgkO2pDXkCIsuE`%<)+`uq7DB9-+;UmK9bF(MMUgd8Q#{`1iAF zUd0{!`qVBfLQiBVOxs+itIE+qRu$`f?$0d+{btnAbZSpv0e=Gs zMiHSb6Vqbhs7X(yB@@Rz720)O1wMAeHi6&$Zn^23uT1oijJC5|I4(GhvE>@FssHjTzoHv89HdrMyL^glK!(G>A_L!$_eMzfSZwR zd2_(e9{lp`t=hfbyCK~+$rVC68}q(Cz5tXqO0gakN*tIh7lPQJb<)Vnad1FW@#8l< znLxAtC;RxvTe6$M2W@%*g$Bu>huhtv@$UCq2(_ixv_Cn(kDh5@=tNfGz zX?yZlX{f^qwQ$B}TBZ6pz;%H;^(YtB&|a~9R_Z${9KeZ__yFyeKM$u=3I{M%RXDkv z;H7#3NJAmyt8nvA6G{)w6n?ckZrsuhe_J4o|I{o zGTLdHcHaRcy(kcQ2c}BaKH*+ap9II+i0konJ2_(H(S(^c^+AXXq#PM)U|N9@pIYuG^^mfgONcN=)Hyb z3;ZcB6qq-WO-?-@Y5aPS81xmdkbo)OXC&<(^3X`GdcR_NQ`Gb8i@$>%iB(`xYZ77o zn9Qv&K1r?3`fT$hXVyr^Nbh0CIdTfwFFsDK6LY0QM&s;%>vjt7+)gHfn)3H5}oebUCRMB!DvgRHUpN${zr-P0Z0>-{meWF>g z*^9Q{C&t$p^yfBM`dn`*lw~u6AIJ*%gX~mNlRf{COg0N{s}cVEl2OM}^QihFqTplS zIx)F*@ugmP_BO1rHFEY);aK;sKO~W_mE>J)Dj-^JE?HM|EYNP}MxwBM*_NE7w^E+{ zsl$bxj@0)?n2+1;UHWE~L<-5wdz9liTaUfy`h(NtMU;y4hKls2s+6iN&X@by)L~p< zH~dT~C<^QKV0?b2|HV!EY5&=x!jgsa7x{wIK@oc#*4+rj&yj<@|1xHlq}ga}lLq@d zPOUra?kxdwx^jLj=L(4K1t=T|F#apJy~kgmpzv5j`eK|AN_#%7mYx13M$3%=Aln&Rvu19VmKS z*o{q)VUOrI^?A%S1N)4u8obmqw7PmHs({Bo;!fDp?ND5_ONWXZUP{Vb$?Rx$DIadW z4yA9UV{eiK4vSRd!TB9R5{ecd3b5zkF7vBVJpGM52;9k@4K0(ebm>Zts*$zAakoe$ z-C=DwCVc=Gxq#2J$w1YtQ1aK;rP`j((2j|donNDstUi`LmVwPh#|H_|-qao31yqc; z4_Ws}m_Yn0nco!H99g}jjgq=OpWf+;RhME|x7;>?bF)Vs#PxU&pO%gl$1g3(?|XGh zhuWjE5!sl^P{dN2E*oUu_Ao3!2)surl#m4P5a7P4mLKyZRk_C{{zchuZg`11YT)yB zCf}*a$j(|0S$1Cnp^3?{HJt^2!|b1V6u7Fcj6BJ{C-Tiz1qmWv4pZ*@-YK5E7jmaz zM2Jd4Bdsf)L)KEN8IOmvcW{Wo_cn z^DVbqt1gADZZ2?PxO&B#=>gc(gEoK$h1y?*9pQ63TqR!aMmPM|E?UaWXZOw{o`pWq z%Z;Ec1J$?hGwmnuSw(9us^QK;%@iM%0UkD_r6C(pRXubb!>J{sy4tyCgi=$cx~i|c zQ7MMMW1PxBR;c!HdKf6XTm_j(Ho>V_DQHz)IV`m;G0@|~zg>D(jgG00_0cij)j+HP zr2@pgCcH)^oN2oXh8qH0QD?g6OP%AARbrw}Kvp=c`g@0JzD0w4MZM+n!^*I6y1Tx# zYyLh)!JdxjLdFPFin5KiN_$Us#g}lAOL8O`p?`3p@)RY+V!-ytV^{6PES^-9^oU%1m^;kj&D=>k}4*B6Zk_IVg*r&6|@>0F%VI`Us9^V#o?mhJBjYZj6FV=&cP@8HwYQp*^Y>B~?kl}V| zy0?E9!&~JH{o14z)#iP8%>=n3Em~gpyFWZix{broWYjFF+$Q+s1waWFe~I7u%Yt}V z@QJ;DE~?u7eIH^wE`sYQ+IcOzAB&Y8E|Mzn62N}xmSNGY^UM6&3(g=Nr&xxKIs-mu z`!N%mJaU&TY#)}mlv_)jha_bVcH(&bC?#LccPYPE4u3TzkIiblX*@pexV&1su@X}i zMv0$GMpRkn`hs6Gd=$OZH_mmQp_@ecY*q>5OU3Z8SJxmI1Ez|SRV<=@3MsVJIAL$v z;PZE|J7Ll4)0%_sm3ij76IzQRQ3qRHmmf0OYp;N`l3 zqZtUgW$oe7)s4q*2LVT&_LGIghmK)Uh=btFfZx8zg&dJ8B%PG;jYUv?Yg3?X>owv! zWEC!ZsjN1RkZiT|iLINQ-P4Eh&%;8snJPlW&PGKzewVW~HnxU}wx$2K0nk?Ze>r~)G;`9?T) z@iQs3@Pn_I7ay8{cqa9Sq<`NlEeD|BvR%-qT)gGJ%o_=Wxk5cc%2e5j)64C2aX2Bx z?&-t1bZCR2&d+!vx0XLs?kh_DLmUhUnbJ-s78(1}Gq)_EbxMW238}RX)LG@RBAlsc z`nsJZOa`M$1S-vCIbPdalx026z5g+!Gd+wGztmtH>gRUmEP(aBW7AayA&whQOOGw^ zM}Xf$F#CrN>zCg7k zk^=gc(ZX}T zRRTGAH(Cu9*mO{1f7nIjT*=blcJ}`^*$6v37(N?@Pr7DuXTTX`gk_Ul*jZjVdw*=V zJ270aihXa}eH7-4Wl{vx-Za~z?gS385?QL)gIpZUHZeuYN#!@rF1tnG*_jo^KTIe%e37sIiCc;&69H9}6VmW`=PR%VvKFs@Qh>HUT+)bGK&PAM#kRI=?i8C-vH} zz$z%w?R52Nzt}}E8&DoRafw?gc5#clk3HFay}RyEdEV;5#G{gaXKYve2U`L8dc%B{ z!&*7&ebzo)#fI9Mh!}iVbMBTu23|ivx-$zD0uZ#v||o$WOC;3AtSk0-6vV*Vc5NH54$a*jbEEq)WD`Y zg@#W-?C`F+!WS_OA)R?)$q~4}!;CN$;{olMTOWt+Ab~Ikf|k0Qb!1p2b|QDX|GUC} ziflMYys)kg32y5$M^7&mq<54~bgxbeBpP9=ZcVQ%lW{AtD2!ydsP(zq*ir+!U<@b{ z4oPgfGoD)q;6Vj~{)S(!x2vA&K!7HFJ>R=h*WmG# zXdbmnY!#;QrdgG!3R8FEfMeH;qY0k5!M{tu1ZAY zjxxPlCw6}aUq1@Ne~Rymt(*+GxI>I~VNEu!u982|0p;2wrW-;qkO;bVkHv(M&X(E1 zZXCvhi(g}%_eW~x4Ht)>`?A73YHuc_K}34@H83L ztQp$xKaH}g46D8~=fuZvxv08Zj=h6KfXSHW@uC~oCww>lIUk1!nF5;z^4_e>xwGCs z3^85_leR4Ma~}6`nP40bVHo$l{cprB722_;@Ynax+{S+*w|d|GYSW;LyS~malRF4% z9XAd4zA8-Ph1-Asa#7P5*7YsynY(kuwpn=XeM(fy+UQ=pWX{8t1!06;x(oaNx*%}O>Wi&IPv9U~A@|W-KT`P5 zSKS2i|5S~=FCk3o|F37Y-F1Y}{aWDy+!b-u=gJU|yDwt7RkzFZvG=VNz)A!q`sA9! zuW`Q1LBdSXeg6rTz_I*)icVY&tGZLTzfmsP+OZ3FrT-m1^!n2MXwFs1_28IM`I4Pp t*LdMwC9;XLXFJQeTy3;h59 literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner23.png b/docs/images/LogMiner/LogMiner23.png new file mode 100644 index 0000000000000000000000000000000000000000..5fe403e37e3a48946c3423d2291f66de92c3e196 GIT binary patch literal 715930 zcmbTdcUY6#(=P0mEo`N2MJZCEq5=X50*0QbY(S)0w$cd*2uO#}lb|9*1dNJO3=j|y z6(uz3JxGTTiu6thRnmYYgz(|-eb4#MKi_qo_xm$>o;6uBYv!JN%}nBNS(*qRJALfX zp+mx^H~zVE=+Iv$4;?!E{4f54Z@L2YY!4kebIA0cYyXD-u|DOV7GfF#qOwE%ub=SY ze=eqV{^+lwX4R(`w2z;ES<9b#zKU?jG(hl#Cja?s9{llVU7kJp`#dozW0(7dy<=ZC z6U>1Fpi^sx2KQE;lC5@2Qc>t6|1rMUC;!(Ud6a0A zTnv)?mZVgW1=h_ubo=rDSjK<<;359i>(`L)avE;l{-5vTw~Pg~TN1~Poos*d!B2YF zH~0TqhyO$2Pl886aR8nSKdY43bnzV+nvw({;< zkY)mcv) zI%DHMN1WFGd;&B=azjm z>HEr7o_VlzdDZ2O>dsB1|worc6Sr$m?+q8iL{#+pW8vS#~#Q}Ex_1^ z^1QeO`8zc)1+96VdxM%2>k7z>-rcG_s&}_m{Fl9m1!%hFOLv0&}h`Ff!fmw~ppx;i3oD z7*T~Gg+rFt0zPe0s|#c$72h`64KsGN(H=f<2Mr8Cr{0>(C@}eBwK~Uz(aCP-x zk8LKs9u4*^XeF+pH_t;%p)-bIFuFPN!BUuFd04Ln}yczj)t~ z$v`myUFKu#-OiaoQ35tA+G#J2EDHf)_is&qz-gK8FAt<%4kTOo7rN(el-YnVbyAZ`U9&M*L&j?`e^0E5k z+$;CSzODZo`rBK<$N31ocZe$#_8?p?wGo4T3ssLC51cb)OPqpQJ_J-M>Ljz6rERea zWcz=)+UdwiO&x=cDyj<+`VBES&_4DRgH4H^W;*?>qSkQ>$N+^h%mkA=$i!+H>@A*; zzR#P}^5%7EA$=y{(#=gD*K9bxp|V@Zttx6cNBY3x|9iD2Z-LGXJW%^RYR!_UeWept ziMkM7wp;K8_XXF|G}m;s?eP~P;Wq9b<{suCTANBQQ}++_oN#}NK`>pr>u=qXzBJgJ zaZR6Bs@%x3!Y7yB9$Wt8d}goDWutJHt|~w2@Ap#BU-xB-OL7*aoW4`u{g;0N*V)4} z#*5=071?Fb2^e3y z&bqOyk<0c~yH(-T@NK6waMi_@R|Ppx>0%gY%KA0fqA_By;N4lSYngugLBvh1kfPq; zf_heo_6=S^6f%rGGST>MubcNQ#9X&+0Dx3KmM4))F3&svk+exT?E5C5+Xr4$&SEj(H(r%)60#e>6FkT@UXmTGlBgW#@u$9aHQj%C{>7W5?0;iRCC*#jeS~MC9!pD1 zD{r0~XzN^G5La>?6Kt7>&dj-u;n&0dRZ}A_S>wXq*Z;;3A8Oy~#4Pr0SQT5Vn{j@6 zVZNOPFd1o2>K<(0DsMr)rAQ$j+b~0i@@db` z*MT+Dr|qg)DBX@IYl`Av-=}=Q$A5@k5Nw5l>M0*E#2Pa1zsDp zcHhs*SXN5}K+t}-d9U^_;Rb)P2o<9RTXX#u^r&(7X9}GbT+Q?x<$dS9$gwr*s4^pC zJ%nN_tKesga1}J z;7fB*UjpUK3Tfd?#79zJSx)71i?%@Nwmp06g#l&1ikP4G#(j^EY76P#_hP&W`Re$U zPj7)UV6@d}2uY9=Z1EpM^~Q>TCA!8Rok9)w(n2)+qi)p7(oMOV(P{pZ!Qsq3tND*? zWR`79_b-pi5OezqN0Bbr6u)3gC-hrivd4XB5X;bZcTPuS^B|_>kky706zH)z)4iHf ztDA+ah4!bVouT9BgA3MEDYnRq|KEn8gjNv0RDP*EfI4Sc$zq5ae`e6CS8rxN0Mc_J zH0LTr>lD44@9;?=-#s>k3vbiCgnDBh;$HIp$7`cp&P`d%E^FHGlT-X)KEClU;KGfM zmIEY8^-iUWm?GA?n_6~R%P$$|l|Wy?ST|lyEAE|sM%k(iSzvuy%Z&68U?`%-OEih& zMOFQ6xMziV{bB6T_n)qXA;uZnj|hh>__mhgO+qKnRw2ZXQ^bow@|u?pHK7B zQG9oSVb^s57nYNNU!;&RuZDZCoMw?s+6g@MtF7Nqzu}$%07hIgAm4Z8i;5bXv5)H& zoM~UE^|=F9`VXZXyzKn4&$6~rKaG6i8m<6%QpV>&N&J~3$CAYxjJ(A^kEu=klWA74 zl2q>lXZhwOeXM^}>(#xAa@c*>APjW3i^C=9NIyFY>MnpzFM$8?wS1X?y8y&ykpWTN zkK1Ew?|ZAnV3;Ls`YyDUsGa%xPc-*o z*GIh@4{;Nj3x}H$Q;#}a@8GV=a9@tj9h(#_sd`aS{3%o#618vlbqJGvC$)i@VbdQ&0}ow% zn(Uyi#^ccM0>y8LKI8;Y;iKG^_D7%-tTO8&OydCXL_gF$y)F3mLQh_x zk1b6pm`%wL+-R5EJMwtY3VMJ!3172d^Dj#qReI2yAI5!%|K$<)w(%D@dN5Rk@$pjz4^4Ct=~Ck75ixZXcx~Z1KY;pcjwO73|zT3Hg-Eey! zrf*H}~X2m*Q z;P@=`#K)!P{*PqU3>iY_CEpSenVxZwx0twwjNIPnMc>miMoCxdC-24W(&}4wu@XBH zhaT3OpMkAUG0Y}!Z>J{h8+{+kB9@~b++HdCWpAzHr*iYYPwVd>*g0ZZgpc-}fEs?Q z((n8R=AiTD9jxS7SR4uu`F7tTsMgq7rK03;@Fox;mfy)vGKow;+~`$ zRG|=f@+`W4Gdk^plF9w&6V~l>M-={Q~3oZA5xA(n2aq4GE zR64dnNo?7GC;Z{qGgS~(iEyDCObQs#t4hv7d7Eszc!PSGR4!%+}GWm ze&gu6Uvt^H*UnD|Y!fz>!d->^_Z8Fm`5&$g5_Zr78lNjx$_5v1ZdcSndUwZ&$Oa3= zx)n=L3m|wu`Jd0ZDnC`;$eKr{fd8~IutwjJ9F-CB%ss%y-JN}E&XkrWdP8?3>h2!O z^^xtIk?%3UGfb)Zxa9ZS=@wlPd%ojpKu3)1$-YOkkTLdQke3i z_aengsv8=YdT;cBY}(@$+l=bx30flI>LAl7ntl@HL)-g%NrlV}_xv6*5v;%nd1C80 z#?1h42399hKa8(xLk2nVMtS&LnB$M|5>Kg;yBwN6oU0o1=k`P0EF4CzX z(w&ev_)@gRUupv-7SVRmaO#n{HYM%Wwzl1NN5frqLcPtcjb+d8xnDNFZ2oqM16bsl znZUdY1&=6K-L77Fyu8KtagW)~ zr(b2XCe?16L(5VR}dvRjAV%B9;8j5Oskpcost|EupA79#^!(o_dfS+O8nO(5S*rz#3Yiw#|$(MyM z3;%5nIUmTaMtz%R_ru9wECl|!h-T*DW9hq4YIDW&N%)jm<7$8)pGSGcm)@FxU&UsG z_JX};u)jQF9+Db@(3DHyy*Tv_aji)qrcX@JZ;MQnyExRy>7D-tI_b7>+G$h~ruf(; z?P_=PTfrmT|JdW0LVG~zP5ZBW00N3}hA0x+WB*+sbiSb1d?6v98BKD_M&wI=EMMsmynyx_C+ocZ_Le0V*8#vu*Cvdm>f&^>5r7kDILBKhdo-(CdPq!@nZc9Qmz;{!m5Eca_p#? zXeQz}7qgr0c*n!z*ggyLs&(?oGF&Mnq*{G%SZ0EFX-cZpnF$)N`ZJ>2xtK3Fd*K-`}vwWdxWq&>^Vi#TSrw<^_+*EK!X4u34`m7yJ?l9`;v5ry>6 z^@^rEBOQU|#5=$V%+{ROOX&=uWae|k%De$1Ss^H&5IMj?o<{!2ISN~GaJyk9b6P*x zQ<{g+)up;?MYLXkF;Aq%IF*uK6L|B+a5UDQD@=;iDIzSmAfdL1feLH=t{#m2S)(emZ*7E7}tj;SVh9iU9>ivOI8l<`7vi;kP-q9x7hg3P=|Q1XMg4*`Z9FcKY7i)+s&D$FHF!mLibEdy*MD2s-L5iYSk;%u3WS zz&%R_mCwypry70KW^ziG9~0|DtmP)4u+yWwi`!<;=sud&R%^}<& z_pAkPm_t_g%!+7S=<~g=i)tdNiCeq4vcsyRE|vlV6FDV(upg3DEuX4o}4dQO%KXSHjxBH4a(5~?PXYurQGh@?DLS0lZcEZ1Q21woI)(}`V&z387vV()&+u2@`rF}i+3Y!+ zH{x|jGPGfwV-x5ZV??AkgYByR7FU>L2tRCXe$k-EoinpJtxI&99j+ACoR9H<)6e7J zzPDz8Dj*z$o?ka-cz+=1Z?v(EzHS|3vR!ACk?3SwmDi1~tQx&3gVbl_CnPJB8VKf| zwYFu=C&Qe`OO&R|l3+ITF>*&2d)TD?54qs}H!AgV41f&#e}K-_1o zj=axkT`TJ!FhtS1&N<0h>Q9vyjGpDtlVLVa#7?wRhKKu0+2tGem(LM26fkwBl%0r< z4h>ub0>Og6Lr)`0tMqdV1Yr&Xta@20XMCiQG{4-6n!D1efKQduMObevN8f6%Ky4*g z8oiUoyfv&*gCbR=+qS!as=?YWY_>ts+#kYnZUL+~tEHRs!|yc9=R3`+zk2t(nF|Bx z{awmEu@VN)*z>gOSh#yOw$78~$%4-gYScufNQlB)=n9D@tnN#RdJ1*RJNBt3` zV`yuZ5g5w3A$Gq40)@EcseZ_n*U8zE!G0(Z$fPBRc+!qzz!g^k=t{Vk=gL_eD`>a! z4{2ccXN-{^%QiNq@SYxPCbaSqHYr)FRX}Lz`V&p?k^-IyvE0wW7|UDO?7yj_=cJ~f zJ`q83^?}bo7SduE6Z&}|ZG~`fO`x_yYa)N{U*fBm$NF&wmLa8_U0qIV_srnU0z;;=G<`8g)UIZGMGDlqU?LK!&QJkNliV9o@OTic#jtS%nRcJJ z3kdGIk9Ac}`;}Ry>FtXZ;@o z8kEN!3MX3A>k>3b*0$+GjQrdyD<<;42KQN3zSk(W(>`uW86Kt2Nb?Gr0G;2VxuD;k z3aDp#6&0+pyA*r}Gu7Kgm;fimS4B>D3>}g{^rV>0d3LAH3nM3W9BKuY}0#E@js`jM?u`7M!1WOAxK6Jfe88wE%$O3 zWOFQz8mn9yW99W)Ne)UpxEtEa*b!GWsG{H>!JURCUY@OtP9aejT8SyZDq8&mMiVXN zDch%H=pVBjSD+a|&_kceqU8Xhkn_p7N4oLJuo`ZOEAtc@5YYnf%E{N}ekf40TjA;_ zpY{J8oCnp3qlH?1AGb^(oRa&*)hlVHjs#2U543=*giv$Ks{(}QQ~SBu3+!YC-B~{W zEYhDbUV=*j@D%Z{+weJ!&H{pUdl~e2H5A&`87i_yV^HhE7aV z3FG8#d0EGC7nlgmVNvO>&#A1%?p*&@KuBb#vWx^a`!+P*#00AQ7o&5IDq%9yVVSHbE6>S z4S#g?t)>bNf0_giW!P8%wUpb#=m?!Ir<)39vRdPbtTSJ&N-Yf&fI-YMdbW{?F+CQx z?6LLkEb~@xqmFWuDDEsyY__54xWOkd+EQ%5e3EuY~RDv(8({G3VX|%7>iG zl0!tWwo^wnN?T){WTYb>+DC}afADATN`tyA(cPBgcdCDkWskSdJp$ar-NXH6pDu)^ zgRGdwwf{Tajis^z+V7GzhDODkz3hvgH_c^aZE|w~dnEwC)G8&~g}EKZHOeiacC^kc zI-wGtHL@pmFetJ}n9N=WhBvxANzvI{Mo^%E@REdm=5neZ5=%;e(FdlCV!4-;=|!j{ z@0UgU!CIbN)$@7w&tAFYyo&AI+~XmQt?S2!O^64b3p*6eji{?LF5cu;a>`x^bW#yr zJ_w<8lD-!qGf7v9BEFHneXSp@sOgZ`iGm|PHW@2(cFS)zX)Ge zS9eKa?EA??{nia#_e@4=dBpFL zt;r(HBp==;_t)kXC$+1VtE|sfQJiIpUjTt5%Ig=!G%QR~yjU#@XGM`V_K)A-^f9O& zS7{h`#pwYWxko*_=4G`{a2y|! zs;3TGsuUS`TQL`Si&pzS5)_e7%PE>$VN+|4jkE@a<1bxKE)->4C_%p~F!7X}`$<7a zamPvGnDe=cC;7Wm`+wvnIIYj;itXt=cllql09G|u*J3~c&4Nj&)pHWEd%rgKP~Ilg z7>M+h053jc`+?N$w2@!_QMD6VlTG419TN32+ATnVuyIw$_9X*^!pM< zO7cS_$K7T>y*D>8xgDCTV8B+&-Pt3Rvh^X^Lw&{E+TBC}u)7?50^hU#)=*XxPQW=P-IJ*65349yi)z@}eP6$&3V@P_ zj1?N|=pEjKH+9l8iyw2txIgslZS%_th4C_eojXke2I24+>FG~Nj#>E&bJ-^vgRs6a zL$#47;w)S?Ix|dd_KW;nCW6PM-q%iU`Q>)Q=3e9`0|T0We(c4Ui&Y)%ivfklD6^Kz z1vdmybfL|i>dn2-kfXn-Z}uB@YWIK1{JK8FcJ%ET0?tZ7NVLwA)7|;1`IF=Cig|G& zQ1)F3DI_Hf877Tkr&cZIpPv4%T+#8QRzI3r)m~Xj!f>av;A)UZ#POj*7-Kxe+CHbC z;<`leu~27d1U0&e51?5ym*l6<`8LDLG&W4BkmJ412lh`be$?`Y(-$nP!KrQ29I$$8 z$#-p+ppImWY>!5Nz8~{S%ThOon%c0wI7J}&JY_6&d|c3f2`mg^8hpA7W^{1rS7pBd z8oKRQ^};uOWp~Svp3I%}vM#*SaJ%fU@B*;7KKD4rr{W~>2N3>KZY5hbU@k#{t=0HL z)dUmCbTy8q^@UxrV|O+gvW{aA|9YdX5Ch5fQ_87f=ang7forfCvY7&VA3i$o(X%%@vxm&RYfKCOknie7%}-Zo z+KiFozH-zpcVLQvdATuRjN1`od_<=(#Gq&cnK)CUR~f#s<}^dk9)`^kRw$z;)yr$? zd#oK^@601|tH#!6rw=gp?!=!D%G`Cfsyz#etmg)-zYgFEW@g^j6_$^n+?$=39vXxu z>J<|`ph%|dCY(Yd1EXVKN4zn>uiMsdY;1Z~X*4X)bflz$@Fk1Qq7lo>3MhKPculiY z`WAu6UB6(wE%?=7aoE4hL^MbPD7jWQdD%aTv`G?(-V0qi!?~rdja=EBp)}+s!+0-3 zb50pl(+5D_QVSmt9A{(^O=%CW%`hORS~dFB{Q#&|FJ@~s){N9g2Qzy-45Gs7e)^kO zN4OG~X&)pS^T_)=!TqM?iH1GBam1ZbZA1T%zwM7M{OBD>z3>~PHU0~vA5^n_Zu!m! ze2@P^{6ajam&_JFaCVnCCfWbzR2;_WUH2hzlj@UaxEnKpzbb4v3m*Dq7ONJ<=D$=Q zEJn{II8B1z$v)0pn`o$)WpxE-F|zPvvfiuGSFc7MwD4ynDWM6Tpy09Y@7gst7IG_R zkMm+--29Kp7E>zyK39plm&e6t6${nrV=j}ihgYI^d7|Yi)~%l1+Oyi{f@-m7VEm`(Iuus{6;%0a`N8ckenamlL!S*Q`JRi?0?FG68yz9cB z@MyMm^b{PZWyjUJjfJlfrxLpI(Pcx=>u1ve#u)@Xz zszumL=s3E;ST`|WGIAW9lAh8xzeKdXU`jz=+%MHJ-G0%*Qvsk!no+om5e|mMq za!!Ef9J|j4Wf}IOHTGC#IL2g(mM>pX`8VeTyhSc1hGsv#N|8>PO+##~PHjf`FDQwg zW>Q6Gs0xUDWmvxNqXHS`RQHihj$p)FGpIY{u?Oc}M{aG<%`n>=_O8-whnsJ=4WV55 z4Jj*M<1t6caJI737%QjbtZY;!n)v(*bfkK3)IXk}G@jj2?p4vvtp zmP%OE^=alQ&h^w6)S>p)Us|V)HyH9c(=R9Qr+tjfyZ7?kDV>cY6j9uurcjG8C$s2J|@uZ;UO zC1`WjT%vuOFmT~~L%Q|R!l+dKw}Tl;S*%aqYzLp8KRZN(ZA;ABuB5eS)x1_v5dqlD z5&mtv0Z9*kJ(o__Yb+6|AD0hZ5o5g4@XR|YCxE<@kV z7-+z}z3%4d(%?Ov)E8+U8QzA<>J203Yyx;Bz~&Okw+_}MO9^XWxXSZpe*C%Q+BY0vRsBsFr@8pkp{o?|QP`_V7Mt8Gjg zN@7LYq>&zNXOLg{q9qi^uH%kIZ_D&cVczm(2DuZBPE$U(bKu*>MQ2Uq)H~Y@9oC%< zusVM0CNRbs2&zq;$=Tm$L^ryRKo%FwaJADLzkG$!0#O6@(-FHRr?s}V{N84Y6l9j? z2MJVO`;I1z$$fDubo$+^C&WE@=crhG+4Z`MSUlt6y7V;bV}*{}YkE;wV}a^*m6Ix$ zeDu>h?1ehKT^^owrf<#;e{)Z`D4;!kf2SfP>AYrRASMdjApQp!r{dOPjq=a2IkT1U zry!6%8F_KWK9qS5uyIYnaDxA>N;a!YzwVZ^i(*nzUHwkYNTUn0O{>1Tn$kEM>s!+9 z=NF+jd#e(GK3tN?xnr_=y}npSj~p=Z^mSCPH*93_RlbPIYX5*B$Uh^8(0;vt!c6Vl zR8EvQdK$tVghmY)$-zI0;(+1#4pykHyI~U8>8m|hKj7c2(yQX#;u*{wsI-y-a_s5m zcD+$yjAYR4eSKZ&-Y)Eq`r^uzvUIO&YXgb!ND@XS)I&hm{-24HCWSuXd6|lhb)M{o zi?5GpXJ+xx(L>>#Vot2lssKep>yYhHXt;wnZgubpn(stJmv51<^FMp$HMd?tg!o-- zp`QWUDG4=Uk6XfCxda>Qg^V6vSW$MlV;`#|Vz6GNe`sMmUdxNx#^<+~Zyo)Z?SYRB z@kTd0a*pjC+w(csnJlcBHsaSgIO09hp!SAfJc=UYxGR28;oZfyk+`?9eO=@`>6+kf z{)77d=)|O(ry%f>q9B{)O*A#Zaz#S=Jwcqab>X>MMU!?P$gRzrXT-^5yKf;!O7qe{?FY*P#7{j zPYPWgKjb|ygtzc;eO@_DUn(nUI*X(Ih4PtfdBYc66{`^#P+-uaum30^LyOYtRmprK zy0pkJmWUYV7y`DbaUM2C-ZI1Z*)`=m<4$-+o?7NxqlnJn$5=urqAnh9G4RBR^DV)F z`QY0WsXaWi*HwBMa&sJT+belo3`>$t;+D{UL~}gsu*j~vxS#Y7C9hu= zAeOTC=C0I}+Z7~^ErXn#cmfbsSS|ZqZ2j0+TA@Ef08HeV^#lSV}M%&_p1lA zz_+26^$!Tk@@aqB)eQT00s}0ht{`u_kyU1TZ)wX6U9gx8WNxo?8UXcpNB zNR51xcFj2@apW;F6N6DxeM7P9)<;PGq=knf1o(M5wBB6okaCkHE_R@tV`8F^^ zN}|dyu-pFocU;5vG3&ZJ-goJzy_wA%q~*T%*LxR{_{)N#w6;q@12Z*l?Usm@WrQy& z73*-7c)>z_!*RLxyE}Vp&nIXX#(VeBY&*!M+01NdsNub{9Jx@y$EXj#-)3Nk(p`8m zQirtdy2InOzbYM`D|yE&o^1BH#e(Xy@HJm+R1StAMD+OYhvqc7wJd!$Z6q9*uD`e| zenVXE;-<)Vl_v95*%#-XE`;k1^1qEaD=5OGn{~U*T+HCRqX|4c8eZCQee+DGaWY)+ zeQG9`o`T_KrN%f5UY0o4+Aa%MFZXS7F?3&4F0;6eMVH`fN{UOp{GN0bm1O=TvJ=lTw(J4CqdQb`X-ZP?4JzF*!dlW&YsN8+5blFIc$kAXs@o#0kd>7cnZ4lxprq&8=*d z#ek0k;f05#K}%|qYt^mIo!@?>$;b;K%+x%>s&EZsV8L6qh+&La2VH5Y%1cV`3ApeAAjV)%G)GgY}ZnxXX1=!-XP;m zY7N}GHeEVzl4}NADpj?brdNK>fS1;jiZ+vrL~lX*{A)%pBjwYV-&vvD89pLU`>s9K zju=cCPUf=T9Gx&rg@&~6WAG^7R9t1za3D%RJ}L8Es;bQMWD}2?!Lvc6h_-0CFo*ig zy&ojoz^I|MK=$^v?ks^2pQWehnzthwPzFn#d%PY9L-eUpap~M30qkk z{K;aUld+3@BSEnpdz+A{jl?h$GhZL)HT1+Bl)qA+QGWp65H%Or>lJ_0ookc&)H|i!<<~Cm+?%=cj*mNPkxK z)4-z!i8PW_Se=b)26b4IwwT?Cddz;xhM)$vEI59p4reslA1#n9{MB{$dNhnZq9Z)| zJcQ0o=&E>2usuS$Oa$l}#$yZF_^MHX% z-b$wq@fs4F)G{K|d&G)-#UuE*>e@{>@T{a*qE_4A+`FQCk`H~lA#cCGr@t|=TwT2x#~Fto|#u2GS~F!x62NIOvpt_yNb(X>7XsAA2Q<2(VyMkzjYhzy@t1NM#!X$ zuN%tyvskI`Efmn776lFoq)(nFg4rdfVAW`eV=BDXvg)LNQQ{#Y^(v3XMy|ovKXX~J zm4-ij%q{jr&gO*z26mU+jOg zjRL{tW2R&MiqcUz-_ufzA-t9n@9EicmmHtz(HBM2i3GIlSiI$`P2_5VYPmI5dcOOB zLk*>K(&0IWelo^jHjmv||5BPg$N7W|mP(@g)p9}QO6P^6{TB#A>+Xa6266!kefHhO z?lH!4Uf|>bljq3-OHU^nWKG3zcu!Bt>v!NMPAD{g*FyHW((-R_g>iL97w_~>MQ@F~ zmxF{9`6@{`Em!F;yFEHhiq%pm^S zuFJn@Sx-nsG#3hcu@({*%3Xc6eFf{e+-ELVbpN7wOQ|iH{mJTA(vO;v__I6=cd1Iau!+iYhSg7xGQYZtim67?|ZJ8h&%d*Vm+ppDk{ANUd_`gO7N2d z4k;%dVjEPZofKxQk1ZzffU5DSAedFa-)9UsvkTM{Fmz#7vR`5*QfpM8#89!^!-^_3a+xn?+M7jX z^8EYo-idna@|a@o_%p3@bKUQ+$v*P;@y&{pYjlFKVnMDROOL&<2&TxucX@xqfu!Vz zyywwCOtys_q@D$jIlFg#>Df}?;+(tBVt>DdT+ItYu(tI{1sEP=3;e8v);TjdFxG6q zS)9XK=_X!FF7G&p9o$#6U#cB{t{ZUFSeLUrJ65CW?i$VPGBYXn96jq2d`LI@LOa4* zheSU7j{P;LQ(l$6oCw^u7`rb2ZIb8vw~z|K+9M$px$F2)(#d+`;RZuXXGYgY`|Z(~ z>RZkku_1dVZGSd84MayAy@oq=`k;6KG#=wC7H51vN%Lezacq(ABE(SxvjZ$E9#wHWIpn+3i5bKSwy}&$?&izFIrpb3Ao0w_hhcaaB4R>BzxIf=`QAEM`% zsvj@OS)T2Qa}mD-W~o$l^!h$bI4vm5CUsf?w9-iAZ$5O{2eJF-p^-jicpqEOv8rHg zog)1MoYB?YzGpHG5tlf4=qfollSLQy(IxWj2t%q8g3GI5KsQ zynSZNs10t;IUCe?kwp)}AB_#bbAnLpA6@esbwl^qYf@*1C!8wK$}1eLjpNA8JiTRf zAo^^;VHj<=ma<4{Z;=qHb~y8`{ftBQafz`Ul3ZsXRoHuA&O%0}#h%)gYU#%;G0jua zxs@HGKk(hdYwRz2y2Zj%a8CVAqV}@_!;9<{Z$igJUzV>-eh+I~#9z1M0@C*vd4F*} zSq?VVpY3i%?|kTAQ~h^#Mov57;fJ8*yZtwbIq?Z$^Zj~%-G+tl{!;+CGfIKFViM~) zmgOjcR*{k;kO*LO8R*m}z*fHHU4QV4UXLNIo<(7#CS*n!7O)f`ExdkX?Q&)$QSzf> zll0`n3`5b>*TMxE+s{3~_A7YpSxasrV|(puD+8*JsM*P+`jV{n@DYb?QNu=!c^48JBLt^Pp8)ijT#88a zXQ)ujV3mK{?t|vTqCIaDVw#`r&|Q)QE#RHOo1(j+>iuZry?|A#dTGJll|&Erx8Vi3 z{b%5~4hdN{$7dlavTyR36HNrRbz$nr-+@$Og!CY5@%kT511wVvNDf-sUl=7J-|7Wg z+$v{^;1kC@#Xw2PDQYvZGVZh=e|n#T#m5*mPQC&{A*i(`=D$S zmR&kRwX=+Uq-Oa?`yfKX%C#2!B?EXVKe5v=C1KgKHw-z~_vdLmUfN3?dE zyz>?M8jS(fH0Uc?AB^-06ef(Gud+|UXDx-Om5Zc8yx#F9_jt}3IBJ1&8_Lc{^^K4?1h|}PY*VolzfH1@j|{`rg!wUL4C(c#9s+J$_B2VyR&^j zC`JD?|Kmi`XKwIIWp_-j3#Nzs@(WBGdRiMFSriHUO=6_^)uXrHBn6L%sql->nYWag zx&}1`q@N$9mi7lc_e$=Hd!yB((xEc6Xecc=kLYYNeye45&D zby93ncAi+f{-h?*W(m3q=J^PH5oPZhT1h9Y_D(RsBC0=SFMWv)@ehWE2#>DzkNoY7 zxl7Dq#0rt!1^tiCUcgt1`bKirGxgz3FuC$9cC%1M%_3LU(l z2+nISK3QRGa-+EkK1iVK8Vt}(%M(1Zdd0uyBjA((Xgrk@Cm-_4LK!DKa#5yuP80)< z^{INk^AknADEgJY2mS{(`xyH&yv3e_6j*S3gr$A&^ilVMv9uq{n_+L*KQDq){7?z@ z6}{)s`dv*V!cs=@3~E@}835bmXxTZ4;9X3yA=M&&&f^mdkuJwecV+@Q(4dWD;)}sH1&(>LW?^1! zHrcYrJTh_x(#OB|j&Y?&Jfc?OJ=FO9-9EG@<2U9>wP(f2hN5LsVcew`Md8_2D3?+; z{GS3YeJ;jnW-*mdggW*#YdOgsJew4|q&@y7Fs(0x3uTn0dT!jeXP6h&iENLVZ zG@x&u>?d45p{Te7=-ZutdLeT=ulHayDon2JI&RMj1xY2mk8x5|Gp} z>ONKt(l2nN8w7hXBKl$vqfLB*hYKC+-)-DUZ7}Z!fl#Q3?4EHk-9S^qU7H$67eXQ$ z#_uAqca-AN|^nk)wHraazC@}tq z?%wlayv)i@j}>zV_$qzSTUsqK#XqaLU1Kdy*^zx|LuW4&uD(c)<_YS-bZUn3L3+BI zY)hG`wP{qDC*T3)&P2S#D0?e`c6Byo6%ZXeB!T)D`$w!M{9RORFIA-2q zF|Tvz^0hOHVn_#s5G>iyer50V;C{g);)wMz znBrgfQP9@opV!s5%f~296S0RQxC@dX326T(fe^p5erLT)9-ZAZi!B@cU$X#o0Vm)| zw?F(zk7z(>>XEMqs`1OJ-~HtKczCizmduTg1d)rhioo^gYLFhAY*D$~a=(RX!Dby4 zWLhh_46}%r7B)3%ns*hTTPqh~`u4F}I&|$<{Z6BuWmc@Q?C++s_0+JxSF>Y+1Dl(R z4roz$NC00n zeV(+Sh=YYn0#sVgARfts9r_hS0e&)h&Dib-^Hh;g@*psu07L}N(+t>Tyl3g+$_#VZ zq)ENNH(Yv*UluaH&VW%{Dvpy6oj{Q(G9r(5MgA7U5p#Bp9&7KRx|%&i$p*`@hsh7D zBWTWA9>BanO!5NO(^4=y+9)Zn&^q5ubd= zlA|*k=I;wWOA8pz592T2$o#CFYBF?c@-t|3$fG;Cq_=7*T?}lwOz7Q_$?7na(^`u? zjOdi-yFdRSCh(mK$0S7uul@#5ka(04oYREGW+l~+8&f6iV2as%wXdFtPPh0hoUwQx zl683iF>QBECaPFIvb-ev{~_x=!3B7kh=mbKCBtS?4fj2YnIp;dR^UtT}Yo6=bd#}Cjwbl(% z<6Ze~qWEcJkPTW&8dCp&N-EaT=ypa&Ws(U}I)Mbr_!-abbe)K$vc!6#RQc0xvt!@s zKcLKR)KSk7JiZ`vTrjNes@@px(e?tHz~2=Lvh zS<1HoVQ#_Q%RwmiJPVwZ;@HlsL)wRDpn3DfPf4$XZksPxMcA<5^N#4p(FSx|t2Mf% zU#FqPa>5sO)-|K=>?4tk-n!l}V(>t~+{EJQ0|7K45PmrTShbu0O`};5TnK1Hz@h53 z5V`moai~dm;-&vG+x&MPN9B1ILYGd%lO`VF`6Iq^U!m6Tvw0sCQO>omEKQ`6sjAWB zvpmb@xE79Rt`E%qOmT**+ z!PH@$!2eKfYMr}(MZ#>bVD8H722)JkMXvt-rU-MM7G{+6_bP(L$ZS)~Lc!RB)N+!W zcyeu`{$&|3HLL7tXjPB2uTs`t+R0PAW0?GwzTYy*JIR$JBRAl1xE-->X~@7#gOGMW zBev#77L*(Jg*`uoU-NCKdeUGOD>VLw{c2`o$zfHesS4KKI@D(5F%#lf6jnPECQd6~uku@4T5a z?Tx)VlV4_uv6m3xh!~Uq9l`*f)R-RNubR?_Kar^K%@`*HYxb@%L=2T3bh1X89b%s#`HCo3Ma;uBzux6d$?H%r@52`#^r)CFyJ`;_NiIRZlw!8EybDA<3&&DB%#`2RU2l4g4ICHldL1c zPjZdr>!asfc=?7orn;g^MwGjbmn~6j_pArY?%BY>gs2b#5!CZuMBOoE=QZb|`IG1R z&YaMr$=PLuvlKqhkHZB0_t362z)4l1wBI%-`1r=GdSBN!cE)Ev2Tsy*zhl^}CcH`) zrN81;T*iT7{KoPBh?^VsTf77Bj$LYsky7Wr@V4VSYVIB?j!dbiB?Nbv-BZsykvkyv zBrT1&^sePH=!of@{apGrlCNFh{T+7Jj2-`~y14$JjNKR)kEy9Dc2-+cRoeZ~V5vj) zAvdnf`c4JGp@EU7jZTpJu~N`_+It;qdU&)d=<7G>Dp|x(QEKWsa?8=Rgfoe8200!ajxymws8E@OAGe zecyaa`R^bUrXVRxs+h(jH37wipW#GYe?N!R88U z>D8ZR2mTWXwjnN7-$(2x5OKBoaQ-trJ8!VEuZ}n7i0zfs8v`fEwyDqLP1mb>T>M(6 z2fGF91orNShNp#_o`k<0y$7$F2tTuLSU-Zv-F{emzV_G8&{sM;%^$xI~bB*8#QWT-i4P|eK*s*=M?W;)?~$E;cHO) zg&tao}utDXvwR#eyLyklrSj z0^%ypHE0?(oUAy%Y-JFMSY{Idf^_9>cX;ZS8(&Is6av-B%MX zUc48bSjf9p8esp-y@*T6Tmv;nEJA#}kB|}^88pyr(S79t#wJ_(% zKsUz_Vr<66Mp0KO52ljVQ?*n2rP3(z?1y^W1$@?+>~XN^eGxYk#rue`Cty0Cz9>>pmg1KD z6Kcysq11ipw2|Z{FCBbQs3Cuz5&-__fzSR@Oge9QS>XHfv(vrrHF_L%inEzGyZuXa z3hvHP4*=+BiVN+LA>%tbgs6W*1>E-prIIk-j;o}R!srB0R$H^oG$G&cuq{jD1+X7- zDK5k_R-BBzUK^h;QFn@USNmYRyw7e6I?dQVc{9Gv|2iTzef6h-ZDU|=rxB#HukCcu zr>OJYnJ3jEeQ86-pbv;d?f_RM9G#(mH>4>e^r8${eV+ zHADXHUmkv>sM(AIr`@0-{zZ1y?g5K+lOW!xa@GC5c2F7<{0|@+A9=Wm*!-+9Cbu7R zv@D0_AIa#X+X}#XWhz^cwQ{hYWC#<)Y+ancJU^45cKS&fE0Ao;6TZ=~AzMayOPP)vj%0 z=vi}jpQ%a7Zd>lLBW!zW+8p0g+>)cbv)g|r$zglV!i&t` zk*gf(ZKubXw%%x%2}`Re6IL{+ZDI{lUTfa+VIwCyrFk}|?+2kD$Qq<-H~zjjnd5rn z?g=8;&e=Zpr#L}~{(M=xVbC{sY{9wToEhF|bh>y0}mo2E>f_{Nf zyhrbHP0Z}V1{NP%1THZNi_6v?<9Ws|$f+kB1H|rKtNldTsNc*Ic%r`UJkDG>nt=X4 zjgTzsy$C7n)ct{*14%5aKVliOX}Un9CsLlx)wgRy?3@B4Og3;-T?O2E0TL!3xY+qI zHda71C+fCu+sh%|Eheq9cULP`@vl_}46gw@oQ}J_A5o>$ub$Qw`l7aaK~U9Ccy1vl z<`v=<&cW5VTR5pf7k)Qe; zP@L7v+`5`P=O%JptfCV7JHjp^&vKyWF;*Xx)^OZ`tcvKRGm2>%50hM1-wGHw$B8q} z!!f_SKZg0pt>&oH;o~*+SuqYkR~<#iV#g6=Foz0pr3IPMo*aFP<3?w$2URqz81u+) zIg-67%|reTbsa zZAx#KDj!Z{jbVYRq^?>pu1&!0@=S4`+$VWpY)M(ssdTQ)x)`;as*PP_O}W{j=Clsf z^jfrV<7+q3zb>Uy+Yz;^0iAmF<*R8JmgNT+A76>gs)eCtnRi7zKQU&I=yZQz^Bea^ zsZ;!9c2?df%dmpIB3G-Ur!d!S0Wf=NPD}W5KfoY6h-5$e{Z=Nh zcq2~7Af=zD3Tg2`-DM*_aBHD@&BK+!e-T#hhsyKJ`Z@siCLl3ug!*xPUc;`>Nf)s; zTKMaAZt%xYw?OEnj}xH=do4oT4*Sr>we?R~foO*}?f24ZiJEYSULVr#*yqn}MyZ`{ z=b!Sriz-nT%5=I1It)^f3Ks^*aeZYrRimm?5X;DMt4WqNO`cfEC;D4m$IIQKRcbV=v(uCE+L$9l*j}_+@QHAx7 z-p$s3I9i?TdG%1qTl56WV_13H^gq({u;Le!JtQ~phqh^O{WRKvvAIqG>N1!%lliMtlh-fK z5R=#xxFn*>-Wb9_e;PlZI-05p56PxX8b;}QLT)%{j7S1HL&NLu>BX{~VIm$W3M3so z73kx7F6np%sekQwSX^POF@*Kik_Y8s(=qxm$?^&n&itiXP5z8}3lva>4R{RV_| zFX=r;^3yL?b{Nw9m2HP+V33?RmI3(vb=l<(L4LIFBbXNB8+ycaVfNas0n;pH&)$V^ z;D5Lksd%~(40Ht>`0}6_{}F_bEmBR{NH)|qnsc?bxC)Uf+EVr$(I>t0s~$#MaT(Mv zCrJ)812+ku-he@4;UTWPG>4#!Y3w&fc4~wgT@&E}P99gN0Q%(sYOxV~TYLm(+ryjp z;5p9NrjuOLxjYhy^^PY9@PGsDtK4cAB?2aC$He(PLO6MDT7!j|!oF4%FaQT&cUj^B_5+dOp-%fzvVxD-YmW8~Ryz)cP zLj_G~3Z_4=Ce|EB^R{2wmB&Z+6s-RhJj%&tI{V@IK)aw)i;>!VPbA)Q2utXB8kxF( z|Fp?05h+$bDs3_(O4{qOW2wF?1mZAnIpQN!N54mT+XW^L@SSPzdl9scp|Ya7KL92j z7`S_1X@vv69VMcsuY6Lto0N}-Zfvk}UwFo`YhDhzJk*RB>h;-I+%v2+lk4X*Rwh3L z`44KRsciTB%DAb;TivVbUwc6iqXUmu+2s?L>@m{XtsUL@6_jjRky_I@_e~l9|M!;Y zf3}vrT%t=a0h}u2C)!k96tr5jG@-d9Kh9(DPlH)pP}vo<38V3wbm1SeL+w44f+z;d2t@Y$e9zmKw{5WH7DXhA%V?3wEA9eL7|CJyf=tSE_!3hGSCvC`&{nVjOw3ZK0GK(+qK_5Su7x=nFlE+5}l zkA4|?*^WF<*B(&E95PoqtmX4KTicQ!v-1yl$JBU9K@hOl&XY5U!j2NmLzTe}JESFx zxzr8t+WmV(dyOmT)s^d?^%vsX@-gSxz1glqcJG)U*O&0CcIi}Vw22cFF*h)dnN-^z z6bE_72{>Upn4T`4#vk}GG&b7ujc)m3II0F~`AJ>k;$Rbs+PU>pQ`sT zuwGbb6k**~EG**c=B*;8K zTDelxi{H8VH6lDQhPhhvv$?Vxz#ZWVh4)l7#@qKe-oKm#)N6W%7*Q3%5-Wu zm0d1~(F5Oy9#fbnA5RVE*8e>Fmv`hJE_;ILbL$*WAAM z9YD2XN0CL#r+h2lP$g#z`!JsjzOn?;juubFk{S<-j&Yzq3#yROak*M5W@)wPvl^MC z6~;x8mGpGrO`T6WNSnXNt=^~i=N{zEi$!^|U|7mG~VWrXvA?^#GyTVT& zk(zX_sj%hRW-Y8-(%ApD39@M;q+sSH3Jyqm=64d5OKRSPoXbfRm?`%Hdz9Q+BV$ep z7Eo(uZlIquycI+!gD&6PHUtyl{0%K5dG)q%-)%z|Ua1rfqK zvU{1u?ZH@C%})s>i5VHnTWJ@`;X{wOxxZ#um@zSA2G~rBP~3i+d$PNQi;%exH*RJ! z7uK%wJ1Jb9--0}xB(GZbFgUT*qJ|F+{|qPqqb-mMNH>rGt>TYs{mj``4q_?|Z0oXg z0Wzzd7Dw1q1uNKxf$Nvz+!LG!@xuGX;!ys+jnNGiGy2vGy2rs`6p!kcUYn~FTP^o_ z;UIs|6&}>SU?Ahd=1NjDfp1NAnOlF(J*jTCU_X`MCK`6!Q8cq>MNs5e&t32%_dti} z#tiJpF_GzTV4=4#-FuI7B!ec85U|-{kSlccZA?|L|19M%TDMEIOl1n&(0RlfK?)=b zWa~BheRT*GvJVgJtiCNN!XHPp);hw*XF~5T-M`oUOqMd$zyG?3^VM01&broo@o=N>;0F3 zBoK$Yz{{{_g5h~A%+x%N*G}nC(9+G1{$IEuOK}qQ(k$$00|5cD5X6FG9eQ|yo!1dX zH}P#XNF71%H4l$uP;xRafn=;=!FHqehh)|wPp&ze&+z^Teib^H0uY!WZrtt5&H6lM zcMbU-7e>y;I0#ROl^3r|qR-AWaL=G@uA++B#tvfo- zt?QxUvd^Dg!RwgR;^9dx6*M(#2=*9n4O>1RBE$E!^fYbtGrNRJ{hs8J`;-;SeqR5x z5ZBcwYS`y193Rx2iqax)fK`u@>@P!x(hJ?;dKRPwbp=Yeh8x-u)zg{h4P zeMtlNCnrmu8`M7bX~11mK$_ZG9cmFf|N)DeE)>od)z+aB!{T%pU*9_ z5h2TZc9f zOEvJVGc|rUc!#%`Mh4FG7!eBh8s`k@FN@qLweGpOWJB*es{iQh{);PNMkPy}l`-

ng3jB@{J2|Hq|Xzsey8>1}o%-}nJ#-`-U8YQv55rn7;60}V1w zf@XK2#dXDT0w#R;mBCQHs9+M;j%P0LNDQz^|3h}lwIQ82zl6#U5i1oD;6g8AyF470 zC(eE!ngJq)-#Q^4IUqO;TjST9wNp!R;qDKTFpC`uQ>!TFhX-M!T};wOL7#{@T!{0v z#YD15VhO675g|Np4YroG4T2wIGU|!KMd&M+U^W&vH?{Be=6aY9qu*#lMa#PO~`_=Q{JUF=G&% zs;Yb!E}~=1gO|TIwB!V~n_&~!j>{jSm8?8&G3|8RM*pn^Soi?u81LFQ@B|NKDS|#} zEAaIgRFXcRn{Q8_opVG|&tvk6i^RV3ABwmv4Tgi=Ez>T#SFqcN3Uo(kq4lrF1}3p4 zasqZAm#)TF{vD@`B9|qVJgtSCic?A*TT&z51Gs>fVBC}kVsTM=OU3MJHV8+B(F(uX z!OY*(7P2D-G8u)cryh1+P~q03&+!cEnDC_$+~4 z(rk`HHf10?+|IEkEf=)3oEldm3L!X3?Tj3@U-SCJ-dmhMJF2&iD=X=$UfckJ+ULU* zrw>1zYnay!wu-%2Ee*wlaTHqxHnFP(gs z>r+R?mw}H4>PgOskPa91`+1Pa1Q)bef19x1QcN#6N@t3^f4VX^4D#3l?`zxbWIlg0 zqO=65+~~QlspWCbkx)l{?~(n+8T;&7K#2~#b!BO+a55s8FJ+m62*Wz_qE#&3jm(ZO zm*>wm!%pfurDnCdB!;~;EPjeqUuM!qr(ZF+;Af5Ke|yL>bO|hUMVoN>ddQN$xpivC>@eyAg`$pCm=DRa9d3PDp_HYJ_BJ zNx1Zb#*L_Rds?Z|pyfY1G~2;8(*C>C(KmRxe@B^99>gV;GzrA@EM=^{LwOaev~^&c zjq(^+hj_WAFJi#|KuxzV!e!3(Ry0rvW_D}P+r`+#D?^rCjkj;rNgwmCs)~&NMI1AEHlMgi@bzWGq!syY63C%;Ep_3hU}niY<1%yI~(5 z^FU=%PBYla?_M9RJ`j(bD4StM~Ov9ZTE-aUI^ZkU7 zjWkp8dVdG-)zTzY+f-=6{#yF-g#DnhXX}XiS1bPX!0zk46PoXahTB%UW$m+GyiCd= z+5_#{bOJ6(LIg@8SspO1rE2%w%K>}ZOzdgngwiYSaVMEMWEuKua@SWp1uZujsi<~p zNFq!&)JZ<`Vj2AA;veHU{-tRC6#pR$Oyjt_Fs}Kt8g9V`=OYi#-l{bdU1qlYbJp$iqx~ znG5Mpj&>5#QAvNY%`bQ?A3B1&J=^`DadopaUDAKg&m-mTR2E9krlf}rL~`I%Q&Wa7 z8a8xWjVRZKBTGlxjy}OXG~8?*O@~sq(~3+>1gAtKmDiutD|Cy*S-@8&?O1}&q~diX z(UP*FRf>DVfOhXdvU%VG{8nxGfn_*Hw_KGelhON(P}hfc?U%1(WP#^)CwsAnERCkq`42$+W}1hhiwWOE0blq0)<5 z`ex=Nt$U57pdYE$?Y`LhoD{Lsx+3G8~f_-1Lusf;+Q~NNwBzQz}F-L+o z%IyAKgzjL-+DUqE@+prb+5UGh1fFTSq{#I9EcS=9`t7?B*yzf%94&&3=S+_% z^7vyRC}k~wH(zNvPi$swYcfjBcF5d)Niuq%EJ#t(5uMhFS~t zO{qbc)4%`hl~=TzI!aMz(19k?k2u>?CF3xzsWB!>@5?$h0i25aYX&f{-ETE6=7>^ujF({|oqZ%&CD@Vxj>Lv=-S||C19<{>GW< z{&yPXugUlXPm=ATQ0%eqp2GP7%)Hn}WnI`fhKX&~O&O}XhCAzg+wj>s@Vw=0UbHu$ zan++lCpkwa?jhBBjfn{R4rSseP^=EG;VhZkwYYXk&wZae`ezaR70SZuju$Xc3H)XI+gHE%lg_W$@Hjad z7{h~ZA!F)17g_@}MI@4SgOw8dQ|_u|} zYhKOhN0}~efz?AQ_C+1Xz^Eh?Qu%&+v2(y9&~HfdaH&A}sSEKkITDJqXG?Zvg7(XAqg*DI=iRUBAF91}N%6MRIa<(6E-I2k zPRtSVEf@C%+s~rq&_zNuwsyh7alyi1T5-(FV1{n@H@!8~$>xh97r%d{Ff}8bnMdEf zZw!SOR2|W#aNHM;C{279ibe*e*ThI`hZ64lDT_JWK@@j_A6?h(gUea~sae*Y%8X+R zmRGqgC}M$bg*EWbd>=BXCC~E)^Ilg^AY-I`Gqa`5Nz{5J&FS&&cvA~DTX%35; z9&GvQ>H9dvuw%G0NsEVz^qE^CCH_{NrwWPmnJryu&6m`stz!WZ;Vu z2g3(~J=>x#276{1NE#Tc_z^c!z%@ z%^^2&iUas#Wwc7A$0}eQlEgMD-P?3ph4#rPn0bwo<<<0Jxudu7?`RIg8CG(E%i5+< zy2e}_9BU8qz>CCB+hyorllhq9db_Qy@?$+|#5_$>lK&=qkwpY`wKonT%IQzaSYz4& zJH!?2V45vfA1fniV_Yt*X8O41j8Wa=WKwGsbUBTYzkP z%2V|wKY;=Xr43RuivuTtf6o0>Or3mF<=dOQFpvoxGoWS7qmAEdEsIM2xlIIIXf-qI zjl+# z>g~*OtNdjy-oWDuq}K!#_R8bf-|P5mXyftT?hw`LyXtQXgD0ph31OX=Urh~0Ei^wEUr3hXqobKQ>$tWQDx)g=QXitl$PJVRi%eOxH5+u+A^$Mz?l#Z~TEbmZ^=JEF^mbY;bB97w(wARh}BGq^A8?4kiR)Wo;_ry;lBC-Ldlc9t3e`y|!8)UZZ} z@~1J1g({#`FxO@Qgq zf9Wek#`!(=jxEa!EO!-r|MilSiA^z~=_`v9$8Y8Td2mH7-CbESJ?I zvdy^RC@YJn7pPqF0}9_T5O;QK7pfGX%2zqWC>lv!@x*(I4H1rX>$<~oFLf>4Qgz<2 zPt+q1*C%mPdjK-+(|h9_M&_vv7nEzuff{&4Gah$Pz+m*kEgP`GnBb&42zR5d z$H{;G?;>bC(vx=}gPhJ$d2sp z9RE=JLlzPgYTk^{4I!fvek%ynQ%lVc1&>LYmG13{RGLQBwM=QoURxtGLmkOVm_15l z5qNdTE*-Qh${iixI_-r?6AnZ9X^9~_6U+(17B;IN&>Ln)>yejO0JD*E*RlpM1oFkk zkSe^0yZWM#U@ar*^y`xk%?8`Q7^$bqcaAf5W~_%Z`vuJMb4t#md6w$rV?_kX7^PHf zl6e>2S+GeEay*{kebm0NrX|L8Wa|@DFGbA$X?065m;wK(ZkJ;>(F(sue<>Evvz5H43$IUBdvm@zs~Wlj#Dvp$lVNg;k& z;5EtPU4U00Q}V@8`zpU!%mI4n1#N5)s21K7JI;-9x-Y2RRCZFDBR^Nsk7lp(9SRrB znua;V4t^9|$6raw=SdMvE=IR(~Q794~FR z>GXhE^I06eSf}b?4v=gtg$d!DsccCj`>eujB z4+tC~yAKPEk#Td=ZE=c0IBIVK^1!p{?nJwybF{)7b&H4qq3p zE?LZe8NI6oG@v+47zo#4w-(M=*m&M)AHwQXQbAifMqW;Dbd9^7V;YQfJiJg7iZLf> zz=fqyzIRac*8_5=Z%NnLAi&Gt8s${HSW@@GlJ}S);8+v`QW(N;sBBHSmUk^?XS!@; z)XOi1@?OjEv_&yaF+GS#!NS^#%LVdh*Z?N}7=^;#;NYgaMlf#G=mXcsQ@+WY5vaIA z-Hgo#VQP#YfI6P=hdZsU`Hf%dgw7J4g5)xHv*l}PE+CkI1292y`Kj`#F}XPAXAi(j z+1SON-2zJ=qcUSe{Jg5km1qT;Crr{p@^HVROb`Su@Y|!#H%n=1szXZmdU+gSV!_94 zVd@QC#koOk$=2e))%?E!ak1995z^}Zo%;2~>-nf6+n5|XpGrHn)DMPk(h(d3#thMb zElL|3t93yqn_Giq<*~I)fPo43uFZZn;6kUj^LZ3nDMiH1%4Y>*?W7AHbGQH@+s@0c zP=bSwhVVjhl0i80{Mwfv!?FjZ_s9CT3lVUsvjfS~&oENsUSmqV+mU%{zLRev8cQ~Z zMN-WdMoh(tSt~}R4zgS^`C?taNQ6{3m7x(mlv|)DF+O(D?syO=sbHm-zGj%vRRnL+ znWes>bli=;({}LL3)_POi|Wuzn4y0!ditis$KP42KV6^O*jKg@_a-h!SJsxL@FaQo zZeE>1&1*Or4Ew~ihsDO#?_29i=jj{xlg23bnuC;pE`68;pr8c?Bo;}Ir+%`2?<&o%+qk@Yb<%q)0I7j zHgbY+DejZQ&Xg37G+~5Fu6zzcC{| zvg#c`!-Q~@UPptOJ;!v{mbu3hi|uQs!<3*Wma)qkH0%+R_ArZ1gc%A8;Bm`w-b%=I zBI$3#{%qK^k^iG{-HNa-QhGaLqfoeyo9n$XV0|jQ` zo3|X~N{^l;JdGAzO5EfqbE8krDs9r+2fGejKbfJ2QqUFXUG+N-+oObhWk6iwu z078=1 z5E4%g=cTx7n%?u;E3Hz&TkfQLr!SIfi;C?`x96GTCn?!sz_^d9IudCoSMPrBMnd2O z#3S53un^!ZXS1mHa!&q2-mbI?EiyM#puVel$RIO`u$cy%hKC~s5$M;*f^9Tg@Ln-C zQZrLQMYkT;J~&i9Mp_=(OOUYo9u-`9OOU^RFHxUWU=#k`1D6!+Rqab^(l=08QB*C3 z77S+c549L?%m75cid10JYjuw%6^y#So9CrZJCC`O6{0G>VGj)|Pv(|zZaSqa(Ivhu zNwT>0etU9|MbiHA4FMVIejPeT-o;?3YGOW0F@wy@Ur2ipKt9>el-Zie{kHi3x^VC6s>^K z6Xx|6;q$hL6QOARm&R7x5=M|T#SK_24D&(t^a3%dIF*ud%9`Nq4Ang5l|eS-Dyv4e zTjReQNB8ExySvOJHSbsrd{P7=UhM`&hI*T}h5gJ0m3v$PjBisGv*$&7W&{8=7`6)T zaNP1}PRNQ+6*RaT;R!IkT0f?^f0F0tZ^+0$nyJY{d{s#eLBP=3E@F4*R%qcV|kVQ&|OuZhLQHN z!)ZO5!0(Btu&%&H$BPGD7{$CVI-Yb4`%0OpM?bh$+`A^YYIJ2*YuoF{X5x*_v0wtU zW5VPA0$nEJ?LEDs!`KgdyN?pDioAY<{@B#oS7{^X%pRJWao)44DoN3}wfV!n@+zo?>k z{(VIaa{~pm->M1|*lIiOIf3D!*Wo>} z0om>+`UO4(j6`o5OuU1#Zc_plZj3Ej`M-FD*?Lo0W@Y`@mbs?{cCj7lCr|<1Wv9b_ z9Zf~PhR>h*j)!ZYx23#``X)<&zrBvgie*l}x1){yu(o}a8y}Hcggm-_D#}BB8h1$C z`Vw4Y39$v+1s~vc_BpyB#84b-${n@JA_UQc=&Kv+147%M zow{Fl_|N_?toAQx+b-lY4#~D;gumUwJW#Y!d$(^Dni>0o-Z^ja3cGQMd1DYB{mbO;|bneh^wal*@=1RE${6D1Vx32zeHVA-)q`ZX!mdOU8lQ*IBeeA{K+h$ z1c4{J8t`8QqJZCrio&bzRZ!vERomVtMu$7waf00#kFli5qM%2pd9O=WJ$6gN13wGZ z>Rc)2f~0Nc7ZJq#uSN>xyWLC>eCQJV} zf*T{tl94sP7zIWZA0pe{CJ1W3~nz$cDw-1c7I(} zG`NcNaXmEfqt-!SwQnqChB`e}-Xw?2_K-vFX9Ni_M6gXk;aMI3;k;T>DzCvE^ZrNij#RO)Gjo_J_T*)9d(D zN!dA0J@V*V5cP299#@{2e?fVxaoGBzgJ9v^Pnq#&C{@g$VoA`GzPZ_*Uc2MvLZ{fn zf}cCTYTP)EJ&LkD7I$!*FL5N;Db$CdnnIx1wtwFL^AQyA5eu=0A_or)JXTLLUbJWI zaT_op!?J}xFT?WDF`+M}E0aM>PzX8x-iV!*OYNweIWe4p7bsQv*wnSt$t9JVYNMt0 zR{?>qs(c>v4bc94ZJmifJ@Z0p1=HqHZonY!D-a9aBa~QooT;9#`Q7cT=?-^GVuLmh zPME4p_#w6R+z#AevZ5i_*tMI_m_UHQ1Ix-0`ZzD#0^hHK%TBCuP8na*3ojv;2-y1A zwAzLp72N@JiQ(`;`obEMq8Q#ONIf>}U17eZNUwGIDt5#_1pL zs{Ol8w5);#xVE}cqu-A_6h(BsjR!|z4hdQT+|FHP zlnKB^mrpD`)6>X`={cND(q29VGifx1H>8Lfe-k#Fv*G5RH_%tA(!BWFa_NE|c#y_J`ID^l^Izu)N6xGm6t1D=9 zLKr(DcTNm=GdIkDFH`n^{|{;MUk_kk_a;X8?h0KJFdlavBp_UCl6d8KSjSm|n9urY z*=3GT->)J+78}8ax*mda*q3eLU-!rN=Vt2E@G}u!NCM_VDQ#UpKTMOVB z*6=;Ota0Ns{*!mY@eI`!+k>7G_c4ub)zqgv`^;E=h570iuh4h{VHUs-R}oyAWvYle=N1@r&YEW=i^o9qbm&$McG zryBeIA@y=(n$u?O;p~g$*sTFT9?2S}?*y&-f^h#epmCG6d3?1xsl-sbgK8359FaQ6+Xijd{j)b>J0HTZu_p%INIJN zr!Ylqkt^gskp_~;It|eqqgK82w@HFbv!NkMde4Twz%^pRonS+5dk^wR-I?^$4W{!* z;zkGB>9lujjV0O|MRJIR?O$CvhrGI#%tc@v`BXeAuB3Qz0x5qu$+zRR&P+%sQ_5*e zkZdoV^!2_1P`I;?|C!;bP{q`)g281xJt5B2G1sd~zN1p37x?h6D+iu|1FV38%2ayW zH)Qx~``TB9?bin8W$Dgi9-FmW#lI#^1E4Xe(zq4AJ-uoy26WvT`gmZs2<`ymVdlk7($ZR zuA@|F*wj&zCT!v`+pj<*KwbG!LODEe-7lFQ%<4^3#V-#emVXgrn^NPsmce9w_7Hmh$1KD zYSZgoRq$Oc-6-bGYWWu3;)if+py?#5hUvma_L&|f3-_Zz1mi~n%pgd9GRR3-#jL(K zajEO|&PGS-wAY5GF3sV!_nidqLho98=j}TQ!KPcD`3Q{_naVY@VS`CYHmgeEB@N{r z-BE#DbF?DFW98S@QL?)$AaO4rKuWj%t(1K7bAS#PR~rH0nR^4OfjwZc*lJswl!KqL z3k&)Xjm6&x%hMFmaiQRw)fxJ;zI01*h9U?_9Lz10=op%RxjC^J#UW{COeNZ)R0Kz* z)`U3KGYVOjNZF#KpD~Oz=0tf3%ET_J*j`kJS5I=VBR2jvB{>bejf2ZMW7mcyUm9ju zfFD;PZNz6CCM`E>!v)ZoZ${?onIi?6GdSNU!_wDB%qydf^n~Zex>^nFyW+fmygP4T zdS0?&X$WJ0sHY|_Mim7%B^1!O)aNR4c7{23(EjZJz(wz-UIMN?s!=6`VueIXPLFpC zA4Fbm{JL#tvUqJyOT!?e|8ek!q5*ELHzOr#G8`IB#E)po3TTp79ojCLn>y=)J2z0O z(XS~$c8$J5ib!eO)y&EbZQu4p*A_hler)NE!=~=b0DAuy2j4B(easV}As!_t0NJBc zYHb%g5)kXwdcx|K3*4M?&iAS zmHz+#Gt;}A$%XejD|eZhCap|yNz4tVampzdnz7Q{NKrAxeM3d4Os!lfom_Guvm|rN zea8|-%>@k=!CX-h#RWkGME&vkUDx+du8WIvpL1?r=XLJ;`FuRzl26)MF)3_QqWNpO zNKoj$Z^-#Yxzi$_+*|G=Kixk!bh_aYOiPL3frvEAo{>62T0WNII7&zc5*NL2-FDLZ zVKPXevE+6gRrL0SS%%UX4f0chy~P&!jqZLxJ?b2O=cPy|_)-+9CqDyalTux+XlrR6 ze+5e`te&s;8$H$DL%<%rCVnEPuf^n~XUOSK39^tt%Y}F41*X~4=T;Z zBSnq8BT9M8=eVofJLJF(Udg-@l1Qp)fegqYMB3u_wjtWPB3)F`F`C0Ox#4GXCvQ?t zJ}LO};D-mcZs-3p)cmu5SpJ%^s7wEH=x9v(==WLnarN@IuGrntti8X@e`p{5o~Y$y zoAt0%3#cV{Dcwr5W!oBE@bhUHr)-gjW>8lNU|*zBQKKz6OM83oxQ8+{e*Sq1Vpzb) z3t}>gwIFWWT0z6WgMJMi*XSlgufb;BzyieuO$V&wKBcs9Yxk6Dq$xBsbA0$%3;Eh5 zDOv}}>^4pYR@HEm5fzggb$S+qp?yP|6MlGgoRK~8K%lSLE`DB=(BN`f*;q9Bxb!z? z+}AWt)_@Y^^^jN|;amvt>QSx=bCEghBZF%G(oqlwnl(jW7|HT-Ar|8|*6gGlnl7Oe zGmcD*7C%%`r2I2`d(Y6S`;l6(5%?M)F8brBOx@Oyu1g%cvu{|J8baA_a9J1%i82~Z z{(PG6Q>pHB@5AU<;x)3JZSBf&D@a*CzheiVQ_%8EE*os2QRf7Ol4owy>Mm|YzdSp8 zTViJw4JQ^HXB)Tnd<<7RSlfDBO-hpnNje9LoYttg)-O1q;%j=&;f@V98=@Hhbzsu6at7NZT-TwIXU1RB-i$?#d9;IBC1h=mK7#auu z;K3jB;XSv!bx-p`Xd3yg#%ZmAz(9JB;KNIwKJq9m8=B>5dAC9I%(_EDgKnpw%yf=L zzB0S)Z&-W8I(`KP@L&XD>v^&~&;v%cZs%Z~5u3k^Q44VqjSm>rrW`-u_7E!m{+)BpL>Ccsx1E#x zcYc-k#;L+4;^LRNCuoN2e8Mt`y3-c6k%8H?+8d4;L6`J~*Yy7|U5 z`k8CNVyP>sAH+T@e&C4>&E{Gs292Rn$Ig2DFCLLW;tgk8Lc6F*U*2z65KQj)vT zxDsBG+wD{l!1_ETO4BP8`(YONbz!1VPBPuKlsxGmZ!JD*G8IVPe5ICmUM~0v6?ZFT6Sv)Q4nm`1o38oh^MzoVHe%=s?@xrh z$H(n5)lVGEOOz=Jv(u33WU6j&+Qu;{orR>_(PTWpJ2pQHi#+g!V?syzUvu12nOriA z;0m<%ko0HLh_Jnz*NZ$vef7?cn&6}}8If!~jrlQZ25N{c{2V&UC{xN$`{7J$Ywk$N za~sP$a~~UA7D2E6m~mlo{zd}TgvB)uQ9Z!FzzqnG%+C5r^b(h`?0N*SjB7^EU4pKl$NK|&VV=c$Y8WQ>wwb5B zfP7DWim2aTti}qN^!Hgqy6saYZz|BPfFKBNB>*}*(luz?z}t8y!?LvpSWG74heA&# zAQ(hgY0C?h2F`V)aP*x+mFnjY^=$Lrw=n7CbM5u5txrFxC>?NhO6>O<2A+ndUdMzU ztl$aF4M0UtQgUUH(HBrHrLTBMw1kWsN2NHm3&*98kOi>FPbIpC4PHG&@EvYF?$8K7 zovm^B$M=c9{ZIFmA!W(^+6`5TU#K!g^;nkCvGV_Xd6B7gnxr|t{B7(7?PFSsb)q&_ zIQRFQy54JL`E;27;+Ri8iwJu<@!LwEZs%g+?T1NFO^dW*zSC^3dS%V3KjSs{v>%31 zq`wys&Usw9;#`}AV~S4I+;a1gPT*}dl(6fl&nnghk*j=Y1@|WZCoPEfe)uCL#3J!v zYa0Pi2c_NDxQGb4#CU2p9ng&V6G<1STx(nyh_;&Fs0;kcg2~yldj6WtedJ77kND=5 zVvnskdN?La@gg?65*H_nHaXU}vXJs^*(y>G3*8~?XwQ?2)}brTub>T?7a^>LE4X$h zdRy-x!L(p;;Cq*z@37#Hur~3d>kwB=1WCy0xTr;v_7d;`XBFahco&$M(c#kxkn>ER zaUb1`XzWS5OS%1BmyXT4)N~;fs$9zIe$oP>u0lIjyk38S2VKBIA24Q4mUOr z=ev;L=hnCS)-M<$89fnqncZbWjcfldmRBc+si~zj`?jQgyB(WtvARj_-&{?=$joC+ zlF~R=sCfOztMEdlmc1t2?ykxjlN{Uvs}2jF$OX3+T%4U(t#&*H$ctR4PM14`bZ)oB z0cTrQIF*o`qJ(Bnd>0jA9xmP#l_n~)$hio&Hs|+=)!9S69LnvFoK7OXIcf+@}uu?nYW3yinzOdBW1&+ zcQc-~wEJlwj8mOrX*50X5_$#IZ5JIoo0Og4vqRJzeAh96-KIe=M<$#1-rN4(5n?}h z|41yjmk7@@W@3hcqMp#41gC84#WCL37efv6`EnR$z~z2!Ij`%lQn3zPv($P3Y;MeO zvYIRi+Kbe+p{JDx-V&ZPF*5LDE?toj75yiiYbgszXdLU~B%_&$#(WI^=iYg*vFcKI z8E#b_328``T@l9FxWKh6D=V!rk&L1U;&NR`tpR^mY^Q zPF9xO@!2I?gv3xl9uEfmsh6@ogPfElU zIl~w`cwB7@M6+G^ZJ)bzYxr06JNBH0G-@LA?8K{HX z_R6WKxe+%m*e;uNNSi=3B5@+0m2325)83ID<Iw)REtxM2!Ym{gbu2!6 z7q*$ycd1u&0o_lT#URVEB5_aC!tbYH;b<>V0@?X`le(W!9a;TH%1(89%0A<-Dv{$W zFytr|#iE&3iCO(QEK`&yMHz1MeZ-M%U1Ldd( ziO!WLm7||K)z7Qab<2N4w6~00qfQ9c0$CCI$K|Xm0<5HFIyw?&&t=An;H+E0!Z^{{ zdfl{Y)+Kros*l!u4?n^3SA&#m=q3V> zp%AOSpXr)vnu|LZJY3kaz zKwA!E_55bdTLE71s!jJ;tfi|Ga*VDLNS|~Q@$FLSIDBtM9wB8xI9MK(#GobxG$!N%De zhO5r0Ve1iNTSx}+%$}piCavi$!8HWEJr;K^W=VUx=g*k|4{4{^4baxdx=%2Y25d7a z{Q?Fa!t{v^_wx$)-e zY7GSjvux8FZS65bb*JCFFd-U>b2VKhEB4jur?IT6SSK9?4CAq`^>BsgRJ+JM!YcrH zp?~NVDk-Gr*S6a6j$`@#3no=S{f>|IvIy(WNekE;wUlY}?js+sIp-~Vk|<`}kfC#j zPdYB_?t@5~b`_|UclyJq>q>09VGC~O6Cff07 zVC{&hqLO4^h3U&y5$W1Ms~#7l(Oy=;!=enqJT3WM=^2FvNXx1}y3VF;AuhQ}$%)YJ zO-=D~Cs;EX^Qhl}`Qy9~qhl+AQ#n$qBijFX=S~)NSj{YYtk*VUh*3$EHGb4aZvJHC zyE7K*T9V-t4B%|ks^Ps0OGiC819w_^dX(*SPm(J2m=z}dj^<9GO8SdlWYx-qPPHPa z!X^syajQlz)}lc#0@8POAlsZ$gmGe*U+%kl3H!S4F2QPl<+w~ArZI5GSUE9B{Lewj zJhHk*L}uP*^-~s}Z?>Vjq4L&4XJRcf^{@fPCa^w^3c&R4rFNgK-vaG5qK$iwb%qH< z>Au7o%%zwWbe4RMy~y9IjF%%wGkZ0oIj?K3}z~hA{4Km%Yje{hFhx zc{G7Ja=Dea#R9m&HE@?DqS3B#I=FNhCUtV&eJ$#KFwL29PF3?{$y5jl8}fqJzAF2i zC#CoqRSTi0BN*N)4&Lg>kTq@n|1 zF~4Zi!O6N1aLEgNwtf5jZPqD(l(T3i`FB(%>kX0)r5v?0=wM*p4}*>8Kiw8^2C3Zp zo(o1*iZ=O*J}evB@WFk{yv1no7vXXdhIh7vFeFMVt~)y6dt59fS1a0BdZgP{J9c7g z0$4gFCsP*4IpBJPkawOie{tV{Bg*?d^EY_xtT5B%I$8o0^|fb+#$Uyxc{O z&Yp&>p1JKYngxh2gJALf7$NIn|L(FC?v1a>4EJYP49VW3jVYSX(j|vrRsu*p2u*@F zHn4BkRD%zI=*A>g4C z32^J!K-i4GS($E;u=~X9-Q~$6<3+-mvE-)>aH(H!`EEvvWO0qVZLg;Xpt=Fsn_)K$+Una9SY!;`?W*WMT} z2a*TNx!9q?W8K3gD}*kJ@>k{?ua|G&c<_ek_2+HtJBuFt$tQP)TN~%d_^P+%fk}{7 z(_|ewPcKxx8YZVc^YN=%p}t4py-OK)I(LpYtu+Ly?Te~iYt1bgy*O|g=VqO(Nf}{t zpJgXdbc&*d-NfYdV0U=_#R6Z{)@xXWVUB6h-P)HKD^4M~0T7C_<7+hXm(SIb>9)l;^Y1#FOQ)I>ekCQC zYKiWB${g@MR-LPUrn;!!`wyfvpgo|O;zzJ=oSU~Ki8y^Bm!xLr8CFIk881b6-gIQa z=76zL4#IgI_J0=w{eKtYw!iY)g>|PMLZ$F(%Ugrl$zRRJrxR^U+M@o|Vt-&Xl#f(% z)h*8sy<$(f?+J!YTgP&Oi0^fU23C#p_W@-=+B{w8`dNg6B7ynsO(5TtL1FctfsF}^ z(dn#lXB{1lEv6YS>7}DMNzM=5-)(og^AG$hC9&NxXlrH=<0-zK8TI5MHYiC+XbvsQ zq8_`v@{xR#NX^Zh9Tz9YNVFWIRjELWEOKwK><5QPWyH%*&qDha>^4d2GsRzHpLz&* z%AKTE)k}9YN9xXFtQct%1)sX`6j(UVp)Gxl?Yun?Eca?O%SyW4PVOel2^SF&51^I} zy!p}6{0dU8Onl|hkG*uZ1re=5OigF}L(sm2_D-pKTnV{#Z$U-Hh4FIGV5%1CX*F8v zekqFGDSHH+R$+o{|4(HE_?-W` zWkBO&2Lm_p!yF~asi5}bErLsa?U@_C)6K7{1Y`xA~Xd-FOJ8_hGF_P@bPE`b5e`5!W1`4=tJ(98hi04vIh zmrQk7Ol;dfRV5DT$U@j`Q~KwAioY-Kem2g6d_jeCDcL!wAZRPAHy=7WhWZ0<7C-rC zH4j5ep2`KM-H60+bGIYs63-S<`t&F|VC@7cmj`(Xlk|MJ-1X>173!0QUn{Rh9`CJr z@Gl)qTu92f$n}{!&06R>ZM9;U3eCe5>jK|6cs9?grucg1o0)2HQVKohM~v3zQjLsn zKmz%C3X(owLgPQ~^{lG{u=<&(tapJ63q8u|+A@i8?GfN%AI)^V$INIurJ6bLL-`3$ z$vu_$TygF!C(Q9nbmRYR?l*6$96vzRF}}T(4s;$&-I<~OBH}#89pZ-?rTk9mT6R^{ ze)XYn(~Su|ph4kj`%|5Vy+`#gs-gla4!%0FycVuQNQ>lGm$6I>(u$qJq?da6+bpYB z*!r`SfAJwqMH@_?FEUy;RU0c^6=ea=#9j6Ygqa5!_kQ_zyr}UK`2L}QdKhlRW@OIX zg|*H7tD9x3;TbpSdeEJ7D4}Kej66%XusA@FSVn;UK;7!#jFo%HTyeI$F(v10K5rs} zZ7h6oJ$SXea@aAt+pZVVHiG9AHFKCLInZ$t?SDaZDX{J#NSv)GmV*vbw$0=AJxrNK z`<$wx%M7oJ`=~z%2+rRy@nQxe)E_g8c}qaW>4XUH)TJsE$&QUci*Mq-P@`KH&h%!i z%wOcuLagFHWc^Ju-5F|7n+?a5#WP^R6P!Gk2#oWdg>hTx9j4RfI{+oO&nYlvb0BUx z6tM(6pp=hRv`qL4CW@mjEv3d_0^%}5FqL$7eYr2dsCvZCzhw#brM$dUC#3mQ>yyrd zK^9Kbi)$rRDx0*Ntt2%l>!@q2W>u!oSlINgRqT%XPARwMMfoczs z?(j>{x0_&dR|lGGYQ4_d^a`M;$RJG04X~uifQdSe$iYb4R)>G=@68(885hfZW-_Ps z3_G!A8Iz-p9`%}|4kzShd7_BgLB8ATfyDTQIS+}}GRY!8=WT@bPGNpx!JQANa z-irqZvh2j6A+8`qj6%pZkYx>q)C2&tVn5}?SWBc@ey-Eu z;%0?Ia2fGnk-)o3Xyyb1!9`l)Q+FoEd5$EhjmgY`79_5|OgBHXI>XE=s-1bGeSm~) zNl$VDf?P=s_Cah5fOP-u=D(w0#|@pcZsn=>qb~+x7{yiTMOr?kL0;vS#8P71_H#wE z!35J5?+=CCfAPo8STnQathJaXP5spNQG$!pX&qTiOWb5z0EQRne6I(8hl_PO^2h(v z0xSe^Z>F0~WhwJL!eE;fUb-oehz5}Zwtl65NPOQ)yOwFEk>%lCr|YWfFTJ0JLd>t& z+R3)@{?GunyxSbzmJ1P~WxHlvir?VK8{*+j#p=m9nj^%+Zgzz260}XDUx!nYGyKJ= z;F|gQ{-V`L)l?i@kv!Qwfrl7%h1hqXSRl{&THQg+_2BI{FPyuERragkUOq5sQ|pWrTATTDh1DcBpQVG6kFtp)>sa( zb#w4+N~w)vQ?pJAWZf*p%eS{6aq-nG`fCaNX;aPFD&ArPO<5m{VEm{D9O=44jjnYv zkWxw=>sLgputC`dURE}g0RsS4no9pgz~g5pzDCM_217JAL}^*CrO28fKjuc zlbZiTzBD?F_r~7*_-|nbIJ;PKwCocFUz9j5O5cO(YX5dNwe+{FqJk<=%|77A)=ZBt zZ=oLPdx!V~K$u`PwA5YhWSpi8jbJ@KT0W}5KRmDFC-Ex`Hr8hw zTD^=lMw?)wPbNffyFD#R=+|{lFT(ZE)la4PjPa84F~)W?6rL_nzhVVloA9-sZI#Vu zXoNJ|#j*BQF$^;j9_!JPfOdXIuz?PiPlnw-D8DEBXQmwXMh^3wRzVr-ZQ!q!xqfzI?D~rV7T}A4?6}3$) z2@z2WB}9FC5HbEPjBrndP^O!AofRlp)pj|lBHh~2RV+O0@m<&m$kK02UjrFwIMvErw5>%4W7!z7H{f_a}`M2CazX{tM>zI zxepoyXJ?iQ21P!4jQcjmD|K3gLmQ6!2Wp03ZMNP%-ykvS_CZ4K{9qh#^KCEmeiMU~ zBgmnd*!FDW2_oHmn5Y}Zlzsvnj0{U}Z0Llk^yYstQ)yd8Ot$&x3`qn7?+8K;p(Y`0 zO}*Bg)f-t5Zxz3iw?_%bE_<5z_L*jdF(Ksy5W`4t5P(i?W!%47cu$ zfY6sUNWA<;KH_4UVaPv3X#BR98msIGwSLs41|K3Mqtf=XN{2Geo3Qe}Raxv2Q5eSM zJ-ct{K7K6oYVgx32J$QUFTDEjiNFkM^!Csd$W;K$`aaL0LZ-`MxYWN(U|6l4H;IBV zR^~I$+iH}9v$=~fs^Gk3X&Sh~Lv=J;Dv4xlTCiz}xcl^O|B8kY7t-h3_pp2!zh>DM zE9f;q*@3n9m(5(!b$GBH!U@b>TtgzmDwB&lxFzvAS_yemc2d+Q!Z6o{47?*47~|aC zPJRg=2m^O#1)<-l5hh3H?^BK1z1~)9K2)oA?LRbJV~rQZS}irO%Cs7>tu$qrK=3wZ zV!qo+i#nEHP06%#HjPEh|B19S7MsVu>4AHWw&<0LMu>%IFGqEfEC@34JwGXk@f@?y zRnirE?4xs{S~Ckx3qFdTqavg2r!)X(EOlXf4)v>o-z@h+UkV(H%#%s=zi3AB*GK?kh3a|z2#b9@LH}+ot{CRESyM71;GQ53f{#N;ptKe;#V< z?2QjT*uo!(x|&2p2-KYCxG6Po0}}Sliz4{mvSCc2MVVj{%3(%RbX`FtA}rk{1o8g? z?Hjx88+7;lc7Pe!u!*wZjh$Z8m2FQ450;3aYjxIp%U3-GJ&pVo(1)*1g;+D!Awu)H zA-EN11TT#5Jj@+RNf6nhVhQ5AMN#E;8C&U_sMIF)jP0Xv(bfDW^gt7n;w^dp?Uz}$ z3rTbx**a9_oiVe!uU+eLQ^YF27D3+CpOpkneZxLf!4wafSg}lkJl7iFNdDizeIpG( z>sEcu*_gX~+q<$5X2FE5@Z9!#$IM;=7@zQs1q#p8mU_|NVP@F4&pi@5T(ghTDG^ZO z1tb+}#K8qUo|x3}Z*TP!5-bLXWM5J7tcYRpJnu9$$U-}Vw#6N@?1U!B=Jpv3^d&-x zz>Xz^KG|JlGx|oIkzGEvDwfi^+m*o5nGbfj@wLFg(t*2_<0GY>swrpov zdMtitIkQ?hyWFK6v8L`tMJZLVtF$QfybgX_`1CX~z7*`3&}u1ext$jdIy{~a5PbqLtrnVq zu{C#MdY)l?>7thVoI3kP=q3N!?A3-RDEOh`6yZ_g3+?cU3Fm|zI%-{)e zjR<5v7PR>#&|`4aWk%7yd1G$n6*T$-YxXwF{ML$e62B)}-{H$F4|Rok5mkZw;vd$BNfk|^=){Td;ekMWH>-Y}^ zPumWobg)Uzn0G@qGlxfH>soVx)L5DK1eqEH6olMM^Gy-f+eqClt@S=D6LRLO%8OLc z8w+=t1Eg;QA)8Is@q_e_J5aWarroyNsw?xQUDS}un_!$MA|!5@9lS1phNm;k)_X)d z{g@ru`9>cKt!}~d);`PiMmB@vH~vvrm@g;zo(|mA8}%^Z9tz_v0;&76u=S1Fe7K0T zZ%N|+n$0-{ZQLu-VE@4P@$_KVLTATPZb9sh%&UqYa}b z9>`17+k0oy7gtPGFsC|Yii<+3_>8AVr*%9Nx%)ONsu7WIJ_ks67Ff>qMCDTLCUepk z8NQ0Swvx(9ls6(Mu0KKSj9Iz^$ChzU<;!r{u$hCjy7pJm8vz5AvS|pLTwTZ5&DQIi zvlWHshnH5k7Pl(o51ynGDvLlRgGgk_LPJGQU3*=|$}({$bMs@o-}pDR7>ZBH8+PXU;(2P8wL;}+CT{W} zzVKk_eb^;q@jDpBM3LQr<~CrmWpVUcuW4|`)2F$$PA}{NE*a5v11po(UGcq91wj&YNBkGU{XH4PRJlo7w5}Ec zVnu*klbd_Pht^sjn8{@L`Of@b*{Sz2tFaz@_wO7roJ8kfKHgoG*9gC(k(0GRb!=vs z2AvJr>|tXq3(;S$2=zRg#;TnCz0c2xPVN1w0TnwzqNZXhlh-jXD;dSJ2mI><>3KJ- z(Z}d+G_woW%rGRtFtQ>!308@R=IF0JHIn&^!FyJzV{o3jQu`stMP7 z`iNu1nVi|fBal$Hf)}MqL7R6(w;bzI05e+Fiplw>A4Z3*e!4XmGq0}v9Y|~1r}-vY zJTV}X$SflnB+baaH5~;db0zt=4$S6s*U@CwT+pw64+7iH%@y0OcM<&(bDqxfJ7$~G z$?pD>rg>wyY<0ZFSnj2W_3Zdn#cezjF4TRZBnU zL(R86kB=p@lJCh4TlQDcFYLeGnuG4oVKVy+ zf_V4pNU7#i3n}+$&!0$**b7+HL7({($%Dt5v-RT`r`8b_d1I!wb|){tos*psv^dqg zkQdeR=uW)%o4w9oBAsRWv9-kGfdA!ed(XW_VErGIQ`#4n6U)tKm(SRT2^PLfP8nU{er6Rnw@{JTzAMcFM#%oV;d(+l zAoGhgW+)MCEm?LcTB=U<*h>~WeK}&h{P`$mV?gOUL&AYtQJ4lID4gSyNK#Lq=1ZIu zCL|;)PgW?h`}3>8?93f)*e;+9(IL!7D$J9RupZ>?CfkSgl%$Y{oOA82E(=~<-SilDz7G8x6Rrx zCgZdQ@a;9|EpkLcN=QccGnnDYMn(zusrDalJ98HavFn=iX`UCKscB?}1uyGK!+*ZJ z`*GR6bMK2(<^f=l4Z{9)JNUYo6=H})W|A!%{O9i=QvsUb^{Y6|>ku&=dNVJk4_8fMJlK*(OcNsWm}Fne$YnoaNi$XEnL zOdW}D`E>e_JSkr;^q_Q&+%uE_VgqKe^N?6sXWUPrPs!=ETVL$*Oq8#MKQe$^cQ7nj$DTxZ!} z7_X?q=OTFn9U+mXdtKF~y>;iJhpf6o><2~vEj7O_>pZg7F-s_CD;C);A0P>U#Lnke zPs2X>rkb4dM57CzrjNed=eNMabg~?!&)CtN^M2tO9jKDTcJ_Op0lGN&w^-b#eh9AO zXUAdE;yx^R9~F3Ls2e^N@lQRa16n*aW3L2sv+lId!T8`+H4taigin959OfE_-7T&3?kSL=nZ`CQ@X%+|I&QOEn{|Y?o1{n9M z4{4h%kb3{wr1~_8T~euXDQOleK|rz4wrA9B$w=Ia0+0@(ZLqOodVh-rM!7tQEt z2TCey*-V@LaN?X=7AMuQ<2;{R1pYb;Kr1KGA$sCLeU%Cxn z|MgD#i~AXv{|rOGySBxgTUlWq1G<|RbsgLN8D1H2?2-1la{E{8^p6SeC(yZdn}sUw zI92UJz=D1$rTeB5uh;TaLgYLC$-M2pqN4+rKi9gW`F#W5?>!d_2j4ank^)>pJM4bdRVgShah}TJkZq8yJen``O}?ycx$+-z zr1(iMQ`#XGa2q4mTtKyOFI zHI~oQl9d=aiJenEc=5j`rtNIXO=Kz^?!O_&qo}_Q&zAC?oEgVe4eT5BiY& zmrTm?_dm7fAJNc{$MCv`Q9Wcc`ESK3uq~`i_7og%(uIBMjGC*V4Sl6q@0*}NuMO<= ze{Ue9A)xlDCvn@76DfyETqOq`LILP=+KaWonZ*xYT6*HsgvTXRI_QMLG^Rdwd$i|Q z>_L7VW^1S9vMFw%xuA1lp_-)spM2j|jDD~(u~#bil>ou!Meq?ki^nuQrF6EX7N`FX zDZ8A<;yrpXYsgp#G{S(vsW2rHC6F|cl(8js*H@D}V?HKD*U+l3=Z3i0xn=JgmZ#?< zK|4KE#TWSwk)UsIeuHOx+l%&jjSk*~fdx}lhUC1a^uMLrs7%MOCpn~*fU64Af6CUL z`j6ORz}MvDg{pOBZ^&m@)f;j1XLZnv?0WqK$lTWFzW@g-Q6KA5F01C^BIb{18)KqP zekR1NJg9nN$(Sewm17+G3?f!^QV)Z-cy&ap7a^n*+-l|2qY#x_;vP8g=4HcYNqF?! zc_kOGQQKj!$z1C<)nnaLSKTq9d(*75`Rt`y3>-vjB_|!^sLzUnzav`?% zGu&-%=X)os0F|g_CBe3OwToL0=}Clt%B&r>`I*%`@krLrV%(gb75sM#``DX_69Wkm zUyG-`CT)Q2LYLU3hxPm4l{VZQt{HV_rG%u5uBw1jpLUo}Q?>@tFgRVf#1Bp%7bASv z8S&CT9yHurzw0(^PI?u!ehC56`ag85 zz-do=g2o%cU0gy-0oRoPRhdwQMx`xP$W4U*Gox~bu4}%jx!Q{S)^HCZ%RQcH=*i_P zy~1q%k{y`8r($-yN~6jx_)7&KH`uQb(OT3@4V_M8r+ru5kkV^Q3qGi@26%4W?VKny zDUE<3KVZyn5p#w_g227N??~h*eeUO?HEwsL0q^^_QPr10clWB=7M70u2y&fN>{Y7> z1Z%>p*(rgc(YKH#oi~2ovcsVJ4KrG@b>WQfzW8l+ zvMd_JXur7T{Y&tM^QoHuDzkNWQz>(oO`>Vb@AZnlJ3VU8@aU!1x`4H7?IV`Z&&4U58&i|T zEc_FVIqSw^L%=Xvt$1qPz`r|p&I8gwOC!}jxFCmajX@ym8vpr=DOx7usHL~ zTr+k;`h1BPr$&9+*+_^ZBp6bBmj4P2{Q6D)i*CK}7Uo%jNh!Ff{c=D~iiDyIbLp|V zYvtcRVVl2;NiU&(!_T5~IX8C~Vh;Y-n^}EO-_#lTsHR@4(!{5A#rd93R>xP1#^B1p zcFtFdS+yFi*RYEaKH3#Hxbtm>Rcv&T9^V?DZM-C$K8)ink-rya7KVjFH&FLrTtMi( zrgLpY1VC#T9F?2nmTAES4k@JAi?A){{;Ozwp9GXdzYt)@cl#t z_1f;j*QgH>KWOisl16mY;gC>EPDAh4&oJcUg?w!*X;?;Wh{_S3dE+6Cqn6443%y+c zN3CLRXV8KmD<|TCnL=(U`}%4um0Xl?ws4-M*SUYRV@=soG{b){#p4w;!KsxdAJAWv!8)0tjazx%`pxAi}d>2|shli7~s@YebIZDR=vbJBVDB?Y#`AVh*V zPrn5W;C3}cTO-&6^EGiynA?|;USPu-DQdA`4zuFy?DeElJaBOKSZg58C~J2qs78gb zOjf&1dJ0gZohLEk&2aM&anCcy29wl1gi*xADr(7{e5`T6oUL15lynz| z;$Q)hjWerV8AU90d%cPp+zlR*x+|nj5M}5j?4@Tf_ zesb)5G$>s7+F;}8##%s8Pa@WZ6AMFr^h;s9s;+NLScN71JgcYF#X@7d6U~ar_^*6> zkDWoy-v<3>z`0JqqmrKo^L|$tDl{39=3oTFU7qG6d82Kog^OmMz^5(PKD}1aI^LtQ zq)KsHQHv1u5+;nHTUi|R9*Ut&Jz}T9FGnl}K?<=KvqKB^7U>M=xqA=3Yx?hw)=7u# zYN(*?=6yQhim^EwOXf3s`SP-A&6zxBZH%8Gb#%`VvhE{QMR)bV3}O@2(MW|+nX)TMe#^jBrly_zR2fZK6u!QbY;P>Lhx z^!=R7X12yq{a8?iXbJHjxngb^*lgk@`J}pe{sm@%MuO2_0Ekp+aQoH10fq4<+igOp z(;9v8S^y-A1t8Srxa0J%bh@Yo76gS$aMshbSF8BB@C%XI9 zrEH>ZYC3i+UlZqCY+FpuDMT+mY6x-X8BSU016tEna_%Q4KAIdPxV)7ELFBxRh*TRj z@aay`Sn$s)%XolRc32@~4nClsXsIvfp1Rax73~BcX6Do{)vLUiNCZqxhFGINauDFP z_ZqhmK3w8fg$zQHRS@yqy6aU<%KEdEoGMiK%%myl>qJzfoB7X3PWTaBdS%kYkFAYz z-)-LwTj%d|tR20}3II5(^6UPe7J!o0vioD5+|0xcfPe_Eih!s`LXv*co0d%c1!$)& zB|q01HXW{%hzerd2fiIae_H;&e=O?#wCSa`w$mT>PMrWSI6FThp1#z?iN6rg+q&9F zcOf&xqOON^1d8xz8qHdhZ2XcPyZzikt2f~C`rYNenH>Dq&Jf<@N1(NRiW(qRG99C~ znV50lsvQ!&oSO1L##H#vlcGxRVL0hB>rX(6gFl4wUs(GFzTl@qVm2B*+I_N=Z|t_- zo37@(539LMkp>l`&LS;Bs0L$Zs8g%RWL99}guL=Ed zEpuMCCVTYSYRM-53EO%T;VrptS9|d0On$B1VLnOh*0^>TQ%jCZdI-|^S~qcMz8iQk zy4QQzIjH7B=VBoiu`}wb%UrThFOtm9F?914ijJFG(Ecve;(ab?=Hz4ceP2P9?x)Yu zGJr)`m(fvI1Z9qrrHHN#`5Iu!a-~mTE?uTw69`@tf%*wfF_*%Xd+T)L+F$pwl7Q3G z110m&$io2@iS?O!TcP*G!VUgt2Wr*Rw9_ggRBQc8gmKX?nO>=Do>FlN9unV%e33py z=tzsI+Wnzuz3O|6;Eg@*bYuMYa>Sf+5o583QlLwBq#ID}IVRa1kjzA{&0XUcLlM|- zwjZBrX%E(YpUgi76f@O-2ROI!Uj|-`t6jQ}(|l1m?6ftxYW~ag@NZReMP_1_64(NM z)`nVhVza(d#l+JBeXaYjiq3N;tGKaqP{TPzHLcpMP2QbshvA)Q$nVLi{6*!2FN1pM?Gt#DLJ^4Cgzo#yT~dMGi!Zw#L%N7y&E$4&dGU>ab5HZ? z3UdYw)0XrqLTP^o*}JW~le4kU;+clL6>oA?o%`M&t*P%Cve!9@(1g&=hsXo%B?R(zHjZQf?k(2t>yiYP&hZn&xcCs(BDY1lBbP4S?#-0! zUjY_zN2?~ChMod+bYKTCtS7(jI8MTj~U;P3N=*KH=tb#pM@;+u0C$1L_Ms06dzCuf&E?RS&rGD%1OK zC!gV&d(B**I#rcDwJAZ8={j%o6la61L2Vy$0y?rBluFl*=l7hN_AH&1=LsZp$2_#{ zag$%ww|;mP9IK#b9dHoi!S7X?aeG=G>mte0!l(>XA!t;Ypnzm|=!-4>j<6SZnZYRp z7JAsGFF}i#PE8FuB#3s@#B+9)^T001XL1$wR}7GHxqNODL5p;mLe-G06LYcArD5rb zOD!1@_WTD;m_*%;5l{vjX&GZlV89klFX7FM8%<`xl$xa-V4A zYvG2j9}kWj!qdfe$j0cE{*(aup@sFHOBUv4G2TbaI&a(%{&hmWRW4Sa8wqbza`K*f z&6XcC6s_5*3PnHtB2)`Zj6nO#povYtxZXVNYXsh4V==Gih*kpr-O3T)466sILFWX6 zZl8Jba%a!vx39$xehUGz6=GE;IEsG;_?58lz54NU2;B5peCxNHmnWH#JlLUR;DQ{H z^c;ol8m4gnL1snAjvk(U)6|l=N&2;ZM(X|xO44Cvrjl?waQ}&#%rm50C}E1aBxyKy4L6H<)iU2dM)pK&~l{U ziy9*>)e|o~+FtY`dyr*l$%%-<2h)t1jBZm^Fb~4Tugl=T>6O{!> z#%i%v-)ku8n?AZ%K3L5%{?j2vJ9v^7P*$qg(9ExX77yN@Ep7IygC3M4xhH82XwRcs zdhNmN=h}Z7sfTCkk(Znc6;%!Fg&cPvrA7@Ud|TMbVqgO0zRy79bYJA%36SvCfybK9 zR)=q1;#^EBQf)G(%oV5r0H7KN8z-Ke^tOKsUBc^5>{t2b8&5>M-s+5yQ^GCvKPXdH zhha+vCtq;)5&{T2Lc?8y6WP@ptUd>##S2)LN_5L3Tl0{lLfJ1w08HQ%;FOrsJFb84`3JTj3`%)We%x~aDh6mzOg!__G-sX`L(2@T{?HI zj0g7>)4)-!rgVf&w1thRmFK@N=Nqh7e{u(c4^nNT+bHO@9{lK1ancI7t*UzYeCdJKIeRkNc||0f6uZMr$8m@ zqU`~m_y}~-bkP1xyD~rb_0mi7;MvK8jIeTiQc<#+qntsb@iZU2SM)Ni(?cuad={I?2M?7Q3Q9eI3F41OBCiRHw5Xsb#a zyIho(Ke8CI?7J#|v1xcizO}-LGkfL-sa@|~@W-jNL}1KiWHj1TBxFNqcEqo=(a& z4fb0$92VuLO*Nv4*6N+0|HPc}j;p8$OJYUBrV$DwfQobQZ3vsvLjQoAet(HFaCSn=)n8 z%PWo7(iLWZlY(B0XWI-wvarom@p(~GhtrsEj&;Je&Vp4z)L0YWP`;3`4|47A(J-+G zGE}fqp%#2@AR)^Xo7!w2^lrxi+91&7K1b!co?!B)!IZi5{?=ezD`lWaEq=w+0XUlndydy*7EYz@H$z6NT z>LvV%t>5kNLU1K7r+G#!wQX1L(hy<6p%(VdouNb9?(u`%Sp$=2rTM*Kc7==w*&()* zpn8b7)+BqN?Y#bwTQy|EWf#~DjHre1dWS?NH*MWg_x>V}O^GxVCDch0ZJz<_0Ea(0 zPbhCPGEqCF44S_8Rg_vC+ti80(O8e))Zz`))Zgu`Yq8KO$k3Y1G8ayytii&OA*XZO zgZsLhccxLG>(DyvqJxgFJ9{N{a4RB!lJH|o&zmiJVDKgx8|$tDf9qaWWPc?L^*}^v(Fsetb z%hiB*&EO;{zW{=aYwMC8C!o%FY{8`?`gs3=(m3PW7w~rIwEmqzQ`$z!oR#+eO7>dzYIh}Q$?IOo>a9bnrl6nk?N`Q74KJ+}R{|``>Fy_8V{ZS1MD?|HLQX`d zD?cB(EdAxKh~L?$=ISJFi(dKKJ;tnqxNW1}s}@C-x&e+G{4Qva759VE`Pg{b45q4i zdA49YjW122C~|d)IXZ|>BlelcSdNkL_p9LI(GAwS!e@5D!fFKnVaFGvg()AFwV-iw zBkeU84|mt#r?&Y{Ns>@6t@o>u#9;jAh@_ijGGDw~#Ug~co+*$C;9B7n^xb~s=yL>m z19Gra-2Sw{Y?2!l|DXl+{iYPWQ~y7xAI~?JSgF~8?7&Y25<>|^;+h1EZ~3QHv3K0@ z1Fn3+is1;5N!O2Z`I2?V)mK8nPtu{t2&8-cL=zxuFu!G;JQsasHQ}^|2oenx?EVQE z>@^XL=U-}9UV(<*cqI390nfi|PPcW_^Dfsl=S>?a5XC9(cW%7{_ct!^NpdSg8LY9Q ztB?9iQh&!q+Q=T&FNmDI!s2aHI&?P$_FkvwzcFSA-e6FmGTYv};m?4L>OTsm(mmkb zmhQhlcJ7~f_WDkHFmB4TxooUiAQB6k#e)BEh1&PO0oX&xq)DBZOVuHmuN*5SX^rbS zv9S?lFsvR^c|?zKgU!d-L$cTh^rGo?VqPrR?{io-V`NKkE5|Ry^NkGg%h1}J;A?h& z*9Khcd1|tC7YjKSPAfy)u9LbCMeejOm?$|nV;7AFnN=6Prtsd)0}}@+6YKpOc&}Lw zSx5{QwoLy(I`|u30&%gnW@dRz|J@v+`mcn0mCM^LU)fndu706yt=)pRS$;6m_1A;J zOt`*pgWst$gg4+peQm?V=slB3lgZMu=VSWZd?kEqny1b;o&AjhRl!pIwQDuE%Q|5T zf8nfPjfOd+(2}&N6`9TK(gJ%a724fc6Qx<>h2swe3yZX$i)?ar!+tA&|@bSp3 z^C21rIQGM&#DJ=e{XRZA>AEzm^5An!GlhmTVMJ`g{FHL zC)=}cD)Re$hdn{ef+beAZ1iae&fx_8J>`$>NG$a@~MZb+32i< zd+I|Bh)&U9DAY@4546nxbx!>EFA1vFNK|f#$qU(vhTaLOJ^hF|xTyAcZRy1>HufO3 zO))WaB-+Jd&v|#f&Zi7zc#ZCr9Pgn(d$y$vYh$Y;NxaFQFxV`?<%`s<3OAU6G+!#f zH{Bftf`_Y3?;c$UNIYICmk-NQyqmSTVNWM1S7NmkD5;Bf=uBrV_M%m8HS2Dcckt%e zY`tQad~_i5&S9}~Ji`7-*$ZDe_2F@h0%P0!`^xb9#ksd9xu$}Ph)fAI3)v!QTiQOZQ(=7K*Zqnk3Oc{!mDnSmHo%&EG;op)8^9qplnu*YH9mL6|?x zNDgo>br)-psm%kIo%2k4REVY;0@#-vvr;c<#D#i0(@otIm4jZezgSLZcUZmz9$`-a z78o@nWT!D8)RVmzz`E_#%;WV%RQ43YH*|A$E{6J_VI8S*Qjb^kso|UDLZg96S|%Jk zy#z8{Vh-U9SzIX9sR~g)1dkeBYsfkrhg9ebW*m58qUnnzjtIROQ&APZ#^w85^r6WI*`p5H-Pa-Ma6T7iTy=FHWT#Xw*`xbVdA+WLh2{krbl z#P@640w;^bPzB^?&(S^OEPROuv2)!!4(az8d=v?7;9%`}Ca{AyeD8_@+M_};s|53g z_$PFMT6~-_M+9Tx%5pDyvtUTK6P?+aPMTx(@v6{GEz|56lv`KdR(+?&^k9+>v9yU# z=xzH7`)LBnvjQDSsvUr0u%)$GEC9$3~RZa z5-myw|l@ z>vtp;kpo3?@uBe$_)vBxI8Cqr-yp|uFI(mgKWOoJOiGiV%CXtBROR5^8nx@MLJpEv zg^$iqazwAaU3-|?L@5*HSFW;IN(!o3?CNd2@)erDo}Z|shP|@0`{Gvxc7D}~Nc_@S ze^$N2jOuroWa%tlREO#lv%HhxnP+g1#xBYx%CPsX!KTJ~{}az`Po7%R;6hfXceE^?VQK_5TewS8NjOLD|U z1^8V$=R(tWuBrESPfemIDq1S^xS&GI0@C!VYXO8jI^Rr3dKIIW^OyniQ^}{&h9ia= zT1g9uP|9CEPR5yP1Fn#*)R2`yQD~K$e8=2122$Bs@5lkQ%y-_47aJd_GvcN|uDll4 zp(gNb&bsaKf@ou5oJt>1sUwHjdkL=)`1**=}wcb2xsz= znnq;T4_hW4ig%MwNzNZnW8}MKL$ss0bsSk^cQ#U^_Dsy%UikuAQ5)xiwB5JFMWQ2knTO2 z-i(YhhFe3fSrKG2NWF5LpGk!EGf#eRm|51C8V|0so1Qmr9Cfxb>tQ3{?^WH_=Sa+2 zzbUAtfJa!cVeHQo8^3+yy@Ir0DJ9PnS)f1RV5IYA}_Q6g8=(Z$8qM@ANOSMu#A zqv(G+BYF`wXvxIspW&>vUE#%L{z{)aAKp~CliFqSo@eHRUG7CKqG|L+w@Y1ovjpj2 z$MYy758B=Or2OgWfb)*7@4F+AsV|8YVtGv`!!OwiIi+@Nl_#fGT7ic|)IhRB9xrFO zwvv-LBI87OFTt}9f_eK*>|1lv`}@tM11)#ls%^3Z$i~oXanqV3Jc<4 zcJ~+%Zg!`^1isr357LB76-u1ehBZoUYGmiki8(-1Nu$u_CcVxX>6x@ zcJ3w6-lw}Cm0*( z_cCRsM{&B7&BSj#R+K(g*mPhcEvQZC@Lwm4uwHZ+gVW;~Q$GPWLlj;_o6!5UUh3R8 zJ;FqwTn8Wk<=9&zJ-qh&AF|OsR*V9a+YEfnB@^{n8`_=k3+z@c1k&ozt*WqvqFoXU zSUdyuI@T)Uv|53W%UP|83T92bm*ZauLp5Ina?e>z@_{C(gn_UFo7Si?@ceooEe5@o zd(Tc#4K}>`<9LFuw*S=M$?ijw5mB9MVt3o*T1W|I+vqd5cC;_n(Jqovf3n|lvREc$ zmD)NWHu8k6NvLT=x4B&*+7c^x#Jr6`um12DyfUypIv*D1$&yjt%Q^?_f1P&oAyiQ@-}(-Oh|Y=PHCY%3+bH*dN9Ni#y?ZY(ZDxjld{dArdu?WPv|7FMsp{ zOxoKK<0A;J8Pf*a!^2kJI5@d@3QAAqh^N81>HWfL>3JoWs9D6W|4;8>Ielu*cZmH3?l&$UH_s4B)ps2q`q6zozCJCq&kLpNZX!HxHHTi9U0=RxJ=7_HK_t*F4dzmo68g%doGaztDTs z!)w|b0qml?qffT_yHK$#mLgoISn@5IZ0Pb%Y$fZpwfrFVB5ycxj((9_>2r3|_IY&W z`JJD$-|WUJBcKdXAXvd)joogG@VD_acq9Y! zUjN{7Uv4m7viZCb~iw#Qs$RsZE>&JM?e=(zjA zq$jm$BN3_&AN75RoZ$Ci9c&_AVIViDw~Wq@oe!$=wFAgDfJNorrD!WP2vG2|9*}YN z@gW8WSa{HPCYkf>-AZEyv^y_LhVqA(_JQ@)0TXu=kS*xAV5ps0{9;>dDJ9Ru&4VVP z1)`4_){NpO16tpmdGD}!nm*pO{8rKZlGpMeyxq#DD0@#-4SyDx<|62}(mbCU-r|$P zly&{0yPudQoH8PTFOOxmSA8w5iov{E9T7d+z+#1q6vt59X7~O?5tm6$Bax~nJWY{9 z68OG+!QaHG9D>=2;@l>Z7KUPy`ROusZg@a)EYaEBM+;zG5eVM?1KeIY=Qmc}X-8oO zzDJxgT;^31wi0<`yvLsC{2Ch7&mPvBl)^qV{mgafE=I&i}s`X^r(i&4{B&{zF~I)@ct(Bj_2Ir{W!!=aIkK-Sba{ifUyDqr|cF;CaWg?_zN zZ$Ub*Gf%LY(zgJ$8if$Gze)3Ble>xz+c z%`QBNfbY0bTXotdE*v(U*4h`v{6-6F>VSKWu9e;c{+Q!`5n-39te(&B=6 zLQiVkTwYFTMX=)6hUNcK;}`Q83xsI)?7ZkIy<1d|jzfyV+Q)s0yw<)vGH!Et|89U+ zIS(}Y3H0g(V8DA&>cxiN&kfalrL(yo`j7cL&jmaV-m6qETh;V962RN^%k16{W9bM` z!Q$_&p%^>)K|oZ$nXQ|m+yl_O-#umSaUm~~FJD4;s%X1%+1A!({F}|Mji zCDbhUMyl^M6;sW9CR%7?S6h1_V|To>_)5tdo;I0b+cGwF)!21M8`s|~++F#}vrj%f z3X$R38>>28Q|Mr5Ct0M<+M>LQ67JFPO-BdgdEAM~ebTY7E2RDry~$}q;pM4G^Wpv< zmn>ygGr8Mx+=G!MpBw_3lr1_o+w~ClrD?#m)70^`JlbpOLSiY-=)F6R`9)B^|L)|A z(-urrJdLSe%wkBgM`7Gmh?XOIK4BVVNMEY&6(MM!cq69N1&aJU36SG_G-j}s< zKaA9cp;dhfq?{6Y|70fkjAU1>RUI~6Q3*syzI8D{lbsO6_sYA;`KGj`?D?M@kWu#p zhpyhGu8@tlS~+vH6iZB(doL3<*t?}NiSWbGJWtKtH;{<|f^Y85bsF1(*4DQvNr~|7 zve_NykJA0#J*jByS;YD1q z&C>YI<`V&U^F{x&%e_n1D=cI4_+iaJk}7RUby_BC6zw<)@996TX8tz9s^b}vdj9F` zs-3i;!14oe3x&D%F4cM979rGNlKMj#Mhz!ZV#ymI*K{N8;3( zN(ZEzqekef!bW_jM}Mq-;hYJ`PgLw#?4&-Bq#q{faUlzP&iRwMPQ;|Nbj6Ur{SlfL zNU>_&h6YTbj^Jsp7ppJVf8DG1b%SoE0^P-I+G|i*Bd#)A(tY9j0a#YbI;Z|ch?T-Y zkHJxcvNmjBr0$XE;j(^L{#o#1)l``*0iqMgQMjG6`!aRD(|zn8mR{YtRPm7UD*Lif z3^zsG?y=pcNHZh?l1xyagx*Z6)-67{CQWgm$D~SR4(Q0u z2b$s|vIWNu6~T8`&9$Vx*N4`QUdg1|bd2!kcK!y)9{Q84_dW%h=6F0#^#}%BDyf1qxolMEc8SYHjU~!g=vg1KabfL0T?U0qHNMcO*_3 z8xDfdwY1C7REJKA^+vz&Tc-F*QMh^rHx(>#yA^F6#AZf)44)Q!rT7<+tv7$_be+$#kQ2Jycbtn zaTX;>S#D^_e{pDkQLY+Hh~%a%YBhD=LG|O=L_vb3&;;%77;X)h2a<TxK~m0_M3NaInhp8HJ2O_F0a&dNC;tH#>%074!}8 zx)G#4)-w(~+;H#kW)WDsgpJA{$-Ki0iMUCjjF0niJs#nifhr|W_A#fH|M?(rOVO_7 zA_T;m!1Bxf>{_gq*BADki6N37Qvs?XFDNn4{94f5l4P}h-pujj4A1$5IP9gG-Dfu& zFprlKSAXct%VR9fPb#X*2PR?FxLb5|hDyPRcCcZ_Cw={6Tlm%)TRTC)kC+Xml@w6R z=#$EdQ3|_W!kqm0(XS-ef#(EEFR?_uF?zd%T5#w?&@`DDc6NtAs2w_0VD(!_qHyw6 z@F%&gFEt)U#_Ek2*8rX5v4I<#BD+|j zV~6<5v*$R<$o72_n6M;M_{8|@?OCy(iW1-C>g`V%hRbLU)`Q#yv-X8)v+pq6H2t&a zPf5MK1n})=?}UxrMP-UXA{9>J42wX|Ng;N^KWBB?BfHR-hQn{W`IUD>xt#!d#pLD3 z2QL&bVXSMgn)pMekP1GQHdTLO`{KA`r1QY}uKbdTVe1SOMmdUqsj$hV$B(Gt{WyBv z0D`NuxUDeZv%$_8V zs!;9iv2bk*w=ttkx^&KR%5*vs9IUo>isCMLL4F7Zx6L~gE+D_K@9uJGExz>5 z)=0g6GKhk&Z_HmeGr67Jw0s!4#?tcaGvTYW{IJhsys1L+q}0q{7EgLmX6p0C_@r{B zixvohcJ&JdKe`qWkM0lM`}kUoQ;yeN;{lSeIUD7ov|bZXx9|^f1bFb?-ix5m#b&Hp z@wN6pCh_YrDH`$bM;ET|#d4g!TXUXSfBQ$OuRub+;_ZVf#h6n^o$-%33DG9|gxde5+O8otxdD9dcY^sHS|& z+m%PwDe$*;&nxpkg_6zNiylTWaAp#>k*@q(QbpS({jY>JPbGOd%C4<{~yXKf2{Q(8Ezs6ACF#w#HG>OW+~%GU5ze#&@|5Ffw5JtX!XVa*c-BJUwd#pKSaqt zwJ=jbg)-|L8G*NpYkyuE0I&LEbykUxSHD{mM%#WjNGbzWdlCdM#$5KP#)t+ zWA8FF0MD=@jQ&Es0(?mTcR!39*U`Fo!8IDl__6$Iwc-KCZa`g~jmz1!GiNk^NB6yu z?QkV7WTVLdJKV)Az~nLDVtk1G^;pFuH0h4thuuP#&Klh|x%A|L)qlodrjY^&7 z|HH7pBO<&uGD0=9vTpb~m+$D(fVPkXrdMl}xnWmt%5AcPNb`78^ z_>xFFZF`F|r>ze32x*iBEW|w8pKr3{a5$~uk0`}_iPwU!^^5w{3+lkR-njgLqgEzk z54Z!XEMJ3!vBGu%`h@fK+$3flL%J=Qw|_VIXzFHq1coQ0nMYwyy4X5PH6B2xqyX(1 zf*@~U4Sd+3CE)qI)=xAvz~{@FINZuy%SNZ7#*Ir11o-r*ZsZr*pWZwFG;hhHF_5h( z!nEPa(szhuZpn#8tz?frCYMLHD554h?xY8>qAE)EX7AcCpAO}&q4s8l$%o_Op z4^P%}I49|)+=49MCI@y7?(OR`0G5a@tGnq(uOcZ=j|rBy-yh!y2}+vmGTs5leERwc zvhKhY|5|IwQ~F_EI|<9B@jbmcPM`MD@9~6O$~Q=-I`#}BJtWq9n5Qqp%lDq!P~#MS zhFb&_Ew@NAa9z%mAeSdmYo>ltQxfr}9I9`29=3TO-p^zhD3LK-Sdt1Hp|{PPFyFeB zsy=nus9gTdVTj*zcF8%MrQo0vRYdW&fLwvG0O`L%7CSTxiu}nWj_0q|w({SPcDN~8 zBv8crj|*~HwzC*YO7_g^clj~9(1p~pH$Z#jR|;ABW!JDP9~#X#A5tHfi{_c8-ptj1 z6i9v}{-ecP624^-De7dx=o@NsuB2J{5Yqu`u?M>S<0C5UMo4laRvldkiXAva!N*af zoq4`9cQc2)nkZoS{8Tvao8+aX3+Lv(3diZE$3DhB{0;2T2-I2@&UDNo9_sIm)25o9 z2}6&DmVfVV$^5JoZpb+&%6%xO`?OCcnT(bQF7t8%e^ljhgeuM;ktTBasFZWXV{P28e^104Z+q7nzTOcn0oN+J`yQuc z%fvY_@Id;0&(Q|>WYQl%36q@PS4k8Qotk6~ zrRtA0HYuXiZ+`39^$Gpp(UZa>nQk{pze~b( z?@=Zq|IFFzQ#-l36^li49eG(L0K@k`jth+m(ZFxG)v$ z@x|hh8P9e>>j*VLO>R>p<}e!vQc2(K97t(Yx3xVMp36EHuY_^%W|9}AD#I`ZH1LY@ zxVr{Dd*uwc=3q~IKOe?eM}e0v zh4=*5-Y)O1C4iy>!n6H0W(!&doL~j#fI511t9gOatn&P$=q$kZJCkgbEXeP#&cJ zqnq90hscQR`hmLfu-7VpSvaKu8k)Fwgx0vcbqIcgJj=M^4cv9VNMVgW0DS3}>bzu4 zVXlIXoSlaXz%}v@B4G?faU+guQ_X@?HRc>M$w! z8hXz4a#$kJ(oK6qqDB`n&?zzTIl$ccji$!-I|Lzrd*i-mgTY1=*I~76{nuFK!ZNPL z%2yR%%nfzj52~+SYr*$4n&H05T3M5>H@k6{H< zSMDt6!}@)O)Lxto;x4@B0c6KeLNJ{0OLK4T-y1=Ty)asMP$0TD)kS7c@kp!p@H&6A zDlN8s-0@d*JKw%TFz0xrWKD&!*JH$Pcy+zoNfBv4q8_-x!pWY0Anx+|OS=Vxv!Dqk z0UNq?wyI%S!S{xKRiphlSmY%i6!x)y-zhqpQmymZ_(k+=Z{xG?aXaDVClHwo_GFju zLjp)lr|G7FG||?~>m|Ef{Zt1v&R?T**CC&DKU=sP>dGhv29U{|8Wia(J!jvwVURWF z69!5h1VyX&&atZoNHrv9{^t4nI;(8C_Fz@tmNnQHzd7^IL(Oj6FMUX&zB!U>fOwrF zXw}D~78}C;kSY-ZAQyke*VmT@Q81GSi_!;v7j5T$M?-X?MS9Ng4dzS2tvi{wK|1gO zF3Kd|zNy2Dl{F0x;Uj@f%kX>dqggvQ%5)9llxzsYr)>l4p(aDT_H`fld0qOT#fvRUy#NFk+6A*)%@U|h#+!U|rJkF4(w-$*2Z!YbML*~j4)PBxujzN+ zW{mK`(6@$GMe0OqwoYSj2V0VxLYsH=h6Gy8^thngf-(7V;yqj9>NSdkMxvgdKhDie z_j2cF5fC{vM?v?Izo}(YcVQ7>cG@gu`wtY>+s;h{2a;axoI!h$t~MrVXNU0(jM}e? zFTBQDZT~GgX*baESSHJH4r#varRZwh1!3*lqWz^gx#P5b;S2qe$A!RyzUn{*pO*St z?%%N*TU}JZ|Dy(T5^`UPS_^Wf$4kZMGwc-CO(Q^qNCs|v4zYov9i+2y9NyA#g>!fUaf2klV@ z3Gv$Tw|eCE&DJWUMbC=w+L$z`$Wi`j#`~7uB0UL-ZVISb+kSCHLrxxB%bl~df2TQt(a0P7u6?1 zdtev&59>w#oG6RSDkXgiq4b=WJ6z6G{9OtZu-L~$@bHE6=>5=CgE-hbi9BS1R{g*X zRX7LM9imj;lwpRvJf8ulfS6lzn^Q@aohpa6qzA`+?wRGE-b`g#CV$%KNq5P&SBwtz z7PIet#HCoNebM_H2kY@%E9;q-r^FoYkmbM4FD>jud#N8y0iAu!=(RxvozTog67b449nj>~W)zSpv@Zfljc_H3bF+LFD}~AeJu- zA!bk^FKfR0nIU5iMm8+o4t@w&t>>HM2yDC*?Uv#3p_49&|#EnaQ%PZg*7hwz1Cf5`4y zXUA)KNM_pKe#I`KzHW0lw2#~H#>i(02jSSKGpQ2;u2Rg^}Wmv~e`J+E6&nEb&3d`muaoe?pw@_f== zHdoe`Sm@2TQ!k@|b1;s>T{f&>4UshxVVImK!BwJibL8)qJveoO%uPmxN5aW{Q*2oA z-!*E*vpvl2qoZ6@a-vk>X^?rHyLU|LIDd1q5bH!9jQkeFA;x?eN%EzS7;x<*$B*2# zdheX7En_S>^VM_jB;@2RQi}^Y73!DNpZgN20wlW!9=n%2(nd56N0cxWsn4|j4iS@@wh)wBg+&tvp0AsAi`JimPJ5zt zP8DA%d?z+_pIH-^Q4TcV+X<~DAxbt6&*^js>gWWo7XyPnrqZtJR4v592!miiC7a1R z#5{FdDYi}>4ToKjjgC5G4gxfh(mh|QZG4%1&$7~6Rb63wduNg@WPgOIkDDsn67#w$ z0_mi-T=|a7@nRDJYSA5%Nafd42!rI zY?A^L0j>kQ(4y+P`l$>Kw4`wU#P0I$=R0X@MkILBzB|BH(^W=pLGa-5c)mF~(ElEU z>yl|dnY?}_LU2zybr5hT8%A3{muS!=nVGIHsSf*^U)uR{yMJyZF`I>VXOm!hzF^>B zJ1nby4^5hfBdZ?=+US44Ra^6_550ohyrF|_ZR{rpo`9~CdXRHOjCe5Y18tq72(}mE9<=NO8 ziXMy0ezj?c`||xpv-9q*m}6U+mZ*Hr4nn7!<45=EW{x)-*1P=jq$}ncb&BY^(EB19 zjjcKTtfly;F+dC2yS#Na(?E?MAQ;x>BHrh}nk&Wui}x%1nVA@yi&tH#SrhkZ&sr!* zC@mf>^=QXJ40@EkC>;#PSqP^uMT)d66~fge(lKEh+Hby8dwFEG5b~hp7a?frM#@c| z!>Ui7MU2@e;%d%k0RGHe7VtdzU@EtHUa}Cp$GB;u?NbFs>Q%~a+m4cG7CoU9_f&_g z%j{)bZ&!s8`gzckwKq}vc4dA>Kq!?n*o-A0QI((G6x$qsi$IUOr@+0EIPoD;XD6!! zH{W>dzD?wCS!mm%uG~u%?_S~ykI%DR%S=)Po2fs4?^F?;i( zYGJK4Lwm(XX5eoi3xF4JT)(Q*j7Sf^uGJF0nmO|VAR#Uo44O}WH}(eQf(8XtjTcrn z*cLk)QJ)kx1%BY{v>CL(wb9LBR2Qejq>zfrz0!&9?TuL+-&=|g4k)mVonNT1Td6E1 z<$y@+$lS4f&3(xxrC|ED>?$#9ZuKGKNpKhFo9IK7#_nX+CmzE9TqD__sPd|xO$6;! zi}-{FD>AdxD8Hq2NcpIwRNT#s(|cDqi!GQX~qc3SejZ{+XUH5R{BUkAiyUe-u667xY6o~yI=Cx4wR@^ws@WP8Uf&Ny3X|CmduO<8OSl6T6qc`T|> zD?%Pqr{b$l{40Lm{dgtg;c0&Nr?_~d_Rag}D>O@BtkZ+W&Vv&w-RTxP(nq+NX5159lGmY#MxKZSf)G{+B?_Fb@5| zfL!=+o4^ev<5A$h&mvfJKOQ5x#LzP@r$0Y7BCzF;{hdpKdT-4 zFx8Iii-`n2IpGria>+0?`MmM3y_Jk7W3Ac_-8J|^KV=bn!QYiTqla$=T&ZVnX=z)+ zTZh8K27NwqsUwmtsVPAR3kD1e32F!#bb`wSb7XvxMURYy9HtKJ_WqE4Y4mdArx3h` z2!ve}^-6o${z<;RyT{KvJd?T})Nsu(+n{CU%dyGaz4QONCgEQpY`J%H{&{_5Z&vk!69?5*qny#T+hF7a*o%MD1mCxpqr3zzhZ zt@RSTx%wnltpa+)cXG|O-8`iFIj9VNn#M5+1d)vzFrV}mVjTuKcZem3?&}kmBvp)u zP3gbnf`4&<-s>4@i*SuD8lN&N58B&XTgbER z>%;no3ISVB6%%MYldH*qE1>t>0RiEXI$6C7ced;{7Vu zi+P#b0Jfeb1Lgpc=+Cw`f)B6`Ui-c`f@`FJ+^NKe&a~?yW5 zH0#nrPuqqxi9i9Lg9yLF4hv2#g0wHyxf+TGUo+8^(J~n6YNEY9a-eEtruE5?$uSYo ztc6Aw`}Lrx;A7mOimVP>u>qol$>>Kmd=TLKLU ziOf#3lemE9Qs|x++OR#f0R53fsvn?9;O2cc`YNR2pF^#7qvO*~Q>ZcY)}uY?!fn~I z7w!?3u$I-7ZIiA_mx=P)_Okje{hC$zz$Ox9Tt2JXQl}7T*S1L@`FoYiPg`j;-Q@VT z_*zRSBTSw*R)k-rP%f2>fegV(Viryyzt9M@-#B0A=?i3@)ULFP=R5X=adXW(*>tLu zh@A3Bywz!5qAV=|)OrIX=%9r>J)=@Mv5Lo(`ja@%-3)^mLlP-9Ojh^}cWh zBs#jo6tlRNUZXpY&n)9gYz;)e5M7oi<=k1={O3*2fxA1%kuJxO%BOF6_Q?ki6o#g$5;%QsH#z z6=;h>n@+CSN}kJ1S_Bt z0sYcEg``hFMqcb%g!S8sqkr~C_^GVgl$t#jJ|57V;5*FJAjQ@w6!K4QE2^%wj(sR;!VG2Kfjd3EqOHnaBOWK+95dEdSS2;OAm zIPH8r-U#4m;kdfopR(~{)9aK^@%8XxvwaPuIDvbC&$y^UPA%X4^cFas>MaV2jei+8 zE%9|S&NH7zY(Hvu>XGGSHTAq#&tc!5%EUPliJoH`v}F2y=GAkd>NeM=t`lDK0F5!oDPB$OQ1YjSLJhEuz)387Dq>PO|6*3X*@8b6nbA`!zryn;6a zu06QWZLdERQF!iEw91C5Ts3h?Xmh5`P!*9lSezgOc-S3QsBhgS2T*Qo7O{+9S-myG z`*R#4p2sy2<`(qduVKB(j$tfh=Tf}$MwNIPaVE~jgyW4?r4eNd>d_sZ(0Lm<;P;bJ z<-8L;vFyyW#=|SMr^KALoK7FTCEzye3^my5&v_OWFaCLWwU;!Uz#E7U zp6gC%>0=95W{H@-NBW9Y*8V!~w*aDgw6Qop?~tELiRbvjnE@Xcn|IgKY%p27_PA3A zO*)VA_c(LOFFzYtBv<#sK^vzeLeJQ?*E%f!6wAUk4q(H4cOISOqQCXXNgSD8jeRH_ z;)%017O@Y7%wRzCh9bxe>M-kEpkf5$sv`OF>XcR8bV1!NzA(5|%f(o)%BIy*~~vcPzFebJ`*k>+@NgLOQPZbwn6G z@g>Cg6wG)@`ya-xbuQS*80bcMwbhrq(6N}ZNe+DY5!4d|*UgeM*7ZrEEJ@Qfq}ob# zH8?^spDbG;7K`@2RqQvbU!TyP`R$#s*|Uz@%=n}~?&4$iod?iZn;YW1D6BHD|659x zHbk9V4UZjE@iS~Q+I>rG+vI)eeHHqLY5$41a~Zo~=izP7ivlY8X|rBmqIB4HZv39c zg`zaGnwdjtpxCuqMb2EIOW+~Pi4Uj;yo$1iJvMwRQv=3xP#LV#uhlUtQ6oL>fhG<| zzicuQaPF+Vj}3W0hB9Zf^&Cu_`5MghpY#2DT%Ed4H(><+#_S0C(5Ah_hNU_LHd`pT zJHm7KL8}1*qOW=Tu~*_I^X7-@nEu#%SL#-7D7;(msz@9vRC6|Xe+ne?DRKIyX46>S z<|Q51Wc&HZw+ZM@({q z6wQlbNui6~&WX+T$PHck)JgdC^Tf{O&z;aei0g0T{L#iS&*)UYKgMn?Z=ocx=77b^l! zlwwZ7#=Bq=bT>UZW?%Dumu!Hv{u~}mD#C`Z{(gF*(=VXkuG1dwAYu2TJg02x*kmZ@ z0o>-`3%)=;HX7Bna$s)v3COlzJu5{rGvFznmvNfE^jFlq82x+vc00tzsmg8p_LciF zmZ*s$@fLKH3-^7k+>X62{1TdeD2ux61=smW6!po9$wK9}9UP^DFBr&4``@ z(rvKdT$5WETeNM3h+wfVE5r^)Y1DsOQz&z$JZUYdo~*LhdOw{nAG?zlBA}d3`4IbI zDdd*c7vadoACtN`D-jQjkWp3c;H}g*kAx91J;o=A?wvi5>Wqg51}gI3{ecMN2NUkS zC_mtyY+&NV;$0}U5Zb3@(V`?Li}Xw%XJU6$11YjMpOm$G=)GI#7t9`#Kdrq?baVX_d~6c)JPA7$(Qq~Qgi7Vs zT_Ho=L|YlEP)VGh+#_EopnU6mW(W=Ns5GY+D+z6|?|W)ZW5b`>dX|I2Z=T za&6ND;ec&=N1}$?;?sm=Y@Yr{tlZr*FL+ME0IcEXOZNRkZNFIH^kp9;pMTeW_oAM* zzLbw%J((i$iuuF)ivd6+F!({xG+xXk#o6O2*z5Wx<5yXcGw_6y*E`*(nC^;P^76}< z+RekN>Z#W*)0szaF1)s39DT2`!kThf`RmM=tWM3~zz4eV9oEfc&442kgkM@e*jk3% z`KC)Qn4HP|nR(MzHZ@9bjXXN{SN)9KkHy|T&v%?EtgcECZ_G|H_0P78nZ&Car9-sj z{3JqAcxyIN!W&B^VZLrEZvKf-VaP|@IhyS(v$XI15Hp`SFx2sHf?b6xRoQo94Y4_BL>as6=SJE|@ouW}gt&&gfT)(d; z89lp45XSA%+6z6)F3~)d1_YSl2}2=J%^P0$a}S4v=(;0nBD#TI=vmc?{lcwX4qEB$@W{)X=m^^WBSo~dsLVz$MJkkcy$O@cJ8C|;_=j)o zUyd7o=dV2UCvYSm7TTz6B_l9P1DWFQ-Q`*{PbPi^_&lJCCjHIoeF%DcFS%|#deK&C)A0GBXoV5 zy-=L%;mb$*5~LJX4X4|d*f}AO?}~2qO1Yz*pwW)Lm2t#dMTG9G89Q%3D&pW^(y>bF zE9g%4_uSY!dQ)bp)*<<%%yiXp(y7{3>(Gr8QOvfWZ;17-9t{TD-DB4CpP#SeC%*Nv zdh~{^0xvrOn2FE0KMMc(E^pevp1l{VZ$D>cjl@=yG(dGa557+`H#UMr#rIww&}y@w z9rJHNxPTE!PjN?x!wXsA?bOx{>{96?V~|$%eYvQiy8x^q;hgQPwPpdwI%5{j%$308tHWnD;@V5!02fLf^Re@aVhTaNK z3?PkN+!tMO1*^gG77Zq0pvry43VdPTs&6Wsz_g+eRmHt4I?=T0Q`(9=qo}q}vT#uQ zV<_RUk16&VOJVN49>lGsBdEOKD8gg#_lO@G67 znZ7nWPuQH* zUEj}H?PXG0dba$%xpsv$#pvNjIj817JhG9Bl{AUc6R~j&)LtBNh;v%oelO9Db_q246kA46cWhHeMr4slm zpi7py&>{WbHOEXU#cVQf3^p1_1?+3jYQh!jy1xCE0^EE)CS3GM*nvR$X+Id!EZ&YUQ>3NBb7G6-ejZE_3TMzQgFw zO^;J~>R8^9 zv1=_$CIJH4^+_pHwyH+(6}gFo(|rGE?gTG8ybYbYCx5JNMVC1Ma~1u{7%(v*LP;)BlOQ~T^&+E;uc3D(``MLYE|QVl`4q57i3VW(+;oN3P`#QA~=5Z)RU6r zgCqC9zj0j#kWKGaqO`IfGIp|RK3JkO_gBm&b+ynYM6*d{*_a)_n39fZ^(2diekV|g z#&_yp7Jnw$3ZlZdWg6PwCRp;xKPObK8xNR9s%YqZvK$?~rF(55SEKVAUi24KW=lt6 zoZLop6jxaGaHm8o>R;S#mBv|+0$Lv2mYJj?&wD@X{$w8^>PEflV?2e6OV2>^E)s@t z9t3SNnz+@G|vQD5uarKF4J(0#o2lI@i09ZFzd(1!W`%Cc(gH~GO#Pj}-@$wyD#ev{XV zqoHh;0fEMEjk4{BI;`E~^FdJ!e)u2WIITyH9F_8&&P&PSAkL;uZAb4gV^02PT)WEB zykx04^v*q9-M9KY|B%PE*8$e;Bl-`B=J+Jv1qI(TUX}z*NY$a^JpQR7M166X ze*H8ej0^7m@JJPy*tpw}9I#Pw+EAmMIjS2#D)g+K`(zx&_EGvApaMPnrjmK;XB%W! z1^zH2Neyul6nOPxZMfIq4w_jlgjpxwLnk%H7T*?lO6KMDk>ZvCbMCL8O@CP-jmzR# z$ECqaQB!AF)k?0-!-Y%6_wZJbNZJd9g@r}d{iR>pJtk6tb@yB%2_II}AZ{v3I|0|f zmEU{CTBi>NvVY<7b2&A z86h0*8~pCYcbGKv-#QeGu1S+Nti!?m+hY#*1}|)`A=#M#UHmj)gqv0*D(q>i62>Ym zN;$G3aQFh;#*}c!=3vs{JH5@o^K7m$IWF&@?wfgh!2Iv}gq!-3^bW{eI3SGr8T&(CT^YTlgO>w-zkNWcs`#nFC`jUIkqhkzu(u@ z`BTeb?=*<;rs7CcyHwzW`DL%*hU*uhTK+yEB_EdVjPZ?)VizlMNj29j*@ zAEtIt&)%MdJa`LceKU$tb)*n?YKaevGB22K&c(-6Y*a8vf?mI2ntOW#m4TcYVSb5#4IRUfy@aG7ZTXykzKQ&zumpt;oMoKN^_y2^Lf2PPT3 z`}($`hKH0BD}wZ~>1N|`V92{T6|A2$-?%-bhLiVXdxSiuG}+Q^swZS+lv(c{3e~lMj;R`{2 z0_7`@m{DepJaTJZfMQpQn!O^zFx;FqDF1!-PR$PZ&g!Z9>(^zR$=SFP;POx{d01@8vTnpVwq3jL zOM9jIV|~yt&dd`)5!ksn!v4x8ZWpPM76!54xbJVP-=UOT4Yw26iK3jWYnC zi7uR+Vx2qA#wT{H*A8w^W4z@C&qwPp465c6S>({Mu&GxC+nx(1^)_7#1|NN0{)V0N zJfgRgF~0D4u$fxEDJ@6Tu17Z?_rAB1kb{3F4ZMiamtDZQ0a492F}qtCs^*VdQPnSbPRl`TJWa>JA2c=6VA z&y5~dI4wR&O4D(^qD>>mt>(c(ndUwNH$@677>%$u*M`lDj=P-e@^*7Ybaf}H<$8ge zwyo_dY<_g990dDh--3_7Zo%#>n48I(>g3xy8QT(;z(_Ai2e-)3X0&BwZ~@cqoqhNw z$w5LlYt|%YM0Sky!TQ&YBn)fnsoM@quW<$IpSeA*jO~kC?_C*+UObXH>Gpz}f2gDG zK^hnlMoq`N$hgsC&jl;;Eh}|onXP3(B1*zVKbQa~RGOgUBT%1wgB~54uMihTy)L~c zX-gc*^x5mdksRj!S+G&0M!(WQF1*o$nRN3a*6o_Cr~jx!+`{+#2r648WtUT56&g9N zF)?A}yFqHu3|DLF&w3HG`qH$?-4e?pURT0$W7Oxa5ANPgTC zrLC9Qw~@ExTZ)WFOt&IemfJ;Ftt^`wW=tp1MV!~t;&$A1d+9Z%1GKwE2vsY3f^U%= zwk#Olg0{Gf0ssbMU(U5!e0|Lxb40R9=Q}5hFa`hgVCso;tWKxt;w@wgcCEVOTT1xn z;oMpJv=T_3uyTFRZ_?LX$$-$I_rs}B0_#3y|4xjGH|F|oxycf?-IJyJWLJoW=jNTs ze~57vz1X5ZKggL%|04gX7n=msdn-W46=CM1s~9Q*Vh~&??{kG!&{VdZg5y}C^rNUW z3;o7v7;Wh8NOv^rWq#45u7s=g>O)J(pUMYk>F!AgTUQ25vCMkiLC;=VPhN31>4)!Y zTYY|isqIo$NKehXx^;Vuq+M*N_wH?7KQELOp68_d& zPc&QH=-xk~?RN{>onBI*YKgo#p$g;l|E3$Y^a*l1ab#%XD|l?so1IeFZPgcDHmyRm z0tImF_&ZQGE)KqQeNul2ZekZz8;r8cW56#_rAdxs$UaGpd<>W{MzUPdXL-Ll^ti--w%iTRA)N!+hMfDU-SmyM zgZt>wQ={EeH0Aij51N0t2<)a*X5rRlQU=7#&A~$>w^pr=w)JDn1?2dD2qFX8KUE|C zD`D?jA<>34*tG9(RR9A`+q+f*i8e;w?1yPck<$E51$aC+4tyP1d;W!iFyr5~08=HV zhOc%<%to6eo@A%Ed3{yKD*#6RQ|K`lvoNT9(sZk4cXe{7XJS@5R>O2iuiG?y2Yv-= z=(gmKT;CU@+M9^9r1p}EEH-5t66x~WT_N~0;(Y_RZ(C`HYU}?|r`ggel-=m%o$A=n zJJ;6IE~V)?iG**DUfYM(gd?0agN$eXx^ym6^Em7L7Q0&qc_A<^&mD4t0Qnt>HrnrM zy0pq`{y!!Ar%>S0@23}nC}YPz^(ELMA86)8Q5v2{w3s~Gp2^mNqf4;MrN{>jJu#cv zKi~fM1$ces^xW^T$QMR0%rx7M4YN;elA3eF(nuZYU>S7rI$dyF%73f<7VR*b{Qw(9 zgEIcABHmtEW@z3bszE$G@=8ZzLW7fAZ^)w|FW$RAr~H z>a3TXfu{-lz=L&n$REZEGZ2HZH;w#wzrY4v*Z*UcID!!W0IQ0P?rQi3o(m)ierRJr>_;~DzZ55;Vyu1=SU>f=4dYwGAppTPhA{=vNF0>nq5cQVd7!SrG2k}IQM z8zh@}qp#AH4Zc3vQC}~x*I8hxbMa>LvpeCOO@0p(f#jA8T4|-6r#+^Z0jxpyldL@E zKrari>uXWMTT0FsH4G#U#K4#js?l38)A>KcdmVaR1q5iAKunvIkDTXpEvX4ufA#mC zYEj{U`nbr?ayE7VjL)Kr=xz*ZU@UO8=KoRkb2OU-ujJCaSp5B1S~ zC4FGgru9Y&Un46(YwLtE0vqo7Qjgx<}!3hiHOCBR5$3-uF@M)qO+HC@16pn82~8 zhqqXn0L>FCaK)9jght2zKJ@>-XZ;h?JXLM>NR+HA{e0WvCu4ddaoB;B>9B8TU-9y*P;n^Y zo4sp zKjmH!ac!loZ?_-P?Vk7F&@K}{>q8tv1nyuo;6qYlcgOUJ$QuoK$%9kUM-uA8^By1U zbfZl7czObbwj9tiySHw*I~EdH|9{V-F889GU#Op%1^pQ*hWNo>PMnl!`!18^=HrxY zLpke#ctncB#<~?@x}!UaHsk*-qw_*-zxkX^tvg}=Yh2BQN0R9r>5nXvuELxmiA*~7 z#Qq;C8qu-H{hIGht;30>adeT~d3_~3sISQp+w{g86vdX|*Xf*d2xgenofNCF< z#!Q~z2E3ijMtAuT2mB=AL-4cMbJ@f=V~lPNlwZ0lnKnG)DuuWjwR-u`#1;goNR_^t zHcWFU-v#7UQ^$P&)8hPpmPZh-d~EZy{&|0kLnIKBfh}%0qPGEY9`Px#l+vF+-1s!6 z!p+)_;LpL_E`K_SB>d;du(o!alkqQ1?r-0FF?Dv<{WZW@w(m*VN>~9oeCfuHM*YHh z=8M|?7jzANS@iC1A7G-r2sjr}0Ma=awIO~{jg@o$JBS?7GL%VKch0&y9=!*_P?yg|Ll%_9JCOFEVK{L{PlqeaW?}GWrr3s z)C9WERZ(2WRVDb#)Z){_f_QNIMsHB;F1ePtG4b`l`k@jz@LlGm2tPv3mXlGr!sve* zt^dj6=@Ot|#YydPjwqZzfOe+L6IpFw6YMTu`loxNWvdf$UXY!8OSt-iTidjkTO@sy zfSZpqc6;)3s%idL|D@`+|JrH)v%3qRVB59yW-}}W`Ew&8Q?xmv9khDDo7|t7kBfXi zWe(Ci&NqxFnk=+mPIh~}HAO`1mWq!)TNe6%ksv%U)OHdNbg|yuOjsgv3m}uOXCG5c zdDO>(S2EVuXVSAC(RR_ z*8UywGS2xNM9igY zayt`1jVTsPQ=aRY1qhJPakYUm_HXy{e|}}maxWg74Bq#@C@oT6;Hp)(l?kY(Sbx3# zm=T$Gk#?Mau)jHJ%pU#i!4<^N-b_*4CdakO`nSEOhxKROXQ_O)9g;VLKbZ50*bz_{ zwqjK=m70+)md=e}dKZhEjWwyCe(pvfDJO-TV+^+@78$!{L=LHSz+Q;Bgsm5_q7pY+ zL3>;YSQgkRZfk~&7W@E5Ajr3335`BR=g^8s<|}Z+=)lTH#RrCO(rTk(2zH9}3|J;6 zBQxI`zw!jrFmp4_BBlgrQ4i?(fP+TV1Y7&sFz|S)|T%uk_hO=kckGmFglx&W9x=s}*EY0E%f~ z3aL1O;hZ6JX@wom*EcPv)Eu>52Z`OV#V<^mbkr>MTg8PaXCJpg6i|=qO-yTu9}Jzq zdO854)!%vQ?o)4P0RlX{X2CYWV}IsOLwX-9baudx;(0c3TldQFnELRUR(`JQ<}su* zF*|IhGW5-82P6MHb19k^fBupzBr@g$)y&K(^mR})@yC{1G&I{}M-9{q-k=$PH|!}K zc<|;^zeneM-&yC2ORLLouA5MQEm3#enMwZy0|%v+fFc2MkTu_g+qz!S7y=Z!s60QD z6!)m^^(yNyx@SEj-gMqMv{HP6Q6Th#40*?{FOVjTosD8-6j+D<4cl}ts74*P{bHwk zq$XnVb!`ksC5TU$j%%5XTcW_bsg>(&p6BFcH%q zGd&{JnM1sWC}YOof}ROiDp0pa34baJ9PJ@tB~y04O*x|Zw#b8#3>XI9d?}mG&4<7Z z93yoq#j`#E(llkMu<+GOmI^Uxgo>_`?6t0RGmJc!1hZB2t6MK44yekYjdGctKGB=* z)_UBT;*`Rs^~GPx!8}w4@1Z zset_iSQBd+I=p$!v2G@#9rh{qK5pS}RCb7iltS?Ew9OkoP}t(p(dkC5MW=GHl%>vp zB;>9bH7q4~)Xu!Hs47@>wT;XeRN&7DS=95oP${($c69--)~wOJKi-)>?yU*VK=r0} zG0ZL9T(wS18@dmZJM6OfaO43XBJjMQFqTiw?ftN{UcewlR+#YU!zeqwt@WQISnRo_ z7CWr(uMw)f64Y1TX#Sx<9V+pPc4RsvJKkpBSJjNz;Lgt2{*c7~W?x>$v^89KC7!Zy zRtVpNQ!ERfwuS}^D3=SFI9TM~pE-#9tOwjS;w@JJS+^IKjyut(o=H)>$z`BkO{pm> zjgf(F^;-_GuZc+#C%2S~Vz}-tnX;(nk3bMSbnrQUsXrqP;u73N0Xs3hI(#|M#cn%C ztutoQ1cKI@r*h&g1(TsyB?96S0K}2g7(K&8*+r=Gd zAEVX=7Cr>2%I}1#Pa4rD4@f}lmfYG>g3;nf;4{MBQ_>o8w#3dwcH!`)*#SuGP6Re) zt`hq-dMIwwpSaEGQFayt;|03UZhsdG-6=Rpxcc|Pj;Qbv_1i`9GmoxXe9-YYuhJ&g zLjJ`^*IgvBNVB1!YiRaY-x4ExFX|_DV{4kGyB^y4c#zU zW*MUstf%iqr>bDJ^l#dIrUttIrl-&4qU_2!Hw2xRsb|E0-Z=_Vck}H=6^OoDJY!>9 zv#ls0pkY_`4?$Tbnm)FaCR**jbICw$zc=k>{Z#@UTVh|c4P&-YwlhelEOmkkf{R?6 zTiPrY8v|~Djw*^ZRiC=!&bVgq$r3Wb@(uPjTjRcr?ZO5!Y@COry3*6s?Q1-bSbS16 zM*3Qtcq##)>r2h%+0X^ljb{>3iz7yul?3mqe*3xjY=w-;sKOt?=~#L4Rn3Y|XuAW? zZH#91^M+7cUGBd^zANkQ1;fhSGucRY_Fd_avx(Wg>4##y?9)7NNN7T-H{WPVO39np zDY|bt$r(Fd)~VJ@_mx=4KWcG%u84ZQ`D>Y{^)U-1sMImKonWNs4@D`8vDkn<~kg$@vJ# zH1-slzEP;Ds12o35~Ar<4-6`%bvn&O*b?l!f6t+0TG&y$r2J` zvWmgvN7i}>cXp>k*9$p@ie6-YsSV2>);MG_^=(*}(H{>GF}0h$xwiPH-5Z2jhY_LJ zPQUF5`L{Qvbdvx85TZ}AyOX)j%)1nRjTP9>1!djvYZyPM&ky?2QK119I^8(hQnN|)UBHn+L6R+ zJ(z*T{AL?l(v(LaaAYSsZn}X6=4G|d;+A7WmZBLg89pc;BbK}AAI!YkNEF*0cB{7f zSHk0@(t?=9(v8XQp;o$^DY(UM(}auxahIw=fU%=)tJ%>)wyLSU#3ahC%G>S7A&jE$ z`$R#|S*CyE2@qiOnU#OT#6>3>Afy*4o z4m~d%CFn~~--5OP284V*hOmnnu!>r?O60n>h^l0fUm};PN?ylsOTcn(83a^uvO=?} znlQ@rE#jr9`q&aGg`zPraodXAU#W^w~lZ3896*SbuoEk=zQIH2uP zW0Sb86Jdrr+%Zf>JypW5HiCstu7Apem;+sO`ZKVEA%XHIZGRMU-tAXzqCPHhF%10M zR_8TUC>H3b^%#a%-+8_H4eRE6&4o3p_?d5Ra&QFL=x4cxh!;W!c7lN;C``mT(tMhv zrNX8eOPl9;+3Amp=jT@hM;-*U&PNdAE#3Xf4#4C`W- zXd-EvwEpc6sE|60JBnRy?3^#INF1}nikR6Uf-4LZ$-jvp*Htam3>cq;imn~7-JBn9 z_bIZEh^)1bA=K31?~)NO(Gm+Am)5xHs(Z|Wk|86RbV&)v*!!Yt(qoKYcYhmQsj50t!j84j99s%D8o>l@1x(56Wiwq?cSqJ{1ImQ%T zc8%4tm`{T9*3thu3)vwzYLUCo3Ee`1qaW!aZB3x6BW7K7gX@23^v&NC^wT98KP+ba zi{p!`n(}2^ME9ueQ|(anr)PH`2Qwdq1b0#Ig|cMqv!X%^WNa0ai=x7k4+97h6~$?G zVV=$Q26YQh@M7vQA&F@ZqJBj+*;<8%4y-};*LUr@n-K#5&Hb(qEx0k*wQNo$xeB1amhxP1 zP;@cbsT7t~)QW7jaif!_%8U(36_$nql<#)zh)5b0z!64L@N%fv7urX`vSyt4t|dS) z^YN>wMEH-(L5QWzV|JnaAF;)mcu;&~vaWL$J-4Z6bf-XfezZN*Yv-IPEOT;hck`GZ z`(1-OSmWHzah~sXIq-~HMxyg%I&8l9Aj^8wOlQfSLE7RdVg=5lb%ZhbRDpfizzhjI z?$uqK6jjG;y)E)aVRqWAHUJjzjHng1kU-pgX%a2ilxdmA;a+1hfnU0YtT9m9d&k1? zn?Pxe;G3bwWLR{GndRU6D<}(nNs-b9PIkp7Gx$+yF(~_mUESyaW5DuVz!J-@$rpXY zX_H)#Q${`^G45+u$C&e0&Fb%TV5y;sQq|cv@Gf#^HlPm~Q|yE9U%NGFA^G27GG-Hxt-h-SCM=Xa892$pPLmv>YpKtYv&`Kvzao zCrG*RhTW;;+D8C%x&iDZ8x(oxpG}4>)CG?_~l146Z@q=S)3c2g993LdFQO<|L?Z{Smr#-}?rc2_E zKw>V$D1-WeZlGOVyQq=&pZBZK=Hiy|z(PYpmZpnNRiwk*{6g#PfkX06n?-36tDYB? zvb<&8zdhLxUQTmc)rL;s@TT(T$d{p%t{&{(e7&K5$a~y~lmve+d+fWU04{tCuT)0; zM`BzxMH%UND)SDA$Ea2o{1crSrQ#8FUy%Rx9a?iG;eZuv#@EfDPxI$}(q>Kz$p?D6 z?{WrHYoLTVM=)H7l<={%OcuqvsbP}v?A>InO7Po^x zP+Y@|T5I_xnk90TlVs(gGpW)DEk6{{Ta%Z50XoJgVOBofc5;hl__zeN|CZA?5*fI$0GH4i0ej@5>9D5pT=!1_$DFuXxbn%I}oT9!W-+9u)Sn&ib+Y@ zzIUo|T0tEVIy7_3QpwGo{PY2V;VYD#>~yUk1TO+%(ScMdM$(0-DO+H;-N)DiWXIK(Hm~BgyFs_Y+%H z=;T9%K9Lk{?%5myIw=;@6#WX(eTN;C^Cw<87wa1mMq@% z3k!3MYH1cPWIE>F|0}L^mGsUR(%Vy{%3~D=B zMUh3A#))K?&xpbma*)Q29TE1qwrn=Wq`FPXgnIgG_p#brhBaRiLezkM0vpP{Q-M$(3{}4KHHpeZiQIGPB+dKsmwqou!kG?e<3R+=r>% z2dqwAlCseqR*Q{hFsCg@K`)EN76X{X%t86pIFuVS)h*;nNuLyk^!{mwn5ana%r-{C zot==a?t6yXr<;BMGrO;yJFm@Rb!)o6d&#!dg(gVphZ*)zxbI=`wXtUApmz3?qAr_WcR+WFxvD{|urJQ) z2&vk(V+SLERH8HllcdaAh`$|W+3lE=jks^0f7^)A2C-EnI=&wcUT&-OTk=j?1Q`GC{^Zu;yP>opfG7gz;lk$`3Y3f-jWUz6Bt zJDY>k;3j!-X-DA|jDj*pK+=4VTxs|P1pz>?Od&QnprBV<(f3P9_!PBg! zXa^nbCzwxekpS!63t#+{BG#qu98wv-HFG+#hJjW_>|v@)8n;XP($Ge{nSQ}}=v78C zhWY%+S}+M_*=i&o%NSm?mfBk^9g#BCsSvFzP}luFD-`n&mW{*g1b;c!)UMii-nA7lN+_cj%Jh>*Ry&P$nSMa{%WXTva8ebS8ETJ$= z>RH3gVtCX-Bf0ljtgjFb(XM7fdaoe9nrn_28O`}KBEAD0op(Y&;ZTRYfWl580vJqCP8> zP2ERW_!AAIk5>jK3n6)o_xRpN9#1lT!nAa}Q%vuWUKttSibW*d)|d;Z#I$#|_iJu0 zgOwU)ZsTs}@>$G$Aj}o6m3N!^tVz~5w*PTr6O^*#nI<4SV3#9O9aa~Gr1sUd8yM;5 z<#mkF%1xmXisE}Qz&$Ripae-l7_qLLX6HkJ(qY#P1TEXAArbz-i*JX zNG`SjsJrbb%hDq|CFLuaa1q65X`-iX=G>zySsbr9KTSf0 zpeI88+ig91Bo*wa&)AIYz#SG@QFo4E50yLgE6X_z0t2`-nbey`ZsZb^tRZ*KXt_8< z3${F(DpI$wxQ($fMvL~&Z05MIri9;UauTtnDL%%_t$uVDCTgYAoU91 z$7mZc<8K>Vf*V^?Y%4yZ^Ui0jeu9`(7KTr!{ks+btY3k2L3E0E*o4hzf4UpRE~$z_>8m9+jm)u^X0 zd>D03Bx+%QkrF3EbQi{J$YT6AelS|i2X`UJosvUlZ@9&5AN9}V9Eq{G-LI2t?9@uP?YCoE_69Cbo&j8CQ@DPvQM zS;pg_PX@P_G_PIr-z;Pzv8R33~7dVi6WgZyEAFLC9p{TFba-+G2H*C6b zSg$)lS6_=!03_&Ajy8~*CqG22l4YUeuheMCKttcNJkot7KwZE`*X(y|33yZ5u z`Rx0;k2b_KjMZjZio%0^3)3QW8UdMZ^y;7~gIXN^gC5a;g;XF~-#-uG`M9yFW1Ni;tD$El1VzuVdzV)m_opsP z90Rw#&v^Y{<8lQwm~n z>uv#@QWN>}#}xG4-%U54mXoKL?vw3c(shid?cR8({!ISh^G#-`x%DNOoZ^gX`!W!- zaumV)h!zfr@-jmleamB7!pb0Y%%wKFxF^$c`SU$oNz#Sv?Y9V0hM^X^cD(I%MVFZpTpqpP{i#5?(C!?`RHLu4EMvsQ^}?$AefEn)(0S->OJ!u>BP`u z5r9_*VADO?l9e5@Ybkc^p!qQ*?R~8%h&c75@(-v?ICE_YEVSochv?y?_^hW3sHXvx zXpc1~zXQ?ohY;xk%g-arw2+#Idn{IbM^IGq>6H5S znJEdEq{R{1rCW%WLw4Z=6G}{l4A=LGfE)osnu3)9HO%E?HY%_o+5pM7+laBHG_RTLdbvzfm8A*3wyccSE_`=fgw3;GfF%eBc zjHXJ-(x=5O{Ys7HKwO4w5{B8KB)hN_SNWH!-#}OHom1&VTM>C=BUn2NJ?r z!_-l!#&$8duo#!us21Fu4mZs@uI~v>^9iF=cT5`X5O7B%Tnl`3*2{RE3DQ5D+80{Q zfjPPy>OT;}YGr%`i@sSh2#%s!eif+0MjT0HCqygX(qWA}Yv2q{YjUUFUzvJUHiTe2 zkKm^~g|2@Rc#(zkzSC;37JxJ8JH1l+B!8I2?v-Z5q zmz0VOWl%y1%l6)M)@vRvPo&6#B7FE@?BeP`Gm*c%Z1IrX3|HHwT%<2u-f|7 zRg1d}u|JwF#W0iH2n|C8kV?Z{gT8vh`TD-bhLu@OLCYT$IT`gnO}e!$dnzb)d>l5~ zNZ*9b>(AGV&`*>j3Wi;OM%4~y64!^%Uu9|Cq%2|gSLRDBjn{9KY+%;-C6BQ4gc4D= zg_e?K*8`CfALufz^+?D|giJ7jxV0H1*2qwuH(1+o*lyU4bvW1uO)7@JRd}r3E=Y~; zEPQTj(?mYc6JUx$sCR*K6)P`uR$ldi`2C5n2*O}4OuB((E&>UOiktsRH*x=tR!BZX z$oYG`FaZl?wI`H&ZvM1$F{l5xP1|1&mR`Ub)=%1bm{B4K8?GqZ_1U!) z{I)cKW$qEsOz{u_)2gDMb;Nyd`yP4FB2Cmm(2iHAj?-6vZlCnHggnPFL-CoiW(~4N zJHOCle?}9B|5fR62q;m0=Tg!&%+AFvAsh=mZ*m|Uht4byoh?cBbP;W}}K8L01(Td3wV$?Uov81?@cPQ7Prb(z@Ya=1Y3z^uF$BJi~XBkyO7!&n>j*6~J&d%K-_ zvqMEDdoryQ(B?vi*rcX(aa$;R=eZs7{ES_TYr%h{ARgCAPzuy-0}( z=hVOcK{#d6PyAmrU3omyfBdh+Dpw^}jwMBX-8ttNl6*_3RLC)Bl)1$)b1ar~j&BWP zuA;+z-*b-TrZB?XlWTLeG5q%XJs!XRK7W25kI(z_c)woHbG6?Sye#MbT%??((!hqS zJyNmeBm>3^5`ZcCCgO@P&G_J6>)9I3UaLX1&0!7giN{`vWjB&$bGztcTG14=MYHfvGukiEG0xoQV0c+s2n6tSBi>At&Ed@CaO#JUz)Lr$~CBOMf#s zHUz!rdR(9o^mC2oQj(im-+ypf{)oEEB&atBb?zA|1kjX>zIypDV7at36>bS$3G<$7 zQ&%H-e>G$HbqyyfLFROs(Stp5`^f#b`+oarjwp0w?@A|mcw74gY+liBcc8Gru|T;kUnxv{~I#EF=mh;QJ?ua=}Esd_^^JEH*~nih%^+U z$)a*0v-xv?0JpW?7xr^WfB8${zi`L&sxe>r4%E@}^B^@lH$(JRj(a9aPdwUd=&#{evFX_~|zt4WqW*v7$5dOU!8v2*HyXrsk(UcziO62YFeCBN6XJflRBMVRxs7Wd0 znA1*!G8}Od{#RDvhB?jH>rPpE6G44)YNA7pBVnl~gQu%>OO_tJCHt#h;YbU(w;mGh zcl}U`NsQpJQJp4kY6~@axlK0hSZ|Rh*hLZRo#XxL=&*LaZlM?ueYqTZ-b`KvJP2E^ zxYVu1nHexwF79Xi^C7ichadLKwqvkI)YunFA=PQNmSkGuu3;`Zx{!db$djOw; zp^2j+_lFy@IW^3;5K*17r5>q)dkd;^z7T zbxqMqPIeIIk{<_g>{ajG3Fos{>{PY+EP(qaX;hPousl~Xb22lY8?4D1TDl4P zAuOaeS28&Aw6^7TipU@aKmP%#W=9Ua(qp=-Dp4qc>Q5%S7yy`R0>nczSMAY%A4f*+ z%n?i07Dnhrvf#x z!2j7SS71b_RBDa)HRYptq-zsFtC2*$&jH#i%yr@l zni<_JtLOz!#cp;}xos zEZh_lZ7i>f4L)X7Du5ihyi__}oE+mRk5Ct0(!*Gt3qob^^YWHTksOS<2)1l4Vw5_A zcu5E0?3=rlJ^=sNTxVeRIz^ukq2zYC@yGL0HRo&UAUKz^R#*G)Pi@k41MSq+YYRw= zfYe?}ufj6iB_e}c%+$YENS9rp!!nYFc35C8Nc2HM)7C6yq zs&T`ash-;z;DT_oB<4js>p-)YYa8D6A))^0j(y z^_BlZ%g^43ZHL{{<*q(nI$dE5xf73sXX_-PSho6IzwDEC@!yRzN4QJ)rAFV+qTbct zAx=YAcINp@=zdQ|GM^SL-1c1L_m~&Qces@r7MNRzs-r*?m@TnzTEwzFSWe#4yF-K< z0&qeW0ZU+ywcra=jzy|C?%T6oiIG#8XLy?GT1T5uA2m7#yR=|xo0ESO2B2#FPu!kO zm1@M7`ACk=c3@WRrz6p-HP!J9+%eRkW(*IO9UZ@FG|eTp$b zjdO!CK05g=c^mAa05Wxk51w1$fsT;e#o4t|Q zL`~3lLT6VSIc;W{Omw%Tn2;WCzSCN}hrlRR6!vSjkWe-_eweRP=uizVP~9xT>(GQI#M~61&lEs9p!l zWkW$|S?ihjwg^r;2?f~T*ax1Yi5NR?+Tjl5*j-e8_gj7z zmVg%iC0J|k=#yQuNiK90R2?$V0pc-?X}LBa|2TExpQ zh7HQ-vJ%jwW?0wE{c$p@ue4crjOJWJSVoS?Gi{I7RDS%WGblGLG*mt}ib`$KpQO+} z0T{a;lPB#M;9NiBD#SJ2H@*3vs~7NX*nyeQi>SE7w}2KxCyE`=;ww4z!)uI5xs7<_ z%$dl=B6?04b~S}`LDXt+UbnxCX8wzuIc=+B*?R7Uzum%{bXfZAr>?SL;lST z9s8v$t|8kAN zZn=G@Eg8|jW}){mu+&N)WdhxLG)5QfiH1V3kLDvys0(OtY!a^r#&Be%R@_GE8=ybWNO^vj`Ng^%BUf?HD&nxoR7F1^JQ~3J|+P&IR;x^@_mbSdt zrQ_2hQ5sKDF`@Avlzns`c40Q@0=)93WuF=MfVwvdw=T5_l`fW@+#RhGmUbP19SVda zlx~-78-{$Lzf!JnUl?4xQK`0YAEMp75K-Wz&xrI=AYV~*`-c+YrA?2oHg(-1FWuRt z=J3G=nL^srHW}RzeRGH!M+bOYwZ5`a4Wrm2iHhY5?H^q~jj zyY$=6|C+g$xM`tt`LdP3Ore&$QXBsIoRB-M-^ZkC(^!1hCGlW-8K8yq2r_$VbrHf> z_BnuxD=yu>eE>^jRczkef-Z}c@0o^0wYr?fe4}_zE|}J89=}tL2U@rVehr6Ajn+An zwN3OPKKdW~O}Fun^`1nHh?;mjb$tvlI@5N#w(ad

Xa0B=mQGZPfp^_++Hlye!CUkDUtimt&Ba1lS@=g{%U4_uBO#Zfd{{qP9LcO z7S%OG@onR%#ICs0RTLlN=Z8kR`!re{ctg!_^1_wsDOpM~YT=w-1NO`B&iqC&8vxVs zjii^8vzD)Y=*!C5j23Jj-gqTqwFj(n4pSqxfo>qwta&na8L37og*1u6F1@y#S%>t1 zh0Yi`wfKmb%=pO8NFgYUzSlO{p9y13tXS9+I7|Tfa2af!!=J}VGbVpvIeRh-AH*i3%~9$Ny%&(!7D(Jr;v#hUSiZ=hQU(qlLcLWO9F- z_8=8}?D;FEkeP5YVr?Sjzw=x~N#u&!ownq`3Fi)igGdqaPTDV)6V4 zSXj`@;Gx$i8yUgiY$jC;PUUBLntt18As#ktRnSJlsP)d}zh=i@Fs;E0-D=a0_pu-X zT%kM0<7iB9>C#S>+D@I#ODHW!X>8d5_i=pq9jEC4N@k3fTDfceOkmCT9sBm3nXp-! z-FlGozU*HuoB2xl{14-MwAY9?U?tu(bfb-pEY)M}!A@OoQ9?uzA5Qg)Mp@u%U8k8E z2Lp3iOH9t_t0~cf!O~*t*YNVM)dB`n-5lxQKMh#W@B%vhD@zI9-7dKO&!jw{M5G~YH%P-kz>fooBw#^+X?5eh%& z^^L^I?G(|XB!xlBbXA%t2F1%w5#z@E{UftQeeJN`zLRlfY(iz;J#XY9fOdo~&6%s6 zlkKJEMcK@^-=K5I3*uub!#kU#nSltK3hR&oMf%I{!^3_kbLdY8g4C2-ZS5O$XV`ok zZZ8#F{T5UGUo&#DQS~uHCw)R$=hs-ZTT(WTIu|wg0;ZYf!6b3odQ-a0`ptR>%h-9& zMEafC+!#Fm&$!UsYjoTF1Nc29)Nq>}vF*K2v^zdCbaYrlcm?i+4aKxb_I4MlgwXgk zX*vk^?B49A9cjO~-vwsL2D^a*0TUZfHf03*^7rfa+x>sv3{wQ>!Cp+nX9djtaWx3w z8^s+e+VjNj>r4S$M2T9|l`|&c90}jOHY48Pwl_*%M0@myMscWGoT(!5;rxK!6zts-~MU}s8fr>V`5fe*;HgtJk(xQ3=jx$bU^U=KGmCePh zHt0C+Kx-3V3x6okJ+nG#@Zg8l2e1lh7{J}MdR_roX5>$p$o$Qkge!$X#G?HOcAY9) z`)=c-np(UQeDR_gp^p(<(}&w-VdqmZKq_JXh=s)uH6j!*2rmk3&$&0zwHbf90XCXe z#ytj$kJm5tc>ZpCFQ=Uqb)e{$(1R{cxU%ql|B^88+8V1f;R;@Sr5jukgv$2P_}e|m zj0B{UiV;;L?ybpq1I?;e(cObZEAhpyjQd3uQSEFj1ENG|8$OY%}&lHvGp`VU4S=A-x6Ch%yIM5-cuI2gMq^e z(MwiMt8dUzt}y1M^%8cCj6q`DRPLE4icYb?uDN6iG`+r79I=W1pqY*kps~GIWrY7-x|HX-vx^ z+xPH2jN-OM{kPsf(DNvH%2x(=%VHk3{&JQtM=<%yokJdyP%_C7OkFh>w|6h^BDj5v zdwom-i84t1#mldESyRsHQ^P!HaQ84&GC|8($NsQbm&huZ$laEc=RF;D1rMMQKpiu@kK`8drK?%8f&e$rBBvm_<*Pd=SrR(^ta$-i!Xr;rDe< zP9)BqK@5Dj_L#A`+nrsex#?dBbms{&F||OJv!cjbyOH2sZ7IPEwh!h*nb^jC?F{b2`wU)LZkX*0--%g&08Blyq_%UoQ#r#Z_ zuzzm2Fn+ClhSaVGhcR)hgYGnJ27R=tw+B`E8f_n1Z}Pc)^#59bq+|Amv&&R&IV#bkqiX=xCV`B)2EP36(>#?gUhg$?xaWmi(D(|d zem3RSVhTgM7I91&7VFy6AEf(_0W+n8>tTerjiGK$encz4Z?7iJ1D^SG< zJJ92TZc@_F#@lS6Plv@^&swU>X8j7i5rCwFO1 zYtVzfAGg;BXXBK7W=boIjp=v|@73>b(bj8zTvnA6*J6w{=7n_1Q9!$g*d|BYrcN^^ z=&U7kZsMTBaSLCqK6`~1%!AD@5{%=mnID(GxHxKGWLzPceZPY_Sa=lsZ_)enK8`Ia zVU}C(_-v~VdjW0)L7gosvrB2qeh-`9pLziO*_lS)-{R{m+6%9%y7W$c=l2%rxAB<$ zYN{zY^uNy-$iUDvvm{HFtZh+?u$S>MU8ZzRMLc&`Sd)M+iFkvHhI+fn-ZTfT>Upwg zOTStg#q(Rf7k7ueH?hFB&B7ye(ufmftQ=NVtOU<=;5@v?7;MPesF29KU_BHiw;u7* zFF#5;%Z4TUtWXrDpkAmM^v>}K9f|`qm|yYXmqasG=O4GW-OyQe4|^y7l3C%Z;lF<= zGLtaoBD5CPFw)xq^jbI#p3dN;0?Y)NZJ!L_%Wv14q9a922F2={?2Uk>9A@T+m)^%^_ z>$eN>{OJSdyHcC|%&3Xo53W6(HsG96MP;IjKl zD{~UCzi9k%hGw8(-BY-uy%CRUan2ZA=rI77z3~Z8++rgiT_e!RMkUZs#heCY`qygj zmbB2;N_B`I&QrE0X=??!tB9LiZIyzx8}f;5IqxRoT{qh1U(f1`V|-H_N-9O{pl{>^ zfmM;*@}U!>T`x7(oxtbCA1!RQa~yT*FevN;kie0dgA3WzU*MOda>F3JcdXCZ|m>2gHPTf4AeQ2Sem=RCtSUntRqU zULuqz2b7YAYwnhq*?FFP(Gg$w>VL2d{@LDzoJ}TL`X`!lFk+4;ZGX;Y_uN5i*E8sPB?8H)ugMaJ6$hc^BRG8w}KyvJ`g zwohp+e4;06biBW$c3aK$B`(1gl*;40g&Js?rAEnH(qE;ql*!~Q+f^5|$pH9$rHojr zt*(aaivN4y@#3{W3{Uai5X$-y*=6ttu+RPI%czF4nJTIEm2tpaWwkoujdv`#OI#2x zc%iD_yRG4|{d#|l@*~|>QD)hTubaN*=g&s*&^AK2L#p@Fj_!y%)3~o}*)mqH9ay_l zebo5T(5%_3KqsVhBet>lNv2bbR#1|-cLjchfbDZk(dn0wy}3jO_`L#+1tks%;lhaM(aL*ciyQWGg{RmA zO=4mtZY*)R4!h*YP8|IJu{>5GfgL@L6_B!q+If~#{$>=T%e_V!qI}6e?s3rZ03H+c(GM!rqL|%5|*wBb0gLB=|CD zD}ofOq#afKA3`d0K8`tTAwxVZC@ZXe7*$(d{;#oJa2HFQ0V0649Xka*W559_Lw0@Yy*kH$iWDz2^@|Q~!@Ft4Dkw zE}{kRahh-Ij^bay@Q_GS0~3F;c^ihWVej?J`#Y*vR6fT(L8RKjVyD-WOdH?8{M~%> z0aop6Iq%TUHQHVt6?i!o_tL=h%Pa!+M@Q2 zOuh}(v+$r5y!aH zSpk8dv9qz-?XJC&^N*PCLF$ifm*>YF|F!*{0q7e{aNSAk@>gOP*I_PJ%F}aiCenBa z+=1mY3fg&%Kc(^b%t=D4O{;=f#gDyZuaL;)Yx|1XX~>L2=vYQ8U8Ar+klT%@1LqxwO5uRIT$|!p;c}E{+)C*4He84wlrL+HN~{qoIU)V^RMyjs(-D*4j;X zxes8PlOQIFxTyF&Ab3dq#5>%fiA2cwZ}a%77^AgMO)q~hh1nZ6HjZDt=Ne|n8y1C5 zVYBt`Q36VheD#saMtJYg^GpT_oXiqV_5^Fe3lqJ}p9g6sB1r#@W~G~rd*1@s!w?3q zFslh)L(c^bea_e6uzp!MVj$Wo5$5gK506#@{q#<~9rCH;slv`U0{mF}Yn)`u@?BPy z^gh1QOKX9O6Ocn{tcVMKJxC7D}kTYA1w9iO4GsBZRehAf;{IppO5ZaIyX2^48AC8 zMfxn{haelMq-PFKV6h>q>@(foRTEcu(y2Ev^*=HnssojWJ)qig9yKY_p{-S=wd3;W z@S6}uKTCJ3Tix9FdtLJEv2P0~4=C3L7}q>-+s6}#@LLL+>i5i*)O)vlYthPdH+6w! zepK^-=F-!O%1h0A?#Mw7@V|@3_RPDM;9oq0qOG_iUu*1jbmWSla~VoeAWL;m(4jTu zJ7~6n1J5i+2C2|>W|RF$d|Z|*G3!6@lxr4i$f@7I>LLmyHMYyyGvE;kyY60cD~dKE z!bC3|`z%D^3g`NI=~z-scIEdy{>=6AuhFU8J?}7Ts(1-q{2F9a)95-xbHIOvQMYdu zFgUxC?Dh?&-0tNcs24pSkW)^+ziQZ@s(yOPdlR?8g6ZUHDN zcHaE%SA4pnsTA{)pJrw?@7An-Z(G+}BC2OD586`$)e;i@{-N(atF%?=Sr^vMAaP|n z#t%>U=bmbrw`Lb+$#yB_$&pe*9&XVK1~r!Ue7=Bgb?abhM>j3+0=aW2QbG7_V>J5q5)9!kmw)}DlcUun-T{Q;7_ZSh%rx|Pbjp9v!1oS zpZuEOg;;Jy$~+q{_l`7>sz046`*Dh|uOdkpj)^V;#qowFuxBkgE48xPnLQ5wRkJrc zd{}d3W!?(QIk0%wTpz$C{AzkXgu`6Bd@$MWR=B&Er)l)8p;z$jBlZN79Rpf>!F)fv zo4?J7q?g|}^e8d51m3Z=_>WU74>*`Lv=5GP6VCR}TR|3(__t|7z6BoW7y(#_R)Dj0 zMfm5^a?i~uSFqy!gCSL1Tf3h_2}8&u6M);{8v&8)$2v6489vFDaiVFOKDWDG6b8Oc?2#`*WgY=XNPyn$3Q0BCVJ&nVw+FU zDIn()zWa&M;w{AQ0Xgz@j}Qmb?X-v=9-5eQJ?z=>&cv^v+_udVG$ng!*8x+c_L%a= zpgXsvE1o^pke5-JuJ_kx_++GFSf5WGbQAvI?tJQv>R||87yC-TOWw0-xcvP2S>%5L zmndC+V-FtZq#nWbx(6MRvi|N&=&AhN0-!cs&6ydQC8DcHi~Sflc)_)>F-9{a{bECN zT#e5%8fOdHsUum^V7|9+}%b7N*c-XyrDmaKAL zpp2%E@ub=e40|mU>!i!Znh+e(zjq~p_~kGj;gKi0CwC#aB4S;vIh5oso=@J92H!*Y z=7?z9s8I5cuCZ%_GJePVaK2>9f;%~Y6%iGRQl*Br%veE@x1qG%f3888i_V&NpXUbm+sfc49eiJ*JMZ&m??EzJxMw+kAJ~ZCd>)1*6HROjh^ZDtuN|x?n<_&PEXXzS1 z-d6LPk$ULdzMw?B!UW5%8FuKk|L3A6LOB5nKy5Pbiv&@c*Y2Ly=!y`vPRchh_ow{F zdxdRsmy|EWcSjAyZXm7*g>o$$>mBsD(yaj7TJHC#)YWiljjcsxHFzHFS3wwS!LDJP z(j}si3wY)io6V#&lKZ+^>XZXEIxquLHML!>6dT5 z?+B@jReE$EMW7~R+l&M15Y-&OEWXMU!;TD1Y=cUBB8=>TXC<_}G9Ntl_XInYNa-?K zlV%&+1tuxa`_;T{_t~9JHl~BlH52m_UmPBxLz#T}%^krjFQ{z`U$c3ZHR-UFaTm@F z@Qr<+%iJ*1RFvt$=F>AT;mGynkG722^!RcCw#5gYYcl)2Vex#3^wm={M3WW(!i)vD(kU#|Mg;eSeB^g?&13x-u|l8+;(|yrXdLE52rl3zX5Ks zW@DTJ^=_rlR@Jxr0ZnxO6V}f&AM24Jzq3n$0|zr6$vZz*7Ccjd5bE8Lz&5t z)bU1|N{jDT^f%m+a?N`wV5ctDh4;=sGz)3G4cDyJ%SQPt?|SW+Y0X1DI6|UM&mf@H zViI3g305qQph}}um8UyZgNA-dV|G+e1loebLPObkv@__aDb!PNj)}Y?sz=({g~j8R z>nd@KHBrj(nRgLkpQCJil|fm>Klod8l6YljNVyF`BZkx-32dDI3U{Z-De-P-kSoC z+I%lHK7{uo_;gzOw`@|VJf)U-aXaVg+;Uv9Zc8l!5EoEzSmajUDHG5c9ZjP~*E_ck zmd;WSZEv09&CF~oSw6i%$#0t>~a`^8(hp)t!iy=QKx@ScB|hF zlk3HU9@{LddNtUDqY*6*n^lP&-y^$~8XxQ3M%pKx6^c6VQYO1OedWW&vz*;k4&;k| zWo@f$xe)6Wt}A!5mH73;KZXwk-8h3J*IYwdKAS^^J$N`0`vTb-1GUshXj?0g2XPk# z03Wf-GlZUV^XeM5zL69;+6xNIzZ~}984!2#7VzS5?U!=Bwy95^k;C#1;VbW9(K2&G zaoif`wb{0o<#lM0Y<3aW-msZxw!^QM|z3jvi*o1YsKXUMf|BVl=f zfma`G;Xw4qXlneZ+ zE97B-xh_~uJ#tX)09vfJ%mX6H&p0pg%HPX*ZT=Yzr?G;#F1)1e;oxcq%%>^F1?zz5 zld6g9(wc8V$yA#YPId}1Ci^NJ?#>ca4jbueOm^l7y}op@nQ6p|L;XwskCLCjAzD>g ze(d~4w!tQq%ezg`$x%h~4T9J$3(doNy0uEi#Q~hpacnJ)+R3hrZ(oLM$46YXje_LA zPG-L^qBD_0FF*Bm=+6t5M z{$CfukZSspbf^>%@veq#N}$!P?t!cM>P1ix=~LZ1HjJG>1FTV}o0o{h!Jah9cvH;& z9zT8GWFo)e_H}jjZi>b;cwLX0+bxn;{o^vwmTTo_@Ncz>i+iUBGTj%PvK){9Uivu^ z!&&}9oKMnGPyBM~vdIq066HEA;mI@H-+g>fMYE)m|Ux+n;Zudovy`ac+Wu__N zt+ygCVRn9dp^*h3S(|N*N4`lB1qK)Hqz-<5L|=Yf6-TsPz!Z^F#7gSN4$c7G6_3VZ z))(%oOwYYh`1sdaH|c@t`GLwBdHBIEoyMA(w>kd;vCqrqD%y&@{HA;!Z!Yj#5yLlY zBGoJ_lEpk4+Vrs{2hG+_;pG=0JgM%YzCrqge-h%a^J6kp+*5>80eB+&_vx$i#7KU@fvSXZU-+w zfzGi|_NZh98G7XtXGoewL)s-CVM@luf2y+1aPIdR&y zNt3Kzs#>tlB{fTp;PN_j3{EyK|NG0cDYVn)+8YmU!Q%C+#X;-jzlpOf2Lp6Ff@#z5 z6?^G;m1-dy7q-D4WIOZW1hPAiiJ#cXH5piKe;&4UF_C_$(Kms`xA296daT$qkH3Wn z_{NH?*f3`SSG>u#7pntO1%EssFWo&!HbZ4w3t#qnVqC&oDP!OaA~(Gf#$$Bm7v%IxP8fsG^pNLeGcd@2jtQo{R7S z=IF zl)S)TrBNKjMNf*R9L9+fCcDG7=M3+IAMgq)Uj>9N`?oa6onW03)Wae=q#2bvDu_RK zPiHW`1@#yyY3!TK$3F_SiPhCz{LX@U2&vQa!;N7E|JWUU^+Y`7s!aAX-tvCc01p>C zpgx1R^HXuQ=T1r2PONl6`YHtRU$V5RHZOi|7X7!;U)$vg-~T=rxh0vvn?kLS7TsF* zKmPG?OnfyOkLDvJ$u)JX&dzV&tfC}kXd zV5c%LkAIIkb@FX)_@i|a-EtvsxZwhx9`4fU{SuNa2H!V>sBI{OT0+X*oybMuys=8O zcctiTID6Qg@~@HNwpGHMp#N;#%0WP?!M{gDqx*-a>yfq{Ud+|AXkcrH1ug|k+cgD- z`dQ&0w~jH+fI_4oLG9vCLNEtRSp51b=FtHu#g?9+!!zs5ThqJgA0dLv>`atDUv97h z+7}0lOR-WqxPMR9{-PyY(|1@axIM7xz}?mHH)LsmHv{uD8ePkZi1xHPMwdNmuq$$WZ^WhBnaT;654-Lle^j^673|!d6-!agnXo_|bO5@uEFI z!>!D{KJg?L5FjO^0i!Q*9l-bS|U_*$xE>_<-a|HIn#6V0APUG#+e zD5h$yR1JAc61$(Pqd|q_vx855UK#PctA{MF`Ys$D>FMXA_%LLAAT0%{TP!8uJ{(yK!7d7!Lzm!*8Cq=37 zmyz^V3a;a^uU2bBK1ID{^61R1?6jarKl7~S1YpZ#50|*tV1k(l8&v>mzls#ppiWX1 zlgrq!;STOcurD&CCt5iBHNT-=9P=}TH-}q5r|dG9IDrRuEG1jDU?guCo@$&|mWzFB zBB(O7J+KeEz4}02WnwH(+!h|M6ilWJqI8_>XlXC}4f-jYe10Il{%Fz?&rPe4SoLn@(#ViJ6M?Gq|}rkZV!A;YOEAh*e!O z8}E_xGjcKjhuH1OhZ$M-zw%xZq%vPxm#v;2Ta;14ULniCDsJMYrEVx_g}-H+h=hbu zNl{|eb3RFt9KWU1w=)#yQiQ|QhQ&Q-f2Jc;tjpOn<*$@o9xOG zX*}Ck#VGUPodtG%XS2oKeyqWef?98CjHK~u3ZK~0`?>8&GZAry>a!S&>4%twxf*+$ z+UHfQ0|MB|M$#wF_-#)-MI;-DJ>3Z^|0a*>2%l|wm*d})>PDME-d}2q%PK(E36#Q=#_P(} zT7pl6>AJ=j0cGZF#;naNleib|xy$CExPIv|=JIuvz|r-G@ft}I$m?oFOI=v{Ju@xD z@;wnN{LCA<$2ODKw62}9=GwaIy4zzTt0ow5%VR)bZTA_r@rYkVP`IPbEbO4d0>wtL zO06DnF~39dOPU3+-*L0hdi2+!+lc+(jnV96UlY3KE$<9DaMeOrv~C~mEGan2lCccA ziMG~dCO_KsWqe2=wKvX_^-bNO)_-5|ITZuN8!cxo<5D^ZQv`_o3%Dg0y13_a-FiYa zujk&5l6wxr%*a*J*Xpp9*7h5cI`_TUZ{m#Xv`xG|YEWZQvJf3j`T&NvsD=eKy%lWc z$GElrLrImGHRwG?ntb5lZxbFR6FcRducaRGw7`Na77RIR{@HwqX|F7_ccOg`i0MCl z4z4CpODBI#>IWG#4FOOn13lC*Io1U+lEIS5S zP2sbLYd7Z~o9Y=g#qBssFb->F zL5e}@CON==9S}Ec5{!LS}Jdz)4(5{U`$+ zXSRJyxIgMB#uXH6as}Re*yGa7JqG`W_n!ff?cH^QEA$INIy+x?Z@a$P(VjZX1v_xz z^B^I`dQ!v=gxUt|$yVJxd&~Dd1|F|cka91VpDa9p>=@nole)872>o%w1eAdOi(iz$ zF!*d~pAzicz7>}qRD&UNPEfZJ{}+S-xbBq7@*yj>s}PG&)m{j%VGHq%kc?B>F3*&O zf=8=f<*mVUuQiZ|Zel7M)~|1_(DuqRLql(b8$;f%tPDIFiQ`f}Il`^pM+8OaY;5%u zo6ux8k2I1z2g0gzi^(2Yk}L0tgzoyBa=$i}pm{Ukn$yC(W5`ZR{jBDIFdF^ux;rrA zh*VGwY+Sf~fAR7it5IQB|0NM7B^+qgr3mq@w`?>nXYl|@e|TTa{D++&DMLDXuVxSC zbGD%K4k$mslqh2!Ohl|g&2{NeGOO#4tRXW1_A?3at(N~!5G&IoJwCS}tWZX&tglBx zDb^h?dxue3wvT{An^vQh#mhWH?(e}I)a;pd72ICpvm0Wc)Yb}gV_EK0U5*#lAqxxtISOVtv zg~qhhb!pqQSYWU>x$&G-4ORI*-0+e)QzAWb>*(5BjnI4G5l;3wY#>9S-sa2$RxCE& zp5N(`o!aL@*Uw4-13kUg}ftgHL-TpC?s|FtjuhjWRz(_=~Wjb+X|+xhikp*y0aekEr~~|B_!(law!LQq-B~ z_N4YV$Xl&9Z24;6Dat&s*Z%?YY*K+qok`pzj8+R9`xnHPEQ!di? zzH(n!sn#e@>w9vLopGR1fPKn+Lpt@nERO5TR4nYa=s7tUVox8!qu;XY&n6e@CwoMC6!AyulWUwb=Tpv8zO70n%y7udG!~Lq@tNG&q47yqLy-|4 zxXs{yamNatdpgl8IXx`7?LKjS;0!AMa*&U{Mck!3>0jV z3FnXEpVYaz82JXhAhU9QRj9Oer>7?`^zMvs6gqQaOny-UNgZ`Y#=zjVB=k|9L*Il| z=CH76dRwb`r-D$xdG#j3t;}tDhq?|){|Ob)$)D4^3x4oIn-s<6{c~}- zmVV2>Tv+KNl*^T{Vzg@~e}(z$Z!KeRBoMaojOUUWSkcO+hUX&Z{9Vy{kAEPv&@yT# zH0aUsCF59bloG%icgBn77NV;Gt>M_bcitw*lw~H4+Mm}#jR%^_IZ2LVl{ZBC43+^X z4tTwB&vEd@u(xGioBRf2LG~}*8x-Fl&PuBtkY9^=nMjQF8RGciyKj}7f{d7;9RBQs zp$=61L7&Awk7JLKJz#q{+Ns|G>=|0&E;_7YgzP|x?iRQTULEPpF6-jmC9IW9#FKzk{xFsuw3w7_3jSv0RbHwK zwDPP}*p^9lmcX(-R@)rM!$ygd?|66~T@`_?JFf4k%G3X$4@v5_X2U6M<~<(MoCM8o z%bJ!No)vZGf4pQ&5mobIwV707bqdY*(#P4Ptl0{sgM9$Puv3l241b}ONwanEE>~k1 z?K7!d$9*!>saP&t>g(~SGr^^YiJAA{4oS&}xm)X7{R!z|2hS~vS)I45V=)G*QE)s=7Hy8#zB)z(SvW?r*g+u6&`} zkvm%DGN^EqzzPjzA1QJNl5w6VE0x^@HZayF=v-i}GQyX5SAiXDM+quXnC`VOVgc<) z8)pYzC%qN0V$yAfYdx9%uWSF-Z}a0t^IcaUW484CTnbj)oAM{U`R8h)n%#x;5m3`X zb78G3J5Mk@dAjM$Y@4z*){po$(*P1DwR+#uR_>z zh+@uXJ2=J4X$}>_mK2q7pzgijEZF>>CIr&?VJX|fFvEk&lL!IHB zRB_euL_Tw?JK7ffKa>41V#cy8S?jB&7PxTITJiH0EJadFZ!0+&A8izbaH~h+N9ht1 z*Ff9|Y1=KH+RJ*iCP9M<5l)fU%?ZmE2z)<7)3%RdWJo@S%kceoX!Ps!K%c#a?$jO8 z8)t+5?EJtVv>sQ99J2`(2Tb){0luQKmp-f^FIx27A6@j20lK;^4wyo(^T#~(&9gS> zkW!b?EB$m!6%fAE6w;n2rjO#U3O|VG0#W~w0FM^j-&JyBOCVzj<)L+ad+nllkvKlE zy^q?_0ZT@s?=fSEt$f$&Rf}eRR2kr;wAikFY3~?ahs!k7-R)6CeLKf-ycfUNC<2nH zxiaVMb9`^_a6BEv)wEiePE zb{;XUNF(;qaDEjm*;BIDP6m%pUERNZG^plsk(h|Kxvn(3oL%}rJC&zu@Od!pvTZHN z?@+te@ts!V4~+-gtdmrGN4^>aFD)R%OOrQF&P0=8T(g>>mb4?oj3Ha>S+V(^$tlV0 zS^5p!C%cC>N8XE>X*CuZ-21+hKL=`}=^C$Ge77 zm1IVyaQLq(QJ2S`OHJ;Fym{MsUJ177a|ZlJ>V`}pfyuaFKDl!(U)G@YhGbIo(bOqw zR7G5jBQ-(bj;_%)OHY;CWmjH)K3W);6gF^IV6*+^Q}LJITFo*n7EW1r0zd~#fW{l`>qgnCxG ze*3qdobzeb_Z_CbiR`tG*=1s9+62ptl0^wqr9u;ocrk2TC3GEI(rMb-Rp8{W2>iHo z_?~a5v+elUMuN)KdnZ!P!1gDRmdd}Tav_B1BSi5V`{TC%+-y6hyAzl|(bbuJs5Qy? zaDezck=kowpye%P#^V6ED#e`lB3h{{K$Uu#F44s$#gKYO?@#xh?vM=G)5Z!yWpR(M z&ty3kCB4I<|Lk1FPR%!;)j>KVf9s+`<;9;l+8HxeL=~QKa*P>Eou4GOB9UdyDYSQ04NU*7pbO({kKD zi&>5F1Zs`V<)KwwWaYbNH$J9)?5qT~i#F#|GC2QVY#NaR)AFNvj@>(Fzye9zcgaj! zU9otzE`=pe6-8LqTaNycdO2lr@GsaMtn)P5ogim6G7@Db>l0_R3Enu!bKdHa! z*kFkKzU%tX@o%XbVc(@mCM3NTzu%h;*@qpAF-Kio_I3rKq?Z2?tQx3?PRzgj`7&=| zi_doYE$+6z+sX{zQ5`m7Rv8z_&QoV+;R7m4+;6BtOT8bm-?ZgOlYbX6=rXe9YAb17 zGM?43u@bsHGy#2?isyXnN9}^N#(!u2-6sX*@X6Z#kKaMWzChee(8JL|sdlJc^3xRL zSy)6foBcZQ$D#JMOINS7^vW~QB2HX=c*9-M@&;?8u%i=1_}udFN(f^*9eo<4wS2ic zz%$$AkbyZI$d(ySii90DKpPB`>kbncG-PyIWee=Z^PMp#L)24x1_=+)-2Q|J@15dA z{L{T15z`xsze!BC@`p^3H}c++)+kfXPDF&UYQ^wufbs5iBJ(J@D}W<>utD%BTW6mx zti9mWk~L~4cJc$w6}O!5j<(Ukv#8n2uli9dFoAR2U->M%Mz}E}CllX8!B_M|IS_BER>{aNOYSd5m_b_xN!kt;`<3J- zpxaJjWL@VNx`TI5V)M5my|J9Uahf9d92=R-$3~`)@Ah-DXwHo&2{~SmcBa?a8r7q8 zu~)f8dr$pi*06w+JzoI3hs-hQIrIw=k=&PEMc;|)imMlrnk^wW3~Ij(xUs;DJ-})N zq6>uRcSIxPhWV8{7{E?Gx;dcG5PF?;%k?TW0k>$k>M9Z@X2liIuW5L>S4h)_TJnaHVA0s%nvBM^{b@DZ zjFBuug;L*=4teQy$A18Et^MjU7>a}2xF$yU7_s{=Z;SwTb6Z+Si+7|L8-U1W#Rlm* zK&J}6lDBm^G5#2*om+Un5wi_O>FOb_HhRBtglYbItI9n>GV>+m?Bt!h6WQ%@Iqic#`+7;$ zN&0`LPy-0jjf}WwrWLkkLBV*mp~b-lrAO zd$)V%^PFu6RQ^zvz4ZdWXtS*j?c3n_D0=ml(OjZO9e;LuNEw$4C#Bj&rf z&p4BeDltc`C4bmv@qkO`3O|K0q32Yk@t`!A+L?xA1Tys(dmes z>&syOO!Qzl8-^eEYig=AC8=zm~*Bx)IU(#^#5F--)7upXCBgio+v?hqr{B z!rblV^*_1<%AB%IHfPgBpI4mop5%jP3339B$FWcT0mq!K1Z{mSa=vj%7RvNG2h%P$ zB4)I{!D7EgSom&c;VeR)cID6=*2g0Io^OGo$@zbEqb;>oGzh{UP}01VgOAc zbR`ylM@8S{-Z=F&V1Lsy+vql<_lb4sdvwFI(IkEwU~nqfYJ=lVzm`Qa}cCupp(`MqfBN?;PE1o_~zshYu4Xoe&X32c4 zlSa1!TVVpf+1kRdONG9bAiTUPLEKDEX^$sJD@!w6_{0WpRYsnYm91TLhG@xe@LWDP z*Dq`yw#^M{!tV@bUCU(JBixqkZ>=WWbk`8Y9u+Ax2zo1!#<+J^4Z7^y=b9y4R1dvr z7C0ThUNQ8`iKX;$wxQVvqi`>%>Eh!WtS3wBndnwciDpL|c9KhU1`1v;Y2z&fJIm0F zt&pCUn47}P7p!wWlRR4z~@9l=r4 zg5FdIrmio2RmcrD+sGP*ST;YVEerafgWroM(RmL-cDSxy!%N+;Sw@&O5hJ7n%??DZ zpZR-h^H>BlJ-x%a*^iOddzRsQzAqsI_@9fC{kAhzWq z(-v**pnLqHlK$e_BawoA;U7>aMJeP^cl~S_7_k0-e`rBx#68W@XNEvQ-=01iw0Aqd zo^u_x_oNBw5!cqLYM9``u6x;P4bF6u1XxZf?VGujPKtEz9yZ5USVyrG=LROiPi?y4 zeSJ;)7u%I3pCdoUu%2U@+|PWscd65-Z9TDja{xb4EG3E87_wLVIp)aZO8pj-P)8_4Aj^s^wTfuPIkdpKCvGYGgQyM z6PwOP?DwsAe(?vh+pj3LfFYE~V+^DqqVRn30`j|<@X%}?Br`h&w8M;!-yiQ?lr^|! zRC-`ys)6+`E6MbWC-3_vZM1(XQhQ+Pa~mL^+4bHGxGV*eO_r@oQ?T@HOni)3Uxpr8 z7VhJu0x~9kQv^#|2S*hHB1()7!0ff1F7oV;ZLJWB#;AV$XiQCwp=W?ucun%;+K2O} z^-grxwRpar{wHkmQsa6RZMmVwxr;D9rtcH%L>~PP{`>vM6IaZKKR<@c=X_CB3Pd#8 z$OzWt-F3~scYj^(CiI0dRtu5lOgr~nT2sKnB+6i=WkF8w#)8tIDO9>G?0J84_h`mN!`cj!$`5kYR13F zIq*IOp|4euymo*lwL?cEii#m9sdpk>_}#bmsVZH5!IM|vj@F3j-##X#YK3=ca(YyGEm9#PVWExfy*#9|1fXnpbd(MVMH2He#El6}V+mXA1l$1WEOe zLJ4len}Pnysw^`y!;p;J-M6mZ)z)_}3U-ojpApRc-#)ZtM?O4;`ZgPe~- z&Sb7K2{_<>kRO?Dy||iBGvoQt-lmw@Wu>XF&$e`J?7@d{YCgvbU{>s_*Jf~WP`{VJ zHpP%3`8Adqk{Oe?H@tHeux_-X^=zom(#zt4vVYuE*!^l=1ijV zN&wLvwkO#Sdye0Zy(r6}Pd=0b*(-kV&bk;+p-YL^H{}qddivEPmdxGQbJ=ym1U!gU z#N?gL9mlb=GSvKIG=VkopQ)K>F>{lUJ^L@FW)5335O#6zCjGV9tAnb}5Y~KKaAn2H z(Q%=sP(`Qi4Yu6O^u_U=!18`fe!5DR0iwGm>Y+hrz0ZPBeR~Y(;C5IDih>40A{1EM^oga@9YX7J! zXq|yy*Zmau^GHYP=heI`F;*ALxxbe#LKZj?eJdWBj%CIsN@T|#VMJhS2z0rMHQAG z4S>%M4ph>As}$NhhS4{BV-S`vWk1C>E=C+HZRKPlmWJ{aL3`7qU1dQtippr#8$4zV z8Efl8=?YF~&!(a2{3u$^eZ}SGv%#_qpkYc3gU3D3T{QbqRx;fI)$0<2InWc`RvKxv zm%{}q*s8UtfA^y@32q{MU86rAS&w-CYub3jDYHX~T=mvIxk}xXX!4%6ICn#LOBTwR zUJkTG&efy{E*{1ejq`p3l$Kx10I6Bl9Rmr6nKK8>+=RGo57p5IQeA38zrWV{#TB`u z*IV1@;Q#>UU~ATEOZ)=@HTYL}RopRuhJth0%IT%)ui=^P`sq2J$vU z5nK9~!pJ8?3;rJqupV|9J!o2L-{!QSij~e5gzCGL!B>h$_M(CmJ(OV~CZZn%%hrmX zN8jK0pJvqxtfaMRWmF(jKzlC|0<4y#d=Uwm@ZT{RAOP z(oDDCGh&++IyAR4*!LZfkCL~}zs?ZMnRXA!>5#g&I`w|{Jo$~bHde=6d$+i)L(6h; zt+6wORE(<}UQ3LQz!}Yt#5_dRXf>@wgnOSNTy&cL((M$?2^9bcjKkZDsa0AA&YfMm z=;AZ}dUnya(BvuI8g3u`qb!ouA+NTXK-nwsVik`a?DqUTm=TJJeP*X?O_9XbKfR03 z{YZZDRG963v%B4)N)g`yuS0kBx0h7((Y)GzZv;;O%4ZNsfcIJi(3sU4&KB4kD#w(KIk zmPgZ)i`J;U!(#2FmAj_9FRE8^9p5?r;p_&eu@9rtiaTSV)++$E8_Hfy`)khbyIll3 zO?}Xt&Fzie>)!LNSWq-zw4bJo#|WB?*-Qq=28ggeSsLiu=OeC|B44g+NDMr2;7h^4 z3w4Nru=4|i?Oy$oI^_=}_;iPP$Oo-V7v5&IQ|dwtJERmgZ5)+Rk#eSZZ!r{tRG*Lh z>{{*I^O9ioCb|4le4+M+OjxUC>d7=KpbxAR%KX)j_IBGoMzirHv%){ZmK%k}c#%uN zofb5d6R){-xq5Ep@9^A&Ir9XjJ5THccarzFN&v2tDZ6B+SatH*oSyXl^b$XKr00+C zhQXeTL$ot*o+hJhtqu5oj>(8^edd@>@+~m|gFC_Lvv?W5g7rHjpW(7}mB38m-kpcL z=QB6IQO0djLxSgSR~@b2D8Tz|(R*Vw>z`KK#hiVC#q`|&U_yAy?5h#DzVayD42Gmx z6qE-}71so^t~@g!c-@T^K;*Yb7m-P%j5Cm*K6Qygx>9-rDMQvV-DZ?C1ew`n4J1tk z6j`;65-JQFav;85 z2c66mTXX1&sB0wrg?wp03MM9GjmT)DHcK?Mzb^C-T-)7vq5^5hC9b;g+sYy|Y;@i; zegok937S!b?yw9s%Fxzr_{$oonROB=1-O7%ZiK-{U@su2@T+D91xskyFZs@3?& zVED{TsMJW{b8gHb_%u$*x&l*v?>r8nf?vRh2_4@3a3v%9T|rdy`W`Xxt)G{sDgkRb zLud(V++WOCBT%^3gqWgTVH!U3keylz_rYNvoxcqIrVyd-eqOI7u|?{?BmLwbTOmO7 zxfdC+z;D>CnMn8KtG$z`G^KBFPSDF^xNNwz)&I`?AtoCd=YF&m9aHH0q9AB~`+?;F zU;bH??0E*W5YM$9FsR3<0(SbT-h|6!&!M9Ew)0;xt1lEWad) zQ4Q^y^uLf44`xd3OeiqEf`5@)zqZ`wNnu+dF!HoopI>MMgitIJ{G$8?JqOjIUIk* zTaR-537P!47?BG#dCthCh&cZ=%xsNhJKrIqNDBb!)Xc`AU*N+P>laVS=|y8%^fr)p z>n|mZ3E9p9aL}aZyDsYVeFRpjj~!`O^~yb|(ZfMwfi=*2H^g7-UeD5eIE^0D7WXmY znl>PS-S7e@m^j}6>lMN}Y^E7r7e((Q%)KU^#oXeyDauko=X2>xVeTbvYiPZHh+=XK z5>YmgTE#87+Y{%UTfx1wJa-JlZfRDir34qn(&iUhn;7z( z$8N>2h3_>xt$T@nk{4(d19cTa=LGdczo1{Fl!$W1e0yVHLb@%ns;CIQkM$qVuane3 zg92(Kqw4!AaO77Dsz*>0z@H9?(N~TXDYw9I{p=}=@%Tji=xi}gdU`@9ab{Jiuy5eC zpNszCgwy))z-t_;8!XBkxb_SsIXV!I>46(TK((7%%GT+d!50ReA+){P7FwdYCTb~o zZ;L|Lyv)H&?FLA{?Ps`;|zwQRLmz3s3sHvSGT_1pc*YXkKC#ty~G_9eL;QW5zG zzb1`=9Ul=nU60_Y;41fmXSi;OF$hOqvU9&EFJ0fisVr(r+G1l+GUR!;&-;4O2Kd5% z3$o0H7(r1z&D3do1J8{OK4=RRW`Lii)!_Ko>(yRztV{Y?q+`i4$_2M{)Eq{!}V5&lfQ`WiO{IA8 zxR$*BZV$(^C3X>`3VD#$){G0i=K|NtTgFG(o&vH?uwQ!vNA(M2XZk|C^YSHE=0eel z_1^!!a%<0pe?fgx**>cU`ftP<6AO~9SA$&2M}RR|SK!@07vANriY>8RwSv}QT0mRF z*$TkLDWGc{eH$$~OuDk%s?qzg#Gf&oQjPt_Y+xz=U6lfLTjQ_w#*v;_k3GAw207cn zPB|U>_H;0fb{I|PxN)??HDj-T52EvMd7zxKMMWTQMi>nc&ih9@V+PcMeHaMFTRiKhOC~ zYA*cku)nPeF2iKJDAd*DHZ=UyIF>Gpsa-+OR!PJl8T zqs$$_@loi48IFpXu|KJ9Y;E(za{yYy7!b=oGrc!l7d2x0dmxJ2JMEf6~|YI6dg$mBlWx&$H;ymEfxC(l1SR`W>qIgN{e)9 zy_OhhA}|QIMiBF0?MI)o^&UvZ&yg(jkfRbR5nQ0YDP?VVB~@=wW#{_4vib1<3RFJ8_vUJjKDgN{p` z0)gFp^F5^LHK{37{cCq{C`HwG_Xvi46s)Z6lP1Vd#xxxIe%fm*CITA|9W6HvbSI}D zlkU_6ews4OwXRWHJtj9|3RDr}W`vwAKqCk{{s%AH!ui<}0Boq#-Ng?)>S$MRRA>{j2;_w_!v!4C2wbU8YhL%nr?BSgf@@MjQ1%=mw>ykQD zbr_q30geAD&;y?bfu?GmYz#)GBQ6-_3$ELg;@6o&8>7V0wZrkfPv50Jv0Z_{P=FKY z;Xx3$F9E8eNow=iM+F}NbTnZvGe~M$j`u&Ob@I0w>IQ;tUQ#@D#_-dnLa+2% zlbdP-M#4yqD@f{7f8BSMj-u;mIp^E9Z=`+r6cQInud1qEJ0G*v;~X(BxgbS)^e{mU zHkq6%QRfA=IV-N06u;;GpZ?7-c~ut}cjq-;Rd^+o3MfrcIgWN@{7`v0#Jg?=eUdJK zddKOP$kO_;S z%%D2eueZ!Bb!4_PWovPM)b6t(7=!dP6f%d8AUF>Ori_tigH+sCH0Pb#o=sK)zp#cR z(do75>qVeHpGo3S_|_AaCOr6XonaEkR(VeGqyGOg&8E`D_jemU|NM?QW%tl4Vm|bR znEz=pK2hS>DN9Y?8arG;d)|JTt)LM$NsZXbeUk$wVE!5HnTmmLM#bs=4Ns&qU^8ZZbX9FnRFhh3@vFL8<|^fY~zga=FHg z&b)&?cYXK*p7~6?yE4#5JhPu!Osw?_oA6W5DC~7)elOtH$*9lh==10_eCj&Dy_^q6N)HJ0;Qoq3{xXm4!JE$CT@c&Wwv(pD#=Kix54|C#*lu*jg-UoK+` z)illZhsBuHdoD(J?E~A|##hu1x{W||U zQ-T>oM7_*w8AUA=2yQA3f$oN}8rv2(wBnUM(DmS> z344xH>&= zj9O-)gKw3y|N3)sFXeYkJN)lY&72yo=`3)Z?c)FcrGMAgN zq35t48P3)NpB)189$c5%PUPrAHkgl=Ke}&|2PzgGte)Qg6}ktAM_sCV2*D0s+D-C8 z{o}Bf#@OB1L{EzER|!t8wq3`TEE)>V*d;5829{?U)yY&^uut1u&(t29sC4h2wNdBQ z2vqF6e>&)a!lg#z0DaUKq=jp~K~^f{l~WKm8ec}_*n+dO8o8icE7SWxsrBT+5WT+W zo$e0L1^l~hflp|c!1bPpeJaCQ9`3`RZaFSctz)qCC*J*GXI-FugtNE(*#o$VD0uH0 zaUXmT-X>E*6NIcN5R2Foh2{h=&Qa4rBClat+I1f?e9)yB3zJc*JPfuRR6q7}g^~Z8 z-l|c$aZY!yxOp#m)UIaZ{g9=(V-5B-4NR{25N&`e5TxbkqjtWw>9Oz00S}9HTZ5_; zx@65F)7#7@m(FL&e;tM|OvZ+RvOb@vMJQ%ng))gUbFy3)ecTz{` zjWeW;3N%kK^yC`VFF5Nfts;bqEoAz6P3vDJLRJ(rg?e^1MeL6j$*3z>poNv65-{0w z^#IB|ZL4kw=qxw3*+e?bQ%2dN9~=Y=L5X&y{Rq-2Lj&)3cdKU(zZD6kEyW1`Lr$4whs_f+eBO+n7Sqv}ySX3)%+$|+vU}6(oi6)mg_dqfqHsOw zJ<{iw(M;TwVRHn0s-Be5B=(az-AGg2yq4=;JMbM7194f`@8s8Gzg{56?*(QUkScY? zu$Mw<+IecUd$R>WrGicW4dr~13*N)dx?y?erHd?o*eXs?9vqi8C=IQNpO>jJ`N+5D|kL3C;gS3|I zoNURY(r=>o_>YoXQi=*MwAvcRUTJOrXzeh6FQb`w|y1WKPBD z9_QrEf8nLWM=pI$2R8bYoX-_$p30Dc2~h-(L$B}jj9%(}YhchX8Yf}lO;%q0D!sdF zpp?`!Z&-An+0&+Fh42Vw+{2leA@u?`oK+~1nJx%sV4{gAF|hi0a!SI=_VZtnM6;FT zOY{-FOpBSU=G3m%jGy#0a+w`NV}x$zT#DQ}l^DY5A{IGOk`Ozh9QaDKytbOx##xv_ zG@tFPG%2+miMhS;Te>LH7S1plrELWLo}8=nM;tVhPIJU9(W|%n42^;pB^LHuK4iS| zxdP~I5r#vaxBrTH)do)&bzq4I=#D(|_pueogO!&+nS6pmWT} z4Gu+UA)SNh|N16ml1Og>O6<>7C!AfO2@ZU_Web$26XZX{*c*Z%0d1R7zMNYBEC*c~ zw{j7}9;@v_pO7bbZ`0tj%rkv0Cv{@XNLkE&%t`%LJI<$2{duZQ(~Wrtd1$dF+?yh8 zqQ|=`g~!c`rpulhJziSJb*dC13*KfcsmVLjkb}>0cuyNTG&Pu@T0H`o_b{IN`IR}#AYr;n@_Ah{f^#1lb*Ss>CFvK z>d#$)K}DU(ny8XSA?%Kwwd=rVoVLF5((v2uJMc|PyjZCXiq5B)UMeda<)=ftHat*{ zl1$yxUh(}=qd+=r!*NDY#b!*;Ps?hk*c!NQjc!UMyDsDJVCCjKP!`z@T{tS)9RjN8 z>G1bw8KoNl-ZOT8Mu>naW_5*q?s%vf`qViqV#Q8Y_%z(zm92u^EWxpk!XY(uJisll zZL~vdmn6xcVhSv^0f~mrN)vXZ$A^nf-`@8+OlnGupNP1nQ^$A?ga`2h#bIM!F?v7-vTF-2=_A_pti;UXkH)MOKlb~H>BaH5y}%)9G3(%aI5B7@uaCGb z(w*pOw<+|b7X8{yxUdNT7A_vCl2Xyt4_$&tCIo~mew>v$n)!0=G_T6c$LvGM+&%6Q zkN4*y>SQiYSi5@vyVf2=3(OY@vLH7l#(0l|t}L9r?mYr&$rpkk8Pak}9;>8*cuc#o z@^0^R!injR9wO^pH`7l(KJEBxmav1@-F+dn7&k}pEs^~dHbw>vd zAt4TxP#JLg+lbv;zO~W2K03BQ`tym3g{T=}CDYjv@#n$Zb`Kh~_Fefl-D8o}Gq3?c z9Rp#vI;qv$|K?VO`?=T+UfHfkLhI)d4?DkOs78|9SR-p2%Kg>3Nc1TKXHV@5aKhf3 zw4ot!eIHjyL{@{V`{D;pm+hfBqma|AQclB}-NM5=SDwvdGz@F$C~ue%gAzLC+dH5$ zc3=ATs%pbXl?Rm{^qRxEK;@i<4-4G;kHBFUC=|Rh3k11A zx?5;E>PEijP}(UO0Bz3Muii)t6N!2 zEo~crV2bAbG~1sRNBQead``?V>I%5Ahb@0K_L?sCFr$kh0bU*fJtG8W-2ou%f&%Q~ zhrZxViQtanJ8B{5cgZUiZJKR_u-`^^FA!Jjj<3yNDz4hMlYO^h+@Tdtr747Hyfxi) zT(D_zg-V?M7PU?;F$n)5RdCa1voXzc8uRz{w~x>`S6^f&l}Qe{`IJGPY>7*uEkVRtC?_T^HOM*1)QcVQoaxP; zHs45xP`@)v((&cH$N43!%7cEZ4$xH!XSwSX_33RtIeeO$+p3*12_}y`&dAh@(PQ~t<==>`69L~+Aa_X+bZun3-u5c-E zGgEH$)qJC1_qCol(v1J!r_iLNBs=%B4{uO;rnp+GCf5bUai6iAPpKyb2 z;o+CBV6s*_ERO#7KLYvKCA!|tEJM%qN042;{~A&-o9{-8FBJgAz0dGPi5O1@-!o85 z32~g{J}DKt2k~2arzM8s$i*isLg_}PQxVOzlqStmg~!)f*5RdQSwY$@%OzShu`5S1JC zS1T{Pz|H1#bc}IYAoYtzMdR2dPKQ8H-Unm1LavuJ?D9RZ?p{>@WWc$0wje^4%=$O8p6d*6XP$re6sseL(bzTxN|i`cvshRcK+G82{N_3L)%!#at$`Ebn2=Cxb#G0s(oKr+QnTTo0? zFVbw^8Klj=!ivtoUt(pNdjFkOi1M!;_=^9MZw*jayEX{o4aL$=8oh!z;;OtnPoqlY zjJpLNxs7fnMBKWIsL{$=ea~UA0Vg@jQxNI=MWjSXneW3ISsRRB*Uh%e@v@} zFuyTJJZ6<)lsmGd+lH%IW49gPom4UgM;Vnbc;q3yk!E^Ph zzI87rDnYZ=uCIN{NN*7`pJra!Hh9YoZc+XEs_%iZdIRV5Ig8D1K;;q-X!k5_lxD*=F z1dO~hI!b$6bm(_UMy+X6CJaHPvwv6=y;KB5o&vCC9+8O$cyZD_wkJ5;tYYH{ztFCi zI{?h1=OUAtC0Xen6vKM1o$e@u-mhV-y!$D%wpKlA{4h3oEUxDA3)WhMjzF z1nspuh+2A~6zqB%=&>A!2klQ+7ET7g>cpaL!A$3xl$n8VDN6Q{W@E$Uw6AhRYE6Ix z3FZAW&T0_m4p3aCNBrww(m(EmuW+swyaa&{V;?WVGdwn%mDY0q_%_$5Y}e1e^EPrX z9_%QaSDa{ho!P73>{+iO!0w%%0@eBOTX{e5raT72cQPYtdEqk-_)Xih>GQRz9~JB4 z54q0|qGjkF#Tlz}>2+^j(-i!_kvgmvuI|=rw{9QC% z&1sisR5(xK+8}-``a_5;;cN4T(mtnioW+QVEm|8}+N$+vQ-=WKW*Hp(I^8`0d?Y%t{F^`BAwA-n^Ln2kVjiVMsu z&Mo^!(oC_3CH==)gBt_$&D;k(#%HFG)6n$Feq_P?XZ!$*W?YXgyNA7+JGnAf<(L77rTY{rtB1?|+0hYTA-h4adbqtM{Vn zSX8SBnP=WR^P(U2GWQ-ZJ&7G+C9Dfi_J18MJ&DM-GwA**Snu2rh$#-6*NINC-6Zw- z^`~Oa*Hb6OjeW*ZGCGVQUAmFcGFgjO_X*t9+ibDlD2jl6>YLk(()Ka*Fd%bva-o*su~!5A!tp%`?Qu>8UYL5+&E01|0T$OjeB9dtA_JC;Jy8rQ^ZI2 zk{)(rB6q{4P2(M?#A30yo6;E+t$Ks)m>hI7d2mgQv(QXjQ7;_YSX`C9=tT9;gd*mI zZRd>Lbae2fHoYmT>h^DR9Xli_;i$I!J>he&k0q6~+Q^KZ5B$D|$Pg5_m zeHs9E`?EzUbNnA_le&HV7QGV6ZY8b8T&bRnDu`ELxP6hpb^YR#4m3fMe=I5WD8$1A zWi?aCH!mD!aDStxgV@d9%z%6Tlb&8*Y2j&V4*(y%BxiZ06I1FK-X}TR9S-S5QENns zjP1(5Nd+GH+V(LpLSSF+e1vvD_N#rQ%wp09Sw~8|>1OL>77kwot%5=~H-9i0vfVBH z;~Tm<#GyMM^p^tq{$?$k!B~&mhIuW97T%OwBFXwLTAcF!!*eOAMAEqdj%XAW zH1MZtYR7jgu-F+BeVxIU4i!f6cBlMTkuMPu=c;v*%V<1`qvO z)aD})F+Ja?wgBv9PQTqev5rP)A;8^9VQBKp+@c_{fFdV3Tl+W8(I456@RX4Z&GYR5*IuvjsNo@C7dY%T*gvWQ{5a8?afHlm%5otLs7! z2BOL}20w|Nw{0#65W74{x`NSk8U0`z5T~KCJy%-$KHYn8G_N|sk7|@!-S3j)rbvZ_7Y&Kufa^+f7$lpA%j`BZUpAfw>l~L9q^kwF%H%tSWAYKH@LA(N4jDFvJ+`;bBF8RlyvQ()#8 zn-mj`5)7o^k(^@Icl9i6Eiy|HhJB%%PI@)`YK`+S=Vzov>>nOQ@UVC~uY$y&U3A)C z<(=DEsO`jP!Sv`fu$#}fZq*^*54y$7Z+$?s(!&RVZfyY9&~NQW-f>8Wq03>`pK}dm zRjiXE?!$$-Zlkpyj5ud^IFTNCHG0?9ZWW>~E}-|2xIsN!nh2?WI3RAB;+E!IbrnCG z_bd^=!x{ey>qfz9gekEtvX)sgM_+qadWH6Xea%o9M|g z6mEK*yDB}64gMR+FW&8zwtgkq^oE*vy0y=Nal8szN#HOlG1|Ihc6m0puY8OS2tJpFc?o1Y4ah#Jj`OXB-bvnd1?{LACb6MPB9M@C?ShF~?id%-i0pci*gAZwA+o zJ{!T3L{!LUPY8)Ygc+Y-0EEGRtBMj02QoRot}G~d4H`LXjy)is6=yd(W>+v*UXWng zH{{{IC65;5$ODh77kl0J8J`>H#e=+%w#z6dD9wWRc;3o*V&{61uLOb}`Mnz+tQ6Qw z|02>X8B$7A4k0}O(%v;2!n=%S%9%?WLIqU>G*!_^q1Ko7a}r`iZTK;%DJo~Z)M9j3 z)Y0q^I*0S)wcgzK3gaiiVAtpDOF7S*+Q-pZ=GJ5AW=Bn~ucU>7LjH<^xS4;m1CTr$ zeV7oE5KzeCl?RB@4Rb6fpGfB+WD)G9_2y6e@>gr^hx8Oe;9dUgCRIW1y{9G?dfdXC z!%;MrWADdwDL&uG{po**mVrg`P~RSYl0Q2K7Yoftx`Im8YgT&;)dyT%1FV6|vH)c1 zNnjtpg|&%?A8+M(a+1hw?|bx6Ufty8v3>+54B|dj{GxT-QmxGBi(GmR(t#3F*V}nc z#yi^ID;q){rU1ocqibojl|RlFUg3GRVK|FhN={r?z4YG^5bZ!-yTAW*$Nv5Gi7l>D z7m^~RJMpKO+Ry_9fihcr`pmobS1yY=p_y}XGO5U$#t<9dy@7a+pJ}%4USM3~zsceY zv-DNi1meR=gwD?{{UDvBj;Fr-L^Unnq<&gdhYJrDo7@JIo^!2+B%e#>wYe5uvW}&< zd8E=`%Bbk}mQ(J!35)4rQqhJq5JsncI?|(7`*yA_c82<`FI$D+QtP6H{9eRn4wa)H z-BgnowU8a%>b)|3gwI8TuNZdwasvldya6^x@_Ff0dX1j~CYvq_!mU zf;7gF7tI7=H=y%^(zQDjbt$dZ2R2&gN?RGPzpIFD4%sw4&K2^I^c{p2UWzCjtDWqR z0Tuy)pQ0(Ympv?Ag*_v zeik?zo$k+Wyw!#ai9aXjlh^)ahFSd9RK7kbjOzcwRSqlJN9Iulu8YSAKXW~TU_r;( zDaBJM$*nAL1;S>-j3A2oC1CoCP@EpNc8C54r`X#wb7jr33!hp{ol?WJE{B$iRgN`5O3pT$Ig|ea`xJ%1aVA~2uJ zdQ1;%JU)Sn-oKlwhAd4hI{y<{g(l9P-loyZd3K)8|0xA)eugjZM6GU5)uj8GyrG=W zTv|BgkU%6)>&d=|?nAAI`1|qbv)V|h=jW#JYbC4}Cjc@!A1cLfTTQCpMFw3o1si(|FdV2K>YS*n4NgVx0g8RGX@pOSixMkVEpzjW zU@jlI8+7vw;}-s+4rvzk{UsN5a?YPnb3w&gn@V->5dC5EQ!)fB@MS-Efu$bvjzilIQ0(Lt}Q{6`~ckfEkkyerCc z{4KzN&fkJdOPDyOyPi$pWNRz99seZxQ^RQao6-AMSbun!v^Yks^|tN9-j-Z8sN|Ux zV6O2&zYff0?V`u+m3!XJi6i^^2{{o_J-*K}ye-KB)=&EYF5ey$zRyi#|C*};t47r~ zCQhna=W$97sJoDP&zTNI4Y|zhdld7VRQ?s0PLo8_%*{0m>o@+Q^D)XZF53;m_B%1p&>;QUz1t4`0{MpT8Vp#@?lwbIT`yz2h)? z%dr?JQL~Cs`s#G@HRY?dM>L=-e(1Sg5|}d=JNET2vi?N!l@z;TKC|dk4nN?hN=T*W z3CDJ`Rt8r$b~3|yvWan)Mxt@~Ao zJ9RFqV#D9-Rkhhi^@PR}ZUPAw#GZuPN+boF!L``{I|SN=VsvOh;;Ek7W-ICOoKiEN zI%0t~-;(S~cThx{Y}B;HLrvp@Um#|A;~j(>8C~AH$>tBYfZ_wrIH(f9Ed>E5RpuNu zTHuRSNd`0?3$K=KF(MgU>-FjCe0{7AVE8TBeR0}n5~{^m^Hj}Ytg&_x^&Qta__si4 zcrn4A(ExA==Ch6aZk{JtR}MT^VgCbv82)H}7&e1FwhUKsf#pYw38o&+IR8uM%OE|c zdwgzvo4sQS@ea{{+;8E^h39LO2gB7O{W44XQ*->hrzrfshhg4%YgaAf3-FSu$TNyv z*#&%ZK>t@Cw^bLMUq$hY0Is<#AiJ^fZ*9?=LNi&@Ib6zG3sCfiO}^bDgm2$0Zj@Iw zu#cS=`W&_MuQX0fG{H1wFWe#U(LGOo)tR{CZNk17(=aA{D(VdZ@ejY)Og$hI)_zcD z4%{w#js0LkEDIs(2P6{gXkQMn7qbRAw~QORVcrdTWJ*kc$(xDmvsdx^02c9+IRstJ zumR66nRX)S$#(w7RTzT( zCI#pjbnhH+RDlX7kH+D(z2^h`0QDhSk4d35EoJ$8B;q*R6E*?ukTArVgcl9u&B4n> z${u8^XQ(?G?u(Ou*lseq5|lIE zC5?wAq*_VS{gX2rzNbOlSNH7G-xSB%5O8c`<_+{!>RIdyd#FiW+jJ%4mSV*>7m76i?($uOn9R+OR(A_VW@S;oS#bv5`qp(n0YGtsL#QbJ zLEU}&!wOd-A?UTVMb&T`RxDw?b8CO(xf_ycw3Cr_m;I?`FC`Y97pc$StthEty$3l> zA9Zozh_&RV#DxrMU2Rs4@y;^t9lkF(&!_8(3R2S0TW$H!hZ zwWEm(t>!}6cyE1Qd>i#%+!oD6uk&PxUHGTYTQw%xnWMZlPd-IhZ@+UVc%XFdDXj7# z09x^m3`>T`;Vy;|qDQ%{$dwsB*)~fTqhOEpq~s7xcoWy+uL~vuR)46fid_nQ(T%r4 zyjJ(Ab7Y3y2kkZP z%)bQRnyP7j@M@c$(tmxS(n|$ZJAL~djy#Xcj{bB^g}d36K+&eHIyOTQtJ4=}s+KV1 zy3Hl|eTv#=6fNU#Xu1y!vl{to#$EU<9W@s%MweBNv3X7Fm2&4+*l?hB8TT0;wef53 zKU~J%G@?>S@ydEOAc-lPdhjOWFFVahxovYb`1tpsJ-FPgpT@chE30$h1iFBn0vHl4 zHbfHWudGY|xrF^p&zF)&R+1aC9@#(XCkb^WI!ZX}B6i&PC(W53YF?gWWg(}ai~pqL z&q%$0E9?Ij@PDEc``2@4v51kvxWbdwjb*mhQ2B`xE30UMV`!#smE)Jxg}Wh?$+F3^|Ea+LYe)<~H6K4ls=hjo`3oeV)psr5?o04p zcWcmDt?8eB7x^r(J)U?0q_>8-HLg&H!jo@-=_p%939+ z9i_IR2tGG9C2})QY?z3JMy*VofMNzG9-}cqmtPDdje6+vK!D$9vg<7;DhiK8r)$A# z%^t(&s?F3KVi&Lbz@#uKsQE!Ir{e@zZO$Zom+-nzs_d0qZK~#a@a%q4Txg-xgDZbI zT=7i7+g2)fd$|qP^wtq&&{ZW=WHXaoU@zRnWy0#X;PI2@p}UwDEBa-MtYa|uWU-#S zPZn`a#q}88VWRxc^}&C#mjm4{3az7Mk=2|J*O${pt}Yno-7?AsE*+PeFg7Z~leHw% zAI$xh-YG%|J1hx5Q;>!9D#4S=9MdRtbgBBBp6O2THp z{&@7?@{Iblg;sCi&kD9M5ovUmKjI!1C7kDWMGMbAxbJ4G@HE@$t~C1XD=E@B$Ti6I ze?C|J^XdXiGzaL=6|VsL_R7u$H{o#bo^;Tvj!Ca9R307(TL?T@>AsygfO?xhX1gOK zpcHwq&e7`(8#Jm1M;`Prtmp&m`azPa#EcS#v~PvzPQLuft+*_s=72)!@!wCD|M`w_ zIbL&i>eKpVvC&ZH4U8jtDc(siitKZ-?4!A15}#$MLzcxLvlh%|U%d%E7q=#ISJjix zA128zp9g-=1IyOU;_8w~(w8`Ss?xCqME(xqf)Poa$5z@g=xa3F`8v%fFt>f^h-nyT zQ%!!bWayW%7M~kZB)!Q^wUDdr!S~>k&F_tf!w!!2zQlip^OqC<{my@r8MhQrzZHE2xrj4TDzL0Js3ZPMWLj3HDjF5s51Yv8ny{Jv z+bG9>NsKf7EFE+A^H`i!bx-1Qan#h25|c(hi`7e8mi+#RlTWvk9Aa3Zs9Wm4*U1jrOz!hxwk%)Nlu>WZ&EzM z{q{tOdsD%}dKLG!rrIiPXTSSeo9XHS!U4j6?)3kDQ5!GF*yes?N{)VU2+?cWFw_3D zA%kqS+2#WBW)BIo3MJDaH#0srz&GBuJ!7?bhr98QS8u%WZa?qc|AA|*muMz>GyVy> zF;b`OhTr!C#%?@oD@w1U3>xgw=&B!YZ2I4=r~g%LQ@4$-lQ&yptg>A$cmVh3`BWN< zi%~H8|JH*2pZ{Q-(fVNReC-4yBdzpSAMO8hIf`!0Oy9Z zvVZu`)%lON6R4<(qJ4l}eM&q2|8!XJxT^UV$Z@AjL2>V$JT`|FpFFYCYSI7RT>l^4 zlKJBQZKLGPe^m9q-W~lfTX#cc_=wU!EjtnTVt)i9RsYJ2csPdo;uUiVAPl#I|G!@k(r@3 zMCoa2+QetQZX2oKxaXv@sYX)%80ySy=jWz+5BZmsYz_8RDOH(>^B&VCge1G@xa zO@}4EAfjDpsJf@7n8{A35`=YCLxF{?Dr&w<1-^nmu$&Q-uSFqPA4E++2YV>}qbpok zSLz7AxI7%pD?{_&^tUH+I{ek*B)xJTK^{tle0z;5b+MdyW14~x;EDPP_U7I`x=41@ zr@44{Eq8YJXNLB_V*%zFNmk4P-(E*|%i+6@sCiC2GrXen9kjeEJgTR(dz)@_M+rM! zP!4ji;;><4xC`}0j&8>6TyA0_XsjiHF>7Z{zXY&$4x@GV`7)AWwg7mqsoU6%Q-K*y zFLQdQD$Rc0TU8u#o35f`061UD?6jb96NHVxz4Cu9@4I z<L@Y3JYevkpwl!;AG|IknU00 z6R!XbM~CM*-C6Br2p8Nu2|Y(P8=P5dI)GnL){}pW1LcVX@My_}Y+0VJwe<1;s=I6| z)2!i4xwA%GgxAFuYmCJ^X#DD+s4LeB0dUm1^R<^Te#g;b#YhqV(m6=XtDmQS=(ZYp{vZ{gW8c+|IPKDnQ4 z(#q#eej+3IcWjw|b>`TySn2Gr~$L#R1sAry=j844?7_>@y`Hp``+H61^l3HgM?&VKTjo25_j2sf!gW0Up zjo@sp5EY>Cz^#uE=m|WcO-T30Ord$0-8qA z%0Gldpm)){Zpz<9CHA(ftks_>kP4Tu&!N z&lJ{oI6#J&8iYnCdIQofmi~&o8N2TW^}1=9BsAbQ6$5U1uMCJvW0~C6xR9&mK)+ZR zm_Zt3_~ppoRkW!=zXlAnN0^Kc4URa4naw4T7G4r&Xi?3JrOA5PjuBBE)D_91DF-ZG2(3?@7$lF{fb|oayzi}U3R&w5w7J6gG->EYa=Gg2GkHXN?dG$6wR3@R9*`9?=+!9a0^CWP|E4{mX5uxq^jB9YM4xLxfIdeRtsRzj$7cAoSD218B^)Y*`Fhf{D zRYm#b0+E9(-@#jyvza`zOkS5#F!~vS-*O$)2+YE=G^0MpMt4ihy>E>gfh7w~?Xk8q zH;_`%9Sslq$8IBjNw?*U)nLjxO;(>N>72{NHeOL171WzX`a?BJ+4*i-eN+O;y;$OD zrzI?7f!Z5BK%~-oph?9qSmDR4+e7t#8nEUO(s%gEUaLh*CSawr%)}Oo$A&{O3Ayl} z2(F9f)hyLOe#~C`e7j*r?}T634Ah-{KoIH&brFcq@^#ud`a4535niT@Y_AWak|cWm zGDlY*9Cl`~dq$rMiJ#A3sLxnjxR^Xc0Rjh83ZVUcDrZ=(@j{V~ji_@xz&okn9tB!m zbT7O8-pxi9EHlqyFjCU3rZPOp_?B++296AWV{aw#;UVT+MnfMRFcIf3^pR1Mxrq*` zzf24CM^Ur7sanaT1mAs@7U}6No&n6%g+FcFqP?=q3^!Q?1RDo3sB~@B^mP=KA8^H@ z7{kzYE@9+=9IZ3%x)e{qk)qil;Y-t$04H&(D}5Gux5+_!@k{(n%MFu*rG~T#@tg5W zP;B#}{CsWSN1ek;XU zo*8l(ktEcudtL)~cE7bQT4Q@w3I%Ts*W&$yn#dy3Y7Q=3P=uw<>4}otbVB&U)uI-* zh*r}s>?8A7`yIYAIDY_soNaA)hVN6@ZKp(IvXQZ|1+R<|QPLQdvRrJD7mx^ai)KLF z656mUzC%G~zG&8x+ngnc%BVL2Zz>!=E7sy%yeaLYBHh4{12~1*_zAqZzH*+pd`(X~ zA`r~+jr^Rl-1oxYC!`zY{}xT=-YVF<1c(VR^Y!%Yp1C=Sa!S_CxIR`c4u&P*Ku@Rq z4AUar3sNdk9HS`o+KiJECDA%Bu20L{;-)1_?Fpg8WBRto5^o%jmUk{Kv_AP1QK(p5 zvmqs+_To8iPdZd#g8P=X$s&n5=S~`wQuZ-lriZ)rsDPZQ4wd86FdDC7+8 z1biQzEYN}4Mi~@a*_{#1*DAK&t7?n)?>JxX-_$HW> zM3(S)7t;ljrg6t?jL}XXy<`U+*D&wF8J07e$2XD*@WylK_(Tx7H11XS0OJ)4s;wVo z^1RoEfSg3!<-bU$Ho&wBD&T{IyUORhL&Jo38@KJN!b(Z8*yVLA5$^&3GqAT`^pd!uZRcBJAul;pKKV6BQ)F~uE)Ea%@*77dvM$)=eZMd~SU~k2G zjX^tp8*2W8N%uoj6p*#>@VjiiCkeS_>J6Ywx9%kS#=cQTatu+nDyRo~1W)vtYU^kC z?z0A$98A79*(RHnluq<5L1+-EPhB%<){$N`X&xDW8+m5#Qea<`Lv=}3vORrpAjR5F z*oc1;;nhock^}xRtH=-4bQe!18{J8$JyS4-GJ%qGt*1Ukv?}e`?e(MW83p;ELgPI_ zU*3e^*HO%M3))Qd$oc@bECNS}`HMVpS*0%6SH?g}rt7MfSN-rXR0GU{Ce@=&PK3Fy zuc?fQ-NrDSx=Q=A#BDP`A0(uH~bC z=Kn+%UBWBu2BY{?OnIs`Z6%jBKcRRCp#ng6z%uN;E?oi;d#hJ+tHbMY4X~)Gtaim0 z%p5JUMRO(q@>v2C{hHn@`$U$d*C}Cmc+I7-EML8`AJrz01phWR)9b!Dfa_5#zIKV$Z5PzVk*_SAX*eDp7n;#%icL1P8jM_z z9$qzz5KMUQBlmb#)?n*QvE7C#)6js{F;iSinRev&11&!F-7&pT>3b-IHlb0)zt*p4 z#7MiB0?YBYW|PK7bD?DZXkmF`#LD{_T7`N8RF;aHf)OJ1)`#Q2;{oChfTeV4c~jP@ zL^E=jXm&Bj{bPPOzPb}BvcW?)1WYsO+L(0F(4*oh-!$FDF%_wOuBB8_FLyj(gGVr< z=#Ayp-Qo757yGpk(d&v==TiUY?)KzMvb)tA(fExK`Ig4Wvmn9Ee(m?;C!Q#VumCIG z1`KUYS9A{myuxtYeMd7P77?;#;~yWo3C!sFW+Jy`u;d0rPzRB=&V;fvgR3R})!1>p z=#~p1qL(?4Mb)CQz@9I~jxL_dFw9!NW}cPFcWys6v)@Ylkp>|`!nvSn&I?D%RT|Zl zg4y|F(+B{0wVWK$s(hw<=~IM6RTvpoR6>aDjjm>HX!kqu1vR$rH-Ypsbz}$d+KIp~Whwxu6mI1Sq6GRCDZ zV_YkPPg&B1T)HgHXyH1K#;vT{DB3*cA-Ua1BZB^(3mLgjFxBFX@iRxoPnRpa?cNR< ze_S(LbXz0}R}>rOqpE$6mAuI-pUC0EmacfWZLWIOlSi%RQS|hXwMepjM`NktH{l$d zJB->m&lM;H3J2^Tx2$fd)Lz(&8muJA;^Eo~+Y;2ACU&7sv9u);B=1RMX&?ZR?@*PC z@#eI+A+*s8lW0%mdQ#5t%VjqWtE=Jq;xxkMS8o#A&^<3N79e{LuqwcNM74SoxHU@g zZhEfP6$NIMIF2nw!2fm4ku0pV&4}I6ZM<8)j$U5Z3M)~#YJb^K!t(HmGb0gEGZE%a zA+vmrt%>`MGI|-B%ugqWfpMWb$zSJ!bs}!;z0hyd9}u?g*N$u7nMtPhJ4ElpDWrwI zeu~cOj3i^88NEeWfEp<Uuqjx&S7BWBxR;F+QVy!Ef4J8Qh}y^m zrw6xNWV*bL5NGM*>C3eMbEym{op8q5JDa?Wm{kZB2skxf$_$@A+ny)|AtAgNh^Uo1 z*Uf1I!7stdgbk5`#^cx3H-(~RZwSSC*a8;@SVI3+E$7TJ%_K|on*lsy8`XqG&yN~2 zO2F6Tc7KuH>8{P&9OrE{T*dX9-GK*DgQFC-*I!gEyg#veR8SaRU`92rX@f2 znqd}&S`=(8y*~C5ucxZk`1{h=!Vr}ZU{90*C~nMkY%usMI3>=vv$yGb*2K`{AzN58 zn^P7Y;)L>tBhLtD2l+kZjk2PE2lz&-x1}@X_cy+r_-7sN9UHVj+tRM58@{_@}1;x<<+h~Mis%MJ%^7eh?P>LD6&}t0JRlDs=+8D?s^IEEdryVQi zc=6pRRDre7QT9ljKPFtXeI1PqdTrP#G|a)co^s>z_Onk*59qLg8$m#v2|!vIAdeWm zzt>@){fHF(eO+F&7dgGn9n#!@hb&AeF(1wanfx|}4m7MI8EZ=Us+Ogb=E?pkT?E88 zywOw}h}baUHo7*Fi0Hq;q`1~)daihTeFqAOy5i(LtXNYlKsI+Z1H>->I8BU%E1Mg; zg+|@c$>BiDm2=ho)s&gKO)bC>m)6O26O_$x`H_Q$p`@(k9NaXV0Bo!K(KeujaPPS(Y}_ zS|oNRZ<^a`SJa0vUT9au1%+HQz$sAh3WuXI_W_PMUi7pGvViU;(*_|jlZf_=h4AT6 zxo~Zamtb2y`Su2F=vJy(_=rBe7rhro9ju(Iy8M8tv;LQ2R5kI>OXm*koAKiRMKk^R z%5pHe;{}=3Zm=@FQ;yseE^9{S8DcivX$`hxmPgCQEX9P{h@PyByeC($rhN$?xQ*?6 zT2YHS*ylQ);eXV2oDcc*kE+S4nN=x&= zo?Pf9c-W_IocbE@GtB&N*L+>@sFg8p0c!31b<%IR^oa^E_?fP&_vz^ApA#$0yHu+| zvLukz;K>^yBf51e;j4`v(0EAEtf20Op=V1Z{Cj(ZBc6LW9VmT^*AAL!O)^+M$ty2J zv?pL1VA$T;O+$}{_&@}0*Xz7odG9GZ9FQfhX zomRc#_XNC%@yQ%fFoTKv!5qS%lVGwP5dDVFnBAZv<%vdCp;MsDh9l7(G*hTrBtqO0 zv9gMvT&U>|IgI+UP}lA$6j*XRv^l)9!!42cTc)liZlH6O;=iJ=ajzUv;XLK6o~|`> zKI2C-xMPBqUcU4^Ft+Qh%i=WSd{Q1!3uN8YVSIZ>Q=@X=u4G-}h>53C_X%6MT>POi zgJnZVjhFi5D91OPM%d>hyd*&!Dr$ zE?7wQhdITbh3y5?i5Zg*CocRD-o<7sMa|lV1Ucvgtj@d*2%5<`qBBpBKZ1heeoB{y zZaENw=3$P%HHkn|j}N5h`>=eMT)*>aqAN5`kcx(~i~Qsc6Fa<(>{*bI@)!|Zq4e3a zE9-08RIY0caT6(EAcKYeOJ0388tv|xOo+~lEVKgQ95Qi>{p}@FytRAB=(NNOB>Sb& z^#y3wWwT8YJL?OFpIgoF*Tzghfs~hAl?UkNLQcSUlOtyqd?Fyzj$wkkY8$N={r5V6 zR_3s!FkAvL-!N+Rd|p!4(66Fn9`hdv+%6jGr_zBG_j0#J zN&W?VScu1NPMG=a1kv}J$0w~qj}dO1s)H0JlJk^P2fpOv&-rNnIB-h^-TGV+#tS~a z7W6};+V7hv>}nZyL)})~J)BCId?V!%q_gRusZ>Cz=(59bYYb#}>9+!H-!Bc@qJ<^} zCWMA-yT>N9#^%P^#|)1gO+3x-*pZx(aV@#X_Di=qU8QTLA*{22iYzc2kp-}i{vi_36pLjpyTz6q>bQ0jXNSP=@^$1XJb^?Ub z`xWAW+{_Q(2jmNv0(=x?BA#W@`vLd6fvO{yXAZB z)k=0NvG(j%#z(Gs@1ztkPWhqV>`f9?ex=lM|#; zz7A4y2hNQs=uY>Zo$^=-!NrF=2PUO|ZTh4uE6J+*V0>0g4K9FT?TkF^#?zycnJP?p zo~8Bl37g(-h_thmg$peYuq{zCSLuU<$Fo zGcNTQW`C#0;h23pGtw`Remdk&CRVT()8CgKO~mf^c30@xvn=H3ACBrw_XAYn1}M@V z=5SFas4f2{86;E*CM#A>j7dLl-~r&tA!Ly0DBfY=tujfocG+3Aa@OF^^mj^V#r_#R z+SDOCkznvjalgb*o9MCG4ilsS&P*w}g4!ha^S~?HlxU-bFGF!YG$UjuQDe1HciE`b zMY8zV%L~aE8V+--ffqhVfNBmfOv6_9y6u)6ekPmlGzWsrndMW?A)uYCSaRX$`P%o6 zdy~ymg=k>2Dzu`4@Ec594a6=emHvz$Wxwwi-QJOc(S}4)r(d1)4MjNwpCX- z<>ZtmI{@dCF$zFPM7fcwh3FzW@S5WZPu=i)(YGPS4&<_dL%&#o`uPlD;P zquM;odp*{h;As9cOQ0B@K`Zhy-8|RmO*l=4?EX=Y#%bCdX3D=^t6qQ3XxVE%Ldqs_ z4K0YYeacLHxTv{h$!f+K|n%u$$FVs#AaGSfmUX zy`JLll9U-nJ67=S;6TZi-M<{(l3$ciU%JiJPN@dK*jl&k@aJd5?PuTJyU>9g!jYNR zLh?&jIS^gSnVAd6r{mSv)%O}sp6T!M)d6ep=0hE}=#7N=-3~jo^7YsBnDXn+N-rwi z9+7;KVp-kCt7m#z8iNxNMg{CYO~O)gW?&~Qe}nU?#V)w1w&|T;WbF?S zhn$-3p8i!yv$(?8E<)IMuttNnC$xDt6(>Tyh~3BIH$Wc*Ke??yFBYYT=>s zsC>>^o?3Q3vwdg8?S*{^)S7+SGcM6UcVOWO-kle`GE0;%kmYe@)Ef-CZ#3085N-Hn z#s9&FpeCl9iTmPKO#`s<5tB1IX{eW%O#T!dRV+;_V2Lp^-W&9Aax4J@2y1V*#ACT)n3k_j!!z1`=__ky!3uWT*NkDB<0}pIv5wzf>P6{4`V!9cC0F z3LY`XYKee`hkd7}kFLgP$W$wQpA`nYNe3i1?%WcE;Jt|@%L4XUN7lcZwp}*Z6RDaT zON7yVqFy$_s%Aphb1!prvy6Y|<~&v_L@i%UYnNbFKcqEL^gv~v z^;Fa6oT<9d5D%YWWZ7CE%q6^fd#|0vtdGZX_Lv7L0Q2zGW3Iv_mgJHCtgcExvXRS! zD>wLu)Gy_o)8)tbya=rNsA9#X^E#ZO`kwi86-c`)P7lSz=%x-des!}ufd$X4k=&(m zZnK=9&tt;S&+P+7Daq>1(qDHbsl~;nPw=;e>pN_%vzeRAtqlO|tU@;)IyaPj(IXaU zX6F00t+ANU<|9nbd-Z+aCQb^K%?%A-#I6n92*?KHW|bM=3n3Tkxt{zbuDCqrSjDA< zp;apEc7*J%YZZc@L;Vh;;MdW%fm$CO#6RWI=%~K64@c5Mz)n%T0^c5Z)|HfBM7}zD zuUoE&6Wd6g#h#zW(o7#(-#y;HoNQkui&)tWt-j*$b#5DdCO8_)_!(oFuG+ZTm5>t* zPVyIj%B8zj+5d&E@9;fD-~}d|L+)wk;~K^xeuhGpeDA&-q?)v0VkYYKa3}&d8-ZJ1 zu4Ww}3gYypo!L6CL;b7l1Hnx7;m2FqAJz=SZmK)s0RP770Iw<}TB$HE<^-?y=|#~a z^@k3fzvjZDlTZV?X8j5(IzfAGeH4s4Zd&GVy>e;tXxLo50qY&~PVj)TA?Wo{)(9yHa3inM zuOHI$poKB`*g;5P6QCv0*+;d7#h;_0qvxME)q+p;y^2F<8cgUf?Ngfcz%`^M{^D{! zxnx~vf5u0HDe_h0Cz?I!_xMM4jHgwI`Ft>UezV?3$xt zirJ0@YX9U-)2F7E=pynilY`IrbMLHNG@h!(70%ws;NH8A&e}~gYrT0cn#H}L^I$?l z;&+g&DVPE)eo?T@_QCa6_;I@cJ!csIAK?*#pq)s3x# z^eyDEklkRhVTQ2Z^>7!Y;d`f23$Q0iSzsFCQoX};;BGW><_OrtwGx0pqt zk4GP}u6=r~Q)P?-FcxpuG&gVB8$UDXvJqog@HOMs*dJ9PDu%k2$t)*r?dtBOd*k!E zdNDz&+8W|lzX-1Z~DE`q5Y2J5UDgo5K}e=P)W=H-oH!o&ulP3t5`wcnAgaYYO8!DbZg~8f&T0R zo6YTr+x+$RYH>B;LeppDs;*i9?TTdE!P-Towj7AT3MD!06V)9ut6nJDar3X4Q?sY? zBH${l+gxF~fxsa#vrB`RK_NbCin3U$1#9V<9q>aun^7f-HRjCsb8OaNH8?`MJ z=(q9l{385){J?WN1~6*6U)=1j>Nngk3kZN-J{7CtL4kL;-Y3*Mbe;)UgZf0!nsk)l zyR&l2pR`NAX?{~%z;tG6CRy-0-;hI8XLJ{uwcR={FE9LHYCT=K(0IpV>h~?bni#Ge z`fcXO*{Q$21V4GV;2~=pi{gIS{ACNO{XRAJap!KkT$&KM7zj3iLSHe_Sh6H~GP%CEumr5=gP?qodl7)eJ^D_SfENC~D$W;1xNG(r{eE7m&Pj@RukHt90IB7~+z$j$CkodI3ma>*~QsiCKAMB9d*r;dGWnV!@8 z86LBTXI~(=-c*o9aMHwv{-w*VH6MFejZ+UZe9IFW}VlY4D^syB>VX zy-c)P0DNYJu#26VSnNN0WgV@fH^}lNw;{d4R^qND>@a!J;k)GTdd@cEq6a^Qf7Ja- zvg@tycDCs5w!(8YrOxSK-#3}|TBsUif*Qd#GBl4euxHcikJy5i-o9P84t0O~kX5gW zsywyZA>+jYe_6{K@rZ1H-)nbh$EyH?r*1<(wx)C*A-1S90yW+m%oG#b?%y)OgH(ED zlV{I7tBtut4ISOKCv@}oEd%3&yiv202fL3Bz9KK1^lI&Y5sT}xl(Ezj)6z+OK^rYl zzViTYf@hEYWPLr6Br?|MZfm&j^D@qHg0!h(0QSGLnc2q8ZF-K0g3egVd--dPfywCi z{NnXs`pdEARekow)6E*zH&fTz=~giwKv>z?$q;Jn3P)Lo)RgMl2_u9HvhHf1oi_LV z6fpH8j^9xL&fqu=`{Y^gbWzE%+Tx7_CyiLD(@rMCt9?EiRCJzzns9G=$BEBnRvlm$ z0DgFQp4}@^3*7kIYjSGY>j}T@ZLG-Z;CDZ}I$(iVFLJ2Wgnke=amrqUi9O$%7Jxhs z7Op9uo;CwsJ-+<#+=kYykM0^a&?9;%#~B6ubG$_*I67rksC-qAU=j6PDcM^q0JOsJhMxxG^gHHL=}BR$D*HDc2CtnP(=HU@ zF_vHLRo8ryhOE1*7wR6&$j6Iu#glD14QB$%t5deyZcSZMapUL24P<-7D~|RM;BL5h zEwA8jXIlK0fI+Cgu98{tpQKWNbNyIz&07{_J=8%}_Htp+Df-gAhkNVZMKfRqX2;bC zI(nMLw!}vt|>oopF=7-g-fYX1CQkR!r_|_4LC>{ED+FV z_JLM)dkTeE5??>lp4F}7$FeDx6>%?F+%^ioy^Z~gv*h>z>(hO4((cjn@J{5MFZQEV zQ(U&s($?i|e=LXE4rLtjC_bp%^jNi=Wm1)#uv_|r755nh5;eYHO);eVny9?TgVAuy zK||Se4XT-bWj#%`NMy=YQId=E3+p8pCoh>Bd=)*7N> z9*K%L_(aP!bk^I0+5}3*1vPMQeP6i_+W*_jI`Pif-F@TT8;8DA9NNd@kN>czU^wBY zwfN3Ay9;Xv@3U3i&YZ$%Xl}J<{6*^23=dj3^ez}|ygzH#9U)GqCyd)JzgwT9nW%9z z91RJ0<`+T;7uv1sK^9P45J{3M4&faV1 z+IiZ^U-$f<0zYf5xVCq&zHE+dED4v}Dhbn2wtFZDM`v%hh^rz^1P<>glGw z{MJDr!c=hT^6tYvzYG(fZ|}KMXJl~?9P=uA-O)+#%gf_e3U7(G5S1stHNc+x7F~o% zDCsYkIwNM=3E!t$p5~ig%FP`qON`gyNYtZ_ZONyPr_4@%TsJt^r5$JjV*)XyWI(C9 zOXVjc`uk|=J>{Yxz3MyA2d}Zi)8SI~C%189=f0GwarUb~!isOj)pMp)ow(q$gU3Cg z^sP3POAI7!0Pm{m*xYgNgiZP-m62P$uZCZ5n+ax09Je6ie0hTJ%=U_U5Fy3UVxp?BrTeoLtjF+JEb z6#C1u!11xyZS>GbweE_A?%%`LP|Of%!j?WepNNn1f4#*j0$t>Oc%`Mud-%X{x8eYc zU#EK=i}0BJ3|B5vb9v~ZNiBX$pz>Rz-UJ=fW(FJX#tx|}6H=6Web*a4Nfe;0uC_bm zuQh2u_Vw!qr0f|--2pA?UOWo?k1TGvX^xYmuDt=|F`b>y_<_ z;a8T$8XEY0nwm}0oRR#!sfpn|q`n^h3ER3mrRJA|_ikNkMz0ktIF#+_Q(8%1kv<`? ze98>r*lVO^$pPc3zA~9dnZ3K3;o`)C^l#k#W&hhs<|+r)%>6Z^Pe=cFMFq`)NWZ`) zT6{6vf&LEO{qy6jCsl1%qy4eY%-YXISpZ|SEBg!MtM71UbN#lR@skkY(iBlLI9qC_v3=RCrBX$hpCO>2VsKuMU z7Um^0+6gtqU*ja<5Mq{!P@&M8Uj9`3uO{4 z4xY^Lv%WEIXD**u<>x?=Fec}aYGV0 z;$HeZ9jZu)){ho<3Zy!QAcC{E^=^%RF<)nwaBYmnyT#N6Ie#yPlJ{HrJt|0tVNUej zRbv8UNix4qe}9Xc!}{LgAe+_f)mJ2)-0+pAALkMN$A>?TR;sD6Q<31lroO?ZCkL}O zT$Cq$s7cq~(sxqtbsMQr29?OgQjB>z?Wq+(w~YVk_PRBL%_@gu(QfJP6f>VU zUzX*N7hTpGF-eu{r-5?hs@sBogp;${SYt@F#Or*N1weRWeL_;~t6WOfi>YV>brLnb zdwyqnX-XZTd7S9TzAQ@bdHToCZ^tQY?Gj|30AIfz8)j^wV7d?PrJfIOR!O)mJ=*0o z#9VW=Y?oUOJ+2QE=lo&Ok(tiKwpy3NhE zX9r&bBo|ur*l^QBlwxJoHG4iT#rgz+i{>7IXrVq7$%Pv?2NR(C$j1u z(ey7*daob4J~sXM?Q%U&wvkcc{iR@-b8L9kow?@Glf-X-?{fqYUjYFc;kf~6|H44W z>~#(Z}r5$yz}un(E(z2yI<@Pg?am=YZGgJmg&Rs@T4;`Vq5Te0}sr!*^BqL1|N zO=8;D{}_&o7LphxKXeJp&(#lIiqIAHl6^9oSd4cDVh_(2cb`=z%QVe^yZZgMgo^_X z&gic<9@?@7Ihlp--C@UaLw{_E+w1STM67#S_egexiXUCVsm?>;XMzs}Q-~juAl-mE zUd)o>)+EzmnWr=cb>YQ@yxco|Sn5BovFTwV;G=6aX1j@n-2MU0uzBBd+a^r=QgpZ8 z0azogF-DIPmC>xXFg{OFnWU+#ZtOM$2k9U0s#=T_7+bjR{_K#5L)c3DHoGLmyGEg# zA6UP&Ti`-ZQob5?r!E+P|D)x=dtpMPoje0O>e%(lvnA%~*E8jlXT%q3fFZ7-zJ$*d zjS7c@bB}ws?G&bMa#KZQVwaDVT$|(U3s$p0zRdLRa|PLYd>$={!pb$KB+#Sf6CjyG z3*`rswzVBCpk9ISKImRNm#mW#C@%+>7Avqs{w&*k2^145u;F(-DtU3}t#haWatJ8r z{IllpSr5TYBUgOrw+?bCRll;To16y<_G{IgJGEM_jS~p1{_@Y-7RDxRxUp4AK}?K9 z?$RFP#|Y`~7bkCN1#o{EmXJae69rl2%26!K}S=>5O9MAxc;!Bx`?4gTtU z^zpKq%3gYT>g0GsdD_78`+pTQjI1yd-)i)a{?P;AI7zcXA08(401ekJzO?}awX$2~ zA#-*e{MEzcq460!3qA+(Dix+G!o53YBa7-=Mi^Y^?1uB(^aq3v+H?u&xt#wn%+^B+ zrar&%5XhjAJs&1H_GKVMew44GvI`7Yg}mE{YPUW^1NvMVDf%Uv78n{T>PM*I=8$vg z&!%sAGk*5&FP*jkD0n$O4(FE{pW*dczZ^ATCsHI#cu8xo^ zhL4kPna(Z;wec$m1y??uHB_H=)S3;$1;%1Mda>{4jYJ-+ZPHxgG%fJk=`#CO#i~+d zhsjGZ>!Stmx>5tK8sUQ-%IF^zqx%1Rrzrl$4v_e@cz=R;zV#_j`@pHzZIW5T{rwwl z1?~-p7mlny5z;$4X>{k9=C)l>6SY4R`Gfmv48O(o%d;JCj)X;UGNHwq4qcwH>;^)a zMV{s%Ud!>2_i(ZfK$6Cea&$Jubpjo41F2%i@0niLBOVWJ`0Cp}*K%_4#MY{qrFyt- zCrWVAs@vU44b7D#w{$VT*~}C#WmdoNCZMi8zh^uPvZHsmO3!P}qO`zU+E29^qY>sQ$))wzZx}Sy9zzi=bS72L-B6?+J-Qtl!vOQ^yePsGC zTG}`JggL|_Y(RI*tlnY6Z3wH;HjsLw32fMoK>9`|!rzG*OrzJ%vT|vQO-!}mX3S9v zW+wID443zo_Z0*6Y{L|GFr;5Gqft){eTsj=>lxO`e>S;OX6?OhHJp)2>gUVQ38ry# zq#oAvyZd|GZnCEpUTsgxM)oZS$5TDc9;1hU>Yoxx zYVEx9k28aDf9%9}(zx47ekZZ*v)5Ucj6#?sH`-;O3$%QDyk8t*@fbBKsmz|Dnh<~D z9{S>!h6mZ>F}VyG{p!DL)1#*r6{RFL~{OL^V(#nEf*>U#Ff^uef9GS^cK?L4D_K$0AMgNqxlA{Tv6KFmq~7 zUgHHzC5Z%YJDPc26gO+aI%1rtJ(L-wk>`3wb9$x#wS6l)Mey+j+5epXOZ$=ZQlT@@ zV5dOXWnScTsc%AqDmS+s!GJn9w`f}oy^$qSIVkmp*ePr_{aEM3T&!kLW` zxG_(wYiWQrJuSpy5JKEh+>A|UtG#UjndkS2 z03v*&1MpRefo3&A}8o z^L*rXJ^>8^^bAFKZALwtb06V&nU-+HkJSgY?>A_D)-ezf^r~}onoldJdGvbAK(E_N z7n!!C?iJx~Co*;L3@m`8kFakuMzzIj+~(_86{)&lBG@{}ETpUq1dSMR0$2UHms zu1m{<_D~;5u*#k6i2s7Tiw=7>>ef>E3Xdv5mJf=&r% zrqV9-9C~C5_L83+Ej4pBY7!T_y%&U%r>RwZVL&GFKg+F#=MK*2_V;d(7Ag+EYi6WD zdU5JMNrU0M{LHklgG&+ze43CRdEZTPWE(z9vsJ0@MLJ1hwvHVxI| zd@ufJ?SvG{9`DHAAWHh;IV{oQ9qm^q`+9l_!Jc&ipd3n~>e+tw&?zg65}$Ei1)TU<9u@vFkL*%M-{KHBY}Am0YRvrn5J>xUPq@IckX(XH2YhfQ zNO<*WwSR{Q^{Rstmd^+KBT}(}g5Ms$VmJ?H=XG+=-GeK@KDg4t8W3_4HgQ8xM+$~# zX$fcb@o5E{*-F})Q;&j!a={O*`m1@(XO67n&4sNgZy%dYsGFf3`S!WJibdKVN&-Ia z%nWoI(_^}qQ;TwfEU!P2E1I0FGEN>R>r`sK)ergCFTt;}{m{pq7^lx5CCbpgI00<5 zEVj+iDUK$Aw#i-qs6YHL5k5cthmBQrc%j&%QPkr4`^>s8kdWIu(z-fYe^K3gE%J%~ z&jOHkHk%p>FY=WVd!dzPo;M)#d`!2-Ii?osFn}|5Q+mhUKJw+EZAtRunRjdEnCoZ& zfeb{fzpvgryFnDv-;51Fv||{xi*wvP4SJ_mVs;4uegw8#dNJ(Gg`G~OM4hk2U6+8m zA1M|^iz6`>vl!G`XmQ!|;3p-)f}4g7zZ5Nn@{782?|{4)o-*HW%v6*Mb?$xMj9uw= zBAi|S@%W=l;I*|W!7oh}i>YQx39pan(`V2@eHR-fxL+-x7}k#JqrYpVU4<<(+P}gD z!ozN779P-{>RWoK!?!H|xWi>aEV7Dm0k={K0Gn;tn+0sv$ZbA1z-Cf5gwHos`B%4u zPmEx+q?C@z%2M()3)NLUHduouhpo^HXOZBr6DTa&=n_&qArWIt9f0w4JmCC7o!;{N zq>6uY^UOmLW{uE~u2a^}KtDPsDm{M`jsFG)-@lawI0g;i2w~y7M=AqAcjvEqqA9)7jV5*0FDrf(b3T zPD%@5t($Y;BgMrB=l9>?Zt`A`xX)g)v#IDqM92BqD026^4HigmEF$W?W0LEi0gJ}( zyodTPbz*XaqIap87S~T0H3UZb+JU031&n@^Hi2g%2O|@G6zlGBKi+v*?d(V^h>;cE zzj|dd3L~f1{M_~P_Y9J|rxbZGP-%(A?$#0h5LL%^C79cH6`~ffltruHP}j(LWLX#O%MclQ=xG+L%k<&;#s@FUCC7XVaZ0L-tv zbh1(k72#8Ugc>XXYTa7OwLEWf3G{evO;DI@rAI;~hVy;redW_BIk;ToCtU*F;tzs- z+w@&`V(a}5Ya!W#>h9Ts?VJIe`d~|e>=?5pN&bV0C!8g&6wLa;$Nyl9L5jgpra@d06w?{Lr*fR{-ou!Q5l& zAm-SOI@;rZPXB$Mi(q2GnPE>s4Q;}2}9%dQ4;5U@w$t7@WqGDpWL z1p?NS+W;_e#%WU-Qsdsw0!KsByOaceYy-RnqL^uWdg^b19^r70X(tlGGNFFX`7mB) zr8V9d4mEA8o7R@AWhOZUnNCRyTy81&FW=Taiiw4rDh^)~y9DYYF-U_c@rNF@#-KC5 zyL*-$ithf?Xf)PUnxAQLBii!b6ZKvL>?FDj&RUo`#?ADt%oo*B`(Pc^#IdYJ5COU| zSAGZh@^KDg^of7KN0I3tAgvEN(Sg?Da6yx(a`8)58swSd?1qGRNlTV31%1p`f}#FD z!_UKGKaYLw7>TQXZdPMit@3(3EUr70(bw7rliVe3$bxC@z>)xDwRgzxA=D#d^B5o!cOePM$fo?3LIx~rk?&$Eq-;Fp zMBc_tD#41^df;)lYhpBGq@1&-Hr3!l_FzqNUiSp|PfHnsVYf)4kTIi-M!00)uUP=4 z9-wIVEYh%Y3d`Zog6?W5z?a9|W?wt^B?~(b$NdT~;o*hvf9@51aVti}>Sghq(c#+N zJ2uzrX3&|80sM7Kf{VAKkn5JO6vJZ{tH(|-L~|0qP1df)e=cig70U_b%H}U79_ke= zS~Ug#V=BD@z7e+ioYDKKBIvP4*>;DMLHQ&nG;|mRB(B#f`-S*Q+=|~XTy646l$`ot zz>jnV&v z;~WkkB0=fAl|?(8PFnp)x#}3rae^n!ACZ6bdw0_Anldd%ZLblhgg+YR=5Qm9PL>^; zgT)1Uu#Ns)-}zA`VkxC81XJ7OVrxhy8s_Ap)Ku%k&GAJ!MN)-M{$$V&+PKe~k@# zmxMYC0U1}mgGB;jvD`dcOE#aJDPs2y3=}e*R*!$ZYYGpQ(*iB)sM}aO;)!an7e|kd zy!r`4>^oK#FUMdyQ$n|b7y`oUL(CyoLKiWy_tTrjPl@M%uP9!kaz3%ao0UQ6ty4HJ zbv2uVx40+TOFU?Qc;1;TaEhlO&azqMwt8FsyL!|m2heM&7cM8~*07Pimpi3MT^ih%a&h7}&%ag_C#jwvMx(oaf5!Q0lsGx*-JK zF3QDx^KG2qoYS&3PBzf!)Y`qvHcpNJI*XnCJB9rDPusZROPF|?0_aS0JWlmicpS=` z`k284rUQTBi+H!~yp}~Y{3a=bkxO=xZq(+KcYR{Ijky&GrBDNMTxsaj7g4_>sOGCC#H*x` zSeLr4G1e@Wki{JxBeK;@_e{3mG3I@xE?CnZj_U@vVf$|Fa4nshWt3puM#lZn=y#E* z{O-XrJ5O$v3LW|lr`M-_+unNn)+4B^GwjULv>Ik6qxrWUW&paywSh$q0KUZpFa6kf zrMwDY8gl1vp^b1-!w`gm^3AoAe8*6!ri|sU-g}EZ8=Bh|%n)8D@MmU3-6t8uSMMch zx#>38nJuZC*sR%Mxodu|atf{jHcPIV;_g3%MS?nVWSS@eF2IUCxYlP-lycu;GVZ8k za^bGmikU6CfNvO-dvF~WrkMZL6LA);3p+^d5-3>yndywE>?_i*b_-vg5a-$4+>7#9 z>E86)+U)80!+9+6y!|YQ!t06vCqvhJU`?o%1k)&P&v zj^lxCExlz28`{x&1Q?_QO_)1EPD;nna;!viiSgpJrcKQEqQtVAdn&Ko_M}mJJSnX2 z7)PHy?jw!U*%*1aib~ML-8k6rKRdbHpFP)N145awRBNKQdIDo6LBuse6r=~R-(iTiSN zJTFeOmY%DVklmGxkCsLWa>Pyu<*4Cml*==AueW<3wV zcz7|Fy}HCfm z!-x?5+42@xP-Ba=m&o@O9D(k(blMUQlPmR~H7ak%-;NxCtE|q*ZV1I?;x{Vl*PsUM-%Bec$=z!dI>KXYeL#XBcQ=Z=0Ar^RXHDv&9?XmCv zi;m`LN~gPKCE&TUHCEj4$Qvlxm^h0B-Y*Qvr?qJ&9YTF>iEbI6Vg5RJbnADg0Mp4m zHo(IPf-Md7TUqKtkDTR!A6Iu9D0#GtgfOnErY`py#A~wkWIv}b zgT9z5&KmJNH_QFmsFP%Ezh9*{kjk-Fgf3G1qe74IPui(|xQ<2NSx!YLwMA)_=GUHq zBCj#qXD>nyqGH4YDw6*)?*-;aVShJZ=T*!~Q`s#y`O_+cmq+u+njfu`TXn@m20u%E z)rdL{?Ljwg!1sSo?Glh~OPzuL4p#2sJ1+87=j_)xbOw)%yt=Yy3{o|T5}fL)a5n3D z1oG;BO*}LC>H-vd^^MjJyQ@Jn+I$p%bB&)y{fjt>q~5m7(@X9iuC8iH63?x7PM_>( zemSgWnOVgT7PR^8t25Oxr`|yj zi@vPPwHchPy)R4&`VShmR{LLMhE6>1fxaAIIC%kLB9+om^n%hvuL{0lWpO4;GE+j@?wO|aZX$mQQVA1ZGSz34`UjUFj$PTl?X}}n>6fG=KcF$b zGMP5k{ujXOkq^l_H^Kb4dq}t1D<33FA669!^C8N(f4NmAR$C=_9lQMw)Yi_q)0)T3 zCrEvpnv?I+Cd{G`^#g3rJ(qc0N$P@IJSPIT#W(P8rK9e-t+gi$7bk$tnX^qE1%Y1# z{;k|)Q=fOf7w7lsgW+{W6)+gDdU-n8;=1zQP_;*LIV<2j4ShsvlX!(Ps17T#U^qW&IyK2{5J1> z!79uzDh1eK&{gLB12OY7$!YWYLIa%2K9I4ZVZ70K4jjl3Qrs$Vc{_!rAJ7sZHfGd; z1@X;f{|E51dLOG=i@M8@V-ly1F6yPl;muO2X{MMrWj`i!hB)Kv+8b?SZ$Y-jGgVyo z^`jPGw<`6#R@*#4S=)M1$h5g`p-Pm6~+B=t)FAoj$-_0ux zKea{s&uD)}I@;(@pwR5<@l!Yg*Qj2gx-O8xtI@py)6%`W|NE^wqK~5o@bh(1UFWKp8mlJ zQ{SpR95=yn>P~H$`Tw0*uEtA4Ofd49D{!D5g{F779CafIhnb1w&*Z*M5QASs)X9hS zwkX0G~ZaW2<^|k`2&EqE#V}@g6xs zh7x*w!#iOJ-X7Hj0(c?UJ`Jdkvz!|vb&{uN)|*3}v6=-gZI^7)y{q=HEn050HI;yQ z0z30LWS#I~dTv=ZxR(j)=|89TDQ@GrW#C^z8X!D2#zDSThucR{bLXC!LKqqn(=osG z(Q>=M)4L$pj@qrrdd&09-1#-i8Bx;ZS5ydl%RixMUacoVdvSQI*r3=)-`&Q-Gn1K1 zzMrh+ZXZQLck^UKRyA7 zyZtkNm4LBHirN}l=YCmYkljJyHg{9eYBCfR^V92vZy1PYmkrZK5v5KtyKJG#x*v#NZc{ znSzkfIvT(bJAWRKW~#R|pefO!o2rtECF#SrjCXW*U#)Isdw_l zk^jTechXtZ^1=QkA>^#mzH!e~G-=IIy)q*as znsx+ViFcEZuY%gJHs+tjC$=&5u}gm@PYN!^E$0_4g(dr!(i5u|J?MdQB}h+i<84y! z5dOI+FGRbX7x#p7Qm}h5+NOhEu|nx<`TFx12DdSCQaGfn`cP@zNz_Y9@x0^TJ7V%} zR~1I20_pD)V=M?xwfMcf(+unEITOuk`l{@%aE$HK>RZ+7<5dIwRMw;5t!wIk8VHYr zw8$$X8JPrLh(KB!x#X#VF~Vo%odhdGr%|^dowe{rvslYni#Fbto$=y+t_m;Y;0bHD z^@x6A^HkOe%C5yIEQNFZC|^)H4?!37nmFb_T6B%%v`JTl8Ih7%eGh=MhT#e98>sJu zdy$R>I(p=^N$CPMeVee?Pn4gDRFuei)Ou(QJ64bk*$1yP&h+2JTbU@1K)>7y(x^_0TiKSS?(?fU-UCP9<4S>jiq|!m9ISDu z!Sl=!-ofHP(A4U+7619t<(?sA9!;kSZgvF&`yrbc2t$PpE8%H(LfPSx?6q{q{wsm; z54F61YEO+~j-E?q;f>*Xha&vRu>lH*kfh+xcW9jj)+A>g-xB-EUS>v~Fd!Ml0y( zU>QAhf3L_KPV?Ngz5t-RECyEhvk+d7IOMT%-QJX%N zoz>Lt*9+!pJd>-&q!R^Q_WZA4PQ(YWjfH_Fjn3S(Ln{m5pRqh?j2)up`-akq$ z9Zf|#mUC;4-Ua^_S2}2P0BtybmYrwVDZ2BKI}p|+ZUeYdUSJ*L{yqf|`_E%bJ(;EC z!_0+_ev5R_e{=}zB~C!U2({ox8UX_(qVGEgEL?)vT=*=OKFQHE$8uBjg5)pYNIo55 z$-7;aLkQLlB=T1;j7prIuzyT51$>-dg|Aft(w6YhxM0@b#6KS$EBv*7qBuaTjNtMK zL{wXKJ7%=~PLlhdR~|tVlN>m;;c&Nuw^%y|zpt8k2^v0v5v1ICPWkKTi ztg>MMXvbdIzP+ePIK3lrH2n`h5XJnie+RkhN_;ZeXy;rcld(JfAQdH3b03DKfMv|n zvb-E|*Y&lRO-cqx>W+8LF$-&=itb*#|vg>qjL9Kisw6~07?E0#!aIXG$UC^1Z&lP1++N7UF z@$4dHzufWATX-6WMo_fK$^h=Tq`0@LK;r>|fcaxQbHiiAL2<_svn0-)M%7uDCN*$l zMk5d-C*>?D_S~x1^$Y{I9Z7AO`cz`kdTYD0>>+1?MpK}DG96WvUF~V|KoszSk-dDk zun7u~bF2|{jAk2d4F%4`Cjz%*u#ehS5>zakTrp!USt}Kr3p?|Slv>X5ejWoA@*wR1 zY2;sB896)@e}H4Sp!y4oJytLIB-L*Xu+BcFe{ro-aKDwoj5uj$6|co0V7-25`uX=R zMx4CWuci*!Oy%B|XYIu~U|S@d37D=y3PJ4B-WQLX9eZ~io~upL8mw&sCu}|15pal> z`+hF2^utDla8qy7lMuPlJvQ)29Lyp6r@36HH2jEHLX|#XZ? zWP7}}qsRG%eT$fdrO1y2HSjkb&lj{4%$Q4oajx9A*c6Y<4Ea76Bq}ugW z6;u4RsPYcH9l(!NAOmiR;J0wwAVEZK(H|zmbF$6VN#=w_#`!TP_6>3t{EnVPBXK5Z z@lz2Xp=-hDfW6zVtjO2q3a+o3s$}0Sz#?k)fN)PjUUzJ-Nsa~OA>6ELFBO|-{k|d` z5;y@ELu)hRT_)_&j{K{iEHtH)dsHH_&ATj>Yq z$DuC0ddb53*sbQ_Sh?TS52zV~IwrX@BqtM>o$s1)lr8!D-DM+OdC^6#0Nv&Lx6Kacp7Q z2Y~1Hid*VnE`wY?#FFvavqj9)W_>ZoMimbrnl=METF2q`+||ote9KKgwWxU;Ci_JM zCjtt9hU=Y~qk7=aPgR;G2jGUiXA%sYxzTZBYh7K$`g)ys!A_v zIi@iOnb?w$EpG5=6+B2vIQil>815$Pvej|4d$fmhjru7UjpxIvjQu7GlGt9VZ?N{~ z)wCjaOtkV#A}ag_K|quS8l(EW2R(05T48D>Dv#S&XD;2<*M(y2aA%je zY}|D&-B=)VM&m$W9aSf3IJ{d_wUS z#L?NT1fAB7zV}LdGva{FwY#BZ!`PnwM=;RYtzHdt*P^##7Y+Q})?WAF7aRf{bW}{d z-PU!*J#S5)s_`S8Om^uAI6THEy^@M?QGX9Ln;#Z4PIc-EJAdSjFKUD-qDwQBB{A6> z{VS%K*rijDVDCX`9S5VvnB@h@!=)#{;C-d_+ov3QKgF_)v`T%$9M;)@5E>zn5UhD_ zeM=Ip@(^hx@^$Ao47(C`#`JGg$=8Ol60W1CJ*e|6V0uv5M*lCLvxXxZ{CZ!fNDg2$ zFX`Rbm4HomZ;x${eMPy~ecfzBL*f9Tzfp*_Y?}Xp^DY|HyIJwB#^g>zv2({}RBK7s zJp-i3`}dxIUXpcOaBWkd1fxgQGbd?fSe{HvQcW`AD0#|c_(mMOu8dzl@WqTWtCD?` zG;p}0^j#mRT#nA5Z?Y2_&6 zVRDb46$_#67(Ag|>XG6*UH9bd25`Fj#g_M-gt)CTNZr<*%@^=x-GuH-Y87`>m_!+L zIBI_IUV*w=Xw*w@dS=4WjosBN&P&mMTY0fbhPfZ^E9~79f>qn!j!PYg&Wl-e?zCsB z2OAt316KUL>3KAq{vF!`UP&WjEaw#SKaDbL0_t|WvZ0t*Y<=c{zUfZXebqoXN<%ld znyWX=F)W7x`mu_zEazow&ts8Z=U)xAqEStXzG0?z>IN56Qra`)334h zKlW7By>eFTzQ`bu@nzlPe#?#=|MV4Tx@aB&V^-I+*?!1}MdrMe(M|5SV&B8>2vW_o ziTR=ZrfCzn(ilnFJ2qA4-X<7nS)u)|O?BiJdentjz%N4@w`{rDoxJxWE~|$sVg#04 z&wh{XA)o9f->tLFnscD;P=(?AuQe}Ome;)L1n_1~=MlvDt!7^^3wHTGlN;sl3aOYM z_JNpJm4&^nmf#ISe7|}@1B)**S<|m>mg#Yyo z&&p~UE*=~wx3I_aFSum5FQ8z-)`{cb_c8L*^+pbx3Tl3T=Z}=tHJMb*6g~8u@#mPHjsYcgfA~n)URv;~D-Qo{s>3Oh(DFS)hp!jKarqUoQn@#? z^}21Yv6C2;DA?q&E*?AJ<5vIaVw`PtmN7I_yT+sJUiFCbd9AW{B@%rC1^@B8ZWCVL zib+_x0k4&F_mR?Qc%RGADN{EGJYHKo3HmyN8ER)-wAtW*&Wl)?4BIAqY!l~GotG_( z(}UkSEw9|Yqq-ikH(;q3=ucA$An68D%rq`BzoVSjQyx2#jMaM%hb%fqpynIE?6+Fr zE3E~jhrXr%vn4GdCJ@i;khtyRZE`<+9lKoj8?emu0Q4|$2zGzp?Z}90@VZt7Iz<=>>5(!*&e@iw7HW~fo3RiUQ=c`HcQ_hxvi{Xr zTfRF`#3oB~?c_^!;S((+T#V9Eo0&`Fr>@s#z+k?sNrvST4eCud^o}MA+GfWJgA>kK z^dfcCl%r#YJAM@&IVYn*&K@d-_wu)4dj$#k6^k8)npH&lkHW_K$@T1VT+?+=bM)ji z^-pJn&dnWC<%a0#(mV7N8@vYiXxR~R-ax_6N98l-eaTDcrc@@f6Z?5j557i^fghIP zD9P&O%N58s;ROhvNkd;1q2T?JqnMg%l3}j?!on=@?ypYWKjx?_&7v36wz!`S=HoP8 z@8svRv%%It=gJY4@Oj7I=X|q#n(TvGhsnCk4cAj-cx%8fh5QLYq_bo>V#EduK&u1F4E+&FAf`nr?GNIUTHbF0L8 z%c-L>WLDl9(q1r)H=6!~Nx{$D9~I@~02Os7gKG0A8S2k^JvO<5+AXIFGfPZyrMZkp z{`Bn^%H}&jT}$ggFT9ubce%P^0%Y6o8wjq{qvmi-T;qv}MIt0VoT^(qee?LtMCd4a zfEySu8-FElLIP-*p*w3?!Oc2ine$ZSJG>HJ1^N)Oded%UZKr&8vAlqLj#2jqWZ*TG zkh@(6Iy2JtN^znqn`XW*;p#n~m~1T93+h|03jmBc&qqI*7z+0_jdHyNh#9t`%&;c~ z5evsvuh!2`7q0(pDG?cT;?wM#dNnP%C$3YPam57Ea4iRz>Hm1YL1spT&l{T}*_r7p zfo{Z}%_!O|7#!|T>(^V=wrSq91M_4gXRCkSo{ zIUjf+aHdbn>M-y0pWzDa5WM~;-kmURgN23UP+`L~QOf7UfgEn*kj!YSQ=TH4E;h3c zS3iDdZ-qUP;o&e<&9JHmmyX_RDUWK+c<6^{mq#>y&cnzX8A{+4_Yh&^YhpSX!_6y; z6qOdao4LF@8eFXE?4Cyg$z04cQW|~Snt>f0d2@eJ`-=BrS4QKb8_p$}b9ngOZkFNe zT`T?HIk*_g#gw*L=UOm5(=(%!pC8Uuw3A?V!AUK&G0SR?3cv!A`E;TF;57rGUs|2L zzSt|ZM8i$Qof3{b4-auyM7V)Ei*>bWa{XclZvjNGwL4oPPga~Qzd zeVEF7L>+@|#$JHQiMu$I##M8A1sEk^RY_;R^D&}Ubv?l4Z`cIPG*BZT(&q6MNq_32 zq%f#=M{Y)UBlpT^Q8gL0ay5ZN)PLfS>*xCd>N5RR3CzO*`ALajpOJfBNc=Uu%B9zh zttBU91?${dLJh>~)K|tZ|5^7eV^DemI5@(s~Q~q328de}rgvM9cPg>vZ$4<)U z7S1g1kZuC?Z=KATy3;v_&AUfcj$6*f{f0%^?ay)Ji%UQC94ea{Mh!|1)Kt7%MW?&7 zXH?vRyc3{bSU*(JFg?U}9 zk#N|aBvqUC3V6MZRejUKWjGmnK7(Of9j0*R}p?nV-2?{M$Y2qpLS+@Pw=9WAOS?>g<-Gmf=Acv>P(LCgshg#_;== zU~Ua4S4-IxZPfE{+`AWHH9tx$kt>aWY9)RiQ10`y;o*UOg|$&qaBJHydL%?Vd`VUH z^9TAE6608FF`Se(vwZl66Yd>!ERS;Vs+MOQyhoyg{(^i<_fTA5jPpyZdSP3T)!M?k?N%yLC9*iWv*%nX=6F?ht}!k7l~i zE+WBua-AuduHJ*I&$r%2IVUyJ*EVKvSCTfPVpgvCl=S|tx!K&!|AzrDimu`>#`NaHECKf

f!=Jl!ueFIu3AIwSOC2q(xNAvb;kW(mNXl=1 zNy2FtnM1VE2O6S;bKYJqN4F=oMCK1&(80ey#Q8^*(CyscY|Z5j;wwBV3wFoRXDu4_ zAvbAf4xfr5wjEV(i75s4`@adZ)AI2m#H-VnraKLKZZhD(o}pMcR(tGgyuLsi$B^Pe zAvVag)>id+<@D)QP3&L)QL&(2`f&wQV_DW=RLwWcGL-PJwqpxg_CdqaNo=Q27cX&K zYXhHeN*?2e?FeqJl}Q*<=T`$A!Q8~vlP1dZ%PP1BQG$T7@YHjlGe2 zK2Fxh`>Ls8l@aw@pQ3CEUsSP^u>vKce6uGoHy3B_8O$#{hgHIZR=99GJLx>qtIU9T za(kl`rvj#5+%)IPhN*?m`~1C+jb3)TkX0AV>B&~3&98^!jW0#~XI8^bYK=w-j>b22 z+2Ko^^P{aQWz#;4G6(tQtw>waA01~L8eg|KZhEV(n>-+_)bPe1m;&Q0M!t6MUj3)2 z?oBX3J$`ws&suFCvT#xJPqCHxljY=q=INHdG)1vg!9};osWMcdq?-La38Ek@QUH zVd;wAlxoPs?%#Sg|HDw15d%dnO32k8(L+aFioz%tl;d^Qm+fE=>Ja6W@DjLB0086VGu#pmtbfJc)!(m+U3^QEU!m?9XzVb5@-jbKc{5Q{vT`a9o6L4^bH>iaFh}~ zctoT`k4K_{Ll15)~yPB`E#Sdjcs+R79FeQz=mqA|ge43BiDXp@kwHfzW#j zEeR=K?)9$wd+z7`-hZBVeJg(@D|=tFXXctUGkfMYCz7R7Gu{N!_9uyiT2A5ThwE;c-ZgR8-W1S8Z4 zyzaTf4qAWL$(XLmJUvw9p>D=yVC5Z=)O>+0d056o;5DyO4{hr+B@W4grH%diz?QFm z@JrR|HFiVZklhy0AzPD*+!;WU+eom{HQeq)rj>sHptCSvcbBb-ZzIz3g})L&(wUoc(_} z$-Os2I`_PLd~A5}Ii%-~y}Ie3hlZv|(s3P8zTke`J^p^Oms^2v(b%l!g`Rw*>-EGm zm;HXa8?w`NEks*8rTyP@c)3RO%@~gMm7`T1Xg(V2giIBT&1`MK_!4mhYD=PG}vVv6(YZdy1uHf*w~Q0?V*z38B6oPFqbvd^!f-ByX?6$u7$D?;og+-`^g zN@4ZeZ&CZ2@}67}bl-1nMtEd>T#bL+bBHc!qZ$0JZX=jM@eO?{HIQO94W~yMs!r(G zsOs63W}-JOi4pF}8n0h*XFMHFNxq>fmu}PSxL`4myBuKJ?&Nq3V6Z&75esAby_sw6 zX#pZe_zUz}8iUws9i+rh+%W9yGP%<9cG(>_vS| zPcR#}#Mj}sjRt;p+8I?YX9a)FfiA$u=?#uQpc_rvbCVWHs{vkp`IH>`%Q%}wSo+Ef zfu0TR9<%`GPSFJKM2S!3yzT_3ShC!FqCDL4jL|(d zi50?mh{`fFLa0Jt`K&h7e^P3TuHH`|dAe48Nr!tO@rCzr2y^`UvgoffM;Eecx7XUg z3S#Lk#>x}P89Dq1v7M{+jqOU2MSe7Yg<<~8`ButN@#+p6A9PKan7yhQ`rt``-XY|R z7vihD{dH{3D0(OyADWYpIPbe?nx*>nJ;Ox<7bnwNM^L52GDW`R8Yjc7o_Fcr67Ne3 z2g=%aFO(@pa#jZGzDS)7{Z@;5lI+Xbz@!jfaP+68)ra#U-A()Y7_!Kv3q`dn$f6Ed z)a3WqXBSS)dBi|;?Rk^2_@TYdc(o>Jex2Ux%WL5e`*iM3IaCWYM$}4{36p)XWZygS zf2<*eg)tQgSQ195==@!kKC_O?+J4cAo`QPg`24f%vy1cCpJVr~hqVKXA(y7W3sXHI zxNlwBdY*Wp=42hL+ z|1myC1CF(YZK`IDF^(FSc$QR@aZ_z(B6_4SVO)(pWI=1Ie;Z9QFsY|Smv2ljT`N-> zIno4pX4v!46KSXN&|5u0Ni-9D*{)tR^BdT)#0``Bb*W*5CS_L6I28cyhaBX)WZJUr z=^M{6y6NQL)1#|0`m`YY5nzIvd0!mlgk?}yvr6RUvW&vp_B)gJwyM{G@vhZKZMtA z{7NY|(lpcgnKeu8GtWZX$>=%t*-C=V%M&JW#0B@|rM6BVftl~Ygatx=$D4|pcG&n* zWm%!gTl+eK>j&3~Ofxv033g{8i^xk}F$(C*h|aoD+iYoSdXi1hxvO_aomx%y@#jrRYW=^f+Qj_%;KJo<+RL(Y=UlHI zw%BcSF5~7)!qQ&l=h2z%=6_Q)O`h0e1V;`-rN6}`+q-b-`EDJTf@702hVGIXt(SF6 zEyy;{G3+0chmE)o;Wj8e=5)I2MiIkmTzWE7(sz7E&zipkf-q{>y8+;m--N8&Yv-U@ zjo1K6?Tf0W7~fMRO#r6GA7L6lP$dnXoD8?cJhhn-dS|fFULYL$E=|Z$6!aFBqFxk- zo3)owV;vVh%l}h{g0z(d`Qo_QND4Ka1tC;>RlZuD!9FZR^Mo^mFM&6n+M zv^S=EF5SBfbP8WJRIV?(!6}Kmu(^ntdJwb^W;{PCjjH^bZZpI3zkiju(6dLVXI$ZW z*^eCk0z_W|Oq{(YIIsm3N$rQ%$B>Mtnb^s_E`Gj_6tM4W*@8<2Pn)C16oCbAG?PPK29t|y`*?H z^*5hft#_sIEB6@4;)R`!sp8K8OVCY(P)-dQv;VJ6_M8jW0?B+H>YWKe?@S=e+1^C3 zP~`drkIM#gkSZ#3N3b7Mwg)tPy9P}yr~D+xwh&&gsW<^nF3?cnp# z;iuhUag-nVf^(@+GFFm7ociM1`4X&7tj(&><OTIgg~ir+cQkn!hB8ncgcUwk0d)TXbV`Lqg=7dM{(9uQ(}Mw>7h7Dh4&>K< zG(V^L?*7XuqQokKw!2ql`9}(&)wDdRX_aAA!Gl045d9DH$eyqS!ct22YY3iI) zVGlMdJ==4Knvm_eLHGjvWZeN~R>-{WF6hLbw2?_y?X|SYp4l#l$MlZhap$g7~=Z;&_85*>Q2xB|9irc*n2G}732 zZ{4|eps8!L$ki{6x%K;(;0ucm3x*$lXlv_jT|}*>TG2{ovD~?C9RoD(i+fUbkJJRD z)4BUPEPYlv3vy&ew#`mFURFT(qjswqT&rL-{7zPi`YE})q_ruA`#NH0Bk<2C+M>Pj zgNMsbg4RlxgndijLNPSR+lav5gbRh%@l{K9p1|F)#oRx*0+MAD`*^`jRshFdP^_KS zOav>Z3yof12tRe0lbJ}l4@-^@p1v%YicKc~OPiLJj21Jk5jepLyVrVW1^my$05DRG zab|B*YK|jv>VylsPY;mUAs&_Z&u7gq*0xj6o!9FE7PXrDVd@+rge1Q9Hc-o*L8PFk)vqN?@;<(Qy_hBZJR}PH%VRR-DbMJC~5Qp1D#IT9m!0P>F0m z2X+J0gq_Mtz;7XWyQ4;1`x_j;2JVtg@M0vZO(}Pd&$caDk-ADuEB)vseAIvF6E*7Sk@!x1n*_I&Ed*>GOo2c?yI1If09m) zPF;qX4FBd7%pz=j&YGQRY{fVjs0m=z~4Ih(Z$ z55KI&Ax{}RwxY?AY>l*nQ6AAbfzp<|&gE~9^aHtcfQ*yNdu4g1vJ~I+^eF%|88-TybX&o6oh36>B{Dmsbe zXoy11bU0V+QO*g=b*)XyQKoZ3lO==LZAvijZz$-EhnD@q5+5Hi9pQMrpLL2>T3t9@ z{Mm%>ViKcbgEoo8r!*lzfL5J}FN^=&Xe16@h-xEjTIrSEdNO|dHQy_fRXk%0NXN7F ztBzS8@LMZ#`l@%zF585iKW1E0N|;}Ic~SKo^>q7ouL{6I?dVNr90Kd!4&nxvL_;6$ z%1StieRmjnF4t|_|A5lpwSo6+J@nuA69rKSrPX86e=+}E3ovs^iOVORo_@ss!)TJg z&a~+{$mJR}Ke?%|z_>=;(x&4a!Z$NJP}}4{Hnjb4I1v%!_7QSdFL?L?WTOqh`nA%N znfka`dt|;Wf>NOFY!K{D4OpxBagREcjdJPJAt!Yq8j|va>XG_l=;oL0p(V?&m~!E= znYKdhvD4YTfMp;;km9uHLADnIid1cc)+;9~55`G6uZVlasE>H5kjL9M-1Wu4Bq6b) z0+(E@q{#k)DI<$g#gF@GvlwkOLaO0}7<9bx}Qe!{SPp_sB*ZVgt_{M=ugS)`; z%kJa1J(B;fuXRg^Fq+idRCnY&cemYai+%yCQZ=33D3J3vSS<8i?)zyDcFNAxBP4q- zDnx?*-Z8`blAr~H;yFdYkufdC2 zqW+NDf;bP(AcS!npH>jym}sMWhUp~aNqD9SkNk$fR!{w*vLw9L!+F?xcnW|mGHAP3 zaRGO7{N;MnhH(gHx<1UU`izY2jtK!SX(h<&3tE(k(NJ61(5k$d?%ZQhH2XL^@-&5( zl|?WN8;{UMw0JqPZ)%)Wx>H_iQJvaOaC(Kci3vn(wb>o6?nyyX5enf!$(nde%FJa= zE(dkevV?uz+5X(ZRgk~ny%Uj%_0C23^+lO1xIiZ!Kf{^w_7ml9UNHA`Z^BQgX&FCS zzTZ9K<8GY>Lzm1>vL+(vyogeB@Xx4pQp|iItTl5v{+acVw39BiIfUKwU43a#dMPr! zK!3Y(AZj3n9&X)$qUE|-=5ZNLHxcOITM?;*ps$O^29Zi8(KpoV`x2MFr*}q6Nv_f; zfs3W?frlZ+j~wbF==}jTwO-KZVZNrw%XfFk5^b>QEFH44bTgWOBBvok3H2{cBWI~; zB>lV8zY)0Jy~S2{Q5%Q#s%H{BaEqqntmt)WUYXu<_0qB}%@!@tn+P&P3;z&FpkhPc z#n)3kNHA1t@@ciU{@dzUJ=;0qk3E>`Sxac7ENsFH0rcO#!H9c z6NcULB=dYN!ECy3kHb*vmrlj$?WS znT>?7a?i?=N+ko%=V(P**h*2)dSj6S+X{|$kDGq&=x{i*az$C}$$`Es<^=W_qc^h! zvw_?88G8%XmJA{6R1(1?>a7F1q%F;3c37L6o1=XCW*GU0Mq zQ)Uf{`hvbLhF5a-bXCp#;je64N#$*OF2nG3lYPx3KK2o_jeQbjl7n6s-y4EX^bZIs zg9<7KyLdWxAlhf0&!x_$XEZIEj`VCQAa*N`b*(Z|z>e-nod%wySgY838-?}TDK3^E zzuGc25haMt&0?3s^pM%vw)fLDkhvP(=hD<1gZ_IDn?K7Y_xuG`O}F3SjI?BQ1|bz! z1_Ayvb??>!PufrYo|Rf<HW*+{DVuI`GkY1XeA1*#U^z?menVW5H z9HTviIDHmgbyil-iTWoJ&8jEGaI*Wa`1sv09fG;uXED-S!J?jLHi5JKFfh5SzT0VJ zM<5=qTMigbE9jmU0GSMJs^Zwsi`8`wtY4^EzfynxRC_tOcDU*Iev3fahl2qaqv<-P zN$OfX-MXu0J}^L7V&4DhjdR&2589HsrV#@cPyr+_wZVLPpNhW$AOe>u3x%*ATlG#0 z5JoQ_H}&`Qt)E(&!1|eL<pczr^vyRN=i#Yfzq&2#j79$-<>PM>UpzhZqTAX<_4 zfr^W8hl<+}cY6wnS7Z~GY&Lb=*{^g5tw+56M(o$;^_(j9PrS!SYboB2O_%d3NJ`{P z4*fCawxBjvw(~dCxq@*GAZZe|ieNHs>CeX48n^fqx%cO&C8P%E=F$#yzHalJt{B^K zr=I}~p{#4ZLnmmuVe+s~3S^L4o+H&mF^mfG360Jo_esEAMMht^`-^I4IPQMP-=CRb zyoZz$?-x=cSHKy0XEho)p%y^***;e+2=Jz#6vuIy)r9HXnKgCMOy^%gA|*&B4*Bb| z_bZ>mbbBnGtl0mwV!-bhWLp%Yq@8C2+caa!IQbN_1LV@plzUU}jCr5=?_!mzU8wM* zWvd#*5CN#;L%wRUn=|@}Ov|+JP{4(#_j~;N3re|tpf|ZB?(LXxz3X;{k*HSi(cH;A zM>Yb9cCRlm>wE0gAibg#wB(?up+4X1_F27?lS?zbN1LuMy~!;zIocluhIhFoz*st2 z8>NIu9#8$OyQsI98**D5#&XPf3}o6?UYF6PxSfb9$(`)giPX6rW8R>#W(H+4zHd7p zr9|-h2Zl>@xwji`(|^acJunwEu0vXLmikP8ro<2w3^j^rps|gYcF>>{_$v-dsyN=M zaA#+BeDcF&^y8{2X`P^%{I}drRZbNL6uu)z7YPN`mE=t~3k~?1Or& z#%(#7XEJm>upDcO5A?bWQA-imdxPj$;Jfrsa=wmW;kG)_i_USO-ok6U(^1)t8M>^e z^bf5HZHu_m)x+#AlOCh|L&CaV5l=Il_Wm>HVqx(u_3UlV!gtMfaeez>arBEWNi#f@9T%?R zNxN(6Ogkix9O5k#WlXupTZ!fCiqA!|J@Do>DC;r+;(V25o6vJQ0T~i9OLM}K$x+eC zAbN_DFG^F5sd?7DPtPl7aakp@XSOs3mpq>$znab6QqVS>osT*kR{>wfI#HtqIz8ii)fW&xdk% zzxmG%!!T+A@?QO*=hW?^<3~wXPCylkI5!3FbG53&taMsHA=T7IaFTOi>Wh2LyXN=h z>7on3S=?lZ2~)9HaII{+cO$Vi4MU%yKebgx4%o>*SQLbZzB5As2e=DG*%h{TrquvHmLeFMI zG1o&thYrCE`dN6m1NzDdO3&%r5T5=3)l3t^a*0Z! zr??l&dh`-mUbs!kBzg89Ng^c*CpP&r>1Lp&wL1frtTp+8HY+@&4QcCn0JK8&TTY(< zCM^*fv=YJ%FtlPnJzRNrL8xI)4R}hOD$4_8=3%3DnhDh zn)WHFNp^v~7om#}iL1|@+h`Zu6KsP2CW(4>KD?kePpZtgl| zr}T52+JV{ZSuk}@Ek3R6qiVpALMXoq{Z(DO4~aI3f?47y@Zo+K=vA+G(}KomO=#~M z1I~gS5XoH3(;EI-ux@3f^Hc_@R((1m)XvzN>vok+gOcq@qE8^@=p2VqE2poy5u3S7 zCiOW|`fq%U7z4FDh^Yt+GCxl9w)bJ_9c-ka<@6FqxW zBph!DKIhVugxhS>j7!%=&yt`;CZMhvAgSY_<_sJMVXlCz% z&5%keY{#Rw;aXRxc+FM32rEdMXDVul2S9D>>HOf5ddua2C3236Bjqr8+oFp9y3vX4 zDeUYD5N*rt%!_%p5X2`k2BP5Eb<%|!9+=x>l5U}5ByKJ*=C6qZL@=!KF@r)-bnP^mG~ zrbl@8*`H|_CT6MT^eFC4Ky8m5)OF6@+2LV6rt zv1^rW$-FO($BK;nv{k$L7H?N}Z?cYFpMSAg{mv4PSb#+o+ip7FZ9r_o(aO zSm;B_aB~uujJ{;&>@t^JpZe|~#DwQYoK2;^NQz)mBjF0}!(meh=9A3q*;tQrq5F$2cn6mG zSMGUZst9&2)4tn}E{+$ZPTw|sQ*Q~fx?n+lQ4q-tiGP+@E<=xv+Micxu0OfzwHSNV%?o#q?XWnPohVt6j90hTkhx6<0?1c?=Q*U=_${8>)X`G(h@_2{b zs|kegUDqPMCA4f#7MIXo$@j2s**xKgS5rS__#IsP_S(HIC(bS@IB9#yGBu0yaIyyg zRb^@`a^uo7aPM3$G*@rOr#!GVM3*eJ(fNnks=drfhXk0$*zs?%h|*1MDPX79UXmlO z@o#+FVdv23e3PT~1xc`yW#b`k3`hzXz+^}PJG2nt!)3jDjoGuw>xZ&jK=EW!ZLhDc zx1G{zAgJD6f1M%iu-{|kbmm3tY$No>*G}@b7h*h0_nK@aA)}f@r6K7Ay5U{T)yYXK z2<;Q(6$$~$;#ElgLA?UXoYZ8RX14jHbFBrol?45J?;X&g(>at{PVdMaNMvqcjv1wm z8uOpw1K+#VC6@|!(zFE< zU2+DEhuTUzzwNWBog)xS-~BG`wFhQpyqL<;)+dN=OQM;&7~wj8^>xyGd++Oc#m)6$ z5e9pFFE<6WDFmiKU&5z(d~8lS1E`6pdmiadsY#w-u45bWGZL`Tj!m&M0@KF3cev? zCLZ0c%1-)K22BD>enBWKb zn5n{GtGje>_0cSeqq^`iSU zKD>QR5j^Z+&*Qf07}2EtA7|-(w6U-5k<%W@ZEMn$CvErclcWv8ZZb6|mOu@+)3~>B z_D6*a*AN1}mQk5A1?E&fpiADjhh8Vsun~*UZG4THNe*g!*a^!!n+ikXINC{FAL<$( zH*ogA(q>M}Yx8?-k*_g1E7EcayiY?+o(??yJJbmDVH=#;+qTx)W;0m-e%pK}R3 zLRuo2UI|#bF#F7g-|X5i@ob=sS)caag}tC=O`PqMeGcf;WVhI~+O}(+PP~g*Mz@r3 z`N4j_<7fMw?DC5EUS)$?8RgM6T8~u%0yXQ~1N$F49K|qm%bd>0hA)LZ!}A{_cAh}{ zTn2TCvF4}8Ykn9p#EqGzX(H>}^e-82ZDX^bYQOr{Nj|_K zND0O@c&BzOxp4Pf2S%)GoadceCJ93f2aF@pdS7J z;E>Y(t;f;WcmL;Aquny19lU7bO5L4m)9D8B{1kRzwRv;(OZE6_$>_cRQp7P4 z%}w@?_k6lGrwiO<^O|bSH%EKgbmq2L9s5Cv@t>UtT8^SeAGYYr^0_pcyep=p*XSyk5oj4 z(OjyAS4TbS{K7~7;|+0G{ucoouIhE*L%yEmimvL_hSi4u@9Q+F01WhgE5CMg%U+*a z)wxzOT-jXN{GUt?d=2P8o;+Y1W)aeg7!A-~&v2>|y8fFRYl{$xE{{Ed`bEMc+6u+$ zul^6poVm42wpx!;m0&aKHR?X<{_n;AVpz30@YVVL4kTdh$uOkyKM5R5#=Mo4+5rQ$ znxlPiOUv0m_Ei=|bx-{7R-xJn`*ko_%qGGcxW+Z2(FoCf>gt)DG@tE)|77oE`1-E% zDcn!5o?)CchS>dEgQK0uDw9#rzX_fUXFrta2J+7#rG5V)%Kzd;wLEt6hxztu>GQGR z73+lmwmCir(80Yn?jEguND%PDvz@%d_^*XSkU#|YlG1*4;j32l>}B(Ff3srYksJma zj+zRx4W3vTOT?TI{L?ptEB{Xh{7amH@QsJI;`yjgE7@Kdh3o2y|BW)wR+(%Ol5Q|m zcGx!T^(7qBH2@zOF7TfAZ5aMHo1&rtPm|Rpwh$=M{ZUrjhp6a1{%YHo>i+jfNvuFQ zfLlO_flhe>B)%;^jQrVe-PmDOp4q3@4l(|@_5bequcO=WzBY(k@#^aAGH3)hqt#S& z1QGV`l5fRy!Kw}k({Q8}`tI%@h)3^-acSTXi&x zotD&q47`qX+8EG1N}U2RUvJ3*lClHIW%939KH8wmi}bCXS!}ro4tA+eB>L#U|Fj2) zBhT+hz5umHz=oC}vf$fJ3|>GwvFf5hZ;J*Ml%^Jyy7bN8I%>tGo@$+@;ddei%L^iB zlX1F@6&6-8<358l*XDg&rMg+U5;0CVPm}to#ML9j*IoSIalOZx-}baK`3)0p@}QxS z0)86^X^_1%bQN;9%(*7gC`UO_H{%IHty)qKJ7ba9+t!q|_`$(>RJ7h8SiZ=C$@+7t zQHX$`oh0!FojwJp9UVIZXOenRx6(~tk~lmCka9oF%%0oUWOy=NZ=j(`MllPu<$SbW z;=;3~lZqteSDyuWW9+1fM<=&2fN)X-Zf!`fj;B`uaPtw-2jf&c@V}bhCB6W7Y`vKK zVBe59%6PX~h2Ktz`x4QcZ4x1x+i9v~d#k{aS$b{ryw4<&F7}8&V9cwN=bDBbfHykb zn}kQQtNc+^rAS7TK6brIbq5hhTNZu-Wh?q~K9Mgl*)a|fhkT+gX#86$G1BIB&ok}n z)U9iZ9q#l;V0g=nmXm9rm(B1QO{bvu#Eu>t;-)G0b@V=UB3DK+!R?f2Q{5`8cd|h9 zwm#2ls6y2yRKq#)j4aUK4eq%*r86hSD`=9Q`-@6^wMKF zMl<-xY^s!EXO{E0Oci!*q1Bf?h+nU6^2B$|tK(K$@l!(q-2RO+j!VBa@^&7sF$L@- z457%_-W^5B=-=z_Eaf|&t0xo?Z!7&|S!%Y@@2z@bx8_xGQy~1(eL2age0hAA)3u8) z6{s4v(|?sWNdebVqB>)goV-y!xaI^UgD|&Jk-WB1Zfg1yuza5G_)>Jb9k?Q*JJR?r zwQ>8FdVK|M3Fucs?TmC!O}EKpeaeW;U$=^6Xt<&#KWP?b<=ED2v))T))Rxa1qoSbT zjYe=VcP@Z8AUk38WC5AhVvJL)s=b)tT(Xp`Y2;!@o+KpLh=77evX!-g(ThUIqpCIU zk6U-OMaKanIX-A6OT%|IcQM70y{~xN%Zh7@A;7i0$!*&Tc92Yah00(wyz*(HZcVg3 z!|sS~(+it73>!*?q3&|~ziRS9+_sc)`6zOKVkkgX}A0>I9bmSqUS4icNOzNL8 z7d9ma;(jF3Pmi}v=?6`hO}$kjV=&RO#k#NTiS?=0cWIwgabdLIJI8Yj*&gIP{z01#J1b^ny zG>7d>y9=~Y?(7?OxJF6(s3*mSd)l>}kl@M8MLp8LcgB`z($+?hf8v-bFj<3#5~uA8JdA9WqvkJ zr!O{AGn^m{Md%{SsCXw5CS>(zAMU0-a<1ltV!H{zU{opsTagn#!VE()Hvtn|>j_NI zz4w>T)-^UQ=RoIhG5S&KRR$*8>n6+_Fp=$ZYT9tlAqcu*;~g@*90YBjv&{=JLa1v+ zdWl;kXJGuCm^Mu^@1lD+gs?tGh+Jt#PI-++)o-+`LKXa7;#$x6U<%?|0>oWR_10(U zwS$(xg+fygpscgeDsQ=godGk!+B%SDEPpOP-6r9;=36N*ag0`ao##Nov?l=fJzc%^ z=?Yz4V3e7ke8F$^*qSY5vo8@>2=1Ov80T6m5P9}QY|x_Hh*a^2ds^%Ct^F~?$>x9x zd55>jaxL=H#SR`lPm>~aO)3Tff=PhWSE(dXD{ib6IL;{?=MNojp$%dR2O*emFx(I7 zqdkco$!lf}EA2^;Z3QORrWO4FxIj`DI{c%hYu_xRV4qq*O6E1~C}9{%Y7GnJzs3@O zoCIu)izv)MRPi>BcmiU4YJFyhDTFYisC8eoDaP15W`OaSb9i^vSx4w%G82z#(vO^W z#r#1**ALW z4cp=e(__5#238#G2%epXf9t1S%QPlWpTC&?OvW#or);3R6tx+5pBk{U#jJjMGWzsI zB5pncyL~=Q2O*qA3OBDf-U4QEoYDZ4L!ZeosD>DU>qh`5rjc*5HqarTYXx4CO%j@KUgNkL8NjY`4LwTvHn z1O5~1^%D_6j6;HQv&!q*hZSWp{Rn*LiwM2_8j{X_!7iwQTg#x)tO|vlky!Z@Fv3Xd zy)8oER|k`DQiJcd?@o+ds;Q7OSbb|!r#vxzd%rr;M~>rt!hKlKP7!D(V*fB+1bPip z#uw)<9k3QYO(Gm5a?&C%%94MpP36-BD7|FzF!X5 zVI>(jmmXB|*ki!y+ont#H!QP6K-R+;1TGa}+A?te=J~gGuv0xr@@3etdS)oyVIFg= zfnm03O5u6~xEs@vLnEaCM}33ZQnx*jyZNwCV9>#B=y|T299aPz}I~BHxr%p*y@g9k}P#4 zR^By}ZCl^uIsJmyyYpdQ_=7(Pf0OGP(ky!sqg1I7xz!Nsz>-xuzLc zm5_1JgxFRgK%fy2Sq@fd;QNs0g6d7D6BpiZo&;dJgG{W|*T+UFKSzPM?QNX+P58dn zOn-rJ8U|Uq6Ch&y2P7wa9iF}}UAa}by6`Dpt<%0ipxF>95A?GDMP=#ZKt8eHlwqa$n^?jL}naCXiSk5xlGPY|8fiCKJi$97b)SNZxIj;n#JYko+IG!$841py4M z0m(OQn{yT^7Pj9k|L%eFPt=#mh-=1S{*p7(kL)fy1V=9h2)CH3e#brB;!x6kbsOys zQ<_N|6|yMN32>!<5@~2R)}VJqZP1(x+Qv z;PkCqU3sj+dsKZAWHujeT=~%Oq7#+l>E~Mfk6%*$yu@lusE+>Ti^aDY-Ydj^_t3J; zvqw4udpw6696Z3A!>0B*X7~#w)5U_NVy6Y!bZc&UEni=Sg=9(wnjqtf){lgB*3ID{ zJ`k$T4yf$>UTpYn64VQR-YdLEznmF?r*Av6408c*h}%45;G=#8_OEtP9Ty9d)JxGK zhfmnyOusXGmXCY#2U_mmg3v@0n?EN)+b_wo8EPt1VZZaJPU9&!rMM zEr0Qq;VEo=<^UtGLv4}RG0rH}--+}LUKMn;=e5`ZIrW-ox;7%W;v|_9sC9Mk z!4()JU5WLwl|BO~GBaEaD4FaSS>+)orIHj2^*##a<@(fVPtx1>R{(#um02qO&64W3 zc6L$YI}E{v2%;Q5k2)U%-Cl19MAf9@ALkUgdAb6!QX=w?nUoK8UTV_YkQRR`-dchk zoe8GpS{t(zBie%PvmH00vgfgs6F@ZQ3jhf(vLHD6x?RrioGj^o^nz|Pl4o(*(9jws zcwiGK`+mxlP+~Foi@z%Xxk}edPREFzfR#U_=96?NzE2}o|ME;fs-*|Q2sW)zZr^}f zm?Vnl13IATolm)kTFub6y-x!G4(}6Mmj)N!9)JF$fwxzDJm$;0a8#ef&6?JbQtt!V z8UHz*ulu^^++^R2`lHK}EO~dgD#Lf$WG~5}zaUGRN#W)l~Cg0ruBY3#+xb*R_Rec zl(}vy4(cidepK4AADx-dTkfPDPMuQ_e$Sa$49T85>aI|>*{7_J>5yE)u`qP;?$-7m zROpZ`wEX%N<7|Fq#QEe`epBCxQLNia%>4A=^*?I)g*;=!_ex~;CBY@QQe(R1Lb!UEW(iK@W2l;}K=ws)`irrzno* z=I{|K6voauE1IJjqRko0bsDeuA7>MI)o@#u=EIWk7 zbaI%Kf`8INVf=2o5&4cL(Q!(5+w4^OtXEo!W@s`)`n>M)o|yyBvrH-in8P?y1eaOa zTv0ixIs9is>Y0d-w1x@6*>TRBpb9?UT=!^HKLRx2n2K*H4o!nu{`{7TkRIr(D@8pf znCs4htkdf;KklvU(pjGAk=Ms5geBN(OT3;>B<436CQX5}M1YN&7uSrG4ZwUFNuxg^ zI-_DnlU4OTCn)J5Dju53x*irz-x$qw5LZ^}1BZ?!>0x9HTaMR0*Qx0pHXePk3%TRM zN*UcaJ2&szVl$mdK0jK0M=sI1oPBFMu3@7cX0NLHlG=K8@#eGiA)sl@pKda6b)c#H zo*2%^p!<`JjQoZ}2bQ0AHvbjIkoIIAc1j1QycoWBaM&aeo*-@;PQ@kXnO$0cgoSU2 zW+nc}D-X}$x1)|Q7&_(vw~-d{l6~u1SKI7-U2t$Y^q$G}!yQ(`V`DZ7!6%bU>bh+| zbz)0Wi#94Jyw_$=;H38cw1;kas$UJkZP4Mqqw$Vzlk{N&zEk9sVpmKahMH-~zMK@$Gq`K+ zkvks4p5Md!7@ix`8dEiJfAh=6mH9mo79lb6jhI{~XqlT+WV&VX?^wR|ZctT=TID{I z|0)xg7<^$gr_Fi6T3EQ#-IvXp-y@?1Y1y^WOX3|vl@#W1RSm+~LCDbPv(|Rr!la*b zCV+I{KX~2#WI{kkN;Y=L$~pbb&==f&qN;rOm1MbIw~%)a$$VAx(&+H-nO~5`amdL* zVyS0-MKYyBiSMFB=DTzd-L|+#%&=RpI4_z;(zF$Uf^UmXXQW+#oUY*T*qGq+zp5Bn zW&~nl3OgkL{PYG|yUkMVkH57mVR1t4!*(^dbJ-|3iCca3FxhAt&ayS1mNCF0#R>X% z(0&V-EQwNVl`PvzO2yHrjCHQ{KokKkbqILAOex3=< z$lFyyOpgPeIP%=!(4JS1GQRpfANuMN7=v~(gzt&_p@a`SyI)F9`VY=q@6uyQqh^7r z$-{0wAvR4Tzsz7HNVQ9of#XwWCzcw6bYHgm9S-ASmq@0QLyW$k#)gYs=tV@BNFuprID0TqV+9WpR)6(U=bm3*Pu<+hWTBBVS8xtW3Zj?V9p>UH;Pd zx5dw6{hWUFwhgHaGq6pQw5@P`{Br8$;?J6_53;Q9`wJHXSdP1o>8rerL*|nq(>J3t z{Jn0GpgpghtamHWe%PP#*|X|&?tuc9k(^M(FQQEi;)ME%qAg2FSkMag9ll0{EAYR% zxvTRlteiC-LX6*0YbE z4K;0%)EUNAc7rxF^U)8%npFq1UR9mDBCBAKUba_eHxnRw4yZ4w*>d+3EG*icd^1DX zo`$lMVG}URy7#-J@w-O;VfvQz`Rv*DOLFi5$U$^={b8wxlJAXVzDxqpK z5y*bvvZh{Qk1ix*BDJY6X=z!{X(P4e*PsmaC8U1%>(Q!!vAYvPTll<*+M0k)M*AEe zR(T$@cw+7fWwe;d3Ojc*CT;F?L$$4aOvTiZkQ821<1J!ISD^iYL|FVsu`X%oTywGY zXSW;0^rhnazbq@9R#&J=t#hnvsbdr`uJqB)7LE=6GII&&7d$`&sj3{2S$H2{{O37+ zhq9VeUIePt)$sCz$Hx!(ygP1FUgF_^seM5-p+(D#5)O`LJu_o;kn~E^2rwnTd+HIt zoX`%Ho%nxfI`4Nj->?t2Ra6yKtr$W3)zX%V89SSEa?@081Gs6H~h`X^#oV;--J~r@?)RdW>={( zFc0+%`%XYsD7a_}ymJ6vEYAh)uQxnPGdmrgX^Z$q?^vV)eWQLMmiy z_@Vr#H8-gSahK53@EoTP>Yb1}VQ?qXe%L$jO0LDw1$=E4bh3#au3 zP+9k?ck0;FNH+gwGh(+5O*d;Vj7mp9|uoCq75^#}3xR6m#}v!v0?9hHxfUAOl3~{gfLF7&^Yj%vC5H<{zuj zJB2fPIdv}VC@gNK(Ru8@@@uPErZ|tD7pT1tYV5!28&C>BhJ&jqw90Vq63aiV7`|{q z^SuPEDVZZ0+c)zWAdut7EuKSFNSa(+RWH@_XHOT@huP7KFK1s3-w73XTnwPRDMK48 zka*7XYG=vR%BFxW10n;P+lpe^78B`ttt`>+n~_^5yHEpo%`u$8O3_Ei+1e(utdVjT zn}7+^As2$&+D2Uty8nh_Jv}#+H>ncTfCB#}smDC0CUb0D7%BKhT^8p_*7L`iUPs@+ zW_;kNZRG7f!m0saPvx`@f0MO_+D(2QRzRn2KH)xeKtq>%&4l^*TB7x1^|y zQ#ZPkr0K!ewSamItaBy;F5gm+Yl9kz4{HmlLpEA8i#aN2E=4A0x=i_Yo?NsrwD-q~ zpEjGDt$;i>-O>)*;zH#{#yeiD?aHzTHJZ6zghgB#15a9{FHULP(NyYs=P1$nfj?8x z;h2#mRXrzvBMo5-%j0D}xU$!=KnDeMH?)d6Rm_S!&Mu7^n$1ts=EIAh7ux%AeFq#E zmkp|e?95B~+`rWO^CEC8kpfe{UUbLkwG4&s#t$+O#!RRQSm%nwQCPE!_so=m zbqa8MG$kcW$gNB*p|<&Nz{*mZNM%!G6XfO>JyFkuWBL3@#Rl=cGI5JQlwG3vh|Br49G;w3F3cFxJXTs@8*CR{%Y*daLNTrzxRv1Gd z!qXh@{G!($XH{rETp3_P$LX%$m1(EL;dgF1ttb$_%)23Bfqc7RObtO7=-8s}_T0@k z!*=-ZnuDSZt{*K0xf{MnT%hy58_i}4q| zXB4-cBn;%-15$%EebbYTgi;*_UK~k$TDf&-D&XxeCb62a)4ud6`MQ3B&mkP^vnI#Ew^azd5)GVk3B_t?Xo7 zZgEDqC-@fzi zKG?My^S?)T{bAjejjmxyJq7(%ysjiz?VB6=chMAl{ZfWzR6D|TO&T*SJV_)XDkxLQZxL#eLk8TaW^d;IHjP7>^$6i)1-wYemZ zfTFgz1@hK=Ng?jjPrf*>i`f?i0_yuAk4?ouNh=7hc8&wWBS%HYPe%8Ljq7B!KsPM%-0Je7a_plU#tdKBtaom@8Acf*7aQ0U zDr8v@eFv=_)M2zAr$>2;$8yz0-?bI&h8Ijc8CMxLWJgG^miqk#a!KU@z6!9E2Z!U9 z-yql*$C2P=1a{v~lfSNtlG7=IM~l6EE5sZ2;;6tC$4|K<`Gnd>b5AgTYFJ6*z?|@u z2~hT+7Ne|vv>BK1F99#UdoAxm%p}@O90fiD=+tQaoArBLk+m~dM{8|wVcBD`lgQwc zH(51W)g`hU4MKX)dOm9^Oe?++>xv>fEtfib@y(i3QY-20=Pbr;Y|QgUE-or-aKalJ z0z^zV#RX1d&5ENNy5tbev^=Rj>au7?xM1vW1s9{lQsW-5MFN!5X(;tzSb&n1rizV?Rm&L+!Z1j@JXo@<=rb(oe2zAWNJY(H5^Zs$o2!+?#ed4KGV@&r!>RUY}a#j~m)GSC8X!WLeK5jqNi$ zwi$OdEUJ12=5csNnntgw^mE*j#_-=1&uZkOPV8coR~Fq*pNh!;)U6k2!&E%5*|S

&4OXD_Of?r|Kw?uWq%8#vCX~cixzeR89Q|-1TDq7d z8`Yb$j7R}JAf1ed+q*UT_HeWFFI6{ln>sHtax;R5Y!o&6SoeX&uHet;B0EG@Uo{n5 z`_H2D9_0ASZ9CfmMkb-+$DTx*VN?*VH)YJ;Az#*i7n1a!jN4#$fB1=4P%Cld&Z%GRi=W|mWWTmLC z0m~_UQJbkP4pLS`Wj-zLdSubD+ZIhsK|RV+lqAYZG`u0jZ_$(rww}AKs6c11zbE#& z3{x%@d4iLEw~%?kR?_O8GWf-Q_z80nH8cNobPb*WR`qcarg1Gu`OM>$~RPXik=o5fl0kHuQgTtY>ODPa)FW!tZn? z?{|%VH2j@-1b!o?e8%q8nMSdoSi2>g1*SHCi>w~>p&{d2ZYqCVvpPPTt<`Cj1;I2D zP2B5*R7zBg=s0R0>}Z9vhS_Q3lMSx?P9pg#AGSWiu~{i-_()NY$r>|VBGg}WyuZ{5 z)l_{3>Zvs$FjihP440Fu=n0Y>w6R42LPm75b&~tta~~J>vTai}lFB|Dibum$A>3mV zf|*fkzH6$OQ{dX_ikE8;RxkPZ+uu+6U?NBVss^J5gTxMk)Ws;!(nsWa}U ztGW(u8YuSFwt<{hwC-xv=-aDlqdO?Mw0QA?j)&z4BBgHd&AFVcd8%{H@`=pVhp2Q1 z+wE~^y=TA+<)w^KO8Ux7=LSGFuJK5fmQJTN46;$c68g&mY2fchOv*~K#=rNp%^gjT z4uid+m?J5B;mjXKt5n^1PuSmUK1<@Wm_YY@9{{r8Y4==jylq}0@iT%?TqUCJv=Tig zjaR#eqU90tUocX(`voDXNaPcfk4E3{O`zVTbRW`zY|mAl0xW@(6^{*@y(srOjG)?1O+jMHhBlczvHl%$$79Pc z#=-nFH#+N(Tobx6$&Ma$lO~^{6v$rvrTU;PziR(~^^F$7zA^5tJeN zqc7UTP*4dbb7*kI@M2DO>};V1RQNR6Wc^$tt$_b8kaBUZgi<{)Q}FewXKY2VYjbO9 zQ}Hs-Wwhb4F{Hv3ZYz2^7$pA@C!gz(+y}$LA0i+(aSiviDVH|Hmn4H}ddLcOpWM^w zyNnR-CEvp6aRana}GZl^Yj88E6FdVK=?#*QH>q>s^?-)kTTOie^(Mpqb%_yB+`A!zuyo)KzUnyx0R-9r`GCutSd^ed=+=(v!~C zU_9?OixK&5T+B*>^cKyjok$ijc`_e(ANuDtbHYwjsS|AB9pXP5_n-d;6=Aeq^mkJ% zZXJCRF!PEY(Ron=oIKAidCSl|pbaKpBJ4SNFiDdYycp2tRzE7)7H>b;@6 zWAVG#Dtk+rZu#vyVm8v2vpX3W!8IMbxbujnk5qd#o_W0Q%=|f+nsb;s5m}V0P%H16 zH|&T%%FlNn?vvjQ3Mcx&ma^oT36Lcn@l}z3$Y(h(h)F&Zr@Wh)lln@PJ}EzIo(FzI z|LUg&%e%^Hqb7vljPvDtCWJG8BU1Dth2ohg2^A=9PW5+GmV@$nv#0#$dm6s<(3V_m zHui(OO^!nSc5LI_{j6E8_Untm#){+9V9kHDVKS4ghY>ADt6tH|?Kyw7(4{I!IH6}ncICVwG^cLu_-_mVlV`SzgMNWutti#*?=u+y87L!gyZ1y^OLla+wyVR9*=<9Umw2)SG z3pNY5u2NRbUI6nT-3M5Z05MIidZuSnB-1(7h}GA!g$8H$aTttL5VvK|4Nd|&}7S+bIv}dU3-nQbmvnk)_ z39nj|HG9_Dy9B*JFQ_)xl*$NIA2C1QJmlVEkxr5fVPkpqz>liAoD19k$w@MLhO>u$ z(U!UA_zk^y&Qrm|0gzMP0Wn;$&a6KRuLvHrvYWvqtGKY8^#cY5{&$E!H;pF;DLBTy zWZ)1-Z5>#?^CP@|=vg*$TXYuU!pY9@kO@BB`tyw$cEcj2m?22`*f;W^s(2`gEnR0y zyY!^}0kfRP5d(jylt88EcZ7fJM}pEzqKP=lz-%uuWnVn)7&(&PN}+!C!$1Tu-;~yY z+S!=S4^~UE5=Wx{0xthZh;nI9cK^AFYD7&iQ4R0O=BVb?&m0i)TtNgX7LQ0})_-QO z|Cv2BWyjz_oX)J8oXq9P)K_Sw$!=QGjfCP*%kdwXk~gpg6qj)1ujzZcrUfEbk801U z3}#@Cada^zZCY|!M4E>|0+fI^hmPeHOpT?goq3R?gCB~J<+`ag*|q?r zVu@4}fP2Hxu<^!O$vcBNZ3Ia4op}s>e#W6`{W|YuzwvC{?WD|gHCzz?0xLrKxh|Pn z5S;2qa;)i>&3aVQ{$sjH3Q$tQ#>qZ;7y`9pg-X14x5FqekyIRtHLWso{PY^%Ns{+L z41>8Nnkq%QlavVdR+8`Z&U4i`yY)@=jJpxzNw_*#($$s=xu-!o^YIOw=IcDr7AfDi z#;bCUUno^O4!#yR)Gu04OQeCWimb}dr=n$ zTl|vtVckX^vCXZ7wH*c?N^hU0DEJhATz3cl>qx7i@}b`KAI$Q3hTX~=ixlNJw;amR zhVca&z)hhZtIW>K@gL~xrb17UOmM48@-1a$G=D^DwZ$ArA=HZ0>}%t&&Qs+r1(>VB znV~8nL4`i2|Hc(4%hn2;xVR(v#(Q2cq>fvl|4-0H{Ok=5gR5h0bvNPCy60P(k_ z`jiPHOhd~`GaQIx1tlz$(q$8Lhh`4Qp)rwI>;xGCIf~5S6s$h^AcR@yis{rwytn<% z$x&DI$VvGY`?T2i%*^kpN%2v(SG8@(S26Lg)L|?fD_9Wts}X)il(y-2lpbbMfbD|g z?iFw~4I^d2Ts8KTVA3AcX-Y6=TmpFG!atWKu#7MXm$MEd>oHxfoMTSV=UA!eQf#cx zc6(yy{=@DN?&TecNiOTsHf2q97YI?KEb|up*-Yh4z}JwVgkNZPaJF_sK=Jq^vnxSe z1F9jIs%vBH+)bg>(#uUA=65@J+U4k}uaYMK^2*R2br*+`A;Z)(xU!^iaE=lZR7-=~ zvI(QZzIZX&zCCib!`0vZRW4jQ&;gT4-jr8243;>>{@q?G=Yg<-Ga+4Y-2qy(bx{De zwbcPC#+0nL*?7PEzKf^vMj7&rf=56f*iyL>#>PwgT>A^(3m5X_1>~KYSGvm}LATbSD#30P?*W}G^C52t_!N#V9VsY6EeJxHt4nk}$k)Idk~sWm9IC0a zlrb@4;kaIvPu9US>og>g>Tn-QQSy0|9s9-T2|#xWuNA5=Dkm?-Q`Q6_lvO^Ry#AiN zEvP|m$CN`ihM+qWWPOxNt*>6B=vidU#{NF38$vxN`|$Qhu%mH_Tcy*gWLTq?d~W1d zw2A|i4V2a;ZpYOpFcy>UYOky_KWdT{0Reg812;(GR-ot{B2uNbhS_NB+fpFl>(%hM zjTW%@706!uHhM7Fm3!N)$5<-*;bW4dBk9h@H$drG1b9nKj~etORcs|PwzkV1q9D}u zSC_Y&`ZbxKN)C`$)%737qqa7tqbBh&T6fmFsD?YwiEnf8xj9V<0$Tip=XQF|d*|?< z*k&g3KH1m44?lX&W(BBLxM zKN~eKI0sv9rGx9{sDmW_Wt^ z+umh-=Pppvi~w)n0VfFvvxn)?8Ya>$tUL(2P~u;GY^s?sr{9d9M!YEe=rhn07nOjFpB+${cN!za}W354ko3ue#^Kfm_lK|o9_v&4MgRXJbY?X@E_;cs01ZFkHX$-6+JYlkKB%pOtP<+zXPh$mUP~*Z@+&T zfBpniPrILGM83MT=c<6Y4!=?uj)FunUaRF9XlD!L5gAVVGO0znA2KLiqzLL+FAI1D#tfbIRKvkJZ0t z75S1tiI=1^q^=&(V>U@$b;#DB>DK9Efr*E4n()+V_^NfawfrGL@p{Nl+Kv(zfs{^5 z#~SSPErNREiO9`AS-O%;t5Pz{xPDsFGkqsH1N&qJyPiwg7zd&977@@;-KLKAP=({d z9k>i2>j!+0m#-&Ig7Q{OJ#$i=*$a|gHZl&z*p$~$(}+WV)g zOnE31Upd;)QQt5-XpZBbPq7+kkfHb|s@M~&O;g*ra=@=+&CKhYgXcc0YYn7e!^-3r zA{EA?v85k=FD-99nuFv2jr^e2*JiXj`An*ZUy%ROL*qtgqe!MG8>MCv#g|y0c=l*7 z-v-OIDZt+FUwu2HUIOCFr-YePQh~bf(^o&&%0g+0Sq%k8j!(S`aeD`?1l7=&j!{JJ zs1$8wvSUrJWvWk(y^2G3VB_V(d-z2K`P;g!&`mB>SxA?S9hpCQTEV(8Pu*E_ZJcwR ztWdSY$8%7*AD~NBu*EA2PgfrNWXM{f z?7XKik!yuSNh)6IbuM0zVS6g4i$e-JCzSZM+g+x>WkbAW14B!sb2Ui&d=LiF5<>XE3iYa-j7J z6C-~^SLUTN7CjjyxaIarQ@hXX-s`eYV@?#l#ys{?f%yYEO45#0Sk@jPmD$x?Tr3wk zZQlneW$eQZ=!NeLwN&Yu6O46*NzQUK0Jnj%uP|ru3Kky~UBBRxRTY)}88I7x^)CR$A13iG9 z=?7K@r_A)R!>$8yUT%z9Hx*_SXslA-0;cUnpW}KKIj?S{s|(JiCk^?&zul{3jFnM_ zKU213sHj^sYR$izGHDZcnd%UDX_Ze?2bA(QQkAvVNY*YTF6@%cEk%!CI+0Riqr!ap zO?_(vD0IKfjO#?qkAuft;uxrneZaNPe|~llCP+zizVWl^nFG_jzZSNxPb$Y5)};K% zH`?pcBXH@^6>z?^s&(S~r05@f$4ZjobF}d*FET~{xv$I|fY%Nz?Z9Ss!J}z*x{=bF zv&s^E_}YiDyEKs=63RKZc7)%Y6dmRXO;{$dU-$5?Io^dOgifKIopzE}1dd^eb5uiI z!fFoc_$L#S-_L^A6Gi;_y2i{Lo!i!imy$!PFC~AtaFM`D|23X;C|#W+h%^7U_!8+~ zwoy&h(chUJL&(FYl?Jp6Ww!@M?!M4j7xM1+YJ;;jVvq+s4qr*lk`BJ>`KzJfrx)Km zIXlhUuL$Jy0g<1nyVk{%uN-}dy}xP!4Rd*)IyAcHHN=%*;9dWOmUOf6JqL6C_36%^ z++%l?e_4O;9xwhE(&;+Dw{TwcPV@^{mu>0I1$3M<^O+N#qHuuzo6j57^uOtuxg3RV zUsRzC>f79+xu;`Y?IZ968GM>Y$h(G?-cuIi>8L+O&knKg?**1?0;8%HN02*K5#Cqf zxqMv;I@v<>$Og(GUBBzbSO@k}X!PcN9Zl-K;VIm^PSM<~L`2&ql4fHE@32fK>>Zz&fE}7(YO_}X8 z3Dv{lZB_=t(6GlELz_;DuoN}W6Ix~F%+?ESD{oGebNrwNqv+Z7_?6)W z0OP!F{Dj-dH>J3d#kkWX-`J5waaztpNuNeozs=K{)W#emeh$pa%3D}QMGnWR|Bx1h~0ewD?b);8cIAH}Q3u&w)^ zdH&;TSIs6@*b5CW?CkwQ&sz^r38Vy9vkd8aEI0&9af78oD8YuPB%2$egZd3 zYi^%*AZEiw;+F5g1lQvuW%ZJpj&=&z|H&Wp&!MJ0!$V!_gJvB6LDpxAWOHWe@uFt< zt(6DNWTVku(jbh=*fV+R=m2*DnIzgCM$u_qK>^~J6C8gOwWan@7f!m$(-0+`P z`2hU?L(_SNCHenvzw)bAmR6SJq%t+jB=^S3%FID&<%V*k=H93{(p)*HupBrtSC)HE z+_)!-TM;LY1Q8NMz5U(C@xLBiPp*g8gU{!EzR&Y@7T#Se(9e3|V?Uz%4e{`AU#Dx{ zYG+bpnQQxOZ4%{6j53v?17hN3B<9D0zX{&?jV>`&^1D**HjX*L=i)(#RPq~OB>;mL zYmgodZZRTI8cH0xViDx_B52H-*vV5&CYn#4lsI0>3Z{85Y_fP>o2AK{co6fQitd3` z@=o}!IjLWV@f)xPw}J)d`yA#EH0A@gB(?}*r<;;^k-I0Tt?_#B#)^~wtXgc@7vCTm z5`Kn`0W~;9pwI#hr%5fB5VQ!A6Zg;L8E;c$b~oYFntF9GfEoh#b~jb&WRD1Fi{d4% zL>iMkT0;Hc!b|Csn-cA21dUreLB>#$(UBzV8PME7pq}I z0tyIgv)upbdA;G7`yjH6Sc=W#;@jMa&KMO)J5oOu`!&6dFlCjO<`53sYTkp=C6bZ- zUPLzcCS!LQ9T7P)RB*0;G^$l1cfGM;?6c<-F0Rs%Xb}=f(1^pnqSG zs&QP}4>S87ndHU1auU-}87fbm>qp66%%>Q~3Ac~ee9ZgcNY-`D%| z1YvilT1@$&1eEc;n#pa12&|^NSbdqFbE3m;POQ}!6hAfp1!CN=957UZx=SY69A46v zmR!h6YbPODYsj=7w{}Yp3-xB#qE~BFfntW)xPSf46W{e2;sHD18fd>Onp;KbTeLH9 z>Dxoe8q)PkXn3}5<(-9DqN{b?2E0KV31U$d9#Aw*UG?#LW(Ng9~^_2>YLC z_~7Xz2bAktkC}~m6#Dty&&#t;9w>L|bcJ(a4ei9I6`>PcqG1$?z@5C*Q&A{H&Y(Sn zyX>=_yoCF;MbAF9hDEhBf#!$KhnMg9dHJ5sx~ON*T~nw2X5rmW+*3@AL`=FN;+)y3 zF5L&5dwZr8=3T?V9MHI&Ot1fT&OKy(%wWGKNJ82vnUxiXfkCp~Uc%L#H=}{`kCXzB z-G$%rI<20frQFN8=W05%xE}Zq6bmp$Le2K`%e%eY{v1eol*b{-JsF6_dlgoFv8MGQQ`=ysWSD{uL+nZr_#*MY@9-xs$QE^M{Rc zhbE@)&nz>a9%{)QxDgKVvZ~_avS&0%Ku7Y?{$bBSS;Da~O3KojL)x)Z6+fMuzxoc{ zjpzm(O@%ufJt@on;{tEoJHF!QA1SAL>T;)pNB(ncE_I8O2;5L3<|*!6I@=`47N^s8 z*Pk;Oo;41v2oI01ao;F1|3IGX*fKf*7cUQW~PIkJE)Wv2Sp!;Ff@QF5{yT4@KY+a_#TtYM;plr+H8Pr?f z#p}ICr_uiB#{l~N*lGDaiLBA%`YVOuH5>)Nx%cUKarfQcB6S389}98W$>EvgGa%lF z(f?dzCa5%HJFR+BPv8My`;PB59(yNv;uC)}8(@<6M4}u56Sx`rP4}v6mLsbb--2c< zG?gcJo8{_uXGth4hkfDzCA@T=W&ieWP)8xmR^NL+VM4m-7$p2Fi)2)C(nvEO<#YuEJ4z{1qr#!)gEZa#k1y1VTd=^2Y%bjxWR zdK*J?teFh4tNLPl5T&WHcS1y2$n!)^0}^OuNn;wBJ}uzp9=r|yc#!SeR7Bd&H|o}L@XJS%qdksk=x#q>hc%?pEa50_Ox^m}H)A^~Xy>ZU z$H=@qS&>5O63X=KN=qKXpCfFV3x&a3km5|&e(D4Pe97FeMUX0ZvEXIr?Z1X78QuAa zL?OoBe`t^6%2;lt_p$Fm1&$aYJZUXs zSmL0{O|`(;)N}dAX?#`ov~qywS8og8ZIyZjSpQzMkAC@e^qOT8CG<<|iiRDA@-b4; z0@1&<{_cVH!h_M_z(fDerR_wW)cR=M8s>#r^@U~K2Y9u&Xlv0%OFvc(Wm<9cqI{p3N^jspy`EI za~c~j@Elbc{27zzWZ(S9RTEj&dwQVVEI1q@J)e1JZo-;1EW#AG09OT-W6E7N!%oo} zyKDe{L`c(R-Jn;$nDgN=z1h zsNejJ95bNXvDnA6Aj^=|q&J&*5d_|(TbqVPckq}Zbd&(6<3u;zBc@DTgGoQ{W&EPX z@K4@Q7*OD7VejP#dMid@bQp2>?Clb74Q@3v;s%jA6FSPEF6ZMx<~K__RSqOA4?aOKzfv+|2Y z^lOR04$plH?{v4}s_kY#zbbe02Vk6!OKjve>qihrb>Mf!3wDSf+vA-K%X2Fs5>WINR!Zz|Xhj`Otgh{TJS3nK{ssYRNAcBbF)4i<> z@`w*^&Ys*<47&XJ*lKckNn&;{Tu-Lo*D4B0b3d%=EGVYdmfo&U1w_U8UDrxJB9wrF zy8*rn<>j$~xR;9-TOioKM90h^~|rlTc)v~=$6+N`%vy$nP- z&@kket9d@}GvM`szxvgF8PI1~S;HLm*(81>@pB-@W%V*$^W;Z+UDxlKAs@3H{g4@! zq{?YNMoW*_{MIdc@X=CHRX?ITVw)Xs$kb|JhI4v}cUyJ01SnH{CqEi6zlP^At%gs< z>2o|oNu=!J57$P$+oaQ(CxYx5x9iDIBnEZ|rK>`yO>vi0)$fKjcjFAOVs7r|4PI~A zdP2U({izc{lZhQyFQ$T-x$K{r;&Gc1O4Q*r-Mm1or8?uC0h6T{TzW^mJqvJ+BE>g3 z2D-e)1;=$Ped0h8L1vQvoY!mw>n9SSih~4dvrUy46VW}tCKW_l=deb6%`4QK*$lBp zomjPRD1X`y}~Dp$kYeAuQFOWhHA1QgBi{58*uNWTlmA6w;u(Uxyi3a9YKFL zzgo4*!dTDUXUjDJmR-a4)_?(7yq%{s{nXz5ZAkLFRjK3dAbbsP8Iep!3kv=B$qrfL zt*kVzLY!l5lcx}`=F7N4T>EWMapS4=U3A&jsoctwlWqw#Q%?2%x8(oH=?1J_hmDJY z!!z$2t))G6xSO(SU8fYNj34p`{ogFz=)JC+|HhWG)BH9#m3O=G`SlHnJ6LXTX;EzY z@mkS7%nQlKZJE6rvbf5h)oA*unszzaUU*f9j)3BjJ5W6`@eE#I^eWzw&D`v{= zNz!eaiKm%0Ts7%5=lQv*XNV~Nd$ew+gXJxUBaBG;^iD?N)sZqw82muxI z?GOxrSDP{ditwBKAT7A05w!kgfQ8y*YI}4|4ANV2uHH%~j7F|Y3|}$sDG%;rWv5X% z^GanfITRkv8B=Zvk(?GgRYnh_abe+)CP|4ZB|0ksX1?NqZ?(NU7(Ho#_^B<*GY7X$ z)q|0GKq{MQEaI8(b4G_ti|lC1Y&cvQU+?7mBg@ ztPccoCOgJWuOw}*^?j6X{O=EUVMEO66|Ma(uVpUJl*(|=t92?~NG#^%q%m>AUVY_;-GBYTZj;TN>3)1Sg3BrMT$eq)H)IuuX9eQH z(*GpO)4=c@bop7%{YptqKbfW2<1-dSc^F(SakR>_2dyfftn1|ea*!Y==dE382abUF z>>QW5mOr`33->eq5)bt+<(i4~K?T>Tw7o+o`OszjB(`roo)14&ml-30pyriBw<0ws zcS95BE&uz0#^A!e4mbjjIc&myy*n)>yxsBsiXRW1+F9;%U;bwSL12YkYCm@ z;{TZ8*STamryb`-E^$6Qd(Dz|#e3CCQn<{q$`tH=(JUi5V0S4p?D$N2t8N;20lJau zw66X+xS{JT=x~?Ps#iZEsw%0ugOYh)sWr(w3j?aj^XbM(tfeZI*TXJRq;DR4b?x&M zi9uF%L)IHFX5)g2=J^CN#dYICH!|LwyZQEadAJUj?CYaRSokS6j0n1yzVEg{U z>5)9yUe(1Tu#WFlL&u3O-`dEzgNG+oSj*v!%vYeTT#f$xbwOktR}W+U@G1~VnccsR z`Jfeqh`T=ZSdaaVKE!tW1m$A&QaxOOhW0D9DYz!M9i(vHV2c;bKhVn1R`auXmmXN+ z9)iru!u2i6^&ak|I@z6Y>$t4BA<=_V^aGq-a~$lBWc5fe+Y)RubjGL5Es+Ok^-1_- z*y2{Qx`pzayjnx2q27kq)61grl_2vJ>SCe=>ErAG=<-k)Hwxo>*;7Qn`qg*YrWjpk z7mduBLrm|bH{G9AIm2BuqN`P+V8+4K+c|j@XY6A?e}Ve;93KPjWJJDRgzjmZwL5rT za^C!6B5b&KLw5G@UqIA|7U% z5z2p?Jxhvn{U%4Py~juHG3Qqib_gYJsh@YRA~|d{(MtqwP}%!OVTA*-!DhDtCqe<~ zYl|^+Zxl#meF-n_{&NGfe7$^;?gW>iP+Kh;hd5$3g{>_{oh{{Lb2gxX5ZcCla(Q)q zbhUhMPD+f;fVp=@pxWWT^^2OqW}K9b)5D4mBi7O<{)}hpW2cmk77KZkN1ZPuv_V`n z&&#Jg_a8qsm z_CD>j^UCr}#j&jbqCzpjMdLU^&+Gps&r0=36PS(!W1-@^dt5ahHxA152Tg=d6+2#%p&rgZN z_?5!M=4mOk);rDnXp4j6GG0ycl38!q^hB8UV(^PC>&=s0YrTS=5<=AE%80PAHJ9&S zlV>&`So3y411Hm;%?Y_ccp+Yewp%%U`6Blk#OL8^h;xG}r$KOTvshv2lY@TBriZvB zZV{0F&Rd$cgR<8mgw--)J5uxX4e~7l?ce53GC{yEW?9AsPmOi_sdT3Nef+Vsn&w|e z=mZ1WJ7OnvpvpPiHzyn$b`3xF;VBDCchslCYEt`ObJi^hQ>uN5ynvv}NPrRLA@2G& ztu-%$+?~EVgL!q_hFzsR1KZu*zWDClG{?KDi>SW~;)1SwEEE#=LC9<6o-+42)S0|v zh#Ab#7?N9AgK;>}@XLd!in-<~RUOeggZH!Gt4QV}jCUps1}&_9_F zEvPjz5rjFdbSS^V7r7Su!hY*a9EAHyTxC$2SaJ?=dh#Lc0ZD7+-2N1!62^+tk}eh; z7gA^bdfc=gUQyZ?^Hw(Qg;zLFM5Ol7ju4~8%tHg@q5Su8MI%sfbAz1s2k-%Es5n^C zd$KApAHcS`s>O*pI1mx;%4KWpU&ZQx+&1@H_H4%>Dw<20TgwgDy}wx})JgV2Ii2lI z-wuia80>Fo!_Cf#ws9;3)*p&HuKgx}pX;Q{j0l{VnUOt^v+k=6QwRfG!h!8xVi)KO z{St@Yf6$My>7lOrZ}cINekZRB4ykM`3T_CHcfD7xLEEd#*$+3{3U|Gu3iT_u5@!86 za*oYgJK=C0prpncj{iYfSH!llXmefP5VHr|9cdWor|g&ByLv~_PAMZwP&H|qW^|BW zs{!SmN1U54zTM6%qM_k-aOz`HyfK{EI`!C`_OEd)6SZyZQmN&ETQod7-l=IbKwHm# zTMX{@JwEH9v1XppYKKZ9@52VhaJK7rq*2Dx3!Hgn-gm%6P9D|f>ySwS!_8*T7m2P6 zyF;^Qs_T*X=A%`~^c@>cbtFwGm6gG5b8PV_OLn%BF(;IM=o_#cE96Nl7mV}p`P{wD zL{guTb%2=B7>m`<(nHy7XmUI^)O!c(fA9r~uQXO2h2bl+RJzQjSXW@+FWQdhJYTAKH;l*5WbHk0G#H(e4qk0q+5rK{c^Y#w)|2&73Kaux>i83Tl(@qb4X>Eh z->&|}k}OZa*;o~)%SuM8lE~uzOqR1yo2CUQgEYsw&;;&6BbMq<%Qj_eZ!Z!ZkbV(R zyxMxl2Oi#X4MA6*%oFJ`{8;hB(G1$F(w|aAv z&zG2y`s#bHFdQw{!PXcrt-dhMAC1AOMScS|oT;DXVyBO9T62P#HFSk}$??H4NX8wz zfWr-oj)`CTf&ptLQj7c;Wh@T*4YsdVK%R7UZVO5zw){cmBE9QO!hF4SE*B}(Dbl|r z`hK1|VW}m7Hcm*wdJdcalab*1+mr+Ar;GCP+Z}8&A0`X!_jf2Ozt<<4&B**I! zp=wS;FPxH?m#*7oR~*?axSi(%R*dQDZwjJ8icVgI8Fn=iE}9}q_-#a(?3WaF8MdiY~V|$ zW!}Jt6QQb~!=9ro6`$^3dXeWFlGHGUCpf0)JASH`fOB!&L#g_4J0?)P%oCjzKX?Bs zDhmUB(3}Pdd8~wr)tp30p*yBV?y6$i0dadH`5inO!?~_yIfSM50DOy54Y{1Xpl(XB z(aEVJh3g7s$iQubv(j$IX=yJ*C69yDsu!mnACrZpo1UgjHRTc^XinUYd-{mdh-&4&CV$8yuIHzrOC}9 zd8phNpG|d3oKjj>jC5L#h~_c`02-k59%0^$|8q8M&MJ44JsI%!g7m?tF&0^bQ1>$k z5QoUTak}6rZCfC7avX)c4a_0^C_1ki_}*IGDSPkPw4krlj}G@mUdH6cXO1G7*uyp9 zucHx6giof9F9V*Cht?7=oBH(*H>y@oX0MLQ+?Y|1-mvekIN3UW3heJed5R3myg#Ef znHL~`%xsCDJ*+8bEqo1WQ$5IZ_la8|Hddi&eEEE>iYv^x%H-0Xq^f~7?DkS7sf2(2 z)sw)x$!d+|#Ffku5jp(U$-XmCC$#pb7m0%f*Za2ubxbzf&n9B`^4 zbaAq+6ED5u?&xdlJ;L3vDwI1=_fZO|B>`Gm-ZI{8TATEi*Dl&hDOykjwoy-^4}~}D z^c;lT36BSMh5hMeSM)a`_Y7+3w;3HDth}Yq$-ka#8xjjxI|E3o937QoX2~<9)kgW& zNP~833H$sA&I`jGxCffk)W@1e&ep;PhxZ+kUqEJRA3OPWEWo8zZl80cnN#g`y>-$3 z#GaPK(rPXMNjY#cgdYNM>d4YSRg!iAs`4H6Y$rX)`r36S0i~I2+v`JlcTAX}D3p|A z5&rq!u4kbQByC|C=X-g#a_2{03C~W4z2Be_?x~fF?l*Vq z{~ICu)~%L!%3S;YDbcDc62J`Cf_f9ygjdWoRhC@mZw_c+{hMx$L7tBD#96Lf+`~Q( zI-_@TZfAGdTS2+Z!HV@{_6H9Y7o3fy(gI720xwz zn(*atRdl}aPNBS}O_)dGk3Mz+cO1nl;sVc(^j7e&!ax`5N?}<;Of{o|i*rDQv=M%` z0efo~Pgt+WPc)9pm_!MYo^3elx{rpR3P6B2!{>gR?;l8us%?(H8_;N7zT+PwxadY7 z=@dJA=$qLl@MO9h68z8bXRkZu3U2d}rwgkZGa?QxD_6h-x*4)UaN2z>03cqpU)M`* zwTT5!z)O>2Bh@i1y?8lOP(;_5@hVq5e55$hJuUpDk`Zf+8VyEUNGijoar(;s&ZjaA zsD_(MfeCp4sApmGyOQ-G{(#il&)cbbX^|F=Xe_U1DHo@eaYnVG$ue=4JtC;~l2a9r zK;r2RZAU$2!a^lxE05TibR`R4ZRG3cRj9OPn|fh2@M9i^@=bs46*H)c~S=_kslR1^|C!nG6o-086;6+38wo~hJ z=kDXIVl(Y%g<$skU=e$DD|1bc2M`rCGFkmE#W!t<98Epr(NRQsOXETKH!mmRB~h*{ ziA^8r%Vfty(VcwoVbjr|QR-Yk2TavA0V#f@!nusxWM9v1!u% zg;jFKHNZ9vW)VYO+X&mcWyG4(ZG$WN8+;o9;AP*P2~^OMfj{-y6cIF?hw!u0--|hK z{aGOf460g8bY`HW{8m*4`u@2!*MN z;7a(!&bLZSd|cEuW9eJCtnSrC3K>u*60VjI1|u&i5fR;`$qNJ%&C{)qluf2Xz{Gg*lPLYy&chvtMI$= zzhhv#E*-;X{w{@gilK9xmYfYJ!+u1qTvc1Z`H8Oao)$;Z5|U zP;SSwz`lo0J=G4Oa;~w%#)2cg=Y#etUsN#9!SG5g<#IxGw&lfhh!<7d;Tt{mIUGzqc zoy2FVu)VssXmJk1>sd_y#c)`1nSsoBTZ^%#8R_RdiA@m1qj03Oq8pgz!1lf^8Cam% z*Ux>irLNx@>N-#w6uRh$TN@==ukZwBddD#F95!ZvU#_!gFgLPgU*N8(qW4N*;A9Mt zQ-#b76>6bI*O)uWGiD^S1_H{Uk?+kJtnbSIw!v^SpT=rA$9;r^tN;iWH3Tyjx=+=Rcv$shoOwX9> zrkF4*bXu3rOBfjbgA8Snh zxgq;m)h8)tEMRi!v7%*cG4uqug?HOI2d-5H2*0{q&ERC7!nS@i9giRqnqtK4{bPHv9)}n2tnhe- z_&`MFHVxD1$p8zE3$E)4l}Ije3It^9e5y%T6uP$e)REZMhCKATl<0Ko-`9@RO{)is zwQ}PK32*!j`d$@9(AiTeowsN}3j$OBqjboLQiP;TkFd`qKLS{{H_)#5ulBAl$^>3u z8K%S{oL$C2=;@Alnh>mn_qf)RmjOL%&YK&_oVuhpnPXI10JM*snt>W@1~5;l2G;HV zjB9r!>ngsA)SPXfVkwByy79&n;x2K*>pPjHp$1f4N6^OJtSN4Z*U}LcBq@I+KA7x6 z4cIVNX0<6Qu4`gV?e4#!=kJza^p)f(J|Cja4`f>ITtv~?;Um4M`B=!hrEl}w!k-U1ZrsDx~L6(kFpG8Ce&SRR|Lhy`t6ZWHcu;gRC`rmeM?-bLkuoYGG$DTgzq6k~&{{6nGDoR_b4J#8UB-`fg@B!J+ z%h1qZ3@R_NjMM0fuYHT^*S~;S`T8rM@F92R>vW-jQ#A)8KRvJ59v%$28T-i)(f z@wHuCZjpwv8Y*j!GjS@DN1D-YGLW%h5|2-XW&98oIdJ%yCz zxelANm3PiNzJ^Vie(>fN*`*X!d-JNRXRo6~4=sN!@OWyu!8hqcT9ev8r{+>_2Ti@6 zzZhU+ph*zcq&P@59gDKh2T%_=Ko8!z(UU69?Lvd@IG$jeJ?A6H@1|LJrD>gFn}pUD zvT`>Ow)3cakYaTAoi_*9P4pfAU)$e3xn+N?-Q+vV;p1<5;k^@gr|zQp#J>@w9?XeA zDuoxGl=o3JJu!@1xhR!9xmd+%<@`&CY|@tw3}h%X+RS@7ZgcUgpC5(O3w-!* z5-42IW1@>6=P`=)Urpz{H!-un(y@$;(zc^CuUbigIlkC%=xWMnOK<^{LW!@=^0+SB zXP*~>PrDIYR`qx7TR&_+jD?pZ|IqSp2gFZ5UMKOS&yjA>$hvj67QYqmWPyr>>_o z4#_=fms;soB6+8X;KPM`e0O=z+(ZD#h!2i~P0i=dnh@@X!pg$7)%nb)K02wEG|8L; zlDTQ!KQq|bgwUs)*E6%aXZMH^CH|UX$r@hYGPWcQNV-9Ssk>qSgCQK9k$H2j5;JV9 zPMqAkby!&WX^R;W^s^et);O){y?llr~VUZ&U}HlUj~t+u_PFe5rJu=V<<+Ag!<1yny! z)iWi=kEX($m|y!tCD+vUJQjj-$J^@8*eu%5@u@at z!hKPA&>9~&&}oEXHXhwgI?>zWh!|Btl!<$Gp)|=QhdDgK{xXK>#{uODn6>_WydvpC zd^+o$Wzn2*8souwYm}mFkXR&Wq8lI_lmiml=h+#e#8g#txGS#_>SA`R+8!@1jbVGR z&r4UX)m4pR_sjtLel|PNVLtNQULBVYgxqqgs~*~EychXwHoxFd{2Of9bsq_H47OEW z)zy6bNvSa=3?IJ~=Wdz+rOVtK=sZ0M)R)HR=$fM{sHJ-3;dg<(f7l+cR^xT zMf~u!Q2LL|w$W-tRe^&CnX7tt=5Rcfm=jfche?{=9OGNb!^(k^SCb|$U}8>86;S$3 z27MfZI6LvzyR)pTLi8CVppeMKM{PA6c}T(6k0h$N$1fCLe%CYGU9gW+#LMS7glARB z>9Nwk^Xn)Zy=s zu9hjutUbM0zgT0e2>55UGPwft(AOqX<)Ip&ladE)dTiYo2b>dOei^gw+gSW<)se(? z+&+7Z7@;dj->QA|1A26&(Iif0NFwT6^JzHs_HZo|dbG?b=B=xh{^G00=^#l zjm8~}Nm=KO>8n!{T=PO{@pMN@LF2>#5-aP8aFYqoXbien0e`Oqe!D%Rgss2q!kDgo z{-bzeTpR$jei!(3n(DrIZ9kzO#VD;21!m;OAZX;?w)3v7I8k#|2|`Bq9|UVCIg9iT zEQ$TC#5dAdc?IElYQi6TNuTal9q*brlKXcd)Xz99gBoQ|Q@FSIsD=~~A!0SAx^(Oh z{EYJ~8U0c8VK&GO=cX8%*qsVGahE0A`rvE|bjH?A5`%dig1eSIfYrOd*!{LKBo-93 zPJmN{W?H{;i*WJ6q-@zRspCIKjq6Eo2_3yBMhfQ~GO*;G*KAyL)4Vos_f|q-TR`a= z`8*gSo@_Hb7RH(J9M})hu8dS`8(y>leQ!EYxd=$HK%WPC@OC0J79B{$VnF2<1^$o);}du9emIe8r>C_ zwL3lsj{+9MQO8+_ck!;@@?nhS> zUkct-YV79i?SB6=h2|!8i9ilkeU@iH;ee5{O22H;8>U@YVbb6`&RnMyUjnufYRjz- zS@{|Prd2&|f-lsXqthWBKP2jQ2Qp{@G0ksC7o~$pCKl#T#W|rt95}nXH&j0^#E^I> zo3gJ^9I&-c#b7Ukzu7uL-m4?g#s&7P;=W#1=;G$?`-|Z}+s;tlScG@-h^xU}`ohuY zFu$W)o20o<8 z@*H9Bg3>ANTWCjeO9)}c89m6o-z~eM2G5kxsEGD!g&NH*-tV8f!@$|WZ>d?>N!0K6=vhMK zcSE%JX1hv*#c_xS-z(FL4SqT_{&pU^a^#e$6lAlnsk2iC`xME^!wG66ByTM`H8WNB z#gV(mzhlZ{pZFJZ>vc|?AS>HRSj=ZX|A+uJ3^CXIPzn6XG&9okZ59e%RUyv%eXjj| zRU5|oKIQs}O0%0m6@#u8uOL=B1O*OUr#^_Qx-F_UFxbs$yuQpfM)=`6`zw00xAOaQ zmH+f}<%Iy<-NerI{^>U&>Y-UOz(u=*>E!3fK7z{E&aO!VsPQF9SgqGiLNTJ^XzaU} z?0||^>ah?wv-7abuMKi#Jl($a8_&GJ{d38Av3Uq$K^bL?pSS+!H>9G*Z`i_EaYf6p z-^hzOgyURU)eHKE^V62_KNvR`24_DwjLyxC*!)b}& zi8GxR%S}-o8X`8I#kOa}0x6BVqrQ1ZeH~LT3KSw2-@0^WAnOo`nDv0#)wU9rg+O{4 z^(V0>*}H6^=IM{3V)6oR*h1GwQ-Mb*J#o`0GSX83Q8-wDO0_jr+xbT6WA&|IMv!)^ z4l7O@UkQkIp7h%91?eEN+=@=jhpE$P)D@`9fO0~0ZWb@BtelD8?|=opv8KGc`;aFw zw3~W|ceqWCKHRMf!6?lMK+?tr;7N8n%vE)qVY;39@Pg(v4uo*1X2{+G>fq+f6;^rI1M%fU8aGs52RXa^L>x z?T2~CFFc$~;BQLq=@bmyQgCmsJTtwfz9AkA-wJEzfl}xr6Ia#t?Gfb9tHz0;sTn)b ziVM6Lic2<6M_GA{a7Rs5SPWj|f(TbtVHg9m$ z_ZiD&AGXgUN&&IS$%1C{Mh7Uq-F~P zD6cuNtR1c22J}zK1{Z>{e^fEIc);#qiDmVyii~14->zyBy4WI}cN&}Y#tm_++h9Z0 z&Tx1ma;jH3Ke#omY*}OF!a7j8Z#r_o>0(+x$}@smS}8;}5RZ@p*9opu9t*_c^89hX z=79GBTfY|>hH5GcD1Vmw#VvzZehP4y% zkvcsS$Ls+Fr=8j!}VAZ#*}=CaCt zm6Cp7mTXZ1pPDY=1Z@B|v^GQ>bGc_4e#J{42-mESi$peeJ8T*92f=z6;Wb`{+cMG% z3qvqrD0g!qkC1>E2=fg&g3wml#rqhtmg538U-~4?JivGh1@){i_+2Nm$7x)K?Haa% zsz!a5N?RVBIovY~un%^;xL61;d$PWI0q0&-T)bIOxS80s6dXXLCpfLeRIiby69d+p zEirt-1V&z%+r~9zVUrOVwTWvSlXGuxTooCSELWe}Auml!#L)aMT+D#iRrGXvZLvS} zCcO+;{1cBHhHqN#8BZlXvoW)Nl6a8wN7$>f>hDo-Wm=^QqeH7M+q@xtM)L4qWvs5M zD+3aAL?vwk2i#&WLR+t7ty^vmZ@0UY78NcXG?Y;+Esm4ZE z=DVH;=i;w!Bkl*EG+K3+ROh!RsLFHP0h75S&kv-NDR)kHxRTlnftdbU zT{qG>r{l<;ttJio#Ouw9){X{gtN~x!vtDmAf=j+=7C?~%L67OZ4P9>s3pa!Fwuj)8 zKZG|?q)Wv+lPT%z8U(xmhHrDk#;0}6hY_g1sNsBq3uLP8z|`%Cr(e=_OJfXwFd#X= zPIWRkl3zF>^lRLlHYKG0Z4#-?#o;@$J(IK&;3h-FjGA#6) zpArE3&0X~3t6q2NcS)tfNr@KD!@U{}>$;4V;x|7^={aGZO4fIS&9uH)55CcO8i}Wa z%N78RWlq}1rdUT70sc$(ottD0^9*&v-l=L1`EE+EDAAtG^jAxTe>V^*>>eVOBU%UX zC6#}WOYi#HtG)0H6CAS#5t$AeEG%9_{iXagzen48vC~+$uA8Ni_|Dg1A5D^I=rX|U z$=JQOPPFUOv^vHM2Br2!WH=$dqJp5iD!!#Fm$ebj7D+*#E`zZ3g7dLDaWH07uaZqW z%3|3@?8xHT_rqRsYhACs&us%ogL9TX-QR5@`SJ55jZ*LMQOU=TSj_+4$<8uzay(Hv z>FF2cdZmjca;p7(z23c=o%TMyT11xLul6rU5#5QE% zWeeX1iwr$1UzSCdH6=R=!mM!*QvJ7=(nOkefa#Ra`1tic5Z!C5vO=_;X6xUK#5c4V zOS8ynGGnIYGRBr36<5@oQbO^tGf{pUpXthOThpX_t1V8*@m;m6Ks})n)7L;}|2v%$ zGwZoL!}NjY``U{Kf*DEZp@lC(4fT5Of$t0!bo1R|sbztOvfO5YiCYM51J-d||L@5S ztBBG7qK#HyXT82`<#ndpw5V@Y8S|7&$r3aH>q`_0>u2s*{`z4@g?n#44^fu~;mzdWp)Iu5t~p%m8n;>D@?m^>JZ+Ky z*8zy6FY9AjON0xlo86qTmWut(m)1kbtpMJs$&b=2qhg|huaf{-syF9bl8;yH36Ifd z`hNC3#6S317FOj(DX?74iIX%>{k%v9fKA6bwax7V>L%R6ITPa+_6y{^KOeuv_b>5=*~lr?5Y<)jC^`aSJgEF z59N&Jep`z#G~sD(TL1>nXWu#)5t5uBfVwnm!uc2S;N=eniMaj?eTeH9V#ZfcqXOdB!oB@yee;f z;xXmBFg4)Z9`xZxuz`p@0bNWmukp`Ji1qrVx8kii=`)Kc?s4r6zJ@PaFW+ddKZf*~ z6hfZTG!~4ZM!EP!nb?&bQ12qBZ9)d#45uXfB2 z8?PKCR@R#QGvOj}tz4+4ZUELcc%+lfVwehd1@CZPElg`2wqyR(X*wfabet$qCS&VN0m1Rpf~u56?|4IU=EG1JiS7q`6Eu$;4CE;!tu62m=$?`)ZL zjk^r-_Z>Qwr+{067`Qp!u)Yr~Yn3w~tcHRzC||szKt5y})WM?-4te81e@1~u<0esJ z|LKN`ppElGuyCF}@xuXrQDm2n*vvz5Iu!v(kJ42YeLRobeu2}&gH?s+hs>;L8$R#= z@^#~YTjn78n0N{iD{?=O^Yqd%8Wcyvu1VvkJu@pO^Zt0(mPHoPFHJS}rjC6XOOnoL z8ZEb;vf{x8GrM-%vRiruP*QccQl^4_L;{k*endQ*n#O9&4SIE|VK0Sxxdz@7+#?cj zIEA-W_AgLgnERCQ|2TWkuqL-|U3j@z!cq`i3k9WGiHHy>QbI`VC@m@qQlrwPOMn1L z5K&PPSc(WpiGaXDdT$}2_a16!385v_gaAoM;LE%B_n!Ukv)>=*oGbt4_00LqF*D{E zbB`^I??j`M)j>UlVJzr_aqtRhF6#zpLB*khTEM$nzc>VWG2Gk#2N@c_lkKXY5F z^%xNI!H!=a_7iUX8t`_@j~XL%r)0IOE;)Wj5EcH-y#X@$olq+KBBid#Y5PrL;23Q& zaktz+W6QLr%Y~qY-4H_SOCM?jKBvtr8(trl-5=85IUF!UU59RHOq&ED5J!835sm;T z;Z9Uk>nUK+>>*4kOv8wglakb?M%p(6ab5MfoW5OJAcu8i8mZ&->b>OdDk>GSvGES( z1o|z>VSeby+AUbs_B5%RbVvW4AJ8`a%1p?8fft-ewo9yC7{3DwKIuEoUYC2uJ*{Ua z?h~Z`So>mj;Z_`nxQ|VW=Rf!}h2t$qC7?;?J%oObthST=a7#k=2U|v>JA6}47K-kl z^eXU+4s%((1YLx>U*xg)T)V0VF7}Q~8T6E5@`P>Ts!!?sLm96{tyXb&l^D^x7@3)1 zuc%@~!0DHKQc^E!754^1?!=RnLYW(l685X{X#38za*Gw|PwA0{i$&}Zaw$B?gu{L8 z2{hY#XuMDA6IY zm18OA)uw_Kk9yY>XbsqZ1)00}tsa6xzhXmvWthXIZg`}}>N4FGY3ls=RO0O*$hF9b znCBh0T#m^1O`p|^Vc{EWm-B3r`hhb5szykuDe6SXLh5g-x;APb?rR4xUiGG)Nu~KF z@_2*nM6V+{X z5ui7sGo;wnX}xc;`HFF?b`xy^0^jNZ+|-FqGxn<0rf0~=Ql8M5I3Z@+Vv3fsMwijq zz2a99G<0`dsa$PHSWS0MQCIEQUpfv3Tt`??gP8y&INo_enRm#0MkSRM#x{|wDa|p+ z_ltaH^s{dsj{dcfbj_o5*h;0&V(Lp&5Xp$=q*Db>nJZU?%q7gs=3Y{&4-Sm%6~z3e zskF#S-t`8pQvlEpzjkV-RneD++V6ag?Sf+0ym1k>_qS4Zd)oDNY z9o;kJ*vbQG2b}1qB!H9S1++9XxsBoqWlY9h?0c+OC5dxmvZ)O^C8uTsUUJZmric8o z_7V$@Xk!>zvdV8^Zj1I>awAO>O8~(oTc)14>4jMvh}T)k6zxoGl=*5xiVw>GO>9!o z0E;iGUR>5hyfdqNA^RffH7aqklZLtf&A#ixhW?--zw|!FreMvx{CoM*gY9mF+^G#%t`0D~( z6n(MEzrQ({F%BrrgNC9~!^xy@Q#TIR>C z%AlSDpLKx?buX^~^i~4=YzRJws)98rN$V_?O>60~!R*NH5$%!rt=I<omE z%V#`KNwxf1{VWB`UhPBmVfQR;sAkx1B{bojzsd!O+4#Te(h zcoEIp=tHhJcqkLEuCzBh+EQt6zy0bnWV@C7BmRdu;p9w%;Mj3&sfqTb;DIh>t2D3o z>frfFcD!(9#gL_Z@a-lAIFlBtwk{$7d-3HoBv8n@_6ra8e518SYv^d*XSP@DFYK70 zlNF!GVl?!UlS~;H-QAhFk|YCr1x6gOsh*wkktXeqarbcma!E?|C^vDcsQVsykS9pI zeypph1mG8=Sah!SgyH>_%Ob5RHo*On)xwNkRqolN+ z{Kr}kLn&)yej(Q=#XkRRRu3sm490OI_<5E+>rXC_?`DiVR`9E1pC_JN_%Scd@H%pP zA&COzsrv(ER}^ZIuhY%eRN5K z(`ii7axUG7Xa^Dm4Q{2Oqb26sx6T3V8V%xZOWG|q@=Yx;hhr4fB8#TxPcp?_%rHwOr)9Rx3y|7sV}^ZuF}*+$=AqDW_q!Z_W^>=S zcReH0YQJd-k2%m_5VMvDiNPE6mcP|;k++`N+KkmA6&n<%8K0J~AD3Z%Y+I_icW!-Y703!#()tPvuoKdLSe{TK646?V(po)KC=7h`2p{Bb^n2{0>htePcn91JP z9VQ%0_GZY%v_33$;N?US_3avDv9`Z8bWQ7rFl;tSpTN>J&IX~fA*@l>)ud1*udVDs znov#fU`CC38M)VF>2E&8P$g?VDb#kYhtP}d|D5T`ZnQcO!m%M)Uzf}EHVE???n=m> z>YrPGf8*`8Z_Y1Jq=Lh%fUTl1*jD&TRD219`k+&laPRV)7vHZkLr=?*w`IpXT-n-x zN{OKlPMmT-vj_(6kcV}}m~qv@#SPztlg2JxZBK$HYTdcZ?+)ks3uQ6tC-Mf+ovSXd zJX_oIJ|WDGI9}CYJ1V06FDIQouyF5!9b?Vnd#^s z5@4Ek%2WcC3+&12^pX3$r0->aLq#yp(;B3^?DBx|x2pThp49c^A2&1xMZh!P&A(Q# zE7`PPtmL_%8vJHcj4PWEsWhI~oAn+i1>iJ@9_EjAaaVo{Qa-|g$>P%W zu_}rwyc~ethTv{K>=LEuIy7;)|I?8t9d{Jt6`Bhwe(zJQaSTXowmcwpaLHmy?&c@Y z&ExXp;_VFEQ2!ftOo|SpGh^U3*P+HaJ85&Jzi&(wZpSG22u#pTldNq%YTp5N79uSN zZ@@V7B5v+4I~4c95X#Fmby@)EcBuQvh!!`#`et4R zuvfl6y;AL+nLK#;h991P1G+n7e43wMUaEI#oEt6r8Cmo2mzisEoDO>-@BQ6K7XnK% zzijVaAIqCBXSNImukRL$wv)%3A<|#E3tPnkD3K!2uuT!!{+mgtkV_HzBPTrW#J`W| zO*^3f5pG{pJPvuoV?}A(L}yY+RqjBrY+?IKgQdN7DsZe($^C$OAE|Ohf#aiuX)Zp74I2@rq#Ut=r2f9Ea=WMQQUkjB+bOoeJT1?*!&-{=NZ`Ga|GqNUO78 zi%&O(L(-n(w$sOvVp4eZhRN`3xMI2?sGjlXw62@nVaQhXIIn4ZeB;ix-qA<1)o#$G?Z0ON_w26@^cg#C{uOhy`L(8g3E#04(;sRb zo{#yxqLHDqUos(@xAK=D6BOR@y%GZoLUd@QY|2`%Hutg86$w^JFJWNbnD^>k3E)B2 zA$Ro>Mk4^7rP{{qP))^=Z^)wYDaw1u@%;gFn{Mt&zwIQJ&8Xh9M&48xlDBbn`J-Wy z71nwu%+gzJ6yF=#4D)9lY2$vC*acb2D_VQ^S8BTYFhAZPdjN=R&CyrAb>h3`7^Yi(%g3p2s4wKy>PnqSuxZVES**ERu)iSKYlWMQ z8W~r^78Q!xQ0`Lbfj(n1ZMh3BtLL2O8R?3slKH4BrE@Q<{+SElN)A$w-(#5Xeek}9gp546yc($a%#)`CU-l-YloAQsFql#k8`9V0{AG)OLL|#-cEQTj*N1s&Om@lkeHwK_ePDYhfXabgYM}86w zSrnlV)6h5Jg^}gs=dLKkDrd!6VyytL;PYvHtb;Q1IXkydxKLtYlyuKYj*@+^3Ardy zBKB_d<{!QbuS_u)F&9IiQIHU#rGeD~e6OglP>ml~4u$pxfu<2dh3!Y1rfbto16e*% zK?~D1n6y(aeZ5|c+j>AgcRcM?RHHCN`~@|EsSrJ&exzMI@He|w61d)SRJuz~Jo?(h z@N~_mKVskdCfSgmcMB_sGpj!KpBIiCkyE}pbx!Bqr->uk9;`>7gdezE>;LW`1h}eI zaO6p>2nK0$Gp#h|Nl37f{O2m)&f7v0WfJnT9*1kMT@MAL?>ZFq51{RA7dyWyQy*hm znxs%(;O)9~SAhhZX7@cs5b2to;0A^f`OsGphx#-?}h3 z+lw#AmnHZczJI-c+bboFS{umraAb*q<|A1}2TsGl!#WJ2;gLM@5^EUsy{`ES0C-I&E zz2kYsK7mm=jFDTeE2Cj5z!vEmg+t7sdj~WN^Fpb-KGdcdxiU@4J3DoYcDpj zZZhs!zvTiy`q+gyOp!@hq7L-+56I7?I$)v*Ah=sv;N|@FK|`6kWB9ty>JAuxCqDU% zaFze{3ebg3W1$iA5tW69yGj)%=*DnBr+>?YXsOE^cBCgPe}9C^9EsxEokBbgeRBWXp7@I;ivM-s|Y5}uZ#mwy5vK6Y}a ziPmowXfrUqK(~|lF~7u@mx^8U=UZ$*y$zqI4`xE)H-=v4O+ft779;vvk8F{qII_Zh$kgO zFgZUe?%oDv?&jag_Vzxw--Q(133c|RVmY>np}P-|d&6HIYmU$|_;oLZG;wX?)!nEhSC-zqE6E%b9$5flruYqX>}h~@L%24Zy>*TdUn%Ek=F|HX->(RV zpNw72Z(vsQnt~L1_jKxrnS`dI0?Wk?0Adc{>tw6-HF!X`PnVn!(oLXqF!K$S?z>qL zqQ5U3QUuzy0B9q;C-y@lQC%Yv9na39{9?Dm&yFnatF^^=EdCiEwf_uNP?)~+LS%my zii{s(xAr~S<%-)ct1wP4!$Iz9Xa1^lR{Hov-}Sxfu{|5noq(IU=+q*3pVRA#;(JI+9j z*(~b`3HMG3FN81DN5+8X7@y)9l(wc5PjHR!P>JZunDVh)0|vHiKGW_tOGkOBLrZ}f z-*x%PIPFxpXe)G|nSu#WvBa z7eKvcu_dV_v1V(X*SmF`UwVx&XB-gNitXK2qT_OxfQLRmMGrguvYd;!qWNyag4%3%kwiKHf^7{BV2^;?Ken$#LTV z{OYA#xX0Dl-j!JwjTKF17d>g=*IgP{z(}kC)pe;-l|HNs5c=`!(CoX@zxG^R?$QC; zHRxsopJYFkN|*ZcmBnAyM{6kWL}adwP|RA>4M$bQd8Tje3*7JEgsVZojDFuKUGhz?Ey{j%LI$mPZNNVG_tc|QxREvwenoRfpq9EK*?FkAyfKXG~ z7<#|dA&v=L$~Zz{Mw%i-0#=xaNU?p=Y3{6uXQbbYF~OIfQvH-Kn>ImggVGy6`wSaQ zakN~6mOtvsPS$yl$%6)aD~>==aKEK}NJqSNReBEEh+#y?F|VStq`|aP0dkkgrm2sQ zdyjv%=Oh&3#$1GKlx7g`v}?)i7gxo;cR+$fAEG1G5GC~=U^wEn?ozBr(YRM5>crLWW`d*%~(mDrKd>+Yu7dBU-Pi%s=1K|bcwmuEbXoMpC zjFZLboR()FX3EcEOU4sM87@%Y?s{lDy!t~p9Dcc{YLXG?-8|z# zl`}Cr zopA;0|JvXq&@S7IcE#M`CKz{1;f}_ ze!NmCsJA|jt14=(fcsq)PQ{PdllpWER?AKYJffU_hNZSewv#%PMlq^2s6^fs zXR_Vs4GU($e;!*3`}Q?QB11hX(DHnO%#DWTuXXA8%W&jqz=Ax3ZkxNOBCfwU7>c5; z9}K%O+B$i1**g@59JiGUr zDnxMD@Yd>+s+d{X(=ixqnd|`6y*>iryR9+*G`D#!>h2q0wWF`-f$|e}gbO0B7wuIx zt96&C;+PlW_t7lrOwBcUP-#PKA;aay_Mqj=lN-`DO5!;C+x?3}Y}nUe>2m`+kKti` z@Rvr0z@?PFV`mXBlc4#9CGt=X=UfJ;CSdJlcIgO8%nLb}Vqd-FYR>}pE({Frrvosv;i!0d;J7k^-en@&??{Eeq!qy=g zGO7D^eNK{aWi<{^=Ows3`MubY*&*mOf$|_ z#^;j`)qz&iJo9Bs?#pwwk2x~lbN$ow#K~}*jQRoX^;(ZIPhn*w@ z08RD!iEbJE!t%d@wLcd|1?&7dS-3fn5dq<)6vj3|V4ypThCc@il~XnlISuUh*qu?{ z6QWe6F!eJ>k)3?I)Xm5ZmktXoZu0Du4t9qghS;l1P1cv2W zhX{|4W@+`0KQZOgaSwvOCCHxhKgVJ}=5=?Rvw8qM@XV*5^BkGe5k?SO;9+gKnLBf( z3!5&h-=|;R1#b777L7!c{;~~!(;{E9LN%8E2wAxyGJ4A6zVoK${HOW1ntB`M)pyfN z9ws5nBzP;afQcJQRtlhQk!jT6aXTMSrgVattIyL65qMHa&Fxq*uI^QGwt^P#SRGiF zeH<-%EGU?Kd?ryCaAW#*`*>UYnw zKV%y|hz8;HvUKpm{t=SAMBJcT$ev|9pZGVp%S>|5FDWU z>_D2g*&ZvED#fAwsc_N@pR}>`UDa6rYt^9>rfwM4=AF60j4J2w1h?=w9wOTu@(p7U z7h#+lDMWhj)tZm!hnCgWkoHah`LPA8mFo>dw+KZ`^$`ZM)(Uy>Q%^!)3$z7I`1uerHB(7MpWFe<<1(4?(w9}MC+xlB2#4AVmK7glTy{f@R=y)7<1oG zuIfI)PI}Luqo@cvxvP43O9$Ns&XJJWpBjOT6lM%7PW6w#NjRz%ALL9kY_)SgsnI>d z1&-T4-X7jsSwGBV?b z$kdX@rl-f{Y|v|0h1apB^PbKBE9Hu)J!28>F6fukD?j=1!6FK+ z24HJ=_x7p2mWK#Lqfqy>NK_9RpMW63d2S{Ff6GO0ecrkVB@W*RjEE+^zC=jX35q9?2gaBLVnOx>LQ{WR@dRJ;&C4l11 zRgvZ`pNAeRJKJ687k?53RmI?A zjQvsn0SJDhj(1P7(Bp$;%^m)a{LQc2xKf;zH@I7x$$MEDKh!YpY44Qlr^2Ga2}OY! zA%uG`<3R*3H@JWf>SHDcR_AxsB@*IDb^+UhnO7c%c76Vsv(vLc+;!e2&`h;ws$RM# z(_j5=&_J}bo>rLr2EQ~XN=?2}RXE@*J4^UVJ$t^1TJn}r9CJ}^vtvm0B&n%t^~H;A z({bir@YM<3#dam!#=F*_7AGpyphY_+g_z|H4KXo84Lg-e|J~b?SIF+Bmdx0~Tb3^> zCvjgIC*LynIHxmV>*YYGHkj#_K20AhutR^R%!=E_pKgTRRz5lW>FnM>txu|g5Ak$O zef{?LY{0$qS&o`75wfAD@azL6ZyR%mVcgIVnRb#;Y*YkNUUO7D-hbY9Wo4z$MGGtjs00r1dq#WMA$C~l&~8T(4oP_1dVhT}Kve_O!5Uu23*ZL}!bVGggS z4_RaFdQCcdq|}X|B@7tcr2>Mx@--+(etpR(9BWiZJY|_&e@(~XK;8>Ry$I=$U1&@S z&f{ykCZ0GRE8cpbY~eLm8wH*AfYidfO2F+q%MKf3n@Zb_t>mbG+V5W9buo*ffzQrp=+4pN!_4GwVc_Zn2t6-_9v2 zUQIXcT~Q<}RC)MO-llFpX*CGYRvM6vJ~l}NI-2O0iC}{le=$sA7OMB+3*f04DmKrF z3qMgN(R(dz$RR0?EZ->xUXm6bw5l;_Rp_0OA-XFox%8yRGiIG>CtWl4SLpb=w=?0U z)8!d8(*`F;zbVo!7lhGI_iFAjPDj%s#IeW#?PC()9wN`%Qyz@ARk}Pq^Hl(NkJ0Iak?EK4>KS(v1wD>m?p9%g_qpvS~%Cyh_IzUN%*`v+?=HE|`SUh;l zfF>(HEeKr>vee&ynuiYHP7OT;cp_;D-aoX>Fx+mX!W$)YO9jb-rx{(T>_sf=A%5kW zY-p9M0VB^$@3As3Q6fGXH48wk?H2C8II^D5yYv$nWyuwRwK>9&MLs{fuw+S>&hOHw zie(=qwInB<$mfb>19Q{|{v8e3ygay@C7RizZjBN%ky5okI$(^0rL?DLd`?j$n@uTs zB67_(7|KzGZvLGeUD0y|Zb(M^bf@{(ribXRzO@l4MmTIerxp}*!@n~B<5@2~cj6Q7EsuyyKWhXKEFh=5|TrYufTvh=HYyQy=So(UqvsDtBxCoQRbDR?B@K zHa*L8SP#)B(%St9Ft3MSa8OR|_~_O7i?JD7#h#TpY#dqJVF-~|TE81O#c>F*ulyLv z_T)%Jl}7UXr~F4(Lm9bP(dd10$$Ey z%_{W#;xUvU7gWuRvt-fe9lsw5uQ;dqxY(972IrR%YPdZgB*FHBbCht10^?CMj8(e+ zH!<6TjmvunW4U%`h?nIFAPejhmSUIWvhK(i6o};ITM~1=@R*vOcrz&NX9%0jbVobk z6Sp`Y?Ia`CJgS!x3PhEsngV>Bi+KeXCtQW!O46W_?bJ-@hStAA8 z{c1~NwL8Be=yvuWFiZ>lzua5}UZLDU-F))Vhyta%LQkrc9=SyyXf>ZcvA$7hVCC;R zU}A;1w~os!{9=hFhYad)8k7jG6p(vDh;s?`liHwyf5*k1R+ha}TFSO^TLd5Rw*a#G z$HGlNMZ9^XM{<=;EG)E)p#z_4-_rqs2i>;}wHL7TffDtgXAaD+K4Jx;wG_ui>StVn z(#+;#E9oa<9v~@ZLE~bDT-VPh&9+p40q3AH;Mv#|%5RmeK6j?M7s256qKH4Cs7cUG zyhf;gXa%HE(;x$828zpSE6WCH(e4|dHaZ0)jlAQjyd|S8*+uj57Z;CRpvityncj-8Hws|iA%&9e3n1pICMEu@EPthpoY+%8fvUNi(KEBrFeqZrp{H8~y|x576&8^WfiZ(9sY`X~5~*r~cZK?@ypwAM z_U|`qk9^?UY+wETt8LTxG5nZ%k;9X=`sgruA${Qhv6U15XdpipW78*kUPvo>PLjkb zIySf7O{|Mg$xZ4yCg^O+f8Dut75-N&(V3CZJ#nZjTD#NpA^-ZXQ-Z`+dTu!W=!b2u zlJx-Yi!PEw`+p5wWi5uJER0H`?=`yAOsN6^$hcfBvZ2P|o1KE(xMnqbHTz!|<^Nx$N+n^^P{7rD<5?px zI!Dg}zQNYJ9jbFiVuV^h2|K(!e=ow9Q60Pb)ie|FdB@$~qk4rgADOf>{->h$!P?ku z&+*DEma>cH~rk(q56(2W4crHBO7hTwTo3FN8 z0^h%~CbwKFxiEThV8q*)8kqYacyJ2t72M8HInrSnuUcso5fZk}GM646oA*xP*ImIn zyA*4<{hXln#H7p)%^c(|r_BcLZXO6__Tldp0VR_qZr1vN`}Hb-6~KS0%m31!HAk_r zdQy=;d$#Disb>)@NkqKCDC7QG_tEnMK#+U~#T z-So$>k$d^afB5BVO&Hh~ZREx{XGubF`k%7eer`TOxS1x}=`>LQi8z&TG*zq;O zq^X!FRm`)RD)91pB#-4qvcY1YdhKidi8}O=)s36`d*57H};Mr09 zAfp+*)9V7V&B?;3J~{^-L)2NIn(s;H^7wx*Vp2iL*12!W0}fha-7ST#1x&0~{O9HiIV)@p?cQFwWt#(X&#e|eq?mWba=C=Q z_7+j_W{iBWBZ~f}jb)Cnx(|`vpI*|e%$0Xp8L4cZUi}=Z(@qH=pCN5dy;JmZ*+1_- zTpc+b70Ux{28LtTZ_7K&ZjL6$d@<0}&N>ns&=|Pa+Dg`2XH0D`hx4-w1q#1=z2WTh z2h-t8;UZzHF{?5Eo6b+26q2vey^|1eJzKk)Tw0((Dvtt8TL;#5uF(H-q~EC&H>!sj z@M7gXSroy??8>fG)x|X53Ump&KE@=} zSm-=;$E@Wut*C>KmMk%r$~xO|w@W*1x+trxFyHW)=Y@kb?4ekw6hJa`QE_AS5xOCJ z&*Bd}kHvWuG$}~3CVEu}`H|#0I{odkhrd zTIBTt2+B*0FbOnIDxoBJ>il$wxGevD>H@!&py`X}3x59;xl&U&Pk$je$Nz@=MYo{f zH~LS>opkVMadfqrVt-pHJ=HUNP>^=4H({~cG(}q@jGh$YI-pAstyKTc8k?Zv_|>ht z73i>cxC@Nk$*79s>u?;0ojXk9ZIq$4>u<{V!N78rMZ%z)ilSmTEIEZh(oInuty+!0 zu0X@gVbN(%;1>ic`#*N*KN_kc9ll3-%03p+k{^88gQ}7QJQn%W^Dk;3-~q_QVCip@ z^+N;`9ky#JLxX1*THf^KUoBykw8N8v#JosrzdLpHdgWQJDFfCYykF&WYRA3D1tI?( zTD+Ufmi`pADgRyAv;2p0Qn@X2`G18}ZX{F9K~97h9CO-_Pko^S7NNOBbLw3F*WDkz z1hl8`cm7CK;L{tsFq>=w0JW&NlAX&wI}_gMQ(nfdn3}EV}5eZn=2k#>h!z+O5SLlu2vKt4w$9RC|2fKvoFm8~$&v>{@TT zK(Qf3mwlt#3|o%9>9YIy#B>((RsfbxAPXk?t`YfPn^Zk19b1JATXB-LV7F2bf219 z#ZRFBpT_^k_NavZ;~h(F-~fS=9YF31`Rt;0QUA}w1-gg{_Tlcj`1Xwaf1W@7onZfc zU;g_iBATxX%rsnKg^ax8vtn~Ka8dR9lv=IbzpT&y$uR%4fK~G9MFJZU`P=T9hX0#> z|Km)Byd#E4BM-y`t&GRowH*-_TMuoN>{|=r@kO3?jy42tj$tPM-8t;vOZm^~{mW}A zfx`dLp)N-_`(IlBH}~{!`u&e5B;JRq9Rn{E$JA~S?rnJQ?77s;>r{c7Aghp;iXOA3cCnd| z6z4h=_$M$0T=)%E_yA%47PS|rqz|Tjf!84#DUWnk4?wp}a4ag8Z8ARfD(idyp4`}i^*ONd8iGExIf-2DHlF0_r zl;=a$_Z?JMpqO!Gd)lDWag4jssIAN=l)oc*c>-ufLU`2dY$55y!}sR6f444gwU?EDYwkUCVXXi}cK?sZH# z;;eifVi+igBcA3d!|a^4=8YM51HdtqltBX$+wO-8wOxeH<6xHMppqCf;KxIKP3-PKTXb_;jts2=31rpdT45dL~`m<901MEq_n_-@hWwIUd-tT0^2`$9}R)+YcvOIK;p88;s(1}jKf;sN}0QP?*xriuic^0N!i zvIzYsjN<-jhU=vM;SI{;bTaOLm4G-kmugvM60uKGJ+4W@1UUE>5zcgtpvEXgo~se@ z?06<4(o!Qg#u7!2ko5G=9Hth}YQbL%5by@lav%mqP514C9!bN%uNo{1yi^4lViaK* zUQ1z+N#+m`v$vtk%p>w0hg_Iq79d)9T1vpzm{6_G;k3B*M2)5cDiTl(la>_vqvLL& ztCHG@)P{@30O0%+&HZHpBgv(5{(8f)&Oc5X2F<(GBTNv34sX-D#_N7WTB3f&#B*QC zU&7I2Xi~j-P1!FroQ1O6UYpAPgcTrWFUmgb5U)gtk2TERh*vg*@S8rR+qk*#yItVh zKZTNlUz^u3#Zy;2jwlo!FOc(?KY1jTCbc$1Kw(b#G4uB>x`CCvN*YQGA$I9G1&{L; zyeh)tM6lWy;jABqO!5bR-fUubC+)TNV~wA=sQWT-T_)Sf*ng0@F{bU*%OyXd6?4yU zkf|utPSQ>#(r|G@^$5Va6nz z@e&OS<-C+12gEg|a64hvNTLWGu=+(7tbml>%-qO0^tG;3uJFBtck9@Yih_O}QjD}$ z?YEbMJ-$!6p4?YuDdOE4iFs`62L?+_S@NltGIls)%PI}DMJgq3#JSTQ&%Up-_2H`- z=j>zdYv3E5AlSE++Y-PU`0mz>?rz=8ez?uP=t1S=1IqWTWX$}JslL?As_FEZ4TJE= zl%~Yq_gTk0`w;yo$5+%DGe?nGvL1cB~TfpxidG} zh^lx5=Qy{?6HQeZ_{TrX_+6oJ&IC=lu=85d)2-tCSsdc7vztab^lM`jo(uBgm0B5$ zyJmgdNQZBQ>-*F0Y4W-TU5#n6JV#42L)yha`jsHtbD_+bJ;i+*wr|(D%&`Z+9@0xd za1o8mXC>bn2ET06O!k0Jn4drZ1UL;J&1X7zWaCIyb&FdLcO+H0Kv+_!2IAa&PAUBOyruzLKR(-x0%=J}EdA`UpV+bHyR3Hp7ODnX0L!1j#C|ig< z>XkXMy)iAkz8_B`lDF1=1 zuI5hw*i>_uNU1$$wi$q?hFX2_f{;|@El7?pn-ZA~S7h2RhQOP|kISa9e{ zV!dl`E2-Q<;PqTqy{Xr+(v6-;~PjJ0 z3*t#ka91{*?MgkF$IKJ8_*np`x$?$X8|mMPh^#CFadHAfKGGSH1vNQ+>d<^1^$#!n zCt+jZNyl6f3261OWtJssD6UW;CiRI=mdQLV!m$9?U&vH3rk)hrBZ)8`V$rT+zvyu1 zEK`HCU9?}D%V2YE@P>MJTqrkX*U5|risFxJx*}9kUJM1%N0*xN#V^VDceE=~-k1k5 zyRmfUT_mzVgVAY*bT=5gxSp`q(Jt(U%4A#Y<NbY>l*PH@2 zo6oSsDgwMv029pMo6-vZnui4L^ZDwp*&k0zO5z7EjEc<1R+9ow#;{A=TA~2Evo;^p zYUcr-dxtEmQEo37ze5G!N+8pjo6z~&I$MWhRLB45`2bxtjd!SG{}x}stpAB+&WF4B z&2N}xI-c7((etwjd<@bW3oNxn|9>=ncUY3|`*vk&<)A*6mW#^LtQ1#np)#{_P+6H1 zQ**DJsEDbVnX6KBOF1(KDvn&Z62pyq?}Zx?1q6BdzQ^(Vm*e>Zo`d_k&+EF*fsKJ; zQz{nO$>FMrbH7oD))a&tmcIC#g9rR>1mW3I?1C`N$NS~ZEOGZ1OaT3Gz_y#L5rA`) zb!~aYJ7C7B&_bIUW7lIr4Al`2vw2q9K`EToXiy}TF^`()+N@-fmJI|FIblN*cMJMK zEr|w?jsxONW(uS2mg5pB`P=lZZMC}pO$Um6`Z=}|iV+yVrA)ZJE~9)U*}bVHDRcfD z+S@JtSe&64-Q!8NR$>-F#o}gmvr(GiMQWMCB;LIvU0q0G2+F?FNFV$@yHn;`iw)*s zXR7y4`9Ax+vY2*T$>pi#dLmh(o(-1vI7OFsZt7_fiou0!vl{9Hqcc)Rg-lW|`EQ(i zsmo{R65b4~#ZI5-j+2SIHgmz)MfPEvINYas?qpqBkdhB7pu7-yXU-X%PWvVcG#@%D z2m=?2&b*wC`ECsEFhWb;GFR^^lL)j63Q$rmUYadF{kz>}m#gvLvd6&BcdAE2`69?6 z&M<+@3KC#i8kXZu@7^tAR4AI@p}2&)`PoaBRgzNRw%72XUL|bvs6!CtgVYg`7PUw% z{zEWN$ch2i?;V$NUklcMxEUrp-XDodXMC@p#fTPF)E4I0N@|0}R$qX@#FF$L+lRpE zB5fo8Vsw2T{$4w3Ab(C$fW0m+@AxfS+kXM1Dz0Uz5w1VG@i=O*f8!Ux?F&2%d`bj;{gK)z5w zDZ6vIJq8cViUQQ-P3{x|8ef<#pha(|^>x*I*0NWU778uTJ_=Z}^|=Kq;v{JuUDP-~ z(FuM92K3)-cEZ>*5KoMa14;TCGM~wfTSDCGKbjULNVz~=Iy4|$EST~eb~-0!F|V1WJ*gx5(`7V z74ieA&wHOh{PBULC6kyU(MG{k(()jL71gMjbR6PO9LV{?Phxa7($^J18Ing`t-Xdy z;%~k0Ou7tG`-?TjNem48VQ$)S8TS9xO^yg-EWxxs@r&0=i)@rjWfC5T4Sn3mmOAp; zVJkb-niombx<*!n-UMiJZ&9jhZYeUDz4F~c2_{!Zs2|0- zyFN29Meb%I!ooOFrMpJ_nmc$DmiSsLeV09FXba3w>b)?b>xa`-eAQM*yF0@H%3E5u zr`+3o+2Fm8!o`d$)yqEMN?KcAURgWJG?|{XNshjMfeM@w!Kf5-~4S55BVk z@Ted~^{TtKMA_5mndMUAiKHiawL8Ti)UqRLYs0y5YrXf)T5HVFT#l{1o;1=Wc_{_P z{Kri!rMrwPuD?1W%;={g)R*h}jTpc4VO8C}JXQ37`5!UCRO&-TN{CR;lWa|j$vP>m zqTO0H^-hetQsRI-fc_XLcCxFb_VbNl@8710jy+`+rH5a~a z%svPF$yZ>Q@XHn(MoLRqItSXq7y~|u0%G+hejV=A zJ})rIP!LSy1Wgl*HEFuApo%f`pxRgbpi2b3!a}~;2T5CVefcXFK|X}=xFqh4ftA)q!oa^-k{=Pt!M&f|rZ| zVjRerq=9Dh@6;v$eHLRMxWA$rNVkXMF!spr_F92M#(@+Jl)ea6YaIH_s<Bj6|1%$7ybRWj^*W)1J*2{ z_x~?#H{QawmH?Y~^OPAgtd)u0%@l=2i(>?2FY5#dzCQ>hS8&$~A$|&hD^OJiT+DI- z{U+>4U}NuCXZ=QJh8EjsVPMQlbHz){mh@Pm4=S>D9z`=j1$|NkXRw>}0`_ECm&Maa z)#HW?X!AOa_`%!#)B`uzVALF3@33RlniNDcPH)&+Z#eqt`j>5D#<-Opv=f0?)#k4) z*PuuByF`%5jXO~hM`Zn8#%*pOVV(WU#Q>!5Fe#8{Ion}m$p}#4PJQT(=rfy=Pkjsm zL`-edEVqO9`$6P#?%+?x?=ElU$#@rcK^!0pijeh^c(w+k+f{ z296SyKytOJCjCsZ`p)>QF_JREkJuH+Tw=S+1<~zAJsK$!M+Y_a-WWBN5{%9UDJ&U& zZ~*Vc%uu^$)Z;4m9^|_Vhox~xA6h@93%b)@Mf~}ZwjaHPfvk7|sLRpfRNQbJ^G_V) zuo+07Yi*0($yL^{{tskx_0sNY!jcyC@z6+5d~Xjb7PFZ)v7LrHmZUbr;sEb#BwEs5 z^4p}yZ!W>+F7!q_(k+8VN6GTQ!*}-Va0+*ii+;i!b`wLWVyQj7pgF+NPF&9+&=MDY z5Y%`82~0E&s?>(5OgAj*HSCSmVJGTxY zWGBsjqZIwOLdxBE9EVrIV5^%=+?||CEA@U5c>CZ8=+ShzhNg{&MSGOrf$2ex1i&P@ zE&ZVW_18|Yg?UbkXyfrL%E?9(SP+?g;#2mW;^YqQ$4hB!K#IMZ$Yn|v%`w#jfZ|Iz z+f49w;OheujJ~k4y`Mi~Q)-;UOnVO2z*r>FU_0RvWofZqamTku!&ni)e|5zf_t5aL z;ASBt;R;tb=QxUg@8D!Ss}(|D;K=AKK@|3+#xOhjho5#hBs6axtl|#{reSMwbCN>6 zP(}_E{O`VvYdWvyT*Ll){niz(O$~|(OlJ{Qlh-XYvlC+py1n#&T!0zxzM;&&(|4VM zm}+dNu$8hVp}F=#C8}Xu`^bK0F!udfkYeoLtfS@VrkYPYnFvabpmEGhCFULZ=7J&h zIkHr!LaG7^zO!$bngWeL_+{@(p^a#^FVHf@?_$(|d(A*k2QxD2-9ORe=zTYRNEIt6 zG6_-8dNlH8U34H(L@I_wC0xnuHgZuOZqD%f7UsC}&D%fR7?ImrlXFMae|xuN+gEk6 z&7J;aY%HzpU?g}!1)z}8n5?&Yw{bBakb;}-;hfEe+QWwx_PiV)q;%nR(?r$@%&y*} zNt)=X>en92jy*pD<%FKeim=RJLr6Sx9z^SJO+A{Q1&E~))KngFyTx7E zz8+W`G;AN-bG%UgVEvTBo}dJa(iayS+mv8^4LtjNNH5(h!XTQU<^@G_?b*p00y!;=%Tf8G(n4fXl)1MZrQzJADCv z#k4dV-go=CG=(XB=Sx|YuW z9l5A2ITe^9j@q-{PUwZfdtZmAp`C-hE=~8m{VlNWy$n?#rM%0K?WdLKipTu9)`@(1 z`1&x{Ui}N`A$7#$=GZWDRSFE8)ds@?oV@;wKV%Ub<`6Sq;-0LB4MVM`R9%mHgm(TZ zE*IFqykxX!E!rjOa{l%&Ha<7j`xp5sjDu#Dxffo##!1x`*=H0_?ZR$6ihRz$qq$at zS}DcLJ=?P$OHvi*u@N~Nac=dDa&yZ$%f4*P*$?Zati~QAIVmIhaUk{HN|YOwUr@9* zJ^~6_pBsK8D&42gJNJ{KkcQaf%20kxa7+g>%Ycx2(Jo1Y<-t8FJ*9KfMVP~Vg!`sEH0E4$yf{vQ=5Y9i1M7zqq$#k$SU9)#cxZXELHny%= zTKFGOAA?$Nud-j%P;RM7+^Hy2knJ}c>j-V25@Yjt>#f?wmNWqbc9f{C@^nJVuSbd= z-}!9WTE1|5a3;rkQj}sa!;pzM$^+GX7)o}WGGYzKQlMq5=OCs6x2BNKAQ3%ms|=OZ zn}gahb!Wb+@b~cOYJj)Q5t{u$piJ7`+ffD)51apDbQ|s@*WBgpO{(p#9KN;I-B~5x;4y;lRv&VikjUSTHP&Q8L0%Y&wbxhVM4kmq#5br2H#8>GOXK%6I0y8S?ZBm)ikS?DaTEe#~k6+p|AK|Y|++!gQ{nE)d zhIg*aa18UP9C=^-<~9U9Y~wo|RR(;N?CBE9o+q<0+;zZi`Eig3xFueO+EaDX%OMuW zQ{}?7Vnn5>smsG|@16K7EFjL>P5K`n{)w2KePh_ofa|)M*pBCW@DyjKc9&|@R=8W) zlg9ZFb5Osj#KXoaKZY+mzPr=MmWow|X$Eu`2rk>L$ZQePSkcr-3L%+K* z&?rwIJQ`6Q`14lQN>^d$&i68I)(~)~ezd|*8FH{I>gJ5JxjYjPUDpj8W(1AKZ}-tg zPjo|%2BARPalLHX^!DMAc8b0-L1%h=hRyMSkm9-j2*qs3s}x4qyC1g$D7emQtM-@) zV=}_^gH{)hM`%OddJmgZL!l5{l$gsA&YiyOl{((4%e%1_YU-J@Xa+0e33fdr1QQzm z_f}V4-f=`zE6~mYya@B+XR?7jmO5!KAk^T+hwV0`mjYz8SQEi4V1%+SuHWR&C2?m- zq?{1_@MFQD3qYZ`wJX9)Kee`qhEuF1gd1ieB)3+1m$R?#WdMthypD^BBa)ux6%{C| zT8?ud<>lV)##OCly&_jt8I;uf2(!w@=yB-s0;)b>olm(O-)`yMuRMhI-`gwdHeHF> z_T&{=Jri-%vG-Ll&{Fu@(%bWaLxg)zfbdnV=E2}W-QcboH{v;dVd%S)KzGnC2Ib!J z_4mPG&vS-wb-A9CI2n80e?HD)@k=DCGM#dvaR`hPc;}>fywQd6QvxBB+nz27W^Iee4ZR@a>ogY&$nmSV%KOT+NKWwBv-?$;wUhGlr8#M7jGa8e7ru z98d%1S+qBv3l=EgLyz=Cx@Pt$mun5*Tnv27J98;~fBzjUK;*DF+LhAo@}-4bI+DuK z2!2??FYdqhQ!&l_N5PaNL}5=)VT>AgS0d8UE8tR*tz6#KB;wOr*!o1*$70_-kF5q zx*^sjd6oG@%(lFB;gu+uN=$ZhI{k#>S)<1nO`h4%K|T@RUy5o!qw=|+f>;{mw_}C+ z4;~(d^?kp-4-u$?vYkN_P5xW>$euTsEW9tP*f8`J(jZ$4khvcaq~PCyq+QGBWK55< zb<@Czf~2+*d>3EQ`X)`NV!kWti7H&_J!$iA;|(C+q2(fbTG7Xkri`P#KAE{L?0%y5 zDSKsiy2bI`p{ZL&DSER@0mLVn&6DAhQlk~_U7ebBu#YHpsr)zWx5N6%1_Z5}KAuz7 zu!%{K@1h9Ssq}GN)Z#1qu&_o|5BTX4i!_f>R|#=84Qygnd!xF^`TnUSapi1&dG$qO z?jE~$_w1w2*?hbQ#GZ}^IYl{E|0!^`6T+y)0e~ zcYR;P%*Bo zC>$@rudtG#2~WH>%`l>Fk5XC7?|a{)QHT{>>;u=a{-zNL7L|2!8zc zw$8{h{$~iSWx+b?PkE$c(TFbLODjKvgoDcR4qb+s#?LD7WQQivawqErVP1x1J?HutWf$r92dG5w#U1p>UJ;k}0mLEIj znOy(^?nrjhmm-i<-5#cbf7w`oIRDhBgE&RiyA)Q2i%u&0XL?JlxR=&qgPAsUH)s&G z%ci(ECls`&_WD(FGNVJ5{axRH53GB^h8m+@rp`JlG@5AyI&gW;z}2!t91f>nZ96Vi zBfc)>3R2eG=VtV2+acd%w~8P`COUH$KbX}d@o((;G!3#|XUcjHVrAKQ1?!Wb&V6cp z;O8;3qF__*vUc@P2=Kr(?PW&NtN9R6(a$XB;+Kb2xDWZDCT*zTg}?(}! zPip$@73c1_hK&~}!%9A8SL>q2h|)-kKzDqPDMJ3XAoZDOcNkK0G-{Z}YguXFN&R4c zr0YJ#y?rx|`IC87-LtEC_*l=Q^pSuRT+)0N2uHdOeWEPhL;b5EQV+*N+r3K-S5!zi zp&q+`mD4-rF`BN!QU7e;y1xbG20XqQ8%RjehZ6~%AkVf4#pVm>@+u#bZw8Fe>dIS| z#JhfMKP}%ykdVKVaY4SGT@O^_@IyS8^>2Z7W3BeKl0cXTxGL{uFL2cf!t6l*_X#L}FYXSq zMW@}3B;#8IZ9qK-8M6CITn~)$)uH>=?8ZAcQ~h~I1DWBm;gOZG{G*%D1&)Sg&Re@Q z;H1fKYX3;ZonUezNiVRh5uK8c#9iO3%%_IUgkxNam>C1RHbQ5*Lf{hzA37xwpAEJz zF2hAZaEsez;!iVcUw)_m_--kV*xZ!dT6~SwpNUtFDo zG0lUGzi*3in%XDrd0z=b{}q{7Mk3ZmIr*J6_Gp6OdJi+ja#-V4SQ@zuCRQ%#;6j>q zYIyM_?w?%MeosmX@%wGnY0^cJv6XYqNY}e*QCHL}f{oqzr432v!fcxkKA{k>-^`c{ z+6&9Zs{Q}T;&{vYsvUd(Jom}$5VhSIc`Z{q7?zgP3PP2Il2{nO5%)T=q2_kh*Xlpl zuKlSC{BoBxzqKVRg;o+|9v7pIeypxKExk@*C#c5)uix`=~yDU6>GPh zvV^g->)sf{sKtJCd(U6usnK|`M(pZbn81zSPeUHnKT%7(+kMwc#l!Qaii6W@E?th| zdTPx!r#}q&!TCTrVi^BUsZ>_|_XE#;9t5uq2+6?0d7W zTgM`JJ{lw({3ka)VA44u%bDcXp>{3Mv)tWapxWex^tW_o@i9Zyf{8Pe| zFwi$tW8sP6xNB@deg4f&^6D5NDnQk?ufPK?=b}Ps6#;rr_SR+`#q+NvMsvZ%bSh?v z*AnImEeS@lmFuSYgl8I)}2uibD{a-mmSF@J{fqE;-e0u zI4*aOz49>*eQH|y9`D1c(z52SWnu3=fqq5Sk;No z;Cs5r-hzn0=}(lK3hyejKDx{0qIBqs$rVdW+lkwtwg|?eqeq0%T|GgcdqPLi$l3>H zU+@pbAbFFsXCmrdWzSfu!TO|D0TVCt(}T`9aMqGK&{8oS|4v4TOKMpyzBTg@)J+)B zwK^NEq}sbxY=fX&FvtjHzUf z-bKH8A|SeY9P@b5MX}N+=uy-unuzN00wl16R9@cgJ;E{+IjS~6bJGz{BaPiQxL=Z} zN{-#sNk?wPFjWeC9T?1CBEWI`%;2lPfY~`=Xs7N#^2%7AGGeY%w--J%Vk!e);Kb^9 zmxcU!DWVNCE5Ca^c1IH`I+4_!ua6OHy`n3UcWBz*@^(*BQ(dDCK)miVe_#3WN{7r< zLLpxin=P-8jF}Le>fmv!Q)2$hgZ`J+r-yvIlb$@A%X#F*@khY}P6-_^`!IX^`%B9+ zuf*otVS{ba6QM>d<@oplf#2V_v4KJ}3qZq6Ie^Z9v^`BMfLz{ZnQn-yk*jb7dHNID zB&`}gJOLN{K`{_t>UXOaM5a3WwVk@|wGNGhVl>S?KMxfGX|ti5Y8F};`5Kx4ZLucw zS+7f%`1o5|zo0rf+ZKRJKWR}GNWS^j0cRs1H5vBRa0wKd>4>s_>vdFRGVF!iNQsNM1*sLVn-1gD#-+>{S6EhD3-;C{r5)^q zz08YwY(Kqt&TT2zqyEc?8-*t`TkFko5M$9mJf0=lI2&Lh4>Kk9n&rg#$(Lad|MmnI zpC+`Kny%-Q4zgaQ+>s@zt65kwhxn3Ay9ueusOO)NOswDD<%xLv?h_7 zX*M#7APQ+=+k{3_8egxsjx6aMuPHu0{^@?6gH&2lo=Zz!8d7+}SDL}d8I{SveD%3g zQOwy7IwI{q12&#pDdjY2_>jy9O&sZj>$!E3$q6N)pf#t1#Ih-{XS zOYNfj?7xxaXzA?3omF00Dl+~og7P@RIvUO?aot%-Du_lurrtAf>RWK$a18}g9H|3J zruxhFc1l$=$f2sZc|JA-eUY0m_TJ;!7(X{dbExPxrl#Csb<74l>#$o^wi-Jn=DgfS zAf1KoN-dGY6w24q@LS>H>T8BtJD^%A-P#8O1!o2_XZzwm8hwtXojY2-zAwS zGfX4xY@9H5k_nGy1n(cGYtx2x8=#R85y#$*e*o&!3hKiHVih_ec=lQF#UfvvIL)A* zoyURuPhqu@s*h-X4S0pZaqnl<`JM7&*3*qkx-t2HvvbDo^ndUt-3I^G@6SjEQWrBl zRzu!My{%MK;F6lLLwI*>u9{643dvC}DKA|D4yw!;qnp3}JX|#!$P=KuEH=FH3?=xj zVY)Fb&teTL`ZDNqk(MHN>RCJWH9wDujUbw1bc0g-f=S0#!i;>zYQyTOTx_d2%Zbw&cur)VuLGx0!@flgQiKjo@5^s5#G+t?}{Iph~ z;8+@;=kp?$RNcYYnOCv{*ghpcWqvJa-m=f_J7f*PT+j=TM1cIwE#TlBL&Kc+Fd zaIJ`mi~XPi$tkNG*|axQ&4v|Ku}R4|2gitE88*7((|x(^&r7SHkmE7L{wm-tiG4qyxkiXHfrnhz z+w-vuy%gf0exq(6W4m1G6b$kJefqL!MmX2NISs2>|GD0RWn6^IWuHk^EoG6^rjJ{e z9mb@&p9bPRBarw1Qu*=KMzvz;64keROwMP(hvboK1sQPLXTicXv8Gxtw#T|9IW?}( z!s}%-k}O8o?gv5rn^qB2ohaYk`ll5lea3HRX~zzKG-L#=$`O@s-~rP?JRZz^$M1wP z8>id*hQeHWVdXef!#`rC3*sKL3;$w?KvZKqN37r7w7O=rd4jj$b_ro7Nj>pW@=50- z`7=EaBE)Dw;hjE1-CmFQ)+mX(c$tSy-Rh!G&|DIiO_a z$mQ=1#gPMzuJhTlv!hFi=81Dq`=t}t%Wt*LgcdQzlv@WL^5w0C6}MzZf$xGF1&?KM z_U0H)IVO31du3J{#RI|G-_;Q)F*qHn5u)P1v|uzY9~yKn#Xzq)bJRIs=)}0>C4<{% zJ~zOnk0pB&!}cPN#=cK(zH-x*VCHSk@~FDXDMX_7PF8HF9JcH|M~~UeFK$aZ(Yz=; zJq>MM$ZXaPbbuk4M|EwmvhL^X-$Nsc_w(mI&w|p=GvcGXOy|%Bj&;h%F5X&A490v7 z6v@>OaIs6sp9;lzb+x(BzdZ1IX4Ct&jht`k^#TuQSbfKnN&e8uf+zQw1J+b1H6|7y=V zfoP#+=cX@|z;hdvNj{>#yEUyAvbCb*Z%5-XUmhxpKr7AUi~n@zD2N~GIj51ntO@;< zbeYe^z#l~453c%cp)1m3Na9GwYrEt^?{_jDsUf|sc;s!~MKj=iiCWFc5pd zi_iKw9cVH#|BmThr8A-V$b8@<=-s;|(kErubqTE;gzB_IwF4x$?wqWu{OwGmeE7>? zYs=lbs)?uXWA&m7Eb%R`Fd2UpQ%R+z?Z};3eYMa#J*C_t3r1Qd|8Sj2uLvMXZu<>| z?_;ljY|g6@rL=c}8wvV&pHfHQsRoN9?%pF>cML7 z^ngbbiF+2L8Q6S1Kc>7GO$g_kHRafuPt2Gc;51i$GVP*8`C@PInXjI4%z1iyEWq%O zhhY!HcqSXBjH^!WdIx}P`*yMa&HTEf)@1w5PuRt;=t)JhU(TSR#kN40qlY!;rYCk50BVSu z`ult?g5@jsXpkWo=7snmYY+{rfX9Vj^TN+L+f{f2vrni8xH+nLO zmzUgUY4VNPXsp6GP3iT4Yr?s>LKUft_FmFrJDqlf$2e=8v^M2}3n=XK>DG*bKuW8- z*>*xwci+wFZ!1}i?>-`2e)TqqWZpV-5L&xu&I=NOkDVORO`LnY5Q!Rn0$aN7zt>f^ zgSm38hJc1dHonLl?a}=>mg~tBNsB@e^S7GZu*qB~nJIU-8()$Pn;(;$4S!gqtdn?x zVh*&3nnW|!H6x&#Vc=Xdj)AhUn_8C`d(Xz$4acyIr}|%*J3KgMh^3T|_HnK5;^;eT z1`t$FM>K$8<@S_gg?WEpKhkFH*Z*+=9)jgnLjS4c^;__s7p=RaLWppRz#)RF%MrW? z;GL{}O#ymg3lC2cbGk*Ua-xz(j=k__jBmN?E$X-p4$E1)BcB6~Xhoz-`Y3Nog0!_422 z`6gf;TCwTx$t7&PVl^N2P{f9f*fyXnZ$&m?z*(a>#kSL1Fyh-YjlMY29Inqc+j;ki zMa-4vhwgRf0nDchO5!nEaBm$ATBY;$StyB*Q=NcM4|6&%{Nvb@B@ZMFI`OXM@<625 zG1YY41s(U62e5FtyLE4GbP`{D16`VRkQ1!uOYn8E`*HI`@y&0F1x>LvZ{(WMQ}sR% z1;8dgv5RM|u1zFVhZjqpwfgsS0oQx&U*}+?jX=2JEQ>Kk{2s{|aFI{2y{=`LM6<=&A9eGmcJ$@TSG7~y>Ap@T$=XB#C!i0NCx5@s~ z*@|;RF}`kBQC(clNO6GJSKrN05m8IVi!{Z1{ccMyr36KlCtj;Q*R3V-@El}j7xMcg zPjHsbo(x+n1yNeOToR|uLiIgltNPpt&%|Ej4@l0g;@n3yC%cuvyTo7aKY(ieCMKG8 z(~`v*nSKFRE~BE?ESrX`Z(rI;D>v!ukJSj}1?Ar@y(GqY$*9UCe{H~P>A5Q>EWTvp zqx}KV9Ce*aUdpZw&#o_bk*I9yLdsaak3xGMNJg#x@l`p$LF^n|OTbbG_X>?tXC#>e zXIR+58>hP3iFH%Vyl&3F<8Hb^CX2<5{Rde_hy9WUqRn9iu0u&-oJJax6*L66crPAIp#fTl;K&H;CSPt*0@4Zz|7%s{|hj()<%4)nE! zL$7LcKH_v3w(~?5CwGPEFR8+d9g`2m;BTHc5=05}5@x2yaOArsYmZ!V`R96L<_MRQ z!o`{2!*h4zXO`_f$5fxZO5-0)!v~i3&kkvsT6OTqrmU98%JAOoJZd&ud>GcX?1z11 ztw_GC6q<$@x^!{(Nw>(XVKO62;J@3al7Xe`IcB0uI@$`^irZ2a2PMQXnL!NGSL96p zy-BFC$B^IKG|hQH`yV?t^Ky2e*9!(7zk3GVxqW#sUk#JwOZdxkgm?uays*?t?9doB z_7r!i<$l7p5}TpZLdM-%$`9-8zZa<5S zpj<)ZZ_$!8*Z*7GVqr=8Qc=m4Jp@q3Ug09i^A6mGTO;J_xAvjD`dI{L6ws_Ti>Lj* zmPR-DwnhX>%(Xn+=4%SlRS6S$MI%g0jSQnw4qeukxW|f&;8n*jn~ePjlZP~{zUDRD zXYd%%zs^+3laA`Y) zJyfar3)4S=BE`Go7~VQ9HVia{qH?8fb-f^-}#wWw-g`CCO4L->#?~B$WqO z#Ro{gce4qU8gMFfsd|iRSOesBjmg*az3QepiabPBdNYT`i?bi<3Ih&2%<)4H@7Q2o4m^N<}jIdnu# zx?`E&_QNr~9IrY(Tl5UCE1<e%kJCnP`5Uit%Y)@yo=3DRXgz zzZjJPRNCUW0XC2i4LE4vcyEFkM| zkBo`k9Mg|zRmQJ!UTYYHF-W!oLZ;q}JeE!`B>BLmv~z#fo^qEKaHM)Do%dL~Jr*xy z=&iT(5=<8UfFiuclsnwDhg~8ghit5LV=zxPYzb zmh$~B4dwUNN^|(V&Go{6)vnO@8PuF^>b)E$;2y86F4&#xIy^fvc578)$1vUf^xaWySn_68G=lX4f-d}V^zI(|MeL;dI7KuH0u zy>`J5`b~p1k!e4kSbgW~_v^TU&PbBfce$$q60a4ycK*f(=d63vzLz_C!BkQKE@AvP zLz-vWOCI5;-F(E9Is`^1FssRtVfPO2I;6j%I-_w0^095#bjc|@`}2w@`}nQO#L!@y z>~{u%<*JU=XnR-ZeZ?u~H0q_!80q&8)RIo5Q$@Uuv zU}5u3=*i-Fl`kCbJkxDLt18pE?S_u!hNAV+Xu;|jvTIY7oHL{9#__nRtaWhESRwVw z1edW>tCVV#8oonP&|cps|2>;K_KQ;k zv~b(*7bpsAm8Az3K$M#tQvyBh6$}nL?rzvsEq?f?9VBuSUmOfN$`Z=>=Gw7fITrns z-9g@UGnSVO99`^T)$TM;yjJ!%i9W|6GP3$2<Ak6EhnxI<#0=o zyV73ang$#c+&rY)83wEK0!(|sm6@vpYuwFEn36WI?7D@e;J<3$T#NPKu`t)!Rrfuy zz5c4^;qMEb{2lezT9f!a{{9MjJQ=kvEn#ZJ-5Z1$q&j4$3at!NLZKz`>MVmT!^~%Z z;q!xc{AI=m%F5ar%90vZ@q&WQ#TS0tD|3f!Rm24*)s)2*t!BB6O}^G1@ff{g_P$!& zN}ugu%!5^rlnq$sL$ua6B18lc<~g!mkE!+Yk5NEKRd$5X$(?zFDkgy>chMoF&LN3kb zMVcXDt?fF)`m(YjFicPI?eN!TXQ9<4sL{vGt-0DgiSg{$$IX4t$97+$?&AH&#mv_U z^b1}Ci*IF9QX$ojEL9x;+_vt8KPQi%<;`+PW5e!ihX*m>*aMJ0dDwq<>R9 z!hZ}-`@T10%jukP=k@LjluKR^srM534~lY3Xw&89=G#6)ku$qA<%j^s4|($8e0vw3 zZz@`@opdE#5D{@-72I|Rb;dl%ugF1_J+ZI{!ERPxx~cwrhtO&ioCVnAkbid**rzLH zOb<-yDZxeV_FXHi9R`@NUM`zJh5}|5)rf9xHy#0%e#87-?2tLIU4n=({k3{p{kX?`RFxB7EmAq{BWM%ZQ9i(YJoWTT zP`Ynk38m7@wSazm-x#&ljQc4Kq*>5kFF+}Ie81YkzA-@mHcrdM&UKq(u0n>s+hGrw zsr3dTP*POpVCqG<^ND5NwHD@{w1>xpLoTqi<~|A?oLA>*4EodN z)?G|JIkx&-Lk9znw#bmp$tX?3;w2@S8+o9ndr*y=%J+=4Y*!Y;3t{uwt_cO1|D3d- zm+IQ{&+{z^0}sFn#iTpz1d?C6a?RO+KR|j|>GM9+`$r;iV4{+jpwhgZJ|xY58%!y`YuU(?|!iv?Vk8-?dKD z5KX3|fB#nY#KrPPKF;Z~sG+~pv=V=A)=J>2ouVg&_j{wMo<^I>d8=2~6P`!Usg``M zzG!vV`l-J#M#%WmgSF&qv6J#pho1%oaJi~x0{>qR&b8GP_+w5?=y8rHeBxc*%hx>S zUavmkT$#1-<%u_*bym3g=|a&zMUPX@7wk_Q2JiKuhWAR5Lcd;#h_i=XlA_30Z0(Zx5bQa_iLRUUx zPfJ;>;JFAm9?M>XRHAyA`O?NkQPD7dGq>?HzNRGz{u~+)eQyZbz3{i!^WaDWZO*Y5 zZCQaC`b*%a)^UPDv{gwFDE*G*3%rnGET8gh{+5ULXGY7(MBoFj787r!8{|&B)N6n5 z6bfXn-zc-~l!)I>7a*Y>G}gTKVKf@`5HLAt(=nUL^vT)Kk6K0h7zZ6;|aq`+Wwd)e9-nW<62Lfga&J8h(FpHT-rFB28wvVohWIT67 z`aU7d0H+4d1P5H4p^u=HSLRvs7uxYoff%u_z@RUG?z!YLo!up^=A&J%b!XCn@i^t1 zZo2}y-DS7yyOqY%X!*lrSyn@PnNjB!Nv(1N5>$=;tp9D190ZfTsHb0!w;MpoKZ-_0 zi|N{jp2=fiOZt3opD}Te&vS5BCI>iXcl{Yg|JOf*dm|4mTTd8hxhr)Yo^n%GPV%!* zn_~}DJ7~roXRrSe!Kgy8eSvr82wHnZR8c@do zY*E``P695SaW1&!0o-x_wTD>F*PC6FUuT+mMzSzfD@$fR1u#d$oS77?;}-59hKE+t zJxw2tb&pZDblo8Ao1*&dCl)WGDV5{ee{2cu@Ullzz_n|-a&tM(yX&k~+vAN21xV=O z&`gf8l1%--ygb9ySLn_N|^ET z{`WDKqTyN5+>MwrzaO+0n%0GYWt693>yO5gMGW?Khk__9s)_R|&&pb!&&=lht$dl% zI~sM-Avmt}^U72DjZZQ&`icLKrt{!xDs8*4ipn?$jDr%Dl2H^ygh-K2GRi2U2pJm@ zDM5;Wbb-)PL{vnC44{SAcd4K^RD&%fn=R^a`K#e@9W-M z!)dg=1<}9z3YBttc8s|5?#$*j89UYLf$nSJ7}8{{#eS(sKYkpKL6`Pcs!`QC0ffX#j!AU zXtY8W)94e|ezS4iDdR1vtE+0(j&95yJ1ZR4+|lqk>v7K;Zr5f`9$P#YKmc5JJ#jDV z!DRW(H=C~gGVSX|H0Ggv94syPu0sW>a`5#F*VXf9csz;lpqZtq)U)`kNvd|$I~A&j z_Q#XMmDjDS#}qAE{U>9I+l@((w*}jeu~nY0%@}juaq)cb>sbEHjBr*qq$cd~VaUXQ zx2mS!z#8c|wnzF9BXiI!)Wu+O2UZu@Z=sOXjUCm<*tX9olihuyi*N%NhwitO4Ufjz zBNI02!yd~3q(L_QhXCH5rm}iw-xPk$^xMoumBb(tffrx~Y5GTrL7jNvhns37_bt)r zpM;f^xOW>!E7~!$E!p-K*)9oev^!A#do(&G=eGA5EPlv^bn`GyIFvjMyD7H9$iuP+ zV@c?kcdMhXll8mCEA@6hTb4_7yT@kTkr&5DULP=A@syCD zrjbpj*UHQ`I?WW4`s=DM=(p;IW!2#AJLJ3HpF(rxMg))qiKW?qf@&olm;B6}}r8}vPW zb2?w)jVQZgO}V z=_F#YwL&$?ql!Ux?txq zH-(rOi0!dxVB$@DMOu-L z&U71&yHDYQdt5{F;vFL zS%5Pkw;|h`W48NpA*DrsM?CiOp-R1!|GRz>pno$`!bR>a=&}+kOD$IPrb=EDt9|NrBmqn3N9&pDbgHTsXIXZKIl?RvdC&VQ?0YA$$jUtwy(w9FG5 zRUlFo2+Xz^5Ff6Hv^pM|Wk2Elu74?ZY5tU5>514w)9)f$&(0BFNnTd%r`{3izQYBS z&~pNsOOAN??BD2#N#7j=oYV{*nR@pbIT$*B|Aiy#FX*05R4qq={^Aaqd8@KkP|I0gP zV3IJY*tjCIsNniX>@z8Zw}qpFv6@Jg{(<5AGImoWs@bN$rt}&tQ%8LKQq3663bE1B-ve9{Q7!B6<;n3#;Uu z1O>`#rxrD4$M-GgIX{C62Hh|Uzb2AjozR(i6>W&+%a=DtX!%?+yYw{D-NN|0IeQij z*b6IPPDb5QgYPWOMcRUv-+F1jd|u!ZOS$w^>@NrYVcF5 zNJ8+eCCSr#KU@#?J)==qMtx1Mf%@GY>?ap@=y7Q4Tk4l3*B^I&p`nwqF7Y-C@E%Y{ z|4@YO`&WYr)eDwM<8{Q`c6ULeZZ);{3k_9%4Ei*WmuR~fYb{6&Tbu{neim-@V6Sx7 z+*)A6pL3Cpa&+V!1WtO)O2R}+M-{A1s-a8UrZ7PNJMw+*%@olMU?Jexd@kY2akq{W z6WzMIj5A&@c=s!{1Yd;Xuq4;l?;k=t#`f@^*mB-$isQ>m8y@`s?_2EUPwHishOdI6FLV&)Sq~owt-Cy0i z(;?M;Eue?rdX!h|L*GKH?bZflnh&?``uE*%uaQ1Pk9+ODcAcM`Kl_W@s#VHPpc_-; zsavZZZ|uHQVf8zqwR)93?>J)MbpbK<1G44T6TvAw=aaq?iYIo!itw6=gQ@vQI`&Gf z{})bKt2w}b)~}p-$qnvhy4(jOdP22x{{yfczDCheE+GR5_QebX7$w_TLK20 zt3p>K#!e{LRF4=uEm^m0^S#LH!7fu(YJ)C;v!&jYp5)tR9{!m~jbapnb zo`+5hpSyT(3gx+z?=8OS!Ks;0z~{|WV1krV;DKsx_#@>|?mbm{dGdSA6F9={9+1L2 z01nj~y)0TRS;b0Ej`P#`9Y+j|R)YS3kqiaO%0L^xx=Jj}6It7J7cl>m*jDn4!UXee z2~P7N{ZZCplOi`_gMg2~MaR>RVSGA9_LjwcP@XPyTm6XQxt4&*a z8)K@V#%Y|GkEsCvCha0!J=GAICwp*?v1y57><`YGH@p1FY`{D!p<)% zS+)Ei0^%Pnct))6>LgMl0}uYAH61w_wIcWkPJ{e4SowN*y{lvP=~Qkl`(wwc+d3^J zMK=_r%5^9mYk%$A-I7RP4YqngT{~MhS2TT+{Cu7=)MW{-;@@hRk3U90iCh-<8EL-& z&sH(EnbvQycCa#=KgG{I3f#grXzTMnNY-{Oy#B?{AN6-0RhHEcS~$? zA#nSoQ1+cKVW8h-Q}oSx|6(Ydu=IxLTmJ^`lobN2rh=qMPpGw;n2mj~dC__H&tV zQE{fkSVtth*vgECUf0$L0)@QlB6iI{cIj9<{DQZ;177q3@UV_Kx%Ftd0k)gLt_ul{ zlydO@yti!ub}qW7N8rq>X>ymqTVe8shMnyUl58y|xIqf;G1DgRdS6Q#a;n18;ITfv zy;XIjcDGFaRAKl|UICgNO(P0fI0)!j(*E9(4CZ>aa1()*6M8~cm85Hl$f3MTygUen z(Z4dqBx3f`hRWh=cb(aJ_8Z^x<}l2)I`0G-PPK*j68MdoB3lY+r`R;*Lmtz-H(>d0 zp|$<=&!(aHN?xkLL#`8Kk4w?ya?O>lQEM*)JBA7@nXP4HO@+#Wwv*V(-VV(6>v!uhJ{ zTVu&_eSbpIBrC|JyuBuZu#ywG+bW9=;}hk1jI4x$b}WSC-Lu&txBkR9_z7SHM`A1l z3)CcQU8VxoGb!1ih{m(4N2(hii+2Wm*S_Y5ezNilT}_G8(1^qQU=Vn$T+L?_ySo*3 zBQgAMQsgHiu-erRQa$sov71?fJUEaXzK{nm;}{xgA!1^Q9EsG1mm`6U4UnLgEK zdjt6T+W)fv!(`FiupH3#O#WJ678hkD7z;(W=cw9t{7^S7ui)IWw8* zTeX(J@RTkGL`%h3a?j60A`4oXry8YjOEA3`i{?j8MvdQ2Vmsw@_+R$SIQgiCUxA!% z9$j~8RKB#Abx(A$hXz4RwBgFtKK3oGjK}3ouiOFBn3iy1aklKR-$=YyU_%aun=un1 zxB=Xp+zP{<&(UqWqmLrH>nLPxljx8M*Hi3f8!A%;DEM0EuPUs?eyrUj zGwZeryHE6mQ}00GDcB%!ZVTIix_Mm-(OcrmxD{FMlx^ z6&F`Nw!H1N&dy(79Wiw77?k~>X4bdX1s8Ny6I|HNw%U;q+BynCK92m5?dK6O43CWH z9L%T=ABmW%8=Woi!;G(z*ksz-!Sf*0x9hEA@R*e5h0v6?$YZ*W%VzrHzgu^~V)Lp5i)A)md>d;tk6}$KAI$Vj-qUA%BHo!hG-}Ej$`?>&r^` ztv>K?sWp5yXEAbg?a#-h_sS2^YVAx|_9R3*5h56)tS^M{*#aQHegLz*6WTZk0ip{7 zwKj5v-S*v&t_jmbUMf4DP71oL2ULRPo~Mi?#Wdyiu)gVt{H>0&qBF7)%Lm-ed{^&> z>AX)A?D!#@2FRnNM71jm35nk{LA%2JwO~Y`3fW@p+DT5gDE?D zw2PW^!Bycb2>3~iXSe5&nu3~F_47Cc>xDZ02mNdQabN%CvX3seRgwZ;9d%CUdAOV% zi?7`bmY)!JSI$_XwsXG$m0LAAd#fS{zrR4*V>(UpbKpn~&4*r~2H8>fa>&h}?%)f6 zLZ^gizo|7a+mU$BboLI`eq0k3a;qPYK>l=5uIsB^UjS9)A5)<9)a3(mvEkoj`d(on z<0l70X3x{8KSEy<_}gs|YAhRzX3Vu@P^0Ph%-(30hOC_%bLy-CIX1+ASm$??W))1| zKIq>(D#Q1s+bH2Tk}mjt$dmeE4Y zVt0*50{8=G=5D*)gWc9wyDSVU(j^0y{M168T?hm%{v7U6FT0A+hJJzX-j}78=%4Ss zvNR`h>Jg+`d@@?VvpaW#ZFSFAklbcdB)%^3bBYH%QUQ^HD=q_a>9HHdaKp5fFI7iO zx+anUAm%;F5L;;rnfoF5=~^8%H?)OQkDgXlLVPJHb~jy1u~d4S#WH6{)C+KxZ|cfo zV%TMVNYsp`brO+pzhuO|3+()Ew`<)N#sVOV^P%HJ9^87*v6>#X^>j30=+AhPK zR3I~&ZFnU$J1B)0YVpfdgjMm2w1t@~8Q-IW4&jK*6(l zTU88(7Znqq{02g|3hOT(ZrW_UI;T3$e7tsO)iR=Qt1-rr(e8CLtQ#VjONEurX-;WF zFZX6{4cIhmPSL(%hgD{e75;p3K;fIUnAO^Ne!*u+;tchsOGVr9-UJ`SbxtPtEDwFLjhbPZ2e)eTuXrYSl&TTF3Y$ZS7H^$^T z)O$Mt)H$vftu|5F+nIL1(yUL_pw&^=hM|$vTWVn+QNu99SkX319*l42(iORsUF zgGBRb^Z5fwiYwZKVY0&|UE-`*E~7o*&;E?!gM{7<-?_nsLlU(3OD2a)DokN;X2_!^ z)k0CjXq}sd_@Zp0Sub;i^tU-bad3V!%u78hzFD?^E(#=9e;9z>70m#jv2$hH=yvlcCwh9+!K3HymhyrP#D2yi7nRo#X!_9Y4U}!3d=bYK@Se*toSU;jW5RzL%^x9KRQYr`2WBo(dJ|Z+@Z7+Mz$=K5z(`UT%yZ8l;EI8V3`<=`rVCb8+wVZ=#WnAS@}0O2Bl;Wx?Qy>_pC4lv<|fyBT6yoSUHU&AupH>; ztgW0Zm1TVG8F6yUlE=>ZENuxPiHC79Qw-|(;641yG8}p6iKF*{;m{reM*`J+bh}wq z4xPK_{l}FXMT8=J_wsv$Uu||xl6%q#ANDY?VehRWx}n*El(e4(Y~?#+ZT;dip4?ko zo^BBrL``}ug6LHBmhqd3^N7TkWk9dD%dxpTW?aeB)AV-v?d6TL74Ex#B)RJ)shgp) zt39PyER49FlnX^R;dq&&;TO=Mf7A15eM_HaMgaHR)skP7p>0WU#U$9fmk5g)w(_+M zxu)%a7nf?+FOnK_Ip2|n4)mL{l+N(YwE-0VpM4gBz67#+(_L2PxEa|~m?_h~*a*8l z#?KZdt3lPnC!5R-(4Cw|!5eF^u#P?ff--nx*29>KhZ=@{iK=z=2Nv#~B3g5Lq_X)9mv;S^eSRpvfA z%=%ESpJUa85B}tL_qEKJ!}iZe?gt&1C>lC7+=M^-o0ZxoJ&IaPx5Wo19>)Y4Ur@HL zeQn|BDd2gh&EF2WyjS8`TJP(4Z;214mZ08CPNn~w-|9T4Np_Ljb1;=2nLFzQAj}^Bq(3PixakYe1?m0EMOe zb(^uAql|`wxyYLEQG&3eX|Z5X-fr$QxJ3>*`k{_%0(la#qiSQlyMBD5_1_Ka3;WiT z(zHQ5?=6NRkX?tRs0M{#vG-C+TcV0T*RTC^HUE#q=i2LF^W0DP6NeY$J@>J3WZz&@ ze{6VbiH$pCUNO&()*3se84l-DK*ZkAoRWO4pqbc0Gmu{P`jmg-e_DpDYP5@BFjrVyYJk6 zQODri4PQ)K%hup2-`!Sbq-=Qm3wFDO&&K+Bm1MiARsC35GjO>rtI{*wIy*VT-xcO8k(Ht0=5iV*0Vd3NVF!UKL(pz0E z1fki&G#;s&YGlSh!#hudB3dOTgj#wts2wjf_m4_r*8-3|mx@O>>yCQIP-yV2!&1Fv z*xJq2@`{?K17c|jjOk%ViG(vldsCu@AvQ_W6s%n9`hgLG1VvP7T;g_Mh!M1qBA^mo zi_q=*=7#aATJ~{SG(^0i9x*&u9K6m5F;^Y#E4Oa@oli_t68+L7v6z=th&BQv^Zi_l$WYW8# zMGG=_;|Sb_a2fb*DlVa{jgi>Jd{DUhiaBF>>5JcQYd>B+?9i=i>2&=j`(nEAVqpvTPR0rT2())5_+a8iz!OzTJI$8-B@W8t%Ry~27h+qwjN@x;a^ z+dc?j(UA)eHuAA?tQ`Aj&uW53XhV8#q!~`v|9}-CSYH}&oh~ln{RV_ydGGA*^j%2^ zpZ*B6Fk8>YOw({096tH}C9TElb3+zeVMVFFoM@AUi8@-1pMd5o3)UI`Rn*Zo#g`X7 z1_uQKCx_cqh@}{Ua6b2!QPBxkTOFDHNNsc3yfF z)n)KX@3>R*U05|@%2LK{>WiJvoL;?&J+aavwUL!k6nrfb$)U?wpmAVFU_=t8a_o`3 z;AQe)b()b;tw(e0BuleRaj&-w|1P?{YQYd=1!LK@v>9a@Rjs zVjR&ZN2txgLKi|s>q}&N2ku3foySAj2N{=K_gNlxDF8QDzb^9&Dqvn9u=-5@wVe12 zJw|*@=*=F>WK;m>U~16?y%8j-OTzZUvlkJ^*lz+yPaU;82MJ>&48O|!Pknv*Suz^i zR`Xu7FswiYVC#pDwbQH}H6sn=Ic~gpm7AN31 zMHk|3J>Xu={+C%3Z=3u|7D(;wDld_Pyy*=oe z@L|%j6fyZyrJ5P{n`H{;mV!EAqE*>zAMXr5S8F*+FhnQYQp>?kx};Pc?kB@?VTvNg;H>X5|L8V@OtNQ>yWshv%> zmRI$dqj3K7MzKL!?dnj3b^Ecou6u_Kkbw`hzazc)amVIcLNL`(L$4g3BFWHgpm+IN z+f5MfZa^LBFNcRq1b?+futFPzJa)Ukw##9Qv>mZJ82^Zqpd#Q(IE=PT^b3qA!5QR) zu8DZx)uQS=tsZ)J$x8EK@mOk^gx+}%2mbv?vl-mn_1T@ zMUDl&0bO<=5Fuhk4emZ+NpyVj?Tzs%(1BCe)%KD04R-}enRHGGGT54KM^Ax1th-gV zZoh^3Dz!Phx}1Xq{mg`2Z>sQry?I_MY$e;dFZk5E1*zKy|8(-Ytq=aHsIwCKOe-k( zZRgS9j4WL3m8X^cX2KD&gJY)WL)9~(Rf$Pjs-_2&tu_7MxCb?jX-VHmb*Qg$4@B18 zF8f7ca{5+7o+SQQ4eR&HXQ2nwqE_?CoWJKrTZrr&ZxFd3sz*DXG7e+Y3EcIi zk>cc_*%lC$_1rO(dxdZl9lkogJS1hx$c?&hmKYUJ4XzYducH@zrYO}OjeiY0{_;H6 zfSe6SewSK!aRB^9P@lb}OV~*-i}%^Q963O~ow41occBR102dR}2dg>1mKWO^-UWou z-^`I8A~1gD51@cd?t?^XA{ix2V@TkoiMIz$gn8x9MA*i(jYCij?d^kL6%5 zr*yPg=Bc56OP2pMKO#TD3@&;nY_cbQU@lJ z(ur@i<6NBgeG5oHwk&UIK1uoNFrNv4^BvN4+U(}iJ08rnDMHKCD6?h@@(FVeCPi-| zD`L)^a##lLymQy&%9}J_k^f$5DfQ319OWdfHhrvgWij~#Mcu&cQLPU6V&Hi|s=j2Z zS7gcM3$mI42T-LO`nu<8j>oY{TN1Ig)7@#RewRIN`pjMq`EixbfK?7;%Gocll7DQ5 zRA>6>SZTpP2l~EG2!8elmjBTfLauPv6{lXPE;wfdz4o$h0P`9dSz9TzACaUV;a}0L zSe{G3ZNvGaGANMrvU9^b)0xHuzxe$!@p0c==SzHk59QslgZsWHN=dr}tKnVw$hice zdAg{gH7~nEJP}t*VkftsNC_`O(x+E0s4i`-Ute^pmxP`63B0w<%ElPpW!TQcK&<4T z_`T*_dBxBmtS^V-1?wz34@zi1Iq~z1En?6tnB)~pE;~f)B#%Zz>J|!wvRm+h3Tj?A zr^%12{tyk=lmv4=w=DHYE!5=iJnTap0ug|FPOpJpQbVy|^NhGd{a=p))24mwmz$)E z9T$snaH_k)ud6E4YH2^6bM!sW7^ON*Kr1`Hh`j|n0JSxrTJ)2L+jT!$FN^0HZH&J@ zP&>sdcNs1%Agpw;2+XE4$afD~H4W3eP_8BZ-w7|gkntn$cfL#z(zzgX39;;>M{UL7 zukR*0Y@f>X9G>Mm-|UYk{cJQORD6~e)l6+D zOW~IXXM)AAQFkqnAC(uiI17QHkpSEVQ9NLdc$**Q%xaL_93s&jRj1%Fk){*Y844T z7tqY=<6u9AZ#LL^of8jz@2y6u>9zR6AQvd`rR}(?A9tly*#GfGk$8jhO=Nvk$4JM4 zR!fy?)>(V7p?GVacS+>p?cl&l1U(d`(!^NLbcap$qNH~(O7tvkJ69Giy zmH<94dccYO!FS*Z)GJEAYepUn+hxdOx?Z;J3 zdt;dg4JafHAP2ji{upcJ^!93PS(TT_jpDURe;SGE?Fi=O%Wv>2cWP|fI{Tu!hIx%V zMf%NW8jycA#up&|nLspp^;B(`OjSPpHStmvkkJ?8$Gx@+ zTkixurFbNd#Hxe?KIA*=Y)zHAm$MGo)O>^LD(AHs)Y5w*I&R~5i5uvLu|x$TA1@m z@YPCg?|ccJxp1DU$DJY z(TeczAWTg=w0bg>f*Wtl*74^)`CZ^_6}S{sn0$6pJ~H$7VM_eDXSbtVsua%~JyyK; zxFj-5eo@=k+8}h>T)BGM=yYR1fcn<*T?=pHj|~CGINftV#VeWJ$5fRHTK;xYl%N&K z(0IGU9s8{B*Hg50I~8>!4ba9#4S8L(Z)4-0Tz6TcUf&0q=`|CQ7d_4!?&$PCkT6dz_7))z9{SXg8{c= z=Kd}Y&b@wH+_(Flxc&e;=#-@istSa zmoSe?)0)JPsXT>^p<}Mx{AZ0A87qAg4+G^ zyuQ`K>Uhg7ZN^V5Oj_)!NdJ(kLuZpRYJCCyAumrVTwt33YG(!-0s=a{wPM{DoEGcc z%eQ_}ce#%E`^vDyu-sg9)X*1cDU4Fv{qY7FXE)4-Ex*9JdfaSq<>l#w(&C{ z*;yN9XF(Q-oZ1{tLDM$yqVIO#M@q5gTw@@Kf1Jv5u(PnHA>K74Ck z^a07yVy06wPU_jdd+?ti^HJZq_Nc|`8`Gl8)g#rr33(Ra)&AIg^tXA(nzJ=D`P=x@ zqP}GA$~oXW6@?u@yh>q@wc=&}?w;mq-m6(9dK9{O!V3PIPuoAnn?ptU{_m4YxX1bG zawdjPlP8ytEd7qF{Yh(=Loqhi4!HNW4123MT0RM7<-=L&1*0$0K9%RqW#m05916B1VB&BqZ4&t7bF_+`nyjCN<=Zuxspj8&nGjhqOi|A>5q zxAtDydLw2&_Cu+KMYu-Z#_}2`Co6po@V=%(J`+y2Brh91vNahhw3ExruX6XgM%6y) z^BUtdPM;o}y^4wyI*eQ8pp*X0LBeepIQ=)f-H%pMhp-Qa+d2vU)B*2%@i{GlOu$2Q z)t{|sP_p%1J$}yBD%xK+2x!Jz_rZc)FK`EX*hI1Ev?h5hAs6h6i8d+pF$v3fAMp$? zKAe%oB?UiE11H70j2TeUs%ms6*@af596c1h^TJayz=EvE$m{p8P_w(G(KKEXEUA+W zJBay1-FRCG=c{?(r8DkHL0Oq9`Z)A-4s`nL`<9GQB zH{)Klz|iE?lTLd4U!ycT<_4_mOS~%YVN)bu3y1$5#ZRFf&Vm#@DZ|$7ix-%k{kqCt zbIy(4pL`+o4yY#Mjbuq49Q!=wSrFy0vo`u*2EIj(o_N@VDbDi_*+*S#A5745C4=*DZZJBrE!81tr2sq>$$Jk83yTMx zWTa%-fM)f|ubHGy|A%}Pj*#+$-zxM=Q?^cV{$YCYYVz+T+(F5!yp|KVULxxmwM2L? zS9E`PWv6s&EOvcZts|%RMQrr({AVz7-YgM)#U?CBL1UYUx<0rSw0wy)OITnYVeTih z{^@NqaJvIdt3D@Zy!ipL+D{4xw=uj5mo+R&R;L@fk0U8#F(f6C8TX=}SsH(4VJRcZ z4?S;S!su|m{a1wW#aA7AH>BTY;etGx@p;)4>eG%5nb3-2l z36aPB!g21xd4nrx$u=3}Nk5eJWywxEk(=qPCDMC#P}kuJm^$y=l?(P+%hF!Z+D%OC zCuVfC`Y!(4aWE&$w+QAou-*mg*_eKWleq!;gt37#zGKKg8e=s}uZUUt{fD8bV`01x z_k}AV=Yq6J%@RnMf2aP&v@Q1uxTg4<@mZkBVG}n5WCi*NnV1oDrXmD3oohW^hncQ$ z0<&fcti10HvzB3-d`Oxt#YwlapvYb1aSmu4La7~9o?ZSSw;B{E`Bguv_FTt~qjm#1n ziG}ZYk6XFYWuTp9-0RK(&}5mN&N6iD597@OiyvpEw!*gdhHKSOtY%q0>B`G*5MxR9 z%NHo`3-yHh-eg_ z>V)BS4S(V>;X>!&9xu38q>-2%uvZG`@yho_#HJB@gEIm_mC?$G_@VLmu)P-mYt#Rz zpF$a{l6(>;TV(GrdlzT#)E={o+QN{}@7#`$b>ute zi8$T4u=$rwI&cmJRH%jSu)MSoRNWBgTUkvJY}Qowi<3>yx`z_>8kb;n%ieUaTs`Fv zB02^0$oslvaJ|;pHKngme*1rt<KGb8R~PBc>i^ZneY2- z5cBYV5ac)aFrqPU$V*6{i3lwjJVim~EKCLJf03xETMQ;Tj8I|{;?^z9n5U#fy6G$L z@}wN*8)Gp{@t*k%%hI8iTi|CaMzva({zDcY6?VE$Mx{|U`@clz>32wu8UsI#EEM{4 zTj2dYs+;}uHQNm%yQ8PmOZ5+xTV^np>ZZ9;S|(5JDk_IG0cpj$VWTC8L~9SQ;FN`? z&|F{th1&V3wP3$dp5*W8oTk9*7O7hNRAN?}aCNT}R@X`O+p(ltRsIJ6YW-{j3j-_( z2cb2UZFQSfEIHmj#LthSjckzZ_}Ps0AL^E+27|8e(F{f#vz z!!KfQT+=vV1MbLbyb||6T+(D>B|cW@;RUd8WE)$;!~VlVzsYfV^)w;c%RPj(P%W*FDehp`Jgk0W?YE_#PCygaJ?lN$8 zt;uOLhWy&Y0|hlx0$)p^*q@-_GG=18T5eNysRPus%GS|w4yYim5hM!1j2a))N?j?8 z%RFEB9H3=eBe^#n4R(#xm87rps0QMdx|=$m7aZY#93B?EX@M}SD1~!X+m`FLzo{D2ZIy(NXcA(OmkS8RJQtd5?9*#Jre%_{+_bTK#fK5+A_nkX6= zuU9o0`?>zx-l=gn?5yKX6jb;Q7r|lz)yJ5&8?lN`#Tps?_bp)6ogV^jU~(XLiN%{5CTT;YGz>hB{dv4=z)HD66% z3!338QjwbNmlf4bCbHWk-ofKvcm#=eCH`U1Crs(BEEnnMOC#hN>EY4_JB>4hfKTPH z@m+7F*xOW(XP-QKD2j4csP(`M_Czo_U)EW}ZM}V%XBM6W+an0>>tv*}=DRU-X?|bt zJsSbJ0sO~q(8AkHXr4vV4+UvO-^wn4ufet3u;NU}vYxCmTS$kK^e<+W6{HQH;T>Bp zLOr$PgUqT$-DhSAoR4e?nN2x5{P(O|{yoQ2#zFl5_`|+Cxc&^pw0T@nkd+KqjE6e_ z#7{$_ET$FaEDJVl%M?3SQp1cK-LXWda+v9e)_hFYGV(tX8x$6mn`Mdrs-piCb&Yhz z4tz^>LrPQsfCbo1OIw4`M6mWRfzEBVK>{WkB}_$^*_JY))6-S7kwb046%iAt1TkhC z#-R2xO4P@*k`!l;?L6tRN7rToGRk!QStgrFgoksqJrdiN)ds@%9f8uW8iew|T-V=V zpV#$6RX@d1lR{w;xBQTd zG~`RdzuEdLV*(ZBo%S-JM`I6M-^Yvq;Q|Sd_TB{G@;mfqCh}C=HajeH{zr3$&5fMa zqPh-2Uc@OI^OSb=C?Bv}tN>)+psa^PD|WR%+%TM%$iHza-T4B7zIm3~xs%`;w&NA= zxKttzK;DQn)e6*m23z8{yY;?veD;O%M7gTx$V`*hMtCEvjeEl_qXzdViCFkQdLPl* z+puEU5K7crxkv%uojYy!LNFQtgnL|peuUb9+Yy^%FYr4MN_cMxAXIOKD8s4M*&Qfs zuH!v13=MTxakTE87w_QyW2y=?^nR^6#l&CJ(;r-SGkt|M5G9cv%zS3;bYlqM=SqG5 zcB&B~w7(nUIqTMrr0&hM;1~$L1?rkL7!59Fw@O@=4egTvfSu0-?Q)WT}c|hQlQoWcGAk1u$^^QQ>^oKX*+oZIm{ZD4CINlr!6{ zKOadeWMr`UfJ-&#?YFt0A=HY_OyaS4C6$WGg@TVtR}s0K15H{=#slM<NV2Xz&xFhPg6c>j*KY{LynE*X2{niJz^_Wh3W_+Tt0YAj zab7J+=>ekVNtXW?R~3~|6Bp*8%9TIMR{;GnOV*yq_Yig%1wQXASKC+K)vHDKzO`gk zXEt7Cby#P{Y$emd@tHb%Au>Gj-eeyZ+P);^K{XnR&%FuIgv&gik>VVzz?0-~-P>hF zOuXoT6eMUFnGoyfJ=EKO(|vZ_Dp#hX*TyyYMa*RT*slT>{+X3=zdJe6UO~Q+7t6|X0$DE!&_FZdSi1DG*c8AfeUZ&%lPLr&;pb3^m$?U)e$$zofRT=6YmTG4`^Qp)6cD85*no8TOcA+*JPwJC-%odzB5wx60)L)HNr9XgwoSxJ8 z)nwFQ;guX(-N+!Aot+nuREuhmA<<>xBSzY=d(D|?p}l}t9x(yaN!dkp0gm|~Vc1HS zBk5gs|0Um>THj*3S+N;JnQd9$&MkDVMre_=$zTCk${ID?t&%i)3*WaTcG3H(He=q5 zz0o$T-AD=dREBfT`oHmVBXyMuN5R&4nM`TFpt$wwX>kG?Li{1(X zZvdXNkz-QPBq}JMQ7J@PtPaFuQPCoTaxp7*T`rxJNKW1BT=Lt2T940DhO;jwJt9B5 z7AHoP0>w>(iPotxy)2YrtEQJG&yYlPSd&0{=BB5+4wUD^mvhs9;AET@AD9F@{S>Bb zRF>Tu?KEQnlo6+RM1P0aY+8aj71cXFyR%vE?0a&@w5g1^$l=%lPc+v85NHHP_n$zm z-Lt+Fw%nvlLSBWE)|Doc(lSHtE|SOeg!n@D+CP@8QO9bXh5XNG`);xk?FMGRxh?dG z)He>$Ozn3Y60x-Uh6%4#1C6O$qdA%Re%`N78kXf`fzorK&!t*V){up8WQ%Nnv2~g$ ze&0gu#!cN*EiixlUv9Uj38X)RE;hb&U-EAsGD|L8w}69Jf}T&mm9AG+NTnX21vz`d zUm7bNAR&W4fmhrzT5x3;xJ(v4F7fdIRM~6TW0Gzi;^=Diu~Ti+n(eAeyaHexCnjD^ z@fzs5Q9w_PXoNbllK{vc{1({USP#`0UAD3PMJv=gVE;YxS?esKX;J%2yy=2T*&Cms z8c=gT^NKpAr1gYp)ymSBmpsO*20ovzJ_lX?QPOqOBlJQZc!($k=gtpdETm{*TWdkv z>&ds%c3bb+|Lv#O-%6|s)4$`ZnJ)C`SMd3<;D%Y?yyBH~(|F8KaV}E$y0$=fNv`4V znKfF6U;H2cN7;MFHI+So<0?v2N>*6`1(J`lf-WV9EHxyHsHl`!mJSIkARva&2@rz7 zBC-gmG?gaORw)q(MF@mQ4@j2|Nr2Fi5LyB$PxkXX-{0@w=lLV|bzb*%UpI5+%$YMY zcji4njbeRy8KM{}I;Lbp0+LhJ+jfP6+wn%BSZ(^731k@=A|MFrmA9zlSs(n`nfb&5 zw0}u8gSZ=?Q!C`6okJf&O$=!X`S7ko8U8<`3&(x3I5WAVYM{xa#s4sr1`TcSb|5ul3 zB8<*)C7+0V(zge;gC0f8kDH|Q;5k|c^>Qtr%jAi6;!3>4zIK>jTz&SAIhe$`72FY* zI26Km57F829zP$JEeTO6EpNUQ*O5GX%8J-E5P_!$?l1i^^fCCF3H7$jhg7YjBliAJ z5}XYF_&XMUCdTZrM4+YwfRU%f_oa+>2HGeNY)Usa-&Bx(Q|`-Dl~{c05p@yo30Q^A z+~*!q=omAovHVF{{w>RQ5KUdmN#caQeTKq%HbiHDQI#t{poOSd9B0pW?EpNCQ} z?{XDR`xi6zaxpxnSEQ0<9R(~#iV_G zULsg;dak9RhLc-+j8^9Ch1oyl>dn8^EkTU$I7()B*RZ?S>!Ek5Eg!nZzIn!`?QF|CDqt@lT8 zT`WIY-cs6tG2?*~ve$?~eniX2b`<|9w9nbMSya~2hGC26XE~Lxz06{Kc&lDGAYp^w z+ni}lXu|C+rR0JR>X*ceIaie)N&W;`o(HPfr(K-0hyhyl=r4ak?8c88L+hWaC5}8o zGVkN^Pm#YZ#&3Xn3r-=u{M%861bY7)t%3BLkwcnYHimzyHXW3EG78s2f4t+Y=+?Wa z&R7wLtPLh)yI&+Mv}QKF*6fM};K$$drUMbv>k|(#juyFVFojJruT}>>)7udpZ*@70 z+7r^q$%4WDZ2s2N{OG9n?J}{a*|?TOjDP-NpRc`Nehb@gu?Ho3e)t}--W%Ym!J)Qj z#2N*)7T26Kv+uZsh)_+s@~Sa4BbStdFw7lGRm^1Pj&UKOuQkH+-ev`;n=jCp-Elw1 zICi|>Al}<9VAPu#eJM%E8UNWLaB3s)q8W4GI9zGowkBRilKa~4>spC0PQgmMIjB&% zDO1ZOEdfmOd_#VD2cyGjkT1njaeyR`j~;$^{3!FH9DKF5@ULki7<}9P)L-K=l=6oH zu#=SQ$_`rvq#q-+rdjF@vnq7p3%;6S|zY z@C`N9gx`FDHid^3=6eJus1%frUc4D2z@sum*(v2BDfv19^|UF7rt@p)4s zm@QFLKZL3};b<4PH(Ti!zz%}1UNL0X1Fmk&zeczSpe#9iGhPJ$_{fMb!gy-ng6QH~tkIT zGLQ2NOj{BMC!VmKdGvRaT*t_`$Bc0)1KDC6@4C(xVQ8L&(&<+W;4VP+^} zA(>7!ugTe@Wo1sjq4SgUZ44B>cz5*p|1oF|Zd1*QD5O-@eml_@wi_^lQ=60%6uro- zaAFS>hi~h8Arx*@k(QIG)Lj2Or5P18mo0#V@9`II-`fES1@N4Wh?3UWi+5hWcz^4z ziS~upr@H=geBrO?lMk=oICV<-(wj44zZxuN1@wOIUMO1(v+fHo+MVlJ(=<|-*aik! z8wTCaHat^bZ-f@h*kzc9x;TK+mgT}%g}3u>#Gfa=tbX~-tf?)761#j{Mul>_mOJ1Q zUEB7mChz;N+IK*$lV2rkfH}ky;H{W$w^zY2teE}dN?~5B6T2cD{B(bJ27LX)WOD&X z%FFlK$^%W>*Js5PeNLLU40LYCbX#0@eLltrcM)z!aHcv43MX|newe^0$KnFr(38X% zBP?J}*34YYi6d#jktn0z#F?&wS{)?{`8`t1B9z{uvB^)wF)Vo(&Y6o*wR9rC#g z@RFPPYVLapVaPc$G;+J;%p1B=55THa(^sRMc)L{ko5iD7=W&t~QS6$s-D{i#C}ncQ zymp5H;Y`5f)g_)R9fjZ@6kBqv4w@5X<1&bchuAN52TaT*oIE9Gy%OP*Q#;|omB`EG@xUWS%b4nL&~ta(YDul31b26=4<-g z;bQW`y^no$FO$C|JRmqr3?)SbQNZ8}AnzB?;$TSaCyP5Xz%{Inrg0(qKQs=x>o&6= z#u-ZIMT<4Qn0v9_s$s#oo9@4UAB1O^pB2o?+|Q|DUvs3bXmQvzZyopMj#Y-59`EJk z_Dhy6eTqE+e@OZTsItsD{iYM30tJXmFB6Z+oRz6vzn^aBoIw8OSX+brUQHw>tZBf| zf7OtQXg_m+rk|s1PV34L%#X^FaW0Uwm~65Hj&vorhxykn#W}OrZ z?1kz`HnA?}*0@a8L|J~(--IEQ%Cj5Iw~PPY%n@^vlnmXKEO04aE4L_l-Bz$NWBijAWFm8NFO&8=@43D- z-!kQS@`T02Q>otJ_y@U_^e2gzgJJ?5-UL2k#oDmCip!ZdL>N6^RkQc|Z*l+AYa+M? zS+9ODvM?FA(w4soKdu04|J#)O{ekcE6rbfn%azk=K*VM9EH>*EyOr|x7}MoTaMbN7 zcGigxu@Yt6w&g0xQWInfLNZqe_O;mVI7Z{i5}<}@6qMgai6s!FX8O%TCrU8g7aV<~ z%p-h{sW2&Nx-t4OuIilO1T$|1(u%2(-=yxciQxcvN<-6^{Yow9A=&{tctT^HrS3CC z$@ce~sP!GpD`|=a(!3Pv<{!|~YuC-pixUm~Yb#aD%G=%sawMd=1%OG(m0J-LfZ&(N zrGMn28_>lhw45D!2(}ZmZZ>yWi&fAV==rNeQ~%0b=sIz<<8@y07-kKsnDgl<( zS;kl*4FjBfkA=*ps*b2EQ}l7$o3HF zJfwTU&lU0Cr1R<}f8Xl72 z3?K7gdmbxlOR1FA^uo5xxlFM?`Dg(B9nfr6b8KmfbBr^q^;Hg?hi)TSI-GuqLsC!)1_v`!I_?u zl7m2tZUDz*e5ib#mJV31xQ0bUu=BEU`?|yj<<5Zx7KWkvM{@OpCZy7<^}nm;>P8_# zAAV-&@&1wF>9#_qaIeP+u!6FZ`7+Da5!>}edW^Z^xyup))`i@c_*+fiZ?KN4tSe`R zEjNG6ke+TX?jb}k)YjbJD3Q~cVaejU5+2Bcv`1@#7;2waEhFYRVP5s!pu?*!F4~3Q!wpc~;qTyPoxQH zsnu5ueNW&!=%Y9Dvi(^Mlwue)MH-HtETAN&N%ygmp5+%N+PGZP+tDC=6d4w(aCtzS z-K_{WpKb?(BjB+32QV8&MTIAI_ck*RYyhs+XUj;1dJYa}?I#SesW;jw`k!ZyqFIVf z%jC}eP1|jwFuN%o$FN48 za>9-cBtcD7uQ4i|t*jnc+TIz<%> z4F4VJy>nJCubdHt9XLAG=$^~|cck!5Q`XiV6WPeu&~#Kt>|JdWKwG!W<;b6W+#e4S zJh)~lXcOu}aqcln#e1z7BD8h8f3wWHRODJVMs%JXYn8a;`X>YF#oL zyl1HCrMNSY?~PZ7sr)Ul<1E?=!tyq3$22jx`Of@LO1)62eZth@y5G;PQTd zbni%kbWK$EGK%Nl~Q&>X3uZ{Y@AJBKuVmTTo)<-qjzav6vLV}NQF%-3b z0eF@Y6jqR`%DAdJ^vr}s!U_9kujd*k}Ji2IQku|}zF-Tukn|8EBt1yb0ipK<80 zU8gU|Q9_OM_O(d}qgW$T*LB|)Rn3fApQ~8M-bHoQ8m{l?)d*y*^#=5yRSP|!nzMArU4(n0b=zvq`?4~oKPe)@rS?W#$v5pG-W zYoSz^4lq2y*5~x{ha2p`5qw4dD1&2|b(~KOUXxH@H_2W(RVQN4n6mLy;Fbn+WYPyB z%y+qW#Jwly0o4=;7Xf#DK46;sOuU3Ws7V-sGz!A_JO3{n`j?%}uOEw+kQsVE=Ql>R zCFCQCe2R02R)CnmQ6KfJ&8+ww_>Zjp_;XFc#4O>0Yf04@j?mDyK>!57Jr@%OI|$60 zZMuXsKGb(|JtS2a|4{Z~(MvOM_y;Jj({)Q`b|Nlj@{wsz^j4qw> zY0&~yv~Yg4#~$Rq6VX_(`a79%g`X}VZf;03KNmSv)VH=CaV2@rX~snY?EnjIEEw-9 zq4a6g&e-^n=X!4luN^KtnQSa@3!NY6D4`Uq-$CI0+V8E*7K{(BEQh4rDoJU3`(uZ; zO5TGf2vtz+Sh!$+b2OQZ=e2>=f7I>48->e!#73lSE$u|(jBr~}{O_%JxZRZBMr4?9 z=pc7SsP03S3H$fCzU!Ra0T3+wb4PD`3vH6d+3(VK9@Y;7l|cOCf9KPW{lvXd2)8+4 z{bs{gbB4XzUe*e04QX%1YFqrm-&_*qLWbobk}`_EaR>>D^CPN+5N^k7Ou^JKf&p{E zk;kUju;D|GdNdPGBj3SFY9d*T4k{lAyZ@-?o#TDoh=r^hUmGvs@}`G2cF31>iI>pf z2df>Zf9b5c1arGLvHQQN7ZLh_^YVt*8{HXmfPF#k8$qr?L+Zz&MxJW9(Dkxc_97}R@p}!0#BYHYq>kT zU4Ph7`UT@Fyjo(BYasglR^k*nWXXIace1EA%TY9C>%U*}c(WOgdM`u1(d!n{0U$gH z%2|?WK^OPDxtb-wKvzy9VSHh`QS>TvSfi8&iEGEA!}cXSBIlObqB~#qA?_5KkX*rpxKn$C(huYm_%{?rB`z4f*_@B8-Vv@0{(F5qB*aYWfoAk~g^yRs#pL&} zC->;>yO9Z>g!dLSx5k@!o74f6mLPnezn>tbPL+2M@r>VyU&l~;zPM{iPTUdcM4AA+ zwmQI$5bp75eQ-X{=pDvR(H#R7UfpPNgK`(M8x~V9S3Lq|{|OnJd1$haf$mET;u(J~ z7>~YryzV$kCiKcX@z{NnA;x?PH)LjW6Xo$z>|igpmW&jZ^`=7)!W;L<9xs*gq?KID z5W%1u^8fP1#ZS765 zXG?!X)Bm&fVP39QcosVPM!O{STsyQj(jD#U*9{-DHRjI<_XNT;rMVc_@^7N9tD~^b z>rWyn`;jxjWs(W6k^gdV^xvm6p7WEl9D6h#c=TvKDC$!moem86!re7O&csH)GPzp& zA~^4NeyZRa#>cr0@*+m=q)Kq9 zy3avJjL{sy;t2d_l1{+xhlo6v*53~#jw#o_ z88skhu}5=EH0pcGa>BS7PvUe*AJ0uZCynU+QMP`2Z@1QMlI9X=8HmU8X_La%{R4_1 zGgkm$s2b9SHag%6Z#)|u?&AJ6h(xB-Y6wq?0Dh)}CvgdFv zGN7Ai|KrfM+{v59IzN;gHu}Tjbel{SX8w?hsBrPU*K~>duY_T4gGHuS@&X{vZ+dEx)Zjq2Fc&{0@zg0^%$+mQB8jU1G>T12xt=6hb*FR9$ z={`!x!l&zV0lUQ#Lm3c2O@^>~A+jEL(jxmxiI7S06sIX{dw+YcI`Uj8cyUUD8!fc)Ss_E;^Xj zF(&Jn{$Q00WG#_sh}K^9>w<%#gliCg zu;S|T%fPZ^{Ux~PGEyi}B8=ZjYZQd@tD-Wl2CIIs{I7+c#s8); z?J|0ymmH{SMB)4!&I z=&<$f5)uLRU;cQBWiah{bCF1a9Go8-G(5DyM{Niflw=6Q4z4;RK4-OqrhwtS6>I4K zcX0?B2e~2q5pJNn#MQX%U2sI>m5xZ75VIx#Z=S(e9BeQnxw~+7w;1YD9W%*jvGMLt ztrDQV|Ejj3N4*+AC>t9}A5dL&OTTR(zW97JW4woU>^GCM345j^=ISG zaZUQunV`a__r$H#bJ=eTQx}VR)mgvF5o1Ko*sKRK3t7~6mO`5~1zY)2E%EY5 z%HTDEb`&zfIekGj;<8x+Zg3z)`!lP#%*pPjrsnH5n^g}UNhX&0kSH?s423d!h7CB! z!6{0n@7%c%#tB6(v+I(yCe!UUTwFYY$#%J*>NGKkVtswPOP$YG2;KjsL|4O^NhyQs z(2ppP%0%UKH7DRMUmpycE98|typLO2m8D?io zf}(JaO*sKjD$v|!olo@e@_kjT0n~KYpizn~8pgm--;)!+NUr=+`IR6Xb9r)sNI$7m z0matM$9Co=2JZhyfBxGUhN&bXb7M%EKfL;|Yw(z3BG9$_tV}zi{pNlTC!X&?a$ob{ z9aV9)cXa)A?%u(ReEMAs&SUK|5a+qK`Py9%lPc)c?{sp}yx(qf6?WQVO>#xN^gXm9 z(D$6o8W}BDN4Eh)nxo#DoR+k4o(vgSjr{xkpcdM{rO62~bo2U0H$1-qjQpnbi1)dPR{3Z-!3%{Xo13O@N}?*z<@` z--NU{@#*V`htL{<6jU)UZC3;3r7Es4F@N2x)Oq5lRVdw2C~K#C!bU0cxswJ+f4CzV zAf8k(;OXTBL~$fVac(>{D^IahKF0=5nWSD;GWuOD^oqlgDfYGb9;nC6YLiZD&Q1D< z5^fvznjDa2R^B-<2$1{IOpVL!M9eHnSh)hF){o={dKMsQM-hR~t*|Lpw( zkJ=!W#;-NBm1t^t&M7Ej*{@HP)1_z{<^6-Q(p>tj$^KiCqG|+>rN^qBF()Di2NL)D zpCXWwD%e3AkhF<{iSyvZO^i#l+Kxs<5JyT9ng`bRqcWCy-8@oX{pL90lPwi}!kP57 zLz8uVCTUb!b#h~E=B$H<&e#%yhQ`K*~=9Z|rS*Ffy zK4ufK#F8q*e;u{EvVUaC<>tifMc054*3~Z*zB%@9SSyR3*t61t3cPR=VBtv6n6x}# z{cNzaC2E^2=~RYazTodEfqJ4AND=jI!oR@%HERv<*hS+r{;_8!3M zDP)JCkR$7{{VB4=%!Phj$=6E`<|;EmLCE@XhQ-x$mgJzgypQEl6JEev-G}ZB3H*8r z5Rbbo6IZi)1Q$apHnuZq{5=xwprklUzir%9X!_tBGUB|F@zVrkYFBL#n`H*a-fEOD z@0U}>;-3b)w4Mc0kD>ELan%fx;XZCZ*6R$ymvI9A3Q(%UVaa@9C1{e$rhG3;fLFen zuF#M7u>dgN71v0{_x&!Bv(%U4SpJ34-WHH|$o1YcyXIdOkZ*JeTefDuXXhL~k$&jt z>!c&8$8zmpL3tl7C&cyK?>n*JMukR_AF>vUf@4ba4QA<^79w##m-X`E)z8?re*ukRh97lFRTVtc&g&c^utfE zR*r$q;^H}-gd|XBp*qnKt98On60bFoprg1P9{RZ}E*N`KKna5$r;Bs1sf7*9VB5_( zHo0B0WNI7Zo+E3b2+EE=47_PPQTFYqrWT!ZI+S!1H-ruM6geD#}-N=+Y>S^H-q&hOs=ZMZ$>Q}Wb(BcLCukaFfpnhzJB zWH(l#T+P94HIrA~tGfauGyAZ&vah6WNLzkBlr&!U6K7+EQ z+Us-#`BFEj)T!BaDe2+^>JZp-D}dETqr{fW!?ZY2r~?)-Zl9&n`;MGucF_4=Q3>E5 z9b@NbHtFzTE7SMJZf92YF<2$xiPzRggfTqGE~hk>ezaK1kTq*M{=QfFD*k)asb;m+ z{a^XR%aW?!qFw=%ZdVJLrQc}rM1`+6z7A)d(6T{d#} zl^6`nstLR?`Vf7zGVd@lTYJaTOYWTKZr&)xPWPwDdC1slqtjIo-Cr)rr~&$B#?TW~ zbAYVIXEv2710d@|KgO8mNEI8)bLr6prGzL6P(|34F`}h`A;%`e@7&b96pkaE%$}&m zgOuD|=-+J=bBD^oX>*6mSVd*HcCD#bdc2RO07c$4HbdbDtA7?)-Y;YJ2z{C^bK&Ya zx8p+JqR46hKo55}%T$ zT}mE$Ub;*d^vYxA>b<&UF|cnOocCt(^x_K>$)@Q&Gsr{3eidH6f`ab-`%l>~&V-PX$5csyM?$UzYKZ4TvqnPZTWyOo?9+5% zsVUaaa|$Q}i2Y+Z+cAMlcS7`y_<$VH2MMjYSeBY>v<%WKaPvY^j;m|bfHVS~qeKk* zs-R!TQc*IFdQjBJo*zJbsqD8-%F#A4!ZKtWrUpHWK}p0oF%r+^ts${G!7^VaY*53P z<@?mST=W?xUt!!q(etn3g6fU{59PH6M{@JtTB=uCG4Y;?XUwQc{Tl7!eXTxnYTS$L zZ&4jGO(Gw+y{pqulf#H#%TxN28tzi0VNbVKk%CV@wN@zuAz8feKq=(eH|Zekx^g(% z*INvOeT8@%hMxbPuxxA3Q5aU5qV^mxByw0X?|dny(tey2Knb!`@f+(6ySWfbx~Vt2 z86Eseqb3dnT`>f`xS?w>Tgy?y=G+iz5P6!5o$Xs9y@=`#v9RSog~Br!4GoIM(|>L+h1kcHj}N zvdsQUORc}j+m|@s*htbR?=6O7Y0i{OfuqylfP}$^+D^zbF-0EhI5h@xMLz64_)B$K zN5K9O((l1chEZtHK(Y>rb42dTpeFlc%Y!uyUX(J*T&#)p9tldGmBsjejm59Lw#hjJ z=5*eXWj~JT_$wWPoe%&fM_Mi;RKhHqn5NnOfn6V^twMFWegPYQL>#croo)Rn2ES=1uWgIpT%vl60#Ay<0>OMHC9!s9ml~OODBw+ev>`>(0TUaX6zBiJH#$20Vg5gt zK3IP>O@==JMXYcqPTl)Z!^+F}{jlF@*4=jJJ-GJNVHG#% zKRSp@!?dtTIh1Hq4C#HD?tnYzJMuVU-vp*@9+L>2XgLOBzgmB#np=V^a5BO@EAGD) z&W7)9OQgv*eNgou4B(_%sK)&ds4+>`D zKm`BH)~Z*7>7?1_^3mfI0Xa~UrO09I(1!O@rO6-ajKBz?4-X@2(p(s3C27|U(Y z#=wplsKBFkUQFUqF6IhWJd>dXamxEI$cKQ0UHaykeTHh_*UT$L7RQtGjuEs2XB`c3 zRa6n{jFZdWA458_Pf`?=9;+!iXh6x^!&?xkEeTED^YIVC(+@3kL$Imv&9ziU#os<| zi8(Fio^gbVhuXxs%Z3+~h2H0H%WC0h2}5SG87}kR84Yud2kZpHV1?;~Sk77hezB=n zDBb$o|C0+~60Y@-{o+Z15$uU)|IjmGn%v@hM_x|?L<)jasywQP8%Wgl3T*XMfmqtk zeiFV_QhZ0hnjRCP-nh%m28A1|pABaeI^OJkJs4t>kD0{hz z_x9=q#>p@Jb5$@$e$fH|hlhE>R$1O^+q)HRV;u_B;Mvi{6E$dy&p}=_)8G7*E&-)lpUl z%aC8oicw1-9dEG?-u@X)UP5fJ#fP{J`tc@~EUXQ8$S&w(a^FuS`4T!u^0S{>&6cKo z0I$U%9-!o=fz+?U5nKgq)^!);rfpOYwN>}=DxV%^P8fBCB%yp6_3xbRA2WSH)Av?n^!-D=)5g$$-sT$B}fi`_^$4JBP^XZb`>2 zMvi0u0+U$=2`Ayy{$nvdE`um~YRrJF4>z@Ty$7-~gn775Km9dkH?Ppb+4HY*`NVQ5 zqyE0>x2Cqb7s#Z3ZscOobRMqv=x{8`%b#ABX6mTZQstMSy*25)XKXi9Idzh`oh$sb zveLDPEUgTR(0zz_h1QY=vQ(9FmYfWne@y{ew3A7KVt@%UHw3zn$o7%8m2)0AoK=zE zAD{Jbo0TbVwHu$y(!JE@0^_tT1@U@UK)*!=hB%C!d(A%se?5 zQSNKNT5;hC8<_T-GK(6m#=p=WQ~zR=I9kIzkE`_?WIYDjwgcV{hwKg&4z%cDnHqfh zSBW>o6ctm>S@jKi+1rMSPE_E2uJveOwq?+Em(2L40&I+|djuL^y}lIGba`=z+TRv1 ze;fQ*^a;$^;RJhyn3umD(0(4-c_L(chFv1q^gyb^4d*hLwjRQ~+2iUs-?)1;-{z)1TVjLOoNh+su|%Vr@qQD)u{z zB!4u|op_-;I4c`hNk=HbmLxRYeJ|P+MR@-6KvuqvC?yp-7hk&lGK}ThTw?J;?)*BX zS=s8H`aq&Sy85s(7(bkH(ce6ONR&*`9}Ol%-8SY;JVO!b))18h;!j_i3mWB7E@?3) zrXum*9@r&`l^TZbEQte#py5*Y5?uTFQ{-m5RSl^pv z4fF3&rCNc<^4C}bV7fD`E@V8k*0{UIXx(AU=tq6{sd_Q*Xh+}sgIVmcp}vfbEOve+ z^&CbGWAD>e*>h4YBS>$8h!dK@Kki6}PAFtAsmb7jb~n0j3Ufc9$G$C`tod8s_ zb9Z{o?Q$>tKKd(oEP&=ef)Uzr2a+@gb>*%K1RD}1AX`*;a4_?+*cOYqGX!>ihH{_jH=I{C%SfbsxnUpNX8kza(BJ?V`7!2D zS-P~K-bAMax3fws-xos!M|{d1d#JYRxh#1<$SrQ~LpR~0Zo_s#T6boKa~``=NQukv z#?+&45oG0^h3_vWW?=Ap0Bg8f`9zl>Nn_?g5^qd(RSdm<0#BU1dvoriMoaSV9vM!d z=r)XnXaT-I4mrSa=>9|NA){fqmkQ8`#$W0ZjorfxmM3W?+KreF=Mt+WxXk^B>#WP4 zm3S-ptC%nOih_aUj3C{qg2Cb1$&E_Kh$`tP`&V~=YABqXv*O$w_Zu~)`h-XPvun;O z@m}@|n=e8TjT(D37E%5;lw8db4j(5-S01g=FYi9hX?oT$^u5$jmz%>>A>0otoSr!V zJC!$Zll~lAGMGF1%_J`2nZGtDa#oLeZuQCCAXbsPN?I+Nwd+5Dh%|uCG!MXG#F$lq z83+RAw|Y=*CZC`LnC08}7}|3ep>B*!wWDRa3S#coTuFzdq!ioEuu5k~c`L?1n1g7+ zi>c|37+V{BNf7mK_ZCZ<^{x@C-+%o3Xo#RmGshzh5g{;luWj5_GmGF21z9}CC@Lh6 z{AY7ClD`RM zAaB+*tk%1`MKFDz*l-8VlK=5827>*sXR9Sf_2L5#>6DaDf725}g|ib20<)Zd0{;He z>hKkRb1YmKvvO`eu2@*=6tcD6ven3ywVGRM#pF0GGe7(;YT&-$>UzHoVuN*B{II0RVmU zA3Pj-Rp_wgD9mOhZlsoA>fEu$Kie~S9FL~ElXgIRwd>?;#4-e+Zj!n5- z=?+!+UfU;N_|Fli&W^q36MmDOun+H&n6J_$@01^uXhMIP3@-M`cKcS%*94d9Doz40 zK#b(ARp8c%hEcdulTTTpRKuxMZ)G(9SqSbNit4Y~JVPhO&FnojrGA7j2Ro2_zaO=o4B6T4*hQO1`^=h-{p|v+gwGv3&xhOZ z1*X{Gy(5|>M_P_tnQ8Ew7*{fK3xCLD-7>o=Ff&9au8#d$k9O3G_V=Y6zG=UBc!$N*8=ZT+t&;U7;>=UnLgV!x1948v&+t3kIarA z6C&~7q+0NaIoXIe#Wggh7&F?O@Q?puB@_jZ4i6sRXf|EJbJSELs?tlY%89aIVwj>y z-J9VzDXs2jMoum%Wsd$Pv>v57@iiM+vDgArk96F|Z+bHh-C1qpdcwuo?h@Fygn7Aw zZ_7Mp+UWEjUPh|3i1o`CYO9R2Q2K(u9MX;dYWiz&pg5;w2gePSbGq*yR5bd%YSVz3 zyeP>3aR7v_DYvYG7t0q4&>>+Wur#L|sD-8opgwj?7@~XCUtk93s1)x!3nSeu>fEEd zBGPgN?9$4fg03?{bO_CM=do3hUcgTW@{rGPPHU8DGQZ-jc{!>4N$r+Tkfq;Z1Q6K5 zsK{Kb50ZVi5bObVd$Qesy zjp?U>1S@Lcu_(d9-gm+P$^-5!tl=j%D)!nfW;Yt7=yN~Gpt~UkMu`+^#BR-=It1u7EM{Ii)vkvmS z(I4T>3Cl4}r5l}Yx$g;FFgHPy9QK^>!wFQ{tV!IIWTp=F%FiTBg^fmhUw{X!ZJtSP zJOMQmwF#@1Qt+LbS@z!hWQ!YA(ARaBGS7I($FCsp_q;xsH+5pSRpGWf9`FysV}_EE zjOt7YkKsj&rP8{Ub3692W9QP1S&42(yn6<@~OxTE%Z@w~y(6#Ua5b<+kSWSN;R4-6I-rX41-t2qD-0 zjOtkPMVsIqsahLn!>WNK^?B+R44;CIKNjYpw*5$DU@SW6&{X6IYn-v&%ou^ZIBV) zP7)fmi8&#rLsB&RfVfE-c2&u}A$tZTpk-!g3J=oVf5otDrlmCY7d#FWFUFZX(hd3% zDmNV0WUe8?#zp$2eOE&ox~S?+2YWCND+SZVlYg(dtybpXE{j0!lrg_`TYcJX=xV4@ z--B&?7N*2U&s`7437k&8dC)@ZJwPy(4Wo0BJAD(YXuI;p&Gg3}+X_bCcWWAcXiTn; zEqDGo>6o7B#MQdurdSol%9F&fISniOnIhY`6xDI2!F(bnY;J!Z%=5&iq__C5D<%ar zcbC^a&NBxNnL-57y{-ZKhrrR=`n|>*yiaESP443&OIaikpyCojwNl1UqW$NCGh+n&JWv|C z{d_`Yw@gF8J-v$%ZiLcyneiSugoqen!$zd%*v8XX8Kq#BaE2IIxPJP$<68UfeAsdQ z9OOYgms{Krm5tJMcp&}!nRX5!K+u+vRy{KPKFJ&3w3S^lY@X6x`uUFEpS_budGl&& zkr-4XzN7Z=R$_IJcqXZ&9F6ByU0t2JjhPP1HyWPe8Ba=X071CYgLiMLt9M5PLI9L0)5#QYddyuiET~62QHfxOH)+Zcfz!ly= z%-9*{w%{tfwVPKOQYHmc{L{;OPu<$E8^zSjYIXh!bV3;R1^7N!+vKKJD{JsxPgs2@iGQ!@bT9uS( zk{?cko6jegi^*jqR(8D4d?9zB#fOqNzmJkDf?6L{n*FRWdL);@nsJY|)Qd!DB%mww zc4j$p*g;uQP{oy6eJ^8|>YBeY_96JPOpIKyI!`>3hY@wpWtjAkS;iCpy2d3?IDEDC z%+2tihdU2pmAz+8<94@QXx#D;G6B?kQ=GN*S`6o@T9)O+y6wG~CornPVx|?$7U;JR z=D!2Tn^S{n;&LQTy46{y3{-$CbMnu+UFqy>5n@z~UgVn+Zf1gJLuHPTmCsZ7Uo>6! zUy|S7uKbvlDV62OL1k&>$}G1~nOeCrQ*)v@sNCXAh9jeLmbr3gYOZjb6V2QM_ntXW zQ9%&}WITL-c>V$4b#v}>&UL-7!Tv%%!TJ=-prv37KITED#J-PM+?Gbr91(MqllPSPQZw(b$TtTcwdasAPw_w5HgL-X zN@-3^0lAgPf^Rp0c%>BdWNn3FkgEt@*)pC<>X6_M@YIf6f5Kk;Y_r)r`I)R%&VhgVuvBO1 z(HFHm$gz!O=Xs}h`Ggv^3v9!&8g0CBKf`wFemHXXrEPl?&)fEz=8m{@<3~V!-i3t6 zK;H?yH$V|`#^p6VAHsMz09@YPBebSp_asvagTnk?rB7{hs!naTP$b_}zZhY7d#JD$ z0N(Y$(=M>rGL@AJR7h0QyBa%Dx4-M^QA}?+>4a7H{`Co7nnCenPe^;_$ ze8|Im6`RX0v7LM@Z#x-Yh8Fjy-5Z_V;=ml|3C66oh3JB5TK$*Hjwc`)T%^IFG5qSa z(3luzc~7x8K%_(7emRQnF6#D7;XfC^f|~ihDm{d6*K>IhOewoL zE;HLLlO&&FhX4qJQI9FUi!FsLQ({ZGvTsC3bIb*3w0EaoOacBIguXG3o*uY+IC`*- z-Q0TV>RZ)5Jtl)^EaE7eGt`Vvtbx{kf8>IvZYr?z6NF-66RbUo<~8jP(c;OHQCMp? z{;ak%kKEwwgu|rdh{*oM92o37>#jtU#!_thI=i=`pRr}@h|jmjS7 zT+o;bX~=)x@6jwGdp9cIp5p%oxKrlTjV7`dXo~ep zztUR2agnZKW{}AWQwSonThdt5*8P2Bdkn2#Q@hHjRJR!%)Wpsimcg`h9Xjzc0H~#6 z9X8bHsZ<~In`j;VLJywv%HK2{>)1n4^}r?+V#>^c5wok**j)Pe)jyD+ly5OfwhlhE ztM5g7N&@C6-H_`8M-?xLUnctB47IuWd#}*7hn8ex=ehpYRC_LZw&z#VFZHur_k*91 zT?+lb?#==;%;y(&bkRDJE8Yu&Nhb2#`k9nz@Cf(EYLB<{D@XCGd|6uitf!VP9QwKH z?wq*<;$KP~HDyDNU-u5ee{wn9`s<4Mi(0qJE|->E|^IHI!6Y0ny!Bw44e}sE8pfI z1+)aBTsAj8(Yj2G{1G=&Co=am$lHoG0*x)%N=&bOR!gg{zuBZ};p2O}{DB%h-s7qt zKdAtp{}l&3NO0g%QX2-Rf0++c=5lUO4ZqH%-!S%8bliGSgIE_FL@mp|$&S^cxvA}a zh#lkcr8Y2s9lm$Y>hXjh2J5IHkVKyR1Rh|}_Zb+wb>%k(Ueew8=Sfilr$jII!g!`H29a?~74z`mFZ>(>amniW1%&!TiTz;R@$Q zY+N{PoeAc)IlDTUfoFZP)G{{R#x7l{P)XGCT75OMO*_y@GP9!l6&-k32f^{H#9yAw zrEsmRfGK07-Ad}6pi!T(Ni1e%gYCC?&6!d)E+_MgKDL`3&R@a|(C{aY(;S9zU$+UBj`9xbsIM+m4;)VNLDB5(8 z4sh+DuV$()anOXm)CIM>zWI@_DWG1w2vc^Q-GP!XhG4w}#)OUy*dK%;15&5uI7Xc< zNNeONIuzCpeoH@CCT3c7v~mH!c1@&Q%-0nQo7^ex`aGDj+=P>RMQi_Pl6zI3|H`UH zU25xcz=^F(pVBRl45=X3wC8;te^M(%Scb-IRQO6Ewcke!bXpAf{~tak5{>zi6+tBgd9>V{*2anowK3`WiZ5@34N z)otr{k4FUXa!-BD+^_0vS4QjkljpwuzZQT`BAr*hse!Xkql$R7s!4bl+fu$^uiJ-1 z4TFCI9^@bxw$-@Q;_wVkp>y8na5jPUNd{xLQsybp&uY!=QOu;fr60Q6 zu;;B43wT2IfqPJKd%m@-;kmv=9_AQ@Fp!F&chcKa9hA#miz^n7!7u+LqXHW!F~J)R z#^BXJL>8C@Tg+WS=ck+a>4EC%S$WeNbu*!#{Nkz-wDJ@rU%*cC-xr&^o?lCs8D%CR+*uiGnpqrl%86^>;wAs39t zJjblvIwaPrHHan32D&TzQml`=wLu<3F_}x56MIGuiXIar+ws zpU3U2eZyk=Wa4oxzf?;%h9r79e;=HJlW_Zy?xU$Ib-P&x$YPx1wOLy-lMrvvL6+-5 zx))kvZ@V{$I|F+3S^>w6%INneo39J};7F1zM9Z=a5F1}@y|)Y#5_!41nNp)O%P$|I zJ&kPj$twMjG(|OqWn6IqO_3ainIJiYRoy|!&jj{}kLv9}=4o>P^!U+&8auRUy)kl? z_W}^|X-=KsX@k>jR9&9=3c-`NY8w60C{Y#nVc{XrBxdhmzecanArSr|T@-%#o8AQBweUGdjECcMaB& zTo?(=+=(pYQTIZsX0TNBabyzvl|vM3ao>^o{_3(^7Gr);V-aG zC97%O^fpWLG2OABT^iH>WYQdl-Nzn`>DF(R8Ed7&wiBznTd3kz9m8oaZN7Y-9ZnL0 zr#T0a=coU|QGzCO$M$hy($ugfs&G$3pB@P1Fd!&nThH*TG1TZ25!dx;8bbQWMiycM3 zWPMa1Q}{LGEx`3pv(m|YzFGohfN7Wd^g5qJTOme<*{G}PS1;wBa&=w zvFDp5ZYQm|dg|j_3bgJ^loqz~f^2>4>+@O5&?>-09`sog^EQy0C zYDd}Yu5bSW(-%gYBx31NEyhXc;JghC-yih1KhCMf??e?(p0#sFf7Ay z$t1Cqr@PyJ8s#@N=bW~w;UO-^^bK-nu3Z7ALj3yt2ojp0+saUJ0`1|!)1bEUN%QHE zraV#AKSB-i=bN23<6>u9xukqcwjNk>Fgp39%y>^x_T+AsxGLo7C~@ooJrs8Vh+I~r zq=m8XZci#b!FqHbt^yS0j4y`Q>xEo1RdBJ|5< zk5wB8gcBpR@TcpSsxUqI?^*YWk;6qm7MJ5Yw}%gkRrEo){NPB{zz8Qp8q)lCRxHNm#b>;6jTlBmnDCRR5R%mGW9<`sA-`JPzL)zr1b)%2j)7nuZFubYp^-|S3mK|QGYbHZM0OR3Sw0Wn~S0e zbN#eQ3{nezxL#iLtE2`p;Bmji9a0uoE#`(CU|b1Q#nm&)e$|KdR#!F)!ts} z?(ue7U{HxK4`a!d04m8S#YR*{@xC&K%M!ts>qxu$?tJWoE; ziQ4yu6J4#{B{xX!$NpSh?>Bs`o@*??QN7sNYcJ>X)cC2I2vpQG65grZq2ibwSi*z0 zce@AVexU!cS6hwqN2_pHzugK*7PJW-5nzoBgsa zHM_l!*{PaqPc^Tv7`V+p`JLyf+vpZJ^HOUNHHkL)U#zsauXkWgVrmG9vDo_dnM zY4Ut;u6R+VEGZ4jD1lJeEAx*8cC%f1_!bEGeSb+1FaQ+uvr+B9MA?N^ntVoxA^@8dr{G1p!Ue2cS3UUFmME@sH@mUf9+ ziB5>hfonIb@gP6>8Q45iCe{z(k$G#cT_lC!nQYvA;Al*Pz-oAM0I%Qr#H|kescT<6 zr*%!0k=u$xtoK~7Q$v!HD(CquGk+iX?gpPuw;q?;KQ;GXssDwBf-|9FWJn>N_lte6*nE%{Tr)vcqsZkKi;uAk@8Ek1$VHONp zH%fvLtnJ2D)r2ch+xy*R(8OiLr-}?@*F_(zfO~)Z^Miy{s_JF8 z9;gQ2jUCW-v$PpE)fQ?MFD|u)gIOamKf_;hqMmLgwon)TdP;xrP1M{?EF{vKc~iNBb=>E8Q#YP3n@%hp~dv-}s?HmXL@h&`0ElMSFuQukHB zhnoZlk?ttjuE>5f^Kqriy|}HW%FoTfPFy1rJSNW*(88tHKQIEw;0=ic#BalKo#9V| zwy3IJB(%`pq6LrcQ}l70m$x3Y^I$id2lKsJPRwdfcD{itIdz&Q&^tb@{YJLcwEBwFQ|Y}+=2fvswP&0+8_J& z7LZ1hbN$yl*$>%Np;f0a_6N9ap^&-nj5&aD5$ATW@pxXHlfmJM2}(_ZRG?Q{sdr#l zOzev5{ncaWMMeqiJeJ$5dZ=rDeMScah}r5uT)s`X0Q;iyq_4%!rL5XJapQCc*CqNb z)bi#TBcW5h_x9szi{2KAoPRS~jz8&rF6 zA7X2j%~+@eCUM5d#euDH$QbZoY6JOzT?W~|Dp%?a=Lnu)g^dl+>GbXmR}5r+$jQ{+ zF+N4IugdPLSH_CY*WR{S&Z3T=^CN~+Zr8w`DKy&#kloGXch|xD_P-AN1I9|!2%`WA z^@4mM{2g0Vz|wK!9B!Y$I2Vbw>9P(~Revts5a`s`^DDh|L)&k2I;{xU6688A97L?v zcs{qd`&5`1c!4A=>YE*+nW?t_yHMXpRqt4i0#1tvrkqxU8jU6vVhCe<_B-2~KnVrS zAl0=8hQ8l1Q55RyrU;1l>MOpD?2IP2!$jbt)OeJal`TdMTh89+KDQL#uzb*Vd_Oo1 z<8&IqocL|7um)qwRx)Kh`SNvKS}hu)qnR#I8;&jEZxwD71iZe{^d1PSeON2B7IhgKG?tX=zLViF zPpJK*UYTR8@J3ysK&Wy9Q&p#C0L*#qF@!CPrpz~4GOv^&J}PYhHftfH4xCm9iwOnD zYxvG*$+yIPT@q`d1NwaapU%{_KJRQYEc;T@vZn8fC~2X*wcC^8O6*$U%vFk+dBg!D zVfWr@XOb{a)2%P)%H@aQaudeCW#ha{P!$s6`TLoxK=K*jmcFC!#Y4%8#tenIH8ArA zG3%X6^`4<#czrX))7meC`D<*7nRV zT=x5R(?A1kur4%ho8Hp6K}%3tE>Nl?uM6b|ASK*GJ^#iB9dQS|+`#~%v)6=VZ4rd} z{O7a6`eBhm-O-X|iVp&O-8{pk-}Xnm4GaUr^bRAA`8E0Gur-qg@)Pz4gifus=l$c0=fT$&Mzh#hBs9crPTOl}b{^MzS}RAQqhy zCyhMm_>>e3;Gd{bWtefwndxfczD_t}_e_81%q~HQxO!ee$GOjGdq?bj6M|`Chb?mk zm23~%f_~!e)q5Y8W|hdmvYV5Ccq$|ffrT~SH}P6!fN6CqD7=~i_Dj+Wn#YQKC#P2; zmZ+@o9HO?@@LVEupXt^?e$r4T&`<#^zd?4s(iBK=4=9GjFSC2I=@&egop?oH?E==S zj2t(p5298q?Y5VbL(gD+rI30rDisTzew+P{|5xMs{-WBMsHTDA@|qwfBa)KDqcV@| zkl~n`xYVyko%w^hbYkt?y?2tAjvHR={*NYmYvy;^rCY)cD^ETi3;g6O+Ir-f=Y77+ zW;L%TQhD$I-hYS2lWGSas+oqKYji=yf)&3?GPVJP8RX!v6`LMkJ6L36$dSk zUkH4idBsc*ghq9b&}N3zMcp(RrOI1A&8q@gbLSpJiVt9L?89YjtNNZ{m>~SPqdUH| z$jJY=xk`$!H-35gj`-)y3}#Uq6f`rGgz@~qH_4^W?>HGb|7=04B|Ax9aD2uKRuh+& zfr{)aOg-k<;uObo$9r!!1b~_q9s){cSMYh(XCHRqkl7p2jh_Js<`Rst++k6bDpJt{ zWYZ7m10Q0V7ou4$pZgq?M9=;k&{h%XO~Es=?Ot}ZDC*_OC-I2AxEMcY7akveNC2~K zRT4oRdN>wv>1fj#MwyCdz>8Pjdmbc=pq`4P!Ft3~#@4ZILR>$#2GZwa#j_+o5V_Zk=^Hb`K}#u}9@% z>1f19_w8C3(_I>CKgE(Rjj_StqUoq#H9H!G%=OrQA^*tPJ$+wg^O?z~FSzZ>u8ZC_LFq5&$T4%|a| zl_lAqv-{7@dkp>c#7!yWHigWmTX9?8K2)46^{P_+8?LUota5VdZ_SX}EFriIcWGT9 zwPD0r^KT8+zy!2qu5kV|ZOW)-HI^Fg-%D99b1PAR%v3AZr1|UCcn2J9Zs=-{kbM*l zz(Ra12Xo7!E-y{tpGTi{3;`-t|3?sj6}D#f){qJgcZ^Xc!0X>2gv);iS|r#WGs#>t z4g25+nXUx)m)_G4jWJcVy{Ngk%EVhW!NN1NeGl<8oCn>a%&$^qq)sg>HL1^RPPHa+ zCxup=NywW`HSP%jltlvwMZF5`*@`Z#gIm_(!j88xzuiHLM7AgSgScJg_d_OU@S!C< zRDQRr3On-H2f9a0Itmp)c<%vUZT|CqK=m%G(e|}`<^KNUGc2a(x3u=M&jg@d)ba)) zaBW{Cg&@8SH4L6_jWx@>2hC^h302MLm~ItkPF4M?F>a*UBmPBa(|S}Y!QT|~krI>i z2u(GyJ}V2^7Zn2~n8HCdkq7e`zLOU$Zg7~_`?bsro(&=fevXpe(P&Ay($cA|Qb)%9 z%}pfm$Jsppk+1paL66k|6w1n*8g3`fe4myX=?e_FWP1P@hK}Rxtig4+{t1-0wTVK! z7L38Ad*?Fa% zi!OiWMD26ZCZ}e%>G-XB4mf6-<@WygNQy%0iEZU%utVJ4?;5W)lcNOez+S zpt#;dDhAG+gkacq++2c4ayYxNi7-3;)q1<3hE*esVq~J6!Znz9sJW%NEiC=*@$xAv zLSleNtsg?8?^@V|vW>4jjuQu!|BI_dJ*yv#9^^#G^%uP*;jG#I;5AZCp@Gx%vaR(f z%4J;Z4T7Xpzp?&5j|uoL;ox21RZ;O(nZWWByfksw0X@qPl z1KEqvV+zi#w1VfvF9xTyTviH1_wv^@R^{2HG!~$t@UIZ>T?jnBndoF!bDTYBj5KG# z26O$`o*pWmKT(GoDj4L1GXfNafq69Ne*>hAlC0svfe?syhCSc5!rv z6G`~3C$MlI&j9PC<>+UCxs~*u<;?_}Ri2iV@3mr@7yI8XlXbWP2M@+dD4*%Q+Enkw z7A`m*gS2p!DCZB6QQy_ghe<=7EO|c|Q7~YlJOBXO)uc#9{y(Ro?GMM zqFM^#=<|6$-#Y!wU5?nZtZGgqNqKhSBBaT~t+&T|^;v^+e}eLH(#n9@##ZBrVf-k;4&QGZnib{HZA?_<$9m}9p|XgzQ5`4MwVSR53(^W|(OAZAv-9lQV=;{r^W8tHc3w(@|J{c( z&a(!LymrnZ2t@-OScn+G7$K@0iwDN0cepy{4hQ>90jE)+u~NNRztMJ&N3W2;TMH_9EJZ}c%HIzz*MFOfjyF`E{vLA{*D$cdTkS7PBDBXr&+cu& zMQxd$V&Am)(j#M*zsz%dE(FdE-=_O;B@|6XQ3+r)!}{P}GH7jrY|I#vX&fjDnkDir zw|pdDh~{|ZoF?+D4pAKYPxp@m+XpD{+Rlz?{}weaAsyPbZR~!r5hTf zg?t5CcQY0B4yajp=Fq-k+u?w z^v!OrlL5)l{XZ9N1h4(G9uuUbJgDl@(y=_uZ5ClkshK7~s6-iMW(SlUrT9FfI1coTK}qY*)U0_t0b zyA^cZN~2RQyI`0gts}Bh=vr*HjiP?u{iDxsGV?0Mmb7JF$ORU#O*;Z&Lee&s??}8c zN_F0?Jep!Tox%{AU$L--uu(alR(fOmqxM!PzxHvpTZ}hF;4Wi`658X=w`Pjop8x-J z%Ub_F^Uq4{VpRpcuu_zluMhl?M1M@nYl)Ia+z=Yh3{jSNle%6(IO3mmI16O_{P%Ev z@n>kU@!*)}a=!Kc-|X?gwRksD>W|ZRyrn(yA=5tnlTOR12PFTkx~8^a6(dlt{OSLh z!s=f2Y6pqhRGS>WeI$72Ynd@T=+Zk7ZTFtUX`XwybuIz^S+ueU!iAe(CD}*SB7JNi z+|n~X_3Lb9WU-W(=aJ9h;dd1+#N~)B$eI+fk2?eF@VVdC|A_`y5z&I0bbefWsLs%V zkyhiX+AZD+4PtQML`qQH1vJ6%I!}^9-x|L{TN1=io7R?sl`{q@6*n#)6(0fy$r{Op!085@7y?0Ux;F6J3)$ugiJ0s5qsvvv1YcJa~s%;H!{r_430AQ1vt67-?^ZC$q<@h+o4~n8}l|7C} zAIg^N2zs~Qr$0oN8l>MYfARZEUrELGnVCLQuK1SR<_N^?P^Bm z?VJYcP_fT8m~A?(F1vs4-Fylt)qvk14{$iV)kDnFu>U3~kU#T~|qu%WkqUBcy zdD~ma3d`7N%Y7|>o(h@SUHL6gmpV`T!z;XxOR77w*q(*M7;=v7!H5@-c?7%FOpuk+ zacgkBAFW#3ZWU-yv{lRy7T z8!+}~*3R)2$^u}a$A3#HagVQQyN<}>___L}_B?B#!j`EVR(VBKd3K#Z+Z{3>bZsAr zlUvfby~K^GeR}Ws5T5UmN=`mBtt+nW6WS+xb(X zgkVOirASsvK#Zs{Fo%}8vmjl!J(`7c8Hko$Y?-5M&NFLc>cz!(`L`$&z*MJH`2-Qb z5IiIqRnGQ_$uXSmz$G);6#?_^7Uw7bkTNklAZWYEXEE-hFr#FpPipvLNB$4rfByV% z)UTIQCM2nQ9wq_1XJYVZ`+#1uQAOl~&5JsV;c7Elfe6an?C1saUuH$-5fUv2fgdtE ze%)syB=sd}==PluqU^>bC5D(V%nvfb+D;T;NbfxN9#1A)XarY*YEl{I^J+`-pZwRQ ze9SFlpc<3C(Zk!V#tZ?cM>||!&(|%}1+A0#9;#vLcI!lQAlfx^^2*UpK#B!ZKK0T$ z|8{qSNs#R^4?+9rqi!wt27>>*hr`xu=t^7w+fx>OyRs_Y05#aC*!_kO5v zvB$oU-sIVdkRzb3aq(HyH>=xpk|oJvAs8A}q!Kj!Ug>ox zd#nI<*DGb^BFhq#zxvkw8z;*0*>2FA7myIPuI(226DnIqN|SM=%2Dze3{K%}jk z=$^Y!8F!;}2iQVxtX6+;5?jqJdlBuJK#f0`BgPe)CfSXW1dRE9m#1>y3G<}JLd{2g zWJ&~YAG3X}2l--Aso-_gy)>aAKB8!{i4u*>kLnAtoP6jur!6FC1(V%)^joTtH|(-! zv#zeA-%T#5O&MIQ*5Z#>k^!9?s6VnygYDPB*}M$bUlJY9MtSw-@;7_uKRPl;6BoD< zxANYh=_3g+RKq3GbwSu9**3VHHhYpcVE3=;ghgP4(K$_Sm5n6#D6x;xdh*V4gHefxi8Vsh36)Y~*4 zT}tai9-~sHwWV~A|J%ENUTgNWaj8LLHj!pMO?TK4W1d(&5?fL|d)bL6B`>Onyr~py zO$+H-zOkdZQyjJXSif&qA-p!)?9zY8kGG@u7a8B@V*LfdN<1Z^t1J*?3`EuyMZ5gxRO@#eu zt##(c$e^RBt3Wr@tIBPH{&;_@dc5gACQ!=xRNAo4GEkWA$>JZYO-xXB>JQSIcd&R{ z7vSWzZ}`~ccLWp5dsiHKQ@mX z4&M``hX>Y^uJYd^Yd zHP6WLu9WkPZTEM@+f#5mb|qR~Ny)(D+3Do*maV`KVx}O%Um(sfP;Nm^BOJs@0F-wj zypFVShRySP zmZL?UO&2Urv5ZfpbJr(zU+O#-9DJ1fG4;dckIt!+=X3Frk!t-j<&#rp2 zLwo+BUguQmnXJBhS8B9yMK#3vlJ}iYrA{B7O{LVCysO;9tI0e1t;mZ${B%Z3yR!SN z;MaU^yLm}-V%QOv-jmrQwZXVIZDtllzZ&-IHg@te3x3sYzgOLEHo6u`2@~GcSjt=8 zdY1HQqV`LYjkg(PqylGVVHb=@TVK^E2$ek&mH?W_E}o?=YMtIupVCB$@~l2HZq|RY zuXgR`fD@0&x6PHS;~MqhXQY>SLE1;|FF&nH=EC!6cV<>GfQ^c5tj8sGG3fNf!Qqd|mPoFf;lU(1P;xcZ$&4m{!YBybHC zdXV`>57^hXi3_tNRD4=43cUQ@RQZpp%=jP?PxPNg4N-?1)fSuOj4Q%a7n z6|(MS&L5KEpBkO!LO5RaZhrdW_PeW)8#1nu0Z_H!s4t1TF5L>=x{9itpvBSZX^SLV zKR?yfd|X}w^c`H;Zub;XH7N3&M}JOf+7c0amQjjar@pXhCF~(Au{QeLbBGq4IabG0cdAE!~AP ze-ESveR(bQ5*KAOU-=LkOqJvA{a25`{HD|6E(d;M7^gp4z3Rq_I?M#HBN+2hNC$d&aT3nOfnQ`m0lC%l&Nl~#~v}Vxr(GorD_s29- z_S1LC;V2Pk%4wM*;&%T*l_+ptqsNRb7m_-V2h)Re#eS9C3}W7vh&QjB;r0llH%H1V zOG`>lROyK!;1g=y*9n`?==*C+x_8`;;t(~o(yc!x$c+^If5_&7AUK;|uX=pqLYVm8 zEIefCPhc~6>zG%Cy#p%`;A|r`Qc;MJMnNdsHP08sWuw}RbeK+=XoQ1>(6_Q?7NPe9 zON!kL01f4FDeXu5)N{Q3Ba3M|OBS&KmVk>~`Y`?Z0(8l%xf`XNc4G9z*Vk;X#5l?^ z6(s~dVSHv_&7wbIa+FT&)xj&5_pN;ti)5({o2tLYuUOeT@oEr9jc!_UzCiWvX8VKG zHo<=TuYU+(SEi1gf?K=&tkxNO;0Y@npF=A67kyEGj}%`qR@>@*KlAnG99c(@FcX(gr7z){KFKP( z-O*%q;UOI={i3UqrWu=^b(D{8Pr=yauY_kO^#|WHl^FT7UUROjcquL#4)&2JZi5eT z>3qqIC4)KNCS_dFf9GGtvuqG0enm-kL0Zc{mV~>EN@2Qc)E}%f42k_W`0X88=F!#f z-6vv9fwcjB6cvk11kF&a%%`=lM#YJt*pVp=v!I)T_Ljv+;OM&RMOg~fq#>C@L(|Qy8~MIKs#1_s4{r8h}cXj zS|i4q;;VtFhhMO!8Iy{=XyMmCp#6Bj>tJ(wo(ma}dK`-a`|{UXOh*M%+N$8rb#Q@C{%J!I~W))@a4ye*)7o-L)8YvkS&=J5H`R@KPD%9U4U zg7I$B2)^#`^KJ*Z7V{t6u4fv56H{*Tof`U$Qd&u(W(K|kxC+|$VndXD!X-eLB8kX_ z!MHx(SiWcdsR4rOP3~jhzRH=?xR8}fOB2oyQnnjRbgV7D5ZVV*XgY`MytP-xsWITz z8px!IYf;_F=YkBjLq68VO8#2ZR--^ggQTT%^`&J7q?&Xxedqrt6-TJ8*HZ;&+ZwXx zqTU-JdlxqXU@-X@{X1!#m38?P)4Xpg*qC3nE!<0^h3FnA&9PaKB*f#o#3O~f!yqc zYZ|NT{wa8pF$qdyD_m%Nv&E|wp3kc_-19%@`+u(g%x=(HqUX5vOWSyKTzWrHG>Pf% z#9%KZ_2bSCmZaUa8@5UVH}@*Wtes)jHA|!oY98_{1jeY$L^Ef+*&hz3w|=`09MLTP z-FZNA#^MRQ~Or$|bcZn~nf6M9D`Y00ep zxAW$)Lsj!!{?po+W3>zSy)LBmKp$_fdXmsQ`>}Nc*JwMv=t}*H;wl@K+Wr6ruW<{-gBwLj#TqX-HVU>M@`d;eyROX-r5 zpmD%w)6lc_#C`S0qaZx`vEsw7WVqvp=iA$?`sS~&=zFcg8P^1IcTGH7v{QF-__pf5 z!g}BTlsD3kM+F*CLwPShOpesunDn&gvZME%9G6vPw(|`AZ7%8+xVEHooc}9#z%f8n ziyaD{Z+PmMQG;H7lziote-~{|`UH@8!0TSupRWf;jpj;pU-KGqxOWs@V|D)Kd)W2V z$*|{7oHk@U8$_Y)AAbrsyA=QTE13L(fpYj$lf8@=Nh;+|8|9S`m?1+C-HiC z4{w8aJ0Zp`dMDr8&)mNHUuZ~DnajmXp~qvANYT7VUOCmJ$4~OM%LFbv@NehF456s~ zP%+u4`!P{J)Vl8+8c;ndnEm&I9s5OuoWIRS6w^rYe>lBT5yJ`PLOLle|3!H-m5iA8 z1lDfS6oZ5`ZE0X)#*Jyadq)|@@WwQ=S3`R1Uuu^nLG8F#y-Gxb4B0W>Ghw_`Zwnsi z1;_v7Z-D19rQI8&KZkK|gtHeutGfK$tR*5p`^2RUH3ipXFbg^7{6&E$?`PkNdW>bg z_L?}z(k_&5d~1f52=aYs@K29HUZ&z5FMm~xH69l(M_y}l-sP1*s|5T`N8LO6qWU|# z)?xN6Ljb^DKm4;lozG_9nW_&=1IcO3h`qs^CDA zjFQY?Y}$xaMN^Knn#qKS)Dh}kX`b#c-Z<>uo2u>&?LYT(8!JtI>{s7=GZc<(mUJLP zXV`EmPOB+mi%nDY%q!YoFeaAq4lpE*V*sMRZu z6BlBfF=%oie3d{=9U>`Z|vM^kVg= zlMO*rLh({)zEHuT`yQZ+{_M@0tLmYL+l^QF%63T#aXl}qkDT+`Ys@m>5C0)edJf<@ zHdT>>(;#0Z+pA7EW-@#Jsfbqj_qN{8)2S9UYFz7SuuPv?xNedZa*M3XyIx+k&TDNT zzM&+j0o?-_Ot`L|aC7vps825#3eKVY5LLZz+GWw+BWQu*0QKDB|KsVb!vRlh9@Cs&ndknf~B0}Xwhu35agQVAm(t=6RTVP~p-%s5Qp9Ma+ zf=sYyW7EG%uwE$mBx z(r;8-KG{+$3&urj3A3o*+;)fI8C>EZMi07$`^yj(`ARpxA<#9Knje(asErKw35~3Q zG_!Z{2U9sd^pfYMN-J@{W?T_53HcL-*7HuT_e)ke*NENt?XD*i?5m2H{Uw9$0*O7b zW_bI!dk-}*t5RYt%Qe|?D2ymW=mN5cw$8ji4MbX<01UpoD1?&e51|k_AYvfraAMs4 z+tssh27{MO{)+{D(|c=>N5q`EJZ$^loQWt0oiH~26LvDHWjVZxLT{3>`2M-G?-=Yo zb1Q(jbZq2wdyl{x_!AwT_u%5+l8){!xn|t<#f(x} zSZu_o^4?Q)ehLUUiffYeI(MYb#n~S>l+`IinoKJ~N+3 zdup%PX^x7t!y!1t3p)MN1T=b?Mn6v=Zt&;IBr}?<=Y{=;;Dl59e}l3hc+M^BL*$r0 zeCWx;AsN&h>#eF@i*cSp(4Rp|CD8s=Ag4KzGAKKs}?ba-R>5K&8hm zm^C;>ePw$aNWzWYnRJ`7U1;*zZ1)TH?E+UTt?<>SjyPxWBGBovW(LW;=eN+R1peZF zC5*@J2hll2)}L7e7%$cRm_w@`_sJI31bQjh5I`#tkrc%(vK(F#qpzbZ0D z#MYMR3~+l7&_vnc!%(g+gjEYfj)||OZ{8r9hxshQgrKSCfd~_LIh7-wz$}RIjd! z&4o9(!s6m`J-OB2%!1xFW{YahKS6iPSnm7uD!8wMVYYr4-TM)dyx?*i6@6RlXsx*m zc{X%UlCb#fPazn|u>?=1e4H^G@{1zDw;n2{<)?WZTlB`9d&^0rD4c|~Xa4@VcT|*M zE@x7C-SdEx12jJl!Glb-r@?LVOPItyyoVaX4PJ+(+Mxt8@iHUoNpCkF5d6-VQh%<@ zI9Jmr&Gu1MM#ABNfg}+padSc6j4I=MO7aKerr5~aK@xf^nu6)vJ>GxB0}`5ncf6OCh{Qe} zPvIXCBz1HU{UTAV2dFhmdBQlI8!s8CwG5^#!hFl;$o?r|tG;LiM7g#fjqUHVxKz5d zoaOBQU~!FRJH|E~2K7<=8|}+-3Ms_8nzvSggyr||kW*u?+BOe`stExve8 zcnVXf`WP5jg8;WG0W%{5>fuv&`pGV9zI50p2X{%`S2tBUV^55*i~|);CvmULFPm@V z1l*o_JfH~8C@0-z%N9SJwt1gwNTQj79*_P3I$hRVpzHjt`>>Gqh=tadtiYbVZfES> z8Aacf0DsATv{d>o3;GK=-5r)nGSzYek43+af59*Rp*Xe_&t}4Ghzflut(vyepAFhx z2m#1^LitDb!#v2gl{+n_D)FjqNGS z?0HD18?+CHJBDk@^^RBMr2NCcZg@c9;eZAcsb3!@Cm|7J*>sJxM4I z5&z8DrMhNhQD7(Qjg+`nU%g-w{Vj(m&|n@TPO9ZBn}2*hIo_q`N68x0ad&(hs&mp7 z6Z@tmvR|RjadQ@U9G6n=3*1!C>X@zDUxB!JUtW*3-5!CU0IJN0^Tvlo5rn%sr|c_N zrVeg03-||^k2ORA(hKb61ow z&Albh{!~iMy>{@$bcYUxxcj7i_qX^Y)O?EMe&gG4Sr0bTz`Ih{GP7?tv%YVXOsWEz z1;Sk^rm(qVZ@d$(u@L+v30|Hz#gfI<@kQKNi?xnq@5To$0-#Xs0oUHha_84l6zLho zAsH#7P+I>kGe(A=_N?1Jo6(m?$S)EkP0La@{UlQkCR?U8N)6-g`PzZWMYDe23)ONz z$_mUJKlRXhOBj{id5{pphUjqTuR;i0Ssnb=VUvwCsi(EhXry4x%*+7aP+(dZd5Bdp z13Eqz4p$2d=JEG@X?y4n@YY3BfmZW2>iuj0Pgh;?$F}$>q=_1nv(o)}ou$sV1J>L_ zzAfuKzo6bfjJd&~hhc594X&dSLLTdJ_hH$xolXs#c{ZhH8kFR6{2=cdzvTF@eb(CZ zl@++8G&-m_JT@(I6aQKjk6`Vovp0N5F4rG(yqkIyq5Z4{{ufKa|DFmU9i4s!FGGGyC#%tWsP`X*u9Bx7`^OJJ`6dnjb> zqB3s=^^QK9cS$AcwK*mjET3qD=`$3#?J{jj@7FIdH%sfB`ZH;67uv8M{Y7h7vQUZ7 ziQICFyyy5yb1@rMG}U!-XMAR^SkLqqByAgFzsbGa-4s;Zd17p6^{NX;Au$ z_zK0q6R!PoIGW0Tf$@wlCex)&;vN_y<8D22h>OA}(^o$=H{Arb;$&9ymOhQA&0i7( z1*cth=r(}_NsT4H?XR`<1vc}`Z4mMW_DfZNm8;_ZK%j+m+v4bSHQunjQ0XRueL(g8{k70-Ay;vGuyz*-DqrFNph~ zTt7q$h9K41f1u6Q_huY6g2-}|*nN0ELNZ}@<1}dQ6<68)_frnGt|<_z-@zuvcCd)Q zM$)CPQ!vPFYuF^;rnV#FFeHv!lZ3b3b@JkrCMYEdnLftMi?^qaV4n?>iD2A~$wmuS zvY1K|S{Qjpn!j=;GO<RNO+2mpDPoNW?7NE z!UJZc%%6o)3@s`T%7sRv(rBdu{5g}Mbwz%zhfa4LROvav6jO9dDjV8?N9(zuU(nK)#!`H(RdwG#-43IwFK zw&4%b1wE~0$reZ>n|E6aJWag{0;eA6h!-Z391<8VmBW~L1HWva4Y0c9D|P>oDNiV{ zI+ciTD)X*<#(KU#?=qfwGXFHs;wf5MXusHG(Ge5YC+$A8D;C$ywB-@z9Dj|qCR5$l z>Y4*gZ#0Duc(WO>n#?In&!|IFP=rHwo=q54MPVt~$TZdLv2_aUddyYyQaCrsFlYzZ zWjgeG+%-V7D;U>}SHF9~*71(VYJ#>^ny051uE9c$Rb{nfV+^_{v}8 zGPX)~%t&{d=wEW(lQf{~;%&!XQDw5x%<{?co4XgZL{-vpA6)%uwoJrRu6?*ro;7ed} zc?<`hz+_Ot@1tE6uP=}40MhH5p8_uS8OEdnV_of1qPK&4hOvSfgejXh`ZbP5_;4OT z?6jUkM*F7;`Gn3(hngSCR!OJuzJ&P3uWp}d_1@_a;SdfQoQbE%Wqrg?1&LQSO&0Ym zAG2CpM6|5L^}NJoCpO>0i~k}OZ=TNl^?l+4t3SPpYI*W2OyWbIb>D_e-*JgxVN-<; zQ&&pQ@Nq&?$gY_pNGj>+Wd&5jALYL#XF`m%z}M$X2r6T|_L;kmM&5xgI1i$p|EtU% z8zVK>A(5_x zbPxI5(4#52Wt=Qd%y(XCV?>AD%PXqBi#VIo0Ua_Y7wh(iQwsNm5?0)a$=egWXn`u2 za8mXAYU*(41IiLKTF2zeZ#?igJoFury0tR*?*hurM9ZEr1yM7=ea|#gMX#P z>~y}4Ft%@&Q8GGl=^Q>iBB0Bg)Q9=((#OwCXYV>m1#(IBVMCAR-9nAdKe0?)Tl5MKtW@BN4rF){ zhYNlo@iR=fAS&+~q!-hd{S&XhN7cu#s1O7IuBQ+etzzJP1umyUxTKl4SR!R??LJya z{=N0&;r>k1YK+_hqm!Sg2Ln74Bg6r@K1xY6b>atV9tn!^+ieiPtZth+e%v-X!luwP zuhz&Dh@zn1j&kyCPHSX39_^7oW6Mat>~ejtR_BgylI~e+TV(8)q^$mXjEvJ$L+NuZjxtxXRQh^FJ0n2KB$4vAz44;m*1X>5+yd3= zt=}&)k!z4ChyA1$2|CSaS?mNe7j<^)T_SQ&xG;jD$@%QM&w#=jX+-QA?)QK)AtK-$ ztB5mF5t*r{I~jZF?|kNyydM&=@^Sa!G`~(@y8t0fD;h6vX7rOGSJ?0U2 z;r6CAG=xm>{pFL6$L1((Lw=a5CbjM3{V>(DX|g_#OEQ6|UHz8+`b5$13vV)L)NYRs zF(G{SI3`|VZSa$PTvR}qPKsxe>M`rLptdu^Br$3__QvIMhnDa9F|rQq1Ouzwbtm!} z7QzQ(vizdEA9_w2lAn2#-D{|qwOtWVvH|>_gPhy?gei)j8Z>vmc#dPpi_~<>tx1~O zc7)>F{)+SH!Xm7DMd0ZwcUnNbF7%Fh4R<;sI3c=*zCSX z3?pDlm?y?6wyOIpT;{jKm2+k0tTAuYALXi-01xqipSb78sAn;vvDaGY$#Xw?c0p>V|g=k?X2}ezhHeep-h7?O zmcP=d>3%H1gmYKl0)UvxQBjBJd$S_u_X9-r^<(#b8dkys5@yC1yqj#}zod z%$*w#qfb${J7+L2CwSR8XkX%A=`<+^4(yEq4GQMEEK<^#%)7$+1T+-!J-Oj7Y9kKgqWWK6sa$QQLY zBH-F_u)1Hggdm0dq%g5&^d07Tp;Soa^&)i(S5`si>fh}#|G%HiKGVcXP7Cu~DzC3s zi%{vx|r3=7Y%d|DU^9OD`jyq%7k%#=H$P-%sE-|dKF}x zj5HS|xbgPOAu|qT#s@JKBKnkAGF(qQZc3&)9hSaC&QYRolh)d#nzj)a}FPuC~da^DC{JX+#Jr+xfv(q%j2%!z$!W*P2DW z5qmYJk2xHE#VgYGw6z-|P*nq3Yw%ZIU_1w8H#004qD#f3B;VeOe}k_2-rfcC8zkvO z^vCGvD1A$vEE$%VYd<%I`@KYJ%j&elyO5P}jy85xPFA18$k=(zJ>&(0>ym52vaXWwxc;@*MJ+ifp#H>=|wQt9$Vv1vz z(vc$92XEM&y`_%{DEPXQcNcokxMIAc<7%?>KqNmjhXZc&l9~!CJ0@wUu*GGWZIPn%X2WoAW}^CM8LGh_f(}Y z<~&49)Q8n59}z_@c#bVJxXX`=MJs&TH6(IvjBQ zBjnwtX!>KwFo-Az%?jULmw2G#7%MBHg+ev11c%CrPJxf4V0Wv2R~-dxj-9K;X7<7d zO@f{`nTGb=S+NcINiNj(@jUWA`qHT6w45ga-Bhk2PttkYyPt|wYc$_h1oX|8>eNsQcwnQ z8Sx5q6jpeKQ+0N2&if!;^qiF=xrRfISLLC?_Og5a?gN?Vfomfkl`rfec#_;w6s=5k zWv{+#5i8yH4xg|+j^^SjCyC1P;%w#k?T;JzPKN%dXp(9xK%<)$pFU8b@NL5Z_^#IN zSiw4G^A*>_jBQz4U2WsecTwUDrv82p$0*y5nJ6WuroHH zwcm(YyXS&Kq1?FpE3Ot|BbA-Pj6^1QQqTHUgU0Wu6$S=vPv>$TaQQB_)vtwi#R`Y@ zTg%5X+xdZx>Fqw+mZq@!#D|FyL2xN!2?8=75zmW1XDp`A;HPHzRi5Rkz|Rc*8vU)l z0mN7+a(CGg;ShKsxHkgaLCf_0;ZpDPxf*NHxMd6QH6dd4w{F02s{cWRQ36}#_NFE9 z^j^1tR)KQ|okW0yh%s_|63b_Spl8Pg8h!KhcWU-xtlZCZ;*{bg?K1wxH~wvV`BTI{ zOaw~M>@eJ%?P)H-xGksbitjY7Pqr4m5|L&`joVY; zqbf;%aCd+_HUjvhsx7 z+%mj?X(ezWUZf{iL+%P1*S4Dq(A>>>q;aiAFLOZDvBuf_K!Ec5Ex^AV2QigakkW4& zpo&lTg>kyG^Kn=E&FroFrQ1NrB4g?#u0mzq8V|*C58ww@n@MSjqu>`w+uqD4jQXlL zXA^k9Vi!xj8IJy?Z_2V&XX@kjdVzH8$dqIpx9w7bE>Vme^fhDhrP`WlfQE|=$3u15 zjkt{;IO=MB0F?cbf#73C{fzctmOdZ7iyq!wB668GkuG&sVU)2}Rdznv&CPT2pct0Mw^WlfgBu1o^$nV{9uzxh`b2B#M6o~H07Nrwm=gZ+l3_-ms-``oNAk*@vj7i{(r(#R)%=I zS=0RIwT@97@o)E+g>Z@HU47#fo7_swJReYM+3X3o&ldGj#t^tk@yJhZoqt@^^_{m_ zTa^F98h-a_u0+)n$Z}rzD8A0@=-)qK+8$4lcFC+1T+&zxy!Wy`4OZo~>Dud&y<$AO*nS5kH0qa*KrGyLI_j%9$OvR9{D)PHER--9D^o;!16{X4jTk zp-+A#SnuTeZ_6feZe@`Pmq6p&np?NVZc8CSm)2X!ad?QI+(WK`FK<~_;os1| z$QgG&g{umNiVrzb$a6MJB$xRa2as;Le;@!<-gW>c1aXh{h#!U}KoA{5S@e6q#K}gk)G$h0xa_ zWpkZ1yOx{jvz#Kn$qTXl!I{Hl@{@voR~*_>8QGhIlJVbL99x=^@dXK|5|q;DIV zk6|kthdyvs6-hbS#Qd>4Q`oW``a+P63C(b~HWsK*eC)N6eWk!!{ZHCk05A1kz?gN> z=8EqYu&R-}`Mujm~gYSF~=5+$T2 z#4t;JJRUM(2g>vqJwH^sdy+RnEw|WI606W1m3aB}1mwl!(+zPjB);f^1^+6f1a|$4 zce4|z&ZajF1$ZUz&l&z@*JO7R$$OAhhcI-(p7THRDBW-D6VCbL3o{@ff?iU1mq-(| z1nwdhXY__HP~2!QgCr{h%nJciw* z<-T^FhEoY`tDhW}=^`d*&D~x|?VH&61Gf69-EKF^EbSemR!wf_jqRHSep8}HNQT!+ zkV;Q14dIdma?~8a(rf`!y!n7icQQFSVsTA7>pb$Eq4#rketun;0HCgo7JTnxn>Zi1k}6&$6)LoTF}>V)?ZFw*Mgk>&v1{VIe)9n_2hvx=Iri^45TrhJ&FUWPu=hr!M6)zr=ez zAkD+0Sox#aL|*YDdzJ1FwoKs~JyY9uyU*-9S5M;R4x4p4i&fyX@@rCC!B@Dr$Tkf& zaWy!Z@`YQ7^Z*WQI5JFa5Jy?sE#=5)U^d>u%VyqdTL_m_o|gz4Xw;#Oh}IDuye49N zTuzg-yXfh5Mj^Ak)rQ+tfY-fzHAppdO{L$t)1Qg_q1CAHX_arI@i|HL^8?(Hb5rxU z(QN0hfI7`yu}$@V?jE%)iQE?3ZK*f}7d0}Q>XmEhR!cIM9Cslb>c<$XFUf?HLf~mpqUh8vT8Yov`y3!TOkda zos4pS3{GY5RJ7NWS@dnvYqV7r?_ga*?C3NWpNa;OuP^M9oq0;;n%+ELg@~#sb3HGv zJ39_OX6gpszW5p8*~h1%_JGS>oiCh$d;EpBF#h|eIjT|*#xf16A4N+`WIaQ)V_rAR z(BwO%U3DLpaYCxA!yJxfkFL9XPz9zA9`9Hgl@x;&h zSovtpz-^q(i^OFa1Kal$K7;m)8u^h)Kgr$MpUex@p@$}2-8@5RW%}@Bmc&R>L18_f z-R2HMMkjC76B147mH8~Kl$*(!y*8Y9pd~3;t{s?h(Sh?pndJ+K429c$Q><8MNLGcPfY*?c4vL&B^zll_fu2r8< zC>R@GgLha5*)n~%p7VW2m83Vs(-N8`Ash?DCN+eH7AT^Sg}5%eLIhuYCe_%gvC}A= z^hT1O@q0h{SV@(=uFM$_?!q$m&D@q$)8qq-3%++e`b>i$hni3h_{fwSI)@ zuOI4BpV%myWl%`1a9McnXIO>NPjIw)J#m)o^D&d(0{Xz})AECzXgD<*#YgZv;0r0< zH`^wP$(H?9GE~zqnJV%DqVU9yqhq4dgsybD4Jks=Gs@UIH86;52%Tv4+$u$hJ0dKt zJ+xm_*|tFc#*E%-LY#FW@WD37;x@9K(@OFFu*}Wg>u8}#ZhfAXSw6*$6EL0TwkMO9 z_cFn@1>M;JE)%a&2S zrd^nYto?+GcJaWIO*fh>A7$CEyhmbW%TD-Y(jbGH z%>FDNOO@Q;X`Y8iREm@wa^V)^Y{ZZgJz3FsK+j zw_;yc{kn0%JwA_zzxP20J;>rF4=~{p&_hw)*Bg=5XP$#_rw>U z>Hxsq+p2D>F*RR?_VVM7E8uUzU=t5jv|SfAXMsH=xo*6JW7`nPT~Udc_m~Rx=V|aA za~?dz=9E5WWnARxU+6j1ZeI_TmNJ>5FbFeVk5nnAeml`%XAOC^jB32bzUCnhQK=1m z@~mAGOmIsRij`j-B~SXEpK^jdPg8OW%kNnj063X+5l+!aP&5Qr2*B(gM}-U^Y1^((8(PC zhJ#x6l(HC07~Qz^*?cp{KC@dl9!Kyie&@Af&C3)T~m2;mj z)cq-hS?7sudHQVC*TBzUwGRuAQ=|pTTlG|KW|deNS`MZ5tK74 zGu+C3J+5#6FQQu;Iceu*!AidqXWp$mMPeiipXe)Vn1k_w*8IjgNzGaghmcQXNW)fb z{>gHYnaH&l6<5_+^T4Z`?<$G@Ggu^SxFyJMhfg4!NaRvo?CkPQFAk*HN6I>aL(c>0 z14d#64MDeO8Apv*_i>#}68_LKsYIqWCsy-C8*`$RF7(Y3gMk z)(pFE{+4hsBVee_pUAwtNTP^QTkErBk*R$!fC=LfDs9bmJ1uelX;+txcXj^ENOpIE zJ5)_u{<=PqBnPbY`IM@L@Hru?_u-d3CO377T`D$8L)VSOd}Ei6JCjQx`4MLr`;Ho)Wxtw*Y#QK>%iJ42G8qg<_z z3FLwXc6VB#LHMm9#eN~OIE%7tKi`QdadRIQ=m4U%bU_!2sQKhchJdnMi?vhpNML~D zns#J6pDsYwai#y!9@LSI?_s)P@`B?58|d=@1Z^KMT0J5r1*081CS!j4fB_b8rQ2|0 zbnth5OSlcea?Zly=g6iJAKap0e4`ItE#khL+Y!Xm+5L8ir;-NaM|9I4Wr1ergYCw& z1`dhX1G(V#bM>lL?fp3~xOWM)ZJdWZcWfkTc(@q{{e!J2(K~ZR5F};ipF&^$u)l1B zsi$99Ez&anCFxj`btHLoUx5!hAm>{@4qW5?zl{$P+1P{*UlV0#dy(_(G8`_SL53cn zajK$quu4n06rZbAAo7YOwoV&SM>&o*_gtZ{LN_f3`FNj}*oCjwrR$9CoA>rDf7J8# z=)GZMi{u^HK4(d+n9XZ%3i^DwkE0>VQ|0N*dEm|K>2VZ^`3qsX9C#9Pf=e6AJ=`rK ze_N$>@uRJeq`j@Lcn40A%kSCHA4Ivpk_p(klt^UYSE6x&lXO;wXTt1d-3cm#!oWef zr^PnUm5=l%t2C;bdEhb7Zq3}aZh5Ia;Wgpk5v+0`)WANnFmmo-HBi72ZWE#;?MuIa zUHCtM4YK|*OB~hPChf84&vJXOs?+Aw*%KqI&Oo0&eab72P6-ZeS_OCAD2?LoUSTT# zeUxPLd+^(k3V#!Fg@2=#>G}van41KLGkm!Nhm)(6L_a2omMgW4=Gu(o`jwhu-Z^HO(=Ih3TIa-$xtyPo#7Sl!suq&h$XRs_%a4J;n zN#YBhlbJ~crhXs%48$^9{)0-WNH70ldDznw0;=(DEh(bU?@|~-HkdvMdykWEd^=^W zU;gUJ&tSvHlypLG)3HNW6mZ`6edIXD>TQ+c4Geov|Z4fv?*Yp3Cl zN?#{KNb@sDH6N#1;B0tqYxnppxiFDka>cbbosEL4Nz!L135IH8i#0Dj6t^)}L_ZJB2Pn;NmsixyiELU+0hbm(-h9XKeJBI^OMSn1hyX zgQ1hOp!pDmuvdw}!8o+MNgEMO`%XL(wo#(CsJtcdN*dM;O;6b1d_F{(s=@PRF5_E# zdA##&;Bj>4<2W9tm7cO|$QdGFhw>7;Z-=WGst(B3^6u8~1HT3>@Ix&;`J*BSx3lDr zo0nE=xZT^beX353xV}#^{g7f;7UJ6fPS7^hBuhag%S{#EjhmxM6Gw13set~io}Dp< z-269AF8kv8+rmwyIj-?xqL*T&d)p7oq|;B8!J*h5>4r>U= zCG2AL_#=v*!B$3gE7$?D(N3hdv-fs}PJK(>~(k z9I;Va(IbRGvt}lMs%2es?`D8AqhmKfN@LObd@2)hC&4q!GsOcre|Wd`t-$c^5O<7P zEV3za#h@D*z%>a9#ZqX}9sTpEB>ZXC$=xW@ay^aqlic zo(V+5;!rpmMQ|9m@ICNkbYTa>O`m?0^{BaGl?VVXwRE&wh)3p(HyrujWn+l}!V0&4 z-x=pTGGLg`!wLLQwS(N@lRT=;CII!exrY+8^*@m?`_GA0sc`cV7KG85f5MvJ+I8hF zlN(4jxBb17KtQTF))&cB{arH>c<}?gI7t{8s!ns<>G_LogWTF?A}narlOMcb;l~<^ zAo#u)MUKCk3I1WV#dx?brY-3l8$GTmFe3Id0ogm|LBuQSdDX?VN}_VAgiE@=G8La} zFT5iy&*y5!!5!&`i<0aUSliB#nE-(J8Rs)Bi*-6BY=O-qRUJN(5jPW0%|o+cIbpi& z(M5`|kmxp3-=zFP$EtL@O__hxo2xw1i_qMS;kJ`4gdd_}b<`N7jA{C>y&`aST}Axw z%(Y(#{{Pyw;mWiz`TQG(fG#mClYB+amolzE;d9(c$cEei-ng!s%<1z7hU|WDSs25Q zZL1RnMdbtQP}7?IS}?hJp{pVJ0N&0sfy{tk!m#;yc@T1-yR{jX{^0;u#b zo+2Y?ZLOabAg3iNeb1}mxg1n{*sVhnrH3`Je)wt}=(?@bCCw%$2<7}ox_YIlM&GUl zHO%9$ZAaDa>Yv#Jh5Y9%*!F~`T_P2z6lpwN5HWGahoPI^cQcSQyZar9n_gEdAvb={ zH{0!zhamlkUPUnoADDUP`<=!BY4Nr}3ZPbTvb*@xX_ zTM9#+JrPQ<-S>msZY?qndmV+NlUtMVqDK(bL*gu{;zjR-yI#F{Odd_Jy&yCv;J9mq z^ZbM#^mdFluX|Z=m^hWOb@Wmh77AhaNAK-gP4NVUt1F5FDX%qz$b*yo*ciKu?|JWc zm^J>bJ1VLZ1r0Adq;34iJu;~anckTSI@RDeKXj@mB8moBy$+J(ffZ$QC%X+vN>=Z5 z65+>f9tUAF<D0 zs|hmx0G;T87n2uM!8f5rgGNDj^}n)`#%8%=0;5vbG6Ns^D&CEtsuWQ=hPFvv zOaPLD_P$9pc9TN2vY1|&AXOUTd1g{5xJ;i_2E4v&Xo4l27$j<;_PgX3>g~VCe-~$= zK+%uoK&~&+VZee|b0EFCprr@jomqT6`A0EV`86XhP>%pzX|xrV|+*5A%W*21Gc+o&)smq=hAT&a@* zZ)up`_}tA$yI3{`t3Ml1dy@*340gc!_PH_}e%*`|5ZW5GGI6DJ{&E0|jwSMDaqN}3 zO}Pu>m2uZG*#e%H!ppSO!_qvkFX!h2vN4%h^X(*?AeF*ty1|@<0LgK1RTej#Xu+%J zAJ`MvP83yQylmq|=mXo1d|=~|<7g^i6S=d!d7kR63`@d`S9!Gv4?D|YeGAzKdUmJa z?W^bDQMQ2VY@{vi7uwXCNZoQ>bK*Nk)M}?urY<(I30d!#*;XcMxJUb z(ExITWclNB-bL!XWIK;)v%(3a(iysqm!F+OW25ff2sC`%h(xMSS6Au;4}8}DV!y+N z;Qy77&Y8))yMzIg=_jGtHGio8t@uOJyVWokD?p}vh-!phChh3pZeov6!-~oxO>sI z{;IH^m!9TrvHb%ce2PQSLdd3f70i<3_GR5>AYJxZ739) zelS@Kd03gf_vMi?cesxdS|*$hR~Egy!QA^5xnPsfQKoI$7A1CvMM{nRh2Mmh{4$~8 zk8zWE^>XI7=k>=~;zAc}vl530&9nP3;{rYgj8)j^8#Me7J&EfzxyS%kbzZo<%+<9n zSSGD$Ai4VUPW|c+Tk~UW>Ps=+EY{?$PK}VgOyw{1?C^wv4zz;B7JO4 zW;EI^QFaUPlP|bryurGCsw};F@qRg+4vB8%1zuk?tt%AF<^&aoYONV93dB}Xj#&VC zj^ueIYfa*d3v0k;_W2XATL9#pY!&_)yZbrGOcM;lilP@(=F>2IoXu%PyHQ+_pC>+= zFR{Vbne{xM;f3X!K&ViwfTn4Ip7DhcRMD7jKB{JkkRq%myTGlxr)k3lOrS(z;D5cJ z$*)AjPD()+!W`;o`%wWiRd%r+c-2kD$WN4tdYwz-vM5xYNvqaK=bn|Z%W=I7uHOK6 zopphGr@?zuj76X-yB4$>P#iDsYKNy?@9G!4cMK!L+vHfak5FgTf1Nxoiaw8VEmgQw z%o8tpCWbnWD&h3q2xypJLw?po{fxmWbA_N}Z_f7gVISI#8_WlA_rkj}J`tH^o<2^X4;+ymhf#kJrVa!VLoVej{Z2n$h#xyM!m#nSFnS!IIHZWm+hljrJw z{iX1CZRVc}Sd>3U?4)SPdSE%gQNj%>H>hN_hpkSeqS|eBUh*8r{dAvm+w+&kLG0uy1)Kp4~Vt9%>S^>Nb93{peu&mz|7g z7v^McpEuz9SP7G7X^gNy3}zqxL1QVNeF>`%FVn^@IJnia4eu6H)XW+=D4l6F_vnsF ztI3Sto+b6?JoIOF&GwnJq3CcahuW2&gZAa`1lxyG1(H=7C4ozmmR>xo`ths!v+)uO zujVh#J62Rnj0sF??dD0bxCJoWt*^r8qdFVpJ*#&wy~W1T_ez&LkD7mVRkiXZF@62M zM~gF!SNx94fw284h*SrG<5^d|GYzH1nz&S;ePuFw-D~IPcuL1s8_mm)H*&!EG|?hQ z*H(qjD-*Y(#(5Qsxn*_VNbqI6*pD)Qs`%VM1zB8yjy6mZFEdJ+wsY8j4AXCev-3#L zRN^q_XZA9`J!OSp`rg@_vMM&`{AVDsCAI8IlB$kJcP6%fKe9BknayL<>U*O(^EgsA$zyUCmyNSvDnacw_f0zu*jtO`8((pPf+rxfB3DhZJM1;jgp`3w>hf zf@g;7_I6)k7yJGAdWL}BR z(39PN_ff3u-U;$YBk<(#CkrCGZ?pf4TsZu2v|48Vvswv@8Ce@|c>ct+(e>M0H|K@r z>{bT0IryHv6c4syATA&2q-!f0X&88r`Fa9*S~dVDyxcMsE*LCL9G^t%j&~TuvH83U z1I)Wh8Q3g+*56aV6dHe7W0f~U>Ncn@qzpjk%=4mphETZe#e8vq$Kq(? zKaq~T)W4n2w4L+Y(4*tfv{;A}>*=z=yqCSc-t?dBOI9M{Z8x=|(26Trdi6JXb}?O} zf!`xWvVM?9svkA;iX>|J65W3r`k@vrjRCEej_am_WFP9 zy;oRMThuNrT?9d8Z+cNt&`k+VYDlc0((Hhgpn!msU}&L+B1WW%G})90h^RE_EhH2P zozQy^5LzIVKoSD^<9GgZzKe5n%5yHx^PIcPmAS?om)Y=gd~y3cXk zIgaMfi~6DNub6zw_dO$=la;f z1|*z)Pk+>ALlE-j(*J%*6ENI>Vw>>xl3^NN4tGX(nz3lNjx;{nR04!{dt>K@<9YP`i`=jn)Rc2`{ zM@REO(Vjt2#sE7%mj@8Vg43%){b63ys!SoblUtOFb$awaG==sy*ZZ>1Td8$;RoJsi zwF$BK-Gz>BdLRxGH+t`YNM3WB$7l;rl` zUoIb+O*xz)_={*amzg^%7aT2a6Hc+)ym%auHiWy;B7mp5LWhQ#Je0`y-R3mdAql2Om#FzGb+d2n`iIsTRkxs(9)bJred(^sB1L z552(0Qrsk~_l_)LaDRML`KCd-d4b^g`55i{e(l`p+_`LbUHI&~qmr19Xlf5mvJ*Lt zQ1PNyjpD8(N|Wp~j(V#~(3{KG-)T4;@c?9^(|dL7tzX>!vHW$tOs``BJa_T)AYNR8 z|H30&uTTF1|I^iCYpofW;rX7LQ(+d?!Lm6XqBX+fw+(b9Tv)|TQY3j$NvuaPW9*E; z`?V$W*DbpU$m{5KiI9mQSqyS0G&=*w;OF|IJ#sG5xFUwn{)DplO*g+y{KoB3?ql5Kb!oDXNXk_dFlZ>{j+!fPv~EI z7O}eaZjH?j9m;`3^AG49gI3^QN5%l z3jVTX&B=NAo5SJ~aL3ZnZ)wj8tmCsSC!T$tPtmE(i|;Kxy~YgG=}YmS%=8rm{%8@n zg`c~<<|rn4<&M*0T492|1hn+&H8?GKK@1~Rb<~?36U_51rafluHdxq+oE&vUyRPq7 z2=A!@t+FQCb1H<6R=MqP?Yn&BoM%sLb!#=9#$q_<+B_$K%XQai&wk~1m33sGtK;k< zA@JyNua;Dk&-`xWFWOdq*ty>|<>y;H^ktLv{Ba}mUzeU7F@;zEg1OXZ9-I_=izK_% zeo*HFYwt8(>NFMh#&0)${w=QzOdvKa3tVMvZfwSCO_oi#brt361EaU9COn9>4l@XF zq42;u(?Ig8$qn$AVABZJ2Cd$}n}O1SzE`J!z3<*LvRuv*fAn4O#tyP#V`9liIngrAs*Sa~i7gKbRc9x`N`VNn};i+aR9p4>I=QrF53N`K#NEl8)cw^Ayorh4( zQ{T-@9<0I9gozd3W>Xl@A-cc{b^L~QjZMoif>r|A#Qn5AqI8$OaGcL|YE(OXnSWjs z!YaxBaKvvWGjKldVafJnY?4Z-m0o9G&rU(1*na(fMXQTbh`8oQ#SqHo%!1vTa8+tm zH)<#!>L{640%oX{J6u^&CVw;6HvaT1#GwQ>Gqifq$XC&8S=zC5khYa6^@eC!g0pta zpFW84t&m@Td!S6c9}o`RJ%jIV4K|xyLA?>1jfmHuZr9|FdlUu5c4d`X$c^2}%$kJ} zgSSmaGHcqme|pVVp5d12zs}<8<8%D9#UMVuX)ipW^m6{OZ!*755Sy?RxdfVII4aEF z9#c6oshySU1gxBYWyKZ$@m9KfrA?$+ij_a*9pLvj{x~S9#w&zyE$FHegP!j8hkfws z2CzQ2vIsx%_uE+c_76imaCY#O+(OVPN!f#*-B|ZnA5FJ*&=y6@=l)L8b;YHo3z_@u z9Ag=a07%9i@GRf^VVPavuAzNv~94qatFEsnOAItC! ziO8&lFAQkW0&i^1m9uu=bIj9U7@Mx2evUGUa@rwfb14lwRu1Ps7d*Cvb1;WTY{AaFAr4JsX-s8U zRp0krxK3$@$FUG$5Z^4@8utRyV|F~~62$54Hc)`s@cNQ89tYHxUE?I|Ip(~%A+!vj zempK$xg&DFG^Cd%(%Sr2J?~Ud!RFh^ALjdQmQ|2b@q2JC%o+hvFy0!|up$QM^2+ zQt#;7yVp-FHk9G&4?(>eTM}$s^ImQ{9ZMEXzoEe|h-AH;gEg}z*XdK1@vI|?M(k{>@Ig=YLS4-og;{#!N6y3w3Rd`OUD zM2JBQXhtZADa{L-{GCzmPyLu5i8`aa*Abpew_YPw%RW}Rza|slS=|nGQ?4J6Jq!J( zfW?=kQfX1LHDOHGl4kY@#q_^CGxejs@6o&qpF6!>{+p?J)r+7G@S;q^r>YEJhbz~a z;gkRRkzcpeV2C2ziXZ)s?jSrN{2J`%WrvnRO<;=ED{Zy>m&*;K* zs$OmoMa^z@I0aR*m-n=&2{m>W6iJ0~uNYYdUr)u}2~j{%a-mJy@#+OWu0cwNxN!=#uc+L5_362CvrEDHDp88pdLxj~4jL4?w@#lh z<{_uEomuqj{PH`uUen#KV*mX4X;C-j^4eB*3X2jK=hX)Kn~Bg_n4)KdvO8|~Trkvz zU;-;!M<(we$Q6W&ZTf&hp`;oICi#e7FY`}Y23pqK{kIWSTqM2QqNIL&!qzfZMY-uK{vb;8|vvu;1!FjO|B{R?jH#rKYLw*9aM2Tj>m>Si|zZ%z#RZi$gR& zZNHM}6eH$COguOZEFM2xCgT2`D15yj0euF=9L~rf7Q+M+$}kK z=RWnO+LxQ6VQJ0QI@j+t5j?c`<;bTU&P&eTCmwdB&KewrCWO8q*BU>$0w>Z{7Ehk# zR=VNNrh1rhH0(PfJCT%-jhOg-COGxEoNzyeO0RzxdUV|8QreQzczH-AYH0nfR31tB z&fyxir*TZE34uaz5N+3WPF0hr2@`lYWK~S)YMQ# zLG9)TLF7REimRm{l!ngAo%9&ctd48@!Ug;8&nIEn*sizdSGjNfgX;dGPg&7&5+(>g zCI$x6l(?gG6S@19El^&^txxeFqgJt>p7&QMB=D8qzMR5~3qL^xgIi1H&z0WFHLZ55 z7-Y=)cGNonSNa35H(q=5Ryo9o`N=m332hv{b8r519J|xowR*L+Cw@EGQu7(}P|Pl# zJGf2T^|Awl(ZTNQ%@4Xv`&Kw>=TdddGWX?!T!x%VkFng-E z5uFq%KYMA0e9}-mA~OuWJWlv|IrVwfx$AQqx?x=5uAD+|`(=HOD?W+iq>h2eTYMfX z-$k+dKQHqrNrZ{eA5L5~4N&pDy!E9hCeKt@37Z13eIv2UKt>{8*F~fd1ZWbZF*Nq+4 zic`IHP4uI$ zBKG~}gO4k!&KMvxW7_jRDfbqq=A~F-O4YAAs!DZ>g{7v`()e4-S5->u!xjH}a2>Ny zu;{uJv@V9B9I_%ywiFq(wd|J|hRBz3r^bQd_QBo{Y8{4rs|)yV(UGLfJ=vo}r&q+W zztqQZ8*x0+As6KwGc4HoZokTf>i%|!!~FH6Fm92Sg98|6s2~l{nheGb+I7MtOY=Qx8j;ay6+Hb7H&{| z?V#VHqE%-l;3Z?Xi9ltPmtbT4=s%@r|4BDJKuQDJ3QAu%+ms3+Q>)PC$#&=^+f;-pa%Pf&)QZ}8<=9a&d;}n?52fcav ze)uw5HbKX8GbQxB0%*%>59kG$iV+je;9VA67I!;;G+?VM>QKIusj!_bP-8~qwKA2_ zl&K0H(=wFQ%Tbl1(1d$x6U#ALX6*Kk!|VQiCaGETC)=y0bpBR5GdKn-oZ+qNL$C?( z2JXxz_~?j~etrq+=8awU@+e~B1U)c5yauC0h|{yDk{$iye$KC96$C>6g**4E9ssBCu-kuDz2z)LdLFn#0a}53hSiqSovwvgsryDz#nmE zhUae=Rc{NQ{bgsp*0I%SG~I)5u=*354Al6@Sjg2dV*X2ad>6X*QhN%EYxZ%kSt)G4H`UOu zY+VjSYh4q}0#~dj7qyP>+Kwvw=VEcOOkl2jFOV2c9-OxkIr_**Q+cF`%{^$$c4+VJ z;Vid50H&QEyG9e+*f|$!SKd_QH*LOnKeKh{=9sV*RL1jBiA~-4J1`0MF8V0^{4~@#b zMC#)Lr@fF;xcs#0^b1YSQl6lE050taru7W{P02Tfxgis*!}u(eQ|J_>fJW{9<2e&F z=BoPS?i+bmLshBLamFU%!qy&F?V2ow+rhY?lMw#LU9dNiF&A7wUSKa$kf?$eC}mI%Yr0o@0T@`TmXy@=y$nu_H9hT{>Q{^F`+)Q(Zw)ScTOLdmC&HytjdeK1v0 z5v~hgKiBBdBQH#@-+wRr*Mv7b9s9&H8S|qHeFCtqUQ=0@^oIBLJ43Nkoi!_Ae1Cu;3|*l*XG{fCF!DMqJ6@?n2D#BC)xikfYR`st8K>aRyo zI*5rc^$h~3Wh3(zHiRo>_xD8tgrIzT7tN<>sqXK(?ARFwo9!o>eam?hto!9fK5y2d z3Y&vp-b#1+A-H-`jkgGn_wAa+odj*V0ruS8c;)@@F_Y6=uhhbP*W~VcO;SsaUVZ+D z2h;<9mbym$_U!<T4k*tyJu~%G5aE=#Kptwk_j1RFMQq=8(-LH65Snz!d&%a<1}bPjl9h~simJG33H1U8M~jtX*Y+mE zGfwU}K>{k-f|{egv~ns(r!c{R2S-kjN5b$au*0a&)4%Tg#rx8tDWj!W=LC1R*u(1& zZlAqT#`(ALXO-OZ_rgCE-0giKx^o*Q{u$_XKmid$>eeTxCTF+0_xM{(j~>|g{LHd8 z)BoFadeGLQ&Sg283xwQa4j&O7qe_Dea5pZ{SFzsRzAZTEkbX>sjqehUiFsSC2m{z? z@yxB>lygKgnDS`F5XD!vj1T5e23DKd)uuka+V*h7?m?d}KW9g&0@6zVc_%&G{&klr z{Y%JqnWb}*Xv$?8WPhzZ#f+A)yHTB0uGpu@BBKhkQ4#rVb@#QoxkTWS#A0Z)h<8%-8qYZBE6!q|F+7L{GeLKSbCuzL;$* zVnZfF)*c69a+x5{{<h=aZf(ybveQVD1jaEOT=c4*-93ZNT8ev z*B4J^G^)>tpm&ebiCFSdy1YJ3Oak&8-c}E;QM{-aV!h(5`W0yV4;f&h$j{vDFtgEo z-C%47P+Q^H`IsiXXBlKiF7f8#XuGt;C%y0qlJ>(ezE_id)zFtumoQy)AJynTQJz)r z-@AKkPcv885a96uHk(wbZ^!q?=Wu|ZuLa-~{*_e}&#KG=GaMqY0~Ycvi zbZi2pam!QP0oZsa;&gZ=Ei*N59dQu@IyVM^e0Y%ilB+=eKv?*s3~xw>m=T+l9HF9tB>8W`sgUr`GE#O7o9f zmLC9>ZAJsBIR!mvXLmy4jWpguaoGe4 zhSHMln1D&~33^&k7u~)dASB=S2+|83SF)O$3@}Jjw(Cq(fP4Qm0=Ul&>(x})+A7a^ zAksK^_~E+RAGMYJ{l6mmtbLx3a&9C^7rjX2Dy)L{XIuExV5~>Q9qwh(liWJV71;$b z(z+{M`c5TqE1i3HJ2BFnyDQom0t*~={H+c3Jc`OeK*9Y0vT9-F4jxQYckZKmo zao4b&@kuo7wGrJm$<2nGX~{l|IaI7y$PYIgV(Z%~hUoEquhzFHSapb0CZC{qLa14?f`d8K^z$U8a&5}yp*;lkGEm9&@wjH{xBkM@S(c9SG; z>hwQ;!kyBj$?i7W;(^4OZF_rfV7V6wcI-1BsQg@cYCARFR7BYxSEGuV`{-9XnP(1g zbzHZBIT+PlB^V0eVNoIWh}VfjL_zXM12F(+UIcX)#4pWvO6JJM`n z+66YxYTF+{5~Dm8+>Q<9$*{VC(~{3u^pI5u%VQ+tnHHskP0$R#ZCGDM$?ZA`2wrkc z;rkm9NLd%Kt!R!A#K7Y8RQ#{Asp$G2PZ)f8xl=k10Ivh&J7!%U4 zqIo9MOAB3j80nDVoLSg%U~Tl(%*!Pr)9sGkov}{|4PWsRSi7-@WzVuXEMrw2sW8}p zm6dLSRT^ln99j3K=cK6>DQh*ZN~96xU2f$jO&bGS;BHj4@Y%q^BuM6BGLCzTi;9{t z-W9p>R}V65=kpD9{#`ZCQlvH(%mgVM?`iiOedqxFb!b8;NC7n=RM+ zE4k}0Fqha~s426V&lf+>_=GYZegFiy6dFPTcU;Voh7k>sy>(&F9h)cB^F^VqIZvOvKpMtyB=pjtqVU76=<=4y z;_=+;b(}qpO?@|d>e|1gorA$@EL`MDuCszdG{&{dE3&QB_5w$-hK8b%kHJZZ>~ ztwe{%~DP)cEtm$Y%Np>($WoPchBF=W1v*TjeJ zoZQb&^Sr?4Mdr7TKP0n@623!$tIu`$GFdq6k_~J}SX;5%lho7Ft%9)$pS|wzc5Il1@tjBh( zFQFh+^1)}l=3xP>|2Yq@MB1df^qjJ$hz*RgoI`5*y#06P64zFqxw6P78S)pT$MF#j z0Q@YH$cI3Iyj#%e2O;jBpGP3|wBb*Xr!`Nm`*t+RGWISE=HQ<>QL+F zWLZKcbNjM=3it<+u%9;adKu+enf$?qW*ZU>Mo|>Aht)F{ZyB-{0)MNglVXbLyG zq6-5gH&Z`7)WhHs)#|9JLxp!M_bv()WKUh3gau;GRgh&EV^|zT`>9PUCEIY_r6>` zAZ7fb?xKNAi5r`Ap>;ZaaGXp3*N$_uyq9l62{0zUlDWz&->@AiYv#+E(89QFhg(Js zh@J54BI%6VhNSt0gpjQeCQ|dDp2?xbq&251YF40Lm=RT-X^!sLEAILWsi7gIB$3Ww zu8~x3GUeu{4A$Q#cK(i*PY3(6Ox0g&m7UI~zd}YU9{|9`&wE8ydClrGJoX;pO=0Mb zmUhf0uleEJFk_8J*ROWn%x(0^y3Nm#yrEN zV?2Rf+gAR?x@CN#zj{4J%iAguLt&15wEMec$#TzaDR;ZV&HsYd3<1ZQ2 z#c5CM;9G4c5n*g{!K_TjwBrG%+9Z39ekL+<8)B*0=(| zbBUf5dm z2Z9{!R8h{8+1O-gSWk2a)7p=8sv##4{^l1m{OtW8-ow2U_l45{;+*&_?U_v9W~QL1 zMpAC1-(F&1q-9%zn&{rAQR{5*HrVE-gw6U12#*-T*tBB0XU zX;0msax2)BRWmYK%>tHZ&h_E)$bUTxu>T}p@bjL8BR?i&Cr0`CY8VtaAD*1c%N^(I z6GB!1O??qLr6PCx=3_hFf30o^ef%Y6(yJ2PH+g+tEJonI4&H8Y;%bTh^Q@+kdkTuW za-V(Jc;4}g32fe#r^6~L?y*PdGJx`3?I`0IGZ~evLH9RL66sChj_w>xOti8k^p|K& z7l2|;ckWwi@Bnzw>TbM~vwT)N0ytOUA_YNTZabXToz%Tt8*k)^JC2b5k*k2|9OqF| zTPCkrwa1bB?`>`%QNN#^o5&jHG6@ciBW|PtbEUVDMi}>S^O|QjP7U{_Sryn*3f;=3AEdFb<@r zSTp(MI`=Pf{{_R~1my`%NlGVQcb>T$h&oS6DR9#0lRx~N8>N*+Hn$Y?a-R>iT!H#u zGGir!loeybOEh&XNIP_8%$m^}C-s^@;f<%(H)ZSNcO25;juMn}bqYPhnu`a8vD4+2 zWtMv%`@@UVvz;T1X59D<#gtS!_04Be%lSa7Z$Y~7k9_*Pg6^`x8iB>9;{vD?eeb&3 zwog?;)r@$v7q@xTzJjw3lr>$k zo>}HWs?Hm|**re|4pQ3jxVp|5yEM&p)Bxh%IRR`j5nSg)rlmp!D;h&138|qV^Nrbj z$<&bb`FPuqJ15QWhwR|2JbFjv62dY%>LwP*bEf(_U-#Y#=?BblnOb%_4Jg?}^h0u` z+7pXcmgTAQ$;vCCiET3RbFZN`1|mKk$P_2ic1z&!`p>0N>5*gQ5-UNXIlR$G7=5Tw zq?vlEVikNOnh{iceXjMre&Ve`(u+7vHN}c*k(>JE3A$0%3MB$tBS3A^T$7@A)&91z zAqhz@3Kn&zUm2Cp%9TH`{=iem`kgj*!&*_wmPhXd#r)2J)l#SlPMDJ3T7GS!8t!ym zR$h=~hLiRqdoh5a)2(~r;#(g7}HFImP3K40twEjF<{$CtvAjIo=x`l9IQe$Paf`+N!5UpI0% z>u&7eq?wNDd=x2E9~Tkn%47`9vw#tl)p#-ak)HG_g4^e>~+|C$kE?qBVyOJK-kO4k-P6k-|c zr+50>_&8I+V9)-gbU{d&WeD{f2yyB0eu@NiBKpEa8;0xFGb-!x>V;jMUj4=|V4}t) z5>IWpr5D&SdJH+}41(r06c;Q=f#7OwgS(xhScw|y+|pKAh*U47pVK@Xf=ZHqT$fZ} zZopCKMEi%BUGkm9!FV*t$Na;~U6AhkH;Re{vXyR^XtPRgu@2IHdnB09J^p8sLUV-cQ*mz>ct@OVcl= zZb655m4Q0`jP%X@;5Jfx*6{{nat%Wl&g_0lIV%C!QnSAFg2&$NP*!0}PFc%VF_w`=AKH}MQDc{%q9LCPjGO7P^gqmoL^U|&1+yVimBO*i) z(d&-&Ot@$5tI7j9n(yitr@5H%)IBwA^M03D=m{jG+mbI)`sJ-ixxF2srh6@^uE=zg zc>{@{{p5(`iiqee$>6J^uvl+ko2+GHoQ7|5(;NvBKE2P!q5$DUj-<^QdL=VEuYGeT z$kEzwrd|UT?X1^(_kYXir9kUCIv(nQIv>$LUC2ycSN~ODT_Ba@${|7`cYHrOpXH9V zSU4K?p{MfLhTC7f4iFkxG6!gN^wyH6jT-hjja2U+F0m6WJwCfZzp8v>mY&SIxuWSm z{30enO~>QL_4Kj~Hp9rkVxc}BroUuJP6bF)Nc7%zu$5%n1p|dFEuu-y63e2+@(Zg-0N$HuMq^wgLR_9a}rF< z@@hv8(MJEnL z?`w_JPGM6NnVBae~? z20eNzrU(qUn%G5M8YP!s4P58DCliCG#MaRxj}r4vzjK1!4bnZ%Dwh)bbT4p+?jtFK z_t;tFB0Q6%uf;m*Dp8LCQ~THLTnPA4`LnXvWR<~$c2X+ z!&yp6{W~!R)cNPXBK)_IBTNAQh)o&1hUDiXb&a&b0Pj-hF9c4gv)rNBL7v~vNI_S(#jV9f0;6WTU`9Ln z_xBcyQ=^|Pv_^3zDlh`KK6%0FjeS&kmvtHDjQ4pb`zaB4iYU;H6Pi(#4hnv}B7*6H zwua~+w4{@Ew^{{kPx*B~^#oEXdwq*lQX0j~d#-lkV%|zKa5i27l8yx4*~JG=QcE|( zDo@rd1MG*we3=(r{Z}z=W7SBAm2L-nDu32x!WlpyTWv+$u8)%8Gk2bKRtEbSsi?v( zU~RH%0Tz|k>oBTi7s9S)!aj?wzw6UT=! zb0U&}G4$6L9D$!mbg|f{DHMgE{n@O^t)Wz-=@AGy+VAK-598|w7tYD9ydc0O!EdaM z6TI^Uvj1B)q@b_^xa$%hzaOS5pq=Qvv8{)Bg(g;Sv{h5T2Pag$G$LF>jm#}=vxBwEhrG%N2Eu|Q+_S#$|$Wa5g&1?I!;Z2^Y7*IdiwcNDTHE&RC>P3b| z)@r0&#=|U!CxVEiJQ%t$Cl4e(e+QMZS8_D5><}5P96~Mj#yc?UvkXhG1tz$}c1;Ga zY3}zU8F0k_{x{y4U_pFtBbpV78AZ0=5!QxP3@3J>U(e%q43mP3tS@PB*k_ z{VZd{=@XkxUEd2!tQ(;1%c>+V5N!4=sGY}9EJHNg^qe|9=+I(Bu4*nfVi%CrpmE#d*NT*7D3WrKIG?hNy= zTwP<%{8-yPxx+(n&>HAjeZW3zZUaipt@aO`D2jb1r= z_aVEA3%UztM^jk&Mv|EBM>4TbH;3^Bd?@#UTbVHToq;Q(ux&U^>r8<)yg!qlyhJ76 zq0r8Xf1)6^cv95cNbn+5nNou}^;(sljDeuL{nEuF%u#ABA88(~nYLgeyfkcDU`s*> zr={+$P;RnKfc%aVj7lbe6fyiXaEk`8^#uPq`d67cnCETdUOQnRvw#mFH+@yt#;s8ZFzh z_MSR+b;ea(tdzblqty}m`E!VMCrm42rrltqwtxYh0bTszE90Xf(3l0 z`e}p+(D>f)OX9T1sgu4eb}3HV0La?SCfe3iM&NwCfHQAO|K%%e&u&>)PJ!0XOq9Gm ziBa6FLd7GkreA-QbM968Sk!ykKrhcQ!zZ*TLV`bI%{WH^edFS_tDZYPjctSKw)RJ1 z0+U?SJHmb$hmWEk@Pw^ny{L`8Z#ZSJEA1U`moM(B&C7%wd$5>w$d>#OE%KgK`%yUd za?YAz>5YO|kOIl)#W%&YijjTW&Z6j+;WwtU2kUc|q~9syk&>5#1Vo$*Bo(%vYI@Rj ztp+j^AQ?zJF9y{|QL=*?Jke?$}1ChguQ6SS=?HKUe=ZtD`GkSe6aN?K@ZR5${)Az7cE1^ zcwDVup%Okj{E&LI=5g~kx}oKBtwL9bKMr!PWQR?q72xicfu`_?R`j3Lk{T0XbV7SN zIJGRB$g^uHW(8@5?hhI~CvXc6F}|>F1{{*UUz<%Tk7H$=`RCPMN02GFZ`=BD3ig7< zx8yO!N6y`>z{dO9uGGY;y%be+1HVrK!iUi38r6{>*Rm?4Rbgfvl8fJ7$DKfEAwKI} z+T{gv17wKE<@e71DdJXLx;}d55;92CycMx^S=!w*Zh4{z;hvR<7gYzv<}N+{I^w$+ z0X3O-HJDVDYRFn0N7oKm0{;Fo9oexw=7W~$jdji*q(bbQ>kNtnq*QvM;4TuV)ilYu zrlh$6v67d4u!qe7P2w=i^^#yNr>_#RHaglG{mBKY|%+85&=dnN*!r-q3CQZ~ZDHWk4?+ECQ< zv+W4lx36(B)ksTX&MgF=oHq3w;;lSS3o_Xdq#wbfz7QT!BEU*&7L$Ex=nf;ppNNna zb8>Y_vN13M@5WsYGkVqRE_$<8kYG@e6TzdK)$yT?B*!xS+EzQr*gg8zKuQ7OrJs z6^Ua(k-f_XbSS?3Q*=*KZmRUir;#TG-Ro(%lSXmgc;&1%T7}mGVn%OLCe&AW&9dH! z*tKj+g1tdXk=uofy!5qZ+Eb><`oPD_LhH{zgOID9Hc1HWT!xAz`bqhS=h?X=J|m>7 zW!0+)`tE}&N})?s+9c(6-7AiOelHz@V|`IEykYCsB%+|M;YsNldSO8_* z%F;^pRO~r_f^59)_N_y0o?Kq#iOSqqLn=k2CYV?1=p>Xf?IUpgF+*r;ta=wgd+BsI zKoZAJwVqvyZjEO4;t2&30}?Ye|Khj}%Kw2Q6(7(~4z#=9(N6SK8;lAf6DZ zC6wj9YsDW*_ix#6C~dYk;1iAPti4ycl*K-YO~WrmWS|!Zve0W~H#91U`)djIYUantjAn~Hb~o+#J#C&OUv9p5vQk`rS=pENaSBANO|vQpYvO$uH53i;~d zm4x);;XqSq_*!|k6Ju#W*|tMSkd`J#|E;Indt-3d+k5V<6Lvi~?LZ$FFZwMO)Um0f z(;|V|=F-~{G?aQiUP9lH^zgl7KI!aLH6&dO3l?N;DO$)XmCo-JXqJQ=S*DL-w@uQ` zYJR_k8P`3T)0c}3?DZG%nDOnUrFTt{_tu|a<8-)7s>laRO{A`a`KMUH68hZ!|G9=Q zv8_#7!FRwt@z6`_7CMdXm5w{Pp$jys6? zkEWh{zvj^vbBpS?G;Ho(T(j2M<7$0Hd%dtnKBKq=UlELc;Q38kXcO*~zMr`sg-L9S{iApGYNc|_2J=AK$2N(=Cpn}>C#KF{n%{Qt%v+K**w zc77HJ0{*fdHM?o|VU8wZ4JSv2uiy7tfM8KSKJRDlXa0}F|KDk11DZQxxKvQCRK{v*ykewnJK<1bh+>lsZQcy^Sg;3zAi`KF zPldc$s>J+~YDnx`n&|t?aAHUW(cR^p;f6;WAs@L%uX5!pKRVUzM!0N{t_qs&^ z0@@5vRK%tj;{P{6JqkR^)BWe`?mzt~XRUhQE{h@x^Xr~`!BC$1INjX(w>gGUa&_}w zquH_g@lIGGInVW$J$-f$yNd8)*rboWIj7AMZp?ULxSltP-uLM;KgCju)H9UmIpBKJ zUYAvLVDDNf?W7(FOv3XRplU93{WpW&#puYE{1;`2I2i=)&01(ZNCF@hL!cXX$xf@c z>fEd8K;Te8v`a^8#x5)nSTJtkz|725IOdzD7X!bI=fU|rNxcR=Ia_l9E~zCa$6RFw z&1uWjf+qh1wPUTeS&Eo_T&_L=-+@EX0D{X{Yi;cepvRmI#NF#UY9kc$?MQ>A99YEL z>xQ2R0e7zn|KC7+0<8V!4U#6{wQZ+P$)KR8=#a-!@exe;29vN?`_26^Q-^7rN_}H; zM7qNGgUKNC|0C)<*qY3`t`(H12#n%Tq(nu*krDx=Lo5SPY2!Ew3M42XAib9~0TGds z0i;QZii%3_olrv)LXi?$fKWnDAc6FJJa7B{zsrg~Z8!VckGXLN(tVF0 zuW6rJ8Rtj3*V!M?2Jce@@wmKg>;TNF9{E3a$Oo_1`hTTA=jvxNN5*rYy!Yc8g1xz~ zoao5@g9%LibqfXR=Xq)_H=k!cSu*f$mVSW4|NOoc3v%b(_W%EX1MT%6gabjenCx^| zSm$!Nss?QDN^aLq<{JFW7THvg=a2oh`wKk)M$CQYuD&1sA4tlh$FF&3pW()XV`wW2 z<{*9B2W39wa_(nOli|!gaHYU4UG9g`!Tq3-iQ6MA+W#Fg&~JhL0=&v3iyrmS<0NEd zZAC}24s4$;n!mnA*?m3OGyPkAHSg+wuGZ(xfZV5X9hALII<^}>rnrL&Mp)FF95lIjm4av71(h7GUp0)vyBhjCpKzXZ!+KpPjLN1;`+LSSB1b;KsY*MKe+i;O!v>3|FclB zQy#&z4BE=i1p-8)IdC_7g}=V+`^T5KWTR-WQL8}?nH2oo!1aY#fAap%GhQE0 zT3?I$AER5x{r9~mT;kjNuP$Y|f>&TVE7@>KeqZHVL^n6KI z7%V)<{})bSC;R3Bsy3hlj~^}CpBLJy4CK1G!}sB{4Rhb+o|+k~LFAYA8~2fMxk^@7 zSCPuBH{i7P{^#3p0YFM0)a1#*L2G!%I~F!U!xQLL?Y6m7gfc}C;k3bgq=ePX8#~^y z2k?6Nbd1rb!#GsJShq^JbD@KRT$4Z;$o+;A+y7=YL`=<-gKp*?Y{Z!-3kRKOYgJk3(2W9&9t1wIWTh|s-=OeFUkJ=n8;3zEVVy(vj zUyZGO2;9^nre|}w7vkD9-ZG4Vza7N<4#bWokPGz7sAK>=@ z=}qTDHg>EDooWn&=(Z?~GbutU2_fBunG^<20a;ODV?4ZWrN z`DG_n;IAcfcBpX0O`QcyxQLu(Sgd_j-xps#8%>wo!*X{na?|b zcV(OG5_nQ)=ZEFY8^bndKoCu$+K3f-*cJJ(flcXF-Ch7M{F7M1Rg#94#}izQcCiwa z4-z_|gdq(HGbo&@WU*P55ayqf$DT{MkMynjSkfH8^zCbnEc|M4LoOIP06!CdBCWDCuM_7E#Lg%cf_&F5_jD(rOu_!@(vX#zU}vtX1kPiiKRW&s`CD@O z-#BXA{sdw>XREPp#9^zEwYAO5jap;@(3E0x9z>%qGy=EqLshrU!r(X6plfSH4l$ZWT-rlto}~ z*m;{QJ;&Cw8VNYwatvxQMza>}v&0H1*MCDtmzX!_p-L>qi8Ah(GUkTo?EJRj09g`Qn526Jy`<(u%_<2Dx{ zT}d!T*Ak6M?F2q{-sh&(GzA50!{A5HyADQdVUanjN`+3AXno3(VUE8&bem7yA--YOwHu<&}e8Mpg-{zGQ}5 zW6j-;v)tO@k!pIBSI-dqcXraZ1+73^#69D1Mu2bE(&_+au8H{tL_`pBnnNDocD?{P zu|QHIt>4EWT7!v@;B^UFw?Uyb^^BDR8s{lk^LB~nEQi0ZK<6F9^1FH-$xWm;ZhQgw zCt0E|O4lL`>R0nNTP2X)%Z4~kLmGeAsCQm=z;@jB!Txehvr9ltgMe!Wp4}J$d&#Ph z;46k+{8e{r2j5_ZC9=?`nP-S2fAFF9&MvLtKbt5(`Moxt&E$sW04L9xg*0qcJORtj z#6t0&{xMnG&sDYBM$oZW*`s3=7Pq#W&k>yhyaoII*9`J;}I>`#-G}Vn!5o#o;oN_*rB0DpbyI zF~DU1Q6G251$I!~#Wqb5yr3at>k{}98rx%E!07C19gpc0BH-(%4>wh{iTT!TRm5J{tFs8i&MXff2rS5-(&R2Q+-D` zgxBga%^Q;Bhh|z2xfSaST=clP!|jBb_&wg&;rGrrk{!0Exl1s?s8(})YfUrH3lI=Y zZ^^lPFJ{O2){5eO{mO;DZSL&0BISq4#KK`DZX3`nxM5>3GOpO$=;Poc79Xsay1lE4 zYn)iw7R=wxAG{7Gr_PQ!NT3&$q!h=_=8vcw+D?_8%3NHgk5w#8lZ;ol)2kk>8w8G6 zYa|iF?Qz^$GBBjYkhpNP=K2GVW5F#`?-y6l^IKV9eu|1`N9&@G!f9*9C~sU8R@xmu z)89dI*&(~RhNK=6Mk7UjKnr}^9d&?X9 zYwYe?YgGR%aW>%i%-CP-u8Et{{nSEUYK0Q;LYA{toD|#ffCV&7!_!mZ{nipOwb+_l zfScxaV|bME@fu+HfFR9>GuH0|X5R+{zbM26TfNT2K1S62G38FL*>F6`?=6`PiJck9 zGNI3JO2De&Kjs#>-t;C1l)Sx@Va9@I7-z@%gg9S@GniynyVha`;!>1j?1!!P=J4#b zt)Ok#lMTlbb;Sw;sC-0{d@>;>VzM_BakU~(Ai41{~;nSH7<5h9g545$& zZI}4A`;nbWy!nkufq|r-BJIa`Bl~4dMdUbczA(DlmiZ~YG1R?pIaDR3ob^innN)Lf zGlIaGZFp+u;p^9r;)uiiyO!kh>S7QKVI$h_)~>m^{Q=L`L}3TCd6TOpypIy-2_eDz z@upyRUDr1~&f)_G5PMypkmp&v9unJ}i*AjBGVk+ZF~)`pIkd2}2~HpltI)cj)MZo0 zNM({(PJtCM73ny377jXqrs@zJ=t&Zwz+v@l@(v|6stALs-`H;!8z2|2y3&Uc)36%OrPM!BXE zSm`So$Y|@}F2l1!W8O$nY0Il?_%qE6og6B2SB2jh!F3!md|%Pm2c_+}s2?Gq^qY6^ zCrB@NpD;#IE3C;Z*ViPOTZKwT5yNB4~Pop|B`)jO!5gV#5D-_@EVgc={ zMxH2sa=5On!kjOZh9J6EYRWXv<=kFZ_2}s0Z*6Bk6E&LG-~Ix?KI|%?kS%5cjHrF- zinJ-rwBAAa6A-^>^sjnON-}|+Y>cMm?(4lAB87g>mk!lG$Th6Pp9A2Um6$bEd}N*^ zD_QmN;;3xMo*q7W-+W*`r)y3Z@1wUm2^Er{NeU=E#9Q$$HuZxJGu?t(EnC0bX7p|d z8va(CeOpj=+^!0@o7=0Oq|GhEK}^dmo+Q3b^MRMDp!kyG7`$uMf|KU+Grq>GJNMER@9fGPkNqVF&3pqtW zmpcM20eZBt9Z1VU%$(JLXy{he*`*@J)rDv&-g?dJQvr9Z~|&rtrf9qD;Mg*AsBMR&)d zcKg*$*3;zb*aB7$wtTwnSd5nUUDWRyw2$o>d$QI;UzlqmwW(~7Y*ytLU5bGV)Vm?4(&5-Qe$VXSqaT|To zNbq_ZwyB%<4FY~%r+ym2+UC!a#XZCN2>k4RY1EKlvgIkHQjs-9)tYhg9u zY)x~&m-HlsTp&S>>U*9i&U-uAAIfb(3=6o6M*&S6CTrRYm?iH;=NL4vW<}kk#fO^T zU|&D8GJ^`(lOqMrZ4b3>r*9##QT!DWj2aEA?2aB7N6m}_5v%8j^+ECpWdHDg&Cz-J zf}Uszc$otFwi3HH5YEwG@#Yh15bGDvRLG`QZ?eK%L5OzjERDkjG22Y7b-Fkb0t&{u z46^r2SLl=;^6r5kIr`mA1tXTNz=y!uH}z&Z=j-O4wqs&77Uhw@iRn5pr1e7aO8OVe z=EPiav^g^=#2|eYE`WPeoumz)ohr+=!KKd`>(e zy?JA)57D;YDTh**I&02#D1@(#iyIZXJsz;&?bJ9u!FpT5{o#@=yX$lk zNi&Dm$1|Skvhfo91l|Ek!Sd9DwWfAvTjj2|VO9x_DIQ{v$HVxeuxa zj}XOpz-}+$I)V9;ebVo#iQBh?6mCm6Ab%9vJM@{Mx|2B17 z1UKi*w0Jnhr8jYZkIQbp@g~IA3F`_7!biB4E}ts5jmuN;=CxV<&D(1S@8kj4Duzat z2P;og%w!L_mG|uRc3@p-z&3}w^GVL31+}*_{UQQlUjBUBro;1ZT&YuF`RLD(uCc6o z2Q#p2-^RkpR|HhME^6fT_zR!xv(l-pHG~0)(TLSB`>4%wxG%FqRWI;e$j~=0&l2x< z)|_@At=q-tQgMXrRBG2ssDOwSbZQ;cHqZjButL=(AK#_CctYB8>FPM{%SQJi?=(z+g&nw~oLi;A)FXmU*a2oqZu(Y2K zoGUaA6$r45jTJScWb?#e!T@H7T@Dhwec&wXqS!aThFx~!f3tYxadYvxN0?cr_> zL`?j~CimFpDQ~r?ODa>c;Ly7RezY;zfCQM@DxrQ2A0%(Ua9Fpi?JDw_*8|?p%E9dv zG_zY0C(GL}=aut|Fnn5%wf^97v`P1)(2=YSM@DM6!JMmQui2RDMh5Ps`FJnu2b+YL zixS`H55e0GP9C$5g+6=Y^uS29`ro*C$DXYCUb?}LnkI%`*ubn zhg1r;yY=x{u1=j9u5^4@0Aq~8Dhj)*?vABwI5#HuXBO3}=%}c|7otrv2cJs2luo`- z*%n6aM#OpTCjJF(7xD#7xSdOxfe8Q^qYeFJa0ZUO zj(faOa8OAV{_6KTw^;t90aqA93a1{3arZ+QK|7I8f#sA3gyqbESn3COIgf(jF@?4* z?q(Su<+}2BE7Dx6<-MMRS~Zsc35?ed&lH{qa2|kp+M){nnO(y{p8>P*1ZQiVx1sme8uN?Zzpye@ne8Ad zC40ItOMNk!#`s}_d%g;BN-W%KU#wZaAgl-X4T2riNNNTbOC}8eUcjigQGujNw-z3#H(NAE$~mYN=Pe@OYmRJKnfA>h4C z}?{=jNn0@MqsKV}Zqpu0je_&=byaMrDo$e=*t)a$cogD2VZ65B^Xh@tYE zgZY86($hn;LzAg_ro}qxQbfEsTUkle$^Y@c(HI>g$?H=Ti1e+AB(*&G`FQ;tqbM?9 zm9~ZcC(ym*_*M zw5EjXK|^^p6Stpp<1OV}*e!yc5@xT9DLUIaWu*+a+YpyPTO{KT znhM>wfz=bgLYK#Ns^*h=o@Ml_f3+}L>;49t%Q%BS>R6vo7+P8D%C=kw@m>PK@)CV^ zh`H~`Z0ErPW*In_PT+1VC!NF9IAc zSw7DQRZa1$TvD@!Pfs9PGJN&>W&H1SgpA5oEdMDAB|mA>Ct@M(WYa=GPIYo z(Gx$;t}&s02C-qV#`qxGaK>A#WpkOgMXctSjjj&dGR!;NDFbHMt+OIvT5}h*MDTSZ zv(dPFh?+Q`))pU$p^tUd;);dXc;Vf*O>tpAP1@fNW?!!39_zBbNkGILd`ur#)&(y5 z-hbD*Z2Qm(G2)TO^?%)!WJ`;%Y5xyiBe$w;5gLtO^I_`2WiIoILm_kuK~ z4m7ma2gVMK8Tk@#{&X)gDOMeh&goQGC;Aor!x?Jir8J2cm07_i0`#3Vu?L|YpIENZ zN-~fZ+y_o+78vh;Y+gZW=rci)oS^DtxiV3?le)bPvcWuLG zmmVaL{7YLgnpaF(-5;1e2>YyHQ5mjm?MwUCuUAx5?;lDNq;mLcr}Gtg8SN!qfwN~& ztjoBH2+GHYb=gMLHe+_7qA};iir8|;eV@R#OFn@k`pA)^o&hoSWe!=cAk~gS=#yB; zf}7@90ds3g@QLO?;u(wP@(}62&p!cI>gYm}T;H{G-4!cC_@f-&>(Kc0m3_v z4}P-+_?xn5-RrpNK=spa(dHNF4qW7CISoCNtJ_tfD>6Tj3mv%UZef2#m zd%h5Z>SlId6!ZQ%#wb=l(1+v`T?5xzQ!?FB$4+N>uymwpvC4-U7BA?xlv za6u+2%eK}$hZLAxi6LA3g{CPP1o`RZGp>NX=LTDJBVC}Y5AbkMd;J^cxv)5;Ys^&`i5g+dh5r{lY<$~+HBr#MQ#)I z#6Cf=5F6+1>49`9u@>iTc#__yO?MWtYoxz~en^Fx0>uoDS_5_H%_;o@pKWN>AB3A)b^v>#ush< zUxuK5GuA`cH&l36PGo>~49O3?r*~Q0+1eN-@i`U6E+U-rce zGWZuWoMrqHOapV`UW$ITy}UBwNohM|XuePyHfkHUZ^8+Sc@>NpvE-i5tNN(3u2}vC z+}a|?=t?nS!mT$IKx}rbdeKtHTl$?4`meGE{WFEyQch-e+{NW1nwE_z)BwMS*oqhf zlP+b?dE}KwHsSGA#lWa`^n2{Y6;$Y8UKpNO!eWKv{<`s(2;t5VEx*$fefec3!cuB~ z|7|7v_C#@{qrypmCGOAhkK{uq2n%*M7QfND-zeV@y@Qved!%QbJ9(pgEu+kmXetzi zYCIp6e>_#qVD@6FqVEytfFsrL<^6QATxL4DYA#6^>f5tMH1`apz@HzGkcx`=+ZF-} z6pI(5m)@is>ECPITk6SIC>ErC1RkN~-Hf!5DiSI%_0e;y)Gr^rH{h&!l`&L%?!#?i z=@QLDANs5te|x$@t4@qqH~Tj{!j}3K7m<}nEM7r|`M=nf1~`{7ErPw+sg}&QsDAK# z!%{!4(K8WR9YQ@CBk?IO$Fp@ar63$(LFXp$)C%X6?dr+r54GRMCeZJbZ@qK`5{5)< zNGc5HxD;18yc!Wy}eHyUVC zUXqgVA3*6P>O@Y7p<`DTuEKI}UYLsS6gH|PZ|aA-#5rYspox%=jr5zc1AC4(1>W&a z`QCik^#a7nLWxj@O0%vSsqE?+P=uDgIghWD&g*nXKI2}4ywAd0)YvNZM+$Y;a?$~A-Nf@CW&Yl78dIlA}EL_il6-EF>p*Uw=9=CG&z zy09-HW$!qmS##cIaTx@tnYRLTf)%WCOHnj0c zm);%Cw4XJo9CxIry#eK3H8o_(*=-{&>VX8VDM?xkU)jHM-(c<{JNfYm@ROk}^CNEh zh{Uwu7==LippP;00~B7{yXXi4jD)^I*Egmcj~#U`0 zE34$cs=l5Zx4F@?#k+Ff(AK0IUDWh;zRFZPe8>hgWbLgGr=tjck;M77!EVWpl(yY_ z3p|_J9v7^~Rp%=qLl4OHHr=gP^;rxBDD=+7fNMrm9(n8QQ5&DVJ*~g`x}qpCBT_)G zIIae8CmWpZQScLYC2I3;m8nPAoV?Atvct~{$E65)i~+Y6NJzYmBZTQVsI8Hh7}fms znfmc+=2Fs)6$4m? zpy*vv@5!K7{rUrUC$8kQ(S-i;M9<;`r07Kb+l>n7+M{DjciF7#>L!=G#Cv;QczxNP zC=0H;OvFx!@FW}x9DCT%$!>V8Q}k%1ajzBg{6RB>Bey4DD#5$lOYR5n0Gsnmt?l`5_Kxe z$f-fEWmbc_EJ0~^l3XEB>Dsc{!~`u5ZJ!|G*ld(o6irA9S@zfgWFFVw;4IXY%dFb| zTesH|@4Li>0H-A9Rw5s29g((zOS^V%d{^-F--q5&li?A=v@$o0TRy4m@63cgL;LBP zq`+d31S+(QI&b6PFy$VrE{!n3l} z2ht~g#XWVyY$GX;!R{&cLbp}JeQ5y^iCkwHo|>(GI6Fy!a|_cP30^y!7#35^7WN5v z?lni-v)^hG&uQ5ir^V^dbspZ-2)Jln9zKS3O9;}iTm?!0Hb{y5yGnRRrM&Z>2b8=F$%3Fz|0y)yz=P-TXX{ z_Kln(=4rc3B}$vd(2iOrkg2SkkGi)2y^J^gJ}C@xoh4W#@1s7W=gLTPHr2dd4{AKUN3L}eI{%xqp> z$S(oB%A0-O-qzpMNaqRe7C5H_pRik4iWfU_NvP%^qoW?i?U57S6O@fY-fAN=S}?iJo-cO?LgI4l}kg z^QY>j5K4G#o;0FzbOpeBF=cj%?`>kxvhk>>|n zz328EAtX3(={{MSwNg>+fzYkp@5CH?A$4qoG>9SfH}p-z$w;4_E)0?~3%bx=WNeqC zfhm@^E-!Z_i#CgyK%GD1{MHVAK6)r)~T836$~n@{t+b|L%9_kaN}wZY4@k~CdM5jhcT94 zd;=1{Xm?c?Hfk*{(J8xLj6Vj{t$C6!Q?sm0#lMB!i90Ob_Gw5j;Cy$6UtNH_O+rAO z`7?1!y@NtRCMlG={PmYKVmMkeAe%){#_(B%w|@`#)P68Darhp8wsHMNo4ilIuuVhL zQ6oLF=*P*S8j&jR(p3d7?|w7I_4wrj4}tVhPLH8og%IO|^;dlOd)sO4112n^;b|MI z^qo2Tv)2Z^G8iB0DZJ?U&g;@u2<+e=fJT z$A06tO$#T80b}Pd@8EDU>k1sz>NBLuXw{G@sC~eVVbe!@rOH6Bf5Cy2BVTK?`afC1 z*zqutZ|d*GYiYk@H=ojxF_hNNG1G_Du-1iv@k>(jNVL!sF zm(cUC_fpmR!GI{F2wmlpKgSf8oxt&9ZNeJEt2HheX2p|5v;UMJUy>s~G{!2YG|@td zj~klAWD!>OFVwT`mwHTg^?eQ&I#}y^I?T*;WvK5&NxYYam;RYE+w$+j-JL+*mM&$_CcO+|8VTl8g>QHJBo)A#k+`IttGoaKTjmH&|q2p;5_g@x^XRdY)JhP0!(~P_Y zq9VoV<9oQKJ8}6F?KU?pdi~XB*j0V_WxJKqXynH@t={H^0|KkfV9d|1n6cK?RVrsX8b(Zc~K`(9 zb#tE*rlIfFB>jHF4LEf%Q^2*Jar?}5hBLi<#%SJC!Fq*4nUtQ-vue|=TSUHxHlFo5 zW8XNx1LUor#)O>$n|PzoY%bC~p;E%=Z9fbw)C_DFsR<%VS1MofShwG+95{*r;{HA$ zUjN4gL3WrbHkutf?iTd%+(ffppHIuyr=DumHCKz!uHP)LH3s;<{>61nl!kE@#BwTr z4_&cfSC;TIfD^TG=Nx++q0h`=Cd+xf5cIUr*n! zje66(!t=)sBbbSSsmRnf8V>JbKxMkDc-kq~UpMPqc^#nDr$za*{9J6c_>GVbnw6&D zS6Y*u+}Hf;C*hI5OQrw@Y^yQ(q76U?Y**II5G_b1{Rs0?LtcskEkUR0U8hkcQK@cW zCO}W)4%HBqOq!Hw;|J z;lE>+PA)kH%B$QUXxn3tJ^iV3RH%B&SNWb?@yX*ltX?mAVCmc&%uRD~-N2KLNg)bv zOKujDIS5?+Vg&iQl7q$9wTh%DHG>IzgYJ9c`Y5TtMBes`7|>Lt;qPCv>3?lTeii#Y ztZ};IK$3oODqSUJZ{uOHx2KJc{$TrIC~xK5BXE3{NU4&7@GA#B<@!9!F8*DsASpSD z4)KHg$1`H3^kr%9OOOYsuk3uMn(H$IR?Xbeis)DTQ(VZTb8^d{=2jv6<13U`$IuV#F|*SNxsQ}ITub=Nqj z6hSTVPgOlFeb2lUQ-a%Wue1Lt3s_2%=+UY!aOOcCGd?1vt7tdjONF5}N}tPnAoS51 z;~}XD+JGM4hD=SClnO3!ee;yp(Dxp=anZfdieg`THBbI9}o zd)IFLMF2Las?LXwys8t5&-mr4HlIo8&-XR_=YfPuu6Sds-n+9aDuAD3&i(NDx~nfS zw511c_m1?6uM8R+;D6<4nA$;Q#bqgLjy6cSN1zLZ`U{uFy=TmHc3*fTFE8$SP@dS? z3gRT8ii2`ufI9KTLP>iAgDbkW)$%xI@R{f1+Oy(_!sFB+Tbm8pps{V{dR&rthIpOf zE2`E@uki1wxsLPYM~x?L!$v^x&XMqJu|m_f0x>bIyw+SOehM>8LU|ea9?1d|waM-W zp;e)aT|@l3vL08V@b-;BRvkN;p}=YkEX1UMUEulkU+;0eyT?5(z1zQew0r=(mZyY2y*BxgrsOKQ;>djB5vdLS z;J)$6b+bqL*;9js^F*?XF#B4GTRY4sh8e7IBgvcS*E%mzenU3#$StN+o2qGfswG8I(> zeY_*|M!e~e7Pb`bn(de@T7(LUQ^^XlC5h%e%ST3-EXOH{q2ik3PKvjU{K4rfx=kTT zJR2UBw*x+Ja5e|IYB6_6Zpj=9`>lm9H~gJX?VO{4)c7k}2vPpjXl~0K@-$h(!QgsL zg)T7gl!|^|-u%%@RF!gJk-UY!dwzvpQc-1C-OGGGonBFq9H`JfsOgh#o1%KYHELM& zq;J(9DJ9y{vo92Uh#oHi5UBVdkhCy#R=s}bu9;XHhuGU`?!d5=H{txa-?F|Fi~Q^vEeFsj_jERqr3B|L(yc_ zKNcLZ%iE_36F$D(aab0Kqovihn|oo_?Gqr8H|LUa$_%@^3yt5AtuC!TjE$dvzrt$N zo8vmLC?!klh5=}D`}A9H*p;3%ymD5&N9N?B6X}r->{Dm)T71e-F||AECtf(TKB<4& zJv;2Y6)vz6;$*((|I`c#x72Kb-`~Us8KjYkVXw-T6~6ezm((3a1w+z3QU7W9l50&A z+=oHN)ZeU!DvQmx0uxMXJxj1=h_b9A_N|-O=3ZgYT0i!4V}XwgQo0&W6E<{v&x%R^ zHen;*VaGis*YVQ%sf3fAv0yjebthCK(H1k3Iqgs@zd39w6WAE16?0{3-%QwuMFqNS z0Tgq*UY@^QeT-nFj{4}~)d8lsS&kctfkFcp+EE(&|H(tOp!$c^W@QL%q#}1SwadrO z3f<^cu8gjzMmc;!Sr~MAxrme|74(bV5;?7Bc|_;DNW4`V{5b62(Np@udBhtLep%zs zj%*5n!!op?+VGq=A)j01m|pQ|&_ql5;(yyH_npr{01T5D|&pmc%N zp~I^Bk(*&KZ+q$r%H;&e9^rYeQWTZzA;)4w3$ecgT@+YtKrE&$`ap#H*@m1KxmG#m z#-HfY$u@0boJEKTiQ_u>ODDIK@gGNz+s7#q@F z|DvHNl!1XH_>6 zvD1jyT%@^Cf9Iakn!@4{4=1-2Z|hsifUn}`=|`+o!I&z7_LTG94+216rHch{VBz}br@lAd4K&b4?8>bE^n&DLjJ^$y!D9~@m^|R z=;!_?5H*#&_GHI)(-cz6yu{vTwzSO|_a`?VCsSg|OZYP)3ff18` z6W(*L+S%Yf zZyvL{<)J~ojr(*&dTKA24)OEIn{0>y#!pDEH;v`~!yOF84P*PCG9*? zm(j(47vjf}#w)YuS9AI^zL9p5`XBjTa}aX9eU-{Cxb1FKlUx^p5 zudjxu^#o0jB>duU@@-~9?>*m(92cig?!!WvDoXF2zwf)L)YGD@op#f0OM41XURBnE z?}3&czO>(P+Ly4N@IRN+4hr^?6Z7Bt;#Osfv=He>>(!p;n);lC88v?@(Fw@7pVN=L zN-nG3WkAS&>=vgA8Opk~BUdSLC>)v~(6Fb5rJD?f?Tyb9J7eh8C7)nBA8!Nve4580 z$vJ#jvUk=R@Y#5D{N%KY&}Ju=_9ZLEd+f{F6h<6C3Vj*t0ax~|a`NW{)=$i@Q)s&T zw7ly;*mDm(pD)fKm&a-=>1&;UI__kyUe>S2D{gsw?;Eg9p_B=8ooU4U$(Klle-8%q z%WD{7*&6wuY{uIkF^YT~j6RU-wi8txHUUGV`l)6^^iYabXny{$kLi1i_Dkz`{o^%P zkCRM(R~)nH9econzVI^DKso

|IhATqu*)9~Mq9$}Czc40vLbRI{!hIy9JnTf2Vb zQs$S?gPwENfO~R5E@&vs{$&+<{L*0t?VOKS1;Nw8tBArguM2em8_JJDor`AaS9MvM zrcRo?RKtswCia!I&GCBfJL50&AC?ShO5P6iXea9jc67xIueCv$+dHu#dS1^pMr;jzf;=q(_N97tajq~gv5CErp>OyhpX5O z;7^q`@_ne$i|suUt1eh62D&(6K4rH}O&2}p4A<>PhrY+gNAu2U&zLi`y&`PNp5or3 zJs<{SK1&VV%cWzjMJVr$f!7F#8El#ocZX@<^Eg`rQagCgidg8-SqgwHF-D2j@tb3c zoS^tlfX%k&iR_u)S=U`tgP|DkUh?U)q0PQ6gt0F)(3gCg6_DyJ3d@oOC1kDGCMJSd z7}!Jtg#C?>|HX0aBz^r>%U({Wq-&riCNC0>K7 zjpvp&m|nt<|7r7M9a5KD^>2ereM{I!6uxJ_@G^@eo{grAx}F@ z|6_sK@7K-_M2Xug(W_p1*@s~rTO}lxgwAK3{A6(5WU-IDf`vT~Yk zMMaiF&L+9zE{9br%3*SdnmOb&Y!i}dM8tAj<&?slk2^R{&Zo>N$2rX*+ZZ$3e0_iT z{0r~v{d!&3^?W_A$5nk#yTZ4XOcE;woz;N}ZJqTW!K4 zhfuEy-BZtm_VYMo8$T_pSNij!*@&PUF(HjboPO9ET7kKKo>S;**Ha z;(Jzry#-ub&&r7h4<7x`3xJ)^%{T2U*7%&qv1R(M7JgnCo%m(;N$78{ za#tNm}; zF5M2fUN~WpZygL06YHi_%|8}Iye8O)IMXThVlCwB_%-`2d+4a*o%n}Pg{gzDgg!AQ z%PQ*vt_bJHY^%dZ@DG>)o+1mM_0HycatXw(A3jA^?OND@Q5v?O{ng5efwWvum6>bd zj0gJYLtqi_a%(jv77|uOwWq33uZ$U_bN(vAzh#vRdY_c$l#5vuo8MHf zo3k*YQW*rvzslpA+5$8L#fG6F6-lAs;Xh+0IA0U*J?fLd=b55p!AfQ?qC_L*bzxDM zN_iurbXD+knmLkas67BE`&il z7%T?USMCU!>W=*4L|V^IwGc-LUniU+Q!!)<;7~Oso|;=Oj)! zQ+lH-yt8IptB_Dv7AA`U%2G>F!SZL<^@j_&L+z!)9B7Z>+5KZ%tS~ zR~Qd+lFuE1yQgkE=>%U>C{+< zU0?1Y&JT;tfp|l=YuYnY(TDU|qP1DG)H8$U7alqY)&WnnKb9r&knWP*x?ZJC!#qz_ z+dq;vbj+j$R$WR>7UM%pDedK4-Y>bOCC?*WFA7-P+}_J)T|d%`uk;sX*;l>1v0e)} z29`3fY_@=9QmXx?BI=H@z+i7JeA^XAEM;xHO`dA4^5hMhcd0BKU(qA^GE^42?K_-8yU)$m z20oMd%3r@~Nbn+jVZT9XA<{8jCeGaHK}-ffOV^1)$G+#53EVes;{t2jkjJaRYj^U& zdmpx9_qI@2S^6;OIwN9O`F*gEja-O0YNRcXDzQAPa?Q*x-%g@`wEn{jUFvC4r4h6leKA5z|(LJHi3j#AWyqkkR ze=?e-*1bie)!k4YamxvcPGhEG12zPV=E@ba_Mmy!If@@Y3euMw20!{#C=F^vTrYdz zSU|XDKx~I)n!gUVm*L32;6J|f7~YZJ8qUX%+FQy6$%g3vJPcb*V| zbV=XAF?IAdA7xH2i@NFBJ&2(g)RX#WMp-dY#)jCH>B>>jb>t>7z!YY53rG4V5$t%e zBwDC!Cq6UToiVl)C{K0f|fRjV9yj`j$R>@okB<9q3m(g9a8P z#kTo->3beXBFh}H)9YXfLj5M)5NIzB;vftO`Ak!-jYPIt*)RBbdwof_GF1XW5jmb- zc$4>4hvbq+CbF|Jt&pA{6D&11P5|$m*+BaRL>21; zkv2CBo||bmt9lL$srr!ZMYhDndtBNrv}fdb_-Yig`-0T}sr4mDPOo9xJHSR(BCvag zqjQat66$$I+?#NKX@8T45E@};e-^trc<(Bq8bwbc4aDygj%-%@Iv;n8b(m&dOSuMk z#C#f4c@j6!pACH*zI%XeP&Z6wC0A_CYr-6078M*%n4fi4&}*{#U72zN)GX0`_ZZ@x zA7eR@zmGP2A1glGvn`|B*_)*Z`~{%RWWJ1>RB0G+N4XSrDx;n?aGHR;*+d*0QQ;Pf zZAdayAICaXScM0+dF-e-vPQ!$@oWLd^mRIVR<9??zF^?ya(>XrZ5w4xzpZsS*!N|5`q2SKzOIH z#Qk6J@`1L{2#B>ocHsN-hg&>$=iNd*=0WR)1}RC|pl^zp_(6$YRyVF=x<|MF*Vd>* z8j%`JQLXyss+mf0-yCbQ8?P_aOCTq2cWIM+a({sI+QTqrnRc@_kX`Zql5DoD0PWQc zaS8H67B$s)dwD`$X7am*fx03XGdc85fx1tdAA4rQ_^eB+SSVGhXOZ(Ec5hX2tYG=QRI6Sh=7eFQaVRULfj(d&!-t3frZxw$;_s;Dhx_xwkF@xbZ)4}H&H zGW?GjivII{^I0Au)UUDgx(nkVR60Z_CE)78OzT4}uf&Q!=w{02GJfBebaql@vcKsZ zeRTD#7h&YX-8;I!tFHFl<;Hqe9EvVJ{0(*PUyQZL*Z8>qw01sgf$;;Y;y`FnPWpBK zh=_QitQ11V7yRW$@LDypgZ!Kva4#MwKA)F6J?yQYSjPzR{R$LBW_o)zsZ@Sw4-Ocl zj@=}9#yBlE4;2ihwhY)i3g`e+am~h=9Q(TeDo>`maW2U03xTgQ4+nDx%negd(PMgM zk7vaYo#jJwVIIDUZLphneeOo&kH2O1 zEp*xv23jMahRqYFpdieC_b5rCyixk#J2M0I5)N01UKftU?|4i6#c6VBSUf_Ds20_5 zxzn{fgwY4Nao=?d&D-iTQib2Bcc1UcM^01F@Msd~=c=d^_-i>*Aol0xabwz-t)Yi% znx`d|<|qmpX%+J@gOF;co#w$q1wks&!MK$u98Ap1Fh83F*X%WUM;b;_G1mU)dHY-p zlT_E(aqhU}6lZvmOh;S$>w(Z+E32LBol)Srn)1NGWXTMtX6_^eh4 zIdsMLv$iAW3bZNkWb~wqoY7R@V00*(`EL|;Mj^AJclk09t<`!1A;;*izGka+4-rsJ zId9Q*$s9P9`$a@=_Bgck#TIDm4G7ldIau_9kSp)k>!bGfv<1L#c;ti=oT&dWc+en* zcv?s%Ox&WCU|Qq(O9s?}1D0F_#X+avtr^`hgE<`9&dT$VQIt)4JMB?I6?(dN?o(w`f9_&WLiP2J+zeu+<%DXe}kV^l;SF4@s%NM1!zl}lP_HcFG z?|OTcwi;p9ch@KG;i`NF&%`Se?qP*?NYMAnE9G>06z7a7m?aR!9Ut`&^l5K_B3m^d zjsDK~oMjkOO`X>6*@;+Zve&>dCw0ZECRo-q^ z>73hIYaH7cjA44jn_jMZ3YwI=UjJ9b6l;z)HZCXyUwLv)lHx?E($d1=gPe4Nks^4Cc1ALxzvM?Y+M zD|qS8fTDDe{dj}@UT*a@1N{{@SJ_mg8CoP65e*$P`C57cx{^B9845D22#^d5SOQd1 zT-*^$l|}~R7?;m+E)EEj&~At$&khTqCBF)TY~0V49`3U9AiN%d}Snm>ZrX1 zuAAS3J@s4v=N`IU-mv3GcDdlEvYPLPCQdj1m_!`v5&uEq+4`OSurm&>t}Cwsx*(m| zb@?ByHYs?{Pi5{kVgm#^eL72E3UIXzFG}J=Gyo~2PKcnus<{A{X1b5c4Jh3w8&6{t zdkkZ51w<{*00{h=yE3dRVCqSV{MtzTfP=Ihzi(LHQMF9R>-RZR)q&>6ZLM;I-Oke+ zBEWU9@#N^Nq@W{PBOsi-b zHHCv5_ZsJ$147~-68hXhLesVeEs*81s}5XHc-Y0UR>P&g%h<2ZNcD24Pe&2UK(!0qP(sO9Wsh+UN2oz|5k@OoEJ6NeZb|0fpH_w2z~SXfu*sc zD}hppp7s1Fta6B&+Bb-k_ODB$X<7Z3M9+slcNPt}Dkw#K&5wRmH}9$;EFYq6y`2j} zP2L_nE}q4Vr?@nVo&%=4U0FyuEH5pTaI}29W7=!HakGG52JLt%daFwMU*s_rBi8UG z8Ap(QfEf7~$VKeC_f0xN$xd#}wOpJi5{^+@BSMe-?7gS+IFKM%{7X)cuC)Fz8PP8< zRln3^pm*EQ?;2eBG{RQx{f;x=_6HR&7OsJgIRh*Mg)~!}=2K6!@dMcWm~N{WsT`TQ)p9 zg`z@E-Aqy3cx&GuxDQbOB+7j#DY)9Xgf9u z?fw+nqu)3-NmYc7^ADHO21|>S3ahxpi9H79>imX{O;XErfd;Jluykx=^PH#_Z*b^U0sKlws`=Be z9XIOaIWaKQwj5TONt!e9kd2~Yec1+tYGY*;F83ZvDsyM5CK_v1=j+4Wc!Ql{2QhNh zH!>Nrle80Q zwYngoJs<}PSzE^hdX6;6!zG}ob(dxM;mGk{+ZI)xIrQ%ng-3VJ_`JxOT)7Mr#r{_n z*Iuc9q%G~v_#273uie3uA7<=inoH>voypYaqTDwo2qNUZvb*vqJ+^l_(J_280(9y# z%rzPpTy*oHy40Rq-K&JXnH-!RCG@@{9|O+vt+-V(R;VGXy%@gNY_ZT2=sTcXmp-Yt zN<4E>Fg?0)!j)V4B9QxqS+y6gHrZORjYhTX&QWsgDyHTXa!mc0@$sS%wL=LJF>!yw zXP`VRy5_gf@dc*0H{U~kLsTnFo071qE`Qe4L9A5~HF(7^y@QfY(bf(`;86V+!@SVa zG;8Aoi9bJd87c3Hf?iu9(_n`p6Dkdix`zIwxYI)M3LKC#;7yyP2b^?eg3}>Bn+C7X zZx?M(At_EFrB4cj7!g23%%vaDfElmRNeoeI;{DwKq0`4$;4Djrq=Ctdk-F`=HcR5iEJZId>@&h_uzJ2#D`l>1Z0;-d#a zZw9;5WR3ll&<~|#w>Bg~5ca*Lj_<@FmyGSF%;Kv=g(^%fm9gr9vA?ZjV2&5>?8Ci6&2y^TxHY2s zvu~{?fw{ZYkNQ&*D_@Mdk_V7+J{4R%^YQpIkquqr5y<-bldhxUz09t4$hrm|c|Cr= zJ8L;Evr7);l^=A3&pO@)x)@|Ohg|TV@on}Yd`F`yO2lVc{VM`Wx7M3Tn$qNVR4H!Q zX0G`4xNBqGG^0$MOw627rLaDiUF;(J21~Lu+cRBnI1Su&?@m@Tm|pC&C6sj!q61Nq z5O$WAg*EB9RVBOsRr-2jwv(f8#e5}<&0ES$nBuAn8<#-}F-1$yLF;riyAlbx@P+Z{ zN9FxrYW$wgI$Hf6pE{D|9dO8gqJmi4*h=uaGC63m>|m2#_Y;T9adCvQ-UGmAF_Kf; z_d;*l%J*_e=Ie4PxRInHrl}}ZaL|~J;ilAtkT>{^R`n_{}I&GYeRbcU91tBi{FB zbYCCs^6gjeI+VhUtZ%f&G@M_=bh-IdFaK8u$7*{sPNcuNzu z&TM^h-m#}wV76j$P`Xe&3fl}me);-b`&SR{f4i$<_S|cAc*8!E&u5SHha4;1%u$rd zwisqi-9CPqlOsCyc{`wCDf<5PRu5()!EhtD$E14fy5m~Dz{|LX{=DI7%+Zu0#Qhk^ z4uWP8FW$+tCR@!s=rMEODJu8<<=rYY@f~2Vq|2ml*&=f)qah1p|DKkSa-iRmnxF~G z6YCghH3Qe=c7I63ke%B_A%s%m!JB**w%2@2ft21c=o69;n3yP5YR*lZ)!C^O<<3#s z4!tv0<=V;{@<-s#d@Q`Zxy{zHB~sMXmzD`Xe3#!O`c|LNT6)ICH7ZCyFJAi>b~$}$ zZ+owKkJB94ku%0=QUm|fxZh(!)jLBa`bYeT^Uz-XnM%XC=IW!ggk-`RjSVOu72=Kg z$L8k3I;$c4U+R54h1W&?g*Jt2-M1)c83~!rgHQI_V$O7Ork&*>eoYqt`2L^`CoZ;S)e#Fm~b=1TGAQUY*c3Vh4S@QMIj-KD=;S z#w#Gjw)ZJ$i}@uxDqO&7dKUEsETMdMbwgBS)3>*K!PoJ~1D`0W`wb3txP3gtV8p)Q zniv|n&kJbdy^jfW=8acu^iV`7w^!WOdUU8@>Idz~#NLPMH&Xnp_0YZy=lE@{_1vAj zprF2QlW8BsVE5y@UleSM5N1XaE@?BLxspM8CHUUpu_J6zkBZ3;iLmk`c&ZaJqUYdY zT%hhJXj!=yxFaCK)=@-*6ELcc^BI#)BKP@(xk1+#5lQWB+!JZ#;iIYpYT7Lh->1iU z7Rq5_@KeQwZgwgxWAdhGyVq@8hYr#Im$~cbPceAh3P5}#`NyC{za;EuG=!X8eKrQuNF{qb#;)1p*%aaGja`m;PG3-++}gaJ_A-z+vPun3ZhN zeq(#z`|g^|7p-G|Rvvs(BZw#IZ{U~>ykKr?*f;$R$y(2{bD5hV+}+UYhGcDi+V<4Y zU6NH8Y}E#|_1uCS`*58fNH0I##rbB`UCwNW?i@BG{~V?;FU1Ki?JHKSt%!;OArH6* zIlc!GJsuHZid>ujA!gu_pTr&tqmlU)sO@txhB3NStF4%Q{U1Q4+HR?QTrTrF)1T# zIZ2#~=hsm`xro2EH^>OUcZ`|V&-R16XN&or>G7kk$5$;*ArnA3R0G~ie62iulWM>9 zmwoyv$*n%lL*O5kzx#8NP?zDJ_}j)Mc|%fE_W;KGACp`WD?%j^v8r(Lx8S_PRrTe2 zDP^4&l^+6CM$qh6*3{>#>vu>&=WS5)`*#vTn;85>;qh8C)I~LHQwV}SYMUBMxu^fQ zH%57lAB~UGh1mdfx&VM(fB=>IqF}7EsQV9BT!4~8Yp5Ts{Bgnl17yJ z&;_ftS)ySkad5D9uNDII5a+Nj)F6s2wefMwUqOygVS9P!%Cj|KnYjC?00t{Q*w;gB z!G!c*yf5=b;|DmeaDC2HFlWDb!Rwb4DkQ6GGO6;xx)W@Cxk!C`=zm@STyak*LtmmK z+X6%*tH*~AJ3NzsI<%2R6hn=j}iWTgOK{ zywzh^2H_i(M(^ZTjs|SKlZH{L6hhD5PkZALW9n9q^(Ka*&l#)2Cw^$osyyd3#lK;4 z+xO#mzrHIbdih0(z1pD?p%5)4Lla7^c)7;E-)Uzy&#p=;$y|3Lh#iuB+5kDMj})%O zYQ#;pGYokdQC&H{Q;43O*_B@u@`_>H|%u7}<9H;4ab z{@&WSk-XN~JQqi8>Y+TXxVPE=F)};fB4&QMN&;frI-KSWvSf=>K#@QNb; zfHlDu;wv%Q*j{?EV7<2Fo}(Zh%0T?69(|jFZk$=cDk=Gvjst{#&A1-=mBqCa3JlWA z8OAaynH@dI-#135-+C+SfgHGb+Ce9e9?u$3>tW=^Dd_hN|8uuqTg^TDm+MuEjl-Z3 z?o2ad`2c8Q=9PzfLGQ??V@_$_$x{~e%Tg8iQWxkn{BcwWM^T0`X8<~-?ZQa{dsRM3 z=y4oi*b0rmUSb~7EMPVZj1xjd@B6y04SdBI8-|oB`IO#@krH#Bsk!*i)1?*RlNtCv ziZRJ4mecGHIz=#&nA=~dM&P+=nvyF+?v0@x`>#;#f5BRZMh+>LkXQ^{MNT_Ceu>Mg3pf}8(iP{Yg{*YroTzjt5R z{rN{NgBukx`1>H}%wt{Z;-mrL{IUdFlYi{JUYB&Ms0=b*e*-I6)%bEPPwzH)m8`}8 zC)SdzW%YCuKM)YP{*tWMi(l=Z%ce&0b-sj*N*8d&*NWm9f2}u)gkzr;MbTGXZs@y{ zH68~s(&IFs0j`>l!6T*0-0FLrvHZ%n*DMVatC1V&#|_{Xwjf%gZ3-4HD5JnK3$j4_ z)x>ZD^2j=LcX%%j=+gQ6Di2XUF>=dx9kPRDNB0KLZN1OfYdH>mG#ojzNR%=r|MGMl z&E>hVjvEmdXXDJCuibpz-7NDWH!5wbHPXJWORpaxMBn?0^RtqqjeZEfa!N%(GcsR>dVv*Ix{ z{^RZ&@QewnAA&RD1vi0Vu(vk&r{Tcg!p{s{hZJ^ZSYl5Jf@ z2Rh?3F!)nG+OG@>Oxh*{etG#vU?$-M5hQHGJj-MgF4Ok048wpC7}_%E5;C56Rcjg% z+6AKk;RALD7GXGi$lm@uBt2)MyFOsPdHMECQhrg+Xc5T}Ja$V=H$XfqlX;9p^VJ<# z1_Uj@)_on7erXq_%evi^d@lrmJ105VvvVJY&S1f7wuF7}NEGnGc|}xL=|W%_yuG!W zG;{S9Zv>cUD$?TkqAz&%^m@M5&*cwIJduS)G6N1+16K!`Ir9_ zo87JGj?EiFJpe?=_l+nuBwG^p7m{0=RT@X68Wi~q7Tup3SeT8%&%($@pfrE(Z8@)J z0R6A!Qd&we+1>Qz&ZfpC*p-Je=S_+F=mfmgFKm3w=Kdg&r(uz3>wS$`Q>lzqtTJTp zs}3|563X%jpQ_}GLN=u8TWW##FXTf{CS^mJsh>*QI-epqxGJS|r}-Iuk=b>%-Sg(} zfV^m82Pg~ucG2nLe>!EON{9FBW0lNz_Dmbz>0qJ-057e8E$(fM%vSOSt+ zm{}JaH9YpmMT%G{G?B9-mK7||UVh#-b`YdK25nr`%hdO{##_tqK+R-Y8Vi zh)%NNLLh)+jPi&7G)t^t!?mN{qL?OT$m&S~EMu>oe5XM{=+?X2axJ1~L~o~`^G#a~ zco>n>Tz4~r)~`PDG+D?@NcU4HdB8OL?$+NoG~i<&7OPa7+fCclwP%mZ9~wOLmsNn& zak=GZr;f)yG$LLZ>(iczEFU^3>&OVu9Q-Pr+1U0u>h+rT!^G(7=%f18?~cE59+H_O zcpcHlOz09|sJ==j^9{V}jO&Z0AzGY#?{xi~piA{p%R&LQ;LPEprpmpS4udx8tl zD=^?X-gw!5Oe>9tcL1kbFrSqtIc&Aa$fauEZ>^428OMYHbXyYrn$CNaP;7Ch7sr|~ zCMXHx3aLQKA%1Qg8o47TXZ-lT>7*=AdO;lVb8h;tK_yYg77r2&R!TXjGe1-HO~cF+ z5W3YW%BGlXDuwth%p@Ug>eN5T`{a7jg)`Ua5tJHiA+Nq_u+8kCK4LTI(h)TRSf~w8 z@xiPP_09{?zN!DTuoalB+&n$y*!K`py-x+DnCX(=5f_dJ&c=GltT;Wjq$r~dbKoy9 zM6P8JwyD0C-Pn8WFfXx|+0}iIIf;P%E}?2zFS}0U^`uZ9^ShOBr9Xx9UC06=M|@=S z3W{P^<1b~13G>epgg6yqdaZoIxlZZ|6pX({YgHxT8m6|ZHDV!QK~9$%?BY;pZ)*zx zxv!`hZl)LE_c*^v==@22!<}V0rF4nzC)0H2OcR|Kj(?u5t+-EgZ*uF<{FA^$cgz2j z)1?ENK&Y}?k8m+lAU!#U>}cAoFmZFu(b&#r)U&MJ6~0d9jPym5kVz!YqrEfO>79>x zdIrE(+aI(h>VJRf9*C1$JahOpNRE?Y3}-~mB=g%}um4>h*Tff2t2Bifgz7AOHDAG( zkC+3~|Dn@w=+xBECv+a8J8YL`IEiN5+wh9F>Hc2qp8w2YceF?Qq9Nh1F!+00-caC| z1|<{5=6&3HBs10A@hZI6Ok(_3w9UJCtB3rTR{bezG0eGJO zPN2Uu=$`8-`cbDKG)0r6?%DxvH^%Bv4$@*tj-->x>RE?%OtwB`7>%KgaTgyFL zzIJo1J`gG0*@tMDs=L$hFnx$YmGETWMU6)vV$0JGjK^LG>-+U-T_u}0lc2CS+p4@3 zAA14acsK>l!l9NnudS2Y-2uB{!E8*!a?TGJ`QqUF1%Hu{%_F_P`=oh1mAD4gY0VFtaYJpEzSlyjz7c*DPwiw#lyrR;^_`o^W@k8Rs&~U7%@DGRg zeG1-|(h5c_r_1^rYU=L!kPiPuv%Yq{>(gIhcMeLr3B#?EmyrxzxlDq+t@J(Dh(}s; z8E%)?UlUAi$RDdhOwCQ4U9DrBq$6kv3D$0Pi0N{bxm1EtVtdvfOCVC;9n1Q#lQp+CA8^bMf+6Mf?JP~a6;*W+U6I6TUaaGHkW{vc_z!#RzIOPzxCvq z-79w>@=gvT=@UCvx0ShJLCqRF`9_VoL-b8;JkhJK@%UrE}^F9(Y==Bx=-g#r) zDf1^~<1~2e^`w`AX}y~SxHSs;dB~Vy;Zp!;AIe`EI#Z|JF$l@n=y!^|`u0aNmbe*2r%&w! z?-7As{qn0Plaxw^CT>)p8F@n^Tsa9fd!O=hGIDIXfF8DYrVyz|(6Xgb+`@&sl z>Co0_zvi8@pL3|Cn=CE!>i#$iK*8Zas|tEl%5wQcSLjg*+#O2@y=SolpT*f;(O4TB zd-{xJ8+1D^ezWReK!*jyPwfubV$U0x|Wo8(wPA!>RbvkP(q7(4Id|f=ey>dazdSz;PO|;gb0pK=@*G!ok z_BlBoz+c_<*~*I<`CM&OwJhy3S{z5j8BDuN)K^5YtNCCVhVz)`R5c`wmg-OzyMbU8 zq29#hGqMKYK5VL5Ktn#T)Xic|H7k8e)+=&*`rw`-jcnwqzr9^UFAQg73pG_b8X2q@ zcyhL9=cFvuC=%y@7^Z&(70bx~q?I3aDDcShXU7j{FR)=xN{g;=nOzXn^w z!bjsIJ_$J^4SByw7My_q{WZN#-!;>uiGzCsSsVHKF>lCsDmZS6P4HNL!cywz#b!Kav*$I(5 zUw7vvB(R?J+$Vv=`K5g?%ziXxk-Mk}IQCQ0U@Twvq)tcfwEVi%PD5I4>F+c>Vy=AX znRrGC>8opK-Bp_T^VtWe6&2jwYSOH7L8o(nOZb}T!!$_lWFx07#EKDs=Y0qW39g|& z@!cGg&Z_c&teX#6%$x@T*OnY&P+rTRxSdRBtQ9q5>prWzcC79I88I6EARZU=)42U` zInELU-u5opoFU#GWiQ`2SX0vn_*t!{gYz_IH5gGc#Alf+Ar~}jG?jU+WQzl>H)sIx zOiJW_)un%-Mk`cT4c?dB)6@2AW53;PBnF&WV-1p*sJfqvgAGbcv;wM9rX@&zFYCD5 zQ7s#h8QbbwyaY(@{HTIcm&KfpwYMPlsK_pxNQKh>CbE{vyrzCUTb+<$-e4jLa(BJt zNQ4yZ!ybemsGm=v$HZ7fIJ-@sE9OKRUPav;1+ahKF0P%@KlDD-<1tmY=aKYc#?Gp6 zy4TnzGYfAT$Z_dq4Dd$M&ZJ_VLy1WY>6#U5@@w#;H%K^iS0!wT0+y(+J2h1^?W6tU z2RT5yoTMSIweU%_ck9{z=P&0XQ^4-YQJXZ!4&vfqOl)ia(;kU6P2L>z;?J6aQ>EXA zQv>r3y()7$;Prm)zS&JhA6Df__Y*U34xiFFjiUHwD&QQ_Ikmqmo)lY^r4wU@JC$!~H zt9}lB_$vE+8Huk}brQ|}^y%+S(he()ENK}bcwiIJ2@IL7mnw|{2 zBFlW$ypG78vDk5**gwZ%AB96V8*`QaP?I6s+L!ukYHjQDFycGL{NBAziLC`??a$R0 zEqtP`WR9MnH@`Z+0XjM`lU*=QlJa9-u=zckZcMAJ8Y~^3=B0IGY{I*=133e`MKR*q zpV}SK^d}MNK8at74{8x?3#v+fcHq5q=)v{~cgoafFNMlR7uC;XX#3k?wR5j_Z+kP4 z7bu&W)f3D)^U@jjY$(Qkkfj9K&t89!=h&U?8w)-fDpUxwvQcC(@YHy{BIyw*HTAx+(LlySm4{OB_sY1r zK46DQZOzSto2Vg7Eb`oX0cyIxl`u1riGpt>xDzdP`2B|;5Nw|kT0aJ>!ORS~4jx-? zZDYJRiI>=YjR#@a`^{v!j|+`E>q`1&d>R+$@A8Y3SlTW8(&gdv!c*IDg|?_(N3lMT zY}Z9|k67lAtjrIWM`D-|CR`Il;9h*ZzDRL%Jr&|IWwN}-CG*#f-wU-@g-80(d>N|+ zv1IZ}IsUWHSM|vs63S_tIz5ApQ=2cN7v(bL#A$Tgao?XaI&AUbs9SO=`izHdO3;Ol zy=9d8!0-$Gk-Tgn5cx2S_ZlnDf^JNkPW2M@p8A8!ZCjK8It+G0funtXgU`BYY$Z9G zTF1+VF_16GZNeu-R?$BhCvpEyndvxo!)LoeF9v1sEb%;^v*m83&PRUxeQ%U7eybbE zm|mUlquTlwrT0C8-b@z@U4n4)13&3*eX^008djnw#tpEP#dk5b%uj`gkCf(vDd2NT zLo*&=DjK${kHLHkgKjX9~q=iD+f1 z=i7|SF+L72VVRQFfMeYL|KN7S7vv+s5l29^5jxcYpoQ#^Jd!gBVw06$sZOs^`(7(2AH)xk!88vcaU(SI|1#~?`>WjS?!Ty=a6$x+3uGcfE^ZWNTCnZhg zYx!$ofX8_F59JuEqM<_kJ4s!x2bRQ8d)it_+UplGAf(_8?Ty6M*9r!%k#nZbnm6`p zkxTE!o>l1;6-Ho=!dqju4i8$S_0ooRPa>H%M2lgUoRL|m;7-+oA$hZbnjh%R`}s7V zAF>jL+SYmh~|LZ?K){?RfrrZN`|+AkUS=h4g1j9r_zdwg z5sCM!_J2jmnUh&DoU0PN|8uF^8W6igH$3Q62kuubG?LsOQFvzs^CPj@pEp+I5SxFV ziw8Q1r+^1ovtNmkf_JL82EU%3ZR)GI4dwu7r-0V{T$TPsh5R%jpN-o z7R%vwHW zg)Y zwMD9uD~vO?`QS$_=17qFqTNG4{Ae>9$z}i`+rRxj zVm9r!CYkTT9^3gBT*qzQN=Ka99fy2PqAFK`Q#o0_FjtTtfA;e4xyI_Al89rrIIOft z;a&DEe_xa&e;G%JPKMZqdYE$T)j)CDcH9MQ$AB|`UzqSENR%xWSeSl8fVTAt<^sbU?%1HBM0yr83?mp{G_O?*X>>|!#i{_x0ljD zdBV4`q%O|Zm)bg3bg+_FB>PX%DBI`@qEZ>LZUbKoLq-0E-`-!s=@F76f%Ay=q@nq1?#P*^@r01R-$$h2EUZr%Nz2iZww=39 zb~BS9d;N!t_AtLBv!-XavAyyjV5p6D>A120nLPp7fmAp%X78q6Y5X3)$7+9C!s<_c z4MoALc}>yfC=e>8Z6IPJlbs9|QWA0~Y&tzfwWUJTzI)nB7)OOLrZqz|ScA3`9S@7W z*(%UF{VRMM@M!;kNu^f_MaS8gYU;z~w0K(PZ)8^19K(cF7Th`WeIrLQgN3nis^c_b zm58@ze#S%za_rNa>_e|YO6F~WZx>UG3FY3M7%U+`Z*cQE4ob@v$7e-@CaUi0rZ*iA z-=H6=Mj~8Vo_u2`8!awl*TZ&_g>BCQp(6e#h8JgtN{Llr@#}PD4!zB`Q^YWR*8VIm z;CpvNN1LYnf>Ml^+B(z9f2t$CTKf@a`7e2=4V_QrDzx`}Y;~6BfTq(5cG)^?kJqvDahAYD~K$Ga^{$Rmhf8iaN(=XU@3-4@gEvC|NZ0oZ)2kQ19ZQpcGy)tw)TwofbWNhALsU2h(ibk_cl zKTW06m@$ndl?u;1PBG>}nd2_ZOp8;b%#0-&DJ_~h?mH^WOlFR`FojBs!j!3*3#I0Y z5Sp4eYGCdwC@Q!CLINUuc|PCoXXf|M@4EQseQ~b$Ip;q2Iq&5%igr$dBO9QB7bkl{mSx_R?em%yScD5VM%rB`xFp1~zO6-#GAjtyzOi z@moB6;kZP7TFF1hr#V<#UT!qP{P)o5i?THM3)m4D^Vv<=Fp*OkFWjUbQPPz|c(E0;8E&DeYTXX?}Jiha4p-)IJcrvZYd`fdm@0^)>)d9Pt< zxTh{FZ<3N5Zn}z4l-fO8;=KS@1B{DUbZ=tOdKT02)>@6SdK8TsmQaG3Za&VcsI9!U z4{PLH_Q>7fx=}J<=CIGC;BZ4qy z2S{|gfDvfpz6B#!GvtUf5s89lYwAeo4I3s7sXk3(*cuvbwAQKYi>BMB)q`rlf>*mWns=V%TbxwxnPgtw$bbNGMdlbF~N4EG2_{F037ytbe(5B*YdNm{e zVYVPM_bL}L0G;?b;uUsg?kgXE#mwta!dZyMW3c!vDe!Ds1YT2|6Wquiu=11sQGoPS z>rW#gR2A3>A)clO_-l~r_?;L=d_Z$Ay>Ny*`!GPp=AUjjM)(Xw-{379(QAy{2)1h| z&Sn}E8?DDgq=#kudGx@g*P>i6U{W}JPBT3xu<_zsY=5xFYh?VW0X$`ESm^`D=c`}* zBwy}2VZoj?q%t5L`f9ekZzix&hoI&3h|QfE?ICkq!naw$&3Q*GuY?jlGjN)`)e+Tm zl_R}y=CO@1n!5N_o@qw%q5%BH8}c`HQM3>M)|&3_dS3CKXwC0n%@-s{5tx*}%GU%@#)_-$Vs&0rl2qXJs z2Wg&Nge0UkYURahl>>UlhA3RdxKV(bl6y${fN#oSYl7&q%ak;)8h)#tbR;>%2~0;> zO<-?tKLd$x`Ydr#B)HejXOr*ZMp zlzhkX(lGb;hKMrkEyL7~B);m!J{s;508|*5W|eU=Ko!k8MM!~EI8ZrZQHfD_2h4wf zy{Q1dZuFH{nde7JTk6wHLPB0}S6dtTh$XeGcg6^<5!Ra7Ax=iR-X?ZliI&F5kZUs? zKRP(!;hohaLMm(wR$(Cq$Q0K?Rb%Ix9vF`2RL_^KP^7u2ts}{926oePlkx_jC2NTZ z1vLQnvD_d#dN7NoN5CZ6pkWvEfrGm{|3U+!fffP(z8KfW>!)APq;#!gbWX(e3^O_G zUfJ?`0qmf)0%2eli^R!{sS6X7fUMeNhtcg$8u_@q5;HMpPV!_+UIW3dj7QQ5L+1ken-}=3k|AT1eUf1l0 z?Yagk_I9n~&a~NbMraCr^ffjZ@7nIC)lm0XuRoh^&fmUDpQ)tb>_#r192?tlUJbHkG)!rg?s{9(Q}1Y!^KK ztf9{N?gnQ}$!>5Bjg^eJ!}1Xg#)GpJb_7{;n_Ge`=}c!)gSy%kBY*|nPpZcUiNv2r zFn^$7cR@vaP25|(8PZR9km-#jz>hiP7tRGq`rr23uPtBJW|)#MJ9GvS)MI6ytPKX zLM&ri`viFSQ5U5nGTw7EyfnBx1?PQ8vi@7fFiZbM8i#8pAK*UbQ5}LGUIA$0e@IN3 zCk{u^h;vF>~|UA$c7H% zHLIMcghlefe({?Mqxz4c>fVAW{(-tmG&~Qm0~wJPkt>&#!_*Qm?1qBI~Gb6T_Qxfh-zF}f~2 z$_}*iVhnl5dXK&K@EFo6f*Yk%%^}&-(p_|iLP)bXQjG#$9)`f7v_(V3&In6p17&JdU=}Z-N(p`_EnekF71}mF+yYzNi);sMY zK5_itRHvc0Gd_%OI>WOgVWY}?3~VMRONOv!1vNr?oFnQsc<57nK2v&PU8bpD7rYB5YtTqsJ8DY%(thy} z^Mb!{vD-D`S`xNIw9n#^KJs;?MwTy|FU|^9AaSOiT8~0(%gc4*f5Rn|>&qUOH4VMe z%b4dFo!x4E9JZ`iR4*YDT?>w!RGkIa+($9@5s)uQr4^Qs;*lifv$EJ`PFj?|5*gYU zsOiyh^gOA$MYtBJJqNxNU=1P2jXj%Jl^^@BMmu31 zE~l4`4?!Rg^6Zc)X=da2*R%StxOSa2H$0bTH?7|t-JHI`VuouYC}OWatfzX+3X4@< zpxBAqwxd%icADsd{_+jkr$mwH;8tb}_Ho8q1<9<7E=RknmaTVMI$4;>?0=83yml+R_4Q3dK{yjA+zPq~g7`KbjL#S^1TjJA87OV2Lu zA>#Gqz3whaM&!%Scccr6++VjhjO=iBHYlQT91mDj+wY*7B_E#gWkEP31xX)pNg z<~o**ae?H@@YMjOqk%X}`8ztM@kWOXGa;~8%`Imak2e~QCl!d_*E zl$a?O-GyPgJ;B+nkX7|e9g*sXIMN%Jmh1&%ke_sWzFr;Jv zZ34PK14YSt+Rp-1AshLDSquPthLqroFJ3>Z@v5YMjGy)Nc(t?T)F6& z6L~Edh8LVXqt%Z=u~=(pcd){VA?Df=YRCscSbJd(KBU;NJt^U9%x4e+!Z32e%QHbZ zm^_4(er+7wwc0_~w#v^&rsUa?UVld|sxWyFd(1Vzt4y_hQXBwFX|xVgy|+GS4Rw68jIphP~m@qz)%b{x&;>Lapn(FbVwI`90iajiPDT zXb!w%LiPs|c2LMxe2IJKmyBQ$O4il?3P_nLGA@e<^lgA;B_&vISb>?%J_eT*B~ok$ z7`OE*xvrzuy|n=pK|p{m)lirJLUF&jV{3PCunS5ZUC(=(>5X@|ie$PbR~m~8j|JrSkdadHY*Tb{0w_6!DM;Y4AZvMP8@xYep#Uv^O9$@t zeL~2B>=eClN@Z&v+8>H=r0(w5R=@G)7w7T ztqZKIGQwz$x~wOG3so9HZFy{BnGuLhRaU`$*Ysa!E#RZC`V$iO2B66)7oxq zy;RJE1uH9xH3if6@k9J7V+!4!U5Aap$w_#9Y!s8PB)j~T&al@vM7O6EufR2ZLm-=vUIXv&hfRi

I>tY96s3hvfyn|3)ig12?Q_IpP9CHFic% z>kaOQOk9Q71ymh>N~y+MBP{|3!zzSctZ+cmHAL2$3?`bn#Xj3907Bx+2@%fg(Nu|` zR@MX6a;iZ#n6ILw)IdAiM;%>ZJ>s;Y=mb42%^s)L^#O`Lu1Yz8zgw>UY)aT}?G+hR z9;|8~Rh?DkWtNU$^e?%Hc>#kB8ltLcr6}k8cH^kf4?%Q|n3~HiYOI(g=X6gt$NMCS z_iB+3k6<2Ymmn&Ir*r(6d*Sls6s&5c&hjB2L0QPuUFdXEhbOn8M+8N#U%~wat#VnI zk3$LH*E^tGEuNzVvi(OPgcMJ3B*VK=W_^i);72^3ZBjGRGZL4`Mg{#t0O26J%H03u z>#!xNEZT#6rxxF|y)e4w_(b>*P{~!*kw=3rP5dZ}OA0oE{5Uu8Ff@SMW%-DQ)yFJm zo(CheV1yfisLfZqeJIS) zvegd&|6?MwdpqKeEHwtaTH5^s%V(nl&+(+&6$$nt6NK%48fB!vvUjKSQZkplbK?|{ z;Eav($MG&6^mesbP1@{9%R7rTuqr!HC_C6t-hAr>tVEhYW6MHj=JF&Z!iLp+O*FZ9 zx0C)W(`^TijIFU6i{AqZP;lvtD#V@??OD(AW-)?$C+$c;H}M?TZpu$jaGOsts8pZO zBPXA_`J_NAv7g@2XThPG3q>X?nLY=R$P_3;GxoWShBtWPl3og(mQB*s{7!2S*N))K z&Z9}#yzdU}(b&$C0kf*3Ii5Ic%2NdEDp=TRKYOoG*n*YKeHFG;;7jPqYDmVt^JX^p zLRTxAAsRgvEg7CqTfPm{DNZ-$?5!BC$UNZ9@Fl#&zCBSWwuYDVm=T&wgM;eWf?5rQ zq>lfNOyE;G7fO@Jss;1y<4~T@l_+y-+Ln{6L+5yhVFl)0@rI4AL5x%{Myk!g0T>(+ zvv0P_}2jYe=Vr{|BobIo8 zk|Gl>PlS*yPlL6x7m-0-y0<-dgqwMu4H`$5JF;l@wIqa$FQ~P;tU(?Wo0@tto|a$1 zhbojiQuT)|J-^A7A%hlfP~$V5l7$0cQg4*YmMpj~`i<`zr@{p1EQRKf>fD>>(hW_$ z6Zp219ENIRR-j(gZE!np#&kLfojUuz8BxC+nIP00XGF)NDcOPgz)6py<3I=wa7itIYsg3igtW*WI7^T|^+>+6@o*nqslNu5yC%GVJ;K#ujk5YP z!F!YyI0s}XuDI%o`jq};^IZ!m87#Jf8V4umPd7)W=~op_m5342Jt!&@ZD>l8KPN;~ zL?)*<$hjAiN~Km8!Fxs?c5S}>jq;aQv2Bi{AK#Y%O=o>PgK19LY2A^k(9?9GSt++o zXxJap(tZ3HFGv{0j1a-MA-IRJusbjlwcZCYNNhf)MEbSAg%ifZLtwG|))L|G+GE&L zjom@Nvc-$^R|R&ZN~O3q^rHrusLol*jZne(w!LVT@@?@xV|2YaE&Vh-a(I}30pkKH znLz^yRt)*CKuzWmD*b1NaoFn^#)k;JgeU)mU{DT&Vo1_ z1{#cdG!cEYeb*7)4#1hI^0)-<(JtiW0FN;fFpgG*!OD49X8PvckMdC&zGJPIC zL!EH_qE)k4@KX`iLugao%;8)0&=|KmYSns=>2vR`G6!Wve1Z|gBYz;jau7NJ@gJzz zr^f+RMM|@2@j_Z%7DQjjy9<7qm@MgK{`l>CxOSE(Yo7)^!-;4#ZL|%27}c=O zNlQ2}Ls-%(K&)|Hq^@Oa?IUZWGn4j0n+s3UG|l!~5D71&LJ#KudstPjPdQ6{2B)P9 z|E~e^8z!S`i%zT-?mI@^yH)`-tS_I2l z=N)o=rdGQCI{trpSLy{2)^)F+_s{%-yGZnsmj4Rke}+J}@Pr|Ve|JH%fHMnL|DHWF z(OhM-_4n{_i5lhQX3Us!)}F05oB*rox9G;%{X8A-6ER{ddWB`8E_%OqTU#>Q+0e%N zyN{@Es5q{%Pu1k#oqBdvYN$Jvse9 ztrxMVLyviyh3vQx-oG4sPcJ-w@R6B6KZ( zwi=D~=Ylbaf8LI9%f%J^qxUd30?u~wpS4_ZmihgE6nj}IeI5Ugn!cWW7c3%%&XRg| zuyhshMtv<-v(GVz4TRBisG{q&&U-QGd;M}#a?Pec)BPbPo&NUscO9WsMk59*#$#3ad#6gn zEGGhwR5(~Xqbi}enD$g?M{vRqFxO}Wh~dFMqO&PGsVLKkkxD;r5Z0mNjIe4Pr?%@4 z5xh!3s^rjKXay=hI_1{cVwa%O9sxDkxqB*Jh`Wuflfrx0{qhe7EmD06A5T{ldS{_p z9biVA=Ly0>d0;p`8T%N6AGD%-@o#3y=bkVW{2c(eUodeOOHSMBrfHq$pRD|Zq)B*H zY9&=)fdXWdr7dMi6^f3iY2bAQQY-H#Wa~~;FBt}-cS5TOo8HHqk@zqF`%UTXb|W}# zt#tLS{cN@`W8CTP=zN-HZ=VaDcH9CPuXp*CJD~n9+=4qMwn`IbO02sG+#5Wwe0cpV zExpLKJd3L7Fb0JyJeKl+1`V>*#11FT+}WP4lP-*_9Haz=<95tudV64zu~X;B>GS*X z&J|J0D5)d;3 za5u-}IC{6f1ie({*F^MR6sM}bcNyvLxZ^{!rKx{u34cOVN%X!};?#jp7wU9#KjD=> z{hITLxBMLA{n@<+w9lQb#3I8>|zn}a=0k(we+;)0*p%8 z1~@{41;1FG%dhM&?Z%!aj_o6?#iKC+&B+Yz>BnF;kgkq*>0x6^u!n;~g{q`$)~Vyy zlG~B2bc&wB5GmPMY$H=$FH$B>7_%HS4ZIWef`?R*E)OR4khb|@Uk>^(?XNgLF2E>d zer~2t8lgY4xdWpwd*s{LbCMY=t*KL=jAxfzWta2>Cm5}rCqInjer4(ZhJPeXs<*7` z_%^*mdY8T#$BUB=Jc(BwS+X)lrf`19!1`jEWk?tYP3kHTu21up5l+ATZ!SRTY-XzX zb1$VRUTs7Z3!0j{q4W;q{7@Qu*OA^m39_q$5VZeG`M-<;{9n%GQ-Tw?f4qu)fkC%j z1lo;=so7$(1&f0q+lDylUtXM(WQA45+vFU9afC>7_gfzP9oD^fco08jN1XK5=~|(( z0-d4`3Jz_|pM@CLGbq!7TeJJUqNKF2;A?A}fq)^$yJ5D<^6l9_A7x%kogi1Gjaa#m|1#uEppTEZ2{O?}F@ z2d9H|vg+SoQzzd2RhenSm!I^k+| zJ{$;vP(3x4?kVY2<29R4Y8xL1_|-Gw6WLMFXTk~-NWZ`O6cF=Lipo8sxE|jSFgjmC z*$}|2cwEw&n1Mf9M25G#sjy3a^X#MqjZzn`BGyKLZQ}=wBkFN^yYem8`KSPx$6N7H zNJh=#MM4gEu?LmvZod-K-5*Bp{8e4~fZjg4k)JjV`$F4C8$>~-)V8cr@I$REB%HEh znX{F%(RFF7+T<4WCacIzIQYbJ5jl?huq!`;sNbI@>MA2Azcl>!vn6Q&S7m4IrDO&L zWq&WA|cO^yc;{) zpqg_58k(_g6fPqG=YHPw-@%QL>)=zt`dCwEnT-(P|eC5{ZA_?F}YFh z4bzw<-6dO%b~9ql9>2LYF1tbPW0z+IJ0l32F?F}jGG%7;0JL|8cnC~(c_iTRQKkf8Qil=POY^K^ zb~}e-m;{Va@c)dFcWkmThe9k?D)lTy4zpD@X1*+SxZg`QIOt66zs|h-=-KiPN|Q+b z2pFqqFMyz(7i1w9sT0D%sw)lbJ8{{3n@KNdIL>#0^R_27^cNomn_3KjY2uDwam?=I zB%ls?JFr3P*2q}%Yi(|AM>l>97eV?tFJKcT7TIdWy{p{&zQpRNw@a76lO@TK+5S!M zDL^~b;!k#(+1E7}?)VC?D+g_DW9`CYe_y99-o^wlQhY&Mi*{3EKYNVKEC#q<&O-Yg z6Xze&5NQnlu-s(XeGNr(X9H9r`#sk}057@QQ!4>$SI+m(P=aG`oBVYTR+zC2{LJ^jt;%Z${tX#- zHAcSc@H!YIu@5*V^BfzD*lkdCHH-ZNSmg@{&S}-8W!`CCteoKfKviZA1{o<3m`T}l z?WOwo_HDJ%T{vfopyPC$@H4}|3J_2m^gBbAsD$NU>WDsS)UfRWVdc)$JsQs84R~ob z!Rn036enemql%qx(17i}>=%0)K#NW*U;Wt}$9UQ#6zA68ohp%u*X~OF0w{AZ6cgKR z!Meee-T14NySF_HIC%>NDzra_275~Yj(5OYyzZY>E@;EetEx#T4!;u@dpk8VsU8~O zF0!P$KN|bqwZ#_%7k1Yko7thvpS4)N=WV~kY%7NkURoluONQGbT^qwRZ_=fMAkyyC zxqem5N?QgZOOa;!A>Ytv`USF_6xqoCo!tSLT zZ}3GlP$fW8{K1I zBsZG76L5sFm;u40@SkzNI{D?Jtm0~!1qwx-j|u9PdQRToRuxk1F1^&udFsUy-yNJT zO>zM4(YWB#s#sOoVqTeShLRZFA;RqmmaaLqhCO_32=fFN)zxZo0pTMm zY0iXo!@MXR>OUEAT+5D1QO#MipHL=VFX_4EZCQD_zPOt0N?u4OCfVJNTj}1YCdF!w zvaAh|2GQKFGCnVw146+PNnV+!{Dp~URL`?wYk$3B#+LN7WuW||W3scywKM*keExO2 zQQvB>G;5-jci(G_>x^NP>cw_5YD&;(mBuBdt0wwsy0>U?2l>3&v@dXiH=g)AA-aRD zzoAH0WI!tIxJT!U(O_ncq!{dt6YbONJp;Mz(q5;pC*bB^yNjH)#Icy5(I$6 zA;lJdxdX$9{i?WTrowRgzzc5`@84?zXn-i>zSO)@%_(k8{5{wCVEt}yH3FI~3pr;9 zB-EsD|5*KbWQ==9cdNW0Sjf^D60w#V((mrak9Sh^9?jQ(xPR&UJewBXp*J(4_UE6m zREycvG*rEYw|^_-1*JNYPo_6tkLTZR5Ow?n@(=~?fu7> z+Lq*v4LoNToo|?gHjhw-k(HU`B$4Cu7e@7J$8Qm9_jik)!(7=%9?Nd{AIne6F=cjt zF;RkDsXx{er}&YX>^%DK#>2O*Spql zjE7GTz;m^&svDSZ5z{{hSR5)DL=3OJEg{8TStr@hlI!X`*Kaq*H!o~1)$$Y&i2^Jv z;YWdgrylT5c%19$4m8woOl=d0j`wm4rMSA%8Nk+c| z6%`xbV%Y{fEFnWi8(^;J0-M30C7I};hj1?75Ko+L*}6Dk3at_ryuzC_? z>}gMJ{jb5eMT^#P`Dk-=eg}H=o=#x1{EE>mcNr>fNs#rR{8g5#-w=yv)E~jmX)?M( z<+YHdB1la*>pD|+$9J%faZj))W%+@=%<0~s*DLiK;+?+!*Os2O)7>|(JT8y@rbc|1B$w&1l^}j$7TY4GfG!v3boPxJRjhsr*l4`z z0_obyE3q62gvlD_olyoEHfTs|^0EAq^`8>)c2k}y8^!Hym9#S_T!5=TKa>@FJtYSS z2MlvvgThKqscrQFEJ0E3*sYZWD6($(4^ z3A(!a$EE|zZS>uPwgK+BjaLW#Ylg+!l0-IJfukuBYl^1irG{LtdN| z(wec7*WFl~sYtEi&K-!ef0ao2Yr&|>nttDsoriL@I2wlzgzsc@G)I$PMUL;UDhCud77&_+M`zL zKJIcXdb$(ir!q9@GMmOe$=8U}nocD})S2}AV-ZkT;H9lJ_6^n7@PaB0e{%oyO{av4 z{R;{8zP{7lp|k9&i36&NM*rA4J1 z5k%=2zvkOXHpr=&f?)S*>2@ITzjpcZD_Dh>EG86Brl)l;ES=0r@GKnLpZob0rI3Ub zH;^je9MhFB%{xhv=SjD@G!?xOR--*y<&Lc{hfVv|<9}HXLtwuTk|OSiCHDiU{8N@A zbGg28zIwAX+cRE-`j#BtZA-gC;p7S>W+!;U=TX8AsSWXQq>;%aqJEpvnJcaqvu6qX zOHe?ff~;36(2_N4Us``zt~F5ly4HQ2?s zpp#-tIkK_WJGBf6d#^De2*Y10HP@Fubh)MOZ(palvo8cN2xXNxdk0np;tm$Va%Ko- zNR$u!rxy=Z#wY$vu`M7w~5bZ{&gCjC@Ss{q?6jNJw^|E|OGZ*m@5ffhlpSPwXpOH{6F6yfZ$JE{ zhyP;!smF%`>|Gd_#<*CN!I=~V=}i1vc3E1@Yow5K6yEkV+%vf6-F9fx`~2KehQkBi zb%&>9JKI|QJDHS{K_w+jA@W!Zv?wxIAdKQmPJTkRWq7rj-E6z7M2JR zhQRY%7qy3~9kI^%tDBzMb5vdxg#60oI~XhA5Idq(-u9*f5TOzjM6O%2@X$xo8(Z+` z^f8}oQkzZI;+mBuFleNqUWrAw|c12#aT%T(F1oOlzS0W9ZxWg2$DAM^F-t8Rx_fo z(su!(=jI+N!Bs9D7v#_6P=?x0r_P2PwYDJy4J)M|o5lo@Xvw4=cXbbE&2p&!VZA*) zwQl4=Uk*Ax(k7)*0k7}KJsnsPP*}<9`gjpFTAK712l;^#a2ah^)Nj)&xm#=L8io-G zEmqPq?_d>fPJlLKDNke;so!-#lMAUAA$dJ@IlyD}GDZbUZk2V0HM=T&3IRo$q*q-9J59ITnO{NgM@3cJo zCT`Y`;cR54;oRTd5qQaE<^m9}dW}#BO8kjcFio+?{RU?$_RXX$q%aFr>IK#DI=C*f z-y76khAd@C@vDy&+5R)+epC3%qVUesgaDVc{x}u;Sm+)#B`y z3DiE1OX#phItXl8p)ZAG%h7h@Bb#0DM%lt#HW)NB;KhuO?Sdx1eOwJATz5E%)YIeL z>L{)eTI^|@`xK($w4)!)7mABt*`to=!(pDzs=(~|`R+sBv3UT3>g@2%8B@nrF3HoO zqx-VOWwfb{D~H_?7Uak1pqqo5K2+h@c2!kKyPy&_2Rb1D4`2f`qDA{#1(MvADs2N{ zGYRy_aOW;^+rh8XeVSzgvDMxjuL@lkQN8e(xYa;pIf>HJz`eVFG`BF;HV`*9HY6WT zTs`9X9m(c;(h;m%S zLAxd{TE+Ion6Ds?kMT>){PT5NaLug8l=y`qczW?BREYajx7_s72)J6}Y@vr;+Tpi* z2ZbC#2X*06+T7HAhqc{z3nWQI9biMAt96-}sc0L{C>mi=lNlbFs=WBgG4_vj&bIIzObe?D_8*prQupx^nJtB zd1(fH0Yz1^flk|3F0q^aAJf(2YV1n^k3Hy=SeOMmq)u6SsivuUC=7NH$Ny!ir`~aX z81?oOaD?4RJhPu#qS$^6QOx|$RQxf;HmQr6|0~pDPZMZ;q-)ZBz1DrH7e(wHo+|nD0K9sbB zUUT*Rr}IA`EK+)qqg!{dwj>KgfA_xBXj*)4KxzAIt%B|w=g$T z`*IC8#xrzLdZe8SNJ?Jrg%CMqQ@FJaysjK`ldfKGF9#p2?{$p$Yh05n75#`$$Y0ZL zJfC)HwvOYR( zKV+aS-FhlM@wwjM9r#ima?98|VCHeZB)dO;{Ts|?!iGA;-Y!xf+37^SbL2x;0=?>s zy`vW|wrciz`YF0G0oj=Nw5kB@&+n)m{JRb3LHMAavN~COF8z{fe*8u$xLMMcvnH@= zvAvy|-Ov3YKz@J8aV+El7>Ao%`%Jkjx?rfZxFP6uOyVadI>F%1YMCuLx3tlr5qpM~ zn_m?nDETXy787-?0xqjE?_V_SI&olj422V-CUFZ%_ze&A?5l0z%C6W1%J;6&NRcvl zi9<%3@20XNM)lhPKDktpvyFSUZ!yR+zQCYyZubKFZ8S+f)0H$VV6*q$4^^R#n7%*| z)lYm}0yr;7boEt17B$}emMv|He-T7bgw5t39yN^;ehO!Dt$c}@8?s#nIL0&fS4+I? zsN*;+|42Wt!mg#WR(9Q4GW)ntP`sOai68+62mMhEQ};Be7YpEq+u?_1W_NwOl&kl_ zJz^=|)mXbeH7?FOF+0+|xHdlDBH=RZYFhtCHgt$bxr^Ep#WFK@orl)N``N@F`En>U`4 z6>N5j|)y+no_4>ej z2d3P0MvR$m>hzY!bSiyaI3W>OfiW9+%aC zt9V=-iH&f3s_>(2xj02B&qU#B($$NI>HU2o2|_QCvc!b$RZwj?(jG zfld2#V)~YPmu{3c>k>ts52dZnyU>=Jkodo53=Byj9Ew(jQFi#C-3Ly*=!_)_f>> zy80Iq}--D8u#_Rguv z@U7a^Slq`2+-mvC8v{D#V2{Pj);p4Rq}A{n>Gfa-c5!BXUCa}e=b<&h>!0HF&gS^q zFC@=99#`B6?=lD@q2{lKj%k-t^(f3W!un&DVjQp-2Uc*wabh2^o?a#XsFU3w)GSFz zR0k72e(;n7!v2g1yU}e(e3wC;Kc+dfR%w{seYLm=J1nJ2<=gSF7WmfTn$Z(%I86kR1OcM%1r+E1QZ-{~HAE>yLVssl8tWOSvzOd6X# z&AbZQx)0r|8E{`Wo>qEo%nOwJQ}?O8rZu)@uie!{o@vr7F{hTMabsn~PLId6r=2Nk;OH-p<8P$# zbW22HdpEF9Uv_y6EHP?cWA|_)hcyY*W-vHp`8nxMJEV$K*}g2w1^CHEXLN2|Ll1{C zhBEus7RMUZKk!CgiHxPhaa6X+0z8~d;GiE))uC6FMc zcuP39@)Hvcv`&Qe25)^vT2VylCXm1d(mK~HT4oUj5s|Pv3loPYKkV0^Nd#qJRuYl4 z;XYVAnR3$XO@!Sa`YI92yK>Y%GIF=(x(3-7c=Y4(KNS1NADXz^>2pe{BO{~sE`_$C z(E7tmV+R1qabI|he&(%}CPwM@l&uFZu<+0J!mhZFoN}v)KRV4fMflu0YbT9OBPRpzuBp$)8a0->G?CnD5r=|Fl(Fs4*Xfn3($}w!+ay zMWtgF5RIE0*3nHs0rA^W<8AxjfAKU3aF0|5y?piEE{q1Ywn->++rJQ5geo=jE0!ws zOFPXFqFWUrSm`<{RsyUC*i9I^Y2T#H77h`PPW8FHTedm*aQ)M*PPIn>ZqaePqB!-& zfKGcSiUEN>T9oWh1v1|I5acvvsG*uL>x5q~FPdOI*etRQbK3y1pOSTGvtq!_;b()j z#amD74QE*#{bilpZm0b{|Caa1-5uUN>y-k>i!-OMe?9)-e{%t(MN=w=wI)4%$PW<@ zuPE{AVBGvJrfMz@zP65J4jufwFug%Hm3U&0q&LDTzPm(Y0+W(gb5^>G%x9Ew5zqZ| zkByRd58a55BkXn>@haw5J}6RU;4L@Pth~W7H6!g$yyj`DQJR~m5(Ll}Wi6)YX~6@# zjH+AD<8%DkfvBys@UW`SF|oVUR+op4%7*?E_zLzzKthWl!0Gbn$!u1lJ!yW8`(^Q~ z!U^o7;rAREJ4=B`k@#cTGrtAAd~M>lc(CzwE`~h_ z{{)H#mOXBwUXd?W{+qD8)K13!+P+agQ}_u<7-9Q>_Q=u~6uMNpY-Oq}Zz$-i>9XD(p-7F@dxkqJbH)hE#PrKp_p?B@-Bkrtr^Ba^yApIi)rn93MIzNGdqB*c5nD$ zODOzn6f5a{E3O39C(b>7Aj>a;MxEoCeu7!(5}A6v%6#|jbB`SoE`Z4<2)mDH{_Oe^ z+wl8c5bG_3$`3W$?m-xvI`H}QRRP_d_Njq~>+CK}WfV;4YlmGk*Ll}aVlS4B=!2+f=i!#1Z_aw?LXPoZ*{bIy#6k;5G3 z%$$c|b2b~d`+hv`zu|LuzhBq&x}KNriQw+t`)Vu{MP8P3EE(FgU{XHkc!=CW7s(K= z&Sz{Z$vyz`SsuM%%gT^Y(t5AwAp|%mZZlM;|KkEmNT{Jl1JnXLD+MhO^ZnUeC`2Vd zZ0f*jYUfA3`x7hN{o(J$I+R|(G7iL?oRv<>ahl6cUn55tI2d>&71A&OfV5TfC!9EfV(9KYN6P9wW^@+AzoHn{g(M3fMBZ~r1ZwR-QAh~KSpLf|c_H`F@q0o#=#;bA(@U>6~ z1k_>r$1BmtCo2(>SMwsKE?{P{;Ksp49xb7!qTa%tKn0;kyWfo6T?Q}q7?(&o?b zrQNe{qu&Dd1Xq@JUaj}o3f=v2+!74uof10d+G{g*EFp;!9lIP!k{#crXwu+3)PLuB z1-bE_-lx%c^-}kKgp=iU2kqSMZae0$jH4uz$$+Nb!*sG0CA6!jZG#rG;q@Chl^Wa_ zo7jQL(+?p5d)TF5VplhUb<^j!s)0SvjXnBo!40n+au0Lq8I3(I^awRZ5!Uh%LC&~7 zt^G@c?AR|Qp(QR^sn(hdb*r+i&VJGR=tQ_wdw+N<*Wv*3k+d>~uH%3?d?TE``<|rg zyue)5eX_x7j2UovyDyI!{fo96LGxI<=7zP}Q5UjqPjBhG4-TbdEZ={j9Ke z+Ny?J3yj1-pkwZt(Lh}h9K=yk!$}&GrZr?6p_?;b5b@J-{6yR;w zwHQ^6uMq=;w6M;Li5HY?NNq^8~Wq5dMe{61TdPdB6XI+0Jf<;^WQ>mv(5Mhr>gS@g39^r9K zd`C{Li=S*a+jKNDb>LU+lHLAoDf~LYPyI6wks|n%V#lgpZJl>EvgMz%*v%0frGLur zQUeZ*TmyMMt#W)RxqIf64E#M$u!!r9FJ}oJzP9~1gL!LjzXdj7h|foK=4Kc#-2Cw1&qO3tnNiLR!WhSKE;q0gexrWCEi_feVr zRwr-n3FvS?>^tw6!xw@^onkCMtq;9;6D*Ps9<$-}Id$xhJ?6^y_XUsjjD<{c#B$Jb z2KsOEa>>M!w(Ym8pv6YaR1ovs27|0`oa~y@`YpPFySyZ~h^=pp&3-{&dry~az1?Qb z{A;2keBhqP zaqHr@!_F^=b~;^AW`PB)XFhqKdw5A(+D(HK&;F6?ei_w3I-cDUdf>#te_2ygWKKkx z>{ww9$pg2`AIR4R53oVRcxvU2&tVboEunPM$2L`iZ8DSVDt2s0nDUfkMZG3m+(Og* zl+@BG(R11dGlARfNGd_rOMbZ+K5I~Az{af6O#7}QbJ_ovZT3VDRbI-ntag7G&3jKk zgv(VIV+trQ4Q27>rLDcwka|6k(6x(L0?%;)Wn*8|*` zt$7iATp);n5qk>f%|qC9Tl}u4;kG!2P`X?C*ppsV+=hUAd@ZPYWvv;g^z5YR#d^sb zy0LA4OOv^F$(o*E#2%r?TXr9s3LL;#GZXk$W=q|dx94oF$*aBsK4alXSq~ZM%k@te zyT;#eH0-2F$fC|R%tvC*4la1F6@yZ{tP{<>cIC4o5_=?`Hjd&mo>C&C@cBHzc_x?Yy3ez3sGP@HmB zN--Zi2<*b%`!`j3MfuE<MZa-*NkZtQ+bLr# z^hb3o!3N=;?mnzxFRNUEG(+tFz%E_MkoS${M3-@?Qdpf1sWGXl&aiWxQ|RVo)jyDq!yMH%EcOcXOgX5??9zL=)tnXyj) z-t9hqY_LQduX>ZaS?Wd-|Kb94$ z8GojWr*JBV-V&oT?ptV|k3fWnNZ@EDce76ZR@gN(h4C^SkrfsL?YP^Dpgw*0+8bvs z_x+dwlY~#Ex{vQ~oG#pXS@C!Kt(rqCVrY^1uN_RXcXmY$X|M4_fk6D`#vkGhd)k)& z=X%iJ^NHTK>pb8$xDtao)$1oAGLll`2A5gB8>5odUzCc^1%$RzN1B(0G(B_tw0zI_CMeodX%eIT12VT)V$YXo6(7b?)P^;h<_DB z0+6ngl2L;#0#>Km07aBXheOY_VeEVCHfdKsnHs@>+Lde>B`)mz4Q_G6EIpXocEGdq z-N?^;jGHOM;7JZASUNRGS7{a89CojRzV zBzIkM1a$tg1#*9MWEIkptOPAxd}@s7Xa+{WOhnC?cFW(qm2Qmsq)V?u0+HGs(HQp*p>R`1=$lZ) zGCwr}B2s5Ta!scK&z&jjx#mh!Q(vNLz&;VMcW0!^9UX-DRTe&6^e*kbn*Es0L%t!W z9SWrp{svLrU&jW=-=#jgr{6cC>C6nFoAVg+Z*MD=5}~sA_Is3;%`r`}1T37cP~@&`D5aZ20=8 z48>3$t=3W-K7g?kKTk4OenRVqtX3zm>M0S!QVg$V87H4M+EyW^soB))T1Dm13U45b zI-+69hoJVv*eN`F$StNyT->jqT#;(p3Cs7;Z9A>1z1^@b(bTp(Ha{ZyI~1y^IX|0& zpbSz%o*T=uv)AxCE_X@H4}rL>BhT!<%^9lO-jtrqTv^xvfiQD4tMLP^NP)X{^1@%* zn_X?hF`O?EZ#~S(UoZ{4;hp8&(06LJDmU@1r!aB>+yBM=b9Q+CqsQ)z=shK^%O6LH zZ;l!{UbxcQBdIW3|Gn?6KbkZjji3KKQ2mvs6MUj9r{S+Fiw!P~&sdytidD@TwbU8! zyD@SYfh*@vDRGL|GT#>g56%37kH~{?vpHge7H(zS$oNv#K_eZ|S;3e{Hr7RyZHQ{& z6x$^~?Z!tyZ&-n0LHWMDiJWZfg#x|tho7}=oNGIhSlPT&BT;H*DaS)f{sza%Xg7#& z<#7D8wLk~bOx}7WmO7>GWLXL!?rERy2XcrUgH|-Oe?ptf&phn?ZX;tdO^nsFdpRuu zY_g#Z4;Q&d9h!m>n}^Lq_(XUrRP#rhd?(e6-mi>gLWl64WdaH#J-wiZ$SRX;Y0RIJxEY_Z7uGd3Gj&O1 zF6#9dz}3&=Mn(e$^$B%FEZ|hSSchzNX1FW6k$#}DpzC)3OXqG5lDZ=*9)#=CqQ2O; z^2?*_v0y-bi>~O0T`6bQKYMkegxqw+(KRsY@PYE?)}Jvp*^K!O8^V*OqCe~UIlDpe zG4v_q@;uvI2oUa51bhM3R)?+a-V{r{!Cf4qg^YpR@ zY|p&J#AdWq0c#T_S^;)P65S(icXaElnzSM2H)bcIY4@y|({j)~S& zor1b(sYv&ck^bD=nh~?%bWtnK$;#ulVT>R?hoxf3T-%CXl69f#6TVpXh*DOXCq|N^ zdd@FnWmBoVrX^L91v^^W{$E-{16KV5?Ch;>?%X+*=6uEH?pkPk<4rEt39mOCs^glr zKQfd|?A>x~qo1eXbkftjd-SGy>^i8axiz^8=6>3A-9L?|=PcQZJvlw>OkhGDn!fI5 z7e&Km(zxj|;Q7x(58kR%CIy;{Sh(9;^z&n&qonbJ|3RN;7|Q{x7Pt?kDr^RZkrd}k zhP5htq3&pX7dXr{U;41U%lZOHqgI>K*Yr-lK{Ckkhnyw+|Jl5fb#WQ>+aUlWx5xHH zGon$ho1wCmWp|Yq=9ccxS3#NkawsD8-(#z)Y$eT}zl3?wn5p(1c>N4rr>b)I;qU@- zXVh^ELPs0v5B}HlYb2$6o){$8pn1d7-R-UIse?6QeAf4vnG!+Pet;rgDQPp5 z1RZbs$_YjxdghiF-<+x?zSG~GhC;&xc>??JhXzoJ-?N}}0D)`;>eLdPf)w}EjwM|? zTptbBTslvkJ$>GoHKth$TxzBwB?Og*tSiul7WQQOR3q*g$2Zt8H?9q=C;lNx#t#8KW7q1=d$m+=fF# zC_ByLI$7x1yN~@v;dxN9foyHx1v$6M)!A+Cr*3^d;6x;uT0S!3ry`bfDSGoDqTcWF z21vP9mq4c`sjyUVC-9>S$s6Rlj^4Mkt04*{eAr_P^~u>p+p?S$FU9@CU_R4$ZV)R~)&0LoDX| z|5}XDe6ca=0`D(>Q1E#+n8~FLt^fX25sjR;lF$&vNPfQH${rlvZ~Ohl^tM>0>i3EmQuB`!4dKDv9xDxNDn3FaHRb8oEI}%q z-jc95!k2GJYBj_nlFc;g5(&gzkK`!LUypx#D9+j#At;{6?kZ=f#waP4^OU5E3y7&DTRB&25BkE{t06khpDje~g3V3+8OOmf_rGZBX!kuAbr6 z3kQ*qB5BlLO7c@e8K!WB@=R+K(R_*VK6rX;Vn9Spd&W;b7&NY$^;>Cwwp5o@b>!&z zR(k8EI5CK)bR{{Udr%buu~;B<)oA2F9q&lHVDvo#0D)sxS&6+>PHUd{Q8 z(s@>rdD6(RpnOnXQ2L*r@>WxrO!xHY;l9B+SR&sGf5#6l`)yXKoBEzXn~v9gONRHG z#x~x^W2Pl^K5xoXDdh|BCrDy{mJOuXRR|bmD3Di~UupuK$^bfBhT9gF+^S+5MZZtI zH!iN~QFA_eQ3bt^S|E;aMIVxui`cq-VUc_pR#&#*n8Ov{sWq4*Y}zr~cHOaD$Z&^| zUY#Mm-nGTBh7m~fGnyaE-)%PeI*yc+gZV61CH%bz7pCCun<00_g`H0?ul8YU`plYN zxt-f_Poa{4or4ZJZ!5&cvZT!T+j`C8UHIBWjDP18U`HtLCq0{N^=ogWc%MloTGj3U z{>H@r*;jiwi;=T1n=TC{yFo@A3BLQzR%upk1o5?C^~*@fiP`yqo_&AE(Kq?WwS^$0 z;uGJ#^*g~umb6|ulnVLl^MduxBr?5*|sQ0Z-7`~fJk*tV7+8yK!g zgZd@P;{R9QoT5ZqRs1rsBkaJzXKW>;?!0@WoId_WBY78rMwLcB2M{Rh?Q#swWc}A8 zY-^wi2RSHo))wq6TdAg>SSag36~vu$%y|PEnvPDIzp4iw>C0{89?lIe?ft%v7%$itmr`5 zjf5Koo`N3gb5AntVlryd8c`a_jk15EB4smiKCgbskcQ)c^&0<}A0J|N4kIHw50X6j zw&U(jw1u-CgL!M*UR#5P{I>J3C7$~_Gt0rc0bQ3!anBx6JOfiB1@?C<3_j$v+Pz^LySNG@G*96nXqZkh zTx%f82Udxc7j8>2vbOIZS{5I@J5Fc^1Fu>(>P;bj4Xm%sd`Oi0`NwAtNb?b8i*M}V z9qHC^t#(Wb7@2zQTlQCv>fFpQ4bal(89w-}&@IfSZkw*+1x)Vy6a(Dak8;;V{iV*; z-?KW*T*okmis$onr$$w#=h_Jsn+^fqc$>15&UR{Mkybt+rO@$v)AFqI3+qXe1Al~ztGAfVpLvhb@N0F!3E@jF7^6krwdsa0p}+m#xIEaKm}!ajPLB^6yd$W{5aBUv-i+8q0eb}}<& z8PwiT+Hd*h6j0_1_2zq+q$TZ!&$hX&RB3TIMoo~#eTpA8cWvkQWCK%%e+mH7Na0jN zQ~zXkf9K0^FN!Q>oVZMX=(9M zo^fC`kABqX6Iw931hO_zOSvTlx0)?xT}3@R?+Fl#!U+p^lbud^j1HRe;qpGpMQarE zb`a0ww&N8q*C}$_BdpVJZz&0?s`Fjrap|1BH0gH8z;UYNq*PdLe`$IUfe&t~mbk-P zs+(q>6a0s5ww^0u!Icij+4g%>hZmp1DAmr-Jrj6B^WcdlT#!lSiL-kQ60}zhrqtHQ zOl!gz>5(~leCqVCeAczNd%|_X_njSEhS8cee5NN+b_p4u0;}7CtxhJTyYFD^HX^4> zK+H>DZ@)6yxgz9Cug}405b{e=g_XRU zfSfr^3Toc)+o+g~o1M9m5beA>+YLD^GtHRsrCvN5yMkV+s^6~hmm3+W*E=gOKxz1m zftB7W@GPxr9uE;MqAtn@HSEmT3}+zkW=>(B*y*}4 zGTLP%_GyD6e=aA^W4;*z~#;KYwkM`a( zWilH$rN^`O)J| z+(nMfHSu-&@UFwk~T~yCslj# zE$p=u$}sS^u}Wf=pT$S0yi^KH&{~XI^#<22gZe$MAcx@E%wh(pAWU+z@5 zulF!ZlqG~?HO3cco5e?#qJd>6qmo5M-Y0-MU~Md${fzeP9Je&!v`(k@GA6w{011;+ zuA`C#9ovD0{$6XB4ltq83%Kj?m2S3w418b)yQMzCvD2IPThpJ5d`^@F`IN#c=8Po{ zPnYm48OCfqqY=_wy>H?r zdN?UUlk;wf{Kli)R1lvup)D~kvImh_>h6{Bz!#pyJjoDLcLS=97oa_X`4H?EPJ4th}Sy>`Fy9Y=88#%v(BB1t{NHdLdWrekiUFs z88>2qMzk@Y-pJpYSk?Q(jB`LgK^eDJK#Ex0vZW}pWV3%!%SR%mu~7HD?bg>Ss5PZ5 ziGC`|Gn#S&n$$irQd|ABa|^|(%PLe262$g+*Y5L4fR^6wpB^3g6{|j*V!7z-Sya_8 zfymBpR0%0JsUpuIKVrSs|CeFs=u92n+JCDYst~gRK=Pm=;M6W#pZBQMvcG0FPuan# z%E>)92pd@;Te_P*Z;YlH;T0IovCf>+;F*%TN{D7!SisMxjYxUs{Ph0Mrjy~tdqhX< zHWcAiZaSA+mMhm_-yiMZJebUF>GP#3JXdd3%6{9|9oU=P&-kR>IV9Nk>#4M>sEyiO zY(46;!dQ)7MXUSS;*9n75TXwt8|8l|6AaT@s-qO6TAr^{KSN}Pze*LHB$ad^w9@#p zUH!GQU+vYrXp1Mc$SlKcf`?I_-5?+S(dL#IP~abNv&X?l`krJ`Wc=ZOQN6hU&fQb{ zRE)2FEo~>DwkGGb?q(!EGxGp<^tajp^0+B?iSJ3nj>e%zK%Un44P8Wl3Eb5+8)z=_ z{n#osv>qDK+vekyU>53*J3I`-dU4#^7A3_5WC;lR=R!@1g}x45`$J z>$nOQU--c@)=O!=C13VXp*dSvr^19ZUlnP^wSL^q%VYh&ln+8@joE(@{`Fc6b*P+9 zUTOC4F35=!2066p=MS8LneD5Z+wB<-x@-JEj?7_#^CQ4Sm*4 zqAx*3>-+cRR-E){=l^(|CDiIIn?1PR&`oy@mhB4CV@{jzJ1Fg-%>{JJ$_z_Df;WfJ zo$r;@b7jttA^?Wl!x3um3)v{IQ(|gb`f`2^gGcf=P@PAAP!Sz#W5B6acWBCiUqhGi8Ll`vvxq>~aS~YC;F;#}snZhTBE2ik^9;kr)Nk40FSs853KRc);#a_DRce0w7IUNpUWXg=If>fd7o$ zFerCs<6E1L|I>FRvinKZ&*;{zGoPS)P5fHJ^>q-9mdsU5GvWAe(;KIcx?7<{Vq1aTD&Q@Y9f2Xx{-@w%UR>8haWT4 zJACCsnnqdxlVd6tEjL1|HeRj~K@t|WYPHPP>JntAWJc-4{>-!V)FI)mS3PyqfTBQV z%hvlyOpgHLV|PAp$id_3au>uuTd=PykE`{dEHi*LvH$kzps$#bx!&@015#&~jYV92 zchW>ZOY4;Qn-*cn>eG7Usfm?yCgz?kOor2x8%wNg`AyrPI+<8&^z0nk+C+GpB~ z_~~T5_27E_;u#EnaVO1q_=pjt6XW-&DJs7@@(qTkNbsxHQ^k#(!8C+5=Y&b zt_@2~*3gYVingaAj(ZQ1*ad3G<2j}B0I90oBx8J0~gL>nc3|($A)d0pu1%IHEyGNMp>!q zd6Hlpr}X%8N*ebd#Q80}UD8t3DIj2Z{;bQ|Rhbv2mM2UM@`!(B`>|{6ny+(Qpk~HM zDO4=O<4fN3FD26Gs`~Ycm@6xZ<3icpTocORenNQoCgm&ESO2I8_;lc7Cs%wTe z(I+8$mjVSZF{;{uot&ES@vN0D`m2rXGjfNzOQ2Db->I4z`+_lX<|gHl){7mS{*sPk z&7GUzK--aAma9k3pWXk`trE(mk}Au~jpn``BsGpoX>?DLM~V+x0sqx4N{TSt1^|sa zjWum%1yiZ(!S}$Af?uN-FKC9EE~R!XT-eWDy=%FYE^C+lrg*`%=X!$3vDK4G$*ZAN z!+B}3KO9!0PYz7aLZ6+RVcMhBU5^C+qu(1&Z^GOp-ITQ%b<=(H+bVtzJ?mB$0Z{ka zseX{;?@+7tC&$3L>Mce#dUy7*pJuRrzJC7VpGGgP;Z#@N`KJKnCdFij+f#1;kjTw5 z(&JPsYCKI5lB$UJ*`BdNyj>NA+x`3Ur>76vyaMgqOr+JGnrx35npw8ghvu0ix~#+- zWO&J*%frrCoJ1TSyQp*S4&+zFD+N}hJ;t(HJ&k&?lVWBA{b8|^?5JAHp>#&9w+T#Y z^>(mKUzGlKIi}0Y_AbzBXgCiy8zJb2y5*dD&p6}0{IV3g^6ZX3D2;XGUX%0y~7BRwFiVZQb{#6#O~a63*YURV_9!Vv)3!z!d`xm|e?B|Q1aM~kD-<$_ z%m(RHJ@G~7HP)R&HT!vj3HWJ*UUB2f5;r5o(~psVXazhM#e)1)d39N-BU`N8l^J)* z%HcI5PUY*aQj7@peY@Aa0w6ncGhW}BIuKff{2+^G;n}oDvJ}`R7k|z>S{Fn62GY`z zUTtJfpY2EOR)P#lQhpuyv9US?aU<6InNQ)=<>=VMfc4H#bUH56N8xB$oHlyaC@|m) zl17~w`Jf|W+pkbND`RW2y}l=SuFxtc%qSLA#96NNn$uQ%d=Nt(0v%kvY%^azeZhLX zRPOkWo=9@n=A74p0p1w$wc#&ahId-?;}q-K9iQtNJ5{A6sAA2ni{qC8@pq;BvdDM2 zg?+3_8DFc`2aVG7Z>m+rrxjCfm@sFvbaj86xQ@;8A-Kq^JlPlJoisW;ZStdLc!H$r zhu=oEjC}2IoORTyCSFN}h$r3Q0a@Qsh%(g|$P%FK&lrD;GK6eP_slG73tU0e4Gy0@ zOcG4%4{7byXqgdJJvA?>FQVZss=VYL1i+t8m8Ek?n$2e$$}(|r@x1DasoU&8=+u=U zY;(-KK6`r1*U4E=U3Yj8#!LBjY`8}xd!)B^!)=3yCGh))PG4!Ca3dF+b@uK_Hl9p^ zxiZV61*0}GyYp1<_xmTyYuEE@-G*Mq7Du1Cye}sAgvc*yMpqDNb@Tn-C*xAK*hT?a zR#bmWyky!BPOnze;Z^7nZm&OUch*;?@!+{vzezfk^?}>(tIV|5Ss(X=FMy{D3?p5F zdDAEBvQR4>d;R-(~nh9ELRtgQJj2AknO7~VOP^{W^M9n1cj z)?yNlHG5QyxD1u{s<`355_)zRhi&_Q}m^pAx2N90|%o&5tJN8ycwI(@-=U4Q}T_j`SM@uH! z3y=By)*@dWvtLS&RulfmX0E$&*!w_o7qqp`m!{}6LZ8<6=redmm-7pj@`&N zwa{i($%mVFQJV1AAVq=d&ZNCsiUI^4*yqD24y)r$gaCRYU#v!j4ar?*r+I3G8MC

oVnioVZuB>}sWAB>gJC zkZH>GcxgcNiHl%<_ys3e;am#fC$W>4$)6V=VF8@9A7WHkK-nQDk8ao_fql?PQoLzf#F6%RdZ9vflaI_7DC9DkKh*$2X| zHs(!O=RtSH^?+x8%sKEr;8GSvOTczEca?WJD5U`Fw+9HimbO;)74f>?lJV>+s#Vjt zfzX#MZr2^wrL)%bD!IP^D|s8b`&3_Ov~&HT?EZlv!*Xub(x(To|Csd{b#&c_?x5`{ z%b5pR|J(!Wg-A@`p93h&A!8Okc6tk1v2A}OTn{NroN%LdM_GX!30Peb$uwh%;BNE$ z{jM=yASTu=&cOdLge64Y(sy8a@#_zR8MLn4ZKc=%=jq zi|QNdcULcnH!R?)FXDiyWcj0FdecFxXgYO%Eh;mXudnNmXtPD&_rMW8UMJvc>WqCk zdBK%?zbLxUL3<*LAB;7sbF?_wJADa)E0#kO(Fu{2T4Fy^dWpVS`A@}Ihu&tur)2=0 zZCI^&m+WL+Kb?B_?vfuZZo2wigZ?gP@i%v(aak!{UKEOoZ(q0&$fIhdDOiRkE~U}W z^dJgDqqFvFY@$q2>_w!9B<jWh|G2J=&H z4XoH%A(}_0?vDXz-r7-m^AvG8=Bp#>U&-d(#LVSu628ol3!*jqoa3EylH?6`yjO1Q z5hs`S9WI7KmdwZ* z;;)*WM-G13GwBg>E333JK)qW+rKH(0ku(;Qj-NhxZGGl_j~rdz2ChkwmrhqjHrWYs z@4h5cRxYq1Aafy?W1TO7be>4I!WBjgfae&iWZ8A%>pDB|Y82ieU~>PLaib00;-E^@ z8ZI1-$Yuv35A_3F7M$l@BH%0BRMd&m3OrjW4DD>9np9j!88>d%wHg~1>H4~?#&8J; zXZ$+kP*vbFGuSy(2E9$-!YjHl&c3FW?|*JKrVI-EDhf!BkKGOcchN%sRNKU<`K8Ly zR2+D061edyid8)?!iW3I^(CHE7T!mJZ&@3(h!ss37PirLr3aDEt*bnB5%>BV~$1@M31`gIDWQaJ=fla(NBY^BxD6X%mF<>u_;!1xu?q4F{d~`8tMDsr45jx98n15>aLx_O4 zrWc-b$-u`bJADq*^;l>;s7QK2VE8^)Z^2#p%dLs6H0AIf8~6ViD)_I^yxhLJYXyaL zbsUDB2h83SP)X&SzWkf-Lwc~mNdaHH?-kdee)r+m1lWa^$22b@6?WA87gIGrk}*bSt-tkkmA;SF2) zw>VsW*pI5J$8d!#S<&tJ<<%ieNG@`L6vM~#dQsYAHzPG2DWxtUjK0^1nyF)2PQcjV zVQTgF+(t8WznnG1j10B%?|+Gike)9S_HleD%U#ICvq_*Gy}$4mAp+hIfIN9iTybn0FiCr%i;sf`XCQr{M1A+0{dRzcdp;@iD22ewe=05{3q&n)aEk$j|8AmCpS#O6EV5+CCVgl=3u#8{)wwZ4 zhC(cA;;djjq=?57{-=5w5e?SOvx3uUdfsS%VYjXU=me?tch&O z*uAfnKD)W-wYesUrTC@An8r%>Un<^-cMqHB3T*%C<(|dcGj(Q?QsM+W5-oW-h^VAP2yFsBHWV$tb+`21XAT!?Dt&CGtJG#_y zcoFf7f1{VDMGrbrJFKZ>F*~yuQP9ctcBX)X}c25t0zjy>5@cVAkG zex<2?UyjP?O1w)9OA&jb6||UGltsyt^~()ooYUxdXN7^nAf8-SB*}_fb=loU)W^fl z-A20>tYoA5NWy#gYAH4}fab=jSa+)YSRAS@wYQQCIE%4|i%^vQrO9h1UVz4{PmgmA^joqxis$~4~bkEN4 zOFb_-(Qh*!@XU=FBJ=ti)X71J0n&OAo*wDob0xMu$Vh(C%D?2Xhmb4z%*==3nWECS zdw+4u&```I+#V6fciHvWN{m7uquQJAh@|WF(xu^ly*Vm?62(Y)|b z^b2)MeMXL6>A4x2+ylBr<&0Tn?d|eluQ_~^6-7MPT(vJ}NrCw*fRoxUt~k7Dj~!A<9x8xgtAPHY7FzcuCn|4L zZ<14gZGz{XDZXr~ICT4j#U77Hl^7pZwLO;bvFC^NFsnvTVU=!*k!keb`iW~53A>*+ z5>1+CE~QtgZyIIT@+|eAoLbSeyxuR?^?jbjSQO!hasFyciS0pd0;=@_K=MRU&rI>T zmZ*@&Kwk~v(P7rE-VLsK9w374Q&s)ke)(+Ln5mT_OxHV_9hA8WU>KJL?G3 zFSv;Q#0I%q>g#2=2ENq2zxGk?u?~5@4&s?cZR_cKUJ|a1E`4R= z>Eoj9{M5MDHVSRuPr#h;vWk>NcW`}8jcM@BXt$8He}DbjHmhQ?BNc|MUcvZTpO3uW z>pU095#xPJm}ZEkjNLXz0}r{HqF+#2Q-E;jAwNm58Ho61u)0vWDSUpGe{DXRG<#d> zCMF(S0=6EzJu!>;)}FYwT*Yj*AGTS*3F&XksnewR0jqRQ)cZ)rIjt{1(8&DR#G-9n z+K<_jZtVC!Kdmp4OHteD9LV%J0KPLh5gS#XExmTBzdKX1}^ zRR_|K77%mnSxN!(LmD>wpVi`Bk=fj(lETJ?XG5Xptakj6_7CvBaGw{@lv;A37>gW~ z8uxo{-n4SMEjTpPz>7dK_Iaqq>RBK@pE`;(^2$qOza*$i(ph{5i0{<-R_q#iQtKGi zhK)&G*{|2 zPE9im(;@Sv-)y-9WDZR*3$#i&{3Io(MZ0HOSNFYdch`}W$L9p^p9^(ZJ?{jaa$-+( z+U1oMj1&t;1g(Ueb*^tC8O*;3K&+r*VR0{jSNA;$&c+)gS8>Dk)xbbJd?jHL$Hzx9rLVh`5 zfst04r6ZTz5xs>(!4li;@#-)+*19?6PpPJC#$&?YecPX&mbMv}&x1Vj*3%P^S5MU@36z zUm!J4mm{(XShsSxXuQt=`V~GmXAdcwj61Q#u-+Jm<(=3x$~9Hh*SgtB)^Ar|sQ0`t zDr&X^qR$QcnQAtFS;rOCOijp{IS_?iZMiafC8X+Nqc#H}BaBxf6^-7-T!1FW{ZnCU zgT&VKW?MgdtEf3b%>250IJdmV&~RMa<&1Tty!Fr}Y|;DJ&So?5fkd$a(?%y6s4vKK zN?ws^27hZ!OP76z0#OWRrF#gpz00qpNf6_E>!w>b` zNFETas7C$6WRwm7-XSr(Yn+zz*@=8@^r+!nAvqQS-HMjeR)BcvKt4Z`Q=ZxKJ~UDk z_WUaM@5U52|FmbxGKaN|gJ%tZlOZa)59t{go=98H=%MczvprqA32(>%)}*Sgk2p9P z__w}s^)jtmOhaciH-w|aSZN74h7Wg~J5`+rA0YN(iG}s7|L!m0z+<3$uVgIi|)Vs~x z_)WYCyoynxQ0-M}++L0W6SL#1R0Q$>+RGFI0K8$~?nV$TGN4%)1+on8I4nUKh1z&S z+aOPXa#=#@r(O}Y!mAuh{LZdASs9{WASN545_swbFn^|*uo7|l3F`w^9U7YKTw4$Cov1?}RY&ng1SR<)HktS?3Mm9L$vcb=N zjdF_Ph*x@dM&A0^=^`fDW|nYPl&8k))vLKk`p#_*xJS}Oa@_Q)M>SUo%|Gpyv98B@ z@7OKhyANadhGz4P`kHB8Q9&9Z#XJT^x4uP|uIt=y)wlYd2I+}VmKx3{E(C}p^of$6v7*@pQ_ z5lN2HRL)Qr|K@?LgWSaD^AS*aRSMr}F-sR#r|tQ{9wh$l4Rz%XX#wOfJz2$3Q4RTv zU9A>oB7)nLxT*0fv}UaCdr3t(V?TA~AI?xk#e7rw$Uoj(k1-#ZuLIpFd*9n3>umKs z^apz5j)I8I6Qa(YKiIRk`9=0AcAg2Aogy$x#Hthv{B)CXaILYkafYZeHug+p@6H`7 zH&s#n1IuK(4Dfy5$XPv5z2VvEsqT%I2VmziGahgGzFuxl-yS1EU{ve#DLG{dX7p^4 z?75L#51QTbrM%LFBh^XjSY)rb?g26uvTyN1B(TyHdlG~V^MwBIGr-7wmk-4HKcu~9 zR8!mgEvyGM9#L|XqY-H~L`8^zQUg&@Q7O@b3P=f31f}-?AyH8|N>c&^1QHQ7D!uoF z-g^rjLV(aAg^+}lH^2M;-#hLY@0WYucaMC^hrRcD_FU`P`&rMNQ)-Cv(xMrspw!!? z^n!G=hzC%)jg474*rLKkysHOSY z`7`iMh=gjmD`^C4nLC{v);T$RRjQpLw}UY&>T+UEeb@4R8Wr1>RuU8jU|x_zN^-pK z{BY|q8Sj}f1Bz1Sy5JcRyTlqw~ASdtG5T$FMLk_Li)k#mc z_WHoWh~b~bSS9j;r?OBFC|_eD)71^DW0Xyc|Dlj}s|dT*Tb*1Yqm*&yroE zCD?pU*q8PG!83bSH0BL1H_1vpQTw>OLpS#d0UlT!ESItr)XRA~%Js+5O>Y-N&R_1{ znqH=>u&*jZg0fAgwTJg7zVF-Gd#0)~+Z@OFB%Q`Ix~!(IVxtH95WpV5&6RtKQdeik z>X7-ay34($*Z>ohSpB}jtw1IiE5DCdX`ddvkxnTMvT5 zX^CS!X{&zq!Z*|3jqG+j$)S`IJgmEV*IVn8SNZ_QCA~=Fu0BYkd2O$5m=#!M z+pUbWWu^bAAw=1JH3lER#x}zh{rCNAS(l=uzqU7#E@e)7odG`~N;~t8c1HGc_b5FN zXr1+mf`~Pp5@W7ks>G?FL#favuQ5++cbt?JXnuyV?eHQ>tY&WZL!PmFb3uk`OgtXL z8l&$q;GGTgye?HKsb*XE>OJ)Hcgh<}nVQN=%=^E^0t3g;E9>`y3)cNqPjuL%2o6IX zK&Pf%fNPh|mw7>q&x9@)MjeNF?jc)WmM_8R9s-s!4uj;QDeIp&pNqWw|7Gyc5rdaf zy9{)=Xz`ZDM3~NBID=gw4B2@%V|X+n>Fwr(;8D(*I+InO>4`%wyB!q zteCziIX!P7;QfVm3wVg^TW%+uyQHYpes8#8EVE6cO`imo^;EJgjCg=Mc{`_d5b3OC-jCWjGz*yKhwP`S6Z3`Uza!;-BkJ z1s(XOJnb;*s`oX$T`AqbmS*EGysKgI&vzZqmg?WbU;R5kie5bKv43K*m164Yf69Y# zYMOEy5c*11zWmy}DY)lcwML3uEL!_Gy(B|oCBuE}j(6%FOj<-5B7-2I7x}MS;bAGF zS9gZAt=F5yL9NEuVtxgEP%Y5?XijWPRh$2E&`;aiPAv$PBFfy{Qc(czn&CbsIUC`j znmR0_aCxcJ(O7I#K)VwVLjR&@@G+Uy@ZM5yud#AqRM(r7^`#QR`%<%cJq&gp50#<>1oq;0mAa)e?x!QKyPbMF{c~Ek#Z%!VLZ8sJCO5>^+AtIpAKi zlVc{E3=!^~+S1JK*n_^k*dz5lwE4b|Pi>j+lOL0l4$XdPI1GF+#!NIif@1lIIcpU+ zbMFrBnTWXNN%Z++Y8rSG>&3etO@DN`m?t-Ju8cEClX7o*6VmonXEL*4T#jI8D@#JW z8t;sXRGWE8!#4i_s1j2me{io%tm=Wm_7iu00C2Oqy3O$g8KQmz;j+}z5r?8$^lW3p zS-H06F(q04Km<-nN6m8yOh4>h(G?-g&v?{BvcAfrKbh-pz29HrNZxA=zFq zZ`)A1S*}Sq>#32G!FcRE>z-N~vGtj1Sx>=Zz18#9w6*{TX&75iE+73xdlYexm8K!r zo#~3kUN52U1%ZigsV*235U0=}WHW)eR3HcTWMX&8+kUh3qZ)*Zl*)ne{mN%Og960UnP>~boMk1;PRtN##lvzV3k#+w)j>Jn1250aa3oK$Bt29 z8|6%RPYbtIGnqSy>x_~?tzC{rHx}vq&FpJ!I$3FdT=<2uZpIUMvwJ3A8@t?*B{9ojY zd0Q)AkN*|XbZP=M=_K{M`d=`$w|pGE^jA#sQSyRYbz|0GwFF>-d@b7sN65au>8_ba zr8MmybB0a6KDvRTYX=@s=-D(-OhrSWfCvdv!PV!9UG#-}C^yL0ty{bO)0l;GH%-X) zcWfj}GU?eLnkP~^-5{`{R1`h6qJk`GuCs%Exap-_z`fW*YIBwq*q)_i3D=LJ*)C@w z;A^XO`WqcUdVR~(yL3m@a$B3N*^H6jqIj99Te^mQY!x(o>g;sU)pUZ#O03}#ybsGo z@2q6ZRq9vnwA{LjEK<)I!&EFD9+aZd+5J-;NcAQVvUR#SK4>S4trk$PiGKJF6unUd zoJe*}^>+%NaVZoehTqz`M2$A3DPgoo0l5$Gn73t>nrGEtM{1Uhov_>-xlJ@kGKj#w zY>w6>1jOO_U*817TO4}^Lz;1Z!WN7I>u(8 z-dSwJ2v+-^;oqDKZy(K>yrrb$TuPwYfN!tn*~Yhe2Z(v7>VySp+Ny(^jLx?Ib>q(N zJY()qvB3w)C!8;dVR;T-Ug%q-fP>1q9@*f!-{ZlBP5yUKcW!h13XH%4Y1pREtW3@X zE=95NgIMv&#{ipeGo9j}Iq(epOz2BN)Dx<^XE(Q{_SuzN5Zqpy$=jwmRdrN1zaVx1 z=Mjn~(ve8l$Zlv;W+Bgc~ee7_N5ePr>cP><*C$coZ7FKF5Z=N+0o`V z@B+Zb=w4gM59#tKP2~9`Di*gqfR1c;u5~Iq9;kj29ubDS3@QLZWCj$wOYFW-q}>B` z=0|`TxL;hBAwzoj zPWcwL=POWQYEHXj%wXpPwV9y!>td9&C+nahjA~bPTmzlz3d#Bm>&j8H-IPhgS!+SN~u2eEwm(TKlr|*~XCDwV5)8vmYX>BonX2&TNDV?usZq5ay z?|$YNxGV9&y#43dCC#%Afu(P|Ubmf681}nS*vgjlUh_OM+xF{^jK~sJbXkw-affij zU*?6!$IphyypaB<_lsujVx|O5`Rr!t13!d|#R+ib!%~NW2fMbKBy{zzt6XNNS9+A3 zENb|s4^MLu?^Acs8&&jY*Z$aNCueWXFNWa~=UK(N6s-Lpun@!$+WC8!%Nh@tC-3o^ zhUE+r!pe8I=N)5??nC!~^JbgNx4L0mK9TdrGU+#3oPR4q&ey?`-uF8Uz-B#A=BjcW zv!88%4k2q#YIUM&C6C;oQB!~2Npqy!cYli6mJ44sIRJ!RbcMf@?4EM_ZS!WtW%a5| z{2{RDEa=cqVI-WDrPB_g%wM{@2; zoohM??X-Efy50Md5QKH=Z<3JJAzAM)g5^THOvc(IOn{LjH@U*bs5B3NCTwK0{HUGc zV65fkd8Z3gii9>hnA>DzCipzTK_JeKziJpP zIphhS{24RR&fr|cxT+QX+@}nT0YO=RYpUkEv5Bu!Cuj4pe{j|U@uAra!Hx@;`f4lE zB*<|xVhYtcs4Yc~#u6+04S(op!?;J2eW|C;RC4Ovh(=6hNk^>ACL)vwQe*7$9s<*tgkogJdJ6m2Wk;k{kn9b+t0$@vvsImFLvjjEDw+ zyZB`m;U4Yl1T~2aiNZ0K|^{n@H7dU%o4?Dk1GLz1dGggr0ybkA|C42L5z~!Ciq?)spQiY50 zlYSa3lHsC^ZiN)Ptpp1&TKNkjKRbpxkPbpM6^36C>7DPC5tU`9Y@Y+_IVh^0cCJoeL6i$v&Tb@gbZK0GMtY*+cE%ozWxRi z?NNRYl}MXLKJrp64i0+R^DK1e@hI_$qLuNFDZT~N%BJxeT0OMYz8s+uy-T1U$R43B`mC1PYrO3!UEOqs(f2BBc=o-+AEGa8&T?(1Bx#l0; zg53RX*z#yJlikwSz9v8FjeowO5Hn#PiF2bN-^a=$#s}!{{U7Xn@Yq0h@((Je<=T6s zXr~o148|4|G-A-a@{`}~kC1r-9c8~rY~;h~S9POtO=@-2r_`O?)e=3dy;HQ z&o@7-ugA4t@EFGyjgRW5?rdpR^Iod4V*|7RVpCZAxw?YbnNwansbItFMca1;rf7a~ z6b!u3KMgVdp_>6$CxFK5N_sTOO9YmFf@>=GKEBK9&4$Yv1T{LGd#|cw(sq9fa_sGU zBZ2Q7L#v>RuqkQ0!RSpu&pD5EE}_CsR!=Z8_Hmk!Hs!jRUV|!Bj@|Y^Bm+Xbht}|;!KIu+sST3h-DZst#Kt9RdKzQlB*?RG(*-Sg zOv-L#_K!H4om1ObjH`|v$~|XnM`N;|`V}JEf$=|_9^`*Xe`GrOyxPR?9 zz?34@^7*6ZhnYoZo!5PPG9(~A^8paKZbX4g(V%P6@s)S^=0$ajE?Wt%5kD+?kA87J z-~hDK8@PR)h9S>rwpm>XNcvMvIn`O8Y@5JmIG6 zjnf_2!y}RB^aGM2O^eTIRSupzxk#z~DUlFRa@Wr!1u8f|Xv<2AX8wch|L#y)_Y@zY zrP0uV4v=vFG$WCkISL%3M7`QKD8y2ISa<~eqvNSqWf`to%@5U*C4V;cK~*UqkR&YW zfxx$3I?Y5ijpiw=vkuAEWu7v!U6KtSUkGLNST23CpO-H4#@y1 zH;GsRSng$+o8G@inPVf#-jrkz?`$DA7u5l!JT~^fG;gw|kl*^vBI=?7avXVUWA@CG z@MoBtmc)iLVVhf$e*^X)*Fn26WkJkvu#DdRxZpBcuS-L5mIleC>}}kk$>vy4%~ly~ zr@t>bTXkym$9e`KBqQpvG<38tmybwOorYW{((gM@ym#`qpxeofd!)i+K=_ax{EIv< zWJ3ymOjX#?S3cpq&f28B{H~s23;I5$*=56Nv*t=jD+3>sq9z>^fQ`SzLJxJxz9|Tj z@+(&}vX{r`fHkO2RuVC0BP)rs+)P|3XViL`T5HSy(`lGbCO?~OP$9}cb7N+G^oks3 z>fcmP${8}G>8yROUiu3fQaRwQaJk4vpHqB@SaVQbE`CS_HC3L;@4X4$5%(|Ia5`BKX z!6EyEF_6OAaWNsIfg{>yT>Q?q8;5lLrnBvat z@D0)+_j(0)cwK%Yr9er8{o$y{-1$7{S3s}fhdSD z6-@3bS&$GqNlx6Qr2u0(UXOlFhe61xVd+@2&xi@H|0K8?(}Jg>)gy1Ugq?(C{)--9 zWV+OQS}35Udc^^4epX5hA}y^2(-tw<})rufmw!(7%fmU(?=J!EHZyLIjNBt5P|GI;ueP zbGBH!UYjER2Q8a+icOec@jAb|1Q4et!JZYj|LdRe&z-!rM`IN>08Yg;Om~;EBy2|X zTJCWe7DtHe19RQ3rs4MZ1x5ah{>p;o6}1oMx=j)%cFyYYs%-hSo2(<>gc7N&1$kG< zSo-KE%iqwd>OHt>#LTp?)YVbPF+z>KLut1g)4+aiSRLoB>YmWk&oUxU|9ttJX}>E* zS4W;*TC3y)^j16@+mxmABN(wb^6psLB_fX0p03l@3S;@cyZCtYrh*lOvXZ*0Mf}=C zmOSS~w(Zk9RwGD+Zx0YqX0VXThq$dJV2?hJ)0+~4z_eueA^eCcxtt4ppTQdP$pIHF zjz>3b9_gamSH_HP5LOxx0`-Kvmb4aE{_~unX1Mb>ExrxzJOd5X7>;?WgOdBKFyE#N z-o6n}Z1xUtK2An#Q|%zF1X7V46yct`ZeXdd5ePXq&UZm=L#Xiv@Y-7@<^dRxz_wZ4 zc+C5oclpA7TD4Esshsm2e6e%>2s7SaZ+_{$$wBgPP72dakkG3RRY+-_W zs+P_Lr$n1VyPvuiP`@Ulg6)%HmmbKYAwO~*kO?HrcN{!N37Me^f%N?bq$1t719jD( z%DH+Y&k<1rQf1pEh}y)~f40qE3UJnKL(kM|;n0HbhG63|7_XUcj@ynRu{ubFl+6p) z3(t3)1SfBY(tgEEIi?ev5tZLvg}d^qv824rc?s8F-2r3UxEnAslF)OO-&I4Lp*l*- zHs%?~B9Th&=N0oYz(w`I4tqU%RmaaT((J&_1$ff^9%YMn99D_VWev8_vC8nuLcF@~ zT3*g!ZQKbN^{usIG?$m1(vul-1#7w~t*7jYOTkO5|f;wk;OG-uKZ&g~w zdr#&_#QqatvVm=R(}mzvWPhK9&*y_nZ4|8&s0))cNDzi{RS5$RstS{fIG8#R%dhrjuv6w|>2EwA1m zzlITPNhWKQ!z|6l+7aao@XRbY&Nva@9dgz;KoBs*%e}a&7|>u_jQp8~-Xh-JCAS92axftFU=% z)s3!hs*kxpLICLv;etCqLc=fb*ruSDUR+95CJ&-q*YG zw2H@~%_oHORCZHg_MmH}%x=L!i_2gH3eBc0LOv?NFd_`J zz3L&1dE_*5s?v-zxmxvoJi^JNOeyRIMzY?ym4K*yr9k=uc{w7kDQ6I@76ep?WVKpQhOeQh)~pxk&~8 z%$0|QBBs_aHA;r+EbOl?w%aBqbeHY$O}w?XAnPa7b8C9RhCiU(ak(DRg1TU%={*)Y zeAMu$5r6sCW1y;J^sAcTD`hG7lVE_wT>zch~9%PnczyCz0M`7tsU38G)XT#V9EAwE_fJ`#- z!edz!JsXc)dPNkpV~IXr?}|cB^@-XsdwVL%)K{Yc*~eAirP1?aqQydpcF=S5_*(}2 zHltCTLXE+7VX(}*m|1N-9wSfpjp4#SVAy8sKGNy=6|SFkD(F4uZu5D&0U4sHC)0;kzr8MZ^Nz&F%t z2S|#$KX8{+KXPKpb%exVP^Ufo4%$#D!7(wuC0qOz(^71brq!7Rhr z-6ep&^7^a4piNt0X3DR-bzX|GeiF^)CZ#$OiSL&{f?(V2xY*I@% zXc<;Unys@E1$$)atKpHl62Jhv^PAd=OG+S+^B4ce(S(HT_bZcO^l?2qeI&6MH}Oew zuMF%}JF08Or}2hK-|NPD0}=KOb6H0!pL^xK|CcuN$UAChM<9_ZJ?eh4zr6@`%0M7E zENpbFHcCz5SdTp`^YFy$j+lNuHClTg#l`6WtoConzBjU_+^7A29zzve&OBWG$Gu3* z^9R~)^rXd>3#q*#T_@)Fgdy#;$Cs~z}R9v?T~z5y$o zJ0NILLj)wqo$XWC!W490r#9i_qpHn%)kQuF!_&(Gwp|8>5qAqP6BOky>WtLKyy#+Pcr9@~trw-E^Q z@w)1X{P&(}Mc1#j6W9qtaAx)caz|nr!w5zgte>q~VN6W^|!a&<~P^#R$ zA@tby+40kV8aoCTQ_P5Uk5P1^Mn+#c^MU$uhoO{rc2qNh+G?wVex$K;?uqY)CCrV_ zxFomfQ+qbqDt{{h;i+0= z_#`UkyHpcL+C1zg7(&nYDlNkqDtDmj_4f6=vRIQ+%zhdv_w$EK zWExXdYN)DtX1d{m(>Q~Gx@jFR(}ono*X|w-YvAuA?rVg&kmkxukxmmkw|><;;{^N* zHAon^HR%tk+FV?DJhQWPV%6#e{2_dXuu)HFm?Nka#L4qdDa>Mk{g`fr{;i`12GtR5 zL)N?k#^s%AOwcD1F z4Rv!;(sV+m!AGJe^c$??uG&|cVgPr4r@{x*g8sT=RfQLzq3f9vLZ3wLL-eP&k(C|3 z&R>}GC7a_#W`efuOX=ObEBN}N>3JjMXq{n&?^*@PN==l;M;z8~3f*X9B<(@x$j?V~Zcxe+r_GL`IzGpK*@*lhN$w357 zNC2+RATDYz^C$7XSC80Lr8Mw`w`;+dNtZ)AG6nu0(2YIC*GD_F{Bv}*QkrB&+fj$J zM7$SL)|B?4u4z!0s2lbKx{#;5mddHm6A)0N;*au%^RDHNALSxf*oT*n=gP(Lb8Qbq zcE?-=PseiA6zu2zHG^K<-Ov)EV>el-Qao%=hF1jBVj062 zttkc(GIQfPQdZgRVpv(9{YBlTk~D>x;WUnZkuy}jbAV1G@P>6S%C!j(n4_*d<`4yV z|6Pb-CC~^P>JW`CZC17(ON=_KrweQJH&is`BXC&PJWWHV=$eI}ZXXV*p9*3{abI~4 zd@^r)w4T-8eW~dLq%Cd!$SdLZ;P4)iqigQaT&C{@5K+jKOA#yqTJ}Dx?3*09hxJiw z-#Kj$KV2&g^npa(()$;*bI|>VL8Pv-a4`XvxZ#+o{#;Q(9-SFy83HjNy${plO4csz zyq8?Rbp;}9P&9=WUVq_{nW_=>=LWkGEFX5xYjP?5pK?xW96xiCIt zT6XhEU8JqYdGhwo$Tj$-MwOkl^xh984lHpKb#`GIjvlW~tkGn66%gwHx!*P*{#wtz z!nDh~jE1k3rfio`pW8d23m(~bK2N2cY#R=qTKUkl@Swc;Rj?|v!$0O}@vv02Qso__ zhp@p|#-kgyD5b=X*ilgqceU4l6MV24OMPmU z{+B|HiDW;|th|Y9Y7uFDa(`g*EY!@P2#5xn8*Vp4<>Tig=V-Epo7XqKG=g{f>J`NsUwykCv{Z}_C!j?pm_I`KfAx4B$iS6BQ9@7eaD3|UHx`Xy$2L93UZph9N& z|74Q^&8dQSMppj!e7JJb{vI#2*$sH=o^1AAfH|%vfj(t5J<#kw5425#ScOj`yM@Ca z9ELg~u$jN)%;}2NR(bYeRHe$fhSbYje~#zr3)02(pgU41=MdcHY4QlBdPcQODl>Ry zsP2H?FWT#X3osEU0wV(v6zc17C>LGP$T=khT_UQb_z=2h?`p;QTZ(9I0txH8A3AcX zruc58hxxfmL;_iBo}45m1x{jqFa|GJAqO+@S_F3vW7Kd-88|;z_a}Ia;WN|l%N$!{ zIQBZR1r)B~E~&M`l{CE<$8bA0-R>rjDE~qotx??V7H24+-}MTqwiL3|tC92o_}@k` z6;8eukOh5&blLHZ6a(U>dqV>fjFYvbr&qsjqv)=xtYhGYzI6*$_jAK%BXlL|G%Hv+ z5bjGen^^!Zo3Z(uDgpl7Mn17{Ft4;mu6$U^9j@chPgLL+a_20#g*_za{_y3ac?|v$ z`@pqf#E!QsVD*qPk>0AcRfBfN=)0#uo{bl1@SUzm8_qXp`C8Z!$rT)!IontQ-G5dy zq-4!MObL9zcDT`<@u)PIAvbS)YRxFYmGQ@Kb$!pf$L#u{;SDOlGBzvMUSVp`!e%o( zWCO5<7Td-=5x_#YT zb4<;TZHcLt%?F6N^OpDk>BqXWp>zd%;^Da6PELjo*rL*jp_qDnGSkzZlM zbNU@6E#CcG!6(&hO%Z}C+>bFWbDHCrf%3Qbb$aY$!*ON%a>j7OXl$&lZW_XDD$J^U zDhmNx7MNd#Y)07|>`=kjb6pwge@$h`iDt{eG+0Wc+iMqeNa&Y1K=PH9#mx6u{ZxaQdk)PzEQs89ZtVVU>ZNtw;{H4Nc8XX5P4MHXiI5@sKq7cw z@d~Y8U^4>sN8nZ;!QZ42-xCObS@8< z%Wg|~%}sV&Vk=LPL}D-PNO~oPJ0%oE6n2mUdXkt7ZU;gu-@V7%d)!_~6#Q5*5k5gT z)|$19stZ+wkNG(q8|uag-)CU6%0=jVstGi~^FT!|uEii9Wr{1OAl~yD#5(m_$?mYXsVG=WC`?L^LGW~50F|ER2>Z5H$v}w`&_Beg0#!+;k zv|#>;i1D&FFk{9KsN8dBci_F=HQyw`cJn*0X>al)xPZiiAGEcE*d_IRARR@k2!}aU ztJS`C9Cx8Lv&V&oSMZ9u9td}3>OO80z2*gcne#SA;tI4xx0XQr-Pp+4U}dHg9k}k$ zm?vC>Z7};mND63jBg>n>VO zZ%ON1Hc7dTy1-%aZ67zvWvN>11PJe9QFXpF3{@{0L?(h>5dl3QK+%zgW8ABSYv_ zGF#F=5UtK}1PhOB?=Z~Rrk}Igp|{bGIKSBNN8ReYj#lk`c5X*nkZ`mdHT(cyg8`!i zTc@qW#{tphBePl0$qXzY+8T5~ZQIAsPCj(>+5xl}4#yIBE0xS4QI@EwHn3&r7yo}? z?msC1#cT?s=5SrRpxgC)>eI)o10_bu;v~7cSbK`o2@F4Qx$=E5q1{;-S_MhomkB^1 zS{TnkC$s`ge}2g}~Swji+SED{>zw)^HPH zv`YW>k#h1Xv;+=o(jP)078J4C3hX*XzZH$$c(_s$Ox%oRfNiSs;;Kn=Y)S$%9QdS8 z7ybE*9P(racF3g-4lH(@DfIQXqd!H4ltW5BidJZNy~ z_m%?4c)Vu#ATg`7W3H|0U{~O*8h5*yr#!{>eSAfxT%L(Yeqq&=b8YaR7itZg~HCpy4ASZoP7O zJD$%^g)^^jE@Rs^ly-WGqJW(}Ky+jH__RW{}4d%j;$o-go{E@nP;TNwS z=DC$7ypcMuUrDxg%}01ze&pC>u`*mf6DBNRr@jM=$H=MrrDG*tL`WekKO9#f#Q%Cc zVPl33It9UQ;6BtInqy$_VGO%#D8iPS%#%a_Ra9^ zcmZK#UUF|-$SQRhbimwq*lY%pxp&Ydym%Fgt|w(f!BHeWi4WeVCaX<;OZqQZTovZJ z8&+JxUloZ%X8#$@yLupf8=$V>6V{pzIN)z;YNOKAfO{XN+#YZT$moA7Z1K?f2Abd` zL<1WHoLKo5cUDrpwR*W@#!$-ah~XR zhY6_blv^fg|MtbTwmbG)0tLotnbG}Vc+<-1F>z-1|G8#fo5a_bMz`=78bVOkPI8q8 ze|@Vwns2ZcI0#FGw#Ziubx{5773AgSYruIgdJ<}A-#rbhUXX*A3R_4PTLDB12HQ{V z44Qq_aA@m{;qv#nHx}V|_-!GMt+%}Wm5;m^Y`B;uTSkow>k}FUG2ZL2-u{6-AdZ>% zr|=Q`q368Ba_>p~y``(9V69>Ti{|3{Sd=FONnm4ASL)&7k%?#22{idC-Zn`*Q_WM+ zCo&ldZrm}WG#<-o&0$R6=jHS%^UV%6Igm;S4*FK5XFUWt?5( zhsh2wALai<;JjA(TSD22wVEF;|6T$6Z{oitZ_Z zge#lnue1|Zogl8Fw*y7?Tly&kW_p6IOZr7<^(cA=G|J!a&DhvDr0kJI)qfgXh|Fun ztB+wHu$orm#_8kY#QUqX!gO=71rlU;UT5ejIc7Wh*nR9)PoZdPtk(H`kz(js?h5Ns^Z^`|Ey>JDO^Isnh|MuiK2xX>S&sc*5331e;7!8f#;nSD{--n z)+k-o0O4xeLKLbl;G*clie86_@Pw#oHL!W;SM`6V6E=y^4I;+`mvuqc@>Uhcj8;8y zV6U;@HxW3qz9cfSXXBe-X$+A!r8E$rn-G~Q9Ga(Rt!ac5su}sGmzxM{iiBIH`5U4r zHYbd4O~Ud>6~E|7lQB)t$;5-M-M9K)n{jaKzqgjAf4ORXwB!C3FJh?WV11V5cil%Q z!+02UIF9vGQQ178g*Ave+^s&g^721#DG!9*WaM9~L44o55%ed|)ftTO#CB_uu>nur zl63!N3hSFPbbj>$S~$3~H#pt*(?3~)S7G_qL^u(yp6}?;RQuu8Q3;iOj}K-axB5T+ zW4HRuBq>U5+ko)^wsYgvIE?%W$8m+}?Gz~+giz-fZZGK}zzXC6 zb7@~Sjl3YacO0(3N$Y$RFJfL3ag-tS&hM|6=PfBUZh6Dn9q119|DS859N)|pQR2!s zLW%dU<;xxj&-fCxXTHTt>k_AOtMLD8i2lbY)^W!-aisrR9J_H#8rp|rDZJHJBz;6b z245JSb1gEsZ{~Q{>pYvw|35~M?>Ms_wHqDcqY64OVea#)7IdKU{+EWG8+FV4sywD8 zKHJdx|M=1VZ_)p6c!VEh(O=qt4&?H<|DuIhk-vCAc4yC%E{blUw~-u4X znTq~fl6&>Wo5X>d`R982v^YpUyM!Ae-%`r!y&LQgV0r#HNz{^3ou}~Sf%!XzNI*u9 zJM@VLprQ`d5vXtg>OIeCcoYk?4%j7$HFfcvac=E7dTpVMOvMRW4Xyw^XQS2Ca6pPD?Z~Di{XMu$j5MM3URD8N^Nx6CmxTRL9{C? z=92_fB+^l3`a#9;YVKa|D*=v#Al-0QNcl|5KP&`A1`fd(qgb2$lAfqv+2)ap)KfjN zyod5&!)tgC$V|jt#M;xoudv9wz}TSnu2`0(v23`ZWL}W(E@g2aul;lGj91*NSxbnl zbdxt-enN*IS48Mi^yk0vM!qTnDF!S^P%#I$;`R85%;&`V4X^Fd{*5~IT$?kE-_F`c z4b4uL=l>WwmWy~do*ZxD(^uSG`@k#h4P2kt?v@~{-*i$pMy5jj061i`mp?fzYRI9> zu03M1o*Tdx5`aGpJs&FT%jV?twYs|kLlInAdH!8R1^v1#6?j5xy{tWtdDI>QTY;uo zzr%F%zJ?rvLRdCFx;!3S#t?zv?lBuq^CsYOGH^ z3|jg6xy8FIInSUbiP#`9p(SO`v+6b=n;ZBu2uV+=PGI8Gau&rZ!3~U+0y#m3*5;^I z$7Y*JoDcI=nu3p5Kq?Zb^}KE|#+A=$e4-crFF>&_9U`@4_!RLICP<|JK~|%rRI?sN zswgW$3>G8B^$+UdJOo+0t;Iz4GwZV-RRVvh`a{g(=3s6N^hJw*yA;%RYmR)bMZ(-L zh12e~nP|dwOrq}w0#L-di0+amsNy%v%8l`eGSY`&#ArNtupxz*q4?R+!g?nSKG^p{ zC8LApu=-Q9U8xKtmIfGMYFN>5!%xN-oZG-D#af9dFygZA)LiQjVLdFRPb~QvVUcnW z@@=l8;gh(An_xnre3C~Rf!&{n4x{XGJp*r1oGeTtF`!MGZUMsw+yJP3hbpu$eKs4y zs77`i<0Q8EXmyu5`1G~tThD9~fU~ka0k`r@$u;H(H*8}j1j18-hEVEvN5QZ34eU0( z*Mr#WWWnEj4}!2_E^z2gD#W^e2?Hy`*OFQxkFd=2QkEmh_mI@%SwQ6=cxDM_=wN|k z{|c_0O&1IpvX=4=!s|7ktg(y~iD{^|iwcFn@X)rB2C*fbF7%rXyS*gCgtmAN#;Y>{ zdvcA%A+|6FYEasg%^CeL8^f2fJu{nJ4QOkTlH3;J8$pX7uRTsZ>Q(u(!IDsAE02aN zmv4NEtK>@V{qt;^!=Kp+PyaX}^<5@GbI;-Vp*qde#K7p6w7<9HqSXksEk?2mdzF** z^eQP5AiKNzf|;99aY4E|qf1}XP@6@!tWN_DaZDErpYqP=gtT*Cg;QEA?PSoLrB5|( z#gxO-Oh0x;dgN6HkADFnbEbB+{of&^cBN8+vi*_tx;|`#l94%{A6MpAOnGHOZ>7-> zC`=Wk;kXBLG^|Q9og}O<8iGxLk%Y07SKrOlNe3}AoR^u0cGpiRs%gVP0^0C}CEo!u zZI;bOHUU0M-%xiJ-h3iR;USeXfBsM~M);)2q{MU7-l;IaL$+`l;{KSfsJj zS%um6Y0XAXCR61W%@&iKcG$J!X(X-sLZQFmhJSJT(knJjAkTCC` zW& zX55YPCWyPe5e`SVtDQlfK4+1G^YlxoTJFbcq}+z%M-QDod_2`;--Pw!7QETa=$>3i z2nM>siiogRo7XJdRH47s&>iC_rz}R)y$9QS>}}A;Z@4rY>kBsjNpc0)|HCWVo36)Y zodR9boqI=O0D3N4GbPyEq&CRUYunhJEGPTudI5)y~$1 z%XquZ1}fBOF*^Es_r|SkP8X}xEE^XtTQUd`m*Opw9-n2=0+d26Z-wiGP5(5GkW#R! zUi{-(?pL5FOC>Vz(q39Hhr9VP#zG2GJvZ0hcZdE`5$&V*|Il>q@l5~k|F4w2ORQ23 zIjj>ZWkqewSf!F2);ksDunJ8f%c)_TQ!F_wMiPc3^%inIn~^ym=8$7-j&sV!u+8w> z=X<;T{@OpUy-oApuj_hVkL&(;(Q4sz#$ZsI>LW~-AZr-w4-OtPPBLuaT7ck9 z&GKGB?$tp_k-zW3L;9b!BW3KmT!!bhS7F7Gq}}3qO`%4_B4$!>O2N;snH!j;N*$yY z=WE!mjStpj6HLID8t+!rMgeUNBUHyYFpUTO7M;8dg4k^!jD@x1JY7$5s=^og~wHynAsC*I+aZEz|?D&+31iO9O?((}n<#@gs{xN=VV*AMyOGGn&f1JUZwi z?myZWxL7t?RthvwQ2Qd6RxKQNuR1ah(@aD-T(g0+aKizhHb3#bpXy-UxUJE=9(yIf zD%R{&94#h`vNgrxvf8^CNSZDel1WYDy-sOqnq@BBN^Rlb0 zcVubUozjifw#8xwa_p0PXC6B?oW$`SbEdFwN%)1uDOTf4T-TY5;~+ErvZ{iLxPa~> zIR(kdm=|L++#plba-E5W+k%t4?@-@F=gSNH77Ql}s4yauP#xu+RCZ(BM;9(HZH+FW zI#H?z0$KUs!py}M8 zx#W;z(_r8le7FbU&o?bscwZkQ@5227Dgde~zzuf+!?OJRA|$wJn{jUlLr}rlp1l?| z!8UdqaeHzpd0Me3L{mhT@DqGZ-Bi4XHh0OqUed(Nl5p36RRuRxQbiF&4|wyhlbhT# zs^fbAB%#H_T?w)2_~@LWB65tqV|@h0Om~hOO78G5@yA?hp>;VK*W58CuHV>eK5@8H z7`PvUi*Z@zA2j8cyw2KB#SRcNq^t2$k=wr` zF_Y|Zr~UV#>a|KP{O@GYwtq`+JG6>Nt{>E=e(827u6?iTq$#O)&N&Cx^GyZ%ON7B zO9Z4G(`}`M^Pa+9*KCY1t2kRvO8{}Yzsgw5co|8lQ%`52aZM&;j~$&*qFz>ailLyV zVwV;zT%KBYzcJ+RT+7NsLr^yeRsJSG$PCq@KDYY7e6ONHXkpIkJYjn6s@JI|hiZ(y z2qEwH#!YFS2iYE$7`JMRxP0+m+h?J@jEy)ej zZJbD%yfx>TC8c2NjqzaClb2_DjrEVZv5TF6Ww1xE=RlOwM?a5(&PPEvmFzy#pmvk9 zIgtS2V6K@(!P+9fZf7UDInAYD#TYmKaQl0GC&feL4rRqyczu!ATSz8+@z?Gy_t6^? zo>vv98Ds%DXmHK%Qs;78`3+DWQO06Pd;l_4 zm;YVnoPtuYi}=qeKn~678bB^r`w^~|GgG+G1cArV)V#}Pznvdxaf~sMv^7z|`!#58 zcK2-o492o_M#>xSg3~R-k|?h4zzUU3p_WQVix`c-Xb(t?sl14jSW^2v8d>rc`mm#L zevD67lCJ4v8QY|On=mDLohzX+UPJ(sg$EI@AX8r?Y7Y7FTO3>FPm1cv*%N0HV#A3A zZ>aEq>j}V#JXP>=x4wT@eTt|nu!r1%=`{7aj*xxGKLt)%If#X4n9j_xXLo=N*=msc z(ImezZlh(LMgT1v%$n6-b)FK`Xv2lfogYLC-C;nz9uTIU7UJq_n2?c8Vg~nFu_JUi zE&O}no5AT^S+?e=6bh2 z`$wC-l(|l%B*Qd)ooO61pqnhk&;!RYf34@sx^4^={<74!_HsP!f|lC;0wDM+y%1Zv zaFRt`r%+6(+a6yq#Q^;eXiGi#7A-Zmg`sX~mjruQJY(gd=4Xo87P@cwtQ@4iAyGzz zhv0wWxwW#9-~s^B8^`GXYT)N2B72`RCfy7UQ6J{K6%R@Wyf<-QWMoEc)oP4jYg<*t z7CSv|;~JzzrgO7qf3!vl8?3r~!vSFD!JufAmbkv3&}TK^xFn5_$;xGexfYF+Ej_?! zN5du=!+Z1$Zi>OYrWTFhs+1?m?;<{6bYM2){+H*U<4RJA^IQHhA~Y z%PUiA=()xx9?}Bd**f^eE;{;$D1$b=*nCDNEa?AQ4As(!#6-YFH^*Z|x#kC`MJhdA%IF2jY9ylt%VEhjb@= zmZ)iNJgLIoo2z1`XcpXG_03Lkzl0Pn`5NYPDDvwX zf8`4g(^_2Hvs_i_pQ@O*(;O2F)^gPS)Nd5JhqJnuLbA5QY~G}UFc zt6;d~IKeHYqgd|UB~fO%tv^>tI;wd3|76&HK; z^qcF9Z#R0K{n}UDD24E13T`P9mV2b_%Pa$?R;B|A*hUZhPDN}NAr&!wJw1<>B@lcg zbbyer;R<_^WsE=gH(xoo1b4_@Y=`@>g`m68>h7ARuiAns*6dH141{ip=}J&)VEe#T zk_Jq#yk~nBcREW+W%-Qi-L;te+h3J@AG3nU!~l$wu!ZFWeJI>bIgG?Q*4+=wiv%A% zV^>s&vPi%%OlLD9WkC2{Vin5|1Hp0&CYA&kaje}Ll9BvUaMiB+!{4FWHfe#44MtSU zJ*8nFb{x^KWfEnjlP5zh1A^*ZQr|!-*|k|1o{gV5NL}h2r3W+_R63JGQFh~Z=Wm6O zbB5R%G8A`4whK@-8iFYs>e3A9QwOv@zzY6VDpT%sj-UM|O~9W5B$dq_qCQVuV`=8& zTY`<`P^`>oNEI)9%i02ZX;6kVihc0=gS5Ebx%qN)7h)q8kfl8|=OL`O`mNq4&ukHy zsgCem&qw60%}vs7kjbPO)KfH( z7(GN|phJ@XMuEMzf;2PzOIT-u&>^|115O`TL)3~BO0z#R6prle1)QFzDW##RL ziRT)F&LuO)H3=N{4FtAjsL4_G6TiL>97S%Ktc%FfBj0U7X#uq+t<-=~7)XllU z^hl_03+5EOnHw712w!^z-A(}L9cRU{-owZ{`*gIO7;yj|LcC3KdYj3LkIa2>EbsE`GJkCSNFnd)(1sI zOAi%naFN*Y2T(?!OTs{Ce~~O`uc(v4o7@-_V-cG&NMrf9?idnum~A}sc=EgHp3};F zY>1{9(3<3G|LJ-$833YqzEFF7t?N0Yia1Cq>7TeqS8~GU}I!nIKpVy%fKf!Y-7y^@p>Ap~}3W|5|AUmzA}b$q;z+RVk`A&W%GWN=abH z{pD*;T$QZwl1-YuqXZ96emuo=Hl1^|B-TUcSgo@upx%i*+joXr%XG;{I2!v@s;FyP zs-#zv4x}jM+#i%ijZK&s;?AV9K-BnD5E}Sw`|U}SW2OrpK$o^)f+&9KJ^4$A~y?%j6Ym$!#DR}H#XxUIxw(7aTK1Q8+r-PpIC9iRtf?jbgs%0%IT0s~Tp*vOO zl~r-@FLRi|eZ;@Kb|37Z5zsFt$1cfwpL}W}r;*&o3)N(We;6g)J4v*vLl5nW2Em>S zPi?Kff|UT0;DzyjTZZDM=CB$G6zmb8>zypf^f?oQj4x7SFbR%XC;w z;lfm9DjfYlWc$^Vpk-4&HvKpNe?}2Y%R%*}WDz!@xQW5n0iEkF3zl^riBCw>Sei~5 z5O3`sBc;s(c?;Nc%MXnKb4OLnE$yqK^dT7yA;{$hUP=KWr6PT2!~+ZCy#cspfCeZ> zRP_erw|OGVQZ&iW+&G8UQP>)MAMp9l%;4^hn>_B_nRy;D!`yr ztV|SdmXRDt0l73LDQ_K@04;ioPRXj4JDw6lgK3t*U_ia4-9vl5GMN3`UE$e&D@Kf3 z_(j4OQgSX%uCrN+ioJ=LcO+Xd^8u{4QBQ_9FJ;sPJT?5VC{ z+^m6>X{*FdNN0Iegk@{RsEoPpXr->s#sPpbgJguE;Kk6J2cW?C`VgcHfoTx%wbWK$ zQFwWyHlD`4BWd~!Rw(YjFeky~6}y2X{)El7JaBP(da1Is*qygHn@s+}ADTd{y$Zmr z$nX0{JucQGT<8CUO>4j1S4zD~21^$D8j@-SABfSi1|})J6mGztQ^sxibVx;r157{_lbd_3ym9H-YZCPh+yn zeLH#y?!VaY`=@XQmpd~Gi2<)K=V8`5mX>qT)U|o)tL%v;<|UPcmFjPzG$NiexMHv^ zm29vi{uz|2VoIe{kIqwRgITz_EH~j{-R2WWMHJ+-@0L84WY2ZRT4?B`Y>IcRg$8?(8iZPAVhXKVQNzIsSB(P}8lj-i{^|0B81w02 zYP~)M-?>g0gP1JqKl)x9-kGobaxLu)mW7yOko5dh*YVWCeq2ufsEh$>@&?LvzI93X zcd2}8Ws-ByPWoG?V5C#ue>TB)un>(eL65O{^;U#FD|ZNOMyjF?92g*Oed}%n0!3+s zR)lz~KcXg$@o9hR(gw7OH!f8%rLo$%mE8GfeBN$s@sGP8+71&vsFJMO0IFuMw^En0 zlU#u(o|@QJ4Odbd*9J+VJ*2U&f`{U&Y^50Oz9r{g&dR`Nitk(q+g3Xdgzr^cU@>p{l+fqrA=Epot?G6gyw^&b;Ut=3z4~gQ(mM?E~1-B;g<7Vrg z6;1Cn#8YPQVzPo6X7!>Ds7k?MpjVZToR z))Z3(57A)3U2wJ7$f?a#sC+(2P1GxpOQF6Kut|b}!hU>p|ClgRYg+CCcs4c$)OE&m zqeeGRW^76xl4LV)1KTsxI3`>i8+7f@&Ea3^To<<5x=K5x#$vH6CdXbb2z zV)s@QXH6k}a|yh$?6$Sd1PZ#&Z$%w(`E=?7H~)31g{T*~-BuD-`PNxl+mMr> zWc8b|z8wf0`AvY9Mtxs9Mwdz4o(~8Zyg8p1dCsAea79VkDZz`%QldT|?y*jYe1+xA zT9jt5RbT$2+N+b`9iQvSJl}{KwEhYNb%#cIY>B@GznMbuV}be`yiR`GV&V?z&6PZD zMGlplRO3sg-82lUr?#|T6)$yawE+~*kPm1!Sz|o`PCPUX7#KU0dlzh{Q%|&u9yJpgKdq$nq8Ijq%bVuNT_&c!#9TRt=lmy{hi-qhi%D0b@XPjbU zH<8I+DDP$bR5Ghp{$X^;ha0ooxW%2*^sBM((PjpcH31-wK>a45gHQWl6R}anl7ve_ zVjF!48qiw$5@#SgJmD{fyoj$j|9kZeg5&-w9nsY?Pc$;L`#ItHH>4Trnd?fC*MHiY zJmcFjNXmCmWh&*r*bkwOwr@Q(^=p|uuXu8y?AB|DjiueYLSvrZqs|sAxBmu@{@>#n8eLa{^xBjs6|MYNJUpAyXHaOHj-~~PaZ~z|=f?_^5pp&9-_0N{u;yaw*sc;a&3B=%#{lsOION7qKa4WS=^W}`fBrRqL{MN)J zTW=K_FkcF_?->6v)nPPX1X!N^&q7^<+AKzm@oQ%!_^W>s@B%B>mC~7aj(uUo&4lt4 zX~eb^wJb{7gNN8;iYAepJo^7O4hhKc;OxliXF)269FF2XN1&tfss9BueX*K>@&o?# z2h^{Lo|>~Y@9>=he3V;R&m&4bHruoavPKK?h@p2EOXim*@w3|^)G!Q?@HeNhgJZZQ zYxqjvGhM%8^6W6e8o?4H=Z+y#Fa7>8#q%XY@r8@H*^hR#+5Z;a{x|W@|JMSH00O=j zTffZX`KE5~vDv<1IR@#N-6@VS+$5X*1JD7}?v%wLQvrPxS=CjH54Jwmsh6bVHeXfK zpk(j}P8*llIw)2FCvB%(o`Pbq^xYrJ@>7w&O9{iJWAB!n=NwiacNFu3UQ#&`)FJ@_ z!A1&QT3-uGs^wtKg_nsxD-i;}|pS05N%sD}nA#H>p zs+!R_6z`VAJ_R2vE^h9G}y%e=6;^j&uUQU&wHnhRj@x#GSa+=*m z)9!Bhz6?%&*HC8%*6l^;98B+fcL}4Cd7h>qV}g5YiPsYd+fR&PCfZjaD$8WQd59MYO&> z)8WZE{X2S!-F$&EFl(R)^JC5{2HI9lHSF883lnvSLtoC=>T#Ip*Ump2H~a+vZ5B81 z>86IqK=23m<&x)Vg~4Z!WsBLVBN4*#V{c@XTaDdyGpjsSMD%9oSnlU+3UJ3v54Ho0+(cSJ|nt{NqSMdEv_Eg{?9 z>Nn)^S5@`j{kweBlgc-xRy8-YuDtSb(de%MxNEbzE}{an6Mm(yTTZ@D2R3h)r*4Nv zjXtUaVtdS=W*bf+MdOV)ZMbC* zC`a4aS708tC)l2?cEO_bi5#3WsG&d@*9dsgLpFmITzpjEUwcd^WlBzFjVrNHb;WA^ zXJEo!h{5dXy}Ko-QgZgEsco%WrBPwgSo}~{0R5}j-VJ^AV>{74vJ2$7o*u^{{;c+0{8QT; zxKW5$Un$i(#E~AGlgBR|XgA#!g&<&t1q#?8b9v!x{ZJ>_CQ16TO;cUHlsCMfPd)LR z;Xx>SQVyAzB@MO5c8!H9gE#ALfWrm?Nm1oQ z!P>3q5?2^lqtwS|R_VK`mg~VnFSUNwn^!>=PqX68!1sT}T_n0M%E3HgHmC4cWb}XS z3K&k73t2@^oKW7p6VPp}B4(otIY~wUv5^3z$hl4W=>c0;@8cnVAr$7J&7=Nvj2Nr2 ze8%E9u5@|Ab8@*|Vn%5t&PdZGKmS-sLoQdH5+&udku-e(0HSM*Xic&lT#t6POcz<< zGOb4GABd)&AGm0ll|zkA-{~rb^r(w!8a+WvC_7yNb~rU!G039BPUKVL03r!IdV z{&W-Ql2m;AWoOrDY#Y7AAjm*TG_QbKDERP5Vi9lqONTx?VB0n1j%Zh3t8-tm ztc0RMR&-R^DsP=%3#?N>e5=2Mdvozbd5bj%j5tXm@FxXEscnX7fbC@!d)Ht6QvYn7 zmz+Fgz{}cdY)t-69V%oL3u4TLJGLET)9uH|*?-3~%i|EuDfpXOsDVySYI%>RtX*yK z40X-a{!4ub0$3fmOZPOwruaD(oG3`MIH0 z4S(eU&OsO->ud$kATQbO%RD)}3cxNl-$Mcl7BpAXs3w%=sN*~~vvfK=!M5cmH5bYL zp4Zh65V%aasOTS-FvvIxnwD}Hu|8r+SUtC9`!o8tGFt0^1MTm0vPydp{iC2+?Ji=6 zUW25e;H&%A2e&qAOLol=BscQ>)_8V=t45LxzsatRiJZUXIer7K^&$&^|AL{l`A?rh zM>8q??HX(Q=5k`#(FWTPtHgOxS~xH%J6Z8#g8k-Yd(V|UiQ{LI23@F6YkoEQhCy`g z2R?QVD|NLJuBLQ!99j^t9-&r`5T>mvL~Wg0BVUQ~7-x}nxcx$Gp$Xz)YkN}1quL*c zx&J%E*ZeJ}eJ$xHL=bDfFSMCe<};^kAi z=C>wXqP!mp1Fvb zl_*{P7d6+vy^Korowo$LR-0)M368^s`Js6WiM1Syo8B;AK^|z7DLuyiy~6Y@QGETh zN$OceOi<0%;UupwdHDU+0|_{oq^t{@ffHEarX-QjiaN^#va!sl-Iip;1Hptid3ia@p1N1&v1Xw<`@}-CWHGThgY4r}gtJds z`en1_B##(Ity+Yh+iDa0<{3{&^+LeL(y;ZE6y6LJZpi*cCNL#B zfmr=cHKhP})i&CtSuOp_sEC zn~)j>SQQ^7^bgPBnA*q33d5yEqz_w;U-J}rTW?U*Z2$q&b4G6=EH8D|SYyLhySY{; z^_z`p17u`WvzBH|aV--J?14En+&Hnd1DvUY+t`P*`+SN@;7Z&im?M0;$cTZwC~9@R zqx>V!3pg;V`2w^~C)GLC@BQMnJ@(Te{%;v=`)%n_OhksLul!%n&&nzzyh21;q5LW~ zBmO9w4{(XPGEDgF1N%fKH;g(SPICP(MO7=P07!C-=;xg>O8Nq1uHDzr<(qcg=P%H~ zhwlVCc$Fk~@A{xEgH;YQg*BZANf^BxPmI31)}3s$`y1eshJoPBaD;WzO^l4!6;->G z7lBg8bYB+jRn;X0)XA3aJvg%K@wKZuDkI?WO<~8cXC4>-i90Xn_-Ru5gvuwpeeSCw zDS-MuoVg6-EDwCA=SWU{)UGkf;hVB4B3Z;N*@WG1WB#P)JB*MuV%BuL=yq2#Hz9DZd?CjxMijzoFRA50+RfEf83l?zOYAW2D(5$%Ln|B>sp*4AGpt zzU{Cy8oSUGzGHu$#K7MDu6zDD_VEdz_H5wWM3(e`#DUsffo=m~@{p@I^mHltU4`<< z-u zMVSVjAh7Mmif(=JmvX^hBTXeMiFh}G&LNpLrKk05R%ea3=cTG_l*`MF)r0*2+$w`TOI)HO=a z7>OVvCz&}u2m|r=3}ZM6qVjs|CA+5=Oy)IK>H43F=O5n)d& zU;W@)H}d4##LORpnuMQRGUKTwCU}O#x@6Ur|LS?j_3Af3uVQnOCL&ei1A$_Upc@$u z3^)wC+7*1efaG4KtAd8GkkQE36TfYrM531{f8hOeQSZlg4eR%c0n>7&`NiaO%jccD zLWbc#7sh=Tp3ZJYjkbG^KVNfY(I3PohMe+w(Q(Ip^yfwAzqg5x8*aM_$xDQ#Ro(nS zM`dSp$4{U9k^`1yr1%wL8K-XvosQX#?2=RXoA7!~zH4SYHDWvd}qk2}W*+Ns@{t+%EZu-y+*PW@&8 zFt@$xP~ivduGbLktjaEG_jI)GQ3je`YSF_9C+RwIHrxmgTdv1<= z12zAbLimi}_sv`GMUFqk|3@)@G=J$UckYO)Mm zJa`1tE!bIdk}IoLk{v0XXjA2q*H-m(6TD7%vyMO8n8iI_`s6vOmIz+wSnK;6(9wqH zQ~g6v>cWb~`XqA%`*O3_tX75psQUgCOOiWqZ&$Nn8w$9vRTviD_t$lE-a|jYq2_?8 zaaR4=8o>&-t6WoHY`Cr1~>fzewy02 zU2mGV=Kjk{MSb|N29>`Lycpk2-ujVJeR74k^&HV`|A~Lrt`}wO3rQKBbRN$#xCQsw zds9Wl*%-j&x*2|T0WcP?vTgj&3v;Tk`4wdSLoD&?qy4B?z{WQM2D#&QwqLg!@I2L# zf1@Q+ywlXSE@=I#h^;j-#zWC94{XGC}f%lLPb#t*I3 zD44pw1Pu_Xw=S0y8Oa)i8+P6M+ja$)9l+R}NZk^BqE8+KKZR9XO^9wdY*YQF|6{=t z4EVJnxA{Lr^A6CabwW?l*5RdFi_maSZp&!4tW-GZ#~#iudF-P20M|dZxoam#>8=$# zlhpOqXpzxk*kx+(#>VEN2KL!?rE)KOyXNIwoJ>0e=D?cPkHGyCaNKNU6VVEgDcp%{ z`HI-t|Iu>hkA3PzhH;26q`FI1MAxoZbX#s0?3_{I-9kG~)Gwgw&?>r(3Mrp#xvNm_$s6(8Z?QSK z{~r_$J32J8X5br%!k>956Gj*YjC&R+a`nn*-*sM1l8)VJh|_>lf61^v{$O*m_;eKh zta1OI9m4!|8RF&>oL;H(NU)3BHPr9Ql=Ij1^|@k0`A8#PAp>W_(JBfFAyGDz&Q?r< z7`>9I!jS#US`C{^J!b@6MPsP37d;IpyzdrtT~;ZyTLZ6CMg>h+%)b&^^RB#QqR0xC z18ic4=vNH|7?HslR)7&^ld%@;J=`U$>bkt6$AhbcmB;At=KWB^=Ik*wmj@2fs(tmQ zRzq)%-{FPUSM?NYZ_}@>1<5;57HUSV^2|1iUi*!cySCR}8ha>fbi`6RWUf!GbOX4km`&m zCMoo^x6P`k@{4Fni9MO^6dqC`%;67;k>q;zL4v`a1fN&Y${kmD+tg^J9r|i3zUu1Um2KC5e(P&2t z6q_w|^{0`K9&jcY(+(6jpav?_RQBVaTlwQVMp>CQ zi-RXq)QGlrgwNZl$*c1MKY}&P&t4@lBFd`yGbLwUwmiS?Z+oTaRO{q=`gHYbk(WeOrJP5UCkG-Q<{anYmkh6m)F4$ZEgiNkuy@YT|nPSe{Kk@ zOrY2ak!NMdYiC4JM$}^~2u|j{iwrTj9aXRqe4zbg`7=G;nr{kU%q`0pwMpLl9@4s0 zD190kuZb#rpEC!6kk;B;v+1uZc4Xx7@Ggn`Ydz0OYaDbq8TV6Z62JF27!Fi43V#>2 z@59WRTOeY$@@WHk^Mh$ge($W@*g+&TIJP*}BmXdAxXqgxP0{AMYJW-MxXdS>6W+F1 z5*{Xa-gYAc;qa$kE*FvtAJ^?Q#;KFu}o>=2}){|~n{sVwa!#;US~ zug{~07>`9-#(iH7UH8+RpV<8N_t(n_9uo&GlG5m3R?p}<&2eJ1;zEt0DTk}sUH5g1 zfQ`H{nmQ8nhoi=oe(hCDlWq5yy1Vpv{33kLK!aGVvGvFJs^jE}VE+bYNMqgTH`TBp z9qE6%VF`Px)YOly=L&61BcgeHxaWPPy5kf;Vu4m+o;+Zig)dv^kFg1OP5XQ6 zYfB-Zih-7Y%(`dw(p~W-E`}uiEyKS`*R_9JrmU^p@j?wJK_xKCYQ?wWj!YP07dRiW zDtwpHB>{5|F>ka>J75*;)QOd){_@8aMDaqb9QNca0Fu|oHMA$LDpiyQkch%;ev1{W zg%j$tIy#{!4Flec{s6wqz}?@XiY zCQMQc6M9bijeGHXqF~L~&+_u0f5jEAX5V4!713=VV$%+pOWe6Z_~SekRr&m0Cr_<} z)8f%N!-oSJ8=pjGcC4%LkA~GD@%1UWpReX^2LC`k2umHo6o7mJktRy>dkVNW;zM#q z)s8ukHZzF1u!&Dv&GnsUvLpb(bXh`JWlbU4wEF|P7eJMu-UA!q7O za<7w~&*R8T^jfdzk>@jcuOGnPEF(;m3*kP3VPz;`Sk(?`=^6(NNMaZHJNdn|yd}=S zz&&}2my!BU*E@-?c4hfmDdM|AU+t2K4$sojTS;GQPnlN5ebE6a(1w*I#Qe;azsYgu z=p_Y{k-5Nsm3_dIiV3x}Fivfm%6DwoWC{5j9egRrmE+s&K3 ze1B*-;!cMa&yGLgK2F)%oxEaDR!UrR{6$hQ+l=2Xj|H8V*GR5poIyn2EO~2u!8LWX z*{OZ_256I_%xsFF@_AJ$Mzv@9W#HBXBE|NwL;d(Me}HR;f`mrC4M!GWOONF98Q{am zBDu*yuS<>g-?JFfzqwZjps21$^xHLw^%~C(*xU#M@0y$eirv|H+F&*DiXu6SVM5zP(U*xSFfbEC7NRrlzwH35qS4+b) z_N(YsZIvdnHkK?lZ$}5qFK6aW_B_}gJk?{xJzdJ0j_{N{l2|)#luL1i1fQ?zF19Ju z3g`c~KiM2wR@o5NIr&|>ral~uPbPde-kuZ#Z$_thZE}xr4_zlNCr$4-w|hUKtd1BU z@$Xoz%ai!?jt+y`w)}rd0d75e06(@jP@jihNc{fN`Ov)b;QOA-DnTw!C9|Yqjz-1? zO1F&J9#?5gLe^vaZ`T>Eyan8LvjaRmbJvkm}qr5ZR3usoUXToeE~Z0Z#}39v{;Q1 z2H)2WIq1j?&$}yW>^=K_Nz%bWOiub+nCVrvyl#Y|$K^j}=|P_+1q@V__hsrt7%t(y zy{cbNA{D1xb)YUkx%kxH1*e!G^E6W4^Nb(oG#L7Fb2&P~!ykn`SLusbn0A;25K`MC z)Y|^UevD}U1n%`NbN_4UE^DU0lH~Uj($F3;?c_ZUTB+8a=(OH}c2xv>cs@wLZ#;C+ zjXHQS=~eFym~Qbw;9HF$g#(&NpeTL(v%)&;ebVTCMDTG55&x=DBDV#cHr474xa_$8 zH{#Rd7n)-cpihm>KcoYWmyBDTN^)vZtJy~Ytn&gjuq)u-kFjZ%F4qkgZr3<=XH9$X zn-x$V!VkZuI5ry3i9VYw4DGW$TvMwnOUBo=V`8VW;-nYaNEPlUG_*N^c;wiFcFg~7 z=U?_Kw0h(78(z%ZNrDm{wjo;$QqT@=b>~4MPwEP23Bj6LM>GXqpzjqSF{Ir&?WQWrH=Vr7;&Zkaj zXK?HKP}ZvhQ5vZ3A=GhMdzQBVH}_{c*JFzW;^tNlKmuZyUZ**cYURgqnVUO^^So=* zI2DmWf57ZPq!*JDBJ+bRIO^BkNi%TVG_}I-Z7W41YEDe7^WBI#tK^!?R@T`z?qu{F zrTwQ))r$MmSGvu60V(fIQUl%2FQ)C_J9szPF3Gk#Hy8O%BG zmNyo{+{ldpb?ok}9Q-Lir;|LImAGtQir+5n*fHU7cIC*YSOaS=V0bwid1+Rr%JhWe zk@YSBoab!xXNZ3Fs!vct#usVDu5X|Mf1eOQ*@hF)!4yPCLjZ_B-AJMVV?Wdy03El4 z*C%fEDU#XYdc_S$x8}J7joCqA?!}`~8SpL6P4~^G)5*w;lO474!$zd!p&pXNRiBKd z20e;OYzVnFupv)uv^}#HS~UT*t=QvE0vMFnz9jyCEdb#B+`NO{)_^X4IeRUY;Yh4G zotxKE3^@5!)Bdxm;n6>Pi2z|CSQ?@0#!0s|z-p(Rsku#^%-)W==)sJ4`x@eSpK7pw zUuWBFW0_bC677Sjlef^T7-C6B`bsSE+DPwZL9>iX{_W~jQpsHNv6>S}D49DS_G84= zE2?*FTOjPeP193K)7c2bzSt%K1Fhag3xa@Nn7v>?-!-#6N!#3VFijHh5zKC z`LY#jqYc?ZwCt$M(nAZwM}s@>NP;VP(fwYJw>kg{np-d4?q|{-&1ayg6%f`{;Ezc~ zFlDp&f-3*Q5^k^Z=C!ajOoNWi{iV7c71I_T@9q~NOE+Oj^-nQ^eMN)kHsANdpGfDr zRdjq!8D1ujOQ9{;y7R4oUW6kRc85rP6MahNO`(p$Dk~1qNpEshO_SmkXX#H(>uYyE z$A3s(#`sMm2NWt#WqGOyU;mWR_%t-aI z{hX#NwSOv3%Nc@Io}0&|{IK}@=S@bv_l2FkCWo(uGljlqQgWn&_S77nydxj^xGn(^ z3|D$FG-3K@VSm{lpNZ2PNjj~_{AksHmyyxKN4q&u@VDLGkF z@%&+7xGtQLZqi0i1;Z7n+KbF-4$I-aTm=7#@*Ma1wKx69dx2T<*sQ9S=-9oHw%DjK z4t7b=03+we=N4j{7iSvu&{9`0&Oz;Iv{?i$7P!rDImD3ZY$;BdKk0Vva9q%n0Xc2P zEc%u14jyl(A49yDddz!EKt3>eT$e0L1YL z70{~N5;L9_0yarehSzjCCB$chxL=T)rYrfo?7H}>>!3oo+s9kHBnUr5iQ>rpaZZ^} zQ^Gn?;(aA&d^87l|Lc6Z2f6?k?FmbMT@fU4P9Z4X92Wq+I3IPfqoY66F~uhOE6GBI#c}#8vm6PKbut2X!^Kx_POcNoUW0C^){gXNfajT615V&^E8}^)5Ka z0P%Zm;Wv940617JsdG~755=#%Jro4SPKcew7TLE`nWSwd`kWlc48RjZMQJ5>O8<>; zcao8@8wg53T|O9lvAh=htbTT|^4<;@xH3)Nwy03PTpTw%3V||PGSWXiJj!cBQlIB8!GHwcTZ;$%B;)FUSxv01)o5&n`Fv>?8>ecgmM17j& z@A_idq1~8X9JRynPg(xpk5PV7&G-Bk^tmLgUy?N}Aj5NQEP1}awXF{ll_c9PA=kg~ zFnJ)ym6$lh$lOEW9(#zW{a!5?8Tj|*{@VDz^-ESAy-AiV9sAn14Q=^ulg^suJJ7iA zo_nuFJ|h&=8kLsq+;UEt;y%)!7}a?s7m4*$2!tj<$8T|;iu&zq~Q5 zdajg)Y`x#p2kauCf+N@@s{Y4|uP5?+(97x1JKQBgM}I+v#?RCkn)Mpn;_z|&a)tp#c;sr*ZAKlcpipgXKvGND z>#)`(*6*8&<*`VancJoSV@U0@HCF%*b$Bq)Y2r|&_nl+U3fC_B+!mQnq%Hx$8(u#h z7C1*lA?;jxN;|h z5fZVVUkm{lLCTozi1+Cs$Me7Dqj4eml$)*UU(J0zB?IsqT?|8#|y->j~e;r>6G&c&Yz|BL@gEMcp} z+*c_oa&K-kNs`B> z|JgVFDU(Hd6PHj&g6*EQ#(uIC9Cp4wEzlr;VQg&m30gfLRb=onEYSfJScNy&hmOkj&p zf{RoRMWwCO<%p@X!`a!bOanw%@!QjV)_2VEHm^P3L(&LY{O8tOW$>~;h`W@QW%MqYde&$ z?7uRrE#a)n|K)r{6djHmLZ1;YX!W_Ai-$TNpfRwsMXnp2J8D@HP z^wL32<$c&|kRFC$W8*}RKWXTa>o3IO4EPi?e5zzmO)f1f^&E&_ZHU}T=y=qZp206z ziT&J_VPE&N>+0N9SVc-o9dyER9eCPd6ec4FR-rI^S~e+F4vZnhyT}LoU?S6>`kZ1~ z#Hi6gT$Qdthj~(~j7rm6G6NnY-I09acd%s=P`45dYh^Sup?|J^6jKqAn9U(Hcg{hK zKIAox$slSHtQhRgy53GyxWnplJ`L#<`PocuusY{9lG&bNb~+c(JYnIO9Tt$#ZC8$m5air&FSaeP}Po#!Hx}UYwaiV zx*|Q*n6-W(HJ_IoqdcK?YP0;moGCFOliws1%3a%Y0F3oz^zWtxAlYUysy+XT(lK}O zWL7o3iAOfEc9)gu+PyKO*gf&Ks8HL^tKC(P9G_T5-T3I=ud{?SuRD88;k?FlFH-A% zM*dg+oV1LZ{8OIn#z8&xm;BbR1e+N3UPbgq$xBKKtEfcJ@epVu6y`sgm0>vCtFd15 zKe|9U?9r@du$(mX>HDrwv zm;hu^m}T>0SIS_ew0r{*0J+G>7)g?mEI=OYzU1-=;Xq5sh zFdhmop#?)wrJA#A@)*q93sV=edxnFS>6c7t4dFo!PEKBWWs_z8wvE}^N+&|8_-&+! z+I}BP06|fi#)G_%I zGEqPv+k2-UU}Elgo7lnpwZ{y(*5EZ7gZ$KOnL=x?e?gN9N|IbU zt>JZktU6xtx3RiclowrSt99^=ET>@&eZg(v3$!}g&bh=`EVdNSs^UW4!M+%KZ(4Ky z3mFOuR&m<-aqm+9!chLPmK()3C;W|>LQ?PRokyBQq*ewKx&{yC9Y4*Utx8|mj{#>u zrCX@_aq95BI24h;)Cj!Z&luS~@!UIA!o%6*O4-T{F=cLvnMJ3~zv;7kpZGpOBib`z zn7_&r+1vZ#przOP&F@D&tyQkxJy_oXJ{2M(JYH=z z&8@YEsOxOBgAcTZuuIH0UVN=&2@$kxqjA>k%lM%`Uob&iTw@!hb;;Rr)JE4MJa!AI z_)7nrjlqi}$5y>@6ud$TyG*b|VVVw`V<-cHzrda1UB;f(VzHZfSmHH^dX-;`8arMR z<(OAZEVQxI+L)Y7H|(k?RR~K$87lKaYe){Trf`Q&!u6y8`Nz`dSGG=$myd*>qDFOIB&5`^<)`<1mS{mY$^d6aa@yCs7rAidR!|qjVQ7 z?$wP>N}D-cB{qFvq0vMLHtTAR4)0b);)|i=k?8S=N1#c%;aQPL1tn9W;lK11tqpY> z2SekXeipt)aD0{;Y<=o}*T|Oub4O(A-9zci58e7XX1z2b6KIyc8=+wopzdptHWe&g zT?!vT(>{P=CLpFpyGJ0G!biNeBcKlbdg!wnoA_1tMP~-U-Zt<<{(T0t3 z9kYT~eZ{1D8A+Zau{n{@BVzptUcU)qE$Ms|=<^<@&~N$~ol( z=ctFW!aKNa`QBxoFf@B557ntY<64-x9Pqmg{y-O1$aCK0+-bFxjvKxyW)A%AT+ zdx-p-$~%FJLQzxB^GeZ?JdooDzE)~A819O}Qj@v)Lr2M#!9NQ(C*ao-V|~bh-=WHF zN;OG)!O0_~`A>kSos;7;s;5`O@HxH%_+W$$2;0%V$nGH|E+P?6+I^?8`<~iTuDSos zq?g>KXimD6j~MhUBUy^ZFwlIEZd-|LR?G$}-ZnDg9I{yfSsa+xko5OGPC#oFi&*N% z*^xHp(aSxYb9@-h?bk;_Yy}>BLM5OL$K;r8pw| zZ!xO$mC;n-O2;d69o2PD+LSPE4Fk2C}kzd5B77Sj(4ab;G$K=`H6%^XZv`^ zpK?3Ic!3=ihwwAMS=fBy83RCNG9o{BFUBxthNNcP3)pD^{vtgav(DLfU**lq&Zj<2 z!U`~oqf?fR_!N9?fYw8G$g%H)j6c2RC5!~XBwveID8@mR=Cu9jaU zR#jdy#m8A#%MRXxFC9f~Zyc^XX?*2XK@KUC5iR1;M~6wf_kddCrB6Bta>?J`+?J~? z(l}YVVB+Jk{I80&hs>*g9O^DV?WTG#YB|}EfdN$0()g_DaAki-<({`IVJa5oqk1d; zxev`Y@#|`?8#R|(^UlMO2O8DZL6mPY@~~nK_&sh*4rQ#LSnKV&JAJk3i6&GMTVKr; zP*RQ3N{cJ@vV>c(ocFeno&4e3Z4^+@!5#?~o6}cTw0Lu}aT|b*8sIreS?4E+?bFhA z^g5|R^3t~Ha7dE@cX8*PJ*1ZY*`!D61<(vG-|;+@<5A2voWmN;^q5wZB-o$fsgP{^ z0xmnetG0B-r6~27Rj`VE+Z)%umJT{D5U{`K zF`2Ky^k+tfAL+b&b#EX_YeDqGeO-f0=TN1p0ieY7>6!HR)4^kci%(RWC0`7Lyuq(O zc27`PeEICfZP~2G><9BNU$ZfYS#eh-d#B_?ohKjG##4zu(IurEmm5V;R#w6+s!2L6 zzyg8k(wFw;MmewUpfR8FoBnxF_x@{OgY%IuTGzaAN|(OkzZV>Hj&*L9IAljRehaz| zJmw?s2&bFp%paZpMT1OBbB4Cf%>Ch4=CccMg}za8Ye3=kQUD-sx8aN|7kV4W)E-n` zM*EuN-P;1bu@_oI*XJv6*}Qa#NCxTt79Tg#1+@w*YlRllIhUX9Is0XPqGx8c-@m4a+{pr?+)tREO#wooua*P9;Slp2kIFe9=T-V^>pXs;| zvyjGscsz@2dZi>yKC&GGp^gjj=-BC)j7c7?6_&45LerHx@Rrjs%N&J`aCpb~1LFW5 z7vhNYdvqE#@0gK+1GO;IE}fQzT3CLCgikq?F<8}EY_}R?yIfD3(|R(K3qNeI^iLr# zCF@DTOA09gX7DB&#UW>^VO?~~=VS<(Pm*0uQ0A?XxR5|SZv{s8{l3OjJ$I=bM6X1ku;e%eB zzJjlQRzF}B98tAIj6us-S)Ue}lX zF_>bNsD4Ityh8T5&O^ED@fgH5iOxB-;Q)3vJC|+;D7{-ryJe(T^oZ}}8hM^*9H#~9 ze2``7=x|}Kbw9;Gr6S80-L2m|3 zPyF^gzMfw5t4W2n^R}0G!btzUtLI=zXDw*Cnb^^d0R=!@g)P@ ztwjoZON^FHf*i0p+4cO^V5ti;+y;Fj#h7)|Mbsxve@OZJEoGD!mJ5#OL^elCIn;-+ zZI%*Qv`_m?F6Xf_(d4$sEWJuUXfpfJd!ixGKec~ij?&~H4TO7Lh_5P#jOeNxVV;U5 z^&gTif=~fkeD1b&t$#gH=8t>x({X05)p;JBqP7xxA- z^iPCFo8^=h`@NxtP)Ur5PBuH+Vh1Z*(Q6)~JArx9PKzX*G3v zj)kFgHBeK6%uL2fo2g84iM$d6-_mhOhPugjiYtZcnhh4_@GS=78r*$kyLHQ7#}_84 zmNor28NQ;_dDCQ%3+i0n=R6THK^ptw>WI?xOyhqB10MIbO_h(v1Vc`7h{__gLDj2K zS5zpYeD%x6V6v53Qe&AglI1dXaY@S_>?$`K1w*|gh?I2%QkgLQ5{VV$g$vjD>lW@- ztN~o1rgH<6nrCyb;!#hXMW>=%x8pDk9pyxQpTk#&&HjM>MlSNI0RuU}w`K&749LjO zj$FxJSKyi(;~p;PAL39N+^!VxbBy*)exRt_*zA@*dTGH zODoBnNHLoRLx)+i~{R6B0eIRDn2c%d!m6ZkS72c|Sjy@0I&w!jv*R-mn+*R;R z*V@U_Sez^mOb37ZfW9-f&{J$Bp6dKbdyl-whW9+WVq$aIEeif+=rc7H@eY%ong%Juxx++!qwSRkYg zFD3lXHgXE+?d;zy=g5DC`X5wo%Wn>{z9Yx!9ezEZtNY1+xGx-dNmp^Ixkuss?}ep! z#U~XC!A6YFZ#LhnUQCy{E3Y47jGLSs)~$pxyGO9hE|?Dia%mJ-D72qD$^81mz=5;8 zdm3YUzigf}2QrNLL#rr#e*KmW0p7G>(tx=dI1yzr>|UVAs`#vrMgk+{f3^ut$iJgz zQM||UzI@TtGq)QGoA2)pb%v$*XHhw3HN2`u@C$8!U@&Tah+65(XI%`x`MxgX*5>D} zM)|s?RC+b` z65E|Vr}+*?#St62aq^FvceE4^y(GtXX=;HhV`sF-dI~4*tO77VK1`wU(-w;O_(cFe zt4Mqi9l?iey>i5Lbx!R!fmp(#;~_cud{5O|QYsQOUGfm!=2KA{`KQcnZDQf)tb)pZ zu5%g=BB|3(iZS@_(4sWwa{Rvd*oT$M5Zwq?6iX3Y={@mB*BA0|!s&DXvxE~;ePk(Xb(zlSVSC)Y&&Z22M zG2Ajd&Shm9^EiKUwFpIGWQdNG-?=PpyGt0_)5imzyBr@$3}LX$|I{c!W)j!oWqn=$ zGh+MEQpl~%{&^Xq3+-bLU-`--=Foqw^rD!9E;D%9srmNOU8zv{fn$rXt7_bg=L1Y3 zN8TTRW=#uusQ{^)qq5>+=F9TP!oYuWP9bs?EtX3neaM4oPO}nxu`zWmM0JrSl{Y+f zEc@Z%cM_7+#vkq3Dgt%af%4C&NJe|DRimMW5`_G2V;r#zvb(fl5&g0Md z5b{SM9RU;68yguFjJLtob@?;W%^4+cR9%FG;iu~-L*b7C4Z9Tk?_Onfq7P+ZnN%Hc zdCgFV%jvl~JDIE}tQt~M0%;TP0Y5DGHB~g}$!O}IhZ7ywbv9cT%v;oH*|RNTlkd6; zTWB3l4kdrKZLZ~})*f(j@=i7e*TIWhGr?dx;w_mP)4+%O*X`p(L$1M1RVGjaSwvE1 z%tSJj-XGKc$~Qx#a(^`#-UEWJn<$0}W0j}AW=fP$@AIrg=KZj&IS4suHQ1MSU02%g z|8YqWlQG*I?UwF55Wlah(NVPX2rn>k;plq^;kqI8o}i^o>$f8FBhKsVUW={zmCAs+ zc4k*=7sL_|E7A^NU@2A87!L{20IXP4n^S&<95> z`_-`RJ960JaCLNuoFi!gOlAX2(^RK?ygzeX88_B6SwlB)D1q z^<+?OXKT@R#FspWn{K_X^jYLcYB{on-i@_$9Gai`auL>CZ$sM@Eq)K1%70Q{_bL2w zCB|}-++EhaxUi8g{r9s z9uipej7Hs7?C<+O0Z(D_Slb3KPTMA%^1C6qcLJOm5Tlbd@^C;oEhF=jmVr~n@y z?kN7ny;{uknj(dY&3ewRpylRwbn*E9b8$bRp;%=K6LqUkP3`A9bpjo&7eroA}V}zXc|JMQ}iEWaG zAciMdiJtR3rZt84YrZ{7+Ug;Q10A~NMQd!+ZFC0Y7uv`&M?*65(srzbGU~;*JlxXS z+{n%VhWu_?mIRg=#yzhU=bk{2i&mJ*W;=0U2V&kk>FOzxS9$1_~bWR@jehHEds))CuKJuO%bB@ab=*tQR_ zf)!U^Zz=!lAf>2PDsDE0zv2H%u3|2RZpy`0xvl#1D69*4qnjw_wbW zNn45DG%dOeh@^fo9quI>GbXpGb5Bjw2&UEBLl*T26*Qthzoz4QUFEr1RoRy8GL;hx z)f!8p++7TgP6bJDyDyNFatGNf?rTBa@2bE$mKZf*2XY4Fw10<*+rF4=V*X(iP zUMDqpRaFV$7HZ~0<@*{O2_EDCsTFxKDz}&->Mv`J`IhF2txbyEpHS)JiC?8R`&CX^ z@dI*?N3@)JSYOMV)-+xdtSo|J__qK}g2LkUCFX_dX5^WUC>`0Afo2Fy;N7~VbU)~{ z(a`PwSRa3~jknu+gcyb4J8=6G1B3XERVZ1MDh zHS9clN#`SenadQNT-Fg?o63W&^b9tRyjMpvSDAriWTt0{sOP*XA3E;o zNaE_Jk;JKq!M>q&d=IZrYnX|}k$AZg^~%nn0n}+Xk7ADI#^|!4~%lwmcTY@`}ZOY;I1gjz>1dl!vX5xqT;+CNYrbzc!jJmKwz%E>w7%iT& zI>u&EttEY_9W~<;m^+ zQK+(s>An8%8z%{pM)UyCpvsYRv7i87ZYTLE zR}EBtR zylIo`5~Q_%RX-A-CwoCV3(acsr9#Q{ZMrIt!nV=bUxQgvn^_`U8a%`> zy{av!LT_`eqBoDt50lCI>JKV@BFAQO&?m>Fum6<+&DZoSGpu| ztYPdF)}$gZlq?S$KHrnPWOvO#<>qtFPGwJY_<#1IWs?}!y*0#m{_<~cn zPh$Y%2q=9tAxB__S8J0sNT2es)y4nuS;{6d|C7H)N&6QADH-MYKD^&CVf+Q8;ZI&Z zA)OU48vvm&nT%dS{j{f8V;&0LDMrYsk0n9nT^bmHW&lgnRf7ivE`bBF16`;6`N|Wx z_4!o2_Oy|4O1;adp*bJu$a31sz)gvkdXl_!$2E-mZGvGX=KcB|=~W2*Q62gF!&Fk@ z!y@}n&;++jBD7W&!_611=6$FDjXdXx!c(M^z7o|(m;)t!|Br7JF0$!O+WJNX;@(Sq z-99m+5`G=QSMJbOvA%8ty$a)}k%6GNh0c>IW(drE&cq*0EVcA(4VQUwpXG*RL4oV(3z}%%zQw} zNhq*=#fXO+{EzaqkNkYeYKN$DO!A+td6w+s)uj!#$s?DS3m8J?InjcCUaGHdf#x?-Gv!+-TnUub1#3PT%$Thj* z(RXIj0Rr((mf8H(NW}@C_M|$*Ixhv^sr&jc-ZSCPf0$zbY^{p_yB^n=^@-t0$odLS zXyo0sh}m!75beLFIQbm)c|P%Ek>TgnD^~mBH?BTces&MP;~rq+&i$X};*Ir!|K2P8 z46}6Hc)jmiXX?q#@O znlelE^URc}u%=)+^6Cu;7uYJZ&uT>c2mMJc4qH;Ho~%ErF{y*~r zt7As=+|NDXAotfklHa!2eut&EDOwB_v0|LKPb(O3G--0&_^DD5nm=i_NK#h3dbK}I z4Y6_Y;c@%KFP5&mBAB~ZhY)DlD%zu^oj6qf_OQ9|i7Rc?LbT2j&MtyYU0wYb6egzG(W zO4|Yb(Rf-YQ|4Ls`_T8OMp=(VQ*Vx@?1QeylZQW);(F-{2@8yPE`gHwwsm*@B$zFb zgI@_a+2!Y=C9p36B4&{siO?UWA#e1`kz|3<7c12pPiN;Iz{szlpGXN6y$d*FOpFro zQ^yc*a9;C$E@EAP8YG7wd&0hT@Z$q@5FseN@1dH>jRA>bdrl#%_Z+^XDA;GiJ5}i-b~#!ESDJ2!80eaxOw>0my5Kyhi85waLy3clJl$IM_vVZRs9d3SJD162Rc)#nhw8% zkSDvBowEDNZ|dDCE+cE`he^;zyz+WwW_6vY%@A^bt_#`fdEZRrEyeSVkAd~%Q;|gP ztLG7l zbUPX2;F>;;7sPzVd*Fl2Ne>Rt->HuEFB^PQoPF7`kRN283C09cslhWs$HO+ys+EbD z-ATC*CE)lK&zQOBA>4w?mH4YFMpMuwa=yx5Dh&#GFOhHfzxp|UCu%UlHjeK3?Q5pm zc?N5LfF2eshFJOs_fSlCw%!~HS66W>SVa5l6orz%M)39(cQ;bXnW#1GUvH_nZr$qo=_+W-2#7c*44 z0=?an7@rweEOsmzrQxaTJQAdfd^NRd=c8{w{8}k4_(ayJD{}QEqixK;eeoEw=$kDg<4TX!ULUu^C)Awe=$*24MhRpVkOn5^^9hVBj<|TpF2VJLC zFU`eoGb+DHo*Bnl{`Am??18a^*~|0`x)oG*Skj1(0a(Yev{3j?ePwKN8S}wbLN`9u zwha&67!bss1|%e{`T>V>E3Tv6wv=BI5N?OqisgL2Z%iTH6?^)an;&-a_}MU#U+V0^ zdaI^D1NqfE0uO|$(W3@H5j0XeR{6~J?l~DPrry$1++Y+z51bj2Qp)lP3ZeEne~l&$ z0(}=cpk=O?E3C-co|_uO$sC^WE?7@liH=GV2{{m$LpbNEUkmTkZITblayZI}9 zF?RU=KZi|`XObJDth&ulH9B~nZYuakO}~#&Q~~QLD0ck}f~(Jqp5EhUTmBD#PMf&h z#T@PFIGoNd_e1fqF`sa!`!6-^z+2>t1mF*3Lyv;gRRm8)GMk=)$>cB)NEOTMkCeYa z<7f7kyWrT+fPd>qL}~#E5n+8^1MY6UE+rIpkssv9p%3PbztwY$6IQe+z$CN&NB3FF z_8|PP-Atbk)H4GV(m4f7M07?!jOzOtfe>aT?!b&^O%O{5dM^s3b( z(F({jB|;$f4NzFH*xU74N-6jzO^^W=l!IksrB zbs-Sc-20vI9zgOI(NaTPv_CYye&ec%Y(R_tqKo{$-&r6JkO4F`H+j!pv)$08AXj4O zm5Tn+As8`3{%oLx+GC14>M+_D>L0>)KI9)7SSXf|yaYrt&oBFtZT+qOhJu0@J9VxM zIh8I7jpq!1P^jUW2Fi-rPFQ-}KF|f9mOb^oe37D?=XRlOSoEDwkV7Mn49zbRJ{b4O zKJB%JxMrdif2kC)U|{at!|=u6@lwo7S{kb1kGFjL-#$Mi>0c^kFBJbz4!UQ$`&0ef;n+j#`43^r6?ZKA44yl+y&E4nAnx*Y zij5u`QuHN7ea(~(4fj+muUMv@H|3HW!zS;=E$a`DEj~#uJ#^LCO_aI-6h}_RpGAw> z_ZFmna4KZ5#OWq}x-G}7)XfBTtgP&*ah|{KKv6?%1s|o`0mt%t?2`r>SWQ}UJSo}I zZX~MK6_jiEU*=ryj=*f%%W2Ev;e-`3ZxXb<*}e6VS2{bQq`wKrI$+*^-bgTN$gOgN zkd@z7wT$|p;SF<<+AiBRERM)Mo`2sS{Xhk*)lxt#2Ox~;@?QNKtZH{l`pOP$NDq0kO)6UAqGadjJUn7^v{IfQqhF_0_4 zYP#@#fd$OnoOmTEaa7j}^P-Q|8Ca5GS(PQ?8eTQJKVwSgLbT=6QTt+CW8Z80txv!R z)DHids0suR`GXyQjXie)MsBzkO7MMhH64<;OnY>}Bh%DzJcn{#3m457&>EJ_fq%Emx^y6u?g_X_a*fJ+TT;Uq|};7LH{(bY6895Z?(Jh)2yV z9o||;z+N0ak&@wQ9=H0dccQ03?B8u6#-)78}vPa)pvocm?9zH5B{jxocgvJkRJEJGzm*Aun5iC zTC*c=@gu+Dvq8f~W7!UKc)X$?V_uM+(IWZi<(HlxQ!THTJY3OI-&qxq@vJ>TtIDGU)35e}c==$=AX9=>86BI}{Q0)|F^?sC}=|+K^9q7FB82$BE>+MYG~EdquWAa=*gcii`KpQL>IxvvO_NHEfqgs?Bk8bnNm%S9?#Z-q75+&H3{8etgCW8p_Tb$FHhTN zn}V65GvzCw?UX7`SrZnX6R2yoi0L7b<12zCkJ=hV75W-| zXpPM(ycC;!rkc|x9CXAVXHevV&0K*_Tx{9jtD6-9TWXgW4o~^y2E+V$IC{P zHqpv~v=suguCsb<76Gbg9&z78#IXfN-g4U$L+%ryz~<4786v$4}{I#$jP^6?c`ugCr zn0yP2bwmI^7f((U90Xps<&ao5_dhRx+E`})j<;9M5b@5njJcR7bq0TLs;b|KK#<=n1kJb13zt8EEW^fx@V6V3Lt)o`}0 zx|5Q4Ge=WHk`)MpH`zx99Zyy|y#MQ`vK;@4f03MxQ?eX+i*!oq_u|8^iSP?Htf{cR zI@gu7GEUred605gYk9_szsD>$M_v5b7@JNTy~!Gv~BZrL@d zK6izdmYb(>%!kar%o3;!we)T^u1L4pg<;Kii^cJj2g`8( z`1-f4_$%>1v#0LC&6UZ`3xoMk}OPEQ>Sph=htFmE-wB_~IQyW#%;lnkoV%ytyhS z|E8K?D!NCIr8JX$vfF-v+ev*@ulSm)u8Fqh0b~|3HL{6i> z@bKp$W7v}Z-dgvpBl^ctv&h=H7u>L(YN_g^{WrJw?~KVu{>ciY?S*=Y!A9C`?OL@5 zu=$KnPSf7bAwFZ7pV)M3tQ{gW6MmX0oRSwOS`F>5#G9KFi_B&5^BM_uNm1lbVQzqh z1rzPx50FY-LrX9X&ySAK3MA9jFOkhjUv?ibje)sPO|IU41e3!DG;)f>%Ee@~kg_Na zB*tAZ`afh5<0oPC+UgUp!t*zg|L8j{BB@2rEq)q_rBC!ZZJLayMAD^BRu~a=hV~WR zVIN|+9CRD)H?vJ#vO~lLWD7+1=46UiII3=CdCQW2CkD56Dm6To`-f}#vrmcHD*3`E zR+}0s>=T;h_S~spH8`(v#HlFPaF7@yIPhVUQR)6+GVU| zJVqiR&MU$)GceJi59b;`A-;B}3DzTP-|Hwy4>un)dE`sevE`MXruyh~(BJQ(?FCXzG7q)C_mYh08`%hvUUO<%D2yOKD+!d%=Av2;cpp=O5UIK6ItIKr%@c8+1b&Z zj)qNHp=yA0#J9fB?hkK!HJ1?xsDtFyP6y9MC|t>3Vm|_*T6{5A`MKdqAr=(+O#xfA zw^D#)F=m5I43=FCwzI(!as%#S$G9ZsKOl@=x~sTl|14&7us(sXKSnU8IlJ1z(tBg zLjkZ6r5bOWuxJ&FmtBZR)2Ty-tIdW-V+Ko4HFe#I^XmLsT!6%d{{hR@;ETXlQT715 zx3|pc$W;7)AXx17p3>O^XrmIJjmki2y_4SmApN4#v@(O=i5MU6|79v0-vUgf`Zy}_ zM8=8i&5PY@-fxLL6gN z_~JWpjK3>yE>plce_F~^K3xe6UDG<6 zv*i_8eWP4kX_%T%al`<0U#r+{D&vCmyxd4S$(<{zd2ASaq2qJjSNm3+c*`yNWdt=( zBFY%tzGvxDLd9^?zZNIVv+t-3 zpPHR@(rugLpBk|HjOh>gR~7#G=@@i+_rXgI#6bqfOrUa){m#|)FRT!~6@+58ax6J^ zY+M|X0?(sE~G%>k(?mVlxADy=(8D08Y-#6&k3jDn<`J|CqaUg>A!3fKZ z5`8WNmrRIWmK4?q&W5;DT?#hy(Rbl;35nPjH#Y%(K zt7Z40pzgJw&>w6!_+nA1VBOcqO_j?2jc1;U9N*^kui6FRMVGr$Ncw$-q-N1rALdK) zj1ku!Ne*!S;=PxUr>ZAm8!@{(Txg?-{5iXqto|^eQ>P!eN?CdN_J|os=gO!1)jnkf zt(6s64VAAg&M@hU|0)n5zh>m?e)Cu49Tx-@)t-hRj^zXOb$F?_+Alyr^xh1Pvbnbd zZl|`m=y4Ewk|ZUNL;(n5Vc~u3FI0PyzPhvco@;GZ!^CFJ^A z+vot+DhoXa?ixkR2g)ClYWmfOC)Oq`+?P10u_wQ_8^YL+S}vY^HE0B%61($;)KyBe z`eWSK!Wt!sorw^`_3@%?RjV`7ZTTjE zfXot+L=D0XF@7S_JWT`b(4oH0e9Xe&!0t->IdQQgOn|M>*YannE#HqWK6jBzsqGWjSlo9At4t@!nH{v&Xu}M9c1Y%ixiLw@uU1eb?(p zKf#53#xY_)mnS1Qc(~h%hti-PwlrhKhshao8ZfApiEBWoTf6KfT>mC5B z+$ZR{exiI+PdL1h!2aHzg?4B6o(XMmhWqJ|lGhI~=?&zJ9Jg%}p*y;Sat!%Y(d=pg z!YiP!Oq;@F_g5qNk%rSpl?L&!N$#U1M3F5XLRC)xl5BuD82V?E#*CYPCc7ajmpw$H zV))Ol16n;kgu>nmdj^fUN55`TTW66E1*`^P2Y^{7x6dUH1%rd<3Znu}kH3N$MA*?z z+mCjJG|vKu$Q~ZT#-LJl)Npyetnce7#X)`x(>2!T6+3T6lQyQ`EAykvYdPIQA062_ z@HlzQ^!Y2@kP#kfwD$c=8-B|(dyC*K?G3ivu{#dc8Yc^bVUq*rN<9iw0(3uln8X1G zAUg$M0S3Ad+c{2HlF8bYtjhM-EET?k=&r-Q(VisF=tBA4@{PY_%GWt8M-Vah7#Y62ftwxk{d$%AohPuWVBnJ2yt)gK86vNcsAFQ$LRZ!PFkK9id|i|NN$&}$ z4rtf^S;z!gFiV~)UVHuEO#K;4NKI@9{Cb?}gGbEmT3M#|3TB0&ubAZ&1iHxv^iHg| z2LS+Rq3dZcD=U!HjS(sH7vJv3@jEc;V$;f23f~v(e_J&_9Tb&63ZM^p%b8HqRVH{6 zIT6;Z3z{OUFXQ$n1N67MaVN$4ihuY`YwYHE9wITZjNi@X0PM7e7i&{ED?QV1eky`3 z1`cr!7UT4{K9=~T!z;VoNu;bP9z@}%s{HMemdGLpXi@{}Z5Tkh*tG0h-teEU@Af`X z>2Cx~jgG>D5{X9tHduZBo3mZdos6DmAnX=>QFyMvKN_*2vE}@@Jr!6#XSJ?iuVgP^ zy+$k(2^A>|nC)DzM)P2TL4Wy7js+HNgE-o)>l*C6SJSRmc6Qf%dl2+f}}LM1u+xF?1^lhIsYSGZ5VmmUNOQr#!Dg))_yE)eDGwe}X?2-n|F! zzoH`WqH|%b@AZkf6PUv=rr56Ra+{H{`@*?F;Devu{l;upAv(A?BJ{ZR7Wa9# z9jiual1W-1E^NvS7XIPxzRHTw!N*ZNnU`H+YF|pfMgb$N9 zo-kxsvb_)Ka1SFs&ysoE2i#Y>e4C%)b&5xnEzbY(8q0E)-KE=PNsr#X%xA5F%B&DR zHuHOWO86rgou3J6FQ5CWu76jVx7 znn+V4pdy6cTf_hXDFFdP4GsUEf2}j;i+^L^a8D^KbsZS}c-$%`EmQgrN}r{3u7k+X-=2X^ z3bkJX_mj>p(LjgA>7%Kn<6*v4f*r7+Ug}ule%$aW2v`rMZ{9DhTpKm?_aQ9r88Ng; z_xxw6{Xw*pUfNr|`&z;2-^FANHlp{$Q2NCi>hAVqKTkTmdeBF-3T}CUznI3Xy75n* z)8E?pG6kz|20onE7zgZ^&Hj(`|wB{Flty;pi-pt~%Ss_Yr^P~LHU##Qte)(vCRGQsm- zVm|2PGp=iAN<<8PV%~^jB+#`HCQT;`91~UH1Cs-Wh=Xa*?gssSrl}g+b5!PU4}fwa zr>gx|?4Rx{pqq??g7q1HDKPz)V`3VTnV2^2RFB#S8-3K1!#b2rq4~)d3{N4#FP?QI z9!Ru*+8kH%_2*Hq5(lT(hT*q#o>XkbXX_x3oP2#~CF*oQ@Hqu2S=5Fn9UG_Qyx)Rf zMz#xg8MLV7WvHL+<=V!bL;ryDWAh{wnF9n>?D9HR-*#ti?A*Jxzut<9$(_k=e+H=i zgF`HwsK~_{Q;H-d-vw-1zyn%LAS}@w1AqEN|NQAXY`1M~n7so_6m#}{%~OF4Abp#A z$mK#-iN{a({?@0BWT@P^*2u>rM>TBdBJl+Nb1CP*mnAYrRliAyR;4DlZxZiq1jfDs z=uY-x+L-)tsuuc4)pL@!^gGb6vT$-)LdMkIz~tcB{h`w>{JpXI*B#bybF+F*7 z+%{C4Y?~N{n}3=1Bu>d#1)Z(=u( zEd7&0k4I;rfFfil4R%&!{PzCKPRBFwruTT}RS0S?(uZ{~8^vm3Pit~C7gggdXJ`UQVRkW@9QkHLu)&Yxhb-nkGei5JpQ3da-){ob|1|NzppMI;kE2*9G4}O zp{ssyNg*}i+^!CvLz3Z0$1+88F00pQPGY?h6!=M9twmu>6c}?N>_&{K0&D)F(S$2u zzcfotPh46*Y*raY_4|%Dn89?6!POBVvARz=CwjW%+;?-vK9XX7AKq?@ zjXOklp7VHUjVuN}^1aiudwog`rhjyAZ0zuBBbaNhHR|J$o>}KwUG=itPy8dJJIh3D zcB1{JW#QG^H@;^M>80eG%`ofEMCj_VX6`1+)-r3pUG-ylxH!RVVOPfqe`Zq-g*4B- zQVm-u&kuAXe;ZZ#YgnnfWpAF&r2pqDaP)hpmmdrd2T%`<50i8G8StR?TuG^Q+Mg5M zI_m|ErBfph3<93dC7gnr2%^#yCZsFbp_@rb0LR9o7G54OeNbe|q?2)YURuzwO%3CkM~U!dS(R437s4@Ze<{!{2v}9SPQ7wFFtHS= zVfA9V9AV;J19D=G4O{Hsk7RQIG8X#E--zvr0F>NBK-hJRzxy?(2n6`-j-{^j(4$yz z^}t%r1N0`|KE2U24cysO&A3RIvgAj#5H{C|iJXA2=|$Qv@a$Pl2vRQ>&W<<$Te+Hb)oF**H=w9nn%KSa(A|2LWp716IOdu%ZF{ zZEPK*ijGs@fKVU25+nfQ&FM+3`#v#1@#W2e3TqscL&bd(UdQi2A%7)Wx)*iV(cj3y zIE=q^$8G}$>+GY1d>5)r|?#T2VG}_X1N1hbEX^^Ehd)3X^v-wC?XD-wG>JpZvxOBQw_< zep1R!OSt-htcsAk2Q2E3Zg)*0|8Z2R`AH2j0|tOh%NHL=81PK8+-tTr4>H}=Z^asK zyotNu-Q)Igcpx1d&PfU3UN){C?L8W071k#qdSHRM&#WgiIx^;2 ziwJ?S3oBZB_g%ec;=EE9{gMBWz~h9BBKc)r3mZ8=U8!smpIb3B7+4T=`h*_RK%=--`00E}Mv{dtQv07WoGoy$&S2oLNfxqJ1< zb4;*zK+D|NrOzcQL#eyvEnZ>elq1H&v=0QJP1in^G_;=Nw3d~;AFcMt&T?vHDbR?Ko)^FT zw4d&Da2crfsb`a9Rk@jSeVrfcTK`s{wQ+-byx%m8^q#l;3Dk-FRdIKc%JDzhRYa_D z{;l{?86J0i?F&y(8Zxw6sygtYjcc-!Q1@qu?$pZ4%HF9JkBI8Jx;?Btk3r7IvYRiA z8r7h4ta{eQ%4V}x#=EHD-R625CPr?ZUMWK(U0Zfe1K_SJ9qR?WPf$7hv~#E8d_-{f zd7hjcC4@=8a1-30gO-{;59jbcYAd3;A|Gh^dPbFGUx;;d%}1;$*+>5L2y(vP@W|=e zEb7Ju_f47T67}Mq;)_rGpQ?v>t4ldKo+&PAF=fA<-oBy}Pz8UK@JIIty#}LHSNGcV ziOH{>*@3VJhLy)cc#|2}tM0}nh!?{hyRC5;9t{x!OR^qC9_-=ify2Jb* z_(u{@F^sO={ZZu=={^u8?8w73>0ww@M31V6UQl_A*u;<|l_EcKKeyq$BlCxHL0QuU z+@M`y(~n`!ujqX^K6S5D_>Hhv*X$rU^&)?T4UKPB^mQzN8JvoSr3Zuz`!ejuwr=kq zt&`!G_f%~tfkSHA)rr3zHeM)$np^5zuB!ZIkz82p?cV{AAL9Jow6_z0cI%#~#F6c_ znb@6|69aRTyz9>iwb}?9xufV#J!j}`K~N?A(zj>^3WVKhe%8RsEBhO%9-b9)UMHd- z7nVuW%+M-lrix|63vn;=7c8d}0u_qM*T>lpreS*1>D*_I`?&FYy9?>83&wVabLZeK z2!Q19>;-pMe`*!MV&L@EuMbtLP=P{l7$cu$)$xe z-=p`roXYy;iC=p6t?^iem~PA?yUGVZ_|#uY(&67_zK`UU)m5@?;xs4jMcYp*#!()P zKPuXAE2(55kB^n-XghjvHDbSY$4r6a8%^BZ{~7M75Ax<$HU0%XI#-T--#@|CJN^oi zQA>Y(ZK+;5v++4D(4tXP)%ckioc(tWY{b;+Ct&@G0XJ9G&D(e+LYktXT6`$yUrLA` zywCkb>!tYtr^xvUauuQ@bKwkAfFGNEVy9c~u%dydTEHs{*b}{EORW0_y(CQt0{o+P zV+56gS$dS!|L&L^Eh;VsW9bAuQJP_GGf+6=rG~LB+OUhLRAM7z|3aB^G z$_%^VoYM^E#mNN>EBtK}U?39*cigw#LPD5Jr@i+ksjrJ7WYOyP9$-j!l&{fogDD(| zg|SmVH;y6@-R64j|>xPi>N*) z=;8=Z)9&t^+Hr{EgHRn6EGT}x3)}oEY1KVsBiHERlSFc`$D~yre7#SWZyJg*gO`qg z%6o=#|pfAWZs4 z!!Y{mJA_2+NCxw`msGL0vJ-b{j(CVSsOLc;k0)As=PIS(fly@{y+R)?q@ifsJk4*W z%E2#bO|_Y}o|$3jUs%v)S~2Q}A{eG` z29C>0?>(2rN>z=ms0Li?Hs`<$OR>AA2qw>Yo%~nUDkVeew^GjV)tWuy@&Khk&&DH4 zIBkX~u4si!{2^i=#=9z&1^<3cwFDqKyof1dn)WA`ed$%fUzGEFvQI_Mteqm>@k%yP zNVlbSAoL~ic2B#7Sh5Ksd!LSXR5U=3(qgFScB%BepXPnme16r`v&uanpeC$TZOB7R zUIGm`bRJX8$Pe_?lPyx6YJ;z^wnc%v6e+k8l8Tkam`wFuCks%;yp5d^|pEB-vC@Ls$qI9kmodKIeADHK7tF@$D13q7 zVDCImwU{_NZ3xMllf4(3ivjoixf$2qI5g0X$CCcK`*2&W4T52F@L%jRmx!+pN%51# z$!!AWvDiNfgCOpnn)f<-J0~b{F$O_VN1_LSr8q5r>EedH8FE67B47rGp>lUq{vsrxZSl_H^2C-gpwHlJQ~b<-_I3I|fMz zc(qGOWfUHc3}3WV4+2rp%l|A9)WDQ`-ub ztVwADVfZbR)8vmu+Ng4RN_4&#=RqiJzd^r^e79y@5xa9P(G4FyZi-6jVcRtNNVzF$ z0vO@L&77}-&qE=c3e3%kuT9U1(Vj+!9Ey1_j@lFV7FHb2Hp+Ynz;psuLcz8=a9@|* zPWO$Z**&4?hdiV|@5Op1AUWMcm`P5Wlr`>yC|3aB9T|uP^~V+ z8J3}e6-Uo%Gt0O7wiLZzOg>Lh0IuJR{6?ERA#snKHcE}KUmzx33oR*iHP7+hYpLYO z%*7KEX}#%TkK#DdSCW?2^^SbYn2H1OIA(aLe~qUBZb=wxNq6~px0fjElU?Af!``{XF8t!A0#*2>FLcP`jD8X-09VNYFO-Ldj@ zb3Kd)W@j>PXJ-xMn{dcq?Bz1)(s#~8oVB!cV;xMi-*=a3sjBBZRXCdV&*tULzKxmn zgHX;ADCYbKgyDEOpLJSX75E2M*w$ znQBmCaCzvWHQ8xf^x<#tqgI;o6f$wdccZTy zhcD1RfkwS=)Yukp*+9-(%9dWwkRj6W%4Y-I0B$zo$b3p>LY`ST5jw>ik+j9C) zCU1#8I*=ZH&a`oFj3o{4Aa7X-vZ}!^D?UZ{%_XBt)$A)19_~JksdfFRfQ}NLwKJQa z5+oC7_}9>85d~%2Cl4^GJG4_c0Uc6UXn%_%@dbI3Kv*#BhHzh>zvcC9@b&|pt&98M zE#QH1Uq!f7FXz{COo>{;qcbckW4A8oc$snelZ9(vE)Qlbh)1 zJ^VY;v=om^<<0(@FK|_{^C^3lf=~k=)D0b=nAm+vvwdH8)EljD=-#p<0}I=S3=*mh zuvD!#rHu_gf}O&y#U*{vVt@R2hT`++W z$2x|$HL9fmbHF}Ofl*>=OfGj4v5qf*^|kQ0;fsp_aB`<0Ax zHT|ZTY0V!w9w(v$%7yrL{YDQPWrv_-31|B~Wd*>~38IuUriS)R*ADd#$_v6#bkOP8 zq;c{a%XyJJ3`w2VQD7!k&gw0iJyiQuCP{51O2tL%Ik7qLZXtBunns&Mfa;lh;u^!& zh4}deVt&)nQT<4;VeI`?&K1F7__(V`9Cq`jC7|@HOZbK}<+xo~@r&}XVrFx@yYYJO zjUM%=uo))xsML&eGG0Z>QYm|3;I=Bk@@_ead{WkTIC|gDlkd*yGF6?UtWCq@aV5#t z$K0Be^S<3^@sMQd`MctmM}fwaEN9J-|x{Z#d!%QK;j3>p6xk78kr6ftg_pNejV2Y7sq&u0N$& zH^53yoWEPg`s&e=C(mAuE`-@Was5YKDrae*)ANZEy;^c)Ol3R9LoEpyT!3S9qCm+) zesY?-^+tb8n`?$a$&q)A=G!1B>J#}rOppIj$(TcgfMFZ#zA5i4Ne`B0Z5dDxm28$D zXz)>m7Awn*PA|o)fU}QgvxB0b?^F2f1tQ);oh_Q5Y5ySUkDXtW&&2FQH;@B8{U;id z?wYFyz0o(I97D(`+v+hcm+4$SPEEcReLUKBk2UXB<`T*%n|d`faiL5`appmAi`rQG zUxlzRR=zv0zpOs25+~irGBq&>cSqKR7{&J;nnG(!>g&gMPGu{Fc}P#~V7HYW-EL1LMCx)Wg+_ zdd}ulbKxc4nEWp8iHW%u{yRdM6}lDKW0NtlsNmcci?TE01h=|5qf@ z?j4LC>qPwqsSD7`2ijII^vdQ*l^#0kx!z}N=V}=0U${YM{W&dB6pp%Zeb=38mlMXT8z*B?#j}lU2AN7}-iSF>i>CQoCGkm9INBil* zGw&XRy^n?7yTn=8Uta*|O!UuJd`N`p{2L3vRy(7h$lePl0k{%QQ`Y<&a!r47?A>U@ zr_p&OHpT>C!E_JbAbVi}ffV7l+U9?=Sm>yl5VsXltzA2QbLpAx*?h>X0&OM3oh$!^= zG=NX%_qpDHoK=PB)4j0lT(wcxn|BQOK^MIigh_;c3+GtP;l1n21*7=Ht1WA8K2C_* z<`UI;kROlTC;ww;!=GUZ-fTG+4#JD8@8+ttImsb|*dkgs*+#!k5VV-oJ8;GNPf2?d zp~uNsZ-?78IoG-;m@d%LgG)~<-xijgb>vlQ!H7|DiF!3D07O&JjnuQGg{k8>fIaGK zbR0-OcOY7X0HczhVgg$WMuUA^i@SBra)N|4}XHjQ^+pYs!j_ zmuk&-DKfdR4Jjxw#q5*QDy&#cKRyYH9_9`Qbe5((Ov}7b{8qkXp%^B}c zNttGqN2I;!5X@FHtk0Pe(bGem+^gDGhv+Ln{I06HVDZpyECV*_T-(chI% z(tyA&dUo(08K3e}sqrp1mRUA~Y*ov^=fd7S(0V>lP~g+QmS=SGnBI)$!_UbnR;S#a z-nO#6+tC^D^;i~W)B4cD9~IQg0o@MG@n?g z>aq5p8xb}4cNT^Tkg|TCq0$>SHK1bg0{JRN?~=~7M3cv`c@enrMXS4d^VXR#c3((g&RvpzR+(i-5Adzdd|sUfF<^GncgQtC=9)nGeK~qof-tr^O^7 zqhvz)Q9n+F0y1(r<%~0Z8I%uJJDlYHnj;ky};Vg%id;;>8Y=uVU?-DKG-iLsC z^GB_oT&jIeet$-pWoqV7`Yilz{KIvsm{e#G?PE@NA4gTq4q{f?BC5Nb8BaOVA|-1S zREp*cw*hy!zclUg%3z9IMoS!AnIaCMHshu4+||_c)I{B%x>Y$f9r&ZPtHE7ix4UX1 zj)!{-v?(G)O$$1{0+J>cMHUjy1!8Vb)I(!Lz*$tQhu}9wIqDPDTPPRmZemF$aIOGl zQoU^iCu5RUZBm=!TD*#0qy?b9+DJdN1N)T!nzX@6IVolnZYlGMV^@{g zy|K%J=A|K$v_#6V;IrqcEmwQ=VZ#Bo&6Gph0=1RF#mzUZ`3#ii;QshMqgym9~j zQN>(;0B`LOb|+hE1`qbryhj?&Gte%xE5%>bnVF1iGx@ptM#+3l5mhaWCAD1+hsaMAL^xVZFKfc@#UrY$o2v&%hIE1#5OxcDWr4C zHH5DGLhs?`_X9`}w4|{z_lSk|uPcD%z9d}^8JxrWW}p|8DB*v=NH=YIkEJeAR26nvS7k#zVZ@gy1Brd}N7VJ!iLPK|JZQh7^jm)Bh=OigS<*`uz$BkaVAQ39!5;4L|cSRvyc%Qk`5c8OP zH0U&E<|>ox5}eKZ0o~=CP^(^gmKYR}B;Ngn<|o1zwepgv?n02?$4j!2?-61pF!&{A z2V*bL+-Jo0Ey^0l`FLCkoF!c5nn(w|5kWtpRhG85#d*_`E%aMt^1;g78|AEvY8Y0O zh|TlHw+=gJMKc+%jqQ4I%Fekdoi;e&FjXbvWc2auV(Z<7A3_N_CG9tw+@w0*y4sx{ zS&wHj7Ek6u%Scm@G;;Tfqo>A#86srih>}MBrGU_0x?eZ>dRichCEJd(3V5BXHPc5& z*km0eC|dfcD6P^KM#%C~D6+gg-`(zNOHREiT90DkxL*1tY;sun@a54SU!z4Tw#5DE z-S|7G;rnUmRCFrv4A>T4jB@5Svm<9jIsyOh1$JEk`-e45cAEO`IisDrkkd_5;rV_Z zvECC6LLv&>a2LT0#vtZY_t>nm|Lp#e(E4(9JaLImiFmurKk>SA*$PRER`+gN3J$4r zNVDJvkanC_W3LK0N5iTkHA(hi9Am)o}v0z#8tK2y_sgJ-j+2 zC=fW2s=A2|yH%}C|Fiu3ZXs*+bmJ506?W74-7I!wAc}EXBOgg%`6Xa&ScSs21;QH1 zz|3hAn@D|9cmmdoU-!~4D>}SjJIBn!X{I)F^SqBIOMkQO^v^R^bH^G57|npd8DDY z{(jH7^&cIC>*1454d_&1A1x`DH|4%lm;|r~rkD@7pYY+ZBHSWc?o1VChT2vgh8xxE| zY7h(i!ZL)=PMgS@fn{v^Zrg$vBrPGgBD(Xg-drVFlb4cXHfK6H6?mrcX6@XRTOAdc zT*Hw4Z-t5T4%~!3BM=ncHWfHOdeUB}Na=?wnc$baQiE(;FXG!9=pF>+xI7NZ;V0*h zv)(()FPxrNKJ1)HAzbpC9KYY@7V8%KfA+QiXAN3S7hv|v*SZrRTL<5++BuzuoQ!pf z<(NdC4XAX+28CQ=?}_&Ljq6$v>2!=;eliy98Lzo=HE!q9UU=V7*DCh75q$n4aTf|K zGK+K{7Yd>GHkZVs0~L2`hy5sbmn;6W0pP!PYSbId&?0P~;Jfo4c9wC*8h{=+llzso zL1hlV)D6eS68lyA+jhnfPZlv7XhqYV1^arTduRR2D6)bX!T&5v z6A*T>Mc}cMqyNK=6*dY2S(Yrv=?Qs3_Xb|!nce^Jdpzce`uP8t3N@G@VdH{9PDls9 z_ZOj?Wcc6x+i(cydwAu}R#Rhdb}%!ylJ?Jr>2jF*I9=$&>RAfU41WZsj-4C}WT_HN z<~tcb<5q{&4A76UFHMWvrmU-nZidAB%?O=bOMWScp&4m7*y&HWmy_K#RnXTYDL6XO z0c~k|WCX{&K4z?L8bTFxSIk!kDO6gt!hi=KF(jQZq!4*5^#S$4e=Z#Vdx}&Kpq;}H zp$kT)8$V$|mHDmRgXXDAtO#J#jV70<%iC zq8e>f<}a^=C>=b8+%C&*z}|`&*Wz_O;VpW!Tyd$O{%5knzq|1t5=r&ymV{`P(enJ_ zMdHFA1t$NcO!L3AX^46c5M2Zv3@)(#W?AxIo3H=jpV^Tc@12eQwiS>{|9`c!GAm+q z(iR(9oH^~=O({q^LI7R;KV{JW<-7h}P`!b27PbZyA2f;jciDd!(Esom|C?0puM1}U zI~otMKJF*}D-e&J%MRVnPUk=Ab#`Y&dP|InSTwSSFYgVtRj{D0Gh|KxgN>|ZAR z*96g>10CF5YuXKm{%G3S#xA++;}2m4Jr4Vma+vYLtM$k_FrEdi&ejx4SFj&eu>Y4xsyS@;e4{ZxS>s=SKb zI(9Gj?}j<##sr8mj|KrR zAA?4WO<7@BFZ1;3`-mVbA^yE4TA~S=swTt6IiCjkO}fM7NaR=c0~YsU`zR8A5=moU zp+$;(a|j{`4}cowx^q7r!fcR6*y&J<#Iwt%5c3bpTe6TTq&kk`TW1@Ty8_g1G`zBX zh{Rou&t2Q{39TN6@|{o1sL80Jd>pAi3Vcah*-mFhCI&Y7Wtdk!c|G0wmOJZSo+_O5 zqNT$=>&@qlQ-LN>cYkWH4xHo+LCrWP87n7Qx)DKubDmLWfxeyRGCjn=mQ2#ZH0b9U z3GA>hTkom$U0AQxo%N?4zXnNxQnEe%RY5b+FMPQ{*LT@u?GY0P6=m{gRO%H7<^fF2 ziohf>@lfN?PQrgf9k82xBq7)Wfnrp8IYjX{jrcPt2;vKePbQ}jR#Skw*)s#6WFOX) z(lUc=V*^`MJzFd~Lxi-PiNo1)b);jJ$zhVd3rDh#`j(~=vb`^%qBFGAIui$Coie=_ zTFz#dLohQQ`xP2|6u;@or;x37{iCUUuLWt?ovOMn5e~wypn$zF6>aH8VkM`^+JLh8 zzs?_h%<+gm1r@&jet{{T5{I0<~q;{VT|NE!itA}(7f+S9E|^S7mZY>}HRa{*w`fqp zLNxAS*i)NO|282J&x`yb)MF;u8eV6~rQHHL)}nN-7BN6`A`AT%1~AsA{FUCZxB!1j zg5XEt8QaJA2YsF-=CMyr|B%WrO;5`Xh`1QYz7>4np`EhuWH0KFyMm{R{&zL7@0?VM z-{Xg&KIc0>mA%l0m%;U476`AdZW#9`C9;txp;8F=JyJ@W3P;&++Y;-l}WbqL1}s1wh~K z=zW*W@1MK(xb=Xl=%1BuKkjRZv4bTXQ?l4#3?5~@4N9Io6kkg|CqJniWi963A(O!T zXs6u$6z7Yi6#x;LumzwfhgPYVr0mEtyo@h(eIe_9TFE?Grsmxrl^VpC6mvpH5~TjY zMsAX*P+Dl%FpOfnUjkB3vm)6I6HSkElhYEhZYl188@a@GtFYYZCud?n%laz=8E7gW zU#~6CxC8F$Akm`Ef=55u+zTWlI68zjH&= zb~e87Zo}&nydX0hvEnjDJKuO7zT1xFr*W1uYK=Sq5&P@KTWVTb-<$yTGv=YCaab>TF= zY(OUQLZxRDEx$`%?W=^NGKNn~k*-8No@tw!U7=OgfA7pAp2{&eZM8BS@VylhHnl)H zrv%mTuuM;OYvK5L7FeZnPRYn;K$@ISi|BTKFI>7&PolCXX7QS==KX2i^9Qm~3nbQR!Bbh)SMaL5Zt4?2qtkS=`!6< zzhKxmW77UeEKQfjn5j7dfTn09iFH^ZrIEi?88kdQv!mjfe`-DRrT zWI}y;?I|mEtetXZaTi|38%836q6<(R|4@@5nY3OU;xV#cb@D~NWr~&ZBmx*;%I+h6 zE>KJ`32Oey76%KiA$C6H)LX|eRqf|bwb64iUNYq=5T^|`C1cIZp9P9GGv#_DCnmx| z-(16pA!T$NDb~8%_Qk{x8H#0Rlsl%Be9NQ1G}>}ivSr}EU9_XK9_zVAfqZA2bKM(D zX)P~&i&Olbh}evkKMUY?M4d|9O;zSBTyHt$@XexK7869ujlKe{rvKSO@FA61y!pWM zFog_T$9NwZbDlnt*nLLAw=X$yT-M5Q=c$cs*9*|{7lhVfh~N6hT#T95_s^ixjG&Dq za2VwvM46V70`d#(<}s&Dn)+CKWfImW`feLX^B7MH;*}Y<^Ot&g7NIJ?>4~^7wmFaT zJ)g~-F++GQB-u}rC1zL>R<*%z5`$6GcwdEwS?ftT>|bN6M%+_d{OG96laYT0d;R$0 zS}bmH6KVIhX*Z*3>`7!YQB{)9?nAb2K?Ms0X#8Qx#=d8Tifrt52o_5@K-N{z&YW_6 z?!+(1(Ntnuf;5$gu5C5KU5yojiDji>pImgGLI_Ws1-9~M?y7btCt2?fI0?E$n~)JN zR7cZ=EK@sCYKa&<137sQLBFw}vr(x^V+OC%ow@`!#MOFOaa`t4VLdtgNOR5sJEuJ& zjt>0hIUvX9@f#JxRAX6$jgjLNb$7tV#u0UOmh;u50NH1ACsZ>`=)6>rkE1?JrPg}F zxjnC*7}TjYdJR%b>e-EJsy7qdOneBVo7of@dzzHGgZ)_rlJ!Ack1KxPz3kuzO%DJQ zI%p|vcE>s=1TDsn-u&c9nHIIghvk+ea-3!PR2?a!(gpzMKd2^ufa^Wh`9lJ`R)q}# z)l6Oi@b4r_TYeqsQ1C^~RH5a4OP@l)ML`^*C0cdV98fo9xKy{~KT#|)(S@yeQI2ut zr_U@bw%oo#G&s&!HC(cd$t**#M{L}qx*HSouqm(KX!a~3G%xom#B+XHQ^7O zl40zX_6@MGpGk@p=IH|Sx9gNbo+AwCtN>!PhGQ72D8qXxMTp5&_CE^V^d#XHI%pYDF~BLe}6L&0)l!x&)9$5RHR7= zMR9POx@OF0yqFY)q7T3^4>T_6ymjOB6=3WJ1TB-{yBG z<};`A@TF;GEJ&yf3D4~GC?Kp_+Pbj_uwiy?KGZDsoZA7@yE~`2vz(|Rl-uMft5qbZ zti>HQ>%NSkBd|;AXs?dFm49SiGraq*1cX z)V~X|*OLvh`7kT3f(o{fBN1_(Tzz4>f=ci=U|fRvOL#Pok=qQ^?e`^|gQ_f8`PfBPQH?K!U0bDfZ9ZaMt+ z8MJ~gX>H$u^(JEYlj7E@_4rr>cgmkc{$qJ~&$qCk%sJH_Iv9725VU5#j#sc;&mW!i zo%TNs1T8Zm@c~@gce7YFiMA*_BVV}&Lis1%DPCwvla5iePVFFiFMfRHSFY;2LDrlZ z`3|twZ)7FT(!t9#$TIWI-7^v5R*qq_(`9FIh#-TJw=jsWKUs6KlK(y0`d%jRbQ`ws zMru!tOwToJUeXCY7X7Av7{_81Ib_N@6Jt0Dq-5!wQ5r)gm6Vzo0CYoNe?e8#K2eUj z1a#z^Xi&`aOt_4KjCm8C1kmp0#>RJyf>+r^tCitfTgV=Xwk)Z&5aCj- z4k~y_7;WII)laQlaatmUK>2?`Ys1He!gt1oP|HIO<~n6JS)EiYGZur~lJ(=>sRM;l zKqC#!5{%Oqt6kqlPdA#gef!&Pma^-DWjVQV8+++S8B#$PXFe52(gZC+D_aWi&X+}oe^ z0I=<`D;R;WvmcG7G-v+V{ERSuI1xMy&BCK4^TR}5RE8D~VY0@|8h@Fst_T94P5n6) z7Mst(W8YnlqpyNBW%r+Ztm=nklL0Gf7_QA`^*Z_k2=m%V0OZktRG@EC_IkOR9VJD; z%1y(hr;Uv)d=YN<)dQM4!FWXCm)HOofI?$OLF2HaSmS*Z)p37wR^#MfD@n#P)**g7 zI(~I&P8kg|b`2XVWyW)XTX8qbt8}g=dd7B0lje86xeG{``g5wi;_zXefev{IVnG3> za_D#y{bqQ$g4y&DbS;b{W9+##>&HqB&v=w1oKm{K(LdWTqq`gO9FJ#fZ=pM-JIEd< z&K~{o`K$@I2_5>R!s<3=wP|Z|eISQ8r#oB{N>>4*e`ENbub)9+IMe2NF!E|#mpsV>zK0?B)9TArn8v-z>Mb9lmD2D%>xKclrY;O$Q?{{tv2lFmxbXr$1e5rvvVRV} z3tB?&${5clL3#MFoy$!^m3L}R`1dQ=^=H~|c~3Q`Wt2JZ$1)nJ8LMrUu>k~um|yjF zF;MUv?No@hYE(n7Zg&*^7Ly+CQfb{V-c2$d+YSx(u;ixt?Nsh$GE2Z)EYRXp=O6s> z;jzrMrx3Vjj&6={bDy4x-<-|yCCxD!iF))?FMtJ}-)m1*Pq zO}m~u`Jwc%Ysf_0GHa!M~Rd#O(Hq zCAho(8UG?EeBGrnsK=1kTGoeD@ME>aIzB*ld<$t@S!}vLu}R6PwstS&xBBf4J~LWA zKhP5$6LiE5I!VgOA1n0?dm7zX@pla;cxEcrh<_+tAvZeMN9xnJb?^==XX(k-)w3fp zN!?r%9r3hy*Qw5qU(?42b(s6t=}jCJ;8+zfJp6E0lbehGCK}9o4X)U3_2bR%^qk4F zu1xjY%aYEpym#^ChGlkkVlVI4LzVyPu zm0dw~Uu#+t%K^@{(blHtdladAEGyttYXPGoCeZ-eq_RWl1!o6p6d!U>dvG)T*>dpWu#7)HUz*%l7^mBpZ2W)5_G1zWW)s`_{;5PNJi|sQ#ShoM^YE zfkefuplgD+EWvHgYg&_aIT_lk)sxl4S(k|6r+W!3F6;rWfw5+j!&qe*9mvs$0(zTAbHoI!OiCg4EdSBJagUb&Rm}N zUc%1b$PMdpJWta)Rs}d1X_Hk;Yu^!Zy6-8;?b?B4a*}9JJ}E2sG6&U4y#sJUeh>(5#t^WJOA6RfOo zz74vZjz#!U zFPGgG^=_@^dpP{|d=4vp8wT6_WLN12JeiX~)y|+2P6T42DjVW9Pk7CoQ_pgUKgB40 z5$Nn*yUI*lEuR3p=@w7Eb}%e6fm=vxP*=U0Ru)72wl-IbW48;1xf-3t3%u0WE*sb+ zKI9nJG}pqV#-bRp%iz5RExMSb^PzF%&#U+mIhhX*R&^+LM9>~h7Cfx9FSFaC{<)XKb&%= z)%JSNZ=86yL0`k$geV#8iZE-P&g@YWvGc#E% zI!=E*OW5q~Z)61Ji}`I|l{5|yO*uyMaiL&Q5rRu zxrLe7a$-(#(eMFtDVl{I6S!$gWJ0mqMm+WYiaI)2;1 zAeTq5GQ)e;`L3g_#5h*xU_Ze8TSX4Q#;%qM_5swad*4j8mwmPN`c?2ubk7zaTI}aC zzq-8({#~WXqZ035+j;Iie5j~dm4C^Dcu+TUDX&rI2cW3Sln~9*(w#|Lm~0-pg95L* zrrdE?X{ea%$Wg|nqJ10D@8Y=U!Gk&LBIrv_=ZEx;QFK}(oj4??T80o>-C|t28dru} zgtj{4B74`bUTBhNjUi8K?tI!+9Sqq}a4Fch<v(xrrzdY+OmJfmXTXhm6>v_XJ4Z2ih z9lP!_J}loLJ9_m=2r;`bwQLw&G>zo3SU310(7{>q0AgYb96D=OypQ?@_bn)8>+SdP zv*Xfadwbe+iHiQTvCl_IZvo~YdAfbrfFY85|+pO-iUkzR4CoeEfQTwEiR+A=Y zKJ(~N=P@qtn!^%V6?!D1w*mN21wTFI`0&VEvCo8FK*I@SPeUg1&3rcrK;THrdv zwHR@+!W&Xuzc`i?wzBf!n4lb`-9>A)D+A1ql@&Mm;zOnEOgnT`+6lx4otv)wV!!ZO z0^qgBJKqNwv5>`_6_Fs=9knY{C|LYE_XvDZ`pRce6E!ryg!RPpYKx8h4$RuGkqVQm z+s+Z;Yrzy4mxc#N3L|{@4FMbRfuV0%bS%_yl2F&WNax1y*G@+dT$dR4$MJ>&tUF8^3NnOXG_7imVDRs$wM zG*QWBCQ0heH){D)M-jp*NkIwu=!ZXquHa`Qod$h*Iga_^@K4yv7U_9gU_cfwjfTQ| zxv>$L<^5FchEH%eX>8JHkLs9er37|Q)@iKFVUCtXtMEsXUmiZRByg}te0+>q*1UyJ zYweKuVXm6|pW48E%GV6wqOT9SPh(*5Gi z)gw-S2uU}*H!SrdXQ*6s4QbGNnC|uILk>xTw_b67bkP&PIr9T0R^pP?u9HIet}AGM zj56~qJT<6m>-y=SqOPF=Wi7#8#WsiB&f&!T7%_`K;}Y z;!@vuJm9GxR8Kio+WdAJiKX8R2;4P7Y`b!6NOK-`hGKbvvFzJA@)qkI;`{W_3l|L9 zu70t;hRY^Ao>((33w2|&^>h1bqlbF^=$tjf3a2sql+9W;u9X$R#3C_#Z`=Cw0AqcC z2z~gyO;Ul)F1%uFS@e|=qpHubTQfbe?tWbj%5Q zvAOJOaF!VQ!+(SIUXq^4n0%I?{dunv*DiLh+7rL@@-3hJR?m`SqI7;!@%h-F&nFke z`6|Jww+qf(=W=dwZJPJS`6@%77hC%qS)_JX*t7YRAk5}=f$^Bj-vFQn2Z$ z-zA<|MDE-8e|N@jP9?yt74^?i8vPN>}?1Hap!X-@)~0h7)Yyzc5kJx~||~va+8D`}77G z1^?NUMtT_271SQZMr7lY1rK1g6C$!Q=Ni*q(L@dMCwGZ2Q&8f83+s+iQAk+^sUut+ z31K~wCL}2`x)rAabA*!8I9 zHebQru3sKz#38L@r`q~Jk(Li0Ml&zI^l%Ooa^~z5te+v!S4$tC$}g@3Q5t~bE?u;V z$!395@M=YWsccDK>bRh&g8GU?;K)0$uSfZbr4&E5FI%pb5v<>I(1x&`qJ7b9I++PB z^LpZ;uJkXsKu<)hM9oH8JhCcykI5LSa=*rI9ae9n#EsFCRYY{xc;lfId5I9m;)MQi zz`agQmbE(`zbyMUKTU;wQ}(X@={_h_yr&O~mEP;p*r(cy9NwpuSFgElBso`1@N>DS z7Bctlg*^I9qq-V`>%o^=(Ng^5{?t#+ZNUd@rj}vU4{RGTc{G}Df>fc@`bboP)0MnO zLTPUk3htOOG1g@KQO_@cGkPbWAro$BCz85MygPJ!_n8RRqGyMACgvcuRSrBqUW^n%ZV0P zdSF^=8c@y6cINsC+3|gZY}c7cdC<6+vHdkvHzr?)x@t)Gd~RJ>sOFcQuZrwiOOqCE z=Ta9M`~tx4wYgHQkKZ+;pnk61FXBI}GI=kyifgST*JPZv=W4}wsS)WTCf&choK9i- zGym&CPMy>|a-&9>hoEe_($j=hlq6nFC%VoXx(Gc6c@(?FKcMbyPbH_xidcyJ{)G;O zEsTn}rm2)IsG`rMK2SSnTW_=b?ET?KUH{wD`AKObm>RiymBh90 zRXyRWGtUSE3&-W(>Ly909#@jho`J%6$oHGCp#tq#Ja#NknKKH6yb_Hdp} zO`a9hn+0~ZTn^jd7g>hXj2gKf7eUBT6_BOLBVjo7nY{{PC1-;l5~V)#zS*%Ep;0f7 z6wwo}@h1zaT35IqkI?5@z0@p^O0{ia*pGcH*Y2ru8YScp_s=b^(qZK{*p2X{g%}=A+dDR%&n$yqO1zZ5-2>P|0Sa)h1v^+))~h(nL+ZM+v1KNmU)p2cgS zEjx3HX0oGy&J-p?Cdr&vE+xBpgp12XD(&&*J{LB#9a2tG=4ty;*{Ko#9ltNvkvZpn z?TkHeRb+XToS7(S#Z}0Bo0V36w@qo;O*2=}JRU;Lo|QEB61O<&ygY5q>dSJjJ|wnS zTQqWT{zu@^uUO5foXkgF8lg?wTIv?dpt^=$LqIj)gHEqr8&Yh3=!k3{-Irv5}|JrgwD=2oq%eArSw%dc5W$nhfZP7|^+e+(5k z<6EgA8EFLwOP}soL|?UJ>Zw?601>2o#Nv__{mxoY+2+g1*DXPXM5#@c>i@PV z`vBfmQ_^#iVsFQx>CFXh%;KA42Q|^x_6QHr%u}Zz?}>?N!R5D`qdZg*PQjTJV9lOn z(AiGSG)szq4g7GFwO6aJl}b<-l>Se?gt9TpznbdfbkbtX`cOx)RBkEx@O|QpM5Q!M zQg87MY|7CE6}{Z2U*V5>zi4$#RK{7e26hGiIDt#KY}f?GImd5;^v{xq3d7WGOa8!E z-oJpOmR#%I-_dzYMlvD(mHe=fU0sJ&;VN~;~phh^xLK(p$yk1^K_$5C~Y$4%{ z!(C9RtG%=H|AfJfa(VhHPX-k8!SEKmd+GUNe>dOJQ_@=%rHc+6N*^|CD-%vRON6p- zildpFPprvPxj$sRho0*^G!rb1ee^(G_JlRHkT|ND7s$+5cTNX@pj9!iH{)a{_|8Bt z`j>)=7_Vm?0D68T?(^CwHZy2Zpa06^oxOxF)iW*!mug2*2R~5fyDp9zNqc&{yE-{< z1qrb8uJkV1umh(phc(Zsj5`qL0H%6qn|NH`dK+f7`GCM3gMNx>>i)$gteo2#8bK@6 zQm-9a*mXScea9oUfQQ7+VfEHDHETy|?7zNSn7Ptj?b|(6=lds1Vyt3?fLn1es#!1r zn{P)tyO*~Ay_fD}{$9!Eot^3*!g+u1&%^v?lUc{W6C9s)bseiS9s5Tft8|w(CdwD} z+$q{_Npj>{PZF%SLj21+20tpjl@GQAoN^FQN0&@6`EU0l$2=^S+jZ9eTsmr$k=wc$ z%Q8-#{pwa#KJOO(LHIWCeW1gS3F)a$QBUp*1hSkh1{Wj}CTI$6owUd_X^TUfr|-{1 zWA_rgLMx1XvGW6*L*$pL@EeWBeoGyTcRf_pNPu*NYzhfy_LRr|W2^BZxg2fKV9C@CcoPn}y zYeul|!rsDjaCmmJZ`A8RkQRZjL`?&o&xV}8?=23WgXrHUXZ8=30?Seeh9NQ{HEhXJQA?Tg6qoe#|21(WvnPRxG~SpuGn?Wq0k6Y`*#Lg6msckz+Y_C9y99=#bZz2| zGS-*v#mMKt<4ex_-mR^&{n`2Ye-Yr6u9xgnWA=^bbl(LrWviahS{?KSBtN35db7GU&vSzECxZ}61 zH6_)f$~UPO@Q$Ih#yCd*-!>s6~jcvZN}#qAARR*9h+aq{qHi_>k^so z_Uqrl56`#Va}muE6jSzOE5SvITlcosKObI`VWY912l4|HAN0;TCh_Z#vzJC}-M`FU zw(-s!(yg?i{O@yN;PE+$Ns5nUZt$T%$h<2t9Nl_bm=dZZ?7MqQzX^!Z@8;GuEOGUc z5qj7-e`eb5dAt)T1<;pc34B;?L}hQ8PiCyS+7OUuQiFB?eoU`L?7c-GW5qT-yWC1n z0I|J#=eED_RRIOEZSV%e5wueqnHBk}qo*%j!6=k{LP%VU!G{m{I+%x!N1MDA8!DqFeQ4oaCv(nJeZ}YTvp5whb-_`<)^|!)`Y`?l&D+h}P zKC6Fdv4xOlxFv|js)&Cg^>&Y@e**F0#CYo%{?hYRFYf(->GbJFY~JM%WR`bTNEr2MwJ#_*M6S)$dgoh_W zhmHS@clOW|NhSO0G>kpH)KHnn(Qh0kVtSX&BE^jMVYMoZb%H2!WLWly9Z9MQG~1@Q zX`C%hvW!hs8<$bAoXxB^Yd7VByPO@mL$h+OM&r!5Tt1`-&%ywer^;M1Z5J z@~Z~t2^lvU!IuQS-GHx6cliv_a`nh6O6EUrODYE3f4Hb=v`0b98G#68=G*vIf7v7} z#Gq^G6T6*39U?c&`oDx`e8PIS1j4^uKxn;W?vC@ZJtWj%BjQ5%?K1s(CDJ+2~yqE0tBOx1B(xUQ zW_z|>Yp8dEuz<9U;))x`l7CPzB_}D?V->V(H7x3tr_n#F6ilmGP&z>55Kv6H=XG)K zk$vHHv;Imu)Ufqk4=>m@{UOhG6TS7h#nLT@YR)aqAgCIPX|MBY_TAZ*@JKn1*p-DZu`N0oNH0Aby?z`5FvDKlPotpKM(rJNH5IQt#^>9CY0UM; zt>Y%QkKaRyTz;PktfIt?H&OPge{;<(zHz|qE$P$*S${5Wj=6da4}Ab8zbq z#=Z0;QA3bP7mc4`}1h zVA7%zD$w@KAQyN8z!0u-fMKVfW;2jCbAcsmm_Yg?zgm0CX6z?z1!4nm`Ef}ZQ7~Ma ze8(@|qz=`6^c#xE2TTsljI|n(5ub{MA8fyX5U~JhAWuwU57HZx6JP{$QylA$76D*~4i41zV;Kf7(esuK+>T27A zZ9W2v!11yvzWy&;V!#)(8fc|zNuIlrxR=L0C8ycfz}wQYM8Ur80#-@iq=9M@9+`rj z8D9!dY}`<3nPoFd?NDJjUF--|Qsy6ityzH!N#Am$W z2-GM>h%ciEdFVISHV!vY5YAhLoZi z^}xR_OncjPm>3?)6|5U`uS8#{9)w!KRE!)rk*ec=qCc)L*_*xo6NzT?h;i8cAurAu zC>1wy_*`r!oM+3>trRbbp8Ny9kVv>()s7G<3MBqzx!U_VRvd%skS!>YK0ISL`GG<` znocs}K};d4ys2+1Tf-!{$8tdI&GkE08xEy|&c{yg=)Re=Jd$H|(X)SJ^Av^t?Mj7y z22C@r)LLSiBq81)4eLWg6-s%_E0;_I0LVRGC@_)*^uPO(fF)zWG)2>|dm^BHIeC+9 zuS?H;WQ5;rvw5GzK>tUtzw9sk{w`$cWX@);TrBIV{;BR_FIWkzZaeeI*s@I7b z<9{Uddu*9tFuz2=<$~H~3Q)kL%+_Vhs>Gz$cDI>x+;j-1$n4ePa*qocj2n}K8=~3n zCt3Zq)=Vt8Y-v*0gl&#dGWCsfRMqlM2;ml!fgFJ~Ti)6WWR7rdps?Dre{o@u9a}X< zg>BB9R#RBe6T)9(lg9Y`HrIL@8*^)>g5x{nq#GipozUl^yLA?Ztl1`{#}1Uy>kr^7 zLS)ke?vwgmiV`;zH;~AGlvt9=B%zwJGlxLZ#JsrUT;aWYQsi6yIHt8QT=_O5Syr)c zYowj2xjRo4HS~YE+pGOvKiij#5$TNpyfUt4M@x}l!i~13cHaJH?7mOu zOA+~LKN-uSPsKV{eRCj-G;NAy&QQnPWyAU^FCYQ_qRYlZ58~ll8ehB@>ZUo3=DZT3 z$&{fC`UPD(zi@URPd8&83y|BGrVH!C?s8pM%pA>N-Wub{&I(?8R=x4f4yIPFl(t`` zg_=0F;!n<&U2v3Y=ibc}HLj!>Zi(tD9e(Kw!Du<}jT(?hMrt;qXvTahLWsS*Kp$J} zo&=ZT(Wkamj53qAug5%kwG&a~M%jF&yTdFNTdeALab^D18K1R${-f*Mc2#Wk9Z0hs zH+qIF5_1glh5P>()LJ2e2F2Lbt6W0$h*O@QSW;5!`*()+-{|ET&SuOBOd2*;7?tin zIhw$!2af%Qwy!J&?z-$ogFiH1ece3Mftd7*K7Mw5PHO2bqttCAS3A*h2bU;(^Pipc zFr)sOx#~r7qkP zKp|so4~3 z%4;8|BOmclI}c@q|9dDN7SOu7!`CGJGu>oYo(Gd25vTtwW(PHAS)QD9vOHIkj&PWp zc=jIpR^&Kh$&86TIl&0kF`ttG^}lOaXAJ#Nf<57Y7G^|`b?1z&G*gtXV|Hne%Xh4( zs;go_B5rRwxas9}m8`5YR6m&mHokZzQ_sshQizb1^I@!@?((O5qn-$YH4l8v{q?wM z{3r`PY&i4+{Ij}o^FIC7Hf`SZxWG?rp^`ZZ{8hxETFwuM!ECHBuiBc2XD{%IulMF3KN_bhNe z`t)9-W!E5+zy<~)2{WJe^WwZxnXWb)rFM9SweE`W_3fthi0RL5VtoFt;~5&wR+5}m z<@l&RHpCz0hN|ttX_8Jx;piZVx40=1v6SdrO?Xi{UttDH z&#*xP)&xkm_$npp!FRu8__6CzLwxc+LpMfww~9r1%c||U68|7M65&xI$A5Hp#xVut zoP=4E)bEo=Ru4AGT8*s%rHZ!f9Ts6xo?b!cM$?5DmfsDhR9^&H!gVyFWEt7;oDe^U=ga#zPDDH=OO0yK|u;V(p< z{@ucN995xbS`(ZF|K#jmI-{p*9Sj0ySfT2T)zG^d)N)qlByrWy6+z|9zMsJ%Zi-6G zW^!kPM!(6KBjQ+m=n8S>GI(Kn!Y#}yW z_HGwj+bcDq>!}j8fx6B?$QLCA&z8W~m;LMqk64k=DoQr;Y?0bh*vx@}C+RGQ{Gp zzRCb~p4q-95m*-bbo6rqxBvOgw;;dIq>@@@e$#jF1k7VGycj8m!g1`2QdkwV8BvyQa|vQrBgFyH*S??hn zA(v*#_8ap%IE^4duI9x)2(hyrpyJ|t7DVBV+mek=;oS(eooBo62wi<);jMIa!^%WbJ;y}BCl&v7uoNE{XsEycJ8NC(FjYusGY_8h%-65 zN%PP@9=@)%mZ6kPBkJCb)TgohosLplnXIwWfSI*Wypn>py0EKRh$(crEiht`84_Dk zVXj(8E*$9`o(+(RudYP3%^Xbxt;v-hU-_){XPH|-cWnMuEV6fUHH3&GrZ~}f=3#Iq z?g7%D@b_na^c@Ei=ossF0@J(g%s)VgVySVhB zqrdmtEqwtxl@UyJT?%H0#(Owl=y7WeyG*EcfxY07kRVCGK%~a00ND1Sk-5W=h_l~JLe%?6sO^a^8*o+2CW9E${mBv zW;S7M(}iX9T-hm|ON`JGJFtbjTJ8i76nFJ}OvSZ8jMh79VIAW|^&Y?Pf;4GQi2U@J zm^1@}kWr!b<%_2$02Wohy&H$_$FJ3&vyfjvrQks^)szyyn}O{Z;p>ADQoTGh_{^T6 zbv7kU7>pD}xe=7u8}9`jjkO9|sn#U!w+kKT;;qJ=ZWu7S2533WOo0LJ3pnurGGH#W zsry8Mul|c$-ZF(H)mPC9!K{U|Ix6I-OsJ;Tybvxi>46A+G_{Q`7*!G0aKJ2H_TW$n z#uzrvD5)4i1%9w{N*l{L^3c))I6X4G=f^#7mKf2jhi+G}^|Ssd>6;DQo4T^pWh zLM$OjOJ`jP#;QGo_ptt)&&pYCM^lS_I7uO;V{Rt0)kOUxy1#%pegrG_bgs({OMH~& zzr??qFj6wuKZ>XWr-q)+ufHBi9;X?3q?@D!jZ~?1OH`LT?b5vshVSS<3>oPdd((Vd zr~LN-cYo*ikpqzV9_BJzM*rwCX@tQGIIaNJr5{oxRadwt*KN}+*=Jm$B}bp{jPJpZ z(qk#i8^5OG2d3()&PcXa{?|>1VTi$Gp7&kZ;DeU@y|+7d9bIfiJ;K*-LwHJsWAAIB z1|oFHvMVa`_UdxKt&PgTB<{UYp7ZSDZp6ZQ%T8pt&B!fzHAk;ub}_S%wj)O z>>oA@v-fqfD0C^RtXUcMm55yA;c1!|hq(^TS1l4w`lk~p60!Jt!^=}j^2kv>*Iu{v zHgQBD?GIY-=vTSZ525$KhXCJs9M(hd6ITZ9rHlSwH1!%b&I!-#a+)Q=OD2tzRK(eg zmqtj0lo;#hOTgxcU*m-{D)o zCs%I{5gfP+ZJhA=PvIw-31@83Oe^fp4j_gxK0k#{nTrdbcss0q%{!;<7t2eDMZ3oe z?`iJWuebE$*vMIx#z~b6#FBuU$^o^>#~)K&2gw}=KfFIr2Wo96u84rks#KwyJm)(z zEYv)!IIA=ME7WOO{Js7iJ{P;0&&s?y`{_(wI)&f{?~0rfI(Ao{3rHK2>#@GEr;pDE z^J$+2*se{SyXkGRC6V>uf%!$*`%WrPx;oase+0&rO5>b{C)fyp_wk*zaoRB zmB^CQr!c!zh=)&{h3%=g!X|A7fkmg6D~Cx z&kPuEkxh!5#=Z6PWPg{Npi_fvkr+SM>%Wp*k}k=!YZ%FkoRd*bn&;a05k2DAvYcV{ z$>F>rUc}-ka0NIz))RgXEuQPiLyiWD+^`$Zwce)RE`YLrun!EG3`u7AGDioc8hMef z#tZ_Z{z{m!U8*j0xsO&(iTjbP@fUZ8{FTXF`THI75tt9M2)O^W#64JT2HvuAmdgFv z3&=WLy~sYcE+2UBePa7IC;09sOjEf%f@e-$1oQ%c!$|ChxxzA7fn z{Y-^kdKv!V35ulaRK_d}1pV%tk>paXZ)H`TlnR;kb$bDdbQOTXH%mKtZOpdg#L#-9 z#GkHs`@?xQ12biz63G}E`01&f63?j$iB{Ko;wjTi*&AiUT_q=;EW7@$vyIW+W1=s9 zT$3jA@&n#DxqepK+>tyU$jj{rnh`K8!Z_(2rwrW_F+16xAO~Sc2gZ|qPC<>|+ZJC= zFE8N_%9}#!{_yjq0nha|2QJuR-DQ3ia(tu;s{UIN>?f|9#;&*s6USncOnZxj75}su zx{Bt<(vnO}_5f)CV@y08yVs391Zx)u&q$x)W^4?qu@)LpaSv|hT2}xZ9$A!8BgVz_ zCH56`2?aI&sO!8{)0eOO@FEI;<70hD*B&=x&j1&p?#FK0y2T0=ioDO!7laxs>tWD6 z^m*>Xx7_|^%z`i#YhII1o03+}6ZHU}6tA9&37=!Y#phAyQ(v9i`jpsmw{+!!`~?M1 zv2)?)*w3XZw3PdIS^SK17E8HoNfaDin%HYncvDK;>UdsGl>%*C9{WiUh9U$b9zK9H6R2w@M9se`v;w0{}Tjpk{M$T6YFH?Rwv*y0x zF7ohH!rH99nU}E04E#Ow-&XEJ)0{gxd9i*7skGSbdxtG7d1sJ)WdkUC7zoTy-HW$4 zATB|x+p5)Drb&j4QMK-1jT zo&L9Q$s33^)1?KENf=O+2D2dn6pVTUP;mcl0|wf6YIE?hxMa*WFt>lIt1BJYoh4VK^8#50O0m)ptfzAHkft~dVZEJ-4 zo&OAMX{$zSyoV{@jb(Whc|R7$p`jp1lf`DF04#&_achFUq;eQTS(1QVPqMiEyMi{W z3yfJbyY-XeSW;KV#Ed+)+M$@NNgmCu*J5{=oxFeW2q5g$!eu!%ZC#Fvn`GWFzffkiD>xlMLkvGa!TXTw;vl&_sma_B9$m1idD=n z(*}(Bo{M*G16C8gK)1`#gU;^@cUkxjxHK7BCfWl*WgPvLI>bQDbmT8ncQhU~H!`)4 zF9uxqZNAucsLt!bKDeRu1X|H^WFsBALUgJgHnY?F9l!q|8;u{^B1q+l;TR>24 z$H@4Deue{}xO4mnl%)OCrhN2sx0Vu{OtcGrV1lz(WcxVMq@Qw`YO^rIW)yGCG4_uMP%KUozIl3QuX8TgwVvFQ{w zyO(QHD>J4YBkV5J8=Po?Fi4vXo;D6P(woiK>o?Pepx&1oTGT0Z@_cvL&0`x7&##M^ zeqa!Fo-t{;xTs7dx^&NrJhy2tt%gmn^Kf#_U2Q@s!z(Tg$9j15^a1(=1EPWytT-IdCk_d%CJQ?8C9H~hm8^+r!nYzl$ZPB zRn7~bfJI{n1fJpN1r7 z>J<3M4sL(+#TLe=TJ$KO>^wwZLSfP)Ds@u@ms%#l$Mv1HnM645@HaLWs9aq-P7@ma z5;(%b(|q`=s*%zW?pm@FA%K1q0qQR~XGvoJ)z*gC(9;ee$`CG(bYZ}N!(=v9f%Y~) zmp04km}j92;^5r42OAQpT%l69<4d|Y#7VLd>&9_S`(zq62pUiyu^x~xeGFOX=ZNB> zK-oA0Kdh*I5zafc*7I(K7P05N{~{x-(F=GQ-zR)h0dV0MlYgW3+NG(h#3IS0%}@3g zNVNe}(iYn44YR2KQ3QlBS)^WMv>vzm>z1F$Y5yo{qQ|gLwimA<{_(#^4UoPM=%%cR zc<&@a*)t6V2W7p!M$Er2*}0?ArAyj+pK5nnnfw@PdpzGGn!1t^x_4RC+n7+!)?7V; zC54}K|1$5OzBp1T%-K9bNbLV+Y-yfS^Xqtx9Y@0{iX}j3!-0)?02S;w1Yqn z)BiAf;%U~0Iq~~9giha&$5Dc#PfBZM`D6ED6hRU>e&H?FQ7tLHi_Ts{A5RF2*kXOu z1olGR2%}rqkepNX#kbsrZQ$g>sU2J6+ivv=i-v<%@EGz6y0v`80JW5}nb6f*)}bE+ z$`Lx`_{o!Ml@xPw!VNf8?)+UGD>F!Z(5F`V|5*T0VR~Ch(68NZw02lu7ic$@b1Etb z_CVx|zKHVeCwYox(%0f~HfxT1qqpa-H-+kVl|}-i0zu>o?HYqYkSDxG8fB=`Jv)w> zTSNYCMDEy1HIg;C_A)L^t?J8LN!D{G|KjOc@h$eq>SoLj}m^x2_Vq5r9q44<;Kfv1iK_y_3EF#*v1 z(SzW1H+D0jyP2#RYS+oz{5h+Ohcd9Aq#^^Wpq<6A8j-u5_mqOKVaq9T_zxjbJsR=n zSoznL8dGaieE_=BPMqS-4&0BJgCn|Z;4^EVd@1M#Zn>KVQUJ=~k7H-73x`q^@RW(nc)t

vkT>P{TvwLeua&?9ciMSrbhx~?cI5)T zB{)5(v$gC$*U(j4B)m7nskR$~-JX30Dpft_Sc4kj{<6AL(|6Vp7jN(Jaxj8Z&E$O1 z>GuS3ezTr2Bl$E<^v3*3oV`6gAjre2rX>L8WbMG5+ zu!Q&~kQPswLjjlf&c($XeI>AodVgW9Q~WVzfAg$XIylrt&Fl?uKjhh=e`yXdec}H; zx&Ry3S#jU1@sG%sB>6y1Ao8AYVAAwpxh(pt$i@OJ@Ks=~J4)Kpzl@zN_GzWse5CM` z?+Nk4sG~wo3`YXxd3$ndM$#i-YY7KY-$2mMa^r(!hRn=t&U9OLpOwIi^@oc~eJaNy zT@|?;?~&IAWa$6y`FO;PS#i}jw(|#3<9h(dr9HtHK;%9ewKTDj6^7vfU)s5bKzT3U z7W2D19NUp7J%Tcs&Iy=7*=ptmN!^`yIGPxc{nn-CUi#;$i^o^>cdS>oH!1_Si*Oob zEom>#vTx*@_>`8h3S&=EI8OZpm(a(xe{G*zl>}4!IPuriT+#e+ehQ2Dlyuy`|O(J4nqcUBsdg0z6I03@GOb#f_K2l?@5FumJ*VIOvP1=*W-_dPY1<4 zMyk84Iog1u8?J@P#Qf-GsuH4*Gj(^F_SkHCjuPcq)XO=#Ec|y?QPDsW%-7VBv|cStVt}W=bY46-gJVZ3y?a;V{9zNK1*SjGj|{puR^sIQ ztyLIhxLx^S=Fa)kF|&c<6_>`neE0IIr&Ii2OTdwLVr$A9Pd*eZeI{|C8;y&4JLq?S zs%O&?F!k}M-w^@m9rum;N^#G~OHF*+19O6WGTplOfLM#eG!cIuIdw>%E+8jQ=shPg z+Pe{M@CiPY!`zF_t8ubZyZ)5>Cr;a6_1#Pavs2MUMNvjwyD0%7eK2dI!nt?>30%&Z z%?L8(1ypQw6>k&w2vr)YcO!8*(+)FPtNW-V`K3z_3HZ(;z;)ipqv7B^8~W9lfJ+U5 zV;}l8e6~JX*dJDoG-(MmO<4I<&qUUMAHxjy$L5J9rWCuj2RTpeC6Vq9#7&0SK=XEd zLckQkE+CR|!+ZCU)6sH8FQDrfmsc5Zl^wZL);r%pc(m^Nr%2!Z;a=`)=RFBv2due; z(ITjiBRvkP)u|J=#@Z#S-CE-D{QD`eHV1N+LeVWaXhwIB#(I`&`C&Cuy@DJfUG8>b0SaoLif1R0h<4O z5IP2ptqSUA_3WV;_9~yA;CRDs{#>crR~Ua~T39lcslLS9-$9=lVNp_}Wwv2eUcOWx zl|FRX?rOVNr;W)mL(C80c+>-s_-3m-rn?{htPnp37JnGBmb%L#cm;I2EU};MbT%(_ zW$QAUQA8AdZ*BvTLaLQ z+q(UGPFyKVY$6vKy3+2OO$WCd{t79hBf+d-sswVQ#t2N7P;ccV-zQ*khODe}F8vW1 z>K{#!)JO}k>56@l?_4}N%(MDqZGa@31fJ1ULj4%&@VK(dC>|L#`%Xwpv2BOmeZ zg_^1FCMDyb+N|K*%t z)3r97)<@r4V)BzaC{rd>2(p;wiL^;V!i8QFkOxl>4-+1EA9%52sS`xsFVLnXZ@Ep_ z22YaFoq@qcKNPU@*R)`p&isxC{NBI`DNRJ>;va6W*H|Ok&0_2x!oXf}O18encmGof z=toW;tPH zIa<{?FYG^oDLS3BUMaO(Ms5iI=dyAsK5*ED#;ZzhKGtaUj4IC@{gf6qdNKJM)>&Rg zYDe*M$6!xQg-t!fO;Gn zbYZ~AbK&=cV8PL@VLJgemJlB3usGLGkg=0EkGP+LIOuYwv&^!xpg9WC{5y2Il{GY~ zB+o!KHjhX<`rwu&%GFr9v+F4$bpr=z|D4#@n}yj#a%>i?kn!RiXP zgy%g>LphA=+ITdp=RKsYTk+{{F|oS*h|mgI?WrW&$T7Wl#hfcCtloo>n%41Vse;4Y zTzP#Rq^$m5#Xx(r0T9{k762mfjGEP!WuzW{&xp{br@&U}Xio`2Cd?#EOjJ(l>pn23 zXEIw9sV1OZ+y6U}?Vu+EIVW*t!%_3ZHyP39FrJjBRr&z`!>8X!L(e@Ui{^<}wHJ!} zm~f%q)a_pYGz_`8h1`+tbxNkauZ0OTWPo3uJH21Fk-qYxrru{YylL49k05QG0c{S(@aNw z4X~z1E~+vkb08aWQGMd%F4FhHM2*p;O}Ao-MJ_JC5#*2kD3p0zq%+A{^)`JXXt&P| zcKpd8uP=DrPKyp&3*fFW=-em5z)OCuDHmd( zf#d8^al}5XjKBGTRvHYneD?qNdh@6x*RXwbx83C~Q+AVB4%99sQ*+8$q_VVfpmLX) z3OS^fpyoV)Hkp-ENs1#@mShe%mvf2}IG~~;jwv|ci~<6p9=~(W?~k+AS>L~`m*;(- zXT5M=_kCa2b&GE@0-o~+Y`1RL@uvadoj(RqTY=gT_J@&=rN>*~#>&)6bZbLV@@67T zCvd@>Dlxw~vOD>0%7D&1ZP{Q0q7R~mC(~3NAH2O(Dzlv9Ew89-$}UnTQj=F?jy8MS zrP`nsH6Vdyx{7mN^^qMruKy}AW23lts9x?#ZI*bvD8JfY{u}Q}#!SlZyCD(~a4})*5q$qe4K?6E}Cc`i|6H zmzWKuTD(R}9qXxHHRdd}AyTymlAsuw)>>y>we|0id#5M-Kv_izw#%zERbsCI*<%m3 zw=}4lb1$dr>O?G!eT!Y5#tLH~?l~R18obuJf2?ClCKUFYm{u9}X3o$e|Dk@{~$-&c_U(@AWw7Wb96Sb_gq;cs2M`ubzc!AtH--j0cDEy#%JeEuX^f5SIm|5lRuNtw?P zlX-~Aex>l<+D!e{3agsUdEP}WAk0&45o(Be3@j0>l z+0&eBI5RIyo6?RpK5n$*GC!0ohnJ}28a;UKasKpzgmLwk`ipqY!<^v1NuMT%-&vzt z>_NFlVAD0br~kmD_BY%OaIo-aT6mAM<-#wOh6{j3oi_gD9nC;+?pk?C#mX0XoapI< zqUc$MOey1I!SX$T6=Tz;|$ zw}DmvcDF-4#Qo){u&QP)xlw8}nx-~a3;9x_-&*KWfNS+C)Z!{AuxTXZ#aN@WL_e zskB&pUVhlc%`>5gBe%Xm`ow_nwdMV*o2nSpq9#kQKQZL77Wch=F*O!BGGaJe!(B4x1{|9Zx3bWZ|a8oVzv zv}00j9XK*|iRwF_12f|F5>_UEn|Md1P74I|A^)gFdUmhN{SFRli?mXx+*}qJ2xy_o z{yeg|d~uJFx9mMJqn4iPf)$eMKtA4z{=yEEkd?K1%CqN;n-9SW7!A)$X}37-8uNy} z^#{4Xn&12A?EParsVhchKNvUf4hVW)_og`E{A;7vB$tgAYwTC7yTW3wvB_D7*qMFF zI6H>xC4z4F+}rapr+)|GrL8V3v1f+^J?=5i^^h6J{Iao^{T=MSVCvA3w_RJ zb;g4cld&nI1JC|64fxj&;{if9{CBAVS0M0^cMh5F`b_6phh}*li_scATc!t6(Dt?q z3yzi_vk#dn&Q9wtTHXBqYk!$;LsJBQ_dO|d;dd>LEH*n<=`YB;Wo=dDkH!~v0j1Ja z7yc{0ia=24=TWWnD9C)Qc#r+TUeBE4+07~Fhca6FCZF&cGiPFD*Quue{qkM47O4#T zrJpt(tvUX44>#Z<_qba9S-(HPhzqwou^GBmQySLBr4GX0ewCk9A0;oAB7a*2YQM@I zV?T(|5C6OJl1tGfz!X{G^7|J~SR8qlh_aQMcZ+4tb7QOi;fwXsxpd#aFIGO2DueCK z=B=i)U;o;y)jD8#{-M-C`iz6&@Yu6ItEZyuZ7tc(#zlU#ExqAzkG8>`>TLtQn2U?f z|0&D7&HS^=n3Os55m#d0kdJlRbnP%KKE6M(U5K5F_;x2?zW}KS5Zq3fl zv@osVhUa#efyUTc$bvCxCtVaf;yPfS$or1dX^q~@?_yO|&hITiIQWB}Oq9aC3fd=6 z1QeSh!>*5}(#$0H#_d^f_TF@-=^Dr{NLG0sa_ebTo~R=9PCk4St~oJz-?A@kX5xB| zK~0Y{DTK=NHs7|V-ftW3`B?$K?OX2``VJZJlxM!tF$h?FoG~h%`mLMoKy?It+O;*< zmx@YZq?X8f0J47`xnwaL_wNqkC@G(YlpF}wk8E`1xLyh6M)Kdb$|5k zqdpk+(O5LHV&E`rZfDzFZ=(1UbVlmnkkF>e8>Qiay@T@|lN4Mu-oCsCkEF|RK02?? z+z*?#8&s+CbVT)zPf0782;@>qX6;QMt2rT57;dXgPo13q%ASo5f$ z=g#4J{L`WA6ll}VV?3yIn{CG&{keV^aqz8X$nOMq3Eoc@ZTrn&~nT+sb&j z5pf3mZUyxE*Fxa+`UCN8E};S$E4`HTD94;2>1?N{Lgw@X)9J--*$wazTtybhkh>e) z6F<9)iPa+{$=$V@&YQ88LGW(*GJGEUI*uR8`O+O=46n||)dWNJJyXG+ijk^_*&nPX zI}p*Bj#v|Cbc;y9rfnILOWTyP36u9{vx~O3M6gFbUoWOAdp#Os6j(Y_U)+;~wQR#_ zkQ)mM=Q&`|K4Y^Dzktf2qnnYBf2OvauJD}mBW!-$bT?(c^oFkcY=;0U&Ogi7EUnuW zUf#o~?9T4M?r_(MUZSQ%nT-R?7!9ogqEqZAOS=8~_FTSMBz;%)w-L}_!P=Xz7af7o zruZyr@ca|IdRaW_&80UExck_9*`Fp|YWUaiRKYmWj1h2nrDzPs#FDbYyD5S*)p5I? zP}-s?RqBN6e~wFO6_tt&M!RQ#^P;~>i)wd5>sFK`rfO3_v}rIvuB+97n>|03&@K;5b^-vce)4#{ zWP&ag6#P{x+j|4dU%k`dbSN?Wo*lw}_(S0`cF-$OSy%hRdbH?p_(l=?G_Or!uNUJG zT=G+Gn|9(``E6Vr0+d+Nk@^%sGS1GDQL5?gF0LRC9m6Hv%j-7%1)E!cSX1>H!1kXz z4;lbgWJ%k&u83Tq_0Hu^h;YV>7x1bX=Et(UK>2TH+^Tx;{r!;p~Icrt5#~R86F= zbZN{yK&&s0B=*Apc& z@Py7cGo^2c2Vuwza6z^p0n`$w-AKG0a6RI^I~D$o039g>LepQI6i=@Need9-r-#># zGFsg4&539yzDGpKRpjfxf4`jnjQjCaeEjmv>0K0g11 zEZL+creg~{<0Lr4n-gye7nMHa(ZRVR6?(Bl!HKj)o_ECI8Pi}JMb^6H!Oh1r`TtX0 zT}|7Z-gD&Z)w>o0dx{36x+6U@8dcm34C4Kb4{JC))UFiwH zpP`_jQ2e?-Z|~dgqwoG4l%Dwpt0P~r*Bw!*bkNB&Oq(P60kD0z&9?WDhCTkF?z6-IK+tioW6yZ(fwwtIYYsftM*s<=p^mp5z z?-7cBUCNP6yeen9wI2}Q%{mc-^=N!|u-<|0a)JNYK6Gu^>Q#|G$;IgAQ?oFq2IIzT z_JANH)NZ3@B;2E8SA)N8pn0MOMOpJ@zTQdhy5u@Kq$@LzBQK0UZSy#YpV69LU6p6f zw99hlf9FY!?3$K!#0MrbGgL)%3c+WZ_|*X7xI+Dazs9T;(OOb<^*Jc@7y9Oe;9La!$V;c~G{*`KAX(J{XrIb6CQ+M$R z)rVBG6BKmcv=#~NAmt*~n%cv!)I|KzwLCmS_=_=9MGoKVoKcWCl-u3v8mSC$Tn7GJ z^$H=(Fx`9~YBq4vECuNx$V9nWlhk^F4OTXh_d~yGC1U0V06y)$TH}#lL8vUwsrb{( zl)Yi)f-J+A59SvNc(K9v^RI$oQ5av8f(mqM;i5a#zyXpq?>w^jd`U~Sy4gqaK!M&Y zafs*S75|9&&Ux^&Upr6jLE)v{dVei@H8W#tZa6zTfi8IPJ!ay>f$rVl6Q@t)gx$g> zk^n%DU~DfGpF9A}p5Fz2M)Qn+;Jjfgiw1q4b&%Pcv6ei}gyDXPsLEO$`gj`r&DP(8 z{e|M!ooZo`b0Pw)?i$dIU)Ix-WvDxm_M+|&OXkFU-8I>D{fOS?x`r`r-wYyCZDajQ zYfYV`;p(Vervv=*6{{bkpKc=Jhvlsrs-6|rXI}AE@-gipAOqen;pt8}v`?e8S{?Jd zCHBRrUdE0jEx*p6yhTD^8rO4@)O&*Bi?bRoK zoNzE$Ybaq(1Gf2TrAMNE+R;lC8iFKih29pdSP6_i>M1Sit>dgUJeAF!E)44aK-}rn zWxvaiC_r(KLmf|l+@08HLx>XEPke@rx%Nc-QL0F>(Ny;z6qDa=Svpj%U_>c5+1F}i|-SW}m3jBje87FYjtb|HU7zbj%bGGe6{u7_7hCc4KD ziYPV7ek@_CW_a)o{l)F?Hk+JZU`Gt|)54k&POxkJzpSOl5I5Bede}v_a1-`_uN0R; zKc1}9dSHaM@+^tgN5wFO?KKBA?M(t%+fLT!u)D2|W#_5CL7=HyvtLiAR}Z$%CB2Zu zYsN!XY)X|Gf+GLg#N80(8`kql~<%aR8dC~sSh)xf`aTUr-vV59;;uAeD+*=GTG#Budc_VKRuabgPF@)qzO(; zmjSmWgUGT2D3!C4Nv~xA1;aNJKwAY{WU}(cY1FjI27A@z-ohtutoQZ0qX`k;N* zqHN-JQ+Ld}9#di#nUJGr)e!zcah?<2txs)WUEFgM`?X8+tKad(16(M7(9>WDSEai1U<}X}g+E?2wPLdC97iE9$b=`njFg$ob!D+nK+o?NU*zNiQA+Tr-`|P|AHPwZ%I1 z`^s61|HB2iS{K=s!p>g&o_$xi>;Cd!Ed=pbaoeePqCb3e*jl)I{qotvO&+n1#yso` z>$#+u^G_u06%VdM9IESnzQon}e2{%1ehaLNJ|sLGo~qB*5RpH&P@{@9{$fAQ`4fP= zmpqMGx(vY~JpGkxwNDnV4 zqlCVLh*2eC7^;@BF+OgxeDT7m>InERqQ+pXp!Ve7juGrZm4%qO!E70b@g6F0zR*$h zuM9}>u>4%9MXOp`gD46i;;_JU`~BI$##ausg?rZz$t3!qkKpQ%S(@U0`uX7MN~NPx zm}mR!pzps@XZ-d1%*Xfq!(aVmLm6J4{#fJA?WT!lz^)Qk>G=4L<4;zrse zLMJ2cUiYLz@gTa|MIKCEs9+@cG8mC?ArEoq3R8FDw$vc zaW9WqftUkx$lMKgbYA=bGw9?Cg@(}cWZ5m%1l}%ubq+oQB`(mj5Q(If0jR5mwVqP5 zhg%yC(fS>Dx~0IW^Nmf!J&wvs+WR=-JI)Qb!rLdDd842Aa!2rT>JgSMuQz7yZUC8Y zIGXIM0lw_sJU%-@6(LX7eyhSUJ zwRqSiKK6W-0VlC);QkQ}hg1luCW>>Xl3~R?^vvj+`4ZzWU)N~te}deWKYxhi#uR^h zjqsMlM3yvKU91lP9Fb%#7=YA<hm%G8C+&4~{49Q#^ zYN}84sj@_oxcSl;ofLF9jK^qK*%+!3BGiACCSEafUPie#*Gb>4FIoAwTR4Q97taoF zHPOS0`+3@qp=*yo5iXynsgXR3kxj?{c(n5m8^#~l(CZ*0#_o?hRY)Ls@epw9$4*w% zMp=WP+4!V9>ud2MilYJY%AUQ5YH#4=IDZw)h9?(9p0PxzT34=L-3*w~tgX17fXfV0 z-z>0MI}HcL&pCo-qa7h*d^?^U=)}N2$+O%~5~qaE>)m?`0G$9ajo!>^c~$=+OMNWz ziIiqTS%pMX1zGBNW#wI!eA}p`w1s@J!zK-j83QwasUCVv#BBv;KBgOY| z$eO*8wT{+O;yPhdE>0Y8x!K~#d65FaQcY*mE^cC8-Y`H?y}e`#*01X)RnWb}YhM93 zIaaG(0fQ|we{J;7vCd%f$3t~e`e$SFgFS(fk_X~|O95FjD@C1$W)%2IhomAK+lI<* zgNYoa&Q$l65B7L{>+DY>_Iewyl{be9IqROmMj1#-^KxJz%Kq$5%{W0$!C%WB?Wl`AVw zuX-BQE=I@R-BsUsHz4w`;S*nEqNx76%~HRf4;5o&-zsCJ>gSJ}j*%N@5vGc+Zo2BV zNu?1grrvU;bNI;~ft^z{1eF$CDg}$#Tfh)0e?S`eI^_J)1JT|q)G6W*Es))FM9017 zJ->HNGdc)^88S-4h;P|bj?;V#Zr#t{Wi0XMH!n*0fpHqz%D{+XZb?uQ|GE$}|4f-v zQI^A*m1+jG42mT@!C zFw-lEdn7~YoSfR~Uq$=+)5ybT9-Vrcn28-dF%PbB-q491&!jw3*;J9(iDJ!~vGr2} zE3xCA3FX(k{?ry4@-$M(e|v#%oN!{v3*avwOPXbP%k23L>sEDs<~-YuGIel2AhT4) z_B`eJ1?4|Wq*!hNY#M@>C_$(#;@@VLE(*6{s zZTE(3ak<0DAOHIPQ*s2cMW>#a+%eehidp~y@&pyVB)csYFqjG&R@Y0-HLJWKYrg2} zwv>v!QC_2vf@?s2q&w#dSTKij$_ez5umH&rWJG7zt`Mzt^#$ zIUw1Z%yX>M)p63C_z>5x=02BCcBq(gQfSYJKh0Mz8P{4i4)ekS!!j2b#1mglP^rITqo~dti!*+zFwY)Bd*AAQk zO8FQI8Y{w7x5yDsTN0`FN?!YiHr?w=+#23H*2j%6m8-cC_~vSulybd)0NrA#^y5It z)R~bJ8vzLqkPN1|CTC#~&XHb(8L1N@GSQ57_TaCWikVcx16_rVz z@RxCX4Fv3EFhXBFQfPEtfC^%EIy6H)9NQgU_n5b#_aBno&Z-2|Ps{zYg4ypXPN#Mn z%tT3k?}?LGP?PGP`8eF4>Cvt0fh;(l?ZNj-Jtu>Q|7!cx8Q8%fCAbs) z#mF3aC1mLl%?Uaj^MLfaOTYFvy_?(7E8hD|k7oMPrR&QWx2r2|__fCvRV6?h9UAHI zlv?}C&5hqVMDmo-K|}N6Rq(af9t6lwPQKm0{zEw43NmTpmquGl9w;r@+-JmmRw&?F zVmkME2kHm|Thl5ukE3M{utzw5RwTPB0WuRk@bUJEs(UEMsy8u;9T{ zEVc9`bR8ffJ2O4O8YQP+apfVTg{CQe{a~jB7sA`+cN9oGAaa>7!G>>aC=0A8JORSwh5@5r!1_uW#9i1?&$h z!De32SEnAqaNX(vCmB&KHIYGu3C>+VDpnYoS%GGo;*5sJ4Q3DX4IkKTnZs&_her!R z1=4~WPKMULc)gXhBeUzl_lBv7fqkd!XElww+EztxmP{JnWdxUYDdbBdS8vJUmnA4u z_tjrJm&~08l>7A=88GFc!aXw1;ug4Dfn4;%#HZ5m;wc;A=$5rpU?O%sCv?q}2{-1g z!p#1q))&#(>6!ew7u*3Ax{Gn(CE%vG<+>?9To?zLZ}#5M6eC$>!adS;yW*+c9&+sO z)=N=!U(7P5-HC^5lpFL3D0p9IeJB6T?wIudkd+7suW>#du352o{1E|YIq%{?`WSTb zC4RBlEn5%z#xZDd?cW1qQr0@iMq=iezqc{7IrTaLcqg#wfkYa2n05|c_^^kxB$3OH z>}7iu?xO`%nN6G118eTW3yI}qV}(SCZk~3ysH|@L5qxkZkW9;m)~cO^b*DRI#0gsw z7*Zs&Bj9hx3_$z)kOpmHX}rvOF=XE+zV)4_>_S1LvkD>!_ZAZQ3QUYzt1~w$X57pu z16ZJ4vZq78aKo!|>(;9}#921W@6lfBFo#MTGY*Lep_g=|+l>@{WuOu`mq+S=n$Yhd z0^^9^;VE7ar`i;z?uOCLZXbS_{>y& zlUFh~w^gHjae^ur*efp+>ry`f=QOp3xxpo9&eE@Lu=HkBOPZVRg zw^oe_-|v2y`b^=g`Di~zRVm6gsL<)-e%q{TxCH)17WB;Oz#hAlOxmf%f6ajzL8ZW_ z&tnzpRgCGwY_%g2&k7ljK9{Yw11i*Nq;DAb1}C}YH{aFds@IqK27iw>F{<2uS%*bt zeBmgQk!~oPl{aAvD7YHQ|D5iK8Fae)W7g zTFLLdwaD!gbrOC4(YP0qz$sN z$&3~d&tziwXZcgIrs$Kuad}7Ap@!{147c}IYcuoRbN(EtRCw8u3skSdzNCR%I!3beZ!kQA?a;MonwSX!~GxG5g&KVC$=j_=wmNZx`RpK2MnncK^LmZ4S@P zbyJKp+f6RT)hmYg+{S?`nCpW>zbsn?|InvK?l&2p-Ht_ht8A!jsCmCy*5ptM_m4?T zmb=^-zht}-fq3=P)*?*HUi~|ke*dYdlUV=Aed;~k;!n7-Vd6?zvQsq=60y2k7hGXce#tsp-_z0G zg|T>QrN^rNQBGIm%f;jc7>AM_*_|2h*OG+D(}hL|@jJhHcQTCq1KGZjd3CNK+iUl> zrP|id!b?%e$vh4JrAK9WrR|S7PkOiB`ZWBRQb0NoT1ahr`8A40ZPVfmIlYB#7KB6$ zs_Q?_2KwP4zrVnYprkz8E!z=EPwgJLUCZ-$ciUp(nQ`OS?(jxWViP1s+U5fDKV-zr zk##u50+(S|*IZ0}uqdb7T4O|qJhNRp9Qg!k1K#G8lH~LxQ$siG8PGOfGidRJOVoGmYj?9UCpr&d3SlW@yGPa+ z$wPhk{=yB=6?Y%+=f%7i(YU$WXUbhdP0n(E; z^>*Y5z3o;l=Z#HEx~Q%O4HYUER0OR<{s`R}2(B?pQjc~^=Eu;-GQkQ_J3VT3mg&Bm zvE7sTRhs8%a$yO#pu5KVRGN6boMc~H#E#X34|(wmg=P z<(1WB7S1cwxxb)>dTs@FqR*)7i$eF=9Io*Xf1DUfglKi03Z$JPfD$5Pj)#}qD~^yp z1#}zmL(9g>Zkt(;ZxJG-#7IBC4%2uj;_Hv#D{TR+fG(L5KuFQR(7jA>4Tb$ZVM1UB z6-^s(A6Y>mMGy#RGA;P6G95qtc!gD=R9VTgHw$^=WHh@HtNIux(3Ls-&QTNT zdaa2`8~k>wa`nP|r(NA#NVb=!brvLk`?L3EE^LurK4>n0+M zeHXu$%rIFPgJvXpK-*5!?^ch}!Di}oc5+!;~wWjyVv=9o!R1YC8u*ZG$uiDQ1 z-g%ESWHn<-W%^2v^Wb54XqL&zqWU zef|9n4(;p(uW}w+1-B&LJ)U?yAyadc!I*j=(vl}E(NY{#LRxX%#ZDm9e$Pr(BY>0- zMMQ1w?LXwlid`QM5L33s#>lcImHtWAfy&?)GioYLnT#DP>?n zP>yq2vd}&5DEB%HbE|V#RkTFzP?rBV^QXa`3>{=fHP*_Weq?q?M#=&KyoVmSVw2M8 zbhKE*dET^NPHOlQ#It(!eg0N5(O-wlGOzVIJlv0kcroI0Go1{3zvJ56Oyz;%-|lD( zUV}{_yiDnyce!D?JG+Qh7&-sV#-Hy2tao*D9aJetUNd0)zscSY$Zv<2@)3iogLb-j zj$#{0L4Mwud1z-K85Yj$N^h@ynAO*MFpI5RL*q9R&+Th2|IhSX2<%pH;@Ns|V)dMz zQapMiby;!DzS{Q;XkWP7D?E$S6?Cb(v{#P*&nRp(A8fTxzpgdQM%4|$c=g)my!;uj z`Qdw~uwCq@giM>z8w0%=sAO!KQI&WIy~q5!txQg^TFzJK=1{?w{dGq&SPd4nBn3cR z2%C0iyhG7c&@Q&lUKM*R40*bBy}9E4PupIS_r(KWL*OyGBbKs|`-ZG!LjCExyZerT zGUnE%U}>ZMN1)+r#O|Ifg(LC^9adzmIWAY_uQI&#kdn;w3#x1APyv z+uf>E z;bU=DWj3MMui5I`=W2+ z+MykH09$gqCPF9gn4@+2SX>V|1X9A~oLGHTnH5K*MHq@eHC44R4e95gapXTC#ne*qh@Mid9ndsT)vUajW)GcALltIjqxZ3^Ti723ZSLN$|fX`s{B?qjFxEx7nNA)RO<8 zoAWS`PW$DhcNQG;tSHkO-KHP|p`F^R$;gprv2Q>XL#y$EIWmb_>AtFxq1gKvCTfbu z*G)4NK#BvZ1QW4bsZDS18q9l zA)#z)cc9bsWNW~yXO`a+P?n0qGnwclBzG-me69^GI<+M{Dc^VXu41@9=Vh!xrdONz z9+U(27Ab0QeB_GZcWgmD_{a|}HQT||i7Ux;I2w(+m%Xod2a!3j8d6s9f3}w53bnW*W=<5v0}!cNFFHniiBHHeRrJfP z^_O#*#UbBL$yWeI`AylbcftGzpt{B(6U&w;&1LM4@M2_%(g5sSYGaz?Mwc zEa}}Wa@RuOr?#ANJ?GvT(NNYY!%F9SADp#p)BK8X zHcl{gZd))0eN@J2uy5n6tC{iO<6Lh0K*xfVA$blMs6!^i91K^j^S>Zxf&w|UTzqop z{NP*7Cs!(e9`)6od21k^3!U!8sws8QrqivB814y1_~LI_7DiUvvM+nPD?;uB1uSSU>M)({g)i3EiuTXCTnn5`CMQ#u8xyCqK6Npr>L56 zrV;@~BCWn0Gb6-b5e^UP-e+SFj;-khdj6@iUk-sIv@EX0}VLp`_>9u^dkf?SUxq%p>kE>dn!4pm!5MJpig`e*^ z*I{x-lEl078J8{cer>jFL4EaTG23auB?fd|4llmowtIsy(rv}gpg(a>3*I_a8ktuw6t z&olTmnwsLYljjHvS~+{rMBU%!8bw$nn)lcwfX-1XtsN7l{d(1?l{WGTC@Xpirk7Au zJdMPC$YClQl=Md*oA~f1rIUNdpZ2OuCyuq|@Uu6O?)Tl!=bu-g1n<(@{B{lu&$=t@ z*9&`aoqZFhse)MJWS))G3Aj5d>Ds|!IX+5MH9+c>iPVg(^ zrL;BSKC^I9GU>T?kav_E#r=$UP0L=6woGxyT3V&-{tJGWK_#zR7c9o&&(j%i_W?A_ zstM!hk4h`G-QBCJN5MxDKnmiI(Fef5?d}aZ(Vga}sp^9r@_?2l z?KW_ULD<8MqqV&Q*L&Xz-M?c)sLw_WFUKT1PYx>-#u1*xrxY?~7pfXIH@3HD8*Eq0 z3Mln6PyW8d*!frV_L5QpdN>{BLd(OOK46K57r>ta5}r6>xo*m|PQ!BQ~1fVDMMg4zAGiWi{o<|Z^)E+3p7 zsY4+*R$I|3hiz0kA0_%%=YtVp^8?+(cj^{cWp(JaeyZTBA2tZG)FdH^S^K={C=+G% zaLXO??}qLO#ot?_3gIOcv&1_+8c0?vi|!(~=bt`O7d&Z+{cIB!YzfHcdzc4S3RO4r zgYE=HVtj^kkYd;(*Fw#bXu~++8y3oVosE|3T>#Q2U)?Ig%o7e~&X*~x({JeLi1 z&YQwK%d7=2a&=K0VeNfn`5sLOvf#W4rHsx~_arzuhgExr(C4X(X*`!NZ9z;aL@M{| z5sYikEoZ~|-YIjy>HGXG`yF1?y6|S=Z0d|CXWTt;*Ps!3^Ho;SYP_UBLnWJO;7Ciu zxPO;2QKC=I5RqqLDz)4Y#qC~MbR?={qUwU}Zuh6?6IlJBer`^tP1&MxL{c{0>#*5S z+spI9kw2p1^Cp~NRetLZ_vRS4Z4DIJ%^8r%sd!>_;FS<(XMei}(VziOzK(MiOw8x~W8)`C(*5qw~RvG~B4`~=umoz`(QCpbQ&ulMp*KI%*1 z38;LL>2#BNb}jcy#QcA@_T12Et;Y)m0F!inPO|5VTK~{BK@!oYj}5_Q>CCgc@cUL+ zMryDs_w~f;Jo_}~$=^h?iBDO3HO+#G$+#SFEM9st*S8NGH9emwW>izKcDwV_iRTgj zHI#0?hE?i+>#%(oWVdpwa)FVuT6)WjL&#z=t-i5<+v!xHgp$+p!2m)=EfrQt9=T7%Ff1vr;DarvpBs!pW3Ba8&cv8c(?sI9ZW(< zO2eHAR86FSSzU7L{IWcd7sdRXdTeCOpl3BG-o79wFLpfjb#A@3ZV*`B))?C%`%`Px zienF|TzHf1ycP=9h!|+R3I6IfRiYZc(X7==fCrHKk57D2C7n{M+O?~2wqh@e&e4^E zt*ai=7cuhhoGz#Fyw>@`K#L+cm1=MvY0coi9QZN^kzZ4Xu-QK!R-4=2~JrG z#A-Lp1>t*o&{Ds}H2P&!rAg-Shwh>d&ckf~;DJ|uJvqv1i3o?LBcPrCUeQ$WI*w!& z5JHaA-@VvAs0!-tjynEg1PI-THaz6m5-T{Da-yN-Udgw5LlTrd*{&87L)`TYNC4~_ zA$~HL_-J+H^a!mH@7J^1QK1}0C*N5ycH&hKMT`^&87q7io02f+GO1ys#6!Nzu_}Wm z;o9tkj9Rm9d>+Un^4qpR++m@!oGn~YS9xqHdO*5DJg>MOo-#l?w2Q}v^fG?a0rvI# zrSz+isntFCBmmJH&9&m@(Dj3o$=5?m1BP*myCs5%Da#{vfyy{V&JXaV9oDl$`r&-? z*WqZu-N1)I?xcki(x7Uc>IH5y4%LPp4pgzCn`)kANiJ6>_t2@g8Q?tzDU$$w0hD*Kah-ep{%FT zxai`R0*>*{hi-YlbWSjIaynl(YqEW(qJd9`3IQRd$~r%*4ejd_F%(21y_2zaLKc46}5UQwz4^iBch=oO1SlfqN z-MRUrwHx$pQNuPHDwdC0*M-YjcFwga@VSWaX%~0h`cQ9XT;A|CS%j&bN+j|{bT#SvUE7&g@yi7#0b0 z7S^;VSZv#Aq7#O0x8e{s)h+T%*roltC&kKoaNQCB8r%EOXtgX$(R}#S7q>MJ(EVxT z<5_IJ^?)>83-rftLq%kYA>4O(t6=`pz~i#w^A9wgGkF^7gI`oSSvtycXVjv$pX} z#~Hy}rAX0(H~WRn@%7B+CleNBxNf{X??h-YO*h39 zdz*)$IagwGdi3 zP;T;&-N<7rQOO4|2xV-Czw6I<~whgONPJ z6wcG!XpH^*5GDniZ9LThn?G}J;9>Rx#U&7s<jM%_4kvWt#5~ z)+SZYRX~Mvp(mge9t`c3w1A9Gjx2wp2s!TW(p6pjxo}Ounsh&MeEQJ~f0P}}!t6=A z`z~Y@KMhd{@@u;00go{VQuVarO$(1xoj((gMpiqQp+v8`7wft2rM^p>TR{ z>D93>RSBSOmO1qm5^p-tnBmXK1&bF$3X6i<5Ex7>(d=XXT64VYD{n)F(pZq{@O#aI zH;D!{P3~5K<%?MtbZGYTDnA-rxkt_~xo)l)y4OlKZw*Ld>Ra zi)o2bYcwiB->Kox5H}=rd3VXx1i3kjwOflUcyh>k#If;Kp^Iip@Xqq~*#COB|IhQR zK=#3(Jow!m1Hwdc%~`WKTT**L0`gms>pzS7OF-U4z`@b6v-Nkk=s($*!b&70;!w^1 zvXlSM+W*hy(E#mLNUa+!ar0uM+0EjptFCRoLKGz?NYbbuPc%pWf4=d*9~UwH*$Vq& z<95f3t;@oi$WQ-Qd)FS-#I>*Op+ZZo9*fi`q+M2Np%o}vlt4ypD+?b`%Oj#fRN9mJ zsE7gyc}>x)^}(Zx9HkOcjT$jPO3?<$OGQXQAo2`aNYD^s5=aP1$a7}y1YPH@^T)Mk zt$Y5ur&+VIGdnYTXMf-K`|WRMO)~D7SyMwTdcu`~$cJ*pT0X!BNPmd|P% zea@3R>wJ6}$Q*d0r#QXqo$mnX5esyh5*SX0`l=p3e6V0#>cK00u9r*^z5QZz2vW6^ zLW1cZ736|OSn;nj8sMb~pBTelARHeIQG5&C47YcL&KvWZ>C#*16B*Jwb8y~`TiX{O zx<%3?YEm7Wk%h|Z%+SDJ3BMBl36(#RuFc;|Q9WpNJaS#&#krC(-sIy9*x5`799(1E zdK`2T0oWZfvjmy#I_o@T+w7PdfRC0vEB_cjP~5wvir7~Cto~`<2F>|~dyK0G_rxSJ zK9l;(BBD6${^-4e{Pbm$OQUMY6W>Hw8)w6B1sIA)_f<;P1_5V8e))fhjEiiAJ32~E0CH! z!z&o#Tku9(1;QYkm8P`zH1`q*fzE{e4Kz(oON3t5M*J9anYlZD(TDQc^W0qk97KRG z_`Cu?LgQB(gwh@=&h30wNxNW^$&H>MZd>q8MXH)`bN_Mb=f>slN_xQKY!4pdCJ7hx z;DeSd&+sRVIu;6t0WE|2Bh*;8z)8%GFY}bAteYLPks5d*GjS9x8GoMs?}Lu;Wd+5c zXl-|XVB4Ix2m0#+#|Q@M&jfKv6KuhT<%dd}RtLWS&_P0FP0-S>8az7pXxpP}gRkps zV{GPR$Vpm~!2ynuEQIOp`kO3(a#?<0tv5Oc{uVPQe8{fuL2GQKhE`WX1|5 zIuq#@G9V7614n|80}xlS|G4=8g9z43Y%XxZOnATEk9;-ohS$uu81kri_@sOE5#uqM zsfNoIe6xN>$13UK-?%*E0i5C?`Z?qkcs=&%w~!p!lw*-|d3=PA@b(P%nR4$7&=CF& z(dnx}DUv)(7^Ns8;~PnCqY3nUB%>XT&3{x+M{tHv=%l=e$mb1+G-%ugsMLM(7Xd$w z!;1i35Xuva&q%W#Dg|i6Ay0#zqftVZyqPfn3$cs$5LWK!U zPo$PR;ff-;vG>K8#>pZNLI2Z;hfl1hi!!_GH!K_LF5U$ttJTMpkVuJ);C&g{njd0y zIZx4Vun>S%(&M}2*wo3f(LCdunJ$vcMm?3U>KNHkpUy0x`iuO?Tz4`hWp#+C3xAk* zg+nZrY?S#DuSGgH8j>xRs!6;y^+}aw1EE!Px}qW(4vq;aFI7AXX~1{&)U^?=5s%k( zFQzZE1yd*~4z8u@Pzr_38%h5>K^6>_R`_>r%(Rpaa-V5a)x7Syd#Tz&@j0r0Ow507 zrvO{C)L4jS@t@BTOmw}`FwkLRFo%m_6X3A0S#ArFC0+?xh#;(G!6p{XsRA4T4jPjo zGTG}}oL`Q8gs7o9dOh7PYFR5OpRA#&+-+5Y0LHtm4Z>5lxdkjnja8@fB z9r=F==)apk+i&E<9OV_SU$?ksx{C5K8&Ibl_Tw2vDF;bF#-yyyFRjKlS$6zmf#n1? zhgqtk;u+VEbSqGbD)m)MP$D9jB~vRzPvX;vYP+J8r7geZ;YAk}f>P$*thp-Ml!^W% zICYJfq$WlMjiU@Q8#~)ja)F*TdyUOGU#!;XfhKMxct!gC0 z^fM$;Azcer7gfi$C>Ku6)#xZ~1XF+>u9H|Vv92zK$N|b=U=kd0&V@+i5JBKI(;javU(;#2lR;QkGFtf zvoR2jk1s4qVBzQ;z2JJ<7G|QnAJO_@HnV6QlT*kP5~AXfx2w6xb**GSE|`CBM^}&e zfcRb^{NcgFTqN)`jG)@EXP2UvnV_>;y0ChJ%txi=pKME$a zelF=2sla07=~tz^rz@NrS2&+rw6$5kw4JF-7N0xAc<$3e;!_;KhAU_bFKo}eAg*xr zj_1u!d!PSX6X0v;AO2N})Lm4W$ulj`bTL$Xw#|^;COhox3=0~cjj*k|!6ZWw!Ur`Z z026HQ3Q%R=OLfFE`sEiVQ~AN3JXJclFh-;%M#nxzQxI;FkjCM=PxBkmg!Z;_X&&VP z=4=_7sfu^qFHuZB!+R2x;9N}qHNJZrm>YxULfKa-w=93>*_T6Tn%km4FNA$ywD%>T zi7+9Lc95Ss6#whPbYK3@j%3T{VKy;$zmUCcv|M3-+6w{qQOwuGMR{s%2scPfxo zN*uZ=+(*jJ{Q)ANE*I(nAaoM9u!B<^8@F26XECC5ZK20ZZKcP;=xmJ6*s;Rmbe?4c zI%^|0#7jZcOQFVTsUnt#$_e#kV3P7IfUG|E7I!a(PTIQk zXe)TKiMBHHRfuRr|@@e_@7t?poMX%%%v<;>d=comlsg%hnStBJMNwAkgDe?5ZeeRN!* zmTW&JIsJ+lq#?oH*|>&9__4z6>X?0aC_5T%2mQS!q;7opPv+LX`aQ*tVuxf;v9;KW z=xA(J8AygJ7wm$cI5>OOYD&0S&4Dr?8c7-xUC>~phctpiBxHHsRad0#?qC)_^7g*# zdQxKq%b7?g(f1#8CP_`kh2Z3{ITXx0Bf0$3TUYw2501I+(~sucTJJ>V6`6B>!DzHc zHu>e=>Fhog&?_j>QoYnn0R zwJmXya$&tI2DnlWZ&o9wIo9oL|e`*>Y~^%hc>u3wE8 zaiKWlO~xdUjHpanGGpfJI)I>6d9U}d&(SuGq|d69GPm@{78m!L|- z#m7P`bLpH(9mn(mhQnqo5{MJ?tUdPg#XVbr!ZhXoziy)G{V|1EhRK72OL(yXyxR2Q zmYe*RPd7=62ga9mxF1K@Sc6$p^(tXSavc>%VH(;6`X3zx2@SF#*)D++c>;vaYd?yO z-%ZM7I6c^6|wbKppO0Jb1GBRF#u7zLz5?f3&Z`=$jMw&sX*c1GXDwo7Pq%CaKO{yUh9 zrscf3bFj8)+p6o6e}gmNBZD2Z<`juSV;`m|1~ zvco9MZ4lsUK+Ww34Y0t$l&>JkrB<|djU0@Wm>;VR&UzTF)i{l#q;$bx8b-Vp>P8J+c6 zXdUxlB+HU0(yLmQK}n>>AzF8WwWfIS1xw}32k@OFS15DP zvR&FKs>my+ZqjGNJLVtbbZYLIbzL7_9nnh)Tc9WKof)%Uw%w6_J={n4my;-%8;{wG(vzXm?Zkw` zed7fQ87CCv%fp%kR^H`zhCbBE4lFuCXy9SJ`&$jI=EOI?-*GWsoj2g9pM$04(sILQ zc)e^p#L7>kuHa0aNYb=IRmoI7?|Q?UbTjY%r)TE-Dh3|XX9iM+N8`LlKeLT&4RT^l zL!TG;oN~>1Yw%OUyDuHVj9Kofh4q4bAgjB!fmeu zPaUl(pdSYJ_rd9|6HO)aeaV-U~}Y z%rtD6(tH7RS)=Xns+y_)+W9T{oAK6|6@74EsQ)bVZqBw2-^i&H-UDrnF$(kX cqu?t&n|GI%yc!lX1AWAO9{*X*`dvr=8&Kk0DF6Tf literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner3.png b/docs/images/LogMiner/LogMiner3.png new file mode 100644 index 0000000000000000000000000000000000000000..aa7457c602449b9cdaf6b0485f8d6ceb476b2857 GIT binary patch literal 163416 zcmYhhXIN8f&^D@D1rZ?%dxJ`eg`&~~5rjY@f&x;avIUV6r7OJ{YJwC&q=VD|p(v=7 zNazHDbV4su5=!U<2qh382>}lKJ?DGR_jg^_de$>*X6~7}=Z<)4sLgZk%DEFKPVngJ zXgxo1;;*_BCr%cf6Vxx>_3l`kbUr?v=UdJ==8G$K3oVko>p9$w$Aw zT>t0Ci5xM(*+=J2b6x*iL;UkuonDPINf}xX2)7e7&S*XAPB{E}FtSvG+*<>I`@s8D zPu_`N$d|)+)id)8M)fhK+eJ&N1H#FnH0fVkUs2Vaw8Ms@y`vH_-0nZQ8$Y9Y;rWiUdqhFmNhyTK zJJ{;xa-=eXu#|M z`RITQwAia1Rga0dwW?HA(=)IHIb!C8tIc0C!OEwzln_tH{|H+66>>IX+&WVil3YfT za9bK~o!`}*pJwgMheb*T_&wMo@n>y@=E)(J^|aTceaed$=1=FP?YK?nD6o8tFTvbk zAmQ=g0hi6smq-DnOsmKs8IH>NE2seMSOCui$LX%k*gK)2i9!l9F_-sNdy#1}b)7Hu| zEUQ@=y+kGVBuOnH1cR>sQfTZS7~));@~ROOoTbn_^AQTi1`iBpMXd}N1T1hBsPm%_ z-Ha^n_b2sk4wM|o?#`+RlIJDMz><(l$lXWd<@U)H-~7LstuiEB7YMNhQm8y`K-Qmp zk-lp+M!m-PCNfG+3FajKtMzw16^e@avy3cIdJ30;bshnYl!6Se zW&`{!Zp*;@jEee7mPm`)_$!GUD#VMxEYfe%Orl8*>uaCg9rho{P7AuuM*?Who@wl? zFwo&l8~Dl}z17(;GT=PA81kQ!#;jgP2Ju@>dn}dfTz49Xcq6Y88;~8$vAx4$4MKG6__I=$}GolMl0<&@DeP1ja9L` z;x3GQgP{_7NYW_saUb3(lTItHUf~~UCkn1n^^kN&b#~~`$gMJ3mp0`cpdd?~5E>~f zoX##V-v>_;A|>!PK?64nTxU1`MOPY5{Q!|>T~|6|+y?SNYN>r+1`J5_;sXsMV174c z=F1*Y-n}f!zB65QNO?SI+o(=xLZJG84fAaKZ65GtsmcD$Gk~Box8crf2IUqeN}?7u zF)v~A%57;@h-a#^Ok_%>S7>6ILt8G8*%b@kPU{m!9IV11evD=yerR}>`;ZZP#2C1n zMT^RV9qqs%hMUNFZR9*92oNwmAZ1DWoTpsBY@pq+>)244cSBiShU8Qb9z|yNkkzI% zSj*_cr%)TgfJSl`vMQQWB=!93b7BlJ0P;*(=H{>O+ju+25wO< z#`?4j01pO$BI@hzu^`qDH0y`8FEbjtGbb+ROBh%J@73gC5NnRwvA*5bG0PokwZpir zkGu230%T1ft3Qv-%p|L@CdmhrnVFq#B-&=gVKZb&rBB6gq%7+_on8PrpzwNEHwx3PgrC^dZ(g>7I~1&-Q}|rOO)dkp?%1p~4I{Wz4$( zG`UOx9?^iY>%M*dqT?3>6&l_6Est2E5LSyMxO9jcCZs+)_XCPpk;OCC1gffAOAkS0 zX0JBXpELwNTpWclKElQ-4mJTto4m6dV5<~&-`}T1-JD=+^qSa~Zw%(aF@8;|XTzKX zQ1f|(eLfZ|-nj8e_}r(q>cd^vqg@v^QNW_S@kNa1?*>+3gP%TBuzvOwv^IMC7+HbB z^;=08HNXB4Q7@;y`t|piJTj}8tXfOOcEzj>7Dz&v1RQce@^EbN_91pJOu25Rbp7qN zG2V)I?EeQb2Kmyjn+Enpkt8QG{!c!V$EG{##hif!v$`7CUARh8qkx~Y01bekXyQU? zqI=ctF_mzv<#zh0@WE=s;p#x7B|SG!8^RjUhU}L?{OjT*wxeAG<^}?ff3PTm*#!fy z=h>{257)>shjWm_Im)qJuCa&zy-WEIzGD+deU|%Ej&H}KO=gW$!6Bc5hYSS|`TBf* z2hhC(_q;*sF!rpVcbtf8;QqKb{AJ7W@j}3ZZOGBKVehe%moqbu&pASPZTOS-$LyKV z+z+t7WPIcPp6|Ol$v)L9-(gRD(GcwFKRzCVr$xI`=eJJ<*3F4*+)wT^J{TpFG{Dfd z&@#2P4}Agzh0Fhigm^ovZ5CoV_8cca_krKLv0IC2uvks&1h-S-FAPgOj&4In=?k z60_)EGoyW4k}uua1s*ZlF<%Rj3(fQK3);azW`}mney^OXZGCdRu3-Y#Ag$bxL&`3 z*pMy#pl7m1KbVy}oRu~Gee_W-;7vQXs|g{d(87wIAQE{1I~azENGb|=mnT3Y?_Ew9 zWZ7LzE-H6RyvzR=;Y1BQv4WVBRCnxUGPriQ*rlRb_mB7`c>VzSVBMR%jJ**Rj_ltN z45SI-CbjOH97N}?gjjGR4px6{fbZ;2?cs4Rztn}n z_RF`3^#^0^U^x52P{;@6lQv`SAGcp{y^knj`0fbGMjb$3vcR6A|fQHM6zLtE~$ z78+a{V#wD0?CLj!S|_chaGqN83{NG0c660%5lr*>E$i;YnVz{`=q0fq*S9Gs*tsr+ zPp$TD*q_AtYvT5`x*Jw4Hb~2i2s9&NQd1O+M9y%-s#in4PTntT782SHNxji*27n(< zZ5&LwL;7v+rmv9#j*GXzDYNrV@@}6mRa;#w`#wf1+rM$Ld<=hdumLfo*Wte~be~AS zg8FtE#UzSGezP(BYvFx^g$%A7!peixn#ECd4i;dC3o+z}PJ64EBNirBtg^l`S$GVw zR`K&uPoX7F6CUH$cI?WV+__!BdL^THnK}V4oTZOgI)m^^5c!?~MQ$Z(A8pAUd3^h5 z)Rcn>JcfB}*+7iU0c9=M^y^ygfVD=Yki8)0>q+OWq&2%`lnKG6+|lg1QSgm74>l6m zTd~&X6NC`M-r5RJx;N_?t#V*1v{(jAMDRN%#()w+>+hAMVlu&)#i5#qbCM_HJ4$i8 zj1bH5vf+EM8K|hOGkZZtw+`NYMD06t8imN)hyOd9*OzcG8?@f$Q=yp|z$GTM`Q&>2 zla1;R^OLfmz`<+aRx8&F9Ub}X!u98zM1#uEWHxSMK~`JNr;!`(ZWvhlgW6;*`Q?b< zEp|E(m*_Cz=teCBik@Zu5P@P2%Qkka8V+3a5C6k>AGPvU?ko0qu|Q{|6ya5nX`&p7 z7({cM_-=52`S^79U3|RH7_H_wEvn2i>X2S=V=LQIzdw1R z6=ta&@h4t>7Z#7xA*OBlnWI{CgPrV$g6&I_Uz+_b?kw;xmmTInCgDGHM$-Q)@1J_% zuV=))_|zgoJwt7D(#^hP`hPhOnzNU;b#%^ zL5zFC1s&>u>IyAuug>1S&^>WXMRjHvYolq(X9^p@PAZ#_Su?J%@NAEHIlO-0M}44jeI@Mh5>UI07>9P?r= zI-L|OMR|uO1=%Yn6;#>`a@Q@rTtB@>_Q~q1UmlVpY1xN)hFVAU$^%9hY5-`?0_zJDXe z%`?^xP0a`VZ3~MBUxVzKpY`^Issgeo7HuU+uvK!j-`PwK? zrilE*Dhm|g{~ID?IC?LhG7%F?c*Xr?bUO9bRT<+1Ft=6HG&?n4>*;gbftBuWkH%!r z`}P*hHe2uY@jgBJl%$3+mT^kB9+BT#W+Bd9U{OJBym47}Ce)P{qABDG^6+W8Rn}+t z>kQfDweBe)@j;u=M7JmT*x|pNPww6Z)xfiM8eo5#$LqIOUTrz2^j>#v|56WkBusna z=40M%CTS_CU-hS6uae>p2TrAkhzWW@$y#%2iX!t*TKcgUOKiaSY%x8Y2!$>gYlygA z6NFYAmgI%{0M5W={G&w=A1l`m#OtA~?|BW)R3Nz=cklHMhrr!;pnRbBBkMdLkUer| zbrhJby!-PPC8H5+;?MQsU<>m)>X&M}0w5`(@m8NEO8GCiV>AY-8g}E2P1)Edv?T!H zwBz;=hRcP+iPy4Pio?sTR6!wp{#;^ymDRc5nl+2zd)A0ZfubW?_u+5d67{n?c@>nF z*u$K?MNO*jv_a*|_ea|Z?JJwL=*c6|mbxp@v5!LG3WmV*8gf= z#)VTUa-RhFBEE^J+MXB46}XH+`F||z3BGypOX`z{!hyf;=(@=0$l~7c6~X&Abx+K^ zsvEN2GkSiNxcf5^Pjw4zD!GwH5B40rtE$USgny9Ej?ws~yY;%D%EEx2z`+%n*~K7U zPfH4=q==D&Wov$&%I`&b^fPEeqHK7P_=SY>TbM)Q05*HiT3uJVCV)C92LMNyxnz7JI;^c5wCN&roHEL!mTHvKgi@*CUxILExD4)5+)I zLC-Pi^`YwLE(;#H4aIt`ptUqnlk%X&gY0v~o>6@C<2GI#mpK!>Roo(ZB{YxK!{tDZu$Pr53dz(K~7M@ zKO|@^q5XAqMbi=kSEiu~?*Le{S+NMXTK1vOSkUTuSJf4J_G_eLo!E*fZ);Rb6;Mbu zME1>wP$}G{H8xyeYcy1_7qBaKfXw~|ng0xcZN^QojLWKS&G#?K1iAksNK`w>?&eik z<*rq+tVg7gjYgn#>TxY*cUnVy6d?q}z{H+?lB+ z*ObA(?1Qksh_(hClnPlDxD&z2JK?+!?qO#F0e&c*y%zA!lgKRFC(Uve$X!*JMZ`Ji zh&Jsvoy3Wqw_F`}$F6d=2|7Oosy^2U&wNfOY$fGJn^^UDQ%fBmV@BKfv~^>|o+*s4 zT0ovyg;=~tKlKbV#!c1_^WuiY^`ix^5Mt;1*UPkqW%Gd?Pv$H9@4H6@J0sUNjNT&u zmb_H~ay0pKpER75@hd3vpkME3N5kqS6MHH=<%oLKkUXAq*HU&Npq*DKQ3~GFF7#({ z;zy=YL?*HbdEaDGGyKw4>-huuky)+onfFm2MDQOS?TyWsBX97KMpMJPz_te$szmPy zJNBE*X+H+-uk(1Q$3-|tNTU2MDDpDcjU2FQwbP#@<*&iZBST2bc6Za)1$D30#~3j* zepmyn8krt+`V%GMvgiWT-K9vp+zKrUfX{kcGDaO)`AUEs){kSxZ7WVsHiJPwas6Ld%(@}5QAFVtn4uRWrs6~ z6ENkDJ3}y}n4guYORxm_lGdIObM!2+)OA%+4FT0iwF7}S@43WZMP^R^Ym0m3@Pr}u z&a<>QZ@E*Jt1qB+g7}E=;dC%hJvhZlToPHkoUTBU@rp@2j}?0^Zs_z_?wSO7VF&XF z!em*ur#39N%awllG`Reb`-#HZrZL7KGqybb8H8%L#7QWyA*TL;;XIHh#Ahm`ZqpXM zL3B~+&97Z5Z=BnNAtDGDS&{5aQ$^VY(pdU=!WDKoS`=TIC%L?(sfN7K{1WOH;pia* zWLxmn#JK;R7t{6|C#EBcj0%T4zXsJ&pCH|noqpkU@M7=SjeFZ&S18n3Jvt*JVD2Qe zGF*c;I36}H#6LGZbGA>7dmTIvU`>QDp}`w(IMH3GxL&f*jk(1^;|%Xb%ce_~W;6}& zEvC;-yA%H$_?iH3v6dNm##|K3$Pn@Jk+SwSY1ny$-U>qA7ScR?A&|EdLLM7+2ONFTtaU7@VspoR1a_)><-OU zAPOM4GfYY>-|BUfav}}49*%rbU;89m-}nzY>h0(kmGNdtsgA`24@G8NG%{&p+bH3A ztz)F}%^>3vQ7vGfaO@9B9OJut1k7x$t*hqQuU(t{H*w)nz4Py4Ic#v5&GQ>W=qy4C zk}G!pd*$ZJ3|9P758iL?5>IMIr&E%&ftsl-?N&*@1fw2sc`Ptz;IIzDfLUDU?x4TfC}5>UTg@Hs|QNdil|57YB>6q+4m%6>YuC^Q}cR@!VN@B=RnYm zvC2%|FnC!K49tyimBA<`x_o=#{q4plN&MTc`B2_S30B5 zaL|u3W;z~GCys~Sx!oeO!FmwIMA*)ll^%KN)ij=GwlC8Gs zMyaL~5;Fv18Tq}8++xw596zivelu|+po<8yh1Qhm@4kWn5zEtEh(k4?>5?XD%4Y=p z-5!c7+;<>o2p*|f1tB{lBvit$*0x<0>L08iA#~b4P8Y;xT`p7^)<~ z6Vz#v$}h>#+4@V)g;}hp(&pzySpPP37>#UP6|P?n?TevcHR~&F4DWX_m2}#q3rCL_ z(_cpYRLgGEnqt)6t|lri(6#yhWXW{am+CGK|FhUmgdzxlr4xeUDfwi&4YUHFk}Vv*UROFB%Gr9i_62E(!9#l7DQ+3QyDBByIrXzvjvCp z0uU;M2K7_eG6=A`>N~UB=q293dh4@C0CBsJEU6OqJALJj$$7Auuet&gEf}>WBiO>% z8-ZD|MUjuX!j`owyZymJCgXC+hc9}ge*Ey9m4qsm0m=}`c?QM^lwwM42G8A?<+{{L z;T2fzs@lRcnAs^KCc^iuZkF1xoyHH=I@vErt(|)O%eK znvF;f+!`PW^q9Uza%VkiGxedrS<_ZJO^CJ|3_ttnB5C`(B zSl>d42AuU}$Q-|dI(7p4sz_4Eawf9h*uwYQ8vfPzW{U}MI1lAKL(*EdL@AXlu+H>_ z4a@qZ)I1yy@G(NOq(YMB8fqUNd`Isr$HngMxDOf|_JUcF(Rp|4ZAQ_N#Y^w5ZyaZu zO0^IsDC<;Q9{WU*hu4bdvlxtqmf)Hb%0og_#%wl%H_EVvAjuqF68 z_s~{1p1R6EzIgv}Vo$-ar*W-Ufp=$6A%b+Ofuzo~Q$beyG_4J(_{u$2$uu(d3n^O* z=-RPmg-6}ORc6BjBzsN`NEE2PtnrPIu69!S0XY7up?TeMhq2qNo%-Pq;M!M{-YJA? zlZD6(b+5dk#(#Fi1AI{e@;$=Oj~;=S7f|@^u;!)0Q+WRsz@^y&@={d5>QDhH14>;z?JyR{Yl{YK$G*eo zJ(0-3+-9&}1;(m{D5HSvY0iy9^{y&gMB*6d;j*3eftPIBy zM;`~fX#@cU=I|K(veJF>%8sryWN^#Cv|6I_s&HCcv&!g7HguI4QyN~Si&^Vm&_?vO z?k@0q6)dQWZI?4tBv^_&pU>boFW-Tf4@uGsl0+uy4bb25tK1S5u$viM#3gD&>ia&6 z@6@49K_hAhB%3TsDfI6^K*n#QA0K`kR7Bsa1dA_mHnxob^KAmXmI?t4t{+fBpJz7g zj;;ju7cc`je&MrRVY~yYDZrn-$QOnX%W4Fzy=qY(ElvfVnTv(ufFUetGz4YptU-R+ zqvWIXGB#kGRMlAQi~r1rs10URyg=KM&8wyuPSX|}exuf){y!!GvShz1c(?zW)jo1} z$N$;xnHvq_gMr&A?wzi)hq?j(M+>loZ7ubaCp0dXwyIzJ_+!(3Y3UAx8 zTCPd&P^r5Bi{d@Y=TV!(9&6oGl*3W6b3(fQC+F>G7-u}q(Q3C(f3I#y-J4B#xC-U# zptS?=t5RIz?JWt?ficr53lbW2vRroD)vnX$9tV0&OZ4@=jfI@Ls_9__q%>&@yfSG_ zbRmOwwE6h9@IkCzZ@i~=JIeRG=g`JRW8*Wg_irjcOF0|h!4VvB=Mh6ugzIrS7Cv1C0xzhW1f8-t5 zX=Uk!nd@`0-TeWbxF#|XKTK{EgfJH`Y_v8Ngv%^?kd-At?2jw*$bNyHp6xNbr1wdpM`HRbNST8>j*?`5VJ(q7QZDEc+~$^I-qJw(umm57wgy{);YrO z*$A4V#Z2aFC^CPtz6G~6<$pGipe?+4*EalZ(Epz#|F`^8supfKK9f(9t{d{@ta1)SC7TliL3Hp!9kO)vqRuoO^VG#shzAW^1Nj27YJPmXZ!oDn+Gx7GLos937n zv=6fBg8<^b#A#LveOv=;ipT36GFQ;_PQIIo`xBoofrk)?11?|{X-06dofCySE!!O9b#6tQj&U${X(d=@9n zzkb}mi8t)hH*w0~!=i7MdOT(mWkDPC;e#o&^9Pv-%heQ8n-l736z*u57Y6HX&$4<6 zTeaTm)$TQQFhMN-gSuuj3>T=Oi8TY8EOmF8Lugv^o z4>B8gnYSiQj#nI_yW)`^7v!%-*d9?=YgpF8e$`QMmg2;>56d(QANgURcxLQDN>j;3 ze76)wuheA0x_WJNQ3&p_OZwmhH zXWaeky|PJQ_zJpX!p(AD@si6!Xk84j|Hd_HMVXd95mG|@r#HyOcvrWmd>}2_^I<{r zH;e=)ds)2MC{4E$;;5URn1iMe`hcu8fN(yWqJc{b?tw%MSaYI))REa4YY@TGU#!_k z{3x}BIRO*Kskv)6_nOqr{(}om-CztCF86^XgsmfBbCg1I)x%hc4JyPVy`sdaDemz6p0XdJoaU$CI-07K9EkZ#hw zsD$5LsT(60(>qx!JkRGk1IM*+tO&}|)F z)3#w*dry9l+S7%??99i9Cf@OA8Thv`Nn}(Sr)YGn`S&aiAAoGXbDstl*=oE z3g(xx>eB>UHkOB)UfwC&K2G`L?bTLSS@nUy@RADtgP`ym-d1&;IUr`D@-R8`8(eNGr7^ z)UvLTZzs2RbP!LvVLF|sL#+P_RW{|mfAVTJ_npL|N56R_ZY4=qza7`` z1P|9-*S#kabMCAncM&Q=)1W^Sqg`;fa3b%p& z&Mhnsa@zx-^z{8ZUQYW+r1GYvc+?TICVcDw@|J2h%<(jr%j$;Pn8*C5K7Cs zAp#z_M#~W5TiHuYa~QMRX4NAjs9{A4#0-}SSUB18ut1Kn?B1;BBb_Di7o48b-Ij5{ z5Vv4C47~Pszm{Vo37q}bC#Da_pB6Wwn0Hq62nnqZAIozLVY=R467m1ZG(gFEVT4vK z2lFU#VA*8q_K%N|2%G+9JNad+Lhx#AtFrn)`X_OL$%)qp4)OK7GnvkGv$3eac7<8{ zl#kD-b-NXqNrUeO3QTF!WAR-*oc}`L@z!6he_C0BuY+%Jy~1n2ni_=()lgWt6bpA> zm?6|(zxC@=;;m=%rg|^a37)-4ljjlhu+;=Z*NBO9xeS&-WO#PZWU^Tpy-`)!@3ydDK$KicG z{0(GHGz$TdOCT~)yxZz!-Ae37K%gLKV%+9li5sakPvzfdB%`+b=JqMv!W#~w?WZj zcic$J@xq((fuG+Tik0>7D9T#*7(27`Fs;F~MQhI(@I1e#f(751+UU%h9-sK(kh0Qd zvhPbz{0%p+UZl>QX+|WB*zc0jqNj$EqKvL*d7Li6j|lE<;2DuIJ$%x;vxIOyEBJO| zs&+GsZEx|(KD|=5vOB#5T zlUY~Y1p?RG7vK5@vm2`~n>be3#2vM|Z3kDxB2&cr*|~C4=eHYb`ZS&CN;cggqMzuG zvffO6Yern>UA_tLpBCq|q^4>{*YB8n+%%pSrI#hjwDqRa!~9sAPt#l(*~($ebLxR< zWjCTO8c(ov-9!Ic2+pqRzKd&~E8 z46q+ZhN@m)eCnVcQS(;(FBO8;b)EC?u+j+`F3N%yyxzY>i%sq(TuTHKB_MwKR6?tD zsmmKZ4iL|6m0;oe>9o4wx|WAHFzWB+9#MqXQ2L4Ku{VMmmm5$T!0FW#F4H6NGn8o= z9~YvL`he15bJAY^ zfwg?IYy2-z_ozVOd}{WzxF(;rvLq6E#3&;?>CwjrU*J4u6Xg>~;k$Mnak068WE zDFwU=Vs~eAA92TI@txs@)2!} zU88#F?lPLHvEgfr&j2W54D514=m~@S{ig&1b?Z7DO40$JC}Z!sR~##6gO)H(m3F`M z{(4e%ztQzP;kK><<|i(&!=XyR5ZeT!f8Q1aO?ncppRbS+)7j z9>v6}Xa-#NR_)M}fHn82DEe!7nhTjOp7(rFH{_jYFOygH#-8`j$feuIQnYf*mhi)J zZ0nEvDSOp04cF{K$)Re~6sE1q&jF*rTprP;QRKakCStDzKD^)btb5cwWT`ArhZ?mh zn9DNM7tUgMhMJsnvPspE_9(N5DIeJG^bifvC9L zJk87HAJXXh<~Ab&#hmLjHE6s=1)S`w~g>auB`0`*l#In3@6uR+k`%rbd0 zl(JJkQ3|n2aI*lhmop6gx`;iuounRk?BtEM1kt$?Dqd4^v88R|o`ijjiij*h!a}_} z$1}n{@t%s;B{N$;>8k_}K=vxQt9Sl;S3<4S*2MyrgYc!CPsA^=O{-0b;3fH@oURYz8Z*&pJ1R@4;YYMZ)r z^9(0(Y@+RGAQH%unRmJ&7g#>$V9xvroM}Tr26NDhuDHe_Lt7AzaE!W|5*Z@cfcg7z z;r(qDr^5{AexsM?p!kHB?+Ubq(iX>UIAmtozhOkli<;!RRd5#Ig3|Rd6lA+5TwRq3 zeQxi0t>Gm*+Hv(+<(V>0;QlHARcK*WOU?|-Mbv^(lC9DR@jA+oh#cki z;|IH!0CTn}clYz_XF3W4uYz&8e7d!*9$c=F{KreP{&RW_s%dN~4z=ka*Ga7WV98H- zgjyhue)(_MY;}!3M65304RwV~_4d4tSbVj+3ra)+5@W1Vz+BS)+v4~UV?+mZ>gH!j z&oH0F)Py;Ff>t~EQpRb57X&4@!YwQcE=|Vv)QU^yR8=O{Ne*Z)7|}U+ zG%f`2YysB{3@1kqV=n$OT*ULfyV` z=ZY6X;mb`SFRmorUY?|odb^tuJoba$FF412lcfK+Xvj01?v?(L1f=5{*%YeQtDRT%I}zj^v>0mIFecV$&1oKaE)DL@ zsN06RmC3Q}6y(#kah^8WdJ0qb6UL({c5d6{IyRg&PVU4;1^FvdYi`RgVwEhPV#VIv zozdbpR$4w@rGGP-gqL7@S_n&RK^W$cK~W|T{nzB_d;7*HIDm6jBUkw?_MX4=6Vb;_ zAw(gY5G@@cQ(1sC_4de^UA(d(DRIMcd-}C-lPzFUe17?#hE+|D1HPjqSRl>JhbsC+ zpY-S-G({v@{+izb_~35W&SZTtwCz`E|6(0VYr>_@Bs&`!iE(;ZfPtgbsKf0xAy%Y% zyAn5aALnJ94{Q08=9W03Q+(QsSiEGtNBr45&W#m6A*$m)b_efcK~JAsX?SFS78iC4 zc&$PEowWuZP}?kns91J+>t9o4PhDB&6dGIN>r3!PXhrt6j-*lIi!(QRF18J`Tl?Ur zTgxV)b!|zL>?Hj=m-&~>i@^@fjE2Bwhui!Wk$=V1p5Ci+6T zZ|w0ws0D|TlbfF%3wele9ZNrs-OLGtpq_2&Ofy3t-yX!P21A_ce_brr(` z+%>Ae*?^Od)8ZBh3vgI|nYw&U&e&t1g97%(JT>KG0WeiZTHnhnd+mD=!305iV-?o! zXworMa^|*vq*HXr1Bq=a)OM*2k*gIDcA zVCILZ-WRBernwjS<1c@enOrimqrXdgfU`bAcS4u6Z1t#OJ! zdj+O!hy5KeFa1VWG$Ue4>n<=VSA3NB1HR{m;-(L^O`m9xOA?Q1yDPMUQra4Nv!v${ z#2dX!$dipR_&KA&U$1M6Y}9NJD;)5hw4t^Nst`?Z;eVAX9(>0nG*M3s=&aA3nqgi# zFqbA0j#?j<&htFeR%C_Ycn+S+%6FjAt+ReOU?$h1f+sHX-Z1JFxp5FaQWH z$GNJdVo{d&w+OGzuvFtKm^OaVVs4}ajL+_n)}PiVt>d=S`jD^=*AzPua7WM=lp|2; zwE1M9?jA6=Y&0gbGA-Nm&oY(rZ)K{scpFpxa$Ax?TZIxK*Y_LeTW%51Wc$GBZ5VAa z3nj!&p>Lv82D4* zM00|r2jRnz-K1?w1r#Ng83 zjqt=>x@IITrVVUCmVO}LvlTb8dSWNGxqeRusE(IA#$Gum!Cc>Py0o$xFD zZuweV4Uam-@?CdW%xA!wkU%4LDFLsT~)-!;hTWRCD9)P0KaC9TLTKQb}ur?`ot^!@sLzm!RLjP`U zgpEQl`r;{k9y|Af-h@oP!+^#XIt`KAzZ?5v5UF({Nx#N&$~<2SYGG~T=`Hr*&|E9Y zaYs$%Ts-QvKFfxvv7(>I$*8QMQidk?@K5 zIG-wFQ*qqL@3%w}J|Pp+Gt<1%u3kxOyJPVhy|SF+v@!~oyi|H^YVD?82{WWYU2#CJ z^o}L<>5@O(r`RrdFV3#2=cUeU3m23A+xKiD!w!^MoM2lUU1Kw3_!a;8t{1BfexDoW zu?leLmT6+!2hRyAh(D%RQ=8IG5O17R;t0qPVLnCgB&pASeUt_hGZXkrX*-DKXfF)$ zrTab2@Q{ks-t&%3+>bAoiGxEXK&<8}(VIPw)*}*(mUT@ot}E286dEnwHjqw20ocr>IjlwUSprh<+1Yu(?m1oOCbyrmmQ#RnF93=`sPN z5~I4de{sHTb@p%mMp;Eh@wqFuC5ecB@AjVU=XjuW#{cRf(E6=QZhMC9c}A*SR3T># z^XfyT`o*|;Td9tAr?L>~Fk%aO@hdB$SKV^U*jy3=TPo`tOKV^D`g*>2zwwd}<$|_X zf^r{xD@p$D`r2u~&Abqi^|zG~==D=b`D8B}ZpbgPisxae^~-TxT8ikdnN*~$^>gYm znSPbbmLu#ubt%GC5pS2Yd@n6+d@dxic!SOAlUu*BH54z>Q`}~^8x&x@(HO_;SVxwO zYnO{odpRxTD1mF{WHyDxD|_-exSotk^9|u~f&){JTE1H7R+<5p4C|O#E?g3LS$0x% z8S+;FC!U(!to}e&go+*6J)y_3PZJCz7=-O~3SWDi4tp25YyRt!!c;Uy#A8p*vXT~( znlNG3zB676zhXhqE}TfRFybJa?)QKe{ZP8BoT(HWwok0*(tDb?@%W#wmKxv9#0-(xa1BbPMmB7_s>78DcExev7zW4QZ}m|#n;bMszffq^=uf=$b&hPxe3vh z<0N9j*+Ey2yU*yY=Ma=ZsP3afJr13eU5SUP<1_s(CcTN`BC?4HLarpEX(pR}2-M(jj_Q@)_S zcc8KQU)3A!fLClmem3L9pE!%6WPL>$QL*8F(q@qR=kenQJ4TM%g4{@aiBW-WliT(d|eXjY;X<4MR4Z^BT5|y0#66!ij{2R;AvYw zEN;&IiLp@%k;TYIbDBj(hbwsow@P18`oyz$MBBvyOt zR_}y~gPn(14D*jqqK4or)s{PKHCTG|RZx!#RS}a0V(4L$q6eBWmBVeYwwnD9BCT>) z=44LVM&e1SCyI@jT^Wx0%e2jnps*Xc2g+oNoC(cD*n%rF5Oo)d#jZn<$ z*57-+VY6-Q-%mM`-c@zUX0>puTU~`cZWi@?cxYF~oM<-Pi?-3KShJVX4SL57->d#`U_Ce6y#(7KfA8deVKVg<_Pt;PotOBb~I!0rTP>MdF&*Qs`1 zALQodkdJV}Ww222j^hHr)wwb)6&mI2Y$IVWQ?PO61m-NF_kX$fqsBvXU!BrnT+E(ENmQgvp6Y_sWq_+4gv#cT)Be=5(vvhqing3v%jjt$2g==1>Xn|%Cz#S%BZGy!Awvfjum z0OXD0RS?P_+gEZnP}Le0Lr9d~rJxcGzir^PyYgiul@N#1ML)x^` zK8_%fSGEY5u;E1hgPN;>H5;pPEPC+b`MLN&JCFtYcNl>`dRNx7K;_@4jIMkVR48x& za|bn@6rHq_`~LN}hOr$L0cH?z(lPT}UmNS4w~Buw0~@{8);#f#%R{4I%7@D z=i_q3aLc=Ev|5kBVn2Hb)*cO3sGTW?`sYEDRax|=zDmk6H_+TSj|Z~6s3i){Omu~* zs0Qpy5axsSSx@pS=WWPE;V7iV_#QFY33>o0{vlr2D3?JZzN=4^$MO)}ze8vS4xGKb z!H=AiAj`c~-Ic`H&Ur;qBG;2otc&)w643ifwX+`qf<14ismJOZ(WBemX!QZnV4wVA z3bpsr{*UcAp78k+hA-yr-4e@)C*LUq_%Er;;Vgm+z(%As)oX`v-PIn-juSLs)g`4P zcw%J_G^(v`Oz7UZOwjUo2$xZDD06Rp0d?8Yj4Se*qdo3jCw<+>Ixji1)Ao3hT?sW& zm{U1TyAqC2*!(kk3+g*k(7;Q$)>Yl&&1X@WC3Tn!lKHwFU^Q3n0nIIJ&?s+9lwp1_ z5S13@p#)`DRv&^-{vRy>M{t_n8paEzNEtqJr|K%SG z=hthUo)156e8z+0hh7MsRbJ@9?jC!<=~!fEa=0lqq0N~hw_5=<%QIW}39Tg}&qChX z)o=w2Cpm|+pZQnnr-i#o8SMN&RNZ$xn{E3z{!Xf>mb+AybQtZeMr}gqR5jB3c2`QQ zyJl;z7)hi?sVZ72szfNIt&!Rz#2%qVsFD~lYX*q~NsM1S&-e4ZUcdjYKa=aaUgvop z$8{X<<65D=>WBElTSOuL)n&!8O9-u}V2hecnZ47tL6e5EPFMocpQKqr#BM#%FU&#* zc8cA}Sn-{FrPy0h%#o*FH=ai@IIlo#_xkDJy}ub&`}tXjL$NqTHX62MXdc?#m7n4R z2%mmOOs{#=VG@L}BbnnTw+-3$8@Txxh25uKCKH!1mhu#9YJpRg!r%z4b*;6%TqH|w zugjWdvB_RaT0>mgjrF`*GQ25Rse~cd{c3f%YD>#xSrZ-_NW|~|&7OKf4i*Y_+!Ncsli`1CmiT_r`#OomHMiYqn?xtjnP1!cv>qo&`X&x8ZUpU%$orIFBfl112? z`jrLyvM)DfITWNP%z&#G2Bo0GybWjwC6l6D}gkAd;3o zUY7PXw@-U*O#B?a@U@t(a0U1g?ZfnEjxnp17W)eOBPEgjjE=(BgU1$!GSB6gJXT~K- zpd>DQ7%Fkb?M!WvAC8*%GNhA{FHsDO53}DvG!LxK+@lGbBoKQDd=zXWl%Hiip^&yZ zxR4}Y1OD}5vdpP6_!sFPEG!xb*bT|A-V9H=ZqoBwzTV6L{c2#hw}G}#$kTeO+O zv^5U&hH=n9?lCkvK^E)38GNs$Be5v{|;dPk@v7yAlVmHF==RRZdD&xaaGou zJB!lxD_7|*q*2YG`6Y#~qzCgoLfEeaGB>22XZOZpZ(4FMjaX1>GbI|^22v6u{VilR zu0IZ(?PFAElgp9aJymC835p-CK|-qrEipx3AfXNYCH@M^dnN&&%^d14PAn}J+r!H~ zCU{Rz{hUcxd)L|-AS&xYiPK_px2?$TgOa;8B8|5KeJx$yp!Z&pH6l z{dhGRi`7s_5h3$qQG*6f+p_7TKEl-*td}JS!6aewu28la%@sg4TB|M&%5q;m>{%t} zGd&=(ipFGx6ynsT?dY*f#AfYKH{A($`JYw>bsG_j=}w=7`l>1$f3G9Lx;EDfO0Vpa zN?7X(&5sV7m2V1Ef;r@>RP@=bVg;5$ z-C2t)gp|40WQ8Iy3SA-)JfN_4+VgOU^xCUJ+O5for>VqDq(disC{8;Zxpq@yj4!WonkBk~JF3y1s8DFxa)0R8`-;MQQ*OxAN{C2{ z%WYL%4}&bnKwh2pwJU_lON6uFd-|3W4}xzjFPf;3%k_g^P&>;9?gHrwDI}~)p9$lc zlgDPuwE~56w5^l&k5;zrS^M~+Ek^MZbP9O+z7;p>*in+>bM?orr!`48#uC5W2^i42 z!ME&KRtVMw`L8*IJ@U*80?BDW1&3z8aoy z4{eE9TB2ZHzNU!k|B^5N>zgk06tZ;-Uj4O?;7VL27VXDvp{5B2WJ9sgIh7ks?U3He z@M$sZh%{;)@SAU#dXb~z8CPx?|3(OS6bd|#DtI=3cB1yNNjYW^hCpt{^f3Qbq^D7Wl^-x0}iaar_ zVnm8CLos1g%VJdLfBlQRo_(eGh~(QxecXPMA%+Vl0f-*xM472CE&IWJOhC$^(NC3v zHf4+Fo{v9dJa~XUYz3<@ioBkl=Og20G-J3-TNi&rk6au`tzU@rOxD1^%MF-~v>=wu zj#*S-+)V2B$?~K7)}zJBmo|edJpBl4Si;hv!0V~J zn;4;(d`|$^wpP}oJSrFWM~dSEA-Txp&UM&B+IH$l%YTTKGN|5u_Zbgma_5O1%{en~k5PVz9?aARGj$g?uKZ0Js^`G&hOENU$V?s_5tjCSGYyB` zV-_UYCWiva+an)w>NluvoiU@{{r`J(|Gk*}Q{rgazFc0!+jYGs>)&DxJkw-*o0AWP zeKJHtQJw4*9AfT4g-RGH9tLcr-&2`tPnw{|!N+D~{tK~iWVj=pGHXp$Vv5$hiaRPi zv#2Evp#XdgJ)Jd?sByKmbjmLYKDK&4Dx5HRTxDcR8utzAWemBsd_PL>L!$1;HQ2E$ z80a)SMR0be^yzD@ngr7S?$*J@jt{S8`-Djj07niA>Zrol-l1d`upUsi+~ra!a?!9S zg7Sfh8gJ+a`bnor(I%c}Zs0MGh%QVXb7KnY1jy{I$FBLR=cJgSmMpP)?`P$2Pg~yt zvMWb1LDxKv(x&xVmtU7)pgIC2c5a;-{iQvm#I%ML7d(em65Jr2q~68B6kU`Ytz`48 zqyOLCtLf(TC@CaS@{kI~0?)VDT6K?%X5{cx7w_RgvWF~PI;}UU0e()yOv86cJG_4QahVIG?@tPE=I}r&jBmt(hlUJ;JAzRZQH^= zA+0Ge{o`mrt}bA?@4RdqlKfoAEeOdFY8?L;b^71o$7*&hAC&h;&XS8vr}AjEy;v;iu{B^dVRP2hPPCHaT!_N1iNE~# z^?7_qVSyV(LRn|-DtmaQcnDQHhDANK#3_ANlAe;sZmuutF_U~KyNaV`fiot2 zb_c3V(!Qqh&d*!~CBB#p=)1)2`+B98s;Gmk*rZ+8dOLi!(a*+@lJ6tc6QX?VRSsFD zY)J}}Li)#oTI41+R)I>Tztv@#OHvACyttG>!>I?IS-#DW!h=+~DFwJ8mDvv#F%RdRZ6tr_eX#(c!z*U366Nh#zKgixyH@ZWQd_{)FGE=u=1P2L zWewA_k01Y$mqAPE8R7;lX=0|Vy4r5CdTOAxVv2ppY^HQZt?7HP@D=NftN6I_3fSCruO zso@RhC+hxQ9=X6HG1s59{k5}@(2=9Xq~8HvBA_V>UUN?x^(0G>aXuRJg;xkj)r@9J z-%#Izh2ZjF9yNB*B6!1|wf!qxD)+1ets83Sv1Z?F*h4AA&4tj5W-Lc@!iNaWia4nZ zL*CZU+qYgF7Mp#_hp<>FA8Waw>^@L(!MtXD(F%*+sZcmyLxkwQw5UCP;kEXM*#XVb zcZCHCS!Wx;_iB);x)iI1a&ulDV*=@q`?ID)1Ag9vMaEM4PHqvqjXj`5qf%chOu0y) znZb}kqQTN&$@KX|A6F=>T|3kxV`plUT$vO+y`b~7==W|JSY*$0xI~4lwb)mkX;xKI z&kG}e)GiT6hEZO2cyDo!Oh-o{8PS2nF4)>WJtZw!o zwGcc{csVsD?1~>9)~)N!jGm9S4yCq{qwzSe_D9$-9T0m8t;;Vr;ad(`gbH=p%-xlE zZPgIJv;9uN9y9R;o?1?`^-Ym9*deHmdB*?#dJ;qHty=8yl{gZgf;cBzUi#h%b*b(Y zy|?tCJy&ORt8$3IxF?m@SQg&NkJDc!nP&}!YoH4@;|eqNc`x}8?7o#+-;u%e9n&1A zN{>yis%AG4M5!jd+(5V4xtO({&9k{?%|B;)>$ju)Swt}DTZU#NpH)$V;oqq@d`S;7;N+-)GH#G zHfT9|vH*b?LQA(5c>RJ{Ln=8deHFzZi!tjSb}nXgsafVpmxy*Jy(o+qsNM=^t;Q}b zc2rZ&o3Fnyr?{c|i{pIinNLmSb!dXrUG>qjv*v+=zkS&^!s?cTVYVk^gS#!P!V0lx z<$?EmDcUS@`2FELlY13QY{^r!cTj^cRmpPP!hj{5l}P^w4AKee)b_JnvzidiI@sE3 ztU8R)ZWy03CfI%N!PY7dPNKKjN}Y4i*(omqJ^eikq@#TRs0yN`!{S$*>cph&<1W+R zCJ&+h!)2UW<+{P7D#n3NHV#1W2(_K&|LAt(Tj6dke&TlXWpjU58qP@ejUf4Y3nmdQ z#p4s4up3PI!qP)X5U)lcobD}z@M~3AK};(nNmh;LRDo>+cB}qrxo!r_*2V|5uqA(( zf_`j8E9g%xXyQVDtOAGjqF<~|uO0YntLnjoM?b>%e-H1T5ncd@AWyEF^PH$v`~c74 zNPzh>Jsha{UaO;ShFmhxAp1txY5*3_e8bZ$b#f*YoH$+B#+?H)uXLQ~Fg2@dny&BQ zO{c98(#$QwdM^hs)~kI2xUZ)R?O9)Q`9bRu`|}aUcD=j3k?^st zjtwk~Duv<~=i zz#7a8^#6|gd)j;-e0A^C_FygUJ04Jadm&F8FycPn1hgfIncoDYlsU8VZZFD_3`P7$ zY}h+DsD@)$t9jj{dm7|#J~x^j;HSO2?SgnB(%Kl3e`G6SS0Ki4`P*~08D(tBbpK>| zK#J7(rHl%kmoHteIy_n=1{AD;kI7Z%cK}^j9w%A zxB4jJ&%WR^Il)1cWh3LArQ}4)3g>(#g1)`gaBzMAVnj_shIvwx2mra_1Q<;yfXysX z_c%M4*z?rqFdw$%A0j8*W&dS&dyaS?Va%aA``tO?6H^4`;YUe#6~0yAm)a8 zw`l9q%m#X)RCB!?;2ho_DJ{^j=2Ff=V|Qll;k{p@#t*nT7~v0fC5eRC zQzB-b>)9fNJr>SzwVMAY4|!7p)@z}9qpIo2lfQjtT+NU^nab2Zu$utJr#$RyV}qr? zI>Tm6!TsJUz@75B2bLEK3OE(ox^!<}(^vlqx@>s6N13;G&x4OOEvMGPOd7sv=aam)x!DG_O5yJMEwrX$h-2+o8=ePp7bm0O+=a zc96-NwjY;u_rr9Nin72)_nsKoOSB|*b4B3eBUrnGw1bv0AVJH-uB_R)Vo*^slSwKm z`#KEDsj5sO3WYtsI+b!Av5`U5596J_gFT^v#4m)zT_#lP=l(^6|K!foSJAES8`fW_ z=>K}E@R#UlZ#n9ZcWvi<)OOgf>M&4$@1Q~`ZEv#8`&r%YZe?R#>gQ6jTuz@j>}Oht z{48E{f(K1vCx{6ILUM#M!YN0_qJ^8tYsIPob;?5XC(7TkqeKo2!POnILyB9UwR4I7 z-cx#7_+lC?d|lPG{xt?=?PpYAiYPKkLx1HY0SwaEUw;s07QZf2Pgu zwUfiqu1DH$At#RlMo2d2>VNHScN&H>(S~evr&!P_1W#(L;KolJoPUi0y9>s_@n&c? zji(ys@+d6jb~ z=RIoFrx(VNQ)j;pXTsnF&Jb?Zp6$97y?8P#;d7(?Q{#Eff;|(fmgD3mg7UVr%U0js zO5dvJrvpE$bBFTrE&%L*6Zf2^WwK{f#Vd&w*&bZBDx=v;&J-BXo&xm*I%5rYrlcED zsp>Mpb%Pj~m(u6ikQW{+N}D?_{K-iLcFvUXrTe#G&vzKjBH_LT^YJ{WktArp9yuN| z*~ib)CakldndBUGs{DhT%xPElb%F3AUoV!t;td7B4T>zLpB9X2j`Jtp5g z1@ajnSHF@iVLl_-5KGyiAHaiYCkApcLiAaN)-4ooHSJrK8AwKt&0>z9*WNf}kD6+zu17x*(x5;=xsL6zjqnA zlwNu}zwx$0`HX=3Ai#?y`lM4%|N1TwyF=_bv@P9roR=O^q; zJ&WiuRImLO)h1)|a~6`QLY~sU^SD3L#icPi`ZX_TeA@||oHLzydSP3vc!r*ag|*oi|DykXsM>+LjA9@^)sX?qA;gkMvgH{ABZ*;fq2l zV$nQ`VDfS$aY-;ElcYJfl56+*ByJ5ND4y{l5Ds5Ff&-yrFpMr3%m}?aE2f zGx+Ynk@g=V2h1g`G-s7?we~keXX}s@?YAT<94)(1J1ZreA@_)L?F`Gey?XKeW>c{Q z-ZX?~UZ$;v&%}XNik|wbSd5_@0cG0Y- zeeaqR4^UEkPfVmaKkQZI?8jhG{gpqY3rrG6U?Lw1#-*N^Pe~3oM2T3TzFd1X6EYegt%ziTbB>J0nh7^|M8dW*tXJ# z;YEmId+lDGGfC|Z1%Uk6XRylDA^~0 zA>UdDkDhfXuIsDrO`XkrcRvd^cANPG4SxOC)(34!#~q#m7br5$?hT$`HSl{e%2JZn z%i;Em6;Wncv8FW~2)9U^+M;XGOl?<1PTzU0XR+3Non>UI3XwYn?z~Y~O6k3JoXTNBE;>@VJ`Z$DSvTY;~ zHU4WsPSRJrz!^O=AhFBp#@~@Yk5eu<70++OX%ZJj*l)Df%C;(9CcjZuWjhA0Ckx|@ zm0C99&nb$4{Gk^sp@DKYEU|WK0!wZ5BswvM@>6Nk{)?0Xat?}T@%-P$dUSJvFYF#m z(#M~91wS&gbAuu(prbEZ(-DWuth_s)9P;32xjUtS$1jVn)@O^k?e9t#|3 z<^-YEkfr52MbK?2$I#N-m0J{Rhb5?AMBnkXoLATt&KLN@DRtrXFrj)FQlwnoV)#c->ye*ZHQR-B_8^Wrv+_?_#z&%Tuw^9zx~0=dyMJLk?Y03{8fGlzn71%}nV? zeRKoQ3EQ(}C_ANWCtir8h`-M06`u+I6*}NL z2nr=K!W?cz%Ig6a)CGQv>{Tph^{(_oIxp4}kPI{y5jc#W2--TK3Wfh)EdbBE_pg#@ z;D48VI0#bp2wa_eiZc*p94HEmlV`(vWnt4zx>inSr9)YA>Aswc`Al=nxTuRDe&(t# zY=}8{8%m73lS-RDZK2k!-3u*mUxw3`anodUQoa`lVix**5=FQ?$PR{uWHc+Aq!3z4 zGgha`w-ez%RyD?x#q~waZ`{-}w4jc#qOSN*@(A{xDpM@S?5tIHgTM!SBUh);hZs42 zCb6@Y{>Bbz;l;QSbg0#P9Q(@^VY07PI$CA4fvp<8>ZLb08WpcgtiQyo+L`Tql*<2O z#kR9$yH+7Vn-KUUwr9 zLl0JE@_$!l58gL}QsD3FIT2c=VJXl2-Y_*y;6J=bG=6P!N{Al#nL^L0*yA?IkrPLf ze#u^MF1uKf@nl$sO;mr26Er_Oa_jW>iHp{--37U$p1kygyWeselURb0_8l@6m$#4X zF?1fyN%{E$aN@3%nN|PCXKq7B6v!{I72QKo^k2UppOBK(T#>W9e8nVPpM80cqgea) z+Ygo7+h@+DiRGs$puct(`7SEckmK zUd`vB?L37no^HygUl9ipdD`9Z=;H8S$4agI8~gH|bJAo@zsY@7-V$^$d7e3Q_sV9O zAnv`b~D9lW~!%%^@u58nS5*oQ_W zsel6ZTXdaPisPj3BLZH}=sM`Q^mj_E7=A&`7vA2l|8e8g#`XhV$MLm5Y)H?MwTL^O z+urm%%gyYj-gU=4UlmiGhJ+8u3>Rj5X&K>J@NBHbRuFT#+u+a5e3`h)y^(}0zh%$5 zksHIwH>5v32j(#lvdp4^%0-nz~^>gx7|<%}10{ z!@Dk%%PvlrO1VzURmc6hV_~d@L49iVevp{Yj|H@0`l3BWgH3*3w`~w#G0OsczH_@I z{KA8;RC;0!&2(IDWWzSMT;Nz5=@`{IsL4f^w$|d*S!`o2VV+F zrCD0;ZG~W=Pl+$tm*>yRy3eNyVR4M3hWNrjmbIFjYo+#PhaD9$ z`reRywRyvLy4f7tkDfC7x{2sWwD#7Qx5rCc@1;IArY1G7xZ@pGwi#a_1El+@?1(=2 z!i~*JIDCrmZDz5ktUTK@<-|rn;GlzIZ^+DIxs77+4N~w8Mc4|n4)9xr)1L2_cKtlq z@3Je)uN&WAR()$iwWffl?@IhBVb~aQ;)ccV%Nx(R!Yjomv6M$CpTopH;-))l-Qrwe z+okVKFN7LhY8BBNJWAQsg2^;Z0|f@`@S)L-3Gv$%P{o+t8Z}^BCVNo1lFqtqO&_2O zY*fRWpyK|-;WW1S)ED(}-Vqt|pBRgGL>72AE`=8tj}hm0(*?H1_YR-EezIZH7ULA5 zakG_sI3^C?R<7Pmlec*(dw-!p<{uxD)?3pM2%Eb^cSpZW4Qa_3vFqKiImYRbx6yMI ztpRI|5u7_$&xOy!2G=TeGhaMQIV)UW*7Fx1Q(VU+I`3uU?qk)xT2)O8N=vJCHu0h< zK+c$S&+RaXdFWDUWi~@flHhEVHFHSrl{nNe>ym<9@-FkI%EOOO3KEq#?}f#E@RQ+2 z#)(c!(#D?~xsH200ceI4|Oos6=jXQ!wBRKt-6MwgR8W!2}l^|Jk|%dq!{ATc@o?KTU7m<1;nxyxBFk z_2jYsr`Fbt*|#dMFrsh`<(Y^yz;>#>2%9yYbc-&fSmQiQ5b5<>xlkAS4l^4+6*@68 zEV3x zR2uDbM)hBuprhFGE3nLag7c3!!?8TWFW*6W1(@>8l8e^mh-6PMR%LkcLq^k{v#a3a z#x7#-HA_izd<$B>&OKRk?oVhD@>6yITn@Z#ORoHdK02KQHn34`ppYs(bdax=2@8cD zPDkr8DnM(JnL%u~rG>+iQHGH3z&Fba^F0eAhj^n2R)%G~wAw5Ubm(;S5brJ<&qsU? z&rR3T(_UIIc0${#>bU@iX?X=I_aO*!TKy8)5;>rV-;T<1PsSUj^}T3f z%0_wCEJ++quI1zt-a3%SAS-#*RSQ;d;$1Q1dIj=5OwL#+9#G~A*IJ1R7<1)zH&T*d zcrCzxY&=a6--mbec_oA9^}3E24L7ds3R-(HcgPA}b2i@xD_mZzw&(m^z|b~u(Pf#( zXk7{sMf%h89YPrw{+z*hS?<4+lA+~cLH8ON-+4Tpfz;F{F?0Ki0@O%@CCU5t;SyL9 zQdwL>Io0fGW)Z|pF&zT-VdJeTOB)pX`$PX=X<;mOhr#zo)ETj_u z;HCVJ+HUd<^x7A+Eo(NfoMWPjM`OmWq4%y~>z8isewh1G9?ru@HaS`@zfy(iZya<# z1X}Sgo7OCmEOaep5RiID4e{lm9alBYIY^fOX3FT4?dUXki?E5T`j~N}A;~8t^mCT5 z^VQspnUs}KuN&6Po2(X{uTtw{e~TeXI z|FNpACsymf|DKYz3RjhJ;X|=!HP^M9(@<9A!nIj*pSq7JV1O;_t@q!rHm@6J@ZOh2 zhlzonV4SxzW_#Anl979~^{y$^z?yt`+LRZw_@}zEwUJ3N=CjN^b-Q)4Fl6^vN#$)$ zR#s&$lUQA^Xb5fM+|v7{H8%w&L#@JC&7mzyD3{~6yFZ#zDZE_ww!}SjrV{t2#XL&O zDt6KUU+p`na2JWylVK$wUsgm~ znTSRR$m`G;BP6oG$zta>Xgn+%y7L?T7rOzWrKl*~fLIJs z*|x#k5o!>AygyeXqH*z8mvD@?#6|-V6elKw35*KgffCC8j2+eBFq)k^8>B+rKd z11=7A-MCXOQ>JMSp^CW-M}%YG;>b=ax0BFU$u5~UpbXiumjr{mlYHpg7WQ+?)a_)c z*K2t}mXC*79A`^6@7$#QXpiBBvhKwzh59yLH10>_y&l75WXl(iNO0&JcogEgnYDh& zzVQt0H@uaeT`5?ggxqIUqqx=kqr<`oTGR2vps2ZWBnXo#?O28=NE5<--AFGZwEU%O zH;}QWJ!xfx{r1s;dyK6RRyQE!z`04a0jYS!C9Z~S^ljWytHgqz1qE$wbk}vx$K0cC z?hd380`cPP`i0|3;ZxK6Si6lcK&lC0UcsSaJ+6UZDV;#**Zy8vyvB*7G^GLh?+X6L zJG#t~Wu*x_w-&pj5b3Yc`?D;f32ixeP&=0Xv%`4_#W{=+(IwtOd(T0wIFTxThi=5z z7h-U*a6x7Sg#VZNY1QZ4UwKmPRYlFkuV@#C3tQWBpyBKk`Z=XKJ_m&R%<;Ku{fw5o z*hKXtW!riNDc00&+i|W?%W@b5mH09e*wIXONn)Tie4e?KH zgb7OA87)z10L84`-d>slej6uSbAR8ALj6}eJb5%KfP_l964j@K52Ct&v#NR zLBQE^zTnuRce( z;a`1f>kePvN4KZBBe+AFl}o={B0Pm%2XDc)PpOQrw3;SW#GmPBNWT1a1^J7tonWW; zc4Z-Q%Hh~E?ds>HDGADam-Z{>?grLurh8H&e!1n{#MbmAc~^Rn^1qMFOAQYV1!p01 z>wNU_+j)drF~{#xd*vE^tTt*L!)ajGojKBdY=}-D$C;}_PzNNHEuBb{9@xndsr%ef zom;IRN_aG;*T-Fb5D)f2H? zqpN#^tBgQ75oPOvc`>_9B6Z{6z{STQV34iaph?X5^}}nc>mQfnU~^dikC#;NT6Vfp z$6jIgcjkQAL zk`LtJL~nMLxNNRlwH@7i@%F(;oi*`8do12SFm%(;VBZbJ^Stq1jjZU6Z*da`bFol% zjU#%VujDH>Y%!pl_Xd=ACq;sWvI3{6dJShn6^c107RF9rRqCIYa)PE! z_78oMCnR{@xH3}hzBqkKTgT@4x4}>6jz=(A@JV&OE_Bu7Qs4*kloXK`QqU)dCd}3EB=WSYigM+2?$|W@g))XL3=3x=@5f7k^iisfH$D^C8K4{!sapI$KDgq zhTBs+l=xc>8W}zCZdh{=ntddMR9t2Q&lSTSDb_ zc!S9ls_is@%Jz%m(Hh%)^i&w@`shl-X+4b|HYYfas%!Rh86g<(IW8n3Y-%QKhxO}v|!o2hH zfU!&?-0r$gpjkW3o+8wE$gI}Hb`*aycsXU)J2_w@)lOAy1@;zW;2iqGL-ed_oM&44cjt^Xg?T*1bR%~81?zZ3^mEIMi0<)^#+$tPJq{C`&w5{bR!c(7y|#vy zp=x7IL&XuS1ZJcMly+pFmyDkmx82ahl*~Tj*Qz5uI5fW{YZSyQZ!2B6oNL6ytRd#n zVZqYHxk`a`5fRvf?K(EgPmxkv7~Ngh87V3<5UP3%0l5>%zEzL zzmUPo?ToZI30>OV?$VmyYiGEQo`GDy8q#wYFPu$hevZBg$EGZ9XCFR^Cw?9PAA z68nnC?3>W%iqC!XdlZe(RdJLi@gJR_%~EKSQ*hgJkH0mofq1?xdJ<|!OLaAStCp9G ztqO2b4btkpvvE);C;o>(f7c>kLF6eh;qgfEh?J(`=iN7jYnF>vqRN3W64-k6EPKsx zW-b7!b|~Y5(9d;PN|)QH{W8289D8#t?_aeuD?qH0%>_Q+n$9b@^?1{@Bbv0kU3wf{&73}(dkfcQgW-SpmZ}q+U90;UC4OPDgR^J z6!YxDAC=Y1t~Aj{+GTaW-l1kw=q&K=Pw@Em@hx$ET7G5IXAlo6)j7SoyN25Liq!h6 z+@ZhQV%M}5GQ+tz_MhQ>*57I;-V3o*{S?L8PV}-2kO}*PV&4;s9bA_F^{-y;rrKeP z$EG7hc!AUN+lIeY=8jB2>_g{>n!^iBAVgWPt^Y*gaWNFr7!`>A)Vtzhxmko9n4`mw zP3_#oV$7;WTvqRCBCW>BUjbi5IDLrU>fpeB!2;~E11Fs@Yvvhd|Hm&dD@afq+&miJ zZNJA#dXW_V_kQ`+hhRuT`E9AMikXCa7CS4I_xnL^&^!D-Zo`+Gr&WNHGZBgyk%fM3 zC4DJ}$^^gWb3`q-XXLwt!qVOR#ZsNi#)hPl4*x2_>0MaD*`1~8MWvUZh~CFwg$wWn z$Jq8a0ev0r$h^7;cjnW0hnYgZ*y|u@-4cKB|JIV{ytSms$`OyuA`La_K>d9q4E-L-DA=I^Uwbf|d72&3^^G zcS4;3ZYyPj78fO9T4tyC=H!ciGJ#}xK52yQ_iMc~e5KPnjGaY2)@SPD4;hA?M%QjO zrT@blg$Jg(AAdj8JX?qqbYAqn4Cbo=*3Ui?q4n{BoUfq1kxKs7AZJ*plHD&He#nm$ zZdAuntRB9!XnaGhQxrR$X?(aViuVidG^R_#V8h=W%p>6 z5$PF6D;4z`4b>{+U@nc5_f=!DX3Yh9Bjf+<9aN*b0;wLcOPSa#iB0ZlN3N3G{Es)- z*E@67Po`Xr@gUFGpKBeFMsGTOj@;>oo(XcXe8}B|dc7rRNyQ(_6klWxO5cINn!-{MOYYDqS)A3<~iIT4&n zxUk#xpTpC!rjQyz`@?-h{3W3EMhd@B$8o`iQ*Lh}2A$m|`;6u%g-ZBLYUI!KCq`k; zh@AwXtB6fT?$yLY?rPN;@*!%AaIY%=yyKVoEE($>IrpW0E>;)vC12{gksU?T&!K6S z8vQ+SJN$HGQ$xMpxoq>(@_KqdTW78V!a~FZbY-4~XdJm%!^fUgwx~0)*fm>SSl78s z!u~pW=qo(DqtXAkY$Pf0jgy*pfn3;L4o?#@{jPGr~6EUx#Jm%&+>T~Wm8pG>QdE%zyRcwE}9=3RFpJStTy?=ojtEYCoWcwzz zC@Wi`sO60$w()bT++t3d}&7nKcQwlW| zPN+5aH;i&``UY^6qsG_x(>}!J&f>;mmE(ZxL|~Kko2S7ygEc0UKaXyv1q!U^7V4i^ zqa__r3uN(L*!S;GY1z1+4=$_LljBDE^!m&(X68BzMe2~k$>Yd%*A2-LWJJ6U-#uDniB;&E+S98VaziBya5VnmVmEB z3$)QRLN4IfH6E4|or^6Zqha^UB>xcsHWq}}JHJ(&bOa)^5r>c2HGuMrA68aPm=-u> zM|T$97kgo3Vbef+4t;(n;{w~4yi)lU^OfO#eEWDk$>V5*zYo-s30wY$OTPn@?RP^XWaN-T0Fcwi8S+eAy)y_BOTN-UsO6N zd>);=3{8G(UuH%Ch0gYw^v*^g+O0xyZ)W%h!S|l_*P60c`phZjQYWHw+eO?ueOw-R z7`q_WM%8_>HPrDh=EkAbaxLK8zG8 z!M$8hs~?vSTHF@0jzs7nA3yAbG(38^YWuZLe7$8TEstn!*dT7OGnnq<7Rfr?EL`h; z{b$$8xde>6mX2b&**6WD-A<30z~x3~olVi=z4q9Z=A(euD$o>CpF?ErLRF^W@qN~! z^BvQe?8Lrf;BKz^u<}3UYj-c=^_ijFjW55?Ju+@C8Hdg!lr5zz6I5531(rUg70`Xt z$1*ewW}WzEQ{3#(GrxDIw%tPerH)nvsWpdGM(xShMBfGj@C|R#Td5Y*n^L&&!Svm2 zPZ@V8Odj>`&BD?_sGtjR1L(dIlbeRCFTKClT{qD8YcCC}Y-q;|sdy=bm%4bjTc>wM zx-XV3jXG>;vg=3B96g}yK;kp6MLuGzBx zf`d!aO+@$cXDj00yMSq(`3Cz}*PnFq872`yJ;xwN{)u=M*S`93TwCl+23hobiDUTa zWD)}J1f7#{TD<(@{goA^CrPq?eK`_j=xT+TkqVI*{8*_ zXjkcJO+&}Hym^k>;zo~~WaB;Y<{V-8j!A#pXTsh}m-0^JllbGn(&G_BYnMKs;^e^s zy~z?+R*rp5Yd-?IsU<~vZ1#S8=*OyNBfVO{+K#Mp@uvg*vj;&go^a$Qd+CHSVYo%y zslJ#sS8orw$@(pp;j5qCGM`mFo*?Tkf8H&agSf`<+8_{-Um;=Y(p zM=z^=dTJiEg4%yD0D2U9aYjh0YZ%M)qq!b!UTb=PCw2c=7kBVlgJ=2svhLPU7#%gouePdXf^16=~F`mVJis` z`)0wapOP4VJZPPKB-vW^&!z=If$n1`DK>y6Fy!zUB`CkJoO9yM;?{|)L)N?W$AJqO zk>?o(;&SkxcQIxYo)0AmmrXCCCz!;vQ}>RR89%(4I;nPH;VRheW2k+GBY{e|ZSx7kerr^b3rNCeOi1C4+`S<-}xji*yhrv#79 zYZLMFXlDIiS3?FKjD3&zzghqox2fSTT@eyct1aB?E+xo?E#LKgUPQ9={w<7uy#O3OY47p&f3 z)A>mc(nREQ>%-DYJeU8``{1dZR8W_b0J6Rr5_D$$Y@58|c{KxKl%b#a(PeaHu7(}K zhD2^ixq|HLWjeoRu+lk982H0ZaC^p8y{ts9I@O zedYUu1*?%RUNI_-iai#beruht>UEXXRp0lzmyXi0o2g)lE^LF#+rHlyW#aJ$85qK> zfr{Cw_pjkK-S)?<=2PA(yR3H*3~pnwDJh8xx{NIM2avsUwiM6o*flZ<#b?DIx=!(0 zj#iPJsumu61h}NYsr2K{+}0qz`n6hunSe8_c)jK^CTv}5sx6@b0cS-NY(_u?&Nuz~ zcm5*OI_d+1&&q|d#W#}Dpid+pt}%I0Z(hKSCiW&W72u0#iM~{WRC&dTPl9PBOH~@( z1J!|KYceq)wNq+*>8lWUalztoC?mz~w#$%5(EZLR@6nZ?Z|lWA0)sN^Mdr#~Sn}BK z1D@Z*w+Ch);PyhS+~$1U*uP14NAr^Sl`#uxtFx#NydOy~+F5z6l|DDv=oS0>dXn;< zr*I6a{9i`7n^)n=-JmT0-a2{1xnlvotvduk<^7cKCFdPyQZRdoF*L-#$Iodx6$mu) zsf$qid`Sy=DnCWjy&Hc17??BLt^imo*U%#r8~=YieRo`vYutD4%z2I{t;}(d2enK`IqNv41ucg{?40q(5hOdOda=7`D^b5C5QqNRe7in%34#et}R>!tHN&-*v< zxj)?Fx_{SqhOdwul&rR2=F=UTItV8(575jJoSMJ(HOE*S`K%NO$cdqiUc*H*Eq%tFxZQ;2NK+z*_Xg<*>| zQd^Zyr?yRC4Egj$!wxYpk0oQqvUTvRgL#Y{lc+TsPmH)Zso0L$cJKS(=L0Ytt5U}< z-XHCV*qSrZ8N8uAx$66v(I`ePOQ8g1+`#lJWe>p}m;l^<@8I|Bp0eF)uV8?QQ+4QG z*w9Cr?vth#p0&+vkpnRR(_4&V32)*JQ(GsTRYHCL>8clcCDD417nFk|N3GT+Z1(kfhu>pOM2Vo2hfa}@M%QK(|W%VPfjWaVN59bp>T#C@d5avfju+?{xB4}c52p9I<_#KN`WNpQ8EyRv+_vYoyn?WCw45>1g=ci?m7(N znI2e@12osxqID;9)?su#UIH+C;r^s)6~pzk;j@)pn>OEuS{T}mPQKNNHKJ!Dm=E~) z{^6OX{5~;=N`&4m-D$K4Nk%X&{qoYR)ty8D`wE+?tt!y4c1g&LEHV&HsE zx;or|HkZd;lPGdHKHau9^(OOro!+%4S)^6hO1)1) z6l{CGk+r=rW!so}d_a8md23)Zb^QYv(R|oiWv_JOx@%ViSoz0HDaN4k;ht&#&yCK% zOv2hj3nNZ^c}kzu-TzA1UGqHAr!G^zU(yPZRsbDq<#odyxN==j+SL~tcx_ZM^dVkb zY4V_Hrr9>j@Nl8|P!w=7O6h}!gEQuOJv+YJ?tueiA(qrHEnCj|Y~_y*8wldEwN}k5 z!7VJ~Qb;(;e1cL|&RJndie;tLk!*o#pROwRi#v;mA*|zmmD+@E1+8C=;JW@|&+6oUwyIq^bt@d}IxP5RUs{Ctf`K%4- zc%nLXxxnCV>KvrjNWCtQ88OobR6jmFTQXEh^jcE}QStJ+E%yvxJj zYx5VwQNMp_{k}O&RA~;RRIxH6GK{a4YD{>No^|_w3b_fjFUa%l{qLW*J5lz7 zCHkOYNUlbZP_92d$a3mH#>6r%_XzQsV`G4Q^rw{T|FFiCPMsZ4{+AH+%ADwa4pGB~LCJCt48# z+dmoSheEpFR87Y*1RydnFrDO2Nz+zJ0w|RpWPB#XT${Xg^ewl)r0)-&-#N#b)Q}}9 z1&~2FI0BqISL&gY^@dG}Q!~T{*wlmE=7qTX4P~~d8|uUh$mJf=cvRn;Q(*3}6xBt|;7r*aI5$`0UiT>{ueN8KB%uyt52nBKj|n)zGN= zg_%3m*{g+*w)cq4frH%v6LLbo{zLBXQsKc*lmhk0n11KWt-nn|xAyLWI{K7RU#=fp zD3k&SeUTxSE#_8|@ey8dUEdeLH4k-ou+>yzFBCmFMzuwI5FPszh}u8sl8f_WGekM; zfSEvGu)Oo)Bnsa=*1gc~FJN9CzNY^yD-Wbnt4AN=#IBKj4f1N&-x*}zxG(Dlr|$v! zVmr@m!k(cwlDY3=JI#LIz^Dl~1D@|$ zfQoRPEd3V$KUiXh%+s;_>vVuD)gCR~PVP<4KiSk&etkG(WX0f-??3ZR21uqiylE54 ze4PSq9-@BH(j)`;fJJ0*lWcLDk;zZF@KKxcLt|$yQ`L~Q(G2)Whjk5`YDA%?^p9@} za9%bc2V)Dg4q2##QvXEIm(MkflTy+Z>z3~Zjin7&P|Ah$ zERXlt9D=DgrGN8F!zPJN6f7BzOU^yIrEfhd{xHgHgz3j=KH{U=hKHF{(r)tFiIuEhrOtM5ih4)GKx+J_sZ; z-|MQkU=jl%v==?T#P_4+k9?62+lx1X7)jJ}?&mYd*Qi<*AVvu1mtW?FxwHY>rzd4M z8=D!+=Agn|WcPac6$^b9$cOqg|0n(fv+B{&O!8dl3Ipz#e}>77(sM3C zztzTIzYd(5_0shiYh3H>_%PPZfkZT#-7zFCL020$HGRjfeqJx~d&xGOO5DvsH;?*m zXG+%yFliJ7O-*5@%S#8@zB{ z%qXf|{Vacc;J-`H@}dODhdk4dei><(J%8kg^%tgTV3+p(jP``!iP*A&5v{EDd#e8g z9PpDA0u}3@oNOpZAANbVsrFmx<#vME#N1D6;gU{vw<%bR$ajk->K_+;?pf(TG%YgY z*X=girsX^}`nGBfWJ98|uKE z`t4fN7eB?qE6FitX@fOyMyHb0bC#=S-wmYAFZ=5{6YuBgau;uj;u^xKrRcpOKcy3f zIWfMv0F(!Ovq;dUQe+pDzG`p}m90ViYw#LID#%`&wGrPo`^$W{pStvZ({VopSr8~t z9HlH{^0C|BGC**CyRDoL36FypwNxKQm{mAi+@@zX9)hdhw6A?J`(mKqXFyuwi0^b# z)%CO|CA1OS%PI@77hE_k>5iYxJ%e1 z)h4Wqy-e^K3&X`!zAOFmBXeKx=`=9;miIxq`2d7#a>92(KX3rN9YF0qy6m&F9eJ=l zRygSYepUgEL0|OwC-*wu`@?jS${gbEt>ZaeCczgG_#$4OM0BrqsEEmblcN<)4CZ@3 zb_3GNqXIEWP@qr=RQ0I-OFE?3;*6{(Rb`#j4MYcVddIeEh`~hhNsSl6&u}N<{Znv=@URx&(PwfF+T_U zO%>JK*$m!fhzRcaBdA2Ntj)r?8S)8ux>l}1*}iO4T^rDjd8udMu@ z153>PG)fZV98B|FX7ZQk6az=nP`UPig1VawWW{g;?^!{bp8z4;FUorx(!%(lW;C+u{^u^cvxp0&r5DE8OzHG50@xZ0Ycgm-&B>y z1Vx8Qj4=n+oA309ZTjwI?a%!j(m%JMjAFh$I2I{fO$my4a?dfWs@P^vt0p>LZUUuY zIwIcPwftp;Hqozf!uN#K*p%sI1`n90SD8DpY8A!^O*Zz~DS`7!fPomxtoUT!Nb?|U z?;}Qup9{|xfp3!N34Jnr42iT8WmY%bF1jP?>lOP3!j{V5n=TXl%;KNC8N0A{*qM3j z%6C;&Sz%tR^l?NKSuun8hTZ*J`yr>;V6WJ)=)T4E@E1mM4bP8ZRe_%KmZ*{0Bn<-r zL?G!=;ODA&Kvlcezs@g}4SH^?cr_lGBmI89UEvVDDW<*~a3vZ93SYKQ*F5|C{dDYy zvPh24X86#Yl2o(EQ&Vv*gWz#BCGTufzKSqr!FtOlFfLjWquGqT!=*!&C``{82Lnt! z?}Zhy+pn;BoJxyJijZ-LA@tj@k`WX=Z2VDc43P6-qQzb-bbHaMB3)^b&gc{2%(z(u z_kbi++TSVZx$zBP)0x{NFM|ItJ#f=@WQuNR=g41=IRlc7bB}GTUhy)vTf}I!5+AyjG+J?Pug^^tZAv>d$Iuu3*HP`$SirdDW2Z#tXeMs#lzSH9=MIhWd-KQY7 z1@S}pD7)g!E)Ckq(@c(ks(L42CbLB(R{W7yr9?TN)cZ+Y)u9=xBLE=M(or$h3#N3y z)!mc}_orSKn3<(jC2*LPrA4zR*8QPVy^bSmOUKuaEKt9aMKLudR+!wrA;%UP3 zA>d`_Y%F}gLWs~>`DcVMbI}iS)Yh1}d}ZK+%+hKv_0ryY4DwsikE^Nv!udYakzs+3 zviFTD%$?unb2J*aIR85LUihsxcAtI2w*9Z;i}sOcVnMCO44;e%k9B?o5ySVH!L@|0 z*^{*!I9e38zl`U*HFl%ZFIdAw+O=QzKwc{n-ySDevv7p75j_@Og9eCFM=U zzrlthNVnZbccy)%^ByNV1*wA2Ei8jw!;|KS`%u1UP|Kuu7 z*qJo&k{=5vulr9Lm3{5YHC90G`7g4%>8X7F?JjD{!q2c7;6+${yWfjOuS~pqC~iXfh4^O&(?-b^W;s(ht~;(J_6efo$;NbHh`k+Z zq#a`vQ!Xc3}o#ykPIvnFqJO{nmWY2mmSVZW#WLSuDOY>Eg$&j>Sn6bk#0PT4 zUq<`Oaec7Fv%%ic@?d(52`kIwikV?K+Cu5h!_@t(fue{zJ))^oG`s!l^ci5{0NZS@ ze)gDp?m)8T;-inHUrv!a9B((E@4eAx+2~YeD@zRXB_KO>Cc5_fx-XL#d_?c7`t5xG zN&0x7ox5T)MoBM;e4lvX52#_6y2lWyWyYnYJf;d~VBy7L0Y;3RqZARa> zO{*f>e^CE5SzR?cq&(y6GRd6-h|7GJ_X7~S2vsVok(*zLHnlKy7fi2fdc<-%PUzzdJUnGx_hHB*x zbByOoCdqO-%SMG4-wibwTFhO^Z`+1VSCtKS!CGb%f=0;Ii36T(U@I)eL(A>)h?~{8f&VDqUe-_(VxVdb?sB+o4b@}|kda{T#{OB@e z<5%jy)1f-**c}!Bi_OtzLlypTTo8Y=8=We(e>2->|6{-u0gRlHt8-CIiDHYj_>`h5 z2}W1;ylmPOm7&d=FmJhNWt>kge_^n3tb&+KKdBylwW1K6B6_9#^xN&5oh{G4gi^;< z>Z>t&7o^h&CS35vv3hUJ!+%-dBO5Gj5rmP%}m4DLi^ za?{*`N`H2hAs-ICom3=GLrY9(ZOU80?IBEX`GIKKJUB6RG$7MBM^@WJ$v>r>_;@op z`>jkf?PdSRQ7@eZ2`3RzfPT}1bKd9WtvhnQbw*2d^>^o-6;`#`6@}>nj?K%87)Y(b z>Rw%5d);o4itXnNc7)BCj|ssWBflCF8mkdkMZU+J3YE*iX6%0vsAH329l6ukR6B)i zrB5Cf>`cChsIp29ln?kkC+y}CX^W{H^$Cab2Tuzcq)s%2S5~i1<LR~D zH{}IZep%2ZQBIea62C)p{&L@t-yM@F*~ATCPdiR{+TP#!XqSUt3~49wo`J@3mv(Gu zmH5ZkAuz>QpcKhxIX9C@%itLSSMJ35n07Rv6QvQ&hh&!I!U8;6nHToCzNK&va^ z29;`P%m;$R;oYIC##slaKnQMJscBBc$aBGFver@uBUh#-z%EGEel^!2p`od9PN;Oi z*l+_JPSuC1&w4Rj4dpxG+d=O7kqg}n7<_QH?=biDD=Mx$fMnwl8Wz|ayB@<1I_Vr% z!o8hqIh+MdHi|sDP#Nr1sidV?btZ>i#f;;t7dEmHu$=XdzZ8N>TDOvZUZ3(Zbl46U zD4orZWVvmgaeSb(xjyt1Eo22Q$5##!J<7}j3-RU;4%WV3Ypk>rn~(#-cT7GdxGx2Q zyW%4(fqSFp8)=+JvnrLKHJ)gs=OKDV)dztpv(1y7PO;fZq|V9}RIIqs`fE*8N6nMX z>)Fb+=TZT?mUZ;p6wq6~-sXc{m@YnvUA8(^Sc=@dSp+v=+XS|2rUI`_c@3C1&U}62 z+T*WyriwKet&F}8zqWlWNq?MG5{s9bqYE(>|$*Mz{0_9S<0l)B=U!cSAG$AD~f+2OAyBmIJg_w*mFYU!UhVZU;>i+1Za5 zS&8xzdJ7xG*0Gg^jycnnN5RO^BUvfR*0#+uWX%CSp49w*c+$V1fJnhhXY~pmEhya# zsTem?o+vmmNy40@JKng#1+WYtmA1;BlO4aE+ zmYxl36dR|s5j4rz04^oo-OVksdA(q|h}{M>cVKYrwQ@37A1Ane#wfL~$+X9`)toWL zCk3_RWb(^GblT>k=ygcn(%nzjBV}6Myj?0Ot<5Fb3ea80Y)|kpW-lL0-sj9g{2IMu zd)FZHs~c^uX>~FVnZ~Ih=TBHE^z+$ak$1;d_Gp({KQnHMK$Q=+S=CB@5%35Yh#1B7!Wc%MrtBc zd-B~amH%Xeb|{Wc-WHav(s!^Z*5xTu2SRPFbwW6H<%u*RfYH@F<|OAU_e zwb(k{dR<3(cw@dO#JyTQ)+RUgr}CF_&CZBalOGN;&53Uny5g@Lp1oLqY%x6Fk@D@5|k3Op$g2X6Nix5<%~ zX%5!Q$4}Jw1Cj}5+hYQ>R_1LwkL^crXWM3tjCVNNcoDZUzjeE06f_!Vc5tw zQe@gWF=Ucr%89tM{s2s?&srfvqNosurjf7OxYU5; zyJSjbc4gVo!=kvXJHY4jj05jf z$d)%3#YoqJhcPpM%w?p`U+AgLU-$;h-C^F{`fz^CHk{+?F>hWC^k)NM#72K;akj@0 z5UNuo-a*LQy>sQ|htHY)wX88*=6Y}=_s>SV0WS-KPM{FXpn+7n8w06$^6URMvr%nr z!_M_ylQgzhLBz1jigN=-^@>!F%$5GO6hAReX2PmQg5xX7l&vUd>fFFn(J(H6T+*(f zxtjx#_v3!MWb}%vmWtqmXFTM`sbMs87`8o5UT9+{Q^H36PYWP2oaI_9VbeU_R#UoV z7D1U!j@-Q7R8M98)WKP>I6-#4C%>|&FmJn{)b`zaT~Ob*?N_MlZNN5UQ;-)$>4UW~ z=>u_l&O>dY>SRA~hi~z0y_YPFQ(QvAdhxA|X{Vtec>I7z1yZZLPr6dm5v4xtF^p6k zE<+P<-*Kpn@%9choy^;0juMRyb$8A6VD9X7#OA_sN`|DeCMr3arcQHHm^gV0GQJZi ziIPq>K??n-A>*6pCSvE(rVTrIv8xMyDyRE~rIA_vcqWq4H&5&7^)JHl2G`x#&*UQ@T|`C0n=-6gneUu=g3h`j&RCj-!bHI%gmL2F@dkZ-s?&TZTJPcQF6&w2Pn%ds6RUWfg{lKwx)3MHgX~EhMQyL8)c+(vKg!7zM zrrunlmtU0kslRqAJjgqQV>diwC04{g{{aZz!thh3PG{uqtV*j$K#VgR5g=lLK-T=W zBYc(5kW_bSAKT$aPX$Z-GqrqjmaM-k-(xmRezDxsp;hp$Tf-RabwE1@h6Tlia} zdOFEr@=cWva_()yQq1^CKt+vQuJs`Ov>*vVg5FlUTzgR&s-qHvXsd};FCKlkz91-W zxa9Lqo_C|^An0Qsb#PDgJDPz8kG3i6SG9Y~)t#n1oL5Q)`LDZtjP1HEySM~s+O%VR zD`97K4iO}GXB1jP9WT4#5J86UAZ~FWmzJ7jhhS3)yCMYB`9&d!3g^!SA;OAiShA$ES$gVKcPp-b8PF7#r=aqGK+mk8E2BQ1^eYSAYGRpQz(xw!Vx**7{!1g^(_ z5Mh@#hF|s@5s{Pqw0%3aI28&QhUus8{0e8(vy8hq(v+M${-2SF(4jOLP(78uyDVsf zSoo1uLns`z`pM5=ngHrlzO%#|DG9#(g-sL5jntG>+i{Wo>DdA0FyHAn280b#K(jN0 zT%T#JIxT-`zuKI>sbe7UGVrpF^&sEtVu%wKv`H#If6oKeVVgHRFsiV8WJa*6rgNnj zaddfF;c$Ksc`|l!I=F89jv-#qcJFFiBCM@XZ_gfAY;m6c2uer(w7(K8`!@Vr%@~tn zIk5E^DV4R~fZJ(s3{Y!RxcRZPja1%Z+P(UG^*<-omLXZcNB-o___vU$A8}{S*LZ)C zCVXN3PRp_O2tMSE=`{_^5Gfhwl(YpU~6^^F!9{NWb{>L3*f%Arl zx)Z*EjEUOQlew4l7Fx^C4O?uA7%jg468bPnZSwxtZW7%jy5agEkRb&rgd3<+9t~_) zANfy~0ONrMO&PE5VWOfbnYn`S)I;S0l(t7{`Xq^S)=_|aa>Q|IA?EQP? zn*R5naFT5Ea$MW=T!eKs;qLV9EeUusy#04xMV z8_Q1^yUkeQ#Ct!v6#iB({1Ao;2k}2 z-low0Xu+KdD}eH@pj4R0U!{FR1@V~1(H1$^1w|gdwsVQ*NoQlURUujnU*$LDzr9NZ zEoD{{N5lFTn`-AaR8KIkmq%r!pio-J$okbK*I(^_)@tmzp4e^c5&kQ>Z^HANfNk6L z1k+>8j5vbP2yy<^S#!Jk&RyX)V=dhVUFBnW?2{wkB;stygErXIr(wWeOLYd zfOHTzV&v20uJeVX#>Z3Q->RGln-w;eKSDMZ64z&qMY4Bm7?VtHi|7h@m3Y*eafO!~p*{XLj8EL}ZrLrhoN0 z4IdRnh+HaDwWQ~(*54bi)U}@G-&d3mdRyKwU3Gr z|M&ef2FY&6PCfclG%1hzmVq@i?_@jkh3)ha2{X3@J z0QvzZ^&nJ&1vxPH(y?9vL^iYJ;X$L*ZV7snpNLPDrJqwk*Y#lM6~AR3e9N?U(=_;= z)jK7e&nqL&CyV9^J2L(lF{mmfaBL2TGb_Y9dOq6O;+0Gh6^o`4rz5RUtNdw^`npOkfFtJhwHc*G-GW)+H z4ygKE)m-Ai-m1wJ-10)yWC@nkJqLgv+G|-wts-3Gq}vM%vJN!Di{zS%h5)>f#G&5N zzj}_cmsVrGYhrG60@}~)RLdQ-*ChIRjO0oR;7t}knztz(bX1{JDv#|n*O=j3D~J8H zO-=x-9sRYw-J4K3NxBI0R+G4|)9(@N$184P9gK%$gSBw{7q#Z37z}0X;k-o~Yg_zi zcxZU;&1;e4fC<>1{^8gd=XRea7(goeDXFg<7F%jcy(aA+a|EXmzHn6bo;`5mLA7|F z>Z+MlR@CdL6b%c-42gOXdD4+rY+(ymykpMuPjO~|QP;Om)*xtp62xHcV8-jCUtVB# zGp$;8vn(Ul*2Y#m%Z#ZgH;78`#pC5#hOvU#u2FzH`_}fibH)u4WrEl4LI|GZE4go1 z5M2A8i;@S1ts92=`%+J-AE7QD7e!!#sk`3ue;v3gyo|iHkviVwsqwVBVcg)dz}Xx@ z(lYl~Kx9?7O2^d6Kc_G1Em+VXCZ-mmC&Gsuc8cxJ7p>M7l@c(2PT2qrMb7ZD;w6s@ z!VLIkDAP{+_T%WNHi$ol{JfxrD)eor*hEg^kRsCF={>y2F9Mz6ZpwY~D&EVhCQ zzgBk{V#Hmvecy_v`gg2;cNgAMHf&CbzB&7`=U$|L$DAgN&CiHWgGOY;gTT9?<@2oe z|LjucWHCv!sH2~>)a{E728uq}LRt!-f&)r3HmJY zYPm%@yI+r0Op^8V^!Ko1O0FkE0eWudG^nzplAl?T@pol6VC|dPjsJ{A4BMWpq4P?9mFFm7Oh?cvnmu67ped=S?Yx6e! zD1+=onDal~R;JFBOCx%Z$C72EI(G6h#O-hYeWhiNLdNH(!*f`a(Re^!^|&;qSv*y` z{Ta*C)L^7RNs)NyzQMRBOCm1;-m?ZgBUc7_jc~Vz8BNU~*-IDl#3!Zm zEGmgT!T#68OG@0SSB>>4RJs2Y%d5A9v#rY)~^9YKhvbFI#+ z-~NDeJfsehIWYIUBlz>;;MII^b;5-dq+TdzVx@F{R^R)DuzX(OV|4DM<_y&?_sUE` z(r%EOetoF#)w=bzkwF!aJ#kIty?kkAm*`&7S08;uWw(3GWZqUtQYQzphOK1|cQ5cb(m)pz)svrpyu-^Rjd*yGZF+JU8%l|W6Z*)x&(`%1RXq%WhZLsW7(RV9gwEFjB4G)BQ;ee*9Jf->vKTk* zc@k&a1A;1EfG~BW$|hZJW0cM(;|{ ztp0`0qOW*B&&B!yj}+|T32y93L*$}+M{-yHF?6l`T?6%Ikb7#kO1tnQo~=hQ_1<~4 z!w-R6<;33+6ua^CCqJ35Ht~P8C+fK;7HM;^)AUZkhb%6|qnq;Zjx+Ec)R=j{V@PqF zmL_wKkmQfO2f%}1IgkTdj1&@X`XNtydAGevVfk3s;&#L|!RP2ysQFdx+)8zmOnmHt z$o6y$+p`n&*?#i1Iqs*nVVlosi$SmMi||{sb!m6+(8B3|ua-<9rb z9kS0#siEI8`4OT$=|ub`nHUy>!T)Ru63i3r?gr#X=mtpjuZsrf1BPsXK#<#XvHtxG zlRk3zk&dBkqcHk3Mq#oy32?Ih@>aG?dR5R`qGKbaeZhABD zK3T$?)EkT>XrNsETPnq!TF7acU%f5DJI>&wVlH4d<6Ln z*uZf5eFCw_I9tKDvYi6UmH4^rqn*q)2RAzsb*4uU^--04Iw`7O-(43Q7`sm?&(lnR!;DB7Fj}U9O#mII zGpBCC$UWIXkTwEN_I6cahH39LCNjgo11+ot`=69E5u0&_zSl5$B`@r=WClM>WMWqU zV=QtXa7TGE%c@%e%!&bmc+O}O21O1UtgezI9p{MD6K3>8PR}^jyNzapF_Anl(ZJO^ z@erpsUFmC!5tfWg+mB!sTEW5)_&!GoLwpMS^3#V9^7?U|Rqh!Fpmk*JSrhJ1Yc+Qz zBjjh1c;(;i-U)ep4S>3o9v*Fee;v39~ld)?Tgp! z<*b_3BYki6XIW*-Zv!gB4C8TfsBA)0-E1PXt}=QG$PE}JDP{z0TsgM^3^r(Ng^_}o+z zr@hN#Z8hSkY7ZVvSu2OK^UVZ;W~hdHftwld9}fIkgBRMi=2`jpvHI+-Rw7d?(rKSB z4z(H*gO&D3N0lh*DxRiyzP`d(JZ1${QW}|Rt)GL~gH8lymPBmcOn^fZ6P2BE$Ua<4 zwbr0f`BZ*tS*cp%XWvFvXrIZADz{RV{p!_>Y~^ZoF>Gak72bz_ql&YDlmBon8w!qP z7;rmgC)t*eu+VC%8@EtMJv)UE6n1`}SB9+5th>w^=UNR@iiLawJ6?|k>5?B3CDgoQEibRl01;KJXDg}LS%G2Tj(WKxOYngmwKo!aJ-&b}hP#9O_U<-E&y zO7L;Zu(vnJkIjFuHp%qdr;6Epv(OW0heb}O6aFiKlH=Fc7w z8v3{|v2Dc40$$l$$&4Rk8~^Bihng}7|NVgVRH#m>w+QM0wDCQJ`#~Y6(;Xt?JLPZ) zco;c~oTrmnM#!4K)fx}jnw9FLA|PXY@9}prn4%Fl;h3aCUF=`GKXdOYD6BrmK307Sq zgaXDdc=y*pI4sy9dR+)R@PfNv1eRsCwPs#&J~t+S79)FhYC_%Gu{Von?+_Ua9avsz z=iHwIdTip`w7kXNjn;|5$=$$Gep$@+TVpQJaaKqHzlVgpTA`&wN9FfeCjPIiaeo9| z%(2p;{Dy=ztaCMT24>dlx<4PE_m?OGzu-Roh}ckKES70ZUhe$hi$ZI23kC{4PI@SC zb55ompgulaIt3*Lg;iE2+v0?Rs!r_E_(}?q_0SBDi@Pj)V03Ni^!jRLw(LS@MCZ2d5DS@UO0e`T!YDPz~{lyb|{ zN~Dx>qhkXUzc3?b#%@&*bv+~Ydx#l#I7APnKcs(g6%}COXb&-%ItN#pHU;YyzK(G7 zH6SkjjtjqA%Vx0OT~KMbH97^ z8y*Ls9WgS{KrXmt#IJT~5A=8wku* zRHP1{IqT{%pqLp#k@$&J&u-P6+Nu3m``a^ng-O77WuMzW4VzG{S^k<2emHg8qUnkV zy>ia?N*&~`b?T&`6XZsfCJK)YYNHP?j1=IsLD(gHFDBwA1~s$sq=Mzl(h3#1{6C8C zK{1$;>wo~H%{v|Wt2W0q-B(u9p371%g2wj7kf$#7=neX{6-c6tB*-|Ona_2Eopd2k zhAl$p`y0-=L!jlhiHd?$2S7d$L8X zw8x)6DTN5QDZETu?T|LJ%NFp2Aoh5K1KwAyIv4I@wCPjp$@rts8miCiESnz5_*E-3 zDd>e5Y5%T=Tp;B;*8++Xukw_Af@=JIqbHvHBuo52<@`B8yOh61$BF$U_#NeMJBvV@ znE4k?MhI8vE1`MKe?e-8YB!X|hxsKjj)o2A6SOt{uJ(tQ4Elv4-`I?OgKBiR@zVMmtWM7|+-4D=J{&^ht{)lmc9TnZ-s?>g?K6mt8a>*v5~Ocg;Hw(! zl3RK~-}$zC`r0`@^vX%~uRHb8GXIO{iXC`U_H@0&*SJ{G3F+UBDWbuHncc9@O^-=& zGcUS_Fk0c${H!<1-wOM0^I;ye&Bs^|WV@%lUrzP#Op4IqVoLme^>q-e)>8sgRXeCx z{x;dT?t_4~<(Ue=iV5wj{_i`DPdhbBhq_v8w z?$kc9vgRJE+W*yik@G7TukO%+WWjfn-0}kl$@(k3<@kc@*p|W@-r1x5uWAvF*#?y1 zGva{ordFbw`c%+Z77MpGifdWNtF=%gP;HZovt?o?3z(Qw1VZUNAO6ZzT=C_XNijVW z#vMS9FP&3bI=D=UNr1*^qCa{kb=~XmS>}!VJb-tWRI%bC*Tfx@ijxNl zzpuui{0?_wR0cUV5q!%|QDj4*-afKAeA1J4^?KU#fmI7Q`r@2>H|@cAk|8Y>N7k3f zcKAOn03%^{Rn!Rd{UDS7+d)3SX-!-GoSWk+mg&at0obJ-9iJv^v@|EcLYp{U^7?eL ziw3*tS`ndOV~h@D?E$>y=zgPiZ(20$ja-l8D0Ce;-&^JDZPJ$~d?ar~w%1L40word z?v-yf@!Bx_$gM-W9+6ufbxM+kngHk6cOJ0@O{W0@q0!yHN^&Md?+R6nr>62l*1)1v zQ5!;@6jAi9S95=-HH*TA(= z{Tci=>IzM@aqg)cGUp_jTR{&1*> zjo?GX1GcAxz(@5b8@MR=p(z75v-ev=e$1G3%vY{qQ-`;q#O3xEBFZS&*PUun-=tY+ z1)oA8+Xmjo_2IvZss`CSN2S39iJ%1x%{)A0f4G*Ss@R9`_RmnlYS)}M-S`zHxEx%2)70g0UjFXJsWcF{{e**Kw!#bShs-|@Z#wRL@c=#X1~%36 znkQuq%!(TyIZ9+K+x-l|8w;1^L^V%Iuy=C>^4Nq}IUJ1%#PBlk;D!t_f5HqvTS$;6+ zmM~JF?&+O&UCp=fj(dVczpB&`EhV9fv6*`n)kOjnrt9k^YNG)5B<7mVi7!$t!NNWB zJJD~UKphb=hIlt9sC@sfhvE4cQH-@=P|#h43Ne4L+qX$6i|Usfx_4x4Z+l#IRSSa+ zr{CN8B!<^4UGfq$*k2!Mxv^yJSNkeJNyRh48a^n`Nk8Me4V&tc)?JU}m!LI|mLv~4 zE{|C19q&#>&@;uz)dBwqjmp~uIY>?qDOA@uUr-iE(vuG=h$bYRj5f+oMb4?|BX!#s zmK0dLtSPg|fHT?$5}`|Qd6k$yeXrI!5Ba^ClxjfataHlVD&Q6@JRO~mq|o`*VFw$v z0qScEscVR`)#AR5)OhiOH0E#D#AlWlXSR2S`gglb7SG;Xx+&lwXo9k6b#IzHH^zkJ zgL>cSAE`R7GA^G^8mSBz5UR?A3&-=S;^WY7zZBZFG{3GmUvZw@6(6aQJ~N<cB;_%1eG~y`47G!`FI^#-Cno-Ct_; zi%yDe>i#tN;s~41r#u7TeWcDoQe%qdVgVsK|Bt7$@N4o7+rG4jf{KWMgd#&l7}B|c zs7MY)HX4+cl#Xqbh^T~&lo*mKJ-SPQjqZ|W!U%yeVq?I&-}^l8`v>gvxp!apRpfuE>rm&Fb&az^W*J4#<>pur*TTghymPr?A8xU+ zzqC?T7~{&Q_eTgym=FOPc41=sM%=!YIuF$1YBnI_xbA8XfCUxs*M*VEgC5kpWHx9M zU&U91z9T%-`kR%)7(ZqNzTF{l*A!LPxFFq-z8X=X{Q0Rxm16K(I4Ovcju0(>;v5cJ zTz@qIsXOhIzkZ(>W~uB(AHa6&lQK2HhLvvs*GqsK8tXg%xq(K(Thv=z_oZUiG{Roh#;DqItQE2{{Us-W>J!Xd;&NK- zx->k<%6?moa*6^c-C<9rOBzq1!LO`Ny8N;BC(l}Hj5Rl^Rp6Ay8qH1v%{RFf*hkt{ zCVyx<1Cvz|+Um4{>bD{SS@wmo=#DWnGZoc#(Tj{A`fMy&!-CN)3OIIaR)JD*ktRbFlzVvyjLqQ@=@s%@I zCBCjrKJ)WOl~Ax9L5!tgN&X9?RRey-oAffu&wSyPh2W#C?Ft>ZgnNxj6Zy^F%JeOb znJ@3xjx;5OvzB#eF0vIXHK3K>4DnMo!}Cw#dd zr^B$R=N4k~l!SgOTcv>?$d$4uJGrHSHm}q}BG1GtC!r^!vxc)I%@*0}aNo||$z4qi z<~kaz_Z$r(OW|=FCirt;G2lHjBwud2Fz-+x-Do%xA2`9Evane5*wB_(Slew&2($loYm zQIus1LuF^J%ezwKqQp@P$OP zC(Mb)+`_-lZ)AiN>1pajIDD9(10w@uFRQi;E^-L^u54CbXon7r4132#^eaA+OWj#+ zaNrJVR}L;vRid=XNd?Yovj@s-X}23i0v^%9e9wO#+MZx%9GoVSv{cl)j2YQ|&pQjM z09;S8$wSF4uz8Q6^Y2OG4aPX#Q+g$JVSeAr3!O_`H)OuB{oVgfRVyK`f6a7ZIK0b% zW@SE+pkjvw@ADtG4b0OE%YCcTo-8?A+EvyVU_TvUo`RWW ziFOz^sr!s@KQLZ0nVq|VS$h8X>`1Mfe1vu9L#N;IXf3oYTQ){J!AYSiznS?4dbi*s87)oYg4UWvvP;Hsh z8yb@<_1DQ!jT!QE zSZn6z{?$DB;9TXiR3WAp^n9Oq4jjR*K>ui~kOEi2+m-fQ32O|h0koA8DuwtRbJYT1 z!dcR69crUF7uHFy(=?2-U=uSGtyXj>DY4u}ne5m_!YtJm;6<~3blH1jwq=G3)~oYr z7wGN>tc%oIot2yh`76Xy;$OW=6lCOb$?9USm^C-{4@y*~tLoRo zUB(Kir`WZ^yF;Uy*LfE+tJpJ+uaicWKR3%&+UUMN*b_BW@bMHl4i}^@xGvf~D3ct@ zr+f~9O&@0P|9&WOq?-K@T8`dYpN;WWsz$36ZgJ0Y4CbDOX?`xLZk#6OZ87M*w_M!k zK&XdA4Z6{EgDf}97Y`bLvOY;|v$!mpm8hu^1OANVGuOLIpRuNQ0g$L#qZwmyiQz3L zb1ebj;~AJRWAaPP|JtA5qJt7;Zj$nJZU!0Y*88-CP^s1Mu@N@|_1xgo!qTjVX50Vi zIcizGCq;x3tEDsN3`H|JHEUSEUaZGnek_qu$*S*xJ|V>UdABk43j#o#_Z8`PU7kQ< z4|E3VI45Ex!2TJ|FRtGm+P-qr(V6a=B5n&%T0E6JS1e6F4agjLkfgh|dAmbV7qc(o z+M!QbZ?P@c%hF}$qu$bLVnP!v=bt$U8Y`K0G3~JV*YrMQYFJu-Hh}yHQ9X*dgc~AA z+XBLisd?;265G!@asz+qZ@!L!llgMGm8mkjSghw?an_`UO09tMz8AJh#+ z=Sb=&6=cP_$$WMfdK0t!#C93)E;V)wjca)JD4^MEg-#$^iER^B!^Fu0&x<7lZ#8Ex zJ2y_hAq(iZr+pjoqh#!cW}Nk}Y{b%!OZdMk)Z}BGoAJK9!)ulpz&Y$N>GE=LH6?tW z*=%UC{JP7;AS;_QAbJNzG^uLodB3J)5L8?YR(z&*>ov{BvnQ{Pq0uzJF_L(hVs?;F9j8U8-=^$DGPXIQWGBrCx;ocq6EJkP2F zcQgWT9zP!s?-GvVUIPf5z0n{gYQdU4wZ(MY6b#b*q56dNoLRCgIk7Z-RSPu{4(8JV z$M#106M1IFHUu(!cd8dGdHy&#dso>6>i2{$6FTqT*+J}k*R|P0_P7)_X>+Dpt~WS zPF)vo!swgZVIQh!EjXO%DuFR^kpVWLRYBHU)&VJEKLXOSSvjujx~F=p3<3Oh(=1PS#`37R`%U%P zI?&6m-k+2SuzYWP9)39)v^Nww)l4h);APCMK)EL{(1?~K^N z_HchnmJk%=>sS=$4<-tPV?WyOw-y({<1m^0a3VxIPo zS!h0*=Xqc98S|}$@k-VUr4xxmVX6)tV7VOti^|sb&4lFsouLeWJgyyWVA43`1|lD| zqg5Qo|E(<{F4{w=u|soWpMra?*Eq@G!+UCcVFHi+N~x8{wc(a>-sa{97Oev@iTuIvM5e@`7091)!U|CEOhZx$lh*@0`B4KW;lsxo zsJOsP8O13sU0?F*7d{9EHvuKCz>w=dGpjTt11}HYger)7qQ|*?`M8lfOb#m)3YsGl zINrA`aO~Wnc4rgc)D+Irwz7QMbYPPie!ioz&o=!47CQ9>7n?vA|7t!lpzUKZy2NLX@ugF_N__63OE3urm?D%x%Qw;E7#Oh7bE ze2qrg{7qS&xG6o+`?gErb9l45;3U;|z=f-`~*9E}pktsQNi zz9G#0QObOe`96oE>@!-2QdF`nM3Z(138L;H7&7>U;#$z!7k#(OB@~F+0zOAsN0Spx{gnu?sn*0S1jRqxk1$*0(Q{4i zw{EP6DHM0ih8(+F*-f$17T(l2Nii~wjN`wL|1k!zIG9;&E-7XG??JZL%I2>Jx zz0{9M?d0xDILURA!kXAkbVBZL%LN8XOx*-^XS$?GiRznW4IE4}pNO2@*NLRL4(K`@ zCAzPh_AJfr_$YpBS+8k6ZvDs5!0{PmttNYJKs_bgzPYWc4c06(jawCTU*h;nBI-Hf zNA7KFotlF1kNNZ@c_kG&xh*b7leEpq@5X!*`Sk+(SB&Id!3)9pnOea~`cL7mV#0z*@iaQFIYO7W2t%XoYVJK%CpO!~eD+ELE0`R=f47#{TmP&er}}H) z#DO+c#W?`KOW(yt-!7;O$$z!9)UzFpP@82k8ynb1Me}1;g@KS!Xje7 zlCO+umrDRGYSteQGp7!jhaL4h@^`I-F~R$v`hG^4<&XQKJz>h-85o`LsWkWOOWOv) z4imm}Wu6Gw2|o}|HDBH64<4@gZFg;Yrrdbb?;ArZL94mKC_{QPjE$Q|A-ZGRYdS^G z8tz>y`^hzi?$^eewB|=ei2Ny4ik#b{%#!kNE#Jzap6F~%N&nf_9PrrO*Id|-r~sTU z|3J(9y49cWn%=|h?ZmPwt=y4@e#3#u9a8uh1Fq6L2#sl3b_T)z-BrT0ZXY*qE5ij; z1BSij$J-R8O$7gjJHdQjg%?O2O|jeBJE-tAvjcW_<11!@Y7YM~)X;o7Sz^-d7GzSD z3x=i|JeY@XPKC^5Sy+;9w$f%B?*qZJ2Raa`)es4O+J+HNYbl?Pb)U{FB1#8v!jvc^ zegCJ@J$Vx>kk~!zj4i_EfNe1;7AGH?(*uy^lH3B$} zS$fwqE2r%!O^ossvme6#1u~6_oaIPVcop$laYD)E%31pC49(=q3OEq_?Kl>R~a?EOdHt0*?9x_hhcg$8Qfduo9jn5-~*eky1xEf6ZR6$S{(dgX8sOq z*^6EQfn!%(Pxb5r5Rkeow4gcVwjhW?3Z=cJfW$~NT=`_yX7>Plfc?i%iULWp{07J> z)EwO?n!PD2dd&QaKi8YV6{806uUuR=9?ITLC@A!<w_&f7Qd7rXOwV+NPuW}Zd)!-OGG#sZ1yxPpn)!o8Xych70Xt!-bG)uQ1v z!9uo~c0G^+NBSq*3;siA?kZ4ei$-Du5WaX`;rNn@o(0rA=-W>FtFb?9YTU^0g=T$~ zD6#K^>;7yY1mcTBLG#u}$2JU`?gF3mp54|r1mmDY(|LIWcq93&NrCv=+#+);l*LsE zR)>vpTc0GHWt=vg)SuMTe1g{*I6nJaZXQSmScw@xH!kk99DQo>mmZq>Yi-vdZtlic z@#6J9_Sd4l9C~G3{cv_0dFXdarlp=7{rWhe{ek_pkRM$Lrevgexi)*k!E?gF)O9zd zGJaMKIR7{tHn(5zB7N_!hI53b+7(rmpx{+M!IY6m;8-#kd+0R& zF?Zp45OL%`C&2m9TW6+KvNrOjDtbjo9Id`(mD2?ypK>*t z2E57Qs;rGM6ng!_uPHCITHT_+v#PB1*ByYXnrK(Cb%}(m_3@vXkmzJ~RFzO~rt(CW zY3MCry^uL&fiaz4inLRrEm9a&uV&0Y`T91uib*-=%0SfX{Iqw39$$1W)4h@#fQ?U9 z)TK7Td68%DH4SbbT}xnXW(-UGZ6=Ve;52joXtuyBNS)1%6fmXlDjc4R2|8@<)% z%~hWG_&BmECsB;#cBox*^4(*7BjUHVLERKXPM>e6kbND;4h$L!ap?{wW=>np zHVO?cO4nq4cTQuQZuE~+kA$pjhS`K?z3>q8f5gyrw&~lD)F|{)EeC@6(oB}>6sT|A z{3X9fxsKjT4C(IS527^w^Xro!4jrUTykChEoqPe_vbQQc4KLW@NuIuNwhY3a8#A#e z)Y*A!XOW$zbhPoG_VXctu!pP6LAJuiX6~2B?KXS_2Tk6SjErOh?+1nAfhSJ`qyW*R z6)x&V1?dgPay++SXh5al$(m-Yr(Fqd+?V3hCReXh51my$J38?`eslbWhIl*{^fzFo z`)r-nMx-^COJlXv42(2d{?m^z+lG^Gt})RuAA*2qLo|1@hd0v9XyR5aYF(A0&b$xou*PC;c> zKTC!|RTtpQ*W)TbYJyu#__KvGK&T5ZM@7x?cv8v(E!B-r=J5YHmK&UYi6dIYuUn&KJ5v#<5e>R+Uy1@^|F*DkpO7U7T zuFRAW#Cnw>1w}^;paL6r+81>wMXkg4*3?#qI2b?T2ogf-1=J3~)19_bknMUGCG?;? z`Yz8}@gp{05?fm#Sgjmr!D(-Wr++CmgVK)VzAv`r2;Vm-LDLZb^>B(8X6?y{b_F+g zKwIJtI&ZGLS7JC0q<(Sx2oXjycAo!v}u zZ&39M`Umpx5%mA7?C#&e7ZcU%@Q-;ZjP4BdUi$lmFx_-$n!+EoxrKGFsg;% zImspK)h{S0PTjazYRSX{*Ja@Won>>8oZac{RjTJ&S&4(+3*}(o#?9w$g&LH8s(Gqh zCoDwF*oeDut|~|M`!s(0>~uSJi__)Hb!xnD?dZRivT*l}5l~*v;mpv)w4b&dFMR3+ zf7>t)Gyf(}Zh!cE3-gn(`SpP$ih^;7*tvZS%G}k?ZdP!;JruGr)LEPk5>F*F+qNi? z=}Ad7jrcM?OOHYXT#r7Z+In+X{(?WMLjuS&{h_pDVj^)vu-SW(_a(WA zseE&MWam<-Z;{3PZj9f}0IT;d^W_m*JQSyn1hCIGX=I~xCV3{gf#VM?P@uOwdEnQ~ zsxCu8YyW~W%%7>Tw2_GGd#Ej2%}V~y`7oCC&p8*Si9744G_8*N5~ktx*x%N(0`-6; zX2-$DF2jy-kl3in5ck`!lCA#;*ofH-+b*&&X>Q8Ds^Lw&Ie6#x-Dkk`z!8F1=}C08 zX4oN@)Bj@u6y~Bpg(22-j36KTLZNfBZ?;DZ{t;a`NqezIZ`<;@ZUWEI3pt6Y>RDnR zfbB}?SWStO(i?fM#lHGC6oDo|Z0Q1eas*sp;Dr&w=}fP?FA+-GhL z|NeK-b3uwR!?LV$B5j_A+66*Ok8mv5GEZ$m-XPnmQn1jsE1!p5xn%ed_4W-wPWl^d z-+?@oy6Q0)Q~XxN14lj4lm|-1rv!Ow|Mlb0sjQac+hLY!#qK$VnpDSQ=_HjFx=)CE_Mt+i zEw@g>n{DHqe2mGdb0o0D9brZ4^o8w_cr<_KIS@%%+IOudk+3}7o72Bu2zD|v^S}V_ z-sRru6nxnasn&4Z+Y#$cUPL)p&H68ig-H3ES(EY{_n*7k;;m-x?!B-IkQ*#tkFxKG zY(Z^(qD9QJSZX3#&pA1VM78c3%Qh!UhnKdN(vklY@u^RMQ~3-M&7!5$&o6B{kqRV2 zL$pji9+hxiFSOdJ2B)YXzPdfq6KJw+l~-RbRpD!MDZChq$N#&}@ksy{||t?K#4 zi?vFZ7jTvuiE%nm9T%At!pzW~V$M%~Xu+%(n0}*rrO{a(IeLiOG1n_iOOcyt2M}H- z&w|bMXiX1$J9d6}!G}tEhZ$C@N(b%??yCp>ix>xJfR9Lr&beS#r2_G(lv5WB{f6JG zvpZmu{Z8Pybx0z4puj;2@o*5m6%Dbh?~dS92_Sqo555fF7rjS9N<(oKB7Arq(d3&5 zGfAt<#V*VN)O%~LY1)m}zTcjMTg?1{cPXD-?+~Yy14^|}IqdZEbgPwnya6cDGyf|j z<+2tFMfiSh?AbmCunMCyISh*exbAC3AFiCYP#qiK4x@s{95ujr}$oT z{T)GFQ8O3k*EPv2KUSTp`V#6!N^@GR8&$S_=eJIIX~$z`1NAg(BWv5lUfU_g!E?@% z@>kN1+VSTZYR(N>10^pTr%3)yqHp`=iTX%qDP()~YEduU*BJWB6_~&~;3W);%(o%u zY_%MT^+kv>eF=1T__=CAo>)q+M)K%-y_QC*2sQ3<1&MesIoGS?S)=FeniAd zRY!CHClWl({_06m@R(7C1A@_<-if-a?Hmr(Klw6PU`}_NrkYi+cqBi|A zmFR-!Zp6VamA&Zq1od#O%fF1#0XQs#xq(Rv;8E=?*hXiAhg6pzp4c>qDPskOn;L7?Xop3m>9I zfkzy?GZeVg9k~Y|Td_}8p@F%O5C?=!m9U~effo0eXa{M8`}{4n4RI+B#x5gxAqPxY z-P2#QPhdnhR0f2aH0m>~-$Ii}C6AQYzDqk8@RpjCbq!Cpazb$zpRBo59^P;zwMoy! zmoJVGYGz_*VrfT+4$z+QKkBka``0W1HifNLnz)!U~EV{dpx~Kn_o?!Whx+ z%(nU`6Wxl-iYOj-(#zzXm6xly*3_qlpy{IlTd1%7a_Z*Vk`6b#HZ@ZU08-?$1k1V4 zyUuTccD@oAN=rUq3);BS!wa9H$!oq2{#6{MyImw_!h$UE)QVUpZMGpzgyaO@smXfO zbRq7iO&}(Rs?B{OE2$i2YaE?HCW)ohi}pR}>Bp(p%F+U9Ql(YxTDv|KV2*k;ZI~J{ z9z;SEGA|Tuhbu+IkKn4Qm3Y%S6p&xt&QR0IBY$8mhYmQUL>@Leiv}ML*WgU;x=U;Q z3yP>W+&VUn{~pX9%+g9qb)cPDPMc`BT(_KuX2s&H(mSofbK$_%J8~oQ8_3L)+YRA< z<*Gbh7@!3Y*|3FZ;>&&Fk77#dXU12Y0Fdh$2YRW)(mN{4U3{Pug88nYC@{iC5IW&jS6@24xz{$lFX!E@w=1)`n2&n~8h?1xqzSv^G zA9a@Nf&10)7yWWh+X@keE8qYikQ&N_p#+va9nW|=^vMvK{07&ZX#|xD$D#^s{#IyC z{pxC19HBiLum|g;Zc-+3A%A-uBa8mLkZ>uv zaZKOU=P4g8YIVDCCUPe7|4tA|QH`cLCV#pqs@nPYHlmn-xU?bEW&ZJbRqIsa$ z;v-=7TTmj+W3EuzqUuM5hcg7ZDCi)fe7z^2wZF>3u19$<Fq4zgCGiyxpw!W;~k9bhJl-bwlp{Ckjo78bl3kWzFVLG4^qk zlq=)A+km~0&hSs*G1@I>uV$vAC@L3t?q}2@s0*8wUb0s@kj30?E1Q>x=PK zU-r{t=JiGXXDPDl0ApKaXe-$ZXxbUdn>}3(K3(0ZtB6A}Ed8*8P<4Z*Zs~*|uBR7F zh)7J2D@4zedx-qvtTpJZ@c%tDq;9K>tPDSP7K*3{v>UAPeEaviqBT0v{m=4AtFAHC zC0;6OPQGP4xUy!Zk1>M@sfY91Vl!khVLB&)xx6#HtYi;*uz)bFuTae(BE?7-gb3+1 z^ro+T+tibv8`CA#1V%xGgoQj`y>Co(3QaO=9Tq9WchSU1gf)(J;Lead@ploL)Qb{c z;F4smG~d&ifZoJEN?nN+Y1&`@s`3d*Pon;uBTzrJ?Iff8}cC77yDO z3(uXIM(cH1os3!WD@-UZU-7Td$@(<#v9$W&c6+G=UXA_cvj6F*-9#+3G5)JaQ=iYZd4lE2VSZu8$Lr73 z2C#=UAomNcRG|+~Pv*}~j?CY1^{B6T-Ih7mfEoXoJn+Fyjq}`!e@*$9Rg|+ry)FR6 zXZosq`n6G~=p&g+l)M5^BZ#wMu@Dl7E8eLcuMS+0O+BWr+CMatA8p_(#n|aTFysJH zOAbGe7aO^qDIKe16mx1HKi;Zn9O$$2j-ysouD_e{W1e+i4#|ut1^3)D;)ZSTY8Dx6F?dZ6qh(P{sAl zQ|hPKA9#Wj1dZQk@mZ@kMt@H|Jc4Orf*7zfh2Dbh`o!plKt2b7WVB6io)`ur;BufS+;Hw%hOAb2HzG>nZltsv7&rj$`&-l!J83~BS*x7O{BzzS;09N%!GH94 z%e&bew1I6TeQ`G@d1H0FqeqY(otB0DF0%S4hlmMoh6#g1GlJYBsrU{FpB?4hAj+l# zIB2xuAhv|!dbIw~>DcDj<}fsf##>A>UIr}v7mU&MGv-`(hU;=&J;gUQLU)&n2&7gkwlx(0UuJ1U$`S(W*<`XvQrW7M$nUwEoTuKbhQL!1IV!2m(WRFEc+jd^a)`7z`xXV(^)BCBitJz!I z^CXMa`*$Cb;mdsUwW)DS8TE@l3rNcIBmIgX z#zn)Qj8AUai=z2DPln(9{t7kFv^#V5tOP>ZV&NiP=+3^iK@*Jl#e!sD*Am(Ok;M0x z9VkchJ{@Cw&kb68nGZ}Xwnn_ocp2~Dit|kqUxJkyQ}FU_-!%;uIV+vLs#?@I^;|*6 zz++m9WHx6)-8zhxX*>23f7Dd#AEhp;2D9{N*F1>C^voSOaW^!bYyYxUbIb9{sQ>e8 z=4Kf$1*`~Mn7oi=S0vifqw%5%tLC;{v@6_NcA8CskHmnt_1~@J-Kvqeas9^PmEHkl z?Rs#$iXz#fwQz1^0&;uf^FgnE;bt}8=$Q+4_jphanM|+eGp0|iogVpgZfZ@{8)9MU z(Y=OupT2G(X&GP>HJ^6cb9f-9*%ecAT0Ojp?{}Z^{Z4P`8Vyl4ku>0LQJt`GU?5M| zC*SbS92upcD7~9uLP&$B$g+N# zx|mVr$n1-(`i1j}InffQ)^hiO-);B=#5$?=_!o2Gma4^a#f49>k43knzlUfxoX=l8 zdYZK%k~@$)T@E(*-IHkv9~{fvIf%xDW^6w1bf++rD@vT}mHgkn(f}c89(8_|4#(zC?(KZ^zc6y@J+ZoXdL)OC+qX&^7i=6<(W}GatiqI#T*ieJo?*wH^9~ZRf;If!xh+FtX!-%K^~CfkhZ?uiIDk)Hw*G6rVm}u z$qcobh<_+~UlVU1(TU3noyL7pv(X=QWMT#Qqdpo{+luMr3;p_)<}$e8a@nB(%XsZt z+;*sQ1bs1frgh%3VTF&zq6d=WU=IO@veO1QKMmJwTa--SNCa^3lnPE26=tj>k{u%PO!fSH*&UpTHW zl6rggqr&mgIL;BIJztboX&qMr-4vVPK9XpO?<%0$Gd(byM9q#pw4lEj`jkvbHF^-p z$V0Cn_I1dZkl*9WD+B#n+`8VeSt$kB!hg1unQD9_cVUt#I!p^sk;(`&4LIFhTHXy> ztEg#yCfIP)0{+JOh#VRXcT0@9310yn{_&FcJh2bFb(J7CLczyF=N4l!eN43&_k^jBG& zlYLy4{>N|my77FQdN%zW%HEA?ynqtE(~|iy?j6a%h4yY&vy9M-P}maLW{BAAFC+= z36TTrQfVpe*cvIvJlfp=Ozi;Q1nmY`O#ezMw*8ytf!%j98hii&DWS?kxFsx4D8Rs zXD78mO=AxcSzs{1aHTBbj4&Dlo!yyaF}>^rylVp0W;(I`&e0NNLitZrrABvMk*UB^ zf3nzpsf$H)vZzJ>IAMSVLbv^=;}ZZB^n)n{8Cw;FbT?x{ZrxXS34rs@}{5zpco=c&jvV&DK`*S*-X| z>UB^(+3b1JnY~havMeIoRl@dY+WvEuNa%w~id;jqp(EVxxn@fhX+6y{Yx1vOjypLl z8DVQ@R&A>(B?sSkclrOD_(c!UJRf865;myF<%S0n^CIJAW_`{TrCrnL7k|BqvO0bN zcchMZ9WQ0kLkeFWD4b5w1dh+`MRMApEb;k%C8J~Ilj=#-n{f`Nso;~iHuk{~bQ4zU z?EskXi-k$$N_#lr{3_>bjGo*2G!CFxp^xq3SUJAtUdVLUvBLB2`EuMSxC8slN-Dlb zA~X_P!lhVA6>oin-kOd8#;b!(3(eOOv8dThXY}#n&Uu;uZx><}!BCvgOvH(GT}N%c zs|gfe5c7{vMyb#92s9}L-1>gfWc{6co`xp>(CKn)W2Nl)zkTkuj{$GoJ8aLHuV|2q zM9U>W_4Qrtb;1UIQGw^O*-DcYYO593_BJa|9tS{9>59)E&%Oj)9Cf>bs)yC`-Qph#$#Yv8{tg+-ib#Guk-FUGlM6PPR1VQ4U6=u8YDe-O1 zV_^Kg&ej_KiR6MvxZXi+{3}RRGm~hXPyBl|3n7j7z7Me#p{|sIKuqBl--!I?7nVXR zx%Ycv8dhx*@X(s9Znh~ptPJi z;w@TDX6B!;@Ps`&9RVQunr#D233lNX3FJSYF3cYxf}-#n)YZaBpb*r9>C0y2v}tgb zPLOHDK9^{NgX$oF!xB`_y3f&^8Ek4;!fv@#nFzd3VOOF9nw;MK(Fd=3h30l_;X|AU ze?6U)3m!mLSh~1XI8E1#aIC)jpN`T$JwgA=&nU~u>MvSt7iy+o?EHMlbP=k2v4rWn zEJl6UMp91e#^9w<+x>g9tJXv%%ZsHD8ec+a=V0L-90(h79L&(8KsCB;x*l+Qr`(tv zwapO#!2n+RDtU=oLD@NGAzBbKj)b&IED0lItd^Y41s!ngzW(xjmP4vH$5$tY#`^P{ zg#Gh#j3Mip`YP+l>aWGUcii@k(ol;KrD<#~gqfGf$heYK7<}Hh05QD6n$Rspc8H`! zoaaDnOGj?+WJO5?5F~)L|2UL3Iddz*FIHMAbsuPb&7i+YVf4gac431ieA`V76^B+7 zxI6H=Z}AA^j`-4nYO0v}bM#0|C*i7Lu~VzqJx>%z<6YL~#R3k-P1`I($g!kyei6kM zLKaeFYP5Wm|0nf2X=p%jDklTwY&5Gw55B+Z*Y*NWXb`(5gb9V8=Jrg)G~lAGB)cNV z3>%)!bm0W6%z!H{2JXk!gj|(W{c|Ex{%26%SwOt2PT**3YAjE#3x`7~H5{&7Z;hy29Ri%1B zJA;nl#RK+wMMT-3ZKYI!^MaHIXMdQ`*NUmtq{`UOBu0k-;+q~LH*)jO(SyPVh{A){ zd!x7XnnB|N|42{8fO;`NuBK5DhcRwshbZ51O-pslf6LT5!=-l{=U{Vc#(}Q2{xE7k z*kxf%b0j%2)jzkV6g9AXByCygC55u;}EZ5?q>@hre7Jd*~nCD4l7-fhVx(Bi02F_%T@(9{+`oCpo%-eS3bD*uHTh zblLSwnUee1Oj3Aael_+zuFV@zikX>K%54v?VFZX8I|1iy{d zhGa^ol3j*Hyz;;~QF1p15*n5U+sE=9@J*w4{2q~13-zQ0`&OF7Y3&M@iYviwGYLkB zML;o}?l8ASM=7m4g(G?nY&21{T**41T}Zv!_YqA^6dAx?sD3x)=eAv( z_!l=`FRx#Eqy@AXTZrcWe=GoUtcr-$^nzBujg=pF2fKo4C4ALcmgb;y6V{iI%(~%X zH{6o8JDZx`rg?mI?Ae?78uK2l11<^1ekDT5$t&r*-|T4KU5<-@*hV7#JD2mFX}$i- zX?O?1O3CRn?ZyRcmqPh6U4JPYKZTGucmy4hjEwq246LCsKm02sq&KHJnZjcH3JAZUbdc6 z$yCBukPg&?__QWTB+@b#E+dvsTuQ5z4)j)aT;)9ZwM8mKRGyTd(CFy@%R9=}%C5a> z)x6{zdw)J-A=GLjssMhPhI&)$n4tPY5W90{`R!58B@jEuS0jjePl90vG!4U<(0@6>1KFu!n)30Qwne{?Kc-3ZgBgVV;AgvxcWhgK= z&1FH*p`&ra#hwnxg6%Qe(_eeP-R5VTODkx;3aV)4A;q?0PP)|_cUP^CmAoc7IqvH{ zV0woSr*;j%zqX)%bs6e!RV)eAME4&pv=8`l$RFzv)&`#?55(-?fo)irQq%4`mOZP> zmnb?a*jh%IVytj%2^O{!DsP68-dk+V+#`jOV_y~!69)o2mG?Uqqd*@@=jF|h*2^AAEiPd*kJRuq0iZVuplCf{ zTChVKy{|ILCxglVvko@Cr-Z0iwbiy6F(~(A1FGFVNnO#e*J@Gj*vk0EninNLPmAs? zH;}-3R3>?uinT;51j_}rX5Qn^w~Uela_0{pG{N|0caF^x9>&%;QA-!rAf`K-mrNu zk{{9u^51_eQnMEmd@>PyVstQ2A180uhskJ0G|rz+olYSZg6cU`WJ*Fgy%*n|vlo!E zBHw)Jw+tf9rvjl?vTqe(iNP82>%ZH{8Gu^BqfkKwKsF#lIyKqD|4q(<$TSz)f-^Yn z;{J=6fiFg3=T}$0erDTQawjHmXhND(l_p5qhM70psu=e3_;>NW&*=x0zD-Mqn7neW2KBM4Z~1B&-m!hsUlz#X*k-#(64@}>H38=Mv{vy zdgE)sr3qYgY!NV++!N_?@;l;ce}+%>jtX+megOVcRR#2F9%D);-OC%Sq9B*9j;eYeT>XV*N=gItfa;{neK~;;RPXy}d4^}W* zfqyH!I%c{Kx6oVYS(bm#1ydO8!UfiZyg1|9WO@AwL}$|uc}Kamai08553FJjEJ;OX zaok&`LM3%5Gx0a~XM6q2yPAeg*?I$CTi&_nb^WGylNU|~6>0-al5hMfd{8I$ay%T= zkej2YU#kV=7f825(K6Q4G8a~Sg1#zw`>x@X@g;Kh5E+L9T>&THh1p8L+I5$dZs!&) zh$Y!tDSZ8<(${sEMZ2F<#KJcn)g4oHe0WJG6d?JwgPTX=Wut{@A`hfzu)8c;2&_0&--(4 zul0NzW1T}G3|`#e%h6LN8q03KTl9>m(F6U`QG7v?_|2VDJ-9v5|Hzx(6BRFIY`Y-{ z2VHCTgThscA>?B>Ap)3YB)C>_sa<(q{+zdG8i-#mr-j|wJ069n+N3IcrTR{elzH&W zX?@OvysjyAxLk-<6s)hwxH@yPUoWJn|MQ}82K{KSBi{U4_jj_v3Bwt!ce-uAMLs`T z9;R1h39c9WY5vtjo3R-){97b`tZn8WzE7vszNmLSfd)ivh39%{`Fc>RH$?d0`WRn# zZUFm>A(mkunEqn&#EI%phEoZAqv}GdU_05eBV8u*!$cssi zl4RO@WvmT^+bJZP1~+$x;Rw<9TkgP*28Kk>9(B>mocLK%v7*3|SNmD{W?|p9!kwPd z!{GZE`d_YJ>^Q zyY4YHY82|+kFE8L2ql*EIR7==5_UR>pdGe1LV+@K>~`n+8LZ8A*kld7t<9b3ApfM- zbJeFbODWVU%5{&eoPHEPCufzQ3EC)qiheiOd7>Ouu1W~~$nC`1%EsBx9@$xKF~%mS z4xZ<&g~cMZAnTXj^8Ge=?b`q*{{t3|2%5_d8-?3f1D+p3)-0yKiCE|9%7xMt=^FKd zLGT^+D@>CHS8&h3Gxilh=qq7>s&=k+8Y=KVzCwZenFxkk+*f#MJ@1w`L!7I%>6j;Z z9iWLXA0AH|K>w4{2GEKhb+Yeg3ymt8!fXr)tb4lSj=@L=cvC!Nwq$|e{^w~@t05xV z(a)sbem-6Za?m&%N)iD*uP`1}9M-k2jlcZ(SbuisR{_-R6A(tjATy;4osv6RL<0Bo zTR9^{1$O^9b0nv@>Q&b^h5ssXQ8;@kHRKG^My;R7%}=`BkkaVFkNww#m?hoPQfX`~ zwKy-n`hDFfz17%DeZ%rN1^U>UJrJ{9y}F}%`Nw>n=V$kc6p)*>T@>J&Z2wsTMO#L_o#Ssu*^Ur0h`~dE!JLshWPoA^7Z{Io3rox2cs5W)wsQ&Ay!X2SI z{WfCs#w7gunK~btJLYH8=JbzDGm^1i>{+!9+D#Rg%b(L~)JR>+@S5VR^ZEp`#=%dl z0Ar(FnwwuyEN4_R-8U2YB^qLs4Xppx7k6OVU}7Lq?YsRh4{bkKf-K}iZ zodex;QPn)caRdWx@(Sb047ru#!$KzcW3M4DOwIchl9uGR#~#6Vw~Tz9#-~eN_9A0J zA)2tN55COzotn;W*JHAv-eiNvIw1A6M>D^5Eird)*Ieignc2;4>qE+K)7zfqSsdXS zX;jFn#(k-vh68>QG>t7qS)^_J>|>x`j8@xh7>EdkOSQL#cvhKUggH40=#j*>=&b2E z-(w!4A|FW5=Ap$05s9o;w@f8xS+>CPc6Jm>nEY{oy$nDfyyxz}rfP~Rmo!Z`?b_q_ zZql#rj@=NQMo1YYdoIbJvcT`RKk7F0t{(gRON{D!{7bwgm{p$XYrpSGBdn*a7tnh1@T#LLZ$yt#I9t7h0YKvk zvdg>qoJa9~<*1G1j;JyjU1d@~R<<{sGXRr5OTM0q9JA{IJ47-@A!~JHuFd|Ao5A;T z(kmSD|4!8{FasJnQZcP7y5RRV)8#LLi~oicB1&xO)co!(LL=cgxB9I+Z%PhKMJTO1=!Oo?(ZRiikZzO_w9_{ua7B z*DrF<84f4QQdUvtlX^?)1VZtdRlA|YsF4hfGFi5E@tSuL`mj5yXY%mah^!+l8wxViG;u4E#r?^fe}4E)Pci+0M^&=TH}(m`QlLRW(KR z;)p&P2RicW;rW$KI&lVfx^tKw(deO{@vsw@fLAKptFwcPt=v?JNqk7plk0+ zXD?bZp{ODhLZ5bZO32Pi$AlbzEULSqjHlpgXYo8+xwO# zX2-o@OoO;|RofX_9Ifq5$bbJ09(yPxvx>`dt*&W!(ng(fyt@gX-#+#k*OZ=ZeH3jX zO#)*qGh}a1B)<0At-#%oj zr`>AV`up{MU__nNji5CYPoK(mN#2Pwrmq;%7egibVATV5ElXk(QL90;S33QHhz3za zCoB^rhrhL0Pa!^OdD5q2qf{3_*4vAyJcp09o0LAnUK271YO>;+Oy(ANZ>l;1rTk*6 zY}Y|)613J|3<&;NRWo`&1lMW_e>s;@$eSYP1AY(VxNA6TYDHqb&h@FfGJM}N;5Mq zfz9Q#QcKNdFGtG`{UE6fn1Ek&zkdGr?mLa&88?{JP2cs~>rfXG&1Pm)!;)_guyro5 z0>whwvQT>UfPET;;4sl>(5cQGF|j%~oC0*OunKfqe~;XH(TFcTVO$?zafX|24t5NvCIY(7C61hZ{$h1zm zU~^v7uH$b;ON^J6KVX}vNB(Bq4u*Wqx_)UW#ENf` z4(nvagea!Cy;qaf;#t8*(7K{@S-;(C>9GCDON8JruYF`SgFqD%v&d3XILu=~&&-h7 zO6w~KK%0p6>~6E7r#TtbZb28fM@{o8FRhO#$kjHtIy-{C)Yz5>54FrUJ z0zK((WP;yVM)YtVN0c*|g|@<9x+$+c0II-$F(rXlYIMRMmI&W#=?By%rr)I30$nHc zjILD<+JA+GKS%;;E-5y!1=dBVsZR(tHSDp-!kY5hON#)C^7t>Fcd~LJDe7pqjG#$z z@+_MaT)f6bDz6_fY)RJ_07*Ni_5HG%7Kbe*yKj~Wc%;o7HtF%=vL!!TK6^XNk9p)v z_}KbV86eo-;<&i2tFb0F^w0Jr;D$rY95)9)&NXhXa-SZ)*34X@e*Ecih_grwqr~i3 zr$rDA7*z{{8X_yN5Vl9IoMGNGX*HkZb`mYu~k4!J?r&Dx)) zJq*Ve;0KP}57X7`71abHBPa3V-4(u{=?q*L ztWImWX5pXQd|Cm}9Hn1bU&)z?`kD-UYo-h(o0D#}3!RmL>ZW~`b+1%IZER{8l@aO; zQX2_l>|9XT3%KIq1!1;mO7AzoM1#g9Q?Y6$kSPkY?F^n|%8NW33{e?1ERWRoVNI?y zw06^CK0YcwAw#H3M(&kU1)i*D*enRKm*^yn&1~hjFKrthk2t@hTq-}&HU&9)QS0pf zbcMEvM%Y*HglIWqB~?j#=F;w!Kfv0}`y(-`26uz}4VtOqoy825+XL+&yCF`TZX!rx z?*e^gYN*!NysPCWIjE87Tc^5%Bd{jH;~t(gu}^RK%fqe1deR34Cm)x_s&wzZ>tJ`c ztS>E5a1hc6>g?|rEDzlt1Z-aH>A*-Q<{H@C?F8g?Ij$6rd4^is>8VUk6jYFLj9npU zqN$L|5d zD*{s~Prx$_%c~p>sRlvyMT-uq4RR+=$u-Mum0op5LEKR!i;4;afk;@-2(GRd_TC#S zv>pOyWnEe9&0QUo0!_-suJiIJEG_0*s=Q8DYTV8JC=ms!xd>>HAp&%%h$sH&wP69Sv^z;J&q zT^gQ1gn_Ib9bg{i#-zE@tRAl}7)$^M5WKJGVA!}23#MS`9dY6%5P3UrHi}zuXAQ$i zp=G4WCG@zQUZZ<68Rv*436RU~4(M&#k6hM@XXxIw=0KQkv=j@{j@5q!E?)pwD}o6a z5L>$n&8X%)T>QnL(tAt%;3JOzWo@_)Td$HI@n7X9k7{`_zaDTR)5K=2{jyit9R2o2 z891NzmIa`D@QQtI^J*j_TQA68R58>3rk>e;L5{Q9s@YVEa(98znCN?rY2%e)V`bho zW1HahkI0b)UGCFiZq7I}mE%E2toT9PvcgoUL=dUCaX-U5^S4ASC=3>&?KX*MzyHaoan&^C?ENqK;iQexxjTby3L?-ii@}Sti5N2+m>x#vW^V z%}=IjLGh0wa&WV>X-F2z{-&j02cR4^GJ3*q*BHOSf?+C1rH#s9iS{}y!TksN)b{CH zVa1EH+Yre~C=NPHY!xN=gtMJi5|oljUl=2l-)lWx$9Qu z=dmg5w&vAhG+QF<*E)O5$>7$@^?3iT3}^3Z$1b>hfG*5heaHt8w6k34>_Y>tD=2Ne z2j;GIufnY1A9&;iZ-`u(Eo5f_8u9gtHV39u%BPAl#sQnWYQU*&M*R5y|HX~xo8mjW zBFu02U?2@1^-4=HvrCJ+wJ@uXhL6_Zb(A;OR_N6VkxH+G;>%%!yvM&Cy3Ir9#Em>q z2m_cxX5iA8Hxg`kEH;}Gpu1dnEo`|)A}rOc{zDtgeWa0i%X9P^)!wkd%Tb1sy5m~i z`MkRzmRTS0qqw_6aW@cVto*fNAVTnmuIh%ByRgU!-f+e|!7#&-s+3V<72K60TVSM3 zS%(Qwo%HYTw|QZ1K3 zwIg4zOC!$be+1C?aCW~ZHA6*TSe1hQ(YQhqV*ylyKCQCgb+*N70f znwhIxLz}Y!id?ZJc457bh{hfiFIet5%4LB2k{$8^zgo>JqOMU;(PB%nQSt7@!l)zP zG6k!JkgJgWf)TEvPhrUy1bZ!bIIc_=>UQu-2g@|z%PG*3o%?|MMjfKfx?(xmbH6F- zHyqQ%L@c{Je=z3!%asRm&>nSKd*|!ZK6hWdq0a6z1(pu4RWWEW_Uc>EOh!WG2a!xU zUT(#;SzPY6~?UqCGvOl=#1Ic_v%*Tc;!FdXYF#IP~ zOV1l2JGFS#%fEwuRmPj z_4}VaJI=DKDL(z`m#=n#9)mOoWQUpmA?(Y&XFS=}`IU?JYkrx7DYQvhqtL+`Nj{mT zdEm6TNT!?9OL2H^23LA05n;2^q2k}k`@@wooRr4s*rMz7?rl0t;(bg=zMt${uJmP%mFZ$E_`mUM^ zhB_9#_C%&|{R-GQqn@)PdHgS>93ucr@5>Nl>3hK4mu>2An%mr{8LIPGs`V9|){!9m zG-YYbsr)Nxar_`_J4PeMZ%K*IOg!Yp%P2r-|B}pzWk2C5XJF#4=PvinA0vR2fg*$B zW*QNvWj(3q4%gPwiojN2$8G5!51Bb2p>l#ra7q1sRjgj3n1!!vX3zGqjcKEJVnCZ& zV#$Bk#r{>>Tjv;(n3d@((GzosC-V~QeeE8aoPE`UuuAzWMnBxpG^-YYR;De)N(RYG zSVuufYv+E6iCm#^aW1!~H{ar_Kz(SgdAajhMqtxZ)FbuNjIc6A)Z($se7B#P`~-+7-SGiNwj@Q<5xO4*DU^suDAGw`}>htl)7K>E~LEgO1q}^UNlHSc$?7 zG)C*mPr^5tMBr_I?k$t*P_jpmzYMAAwopp@fkddk1}#+@W9`wU$9&~Dv)D%pR?@gq zrl!n4)5v<_v^DL~I8A$_EURMW=~SN04Nds>-YGWV8pCodHiEj`R_)5&+X=9 zRZhvvv-ooQ*Q2n$U<&VVgVFw;jsKekkRA9DJ>qCwe|O{x;Z0HL9EL5{#7IJ#;jTQ6 z|AwB*D)l8YLrI?v$fcy1)%!sweBNi5#=N(^ics%s%2u>-z-AbZC6{`_FAR`-zCj~X zG5lfsv~9o1av9b@%Joxy3tyV?9Lj*4xIt#bnJOzoTH)Y&=&~GK8S-t~?VhICoYx;Y zXhixxIfJ&?Ur?r-dbM7~IRr)vV@f1J)MzZ#^-)|JGxW{KEp^kn(>bn_Ij6*Rp&tJ# zh4L9{#$^?EhsN;B^7J4J5Es*pe>PTP-Vj?x>Q8$%ZOUDBCK1B-eC@=B@Bu?5ci6H} z!1b*A16)oalscL#BXQ5p4{N*_3?iiLu;P1$qt^*Bd^G73_P8#S#QxD>hS^rD?C(paUrw-F@2dtyJ1@93|eB-+|m6>lOSPB>3UIOcU@Jhb}t7Xc{HaN)sWFzo>0o3MZgT_GKgti z=Ofk?1+&k=5EWoJdyg$-=TDXiYCW?BleLcz>leAQ^KG_NR>1U4&A~s$M`H7}t$-(9 zy_H^2Mz>nqXBu;gTI_K4=7B2n$3?jePe5jntORGSDe6~G`btFPPH@DTY2T7uo6&K@ z=Jc|P*^>?0FTXk|e_(XN`aDgR4<|gQ<@5pDRO@-{*jQI4q|Vj*7)(kvou=Gxs^zc% zRk(*f_Jr&rqLDjKQxn!=1}mt0-m9-QefPB1oQ)jF>xWdP*}t#2D=V*zjeC@R*F<|P z7QIQ7WIspEJ30PN6V1n2%24VZefz132_XZeTWdqKFoJN#Ar3tL?>V3?0XZdI4y0O* z*`F4HLf5Y7TYi_Pqe6gHb&-54J0tWB`us^fJ^!;VI|iyu$?JnRgvviy z&UH8*=_P{gpTd!Oa3X$AnV&~sanh~K#}XFkFW^dpGx%n{v#q#fT)lz81cJzT|?Qy}$O}i1`iEBi39Nx)n@wz4Cd)uq-Xz1EOx<|{k zjVX3BG7T_2rS<>r&pj79Gk&a;fZ**MN1@_!&n@QfJm1GbINzHVL9X-MT=-Y0tbeuF zRk#7iMshNKH|ha;%9mTbv8&c}V1|&HJ&%AQQT1q?aAKbe(@ndo3BD#_#y)%sf$x#^ z@YvhI-xuPAeb=ipBh$eBJPh93lpd~?3+LGJ#l36nuPqxiwS-%!=P)z2lvG<~`$ES1 zD`+eiMsAJmWiq>GYEPlko1=ozLCXvjX68|%|5LN(y{Egh@UM_8`BGokSCvJB1nO(R z^hr(vQj>u*k*>mjK_bL`Am~gk!=78I2bxA83BL&ev6}|Y)2x-=bY=u~qIX?J85zH7 z#-Q)N9ZLkOBJ}(awsX6!?1E%WDIAq93ie^UD^D0^W?@9e#;Kg##C;evhi>XkoA~L<|`$>3A4rFLcU42!} zw1+ape%C>mzjW)jUS&{=`>XL`31Np4kO<~Ba2?hpNl~W+FEYg(3nIxM@9DI_Oe=81 zN~2~nzQ&&Ww8GsiZogx{<3BRdmF+UlD_2%DT|~PIr@A7d?+EAbDqpgVcb}_Ic922d zb$X!tcu|fo7(P>Q+4p&qEM}sB9~{K&tp+40@1Q4JUfb1+yUAu$rM_gm1LnRM0tv1y zy$4kpcU?eEGgM%rq`HQ2E(#fg>L*tQXY^|s1ia5kGKnEj8Af4VqdD=wmxwRhOhA}M z@cEgQQ2GzPzNY<-=X->XoJZ7uIu(tKD4d0RhW)LF{#AO0#`8(_Ry?doN zz2_+533W9cT}8UY>EwNz#w>TN@gpj|a4a^#<2J2-|7e1zcLLWg9esb&Dn^IDw~@?= zE}2W<3SKW50%1@*%dSuty?Y8)59DU)JMnTsf*>slz1APF~ea z!%3q>cOxDoP*eb?&I5B$Fz^`^nG{e3^MRm0&anRgJH-ulxXa0TusaR#BSJ%6AJmUH zYN@LW>RewBYgY_vD4rqm+nn^e&$`Xh8de~h%2f}AcgSa=hfAI@Hs@7p^*egNJi>lt zMEM`;mfM@p%V@Db?a!JUV0(JzIqoDtJUuGn;o>iWg?PCvVBXm2VdEeNiI4p)vPE#3$gJEdu?3Z&z{UQ z&R~)_%WlgWPFQ?Uu;l=-31x_H+9XD%uRog+nmhusxvdJf#cEM0EupPm#@g~)4ooPY zcV624oz(eDj|boiESeX3oksi{N#}vs{kLw z+sjV=2cm{q`!Ooj!z@@U8d|mlAYbiXd97gMt{3jpCdbwv`;$`e&q=p>+$JM!v0YHO z?B?^P3+K;<>vJ;1nHqA0gMHdG4hc1i5jwCUjq0;V{r7=o(Y#I=cNsw5z$g7f+qJ69 zr)@N<)uTVKa}5T`40oae;K)hX3YYjm5<)ONa@{SXDtKWp@pahTMO4;}@vWMd=yjEh z-ryAEg02QU)2iI#JI2LUo|MjMsEU%DqfI6DzqSRQTKW_gU0wZ`RKQEyIq2yi`kB zf^99EyPt{X;CK7|9`VYJy2gf38U#s=D#XvujQa77J6N{CUKsw}q4;kLcjeSa$uRb^ zf-S7wgxRFU&XVz@c>V>+T}Jrs<=F|~v!9g!D1iF5sf*z%lI<-;u3@d_a!J$9c=YYH z(kO)|0imX|-v<#IE--2gKn%&+Tw4NNVA2&Z%Lu?6isRX%W>a+dHl<}3mjIw^6PV}r zSGPkLnMzj4!()6ArJ!r;rL*Ha{pmcuCuf!i5>a_c^ISPy#1@eJL?8Qeg(>F6du^Qk zo>s?{38iBAxOlM@eEVHoQ@iu43V)KNNVko(lKZ+uP9Dzlm(V8+^4%kw*?t|!2Ceu! z`OIcgZ_DwJcAcUvco~Q0(RHoIo3L}=gGHr=n{xImTcZp69kC-9fDfa@P#QW{-m|Ms za!IK(rR!q>9Un)NLHN!M2{mP)lNPJ_`c0)$g915v+u(@v z<0|SA9QI0>aJ5PL@m3F5vx%_xpy;>}S7Rys!^9&2OM+e>GdIG-D=bf>qj1{)?YbW` zrZ*a5ng^8G;(EovaGd5r99XquZ1j`+K+k98$jcs>oG$!P(=9&Hu%fs@w4yC zW@=!9erpJpVEPKW;dQau>ake1X=6~(7#RQjxWqCmsBoSZf<{Iqb#LvxIKCxdGm{Nz z1?=T8gvbkSM5GIm)6~!oeg`FwX{&6~xK!Inxz4hjP9nn~)CUxV5Bj+;rX6Z!9hWoF zH^#CAWr=*b@dpgDa%!mSz49K*Uf{jYMmRQHHzs5&SXYNER&E_k222Jtavg=OF><9S z{5(7}@!KLw{%kr6M}=S%UWNWwf%y(hBNXFO6|bliBwd7yfleq7t+6G`4W;9w#Tr?p zCnJF@`(oVStbU1ZLHQk4Z+*LyTc8}SKHr;&tEc)|Wucq34re^kuv)RkQHZzZ(T;;H zY3*r3g-P4`C2Pn)I+i(5!gl?t0jeH-s|2RfFUFWnyO<=pMy-7HeF?;C zVNuML`Rq0=TrzMg{`G{@?ALOkIs@;5a;RyArLK{!Or{$nOz}KbvIMSbT!T#7KN zv(cOIdQ^0(PDXtP#$k5itSM_Y+iH?H*xpPv43>bU+iZv`D({3FJ&nP@kGv3&jYY-P zgWLpNnvb5dKj_G9v2io+Ai2brqP`4}4c;E+TA1BF?LKq-ER3a*BY?lTf@Vm_@XIUP z7ccQ&@)?zc!$OI`(Aq9>uKHhiI8LVf(V~$yqrAX;z8=VOtlCK$J-G^tRXiSqIToib zW#xH$Co`EUxZkCziOb_b)Ru|_hvI`rF(Vv1=*LWfKjX>*ubjRYd)%rIQ{z* z%pGkG9>TP+-`EPf2h1YA@XHB~V;%9F&~ScKDI@f!5>i}(nj9npJ!{usl^PFqG=>J4 zO9Z?ibE4gBmF^eu53-u)L?KK3o2}0Oi6Xub3cNy=Y1Z;CUeQz)OJIIdJNQ>v+2K|e zW$kNVYRGEieO1b9mB!cG>VCcoaMmMVyccH zR($s)RnR*Bb1&;C|W?IafFfa8yA@kVvw+lnN9&Uf32+pcv*tr@jjU*k7b?L@t8 z(Y9?dkVUX7%c(ErBXV~GE=o7s?X$EQWz-kJ8Yk{6n&&!Er3`{n-^G?3)pn<`gh8!N z;}rj*y7c;@Rac1Q!+3NStZEN6V2pA|SA9R=zcAs)GP~9bzjX6NN!> zm}+lAk{JzgYj5EvnIy+GO!@{lz(`4^NlQLeN62GgjMTHzLPpd;_A*EmjVU$>v65=9 zr6Fa%0f6;htn?(&m{K$Q>-T`7gFci@`V`||WQ47&q5@Ub$GRGSEB<}24IO=;Uh}}5 zG`IeDLhX5wcL^5=#|UpGM`BXp)dg^cFB!hBA-7_=vS|{hK|^sI-t$M)fAo0Tq){H) zNc0?($Pt;!g!oEyN;MJG=eZ)C+ckL2lPDDi1oXNasX3h+CSqLaJR?DH>w1iJx9JS2 zSR$B0dZ#yA_%3Alw*qBI$m4Exa!!SLh1(w;I0q(?^Bh1b!?C(p8iIMgR+6nNM{%Ua zMH}Erd5O|i{t^hN6VN(%`z==hYks+;^^PIMDZdl^@n$tI;lEvZ_lxqkf$~mXF=K_(A{@Cgan`2AWb=!dvMJyi7*3pNz|0`vG8eq~XwFNOE? zTvcD!PKAHb%EdUHpy0^FU!ro2z|cJ<*qycH3)0OWu!{5ftkGP9-z298{+(TQZn35X znBoR2z|-pwtms^G^<1e~gS|C7EQHlYLa~`Zv$_T;Vl=aChIUx91I;ps>c*x&^t`&X z?>VXrawQr<_m|AUOYr$NQOChbg+k9k0;z3(x_P{rHuWb+oBBiZRYqWY)%&stY<=6S zg=6$VX-4M8C;9~-A$|WN?&2+lOyvt@zPR)XqH#?1c2T!ckg@g>n%tTjCMy9!z8aS>e*5U-}suBcsz@^epoWRA<&hrEVA>HqqQ3MY8}~iNA4Jj`+TPKkahYPdP2!EJM)aU%Yh*);_~I zKFQc4{C##Ef}f$EJoT>QXIIM=eB?wchHEvSsbq8eXzr$xPC}3s&lB^#x3f%j6ms2iW)pbymUhd&@7L4-y{WY(J z&Mcaj*6v=rI?AeGbAC$_yYJOiV2oTIco%CnV=PDmcG$@q+x{T;w+nIlDpdK%Fgf=T zQ~b5dFmJkOqb4S%zDJgP$HVFBn4c*zVvfWykNHR^%e|s8rgIgpej&L&k8)@LC7DUh zW()TWdDzxDcxCENPS&XYW`Y@U%O{;VNn7oGoqR>5}Y0r|fh>oP~~>a=~-n zpOH@iSC!fnIb%p`%n z`J$}Qz_m}C>n`B#<)fNWTt>Te&O?7+^mahR#D)+2aO#Bw|MW%e6PIpl-ybG%ZY!F9 z8CC8XJTX1%| z&`ORfe$YhqKa(<$GoP>hwgZgJ3kQHv6E(#W;FM7hz3RgJ11%l zvVDqZ96@xstO7q0ePpCQfBu2j<}#)AON_xb4)a|)8oNGEl_cOW{WW1_Oy-YoAroDK zz61G)O-KKmiK7N~Ov+vrXYWLqizZiAjL_R>RlCuq2=ymc(Y_0Mn<;P7I0Jw}(iv;_ zJ@0hi&@c-e*t;L{dQuXh#@{+P@Lem<=}!V=g^^vhfkLaSeD}}rCSMP`ZyQ81wsr8D zK+u)wJ>7nuy2L2)Zk6V_crF>tbmfKLPm0T(HvQ9B#ML`V69EfA&*vp%{Gcw-2a7Mb z4jt9!)!gzAV#3HVToV}M;L#>uzm$Ma`T(A{i2=PaVNX}po}WB6+t3x`tuZtb_Zh4i z>W%U^o3d=wd>iS<%4u!wvM?@<&bOP@(*p$jKo78=RV~kJ=((nTEJ%*yBo;Q6^&l!F zV!7^5iU`HG!li#q6ceGaHFmCsPf0mc9&;~26;0F*O95^>*Ki>u^`IEO1x6iii%?sm zv_>6G8O!*q%A{-87mPy~&a^NGn|-&qburI8w7Ov)8U(w?=&weeo?C8LP^9ydNJUS) zH~=caE%=jm(3BfT8Cb~j^9MBx3(Mak&#Np))uRxOxX~|!Z7&ZUiL*tAwkJ!r3D^ka zrRS6jw2tKVD5W;!D5unOuJTmaQ}1ta00N zfZ%`Iy^RK;d1eMZ`1#z+DeL28Q!Ou){>p7=dW|JkrC zS2tp&sP}1}xx#~^%(DfHv{oYFsi6__Ez5j0u1l!N`eLLGZ^MK^7M=^yJMmU1xxDGv zko@Gds&vg_xeKP~9ynOL|BWN=8uSp;+b5R#bI?fnXd6b`_0ZASn=}H1@r&T=;{Er= zWiIlN5tQRJ{?3@0Fp=9Yb0| zAYoq0tF~k4QMQj*mhYKdpi&_66ILNa6sA01#>E@y3}T85wLFhH+fSij!%m840{erD zLkZY~Ja6v8%SC=hHnQV#OFa#23^*)1I|-&8702Y%_Ep=g0+x6PvbcGe_B(ZMdj)3EN&R zB)t~jGAq{i;~*!8l9QsTk~g8-r&R%<%6ciY zrP}e9@uenrN?(^}dePRw!YFR&<=uAsuQ7Jdc!k*e_S8dr-$c={(y#uSWdsLKyN#u` z972%Zu`J-<9Y&Fy$DdnL*aKUrBbDI8o;xbXePZcMFP+YG_>-akP$NLS99(xX!ThkW zd-34xFG>_xHN)+D_Y`1+WbWsP*eza>j!4q|I7<-V^4nLpRK{MP(O zkpByEHF44Q^f`G|Y)>PWVUIQ_anx&R_myMhvZ1Yr2z1}#h7-I-DnAO4WO2i{R+AT^ zM!PyZief@hN@Sr#o_krPQN58xpT;T#k^kCD$MF@4LL-a0eZE%PY>*1D*IvX}Ef?y} zdDX<5S03v|rk*599GGNAffqeW#j!Z_NiOXYP zOZC-rpw@N2mrFnkZsc4$Txb7HyR_=YndBFaV}+ut>MX&2_wMOH2F|J_OKsFunPI(v zGQKBs*OP94IXkt1XC-Bp1-ul|z>QoG;0q)&-Dmj5!2|n_D#Fxet(DE!hfdZ{lZfgq z$N&E3-V#dh@2?db!qG;i3g&@Fzd%R7;!VRdpb=m8dlee8I$`vioS+Pf$ze?d7w+q( zTW!b{1BF7l*&7`f==}P}wpBUv-I?C`(rDP00S=KzyEo}_yx`>DVjiJW3~qlvao|$o zNu%|(8ez$v7tXf`+)*u{vV}0gXy_1tXX6Tf;?rWR{|}~K9&2iW@bIsy2QhXy>B8yz zOFWlwX$EfR%q%iA)7|Z1i;65RX1>rcdl&AFE~sjDr1FtH8+Zi0Sxo!~oo~kV9ZY;t zSt$X$JNjrAzEPi&ZFWtAm(uO|=T~onf9QB$`D{E>ghu^N);qb?@`v;ej}I-jt7{%m zC)^~fgX}&TYD{bS)|nBjpO`n#if~|<3yRW18K(XKw)cy5GXBg=SCTHz(Exqx zv`I4|nN}W2g0Qpq9UL#&C(_3Y7;UqA=R>%^JR$D$JEUXT9}l&xDavh=HVV>OqpwZ;%%V%gh}?H-7(aOmH9p zl^+8*zs%(vD9<2F2Ey1*^Jd%KOS&fW=4zzCUoQD9TO0ZhcJfWWS#eB?UV@~f zHaI@doYw}scx&}3pqYS-ni!I{hoBks^7k75Zx$fcR~d*FI+jp}=xCEuxTG>`zwh5e z#KU-vkWp?tXTQs-xR={7q5Rj*oKh`hCHq%NF)Ff>6=W@VS#F-d4xV`+WRPnM8x%M> zkZP7rJbICC-MDJW6GBF1nwnLUJpI0y(Uo%U%?cf%g-`>})piOr;ufymI~;eWt=LkT zrWSDS^-op(qw9(5g$C=5q+RaLz67nRmXinY=B~{dWNXvM7z2VFbXy?fc+>VBWRX*u zP~Z8KCG4pla#3dD$>Xn~QsYc*5aPeFK`f+umT=>qmta_Pxw@n`VSHAV@9j#N9## z(|KRqfW3rzWNBTUuZ+5Z$~I7{(Z!zZ=ug{l3tXWE>moE|PH7t` zLGP{^QMea7xTUmU8yITN3-eQr%erCo-gf*F9C{XGpUhIV6a!}-0A zn!97=YriT&JEJX_iF-ntI`s;RaugCVo$h0go8D$@>jvxyQ;!L{OoUwvuiM_Xbfr%b zY;k^d5nhAd8tiNIzjUMx7*3mb2A(zLI&Oc$>^&G?;VW~R$^v^g<2fk}Sz0qJ*?$69 z-0a#yo@L+0`&FpT*qhj#M4ont{WYs?MNLp#Ry^mx+Xh(X1AmWm6E;g3GVFmIC?*s` z8gnba;IRwFW9 zgZvTb_HJ*tz}jx%uAnu%w!! zBCQqaks1v}XGM3rYJCt?NfnAa6I@$R?QuzXXn>ykbmpE5{i};&a=f@FIjj2(Gt!8# zh8Py6h{#_@TmeMTaq-ZKQDo#rh}ZP>gSLPv2MKNlzWSi7)7|s{0hls(z*U5y-ICDRdpVAZA0oaQ^B7nDsw{WVg^C`>j3+Asx{=(%PiM zm)*VRGV)uj^&1*f6)XNj{_LdG^^P-lJycLU0AW%oRzhFpP=q6h*?~JgPq~?-#>C&l z?T-7)^E&c*6}RR2&z^!>e*I#5QY&fGS6+xc zZklomH%jW&mftj5GWM$p>GhnLy1?M)&=USQ>`y@jB~6mbc4i47@*|M_>RxqKsr9h} zIZ=?ESGC27AYOb=^KYgEbPZA?uurA5u79Iu_g2r+wlh`Qe!1 zN!YL2woS%M1FMJYrS8KcS=_N|o)Z1rF$zPKDQkN6N~5*@7FMuV^(R3aGy61|esp=_ ziN@OJ52qj!Qn#ggZeP1=;D&eYm+uc-6m!Yr%qlG(2srT7%}CHp3~1K6xslp>Sj>fT z_$naNSP69H{$kgs=`4KbEdPz;pB;7{j*l4?`&kT2(ckU`p371GCU+OzQu*Sn5o z{@$L~rF|+9=QUP5(dof}Gr4LQ`d!7A%+BpkMeb(?3Hln!Jhvw3J-PUtVT}R-vXbEJ zSI-jt-}jHJUv&E>r?S*n`pc31;cN6)x&r=-DXf=j>%igg3WgtU4J^A^)NPJqLHZ&U z#ZQ*LUt98`A&Cz#f__iL8m-+P1Ddbv3Bxu?m~>9szFLO?50mJ;rcRIww}Fx4_~O=h z6mo?H4WajuXAnfKf2mZV4h~$Er>Ilkx$PdJSFSvESpNwtbCRq9yuKmO^)*Iy4f3AJ#H=1d{;OhS)_cs6B2&TY2#Zu{3W#X6HCw`YCD2`$5 zVd(mf&Tc?Dh{oj2r1i*r`WI)H{G|Z41^l10&b2WG(4}hsHHXwD{vV#+J)X%w{{PQ8 z36-L9EJfuoA~oT8%J-u1rmg2 zn-a^Ip6;P;$LP1kVa(YiHrK_49-gkg{8B)2c5Gk!F7mOY&TJ9iec5_#aL$`7Z^m8` zSd6Nc&o%9CedPynX$+BNL9Eb&g2dK2{lTu+@#De=?!n=7P!9& zMS6zrxG|^!$k&NFXYbr>tDp-R_V=;zB7X)0U4v#K>IF!NkPL?|Nv(brTFDMWOtmG% zEO7KuN$OnQ0SJ5W2^&cC>Sfpi-xgEe_;Cj3Y;Zf_&R(=YFYX`GRdCxY3EjMf?4`LF zPoRatPFb1HuAMou{=Zh>sgBD7WOcL3~z;z*|qOutDu^4KV~h!X7@Fs|A^Sqz-N z)M-Ar>BgFCdHi-b<)?>N6uj6;SqmKApU<}zr!LU8T6A(he;3%~6D7XJN>*m})J(qmI+$J>tTe)5x`n{;I<=mB&a!!!LaA=y&15&Is^ z6R%>l>EdBEB#YD4RkdFvuTorE=ep}nj4Jpwask_c0NL!^V*4z%-v~RimP6C3Xg0*# zYpP0{I{*G$s9zaUSGf4OoUb0vLg*Xs2yg6%bs z5T*(<(=x>ewtJYx*=8s6Y$6>ds9z?Z2L(C*W>27-noIavyBkMCxijp}VF|Y~Aj{bk zzGo~}pwE0-v5hwmeI*X!4r&2h<2U|*mKZlS%){B=vt7A^B#rqWiO{1ziF+jyB#bMx zP1p|95bdApW9na%x|@tnN$Elp1<@=(-VZiGjQo2co%<=eSJl4?atK3`Pz*3u_p>&V>Z}N-PYcQ`q%)bVPp$cTE>6D0^7S$cqjfB1xdZ(d(IR$4D4GWy%++?5Lo@Fm@iN z+Kj9Al{ibOilQ&H4cgW>)|r{P(rIPU?L!}FNfxf$xr+S_!enA0O4qC*(ijr+(zu~{ zaQG|RJ%r-AqiwOeFxbMLb)0NLY-_$P14phBfV9)b*PWXK2{HkcCW6f)y<;@TH_h&A zD0jZ5xmiAIJ-#!q*?UM186>FRlFFy808|22%Dg;3iZChBU-ZwlPL1Ll5l^@?f)7*| z)r%62();JJ_GFW+k;(wI21^X82kp$O7rjfNG+5uSnh|WntEjUKO(@@5FY^}u9 z;6c>JvmJB6r!oajZ1mct&i@t$NiVk4hyVWLF$6XFY85$z9Mo|PeIfe`uZc%gr5^-F zn~eKYFW6)2*^Xgr-i7@o92-!UpJ$Fk_cz9iz6+5{XYVdiPj6qaz~-)%BWNm~DDTWp zMV(_SoRkI_wR>Miwf(UeUSO=OVM9#|IIyXY>YJ<#xay8E-^o97d%VESgoVH7^L_N~*om_gmDz+^6LSBxFGqy(7 zuOaSlHD;T+Nd1Q#RpDRqBPycag>@FQmA3b*FAi1RV4d4PzWUeZO3(+w$EmCvvtf60 z9i_KeFSZC|FVfaUR2kdVaU^@=oCsR=nXL>M(fRX2AXYBe|9E5Vd0=HtQ2$^TdWlFg zH~pB{H05jbwf~>_$D?KbIAGvkk!9vy=?r1W=Ch6B3w`=}@zUke4ejA1zefJdjF4^N z>OghF+Bt)pld~_wLjCSXaG%qKK(g=h!wYqmqwfk$I0=m%JA6g!CY;b}YABYZ(dZ3w zY@CYx_C*%UYr`}Lnv?GKS3nKhizTzMA$Jr&6s}=al>czCZ_SLo`C^{4s=RbXhIG7BHS#RNKUDmJMO zTkDS@&Zl2P+4VoUZPg-t=7e^fkS zA^sF&rvA4%8K3Snw^w@~ip%qR6Dmx7JCyj-IvWSxbf(G|{>8eFbVm{GwnHslJr zncdFRTjE`pRAquh92*xD0OQpr*&5qKFoP?XH9Q(8rgz+B&qLua z=!5nttz3KKrwLN;%-&h{Tt~6P+ZN(C*Ty^i^@?(_b3|Oe$7FnOJPr})*+I#xK672- zEq#Kgp>?SMrOde)4^cvgs!sM`!_$>vw47|+Rx3!|DRno!NKblc^VCN{QmLE<&}lfh zccl1=YIH<3t}4XKnj7Ax(`$55_0o!`wjuI~c6o5w>hKP_^!P6niumFppd{j#-snmI9;XqbNtq zQ~zrYg0<(VB_n9evFO+1D_77uny$(@ZkNI{e0XH1XxUnZvDDi%UR!%2aPn-3; zS}M649PjkU$2J0LEcczKCPR@|3RyXJ+;lANF;v@0_tP)9d@eX~E-y_PKjOV9-vVcm z4x1n;`}xbK{cz z(lPcZ!IMz8Exs%Q3*9492RRK5t16sJ0*f>8MspX+Yx;$9YLGvFe4neWvYG_Fbq$%Y zJif+tz+A8OeZbnJzV482#xEuG_g#;_2yDvJ09rij>0;R-)867(%_Blg2ChQK9j<4$b!HFNWadGH z659^8w%s~VILR)?f7VSkscE|nf(i)7?Jw0TQ8%xk-8M4%=BE+VOlcl!YIw=VFzyVT zWB%o>4m%n!7ZHl<|Fz+_ECpQ@{)>)I6hMY00?g(LI{FisJu$Rw@kIqZ>p2)AHiy@) zQ)O08u}90K39Keco_o_sobL8F*fZRAs*>t;5U??OUJmm{V|4i~ZsYH>@I?<^ZeXZh z9}5NOJ-v0dC)$4;w5_CmfXV(kc~(*BvgN~D-@U>YZW>qlnp&0aOm7ZYTV;Ui)s$2} zIhqnuF+E%6ABI}j9U~uGhu*0R>iO5Hf%pD=r-E`v;p)h(or+0b#VBk5_7Zcvkd*GZ zU(e|5?)c>P_~^IK09Or7FcHdHe6q#vq$QQfHh)C6srj%B2>Tv2HyW%+?%V%+i@N+! zb!$YY;C#g1SBw8j>WQ7ycq=Hs6>KP{u3}gH#G`#MI<=#xI2Y)9f9?N~S29ExkknXc z`yWlw_?YlZlLtn<#xB$7Fzg}R>eB@~8HVRSuVhhAyNk|LTJ{>X-!^IUNYGk7Qrchj z&aVev*WdTyx#foFN>*x&XEj087gWk-UeOek6`H$c4+`v*7Cm0f7YP9k{IfUbh6ECy zfiaTdXQg(p_Oe!`m2Y}qK{ZRwvwXiAcRhMe#*nPZtc}JNGn`>CN9hDN zGFmVCu!l4EOKCUxx@rE!XoMKTS4Bs?MfdgL8P;O&Jft{_;*SByY;FA6fh%a~5N7~; z_4(5)djEFcZl%Rw{IRZSItASwf?j?w`<-InaxBDx6K}yOUlMkzQxN^18?1Bmh@B*B zUAC$!MaeM&p6}BNzy$65C!>wt+KXrV&=^H6(~mFnscw@CO6OI1DSP+lA@71QAA zQ$S>A-H^t10)QIoscN$Q!h$}3*&4W!VjYSU6MF)!yKnu76iOaR{aV0FF`j4`ZV&%}i;(*F$K81k2y5~D+W)Q%#O3wa`g6WBT>iYp zuHQ_}Uyn+7kyWDJF-paS6Boh1PYqc59SX}L?FlH@{;mmpGHW0tX#$`BmYQ1dOdF>_ z0HtP|g`+H9or1n4Q2Jh3Od(kDs?FxWJRjq(C3|JEOybMNaifhQ_2Y8&HwJW_(J~8P zwrqgjaRJ7^bx&Gc(HuAt4%eY!};FH}D~DU)`?_3ue1CPST3mA_qfL%I!0PCop9^W94d4}HuhH5I<~&|BO5b; zqi>6cFigJDdOb7Jrmii+HAzPnDnE7qdx^CLNrkz2RFNm`Y%$+fixEp2cjJ*;DMPBK zYNgx^yMXNV%TkA}6>BcIwiZ`dSD}ynaD(}B#DQC)qg@xawOr)uc?F&7Cdk))(HQwx zj8O+eFbEW+%aOi7xAY@_#K(a@oK5E<~Xr)sNs$AuG z;8a17w$)pFGSnh%bL4fcX~}L?c7?G0&S|HC4;9?DK3AhpwoQ0={dcAc2W_?zzDiSKl+EOx0k!ZJF*(q;%t@pm32 z2794`DO)WdN9Q;9P)gzbknp5}eBP_8t&ubUKjifCuRQ63FL&CSF8UJ!kj>ErlS5?x z)BFSlRdXZh5l<4-e43mv{G@o2f&s+1?lkb)e1A#oM<|PMG6k`sQA=<45c!{$^epiR z)Shq8etjR?Q+heP*gLM~^23>fO3`dIrdfI*O8c8=@z~N(3q(TS{*ZWR(2x+-Ao@h+f3hi5YRj>#01EGjT#F{vrmLra@=AH6&41p z=369By`B14C_9gtjVgNJc{RSIq$00%*z#vPbeS;rFrWdg@o99M_yJy4? zpOM_p%F(m`l#2Y`y3w9lUpY-SmA`KTTFb37~EIDj*t8;nHMH z`8+MfD(eC45}Z2qYiKEtJix2m!MZ|*Q~^EK=0u$}&b29m{7$xJy;y9gh^c5c#MYB^ z`h|U+a-h}GQ7bP$0{xY#{;!H18)Ia2)UZWV`(i8G@3%zH9Sg)Sq z;k*-d`LKoR`^4bGqWu$NADH7RY{E*D{9X+JP%5Ckq~Pbw6^J?G;7|31%^r>QHPa7C zW)~kGIyQ0rSPonB__n2KdeX-p7O|u3g0<@nNUnVP1N-r#nOorsB z*$u+4l$L`&i^F1}B8B804SDp5M)3ez-2F9;$R=&L6%#1G=Y741Z~98f*~a}7i`WYT z=$G9XVA0TP$R>OGgE6N})^x<<5Ke|-UtbWpLLVVB3C;;drdD#LgzrUiFe7b@zNtP3 zqMh6qeo0Zkp|K3ZGgt(7f$#>iF6M79{Dj=#_Ee=(E0afRTTm#&z+{#+i zQjfBXEgXN+q7JIij<9ALT69A76AoV^ zHR3%fO+EYDjrww@7^Dyp4zKjZMS`hOKCz9BE|sqzc-`IcivHsj5G2?Wn7l*B5I*GQ zyxSMIf5U8^{*Y-vJbumoulu!r<>$qFE}8~QyJ$8s>fuLdh6JJ3 zdtp(b_3O*a=FK|lmnTUjq^Rc1*|44Ot!$gwru<$B!Ay|XW^Ab%?t2p^L6CV18j4ej zFB{Jl_a?KC+KC-7y>YhQ;IY2p!q#kp@+pxDh=PdLaj$2YXdhOw(}AJ6^lJNXG4E-I z>yPA;s)YB%TyMVk7$PAwe{ zf3sMd36>c-@#$_~$@c4E)!!p+lL)^sbOoi!q_1Xby}W%}l=p4{!GRy| z#}su3gsq!-2B@TbiCc_bjZUOOz-1!JeJWddr8z4?c~3q)+%w$Ka8-s1pVz7mzxi>O z6aTKzJ{rfny|K4k3=`eKtB+ihm$I+=Fj?@ecJ=>x0h$fLTI%4|l}8CM&gI=FcY5F4 zz#1z|C=JZUh+f!F3I5+JAWndEJ(kFnS%O1ABUbWVoVmXWu}9F> zm$Lq{2>Q;2H+>;IGUud-+1m-i?FD*Vih_$HRIIm!SDU#IGG^9TLSO<;Ndq=OcRabbMOHf zhuAlv5w5%ICQ5D=G+Q65 zdqDe1rvZns&?&9h=17-4pW@Fyl_qk&!#+K~9U5hW*H7|K(hPlwiL5*y-Na;+V*VQN z-sjz1%=W7pA8eagWn4%?@s5D$!kZ(p&zlNB+nisif&TD11?2QdT1B7=A`UXE?y`RC zVeYIILJSwPo>e~CVywvM!RdMC@N3ccHpmZ+~t@ zR%;fiW{dyf)aCAvUy_;2ao+JRfSA1{;8xq$J;0u9^FaFk88CIwibm>!Mn@EaoFgu> zYL5M!5!7A!G{+?qogdZZ@qR=5vQ3;vS ztMxOHY9|=H`pj|53SL&&R@Wth9O*ivCb{7;_k$fw72`%_OCNS$pduj{tn7qNi&5p> zw$l7LXH;>7%>-4=GOYQNi8k~i61IXGWcJ7#mT9_V;wlzOZL0($wLnjRgGt_CNRPGp zi!f33aoeP>4u_`SaSve)RV5qAntU5 zC$?J3$_uf7KPM;qRrt}cxhz8UNbp+E`9rX~xqwZ1t3V~4hDQ0A;@vP+JR*oYC)g$g zK%=sg(v{NB3Sw&$WB{%4fpR_<-bs}5^=tYbpe;7Tk3 z60`Uu(W6rismE8pSMa6Iqx-Bngf$=U^*3+7-X(Xg4s`UK#S_wG2aG~}Q@#d((P>u| zVnWZVeoqYW)KGuN_U_2U{;=MkE4Kb>;_s6f_t7nEVQ*g)tH>p1ptsOPamo=NRn@Tw zGrmOe{+uZK zG$xtofG?(0aR%1|^te`QTJiQdF*Yj4v@~5dIUVl-dCbn!fXM>M`nAFJR+g<|jNUQ& z{RX}!vaYK&QUP=vNuSG-{YJKySJ+m6!wh#g3Jtw(%_*7sjUHo%7a3#RJd5MY495Nm zB32uGohDj%`qq?L@M!(AM0wR1y@_IQB*(j*_)TXwGy323=G8wlY%&cenDPi3@MPi! zr4u$ao(7VBP32@S-NJj)I>b6OaR{3I9@_Ha!qjJ7_kLozSw!Y|VR4xR#LU~aTlNvy z#3*LwLn2D|&5xhiAB|lWY%Jnd#0t?+S9%DJ8Q3c*zcYR#6FB5ehJo?y2_2-PqgiIA ztZwuWC2{GN$uTc37kuw)PS@@oygs|QeK<3k<+rPol$gA}I+$^7>#)F$tV!v0+D3Er z^^jWXc}Y=6=plJ&%;5-}(od&w&&+vMeGoPzdVkCx^etpBjvqd7U9;@Vc@+*Z4vZSU zb!lP{<(-XN8^k8pM)_nd+qBBYC;FZUTr6+h<+YiL929Yk4~@SWU(y^KxJtSM2dMVT ze6Zd0woC?4qqs!4@Lq|T&wlH+^+V1`hc~&LI2OTdh&^Za!M-~y(RPyfX+P2|Fh9^f zXr)_S)zQBZZmnSFptlp=V*_inj4`1i*z%frI+{D5`3%>c+Sf?>m02bAw+tmm8Y`y{ zK!QF9#G5Xr%Dd(aex(7@ z9UM0DSyw%}Rx?`YR$z-B#}$cHMewXhHa_x-BJ2RNd0JjW>$qN17Iy6{fI4?2`zPV2 z!$9?hZc0dvMzavir#Ta=?y|6G(Y8?a(M6Jh`$s!c2a=fX zo|e9QM8jCkUJdOB$7RyBWcyw6C+4oo9(V1XMeC|=%QQ-N*~SCza}03WW9-kq;f7__ zDvJy-b&X4Ih%$%B%IliwA^%I6$)4Nvb20M;Ee4}hVEh#L#a_P0ien@q| z1y=Wa;Bt=cZe{npjVZa4Tj8njreY~0bn%^5ee{3CmW}BwVtOqiXXD5}n?6IcDxRv* zfsiA+W9Wkco(&T)iz^4E#|M`mbr2&n96nmy{d%Kq_z8FWV1HC+6YZa`7iyNO(K-xB*e0cD+;WaC2ziMKCIcAW${3N ziEHkUjYO?{CedW&53lMzH5PDwh0yLm??%+5-L}=~>cF9H=1pfA0Rc;O-Bf z)9-n!H@-UJbwzPX0FCyYlKOVxrtUc7jIv`$duMi+Q$cfshnN(j=UkpS>AUJP3yEoB z+)CAhC-j^7`BT3o91+5XVIGb8J>vyCN*M^f#z4Z207Kh)Ll(`EDv(fHKs%om{T(B% zLRZcUL*<+>SkirSNPngsD<7+q?Qo69u5E6*hOF5mt++)6o+lnhMWG5jBK8 zS_(D1Z`l?hl0o0fN}_Iy?E=rt$lM7NX>yQm#Sba$9sfc_V)-D`w9^B3RE^aUbX#a6 z;qT5UKVCbhYWXIBCX4qXS z@A1&u-!e@HH&YE&2+~==whG-l^8oWa-4M~5=uwXM2iL5p%vI#mobsF1Tte>mtXRWh zrm8$?S@@}>-2~nL^+g452*nRvIvV7L5rFk{xZ;LBYtruXd+tR`5RS(uiw_l-nYoIv zoJ5MM7SLafoUg&_Y%qrOy|Odw+RzX}iUwD6rwEs-7PYAM_RjX|xTw~r&wf5+aSrp9 zMoaC3NUc_hRnUQyHK%5-#NrLmgzg8EucH?l*PZGns_AFP2!aNl=YWc{^_S0720wRR zTll@iD#3ONyi2wYk<(21K^%Tjzhlb-XLtB-rIcyFcTp@du02f3$b6V*F1uT5?u zCpZ1{&mCaz1Ldv1+~uHl8(LEP@$MKkT|KZV(_!PR4_~+)t;>tLY2K_+XL?Q$>|rPf zyR8Yk)zRyDMFoS|6HM&cd4uk2zpM%vw?I&3FmlZg`h)CFM!<9V0(u^S(DiB^ zSC%)whXJ=4VIN>=Iv2BW%Wd3KH3c~3?*F8Q2IRP1<-0z{p5Gc^!C_y>*c|)$wrZrA zK-7p0^*?FJrdh39CU(Qd18OG^#~f1-P4#9B5u$E=8b=H#n|Gzb9SCLlfODpJM&3`J z@#MwZ=_<2sQ1LaF2Q(_Fq>^-Tsm-B^@5X(O zQ+`i$4zm+Hy7!mzJgK|(<(Et|6mHtbX~WmR^bZ>d@jtK8DsUsRUV5}j!)&Gyb@y(< ztjEZ++9OldM?4Bwx*BHulfHBwJRZ^&(wZXQ%1dcnH#&968#m8xAC?dp;dYB;4fk%g zMBv&5(i&h)7>a(n2TS1pr$_m#T}YEWXEyGBzzE9YB0 z_PXUh86~DKAAGj`UprO`jvxsUX*Il*uOTbc*uEo}<)Z@PgE151)p_Dz^h?Q~MQUx9 zB@>$-2_p7E!d`MIZT4w1zSlzb)s3XL4hZZiUqjNyh3}%R5yW#S+3`y^MxK|Duc+Un zSATT-+l@l?lOk%WC5bcQDS0~U@LMe}e|x!*7QbFbC*9F?mzS$aM7kz4=IjfC{Q5?uV2QX` z#p5ZOk5(y+{&_1ys%_vYwcI*e)zgr!Z4B=- zrg2+l!>RXg{2YX}FYOp&)Gd z0hV8a^*HBC`cijCT$A++wr&nH9Py|($rA<(y~fWNpBOMC?fy4$v`@?nG7(E-y4UB0 z#>NM)hV1=^H1)-jRKGyNQK~iyN=&!eM8bS<)#z~8pAnY_h7*wk(>hXa{(p^qtj{YC zu5J9erY88B8X1(rze)r!_274=?)LoXT1QRpiMfXe6_6&~wwry`jBi)it{a&-?@;KQt0AYdmc!awd-M{Jd!v7nbZ3{@ zG&Sn~e?B_AE^?}Z94dY~-YNd=xN$?OVIW?uqm5j)E3P}>e)f$&**4ANrjZ(T*f-GN zQ%=vcnCvwAs%(omkN7y;`he&`4YByo5;`uz3Bo2-%?Wn{)Qp8rXOJw*y`5fu1E?BY^vykq3D^)DO{u=WVaR%A z2r_R!Ix2*Dct0I1ukW$fec@CGOjsH$=FFjd(_~(TwrMS-18yGt?x7eC^fqVteYhNx z(2Ez2jJ-7O%@{4fPo7L`T|doUyCfuk+$ucMLD5ZPIkais)5EOO*Xol4^`A6Fe`_UC zPxv*RF{K;Ril*6I-N@GSR^HrdSJki=VTe9D*k_r zm1H@%T^_DpI@wR^Lv`hWGu0%R)9%Tk^P-DHYkBKY*qUe3v$+AQu!o9J8DBt1o={PfGu{|qs9+U{@Fj?imwY>uaX}6wgrUf_e$p>(>a?m-l)BXBF zhoTDYWsYtW3hnlI{ENm0Xc)}XXT-!3++1wD6e9*b_P(pFum0?|%1!|lGCts`0|)mC z$bi+|0UB<|hrMF(J~M_fV(&hEQDrX+8wvBR%3I?ZXq^MbZ%ea{YUCX2ZCT`HLG{`Bw^hzaOG3ngusTO zh2Q?`Ae;;Vn^iUC`K(v_!orrB)&x`kjje-+*GtM6?J+9IIoR$74A!dd-B7g3L=2#NZ6amX8vpMDtaP$3fvACW z82|$@MBBe*!kfg|n0+6PJlp4{Q(CzAGa=}qis|COW3BDK9jZI5MZYgQD9t=xWX%1C zX#IgFr;?1Um6L}U3;s$An<%K>a#*@_f!YYzl1A5Css)#$K1#IM%Y=TpI7m+JNXaAE zJ8-4&!9SY#Pq?#!8CJ7w^SZ=2kuiRyvZ>uYd-z;rm8|`GNEa#^M;MAtHP;|J?Y@P+ zU5Q`duUwl_O1XO2G_TjwQRvmfOAF?VP78UhIFY*TE*QV-r;(Y4FD=d?qA?wytZOPp z=rg3=l`rR|Oeq}RkQZ^oge50vHx;i@&0V>4p%tl{k3~Wc*|8pmM1#amtlwg*nCHF! zrCDZO#oAlI4fcG0SuSrpF!W2q%|A!`L3nm1c${#3NnJ5t0ppMabmY#{VqT@DFz!|U zHKX4^b$vOtcxoz}tXWfWHF~E;DOr6!~(oqy63|s4j%;|8p z-8St2Z);B|jx0Tx#1F?h;pca@e+!DQ7y(Mx=J|T2$yy%Zg`;ID^kE^DC8f)mLl55< zEZ;K=@Awwb?VV93Wlm_Enw?ys<@6eZRKuSm3vSfKZ0tM4bd6l}YO81TPs52!`F$D3 zD=p(yz1^Jy6j1wxURn+AnO~7GHiX;L>_3S+drb^J2s~17?E0)z7SM;Y&atb#pHMPG-+}Eo8Y!oe@9(77k-FRZf$4<{9_yi(}1 zXqyb5O&PUV7qhcZTlm!-aI$sZ-kSJm%j_8MzqTy6lYL`Itw?4*KPQjI{i%xU(L08I z@Q?A096sff{m6@JP3%};c{GRAGx~YPczftnIbM14I&`C4nPcv4=}FB?mj1BV&`vR@ zl?52kbtK%3%HrK$ISzO9MK0PT?0kp%RUf#EONDuOZFp$z5IIkiQwAUdT&|xiT~c5F;-@L0X-X^i7v|nxXf&SBm!6QAI9|T(JGNA2^<(N$ zN$2-*3~~QWIKWN`o_|~^IKSXqfzQdO!6_)K>eW=Y>3wEm$BEKuZcpATTokZ%A9O4! z8NTUC?2`j}8}}AGCjTqqhS}Bj0r}IKxN8^wr!z3iPHZ$_O(dwS@LLq|m%7aY2 zVfIH!+sOiJy&7PjGrHK)ev*#BG`>sQ{BcfM{xq|wr1A9pkI@!*c^pRcGxd#*hAY31 zbx!Mm1_ofypjV!$M3-V!Vug1*h7`xCX|m^V$*{b;8YMutw^{|97t#o*WqB@u|57f$ z#l2h4xph)1#(pRaQx~RpM34x;xo(5iPL!A=sU_K#hf4 zi45Mkf+w-9YsXNAV`T!cUNp{c^HbvLAY!&ZgDZBpoFVJICMUI8p#`M}&#{i(M$W%=63wtrsh@MINhUKhG`y#rmG` z&edC5t=&nNF^tY(+-?g0YYEIT^X3kKYbq{#DnD6my4o8MNyvcaueq`Ib^ZUx_mcR< zLCL`54f)64ERotS2z^()^ICk)UQ-k3IbCFYFTTmX$T#?S(FqVdiY+s>l*1P4`5mkBs>t=-XsoMZ~PYU9)}gBT`0}) zB28B|#a&_m3-(+e&%BJcxV*1?7IGAJv6&)z4$vAjD>BV>YRau4hz(yIpB=BtzTrhS zKjtWyoyR;Ya~oC|FO1~qeExe@>llktyK{YFql|Wm=8v!I2O{YA8bjK++auxLR@EWl zvnWk)9KBR&%rF}6ay<@w4Ya<$Ux6;#QD|Rci12K$-L`2B76?lJ`#!9cHN5_WQP{qq z^agG@(e5u%cibnUUMV_)gwwIjDKCBrr@vF%62ga?1Qqd-FE?6R!4&1Mn-3|QEouWCa2ET1RtKRV75Y%Slzmjoxe~D z=DLTdjm=b_xNf~8r7-zTrx%ZS#~+GOyyD@Tv?sEkcNL9g51-(Gnt6&=eW-T5nc}=(h#8ltJNhjzJ$jFShBlmXw*C+En?uI+c!|M3{!{uA<|?#o(Ce zf~4vlJp6s{$M80%=%Dt&1sjY$`Uz1R1q(f;2|m}VX$=`HF>|9c-ZHI&W-d)rOMak# zIDu-^kc4xX6jpKkO4ra8fU55Oz@e2m&BZM987n83NYL-X5VOo7Wh~t3PsFqSiG`(h zr{Cu9<~zrS!elp_*;%@m)=_NlqF#wv9n~K@e@@W;pBDh6sfsXc_t181kn570BMZPw zv=*{vR=^OaG)8}^RyCi~#=M8P7~}xf5;4xRGb;u~8$noq4GH}p=GZH=Sr`k*f=hVo z&7=)VyGo~-zIlI%9`tAMhP8wdM^~4dnTbkp^`uls7IomTomXYCKfLSsMX#Zaeo59Y zwDu5D$G4=@g^`8Od}92Hj`Y}!5==Lfo-}@SZDDI74v3T4{jQ0qqE~Yz>2aD7rJ*WFZ45x_q8kwd(h0v&P zSG81Q>Z(pheTX{vCVBWPUEy^EyQt9Ql!Y0OQQJ(+zD8kZ$pBD5J{!v?w%enF8OeY~ z>rl&VeC>PQcTBUlmLO-dIz>zBRS+B;?6)jMPI&8yMj4v8#(U%ySk;?_!_uBRM$_2@ zSG@3P!!mUQZ;tf6DSq>;g?YQi2B*4=J!~7m2`SMUpB`$eOAxWO#^kcSE+Gtujqe|5 zVSBe_o`66~J7&^_0wOq>QVJ^NsdJqH`x)eu&O}1SzCeX(Sa>D*)KVz|jITYNpuk!e&LerZU0Kjn=;H|IL5YT0{f( z1jO@?f&O2(qDr>;?&m1<_$58XQ(*;T2$8=t3_VTVW72)L^(NVlyqfcu%9sQ%I zOb-xnc<*V!pvxNXXMGed=a&Jli~wa&I{Mytt7ee8x*whA4gl4qi=OC@H< zJQ*YfK>vZ_gD$SE6agr#a0V_Zd&x!MDEZ9yJYnkQ+@&5}S7o8h{m^W2X=2tqVxSx| z({IeJ1y_FC zbY{Hx@qrfO>qcoyYFjTH=3N@A4HmDQYzZ<_hFsJK1%%#ntqHM#0msgIEkg36Lg;n@ zdX$`gU2@&m&0iP%m%fV8(9Z+7ON6+;R)0xwy0`E(5fD(BfD-&%Uvvfc?~cFzTdDz4 zl2jXv)f`RFo?!P&&6cSMzNB{HO+d+3MOoK_!X1j{?SV`$JTEhKDv6lV-)-TV)IFJxWtEyO z%$~vy4F)V>Dkg}lDT+EZ6@NEpM365MwxhfGb}~J?Sm;>ebr`;Lbf(93dyu%UN&fIP z^@qZc(LG#@hBbaiXg+q7UD1&l9c9+?e*v}!sv}mRiVXnv(Vpr4_8Kdvw`(%x1fTR5 zaG$}Z=3WqQ?zyI>3&}riZ0>;0y#jgj>u`2&dZSQ zU31h%M~Rj)o6vVnPiesguhAUnPUn=`hHu=dTcd{*XG*ak3CsC|E!TXxRf?|AQ}YL1 zsdMiyOgAOo8*Q|)IiTi#yDo%XGEQ6#UVo;gEPF4~v(fnn^hz1m{ic45^g?Uc#)eWDh$6Yj+5$Qcpy;R}}*&Vlb;8Wkf`MPX~T zg=SP0{TcHitzDbeVU=uEU5oz=R!kA;nn||KWYE@N8l)N!!|kmHz14V(1LvG-(O5sN zg)yyNojp=VCW=GlGRf)U!{SioYi&$)5&X2>A#cTSNbZtcxuV_~gB#(Y^M8SO;Vz8` zb~=*rStU*s>TEDw5L+LA1yv&{v_^P{n?b&{Yny)EJ%rYcoVMiSi;8vmLh~yoLs(Ql zwo4*%M*5p%xY#w}{7lT!T$bZUgioEf6uP=@Vdyhma-XZHD_;B2<|_+jOhLz$9ob!$ zY(2nuC6+t_Z^|GE{<7yhWvixC`8s>mG!hp$>=PDVZFOM2*EiLSqn}~P+Fww*D)~3Y zjkG141#z@WGKspHGND_Ycfo%#DVA{^g9Dva2pQd8KlVtdB)XU{5j`alK%-v|TX$1s z)$_vtw#&A-ZSWc+cesM|md;IHn*=a0YTPd@%Mo!b)h~A8zd0LDpo}3WQxYL@I?-Qz zleaw#e~TAJn8=)Ee$TP|ik$18tjqxDEZJn!2c@I;wZ1UZn{Nxa=!qMe6@(BN9EfmAC zH*PRGJsY>oSBC5~@c)Y_mI`XZ_QVK4NnfQa;rH#@#C`k4L*X~Xne*a0Roa&=UPMtnh zwMShY0>d6= zMr1DiXmuP|98&HFFu44|MOW^g?Wu36tI18iEp6o{&Bjgk=TMm5k1)ig?U(}8c@FEX zt$w|Ym3h2q{HZzhJl?H;vhLg4^TqGp=YnzA>DG10QFquod>C_XNMExi>OwiG|Co*D zw?PeYA@-|Y{{2Xa-D5wF!~jK?>}I#=Un}@fuzznE{L{zE$pRyy^aLF_bQkXw|{as%apeMTu_UU#3vi-l-qhS;+F_3Ov3XsuUXCZ+MS$& zyO`yMeXxLfl&=o^{_=UH5l|PhvQ)j=na-4Q2F)2O3H60>yCrHzb4`hV)b=Y?i^`&u z@GMj>V5ju+cP%rO42O-}-f7gY!xJIv2uYbifMnFex;*d3tv{w{d%MsH@1qGZS+y^0 z1yhzR{sdz5ufomcu4e7$F7uJ;2J}4_g^OcTXc_i`SQuv845ZjcJ>Oydw4z;(w%%4gM=>~c%!M%)aKXaoeYOw0e~Z6CXDmTneI)BeoY&&biS#%% zI_0U8R5%u;5A&Nh6szwL1$+@>e?QqUF56ffIu%V$8@y2i*PG{bJWsy8Xu^%_`*sLW zo(@P<@HIN!A8U}P{?2Q<%k_G~LvhAZxtkyNdi35U?vb{2lzb?t$)#>mL3im~Hjq@fc^oGxH!hw`9Krgi9LA+Wjx28D zVqS%PS^C>L?!Aunc=6Qn>3rhr-RDWC!Z*H=pG7FqqI@VRs}1_W4k7erjJLZ0!l(VU zI8X8FaM<#&7u2FTBh35PJ7~j6+r1e}^o;jbYUAPD{~3O$T;b(6tTUCr&f7?8Iff9T z@bB|m=+^td{LafCmgYOk1Jo^Z(wBo;YYs9r*K1XidBy1_G6+>m9wY5{#3pc zbg*V{;3w!Lw4^Wp7Ww#-gwxm^b!}&lhc&^cr2myuuaOWM&v6lx!(|RIWm8rZ+_X08gMY-r;>gYGeN4<7HM{FGeAh@v ztQLxu7bwc#4;f9A8CWYmUj~pby=%H(?aPxHkyy?VXCEzG;$I_@>~|hyu=$b)Ee$S2 zH}#Fkq)H$Phl3on>RR1?S!S%o0q=23%+T4~%zXtc=yaJZY%gWe8}VGi(m`r0QoPgl zsP%8!L2=V|v2Am@UO+CocXTdE0-k$5T29wM6jYuyv~*m-HiNd+>5Y6RuFdGD^@CD| z|4`bz2U_l)c;^6l=26~Vsn`^{P0k66gY-52w2(P6vRzTnb&T3=!?~Rk)^h*FiH37p zduMFbcc8`Sm$&8q5o!%sIv>*fmqHmN{$b8yd;UMK??%Y|G?`U?rZ&bDGj7ZF5&wad zf0FVlx~lqD;^>v~pQ}f+t{IJ;MXg(QT>Jdgn;wXxD}>jCg>J2~xZNQ8z3Y+QwZi-I z^@pvaKRXhrWBJ82X+c$j?&ip+(L)L&kL1s>YsW5mX?g&>t{;izf01Yfo4M5W$|#%I zG#n3C&8v{&RNzoT1<;u^&&+f3Bn)2G6)ftaSD9p!)Y zHID4{K7w%MwjZh*9_}F?KaL;mz3XZb zl4)cfS0&EHV57Z)5VHHHnlS*1M9kI^gODSmv?y*q1PBv6GB$`mbfFG(?&^p{{&AQ7 zF@n`FviLopsEs<)bR(?in7$ekESStUZ+c5z}$ z2793CTX5~O2O<9KXz0S=BG{La>u&W=na)w)io^)Tj3e#yn|NNH3cOzEKjwYc#3Q3+ zS||dER@=K4GM`l-p)_;5Aj11bXOLh6-eo-ZW{i?wqG1PXf|oCEW+WQeQz8(1->RHb^+kWht-DL{ zRi@{u%w+9~KVXO_l7@gmyFSR=ujsa*&`WSqJ*&Jsf4^zFYY=ZW<`7EFXxuR6zZ0Ej z|1TkT_~rhOXR6DUcF|u6lTKb%UwCc+j{BsfpIdIE+);xIKK{(?IHQx&IvV-uRud*6PBJxB|U1C7c0|8uosh8P9QRp0=FS3DqFY0#<Ze2&4@G7quvDJ*WaGcD}B$h2^dhg9SDYL?sZLXjxeGKaMTh#n{9ebrY|Zz-7O z(x0G5>S53MaQ#C_mfMS4taYiT8`FlM-ms;PuqD{anAYd4-JszQ3^4Vb5)M2o}_eN(XzZBedTwy)s5WC7h5GAcuMuo`equT zG_QSlctodPs=`zu#4Ay{yxi1KxRY9#@ry`*iT*PG>D$|LP20!M;sSzj0%o;~#f)#lo~+xr?OCGU^0|&9GEK)Go~ZF3h+_6U z%E8Lmt46NH5r;~JXLf)0?)}EWo(XbSIE&PqRSoms%B9+Wy4?iw`BcjHaOPhx^Ws<3<{DtvAl! zIAbLaY-Ewfj-e)40}SDxuEFjp{{}MQvw;s)&e+y){YG_37Y#Tzz!AX8v4cb7tYst zz7s4OcF#Qg!2=Q5#QpPqrg!nbV&@f6Wr*1naN2L5Pa4dU%z;5r-I@nW5u&Y^7cslKV*)y-m$P&MKFF0yUi>a3d$v72R`jCC^cZs z?3{^LnpF3?Y`I#A-n?R59YEKPkss}yTOeG;AY zTRN%&Oq#AEs1&;`_pvEOW1u$it#ZcaW1`|F-s%2&{qlj#42ONjV8Jd)p`rG(G57@3 zKVFw^#RcLO%ub_$YyFXSyD^RL+ko}FW?rZB}dO~Xk|Vb zEyQ8xII~jurbCWTdzB967>#XPR;o8#BS`M~i}2S&EBy24S)_v7K7;mdzM3;lT}>t2k@A#y z)|HIYC^4BO3P{aF_ZMCaV*YNm!ea-kbhss`7UFX0L}ByZv@fe3u$&%#Nq3io znn3Jd@MFTK%S~iW^|pAu-17{UZ&&H`utK){qv-eU!l-1Cev*oQEqvfnSP;I1^$B~p zaOn%=hX6RuTR|{KdaG@4zdyRzwsAYw{H|}ag*R*JSlQL}FUqbms2bnolPk{WgUW4O zH2w{wZC^(0f7xf|RBY9<#N@r1a5JDkrp%3-Nl&WcpzzoGEVfv!SI9%gC{Hc=>}aqZ zJ+KFjHc$@d3SFUt0uwdlN7Fa9QU#>0N(>*8wEO6H81(#H!iMxxD*)1HB(-ui(Mw=1 zgslk30B@$DMxEa8rXJgTEMrDV{qAgwp1G39Z})LVFN!%aZH~k)H2F@wF&j-s$^*CB zQroMRldf{Iol1~CfO?R{Sb^G^)gfjDpDCLNTz=5g0*_**fuy|1wz9xlCB;&cTD|Ik z%USO#Uy_-v+94rn%c~*oD5>9AQyP5EOJJ1x zTtlGI`$^%c1s7{rv?!v2oKm$$$|TDo@H*q)K>8|ewO^?>H4I|PTIK9Ppn;X+nx1lQ z49|x@PiA;CZWY@#`Um+Oz{SEDDPynjvjh4>gDxwxivA;cJSbfCe3VJ#QM2EP=Q8X z{k>qAkDzR5lRIB6KTp8Iov;N$q2^QtrNA9#o$v7!l45F(0Is6p{o#~VnybUd> z_p~1yz+u62h0}?1+>sLhzau5}<=mOB*a6|pm0AB%vO_odoOO(pwD}k59@MV}(5 zxAKxC!OP<4o*81fzVX=1!-wDNJuEn4**XKKWh4imuDggch>g5T95fi1=Gpq2f&&Ejd_H-QtMlQ+u2<4JAxO8N6hI){aiqffh*x zz{N!`CoAhcw)p}3;CQFiin~CKVx9VjhAnvt0|=a2p{z)O%T|k;iiy8-7Q@DYLdh~- zb3W$ZJid>tMvFKV0=CL9zP;Z>?dAbq(~KQEDcrGliL!EIju{?@n6=v`2fjI8yQ4V7 z!0b4$Nud8u2*rO^jm5JqU24*;)Qzx8$F zoblNNZmFfkn3LT6wr=B}wJ@^N7Brm7D_6ax7tG4H3fzcQY|??H+^*824V)B`6AC9Nc5?HGTy`Uh`%!6rR3HSG_Y2gn4UzB?IN*2o7jDtcbF4(L zR%WLsvF!Zk05F|F94A;?Z=KUYc%ZSQRidpOeV$y$knK)0&Iv;A)d2sE;VafzuRGk4L(8ROw9hKl}2a{GGub)(`s8#-HZ$+KnA>v3SkalMosJ#XMi zp*qaSc+uh0vu=D(NBOrMBSIbzo;3J)c(D|uR}7&D2>zxQc_%$Lx*02){llVRwxHN= z7FaUXD#;py49LCPzw;@E^tao4a+e?hM4Ng+*DhQ~gR*|Nc8jj0!vK{X_LVE$cZ_oq zJD#ZK(i`0%?@Exsoy7^4&G$HRht0sPII8$WLzi1^FWMpCc1Ad=Ca9WQ4mG)U$9X%U z(JA_H&*w;)iP}r90p;CnfUCQv%pPBP1b5z@d%Kz3jO5ol?fzf%PLkYK(%2 z8-X9sQQIct082mhlj95|`+tF1sgd-EM3+Uwa_U`2oSalH{IQOH<86S}daiI#@VIbYQ$iTZV~#R8%>4b;)H|$1*;B9zVVm#$&;Mls zd}OuclCMzj9IV5a=C}W${v~hyKHLKw-BAb_eF-&sN_aZ_`|u~xurG@_d!sqy6e8dY zT^zm>TH6cG&f3PC-xf>@Z9dX?O${<6O935P&Xb@GZ4RmcweEAbDl+>|d1I z>cmBom7}tRAv9LyHQj$}QGJ=J^^Z+XOl3{UFog*5E1eZ*zS6L#b@)-tqm(Ne+8&w^QV4gJBB#wFuvt3%lD4URssz6`b78o)G~^2Sr)!aTEW0Qx_Q&|x=sjPbyS>DD93iUM;XY# z?()5gN-U^B7aBn3U4i<=27To{6ww?}K%de%=4maSgjzVCY3#uUVcWulQ6Bl>)mzz0ZhhPK{fev18YiP~R;&Jtn~gL7jyzCC zY)6x*A^6_o@U6T=f9hB7ypBaBkvdMEif`_Hpn*7>g{kFKuqnwa5;2r+OxvDAXz<6L zrJqQN)C3i|S*h!Lf9ma-UtMNNoMzX^R1Zkd@~>f0B`xWobL7?Ll|a~Fc$5>zKB-?S z&XbtEAisL$K!0*gAF|e_X;eA+W_%TR`BrRLLE?=Ko6ASxO35i>w80f_k*t<>WA?DO z(q5QgWB#1xtGhW}pczk_{a|G|?p1=D!8PK-NB*0y{OlJK)y}^gnxFu$T+271o_D*Z zC3xFpq0!Wv>h*9|TGyK267`{ikS-lv&4ZRM50P?Y4m|A>f2Hj=xAq<8*cq>U6b;nk z!9)n@*Auc!!qR@56^c zwk~mIx;F3JEyNmY#Tm!%aRZw-Kg6?EBwY;Q4b>vfyBGpuxstIwarUEs)Be_ za@JD$3fg2sf*V!>yOmk}8yUl~Gw%V1EDe_b*cI&(RPT@88q8mS^Ltl4UCY?XEJe09YpOuwupaZ~BZDLv*7UeTLplMDM~Powxv zJ>I^4t$w58?GyT&xTwo7pclW2|HxNLJ3g2VzrcxjyGnrVAX#&3V+X4ZR)xc>jhp-9 zf%p)BpU*Wlk0EkNx`#`5Yd`{irr|9xbe?!kKtK>UTepC2sF2!Xe$Y}AiL)p&_!a)r zsR7`x`cxE+DswPhqQc87*Q6anzhj$UUTQreirkl%S<;C3JG47Y551eW%IF!+?$lMV z3SMH+#yI#v{Gb=FxrtPDITv-T_7KBa_730q_0qSfsD#_ow1(-ibS+aWxixcS{`Z)} zexAX0ULYiVZ{1>UE0H^p2f*9{6|GeTcfG_~7O4$C0-&T{ieU8=+tm`IXY0SLedNDV zd}}IGMl@JQa-{VAOGoCETQ5@D!PV6dmVB{dJ63M)&_UTjrOnWb_|!8!?<{Xd^L6Cb zvpazUYv)^sHbui2(c}#US$?Hj;O1{XtLk}Lp_FVY@6U*i^X{h;5wKPXC<`j-_c#Uc zWUGJ)P6WTB)23BaeYN!X`@vp=7wsx642q52=+~_3gnm;OM1J1_6+1~+eZ95&ihfVF zqxD<@IGdR2AK;PtaI{i0I4=O7HX;4S>4%scuR-0INKd3mnBFT-*}cD6Zj=%cci31y z?qoJdmwK55u$2W#W$&sWXo7&?S%QV`5O^^0`E6!|cMLMRl{nWtQdS7Tx_k(m^?S%% zGEtBO3zb&1(ry!ndq!_cHe^tq)py2_AK>1^R}6MhXuoI!@OrJf(#*ov0=>L=^JNoT zN3qosji2JMk0Vx1gkvrts3*%^N3J=HrrDbeMSJ8-IlbW?XLWaueDgmFIa8kQ`d=C3 zGpKs|zQ(Ww7S)I&8rX#5ccmRx&&k5xPnF7~Q;S>?enkbXFSBbn2#JB`ajVO%Rr+{+ zm=s)LtQj;B`X}AhEUOT@qBE7r138GIcWN8)n-!ZX*b0m~I@eJm-0S)bpFUKrer;>? zMQzq!(udKnJ&pPa&!wFh{L~j*@>BAj(7jBpTu_`!rXmu&(d=o^E|ExT>HnS#);`XB zA2)RFvYFiVv-bwNV9wRj(mt@Fl7ZAxp1^L!CdP#n)jp*GODAi(L0!L_50lkbE4{vs2qDd}>KP+fO>j|Ef@# zowlT>3BN4Ty?K^htR~{3j+}^i$)a8^*0BOsu#>z)G=p$qQKyTZ`)gNcS~`DU$U;$t zih{TPI%4F7v-Uraq#OGKt_iQ-F5#~#G3j`t7g1isaj_NV_R8Phgf&@}iUfb{6F zE6yz@bCP6^rc|p6pm|?l#UKIB&f@|B1o_^v_Q98mn5x;+q4<*nko1yIz=Pd2%=HLA zuffFeOSw%-^K}&o(i0r2$A6!Ye?qKm(dD5dbRTuM0n@A~L8ei;!DESKb|7T zc=M{m_HA`1+SC7zG)+q|t~zM7Uz$`X)A@6`BkvpDA~;TTn*!xhbH@3BC!o}e7#xCs z0C*BoF3^(LOPsl3td12wvQ1k9@ojxj%4ipXKa6#Gg^7wMUn2vk(@nsVqN3A^A^$Nd$;!@vBx~}PD9p{dClj3lS za2>f|`Pp)oMWorD`xqnTr2cPUd$LPllOj&+S#f&wolT^a&zE24e=kL=oRK78Ep^Nq zO3zKZUpSGCz{W^BY!~~9qy;m*{v2&#inzmfktV<{luGQ+HrxYS8}qX;~vg=gwZ{%Yv2178$1%F zxdA-ASb1{mUcSEMI|d1N!}te z3>;n+s1=@{mljq%!C5wosr1#*{g2Tbwt)lG*Bs_S#)*s`qPaz@B>)$uj-JWa1~5v# zJ=r@PxN8LP^r+>jG|RTrYnJ-dk5-wm0NKl)ozu~l22Z4PZT(=xNK4iF9?$P3HtRN` zWHWi+Rjr}OmV?H61^Gf+vSKiNW1|eN~$Bi2*Ni`^YU0-dsd-)w`O0pPrJR z1m0DZ3#HB4u$!wrOD$^Wb45Xfx9DBc+qnCOLSE`%l<6OK{jrA^8+`s`WR(Qx%^g~Q z9Ww!3s7`|vrNypbt>?)VbF_bovMdBq0qsGURn*gGz5_5- z&lUQQlLe8iAa8={!wQLxd->$R0f&D&K$1)CK3bS;G$TWOwBvSP2o+I>V36+1?s&+_ zSF)C?FV_K85a;dJp*_sct-SSmoNe(s8-n+7Tpr)yJy*qQDkWiqd`(%WQpXr3=WTQJ z9B2oO94$1p2@Yq@yhq<9-!6wehTQ0aSxCb6qp!J-JsW#g6$!u#|6bkIhU(ZNlPs@C zJM$XK0vf=L{$xL^N3f2<;lPj3=`w2QU|2}e2J@IuahGg;Kr@s=E?<20@9b8M% zIPJ8F?bh0M#oegdseN1(WDmP|PLndEu%jSA+iSLRuFu^NhZb&IS{e#Wu9=O@FA<(qW= z!SG7KH-Dv`_A2gQ$S0l$W}5q&Uf4BDSk6z}znv_Scs9AUhOPCsmFXnd191*|oUPb+ zKM6*Nta|psdGf_xad~5~vmaLiE0%&{?w<{#ykA>}87`eAw#H4ce{KL>obq5%1G7&_ z&!&pZMh-y&zL>Ar)z%J|QQ?4wIr&JoR~+$&G3C-*85#2Xp}T{1FzUs-ssZH*zG)2^ zYxmAH$laYIRkaGN((rBgHtt2sO{(u$k+j@X`lW!vOflQ|n=-}Mz5Ux z6wnN}nI(VSun1EW;ok?3^!Z5(Uj>IwgVF}$fD88xe9u<2k=+-l5|1;z6jeF1%^Ww< zxfvllJ)sEgHbT3zWXMOr)4Aygh%5KSD2k#e-|9YbYMxG2? zx-C+Z$?JT?HDvkS3M)|-K{RuHDM!C@af?5!zC@#i$vmA+Q4;tsz0{FXWG>)5&@R*U zea+Dpy*GMJo7S^&TF4B3b2&j@OU?(&C#5xkYH-PjbNvhgZTZ7a9ch}UefSilIp5)s z*#mp*v3x@l8PGbiTy}7sSW? z9AhP9ixrK3IK*zel1Q|`Ukk#0T#)I5b!sv%L%slK$cakxCE3KJ)zCv1&1Ej-l)CPj zR9EB08{Yu5Iq<UX1lG`mxcZJR#O-DXh$_Kzoy=FKt_EDgILMc&7W7XA}luj%_iY zl$mHO^_KFITG3#HQ}|Z|%l*i?3nwf|OS@&o`EH99i>g*1skCxBRa>fG01{wBn-!EA zX(KZ8Cx3`R$ecpkAfiGFMA}S8Kv8c>&RkI;?2dE5_4-s5PLbdT>=Df;y2U{i`Z8sz zS$*Q7W&+M1e1g_!oU;hi$5G~?faOP-c8U}SRDd={~sU$()6D*7bBCRN1lvYlI;Bdhf!Huu<*-Q#5Av$D49C#jS|vi|-6(?~EI zsOCKtj~h&Cm^z>Dv23g}S|v;_KheZ7AA!#NSk;+uBCIK2S^SX&6Nt;m!ZbxF?xP$hEVEuKIey8r-^!Jm24VQzx;fzEjpZH8I7gfA;fYiFYNuX|KBSI*w=1 zr(*i^&FhWL>;0HMF*K*dn~bR#l$jn}Z%RUMe8zQ{1$IE^PixOYTIOE=dF`s2i@_f0 zz+6WIGGA&Hv?lDob`1jQc#SxR{=tvo$Gmv!g!P_D!OICkz<`;S%DikkL-rBX;hNTy z8|g452TuMNI-72jk<=K!adI;AIKI<}q)fb@IH2gjLBFCudL4Hfn{L)verasgrTdgbpe0Onv}xs9R5kPc0&|4TXj*oy9Vu|&-@JztDUa;5 zra>Iq(O$X^J=t<6A#9;)8eNqGt%KH$^K-A^uQV7a-seAbRw~!ZrbS%isysaTo27;D zDLsX1vP<`cUF+_xwv5e7Cyt85qvH-zRKov^-vv_O$Ko;3yVl`iz*)0Bd2=QmF590c zoW1K*#<(JeM5xn15G|TT2dAv|v#mSkr}2~pm_3@$hPHmybV$urm3aVAsb*L7YS&*x zsP^XPz=;&v^7a6<3EK4cS5@PM=c@x6Jp{s(8*~@yydM^qT0T*97zR%$frnP!dW(pK zr9N+2d0@7SdZc^Re&3#yp_j;Q1ioyqN3|&T|M-o|DwBVnVB8TnB>~b5GIxE+dX+=2 z9O$%tst8@Lohs<~mp1bz%oNh%H{gW68ARq6)2t(8aeN*K;*1efJlWi`)6o~K*RZ(2 z$#Sm@uZ)$y`)YuHMJiv!=Fhu1IA1XiErM6Ax5@p@$dDSV0EC#j(%rwWCS|8C;-QDU z&3>dLSk=pOOdq%tU!@8#fopH~znfwX2+5=S&g`MGGG3AUs*P-$+3TxzNwq`uz zdlQ|jnXelE46WT3o{=O?Z5)S1u^7ez?#Z6)eiz>(Hm9fM_o1#vdfm;b@ zY+hd9>BU`_A^s$`b|rX+@u4$W`Tb>Vn*NK|<>A)2bj4B+-9UnzsOG^|dFg>94s+1c zgMBO;Zt_&I=|6VP4AnCUx<@d|f_=XQbM$Vy#Fn#zZIs&{M2Q3B9F{Nd{Jyg_RoSZF zy!v;Aym&S7kvqjA0qr_4nU|W@$D|_8_yrDVo@JAa;2+vMCcIX;1uJ3xwI=6xz;dFk z(+6ieR7&lJ7cLw-xUwAnJuue_*2nLsGtxP2@Y!BzY14imu;3dp6ARf+{V!-s}P!@(g{PN(&xV?)C}!c<^;wLS-j+*?!IZ$wBaqH$H@8Kz;e*gq6Q@6SuT^pbGm z#m|R-Lo;+h(-hbGz16Dl-$iv{9R;45l1AyK_g?WZ2GvR$S^G=tbL~ ztT(;tKxu~_2~eFSlT%(@SwkiI)EtcqWR>S!em7T;d)j_^@<#U(b?QC5z?^P;)A?m? zyi{9XhR?`pM@aE^B4?~=Z%piGK#!Z&1bb@!Gj6K5%=GriV1YE_b55>S#@I`roo7KD zCtKz*^h7V7@b|4)(qJ@lcLZ-WR5Pp5sN-$2;@yBHq{gpneo?|d%V2x^n@!<1ZJfV( zbu#R$Wl3TfI}c-2f1J#f$~)hpp|>?Zy*xw;l#&X=pNvj&R{aiji)niiZz*Z}%I8nS zd7ywp1^&%Aqs{2bu0I!<;T2ZX*#^;kIETMC{K z76GP{X_jc|JZV!A=}d~8oyG&@39rLWR;!{?^;>^3FsFN!Gd~2s?q&Zf)spox$paO~ zvlO1g_SqPNP;6ZATxLyziZsW6xTnDY{#~)-+Exs);nGZJvF>_ts9P7OaQa`pL^nq3 z@bep1d)iYcOfvc$`d%C?+)6in+*j)}JL;OFPs6?07lC4@awd?-FM&v3v9}Dtye#)x zdeg5r_%fvm-A`T>Dx%Uliv|usv#BUqmnVAFE4q?;?h zTo+CuxS|sp|G%OWiDD=tCD3BHTk0LUSW(o>p{8!Ahq^zRveIpq` zYDdCSK!w3fU4CFN#Rl5^ahDKC5ZB?<`9KSSx>5T*a?nl8PRC7v#Z+ZhoL#n`6UuP=M2oo6rLdW- z4jj#DE&fd&LNhYeLBTf8fxka(%-ut7-WrFiXeW@0vPWJHV%{8@VCXxUS<7p|)wgs9 zj&TF-VY>WzEA|E}1gTHl z61xlEK5#k(Rw)SNqzJEcwGhgh?8EDXcMs4l3K2%-W*KX)hx}vFNzK|tI*+ry_{%gP zDTbYr1NYBrQEY~`YzR@kNvd!^uoj75l1nY2U!rqssjs?4M=2(j9+j+s-mppirZeMy z4pW?JlT|$ktoMC8#Ve56lZn^aAWie0-GyrU)fR{z1T7)$#Lptoab&0x9#h3hcvI&!o4iT z5a$6AqP+~5!+BEqr%&5JcNScDb63xQ7|-}6Zr5?O=Zz@H_FI%ZWa4kXcqW)KatfK4 z)l4s3V*h#PmIxD|Qh9P(ovycABhiT3vf6)}xVo>#^eJ}_c|Fo`P?nn*rP}wanyMmo z!0EC;#zw!4O)#4_Rxz3q?rs;Nnpc;7BNHzAL@~fe_!NI^52ei@PB0XitLKwD=)#(g7qBNDK4_6>^l7ocPjV6_ zwTLm(OLE&=mwT_uHOxdOL1*9b3cFaod5#j6mt#_6;7?9#z|HMuZv-7PubKBsKHPim zO1@y`-;p0o>2J?Y<*Cxw0R~| zzZ3E<*25Lq&^+zrLoIQOeO{1pwZonI#2tUQSeQ(Xi<**mooTuM=j=dIYmfVk328Wu z_=^iN?)cF$9qKk_YkMitVQCWNmiZ9&>0*X1@Rg_z0uoFwP;8t?m%q(vUVA(wsbDNp zeo=-dF<@c)JbYcLWPddO&B=_`c%*R3MUdCfU`qJ?OfyOu@OI5gvSw&~xxBWghd}F& zP2DsDYt-T7x#@R#)G4ewRU`CNfoM#++goFk(| zxhguMd{Zoe?>Oq=OHfaRr4C{PmlL{KGncJfy5KuyIx|>v(JZw?p?uS@c0bo=p>cY3 zdGi0;3=0&l%@B}gKIc|!#kcTI8|D`y@jK|ZjxSy31#g_uq+!dWKd)w`&#G@9mPtqz z$WjFAP-eP~3^7o_fIswzQb(U=eXxcezP9w;OYs2#<+yaAo*05)`Bop{TEyu0JWbWJ zXXf7lBr|XNc2-i_fN*QeN5I5gDxC*Z^NNsfD_JoBbso0X*&cau3ern<&iHEZfsja+ z4p)u77AN6*)zI(%vH&GUsrsewqX^8`Qdj%^UX*xgf6U#d&W$LR^Po{;_OmHvb{uWj zZt>D1fW+q=NkMf7bATW2KDNfv2S=v1 z6)9*-YZYhlhXlMZw$aiD%<^%c;MP}W z+0(gtOI}!ZWqIiGDc^7N*?up4Z+b)3vYNFrDu=(yHLe+Ek>m-Khet8*&#VZ-1O*np%q=GaZ`V6V`Ss+yY9iOV}c9gEwR*d zPfQh-QR+B%Mxcd+5($!lm)<4fWV-0848MB+GJc*cjEHp{G-gU_lGA>r_rWun^7e~n zD|TOtANBL9&_^YEtM#%BWZ5Tt?+$1>=y0~apPZPDiC3vsl_Z$_GX?dr7y@fB)1BXX z!cwUWdK$yT4> zH<-nai^j&@>Tpj8qbPjnI~U9~P{=1y>?2jd-L$SCKHVv!iIdWe*V9itKH&GAeWtr5 zt?5kHv~kP?s>>_0N*y+sNQ9W|a`wcaHs@DvH3Ctj2Kpy6&^27-gteRLRI|>XPu*mk z+N_V>D5x`%Ig!$F5-dPw(!S`bIbjI76b;H3=EId^?+G=K9#q0>i4%70cWb!HaIVvi5Dc0_r=aCHc_hYXPkCeJrp;b zRLuPaZLOEcX0J8M5%WYJ@iX?@;(yl@=p4Os{p7@Y)#B={N&baz&~=;sZ=aj5e4F|r z<{w*yELW@N2m0&38-blYtC)*aocAf!y1P0p_zl0*S>f9uNyy{5GnXmGZ-!I_f}W}t z2W8^;oq0vGN*rS^{-XQA6~ld#Ep4>qbZpl1HRr}PzFLae_JCdeI{Qq5)PP&3T>-An z#;$_&uNFSLo}-EaHB-`#J*il;kEFNq!3b?#=_|;DQuFnmTqadQG$1nW{=4T&l*no$ ztF2a#8z0o1`<4?=!}x!FwG3R(3mqv6Xs=~W88t1Ps;_{3l{IzH;+L27_iaDz#jAG) zG^BkOsRQB{U2h@e@s3xR&*ocoXcj`}FDjKF=x2T&iL1}FXGnXSB^~=LnmFIxUxZ*^ zM4?(vu9a1`nG~jiYx^zrKb%TJ`GxoNce}7Zh^$_<(sBxsD3!K_YJ9IWCJX$Rd0Vk; z`-oy5o6-20#_5>DpSRbBORFwJLMm9w_T!Ux`DNu#EMd&*rC?aSQuaV3x#N$#g6H&} z#vA9wCD-9rk=Li~bT6{mMu1>Z>jknI+CGz=JsTm>blPk9b2y(zQ|WF<4&&zVONkDr zUl|qH^bxIre^>JK>iDK$MIXO>dCpz-yuiy@>(L9A)`Wo&?j3#S|GwB`&7mFKE@V7j z+e9aL9pw*>QKf;fzfdHq{inup0xu zip=cy>5~=S26OSK9Y5#%eH7+@Ip#^@6{odzx2IS?^l3@mgiDw_0hTZY?t4~SkTHde zWlc$VoAC(*|9P|7oGAYR>$5~$_nFX&@T1<$TLKl2vV^9D!5=~C%KX$gu*SRM)%1l&``q7^XHU5~* z=v3?Mh{knWCkb>Y^V*lVU<`gsqRW8Mf<%A2h;Hbe=pgQ2oKK;BajjZodSUXKqs_P9 ztO*05%`^gUw(s`0&Lmm5KHZ}BM2@rr>OC>AT|0CJ2ipEJ$FZ^0yzdCwT3TuS!9oR& zy^ns+tSJmO>^WKM1 z%JToQFlMCbW3F+c6LtS<2xn6?7=|xpGMCx>L!&hML*rhNV5?o;9s6g38r@}n0cXnN zDshoRZ6^+Xu->1+eo4fb;r9ozFSn+@iNSO9yc? zof+_+6ShPfnQ*vUA9tu`I{XxPpOuf25T#Gu)WYZX62D(SBbj$3L{&F5PA(6F=}AlX z0YF+?jGAn>khamp&VK#NSLHo3UB(?P#6>)lXrS-a>bIb^#g~(dfVP9`iB}=pc6xT_ zWrB}veKHp62h6a8lu8K@ra5e^_VxhCur?~3)q2I zxS`5bIcfWkzJ*sA#45IkT!Z<9#+b$!l*&8AzwpGe1kOvN@K`a$3`Rt}hc6d$M>Z4r zv4cY>e7&RSExB$XfPMXOfCH1|a#>dHDbwv0&3wd^dEf7c+MM%Ln_rtj%dslekHo+j z0$J3IrR;`iLaJ#)C zopW_KrHb^?Ia4Rs`sHj+E4Nd0|MKdWck4v|K-fh33qRkP?5LvYzIRi~llHRFII55#sKwZNUkDjWxQ@)$ZT2~7`2Ta$Xy2(PI9ohl9vp-<^I z-nn86yQeH)!%9WJ!CT_Zt7jw4f&HfS$Htgr>*TSpv9LN}#5qcHw8+L6KcE$HPG`)I zBNf*8{U;$*#0NcM(on)XYl=v=hKq%56=!Cq5$Ei5)3uno622E^EH#+!IF@|xeKlH} zU@uW9%Vus;)cN%u)G$ZGz>9WBBu=II;;+xC{89__9HbAWbPbmDl<-pws$4#Hck z95H&Rye|3L)+xwr37s)da8T0^u!Y`$e*0sOc36|oS%Nfl*UK1iHSjUdvH$T_2KZ%n zMNDFmRU+VEnWN4A`@;c!@e#$Aslo=69PFC(ljw1$s{0muCEqgODdD#T#1TUw-$otY z#tDGx*(tadQU>kEmF(Y)C_$X{d1=m^+tf>CyDdE-1FZbG|KouhKs6c9h2#mwTXQ|% z=B^CdyS#Lr`{jCx?02Aw0YYImx*gEOaX}5BS=r9-T$e4fucs=9r}PMqmjdJuh-}M{$2L({@CI1eB2+8`|WnS-nE1EW6Sef zlI%#Ew1|-En^iac#8{`Rk4v~olo*vK4sN3Mdq>=#6v3j+&Tn|!Nmaeb@ww>*tMakx zwDL)p;d21Qig{q>j zUTv~$?Zt5J`&7>sT#=O=6uq|jg!y%?|I|TQuYAYeEG%F~M1|gc`^eoFnTi*G6!wAk z2Rz!cw~0I2xGs&-uG?Mu9xpf7f;m!u9W*`vf*;)1>)Bf&ufCsSYVmE5*(EHvp#n;E zy$mH6suOB&Gj(~IidX$DI_r{WbGK4?@CUCGH4ZYWc1}wMgjWBwIYbspNRU`jQNM?@ zU^zl%s$%6Jwcm5tL)@RP6m;Hj099KZDsEHZDv6KFu+c4L_#qDdl&`a2YzeK5PyOmM ziQK*zLGiR*-uyasj8NHd_^vY|UQ;YJH0^Q7GeC@h4i`Yzm2BG)70T$} z)i?{tw#@RTWygwzDtJrgC&(JP1OrFqoKF9amp{5fQHarbz=#wgde59olChzsJPG+g zeUS(QmRoaPoK;)+nBXBhU;1Z25*=3X%^xV=w9iP1ib+)=g?&GK5;C=%b7V z2uu^eblT{VRp+HatruofT}j4x5(`V8^lS>r8%;(RI?MAN1jg$a8}IF|j|`EXHujUY z*>k;B0W%ZvVN-Q?dz##qHbo5AC^#Ms|lFBE$4K@+pDKt!=f0c3kvrxh*R3@6;Pa?5aF>VwFES<9eQ4JF9+eL^4QOs zdd8+ydq41}zC0ZU(U56mwV?x$9^|QlE2VnHv#YrTZm-CbQh)$D8k*DOists=1zhT~U)}nK#~46O4;G=jU3>r%MlXBe z>^pVyWhaB^l{OQwgvQ3#K?`K>2MY6Y6J@7bcs@{DI>&ArhKzK z+8Z^-y%p!#Bakk7yG^;B@DTg6yM#I7wxAWVT%=aoPC0V;)_Gn07b3B=@>c?<#;$mf z3NFMV;@M%>?e(X4Jj+>aEwAujg&_1*n056k`R_7ojkQHhVqC@y@J3f0R`DuE>{s%z ze5N-n-w}V z^H5IeqUr&)Mc5=2j=m%b!fX90*9xKp8-Rn?W=dnr%-=x(;}?o|3W-Ih29vBoOomL! zwLeeh6SIMj!ZD4_xdKT{#IsOYO-sinxZhTqo^I3cf4UbA>t_!(^maNlcfR9>Ekgdz z^gCT=|1|97a7Y&+s?cqI?-lY=HLFEmMclZerPZtdjUP^HK)&H$O{<=xzwXjgblMF> zb*0lmhP3<)2c_@FcJGRTXc_jsfth8_aP_w%)!E=Jre=Op=zZ(CEWTB<6=>_-5~;g5 zV_HpIN!c4QsS2l%L4ElKxar2qWwkY!n)ZK!(D{kxgO(VMoT$a+y0>P!&hO{HKGp`X z311-_w@>dc>uQ0mqg=O76({zDlYZHs8ogHc?OE(EkawsN*UFwjwTf2-d8_W0v1u0;4k^okwdOUDc18`Jf_H)Koj-pYM_hP%LrY-3gBy8h^jk!Z@@ zuTe&srvF4mDCLl=K7$fa>ztpYCB)no@-h_bYJbh$GOPBMksUqL^zZ0%?B3kp-nnim zxboTko>}}iF8E6KG0)SEBHMGU2z95W+<#k^>V{Ud$kDEg;v6!K;@(I>sJ1)zWIi-w zZNW`?vf4D^Q_u773zylZsY{`tB^u$~Xms4szsf~{#P<)Y$>WCU$@iztj}H?oBm^ZX zIq|?urhOnd47IFMoOW z;Y5e&q_dTXWd=ESupajflqJJO{P?k*zF=j1kZ;o|_iRMmM%_SIW5y&=?D^7bYQB-- zWpV2-99*1IKykInqhkDyr0e_&JGTQ!ujZDGFY_+m_~;+l^ZVmqaY=Xj@JI?yFU{NE zs{Kg}^NB4B_KVb4k8QjZFmXJnsCv|Ngci+;oG;RPCcpQOZ?+Wb;FP*iY284qSK@Zk z-qT=ak<7LK0wZA8wJvWb<#GLd@0@M^j6#tAFALL>t}HF->d^ZK^YmS||GZ0>c`PT^ z|49?MAMbgEwwF?(@EIQrXf5!6NG0)Pz{zy?)kz%`)rVZEu-8ftjAp41b z{O$X@8BygwsFA4=5+NKMP+?C71(~I^c(L-Wj8p@cEipFY{SBO4yU#rgyWxlAt(v=i zzEEsx?w2A-Ln@%EjNN*cQKWZX`)>CAm>H7|do(O+X0OET zvFO95ZObNKg6iFiNpeNsAabS4Y&&1f=rN$gKQ)|gJCK(Q`V$)?iCVU`ZC}P<>QQm$ zSqTz;Zv_X?P#b9GbZfClfT%s&SaDA9(K?mG_`t)q)jFOLY5v|`OcGYmn-&8oh%tHerE z;r4esk$Ulove5VY@G}F0e==<-F;KJPMG`4?1b7;q$+49sKE5aKvS(40XX$%vBV@qF zU@CuYh0lly?@#x-3SxrVq&DZ?2l#R%- zx3qy`1uM>WfJi4To%n146CIaO5p#T+j$$|R=l{~x)z=CD2StU~=<3$-pW^fkG=GTb>WwIF7`@^fB3x@7G@@3_EZgUTGx`>N9UIXz|;pg?Y;JLQ(+XL zgjVPK&Wk=zBPq!PmebL(VTziE^dtrk*Y)?q90_fNslu| z(Lg2;${)LDyA|IdT6BwT{a%JQ0yHFOHkD&E_S#0fUIP9kOeVt3D2yMk_%Mhmh@xHF zRPn-7qpGW1u^d{kUUx(XK$7^$RYztjQS!U2=-%aR=hS%VpV$7dcj~}0eZg-tSL(bG zmAb)Hi6gdYvNv%UO1T=Hl*Eh%{kWf$c)vyI5=34C6X_{J3<_l4LMeJzq;Q}Q&HgA- z3C6X}>+s7#AxYb^d&^x{o~ce8Mw2GVT!{Sy0HZ>P(mzsZ>mo~pnWZB)6KAQ%jxxcM zRnt`mkE@TPlgn$tcX48(tyQVVYvo&8iq@8Y!WRa7cM#^3NY9Lkw~@piQ6*O6Txl8T?S1+KG8|xK*bNd;5XgnhLd+VA%Fy_ zMoA}%f+#F^$*&>ei3NO_coop3kiM2|RSG%cy~rsN^%`gcdGh>wjF`2$e)l}E@;d4U zle2zcc^MjiEA2FDtDVhuGhcu90C_49zSFnzuH~)LY3sT+d9>TRZ=Ocp?i`6b&HWW0 z^NZ5Ozv^&E|0&srG&1;_`HB%@V;~_SgHZjCd{2LrF+lM4dSAdJUtA%i+F&es_-$rc z)pQ2Y1S6`8o%TiyWjt1Up@s8U8y8m0u)K1wH*WP8Kx1zAFXGyu;nVn-g3&n)&SR*m;*R7=WG(I%7HR3bzsiHg|RCEl< z9&4A%u%ecM#9nn1KkgJG+aHKOMDil}6f8F<22sD}WRs>B7jpchG4Bk3!~QfM3b$Nt z(urffYr*~j6JM)xHw&zY`?x09OAD}Kj{7yhk_CEuWB*^$Uz`;7e29AViuTW1-y7K5 zwG;KyR>LCdY)`mZvRUIHay@~2BE{dj{;6Du{0uce+idR`%s*`is%NXxJqi8dA&}%|4dh3_cBgzrZxth)9*} zl$DUD`0gZ^znnZ^2d|Bd@yj=emmVO0b(XgnhC=Q6UR3ds3d$js^I3&%x4Jn;7dRRt zAfFqVqzad$+tHT3LUnJWuJWrrH+C;z@>0>Q-EtJebd(CQsp_g$10kjALRbEb zB}`Dr$-cT%_Orf@lb$5hb>iS@CXzo4w>I(J!GdB4nz8kDd7AGgvpOK7PZII=GyF;w z7s#qms#s}q;O;lT+bP{!J-9nL{-RVk#zTBY<}p2d^?RUu_E{_2B7O;zV^I!& zeQRi!pIT{ATWIJj`N`4Yuu$TfRen0+EilCe@4w6mB#?5zDn)x zOA3h9bC$Kdx=STGnYL*r=9*YYX6%-$EPq4F@m;W{tPVu1@YqAO`77=t7~Xx}9SGN) zK}`kpf{USL(cl@eWXiZqn`g02RFv1LmBRUSe5c3_pgO+`4tQAqD2y_f98>+8RTNBn z#E*p)j}!+Q9%SFLuyeqvlF{+Sg`{gZhbz=`p0Xs)hf%6rXN@eUAQuOXgX~P$nx*C^ zpM~5w14I>pFS%-q+Isgg5}9S~%(K#Q`K?ahy=b`1Sn7{!dy(<3F&|+)#(}F^o zY$ANpi*!y!?^zSdPR(5A6g+Qh2#F49(}#5M1&;pAipSQRHUy_B1RedKBYtP5jVn|Z zk2Lmj*9YOlN}3M$s2cr;6*?6Qzcp7?Q8B;Y{MHPT8Yd<+%X zcDNaX+};0l)0wY$8IEy+Zk?(=8 zgl?mJE**9awF@KO_kMpI3J>+du73;Km2~e`2{DS-dIAflT*FE5-R2{(hdw`VbvtDi z)QamAP4ka%aE!r*bPf9I-=A4&oupRXYZyKwNem8F|)iwUo)7!wC`A)_MIdiXn0XQ8) z@1M!IcdaBz5Ut7<-sb|OfUGX0Vd42v&%T~|3r%s|&jV+qytW-hoK`+$qM)0u?)m}v zpe`=JXPSQv8ug#6nb&!v>%UzOk+Pq8l zQ}A<`7b}z7I0;b{pi2p#G}|DTH#}4`lM5So8$?ce09xdk12+863*h-@;6BhdQ6{up za*z!<&_-I<{-C0e0smrZ0=o^}hhS8TPMF^jZ4YQGGYdd8qCHu(%gr7-E(PGEszZ+B1@iaXt zr?c|tgaM?-C)?n~9@g`G1h?5Z8^JAwVLoh{(?%edhGgq#^HNfzUzWSc{JLp;fc0oY zg!h0>9h4%pFQ*-d-Lj^skt2?hKNJyemM9?XX#{C>#^V+X_mjdmGbzTvu_1jMs-74x zCPmV1G3a-0j(iF8$?jaVstdq_sigIpc@{$Cj_>7{!rM*?PYT!b;f>inJneR0J704G z`Fhsy8==O0(#RP}(@eZ_6F+F5c`d{m=rym#wY3mqhLGq@#|G2+Bn1LG&!!4gBUnoW z=W=)E5)FVtEn|N<_U?C@HR2JbpAePL)xEMY-jUonLdK1({&I!}p_ad@>#bC6XR!g& zIPqaXe~c2fC6}$`(4c8k+&W?hhNOii#>bspHu&c6cvVD&uEOr1A3Ys!CR%R&Lu`6M z3Zt(bEHhibpb_xlmam-6SNCT-%n#KB+OI{5sYlg2TlIyaGP648{6>22=q~~OPYK!q zNPq7ft#3gA&N4{|t_(Q+qgm*sU6Rpx1l-jsP-~L&e~l3i>DBiu$#94-ip<%-xNxjx zb!Z_*UeXFUNX|S}wP=h`kCcc(ngzCG&>Y44wQjgEaHxJG4WF+vH}E;9XFG$)s3>C; zne=fo+$0k=f?+=AEJfV&M@|zk_?D3}eYRi}jMQrFy0-~8KK!sj>gLgskg-;r`#V;Z z8YR`49>5qUKQwZZD^b{`AD7=eS+#hd{cax4qp6@vTB0N)O{GNR8g^%})==D2F&{Na zix<|S7x+-F%1;prbbiM?}IcVnf`Dye;$f&)i{Oa zmp%NY6aTx-Vvyczlb7O7pgrvEvOet6!V?ficspJD{ceX*1nK)?tO#3PASl~_>(U>H zVHh!87c%=e*wD*N96#co)U2M`7O8W~K&-=OG2XZl31!i*S_ceB2MPc1WIT7tZn)g$ zPkCp|kSBp$WH!6xv@A^<2l_cvAE?eQG1y1U3A-3%K4PD6#;Ad!KPb)0MKdWKu8Y=O z8CBOZ@rXjaek;!sig(xTe5S&lvmZ;7j}dg@;8lcs^>={~v3k#NaQ67cmE@5YUEP!@ zVTrL?{u`ReYJ|kcTA>co2gt$zs0q_YP$@27J~deUP_QZ~V-unD?x zNZaXqs@>>NxPFrz&W*|#FsUIVEtB~$`>b#g7vf5XxE{U=L4#Oms z{Xr>#bp_FJz)TCNFo1NjH3QfQjEs@zl@C=yd)Nu8qF7(h-uXtPVj!(sP#B}$0{(IC zC|zs5L4QGU3cZ0%>A4)>S5Htc?Es5Tj9hLXrQ#KCped)TSOi~IR z$|4Chi6#~EtD>YO1y$%|uX8dueCK>V;&!Lvo%@0}#A$XUE0Mj=6Jaj(6QHSB(s3-H zj<5#YzV6A$+--sGk_Fve;FplrGNH> zAm}5qDeOz#y>|Jmeg0o&TNGsViuxd2dwxf)>92E-{$M=njDi*LK?}PfpKa}8Fb9uj zO&{iKd9t8YvlCA7_2kWxi-)eOpY4tcwvVP=I=U#He;qzqO|_PbIEf8bp{t$?+`+8C znI#rSlK#}h*}j5V@?p&}_V27|Kj&Cz57fpVoGr5bd#5?)j52B$X)`5wm@KWiMK~j)I@V?x5RK#?Jj46C5pl_!Mpz)E2OqqjKDrKq zQDVeqF3d!y{$ieqB$Y2lv}-l?pOEL(+*bNhZ{8B-UYQO;wigU`K5J zl$6B(MO%7PdMVb!H|=B6=z@v5#=Jx*DL4MdCeBLpPX5tyWKiM9+N;fBx)h#+-YQER zu=Oyj%jmGWs~~({k(cptTq~9mt5uSU2D9DyY)D+B_>s?g81AsUZs4OuE&2FjHun*# z*p0$H=tGc;b5%pV;up3)@ZWie(MRND)ra-%L5p{?jmBET{OY~M6LFq~Y0Gxm^6&!> zKho{`WR9spc8@B;XHpB=`4sDB1Zkq7zFcFe}!4J$X`Ny z-l2O85fEbnBWja=$!b#H^5k%Tl3O0ZBZjj?IOP*k;S!+%gyCzCUS2{W&OMNqaO2C- zFCjQXpN|*jie2Sumo9mihemphrLCP@BdIuMM=c7|t-A zShfPU=v6*Cxwr$~c5OGCD0(>%x|Wo5G+Bpe>G7ra$rL903@xvb`mj72*Y$5JIo6BM z1Q;r#y58z;bJ|#67D#CQvoq;w+xgk|qnydc6|WsT9kbaoEW2oErqZlj)=)JT{8Nqw zAs?LD>%mU02k2EKNx9S67gDd?N;OrLGwXagm{@t!Z^16Z&e5mT`DdE%Rmt^lM{V(` z)^QV^b(t2XAoTYIAWqF|z{xqC*P(Dl~7j+9R* zfb0#Az%ufS6DN%JgvnQImiJ!NiK@fNoG(i^!;b!JiXwpLt1^m;pRaqB58~<9LlPb9 zK0OOyo2=Vq8{B=aOF^9mgs~1rk&i!zvgf!S1(vC?WvknhUwEWEjQ3BuAGdTT?{iZj zk~WT|*mcNga9LiUk{w|lRznum=!ln;$I}ScO75PPips209V<^2dZ>~asC=@z=d+}? zpVzyaAP+yED+MiLA-<~Ji84TOm-Sz#&~@Hc&33;?DNq>k(k-t!c-^by4pH2G#S zQb}8?gZt-`Po|-8>eZV>O-HX>paiNL_B?%C?rT%=otF;4F2hZ?1OWEUV27xKkLBiS z*F_r>E&PBdABjREcJqAer5U~|kF_iXMplB?Cw-xEa4)@MHOd9Zt!*i_LXtoiwwA<~Ll=anc2ZTZ9t z9p^lE%N>kVFV*1Q^PO{^yU+#_M6XBd$SWkd8*pG`Um-t8c4lM&r$K+Y-TBSNC7A%w z(^B%Y$V4S0_L&26l#DTXIY)w*PQR_r?~hEscNLd&nz{7;Dv}N4jp+EaRxCLtSUk_g z7^*f^Cu(8O%d0P`6v_sG-pfd>aVHIS@l&%yLAeVCHH=^e5~DuevG7Lvr9wUc`j18o z%vFspRF;S4ye<9@E~iTy0970Ybi^S~a)tR8OmXh8uH)8y5<#(9rueSsXp-u7=b?6z z^r@=7qU*>k$-&>TTKl8|_lH69-o4(V3=n0-CZ$X z({42f55gCpdf@ukwFUKS*v<>nbXC&Y-umn=dbakXj-0q}(Tw5kT(wj0C2{-ugKS=n z(4Je!wGutEh99GUvWhBJh%#My9&?{-{FrxZ`+KbUd)_+1|{!E3e6JzD?CAQ&w&iv3!zI)j*JCZ-R;5cJ3VPPn9K ztj5P{P25R{&2zD-2@A|bVZ!wPi>j&9*}H*0z@@&prBUfck@?D+4cl{%bZ|YP3`Q5m z&O0|hwu>qpzU^gHL~P<8w0kkPzf;{5;++Wmiu(W@2HP)5t~!z)`Y6KNwsvD|SuumA zDAEO9wifB{y+|X^DgTVliAwDlPmfAxAW7ff86*mw$r4H;{$e+ZtU*BWSn;%LvH{~l z{F4^L+MCknoRqFAzj=VPE~|tGXmezDw*0GhVdhW(e(Z1?{La?A)Di312=N0^JveZA zabD8Th39;OkaZmyW5icsqMCTQBkQiFRg`K+QqNd~xcg5#HQxxWT|F}!oQ?QNTnEgc z_$T9uh`6ug=}qTj=VQ+#{OUR%w&$AD#a5h3Y&s)EjP`g2l>m?oAFFWw39}lMj=K#S z_HgY2teMImYXGKLxj4AtGHm&uE}5v^Apsw=-~*BshJqa1*5mW#t?N_&8L3RhI5)ylT<5B~ zunt>e7e?v+xp%qPa@@t4ssM>xZOfFh`@f#tgW@uy9Q$@CN>~nCTso&Gu;_VKQtD$# zuG8Z0PB$1$-Fb0(q@Z<>$^+tsShg-XhKr)e?~-&al1CvJpgFKg2+J#>X~l?6Jl)o+ zqLmh_#2#y&%1^(E=7kwui~T1}$VNpUz?ptE+)K-jRLrKlAt0GK(U2hK`2 zhEU|D^R=u?I? z)p9{zQP~oShAkKH1DVVdB1?|_$kyVS{sI8FFGI3wfPPbJDLDl1&(=zsP11_VTOTm3 z8Rge)CtU`zr=6D;csk8=Gl>TS7!J&h3I@pDWX7GGa@>=$)kstc6`P^LLnoVd-0&TQACO9&gju|t*7n--sZ$XU(qZ!fgulSa{`J6*)f8ud*Vt#00 z(uJIP6tabcR?n-HbsCb&fTa)J!zKZAc2l_B1)+Ya`G8>f#7o@^ksbOq8270EnozGM z!i|1m&cipV(pJ@r?ujN-OwaTGtv;DmzBOpSYVMm0Ica%|<|`j;xk;acn}1jM3Cfj3OBWn>g2V&HWHVkbg{qVTY;&>KJu%99#ww9tlQRZ|OtDoIQ^*4yz#X{w)9$9}L?;P3AYa3u?-~Y;&n&%f=I%YU2aoJ@ zGRAEdHm@ zuJY5!rd)ll>x<#Fj0!OMsu)c|_jviYd=w|st>=Ribp3RKQkM$RayQAUJg}&ybz6(=&02E4^w3jU+%0f@KmK;CE+iwC$F9)8B(C z`K>&&s(j2kG{IHxsh3LNeY20Ts75aQ>!4LH$wTv84Y6XkTrS-Jh2*Zw?K z$A;^3f(6zptuY2+u`MkkkdovwbB5-;M2P>$OOO@yq?0Tjvlqn%-BqJjHzU z`n?8Ft%?v+H8bAR?6|$Q6uPk#`r->}&k@j}z1&>z!Sl=FPuSCsB?jZ!j69yD*R3Pp zPr;-jNaNr5(}bS*UegSC6Ex8Aw)*&|`P+2hg(6t~XkimG&e+ACh4@;&q8%h1zur7% z00PDLXGEu0xRh`r=zc;pr!e=MONS4!h?b?xKBdFy9e)TA@ryWbSVaR<8)w+;HDdj_ z+$R0epVA(+O9BbJ>IS^I8f&v4J8CD}$`}B``^1=qZaWQ+-_r%xQaW^N#|(l_`1}}q zm17ErmnY@~48a%|*?^QV>s?9H*NpL=CjwinN5RW>1K9^aPks;WZ1?@HlKn1IiCkRV zUm7_gU0bY&qx?Stb1zOfzv>w1x;`s+Cg{)WYz9s@a8<2Uh<(Sro6$oSY7iJp?snuu ztK_+F<8CPkRwv%h#rCzO@C8EWMj&K?s{cKmkJvV#w`F-ApGth@k5L^#=ZjD)$J~9Q z{yeYcujlXk_7}7}{LKR{oI|^+Ze-Mh2kJ$4-rHZfyCDe*EC`GVHKqt7lJ)ru&VHoi`me;)sL@p;>00LhTUwx8Fa(fKiUep!-lA3_(uRQ8jE5W6Vx z;fvVPanxY*%*y23l+H+Z2g`!anrm*dBb&HFZL!?Wm1;-gua8frasEzQOFZjIiQ?)= zK|xig$fQ_zhu!f6IVGDvXxJ{qaXiC8B2+5MqJO8+obF`Oah*WAQO=Q3E;RZ4zU=Hq z%x}V5Y3Yqex-%9pa5TSAb%e9HutW+v3TdC!O*@YADe0m{n^GnhS zNq&xzk+A|`3N#EiyF&c?M%^l5dD%_~g8u&fu-E8siN*98n%cnpv{|ZzMtR`+yg_0E zSg6A!Lsy?)Y}FOmF!9eM!jsI04gI0!syK(X?DdFcuPDD6zv%y3*l{i#zHSiD!spBT z7sOGNLg|>L%~PB)RT`t(>>w65Q%j%YRU#A`X=ylQu~I>rluRz>7{0qoGdt9!Ixl$L z^HFc{oswJ%@LfJ|5R?-g&blD@0K)&ixW4;S*`t5U`Y7Pu502cu4f?e z0+Lk)A;WMmr>?|pu02L3fJvee)m7|_RZfq{^dW$)aBOTaQfJRl(oXa5JSl6?j}YEB zx_&~&vt;egi6)1el>xI4mL7&pc~O=# z{MCa0nX8QL#cFI__%^EUh?&d&0-Hy!@@jB4`5yZ_wwy2zFtEYT31hNYh5`paQGrB6 z*(rJkJ%eR7(pCAI^VD94k~N_|QLC;?GK94}e~aFneMZ&%524<@7lmaN2J3g|zR?6z zontyb3Y3_(x;P;EV{D&Grq4kd3*W>lQBd*5ng<0O9_rD8v6&NI>;I^O!#@aU9Z+I3 z`1_OPP_$+GXX0N#b;0Z|KIqEw)KzG9-J22xbuZb`qFg`;Y~tbp+3Y!jUn)!>=)IsJ zi>fUpr7%wB2T178$`}}_(>P^AL@s6G+?%iKdWG1!18OOH@g;j?F^qGxfaPmiAdxe zY_P%CL^r>eO2yeLv~LyZT3mnFQX`C#BIU+6FMq?6Faku;R#2TMg+slmb&D}Ra(I5N zMrWjas6@|oz*K>MX&jFVY4C?LR&M#_XZ945L_1w|KyX+3UUR`On9NK<>()J4&zKZ{ zIfD*X@@rfC@jq4;qzAU*Y-whfjxJRW<>$gf^FCz4J9|HopsjQL=U)4_ZKBq=F~Mci z(fzX1r=W)k#@2H0=6{FMa*b8anT=f2#XR24-w?#IF1*ZHru4EF{Lf)RuxKUOjV<64 z_har(UyI49Mgx=+`XT|SDd$rp-~B7kFV(S|O*(+m&U(Cr=JBXKof~J@11Gkbm)tpi zbIpQFw0~R;9in!PQir+YOC%haBqH;+Uj;*B0o{s*D`iUh9BWb4Box>x0+QJC9opFB zLBK9j%f@x@%~4m-zSK$2yc+t(HW<2YL|d z#F!#j@hRGeLOt7f^hxbS1>+>UySGpCt3en1cjo+O@{R<6vT|AM zA|z(@VoMUZ4?GvgrCL*iUpQ|dgGup72}{nrt(1wZ9Jn2>e7ca$$HF7@OG**hAw!Hd z)!MvyJxFPsv{sD*dhQ`sm!@5Xfe*JYNF=Qi7&NqTa5U0iu|;;WO+y3&r)t{wpAeL( zK5ev1;>w^YYYvXf?{(d<4v-1m5vY1inK!X19q2@r{p2ZO#H4zR7cl$l73>!bC6SWq z0jw5lI?~6*JYXJ;(%(hrWH?1jYjkI$sQ<9>wy#!K@TL1KBeBduuHb?D`jXZk1u`=Q zwX7*v-5UBSEVxbZh9POW^TViYYtj4#HZ-q zroFU!&L!5UkcqS{OCmTgC_ip1F{(AfGg}#%49LWSjI0%P;Y7V$u09?x6OC@=whavw zPq}H`D)M^PE*;O-^k6wWP=>6W=c4gAvbQbHk1q+f9A_->nHFg(9FQVFY||J1Jl8^u zfpzOcDv(+zm@5y-aC+{9K-j=15lIv`4?BMp6_lSN=*}dFVp#pX3RA3KcmOE#fNCy! zDv6mHpwu`~9XQlTz`0A(HMrvaCbmz?Gy!Z#DZ-o?`-zOG;Rm_>eN^*1j3(E3-QOp#YtU?Mv{ zvzMxq!5+(q-LowQ3Tjf@=$X5n-}aHCZSg9hpoIQ*DEa>EoT!og?m*^1XN_C}rx83x zwS}BvBN#U9zF0(Oy%)1qu&?44He&0ptY?X|=oZ3>)pP~AkmULhz`P z(SA+_3@%%=?=rhgwmd)mb~V4Wcy+tdAL`MX6v~IS(G8#=W6fM@RwkOB{(1LFe)ZsW zp=+dBXr9?LpqFXg>QgauW_vNLwZD1W>wO*b$ZSb^v`Oq?*f!2q_^OQ$OxE?p<@l&^jM^z&l0uMj3%F`Upqe3=5k_I{)(m zgvjyHpM@27udi(A2T1hSX7x9Pu$}KcgY9yMP2HR(Ve16jv-^HMB^&$KNI#8;<9qEk zQlDaSZZZXYd{>(qwSDJ$1!x2zqjC5reqkG@^nl9QRTf)yj_1L#i#c98ZGW!cNFrY3 zwUnW~w~S4xgk}~$A+2p4JmJec?#%I9t-KOB6!B*?SO2e=@)`kF zu{PC7C&%eJfw^?1fh(N_OxFaH7E;B{25s*nE zw=}j>j8Tq1maHy_WAtnrPRK@2ev_}2v!k^JQD~mM#cGC-I_?zL;S)B%{&U5&wSO>C ztNF_O`LpQVj&J%Lv?LpIu+0EfK8(SOEi=|^C)L2(w4NTw4{Nhx8It`RmYS?WmZ}U!uTr*WTV4vGhd4^K!!a$W1HWVxq+%KPJPzREm0K*-km> zs=iltL1H(zl2*G`w0aMpOlxIY2S2rJX^oCVkBc82TJ$WUb_>GP`{P9jPc~Su%@B!P z?0TFnA}j!ql-*GO_Bc}^(4Mxpi|A26u@x2uAh@Giy7FJEc~TzVFaC#9Ipt;Ax$_>| z-|)$=zFgJd`ihFdvp3lpO3|bdV$8;| zlCr&sFPm>)a7gQ34a8?>P%gU?=7os7t@>J2X|dO;PMm6F0d_R0rTTL_2`!7K1rpHS z5KYotD!-DArUdY9@rxg=M-d>=>EIKd3}P`oxmS2xdY{pXVRUi)iEm=X%W`4$pZ1xj zIC03v(5%3h+)W=(bbYB?yxfYdHl=$|qqeBVVS5X}hJ&3+v&4dh0;Ij`jo>;#-lF^=S0-Apz*qLEP((8m`+77wZF zulD8Ux3GRip8YAKgbG1^lG}hq<#3rYW3oy2D}7-Oy!4ua8o=7B!s$jzT`%WH(^%(dy72SHi3o*aVM z9_(=osKR_*?cAkn2?dthe;vjd2a^}9%9YcL7fd<9hr_l9!<`XZnqH5mUr=^#meGEm zIT-8E^fTQk&hf2?F;MOEn#miKZdgG2I4YIG{nC%sTAz)4AgMW=0q&H9WDIkwwONJ) z8IL{rQ~Hhic7idV>X90~f(7@*B7V78u9-b)AJ`WHi@-j^!|1E8^rf2H`rmcVbgIc05rJv22(<6}p&@FAf@MWG1mz zT(^y9j&FavS=urbVYA1hXmA_fTmEiYWT8Bw@56g|HN&vj|6%581KrEST1|BCw!<4s zIePeJJ`wTWG&pv$bNpZ9V8;OG%0pix@srRUx8u)!Z`pw-P80v76nP^G6Zbp9_B#rs zmCe=aId^dEfxK8F-q70{uST<;TX?Au?`(C+mY`7ops#46y?#;q?tUz49XaC@>`qtw zDUMfsT$Wpum~|hpo`5ZS@VjS}BzB$`HV1FTcz~%0BJ2~F0I@>vICOSbUGTVK({7Ka zO^TQygBU~H{owhY#ddhY$a)?Wm&Xa3h&oe<(ZvJM{~7W@pN;Le9xrn-Lo5;{_MQUI0n>Z2VaRdTLB zyr-sO*vU8hCd|1ZW5BUJLvjwz1~0+FNy`p~l0zQ?w~PIX_BIbUHf>T`{v=gc^B| zXHElq1!uqH78YIn;j_Z=l1$%QEy2J*X(=tu*iawGPLGR8v9Mwq*$gzKB{B9saqX@y zE|!$*sTf0lJ%2J>(b;tw9bj6G(nd_gFmKeWbNmiGcT7t`)^Q?&+q^8A`re58(8FE- zpd4^09>2UUY~C{fS?*3>z@Mklx1R`G!i4)^pFzCLxoGd_KyDbY8*-^grb{*JVKqG4 zp_I%3*1=oFq&88H8wgv6qynQMS?fZW_hR3C=k7C3Tn0mHyTBODd=a$$IK6x@&wi7O zDBmkF8KidoKV6To+S`-$6PH6)3gyYCKj$Ki z{Vw5GTqoFs!tB^XA;}92YCrJ|LhwO{f1l}ttWKQfXF+4tB%d+2_)l_sfyA0B&PDrp zH^;vgt-Cs&S#RfmNiueMUtaJQ zp5O?3Wo7t z3QPh5#iNAGB?5v02Kc)sySy8;zOl%)oJ#TVF3okNUGIm$oX+MByI-}(x0<0+jO+6Y z_Cb<{MyX|`PGM?aHTfd@h4P_g;z+xs#zz7(AI_*a>2Q>RAz`o-rOf`^`p30nZ!<{+ zu9Pm4a~2OON?VppzZt!$r&*J<)mN1*TXZ9Dy!B3rO~p;5 z27LC<%x@#7cKtsPVQ(E56Eyf5yv4<`JmZV#7@x26PL2`T?tPy zlkt5zc?E-xUdeSh%`6G|&_U1gufJ*MPw$Hh-4m#4Ew2g_+4osn_WYQ~tNAp(dR}3r zzpG$R*?#F*S?6fe;kN8#nW_f;XGELyBt%LE1aV190FAEOyARyoGS;GjSrl?35#nw07RkO&57ZK6HRpk#b(5hCf3SLI9E>DD^~Z#0A*0 zV<|)i?GeWJ@bzO*;eIirpBy6}8?~{(o#%7$XIZwMnK#hXWHblC06lUpEHnSG@H1}S z>BQ9Em#0_{x%8nN$h%)}TX8(*_hhYvOQ(g*UBljv{jqRwx`{^M!J5kXE+t@s`CW*2 z>I?$xIy)sfhwT`AnJZB`9sRViK=k@a$ws6R*n*QmbxZ?#5xo&b&#q&I@)58J(%4H= z8{IRs?E^DN{_?rCd{BFpobuDHW-SnnZFyn zbsPr|S;Jaim4~K_gUy5)g=h7lNCNyEXZgnmTtwGW8m;dE?HOREb45oiT(|1~b<3>(eWnvoU)EY;I_MGlV(p)tyB;(6kF%lZZ8Oc*r59t7+X1`r78@5o)SG$q z3UugY_h_GHwY9H}E3`=!uKhfePqK*^R$=D*j>-vJNLd_Hr!yW{?fIvPJ8I`>16C=S z%$Snaa@}~Su-esnr{iAI z!0n*ur32FoEceNfFf4P`(ekG3J_ShE>~Q#xlOOZVJEN?!F9ZtO*xa=Lkq3yR1hpS^ z@5zgkG$A=Ru%S2jy|2eT(8@^&%4qGRsMG9uuic{!eSKR-RPg23W_T{4RBo>Bysz=S zwf9NrPlmy+vkg<;`sv=43Io0c5-~akgZiKZJQo*Q4@^fpxz3a~WV#k9G!8rCt_~cV z?gFEF#-{jg6hC*YJigTBG+O^IttbtocY}VC(st-Q8*+91WM1k@NbF_@TJs-S^XVFk z?{fAlP9I($aeP$%sP83s=XD(}@-U7vHVRPy9!iKq~T=!B`^3S9JhZGNM1XANiksuZ#X#&ea;QC+8J7W?f+{d=m6$Cc_!o zQE07lp8vYKu89jMQ*iJ*2&j}~db0gSZQ~x6vQ_F4mKiEEJkh zG|PZ>Q9ZZF&#hn9HaLqWikgz5qsT_9PZ4jdun z!fT+AKa-uUvsB8xnii*bJR`h5)3&M_Q^I>xH{sO6B_gVZyEOAu% z%Bf3w+v_Tqhbw9cEog~WO=Jn^_P|}I zVtTjiWP;siw&o^)MB89)~XsMTt;+ zHMo^R|DzC5HUE>)(C^OzV^#{c-ggEJY?c_|Z(sTFcYw~am-F8vhtE=ChMwfyk_d>C zzk}iouUMtRK)5YD@@jy4-L95 zRNWa7!m*7+%WLRvy4oUkhbV2^c*Rg)hEBl)Yuy`^DfU!5HH8i$KQG=QeR}%#8%#NS zW8Jz1#AEL?i(h#9-q&Xy+{!w;mPUef>4a4MwR|i1$H_bugmHSug*FmORVtC&HAU6# ztz4r3Aag+%Xqf`7Nc&hewb{@k982Hecjfs+`5ND@aD(KTOl7Jd!9F{Oogv)k`X{z! z!Oj5B_~!O*H2g1R$;pe(+bNNZdT{aa>JNv@UMc^c( zl-|3(Ch4XIlpcYyYaAN;onvu5Tezhhfp%gE|5`U_?#Xb)$+;8zbK*7G&w4y0{VlG4 zs?3quQw~2bu{D*wA;vs8$yLQTf?2#-#WfCYnL$qdW2oPP17c8fGAGX4O|XY5m`D_p z*|??te)*$B;_$5QGSBYuj81Okn zj16YwQ=NZt45dc{dR(gn+v(mNBh;1T{CH5$)YlyYwn>?)TS+JyJZ_GIr-d8{fah*y5rp)}SKN^^j`sRP$JUr;{`5NWc4oPfF6oxX;^#r)ueoXUL&Xx| zZ&A|@VCaRY{d8k+R$Qpk<}{n0ui2l34l?&LHok=H<{a!KPWp%2pFH#TF5S+K6q+i# z^_m4iPcdSlm0V&z=0x?rXI{08C})x{*955@R(EgJHvjBZkOg9>$LLD+qV38r21jxR z5i2$NVXonv8*?|RFv-wTH&v;LXLJ^{lErw3kvaTtw8VIy)v8psYG(5WXou2sE~CT~ zKN(%qwj9MvJRGWJwt=Q_7E#q`(A2iil#5sO60|vhbg3aIh)OE40P89yNZ$@g@0FNc z`zl~fB3jvoQPVtbxs7`V-|3C>n<4*v*9y$n4bE;ACe*m?$zujaeFh?FqbqkSjzeuc z2mv1SKO@F1ot%z3BdW8w6OFcm6-Y{*s{X-BQPhkOv|3SEUe+OQ5q8}oa=+9!t*vzM z!Jys5ifIVrD|Pyjw6gX#XThU-07LLL;sr>{r3lbd*u8gAz4QKf|^93?L4Rw}ObKeTH2I9;x zt+bV8WV%UD!`_`CeeDt#lJr=Oo;QS+#<6M_(I z2P__H<4!9wvr@}+bvLCgn8A)O-kH0~zwv9`N5Z8uy*T2jFe!qi0I}HT!7hJKP31yK z4}0;Vb2XrocQKmb^ho~%^NCw(VeC$;aV@jzQ^iU;)h+6k?SX4{naT<6RMtJ%(&2>o z9<$S(Zg%*<6&CwBkvZz!n-<9hq;p8zMEk4GJF#%K$b}!TGhENPi?2mDf{Z?jS+qYgu(in{H1N?N&6 zMnVOv8}V`fVn1Lpy`q*n3bLe*WbbJQ7Xp-WHasH~6i-C-kVt9~ptTs(bg-6Qiqb-~ zj)!fpgmo3^>TkcjKl4!3nrhUTOL4udl^gGx-RkPfaSm2pU=21v4@%R&EBlksFUGwI zzq@_uHJ_|!hICLh&6+vMkq6s7HPy@fVirAP&c1Q}dxiWe8LZQI#Lqs$gdZE&9fyNL zi#j8t7Skj5M zyzv9@)Pb``CoNSG7Veeb0($mzyEHMtIW%AlpYI@%uid}G|HkG7*Aj@Ts|-?K*$Hl> z8qr3#(XgZ9A2^P*?-HfV0Z;_NIHHS?mJAXnC2vU|y-v`yBcRSoe`{fyOXz>lmyk4e zXpuj7pTSh_l>lpZ67ll|ak?ZMcRf%d=or6k(^~_c+E!OX?p6$<--&POhXo{5SgrNF z+~d&B8WJvMu5c~C-n{r>a~!zg)sJ_gFK(M*=qqK(5gH}_VK(l2up`n7b3Z*0v{v!t zF3f1~e{WC2fVqIcxQBdOgrQE6A42YezdP}%T$;8o&0(Q>MQLft;150;35spBpojIX z9?ghwwAS$h&0J;F)@#kCP9T8Yc(>91s(P;k%FSf8M+=t=<6$#PG4s1$WW%;?gwmbF zUFC!mF)UxkWOmZF-6irfUdU<;++}N8ta^;R_C^ES(L3;lR|l*)sH@CLhwuCtn~Hueb?VG>)tq)> zcLlQDejy_3aMoeh`x!zQXHp?FzFe4Kpxc40_21)#f$eZSGEzD1zH9RY@Z%NW?PFFo zrtdDRKahk^E%O;8+1g5t&=Z|7oIBRfj;M+BBuBH)_eMov1?lvqhkOCnT`8-2z63Lz34$ zAdvxf(JJ$*mC} zZa>sH_k)UYVrLMe=VP1!rMuN7Srx$=^CEQuE(9!Z|0}h2IQ%j5E924Bd|IWhRc|u2`EKL%3mf z?S0^3kM9VNhlfo_!NIr*tv6gJ?Z#^}bt*VYk|zPYL5Mv)Z#cU(0}u8IL;%xONAe(A zXdYLf{s?zBJ&Cqiz12uPJ=b2mc1a*;gd4^72Qa_5XE{Y2YpYew+BegPF>s)NooRKl z*`yxMmBXtqP-h+6Mb13A6|}ZGv(ygE88TuE46auj$0og|)jidrICONB0HWeNarb-2 zhUgkFP(nMf_~d(qPe04(FFYR<46WXRj$yH#MAZEp{AwH{OiiLj@Ul0(o_rS@tkTAO zN?%^%*s25%FK~%=cwS~p?3}jr9{CxXHNUI<*2evmuZWziK7N_4KzME}L^wpKD@ac4 zXw7V2zfq5dFWBG>vATp;G>6yAmjn(M$2$%>t84Ox(0iyRQP%r*Ho4HE31QdE5d=5c zTs)Trh0vKxA$|7CGv~F~g`EJ*(E->UZLcPbDvA5p{Z5R)hP- z1n>SoMQ9mlt=|^aC~~hZOKi>(5<8`mRoSgDS-otI^-kov36H`Z8=S13{Eqa1Kd2e9 z^p1>ZcUe3CBy`@jtzojiXBX7Xb&A~G5d3-5)CvzIZ#IkOoJ?-&Jn{W^wMD1#wS~hI;b8JoPPseW>$T?7{tSL2zJVPV z@JICQ3ST8O_JMwv$IMMP#pTiyeWzU&n?)_%ft4w@Us%dPhrdo6?jbk&IHDG$lIsek zNNH9LIb@Xhme;ect0*wVq$@cZDi_xzC8y;#b(L6uvEp$N6yTGp{x<7Ws|{T4%j^YR zaluTZiIC`Kv7c{erRfj9j%UO!`C>kIvFXL3pX8%%HQ3apY^ZM9{;W)DPE!|(f!T~ z^6-dsZ=QW_*I2?oc%XucZ?*rvC3;2owU0v!Zp6LZ_K!764fEvhAg7BvT#;_mub@*g zF9mdq*F41f6(6=h*)wH7Hy&f+f5^MTc{Y*ZW`^lP8Y5#NTOa$^A8QqPDeAVs zIk}Q)Tnh4$NA=FeP);S~%Aey&|ExV(eeiZl?#xJGmVVJr*bRt$jnTIb>)ZW&U0>q6#X`#Fu8+xSc1kK#F_A@|7Ul$v;ED z{3L0t#`*cmy@4yr62F($B)1%G#xpn0!=^|-@gUUjf5&v2Sg zvB~dPWCdA>%1f?tkRC<_ri1K?lN3`d<-=RNw%tKGq#|p1;`PSns<@*I7`wN(O|=}4 z&s?E?!R3O3>o16uPJcYhEI8}q)&PC5_cxvIz4E<6dQ7fQt+U8wCNbrh2t2~)s9r`P z;d*=G)a;`lR+S7@t4u_&T--d=Oo?aMVB^=vD%90U6WM^L`;QD1hA%saP{g9P#TvXL z_Mp|YPliBC&0)K?z9BJrCQBiVVz#e;-E%k&LF5Ip6-G-78&viBn;G)19(0sr4x2C$ z|Ae$8_^9&i3qCMPgRHS5j#&MPvc0mHoT*$X(KFLGD_u^nIDXNr2PRLjKlkNoj@*gN zD(@?wi^Pn)mA#WX(Oz+BvP}9yP9#Tc8`T4eKDHaf@l?-fyYJnD1lATDPnNev-;;Ra z!2cfc#>W1z!SDx!ZgAe0s1-P5ozRW(4J9O*lf9Wh8sD7}?YIG(1Ks{Z zUmo{8Q)R` zHQ2f}uyv&~!m%O);l{26DbulD9g>ySwT1g+_DwyB)$*#H>5-=SyoG0XJYSNOnF`43 zEeB2+$vg+a+703+H%|9W%AH|mU5-ezPLJQnrMm98h&VYpcdDABl^3ZYVT6s%AmcKq zudjQceVe>yFrnmog;MD8_S!F3(j9Y6+xJ8n(W{{yu~0gZCgtCuYsfUn~g$n?iD?oO{t&a@Rg}wh zEq?HEO=rGB3)kEKF$A<&J<1v3mp=kiR$*+HrH@LbJal`>)e(IvxMERK0<%r-= zf$88L7bDQ;ZO`5br-%(31K%aBrD+J#T)&NLBn4Fh%O{nVT!1>yb)~}!koWrld%j2M z$18uv#M0Ujbp^*R%+ten$2NXE@|9~G`!qM_$-xh5J+zIk1STgLlSsC%7}6%2^Rh{Id^3u#*JV+6tT*)QI~Pm)+<8&G)byQY%v#x71h;<;3fogNbsG8 z0#?;7r#-?{g)ZIlnhJ+_1JyV6U3po#4;*0HY=-B__yaY|+=ZI(1N)|h*^+0Scj2GW z#j0Wp323SDi-$@G&SLn{`?|k9;~{e{roZl=IzqSkoOPwBW{4V3X9{zEoXS(#U`&Vo zs!-Cz6=IxP-b|(Dkhpeo=1ZjXbT};+*D-mKw*W_ZhV_w~T8X`WC5*!UtrD$EIE38? zpFox~9 zC?u%q^=!a*_FVkkeAKh|-(we!p05AD$wN0I+NjdylwcvZt;a1v8eC^ZpPi@kCh!?Y zmC5zKEG4qihd#?(C-L)$1Ka0$x0-nCanG~41H9szJ6`+_iXL>~8|GvuR|6Nvl@4cK z_qcS4O%5wl{Ise&W>>n};gN(fm&)R9jNvF-xJ9Z@UR)dew}s|zr5tk_szWzo7Fp&- z6kddV(w3CykcW7cnELLxIs*NuP?z4 z7w`^~lyz~+e)!K3Zrx?@$;w_ELLI|xIq3u}9=iV*e2V_aUzfsAl&CfxF5iSdTKqXS znENc5QF&bM2pyAA?K9QKEAP@w@;XG1(V&J4T}v@i?6>y5d5p-tvaEN;ue;R8!GHB# zy&-pNcCEs7c(nnPBS}JvZr$O?U_TrDN+_wf9Fybe+hN6)I`5A4)gasS#+tmvD#^># z$U=wMS?3!-8j%nss%>2`GC_X}OENur`EyK26DE$ee;As@;o&GNA@wR{ev(%Ve4Y>) z1t($L$I?N7!Uol-zKO`5b8e9Z+vi;t6XS@9W^3F3^&GzzK3t0bp1yp$4kO5CnnfXD zO2%SG*?)!JY(M@jbD`$CgbMA~{30QxUxu}0d-r2xnl>nzg>CI6A6xHU*XCiK4`?&d zT+BE(a-DKj>vf)%+VttlemAi%r+C_=E&<=nax_~kY?N-LH7a+|Fw=s(GIjllhf+H^RZMBYS~NQ_jLA%6;?rS_ zfe7WYNO>PO_RBFBWjt7^!727j;%gUa8X0@0H*4kf+ewmY;lR(bf>Z74qa6DL-j*`Y zaPnWg8uT=QA<}YM4JnqzH8s!lJF34B^x7l@QVkQ08+S^+*~dZ&QD`~i^Y8gfM=BK& z$2{l_c<6W;*R-7qP1c`2LU!pL-})@8KpvT=(ns4ZbTB;S`UIGXd#meq2Kci~h-5+F z+o2@0wI@~l9`9Tij});~MJ-r2Ap-f%KlNQ3KIi($+(CQDj8_@++Bts>>hy+W)~**; z5jiJgGEy)uN}PFE*m0{a}^@*u>h~X%$pE&kw zGTCs|`LxLOEh4p-kivSWx-dEi|SR%Pf< z30TMHg}W($LIA;p@oVI!%ZZ|%N z3WumT>V6pPo_^QR|JiBdXR+x~$bozq(<8qDRqU%QLW$_>7c#Z(tK4}tSZK8x=03H} z*?z!>@om%8e7qzUG&xe6Kos^n4nirX7{R{+#TgS31o?9r9c<3FsWZv7(9ov7jd`msOMmEGd&Z2qCauNuC4`(7GeM)?A<(0VWL z{WTCGS<_*szu(X_t~FEaXGGzm6YIaPsTh`8CUUH-TOXrDai%XGi`L_M>#m=|*dO&;7NoI&su-9Q?N*>P~r`srT;Qa4Vkgy*izQH_SMMd}3 zP%?3y)x_uO0n*ET@KR5L+++Xn!&}R3vn4G?%Co5M*^y~SKP(os=c?F*YXsF)d%SQZVvd9+F9{wftaz8qJ%8K3wbs>e zL{BIaC9}IE05urNFAk@j2W)le1GGobjX&KZ@%~{2+7~k!z|W2mA)LG_dPGYTq^;C$ zh>o%fF7S6uQiRQH?01>n_Yx9EQ;#GZ-t+p9G`J;Xi6SdhY(0g-74g+t6BFD-^&d1x ztE#{pkqc6j0C?AXxadCtBYEzpeDWU5ElA9#dpmP@?NYPS#X^6t^CW=&rjK;(xjMK} z^#e!(OAgbcMud@`s6XARt}gK0eNYRPR{erGisw|!N->!ECGA!+lwsGh!lj2NS3 zRaBD)4U7np97uVKW2Xpj2!C!`2Kfak=7B-@0{I`zD!k@#n9Z-WNPrc!#we(6XdG;? zf?Oz*TBz#WR`RC!i|7H4DV;PeOkzY*WhZAYZ8#nLv3i%&+DXIbk+dZP1&Qp}f`;9* z3$M4@{+;!933|>yII2&2U(p==A$cKv>}kDlE-lTVP;sv41!~BO9#Dh=g#RpK!R%7+^hR{iQj7KG@T_yZY3Kj#{mX|PN4&d|3+F|ye~zi>tcbau%~oBF z0{90uBqmZVSAw%$V@65^0~W16BvM?nsfr2pY`K=>%95Ex8N~#ZtmQ8%io$*v&l*rJ zwO?ZMx$CzEYS+i1^~>tf?HHUKtKV+(ZE0w}@XIJ3QO>zW%wGx-+n{P*Sc|yRB^xa+%HgF6o@$eZP`${%m$<t2LBd3Y2C!*3kiqYo4uTmRK{8#A5x$Grol5zn(d?7d zpw3Jy0+#AIB7_OuFp87`X`V4WZUxaj8s3X}CbAq2Y~Xmpu{`Rd19<5$_W-UzF$X~- zy$ZXFSq?l$IyEPJ2p6%}d+;H<#^U&!Ts#@^QeN-e&>9N@tX#%)Vw9ekp#>GA_PRm0 z!s%mT7U>9e1zmc&d;gO8#F#TRjtJq-(meT~r*30ay}2EK3#K!bVD8jz1oR^ntw$}Q zHIJRuN9#o9ryuI%uAmK&?@J>Xs5&>CouiJEXnDIOw)8g*^5;{Uf8_eP_WCO_mLlNH z-A6XZtowi-iMPi=T1iSc({~=ftiMQoBr23>U@Jb|^BTW|T{vq9MY813Eh2b0Z~VeU z$FIB|8W7U{#@DIYBMhy{SQsM?@;(mnf|vxeMU3*>{%^=c_~eYz zi}<%r$F5!e_PN=fsEpDYveM822+LebGukMYHm2qmf2rrP0pC;&Y!zt~(bvgi$@{WVC_%TDDcMO$Gag9TW$Za;gYe1I&_BfMF_yf`yaKD{cf-$rsm zj;Qzrw=@kD=jwrHA4-_kAnGQj&&;|69Es%&ec5Yshn=SIs?FDzNm_(ds_$K{Va%a0 zNufi5tfG}2v+5(8>pBQ9OD^{YAC14bOrp{i5a2k%gL_c;Ti>ZQhbC~#pP5F#Vd{%F z_|&sQyt8`b2%dgHVT(plmgA67V<6Qg zM17(MrD+HH@fwmmQBX>C4lL02<&|lpx~6eM?MJj!2rYyGzidRUclT1dNwcP7+eo%r z6jz$LK~GDhE(t$leCCPCfaDH zC(cYYd-f({yaDP?W2IhL2{!*}mZQe8GhNEW_ZygBvIXnPkgvM1t~*a3M9pX9xahH% zvRb+1^a9=aQaHH zmIyPWfEg(tPH-%EurIsTWLC6A^9JR7;s3~S<@Fl08Tz%0lSrz<==0eU2PP{8z68E@}wAOuDtkEw)L-{WTi~@t^jrwzw%MqVDM|)K!BH-Lfzx|Qx4h@Qq-5I~z z%H0}Th#SnwS$f$#qYQ)^Tf?c`Q%fGSG&xkrY;z5-6Iabz@=Y!A;SfNjtY^~46v=CU z`Klq*33V@K<%1VIdim3(lN@lZFY8>~4)9eR70?mhj`!W~7Qap%U{8*$<-d`4`5-xa zypWP`srjzUo_~p+68%sn1qZd8wvFDN=F}skwnpKUJBO~Ss_|PWU6d8VyWdn@uR7( zq}`}~!B%%7WWkYFndsd+5?txG2rT`CN-FY3QKwX8tU4@H5&%ErrLAjSc}l&3kpAYd zd-}tRPm{`KpJ1;ZtE=!xj$|KcHx2RE%|x(F@L3g!TpQV`5A-pRKW4DD8oQG}6383+ z3|}pdeTSdP%%0EyJ94mi9ot!%J*~gT5vc1-2anrm>s(@ z+*%W*eCM*of(V~@J6uG3?j5gy48kOCTT)~BPz_QSfbE?V&~*=y<*)z5yw1CmDYW{C zB-h=snK+~K9+7fkU>r@$%pSge)f~0#wRAc!YoL|-_vwYS`I;OG9A)7tf0m8Ax%|Ao z+b4hqi6`6gm|e^0jp7}Wh|l5=7%^gVahR`>_UJ-7b}{SQ5(EqM5X*HkDs}#TwsMV7 z$Fdo1Z)@wMz(r~Hzw@eUisrENENs}!^(XnipT)}~>+R0m04jX=D|8x)-wNVo>~S1F z_I8VfHg7MS{FlUXINrgWFo7ozhvM^IJkX66y~^Ous+E$SFi|YVAdZZiF33CloLTqX z8}Z0t`=!P8!gZyl?ji;-!gH?g?Ay_iv*R?*Ta1_c^>@awlg8-4+R)})iOnol6I_ys z)C!kO&?J|<`S9pql%^5XX~lZqmDg56a~-#f1GduC;o7kQF^?k<*8*WQyJ%V z|E*AXo2Kcl+~rfDwGkSmh_m-{#zWTks{<#jydndA>*t??g^Q2scnyYU{T*P{64`>= z{5O?dyDGNEJgyb_*Y!Es@AlWYaDe%(TZTk4MDJk^f3N1!vJffzyIFT5soB=Q07s(A z1Iowak6NlbN6zn!$<(=EEtlJ=V;b$H{u2qe{x%OW|=LB zMJ|f@-Qf7x`IFOA#MhZqFj;=zX@gJNU?*efH}}1k+p!!$(x|2j6Gxl!&*5iBy-$f< z7)Ekmg8ywXIasCrVD*WO*9~j+>7xM6;lkm!T(bi8Jz_uCLTrLa5*&ge#ruD-$E z1A{$R(}ze-K+YJs*HC79JWHri=44w2J9RPYsBT9(y2%bt7=tRAd*R)C;YeD z8*Fj-x0e4M8on{x;e8-jE-QHc?%T(I5yzuA)UNP|4%Dy9-TBM&!yWS4^+x`OmMfIl zcnU5*-ihtLTkv0Ttru<@%1>_R$(eWDGc-+~sKi@Z$&a~m;Y1|g<@mWd#qekMUCt5+ z7K+@j!60v*cw_gy8`Nv%PY0%Ae!KSjXVj|eB1QjddvdTvS!)S!gUAH`9ahX=Ywo_P z-ImAoEmzK>T)pBKm(JS4m-4B{mmY@bMk_uGe%YY-A}a3jp3lVOzh}2cE!! z^;dBQeBlX?7ysPho4?`ob}RotqK4;cRQh@x^Ji5Wanq=ko|$Ir;RFFz6^JzQ;b9zI9^pBI3$t|CMb+vJj++_CX zP@`(HUP(*5(2a0TPtU$99l$%(Nd=N{Q~V=L)|Pmnqde&d*Cx>=YdzhoEu| zwmXR!c(U(>JL=5zq2#+)KDXtZ-OWV%XY0F0v2b;KsMH4ZRy+4R*U5QJ2R6w7V4F$HmCQ?968ar0~yv zA+(SqnHTz@+!w(HSJ;n7)=&?u@;TS z=#4HqJ^VwhN>e?mKcDi;UldQ7{wHVeEdMiKXGe1zO;UgC9F_ji$sTsL5h8u!qGk`b z_nCfo_QT7pswPSRui5HFof1(pPC=07Fd5wnD>8l!ADQ4r%1*UsnjHf zSho$LFH?J2&%Zk4f~j9;tkG!L9gdRDAr+zQyz19K#G$BY-2J$2t!N!|!kn{zh?${t zNMH2yH{Sr!=&l)@$T^<73jV`I-P-r*D-aWZ68R?3YTElwi#t-`j#|Fn`L{xFzup)A z|Gu-*%4D*yE{dsJSjc--+}WU&wYn;+L^jr$R?6mShx}V}@l{1_ct&9afm~OsDH3Gd zZ7O;WF1G{O=LxXLgePRKNSfbQt(dF1AF$GA4!)>6GqNJ#5l`~(y%M=^%bCL%V&Vr( zL~;xuyx9d+ul_B5)H?3uCfy?0a3ZH}0tr2Vl5jEk`(CYK}~O ziIVM}M5pQ%vNCroc?9o(F>5mAqWy@IJZNqj~bpD{354UwyKpJGDQm+AkBEG4?)IQ?J)e{hcImt5+QBJ0U3F<#|-@BqX3 z=@3~EI(#Gj8;m_N40&_Leq}X<{1>Gj2(?2Ny1>}+5&L?AXgyYhedEUAQAbTlZ|r;I zjOk>a>eri;ItPoOY#dKcL~QQsy1^v{^0^<}7&hePn|wd>UG|pYC5mr81)!y|apl83u1|xl zggY&F*lV4DT5N^F(8kbAft4feAI^TO!gE$UVaT>qgB_}qqGuNDrf{u*0-$5YWeb&9 zHV@D7@sa4*0~%?kPiyf`1VQ<8)JENeU&~jgI%-aI>4nQ5N}U610ViI%QQxk8DTxS9 z%5`kUFAu3C2(z_tCFEJ=y_kFcMmh)58gP8{!|CLwnA#U_Fx!SawIte5mk0>@IDuZ+ z_%tn8T=MJ%$;<|xz6yxyy4Woy*e4P|tU#t)WL2I{f)hB9wnvnbcs+xitmCVbkojc) zLe+?0$;=lf_8D-HOB_)+k`kwtJ^P@2&0D^W(9!Q6EMl?!C4ADZ0*V{P&=Pz_^Vi0= zlgO7#G$JVIRStEX(R*RR`B_d!!<+3nhDMnN*HV{H3%q;$*zMZ#!5e_GHvM9TDm#u6 zo+Bt~EbY|@xrni37E3b(3}(I@yb$zF#o80?hKVxlmi4>bu?DSGfrY$tm)=U&m#gkt zgV#dGYiPIile!sSPINt=B`BW64Q~#vfdI-`%C5Wmx(&&hy#J1P8eomEFr$`&)2_9| zhEor;La{KmnrltVP}dS~hDs=x_W#bV?#YC@ECr_j+$`z5zu;W_J^IPQBRh`jS$<%= zQT6K!rsqnzfNfR>;xZgjt5R)UXG8fkDCZj6R7_O^zRj<6znn$RzOG=C@ulUN8FqM} zgE*{F><*H}eNObC%Zf*WH0K*SldoWJFAdi&%cYnYc4&zTjMRNFL}B~3LzM@(!Sx)= zKA&eh3AkTt4fU6SDZB}Ok-#$}FElZ8{F^ARzrAotYPX$oKMB z@07y5)1SXUP*)821oj6Mq(tS&g~iQss{=D_^|Id~u@8L*L@WHZb-PPS4L=}S^4nwI z2_>-Wepy?4YVuFc0x8MQw-1m8>1h~LKs}JD7{IR8aY@kHqiwZeJWm#;_~8OBdxvcl zNIhS6+r^J)OYezKB*SINt4D{uL&P<)AX`pX^WfKm>quQpWC4EnTFk%Ll(oa^;l>Ua zBhmEyrW;{t^#K-AAMvw0%&UFd<8JWA>C!b}&s`w@PMIQ%^lXJ9ZjVefAW=VY`F9d( zrZHHOx@hVT;oT~;x6Me|CiZ`06m=P8+S4vokFwUZJjs;NG22%v+OaO`)2I4!orA<$ z*)b);tWgkj4lp+z4JUEl;I=&*;WOM~$}zzy;{Uca$M7x)1P)p(+~SIHP}!Ku`wK=T zLUN^W3BtZQ1-t;2x9)c?XvcdWkRZ!& z>#4<~i|aoh*j2wDbD2=no|077JLy4Hq+M2|v&ZS+o#Om-V*{CqcYci?AM z0w78F`Kt9s;HbRVL$TnLkbWom9O!|oo((+Z_@d+9*LI@x*ZBF==G^L3HVkwz!jIZbp1rlj158ajSFfwhfEhq!{IUcQD-?EiA8l4jf7QUSz-+ z&#>H%C2o$SvLXql4q)-(0V>hP6fyM=U0-`G6-?N^>#skecNvrmEJxz?aC*VU{GSwm z+cl5{P0+`&sX_98pMz0W47RtG6Fr`iUi7EOkCw>lc8~j!LV2({`TC1j*Oj7jnfl0s zfFaqxr&oR&;1$s-GJ5o`)e@92IMZ1@9rT-bJ3DB7CPKZNs{pNvT)T=TY%9(LX(S|^ z#BSx|kec@8@RsR~HI4=_#oct0tvo31X`<*W(GOXP?H9R23*ZMiDf} z->xkK``@W03gS!pk_cbN?GuB8zBV8TrO<_B&b8TF~=xIfL>&mWfQpv)gF=eCZQalUpaCEMxq_0BMDtE z041!1h`gAPAcFOK9E0$=``>_4bF?_~8WMiH%O5?=#ddhE`?8nd|psw4Q5yE~}A|s2vt>Jl}fWnKdg1idn~Pj+PA)P+(t+!L^>P zFG{*%cTH;;`JFB5TZ>y$(o(Dx-o$>8YVsheo@pmW+xb{yYNt2yNHj87kssR{06H1X zRc{c05%|S0gUSYDp7{cWy?DGfMzYN zv0hBu32s-8!s{XH!>X53RJF|kV&J2GF`;h|u=IXBzdROE+esnwi4E9@Kx_FA8s4bR z?UpAh9{sryNnfIV9|t&v$%Otr>2*pynjx97b)Av^mF+8=kHjfIfEp_$bJaV~LA)sB z(v%VXy*M=7>S+9NuyI!Dk9b`|2Z#fycR(B!w+wK*%sYq2c7)Ae@N9JT%ahQ#+s*?{r!hK$)ie*(|WbL>|fhW71`b0Jjj zJk`F7IX3tI|EfCocqae<|K~7=oGL}ksgqS0A*V5vsGRaHEwNF~Ip#2@nULfyR6;pT zA(g|N$ze`qMwvs%95$!Sd1jaye(Ur7{7%>Fzia#Jy6t*hyPl87{qgAAc7i)i1$tBW zmY}*r=R&6={}D>D&fU3_yJizXx32Bv2V3p#lxa@XoNKI9Va-MLYrcbCpr*P*YtoV_ ziQP#=_hyttN{a!`&fYMYFGx2bYp6PF?K;fQb*IzMc8tZIq&GZ&wg+wOBF;WxuWcmS zOD<@%L>3=9D$+b!p}%k!w-x-omRPp)gED!#Uhj3V*w&33>L@o$Lla4rh@1RX?^LE% zN8gsF!D(sMR1RU65v6*7h;(8Z#V#xy&)_+^EIivStqlAT(_H`zl_Z}%>gzes*fe?s zUfK5jh96>wj{s3H3-OD|hm6S$0LbDi?E^dRbKbJnQs&z=*vOTw6&r2ObEot}N8VT{JSZX(i3Ajz*EV?YrmTIfry;A%Y9 zRqS!IFx&(8nScFMlsfQ422-V#GnQTG!O@%+x~)DPP~3^>b4qLv#yRrH@Skn4?dTb) z*c9-q>=;;G0aDeM3M z>)4dn2S+SSwM_U`Klyt}Wv>Y=VOKo9^Sh}w{4}fDkG4nYgL-Se0p7r!Vus^nNtHck zd9QGs*I0)m59Te@u6JhU;*n5Bx||!75q0!!ivx%En3)(_+w&cbnI9gW<2iz_DPE5rjq`Qw%0eA%Iv#BP@fzXT zh|nB8V(}5>S?iJL>%O@7E6q5i^;+wk+oeVrdqLXHRgwNTVXV$RhuP}o-3afd-jFny zeY*@VeZ>Ddw-f`bk7=Hm`YB^*ZFD}wCY_j=>UX`ZpD6AkyS$vzGGeH4u+z4$W@T6}QxdJf4L9GnQ-qoKjA>GXw$z}qf5L}yf z_i{JT5S{~WM(q)vS|jPf$)T1Wib{IF%-#t`>xR~s#e1{X0mjB3yxdl99+v12Ii~`I?v4^CXEUE#-ly`G0)Ic!7UBzUcjq~1&B_DV z##|ZUFP%PR>h=Wl-w{SQL(W|4~p`$|lQRwvQ?lL`@}KyNR)v=>-A4}w%3C@a6rBH9B2fzTl%3w$4V>cRVT@r#ejtxD^6RRGS1% z0_VxAZoBtR7K~aE5CAv+J0kk58ZNMatnAx{(a^-Pwm0t#U>qoxIx#?w*|E*#4U7-;Nt{t} zv^V(@c?i1R=Gn8@5b*lDpdIHc| z)3`d96<3Wr_hd+3`3}>u)&8K8c|_P#yb_RwyuRn#b>b{#-QGqOx;|z+HvNyk@&RYx zbF7QYQaEDmr%>q?c@CZ%rEI~B_fCn+QRpoLqgxDDe)zHTlVi?9{_Q3lV=h2gm&f+* zjy~9?GiH`S7LVFVZILa01pfC2wjabnU&|V zXODJRC34k16>9gqjoaR$+N~7E3j5UH71A+#-s&IQXZjB|PvAu>xex~$ox$Hgx)S$E zePbdz`Yz=oFOM`(qiTUS-c0rWzBffn;V@Q)fQl{OtK%vJ-SwQVj#2o0;%i0FBc(Ws3SN=1I72=zAJb!(? zTRFFT`O53+~t{@foGd0Z7`8q?A~W7(wh97_GB+F>BQ?j3vo!W5^7n@{}CgP=CP-bzDNJ$31w&_&u=qlvYXIr zE729Ne=I(Fbz%kJxUU_wL!(a(@#lUPV|V$`w5L`MDc)0ay1WqZ(V2_SXkHGeEP@<} zk#*(1f^8J}fviGKD_(UvPoaex#B1B}xzH{4n-Sk`e zyhkp@h|qgjmKjc~c)YF%-|X>qje!{p$;jyhv@i-)4cg9C9ZHf#ynh6Xq8XFF8bCAZ z?ByJBBz?Gqg$HEdi0pxY8*dQGGi1D9e5X__Z2gJL*6$tCwU@{~^5M`iTaESUb0fTE z^0imbh0dr(rpa==TBX*Wtg7OX>e=#Zv;u6n^8@22$p=JXW z&HIyZ${9(VK>z9fUL2hp;p{v^Ff(NJ+d~fMst0uONEb$!dL#rwC|CQX+VuIQd*j4| z$R&M@&r}D0@}W}U=<>(bc@DCGd6zf*&EG3~eBQgEN6CJh&6F%>_J5~ZW4o;>r~^p>JdL;0>1b8f$&p4ZEUOM$ruo}i`SQ%D>je7*OGI(F;Wf!;`@m_ z2E1jQ+8p>{R_n=t`?M7n-Zs$Yee`V4Hp zf-qTLU6NVrwU6r2msp6n0i`Fy=|$Glf_j~&{nc!Sp`lF!^O(gWkzwHmG*DTS({*y? z#{lP+sfiqwmcRzR!d_%a!G%dsJd;yR-tnXn*o$#Q@Cpi^pCa z!>fLAuDx~@rvTq$z|j~GuAs5&Ue`;{HVa3U65vLW0fPdS_3Xfm{*0IHv$*OM^R&b853LG73Gu&4zW zvn6XVvcj^D&j*r@ytLw!=c$mAE9^;6mHm;zePsS{2gk^s<{E#^pgLzHnL8r2p<%jb zmHi67p!WW70k4*X1~Rf5Q!CljsoFZY%Fx04{Ns+z8Q=883%^-VVu`|$CA^NDOP(wn zCL}FXQlrr*;TEe`@l!W2xW>!Gtu5nPOcR03-M-p*sDI~?`fpFzU~fx~a;i?);<NV z4;4VV=qNwz7J69icOxoj7haxfxDYh?o4YJrXm(T8J!0i}IPkqM62G=Hmuo!&|4M;T&i?QBAMeKco zQytyok+rw4YviFrvXC1l#&?90{1@u^`+tE4NDfh&AF1M`a&kg|&_m=S$A(Q#&Y#K1rN_l#F8yx*evddJF-AHmCjC+wxudcXs~z&U@ere|vl6foi|cS_smaf@ z5iv1nXM+i36XDXIzQ@Veymr%YG3Ebx()%p%h}=R<3Dyhe%89=>uAsaLz!E82O<&r~ z-x>ExA!0x4O9!wdo7+Z>RQeJo3!ILO2S&_>M*6gRQvlkQ?y0bBoSM|o z?=x^o7o1b=gbE42TbnPW2;Qe%9EJhA)-zLm!}|yG_XYx2zt!aTh%G+&pp`JNc2vg0 zi|e><_sWsd(xJ=E_U`mOr5v9{uZpuy&XQ7ysn?~dodS?j9)n<92&QH5!- z%`(;7Z5`;}O5@{NOV|DA1apvG7`0yi%->ROtlNN41;v)K3q?KpSTmTM8kPvKQt;ah zAtXjBQQlAXgMDt03rLn>r|iP-c=(En3^O5WH+7Em$`Em!_4A8+B*&z55BYZ_!_2Y> z?kb`ui{ueP_Ph8~OmhHRghPM~7g`$h<3ZcCj*-wB+j{r7n)l8(ciPv6?XyWQl`&?C z&r~EPmdlw2!FH&ys+Xm#l%D{)O29ghsG`o>zV4o=FVDCv5L`7zbd0$d8S9HvSG-8f z=mbWRXX=wFV-Lf2MMyyi+F}1PC?~h!AdWM>H*!};CikUI^QrB~<`rbUY#pH9G;>7Q zK^`bshgH+u`?%^DbEy&Z?zPtYYPA8W(OYk-w7@w*0c!-kxe|C`ze$~E{R~B5^a{rdF>#>>R1ihu~mX;Jl+%ie}t%7*d=KPX~`1 zG+95_#rY2u0IL(9nv$w2=XSMO(?gL<|43T~6_qT<5_31-YuOOH)83!4H`prK+5ySw z%SAUCos$~<$MF=03L>OOxDw#fKGSx(g~chz`4I*T7l1gM6O}5Y2pLc8YR0D$yRa%& zdMkaeK2%viBh1ZiM_mw`?C|-j8D@^x;wr227zMt_NFsI2%cwK}GIOSWhO`Ae@#jVe zoq}wU1|=ma`C_IRE2a-nt0VZ-;f5+X77khlB?v-i54u%D0fzq(e3jOCMEotm5RbUT zkCjeLN&@j-Kra_KSJq@|7Us8*7Ce2?+-)ar!Ethc89r_7s5@>jIu!$gGt+TDdh z`c)hkP^j)#E!*EDBX`oo!9Xirv8fpd$c5cai!SzZ#kbURy6`|CyEOOPnp15t3o5C} zAkr)tJ=i0UYk1>mR|taz5e|Py@_h{2>wZ(#RE=T}xI^{S#w6?LwfgTow-)kx;I{o`}VGkbP=^tv2$27$|F~kT7So)^my^>ko*ad-_L>LzM=v*qhY`JrSpXQL9 z0wI3oPnUFoo}dI~{ta*_$;Pg_tws#oAns_#6rWDiCjG?XXdto78c0J>&Rn9ZtRcyN z0WCb44-27EBsmg?%MNslx?RtfP>fBHy>dB;db zp%8PlqH-n$TbUP-!cG{|%VE#WLSgs z(GKZmF1yTX4ZtkG{$F=C+PZv=5s+tz*6X+_*u>8Xejvip>}3 zU526CwDE=i6+3t)_$Tk{#y*T~ex^;+dcvoqN`7)5C(09r6$X=>8UyUpvuhr>UbKe( zK6~_q7DpkTm%~-2mEwkWssTBb%q|OGonWC1kv0!4dGNhaXWFTqDOft=&xYhqA|4p! zO81y1$orcE_dGQkh90Mrdy2#@%+xD+Fv#b3(F^_GQrVwjv+i!ErxveznKvz< zN<=|E`r9=BsXEaj145{z+mXR1TqyODt%nRR`z+sEfU|})qUIa&CuH%RzAxR%`$L3XjM$k>|P(H$+~De4!Wi!*Eee35KEF zK3B#97IJ4Xb4nnRM@;_{4&W7JtT_PknG-5rx&X;!WYe_-615@;%JLS&gOqNiW_^DH zTEFsY0a>t79odF)1>Sn~b*o58Dvut)EWFTMy>rAjIL{KnoHOTqpHLP2%E~M=R zE-SsDf|bqxtbCG4Dpl592lVHtPc6z1Z0@?4M5Pn9O{&6gVoUq~f*A1sku>G|IQf#8 zpxENI09MH$I@4@tK>JDVE%)dR6SB$2lD2xQI7KOc!0|+Tk|ik0lMqk7!DAgheWO>f zt|{n<3Vi6n=7$Br7Swu7S>RUbJxphSP5c`F{OlezrCegUEgMY84nfS1R94G4X5wrunJO}9j*V8O-JI?f( z4R?BT&*bA>f!-TZMorMjKdBEnS+O1&)(uU5?E^UzOCy%B{nKiZxG!zEJrf6H)ARAU z^=1#uHHLw+WPJTrE%`pKz#PJaR{4c)++HzG*ObP5m?1X13Qx|jF*Av2yT@hi@~{`O zS7P0R-Pa6q(sI-(k>Njz`i;V(Zff|3Jm4Gewz#{g8r3G>Cy16Xh5;RYP@B=C3tdut z&Z%FcHdtF=wrD2gmU)I5tUOcsWmCkCEJMlKT0P`pnkZ&Gt+WJnL>5hpXw&`lT-p3jP|O4Jg)J3W2Kv02lE4`2ZyQ!~r1)Lg_52Iv zR>T1;+EY@T>L_svr{ZON<=I+8C#-p{&>$@uw@xZWj5!X-JLJw=+J8cE8gv?WVpUPj z>Eon7VRJc*#aYNWn*tfc`79khyY*F><-l-id)m47lBjG_oKN(#%VzniKqB#4@E`%- zsmKrsOY%k`Ot$|I*`+(9w}r!chV7|JZMx5B%EGzUibZe{N|YqY6niYJWZo?NtPJIudICrd}pqg=4OT&nnzQMhl{x-Ebua)hVoc#_4g^eu<;ct;TLp z;7<}&o1&iIc%(u1-lg^m#`HERbQoF?&=6G{lkDQU8TxAv8H z#V<05rOt@d#$&&H6Y&KnEXAL*MgP#tMqINJu1npA=yU%|of_S3Y}swB8_8Z-uGgz3 zoRb~18JzoK#f}(1BO`CyGUs#J+_$o?yeIw-;3lHTt%Gj(23b=?uxtE!fp{Vhq|&l6 zpV7je%TgW6FzgSz>}B>3?UTXI!aV1)V1Au~a3~#TeE&uo7BwgF#I{T{NMQAF=!ht6 zDU-cWu5nS}I7<0vv*wPA(jamCaG_xzwij_*Dey7(1oyGJRzZ(nh1UQrLiZWkkjF0d zr9XQSKB;o_FC*>UD%^mp$>*NLkd_(1m?|ENGIG4WeP7;Y;|JK}OoNMYB5W;aY1it;!%L|S@40HYq8#%NBSgx&dF5gxMjjtFu~=OFs7+mc@(9#J z@jjNSbuxA4t{N=k^-Qpz_%Vc@#c+(8`>KPy7r$w~iR4pr`RpgRjh@L(+J1JqY$GX9 z%wumc8v}wMW7s)Qv?k74IMzm-^WhE{J8BoTNxbca&HDK6kX zruCs8G-Uo&fzQye)$>gFkTHDp)cNO%5IUx*2zgsuh*z)akHm3~k@R@AUFau%S;VU< zp=6Pmn{A~OK@pjQsqEFqvvGyx1~8oOD-pgmXep{)-VA+o<(9G5@OJTECH6 z7Cb^fc#raJ$Wj4ZiX8~ms0DLJ);{)tg(dS*{$gn@09ig~PArraW!Cny;CG`OE4>0h zSNHNszjudv!ffU*TmFNLF91N~*vsk_W2eMhU!Us(t9{BptoN(X31pBrp-t!IpN)W9 zVJFnZk4hnA54$ZNJFE#1O>@OtJ!NF@1I!f9YK8u)%V}pCZ@FZlHlJ)r|DGKI%k9n9 z*X1TN!d#Va*9s7)lnny=nx zJC4(F`p2AAEuX_}8YH)8l=msy*Ol8RsJr>mu*71A#NIu}xWE_ZrKl@c#^R383L#`e z2(=7AvEv-~9d$_R&3L13D~Pn8!JDOd)ca&~7AvrldTr3EDlV~UqKEZ><;bUKUOv|A zsnzttp^5Mrvmk_;)-D#?xK@`YrOn%h@;mh<%1*)UDsE0Pag)Y!Zz$Ld+?!D3-Wxq) z@V4apdyV08;@Z^+yYuT};VBvi64%$mAVx5t_;R{7-dkH&e|q*gkAXLT2)@5pE_)<}aIUC9AZWS3Hbp@S zkQXEDxz%^-`u?)J;qEsuRw@4k#+C8ELi?8ut2xg(&rWtYCR~ zo8ZZ&+~i&pB3a_X)Loo{sU5T$@)TE9>^X83ADQ>?ToRI}7x||BaTo?^tmD@$lV?&M^)Q2?aQF zB7ImZo4-ZV6=zk^>xp%O<5K@X>J@Don?ySGo)ac#r-eP;snXAZ4CC158_vNL?W0k% zXXL+Ho?Sln0^;oLZjq_X&Z9ivQ(Q)#sY^Y~#|v1n{Wuv!QhDswyDoqUQGYxN;1yD! zeb%rCa8a!y2KiGa#VyscH84nmip=!P2sM*&k%#z%x0I4*Fl$=uvw{R$z1yO74_dvv z_9Dl}ov&YryMMPJy;gI5V~KQT;%cr&DQWES^Pq~xjmj8dVI`cm*y{)L zZ=jm&!pJ=?!6c_QzKwaK9;e!neu}(mF*bhcj6k1a?BTNzce}n!D>b_y(%zsSGe?E{ zz7}cj4Cg;qGy}nSLD5AbQ@&fiOzV`Ab2^9 z@tTXwo76)0=B+AK*Y6EJth3xyF=AIYcZphMd+X4_OVP^IMmCMxsj?ZNX(QCdJ3Hj+ z0~sXzK+c5@jVXoA+1Gl~tG<_&hqIcF$!)5g%KV+*=7qT)TEz8pc+q;$> zClzl99k*7mYSDi#nxa*sRO5P3mlM&Gbdrzjd_(i-!M|Sy7jK^LzLW0;9Y{~? z^EY^N{fikXU_I|F4UBlHk6svqu(0y>X8iz@`zmy;G#q$*Yx-nD0NZLDGuwZP2b9#I zyTcZ2PDn+2^EUJEVeUo-ET-&_^5c)ImWsuIaepKSG)4oCmyDB?9WJJEktS+sG3xxbF6dOTr*rUk=j`O$OrvyEv_}=_7xRel4&zyJ6z8kWu22OZTsLxDzX1 zrjPzNyW#r1f}_|JUYbk17b~&Q~y;kaN>X4n~ ztnF$Vl71~u*WSgX!_F=;f%dv%og44)Tczmsaf^bVQr8<=&)ALX?(25Y78LD(nAy%O zlqdU*-a!(deXJ0;sPAq4`3v1{FLbE1R)3G2Sl43GIz_-neeBkUp3m(-e)~~V@UDm! zqA>KiHqaD*nw@T8?|e3eUyrsbrF7J{-@|#}&k@<)BDBwGlv@#G&OPe%yQZeh5wZD! zne(FfRImv+bdUDZva2S1@7WlJ=w@_X6WKRlO!_cnacM}M3=lO^c-tS>TAs#1AZ?W|IsuM+;PyU(wlk*GlZasVBT`xL;@2!mGTd2y-NK>y@Ki@4MGf54OeGn}X zclGugqRF`R&b5AJwr-HWPO^lAjiW1raW3rU*hJPrr<;EIRJQNYlXEgBGd?OG z>y=j!nOkih70tP)079bzU6d)IYS3(+>Hr<;5rY|Hw-h8$>S#ewjNGJb+|92lZ( z_N%Axw3ZeBrnt8j21Sa9dEm!*a5WEnPml7v0PDFy@n+CI$r6)e@?@bWftqW4`_#n2 z(TvyLq0<%)4V7WW@+p*QRR>zO_S$`CYfz3g-&&PE=uKd}1T?wwNh6CA4Sk`6&}TK> z)J9pIZ(R6r5dup2dAp$^OgwI+6Up$Kv1$Aj>VcPsQ$}@`%o?|ibM{{t?+tFsVC+=o zmEG^k2cli8yjz=Mo?+Ex9-_)+*Q7^TKJU(mxXfhdi^NAp+jP1&owa9shfk!#;)^?K zqV308B?B$d+4ax|y@d5fW8u1lM#0VJw78H1R`iL8b)!166FHY$dhi!#PTLoB;)BBIZPj{50 zWc-t?iXSj7Z1^0X5Rs0L?~(*w74lWL2%mG$6wb!E_iM28L4_k4VYF*7pTlnQ^i;pO zGg0d{_O!ybeAO#>(IHPU)0^>HPW?(rs;>S)L+4uaE%6bIw0s^|B>P9LpsF>@o%Qdt zq35SVk8Cc^!*i6hu{mFmdLdarO>1|dS zs%fc@ez%M2FC};5HODReny+zxRF|LnEOs z?~9D5pV;XAkTYj)oKbuF_$~76>coDLvx32%OMB7J*BOBq8N(P}-+J+g`5fbUse6F4 zPai*Gm^l;lJM)Ij=Ubm|1%1Eq>HEx?dqE$7GZ5)R)q~_=u-C>WhBkVz>a(u2yY912 zx5!!>uzi0}?x#x}+U8E9a;Ld$C8q5ssy3w(==p@PnQ29b$(BTF$4Z&OKu4ti%m+1}Y^AhK zP?}qafQ*Eaa0t9P$4I(K0!q(Bf(y&3z-u$IWT6cT+mMM@C}m!UGl%qy*NMC=H>QXi zuiUK}S$S6(IKk)?DhK!KPhr*GXp57xJsV9ySM=k)P()^n7pG(cLOUPZ z_zJBLA;N}ZuZpIUuNL)FnoO*<{K+Byt?JKlyKuojp1dP4^M7{8@#+gl2w{ZMpZu1Q zr~zHY+Xr#5*|I-e5-y@$>PVJGL6c}K^LIt~Q6bv?^th-1`)J=J&awxHM6*Jxj>yl``Exf+A@Ep>T| zQd9wX+F?)DY*SYCgkE&wy|r<2yrFtR6+Cg-Vo;_qw6h0*GqtVN1@>auQ+=tOwVlQT zl+^8fTNt=^-cs;jzcPKnqV!|ITlVw}Hy5r{StHLN$LVj29|r zPJTEIlG3bSxx){s=?|Hnk>M9j;#QjFCIIP#lS4v=#9ojD@Gm#4<|E6F$<($9@y?06 z*{*xL*6<$*aSXU6kJOZF{ceru9zM(|_=bH@9x}|6B=)Aa)sl)CmNa_bl8uYn`-GAqdgbUs8-l20R zaY1L8Hty{AUDQ+koU!SAvg533wijYn73ba!B_Q91b{K7=1lhba7f1z%bKg8GuB`oY zuru8iITI4)-_L%%!5p4lrjvN7xxw(_Nsj+=j$EjNqGzmjj5u8Qgc<656Nr0Kyo*~@ zOwK4Df)rDRc~+PUBxPy9?ozILPdFew4P&mfzm>& zLThXRd(A{Ft1)}G^2w&MVBJndQC*N$YFJdp7O~-os5*7LHdU`VgCsCxI{^Mvq5LOa zLYvBaDCH5v%?@LioqX3U2zMtMIh5UIyvo_9NDJJw-`OJnb$ziYnNWy>^^m)>W24lt(?i^evhY< zk>1cZop|NrS>?&O3mFH_M}M77q|}Tqye>MSNu0QD^+o${Erm${||RtnOsD> zOxpgR3}t$TGWd<2v+pc#w3lj^zhj`mnuRUjhOP3!D6b%-d^YW5Kh0U>^D;}3XT!mS zs`6p8vah4F)tJZIoMrH+57NVEY~_7X-UOouZ&kYEcQt*x9~B(i|IBC?;lAWRyK?Hy zHG7=Bg>_JQX;?9hWEcV9IrGemDb&T7zWy8U&{pDcY1ReXSN>)`>(9j z>HnmeqU;V1%n6N!^p(qd*G!6UL}=FHdD(Q|srW;@nqSiC8{ z|C@lb>cr8y=hT)9RxQOfHmR4q`%xZeA=0j$MjsRl_`9jL9ms|&I_xYOpF8ShJL)X~ z(DFM+ZOU19m-QOxOWn@?n;+Xr5jpf&XnmPM=aZs;*QVIUy!$F4;|behrP6)})F}#1 z=kzC5`Vp~D_Wq?8)l+;PV5M!0_FVM`L1 zz*a7UNXS*rbd^!Q)rE4|55)H7`;I=Yh|(WTRGyx1tK-~L*sre z4OP--tb$zN74GiMeXstOfx%QAM%Ul7=JPEr@{k@W)tnlDmU!rPp=aGnN#Xp^B+@0; z@GAID^LxI<;oP@dKFC_0!5xnV<~+R5)W|^_t22Y{CTLUF6@U4zGqXFyj?eSP{+HjxZy>xH1>7-AVB*P z>S$fk)c7?ljYHb{yKuV-Xi%`)v%1SfE@=xa*0PaTznmv$*z^9dPv3$RT(OLQJs0tX z+Q_6td13O|4KOXD%tO!IwJ)8wFIq0~P7bZgXis)p?zPyR1_kly*wzEnR|&aBUZhJf z8N+AlHePYHl6wN{4Ez6A=dpoZX=^qYHLtOq)~w2o;c6By>?4g=^C*zK#n7#UFyaF! zE?DygX{6L^w96Em!X(uJHm0LalUkbGuS704$V~gyO>B?-bPHtqEZazrRYHQ`ZdE(X zwZrXI((EH$SL&6@g7Z_lTE1QC0=6Fp`i50#DHc-r&+_)+#27c5;DH+xxo9<>Ncwjs zKgY_(ueNtvmsy3M))z?h#5`0SxNxinwRYo5Y~K1;tlO)W@$ne&zf9RcZ5}x5wq_*~ zrm>IhGPEEq1*Nq18Mq?=p#re8{k1!sITB0*f3pzoI{TP(?|BikY8bj74-(lLV)9z; zy>8^2hvJUsoKl30e%z@UU9tIJ?+53t8f+9|Hj>wb^aDx^S>ZiT$Ie^RDmATdQAdtT zINqJ3A#X2U{?x@Och&_1d@1xKt#H=oA0CnK3V$@KvRoH*0R%Gl1{>BF)h}7g-cHcD zj;Ob7C_fKC$vUaXfZn5S8h(yyO=`d^ep-cdu)BlmTfoC_uqj} zQ*X-Hqo_!Ecph8NSD?zK-_))j@jXhde^^cE3X`wqm=HHQtPCjhRO7s8-XjFurmf}Wx^Z7R`(j2> zek?jdNE;#a+Q?QnfI|NVgtaayFvu$35p)xAbNG%}R5S-)-mT7OOp9LmlHFxsj6#h| z4QBMQQ3_g3m1uxi)Q3;!>%QG73yxE(g<~)Sq-i@gPeFS<;$wimi2UN*3*{dnimQ(Im~V|xct zMSZ?k+nF77Sfff^^GDO;o^NmX{>M#jT3Dh$Pqv_XL&|l#6c6y3)KbG5Vv7@dE$jFV zx!$=O4?&4HG$@!e>=)W?<**zK>{&x$(E*3EL^;$U-}Nn!fG$dd(qC|?RiwHvUhXT0 z1a!noJepx1?VKmQm;9aV$a3E>D#S26s;kEho;stCAg|>ZOACdV+xYuoyY?1Q$)%)rlD2$jOLlLJs$4>Ery%#t z#KK_^C(gW9r@E#QHK5paucyRCH{WBHPm?cgX?8d#ulJi+-U#`ZXFOlM%ygw9eI{5?;=#BLPb7hXR8`}z{7-T zm=UR;0rLCr_ai~dt)u1)^C{PQq@n&}nF!wj_M#bfrt#&-ngoDNsjw537Hlryxd!Jc zc-nKQ=wD^Fx?MWl^)K&@_sW?!B}N5>{ZRWERD^!X{+Xc|Yf1mXECA zzLx*ZeC}yO=lFp9`Xgy#0M-pF*$hYF*@Tox)Xu+iA-t0sOvwnDfO)|w3aJp%nQQ?J z>A0emIuA9ZQb^ZBdHt#VeW5=0Fu10vX5LD|8q7x=<`WhcrJK|Lbs1S88LOA|P0*N2 z1IQF;7o0O6!kLC-lEAS0R8av;zFBOQSHv-U1r)yWj7pCYs#TvRpB z(U}Hpr6$+3He5MMM;5EG$k%sFG`o=zm5D=HV`gT*pO>c7Z1izTI^O-$GHV)qj;-B) zgE$$XX%%UOrA=wlsFb@JJv>^VnJpI88gPQE?Xf77()orbO-;h@g)T)A=eJF$t0Rn0 zepAqjlT)~%&#$S!n8NSUCM<^Zl8F+A90n_P5UL+2dTIFC_#f&T1JQ!Gr3dQsEJ_0u zy;Q1x*EQ3vJ*H;fiAxO7=s8twF1{mrqXC)pw6<+FE-Fhn+HUxYc+GU*Iw?_sSh6;u zyBu+(VN&6@mL}FIA>a?BwV>KX4uy86Ni4Qz%*wb`cbaB4j+xDqmby=cT>BX6>Dz*q z_4?ebd|%?2-FDe9Ep`eev=@6<+202_yVFnyBNV_S8c4(GHmXVm1F7+zZy27H0eH$R zi^|Ee3ZA}=Gp9TRGjuY2?`3Ug0_2A?Jk<*gMQW!d3PNyJZIy_93O-0xS;cE9Ps9st zYB)J}6>3p6|1HX?^FKD{kPWiiN&=_NtuBv{(P!y9*V$CP>-9B!%w;G-g}xpv@tPb$ zD&Aw;bN2=OT{BpmX;DxMpl}HFzn;C8XubXU?uUHd-a(uKO-v$Y`rOSE&AdfvYcOwq zVtLP;1Luh+ZNJLLdEEM(oPsfHpb)rt)GLesV%=uShA-q5L2K;$m7*o}gP7%iV|}HM zZ%{X!|Bh|y-k>IAlr;z*)Ge)16BoUe8C^t3^K)M)bNoisF{8|W&1Vi*8c7G=FM`jr z}Pu3vyhkjx5YeGi3 zYdMO6-G|%~(VE=O#wCr%O^HWhla4(T+C5+QuiQdPXNW28<5PBcwIGGpYc|yO7f-f) z`IqX&+)-u_7AnSt?j=;n+qEt0=QNW|5FLns3Xs-Gk)tO&9*W;Q>RO(vx=DmF{8H+6 zU;M1TVEkK`DOu}z@sr5DxkczWlFN?Ac>0-{etRn9Fs4ya@n7|*3>2lK8S9Z3XWhnb ze2MDQOuGD@cI)H{!1&~|G-Y3*CK-vJ*4jtw&1*$TDt=bwO@j}lmtS$5(Zno^)%S&@ zjoZC!-VF8FC5~_S?<)Lto|(VW>c&RMx-9@5=lrEqQG1Xu?ltFhOG0G;yOgxn*xgn( zn<(M`Q@Q$gzdNJpwbLL{^BI3;?MOcH!H3L2>9l_NiA!A#hM$TQQX|6F`(vJO=G!Vd z)Pg1_^g8ZA{cHGQ^itJ96Y@5gXWB7x?iEToGo%6%;JS!u|5!&w=At#nAV1LqcVk7;Frn|I9>4u9WE188yb-Ds^R z(Y%dyZ;|#d^>9L(<}#PXqp__PMwm3B(j7L2<3)N>gYHi!!H=-^!LZabf1S?P;hLO>Cwruy8VEt1@W1zrNRi zn_0$3w&0s(_l}z+4x1K44O4f*87n!vpn3rZ9=uYD(+~3Q(zdQNu{JyuU{rLsZIc%B+ptNjO|@bk$NNeH|-RLHcs}Z#9ROs+pTzz#Hm1R*>)iw zw`qW9V5^1!ZZYg}B@ZjG~ z%>+5tD9@UK6gkE5v$wmMi%&KpziRpa3=2IQYs`*4yL(r{t3&QP*LR-17L;Q!zKa53-)&-lrvMsWSY0+|Lih$iXab z42zFx)(-&#F0~IU7tC>g2YFBnqo|a|*2(}AytWymNli&8pS*W>%`!}Ol3W+MIATv2 zCHLu;)wxQRC+=9W75Pl8MB%;^r}$qnsy$#zJ$^udk+EJHJ=;E(ANDYy7j#I!Fu9p_au^v3)3>Mh;nmOj zMY4_B%>N4D==3xKdP@W$rw{6P>zsW+)R?pFp9?tP5@z&RsmGw-FB$o_wJpDwyiy*} z1A!5{w3*?%$VMH(wupZL&<>&GVyF|%;lzd=Z1okGn;Nt7&pC6r0M8Oo5S)w8di))b zt=+Y2(+|DZO!A%13Iv_QXB3b+sAhMhHR#j-sJ==qr(%|=XdzZ<xCIfu3EY zI%?n4bRLf81L%kg?YZG_jv{1#W?M#^_+`1-OIZ2uTXE+u#FO@p{W)Rf>I%UG{OHs8 zh+2s)|RN{qhV7o_)w$|=BDFa_fzfG4nRh)r{+L$Yv5p{FrZ6|!woTB@+`7{fq_ zY9$%o?H2@>J3i9}8CDDl*BvgUb@o=p^mK?76wTOH%(JKMwPXv1hB35Uy{SD_R?+Ix zJTPnBtsflagi$<9;`tE)t zs-v4%J)xckF&v2dU+ID{*!$q%0_ugCAGN z@iN23^bfL;9cOm;wVUilft0`hPm!o1))z1e2yM52EC&Avm zX`jXd^6SI$%8+nx4N=``VEoebkSU)O%v?PPqJFqs>isb~r~P_jsYcz5w*I7mX_n07 zCA2QA91NTu)0tY-Al6tZ>uQhoAl{DWv)Z`!wDsC%&>y1}c+apwleB*{xV{WBl3Km+ zpf;V5A$YXZZMxwS?c-QQ-e|BvP#BE%bZ-=7e$H^vL3c0jD%#PD46DeJDT00$>fCxTTDE0=~q5_Ys5hPBn6k z)VgBklWd$ISG{Fp(KJB9RSe?`XiMfok6CT{|GqZc`oRgy6W%?AR^^(YvzY8pExT8} zpp^(+jXB;sjxr#Xck{pJ9DY39gTI;My^Q1FzR^3qh<@X(Z_v2eEO<=$TKQCchVe>I z*ke5LiZ6mug5f9Zu9wjJo)5k#`Nd9<09b#2`r5rBMTB=tj%&)brfuC3TKPj`XUuY= zQCfK*;V-wzZzgR{>~Kq5xFd}6Sr7LP6}X#Ef1zNj^0Cqw*XN3eRW%+R%#bVV)UM^R z8>%k2A6>9x5)8Ujza`d(fAX#x_qs% zuMh7{?EQ)hNlb8m!d_8l8M?u9Q6m^!sOAs2BIJqss<=0LA9RRlzUa5WlCF^h=tEx!)>H;39Gu6__17 zGP>XeDvv|R$elL9@YK`SC)}Ab|DxXk97FMbusLY3Kj6+L@l;j0M6o|y`j&3k^fc+&$8iKoxRA|-Z>;80 zR8KXRmg?dGbqm+b)|{wSq5-$x17!L)Wj}pGGnD1Z1OJ}?9wgV>R2Vjt zJ$Me_@zbpLXUR@`pmHP&3PIdYUH3tj91Kq?aent({=&!4-(&GsD|qd0iYu58ob|X( zH%x0{SZU&B#$&1s*AH2*v4+Z|8-z0p-JKqhXY<6w0r4G%OmbXYFFHyh9Ilk>EU+-* zSTS`e24sBm9Swo%Iejlm1!0Irp5+CCWVXD=6Q5ED(K3ONciSt>(?&4;m3j zoxnPU)g*buDW2bKYAP&4ZFv!VCd)?I+bi5blY} z8hQW#@I|GI$r_>Nn+wK6=3V&uNM7G+$E_XM^CWcpR|@@ZH*;d%epDvnD9@>u^m3uTjH67l&+ZYXcL%k6Nx}YvK!sYyL+YDQRfb5#bVf&l0ZvLpFCJ3BYNzwtXxa9=CaDF)+eTkt;DTc?IY3$qoU{n zr&)kuUfOm==diwB%@-xpE4eJ-E_QEjnVz^E>1UyITnXXkj9vaJm*a!b5hne0}7p{=lYY@3OEUK$BkqGjuQi zGuNH$=05Y_w!o=4)%_=C;Ks0xSncuY_mmQF44-$r;F=n#g*}aI_xro5b5xnY^!Z&~ z+g^uoP1C9n_UC_Z0x`>vArPVwATDV8sYK@Oc5AU_4F$OkQRnN8B;lF@T{{d%w`t}ET8HhEh1y}s%R|Y z5eu1D%p?Q^_Rx$|2s;O5{i-MP1MLy$p7p>iZBonBEN4T&8Vn(VeVvkB+)=+}z-!y; zJgp#d)QwLr9Y)TF1?*3^vhY?lDHciAH{I`gWXn7i_{_Ot(iH^WTtSyDeXyMujS%d~EKCgU&D}Jup=8sDROeQ_Ik3?|A>YH@bWH&!mD;2u;5YqIc{Z=m!Mp#CrIxXASsBg@mo+rmciLmt8-d09l%>FPYCBlJJHeif9 zo9Dq{BZJR2nHtAh%2UG|zTKtvcU&ymvbQKF13bs<_M4bPJ+&0@WU%W4h^dBl$<~w(HI>(!>TqaVoUZTGcQ@Qr zpOyE*#>iQq^{mAs2o)bYpaS(&fJXt&vnE`M%UXNPeavyq*KI5;vPIdb$$3o6HJiT( zC``VxHm8S!l#_21M#ya3nN&D@H?lYh@KM0vPgjH&SkGMjr??oo6okLYLv=SdB=_9b zoaBrBt-*9?JLzl5qU3&AsKt_24XV*nk`W;MDyz|mra@VRm8ojA*^A~;i!F+pZ3eU% z*=-QZM3ftq>DN%y`;|?*+Edf1ei8 zE%T*y3h?W@aMs>;F#D0Bx=)e(y7wlZa1^ra+^H3+h=Z}-TmR%*i&f>VC1xF%^kj&Q z<`L>AJQtuwOR5yrP>R(EM02dcTd`9|S%5yhUFUEW_PQzsKbpS{&|{ z^VKK8X}L9gF{{02(U<9`fBKJT-@{qKDL{8E!$Q7gv%y&Ng{*qHRa(cYtRZ$amAjH@ zW#HIMPnmTwQ__^~c2ua;R{g*A-1Y@8swgCE5uj|wOI&d)BCI$uxF4UdzBzJNhfx4! zOhGrfzxpC~k%vofQkV*4+XH_gkv(+uio>CbZts@lXUSKcEGQ1lw)}s9rrO6h4MNTH z0F&hvI=f@Pi!_#UG*HI|$U^C!n5d#kKlYnBuANFO(fo@utVnUzIpe|lki}7~BhXf{ zKE&}yRqsjeKTgHZFiol3qaTFPg#+hI?mS=<;O<}vV3SBSS%;NYc4(Fcj{b5-bTv&P z=nZeb`6NP)+NMvr%tdPFMZ?P52MWOS$+c|#~xB(Si>#V#2g|$p2 zihB0RHT3^Q!s7G6<4z@N2d)JA1dM;)j5BOZ+RYl(HblC zH&!S$l4}J_&{mSpE6@IE>)UtNz|Q_PKk0a{&Mhd{ygr&>)HESkIJNj?`gMcraBTgT z?kpx_Ev(Y#Z!W*hhJ*{uFiS>l?8>*QAY`3#}gw7W9c+**QGyhSQa+|qjFR6G$ z8hsL9iL*}5H>n0dkGo8FO2yW9ug=x%q(mM2It5j_T7Kxhzd0r9eXtgIHd+foML%K4 zZ14VgJh+~wKL_VCT*wZkRsDeqs9qHJJ5b+mfl;Chzhpdup3tNck|lw+oH#$uy7<)( zNo;opQrP&UV;?Iiy2WbT*m~8l9Apf8r_lMmAazL;mS|S*3U@Sq@j}oGHb5d6^ywCs zh88(F!t8yFF7;D19_=}(E5B|HO`UJPy14oPYGiMYxCk9*2nZ1O=Z6nQGGoqT7cJbJ z*oXaL3MEWh&HrZM0>U22d9BOE!i;`{m!^)YtlutZ_n@rWD*Kc@hTzVBz3=1>s>Q31 ztA~Fp;rm6ab1YGiV9|{Q9noXObjr8zXqkoiR&`W;odTG1im@z1ah$kpGty|S+76}C z(ZJB4j7DnX_?%S}7OE^P6$NN#hu_9aRIY6&$ZeOw8rG1!s<4$eDt=khb37<0JWx zo&u|Q+qSO){_|=RmSA^GQJk@;cmY~Elhg2hBdVvh4766E?8A>KJ8dIgp~5*>Q2Ne~ zvB8-NckJHC>c`5N>t_|q-d#^OCYs*o8GjjwZ-0Y@z|GDiO4d;WU0gMAGuUELw{Ro+ z&a8$@o+hI?Zpre?=7=GBL+YNl&+O6*`Cr-US|gSMn+e@gc5y07#v(#|ACb=`B0>Y@ zkkXm6un={xd^z_o181Dyd{PbY-`Rd!jA4ZIi3l(L@U>Pi3}b|fDC2>jbOV~&+h&oRAW7zll(ep}n4@CI9`&4}x)4;RkNo0dPV8m})Zf)8>(6Vc!r!&KbsK(vn&02d`TS>Z z+W83^=cduou51`=Ehz2 z#=|-q-9J7}*T_yeqA)I8X04Kc;U%I$boSX@ASP|{immu`;-7-y{F6ENYZC+Y!1|g74q3~G+&qns0wJ;LLi`Zf&mm^_~nC{?0ReE6M7aY>)e zIfn04w$1n|D!S|uUm#)`|tT=&{d6eu_Lq4|eWtjZ|pcHI0wQR%{?{=2GW5VNB1Wi<OB*@k4Q(i<@m;dO@xWyXxakY5pHT0K9st-}gSQ$uz{fz0~Mb zG>ta%+B~Xi(zQ)?m|c}cdi1h%wOnO$t{B~G3~PTW;ukMGie{@$oy%nMZ)G9enl*IE zj;Js*5h17`w)<$mPv3wrQ`aYI`kO0KG4@scG+mRwh^=9sn6l;5q1mIRY`dChtY{sf zO$B#9G~N$|3s0y~-OqrT*?f5%k(!kL;anA*lav00MoLeoTRg077PS9%w<(cep9{Do zJfRtx)6O%KF%k3Hi)H?sr?&O_US$D37-l6{CNUhvE(dcTgI4Wn)Lxa~hmozG`@G>Y zMKt}=Mv&s_vzq3z1*=21YOnjv@WZss$jQ7~DbyqyV7{Q(phqoy__L2#jaI5~Yf;FC zmK%{Lqj1c00Cn#Fxc`gcE#Q5Le?pVX|Bc#uxV#~z+Jw`*N3>2C z+meToY^MxCV(9cU6@F@2Ntt!n~>}5gt5tf27y@Fgb>uFVWuc0+G7D8^j zJyP2PWpLP~5a~~-m16o2`$cgH<#@C9iVZLRWfAGU;5^f*;4R+Mkjvzzu}id4@;X%$ zij?tBibzhbT#jEwd~VnpwC@io`>44T+7{MX-#;Gv@R%H{)P#T@{|OJE&BqBUVfH=- zxIMx3yZ>$NCRGniKC(1yi@*8a*i&oIaqCp=aIg>@d|M*pLNa9_;?^YNOHU#K+SBPw z`>+B4q)B_~F)CTLInS68kXrHTe`>lGvwshe^75M0HrBa!zi5o`U3pNcPB*@CvFNc? zzp*;s#_|BCPAgjo!jtne!NafG)f8+TKB>?xrf|`kt^Sb}%Bj7OHM<~aKX{xUXjkfK z!G=6+{9M~}Hh`S`&IoOW#uVX}$SnDEXZmD}#XM9st^eTZ)a*Hx8XbS{kN|B2Lc1?{PI8M#fzuydNsEx>V+lrGht;kq;WK6pxAc{$5r(YU!KVMi6A9|q| zF`fkTck3)0FwJo(0KB&Aw+0FXi0V@33#essFSnjl^4E$mt*uT|lZySTMy-QRBbvU4 z(F0o703$PGy4(s}XGG>6Rqtk>&49Rm?veq%N1KP7Lj80(<_V~_Y#{)K1ApRK>(zNC zGTxmF^Jgp4UWCl*sScMN=R}4b)P?)FE!08hLfbvY3+DZk%68DJT}RNP(`Y_~ZN2AKlx+>O-l0Pu8=G>%+O3Z5{t<`MTOa>0JNocB>+0@# zuis9LsgVNf50~xpxur&0QXu2y?{$PnT9Oeow2bv1BqXSn1!6K&HuF+dkERS1!8 zI{XKk#HbI!s*n`a3{jgxIq7qLUAAeJ;XmjY8ln?rS={!8s{)eT((g1<%bU5%lA!>hn8=fl1cGibT>&#A{2=KeMX(JsyqkGa=lwa z)$E>gTNc?;r~N_rj@BRcN6%CQrOO1Ab2k=uHjnk<5AXZ zZKj5IXWX>iP^^YTi#e-I|2AeqWWH4uX}_2Aid~#0Fwzus=-BJb7mHsjjDKzv6YAVC z( nYH-NY9AZmN4_{e3vQ;%1`?&cWBf6cF3XG3GAhQg!J!?)+rqA5yOgq`U6>8ae zvtno|azt}rteD=`&Za%lyPD+Z-q$@aHpDA2Irc?B_Zuan{ZRSqlx0=5Zf)f_ zCHa0%O~qIi-!hA-++%3vV|0erL{h%V_*-JU$=Y)>>q5=U-NkoZ}%)b}DVyA7P#bmh{oTaz;66rDC;&6iNHtYTz#! z8`A}XEZf}seR|g$fF*CzmB}(kXD2#KhwS`bnDu!c0T?!xZ-chvkG(*ywYBxvqe?;? zAqxgC%ISfyn|^Q27W7?59654?Q%IMnubnqJrMKSZ9N*6th^f1F zlr$D7rIFDml`jsLYdP{^oQ-G`xQB{aT=ZrHH84z%H(J<%Q?+c~47+If3q9EfqvQks z?c$RM&P2Ota^u{5Mllzwjvry7z>v|+$UbfX!YB=A2m$=3jQU>l#d#XxUe_`4ovHm;PMBc+^R|eL zurSk2CC1yziaObRS+vXNFrxE|c3+^*D4Z)2;iRlL7!RM$$w_`k?j>-75X))`V{ML- z(NS}7?GhXU@%O>^(PFGMNcT=( zjF69N()Nebm&-7!=@4Y6gxRSUjL&Bvd9u92Rm89SSn#9nly2qHKzl;pvW_xP+)RM! z^DJNe{u82W!jkzPE9jV^QG)!^T8K2yVG_e(eKz)s^Q4Q?r^qDeVLPSSXAvpfIWomQ|A@<(ab@Irn|QW`hg(hGq%klwhpd+H)L86Thm^EPZpd(p!; zRyE@oVh<7NdfWF}x~4(Y%ePLYKEhGozSOZks0IJRgv@!VYX^*ZEF(_eQ}LN4>>I9F zcCI*Y49N;mh9W!e8t}dIh;bvYy)dl9t3l^~;X8)@nc2yDEkCKJ28;3J^k$-TcxI!%-Gd)}NY%Yh zl{{vynQYe}|Q)a2Lq*F@j zWpHriq99Fj^esAlqb#oeaW(slASxQU0Btl*zi<)ihMg* z#SF*yTwIJ0lebembFL5|Dl5Mi=RenJt-d{s;MW8zL2wST^QKj&h zK@nR(l(&`OkmlM*uGRcVvyuNHs1j^@EkWuGyEKFO+XxYc6|^E!=*dUR3y;9FjqRNw zTIBBZ|5wy`M>V;0;a)_Dv_wTjIvzC^%j#kJ z9g714)-zp$t{$(0W0XSs!~e}av2mk}8W&s)S2Ta5p^ytt*vHa!3?B174oj<6AI&FC zF3ueL@jILNE&s3g+F!acO3i{8|L4`E`DI;s%$n^i3_&XN85I1|n+d2I&JUgnIu9tz zre`e;{T*L(Y5I|#o#g)kvp8C-bM#0R9k%6{vQf{M$rKB|v}SaksFst(%$BU$&$l-e z=6^#2c}|JRU-wU^4?WZS%VvOSeS1<`6O-qL`evJV*{13&&=v5loGjo;24;kD)<@qgj)MJ0Lv_r@ z{_2cUOcoZb8Ue{ZGT<(Iu9~>51S#12N^?rdh^47| ze*eo<2%s)(8(B11Y&|mCZm^_U!@?&`D6vIPMb0nRQ{GEVPd-ve+ZV06cSjTbiw`cZ zT2yGOUHT^g6%@C`B!Ou{f>O9DEeh7PO$iidb#$MBmG8aQO!*5RGu8g~CH;s%y_pD< zO_Zpx+U$3%2Z}55vzu#HVb_0Hy^(b7G*pwm63SBC;%q@?aWpA()+-DClj7(*BF`hC z)+%S&Je0|~A*>Lm1bj+MVHs2`&?$Gl<$bA*Q&!SdKJ)Iv<)0buOau9shD^Q~ z#P8?xd7@+T9N8~)4aN8oNM$ZXtF8TfA~QPt7PcJ2o2{d1Eol}OUszp)CtGWm3m{zTTy|>k>RXnZ8woJ`k7X_K+Ym5pXqOBj+VfaeA9x|^COcge9 zgIwImgJbiSmr{bIQR`~Po89@v3hAZS6LbZ;ev7qmGm%R}fz23SQ!e0UCUn_RUzLLmHd|k!Q zS4vWyV}wL-#0-6iB(n4vJ%ej(d<6hzmfJs<(lYyhAWCB-mRHMZ!N8uIP7h`5UK)@z zW4S;+h7hFwxV-$Q0MxmU0o35BQIHbUrd|I4?2tbK_FD?$y0_bE%NNEae&9AGBG{ zH?a)e(=q(&sOOn;vZ@p1v2c#455|6sYJcGP&TG9WO}@tA0pC!Qys(1nX~AJKzWhcm z?Bz;VdFGG)YG+flN!_rX+28#_X<({ao%!<=#@9TCOVXtug#mfl@YMK0H;o($|0RJ| zt4J>~4v;;r4>0-I=#gZfYzj|Iv(I+U9VncPPD^ z(t?C_3k!l~>mtL`*rCrJrt(B~#JB`MR%l+5$pdLm&y!sV!*j7}hMsGMmjbHSm6TsD zj>?Z@W5c2<;-s$UjeI)1%?Wp|l~=F|&jqXm>1XIA-5Pj-p0(j%@7kIM!UQLyf%D5O z)YjypFI#m7!FJNlmHZ7|>z|$+urE_(zRI{X+cy?$E)b{i8~T6|&jkm-uJb>#)b+6n z)oNOkeEwvLnHA&V%PsUs7>r+qV^z=8cU|do+`O3j5x@RyW(l`l@wrgIC*w9Ax7-#SY~*)wr0kyP z;r$n=Ezs}(@CCAIhu>Kp&CNzS)&>Fmv}{K{0{~_~ebui1-NxfUr6o^UhHdKk__EBI zSgq9lvXBMdD0%madyp_e{#%uwf$P)SWlH4_9ozdxpuX)bd-jCCl*5nkDvP=_({s%8 zcIQdP8|)%?oOvB_j*MzyhZ)J$o;}3;YRRN#8cF-5(tazOlX~k^7{fgyg`RS^M{u>^ zh%(I)RT+J_^Gjx?YRgnb_c*S*laK(DY6{V3J_1O8PFmJUCqdIZtnUWoP`|3- zR)@>na2r38j8h8;B=o2ufn?%cm=GEfR_Jft%W{DY&hxMabC909)1_EryTHvY>JD1z z?YK_;;5Eu-w7;moLJ1+h8{#ZD5I!KYk?pDum^PN>>=ru8s%eRArK*%~&bz?~!+e+F z`K^AAr+|FbSn-rk{5lD*_`C2WJa`$N0441fkn9r2Kb%M`uhO^N?yLdTr zUneX%h4!A@>+%Jf68Cl=XDA?-lxTQBF+ONXw!Yh-5N(bfQA4>6s`K3yAiZ5|b*Ej# zr`X0AU(8iH4gy5bFU0sJ4ZpF-X5-&rW@sU}kkWJv8b&yDW2-j9WqR^bXc|(vs3?O7+@Cx+FssN5Ng%BYt$fL z(Oz-DGW!#{RP?A7Dq*x{i-RxFHQRr2%nr07e=vK4ZGOKkjVQM_?Y#A%BP!ZFWI{9Kv2h0 zqcqk`1Zr(H3q(xXkrwp3p+W@8QX>%F7IcnIp`<@86Lf8>6;G{5A-8EBUh4a|oym&b7_{d`Y}fH*gIMT99K?3B<- z6VzznwPH-qQ$ZEZ-2JKAT5$pa!2RLEg+aasQ-c#9X0UEbviWk!jJ74;B%2JE6s<5k{F4g49z4JlN>?!a;I=eu zlB-H-fsQ82(mHMBLU6n8<5b3%^$9LC?e>&J`(Om}HA%40!*Qfeqab zIiuLXmf|?@7@|a^Gx~~Xi`MaAw*8?VYVpY3Z^NSSZ{jucp^C7*w1cvf;IVD;SB`6V z;@O*$O%VbQ*pdudxis^ETlx4t=x%L%d>`cf8dvWPftbrtwtX>aH5qE0sf0zc!Rb8C zU*LDNTh`=GbIOG?o$<7;Wf?1kX18* zc}}Fc{eIT9yVLh|P&d3?fE?%N2NA~}9M}4^l#s`2n1#)SxRc#qqneH({a>r0%*WMZ zjt$$MRIY(mZGT7X9ka9&7@wG+E;cw`uWJ51t*wPk@RU9GW4MqmAuCwj{rPpgnj2Vm z56e3%Dm}|7=s=cUATmWDjj(NILkn%WJIgnmn_HBx79DFNRz$~RzxHKyLv@_inne_i zUdc$C{>gxAo4l_LTT#4bwsJ^tC>NiUzTx)T)QDUZ=WQvmi!Y5$qTb$TMCha2lAwf7-wPfrWEipbxOM?_W< zwrb3?;`#OZU&dE`Ijx=UwroXl(}fALE*TuAUQY|2x>y{NgCgop2bLe^8VAy5NE_{X z6lSW6%_Nh43dEZuQo13qvV=hvg$wX_c=qldB}a`g77oT7C*`fY6TB;s8E&}t5F_1B zMu~qvU(Ek-e^7^qco~^Pbf+=@Zps(?YJp~Nxs6i3>VLNSx*c|l4a=eOe*y@-I z32ZLF6eb}kz(aYWiQ8Nq^$~B7RI(#1R^`|SbET;T1B<_D5^~e%wQd97tRZ!_HItcY zE<^Lo%zj5dpADwYR4}*hZ!>(iW!wLKtM?qHNhR#B{cq0$VmaK9=Lx+BLQfcL2P9ah zF?JhIYFO{HxO~8-b*R$#F3S1)@P#mr3IC3oa$5J`NemqFoT0v3daZcIvnECwjmU3Z zsMeYLcop7YoAfxPI+wX8DBZTZ+#v)C)-2UVzhmKg{G6&iW*1K1t zoKRjI3bXoQBJS;wNWR=L`}u{MM|A!BdJxd*RmFSxgX&r}0pAuwiOsBJBxF|0)u%cX zkCmE!u_bHbi83CX1XJBLt71uG%Hn10>mU7oa!u`hKawbjU09I1C%3Vucv(`;3W|`5 zHSn{Sj+C0BsWnYr0Lg~uRt|0Rx1MS}(mWh*_f(XaTKi%e|MTqn7GGo^f8Zt7`ItdtOBWIz zv`vY&OsS=X|9~^%f;HXNdY?VBLGzOsQ)%9cUsgq@_I=GvhK}micx<(= z?WbOgX`?RX4M@-0#;P7rUT6T>AtCy4oaZ(DzBvQ$bi}0j|8Ys0rggqGWJ=?Y$W__> zHq~azkBn6Jxr0|2J$3T$<#B~VQ+10bIFHt6+45gtdhFNiHTObuv7XT<-|Y1afO0%8 z$Kk2&IORN0pUO-uV`S<4Hx*dU3*wlKZ}+z?j6fy z)^xyQox2pBl*N>*oN-(Xb~4B?_7tQZ<@{e+P%aC@UHSJwEt2EO+ZP8O5^1y9NB2Z> z#S_XNcR9O-Ngx!^eT_v9{(3yRSl#heNXjv7EsFc4HLG}`%JCoO`h0R>tS7mi&s-k6 z9yAnQL^Sp5l>#4?54^b)#-dk_;k+T zZhtB&I}y*=uGKrTs`(GTn9OnAfx~F_i|+Zt6f0q7Kb&Q7(!hsE2IsUW2DyWzb1PB} zWZ!%Snt-tEH=o+JF{}B72aPvzDyLW2Z7MoG=uRPNO3daaM&P=iTlWFv9VTex7j_`C zfP(J@r;+#iqvlPH&OJodRQP<7!;+!x`7#bhh=6Fsz#~MOkjIn1(O(4}_!iYcp4M?J zk?>HcWmL_dJYVP)s=Qc1kIO1iL>M}wyR&aB&+1h1ue-%AbE*ML--EO8z}z}mNiIF^kB(#wAnm%jA8bJHuM}>cftZ(TUn$NX<;?cB2-7! zv?MpSifyHYVL1!K+vif{;rtqXzhXZ}(qXl0#2cJZqd&o%zAGiG){5cvbcR*8oRVx4 zJ+nO=@1@-RL2p-80*YLpWxej70>~Hic=p@jgMM~FSEsB}+{j*Z{|@z|#MGOqX#e$Y z!@v5s}N+e9K>y0mq^U9lH`vB=*`*weSzFY(KuXbiZb04Q>tCc>P`tBL|m}dtn z)k+8ELO-7V6Poe@m{nQXJLUy8EBhLe_qS+B}9EsWSY}w_zL_^IkwlGtv<+275HI$F*VqDy( z5X*{)rFiiooyL{_x+?FPZ83DbiT5hMa%HX=)1JSh&LJ;9nHWseL(4lst*i4qYd1V; z)fHpCD7lq4jP4~;;dCMhRfYCBYP#|bOh&i$g4}?slFo6;`+7I(FUJ^)dh8Y>fIFKb zb}-O2Pv_cpx~o@9>nxv)l8H5#*LixjPgkn4L=?skykw?p^*@^HzOSqG6B`d*ja0t_ zdUtZ^S}_61&0oqf#-OjKkuQc3arU{TmIqw5{pS~d5`SaiMyfVkRDuRq zEw6hOt-hlp7v#cTk8nds-=9x~uK7MU5*X+>xK4we?|YUwOji8Z>*{)EV$247zfTo^ z$$c1>3{0BxUt(Xn=@&CSzZ|mw@s3j!aFH!2FOjaM6j=R`t85a-a%2|(4@gZNA9w|` zCvGbX$c~0qwifuW0lK=vgAs7A&CR6ZqRH3!PISUu71HQu4puThvC>|?x$m~|u9ezh z7S~94L&x86yZ=CT;w2oWrwt#qX4XTrCT{hs-kZoe=1-{8^~K4mHdnl4?0;nUTJ{D} zn#X9+PLiCS;Gh$`sx<4btFT(fl-bl`rOKTzhJ1-s*7?U)w`+eQ^vAFu>MM)*Ot&0W zR7k-s#e9u&$1HWQam+&KQ-7*FbYCxxA2P+aFJEr|I@HpwMNK&&x=r5SDyiB%r!&tV z=Jcl|S9a}^Q;pkz10yo0%lrBDw75&S2FHtpmlZDHu;6n!GdhcZ(hF(0m7ds|Qn4k% zM=i>&L}-e1W7ELwL)oUli&2qXT|b@~dOS0)WIIYpbL{L;^T$sszJ0+rSSzt!9u_BB zOIUM z`5!(p&w?nvU`D5xt2*AWnp^MfZc`U)QUBg=79U|f=j%JIH`VeExfGGyC@=Qqpj*1B z@iRnbJl+uY$1$v&SMtw_@CWXW=38u#*|lcx5-BObP&fcS>(aUurMcE@A>A6xXia8v zRA`kmW6cjEJ@gm^!+l;MWXM|FQR3rI7>hpk-Rm8}V`fuIq?Q&q?emo;tK4*FN2C+t zAo!8~V>Q=fDE_Ihy-lF`C6GVQ$Kk+G|{8 z2?qLQw7>;yQXpy$N%sY}TP0McvSEj`rQ42EZLi*#*ob%i$(hSt zTu8mD=fC^{u@Db$F-9+NHM7NMs^Mt^-->BXE#|LqV5Nmv8we+Qrs@#Vj>RSS4Zd`;Ys}PRSnZzn>-?8vv(SlBr#+p(<$0 z#Yevj63XEAsG1AryLZIc(IlPm8wab4fkl|=HMxz_!)C=}D^zTWTef)(Qs_MmJt6+# zv{vxwR{ej67`zQ52a>r+{xJZ5fn&IdcW2ZC+a=*Z2?2RMSMWI)6s)OLtB05+3um&S zE~0oSpWYr`?8cYe7Z*UdA`}p5oSFnBpGStU?i+&egTe6JxsbM{ESJSTjMSpm_IDHO z5&`)GgbHVU`;r{BO}kqnQZ_Y)t+EnoBtyk7EW>x3AW9UTD9GeIF-myE zO9^$?CL17WAK2(xLa@VVFa0`jbgS&Y0do&97nz19GWvA!&hw5h%ia4DaEHBtKw6F9 zE`sTLK?xpu2)l)^s<7+w$_wCH$!QD{)YJkE&_xgyT?`2o4lXU1@t!7Ty8c9XtM9K~ zm>PNO(^9!hVdqnPh(!kR0IJe!F{;B_^6rc$ejGBMn zl!@pk<*SU?AL^bqWte#b8g93EJe}=QA`5jD>u@?p{l-oG2R1Y{;!6mg;vc?aszA;* z9Pi<%l$4{kb-?t%SH0z)=bHl zYh6D>IRUd@Jxb9iY>XLipB-I7UjJUYSZaFmRSnCUp@JFW$=syRZ`xlLum2);(QC$d z(ax3O)4$RInijRq$~U%Unz91*2u;P83c}L^-B=?lwW3nE;HdZdHaiynWApAPyyMX@ z7kS9O>K-p}>G5Nm&?r~#qR%7h!8j+qWePQs z@q2Z@=S<-uO{K4oPzOU1pG3uJVwzn(utAr!Fr5Asn3{y;51ENLl=UY=JJ=fPj)aQ+ z^*R?ZH5321s!L8vAS%?kQrE2uDpp{+mdszh&Eh8U6k|baVTck>vzH%;jlWy6Nd5P1 zbzDQaB#N$=*Ycsoc|}DZ<+QfYA@cj%?~>p;shC5vYJHXQ;X1-PT+m*b3En>-pth|< z^Gx;)oaS2wRIYiXm3pD%EQQTcU-oLIH{7ZMUomnRsc?H;=IaQ(`88%5bUy75rxO=! zXhWDkvV`Ghbmmg!$f*(#Akm*<0gj^O>Od8DHUeJQCd0t}-_-KCq~UdqVCDT-NCSFZ zzI*l-SDH&|9`5rriTJ(1wuQlDmqpffpcSDs6V$Zz8IBi$t5AW?V%n5%e|SNftKEf8 zQa}~!fw!bwc*;(MQ%)NMPfu)L4(fH&NnwG93w4lhX+nIIMt-_{2|RWoX(l64QL z|JqEtme6PM)1-Q&2le5?c?}5>R8UbEA$29r&#gGBPKCZlaPw8m+~XzaJmd#7T$G%j zM7ePm#o?GO-(vBhF{_+ZBmYKgnReW|QF|SX@p46XRO)PY;-6eSoOtNcT@oix z5T1_%O1c)$o~#f5<~t@eKl*o8cercK-DN8$wIU39Og2QuUq0jtk;7oiwZ@L2VG+B> zK7nce0P1K}z2^}Ty|a^7Z)2p=@lJ5rp3}AJ*_b$GIJ{8MI$hJJD+1va{c(OID!}@u zn2GPcb=clL)i6*$e6m244y{5=Iy{=XL2Ue{aYrAEdc#%o*4qU;wdhA z41)4rgGVpsob}w0jb>SSW+!*#%y#de5TAUI`sv5#bg)C1MU=AczV+P5%_}2}nM9?Y z(OVSjBL6DXzjAD}Am96WAQzSG#h5?!`Gg~BX$W!Yq^v_2-7K4!6svm$K{sTO732pt zuuvk-qwmeT-#T2&XqsMLYRXIRW9~|I-fMZhlp&u~r8uRWo^g{GVK+(WpocmY52@g0Y zIqPe&kjq1d-&r7)${hb!Z-j@%Uf+R06y&S~#$&h7s;l>Zd1FQ3R=PI~$u;@Sz9ftNe{#E6NE44(;E z-I^2LFo%JW!SdPc{zbq3eqnQRlk8OFy9cn|VfKN0JhGJBYJyB#G_yMXa|IwRpTM4! zo^k)P-?~12lAraM7`7s2IyqNON7W+)547cs1<4t2-5PVIDPTfelslz5k<%qfu45DQS4|XByy}-=P{oHnVo*wwn?%5B3=2^1ZX<H6n|+7#Lk7BS0pcT>e}kv+#>o}NReb<64YDs0ks+qnoUuF@)Hwb9-ZFJIMc{Q zyMVm@H9DleNg$_+HM_4H4NZ~97B*jq0{uh=!HhRe>Psi;!{yLvh3|+RVcRLP$4&DY zf6T@Dp16ge&d>if^r@cAG=DRoZa(C6XrWY*%$0#bq>V|Rmgg!v4#oGiGXi=+6 z>;B9_2>-#iVE}rJvEbQPd~4w*>FqNlH|}+PVyZN-??T73K+7xl&K)9R0%zvY_7|BFjt6%)a2HKHvamZV- z6Uh}mmey?2RE-Uclp*W^F@H!bW(G8_N|Z#hJ1Y z$!#FS4i0fVYyj8b2rY7a?#VlS3-Ok=WX$uujLYfy_DbgZx0=Kn*K&#hd=(<0m!jln zK^71U3sH_ME*#u{D3X$*m~XrDO(g~$H?;n5&4%+T|JwtPiYJc(K(m65w^C*IM5sC(Xa`G39}eK)IEob*@+`v{9*XPg9p0mWHVWeEjX5X+kv z99R;>1Oqyqn%`NnWftaxzgcx;mPZ2x(V{vpk>_G3tc2&j#7IqEqiPiqb!@epUpagB zg)ii_M)CLE)XbG4O|j~hrUsP0ZyIFCEGrarOh*w1#s4D`$_XMpR+|oP4XHtDGU6*q z>Sst`L{Izx4d?D=1Fd5 zdCfw|H;GDhL?p8y5}iTie9Gyiz}zvlcXh?~r0`KCQ~XWG5izrJ5gy=JHv)1@tXfR` zNP`{N7W5cD8_KbYceZ~z1r0vUtFI{Dnm2ZiTkv_W@9a9|v#xoPv#}df_h$w&_rDd& z&1Y*Ni>b(9wAmoGF4=|8yh2Vr^ULwIO?O`%Kj$RnzTNa_lK`5gxO=2*u^?e=aZVQG zV6Eg>-PxVwd=0P=E}A%#Y+DU}GxdN3W+PX4zL!X&=!(;DH?{-Ix+Em9`8W7QaKb*; z;3ckNOdEAnx0>qEo#%MDk44#J@WxPAke0FO1njf!_kE*q5?EeDKeXBa>bXastw?*U5wX!!1az;1mC&#rJ5@aBsrq#x7 z+jdv7EG&gQH` zzb$KV?oWaeAAyB#t{_yyVax2|Tm|OIzX*2bjB80t;a$-sRAf+@ol4~r6O9Nbd_B4- zcazq=cZHmsq3+j#PDy!uI&6GaR3P%AO+}E(zOS0o#zn-G==@?|u8S~lO5KcHqAUO5 zhaLtp(HyIb+ju5~3?wZ0YTc>6N%*LkW+koK?O3U0pizWxnTvlske|#e_#PyOCmqzQznC98Ud*|OB@^x~iYsiupUO!RdwOend>=Vx@ z--O_b`-P4%ud>yZSqTL>Kgt*vPcPa!pF9&59e-}c&UEA{{NGpopE2)}7gL_*|ytKPgQI~_-T!u2R2P9%L?p-zx zd}9X5z2lTKQ^+ z(?<#HEVKFC5X!SjmTtzR^Tgq+1?)fjp`jJrGp2E^wP`+;5l4-Q#qtn*k zra+#qWAO_L)ZNfwZgZ^bk21L#3N?H0AMKY9)Iet!pAi4@UBTj%PkwPgRhj$=tJpUd zdmRoK6Qk56iGJX&OVKAMPwMJTfyF5Fg5L8fKqcrH@i&mU@}tc7mo#hLD=va2{~+0X zq?r&q%*5po#cqho@ zK(AKbR>F+uG2$5KPil3qG~U-z^phet+Z#Ins>sN~l62&qlj^|`)2Aj!0(Ng{E#~GsN=YTACIv#TIuHJ~X1p zU%&hUZM~ppM#kw2_Q2zq;S4MnER}IPkGc(5hn-#Bph#wG*F6u7fOtB zPXSI>4jWTdIZ!LY`Gca#jITc#PxQ!1syq@hKMZouNUQ1hIBeX}O zryKU+xOK4$Wr%}h9jR4_Gi^S1hSc|HT=b)$*E+tp}?O43oh=hvYb+ht?Zu)KB*uN58 zscv{GFU%K{th|hbtWge8YbyhFQrqowyKRTBj~XL(18Y?a815zv9d(G9UwUofP>SDi zcljt3;w;G_A>?dL9(>$l$P7s>psanAJsy@G+U(ogF%4e$9^ez>IVsnTR*sS0swq~g z6$LGdDeuP9XGRAy&bf8p+hqaqZ?W9=cY7J)h1|k8bQqSrK#7j;5n*#8We1RA!HRw^ zqzW^cCY-pE*#tn&F$rU>??o%U|F2#R)?-aulT52HBUoUY!g2mn=t+(CHD~gH#pHhF z;g{cAQ*ZpL>ZepFqki*|gTe{`=>z6bN)tuC9y@UBl2{URH#5~Z6#Xl+EnaC&3pa}c zAB%80BV?rUX`C8-O5{*LnFJ3RQgDh>F~veY!p(-@E~|(%3THbA%z0Ots=KJat%I0c zM&a~%Fq5et!x0)P;Q_w}ts-cNk*BS?KjTu}av~j)X`&r?!Z$e9&cj+op{jfci#hN=Z4*dLzkn*V88ioA*)~$M&6D%V5l_x*bPANtXTZ( zv{*z;w$Xuj&gzB7#lanwtr0&-%EUzS7$y_`syx@<4VtW=ciKBw(Ns)Rl@zD?_5Ndr#yp zc0@?kH!3g`h&$O3BSZj zEs-!e`ChF%#)B6Y2F;)daU%r<$!QiGhX&lb+Hfu2+}~SSGE-m0+Q!#Y>q|PZ$_sn( zY5EXT^k8NLWX5Ifj+rs8#aM90kMdQ)Fy2aUaz_*=F4VJ!fyAl^m9jm^XERlR1bD2S zM_=LvB?ypk_4JQYLTJuz<=0v5mJ>Dn_v-4Iitqf?qjqLzrZ-`ei`Ndij|f{8r>1ma zK=mYG`_wGp?Bs8}YOnp6e0Rq{Up|A*9)}yG6=3+l#oQaS1FrV&e8FSNXEgbd>UA4_R0^#ZxjFmZFkrkmprvZRLzCm^PS$U} zpQ>i_OKPOx>h-TaL#n{! zRod=L^95`D;vn3D``K}(|8_n-78FMvxp~w-Uh#|8E3RKEtI^OqjqEVsU+?7Jd`bs+ zV6puM7qG$RNHnG(8RbGHx!qKBsLhh3X?RqfJU)*Dm z{dnwpcN*t+z)x%eaQj;Z)TG94c0JCSahIw#23U5Uir%yW&1vOx24uQM)HJI5F}e1= zvgP|LHhA;FWs9Hf57L`gg20Yp9htqu28uNWFG~JNc!o?h{jAFDxBb|C!S3|oiVve9 z{9^lsw+`xaJU^D-t=dO~&+YsHO>dWQ>I58ZRmG>zdW{~Q9h?yS-ul(}&9@yk<`btH z{Biikpt6VSB3loz#uP8t-f+hdjq}G^X?3&9z zX@oU)nCb1jR8{@0a*I>!V3~!P;_>M`F9!FY*(1Ygxn0qv7lK*(X5$9R59AEkmn4$Mv&a3)d$k@CpiL{;vBw zyFoh-$!FSMyG*!l#8FB+FDPYJVw(--Q#sigVnf7>HV>igvwSmurKgBdHO>9M)Xs%D zIcMl9n7D4*doo8wXb&b0J0i5$fYd>Wl9nN{jiDS5`~Xw>@j{F)L&iBZ;Y$){e=hSz zoqMj>Y&R67FNfy7WmK`G@2DP`o~JE3((?1J)V_W#tnBu`fi1Z}Rj8&F-Gy@G*47}{ zwr0>|UM;i{dZv-zS4sS?R=O5wpUmNRs-e^9_74g?F(uA|$|XXHYXfeCP`gFYb-A|NJh969K(sJAT$76mYC=csLX< zKo$x@dR(~cXl9)+b_*xT*J6A>kh8PIc(Uv2fd}jJ&QcCabYIzK3kZI^k&E}QDY*0v zoqvB-M**pS0dk-feS6@kUPGC3<<|JxW!q*l;27!gJb$R?hoF z6_E#Y&TD;u`&@DVD4KqJ{Hih_+`a%-EN~xDKdRJWp3!5Kc!Nsh`{5DRD?F)xsK3(Y zwFY88vdnp}q|)!Y2P=5@T4%j@A&>1TgvVNQgq1#W1?7aHX1Mm zK$U|CSDdc=xbljtxd{pIX*i~JYhPOW7@|SxpM3Ie8%xZ`a}3`BF*TReGc<4KhLEUZ>y$<#gdIH+XZfn*bZ=(F^pI7RJ;K-{OO2@7&( zlNnTSUHI|kSP$W86DK}{R=-6VEm<6CjY)2ZGUq8QEkg$0{ z089Y$Lhpkgc8>v@Cly>fD?=Zmo8|7v+KOt|mzRM(WJKDh;E$m)V0F%HO&4q5wvdR? zS5`H+-@^A50w-eapdDRNK6z2@_jPO5p8QoC$Z|B%sGEopsON>QQowN|Y_b8ACP1+t zb-VUAs!ufWI=_yWQ~2BkXV6-*HS)oJnL?g#{;k#hMtPpCR_5KSu^yvOl$o*}o`NZs zY%lZW_UpiXmB(AAI9g@mfDzAkUDntOp6i_}l|$uP{$%MIo#+WWt`CBqYXk%|z{*IO zbNr(4+3}zL2#!NTDVp{c%74Kz_(H(1A%8eM7Bkd)4ci>n@0F0k#M5@q98b;93ePG* z1Q#J&i|#>Y30z!oGtxyT)YfOf*x0qAnSb>Hj7*JZ&W#iaE9lBJL`}A?V4uk3ej>A> za?-3crC)ihC0e5oRg$a*PEL=0n`5syN*XhH17ETU6LecOiT`e^*v4r`KYiyp8m|)d zPCebANAhNMTTba8AqA~) zZhpV(5A_5{M}#^M%~vJ1xNaj8SfQAogU`%YN~7m)&UR-GosJseB*3Z4v-|%t*@rN< zT5x`)X_?4R-&Z=c4O#%#d60EGnP*)vNv2E5*1EGqBvSqm69?aGm9y66!G7i-HZPfe z+`0~cnXqj1JoSib5_voM`DEb%0^^pw1nNfyMvm~U9bYxWPQ)EgKJd@UOup0`8_zC7 z`jd#AxbAmJ*P0ZyhL@5%!JMi1$(*1R^6c&#g(0-Ss2 z#nG>Y&oo3r8KK)8F_8XPqN0s zD;@MU=QplmKMW9k{LRlk7mMUet50*}-D(ayA)FeY(X$4IM~zUjd$c_z|JAQ|a%(yd z$V(zo+87b-LvmwH?gE|=6*N*8h%XpCC+m(%^_u%0@g#6jkTQrAU665SeX z5nq#A5F}QOfr9iDWB9}{wDz|`AHxbnAQSVk@^)F>hy#dT9ol>Co8?f{UvdYw^>?mw zH^4r5-cpKQcJwk9Dgq5An4O$EcMh)gNc9DMHyZ{xz4n)(>tv13oP@?zO8hL28utRX z7|&h{8LgZ}^vyk>vOwQVsT}dm{J)1)^rs9*dpd!ZXc6>(0I?-Z Ad;kCd literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner5.png b/docs/images/LogMiner/LogMiner5.png new file mode 100644 index 0000000000000000000000000000000000000000..b935758e20f359fe82e18eff90d81c19f7e73f10 GIT binary patch literal 44106 zcmZ6y2UwE(|32P0<|$JvbCj#o5X+TWE)c4dSvjcVDJ@s2rR83Tig47-+?D0XL8)m= zaW52HXt+=*Hx5t{6%|1RMEv9Ve7~RX_4{8J7uUlD&+}fddEYm&x2?=14;(+RYu7Hx zn>VigwQJX&G|}~s-}i~$8#FrJ@7i@@*Uf8J{)X>fq-hacF2gHrFYPX?tb2gesTH_NggkG{Tp<1OQoY5Om`gJ;PXee{`@xZ zwf@=NjjxWBu#HdJL0`<7!uD;hDsTlQByHDTX2xO?b;jY1)55`RzLR*UniDttf=o3g zTSdQ2X5#q9&aZgD?Uemi;UiOI9)5dc8>|pk-`8lAdKN!m4N>Non*^yqv8CCn%ELKj zm4!@%lsxHpZj8B_AuT0`7BMYFHewR=g7k63G)U9*6TH3=6F%TO?}Z??0k#_mU3@cRqighr@ zmmD)PIW2)cNZ}n)&03%S(txpTMvo)1>zbKS*4m2PBpYx>c&M5ca6Hf~23=w9#P6q{ z`%ZAvvPMm$H(Ta#pbp#uFXmajEpvbZnCEV7V_47ljlDOVN|QG5YQ;kJYb6~QbqeUl zfGXbjY*=T6mbiAQQLnQ?R)jmtQ`eMSyken-yZ?`ACck1T&$7V-$4_y}v{+pncmjkJ zd|$5{$~)zhl2J1~c;1nQQ6_N$Mm!Um5X9x_WU)f~x}-BvN!2c10rU6CKZ@IbH{d-J zFT@RlQ_-wq_A@Pc6Hu+|2Jg7`b|i5SDlYJJ!`R=H8Uth^j3ceRJ$&N3|FdKNInM{X zv{Hj>T9jd!Fv@kJ6L|R-%vnDpfnUw)2P&?&7`LSD0WNJ3BZzY>Wn*yn{0}*Uns%{VDr0W&NXc zxO;F)=GgRQ10+WW$%@Br3zxUYw`L9S!wt*f%XV|b#}fH^4a1R8X+xH~HMD|=FCYmk z8Y-(zAMe;jmC}&^PwgSe_qA{1G+*`g{e!FD~?F4wF$ zkh_fBz;-roF?~>B8>16K*oNpH_u#Dsr|l2DC7}8NG$8BZS`P=R6En+n$5ijOY z`uFZ8(l#HnJ+j4sqDDe2;_0za1hq$jJIE5<7|P1FG>sN=^qekNsqlW7EeQp5$j&$u zTH+=sUZ)exiQ6nMCAj_1ameTdx6J@1+Vu_j@dbOjF;*v>VUFGT+wmkAe80@-ZlsCMIy<+| zJJs}l>3TlEHk)m2V-(I0XhK>YiooFG|n89bYRB&i*!(9v$HTa%t#G z^bRbJjL12VN(x^JkRW;agpEeW2xzL?+%w38;-N9T98 zRX!Ucn37Iw{#oA}LRoMOKAYbi;jj`8w#?G~5v6j4uKIF7fz=C0l_D5(U&Vl?A9`}A zHVCBXYz!O8zv?Z4@WKYc`x;_-LEa~6t=b@9t)#a^$A6>6nbYFhmDu%4Ov6!Eov`7~ zR8zS28@byN+%7k8`O%bsjkccb>~DKMeMkpJPRZ_zBX88xO*(t}abm#nyob@|SXa)p zs*#siY4eKH_L@_gP|2L|J-0P)`>Dr#=%R}%c9Aojo!~ZKOVIKzJ*uxF6C-`Eb&mYy z#9zOk?U;-6q4e||r=nJ(RO`zX2W`m>vy{Q=(eHgs$H z?qrUt=jd5v<6f1L`gz~BV{S?-Zc&i9&RNCr8d)wT^w&2Sgxe$ygKYCi!D#-^LRdf& zYiw4(`hJ^B512=UYGZ&khcnor-5!ns!vx`?8w8sL>?>$!dj(Y-lY9FghZk9-`E9AJ z@|AN(@f!`xOwdw2RTcxkedG7%rh}*M0I$jdCdEbLeEQs5RT5h9^~4GCRmJU{Ihx{| zql@X_joMCNRv)qA25GYT2911Ab6BhV&ye9xM_8$BVixtUkFGXI{Wkjw?ks;%+bWzv5hs}Mo^ycu?Mf8K*c@NHbxsh!wG%@%S$F1GtWTm~a702( z_L$+a)A5vzd$P|C1VyK2%0z#1{^cPXw;T>G@*8*fv@BXEp;-qByW4FYjS1lxM+Iw< zj2FH~1#7c<;AAZ*WW6+5d9EXem!DFv{5YAbJg#S%jB~h;Q~yggqwRv1`VE3*&e{?n zG~j*lqPMTz?f>)fJ4O$$R49z>|JQR?Zjikbvy0^J#;=-zS6@egjZC9+KM?L;nKi-6 z4Ik**^YuiT`C3(VgV))kbDWxAJ@z2i2U&$VBPyMC0xiBA9Q#ql34rmXrd*XrE1wgK z%H1zbncIns(tP5qkMT<1m9%U_tRvh+e*>ADUFXW^Hmeyy!Z|y+X>u)+8e8aBkhe$L z(U0y|71?2>tSWd&`1U)};*x5+8eaOGu$3rP&by@I(zNipde2SWby3a;Im>6Ad&VXt z7l4cng4?E7{?u9iaP9)*65aWFn$b>PLW~g}`~7urwaDo9SS&tQE_p+vyY}QLVq7#a z;LC>6$+K1CiEBNN!(TbN$owhrRrx|@Y*}vEjjyK0Thf|OwK)@XwiXR_#!j3aBfMH_ zJKS~MUu2l6|Bqqn4a?bxwTd#&%1mF?)uv_-KQrr}S>fx$oqI+nsOr5(A##smoEwuK zM#n~1xhVH)*iRkk>;Wjs0v9JG)X0&_oKvSDb&t@NrbT9(T&C|?!D0sivesMlybe0%i_E*c_*JX1Y|%{g1sR3{no#E==#GiYJ2ncsw~`2 z2+_omAkl{#VBeXb_vWnc?z1t<4?)7SC(z4&$zhtN`O5ZUshJ7i9GK6{yB6)K<)d!F zE4D(@!uV_~h3OsNWrOGcqR>TNyEl_M8*|$c`vYp4-#*0HH9Y$#aE`b5?tsB@n9Y>Z zwt%!pF-e?WUN?Gz_M4|YxI3OcVe@l)IUl@ID?K{l+>$eB>vT?HCS(ft7rcjSYEaa7gUVuuU*`C+wW$f@*R+G-`a8jdQ($>Wa?SML)!23 zJDOq0#+01EsXlLeJ}*C=ZD1aUFnz6jrMJfp#nM}I^^qim2tT-Wa6k5DxrpUnb-FTQ z=*Y>H*AfGa3j+3@YzwXT*Lff~JHek4% zYI3F1N~FAI@^piIY+=pRU5nAjXgD?zv7QdeZu@roU9v9;C1j_SHk0XGZaW0G$Sv^T z?-@J{RzOCf;``*{wdDS%WjLcI-g9gYvPFg{YqrVV1)3bl9IT(c)Opea@Z;(j>SC2O zgOw-tD=ETt(2 zvpL5h`qfEd@#xQBTe~jiu3Q5g;_mtWnTEE-b8}sJ=wa`Y zK}NPlrzX-UwXNtbQNgtywTG;<=6H5uwq8Szb|$Zifi69Zr2qpz(C8VO-9s1e==6ThTf@%w(>IxsBx@7*`tAc$Cu>T3=y zFYW_kKh>xDkd^Ll#*44cX_LP+4r)72ZCpt%G67%D`)0e^BKh2w%Q_Tl`a0Vjn2va* zNL`><2H3Ec+U+?_8rqfRAGOenSC`mNsjD#w5SGo44n0()iX?u+LY9r}&3|~$RbxcM zvs@Aua6)`3Y+_RzA{Cj2_4XV)AN$k*80TtWj<|i4#){Gt%MZBRa|s-}c&yiIPX^Ebb!h>6Z@eWhWy_e^-%9+K7;Z#GZpKj!LCW^gBUAg_?eK0T2JqM$|Lhz6~;Lm%Q z#Z8P(oc$#QFs2;fEJhJv)T;HW&;PdWJ62|&>;FM)IM?0J#@F`znzEK4G8gmDiug>N zq?r5ikf{zF?ooKkers-J)17tKwrzRK6xHX_{&ZqVx%P`s72}2v{|UtLBJ*a{x5nSg zNYbe6l&m(I@b&&rKRA0!_)V_;yE_7wzrSdq#Xw+|n|ZL+KJq0$24%9U z=KmIO)qip>HYfhmKNTU|VD?qtP@)rl z=4w1E^{82Ua6GA$rpLZjzi3aY2OiedbZRJMMD20Ro+Al|Nsib;e@KSRBNW+h5j5mn zL2&<1KWFIWKk06!aat=s;I-36pIUfTwX30Vn;k+bIT`frgGON_psXe{WE@eFqE)kK zXOu}!3K^%C5?uaw_<%|GcCGJQ#v`W}?EaRvvz@X_h_drnQl@Ev+moO? z{YOlRBlwKG8CVE-s;=akMDUeGpR)FxO|9#qy!M}diL5Q2G%=+DRnaHP(Q9qdFA3Y{ z?G%MvI+J?nTZVaiM+^JH`0AytwFZ~kwmu2E;cIY~Jj!6){NA_^yPY?YUCV?jr{PqNSFRCWzOmRb0Gf~S)p7F1OL4b`0l^u(tj^xQuhKZ`e5z8eYGw_v(B0E|NUm?!tkz9iEP?v z_WyNL|Cxv=Vu?bPOu^jWciS^5%pTAG--K%BrW^0sdk3HUf2RL$oC9czTczi+!`mlG zU+HLT56pJs$aCn}dpxp?m8xFuq-f7B1mw(d+%b%)LD(B<)P;|j4p*I;b&sSxO!vi9 z+^v+w<0|?~Z5M-D&Y*?3Co;$mf}S6%=n-gYf!vp_R=m!-Y+R`0p|}=hO$RH5(gt!s zl~IcWR?937ZTAKb%q-kEpI&^kFaSMH4DfpF~+UnINKVq?$BF8NI8qNQ9(-#=p3&x5RYzERtaSD0lhwrgiXUfF@$3 zy*xGB@4eUR0(dKi(dEcNd`ZR$D^5Mor?#gR?{A?c2hS(rUTne#XrI+jWI{Yl*76Fg z!Nz09owm=1;JR4fd3gkR6Z9ZQuq&?{wmO-QVFlRpkjBkzm%`CBDSR^Y;1i{1?TYy> zRpiBzlA;3e?B1XWzyqx{C7dhn}^84#DzD~ag%bv+YshdJ%` zS0P^JX;C#2Q4aAVufL~X+hF5sdMAcfzaWu}7>VoVPtZB=GNX`6`fCUWAm8;wuz(CW zYGSnI&On1i&7T74&iUb>;xkd;wWk0XC@1K37K!_;-40cxk-fP>Z2PnQfwYPlGP8Vo za-H$XM-@z&vxctgNR>1fmv$K*t#;>|W$(4FS_dn5&!abOqL1vGKU-kDNi-}!g)DPG7w&QX%dP{ix!>U*5W{+j(r_KYYd$3T``=L zHXa0|#5A4Nd(6T-?rpIgk98um%XLX?Ek|D!#`t9I-~d@&rEq|CWTE7rch6a3XTChA z1pn20_lO68)_fQ;yyB5=0U}Oj)>pC{kn*3WbI}J4y?#pFn_kgSQhfqMPd!m|+A;AdmhXga4A-5g3D@WSU7Qb`ph-*XzksWGRnel%p%?n*3xidW^NS;w zf~-Ql*59)O_tA_F7KhK!9_XtWSFkm}hj7TMf-&%e8rOIz_e7X=+B!!Md|sJ%{*TbA z;}SKgm}LDmOGGHyU~9BpetRFOLFn+D<~OhW4xv|Ck$mgN5uIb_fQ>xcsJFFw?-LE>Z?Q}q3@k7e6RSIH$X{@NDrleti%T0R%=Mjtc0}H`>b^1TuY@4{#<1zgyXL5Kj z&;GCO*7y4dCv(MHqCCdxAStyedN~h6XJh|T{BFW)IRgwg9`9xO6sUg}Zc3nS|lpVB4R-Fi)GEDU$#{^nrtv@oW;-)UA7&sM5_e~9D>U&^V zt_6vlCwh>6OBJo~l}eTss$lVf(*;Jhab5)bWw0cKQr`@J&i{OHs8O{=Y9AL;IT4e|l<(@CszZ6fv6-1-!^F3pfL>ybwH4o<3>@B*3S z!w|z!R-}Mf4KJ`_D}se*@s-j}?b^AeMr%^$pO4tDQmAs(Lne%y&=C>YusqVR_*iF@ zB0`uQ-z;57tL<8<6``g}XQ3e-OZQ2k14~y#Rrb=V2*CB%%>=YTw%L&79BwOMXHRshg|gb^GAL_+&Kh2Y7RsG)|$Z zYz%GDAVOO0fwxw1KnlYOxZb-g!VH&$Q>wzLWvdSIv|6@fUNI-D3hW(?d40%O)w1y)q4ya3~2Z>l2 ziikXEkM?O;xtPN~&^;j|B10&OA|Rf!gXx@+76lR&Zlkz1&n3Z`dI;**0l@3O=CiW4H(qT`Ue~dZ=b$2s42(v$usEK}PQN ziNth58>s2-F2m)iZRUi~E5g|@dA6rMHhV!92yaXRj!j94R~-e+vUV2q2373B>$f*B z;00fJM0eLu$Ur5-TTi9xHnHhYHza*zvX0A^Dh0(Spr2i}nWsp_ke1I`liGa%NR2S}zu{!t~$; z5g5yUPx%+mGT9!;Z$lowS>4Xc&Jnr*CtSnef3QE=zPv-P*Pp0c?+vf~=HNwi^FxE<%yst_g1!8v@gZGi zs{`VLvP?k`MFgOVXs`vliMoGju+p-~uLb^reS^4*>}88o*wm$nl#fERWVSbnU7IO=68Sgl!3q_7#kf<**6t zbox-5P6_~GW%l`lo;FKJ&_Vk7=Ddu~yiI>&7%rk7KhQ=~GyBv^PpJ=rrf9zTrz*eq0>IfW=UK3oy@KxYYj z=0f$Nq-2K-H9B3P0O1T%GeoMY*}Ue)pZTSO83s`wO>wXxpW4S zkQXVHteX)QxbSNbw_a&_Jt9rD+koQQJgBB4HDJN7gsenJYlf_f2q*qSZ~_!(;O05L zLuznJ|3GhZ3L#oJbq*sKu@ry?{s^ce9Ly(8y^u{doRx9nt%=;qw@(_=d!|#I7T9xN z&r{`b=TLv2p#l7vM%9nJ`w!`mA17&h zhK6<$fQo(ZRNe1ocSfzw+0AyCjtj1rH@+-Ve%?$hWMRpAN8_>)_(xS<}BC%UIZH@?uh&o+w$lOm9kva-rDPn(UNl{dJsdf3Fo znLayoNkb`nL=G5k3iK=sWc0W6cBC!+Fzc znF*Qa34Z<$=>?#pb_7XfE)(%7P0EGfA}&p_%w)ew82f}#k~au96Gp~?jG3qFjx~sU zr#kDDcU9a>ya-n&Sg{W**DXE0XPhiXgy%M;D&4UtK!4xUaXohZQn6p+OzaCd@e9{} zb233UY@h-wVk*VMxm?ERSsA{4mX-DYbK@@QhLrC{$0k|2h0h#i8^3-V64X3Z1t*fhHu()Gwg3cgFh)p6}7(hF<;!jI9eNcb*ePIEU&TD zd|%-%!ggOWGO!kIykBc|hu%vy6?44vekq)0C|lDKXD$}Ynzbf8XEhJkw&wllufA++ z*gB-GJ0_3k63)n($^L>F6EH^izPpm6?VFxz6T0iGTt-f%28*;1*yQj#rO9x~%+0G36cmNLUs3N?UObsKnM%Su4(&)-%@gjohW;YLsVC33SQ_??g$X}g?QGC)I9i+*Sas#X z(-A%CiIo=yn&6RlZt>Er;H~P~hmMpVblP*-v|=6CT%zBgYbaT3mN~A$A>g}n;BV+= zxZm=;Rw}S9g- zef!3(dwixv?dM?Ud%NKFnac8Dc}e-}!6{Z!fY|tyl)HAXX+3P?!P=gaX^W;)`G((X zj!H;;%q7(@uhukfe2tK(Fe)W$6liy7=dzv8j;na7`*tNZ_YsPN9s zN8?rk&sDT0OB24fyE2pDtuuY!jc)%|^N=XU?mhLitR1WF>NcCEEDL_`A)(~3V03Xp zZa47Nfr~CP!2#X|Yn@KQ)d;1n)yRp!*HhKX9}p#aXWC4jyv?IN+Bk?0c|5xvq6*JA{IAP7cXdHCDJs@?l|%dp>>h%S!4M{P+`_ z*jC_PHR_=o*rQ&SD1%m1)=lZ0^~J5<(OlV@20(^uXP;|m%Q@;+eDea&Bnn;wd!k_I z>8g;=ALbtDua)X;4jUeOrMy0O&pz&5{ZR>^i~peQbQXIbI`=npy}#s@LH&EQw-fvi z!FMP>W5u#fYy1M(J^QlZi>RyvGK?`xO9#fQrL3!E2a3={?gkTW%dg5SoPYKn#tPRPpVt1QzF_;pQFd` z_?d+s&YC0?ib9NOWoaJP7ih08(j^(__#cpRk#`})DeG;)j&|P#W(2C&UzUg$`wOI* zq&+`tIziQhIiw$D1-{Q*ilp#pJ3@zm^J%aO-nn(}YqOiF-4a_fSB-psX0@i&17Su4 zRhQdi>)d@9(>)GK3Dyzf=lHz@@1+|rlPU7@0kn6}K#kAkMIUqrVBc3nOcClnCk|le zS?5=}*J2z?j8j~nUbP>Sy~dK?KDHOeDu3$u-XOrIPyNM*%w5Uw?mr;YEkA?mstL`1 z$T=7M<7f4f1)SjZL@5zmw8NzPEuT9m);RxTW;f0`Cv9IlW8D10bT~!IPo9mOyne5A zV$<6l);yi)aoR!leevVrhVu_yzMY>Az`lbi%U*OIFiIW>G97>0K6l^`*1g}(U0)ey z`YZq1mzaX<(bXjfBSVjBEX7$1W=e8Z@E%d=>i-mp7yogk z>%Nq7c#^D>3pF#~uPpBT)>!eM)D|5a*6$Jwwpi0&H2ER`X@D<^CF@fV9hlkES(Z{R z=bsEl%i~s}#RrRvzj4mxC)zwRgP1VpgAVCd#h%0=kETOd9oAryYFq!~@vt(oZGE{{ z{Tw1l?~iR>hjakh(%vOrek_TszfWGW{mi>|-#0}vSFW6@)#{SFyI=j|nF6t~{mg4s z>cP8%|Eje<_eb_Yx&5Kp=d}$v!x5ShvT1?CTlwv#=4kC0=X(+JS6#1&qtD@X7W zpMu@SBuAMU!BN@vJpXea<41A9+5{38_=UGj{o9YRqzVnrch^>tebM%OVDoN3LSP); z>1kXc^)MwYGceT8exfJR=(KV5>EnSA^70u&|XK?*tP z7BklK#?%bAw)ga^%FO-WXZDXj1i#eFX_J~*s!KnX7yp%dB)P>ZmD^=Dw(8i|us&QY z9wL!ZW2-sy#K`lCI;??SJJ6(t444G{NN)R}(}v7ylNL)daIxg>}U2@ZF?~k(SQP+MWaZdB-Cw>ZQoly~0z)f42{UK>-thU^m z{T=3Vux|VqK3mJcwCB#U{)n!@l1e;wit{7WsrW2$0ekF%cCBmmYcMd$zbmM&a9_`U zdc7^tNVmOHCA=8GH>sQlX;(QueTC0CUGpNCNlkV>3ubo|fHuHts^4YWWvr2$^x@>F zAL^+O+IzcIA!DEK16sd0n1f#hcr$@ceZW-!9-xUExG|`(3}3xc8lxXH^tz(BR>e}8 zqtRGYv;qt5ZVIJ*fF6FPL{&#}ridJ9Ze-Ow3FbPrw?d*>H27M2tD<27{dq{thLiQskrgVLgZ59%wrgfW7yTb2MLQe zdLJDB;m!=o2V5fMsWij zIWLz5mZ}LaAAVF^%y?5}DlQ^(w=N@QKYS>g){KUF#yw__)J87pn9c?E?tfqHpXFG$ z`cyfFT;$W$W&@T3M86;$X=rh>GJ**H$y50I=S6QZyjX^X;j3bpFDJ!3nDut=9N)XR zm16yx+drCRsR*dch7PaJzsM?CnDhJ6PAwi>wK^#KQV`G`)RKV_^cRqV?jbuLN7omg z@OJ&tVHbTaY?YjNzpTe?FpO}(q1dkYjKAUbT<{$BN`780p|UfGvY9@+bV;ICis|PemGjXT+DCSjt zp?fh?z*N)SlHF*VYHZQrjRl=nUhP*HaaB+9uQ!|Ismpuba0NK5QCw@BG6ku)~SnrY(@*cWV6Qp$x-VkT?@k6s zlzB?s0gZ+Pb3rJ*-{tn!xZMSnpk*{r!wTXXKf$pFtsFcz$@*@vE? zDqAa&sEK>BR`A1u@b;wnG>#7Htye@Pqxq4$5)tIBkRtp16D?+nn&}n(Hhi$O~z`?ms7>X+>&YS*MXf}XTvozbD5bbe6S^r2bR@a{iEAAW4d*sjh5tO`I zSllwwBd))AFZ9SM?c%#}T`D9|1_&GSGz7>JygPWkyl0x(m<6`P1g8{iPk12gJO!_9 zF~A%<8U{BlRDyZklAgPyh>ZVMT4SXcwQvjL-Rk6bQye=Cvz}Rdj%d`$$=OqC5&$|j zC`0(3J$kbpuUr^((5i_226H-K&(NCRoBINL&#}kGhfibUpI14n9|L?0)-mz+eRKE2 zgD!{qEDonKZ;oOaUo4~I;CJg@(`5i^(YuU%>d*b-H$N5mR&rBBz|!`{OzY?L#Nbt@ zqgJm(#vm`XTmfVG?GDbhb5H!(PCZcHRWB-gLywFUUP?%F(-r z+U8=UPp|+D-*Jt@P=9Eg@&6?mr3ip6e$dxxGp{x!r2n9m9%4_3O<1bpYUiem?s-DR z>&E1P1|X#C!#nyAs-|OGFYgZ=eCxyhVugxyYEW?oDnvm!$BIygIBec)zp+;?;Mw8p z)~$7=O*-5CkhOPo0I-c*K<)2N0maax@;1(F_Q={T&DkDh@Zg{H?}!U>elr?igICmPWlNV^0LYX20#vbsYoe1< z(QE&9%dP2C%i<=U8sGc+86!4#Bx7b9ZnK%kWmG@chL5kOdLk<@Y z>fet)2?og3?=jtjHS9apM5{DB_jfit@S}#6$ydhdz^g5zvJy7FYYs<=b)6jCCt&%d z369RN^?rxBdU6NJd8&BSu(e1co!eNOzEX{yz!@J#H|Ht9k5VI<|HuS;E$?}dHKm!V zr{+mx^@n(c%Nt;$4gisIHg(+PJ+pne4zcclbHxWP#78QM^M31)#GfR|O(d9kN3JgJ z2*b;4Wfgga>X%dyalS5Rmk!(tya>{s-%9V|YQFUhmYNJ$xq0PT#HT#J4H^LC{UfeC zJzf?<{1H-{?CscPO}~LFX@Vk4Fui^%y?uFtwmTN9w7}n|1$mWv zIq%Zd(`N)q24G+gQcDF_Uy=zJ_bjnN*4%HV9nV+zFcD$a^zgNnO^MNGRM{iihHaKp zam4Vol;E~A$!nAo$5pfJ_bGd?Q!3*BG`Pz6QsY^esLXpg7^0@%6Bf|?;^YNm3Rh{m zwft(Khx{?Fu&rxtbgjGlGLPhDP`TCp0H?gqVeWE#!p_@4!t3q3Ml{SQu6fF7B97j3Sy1)HMuj$)wj=G4uc$Uq7e*c`rjd5F~jTf zb}%sm96#~tIxY5|YI6F38QQ-bpEGeb3c95JvVQed(h2^1)g@PzVaHJiS{`Yv1LMw5 zgd87QQ*sJz8g7>_f^Bic`vzfZk_~&&?Au+uoSuh>%}GwO%GTm5s2S}Weq9o5gfnz< zn@Uwu9bj2XC3~c81Q6-VZ?o%VFbxBW;Y?sd-xDBMzn*k9XN$ZK%$?>E%@%xglV7A2 zBYsUJkuU5k9A{)))7!m~{m+~4DIZ)H^NW++oD{vOZp{S&9mOGbW58qUte(>ETj+J| zu652+6Sb|mlmSpzX%B0_3YG*gu?{XBwaIUV1i8mg{L~%YTr8eRu7)miK9;7SD~!-& zoNl)u1LiPi61uQOx#U!9{$##u={;)I5?$y+?D4ap@CLK;-6+rk(A<*=Ep=wlD-u#oez6g5V}Yg#`_PjomO@d)S3ZPQ1`_Rk29Wg}LO3a82oRWH&w5 zP)yMY@m0;i1zG{g!Ux_I($WL9OtU6W#l1ZKLBVPWPArNW~?KmFc%1okW$*Vlfc6Hu7IX)f601 ziSe+qhr;jemzXow_3qTeIF_1TfT4XoF>*KyBxBQ+7iA6Ix}*&OShRD1V|UI|rdPGn z?Lqu<T zVM#^$wqb3t+W`kz3N5V$1w=>)`B;`x8|ee1HrN>CYfu-}>UCWvA-n zazlG7rfD&Eu)t-+T;5g@qRQ42o~T=|z^0l?m_Y?o1tLn_+sop~s+ww7JD*tmcrMOh zW|KCY(&1bK*cwUs+@*RHmogb5!Xi-5- zlk#0{1$sRtoyy7Bv~ExMRS}?o0$YsJo4J)`%`@u8cCtoEdp zOj-!tn>d(9t5X|iRyby1{MTb6`X=su8;ph$zW^nkm|PTt)Ox9Ea#}-po7jSE3vgu83VbNjaO=w{ z6<^8#)W#)QC*5nSavW}DHl&1U{~NTP__Y5?n5xd$dp8QY_OGm#`<@<{8L9H%3oGRH z5!H%gcz6`yM2=A#6-{ED?z~q6&lMZwa`U2w6RLIpNrLy0JR)>%H0w7Q2qVl#_PPQwjz4yNOzj&vl zslFaLM-DBXHuPMXBQBfIh67w+gT^g|K3)+1gQdUM*lv6*cxpks5M%8aHs9yCK`T5J zJ~eP2$Vg|aES*<&C8y=ECCKG37!vsPGyLPlv`8>p$|T$4et5;4wM~s6z04YgSWuus zC*Bt#&)>h`5!Oe|MrL_4jb5}puAhoZ{AT+ca?r|w_`)o@$V=N zb*0TUF`&F-IwRBO6Rq9rKi_%QHIBG@*rn))knziPp}x5ydkc z-SqVc=M(dF?CTl#Hvp)L4}PcR6Cf`OVgBv&G>z*FOW6NPQ<7-euQuPlRk#AGG=I+b zNCzYO3hMPAgmTO;2xRaWk#GNKQ-bmfvN=mpj2rZyic%zwTM+>KYHMz){4frA9whWF zSJqB<^q*=RAzC$ztkLW?Qp{23ky~Hr=Y>QvqF0yYn)ldkUL;T0A1eCKCL>ut+`^kmHP=>cRy1dZU#xMB3J{cM zgVOQDc2S8@-PY~*{V6|*nBpA@P;si5Zd?=x6zg8MYvD}B>(7X^bGx@6Z2wH_5gAVT zV0uBe>w434|K4Ih{!~A3GXPB1V@$m%f;_^ViMbahnR2SUpqoX4iIQAe5?77%h`FXv z2H^pqbu3Ge(4ANy;biE`|EB5$S^)E6Gnp-jMm^@zLV5LUhR{^P3TY_f7Fb>DkG9|g zp`1ZhhH>wjGI0QX9=G`6mt)KHkMB>r&yIKk_FAC;6BS0h^4&2Lhr|1uvyT{NQ!gjU zK4<&{k0=d282l+OX(1iTg^MJ@BZISp`Ot6~d@30&GentxyFU&3&})@E@KY69H!q$d zHt3fPVocNDW03k($?F)^36U7E4}hw1<+Mxz@8P*q-ROs~fhsbDHKI&>jG9kM)l|?H zB62EPMp(1tpLa*$Y9^z@@jj}cW4tudunVn(MfkHIB%{7BxdRU`%gkA6`jUoou4yL8FUiY}HK~06Ma$PZYu!+i6n=I+=Q^Y1ULIil3F&cx1va;d?G$`$B!+I&J$FIA_ zLI|IlEmdJpV^8o5ycRv6K!C<}S!vRK)RV3if(%ki5m(xZNv=M{KOPrg+*g2$b?RiO zng_2kek(*x6)sg;1(-Y@fR*dA&y4-V4;)6-lf{{HQ-BFEBL0in=4T;J(22JGZCjtQ z842_=s*!Tt+9%aRWW{`S#nyZZSj`72kbWMM>}tCcSW)T2_sHNg(Ba6y4}&?Nkrkep z{s2+lWSkR`B$v#$uJW@O`MDQAn?bxj_%ki%YzoddWxE7+fdNd=LHOP8wA=MoW8jSI@>G?~w2U`{XZvJ%)=OG1ymAMl!dZ z6@QDNpY~M;J=mMg%*v^8&HYq}tRHTkD4p$|Jzn6^oS{B7Rj{-s*2U%NF<(fKNQ+RM zVEJ?sWR%#Nrisu^uV>|b(ip#ls^eesVH>XflCo+blF<}y?k^{{g$W^8D-W*{44RJD zRSB<{9dffSeB)G+cGGAUHS%dZ9mkvC`2^_uG?V#db!;p9_VG_~KGp_Sz zMp58AtsTgVg9;e8k}L$Q=c0MLlkqMys>jv~7T=O$ZVtj+u~oupbJC+hh=A;~6{sd5 z=TbLVO2mhscOV4MC*XP66>ZiDtb7dxXy|MWs|yE%_0GN)RA91`rsw8t*-tzq3ztS_ zT7JGBdsKr+4`jA|Omsu%ESjE*f`o9%m$r|ml)96I3Eg*iHk#UjKdvP4#weATaOO-( zU^uvg&hYk)Qq+|Bp12fX318S$`yzvd8u48JKf=xfs;O=3_Y?sIA$m|yfuN!w(gc(m z0?GmOP@FQ%rC`}phri6B#ghs#_d&hC6YAc( zFfDYAK)|Dv@_f7OCA|+2iyTMuzpxEw^9q_?LdL8{Ea9jq*fNou6KX8u@qvMRNKTLU z49P=j*&aep08Kq>&Oo+s?QM4@xxxaV3+O^Ufzgbfijm+9{hX$XgxEP*0f%J~lmJTW z3<$wi?Smg+GdtB*zk9Em#9+R0PAaHB+YBrqb`*KH;N}OInR&MJW%HlBl*kLL7xvOZ z18>SLSASm;oPsTG!}x#3CcMd}@NBYwapAd7bgyAaf=i=Y^Oo8x4hyX}b`}IbXRKaR z{aFor-=N&n*S4iUc_Dl2R?Upfkxzb*6+Tmc>^Aa{IFA|udnT`~3Qi?Wo!H7Uy^+6Y zwy%V$VomPYF5hQK5s7)b`uSKZe6G(a`kM26k?⁡U`|+*8S?V;abxC2emlrD-p<3 zdMWb-)ujg1vA4WW7g#Mck_qya?Yf_K0N;4h1nOviDa?Qmc^E3VKRq#OgQ^KoKYvJP zE0CVj0CQQn@&RDX=dNy5*>H{Ik|AKr*D`i!P;;p_P+OFlM_`&=5a1o3QdyKn8m;x= zDS+~(px0jPPePnfqaK?H2(hg04PX;&cv1s|BIA1i;a=+}Zf;IhQ(BY85nVm`MM#ij zfcj}_L+%y!w$OwCWoP;!>X3v0#A<1@0yKes-ggS z(5Ef-BTN|{)v;}`MzsK&#q)QGel~S69(_i0%h+Cdz zkea04lM+}Xe)B1q>8^4Oo_zyCLe2D9p+P0<7R0cVQ@w59^h2J;d*ZYjwNsrPu5qnu zHHJ?nFGmV0sWuDM@YL=v=9f{mTkgzRggVSMse{f3PPeLqVF<4%Pjer@@n@g$Rs;oP zC2Zw~2BeQ7;#?D$!ze(-nQE;w`#8C@x#N;>ih9?;&u@uA&1_BDNZ+hCQYd^*Z^u+c2Gx8~L*<*M>u9y6(9TDh^`?E~Sdl z=(o8uuwZMnvPEkvda+lt+`K~W7Hj2zCnE|g-IJ^vM5{V9>)jVC3%_mu2cO6)&Bk=g zc*vJcerXDK)qLmr=< z7$&TMNT|TsCkWb~gZm)XOPvSI|JSeHnVAK(=&RPgoFNK?sY`js)z`vUogsU;_O*<9 z-I{HDlAFHV{(Pb%Mnv{#MAZS~E>6rr*hg`{7{B@jjN6(J2Fs>=wq_I5%fg$_Mq5NXPWHWv0aQJcgMca$C z`Fm4MxzyFx#^;VD`aa1I=g@A`N$dJaJ|_F+6z9L4*S~$9#GfI#6_Bux2VM`&Ag~{f z=H9IOLRI;*kK7;#W8lwv&7er<#9iuak~N9+`%YWzF&Q;U{;0XZ&)Jn9MoKq2tMHbul1@=jxyGS|`a5oSrn zdAHje>%RMv7c)Jb&9%0=7htxJwXn#+&M6&{OA-jun9bA+yZiGORj_IGT2pzt`BL7L z@n3X|&IDb+GrNVZL3ekVA_-I(laV&_XY4(8cc?fDD(okq01& zzMum9lC1W)=GS-m4xtG>=xTSjJ?hVL<{nLw|hU{{_nEK-Y5LILLs69d004*}zk3}l|3!(x}D$Tk}rRyM~>;(aw}A@{G6jSj{Xp8|H)O7CMGGd6Ord~ z7Q}ML)mpyii&JZM^Ar~{MS`Q$thCd7r>HXqeFzGoG%`cOVww-#;;YA>HEyKeX}>ci zuV1_k%#;K5xJnYWG1Uu!p)!Q;t^h2DhfSjW0lR0B&YypSop4eCGaT`=Oq_6u>!w5?GbcJF z`KocV1G_&5LWg3``#;OL`(%92D@UW)slLSN8-g%CZG^E7-)P(C1$Jx zBV$bT(bu<^aCDt_SWmP%gs$Q82H-^HT#MUuBg?GRAwFZm>X0rig@Uw#kPD1N3X z{f0IFA{Efamh{$0*744jS9>ic+DNh}jqYppKm|+JTi$wyJtL>3^w4Fl%F1~Wz8aNg zx3MVCA*0R>akEyXGx7BK?LM1-zIxwxAZfDXCu&wdHr0Gi=2L`)Phi3-9UW3lw=_h; z+}Zws$u-rMH6$euGZ#fpCd?FZ@?z!8x30@ymmJ>e90D94an`LVSdZ(0XGhn>?yJb~ z4$xOlw&ttKc#cfV(AMtK|1vR4upJ)pE1)a<0W(1D8ELN_$df1g3V<%zo?pCz2i_dl zY<64oc5@xT*ckt&R=wewUwwwf>u}9U)Jc4hvSy0A?f26XTzn?^EaL4f40n8LXEMjqOXe%9p^Juqjt8FXFJUtDjP4yO8MOOpJqMFMOGBAc<-giZIAOSK{yYP0Y-0H^Kn( z%okB7nfJx;(mYXeY7@{B3;gWFwXBV0%^lI>+|XU&(d&}yR#?E|WVoMICuMH#egEcI z>}z!cOrn9;^s@Z$voF)HUxExumn7jEcqcF1RW`L5KymxJ`b;90+o(U5ty$gJ7>r_;Z01*SDtdq#Xa ze_u85zQSYEvf$nB1^SNlZTfQd8~tO3L_a?Rsn|a=$*mw^%Ha7xg7WE87_`LhuA4U(?qyspja`wFrnt zmYIrjve)GU3R`X&ainyc`69BcnOpCYrkpBf={p0-65Z0ktC4%>Uced8H61@o<&%L& zdsFcJTY99HpIm~OQL_&VAwkPcF#^!$WZ&(%Yk{-=bWRi{q*d=q%Hp8U846e#U3*8L z7=t!zZRs1Ib2i^{?gEkn?2mlUO59E57p##=LiFktw@x`RHBBV^O!D!$8f@B(|1gGI znt4`qc6OcC93omb zdc~iJPtziP($}m=6^?^az`s^2<|R&B=Xs89u;dYP7)DPCS*nVM470_toqZKyyZQW! zBAA@UCZQ2qIC@)oE=u;(Gkxj1;bwO<;>@%iH%(snqxJiqYw7eo+m8@zj$=Z@!^N7Y zUxOc~5gRuyrKmVP7g44DxeLe$KWcP46;0gVk1)wUKfhO)mZDefNqG)=!K=)|bF?<7 z$$Yl>ANt!TK|bQ`OO=y$^KpVZKEsa!&ZezO_!?YmHq7Ih8TcUdB}4{X9+)zlWP? zBjgKmev?u;NNZJSS2z>WGAU%IZ0d3C9hLV{^0gmCTInq#WZwEq(femQBGgnd4}vf7 zR!5J{&4IUf#a?Va>2CPTB&3J6nma;-Z?ZYrk8ce6{W~e|lCisS4v8cBXstf1b%{;o z*;A2+weTT-9n}Lp5wpdQDE0xn;k?|6N9W!rx>4n~UyHF#F!bgIqe#S%2$*uNgEHZ< z>B`+1$}90w_`d=}mu6lTt($}ACqBuWO#YsLt8#3{_s^Pn<>4w{O_jI#mDmw? z3~w6MZ%u`=OK+O4#BJL?boHR;c<{TER`pK`-VrCCR^d_}^*(y@4V2TCpy0@IS^#{n zcogK)yZAIY;`ZkI*X6}`HD?-}H$`zoeGgpLqm$ItE1Qsp*j>X@iqCn6+}jONkyBubCh5kv){L)eyKXxYTsa<4o$Ka83qNPX^B&S}eVsinx^~>v@?mn{Xp#PCo^*1k`DVmUw{_s6$&EwK)IC$&e<&6J_u(DCg2)Xo z$+OX5K4UABf)hq12khg>JNuU^1a2Z9Y#-F$0;_61xKAndjVw_ef8s;;>6oyeLcSX% zntw^@3OuoXT2??}(P9(Qe=9PQRr)}ii_`pea~@N--WRjSET?Jo;KSNrFvADqA?5|? z*tR^~EIqm%BPP(YRH#&Sxy+Tu0`X-B)j35^ve;j8h4}QGe73f>Uj;drN509(6O}E zug{p)+=}$YKK{`EHXKZ9X$}g&b_*@8pLs5Cdoo-gLfh3@oKiBjP+YAe52s9x@5Db` zH!T}TN}XLjQZ4Ng_4RAI=Nc^^xP$3@5wNawAwvjUKs`BJF^xaPAusQAobV>3vHAr(offuP87Px=SPEDaL{Lv3v4VOECFD#T{&kR(V0)rS*kw%Q@aIrSClr6r zx_jF>gsY+1Ul~U@QZ>eB3&bfI;G&C^ zN-9>hj*8DQB-F#nIy7j_>dV&-0Tv{DDwKan@9>77l8DMU^e$W}l=AbNnSK7_Wsz!_ zROKy!JR-XM5Gv5lW&@s;D*u8m9zQu7EppD;1viBY3aS}-2GDE&P5YQORqSQLH7LCW ziaO@X#%}eL#aRJKGm9|hN*raUea6nax7(oF!E0|#Z?y1bd1u*<;ywlVZSY1zjDK#l7sN#Rolwu`}fGYzfEo*H-7*J4^7p$keq@LB8!U2{0 zRvUS*4TZ9}u??Zm27i!jP!k@Zd&}+5l@AdZTYD@;zQPRhUngUQJbs1nES6s{SHA;% zUbgbEsxXKjjf{_w@+)XvCbAUgR}dxBc9-`x-l?kXwi)tG?nNNhGw;~aWXA9(7aP}l zc0OY>@$!=E_dp;$NJ?LSzM$kAIYfRze~L4C@IeOwj6YVJlDM zkL+}W`Q8ja1Punf9@v=^-nRk`BtgkOl+U$&^)0-ThoXFvshr^9=K1NYMxJ~L^M9So zuPU*1*XYagP9iwAba=>>#|4K{mD+nW;QqR3bc)74o&~LYU`y-@thhLene~T5Nx_%+b~&t%h~r9Q^-$tIt5hFvTdoCkHf3Xde55=2_fwn1EKwy< zSFFk+j&UcdIK=8fM|oWKZMXYuf;w~|SW$iDH`x5PBr;+>q9!W{_p2NIwzc+CABxzU zr|Jl+Tpl%CO~DKdi@_Si=p2CMdxE zhj6w4G(q>kBe>Wn(G5qus(cAISd%XpuHwGwYr@=ROHjY47x{9iePaYaJ; zGfN&urjglf72btWsCcgxIv~JBKd@|$(PUo4DHrKLs;6uZL)YR|e?FnT#G9=ihsVlN z=epwWWeYS_gJq4$YhPvk344??n)XHq^TV{~L0!c|aa~p~Dc1qVeY(4&pk#pDb*6#y zQF*FkeZw~<_6YjmsrlO`bHWIF!QHS0Q`d>vvS!DBod=IC)IH#^;~<+7OcLUfZWx~! zypytDHT?z^bNEFBp&!0u>@^`0k6qI>n<3}`7aYk_v2M&o^yer11v}-qlF)3vGwg>x zEEP9)BZlA;_T%!R%XYL?6SL$eE#zi%Q4?A6Ce!ng@(_h3%h|Y2@Hyw3F8!&q~z zVz;6iShq+T&QrL=GTnO$zZ9{kT+Btljcz6wOA+A&#P6ug^#@;q9FPJ0PWJg zu}A<^FSQia2b4g781>P+yGBF&R%s0;fBYF zaC0TEldoh8s+b;bWXv?Pb^A0W<*nT;pFum(FS3KDCm~^#xrlMzxZLAYPJjNhBY4HX zxb4iC5^2u?EMXb~Q^u@tH;EPd)XKe^)VK_JqjfvwsZ1_~BBC8kP~`9$)s`ANwH->- zKw)QHUug?h?C`-$$Y4MnAa_9!f|?e|^YR-e2i%1RwD)DJ?D@>~YKUv2gmYL?)1lk}f6CcFOao0kHD05%%eC zx&Tn_jtJw?9TPbuP~_ylI~M|4nsDeF*8Wv}WT3b^otqK-TFO7|L^bJSK4#=%CloBV zjgT78aHsM2wPl8*Mz1WXW#_7;$3w6lp){yj8!*euR7yh~jEr8YyBVPNVp zz1rrCDj~n^3J7ntOm6uyWr)}Dh7yu|#>; z&pT3lR<=rgG&`Y!C5YTbI-8!I33pVx)*Fsj%{|@Id0_ zNF={snWXI@u_g}7gaGJdVQI-D)1Ns0wW0ZThiFT%{E`3;#}femdvAgclU}F*-0ktACSDs_-lVl(COM z{YjR0@8alwL8u0sqE}HYoWRZuy})r?i_Hw@)k0~=CoeK53*2r_^s-DW8<`JH$(zrN zZh0YpG8KH-7Xs%iDLw~AhB}n30W8VpP|bMavOJcWQ1S91X;@PpyptIy*VKA$L`|yO z*5NHLIPiD*$;EbUNgjGrIjy{+S%T-hUoT!K-)v-`4Xuh}H{7}`KcXuMm2t}`l;6B9 zCuMth-d@AH8e6~q(k)f$8e15Rr&_}6-5EC~TbQ`~S>Q|K$umsaL* zsgy2d8o##;AEwEiKR$VMF(~Qi@92CQvRG$o#WCr3J2ld|n5vJ~4s7Bhh}FY-vw^}MM_SQWyNg%Iv&iKGG4XsX}hhLu|! zS3HtHY~@DKBIG<5gaQIe(O{YF`qCJhK73za4r?11gwsNm(Cp3dMyt`zE4?w!x~LL| zZ~iNHxg5*l-g>WOtJvYTi;L^s;bLVz4#fYYl9HHtrW<)4P1_eRei53b} z$~+itaZkX`51%?B2lWqA>^K~CQANa;(Y_=tkqi~PnSO6^6QXX)%>Rxxy7?0#3(LYh zYwqUrZu3l0So|f&xshcsk<{%g&G9;9f_=@v;sY6dm=~Y=DOibzwYP-oL4r*C+5GuB zm(isA{ws;=9`l`0+-Ed2VH}X^;gECbx+%DT&w;wMrU41ARYRR5Vkng-sDuF2&46#E zsP@gg$jGH`i6{fS%4FV;>klYXW@Ge>kqvz#mwce;0^wNRzx{gp3im7f!Wi@0ryRRX zf;wh&xdd{Scp-C4jplQYGIbpUIL8le=a?orNcJ_qU z(q|s7pU+)7HsVaiV3G=WY5_>zQ-ni)OkdAmpxWaY@k&0Y>Jq18m-60dqx|-(sp(4h zpvYDpX%lSG*#-b4=V~eHBe0eO23A4-Fp?icq7m66`4qgxLg5pvjmj=l6?GVEv z<>5m<2U+myz;`jg?i_A`eKtWJeP9uXe_3j)OAKERFo^S^^an4(K+LQko`_dN~|jU|5XAT!y>_w1kF zCdruWQQVyb8a-Vrnn@H(U&EZ|=DbsG3W+{nhLM3krpfwwgYYHJqs*RYq1w5r-JWCm zHUaMPDyB-SY@=#P2WzZ-U)azS6gi*zErdYhtO~%`oQ-*rzvSJ47R(bevyFoi^;?Z4 zu|6G+2RM8*?|8U&>r?XlToSa$i;S>;rc{JN+9%=P2qx1xrlmt>K`S_%$l<=?%~gA0ah#A{F5g{)9(+-mV4kfMJ)Zky-o~EOM!#Wr1U= zBeHQn8L}VcOvAM3CX!oG>$KJ{dsZ(X5>4$d`$`5l0af6-XnASRl<^ZxB>n>UtO3T| zsY+4Wx$k{XOrYEsy!GUCDBjl&X z>Z2IxpwJ_@zxQE|*@F~NtxUL~?Y-E`#`ij|e7b-6gMRxXrrX-Lbw%H>>WII+!v5E> z)2Htr6FJ6m+DcR7`RQY)Pu+IaiINTu>OUAlPi=Y@r*@r3xu?pitcIJZ^vP@%p(6NWO_p+%|OijSanS^&Fk-*0G#T>YWCTDjU_C zYvb-c0`98TQa<$@E%`b?50Rd3I_(Jw|4jSlxN^LYrhYEPJb>7op+d#Q9IA_utBnz( z&b0@NMQuGF%i=Z&iW51Hc>B_86A?g)J2l0l|6Vmi#sY=1y%r87r!O%UTOw7wqEc#+ zH2{n3f~6A|P%aTnLI7$W6k=XT@O;^uAS;E9in-_U=6?9K4zc{?l%PXu@f=b$@PqZl zTk-x|Fa${4CdG4TIkfkiO6~At*i;)H@RM2|ykE-qZRakMCuIpm*X~>vG5O_O=9E`p z6O=aqTMr~>2nz6ysVpVCYk~t0Nw4W05mM_>lHGu|B%OP~N>;0VJ-)USdD!qepXzv+ zKAd-7d1d)E9=3094scNaBp~<&MAu( ztXl6aU%zNc#xHs{pEdY-WO7)?3|e#B8s$Qa{*ZZ2{7y2;*{_}mnx@N_kVg%B^83FG z;5AR(^E--7O$Eh20URUba{vSX)39JL7Bp{EaDbc)ng?RQ&5eFGIj=qq-smErVrnDH zi=+2gaaX!o*xV^u?eO7<5U!dmRl%dO8$vGx6|)HjxHOe+Pwry9yXSyLD%)#xM$7j| zJwAFFYH?wnvn^n6Yzn#|;jyv%1Fvm$*LY-jW3_J6e!?IS_>XQtrZx2wV5)3(wfbJ2 zJ~sVeb5I|af^6gfAXJ5NkVDFxB}AncvbR=j2`SwII2>+KD$4C6H2{oCaOb!F%j@W1 z7i7HiKrr}@Nn_jbQ}5LU+=tGZ+y1m3wt;RXjsh{NERJQQw_s^R-Z+?yN29=UYfrDG zKC)gM5%3=+{Bc7^P1|BO#|n`q4;GgP2J-mMyd%)N!VFEjAr>9UCXVtPJo(3v zTo;algO=07pVJ#(w5l%H*L0Qfk=`%S=f&`+xE{R~#-dvEdP2T|ndh%}N5E5+8Xub? z($wHFh1l()r%R((X{-5yTePcEzU`7FJo1r=a$x_pLg}RmK7ztpLF1xHAO;hBrQi$a z#SpDH&aLB}8L-GJ!9oHxdjlgGNT?EzzFRHOzCvK3&v$tcZ||t&12m~oA^(&c3Eull zAaa4sL18e2L5NKwH8OG-TeV+wq|#GmG%Ao*VIvSgsJD#jvpN~J#s;!*2X2z!x_?En zdNCjqftPZ)1`~1i`4itHcGz8*%5c5H)t2x@-TC^>7CvYGvQ9Z))@-h}`9uGiFy0s? zoHD6R$flLw{imcyKW-iYJHWm6+mP}A-$-yv8^;;%c@5~wbEX*k`^x{Co8WXlIE~Jq z_pB(ZEaak%c`%lhNHW=-WKp=C|vjMcNw<=_u{C1dTIk+(&?dLpGSt zx7|4v*FCqs58zr9Yk5-wNbS|)G0VZw$-h7f-V>g~Z_rb*LUw6$v0C8sj&@pOJb!YO z-*o=E$7bOZ%yfhs+rz*(dE7ph3 z^{{MrZ`JPJLI6MBy-HdURh@sQ?sdMAFobm*``IpwO^h4x6;8qQe&^*oc5&3mCw<}4 zyxo~V02>FkR{N~TZ2DTT=ZaE(!#*IqJGTYE@FH87f9b@DHtnupz{Xib2Y#5S68HG|8V9_ z8YI*Z-Mw}in|M-K@0PKvJQw!|W$aFR|F8X+{O-wRS3l;;-AVt}*|93q6iMCz11DOC zuTG{%;SrtMHka9xWU#7(msfH$_Zy)A&46w}-asNDVzKJJ^9zaW!EO8tt|bY+s^Y$r zwextQlQji+403o>E}r!{IrR!Gz=TgkFzZEYo%nudTL%67dx=89NbkljKFgjl8-A6Wgkyg3eIr=l ztPgA*rZwI5mP-PMK2q`a@{Zw>JB)88TpBsulVW?i=ZUFX4!E~BkPGzEUFsM`d6F2z z$X@g5DPQ$V2w0O02O3K)lI`5DbM#!@+kN+1nAgZ%mF+U){ot}iEmrsYSnpe<+VUP9 zbAPoiR&NUA)c4up-kSF5hOf$MJ;`d-wexUF&OYPSxT+g{GwvGIiAUI?&YH9>R$@|v z9Mgc(JVvqeH91w2l8BKPZY`R48Jg)eBwNFs=}=z8@%1OmAY^Q=lKC}rPL9*}E8az| zvDWD%V*RxA5Yr8Jmge}!W*1An2YZ<~`gvp!tX#g*zmV8hzNn~^Wo5o*gG-9vREtQD zjQM?~7L3OGDhM@8ss606%|y{9V!!aAZQ19;@|T=Bk6DAmH;N^&#({LLP51=kqir*@ zfT&7WD@*V-A67;+3)yvsr<0A=ypjx(ptH6=QoiXmyx^V1J*^ex^}AoSJ!HZLD_3c0bN2-QNgQ#WWw>Paoo`Z-G@lM17@{^zd#!uY*n!}o~>eP1ji zt9L|5B)?Yo^ND>#uyL=0nN9lp7=u{!djk#$5Z9>_kd|z|L&yi7Cv_qGYD|?3f9{^( z_~9*Gqi4ye73V$4VBpB6!<0^Aef6Sz2i-D&mb+~zyjw@K@F?cu#Z}xsR z=*T_2c6}4~+B|^+wz;R&UZ>^WB9NaWmZiTgA$LQ<(gyMYIGpS|e9RU)=H}R%!D;x2 zHH~T!4gr^Z)dy8Hwrbhh$v*XX8IJMcbC*%?2#e)=_bbrrm{|UpQ?(-|JYq7r>%d%T zTBe%4GDx&BBXeQ9)J|!ns9HEQJucynDLG#WVfNSr%9* zS$gj79c4o$n1M*-Z4Yssab4VAAVdP=QFq_toS8_CQTn>ZR?jT+J_&04!S|+k!TeuA zO}>)MlJ@YAH2@11EO=rh`qa{6MgoIoZNyCZ8oLp{x`jK4L9Ex>nQ5)bUf+*u ziNk+7%qe$WvYeSSI%xO*MQZ0k+>I;G)Gu!=?TLT9?4r(>#MgHH*!ZB5XW>QnlX7uw zY{z&w1(UwQoFw0RS9tApaEd6gopOuP5HN1+&^Z>Wa!D* zZs<@ZNe8e+c{e(&k2dTSMpKVpAvhGIki&l#rr?G@7#Q0SjOqfiIQjj&`{h#JExk7P zxmYvS9{19Fp?$|kIfQfav7M)Qx0U9mxUSu1xhR`e%~H#%oiq#UAlrMoRH}O~*6Z@- zb7Z}j^ORwe+=y1c7(TvG4LLO2!|#a*FlncLUzOF z6|!ia`OUDYdlyE+`s|q(agM$g&*prW-mkX{%K}kTo82B%)73Q z0*CH4oGJtjPDll=8#vQ^PMPCY4ANBRM5xKuoC%I5~a_p>zAF zzcTAfCuPu8`+y6jI-I?+a(SAuop>E03AK7U)?Ld@;&(WEz)HvzTtudRkYE~J zy15)5ehZWldv*OCTg7}$EN@DS*d;J`Q~0=b?W~p zXklKy-J}wkOQ_&z$Ipa$cSD$GlN0d=usS50tlL)s&-d|xvrhwy>>U*s*@FaI)2w@j zehhkk<=L#3E0XBWaPH1{I{`F!OK$ZWe4xe2x8u^&ox)$gF^51x>eA!{m_K3K*3^Y* zEc5utz_u03J6mENUg);t;vyiaT|o>&r8^^EG9^)Jd-6l&=N;wo1K*;gsxboVQcGD9 zld})vbqb$y;snHpR~+ng$aAvA>C0lmC#Cac+N$M0k}L0xY^2-!d3|L~*}LO@U7l8T z4%VrBu&)STe+&ti**ayIv;28Of^~ES7vVSm>!MA}*?!)5mJZXaK@P_YB2%pV&ETW) zVV&Vnyd{r`-M-JGSrs-H38yvgAdWBVjwXwf`s(@6YVs}VL&;PflMgdS>aLNQ{wH0G zI^CrH`ks?NG0Zr21MIQ>M|}B(lna*T?iZy+B&`CXgpw?Sd~CYjp*dIECCbNTAf6=C z#T2%$FT8C_g-$x#@tWMyl7$Tj%5`iItcok+*=zh}pet6))n{xmY(`^)F>KUi{13hW_|8}%d+4_$NGsg`0sKjqVyc&-JlDUTM-l`#vD<6|-4_SZ0_YnN~Rl0V9VqD1$nYY>T9t|D5obR;Ey;8iM&6Nh? zj_dZHun(L)(I-B1_;CB<<9GZxE(2Aqp|oO7M~6Ulv(P%e4k6YH%&@zMO6m4Ny@Tx! zI+d_~{+;LMTRuv^iHX-P?~Wf*R2gd4{H!-xeuEYbR_8kq36gEN&gc6~%t{2-&?cVa zCX(v6G6+RuF}*`>WRm_TT7zk<^i$ixd#U$LX5vn*KO+)035sU3GWhKnIn4%eJ0dUFs;s8w+ttC)Ck;ALXg( z+?uNGLi#J$vWdON8?}9H=SA|?|1wzh+OKq-ylCd&6Y8!$nZ}h@S#!?ejvH+5o=yLi z8Z%nO^xyleG<_s+>dNa@@Qzo+*k7K zy#mIPp20=q;z=Lwp&^b}{)Eo~jD4!lBd93qb=K63dzERlDeW>%omR5-Uw(Clft)2 zq-vWln9P;xt>|?&q8pbys%}?}08g)G@oj-*KH-eWnX6aSPeK+-#aWfVWm7vcxi9)Q zG_8JpAMf?YYdMcwwPn1%>hQ+D48~5ONX@E_uh8TVS_`Vt1~|f9^M=8DzK-(N4Q=o5 zA6$p6w1YN#zoRf6vaSda+_z7oLwTIK8v>pK&y|JH2WN2zr@d|C?W;oUX?GJac2yTl zB{(cj-00Z3Kx0ViZHrtgBHf+9wcG?DBt1q`-|-&mtDOOt<~9kLSg|8Npz}x<;<A7sdmbwg@yM|fZ$0KW;Ab5-jP_QWID-etd*Pb_r=RX zj%=8f;inldahyj4Yuo-I3{PP|&ZnO+^I_5u&39d}{dj^o^$y?RJs34J5O?AnJLLlk z0%;K0&fLGOP5DJ4#PTYtmm7Ys>9D*2Q@R9=l`Rr}BbAmarxb?V<7>_3>j?>~8kqaDZPx7- zb5(X;)r&>vliOnYN1v{?()D1jC4a@dcyq|~>W@rRX-{l(As5c|S_y_M@5wJUlU&ed zWY<6WEZtIMzdI)9v0%4sBWm1(Tcs}?t8{pE9i0dNrdXBHkv>5dB@<7f#fuo@yRn1u z3@)Dhx$1K3@53M{06jZ$ z|yMuF!*)fii0`hE`_W6GrOgL zw1Rxn`rr$7URKfkcxT;J*@ERYHAso~bO>8FT;HYXWa_;BhQrPE^rSaPXCLL7rj@SHF>S4#+w=1;D@VqZNWMXD32e+%`EK%^pUvj;V z4}q+PyOM)hi6oBo@4-_TQll;%tS*_B&UB*o)f|yM;rMd;4Ub_n?+Jnj z@9#l4aoH%s+!PcqGl zwz^x??d>nTQw9<%ix>ZpaCtkl_lAS0J9#G^zCLq8&7K>yDbpgfobp&{r&DKP(!b;Y z{7xru0Z43Y65MF8Y6HeL_KQyhtRVI%A^jGS)P5gCw%)^@y}>z!=GpE6_^p3F&ywb$R+!@aa=eImYCEtQ)6xvY%9EZezHo#w(Qu{VINmf#oaV z&mrh_)%B7AO}-Uoc({Pq2nM^WQhER+OrzIFF}ufoG=>O`t4-W6evBlB&|p^fNlf=6 z3+KFF7R7r49Q> zsog|gbV=}54tWl6?LjwIihGC3M8GFu_VSh z&2W)PZFc4_Z2o88{+$W$DaGkKn~j2LEvJ!-}5y-=N5|r>? z<63sAay=NdjC?wolV5<<7CYTjbxp{4^}5`u_H0)OU?u@Yt^3t+GCXopHs+fQ(y3nA zVqdYJ;4$dl(*kJ;cn+=seQW0`nh>6wznDCI@0LET#*%c#LEP`S_0YR-ArUtnW@PYvmcbRAOzM&0Yt zy4SZ#L^*teFNXf8%UqoD95!p@v*A!s(_LP*PJ6SYGr|7o1Inf0L}NAU&uj6#pY0NM z1>FZh`R73w-|-qb@cK*1wgmOwW{Y08p8T0*aO$Y-bJ<|FD_Ch#Wt&mVg7V44>AVb^ zZJ1Www_ygtC4CzI=eDakzN3LtK}QE#6CX~rXY>TyLzf@a#;_<^yu&Wo`vaXY&QJg0 zl3z4*cs14QGHS<1C1lFZq^#H1wPxzPqJBQ`%t`&%Gk1e;1W3=P*=4ZF*TJgGOj4xm z#C|a@{?2$xW8xY4)FRhy7@D7W)WPTVa0c^rCI_=7i4xB@$&4SM#xqnP!z&UVFUwZ8 z)i8tX-uu7{4XEgqFOy63h_TJ^2=ji&7zt5oTn$OA*9xG;XKe4K;d7C-SD_NiRUVRY%v#b)7 zjNxA4aP^B_LUgsIca2Rxhb-2L1(;gf_ zZUxFH*MyiSZB4QJEkbGHWIJ`^%A16om|E^{6TKWYtyxmGO$&^b4m#b*QsDjXm%g`oFhfaTf}V`$7rV$_degH?JM^7r^xFp|ZhY;nBe__;!r zpO!XbXWG=b(?0n0pOl2WdMgGRx%cJ?-wdMzwPW;?*GhU8xsJOBjWqlbkRHcNV|N>1 zt-^;vJwFgF-iOG`IbP9GzI7&lq5tvWg?#H@mPXboB)tP%IZk*q{zYwIn!eqDji;HN z2=#=$$t))Fxj&(Iy1Pw?-|H!?3}ykBC!mUzLKq;F7(3I6dJzWOE0dO+^Q z=$Q|Xp>GXnZ6$_A-!$ULH&3GG!{Ma3)Ys{-@0gwPMG;AzfWP2WHjCF@g(XV156yjF z1(7a~xgD@0!u^L2(p#%rtAD?D3o{1`vjhj<`=c%WbL0Qo3jbbI3ogkOT#CL)Sz90e zcU=6hP^OXfg6WRF&hICd{?z_G>Z(TTs-#!Yu{CpcUt23%EB~2ZV7(IMvvnDTU3X;u zpJVko^J>DgI5Fp^XlFsYN4Hh5AA+$WAv@;?_nsr`}SWK{yWrku$VX}5oS@HgL3)LOw(WOaQ#32uJ0(#ofvd1cnQ4p z-xKi7A+6q8HS6z7{^x5>5@!xy#wIOxpb&~l@LAO*7ZtKr!|9YTllYSJZN{y>L{IFo0(>2Je`N{4a z*v>NDV?$eX6)1VX~6d$PMluL*Xev(c* zS8=p)%UD?B>_l|GZPHrYRi2|}Z5H~ZnJqBuo8*<@ zqGg#hjj9#IEv3i#L%GBeO`Hkm)Hhp8G24D~++D-f@XW<9J)_cg!Y9GcI6a?15?R)_ z0T8V=C`0DM%lgrn#dJ=T@g(ur*b#VW^b{I<^?(1Lp|~VQT+zbViewo}_BBF9rb6~DBr#;4vAYzx#3;&6%93QsGPbcq#K^u3 zGa37CEMw4E?=xM$@BRJ$?)!23-^`qIKIfdz`}KalUT@yxH%SG>RN(_hab$-9YBn*8 zKuTgq6!VZ=7p);+dmxU;of`f8@vPe_1$ja9JBdAi2*A(|@68ZfNGZQ*wxMg?m1Axd8d zD>{vbK!Bm9Ql)+QN<#QzkDiP7uP2FY%W>a=lk?*p9|tI2(jnYZV~^i3fd;+d*(*ed z#TG~F+3wZe)x4_dclFL4v(RM;|L0N;dnQDS&mIe~68xQkZeXF@YbBmO+162&$?}AO z?p;a`0s5j@A%MP{n(2)bt2o>tdTM;RFPa!2H%k&jJm4Ex(L89?{Bd+RaU9|gyz_fK zU#sSorVpb>i=@5E59p)pR_Rk_RzDS$r2Pg=_`Ev=x7u7PoNPYXu=YDd^C_sVe6Dyz z-;zn~o{620V1_EG9#RKg3IGM{${=1MYIT)9WNek*)~^9bk=@nzv6V$}^dSfpXaFXu zpj-C06lUqXL$(b-#8ilvV!i$8eUKwyFq^k6bQ}sa+I0#Rvcry6vyW-n-_`S$)pn#H zVS(SR_AO@HI&B7D0ZrOcm*!DAJTlgFNNkD&+V_!d)LXk!JHMxbRWu#%H)W%@vk%9z zy8(U%(oJuDzjBcPidG)<7wq)k;*WsV-`eW>(c1b5;^3j?tj1(GW0V?w%k(pT+X2{t zrDme$Q!5m)Q*1u(mJ0~ zyV}CaPhIkCshj|$&oIb}dl$>K?A)N5nW876u^g!q+Q-0EQ!mK|y7Cot1HEwQ!RGf~ z^($7lcF0n(=_3{nBbnu`IqaS|=>G9uYQhiT zFvt&8eV+m}P4@zo&e4p~YiTaNpy zOiIDCuk`hZ^avmqxCU*uOa?N$mOWC)Dph0ep&9;B&4`m6GR>8JZtIZFiLcUSsYN_J zMrn))oZFm4AirQP(l&c;FnPa~&i~x&(EyV_N6%llP*-z&w9l+$YDQZCK$cowSjdur ze$d*wmhuIkNM3DfA%3UHKi#enQOr1w;q>=NSn1n5yZK2m6mgp$HT!C233o>a3X6%^ z0Pd22(8$i*i7UzhPgHADH!jt7Zk(!BuD2i$X?XfJy`OpJdQ7i+Hjp=g1$0bX1@2~k z){~_Mk1^UQ%6q_QOAiS>?rWsvj&^?uU$rvsJ0KzFWrjTNPGDg@Mm|K|2Y7DJxW@MV zbK`w1s^5AUpc-n%(fpym)YxuXo?~>X@;ThbLx^+N=OhQA8r-z-+Xst#Q_=iT@(kEI zAN`8aCQd(|-K;um3=Fmg5*|vs)l7!=argI3=W#m8Ub692i1lqon*lYC0c=M5E3THB zpfdB)O`$G8h>>4S5#_W~*rvx<{FrJ2<`Y8884_?EUx2d*vDy*H*FXyr=i9b7e$h#* zT(E=RsATQT^AHEc1>QbawWnsCz4=>hTq%{( z6W(2tAL14AY%r&Wd;jW;#Z|BftR&*kc*DQC!f10hnFgYBN z$!zccBJb$T%g8;JP6eVs@?>@bfP3zp&bknD{UohKcG!Oo*fAK+j7FS~{1l*w1?D9M`Phi}DqX&&AfNSR`6*r|CQjr>iw>Of8`pZ?(y|4- zPLLA0U=^$DO-@h0M5lYyxju^Rp7sain{J+ab4%IJNs;ygdtF@}J{Ao?QVjg~8aHMc z+%~8d)@fiX+dKz6cNN^Fv5P+#6LdER3}AfWLZowkpfezDb!CZiZn(XyKaHMF(Qym| zCVRHBI&Jo9+w2Qwsqn2m87%5&R&URW=z4PT*U*>D_Tjn9O|oyA02D~-K#(*@^F`8;JXpxT2+%wXHnA_^!MW0qn znV$dK-v=@sfp}WG5A1yi-=12rYARCH3!Bf?D;O#{c&uU1V~A?~7A98_4fj@W(fnmS zRbB!Rrd1p_cbqKVL|{vdyFPuI^(x`=|ApfLv6RqrQsJMF+?Vlmto% zU#vw*I;wW|?!1fEnHBy?D^gOxNi%Gl7i?2Mu=B#;p zB@!JJq&w&Y1)K@YUJ77>^QR#7#F;g=yws8WtNW`6&(wq|xA+hwp)`ysC(J%lJV_Ib|93u8$TLSKIIPt|VLomH?H zc6RezyQ8PoPo0rCMUqnee45)lL=*dZHF1~g@%$n4)rd!1M#oh%441}^dP8?AHU-l9 z#KoQdmZA{;2$|y>R(qkV!T;->XYU*)(~S6FgLWxKX01~?(d42B)2;ohQi^XXJ%vO_ z01PLt*!vT^ILMFz36y20aG8m2jvNo{m0~RYcpDlfPr}Y+$!SF|4CpgZJ~isppU)}v zMVTROIy_^$g}P?U5n35GNa@1$>yI3m^}r1FJ%70t4Ytgig8V$|$r$kMipN_woV)5V zGas1OZ7Dw!BJ~YW8d0E2Ra30yb`frJAO$Vx-5djzg_GrTG5Uml^pDu?KAMqxBHuY< zl0jg6DA)4VwRrY(`@jTJb=*t&`b}y3I~@wJvbhK_`iJ5=GSHk+O0LB91Ezx0rU)_w z{(I1vvqc=lUXxT1kML4>5@dNnYnVT4sT>Sjhcek>Gtb&(<0;rYGTM-ixwK8igm(3hYNqn?McR8_TE;A7! z3)wF~SrNnvSNYK`tup5T;*eEfDKg^570vO59bwT8rwTAefCX*7_Nk?I`@@Ru$1)WO zL8BYcOZ8>p;IDKg{-@r6A|_JdT{&)@6%kvCXrjA|90mP&6C`zG%Mk7Yf>s2r;z|hL z)$DI5P4KPKZu5?DaUt=;Ks@?0?VS+6;Z(2OsgEEL%;hban@HBdf;9Z*0&)AkN@vJl zQYgtWeabV_?}IhoGo6D4KccfNAkRp(HyA%|zKw>VKoVISc?9#@Ye~*oDi`4C@olW( zaae7VNqOMM{`^1g9ea=WN{>}=AuOfp>();g|6qfMNN4lCY#FgpMrFhV7A8J6YHEt0jAjEGKw$*CV0iss(R91)J^0&P{~)FRAwWf~1|T>!Z)NGy11{S>ZWd!Id=@8xpy%Z2q`QF$8ls-D)y^um#cx4nE^{ud+CR zPLXcN0a1;hG@K-s_FiqoU3M+Y$D2c0dy9yBjdPNo zV{@eUp?e}E{Mu4DyA9Qj-|_?TnQ=$~doz=S?lCq6l;#z?K%tL@MqcScHot&)(aLA4b)l%2Tcub8vnI5&3(f(G^Vrwd~5uCGJ zG&64BoIu9STp!-W;vz(nQ4wKk$%{*|*1y_saRn}{D%AW`ug@n+K1HwyV*V21eiQM< zMf-v@+(aaS<&};DN{KQodchCVJh}GA`Ev81*Gx|{nmUSRYy`}x>U;$w#}uD>f@_u6 z`Z>#wix+==Ap?Us%5Q#3_8N0nPrMA}T{lciC(YfSpt2@xT^1*PIs5(Xvf!e|zfs#OVvI` zpI2+E4Dcxxu$9IAtQUgM!93I0$FIFSEE9L*JQ}%jp#Fl$X%?-|3v%99+d?5Xzoyjw zSn1G+FreN;boTrd!nv5HUo1Nmf7=Yp-;-Y>U71-c)z zal>?Nw3I2isKO6^2G7}ie;U8+xSQ!>2pM3VWLw- z6(7W6t=6BVqqjBH+YhWyW>Ig>iGjAsl~PT4Yd#<5!)VRUPG9-2m^D?mB|~>#wsD`X zE}ryZw7Xp>Gjh(S#nRtAEi9_^QywiEno z%q}5+jkrlT8o%LBSmLu=KP9S(i6Xi2&T1u7*sDtsh^a3`+HLtShM%4>3A=8Jm37|u z9Qj~t`Po~$t|yt5iP#~7lwb#uz`3e(>xI}iz1m*nJR`|$N-{fHn_A@&SHx9YkR!{p zmz3u*&|-~ylG!J6;-CeFDzu`V6o^uAUVc%a=ISdC!>S$}fAt@>^DVz?i^CY4+ZQOU zL8XLVwl0A!n`$iQd9>hkY>9*(yMTwreyd|ccZ9Ps?c-$2I z3LFSYc0|KtZB2w6Rm{%yoW`JkYbj28uFz;1pYZt>UZvl$JAFr-e5&8=!}3#HHLi@K zlrTa%S5W$?Df3-qAzLn7oLsOMyO}bN_l+y)W)C0*ev9WW&wEZ8kv;0#On+N_NLI75 zialJ7s8hB2nySSbq2ZkRR7*EpcloY=ULoxbH^f;Cp0X#e&e!aEzVUIsTG7EPlw|;} zKEFRoWBYkY_!Y#ne2f`Q;!j9Z2Xy=$Thp_`>`MRpny6dR53a!G-grKKLhRJBTQO;@ zG7McVvYMHys*Yk^`aDlQ(O_a_ZV{G5ya@*$5ZU1jC>;J&9=md#Fvq=&POJ5>qh zWW_o*@c_tb!>GFlot&w3RFP$NsTve{AYy%zGnHzgl`jY_6t`7(pGs80HRNVimbDkD z#h7hyT0DV`8J}MO-$t zYZVSQ&hlkzRVA||zlR`;sjjfSn--Ma5$U?zhF`hP*^<*q?Ov~L%CR7X^BBh6?sp3TxpSlM$a<1euG4Lf%mOVT^5s;q> zQ~a>er?TIb&6;7ekYHhleDv_W-<9GI7c<1>dAyEpCZy|Xhsk>Jo`i~KA*ZG6aXdi; z|30Y;(U*#No^qiwl;;m*!h(>1Ge+&0>n} z!EPE^gr^VQX)5O$Hp(&a4GY1756NvxHzu@?k#Bi2mARzKqu9~2rlHe4j#P0u$M@=v zmnP2IC_idF|DyfJuVOK=_Ud}&**f-YPdimz z(J$cL;wSu_b_SYb_kJ1CawTZS>4`=b(W>|(1uyqqXkMD(QO(o=Y(ZE9u%v9s)fxc} zTiW|+d$WZ;Wydau@t#TVdAp-;;LqwiqmGg2iscO-{}i9vX_OBs1$$61N^_t&GAfr~ zDLAhYTU;jVynv;Ciqin28&U}^QLbWz4v;3*&7J-}@n^=Ep57tLhQUQGjO;?n(h5D_ z8hp*TkvDA4<8^ZYb&|KzazgphY}Wje@h{aiJ~)FD8BfcF@;9UxLHB$C$a zKxf1LxQowO3je!_t>&K6kZG-^&=t+acacoyC*NOzYMn4T{Hy2skDlK8rTla~)t-Kx zne~(3`NIxsz#w@o=mv9&u5I|~KYD6|u->zGVFC(_nJNR*vo|6OCemoJC#26ljfr>P zgDRu;I#?c$MEVve>iez485{uaI4GiUkW+X#-KSo=Die*Hl6@HB^s2&61PpiIM6I4q#UQ>Z@#&{Fs(w-R*N&m)$rk5soL z5ku05VQl>R!M^Ul8{laO`|i`+)+yDv<`c7q;QN%$&cQY=G4{n@%z%STQ*GT$Ym{6) zp+Q*=iyr>krD+Z>qU*{?eQ@nokE$p8N2DJ3eAA$@%PH${>+qqSYHuF2Fz8PUuu?F> zM!Xc{gM;&hFx}V%!pgzea7Czgvtg}fSi%^$7okkIgbj3|S8Sp~#85fZR%V0e&r;6C zf{Bgi+)vRO!WCgShGeH3tS187p6q>P=I2$>?IIdTHL2+^+AoWu z%O`Sywi0I0C|Y%2qko1tL;T&_D=SebEBP@W{;-JczLtLYFn#Lv)cCZ_30Zf??-!Y< zPF>uK8(H~w0budHw#W{j)UYyKm*@EGSRnJ;9`)Uy%8waON2spv>nb|o+8hhw?Ha|B zJumW2p165f!Ch08$J!jF-$I2gZQR`ZT+wrZxH0TQtHU>VRI+fqAH)YVZeqtaB+zJnIl(ghb>)v)T6U>#yC8y<@ixy@=2M%e z!zdd&du$gnJ9{+?;#h2*rq41d#f2aYc+sx51-tPjM1ht*rY%C%mp1VkVv@9F&!B**ix5X)ej;E7TccoQOs6MyvmGq8 z1-FR0SbHGb6PAvGZ@Ec3V-bWCDx3b!MDzMWv|0GDHWx!N#CZrJz*OdrPfV2hhI(Au zutVrK^ZvBpDB5&pP~U1!$_qu~z?UhGK}XKCzt zR_zhW+Q{-J-VsKztIn{C2!@!@Qc*G(Y~C(>2$4Ry6G0P3t9e&m6d=z?0Gn$#1njg8 zzBhX+JC|p`Wr2JUr1u|-*WLckn`*T(CtmQW7Hd!@B%nw0_~48DX_M#N94skYuUG|y z4i`V5*_%(Ks8joNe40B~m~zJ+GDU5CHB+a7-lM&<0F{5L_NCflxQgvH=&f}RrUWsd zyeH~uz}};Su&Oa|kWiYn*1hn16i(c(v#P+!q|0`X4b;4sg*g^KKq=y)j0ySZ{!V(- z}~br1>7Fq?Pov#Y;XN6L*RGDo3n*&*}R0e{HUR> zBBoz6$$0Kh$)orKq_zK;vtve&@}x0 zN5^Eo^U7(RDYYOh3=jEhxnkeqz*hy9{}wMv@nqS`Ww<$3Zj{}(@|}8HReO?CX%ObS z6e$~i4sQ3B4#Md0XyjjK?0|+)`zwGaL@?63^XaHVG@E@Wz$UWWsx>*a{T8MK-exol z9jSe=mHS8``Shx9VX0js>U(|Q6b&j^53Mx)V(R&94E1jF$`!Qrws>giSy2Bb$4p*R z`$+*m*NZRa1TR+vI98<>Ez6+zGzzl+6Gu9Kag=uIRPzfV?EM6<7~UxeZ~g=> zUUjVoEP+D9H4mv?S`DF$8K{)L0Q&)j8b~shL90C}+HNn|H#+D$faLxQ)vzC@a}wP= z`^DFkjNjRdU%Y$5BV090>Kz&{n590* zs2vhsIiLW#XXk8&`qsey&L%K%2cOX)SUKM9cplXcN&(5EkQ&>`?C;S8z2%d>46pY# z0Q8w1B55=I0UrNQdx24G{V78+rRpp#|B!8>a&W4r5l;QJ^9%DE2>D#Z_F23_*?ja# zq)7w|{H(pB0X}EhbR_(g)}Sg%ESrJXS1mRC@%H;XJ_JafWI*lj(MIGJ4_!BIcr5n) zJM&FkqNpL3Vk=P+*C%RNPg<|s&c-A*ohwa4n=LNgtfQ1V|02HKRm{F^Ohc_Fk#x8k z@cw->7fAzEuvi*BK_I<>SLyXw8Od zDOFjrzGCT=o2WUv73NDh290F{1rl>pVyDB4%Z*mvr`t}Qojj+UkC@4mpo&k~5O1ri zeR}!$K87O%@N~Y>*nZFg!y4m_tUxXJda%iQ3Q% z;I6HXh0+H@afU3zAJBar_~_g>0BN)@58Lb$i8E_A!p@6kNmKO01`3T`za>7%3glZ; z-;IH;Kv(`bba`TP+Ijb-lQn;_yc3+f9Ayl?aq_ts50kseU6YmP7>w!4Nf2rQ9=aUm zxh!g8+m&l9daa&oq!AE#2)21vC7DzG4b z1233g8qO5*XuJ`%q!nHS8KZJ-TTU(a6($D$dRBr-^rni}YV+GJOU(8gXYHz*wJ2VD zV@A0Dc4v0@V6kA}S@P3DZ#B1|saKfSM&`sj89n_5_`pKBIapKa$o2o`j+*~=x7WQv z40K?7Vb1~Pk%xw#NqBWF?bJb1v`D+@;IipaPNBmhmr>UfqH&tox@#RJt$p)U2$`#H zRx~L(lyH>fG64Ro5SB4=$MRg=?DO7-4vTN6Rg0~NsX%F6Xf_xFr}+ggOo#@)40`$h zPI5QXNL)XL=^f%Og86@7UJn^pT5=Hq@% OZs{7`DABh6>%RaXINjF( literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner6.png b/docs/images/LogMiner/LogMiner6.png new file mode 100644 index 0000000000000000000000000000000000000000..053b4fb3a8498bf605141a3b6430f4e774fcb3d6 GIT binary patch literal 68355 zcmdqJc|4T=_s3mHN|=gJmMJYrh{|pxNo7wAk+Fm!+rTI-}-!izu*1W{n!0Z9*^d6ZSU)x^FHT!UgvsYWT3;%d7N|C zu3g+WZ(O^*YuBFPUAuO_-@gy|p9VxIV%M${yKY|7yyLrjzHYteUh670`G@|~+aq#SlHab3ltXHpGAYaJf` zs^M0cN^tBVu77u0Q1an&4sq>e`{z#sv*8NoyqeYIZYjb2 z`@%wPPW<)924Z)DeJxOLn&~_7B3@0I<3C4SxlsJT{sg-Za8${@-~RUy*JmBBcMX~G z?VtOn-?GY6X~{H!Uss~Bu}K!2*v!1R{Qt#u)aYa3&F;apv;&3*Rsz1GLs)a$Up|Vz zrEITN)@Nj);DjHf*$)U9>GmqY((c6>qTurT;Wc?*+?~}2>sv{>0o6f)`TdofO9K~u zCJBa9C=_UuLejC0??A)7%M}%=b5(07Y52l97&6Hzn1DEkR&xu$VwPjs^J9EsGlJD| zBVte7rpObw(Q&3hP$6$rEk*=yUb1?{&=2062qO(=51Dw243*4p|9n!P_6Xhv%6OhS zd1~YDHKw;;7~@m`6N8O5cN;V|A_5k8pM&YkQVY?14g{V69$bI2=jN~}j1b@clH>AP zrr0KHF+}6s9V*Dsk`eb2CgHiED^ooga@4CMKQz^T8;l8q~ha9Nx@eneN>| z_X)X})jm=uBCa2Sb|-bS1(9DP;&i46USYLh6vYM1Kw6y@XxIMBh#Cx|LJ^|=5AGBz z9Re?_&KEa7`td%Qr}C?3CpAxlh54`g%R_I;As@uM7c z<(YJ^NcFX0`}<@LEc;#++L!A+Z`(^s%Iav{s+CIaH9vC4C!%FVeZG3&6GpofNN!3? zExd?}J!)P`td)L}$DoB1w~naa{pjeF*~N+>>#lwEIAbcI4N|krxIX3LfgGsAh<%yBiu13uE{6`a;87!WAky>0=aWppE=}) zMFkQ|RtL<-T+uv16ZU%39vzZ-Tr-&g52KyQv5KOvt^I%S^Lm`zdFbPY0TNp;;M=Tq z@Wa|AoH4d5)$3lsLIo*Z&?_&)$A$c`p0$?aH*rvCfg*@euaA=gN5Q57*~-5A&vEmo zcW-{o%RG*yPehL=8bXf!te(m2+c1kms$jQPMmtd=rTzug({H2&(Dhpk>Ke(t@~fm1 zS0K!rVA|Xa(oiey9|czA;^#M)UFj?RhKPFBf;(?_8iszm9`~Xaspv_k2Rv$fKA6%S#|j(=Ljy0UlBdNecww>D;)lVQQi=gU6JFEj^?I3$Yz zw^=xz=8ipP4LsD@O=>d;@zEqNjx!aFWw=z0M+)JBdrkJ5?=nB3bd%okt|eFa-1di@ zxZ{EdC2QaDi1D%>h%0>J+DsKW&Xol`MNR8-8SNq9MaVB}B9%6qTID2fEcGz6Cnrem z^&-}RORarYl18I`GZ|;aw>jYhWA~ymG)Df6SY5U=7@Agmf_dW>pAg|<89J2p&nzlJ>hX) zyES57h(IQD3@z&~2TxCBiV_>&%ff(P7qH@0Jo;**CgJ!a^` zD~x9!uTZi!tS7uh*lPl4%tU#Jq+21?`4 z%wC;vib!mt0PF3(e9vd({ZV7!#cX_dlr6~G9NmU2!!?`D`7c-OdJ5gd4VZZM=GmMa zOoCch$7&fj->TS>jC1PARJV=j-*ce8{6*qK9vA`fa(e%!dTSgM@<1g9G^*5VU-9bg zqc+HKw@6XKb-B%gwVR{D%f zUVIKL`1G?zTA-jf5wEZKt$gZrm>*;1hqMBxPSitA&3XFZ9;7>V!!fa*K|x868cm!u z1D{X%RHEDineGsfy&OD6Eds3iwb#U{vr)eFmp_oz0GDq5_GI+ivMKGOto@a06NnAv5cSPE8hAGJ?Q^l03ny2#YWVl@y((z9eg7lv=ch#;eltEDx{Xk%C+n0w z2`BR8Yaa$QR!Qm2z40M%>ULm>ik`7|>rv>XAP0UyBeP=3K*iBO+IGzkRUWB|%J2TJ zi9;I~_Nl0uSAB7_GQyMYC+go79!lRvtEt|`ovLHjXrE4@(2l`YkuMkLr(jSRW8K&82caPi-=OLE`Jx9EPV@7h- zU#V}-mo0pCubz6{4EV$lcHBlA3BNeab=Tf0km)gxAw6sc0*A6&dEa$?SRLR)K`ST9 z31*fRa*Yxo3*g|MN1x^J<)*GIrF~0P@&#PTbhcu)ha=aU!Z|?3)1UyH9}skiE?cMK z)xN9*ozU)97B{Qs9;?YR`tCh+PkfSEf){lqX#w6;WX$utQ7~I=zhIu6pe$;stx+Cy zY4WHLXeCZ*;`#Uz9_+3h20~}r2SrS?$^Ns|o9_}VwxKqs*ddO#1MYxcFm*ion%6k` z1O{O==@(Y)Q1RyNanV+`h*YWOz_Fn)yV0O^cmy|N-m-2i>|i7T3Jbd0Xp`_%1hSxd zmi-87ezAT#SW1TCQlh|zFUY!mqXd>-Hy|8%25rcBeJypa@-}mpOhKU^g7)#H@_8|o z#-6eft$-^p9#GET_#)1(;599@+pjLO`s&Yg3)8d7Cb@dfmWSGod14%mY$ke`W}mQK ze($f?8_b_^l2zKUNnag;)gLY7b0797_Kx(GeEq{j&&&Ena}`KsHVcT#hnvyOf;2w$`Y9Z4LK5w#8%H8qS z6tZ+dV1Ao1RF9F9w1Jx2M)Xyw2_EW1ZZl`rq}f47+#)=Gsy%qAyWD{}g~>0j4+-a1 z5ZtfnE^>ynUDYY+-9b?CMG@UXNwR)cD)t%`ls%KH+q*3P;Q7Vm~KX0bzt1YlU zy4Xco-K)3HtI-PBy_s+2Y(n8j=tg-D-buXukliqWhw|C*vs39RLiaR2OBPF-Nj@I8 z)iwSQJFti2){+OUOR-YDbC&lYDr7bvMsCU_E>?~P7dD&~jXJbOtVv>1kJ6#@8Gr++ znrXc38^D5%LiYDIz7`b5d%WkH=98HWSI5(BSHB;1<-B25WwCHc0iKOXFt)j9$Z1EX z`U_1MT^CYjmyzoE9DdYvdhh#TFI|LBoOod%)^Yk|F51W4w+4vUwS zZL~9KP0ids(`DTmYO7LszLMdK(D?=_b2~0OeDGXh<-?7q@jN8|wzQ3~nMR6Ygj!IK zIGp?xAFk%WeL&Tl9j5M7!@Fuknlk)c)X2SDt&I8idS@DOULz>b^)*0G)q>9+Jt zu=+d8X5JSZg9$QU`W#w|_M}!^Y`q-8(JG?xjNF_~(89_?!MBXBFGYxKnLF5D=1q;b zPN=+q&}qDOQnOD>E*q;I8RWcrv*ebC&TcJZophmxRklLOZ`qIqkIXE|C-zP9#kGlm zs^OSaqbsF%IY;ZtDMl?9-LbeKG=~1zDk`U*IVB5Z3Cs(&x^W7$tEr%k@jxRzf@uK0 z(NIIctLv5&>8QW5LVvElYjxI*+y)1ONgToT@mH;Gja)%lsi8^MOD!H}Q~}>`ppa%v zLGD}pxeJ!p%0mZzV?hwVw>D>Iemc(}q8`FeRtobqs2LqTHNymN)A15!K{+}nZ-p)Z zv2(0sJg{vBY1k%>A`CATa$-|a5s-*WpfhbRj&_(^X|>E$b(^Tut#k_PP&s-XE*hZ1 zxHD7fhY>U3R{e?!3#*;O-R(+OX}gd7Q%{_?;VM4v_+^fx3_Kk8)I@FBUUPkB*`4nv z8;)N!Nb}MK`Xtfa`xAy{uthR znoe-yyd0(Tc%W?I*4fr-H5PG*LpuNo#!aI4r%GwiTlOK14xVO>C7W7dM5yFGif%@` zgvQx-RQsv)qn1(9W?w8arrZzA`Nr69xd@3$ec&G+{-AT-wXniWifEpIkCQtVZ|W|3 zZmcBeSzsyqRZFu#zp)8KQISo~Cm^H5fOX#L6co_=&&syNjwl3?bfqiq&~BC(gc{1V z`v^`^CO}N<$9>w;2BHiX>jaZ3#wne7CG$cKcfP*mK#4I#4fUmm^?1LN8_rIKc;nbm zlKqOHz_Ec8KS64!mIc8cU8UhUuut8rYpJWENh7XUF$0`YBYhn_J6Q5#$f~y|LHIuf z;lcfE1z5s;lM|lwMs5cZyEczh3g3-_{WW2c((H#Lo=QCmEI#$bK~RL%H(ie!F>%Rm zizi437o*r+q%Rd~S^E{eox{Ux7{cwY6;B`wAD zs5K$*nxS2{P%3et_p_&=KN}>NABND^>aUKG2ql_n>yMNercs z#n5nONkvrh^@JG}EcU8+J?@XR$+Lf#(pdYads;xQoV$5Axr5)~@=t5?gO~X}GqYlM z(J7ftuRxEZAZAY`%>zrgRAy-v>Kgm+CnXm0Gl!#19fJ$B8#Rsl1%z<&58v{GjiPRG zl7KY5zHTpK;e0{&?Rn$<6NsjlRVfEOj0E56oR4c(GgqF$J5&2?FB1iF#=--OOK+CM z^3bwp%pUNZ`vVijs;qxZSBGLS{qTv+R_Hx%!rIFl=~Tbj zlX)!$NVpNbAro<(xojtD_EGuCXAxy=?2EZ(>GN%B5>XooVp83c&kJR08xDvHeo)?H z^3ZskqkO&^2O8Z=T{vui0fxa9i+@8v$&WSu@!3 zT-2Bj(~;9`Y;8PuRw|93ZDrDFKBC?@BHIOGP+du@w`iz%^O_qI(^rH{=TRUJ}o1+_*)%v51ol2aOud=eCrsmSRSA`IP_U9-IQ{IDvg zMmJb$W>yZue$mgSh%a+KTPImstRTz(doLm$tHJ5;^>`YsP|c01sd)>_w7l|INy5lN za)twuF2D@$nR;#c@Rd8ugU7dFLM_~0v|KPz@xk(y$dLPd%MT5k_|KQJBGYa06#3EH z7jj3h2&yWNy=jaao}%UGO|S*ngq5@$21(3*?5WRtWbK!ti9}qRq)&#%+kpdk9^nSh z;Tt)8rvjT-<3K;bAEoH+QV^G& z!BXyXK^r;aof^wvC38*Ng)*BWJaqNhCKS#KX?#x zXB}J*X`?+{VD{m8vE<&oT~0E?LTPIhOCyuEiuktRxpr;O!+-5F)I`8@NnzoQ=MPa3YT(JXEGmjed-rxj zL;X|gse5x=cJqK#ORY#bWmZ6=WnW<}WmHZC zwMIx3L&1ZZ(6qcBelgKU8`q$7NR8zHe25 zJiX69P){ZHqTy3UZgW)~ll_Xn$XWZ<)ddsEMmJw+#2LJ{^cEpu>2bxsv+G>Y#6lm=P!R^HXE!hz$qIpjW)b` ztF9gJJNIlx-?qtXFbEa-C!zRr1H1s1d$}h7a`vL3o1Y8lpQZW}irxs#*rRG#@Q>yH z1yN~-0j4=)FHdWEBR=>~IsX?xu{L~3L6-h&Lxk!%Q zUu#ZB$`uzMblZcWqaqxpESW12iR^!PipwH{;Zn)BQ=|}U!TFi+k3K7KT`3SU2|d4F zzs;J>-etXja<_g-D%x^k&OT!9uv!2CoEmy!C@!Q`AA{CMyViqsNkBN#0{$87uD)2= z0X49?9I{-3VpJ5y-8nYkcZ+bK3n<9`L(iuVupRu=fO(#9tr82#fevpjNYhgG5SwrFLa^PTc#V;7x4c2Om=%Wi|VJe`>5gm=K9kPC#dp zCD63%I@Mbld8NnWA$a3ehFEp?O_3nOX&oVZlTqc~C&Ip_hWGHE)?^%~tm5#m6-nXy z^EOg`93*%|^eAPn8T3MGgx6Lz3=A{3-VeK=H6ABvB{f&7=Ovlp^MfA3-7$Sh`-S-z)O)AfLAkr{5*S{HqWcD2gGvc<_H06rLT|s^_$rRtS-2f zwx2Dux$Btf5P7j%$^NnMqlVqh9a3nubGa*@I?kcE@&GjZXetTHGlLA-y5LP2FfPF? z;e-?e9``zZ-hod)&={giY~{vu8VG=YgO30aH= zG)G8F$alXPZXgxbO)LapQ%;IRNBdC~)rE4Cw1eY@f-=$R&Lj5!mCB?G?pPw=_o_u#^EY_`COO+;=wApGMb}!xG z>(gEU`ip%6KqfAr8g~#HT5{}qLx?Inu4;dB)D>NK_i%TLD}ehxn4@Ycx~fn`H7>O6 zh00ML*}9c4Q~<_d1brV|>HgZz<}o(3_GC0@2>0&8;0$YhRD%i7&t|V4b;UQl>`EmG z8q?7t*asJ6gX5w{QxT%GWZ};J@|}y@rM+nPP6-)%L=JowD^zDuBN#_oVJgutS)d znfdlPuqWCjnz}J8-RoAV0Pw36bKP91bY}<4N0kB12+S%_Cu=AuqefynJEXN|B846* z5tBP9=h+3MM);x&p2~NFx2TBR=8g-kXKDeo?0+ZbC%`@nZ`i+IF$`h!@8D%UiCr&7 zm5I4MjU8%RJB%DNX!3As|7~4q!hQbSHj9tB5>qFiz4Zj#!&znel}Bnn z@cc?&0~YIYe5s8c5QT!}L?PzVQ!23Nl}X-#FWGQgH01VIW>(qYUDpcc1j)QOkGqGa zD((^(hx>$f|$3xmHkjV=5W;onC;+>0~=nG72;ba;n<%K>G@cX*O5pflK(7`B6UR z5u5d5t8ufhGdgrt6?^`NZSosjwt*&Fv=*Y?VgT}#GJHKtq_tR^Pt8nHGE(}r1nztA z=E69Dww)o;#Utn5f&d&_V{tT62Wb~uC`qw;^36LIe@>BI2=#HNB>|*;Kmn>z+`$?P z2l2Sc9aGeaz7be@X8*1v#o{ZG!ODPwCJbLJ1b&x;#Bf~EKx&QJHR>+h+OcrLm)4;bdRE;dr!XgSuv_& zWH0^8LQd@q=3&R|9t?||c-hq6^z!QKqZUyoqRcq0InS@n-!?A|(?b&UqSYAlH!(K$ z=|IFUJQ_RSKZOY9L(#7yuBJ$f4Z}-X`D1m)wfhWgqqnp`>I0atV(4)pJ0Dk^CWpuX z=fgEx>C{yuc{)#e&CNDzEJvb+e{}w`4MQAK!)}M=WN;^}qns;EPn^pI`X%a(yO(hC%Yl{fhSldf_nkTXBG>uyrWLov?e7%IX|)1Na%&e+~*S< z{##U@RFXRNX5Y%^i#_FG?P7^Z4lBdWpF7S#hArLR((wm^Pt1tAVG^Ht)@ETFK2MK&ao)2x z#&6a3y4c4Imt=$^&O_%2UJy617zF#`3Jii;`l&TjyQBpb9hS%plJHv_aoTnT`7FXh zX!3~JNk^ey>GdftrZDHc@r5Z3gyba)^(mdlZIZRa9gR^_K)<;k+mV;`IC?*d7&z7Q z-HP)R4<{;wNe1U6nfmhG|R{wvBwLpKE=C~Yv*oR99`4A zBC35kp#KG;x(&|(;&n^1aXr8;t~@awwg8${J`0}!>;c2qQTsW9gP++<9y=Mn=k-SK zqWv@cw!T*;eLQFMh|=re>1!Em0}2efjU^BHQ&=Gw{D?ku6gOR8maws_r9*WlbdM$S zHq)->wJ?^Ig^tq92NbJlqP7Px7b?H7_u>x?Mm3UsVwu zs4umT#sQhbh*{B+Zs?$Z^XH0g0p%dG#2sZx#nMIq#QKt!ZMzRQ?gBq?@~QOs7Ies3 z^hJp%lW}SnLNM*cDlt53bSF1VQ!%BpL(QX9B*IR_I`;6l z?NysVxZ5Wc>w|7ykuu8S0ziy)mwJ4!VKKKB0Y67dN@@Ibl-?4FyCHyO>29d z_9p#|)892{{SB^_h{2{CVNyB#h&79oj_3>8q7m}!mm?%B1?cfkzI?{~W-UMKb&byV zJ&8!#^D4YhygOk}l&M?r3f^U@erpxI1|!xMv=H=ySK53)Yd%90-4mVlA@am=N^NRLNBH9$k^3KBvwD+NBUwfxByb5&UsJ9KNoy zmW`40*|0q=MdR;byv$b}G4FU!%mx`^gC49p1k)Q&ZJ5g;cU$bQEr=SwtM8g6pWk-OA>)rC7< zX{XRozTw>yJ$s(}eRxA%(CfZg)cKE3+Z@ILQY>xy4|Kc1V^%Wx&V=nq~U?|6n$kBE6-AOP~+XA${Vj=Aip+8UMw*7}7@Aq{Diztpl zvT0ohc<6d1C4NrTfRLzo%=K#HKW!}cHdYhCJNX z%NeuypPNLpVECy@7;6xtY-EU}+!*gK`s=h8uWQ6%vS~|>uaBl9M>!3;Z+73@X=C1& zqXJ1@oV2Km=9N#f@`0$gt`&m6aCadCnL z1?k3e`;A`LQJ;)Hvh{pOSZO^tE(y67av{m;w7}nMAANcsV^7-9V|2yf-G!uYwUBgZ z47#jarTLYl$=x(D4pA1Mol)qRs~a-6kG1W(JS68N*q+dXuq!=PL$o$9}=d`j-<-IaJAtPg)|m?F(v`JG%CzF7@F+kQYf_;oRhY zA2sGgcvEV)pXIfoCO?m6CGNle`?5R{e~KFkMU%W-sT>lI#U?h zlQuEvXWyLe^1WT$s})`H=wTas(Bo88^#C+Bq8gA*<>Yt>=5&)&oidN{sIT-YgxC#n zsB|MZM3-Bu@B9>;O^{vIbQ?Z#>Z~1;PL8)U0P4&4#*p**(xufeiks?`CgRgQG8+V5 zF_pHzstJszIc=<+ospZ7`^(F1_Ux00v7v|doLdAG9;$BGjp^8jz2fB+i1V?T4?zdCz2?-NJ~A=lR@QYq-UC?T zP1~J5w?9x4*jr}5A4}ZPD_zRi$^ZFOJ&v4G5pH=QO745InylbIe1v0bykt2FghLq4 zw^RW<1EUZ)-H~7BWrG!NQ)+8|c0YdO%GpKw`iG_YT3UBu+e|1MR;A@?=5xf?v1VGU zS~|$%WL|NG0{m`tEM)Zbl_jm(f_#<860N>Q4zkgHdjz){kK6mnPOrDI)h9$fdv4XT zSKRq}Q$UgVLemSUJZYseT0MSh`D^ii;+Xa!HUJSkkvcuMWe#CVKb4SIE>YKhnD7+7 z&?#_exWh{}|ITPfMd%tN_P{-W#N+#5hT{DngM~lcUT1dn%Lb9?7p_Y#SBnsb(pMBR z!QKg~k$e{FhEzb8MGu8;l=l^Sy_!)n9hH08j_{NyLkoY)cL(3n>oydbc~_q~SZ!;$ z$x0dSxQM@vLoFYNF7~HTzUCXOl6|}L)>Y$R-d{2|y%UAc==|*l8kSte+AU3?C#1Q~ zi;YP_9qSElvCg5id29G@Qf3V`RubBwhJJ?lb`iFFtUIcHI7e6p5t_%iTEsKp>w2E- z%K+=34DtE)!)5@fD^*!n;ZX_FBj^xB@K0hWqi-=p;?E$PJkVp8(WhCQruh@mDaRqyZ^J7o_7>{1oDyQcOi*g);FljChbrrM&&UrV5@c1VO4N@qI3Wp*_F%c7GlH|)DxQ2k2hF=pB2?PDlGZCi zk0b8fBVWP$#5$KQl_Le1%8nx8K5>(OYmiR}nt#3kh*Yx2@{!t+ft$CVW^r|7yUCyP z&CjT5JAY-&6w8>sGomCiMZkGz30$P+D(QQO{;CNybhylI;-w&xzS4Hm!O~XJEdhY$ z)!82AM}cPAx6Mf^P-4oG6@mR<%f#k)fitkJObU*L6d${9<>8X;ddL*N$5O<(UR~1? zUyQDcD0_9)I<>}#HFOo0=}#XMbV3_EkjejjBMl1$8je6q`>uW;WxRpEG-%w#xF9p= zW8nO_g>WY#pVT>fQPh)MQm7uZFlFlUH7VIg=EMIu!4_z(>S_i5hmLW=)ikX9g9?w! zpZI%bAIO>(6$XhHg>2Fb)ed^hPtNJ4^;+HdC+7aunmD+Zt@daMe@;0|<9?LsK{LO7 zrtq=8WR42oK`i4oI8%Z0qyDyoH`gTJ08w}Kp7Uywk4(Y;xPu)~_G}^r>InAwEVsZN z!Sh*&JzjhAYa-yP|!Q>LF|P98GY-!-RsuGR0`WrVhZLfhEV;FTy_H$wm< zju(`5ACCiMzP35B3NEd++Lzc_BTmR~E4}Iq<^9*~mj*SzAKwoWVG4V;ootShDpF#c zk+we`@<8V8zi)^=a`(E;r(4Xe&BZ^rk4HmGbrSVSpyLDY?W>F9Z+_KlFLn^YkIWA#%d#JK|Bsj+9vHspLBgZXFPk zZ$9P|a8VGTV~1HD?!KkC4B+UxqhkVAk2GoHQRN;*0C<24oArz&>dS2d52sLHPJr1M zo~!3;xSz=#-)L&4`I~tEHdj?x?4H)SoqJYW9WW{M{RRL|B8JQCwZ#Z_jp&-z4eXcS z%rZs>Z9Nut0k`azs80KQu^B+VM9zEj7VJ(*et)7$Id&CP7ZH%M{wyqZ>g2!?{vir3ekkt1^)(0Ej!IMmWHMk7%xP58bk>5B<8AKl4gYu40a~s{`RlbHJsl zkB+$UW-CD!Yiav1p2Zz{CjkipNxrCRA}Vx2j1M!9aP;=a#T;t}jEYDH@&eJ}{@9;@ zM4HGs4IufC8*~7mM+#W`+A0Bz7SSp5$;}tIR|Ft(dNjPF!qilE=y-*B+qM1JzBAJu zr@q{FDlHu@PIm}VxcSzo9>{)(f@*?oo9(S01bHBLf#34#uvin83Os1dNZoRW z6dOAWpkPT8D%v4YTfg5Ezv#ZNVpqpMo8SSH26S!~nt)O(DK#v4&pM#VbY)jSwY#mS zD8Px&8b+`sKXTJBZ~c?+`8aE8G)2Lx=U}43ei|1O5N!#Wr{Ez;;csox6zc(Gc-oJ1+&?9Zw)7$_6;!)1qQN}!G z(K}56oGP_}jZ`0IG=>Gh4?KE*XbDG}T?%WE?^9cLoB79F-|s*+wQtKHp77HYdanX5 zlD*#%ec7LnRA6r3S!EP8wUd`sNETbvVW~|A(*T9t@Y^PH<~J~E?&0C=ZNN|~2%ctD z-%zpozikbFb$CN}q|?#RecsW}*~?xPp5Xs3p6P2R^>!n{p8M6+R8|jP58ya+d+Xh} z8}V_fKZohF^vV#rxo7*CG!aYnS;GvKiq(;AeJ)&Rj0ba0;AYd0nHxO+ zwqkxI;2ktI@Db5lf?@w^qU}5`5dRLkMcvsqHLAu8aS6{dcbIG2;qZ^mPdc){2^UKS3lYD2 zc1jEN>=6Rf9jA_e6Y&|&UUjzDGVX%Fx0;Cr7q_o$KS9!cw!*W3N0ZiUPT6Zzd0gFq z9-;p)TM&C>=$7_HZyT65PtTroyEF*uTahqSs50#aO3#v&e+nGcqw-pQ&9r~|gHH$E zm^C2yrMva$ugD1ONYq5Tr_zFeo^9Yk>yX%>rnJ?Mb$Q>4=Y=&?>8Mq9G-b}jZ^7~z z)S|-seDs0`gP^eRzwBg1HkKdmR$Y~TFm>!@(3FuM<;E&XCZZU~0b-_X#p8VzEj55- zZSv-nknaE7h=tj9x^E9w{jHX6?cy&exIuy@Vhjms* z*H;VS3wasQlK-x=Lv5admG73)+G+`e6jYrIobXUSs?-a_P%)mut z)=(*Q$^*1@EZE|ITB^+$AV1D9D!!YgqNH7^?FJLhiPjTxmIhLKtCQ{;$z0<~u>4OT(dq`oy36#iO-E&6}$`*f7CftFKT`D9GX^v|9ebRJ^Ne zJZ$PVp0g{0?_Y1B_{xO_)ECL?n_-wmUOnl%uAh=epd7FYqud3bS1RrKvM|fdB(WUy zJ$PrL3zTf+7#dfAsQ@INC~J{2%AAlWL%0K_XHAgXw{i}A&w;49Dqg6DHyt!#W#ttR z`!mn)TUkZxj}D3E^9%&$%N8=IN-?^3S-qJB=-@>Jmh1`(I+2VnWZfcorBk#r@2I*> z-c-+PbFmb95L-zh$f2;<@$8=L+Kx$65&J9X5!Ql54WgdBwI)Nw6dRkfAj6}TR zT5F1qvse{j`|BY@`LLxf7b};_6DS8MRUQ6!y$2$o8w4C?*(Drc7kP#YU3rWz`Ji|$ zAxV`Abqj!3fP(OfMHIH`K==YQKa+e1oTO-AnEufzq;$!pcJ>1dUr_QxzPZ{&`7i?Q|+LJfuPdQU>Zv(6mI7BeAVQ=M-f0!FK&n?|tB<81J$Wt6Dg z2s)TORNg+~xUOuPN&UkTInj?EKZdYd=a3kLzmYepCtv3i1uEoV!~&LHru5t*OU@?= z8j>_hGTkQM@5p968>CM{HzD807DpAY$%$Qd3gS_BN-`vAseY&-I1zI0t-|f|^jH)& zb+k5|QgIp-mEqBgM+P|)(C|sGFUGls_5?)WX76_=s-hD!wlmc)`46X6hE+d$_~Xot zPcAuN2MX{Adqa59XJ`&GUCA>yf(a;+SW*8d8`T@R2TubOFge@LpyrzW6YbCEjQTPV5df4TC^*PkK7N>7|uE88mO>Yu;l`lIFa zJJyRr-b;3shI8DTg||@1s9pW)D{DSH9>fexg(k^(7B@pbQ<17G_SPy{uh0qQl1 zO!Mby!Pd07X(_A0ZugY<6LJ}uY1Y*BMS=ZqG-yARO(uiO#snEpJ`@dzuzC@}}(fBEF>om9B2!KSw`k-BLoFiJ@qAm(*t%aR!V*EO(u zc}M}q<%bN$0Dry+d0-C`Yu{JIqC^WzB)TZdRLmQHI-1X2o_ZX~gB}1ucwoglq>ZUDPpJUNIr}%td*P zk5|~8&$R3j7(iTw4=j9KzSYWB>%k!VHjcXMfZowIw>Y{#SEuC2_IQb#sfDbnTu%;O zM6DZ7wXOUR>>YETsB8E-VD)91Dm+}Y(b_$CJ<3W1o;7<5m7&(d%NlcEs^0eduHLgU zHDdivk{y%fB-RO}b35`8TOijtzN5gfty>**<){T_Tq^JFE#F~nwm^%s`HgGQ2z|C6 zwxdYd$-_6UAlbG6`NlV9VGSi;pJPj9uAu)u0fem~_V4ubZ3Aa)(oqGaw`eci1MTpk=@5?-_K+R5wQnC zbPYWlrCJJibKe5w#0!R9|Mn;_f36$TS;_5T^7M> z=YnuzJjJ21ZsWEG!2yrQ(Zx-i`upO;mz16#?$F)~+oszX&_j=i>rAnSL@RD7ty^gY zK8=#3?gqQ0ZE_AJrtCh|7FNGK`PB7u`uTI~8b1#No)uJG=l`km)nu&CI(TDBXC3SB zt~rDMz8zK{GMdw_x-nVv28pZ>bbQp^R>mg^Y7?d^_XKy-j@=NZ3bf3)Q!5{7gw$m= zid2nlwfe!{MN>-6SERPJp9#s^0MLmXDT}4s(E-^^KT43H>h0HTW0_C7$ZVu%)y7-+5Q3yc6zOLZ9m z!%eUWfPpEkuBRd#_JP0@XzX`DIwEc0J3EFKbS66`>MJ;_F@$I31bCQ0Ze+_wvVV{h zFC4P+;spn2BC97feIp@!hZru!DNm=21F0u`D+7V;ZQi~qo-y!vd!r`=6Zlq#3)ju@ z%*#gj{;)j%0?^TR0fzN>@5Zd8MpuueUvQu9sA`*3c47bp5tRynKgD|LQrCtZ(4lUg zsTEQUv@7nZnw%I%fFG7L=@2T-_S$NesW; z$W4wJa5{5%6GmO{58m>Zsm}AlwkaYtc2MW$=W)iWR}yAOkr8O!JClX}&SWyVx3=yn zz8sBM+UQiCdBe|P6y;i9Vm^5g4k=IB+j{vG;n@OE#H2ui7{&bts@xYHKMP`ZP$J+| z!WaNJ=CVZ~V(io47-v2RN^fVLq=r7?ffv!i(BBE}9U$7oXY$pz?9?VamhcKw8d)i> z)8r>m?hjA4VSou!TZD*DN-rKEXO|a3`ka{O==Op=Dn@}UMn2tq5p2Y;bSu%$`J9xt z3tQ4(30Z2qr4}?Dm#)-)1}lh=%c7sKCQj-Jaa!_B4yzc_T(G@5908B#c?G08`Ep}= zl(q5cIl7R(KAhiJyaGf`oY(~vXB(Q=xw@15jtB+fLRHQR=D_fc8jx{SD3Kb&w%s_o zVr;-K&I}bOI$3;@e=Jm2LU1q03jGz@z~?VKw#R~6`CxP#5ZdJUWY$$aGN|M5bf3VE zsfT)T%QWnU^ZeBRZZftlyIw!Gn8#vH)jonK$pVd7FIB_zix2_jHVu_@_nwbP`-2pk zBHhv#S#LEI+#SwF9DIA8;R%=h!eM20;@&@60I6Yv4+igMMxMME`sQ)K$UR-w;Y+@j zCHAp=bq#rC8i!iSv0QZ?QeKtEXcue*dT9)2L&oNdq*9f?d7sAzq)k!p=g`9 zQ0Ec~v^urQ@*ofR9oG^mTQ?3`FLi;D>S9Yfl85^9wF0cCDETnf=U6E1*+W9!J+IMD zK_{y%e^s){C>9&Qig=+hGd$C3{^DECvJ1(N(z!9{cdogJIEW$NKot;&Fg3Rw*0Ps@ z+zR(Y|11<5V(R`-ZTcP000x5ZvN*=)N9co1tuodKwtA#CwR1gH?|xQqz4f z)zf#m@8kMud_RWLhtv7<FD7WEvMX=C%>g*xmU*AG#unXVm^3m+rrqzlkL(lv=ESTr71{=H2zpLuN7NGOD zk@6YY%xhxTmiRBJ@EBsRkV>ek(>NmeV0wjVwK$~jnLcrSmI53b3UyRDD2MTe8dr_{8 zTE(qc#qWZEz%e7&YcfGcq1t`&WaT^`>6Ho~b|!djR_@-0A|)}Be^{h$0pKh0 z{p-6fE5~(@JO&2UL4Y1~wzzaIP>1_T=m+OP+d9)NjuU52vWA))1%C*7;I# znnTyffl|tR2W@9Wa_ug<{L8HzG2xMck0yv6RC#pN?WiBH?g%X& z<#$I3d}BmifzB2OZ_j5h(nr4v|ECKAoL54y3Lta))~yp$?($jMbk6+%FrkW-&xk`a zpzBi%WUUT01KdpIT+M)fscxyst9a^RtG)j;DuLHlJZzV>r4QCFX;px)+0fii0vyFs zqnBnA|EGXXZJSBsi1aF%>!BJs<&DLeg( zuB^I?<+U=0Nq}()9vvotR2W91OY?|n$=@{6z+&2zH0;_A|0eJWAX=M`28?yMd(>!p zB&Q%?+Xn4WoXt;=v$0*kT(%s*`ykD`IfO3&1L{W$4`~3Ojkpe|bS&A5&-_sZeZmkf(|MY}z%y~?A4~wf4Oc@1D?k6TZ{|ZyU9A0$GX2v6apwEE`G1iwo06t0EU?>Ig(7Oj)1rvWHT-?L=Sx~X z+^Fm|E8@8mfH%UJpSfgt?YeoJUssp>FBc9@5d5$$X6M0GA*di(F)D!L|4dn7eMkB; zc)D7D`!LL(W@s}8L>p(RxQYiw@Uyp&Is6?3cl)6N9T2xo{=T)jbCvBT_}~Zs-J#GH zWn;FL@Yj?}M@q*1usp45v1vr=$d~mqS}jl2LlfU+96bN)hIVW&q4{Y@zs?vr%Gz&B zcRzcnnz-wdT9LbEp&ls{7;O?T{?{9C5R2XYojAMKC41gdBxyB>(;?d7F(a&+aT~p! zz1VAY4Y>E;KBg1Ry-(C{GNFVWqbuj=1DmnS3XJZt4yY!pvTs%u~w$ z85Q_*na$5%*pV<`CI-V&hB9dknxsiC7to;nUJQWx%J4NHFj(5wcB{;B?F^Kvj%CM- z%BE(&vD?zm|J4Yxd%E3N)GFg{J-Ip!WXAeAJ7KxO zHUv=N$7%r488_oSTf0X@h^PaI-y1;U;~o$4Kx=`Y-rU3MXJ`EPSWD|3HojWE8eu*x zpy8$z17l}AKy+sn100!~1oW4SfkcqIKhMzpmGlW9LwvBXBVG^V5`khXfnP$|?gd@3 zVEM>mBG&8@ZXnIC(pG+p(2WhULTo!i6RCnZ7x@TutNVLOTh4{oLyPz#75-l807B6~ z=UcB@%Sh8Iee~aFu_A|*$wD=#u*xpJ|;7Y`rSZbA8>XrbHr>Gb2 zZysec25m-DP+#z8TJ}`SzIE07E-=mbxDBy0R_{hKva2P254HRE@%wR6PvL`)@b~Z& z_#&g!XODz4v#{d%m&$$ru^gD|_!} zJ?klR&UM=KV3$9)&xddOWU>!LH0tFf+nNU=Q#U7m4PX`8AGlHfv%V+1&AAEepEc~Q zL}p){=iwTaU(%$9j6>74h9UQLcg z%}^XD>HlZ(xQ0rl;jrDe>`nE4`-RUz*StADqYzzW{@#G!_pz-e$n+4Ho1vRS8nTjW z5wqPh^mt{1nNt&mo%=gidj7F9cw$5lm)b@Gt_f5qZl2t-qv6pmEhd zdK%a?RLweM0-MZVX8QA0Qvzl1ly?+rLl=jB^KW!K}R%Orc0|0M_M|qx6N+kV~Murv< z(kWsii&cRYvTvrSb&8XTb;A__#0^$p$^bhzQNg9HrxIGd$EO)H?;~(M{9maxAheX% z^5OV2GuHlLG20xx!*t{_WVjuF_`jwnEC~doN3#vp%`A$gy6Q|yp9&o+Cd#jjU49>S z?kxM?ezZ_s!Q+yqQ|wAbh6hj>5kUhIC~m)(cbEG^Pg9wf7eE1rTO#;jFhCc{zb?CY zxoLO=Ur*8%dhJw%=g-0z%_pO|VI8yj(}t7~BN#Cb0Jiflcy_xh!Ps|~wC{-l z7^h706eo~=;_JASo5_T`znb(exSu{~y$?IrmCvTu{J)_7Z#^;nYdxi#WLpw1nH6rt59;X2a|op$ zJ`^fk=%(@BoA7dawEA3?NBZuE&qY=tglm^aE>{WVm-d}fQ&>++GDeNBSxEvYM4zkX zjC!xSM3;PFf~WAF1K(*6ZIo;t_CaDZQ4|M;}>&;i%Jy> z(SKtdI1k;S?-z5GNcXc1aAP$ zqVnb9h+jp+Hy;E6=&(>$p@K^Dv6D`{OM2+5-5|&dvzj5(sTZJ1ZXx%{qG;pWu>UhJ z@F@xsbfE?sU;j0;H77SK4E%(>;ms{NL7y?Rkn@Hvzu{X3dD}xij^s_%Nc&|oJC=h9 zLH7~X&J~F&oF>wZRV!dcnarhp?D9+G=H|cXV0RUcfF)l^XnS3jJm^-R>J1QI-pfvU zTioNXCjOY!^&e&1iT$?-R~XN!iF<38jJE#?s$TReqJ0JfcVWnbP4O}<_X7OzUs>^H zzv;m4#%5NbPWz!(x=AYV9!e%2U{-y;J}un6sf~v zd=kx3D_<1%m4#D~#8~x%nYvTZ8svE7>tD`&TH5u-f=7ENU&5A-EA1r7YRCV(x3Vkz z%Pm}Z5nEJD22}rYpX=!euHU&(qG37dx>{eTZ&^}L0q`MCk3_2ufuM^1*eNCK9ddB3 zm!$e}^HfyptJ|?&cm3Eb*qJ4LssFn;+QvEssEK6}&`wp8EJ4uV_q&+5e>WEf@Z^B|a^?GxxVp+a2m6 zn&9(WrRy)PJZ9;;}r@hE_d0R+JJH=`MN;Cd$IPoJj~~}>uaJA$ z(@2ruOec;*P;W9dm!j&+zckN@|4-~|_W57c1?F)>`|PEpz40U=8xh33n~Q<-5O*_- z0irs%I#0a$2s1kBDLfmEIayIj1XGrXi1}|jkss&oR{d}Ap>O#wq-}X!K0DjdL);an z?Hmn}yU{hVRE^=$RcXc#g!Jp@MocYn+eu1MmXk*l1S2;!7Ee|rFqJ;e8@36|j z1c%o}9_3`7AJ89Yj&k@RPr!2xd9k!p5_XO7$}lC{`4PQ^6|yCEI@ic!?yAvX2wK!( zFICy87eavPjInlQ&Ckq`r|Sf1EBesSnqc0-_^x{ey2PkX6Hk4O(L8e zJy+g{5eRdcPGd!l#SXJcTmV+suYNF8(r33UMAaxY4HJR*;oy*f0TbhNv0(f?t-17e z`ys!i+DQNivmAw0jo>iy2+ysLPtSznU%+xAsw143Y{!lO>l@{rd-aZ&g0uys1 z32p8|_2p%&NWIiinRQb*NS%V$HANo1c$vnL-f9p;0FQ5e` zmB?SAl|`YvCsk8k;I{~h^Z?{+W-3{aCk%wM#qhWirft#P5K;1IhhV`97m{VSOqw3~ z2jV~&j3+^S>$|FO1uXs|{=QY7xF8LsX&4u!uC^lENwas5K1Xb>eQvpelDqY7@mqd1 zq z$zTvSB;UqiLE+OR_I!pWVjv5X<)eyP_yWk_st2xi=0$j4)UcRRAWe<(3S; zFokx`@h3ZLMEce_PRV|Jv9{a4qi#>U;T$~}#=ygZT!w)LHBC1{upIEgZw}{P%!k9G za@1bt;y@}JhLYn8{8yJJ zpY7Df!0DJg$`Ue}1CWG(@c ziwnlFZFrV|(Y=>QGO4IH6zt&(akauZ*0=9UU6rn<^ytC0$8=Sdij0qP(LBXDi)39b zI7Qo(jC+pE+3+ckf)f`83TDmhzzYup_8)vtfhdr2^T9|i28HdFu>2kvLp!w=bXYJR zz~;|BT<2VQc`#alwdbo4x#C4x#m73GBYoTNiE^drs$%6;7r0|PBg|*N6PpBb zpSSAUYab`eAZA-0D_U-6x2Fh{ZtZexz|BO()+?Nf<_Z9vgBNS737gBSLtx@dKeqx_Lfj5$ ztTw{!v^fSLhK$RUkN)oW*DOuLTq}{;@5Ag!OnUq=K-YPPy0r(mfxM@KM6{&Yk6!XR zSPaPemcoE;jwj(1ot~M(r-C}!UZ7Jrl>U3Fr#QC>l=M2@ixo&0BvNEQ4qvL6eQIY6 zpej-hCvKkDu0Jy+K7|PTM#QbK_%=jUHy8<=`5no7d6(xxwo-xGb~6y-{#lm_1)#A3Ee1q~S-Vpx>0ZOB z#_M?_Kn!kXiFbB?X~=t8$StkwBgE;fKXDLdbZ#Yx2#}X8E4Of5Mxu@7E`iy6k#gph zYsJ4urv_hyknv+_bhm)8Qq9l9@lYLBMMA*sM1^(9=PrrN&_I;Hk?wulze2I1V z>A%AGe(_`M$-mNz+RY{#V^L01&Ze=W{pr1sD$3yA)XDMQ%&AyhtAN5~vGKFiA^_KC zGyFd4ve|g)9k4)uF5+kf4<6YNq%3G8f-QOkIO3|P$EB>hxGQbN(Lmm6zm`&aA;Lw| zBE@Uchd}*0Or+cfn^UmUqZN@K1>+G@p4$)ZKL76I%j>*J1_kb6c~9auh(EVVnuRjH zZBB@GHU;*#m!ayBBll)!#n!I>^r}uy0d^N-{6_P$B7Xw?Y>L)#t{{B!F` z92e~&vuI`dtG6eIU!x`g^yFXxpZ|V`nx}XAU|PM}xGg%B9!~SClQ)svfU$Ux3%BZX zkwk1sUSDVpDaP(Pac|BOkJn%(I9KqzF|96|@-kjx;AbgCr=oq!YHBNMWoUoq=LQ+Z z3WQ~NSlrl#!_4Z?7XG>WE({|=ru5YNM;KMTth$VqwCB#<4p+O1Tqj~`x1W_{7y1M2 zm(Wunw!4F7qGYxS27K;s*aPw@PW1?;bYjqrjDEltFuAv)e_6IQ;*sHwgL-?ZENDSp z9`$rQR4}>{HNf!P9lY6`{CagU5z7-JVb7Ntft+vjvbPAvXPgzQy%`E5)J`QocQW!p zgrZfCH?#CUUu_gPSWs>SuaXomSCdW5JB+<8;`9j_#@luPw01GW^0!TNj?w>rl@ zwx6;aq*3Scvfk_?g8(}f0z|W@pL{J>X47VDL(u8s!o+zTo-26kyyJAgKhV4wQ{gWM zBQDUgI-V-Y+U<-iXQ*w|chk9IXgxln=weQ#oMXM15axm1511X892TFw>Kt!loIUO3 z@LhFVkXivMBu<{cKC(ba0Jz|@X$#7qLrtNACYh}%A}lQS%vt)F=RsCu$t((S)_&tp zNpE)s6CKtZN(>tr>C`DO++8|Q$3Ee{?(7=jPN6)xAZ>*;8LK}erd3$0j}wrz2Q*LI zz&7)V6$pjowDMK9F zkg3g_6bP%Tackb!SuJ>ZZ{nIVo^#*^1liy=xpJc-|9s0$SrJpmOo-&}8Vt4Y;J_3w zujl8Hjda`%s@;o10czJZ6V8IA2m3_9eAlCAi9#)(7Ih&qwfc*I>Vgex9}r>m&eKNQ z6>a3_n%2-+%AX%W|=>QS0Pwe7cO55^Cbmq(%t5Dr=G*-CK5xz?pzi!nR;)VjUNnu%up zM>E zfkGr=@v;4|3BVyEAT~RjE4Na|EMEq+3PlQ`;=`1H@>=GOkyQ+N5ahHyZ)8qEjJ1Va z^^Rqjs#?z7Xx>YE8}1|y)LZJ_Q&Z+>P^X3JS!c|=e{03fx#V{YRw)+j2X`t_0D$Cq z7e#&qQ?v_)1o-S<$DWES*WHU?fMS*YLb`*Q#k5s$mB99)#TDP}IGHvM%_1z&53r6Z+}EUH_-K~>RX|X$Lj4?QJLM%LuWZ)tjvj7c2FDQS9sJ58 z5%$6fs9tGIaZ%-d4g3VtaC<=APs<7xr}>86BD87m{vQ^A3;y7>=cHh0?0T7} zg}I8s`Ly5QyqM}GV-&bQprdsjr~+-jmEb9L(#ZQYmm+8F zC^+5B$YoVp#dwKU3qKbY%CBb}Ew6Ql0Y0?j*#uD1USDP~vZ#FG{pE4!R^`5Yg@+xT z&}XoZlxmtI(nD@`AmNaH251oSL`0y%!ll>tI$9yzw5jF(Q8j3 zKVyX==(_j0e{~V*Jbk9mpiYaf%|$Lx8gCB+B1ckD)6rMt7u(6Rjq!4WuUZbp0?2B~ zd=RECh<=Zijq7MjbJjU$4{Oa0t^L##t!vfOTu3fHk*1>(t%-?B90td;Z)f^rWQ zh|iJ_IQH&Alnbr1WB0Fe$+PlJmdI_Sz%2J0K$Q}X`zfrrEM?1!LEde*h05l*pBZ7qNK=_3K`XbC8~W!y&(Fgk=m*p7B;(RTY(=e@OcIJ^xKc z&8ib;4N2>Y$jwWQ-g^4bUAu~%NC>2Y~n4* z2dIThJY%h!s$hhSOJuYn`{I?gzhUyO0sWvbsuY#=`EkGA8)9Z49%d$7hTP4^9B(*P zpn1El(7)Gav8?d5UhFBO;?vF8l&z}8=Jz{hWo&+3bi*^pussh&&;m_lSu<49eBSU@ zH^T8^RFxDu#f!;(vrRtptRGOaQ{8cfXm7ve-Md0rh}BiwZk2vma0qO!Z9+uHF3Z8kx!O2}E)R(g84;xmOuD&T z4+b9n`t1`9!+5#d9WF(4bum<4j~b8ppj$uv+0B7FMay-Cm{j-N??$VjPpV!Z$m{@X52>@>_Q!tymdV8S|_*CY>R&RVmclP4vau z$K=(oZk8$9i#k4GCSx%$0o*V7CoHlaj&rTcDs>6MGSjq<$`RZ~RKG&mTA1nNWgVIL zkm(GYf?z6$%jPw^Kr!vXc_CTH*TmPb*_n_}0(lgV6j$t@1=~+tbOHvDSj-11mcR^U ze*Xsg&CkqK)W}mo@`fE^R*}m^?G(2gn}GMYaC};8U7&?>UuNov^AZ2^h{I$cko8AZ zPt{C}5U^W&&|p)_X5uRE8ko`05gd^vv4_@`b}Mznm9q{U%)w;Uq+iybfG4vic% z1%3QCU8{GcGq&n)7%Q`DYt_Q{R^G>l&&`rp9la-`tDaQ-bXA%8iQ?>Fpq!GG#?D72 z=aRGwn;2N#EN_3PY~ukPwhE?q+?J(xywhQTDE#y&t!JkB9fOdgi!MlAl@{Xqvjl&5>$ArnT!*Y*u1ga*=-l5 z|B<;PNa)k!wdorwQC!?EUzj0h5`~79F*v5ksx}FjTVP7ESk-rBes-p_lv8eN2?r>= zt!%eD;MnbDf*ifuZ5Htk50>|db|yD&LCq3YuZ(XuG|BBZ$hJU1hQmQxOd_so7j4h| zBFqdAkG_sH#v>jrH(%YExz@r!r<5?y{d_4<3TI}I-0W#XQuzU<^MU49zGYr(v;gn> zVbzaf=A>N1F6r&+miMfd;?D}MhPrhsg@JF(Dm1=ViX)CU@CnRC=*!`!6XTwDl^o1% zwT0Z4*a?KY{jJ44cMjEBz#T=5Ux<;%vC@gSL<@WhgdQzx5pG0=zN?yH_0tom0=Xhx z*9BKTr1}|x1dQlg*;ub2YXSVDYFN-};7(haUBRYCc*lebCo`8vdN`hQj!_98`a;;H zd%D)uzjA0s=UmD4KyIV;naTQe921T+bd_q1QutRtyxZg9j7rcg5NOyHI-3&02w{^~ zWq-c3J;sP|gP&;z|LVt}FqFCDX4(qm9SiLbu8=CN%}si&+}`+5M2N(Ui^z7B`ea)K zwwC|frr>_uOvoxb*M`HurMW1&%ft_KCLA@DC%uc>_+z;hT2R6*hK>-sFD|vP7uQ`; zGYWLF;C;7~Eko|bitP{0ax?w(-NrlXpC9ABbFrSA4d?gZ+ig_>C%a{QT7Ri|Q57AG z+#iGvpH~Fx7Qois%myXe)pPXU9!HHXMAzTL{F{-APhCsA{K9|wR_qu;VZ zydnnOwCo)(-jG?eOG`JV9oK3Mb)EGHZ~5vfW#y__$v=H!v4IV!Fv_8v;Dbv)1S#LS zZ+_#!1x67XytecU2&eW>n$NCFx}VIbA0<7}{S^H6`{nytPZ_ceY$yUIQ(#X>_nucO z!DDJqe$oFxei11JUF%qrw)l%Ji){!4q2H$l?lMNFt98|EWaby=Uq z1KyBge~;%a98Xk<-EUPxDey}*rZZmQacIBvmQg&3yZBXmVKfbb1jMQOfJ@~x% z(`304E{qwKNpxIl3HaL8(IyfDbia`c6SR&EL^t%F$w})sjd{p*rc_4NbF!@i68`OA9GCnigxM>(A`=7SveOLkmk`=tVi>@o9n%>GSs zdkRZNGs;)=sNHivP39~Wxn)&vr7vEXyxGHQ-Dkoi8-XP0$r_Aw&}%Ud->%FVH!nu9 z-Hlm+S^T1ZD=P4wVnCH-k7&=qJCfH&&ghK7qh)-;H57!G3g?5_?w_w|D^hMQDw3JM zD=<~t5~Z~Kmzh(Xe^VBA*1S)I^U_-7BA1+V^+RZtcnch%{5YPJ$tBkN1wghd@=n~9 zV*NXTVkxOokiuxX|CDc>309_p&*SI={$Sg8+u|BQ@`JsT&I_U1M5~_~2@R22#TpYU zMjb?yhSt69nM)=nJ2Cz8cJJA$=dx6|Ub#TM6wchpZ1_#Vopkbv_kaH(BPhjC?Uk+O z%*y8wEA$&cbIA&)=U@jO>q1GsAI+k97EB%3U#_qQ>FvJ?H4~60pIhiK8p%#^YC{Wv zlDK-#EK|yqe}BX(*od$8+~CQkF>n|}?cVB7!gu}QIK~Ze2Z{yAU77de?9#LKss19Q z5f$T+@Vo#7u6og>sl`!pvGa9PTuOUa+;=F~)=xXr9uRIXFIgkzJ!qnz7kfeM&X5!C zYDzUe6V*!iSycIiM=njqYtyXjqSt3>2lW6gkJp95k?AlVM4D)fMT`c0ZIX`tkLf9; z+D06LRS~(V8<<+(7sb$d&7fv{XmWqMH>;Zr!Ad21_USfDXW&K%E~Zv5;_(+0tmsQU z(ZSs-qgQAdu6n8}wZG)mSGQ3pPdz*<=54TWgB;ol!s?~C@$Q|BOw(*!6>Iq#%trbMqGiw2>^WzjskC2 z_VzEpL&Oww)&6uO{}_;#z=fez?~mrfcR>&zT6xo-FsA8W#iui4s1nW}&VKQdk0_>|-a~Xp;xtPsvE1ce zm+YR~7iFS_lE2rrw_emX$Ramdg-09kAaLgt>B`oPQs%|pLczhzp`3{63{}4f>-pus z0>1V?GTtff1yj$S#dd1hY&9yt6H%G(>=deN7c!BZ=h+k~WmJ+^WlY6?U?p0iH1Y0= z(Jl#=ZZC{tJ(esql8zF2c6}*kv${%x)#W3ImeoNTnF@Y+k%f?gO}=+Ci0}T?o(jW% zFTBU3ATnk-@^YPwGkh+o9*^6vsw8s|EYQptX)L;Vy{0s1qgeIs4_W#hF;jP$#QM}$ z7?ix;Z(#6qtB+9$Nf+d&IsV{IY^?OsC%0Onscz zMbo5N*zeSI?^Q0SGmG_teK&jK8Ll(T9teuYLuE(27Fta)s6Vq|geAt7n9=|0g;W)F z`J;!^HWwv*dQ`tZLa7sLinR>=9Il;fpN#+Hc;ntv#fai)AImwDH_R{mn{!jNe1((h z+*@Za+LSYj3qIr!--@|!wKc?|AbdenCuYVH_p1rjru9BZ@PhkR=6O8|M;(-4?^ilaH$kLmje3NuG(jhb~y&lJh)b%*u| zV8pYj*!=N1<`VI>I8p!R#kYd;Yi4L{uvL6I8xJdLM@tLWLpLk)cbiNjb}*3%gV$$l zq4D|qeI_R8`QcEm)<7As}_PIZTOV!=+qAt7gWKm$%n zxSHx9$_WJ>hpUnp|I#(e%tS1beYp~Gm;CaC$kHYYX0zc@gPA4g!?reDDy6^g%~{t7 zJU+wyCr3tSZpa$(R28eke>V84Pvhq;@SBj6V)t98Hb%pKDch|1tHnwoIsQ;U>wqCET>}=RsQEG+Rnn z-~~Yb_0Q__nFonxUHfqY{Z>3u=;;`SgWw6@neyLbvjrOwxcPOamiO6&n>KpYEzsY|k{ACD*QmXxL z%aUI>UFVciintLXMvOT>n`IZ~5z{?47^=1NzWKBL;|k5u%ldYmiwQrNXKdhY!7EUr$UWtPuEF=bmg;o?DnK3HxfY@2mCvf(4ZkV)QWKv~o)~aI$U4)V#my~n?n4pIk8Shv@~v++8JduGfsH!pkiS#*@{&%=KDj#_*pJbbD& z+wpMH_PVJ|H!S;$n3CsDi^HF?9)CT=%z`)E)4PusL1=mLv#b4D%WStAUR1*HuN%ET zFm#&RM`m_sJg$duD8I2rKR*1HSKVjhT+l6K5hb~|a@n5g9NACwu%9$4M?Aav3Z0(j zK%>7CN~Ne2>_oWI(;Ft{C;GNtbJdUdPTkj`$DzFMm3q<1u!UE35Ai9wlA>JL^SbBo zyNC?+h|JhsCy30XJ=1s4Rtc?M+nxPCnhL3Xf*kgDWfvD|gD#}Ge`RXwrBg4HPR@N=_{QwT^Y}&tz4fN38GYK)JcmCdvMt;&FaKb44Yd5(ceI5h1D;zy ziYw@|Y|q^)@YUaaO_`X~y1ANID=fc#saG^; z(KzdifXxhy@Z7Pf*v5uKQDpdVdP1IBH70X9l_ls;%3SL_!^TG@c3~=Zv!J}|!RHS5 z?JY|mO?_Ux)qTBR#^K_y{;%YvZ58C|t%6Bep3g6AYC213U#%{W!|GVEUolm(diGP6U54LVGm2` zoU8^dV1y>%bULfMbmW3w-t=r)|0KfFfj~MlMF!j z$$paw)&JaIW}~tX4N=cX3>mN4-g~#HTH%9FxlviDzv^WnhKjb%E1c&fQC{gGAAX+> z(Wnca8^Dw|$P05-EJzZvLomcK1mPR$NoSoU=WN>k*$YIWb9ZEim}!zjhi8=EtIzLd zF=IRxYHz4HXKa{I##lc@Jl`pjSe0arvCt764z+?&msh=%???=r?xrxKaQ+yk>E+<3 z{*tUsA}7U-{BWaJFQE<9eq1~b;##Y+JKqJqjDAyEZ+PbO!9^rb1&yI*o;lY-<@8rtLnm`W$DY1+9Py z{U^r6RBsJWc9dWkm^z#~YhsSEbm@xswdup@7q2CLB2HSLrBYN(uRAnU^z;c`?^)37 za4_B#8sWd?J8WM#D|=yUZ}oUok83XtQU=WAN}zJ2T@b0?ldf78X^{lUnxv_E3>ZbGi3oF0yatr~liRKy&Mfi6*TPXIcHwJ$rMa5YNanr`e(X(u+yx^0mD4*_8LtnB+s^I;8t!&j)V&Yl03ni?&G2+WG?|v8&fn}XQJFjbNy3^Oyic#Sfp$I(_Qc;Y*0AVy+WQ%F7bqa<1zWQK4 zgyZO}k!5m!*>CUHnb&zBWqgS*NXzgYDXx4_iH3~FTp}$|5MDjRKFjzu3-J^`{#G5- zGw6LM-KK+SIJT$-tbt9JIk49(eN-h>^&-eFp`z*~x|2(OMH;%Lp*43#D)_~rXfmUm z6et_!m!CUVxzF%^*-F(T;`UA$Hd_ULXO>l=I0*FyX3?HB#QkQn__4g%a}+OeYZ4N$k5+k2I~#4BZ#KkoTo~i*psaal97RYW)*=EX zxpLQLb4Nd?ii79lP{kREa3b5)1!$-h-LUuW)9vhq!m8KuDGg2{Z@FX)V(6yHV=1F@ z7Gm5xxe^%~n`WANQ{ETMVk~ICcKTaPmns?M#dX5BI5Dr{S;%RQyGFd8g+XQDV-F)S6%WvNM75#e+|Ir`$ZMy3NgMnU2-_wK?`QeZr zw40m0-xdzK!?t0$3Tq4?zUdxlgRkYCpljFEV4j5eo|%Vqkj)(H5-XZP%vLw0EgYmp z%H2o;kK~CgmHnhpi~`d--Q)0?9XP%(V_xDS3-qc)LmKaa+*_bs)>^21Cw`J!`ceU}sV zwqoyE)XV(A)?t_Ma~;JN`^mocO)EMOOvkDb%~#WnCjpf2kv4NbFnO-TjN6JxO{rxw z6)Qg_%aY&Uxn!;}EQh18+wET}x~*aJ*;}^-e3u78WtQh-2pfuJA178Gp-A7FO+0pZ zs>$D}MC#f`e!DL5f6#R_Ei&DuT$|lb19@bTdwM~X_}m);%T)_nb$WcUkJ0WzSXDcXP44dl6$Yp*%B7f2^6!E zSvq|D(pEH&y?ee~(RhQNDzIDm0G({FoLuBF z{;JT5(=sHlbh?uHKR0}=FIS(K7t%7VB_`5ME*qdn2E*#_RhOz6kiKTLa7E<1>tFr? zoA6eP8q|t3E22^~bj@PtzMqY^SzmkyFI+3dT*qu9K0EcI58G$OQo-|OnTX|BS=*9M zzv^N^AydSBTdG{E=seBjh+e6Imj8;nretKcACx;6YTGv4zfcD@idgAIL(ofOPG6xq;H+o^XPyaC%26&9|h3};uo(Dt+tP{ zM%6IOmm3L&7Mc5T>Ab zE$TIk`>6fS?A(BmkS6%x)lZ)mp~nk7f_@k&pXG6MiSfjWnv%&on1#|>yO%FGpyF-4 z%-VdG6s2}A^O>X)p$FS1?+GUB!s^^(y}G;hv)7~+TCD<-zwCAqt-7-NIlTK5MQsna za5_nmD|n70&Z#*;xMGc#scTmI7cbG8&D&~|L3E%&)C>Oyc4 zL2b2sf`xI+o^f&StWe!uk8T04xKtOj$RO#WYt=2LiR3kTLQiwlzDO-x=+a^ zI4C}Dp1QKJt5X_hT2pv(k^^meV7t#GwN&u2-dpoG&N3!`?I$Spyv>M@VfCW#=XFaM z|HLZ6|4$-G*VI{Q?CYVLQtEN57Eky*Y6;N`hS*w1N(ntY^Gsl>s8Pif z@dBnmL(I}G?*m~Unf<+Pug?7vt0mCgjxrfv+M(z{yQ=7rlii2eeT#k_c01S!zqAq3 za*f-(z)3r5qR7>QBC4*B=7MJ9=XCYzEWog@U(eaPE_z3KTFsm2V&xsbpO=CzkzS%IC!_@VI$eJUQXg3vC_Ax)YWU1%K2^yY&PbJ|GwQVnYl+8 zUhNVnnTWAo`O3kji${|CmibECq<)LG92WVf_~&<$?@oEb$pD>IJuv^3#yIyZtZ?%8 zHcaGry?yH9e0YOYYU8Vw79E2<`BG{nVoOb5f9lKbFVGnabH4G|H(`ghJxzlKZ&r(m zpX5+T@Ik4(Ri6U+hvu>YY1#hzmt_J0k4pnA%ND(W%ES=KvQGj3CH8(AiP$CmJ%<=voiE&ER3G$U-Im*}Ly zh9Rn6JJ5vBEtcL#3l{+03kW?5uujkI9Ta?oBDcvdOC@}=&PGdpPLqae8h0FWhxZEP zdsTHA`o-7bT?(A6Pri!%Sg5eUZ#sWHGW08b)fEvl364-THL%43n_*Y6)>i@^-=<4c z^A`7PkGZI~kZ>1+=nl{&;-FTKGmV75ZQE+0`1ZVv`+Id3+fOz%jw!WIJD)y~V`iS{rVfX4 z<0|^pXcZ}brX}#^2c<{I-IA~G=W+K*-K5Uq{p|Uj#OmHJC4FiZ_Qt9H=Tzt_Ce#G{ zzwmhe4fFTtudCfPeZq_FFT8Ozd-rvWdtb>ir$cwmqAg;5xvlo4j(_5D%5i>Y%%Z*z z!djBRR!}k<6=Zdia3)HOn#|`RFnkSlt`-tSU`vI#t@ZcNxS*svpJIGE?aHC-dkiER zhk$$QF*yg_N4WYv(BvaC;?c^%yP$s$sdqsZ0>cMXHS~<>aUL77hGxPIjtqHg-&m1L zBkIqs4&ZMS_rJn=dgJj@;ikIo^m7^Ziz_8fKks=L`rk|6>@h zN=%>e1IbBMEBn${(e=hnQ2ta*Y67H|AF2*PrLOUqo8Gqc%Y=l&TiKY%(am!7uGM-Y&zCQ~y7P+@o`~i%J{v@>UDOG)6&X z9NX4`6GGRw_G=(P#acm3c|v=Frc-hCV-xG1H^*$tsVLaJx{VnYugYf6?W0{Z6e$GT z*oq1`oH=d9&P?uM_da(0^z)zydQwI&r%b!fyqS6P%IyD*=>Z9G_(ehG@4r444?&H+ zs{2zlX=ct{usj*iIvOC2;&;;aYV&Iw#=I8jhAq=SAAFasvpv9nHX8rSJ_gtDQ9X!o zLd-tZJ{}#ydgi(TyhoG!dKA6thI;YBe+*}S@lD_WJwN7P=196`&CJvXe%6NRYrlNH zcE7LNKyHj33;trVaYrI3Wy5qX6Ub!yv&`;+ z0IP$=Xag~yl^Ktt8PM^JXGR2^T>X`mo&KMr>lXbavA~PxeA*thM`PjRE-GL8p?QI* z!=ip0X!tHmdDOsRBj~WvbMy$XXdC9dgVEkZ_oY$pyw2W#-}vb#R#|$5MhNUWn3jN? zMhH`--6zG4JoS*QeXxCMAe)#I3dYF!DAKET*7>*5jM!apz6)KGSb zmS#;eORA7<@x}e}z=~;k|7AqnEd}`%_j;{uDwEt<4r=~frZq^XbE%oxk_(%63#~>Gh z-*_5J@RD~wo(VWb%5r789B!PPWBPT=&oEBg|dzsX$hyq`KnP8B+{BQJDSYPi4@EMyX5_y3mv|2Trv=UxBd^7QLMt+aqe z#i_l-yGzFZn2eM2!kNWr)87{? z6;s!FR$;Y&uLzuo2EzggZw<*X>~>u95SCO%@Ey!G_1MN-_aFaZoGx;7G<9;+9O3Am z$!0=B+%0f-Z)F^Xn&+|)j z)lr?bfJ+)P-B_A>cy|w5RzK|)JP~Tnl#_k1mpACQ6$Y=R8wT``Mm}O!vI4U0#Q)o4 zrTCmXTNp)Gtr-tL7#(ulFyt_sWfNvDJlZJpDggoS+hN1<;77;C@%n~iF)wiBAPBOP z*v*}`9gF5(2gGiiK;O2~NN5Q0Ct-k+fR);4B^C^{;cSWXNt_)2;obayNc1q04udRM z$R*H&62h4S%256Bm0$p;;u1^%pKd~Z=}~1b!|P4P%3eh} zE&Q_~s(-$Jea^}(@#(Oz6Rz`@o-lVH&2Wo@%+t%4-eUGJL&}t?hKVm#&Sn3x4LL_x zOqQ&Q^{r7uMY-zA%80L1t_OI>4r5Xqn z45|HDz5Snaoqm3G`tMAS641R;A4CJNF>6=!F+3~(eXZLeL1E_$S9 z)vp=R+M1isbMXGB8H+NWOL-jFd68yeIocigf?)Z!Sv!Khtqv zc;_jViy(Oar!}zCiz3hEQTfpg+u^JKF&Qx3r_(D6D!37zh#m1Pul#?6y?Hp4?cYD1 zLUj|BB3awaEm@*Qwwc^26=o_$h*7r0Ap0_7P|<>kv5zs8Ws(@imSqMhLdb6HvhQZh zU>M_fxu56$J@@B(Jm25%m_KF?$Kkrp>%7kMd~dJU>pj+0|L>WS_*(r(zIh1M_w&F1 zy7Qa#GW4h|@q>nH5#D0LrM}`nuIn*lXDiSvA!oaGo4~o+l(TYM#%FSGP^{8IPNL~z z2HmI5ocefE###Gjyd3VL&zW>EOeWr%;N6(K`Zhj~I^f(x6B)8kFvASjO}CH7!7NdY zKYE*TQCS^zHxQgRa_&tLO%eZ$s3#D3xJ~cN+ItF>#6koj{RP}i!#&sH>*dfx#5dxi zl2G0lH283S?3@k9C(nzZH2ovI-F|+lLD$iWYYsM1v0&!e(d1Rr>`73`=}qou(-1=1 zEC{|K24#qu_ui}5nSdoTIc8gp(G6*VW<2k@LFO29?4P9q7;L3#+9lhT%Qp{}R8`#~ zNw7?4RB4CD{;g*aZ_VkSRS{d+cI}@uo#~mu>W(|V$!@`YY7V~&-K{2Y(FwWsif_mz z-iX)ak{h7m&rl>@x635$f|LSsLe(u&c^mlrw{M)PwezIE^{w7re3tXrT>N1D?$U7+ z3yt0ibq)(u<=touuWYxFl|jx%2A`jaOO$zRU2ngHY|2wKqAnV>Ew_vWs6w~D;q%(p zaRsC$pYK)eZ{(c*J(o^`e?LrqOURSqnJFcF<+|g$60&9!WB=~8P+MkJDl^l+^6<$h z@I(fQvWQcT3J1|Fg72+N;czPIYpBK#OFrESy7lV{3-_g>+pmVLtMQ0K}OLjJMKV$NM_u(ZFHr+5l+Vnl` z0mL&Ciunl}oi(Y#BlTC*lJ^aydFy+I0@)sQKQ z7ToSA$%!Rucj4D2W)z_g@cv&URTmd$3QhxAwd0jCX}-*cXw5#zy`g*bqKlhnt!BPQ z6;D)6RQ+>iX*>h=2SWP3(1$$0ILoE6GC(qE+;Nbco5wRLE;eN0uaNaJ;$~Sb(A$+; z3;2WywMqz1W9<^RWl7o%H74aAI)6%*@v>G3vHrETkhnEp>qXogsNM0Il(7h2LTTm1 zD_YRdN?HUE!~{fa0vtZ8!FjQL5CE(U)n&aG)e9*8AWvbH2G5m(^;!eF3&B?He%PRa zvD5z4TfYL{`z|(-&;mam_c8WjmiI;g27(~<+;vs6=#8NbR@D$}ISn1}*qJGX=@23XnLVEql5CI9+b zKtd>ZokQH_MDczYLOfTh$~nf32+{(HI^4bWJ*xlTQ?`BY@8!K~^4pzT>@GkIet7|C zX1+{VLQQ$}_Ye6BRp+_Z?*xJn)CwMu-tVv4>L1;il+m3z8r|9EBG|yrbg-5j=Z9el z69{#JB7ivDDq<;ktwKswwx|N|cBF{!7Tvd3-Al`NyaBLfP3F79+fqZz;bQ&E<}KZr z^$1MwIB&CRd$aBh!bX1hm3!#eXFrGjfJkhreFCq}om)5AgxTuB=%IkVyu+@)R6Ff!Bk}}?zhQw^IVKCM3h#=pcXhzo=K`E44++KlhdyRECq2PHVoQI-8y}npp2;eM# zL+th_+5~)LsfA~9+z~T`o_F@#P7L;H$PQ^!^hV073Dbo6&x7vw{iayD9|upyt}uhu5_T;v@SfwZEfB_Q?5vSs^S{3V4Qn-Q8whwb5mDs}y=h z;;#0@(fhGyMmFihExMS_v_75>?4e-J+wpO`qr+}pU~AVb1ZOcxPo4b&p*#jw4)E~E z(y6n!t(~A$=m9}~hxj_Y+YX5u7?broypCC~m4l#myif*l#DZ$S*wMG_gjsdMBPt&~ zGYlf2`oHz6{T+O|US4}1CW=|;2}X8o`eD}mB3gr23qce9KM3Tx)NuEN$c~!ce;=^h z8hZSHYbLvE9t$=yU28;e{jDGmApdw%U%LOcjCV_tmvYU}NZG;g7C9t(J>9-IBc!9oXx-sPCzTCc}&>-%4E4!HZ*;yNsK$M@Z& zDvWw>x&S`pC~(I$@qLbOc2&I7u;k10^`EH8Q5iagnKdPrQ#*rVZVZ-0HoMd`_A$P0V$B0>PUw zOUP}gO;@%L8XnAgen6nn=8%5J>Wtx*(#UGCj$~LMuWpLq4k;{a+n)JWnm6(6pY!iU zsGR!Fd-K*FXmB7Od=S6O>MbZ_Yv);%U$S6q5R}uP>~VK8C3IoVX01G`6?^tT_inA4 z_ap1>yh(R(-_GMjEh%W0H7M*E*6q5NIIlG#knNwEg*)T#t+!qy3K=y85egT(n&B3G zoVO9W_Cj&Q<>w$*?ZOy{@@n=Ua<&dQrtkSs?Cb3APQgg7;A9m z;^qY5$)?{?P;;0ksev1=8BE65OVB4=B094#LFxJSZ^T?fS$483Nwq*S362b1L2$n# zq*HB$Dub?G328<&#K+Pv&M09RW3C(zj>kX7@#l_lB<(fhOR`+Gk6t74fwiSNlx(>> z89uSHnN(fx?!5N-FXhevv)tesa=FKEF@A9Rnr2_$d{OdN--+cex9xIf8#dT`Q{oUG zI7Qo@r5$NHpKaH-^~s$21lJS3o9IWUK05isX%8_7WcAjhH0od=Ux{{u>b)T)5OdBF z@;`aJ^%FZTs%l=LP&dCB^)HWmHiyplI_3|Itc+=qrGAc>B>RO6FlK++wjee+p&v}G zx1APLdHiWMR<>i1>NKAT#yUtO&_xW5Q|*Uom;{4+ANY6GR{6avq0A8S3+gIF&z=`kOd!E-SJGj z$wH@03C)&-?S$=rf`yhNf@UFTMhF#1B~qy=j66T8ixAI-`&+_&jW@i}7*c~hG`w0; ziVS5ulV!cbVg3l9H`(~U*k!(J%6+1eCNF>FM_HxZxp%CG;v1>DW9(WT?YkjIK@sxXhSCEP|bx3OCsc{qQJ(fDdxy~m9-u_g7! zFTcUA3gdpmkPnQZ^S8Zs%RW%U>WN2g|E^ds_<{mH{~Zi?8Ts^t@Z9TGutvxb~6_XNH`mbra zkm%MIbt&_}|80v%h??o&54|rl?CiQWbu(w`boc&TY%O;j&Ko~EZjv^hflIrAQ~hFU zHh5=3ql(;5Edm?Mv;pYcNf#qeO1~mo+3o3@W)XcaR=cwiUtmc~(Q3k{2+UADdA@h@ zyT(J}uBzyAK@r8i`5kgHUQS3j z$J~-IpQ>GaipsdBs*|CC;B=W_8**d{Be^Wuj(f>P;M1g{z|E5HMEG(SDTwMn1|O;< za;RzRz_Z%9{l~qccA`4j%Ru9qA}iG0qM7fAMhQI8xcn{MRc@p%$5qbTtOQ=@z+9T^$@6!@H#(W>nuAQt zz_D5juhZrijR3?+H2ug@fjpYoRBvSCFrVz;x(JbFw<9d+MNfg)sq#N!r~Q9>5uuj1 z%xpi7rCaagUtg_UT&qdmR8e<&r{^#~=U}9rAy7;=qgP^mjfc)#`jFkU`waxQ=xR+_ zSo)#X&I*#B9JvU59VclLQ2Drx6m0>SnIJ_!R-&#Kl>B?yE9`{u9ebYWoAX)C#0z{C zKHYU=NO-wXqWZh3wvSBug3oF9_MB3<3*6@{QJefExbaU_y)3%~xiNar)8TmQ1`h-+ zFi7rhbZH5k+J&b$5vG;&sOL*iWvH@$ww8k0t&qX&N#TRz);y_oK$Vj>3Y;Z#M~6Vl z)o*{#+0?FidR`0tV`zO$`MVT?9mf&1!`zrE>|R9oen(`KaRsc9y-jDDhWj~I#p9!8 zoMw00Rxy6t5|Sm_T9jSRgL8lbn&pULD-0KKdF=U-1;?mX6B`#MeybRM5UZcilX~utCN*_JO12SW31`% z;q%F+38=N%#W@f(VCrTU-t)HsUVV(-M9|f!n$q#DnsSHy3`cW-;9~fI7w^VC7-Bh0 zV&Njp<{vCTD(Zx}3|BPZhk;0IK#Lbxchl(3T&T|HLEwBTh z7KiuxF8fGw`1LYD5cw%M5QivN#bP1o-*e4ebqfxm_>3aduO{0tP_aHNkv6yq) z$*rhK{>1cu{Qka#|sLr0p77t<0jwqrJ&d&lC|f z@WRaahpSjoY9(d9C3-za&RXUehC3!##df)G(rvT&S(TW2W6I^rRFF5)9=nY++#cBr z_4fzfTp7ART@t#Y0WDMxBcm5wG>wnB#Gf(H?oWuaPj3awjx88{)TA+@;2VO&4_FPx zEv$z{ci78y&ulD;+AXq%GT=HLxw;mOwf*02BI^??Eg%yLr^>prZ4_yzUwnfW*4nM!Hx_I03U^Knx8#$wm^l~-^e*il#RYuy@XMM@cOM-k7- zG1Nu<9-mAKw?^b$Bzph49Id_k`D|9t-p@cP088jCQeCoX%$qd{3~<;;w4QvZue`bO zJCCVoFeKQWaz#+?G{V9&2Z#M&G=r~v%vl3m`5s^Si8&fA`xaFX8d&muEO<7jrAW2H z(Hbu`ynm-S;r{{q6Wt$6`l-1W0^})>TRC|?7XfTvk)89{O#mDaG=W~yCnjx6&vLn~ z+uT=qcAn|kN!XVq_Qq9<{+P(AR48{Sy>76#%rRUjYt!Ou$ z-qeLx2*M3j1cwK)%F1n1DeK(t8~w3H!^hH9b4b3~LhIzvqL1wb&xt8CYQ`HHFt9hw zLU3f4!zfmksPFt2I~nPar#Orz4Ryu;alCWSpKIf13WG{aEOs_Xfr+31pQ#N8Rpeg9 z47g{bSN_T6gpoT>iJ(4?OCNY-@4-r5vh!)bT|V^`UiE!;C+qRbodVkGdNV|23VKxf zGjltsbItC-D_NuvWs7)oVs2Yj+WMH%Rjg_ny()fmX=VPQbgnjVkM;(fDmMOoadn4| zPa?d#5}UCl+)Cp!G0aQni+1gk z?!u?y6-IN%(>16aOQUD?+ShNA9t1jV&0aN)_-QfqfYWzt z&_3P_va|*^Nn5|Eq;!=Cd1YfsO->k|Bj^`W=z;)%ry<6-t)hoo1$WObXRgTRs%Arg z8V*!(rQqC`LXk@iV%344_TAci5 z=b*~Wb>0SfJ5XqR2uLBh%_W$WcskG}x69gD7whW`*E(B8cXy+F9LCKN)_f#(}lIj}2(6Q{*8wG7)`~O|Y86i>T3#I+Zof zAIi{UhGAQcKzIvxv&*^2xknD{>Yk)cN2*p%)dHd9b<(1Rd*9381{>>=>+x=jA54wJ|4C@v8UwOibxw&%2Qt^U z7lAsFt;CDE=*C{>How5O4*;mt0TQuLP+EVs_>Hj6{SwIm-woId$3RVtctP#xxc?QL z>NMl&u>}d~$7EAl*w(rYGd9r`_o{?f3+2{=-}Yt<*u5_WQacf{`DlJq>s-fhffbWl?UyZnu{G@Kj(je^l_t&|CSt3 zqJrzaX9(Efu(%W8<6P0;oE&$WKHcPTuq?!vtBgzOS*)j6DFqX#wg1Eqw+?q4LH98 zXrS6Fb_&@6n)fdEUjvefS0W0Y%FDe2eo>`2NCl}*9)){Fq8EHN z8c4GNT|X76!Cg*u3fg6y*%F}M*iGK3&7>8W5gC$FquAjzpiu1&F%wCwbENdzKhQ=N zCzH23-w$6J6$25Qv%4IiM4WRdsdb(C#|y3dH)UnoZFF8Hq;iX@{5%~ZXZ_&&nyq)= zN7-MG5=#2zXm_ZtIZ=acxzSL z-K3RliTV7d2bNZuoQLh>91*khF)v9(Un6XzQav7N6Mq3luvtBRRJiCwlxPA=c_3N! z*Gx7#?7A1fFm0#c-1s%#_iJTY6Ut8H0>*n^95w&Lf&BD=98|+g!^T7z=}zKw_syg* z&g#AYkRqBTVGt7k-F61y1(y)#}QS3<~uo@ zBK6l9wQ(am157}d{xIVq-%03`y;8OZp?F=%%D1SAz9%iewkNe#q;y(eJs^MV#uN6uf$>{>&%&3_1q3>l!A$>*2bp47## zW03b9c53R(XLlv#e$^lZ#0k^&LOtx9SHH%;8=@Z8yMNFZn`eM*q+ZVIT%0sGM)9Z( zLE*onzGL#AoQsjMwqivTDBS-o`lfl{Wxxj%p1eWm&J^lb zcU(JFZb)7JSk+lP`mJaG39ZK)e4=O%;jx*^Jx(5OUo~Ot!ylT1(T7#3%;^rf$CD>K zJI~ZsJg-YEPh)gQGU9s?0IyJfIiIw2_lb~e5^-9LyOaGM;0nrlr#LZR9k@URrC36WJy_CQ9fM;sTM(G*=Nd{?ilAn*NQJwHOv757m1 zkCQX!q3rUwr-mvV>+aL&0=fFeB>=~PbP?y|&40=(dnvneH1%-~Blm%ofQlMlE!TPA zx8`E(?QNm(pGMQgrZ%>JU?%q;3)ff)C+`Ot$ju0SIk@N7!)Fp%I41)~n-e)@@xnZN z5+@QeCxc0;rC^1bu6J0uK$FV^$JT<*_49~*pQH9}k(Q=?4jKriS{m6(to|%(oSX}( zw@){b>g{tUeAb`( z04PNB+&KPR0mTgDKSI@3aKojArp>p{i`xqfv69rfl-}Bt3}dQq?TT3t--9fXO2fTu z=y~q0-s+}*EJXZn?86v+yikB|hQTK!_m66BvhlO_%lWuud_e(!yO-tVFQcUR6wa4}5Wg&{#;MvR|o7DXwb> z7QGV2DdAn^8K(`ZVn-QYz~x!zC|zD3?&|4kz(u7cF5QI&^gSQOUMoJjIH=_0WT(9S z&OAbtEuixY`Ele|QflBbNtM1h!6?41;*T{v`^&D1HiAgBZM&uh;S+#XU-8WJ$B`@j zE-9bvb2J`#W?^?lZ9a`UuCtY4Q1*gM3fbrv+xsTTwxVPfUc$6P|7FvA)VSSl;C=tG z(W4FR0sL8`M$0Q4->!u4{{?1+Med@4Gf=S5imcHb@9tv3~IH8`vxo+Wm zov}aG_KL@R+JVNOIr)!Zw-IW!L{ywG*WPkv`T3};i=89nZp48)1w4zYWdvQhO&PJs zr?^LG-5jKg_)xCiB<|9&XqdkBA^3$IZ|&wTS^pPMFUg7#BRqdcBT1wtIY5Z|#>As* zBXW51t#SD>{rMM7^X2n0T*YcwbYR|Z{<#GH*DE;) zI_Hjya=`qU5uCrXr)niGUU>cDwfKrw_!JHyDfcmv(B0Ok64ySaco1XRw7l}-XHC!Dk`^X~5<8j|D8in)1l!V)q+|Rr|9HRS zC)IMSZEEFfov#%sOo;!Wn}uGa64E}|eK=X`a-c6#uBEu_yBfWb9e=*fMU4B^Y2!}W z{s>W9jetzfjhl{3-9oWSb{JB${Tz;)OAL}ZBU$%Bp1eV~?0fFE=g4!X^Afm{nF9EMNH%eg5uq(KRrGQS+*q{7Cog9d#)$GhFJeq+%jq0 zg?6zd@`ifv+*i;`ufHJ3Ts=|L$R7Bl#3e_BgNV?2Ye&+2OWkcu6F~8IJz*;-ZZ4J;T=`Mq%^QHXR+HK5TcEbd+8n za-gF0X@;ua_&#qd^LYB@k9H;=WF%O$?gj5|PoSX<{y^Zv4#3%@v~uaAM=g1)SDM6P z^^k(`jXyLdN58&8aodHg>z;T;mZV_29*Rj%>;+32qH=UMUW_ksDo=ie(*xdb%$yCr zRi({!)|(!(>=%jktW_v-X&G_N-t$Fx8bYe7$NoE*P>Z;XRr9^dHFrc+q014y!5w+KC`LM8Vy=6@7(7y8Mn->cre2*`l^PANPR^ za7&RsSXW7dkdQ{{RZhDlK!#w z+LUC0x>&z2s4rh30_*thmXwmFYElpP+O=UxrN$PYdx*2WckB1(;u?d69zD1!Wx*S2>Ivlev&-KUT$ zyY+z>_~EN;O%5YZ*llEALapm+O?k&cwz&>#3KcOZrmp>2Z1#`5^+J-*tk6E48W?2Y zW6#{}zBIWalPW)W9m`2Qt^-aCkQn|v#(R)h!Z@}4BnKq!Kl&~8uHFRdiP(#T#YWY* z-+pwD>7KHZ2Osl1r6gRMb&cn5G=5f9c>2Z8?V!-KQ>g1!3ZmREVuu#>#1&0y^G)eY!1%&qwih&y~j9>I>jBFX=_tR zieaN*07eJwFybSlqnw{4*%^{%tYZ?A1f@M;rd5qJi&tDIpR#wa#`WKjwm}w8o0Gve z`HDsmT9@~bP4n01)j|ha;&a;=D^EQI$m^4@put#P?+nAWQo|$-LizDCn7DrM7LFVR zYqHI_VPKmzIJ9sQ|C6hYyl{&i3+;%i-<`c+Vmpg>HHL|UJprz?V}3(~fS1SE(L?wO zdm&E&A3d0X<_VnxuOsH?i)`}?4BM8SaaYJ!Nxu8@O(u&qEyjEQWDRKshTfAB@;P@u zZ}yCNa!!!Y#lcO1A4|virKo1NspDmdnsnx&3P+2HJ6P`Ya$voUh|UAn^w{p?`DcAcEU7fKf~U~SnA`959!SliNg zMnBG~$mZ3Y;+s8^o{`FUScLA(?VkP6pPi-gH=4cd!LZ(@?b&aM!L%++vpcv$r`azAJ#p_|n8{zewqSco zOE%F;nxHK-ZTwQOM)=#gyC~Pcvg!q^AK4e;s$!eWkonUr5gDE0X8Y-Td*5UR*SLH= zQyZPE8tCx6az0V>vhb16Bir|r*jQW?HxvG_Hv8N#AooUi$CL?hRV%|KH?x}X8Rujb z_Ty-4Hy;}I2c79+Q{r{fN=zhK4V}D5{v#dyl`CSQ5Uy29SUT$EwTX*VY9G z6~7q1oFA9Ky}xWh>~)PfajTQu7qc*a)@ZL4|Chc>XpP7o|8og_wx)krd0Y?2okH7l zYYIYa@&}ytWWhB#+W8Hd8_~)t__O*QC`*U;M{J#3R-H_p)z?jk$3SfvU1N5>U#SPA ztfntQ7;SbjHG2@K-)*@CSxEmPtA5-2Rek$bZmieVnxn#0E-k!@RhN*O$$tul>4uI7 zX;b2r{r@=9DMTvncR9Bqj0nCNvB7M>8G0?LOB}Zn_ZYkT?)OkvY6!74awW_hX*Bs_ zd2jk8T>n!C%HLfVhtH)#PXF24{}bBxxS(a#X3tpAs`uH&&O6X!Jt~)sA`V2+E@eG~ za?+4>ABvl}Z>l)IW-@ut@pT!rPyy}U7!|l**Na1-*6w$G6L0_8^+C99L@0OoLb6?I zNw-<>+8_I5wIfg1)rYsF{(yqj)UFzqG^~Vz>HG3)QzG)n4}sp#G_{riq0aTMXOh6= zk1kuCvM$zN)XDkJmTI3}h%7Lb8M8s)g{IZy3iYjif?U1jpZ5&(;+p!frAn6IWmB^L zvLU=J_cknQPK@0<8VkpGR=7Vn11+b8?upQ=3o`+`vg691TJ)11aJp~SWNFo+_84)q zQ%#l~RzUkS>+wCgE?cLLpwlL2yzt5^FK7-zlHO8=%a5tdXV>Ol6%|+4g>2Owh}0SB zjM%dIhI+JhQ53)Fn*Z6byIthl6FkpeE_K<82BwW%lO9{vulP*Jt?E1OPRV%Off1|6 zEg4S;<-4f8d$<33i{J4_ww@_`d&Z^vM7#M*cAq9JeYLx+SukO2%eb#n4Zo$6{EKnm zm_uQuwtd4Mg2=@~Qdg$++V_Y^sD0_RS9icx8`LXlRtsc_4{9I!#VEzWi5KwaqGyx& z8ON`Phq#VgE!ZMFb5x5%Jx8X8mFL#!&=<0$nO7PXAGr$78F6pl6RasvHKxiT44GClt|_NaH# zF3ua-B;WRg+&f4;^Qx4)Y$apLGnl`-zO4{uzsGSS)Z|wC?o`+JOWFZO6pWeX+}h&s;C~M_@Mwj*TZOV08Ozh$#qu zwK}b;4kx}2_eoLibY18H%iwC#xuA?ikjcBYpf}USncCowYT3Ce!FVi=L9g!pIqiph zL@r2B35_qQ7AMwz{C--dJ?;K6jr?qK8{N)0WxI1hJ%V-oa^Unde^8@Jt6`tfwH0x_ zX%!SF>q3gBegOJw&qfD&z1C$6*%m&|FwA# ze~0S}Xdv?_-j?<>D=)-mS9tw^6XAlY(zEk(FP>gATp*TDL@C*om)#&Tya1t@__6cd z8PrdInVV}5;1<+LU#lv$vM;yw9u+n>e-i9zlA>D{L=Td6;j8Fvl;f%_jB9Gmzf~m! z(A<^Hx4*ajg9QjNT6rIwo!9a7z8YeA&#<=Ii8ttOOMjw3e&%P(tgz&h*Y*ejO-N5E<;F-bnAUSHs?B1DmrRh5o@8n z`PY^P`e zy~Z{=FwUGUOT1i)qG^eF=c?irtSo$s)TC?$OzQjY z$&jx3zL-hN4s3|3Vsft%U(rH@27kLQmn7k0Qk{(`862-w_cs;V;7iN5Ww;7Q--z^e z)(Dzw$M^eZ=xo2Q%?d?6T4R0PO;Emr)JP(vFjmJ!nu~>521qeIx z#(E5vL`%1_K%S65y7vs#vFb&qPzg3H*91Qknf3Ns1-nMay1LW6Stqac?8>`Zbi#vc zwVf_2pgk3zH$`mU5*>3Uf6D0LcVnGHduL`bh&e~BCs8u zMsRhU8$gXUNi?Y3GYh(jQ*QQSB*N0xGiF|e6C)R!#x%CedOr=sbJ5!8GkyyA!HOGF zN-D4lXUx^6t3#y8s?&wMQ4pT;9`e@|{22C{P09CRwErucPr5q4RyRJpUux>aU*Onh z`X!KzEVVct&~@vew5_cogw5vXH(i7MHHP^;Hu4|PiGxl`V4ZPvh=vfnel?dLf*pN ziWJEkUHysKZ%S0*j_jN%NXiAd)9LDK9ckld44(iEZFTn|SL1#w5$D_Oo0-g+y18$U zFo%6VW3uw>jf&g$mkjzGi_%1k-`+E9`Z`+&f=di0Aai9h{kzV1^+gPwj;DG5>2>B* z;98fLTvTM<>~AAq^~|=LIFd1qXP$<_72pJs1WKHf+(spwXXB)91R5^NWGZ?!rc}-( zw4l^K1o6AoE{xYXJoS6;bKLG>iu!aiSYfWWcj66KzcjGWZS`CFx#MMRAY|WRy}`y< zvMU)L5PWmB^VHJakM%{HVhW&WaEvwBA}xOcN>+-A`RV$!H7#3%HlMBBmf09R3a7jF z|1@{3)K)f;<;BYlFTLZlNPxNxgIIaTVSTN{K1ltnxoUsh zs6qeAJ#jSihaVyZ-Ed8n80n0o%qG;CZ_?+B=qH|=?dGG*HnL@H zKh6EB5^XKhG}l< zsk;kJtXCZhgS_Fd3d!E?Rulz2G%lRH2`m?qbb0KdYmQs~Mhr(pQF4~SgGu=o#Y!<& z!zsD$Ws;orw$7E`Vvs32Ma)GKp0q6M(qYq@{AHMEJ9VY5l}l?Ytc_-Xv+sf(CMsu8 zC~F2R{)AI2agmxgo7s={B&90qNKV}(sioqdkpCdCA0*yP6i8-#t~8AgsoWwlzw6hpA47#nz&eW%7n~JjuN=TCO!CQ8fyy z{CF@21SPXfk{R%T$Hh7uJctF%wK8W$&vWT3^JB2B$qSF z{hGn`rdt8n1c=XrVntrNy*bIdC6|+V1n$55>W!2)NBdryY!oa7`BWpjXW+_u*gGkaE! z5Si_sK&s$;ld7|%5q zkxm^C$(%JalyzUrY&c6`emX^E_@v?g?)g7r+?iRXo+yS8*Up8D?s_qQFY6Qz> zHW;Sm1kSusK{9VTZsd5WB(LX~YCGJ+%o=xhs#Z-6u5{7R)WsYG!bFFnJmmk$w7m3` zlq1^LnAf=KtlLI^2JQ#IoKhHPbQ`*|_&peqGgHigSkLsYoIaI?lF$pDU=Avg{NOn@ zv=nZ={aslM+bLzC?UU(o>*KlJ3GpEJk`+{C;W;QyV}+~B`9TD?s{61J6NRpmPIB-K z*V;}o(Ug+1;klPi?%S238Z*g0zb5rK9imKAcSv-CUPHo%X6EcS91S3Rd|n)zgIKkt zXX8@60pQt^^XBHbgJ1bvp{uDXrtXRaV!!_~@UPES>kKbLoe&YDA3#B>=-a0F%5nbH|U zTni0okJWRYi3YP3N(1Q~Tjz*-9NP=ii^_`~+6zylvEmnJ0CPxXc9}u~R#EYzs)&@9 zMaJpUqLNfHugIxHi_>5-8CP6ds!bo~CHk%}qv)b#HpSb7W`CUl<0X?T%Y%j*S8k&+ zaTuntOptDK5GrbotM5x$Mb=i43CUlGBA^_rdMM+MNHa)WdvB0TX1{&pzL3oWB z#f2AmcXTx{d7BLyD-1(C_)4v*)C8#YrxL#Bvz&}_pic#s2PZ7d z`Cvh`#}7TK!W|5`gGODs!E21Gs!_qTA_~9x-#H_>fcoh!y)1Hh7hcztjM8mTib^}Ds1JW1e||36IR!{lV^-^@+s=09xFo$uoP0O1B;`S%s5{D)G^IpNfkJCusoEtTMf z)C6T0(0|IIcr@=YT!4So>Hfcs)FsisdH!(XaN_X4$C_vI57*k~|Hid0cAhG7wb0@G z=Y?H%IJEztBYjZu&BZbC3l|XO_}}LAegk|?jo^(hHvJ-1&eoeBM>EaIcP+RYP-p*& ztk1yW#W@j&R{-d32Y4d>#+pMT)|wIDy*3paQxy*%qPj8xur%DaTov&eJZI*VS-ciV6>rGG_0m}a;gYikwjNz`e@)}%O&FU}t4Gr# zAlp_wS6N$n)~1dVZH44bMTAtZcrE}Fe{ZtEvh(JhLkuqBVtCweNYf|K!W|Vdj003l z>xeQM%2?33D|pZNr3r5Y(F|@WDXAd`r>y9&DQ#_UqBxH@kN)35jp!cxlC@`Ea^%Rb zeXkpb0a*||T|nPJr;?rPu2;GKT@}J@SKVqC|2a4!sCh|uo6`!_T`OzVs~izfy`(#P z8>>CA9IjdU*Sj05000~ArVHS@@FF|xSrGt_21OsXAL6w5L3F3ITD1mPS(lFqe$J`K61wf%&q2Bm z(*aDYU;pEE&GZ7XQlRDW!Hd`$T8H^F^pYi=Gq@#n`jZGyt; z@5i@Ky*pyn|AbHkvjU}m02-ESj5$;a9+A%Ncw(4UoNc-2mtis~*Ed|5uoAfOa82uS zN4iH#z;^zM#atmI(I)`f8US$o3pH0>+jEPdlP`bDMcK;#SATo?W0xF-rQO2@mp&Tu zWf>>8qk*<^gJgjIUZNB!ql24Yz7u}hAhj25Zyrln+(BMP9aOO_`bf_Y+OHLUVQd9p zvCs}>@0CO4$NcGg?u3VgwfAk8MBB02mWz;suC5;qKJH*%^J2XBN$O1YUj6}e-3`C? zOTO^noedM+`e%fa!Qpl`Enr@=vpe(_P1bJjSBCDgiY^cV(&&5 z#ZLOhBbO@abT-h}^UH*tNR&w`K-%7mw5B8Ld|OhrnisQ4X-|9yPz4aK;jW-b20(sy z8yf>CogV4SEjmj@0pA4)0ogt3><7uu{?AF%i~RA<&M#T!hbnr;C@SM-{@a7+uOB@< zt24}joq|sHnw)Gc}hlLTJd9&tKPkiTk$nN~ov729oRK2~XT4dG_4Ihi%>&XS$FND@~X={aDG`c!M=)IiazZ;{6xnt6FzW1yS`=H*u&uJ)^`cSe2Lqj_NDIH=G|N2 zw*8uJk-zj?UJ29;RR*lct7z9}boLCya2_2H3=945IkEd%%~0s|zH{JGEHgH&JaD>zz}cI z=#S1=>N|gJQsC>7NC~o+2T)EHFV%fLblmMU*tJ(JNz<>rAi~yw*K8o-mw)D6(bKfl zGF$1+vKd?2m#3ruPh;O1)>PWHtAhhFBF#}yNISM94cLw1)n&kFZXsscd1RL zWM^yFNB^VJxrxXam7o{}{M6ibAF}n1oet33l}lSQ;mShG`MKJf+GkdRFsNu+ZFEPV z_=8Fc?m+C8Gi%ehf?^}w*=b5`21h4LT<=RznN38EB}Qi|Zgig3#;*;BC|&DY>`>`~ zfwM^0Fdvse`zO5HQ%}L!;!&aKp4FrMu4eACHi(6nY0loVykqwm z8D2Rz_VB)W+qgLx^uR#RaPZQ7Eq_LO5xm6^aAzDK3?$vGe$jTB5 zj(C)4xiMMJ%gi!*m78_gh-Wb0YNfNDaz2jj_Pj1b#S0vcxQm_S1JhVWXx8IQF*JE$ zboa)a&O82(2Cjlfl4M!9S(26tH)6c^M$7Oh&`I=^u9co%>3+47Yai8ORCeCEs^w;$ zB{e08uYd7jsvtZF$D;ne5w4#Vzuk$)cFKar!Xico)uZ+&Jyu_=Q8zv?E~IT$<4QWH zn{o74PRQl{wreS!&Kl|`^{&kG|@3-Bh32H z_)e%$MsP0ft|FU8yuG+7V}$C6k{}^x4#Ozo2R=m@OzZ>yu+K1#8gYW0EWb177)%X{ zmtA}$2tRbv?!(@b3EoiB!6lNw{Pk$NMHqCiNUMkhNorv5!7*Vm_YTG>e8zZ_pL#Z(t7h1x;ngOF@6mF2;soE@YoSK% zA-n2dlm1M+tzLeDSNUtSzgDV1BzY_V{DkU=$kQNo-V0%*=kbd}?xa@)b|n4sPBvj9 zD&+hNx(s2$V=^sb!TqrNhsGP-uOtj9hFM!Xu`T}l5sw-HXpAGV*B=Q-*4QpDdhAEA zG{A0-Id{*4*2`bA?}`U3H?L(tU~+LQNm-O5(V?G1ew+NDVcEQR1Tn6$@r#h>$Vk|1 z2?GsbT&K5A@}!7kpHkI@Z_P%ci(75(c7_JJ_iR@j>UR!@J6K8Hn2J0&#iRCT@x!ha zqKI}{U{fbg*Um&EpZGp^b4T)(PUhr_E^fWz2c@tRT+a)ZdX!}n@jdL4VcA9nFt!ohM{@Od^-Jk5l>J@xmk``m#(FyEmIQDw5fL0rzZLn zL|hW$(v2cxR8dhq{`GGH8^(3|+Ci7k=xn@aoG`K*(Y;^S5k zODTF=1i!}l0uY3#0tBV}S0D1;zeBSq+w?Gh^Y*CSHJ$@YJOzK+ikLXb-5_2#|49Av znC10c_41)kU$9qj`~@Vmhj5WNiu3k!4C(`lXn4ZnNxtd^BN0zpsP;7B7BJ0T%?tbX z6@y@%_QD2&3MHRi)^i>U8zfunuSrWMEUM^M5TCjKMltAWVymsgrn}Y*u6KCD zr!+o@y&1-aC3Juu{2Rf*jpVe1vrTV)@GmI3tV{cF4PSme<>F-wo${v>w!Ap|vi9Yr zt58>4c)1wjN~W9tN;A4Vxut0137%q&37a%^a?vyz7!+|AKc1jm)>4|=vK=Rj&3AHl zf_FM?LM$PnFeiG5Or*^k!zdbJess57s$f~bA9b{=ZZzCJ%>XX|dJTOxBxUsECqaiy<9 zXX+RvqJ4^P6_-=idH)73 zVVsX64!x>YGLn9oQQBeqAHDq_^{sG>`+S+zhx@$EI%Ak%!o|BoKP;b{vfy2bn>l5l zJ1h)+&-{afTwPDPOYMPh{{EieUe3o{e+PcA{a1>{q4}~;lXDz!@L!B@PD!awbZ;|% zGg-pt$FDax9RuDJ{=0%oaAaQiYT9}S@&{mH*LrR*VlRSW_gw;U5WS%gmty_Go6m2h z&F609XZK}CZm-j?4Ak>y+FrBD)oExbL;{G6E+=73NyIJwu>|0opn3%I{iEZjxHV5Z z2Vx}P<4-N;!h`~6dJ5?A5%uUHt%061;1qca@A8jgpvYyVLo4H?M%)*D-7byEtUk-e z#g+%Li_1rDI%}OrQL1iOBv-TUs|lQugl5LtJql&6m)|}ZNpd?q)9RKr?jJXvA35 zsf)1*T^r0<{FNdbY|Xfo4!bg;HA^=#qPzRme(1A=sj11?(#8>FjFsc*z=oywC#Zli zqb;zR$sr?i^0XPUyqN0y3g;c_=$6`kIoZY+x($qQWG+n%XzfUbB)A zp~O!#b2;6x@=0&>uzohrR=Qu`U_?rkI$3+U$u5g)Wle~5Mong`xSF3{5jD5dd5GjNWtT@EH6kWu0>n`T6Ax}IXb51UW`U!gUM70m z_)JJ}x71}@NB;}e*FQe@eU~gpeRF>w{zPegMX!jW^RV9C#45MwdVt}0Qa@8zM8PQG zdLByy*eKp2z~w<=nNd9O^9qSNcQ@98LhcUsO;api|ChPW%R0O6>PnMf{7Qw-_EwzzI! ztK%iF?Zb^wgyqrY=wp9hZcznMSKA6o-|?5x?$S5xPPXIVTdf>2vahCWQ_OgU4Bj(QwW~8kg@9Hr)KQP>*_TsY$XZ z*+XE|C`Ezf1a(V@%-+AkQS{QFmx1S;vdfR|)Np6Sc3k+c znNri#wA`gTy7$4oJ9;KI;!;5mW9$C7v051RzQz3+$nZ>$$-QE+%G|tB=Z9#WD-;oS zBHJt+zH+Iwc==fgye(;~aKGoW1J0?zze6wk%qO$lh&X5g6pGfAN5NInl3Cs^^py)? z`yH%<)@-(>^IkMeTrRD$V(DpDlvXm3HV#3o_B!xH`G{YB0!jkcGUle%(KnD3*=jj) zG*IY2vQlY6scFz&Q+qC<0fOQ$6#kX>?!60{M~uWMK!~Ls2%a<}d-=V$HLXAaOb|Xx z>&cpVR8^a2Ti#J=sAFGqT|umY;V@7MK(|5h_` zTztl52?mai*Li=zrNXUk?)EAu_C^QDDfJa~mTBS4LN=IBy3O?*+d|Av3T(5uFvp&o&vp}tjR0Xj~)cBT0h z&NkPr7G{)leXR&(7AlT(V{OwOt9Cy5ujb-=du}JjICsGKv#cIlDp~fOP!Mm94z{x* zIRpzMBrUBhGuZ_KmD40MZBWd2Wy{f^R(rVST*YdHos0{Ef+MlkpD@|DWCxh{AYujU z2rsni^-)|$qzQ|i9xCzi(v?~M@{f-HrVQ44l$tWd<%$KZ+q~t9Q|W?1!LL7FIa(ZV zkfzB(i9t>86~)7euY=zAF;x`;UWncCP6^O){H~)p=6vok?ht-3?;pI36kC&1;Kz=( zZ;C8>O_C+fVGI0WM&8z&wEes2taujClf)+e67%?^{=vU@;M{`NZx2QP&9>+6V8|>0M%V^vjAm8w<&d|ItOlJOHm~a z@9Tw!@r=462Rs4}@JdyNQ^#q`K*dFTgL|ghj*!DAt0cCF#8;+ z3@RQE=*p@j-!|4ZdlQe^WT~>B=||T!gs&nvMhK;-;$xs*t2Dm;F9!$by#0&Ra;VWV z73*W00tsf!MXa~G;QC*8lV_Wm&vaY!1*Sr-X!ZlT_KD;ueWbdysLurGp9JBbj`Ubz?>I$KhWOSjIqgflni8yg2j zL2t;aP-5I68ex;+!*;9>@m~dKzq82=fVOKvuEcjlpZHtWe*0d- z2xzg;rHOb_T08M%qQ$prk65EmX#?)C2H9iKGj*^`N!ySCFG$+m^evw`U$ zIHZKkXB96p%p-JTWxDf$*si57R_5e@$|3&UlT&;53~lQVOp6#$)7_ z-hd1=0_T`=NpSC}J%^8K1v7?y0gX=Q1HWK$tj_UNiD_V9W!aGvd}m2or)$TP4PTc6 z2b{s|ckz3%L9ZV5^{SL^HA_8tqgAA0|KV+*c4B`(h{yO!x0TaYCtXc6JpXyAVqSHHXd6^BLLR=A+d0>JG9+S>+*9y4~FB_fQn*jfX2+utxr<%iU*T9qaBqn1O2{r3=P`+nh`B)shS{{XB4SH3} z9NltVe18Hc?+KjqB^hJ%bER{8)s#Q@Z;MGl2Iib??>gm1@89iZw8i0~D<`9q9jbld zYI|CLt0hNI9@tq3NjoyA6O}QKocKo$?6foig&rftaZzktp!g}Z+e(+Z?b??ON0j9v zm)^aKb`6xhNA*_tlsIB|$P$ zzef7Q+*7kMAD8Qs#o&8cldj!7?!ytA^!QtW^%t_)u^rJmKb~UO&(_Vm`~vP}1(fS| zRKC`z(_0GbxJz}*Ri<}q0<6_M7Ahs^5P_`m?k!|->GOiO8NT{up)6I%V4{EIc#IL)*hNbFvwl_<*3d~&*iYom-AJ~Ij z!mxM8bOVrEizFUAWlH)zlaKZO@d>fR4~#2g`96N#ItwA^G*1+(5(I-~yIwsww@fq~ z)5A#*t_32FxOe#{SX*km9LxsyK zuPR@ezIlbIvE19KQ$r(Zy{{RXt6?5`ln4gaq5ds*mfof@D#XsxOOMv0XI%YW$(kRxxRzdqlQ~%9Yf8^M-ce5dV z|JO&ZF{aQFMrj)F#eeVpun#r?E^0=SYq>aF2m@L*7^e)orkemU@zI zy&$ObpmaB1pEWGWE3$1YaVf>Ax?qbo+&p1L?6SJhLW#N~UH(2$vE+8dR@EhMpPZ}x zL|lZEcE!Zgh!^Tc-uV(d_qT7mLnUvaK=eWjdLXvcbMh%L=1*BzUrSHm z&HeoL%UAG)AR8i%B2bi>PhkdXGTgx9C1LMb_bcF6)Gk}X;iaE+UaLql-6CYLMf00I z8^)eUA2DXwpc<^Bd)}qD{|utf2YnU6Usbfe5xP)Bp!~53m+5`?))OB@TYIBEYrGkq z5HS_q%Pc~ua(j}rJ6^Fm#aotasm2V+1_W9#K$KF)>Xlg{N*-uo`go z@*=8{{a&@37wd;wdXZ%accSmYiGlj*WWknj^Vmd+D@FJ9LbX6P(7>&K-bs1;iOPSqa z3&F6_NV3|pOFKDNb5as7Q=+}>GE}}=IQ!D_@D@tTik!R5*aDo2L%BoSUL@6Kv|aB= z7k?Nu!$o|`OSi`XR9RtFW(A>Faa>%=jfJ&Ph~>YVDO=P;%l#4atN-ZUp`q;0E6uGh8Z8JU_W6h2KD}U zewAqDwN7uHsoJuaR)8zbR}D~JuVe&jlh7%e(6^&H!k|b>Of&MAYs;BYTN~;&GaCm| zLZRaMIbq%@j~+gh@PuQfquBtO75y8k^G>OT zsDcP?>Utim@fc4eZC_;Z@loF5B0J&Po@T-H-6CGOC`1d_+HO$Q^noqZ7PU=FNg+a^ zr&dd-ac3%jv}GkTmie_3aQ0zWA;d)`%Q9=yUzUwf_V&JQoo%ugV+()wNo3@DR^2t` zyg(%6%aNdlX)fHh_Gd_QSDmt*%dNF|;^^q-`L%8K!2@IWf;_4AB644q!+TW*kYyYs zO&_kU+-w1DfB44yRQtJ9hp#d?e2>MF^wv~f!d3fFfFJ6MSoqxR`}VJc)K?anPe9vM z{`A4h{Pkj&M#5Jq$vXDts+!Bp#N9R_{cQFi4(<^9wmcQWsL6xen)=Z6KxfQKYmA(! zR{B*%u{;7OfmRKk<3Kosn3^w))k;cOgpCp!06t=Cx0iYAtL%2Iwjp192{Vv{mMC#) zl@ma;hSIV9b*esnXE{t)?Ne&4r_RH-7hK*MX~`<5TMO=ame&=T4#K&>u6<5$UU5CLN7(%b600D9)a}AW72a zqYSR&r=G`NMb2y~>^9MDdxyELF$V!$VTn9tv}gg%&3q4n;{jBRMdOqJu-TLtx_XCx zw$9B<@@nw$Jj=r_?oVap%cC00<8^QD2Cwt~(p{`P`-hHwa9J1TV6J^x)7v^)thV~T zZ47KwnGDK3pd-^E(f@FRO$rt<_)GOgBNA7;6+weMnLpPju3fVWGiixIColC4juqt|e zBS1Fu2x<7r`FnJS(LV91+Cq@$q*3I?QwdF|SbqpbjLUu~?%9j@fq{%XiWrYmueJOG zwo-zBQtya3<#>5{d_|f+?0`>Cd{N1#^o@Bcg4TYkYF>R*Xhrga3yD+fNf0 z`1+!2h<*KaqMt_7C#umsBrvLAmCcj^d7?Iw(>F95`yK~r&9r6%6B9rss`6~3qbh6x zBB0~j$J8eRkgt?M}vva6nm>j z(GrB_(J5M2tzq3Y%)UV9{tNXtyd*YtEaPIG!{u1OqGz>s2`o#KvmuktCJ>)O6G*(JNjc>@$qzCZUHLqzI3*o=u${=#^$(dM;KV(6X}j z#hSMHtT6PXK&)G4c@>at$*oabN|_$2F4d{(W#;+rFx7JWS8PkO=_rk2s`$W>{EdQ2 zzdme15ZwDD1*`Xo=cRHXS;t%7DVv2V*stSE5zX&A)n8tuKa*$R1u8@bz>7SG;%wO; zdfNxCcnzvkq`&G?7Jn&-bxCYy#(TK1XyYo_4|^rrNdgk27#2!q)HSg>uK@=VcO>!F#?wb|G<`LoRT>O%`d4Senqzd zr+U+x4byio^@}fvkLZ1siWQTm$6nTjjzPsdnWoUo{UYI|6wAY6^ZfBeJ-EK({WSwm zX-B;pL6kzZzOuS);+e=%9C51J-c~rOA}t6iP-L6Uwms6XURCx8?GJAGC6~1WWVfz3(CT#7 zdPRE?c7r-J>vrN{`Obo#Qew3Lx_Zoe_7HsBh|tNRO{;YuQ=2rgH}LqY)RU!ngMP@VoL)_i?Ac;4JZ; zyy{HTdY{3NBpTnjhU~gxzdp6^Ca1g>x7N`&2FB@|t?5*{Y0kQcKoC8zGm z*Y4puzzxuktGudFY3EkE%s$!#NuDZD*0NS{lUK&qL}#mY)=lm^`@@^hayy4jztkPf zMw}l`dY~1~XAl6-y!^aG@e{7lOaoN&vxVg!zdIqOM`Fq}?!+@pSVFv#MzY`gdcyAy z;qe+61!c&l6e764n=K##vI=ozEpX9IA6 zn{@zuD-54{7ESTV>{6%2HDj>l=ZgcQkNm`jC_PhZf^N)Ms(9XFueJbC2;VS~_3nmR3Ugl0>u~_Xh9{+N&zH$8w;O*-# zYI8DDm2@|z@;LsyI%_NKpmT3B(m_CLHn`Vjl9A*Ml3`)#foR|K5mpC`*Y9ewYPK7H zxvV=Uf6^yMvE%#30`9YcGPjo7q9#=teD68@+fc#8RP^{0@WVB1zw;2Hvf6KS1W^|{ zW-QmwxMwx+ZvV8T>j}}eG}w1@*<6%Q3y9i^|Ghuk=iwqPL3d;?Yy4dbkAaCW#^brA z&iXimy*fHS0l(EO)3-f#n(SH*Y1aFRtmx7fBvC+@F%>G>aRtry)(gd#=wZT~zwakHeGA z`Ae-PFQ{zY_6JTJTqILd*SFOd0OeF1eCSCI;xzh67C?1H(_JASg?$58ULuk_-Oi8b zkNik;h-Ql3lwQF1dvn1~ zH*ng4_A*Q0%J=;N=<23);;yddYr7Ji-l+!hRtjgg)}$7pMVBW!XQc+S1`RQewH5eT z4E{;|CIip$499ntY;VQN&mQ{e7T!E5>dNCFOh;F|u;+b-8OP6@7)0dbXR1m*Ax-E0 z2X93D^_zq-^p$JkD@?vIBj`qH_@GB*P9N~>aPJG;$n~L(jo)<^Ue>?dZMaI ztmK|qpgL0m7+3$bsKBTIN4jB%+*{$#5!uZ)?vCwj@a#a3`c#QE`LV_~3jLqZW6@yI zDcx74{AE@TUruEev}slS7!i_e0Y zp9Hc!>Oi*aq_MG-6M;W6SwKvMymFJZfowyvH}_bAsKlRgt(0#0=2vkR@XF%(7bYo3 z{4t?%pbUYFKW}9XP(lZvJPuHw4VgNqFg3t4;M-Sh|IK_yJfmT!60g0l^y*@eb$7OW zInfRY+RS$;3e~o+eG&xVB91rpQoOQ*`flGdYa+chxYn<|6ZG?xcX2T-(1IhqMYh$p zPnNxx;X^oVf6hE$lme#X#qh7|a%qmA0H=uG0woY|w$jlOkh%n&jo0|;SDAGdnP4o7 ziVqInT%89E&?b|9&wVTrkPYeme$_c&^S^(GIon?i*&hs~(`I}+Y@v|ITN|+6p_1#XlK3?cdRi9dslZTAbynhMIf5PPZ zX&+v5j%k3?Lq8X_A5p`_HhS_VBnbkth_I_*pzWE2TFHVeaEuG-6@-%EUF@Pce z^I6e>^WAfu^i@`~{qex<;mV7-BRF>{tHWxOacn)nNb-kU()2JI_&>q9;qo5^iXdOy zQGo0;@;?*E6FNrKyJk@#|NWH7|W8_wd*%;69c_J|6x5j zd@1lr=g_Z@b=}x4&c9f@NC94wEOdWF;oZ$j6N~ESo(y3{PEi=Fm+XmmMc4-6_{`Rv{EOQVp4Y zk#dArO&&3D9XR#YKcD*aH%;Y&Fp~fm=a4V=sUr>Hcn80RpmQp4C+A*rHc;~G3AMvy-}&Z zkKSzAA0ILr*rgqXPY3M(0--6+-G;6n`Gw4nTZ{lQVN`QH+f(-=i9H4rXw`qo#Xu}Myj$mmwSv~bzEeOo*)kNRo@}>%F$M-!7f9CIl7VCpHE-w zExbZ)j^e?~nWf(1Bvr_C1AYZBAJRN)$j?tlTqw!$1gRAmb5L2IeJOJNhXZww*aXxs z){?3F;e1U4xYWWFxJ*Vk%yer3XajFk+ zi;kjW1O;5sF>^q0l+aD+v_s&lEP)?D2}kMi#_!$>Hn6#D=2Z|F@x;q<`H|pWgdDyX z!QNofwU`pY!yl=ffN3%(3@zewTesdo1_AKB4DGwYA*gm=nk+b4|9S#Q^O%=JE;Jhw zq*uhH3l^=W-nCqF<*ls+q05~DlAK=|PH04xSa>ijfBpbboEr$w zzJ!RTEceZ59eM6?yNWqWf$Z)%-j%ML2cgBOa8K5JTn{)5tCtMLU=Q5X~ zEVI1(XFa9GmuUt){@55?Mo>wE96~w3fhi&tz-(rECmbpyYm;ix>2L!q* z{)xk88#4W0plc7e`m5dR5;z%c*scN7iW5Sq*KLoMWc$PPyM&TEGu5hPWI5gZhJ=I= zrP${DdB3bSWX;d-u(ab_khuWiF~RH+L6z7FV+4|AK*?_O$5q|evc<6<)=$R@LhL zy43fx!;x%R3mmkV=xkE4XP_?XwTGxvqZ8h=m8oY-_c^Rf|KI?2_Rul@`F2COxx~OQ zbr?@3diCABl9OCB#q9YF+7ZK4tc1#ep2#`V8LoPd{mU0bew5~d9AhoPV;BfRmMGP% zJ}vExgW0cs3ukEX2XalQoM;FH3h4I->*{z#aDV$(v%T+DC>X4m7`pP&C{wYkI~8er zH|PH03MJ_!>7^gX6lcM=z&C4(2t7gL=+I0*La36IWF<+ZBs5hO7Ytn0&!-Z!6f$j} zE^GNvq}vB>;>4&d`VgMee}aCvA4qiDLD*r!9?8l=WdCJoIJ&yhi1G2iQ|Uv1$kn#((DU@Q`6f%n8z4 z*$GC#O_fw;hY0k$C^HY?3o*F&@V2==k~0XLkT?1-^NceuweyY~J$>96$VeGV0%Syl zjLn<@ZPp{e@m_ziBB0LKr;P9k%k7LH9psqY8M_~FhyVTEIZ&pm zt!{LfBY@PneqnP0OC)4M$CHfe{D2;&)S2EQ$SqEcKy1+Z*EF@e+cQO_2(z@)ba{&> zXQjr>b3BF}Y2N26Gqsyc`cz#~zpuKIwDM;l$fm&*k*}fOc_{(N3cGpkufb4ftoIcl zr*5;63S?mozy~W{3i*?-mI5y6a~R3+xpD+pEE2{^tU&+{j)~hHnO^5D!;IHqt zxe|uO-W~pG<ELphL34W(weE zdP<$}xam+yT%GA7jc)kKcPzkC8zsYCfcVY>Q1|8C?=^XTfD}r~%0M28TywGqqC%QH zr-_f+@0M!59FXCJM+9;D4i1IF&~Xb>I?gAf<~iNl(NJk1gKQ21gpu%{-)`ms-QpOg z?LmOE$GqbmIr`-6Pta+Nv}?~$p0pq=_t^Q2{Cf-fBmlUAz(PQ-hV4+i4uC!(d7bUo zq=t=+*OKMT>NrA1a~p7uz?om$(?Exe&Vw>k?7uG`2Sd@gxpJBxh?-StSX(qmF5eEx zD-ub$4Nz4%I6&oAH@H3s>|g6OQ8DTTtR|c9NBo=B1dPXF$sys6dYR4*8<&^hwyPNM zqfQy=`Re=}fj|HK<8!DKnW#Hb`u8>s>;s1Vy2-4owj^KGFuZR6Q=b%L^SYi zKTlKPH;ykoIz996=>Gpc`L{3qVL<<*h;ja(f*EMh|8Jz4AF{>xU1c?6^ZO{&^^_KN bb~Dz3PsCMj0oAB>?J~JwX@EKJdiVbTpTgIm literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner7.png b/docs/images/LogMiner/LogMiner7.png new file mode 100644 index 0000000000000000000000000000000000000000..6be444e437ce2e5a6cc910ae07b6876ab5276ecc GIT binary patch literal 74021 zcmX_{d0dj)*2n9VSy@^+mIIBFS(;fmLsn|#KqpUWX<}*)$9c#J(9+CuqM}kmrP8F1 zra8}=0}wf&Q7TTTs7MYBA_9WGc+b7>2mZkaw$HQn+H0@%{jL4modo3nb;^2pZ1n|JLw zxp&LozV9BN*nD#9^Na(rdkcv|H_5P0^@QPme-fNb7*9s&v711pUbiI}Eo{2HI z@rvt=g05#zA0x}N+l24x&q9)pWx9q5VC!@7nqOlisC5n`gTE4eW_>k&on^}@1dgo> zA?u8HprHUF{J_zM%xrF7VzdujU;R{cR}eirQkw>LTU~qUiCY_+X@&%LfrJr{ppgoF z6K!)qfI|!44c5pc1sOaX$O!)ZuoPgoo_%m^eqLCZ=egh+M&xOtb889Vb^whr5}EAT z2w!(v|1tMHg_uJ8{_xcnU0J8EoB^}`YMR-WB~k+d7;#BrXkrlO7a&=952fs09`PraX3-_95szL03EZG4aXAvdsbh{*GT|M(m2*GL)rX%+)) z2AOJ#maf9MZ~%&G@7)#wb*;7(;nI0CoiB%LC}?xzX*y))VkM%NoXNvijO)mCo%q(D zoVsDKoAR!&!p3D&C2@)Y6;=Bejk-h7jCC<~y+UTX(q5T01tr!6R5Qnx7Vi%Ubl-a4 z?Hw#vXtU2+j9#ICyJ_mMv~Jh@g4FT}IE-s0B;>`DWb>KHTUNx&70SVqgWyWfYI?S4 zqOhbX*DJ#^jFbWe;p7((Lvsg*NwCw-rO|g3BZD+^YSTc&*s?q{_6J&CAIY>;fW~}x ztX>;&$BL(bfVkC~_*E~@TujvF4w~{}z-bFXz<5;4>Zn5e8Y4a~y4&9^TtIiVa-UKfj~O2EMjPIwi_s7vdLg#U zQ!D%vbY}}&t|cUT>Ifhv`d5)#6@Ci5Qb(S^#r?*~D@1?Y;xLuB#=`{%#+fgMqJ}4< z7W`qW9keEiA^z~#c-diZ{ooNOe25}Hc~_$qw!RF5xY7>{40=!mTHI&J8gYw^xW((9 zUbc66vhi%$57DJh~XUrwBem*UJ1+ zwX#sT*7YeY36Q>$9*m$*%ow!@uLM0_?@b0r2nOOM1MNo5ob-0}G4^A{xX2Xh&vu%Z zWn~EkVe2G%0a09k|9Q99hTm>DO34M25Yoh6RCLFL3r*Bt&0O0s-2{dvp6PEE{VHt^ zEEs56KWR1@L?$jJc zPo|^Vj<*a7##WkZnA)q@bs=`R7Q%T7%}KNG;|j^~aTO90G>*&0>&J1lYxr>QE#^s_+9J6 z8Oc3;I8_A`z|5yCSCt|IDJx%+Edo*p7}8lj)eQFXTBQTl1q6CKevBNgeNrABRn6hz z?@Q)j>ud$-+huRjSNy>JPM5K;)wiDDG4AV{fVF3q!6ze_s$`L{g3$@e9^%|QIsgj^ z+_|wHi>2-#@`}6}Q=M@6?g!(u?kJJgx`2o^=Z0Tc9%aQvnU!5^S!rT5DQ~NAzUJB_ zwf)%C@XO(Vz%NySU4cClJ;%Oz351`Z$lv3Y8EGDUJKRv(!cS~rI!%zC?hs$iUb?t< zd1Lm0^qguAWuo$TbxqR(LD2%%y3)MTgwe$1%@T^(CEo~iQD=*|^Yh0GSS~zw8P$pz znbJ;{1+yPwatUE}PUSO(;H0sQ&*;!c#3J^MOQySjwX|z4e!;6QIz-h|9A2i5ni<{MRkFNZMgr9_zMNTo1wDc#pE~r+>)otPGx;#f? z;V}qQ9527-@PbqPvJ(`<7n7oZe~WXHcINS7D*riM$FvWQeShSQ!RVw)&ETw-E!1QI zvD-8-zsN%a8v5zWV>8Nh8As}&5VsS2G{i-5z*#wbr)wo)(+kdJ>W=p|Z05BGX2`~M z39ChU5*7P1zdB~w7?QW)N0=OAV?UmAR@w#0#prl$4*Inww1fL$dGFBP; z-D|}bq;i$2MxAzSO1W%5g6~WERC!+E|z`XpF@1n*r%3bt8qJbB^{fM z=%ibfEAMQb-^To>8zr3ux#3T4NosYoDb}b5xnVjF^4O1n0qmN%xti+kTnj5#R39@h zvMN{2^v3d7!3rR^Heu|SCu1dkeWm6wDeHg_f2H?Iep}ZLugGso6(-fxn_jv%gF+J3 z15ht?4{R*35Z&)e=N$nJ%ZUMq8AB}9!CT1Ms?6JtJ%vl~_|R)>eP~zIoC__sU1!g- z@>L-t*WQyWe^N3dmPoyfANR?IGSjc2*(qbn&x|<>+*uzWmYSXWIKOkStB7I?*d`6K zQGvJujsudwd5%I%u{UDqQQ#XzlXq%}l?GZ<@Olif|b|1I$E4{fW_$J9T*?|CfP@VMbCF8D|G64v?} ztA>IDT>~a-U2$sE(NogwTev zz&ShQ)XaU=zOg@~)0hQ{;MiDPs7f*A_2K7@E614ETx)XHO(qTm%nZkk@S=Wv z{B}f672s^Pp6Jk<6QN@#-9u1;_zaKBBTCnK_P4@FkuN^6+8-F}{!F*rT)?(onxz^y z9q}D&%yXA#tUhq?!q43vsPEUl*i*DafY|Za&0P6D&!kSpEGd)o!h~{9BN#qB45CfO z^DD4T(|3U}8@o--ALQt?R>@fn$#fFM)9`Lnkg8>N3Xgin5UzNwaaSEORTWL_#L7HkB^FJa>+o^OE+?unoAiSvd>Ikfs3-cTDXwl9xE znJfqXvvaphVLJ6*;_~Ia_J?j5R6+MeTRlOa1u3a1M{sU#FVKCmXkks;$=^*C7uQ~E zWM4T%lCKlXOei6C<~GH1Oy6}-jx4g@%8fp~!7_A^C?|*LS>+p;Hrb;l^Zz*7oPr^w zXV6IcFeS#$!Z}xlY32ciJg*;6ATr{3kP_Cn&oV+rfoARy8B5HdOvc2{s8iU;(+B4s z-AJ9N<$Ban$=h^@?{w#PQ z{P<9U)6*o*QH6d)iO}`mTuij`=B? zQUaAm<8}3_dfR}Wi8r1N)~;5U**tJaf3vHR19c7jqTkfJpm|m%hhlUVm9Kn3>ySe> zK?ZmFH;2blNDl<2M+iVcrtDvQaPE7O=MXbEZo=`~^U$`Zsx;na^3UwWw>E{%UIo2> z9@PoT`4Y3j8YBhC*nv)X=WLC&2_E`k?R56X9H;W|4ZMeSlP;8dguQlLex=4$I(6)PdYHZD#i6*itK~rxAdvZ=&Clv%=gJPzT+yD7?Dcs|CVg`*~FknQ$Z)uJ`aaQ*L z=UMpG#kgzN_%C!1Ww8bZ%tVF$;&ko8qbPFt;B)ssd0qe4VE5TP=oR}J4ZkawTyVam%;t(s+ zQBg17O)&}%d=K)k3qveZoxTd%9(g?6 z9XHh5l&TVMb>vN|0^-kIUs|2OT;*;BH{!78)gXYL%N-=WtIsHq!44uY z=KblZ#5&I6F|5G#(BZ zTo0QaidaB?g0;SGVk%NU8D<|8HFu-5hFMIah-wo)y)@Nr-N@MrWChTP;c2CzAK=_1 zOnoH~70O>a3vXsJD$`j3s>6=jkoJcZ2q2`3wuQ%_ZNBlSvhU;7UwT<7XFjo*Zy*7;*(DmqfopDa~-y%Rz#`v1O zyV{_}PuBitKF;tkEknUbv5zpn70J zCVufn-MsLNM~z3#zjr40dW(M6q)Y-@9aW z{4a=cjMM7K%zeBc-tXUPQT3jD4wVTaxWo3Hrp|Rtca^yqbttO>gif1dnwLrrpD~+` z>uv1ezhUSG{zW%LxKJ+}SFpCPPkgw5Rndi6|)Qb&dN8Ny=A6+q>ZhkLU zz4OA-i$&qP?M)`BZt<<**iPjl<)Z)Qxjd7Qna!0~dhC^gf(mqZ^k}pzV|77b2EF}z znR)QodnTD*FIzGgb81Ykr1oDtF@6=Z#pyQD!6E7nD+`Qk~o z#+u2!(c`+%f1mF^P~-bUxtuNjf9SEd7Qc^C?*3oJ_n&L{XIqMVU9IT<8|U9_M@m9& zg^w5SQoT6cfN;h3{+mJmcd%Dm7)G);`)a|r{{PGW`*qGa*`DuT{^2jVRFw)!1pkj! zHm<8)lrMfCiY)*C*#562iZzV=<#S$55N`_e`0b>LRfB0{JcPtqxQm!r_Qx5ju^vU! zvE_=gkofJ$?^qO{R|9AoTg@(ox3CwcrCg2Ki}lQ1Aejlv z=4A#?$AZ!ykWY+1kk^8Kz0S;&48hg(v-Fl+;FCak@Ezh1$3d%BWLaJ!M43-aUU3qn zDr>37d!8+v4R0Lv@9TS?32>Bc7IC_xt!yK??Z?8?f`6L;76KOUGZYaq`^XwLhzVZ% z1U67I|FCLBAetqyVk#<AoE<9QR`iO_QgPRSpRViaH)~o(_X$zEw4>}~*Q8-8 za^e2DD30t3Vio^4w53>r6@9`EfnLl=lPo0S%^GVsno$@!r1ryv=@_*zi}((xW%IhK z;}8}0W3NU@hQdnbFa?8|LUArN434MS(iK#O4>k@SRU9FDa)DcVbhU)#s2XtWz{^f; zS5s|CN)QW~A3s?Lf!}VPCt7N27%v~IAa<{w$aAmvZUJ3Z&3j7eiP{4NLFu&4xv1cy za>Sc+KYwBr#tp=7@oNj53ampd2?iIPo8}8X*%Ye}yrkCPy0Urpx62#`;@pU=enWLKE#oeK;}V877vy*-kH3GX7H)%3Vfr9O}aTj7mtDwfTp{0Q(U) z?IzAvT#uJ_pvYjMP(L;`wmjljUZ{f6LgA6@Xf9BsHJQ4S)oihj|6x>Ex6QsK6>fF@ z>>0eihV!m`qT!)tq=rvCl(tf_d7WO*RXC|l?E#!!5V?m6@zshy2K>qT4d2EHMLlH(RPMLE;wJ>ANqL z0)mB`0D-g&BTNN(MVyQR-dR=^b4FP{PnVPK7d)a*l|9O?-F7(p>g*K1mbVQE+p#n1 zqyg!R`pD9ATIaM+$k4I)X1%z1q=~m~%U7v8sQ?g$Jw6qlUBn1)mRbh@ELmR4cYYj< z;0&~!Je?~HeRB%I`GVFk9`It3tO0%SALoaXAqC{<$2y)PKn(H=CMy@1AkC< z7!k7osz;Pk#d?v18vT8ou3G!-r`RRK`#{KD)09xq90Q7AZb407LpU8BSv?;WclM?v zgc`Ii~0RMt}M%R$x{Icc5ugB(GG)3C4e#P4@!HX{1UN&iUmM+(JZ}U--NxECTizRI83WL_=d< z_-S7y1s)vTZaCq6y~k@*!915!AP3gCGqJ*5rhkS_qC2fa$e1qSBfO{HW9sm#A@R*f z$9@}6Dp4C@zPtkB(MbwyHNCM}_#Jl0M4}_>Uf_~XY3W*CVSJZ^*zhsuSOw!D%Dbm` z4k)M9o}@2J(F~UGzLFa+>&2c%#C~nkYb1s+Yt<$+pa#A<9;ZXuhOb3%QCe$W&7ecx zkoY{x214^TTLY+0kquT?LmB>Sz)Y+(Era&?}j-f8G=Yg6X;o0TRdg7Vo4 z#?UA2X-Ot;Leo^_?VLFOK>VsKO(#h~<>O;GQaE@>_%evU6r525-v>a@zpBTtB*xJ% z+nk}cDxS)9pK~+?w(vX1kqDxD^ES@ZoWYuf9t}+j3gNcn0d6h)N4N=~0+P~G775Ci zVWRpGy}6@Li*o13WF;m>%uVQprqITT6zWU+DAV8if_zpmL&w|cfKBz*W?^@5Tr|Nj zFEo0&2!`CKC8BDj(5`;;8f3{35-!+)_U-6AUO_uxodW=9iN|FD<5xuq;h+8Q%&~vs z!1vMdTr?I@%aOwJu!|d{b|gYt-w&A7^~Fh~!1Mu2%HkOyFrWG$T+$yb>7T&Gi*V40 z2+9Mg9ATj_(6o5J1~a|E#%esgwPTH}n{)NEpu7rU1!T2i*jTvU0R)Gp5*4KsEYQ_M z@X^NN?^M=EW2t2RpE zYDl?&oq95F1OEf%wrS3!LJ^XDh$z4P&SF?5y6!yIbK_D}Ezx4J48Lk^RNqH{$9S4D zIteRJY|JOw<(Wh18gAlA zRe&2O#VG(ZWuY!c^6~-5Xb}b5Ji){b4ECv~@=GN{rJ6zai(CS0&1k}&WGN?xjSQh> zX?VmrEZ12hy^b@=udW&O;MunDY=_l4Go?UXG(19+V*IgwU5->S$*bnPF+z&8R%(fX z^;-Nd$$@8Mwj|4rBLAG90<3PpM+GV5j&I^D$ymfEtiq3IG8DgKZNS{;!^qgGqB{#m zwc-WI@zK#6I!*|b%Q~bG^9=;{JexP^Up!DtI0wo>MP;$JYyFZ2a@ ziAqf^{;69e`zfOxu*MD?4F%L0jpolvfp@M$f3Dwvgw)a^DrzxpoVC&)Le#{q9S?n} zxOfHJDJQBp1X?s~u4Rn0uL_Tr`ieFrk@gMT3m6j+jDpI(jO7*qrlx<{x^Z7D#MNxu zfWX`tVxpAxp^cwB&r++XWp|Ecw)F3bKu~9}CP{|Ys#?$FpXI21TZ(w@;}T?-Bbk=b zk8Z6dtTC}`%xYj)mT$!^%WQz0n_oXwj`ifxi=&|F;J77BvJ|q$Ll9Ek$^;=Lq^5|~ zVcJ;GaU=Ccsil`;(D6>#8W)YZwwWts(to2@H+%ycvhUwKfx#M@FcvK<}k~cl6M(B%<)3}lcQM|ofs2w@Jq3Zlc z9SR(fYHjfv;8iL)Nc{-*R9AXuVG`{5W*^I}*bPoKcQ8)@F}?-rCDj-d0?c{diEeTq zoRtzc25?COi~3MOwZ32_F+-~8J(VvPJH9=t&H{*A0Rh5RDB#3k27~*R$Dj{1^Q+o7 z(0tI^+5&BDfrMz5VtAT(Du3{Rz95HT?z41p!loyBP3puk`PftIN`LcorqV3S4$bHZ z7G-1OSEAxqyyTS!grDCMOAlts>UmBKXG)c=b|6GZvd;(tb;Z=(-8KJgori-U_mR0z zwbcJ}F06E-ahhbI$O7Nr7c;SsZbw-t0V z{Hx_d1KrcY%}`p7QG|?q{kgd!DZ`$SPTS&#e!a-g17UCOtQ5iVt}Z@3=b8m2G2Q+_ zeBt{C9xXAGhU!9}oDBm1MVYK@FIiz`^uUin zCVjvMyp*kIu~0LA05{l6Y3v!y4V+JnYyG=M&RcNwxm`0fYM)!*0Pv)-`fE%l&Y4vF z?0MRY$~LytETfMrbgsH%7mqHo#67G4qa;OJP{8XKznB?d3oyQZZ%*OYTLEn2&`T}S z&|P)P6r_ZG(Qqi|{EMX>b5wF1zuF|C?IQUVQzRN_i4$z#s&1S(s_pL_<$dDMz6hy+ zT)`RYWjmYaoIL*Q)EAgE>eUQB6H4XPO66liljn%Jr{ofxtGH6w#Xto?GA0!26?q9L&qjqf=GWqo?Z7%CVH)-%Sh(JXN~D%S(3->Bvsk zHA9yI?Fa*fCEp5gj>RUA=Q-WXB&%H1R5x1q#vDK6kwoywm})dI(~0~MP@H?)LaMUU zL@8nPzvkb%U}kDb4FibY-n_GzlB!c9Y457*PM6LU+uYw-e(USFKNtn7Y$~+z#?zXAcB7*FD4!wzWm!pWrXg zcR$QpJ*)E=Kgh%+ZA#Y)^B5j!)dAtj8t5C9UXEpYYhvGt-s&f!zpj#L;%-`idYPJf zYL@P#a)=gv;%<%ZC?z>W+Iund6mW<0T9*)1%npIWb5))k(cll0ePCJ=nwWFf#Q!VC^J{svol`2HC!5Ra{p*^0emyU!d;>r z8rUgMoolfxq2Gc$m$qvKqsAG|pPAg1SEYz)Q2dMuEbT)i;2(B7qh`ya{(3P|+Fw$o z`SDp5t9)UL&8pRD2>Tu|t3(D$P6UJp=oKG1_L$fer2X{x4%|NYlMot z5P$U7Y~l$jj{X(m+&k#^E*pi*eeYbXt77_j{_G34)O4$~oT%UbQE!^Ru146bW5#%e z3ms)qgN5r6uHWYDtnx3|xGHIR6kQG(BE%=X3{6ck`4V&DyYi{Y9X_0OZe~j(Ntdr= z;Sj9+hLA>HMXJc9RvjKI^pE@I@)KXG2o*a}$8)y!KIBXFo%YaIC)f$7iQOF}jQWdoj^~*HZ}!6&^FccGvmr9z@+T@9H%7 zJ89j&^DO6P1jg`7)JXAN-s<@&i5ZBbO}?pK962{Umb^q|74+2~MBzJzbg&%-Xlwjq z#KTONupx_pK;EI}b}I={)3Yz=5yp26LEIG6a)0UUA`5g z{wdZgkKl^&{+Vq}aLQ1tw&qOzxdvnMA1AZjLkXU4xd&r?&|uHh#Xlq9`*Pr25^DW) z+_?;gdi3z{j`&?CR4jVpr1=f<(@GC#aoKp!9@b8<`D8kIY{sDZe&J*}E{^#EwN!Z~ z{u^cmg`D>UhVBB+;D25z?VLy@ikSA$5LT2xMnB*bXPqAZA;iP%1G?+fW6%}cX_Z^3 z!c)d7o3AkcI)C^!LP_!1i)O1D@W*cl86CZoliO}QAhbVLAI13^r#`nIo4NZgpPW9W zct178{?#cKzpig~UZP-ndzMb_PfDox`ppP*YaDlvLsu)P_Ea~)At@G%beDAv;cirn`iZ5aIjGGhYgH4^o%fh(26Y+@h zn(W@v?ZL$$Q@W}wICYCw{p`)JPwK2jQn?=EUQIUs+!Ff{o%F+(f`+!IZl2v|6FKTQ zsY_;(Un|~^$TTnx_f53kB+VR^1cue5ZoB>Gaq5cR{_QdA7?aj^UnF?bUf2qis3VlWm z-Kv$oE;^bo`0u2Dg2DMzv&d;I;+5fiYUcZ&LsbKcSYrpIpd&-R^lr9otoi7-ZE>EH z{{Dgj5@#$`o^|cQ^UJSyi@NrYPI{Lz_u#5zg1HppFL2%#)q{GfV|(S^p-ttWn?vZg zyaLBk43qMYqF-8Udsz#XR6o0FZIw3lGD1%Vx})S^grmUg=JgW?j)kb#;V;wP+hAq7 zap(Dgr<*rXI$7XzX3s+%j~_*iK1pX>xFARJW1T(WmGUb6TCvjckQD21cY{5$p=DRp za9Q-w3!W3o;c{2$KL@G>&c~?_m)@UK_jpp`T%N^{&2etOHJ;-z)M)W|CNZhk^AwOW zbJ%R^jpKr+gblx}X92$UO%dQA@Ix@m_w!3`imAN2HaXKU@xhkq7ooJZ*^l^OV@*rgg>&I=}^o})8*4k^I+?yxNrw^I1&VIkMf>~}erZRK? zTxo$PTZAVpw!an}Hy*TWsfOAzgJtZ}H-zTY0N{RGK(*Fn$EUvnc=YE@hn~YIH_VHVFPX zpLG8{h76bA*9ZI(0!Cg90iM*lRnNTrZM4AQi_*T)2mAVuqz5@$-rvXk>vDt5!6ms( zNRNfo)1a+JuBbg`!k1rFPMihWsBBYRcOA+Yp_10+j6Y@8Z+58^P|^pl5HQez)(^@|BG-cjc_)96`J$#zpIgo)1qatL(s=3&+BloU=J~&*U6#EB)m%4`Q zQQlFEQK(hv~m=HhA)`JCEF*3-``D(5|aDRT`7kn0~|6LH7VMYt{l%?W z_Y8u+v6h{@*n&U@b9`2bn~{UhTfpYd@tc3=;fnc=_1CXCVjR?^P3C%+YNi=q?dH=X!N!Sqm} z$xipX{#;EEJV}C?e)Y8X9U~2+*k-l8QLyj(SLQ0wYXNyXW+1{T^K*OKgAe4j|ebal^Ys_kIB`1WEDcb%k9MPBtsSM0gW1`AjJdPMV6iXkobomaPrOKpY{ zjscPLCw6VcA|v^@(=P_oXS}+AXNbSG?CHlm_p)`WF(1?y{;-=RrROXdDyQ#c3ew1v z3?*`O?%5O^^h49$<@KXlB4N+i zm!CrsvpjWFStms9UU@(?;8VA~r4J%JAYwul63ULLl`+d;&;tcC-Yq%qR zN3{ODDF?5c=TVpy@%;!{xdLRdw3vulJxWVtwy3=`s7=m2%Z0f%`QMv&xO_knY_)ex za-<>po0jeY{I7-uK2P1jqs3z%w~k` zXNku%*nGdX5Ar$PLBD;eBC=jpJYgDl?1?m56sqsz9k=}BO0;^B%{CPsnGf&(yguwD zv%95xR43!otFJGHfk&}JjfQmZNJse5s|1fj)X|PvEx60UfE0_!xxM_ji^uez2mVC- z_3g(yiN)7wBI>zeTg&%$e-PbnHKL2DRp8~<% z18okQ47B|F(7Qx8Qzg&nfWvN3c4CD?c~@N&v*Xzv@QG#t+i0=12@Q%F%?}0Mi*B@bN1etlK2JFW zwMp?@%EVR#H%&U?9)E_+^a(hz-07=;+0=Ve9T#D4B%|A!%vDq3L{^_W%<@?=5}P`v^AlhmWGH}bM&Phi~f z2&EqsAn+4`WlataO$evBY&~(nC7RT}n()Yqj9Av8z6a{f*jrBa?HgsQ5Cfh(jHXw7 z?+*4n%zv>AciElc!>W&yl$iq=rrUYjd`yr!tnXb8;^AqF@SV z+(GZy+&rkO1lIwepp$OxPlCel0lM*d;M^3bl`SA)_I_BaYy>8<`2foK`8jPs#)-&) z@=2)PEhNVX>cP^QNVU%cq^z1GQ|kubm8FcPRhazwu`=joDp#%a!U*D=kH%O5X%7%*l({a~xEAJfRKRumJ)wW`4}M@wdx?w&PE zxe0J%)Wk9;j|hi^^Q6SX$45Cap9vDL^c(ZHowQNGN0;8MU`cf_3w+iRE6&Dj_`_@^ zYJYGj+IA<0X=t(Yy#Eu0#6FF}TG}c;iT(CM8w{ZgjTH};BJzVNyirZ*BO(_ zfo%aqD6|%l0h%tEEriY zZdw3c-rLZ4IXPAJlsdYUqe}OKKL6-k^LlZbX;VWm-Ts3-6M%~va$PWFw4n^ZPVzS1 zFAg-ukqT(5V07tG48ky^59px2R#{-@WX_Snl|=XbZG(+_-0jOLaB z17gI1^mLD{-iq-eU`EfUBK%O>Sx$w8MDXXZqkgUsqg88oDhD3#a$fndLv(KOj6Xex zUXp7OGWlK7q-hF-antXTv16WRgsIt1DCT9pM`CzT%AMLB{_0rHfP#vmWIvf} zRJR|S?KJh{A#kYjz8&#rlvRzo&1u*HLpVXzVpUiIiOH7*rZ1YTMQ1PvJ13aU7%`E@ zIVTjC#CLgDMlh4x0pL~iW4DuRJRAhT0T@5VP@b%daXAE{{uyC?&8@EQ)1R>O22$0Z zPMsSg9*YfIR&5OX`FE!4?9=17mM*@}Id#jywmd?^R?K^m4mBI2?28q>D5b0w;A-&s zYp*uXtny@n`2MOz6cL-Lt73W9;8aQLC))<19n(~0-LnkevZ6nlDWg@fH)v>wPmsS) zsc{>Ns17~T%5HsHYT@~ZQx~bj5l$oHK4F_J03u$@kX?1%RmNDRK+oFOF5RCMPk*GZ zH}PUeUUhb%BCHJRboO*p75yPp@3tfWYQZQ%2G|d;zFr`hGOH3fpdP|uRnFV|Ul{*6k`)3Oui5^#nYS zHt>qdE&QrhQg5E2rn4@5MIyZW+Mzsg@VoCU9ra+(Cpd5{UEXM73Z)Q4XpC$WaMMfJ zt)JG*t0{Os&;NHKN?%MBIbuhsOXSddnL!A2P2?V^815$UWxkPwxDOOS)%zI8!;nb@ zLR4A131C&YFVi9@BZ}}|!&*Pj64BQ_f#Kq+C`QwT3XF=}Zi^hyk~$PI!9{Xw`oC(X z1}>Ei7_2(7Dlw8P`imjd41MGK$*Gw+a;VgJ_gFe)w|2Zc9>-7f0@=XI^+1qb7d9V~Dp=$#@-dzF}|mMZ7L^P;@Da=pRJ`z-ln2wkJ;evRpYd>7|@nQqDyrkZ{w z9w~P8hN4*IZJ*3Eg)lb`dejeIjW*l8NX+i`G;Nb%{?wCK)Mx$18LlC&RL6OymXm{<-J?V9I1Q<}IuozH zc7J+i^~IMb2Vk`UF z{fM$XgA>^X=TU%Dn#394f$T}8px?WaONIUNkeG?mkEv^fwifOo3A}-$wIr93;q{wh zrI#(D8t+EsmCS29o}81#X~v>4&ErX^*Yl6?o1x&h1M&{psbzr(!!^QbdSMHS0_LOh zN+`Vd#EUCGWniu(Y@z2EfMGZz{DNxbnZRbU3Zc==hp9ldI4%^t(4;`J#+0K8C2MJn z0_XwFa64uX-z|vW0R8qAhbxo8VSTtdqpdno#knXd4*tDgezMbM0r^mq#xrx}15Y(5 z)-bTr6P+fH1MC3pr=sx-@XRBxrOV0TMdKsW*AEn(Uf=;a53_2ArvApt>1&Be5fxDC zhKKs8mRk@{4;vPWJ6Yo$zcN7g{B5?h%P<6e^xXj3qbj<+KB96;U(Otz5%2?^#!{0H z4r^_Kr5{4XCa|Y($@!elm7W*FEk;*taTptF)xlVtafB|6|3sEmtH&EahO^XHwH$=| z<;uDU9cTrG{tTbec&Qg6+5#4lcf(P`OTRO1&_>NWICx6*dlsj!JyB0QQHjQi`x1RQ z-9j_#Qey}2q07X|54m{Z_Kc7at8sU zC%J#;lP5dzDOM2#tRP)P;igjbpEyaoSH2xngvJ7cMTwjOPi_zY=M~WH*x!j&E2i`$ z@<8+){i(hhYApZH4?lPy}K}KXnW?G`xy%1Won+dGCRX}nbV<~BdSVWa=RJ-ef~ZZn4g>H z_1+#%^o3gp?hvD)`S-(&@844`iGH-D{hg1l#GT$ZHaFPWRi;I$VqmLr0sIxdHaWmY zHwyuVj!~^kp^?=TTHj^DyKxeC#G=*D#jK8Zvsume53`m$-(YrKsddG@pge3hC8UR89; z$tRb!**B16(AlPih-Ny*ToJqbncLYf`~qklGKXEJ2@Uy<)lMXy3w(?@XIuDFBVaP!Ym^O zei}^Ux=D)QtPJWv9+FF03}DEv5x}8UK)V%QW)C~MYNa-!qXw6ZMZjT8q1=847&5jH zMVcI*>z<*yhf%lxXLt(CZ2#$zN`OU9oUxeT7*9qwyYnn^>dtwdo7RR34kBkON-b7Z z53dV;A&h1aL&RO0(he|_)2<;uN}ln@KKac$H(a51GLw%u?WvrgD0c}dB1c%uaY!I* z+1sp^aVl%7u-gwvMQmXh{{~`9#$TqYro~Is09w0O8w;5gIhjqjk81;4w0EoP-C>e$ z3J%7aPbA<^0kmNYQ}ogH2z+ULac=q)=NFiO)jEsdXrP}O<#JrJmrA4Ab#?wZ;|26= zC%Ue-rTNhE>T}6cTc|nplQfq)h90#5uhTIxrps0ei_6nnV*fiHW*31`G`eT90CA#I?qHO{kmI_ zfX0E;kVB2;bF`lp!&GW}^!S}sVHW)oHe5}g*5}>jjY-$_T$_$J7-ZxcSYbjS{rl0R zsH7^n!hT7ZvzzUdynn)6UJi2FgCpuQg==PrObAj`IvlHH7tqfaU(AKk! zY^MB-e%_?w=N+FFBJGDm9LcANBKP}8tb zMJ%$B@@I~;?a+?%uGL1)yM>luM|6uqo`(I6)7?&sZvWsEPXGc+KZ-GNGZ$+SfRCAb6xGyxh9EsrkaUaHWsWLJoFQv(Mw?4qjV$N%qcSaO*_OPg z&|uy{NFiC|+sg9_X4#Z`@i~XW3&j|4S!VipM<15lmngi}i1ybI^bkdvknYW96WqWy z@i6_{Q6taV0VBU|sAq83GRa$il=IflD08_LX-18nBMuL?%l1|2&xw4B)04FLOOPxD zC$)%_kbLz~--iF3`HWwn>kGz3y3K1`BJ7l~^Z&Se@3*G%we5Qpkx|+>NRtvrR8*uR zH6$Y_Sb&ToAWD@ky$1+HML=m$0qI0QK&45MCJ<^uuS$myLT^bRKnjF+W%izZ@B4V) z_xS^!BfoJhR{PdtF4#I%JIEKd4&muJ~72&Frs5a9|;p$a5a5F zIW0miSR)z5*lK~#mUo+tCS5+PGmDYJTeR$Rvvvj3qV(p5HyS@2NUscf;Z zK|rNs>!mov&Uq$&1A~nrRqmkTWPDko1!c*(JN^C#Qm(O22C`K5%rtx>tWkjVzY5BC z$lAkT4(N5zM|%Y;tRGY8rt|?8&kQIwmu^fW0h!Ck%xwP9WkMziv6RPho<_e7q=#>@ zwQ*t?sBz$+p|G~!BK6G(%%36pdIYC$hd@fN>izyYNW`qlGAynB(eo5&RU_KLB=+g514Q0}!5K>nvo zZ;Fp0&9ViFhFn(aTNCX!4d&w;??Tz;z!0&`zYzsni}+SR5|{wnl+Aa~VaO>a3GXwp zYK>)a2~g}ws~);^Rr^fnxxltjanaK5u|u{elIV+0RI^X7m|*UQ2PRd_E$DuzdA}yO z$skPkFBojuk5^b-dRa+){T{kG8rZOAX~x)hwIl=*PvJ|tzq1CVte!l!ay7dl{Tl11!oJ6IkrYZ$Z2deQ4?fot;?W<^@db4#9TOJRz*CQmT5i!u5z zu0lMWj+0|L!{a#Tb!)+P){3tqqoBpE6LATRGEWYSFZI^;CHovvk)-Scf}}?JxkYSq zRChY^iFeShwJVDDumB(t;5^F&r5h>6>Ab9%|6FYg)X8JQrTI9;(g58rc+@Mumtd=T+FE(8#IZQN&l zjT(JoZ$){ND<-RXMq=^zc-eZy#ZvdOyFwrdMs?lKV%1Ao$!dogdar{h#g=&Wc_T_U zG|1GT!=i{Tz1v#|`#k5(rZjX*!|hZYi9gjwXoG$4k?=HMcQa|(Cl(noTFXAAZ)Kvc zZ_CNojSeNF7r5#}npS#~v=96C7}AlOW{OGO%8QJ4F*k`nh98}MPX^`*yF!HKdt~w5 z{U)3fef*K_^d+_8Qy7XErmXn+e8&KPsk?!ko%ZMs$&HRPVoY-{EdJ8h zp!XgHoT9?IC&aBU(A~AHMx;>vwLA0m6v1ixSyX>J9=@Vsy#@-Ab8R?y0b;Cgn`5*) z)OXq|5H2$W)CNX4gtdV)X94XUg+GwWf0BsUyRG39(IS-PajeVfF~(3w>`*avI79P2 z*-HRAY1}^}bY^CKl&du}-*qcvB#W3)=jx=3Sxaf~x?b;lb!S-dkH*?Rv-6~x*J%?g z&tjk!UUM*DL)iIz-QZC`D-g{=KDxSx=p67ZY8bt?P!XE8E{Os4SE{*u>v!a~hGZB5 zyF(*@`cbiN5L81BQO7kb<=T;a!Dksg7`Nlnp22|UVFlKxXe8F@hzavTmGWbCk*c#5 zxuF*u?Uyx1X4cXI@(luZ6%H>-ddxY6Y?TS}_CUV^PM!I@f-p_uPw0q|)4p-o1d0!< zOqlQO%OG7=n>5BLimjY9(xSBHNsFf#*!7^v#>eFl3YH;{d<0NxTAi8L+In|(?nuD_nXn4eChV5OcYTHzt@P_d>RFYF8|Q}+(qe~J0lRzk z>~kjsuJWM`f)o!@_i3wKD=9RF4S1y#J-vMl`S6!aX%1=5ks-{JhJW0j|8}i8`_AV0 zBLs7aZNAgbWKjRhCbpfyA=9*^(NtJt*}n0Qr|WMxh)KtD6P1!2?|;xN{}{q%i!{7U zBMWsL>akxTgyMfNI{*4#IqUzEG&flG!jC!1Z_>rcR-37}j?o;ltLLAAJ4#@PGqi z39!V##0WkzNZvw_joRO!At;MDW1aI?FNuv9s zyLVZvr?2BQW@vj*@@0?lUm*u8Fg(k)1B>yjsgk;lNA8@+~RexRR4=Ev}r(-vODV5l$HM55|i z0;r)MFfXWrMUm|{sxc!1+=ryko^FeyHTwx17L*L4R)Td*z+%v2iaEvnUv5JaB`!IQ zeYVPb_wWrXx%%+rj`b@xa-oWMho3deZ^V)wIs{{#a^Igmc}iXBV2>b-p|!WNwd8T} z`qzIPo)25r1x?U)*qT&NvtJ zKxc!RAZ4w3+r#x#bl(04F;qAH`QWt!D!R&B@E-qaE#kSU{ci0L#s{C+~HABJQMr)Rt{ z%^0I~=tIg^LI9m;aHWk3OSHeix)7OU!PB3Ha-A8iF9pUESs;reSv>1t-N`e3bqA2{ zCd8HVRv4OEXMgmn0X8kj`;|>HlJ0XX8Pl>^RJ;2!n%?828tR9b0~9_8zy&dFG<*8d z{As~xG56%xzX+ZMOdm+!@@M*!utXh?Aplb(=1#Q19X^=b!+$kOtBf-y+#CG0!1;mO zCEAtx48+!v(xXejHimK$HCl68?qrFZ%mcy8EZqd7MwtLXGY0n4*f!>kea4>*fxyFx z=*>H3kfzrHkBvI_J>`)e-Z76e$a5}XL#?JYYz-?Hr?1MJ%w>Z0Zh5jhi*&g3!zGKN zIjvI4*4(Q#0;6j?U9#@<&%ZeZSKAXBRyl+nX7W5yZD(oSjMY;Bn zaoeS@S2W|@m!l=qtL8c#W&ct&lfF-8iEB@>*+TBQyEu|sJ%P9S+zOkVQLXdbLwLMF zBDCO6Y{_g4-tO_uh$p{ou(ySJ z%uGBz)7>EY;>{fW0()}V+PxLPD$}rWex1!OFr_X?gPzJ96%3bTwwl~6j;-jrs==@TCCE zR#K@rApT~}Vdx2-bf|yv?!?*u%s1;o91-v+!KvnfVnZr79rRT* zOr4;n)rI9_zSvmOxJ|MIs)OEdo%|%po!ZGvM$(fXPt`|hgbIRCFGBDF)8bdsR%|f( zLw@~1(0!nhnX%p=xHsGK)gTx@xAct@bT)Ru zq%`>F3G1*Vu1<7s=}l{~j(^tA{g?Ai7bZ_+>foftf$6_{((%i$>oIbf)JxH)PpP^$ zmwleF5saJ11Kuw;AgTE95SZ?0*l<`j(H_^CxW*pxoy2j~6~A|Qw8~~oj=6eOZHU3@ zJ$uGnw%Ow{XN!~k5uf+uG7Ts@Yzo}%KR*7s{VG+`dg9*f<+Fe<3>l(TycI8PX;cEP z+xTd5Z<`Gh8(zA5W8deFG~Z0fI>k>q4sl8V^6L9soB{D((UoUy)^At4C2;SQ!YZ*P z8<~1`*Cf4d@488Y%%$Ui@veUtEuZ=3kZn$A6$4B-z4~QaoCznrZ=S_l_;8@*kKgPE zSqIzZPNhPe(JNnm@f%v+c{S7d@%z+|#psA%O7R{V^$Ve z&*ZE6>pSQ;Yr?`{j?OK`x%&^=jgQP$-a4Y`{xzvVy9b7PA}DWc^!-WG>&%w+YG;Cl zB&>DE&+~z(4VC8+!oV)x|FfRJsKD<-r%W0*Ptj$-qEMlyX+otDh5`tl1HaqcSx> zp{P_!aPm+{Ja$vji&aj1V63RycUwvl+7J%-@7vE=(*MM~_)$}-^*m2{Tw{dV6y#?y zD1vEzHg{^^JZyVU0+qgus@L{JqP=!;$X9y<3Um;=y1qK{c<= zro9S?r0%M;_2a<~7@lg__wclG^h1QvP(9bON_WKO$KKo7jW0gsyp^<5;Y-fJUd|A> z3YAIZ4b$9M{rDC=kGL!)StVA0*QY+TxEGGI3vAB`APHmn^K(T^-iYF^o*4!~&68iG z94>OoYKQIpbwmsE>o%|&xVBy^J&<&YNH?~iOd*?ZATp0?=JjFL=W#9yVG+{MkTz}H zTRamt?3OYdt*1Im4zVBwstgOTRG>mE`$6?u3_$t%H|L8ikjSBWNBTwdxdMxeppx@n zvo%j`xAR3jGn{;#Wh@One9`~;e)~-yBj4GKCmQrz>%m18Bf^K@bV>IDeWLkqnqf|f zLQ*`h)1pthN4$Gs&`gw2FZj^t9lIE#;ATn|N!R&}uXJzfbi_&X{qaxmElDXW{4ag^ zfpZ;AGO^1KZp0+$uQ!Si{kmz|UHQY8=l3v&2eOiLU-(Ve=k8?$K?joWo<=#-IqJr` zP~(f=&rJ*ex!jy^^-*%5N-vR3Oe{{>^|%7u#Ep5;!vpeVg5L0Qy5O54eJN&t06rW}tGM5W!(jKV*W0u8N%N=U z7Ck}e=q0K=tu@2E>|xoz5fm^#zWoo&#%*x^Vncr+DOHzlM z>e)`3!r4fyx*aSYCb{h@2GaJT(j0wMD@KVf?dL7&UT!=B$(F=>V-*g4(E?Va^cHrUTGx^hwk1?|w4oiC9M~na3 zzBQT`H=~y+`KI_Ymt1f_Gkl9z4pIO!>e$bkozDcX81hg2;6&zA-r1E~a^&;JZ*kz;;bkMNj zwr-19>CrC}F8fg!kd;bNUbo;QP&#jidd)0#~c%}VV&+$if z<+=EX89_@l%T9BSbC}yvITb*%Nhe6|6NXN?x2!y-xm|z~=KJ+1w&p+A6ZLPAQIyyV$N2nWu!amN$ z39X;94|uU`?+E=4urVY(RS8sS{6`0nfn==H&^G4-=X@=5yD_GQzoe_QSgby#kTNWB zFa|G=b7G(k7*tD26u#BX9$HHq5}R5p#3bKksiaIt2O)gHjF9vNnC4&wl1CqJrH|ld zDeReE1%7`sdB_489PSp*Ba@1Z|bbECBzg|<=QQr@7is~^h zyqar|^auEz_^9pTRRP_J;y2O`nffSEac}FQ8X<9-Sy8ZJHn!lQLT-d~N- z<5u&(0fUZR_S*NNYk32h=}PS$J(`{Bz(wuU*^fVGCfmR7oq!>l;p8apd@=W`Txn&O zkeSaQyy6UX!^N-Iwf%33GaHo?_QH3TD*=N)tJP5U{K}n0y+?%KWz7$0#}X|nPo^K$ zrn}8okg8AYA;t|rnxY4OTnQHh_#0iM9@TFTN>BYT$-ZZ>Ji*M^oX_yb3!sICtP#?!}LLu#c8%j$82PNk9RL$_N~Ztzy8BB0V4aU^CJer za+B6_5vZkwyZA{}(#noP~`kY2~POv)HGn}QmcKNZEgt7Go-0cUmSMy9tu_^Sl}0c#N{4%Bnlu)dBH z%cvvgafDW(D6sKmrsA;11_{@q7kn{;Xb4fKmumscB(gMznbs?$e8T$61SmrB4kXJ( zdPdtrK0yu<@RDD%5Y}aYg~&C`*$2;;dL`*uM_p27w@Koxtxe%Df^gO0nG$&gmhpu3 zq2co3N6YB5hhsYNJj84?4r$Xn5B{Z}BWi%?>(rg(akr z<<_RlC9c_#>^67L%L2$p%s9sl;(5vg0g5s*A(w+Bt{Yl|y(6!cz;6`Wri@?467T5Q z9)*?4%EjOkrbs zx5MMt^QA(t(elA>!T`$(a)>(oM;5%Gba{>ekV@>BQDTRF;RR)ST$a)JlW9i#i@l$` zR7Uo>I`g*K&fML8O$H^!n6gk^b*TOBTG&5Yfb}pSn^ry8*+N{9b}nMY>5FCd^qgu; z*6IW@rpa6f75x!?7}4)@n%;$>f)AJb$k$zaNon|hy_gI;K6Miysp8?!P6#1t0N^mP zW_rUTum*VW8cT+ejRz2klmw6@-`#%O;0vs2#TIV-xYHcD#0EoO6wa(qkS;KGo5l3H zdsI*e7s5x0x(K(g{lb+9ADxBDRR9;TKwz_A^SLr1!5xQzb9L3LKnQvcqK&Mn7YdEY#r}Sofm&ZFy&l`HUV-^PTEf&T??2{hD@J5AV-7 z+V7Ux!%`-pLLvW*RZ@=@5sUGjgXhBOXn22I&Li^&`2QO+qBRC#dG5>%Ag7m99lUO8S+ zw5ymEoe9kE!w~^#ai4ulrRe)(t=geoOI(s0E>ZoA!6AVX2%JZ4B?KLfOY9Bvq0A3w zS%ECIYG)=gq9w6^1dQ;TMRh+@OoP^ixXL6`d>?pv=UdC!m^#1 z`-w6HtW>z!QXUH~~;NQw$!}6JB(F^xSRT*fAW3Uo>JG98mppzCh_K z2Beo0f_@?OVZpf)!>>moAP!GqwO%XeO6oJ?Q8%1!f2#dnWd)@`c#5g+f}0o~Nu1u^ z3{@8(Sq1DMQ=DXkfZxBkgwczCiY666Gh?_aQZxl@y49L_#2Hls|cs`W$E9-v<26NI$sMNjR3Ph!@H| zJ|9jS67=9Nh!fpw$fR`?1nO+~!O{g#eR=_NliT7IZ#UtcBYJnu5R7la_0e)+M2??n z#A~>J-IaQjc#4?Qkx9=FWWu6jU2>c>LZLxmf;PC6lu0X(C{Uef1R)26rO7|=+ptHR zx9Xe8!<&u8q%_7;C*`aLs{?zYq3R5SM&rm?c#IIZsO)Vl#1iSJa>AvZ-mITya`V!Vs{g<>qO<+MF2P*{d0*<(Ev@6Vbu5 znH6JqkB3@OjiwV!wX`V?fu#bWs9cV}GkhY4waZ-?o4U1vwTB?jDp+n5ioYmxbDTdJ zln_u9n#MnFNgk@Ln=709cjN+HBTjoQM$U__thlgW_CjhZ)!ylz&&9@@T6_D24~3L{ zz2vsP&V_hTGU=K`i~XGNc9z8)T3ov#bH=W2j@xM5=$W*vrGVNb(@f!vHP~0&OQ2LD zR37J1>1JR(ICL#&56rM!Ne8p{yp8t4`?lgvR~@HXVyqQ2s1wbd)?#sP;AU-0q-QSV zQSehxxR=xS`30&|RYEV$hGqjVAI6Q#9~KzYD<8JDn?c2zmuG9q2CScu5UUKFh~-~I zr-ncJh)wZJP?p^ZWK z*{Jlb1c>|W2IS~Aswk|Iv64kwfA)=-&iGaU!od?_gkF+P3iF^hvA;n?ETM<41>8k( zs*@|`miLk(DyWCh%pXP;A}Pz>5MC3JPsA^%J&b3L zq6@TvNo5IoNRMJiG9LF#^3YOyYH6{7n9EEp10;!_Bi!KpeG|wfx~uL6lYk{xf?Zcc z;_rGLhbk%?2hiQk1N!7s2FKy3#x+F)J<-@Hhd zxB2c}zH{#_pARIl6-{OI* g$^O<2Gf~V&{2tQ_zZT`X+qi2Lvhi&+Y_nn1im54L zXU_y^@)}GaGOoE@2&2CpF1-TXdW(uJ*V&*0JhE$Hw&s%`jF1jnk4=y#zWCFiwLa`L zrz5+(i4$Av!G)^CS^N$2swA+3zq17V<}iiCf42Y~##oP4?n`M{ko8Oz`)nWKK*fps z_N*%dA^vq=0Fprqx&QO#+=;?i3F|P4Ld{$fmA8l~^OIdjgkGM5Y-%M5t< z3yX$>#v^0!zWv%n{G2soDET<6;+Eqdw^M(<{M`SL4zN$OXj7|UV;IqcfFlEKKZGgl4= zQDJ^mA{+h|1X>49#!-j?YJ*@ezJ7ZQtswm}flCr@6VNSN-P1j$?%)9+x588zlD$BW zyjEcT1!bFf{gtPxmi@Qu?;MLntiRgp3J^z09t*C}8TcC%MPzF7SvKXdxymNt%(S7} zvYIdOV?iax<;i=+?|NyKWsBbameo+=q~LJCK&D(-x4-MaLto zPIAuydHt9_l9`1ibx+Cav(GppGv3N`54c)wrwzul^PAhXSarva@swFmuA)OYl-qr% zoL09~{D*Q-9urg7PGQd*AI$b5gss$99l%5S)D`G9Tp@8~^hs>i5u0WfkQl1r9U10x zcnbFMui39p2n+Yt)0VQt3||xHOmAt+9sb-XdN**&8&^YZKv=2N&!sNid&zm+n8AZ5 z|E{DUY6@3Kvmhs4liraR)C{%l$m8>?4;d0ctOknS#YSjGs6qkY-Ds4B;+bNHw{dE3 z2vJ-;EI{`qS*{fO)3!(HwB533f2>^=j83b`!G-f_I(HFBHK2TuEssvh@Qwf@qO z*$BptUWbawHT^QCsp1#4dSsU8wWp`EyoqMiwz@;oCc)tcyEpf28cqM_Db+S|@M>OeZEQ7=GiU zsMDtQ@wu5Yky?>}`hFqlm~1J0N8eib#Tn3wc}*Jz_(q70)7SPPy0np8fO1E(?td7F z?kJs+jf_vGJZtS54Ruc_ubvK={G=Zg1D(E!o_9R^E)z1SwX2&@1>7Q9VsGM(DZ;{8 zKw5WBLe2`ed@0n#|9tO>Q$4S>8eY9cm8_bJ8}jsZEiLDwBo8{4%a6=n=*!m=J)Q}% zSEwx=YsO6+;R1km$y`X4q{F9kt2V!A8pThsr5Rk->HIrv+L6SbQ&!O82V0dddl|lc z!0mb1{?5PAhQpqWs~yd~juq7CJRPpOb0WwpE@8QHk#J%UJ|HN$q)&%c7SK3#okeHpz}S`#&EUoU=Y2%{&7Oi<(5K2|Qpti&E3q zD#mY&E}TXVx(Xdh(DKabL@%M@Wj{&c-Z&1XZarLq_|GOm65ji9e`cVxHfzeBsUNp{ zY0e{S4-Dtohc#`!_8zAo`{Y&}ZM#THluT@}an1^uaM`Xq2O@Iw_uEA@b_wL(8=vQv zZJOwlYLTG=C4~0==r_%Yxkq^^4Yu6{J~wUQCn!#DlztVhp6Kje7MavR-^`%U@Q19x zt2GmPM#o>>oO}`kqnbDfkX(Z#9uy6lFu=htp^;Yc_Y$mDzQhlHkGfaSW@pCg^UHl- zFDLX4BQJBeer?TlQf{%CVE#QF`&yY zv9tZKRn}tbsa9H)V*gQMw{zut#ovr!UdS=qsJPuXZ;QTm$vmQbGQzEct0$sCxe51D zoRTDxwgcX&F>GrR>;~(9PwN;aY^hu8AGn>lzG!2K!R}_<}PL z^2G`~^`E?ewVp5aYS{SIoWdRT=zNS63hdLv4gN+;lNaHmu4u?@sP9era>Y3*559SR zBo_iiU$^exn7^H(DTR#Umk{tft^G6XOWixtsbA9vMV?*`_xM6elg7gbljZtrOWTT zM`z3^{2u32Z!4MKsw*ZMwJe%N-C|Rsz}V;L+GlTzHeBR6%rjNBCf2sb^m&Zy7jOxs zR>!YuxXtI7?YH=T4hUrEmNmv}jt_L1T~ipjC(8d;de%_HCZUs3dA-TsdRqWi^LC45 z)ttd<)99}dsSyCGjQhNY`sZm#aY=H0|~Ceq|ib5R}o61d1cTk z(T9ZI@J~ir+}U>W+`^v6!+A38EbIpC&IxIACq=|R*OyaS71UI6OuPeoG^dkwM&Chb zpyRA|i`YwNJ#$mh*1?Nw0@a-6ge9i0&TXAPIYHG6WfXtGiPw-!Q+7rJM zeXsZ?Df;`0SGSBHgdO^qR;Ui5`zeuH^l2Zsg4`q|;fKZcY>-k}wb@T;E4 zx)8~6V3m35Zf6d-R8qG@l6xsnP*2JV90!Llj*58m6laOl#y+;Yin968`}UjErcFP# zCl@5S{nw13hPwjg6(P0irvA`^vd?fBR)*el&6trlF|{&@qFfspkK|TBTa4D&y7mcw zQigY&+`o7%>$kn27y~eeLq)ZVOhTF2%SFBDsU$#nF|km=T!n*nk#IrmLvpy&S7BDR zS(dc8(ClS;cLW#_TOC|kWaScTlCtRXfQw^bF}i+>Nb$2bFrvTZzgU1+xapL|`5vSZ z^xO0#`J;Ar9!yN9TwN2^=>55P+}`9MZ{5e8RG$gr(E8BI``OPV+?MN1#UkQJwH(~! zVwT>sYS+S*<2R1qG$he>?PLYq{kojzPc5HtVEWgJvzC)fa-I4~<~Oct`bm@~FY!@2 zQy#__)Kl?^Mw*gl)}cEhxDCXNGf&+9z(;K`65CMiVGNZrIfHtw;y^wEK`IXSI2HWlR9&79`lXU-DCqaNI#C~Td@@*4&;Jv8lmblAdkT`BvJ#R(I#H;91@1d>)J^IA2xCey zRL@f|_jRilzexXUen(|Da6#Lf$|lF_zwx6NbP$$`IXd0lw%O%2AtCRa3?Ewm{`LBw zcK5bJA4I-)I^IfT$0VCxDDmiV=Q)9ZnMEHNUmBz$72+-fJ6n23=r(r|rI-5!YU%vK zV0hfQ)T?cZG)+d&nY{H*N`4p)G1IbKblxy6&LdyvVoS2wJa_!V)C-qD3Nt_N#PUeU6W{98yoa&DXDSxPbCh|G16DnKGt;DB~vOMnOOD_ z`ZEvJ@aqRYA9W$r+eoZpI?G&Ba3XJ(p8Rr2Kfo_mSE&|tDqD6!kmqr1kZ|YkkeTy4vh|7IK0Up=C4i+_ z6%S_lNI?9Gl}iT#83r9Gqt=B#g|9BvK3<@r%ICb^(XWWyi!w|>U`2Y~HLHi~Qe!KA zc0KEJ%67X!Zk8e@U0wZ79lBesAwkX&ytt#iE<&!E4&lW|`F?wJcW{*t=3qB{`{YJt=;y*#iB?Dhr>?O?iRoywA;bBy+cD< zez!7C6`D57X#Qbp8&&Br}vX@Y1>=`91Zfw`Th4 zDN6}~%cF#NyHH;L>dP-vU~gQJ!@e1H$H^%QM-CRoj^JVem zH?azHkr%I*tMcy_?m6ow`^ZFOqqjcS>lhEW2`9Z(P2x6fk&=WgglEp*=60ujG};qV z%)VRaddFBI{YTDuIOo^vTL~AlyiZw4Y)++o+P=fMK)6#^?Dvh| zxa7&bGiejZ%OIWw$by;J@#Oj1(YQ=7-^z$TsHn{t|jxRp1%P45)U`r%cQb5monE5fVrXC`^ z;`CN7;r&-5at!?HokDJ1+Djv=p%$$@=QbI`q5F-I!#e=O;_@Xd`Dana#Q|K(6;EemayE9OCiY?75rTeVXo3=|oq`4o9E zw5vKFMl_aezR@xeCAW8tXX9L2T0zgbGcHNi%kP9FYxzSBVUJn{0&0S2^v$o!Yb2j}Kgey{>tLC-I4K$34gXY4<_#tC5KF&C-ONWEH(a#ar)# z&o4kCa}}^TUUk0H%K?4|+V9VB{B!7IJDX~csdteRjz|2jiHN>XJXwesu3QPt+kg{m z6$eC|c~p+GkG6dI+;m;;{`wc*k*CK-YEan>>I+W<-NSU=-ns6I`Hk~-+jPeEgJ7** zv_;a}ng^DOBE;*|7^TP@bba^tosq{lE@OV+%R2Lm(`SZO1MWKTod4XNjLiqZ$`>yJ zKED-0*;+k)P3f?yUrck=o94kGUgJr9_Z_7cHtRyB{`Dll4ceeunJ*Ua>r=Obi+a|r^ zZ<`yC`wZ((hdy=SNjUrSfw;;oS7QS|GdVd{!|SN&cJ<$vZPElDufZtV%Gn-E>!ws< zbish+P81sKbXG|O2BFVW`e8Hr&=get<#Z;WSognMi2ac3Iy{v=f!RJw$7g)}+a^<% zMECH>L&^}hCsDR~N-6%+!C6)_7bqi>{V9eDtsWd3v*a_&9Hsr!de^Xj%J{_z)?Lvp z8>#J}0pS`Z);BInGI&`%9zu925cx;YdKr$771Q+~OTyFQzgrS3;v7*@rc>SR9Gk4D zXQLx|6F2CT=|2*i4JS41#dlNu8-Lck^$DET_V}M(g|oFmxsn4!{ns?kZdv5otv2jV zxCXF8Z>@uhJHV&SHKIQcZ|xJCl7d)EGgR{5o#yW+EfX2D6Kiyd#fczKD2q%3mzb$w zGr9TSUh9WOR<&Zi|Endwi*OXrS{BVli6CNDW127d$!b;^O{iG3l@6a7{jc{|q%T|LzXrmth;wdPpwR!<@l`)YXYPAuRvC%? zGa3g(#-9HFAOC;B`AJGaAfR+Yuf?D2#52NT4v@)9=>_*9fw};--ScAt*A4L|dm%Y| z0Ve6;k2Ov8zh*K&o&fE9(%>cG?{@)n$U( zICLS4@&0=$LI-0Z!gvW(QO&p5@{#AUY)Mni1E%tCu|CFAcM2VJ(3Ml8KDPyoR>*6v z#w#|)Cn{W;PyNMF3g6@)pIlwox?+hl!%80Lp{WavMykcFN&Yq3n!$hrLwnS-i<9^b z^c3tDqZ4Z4(q3>N%<6(Mzi$;GI#R(agtzVYA1w6={e2{Et=$-Oj$CiTNc+#8E!_36HtQ=;VbQUuY9-$vK86QXGZMYF=qHj{26!3-gNDo|NXd#ke zFLh29(Y9u?cE6Cjt5Tt!x-eEPA7BLbUCyjak+5HQJY?i-@4V6XL z-$4g!P6`F0eH2U($9+ybL=Rh9NyGW>)JAqi~Tb&;pHvQZ?F7Los6I9ZaT ze5A|*ymL~Zbl4~RN6^*w(DYpJmy^)ecR*pz zrXdmf_MeDVa~G8!XyjAKcQok097js;EKLHrPtnB}FMSjJ-W<>}J68*MjMje`!!k327E>#LDznAgihq(Ob3=&d>^MkQ z@4(rj-%d+FxOKg#qA!hI(i7xwPce|dPKr8ZC;}{Zh?>^?8{I{s1{hDxSL2lg z>dmV;k>EJ3Mg5mu$XO0hN+=L`%m@V>RwlxHt9~R_NSmnI5Qhs4Xd5LYb^(%g zcCCtac8jgEO^dG%fgDLHySf4XmtfA2IM0z?w}9GZGCRJ%4#XMIY)z2ZD3C0woIa=; zpR5&~FNB8<##pH^f#emK9NMqF$J6lm$EuF$N^$}w7<%8zocp6Ul^u`{*qaVSY5v#& zsu+39U}C`sEF6^Oy;}f1aM=Y)Ts!PWo7F$LihwJqQr;keYUXx`*#2^X_ac?Vesi`diXxPNX8s};6Ms09FPCmZ zdx=i>Yz(4m#R8FHE8j)}=F$VM86M31WbEfPwmr4!JHkDwRF+HJtEp+r3`pkmO6D9j)5gcCZ}lZLjx*Wr&p!g|FPV#rk+o3rF;$(*bHF!kLj3D* zJ9XQ^d97C?n*+U=fXK_~>MJe@zkv_72=g&Gw@$iOZR4*<74sV7c;PpJr+V!`%&~E{ z#k6cpA?uyv^YYs^aCUuq(U=k7>9iR5Cl03=z*A`uzo}d-ty}$4CSsx?g$_n(u6}x+ zCt##*Td6xCXl9}K9muRY30wLNUg{RPC7OM6`Y5hrAadd&eeoGwazXK_Y>C$cKEn;-?z)Ybh?SeR1sg`Ja0pOpd-bjoj`7hX!w_=^2%rL5P- zPIqOGE{nSV^zS;3g9CF3CUrWDxxsWdHirlFH7Ga40wV;^!oj-*(0dGlW99jN53sZ0OhAd zkn=xGA6|nwL*;Yc#Jb236_g6Ok-`$6bSGm2X9Ci%CNn#-0F5eYh4)8)zMvalJ@11) zgwJsn&$Et%PGZ%hhnQ9TRJXoZ9bcfh9o6|o2$1xJ`aQk(O{6+3(4xQ?te4ze+TL~4 zxA)p8kM?7gx$CdYE>s#@#cWoHe3Z1ETNZHo>obm$?8hyyyTc}Zd;siy5M`mDm_MZ6 zn!OCf=AQekCdQ*@JJezgUq~|R3j@4*{SB!@B0)6m?bLqp9j!;ksy>DIoNl6JPy|vZ z=(3&B?CZCV&-41(iL-Ed)SGr=t*bp<+jpMc-ITy7J1^TE9_%4+J^qnUcOMzmje0!+ zWwezQ+F7hDp4Pkflkk+vrf_&ui{m2Wq7D1l;Z+SI@gvF;bIpa|pWOoq{W0h|T`8zb zcLWdQ|6%XFo0{7D{$CXZkpNrSLg*?;?^PgxihxKHk=}dnEuo4ul|ZC72_j9pLQsLw zA)!cbiu8a45J(_MhqL1KyZ61%%(>13ICEzFi&+L%$y)3CF0aqGmmiDKK35bH;Wmi9 zsiu#&<|r>012>InJt`bMM?%1Xh%q;7M~XOgE#zpso8jP^t00TVt2S0s5;P^oC+qhM z!4&nxWRVI7w_t1r_COx*)Nv`3!kL|=aVFr>7iRG2(7;Aj<7Rfq#pFESoR8XO+C1_J zO4Ti=hvM}s;pKG>lD%dD8k3C54D~A?y89ll&|0^OPd_I{_tlkd>p*OTosw+0%O;;= zLu|xR-O$A0Mb9L^H!(kNi^6)0U%>8@>*9l0>8(rh9C_U(Nb3)q79n@d>z_7wg( zNvCaQ%+mIo!b@pa@ekTQ9%=$H1U(J}K~K$M%_bA0={#zuwTZm|#G?OP`1Fl#hl>ULTNmJcs9_}t_xu}C+bpBai2`d~S$ zHxVxi_oe(9FliwRv9d%x7NJzLf0SFl<$WOS{%WiAbUNhX;)IzO`w5-sh->|_|JR%n zU7Mtgn$R? z94EFz%u!|e<7Ee52yMpN&)Z*(wM{KSJIRc4L@q%&b~(3_z_*{}+GRKDhHb_A;$SfR ztrBr99acFhjV*7Ome+@HnL6*B3lFY;U94YO5F2818Yh9?(Qbaz&Tp(6Ie#)z-lHhm zUdTC7mZ+FU-{cxXa=B^!H|(I>FAzcrxdAT%Cg2GAiMIV-qd5b6*Hy%HIJQ22I+yr! z71eF*4}JylPRxTbvGe2FLYzPiLXVf~yv5sV_Deh8P{zOVa&t9sTZ4J<7{3F^ZSrGBSmm`Mr$UC(o99otM8V?7q?4C*~6OIJa&V zRnQSv<)m-1s**wK^nes&N6UvD4l|i_&%_VYI*hQ;*2S`(FhS0iyeRK%h7v1{zzS2F>` zy~n#fckO=x$fvVhulr#aR85g9w8}QiR-+2d8_9^Eida zykNegE4VWl4jSJtq)EupC2iDD=4JxQO&eC$R`bmTRZ4!RjQN*g;oe9%3lkEJm= z^i>H@Ok0KhnJ-etMRqet=5jBUz#~z5zFQ)@YH{})makghakV0$8&LsVKkl^{*q<57VN~is(8HL0f(owB#>+FB zi?+TUyzuqmU{0aslwNF->2fL2)CkMjkZxcx+~`)u>jIhinf6Gz$jraGLgIiMY#tj2 z!LSx|7tps|QOds>w?Xj9TM1h46OUih$;N^g!?0N6`BlT}jK=w|7$FkJ$Je|wM3g@w ztW`8hx&}7zS8o5`-z72aSVu~wCN{1)wSH*7WbOhNw-Fq}rYmS?E|B6B4a__)s~-El zpd#LUUjF;v_rIQRI)BC3VCK9pggNjjB{ju}&R`XwVZLyy{p3*IqgsvDA-NNGC87%$ z7-kIJ`W#baF~&<*0@-WZ?;twGv22nYV?m+rx8^w9ji#VJV-DvRmp||kn;k&nI$?~w zb^y^ENl`1d&27qto55mtVlnwEr)*A6kx!L8Y@$u3h2FS$EnVBLV8J#II-lPeIp@Bd z^}v!PZ!DiGUc4ibS7wk!UdgapUg!mc-@cc1+Md`R|LO3tJjHiS5pOLdYMgZv8H5WM zGo0;#_!U>W=;9fQx8%*kh2u}JjbwyRPRj5cUK}v@kN2#TAJ_emrTg*Hx_9NA;<|H! zw^!78R!s8e--=Hb4lKUN4#FaPF-{FhrjHn7ZIl!*#dG8lwDO&#dhFOb{0YglSI`(| zk}aqp2|K>i{?iCI6K~Zk9y`oxx82h-O1B3*GNy-8KD7yDqsEQ9As!U3;<%ucrNnO5 zwf72#PmEaO`*2WxwhFt$VkYl#iXbDt{?4Sf*p^5eZ)PFN0_vOU&WZU$Z|9KIPDb5L zjRZJ{_H%!ak&ho5wYD2I6=WOO+hC;r_eVgOLVEJE(|L9xPaZnJTgAS+gs~n8Z2V{uLPn_g-O1%20ZAmCb zs$v<&49MTdW@_YpO7PE&lpWEEF==J5+nG)E{2_)od-2+J8-w&Yi_tUg|z< z6w42aHaJuecT1b;CbFsj?n-JaKUB;f6 zRzTsA<|yPMIAy+;IQ$-c@CMXdh!(4PExwx6*s*~<6B+qleAS+A&0=gwyDviiu7t6K z%QDi^6><=g-Q3W$F6*eczsUfR%P!k5Ct7^xT_ZBHisCNrw0FYX*c6N?%71QExI@4H zC2#Bhl*wz^b>EQvcQL!OYNEF^bYIMZacQKHZ@=MDW8(`G9r));KZIF6y3n>?x-zFW z0fPGTP*aShQQNV+qv?NyR%rRy#0Y*LuVQicd4Jt|JARJRsHE##i>WfW^Ofa^dHfrR z)1qCHH6HZ3H3m`Q5^>lk0k>N^X56$ye@YL~hFv<7 zR+M;XMla_Yw&vgRuu8x#A;stFNh;!K3aRQKFqgQ+fiusW49}YtogC!P@RplWh1E?z zg1UunazM?Zqc*hMRNYJ3C#Lb{o*#HbZ<=iIlz+A@Sl&7eniEgGq?M`4Mcl>jnU);h z(oNv4OuTw9dc>Z@v}Gz1Fpf$CgjeVk>S6Ozoa61;+?FGF)2o#6-{lhdw2sm7AFem3 z+AP<;VcQOjK6@vx;L&Fr)OSURQ^<@tG~nE;$+nzGE5wJC;{M&sHiGMK9p=z>9~M9U z9`&WsEHaCSvNLJ+=&zTOq{Gw~1JK$(%y$es6qXr^zoN9=_#@Wv-hPQ6m&H)#v#y7J zb;~8d&S3eg`TJd$T(bLLwz1SU*=8RZF+|sFotP`3Ju2N3$E&i@>WD z+7w{9t_@ju%D9Tbm?k26YZKdCW&~%9vIyp7w1ULdV~hBM-3c!Oi`G(jo$u4bSEa8$ zFGI&Aq%HeO5|?PoCGF%+b3cbH?4GUC&3vOv_$(In@ps%{!hT_-RndW?5v5OYKb z(8Tf$8jAp;w!_iu;}b4n%)9|@_a%$WG!LGvBoZooU&g6xTct8u@K$%s1i6d0PrgGH ziT!1o@1y7Q>1megYVTytM6vups)j{;Z_w0c{IJH{rZ{b`?l*=@?ypFo3DA?d&c?mj z5UgxtbTcOg3Qx-Nmi$!YFx7;8%9v;!XtcNY?l3vsI|CJK8?EXnb zXnwb)d;Y+ygao}UYkFGMf_?mtk@Nd9=-1VRX%Q|@u4jYR@Xy1O`^+y-&-%pB3$d)2 zh`U8$K!}$^eJp?GvPt6rhvYv_%{MOb59%_(qu``?c#8_1oIRA~Wb`+XGG{xxzssqo zhO1veD#(f)nI$C5AW{E#I3E(h&JNWDHJr8;KbbKC9rvi%r7T11$zmVbwEH7bZqyDc zbvQ$|&^sLM1?m6IX-!awwL>~__cb4FD!?&hg{hXT-yENGA@DF3vSYa5V!o6={r_y~ zmeGDrxCpa!e!0MhKH6-={uoUxo8bgR)7n4DY|^m$pmNYqtessDm(Py?+0#GfT`Nai zq+(TD&d6(fFb2oB8N;NuXyTMB9RiROr7WZC*Q**GXDia@DC&HE{L8qNw|jfNdEknb zGokA*t$iR&;222kVxso~jK7f0dZ*A@aZ3$}xIoG7ISJAo*Z=XyT0IewGtH4S38?-H z?y-3-jbjBrk2h3jXG69WcOba2yT7ccWjtkRV$BjUwtvp&H#@NpSSkNe7KAt$c9o%8 z%@!Rm&chJh{u0m=`7>8}fG#ZPzgw=msNh!*}^OUBgjdx4?SYFjNgCjugoy+mq zTJYg&7^r(G=$o3k{YZSnJWPyvbl}7@vxXBn}(2QE&?C##a%l~ZJ z$`KW*Se#!jF+ah&zY9|fFtQUPg5yVzchl?E*FTQ@bAT0DC_?;kvPCz)Q`Crm|L4b; zR-VqgLA^Y`Qe_;^@-Pm#ntu*K4v1prW=@65l(l=*$A5mB5ZV7CQikP+v-|)23FNfE z5ZL%Ch2S-JHO}FJm|hE@CUY( zUPIf?jx{Fng_1S<5Mb%8tN{4!KZ8AIHoth5ZweUNoMhT1X8my${&)$%IAey|6kR3Q z?_${|mqeG7g)&I`t4W_oo&ZHaH_mGUw6L5o09BNe>o*`*YXs=|T%rfm2*uyuzLVAF z?|^E(3YZ4JYdkP4%(DPXHDTK+FYNa(3&3l5Er`2*Mi!89hd`WDNnsV3XlnL6zZ;G? zR0hVI53D~tB6pH#75Q@FbvOVadvN)rZemA4Gm9ZBX~j!@=*C5McDh}*G~-C0TX}lO)hltC55_% zRbd}Vg#Z$)`g+!@^iv#>^&d+Iwv@E<(Wmu1)mNqC1^PjeCelb`L5b3C=?+?T_t!&~ z?|`Mjo6s<*e0D5e43Ae=T+^#1SBNOi$aJlh-A7T&WZ9aFcEVae8<=idi|=o65)tDC z>yqEaTcO)M$K(=Ls_Z~J%?6qFrboq&)7Bx8ROt=@S-8M4j?K55uOv}^03L!94jwM< zR4`}RX?!dT{F}d-1PU#4c)|HMXTTZ1zfbO{Rl>J`@hZf&_ z%pS!l6i}s9y?k8-oba;&Pc#uzUlkPVm2q+g&(bJsR*DzJ z7MX5K^{EzGr53S}D`wYmLLNNdc0``Z{tFqh$-H|)t_{RL!V4>5evsP*SKoz=ms?67 z=Al7CzNf&7JnsZ3ih#j3&yr~oW}fv9|FJLt4>BY784T*GardXzGyANgb1+}kc$k$>m*^SsaJqFm9A89@O z8@eAR6-QW8&ptO|HTxrp%N-`%J6N|m3oZm286FfA-mgTCgLt%>$dv=I3ZYbKU;GdG zvLD>#y$g(+cXYS39x0CfRRY9a9@-UN>VWda$0gi4KmwD55vZ@1?)#HPFf}DPa5Uz1 z!_tD&Kr@gZiWB!>)0EA?X(T!+uA4*+V2ES~!Mw!2LtU{{GYb_&E@91ve%A9H>+?mWHzG!p%hH$+96N77Xt}f5hupYiUX z<(g}EiX@*NB^>p=2tXT8jETSZT7LZws=gLVyvuV&CQ<*|Y}`T;ubJYZPnVKU|J9nU+SJmP2o&3}kd{s&k7O=DfTa)hnAa&fUJSoT zyjjGF^RSC<8zHBn(9hEL5;4qH-nK$)fZ69~9svOI*l^E>opB_f=M^Y&1b7^>ODLb4 zB?-xat?WW%JOS9(4?Z+rn|Jcdhf~uH^$!=|O$(_=Gd~v?{99jQjv6W2ZU8;t(HJ&X$Rm0d+KstfmKS^Nnw9+g6u zik$%{gPa-IO76~Ka`X2esOw7wnhfzTUz{?7T8@`RjI4Wm$5$=laJ2*x5?`fyNM^Shql8+a4XGc! ziFC0+OBoSl>F?55d!_^`#H( zsMsdGJpS!4orfLz9jt?OR*X>tVf7Y4m3EWR$*bgJ>9l|QP4CFwkHnW#vIepvup?3o z$*sC6(ET`Lq+F>I-N}6LGdz>He^CjpcCw~Ny$6}{ej5rk6ATBa4_}W^y>(o~H{k}= zR%`9_o(Y0#1tB#%&yq(olYfGl`~8qRZ|co4%7ghIx1Teit%sacinMgjLlPG_0-0mc zNCTtcTu4s%ZZKLi3(CUPC{+^zih+2At($$$5@bwJb~rROVy|#>Fst&D+`nK*aqced zgTcF5Y`H9k6AGVnbfk)QF&)d^6?wgrd*(s+0f_z|NlRx^<} zLXWq|&)qijM;J20>dc`UPH#*elOe7kQMgmCkQ6el=8GhnEz6`3w(r4RDE{F|iPMDy zb*yKd&aRl$@XjI7F>3F#O|qB5@TCv_XcNrrzd1v|clSMiVKw!6(lUfrC2~)_K^t-D z^&33%wZ!*Du;#1UiNA#gOy2rUSDhpUP3Zz6_7O&9BS`*)+)G5akoa`1t}r0NN3kP{ zwyobt?h0BGRp57v6PHk+2eWoW>$UAZQO+-P)r8rkzvyQV!N!|{RvvaYl6z_+*KS}L zkc8+YMqRFBW?9IBvo%<)ziz*4wEJPYaCgIZmkU|p+@ehdV#*9 z>fv_0m*DQf{*VN9XoygUc`J^;^0&&w;?zLu2Ji>A(XF{z2xL*%QpHR^T|N&yu}_I$ zf69WD4{ZD{IS?C)U=JVFJb$+fGHs1s?9~a0b}u^l3Y}b4+seLHdoHiKc9I6Po_sqC{zc|bLwVB2*ghCey1hV@rG$n@&9AjtJM^KTN z6OcBkMno=TK^K%IWh>?aZpIc0n1ArD@kM-B`qle?z5yw2ekJNVHsLK!AV*Hw53InwhrwiVha*@o;p-AVXUF zOn^&!?%UmOt$43ld4w2&E@D~GidpI~GfwU}RkmuL_m1vtVzj*tXfdN-HH~QyI<)?? zmnbHef}wGNvnmtHJ?vMOOOzo)J4%+mJ-2Itd#w@0D7mqq{$qabDq6xj6fo{-t8a;x ze*}l+?~Tx&E@6xy*n?Gi1oUB$?Q3GWu!oq^-#CTmIk`f$K;)UeEZ6;O_D;{ai$Bk$ z!zpu}yL*jV+~|j1e#FO% zE3>|-8K>|4HM`a`FDGp8_u_o!!*-_A3S^|Vh1wbIV$&c<)%X{ek~27SGt`G(_s8?# zI}1`8YWXZhBM8TZ5&glVMfhXnVs6$;7pJm#nomA5P|IXX3ovQX3jrRjP4_ z@SmJAqTC(Fc)6?x0gCM)g{x+W^K>}lAevW_wTw;I zDWUEYi=>SE8hsw#IXI6*mw`EDS#z`yFiC0q7@IIF0=r`6UY%sy0`9!C7J9phIaUincypmltQ7`f9c1;RW?#mqXx|6C!Uy*yh}TS({4O%d z-OmWYy#bk=OH)nxtXz98c9wXfdFi=@hvQpUqa`11L`h86`BEW-)6|-h?}J&wmiMao zUevV)(CWq_~TW)Idfz0P! zT_Yx@((}Z?)z(wQ7daBtrQvaq(aSp@(s3?>Qw=K{MrsWqPy^mAWv_Rix>DBpve|aC z)B=V-O02~@B{$;-i>c?(9*G>4x~C5Zi6|+Ui_v0x^MnP}y5&Opy8W|$>zwBX$lQ$c zdUM|%Fuj$14>1uq$#GSJW0Slth(4wJR(`G%i*e%rz+lyJ=|3?Us!$x(@es zoQ@uMV`<*eVd442;xf5bv$qRYDwfv|k_7XJ#6`MhXmO=U^lI!!6U8Uc;sFA}oPBdY zQ>-M@?0FNuW>$BXP7u^Ow9U6BdIByO-U-1&M~S|2HOs>^Eva)!e^EkdiA%x>=x~=|>yTSiIX<@}ev|nxa<`elJ)XO3*fp-!=h2pFJN^^vXN+{R{Yt!$`Hfi# z?T+`WG(6QJ;TeL^ef!&AB3JUw;i9oF=&XZ*TZ8$CQ`17*+Y*>Ed*>4tRYqB z1vQ2)X`fwSh#F}3Y};MfAB%Eoe(^fPjc|ivdRg-Be#$-8)L{-AvFr?XC=D!^FgXAj zG}v7}5b@~0?;7bqP-9A(JaOH%ItV(>w^MCdO_|p{gUN_Dyjy|OxU_wcIstR>*>#(? zdXuO*P$SItZP(53Y>#KS4aX7U5Df%r%bp?*R`)!&`#631gXqLi=x1tKx-=(Hat^PO z2XDde`Bk^`-X6I0(3?@L$~i)q{NMxz7w=!QP(5`>f&pT`z`>bvZYg0UmscscVz(;o zRA4Ys**{b$KgEE-lOp|g=Ef(fyPS{X6|T+4&H-c!^_-$%>Q<*4TQ2fb*EcLsI9=sm zuMU4dK(>iAwR`%{*WaGIZ(n$6QX$G?H(Gb?ui}C1jNnwCq@wWZ59xb9rBpH^Wx6!? zxfMMsm%CCpVNa?<;u+cRjNoE+uRXA*<0Q2e?Mo@|^&ZMtti^{eZ~ijUYi;g`}Poa+!RT#qQQ{d<1k|j)}j5_ zt*Zm!T1kz-CnJKKuN60saaOE^o60~^tX7$p$Ag~~cd%p6=j`$$e?#- zI&Xl~w8VqdHhNDCopFvC;_}F~0KGi--=EwK56VQ=3psLiN0XbMgh%N6N8Y)P@a>Jmy4g=q`W}LZHda3p4<~rRPpi7$Jn}QZo6?Cih_siY4%e3+t z44FGK-jzGZpDJ5a7si*J;mz)B7&Yjnbej3ZX7Q_TcGExFV^*U+k`V^#-`;@LEe zWKhzg@^|BcmaF0ABle-Ug{7!FekdaAnKIIe!j?Oof`dawEG}V{Idl`1?}WdXo_D+? z;>{Dr5|r1b@4_;qkXWlF;hza@pIA>^mu@KND^>FI%6}O8YNar4@JYUJ+Qi~(Rclio z8vRN^9JL2jSRhYg`-#$5af&K^Kz*L2MI6PHdFG%FyH@nO47?RRq`Jhts3Jp zFEd}GsXTHqt>pEWt~jXU2z9j-9wk~R$ppeKnT;4%w3b_6dYXaiRB^lO@R(c6GR`f& z7AZBMlV|e4I@L02@6|$Q$-?jXh6(*X2O*8V`(o^7Yia3+f*wn0jPk_nDo+@HUhq)S zpWOWn$`fXC*G+EK#NN$hwQ7i`BDAA=8zYEH!%7Lm*gem_WUl$@wTw4s{r2kR)#99& zTuwvrd-VQte&J;i(ayW8?zpS7UbWYwTkisH9+kk~Hw%znEx9h@{>0y!*vDf-B&S5SZb>R{OYP=;!oHBHlf z-PIIjAg#5m#p-&%V)cl3LD2j`-XVjbQc``fHqHP;c)!anY_n9lC!H8!RT=e<5)K0W zn!RiNw-~qWW$v*9C5Cy13%f1D)BE@^t6tfWcmbqm z05y`BAe%3>-T9cvc>5G^vEY+q3;hOG^LuU|hYH=H)NZ+@vXJhdDO}A!4I$#K25e55 zG;DM>-aoll0%;dqzNW^QzvX+7p6fMRugjb#m%l>zTMbFLI`UChW}oM+SKz_e1+I_V zR}gr9*wj|Y^YE8NX!)kwFF3T z^Ob3#^6f8Isi&~8H$P+>LiKxP-zQ_Ak+kLKGPf6=%)Q=TZvZEPm|LV$GU)3MQ}0*u zVlGF~vRfXAqk}`0xH)AEl}}5RLyyN4&qj8|-F%lQ-skHTV`fS`V^g=H4#xGHc}$(hZc}jf}3$B|gF%8F_qHf9gXm@aE z>^j9j7>7Sy-ZAIooJ{BmXZ2w`-`uB^p9ogx)dwMk;;#DY`!2fvsi+8dP1iTGyWQSCWIwU=XZl2L|` zQYu@xzi_|0Cv(>}YFxIzJLyx*Y}jH_ZHJ;=6+V13{68j)rmb}k{NP=4S{hS<3ajnK zeP5;O6wWV_fwr0XWE(E_JWdw`635&`+*09PhVd`UnHnl7!;F+H%Qe0&?_6^9w4q zy<%+Tzt&0vxJ4ewdW2}N2v2w&)Kb8h4@nwy*i&?}3H%_ zr8T0n6*-ItUjjX#R6*9+-7zQ__9;7i%>qIW+r2-aeS(e@4`0gjAJK?nqUW&VgTYf- zz0f{v9otjluZ|}JhtD;h9cP7MpkJoyMUTYdT*Cza3fShS5 zJFzU^?v6&eFR|v4-F)(O1h_3p5i)}-s3gG_VweUj0UesY9`1nthi+6q49-YV4GjD1*8A2x9Z8CFO4 zPk&HM>_Z4eKFhg9&9ZbfBJ2EIVy&UZoXLUFAB`yl0G9XS9;KB%RLz5_{m5{vwt*F4$kJxrKzoH-u-?% z;xXlu6Xqr+RTM%AY=Ue-1M+R0`X-si696?F-g=K3)Kq|!E&Ai005eEj^D@l9uaW%z zUxg8RJk;OOeZv}lsLC){c+vnWYXJtxupk!Sl#XHF`?2!OC*OQER&(`08D}Jm*{)-q z{ZiMyWM=g4z(9F|)o5REr2boIqNgidyb275#U>?n+`%okyY)s57dLnF*}vh(y`S#m zFF^*ciLHTw;3W7JHL(Isd=Fc)XPa~q9e%ARUy$$>?Y9UrKM3ck4g-m8zqi7F$&D43xg!=*E>m#R<5uW$z7Y|wKPpysdTBChmrM8r#n4wb@Kg>dxQt< z;D~rS{3|r1rZh!qSn>c~n4pf!uY}g_ z4~hk(T8uJV!`aZwJM+Yfq;d{k!D{n%)o1BguHM$gQa^;qt{jKb3279&lSOVD#I*r6 zZR|JSrLQ>NrCeOo4%`8WAl=u5%LOR zO;Pq(Zr%Lg7<|vdtqj~5ieC+)BIYu+R>UbWGFSO|R(@^1h}(%|vI-MjKe^QuIJK$l z<-ZMDl>tshIau1W}m9=*QT(M>Gg2HcKA8?8416DA#*gnngf zA9&w4?M_?W(qoCE8OK!>&e1NJEX05ME= zVE4^9&}^a)vk{5lcDA3z9|Hju_5rj+uQ7 zm#9r*&Y;2;`d$ZD7Soeg&mZ1P&;#_2v0N$Sok0T3EpuW05)*O=JrfGx&S<^( z?L=dXGM+wB3clhM>o5|XIL*F?>3*mDE+!)-k!%+(FvIn9(!19mW`+7|uPE6M zye4J65zxesTi}4_b7t2DD#Vfi!uEKw=8I&FUHb16)(FEM>h%lir%9?qa2os%dlH8)^SzJLB*e2QVWCL zB>Xc?li7!sKTMsRi^k=dUkv95y3%kg9{fNk@4r@Q2o+s~o-f<34lRgU3i@(1$S{MQ z!T#rYq6(^bC}w5+`JF_0{=HLIQ0FtPU{F$Fh!3}#I z*^S+jdpl{fP`!GAzY`eXcgtu^^GthnXh*2bu9w$=X*?rXK9rBrjJGo7&pYP_bsOtJ zAg@3(Uln@~1Yg2J9|q&?o?Z6q(Uc>mYERwjC?0vaQg&Qas;6LDJdlR>I^LL``4BVPO}7uR0?C}xO9SgO5KpCfF< z)flD))7eU+X+o<)N>=Mm!De;lc)^)>^2GrwN*-+`+uq8{I3*YU3^4~k&W7&nqiDP< zWFs?OT~ZsU{$|B42sUAW;%83!*3nGJ?EhZTG~yq`eEOK+@?7#N8f{Aj<`B$#$|LsFCt*^kwN!Cpj zlT{@_kN&l_6s3&lf8GAA@M*srF=)z;}0G!qTw-aFD+NAW$hMx79ft*3fyIr;8dYps^U6R@B z(Z{3>q;FoS9=dG>J{&5lD&*A)?j7@Y>4_lcffl7B^qTl<)AQx^ndGkN{IsQ$0hqT9{2t8 z6F3C(hNx9|m5JGwqlM}#3s+%+7Hg^Lr0BGuW${WY)|9*&vYEef$EuT;WF`$4DNx=+ z1djoYU1VMLh*m(x?@=?TTAZ}7fBVpFT8m!p>fahmre&K1lX5W$EtO-gm0MZMMi#2b z)KZv6xyK{Z-lFh5plfa|bx;?IcbiHT35eryXpg8{K$-8~)(YPo{XF7VDtKs8I%`Vl zv0q}|ubZpipt>4A@BP#q}}a|Az~3%U&ZGpV}8Kf;=YtFtDjqmM^pE zuO@b9OF2BLDo*+l)~{=Je?`fmgowXs)4S9REceH5Yt7^{2nZ^opNQ-^uibjmSH;u( zP`1u|P1AV2d|NI2AW68!D16a?Gc69?@Zsr{X@qleO+mH)=KA&eYrP$Aaxejpx)n8$`q-#i?O`y?T@HUFs2JL5QL3;P-|lh}zb_*2nbBo5aJ(Vo)uhb$c3SgV ze(@yKIr(RJb~?mnaY~qD3V-uJvnWo-c)4m-nnMX=0qx71u-JK`WG4-y)(@B7Edb61 z4TG*l{!OoKlHMZqFre{h>zW%M6ltwJ0?#Zd*$A^{kHr;Fq)4!PDtLK)P?9kOTT{OP zz3b$^{p`vM0gqJ1?CH3PFz9&ORr&myG)wesoZ(cep`|mRmL3znU$>ch@%hWAWZu<9 zgY<5;LjnD5*GwqBU357`R1r3koT!8TsrqS%TNIV0rC;J*s#GBb>T;Qpj;O!Gd`q_| z-aT#0Kk{M7{hT9$$)iA0nqjCwN2=3iM`P@QG_6~fsrpC{plTP5Z;LmlFg2KN87As} zl<`<0S(+8dHk3BRbEImZV>J-r%LXS6x?%a1%Uo`2)u942rr462FN4?7(L56M8AcA* zb?b-Xs(Rm`dj$`A{6w4amh~>d0JMQXUp<6R>CQzJwewq~@xw|*ceSRdBf-HgeIh$VYRFOm_7wBz(taj=pGHY$&UUB2R<}*KF zaq%CF0J+nt0Ia5*HkTDQeGob1q$1lS8NK}{4)_O!y!ZN|H4rsQQ9mG40Ae@!bz!}4 z&$m<1L%e?jRrkk@CMY0Ic%w z{3UNBLWuEJ5Y9K0!oeO_l>rI^RxuX{JmzILtr=8jubWx;A0`H&?Us6!cgOVAa}N=o zL-SFa4x%C7e-&_^;D6NtzSu*=R{+R3$G7uRH`MkwF!$GMsjsgyRfd{OK}VdvJ_B&+ z0o_q8=to8gwM&(MkL36Qb6csL$hkAemAr^ke8m2ZmTfE$w{E30DX?mQ9rRf#W!wwF z|85l(u6IpSv&iIec#EOfA{!2BXa}+z23a7!)pORDwZO>Y$AL7PohW4|E zrgP8)-5F@Q!WcX8dAzT62IW@X3>z}g1Utk484pf1=vQ$pJd{k|3gDs@^=}8T z`QBOrS((EYu;sK`axJ))%zD_V8DZ95>$H+v-ZcS<*K|4C-I)Pm&33hK=g4LCHbD$S z_>T|+<_G|2l5j}=Z-e2$FdOzT+q^vf5LQaQ9O79LcbtrjAClx3zzTm>x>Tv!^?wIV(1+Q1C ze1?wR4h~Ii(=-pm&>`ERWX=e2%5N+JzdJ;US~62z_e-)Xk0ohRVIYbx&VaMXa;KKC zwnb(J_{|L6+%w-}04twL|PwSB{Qt7LUWZp)mEIy9m+(1!WoD2lWIkPVpiSwa^cdAdt zCh##jUmMq?JqbRSIu^66#){kz^$*A@Z!IoS!7~*bm0zH$s7XA~vyJ)8gm3<%*)s z)f;}OFG|q@}e76!B;XGhqC70%Gj+f=BmBm}j)qZ|^Dhh+jV@9iKjR6EEePMqyh9#WVDtVW=RRRp{7IU0eK-K8 z$kUY&XB$@KOu8K)%p~!oXMempJfPY7^(c4A8`aMC!Ax(?S(g%&`=j|mMbX{gi}dti z=L$->c8Mb+H<8NwZS-pF+ob$UD``L%{bB9ZRvg)#F;_-M_r$?Y?(8BL(li@U%z;on zcNCeS;)#pVoGbil2wFhuhyoJ?^0#;)3xKyvL9?np%PB=t7X{ru#PPqRdrlGDa=hKQ z5?sg-j9RcJJ!LNfF@Kd^d$tO}*bLaU@=?r{_5qH0c%uLZo5VgCB$kwL4e^DjX(i6` z#{$w0S$cV1&C$}_IK~DtlsQ^A)8#y7D0TQi$?2KgOP_Y*RE$dOY_a6-nB0w>JA+7A zjLI!Pqn}h95|G81v4P2^&47*vB z#$m5fv&2%sg$#f6{EuQehr%pLIsVRFih>ay!@R|!!7)3~+n(j-ZH?f#v(aJ7HCtTF zIS!iK^S>S@NEDvee%A5jgFY$>UIUiy^^Vk(v_3z(lwC!MNexp>Xn3iOH)%f{}`$r8zy0S5Z9?6?2ul zf0!q`4S7su%Nwz(>GFsK{Nyt{c*RvyYLv^*kjpJ~ubnfD z$K&>~gOe_OuQ+~Qe4k5wtu}I4nQTs|Uq)=>``&pZ`FP>F@B_D4D5A8sCF!0Y-~}Ys zy}nFv;JUcCp2SIIQhW4zjz3ng9!;_yua!GzS=YDdoYH>ot``L*7Dl?U98IE2(LVvp z{)Ed<5a@D077swbmG}>P;q{$Hf(+gx$GV)O&j&wq3t%}CYs<8Mnlh*eDmJ%Q8|3Qx z@(uw8!)bS3kif{bmp2iEWVKmbX_cDO?=lP6@+qP~|HO6Xg=aVIU{<5`)U>d&gkwcC z5NWtBK&5~tz^wy(gQJlpN%^U#{wy)YNb8}Z?eWK^9w%!)B#;7JeY;SYnkzme$9A2O zLgawjW3%$3j!*ZSls8Mf_WSM1hu!>IE#;!9pITo^jCid{y0J?}rY==@0pFngmOB3w zysh75dtR4o-u92LGkBYH3w0xDljsYP3X;a#(*>8B8nDCmEB5E>zL~s+4YW^+&?AJV zoI#f<9>e@WjMlsqTSYyQ6_nR0G&u^s`mvz(d&gzm?cWHI0=upofwD$6|Dda881Je@ zYouTGyx)`5qB?rf=>u!G2<3Z?8)0{;cr4X2UP5em+;(%G3SSQV%=7T@>Fk5{0*tY~ zh)dwo(uaFDM4K6HgMWu2dpMO3PssqS$)KrEO$;r;utiiF)u5l__m^=@EIHRw6fDHY zppxdKf?OG1qEsPUb;|GI+(2ihnFoeQ2IHC=TSa~?_llaeFE}@QmrJ>U$btX)RrDRcSBQdID%J zI?YS_$M~U9hV~VUHuw-_j?VLSDCR3Q-04edNhOFC^EX^TP}h0%VmwSJYWTf~`~)wC z;+WZt&oWl){;qg>2qzvV8ekvra0Nm71&6)N+py6(uz@Cn=ShBBmux_e=#BDmieK12+!H1>yii{JlBn zbH2ad>-X|M$0P85p8L7)>%Q*mT9z(^6A@<{&XEVEZO@>Nog~}z7ILp(T!SuOm66ms z1cy^7V_@#U<(c&FKH}Z!wJ~?*9u=;BDEK?i*v8*>RqiTRAMKYt{tB0r_%O}IgGW-} z#c1-hb}@=S_BbW?Gai2YsEIPOv1eYyDe+>!prT#DK^MU@@Wcmq4{{RsZB735>341g zM67~%%)7_T@b|l8kPbN~cKvQn+Ehk>omQDgUyRlT+_0#NtyEvy>>nY9btExxnf$Iv ztd`T-`&$28rLElt2Q?Gl%ZY(K*_lW{U$r?u9;^X`4{Hk)?Y9y@(P`uRug&_)9r@#o$P zjBeR+PFL_QWjP)4W%pUH=c*yED)V*A|Hf+c`g2bi$^ref+kYAv&$hpsF+CB>%tW7A zYfgXoC%PR@JjY63p|kqSZ@&Or%DO1y@3#g<{djLQH46)M`)WC#W1#=RC2)7OVx5|- zZCh+?ZQ_oQUre}kA8e*0HYES%_hW%8s)eJoX7m0FPva}!Pxc8Mf?7l+0r80)x1Kr| z$v?f&QQD9xw4mZs14G9U(|g|Sxv=wHjE3QRE!dvN(Z>^D$^2=%vY9zkxm_vO{?tUj zKDI~6qQ(^wa!Gx7*D&wm=6t(`AgAG3rfo|*F zGl43rSU#jhTOZl=gncrTL_V5>l>_%0MLc!SuAF4{FYn2qCF8Ci48>WYCCLX(QZCYM z+K+q6+Rq;tz%}aMe7=0^-C0%lvvb65OWn5Uwc#rgPe6(hs$%D~4x!C_B=y4Uu@!T2 zyOWYq$`lej0S@FL%?A6LNK zS6HpGkLEKE4eNJ*ZQ7A{C8FR{PwJ|JbiM;Stjlt9^02(0(z#vAwl3Odli_KP?TE~? z8;#krFW{$=v=(EuY7ag*qh^$O#U;CIrsljaz42<8VS8| zA9t%)a4!(G7(Vl}CP+K+qG#lj9hz5O7CReV$lIQt(^3hsG^1JJ-KTz%XkWr@D7X4> z%r@20Pe~km%&^8DTdQb7U1OWt*b6_B{Q<7r#?u08kAyYGpW#ns;jm1>h$ z;g5D=xX1e3XdXZUtX7huntesgRYge1&Q9ThN98dkZ|$5>%9CDfUIHz>Cp~UwlKw5! zOJ)?vNY&GSKbiT-Az%OWPE|7dol^q1FebuUMmap%(LOX;x6KNa-&S;J%lh8;!?6J} z2i6#4@3h{{n#yr{GONxO*(Ck_a6-th?4J}) zIiULPxuwUYeLLtogz0tH3vazXno}=yRdc8F#n2?J{eSY`Z>EtaHr%xAHy)7#vje*V z*Aqy?VQTM)RSv4h^ zTKNS${^(iY;G-a+T%*Fhq&^)A*O54pIr?!SGRR!_(cG6#9Ua=r#gq>(OBprQ3?-G? zQPt|n*fS2XESwhg^_KuWMNv=;E)d_^2VXc1zB~6e_lg2NNFzgR&Z0QhJvQ;l&bmQ8 zciL^n1h&|6Z0~0Wt(xdf1u{zY%eQ^y=SH58s`I{`oQ0xrjDzRFv*m0N;rx>}@O(zq@Vrtv!{ON$b773`coG zpTnu0WcbMO7p)`OFFu?oX=<;-1JAyGA&ABJWIY)6s9f7oRj%U6?i7y(~^L{SkI>DfM)( z%w4fM%sb2GX%{>HszjEjd?b*SeYBBIbuG>b|wBWlfEn`l{B^^ z)1{!(GUioOUvjaY-;q0QC7>nP%P@%-dkdB8t6!?x7@bWOnYBCOZm1NOkvIFi^HEy1 zhTAYmkyK*?4S{d}RK2qGz1Nx6)9YvrTR2mll7gI8)cC$sQLbJx^{CA*GuN-(d3#C91(qP2RjmPu()g98?X*7~}QD-~8Wg%$saX9U!j@}Bd$j(iCGF1#_oXc!K-(Mwt(}7|? zJ|X2jEoC0x9aN-Vx$tzh(7ByMQ8VC3ZUq&|Jv1Q<*x#Ga4MC!jM$%%-Y151XPQg#u z&rwEG^}!d=tlpcv<|NcFxtKcG+%tzUpEna>??=<^=SKXh3Yshon5W?@+$WLFfe#da z=N`2^o|aJ3ni~(O+>Z>|eLY?@3W_{BXW6=xD;6g?6A5_C!6up5$D;ZW%bWe>=Z;-e zpIWG5gxvo8u}n9phb}IoDUaFbwbB!3flJ6nUrTuytd0GfE2f>1sT@XW89}KWLW^Z5 z1PkjB)DAgF(L8X)Qzo?ztDPpYrZv7AQ80g_Ve1Q?HPN^Cy(NzgkThdX)s6}at3pnf zEvR-hmzg14J9f)2P{K20?IVJF%#mUjwyYXv2TX58dhS_i)17`Er5l2`yV&qLZut1APSRNPlLz^wDi8V@}Xu)DR*z6~2TeHQtF zwYq6evqK{%$C_S4h!!_z?(EZw6w}NOXf1!?$lCq=dG|Y-pTk+5_%j0+svpQg6ocG0 zekX>wER0lM*A2f#d(t;GA_;-O{36;k^-fnd7dBe$nS)E|ID>*~#6xNyTS{ri`xIOE zzs#dNX++u5WUq&ujzE0zc#KPN+6tiT-LDvbb?&3}!;WahFz_t;m0s7>zAlRs%3tpU z8h7MXTe+^oYuALatS-*bNR1wB^v3<~;tgTvTW#{k(8E5TG4sxclYPM73m88ef{m+u zYQ}~qJPcQdkCYuu-Fxs3|p#6cB5)Q%I@^!f&#LiXsE)S+gKWMuutUQbsD<=hBdxn7`gXmqXm{zN&gVdzuY z83T@y^Zd>m&SB5K%DwDZs5-eZ?@9XY>&@+o$=-FdivtPr8m8(=ZF@=?l-_M{* zvVPYyn?|umy|{Tc=lIo4bMK$@hCySqY-&!Pf~CeRcMXOa#$RDBpD~-7Yg{<0a##8s z@x!H?yalP3M|^LVPXraaTA7cMHmZJFM8zoL**|CPs!AglY;M@YInEut9^Z{;du#Ru zi0(z%JlZf;$dpkFNtnG5O>SK&RH(eMYJ6;4y}Y8A;y#cU1YZ2Ri`@kSYnsyDJ{W1CY2|sw9v$2H;+JkG zHqgn(ZelervQyK6wS2qzPY33WtjsX5w1}&8{K8xGtP?@6{HYsR!NH-pONIi&aNHWm z#^cgPExpOL#Xro79~*L&bZQDtwK01gMrcoil}b*GlU_(0!M++kJhdzPuXijn`=M44 znZC#j(%I&GVIQZ~J@Sn4bkV121siGo{Ty{poZ(k+hnI-*W$@P`$ERl8nk`+Z%`TNG zg${iaBd(M<8BLd$u`ElxX#kGF#a$+@{^vQ_eS;M@`co%$b{@!aWWB8X&<|)M2 z7p1%JBYdWqovh|g)vehkn-TarRI0$J#cKiK{nbabL_~k^OWg#~DsOAt-X0!%{3_|Q zdUe*qgmK8<_Fd!z%N0gye_o9$WUZc}xS(ZPcG#i(`XKkdnf95`h@g}VL)p(5-##)T zP9}?a{!RGlL*tGSzq8Ml8o8XMxZ)h~{Svd|)0nbh8a?|g!#$u%{!N(7YkKRehwUkR zuK`(;m>jX{OM{nmXPBz%3Zz|B&pm4!t3v}_V7|ueu;}Ji^&b#U>FciDo~_Eh5e||Q z3RYAv`y_%fclsg<#A`kw3*Ca_j15I)AT4iEr_g(Q-x+Pvjg)WPeA&tSRk(Ndx?i#y z#{|L2END4jXcCsqWk!)iSjWa@b@g9}j3Ggx8--IUfZaHw;hIG2STh+0s84c9^&w`N zn!i3GAd2Rw;LBStMYEWm3w>!rmd0|Bno^81;E;VLME(>w#O;Cp8r-b6oNVdFb}jV> zQ=x7R=j^BjQMwM^&%4;it3-E|dtn}=lsSk0oFRxg$wFlBCZt%OSkQe@DmKmESu=(F zNkkF@hygS8KO$cX%^8Vw)kNN@|F(1K5JpHkO5?6{(y(aj5O(p&1G@ptYoY&{eg7GF z-(vWL?xO+P(S=)?DBC@c$@t>I2YYb!UlnWnTWWX?10DbVvVDb~-;g*$+R;;F^%Q9b z(MkCJ;_f-(lcm2cd5;U{y9NJMu&Dh`y&@I*f%NxZ-dEXD4I$MLw~p8xi8BtC?-Uxo z7T}rPMK=6aRZvv`7||C6O#D4p|N3r`=-YCa;Gx-iJmbraEZKZgsiqQW{BfaY&;PAD z4XjE#ip^=SI9ww&`%vcDu+bZFCqnVF0m$w9EtM1BH3ZJ~MFIC?@v^r6aRFxPP1A)u zZ)r4MvyGWLQ4;p$uLWIIlvFG{ihZ}Cs=xp=)81G(^3Ury+7;=vb*`6RRciVTlOoeX z+a*Pg@C)=eSJL3eec2ltWR?bFb4kCjs|zuiwan<`YfX?%ra3<7N+yrgD%mXmZT&sw zz1o<`YTc-FOmMGds!{W~MT_t2tHH@9wtAu&eK;d!Oql!{E_uZh+C+U}i8Kte<66yS z`}R_WSWX6rTC3LVWUxP4C!|~Jk*gado20Xq4qvqDp)IB(`o@|E#Fa{3pMIfPH8;9} z7K3d1v2w_|sT_709FM-g4YS&y{5g?)UNxbE);9gzlq0 z96Yx#Fm)8(roBW;Fb?Sr|H~4&u^5!^**m#{^o2rf!5gcq^?A^-p{{XN2)z{38KhD( zw;S2EG;KPL(v+vC^E#W39I6glTvVFEhL!_<+7k-Ch%gA#5>FSq7eF_ZV93f!Zr2+t z4#~ewHV4V?%X0aBRVCe0@+_yZRcF$R`UOIn94y2*t(Dpm3Z+~P_`JpeUFL6Sh6iZ? zYv@5wD0JO3E1^qa&RK7LLKadKdQ&v%3BJQm+hc<^G5S z$N=vc#Cwi91~zos;JYGMS_7|ImznTdq>b#>IC9AgS!P=v_rE7dI)>((Yu5(y@Vn3& zWuFY3Cf?5bO?ZLi3ygIa>B|rEnsl-kexhW&3@@_l)T`JmQ25X@s2QGUhwRP6c=Dw^ z^%XhjZ@Wv010-3z*76bVs$OWogfZT4p*gdYdoGOuPUbbHb*g0|gY@#3=wW!pm1Be{ zYy?j8t(-Xl*BOpbQRn#Gue#5&gNz~0Jog}4APcV}d(nChe4@2si3DLoeQaVZ_TVzS zo~@F#NIFMhGV)*%`P=)9?E}%4q48=X=k>Om>uuWO?iXi-2+JbXdUXef3H?MqvZnHa5plHtRm!>~Jg$S889H*O+gN{T?`EhAE z(Y5#s?9~)akC7>v%E80>VKn27IZ5d7(2HGWI*RNS>1iQC8D~P$!mD?Gl3^D4TTg=s z4?Z{{Vo$w)Y;R!}Mp^mCMLtRGY{7Me zdd09f6XroZLN>#v;@aV~`q*E3&&Sf+Fr+o#E=?yl2-bd{V*#mKu(0cDrz^jWM)5wd z5Dwdi2sRl5ba~}fCWFZS0g5;%rsUG(qOiH96T#?SLx~T(PB_2)qmCyf5%?s30KyB> zCbVQEuTW3LBG;^u<>B{v$n6ber9|B3zG{ieO^@#q@ilFme}pI>*#;zDO2XeO0#r35 zd==0TVbTCpl-)uxqmuH_gsFJ7WPv5R&x#+sKB#0ai~tz2yC;WqBWklHh5`MVxnw0) zdaD96#U~U^Mo~tWVDiQXLpMG2BUS+FfqSJ^YD}JHwm>7mE-E)hXee4>K!GDRB%utCsZ$Vy#t?_|*pffnCb3l{5amzhl9wG&2ybgU1D1?eF~ zo+PW=!P)D8hwvor$1K)Hdd{}&nIsEmIHQ`0Z6Ov;^}{vRc%BM?rcY~fHg6i_%Q}Uh za`>8bu#0ngxx9T_&uo*IN2&es4j_4$Q9>p*+laDX)qiR5#H-O3FEz!v_L@?!XGEPC zx~&}&Jb^aQqWv+euJe|Lve)^T<$0`4`oPu`*RjKK1Xeja9x4*j%Bkg_*Nk03DbJ!L z0?D^ii`kclyKUP>Q$JsiOMUB$1C z^VCsAssxoT*!zX+NVKTB4R`sLiPiD-3%^~PdT89N2jR3?59B;fnl&&!i@S%^oZ+E; zXK2;-+wm#)%#{pUzvIb<)qhhyvkU-;)hxKGNilf|Pzv%{G#CYv6ToVtuCfCC=av%i zAbJFLDvJi+H@*+lB`?7$YrMl{)qO-3~x;G3u~#J*%yx6%t!vU>q^En=E1+Cq_w<4(G;S!8P`e znlVYc^lLn1_|ud09Ygm~h4FYqjx5T=u9nuOB41hE>0M#0q;KENn9@?J0@|_sR#qcf z5PM%}A``Z}@`mo5*TA4`U3(m*7cDxShdk(=4KmuX6_#~s)>LvRcz&pN6k6Q`(~>M^4%M?>wy<5bdO_=ixSMscK3-5Xgw^aLR?+A zQHAL*O)~Y-WVi5Izo8x+2p`lLQ9`m9V%O6qRh;x!{Cfw#O5}?eAF|h15g+SO$QL0; z*q1sjwB>95^PkW=n8%w?Bu_`i!la{$k)#URr);=$V|qa23bWyaNZKeOQB!xVF$O2} z4PK8sHaty6vFE9sBHkiK%nzRHXbT_iGvvSTjEKh5=lgeH#o>$7BYrKJou6dYL;bjC zT5T6{p~(CtS{Q#d8lE~d=fpP3S_$kjKt%<1k#ygpf}V?zEReaAPcC~J(inZQt7g|l zbUE>z&2v`zQ=d>e%Do12Cmcvb$!FtzC2@1Nz46pZF}z8wU)XkIr`EG}?5yW+?E)Yp zpTa6d%s6RpwHzwoE6lH^DWi2V!*plnvcSm`tC5L$y6?{!UiINh?kzla&p7{>Zum77 z#XebW{a%r&cAZ5(lGDR8#H{Wm-Pe)u^=MJ-S3DTMyJ?y8TSWL1%uiEZ>2mvBHXCG2 z$P=E-G+zl!UG39aeAYhZs8UVB8Epd68oX=nTkzjF16}51um4|`d|ss55<23l(vmrx z@d26aV89vXPFg;cgnoB3u=BvNGz}W+>ZZ;xK}fnC~3sKc0S)k&;T zg(rb02r3}XzEsH2usP?$h+XYTS^A_olGyZqnPRn(4c?>+yBma5A@=IbxEtUJ}D2ScSK+mGV-ZhZ2*yAS`}$nc35D&7eOv zP+NHoN|aC9A`yJCk59wVf*TONm|eXNAxbYho$P+9obrpE0y)(%4m%C^ja$~ zOv(Fb3SlA4U)b77Ml3c$A(yHyZlw{T3w9^0_s13*hZ;82Ex|#KH7=U6NcM!IF@k!f z5R-hL?aH{484C1jke5N32=B7E%o1YwkGZ~AJuTG1KDwCBli2IQWxUT4Z>%QQ&oVfY z>giu2st#8(?&~+Y7wRrB>3WY&hjCv)Rpc97qb4`}ddpcxh?gVp*2|m>6dM>;#>4#v z^f1U8E|FjDxZS4o7RpBaWtmx;G~n;y#ei?{hRnh8YiOZ7Ve~ibD7*=pIXSgg<=i{4 zJjr~xe>dcjm3x_yxv}mHbgC0I;i$p`)-`BgkluT)PfkJN#IYKSft?UoFWREJ0o0m} zuMF=kKR({~L0P(^nsJH{nUix5>n+cWg{v5ighUatJ|3Q_Q z$_)fKidd#)l`lEMCDN`n zzTv-$^Q*#Oz{G&$JKCKs zOmIG;ROIse%S$oYEbe5lqyq$kOD0tgT!gLshGwhU8ORS=g$15!Nvx+=(}Y*3L;GZup3@@)J=3r}@1f_qrwH8#A)V*JiGq13I6s}cjkl=wQdrsT-8&?q zBbJF6-qZ;>WR15NyGEIR>q1UrZ|JY6E4tzm!S`0LW$JG%PB)^lTF-pHGNb6@nJHDm z*6S58FiVy1I!R5s;)aK&YckNF(ZyT6RtATLydHvv9mkI-Q!z02FzywkeyR|aWUTw7 z*}#98xlAW|kA?|M<;e|hsflcMd;&=sS_Obry0xXc-;C(fB|2M=3>&UdDkVr{EvUY2>#9z8m|Y*) zM89qE;Y7S$C@$M?Vn`to{E&!q&qGymFW=bL9#AC=uYU+zA&dBPgy^AI1P6v zEi^EWlr`2sSre1UG>qs?A1BYqM@Imy(%)WBu?sfghO?R;TKXOAHL zJ;sb^Gf^rbjRS(WTjw{Yutr*LbGKuW?O2zWvTGN&U-(+v@um-lRAEn>R+6MQ4gSP(uH zlDzinsr0pra4~*g_IjB`n(rAvKG$hvm0fTiFK{uF?-Z$1J0KP4rHaU35*Srw5}Y_c ztdp5|BL~DOa9h!%1>@KG%T_kq>%Q9+lYD^@9!(w~oV#rRx{{2$MJKN?WlN{@S~HM% zB3B?m$+Grx-!VGaFic`b8-MSU;I_C6FPIsxqTgQfPx~sZUO<^rSd4s)p{S;T2Up+~2{tC)Ld1i_%9rEcN-ZBbl|7vDEMY^C^!Gq` zO%R0*==nI(Qy5oDDqeV8_=Nj_c~ARx8h-VI9rGy^+i^^Z`C_lT*Q8X*>h0+ffLuBr zv_-=5NMH@&L}v|(D2SMjF|V-^5Y8)(e$74DRqzI5YpnWAs(k9_;NUOuUhj?D@7Myz zLdroZdbz?fdm7ZCX-X9+5xelGXElzD*p02c@wOMOgVRdeKeBK2I{}5&s54Q7b!gmL zpg32Sv+| z+%J9SH))OUaX6CKR^XAS)C_X82DD>)0MsF_Q>W&QfZtx3J1-boxclmZ8WwaqIsUl5 zIJ1$sA<(ieE!A9YZdVB9Wk+5^fUgvTrXU1I-WeDUWWy48`{Pq~|h#$rbY5`h)Tj5w(Ir3BMMa|AHAH znQbv1t+bk>ZmHDXN8JxTXqw|Z1Dw6h{Yr}hU21|e1bM*#4FN!tWJh4e!ABSr4wDq@ zJD+vwY^BjtwFDMs*^8Cic3U^p{bdpYmr|jFhEto-23tiNX zw<5m5Trmw~J(j)M;5Cr`wfF_9DLqv7)<|zk66=n(%)r76bjWT^nU(IE4~|y*tPS!+ zTLmomFCO+WKMS*PGd)r6wNJmYzXf{C)X%&YBAh(5cvAJ3l(49D&6(iueVB+72Mr_c z*e?Yi3JbP)hnu){qvsz_Otq-fBNAbQP2jAfmmA*Ou+sgI{&&4>I_4S)b9((C+(+li zs2$1_=`*6c0CJQ^R{S2_nrE;(qz;TF@%hmk?>zTRO*21k94p`Y*vzI&DWhhMK3Aw+ zOH+0_*)P@Z#jvnbAf_uXY)t@`rlA0mN9bXV`{XHJ4^2!^6wc7nn~+fKkxFXGf2w0i zR18~fH5rqzNi@BgZfv-aVx35-qiRAzM^LT0^y9v=pj1`G`_)@c#P6(^Fp|0v z-{O&jIcxe7qX2n+S$zU(a74YES))THX(n%@_*2+tNHwuolLfn2z3tJ}V zetG#e$TTmi(Zw8&a+6L`5}9w6e{Xf<@|6cWefN`PyQSEYQ{SKMYZbgDO|o8;y)~#6{MGIt^IpB#Qo+>?3o~0obzvH zox>#m@3z{N_NvnJiz%#hWr-oZ0+xd|*v*tGkjO&P za){HSbj|`7WyM_fxmH zq1Pd~3~IUz0!%Bdnc43WUS*|7Qfp!NgoQEae}uh9Ak1rmW+VH%OIw#%Y z5&t(N_r~R6)Avr34a%YHdy(#Gy*`+oJrY;kdR(pB3yx=Ps7eFlv2#`NnC-eUPqP-P zuQ8{*!t66}p=-~0%1RJ>c}MmbZjxFzfcW)9eqe;y+~*R!TmeS%Zg`cW2YmwYO=>}P z%9)B0z|$_)a&JelXTrl*m7tzFZr%pNUpx%gs_!6V?fviloBsp(jjjFG$@)vnDKvYp z6E95f7Uv#jbzu}$#l8_7%sU`p=ZS z>km{dY5c$gW4eM#IUnIP#j7zlneVM8zsTKjz55s1sy{wYc^Y1}=wN`Vr9Vs)H?34W z=dxm%3rDM8fv-v^GEW{7j9V#hzBTXeQr+wnmRG`Tn0(*|;0OxN?bt_|I;`JMgksV{ z%Eda7>r%g_^urCXrg^*rO&}xswM<`8nAW=ZDP^f7(RDgX?rUpbEX%#+(f>CZ=!%@q zjz40Wf+-bE&N+IeC%Rxl^zp`D6~Jb0NzJQtP_1zp7F89F-Fbcg!#Pd9Lho{&c9m!r ztpUy8gReFaA+SM~M~1!W%V3?gp%U~9@_;3;F+I!ds}%tQiFs)@JHdNcW>%RLCmu?Q zkBbtuxVThTPm3=-Xr?cn*J5$EW6|qW6k_jlr053S%D>HV3tkU9S?-=yKG_k+lu>)B zS?6}xk-O>C1DX@l6AdtIp?9x8d;_y(ncaM(;B#Q2ac<+<8RKq?r&V|MfK>>?T@n7m zn=UXSivn0dP^%v#d{um)b7}Ojxg9fWdZ4i|f%ad~Z2w%3F&|^QT+w;ItmwICgLw8c zGnCnpcSU7^ukju-UaQ>I@)wn88$#6a%pePc@*z4plJsZa>(QW?0h~3&3FOU9Bi}Ff zJlzro3={hHp`C_yPUbr<4xvC9UOz=xniP^qU&_{}W7+q=$6LYjJG{`P=jbirwU7Mlj*rPHL-{ufkK^tz}1 z(!b|%{pmF`F$jfZnt68e)K2w5($WJWbcHq?;We33uyLv`dTt8>=}q4nEA;9iFFgwj z<1*dlk=_-GXiY`7TbQQK;+6KhB_t@K0*8Ry7-~HcQoDHxEH7z^v3g^%<=0SWv_d>q z$Fv+RO&VkOrWsMsrs4E!4_z>WSwL@^CJ{G2d0fSL)5g?gqJ@lV58{c0Gm9H(qBQ!B6)-Ibe*RR*Ra?b4Mdr zFH^Vs4Jgc)SoT6^ai2W$Oa_-U;}rf9$&rirEur&(3&13$VwMhEW~+(Wdd~zjxTD(A z3IARP!@+cM@7L1%$m2tn8?PJ|owA+OWPmS#f&f4Oi2ww!FKvJKIko?|0DBlc^)jPz zkY;%eU5;H*5A8YuFN^>h)DD|K_Z!Bb&z15cDm5_l^xD2OhO}H*wO1swr8I4bb$Kqj z5rl2vz2$B!E;P6E6a9nyWKv2Mq54sQrzaVY$?W>h+;9H%eOeE3-&7^}x6c?g(48u{ z2B0l=Z}l9u7^@vX%9d&elJbL2x~)mSe?-3)`G+b2*6ZH6y8_-NZ1nwUS@&ZqM!U*3 zv18phJ*+w$Nye%p%9%8%Q4tQ%S%d6H zrQq~$L1<~o)@BQXQjCT!uP62A@(Z2qDPblk+V~Nd)Wzww1Wt7wJSe?PZ>s1WzeDBIeimlIa(+3p6RXYOs8i20PkqbD2u!sH~z*8vC; z?>T37)-Qe!SARl39}{=oI1~smM&iCzCIORe+0F7$>LBGB(ytdJ9?{z?W^KTqxOTSm zYG&=~do{aKEZ)TLx8Y^TGZ%x+29m6g#D*bJW_-1By_ya7{<^M7E|gA?PWaym$O959 z)5~J{B<~)KWReouK4^pIy0J2ys@IIgtco;6`&lh}K{0?Z0W+$6L!TN$9)QnN<1}@> zJ;|7FbxEs>4;yi%5&O0sRTQpjj0HU#z*f~|MNs?3=07QW|L#rBPDU?C%Z4(VoshpR zK=)evGrbTOYllm*eAlUKzO_S;I8DAVQV;b1*;^d}Ii`KhWW_f!vr9+|i_6}f5oi5MJ$kS%Ke5KgWzFaMwPsl1zV1_{JHWe&*55;`4{kHO z2E;8ns^^$}Z@~9l-iA&LwL!K*%#9{VxO4Uq=A-3(H?bXwk4hSXRIHIdmKO(5Aou}e zj2Qnnq7eQSF2HgAf#Kc=spp)yEcS)}4rbcoXbDlMmI?D_2MJ3IWrA_W*f^8SijB}x z#SvD1=E&-U;cMXWdOd)f6g?ETt2cU=rT(yCiO!m7tk}5wmWt2HBX>{+e10S1_?T_h z;+pmtqha+Yw#a`WydJ$ZO&x7oB|;v$8_1JDIOXKzmUnxxWA_8Bg3tDp+(9-tb&gA; zq*4ESc5Zszm%e=TxVZPtosFlfRKD za=F$oW@Qn)1|HE0k-I2X2w>0JRSTW~uLRU8VO9}6xlv+eJO*#e34R8>0vW*@g``T% zCg9im_d}^k;P6Z)sdehQaYkLiwWf34(BUI`RS4ZzNYYb|vHzE?cEfwcHvfhHhi`>o z#Rz`qq+O#<%}{5u4=^6>6W+b^x1>J&TqhY#;VB&YXL(tZSMRz442MNQ7a~V`?Bxer}n+wCsaD<5kpwBOA!R(tMT z9i7;vGw3>~tVh?PljS`LL*|o8au#!fDuq%?XxsieHxa*zR2cH;dVwrJ*M_M z)&2v}K#~G~9}#LL3rc@8L46Ch)fK@;0PTC9?;MiP_dZkXx)qk&Zy~~mce6PFfPj>D z886X@r0De(()N>8H7x~|cO^U=u#Op-vRxQFzbhP1SJQ`ZNKmCYA&2CtwRaNX9(|Fr z1#(idtmDC}id<%##0VP8o6Zx%AQ87b$b-^N_x&CH)DQg^0R3#|>X&(4%L=2w$dU1P zz;y;5R9Q2+_)iPFM{&t7HctT7nF(s$3bW7&fhAU6*a%fgehE;k94Owyu^at;*DScY z-2to=u-d)5LMS%*iijPsKeTqls|Hoj$+G-zrX69;F4Hn1CsG{N){}y#asE6N?0fbk{l>Q6_pmj6rXD)hRKewxtjmr0M8@-_r96-B2` zyXH1o@Vr_B+hi2R76wqk>*X7*uMGOLsU!2l6pLUy6#ha-A0Rkp9}Zm}YnX9AV|>Qw z{-PBf1sWQHE+0`C?ze*&t>I80;sNsXm(Hbk;8ClASJ<;`amCmcA=zyFC-*4{x?NS1 zzoWmc|9|a<6rnWYVF?*2wT6L_!vCV*zpt}PSEMW5U)mA0;P%bcPOt1gaOa=RzYE|F za?Ykt-(eI+A^+b)0i+2fdPVUqceMZIBL0Ke{u{fpz#^9Ieh@{|Aj_G=-aT<$w- zKlFL^&=JV?zI(UUV*60Qc1q3$h&4V=0d!QZb+5w{y*uU?$E(-r4FdlbJIgh*5w$UE zjWME)LV_yo+)FaRghBBE#D_}*NIdKY2nK>43@$Jqokq;i1!5^Z@?(tWc{&76C&fC2 zvA|K{cE|b{*D_)>NAUWW!59Q``j~PxiNH+qHC*`wt>PcXM$F-;8GyD#kHGUMPoTTf zs`qN!74e$nt{&d`my-CZ!xR6*6y{fVBhO7(C?BvwlNdqp{%HvaU@gjEDlQtbS|o4} z|4k$F7s$!28HMb`HGxh_kHE)fvHcn;N|CN^IVsC~n7Ce?UVMOXuqktFHQ$U3Z#h?C zC<>X$bg3G8V+V97N>To#F#sFUOI6nS-jkH<6PQWyGF-XBCmkyj(0mNDp)4e@-#7lj z^HX?Q8-RMjR&sb2Ug7cP+qbm=#-_nbKSG>jHxmf#gibIrSD+YT914URe0W29I-6wX zM0>12Uz;hIp04JFw;GHwJtMdW-s+ZkCLSgr?hVL=Up)PtxfwsWy@}jjwh0E$+Q-;2 zrbqC96jy)D;*|gS{f`|SY2{gABd5O~g8-jn13nwI&9+H4Hzb{gFNMrHZvclad{d>u zH%h=<8h5U9Ml5L=sVcVzi`5ILHj80Y!Js3UJv@M4uaYz9BcBGFHOL&_+5jB;C^~K7 zl3NsDgmfLl5-46!aJtk=8XH#Qn;8vP2lK3TPZdS*sLH;0J^+FZ23+0|Z`qm#SXK!% zY0M~<92*_0cv8BU}0d#shq&)-9d=Vtz^e0E*4z36)m`9U|M0uMb?x7`<+Q!#q!WV82Ficr_sQHi zG_+l2?r>P3lBIOCse#@$*~+T`7Hzi^gKNl=9ZjvoBx&dqIGKcM^?jD2pjy+5jZ{n= zQZRmHg}AVWXB>0W!w{-{g<+SBI|{eiiV1=OKl0-Mm?4IwQt^OtsF0%zrLjpO5g72 z$=J5SEq_Dr*9jW8sVs`1Uw6{~m3JYZBL33!Zlv>$a>-_yZBpCfI6G-v_v~HXKe9wy zZJ;xdvH^dtoK#^>^{s^(7;-{Lc~F!)D~94E1uKuwW{aBw!DbHWQ!8p#eH|cNtQYMC zwaz3v)T|zd5=SlUO5!T;RlRb?-e=Aw-#!QXF6X6Q^ zh6XxyC#8HLPWPK<>> zi;=L7W7(p+WHpMtEbrQrjSJ4XP1zJg$q3ar0|HQH6}prsx9fEcf4?np@6GS)K}4#X zlMI4=V5OPf3AYW7$~c0J0$R2O+RyWL&!p{OrVBPe%d>DWsPB49LGmazNC&(s*zn`9 ztEcaA#SPMw8n(ukx5iXFkC@ApNxs)zlM6&9=RvntnRk+hwfb9|<^-9N-FSH}{FEcp zkakgGSjJxLJRf#rXsvBuR|@_|Rm~mOawXXrDZPm|EP|1~tXu_HrW4Y62SzJ{O-7Z!&A5aDK?>*G$chF#a~v zWtg|3f)%a9_5dd%t^auN&TAc(2SB-KQseZ}XLRW_=Io@5VWpwub<;$4b=%;blKJwf z5Fe(Ys*l4+gP^fT*|{Q6MY4M)NEH8I2OQJe~Q@Igf*>#zrSuMzHb*^sdQ{&0q z(7}bS!1gI0|6;WV{#I@GvL>W#=%rx~Dw{e-%1FtVBnb`#HnYE?#(m@AQg#l7UZKVY z8jpb#TP_gts*FT0OGXD3T};~`N+rT}uZ^zSm~-OB^ZX4?`$>z9eQ>2TF7Jy~s$2$S zde3EY@DoEAiE|ON(}JFL#ipHC=gf5Evi|L_RR421Cj(8K)o;K*m_`0MccpD_n&O3! zMU@>AijJhB9Xpom05P1&u?QHb`B?CvM=Z)xN7)b7Fw-k&G<$wPqsOW-QkGvs z9{6Ud(F?zi9PkX(av3LnfNR3nnHpx%&Tya`7op~1zZ%vt6IIy8^m#PbT-_WtoZ&5& zk=wC1vNbS>aJs(`o zW3dCF2#ZuC=aUvns7z#?bG+((;vsisu$ioX}=thR^tVes%<&)M#kCL zrsE><863|hYL0nPI&Z&k53JE_nM(KB4zbzwOvwOU>tLQiHS7cfa2)eZ24(Bx?2*3?#q+wVZa;ojQ zeDbfu6&O&!(m#aK%-Wy%P6ebVdJBeH#zghE^2NXOJu?mOYL_gZ-^*{(8$qa^O?an_ zFN$lm$j?;rHa)IH)P_h`6Nf_DB?_C~0?Rr|ArzfE1*k>jC#m(XG)McAsHSq6w!HQ}c_NX9+^m-!*M z;@aZ_yu~N|*uY+cFEBc2EcAi+)?cogY05bp{)=TblYZHx9R}m1tUqVmVm@U`epm3Z zl=AcWqXEflMUDgw?R^rsOEY#WaK~r_FQyQ4MOPZ%ewfzP# z{LL?A_MA`&YUe0H^C-+`20pmBAac_AeO>DLQt4 zf9fi1m;GV(Atd9{Ceos4TzDjAH2G;lcl`2eNU+^R{0Z9VX+58tkB*c-4VnHdcM6;O z4f$8T&Y=udcUhXCly$rAFQq)_6X*Q*JK=#sL+J|o`bRQVolv$OksH&sQwEcDv@cz% ze&~hx3Q5A@^sm)sp%&S$otj|Y^J59EQ+vlzVpEl8dfPRtJv&5iQ-3I&~{*1^Q_#)?>?J!KNjw{((d-WwzThRm*(rqQ64j4+;!p9kMmF|2D97=S{c0? z8xaIy=QedJ9?D#?j#N-aXjz$;7`6UfniF$mL|K?spoIqfdx#UcfwbJ~ssB%1XX2K0 zy2bIzjHVoI#x2K%X;CW;r<}1=gvnBK#$d{>OXgDQb;%7E!UQob%`Ge{D_O=gt+8AZ zcf>T2&=i#>+)+ev1-DcLabT8v>$%VU1J3)r%kz7G=lz`TIR_SaH6I45+x@3%hPvoK z5)rW4ag`Pom%;Z-#TrWPvtEMa&vbCwQk8~Dr7NWJvKu;y7jk0mVb7LMtXAL0S3j=O zgQ$(p?ezFur)@)^RA4^}Ef58D<1-j_=+cut`^*J746h;G{PYH5I7*t<%j4YkUnY>X zOz9V@`91#S4#-6gJC3|`m>Qypd0SNu;=TgY;2SrFE@?DKU-`BIp#zQqzO#tmdJb4* z{s5hLCt7mwm>Xby!O`G6h!llf4l)2gc%VKG(*O0z*%QaE&dsrEoC6=TBL|#KYsYag z*cc9&HsTt>%WKQTUA_Z~9S3gdJF-&zGD@|c7$!adUGKBnn9Rb*e`_D0jG%R%Nr~B2 zUl^9w(y|3d4n@UR)@%{!wF)yFEwsQZ1PHFI?f}t>cACVvPQITBON1RLRi`^nf^a1f zhS;OpZM#8s&}OzWBMvI`)7<6twi3@rbUZy?kj@m;Bvd?`W7+y#DvL<1*Z!yp8H*M) zKDzysu7nN6<(ux6#kY)ppTcXiMvJ>QK0FF+boq^X%f`GbZ4=H!*FBs$q&jeZZR}9sO6l=C z<(txr7;y$)UE>yQ=>nrZCK|+s9&1UQ8K=1MR|w@1J4yXkMjyso&)l^58@ZE=H8=njk8bR&fbk&egPXVvU*0m z2utYmL0ZK&{f?7+ID!jy?=l9dubrg9WDGjpy=KP;`^9Pdjmd&~)-*GltW06ih4`s+TCUdBS>_$Yzc04`u_XV^5b!UgQ_3 zf17zYXL|yD&lsH!hbR{g1d}~v+HF3F0nuDCxztkaSj_iTQN^QIU>})pOB4+ZJq)JF zsk46UgYbH2zsc?pC+qq7-XoKKc@pJ@!IhTo-5=+3B}IBt`~!KBp$?0SWw&5P(2UsI zx^nJK@(3drR(qcBxOmU;s5%;Eiz?kpHsLwU*27ZoTEzutdyA3o6I3aMlc&WB-E_m6Al@j`RL2l z+bE`e4ulSX0V!}^&QUo#b|Ztd%w=@l5HGkf4(1T`4Pjvv5QY%NAj zNK=o!)pnrWB$rZEN&F28_7-4$zchE=3Fsnu7bOKZPbGc>rDc&)u6r8R2AsQ+{ca@b z@aaq?E;y)3?xu+?%zP@6Oh^GhmX0~XG^I_XmBc)TU=k+v@{eJ(O7I<-Gko<&v0+cA zv(QHa#af_#XYbl4bX^r_Om5660KWI+D^-YHj_j%LR0y?~jwh{ZQzK)sM)Ox_9m!Re z{Q|`5^KxVjIQ9uKH5AMYE`;Y8I7EliCa>OTbD_aKC!OF_5&50q4AKZn{%K(nIIu^`cb_dX;w~?i@vaFGgJ#>KjvOQ>(PO!4 z*h8pp7(jCPsQQP3axn>+C)nV#Nvp`cc(?_2B=0F8LyZtk(c;+9ZictQtmD?1h2c{L zCTs7@7%w4k8TYqzdYE+`St8QrP zd)T%YXN<1RO38xt_o>*{8S#WW!%inDz598(k-ODFSmNDUd>&po18cZ`pH+efeZQDk z&O?VGNHuZiI?|#j!;f0(EO#Ko@)xrp6eby`?!e;b33g%RP3^5#BD)HOmM%-EVK%#Z zN`#T?%Oj>fKq#-@upP$tuX(Yi$b;fesfOTkyaghpqn%CQJ~yw&AJZLKDnu`xj93UF z=lhD@+{?K}-=p7H4|PT^IMGmvW3x1}(`s0C0lWx?KJgxX!AKYD%JRENPOuYnO zf;X;Sxx@>IV^yu0lzavei;e?aek|c7kYc(#<~f6Iveoo0-WhX2WwP=j-q)-U z>c^2j%4@+pmKyh->w305x+iGH-v?N{A}GSFuz@ev*c{z~;cYnvk@H@sH?n7AWS>yZ zHX{Hi=emJuA1L`0SEM)9@4loal`dB3c)yqnm#inr=))@umx$K?z;?NG>XkegA6%;K zKT?f7HS4{unQ0~$f>wEVDRRa7A*18w=OJdlo2@TLPB(kjP(S6m1cshlqYbXnCU0+y zw^Y3CrD%AqxPoY>%$GBJ#+R~ouC#q^^SiZ;V)NFbokUKXthVy+tn+WOl{nR#Jv}Q^ z?*3Mo-k=~F4jYTmjdIP~-m6X8m8@stAF28b1-4<-E{klvCJ?5Q>5KnS&D>YwOz2hH zs|l-L*Y@D6i7i8}QaSNME-8x9Nv8GbBMNUdv~Yfp{6#nq8>K@E#<*5hpK$0O4greK zCe`?*HN&1DcQ&e06oS!yZ()KWsDq=)v+;l!fi<@`ek;4f@73gmh_Qu#D7E6WA`GH@ zlvlvpRw#BMH|4i-fQ%|n0~+B{eWaIex9iSQz`US(IW&nUYqLiI(07k#tMbW?Ft znSQhQ*_wh=b}9BL$!fHrQ)VWa*860Y*> zBViS;u+N4U>;bs^!aDg2>-p)I+qYg>eA#r)qYO21cA0)Od5mF2{&e{kfZ{SG>%yj4 zTf=MCndNRT^bbG8N#a+P|Ie6a&lAskbqY6a+LYjU>csh&Of2#53J8Cbcpd)h9o;%x zgEX&CvoW8Y^%zM3?(IfD-M>G>u0c`56qW3_f7Q1%2u(lxOz$fd{pMx2_&*kta-pD7 Y{PBIofp4!Jm(OLBqaEy2v8_+yA5eVOdjJ3c literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner8.png b/docs/images/LogMiner/LogMiner8.png new file mode 100644 index 0000000000000000000000000000000000000000..1cf9b8b6a8a6b3dd27873dbe35ee9fd06e639eb9 GIT binary patch literal 110133 zcmagG2{@E}`~P3JLN}q5eN81HOJ!%23R$MezD%|hhHN9tj6_LxvSq7??EAvMgUb9tTbbHeWFYo7d@=kFs&j-1ri zx@B&Am;M+xcMC05_asG-NmBSGq1Ofh1Ab)6Zw zbyiNV^0}7K8AE%&)$43RAAD{;KUR98VOzk@yG@;%bue;uS534l)w8T=wwc_p%4%`7 z;!)9c=SSZgwv|2S(>B5bsT-M-QV3GGC3o;cAudMx7eW7gJiKNr6rviQAc6V)Gum(5 zk)gkd(Im}Aw4SxL?d;W%W@A1^*QEaI!2L8Z6(3ptc(@Q3*Eir+g5Tn=@|Zje6H{jW_Y;17>`&2#7NRPS|7WoO z8v6Z+AWcEISKisLQh~c&EAr@Fz4+L}krKqn717voUwlVuTnV|kvn$KOyW0fa>E!es zLs`09P$>JlPB00jw}R6v{v=?~V zcyzLA%o;c8yO2IPVgi{Y)~%v+MoQ3B%H+6X=Hw(|k2v{~(;B0fBI}@j)5OYq#Mq}@ zIFwoICRtD5;|=pUIGgrk-J&-1ZZM0|=eqSy@jCPY(QVSm#A@XJ=t7k83MM^}x;uHC z)3QI^vsY2_qOIz|<`Bcy=FcC&tO7BY4tTQE);w2g7GRF)-o_iBSUb+lxVLgUdR0$& z==e-}k9ngH4_1}6hEIs5^{4xM_nmx_n?5@IW$|OUIXCy%Y)3ppSwFb#rNXA(ceU7( z5ZxLv@=l)8JXmq=EeB|(Ng>U5K1GtAqToK=GAf0Cy}Xze?J7GXO>e8T@@2*xbhy6& z^6_77@Cnb{ip})H=S^~0xX8rZ2TyGS6L14W3Ia*F6co;n+?cN)Ls4f04;ORvQjup8 z7gxKa&=YXVQtsr}mrQkgtLVV~#Dm6A#HJ2E@lpxYBDWAPz5Q?%81s_a&X3@gc9H1) zs7zkB`Z7=Vh$7GR>2N6+B!F~V-tBw+jkGv7FPFCAExkRDuI%R4`*qoe-yp5$>`wh_ z|4ZX#*7`MAPr_cE4?&>qvqYM%^4?arsJ>@H-w9gF-jX6e~;r(Y=BbD_&?^y?ik%l>BiUXZ>+u#L*vplF8UQ!jXKp06_lX-#4G5SJ_Jh} z_OV=_J%|q6pLwpJ{np2eYzfY_u4t)dZE1UiI-RQEk`>jKXyH1}AGlAVLr3ZGi|k9^8GfR(+ao_k$Usn* zf{LgEj!EyzrrD1K+&FI*F;Ft;Gu1He0_=v#>M4Y>P-FZ>J^XjzFsu|+e|M5wMeVur zg4cqO6nek>TSx9+6nXg8xT8gV zmE;EHV`1RNXo@-5EJaLt;YWcX^@%u~+OqXUTUFqyB2CQBVlTGf)x$G)8?3N+Ld-%R))weU=TSG7{ z@h6pn$PMb=s{B`L+$gM~N6xv--HVai^XH3DxiV6GlHYLHZJuxeSf%i4wCa|r_(n(5 zrzC4^*=Gu&qOTUcKPMGjCM|%?gTFr+)vvPAeRUF8*wJbA&}Cr8iKc5YnxWuJ3i-$H zQ=0{;{pkVg9h1d%E49PP)6rJYsFp9!n*5L}d5XnCWmwDZ1uXD9u5lt9_z_bh2kQx6~kwpSf{(}LXQvi%8Vnj3oq znu1j_kM7Ui)K#5-g;eC3hu^g-j=1rU!vA- zarV)_di5{Ae;#t<;bj}mD`GGEpN=MpoZG40nGR>D+pRqQAs=dslhU|rD)cnnYfziN z&1+LiTJf|7A7SNYea((nl|>X`>@ z##%CtUn0APF_%&ol9@c|n>2g*$}0LkHe!-B=Oe+YjZ&l!;=A$}9@&zTVjee_T@(l? zSMH7FP&=^$Ou90MVe8kA&-ad4t@ZAxzbkU=Z4%(JcQLzmZ|gCaUkn1Zs;-)8KJGJ* z+~Pgto4l{jUsjH>KiZQk5?~VijUiljp@;gE`Go5H5sTIyC7iAjy#U56Pr?|_M@*XbSyt37UAeBrmLDpJc&5%6^MP*Zr-Tejh_yMY<-Pc+mkLvW z6A0dzXqisB6M1{^epm#13#YOwgf(TFsWeTr`lQxLcQz@%fEw2IXUj#^EViNhY`JQC zVvytzj7{HMRAHvNdH_k;t~FwHM+jzH#wtG*;3?9WQI-oH9k^19kLO{k>jA5fZi&2& zQJepEbx#5&SarhbDDC3FFtKYV-kgIS`G0 zJ6=<4dV!up7gHue#FVn#Y<U}Nm%3P2d$?>Q zf8APzW1m1o0W#t^=8-7>7Sqw2#a>Q&c>bW zIJQtS-sG^rcdq%{6$$o}M`vCi@eR9BH@v8RIp#*G(99^Sh|KA~8HMa$PFfO<#Ut@% z#l&)v&ZLWG(&{ck9xG38+6ocU`bO_S67xcJg-c|V1Y*Fpw!-ft#VCZWX^pSNR*`4n zS6~#9QD5n@PCCM=p1?CCNZ@wtjjT{8w~l-CWNQj)cXmECEg1##h91Q}*$C=Ah!5`F9+GvM8bMbIq$ZrEGQ| zUKM+OE#u%&T3n8P>;L&XE7Hn)r?O+x?O=pjdSvE{-qh&oap_e{37$o%va+X5p~*uf zM05SxIJLBjRJAYLFMd8$0OIdfRr-k=a~Gt{K8J*;2rg?3EByUJpstjyRq0s_hZ-`8 z`6dEmM2u(X%2pPDqWM;Ix;Z~W2Bt)whQHcb%!=dGP40XQ&W zbrgrt^xMxJF(ez6Y(nGB4AX1(awUDc@Sd;Iw=fr~5N2Z3Q8UN%t@OXz#b%S9VTgf0 zUvkp7`MZq0mFuv`;g1r$q13?}?~6+AG9_&!w99>eHZw}!mZf6*HI~`S^2pCrJ3}^Y zHAx$WOnz8^AY$iduvCG^aEny>=(ES6Iw>=KFJ2+ahx%Rwfn60`KW4xxo1(?Yyx3@HFW>iaVN^Miozo#Zo7j;xo)9-#9jA% zSq8giC?u;1lqvhZZ-AYHkMH`Re5QWsj1F_mBQ>KuD)-f>&ZlWPQ#oo8IMbL4Oz>hw zL0_`FK4gz_;`qYQuD3wz3k(I_P1ga-o!VOacJ8?nZr1;7CEc!Obf&rece7{ra-i*3 zpMA0{Yg29cdW8RAJW)oYwW)p`(fO6`X0#=5fUx7f=`}Hv+{E4NU~nFnd|SoTolfze zbV_vuk?&WVJlFVusCwpaI;!mP^jAXauY|4=)Yfm`L_O$W5ww4$;zx_+MXq9QqJ-yT zc>7Lk3|{io4U+oleq&uVS6$)-1bCp1f(Fi82Pj(g#96Zv_;wk`RY>hyFO56n6*OK|7C)QU>Qj~`6cF&A22 zZA_%4zv?#mOVlRx!645Hv7nf$#dcS;Y97=(n57Z1YJ(yfY$iI$XWn>dQm`hXL zN?Tf3ydOztIS8J==Q~*6{husf=EYIirR6BT^KY$G1M^u)U? zR^a(*$%gj!Rv~Hf4x53~Vk}UFS{YqRn~K|gg7o70Lbs{2^O8i?C=ERy1H; zT(kRRxN^XNU;ydF@$(aidx0&dG_U#F1(^=fd$xXz+E{!0Bn#QBuNuxl$PXCOlZPUw z-ypwS+H){&dwr-Pq~H2CKSvuSIATZ>+T?Pme3bek8htCc=-@rNMwb4!O^7(^gSqfb z|KdX7?i?GsQOqokQOjH?X1|a>Rukg_-Wa}8$r=bwCE;PJ+CM!cJ;3s#4 zp`3u>%=;*&@?JB~10K3LDh3a+!jF{cALdkAY+$i!Uu-8L!HV!1c78_H%*M@`0q|hL z&W-fjV?R(#QNc3+jp0h-dg>U{%PPXuzIW_sN_MbHNN8s7MtBNE#t01=bxX)9K!t?( zc^7FK+r6Iy|^7m&M%AP)oYy*wp}^>$?t zT(xtUq^TER+gz-IzSb-%8SvK!6xGF0*VJvwdEe+2zk>s%U~4qbv3v1dE{JdBC*wP0 zYj$6@Z_|IWN^u!_%m@3wgpZv*0-?0GTgcspoO8~oy&V5|ai|;`>o)>(X@0=pX-OH7 zB3D_FwMUAm34o8R>aBM-MOw{aN4Gv-y+|(jE{!vv>KlTQ4L9Icog-*>)}e~1VHLkb zvW8OzB4ri#7#H1C_L)3l99Wg1Vp4UxWXdP-P9?V15aO_oEA-Q;51}sr8c1#7?`Y?6WLe8iJ zX);s(9a&YNG#dHVn0Gxa1YQsC>g>9{A7z)yFO?}**LtvqZt<(A|zMQ=4% zrN5s%TAL4&i0&@H#x4xzv^rU#t*x|`$z6CsIsBptC{ddqGL&yfw2BZaKuL-WjaDM# z1FQ{xADY)k50R?z)u;8Xci*1dNZS2RGzb{0hYeT(df(Kcm0# z@IS-)7wXL70v?g|%%dD{i869*6`MKp8-v<$1J{HJX7?HXArrs;Ho}%Uq%?ZEgS$)o z+oq(l@Siu75d42q(0HST^&+LK`rNs{2hP>?|Aei*rq*IPZsRBZc>9MBcct0B3Vh8t z>-|p>`3v4QUTRfm+`E*KR;ZNZHj?7rsqml8-_&RdT>HPje^yZGA!Q70?PTent>@4b%h#Uk}qmyDjnt zl2=V_u6B#A=TB|lCps67m`wSM6zn?LrM4DXLrdnl>Ff;09-yNPA!>z%C%r{>N3JXl z?({sfSNV@WJrt%#xPD!kH7?I){tKIAA(t?l;v|0G(E3EOtk;@d&8i{f`fY|mO!W;- z{~@Pbl$iJRlHyE0)8b=i(Qsaz?6+RjHhl9$11f(Cu}2ZQ&Os<0&<7diwi?2jyG;By zkU=O}-0Gej%bS@#QNq{|04cbMkB2xh_>k}2M`|etvGC;jj zRVe_!Bt1Ve%`{%ST&yhQgo9RF0n~TFo7biE#!!~--p>0X4OZ1k3pjZMJ;Y-FY_(zA z6&CYdgGDj>8wTR_;NvC5?2=U1+9y5$Df0#hvdMN5djC`Eb?4FdAR58<0a60wG5}>< zN#ol9lFm(4@m&|9s%}qIC@&XQ)B+&;acOX$=_%!dkAj?#0QaE}kR8?C*?0?!MFMJX zg${!G00MI4ASuc#Y9_#sSGI6&Zfey4&6hLYY*sTmO{izl*_(0ydP20)bV-z?u(ioP z?iRR*8=W7{Gqlj11~MV4c2SBsyUhzUWV9ZEv-T&BcoEu4 zlSRZ?W7vpI4W}%RasK;AA6{jSp#SQ+AOtymnQ`d8CjUrJs(i`bdY4q4iM8*iz5R{8 zI)J}kmjyX=zIAPSahym2Q1iRq{oRc^fEYs6d4wlRP!y-oeH!Icq9J64nqVy&`|dfc zd@@8&%<$+G(?bH7w33sQ;+Qc`Sm;g1VIg0h+UN-TwdxhdAg1kqKK&8IASWonk{<Z z`il3CFxcM`SVC)&${o9bhXKF>=vM=v`C>Ecx@cxV`MC6agrr>H;ocO39QI>LWVyqq z?n8F4%7AU9TZ>La_a1;Kh7PFP=+(!2O$>s3JGtq~D_^q4%f6TbDHOe2CO{;G4l#Do z-iQ}#Kqh>>hN}H!CQj#yS?6lGtv+0!_qU;31GP2w>0vH-ADk&?yb*e)d{0+ZIhUZW zLBQC&)GgQ0O-Z-Wx@_-7*<6ZLFIQ%+aR^NadaDKTolEGM;&x0M(87*dxT#_WG&#$^ z%tR^6@xwlmI$Tv8m&R*eZECE&j{PDuy+^Euh&+#(0g4Cp{)Y$KscD$Uys8`emtpNf zVQh;B+f(SJ`+fSd?s>~HNvUqAnx$d@6;I_^*hOA9p9JW;xrBsnX$kSBsTrU8vnNyT zj_x;@Ps6amJLl}`*D3%kQd#z8DJPY_#b={O-YxtLp49fSDLF~r!E)d_2k~*JDGm6~ zTPW7;b;oXWPe=EH7O;-wiL{k99mEPsWBO}c4uYjU1XYp?0Yxm5JbQzV@PE~`pVbGwMEm1wLsbT_0ztSDg0 z3!IJBtR(_yx*xgKVhf6C*kA9O1b%B`UtIozzL$gsCM=ptUREe>cD+`2xEU@YfccoK z>HzyZes)ZNyM;FuL?uNBZ((+Gi-3Jt+ac80-2uGML}$XV@nO4|2{%-tZ%)O=pI9}9 zyv-ewUQQbH;qdGO9R_T4C=7{mz$%+$E=^ywF7s?W%24TBDFs#~KiIq0tU#I+%oSum zT;G&Y>I!`sVc076k5k_{=*hG6QFjusp*Y5~kY}@)4qCn9 zHBZ|^p$pYUk|% zb-&tB0l&|4iuyOxHiuvXP0`8(is;KU&z9F#Psp(YQm{`%Cc8)h_h|-MH{Ovr2#eP@I0z1MCzmt!y<3 z7;-EsyQAdW+KD4=FriYute>3L{h{_ed*v4~7zJe0*B_8Wipj6y9z>%|Uyu#Papu5t zO5zL`?}&3)Bv17 zQQ`*%9eC}XD=)*M)ZX%f2UR`?|3?t;!?BrWdqYo&Xt6IO3FpK}h4FyK8G=*@rOiqR z*HgAV_v!J%y?pM^4#;B+6t?RKt2?J#g)~-#zt5_kZXVzeeQtM(tpNx+iaIbxqAXZL zhI3Kic9jq|S=OdkYqni1*JKeN@1@2X?76*xFkVJGZ2(SD&VLWg-}Y$D3hnhK?d4xH zHYzRZ6Wc&N#wXw@6*nW>`mImE*1zZa*fscco@3LKX0FK|q!ooA`uFJ1u`f`&><8%K zvD;DM>>MVQbDtn;-Zm0D*FWG>Q2;yXLU&d*G03V7JPbu4BPS>bYmoLv=^xwS(35$8 zA|#zfXK2@+JjLMHiGoIPTyI@N@w5f2cRuF)pfSOJuD<`FH`OCld1%Sd;L(pkHG zhf#H5=)zOJ*d_)??DDPqtYwyg2V@IXCT4`(X{)F>Os>;a304{V>Qs@-F;ME$4k-(Y z=bNqQO`tsN#`HHQT}xw<3pXdK-l;Bo--*TU!QswpnHSw2v!5v!SeqP!nx*FWfCEh& zV&ssLxck}hMvEv%c8r4>{NeqkXUpNw+7B^Z$B%BY!6#8Vg@(9u9cS3z@EPDs zYSlx8quDI9WFZpS-BHh~f+IX{1xxXsa|{(QXe()ZdgjY@K zs7lWp6O^jSSOb?z&+bc}-JwV)2q_Q3sLb_aL4rhd=UZ#ylHE3fC{vpIl(1k*kQ&rI zq@9azFLpXA;JAQ_c6F7`;;X655c_opsF?Z7v>_dG8)Ol81r-M{9t2gJR%g6d>;qNGN0^JJ#0DeeRBCtw5j!6_}1 zu!1OA+yJt(Xt~Rvm+9O;3&Ti{;P`b)^>w6$E#s~j;x8CwcNVx7t+Lvp_5E=V1br_= zQcTdRZC?s=CB|3xkdd(la0@rHsTK!BGK=@9iN$(i*;TiIL`9E}kb?*C4ZOugqp4cN zo8Iy((0qu)jywdX;ffn6mKs8K?j6Xhd23!42iFo5d$=4T{*&cpve5zgu;b0ZuP3Zn zUZjzFpZ%Hqm!7r(V8w-Zg)t?*&nHYU>ugSzUxdiqN}4!PgKu5{wBY+c~1 z!0(t2xGMcdhV84`8&OAzBIcnag4ciIJ>c5f%>Pen_diHVS;#LYQuv3d z|F_Odh(HEAPyz*-XU!nXF}@${(gW{a?bc7n_tgvCZ+ie*G~)J!7Nmi=^Bsr@lHzJ) z-N*u@L&k7D@qyV?D9R~cHII_rIZ))_P4VcW6#G#>#CzjM<)G1|f^kTG`h#(r_-pdTaVDkB&uuIaOg9jdDwBr|>n`XddI1U~{+(SQ zeNAPCzn;&ER>R^+MFL7dyr;L}4(P1p4_j|owC(~5C3yp2EY%~PU0jy1*hh=UrY$^st|#%Z`lRZ$oug7_+LL9eU0-kG+ejLU zb&wj+zt1#J}yo2T0m#Y5c+Jtq;IETmeVtR@d-07f>Y;0Scm*BwPEo z%rTeOidZ%YNVCjS((h!Gy$t+)K|n*sqxT>Z5CdI}V|7 z{C0#eI&Nwh_vEkYao9xeUDsz2qlRJOrQc9ARcFgC3 zV|>0B21H`7&H|hcJ%^R^#eLG-h?8@10h>kMhES03S<6M^IFsc9kXK#-iu{BZzdVMy z6gc%sa9Rj*u!wtL)!QrLsj^v8gnH}jXr|hg1`%B~eP43&NuXtEwY8uTL&_YFU6nA{ zD)JKq6qlKBB5<<_k#f1fE?B~(^$!8wTGy_~pJ_wLV>px)ZaOGHN_2;%J2MRt;j3KrZZ zvoyGE^INjT#(ai(?Lz{h$=96XJh|*ntpaF|2|qM+ zMBEUKXAw!Mg0CMSmfJ8AS z)etJf0lQdj*fpJ133&h)V2ez{mj1jEsGt3XM+; zHFT3nFYCbk+l@t7Fb_+m-#xE;;dP2kJ@L%Dh0*pW(#da)}x z*b*`#w>v@j?JsF8Tl_3N#-`-xwswiP5z7s#nylq=w@yAu^n<8dgKt;WNvNh$YEe(A zX~Kn`Nbdh~tEC3Eje$!1cRq?gNj$@S@mc@4G80 zU^RKC{|4xHUmgv-K<^>0jil?O*iTTXQPo@72@=t7+gNLN28^dhV_7OnWYNz};lmXC z)!m8r|7+CqKoYNTny%yJrbkT{)bV_y$jJRwsgjry3MFP|%x_tDkH{kOlg(>(CyzWE zc0;iB{c3iN;_s6{+k3`J%<35NGQeKh)fs|Ss}4Lbq<@GC+)@FsEe>se4hKku>zJhK zOO+4zH!S>m6&EV^!$dPMRN{KX)!xT}hS5dm)6qpBzyZ`n%3(k6-eM5kQ3Z5xfV>gs z%Er*bYoO1MlKS%F#R`y_t@Ntx5sc?ELA%7K_&(*;`img^=d4;ltL$I8W4d(l9Qt)^ z%^z_FtvdKBh`L$|6z6f1CJ^|D6=HWT_^3aSF4p;O%zM-1#JGW1E;QZOFTw8a#)3~c zhosXOO^!({ZDyglCG!Is9|~{*#-zt8_>0xeFW#fl_^G5Au@Df;^t?jXwn_PAFAmbA zu(eNya;lEMBpT+Tac<~_-4f+h;Z-y3-|-i%o>N%KwWgEn^_-tn{oSv?qW+otb}O&j zH4hfWL^hgV*9+x$xA!)gIQl$WBU|{u#M1q?$Q0p)c&d!u9h_<#b)|0Al{#wfT(pvt z>gsg<@U5Z#6Qb*vpRbA{X;RjN<=i8)H2Q4ZcSY3!>(JErmo3eM5RG|fbl#<2|Hp6t z;xZ3EeN{i;hy4`@u%|j8wfuO>MQiOgr`0eF1)}cEtN;UZxs{xCiOm((JK2%Qw+i?l zX!Ra|RTjSk+$E>4cXtR3nuK-I)X|SN0AK6_yp(f6uGK6d+jIYTX1plab=G1qIzmpx zw&ELyMJ1PFVz8<=Uy5+CS;eE1_TnS<%zKRI%a7iXP4fLh{|`=VYW&jD2vCRm;iTJ& zr#%584C_A?a$E?s)PK|tpI7~J;>tpgQWbzoW+$)}hh8h*O0|<@b8eTh z$BM|rxZ`8s>m8!}b*tHF?o*Fap**o_`)kd0G=5D)YIH09-CV{68yvr|z^t;+3j!`B zPKS%g78D`PMK%p{be}*q8ih9&2k2oj?!8}TR)Lu%t^o5M^+gP{*{@ObUG3!;#|wXW zN#J~!8rU@gQZ&ogG=2HF@2fP=p9E`-cftg(#H&key8saKdwRb0#>3EbCD`Du5f^Mk z(GdmE|0@2CrJ-E;z!>r`Ob$*{C{qT)8|=B~lK}xFXIVnGolcEFs4yBp$jC`AwNuaV zdAa7Pm0huX(Mw#0gA>{R!?;M&pC@=7`PfzC`TAP+4h{W)mbCcma5U*){KmI?jPhtk zyEave-Wnf7#~h&fUlE#)*`kTngFpABD_7%yorp`)7oMSMo}WhyeP<91=#QFaYoq6N z>1RYaVxSDFc-7|K`@DG5hioED%z>=dso6C*{)cfgXI`s_TG5I}0U=RjDNgvc!g+34 z0*kWas4xMs^ww`vJN7<p*WBI+wxI!@E=YsiVBjH??hWgQn&E7vb1 zS<0>jTV`6g);JU_Y>!7C0hGiuJ@+hX2aA%ML#b)+oFx)v#&6(DBM^ zLtJ*}t;>eEOu*9Ht6TfR>`;%`(C&?js2BvI>=o>FrkyI^k}q(iannuz*D*W~RJ!mi ztOlZ#(1?>^+yf?etR1OoD>NA(_Ezcz0;mH>&T*P6oUQ;ShgA!tb}6fGfGfaLh5qYF zv4sE?^j_~SnU=1ML+eScdV=e#Nf6;pO8}NWUPv-f1Vijz_RmyobpX$f{Gab{^FG4{ zvh<>_k3+KRQs@5M(LXBHO^sYFKaijHt~k@o6VPMxt5j~r z9i5)wJ}>L}hZhBL{6Fj7yJZEX7>DD6f2>akN5z1M;s-~IB#@spI0VF&I5=e6oeJFS3MVaYrz?>Z3#0sIn_CUg*`x7(#0>O2lyx@b{!eJ$vG+Ys4rtuAd#8ly_Zlr+pM?H@#6CQ=t%NGJymO2 z4?OOiU}S8TFF8x8RyCZh@iCvqa6SeO~DJa!BKbx`WF293iu;P=d4=8+c zNuDp%z=K+3pGIQb)O2oypz zKdNXCrQ&UEB*+aNk5bAR9{Cc>8+;&Aq(*Ys$1Z~#5dM^YeYk=5a`IAcXXOVLQ0tSs zs@_!*%#&CYs_5#)*)2W(W!;3->E@l00jWG+eE!MxHw^0};nuJ` z%x*cFpIW$ws(0V?zBH6be4x5aDdV9d+K#PC$*;lv{RkLx!yscL;->#s@ z&o;j^&$Bpo+dGDgyO|T22Wx$j50`W;&GsK`e^cEs>>j>>>J1Ae5%{`;*NOHYFWBF2SC(@{bgD2 z)DGO?^#(lt5={s}^U;#gsVRvn`-iM!dYX)`07=AhbMJ zBYMPYzG9gP<9BH5Q6Z}^z7B)&iEKuAPPc?j0*yy9TujsNd(T)+XHPn5roj3y=S0xi#hosw^-$=kB5Vi;OPG-yjUR`lI_lolaa+s(j96c(X%FII)u&lC!f z*poKy)gbL>QxWqC99yU!sZknPw@fEuH*S=>tZP~cDk8GdbIV%Ws8U0N) zUSAO0zS<`^pS+i_aCBQ~qg(Z)9HsB9S4&vm+0I$`#d~+6*UK{ko2Cv|4weJfCgxD& zj7R#u)rt%4z5=8PpOt6wwxR-BaH;ouhV_BL=*;bdWy`C5509vPG4+i7#y3Yfw7VDG zQu;y#h;M59)bzZvxRjo&#ualQggV+>t0@HfUfE^aNRM zZw`T33H?oNKoB`jc)VIz(aVFetpJn>IT~*SSo?D`{G9q%7BU0B!vXhNq_=5ipZ07z zESPJpL=c;*?BgmA#a3Rl03;I|!D&pYl-p^vmK|qKctF3J_o52W4@f+E6P-7d;kO60 zYqj~%T80WV@9-xf1mK$kE5n9hC~1EWc=tsHyO&Y{wshJb3X5iR?^*&)arnoY@BnG~ z>@9~>D3ZGcnl|h^U^3IfisE*}(E{B%pf3O^lk?rQu#FbD3=Ig{wpveRw{)1k;4gbF ziITW_@nPuh)51*~e;T}Mb4c-VoODV}SM72j$YM8zrlOjN4RDr9~+ zTnc51PY;gpIlam>AFa3`Fweibk+D0+ug@<#Ke2skl{H}h%<2?yD*Mdme=a;blwDoF zVEdjRDk!|9!EsrIjhl<%#F4&anJgT9d$R5qDg%O>I)K0xShbsx@8!7Wro-^G4u_mz zHS%(CovX!h!<;CZLPIapkC9Ueh}J!;e^D|b(qiaCx)p5v)2>hk(DYTd^sw)x0C76l zNKWmNp{7Lk&U*o?00PiN5{!9$T=k%|fX?hr>upu!&X4H}>X5K7zv&C$RG=-U^-bvC z2SZY1%uKYxfN-pg}2dB!cswjV1Xb8xW z?i(iQu9>x12A{|f?LK^?cK8B)NZdzITl4sNz0-I;s^QCYmV-FO?C`hwq^2oNP zZfE5XYaaYU35Z+}QSY$-IlPc^uOw%2*)!XBsZW@K{n4c;1jZr3Z!%;z7LEKnTl4hL z)|rcbWbgbyJA7LdQR^VPm{RDAD|A0s!SAh?SK#^Pd#TzFx8w=;QY%Amy@dEdBy`jd zK7#2Nz36~l3LQqpjb&1d?tGU;xPWWCud5v-=Py4qzewy8mK!O?_9M%9LS2e#bskIKy>v4l^ON?Etg zTG>?>ceY!wP^q2>oI4frv!pk;&(I#ZoF+8mL5(rN{cwAN7!?Gy7Jx#d^6V_3T#jw; zhJmP#%6D(e+OVYe6q!=lW25F$fM>TL8*Pk?ed+V8Z)p-P+Ev1|r?2IQMtLw83bs32 zRc(AI(9?5HKO!`@o?}vMMYh6KXPn{1R~muSZRU8;zC`UF6N`=E6nXxOS^A*XX|BhP z``JduZFGXD*TV&fvI^E=6RS?k13!D@2bMgU_YhoZ@v7zl3N$zJcT08JKU#oLUwonH zh-1PGSkV_}V!fQ7KJ)9jWicL#9Gz7Ks16*8oQ=7cTjYml79ZQ`aRxh~uzAat zy(uyrZR*N1qDAFO0z03R@q3?K1lR0%DRT6NvjUN5x6 zDfN1EBrF?yIm@yw*kf&bYA=4*qc;uQR%}}srMPR}mV8cx80}NlT!PYb#?9-IwLAH+ zEOYn*e5%PKy+60wc7^07<-rh6zG1XF^Nocr(q@~JldUryt&@*G;GvF#=MM z?DQ=q1vt>ux^WOtrK2Wq_FtS3DA3W#G_HQ!KJo0r6w4*0_T?)|bAEZtD}EzYSC&}Q z-V;h4L;nkEkRp#BDx5ecIZzzZ03N(BL9LH=AnqZn)OKUY|DBjU{J~%gWO_cilU5r_ z$*0}DCKTnXit(a{bfV|P7f1~rz59;T-zoXS^UP~)+AD<`<-)U}d>&dp9T$?k7SesJ zeQ1sToYv25e1JBss=%)|8Q|0)_HONK)MFVQbNuXO(A*L*A zrC15oU*K>cdJr4`HeusPR?|J{KeFW+7CJy)HvF8~Xj+d7gn^GWZ7{wi-hiajL1Z1k zp3rK5rZE$^RW{FW&A3Wy6efMWIS*nT{&S%ae=*ZSi%5H4iPjr0pV7*Z#Ko?bF!t1_ zY&flSxzPuj`qiVql3-%J^r$yYQHVTlb9mz7#aYlS=yw_n?7kz+DgY3K3Q_8&-zqk% z1yc82tAMg8v+;#38esDQh(Wc~=k-rRfRGO=X4ThH$Ed|~?$7-D*P^L2s{jm(sFO>` zxcY46V>}x8>H%CVS8wU|i3ZBzjm0Uzv#)&Pgz%+7R+k-Z5kFe&Z>yS~r&n&$|2@SK z&dkFg{rvNMUr*Wqolk++s}O)@Pawvho^_H&`3{D&g7b>!vTFu@ADe&GO9X?06g}r9 zstQ;J)E3_d*8&XEn+6tfHs~KOn9P0Cy+@ziN{1xY98&k{M50y7fp)}Z>;N}no&-Q& zS5z9KwsM~G%Ju6dh(SS_pz*Li{ea)&I!0&5OUDnAnmj+5>>C zQ`CO$8HqJuE(rv$6xd|%+)6<%v{kPLvV?Wg$91LuR+G|GqNWnY+$Sv9T8X2zwaV_ znqG(yXRj^i;M43ZkP-rvt(GVXG-)%fQQxB6UVCp`lPZ>wz5|?-ob*u9=;Cavy6pcw z5H{wg0+Z>kC%)v--D$A@{vb790|C@0i9$qL+~@<4!CzuUpFwTWs>}eOHICgPJ^Uzq zNn|PJ1kaOCXLlcd;GQ@p|HsI3WYz#gcx_rlJL$DX&INoYA7dx^rgcZPFWmg1QBFyA zP1FBe!PTZCpk-6-O|5M6!KmV66G5N4jeL4XM2kG29_Q!%FJ7oy7*}Qe-Gu?12R1sg zR>XN;1L*h4&hwMQM(f4J<ogNlrl4~DHE9E?CUaDS$~vHuIG!0 zl^`}J!3D15*nQcHJtM}0h4sHp8&GH1<4RWb-|C0Bkf!vOuPEN)iIaR1`rOc1^QLP< zi-n@Y2KICOwe$aFX@Nm;y#SiDxcN@(lm@+csDsQ)>NlBmVg-P~X@3LgO>C&HmT?DN z!0Y(zChgxF4A}T<;9TbCbH0buqWWnV zXzfxC5o;SRsZ^{dH=dm;>u^_RKqQ(^btC(&C@pKHeKid5tt+(eP$(QRbUet`j!&dG zq`1rf5dJCr^S{ReIz|VGh<6?~3eD_`FHO{V{;Kl>>DTIfPwJSY<(E>6%E$NPp};p# zNZximnEn=?Dp1a@ukgVi;OVI+VZ*UkOiSNe{U5&GJFcm0Z5yT|BA~-akrJ$kB1kc` zkiamCN>fo0sZynP2qmGZh*U)>Qk5pXcSz`=hfYF=(0dCdA@J>-=bUF|&ilRZFaIXl zYu#(Dd)?)_uD!OT(!W|17rCGN_3sNotr%XoJI!9h@O@$%sA_uvatSsSi%H(@REYz% z|81qC&ji|1g%Tn^|5Ot=qL!ZH!FI1p0A8A#MOSQ}`-EMm0Y$y%Ow_K*OT#_^QAl9s z^#1~S0V?zFOeaTJ-w#Xg*8ckZt?TP^p->&cA&AVbTAarxRiDzD4Zz;a@)h=EDxKfpiiI zcbT71RVl`j9|I7YCd|OL={W&3Df4yH+zSiT=rqO|Ru=aC1mNHZmgecBb8?G2p1Bbir;ny_u4<5%)~Fm(3g)oK9YU$iW-0AzQs z55}fW+*1PYr0JU=0X+^kAm>K@PZ*)^sO$kK1(hCqt>%qT_w59bxlU3gC+HZ9 zC)DD@XDTbddg}D#AWvfy?|`OUlCx}}$r6qRZpn%Zr$7g*8h>TC5ervn)Q=1U>`)l~HQU?H$6qq>CpcGUeDtqJG6P5AHQvh^B zSH^Q^?m@>U0MK~w(PPvqa|00Dv61oE7=LN8II&HwCGdG>gu+~@-KYk3Im!I$RIMXw zi!elV2lotXU&oL@ZmI@Cu9|i{MfCvSiYx=nm9htus{u_s4_(a9{pHM!-vh@1Wg-;N zB><$bVXuk@dh*x4ifA*bO1*>!w8}0=or^kIkU?8rOVkp<30SVlz6P3SW91W@Ir-Ju zRHO@Eg9UTagaAG#WA~0yPtUPN zl_S~!f=X-FLTb)`G*ijSO6xum93tRuQqNJeI#pVpD!Tk>!hshZYVC`K1`z52J~2~`6~eg!;Rm+ke}3jbJ=^{4|l2JCG! zl?+21LMy-7wgx*v<|0)t=`OG)hYOf?Ubxtk2-~Ws4>SIkq8C*fD*Vb1Dqsp+b<%rq zt_GQKyF={P;j`P#fZhR-U2(6s{H^RltZ6m1v3%#e>_RiQ?sn&Ao-=Y7$3pXpkYxu=%k*FF?G?`)JtIRlywLL*-gTx__K5Hzp_ z!N2v$c`bSX#htt`YU`TX+t)*hF?YkY^(a`~ZvQ=oy34`x|dV zIic-RrVPteS+g-QFuHWMGOwgZeqEiaku)COtTYJ(+UpsofZr&ca6|nf0X#JszBZXZ zlL%7ZevQE)RTk;;$w~0BPWVe*LzdRg!e&9Q9GE& z{FZN+lZ?FN0CL=zcwI^-w<`6z4;Qa=W=aGD+HaL$-)6=lOJTvrjbw094mRf5HlZa` zlz;WjOgk|7qFX-T0S-kfCL`Ac+YVyB3j5PlfP}X_5i_gJ2^nft4*7 zEK)P>6Z}=eJTqS0g(_aB(Nj=;x97YKi$utESAYkgp%JI0fV(Jj{S*T`De7jaA+&A3 z7js+d1;}8-cAxzcU{55)p!eR`TbTk`fGt!ID85>#gf6Kp_024?*BrFUY=9&?gZ)DX zJ}HJsB6-pOTEEu2fK&DT-u^y?=`Br2m{29_;r^_r^VRW0bV!M z?9#N(Z^Hgk0x|Wkd+t9FxXL14e9Pnb+8jc$Gh-p`%vnXOA)NJ zlrPzt=U`1{*@IWtR(3s?nkc#KAn` zJ*_&I%-V9(q|#ySa!qgT#m@3?rUh(n!|@~e<@Xc8xSZ55t09Rgt)ZbzV#l5M{-hXi zS{n(x_q1w%CG_jhP26+z=!iT5_d&#&XfsKA1ISYBY zc8)IuD3O;qi7zm?(gHBzv`ihfvJz$a3rZ%fytP@WjJq`w;r`VIwL-MZ)1EvSD#*;u zw~MsaS#7_qs*0FfAexU>{D6Gvs(!77DmN#;@OUMt=oXeoN@uTN%nYrLdNCFK_+!NVFtOe`8yB{_JKsgU1ktZ>-pPeu7R{M6qDR-Su*zQV!-KuI_^$TbhdmMP>i?vo_ zkFSmPP`?4lGC%k-R zsC@50McYS?<50$sf{S+UEVEmU&5mi5(+=Z=t875h$CJc7EtL}_dWEH*2?Be0y zeS=U@t-D0;X!|RF z7PC4MVDu$Zz0PuNVxQClx4g{UbBgnGtMBzeZ8;iYzSqbrHm?j7#tK8libp5!dYi9U zY3b@3VVwKZ7rpeunHAPt^`|45abLetn}ZX8=H1xI`{IC4q}TcMIA-Ia9F)yix#i4z0;+90LFc}wOa-9K9WS3S_=2_+-ntHeiB zD{)jEy7hy0Dh+#BYcE%d$>-Kpfg7rrS3`LO7l7sLis5%rj}IjEYU_->*lsCqi+=5=%$<)vX;+}Szobq%5RMAs= zy|U0yNncHKBeIuelBY$D78Ho}_Tvvmw-kuSw%5^a9H_0-&F?zsRuiM#tLsLgog$eD z{Tr&sPu+8SuRC~;4ttrk_Kj#8CBDx0Y!yuY!a}rDMHA&x)nOE#rJDVF5rWP{vfT^c z;i1xDzCe^{PD!m*%NM&BambdN^RJB-`Mt>oVCYYlx0VD?#Y*Pr z;Nf5vv8P029T#2RMekCNez6)@N&?Ue##19DxkwF9OCvxGA&`ub-BYzzzsG(83rwbA z(t6>c>**7^cbs#wE%i*7|Cgz+sEtf+C;%T$d1DMz`{*Mt(>%Ngp-B$9j@IsWc&@$J z4T$W~K3v~=>UMJ^#+Wh3i>|udhyoxzQg%me3d-DjlOdkYWW)wlYm$-y<}d`^#93=~ zObzayZ;G(fjFU{i@Pk@6Yws(`Z~bu&C=X4PhqVB46AXU-jb-v+nH{F>M0aE4s!oQu z>93Rak)pv5$W0)eSmy|UKP*Y9&qNIlL=RS@l3ZA_2oV858$G_^FvSLB(=}K3wsYSa zU6!gT$KhdU3sCX5m0qq3(2wOoZB^prTa~lfk%KuW-+-o``TBhYpy8U{ssVXnG?Zy4 zlB%w^kCn(kwWb&nbHwdpuZo@MUYu=ks4Xe{fhrG5Tz}#F0*ib`MT5%xK^twydL&k} z*1vHT>P80S7w;a@-&_F$X!?Ci?c=PASUSTVK+|#v2o0L`Oa;04#P4(65dFPq$gss~cCOWw}M?E{TqHd`Q~=Ecp( z(IF4D7*0(6R{>{ZV9ozFIRx$*QwwmS17qKY({5_=WI-iVVo)p-uHmymo9sL(ra1O{!)PyQ| z8}5`{WU9Bmt%?{b5_qq`Eub++B*1ieN5*Wl<&1(MHpDPATM0G5~ULmQjPGQn~g*{!$!p8Q?d5|z|MXnpz&&bdlmEh)Uk~yD|2f*GVYS2(fq9Ch@Nm-Btn z>P^G9@uH9}4iFr*tT#7!w@(;^?fcLLE7t6_nm?}dw7YpgA$H8RF++&rS=yC&&hkoIc4waR(y>nd4elWk6p$Gmn#&% z7?Dltu$-NwvV=YK0k%?*HKO_MnMs~G#&e);+8ux#>*nZE)QQ>tO;B_6U#$hg{Z1dB zc9!)YZLW73NFl={LBid~A>DM^cb!@}IrTu7|IP zK()wz1dyWb6?APYB0M}-^M1Td9Cc{o?Z+A6JFGoc4-#i?qw=^E;l@`o3#c@()kQ!Z zcZ8Zg2AV1ix)?aMTof=%PV5JYj``~h68n4K-UA4wPWB5$TgJV^+tZxvt;~v0_m;^C z2RC7j?}LWFyS*Xk7vFw-?mRdjs0|ffP`2geFapmV{fZN~vKU%4y>wRGkIG_h`5@>d zDKkwETT-sY;b-5su@3`Jm8oX|>o4^?v?dcC_ppe-X^d)VDi22nND?<#MQ4p;=7A0E z5aaR(Na(b*Q-u;HGx^qqN!fr(iB$-{NO&x2%)d5-yuhq*KU6}(uZ{|jPMr zKTAnSICQZp_zw=iR$|3H$QIq-?=7L7l+^jfU5}bS;%ozX@H75}O49tWb-A7+%-7`JLoiGar6Y zKUe@ACli@)^EzMrn?=t`Kn|h8lKX%I8LS1mn!Xb09JY6y8V=nVCkW0X8v1nS_TO8I z5h11x9GjvGtw+v~K^Xy&@-|guei*)JTP^dVv|lLu?%hBo8nF(#Gm_L&hB(o9v4x&@ zkc&@{q4u3zk`n;Il5VtAIGpuwZUTKdd*R#CcJEe$at&3eeISa4CrCsFBBP+-GKyz6 z`^*=_9ja?k=nbR~kT>KuXdG>uwj`*d~#o6m^ z30sAPMi7SyrsI{!t|a08&vV{_JCULfc|9LW?=IG+HaW4ji6ikB35=cat!J1R)5o^w zH{_FWsZEeAN6V2=iJ3R-Su+FlYjF-+LFee&IwR!X!A-GOnA9cML$8a&M*UrZ+|g-E zvYxs8t5u)`4Y^bqBHFg~lpC?+XG2mG00^6&H(fTjX5ToqD%ShV&>L_;!|8W1`n&z} zQCCP0AQaQfqF0u$s=oL*acIS?KVU>^pAD+?d^!LU-@9xQDP@?_Mr>e>I27XO65eG} zriq!v~CsVgG3&VSxJh z$cE+J9*X)J^6{gD*0WV~oo-D5Jh3tK?6N6g3Ov`t*n-nI9UvZ#&7Eja;{{q2xdMwMFmwL2aA=gbMT-Vbk^ zwjt^hs+=&p5l80Aj}zK>EjQ>%kJWJl`v-NtVSz|BT!T<~_ETWUOB9baBt<0SkEWpKo3Sk2jL#U!)nN0erLv_LQm1~b#0;NuayD$jt;nzlKpl& z{MMp*2Ypmb=6dAM0(Nuv9k}dcv0PJP1pl*l=NcJ{X0KzMSj|=>H{0So%nxs1#bAGe z0y@>An4@+Z1A>bJmv#%yp_hef&m8mgL#2{xukmmrqMkX#&u!DkT@f_IEh#z1vbGEI zOuq-lUQBq83VFJ6d4Dfiow;O|Q9k?4eCw@rsKYu9RoxnbJOp)kzW`-0ruqhc^KHM5tA_ zsS59Jox1_AK8pyOU-s8>R6_H3oVhjkP|@&LD1FFz^Fud7gReD?mZGurNghiVWL(ZR z+_f)c@%(vC7+j>Ad)U1=7dpH z@m{*x#6hS}0IrSN+^YtH$3w5`!O0IK3?un{UTHn%gBX#pMkas8gmG_}wr3F-PCmQ% z7B#U9Z#CuLbuU-pGo~>y=Np{CZ2qx*?tUvdY}?E*PBZRi3qiH%%0eY^Z(pB;|RDfzYIi-`w|II8zRP%$-wGVtTCy_<%#&KQ?=e!eya%2p`yD?6%J_N=3S?s$b46ZIQhY2%f8QcpLQ|N|`n>r9*7ef0R4; zg5^sZ2)!M0ee~5Zc|`W1=ZD!bkI$yh+&;|n8qGCN`kpKYRx$WO6@p`m`IL{1nM|9# z8DY3S@z$tfyIyv;GKgqK+Xh>Bfe3iI*{iV?vpWg!WT$Ftxt@a8R;Y^<4Ik+2U z#M2^=MRfU{7e6p|7p^~)KHvB2!ad_Z&o~CBFkHCTkAggTt`r(@wx8vvADhtQDTvbN zq@cYN*IQwJhWDS_(i}w%cUq!+WC)tWP|IW&`x5a?WNcRU)6|@&so4*Pa_?~qGQOGi zX~>G;+}_&ZU=^FF>(8Sc=~5m?`z4AhzqUIwm3Bu~Mr16vNKOy#O%o&@`4qOfY+2Qv zKA>0qamdm0e(C^_F4?6{bJeVp+ki zTlK&t;C`zmLfwXAaMkhT(;7rDX&3E-AZ3>h! z>#tV@gnOP{nKP_^Yg_ZU2_T$rB}NYG*gkTXmxaWx_s8t9JP6vWXBMc*I$+8Z5>0rd z>P`x1y5PsNdK@HL4QCkclTXh;*B7Beu`S0D=e*A9Gx&sA?#Ne#b1V3JH%XCgSANvl zjvPN%=&pL(einu*d-3=~1-P}Cv^0;`ECkg(ug?L^+xg)?)TO%6v2$ri>d}XWtn@y| zhJ%rl$G?)@_8jT>x|q~!x(${DG;i(-*uCMfJHXwvRM>;Z83jB_l;&Fun3>Ld56)R4 z(OLz`xef+-qes{gPZef$6hJ$XLALu=oX`>YD&v?2dbQiiUu+Uw?z#gHNn4*hdo$Ub zqQPx*RgVSNXRYl_#1!9&RIJ;9bXNF~cRQ zfPeCU55$RPNh9a2Xk59$RIoc8U=?Ki!e{4-L^l)j+P0(8=*&^lHHyL?%P#U1>*oo3>E|n-!gm|v*qPoKTZTx*(PL#Lp@+w+s`XW!Npg+(XL;a~O6h^{*9yPZ#&U|ieIy+eF+ zqJ(`r-*B*N;*82< zlFHeuq5GE6hwBCSi3M}WuSaRx&ubZeZ6&x$#}3>twEVHepgdOEX?6$Bj%7g)Xb>CP z_Rh@Rsoj~2t5p|$Tl~#C%k{1VbD+4~|BXn`uZ0&c|V3U4}{ zMhO~%^lTGuMYfAQqx}Fv4RKi=_ubZeTUJm$C`k1Aw2p&JF!~2HbHNeK3x*>PZr6`s z;ICe7AFR}Pab^S~lR{+`37m{`Z)dzk{|vaL%v6cV2uWti4p_D!Tzf47>(``oJTUGc zU8Ia(V-;fbS|RAlW735(T5jljt&AYfKSemFb46(S-Iy}_QFQr=N_~=mMP035@_JZV zaFb;Q2OsRX8#lT;%>lOKx-C7l>Jsu@C@B1Jb;NUsU6iuC){gF274|nE9>yG4qdaLu z@!R!N_s%M}j>q2Hm|2C{5jMAn3tBau(?%Cvqd=~Y*(XKb6c7he-TP>b%$YBMBaFAQ z)usY;TtbR;{LY9cz|(W&BP1F;M%8mH9T0bAGy*)=?}3-UChb+lp5i}7&OBl8jj#zi z&Ryj)?k0k1I`B2%O2g(>@c3MDX0aR>GU(xM*emyLd&l;RYZ|;eYj1Y*P*Y>E+_+LL z@kg!RiK$b{fCLWIY$e%-z5wQgZ@>g^HD^NADMPw{CNM zM1lOTPu$4XB)M7*|C2$CS*bnFm2Z*VvEL}pR~?bJ3_d@{s0A{oyzgP+t(Aue-0W?V z_0vmyjyDarP$i`n;pep|_R}V-4v^){)8#A17(VRU72->kQ;(b!QgBJ~X`|DbH=(Y$ zAD~7Iw{5Jo*dXf&Wm-6IeV0!K9*b(E^ZT4gUb44jbKCo5H zFb>M4$yu%J&~N?ud~6l;a_qgMGxbYBL67?8ZcaZEw~A2euu6E;*Z#TXh2B`gJK>Kv z3)ME)tajeO4!ROk&hC`fa3UZK4@cj=0cVje=fB4;JVHR`HI~feB<{T_4&=N}%OLB; zdB#&yLp{Ou&27%_CyfP};7UoMdg&i>{v}Td!i)y+E!t|<(ZtFDZFGg~Kug+`!9teI zROh6bEbZoyXcFVv4gK=-@V=oJ=k{Grzp!gsFPWQo&@DXwQ_0G|V}a`$WG`-id2>`` z=%q9rEM6j>iFI^M2X!-cPA96|2j)KHPRZVEvo&qtD#Oy*Ngj;G+f|dKiNDzvCknOy zaaMIIJ#*T>fG>(v7WFt@)Y+S$gg&(3xa$$Zm`xXrKOkPv&wS2xiO|j`9e=*oE0Ese zDWS=X3;y;t1>eB(eUNUU<2;@;=@WRvmtn+kw|S5mk~pV`XVp*M+I4(P zNaE6^5+81?=FSnyvrRUOvcM?Y;UipG#yI{8oIy6APV+P~+uRc#}qjD6V{{Hgu!gZw}S#G<^a&hKFu zlHciD)1VjL=s?RP6DeA6B3>{1H1wXb0&;R8mh+Oh-5CQAom9{lyaT6L`ANTW{PuEQ zmfKMLO~g+q0yhy`8XdS!d|#7%x+kEg7k?P8?cyob+%>TjA2E=A zqPHAlnr9wTAm`SAcHsU~XKdNV-yJLd9sY~k_;qhJ!HUp;L|EOp#FdsmQ`lrQ#~u*cMu6$#dRayAmz{ULp%rM)Ft@#qt@!q4oKOD^g% zg!8R%pR1X--SVwZ10sc1TF(6jg6wD|hX78$BAfUqLlxHQ(sU@dS#QGwy1ecb@QU{d zK3HS||H^2Q&|Z)S+#|?4AKxgjUJ_V^Wjc>CPjSLL3;gjh;iGG%aU4q{t^7W)n;F8V zeHDlJ%4R8j<+HDAr#;TzqRHS>fN#Cw6Rv{4!r?!Q)~?hbPKy-`?l`uy3k%eTvB-aE z60WztUBjp96HLJikG-}ni#YMonHcBC8SixB-65hq%TFI?91pUFin=h#o7*Ky8;@pf zE^8T8SE9ZeOvAs4%lez@H(q!$&By_j^?ua;d_;vKp4V>3!pp0^kIxkRopd2Qr3Gi7 zkQQ<>&b+p_J1C~i0bK|%UMcB*ryDB}Kz6DwnboGiGxl9k51noa_Y_#7Z!-#3Y2<*_k)fk;XB5^>M@bg)I( z{n|DqEpBF2A8-5Y=>b}hJoforJN=!@1m(ng!Pp{UZ|REZzGDl;d$5LsR}o7l(2g6e zpQ?Bg-95Xm9&yXBOEBY}7t(`NGJ9Iv#=rWeRy~3Oz(k0TW>=n$}6|tLV zPI|9hzoDU0827y-kK49UP%u8I^?Q|4vj4zP@I5`8pV?+e9{N~kOm7!AJ|Kc1!BsMR zy|2<5dS6mAb~#zS!>D*fH__HF-s<7PYaqZS&*0aT>8TRYaZVU^z*yJ+L19}s91_pUpP97%y|N)K ziPUh-YDS0^s>{U~)zv~mth`7P^sX|COFL3>zVnr?j&590W4$2;wT6F}sUNx492Akw zxwTFX!^TG5$>oajwEpu3Xi*Q$Yjz_hbnP>{tK}xI=yPUY=(0S82exPM;$f@$d>?M^ zUQ%{4xc|g9GHDAkYD_S%z1SAGs#ET~3DaLT)UoSePq*{SU~m6k+ZeQH=9J{VX?Gv9 z5t2HP6nr(gb{g-}5w|m+8ufr%|B@ZC>+WP)+)8>oZw~4B!}YrJDK{BMz(x?Lxe2mxWFq*F0+aT z0jV|r?PgI^l1K6Kf->hhY?g~%BGIw;%oamBrA-kouWsn`481aSC=4>h?oAwC<3p0# zn6>O7X5wyj(MO`3Tr7M4Nkkh+(YC)3Tc z(iowIm{+Q2GVAXQQY}DC-17B2HuzS)bvE;Z%kQ*-pbBZf%-yfzVfoUST!`?`w~uF} zaT%tqO(Gk$5j^3fo&I)*h3$gzE3{mvaU9~PzD8ol_)o!bdAQ=VcQwZzbR_cDx`;>eyzyV z=8l*n-qDY`qNn9mdBZ-_jB?(3CzAVJ71urbF~=|8P}}S%@ur;a?VE{^M!8lDo#M=r z1uvRwdrp|1ALVQ%3t!T4kZLi$+H1fTsrTl{dxj%mnD@$kjczqzoz}c4i64 z8n$$y2Ggc_C+mgg4kit>*67@g%<|{aH-r=SrDr}=j7TL7VO_B{v#r9%D;dd=0)|9w zE#aS&X>y1)k_loh(mxTWUiUQ&_jpq8esPqM0zq7ZySOQyXw1EfK8*f3RdN5k9Yk|- z6rqkjR7cF}Bo@~SLnIeGwlj0<5-6|6n2olZQ;$pg(h5+n{eBv7Qd1V&vfS8jlW7rJ zCS|9#Xdh5r+F5V51Rpgttb3Gt25ks&4> z!PlI=ZlBRP3qE>LAO<1sxUzsmCvKeMk1!MQZj6)6iSpuXO#*nxbbKNCoaw5?(@gPu zw|&1cuO4^Vkoez6IeuD(e>-p^`!jNA*3L>S2ezms3L6e#a!AwOkS!5TWM%qxtmJ@% zT!|(!I-(EbJ=^2s)to5}ho}t*qo$*7)W&_N(VAgUXcb0rX1>==%&hJ=l^*)0S-q8s zGHwvokR`4U%J~#pU5!c@5R{&2)?P__jTm9In6TPudMk(~8aD)NtmEna(F>yb^GS-LP;dC*Gg(gYabs<<;}ybFU3sU}nCy6SPlxOJ z=Ed7C_gWQ?9B*J<$1`V&U5*qDU5Nr zbi*uX0$Zr|>X)zx8=1+iR)yOBC1Lcp(;eArYr8&y!Gya(S^D4o0WYp&5S4QLPPL+c z-nNrDnnM)y3R>u6d{n>VKTqGuEY`$)T$yf=WL)d#O0kDUK%0g)hjv6cXWbP@-$l7S2yQ(I~QaA z{;bs&W{djp9|{p03Z!>0=#J}~EjcMPoa%)e7+a*@Y?L=tVS$`BaW7sO0eM-?@BW|h z{cXVJhkl!>9#_Ps_dCsvjF!eQD+~nW%|6%{mT5Ay%prr+n&WV^pXlauc2p-*|3PN1 z_DWRu>`wAL>>u-fu}O~zn#l5`t(n%J-l)mUZ$EtC;44jog_)Bw^0Yme_O$F*$T{md zlfQ0H8`2*9kDdND4&pq0e)E||Mm(K+L-l}5vhxnsYC5ryJR&2!h?>|)<|JE4@e8j* zX|KaJ8*dj_;{Vs!fBO@2Qa$7icc_9ccQd&SoiYx4QH|Ze%E&8o_o6oun`3lN4N!Ss zt99$HwU1yrz_c;=r#1u^fvMNX(&M*(w@#(CkMXG%K;s|}5+*j>E7nUVj5s_RPiDE% zaI!Y9zg#tMB;GZhfA6XRW`@b#5V3qz z?Xrf$Xu!Jn#H!(#!Tuub^e9N&XNGkw#xx;``f?DM&tVg2(otRmgJB`BeQft=)H#t< zSp9%2x|1}rjh~P2l=;U#a#^I$N1w@MoYd@qk?~l29g~}9H&#NF#v`6GZfxoM9s#fR z_N^r5In*&KRMgJjsUUA`A`faRoR519Om!Tq0|uzO{%;Gh;~yW-Q474jd`ezJNUV3? zEjGXA=F|b|U?RXx$9((^_d5WM!7++_x5AtQU0#w8jXF|=k@|#3Y_?XjyU&h~XuAD< zHU53LfaT?zI&57Z)OQrC-~Yx_xRDaby27}#QOMfNOBT`c>~O!tu@S$9MQpefxRfzz zR_3^fmAKg3*Pzok<%|AtM1doB3Y})o2-pd*@vq*2wk6%wv6NRv69hIZ4btNd9a#^B zedcwhQGcE4|IBatH$;6|bO5n`AoS+>WPfis=gSk5+Mfo?*WGcWWylr_=cC@8AAh&V zbj~RA$>u2qeLF)vnDaka0GijzTmQ#7*zw<8p}%cxq|jfmBCzZ!09Pw0yZI0IVoCJB zSXkhPU|jnzlh4yH;+S97Bf}LIb8GV7SGX_z*B1&5L9b}ZXK^FpUjJ>~X?}A3Ssd1B zF*k8XXH(OYxV|#>j~V1b3+eWO=L!)2k0HzL0|QDX^vJE|+sxz%kpB6F>6{T@P$;_n zvQ)!=`aDsPEQb28F5_}Gna4vl+J=Z4(m#9@fUVA@PSi%V?!TA8F&tscxVBj7(o?4eDzemrgh%a&DtIkwD%HZ!K>-pCL(2_@={vRhg z!IWx2xs8O9jcTsR(I5Zy1(X+cP$LfYCvHXuZo)k~Wk%#O;iEDe#hSzj{Q4^TWEI^Q zZJStkKx`nSi-KyMDL?fEaC0&<^8`M`{EAwR(K4Rwi?_yG|FuACKEQt}1nMtrl=_7Q zUR=vdS)VO8PhETd{SN*ql7pEwKi|2W!#5dxh@Rm?&0ISS;xycw(KvkPAH}Qwy6SgP zSSS%!R_KS*{9PKn3p^BtZsAcyzuZVZ)V=`*-f~v+&Lm zJ>t%ag{Q+vzXy@Hfrt!8$(A&L(wcVvAaW>!dOmahUcX9V$>t5+cwJ=?) zoXVDAWNUjzfzo=v?e~cVrLvhgb7dbmzM>u_4R<9w!-v!k3eA3v4PdKHj}!|;Ea%0$ zg`?|$&tKQv7O=ZGOcO3E!MZ&yXoMX(*EDceuB=D{a*3{g@qo>|6($br&e+lCuT>kD z?ADOFTB?s{@!p!dS@HRom|#?akNo|b{?+~6VoWjSZ)0Sj8sIw5>O&>*LhosM5F9>S zlH>+I>NG;a2U7*z4$B?Lfa(alco1Zwm9dtw5x)XGsKU-1_?{kcP~9yDdfx_J&e-9I zCyn@0MmS)U0azWLr)w8@p~Uf4Jo+eJSSKopG?*zUxR=75xJS=V>*-&L^xBC^u2lq_ zm9}QT-V|v#NMU@2JqW-nZwLL$2l}IFC_Nsi-I)sHY6Uxh*;VulNQCshuECFBNqyi%K7RdfcbiI9QUc>EL5}ZP!<$%}Ez)lD-_R}5lQ|Ch<_pF)+ z23ld?u$f~-S;;%j(n74rA7G{^8y*ga@H6?RM59w8O+^?D@9y*CZs1jNGgGIODUQ>l zyMbMUDWX9{-j}wuNL+jJ=7)f#|vG;&?jZ(N%3)1y4GXM=e1>STdn{?BO^-`I<5|L(W|V$;}w? zZEL(N{AP%k1>(8CB09j?173KZ-k8ZRVc8T!#x-QEzF2={d2X21merYYIbdF=Zsmn= z!@j|$*T@|j-~Ezg#3Af-@zj=>e6p9x;d|iW>*3roUvNq)s1Qo--u~?sdgogKWQKN9 zfM^Ds25Kw;X`oD;7_nlSVHG!;ZyVwo^x;x8nM^8Mrb z(3u0S7qlevcjdTRh2GO+?5PiYQDLe|Qx|(acXaLdQkOYz0Dz`9O=%W}ZZjZr^d_uX zzb3|__sa?g-bT;Z8CPG!daBrjy~oN-wnru<9I)z*gV-*!g1wH`iQ=i^ss9{K z{TI}0;kL^m-F`sHNbSNsz3C-u^}?KvA3f*OKqbryUizvRKAv_I9Cuj2zIY}_aJ)*~AwONF?lO(?n)LwZ z5hsiKjh^c1NkDXvW{Sg<@Gas0x}GUyEEe7>Rc_oL^T*3Vq?P+R=Ep{m!uH9K5oM_R zLt-pzsL?_7h`8J5$7bkHq7p=Y^v62dM*ij%~^5K=d@DIL;Bh z9FJPQ)`o0T5iD*H+>OQ9@&)>mD|{7d*UZHgDy24)AOh_|0}qsupp@IfgzO|^F&Fv@ zTX!phCEf;a^FJrTX#QX3TAkjwPY(DvFhcc1idF1Ibmbvq(P0^h8_qE`w!)JLs}_d2 z$LGdShb`ih0gkkFIjR9B2RH)_&nfa`MWQ{1RTC!Pa<$S&$Ed!#X)@wN1H+o;5@PrQ z3ad8Cb7s_z;kcJ)JVFz;9js;RInSH6Ne?Hk>Vp3fR*7}~Yww2>pk;nNBkHb=XNSeQ zGo<|7W5$Eav9erdtSJUjPL1pD47S9KEmt>+S$iq&G+~(Fs(uSB z2PI~En1~zu0tc;%I9~x6#X)#=Yj0QUWKPT82N>~zAoO^KBe|`2({4H=;Tu}o_tpOE z{r{f}4*uH;v_z?gaQlkpDCMyCTX?>wXl6}e7;AC5?L6k^+L1T2-rZMlF?H@vzc*GvvIc9m3FT_r2VO_t3`VZDKpUY9Pdf=;huhs2Ozug#q(5$Y_D7_pSfVXQD~`s>bNpE zVae05_4}(p&xB8@pHBl}hkfW^-fn=u04&!nYHDGAx5_$ z>~UHkNNX!j2PKm1nj}wD!Idn%7aK)fLO9OH6`VK^xJEcoQnV6(k;S|nZnf1x%`>Iq zq8yHnZ`Hm{bo%KjIWGykC(=fF$o|1-UGBAK0|27Na(Rwhb6FxG7>$)dsSOdz7IhF?ipiZK znA?eJ;z1)4-w$)*k4ieSx}!W?dfo|i3Ok6ad2%Q`KpPtwxD04aCR?)LM5;Wz!V{#X zc*5TLef=Vm%r5myEqUcujOHP~s}y<>W>wrG2s@E6_Y(2b@5 zWuaAymwvUJtM;Jso4D}MhtbcX^E)w3yQB>KL1%iocfZY8ns_o>+Pn>BJa=QcscKyM zC<-a=JDhi{Gew63>QPUR{7E}RU+E5@zO%X6r+z3H4f+s>{d(a9nphmM%mSP2)BJ9% zJuC=IFkPOsm{I5TuG@lziILPEMSWZE4PZq7ZsZT=?oeyl?Qy-rKJ}FLBXKa_b5I-^#Bx4ocUdNC7sz^lf8-^?_6!8eIE>bEB#E%S5WR`cj zSyVV{k42e7Dt=c?P@HU*i`Lpm>x=$0EK9T2bg}nF!A%ivcZ5Hk_QjW{wkihi$@` z?>*Dh*E-OXy_7}6k!fwM(!Rh=W+B1$=P>Rs?Y=v;MZ3#d$vdvGUPkJfRzZlG{rPt9 zj+)rekRf4JdGh6nw3)8smcyiUM*B|Cu2g$++?7Au0Zz7HJV_D!s8 zhdkgkTpsk~$YH}AGK5%+Y6!r~V=HHJ6j)E+ibq>0Sos7Rgzk$Hw;`+>%~^HnD|^`V zt@zRX0^w(Vh>R_DGG#Vc7I8S@O_Q@ge&7IwC42^>HU?Bv5v(-iLbUb-VICo^~KaT%Ypaf~&Ru?884 z8aqg~y%C8SNOr5&w+h@3d?_a|o3|F{)h}NAh*N#61EYE=-*t9cam**weWyxd*LAx$ zPCqkiUOY|J)$#vh?!CgATD$gN=^_HUP*8!OOC_>sDo7`YEJXwo6$O#1AVsPWdI$m{ zvJj$%-hv`TM2d9jML|GXLT^dv2`zyH5(wcxS>Jx!{@%S$_r7w#0aub)o;fncc<$dl zS_INpGFUObXBwOf&$^*P4ksZyitDiT8S8X0`s~sYo?b0=(&ARmq+%()y|=|;r_T+B zT`q%xQEfXfx)EwIy!KL~yG-Dxeo>as}xuxuqe24*G)ZJ!^AI*$L&K)60$T8I^qk@6Z)O;k~E7 zE}Uem);Lf{IZWx++9L|zhPQxIPc2wQ^gdF5x@WCmkNp*ZXb+Ie+APOP0}ZL#Op}+% zOH>P-1_r;8xd9<>y#VVx+CfWWmN$}V^;zd~V$!m*^jK&;&HZkXmgWx?BKu4aK(vxu z&w^;L(0q?YzVkb44LcE*HnCdGUY!Z?^zDYVorV>B6bJrFmovP#Z70cAsQGbrVyqhi zL$0Gm_hnvx!e7n%k?=lBW^@O*+kXL{izJ+#fZk5;9Le|4r$zk6@f5)q@E%6ZY)!P@MLDAj#Q_Zfb-Ts5nVvtJi&5{}aY2McWdaP#zxYKt^j!(ItQOj4YmF_&^K@hezz?&IwG+s)nt0|TJ;sGSxKK6u#aurz2fk^vCr z%q9V-tf9_oV7s;_VTcoD$BYuoqQ&<@DqLxkEBklVPG4*#F16A=WqH320{xv#R{mYc zT5ey_cda@wRP|$RKNGCmto(#Hzw!ij8M$9*pzoBZ1r4{&-cu4rYP#zD#6}-D`AfyG zT54)oKv)_%?~I(ME*k0uZ`6(!JNJu1JwA%+9sSAVua9yY+Fe9(7M*x}x%DU^8-XX> zI2SwB=UID&o_+;OeYUPvanG93!r^h#iR)H$^GpQX;YXz&W_X9LXVtu$*;C5`YQaqUa1?z`S>y!UY;(UlUJ1LLV7HMrELG8Q$#%@g zpf{|2R%xW$O{DhDwj2Ml=^wr&soojK>Q?=V$EE5fl_s)|#Wj?|*Ps5LR^lCD%4w$h zrTv~gCZIscXh^xhd$a>E@$o8QBKLse_l7#pCI#o#3nO}6^8PH=c(e`L*XfA5Z`#V^<{P-bLMcd5E73I*UDfI8N#<1mT|-dhZ_d3sL0$(;KecA!zD zDRQtNok25tnH@!7sbKAM`+vw+m)F?0m69DrK7gyE}hkvULIM}nlOWK+V^Fudm z2bz8xyk4a&P&_$kOp6(Qkt1mP8Dg6o0_u1J%1JMoU3o-12zioNb?F#9y){Df^ijB_ zKobY(2)J*w%CMVdCYI&xx3EtT7NP7i-|d8LfhS0vcKk(t34gQ@qKk&-zf3B2~kQkRQ!<-5?B{$-=Fd z39o12UE&7?tjK-~_<|k~kYqWp*-KU1GxxhgF&p2mP1}J;IgFHJ>_nYYD6u!mb+hNA zQ9vd>{(a7CI*i>O#@33b@DAt4Jh!YDT-}d_f_9m2TleBvF;TzJ}wfmqUm~Q#B@C(d8=`6HyyFfEX(3w8)!aIrA=sz=TW~i$_k^HVaA{q zCIf-0;U(RYQ`DCp;og*-LipV^!j=Az!L8I@#5}cOtKO@28?7=VUc9+@$nZ^UgWD~} znMn_KBoK&$%KXYHHXgxbW4jvDz`*L?;p5{|n~nxYMT3DS14W3y7Mb?p2ao%&Z);jx zB#aEZxowvgi&=YJ``PCuPNb)7!k<2xX6NW_=b(cTiUaJclBPZDw$bgad2w-UH@WoC zX)(;m_PHtOHzwAi2b$upG1__mjjI`3ha-3Y;+rs8!;AKaMT^VKGGyUOpyquazwdfe z!NT2o%p;)f!I$mV@-pFSS-3NVzWrZ-`f+~2EJ){Go)WZD=wefd!ZkGr{T!ifCZu5@ zq$mfRN>?){R*Sp=wPBFgdzBd{Lo*&n8HfjOmkqKd>{c%FGq=Ectd4H44BPd%#oP!v z3;f7k!l@0tO$WWo3Lep6iE=w42WS(yZ^%@-w5BUkJ0ZDdB8^A-f{ahw_sd#{ewZoP zZfX0GpH8#3dlQar2%C{4RI_Ai8|M_gzV-OYS)gA#K$J2qlsQzLL04t=+Rs0teY#` zTYad)QF5#ud)j~ZcMu}ZPE7`2wnp!sf}BJozMjqQ*Bk3pXO?IxOZne%Hhoq~NLl?1 z4W4!oQ5CIXnG_AAp9}pilTq4=*-mrb@1%I3zO_mv6J8V$HXiwZZGi8}T6(iyRXk~F zij$kM3e!k=k7ArpZ@vV5eaE4KHE;qJ9axHzKid|g@jjRez{~U0{isU?x}=D7nUU8G zj_d48T(~-dhY`-_d~w~2j9BY9b2HvNDBIzs9`+fJ##IBo;PSyS*JwM2EOqbhKD@8 zQzr)I2KccDzQx#NUL9Oom-@xq^h{U;9hcoUD-jmR?Sih(wzQw+qzj&eq7|g;($ra& z)wnZ|PHv^hdO>+rv;sGWv4YxI*BpdvwOIe?8WJ1 znb^W@`1!7gN2EOEKtJ(^`aGJ-VXUsInB$Gx$3fa;i`XMN&99@-nS;9fn{T4F&Y4}7FZ_w~+X3oX4i{{XSWIwt@3x`xLV^CRgZkc) zz|Rs1i-x{4zL=c}vAugJ0Yn2%R{V=wl!s0Q${N}3B<|x&3P3Idc7r5z1D?Jwh{i`iy5o)*dmKnoTvBQ`u8PwS8!W5p<_f_zbG;T8sU0o1{A@kVi^jZLOIbnOC6)Tz;i)$xxpQxNCGY z3R< zvtuUgE29c&B$MBO-zx+&16xvAf;@!YFlOr|@Po+L;LT}mK#QFX0$Hsqavn?H7foWr z6@?pc%DomA^@?d3zMl_&Sf2W#|MWwKHOBQdzHa_hbpPTl-y4w=`PqUc zF|xL%!Ts=Hk7mUOspyybj&BI_YE`4ND`eWyt0{+n!s3_CE-IvTlpc`$B_0VskLP`S zS%MuEi;59wz?TvoyNv|uPmkx{=DJm#+i|(s>ZeJT%YYl|>Ik<7?Pi$VRDWp4N{H~HDS8~92@^{dzpxKAACGFL36e+bl^a{cyI%X+>y-Nf1PcJUzaJmM+; zPT&P}Z^L3gE~*~67>#tnXNthxe6tHXEQTZF53dNRnQ0w*IpWsfIn8#QKhlpRLwuDH=sM(8OmAQV=jvk#WaVySvzXQ z_3Ws;pyFqw1WoHYr=9-$J*(bSUT2|7p4jMT?d@NVvo2WI^-PC(qdUyECG<863p*UF zkdNd40%HHj?R16*A~(Xz+kfleGOvOAF`1RzG6pYnH&t3nr>`sg@F}%NVd-`&t8qkY zKws5ID^z5j1P2N}FX+l5(XN+w&PF*K6pHpmq%SP?PyDz+>GpMWjlN=%4ZE@O!y;Sp zYEhj1ED@ZN9IafUT4f0)j%a+vnmLqjM|S_(rRa9(M^>U~V_falkuW!RwX1F!C2tuY zcn&Q^RgRXB9CUBMAP)t2I@OFRy``L<%8e%;RWVdTxgb_&2E&+5DDp zsa_+;*NW!+S@=8X`MSI43}|Mi%lq_5O^B>qsWnE>{FDq=ZHFVj)?gHHvvq|JdW)$@ zXFhIH_nxe5KlBeRfMj32Qe^Z?1P@NstBmJV>_@?D<$M=|ExF9AInl$D0s60ef`8)E z6}5D;(akI8PP4>?OU4e`N`n$>=kbGr-V7*n5oIV>`$xLBb2>e>L*}*Vbg%L(U!Mzk zbjiNDsFNt*>HGWI(bIl|a-kWxRCcZJDXB2<%?b(SQeF*l(S}9S7i#CyNR#*bSrcHTXwQsYcn$}hZ)54pcX4#e`-_w@fh&h$P-fxkw%2w2g9 z8qJZ||KZNnd#Sb{FX5l!aDlUtYg&NO!QH^+vj>W8?l#9?dftOR9OT0)B;K|<6+0Gb zZh0tLA=blhvEsUSmgr1Q{Uql45^u6Cv`Zmn!>};0{KO&KL|wJV%V$xBR3P;xJAi&S zd{Y3CVnDfBT)V_Y2^8`I^?2XcGj?e{GY;;3@-|I2-@dYb=dphCkuX}Je#af#BNygi zfO%_&Ss}dcE4Z!~lv^a}5*sIy@@(n$@8ch1wMw?jgS~we>*~n$Nvk->ct~(-< z=8E8TSbnZ{iYs@Mn5jUw>2E%(Aq8WzD!!IP3yl=kF8+L^ciCEz*HA>w5q9-ZUfr zZ+CYeORFnsFRM7a8mQ9yp}vAh#!l1_!DaNV65>wdz-DTa;aS*AdO1sZN^J=`Op8le zMHqf#B#S-~@O_yry*Fz<{2&ehTlPl2QC94qby}V{8%@5SbQLQo>u~ZMe+p5|r2a=| z2Wj@a%drgAgCQ#!QexsYf!`aCDTgZsbA_+ZVr;qPB6TyE>Lb%Li4(JvgM!862)8Q1 z_h>$E1YaanEdeW%xf&2~H!~_OG~@0I`Bz7z#NY6>_kM)s4X3Le5{J_xGU=BsPq9r^ zcerdU(@m!NmfeY8U0d)k|Fk!?i%qw`rZ==EeqC%4a?wOI&||~|IT^*DJRI+=yJR`u za`R>78@%f|$a21-T5RX&2cx%{%qTNeu-GhVNmUe*zU1X$>V>^O^_)m@w6x-TPTSnh zYOpmN79TcdnpbwdbYb$QhMp9RyME%AJR=YuV13eNoA`uQ#vIW&2px^Ig-0zE#`)7_ z&u5XXQ!xQ8Q_j-ph3;_9y4v^op~>57*eZ2<>d*DI16gVkuRgmS?y$N?9_K&hzE@J; zJu;zz8|FV8#w8^)Z^15+|C9N~4RZ9VhNgv=(jCKg=y{eyuwcp7#v!(Aj%$G5{9%EE z>l@+VSK>!3iMpe=AX)6+Y-5!{40$PR61q`qn5*xF8;9t%{Ufk%P3_tu3#G+*kNG*| z1)6c}Im#9WHMn3=5t`?ZpE!7fdDrWB3rvR0xl~Dy+@wfwI{dqH$FAqOosYP-{UaGf z0dN&}7lUk>`u-USwG(56>R7CpC$;F|yOld$k*`oeT++2E5+5om4?gYoX#Ti^SUQT_ z(P~yXIr^YJsLD-fUee#bATYg*{NWb}6y!0mP#-lEPx;6m*KpwxHn-yYu+!SRzaaXc3{fGh3Yv1ELeNm2K|BKxy?WIG+&mU5+iETUb? zp+Z!T$uRJ^O#m3Kr?Hn_1q3pw(!SgfwX4gnA~!2X3WRSX;yYe1+-jT0)y~`LQvBzjF=0Xg;aVdVl2CeXI>PCg`rq z`-|CFPq;;;iuzYvx5MvFFtoo0^w{FluNbu#7#kB_*|Xc@{Q}PNu~J#K#r&A^igu%L z`*SCW>tv_t-01Ekv*TjWEn&Wg;;O~+c&iKIlR9IE~ofuP_#LNH}#{(@6-ZR zjmqKqgpB%G#hdOIcLupXVtC`muIYiL#>RO6s`MAC26)87-Zck%Xhe1L{Jn2+_A9Ri z2w7Y66&Bj_kKJ3FUaDxAG&)`odfZ2`c;ZL@E!(xgUK`=6#XDmcA*;2z^lssE1jo0k ztn!m_mTK4$G{?JlQXt=c}I%(a88lbw0YX@@N? z+&<5m_q|**6_xz$HE5S7u_y-&$7S>(s3SMRu&I`dPhH)ERU_b za!p&Plyz9f)6+jVZc4-GhOb#oQq6~)Js}-DpADb}Ei!UpsRT`(zzJr4{VsZFM4KEB z+ps9j?fBX-e&9s`l(5~Od!UXsL;34ULDkhEy@p4TsF$Fzvx|j_cYR$X?+4=ZI;Ee- z99_J)6eqLEm>CX~HKrVmPB}Z*Zxy!ay8Qf#jE&1~MgR2h&?~iWT6(@Z-?BB)6jYBh zO{<%4Tz(2&kQC7-ek~|v+)JJ6^w=D5tJS>7A>YMZ0MxHUEeKJ>Rs_9q@{6nz8u}9- zv{GZ*B70oKg*#mNc2qA(;kK-!wpH^B`_>joJy-qZ7l~llo2Lg-UGSz4A@#zD$Z(Tz z^WhUFZ0P9>&$JAM3<__(tfJ6yts1L8dEZ5V18Hro++JmGt+kTWzoq}|dr-%F53ct1 z2TxH)kZt`Z5#`vUx+7gLPE+qV5tc~_W4xc^v(!YP^9wId!D3T|-xVnBRvTXQ@W}>C z^PwUF{mFkE&?bXdvQ>~3Xb`t1x-2Gc*W`t)+4@7(a`dWA5Oqf&Xr%?EOFn(unBe&K zO!km~tQNYy;O27v3OBr3JdD$xrFI(niL6)+9GIa1%Iw^x&mTXAh4@EAdmV~dihqpC zYp;v$nkZk!*@P_IPzcEnco_9^id$X?oN2O3Aw88zL3$tRv28=!KN`E9m8q!1kk@5I zLhOaG8f>epj}+ixm9&gdoD2+g*%oPhEg3rEtSbcrKZDcN@|N`D?t3r{%#cs5{R)jk zD&C^3mea&)ZeVSjiex4`?#rTz4>rw_(UUmmJhGd^l^w@vdR{Ygc8@Mm{Xs5;sJGqy zvf!)iMU|nS3te>h`a)dvv(FQQ3Gjez_dDj!S+5&|ea~3!gntV8wmmr+QiqqwBECS{ zGZGP(>6Zv|Zt0bGw(`wwkhmx*2IY`a2pKQZ!`wmALmK)PJ!{>UJg7fQ zUY0iE-Jh@*JEB=aJJoq+d&8jWF~YW!;$lhjjd%60t!A3o;n<}YbVmuvtDy_gd+sgb ze9IKFJ4mf~fWtPp1&!>PoL@yb1#lxbHPF`f$6P7Q>WL3Hp`KvSQZ)zFKB#d zKod%uNJNeX6g{>2CDW(6m|@~mB)gKK2EHq<@bH30HJ`D)LpF)@)!x@W()8-1#~xd$ zQjvo%u4rd#Rm&b}ey~ouZ+EKj^bc3rAby@eyP>9b2cP$7LwkZ?ZTM%<#B<6=kZ5?4 z;5F~db6xl@hYv>`r@1WboqS+k`Fsx#TznK5xcDd=y>`#ysx!CZZP%mOJRyF16HC|9 zQ4xLak9|~|zeaH`<-c|4&z20V(ds@*CtGnP3V#1^cjqHRyY`*y#%*nd6A^zE9Tz;! z3mQf1me&?vNx8shAdZrAfSQXRLCa3EQnE#aHBOb9_=~G~(5ROBmBC_{thDOi;iB{c zKJ-G}zbNif4`+fUZ!K8_yF_Oj*hTT!Wn+s!^eWwrmDzdMyY>1CxKVt_{qtnQ&G)*a zCNgAnVA7!!h4Qi4e7kybY0L*O=x${P&~{~hDWqfVRxS*x)Bm*4{+ylfbgNCx3f3$ZuXH$Dam~?1*FU@!wa8N7EPU6jOKqtznpt3p z5WI7-YAxzR?&5Tzy_d>YT@|zhiyF%)Yn(%zAkq zIQP;Wk$FYYTAXrV@Jq2qN-mw$IIrU|dv@v-F;{W4^}*n9+kLtjDwc)0YCJw-6|z{+ z%?$%^Tz6GMUtU!Lq3`R`$?gDlgzfy*aOo*WVs{aR?r}NeC)%d=-JP)T4IZ7oX$Moc zi9n&}Ug>|REqdtdjOtohX}>mBEi{6r;bCb~a@sioDQAeWqC1}@e#u6Qx}`VZ0+wgG z^=Cc~AikKgC5kkU*?GxI9r!4uS0y{tc-Kqi*W~h3P#IL~;%!6G^g}+eVzPxgnr%lX z>s@bd#VnXw>1*p~rJcoJx7+C7{>Um<-gwimV`DYH?Kbmgn8Q6ilRaWs#r3B>-yPb{ zk#+F9Yzw`9Bg_K~D=p}P-46o)s74>OR0p4BCcy%9euyYh_<7mrW6LL1p|yED_X)Fk zJ@#eQkrh{G>tqjeMICU~%Wh5WY(lBqLK6>zur}1Z8CZSavR`a3D61|oH~QGG4k6vN z^oSo5d*nl0_PZld2wyiqp};k|kW+ejOF~>`;Z*9Pv8UW#=j9B1bZyms*-U;xDtAppB^$`43o_;(Jd4ntB!P=d2k;~B zM5DjRiN+f!`37xA75oBB<#DT&8Iz8zxG|!fl%e+Gk*G`WZ%P@59#cUfAMCO+EjM5bvo;)tB8@>`) zCyTN#0%<6w)_h*NMAPQMnH4a%t%4-Z z^$((GBjd>FL%bD(fzV{!>;*%!QzD~eKL6LU6S^MWJ zljOz`n*F6mjb($(^4PO_OL<5!5VC}xIXByVPxql~btl8B#N`j?WIR1Q8S2ffQg?MH zZ5uMYgX?>pGNCmy9%ieGS@Ot^A8zZJ>ygP&>1G{!+SBf)RxxBgQrLp>wF7<&q%)>l ztXb^X<(*4@fJ~~lSr-0o3#|!n4`|7I9ca9T_yhK>zfHZyyFk=7$WEr$HPU4|j(=(( zw0Gge9+hS!q6H>Qou;aI@v-g-1kK+DbLxK>s6Y9k5`>FUdDfcmLBZwJ=(?{Ta>ktt zYl$b_7uCzWMJf2ZBE#k9hNATQ)k|4ltNo4R(fZi zH|#oq)$ID7&RX;tAiIR<%<;?Cgoc*qmf3Pn=&)$9ZNyvVC82-u_cDiG0-AgWkuf95 zHJ6;20N2a19FT<#b5MFl``M=InC))D@nzM9X19g!?as>1>shS&h6<~Pjpt%t&z^I@ z6~tBzH$%a)DERdm_^+i<$u}Ji)6el|QxkKuX3v-T=){vx*l2W8`(&}@HDec9`LPy( zDnA_f%SmfyI%PqAJ8A;a+8aA+7hoA<$G#jBt?Bk}6M_bs72CUX85X~u>iFa}y%aJm zGhvYUkUAmemzig31b~Po#K2^|ZjcN9(R@z2A5AIK&&=FAP%M6Lz;8ZBF3#NY_Ns7p z_R}@dB8!rv=`6)}I+CoF1CobMQEv_X>gC*R96s*Jq`*R2I%J+2Flj(_4RN)WLhFqO zn@E%_x*1>2?ReWOihtBHyQO8Gr9%vgNW5WJp8y#&b~7%CNr$?`S2&=o(zm>y)Zyw& z6DBAhjFSo;*gWyk{pIBtPfpLG{pCfTYNHXl%;GKla!0YMnZ~61>KnAPzC*AKLD4X~ zMl8mMG;|{f6^zJr$t~*tnj>6Ua06*PG~P$eLA5Isv&p{Q_Z`w30@CoeByWd+4`->E zrX6mhb@m2R9sFo%s?>N>2-+?PzV0IHPC4Xk2lZ54Y~_*{RkpcB3LfYduwieq!8)f( z^0&Un>_M&zDUSu7FIKh9ZC)~4DJb$cec+dC1`R?7FmUku#X1<57K@^lkdaFh_KYrf zLqA*~DfIQ;_ZX$L&K>yJ`G$7yrh&DkKW5B!ZRZ;h?vK7>H@gf~9uYfIe7e?_z#eMC z0lm&G;7Qk@?cB3tv59CIQv?>&+%j4P1~r3iwQlN;R{PW?c9U#@7t4n6Di`i=3F9P(PiH)VOqEqR=>~ehHNL>PVeY^ZBxFkcaeMx&@3Zs0?fHHOyBv)poyT5LJH?}D!Ux*W}i_w18*B$JK9$t*|<+bQNnXmHPRXX;|f z>I@@L!yNu-@9cdEpvIMl)xjT<$*%G-dom<1-tlIfqUIrPc`G`SODb{)R zK{?6_Io-pSa-q`FRVDDDBGdUYwr0br(FdcRPhCC2f zq;o8w{VT(!tEIAZZ4u6YU{*=;Tc0q!Q!gvdj=8Hm`Dmum9vH3iq+2Jglcp(qR>yTS zLIQz7IWy2VY(jt2siS9l2b)f|`~V{u#4PXDSPQNDB6>QE{+IHF4UiH1itFYaU`8y} z8oQN{&Ze8Ik4x`a>bc7H%2_~k*qz{zX>-FHEb2QM1m%5^&pI(mvJjx5<%fGhCv6Fo z87r-z=2Rc2ph~}%8MDvv%MOSZxv1LYVkJvbBJ|t(Mydv}eTKpaLSa44!fu!24Vf#h+Eob z+;9I>j`YM>R-f6uUEuTRUY$Xb6q0Q-n^7JY%4bS zA{=H*;zc#8*2Jhhk#3FKx)eI^_|kvb;Zbb_TxZy<_1~&{~8QvC_sFkH1w4Q4CT1cM@AYK!z$Y9pTDVxugum@zFSM zTJ5TK+Lgd79L@FX%EA5OA9s=#_lY4fEjkA`M)(<^_iU>7Ve|H(q5pPZd7{7MFTGy; z9IIzmGoR*GSr<0%6D`f=Qx0qv|ETSu+$WI|vfZw$qZOoVlj)n9>FY}nR@xnoY5N!T zv5(9UJ414N zD(Agu=pywk3gMU&JG5}7f9T8-eD{#H)5;APLJsqgyq^Cb(C=qD7kHU(@-UHJx1`jG zwEx2;KzJUQTEX!j-WB!=Y&CC~T=5L>`JnV5tT$w;EYYH*fggWgGi6BD&7<_+c*ON^ z$rgc{-_E*7xa??|5S#=yoQ)#&2Fd(w4BB*c%8 zoZDRsR1bGd7WnAzVS(t@_L`|I25o(HuP>Fr7Z3_Jz< z@9!hF(ZhEGM#dQj8-$v+z-y#~oz4duMcJ)Z>ceWfu;F5xUi&u6h7RjQE;%112U z!Uo*N@K`mBJ*b~Ns;@r3>e2BK%lCgfxQzuKz(xVyyj|Cc>j>5l(GLh`zDQPnAR3VS z-T%u2rhvAnp!LX( zi*e|G^VLbB83uo2(#r{wZdH}DU(kP>V*K0NSzG^uu&;dlU#I;qIGdKWC%s0=88rNl zlM`Oqr{n+M;cUs^s!C?D*#G0V{_jKn|NL9`Qs@)^&;tDb>%7(d{)c9_0vqr1&YS0C zbVeo^;d~gl_lKQrfKe4&5L8>53HajK(h_UumrLealLW|KROdujoZ3`pR$$^PiIxx9 zCgzqQ=iLkfag>6BFGbFjkcXhj*=Mg|Y^PJUj+WGle07E6`i+06klKAFN+ zcK6W&Q@tR601eObxhjOp*qK;>q-X8(oT0cLfJXPy*4ZV)^mO)>xD@Ku1+Kj~0yEAu z$$yWD+G83WfkCWwXhV8qfXx7BDH_m{@FTDp7oHQ=k-nm5O5kf0l>PiQ+1&JXb{Z7{JPBf!oblA)zts*}O91{oQ?h zywG-slVXUy7{9LpLKciPk&o)rvFWmsMmLP_`a*;#$qt8Lt=ZQaKi3Kg!(iYZHq z1n$$LVZTWM{Js|brFfGAj6*|KuXuRXW-dqUj$(F4#cxjn8OH?oB`#S5 z!H0VE_oCAKTyPd|3Dk)%sBycXv4*5W4Huo@Chsqbkxu9oUsS-iE}P2dXlwAe{ojVq zx*4~Gm>K2h4I+>|oqMp#E5&{!8Zz?I(1NYE`4#0s=L;LNI8Y74Fupi@G*82J2Vj}J z4ba6k#DcchWnW${>b@wsP#8*!lz4$Nw6}UgId)1UXw&_+nGfjXggm5D(II(<<*r?0JUj0@(w=%Q9FQPeg{_ubJe3Q zxU>ON`<;UWI__OZ686$y0}tIs<=566^+QF~1DgUmL@ui)TPxn>DW`iFsM$hlF*4U~ zbNor{lh}VHGu~%eO+kl7{A#XtZ?s4>cnh7jxxyu(<9AA7?M3}_UFV>EYRX3|KDx>J zw#98(``uW8D-jk@9=6H&<$oT~$_47z6&&xhdHZZSO9V)Qt<*>NfhmMHc6&eUR#tHp z6|wijkALUh*l>e??XBr;)9vF2rmbu%tXWWgbDYR& z1wN!Q=#_uo9v!v^58Pd{YaG?I$tV$u&)kfbx0hfF?8E?u5vDNg;M9R~u7h@I(A=`s z2U$SQvzRdY4&h}k8;|h1R`<*$=dEe574?@h#5aehDh5BS<{H?zk-4T*63MC1geOnF z{O$!>BpP8If|W<&RQ2QHR!Ui=tbb(q))f$M01P?#k$f=x9o|*nJsf@Dj`tj|_0>%S zJ-UI_bGe&H7T`0&1i-2qQcf#RCjJq&c)f#~0}an#I=fFuedBB7-lPDVorgx6weI_m zEgg!h#jj#(&Fr4PO1AIYeacmLp!BM4g9?);-COU)E|4;p(wM-#D6tvz_RA4IG#247 ztZ%*Z>^#4Y?K<~uFnerGE5@IB)P3`y#021HVwnAhhW{U#3jURvCGVdb%IHlEnp?zW z42(O{b0(RvAX?+?nkv(3<)98@HA|blRJF*QUPe@v?3BizhF;!VEXF0Pz_Zip2S#V= z0c))xHMJc8Y-wM4F(uoch-CuS-<+@R!0TVnmFG2?>UDQT`ksjRFI{Ii^gZ9qdS*$hF9jqs2QU)PtWL{+ z5~H$?7RX=P^&XTk;vIHD__Zaw6SptNK8Gw6#dWr?(|w~SUF)0fXp;r9R$c0YN=O%8 z#WlM9!R?xUioNqp)=IA30JMiJKWQB@FIX!U;?bw1m8*iNhJMxqf}Z>3-RCM0x@*o` zPYGl2%J)#t2BvZJ5y$+1Clje|k0#MG8sG_Ih?1!6Ny%tf*fu_T-yuZ6>)?ku37Z&w z15dwoyptI=u~86Q@U_``w`BX#Kg>&t{@A|?nlE3Xxqc>w-H=#YnLiUo$rUqY>B@@=QUQWgElK_8M2jgSXI z*+y7Uy(zz4z79UrHVBZ!{woy&Dgx7a9w1YMNKSQpIZ#ksRBbmQ_(>+{(0AnO0@71^ z&aZ}Ih@}Mr)L)Qy?)P0M{T&S84&&7fC&)?6YCZFdn5&$9%G3W^rl-WM+PmCX`+-DB$u~xZm1`KG4;9L_ zP0w_{ZwNI~g~rJ>Xg@tCzqBc7s@mnEQo7uY^}g|Q2S@liIvu~kyi}%Db){UjI?MF5 zJS?($%LiZdBGySs#kN+o)omE{E>~@LEKEEk8qHI*qKH@TX8lrdM=GsxX&^x{-5t{lAw;#3wEUdX?L#m9XiU$O(83gakF_P*k15>tJ1_en_-p=V z&exNVC02^3toX4w%7`%qmdw_^FvVmf@tShM>Db(gJG&nPqBE81&m1h4`4O!0p}788 zjJh=2Sg|?|6qv5eqPzQk5Z<;386hOUq`o5D8s`4D;0Boe&F;s1{~JecG8_}QRqYPY zPPRi6!C7oqF7fJMeS3`Y>jm4F!II$-zwm*1ZuIs}mBoxe>|DpO7w^^tGLg%gbGyEp0*VKo95KwkOV^r5kr2M*K zn7NKDw1nAK;8RY^V0c4;KE|r6e9nD>n6lP+^^pdty)hpXXgLYk5K>~Abi^u!8b`M8 zm8*`oU{q-_o=M=TU$BgIye!E)x(Y#6)5XOAi$QlXIr%?Lsf1^F7S7u%-^TUt;Cm>C zmYv=_$c`!sW(n#gVM>CYJl1*`4w}g72x*CEuY#{@E86WGI?})TWX-iT@oB(%gnY80 zmF=muz;`mbBMl(;p)v_b^QONjXfHN51Tm+380DC>O8anU&B&A5iF3&n3z+`G7FH!& z+w;`AH@N?sJzPAq$yB;T!sNK>{UJ+}sG<;gBBbN<>RjSe-?2Xu*B zcxRXetNd6mWm-mgMOlw$Y56xBjS%LnaJyLd7oJ!FWipHX6IQeXXOiNaIXW)sH~VU+#f;_2}=ShPD?!&hTl+P#g*BAtE*M_h6E}`eWO3&Wiep4 zQiZ$H{~ykE!?bWtCr7L4EcQvno0E>m5{|)df_q-TYU(o_!tLrvZO#~&x$EW4^2FS^ zt-8L?>7_W8hQwUj*s!!b3~+ACH)GhhFW;?C_)_1Kh|lqXH|JzB%9PltH^fZ7A;`1O zz93OplV~DvNv|~Y-%o;O2qqWTB|rPsLwl)Bc7j;uc4)+j7gLBygr%}hw}}5(z6~t` zn_MB`hOWGPYJ8^vBiiczgwM*2uq{MFtMVlNLT^nfhV8_;rrOSoG_`3tQviNIcBN8K zxn^6GX54Oy2+62*d|}4wRJTGef`QGQHL=8!!q-@{DmzV4@dVPNkuQ&QxLnU_?8oyZm4mTqy-$A@eiYU6Iy&2c~(B`6apnd z7&EK)K`4D+dN6tE0iw6Xce;q3*7tA6aetKuuvAO0%KmuTB>PSST+0ZY34Gh4^06=9 z_wKqvXHHr_1g(|vTR9Ip^y<|ZM;Qh@4E`dFCGi&*cA8-VO-Ddr5XoG`D+$(x4tovR zy8~=Cr>d4Arz4}4>%XxG1EF(3^11o*e(RiA7x*82I-&`LngqCgp0n4xagj&18F%;# zQ3Ada`DL+bp$yx8c7iF_+oZ7Vg7=;N6p2;Rl~0Cp>^l_oXn9}u+NVdsaEX9tc7B;5Iv|N6l5%lyX#67=uf zetGq`cNa4@By{a;RsvSaaLWNVZ8d^gL-wfm0*Y-7fQc(RmXozxQ&);tZW$DHZa#U^ zmdgO2%wJq*>n8f6H-3CQ{FE3?b^7zZay%9 zIdnYdOa8WLZVsVcYy6<7>Rp!H63n&o>$*8;ky4&{zpdsew$m7v7NvdSAb&)-{b}IO z)Jv|_5|ohaRGWruNw}dUchad}Hbg?MP86*{^aX8~ z(mxx3%qVSx1u%%8QSsa#qV>u-Ik?Zzje#zQvGEgz#X23%??wJ`R1x8E7jK6Ry%lTB z1P05esC!M7*f&RPZqaAAfRUz;fZ4+*=X)`FRIDcMf|+4zV`;QN5T_>VvQ~C#G^(+T z08D(0t_P+>IvQ9BNhnD{w);_)deAYyVViVaN=XiAeQ^YsMV>YZOvDZQUIpw>2rph$ z56|_;i=*fxdv&+mJw9e8RKP&Cps`naADbN{OTr5?{m1fiab&EhYM`3WW(w-ND|kg8 ze3^OB3bH>{@wsweu1tmgUx&38%T}%>`OipF_6NZZ*^&E*xtPT!43k?LM|>Y&%6ZXV zS~D2a_^J3oN#@806i5HQ43lz}7OhMF9R+d6fws#lNGQmN1a3-VQ`c2|{_p$r_w7nK z!h<9ah_!6=>V-WhbLyr7eh9*D!$s2O#kP|;#?r88alC6%^MCmI z@^~os_J3O>TjdeyRMl`;b= z734&vCr|~w`d=@Gb=(0v78xkTWNMSlN~|%OD!W)Q1L(~ebW-tk}K8O z62UK*5B`Xbex)+T?#H|gRx&TytAH;H>(y6%AAl%0@37swHy_zWd{ZyB*s3mhmX(3N zwT1snFnDb$cOiQ_FDsgt>&6pVF=#{Wg2>i`aF>Z{7NUkt)WI)&N2mw*sRfg31e5v8;Vog!CJ`=V^+j-`#Uq34wfS}I)!4K z{7#2wv}!gYFTH37(>g(}VIufF1?(#0y$6N2pE5xN?VQ9;zsvMAE5+G+A-iGVJ$EX>?`&FVgg zlK_vDEAwihWvX;DOs{T@V8^wM9Q@I%A@7Ss&!QH&d0!io)UNSDh2M<{N%kc?}d>qp2o?#3d;$Y{F zU~blMCtl9L+7W8b`Ejh;9e432Y@uUwi~@mqZI_cC2i*}WK3sG}_8BP;I|`O+AW{XE ze-AP&A@C+8-BwE3)IvF}{&`#tvFBR*WvUk}GmhQ#*|Hs6FL?kSYX88mTZQrCxTsOj zpl^2EcsaT9MFX7BgVonsSRosx7x3{Q3V(FNwr~$}sweC7hoLzVjxJ_rBq;e6Q{`T` zjnzye(|KgUy|Iy1SEmId*useJfMy-(zO`0ew@%a2#;V8`_V#LGp^cr&@EylVN|OY# zt^-AJfWc#q?k1m7{N?e4+i1XRH!p;|zi4IF@`pK_9P90SH=jecs-hcg4E7765QuC3 zm$jRQ;{<&Qa9eo>{@J8hw4Z290P!)%uzG2F)P2lwYdK$T48Jv6e(05&ATyTqb~Q;O zGgDKQH0my)mj-tgn@@7~pRWo`Rkik=PmFk^IzI+KQw;dwEmCrHd>Nedpn6~qXN?;B zc%!Eq=S@!TPLhrE`j;(JO**PA*yt&%{`L4u-t2q7dm6*YvGNL;Q!3jInmnt;@lbM$ z0sbo5p6!DJ)Esw_T-IMnx+63ez9Twa8<3cG9jy{DW<;r~VMZMpj8`sIeVfB&MA;8m z4T>}pq=7C(Hhc58^!f^Ga>nC>T^_S2LMQv%p@bIw@ICl0uenc%N{d!%S^pl?7TtRLJ? zFf0`Ha(RB}pE zx+mH=5e6YF7zQmAdd)i`99vO=1#o$E(3=17Nm|(Et$=gXx1PFFzp)ZUr5M>6&M)3S z2*$X3eb7h^c`2fD6J|4pPJi zoREKLDm)^Uh(zjrs*Hf(CwnH`7dG(E>PYb$WL^u++$NF{bStOYG_YrkyIN1c059y4 zXxQvkRn#Cmynj+R54i|GAl;_f$Iev_({kVSsTokhyI5bS&G;6{kKv z-__?&zRO?usIW|viq(ft? zOvCWP);!+57#+Q`a`S<_gdW_W&oPVe;xWR(ob*2RKVI}-&eRmdcxa-;N|rt|QUtS^NDjQCyhVN%wvtI>1I@ zDb+*n%;5kqKt9K3qjA?ccexYhJ!ylvnq($z<~#dheflAseuZ6mZIFNlh8oX)h!l6& zr#6MUkw%U5z=ZoM6{U=z%}5x(H_kryVMEoRmOJFKJ#?Z)ojPBYp0AZoRsUdL!7KCJ}W-f*KuUA=;31c^q3yDVIRgM$v zdXW8j*DvR^WGUfne780BKEGgzFoGegRxa)>{U%`I*|hgPlD+|Z*aJJ3rpC#>cF4in zh#e{Jv!)-_HEZ$K+tV+Yx-@!?{$TawS#~;Ie(8S_E9!$-rA8&_)qQ|+nWD3z^9d!> zbEuS_!ZWnVfAch0y^{N#VL5`|y>t1uZ!>#D42Mgr4DAuMJ#x+*sYk*kM3q*aR@n$T zxC@Xs*zy{cf9%c;>rS*6~CgVBdloPJKyDJpmdO^E)?ewsQr0 zG`^TQD8;bz@s*-YYn=B0womzxj1Gc_$b@Xq#i&hr3IrW)(!(6CVudX_#8Y0x>p|*P z=9AfEb+NOv^*UbH8&nQ{d~zA-NaSCwOAdpVbnLwf@zR!2{QDC;%~*Yx^$=i(cuqBV z&P>{(K+d1mt7nI_1VKPsm1J*+BjA$Ay@}@In_t~~ZoE!XUSL?Ga!UB4tw@Qc+HbeO z%fg*tyB%DwiEY}LZg(B`K>k6wO;2((^}z!}1CAwAgO2wxRC;H2B^TvPtK_p}&qS-7 zyXSPRRLde-_eYn(zzT#`)#t01%DQH_ocfjl{c8NPH;n)CF#mQmPF@#g?y`mygNIZ9 znno<8t?OhlEU0|t30nD>6b|&GW?KI(ozwdVs+stiAp`DfV={4_@1ih=u59h0W0IOHfUNc`Gi|h);$Cu?_s*2&;U=XFge$yw4L)< zAUC4)jEFK%%>)6B z%+?EufXt!VrT!d?LjKqQ-dH!5U!@Ch{k#iAAbvM6_qsG6?Zm~i1}HWPTP!(r1NQx` zB#USe_0@JMdQ75z&w|dO9PNx0qz1a1NX&e3b_LVCl+HpZqlco_oHMc9OWcl9FZaz0y7r)u7}` zhp~bkN7*SOaLbZT?(yV#68dUk~JF_@Shaj)iXv`yG_`|JRddr#}(|!@WSAV@X5c-+4ZX zssBB)H84rV1|O*^uQ{FVLu@y2=?-pcXon}8~3-u?x z`(r})W1La-aK{lwB!F~p46=U@Zd7HHJ{kbemGzA3DKy~e!O4Qajv#5n<^?FX%?Z!A z5@HAqfzLKmw2a3^WppFTNyftDhq}nb2&JBz5`B+M5E;8(2CvcO*~ce#Y#w!0(w-8LiWzui`wNA%SLJb?~5q^A99C4>urkK); zk&iY~C5lL($`hNaX3;w_4GUd|3MIOzye!I4Rblhd}yWA+*!S7z}EI za~~uM;2t9T-7ZbIViL*d$wD#0ch$gVREFJZ$HNTdKEAs@HjKrGhzyr$| z)g3#}d3J@F;8|*W`MfBxU|6lIq_DQJpr$(sY?qx^ts$DZLoIW5BnZkuD8?z~sI>&P5(wubqp2&8>(l zU?a?aM9&YsLT3&Mba!5iY0lGpjOgA;^CPJc)H#$ghJdyx(~acUeruDqxYuevSj&i< zl)yr60aCV@LzF07X*nY-S=?#wqL!()y^wdOL5%ABvM43v;cfI$2ovJy)~8W=2^*l% zjMwrdG~4jZ#x$fAWlDc+2UhSBkW3%^B4DNWzK&71Ejl0p5!zhrb7W=KN@MkdSDf*J zQTx>9v-=hxEHKT7S1mxs86e!;dqEr9KNt9ttPFc{D(pHIj1@+l9B-Tm zj2D@VkXFZw9rxMzPZs)JA#QckZ=Z z-3`YS>GeMrdu0ar1p@bG$00a&kSTGv8c-}5-B1s97F=Hf61Xv(=eUV28wX_10>C~R zAj{btOKCpPxg#p2XE|12c)w;n116S`fQZ%Lh*@u8M84qWg|~(uhbOhMG(SKik91Q8 z6y~C4pH&K(}{g_*J(Ur-};h`SqA z$KU);qY`e@)^^rJZquubXmTM!@!v7KfGq z!g}r_tAx$`a@yi7Sj+X7Hl4pwAoVPfwvqe)irU6TO&d6In9-rH|wyjuK#?kqp!zx{l zKY-X;d{iCcM+t`>s>AF#rFX*oaN(?iNgCjo8>@Eo9LHbS$tpX{N28blY}|J(#GSF8 zE`_6zpviKZtQ~`PVJo^ArMMT-Dk3&cgGP8Z2{QIY1V~zGaI%ITl@X@>GRjYT0j#Hn z=*+_GbVabYKbPJqyPqB*_kjJlTKrHd+9F`Up-Mmzbad}?D9YaMU zi`MpJ!$eKAje#A+Fjw|qeBxnv#Bb(=im$m<>Ge%eWx^;VV3=Smf`%f*OkoR-O(cz_ zhs|Du>=MeOz~LEm%61xh`)bo?)Z6=+`;VIMa&jZnD0hZFLV8Hb&N)a`8>cQkTueUs zyXKUGQM%iDX$h_%fMC$YZfhfjmWfnNIK9K(Yg-&2x5-x6qmMK-R zt8X}UcNZl6<~MdVMz)7I-r!wkKlO?pwUd2cj)*HaCq9lHqZDui5(<9cZj(qDou+EG zWVlvL`J6uak)RL<`ta+wLZz;)YdWZT!^i2};MWJM^WQjdv7UlrP; ztzeHI_l|BB`+Z?JtY(IiA1Bw(SEUgPJG|fhfv$H-0@gdx;62^5SBT8n*b!__miOv) zNP|>68;q+R6$W}LhA+!-q?LUr0|Af zisLW#Ngv)sf9q=#q}ZX9;Qm!Y#iTBDjGgUzE}0OA#uYY|3NAk!e+0@I20Q8%YwCW= zZk=wA5Z$xBe5m2$w4HTalUn&j`m5mBC55|7V=Cl?)oj-{BBFX;axLp2=BrhPUg}|5 z5CdOI7b9Bfo+F=mR042thBx&we!j5bFfqapZdl(?)i^kRt7O7_!O?W+5}IUa)Yrqr zrnm1{ zq7=!$(DMTjNpy#t$i##J`|13=-Bj@dE?W6j+2W0iFkea0=ZoVpb zxAAn(SUCc3Gr5z;Lj4aPp?>f%il1t=c3o3C$^ZAD?fcD-PBaE80JDS96-U#G{VrKm zqGYgS7>Kyn2+PAhbK-v;?cLyHmqJ01+1R%u?vQ%xpM5F)-~as3@%^{51I<;#NkWw#WJADsEFfTYW1ihUpZ z_fmj?exxn9`Mi}Y+(LKKqG7mwlT1clMBf``ApY~uT3dvl22b zj%BgaA@D)JAZUQtEhzcAj z6OAy+>y;yUfx1dEtzMz4|E!dKcC&p-*5Iju?6}b?=P9EXR~^)Y8N&Y_lq!l zbFTdp#OdW?Edr_qws6{-R=JfMHD9G&ujZV5SfTcx=IVq69g#8A#6Q%R-X|1Ee_SA= z;qP7X-zM?QA|t(Yu7%fnm-ds9?GBK#dw2JL^G*l;#|9=pHyvl4DBqXi)Sl7%_L~_l zF8;;ex4Bw{@et&#*U8q+TOjUOh<@^2BIT)iQC*vjMiPv&| zt-7UuuXoj3X?(D3!))!^0PJ&@l~#Gd?@52s15pX~BLL|f4)xwIim8aK;Cu&`@P?JO&+f#TvQFAJY#Rq`BD8D1@imFo-ojFAb(8so>F#m%BbV=ADH}Z#+B&YCpJ|6MR3kVG@3|<)1fsno+o=C4LSBAMECIr zRs=mMQoKIz=qW3B!z_2!Iz7U`S3q}-ZBgbLC+l7B?vbfK88urU{JMO@l9Z%j3skR_ zt-H{mNM02nhr_Aeh%uyvo(L9+hSG0HwcW2C5iH`h7F4X9_x@BM{R6Tvs*C#Q_v@2*?Ozx+Y(v690$ck?tFv7qe{p&C+GtIaF+ zZ_xAShvX^79KO=kJM=C+*>iTt2mR>oM?Bqg^VjuA2CPu;uY8u*eVL*ECO&|V8P!)I z`E)yeccbYYV$HWmhRM6wk7OI0srcyol4%zIL_cQITGUTEv~OOM&yZWEccgc-nJKL7 zb}20pAL)({@NWCBmrp$+#DAl)Wc%rCL^n7f@k1c>UsEGknU~yd+?@Qm= zzq8u775DhRu&Td84mO&b75i4Nr$!5U^h#mAw?#%=)p^Sblxy>@qYt&Jn3N`ruNnC4 z%|>wOB#FaShz0}96j~(w{B4DY^|^aQ7Pu(iiiJ1JKxdKi#{&0quu_q z)kAN0ceF8VYJzXisH4ZqJTMITm&W9%xzV0hO3PoL=EEv}IYVDPi(ut7#?kDXdQ6U! z5{Zu?zS*m1ZQ6&}aD~nbZ=UI&`UjxNKo4T1NTjuEq=mWrrH%Yo@VfDeipLFBX#dyJ z{)5Nq{Fr~L3}*hjP4u#}Ua`pay|9RKi*Ph$=G03T7U1+zuzMReZ>V}#kQL>)<(+8-c>vi zQJTNphLiUHy#uE1U~ask_CGi+Z;Okx{&YKQdH;0a&!m7A4}5R!D)IJSbC6-@iTHK5 z*f;<84ph(UidTEQ`P(+Pwyd=P4}oEA^5H*|iQcEcicxK0@#gDm*s*8)((J$Pw)851 z*ZvZ`&LiG8yuJz_=ByllHl3;Uju#$i3s;X0@*X?elZ&s3jl~;-{W% z>5m3YJ}#9B1kNT;1Q;T#Jy(W*Mmbx2_!OVZxn$fEu2b-Vr-Jk<;>jLe#a~%!KTtV zfF9-;0NqqI^BlT`p`uHBiOTb_n&CR)b=hN@7-1453F>D*QfA%t=CzXzZR#X{D#H;{4scyytU2i!? zz$aXQBFSTx-|D^K4W%KynGpgwIwT307LId0-0YACVJ{m;p!yj2Xq>L^QdZIOh+$E5 zgZn5uTsUnKXaM+Y<*$FPoF=tY(Cy2)EKV{#yF$jP&=|F2Ul!b=Y;DfDQhyS>RM#SE z<8;$fU&;iXJEZPnFtpNch&X~et~>=y#UgdP0=ZRmz#)G6*xMH%?ReGdPZzlWoYRnS zu$y|#94*HHz=eF3qX86VM|nShftvT?!54yNLYR0#Q&gc7HA&-z0~rrI{&pI8e|bax zH13{KMIF~kThPC)(1Qzj*`^hVoV9MagSCnQU|J7;`Zn!M>Zu7?I&Wmy#_2q&28S#5 z1UR+Hv%dlGy4K3;O*=Tp2r7+i+#HBxS>FE>qkcf8E3a9AeT6f?gIW@DcMuL1Y)3gM z&bVt9wugJ3&qsV9H!s=20O)O`)sSC~4xH-xho~BQz&|8;6NJx2LQ?JO{?_c05BD(P>$p^!SFo^sK*BZ#4EFMOLy{K`3D z2Y+N}3Y`|dw#3WYN&)h?F zb{u)~Z?_4EC(f5P%W%@Dxi7uvoXkZ9`T+4 zIR}2wbaH&av{CADeqcooyvl&5vYX4#LEob2jh=pGRo7Ho0jyPi7#KWRPnG4}4vPRf z9$babM50s24_=H=qwT3R;<-{God*Np=G!m~Zw~IrZE(LP*rRqz5i|=i5D-^+FVubhS;viL@Gj$8;L4*EXhm5Oq(^%iva_ijaQ;6* z@g6xgY6Ol2N)M)v{?8=ES4Z-xZI>ry17FP?=7y&Y*lNFASS}<%q8yHh*q+td8*6g@D-wx;?!Db*Hwth5)!C9eaGZq$ZfQ z`1p8Uen(yd&gR7Yb-&_DNu_IdwBEjMOOHgeitBKHk}h*{-FeJS?jLB6Zr$Kxph1L( zr)^qBI?8X}ferRx0n9T1)IYP!SuxHk*-kXI^f8cT*}tH4?x92Z?fY3*LcQs1>D)r?==_`RoGDI!Sz#aAu84gR&Tj%!ivE=McU@G^h(I}5ta8g`y6GWXLr>k<8$)HMcXGZ< zK`U+7OgPO;8FB{jm6T()a#V+5HSZa(vV@gfX4B6jvnR^dBt% zIV?69?$Ake z&;3tkVR44)(LQU8sDO%_*S|5nrk(u)5%7_YZgwE*lz!WKDDk?1wyvq9vRkXMAsZ(( zCg+u?*T}fy%sXEOtt%L)&*q%F*!0Tff3h`gz+{l;m+lD}qeD#Crp{OD4WJoE?&*h_ zgT*x5$ReI}i*AslXJCL>eXIKsH4({We;4L!YCf@&HgH@l9cs_sb&J-oz6H_SG zKuRglYRZnEyLkL6=zkW+EZKr~yI^s)Wjne+c8l>b@w(&PSZVPGWzhFe-sZqp2d$>+ zyLCV@>u+*vU|x@)dk!WLJp({22#-g5Uyt$BdpR^`#kYW#%NEhA8y9jk5g?QE>xDl| zP#ll*t)Il}W{9kTmqPek?J<+B;X8-|f$p(^@qE)s z2TTJ+43H11cE{h9DR!@d)&~&Y^>ULg>r9o4crI1K`E3pG*|?jRPwWNrt>$a|GZB z5rNML>;ku1s1V?V*Bm7v5-QSvdw(~nhVUVfKc<0jDN%19$GB;4J|NN9{fO@-8f4dL zQX+!rA>=qgkySHBlxJCj#k7zlw)Bk+~#O#>yf#kd~?p{fit9B&BMbP>fpp2zCatlv_ZH`eF0?c z2T+1yJ|;nx0!Z;F{(YwRZt$Y|xUrJzb*8y-{P&vS0j1TVXY@_bB+4B`!0e9STBO|l z1*4et+$j~~+W9v+yzOP^BDpd6Bj{t-Cm`g?2ONyV&**+@jPk^X zLCCR2YFqPwQjU*Z?U4A|^HRq~ry4+lt?w@E&xu$se^N{9+L#TKp$j0}hdok@TZ)UQ zt`jNk-%)}b{x|~q670SXEH@C?zNlrgUoZ_ZaN+ATs)s*pLg)5sHY&v$^;hB&1(QiP z4H=VD)m-rg z+cI&q>^uv$y2(Y^Qvlb?+p990#(*N>Pg745?Msi#>(<63Wbm- zZB+vSfA@BM-mZq$x@di#L5=-K#{ngMO0eLH+FW*YazR%2gm*?n{S&mazL)C$d_1P6 ziCjEn{GcNXB`P5el;I%i$@!fVpg*P87*(g(P#i>9Hkcn$9-`Q$#1;lVhaQ9k`Cbo# zgAx0)=Qxr6`stWfC50uytD?pX_dg#g{nhg|dX zaR6Ewq;?3jJ`%WJP1Q%J<<}kU;PEa_%9aeu(h%xwya=jd?TQYbIox^-Bb&Yv z_ANq|Q~GE`-??U3G^+HfUX@N$lR71^M>#tB8?_I~dRzM+5-xBQVzuCgJ@L^|HHreA zg1pr8DvgyXVFGy$=7B~H%@Y#CpzxAUa!dZ= z!9O(EH|xoZfA++_5#oCtHl`}>a^R%Aq5vA6*Gk$?j;??=?i6y2If6ux6sm;u37UF% zCESSJEa96i3W=+4U3?=~v0{5nnP$srkSKNe)N2){L}$8s4<6Z6b@aA4DLNH_^XirA zU~Pzhhdz`oKFlV)A))(#x@Yh9dOWvjwJkr_s7*?xOSf=@nky!znKFh>7AIdSG}AMH z)*t7H^~vcQ*#nYRFFsB=J(69AJ6f<_$&Zgv0|hxfZvreJ#_0~)zN#maKZ2Ek;`9_K zK!8L1(QA+EUIFxbQ0J%|PHXul+SymManZhJp}=4=n;EM`L(BU6GL0>O>w>7lQPyUl z&Bkp`hWd>2>f+mPJ3i73c2L|h*6guLfK}r|*QVQy4pQm1aurig1DbQ3=cb!}X{|JZ zS+~-JTk9ZSMdZZcuN%iDQ*y>Jeb*(@Mt81h)(zXo+u)fv4hI49%nu!*=JZrU$knKt z^ypv^QQ*AF+^KNY{cGldKH8l$p=77BlS^{cQQLfqL8ZDH1$tc+O?%{@$Qa{k0Dt39 zK`0;}eiF%jn)+vcP4^>3@-DfGH6{Qzq_$(4R`qrD!Q0V{TZedEvIWNzz1*ql*g!xv z>YTe|_QK$8LWG{;>30WeN9F58$%)f{5=|1dYPNKcRX2u5Z#gs`2`_&iC~isj>c0PY zJYt6vnJ8#Fo{7d)s}9cQyW<|Kc|n}6C*9t9O8W)lqX}~g;&vZY}i2WP3GFnBp!vG58 zD=~p(MeJTF5I}};!vd|;D=LcKcF0AKXf=wh(w;xFI*7WyKFOR;WdRJ}CSkC2Z6eDSdE=b2NUz;Mz5CP?&^7_n9&P~43|bd>N}dX9Pq#K9rhho3_$RNgxvIc;QUkb4Wf8-H7emn zdbPU}=CX@moXzKQyRD7)snw+i&st~wULkKqAWqhYW9BkO-U3CBk8#=4TL)VG0CG`+ z+NT z)4eqAQztD2sc;L;Fwo&km5EHVN;A(`%ZIZ<>H(OJSKV3h+%43jM)j>sR*TqSL9Uza z@lCFwZDfA@hd_PG>(?8ByPvZBoFXM~otWk~<1rX!v9hj?)V)y0-=_EbeQql)bE4Xb zaWde0GiPAuipDpy*|uNDo6~tiE~D4y8ItufE^xOe>e(;D%spEr=dOn%&dnIPw%V)u zen2kg-Vgj!BD}04_fnK%(;k3xjIRxrP+l4mtv+udY<=wEXVP zFF~;5^k6_NDp^GY)!kmiX0b$>1#Gs#OJxE%yCLu)|s#0*a5C}ymy zc?d_ZrQki)tzH>8WC}P^ClrK#3#mdiZSVaVOMZEgosYpiN^RX@twr|eMSlZR(DEY* z3Q&IP3UB7_<)fM;pGAQk83BE77{AhKVM5aWCgn4Jc7L+^Ml z8{rNBbXy)pRdTJ*Jl~OIWVn>te@t!4afcBPt;jN`FF) zJj>D^7AUy`_-J$Gm5GQ-HvhMUaIY(AbF`l%xLPi$)39>Hr{3asHPPy3!Ng0_etFt$ zpKl@5`IPQ+53`(x6pOr4;;fKHC{t*v|Bt%dZ#r{Vjq!96w3j0VIgL!Xui!B&ks8n8 zFOu5QZY42)u{y}J%Dh^8>+Ng$^$vDOegMIy^+S3F_Ew~JFZk?shy{ac!@bk8YfK_6 zGAt?_v|~U!sJ?N5`8q%KW<^fk%uXiQ>YTkNqbw$7qAgyLfa55VW{8DEHcYQeDD2?*z%o<4 z9Y+X1ts?w;yk5YHk|ut?Y@X$d zVR%aP1f!#b9M%`hF%Ixc?2C$XU&7zMWrQCXFFh2cP7g>z0`l;Q` z?7av3C|Z0bKF8a+YE@0aHru4*cJU?}RApR~H22yZ@DAeCl@YKq?Jpb48I3!fs`09c zE?+5HL9M@SaW@t*_lh3&%*G#oMq!_&c}IPdeF{^~iLVmzijSG}A6J{QWV%Av(|_Ks zLfy^h_hc2Z=VI1V1Des=bCIFav@coIY^;%kr#kob`Qqca)8bQs@+(*@@(iUe_r|xK zSjsIyy{fpfDLJd2wueVw`<|8MX^2L0OHd%{7PKM#Jz9vh03c<6BQIisZuI_ZGmRhU z)lz7=eDuE0kjpmSs!`8G-->Q|Vk+iYnV7^C&^@S&YsJvrf4~B3TnuC9V&^&w)hEvC zgx<1)h75AiGDTjl&OWzhkrc1RCD8snBfTXeNzRLrl|f6a8k?aSRd$?@wBF>FA!zk) z3fbI)b4WbGX$I3DXN+)rhRVxv&hKBLgH5vnJ(2wI znazZRmJ3R>p!KVrS-o?dkFsgMu!Cf0d#X@7n)N%1|MZp_KLmY@|5Dd(|8esBUg%FcZ@3v0(=aBWW_ z)XYsM8T1DTHw7e+fH?JV|had--4m)?%5TUq##h1wPZhV%7ONmPXKOZ^b$V7BM z-yW(l$SAlo)*3kn@b2))OVzVNS7T!%I8U!#z&iB!>3e?Hp;HJGSXE53XjPZss9#n- zmM?wCBymo_1|$J5R~H(1JpEH%$s}95%AP1D0oEjrOGu9d5||YfiY=vT8Y{<= zn7~pAN5;P^>Nd=L>}$Z<@#Pxqw9+&*J<6k>OC=}>V3#bdFbWFcqcY>$`Oq}lt;)8M zKgby=Y&S0KGQ2zmg6i_kS{=^_>a8cI124h=kY6^`*OB1O6HKk*6VeAPILruNlo(yhGYPWLMH6N8R`h50mc zc6Y21w}>O-4%_K`AMqKr&;UDvcTp<`t@55VC^q7FL}I^c!9N6XsW=BaKcdJHmjGU9 z!RXPKg`HgB4uug~-*53NwW=aJ5NyK>PiLj$K6pG1cLhmNKvYC{wzU4?*FpXdr%Yefbcuo=8EUGK^yjlHJQ z;c#V&=Xa@OgC@(T*x57ZA&fQ;FW#ZMGioyM{Z1Nqt&{$-?zK+L0484?j2F8`{N%qY z8>;@Ijj8eXqUwl*Hbw+EDd48fu=J~r%{)HSo2PHdgRRBxrhX`hNje=F-g+ru+2-ly zLjjwkVAfWl>Pchcj*g1$B<0{iUm_dJDFoc4!#W2fcKmf^;e{@W`H$IF0SR|p%jHED zMn5=9L+aa+z!E`2<$wfT<;mHt;iqNUeJWo3JU;V{-4=XfZP*nTO6TofxwD<5%QH`+ zqC_PCsIPM$9cYR0Hoa{3%9i_|nE_MW;5`mXn=+aZZ%Wa@#c~v2EG_8PHxk zeBxKX*7L<+6~`rKw-*w4$498WN2B>{X&4ewY6B81T;&L5qkLXLdGsf7(B?}=7u%4B zjVO;gUNpI%Vq8-wspM8}NaZ)$OC7j5V{?LjSAOLB@V!cXylF0JnxP(&CPzL!sJ|6|L72GYdvmoqL8O1}3(4Ht#B6I#8z zeq}j7MXzI=o`2kw?8tG?_0)4VZV#I2a`ELL2KYiE*(Bhe!Gf@h;;Hss8VX#zbhK#zo8sIy^ba7=cpq*%6zob=S8CxqE2c(*Bx9hf zJoLT+c@=6obK`1z7qsNKiTN=;)qLt`xBARb4j^cw*Q|yamN=gVi%JFOhK5TkdvM|I zYBAe11$=q>FK+@R(&c znS;wU^pEu?xxKZbQp;NXr`Sxm#JDbm!mSafuPmc-iu(Mi{~iW#$UMyM1#B|k%M_G*hw;4ss26r1ndmwSzS!Fn%^ee765 zBpa|7jjfR_?379u)%-aos0#Col0X|{p9?`wGkT%ss|ApwFyxl=-N`FhA3DI3g6vcn z*BD+bZ)ZARKXm(@hQAb;&+AD)cbf-UNeekBfD@KEzy*7s+2oBL`pO3zZl7-u;`kA( zG38C*2-UEz)zxV9+&7UJoCP;U(QbXKnp{Qp-gLS=Q2#k~h)cv%t~n}lJ+9>bT9^TZ ztUnH^u-q=V?!{2CopxW%Mz@~F5odIuorXFSm&>k3W-^?KA;&A>tt{5T908PD(~Zq~ zK_-QzkD}nSu0!l>{k4D*#K~t{ zHEok6UsUjWhe)@afjyn$CweLE#X-c{Ase8z&rRB*SD-1csLTjmVlan%SoJw2?|_V@ zR~)x55DP*XXfWoM!lxq3B0s5z#>9-$2uX+w%W|S_r3W(xa-A2J{^ZQ1&Gn3+sG$*4 z&H6MgwTfMUPK{1^<1ZZ7Hb#fX#QU3Aa_ZgEIdEQfy0&I|$2Tzq^h*O4+cqLjxtpOI z#GHQmxHQJu>0H8P7=DTFv%M5&ObZ7hREJCR$YFX}RE03cG&-VZO}t8bQ{xQ$GBz{4 z?F|hxFN+GT1bDOItJ#Aw9X@+NM9Q!t_GLSSn@^U@tb}{>{7f4L%j28D$tPx4t%ovL zycN#QB}*9y&*gaJ%UrhXrb>rwu&zDx|0p~2Xeh&m|7R~^s*&A{QkGJ-EW;Q{NMe$N zEMpDX*T~qjMV2IF7)2zquVr7xzVA!c!5|uA8)N(3p7(j)=XrnU{N8hZe>umo+~>aU z>$B>Eq#O>D0+H<Q2@M6$g^Z21?kBG7W@oy>A6Rk;NyXYHjpR|{`L>-o@ z>nM$>=F;M&b>$J>n$XP}{~m!6eWp!zUyH8FUd>;N8S8LjfEj+s_@G4&m(*`Ef;zgR z{hfu#IHE7Je=Xd5^|#H05sji^gar91gLGCSb}M!NQQ^8e{}%XgOe@HSvU-KV-M5}Y zdzp&fLfvreLWF2xw-ns3AyS+MHJkjalM-1UyWjRR#Lpo~qJTrMkU7-5Zu3}4My~Q+ z1a?j*6bb$e%kQFFm2E!L*`F}sVTozEx4V!uttZF%_iI~UwvGGpGL-cbxMy5)fS(2U7%#IY6YCzg1K zH~KZysazjzqG*=DB8pfXlwRfJ#w8<)1q!3dZdzi){S$Bgl?7m#4bYVLp8mZZnRG@0oqI+pQ-8mmSRe zGt$&sj^y#_lF2aUbY{W}7uAm~S}7yiOLJBlV$m!Da9?Y)4>7wVgRF~vy-n<}WWuy^ z<`FqWJthy07Jq*x)N_Cx7v93XoJ|EY5lX?lB0Th-)!)3%brcw* z?v*kbD3-ShP;nrZXqo+}HK{?qTD*eY$IpWWks|mS=r+4t+-EEq;)@^0@HJi43V74Y zGsA@^^&?!#v8;6xh&0lyj)^{o!`oba@*q7$KEsrUcbmc2H0w6WQEo?f9qc!My`BSj znSW5n!A-Vh@ZH@;A%cwU2_O$J&q3~;(pHzCp4R;+VYzP$Uy%wH&6Poy51vnkJ9cE+ zKl0+FSjnspXhHV9T!LJ1&1(wP94j}=Q-izIoMS|O64)tm3o0sSLT^ppoxOW7Zez#f zUo-;H+%0}j6_wn;Uri2_c&n?c4GIda495!f5KHoQRI(O_4ODK--V^s?H56Aw{g$=- zQiF*^JIHooeKvhPQ5<=GqMu3?c3tpWA#dhvXh8e1vK!)IMji3uUG=rW@A1{gR^3_t z@a;sYuwyayStZaKORWUqX@#6%6IAYOtt@TBXx_>;x34|AZ&CfGyT_V_eIX*49a;NX zySJdE&xzV(gmy%r13uQUw;fqK>qPRfcyVCI$YR4?ooW9QW6i^(WbONe-tW)2BVj(y zyYW*4xwz*^>yCu}do>|`2q^4jLr;L3G%&KSW_;d8JDF?|;MNLlf`Q zy^o9)_W1L-W+DD|DI#=F|Di2)&S$RZdXd}hiA_8}$emGcNTq+GKpGf@9Z44ht zZd8^(H^srurm-d~JlmojOhfKhmN#<8hdo;p)=ME{RX!!=Stuz8Vs|V|7wv)e2gt%_ z<4GmJf#LA=h#Zwzv>t>7-(fSMrej054oh$4fFc$5pg>csFIEk^7xN>xy8QPCxk(!4 zN92J5k8KssC(_>Z*UM&hYbp#@#+snCv=jFPQ`nazxo?n%BLPU&xLo~tf;b@?NKZI$ zQ^{uMe^|UyplHK=dq3|Q~$ikLu@T!>AS{QB4Y za?o!wx4mqPJ=C^)lEtQnZZceJ>73hhPP7#SK`K>&KI9LglhVTjajQ!NfUem?F7!dDVvYqu5@oQqINVc7l6^iLmS=H2H9aBVT@`9=_&U@h zk25cZ{FOolX$8To;y(AB3dV^0ZMk{w1JA9!uX5Q%`VD_{cosV3xkpJVXlu&A=eA)zBflWdpfnnpI|)|y$hT%v2y>PxKU zqk>bjA2SM2F%`^mCo<}Le0uitwShjY|9$I+*WNE*PU88?m`R%VGp5xu;ddYi>7|ex zHCYs9#MaOtn+@rEWN~efbiVIMlSWOXm8juPZmaaR$H*jv{N02^Yk(of{@qW|0Rl?A zoGUe=H=)oYQqZ?Mq-a#x4+Wcqp4Ly8tYIAu_j(Xi6s#pKdwrP2K?f=d+pu}-;_JvQ z(&@x$y!3-#$|==BN>}TsJSS^4HZ+eklsrn0PB2+)H(a`o6NOxSq%Nr&Cas=({2XO< zL(zLeV9CKc;m09CJF#3u&tT`q8d9sc|LSAn)41-l+&;v5vtZBRcReNr9_19* znDA`fP*l-&=ZE{ET~)TOF|jBIfB!PVm)4kf;ZNnr<7^(=Va@R{B!hnEt#~cH@@U!YKy>WRu=)`uii~uC%V*z~ z&8)M6eqVnXsYTe&Wbk)=2TqyLbpq6Czcy7iLj?bbY%4g^ahXb>l+NnkivOZ)vag<# z!WcV|PiotJT{=2y@M8WY=+hygCDclg%gT1x^J`4*sEL#EPl>FulZlUdziRD=g^g=n zawtEPL&Iu6Wow;0JAMu_hL9cj;U8`mlp9_1NgYY~u8eTk2xj;hBSE|yjD)QUx&6h& zXx^N@0TSdHDxnIjn~-pNs|Do1o?H9m<=U35jOjuEpraY8Ix+v^#A%aKbnSXS}MiS7=O3`eN7_R(Sq z1>=cdwnHQX8+% zxkHuPB`XJS+o}GjnV>r=`3@x7)q&d>i8D?+9n}C z3J&mQBYFn5SqrPB*1WdEsIAp2AtqKY(_Z_E)C>$|KugVD*5tVB5NfbR8MmHzACx94 zC%O_l0B_~`W7X*!f(5=Mc?c^ZySC&yDFxeR=S$EU7PQJ@~rAp1v4=A9}0#b@>wPk2`B)e=8rM<&M8>` zw*8%d|72$ivrC7n&1ItMxp3#Q&A|5(+UJY>Ag8ao$#|OTe}!BBnVmHSi`9$j5yXP+ zbylQy-h3Zu16S|4`c@^XdxPn}|G_U*kZ=7fQGCkjz}`^hH;c{E&rdnwV{8wrq*lr1 z=jz0}dg0njw5ce)*_kOaiFS{F|Ed}FHjEZy&!@`$TO_H$_g|sa3+JnIIf#2E;`jP` z*sqf5=xeVyp3xHP5oIQ(5NOv9f6^=Vwu*v?I>@2T7FNIcxYq?8vHp78*mUvZ|7mmb zDzFH^dR!9|tX!JQ4kc%Rs9&8B+7vnFdzEH)tar#6U}{%CL=0AU!EjOPE&yWve$k=~ zL?-`zB(yXsfmFvixpL7nuq8lPU@dk6dpBdWGjlk*I4w{0bJQk%6Z!V|tJ7D#-|bGg zGc}{HqE=>9w?y5pxn2ABS(y$EynORnDJo(PkHIR>Tc+yl0fgSK)Ka&^)PUc0A5;TR z`c$Uly~$Nq7i<+gj;jYIqXZsk#IpS>&i>Dg^dk_>l$l_gz_+Ch7yu!U?PqImWT5~$ zXPxP5pVQBkES#F@G`8y$l3CyQ_45%p&pMIg|N0wUXJ}}r=|`u{_J&e*woVytniGi! zvB!YIrI79qrtvf`nZZltTJhV{@R#(0>9;}G0nYPJJhA+FbZQOS|2{Weu=5u%+(uiY z(?-6V+h_pM%eb5RYdx*-1Q6!&`T-n~6YG7K8z%r0{?3N*^e@KkCH0?+EE2m0PydtL zO1eV70h|mrqkR=^zk#@!7cyf@Oj$5 zIa!`apBX>%@8mcjo_!@=&+AG3(myEfE@sfxhIZjJzc^BWn$en2FjHLop!2-`7^p@v zmz{e94*j+9Gnej9XQJrLgG4zn;KcgB;Y(>SjfsClAg0(&?+6I4a2{FmEA5>|hp0*1 zj+JKH*$~a8xJe)Jl7=bN!|B0vzZ6gQqohGy}Yy^b7Q>QiTG11xhgZkcMPxxQWMg zY=FYQZ^nP*xyNYFlfMqiI)E`Bjn}6phF?@mPXFuw-WB^VkS*}FwB5hIcM8Y-3+4O& z2H7GR1b?k-~0DH2OPpF*a4{0{vZ6KmS3KhVVgB0&G& zMz;T70yLSq2WT9vtyr&~27<|T?>?%FRbe*`b|&xdgV3`y{4=XQ-gduF`CS}C3_2>% z+dCn_u_`*z!}s})F3drXw;hQ%-b5b$H+ugl2%y;FWvxF!X)d-62fV9pFs_#=^Y@uq zktB>o7kmZd-Ayqtn`eJF}ynA4|&oA3ngOi@+7^<+)4ilJ*VjM_Ks?$@mA58=f1= zx|lcri;KXOCu&e^SqaFz{=vT0X9M9_U#7;-DA)GuO#@#w)RbW0S>PAAdldGj`gPfH zvv0s!Hdetl?%dA)b4)0X(=NI50Pncs+;Um(yq<~oqI16rJ2SwwEr|IaA4-P;AzM#C z{{`5x2+ngM>?{C$smASOZ{oMV_>SENUb+sX&LHE-{di31$rt~nEg3`<=$~`r3x(6e zKc7h-e17{hoRWvC^V6?+z2gjkXb$cZa%+KNOhL*t3%%$i$?mX!Sl46x4=wb6d9I&w z=4G=07VsA`^21OD4sd`?VTkR|+NOu6DL2ttviZVXGdIxifEWB>MB{;@lCL}7Dc}a7 zbD#a-dN-nrt z=dFr6b*?}C0LyPheUj;Gz|@vu%9RY9Osi7$FA$TU*N45h4(7lVkpXU}>Y|m&Qx0$E zw5u=_(5u$aEdv7m6G8`5_NM*Sv82yir}A1NJ{}p3A3_>;TOVdaPRK$C#(zzd0u8`a zppKDzr(+#lcbbK4a_d!)K`(I6*fcB(A~qwm#W>Gs{aHX1{su7rutbz!RciFwB;XAD zvcLVwE}wIwdaH-+f#NA00XR{me!gwCm2vHGCiwzXWtsRl={%Y1rpr&Kd z-JrDrIF1MMzfXgXr>=%{I*%}@pO=<_;AToZ_+%J)l}*%S>af+<*<1Fy;F%L32|Y!< zD`htlbqIu?$Y;l3W`Z$VLCza>;5$0c*cV^^4dr&UNeMItRLIwSPQ@5we0QzE#1iaL z-U{*I`By{zdw?dkbWLo^E^Kc;QWVkykd|gAM|;rmK}TS;mi)(P9XuVaTj%Z;k^y~6 z#cXWAJJT3Zp&naPO@LiGzVdC!pxoX-et0P|Yv;|U_zfV!XH8x~`pYpE!pu&9wMO4z z7q92W`;7QO*PuP%acq|%PJA6j1=mYHMVF;~24WRAo57O_NdWk^wch#7D8>NrB2Hc_ zym)G^e?D2Lo*eI`b*O%hSoI`0?R&|t)P8br6=-V~qGT?T`-rcr=|# z3^G8YzH8@dkV7q#saPE#=PgH>RUK2-CeO)dFQ3XCNz1klz;c;S1Ds3~0E#I3fh<4kc1eHm?7${dpc}J& z1)!v4D?FKJuj3e=Qwwt02zToaVZI0i3FiFs^PG7@_>5EWFf$pk0;Mv+l_3{V$nUX) zQSh#)-)5vHwC-6`5{bAbiC$rqKa^k|b){_T#@M2;q^>2S|2Q7lKaPjM+;LuRe+xog zfB3K~>)W|B`NJGUBF!79=pxn;1`*;5X~d*}blBe5g~T+KSkV5eEf`W{|S+hkMl!Puaq z44nKneIGFS*q~EwuITMIyt4)hlP0{p3l9ITyLlkc^y^KzqG8 zdqWsg3e`W zZ)br$HE<+OK_xg^F8UW)mEP{`WmjnMgKWY+|Tyw)3n<<>F)iIJYQ|$3kbGWF=w#;~IV$!H4F2%AR>P8eZ zJiMOYj9uqa3sXa^cBkAh>|C1D6GYck1Eilw!YJ$Xr|dtkpZRM(eBPUTYwcEyMr-@- z(Wh@t2~L;|&CkQ-pJaZsDlB-dWH^-7Wi5~eGlcbgB{=Im=dL)7yGx7@Ii%Y4uYQjR%Zt{lj z_gx`TRDqft8a&)y0uz=SHbnM*b%^fKOlbHhI(VY-OQ)=)yalaj62B75o;PwFVeIjY zw?kjA)tEQIKg)YPJ*?VlFnKlR@Q%r(j_1xS0!|LP5I}RUT>F7)Xe?B9al(ov$irOJ7 za!v5G&J2DK90m=%o{s2f6D|=h7aU-x{4t4nu|s|&Xj(xN;3-hA*PE58#cJ91OK6OZ zFwiD1{MArzKl^zeCS6z6H}lcNONNP$em1!dg-0K=<9A7A{;Rq0!G0JoWPSVR0mF8WTCq2T#+ZvaHS!`CPK6u*HA%}H$8spg)KB9Ce zMXzN#qWyW@Y%kN$*8m&0$&RO{+#iAQb70QJ+1#z#`MQb8q3YKk2GfLhNPVFbD66cL zXdl5GZeB{`kn|(H6HByi;(C%g3O7DLzhF7-&f|{^e&t4qjsnT`XBhA7ls`%1wi{*R zTn4V|3ByyRccf7vPeMrP7hSs@}==0MjZ8Ac$NWkWW11C297U$(m3te~ux`lM;RR~?tqyAbSB=9Dg-Ja6hi+kfAt2$HB&V*P@~sQB6Wp!} z=jyHp&1<>=k`UzS0)HMQJOk8=NBO7YvvE=A*F(dr z^aShaUSRpU0<7(L_KYai*X>KA-|dTa^XyJDkOp(|cMh{#PG37$fTf@jYf9I0uF1~# zmD4_2GN=CWmHqb`w>c%??3IL)a_6@9p7y7%w}mw8qel<|DNoSpJ@l79fPJ=}+;uJ> zZj}Jy=-Z?NKxNX^tZ0>eYYX8btT##0ZBumEvUWXmYCfeEL`Iw}MEr*#ERQ~IvT(V@)S z0NHRH2tu14zL1Li65VsTcp#0X?MrLjL)Vh;Sm3hlRqg9(pBz1Te$VkwZvM}gKY`sP zw&JO928h?J(Dn>a0HJs--{J&F0w?yLuG5mL9ga7yWs?=$kFVu)Jgd9mtFcq0tW@um zR|5OoJ9ezQtM@M0JV^27RkiUC#TVV0svqVLxaNmTwI~)b{KcF$hC@!cPH&HBct;pl zow7f`nZ)>qUa#{6-s}876})s)INFX3Qvis{w`~Fr3O_$tPogWudwa@=dH{d9eY(ko z4@?;WR*jAg4ADkrZtc}ut=3aeBi5Wl7cN+IS{Ku{8_Mm}(E*WusZ^7r;)<*oTk($*G0_{I}mIUb8>cE9B#fI-`h zpJ#T^h)(gpy%-ZGawZp=i=Mf3=g+2<6e5|}uyn8Xdh7mEbW7`L6&vdr??x8Cn%BHc zduOwkffvl9c+1a#zEg?I8gP()Q+onn@Uvl!^kcMl{G@hwvd=3m{V%hVS5qH@ z7U%ZpaWPyu+ZB$ocRN#$yW%#!*^r?~xz-%*FefodId{LJe4XZWcX#F2%+W?8q;iK4 z;-ru$?Q3G>)-9#%s4*etxQh4Q1Qx=tjgJ?-gh}n3Bcdc*g@t&vN#LXd2C22qU+dCZ ztn&gHGOBFl7$fH=aoG+cpAK*R67-!a#rPFI=wJHcd^oj?kogU49%s}qTEaa{!F~wO z?6b>vLcm})cKPj=f9>c=QsT!y-r*6PIA+RM*?^QF5%~4FtjJO9d#-yNVEffAZ z`eUF_1*WwjJVD-jXp*4LjSCjP82MA5$8*fA$%*kw{^?s?k+LSv4RkbuP&slEB@yThBF2F;Prowx zSwt`&vkD_ecY?zRr&=qQ_l-;8npP8;h0QUiKzL*k5tM}xgE7x3wFt9&t2hz^W&ofb zTWZ_HaNA_B9)!64^=|}s$QrZEl$g0n*Qwx1ynt-$CtkoxA$JMNV#MY|VUNTEf1j4c zG%;*Yy?tUY`)s1?@_sjl^zf{p6VB*CgI3@sFbt2?NqyceA@4HxxImh29CFg7?MG1% z6Vc#MC|L_$^zNjK)8a~P8_b%R!p?f!CdYzE=3~KM-V=wmZ@rD&om<$B3lqrpi@MC6 zf36w)n~YOfdwo|V{R;MH=~`t#Py4CFYr~fG#wS7#T&dRuvOgV`P_K~>AYX!Vq<(r> zBobPQ0xDhl7l!F@-;9?6#fx7!zg?@o>a)L?7*nmyE`e`66{5Gyosq>wh!h3jBEGqO z5m{qv5pv0R+yEW<1$6ItlbiLo&$Z?H=Y8(rE|~x-j^1*J|7{NPlB{U&&E=26!mXmK zut~}$x=JVC4=uE0;if!YGyqx6R)gK)UyqB#7vJIQ{dC%+w4gJhS3Q53zm=q;WqzM$ zpO)3QV?dgIp0Ln++Bwq3X=@+e?vHz5fv*j8gn zIC4zFvny`xk~`^>_1B-glChh`<1dUq|5e$zV8_y;pIC`%<8_z=WAiTBEyYNYq9l9t zfsbQ1&5f_@84O0q*UF}qrPYxe_1jzyo1;&>eqit7LKK!8T)18lJ&rU?NEjC})ii%R z%j-Bb_TC3eGzWQVLl=iMNrXtbka=!IoZh5^h|q%vQC0ZatfebGO}A*8X@cuaUy0xQ z*m8%-eB0HcD3&xEs@jmvdnsv97t%NN;F=hd7)Z%Kjp=~sGFTGeT0&ru$^BV>+8`Lo ztqif;=fHx!6py!;0>h|-QW$=ZoxZcnLT>o2ru%sb&yNMvBbCmwK#2)OC4?)zH1F;u zs$4oSB0dq4e`emKUzNkj4|?fn387SX1hJUkYX}r!ZxLXks=cJaM#H8pEq~jH)Ua?< z92enIk~AyG)FP<3-@YkkIyG@&9t`FE0>!w@D)j_)&;-q6so51dxzSNOr(GAsxG)Kj zG%j2b7^kiA?hP7n?^zzeuEqo{RN?XNj_5wqqUfeJr)3tv!>zCs7+EAnZ?0i7&%n(G zfppO@&7UsTT)yAC6708hNx>*8;a*75Kv)l5+LX*Rt5ei4-H86CGhK`VDb)MAD#6@D zxY;Xl7wC}W4Po`G-F)X{)JzRu;G1raG+LT#mfx8;Q(V=|&Zy!ZOyj!woujHjIW0^K zykrZH`v|`co*CpfAH(wY#-U|b9}>3sp`}J$`(A!Y15S-m zM2o_~?;WW~AbRk55yLnx(ctbC-TT~l)Z2P6E0kV+Qq@|h0nxV zFq~W1;ga8`X_$VpyuH3qN0Q|2w>|`;)zhY>RxgonQBZH3WaL4NY>A1hq8GVgi+UzW zkmXYnXo)ptmE9G~w}yK*d3WO`338O@Fxz(1Es2?_S5lhB9?{Ocq{7R(M!q}OnakV! zk%nPI-g~H>qNB*$sy3o&sV_K|B+0@!f0!9B3kGdKrTg++@*>D9H9@=+_BYzBD1=uc zXqj`U2yQ!&H}cP6^zc8nua!rbMf_aIsx0xKHZYTyfnud#n~VLA<^4#}wi2^hSV#2* zz2)Ci%Fx8xW!cQ`AN~Y0ns=zKbAIm^#y*GZjWnGn!1<%paRZ0yiXWWD=nhz>Ew2f; zH(FiQF^zwNi4{l-iM8Ci&&?Gw4ou%WD#dPG_b<|R-``Tb@T9QB+0f|4`z6gcF(LwN z=P2Jz*lZkK2_3e|;ZAV<4^&TRH~YG%L%Ea1^$aojh5F8`fqCM7qlf?tAfp`L*I3 zEMoS4d)K>B!dRfBF8SlF^k)S>WxXw3vdD{Oy^X=W&B0;da__T5F%_taw6s2fhre9t zEk{O5^-BiGZJe54dLlK4qR@NPP0{`VG%)fVUOAoL@(8c$=I2q_}UD*ttR~e9;2YXtR%1b7`-5@(Xa66}=o+u|yj_+KWg+1hZk#IoksLEz=(j^(IH*K-m_R$h7rQ?_j zI-pRVo5fKvyWihI;{pk9ddAhFOTD9q(*r)0pswMoSC>$Ez!Xd6=+)08!#6TG`U zE`I&?q{mVG%-{aa9Pv#n<12n}rnqJ2NU7v^j1uG?*@zrD4EKSpo>7zF$k%lVHa4mA z)3lqR)_UE_s&M$V%jz1WqV$M`=HN_S5b(GVVUv7&T`MJa@dhQaBILwIE!R>EW%FaGg*dlM=+ z6BVg~2R^rgQdZow|Kx-f6$Ow?X|vG-22=eFIp^u%HYEi#glMOMY zu~5HuFmH6AKIx2kH7-~M+eih`vguz>G~0Z4?>x`3YhzDS*dUx@WN-R}L3SQk5d}1h zlGLE#=MFJhtK3_RJ$lxC4WTqpH#d4kM3K!wk+j9)y|hLsgi+qvUZ5=Q@l3;J`V~Rh z7LVKMKKDW(I8g7YbgF*|%0KZCgZoUyYRTl0dl zcnB==5wBwHCFK!$o7F{H5axI@;(T^oRH}}LIquw5cTy{PmI2r~nc5obyM0ITmbOgn zd?->i$!VTGdcGI=iongkxIK*U6iPdK6YG^8^Dbd4@;o>DVfUyX&jBzPZsCw-ofFDN zMGcjQ_wV_8wS?|lyPe*e}s;MY9Rre<@$wg&I>q{mvja*p#+|XGPjl6)T zdouKdJSZSq^zN3`*wkFS=XOfBV6`LTY-<|2)Z65F;mvH#zd95=Q`OvO>fUoWQ=p=I z7ra~LHnQ7^&Zx(xH)x<+jYhF2aGFB? zRB3EudSaPM_3{2EMvLIE$bD;(v7nfD`P@wSBi0Dy*}l4wt;8mwG@ULPQgl2rf{@X4 z^DnzC&>J|rN)d$iUeM~Bb3-@3q2Y9fN49w{Gr*+9J2H9JYvt%mV6!<9>H^?McQ(I) zKvqeSYE~X%Jo=hRT2^4T4Sqq?@$UO!uqe3-CVQphDi`Or54bO@?}c9PyVY)f)Rv@4=C{ZO*W#Ar@$?e0<(nJP2&%WM>vym5| zH65MvkBaU>Cn>7fm`sB_FY*=Th7R+tdrVNz%BqMAS2m^HoiOnt!f@CX!Ar9S+rq`( z;iLdO)e?@07jve*wf$ia-5pW3TM$(^ABp!lg3Xf0K0s)?zc+_|!OSfazIhELv{VmM zoxtR67Mk}63+N9i#BDJpvLfJ_vU5Xf<}JaOjamPY%>v(;;8YL`q7p{vpa94W(a@M?vv&ZCQo)q4$F zMC-gR1;OkwYkSuGzN$X;JWjzi)=s`l(#_Vw-J^Gv)otcm#Sy`*EiV1}Xsi3b_q?$SeW?dp6B7A9jNlCLo3U$E~O z#WQVY5e=Zj&tgrW*3iGrv4eMy5|=7rx?lUQ)y-P1t3!_B6%TlXX88*assRBO#7_)G zFgw3=#_sCz!c7uqVi^g<%lb7B+a^=p4@Rw{RWT zcb0@u59$!MwT%f^61{*H1ZS;4o~L33WbRBv-Y!Y@(=E=!4qjyK?%{A$4W&v!&mCwRl4_>pKBOiU?kG#^J z!!ltcQhqHML@7)ygd(8lIY7xX=({%zl>7m(2nBG+ZW2?QKe5ALuBn_e)pfg4)2Uz!Kr}SbDm3Neg{A z-Eh1*yYF;}ZjY=j&4W3OW8!rl3MJ5wNdYAuw6S&<^BxRkl*gC!;jCq3;jtLKb%}ym z^y3Bg5O!|W^=#4VXVi7j8s!D5&gxj1XeLa-^UrE<3ndxpp-!5JYG;$V#ad#`V{&DE zu3GpEob2j$4loF1b2JBex8%zuNiGVj5?g&3;MKcvp9yB@E`HzJJL5~!t!W`~UZl}M zV@r-)7-myfv?VtrO!kkP=;mXb;_bJ0Hh8^wA`vU{6maX$c}zI_r$%;I>t^({wrL8ydn(omWM-$D08&XU+y{Za@K?Ru? zBuVF?`gXi_6cHBg4xOh@?iaSJi7w0{%ztDsi?XMnm)`Fn4;srzG2L=S^J%)F)XdyG zj0E;QR%!TL$f#G}3oW4l1*=`KtXqOqh)-lUsMb#7=&r+sR~~|vCJ_~K`>Zr6XuvV{ z_7^8%)b;I%yKmEp6bJ9*mkDf+cgxSH^S7{ukq;U zy)c}xJa5oaIrit~vMkI_{l;)yQI2fxCKQ<}NxyRF43 zU&qr7Ptsk8?PgKfbjfUjxkXEozDBrU0TFOzAF^(@Ez|Mtm!@tzD#A=Ab>aHt)c|ka zHz6-KG(?E+KcN5gtiU2~%d0@RFtK+9$H&e-Hq{Q$J}l($dh()iV3}WZ{g>Fc;Fy{R zFA+G2h{dl?DPR~Ioe8&@cF6_&TuY-ewk0A=(ISJ8dB#<_p1NNB9Mq%u0!5sSVjx5) zdGMO%tAiHTe%rf{sKHy?h4qDPogjPy$4(Z!$oz^KMugNHZywx|M$Kx17mn3GlV0?# z*kGR5fc=bl`saT`hEJQ zu=W>jvoy>mjQXx&%=FoQ=#@mIiAa8&WZJ}I#XC5&E3NI2O-o14U)a}$oDE2GrH00c z8u{>L}=L#IN_FPG^*vizL^HY ztQ^M+=MuAf#=f4(KC{&Da6{t7hTsyD+|TfHvCN_^&}@UQn^o<$*Sx#gf0FM9>fAKf ztd%HebD7uFl3<}2t@cRo3A^n0#gC5hEy$S%MWi$#(fvmfK(AjdkrGolO^ z$Vx_4>>T`iLV}pSWIJOWsWl?xRoP3}dNg#q zS86GG6js=Ta;rX;&Oe*f`Sbep*-xN(^j=aGC6;6d%1XV7&Hjb|z{ z`vYOD(kgg;JRh^wn8oB1C46Mrm6x2?0@q7-2}}g}uTs=*QKh;(p(zq5JT5|*w~k(l zv!dzE3bQ^3!^sl4xounhn$|1DL{lnNZW9Z0WVj;F%VcHe^=X;_&k(<DW&6pAtj3 zp(wBLV%e=C?mLWHHf^pnMU{LpjxyZn=65j@LnyM)qqrtEIbsu*GJs;kP&NgB_9+Lp zt5cK6ICIUz021+xMFWndwpjLYb&*X+22zB`))SS|eGZO+m&|g4(O=K(3?;YFry7XC zUhugyx-)V)Jc+ubEKNmpmg}gN+aq^;iTT^-T`#5zk=u@diivs&1$wVe=>pFU~AkvBlhZY-`v3(=FbHXK7x6n4Gi;r zli!P%+dcY%Dc?yz?04bX83bKudzaoFgojC1wbMlM2!M)0c%a;pP@}G!Dm0+GJ6pE& zyS(yV8=OlhRNvRX_m1>H3{=y0BtFs9py(>W=q_eNNt(J^=zOR-BWf-f+b#IIjnfZC z!*xvE?(KHY8>dnG8S6379lv1e7^}nvaZQ4|rn>y*)2e=w;nX32n{QJotc9ky-Dxvd z%Q903A3sBG;L?hoXR7+b{PvygVN2(3W{_j?QoS5)jI?!vfzNlym0>@mF9WQqYVT6>WR z*RDEVbjfGwJa2QjRA|-Qu`+|4o`Qd_ru38M-^d#~GzjSg^6YCzo;O5S5U^oof7WZA z&il|3z7*|OENuPzjh0B2*R=UBN>0?V6#A-l`I>%ZY?2sj?o@i(>II%l((!CO5uIOp zf@XvHXwI>4skpH*<_PWxKoqVis`pYaZ|FlmTg$III;PxQLZDldBJ=QB zqAom^E>gqq=nK-&WQl5@C*uyAp=&VnS8?RbSa^{T$y5bY)UiqIxX6@S!2H$TVF64> z2STrB_a@Zs#?57*AdgCGW_6$rxb_wH@@=UV-2z`-%l{>~;GLc3yl@c?dCwACO|)WMzHE8Fpj&r)t_)I50uMZXhQ#QOgsn)IEz>2}0Hb z+SQcUp!WyKU$p$vx0@XtCz;t8gb6=>Zq5{m72s8R)|MuGe=d5}UeZJ7TYNu)p^{#W z7__p)cmq@-DfHeaQ+E0Lt16*W9oD(!LktDI)Qj7wv}~G} z8o%?d=Qqp_0NgZU;qluqylHqH>-nJ>XL->ng)5R(Uh9E_k1vUW1ty8_pI%FE|B5Cl zZ7t~ttjsodW+SvN%fc>-VkO*OXF4zrK`2ukd`N7hPAD41&V~5uJ1GTO+f+92+Enr1 zZa|<(L7faUH{?46L9dPA%_SVJPa2whv{Y+#F94$-$-UZSJ>M=MvMlA%!KCFo{|24s z*pRPad)GBA2s7wUFdFFB_>8kJMqnu&lX##TFy!O4`_Ad4XdhA zTB=*zyttCCLR!eYhvLXk&p=u-h-} z&~tx~?NHk@PQ&0^<2g#Q(^*+DEbnjPsNb{|;Z`&0To`oQP&QF(v_A5zPI+0kUlD&V zEhb;m?B(QrzPPl961v_VRGQD?xn&c#Db$aWo1I?zBtwpJ-4E*R^?nW9@Y6Y<1R4z| zm+lZ${E{E7mfte^`iu1RXdom-Q|9O4Vn(}OuoIH$MtH*C3;j`m)q^|5QL}w-p4!B+o-#NhDqd z^Ts{>@QbmLN!sp<2Ma6UpTk(Pu7iDVd)Hi_&VG$QZ_^FKl!rRmE=tqltJudg(!;os zCFBY3WESOvT$3i%^-TTcfJr@bjQjgHcH>P20vC@y8ASv? zTbVC3*jZtCjF{y?xFE|?%K`I$d0WPh4u+a}yfIyzw)A92rq+bXfL;U4V82M+5Rz`u z&44*{dyE>og2X+P@4EqkISY&4UVI-7In>c38GQJzoO0K;beS>_{?eK|Us9Ks>M}HA zRc+TVqo-Aa>yx#d`nY1kgq4emzc8LTDJVcG{&);SvQ|Aw#}ipc-2d~~_7;%&uh-WI54{@F_I zuHMdYMojc&PTOI{9rcR3i)PWel!~0=6#h>4EX00fJ%5uEqa+@;_Ec#Sn zPGTQFJ|r!-e3Bq;Ek%akgQbnQU zqBKSx$vk(}vbD2k+57F1e>hO%cYCgpM`>5qi@v ztvgu;%u|OlW#j$kpP2-br*2DOgG$qw%aZ=t8iedfHd4XXYGQmBIwuUT$&gBok0(au z?mP_Jx#LMgfKrPyU2-DFzMWJYN&S=^;{g@**s~xtR@UN-0H0)dEl-tdIXcHG;{-&^ z;kU=n-C<}A5YQh1SvnCg4R?r{J00+0=ku2G%^~VSwYrBC57!?f=w9k#au7r6<8uE? z*ZF;xt(^U<@7~dPAh)R4E_E$kg$3qgVS10?ye4g)g4PZ}%HniA$D!a5hJv zWN`&CSP==N@#nOn@dT6O9pu3cwr{&PG zFUIgq#yOsk6@cK-{5g{v@`sr1{`XK?2XX7YfSLWeQ=@juTn7T7BCSNVA-7>4pXZ3(Tw+DJ34Nv98*j)#-4-wFTy( zhEL4x$kki!*x&&>G4}MunXjgT{Q5071{@Aj`qHOvH!2W}M5UXhm zDSFcxj~JVvw&N*Nl+w>r-l;>6L6p)O@v^Hx&d+rwj576peP!Lu-!s*?@2To`7Qg%^ z!^OlMw(SUJv<->qXSmwE^tysbX|rG!t$(FQ3Ve|G0e|1M@kASCnvTnx_vbl#nz+ypaaE-Bi8}5-u(O?6Z77-Xp~CGVWUigqusCC?QUH1d2e-$C~pTZ8PzV zc3=)X7p5@MJs;iotq2UZc%$Ux=0%#e5bjp4D3LYY2G$(6CqA)_%IFbzyOEduqmR=` zeUs{WkNl62UOUPksIgqhS%nRXy24y>#A~kh!JTksqM{JL>9D$w-T8%FX8%^#_&)93 zj7(|fam^kmwE*u8g6M6E(*oalW&Cs5MpJc!=hALkX}+%rS=dMX=MC0JY}+Zon>K~M zb)#zC|G5f7y)t-Kc{SClFaGe<=!0T>ET`*tk+ai1Y`Q91& z?-lIC=d%1b&DKoy_Rwv!vwY%*S}ii1{1{c9rhW`^<7Y;Y zU?fvU0yRSpNpw5T7u4ohNozq926z?d(~MWmdKh%6e1RV71G6+7;x~61YE|AZt>7&y z&5Pbnc&$omO<`94i1zxeJ!>smLtXQ2@w#nq^Mvjg7hi#aoyZ|%7)CY2)o;QWJW@Ml zvR!Jk-OK6S8V8OZY#6>@pLwu~kkjXDFUfanJHe=`ump$$I?uCBWsbe#_a~f3sU?^diik!R%#9RuzlGeI@j~&aK4{~j zu2b3{yFr?jh8r9w9?{>yH%TxBl+`!LZA$veM@M<}a`v*DH6n&;Mz)Wu(N zc&+Q|G^Uy@?%us}**g?walodUW^#OlmS}W*hBwgTm+16A*U&>&-|}Izmo&gC8(7PK zkm)TdgMA}kyG5s2NCW|kMUmbNBbxK7$#-k3+qP}ponK0N`x6;nlTfj?47n9qOvThu z&R}K|gK_itrgL3CJ07(H?BxO&1rPH$#l5>Zu$Meq*$VKny8n5!?W~)HnL2nF(r@#e zpKi53ETw>Wxd(%Pa>&oMW#Qfs21~5y@>}8R2ZGi>RvLA=z_;2t01(C3 zumSQu`{WQCZB9U{O*qsV{soyv|0p12dyjs&{9Q&G#Bqu#TNXUkJX68 zDUbs1rnmPsdBgvnw6@y4Y_kOCgpBux=ta(6a|`H00%?L!-yhzC$l3hQ5WsP@h=x}-4g;B3 zDlE&s=;wPiN$Y^Gr@9u<>;f;y5baPPQ*d@XaNZfAQ4LtWiv_TiFFw)a&{l^s5F8?gthC$00-hQeiVz|Tg#%t-D(KE zCAl_aAIDqsV0YdvScL8e_z>d|yUf8pmQ!kG^z%^SjVU`Si-5Dwck;OkpBTV{mB1mk zX_orUtd!Hnq=d`qs~BDTliNR2+{D$Bp+8S-uJi6+dox&de;f}~I~+R1?8Oi-Z8?Um zR8h6uDu8SNArC!%;c`g&-K9Xlz_np$fagrQ;IGtG%bI2UT~Kct+rQT?Lv-i>y9PUk zJst0(Nmc7QZZOEQ+q-UhB#D)1Dj-b}gbDr-^pw~d=QNsF1#Zhva7C~`RjIzq_`JcW z)hOe24;7iZ!=guJhQaPLfn3t{!ZReJZ;LYJQ-v5>3F_{UZ_lgr1d9+)3w;J3$ z!{YssVgPTjk441Y2A;b_ijF%=T)xRLobV|#A5_vO%tUz~{LfN-mCq&Naax_#$-SVe z#V^+zJloEL*&5!f;@$!JdgU^rve|0MxLZ{-xhc6UpP0#`Tn=h?sr_Q=II1985V_n_ zH!C1G2bcuR$#jor9?O=w$%$KM*&?Y9gMrR34G9D~<7#HKpWP(Od3`xdV#`+W$L;;< zouUF*@j@=MV4;fPr;xhXF52=Bw+AjK7Sk-PWk~cNkHsp&XRv_Y{|w0)kGf$!yyvin ziq2w@?%T!P79+4y(B#UoC^%%9-kwCcCCXN&R<+Dc`AS!}P&Vv+nH55XJp)?Hu^!a) z<$k?oSjhT3j=d!v$b&qMU>S@_>hJBkZ&p2Y8O)UWimr!e|GrZp0F5&LSS$Jr_Zd7C zMT1Qq-`?DxW7J*iFe>eal@m$qQ=fQ zfy%}ohfX3zj>A!z>keTd5bk?XTHG#>pK7H*SZ+SysO%aNmW{)A|wi zB`g;+Kf!j^UE=^nYnN{MmWc`Svgnn%)?@zA%BEj=z(DguJ?rz`qL&_?cfwl65SP*x z4x+|0mY9d|gJy*n4Hv79)N_Og__!>gHNxpEhC1Dt4XjRB&go3Fzs3G?P+O8+V^y~M zQx<;R7s%C(OH#ZKR?JWx*q&Bj*MF3c3F9~wrTF3L} z1wFb0BKi< zY%aJ243*Sk5v$?Q4R!eTl?Mm=`ESO8#_X_PuRgNv=I39SnY@U7G$p6E{?#yMZ9{hM z`SFxv;LAF%w;o4)SZ#&CMKV}=`)XVhRl|J)tH-R zBp%}i&zVg<#$A~bc2-3-@nQEY$|(raQ`@}ubqXu>EO3lZs!?a3u+ zgHAHf)OaCa(v~_8pKIwyz+vFIxvHXc&rR{kt#0Npcc=>N>OUJiBhH z)|y&yBjm=Fs4SVBTNE2t$dGK6|5*-y;w9rum&1m21}O-o>%hFZCRjf%)6~-}r`Y{s z;twrEJRP;D8U8sb%j^$vZ5ckNr3!9F$##((A)zjIa{ryFFQ?P@tpK}`q(^{^PWS@D zqc;w%ygxSjRHB}Q_Hu+CCfd@lL|!{6=*A25r@fh~!9YSl&A9aH^y0vgZ4-pQYxvtF z-kW5Y%KVD;j5L9jl@wdfD3$R(#)GbSb*t)^H(msrbOmSY81o3+0h-Z)BgmMr!uqpu zhtfse+I#|0OT!*CRquZfIhm{x6DZ^aIj0JFCR?$U!yhG0oV&7@>p2V_HKiH;)B8BL z1=PA`CEx8iSh{gKw=kz{3dc6HWJR4G_ib7UTTAY#lRLGzE~i*k&lhJP`r>V}rQ?vy zViXsC-b{x<op)&#&*|CMK z-!t`3xN`gajDlJ>)SJ{n(hE1b9|@7iTlSI-tVU>NxvPyT-}R!q)Kxsi7Giz>w;ita z!LnIZ+Tvu2Z0(fJAE=|F)eSTZE;U|Y7e#Te0?FQfg3jFunL4@Ml*A+ML6U-Kc(gyD zLjc}Mc(yOiPN*`qO`m!ug&emp9cw>zS%k+ZD%fxR@q3^1JKc5nL-xIS`_l3bu)DRq z>S^kH_Bh_zOb7O05?2%Z1z&ux=NSY|d!e{a;zxpAGu*eQjkMLd&94mcVp4n1qS&6`I2#+TL4mNT9)6Fioynhn zSTh!l%)9;Pjnr2oYp_D9jx)lp=#lj2JlLWO;tIr?+A=w}wN2fg-wr>8=3chW!gB{l z?b_`+zPuW|_+7RR({qjSt_mjj$6%Y#>A8 zeS<$41iIWAZ-x1D1GpzoJmYSPFD>$Lco6f8-?o^~dU?ju9Ug>VI7G6e@|R$VP_@fL znW0Yxx-k+}KFBXz(lLrz5#{>A7-GkBe?FY(B~sAvQ4PRY;lv9%nawN756y%1@M7|h z^2Y3BL?D>D-^v}c!Ve;@8cMAUyA4JEjvKH}xzBGFTD=o^cD|r?$!f+J-zcI)R(dhM zo_H7@zlb|5uS046rpqaI2|eFn36e?ZPCqA_6$b@~P03ul?b1EJ7y+UbnpFoDh{kZWr;;?l=8yGGcS!>U$>l>4tAXtu7FuGVNTOm)R+t} zJr@o7a8svfoO&kZ*p&CyYIuur*1T%AePA^P+(E~yZ$l&!2BewHQ>G{EOF+)p>T)#Xmm>l(3_+L$i6#g6}`+orOcJ`!300uEo zbd6RbS$l1b_h2rO%)jlsW;5JHKu76Y(uA}fEq@$674O-8uQaE+G|FQBos|r+Z4ew3 znbozva@zh@b4V)sq1#bY8`%`~2h?LOJZU2H(}8UcR%$0LNjHq@(K3G@=2(ASp0SP> zm7#?#MQ+y(6diO8JRGea^yG+|y-{V}KK8+?!Yus*4AsV!SPa6hUklA^!i%o}#6O>K z#$EwHf-qjQB^&K+E>0hKit;`)_JN=%{KeU7L~mtERn&@cQ=$lj_3W)35qb!mYleQ` zNmi@c~AL9x{F504t`S(l1pDB#! zxwkp-yip}FxG7dV7CZepb4nUQpBY>OvVzzP-cs;n2c#c0=w<|Cc|h{{j;O*6LGHW7 zQk=(B66Wh|F<@$%_#F)_WD6$25Yi_VgSNAUgaqw`t_hx8rhGd25-xClqvXAqU*Z~7 zxOh5VA~lGnZ=qgN0!3)Gnb%C$hlyTrm5E=tm4~=PGQ*UpSw)6H;Z62d?-tZ7oS30t z68Sm1p9gI(2I&w=(lJz9R5NOlnpqqglwN-zCuj34#&Sj4l@?z`%ZXE9GTvE9BpjB) z0#c8tTKVMQX_HDP&as)b0mXlzoN3Wp$!zI`_4oE(lGf?DtGi!|M{EO@Zuf*3BqS{a zYI&BG|0pv^&+oyArA3nZ1Fzc&vOG@^B1g|*%)~uPv8$BB1#er~{%~a6*Vrh^mryhG zZna!Fq(Pbb3f_{npOEWHm3JcTx6mMY{+e?(0UG+f8wBT{7$h4f&Sbn-K^?+rb5R6n zVJL@iS(6mG$#82%TVJ8@^uHUM&d|$VwKeZPt7g3`l+hh}++lYmDJqdYB4{<9znWJ3 z;)Y?>mQ#d}kT&D~-ZRy`uQ>0-@>((aAa~ky?M+*)wgIB={JDiP3*F{DafByB`i`_j zsZKOaR~YFO^Qkj#i}rGiSTx;sOigILp*=4)CZ3p`>T3VAsv_e(|2xR`)!QT8wDs&v zOVkV|0GO=02#Q7grrOzg!Ld(G;S8yQ5oql7=B&woSEj=H61LzTu;oby_oDT}mRlT& zCdZ?a=Nxs-T?fIQq#G(`V%_JeSLJE~mRs$PRjDF};7_r|`H#j^C5$mRk`hPTNW_@O zU|CBZdYUn%nr&rEvJ~GYg}g2jDrb-}u6{5DhZSxA0Un6DQmel9${8CXH}xLj{>}`7 zY3|W`VpaeB@B3wCPQM7T#|8~7G)Bz5cJHJ*fPzRDuo9EEY_tx6cPhj+g9xF^zN7wB zr!K8E(O|3K@lZ}aB7klALwj{6nh_&eT|97rBFLdWt&icDA^{>+IN)^eQ5WaB?4ip_ zU~SGj)bbWO#-}#*LKoMiirxH9DK$fd6;ai6{e z&07yePJn`3?#YxjW(p&^DSlnx%{fI2hmZz7^$YfuE~jK(zq~lA6Vy*#>eO)S;8I=Q z2bfdSX8?2%zTS50Ih&%Ac+O%g{c#k>22BqOI;fmK+o7+n<~yrpBUr$d+W&xUev~W z)(ZIWNJo8uLDU@70qLw0T?AqyX*L6_WZoZ1UyeA zJi8~ffZRTueVmrh^u)cSQ%BL>I0YB-_;n+(%hQW2G=>L#PxbwGK%pW6JHuAA&KLXm z034O^Twz|RlXZ$;4@aYPIhDG*bw$#~3c0t=7y*H#Cxqtke&;A`dXHC6B3V`cE!HhpcU2Lp4n24VzVZg?A^M~}Y|5nIKfwQ1yhZ%gM zNrE(mB^6G7P~BN(7Y)8Q8GC;BfMLa@%z*H=Zs>734cCR8mBB%2;n3u(@wmHgkl#Tw z)@wq!XofAWyYvFk$%}D3pelD#@b0!S`-Rqf~&W`yLk!pRrVU} zEPr45Li)YcY^b#?jaHc$nFIi09)qxuXymX1MjkQ{OpivGvW~vEc2G@_;3_Z(t#XYd zP{;saTFqt66(uiOGPF&IHUMYd?gIvcG99xj_;t&58*p@1*#>SVrBih>CNwK<4giX& z;RSz!#>GIe=sI%Y^BB4h(2P~el5sx*)sY?Q zSz^I%9uFU7eDbQ@%XttDS*`P;1q1MX?DNKG`10G}SJw%5#|r`!OJCh1Syl{wT5R;H^T zG*|+3JTIaE#bb#^#f5?;?hZ%rQgWllT$TL}fXpMSIJ|(!PTX4WgxW&YQqzq3!pAXS zg6~UqOjnCSb2}qp_q-|~9=ibc<6Q*~1EprbYRBB`c_(fB*UN%b(&K|64bNlRTlDjPUo%mRPlZEY-0E-Jh z6HqEvTkm1#S6&SLx@U(Wu5k(Jb)yTn$7LVOG}nWMwI}6DzZ=v4_0{q})v8@hV-uX) z(K~4HixrN!zGungykG+$lL0ac4FbYW5KC*1#sfk)N>x_pEJIoC8Cg-*EbU#VuWb`~qS)zIpgSZI*jI}hZE8uh4 zT8z*Z4JU{^4Vyvbkz^ab1CvRp;Vs()hG5!6PS@e_-88{C+-3!#>rVFX?*<=kb{uA9li} zHOYah^!Z~Sr(LUKJ;!hf=7BV9)E^P-TB@TM$Ku*n@b(mL%(D4kZo^-%Y*X4pSu&;3 zZ&TwvYmjX^GPQMk)v5_EsQfS1lMT_Bv{T6lhY5^*NNV=d!*-TdlR>EIL`Q2yxF(9U z^=Ok7^xN5*X2n-zx@8HOUUm@OPU$^chXpD&T8&1r+KaVp%^Gt<6W$^`Ug-KXMEL}4 zpU24eC(BKCFIBXBt22J5Dj@mhF3h_FVNr!Bt0?PHuC+m1C7mGp5}-LmmdW)y`&7V( zsM=;gzeYbeb@@2AD_T*C;_sV=cr)=tOcO_4l&lvv-eAaROB(m^reVDk_@>BD#4A~v zS4O&VeCSUcXo}4A1pA;t;MmlZE4l)faeQvsv$w1b){fF`J=QBLTNy9)q-aHA6GX$CB|k;M{(JwxQQ1okkl9wf_Hwqhl-*sZk8xh^ zX`@I830lQ$bh4Bol7ta=Y4-Mo@bVA1O%$8&KM?3&=WUlV(eWo*(azB}(YALP9|tU+ z*1he@;}h4ST|Mr>d}U_CxS6D!eYRi(CGdazGEA*jV@};PG{r!n(PO^d`EVN2sn)@d z=hsyJR@*+;`mC?#DoxRe9p|dOzt0Z9wy%mL8e*?S*}o}HR_dl)tC}w<{U{G|hgdfD zKy8e>cr9pY&NyoVY%Nu^Y|8ldh%r>GeX&KhG6r!Q;9yXv`do=i6}FwdR+me@X=9`C*<*(q#euBvDol?=D}HXkz1B3X>Xf@=A{V|EtDp1?&3= z^&-+?y>>!XtvSJ>=f(V1AbEun<3zalRE?7mPc&7ax5%cfyWkC=qw@u*dR~?D+^gnc zj&@2zkc2vRDJU6MKuVI^>gUA=JhZaXbG>;RAd8p_7j^f(Mt>Lto7p51fLY8~Wc(-j zl-RR;S}|(75TEMj5TAZ&L*U%$SfcA)+7)%5@t2RZDK0!P@l;}zWdzEWaVDLBTH&FR zKR)uip)f)^@-VB7#7Ro;#vz+CMoT35#7XT)V+`Vl9-R^S86;Nadbd$B zOH-5ffsf+v}CERN|_!T z<@x*{T$z?|hbP^xmbX?^XBP8cncN}35QIEp!h7&A6wx3_Jqsy2SQ7=Na;l=ROS&kX zjfYm%dJaWc*O~T{KE2DU1PdzVVu|*R$qe_~YAvkmOTunueqxOh7lmy3jPMM&F z3((o7%=r0b15Hmj`APd?-n17wq;E}4qMtJXPZ*H9DYaQLNRw0HgelJ)#m51~9beB0 z&9d>9IgLGp_LKX+mp5SKH^5ecRd->7fk(WGyeg)Ec?aOc#j7|S#UuW&1wvAoXS7_U z#=e&|%-J7Vx*qVU?dO-b+EZIa+};nZ?k+e4?k<1&4Szoq6^WCLt7YLX~75 zp~T>ti1^INvkF)(wplsjn%FrtQb5|d+HogLx$sMBwnL~5fJubI3lfrxFH5H=^-|q? zk37^D_4JSAq}h{;@~z1aMZ}wpttwPE_2@JLNGvI!L0Vy?7t?xH%l`%^*e?`%r_Cfj z_8>B|s(V2CrlGjQ6WUX_f8y<@m~?l5cbK2>Nz2iZPdZr7^_ji`h^~V&wI(E*Si9DT zrMJlyFrqPIXohY(c)r0wW1}_;C|EkM>z+VuHoeRR;-x_`9-^lrkR8Q#+7o!a%3^X2tJYe2o;8-weE;&c?=hLk`5uZ zlj5(50;-CfGS?~yPZ)ZG1!l+n6Api{(qNqZsQ$`g#LI{3Vy?#Y@;Mg^W`@9N&@{%` zP6C*OH>*C1MXj{UrS|0qFJ{M*(zAfOm*s(rkn<$GiXyYpP%C?Kj0azv!KX<_&z+7_ z((`sUB0e!U>aD0w9uq4A7$=5GKmQmOy6^tO2@uIpdl~te_lGg``-fJW99Vf<*7K;r zpQ6@{l`#Y%@qp5t4y89+v2{C2Nq~2c<#wb1=vrDS2)k^4Xw!=^Bh0w$n_-z=a?uE{ z{tD|OHY-|9vX1NczFs12D(?C*M`PRSe+7}QATT{sz6Vz^byFt6{n)aS8dC%X<91wp zlk`eB3EhJ)XRBu1+_=grXjMsq5(f7!FuMOJh$J8tr~IuqAK|6`P2VaP5yZ?UI$WYN z@*l@^DRNIh#YhlqN^`GiHfd25U13DJFW->feABv&0ZJf=?g zswJPeJBb6TEs)jLmZI764yHZNvtk6F*AvD)V)hnm85t2th1xhPv0CQlvXjYecN)y% zP4}%K40KqndhwZXu9!O`M5TcCTE$_)ShH%G($m#%1z74p0!c=ejGO+hkr9I?Zy1Xq zeH?zw8OOv#75^94acGh-Ev||bt^oP5oZ}c68yi%5HyH9+J;X|5cQLbxCv|5;nIjTU z4}GfOpWSDP9!@|r-@XccUNUQ>Hj#|hM{?5RwZ^Nv z;v`xtJ92$qVF=9ZK_c@T4~*ERV-i51uw%fFp`w>&WI=p8F75GUZ7|pFE`tQ3CZPxS zi+kezY68Z$09)bO4*J85CRB43!KlSByin`$(iB#qR@(%8+Eh+Hc%~G~n03MhQ6gS1 zyR`Etp!Q=`AN0cDqFckSapXCDjFH3ve4&LgZwQx>Fwn6n z;{&d-3US1DCPAxtvHh5!5pNAZ8(gfPg=Nh6cLk0j0rUaFDe7MPKmrhs?c3R0(AEM- zND{jN++w1rlPJFlT{K|NBZ5MPgc*tufm5v)34kVYSs?CReJcRBSS$f(iBJ{?3&$dd z>1$t_dl#Xs&cXmP7hdS|-CmprkX`}x?rV47cdfq@pIRGs#7>+>_YQ(x{mK_#`aKIX zk|+$#XITa4&|~^|L^apBE^+ksG>2o?!yxx!s+U;^V1pfmv*Lv=r~Nn3J$lNJ4ogJ6 z*{?Ay4CC=O%v+?3DhK49LhT{-cPs7T4!`U6v(Zg$r90Ge2Dk+%#m2sG2*z5 zd}i{a>Fv^jgxZxVLim1sa`l zvyKIsQ#_Zx?I$g2C$8Xuudkh06YY8;zrAIE7hW<<@s6ZwbSYECTDCia$ee4MOKn>h z2D`uVV!f=$8zUQQ4HwbQn!)*}ldOIwZRG`Ll-meT?*_|MJPSprdvhY|*o95`H5Y3E z@)Q3xe(?G@i{r?@4gfBu08TK5oO)>lVc|Np_04A`lmG?b)n$O;={8;)7SGutr#hsA zR|fc{>EWF$M2W;w@?{;a^&wV^Ub@ukO%`o1EEeLo_*Hqau%1tZS!dtu(ao_&_+0I} z&ILlB<9QAiIsVE>8y2U|GGfRKBMK9{4JAG`LXoO`WFX+t5B76!4enHe;m!F^PU5`r4YY80DqBf}>ZjV+oxQUZ04YM0MCR-BYawHfiWaR) zVJlJv>}OQzBBEgT9uDbrVp(TRaL3n+XLpkvXj6Q2gl?UpG-qZz20J@2jXh-;M#O<0Efk6@50xqb-_Uu+*0JaAB2<7_`w{HqR z6YfYZ-1V#e9NIbCx3R@KGO1NwPOtg6F*@>n?fX*yi4HAnVfQaNyB7%cvpVS|Y+QFZ zMWuzS7q$TZx1R8_-> z@xL#D@an@HZPI*VN2RYN`84cpJUzCt7T2gvt!`=Ls|TtNTBlGaX8;@#cd>ClGUc|FY`$7Wo~uvAAWNMEV&9Oi63 z%a??jS=Y0|K9sPcT}l4{^1q>?#K`?M)kRQFFvjM*hm~{7{Q!)Bv^SgfrTS5F%(Z$$ zXF#s_1Y@m%r(SY3fB-pi9fn=~_C*rmoqMq4sxU6DH?f_Fe#Z8=7$l~h;A=X(uoY*E4eKGWVKDUH56T8h36XQ;0TbipIq_776Z71M#Fid2nF;vUxDidsV&1Gs{J5rUFA6SmV164xS7ACSk}eG*$ej0HM3-drNB zhI=WIFcC!i+%&%_vEqu5M58p;*29hQjtBK#e>>4vYr7w|F?;K$U*7y&J9g=&>oJo> zhBTCBBJLNT>q1^Z++mEQXGNL>mgJenn!M>&%r408A%IfjufxTeO))@qeLHCTEX2J# z=^-Z7tyv1oIeaUYqbtjUd7L7oRs#6Fk@zCnGyMa!NogZqdgUyCMSO04gNAVLtFcwl zYu2PJ8wXG8-Y2n=-e;c^P!p*!3`V!g{Stw=DtilIv-E~xxM50PmcqSdlT@bi(E>|; z<(11LB+EF>@kWZK`EHEEU$sR_U$cr-dHVlds-)6%3I4oUYJkYE3-jZEsoS}*Qwm|! zC&x`QJ$vVQDDADGua(c?FVS^Y_Vj`L!=)6IhAk{y zC22uTN8ePew9<1Vy3e`M%q?y?H1c@8+>5Dj$^_%9tkM~2+be+GYSKNPS;Q(I zm_^8cw5hVCoz9rg7O|v$@%yrLa|s$q`*=u1&8@f2%iN#7sGEmm()N%n_>$JARQTEs z0PbT58AL4G8zNU-Yf9M%&h%vfqFF`c{+Csx9&k6^!OBLDQUsjc zolS_`6Yc&0Pz2Wunl50j@PPe3uo9%GX2&y?b4G;QFq1EGgIDt}pMx`fc$X57?bEg8 zy~!a_r5n+eAyq&aU<;Ni+Z%xp&kS0HNqaKSz~7}Ff;;KC)6Aiao}Z6jBOOAfuZpz2 zGlJ7sdm=s(?^#tdu4aMw0RL59Q|$qHz6Vyj3E{%1-o$+Sc&OS2#YAF2vLE5Fd^Z{5 zx(6mjnVQ*>lSoVq98{%<9`dCEDgINbSK7-_6$}uS1PxfW(Vp%Tf<798N;GnL&9&)G zz5+Z4NTLw66<~2l-m({uaqZ+%wbvRH!gyJFbpM>54z~B1v@~$mxe?(M~=j}x*d|dK?gA5M75-F^nJ0-wbl7<|a5hJRsxpu@cs6plXT~a6_Qt`t6HbQqw{Yg^Scya>cz(lK0+XsZ!u%)4>d$sb1ul;+k zX^{pj5(V&@C=Xg8KR`eL^lM$6^xp(E8B{a!U@~Yvsknf6s7QBjEb->XoV2!j{C#bY zl7T$_Wt$gMN58~x4S#d7e1zXIP4l(X3q|y;Fu6F|SrK9;(pW^yA^E zP@@w?%P@!)z!$-KT*q(G1f#HFty!Z}eHww>j7T{#0r-LM?c4?g`9!89IGJ>Zr4V-#r9u}fV#_qrP#&8V;J zWBIMyTKg6w)9y9Dwf~o}@)vUnphbKK44#TLZuqL|`8vv!9P0W3@7*rDMZBE$EKdK; z=XkZI1t^Fy)(*$!5U65Y2OW9`|Bq&EuxX^OEN>sw{NHF8pSYCj6t8Djz3O3|O3_hpuv z_SV3KUBd@;d}BpqqO{%pQx$#-0IXVf)<(Ka+<)rjOdY30Dv)dy zN?%**W*tjrfT&jTubSPR86X0hvIB$-`O;OYCU{S`EV_H&9S!&YPbllp7#pl(FOSc@ z+tG7!1!Z=E6b&F0BSVy@v4q&;_5pD!C_-@BG1a4K`7yx2H0l^fSFD}vW>MMbsPj_V|Xw+D`#1oQ@zoBHt$BCrl_e^ zTfwI+sAjaP4>P4pc^-*c-VRV;~QcNYzcRl3jNOwzGP$3aGDo# z0*lha*Uc?$O4oaxgeKC_M-`-+5pxuxD&~I!ezx6qKk8h?{ih$Nyb?tuxBp)ww zKB}TO1)qBd2QsZ7RBZ6hpEMm%8UtqxJg(7_c7@7=@Ov!vZ&Sb~Vz$Xy(Gsw^SU*Bz z*(1_%SNCI2rR$}=9kr84b$zfL`1YIHb(?kHg5QafK>P=*<__e!J^w<2xu0ApU5YSf zow#HgBv8vebcId8+)q7f&KF2f=>ral+rjfcjGb?q9BcBhJaPE+f!;$UiiZV7Egr*k z_WLrq8p7Y*J6!B_y5dHvj-_4g%V#gb%MRTXJp9K6uTx}|5A91|`l`e$^woa|zgiAH zOgm_ftHz;2aWo6dw&i8y#u63UFCrlUD!r3WG3h^u=9}Kp+@}#L#&)cB566X&y$5ag zu(cmsR8ZvWPCq_qvh+$=BJZZw?!kix69TH?%+1Zy?*gw?IQDFlm&&!9*JLm=qJ};4 zoFQd(ynlaP%f2@BNAZ}N+2$LQYQZtznLx0Fh)DFjc&S+Ss=MWTv3n)kSw5*fm$jPb z1AZC^5V^*F`|&)*7rv3VulaS>SzVlo;FpM9`IVKG^@$IC?_Rcbb-?A=QiUA`rTzJF z=69dV*lQw&-+ueD$fyo3^vA_nwIHf}wa#GAr_awi29>u=;9svqDaX_*{&jz>FWk%M ztNm4&zo!5z`VCc(ye8UR9iBaMLrJOWpJq5#tHZ;V#<42eu>DG`AN8x(Ta1SKwV5EV z9@%p_%bC?ufBtlLC5Jli9QQEyxLVNc_ss8-RVT=oXVZP-JhZOrY5d(PtdG75eYD@F z#w^^EbM4^+-;lL$aRpRD!h6^;-L+o_-VkvMe^>C`iRnv)mX57r9b&b4b>FEUsP=<7 zXG-k<_lvCC$gl(Mpg@>5p06aKQAM3C2d7=1iJ?7+@-iIytwp%!_`cC6iFhilG4PO%^kVe0Kl{>O z<%c>PtU_;Be2Osu(EyuNhx_+)x3)JwcXW0((Z77r69vi3Cx&tI3#D~+bv0`Z*jQVi zhrwXi;brcbrVz;M^78U`xWItetED~Ea{UAI?Z|_>n|Tura2fjJ|NV*%^u*!`9}x8Y zz(Y#^dl+T9C(FUlPZfX#B}=kXbw8B2%B!Z40>y6ht+mI zIR5*{kw0i^YI?~~nXfk~kP|JUQLjsX=o~qE%ufuNI3G(QjJuztqdCu`_`QPrU!F+unnh z6c>T95UfYI-=bwzApDxGtW!(>7bbbOvcP1pHOFxrlNUeh^*y`qn3h{d&&BPlHFke@ z+K!MKB+d5Y$@$s>d(PRQYu>0IXN#%(okiX@?E9xW({Dp9@$kkgO?8!N7%N`-Lo422 z^O|uLHgiMwvo01WK5YJHw9Iyetlh_20hrlfsb*d}*5H=($N!Ye#&)N^AY}dpn$ni2q@;9%gM;J5=^P@FSe9FJ^Mh?c z#mHwj5HOig=#$M9QmJze9Iry#yq)fi3saxY*!3!DDi*n~7%ctxWNCPK_(tBq+tWvfs6T(U)kpA82D!L=?C9#k zL<@tOHp;!I@1gn9kCEltDs)Uz7@rQ9v%O5P`e&BRt84oaCF#Cwdf?ZF^#3adY?j@h%}tBl6F z#|8dfzY04Y6T7P7=&SOmo{yXH($G^03qzl>4)XCeeR3Q+Iz3hsJeG8APfc!T1P1M8GFdP^pPPb_M&;H=^s z-L>;o8ajbR+5yWluluGlI@MPdkY=DbsSP(rW%hptV#2jMN3sfkfubPSt~DO4?2L$r zIQ`D~zRktCwS_VMOIYn-1qpD_vt%T@U-q=VZASf1eT`^yzqG zjLZf}Zp9am={}V}M|y<) zGxxpP3ks%xEWr>4NNhrwvF(C z`veT{_uQxU&ls%LspJqHI{NWwcPSXS1+@oI2&9RLbenw_$ehh3725%qfN>}&3?;&?%vtKJg1)d3O)iz)nP<7TbT5X*{vU6y|H>YS1`v|J?1bg+?g4NfmLtp z!ab1&>D%`9`bA_fw)kgzMh8|&HZ*Qc23A;cva*&d!qacB{%WCq0Xg#~u(ItEy&-ym z^QCsh!=A9~=Q5`5Tc8hZkVr)EgR0m#p~ah|Zp_x5nkeGB0#Wiv95`@__Yb>drG@5( S#7FTAK;Y@>=d#Wzp$P!xJ$dZ_ literal 0 HcmV?d00001 diff --git a/docs/images/LogMiner/LogMiner9.png b/docs/images/LogMiner/LogMiner9.png new file mode 100644 index 0000000000000000000000000000000000000000..ff7501cda650b328bc67050a248822437d7f4ccd GIT binary patch literal 33540 zcmZU)XH=8R*Z+MyC>%vVMM0&+0z^bW1?eOzDk4HWqC})brHPQxdy0yPNK=Y{0iwVm zD!un0(xjKrTYyj!dO`w8NWwpyyFB-AJ#Sdo0%oq6xn}P@^Zo1{e$U)k?7+zbfBf-> zn91!M_y73g&rrd$-(SLlpM(PHoj?9K^@qugYYzhUEKh9ZV@w~edsd%0oY8ykM)=Fa zXB10ri>CJMuQfXNo-z5+E>rrSb5mz)wL5IT=nX@?m{qvZ8u%H9hQ2f znA4GyY^4#5>0*ca*KJQ2*R6}zd91UWVB99L9n(67X^W2|D;!I&_8`xD)-u~@I4$6;pTs&za@&p_5nB-JM3v{3i# zmckqu%zn>z&ZWA{JWZw%r}1$>!DZV7sI6qoU~w6IP6eF>GRP)b;OIYqoJ_fy4S;mk7Gpmhg2)&R%OQJg@O+*U-CH7_!G=+0K17|A~r^;s_X0UptiWWP8ZvL zsO>^^7oMsf%#N??4e=&|Tfqjjd;{P#6oj7Y#8Jm)Z{iH%{5;#)!#cS^i$cikm;gMO zSjQsb&4*^gB0$sU^<5O-TS#zn7B(0n-REUUFVyR@x~M{&Ol4q*I!aQTO-|OS@fM$M zt#b)hB`5n?a;z4!qaOk|iz>C#Gf)^Rx@;zHp9%yPCmQBFBi`Cr-O&ET`?xkW7_-K! zZTUH;AUwH|wq^62m?m1dFKAF=U(;!hNo&h#wy97{nlsaJX4Gx{fjE87+UO)>U+);} z*GDs;4vm(U9cAIPh~=l6wzx1xOGiyGjE5pS_`6*8ad29Kmhbs)U?m3_$o!7?x~RK; zNDT=R0p@2O20hlJF=W%t2yOU!9|sj3$1GT)0za#(INo((HkJQ%VTW-EUD2#2n>2aK zCTPaguL+`;wy0l|;&vnW&#D%#ybxIC(#1OE6qyE_$f28ro7HB>+&Dv*UR;oRF8zgT zE=0drRTA7z>&ZaghLtGe4}#fH_N)?UJl>jVHy(ubugb>fW3TXM!H-Zgv0o$PFV<$? ztqGe2!c`Yw7kTUqwO41P#d8h<3@03vF~Pu zb=c@p_;or{u_lPYO{%amSF`!RU6y2)c9TyN<0hWKlY;Q$FU|@!O8CrwZ1h3@4Ydbhn+D)q2U_f!xzziV=?MF=A$ETNzX}V-X{)O)GUw(AC_u zK$KmPqoyI3En&X%78ecUBoKWY8px`U5urHIG%1Y5n#-@?ycwM+sLVpBV#4fAEs$h?m-s%FKsLDP?F|+>}-`QDFvK3Wv?|LsZZ5AreUMbLWp?C~LptVH8$Y5eSK#i!vS7u7t(_cq{=7Q^G{K$M2A9coJ&LQ} zF~%^Ajk}Ta8ZmY|oFPChcTOD~L@@$XmF^M4tmM>1RhLoMNSs{KD5d!-cFi*;Sk{)^ z1_hNF8+)IAw7roaPG=PPZdH>|#EIk0Eon4NyY*yI>Cy{m^=P47jnBK08P&?gh4*#K zse9S(7hLN=6;w0#pz)YldG3^mJNzP4o<}hT@TdYC3PsMw1q^nzfHn(_A;F`>va+7< z`ArhFLxjyz?MhBZ%1`eGnMscZ81)Wx)U9^2G~Y02Alt6Al()MAp4=fnP$%64K1SPa zu4wuB@9u89v>vm=iWw6gKI3)`Gu$BW$AIlHVa7ZBwH-8LcNb7d{1s9(pDFk?xzc3&c)q!+wB(v2u_M{i)QpWU7m=8$%HB9G`*L1`qS1NUSKhWY9QS-k$;9*#KkJ*GJcPx0*4I-QgU)@&U6uiLad(s=4g! zI!<=3I-^Aby&jUWhFS~8ny+-_FWGHX8N&phLmNiPb8Fb15&>6Cj|}0Ki%*n1%>q8! zR!W)UEFZ#Dj6kPZS#rz4=fP8ZB!-Xd;ZxOlnahElu6D&tGt|t7pIiCJITd3PyUV|f zfSM8lR7B~@rta<;?Gt`r=647@NTJPVR`ikn3z_kUd0q*f<<5d>xA-YaBVWZ$>3#UN zL5_fNXXYb{X5X6cx95kHT3{6~#>(q>u6)mdvo*K#%Z!DnU)1$|&Y6B9YeNWv?c>hc zeE~K%AI;1*9<{(yD>7+`wQ*C%Ao@r0@InTG)2vPK^r>C4b>AS{1Ga~?qMI!+$enEx z_%aT?<&W74NDIig`l)V%T@2nX&V^8sf>>5gRrz|we`?JKuMZCZztE5%Fmv7E9BLtd zb}KTft=BI=S)EI^M6He0x^EpHwx7}wp3jdN`wRX=%NzT-Y2?{ZiO8vsEJvXZzOe>hrTuXp6f5#*lrNRIYh$KdN5y zE3bxc_D^t3K7+zxksVU63h=BAw!TSox$8-bugws{!#uICTT#Lv8- ziFKETT@MKRhLe3Gdx|`SnY{i^I?N;tpf|B-da^WCHQXTCh-d4S+$A_v4gF87wEy{Q z;N|L1&z}1=7<9525p7n*oLkL6PB=R@P1FnFX^brBEMUD~{Vb z7(IEWA3;BI)8uR5C&K#tD{C7)S){@np4@|cDwExb@GG{j?kVVreT|{F~XUuM|6zk>QM5vfSFR))DN!lp9y? zC)w5LE?TzCKaYE2`sb6g#t@j-NDjg%#J**oQ<{&0Z;QXj(XtjU#RPUZ8NFVrR`U(- zN?k)P*vl=9#s}})j7_X@*cN^-t`mM3hH_QhLo+*3&Hf}IS>;%GBJpVKP~PZU|G|vy z)#Ww*>RK7d%ak%Gf?GQ#2+lEpO5SPx?MEooZZjfM=s^t#KikEEt9TU_)XdL@1S@XRj{EF7ZQ73&s zJtIUhRI%3s(TsX_4`Wi*A*m?=?$jm}nk*{K`3PN{EXhGx8 z)3#NLt`D-6$3Qf+Je}8G{X3{k&i0$rWzpF|az2FmQ+1SvH4_kX`*S@gqz5i)qZ2LPi}A5+L5&wZ`D65CO2)#ohO1^O|* z+k``Tds9*_#ebPtLvM}6KD6(NIsvY|?$lE5=SrHj4k*B0H+kw)4mnar(1`zF;t;7i zCVmk;dmne_LX^rj0Va7C>GARq6mfIH4<$H|j2_WNy{}svdtbXOg2yR_#(IUvLQb&A zpG3iSn^9x&+CYUtr0>~46SDdQ{#tkv$m87yith3CfjuE94Xvl zw3)-eNpT0s7=^#B?HK9O@WLUW5(@ks!j!B=_G-s&4@R7*19^UE)GKkY4@<0fDgcs# zYgTr@4VOG1Whe7eto_W|!_RMe{zP0VKRoj>lC_RRNTF?AEY6*veePE(K%7@yrG4Ly z?hrc=h{edQ>Z4? zw*y&e4#<$X%}C5d(X2>i0{A4G5ri_e15`fc@dU?^PJC-HWx$yS42myH2aD*F!{jmpCs(>yamqz@h&L$JB1e$GK_LeIJD0| z!kB%cT1?>UZulj6USOX40Z2q0Qh$HZ_o(@NzdO;2d<3qSH5I0W^tu{<@MJY2lG-FP zb0r%27Q1pi@xj)e$7jxuyv30(oW_rdoyTr?3+ojRJo-q&)kc5IRvn zVMxvGM{PS3Hx7tTs`hdBsq2;_;eRnkbQJ#5iq%b;CBlUg&Q*Rd2p>BoPyY}m>wYzi zn_T6lMfAsvHmKgn7p&QZLYKH!98zZ}Yg{{&uX(%7S%&+&S)~D#6a&;>gufsCk*XN_ zoJCi(VkT>v#&h4bn0?*bu9|q`x0BkXJH}eXMBISs>#Wr0vDd>x2t=ser8}qGmU$O% z#Wt0V)w!`PF2??ax&9V{kfbv7dR1;9n{QnHnj#A*N(Fy;Lrm(}dvCi5hy68+SL1z$ zBP6KtNAKv=pdY|iSo?|yj8qvcqOoLPJka-UievLHkQunC2JF6%#*5Emz zP_A3xXTERWvs%{u$>apk*ReQ_R~G(PB}ECMosfVP!QtsyvxgokQJJP-j7IUsWFtwZ zL(~crQ=dQCaGKeqJCq+_i6b#aianiOpN+G&U-mZMq8RU6EEbE4$>C!GRJ=Oo#ZY`kw~8)Lv3C)Bz3w;j z&Y+HueZXz>qE|I#@LjE2Mz*P!!u^T7k)MiHoEB1E!-B7&eEuA9N$YqQJYV74r5N+_ zOP~Je?>d!64Xg4CmxG(Pa2+*m@@4TgeABVMtOYiNTXX4=uEk1MQRcCaRu+%QxXL_&Bi9=zdD=VKezvRNDBPDd+6w5RvviI z%vdGDp?7?^L+n^rT$+_%gMOW7L>S-*rga4gznwpnVW&(_1|WF-D5GzH(rVauj#EBF zxUZ97twXYSD};wlyM{p{mMPBKg@~%ixUt6uv8p&x*5*LWnaHTwI>4>?qa2!~T(iZm ztmXCIB{CwOeWtY+d}NrT2Vg?xkg4G>TH_nyu@61>1J&67U|3X(gJe=}QCW%#zcIOT z{z@fZ3NuR9HRZ0CXI><^n%=u!L0EK*S)13>B~QJawftKKX<46$)=WU)4nB{|^y1(< zVs2}%M=6g}bM27r*(=o_Y78PWi4ZK=Y`!WPLBVHFnUvbt~&I9 z-_q-$w?o-yWWw-fxHQBKg@cGyu1@`}bT=|$qdTwjyV#2rZaCDA=WteEi+n^V$jx9m zsH8!JpRjT5%wF}wWy!R|%MusI8Yx3>0fOtQwRP!G6a;$$S2OB>yC7pXRLPt%Hu(Z; zADvn-=C}qAaop*Bxcb#&E4hB0GxrceZx;q!nw8iX(EZ@td2@ zE;aNmc)H)Bw07piao6I#QH17jHEi8b8}-f7bbH%cvq{ktH%6ku1F!fl!iED37Gf9p z=}U#~oW^IG<|b{D7VSSCd!4`3f=nO#x)1QXZh6kt7qx2n86EbvwH>mlaBuzDJ$=eC zBVN?6=6M?$@Z*3K@2bdzeA&j5P0w*^H&-H-h`v>EudW6&h*Zy?ckTs1L9Ny?>O}pi~9E&Nf0QCR*Q&yr1w12(XS{1G#4vE zbJFL}#)*Q*WZ{eV%PNBuL^r~Y$El<0uHxYOj3iysaL!CvKBS#oWC)5jw;m_ZhW>tnuCA)5w%xEr~GYqW}J_+ero;f55VhE$I(}3v*Gp1!LgIgr!ebglWq0iHLSG9 z5U%>S@5=L!yYK%Ras%_ig}yu&H`H3|zDXF_jasI%LMRIzX?i8a`AMHj`Sfl1+B=(j zc3il0}1jEl|go<`KoEw=c$v2>}52_!IEc?|OW{INDiKYyd zAm*!je=NVf`|D4R`7w9BzScya`kSz^!;k=GKgwpX z5OB9{{X=)m_w2H_qOmju3vQMQfdUfp%vdzLf%*_FxVTOJ@5Sv)o%ZuRh+>zogY2bW z1}9wkNVSBv&nFVHz-)3<3Z z8r;gM(bZ)K`%>@QFtnL1b3ebWmw4LbFaKCM#z-EE_!Zt+7^OocOB$dQ2C9fozJk&}L!IwI6Vppe@W#DvmiP}8S3KjaR zSb~BDbkKN&wd)O zd((2q(+!p)%f6NO7~{l4i+__LPV|)0@%fuARe^H^^h1ASFB0Vzhf`u^jOhGR$WRTT?8H@feTzgF+Vwkh9dm*n|ze17+%9SBXl`N+n zGeZl6YKyibG|j37cXu;{<)k~ws!sqd6{mFcpAd)Ds5HXxR zasgR3WwB;?K^~kxjny#Z{Z8tHb!wUxaP0!H_!sBAZFW?`&b?ZNE5QS_7eodmFc+@@ z`ysLsWuw*ix8dcB+9Aj1`Ju;$^tOMe;)HC&%~X4u}k z7it)gVWx|uV*SJu;@rtr%o{LTb{?xw(y%6I4w)^V8*~VbCFSWhW=2zIZkZ7N9T>cLr4cF6B9}_ubR>AZ)VJdOXPX2f50TCV~$PXv^@q(vTcP|hBWlaA*fEz9q07JnIA;rKUu9h> z|M?$}v}aEs2f8gYMa{xNfXf%@2g%NHY7_#eYqP!Zn5{^B_l;KX-Tv;XZ$@W9JjM>>vGPx2JX|Dy>Q za;bnMczFoV=f73fe=n{74o^E23^~DFT|&={|3_aN$Aq@crqP#&Az}Z0$Nk^gpe+Ij zMJu*z=-9FUXoABZA9g>m6<3ee`_H}v)58A3;SZv>e8HaoXUPQ1%6=~xwFmgbGQQ#e zPJVY{b;<^?kcPuxPjH^1opdAEC-AfJ!`I_74HJITV zK28YkLS6m`Q;xbzl5GBh@`lhWLuyAedhy-4ds^>~x;R!@CNmnaM!gLpL~FypAWFKv zC5v(DkE;fCWzfH7soAnNOtZ@N%nQo1cg1aK@1X`kza-N)&Tw+&mkKPpz0JSs6@^z|qcxxDje%*#XHgX?>3c+xC$$%YJ2GEddA|3Rv zN=qq6FO^5jZx6S~8w#Z4iL~v>ErRiDB>3RLq+q=FEP(x+d;{Xl-Ie#VHrk*B zdZ_SnZw#6dx*i~q?PT1wRt0JSpq#vg+1ZNWa|DtipY}DtMwwbYcGi>SC7xB84niD2NJ%Ged*soh3AE1|0frK zZXlpR!-EHc0L*X3a<&_vhiZEC64Js9W=|Vqw#zZWjFbg~1iY;xv(S$>9L&$QnLFPCS{M@F%3Mb_UiH-Vl_q(#%@ZmMx)i<5CZ{TQN6%=%a9dh7 zQkO&I8DHR&b^-tbP80F3AYbbY@<=jveY&;4Oe*t{eF_ zZ@_Kk6-}zqE1$Ew1DZ9;n7<5&^2njeE?+HhNpZ+i)Kj&@-|6k@lu{6AS7;uUu|<;$ z?P}-=`36BNuCVG{Bd?PjhK8Ix^GTVS1#8v6F4+5}g}l&q?HbD6^plRTvj%HdFwYPm zxF&2v0D-sS?q1Q2;cplVuopC%FAys@)8MaSs6-+3`Gx5eTes(Qh}8%B}1o z8Lg_Seeq^kF;v>?5IdRu$<`#!V2hB$so!*~Gmpav>!Ir7v*4}DZaDm+jxa|A%&PC* zO__*-e3&N*_7VobTdLzP_3qLzv(?8#RfDvbuq+|EM#`yWo(Fp(fWO?$U#1k}c+0&Q z-VjC(#UbJ*=RXAc@3JhcDUQ>RFJs$_*JC07VjKRt#Eu6R)6})A_`|!%>q0%Fb9L2UlMV2~--Hy!@Fu(?DxtK~gC`8J| zkFvAtSaRi%ja$nC9!D}KWbe1ZYE6|z=CAntYfb|eS z&B(P~d{n^v+po`d;$$`oxmShg!kaa&(XZilwQm@1HYF!@4mR>O1&~k*J74?eFIdg| z8}{_!mbqh$B;cHwnp13H^gCu^9e1v!Vp;yRL+&0o?s^z_e12|M@K(KMioenr^a?@1 z8Vz0wGu5M^rbS^oY0f^SdABu(th;vc8F)2fcJB6QyMGmoE=;yj8O! zSg8W)4+vF--LYe|0?MzzaPSyC#Hp6?Up!U|7-V;2`VQTpPgdA>6prx5lFfa8)d^T$ zt*T|Ls+@bFB3K5GEX}rs$N4{l)=i7K2Pgz;9M`VujVg=EnvKNxCS~?KXgmVFf%_wZ5vwaMiQs zdF5AkHT1GQU~E1S%?SasLRL!L#GWIrgzc=Z@exFOPrqtoe~#(7iN;=|`3%90uPnKL zXJQH^5Oj6CzKD^KC;_diQ_LM83jH#9PA%!N&R6&-<*c1xAwB6cs}G*eIeWx`J9tQ2 z2#Ki|wJVtB&qDYP);lcY`CZ6cHhXX8^JLx5b{)x~pStA`W9OPDl;hwx(@W_Xl3R5p zvLjzTt2>@!CktACB{6ReZp@^0;@?K5nM#yrP}a3E6Fw)1<0ni2+&|sZ8(2r zZW5HWr1=~@r7h7}yz9wEHUsFL%7L#GF5nunhw_l2M)|3|_#7{YPx<@N`N5+r;{Fv4 zS{T@_09uuY;VrqtEru)&#H(1v39tcTYHs_%J5`s(?;KEc&bv3DODVmWawEcC>kqvD z+{jBQm2x3rZ1l+^xGIZ+`mGnhD(DU2p@gDrr_JG)ahi`C6+;&%#xHLwwmU$)$e%m1mrBF^+)Rp)@y;?!O@P$x6f7sncM*ZTcik_&DyQv+d-+%&{c*7SBa;)I$&Ws8@-Z~ zxea5Jn`QPBBMGy*h26)^gu8CtaNe+?JLU$p=ugrtwEwg)c9ltCLV`D1KO2R~90=N> ztM7PO2j|$cXjU;fkip9p0)o%9w$_I((t^?hTur^`QrpyEMa=SBWe_+;szwq(Ga17cEOg}1YA=WEkH{%ji`kwV7a z+Ou%1wB8f^`b8!@$0j(u!$W8v7C^cJq^c)>-sO#{*|#Cfvhk@CjKQoICmN@xs@ zfqUHlwAc4T$*rUzJMuU)&wm1R^HO52s)L)_eD+ZL{47rpBE(2$PhIb7355HLMch^( z_lWt%@VNha$tmnofI_5JI5)Gq6X$SLjaNT-wWMwE@a2H18pUbv{o~%a?eSdIjWKZk zSGRX44Qf`;=uLMx7Nn~~bU_VDX&)X0?q8VAu4zE=bYPcyY;I$GxAP%|ldNWK`H5vG z?Xj6kmgbP`YDe5`$1(2~*M~YjxW8{7ocL1Nxv|-kwo^4!yZT6tZPb%e9Gyh!RZK*T zC_do^2y!#ZSJ;C>nxMy$<9D^TYQD$+Oc46gT~4A|#tLfrbN54$=xB@tGQL}S`1dB|dIKE_W=$NQ&>`2IWlqbztv z08WppiKa*t6o?E{69u5lz2^1w49$1u)7L(9JoUCJav%idUI_Q;+k7$TLV7%%RX@I* zAS#RUkeXO$5Xp^*+KY*_r1mznj&oFjrB$_4B)d?qP6?40&IUw_2i0o%t^2AxHR`qG z|ExaN^J(x9pt|@giBd?Vo^_OwBm|5U&!-2!5IuLID~7se#~(@7*r|)UNwY*+`u~>1 zp&eczMizpxFI%|Um;}SwPU+ynz-L`aB5DyGVpX$JGvnHo&QxQQue>p2>sw}<{92fU zkB)w&2!tO1yx`7*g54x*-^q{2^kn}hvXU~!~biX<{$DfreU=qx53Go=Y_tY#E z8R`z~m9TO{%A)UkhaB@yvnMX*Uw!3xmlu+b-m}reUEMT)ZJ(B0>>k)|UDnjGf~}nW;PUig8mS>UU&=Z^GFLI_ z#jo(;WbFOMM~<}H`F-UBe zo~g6=t*(unH^&4ve*T%rb)S4GACRA@7{gA^qgHzrxh6}9H%PiiTZ=48RLz|Ng?NYj z0OTlA59Xo!0+T_Cp}ue5jj*z~`xsIx)kSmJH&56;`C9D~|MPsY@AZ*|b9f|I_RIsEo? zv2==7j(4pTd1*I049=t8b2aR}6-Juf9w;5No;w%Tu)T3XZv^KveHCC~zA(-?ao0y^_I>}1PT=D4 z)T=O`)ahWwJxcA;5428`MAF$766O+GhrP6vNupDpH(!>dtHK%4Ljc zPs$bP>6$yM(AGf!hdb2~^dwq8RNA%Eo`2h5x5GN!bo!KaPYbi(r zB$sS8v8fUB4UvX*SqZ>l%u4kpY`e5_c56>_dWbEZ_p;ypNQYv6Os8ngVyC)0y^F(n zbpN*#?mO%69D_G2+)|~E%!^1rc32&tX>QGGMj3}XdqJCKnUEsulDD7LC*z}R3DQ8d zA&~X!^={C*{;99g1r(E%3ooS1PUJ_|)r%ntu5`3M0tOtLDSZbmlDbyzEn9s0QIA53 z`EQWGf%@*p#pnm4#!36n*iSR3zh6D}xP;(_O zwGAk*@WvxCFWm0f39%1HxXZXbM%<_&=(=3&D{a^){%eKDDJJHt8i>D)Wl?l@0i zD^`BAF?n~P#(TNrN_dR!`4bxD)nW|`9_MJqdxmW&#ovg|AAPc)$Wgpwd@`^NilOr^ z=&Ixo7diGN^Cee0v~ShgPGx8xoUd3*&b1yC#oTi4^5@(UE|#(Nx}f|(V!_yd@CZqn zJuL=sGv~C%*yiAC6TVgt%?-*dodI7X$rb^7GrPB<1Gv4`uHTDF!=v36Na7g3`OY1P z{pmr2Vl$BIMQz?s@0mGq@re28h%?@ueD%K@-OYV2Uamt)IpyJTvNe^u%(Mrejy_JD z6|z%|d#JtBnsjz>ulKm#leFY*O74Evto+0aCI8sY@NRcpU&6-kaIWtjS8H^^%9NYD z^;v~CawWOWosM&iFwtMm?y5lV?>`CcUn4zlO&okYd*!}}bq~kx5bkBcy$qL%`c3!; z+{@~R6qy?l26B{u)OcSzQv2|8r?(fB8_4~LYd+RL<#7rgK0R{}Sn45a{r6;xpkJwp5HQM-2>4=n_Q(xRW zj4?m{*%uwzL`tDo4c4r7-%Z-+43-JAsDc5o!6;0K+MZZ$!qKj#-q(60ucR@z?Yyu^ z-|(1IfbrW?AA78<$Dh$jS^d};hDd2P{jO7~MetDBW9>WK8m+CMNOi8qBW8052u%@1 zmo%60hx>=&QE!ISpE!okGt2@pn;;$1rJj^y4R)n>);mOL9c4w`a*f}yH@wZ3KMTd) z3)evDcT5 z@~Q1soRTZoU!Mm~N0eTbH8UT? z8Q_r@4i#2>ND&TH`+VhM>S8hS+vHTHe%iq14kb0q(wf zGh)&yWwd^%3TA1vq_pg8>eP9dzEZc{ z=4r(&pTYo#K`~2VY|mSTKC4K2i=BWo!c2}8R6-$1@v{>2|x*X!3M)6FmgK#XI#H{$5{ zq;7ABIS-*fe&HqW=01)qFwp&WQF07X=`+V>xwQrBM#$9Hy?F9ABEhUhe9L!9AR5t( z{XLQAge97sxGQX*O!vI~iEl~Nj0pA4#PAs5JQi0kLcP?t3+lDVxNF;{0@E6=DeEek z|5p9*L5sGp+z_dgL&R{L2tFaC8%Er~cl#s&2OR8L&2o$j+W3@Vz6H_A_2 zE)g4g@=(}twn#FvtO510RzSlybSt{1&9~Yz_a)o-ddx*RbG(mqkLD>8-`>_r zt$!E|*gYo-RO^{U{+2@9-n&HB0Z^B)>AOmW?!4h$>5n{M- z-Wf>nkuQ>#NUA+$gZe`ngzDtAYA|hIIATKe9bL&f2}S{4UY~%5hEpc2k>NPjZkY(y zB9A_couhi!Ot~1iR?~1dyZh@#AGs_FYuARknV(<;010%rZj-rydZQ}K6i)vY>c08@ z!25x1o%DWG?DA}KQ34RPsIwoUHu7(eWJ86-=vM7^U~5XVky9U}xRyRA5pYtBmmc{v zXd`NMq|9Y0(@XEC16eG+lOCV!rNS=>Dzju0mowezvV9uV)|bF(YEn>aGB=^VjV0ci zs)YI6;Sw~G%qU)LO`Q617FocQAi2~4OM_?dT$8!w*R9PmH?VWc8grIh7p7Y-9Y111 zn$^v8?u*+4`9-|MyqnynV#`xNwch1iFqTMb=fBXejGJ;PGtcPt_ndL(81p|wjs7K{ zUYwdnxmIG8H{lVs-mCs_Si-F9pLnAyQtsk|6?&)A!X^BFH-C*%o?sD+KWzY$q8^s% zJqjGSDFq*G8)8_B5KH^i%u$?pu?i#J5jh7}W@Qli0YnX16bEvlL6Yw3Fa@~SzHg*n zvX&E{Et^93V4noZ4b-=4(AoqZ%AmiS+N?3>q-$O8mQyrP-xkJ5H{-DrCl{YoS&lY+ zht%Q|pcAxT&^$3gb&|Y%j#!+f?mJ^$^u@;&i?sT!8`vQ8*a+=+`IjGA|AM4asQZF{ zb#wZ^*R+sA-PumyL8wQ9j0n>zjmZiAC#?IH{cXR|q_T9mzAJNNtd=;n)vsjMiFZhC zo1MI}C}`ZLv9wSmi5ifWchJYY9{H_~v}lQkwF^AeEOh)a&VYTW+iH;}+}fWNuKz$> zB(E&JzS|4wiUq>B#p(_FpzQU;12%ELQpBAyn^cTLC@Suk>y5;s%NGBR0Uuv`6Q|W~ zLAnr^iYzu@cF`G%29;?Z^>OR?UJ3S|#I-QwZNPETV{`Y5$yT^I3hH` zFXA7csTg5=&Ejn(!jnx)zK=1EX=TT}JLWu~%g!QL+3}Jt6LD5)gg3w;Nwt6@{2T_W z>(7xHk{kI#22t|vLXHY9J8k^5jw2A&v;Zx5bVnGYsWbP`_%dxDjcm&%%Ap3G0|zwV zxQ7d~SdLR5)4uI4#wn0h!7DFwlE=UVk$ zR6upFo9&1qvzXPnb;Z`Zdp!V!E6aCDM54o+{Oo8oW-&NvN`s5V>2PCM; zHDMIC$2*CgMioF`A-&Of>xcay)HZCfSuQP)k&6slhMln?9ZfSeNtI5CRtCg^_ha7q zYw2-LTlXp%I4m4Ov_0s$N7}A|ELY;k?f7a(HdA)~3(21W6k>Y<(`%v%*QVX@V$?t0i#jIAXA^To)aY4 zeyACzz=ZZ&tbrJ#B1yc#5_M0;&2Gz!c8uL#fyqy*iX};JvZGC^&&_f51&SH>JytJ1 z8$dtHIX&u9QT*Fcd-4en#Ox-du^h7UOrX(t`W*>e6eW!N&M)~I=8sQ>Dc`>iK=Jy8 zDYH|{kwykoTuJfospQKE4f%TX;3lB|Uun(3M^NVgr!Zaq1HRM6l;{ z#y{S;8t z`NFRZhzxPas3gHZ7f=MX#;>;3yDWW(c3Dv1m;`L%`95ZKQ-j0TV1cpwBzmE8aWJ+5 zfNTVVr`?>tw_PNiVlbkUxXRVbBwr|rV}#*A|EwuIX0gmeyxm@?@V}oOlPssp78TN4 zypN>u+?qtq>-vX=li^8T3A??~gdud#K0K_=C+HVME)4la1-X)Txwkm*WXJr?kI5X% z>fT=quz+Qde6r$ZjO8hE}%Fb3Wt@l@W>s2pkvTPtU1Xo?Bqp&vhoc$Uq>BS^8O-^?)o~ zwp&eH*j>Gi;-C*HUNuwiO|>60tUx#L-lcz|Mos;GJ=|)G5g#)0qd&+~|KWU=?czLV zwJp$&-hpkW2NH-;F=`hLbmlfRf@_7f>Z<~^Ay2n9?xlP zkpaP`oGUrY*$xu*19+=$ses#lrNJk`lMhA4f9MO!_)1HV6+M|lm??e!b;0Jhn`rXt7JRojM0Xvdh(?*RT3=;h^TaN=$Hb z0n=GA>ylwosJ5^ML8au~@wVd8duz@#3pLwTDD0_^2Z%Kj3R|Md7bVM%p60l4Vhu9^ zh(*)xMl*IBF{Tw8#romwOg|4H7%`PGpXmMmQv6mUKj!HmbVvd|*MsG(n|2H;XPD+@ zrbCU3K{Ai!fim3}Ut2HH=By1Q)8*1z;?$`Jz)KeL=v?9@PYY*9M!;)9HjJdUW05|) zXNVIIHVte8%;0?HB4&a?Kn>r}2;dm)E~g}07IInAoHLt|&p7}E^lAIn=Q3HAF!H`$ z@m{hZk0}(#6lB^+V}gE~NPvCMg>`qGwy7dp`G(agcrfh;u26~SZ8G!veM0 zdjap<^bmr%II&23Q{6w!jnlkrxg*jK+pOaZtXO>1!7GVA=zS zzaCSX6S1)j>INI?xdp_eQTz9_HJwl-?{;p$}4qw-bFyut3ux9_DjJ%=9`wUC-f z80!{hHJZ88wm>>o0?G-0S5?}=JxfX&&RAAik&bd)+Q(~SEw62sd7MH+wuh|{2dIx% z3igC)*#oV;QMyp>h|CcG(a>G?`zv?u(hc=jtnn*#3y*gw8cICX=4N~5^!?l|BS|5% z!n&C?m;rMwjhlUJXSKC1kRFpO09#A)J4-+$PyZa!;oq$8S^?QF8ox2*ygmZ=J+ev{)>=LE2oy!Gzme49woyyd+FG$) zY!iRAaK}%L$`Ag;%2t0-7v#u0t2?iab2UA*TEdeixh%nWiJi@71M>Klj~IvpW5*9R zstWY$j9rXstX)0QSyohKEJspBct&JkN!gi0%#xh#C<%jbD~cK zBgi^=k@0GA`NccTNwc@yUF0iD_0;q1J7pxX9YCAd6=xX~O&tSD>ApAK8OrtdT8zaf zg<2r!^>91Cb>>eKJJs7B4lKL6k7RtA697;H)aORZN4_PQgyuh=om#AR>wVQ6wwo3q zlPa!L!?|N!O54qWxCAmZKf62&r~pOnPi4oNmHk^v2M;~lUe4+pNa{~4jU#y=*UvSC zLM6&uNRa0FzV~4Kl$4IWL3^`k*XJh}c4pN1&w?XwlHDTJOt`&s4rXDj3zm& zk23QyyxcBnzutO7J{mm)ra9R}Wn5_ph|utSYLlBknEy;1AnzN)k;h0ci%I+L3@k7+ z*m!U!d!FDWVH4ojVN~>`GT`|Ryym1emldfj*L)_;DVZJipivJ)JCQ_zhvjT-H(nc&A4IR}6!L2%~? z%38UCd)_B}s!;&l6$Ux+|EhZTcqae&|G%Onr%I8WDk2G?oJ~5AuvOkjjycWwe3&&9 zAvxv{8e1YuPBG`hoUP6Id_K%^bBJMV!~E9g`}y5|-=CNNxNR?&UDx$`K4168{n~Z5 z8z91&)kNhUwa%uiX97$e!SI-BDa)=BR`)AJ04gLSke?Su!L{ zq1n7#@Z5Fv7N4d%ntO$vtq1A{kOIv*W)=AvAdF`h zGiq^d>h6XNvmPA;br1*&9r6&vI(CAgbQ27b zio&dCap;B&t2*?VK&xS~h^fLAU6u32P`?myYq;aliaf(n-E~mUYY)55L3P@~qpe$h zD9nP$8pAqh7Dq3S4L|Z(pPwERO5zwDGX^osoV0s7TKwXz zJiSAyLKf7^Nd(i4vq^Evnvx`BJ+7n-Z56btF2xv7K9IEtvHaC%c%5bNY?P+gU{Iv5 zTfjpN)yma#$Qda?cl>|A5~y&V26>U^bRRysXT1nu(2y(%py$Q!mehmuL7LLX6JH+H z7i^d%gvJlL*y`TCtvGpE2J&ZA&!|{LLp=Ei zGUSThZc8rdCY>@x_=(KFheIVnwM4kuK;t&z2Y`|r+HR;BdrVI%eqWqtIdxTaM&Cxq!dVlHjydowWPOl zU8Fica62~!lP`88y=k@z9&XhMa8yMO@bbW_Re=Q{%F^QhymI#2fwKGLxF_W}Yub64 zsd7|3zANAf$4F$p>p)5Y=vB~!d1ut>ctQ6tXhHKad@rY@I{zT)3uqN-EGIgj#ki#9 z6c^=FPcmC0B%5lrDY_H#MGP=dsNDUrwS`AL5iGvnfImgLvWdgt@Z|qa40iK>Cx#Z5mJy+v^@MDsQ>YSgheF8^Zdzm|zL;{T!HPp2KMz{{ zr7@Z2R_)hjF05%0A};u$WSWeSKq7D)X&Czgtf&UN=#gzJGnjTU(#MC}mfP{C_#3s# zFN2TrdY*+;T=s+wqksyuOMrBXyGOjYQOcUqmcs7Lme4p{X>0#myI`?eFp;L-Ns3ec z3RO{(MhRwya;6G>+yK@xS6q{o{X3P!U3uGzcB!DV~W zhe9m79gbg;d5Z16#@jC#Ar~%b3cboO*u|6LP=aguS{c`_GFA&pNM}rs`v2(Yi>e@D z=X53DUT+UAs~dN56D3|0Upo`g5hM|-FoB$2M3W_Vzz^ZS_CdfTk zkVZ2sIuH*OEI3+RIN9rBSE1_v?S~|?tO6eaveL9r`?Zj7kU&rAG*S9~+7#-)Yg_`Y zwqa^ne$`>3dXK57!`sS1DT9jSL6lz+bcp%pB>>WIBXx=YM&juWN+wJOk%wTya088} zN@_;U?Xp`Q-JAV>8r9>htZ8GH#-N66RoHFk)m3LpW1MWD+_Vn}IWQp$DrZvTd5%Y* z^Rv(yaE5_~5kR-kk!?u6U($jC4Bl;1_ZR2=2cP4YIfO*K2EaiRpxH$bKStsrlxl2B z@jJaEY``Z2Js(`D8N5X{sEIb6VPd|r9-Nn4+XXZUc|FkuoD#}!mi&y7hy6{A?WwTQ@o+Omvwk_N>6t@7y2@&}<3PpTfL&QExMQRI41i zrl4x$eHTze<;`SnuX)E~14IK%yKytvyApF=Uoj7ySsYwFkOcR&NzE52ftE5!=136V zBA6QVS}Rvs^)UHwBkyCD96|w?uSYXj5$3;*GXPXWB&=%u73++^3jJZEk26>0Cojlk z*igqu7Po)Uud7goMjqA9DZtpWSt1?dwfu|Kew8E| z)P_feiM>t~DbpmyiFBbV5=Kgm=Rr~$8;Qyw&gI;p#z}YWnLaY~f0nF}Eru;|D z!+T0_-6+V)6-U9)+P-8S33Mf!rYKK;61=vxXw7oFzW_9T7!ecSsBpN*lOUOKIjup; zR5cw|NlY*I6AFW%3^ghqscKTNctOqdIDfbA$|6)E-UPa!-^&d`45l&b`ORedak=+X zB?zCEi510dOS@+5Ir7l;PWI)5#evp@#V{~ghGtJhDCG>^?<5uT6s(w2I)(Ng@%Ff+ zG%7HvnSdqne{k_+Wf8&a zMup% zWfwUyVdLP+#w5VxYgNID(5Mygi~9OqS8vKYHjO!=YHv^vU)B8c`f%H^(5VXx`}HfD zRq3FPW5y+{GKL&ib?p1B!=kxhN^QY>R&$P5quZSEWDIG8DFu! zaf_(EO1=Tz-hVuJcNG2m{YTb8hi?t**IzeM*^xZoR%DruqmdDkkp?ZKn351DI8D@n zi>%H5(wvnpLb@+iOs z#ZcTIzsvvkpY~<#iM)t)K#OT(m(B6AY}8>Vnt>!R>8Pxt_2bP3CY?T|mY$>Zp6Nm2 zm^@ZEWU5gb$uL@{^~^7g3$7)ScY`(xi+K2z3>!Z*SX7XVW&hrmjuMEWtZ!%L6#~a6&w>STNygw`L;O4HX z9K)ZRpZ^1x8QdexK93y2$`7 zq5zb9!;Z$zaJ&6~^>fDo(~=O#QB1c3zG`L>-B94jBR_*x1@-gKB9{Pf|KY~n6$})( z?^LJfalw>D3M3=Jc9TVv+SI?#bV9jfXvg2|>DGwlh~p|3k#ruvM0Ee0Je$@|Mz5R0 z@$Ar)B52Fp^wUO^=K7mS*jC8^fO)cc`u;of`pKD&QLz;$)+7ca!S~p8|6kCf6j%!f z;fv(r5^DV)OcEsu+?k>tY-PZl*gn%n1c6sMDYcGdGD_67cHak0XPSOt1t)WB_^CW! z;bC%@81FSCWH9u_h!#8rGXffUy75LjtAZwe>&=p)@I=nxztZ_n?XyUPI#W#~*!|V{ z$e}X259)`s z6Tj3ijfWCq>&AYE66$^idm5lPtz)Zdz#^{$`v^CkZ8vVp1>yKOMy|`t{H5A#-a~Z} zCUz0~_28V)e<*j-;74pIH7c|^vD>*(f%Xsig%?)KEzcF{PKcD)8yzwGITOp_Wv##v zEWd#%rIWL`BtfQ6?YU03)Nxv$?a7tUmsf>2-nW-eDoAPwGrsdTIH^0h6lyc!YZ|<$ z8Mw)(nQ+YZvsGiC37N*6DVov=x zy*yM%TUsWw%+tWlL0Rl{vSC#0HE}Vg?KsVZcdrk48YRq8B`#bVU6fns$6# z1^C9_$S6PAI({#fy#a`?NQ+f^$D4*!uXl=>o3ZAvKX>>)EHW>-qUVYH+MHy^|6uMh zTE2!bwe{#HF-oLKD5}u92>GL~o0@p+a?Wu~yrZnjG}`WO-(W3y zJFOAcs@_yK3xf>hh*S`0g0hx#79Uf0c#E(_ximU6ykNV-gJ`^14f*q0w<{D-*Z)BO zc$YV!>STe82GhiKT;BOyVbgyqKVlDV!z?s3nEFz-%!8^c6ccxn@gP)lk!S2|sYW|mpHOe#)fyh5b zu|DUVV|LlAq%8>N;FhIg9fOZ_$ziixjgQ2Hr+3GFh12sIlJWn{D^&sd_v*6R*0v-i zZT?4bR>@#`tTrb`ZmoTdxmUxcHv8utEg-33Cw?>UH*U1)%agTxdNI**1;%MXb;CYO zuUYJIz|&T|L?M(T2TKEAZvt#l%>3Q%ZIgxBy~-+g$*6y_m&qope~Pug>En9-4^=pF z%96{zEig-Q2fy$eLGWGXwY30_gjl|x{P4O9#l8J6SW^`+D0ba9%gsB$#ZXQYf>V<* z|A`VkwfCr`R~_m;Nz}WtqaecjJ#ed?@!m8i~V_ZsULB)4i=U4D3JB-()g`YE@CKx zG&*FaIxu~b*SKL!1hQw*U(xe#DH8dOMMg_#o4OtiSTY~&qn-M`KyjHoq|n)9g|*j` zxp2_Ak+#U)e?q%-Wp%%XbqBdgKSHmL*5X24ZnwnRUQ4awhTeBs5*;P%KH-<5bEu@Vs3o?FL|F$F=u0T(!`{zE96|bD@K5;z~ufaDuFi?>nuB z*Jo=_Apva03cg(VMdVCYlDFy;D{lKZH%16Erlg6{*k+GDs=Xf$phZx+rZQX&DtlHR z9j0B)oW1lCuN|{&@Bo$o{{2??&VfHVjW;uG?(<-&+Ry)kV!JoNTu)e|Ua>mts14oF z9?3`csELg|G}f#GmWN%>&@<(OF|~RaWiMKfz== zJf0Nl{?Rm;$%Ivl%i9GhSnojRhH&=lrt;H zyXOL;@08N7I~mT0B2PF)UGd@wn?*E7hM4ekj66}%vst+52=jcEwc&i6ORZyW(+Z?w`!(%(uu&Q{5O)aQS;?1_99&BA{>jzjfud8=I*Isbj^gymNi z2vxlK&40FMdS!1(m0QCP_Kbg5a3o|t(SSy*tG9CV(SnOrq@>F8qU}8O#{5-$f^-r% zt_K^{yJ!KbX)-c3#}umNF{C0nt6g_MlI55g-g;Y~KB{`^yKCQ7f^qcw{(K_eJ(OKs zz{|apWvAEwMQlc#X8C^2{*xf_v1fr>ovxj`N^d=lChg)z4^M&T$}}4-uWttQyCb_F zK9>YZ&fd%e3SdqqT{_3-P@CfZNh$NpTl(Vd2Gd?Hw`5&g?3c;&B{<@3sdB0Q&HtX1 z=$)y3GM6Twt^3>j=>gD=5r?CTr(2^v_qLyyv+gtKdZ8IH%Vji|?5d_u{(bwll;j%I zq`RISZQs-R7yK^G+SZRs`$2N%B(iHjK#l!$$>sN>nq)|Yo2d#aVGFa@lqE z`^V!IR%C5>?etHX?wr*IlX+e(oB;wBJ3GY;dEkQ=hSqK|+BUEK5v9Y?v`$HQ*X0sO z`^Mg-*+Y|F$02&^bWAOS{P|93VfTnubfq{#i*^Zg9uYf=?Jy2oeNd=XWGBnAHO4mf zJ8xO+7B}6I0|_lD>K{bL=Ju(?2q%ztf?tzOk?lMpErxU}z$DZulaeaWewE`Oq2`v( z(JMTF)NStB)E060-TT_?q^5ipMqh!OR9Ui(TflnK4653OG%hI48%@2ctNk_~rO$_1 zc$D0xdS`eBmI;tXi-88%Jf1D79q~Qje!?$W#0Eo)+|yIrHLuW1Ps!`N8I|>^2OCcJ@nMI{571649mfNeiasU)zk@`(P9vph%u_pnsyV!qIdO7d@ zxtUC*z%F&v(Xz_dI9#TWVhIB1dJ$F?M~Y^GX2!Y+qY`$$>|jKdDj|>=G<4f2J?U}) zi+NxI7msiXev|=J6GzZlfwNzCE;hr;zBJ$DjScxi>HS(%bXbLONgK7m>;pxgf=lm` zJ**1XX8x#@Z)16Opb^X8xFn@AgIzf5wX%_2ZFs$YR0qSwRRHyqGCud9j&qfX`ks(|$_jij7|OCX!S9 zYwn1DNtvLAS{cVm>Gi$r2DMt8(Mu<>+XCfV0Y+oeGH@biJhRlT(_NP3n$?;l8oivj`D0l9f7ZL2Jp3!`w4ga%<H8?)95t6F~_3k;G_7Kus81?pBey!@*f6zBKD^~K;xJ}(DT zoPtYEn>luDQ^i>oy|ADlM_~snd{mHsGVYuNG4%E23kvqj> z`2B}@pMDBj(CUQB5uXk0+pOyIFq_pr{c%Sm`-;Dd5;6JGM3>vG@e{&N#}ju4uIr)yj>S$sE<`Hj?7kmNV=lyZ zDMu=yS9PNs_6(*|!PcbEs+eH&PuME6y%SZfNj^)mqJLasDr+jb&uFPVA6?N_yBWdV zOht<}D6NRX*xq}o*Hc}skA&}`)-`3-uktK#tdvB5^gr4-)lt32@-m7~nYh=a@2c_X zWS_>x9lwH|w9ur7r8y!Y_~A1?wtAL8o1^W>@`k!S&@90MNZ3fN4wA!Yj$A$57CIUP zU-oLS+<5+F26kW_Lnz*I{oMtTy(IKxF#qCp;l)W_j!b*3tL-RO?ZNl?HmB^@{94NH zYQ%q=!C$;?o4?~cA$X=$`mA=&%maJTOvyL?0%~=9U>1Tzb4da{0PFt4Ff#Pw13|vV zy5h@I4clAfx^|NrPFvy?M}3bEc6j|-iT60=&X37@2iz#lH~WuE*3wF9v@xv9@E2fp zl4CmzAIOEH@uVkAdhW=%h;v!*|78J?d=E}8Jf0UU^f1}lBO;H+59X3O9(b=A ze|p6`spK1d?v}P9TU!IAc|!m`KcqJf3xKSP`3Xjk@h$khpP#H(SP8!6ape7cTk4!s z79mQ!6}At>9{TzA=Gj-YBi!qkZ&~3+GI?Q@5>(c)ii)ok9*kSvkgo>i|bdq_wNXzAamjMegs3?I!)+B)3&TubS!eXo1b-?=b!L?lYz zwcJy_$dKWDGBgi#*q0Z*1Jw0pkS1r z#(b~go3$Kyqe>)^g9k%0rwUt@`9E*GiltwOO^0IM+b3DX=^-2J=gDf zeL1#Wgv1+1LY3t{U;pij`MNfOn0bvi?=o`@XYMzp&swRLL_MQ_EucgnKAbEHCyrd$ zn}Z#nTk%@lh_0>5c1sUDG_)=)*!*)?qRCfw70WK^)T5^9GtjD++;=Khk!;4v`tEca zYgPKI^Yu3>yvFbR8GcZ5lbCu*AXiOVZ4CQ8*gVaeJftU6so*ZW=`G!)1)4ob^sVdg zxEGDkc0IwF`@WwW21|3J-%sUjRC26hB-RVqKuUs%5_024%L(runro9B~2D-z$nW05qJ7(=3{Bet=_&qX$=JZHk!sCc$EBGUzQp5)XbK54=>?ob#h~uYBCUM0gn})8F{ewlagvx`ltl%;_oVknUGP+vxW@T&F^?df#ExJQZC>g zcTxlWYRRO9Z}uaO%%f45;@`c)4su=KO&E=GEAQE_Sf5FhE43^7Rm6Z`#ylJPtfwz-+wCps9U-(L9a?T)a2mIWna& z_Xmi;Q_L_WFcM3$p6FAL`Ef&2s$UP3(XPI+zJ!+h_}1F;d|JDie356%<+sfh%qE2E z_tfHV5JS}CW@!S2?aJ2rt3arXaXsh&ZP9X;A0YI0ymylXhF!zB5h8r`H#a^B;&$(C z{z}fz{JZO@_@@!lT10PQ+QsBPMs+8TG*Y*yj072wr^EKmr$`^4W*BoH8AlXbFtPqV zTsesM%Ts2ol6b6-av5=$R&+@?9lEzUQMDSU>pgI#Kw09T3OhPUDN!9h?Pz)uZFjqA9(M%_>t;E2&6Edr!kx^HZAJcGTh5 zYI_OUaU15+WsSwXq%dJBk6L;?7`L=mu5C(hf}kvx^ss3byZOpO7{B>VZR9rY@7DKr zaIj$Jk@7*{7tlgnWe0xftv7xgr6uTj?`VHx@=`9P^tIm^g%iCu8e zYrK%h#riINRex95>O61S6YF~(+TIgaitdKabG2YOOjQ|!*Jf~yBd9ym<88{Fw7Wqm zron?FL%}aq1=qgvhwhSiUQ@4GU=~|*e{3qe$1o`a{|B0qC^1P@w{4VN1g=##yg=xK z>jk|t7DwqE#FXx;LHiZk$4VKy?@|ZRBLozJI`G3PsD&ADFcglWpU|vH-eofS0}ZX6 z0oq5$MUA7>Um}_0fU{M5x`)iT{jc!U{snfOgV2p>{do5Z?so+B(~d1?ubdIfjyiwQ zFOfKyhjp`gW@=4h*!Tnv2MrZZ_y^RKrlc*X#BZqJUYj*vMr=GaT`8sBK}~uYYTzXe zl24?szfk&JJDfpn@z%o*7r+M0tdvlDtDZ|g!r{9}vQs9l|6b>(-m5Qcwb$8IvE*jXT03}PSNxGoKztWa%5H|SY!n^^6J z+%X&;jD)y%-@T&ZDt8F3J%mT7Mx6c03${?x$+|n>U#*=W-wb5A>gJwuMzCHX_N568 zSMBT+_M+FwjCxzH@p+I#1m(H~p^xz)0g=H!J%w3(Qs45Nc{~KK0&(_F1bd0w{7*s|M*? z`C87OMHyaf=C3e!MNA}8_~SF=tSp!2f5;Is0`}UBp!4ZL75>rj6RbI~N*T?yt}?R{ zSP&X`cgZz}x#P%u&(^Css*$)Bp@aE2>#e2Gzhl6ljIz7rFG%jkzIVX@s?Db~S!m~b&ak_dDep^TBNBN#8FjfsrNVJ;ft)TRKA{o)KN(XLdJ%H(EN|)o>~GIma`xyZk|ab+(zzx z1ie4k`ysWecYDs);aX6pg0V;@>JZ1FRBf+%6)zY31xdduR8NrIY)k!%D=B;~O!n%6 zIZzyyV_H>rnj&-5wk4+L3XD|NEb&HwogZrn5O-(wYT|PMlHCZS)*REj`{9CC=a2B{ zJg-#i27CeUR0e|LSYHXA={VNJl(Z*w6hE))(5)rcKaSa66|ds`n_tWOw>04U$#rKf z$D?r|S${XvKP-y|u4i-~u1duex6Fmrq=XbUHe(<}-;ME)(yUW8OLzhPbmoU z*gX1+U}sYcVoZ#+C3;~^d5O@+AFAvX^#wF^6xpe1zFG^ws^6Qg`57Pbef;3bfW%2y zlk`zvWOadJc{pHe^$CB-uT5^*jp{Hg>Oi(DcGC+QsC!%lNO?OK5S3n{(5if(xIngR zoxC#mcJ3f*QBHt0t+T-S7{i7)C4}?53aQ#N0iYhjA6~s zZZ7N5F9L8?kNJ+QnWX(?--U;c-GRHxwCl->$s(WSYx5XwGpzytqgZX|k6Syj^jK^z-=@hA}g#OLHnyfM0JNE=%T9(Ns#2$2)ycMFf@*4E4 zb``7-w*c-rpI3c~xwjLMQTM$qI7s(}*wBMIPmb8P=qc1n{58{B$OUuE+xIG_=?gCG z5ZG|kdo@X<9Z6cr`m*5EIBDW*5Eq0p{o*vZV8CYhbZ1jGPe|Zf+NZuBDMVI|o}2Yv z)u^te{Aufsj}+IdfqW70>m57^8ufk3?q?}jQ)oyDTU0elot>O<<9(X$S(P|~pDsVN zZ^ANaS~$7{Qe>dMs_a2tTYTZ?Nv-YJMNp4Y4nM0Ww2KRiVP(rCP>d-%`9(S1ZQS9g zb?DJI<&gcyJ-A)vS=dfu+IsDAtjHl5D;m0Hd1%nn``nU8muCh)+zGe^_MOtx2_3up zTXswbe8w!)(ox^DxjLNlNVcoeDP?mW;tm;;U7EJUkFU)%VJv{iW;8D%giWHiM2T?M z>|2di)1TIyEvLzbABc5UD_y(5wo*;9!Ou9wZa@o%vmFO`ECy#O;V3}OeD;Q#1@GS;yw5*b*LdDgcA%IUd7Zsuf5LHr^SgH2-Ny>V zl$7)K^+j8Lz^J>!pF|#6uC8ypaN7C_UfNn!yyexjMy?Z!J-J2?*NdX!a$^I$WG#&w z{kUH`d0{F3I$azbyXIo1oP$BVrm;aB&P&H1lK>TwnOr|=k&`o278U0x_eBh!T$huMqOOE&QbzW!B5aOWgxl8>ZlVlle|t8fGJ zxu=tsU|r(XD{47Yo0BQNxr?P@veAat>sLzIb|ZOlwCFT@q=&)L)8lhTJn304Bq84Z zANc%21+WzqSo4+3HlkuycW)|8Qkb6>O_j8M6ap*$haeRv z;jbydnVg|e7wXp3t95+%pbECJTS8oonvLgRj75AHoX!5)mFdN5eI+z^K~(pOaK~c| zBb_H!FEn`D^lE&zk9r{S(MsTK@P48)*{Up*!BWHpD{)u)awpxOf4~!DuF~o(Qf6Zc zMpiY4tB%M(RqQi0>>f+t3cc?YFW*=DJM_G!5hk^HYAr#))LmjHZSKHSxrTWl(b28s z`4s$u%GatL(bpfJwz@Yz)Y-WBtl$x2ebv&%clrH_TpJHcw7<~Oha2&?Pnv(nb#y+3 z*V#KmOdyzC<5jtT+k3_<_Cbo}Jr=Cg<97i(Q~vy$ zWj7w(Chrr>r*^Q=Stu6IckEOmv*GiM{HX^mPsD4x?kCFpH={*t{_vKJ~UahS)*KjE8Oc zrLgf8B@Gp96GWo5Bh~tWWNfHTD=v;A#%)@x=gjH&w5BvaS>9osU;k=OY@Vwa5CW{>mI4?GLSr#8XYi#Fa>s zRCms`w64Iib}861#%_Y)+^ud*(p#mbp42C~?Kd?{2G{j~2AlJJlAAm497L~-()PLg+ zV299D5O6eCQP9fg%3CO3!FyG1Ja;*2yQM@fE2S%-OO)v1&1mJ(Vggbn+{>3$j`lc? zMbVZO%HA=XJ7pS8`-`Zc0Z5YJPQLd({bNvf{5VMKX!n}jau@J~pt_*;;R!pF8Q%^b zOFm^L5w9TFjaQBE`^xAu4~T+QwbF!W;j<*xtkNsKzs8QTWT2FU1mZS~wT%5=-W^gm3QyEq;A{j9)V*h1A5r~8q`W$6YprC#x$CT}^~2Y0g8ev?j=#0I z-gIelJ7_uBsG0>7Y6zi+bW|b)JpDgmuf<=;G^%dXm-ZEUsd>tMAhGwFGswCH`({IT zJzz%-AhAwA`NggyXkpG+4r+KVQ;Q~*P`UL>>>}0?V(`)Y#eLQJ_poE#4>3N?4yNyZ zdwU6lgz@wwRHu%Ha1-=QNnF+=kQwPvoTy!|gx6~o*TsbQpgrQ5;IC?(67PJ=GYsE_ z1HE#`PIzOneqRP{OdCvJ6WQ4MKlpB&s2h!@DwcEJtGj49EL~I>I82Oz`3-n(yh>-T z{K8j#KB|^V@cO-;hO?^wJ>8j4XWp|#-5#oa>IRy{cyl~a zso#t;Nw9q4KgM?W9-IFVcDR&gEiw~4#Af%cXnb=55i~Dx3qqcZW?6}|T=?T68Y^tM z@jD3kbaz@W#`IiDZ{^US>25cyPB{*Z$uOb|K2!N~`Sl<6tC=DduX;nmxvYL>Z7add zCLJ}B-LJoW;BnYHT1_d>%%p$$sY}Wih`&=S{5HqEx5}6F zC%3F@S>MNUbBnwdgi(G;i=SN?9iDijQDJ9X5TdIWzh9v4Nb{`cPSwg?WAcu|_ejxg6xYO}v$snQT<~K4%zrQum|Gwf$fme8)+dQv@5c zXps&>URR+bvzO-jy9XkMb7tc!o)y52{zy4|GlV#uUeLT1Y^Oh_vibD8S)JP{z;AZn z?6Y9lx6Mp2X`;NT^M$>j3N%w=!w=@hAHG4Ps=(Y*Rp8&;111(@LBuj@p)eu$rci32 zZMLPx!us0aR1Kv$vor?iUWryA$E-;o@b^~8AZ2c_EuVCE>79RNw#VTP@F{;VlE*kJ z**&Mv(Yt$RlIrhfv8@>7&bTSJaV}X4>R^>4DRhStt0-z4Z(I`Sb*@BS2lqAbGtN;I zWe6qJ#idNZ9YzziI9_bKyvXR@2HY@@22E5n>mVt0dkwc+`F^rUH0o%iJB1AQ$?~1L z5oYw~R=}Npmn1TKk9wF1^v8uR!W#{@_NziV!xilDVt2!u!R3&ZRRM8lgXZT6#*yqT z44x>&EmtEUZpz%SDet?(_>ojox$s|8J>NrKb4lrzM7RBV9JCQ5y`N$VZC$&9 z+wZ_ig$A(fhT^R`ZgcR}y~7VGa{%dfr8_IgPyi7l46XByhpEJ5MifU9h2iV(6N7(W zjH+V4y^vAQA0|)gOV!`z;m)!;d2-T@;IwjrvD>&XC?T9IJQeVJjICN|a1Zz+>)~#s z;@3rg7^l{Rj2ih2=L|9D^o@Fn|7Z;g3Rx}mA}&b*VbM-7YMk{bX4y$9HuT4E^D*F_ zl=T%iEOxuEy=RR#idnbE`Ez>(4Wn)DzEKxY0JvG+CB|9D!7XjQHl9X;BP$vST4jex z_1D00yfu1}aX(6x<&xs~lwgtZKK%}EG(kR|fw<%QP7;s&hIC%sYmxTAa?1?3CD)Jd z`7R~2V!`t(O`8=L-Jw3`Xhw_U^Cf#5Jvrm*!~Hp3zFY{HQhY$^!w2lDvV&J%*q`al zQvX<$BhgJVO87l?6K&WjC4NPimzR|Xb>7H;Rrm)K)lpNc2vI-lb#=%I_uwh^51uo( zx$jFEyI{Ss^h&3?I?*JGT;+2j$DMNUi(h1uc)<&SNtJ0ehergc3a+cZgV?a0usHWP z=*j>Q4ebMMOjw@iluC(`8)zOuu%9p_rl1Ual!O8V!u5ae8d;I>$}r)DAWzuxx5VDX z1PghmMLGC`zqp@rbPAHQOpH^kCXX%PTauo_RDPJLe(uKO*Iomf)(GgGRpRBWt^Y!| zRfO#m$9ZBDmzw#$;~rj|j)yK5*d@6{y??V};s4{m>ZS1uTZmHnQY(Rsn6kKK(gW-= zebyI&v1&Xi$wdDyuUw4fy|iH;)hgM>783|ruWpZV3!AQ-laA>bcnHi^BG@o@XFnC# zDY&yd5UGvTJ;a+ZZ%AZf8%tzzQ9?kfO7SoYyy26hv)u7oBdzVq`!#m%;3an|PiQfY zqdxwaoE4i$k!lS{CsoU%o8&QkXV)W5WnxpI0hcCEa?kpbxO%5jG87!D2F~=41Mfx< zX#dgHQd5B|^&Vk7)aFWJUBq1krMOH(@=~T=-HN0;UK7kH$W;f#%g1s(>MuT_BVeG* z!I@J%4akzXWTmk@_2ymHy#g?PbLfy1*1=)l3^&%Q z^x~cqwZ=H#guB@eiPWk@*a-cQ|7oS;&__@tol8~6h-q-T9-)3>{&vXSOG(H}1~CtW zLgWsD_KX2VC1yTblcS+0K@6ekF*C_gw^I%9T1Q*S%@vMe)rh%qVcw5&yQn*Z`n4V2 zvAzA*vaF0zSr2vlcCleGmNqHPgIcw1f1T5~`u~=*SK#_9K5ahz_{cf?CnNX$#^R-~ zTWusUcy91fWln;K(#hF!a0{m{c8%{_1K92R?~j#E`+yuk(;IS^USRY+>bjqu?e>m{ zk*I$gnkrUgnILSv7p)opWiDdXi(#Rs=>$C_l{5=6m3;=_rU0pX|zgWQYukp#&dwsxfN{gjDZh76eb~Nhv zc$$fwLxTd1e2su}36^(QbyF;*Sa9bJBL$!8eDr=;$o>!|rRq6WdX6gFD8cGABF}q5 z`-Voq>RrPO?1^&?1*eeH_*iYqnn8GxjIPX_s)rZv(^!cU&E}^CuiV8Kw%GwJ&I~-x zwq8c6cy8gU%<*GELzH_L-lcUcc%2n^eeZ|Hhf5puhrfrlIiSsO z*2QkileA2bd&aF?itD!S44^Z71^+;Om?<~YWBdu`PO~gpOj{q|-FDi~!g#d|6@_p- zi7Aa0e`FY+yt0n_AqGq~`i*NSNh7fe>I9Ow8f-no-1*PNA&Ni!viQT7pMwFYT@CDt zi+6i?^uQIEt+C1H%qw&_|1dtmhh(Mrc?2PBd^dh>as2b35{485$YzO6pvFc+INF_7 zKH^t?YAXGOoSrF`eHFm`R<_vQodjP0nwy%jye{cjFSTs-?-RQ^x}fs`gY_X7We6F& zNC#fWj^<`F!HE7>v>$gh@Gj;@+Z>?GC4S)47HdhpO8P+BDy~4tt+&t`@m#&%;+7Wm zA*_<+wwp%N@JOK`7dk8MSdnjZ{)rtghKH;Px&DgM)}WtOEP$PBIjAADDK79@Gj&^} zIgS#U(5=Y2^IUG9N)n|L5YGFKu)QVY`57`KH8PT{GTEAWMXaZDdALfU(r3|MzjMW* zH`EJ3RSFJX_^n}yn0a`nj3Yi|;m{MMUBTQCj**;o!lO7vX-*Dg=-{GH8A}^?xmG-+K(Ze<2i7sa6h!@ z!}R(a&_Z?s7}aoWqC?j?)8|&&&qu~5pZtll ze}@=6UVl_MC*RO3%X)leN^=n#^Dube7(u@xH*ZK2EZ(pXcqKqhr>R&+wW z=WtvGFW#Q?I{nsRF8I$0^`@E&e9C)#>4Q4;{6Q-21i!;iY({D!4g4bRTC-8^W8Rn+ zmDvfYuAd;g`wN4AV)hNvFMp6e+Y%S>K43jy9g*}te`S6=eEnVRlA#8ArmGPPfn{j) z_ipzBkX~R~o~fkinVHgmGk_UT^9X8jA*2?pmm5Q!Z{6Q#bGaIhSXsP0u(2Jn@gchR z@NLyea?bPFUI12*HaFjYa}51AZ#1%)vtCi4y29wsQ%#4yi+`NE9IeP)vl(;6gZrV$ z#|^6wfCz>m+seyOUx;k#wB5{2?B7QkpLo(%O)3c!gQ@QWZ?jYdoMX2F5@Nx>9W%B$ zXEGX39Rg27vRdl`xCi&wwFyJV^9d3A9)h)l>+8&?fn7(Ip*5^#)I%#n)s+i5VPTy0 zr%_&P>%3>&vcA;?gqiR&$C>{#JS14wE){Q08&F4@E!hGcZ2ou5ZTx9ci(vl9s9NFo z|24Ld{^v~3j+`&#{QnK3>d6sL@FWy#i?#j#UL!!^w|YdK!q_kY-}HZv(390>q)qwf SvuP)oUwTh~kBcALg#AC#S1vjL literal 0 HcmV?d00001 diff --git a/docs/images/quick_1.png b/docs/images/quick_1.png index 6b1f4450c722e781519428f410bd4602e02a8556..71f00c6a907c4797ddf2c50438e7e62fdbb7cb08 100644 GIT binary patch literal 174058 zcmeFZ2UJwc)-Jlp2$FL~f@F{kl0-5{mYf76Hcbu1X05qq&6+i9-Tt^;14uPhHBnX{LnvWS|y%H~h|<1LWh-+jUG1 z4J9QTJp)};jVCI9HVTu{*3;ANkADWc`+6IwJ!UmEGiSxy1TfLwo{D;U zPyXQf*WW++|Lbrz`)AvMDSI4CWmEcpgYG(EUbt z=ilczN&tYw6aX5-J#D>h|G^J^#5lwDaTFE#^D_ov69C|3-roK)0RTK60JxdEy}fF> zy}c<$^IrgfuV=Sk0Wy3nZoG$B7;FG083q;^#%(Xag64yR@hAPo7y}av8wVE;{|*5m zxL*d7w;*!#`PqlUR4UJ7-np?Vidi(kZ28X^)PEF6u z&i$BQfUR%*+}zsU+1*2&{Q7-*c899$eU;o{@|LHGpte-gnz>F%FI{0EWzi*C_O zFwi`(v9a;czodkB2uc4R)9o_ac=_Hg0C%x4(8`2G1}Fg6N+06*4oH#-xGS01R&`ot z1Oys_8*6GZpPE%si}J*Yc#e{iT424yN;l*&laMNJrJFW6x&^|+o-52V>qbe?=Oug4 zB=>#CL^SY~w=U8Z87qBxoT91+R`8dQl_Xwr?BwqdzvhNJMT#xzQbCAOo0abOzcjW^ zc~EK3=KHwqrK$1V+o@O#WSYWx`O*d_6Pfs;cZ2=qD~CXd^jg>b!a8@Muvh`3rtz!0 zy4dzEwncIN^#)+q+`8RF{gk0Cp=!==f!$!j4t* zEigI!@Org+;ub)1ZePEXue=594#!ZLX0x>Uu)&?~TY!vZYBs{hiDlgQ=9x1KVk=+K z;`HR;@)i0kSqZp5g#TUs|L+5Z+wp(t^W`nDmUU29dacx1(>ixd{p~w-=K4}%E&epm zQ-3b*Nnv&d4T1z6k+%qih)V<4PO|!x!P=brpuWUGqtr^pEL;1pqbtNDa><%k=RMh$ z#nzkrdzi$__t!FT)aSYE(P+-s#&+LDt-e!lDTp)U z4~z0oq^*ml*B_1%fVxA)AzJHl>#o+EbG{@87uOykVhf?!CyFhPCi?H@urs{<5Qpue zrBOG^>z~=>@eIDX9sBWfj^X(4L$2;w!(zL1Buy}#JTx1rldg0o1J?v%1hC7Ns zpNjwXI3x>!9qO8XZkuX=_*q;peXPU;V%<4LTHN)h;xLC(Oy7B{(OrD&n8S8 zpNEp7Ld)~_~MSphKigJ4fp$|Pvn^zqSlu=$iAt)(H^qX_qOK2p!-_= zmgHavPI4B6ltF+7Sm;sAR>&(uI7u>ta&4@iMWxK>WEJ>evp#TTzOg{qGPVhO$+F^k z-qRdQRQ(V z{x%`paxVggO1pIn6USBqcV%un6Jtkr`X^BfpKuaiPGEIuyN>lZ9|{&nSd_jGh+6e{ zv-k0nyOdS0rwD{@-)R3F)MXxUaK5G+(OIw4PSayUv8zBL^gAEVwGLu$6U!XmN5lE;|IY_^i89@oXaX9cu}byxmpIH=rw>STbV+<=al0vS5W?Dv(&*u0!6}s zCtR;r+w@*2J#)HW5{nEp{@Fu>@R94RX1}P<(wMPL(r~QH7K_x+&6dER)*-E74_}Vm zU#Q!^w7hvQD_4R{*FZKD??o9mHp5GLXgQOlt;HeRvmO&Y*zD2rhy9gS7tT5JQZF0qP?KEUd0RA{`URHyb|f| zK~Bz6BVmd*a08nctHi#Y1dywB_?XtSy@_kgPa$`8DtG z>)~GiL!vEc7J7&~Ksd!h(GLY`x>gOD;j8#l5V|*~{E!naaC=fg@auz$|KNvrIVjM9-{)>n8CQcrK@eqB4m; zsHgQkq%cMJt;BxnfhGFrA*d?*>is2~NMpGcP6{g;YdE$@%gNhbbx^9u@7I`%m!gWe zJ_WHvoHb^dOjTCC7V;9^c2zCw;)nFRhPtg1SB}jda_$nUS2It0KZ+dN zHOn3!%)y7xyr-|7;D%?fkoRB~V|T2EH%Le!-g`2MkAsgJRhZ+;2Af0zdtar#!oRYA zY9#XTy36R3wY2Lbs^J#6S3x*ogMJXkw(BE4mft1o!cFV+dVll!7I5-d$m_rAUAY5a zz6E;ZlpZ6KxuV5mi>L3mr5iU2R?5B*6lH3dx_J9Ek)VGbTOEI~50Z1{c9R{-yTd|S zfHZ>do5B67;f4K|Hc?L0Eg!RLL1*U+?}FSS6b|u7yL>X8jQGj!Xdf9REt-BgySKL4 zKGwg-cZrM0I?%u3Sw(tVtdc-<#?;#2$BDASg;q)>$joD-=CnD38U6$JNzA?)B5C>fG3eC8M2N58!Cq1cq>JGodU z{6`GOgIQ+p3ITBV3*nLlDu0JdymmQ_?1SF_5^g2)WY+0 zD_MBVJfEZaai`iJgM0po=VjAPe3QbZnUl>ss!Ij5zPH^1>CrQt2mf^G5_I7|t!a=k zsJwa-^QZV<5sx9o+60@jGy?DFhvt9Ezh_KaNX`v7R?m}E zxWD$rHmXaS4sScxurHsXUXg6J4Fo^?s$;2k7URCYVh>Jk3W1N#!fM9hfz<(strvK4 zZ7iRV{@X@h#y6(#J;+%!Qx;YFl0fD2b!hD}LM;ex#nSa~?3FALEFaf-3}!=`OT3~L zDCidzavY|ye;yx~ms5P;=L-*0pg(R&i7rD}nP5k5-H=i2!& z%sGsFIdtq6pgFCMdFbhUh+;!J?)7Q%u!1C9%RSibCkQb1GcUtQI_rhVAYpyf3Jan1 z(}#W_xkHwG7ZF|R->@vGLl*m&Q583nO|=-XtPgzl*Viyi-y8G&o)&Ocg$WuaaO_*k8b% zL;Yit5fN=_hN15ZjjOMI1tgO}n{7fHHg_RoS0*U3oXrkYZ%2-H+d@@`(Q=3E;OMF4 z;gYmv*@1Q_be~xs5%v-#GUT)FLF@W3FCmXDTBkmQ-Ka5>mfew{(PTmE1)s*iv7=2+ znKeW@d!Ubmc@Y`h`w+Uq(>hVVthQdX6&p|<_$6Iu5~WnjW5Y&et?rH)9!&9wrSz%E zrhFxWWt}(|mdpgv>b91d77r2aLERre_?;Vagwr?7dHi;}`%`8|Xc?-5vZG<8A{T+M=xl#0l?M_onwU`j? z;%-wl7-o7%^h&=et>=4^$uH)cl(ks4tY@+puFhBoym9Lg@UyfnSg+UPqUB^2BRY;7 z9H?(7r==^sNB5vuCz)K|N(n4P^X$;>3;#)*xvyL8$jjVe9uDelf3HEQLHHgdM{x>@ z^=TDSXf(Szz)aT>qRDranT;i4s^-U-(jRL2R_7KH%%9;QCB?4B^rJa06)uLA zGa0}2$FK*z(^Pn))C`wIQdF#~N2{YlA0VEt;zG3qTT7RB=vSkl*lEt$ZWOdI39KhX z)TK$U0w>$bBXqBWQ%Hf{I{(YFl8yXG9?rAGbtc_j$?T?x2C&(b-kYTlt`8xN&-3t; zhv`#g962!5u@c7d7N5A-#furq7_IRbjd9tP2DZI2el(1XC)17O{PS-Kl#5r z;IADVv^OA4!u;vj+*+MrEshXbf#j>q2nzg8j+GeSF~{uvE&y2^9O2-v`$1t%!<=Yxr+G2 zA8vvtebjLex!5O9i`d+3>K^e9qXtdVn(ucfgU@mx7>SQO8o4sXZib_8t`jPB*4~Zd z&Ac8+B}+A38}zSk?ot?}kLb53JE+|12sd9Eq8F>dBgbC~H^U0Y0?RsK$+=|N?4!r2 z{jytN!s8JQD7pce17E-D)rJu&P`ae=$Mg}USwEg*m^&8AWblV2nb0xyDXbgpC&Tyq z_+V&PWf4!agWB3U;HG3SjL%(qxs&$1dMrOWE3CHUTJ-ou0fM^`9O7<^DJSD`wljlW z!Vzei&rlPwD!80~FpjsoOm$&}P+AX*FPxF*Mb;j#iyueHDy=12GiVbk#=+PcTN=rE zqk~nkmjX+*wsYHchr-36A?reM^rJP1!rlkSF*Eo+gl)LzGO{YyMSj@tOLM(Rrlc`; z%n5@x55}8_tHu}T$B@!wYh*P}o*l_&iHM4qOgPnc@C%E|lIJ0}(tKZW%BJZ!uKMX- zqNnDEBoS#2$1bIY7UissdL2dZ6pSZjcW0ZftMW=~Q3e@0 zVJ{mX{5%oy*>7+yp^TlqJroS(M84cOj+qO?sSrOVTkTz8pM>gr1dF%zOFsws@IM%N zM+!1$dN-4?QDL!%ZT87Zq)ysy6tGvBPJ zzSm5OxAb(--v0;jZ2y51-uNWaejpDYQN5r5Bc3}DAFD~J3yIs}OXknZX=`+I!lTD4rh7qrta8JD@cp=P#mBX$L9D?YK2WavQ~rN_)=IYZW$ z?KkEk-2DidXtX32a6i;3eXwHaOB+hSgW=td^pcXs6P} zvC`eWL?19?>D1TSZQHa5TC8qNMd71TRgMaAHg}8WbR*Xh{z;dExs{>aO=nyChXj33 ztkpxVdN2t#SP^k2#~4VEBD`cEpK4d0sV1p=4n=mz{5Um%CiUoec<><9$zQfkm|xcb zG+0k$V+p6~h^^WQ6`8tLvaf)&tapPs-HfHp8U!>A<>zf^FoXS5-yLB{R6ntOgLRa> zeRypI8}1~tf@n_ZM}ZytIrG+M7{xyKBNv`y9Ua}p$HR`mU+=IS7r&3di-D@FcamQ} zZXXv1GxQm__%FCF;nZ0;S5fnU0&3PY=TcKvY=<4X9!t!jo%kM!7Mo;wo;`i*RZDg8 zTY%l6Z>dhzsDi(EN&iO`+_co~IV(Q#I1Uip->@!#)nlCRfnY8H6PEf2C+R5B$7EFY4SY}?QwPf*TnVHJCqjU`@%Y98whAtKub z4Uz6uz9W2B_XI0vsnU36xsR&vkgKzAb|{ULm0wA@W=AzucNB8vpB*18##}#gs0RsL zM%a9*b7A|D8jJeQ5{i46XT1?#kMVmo<~n#S3~$NY98%XyADSqKV8d=}iCZojzJL%9)pajNqBO63SFTr@Z%mKLT6@- zhPEV8%1~uypHCx!QilAKfR42q_@qN5kgEt9U>o?c^I!sIL7OSOp|nm&i|KCDDA>;Y zsT1UnQg{%Vu_}Avl>d4wT)uy%g9j=y?iVPlP=#z&n`%&x<6E#?UF`8JR=FGAd&Ed> zjO{hZn$)ncWNEQrF%%HM+k?$(_0yjq=97ro8#ut227=uJbbBqUFN%>l2$mjev8j() zPiEPh^Odvv%(ha&;P)10oLS^i8D!8x{KfQlP4_}85mv`9#65^XK~4$V^&x_z^A0oM z?F=vMyRu!|rP4;vJ|@7wGCtSC&5weD)@}j%MXkWK8}DI@2;6!0c}q3V(v<{z`v|Z8 zpi?T`7aPY|kaE4&$u{A2CI#d6&tnU$^(}FMY+YR|rM*dz?M^1pX7E*rMSmVCx`iO+ zSl`7Fryfz41!#OqUh7Qq0-?5s@uV2=A6N82pZb3?GZ#?rS$QMulCM^IJK=z**~$(F~ckx=o^EvAe8 zpmicuTs3pu%MMix4!cVufA+kCZ7lNQy*4+ zs@oe}1N)h*P(P&0VCuF0u*CS#L~_o3qZ>DMD+vNNHTN>(ALeKst)~vOD4O z?l@k9v$YIdH}?3bzLN;~Qg_CKKWwfCge z!PQP9VClUGoxN~>O?kO$+%~hdsohp<3DU<*H!20fVuRM4hch1Hjx|=Aqk-S4#%u8sUnYtibTOKR3NIsu3qMC@cyY?s91XA&LtANRRUJgQy{&1~h!rkMj-gKK( zYwIv+QY>OoWPn3})Fl=?Hq~=Dh@9$@Xe=0)~Qb%N0jFT4_^fbK9*bt zr#);Wu}=aO%v>uKIMYLB8V)rVwQC%eDVDyc^qMQZVpxhCrii5b`MWv=#b&wB%!J_T z(`7_@44Av;`n|W9Umhx2`oaHw)v4AviEag4S!r)jqVDyhpj{ju=r}MLvBqsf-V^$&i``Rw zlUmW}Qp5j9)GG0rtn1?n>HI^s?c4*|)qFX@-7N5&Q$hplWD)U*-NQG^oZ zYx#GI+wW?6a%st_bWBh@qlj(_#Kq%S)5F^l>)?h@(_hO|}KiBIIh{c}v52O!`w&8*|spg{nvhc;ctY6&rTu_s4j)S?np4 zxu37^A**2>9x?eczTzd7{M?1* zRX{Qo#CH(t@*O{z(!2r~r3Cs-w|Wnmfno&lJ#a zE!fP=%*b)v>~_B=R*jT6dR(aZUx)C4Q2f63vR%kr|FI@xz8+bE`6g@mHlZ@M(ZwGBv!A?&J=Z(QaAOzKAQmPIW- zY7vK=4c}Lw2Y!uFgmtMx{d_)YLWy8j@~hejY4WSvmO6t{OZ{|gPsd8W<2}i1_`P~? zc;$c$*tlQL@)D~1tb+x?@k#rO8R4Q;2AxB~=etRG_ZXxBSnPTSITBQXT!=!c(C+Ki@;Kwum=AQO=gMen{PwP*i8CIMDH$A-HPAZgoc()UOErUK}A%JA*!NX#H-vZ>jPBagpoXP6%wGr_p{#H+^f%&o?%ONGoel$?OUK@$CEL77y0}G|7-Ul z-E~)4!I!*QeewFlyYUmLai1^AEZ?+H-S701r1=>d{S_K?;+kmm;nP%bU#r={!NIvj zaf?9mdeWSMg}_E)O|FMZ)nZ9+pW&&{GJxy!NriE04Q{kmfC2p9yvy8 z%4**Nios*o`03ZKMYA>Kt2|nZyGN9#%m(EbmeOgc|X4$7FPbTL#(V z0Jkxp5GT^K=!{*+Y$v(|ap_DMf8U*0?70)+Oig8yF3cz$^O?bO!}Y~Qh)R&?`#WZ0 z5O}pzF_$(2f>G;h)TC=pMp&us|LLaU1~giF(|B>v#lL6zELd)c8DzOnv%nRSXj7Btiu=$u*v!LcGL-k)8kvk}?W=`suV*$1wpYpuFUEdUUpIxt%djVL8?h&1 za2%=tHI53RGWl=*-dSyix6qmr|f}v)VO#nn0NFSiu3HwR@Vnz&XG812o#~BS;XA;uTvR)IhhXXTAc309$@=*OtlJRW$$Ijp%Pqx%@RjgL zmJ0&3S>L~F_in`k)*^+xIZPgZEea#rjn!-Z$#%hlMgYvj6*?RK9A_=i;7X8etFuU5E zp=rU&;`&VR)T4vN_j$@ceikgJypKrzHA5LpYd%ZUn0xFWqJf||o46^K=b3l*mlr`) zOsT;ZcN(-rK^|bc{jfXtWimmgAB5PPHU-`>d!=biA_fkpt_-i0UTQQUB$GIvPk0>$RcVY+78*d6dvk7F2wop*spl`Xv_mScBOXhcvG;d zmcD)-6V{XzURRlX$yrny>8?o!P=D_G!Xn_+D%FpR3+my_6UqUz8AMZBpqBA6?-1t2{&}P1^UaUcRu1_eSq>HtzeZpOddUX z#1r+Es*3>`J}xDTU1Gu0h2S))nW(23QnC?hN>-Pa)`tUdsDzqNiXhY$A z7WAbyMH^^qN?KFm-a%Z%Et&=;(64+5m(C|g@^!6Im+!*1roD$6=bo`!cXJ42%AZ70Y7!-nPwHh+W><1wH6rz zX|W;R9jg+=@03VoC{R<{2Ct_6*!mcT1-Y*+qd)Nzj>?;*?>+Vlb_d&?R8?`Xw$&v{CwgG}QL{7l!u!nWi+S&K9bA?aPSg!W0N`2tMPer4O{nQ5Cy+PI%|>^q|1_DC!CM9I|Cp~ z;?vt%P;6=Hlz709BW1mE2U@gBDFR-lmk=pkr14I`YVek#okZWRFcJuRS14KLUKoM1 z1ie^Ruh^jjh^kdnRb9~DNMQQ(v}_HY94S!g#t*Ky;I6E1Uf7%DXpA*jfW=d1auUAs zwiC9vv4)IIM){=d;KMA^XBI#RVI|mMGn^{Bb?7z z$~N{Rr0}VWg-;5={{xF_dJQA}$J2T}41EpqK}0LXn;g%9s>wDKZV@`Y4p!Mgdo5Lm zV1RT?bzeSzxFMXf%K47ZtG(mb#ft!U>CL+Ir8yt8!q8{+ptCggA09As&sH0&R6`5A8*TTbh2RV|I-3e+x-+RHNNL!&T+GR~V;i~}^ zFWfPveyDuaHY_FZA!+$Mo+#`X2dU8g(04&CvR+GotsnNbC}MI& ziaqtsv&^S}Fndz4*JKVlkOdL#>fb?XDb`GA@8i9jw@~re`aDW_;o4f0p7499>O&mx zA=@#Bx=^r&=_5~=7r7GW*F@zpN|m={`3!Mo{=9d-@~iKGFcknKJqtMtoeJzwPJ9cS z;j|&fifM^uC2}>5_|6!6+`e-Gv7HAtQevZ5V|_mnx7*5JoS^9tMAl+t9sa}wGT;uW zEY}v*{+zwH-apCb;!*Ugidkxg-o>9k_$MpoNDWm0<2!C9puAoWk7S)CdVwUSLF#lX zxKh$2Nr=_j)N%+Us*>Bq@#C@PCk@_OJ{*pWew`pNyS5d(PPC3i>`|}u&EfYbvm764 z&UYt4h@A&`nDwOv#D=}r*l^wW?fMpk^$XjIk!KoB!NaCi^lY9wJ#1;stQ1LWLB`9Y z=|$Gd^Q$XnJ#w?}M$c-8=<8F&;`9@qsOr^{f2{CCd^|;>!@N~hZ4Dfuhr#vRi#qVt`6tE%FFu^Hb=Ny{mB-w?Y6nPG;( z1)ldAN*pmgIHt@6b9}I4;0Mz2h^PL>#3*|Od!DLt=6X)ApkHCL#>tI1XSrsf^%FPn zi8|Z(ZAtr5Cbj3H_;qn`I;CrrOWY?qjs@Iv-cPMbI+S}oaA3|n6(ouI*!Hx)IZ{DtSLB8yfXGAz=FDS(Mwpv;D4cg-@@yG^8D>a}4 zqHCvA=uyl^cWn%B2v)PUJ$_)Jh|f+R*qD6q0%OpI%l*4kUT$EStT;@F5F)AjQ1?}R zyf*z{jStbgJD4=G)V$%C15CQ&<7iuy)VxN5iOe^wpM|Qy4i%nG^W#gL(-_xi^Q-8n z4*2mu2&9!|4w_RebIEqEa}dlaUr-5gk-$zFqaS8ii;!f;B*w>n?pHpohhh2Dh6u&n zwXTYI%}h6CKbISyb6Wq(GE0{F*W;rlHJrLMnK_UO;1yKTlN`OU(o7QDkbHNyX`#K@ zIO!x<{rMN;Ctx;JgL%uljEIF6P^GAL$Cvun_WG#x;5&C4CpG9>yV%)PttD7r+a-Kc z?^;mb(Y8jL@Hpqp=lH$hbQZ}^pY@zl2~YyY${oz(T(b(WdpX*T=|46pd?tC8F_=tQWbQRNVGiMk)>o4F_zdAc3nPre zPX~VzoD*ddh_kr z%m<%|AR>@WI><+ltCk7kA-*!M!Sp|2N_JLvF8XK*HJz4DhZEiYwSnE6ftW*m0` z$?fD{v6=tv9RoKk=yYf6u%Vv4)ic4RTVU5AH`LQ)XW*KGbX37v&U7(@2yM7d$5yZt zLTbmx<9g83114IVb@s~_e`UW#k3XM5^}iAJW%;^#ZRK&%GEkv#-V=uKJ2_<0NzCSB z7@3u`y#+cLZh`p1|A0UANH|w`NSdBTqmUjb!Q_9Tc#Jv~`@p~0`2HK268#t4zPe4A zuDI9ilia}3&0zi)qQ-KJ>;L7KY#Mfq_5;>;{sa8XAZ9F?QxSxAaG;Xt)Gf9DfK|fh zKh(4>+t0FG!8(yGvj0N-xFb}FTl>tPZ-G^G-}FuVNBrz>k$nHYOaD;8zZLJV=j!ig z>7SD*6UycUIV(AEF&Ce!N?BqTgQ|iT^kEf!;9x zCtmH`&Mk0F{tKnUVtfnK_;23=TKxNW(Yrc)ZT|>&Z$H91aQRI&{>&H`9lK^m!%iE) zu~NkssyLJez}G^w$Nl&5?;iMn-vd~;)BjR%UJwWrKO9pjz!tcjJHwtxS=U@6Mj*w{L;i%=mIVH7WwVW!U(W8tgbmdRn3s;pmh^UBxG95BZwV z*@(I^d8D7?jSwbLegbTxc2tfn^_jHi5}rl5kM4SUmf$UXiPKoMt)GRdP#TizTVyiR z&-uyGJVh`?zE0n5;YvN5pJQOX(s~^qAbs(JRBoR*J~9G5H~nLN`d^m+_JY*EXPiNQ z%{a0CTA8ZOa++qX7DUb=IIdZchU*@&g*vENd8T?$b!}jWT5kP|rtd$lf=(AgGxo$! z&JWRP{P)WdLZ~j0upKRR5?c87q5hQ-vZ?3NCaz8HN^Cf=2qf`&#au_$&@sXy_!YVG znk+`t<}=Sn>|d&_Yi>TA-UP4H^eaSl(7-*;lB(e?qm#knG~c3)^G=pVQ()_ac;7LP z_%B8wyViJD4|l>EhoKLU4t^z&tgf=PsluHxsQ%r?hmw(`jF7~$jQIJV8 zvJRywbU2KDRlrO%Du)A|%W{mpcX`$hV`4Bv7^k2UpjOS5#8=)qnvGP97PML0g|pah zuo@kDm?fig*Ndtjf7tcCa@qdDfbpH%v}%o5G{FgvIRlwX6Ydtujod#|g5y>O`Z*MX znDO2U5xpDp5Nkx3wM{B*-MdTuEYD&sIU}ap8tJemz_%uJXoz^ZR^2mHB_K;t3LyY{ ziOoBvJ&!ni@*tv;U6h$wM?)YANTbxlTSce;=JzWgZ8~w=6cFIwsqe$wZ>n!d#;$ZT z&MeBnowFBS?#r9*S1;)Xq=wG+P>?W3ChLv$o2IVg`OXcFEM^9z4P#KoSuVbD5($;7 z{+_7TPc5&L7mQCfu|e-z$Z}8A>dvq9r>_OpbcrS2ms^=3!xSg)skk^n4#5KNpRi$m zH)2=pP73$L0U#xZe28-t-F7#M0l_+c3p6Fi_+817e7n+}hgLTXh~@bo3tBBgE8M=0 zhQbFFl!p2B?qxcej{7;XV&wbSRjdgenEO2dLAPM0fy$aMO8mOg*_zep7zfdDvH^qB=9;Y3a zt^OCS9k5a1es4L2o}dNDOU234+z7nsO4ins0C(>DkC3Cdk48mbWM4}lKAxjUkyuxz zP>iWl=~EAUsQh%ByqKXE&)b8I_Gx<&^JqJrAKxG0Fya#H@`J)Q)Vf&kpsY8UP@*eC z2!%~Y{JLMuPS!}3_)TwRY5N`zMA=(9LHSefC*JC#@xmkQ2#l*l6-Vc$I#+YKs6ro_ zbb)CyO~ftWm5W7neWy>`57jY-YRSK${*B?^DH|$%?Sm{?JB|xljk=bsyh+xnyOPYH znXmk~xMiWj4xLo~?)_bt>v#3|-gVYZap%t_WI75RbNZb|e_hx2y@xDn*k!ussV8*T zTA;+;%*7Wf6hqrm@2QtT_oc((6WBNF8QLOdBhhN7#D&tBzXcH43b5i1bm(S$=oU!x z54(2km_@LJ%d&t!fl92zL{QhJY8-s&Pz{_ zIasO>aT|*pc@UUME%Lg?z+d~?pd(HHxUogyzTz!FB#!7rMd@CrqgNj{#KL;Cp&qxu z`}A9Y1HF|9ITGR|86i*8voRKv*@B9;2Y;A%$@ZaOs&_dv%5WstqMrExyjs=xTNV#R584R3xto8%h0ay$cm7}c z^fGAJV<=;gO*R^8MNS?t(l4siU98M zO0ZgV9oFP*i&j5z7+ipr-}GRw1tbwk?s(fM=o(Ep`T|MQJ*?+A+wC7F^XL}2x|5H&Ow z!fu>&n%X8xGRQUDe$;pP@|sd77IG>MNOh-~Y5SS9NF>9OS$N3!JgfMe%3cr?G$GWk`8>h0}ZR!0g%3J2$8nd*7 z^1BXrHR5CbMe|2dQvT85(XY{?L%w78_MrAIK$GEq=X~LS zH81kTn%0d|-(06hTeLyuGRH|_JQ@FkFP6%s4_g^GWKVZ(l;(!QYMQRonG8`q6k)q^ z?IA8UQ&5VSG83Mj>2fX}slCP4(8~Nq&k>jDull*Fi=cpVbnLt1;JVw z_RszyxkYEJ|6}?kH?~I6=dun4nIOK_p9xXMh7ljV@!8)~dIz_?A3OG8F~8=nN6ciVBV5B`$X1m;k`S*J zou9o)9|jw}#d+f;iMu`8-yLc@`fx6=0-^w0*ridRIug5LAYxdTrDx799$*v>$qn<~5$_EB^XUUX&=USc|irmYg8|n2@PaI^7q@_CXLuXri#5}CbP0B4p z<`}j%OfgFf*kztLsWS_mg6Zo>D{?6lNTsOaNzl$%h$k|8)vzVrR8+jH-7MD~y3J9a z=PYxcrXlbyEoizw-b#&k=peb3p(~%rPb;4h$!}_=VhhvC$3upk(!!E>;&1A^N|)XW zr*b4&%A6Zn!g2gr$SOifkx%y|X;2Jxd@!P59Wu_QsI9tK{=UJa;NZo#(7|f##*61Z ziQgrwlmG>(sLrbR5Y=&46E-5NiDUa;`3_V4eG|o&>yIn6Fz35Cl0YU_h~p$#eOr$r zNLKH|s7eQC#VEqoGN(9^f>Pe3UQ7&DvQepBwVHq>x!T5oos6chol%!6OUwk*72tav z@XTTq&vX^1=_V)<$K8YMnVt8O&?nSK-+Etc<%#HoiHU1Xl(_~3#dR^2wq$E4KPXi% zT4J7kEFjP?`)OgORb9y!`f%?JjWo^ahR}tLLB=%;*yETQ4heYUbgYDxkHRFJb z>PI)3p)3%eKAXGOHV$E*Gy>t}8d++SlNtrW+8$y=?jzpVKwFMwihdy-iqXpoXYSx; zZLu?c;kf^;kWhcqItfW<)X_b1l{2Hd05u`{K$YHR#Ubx6W5d?FDgZRr?}`e_)`p~k z9f#LhxpvNXP__gw3mIv^w5=!CwcZN8-N>~ii!fKUIW=8O-q7pWNd$du;fia+^MN1t zt7?ZmndT4ef}E=;d9d`x|BI<3AM?Cbi*{iz_X%tLOhwPVabyc`r2M6xfNB<7Tb z9GToFAy4tf(s@Cr5?Z%=Cc=mJIF;Y!oE8(fr@`{T^^_%nr8fCRr~tBm4sAuYTGx_k z7A5BH6r5?$&&JuoR;Hu{m%9qF2 zZenotSmV?3QlG3(?DIU+TteztMNGLb0^xM=CR8BwYfr9_LezDIx?vT_bR+m`^tMI0 z5Rvs>R0uWQ9GCZEtvisWp|{4EqG({;v4alJRf}nJWFCFA7eDd)3cv=*g(tYwPi;aS`XHy~o=hd}Vgy5swa+IPPYsqvT^ybiK)g{67&&)6HIsbB6N1iQk9XY1uzm)WV6abxJ~7D+@0B z-w&mb$n1gyoiizf5?B|39Edp$J8{S8!|w3;>8=zI(WkrbJXovjSGjcQ3aQBz4hpeI%!0lGGO5Q!@p}o586=e;We|hgdL>+Vq+HCAm?LO1Zqj zp-!X%Qe1re7kiE^89g1Yh8ZBe$N{iVkMz!yxme%#3)HCuoSG75CWgUuLFPD%7}cAV z1a*YWM3K8jj8$e1N)9^rcj#`{(v9ERl;+g_LXb5GkE zLvb0rgI9 zVo6}dnC~#8$fHIeTX%Gjdq`kzum6>A7fmO(<+5h5BMK_E@@sikw>Wu#+}r!Ex%stg zw-O`nQnWp~uqQ`xv>1rhR+$)#t?XR3PZXoo0;u5tDh{rwBb7T;>V@` zn%}~O38194cXooi`!Z4y7=JXshY@-PaQ?HTN|1t zccDg53{@bXlSy5h6$tjrE7G1Lqqhi(|6(EtH zBMC4^gv=vzY-=q!EcM!&#ceOeg}d7oS@L-d_r-bu3C)Ip=l{JE>3{n7m^b0{oeBg+ zye-ZKKWM4%-ZB^DJ6SlUB3-VFWTvcpNfsH|Q$0!n1~Z$PAfLQR-)368NF7NWm_$zb zxQ}UzS3smY@=LY+)sYiPK*2#x#7Hvbt&dOcIrGX%J%7&~lj*l!N~BBG8VB*m@i0Og zL)4ewXh^f3cN*jp&|=)V6Yo~d<7N{4q?G_vX{-6mL;$dvqw>+@dsV z$-pM()5f$T>zSkZq)*3H)U; zSX;_8P@sDCn>d>H5pA)RAah#)e{Dg0GQ_bdYAXZ9Z#o(PV3{q8gBdEEGAtz5CfH%2 zgG4R_4jffYGy4JN#Gw0aGR?s^sndcqnq7Cp;)yj{=2-M;rdZvp47=O_h1u=u-@{3+ zPb10=RwV6i*BgBSvc|gG|AtMe|DFAJ((~^>{a=3lFCwackW~NOp3{F#v51HlkHIIc ztK%(*^ldosPvYu{71Z%CXhPg?!Gk|irLj@*>_h9D#8)4GkTn;yGXnwzn$x$tn5^(m zdRWf;W1@}$eNS7rUaFQQ8>H7`V?F3*@O&6)Df_LtK$KBj2pujeI?AtjRpJdSe)2$4 z(xuN&l1o6Md+=j)T=E>*ox2;!u4EkrwFCpS!Ofl(p6MVnwEk!3Vu*~MkB^GDOgvjg zecd%GE1AjBUJBHo~KM;pC8U+vdlrkM5Ln1w0;>NHRfOGK=&xfe(Akkht{_vqqf&%XpSa4%GV|E^vTeb9-wW@~#zTidKiR~! zjSBj=PO`>y2$7noP66U{C}$^&yEV%PG&s_CfbQn5tm}f~eY9&yb4%I_UM`#B)MtuG zW9d=fuC}lP)hAc=h5X$$zt{#S#YlG(kuQR{-@9;NB9|bkHg%z`lgw-t|L6sL0sTv?V6Y#x|$x!b7PBNGVKEV(h{dt|UOK4}K$MvBi*2d~qntye*rA+i~(<_9; z@jkI~p%zaL;&dYqEuMW%J%JK=2zCcEOz9=)PV%=rJ(50#nY*q9mTHv6UZ)Y&xC0XT z#zcDT!+Zonf)@DJ`_#B`>aiiBzkNHYYq2G>%`>-VpvMM0-ux2J+hRN?*t=o#WLLUr z25|T649=Ya% z38fnZB`P-|ljM8lApru)PtV6QD<^!KcYKXks^Ui*J0HDI(^)ZYc$;bLOIVWc?`o9H zio^XS(ZLI&R;l!=E$!U0T7^DWQ`KGV$SQuFBt4&|E=V{{7Nim4_Dx6B280C{YX!OB z>QD0Uk=SfwoKOV>+Y8m{38B;X?VOdq%`{~&3Fpi9|^T({HE7Px}K0ehCq za#f#oqp%PTd>d*aN;&DI1%0WxryP>xsWQf)?F^ee+&S9YX*^lwb>ttQ6d#{zt^X6c zs5oMP55}qVt23)gO2^y|jbb&iax*t?(hH^OJAEkmbaL;1TOD(=nsj#l_|O;n@;b?M!y*`Ax7hE&=O~@ts^ELXNhkA%^T`qKZnEeuc!JY>yXTS z4FP%~4dZ!O>0S==x>DDhu);o$y*IY=t|Kz_IaLyCN*39SZ36kWzSs!xhGTQopgO9W^`Kdv~UDO0H~#X5^drvL4ArbUfYy zZGp*iM0qBx(h+G~vlJVxC*rSGwtkvjfpI>z=@KnK%tM(L_nl~mn>{6MNyZwo%@~57U%*oQDuN$70i#{&fiS6X?YxSQR#L z67dzQ)T1Juz;PN=Bk5yjdEfIn>dfjYX13IO#g;{Ps6#LHjP`SvU-rVQ#&(YNlb_R} zsQY-&b?3pTss_cGb_>=RPpDK)JHCqVs3;yX1_+X}hWLu$gh z=oJ8E)Y4vig}RO(1hKZ`yFc5n%-lsb=a@7qP9eI*`kw3E4lfRwQzr@1Sqo*xU-agS zq*&EroeNjo7_^vbmn!e;8PqNb81_C;EF`Uq11SZNoSVku57ywA+v}e3_99c>c7doJ z*IAAk_~Ke?{VWu5R|4h4_^>tetHkR?g)iDGoWGuxlFM>k{w2Xc*Wt7{;Y~rd+AW>_Zmmpzw8`I`t?!!kFbZpWeHc!)k`%Ly`6x*MlRTF5pen6+* z-U=s-R$ZZN#S)VMLT%s|Sne-<8zaP0g17>#0HT*0oJOx}%lAe8?i%g&PjpWT+WBqp zKUXLw_iBKV`tEACx6klRv8dRcC(2!a6k4PuA-nH7KxAImV~hCmcKx37-~e3=ODAtk z?;OYQV7Y++{n-oAMu9or$FllT6H=m1v~7;Q7I)d0NVmN$Qg{B6aD{L|lyZM%+-8@) zElsB*k)1Vr(j>2{%}i|;|8Tfy13I*oCPzA`v)08mA9Nq91y3YMd-79VJXeshlgJtz zb)x2f;VtZ8sM>FCs^+|;9OkEB5svpmTl7o14=QIz?6-3kFXY|alLH&JZhVc3zVN5H z!F^A4z$_`wT;zK9@(|x<=^*&(c0Si6LY-r0wy~*oR7r_`(2;w2`i+jebT;o>e=2e4 zaMo2)kSxiUtWR`a1??k9FuNK%3~3c}A3NSb``Q1^^?Z<|(_x7gDSY^SS+cf<_OZD{ z9vhh+Y9eu!y4|K?>@z;pK7MDPZ8L)=O^2(l+xHuGgiqoPAEH@3=&dC#E?D82>H78B`GM-|b?%kU73Rk8jGLSo z|Q=R0^`WUzjhLGWt*4(Qo2YJY+rp$uH4L)2rUTBX$ zT#6nlNMhfRFr#+9^TOU}l!_}0o}e1?ueaf{vK6a}zV1*3 ziuGSPfE_$PEC}?!lC0@MbI9;nxoq1nXn><_Lr5}-bk@7@pQf<%iwClAnw7FV zW|1RH(9?HayGrzLlI9z~+>!cZRf_{3TeN$dzJ@UAmE2J!d0P6@cN37H*2~|D6K!;q+Qo$@`b8WZ_e6M!Tw>e3vKq0D-Gmfea z@`Lg{V}SN< zzY!A8&&k=O&&BCJLiZv>^8i>uk3M%T&;Oez3nJ=oHyserF)^`lI(ObOCc zNoI5>EB<&5&bo}}n0pv${4q#l3S?IM(drwtAo@;UVz#`LsVIdnM)=@WzJ(gNlsLm` zosHox4Wa~J*n>{FI*?lFI#$^b__{!^%dM~82(<=!+4gW?t_tXLjVo-kx zV))G?iaW`}ZlN~vrN4(T4N~U|P1~3xf9vny&|{osO&T4eb{jo&-h2d<9cu_T*J)VM zl#zeHT=>`8+4^&PYn_cQb~T%Z;nQ*_&uFjhy%r%WD*$Su1rVK9lPqx)i|+h_WpD? ziA&b`X+&vUb_|inpejffOIzE@Q(s;EnahH2Lyzmpy^grV9LKmK!^a8DGU_6fB%n{; z7Ge0NlVUvSnw`Ul1`eEvk&v!IMWE=5en@yNg?%4V8AyC55f)KmuJQVwR^V^mFz36J z@3crHnM^dU{=fMOdUYWWQRI=uQ6_iFx~C>|csb&=zUZ?>Fid%=gVZIcVXL$7l)y0_`p*PA^8wYMm6Nnm7}HNhH}x!S)P6R?^h+}SGm*#N>h;gt2Z zDVqUV^a|xmA^gggQDdWNdc)rUW?(uv{dUt3-bOiMYGN2@H7wJ42pP7m^Fy4R?XAic zi(EYB5@s6@`ID`{l$(5Nl2P@wHU5{|T;t_DU?$<|bD@fd4{CGT7%}0&CI)gZl7bc? zTt+t(;)_xv(il!-zb(mJyCTg-(77xmkf9do60FZ%4`Bfo;nfYXCWH%^6dO5>EY~-( zt1oC8?y{M-)X3D~gV!sJ=Iha((ZyKAa(=lFnmldQC}PDzl4Amz?r|ku;t_LW>jty! z7W1Di4C`n2fYrUzFTY!*;IGQu2?RzBWG5-BkaedoYLsLj;af5FM;fvWDj-QV*OU>Q93W}g+>h#!VERrmywGl6 zuFF8uZRwZZ_mm9`Y1|E?W*1O=?5>=*MBbm+N^4FtI}XD>7~*1jR}&OLtPOD56=Ex=$V>dV>QDq+ArP+r4lYbNIbCNAtobL8c z!Q#mNo0=ZGQc#`rldv}fY@3aEPwC!DNrCbyScvY?7y`p|*0HN{^hPoR>cS13K4DR! z$KB@qc-f`U@+^Jh)#j;Zsbv1XMQc{}Z~q;F75>z3?MzLwryT!!(a_l{+GDBLZ3iK#xQkIQ1p(F=crElSKdk-MS{@hv^3uG};F^=peJ ziQ*Ugvo)UZ-q`{3PcwQg;*+n9@(|MQR0Naw8fKG$=y2y!1Q9Tq@aIt^kd=P_Rvw_ z(4C?h-~7fvp&H++mXmh(9L=2>uj}fNJJoia22Jilt<@b_rFHltv~-`HJKB>V3P4LD z#($g-k9qn?K0330*rYSi_mGQ~H{+1@&36)mr`S*c)xZ)WHyc71WR?5_Y6(emW8sL3 z&xqey&3yJP;Lfc-Btez3-y4k*CEIPV@b#qA{v>&1Q>QTYAa2hA(V1e9GTw&BUlJ>f ztA@gFoHhz7JsyT7PqX&Q~#%4hDcM)HDKZXWglxvlW3T;jVDlW{o5#%Oq z9N`gS(Y|E-2EuCF_Z z2&H#3c0H#_JkPuw5@+scFi`kGf1Q_s^u?_!FmACvdrlk^P5=w>Gu638U0HkA{K>al zQ9a})uYUFyl(ryRzvfld@WO;1d+0G$P$wovM##~+-5_T&?+5&u5SQWxQ$Bf59=Fuj z&YJ2;cnC$UNAKVH*w3%Acja3-Uxq_bSa;;NegJ|RNibL!MO;&rEgdQwD*9>X=J!A~ zJb^7Q$|7H?}QK%urj&}4H5F|SMpYA#Q-`HztuK|+1;~Eyi-Dw37-Qwk!bN@YV z{*}1Q=tFkf{nr(}{5(c?y0@-$n8s@a^_O(@r;y{b2Pa*5$g?fM^t%+qf#ZX{wfY;g z=HKaVKQ39j3bVQOP>)-@7077q+)=+$QDAbTl9IBlMkS-bN`ce@q@P5E_&}=6k(H3& z>ucSl6sR=upT9qmd)#LR8vb8F{lShW>{zAVOAE~?Rp?rhuieYvQ_INBl>w>hIE0eS zHym#kJCb5GOoO~eAf$0p1N0GGzuMasuSN;<6X=qZJ$rY-F^vHw2{hH|Yl}7)rcHAv zu66?cxBA<893wgEs8VEeV$9vt>rjoFdj!#}^}bGi7escSWHscg_3u_U z3Ru<4O1?BJbGe_inpqBhvKRebj{pUZcP~3xVF!buh@`pxn%G>IpFzTFH0pai3~|4r zJsXzoQ~oR-rnWUWk#358W6J#5eWj!(d6jFMIq@$PF$N+{=pB*{Nz3hGp6vwp@E^UA>m zzGf8Tq5=j)V=)q6f$*?j*>6LgQx@n30=D`EaSvPJ5t0OdR2z*L%@ecfs@uDpht>Q_ zG6pB+Z$nO)y?ez93d7jmX!n zA_^66+fzE|D#c?tYS8GZ0x>r`JkLG4_vA$OvQ!nMgsc+o=~VeThiOQ;Iaal_+L%c$ zKfi3d5Aob@-z_*TO0R!*R!jLz{CFXrgfMVfsm4u!;VUY3)mY&u9{BpTDr^uZ8mex_ z;XIId^b%sfEeL-z#92@)!ptVZHpCm8yp&?ci<>bLg4EldL0Fs zgg`b1wPi7Z%CEYo)Yr@&KC|$8_BkrJ9KZ^-;fQnVMiC)Y)-a_LV=Q2y(JP>A*L1Fh z_`2;Y+xOIfaaoT;<&)V%Q^V^^npbZT5XHoCz*1#muXsHrEIX8T1MgpH%W3yu)pQzB zVPGHZXLd8Vz<<9OF{JIt_}S&#mHQ3PX+jGBdCt{9c+U$hf-8RLbI=ofDryG*TsWK{ zA6mS9$T<@H2!hDdrff>XFfPZ(I?^?Z*9v?I&+DuC2nnq_z<{t9Xi$&15iP87&b?L06u7UntYYCpG^OC=*@{ks%KMB>@o(UZK@vmFZIU zj_K&ZcoDqU_7Mca3%|U{YP@JW<`w7R>7k_b-eO}(iLnA)_{W%Xvkyd~?g}Oc<}n1H z*t&@2Tn3bxtk-1Q6JL}(0yLj=ZA-$;JeU*@UFz*11~ff{yv zQ!j_sL}I6?+pP6lHP-3wvZi0jmu~u#)PkqZMaT|4k)V3k zz6maO5VOy&bj*o5c|!^_Ov;F6{`Wh(#WCjDg7L3ZEn4pnCn=^FRQ#Bs-CT`Of;9gA z#lw{#%S-s}F7_aqdTeqMw$!lehqMRgSYIVa81zg=$7`~$CqhqHB;M)S7;o^Wh}8-O zW6JfagS|l<_$iJyYHdt@tuqQmbqVUnxtrn-^gE5eT55`>i*g+qh5k|tol(i4CB61( z-v-X{m&6$y3lJ6=L!?*np(sN*PHGNaDchJ@VH6C7D#dsTXj^TFzB+PJxlX2)7(1hP z7N0V=BSMA(-5^lm^H5!hU1UBT2I`1EfC#+umi7$dzAEg({Xy9$fn*`CKCd1+t~!eL zdUSLYKndAeKP3ndCGkqaiK?V)BSRph^;5N5KlZ9zn&W`V?~23^`4N8UF{YPa_$VRd z2dn3iS|lN+#A!HXs9RTDr!bC~hr+OI83PCc??v(-j-V8?adE-&c!u6!N zmm1gkRi?9eGoE$r7Bi8am7>5u57vF);cCaJ$c`sQ4Kv5h1W$@O+HCm+O=^}nNu&`V%0K>9 zM_#!l>GkiTGE-R1{3GBI0S$ob=>`EEL<3yKfIS};0Y@h3_b(V`mTc$hPT%UVX;Bz? zFJ-M)+SE&erjw+)%J21Up#}PvggRt6lwOSw!lTxVgGV1rUvpPJeLE3@srH#*=&u?P z=(XHuxMHJiuHw2G<<&oeQRSsfnaKtIqstoW-WJa0F}@9318&1Vcf>X>qB?W@AXPh! zb0>3huq;xRaRb`;rGK z3RT9EQ3ENSE!6tYj`URgQ|XIA0+^nK>cvpYe@+4QEA zDf~uFRaZ+xtDLcy_?V$IyC#8~S$&_;V%;JJBWdn`EY`pg2p|#yQL1vT88NLSX_|j; zXmi)k$xQg1CP4BQ}^TLspgg2jb4?`i_Q}3ey(R%8}Rad9CUlsi*o-IWb(ZAURHi`ZP9V9qb z8H_wj{_tr6dNn4#;nH0XyD>L8@d{eHpem%xg4VNUmNM64RF`*f@z_g`4`mFxiK`z_ zZmfCXJ@x?oW8G}ZQzEG9!SJF%^uUS@#QyFl^PBqK)oD&GPIsXr9gNK+w)o^w~Usz`ZdY+`=x*m{C)yDNyvcH z^oIm=??u#R{IfB4j{aZ#z5B%=4+{ZB$>+;Q+CRrOp_&rf?^VMXPi?0jGW-r;K|o#)Si*-}+B=)F zm^tm@Rypo2wUDt1Fp{1p1F%?DO}q~Jce@}Ku94qusfRlHQc^G^Ah00$3+{hKW&s~G zBIXsPpd1_Yo7fV&L1EbcV;!2La_@?I~eaMgViKssZ zFc^l zoKT~kQl-I;OUH!p;|L5HST|Hja{6+jlKcjnq_jgE^toV?pv-5MZApx)Kr`k6*bCOSm zDLN(wyMPd`Dr62s)hWE4>zRjw(?XQ{5tplZH_qhkw9T0hbNZ^9^UL1SZdl9$+kH3^ zOVJBT&_Qnzgoxt)qZ3Le@@}KvM)~|K6Lot7-Lz{;UijN*Tf%!Y7drjeB z$4?DNph%XJ#|bfo(`X3YOlitoG%>|~FzbFu9P z)vy{=a-n-^KlOrij8A`KQ#^yKTuHM9gufQc9Ca8GBA z<^9d4ICgAF&9m~GUvv};%aAJXf9O3D(Nyg>`&c$W1>FKY2V!3N3l-QCFjl&|h7_;V zZ{GEMR>6*Ee0HB^TRKz0B+6%%Q%>>PPUZbQ_mNJeAX(gC0)W-&3R_XZ?teJxpz87C zINF)x@s_>J@q~U#CaYU^N)P0xTdr^fDN=~<1^~Z&3YrdCvp|MQ834zxDB#uCHgJvE zxX4F{$k#h8vqr^kidmSKDRN89M|`{sM(_TTe0|lUHiwm7EA2ao47m=pQ0^)9HJZ5E zv(|U;@O`~d`gl5L5mf@NAOGp3>yn!6r6g@KPWjJ|zcJT05e078>RXLHS1k3x4jOz9 z3jVxn%o!SMZC}Rs)sBziCE2s1_B|`Mih+6IcO<6nE+++-`D(XdO=y>d+i*&P6qr7= z7hHmLp{*$0#S}l3Vo{e+t8*6k(xgv{9IW%L)v};A0+nYNQ;xx}6B!A*c$)%(Jl5j_ z+D09()h=3iFAL#(Vt2S|pTQ{@@mfzWZG;T3ieXhTsV(W`Vp zo~9?~DJMxC@Dr{64e6_>mUsv5ZbtdiEV>(oSq_B>Nki?Fn!d>5><<;O39+t zkX!_do`Z&f59Pbcvs*kf+ur|TE?m`jcFx_ay`}Ie@s!7!R-M88A8$j&09ujPqJ zI;)D(6c?TPra!_#0_ts-&pK_ZI5LwG)?ZN z$I3efx`j`?r;Z%ee)-9@2^Af(Jr)cY&j&qUTcrfzdn=x(N-MlNkrA@r&Rg_5+q~T0 zow`ve*Xv#8i>&*cJ-pe}jxTOid);TA2Mieu(9)#to!lIwvP)V;_y^p4Hbu@(tY9~J z{To+;#=;!Q%g9Mr!~bkDk)ps9-bT!TKxsY=*0%#SNd-BVAAA#f63ew{n0Cv?IM(XD zGGQ?zgOfxx)gHcX2eQP`MIx5TkPC&bxh2KV-#{TD<-cwh9X%Z4LdmryzSJa5{peGj z3nUE1*si^pi-&|ecm!7a9$i02NGVF4q`D*}m?GN~aw6jK&L1~IwbanW2o4(gRVJk9 z$Xl7h5On3T*jt(+$nY|!ofBmfYy>=D3_)5#6)xzQ>09yKY6FL^PWhr3o727Dbyv?w zm#pHqNGx`ffI!d;*LLy-KjOF^m@pBx!qrtbEu}CfY?NaAbxx9Z>Oeb>$t%1H;+NW# zv#I?)vA><3LR*jdJfTQ4*t6K`{ufnGe=def%#FR+VJ1qE(t@mU%IDVq9V*TLfWT$S z|MaoHC%$$H@ad?;9~^cyHJyz$5gAg>sJBL52B(7tcZuWfzdfl!ztt6m4m|+^LY2@n zNnGha%tDO zcrqugfz2q+(VZqUJKo18Fwol)9j~%I$w2?y>?uI5lOb8VG4Azp_NE?-no#JsgYwXC zjzmj<=NCwP0N#0W8A==?4lER|!AW6Av|^obOgmliNafC}MO*veT$&P>=vjG*5(NQP zXDwj|=VUcwgJ*65)umNnz4Be9bnuEn(vhw1kKqdqla{44FA_50Fp`Q8z+ES{owv#w zpWYi#JeyEqS?3-&BD%7GW^opKq|a_)`t+Sw0Hd;*)#3xGn)34t>|D369G(gll^svu zri%}1zG>tg2^Eys0=q3=}yRT(d$5nyd zI{9kbv8OQ&!twWv4D0F{@0WEZS5}Ex*{UCe@fK;_T~U+_`h4MvSES*&=w4Br+#{$e zNHasy5y_uXmqMX*C7&9zX*h@86pd2Omzegv4l9ah4p&|OwG3WY>c0%_JNHXM09kc5 zk@p9yb5@~R@f*R-FGqdy4bi%T!YYuwdVR=MyS<;()~OtK+>RZq|O53Kq>c=iAKx>!GP1~89K z(sX#9-F{F*NLlW9F?w5Ia?g9w0HYSiu43~9?k;rgj%+e)Xz(^}tIv}f!b6Q4s7D2p z*&%R>A4-?lFvsHZtyE>P)YWeU=->c8Y%9I0cri2lT`OSelw zAeyvZ_Fru;%agbUmyN0Py_U2p_o!f{?nw(@4T&ifl0I`)eSW?F1KW)X<#L;k)Draa~CskGn67^uSrqSd_PbV_s8 zwI-Xudhr<2DTz}|E!HcP%*sKF3dI<EQJW58I(Laih%c%bAvy>i-?HTz=Ci5-HAuXo-sWCut}rVF4WlsQ)^?4A&Ii}2QQ-U+AF-vt4NW3dWUU| zVW>u34pr&F2uJii*-XL&RvidVp1$qe-J1%2&vkJ<4qpFRr48(YC^)bFk+KS%YI*}S zI->*rqt6*s44ju}o)#2{2hXX|h%-BgYDjvqYL>R|PNR75kzd+RjptVs8(W^qVbZ=( zs-9G_t0_TNX$iup14dW@5g==-vCUW?-fh3{<0etsl*^4_dSWHnPA(C}acAk)$klHu zk@R$>A~iWqc-*>vMUu(`>?yJ>NvH1(M89(N=WT*HdSKSYIh@0a|7>4?3N9kD0F&&w zO_>CgBOYS=FOw6BVbE%Xp7;CXgb}FrIgKFZ*1LDLL;^@|OAI-}#^vG*VDdG!f|>zD zR!Cv>uCjd$cB03RN3?l{aL*Gh;%BevR8Ib5=%$vqE}MA8Et7`EUmnB76?`Kn5jf9; zb~|hz9)#SMj%kNL%Er8!l&F|{^&PgEdmSMQP|GwOH4iUE%n-@>PW+j$7JfOj1d;{R zg#G}v0v3SH&vDWQafbK>Qo|A+=e3^(!bQzR9PQdz9Tl$`TgHDkS}^~0IHK9~BZ&Kr z4`7IH+4bGSyRkK1seW35PwrclBDE)v%wi9ScZrlALUNuPe`dda@*#PoSH^82J0_&3 zy0c3?daJIrE1uKF22Ve{0zH!IVb{1(`@qJFga>gzq{dzSOA<#=aj)D7u=#x|e?ar{ zaL|il_LiJJgUIhS3UluJ>?9k;lu!O!Pyl1a(5NRX(8ZjuCbzSr+w!h}>3$l$IsuxK z`+UvboY|v5OBBhi_CTE0W`YT39vWkePw&qyjTOcT)o@T!IXlI33_u>67O@NqCWz;? zm@h}RB^pysDOe1i>tHm|P={6I=}8yO{=!JVAFCpVQ$?8#@%&`|BeRe56D|SmtufW4 z(hYucj2&^xw=sksF})w8I!hF6nXUGw_)*#s%~sG+KVs06eWJ4~=zNL~y?Er-DfgkL z>AWdI_esKCnfQzs`7;sE27L>HbbR`xHpSnf`TwdGI_J>z@4wQJhYJ~}au92UEHAZO$xnGG-RjMsbAlTTo6B6x!kZ?*)c$;GG00IfU=+JTk59*O zkfP?>jY{xu41Y-P^G588JfA`<9u9ds$cgyd4QD;Qp+v4)-mS87`?Act_F7u$7h9UIIP(;UzAeu6E{#en zYeHiwQUmnV1T3s|p$SXS#=0Oc;Eb?wsDdc&d{YuWzfy7NuZyd*tHx^Z3AWekG7|{5A3&T9*7 z(nGTS$!E8P7ukMAI(|$5men2f2lA}>Mmb;eIO+9oNakHzcprR$49#5w=s8jq<2x_UlK(7y|Uf4xQIO# z1})K;eXi`IS8a)opU0j<;2Z>KIsVGrU}QHPgtiR2GOUtkzQCk7)#r3!Fj`3xA=^zx zi)rRAd`F^|RM4q33p{VDtUC*!NS~UR6`lycnIHq9$g-Z>doL?qT^V`Lm&2TzW&B~* zXE=%ks`6E7pQr$$+{u^IbS&G=t$;-%wWboZ{tehUI=)^@YJp4oj&kUv#VG!yder=?|DI z2~xo4l*8sMaMDo(ebfndr32*;jT)<2FvW3jej9!FD6QZVZ$vFWNR<3V*uB^PlWS4^ zV_YFda>EwDzeD)Do1dA4YccJyQs0mfsiF~nG>up`oYR`b{e8>%7^7N}GFm$B9+c*kQ%?*Stt^vhR_4R!Jo;YbN2=xr%X>_!QF^ZR~N>KzW(1 z$6-~Kga5B|hb(B_Ckd}l>EYevf0!%K>SVCFo)I!w8}d9lR?6>M(s9vj?}HBCvoCFR zsUN*dog@uyfZ>T0^mqYk!3_;i&=VYOjq|79jToQMvN7(CcJZVi`r=o$*t<9K2>~;K zyPO?zC+h~&e6CVe0 zwR97@5B0*yA@9D8dZ)bs4SpN8sJEayX8B$9UKOnLY05+FW#KCLaayY;kCGgcQ=gZ8 zaQWd2g2-2z$1i~yE=ahF{LfLwi`3f6h;$dkOZ;p4|MSgPGU9)-iPs%uFSG$d*NK!} z=s(UN(7A0T0^W)-hpzHD7^Gl`OBUlj02Ez$|!Wi)Jm7Q7+Heh&at%e}l@c;lO zyt|ACNO@-}*Zf|1XJ~cQIy$h?EbCaGSc{_Z^&tC@AH@cq>~hC5 zwM%y+3u$#V^#S3#ALgwynEe`kNSms#5nf0lNFF6gYXGU1BB)f;1nl6}&JOnV-Ve0| zO}uOQaNu0aM9ovk+6yKTz=kxv^u~BB;X`_l51<&@Y!6S?H|#7d9S-A;7I(>S1$nVO zv-rVe29!F^QY6qjc7~^?j&0CYD5!mJCbuIqRefjt1~x6a4uQyinAtSrTAhC~yEWQc zy0_>UDxB;AJsxaq#yR(QjD0MiZp@ZYqQE4&jB*l&;v7$9r&c+RZ3@U;7Vi*Fq?$wP;#}lvbf^RB7K}a~Ws`2eAv7u_`j)?JZKdH8hwDn&&)PFO9 zB3W+8w`x@CE?f=14s>D$;AE+xEsTZto+RILc9kkv#3z~+T z8L^4!R!`|(!Yz7d;k4^mc&tD}EB1Q(6UqMOmJ9EI58^9vtN~0AHWqVg-shV_?2L-5r%5aaK36o>YSG)E({|>+b+bj@c5TIyDMb&M68cv+QgkLs6 zBx@E_yOf5HD3o6QFYLW{R8!r$FC4{A?;TWn7im%>AYDXy6Qa^fKzc8NfOG)?0qMO< z3B6YV=}i(^LhlJRRPS={GxmPZIp2Hk`OZDxxOd#|{Uc*6O%`)yW|n6@&#%m;RBY7w z*f4cL2>fbyn+TmQGnf53sJ#=v?AQ9_L#q=I0!Y54X@^NdZ@* zt043*klfYm!7mW<=EUw1qrf&oYVtVKQKGrUYXA5$d(Z5kHe;AdY+-vgIW>27DtTYJ zIGWTL8{3aEB&!Qx@(d8=F|Un%u1al;YbX1aIO6rCG|=UPMHe?PEGNN-;^KUihBGz| zymbwz`Ml_R2#;{QPbC~yWHA5)JYZiO(a;6f47xi{dO?{&a~CYR0e zAL*CG%)%vWKR>ODyQz}6Fyy;CI@LS`z&sA5u>^vUJKiGbY#U#9tdBPzULL|8!B2+S zo>@)jyL5F*1ioqyxN~7qxHJNVSd{D@KY?VcMGwk_0B zSiJs|syy%So>JF;hbAq$j8!L*&R#q!?en*>K_Vd+lto6Wj<%>u`4*UA$N_8c)WHu3 z4|O<{^>JU?U5V*QvX{5Rq5~^>3%s8;h3i1mM8~!dyLCS{8icD@m3Bq!u$uF%?%v;# za1s4}fLNF*I>v40(?GX|@|4)$ALVvdS#277Fu{~+%e0TG4`TV`K?+h_Zdjt$3~b!& zXUAESGz9cGcD$cTD4@4`qLC@ZgS-n$6XANY&b%7a@4`?*`1d-+f?wY&e4)_SLtqa; z%FKPhT@66i-^%4QYuaEyB}8Q+;=!Ktc{AUShg*ZZ5(`i24oo_3;ZQG=pLqiFuzzh4 z@Ny~zIL2K(GM(M3$O0h*{lk7Q*m!mty|f9;a+l%Gyb)T$R#CG#AcSf4{-aF*l5U}@ z`7Gf!R&(jD1bd}r4V-*`z>NL`e!oiN(~sOVFt3Zwo5pz2^CqVwaEYR&HWK zCn1MWhi&dMZ#EmG<%fhTryia%hNT_uj%NfzkeeX&G+C$j8wMvGHhx6za6bxTYwmBa zu;zQ6j)vS2ewF%Ioi8e_0oS3pRV{7~fm^_U9<00|!F+BdyZ|riUl^JBQ3TzsHf`(pm5!C{$^!U!WBIOBZox_?E<&nR|2x@ zKeg#N7hTIX|5;kWM1SBcz)m{q=n=_L#2>L#66s7d`E~-WX zil)3GrIun+x~xA|!>a96hkN@Z{y1klHWar5^6o7bNv-*Pwe$up!CC%Nu0SRF?R1Zu z6gP$knXayC*vXNNFFd0GI7Hec@i8TEEc>LKe7jeY>6nqwzSMm|i5~AwC%SO)DY%eH z*KZs%N0%N4N=_A9*ykxV9F2IqGi>{zA1_-SM;iS0I{K#15-{KM&CbQJl{P6}>E!O{ zEymX)LWG3yjB$;dQSUsb^xxf9Xfq{k)&WGrN;u(6<$*HVl!0-)Nq1#)N{Y>o%m9*d zV}ERA@^jDez{6}-`e5?7f;OsBd!no1Sew`_cnJfO7d}uAG^)T5q`T1gC7QQ3vqMD1xopSQ0%8?}!f+6o`t$3}y^G4mO2yb*K{<=*=nxm;ygR=@(;iqg zf&SjXRiBh0L~Sy9b7Tw#Rg;-0g+)?q8q#iw=)Xy$;9$0E3f7^zKV~DHUwDX@jxHZ= zza+dOX=U1xRjWUzoDl}Hua{LjC*p2PE9bmFJks1m5z9`ix?s!fFipbM5_yw}U&=KO*J2 z9I2l;)nA)C`343Hrkd!E*e^{wb8Q@M{gl0t20#q73392pGVw&X$N_(F>BWiTvneIPODMwdO(-(Z^-F-6D-wXG>gEhRB#+=w~Och0z zf_f32*I~D#y9PVRBW-J({-Q^Ym%VH!@Kt9jISn5hQYcN=f>Eu{QO8q<>Yn#2-9J!?iH0-n z`tWY`o4v&(~}9#}S*g5Ojrwv6T*sd1~`oQ8;BzGO)eWkz3n2;)Yyr%@y# z?L#!%&Ti#0MtaMZSiwJ>BoGfgsWEy5@2UXP;#ocDbd;aK0lmJsZG|c)(-UZps_?KK zo0Icg6%PfHxy;p0UmyJ__@Uf7uc351**lE~qgAB_y!MtZxW@=oCEqPzY z&0NGv%5fmTaFtDBWD#uX``%rl%~gu38htwiz@6=P_K5j?6uNvdY&2o-eBdM5q|@*o zrhT9r{xV@})fA~0@Uv&pR}7OLa76F5*PBF2huK1k z)(h3_d(tgpEJPV=Xd~k02oV}=q%yK&abgl9yPr9$MAgF{VbsmR8gt_3fAPSiz?~GS zlY)ZMSsNt8<+%@NSB#d8MAy#AGOI#yyWagM)BeJTxv^V+yAW%VJ+s9igjHDbv1mW` zCBqO@@O|9Q`&LmS=B+#R$!dQ)@%s=_zzQxN#gOOC-nG2^rC%U=8?mF-Fc%2XCZ%V0 z+QP$ql|lm)QBFMZcSFb5vGQZ0tT`>%(4|W?!;r3q=uj7~RAabs7*`)3tIfwH7c)~O zTsv(u^BXt;RDly??@Y9bk5i+fv)xlV=Ls;qU4%j3HqLTe!&$}+l}4SXE8|{{kB#O# zLhsJ;2bi`q?`LGw?Fub}t9{=Cn-*5B+d#4=%DLHaXH?cLcO!Pa@&z;%how6rowHi5 zrZ`i)$Bp5d6lt-yEas}!?@iHG8p{L>caA%_It{gNWmfxIdXBV;B!rVdEASJ(ItCyH z=r7TJDVH!THQv(fXk>~#MyUkL9cnW1Bcf>&?^c^5(mHeM+M_w@Klg&F#7r3 zmDNVt_- zsq159yF!s(T}h^1@fA_%CTTsYi~1DH0di<(OfCK0(lah^UREt@&Tq5P0g=jw+ljgkLE`un&Q+#6@^yfq!uXM!P_+{&Vd|AGK0md~QX{5nkyqZVu6& zZu%{5wq7Km7||hDBalNNU$g%k=Z`7?sITDTfM1}j)?c86%fw$Ga^nR##w}mIQx3ny zFvg1#@Z?vKg9!l2GU<0J(DiShR2?`t`RxEzEIB{mFn-Q=#o62v2A;Zr>}vqGkbzJ2 z{EDC0uND3R1r`BqT73L>o%Ir}?7k;GmRp7K3nlHQj22K#G zW^)z0W_YN}0RG5@vO4FC34qMyDiiKSkY2nfVNj9(aGAp*e6gt&d!EbMilcB2Q9la* zyNB$KkHQq+-wa!avYG^@1pQz~jC!qqH-5dEp7445FFv#-o9usjpZ{NX{h=5C>Wx2+ zslR&Tuip5x_2#d!@z>b+Yi#_r!~PX-AORoOp{)OSJp@cJ|Ck*4uT;ca<-m(y7WUsI z*ItHCgU|FWAgL~A9Ds!d;CB`GVX}4Gxc(PtY(T8P<_A2aD|#auh|Hs(Tm{tH;Qr&P z?{|sP*@}5AhuFmX!+W6C%a%r%3%U9};qwrP6#;*Rk2$wo$g?kF3eMzyfqZHKYKp?K zeD((AR_+iNWOs%D2{1yQIZ`aGe?)s}Y{u&$q*US`0N#QGn z7&5spDOLluMwxY|KKLqsToj&CPv}Jdb75fo=b~^eRn8Jfj{ak1Q2*<;zq;h_+s9v{ z1VsGT#Q7aA{Iw{M{a?N)C`kwysxGV5x{fSxFu#r!f4`wVat%n5-v$gf|5YT+C95sGMG7b>p}C*r7AZ z+PV+%?#micC1Syv0fbZjT%&SO8!7s!3khyogDNlSB7pdRoEIkJEC`hO}w9F+aL6NbLdOv+1a%q9Q2A7+hl zL}%xX<-c5ba39B5cV5W|x9*8t`A^R1(!TixdO!;~0bI47T5lKrtJeN)>;D`7eNB1# z(r4|`h4GAsb4LT3<1bxo3sO9_j1rsboW@z$x10u;tnyT6i_ePOPPI(1>eH=nBbb|BL4 zn5I88b(<$*5oi63YbBk^cf7$je36kDo1%ywD!i;k7M8wk9J61fIHB0`_@Su^n|a2Y z5oBt9`t6aZ5CR*wR2GReRNQm1$HV%|CqUqod6HnyjpG?x*g(k7dqFI*!mhXn#DIAb zV!b@Bh2_F>%3IQY$q>LPY*x(CGc0s!!aG@#Px*U`=kFNEeN{AoJbtWrlZ^G+%}Sds znt$OFHz82JK#?wx5D31@9?g2P{V{Ae9uzC?>-x1J(+zPz7Qyd^+o%0x)nAe|>@>U` zLIvv?felmHoXx*S?cuJ&P2myK^9+3SKOW1pm>Bk1308jne506BHYQj)b#e~iv`;1F zSdAKN6h4|1g5tcKDxqG|B+32E!OC)4N&A=&MC1VwfGS2ItXQVw;06qUxe!2}<9Zz0 zQ@Y2g2H5b(>Ws6}Nbbz4=Ol*S>N9bJ-$?Oz>dvWW;)o?hm3Mp9HL0Q0=%nJLLXh-t zPZa8FMh_cjTyP6{0&<{sQj_eo%Q23*wD$-U&)aaeXt+8lj|0%J#29LeH;;j8s7uBx zz^KcrR;SUvYkC=1TX8hiaENXB0j9l=N+<}m5UD$|-aMoVgrZE{tC_lVf(Pco~u$VkPM?T>AR&8A<$}ysx7I5GVxO?Mu zfb=bYF)?;P8-%@RItx>>W+a-el1+QZK%0*$j*-JfBd6fW_ZuW)#7`+K%RdW7GhR4I zfid8qBg7uTWng>Fna|O70^I0a)xqAC#Z8gOSd@gbOXi|HR-xaM;Y1Cot_Ze?Zla-7-SSPQRXOO_bRggZgDF~i==S;q8hEu0)9zPp%;<^S}HgA z!>f9xoTV;``o4m`i3h`@Um(#}oHf^Mfgv#FtlO7P0w6nR)A7A;kNL8?YxK~T`n`I3 zzoJgpG}y-3gRiS7DmvUOSljkdCrC0n%RO-^@BQLPzVVog6|}T;r%VW6W~wi(W6hG*N1stb(hzE48G=Z#K&xFs}z_ z-TllUG|Otj?eVL&50IcNwGor-NiVhzMIxN*CUy_En4GN4R`Zi+RPVnDak(FHJ%g@_ z{YJ3OLbY3fGnl(OZ}Y5(HO0PoK{6;ktpOT8wil&RGk@Jo8d}&~Q)p|F3_80WhEc~L zI@}mU0^5=$gY%2$25-hMz-@Ov)HuK)*ELW|;r=lN?Qi8hXk}lcrv0x3u}&zKLn2F0 z<6`p23yndOf>$$Nw#!KpxLHG4-Q*;NoS|mzBVG(?9cFs!2)t7(j zLgiMdi@Am-9+7gotw(@I<9>~tAi8qsgc-n;B-hGqK@9iV;d_P#t>?;CPZ( zyV?mdS&{|%=<8kaUKNo9$f5M^kt zqA{pfd{Vw6sFUKATbJ*H%UlT_WOyjI5SYL{aHpohRIGchNr#I$!PPMl?}XM2o~F~e zir*W!zqbQTl2>~amnH;h%9V7!%^DK$#Z`hqGSLodEZ+m&WSslV- zd^HlIR*t2883tbi#QLXJCtIeaDr|WSYz%q8I}Wcp6Syl#+|5BXgaZ+z(}S4v&QO!g zSwl;)xs|n#$h6J=4q73RXbU?np#gc*YfKM57nl>YMX zdznfx=8>84+C5L*XK%^V-;4R=E>!q0I{doBfKk$8fM19WEOz%&uL{5?S>a_J;mw|F z*yX|vhSl@_=+;Z@%+P-IU`?`Ff4Pul9vyNei52fkUMWj_V^+wTUZ5rfODpFo8+^QJi9?~7KNpu;LKc55 z>=S_;U$B&&cluM*ZX_?rYj(AV?$BY3;-47u z2gJqS<<)k}8-VoAxfvFVZ|@AKeS9advQm^fmzcJ&3PWhmF(BkT%u_IKoiusjNP5?K z+}v7BZd*+y-bZ>y)9CXNz|=e9a%^@9y~y2a&4{y58c}n`+BC8m$*rRij^F3vgiCJY zu3SIIs&Jm_*cO13-H{Zjz;choy8+DlH4aF(nw{`#qa^S|Ulry0wBG&&s^x2DR8lvV zMlMA9l3=*c@N-j}wvQ@-K$O0)si9| z$n1~qMG!)3uw|rU53YIAMHT?wKmJ^LnT3A<|;)&3;VjjNx3)eKNggfpFd?W?eykiYE>I>N3D4oC!~N&oW|P!Okeqjx(pgMRhLH z(U<%(%Xvq(?XmlGHx2#Omj!zo+}*t? zKbuhvMh8Y*ybay?Gx=V6Azfd{c_EfH_5*WOZa#umWhXjPFTffxAwsLo=Z~5iU@Al> z6v8`9^3m_YA27L&wQG7o+wTK73#|jj)BTg$#q{N=M(f7n)}uI_Um&oqUw8TFMoV;+ z}vqj5|9YKZ44$m8RSohQ`9~o@kwM_{}RJ>JeC%`EpX?h<9I->|X zO=)Lji0d-GU1_-lRqH}y%_Z$MV(*0fv|Wuw3`U6u&*vIw_CKqKw(`SUg`A|wvEvw- zyS_Bm48leP=P$6apFU=czpbljuCm%0>}cd3AFAxCgD>0mIPE-Tgp5LjEf-*H{zt?2 z2x$y9Xy8_EB<;|*`+7(i?LderMU6FGQm&2$EtPcO0UaH*_gofxAM3lMTVFY`BZO=8 zf}Z{`wA6J|<_hW7c_-XMcJ;w*`Km1wn;YO%f(pZ^3`eEWSB^O&`kaj{&NX$K($#9l z@Y4pRcX>Ybaei>qJ1)XUbfw&B$Fp%$##SKb47Jfy;q-ppnZltdIM0KN~jktE2hh25;%%eM% zK4Ts_Y*c;F3>w$mb47PX<;nbSP2(`L%JH?D>Is(zRw?*mo6D>}dCdv;8~*A$<>F>i;|`08{PH7|K?G0SUwY#yqwc6N zF)AI-*v>#QeMC=jHmq4+J<1$Fuc=v6{n+ZvIgwVGrwj4^HePNG7xepFV7E!DMW|HU2f%kEK5U&#MMHxatK8$kSYx&-GFpO*IVoEB7 zq>Aj=g{+qYnaRs>?OVCm^(FiJXsg>%;gKDxAL^-k#_Kk)kmFFaLfDa)4J{YG4nlQ8 zwd1Gtw6Hmk@FBcdFzq=a6wFznv7Y%&CKlLGJhGai+Zd?@2|rH1?9SdWfaR9%G#bqq z&+C6FwZE@(&zJ*8nDECP{p+IC4-{9M9f6Sbci8%$dx@%=T*-is`@n~6C7YvD;AN&? zAX{L4mGKv-Y>sD=>kQGPDSxF9hofveUT}YZ&0aKo9Li=I*@ zWcddPan$$2a=f4T3hmQ1gn1t?gd~Wc7K`~$b?jpQ~eTXoevK5THr8T}+?U7X( zWw>FkGEnUE*#65i4QA7o4_={tuhu&KY@(|fN!;;w535IZ0lH?w{!ay3nS@G-xT=nd z3huB7s(Am zzt!&4T@$ZZCwXCB1>^nt_H3~zxd?t}W)8Q^uKAOp$4a|aJ^1{}R>h3^7a~}%5_c2i zLN_1og?x3&)8O5;P6{U_VRD&)g6w)FQF#P0-zP^ zY76GuR@H&`m6kDcIevQ39Uft#9J<~2KvBQLdn)J>;!4O1SA$PpGXo9nIf9~w_ z*zx8jP}SfU!>Gd|GZr1%79F~ID=;{{z7NKD4)OiE45D)L44{wTvvUK2#Gh$(ZSA_{ zN8UYIeD@?fW*#|IYI5kdBo$8?+GLOM>YQgm^8^9tXiEX>V}tz}LETD-9GQk}1NN(8 zWxe}F#fEV zmfpxBdZ--5BY&a@yYsLrZC9UXi8$iEUQ(EUc2L_AjOAAFCYw_%zy;AH`p32k5M#-f_0HQtC$*;Z!9S%O`rx$Gz) zr=tY#mN@efSYNRRz_}dqU=x_~8QLFe)vwe--VKj%t5hhRuNDF}W($BnB28ufmf~zgxaZ~9?H~FMffX$-FbZC|0M;j%d(UT?uO?F^YKXBE$YtN3NbN! zHGFSY=H3%?C8EH1mz#I#${a<>rvNc#I)wLij9-|90xFsq+cGYTuG&hYZytGWa5v^^ z?BvUwi{qFak@zEF-m&@8HZp??I_zl4!p5}mz|wwD)#3YK{gO}F5 zHb7p^M-Cv%sUyp05Ivr%>0L~2Z;(l(JzUwRVy}93_uJ*u%nnSFm9))f5tFfRVi{8d zEe9aS7MK^ci$Q4nP~K947;vcbrE2wi9}g#}MV?`SVYrP~QsNK~ufBf=J6k}r%}z2P z0qk*g@8T&WS7cV3_sKc>*LDV#;Kjpkgz<^PXFoLJ7&+m;D7RzTAdf|Mb zM{1^e;V~VIN2`CpCeiHv!LCb!c#ZsJGbD#)MzF39Tb}{ zF7%eI|2D}3?=}9`8ZKRx;@SJgqxb_ibJiA=-lmaA2H&lN|V5lJ|q}TxG~b}Tw4RDn+~@xR+5q^ zWUfp(5$<71xI~DuMW9-~{quAn<#Czmi*KC{M5KfWOI?grtpW4^H_6modUhTz?l#B6)e$|}b+QFN5F2EFN%KC*5wI?HFhKKG}nGaWjk`}&VYU!p& zt6$#klDc=9>dU--C4~+NYf|ybT>uB~*{wBB;7{)KWES#tJ?@Fx68nPUXjVuFv3o3w zcT>O`Qk58Ge?dtYWS*$l2oOkYx^!uVKC#9w%XvWExc60V?snnhzP07*g2X^<0Zjmw z4aRHU(l$_J<9mxN%M$~?uu>SJj+wp6|XFNa^E8V^UT!`Fwt^iS8XlZv+>-i3nmZb zks=>oS@^t~pa##6gV~<3M?R5_I9zSlDh*mrff;9ptvv_JVVvD56{hEzvfX{Tx>~gp z6j%9#(&KOt@vlhf+93p5uN`UQpD?gvfBX@s!MGJ`+_x%<8pYuI+i7UHc`)KXt$bzITu;PrqdKSqID zg`{_!+tDGePiW@1{+PDSz<{LKkdN%kG;u>iJ-EJEe1HO-rHDYI@bw~pCFaF@i>*ip zZ{Tp4xgZOS$#?*tNLSV*3^A^>ixlB7H z-mb#+N?B5qn4&B+ing_lS~Q$G(&W5P-<*poASuxD#fvGYV3@an169L0bgmPjVkKxU zJ{c^~{MN&%-nA$q%AuVBa09>%M>1X#oH1VEm9}D67}0et#J@lzdznBcv>)*c6s0@1 z5`W-LFCrhr!QOxJs(ik|p=w;){P4gcFd9&ax8{Tu$-w|MW);ERM)BMerXN=ffP{PT z;*Pi)*2AgL3c5JQzpM4?S!JsH=G5EmPYsVf)W2DrrYbRCi*i(52sXZrZCph7Ft!8Q zOjs+!TI9_h`w(R2>)d6z*E|AslGc)-+&}x%o4`nAW?vU82=HQO#^)C7U3^RD7CM`F z(DtaBPgl1`gj6mcLI~YyUwVBsW_r?CAC#c2vDjN-bmI%0fR;Idinf9A?gb4-DiB#7 zd?qwEnKHO|`C|WgyJw#~Ry@$D`q}f>2@H7PMA(hk z=;Wm!%B$l}%R;G;m0yeEzuK@t$yd|DV+U6H7#n|JYOw-KJ8UoQ$*_)Fsy^o9-3EEJ zHsk|8c$TXskoyLr`b@KxL09W$40+M`7nGQzoMVINk9^f@%?yv~F)V#GR?ptjhC|o} zK1z7ZmyhuG%#zG}FE<6w_}%E}s}LVTz=c9*;leQ*`w9ZEn%CWc2o-Gx09$%F1|)`N z>5Og{!A|LS$`Wg)<9y3U)cYUDv#@B}dpQ+DRDGOPmc5S_7yQ9Q&e3#OlWv9wjpO#r zBX(tG4Xx+e%O~)FVY-iHTaWmAG#w+u942-bacWv``aUqkDiadU%x{FdAft)>?A9F{ zpZ7hn%8SWrHuBFWS>mTS?s@D}{BM5Yf8VdA6?6(|lgVG#n|Y4us~!E$z2j@DDwSAl z*G={1o;;B$n)(HTx~N0WZH-PE8k^@XUSCjPc40mR?zLTsp0=K=RppkRkzEvnd^2yV zMnms?+)bI+(Qt~tr}mD^WQnl@LXJH!Hd1?9JI0j6qeE--LaA)~Nc^rWKX0s|>GMEQ zEuRd2$`RpkHbeH*t&_p5Tz|4f_9E3r!{(2>Ne z>wrf4nc`~Qvy!6TU!d!dRZk)ElpGrx`PTgI;&qV`YaM~z{av11{Vbmn?V1LXe7rkP zB#7j!bd#PaKX!<%Qt)QY`JAD)@`pP{Id#N0QkP<%Qr)&a2x4YXus$gQ;;^l~nc*&x z1yB&~&j~NaTYQ4E7^&GQ#3k?Lje-SHuI zyN8DE%H5Axls;C6R;M|F$d9}K0o%E@%1Gf&UW7g9g!{(jsNe}0Sf3f4C(Xn^*U?oU zzJF6Y3RNT%@j5Msz-Da_atxWyufojbr57PoC`wn_k?U-{F;+FP3uW0Jq~!8U-m|bCM-S)IGoc7Wre82x##*kynrf>P;0z=pIzdfsuh!o6-t)Q? zl>@m;m7|OGCe5+0%O&e-(bhi_m%M^w%*NA(PZ-(2;%lpj{e$VS4y^tnW2Et|Z=)A? zeTvo>&|z2))?luWOvx%+KV84o>Swv=q1Y99)ql1;5?)R|(*dF*HKa&8gxZ~{*9s*b zH0CXyn_GgO{b{K=m!c}~p|AX6D(*dgd1ByuhNW2fZQaSw|L?5F>GCQ7R1@R5Y5E!I zl#AsS<36z2*C+wuhor^dd|ZCJuySRUhRcAXfuh9L*yX2RAiv*N(=7$V6$fcklKsxw z!IM-$ffp}c)s+rLgyl@FVJM{YNo&TjixCC^r z0!;k+I;XyKF>?Ef15M%G&8ym$JWSq?#GpNqWs1xg2~T|vJ<_oB~D z6tAYP+RmLB;Q@rzmvMs-@b|f@!Tp0q^Swk@{xpGP9w8srtNjDDisk^+mYSL27n2xc zMe3dv5_aZHv*lYjOfd!xq_9)t=~K5u0c{g6$iUjqxPR3M_@Av8!|v4A+5}hYzZ?pq z&&Cr8vb)ALffE?0pEKWGBYl~Y?>7v{9&`YkIpa)&+r{7H5%{p$a1OlOKYLS)Ca*SY z-Nx>0^0FnZHjbs~7E0@*ibT>3jXjf74FQ>>LtCkG?F%Iip==ga;W5vw#A7=^97d|b zZvpp;%+Gvd;b4pM65Wc_wJc!&(^j8cAV_ua*O-J1^p{CUsK0B z#NO7pC2wt=kW6F7_s3uPYH+-D?YByRUq{)f;(zeo(?1^pjdCE3Km46Pjv}t4i54br z@aPftf~(j)QMGc){GLX?Oegwa!?5-C`k(fTHqX%|&7MIMdIvV^Xzl6z zZqUJwaG1<#cYh`%+t$_x-_ankhwwv#&}C12i915Jq8a8yH=ckL3tD4kSHGlnOeW4=4d4|eKsMjwz~${lui_$kXvX?IyV!lrRz+UKN#PJYBfz$l5nfb z2NsLh-^-p&2D$yoDAF{TGc-oXd6{v(GMs(bM;G(W+1Vl9>`nU(x_~DS^E2*2{xGoB z|2PDUviv#zFdP;B&;Yq>80gQ3rYav~6jmH5Gp*bhwF%{1gwJu(eX+Pxrru02rFkfn zEjb9Q8LsY+@hD?b-KKP^m80HNdo$(#;17NI$3EuVA-ksXrxUpP<;_1Iga5U0_CIUz zmeEv5)`Fp`nO8ny)^L_df!Xm{R8-WX{(k8jMDL^*!ir zYCz8fd2$p9(b>W3m10H zG-2Rc{?>ylzu)qVf8G7R*lS(mfI@6DO~0gw!dNM{Epq7l>wqWOs$CWEunfFdrM69_ z;)OrVDs4E|(AO0XQKh-61|oH3ejUX>pF42Qd;hgb_aE#Bx|aW6%<{h){KHTD-?MQ3 zZ|G8-=!uNr`HLLl66?3JK_I+cd*yrsvEGXhemk*#>uSFlA97^Z{^@jgkMlS^chj?a z5SnYC>cv^;_z6h!zI@6YbYqDb?|j;B;jR=5&&lx{^v(kOYKq(YQ&QY88?9?#YWM{u+yX7#k2u#Jyxr;h5Tkl>41hrz zON~#g{2mV7u1kD-1dR!{#f$i=S*O>X)oA2~EboHg{-oFKnx~j_4!_@ycak80=%Q}m zJtg8*dy{4pmXmsXtF%ttkRpw7%Ki{4+eP6T&6}Dlao6Qh&(GxLZy!bczGh!LEIC(E zDLyZ-wSD6io)Q9B>Iw1gi+|-Tiiqv{3KC^a?S9)#Ug(gyrIXy~?zyE_7ow&A?A9pC zrH+)eTK+AQ@BRznCD8&GNj{BH`?WDxxKarp^h?e+z`>5&792F%W+v68{;op)KSRWC zmGxUpX55FsAm^03eurgatrEqDB3E~T0|LuAAR}Gj3v2+&J6w2b=w;`>*`KKZTg<^4l-ar^xd2!C#;ZAy+N`KXZ8d-~Io8tXUTt`~sCX{Q{L`1DaU5 zMj3=(uY_i0fn*X^e2HsUMU?@0IH>)};u7a2t!#@Z7hb#GwYlOo!|(+ft7NtSde*w} zWo&lAX2WV8w0v4MDQT2pXwKfquph!+h=`~WF2nZ?v~GA2U-nZq*T(hbCWQ#-?)|z* zv(UB&!OQQo7X9TF)U{xKxLBJmnzxn>yKz|XP#x-2GH^Cc1F?KJ$@p@G8|HkJ7_%JMd=BxKFf`*G@NT~4(rr?yZVWN{t#39Sk0AmM4dm4tWRSjzI$^eN0E3(BA@xO&m^6& za`K`5OjQp93uZpMXcGYUTO^;WR8>`1S8;I@tC?;!3$ca{(eJ8ZOjdB{sdPtb(g|7^ ze=7KpM+Z9(I8RLE!h0?5g?})Kt@N*RVT&b@_b9D%#=o_o9<5h94$W!}n>AqR=m>v| zz^Rj=!|!_S1{I9C1Gf<@g%Wfdn!g}TzcgdUufq{RDPzY2Vt9_`=0$F_KiSd724b6URJL;rW2Jm z*tyQQ(pU*^*M1&zTTUIb0y`Mly$vtj%~ydRWZ$Ty99OSMqae5NV=yfDL8 zI2Rfruj4u-)0HJlC9#i1blX|B5{)Zh3RT^Dz;4ho=Vs^&Ah@FaVu4-Eco<@Rke8Q!$$2mvwA&E<%BO+RAmb zBvy1*shyhXra$PbDXjrP)zTKN{q?PoZn3;#kEKBz{e;cCxwpvDrhcf}O^x1HCk?(& zB5IfabsBzbOonbr#`NV|9=n#~S5!uH+tZN2cK2I03^R1@Au{G z_Q|ZQIg1~qr7NSIxq#52k{@FD$0?H{Hkud4*RIOh+okFwm7TrNQcUtHiNWgOxY(?} z5;v<3lb`r|rcXA*rQVu6t)1X&?$(`}leW917l0P#hFL*-4uV_dfwW*P$@Trm-%#W? zhzIa=XFdHo7eE`7t#AX~;A({Ux1lR%;dj0Ft;n)J^=GWahsC8y-#-#Oj|O2WcJZaA z_BtD!M>fEi)o3eRmRixv{`d~3;ejaY%$_7~Hu8If@#0(1J?+Dq`B9c4*XZv_=rUmk zJC{U(@v^DBh6P{w`Don(DRFf5Gy^YYxP5`zJ<+f!3;^2$m^^Xb2;yM}OEl_H6UV{X zkHCN~ zT0pFo&{A-~d}RB%7*{Aa>GmkJ+U0(*KeWq~=nh9( zU#;TUeT8`(H1FoNmUgJMRKlFi-as&A#pgQJET3HyQ@!M6Nq>OQT>)7)QJX-U<#ASWUr=zP#?XptDLD*T94i<_&fJ|j#0JptJftQo7Is4oszdjzy<*z=C$gQlST8o;r zcm)85E>wLe@=#QBhOoGQH_!m$7?-KS3^?l7^BaRts6!s zSejge$Yw>3O>>kN0SNK!jfk)QTwAG{$n;Jh+nZC~9JvtxZ~1%MuM?MRZoYve5fTT# z4$KO8+Q!--S}jSjy4tb7q*j~Y!(LGplO^x1*K#XNGK$>FH`Y=)QfHQ*ZEd!4svk$?;gF17(52k`O!L^Qo+L)0L;xca(r zNWs_=XK`=FzUyq~xE{H1+r4OiNoI5#IcZl->u$s?a zy?OjNesXmpy#s<<;_L;~iR~9kkF)dPr@p3e^MnDx0`IlR>wO%dFIK6I+DxZMh_QU_ zBklR{?o2&qv1fx$rr|b3hWc~ig5Uh>A_oLW1_G4WB$6k@grndM*!Lx`#j>746Slir zEr|qf^WSx7ee;^C%;6J(z*EMwWQAx6xi{!eATfLIh)4OPT|o&W<8H6rXvGwEJg&;G z-Q)@AmB0N>S@eQH6W7QN?RT^>d`ZA?2Uq0XD`SL&02iy$}Dd~CJ$ zQWS6aK2=-@Xot0>s}-#q2q4yE3)h7yEewfyP7|is&h`A?K9E|GvPgKMI)gxv6Qdd5 zBRbR3A+;gZ2N{^F_s4oI1MS~0hvrf}Hl~w7r`C+5^na{LmX&@2yX=rL=BA@MV4+6t7#N)H8G$k;UPb8Ov5#+1d4?3n8lHGD`KDK^NZ#e|j zQBH^pohgfoc|2Nw9dL7`fzvMN}ZUsbXDQUPXmn8)fRH^;E@ z_>K|saJH+-u!mfIXleGNyjk4-Djt?OtKyv_ToBs}BNqVxb0>u$3=<58KK7-hr$nU? z#?XDXF*0Mx@pxXrHUCme_YSv~@P!L@de&H!Tt^$uG#8KarG7Wed#^lCsHrI&ET5~e ze=UgQb7`)azXoVH-tUGx5O-SvA?}Qow*XmwoQ<^0cyGo;_~`EI%S>yKCK0o~z#~vN z24ng?x*!mg3yIlB-J=dwN74Z!xODgVs2^MX8|m@YmZ!n)T8r#=;=rz03pDkwsBAAB zj0#pXk#6B__o1>gwJ~JdB4YFuG0(k}L`T^R^i`a4`#DD)s0Q4}M)m@Ou$^w{ta-D0 zAx=B=5{oR#wk2pL>}u)w)wt{|s)IKC%`O52@9RDR?hwnY)_%_FTq$9FKgZGni^4OW z$xw;*S(O2w6_YdvM$&!1`-yBI69*{NXuut)QSwYbhTpQ&3cYomgl+pb}2l`#8;KvwR8UU2<62*wGE?Z1Zy)%+%sA@@4Nr8%i8Uh}Z zRt8UTl2$dE-+ZmY^3L<^L-2ID7`RvmmG3od2`|qh^>pejeHJ@v!B=lbbt(f>)iZ8#GuA+k+k#1ga~9I>J#s+w-|Ma9?t8dpLw)%a(6 zJLVT`INkOvF^pOVp<*0}gKh@m)k(V+VVwHOmOGZvnWFa!4h1?!N>0qLw!PV0%@mj- z?-^u7J8G=0E~iP-x=*M~C-o$XTwb=7#+GupCaITogpuCu-?*{x@qQ3qx(P<)&1Qz4 z_Ruk~B3vH*qBu(Tt^oXCuln-+lM!>?v$RuNZbh;YJQ9c0{;zEtSF^5?QY}n~gRVm` z0|}Z3c|%-MZrp{Y;--Pzjf_n_U4xpz*VD+A$w8}n5)=76Cwd(o8of5LDj?FuXB1E0 zo}-WQyyCevE61H>eNDg9Rvpey+!u>(mck?$0aCFFY;2$Hq1x~$xr?VZ)V=F7C(XSq z2WbvWmqp8??EAa=X<{Q8Z175hr-L+UC_%rkPrnKH~F@$pCL^=zpN)u=QilSdhM_0BO59>Q1&k_MYd47oE>!;jyxnHDA8*wK_PO1GDKMLmsn#A6b;raM zbW%emsoNqX>b)J@Plug>3P{!rvf@&V^*me1UuhtgwHf(wI0c%#s0<6*YVLr4mz^(IGXpaVTp}8&?=IcRKs2L>*nCs!LNNnX zsj!Y6_OfnP5AVYcVhR?Bg?`%X_ABqGu_XjCPXR8a-RFSo4L4MJ`I1O!f^UICBN(Qm z9LpV3vNg=9QaA0`o6}5tZJ+X^E%9Mxe_#AItZX=tT61`9d{$q9xl-!zkqm9z>X6G5 z*Aj`Td8vVp(Z~9kjL6MsF>S8mVF4+sxJy91{%gC_4^WToKExrG1->>KLt4;j+9{!i~1@}^1K)4TN`Y-wIuLM;ut1hm*SAT%o z3jnF{GPaAT@BC>}loW@GTuAF!)ibLHr8;I@^ZuH}Q-F7nFT?pp)^Kqd=(ov>FW{p6 zPfNi6sH()jIPQ`F7;mk72YjY{ca0kX`JU!y=);OHODNV5bkd&oRqsXCB}YvuuVi=Z zSkTp7$viHiE3wt{n+v6X+o(vgocy0nLDufcqK7&mMJ%kf3SdM3vw04kS#@A~Mb$h-ui@r0!P?kMjxzu?@8{c;eb+-4Y z;_*w?%T-$sTV}VuEWUbx%Ibo-g|^^jTP602k$J&~ zAM@XFUTT@J%UlFoUE{MATL|nM2faXbN^*Jg0i;HiPL*9zdqNn>`RmtN8w$_7rrAY9 zn8Nj`6Q6O@MBKQi(WC#h|Z`nPY(R{j496k)zm> zke@pLN?pp*^-jo0W7kK}P0(3{?q#vhJ$F&X8H(Oi!X$w*r#zIC)|b_ZxRVW`i5|Bq zom@d1|BZ~j1;N|b*wgm!ay%gWF4w^mx+sJSd2e&DdP#1`qc+Y1n-+bOw@t0o_QB3H z`e{S(oVsSl4Z|D@)34^tyjWUSwFto#<4}{9x@l*Yrsy9ah3YRG1Y)ZqK3N>2VCq=L zdt=XbckaVgzlvUI_RB3Bhh&nV?DJLf+@Fmrx|eSV3X-Teh_CndtZU-O8yK|2%g?E4 zkURN-D1l1s6#^-Q{|t_RV_kBNLAFs6Q%rL;ioGAf{gTXddZG-6T zj*BnY1DOv|@r6D^`3K#xcx6t{+7Js@WjRX4OGY-@^&)$vl`<#T4UMU?hZ{#Lymyxa zm^;NAm#BQkjOm~ph3Kb^)Y>5|V^_EPvqwuyYCkU+u^hh>q)@QJ3SzRF&E@4wdlOX@ z)tP>Yhm>CajDCR(@jK?S*`F1YT=rJrPpG+gBtF%P#@Vrx_+ozZ?H!Z-%oh~q#>p54 zwWqr!h}E8~+h|SXqki%qpt%>QfFq^m;CTfH2(7eoT`+0%ifM{jv&usTel3mrxo@%6 zrNp+4VP~E5x?wWa!_D!7RvYIs z72_I2MTkke;ghSW38etpmWJwy@9S~;C1K^yi)sbPqiIvF<VT;@QH%eq&VBGl&a z4CQ1IzCGZSnnP~W>de#bQ}W85asAKg7Wigt=6(v4uFv1Z8=C?-cQGiVaR`i|YX^c` ziWXieYMaRKg?PBxi|4L=dLC;i^KibTK8XB$3;;`fg(w)ZTl(FYFqiz? zrL6>vr6k@{SxP-$!jHBvkA1^A67FN->K_QhCB=XP&KH0fWvf$%+IBr|-kc4d#c~M| zu?#Kvk^Jd-(81U2$k4(1*oGoe)iz9%)ZdGT`}Qz|vcpA*LiLyvDMUC=;IK1iZxz|v zwg9;JQqvnvnKUSOY23?X@4nU7WNuQI$r3kEHWQka6HoyoI(uai8TkC-qTyMp&)%SY zT3GJ=P@iS>n}nqPh6jEU6ey0BsIv8nK+cW8ZNfN%Q5QESU#Ca1ADD<7+?cDXvZD&~ zP6}6~2Hq1%{j9q24{T#poYH&)3)E!o3x1h~fJ^|h(|#lX`LRaBil=}uDC$b`cvvz` zQ}6$TW+(o6TrO|xUOA0%(HE*Ec+~Bgv0l3B?AUF|7u#^GrO6nR$J!VBNlnkQF0Aw$ zg|EqsEjQmz#s_psawe5PW(XKiE1s@q54~4%s;5iakqh3U>0AwgJai@##%^>8p~o#FUga&_GN+2_USYQ`80J_ zGK8BsY@Eg)4-)^`@mgD-5F~dQ0p|XXsagGBJIBDh&(W;tl~$8!9u@h1B^fh@OWdaa zBoUxdNUHfRfv@=fgdDqR4AXN%U&;V-*%hd<%RGSAF=>1XkZhuU2f15YFC85Nz5(s4 zaz7Qq|2xS2r(+p@DrD++tmaQw^!?{Te#hJXbnLGS{_VZ|dZYk3n zt}Xmc7Yq|HVEd=&f#?zl1(rvWp0_lu&X<+Po6H1pT}#Q8fIbK`JYzVVkqqY8He|XY zzOZHn;Mt{WGRqfP`}fPTJAQ!tnlZ01jVY}~7OUSIUba7JZ1MtyqR z+5c>z70^|@d^WC6AlC!Sn5x-Um(Od>SalJX=HSI*kk(ZuNbRzwfO^9XnR;Gq5nEY1 z)-@>w%!V}IfA#k4=(%B?x$mHX4^=4q$PCryRH zTkkVU62<*XsmI1$qV%OEUrv4JnEk%9WBOd1e4OuLpFCrN^PS+wOmD_(Y7+yE4qhC@ zV53Z@06D}#h?7)|-VBs~Im1b(YV6>yHM#YYx_xyD7wTb&>@HlQEE*VG)zJT z;im|f{4BWSZIoh9z{Qi=dI8r{McW;+q6+~h2H4vKW)sOp1Hf2*G^WML4bF!Q8V%*g z<^PIOTy^1rshUJ`HrE$+GJ*WxL?TLk01-*u>boSwrp4GTmPd&fmyBom%c2Utnl{(?$qq;YQ~AK0 z$|no@VsQeiGki<9^}VD15?Cmef>JOXr<8f>dx3W`4t4IyOl5VZYjUo_Yix8CKLQ9j zq4nVv6bo|R@L|&<-G(5snz%!_V#rh4y%%>(Z+LVZ43isiA&t_+gGMsSj9|lga?D!x zWJ96TV}lK!-X@rsX)n$NEYtOiE4|3y4bvo>Jr{;Y&{6bNq50o*o#?r2hKR9ahh>Lc zJQL-2N$V@Ja2GewjL(EpYGeg-KhF=7HU&E#)58CdE5(`dSDw>^v9M z@$j)k?)B}Tm7It404eM00oD?@w>mOEK;QC_(dZp3wmW&D`%cMg_wg!Z8OFbk~ml>>(Lr#4iTW$N4u4Ue}wHb@$AE1b)jfmBI=$G*6 zW~2skPQj+aq0pe$?WTP1ss2Jv%H7vP{Y*@o(3s>@yLU~z_e&~IjY5`*Ztl3zlOTyA z#fb7zPCE1y)5p5b-@f-*8SUd0nguJ^s+J64DVyK26kBodkAC^RBKQF+{G#{VH`vw% zFTxme+w)!GsR(nbYl?%5rP!0j4I1ILZx$@ZgSG;MIP>7OrBDW1r~-V?US<+4VJ9X? z5n`GsO2m5Nu|HN$mz$}}LmOtKfCueoH1oq2KAJnALoL0S*?OitO&a1FY zRxnSGIDYxmTQx@O7_xqnNBwZSZ$~csVeNw9@t)~{Zrn6%pnUBiCFhC`*Xk&up9ET* z^N}oGhHTt1i1aVS3VebA0wbT~TDwjA3C}#hayQKL@zDNekG?H^rFa6Y&rB~zuM%Kd zlvyjEzI6fXScdV$kbg3cpCk8k17)*(6L zs@MJ0#af8dNgO5iK+bLeJUSz2px%C!wP46GYe(uR_Ce)PQXerS^Jw`qhC%@>{!#`Jq zf$K!X1mow2MFz0W%^i9f!^p`=m`f;c4t-KxOR&4{<^X*#9d|Oao>$9Z58sKoB{_5^ zFg`a-Qg|jg{pRkP_dMp2GuE}KV=*@ecPa~vk|A*+%Aao+M+2{R@BYn$#-E=j7e}iy_jJ3R#QxvDUo-s z?iRx6t{lX@zWh~V>XJjyUWblC=x)5*@IYTz3RIzqD>ftS#1F9 zn09^D*3aNJJr_*&O!L*f-iAK+a&0(KHCPz=!b+j5>g>oJhd=4Aex4)I2!TJGr`q&BxHH~SS6YImyh>h;t z#D8VX96U(;rp`~I;2(&e{9TB&Tv{J#d?FYwCb-gGb+&|fD?aYgd~zoFDsVWXS$Xek zzgl$5_wu%u0yxkMTliL>^m`WZ1G@HCrBe6S-l#Suj?rS$Ax15p?dcq2^G!7IKj;(X$9`wZfW>cJCCTIb6VHrM9N zNd#dU^aTOJa&624S4?A2X6_aSC+LaVLf#ymRJJl?cjQ=cU6@zO21((*J1YfNKzn>qP69#Fgz>r4x7GNFhKOeRj@Gj9)H5-0^yE=HzxPEMG(*$OL(RH;G{OaOzD zgk1|i_ciXaO4sOZeW)N}KkTIw&_r-4f@*n-^^&Zw=T>o!SBA~A9_!vo`SS6JM*5z~ z122Y54R@5-|r>0a|Cq^fWDvq zxzhi@|Liu#hlEW)6c|u$d!Ksv?6_6&>hX;6@xTwz_uEDaQq(YGG@vL05byM+OEY+` z^nQRke9OT9B$G7tx7F9b$jSakZ{`2mxauF|qd$jJF*i1wXTa&LC{+V}OO$;qSWCSV zeb3NIhX6t?^gUjUCQlUr7L$y-0&=xT(qH?2J(pi?;@9`^t1tM~NB%ZGU;=(xZ(U|C z9eppcX5zF9ZOA9Mf(B`dba2v_LlHhN`f*ba&q34U0BX4cjeR~{Yrf@sICF)03D*04 zekBPw@%$Zz1;iR=^^#7EPnKpau11gAMo*$bJ|EQ`9G-q(s84t4=fijset#+t-Ge?LvOHW%rderP;c$`jd zfOdcdJF~5(|2B@$-JE>J>?RIhT=&F-ClNie^D*7@EJ+6O-MrR<_N7n9 zIc9N~1@C_;vc?GIMyP2dV!g&-^8*w5>->(fDZ?6fQQ#Zkuc=O{`DUR*zb4y9me6St zmD~4am-KD!nxBfztF!{6?en1W`24OJilnOCTRrU_Yq0^@F^=-rB7(v-G#3<=TgUln z%Gh26l^t;AxwD3~9V*a}xkxMr5UBrdigfMI;w8U?O8#!H@os5PN%BJFf?^N%0$%CdD?d-9c?1&EE`gUQywb?oErMiW9ZX z4ENn;D%ffOsG5K^B)Iv+IhEO(;&qzc)LY6oD}HnbZ_2-ynZYHyCCRsw&NFhhB^ajm zATpO18}Nj#^KIT~Ep`q}&2c~|0M3vtvG9`p<+E~9-@UbB-kEcE&AI_s7 zPw~G7pQ`hwfm;a4-V;BQJ|P&jIV5fo3OrA$NHGYjxG}Z0`q$^qh z(!>^*eeIx^S3~W>6PWjXCB1zGW5uWM^!LUSzjTxA5$r%hRsFB{xdZu&OQ-Ed2ZtA0hKYSz1R-|Hx=LwEzryxm?t=Jtp|vS%MQuigdS^ zpLS{GDnG_cv|t(t20v{~866o+(wQAF^Lwash;h75hPb<9bPuT=STG|Ob{xSv0liZM zzJ+6#=bw6rHyCqQA*sYx8+p*(3ht#&sEaS? zbtvMPH+=T7r}V^@l)@3jj1d0K5Ux!CRl9nLr#i*W&$Jp5#ULw!4MU4M`y;h3-8nSnoIcAfT`3PJ=jsgZ3!%15g1|l0D+Gs8bjskMtv@|bFkK0cq zbjhVT0w|#c8-}F{j0~3A%wpclW^iMaGX@`5?7pw(!I@ zDwKbYjerpa(l?mLWlvwTDyf^CdNSB+AXB4JG3@Cf6Vna(1yb@bUsnNiv2gL?1Eh9n z5}c(iYo&s;6JmiS!h!NA=ilkP%z$GPsQyu^@a# z&i8#u-IFw?4vulxXZRz8J~TAC)z#P6wIwjge;DcQFm4U_dgfFE60iHmVk;ktZJJ(% z6^xdEwcmc#ffacaG{l;Q^U>)peA1*}n>q(Fa?!&CbrCCNW*P?HJXJ#ZE?QM%lUHja z8*~8{V7`O0Y3#ttlSCm*M$BnGo?BKD_a@Ayq5Oq&+jX}|NZ`{NsjT;3cZiTYB*LOV%`rB<4(mn5E(zmtzfRu|8r)#3^-uW7)5kkW;5{56RUqhwa@%*IwinIM+i7i zdd6+(nY<8{;{&`suRXr4bcb#BR@RFzPDTJr3|HwC&2Eg6k*r-iY_CHvee{06cZiNW zICL|!PZ@EDT0Gu_kvEs1EH&$A_-G2M+G{z@5_lWk*tK?IKgSVORy+eZ%ooTvSHWH} z7jT}dyV(j!HKKs*yp2+x&C91dv=7k*ui3>E`OF!dUwg|*4|`g65J>kn$@OO3 z)<(#_Z8|BlI7w0O`TVg4>7DubVNTow_}yNI#7|i4TSgY;J5B9jwfa#`MonIG7uop= z#x!X*K}N~6_ds4Z3tko)Z)tzr)-xQcb93FLXaC?DE&tWo!h$hC{Bi!&RC0WEljIX9 z?~3tFFl4#a@e`QZH>g7FB|*k%Fj}DsK2`B5__>1ZA;?^9$d*nq&c`nO$oP88b(fV^ zjxNu(_78NTT0jM3UcQPS#ZGcq>xZz%@w;NQv_HW2b4{erZj(l0_t-O_%E25lB&ZnG zDVB*)@To15>thx2?HNa1MOp-3{X}E9)f=3( zCnK&+!L|?9%#Q|2z2j@kA$w+DFM^&bq3462KQHs?w~^FY@+rFr{@dMejuVvSHUeR{F4CS7(evS$>b&+FCO?53V@ZaOh6 z24P5FE`z9gHQJGap~p&2S0vsln$6AAnEh;X78#41P)3}&VN+wr>#7C9l$2YV)Jk^j zQz~ZPyQK`7;FkSS_Pq#7TeRk~{;*H~i%^J4y|s_hRBgt%e}K8)y8vZEZ8H)PEuNqj zB1DWbg6EAhMEhCr)kIGwz2Hoo^!s_^ZFR%v(c=Sz1;iZKv*{S?yP}%P2W>d$omwD# zUFs5smNqL{)Zea`eZfEPM#9n)3_Z(R$ET}b+aEZqa?;*Kd#y%^N+orr`i8pTLC9aj z6)DT}6fH?C-Fa`|NSp6db{+uPby>agrks6WJhT|f1tu;EY2`Fsce!}KY z^YHR2G#V{3o>y>kb7E5D9B=esjvd)>eZ^eOE6ki8KWt465uG1#xvf=s!saiA>1Ki0 zV7QK!u1tIWMjaKDV?9GWoE&j4+B}`Zi^Ci7`XLVgj;kkKW<{CJsmOspK)ARuuv|g; zO#sHyn-EqRx5Ya7^r19oNJRP*a>MSo%Da0e9Lo&E?nix#Jkc-)vYpfO9#`E%t-}r7 z9*=DeQQF({$0d3dJz1fin_qqFTGWaG@Q%f&YCVtBYK0Sz7YT9VBd#JRF9SrWlN72F zG@2N!pf4x1dqf!e6j~IQ6&Zk@vWvY)nl+)g(|me{f=B$|9dtHqcr@#_N6Ol@xh6>e z=9J6%K@S5xDn}(o0m&1_&!zX7;)n#Vxv?P1IJSW^vRjBPvn5M!?fzER=ROzK-2H&? zdu8}tzZO-0Vv78%{#ccrD!*?x#pofIN~rOkVS6Gb#lM8XChH2ijs{p=@Nd^PZV0SK zCl{v%XIU2ODiv%K?r97h+0Ry$B)A9 zqNhc4KR+i3hw*o)-BIs8#}576d_@rdLVcg)WPV7L}9&gyLo#D87v zV+mNN7A8eDQkq0Yds17%0D4)LuZeNN{B%0#aqhP!t&o@WDS7&3t))+^-VcD{8!{Edm25P5ey+n$=l>qM|F!R5 zHjdxI*Z<|uB@lDcIj`9Gr>1+M@dI?U$+OAu14JQ3YleP^C~aek>6y8<@3GP-SoxS` zuY0!NS>rAp}6>ih6y2v?mE}6clWCS1A`HWpXUtQ(07InFb!1l zVDSw8;Pgnxpr|(Caaf;14CcBR;;e$_JpttJ`E#v*0c8I2w*}+>%2fKuV@BX zzjkawHhMy7(r!2B_S}3;5MH2O0rw)_Ms`$mmnA@1`2mwJy|)l}-rd1*)AC zHSbw?$CQ4nhyx%;R?t_+UnCLa2Y|lKJz!b*0hu-<1%MLt+ppv+z?ZVUzG4!mS8}me z4J++&#%}1_JI^@Q+EIE8RIScL=<7g8cP(7<&!wnU8jlhI!Oge6OH|9^pOxkK6bszYyG3-RsoUxqxc1qM_w*mSuf4HffKNoUL zLxx6NES+JWuLfT!crPJT%I^KURm=HX%^v@%*l+Ls|MoZ5dbCmyR?%X)Oo!d;`Ocx+ zr5GmXaQ6vLly0BAYHzee?o381VuT4-L8}nH>Sc9MqI6=I^39^n^IpatLml>7Psyuj_Xf_Kj&)e zJHp}}ruhu*?O?MqC3wgX={b~nH?b4VZ8l~$o%`kbS&QMYJNEupDW=i5y+EG!`~CYL zX$mnHttv8jB(1zzySrbzM;|C!#q#O$E20)xZtrSeTj5--b~7QZNs_Sj8qhRm5Bpx4 z!EqAf?WQ7MLqIaBJH4dyM*cz~tEAWEmC|r&z1k2Di5TfMR_z-PK1-VjfJ}%Atttp7 z+PD|CeI#2@KIaaHuMMODi0et4-4>chMKo~YxG=3)skg(bLKs@3YqAACK;^D0tbfxM?h`z#h9_c))KAQ%VLye*ZE~u z0L2J0;cM*#BX^QL> zwK4%;aeA89o3cJx)N5IFbrG)KF(LeX&hT`SWvT{Dr|xBD$udv;`AZ7ZyFNo{vZK4N zTJ`d+Z zj;tosrgLiD`*XJtg3aLXC1v*)H(yZLlvQg*FTmqCd^wYFPlAqa9Cx7y8{*z$eiPy7 zU|3bIylk{fydtx15%eg}BI+)R=vlNgOL(6uTYq?1$r;TTVwme5&9Y8&TEZUfv+CpHd4^5@AGWzk)ZRgA*C4P)QlLckpbtMJ#RY} z8Q536%JjSm>Gbz(3@c2mnIO}17{q*IeD~o`6wU5q48KTQP3Lvzd5nxXc|-)%V7vD@s{2u1gy^18X8nUEc%ZC(+U$;mZp)lioDKe^A6Ak~a zM4Ra5SmTwRE6F+buDd@#;$?`~U~lkle5RPr6VSXl$)%`f9o;VPP_%p=} zMV*#wQn1&=rpNym2sQlX^`fZ$2h>K{_8+mB|0Z7J@5-tF3+Ia!!Olx(1gPq(?(>4H z7@LdKHmV8VH9`331<`iaxp>KU?i(mxhLeMh(U4WHZ-9cwQk(#wgmS^O`Y+;q|8AFY z@BOvb{$)XQY!7>Y3;FSPq|wepKzZ8o^*DTC?HV6Kda#Y(E`-EQt;@ z2Vl1Vn)92AqSWoJ!?4V4&Fj+7gqW~=_O3h7KR`ji@L{bbgZk-X9-<`~2+O-_>$qN; zQ83dDUU`dfgO|aY6SkoW_q+LFTdj8OJB2?$W7c}k7^~;)ol)4@(@Pp6<#}LB7Z$)$ z!TE(OgeUJK4x`r_W_w;n+3*Vs!xaq)uYDliTJxnPhbYVGO<9!J0jtj>t;A^KRRvSQ zVn0&*!tBrba3O{6NNgQNI!sqR&3`_{a2_O3U%5212ARKf=a}WgcDS+=7 zlWlwU<+#xWbua3DRb=J@+dCp^2$*e6FQjPb+TAuiOwxC^4Dx1nFIK{8A_#7d>Q2>M zaClSCNVO<>d@$bQ5*gMt3;$Nb=@jzXh=J?lE(XX?d;_=uxIHM7uDNEqG!>i-pW*i+ zcx4j##-cCdM1`H{42L+YQm%nEKCv4=ss=5LXfnnF3sOfX!52()42>6U9$LqokXJuk zdVch|x$P6TfJYn&c1CQ+0NHkpDt7voFTOJ!lM=!%0pj2|=1#Bj@Kf_0Yob}!>PsXJ zCK0!*$xvGjEL5>EWj3xS-(J1io}r^4q;~6FoMD#ryZ5e~ z6yoAGz%5gtUV@2&0N&%brjvqPnS3e%6*~l99CLGh?$|K?TloTyzHe1H*X6}#Uar^^ zZ21r(7E@LX$#gmS(%knx5YDBUKO?mhAL!?AmH9mj`B%;IU;CY`12xm3hWdOUKbbv! z@SSFlL?O4nVd84I{S=v;p<~b~atjEgIWu@36uu{-oN2??Lxl^$E_y{K-5hlV(tKU; z6DySpXOpb4v}qs#1bOouPbwSmzuLxu9XD6BgUdOO=dQ=WariINXj0S#r0zn*8KYSu z?!8WfOuUBl#|H@Y=Z#4*a4rDe@=8lR`Vd)Gz)7lh<_D<$34yHKo1wFeNLQmL38}i2 z2MeOI7?v`NU+LdiBJ|}Z4o+rwQgih{Z6a~sA~^4M+NIjp@M1}y#BdI$h-HZm&Z`X8 zho9)(!yJHI5q+y$ystuBUQ$?_u6<-vxy8k7Jd0c!DP>N-7y3+r8?Z`L6(z^ZOkV@6 zAc9tdcJro>t#{2PTnJc4rZ(OTR8~*EW3KGL!l?Qn+xNB+gz-62yAAkAz!hjfm^tjW zn$i=mJnF=^sgIHuG;)cX<>T&rb-4FxL|NPa@OW5Qf-Gl8Z70?}N2fkjr;KTm92c@; zs8EVh;JztEMHfN{f!!KBjIA+k)e8r00={<^#H+2 z4uc&@Y2Wh3sql{h-ORuCKL5Sn$OcfOoiO{T?=N$09x2&;86}vi9j*q9D-Grv-1Eay zbluqLsC4B&f4na*Zb6FC>(0}0VuT-|7RzJjVyLLwaAA0Sxe_^VFvO(u?qe8JlqQqR z)2A}3*w5osz%r!&;K2Qd^Zrw#MNjU>1ctlbGu3YLkLCd|1n|LJ?lWPAf3(1srTLX(lm8EJGP`79QBOy|b^ZGm1j~$vIg*&vtFnU>!!*3N z3CR{>>-_1sWI=s;NjYIPirYMcgxtg6?!C7exz%rrG1k%v!8UPLkO$e}?4ca@to3dIvk6aZ{--@%&QyTRq*C> zyk}u$hDH4LG0$5W=ovw8f7dAGRwf>zY+(9(ncnj#%EY(>#n8-G!OBlyZ>Cr+`sKf7 z#kCsLbDfW;zvS`e$Q#Q|^ZX9>cJ|!Ph(D?X*3KmjpTP@u!^cW{1^Q>;x+uen_q%&s z+{{fLD+-j)ZsNM*9crVdtjc9EgprX|$I==ci!G zyjMDF@eVYavy>dP*EIaU%)Ikt$@&Z5;3^jtN_RzR-8B8_UFYLEU+xzUPNYu+4~ORT zz3>@0DLOZ;xF%B93Q(=ft)l7^-!t4)>nH5^qn>z6^oUP2hrw`$(hlPUdhJMWblaP& zzpo0w_~!dpMvW#0?wehz!^N}G#ju5KYED4i$^r|i;6mz9zS_h%mn2?qV%h4DNrmdq z+c|}CMHhUq$)uHFjv1O@o>c$Z>vmBgYARlORNNTDO`;SjC4?1l4|DP_y{+FbvLaxx z$AtoAR$O)_aog)bSoHU&G~3V(Z}n^+E{N+E^w?7U8aS(b66` zav2Y(SxiyfjZ|u%MxTLz$>l#gSpU&K`)E%eKJ(`LrM?kjxNMgq^)#gHZcAqW-UF$1 zmIXh0Yhn!5M(>{M_<^yFPBVk>cD%C>&^0w)$%!jj zHf>X(b%%4h`XEaz`asni6+gL1|7>=yv$raVonLcBeOR0Y1 zW`F3ResX*-okSW3tzo>_SaxKzqWh7{2PqA0f=uTyt@;IGo zQygA$-eI1ZKMll=Hfu%fLTeV;9?(0IV(xM&xko(-<_h$+iSVAcdsuO8e<)n;Ny-gB zfxrI9bIjoO1ly?zeKGml(@Ab8{*lv&QQG*YWJ8cvvIuTkGGW>WOl6QS|DPLU|GMD8 z*>S`b${6v-Ub)%eG+a#1eg$%;)_Tv5CnklIS0Bb0=aH8w6U`^}tb((fbEr6nbBOe| z6m`~L4OSi@Xm0Tvk_3Qdn8$N>I z3@C#poiTfRW9i8x?TK0%@B6EAz4dc(`#WY&-lzH7ecHO);2^NjxcT}%NL>4`K9=&E z0MJI%nx4LP+QpY(+f}>%G|bF8U)~nB@3DsUUMqZkuY-|yaH!#?Q}m;EOH|I%Il4yr z*4jtgu;3htc*5usP@CX%C(LTVIuIlqTV0dsI^1w4jPVwc`d^s6b6|F6@=ieQb0ta& zy7_bF=9R3Sz>dX9078bP#;1VrN)W?O?MCyz z*$|5FEv@oV5id#6Dj|`LF;MmKP4AMy9Uz4CbAB4&Li~;WRps1%1_tt*SG8qQ*8yo7 zh95uaerEq;`0)R}obG>Q=~U7!Y1NK5xN00UlSx)PZTsD;Om<`J4s~E3p8f#2g@Yvr zJE7L=ya!plyEvaJoXXAmB=8aaF=$Dr6?;nSts?6rFX*TAs>1@us;XO`KFdn8W4ZDl z*KQqksqM$S`|u*s#xN$x`3Hz<%lLW))WaL#9BuGiL#-+yIwwi`RT>p)lw@JJfok)C zfpWFu+TAx)F9f0U8_gHKbZBD)Lzr_r5vsbEe34Uy+}-KL6xSjRWtp)`^T!}o8ff;@ zw?f_}E#%CEo^STANKrJ1{?0L*u`N%9Mu_9mZc>%;yoxXL$ycUNmW>`n2?4U5LE@b` z9a-0|$k6TWif*Nq&oK5}5rH1OCoHKE0izV7i3p_x!yUr-UJ zu;8il-7+Hk0Kmkrw9MHoJ^Eb36G6|}UW>w-upt&(6swO@(!g20;Y6Q2gRg;c2uZ`* z?Gtf^x75p`s*z1R0UyyClPwC?PA+gI1!pCZggHY5Q53xV1Egℑ`{(r%42j=vai zPrW%`x!uH7{sT1Z-5&sbf*kFw2xuont9h58Vw90Sd3MgMK=O{w4_ZY|!>o_OU0cH)@EBWPCl8`Z6L3?DLU zBZldgN8p@Cv+m*7Yug_gE~tnh81LK_B>6mnrK&uo4RS3$FS9d0$g68UI-JRG#XTS9 z)+#GItvWoKs3@9t@7n1Tar5KXAH@fhmEjRyxYSe35{51H5e%OtTq(7D{L@{php@Ks zwjmFq(Q=3ict|i*8XhsnL#*R&x~k^IB`2#Sa)mR^4x}pCbGWdk$M<-d3l%?XI@nTQ zqykz<2TQU3=iSgo{>!d!UWWzHvAdF4XWig>5Q& zCDr%8n;{>#+xdv~27YM2e`Y{qUO?I4{^vOt@3OOn`T)q6)^8g!D^m7WKKAk+eN zw5o!=op~3@5O|zY;e2p&{%faGflSVRWmMjSvo8zV>K}X76cTQ|4*Y-Ed+)HOwsl`P z2#QEYdhaMzrAd*h6p^a*CLkpugen07L6F{2KtVuy@4eU1oAh2nlb%pRi(l5hXYak% z+Gp=`&pO{Z&%NvV2Tz_bGc#t!9P=Gxyzj5%m9Vd8q2iHTD{3il;xO@S|4U`tY?~hd zg2`vKRb?@4Vgpmjy%ND`aa!83c)YTvDSW3sModto`pz!Gxig_Cy%78XBfnx>Gae#5 zY?z;u8mH8v9h(}W9XS91u;Vu~Z0C(NQG$teO)0X{Gtb&PH_2?7nm&7vmB=xNL5d_B zOLI);cZZ%239Y>^<`VE6)x`++C5+l#r`N_ww?X4$)I_+7&Qmb1+;C{80G9H~BQ_)C z^{P^*WW0Eo$n7o*hMKav2>IJGx$$I=@d&cy2ByZE?xWrzvs-7Ve2(Z+NJO%WFXigu z-1(2LIBA&Ft9`#&d)lg4Ih2)JbbPq~aPI99_U4t_vbqmM#o5iha1#nlxKQ%CVRg8! zI`by7F||$Ut7YxNJq)?QJPQeGXTnBp$dn<5f-4p)^)`#i;}R|7XbM-~+bM>ClPndm z!vb(3h-8JeTF1*P8il=%F?vL-7NME0;PdbGP?PL_kwpqLxBSw>ie@TPqwi82DW>tN zB}p!(6JCMy3-*U2d@N}^ zF}j51#CSny`=rI=!@0_k)XPRsf$gad$eHRrBL_zNa4|0oLjAu z^}F2_x8P9%8alwcC}n2a^H2p87Q#!O($%mMUexyUO59Bpfq z2#=J-R&Sl!o!Qn+df|g-28X~hzAqi|7@E+{yIP5H=5#A^0#&|Mw3R~fWi|&Ki>}LJrp4DOl zbeeY=mJ%1~rTq$pdW4eH4pqlE8r6xmZ=dPR^eW=%YSftT1MF6~HLN#Qiic-J0@F~= zs)34#tM{~){x!1{jTI^cN{=W=0MguR4PJ=8R@2*W?A|~uvJ*?q1X65;occhHApZMx zcT-$-v6#tG6TGRK`pvDZr~bV3Kz(MZ~((=1#rP^ z3JBvb-}FGWjSWzdgOlZkzWm%s-8lK4BW-GJ`a9R^@i4$dHm)lt3~@rv!{B=WcLuQc zqoqynr15TLzjfgIY}y!ITM*BlO4ji&a!nYOsVA9WX&7cZl29Fash@6WI9rg+Fj1J? z-a)+1Ej~Nb8z1-TnyiDRZum#+j?WJP8w)}U)uzW|(C-z6;u|7Tsu(F&%&K$25G$cW%Q=@WWZPl@($6r}JaVx0(rNgPYhw4Ygt3-BFP&)& zU3Q9+3V!kKeUY{_>g$IMSxW+Cjo#=FK#9E&Lg;8@ zC7Pc@+msWoa^w^jMoxJl9?y&W4BONQvnk>LM=tM2*z+|!w+5vN?<}w(;$qF?@gLPp zWBd-J-aLqpHst5hDoTyomH6bCC|>OYF_jKIW)LAJ45rdVN+lI_!PP}(PerP-=94Mq z;41@o_=af(mIR=fMX>JpM-ANmCK8lsp$!@8)~-;{uD#2EvvK{{s$wX(s##@)PT~fq z%pL-g?#RS1bIb8){NJx){JyI35ARJLAvq^IhEdtAO?MC{@0{V(N5Fm6e#&wK7 z*~dxn7*kS22(QexuUv7p+GcM~&LlNQz8Z`88n4lK*d-J*_khmw2S7PX7VAOu z)6(9MlcB(2h&qBZ(gHEKh#9#yk*bX`LKDgGIX$T_r+my&9c5S9yVp2pG`cFfMQLt* znH+ALnKGg)CBvJR`z#W^rpQdW9r-%hN9x7sB{qDj!OS|VXSpQQ8Xh|D-k`ib>wPe> z)W3Zb117dPeEPJipTG>qHfsL5V<_Sfp0xO0s2N9jIk4K40QKDH8|7xv?o*R_qgfbX zslSd^Z_LeuXQ{V7_yTC8L74HvgQ{NUFXu>ZwqCZQq)w|xWQM=jWP_2ZxLZn=v9rGd z$W>8H+0iwf8f;+paDAO7oXGi!vU96q$|N}XxAn)>DGI?t8)e)EJfFEj;LFeWC1E1r ztmXHynLL`h(|jcBmqG725tZ3N9L*&WgQq%^uC^1dQ$s?cR8Epa{qK)M?XkqLeX*#E zHGjwY`rq{m;t56h<)Ye^5V2PqJ{8d20Z$thLuYnP{!?<~?0cypiI+aN=}q4pYK&NKB&u{I%62rjKX!jwHm|a7IH!88KBIVzr;hvM3p$d! zq_3s0k+>mz7~10x06AC)gS&aNHRC#FfHSG7=zPK_MG^QwnS9Y2_~|oKFYuYANP3lr zmoBCyYp!-CDpkap5EP}r5!hY~x5$OIi4u;HnzUx{GS&OiOlXp8;g|4>D2nfDX})40 zTqJ}@BPZYUNM%(&ipA^mV$k= zDW;;Yk4Tr9bP1O*8oyM2Ep&m=^xHZWPuJi71w%&p&^&nILAQ}2S^U(~eXi?uDbN34 zH~;l^WadJ<#kbM5(r91X9-jY9w&rj8=l`@&`yVOs{mHU5>(H5MgDQ+mwy~&jV^ZNR zucUmMx?~7-Y=A`fjJ$u3^c_in@@B}nc}+0`>b4TfFH?V$cC5UeF!m}rEnq%Yug{Wu zfVS$`v82qW1)%g@cD<>XLQO0CPS@MWxOP>A3`@R90fb`5V$gy#;`&fWB%uu@Ra{+W zKXfXN&}~g^f>o3LgYh6ibV0@qN3x>lbo!yI#4@OI_%5^sN?Zx0Kv^i*d1{p?%}6{Z za>I?*oC_MCD^TiGxRnu=L9E#?ix8C?<%1|@qijtkAkvm_36v{U^=Oe6PDM@lgtBTb zDOI@rE%X`wgD549+F~UnD8TO0^s6!0d;R24%`oY`(Ug&JVgIu*1_QC@QBF5xyV+RY z;e_;(!X3nikEB@;iBpEDpK63nwJpuTbV|ZT_}e8=?Jf z3fHuJAh@uI0s)AM>Zw^`#4WaEB54wCMG8JfX{Jrv2)@=epnH9K>g<)v2Pd7sN&Pqk zHp}vnJBA3ZS9OE<3GUABQrB>US%&TDd0eyeO3v3n?dHY=>B^^Zc{+gPXPO$=6C3?Hrj_x*!b}t#u(;1=(g7KhL{Ype5yzEg*ZS=S zHlq~m-UiV1>vx-oTGc^m(Id2rewW@Z0Uu-=N6aSpTxtt;vubL_MkGFyu}Q|TdL?to zb$nu4??Pfvj=41Zm@o}rDvA`RgZ+$3M>fXjwCyPIzcD7BRNL*%YU9+?pshj&4tlm~ z;3dbP>A=>ys&r4Mne}Sh?8VpHMOBH{$5dX`1%0-1_odb!JTcM!lGDRYQPRiQN7dI< zjczg>57oZcFblkbn&z%IF<*T}-vj!=k~u~trPEC?j$-cN?4Vb{6B z5XL&V!qYtvPD*k+C!4!&u=9g8bbOH?#g|7YvD`8lNPvi8t@V^6wz=e zUesz7PB@tA>J3{tWc7M_z51gJq~VUX#&fU}v6dovEP=eqWph3>u{sUcz}I^VU5v)a zYY4VT(!im?f)IOz{ku$~scEnp6x(2nO>Ll^-2sm;y1VboV8N_~6}!8Uh+zT@!LiEzHwbWUFv6HhS7~qN89*H%_E+Pa=}pi?jL0r8Ef6 zbPJrWHs*@YAm8TUv8h?{p!51#$d1&Lk&0L1aZfDD&E|(gV2Jn@CSov;c6Ex}tuE<8*^B+NeGtZxiZ*7G^q598=gku) z%@nu?D4O{4(6n3Q+EnJU?FaI2SnC9r{;d*JsRZ0V0Pz?=?GU(me-w)B^;6L{t>g~F z5|G6>lZ;gK0A3-k1D+ooGo9^IU7dr_Vv4{1bR~UsA8p@lTt%$dkEq_fecK;CH7Gdv$WewuvQZK3iAfT8Ct@_2G}pGI4;KV3jutPSioVH^H~c*K z|41kB@BscIS9er_rP7D-rEf4j-E2m!G5#jRaPMLmDL;??TfqHu_C{n52dbo2KSs7A;w3z;%}G$eeu7`zyC#@l=#P=S@Zpm-~;|W z;?WRo)5l3^bHgdZ7P}_BXtzCvstvKiTl^BgKrDPi)&(T(mIfqW+d+#rPud@2{5SZ< zluzWvI^?|iC5!J}4WQ;{g?|e{)>1dCsjS)JNg9_lUREW8!-w=H-s}|F@iA#6+%3W& zyUQb4_Gs`3Sha2ngnx~w%+B0Ou&>UV*`|He?^Y#RnlY@7j~k&kXRo6}Pa@mR8WYo4 zBuZsF)%=89n~YpjdyjwDj>2H;)q4pXtsQd8M}Q$MuRGUZzc(xLjBEj&u}NWcno8`N z(S9ZjCMyU!v=XLZxE_DJqn*gk{=R58UTOHgvb?-X-zrZ2{bRYcPbo4y11^N7os8lW zcEZ~>tHTfuyU(o3AMEzBlvs?v1-$gJcJh_%T8he_Y{rjQ%M{bGQy|?sX?|Q^J?&A? zKHU}6rSN4%m8a*(xW1{!%{yeT`*>HhaPrEP>70+J&6@`t+0h60Od;AdVGnJaFnH_d zm@oN3mY~mLc}DQ|XD6LIzJ6Tsm>mwd#>`~*S9y4iNeIibG`TWRG`PZ$&p`FjWuH}j zb%Zni7s$8I6jjN%Yz)^O7y`S!=}_^7DD|biTyl?(Zxv-wz^|`8G@>nf7gT|7{X3Bt z{XGOcm1VUpUH#70g%AbgT2=E3gW4j@p&hiWYO7S?_e5lvsv!W4{{SfzCOI%7ImmKH zmWT-63oy-aWeSugt_*rpcnl%{x|IT>=nP*Z_u^45%w@Hc%SOGAwC<~PxZinS|E=^b z2q6$|C*MgWu3d>P_Jjzm)Yeu9DBH0c+VEA)c0Y&MR$aFzawBsN17IDxNO6xza(L!! zom+;9PEtOA$TS7hlDj7us^l1xva|I+adOxS6LpMIGIA4@;zJS#KzLjxn@KE?gjbga z&kYh)=oA|xjOCokg3^cyIK|MbgrYaKKhMbAEKW_uWT8P0)ltB>(!oLrJW6sR+D$(- zd8{wJy##@%?(1bw%w{2?D0j;YQ?U0?+qn^~N!D!9u0`3qmo+b)2uk1IT4hvNvcX;p zCx%H0BeN=(K)a8;FI3}IkoMUbL$RZmeX)Mel3l5!t#2ECet7?0aFvtZ^{Bh5(!`gF zsMKOq9o%qg!}!{v=EPAgUzQy=516fl3jelj)ynRHkrl;*{7zk4!8^C|29D;uu@JoC zZwz>)7)H}amy#uNb|*$iZyCv5>*D@uoc8$EwGtWtfS5v3>K4Mas@4_m>2<4(&;0nQ zx<^$hUw@v$t%g*$5@vubhG@G&9Oz-6j<`61)}lhwP}c1RjM=tJ#gztIb#FPwvSRJX zv~53nzA*igGiNKnlXl%NHiU%TZwYO7w^TwS_v@44j!n-VA$C#*|#A#k*5+H?D z-|aNT^7fjx_&x(_1NnQ1&NP=^(xgpY)!|mw90+aOG;F&&Hr*BJ6jhj_`?yN)#8XM@ z<7qx|Mo(!p|gB9OR-U2k=1KU@FBnr7Wcit_`!tHVy9Wdp_rBiR;}Vn$fy!^wRm z^hkhII?-*&pp&AF_d=bVxA&a3#4dUzB7;oWQ6CVO`aMwiXM2}jaSYdA?OTV>69GbC6x2SSGj(mct@(}a;93g)#qhKk zag>wu*!a9{E1C?+F_5$$RH4G4wW4l!GGc7_M51M(v^JLg z=KKM~8gie$I~t;fau~Vy+|Il#WZ$xIzQZn0;xB(@ zY5Jgu+A;a{X;(AZdOeaj)NH)D{Am6fhHM&Uvu}$Jiq!P&*-MKv&yvyIC0vVrsa;<0 zw5&KRcqxd&xY%WFPD)XXWMYb^w$uil*bKQ=bIH%l*Z&;yCd%%XgNhU_3c8$?zO*ZdQE-ltfYLVAJ}}yX@MqP1Y6dsK z8g4{&jga0D`BZxFZoreGt2K%Cu#0EKb6(CpS)!s*Pcrl|r`{xK{#*g;+H%$?^yT#6Y?Y!Yxz zb>qCLj9Dz*rAwEpn+*%IfJGTK$#@#wcuMF2+MH;0!c)a@Q7NWj5Dt7_wd#w}tXG6F zBHcZ14!u@J^c`(+z!d+XOE_$i7=knDp?v`csfvoz11VGPmIr!$alGnySG6u|x{U85 zp@GYQD7t{n98U%d50PwJ?G5r-ycpftiCdsf_ob7lWmT}&K4d`D^t`d4x9?*z%ZoGc z3P_@l2A+H~$SCU(EKtg!)z@o?kb4*Rmg93IJBDaz|HrVPzi0us;fGVTi<{(44OdE& z04M6ca*_7fvUruM%=X1OlG_s#Q0I0(lv;ivtSazq;^-orOTQuj-MLBNEbBTQYgFJu!Ax02bGWF3!sDe{*$93;Dyk!T&|A z;9prU{DJuOOMU0hRm1tm6*X}uA*fBnL=ef;S%|6GxaOgl%XXTjyG$i-n)+L8)D6(UXHlRgFU?Fj;5nIDW1>VdpPuAa6G6s{DuFbs(b>gmuM z{p5a+DKDsNz;C4KI|5U+=UB?0ch*D<1WxTqDtny!U$~d`Y#1w_W^_cfI$w?5Y8vlA zt(XCK2X)X-`5P>)gF0khNr{VD?e;yNcDi=RK(N z;mj4}T6nzW2@=e{TLiwy-Kw0ntgJL0DoBAFRBNE?x|v%}{2thUw>!eTZa zvjDdee$PrrjQYh+BZ95td@#Lu+nXZ7b6&^M*`d$bm8`LIcU>d8H>d3WMgLby8k{I4 z(dXBL93bTrSBs!S*5n7i$)zLn(FkZ~GwS%N5W2n6+I%i%JA!7t1fo{lkW3v5H&JH7 zYhjExN8hk36;#yQ$(X1+zWK0zpb|{diii8sruE@jWB!g8Xi2xNnY1|cx&Ak);U;lg z&I#y<{qtQ#^_LS}s~_DxZ~g$pd5S_QU&n__TNnE+l2m?@l!Xrw^UYt9@Ri1L)%f&X zyP8@j>Gsv8rhjPvX%lA6TUFM~UI-Pb>)8kG3)GB=`7?;Ec)D|q@HBt87z&eggb#S1 z!ZSC*g;l6`gsV~a-Q9%?N;8m8oH&!_X~nS9zd9&P63~E|!zLz!ytyvlj7TwE3N<6@ z=YYxdwTQ~X^;GxNP09rS4pDDCW@YoIydtk_IMOr#y5n8UmOEF;T&S(Q)Zc%9#>?HBSswq!} zuW_f=&|#^BQQ6zs2=3$?^^iG??n<>fN*X$kPbW9mkm7sz1Mu#YfyvLbtJn+9vZ8;~ zQy8xq1+-8*lx$6toSru~5(ML-Bs$7a`6MopnL{?N>%nxc_X8TsaSD0*Z zqN1mwbWq_CA=#s^>-psi_q#CqtHfpXKr|`h$$BkxITXuMI#}S<7-&kF?QBx(>a@N5 z`jBzH{}K5gRdWC{`Exi4@x5K`M;wupP~7F0HU3L0O+AdpOZ1Jh|p%}Y|JIx zW1d{oG&Uu}|My$)e|t~u7OnVb718gjwEB)BHbDNG5sB;;q5IqLrEtSm+T%LZgJtM4 zMV%FuxKYP$xj8=Gn?w|OH6h-ad#{}s?l`8L7!{Us+$(*Bw`p6AADR2~K{kU0-{`oz zwDz~Xro#`UzW&MzAb7+8X!HAd;io={Ps=fzMq->A-g@(a7#XXZ`HLXGi?Sbp#m~~H zgkSNJM~m1BGEY#_^#WICn2b_z`YrHp<%vTIDh%o9HrpZ}SfScr_&=AQ1ghNec)ZzRt&Zd75w z34G8c>50mGE{t+&Tr+}c+hFV${TQqtfAdiJiEaBoR^rTjK3MB4gH$44!H0rom2&8i zf+qW4M*9s`qL)usPL#h#wHaXuVjxe{Emswed z=iyvNOs)IvJ;tYD<=6*2^zSw)o45C+|Cgqjk$ov9uL~mGVr0Gm#hzTz97PKB&6h*M z(Du9S59h;cm<#~9j!46{Vc^Vqp0E3^3wecmvXB;+|jOqN*^uq?7!vi@TndkvJ8rivSeB7680EWQ~QX_Oeg2}jMbYu7kN|70m zn5(}TH{&ZDn8aHKw8@EijP?kM(|$sBSN9Swj z=xZRFE@b=psH^QF(ezLj@61FvV^)k_!@I&C05|r&Dmactnq7w&uYq(T0DIJxDb#}~ z$u--4^q8h1No7Y?_*M0_Rys{fm6xyFbnSMw@WL6MJ1pqb`~Z+bKoQcUzy&4*`m=|5 zqG|QFjIqq>8*TdXv9$qrvG>UoD+AVL5@UVL6?kwvq{}Uv%8L!QG+xJP%*;r|Ac9>! zW!6>FDu=!iab}OsEFcRiL&8W_zc4g&xDX>uP0%w{l<2#95Mj&W6q5FAd=jgR$K^r? z9g|XT2^5qW#%{jTYUl7>^6|p{ox;yQVB!DOaU-sv26bo#xUbEGp(=v2YBHlydAglS z6J&2`{Qe#`?>#4ChWiusb%&;mV8IS4wsDAbG1yeU>f15lHN#+%d41s{ExvwEO08f= zds)CLL)WGZ#icXSLIJ^dHd^G;sHhXSFuX;TGL7JE8|=PsDrGkq2;@e+Alk4sQzNuo zgg77?BZK%Wq)T24&Q2Lk8O*=FFBfis{aWR=p+_c*uI_0gJrVGE_E@+lc;@>jjxbf$0P9$_o!F&Y&7!S+vKgqzCnaN2 z19nBmK z@yj8S+-NK0T3LzJEBshXt^WIZO{B;`SR~E#H274>@0k=`b0c_Vl=d#=l2pjC9zjdkh>AVSx>xh{dXGFT<6OaZTLuy7vMTn`1g~>*WOMO zM-@a&TiyAZqB87FfvRp-WeThSQF+C;k(fT((E(aLInVfz7*KHmN~+$u=waG=oUA|H z+z7Cny?lwd8VaRq+sxj=$B7@=jNS^z=qgzEK(Cc6+n0eBYo9(??Ifv-B`^$$P!DIk z9okN-9!g7BZdvgCG+~ZeA^BxxnLlT=9;hWMKm02h-k{R9zDUYVR}~Pc%x-ZyN@r9r z#W+xf68yOkmcaC(q4+Bk#s=hTm5j{<{@m2flA@H- zdm`(ZN06Ks6ym*UryF>>&3Cq7jbh^*Q|AS2z_V0y;P%%Y;TJGq9?Nqj^P zLz|W9vAY5R9@*W_RT_%XtrVIgRk{~`Dq}8QS`R3@7yDU*mq!_|eA^jD3$oXH=o(U4 zb67t4GmlDYBQOPxguJ8|Gf66Pp$b|o1m<8%cO7q|8+4y6A1-nJOv>DJ zL$szD8|kaKJCZ5dali_)?vk;sQ$a|-?BGWT)^kgouR`dSO;L{vT=4}ey^42d;!<<= zrq^~ZT-)c|!E@8o&GvhOVM{JWb$=|W+%>r;TyM;}4kFTrL305(Bm)A`q*gB<99%c2 zke!>-QlbWY(Sea;9m0xX0#tLC0*I=zxDzgz)ZMY$Pebtprp!O}-Ej}|w*J&ZuTj5q zh-D6#tL@(NW&)>o7$oPy)$za$hRXTi*Xz^+17AQmR63^`1>c^QDzg{T5tNDVn|j5`!s>{)P)7JCm%JRSz#qqW9}=@Ubvb=#=rtky0T8E%PIq0vzx&YgAzLPb$+x&c zK%~0R_B`bE%zn_G*7^ZBRb&{8Tp34G9xp~KI*s{au%pz9>kEJwW5oN1Q_+7-Xa5&d z(tl<*f?}32DV)m(W5;v@I@{`XhKXkxHJ@O)i*4b&1^QF$&UyUGgMe#|9wB?JR^Ts! zAVM7tkhjh_dfmw!L4F^P`)I6ip6e_bleQr^LzJT-pk{<)F7!Z^2{Dv{FTK%;A&cPB zVwSb+lNH}g0?$W3V-#ey)h~%m*Bznxwa_cz-~S{}RdQ&dSjyG)7nmm3&}QhOH4H?9nE3LB^eVXe z_st3Y{(?+KpD=B!S!2qa+R_Ly=%=4Q|L5KR?;89jnO|+hbpwP91&D1Nyb{$%;d(D4 zou#_RUJ?%KM~=nVnuM%$ZanON_ToN8rch-?#!T}-(nF(sd^xojcNGNcl)aTOsH8(R z#J>o>*J6v$FNV&yq>l`=XL>_s#ER^v`+PaD*P_Ru`D(o~`AjeUE=DjZ(M@So-d~Jy zaF`b2(LVsn7?(25lbh0w0#|5EPBi2%reGzqSDVoNqm)bSpQruolAojG=YxZB#QPZ( z{#0@DAAG?qZ|ND80G(K!`^!jA)y=1b*N(9MrIGZX+2`S3OtpXA$NX1be=}YG%J1}G zWA}R6!W_y^b;9L}b=$PQvMk!Jz@TjlqchPk&?4_<&DxaW_U@e{^AySP(lG7>xo+3_ zhK~UIB*b*3TgbB&Dz+bygNx2+8g};HRA~5*AyxvDcByL-K0A4`jg>xFsVkYkON8Nfe*92&8qr z^yix7PZ%7#&P!|QYZ!ya zVJ!2WJ2BR$wE6xZetcy*lJ2dqt2uOxObDT5j&elRyIimy?5iQFOxeo?DRgEIQ0mdKn}Xc};#jgRdfQ zPmbuG!lKV5-iZJ;81plm138n?8||W$(zXfXd#|cvjh{Kp&)tFY%i0XkxKrc4+#*uK z!QRw^I4-M1l1!m=7@ja`kL}PsuYS*Cr6O!8f9H)IHvc_50(WK-JW7{h^6wa}5Od_) zV4lUzcoN}$0dg&>xO;dv9`**BPo2paRZ|Z>fjmgQiAqUFiJ!twXdlfraU+sCOlU`j z(?1tg)EfdTPC53r7kJ{NC#G>ptl_j-?m$jJt<^Vv%Z+u|ZI!Z*3hjKpj>wA01@ zpiA7mDFf6bn)C;NSk)S?YTYg_ArFotD_5bH;YD6YZZYGe02$OwBIJsiDbT0Zek zDlpr201!jSzqfUluduna$mJocAh=LnwaC zRX+ZHG}w6MPO&YIA65iV4v@1bBHon;5u4bkLxRGDABn7Q!{WqK0vS8^*gt-02fgH9 zt;A+dzZYx86V;n(>c51&vvtUhYDb{eNr(x%a-NP>PM6#cRY@sHY38Gky0}o7r2`0v;x5(e*Q{7yN6|7cn$m9(iJIl6;SK@kD~&3%L!+}oVCPd zsH)SJIxas}#9m1qMof+J(sFBx*bgqP>R>%FhqH5S;Xq8f!Ark8~)_QUUyh$uB~L| z9BA;OD(rIVY;kQ6T!1lXlhzrRVl|xqeiBdX`Y!MZh_JzgzOg2*_vYL9U`{HLk|_#@ z1t%kg+Ax2K(h7jr@)K};D+Y=-WAg!(-VvW*J8LUmR9j%5?CQufL$_%FNIaM83)E=7 zJGlI1y|L<#W$y+igu{8-&n*Wli;Q6wW>x{p*`IwCLL|x=uLEBQ&Kk}uuFgL$=UlTZ zN)1?j+3Vn$50KrF5yI*y?pVZ;_Wgt;^*GAe^4NLI?8<_GHay?Y!T~mWp>`>%&eg1_SezSRh8;Yz0)N2^c8ZOMLSi6%A;e7wr zL+8``y`7+d7>C5+iTwI&>BBaGi}S@x`$$2I*HbbnbaCPbV8sQ?d|?4!kFByQ+&qBd zW{?keUQP>u2!QRwZIJoTPkI~j^B~sGPQVPGA;l#o9C^4P992QbF)GS6KP~h#IB?=R{#W? zF`;~O=K0nsD7OHHu}HF62@tqDYI{HYR7nNr%qh?n;Hgc4)D4HYuY^O@km%B^EuiJ{ zSrdcdwoY6c4O?Ax%n2R5fP;*%EFf~-F2#k_r=VbHb{k67;JRc;Olp9t>+IZqURpL{ zzvJCX(I35`Q}4+U^6G~2t9#vF@BniN$9 zVb8UHrqL2kcC4b*wFkf{^c=>wI+cvf!!uH@Qu!yb-lu3tFiY;wz7;g2?a}{ugg`{ zY+|je8o0TdCKO2Gt|p05;j8Bg2q9V^BmK8g{YLSq_$w9;_lbwhHq$uPln0d2e*V4T z*yH`%x;KyL^(fal0&3nsAb23sX zX-!tpkVTS2{qk|X-HKwAf-o_;bRb)BN4rL3ce08*m(}80sax<#R|$nyfr=A%O1l^9 zHf(`>REl>C%|FH~+}9`v%;Y>V%T^>aX^@^?@5wN0V^;$}fG`XM+(AP`N9 zG?>x9DmOq>WSFnZZ(~+<8y->o4Opy1^288$O@yFR79}Kad4X z=%9TPQ3`Yhk`G2dah8Db5;1BRJia5+dsBzNTQ+12DbT6T_}yD5zU|yQPS`OOYSop? zmlc_3Bs^AaX7aW=0gIQikNypfj+Ijvw^An@>zI#XO`1qjbw!%e6zqFpXn0e|lTck_ zv|y!L*&9>YFs}terj5UR?KZO0-V(J3>@xl@zl~o6A^vlJx{;7q&12@H5Ybe& z4+cSS+eBH0cINNWA_YkTXafwy09t7K7lS_BAAe?D%m2A4M3-qrAi#tGc~o`Ec&9AJ z-7YsLn%Y#K;czzRrt52wTGL=k9FV1or(e_doWPg_ItRD1|H*)yL=L8 z@3bF)rpZei?8r$e8ZcLf(*S>dcHD(pB4;EwOmdh#jx69gc7|F=FLqU%f<}x+u2}xe zkWT{#oJJVWHA&K_D%X-S#o<{D;mL4)4{dmXNUS@JUlBb$WRAeabjHB#@(QyE_bNtx zyUN5k?2MV7&R}*Y0LK}{cBbp!!!7t?zZ6R)flra@j;@Z*Vs)Nw6%UkoMQp zeW+V-u5ouEQd*<^JpVYxCt5H=fhzVB?S)FP_9D z{QyX7LKkh(LQo)Z@$i`H>ZEx^hW?WA2BNSjC3{n2YPWJ^pH8Dn{ek@2TzP~4fg#s= z^xy=Cd*{6y5A3}uz?A#2W<=spcKz!n7dPG~11;5_K^ltm`)}AugmQeiyT8Q&2pDfj zfPUS3=#kOpjV*zTU1+C|m`ZUVBT3ybaX`X|Qi1oh)YoLye#8L7e8-59GOu$NYabmA zhx0*Ai+MUe%pT>2VX(eG`c(1g+U!o1Lq!CYvl490nLT1}Eopu}RPa7;M~t$t=Y@(I z1__kYypNG5p_Q(v*grcDMPHaA|AI95XIH}i?M3t7KbroV@Aw72@T;r-%~-g$OQG_c z5C3fNuQ&H!cc~F5v;liLg`N%Z14%`=%9T%Vt=xt z5}J5t_vJX14n9RipVLU%vc?AoyRp4i_ZTep5Rv;^U5$}T`|VQ=G7x3PB<31XDCj_H z()onN!_cvo(%X9aJ*>$bZL+&4;`VCt>kq)+A3Mcgw~YmpM-;bIq4_aFr*OVUo%tJ& zx;fXEW(zxl*Ce;!ygH!Nk$ch2Mj$JBU`tE<4vN2!R!RAWD%%wT} zj%Fs$E#%m0# z&d~hVEsPk0 z{*uJ`bszrfOJt~Q^8R^zHBS54;NNDO-*ss#QXoJ`n%d^jMvArSLi^(yb&gy3_{z)g zo9vdx4AeJt6$mn}-4ss$r+4LsfWIUIM$z~>@L+R+>D`J-a01+&Gr{olMp6qQ`{>{h z`!B*8<}#RkFU9D%z(0G~|8*$&8J+*HpYZ+x9Fn8=$iJ7P|JlH8AM9NASgGUx2psZf z*5do0TP17H{3_8h@cIXUM3O5Nm8gz*o9@w9uoY4avZKHKY@bd#zTwk1->>#GQ~oCS zBsaej^@bes39N9n8(TfnvL&=5Fpwm6b!3lLC9s-#pq&-G`b0k+#{%+X@*zzR1v9E1 zaf&9LT_kVH`lbgw%q)Pb*aq#9!Bw*!HuWQe`)XY8q&?6h(Pi~{#2Io{H@h;vA7mdl z)RG`vRnD;y#PyBOVEw?5y1ROmN;5`TeL5?eK@AmVB^kgZB8FX$7=oz~KHWN%L#lor z7T%T62z-_>R$CjsUK_YeA|%LKgj$Vc7vv_+UL>-|kN7v#suA z)^BLOn@Q8hT5CyP0-#+N5(r7=L$XvO8AF#@T$e5qH_9S*`KPw7lR4YlqhYId2co&s z+tp`KslhR4{K^-yiXf?!50rr3MI+2N9t{?>O_EAo8G73gaKF%NT@ z;GlQu-wGfy=V{^6TBCcyD(;Y4M~X7^OziKVAKPB61*iuCJgUWHZKHv}t|^D`PywV2_iK6bRr_yeHj={cau zmk}byXaD?VgIyH9uLZ1LX_31CMqN!kM2@vdu1klRK!X@>BXvr1znd^M+V&05>ND5l zp(8TnCVA zKKU>GX^l0nGPq)N%rU7yS>e^#|D5{kpINKwe{ROnrEy%w&o2&oj>^*~(O1W$Sgg$S z;~dtV*+}2{)GMuuIL|h|efxI`w5|>I%Hqw->&VFfp}TM16crW-dkHFhP#Sd=9zQnh za6aQ^OUpQ?ADomPp9Brg)!55UWzhCTC0wkBi}fFcQfclsxCB4G~3qtPIlLi zD5R|6`4>dIQsgLKGi#YYh~ut*4&gzr{s7n`GY2M8q(k?PLkTtQ{Q4ME=*1xlH+OM5 zMdClYQ}?ZTom}`3FlFR?B~HF%z&IYgW#k;a)Px6D97=j5m*%K4$$rGSt$)>(LCgK%~G0FLg&GEsM79Elk;*SEYM!cs)H=^favZ%&}XJRDVSoMUB(# z|FK*N%e-aHaC7Sp_S$t6DZEj!rH}@n zvZxI@HwnRb+=?xS&NIOl_$K!l*+9iZKwj?m`1K6{vxx%a&cxwM30*I&A zn+5FIse6ubMQKo7>gMI&6T&N6P!h(^wIdx_$G$nsu5*gZ{AH7+a_Y<@fwu8b)m^%ape3auresvWdTp>rc zJc(|!u=QZRj1Lm=B%2!3oePdrY6o_6MWRHq>|~Bd!sJ-UM}tZ5G2}$z5`)O%F0FBV zSc)P41eNcyd4j(E8jzpy{%cm9Sl_2*<-#H$d!yI=7rCv-M`tQpT*p>9`ns7(VerV0(BQD0wUH&$?dHuN2(1=iWS(1JjfWW znB+v02jFEt2aw{B`fO#MvHOBGk1@*TON{qX>P0BuN0u(Lp-QwD?K(4$>H3z84WAv} zb;#{7y+=c1Hq^J>hx)M893rW_g0Uf8G&E4LgOiVDjIoF>6cXq-&u|y`@_3rE@^rAs zhV`vRO`G3)J2`fwfs2Qk+S*Q=$VV#;RZfFfnd??>Rq~_&3?Wykjb*4x>4Uw2iGcQy1KI{o>YbIjzh6;;JWss{Cdn& z>@*@p1|E8Xh^Wq-->OQ!Gsa&znm^N(-->Tg78)(Weqa~X#Ty!9tl8mY7G+1L#<$oy zM6=aIKW?fKsBBnh+M%%>PfPWgXY)ZTVVPl~vgLui=qC&Pq)Xo{nFnElDY=+ib{D0x zl%(5At5bL*lOiiQA?ad4HVZg7WSRXKsSt3JW`tE3M~W^KJNuYWdZPaWuyOGJvG>+tRrTAx_(DphOG-cq zX^`$vq(wrdLq)oKA+bQEOF%*z>6Y%2mTm#5MfY0d0v3zk^xgaJt@|F|ea_y$yYD{d zzJJK`_{_QHoZlMp8Do5g(HAKceTdP~N$PK%C9^L|e9xoWk$l=;5X!Kq%A zUDt`5L#i&wZ3-v`P3?iOT!Ik6lD$$$l=7+kC8+)BbTI2eQjNdtyY>z`3v4TVr6gzZ z!!)3D-y{_UVM1nHf}kwtK&|V+DfbE;Kr^93%LtEqD^86eWXL_q;Vf)Rk*VtCL{?Uh z7L{*eg5f)WX{~K}gQ_S=H$2XXY$bE6P9)tWO33J`2_w-i9*Fsu-;tgRkT6G*KHhX= zwYoVfV;ja`gD21y2)&mIb$gaC7KI5C+b##P|CXEILy)p`fBn4Q`UU)P>CcPxZ!;T@ zQ`J6I)m4?&)&{dDe*SzldE8JVr1mwdh)qFHr@aV2DD|&P_|p#ijjHZf*Q}ywC1E^p zZshGZ?V^z5OV9;6YMgxf%(g4K-rHe-XEfzK+K^W)rZf=w)os_~y~4N`sMa z`rW0nRBM!Gj-;ygs6eF0b}Bh}rTeNi4;ViYrh1$NDj{n` z*20ghl*TKH+*IFa4s$;rOJ7ZCf}Z;ep{T|20D5spiT899wWy;J1_#y37B`EFS-dZJr-$2eo>d zDP0I~wLJ|h?I;lWC!cjKU}Gz`*1J@Fjh|=HsVJi{s)TrDlN98xT0grmu|hQ+pz0zn zZp1kIyc)vpR0jt@_2{l4jCJeltH!;VEE4$ZBQqZxJG{K1V)k7B>ciL>CErjb?a+i( zf%@78nrtv?vTeFy;=~5J^}OJsK)bF=@RdoQIefr$#{PJ?Z~1mI!}*8PfY|F7nR7RW zGvVUicN0XI#ulorv~^Y@GyGX8_*3*{8!F7$>XW#jq*oj|oQ1JPXw1=}csJI%bUz}+ zyx28IT|$QQtqP1?92{KobQ{yEyz(bi$tvP7MP%Qz;)2BDmnTzcJld(u{E$4)T?%E= z^ft+9>1)Oh@>LgSJPg1F8Pxo-bJW>5H%@ga^g?_OqP(}f-;EFw7@DIW0M?pW`yiRQ zt1h2`6#+qz^A3ShgDGmHW^|{7IVN!+YVy&I6qD8asS>v0l|f3du-F1gfk_*?1{<>M zYN*FDoZnD%m)NiXe=|`z#otY&LE^KYYI!63v8bTZ1X|1mQ;vt)F2<|U#&|6Zza8oZG z^OVR#I0{!Fij%ZEZEd#kmQ}8vzSO}jzWRJW^@D3E<3)nBguIe22w}U|p*0LX>)>io zEto-kw4?M2-cr;tJtsDYy6M?{TX+!Jb2X@8_zgqpeT8ls>^4P?v_DbY+1&9kRjMM<{097Efgl=1@ksz)seL zkFEZ6b`V_>ecX1l1EDni2ea!q3a&RqriB5!UUxcX-q%@$zPDYA7ZITdGYh2Cev;e= zvlNUi#Smf^Au05o67rnYws-VKv zsYBvCtUNrdHZ=HyZ*J>jo`20OonZ%}G)==+jGWp}ty`|V= z5ymoN8POebLv5zpBCGd#DBA=4O3X7#uzVzS5S&@^OYz?HMud;J>;aZsjiM|{+72db@@l>_dl$Ne{}m| z5dj_3Renq72!{Ugu($xm<{SPBXhx$VSApWVqrG-Gy;Z@98|JNqIW9O z=PZS=cY64P7)fTj>X`k7!ey_hSK|u>d~+q8o{lxbrvDW3IANxv4!T)a*-p;4No!M* zWH?81rQ|(lQ=c)5y|KB{ODxdce#x#y{N%;>3Y=7IBYmBDu;tv@96CEghYM{w+aRw2 zcU*GzOC9W(l^6ybzf*H`87Ovp1q`p-e@(@sBk^H#WA)2-tO={#pNWUAIz|ds+Q@;L z-}J*&!ptfL=`oulF{$l&Ufxl?-P1b}aKs-5 z-C2sAR`Pph)jZ)%W?t7ZmzA5H?(y6!cT+PVx2D>>dgj|yOKkTG;s+ZQo$)3vp1Tsl z?XL59bg~oNF3kj1b#HkF9UQ!9-|wf5%I!7BBCDzxsUXXWHo8X#pSZg?^Q!Km;MGX)yU>=p=qoSwD< zqD{n#wv+TnHk$pP#C^$QVR;%UhFGgO6*kw=-dMnu^}=P+5XOpe?XOmL3yKTv4Gp_& zBXu|23?G)PF=AU{Mezq+=ksjgOS+Ly4ynM;63CfB3N2C!uzMI_NxVAmYfxaU4pbC> ztmpT2aV;$E8R%KzJBr|p>xwUwt&lT4K<6bkH5Q9C?PkWd)Aw;P zWQ0EuW%jL9rJC+KjxXfjIKp@&A76Uk1bxgFoos22aH7HEBRHeg)sPvE1Aa|FwGk&o z(g~YsuX2lpcL<-ezUDVZ({iTQjP$+gwp_KR5byy0ip`uH`(X`iGiORygxJ{r*w@v@ z_?_PudvK@6i?Rl4Sb!qTyTGhKuq1a&HB+Z4H7g4lDQa;XMaoTjM>xZjbLHa)CbMWu z-}+){3DdtvE%f8{hP4jSjM0jA4mlTzki8`y(NDFnop0r#)+G0U*>pSh+GS|VPQimg794L079Qva+>{~lJeld%(jjA<9b_T=rk|ms%Yt7%9-n&*yPOGuYy}Cn;%(?>CK7n90 zJa&WMm{AY%%L&*4Qo44A-l#_|97viO)5N{YGL4lJJ9#whqciMQg`GBqyfJ)ueo^&m zHSdNGcX(zUeecyzdfACPxc3MzGJ5$;Rhb9&Pe}H6pa2Dl=d-3-IOAQRfG=RfEzMM! z0L^pVIIFKz3NtQMEe}-qRj${(i~9P-$LYx`Vhz_2XUNm|BD4b#cAHeNfI-SoZM`n3 z=z6n0qrRDXbcz82rS&2d!Ftskn<`OpL2hM|)`4Z|1}CG@6nuW5kTHoRP)U(%u;PoF z*bSVNq-Ck{F25ze?W~C-i*4gk7rZ+Il#pEc4atR?Y_YJb~0} zOsxv&44)SJ>+e%~YY2v_VGdpaFX_UwGm5V{-_EfRl|5QbB^x!k##oy$;OaN^oG2{U zb&csDDJ&DzHByia>~Icob_;j2oF0xK4=;D`RgKIbJ5!5gbjy-2l-+8M(w&Opld}KC z`q4kvo4T8IKnShsB$LxEByZMHd7T8gco=XPS}Rgbi#%P4eV~JWJGwV0Mkvo<*+eh> zNOT!#wy-1Ad3JY1fHJ$%+{}M-dUYy>i?i6~oA~1geTDV!!$S)*O|;1v3GrnXn@XWX zC5ZY~TKX|PT|2y4J+FkbC&3~sRSnfTG0fV;!pQFVS1am{iHejxlGXZJuaN{M3*D8J z#K&t-$0`#eD(8*nDEd44J_`gXUyTwT*WiEh;K5Una}VQj#rNhg|Mb7f-%M#;)!z^< zpSwWs@+HJ2Xp$dg^VAYZmP115&Q??V^NW5$4A`vM3EHpa#44gL5_rfI8jroHu@(Wn=qWrNfP_HRj3Yo`2k$xxs`SD)5 zsX900SqIQ*h8v;wozCXRn}2x7ZSMSZ7c%Hu z;Gfi%6$Ia5X}ccJ6F1QqwiW14AqtdIXfPsk8{CF?r-&wZFhqzTEpTa?pAN_mYFOW< zx&z`@9j&LQgXoe=(VMB|xTSD}<7#0?-}S+cf*>}ASLjwRZfNkNq%cekt^gB(W2K18 z86Wt6IId}$U0HIy1eNa|2reb<0F1MC=R)7>Kee0p{4PP)rOKW6PLD!R2qRcQg&HRJ z57Wki>IbM7Cb-WRA)kwfrAeFrdTYi{�FLY5Z_+#1GH-Et~PDoBw#>|L7@Y^Xu>n zp8NcI?ryfoHEME<*YP`hdNIjPK!z0zGRX=#WK6pXDc_b_oBnNIMQooFoF4zS{eW7o zWVpN>!XU>&OEfHOx5%r+s4LrIg38l^segRAw1=&;$*PMgEK-a-a$Ndm(vrBc_ee9| zVWZT8T6noWLhwL;JZspvbV5^1FR*wqwC6Ttj^C8bpt&h*j@JNWT|x@^8)6wK*0LPu z2_yxOcM(~cx>mG!aJ=MYw@EEc`;N6vX7}wWWV?3T4d$1o>fxnC%(J(Kurq=7XHhj^{i+N&!{Ta0PHVv?ie;qG+P0W>ggt04Etr+-g%io=@By++Esap0UEHe- z^z{PIT6mM*bTc%9o}vZ;L6&O%zfSI7X0d2knoR0HoYVqmHB8mN&T+=MWE;xsJ1-46 zl5Lh0?eP!)XZh*rt#GrDie~JMEw9|D9R|#B#?BK77{N0U=Ceq>SGE~u1z%h9EcSzn zc1d~f?nsUzqU|Fd5_WSr@g1&($7D_pwW#ccUT91A zEURIa{KDTd68`EvHb+tq$EY({^LM!rQjBQlLp*n>59E9lf0jfpH`qH%)))P;=k9a_ zBTEhgBT4WHJHbsb`x7|L+MB~cLW`4xy1VM+5%8y2H zaAp#%zTWX4OJjOF!VnW-j*;B{ObuK4ml`?=Ob!JoQM?(z1?`5^u&};Y$;|;fhg%Ul zI65C|Dr!|8Jbc^}b?2EY1%*)aqod9DXv+VPs+zn5IR|22i+)lGKyJ>z7V zoM_U};$?q2w%7L-4y0)Yf3XL(%MddFCscwI=3vPN9N}+{92?c$y!~nz;F@~Ixk`3| z6C6C+)r2)BF6+LCLqKFPDSDigVlt6R_Kn-^3IX)DE%&O-SDN|~^s8ap@4z98Y#UKq zHcK{y7DI}-8v4mfGd+r2TAnN;?ujxA&Ae9}+ae>iX~}`Zdtd;2E6ESERZzv#Hn7z7 zk3$>VU8p8e&6!g+ozvs9Gk0(Y)v11txJE#3nc%fsLK^3) zd3SoF7w4Ob1aiaXdRmIM@v0&@QxZs|kl-rP1!fnxgOr=z){?wl5A%Ij^?lu@yA*B7H7&n(kLmf-}s`j z?03!HsMq?PblXj6)&%M{7`sid&gCRXTshl9bGn{wO4-dF_nLDbnmUfcCL<$}Pwjv# z$l2&uGwJ|TQV{Z|B9tq+uWdT&S?uffm|+(5dRQ7BTh+Ww{1T4&t}ncRnvjOH1~b|* zkKW4|YSb1%pwpAPg<>0V?p3hue;choe-GCJT+?6AS93)@GQMYaN18_UhZ!Z)U3wIR z`&dV>vrKMEOj_ff2G2W*1AV)8f)H9vgx$x%d_;(Lx&P9(mQYo!stc}`(3TZZOw<&k z2`>+NSnVczF>In5JE%b<3mi2q=ue|Lf>R;ixuUZ~4EnlsOAwz0olk#QuboO_Hz4=k zt*U0fh4*b@GIx5gMynLMO@ze>?-iQCR5BAHjx)L(O;G(d>~=FR_xtPbzEIk2jbpg5 zIE8n*3#^Z+VKb$#7cK5k_1+&%bF#|${7!atU{tY8;Cg-LyI5o2?=8*d9|iT58_sl& zC`?iNya1{3L`jtEuZF;J+UiaMxsYPUT&C6M;R;iN^UW(M=MeSo?$!DP&@WN6r_-kq zsuy_;tnN_NZL1Pot26KTIr`MkrCX0%&7G$Nw7GPL9^h(e=(F|k?#P3m+zeevQo{z$ z)UOuQ=>Py#TVm@TQe8|2@qcyhjY|+NPT41WEnh{<|0U|ibSDp~UyBqAz$^PmxEa^e z8zBWv4XA^3P}`AhOi11w`(Uh`xRq=3^v?5041amFJFOUwm)?fVn2J?EpFZKG*5zJHJ`_E-eFr)|I|3XSGk2a zY!qx$4P&gWZ-^CP(w^)GFWoC+ezC8R4_ZF##)kY-HYb{=QXm;T7I8`_5ubV%^bABd zwo4e`L;}?3diqr~$Gn_}a`!9&S;%%5ptbP}8|5p>T5uYwd>d9&R;sI3GlZf0RmK zfT)*;KZ@~6*Cy9XzK^rcRPna!q&=EFeyG^J7uPKfKlN97w)h1kND9BmlPyKG7I*@I z|8nQ7`d*sejc@imfV>nDCrF4rhYGdLa!r3ec?_3}v#`tWyi*=!^_4+6zLy@`??X$Z zn?guTcW>@y+btTQUyV(tz|#&!77@-*5HM|w!5IX6W1|DslW24v<18Wx}7F#O3J-S}6- zD!DceF)tyoKyi(G@)<7{xoMZfj*hAl^EaymOO;KGU7Slr8GZVy@ySf(BB6d0cOBbt zBe%@DXs^?<(qc>A8tn?~^R^$AoFt!VXt13>_L?*O=9FC10kZj}oznZWA+@4=|M8SbMrHNZb>^W4mxH zyn4XkdBr#e?T!fO)_ZHB$jE+>>o2vBLB4KmhDoOOV4fv$pv}%$dFi%Po&;zYoQD3Z z83D8`xJwBhn=(0V!7?m3Mek`RGjq%JdAF4M7}XZbFHfMhw+P%d5lDP#HM|91Ex%cA zfx0=mTgm>m`ueE2pacIvX~0X3l_!3f?@7jd``wLvrOfT{z+F0tZD#|11z@&Q{^p(B zI74M^;-yEp`6oomb`B0i+P!J-_zJid6k5{=?0@BVL`Oga+7z9yHfWKDI0pQrxY5a|SB-@5V&U6TY#*LLb6A@m@kecE_av=7J0*qZwppMms}V=sos zi9+br+wTeGMbhx2e>FKJgs{OKw8eQ6M4~*bqbb=5V;X?2f;fm3uUpH*##^eB6Y-ME zO1~Ia&vjUXQ=aLt&U#xv>TUI#A>IpV))yIjC;b#=oCk zMq>=38X)-6z0WoAr4)k-v;)A3e?bp__9{;!;>B7H_FbMvJ7u(GR=S5NjyRy*g+BcS9yrV}4w-z1qv z=`?G64CGC#-zCOXvo|P}V$T_~x!IyqtO=*#hx}4&j$!d-X>LXbPhOCxvps_CROb< z`&U{$0o8`dtHrvv8hd@i579<`b*6(B_ilo@J4Ov=5RZ*UkMgXqhrBJT3tma*;{AA! zn&-YegK&58@A&zEX*d}DX!P~k_HV&@vM8IvrlFu?Fie}Erm1qM_rwYZWeB9mdO?yw zK+5tjgg58%o|T2fn0;gK{ZJ@N)oBzN<@NgD;?;Ft=|K(lHL0uKBd$&CaPNXeoKz4Z zuC0G@HPmvl?&F4o&c2fL|#>pcdPxEy$CHIqW0c zFbYKz6x@_qJw(m)ycKIfDwaS(NE7!GMBcze5azshjkDcR&r+9cFOcN1&PPFZGh=}U zu7~xdZ;pLhn|8+;6F`TMU!1twuYgyKWuv7!ln&>LKT9dP{%uOPX!(vuJcX^!^H9kR z!f+XuTj5RyUjLYS_&ec_o8DO%f+s7t!i*mi-(&%qIlX!+Iy@n@%y0>^j=lule>ISH z1<2PUxcqlyl1xRcEcTmqN?}Ug0?W6>UV^N~D}fZVDna`_1>{Ufq9T2eQ1?@Er%#lo zAQ#5#>m0zitWXD#bv%QZTaqB7sd=dUTYzCu6t3l)?%O~c=8?B0NFvH+F}<81&=FJ< zijDcP!JY77sXx6euRTo|_&REku0pds16;;_3A)x}BAEjN5Q%$wPvh>Fm)Yp-ahilv zv+yqxDjO8A`t2Gd;kEXe_&_G7SsL(PAb(eU?w$1b{tl3D=r*EKFd>x#Mp19IiFnL-ivo7*4I z?mxqBy=9`Bwk1s~a4v(;*y@xdC>?2D#17~>du7xmuo5tX8Fr}N8Bw+R*%chaW$Y-p zs@`2D(S?wIL~3V=hNX62Y*FR9#}QBx$T0XCGO15XBu^~I3MCE z>%^H=Is29~-{S1cJcp3$?g1P3d#(&Dkv==WTexVc!`Fd@Bv40C!#$D0uy2D4Gm%T| zVyWbk9DB|o^&T#h0?aiE+?+1AY<8`83DYB&T#~OWnJS_mx_x#{hzanku%a<#Hy6OM zVh8UslpDnM@XDo6kfA8rt`NHFYhL`6w1&1Ib+hUAh7|4&r}#Z60X7}s#>k5sb@xaT z-57V|iCNi-kj%o&;rM*(&G;^SMPQ9l9a=g`O69$IdDGS#(xF(BF0yKcZPjW#DPWqg z@vN+)dfoZ^afFgW^7H{RZzA70;$Q98E-|d)!{NBe^4Tvxdr#6Vf4$c3<$1G8rT7o%<5J z+f!L#%N)b{Jx&&tZ=*WzmK6}HT2GFtyX6&hOYrTFwl|T&g<82^$2AEwjAp=AGtvUO%!YuIgY@LfucszX7Rg8lwO3;`MN) zNG3#tN_&?{Hf+?95A#9E$Qkg;H?oM^a2#C(NueiK$YOGw&(XOmW_19$mhz0em&S^q zZManVX~6cR7-fobSQ)#5+-EnQh&CfMZ+hxG6OY@b8~s++f<4K-9Bc}V%w;=@13b&V z4ZPd9aXisYA99P1SJCvTdZ|*b-H0ggvN^h`W~l61koRqbv)>4kCwn7>;Z3WS75__| z67m-w+52N3=+AHZLN!`Un3(u_XkD_r`%_8mB&iW~n6SAVlY(y>x|ngtu0@daqT<{- zf&JOWOot7@PaoTdgKjJCGR^gJO-dIeT&MtyWwIA3;yBP*94MrNj%F(bBcz4OC}dK` zikf1r;zO^FjQHaZn*DS9(HxmeuPGe$yd~@Y%%sNN;i+7RfT+Ol4(~Nk(j2WEtg=41 z+nic+vC~&Ocv3#vd~>q$i486{z0!Iie9emRnm)|l!>kd+JB6V+EJv?>g+78)a`Tzj zun!jYy$)4bQZ!g|SHu(`&^zdYQh4A5{sLa|ZIK2D+DYZn;%&Q*LM_qvT5;t9l_PSW z9}_7W<9@x4o?NI+C=ThIFOO?iCEl`ECVwknASFN9z>{X7n6vT9-w*c{X`N2AS7x=) z#7TJ}PaLtVKV0hh3$nnh4NWf#BylA|j zsoXA)b0L`Vfs;#|ube5i?1JdzGE*NwQSyBbY2R>K`@JA9ETSso) zwWOtrJ#wzq=;w_3vOld{%Sk1t7IK}TLfZ4I8Yzjdksu=O*b0$lmIYpt8(kRlyvGW6 z4TDk}=<1!bhbilha%OaWrPwrdX6H&-t}t{~#_@??ZuI^Dj4km@tCcQLlDG_}8k`PO z*$XbLNcJ!s+iMWYn>QaZ%o|TvfYwEH2al@AhfT=eE%;<4@uj;9%^a!mT`OCSAPiA$ zt<{3l4^znMhl<6e7SH=hyRucB9M&lMraLNvmX$(@6*=QxQFL~gw3Lhhe3Jl+;DS)Q zHm9dW?uILYd*Ox}LQEaDTs|kgZ@4K*?Q>cx{`D=oAQg6$!bdsc*y{!FgpG(%19T)s z6;l3Z)yFo+CAsouC5lini8}_=J4Bmv3<>PLK~F2=v#X*wp$ar{C%#umncVU&_&@qc z-4(Yw7e~sEI*aY$Tci?1cUcsBik8AELr>0PLd(|4W9iqH-LsS*vyt%ImbDTZGs@h? zqVNttN{s9sN`Vko560>Dxt%X~2j;*h?AEa?>NIYwZ1ydoukUSJeFEqXUaw$OjdZ@y zY0DtsL~Sj5jlPp3kO2+`2wdV{nM$x}ct=zs0$`wMDD~`R^(WmIOl_>y5<%^X9xo*v z=Z8J_V{}(1Gkj?K48$3ZrRwRL#{w$po#k5~XkTyeaLN4&rdAtusX^VE-#VV7ubWs% zR8Zksd+Z*c9m}=aH!>}D>k=dv)F${a_;a%wz!y4b_tuao!ssH~(ce%-mpM`y#Flaek);V1Fh1f+)7=+$@M8)#lSLt&Ig(=516+C@Y4=`|qDCj8_bD;rJ(8yO?ux7WKAWwTp%J zUEQmDaH2~7y7QRvfmgMtSDW}fTlznlOkeF^ zonMfmokr$)5+lxY5qxcKuPQ`mcdGr7I_|mJ=7YH#y8g8`8T#7P=IFDQRuZA3Gkqh3 zWqgsOZ#lBY6?Vk0o%3(H>NdYkgA4iNtp;U+#o9F`4fC9gAxy>T`IdJFaWPuS8r=7B z(`kt0?9EcF25KlHRXwJwm7S~VOHtxJS*47@ga`a zHbx))Dj4_eO|xaDfSJReTUOni55a4A#ZfJ!tF)!Am&#K#^>l#rW?8}OYqZG@Nv~-2 z7FY8!tnwuDK7w5(=t1JmBF163IU;O=ER6VT*qQ5%)tfN!ypxwPf+8EL1Y&4k3AX&e^E>u)Vy+6ndEqyLzT zeK-y9!7#Va<26|1(qSR*jiQLATW!)1(dy&e;tVt^kqH~z)+&W=gN+mI26Aiep$fg> zPy3Qtk#rZy4UaRx)r+0wOQ|?h9XaN%DDH-Hx(}FrR7{0UL>0k4xpaUr=8IR!}h32|B2d@DH>cBrkLb$>ueMY5%sfh6<`9n0Mb z>y)SNQ-tCkvJHvvS+vxI2mV2)2EnjswtWPpyIS2^U!RIIGi9(I*Tif`q)=i#U~{Cf z+mO!39xd9vL-3{kc;O|e!6Rb3bYYJeDTZJID8fPDdKpyvxVWf)r$JdOhg*u~mD$cA zG0(E_*D7UKu9~c?4vndy>HnQA8WjvW;0{R04lB+)%MM!ndKd`WN3MkP^Cm6leocEP zmh$0bKteDhn8Id>NIHayTUHH3xt5m4g5K*b0(R1YN3^O2(h|W|nx~I!?Ap$X!ji;B zqy(X>BMKi?_Q_3v01#^VKLB`_nx;Ri1wHb&#OZgzX*>Kot*-V19Gn#~+Y}XoCTYH^`g2$78V>H{aZK?iQ>w-OMEi z@5Tsr+33qEzRKlj=HvHqcX!@BD6Nybt(I1LF@p*l1`LD~qw@KVWX;<-_be`=PKc3@ z)ZLE*38uP&m^rh1-@3kFd|cy)8I5=?$o_5!Vx}5YvOq`No8JbIpVABiA*VW0^?Qgo}Atpz&lbUxe=U88_pM2>LG2fVkR-eV8*Ubfb1Pv%#CfkLy^_)Kw%CeemaX9t>cP=P8W3xR_t{_T<=PXKf(b0k#9s&&53nV$5U2nMf173e!r?Y;e z!8D#$S6@#;WOD0Cq-1CBZKqiC#8snFOBVE{RFEhB_|hbbZL5iEs3=YYZr~qnV&Y~0 zye3xmjjaEKy^eAn+c#{yB-`^g`_>YJi)3dN$mdHCh9~8sD(O1sV!z(KQBLQr#K%kx zty$$k&n;qhQXa6gs%%BqW3Qr>u}_<)5d$T z?rBI%L_9oSl1xXA^7RxE3bqwzh2{$A6|j4R=vk=bYa6!omAG;6 zMA5#am`!mCMUMbYqKG`r;VOF}k^+VZQ+7`8b3<5o#`DFj#%d4$QdnbG!j{0ur>gS@ zDC;b5*C*HW^$#M30IT+(>mh<0PS@GQT?J=uCqVBq%bD|3l4MHh+}s@4md(xy@Jk}@ zCYG0Tse7U+`%NaS%@MR(66{Tgw3v_-x2W4`_V;V@%R+Q^JyA|1`Z)cyDTb`ZH|za2 zRpz9xVz~4Ohqv1GeB|{D3c&wE{~io237qy$2822;vtMUY6AdEHtGM#;@zkjsmCB5z zMu%f?nY&NNaXe5G9Du!>r>1bAise0Mbc3^E5Eo%;cSR53z17l03+GC7FYItP2kPk~ zzgfEWV_cEqMDCO+$+m|%lUVs+I~nC3s$m1wQ`p`(02Gange)X$j<&(p&ujwP)+-ED z7R<>-GapZP=h1g|WIe-J>u=-cJXrDfZuzsz8EjOqEsv4}5>Np&+ZKSf0nnjq46;{p zkcJ=JbQhk}rW!P=4BtI(tHVpl5DH})no$Tw>gvzT$y=D&;Y~bGs<=059N6*sEiWcB zYrj`&q#UqOjVz4Nn%JZQJPq8TbU0g1-wY5X%9DrbH`3GE3Zy5d&8s}pan*}s7=v3f zTDGXllJ?xx_Eg*F%gwql#N6x7hb^*L4dFHefi)HhfP~&tFU&&nP#(o{GK8 zeLJwS?Y?-VFk@7Jm3Q{CdZaWRnpG;EB=aU6_2GOVEd@-xPQyy*4DYzO?){l$Mpj_q z$8ZttbUb$}h^0>#x~}-UdbZl9rs8bG8bsi0KsNh|D#ET`9`!c2879$; z=lv%9k>nj* z{C)e|pGogNb$_vtT>Ii^HEJ_)is4WQMSY-t2})<s01#K5j>0YK{|veOk=IyjcF70MT~|0)&v4$}U3uz4K-{+R+ULG<4K zKnopjOQ}O9AbH#uh=F#pT!Q?OV~};FkV_Ey%q1vPYb}Np(oXT$FSrK~Z>vfI)h7|i zSUP0T8IT)^cba<%8j2xDVOL**C^CK?oKeaXV5o$~{>|&;mmp?<&8!C-DJy=CdM&lD z4cTX2xBGeY5kF12x4XJL@?JO<@IE7CN)9mlbpP`4D-At*m(|wUOX?S+|C%g#$sRkM zfkE;EY3dtspf+5(`BXv`r2v7Gmj7v4ZvVC{bboJIq!4dz{`KcZ|LReFnVuDn&*b|2 z4Twyu#tLEfzQSwA_jt$psZF4-$IgygnWK=52+h=#83YnxOnIFHcjLV6$iB<8oeaoTW=ZtGku@RUR1sWxT2f{Zc_7q2KfSuQ z7!ZC0L6=>8JEo-)=^#?iLw*oxdj9%Or_fT~M#7@hV&fhVH&p(M&Im#cRS8z7(ux}& zzn?Ete&{H#r+qLq`3~rSZjpM>bqS)igO?2OHQ#6u@3w5?=yyCjD4}xB5|LT7&0={{ z*9g7@2{vt(H&#dr2}8G0!e3zNflAk%6dk)BWR9G3S37N&AnZIYL1)C!KPx&wWOx9~ zq(gCkRzf9VkW&7^)1^yLgb>hGIw%`5NQZQ9?rexa#HZp&t!o7#rB2fh0I~;sCP24j zU4ndo&w3~M1rQ}C&46m=`ip9A1c?2wIs8ezA|}sC9$tb{q!0&?FuJb^W=J?-^(7+B zft0o&SUJiMoC-*^1oR7gsL|PVBLptw6#c*Ut)ojr985NmdgQi`NK3^XJ%*YhX>UlK z#y}wiD070NwjjtlD`bHOkU9jVm!Q9y`0sLW4+2;c`hT({ngmeoqf%=TDDn9veN6Ak zg=6CuPn)BX?Ltfi5Bx`V7I#Vp`>h(5?Ag}e3$F0NTDA` z|G%yCS|!=J(a-AH9bl#V*+7wk&7A<<-kCanFfo3dVxyEX;JbekW3Y?toTmjnN)Zs#gTEBhgTE)H4{qhLwPCoBzW;yR-1&dq-1}$89~^%A zHLEH#33uUDf(>ANga0tT|8PG3VdqQCc>6#w`9r5erxzdK38nmwCj|L>F2^69;2)mw zKO>AX7_x=(^nK*ycnM@wepk`{u0QmzHle>;ML!`F|5v@iKSuu}?(v87_%Fsmw8xequRKf#LsJ3uoW^KQ`>8+6cx9r-ts zf&9M!_PwGP53sh>UPwYq0repLt{1Ldzb4Eaaj)kmVH5o~;7jxGS@rul)?O*5qvgT_ z!h_j>n-<9TYYd3{XXNs4P_5>F1_L^L;Ew-wth7}aT^ud?^!iVBVf;S;%4V>030>jk zEfolOvV>+oV66l{<3j9zXNY;NHw%O{o;pN_W4$s>{mI_A z!~HYn_50KQ&w@mKSK8S^qg$>dMRxCBjZ_v9zK*3R8UIg?F86;dYIZ&P_MZ;zLvBr0w1(w-o+tvc2ETh1mPYumys+eAWMqNp6v1+Ym6iHlE;-5kbV`j>Z+{_Mv(CI2! zsg4P*th=FO*vzoE<~>sR8)+tQ6x-Npknyj zjKZ)o3AX(Gr!nuB3FU9S=@z(vEHKeoynDfVlE-xNnWhl68nG%x$#yQj`u6-jHxTWL z^C)RKZ+kp63SF5GFdM7fd9>B$=FMy+7Q*Q``(mtc*N{ABC#_cFUa&e{ERpM5ueZ4Z zowBflmZm#H0fCFK_f3>Q)_H^c@Q(Q!#MdaGjCV zY$jRQRMN>jnjhR**-0WQR**~Vq0Qq)ayHi%leFeNb<_We{9Kn~;oBN) zpY~5{2`_l44nymt<2~XxG*Hnuyl3Yd^zw2t4(uk z*g&TxWX-x*bEBY1`=u7G2xSPUBZ$J0$PnzO_qNQHwTCO>MGo>&d3kkG%as}X8T%gx z&YsTqK0kOLCGq2|Qiv0(mJ~cYuH{ctpwWbi9XK>}EscS+C$Eo2p>CRfQltl6d~I4~ zTU%&fmA;Bn&TzNRO|yl3=JQ!NUr!5fzESt|k>WNIwc1%sYav2Q7!>X>bazTj(a9<7 zDo{*LggD0BDH`c!%@$*vR=brGXlaB5yI?S=D!l+UyP?DG*ME{g-=J65Kn>;8inY`$ z_($SmF0-bX*dQI+57r-VVmB{8H=i$Wvv@F6u=(nQK8l^R)uD8Ds8N@|OO2-Mn#kg= zit)`*PqB7hd|dR6ib5UNlGS~NXHI~a6UkdW;5s+#idr;%SMM)a`Kfk5&HGaV)>dt_ zg^h<@-)2^i*#SQ`!Pmdrx$11W=oYu7kYF-%@26-i2{V~6M;*_k z!SOt`iS?bgG01B3eXXS(=+$X)t}*_2pXO!1_nsDZ*KM93IQvVjA?V@kBv;>Anj_Wp z{6qST0*FKTOUs_T(Q&m!aI3qB>|WlBVwb3{G5#uHmV246sQea75$EBz`b$@J*uaEp zN7P$OjzO9BO||u=3!r|N4aS?7(btTlxo^$UzPfLIo0r`6jCg&9K2~(@5E@pdaL0>E zaWy_<&mQl9z}*0yPdv^*Zkk6oFt$tm`AG2a{KaHNxu3T2)z5vSPI>T0Ru9azV6mo$m7bf;1@o~jq z;e3ft+N*}`w%*Uq?e^BUbY;~|Kbb$%IO59uj;Cpd?<37{;=L& zUS7V^V`&NDv+XG zBI6F+G_H)0RON^!_MBPl!yqs#!6m6rjC+M4DV&xpwKX2JJROtTz zwX()HLEqhNiYs23;2+quY%xk1&#pzO(&p_yk1RT144~B zx&SLr(QZeXvZ_%p$ccUu`EmxYevg81-gk1v)`5>64yiH9KghPI{C)YeiwZ_{+<8N$t!`wP!6YUwy)u%$$8A-KKm4A` z)AS_EfYJTR%h>#bN|ctMnTG(|#6BDg8@8x(mFij=%A+#OU}W@1o5_yC_QC$(PE-fp zK0tU16T*3k2n%{Xpm?97HTXu6L!8Q$PN8I3Qs~P|qq^3gDrkxVyUiCjvpvi)-vYn^ zG%YZ{{JX|OIqa;?i^=?{U!i6XmM(9s!@CuKzHwjE)79~r!-5PO>1z5Fqai^>JQBix z5$)f6J0B*%(d{@CoxgH;A&sdZ4XT#|y`L)P@4zZ(=vI$ky3a!S=1qUN@N?rY`b~X# zpai4Z^iBJTW0!dhMA^91Dz;zHb|=jwSj+RkT-3V&hSi4X)$Stw1A=F2Z*Cj?z|?32 zU?!aQSffJLGm9%K8EgWHVkNFL8t||i+k2r;Kg`F5umr9|7i9amPB*}HjRVuo#?bGq# zDUId{Gm+){pl^P`c!~Ke#Z7ec4CZP*Oo`0eJO`5l3HnQ;-MM45a}yE-yy@a%&3>Yh z6->OGI`ADI@#|}RdZQ!NONAMjnwmSTj^k&Z+Y=p&+`Ia-4zKMx%Q-uixi+w5E8c8Bu+aU zzbf|?e^c*DL;MvHD}_RVC$GOUnZsw9T@JGCIfaNS!2)rs4TrT8(bB8M8YL-@So#Z@ zTki8HPVHk{%s9nYq#FJ0y4N>7H02yDk9}QA;Z>fRywP*{lXSZk!+Y9DC@faGuwcH+ z4l7%0F+O^@uSrM^neT@yOmTQLJZT7R#a0=29%s*(S>NBY6zFk^epgf1NeLc&?L^3jGpml;(-K(b1 z+0zc=-afm?MlLs>6<}FLS}IvGv{V`M&^}Hd)d71z+B+;PFrIOT;+s+fIZY&EgDbI= zeHPZ&i?x&(*U|ot8^tKc3TJtm~zIM0F6S6!Dck7%ve?-N!B%6{~soXP$heWrqjYL#-B7HiIhe zmGg2*=zK?T&tcRCvN1tSn^!yf#ue~73bkMp*y=ure5>beXcNp}46o^sZjbFVvy3X$ z>V_Ia#Mk;X;laThb92^ip`z7E2giUt!QsQDPJE04{6Lj5FV{bo{mz;FZ6((20RJyKAr!;F_XLll33Wwu{ z@0h6#6svNSN~&B@e~P574qBv04Vxy~g+FyFe`8h?ISCO>u7`wOOn#BeWBR-i-uw2< z1NPh}7bHN7q}Q{ZxuqmAxMrR<=f`Efn6j}Zh@$l$y=NzRZcS@Vz5bRzU;laT7llW- z`_8AhnvWmQkkyN$GgS`Fmb1;Ka%A(n@%e`RDKC4MUBh4`=P!OnMBA)C9Z_TZmMp!) zs{IwF1#kOw?^((zbZCl<>)x(hq9}9;7vAx^W-Nb8cm(E*ON6-&rqY1XNn?|i?R@*9 z*V}HThYx%Q-u_YY_3u~VL(1OW_x$^sCDo-hA@(-GEF2bE#j(uX3%Jy(jnZEf2mGl0 z8TqnUjN<*kK1&?o3UF`a_~M_x#89 zRNn6o`9-xduvq+jjJ;Gk)WJbN-vLSg+P1-bDWEx^nqax2M**$OKE3TeC^7UmFI{=V=wMWw4lS8Kl7gWaBm3 z0==|RnEJzQlIcORaQ4qT0u*Pu{Jf5bgD`+^{SFU{7~|kFCJOyj>WDsxkifn-c#(23 zR_Oi6lt)tJ8^VQnqCzuD0N))w6HzUN;L&!os_~Qr_$z+ujV~9g12+Adlm%;KuR3Ue zK8~1oE_PnTGsN;vZucU+`+y=rmr>;MI39@{wpOt(bA7rylfv^kYeR7(1>ogkr4BeW z^sVRnQB8$1=NoOMYeT>k55YeiCg_NqIW{vsApN3VGTpQDYxa`-;Vd?q-ftr3&b@Lc z8Pl;fqFjR&Ap{TgURU+B>|7%65b^_TbgXqz4cYQYgCdDGI*FRRiEORf>Aa~kqY!Ua z5ZBdcd57*rHgkIe*C)Fy+5T4Wc<4{%#u>H<#9Gm<(yfhg z8Ov6^P>m=pBDd!k7a(<%H7)oIq@0`$^|jkEvYtR51Cq=00a=Fv?+)V13WC8VuHP6( zY+f%%`#Rg~9Yy%-^9CbcGA4Yyl5?*5(p?nk>d93Oy%WC0qnECvPe~()OzfdVo9h^P zvU={^)@(HDM_(O}4^p+RdY}|>#e8HOTI8Q#Rc+%t9>$k>qj)_|Kop5gDeKZ>`HUBw zA2ieyvv1G>-&a2p^>kd$=~(yAV0Mq=?A4WZ;PYJD2?y==qQ~vmf9$&A#dYEzlQR6` zJ@Dq1dbC%~*%Z9XBDGe@`ido&=H0%-Jd$R4hg)`6U{zBCg+MkQFNQMn?&i5TGg#Z2 zUz{y(On{EB4Z`;L4}$4us}lS!VJq{?T~R^)s-cyk@+w^30hHDHh9#~nC*4JNxNeX7 zRywY(Cla{4qTa^@^;<6E#hW-_jluCvRW9MYMKKabKX3Sdj@ht(H*aynI@Fwza%wne zDEocEsxrrb)!+`z^3Ldz)O?RCkJ`-*W_cU#EqaFBsp0sfDNX@8A&K&GzqPgglC9W$ za-tGDn)&PQtnp6WC_1~aW^kjXcMZY8xwlTfn(u^u?{7d@&)nZY3_YHB(N5b}G@s?e zNW-+4V_CKfZAl~n$sWld!eH$l+TT69(ZR=RAo@bQ?MKaXazC?Z*ZAGsuC3A1&`K~K zOrlp=%W=6~=v8cqquekhi$FzmR%{yuU3U}m6$Lv%zF^&YhAWD!_S6%1$tTfpVC{3z z)2dpOeVE}#Err;rotZk->-DjY0XlhnYlY<4MXD?gS@W7a3kPREmFz~Mj(mWPJa%1l zyVQomYkb>cv*1`)Is6vpN6))|EaP8G`7d6|B?Nl(%qBftj-b@jXcD843~9c4KdPf$%CUZ) zGIyBxXY@)K5)p?5qY~8%EqAwv>a5>wvCLY?G1ZP1_*qWuAZpqC>oA5(l8Q6(YBIdL z<>Sr>uU{12py6Ko02;zYP{ae0FY&5=s{Wv7re^6qyWXI(rE$@3_}3{CAxk4J$x&JW zZP5P63U_UzOifQr)Uvx_kI(CJPcg*4#l`x}1KnorM-^NWJ*ucJ90RGbx$de|D)lN1 zFHtqR7!bYyK}gwpZ*Bow9Cp(V@{lAuk7qY6_EpY;r-rik4@!C>D#>N{exJm>07;Aa^qIh{I}jT4yfUdhkl z{_w}mN-0`l%!4dOF%a+Ikp=h{1y9cTcScSR(4cvT(c0woZRJ0S69u#4SK(ZPyLw@^ znq}GyjK?W`+K~xXc0q=|FA^&s>BmVj20r=8%(%7dnR0^|A5mlO+!J1g9)F5I`}MeR zMw9&YXZQG;dlKzz;RUBei~@JZ{M+^jEs zyU4~J_AEp?>PgiH`Pugr#t+qupD573^u=62!&rll=a`0uVUj&`G0amM-Zo(@5wZ7l z8MUP(&Yj~;$sCT4?6d}5*0`KCHCd1ZEBo(s4q}19>@q0+ zt04*-KCG-P8Oj%1D~Q{hp?Q}g&q13fj1J0eyYsX)sCGNN6D+)%{oRU@c>>_tT?3k*u5aTk{Yd6VIK zGb+?I+bbYY#`oWS=VVzhoz=Z#IK#z@WXHkkW(= z*XOS%q@&j(-yWDOK^9yn)V}rHJMR;Mn}xFY6v|r>b!!p%e3hbCCM>kwE@3HiqZ27- zlT2jxa`GIVX&gZ>$ID7JDb*cVRoNwId7UbfZhm0Ypzy@liA;y-8$HKIPGu)X?T?SX zAB?FhFN=#XiW25(qDG^yB}OK(>K%Q-kBl#8ZNkufi~T4ma}wLMEG`c&Fw6;Hd7Rw+ z>n<{)ckfJp{#jene&u|$iIGNShwMH2j_V7Q)H~>QMb!+l6A?wSnukkGkKbL?9h5mb z;#26&cx?%JXfqwT&v5gd`!;TZ%_y8y(l{YP)OcHHQ$r}nz%unDeP1eNZl1gAC-Fd1 zs;x)02V9#;zDHbm<)Rxrqob`OSIm-gBs2Hppf3f(-^vE0)JJVg)Ua8433X{nF6dZl zgJ61QW-licqmb&z_=cF0o!d!v?toyWmGQE9rC3Z!Z6)C%`~*B8A?ahJ|Y_b7wj$LXxk`PNRobb)9!&oD0pMMy0*>JF;HOk3#XI!L*EntSa?QYseNDyM z*Ce}x?o+*ZE0&pYo2@+N42#SxkbM)R>wLo{i-|#~duV#JT}mj{jM~n@E&tw&=Vvuk z9~alw^t*Boc&>(tx)dZo54ZoE8Qm5t)Yfs|g!1{Nmvpe6P;Np+l4$IgN)>Tm(W6!L zhP#F$s_99Slb@q-kNVzvc0|!mPDq)kMhah0$UFUfV;!Sthup7>sAc?8eMt8S_g&B)+^ z?7nYj@6XiP|4@X^V5rBt5Yx$4+ggw2 zd}#|Lte#6=t*CUVF6s72+P3tR2WPq?o3_=vkFrN`{4ph9jhNoSWzo{%D!7`?bcSl| zvNK8(lIqOW+*KH1TvrvaB(v;gA^we1aEN=LMo&3$_Comp&)}bD_Y^AAO=S1<1#Px? zZhhw&WB7c{`H8#PiA<2QVSU}Tst6GDT^&D=w4vBkBy(_`vzI?C;{rtP!+M5om=7Y#`j`n5X1W5ju{iWR;Hw_3yJODm@k90p{SY+B&nWR(WoCi8he||r$_mE8FCi}ppBXKSWEW4 zvg%PKp?z4Y16@-XWp&HYIO|u#3bzsV4|INJtrxW~uUp75c5i<1uFjtBTETcNI~5Gm zgPMh+O7_A4ou(Y_=)L1zvRj+dlk)i?|HpgU51H3_pT_o39?hhuhaU|@y|>FD3j$>k zGM(b_n9!5KYu-Z!Y^!MI7^hKfeowO;EPZ?V{g2g+XVb#kSa?>Qt@X9_SR4;y%XTLs zjw`3eB^QR99^f5z*-T>|^)A+$s_(`Gyl0PTP0n~m8KeWhk?w?F3@M`=tWnA}cZe?v zVsdf-vrs6h&v8%fbF6}aX!Qw(OLF1U#Pf$v26P2}Vp&J3;^&++tKH2hZtLEbfYPNB zvL4&=4M&9UJ1^|&_NW(l4XtLp0UT8eq&xfVQbs}k6 z7({FwBgOFKI~v&Eg(5|@_$iS5n-7PA z`O=E+2CO%6*-0LpJo)e-O>rE(;Czm4@BO%&R5+@$weC*8ms7339+5u7Id3x3+(Hc| zq<8xlMRv>u+Q~4P?_=hAFvWO~fpq`l=JByOJKH-VOLCHuR=t;vC5UC3QsEmlzKmb5!`$fTWqSDKSPvHU@@iZrw(!+(PdcLas z`axC&Zr2#1{*`C;AAR;$9Ac0?qCIdBu?DRK_y*Nlh4JO`hfTu;po{P>YP@JTn?*nTrpL7}!!;f}E_-Ryk zR(w!D$h6T9Ht{KoNXriS@g8c z+lB}9V!D=FNSK3I4A{Pk>wL2^$5aXYp}TxSm!?KMkkNvF(zMn~MG!Zgs4Ro{0gN2! zol5qLBH8x5hwg|W=Eea+5GQ63CA^B5c~*is*wy|;aV8b$6h2A5FM>V^G&%w(2yI-S zCJ9}sthgm}KpNG0CrTP*DZ3WT;dPZQmwq4+3$)S&P+S`ApxZ0)C+Ad79;?hy*+93q zawfS>?vb+rAov9dkl};H6CFH)xT5m=aBwE}87KT~cY&%=MmPe?FN!UI-E&D>2x!MD zlYr*MdAZ;F0|-U5BSiE`a1S{VfE{NC4@&@o%DSdA&Ln^6LYgzjZf&_`7)Wdo{?9WxasHt$Bzk zDIkQa@q|0h_!otWPyq3-fUXYL-c5heT0KWiA!OI!BxHB|$i%d>iA zU~+Fg-Avoawwr9ilcz~Z1bTWE?c96`W#vT~s|+C{tlo6II_fB(w~XP!%c_?P@3j(T zYW%Ws$nXD&nl1qNr+L2s;EzQ>eHS2)Koh))lO~oQ(OJiSvz&sPQCS0Z~ZrT z=oGFwg=_wI;dSO3M?!G_Sf+=BI^%EL$K{=AzbMK9N)O3~D+;$GjF{Vv|3$%t`MJtK zB4Pnjn*%u*{Qtl}XI03OQLU4QjJPeoP5^W51whFR;{HjnD>p`H3IMCq zp8|pWjz0i59ytUh2S)vEI3wZz1_dPnsOoPW?jRa(kplpb8eka+V#E-S%M2)ecR z-&it$Oor%_jsPqfd!lECARemxZB97bTVH;Ma&iAR7X6?6{(pfW5Wh&H45MXv%P=3KyD@JA17SgPu%93248z}`rXrYIDH;Y{}-qF;Z#qa?ic@R-@JRr zC0*^kz|&lcb5uVN-+)tb`x<}%5JE-*)?9AxpHzXpzQB1p-GKmb&aHtLS6Q6K@dC$D zjzYK2LQ5?XJ&l2yzbKk|_ab@@NUrTFBvg1fMg02p)NR6K41y>fkiLBq$hM_-!k?c0 ritk8mkxZ7GOC{@Y{r}Sq70iRT(Bj=Wfpf2S&J6)&UH`kZ>&Slr<3{w6 literal 159395 zcmZU)by(X?(>9ER65N8jTksSw?ykitE-l5q6bb-uJ&vj@_Nz+5OGzoO4dJmWCn@CIu!E5)zKGlAI0_654kpBvc&W3&cOkT7T(~ zkm!(<<)rm|k$?B12UFWyRvusHWC>I(9Cy}wf0OSdC2ies?Us{=49uHXHq*pzeRS`_i+9SBanShbPf_oxx zCL6hv4C#r8C-o4`1E5GdEw;F(+a{|q!)+T(31M>p zMBj)xUv?Bk*F|4548-rnua^BowFUo~s+Rbl!Fxn9>)-QyJ7}C1GpXgZPg`CA=`zOX z%eISwi0*u{TU7sw9x7u({;$wd06k)wi1>wI0Q>(eszU)JKe%;;n29YW6o;~*!jZzf z7EgbE#AenoJsh!EZrE!*y|{l?{m&Zy-N?U<_=UV;w#Bb*j)-x~$3r{r|IDl^r3*s) zf4crN*mM_)5Bx{`5Od*w_3^*w`&WXn>i;3(f64540svQqnYtPL7^lCwiKOUQWwW4_ zNJ(V$LvX;V3}plV|H}RycT=DY3@HX8Hdk~c?8O>WT*!v`&lb)Uc3u75Ze!2ntAT0F z>wig04r%n%6iF4|4&8yySsdleDB7zn`qvCTLS)n&W%wU!peMg!RBHbx{GN+A!4=l(MDHyyMY~C z(qpR$q-vv#oQv(QjBzzWo#oHmUsHU38iq5ow)~1>Ry9y@GSw(ewc#hPU&1bTwEDla z(!qyN`rJX^)>$#ze}%cCt2i2;lb|r;T+;p;@m&jq%`t-99ak%%TBi}CzziYnxh}#C zp@AZtXZZJNWNua~_-q;>Wd_z*J>sYlf=W~W@AGg0%K&pVSZNT~LHpl(GnOS}sywX! zhczX4bG@84C7T9KRlb?Qx3ZU+Q>zRw1XXtc9?Q;;4I>CMWC$DBbZ3D|&8_G`xU42; zn_b<9JF#h3;%rG;s2mnsOqDPWJhRnU;{VqvBhF`tsf;OSf`R!aKmsJcp+h7?2VKWz zXVy_M=OM&Efr1`1>3q{u>7*E1K+x22{9G0hbxbt)1qQv;IT%BUI{^tm2q#Ag0RRY- z7Jd7Z2%+mBb`cjDU}-9__ub?UkKIYyAM1d1I>n|3Dgl?x@@YI6)(O1u($(hSK`D)4u3 z9j4Z#fm!Ag475RfB_;tt=t7Q^4kLIJ6s5A?48kPD4_(3!Uaqg?#RN2vC*?8(cl1wK z7;$2_5YP!{;9HcHkiji^J;r0SraNpMo;d83?0`vx0J1uX@k_E=A$q_bOvh>q<_Z*Q zJCs$4#bSjUyMY}~jon%v8f*PXmpow~DZ9|o(t_vJ<-l@vxd?4wz;vPbO6<}|gix&{ zveH4MlugHatIV-H{7|KNs+q$3!TnpBbbDR@E53{_~Tnlb}ptu6_cwSX#VrykmO5Q`XEG)ZlFyB|!BFBmXmQ$G3O9TuHes%u0@glCC^()N4Af zu!WRVLV7?gDLu1{RO_6LHk)SWu$(ju0`eZ9rcx0eF~n5DkU2G;HHB%#hj7|h*qW(b zIoJ@gWW*S7md!3G_l|^U`8pyz#8fg#Ixm^h3EjIvg3t>R>=$mrRBCE!rz%jb%C{ys zGF#K>&k7^}f>tR03a~2lYZ4*i%0Lkp4I5d>&Jd{AQI8vm&0G4A8=8suOv`NCPl4!y zPr)age|EFrf$q^5Ppoq-P=u=U@~9D)waNKo#GA4s74PjUnGqBGcE-%#7lv)KTHj!j7uc}a z$Ke+ksSC4O1a|%oLY*ZY9L*KcBFE-dtRMDs;;h~2#r1wHXlwtLMpAei%*d*!Y%juP zC~a?z#;uP8yWF?Ly{zyR!PSHoCt;y|#bh5Ew$99@clCdOhDzws^LaRyS6FR>^S3S) zqi)QAJ>vsWLpo$6Q!^a|Dq-9ja8|NxrBfZMC!jnx{Bj#)LzvlHuQkvIgvIx}uoA}M=7;zUWc>58x{TX#Sv!)RcpV1wJsGr9zQgB(eesGoRJfwZUYuZ;@e^6UUvb zA7R-V*mP8zB*c|uw#?hsOZ5!FVuW<`^3rfl$}vQyiYv}_LSsXP0ZI*0R&R4{@^v8< z8LmuGvaadLc3_HrUgjz7f4Vy}_S*2rGW#X$;_6a)mmtk52?z?t!;P}XRM|jsq$CG7 z!gK5?l~>1bxzUxZ(iYo87ICkLi{v{vQsRC<02vbL~f=3eRuex&UxiY-u$w!{PJZ7 zd#v7fzbNwdPr)PA9K^5l_A&6JWy?F?cT@TA@0GFedT^|#X#84j5gTU1{QNw8SmMs| zr&#;QCl1UcE<+HuGT6nk61F+yzB<1&xa{1X_9yU~=A`pFQ0D-W5h@46GP-<|)=y4s zBm1J9gWQZE^KFBxI>u#-Orc`$+Mk|e0#mTQK08Ou`jx+5!nOeuUFDR<#qxnfj3;`) zB~R4DZyE{jO`x?Ak0uEf9W$`^Rwua7(sk|zm{;crmI5LZg6V-niU;DC!pmZ-(4tRl zN=%q2NizM~-vIXhxtHJRzk;RQp8zs5_%%9ezcOc0*pq@&gOTi)Q0hM>T9!pfsW8#~ zYiq{pTlD^MAv0cihld=$xJiLYI{d=Ia^m?2OH{^^q}Qsxs-0zJ=9ZA8%YdJIMuYKz zh+>WQwBA;Q)!t26nLUdh+M{b+7!dd(gG7M~RMd6>5*i|3s^t%ymCH7Tg!Dj|${p#; z!q*J|70OA$Qo-w>UPql|Ab?b54noxyX0XLm!>F?Y4^fw7C{Q76A|bMpP?95`PBNC`&29Fb|$uuMT@T$f<8R*gP*sEWRZs+bt@1H+^R#sQrJU!f+`mB8WKEJ>xA~G0v-ErSMV#9xV zjmJwFVVR^{k*wn`0a3S}IGhjoZB4t|w7jaKtc)R1ka&C1-g;0@SyfYQ6%dfSF9$;S z)e?^b%ygbyP1%psS6*J<7v{C!7!3Du*6cmf-XOi(Bl${|_Mbl$Kbmnd*MJ^TWtlv3 z%Yc_V$Dcc#sq+qzfBc~=wm9RUF!C82zU?7QVMH&6fO)KL)&(^ZaylsV^$&jUTX=fL z&D8Ok>zAhM|9J!lJyx0YhPocROI(U|kHu5FPIcxO`CtDz|JHp&EtB#*Zr$#yfAuA( zxBGGZF_m_)Usv%S?$Pzo=(!4~eGa0vSFbCmF8kzn-Co0xx;!*@ucP(7hxpGEuzTJ} z)A8f0g_S$pldkgU-V{xDt4va?A7A|kIgHCqnQ`#yBu+Bnj=QX(0(F|-t767xiGnSj+#zqr zCbbonyb1c9sX||JRV^LxWHHRcWt6Y1oZD<^mO`Y52+8fn^!28G9@UhEAmidN`zgn$ ze60@`@v@a;uw+Q)O;*Z0Pv?Hk^;VW#Qo#i>JmwB{aQahNS0Cldr1oA>h|4nXW}7!! zyuw%Ir%{79fqt1pZkx?p*HEgdy@G~NlUj3>{&upc3oBfq_%WAVb!t)8`v^DvrWqZ5 zH6}t9LJXp4XqDrU)$MHSw&saFuO)9uV_D8g4fkvCd*;a;MJUuS zbU@4lZ9ZLTqReoqXEHyjs|bh8_FUp)dEr;mDx70yaKP4_W9jx_HHq6^ysGxr@IlT9 zDvD-d3>qh?)*ItcN&M1c;n4A+sIqe<^AY6>*f-OLrs$dX=+VfNv;MEm{D{90gw!Yx z6}y=-d$d|jpo}t@drsK+`y%``A!gA}KhqCe-#%BSIIT%WF*E-+?x!^HGZI-a^Wnmc z3=@Fj8>Ta5DQ#ZdM#(YAF%U2$1Cx~i&BL;SIfM0Jb1ZfB_|InT5w5}mKkS?M5Kl0O zy8Il(uB4?hisu1NtG%!|u6fTdZYP@ghoA)-YNAV)7K(n)bp2b&ZGRf$CxkzZZLs87|Ls;2!JSiOk`vpa;_BHL0$ycbGwu zxapScdGFGh_U?z-#LETKiI+z;o7wMbcC)<<>YP{RKc5+b@O~?{9FF%S**OkQ zZ*7JyJrQ9fsjN20RD)F@JDb9&-WU1hOMSi9ueJ1zqQ`cuoB~1vBogsMRjWpA&&AZ0 zYq%$ii5HScMQ=R``w2l{Ugp#T*sWz46ge^lmz!B$TT7pK;+@uak@y8CdZ*9DiwnJ@ zlR_cT#*#==6nACrY!jTVITgM#WjAGDh9YauBqJ1s(A1gLmY4uPX>3~?iUNe<)6r>@ z^(GlX*E{^g5kLfUmeTn-U_Q^lnhfm@&sk1Lm2E@is~!jqABA2Dh>rrr50+AiPPH&1 zl-I_OkutVWj7`$FU)d>8JWfK}?nkW%xjE z#Z7Hcdx806<<`&Uz|6$1cIf@n>6D+Z_|$R1Es?}cmru$~r^^|3#_OG2XNj9#⋘N ziHxoXHgEvtm`i8=TMe<()9c&u|gp%zA8F4xP@;c1E>!AqW2LVJYY59?dZApHL-*#YzlBli7#AbKtoUsK2lr zy2#y}9%XYUk1Lyv+T?h64U@|yJrnZOaZ+<_3Cfh_9A-qNm`mwMph``Bt#cA{egh}t z8RL)2UxY^x<@PJy6BCTa$^I56-y(*i9PIPuuZn5VxEgTnHc{@*kRMq(7^IT#P-9hk z#h;E0u^z-zn2N*qL%zhh<0>Yd%$Gq^J}il4qS<4tV2QCeDJWBNn~L&mdWl8F)7UMi zn2-ti(M;^vp489Zz!|$Jt}fMo=lq z)MVrrL~Gti6zDmHGH&^acfDv^Rv`~O*A}FvD5ZNjn$vFdN^+naCy`GW5*`u@bH^hQ z$o=laUHY1KFC)>Vloc!gki$G*-f2_VM2Kq7RG!*E#~|B&A)jUz zmGjVk>BD}}@8~W0h&{~WUYSov@G3DimsI|$!SzZkQ-cION$t z#Z65SKYtp(K#vs8C}#82H*nvdw#22Dh)ad3!VkV@i8WIv4y)_x67AU9#>FML7lFr$ zwRoq!qVYqqzS$gDQBWSX?j_sbG{CJwSKfCpvt@2=f+`r;X7$1I6AMiw!mZDiT{jz) zEgKP(hTTsXrKzUab7FngAz2;C0{LJ^0e1a=^Oi=FSTGQ5E)9ZgDLMTDdhs?V|U@bn*DC5`PQ{WFSGUF9Krd$JmW_23w{$zs|Z@FFXh>H2YsUJu97 zNQs>f92@-A*M*u&^(~dsp(%$(MS`x6xH&MN4iE(Eg}XgS)=L ztZhgx^^XJhcYt5X-NNu@U+?Xu+1wRaj72Dle922cs$pvXLwSjJ#&B|XUFYh^7iRL5 z)3wTI*d=;M+7bP*@qwI%QZ7Q&(d4gPZvS(}%7-BlT14D^wvY4#dr_{K7@9gBkxDpL zpNSA_Yh~*%^Rb>OgS;q z#3J$yI_z9rI|*WI%DiNo9tDr$jEszkz=(V}f!ccNr1Rl;?PJ>Aoj(+RJUtaABQbUR z*7@z`N&5*?3j8#j=YjlHXJ=zm<5&NA4w|4%<&)OE4^$!nsgh4yQy#-&E_N z>Dm?zBHqCFbB~D7eVLV;%aAt_w#$T%OEk?TL)={7rdofi+3e;ds%T7O_c+PgvXkxO zKBi97v??~*^`#`2uih`NC>gwyZ*0fbhE=OulBAk4abL-Cp*`|9S&d>m^EY(_^TbUU zP1F!u4TtoQKYZI=Ntc(JnqkXaF8QQ%?K=iZxTU)g`xpvizo92)NlCOjZ5+9@N@(B+ zHFNrgkMUVNu@WnsmKE`Q9Xya`Z8~PL4!tI*&V71E;s4RGKBF%)(GSQH$-Z zQ+dP8;8OuDqxrf>b$|Tuu@sUHg$Hj%lvs<$orsZ)YfTS`Sq*TA>oH27AAnRn4~FVHN4PaKxo# zv7OlQY*O^4kCYu$$suFmvD)RQTD@bcPhxGU%x50NR@sm2`HQYmhC@~gq9POj zOuy-U0PNuhRIh`IDqDU@$2@RDl?wNf17bp^%>XFeUwe-Ku8-L(*jEGbw)28gt?jI3}EXQn9 zaAbc7lAd}A8=R)`)%Q_X=~6^IP!-X0(B~ zScZLlL27iRbU&Od*n>fIOmw~pfd2Q|r%K{7yp%G)4s%+0d3i+UvyrJ1lS(q~FM2_+ z9*Q{4Vmdbd2(I#R&6eC0rB&SW#RU3fzt)37YwbUj; zldWU!U})A%38Nehk%q0WLdSJRX7@{%Sdrs)y$BTnQH6fU+_7{eM|Qb;aRoX(4kTcG6cv{nE!swb zZBteoprc|31Yj97GK8Rw8Q=2!u_h}IujUBe{1mhmZ@%!>?yt9YGICwrZ#SsEU9Z%x4%(n(fJ>6|IQzL_Xv77bP zHK^sS&S02sL(M_4i!|XGx{VM0o_gOy8q(YJyC*urCxk~Bci;3r=*0y%$AyFt8XJ3^ zw_+ch$^Q0VHz}=G22%mXoXg7V2|hI?I~R}^?eBBb7GLpHnp(R51c^QGOp)`QcYqp_ z7zKa$P7&z;4;MhyZNe0?5|*sWN13IT{^8`729dP3v9L(0s&eFJj*l%o5*^Fn+A5)a z1`a3E?iF-j@m&7>dw#@JW`jWBIzL>`EG_jGhhr@^_|VQo|MQ>%(*vlfJsjWV^h-WG zE+7y?85MXdB3gF|8eTs2??gPN*#1dk@5$=xr-O~utgTTC?tkrx1a1NZM#shso1EC& zuYZ?ycNe^0ZaV9yeTr&7X-kPH5adNbh2c6JV!msCf+BuKg$eO^UiA1Yi;rOqh&1^K zM1{$fXxn}F>@Mhj*c3gvzstCpMExMLuyLHTv`%f6O}NYLu$7=08258#_-3=fUP0*LFy!JJJG-3&2+mjDBkD9pWQj${* z+cj$&oV0Bricf*$L6`4aF7UegSbw~nX@4gmE5lq+WgTdrU+~z!cf0$MRXF2ty7gp* zcJG3=EiYyHkL@2<^5(T)@aw1DhKK&^=FO}NmZ$yNz(iu%quYuzwA7s!e&V7ED(B!7 zYDVn{0nvEX+RzVwwm#!e7WO?3Sv%N@Uo*3-bztZVbM_pmdP38D6$zU$ zZ%q|a%jFGp%?%P>Cfg2HTlqH~@1EX3NH{`oJ% z6vT|R=dEQ{lA@j^tcOR|>9i6MRc9EX3nI1smiL`Rj296RYHzG)+DwD-|TTv&Z2bRPW4oi}TQae@iM9-zM=;$IqrHuht7Gm%I=lT0Bd z^vzxwTU*X(lef*Y;ERfsTn2UbCG7R}G7;Bmi|iLHJS=7M+$kG7mClkc-+MT6V{PDL zuDSwkf8%!>cIU9q^>J*&K~g>!R=sblQAO)gI!l=<$ih+PX2hw}=&3Y_H>P_bN$G>J z%HuNmp2(&e<}$F}5M|@NbS(mJf&zZFx0P~3h;vpWc$+yDD)O>LPsr>2itp$U`9Y#K zIruB*PWiWQX}aSO)K~r6;`GLCySi_F_wQ&On16#scdh(ZMX8R3wQ)1db7af*)Ba7N z6zvwA2UAL$tHJ*H@M@BZW4eOBgZaGLD^u!JLY?pNakJ1e!A61UcHt_r$REFvSwxaFn>;C3Ou zc+FEjS&}?%$2)6u8HJaIQq3x*dyPV&`Y4>o-DdEm)ZLSi07Hk|d%ZN>PqicN`owB0 zpWs(ANsYPb=oz(%aynOO3-*7gIa@*lH#fqX&Wk<7?-pi%2u9A#XrqK6adC0|UTLN+ zm{R4>V#jNYE0Vv2)^&juO>zBX=Z$2^)4h{}% zx#?7as`Rq0hZAO&7a|?AI|#@45#ETBHKCwq%iC?IT3)@VC&@tJhUy^bshy2m;;T8Nz0p+y_G z6p6moLPtf40SOT{k`xx!1mTx`BZshl|6bx00AfsIem>zTl(_wv8q)teDe&n}w(AU+{59f5L+8N1LzDpRy zNAWZp-0>)Ehh&Q}HhPk1>mBwTdrf-?M~szhxF59Wz8qK#AJ}A8Mw>8Ft~9hPe#Q?E z2q7KasaL|-z9W;UPadiU3!wY8}YMnw^8Utg$tj-Y^u2s|#JsOK-WSnE!D&cWfK&3yv3X{Mla;C19o!ej7o zwR0I(%TvPj7S-Z#fA?aaus!zKK3?}^$%D!S%E`mcl}+Mj`?-92bBDd4pq#uX9g?*> zdI@i9g5}RB#C>D)LTm7Op$nLMgv$0%dutAU&}|8=2cH7X4aJ)Y+_N1XuR@o5{(^^b z48t2i=MQ6C9fT{#c22*^pMHOmw15aCYCfG7v~HbTpPv5hW`P@Z2|bb&1pIkV<$Ec4 zQ__7W$X-IW@kr9V;>~saSyFi{=Z{6259RaCe7Eg2YY1W%kHcrP(Pc1X^bvVxQLE;4`hx}lgo;)`0`vhc`txj~mj%&K(h0eV5WLCo< z;hy;BFBY5d$VJg1qXZ-_ue6qnN2;v#!Cc#Mlh$m(M<%>i^(kj9AtzOKw_fjznsr?N z+`Y=(1nL<0<)YL2=k})3nJ&R!3W4pE-LqM7clHgv(Iz(OB>h~`+35_;!bgpEJoYx# zE(!BuERkcZc>S$}J}6eORXS&rn!$-ZmPOn>_5Kn-gFYCb`9=b9E1x zZ7UzeZ$<>-)JV1F>6n=@E9A0Ui`y|nu(}!HPxgWyg_5W?i?`p|mRwOkQFS``Jt)7> z!L4qqwXh9~ebjt4`Zomw5(M>oSFMF0vDssoasHS+5r&81x!6Nq_ZY1XT`l;?M3B0@ zIC9_~bvr+0I}qQ4$4^xevc0o7NR!59nnV4t62ddb%=Zx~LjWU zKD`9D>~e4nVWqwnUTQsKzhgNkBQjaSPwObFPh%7lY3;#gJ#{^; z!okk89p3=EIC72E8owA8JR8|9VRF-2ZA{I{OWK9JF#@_sm6cWHcflcj)WyylGDAMk zq4A2FAR(}gnL$*)39bE65r%7IR{JIg6hD`wl&Nsb;H`u6B}|2BnsEYbl(c?uWg;w+ zTx@$_Y#gR5{unzfEST0yRBgTU?W$4*nNei$NuAR5#>L7~Ij({>07@_AAi5$nUpFE; zT0Y#wpN%3eKW;V~|IQOn3{TOK{lApX8ytQ?iG;O1`=aRGsOWfZ>pjx{;BV?e zT^I#Zih?=J1ygnJu4(uWurords}V_jx5ujmZ8p$ivx}r+B8|0$YjDwqVF5{gLNLctCOo|HlBHMIEj!En( znYb8AOmz6BKze|ZMjaR{$-*c~3aMPNunEl{H-wj$TC~12yHCA9N}yi+%4N-Z?MH!! zGQtyQO;}OAig_s>mjoN1=H=UQ3pM!6km55V;A}-lmY7AAodl$pLV_732P27#Udo|4 zSCoEHnbo&E)70$gY8*-@dQlgRs>Z~xmqA>@!C}UA1w?Qi5niuG3>Y4YfDGb3o&c2JG52<%piki!{%RdseTS42I5XAt#d=>OZFdsL zSyQC|+G*q50hC^KJO zvS%xC-9Vp;o7Q;>MMjFYv5oQ|lc@Ch4B+>g_d3F4_tY*5ExBX z`iGgv%e1sKvWVU<&0XOJ7pJm`r+I%KC`nc4AQY682rzzco-Ia(jjXt>E&JerEZx?P z56W=rD;}4S08?T1qwnWMutCv3T%T;2!fisA>cQ|^SJ6B~0zV>Bkrz8m;O)`d)~CB8 zVb2?0R8-V)C6*L5O-)^G?a6Tb;;d({1w}uPv!IJx7d-dlKPxJvuIJN%5~pWV_5Mo> zx0}sB1bE-hw*Gyw>UD1tEi4TT+2kp&0jtq>rB6K_plA>ja&RbN&A}{o7CUa0yB^AL zE_Ysizt77pe%rWmi{4S;guZq+`AAi;o_c@o=HGy;q8Ixn!|6AZrldRVqjm5RVCr|l z`s1f%w>t(Vbq#Hj&Ul_0vzVqo^iNmM51b__7<6N@iI1cPHBU%y9x73v+xP>Yj)OKL zhf5lf6f7y`lun-)!|-lIr~0O!yZ1h>33z%YG}&;C)pL!>(wx&|O%8;xsI%c0n$Avt z{O~~$z4^F1d{)xc#ia0YTE?N!mN;8VDf@uQJ9pp%9$XgoZBJx&X6fSN+FolqwY{*0 zl@pq)Z}(dB_vOjmVMVv2m%joQ%uIdmA04))PJQr}w)^|3@bEkm3Y`yhC}N2phw)|v zFqSrRj9MI!qX(Q{SZRs7Fk`7O5C7EUr}$Q7(vm(nwmV1pYCvvuVq8Skkz>+zdq-ET z5(&$gq~6t*vs1P`YRh)DSRRsY?D^+iiI7<^n>l=QP<2GXTOjrI`nitjLX^Z$Wfvwm zsK^mmovCk$f2>rPCWy~l@2LQR6eX$f4eA=J=;2y5!EAX8T^ms8M~7U_Zx#^V6m!cy!1^Si5-?y7le!WBPp%D-BE7(q|{jdmOc8^Z3zLu`0 z$9rV6DH<#ATmHc&>$rF~4UUnj6t^DP{ysDamvMA1)rO-4$d<=+PpL&gvLG|K#g zDr&O=5RG^0sJa0!1srSGEq(9lL-|tJZ{f&o3b60#?#R8Lf4s&_VT?;qw{ok}`Mvf~ z&`3$4dwG}k%d4Nu*R);4(;b-7Y+qr8UG@@+^yO=+QNx~&v6hmv9g;#{2V>b+cRy~G zC1{)ZZXF*UitBx>QPY#Z^C$OZlyQ3b*ze;&BBzY+-pTgZ%sl2 zcaZO`-6;jnyjT498<>+LEgc+`qUFL$lr@-3$_DP)n&jvI)nBA5o%&i%gVTDVIxUxC zRnu(F)HFr$;ILr(vl=vE6@Ot8!!>*k*>3cOt7$|W93PbZW1H|}2Yu}>8Wx)--Z!tPI67kBy}IO0 zc1UGka!MyPUmS|KIx<2a)7IA30nckeN^joKHod>lNYx46bVtO&G&e(G0lT$Np^}D@ zDIdkI8$M2H2PWMkMGQw~#iURk^vwIDN|*o_dZr&562H3E89cMSC($m z3D)Zf(aB{7%xc+;GLPc3fo9QwQf@{d%rcOQa;i?0DXvGqI$3*xykrwmJ^A?cN3wRS z7d8THfa*eE6{Q-bI%>a^i$l-zfo7O3-{9gm^sc{1ZH<{`wx?B;*ixU#r6*mtyI`RR&(X*+kauKE&u`{V~U46 zyqs`7#h88-9RK3RGg_($-;|pkK=-vQ{JpBu(MX+wjICnSwkCn=0<5l|>D;>XUR5aq zBE_2rsMVrl2EUOk=&I2cT}BsftO)xi)Tp(8ZM6%As26H=m|&t14=|Rf&7`8!1Cr0c zstfulaXSvnl!zqNv}cH=L(4A;?q-EQRlxRqY9_5i`m~X`MbkbT-Y|SRh_sPDGI08g zVcM7Vq?*|Dx~_Y$iHmz1^LyWE%DQ8r|Hg1;b0L-NwbnjQxy7Vt%#o>$?DFEb+j|=U z+QPTt1<8z(MLwGp#z9-$aZU*ypuSUGk!QOY@Z{{TK+|2`7@zd1az1Q48KD+a1YI*; z=>7KJWtWnQibzI9Tq5j=e=Ueit4P$arymedzmTa~oF+{3;Nsb+%su&8s0# zUiFl@igt|1eqfOspNy2FBYS*&JR&lkU6Tw>g>jZ6ctB%2GIO5Xh=Lg!O$M-?# zIjPnY$K8Ce*el@n&r@g}`cwMz#eg@9Ur9caz@n5IO^ zrg&OLAthoz>7R1NBN<0Mmf)^{QLzHYr2!4-Tp*o{<=f;4%`yq4<0r4 zeZV@sp$IXbbU)b7?%`(@;S?XOI%S)ltg{Z=U%SDiy}yy1y}G15bqF_b`GwRcK45;S<4rjZRbTkQ-}sJmI1$eRCHZsVL;CV!V*s>5Bcs7iALrH7 ztKq5;IzfCjjKE`qnw_C?Y>gOciZGJ{Wx} zV%fPd)_*j}4PA7oWHAX4!2UXk%ebcV!?&4YF4Wvfd{F~Y4uP*emW-RX=ii^?3_hWv zA5fDqf+kQX=HI>6%1W#&`GBO0B2(hN%J1BECR*Ld)(eNU*;!%Xn4MBszlhxf@h*C2 zCoAW8{Q8a{w{4HPTQE(@?+^mQFvVgt5yH26og?rSyvLv75kRnMN|s74k*EHFM;mzlNk_+z*^KH{3QmtR9S-(t>j1 zIZ63N`FeYN)0@jTxUwd^noWm8MR}5wG2GiIO*kO^>(onx?0~LWhW$j31gU+BQ5YLD zyrKj;n@f~S#~Raem5yUNCp5A-DIFx6qf^KInD+ zx(p^-tFp#lO)Czm9dfZ(GS%Ng?nJz(a1~nE98(0;od!i_`|7zMLF{ExG3+mJVkwoc zXe~VCo~CzZm7A6s=wn6=_j^X$MFY7c#lMFht2;n=`gLp*YQ7Iak9n8+P#*X7%rlOx-^1Z3OT5lOLjl;LN{s``pjD`}j2nC+C@O$9|gv zgko#IP7o37A!0nW3YC&d zuqFLZ?SKjp8OUv*sK|;ZaZ~%SLHmUKd@3e%-m598G!>nJ|BWjtVH1mbFX-a~$w_M< z#&VPM&WE*o3AhcW`dka43`WccoZ<<=7y%HSuxybDg^n&8suSse{jwSX30pj8Ig$xS z5@GUGuyfKKZ9&O;`aHsmvD6ZUflL5X4xA>XA$Ds`QC(VeV=0Q+xd}1h+hb(t$!~s;llV{Svb=&03t8y zxcT=jV!r8l8U>)13QCL*43FAsit!CSNxWMT%$C;D+=$Q}^|JfY;Idm!d8ha_$;#=G zT%!2+;y#;hjQw~?Ca#9Kckm)W>o@b-m9@s?9sW}f_9&&s%wO7|!9fQ!eULw^k}b1~ z{G2BUGj-E_DSgmJSdEv=$_cE*ontLZZpy;WjQ*Fl=h*vVx%)9^JWn8!w8^=oy`8cQ z^ry*&UztB^+X{-GYLF`6b7ERN!^WzSOGlF^KwX-Oh~nzqwiFPF%V>p?jVNGE{*$U;S9&Hn5tSxzBI96CxtA8ov#;koEH#{+gAMmq$Ys18%{_-%JeG z+%p}K7Ho1p-~7P_5KZ@>`TN~TN+|m5J4@WHJ;5Z&1G0G}Pss?{!>@yGtRemizT<*^ z>aP}p9=dh-yyIM&3oKv@pZ3x})RZ+%uY=VqyaUMx2fsMA4cp1*8W;kW^EZ|l`2sDo z+RMNmTb0hWn-G(SgDTZP9!wRM|)Eqgnehb5k_! zlm_M?3*Xl%YUwNnfje`H9aj7@kb;Wp&mo7688R+0P1IjL`e z47Hcyhlks=F9_c5TUmfN-I+D$Csn>>2Qlw+{z&lMj8R!1_ZGNdK+tVX#ev=LGWs)Y(jVlz# zRq5KUdRxs!E;EO_+3Bv#p65B2#LCYGMA;K z%0yYa?btn=FG&vK`fOE9 zovGEgUeb_q(+hJFF{i&gYyZ{a%eV~D&VrPeZ_S$R2Ty=dn=K15xU<|V_JwEBY8dP$ zxKTQT>x0oZxb1#?_E*q@{U1hu$Gg4tb~h(W%!hjmk*yG%L{LCU)aRzX;;XC4q5Ub_ z^+u~W>9%z)YZ3@-P(xcA$wWm(RYj>>2S|^ANA~iUWjOM7D~d31=m8n3D)5t6AF>2| zaIaVI37#%X7S1x(0urL5QR386-x!%sJ$BdUhdh}-9WSpj6Gk&Y5Q7Y=TQn;w4OnL! z5T&+gGQd$$?9!sBm#R)?pb@9MFxPrmZ;kMG)L&s5j&uyqL?_^TpIsv>>h#n;Q%nPMz-DA*pUa*2xTmm1*7Fq z%ohI9QI#H|#@J(by!YseFmV`xKB~`UGCHCInoF79t^xJy$d#N>G{ntWQi)Lg05-v~;tWtY)PF zYhO&KfG(Nxc$6|>S$c%N?SvLK&)UfMF+xL=i!h8|LuP-?N+~WQ{eQ&0RahNCwgwsq z?iM__ySoM7xP{>E?k>UI-5o+85Zs-OI|&ZK-QBsJIWu$bnKSRVpV%8ds=K;st*Z4) z<$G|Osw%g?%9Jjcph03qmZq}VwO1tA`<%WDl@-%MSWe!)!qA;pQX#bsnLIO+{DDf@ zGe%z=tT_&@KSKpjHA=#WA|;)O{pqlU$s=2ngO9}GIeriw-x692CH3mQ;>l~GJYVIp z5-pDmcL-&g`-XQv%U9~wMob+9Nz-~3d3{f-XSptlbF35|)pqnmh*u|~wPpH3m96_LWsg{I&r+{a~|aGe7Q~QrZCtzct0W{97WH-#*4@=+&lXA=+FD}kXC)>ckj{lNFyZV{c19<1)qFhZJ(Js z&!>gAy4?5f--y>h@lJD9Prhq^Hg`w$U%iEHsq;31@-zn5x(Fy{Z^=cM-pq=+veUj& zq*VUUlWri|+}LXgr`jD)5BP9Kz6vG~FQUEV&6znA4KGbMvhbseB;iY>6vidn$PD?T z9~p&3wN7`ff=fe!hh#Lss50O~w*q~g(7hWygAu92q>^ALT8>xjEM%r;nZ3WoK>GdL zbymN3qDdu!Iu{Q6;{1fguT4%Ov$^DW#p^`p56XJm^;1 zpqX6(I&*aqi@-V#?tn!6_F-00b(9!(oe(wX6$?lkXslmKUrgc&@eBnf@swjv)AwO;Bc2_IU; zN=X!%IMyAl`I!aVDDrfsypmDOJbfN2U0WZeg~gb)DjE&SEZG1YHe_Dc&Zs-6i`9*9 zy|>=URrs}RLCP?IpoelOh7GGuLy0wsY#`!2kvn>WY2@@8mS+^A1r)+FmKQq`jO`kT zA(blaH_Z>G;0PX7jvw(OMxQGA#h4JaNUO;~W&`kLX;Vrk8&kOX`T2!#sJv96WM9Lg zoN$y>)(6GX3C*EoXqi#s5gXWtc|&5HX;U;?sl+teGX}rNl;b37ia2jDR~7YVu8SB2 z=YG^}*Si=qi?p^Law9ai&)b$gPwsmJL|Qro7Cyd#%~XjMkFA*K zxVYIhzifazsH}`G3YV;+Lnl8EI3%#`dubTX|i->v{hS|bQHze+Z`Afib20! ztsPyoVCg@m+0V?*29+Mbk*1?m6;F)e2nOS1_2b))K}Jv)C{LgE=1kEX{RDkpma9+S ze!(Dx&V;9_3Qy)znPm;-Q>IgVEJY21>D=pz`~5TC=gLZ)DfX;HiWGYQCHbq77R|(z zYw#p&V~j&fI@!AjQ4bwlyO;s8pr;WeEXV+(FT}?_5Lju7ppZlaN=m>>11d!Hor+Va zNPhT|X{BAdEDsmR{Ea1Qm&<6GOU4nLtRjbpi<@6llaQK6bdU;cJRd2a2Z=Q8OX)WDhOaK%1<&OrIrj< zwiFl&=Y6nwu+SN5v6}L~IVLK4P8eC*x)1*7w)y0jBlb$N`a&U%}SToh}ruN%Kq}0;Lvqs6;A~_-}rNn+@`II z55~K7(#zX2|Dzy0g9)NHS>^&FwFupCnUJfn|E>93)}-HVH6AhX-d;gvd-TqzHXug^ zphRWZ$Jbrfa+!c0%$I zzaR-O2r5J(h5$@R*lM;F3iTJhv)Mt4rZGR)3)@{rlcLeBv1#|}@ue7u#biTyZ)m`DX@mnqa zyynv?Y{MJrj^3G9N+L42TYop~5z831>!>;lxhq~94Tf$Qmo2}Gt|_w(lxFBHXw;gk z)RWeqA4nC~&y86~!<_6f%?@PiH|Vpv$aHCRHoH!CBOZ0xJI8h5QVQr6_>rBU>i z+39gSE&e#C4;O?r5;l(CjJ4e80dJX1NQZ z2*t%kp88pZ?O&L+l|Z<+=$4#2O(=Wxy&Mld$@O=wj|66jR_>Pwqwk|W#X(ZIjUGWO zN(9~tza3s8mlyMG_wEg_)2v}`e$!GNdtpBY9iK(#e;o;_u)N&mZEzOGRWYQ;5p+{4{dbt!k|ml>FafRPvsd?oqYK< zZ~NSi>^T_{@uAz+HG8Qx&rzg8AMDyd8%xB~BmHaBHp6pMOl;xETveYnbfhh#$prql z9}X##L@K8C9qZn_daadfW>HSjm_{AUnKUe{cNnWH4&(eE1;=~b!6&0;#3+qMC%oNq z!FqWL59*eUtz*H@qnwQMH6})|@A1BAcrm&zlHT6jCHyQ%pI0pw>&xbu>v`Nc(p%{;>$=4ws1zt)vCJ9E)X=a~f-POLo$76DA1E>W#v_nS|mC%gx($- zoabs==(R_icoS=#$Ydf8@OU(bY4Axc2MF0|w|w<1VpRy6QTxSgv{tF&g5C?HpTE$G z1N>$LPmKnW?mQ4w=i!jV23e3+>`)a2Z#XK{tGZ8|F6`2kKZpD|GoLP$V9;$126B(B zEi(NV;CF7bzc|u-pKc8oxzc3jkvZhIWPmkiI@}dAs$Y`-eOR@_*5%^!egaOj->X3C zSosD&5F`LdUkVP`uZY5x1}Uv0TmL#eSL(o#tL|rkmXwVD))da6olNYm%u+DzYmPFG zfy;{=u8pQD*WWqu{bev-U7=%YuhfK=(}ozl#1 zBpiecX>qVd^!1-N!I9c{F)@Qo8j*J|8KSuhMcqS1$>L5EN-KZWxJr;pI(*PH$!yAy zQ^-((b;*!WG!cmi!M!sh78V+!x(10sA6Gf%egiBCRtOnd4ta7ylse}<1ubpu@~SE` z84mm%AdN!>ed3T8qXlF2q$D8RN(vn3aJ1BrMG4M;cD1`;hgsl|QzD8|#>MkGAg}JZ zoD^RL)%kk*`oI(n$Y^q~%Z;>*v|9sU?po75XOk;&!s4;ot)e$YweRSJ^KdS1DBH)s* z7PBaw4V@h#K5{l!SQh;o4h(ge*@9RI$v8nR^`cAp?3Fu{JIk;(U%M5YxW= z^zfvN(8wKDg3<3L87D!=YARKtgYA6}w-#wVVP#c8eqG8tIuVM;cP=JbsIzXC+TN9k zm@y3@Dh3-ar1_QUD{t4t`23jJK!R*(a9{WG3)fGI$PoaO#|Z?QF3RBqSsG|!^EaM}(>A}o{#k!je1UnpdwXUSMjcIM0@up{jvW!= zWbFiO84SX&7m8D^L2YN9QNJtHozAC7srr(B{hq6>cg4H8xv}xykRW>85Ce>)s)9Gz zrZ@n4=UIQFEs^gvE1JXPs{2)w|C0nDPKf|-s(qQvud0fjEmORD@qZ)rc02n-2QC4K z2|7aDpBqLsYKxOUyZdFQNHT}zdVUS?`GB~$GG6w!VDu-I5m?5BPl#u3E=i^`q=)^X z^hmLrT2Kn^s*G^uUXk<-o>!Jba(B${h@=02igJ0mOU>Gvn9a%Cz4jyP03pXj=Gzf& zRvvo&jf47p!Ky&YuY=rm!et5<69PpIF3bc9c+aonioyH1MAX8(BUQ|=wEA!Tz8ru2 z`0t7ZuRHE-JKutQqX#kT8(!Z3@x4ys(657RImvq7yX$(lDW>xl@sutfYy5Vf=3~F? zSKEEM*D0&KD0Sa&uNNfy&Mdv!_fU`MV>r8ntRya;8zyLpgwe}g0Oa=&!(gd9DAzp^ zbN-C!jXUBnH}$%=fUVmb7($R9WNY4_ze{9# z`B$vLCSftB-;&r>`cjzr+h~fywQ2B0$($8tMwDLrcnR&w$zs}h>)A+cO)=*%AxRb$%$RuKkmUN|tgp4Wg6D*Bu-cj}7Q9F10c^8`%a2GO$u&jDlkm%)?n%9#uEz9R zBE96l@{TzR^GXiY_YiIjDe=x%X+}}Kb56v0y}~W8^-ASTWnXDTRi&FqGE1G%o!qd} z?FrmD`dR1LdMBj#tusJ+ZIgk!UlIlc6C}+Jk>)NE87KH>SC;e0jTEXMK%>mg9rCdWfF(7+DBwC zA`mj?s*57nlTG&6< zjC`D6(JXd4m#GjFo&msG3hnRMO=jn3ms(@_L^8K*e0^}<<4v&D8G7+O!BN-Ihzv6b zhUrt~>h_H^SX;hR@#SCivBHvd3rxm=m0 z#*H!M9gK-YnZ^4^N!KEhTJuWexA%yKMtm@})oh)KQH$)NdrLShwmgd8HyjW9ETR@9PykkB4$t`YGy_@`_9N91 z#VA4KV!aE>l-XdWi6jCUf0JQvM zBN?GIiCIphK7&V^WEFwSRh$||f^w|5HwXTLX*jz$M9L3DO;LT`d>L@ya0M#muOb(A ztXGys1$y7j7z?PM1@KtA9qd`fI&v0LxB2yjAQ6J8L;_S02#ZMi24*mFE2l;~vwny=3U{@q1fasJWR%9Iur z=W{Pc$Uiqw)5t~mEcaoq!C{&4Nc!Z3buuT?@nGfGkIsxc#>?}|IN~CD@KLj-!z`?Z zHu`Qvn7&b>dxV)%(>1o8K7+@_V%k2Q;@014$lvAYdUwz7Q5K}rr)~d&0z;}v;jR=5 zh5;vveh}c+#rKM$M8OIVA-M!KnZj~=zMjLVTjVv7Cv}H=~U;I)Tb$gc@tRLpO zUI_pz2H6d#7C5NLDc78X^}DBsO&p=YV_xS)?{%8-tKH2jTpvJ-zP(=12S@6`O^c;k zyOTH(XGY6guDW8)!uyA!R$QgwOPN^{0eQ6}Z!p@gIkhV(|7T&#N`}(?#cI-Gd7GCG zi$y4jC!!vPALM=~)v6B9&^~2%iB>wXz9$_%FV?8%$5-0GqDMtdlggh%V5o4Ir7@XC zgeC+MD&Ff)z)%E`FK}lkIX(ip{!|!a=CW zByWChttv`3id2aV2>66T#3xFZvKdJW*69L=S+Q5rkE`yc!s#V(+w#p2 zsko}12x+iEBpQGyo70v6AO#Q?gVAV8e#%9Z2As~MZ<8b$G0UX4GtMK??N#Pi{)$@P zOdi9q@(-2HWY7GjttDG{WV(sRG$k%C3q8H%J>(E-rZiKJPQElDZh=)~fZo7(b%7U; zm{;6%gq2q4wx+e)xF++2h?*!L?c4$gblulnsn5Ls*w!~RT#dtaNzkPjwTAz;=AtQ0 zD76HffGsnhfvyfsPR#otKMMy131HEMmiQ_%K&b>kGH5Ah0cLlCG)271^c z^b=Dab&iao>~Ot=F}!n`O&v^_mQKDlai;1D2u6c?w9co{$KVCr0tL>IoGQp;xsX-pGyL=ItVmqS-a;Xp2F69KVY88ln+`vo%lJ`m#(=dIV$hUnAUMM*Xrn}nCj;X}hs zpuS*;x+3s0pJ&uYVbNk36B&l8l zT1r67o`J2hl>9|aS!3UvFS%t`3C99l?*vSYc%TLQ!envysMmfWHaD#QJ^J6EWag*9rK|LQw7`l?G<@!rCrwGk zGt>2bGZ9Bh1VltyCR(Z#IrJz13o^RNL`H!966C@s!IuM25QAg^WR`$?OEA!?KhLv;c z`{xqEP( z>Cly6&4L_4Rdk{;AklQPH&?0A0=boCF7(KiXu!OI>`-T1C0O=Sa6Qb(q~G*C-ZvoH zZx5x~C%{3|CNd`1AbcdIQIej(1a1)$0$APPCkj|T!bf#%H=upt@6+vG*=bT;{2AW3 zM@*6zdl{urU6J*mNvMMyjE2kfzNEX+=KuTN{!Z&Vs<>q_!vf+ymU~k&G&yQgC}G7w z_FzD+phzm>5V}5&B}@QGdQbTe$1${L1~|Zx05J4~1B8Wt>n_1)&$S>0l&#xGOC2xz ziy^ccHCSGBD9Wh|!*IZ$jD$xP6Cd(p2nDsMTx3UoJ$X7A0x%lwnl)U;!)U54H?P;E zq@|=<@c$7h33cR?)l6#f*fKa&Wd6^M!jMAg*+Ybcgus#hMLdEka!T92*=fldA5lFn z=^HgWqnc4g*}c5=*4i9V+Dnx*Dk?vd5Spw#(+#~`#N(-g<$aCUI7~=TMS^7}0DLfv0$!j&qs8wa5dOYTkTy}?zqkPZ1GE5N-h;__ z#2^Ci_(-9c(h%e3qvp53f8Y9=YH+u{`egP8;_lB%qTjk=7CeN9hj1TsF!=K%WU_#g z-_9p@&vWclB{pM}&&aYgRo~37ul2l8g+prPtEm1t4e9?88MLTMGo+S)^_vXse|Ckw z+~1A|l8GU1^Qs+EUr5!_KUO>5DQ3;(K#y3SKnjGE;Mx(-aMfJm727)AVB*@LhK@!h zW3Z=@U>{QnOer`yp;6jaas9JN{t_%eNLzsQKg*_n4dCt`W*#7( z7%?;1ziXP0n{ru;vSF8E`UoDQQkHRQU~@Rv0|v&tE6_duz*!0g*{&P+Z+Gl=`c7#& zOx_&)@<}d~Imx1$&NpD{q0)~{Zl1XRDNJ2#KMyzIvZD*p* zsMEHco;-7f3uYinyP)kBb`>)r6l%&Xiy$pzd z3-bkk{jXPdS9xYBDGABBK491zOHN7(0Ra*4rV?rh*qYZekdvI}nc~8P8j_5%L53Au z;zt&e{QElpGt7eIqAR701w|zu12%$#gL_VEdpxYj;IhRBp5d(oAWQ-Oh${;n?S2x_ zpK36Kf{i|{#|QS1U`Ih~rvAUraer2H#JAyqn>8NV9Xtg5qyKgtlav3u?f><|&$#IS z`fOjjh1mYHFaP=ZY3RST0DSlVcO#L&lXMR<_216F_9VS&jK4eEm@t`sNQ?h-i^Tk$ z`XU9h&O-MF%YQkyKdnjto*-=9dKcB-EAg1Uvv=5&i;AhYG3~Yc3R!~T>5vSAHyJ|j z^^K_1_;rV&LgxPrkM%c-4}afxT+V=lIbxPKhXSu9zWQe~W^FP{H#~i*_YI|V-6U+g zMeb9P6NbUQ9KUQOc@?&XEbf<@RCKpU`Zt32zm*r%(EOjl)vX8YCIF&5@TxOQ9qZXL z)-#gChg6Ui!{oD z%+Ciu?6{qrs+x#B=`G=7Yx8X_pv(kCf|iSdiivGT-q-hihZy=;1Z{F^O2VevVxI`& z!$M6 zEQ^A!CtmeR-dtH>ZwN5F2t)6LzuNvpZw1 zEPyO4o!CDulSpyw#t+!!tgYz*Q_id|166DXGd?wS;9)OM5unk3|87*bC@m{%d@xf2 zIMJUE3cn_BTF(Jnyoi5HTf!0~-9ceK$rE*AW_QW`S&uMKLOGp-qf-H5obIOt9v((O z{y-8w(q@tb9W_Gg>VC-3x#6^H)*6m%v;9T*UeKOPAM}auY)VY5((_a~*AldSIH@mV z`5id|eg?40TS;nPNdu4Tx5e9($yk_B`?p`=2s9ZXEXuCt%Qo9kGPFE0TyA15aS4vU zC<-XPZu?;G4*ut4Klhhs!KI+2ga#$v+Z!}!DPuv)&))x36Ege$-D%0P%yC)*Cm}IW zlmov6s8O5edWQD&e5z{RM5yX~v;utnnLF>v0N4f??nUv~2z8BS-2UY#Q}{c+jdw5} zRbF4&%e;A=Yonl&`k!okvI)A*A#3}IY9ne*-7f&~D`6y8OJGY2ZNDa#12)SrLtVM? zyBSZZK3V_u-@Bv}Z%gBHlky^)aM8zRpT%I=6+4rMY3&Ah2|r}FvZQ%)RhK?(&n7UM z(WNq!3NQvZwwMm8UdQFt+Ir(SUvAa>uW$x@IQNe-#HESs^+yCH;+o^a5T}j|%FS}* z`rc|LCMGr?ey{ksXam&Jif9lf1+I{~UQX-*-2)jbD|+1`z;6`?=9dK~e65!%cUKh( zi`99+voZQ>V=IBa#$F^|L^BqOT75hLvoLy?rWtBRGZJbTWd)&(s`u9!j*#)JL!JVT z45*6U-~)VPEy4pP`lmV`+h8N1UeYs?d`OPlh^MaRM{{u1mBbM(~2zd2HOEC!B6C@c&d&*FA279c`3&7jlL z7nG8@NL8K1HnOk?C+9RCKvGBfAxPA0p*fMBDJOKTHn93-A05Sf7^id6GsV)z`vRWG>yXpy-hgQ`!{_QP zN3k{I4tFyp>orUm_37c;%q5lc_?OOM3TN0jzbys5kBGHh&z?~aE9Mh}f=k`{oll5s z+p~#a+CAUgt2C1u>z$yLhjp1)TCdPC`yL|np=N8c1G?y#8BlZM;2v2ds~dkUi2 z1PIOi(fx6=cnK=-kovlDkRTnSqXv4TLw$J&v9aKR`H@V7T-#O>khtu2s62s z$7Aw8T-VP zSUvZzpB3|pSQ-7-MluMqKSu~Z_inJzt5>c|x2)RPsDAfooSjU_%sMWvvrhkGZ@1T{ zp%XtA`GKBC1&GNjD>?v=wI_j&j;?UUQf&E^J2rXPrnJ#5_++`sxi~U1(xdal)5t5D zyg@u;WJp!fp}0~4+^on4gt>nw03NGypD_wh?}h`gUd=2;CnlZ*3Z5VZh0@B(Xhi1r z__92|OeGp|Nl6&Lt~?+g2nivuU(q)oUxT;7Jr0dM1DumJzGC5uxuQNstaWkP(gR-u z7-W6apjFLl92Bt6aKyE(%|AD%FCt5U&arw^ql}o8q0U$?4TX?jjW@3a!>IC{StI z2g-&&NuJ8ZYV%|Suk8?8OFluiYY0nf_Zt+F70}Q(p|CQ7Wg=uZl2X&R4QG-ud72tS z{|G+1m#$s%)}F9=m!nxN_S9uDjL~XDGtMZ8y|Bd69`p~Eu{KczdF%Uz*s3y9jFw{% z*>2=jpOKia@p_!(OBHc+-txszmL=G-FVt*MHp0v_Q!syEhlG@(j=4SLUZ`It;iN~b z=9M$IW~>Me8mnLcgW;CBstlLIL$Z(6>U+2vD>bEDP6;CgsNA^i4ONYq9S@R*TyU@5 zPPb#`VR&DEZJmGr)M&3GuiGT4rOvmj#LUm#sSvE1k!mZoD;SNd+fbqLr#vAu1*G#W zY?tu}uHQ^0AuSMOgeoX{w9EO+bzcrN-nWj16jReda^> zhEe51H6#)hjCZ&)l6B;^KEy~J=ZH}|9Jp#oJwW8eF(hJ7%-Z%oDk5m65W;J@Nx8$6 zAfQ-=Zm};MrPgyN%Vj{v#N5;%2IXgw9cSmd8eN%OZIY)~Xl0{3)j;!$23p8s-qXUhZ)|JcY{9{C3Z zvqWeQ|Mk0Z>$QA8B`S7)#JJO<62p&08wYAB=Ef=i9=B*{_i+AOLHc=4PH!g`5%k>h{3- zBd`q@U~hA9a6rn-o585vc<#uyteURp32*6WH~?w3sUn-9$CsDJ-9Ex%?_9#xQ5z}H z6p@h2B{$r{gB2d_qWm(GRe#B)Noo2RF*fO3bk0RS(0%L0WoK!W?#P5NYNpJ!7=y6i z12Jd6AfoLa{6aaVI~Hg{ur<(n_Z|b_hsCSi>nE9 zTNfR_5&Dy%FoYI?-tM&UuOifwcV%g{Bbvz`CrGZ(i0x6*^1PcDs!Cetu;9cpq*n_N?@WckC-<-@{TRE9L zg~AbtYiZ);rGcSLwm8cWy<`ocs#flD%acMDLHHzj2*I)LDy_^=n|{p!p6rCxWsRnq z1Z8s9;#8QDkV_OQ#%fZRl>Ce0m?|V_x`@2q#@>j>3S9Z~OLKL|$M;GC z)(|DMTmvO7Rtq1v!Uwu5<9@bP8g88IrY&F0iLMR4nUT$f$@s~u1o`OUQHf|ep!9o~pFQ!JR|n0rhlT_i*wa$e^c>u-|y#=lrNznP_lOa>qD6A)#lb#Th&Eqs(2=i#BFDy1E}*tH?*$TC1)c9jM25f7LnH#Bg2JHJNm_^TciK;#|(B z>6Ns?O4DvsGW(^C(hEf29``0^hlvFYIZ)egzXazA-w4=x;y51J>hBc`a3k2W8zFI2 zZ3djJ4wK)s4GA2LuDrIC$T7PXL3%lSZK!@tsa|_T^9J;`C$VpXpaX@07`IA^mx&UA zHY*HsJx_=~&r7Z*;J?wyj_SZ%crR|UYB&Fy6+iC<(Ou|SVD)%X;Wy}*I=7bh%jFAk zwp``zq(IjduR_P|vX7w~U*Ac!k%wZhF*GCJ*kjpN^A&(_c4t%i?7{xon$bxyCwlo% z-O-Q`blG%K(dI8FuL`?`V5ZZz+SL+~m^?f-l-WBn&eiF<_aoca2p@ak;NJcB`L?_L z*4oWYw%gW^6^EYkV%`q}2L+^a4F2) zc3%U{Fff~mm6;D}`T6fLdw7AVj3S18cH7*YHVWx3H^;tCXt;l*&-D(re|^@(xTL$^ zbU#~v<^^kfj&NlCgTm(Lgm`CHQ*#}9cA0TWG4SYBQ(SCxVY}PvijcPIg;zHDgJQx2 zZ9%X*ToK$~GZ>C{eG`1Z&SNA=A^8+8sRUbQLux)jtD|i+XqW0K%$~HXKW4teAVTRp zkl`b=9Pj}#X*YMoRW)1VbB5r`*uya!f|CH6{T^`q5IN~+FAt|HU0%m7UuFFgU2?@5g`-gR0PLPx5Y)iapKyG*r>}QA8p4)uwISd-}i?6eRmju?rRBt=DdFbO?GqFmfeS?}TFD=$1S zR*o%>C@mDo>pb{0;q}xsjj{{rXcB`p?33`0&hkr#rPL3wo|m~Ccsi?Azu~E-8=@-xSr~;t z%1Jmi(+Z_PxMVM{J~b&M3t)v2V}ssF(eUD$9_Wz>eBt1BHyCFi=kMul9-6XiY;al6 zR+F$RgxdJzhB}NTmlwO%mek1NJ$x!xDi647t9zzdu^HmRv>KvCUH^2I#i;^$notgs zTN^&Q*%k?Ab|(wK{$3_)r)OJ`A3E$cbtk*xvL?(8hH|UFMI*uC&qHCvH=M$u_+i4g zW`M~Vbo>DswZ?T;%ZLiLb}{gK3@b)m!YL;^8-_$m(Y+*@M^r_+FmKA;?Eas{?kZNC zmKM9lGkn81+?Nh9afV?E%N(0E77-2Aa}zA3)Fa8?QWi_K5d*m-7Z+|uVPSuNQ~`69geeBM#keun4~hr-Erz%I613ZMNCF}~u zpp+!W8+2^-)VkerbIn!$;E9c7gN$cSxjE(fa79e1-;d$g7mis6rN{l4T$^Mj)|j}n6G~&Ce`_!if&?Wh z{lk@z&3;V)IP9b!I3yT2=c^C^5rOK9m9@3!am}EGz5QOC|1&wT33|BNb3QCDQ&Car ztu-AT8X8*P!th_pz@i+v=6=ihaQBju`c-u47ba+ASi*=HluEiXAjr*VcJU*Q%>=X& zY&y^8Dub7n?ak%98Rofvs;%jD`XI2nKB%e!PK-hT$qOeD`y8>4F+|kf?lL?7-s_UK zB?-`^&K>=N-L^2A8hEfHtJ!`}AaEv&rQJ5D(Q2>)0vP-Wu{p`(<7lJ(Y08v;> z(~{I%dWe$x@<+8_45HIX`{TCnirb_<9zJ}Ha755@=K*u2#i3YZ8;vVP?RI8KHk+8) z74hlrQpw&$$-LXdu4Lx)gl>~hh-{V4K&TMr4&vnccX3qja;EE4 zDPHFb&ga{5uR_N~9j}35IYZQ>zCioCBtVVHD(qi_V*kp}d3{qs%b-B{63YK@nQz95!J zy3iWCjKgep0j*zfcQvZfYCt8kT;X$iIeLHFVLg4R^NrBkWz~QqAAL)V1xeKKqx_yN z9SC}d;rrNNrF459P|xK;L=gBY+FNXyt)UqXpHke5fIU9eS1f&OSZ2^8x@*GUvuir{ zEP>@EcU+LD-uZRQ{KL8$Pkiy_@Q?Lkuq@BkFU)W0+n-VwMvrBT2cDC7jEC{EIjRFG z+Iopl5U?4FIjB<-zhm;ejLnv3X9wth0;*o614)9kS>k08j$Rzn`cjB~Sz&5*(1KV! zCUpk2L&t`_MqK2uVHa7I#U`dn$Ms|op+tp6Kavu{bLx>0l^Fxg+!9G-+htuf5{gJg z1JKSe5rT#T$E#|?qQ5V(fCtVGSdCsGG6~%C%~Im`8ps_zEKQv=RF$Qj+L1%vOZ4gf z;<{eXQIktuj_mK(T-7U8A2*fp!&lebTu1WZKRV!A*)}gRR;L)S3_x$?Tf&9IZSTwd z^t)X%SwW`9E_E^QcM4#5g70=a#YsFa@5_UM=`c`ku|(6h%#U{sI~_1xfQ`Q-hK04W zRYP~Ih$^XUi#>*&At9p zq1GeTNQD`@`6)xB>W}M2H?-{zB*->kzvvviV!7k-k?tw@99A?~YH2}jR&+{F&X+2?8Nob%(jLq#<(a>is#MAw#DW3Gs+H|9R)JbSpu*JjJ@77sdWxDi-7%Q%L8 zFJF8{`Y{a^X)aIJ12zQV(PDksMmzF~KyIkObu(pephN$>aXwRKC?g|&)R|>*M|D0i zm|VqzZndaZx-BZU1w&+i+?ynv)2@=ympEQ*GDh`hWm33RZx7V<$x(~f(h(zUM;`1q z;v(1M<+bL4=uh)uZ_~yE_FwqWVDY&Hv9?q$LjGS#9-*kP&5u&dA6A+*PM{-96$ zWtROn>tBzZ zII5t@TxEr9tl|p3*kMqE3$i%PMCOP)bG3o%4HM+CVSjJ?!?S(!TZq{@wx9P5{PScW zA=B{W=74r*-`saOhOu>P_vc*b5<8N>#R3D(+OG%0*rA7SL#x_)23H37hjsqO6V{67 z6-|7@!nuNyRv!0*@*H7DrD>!DOKJ*S6(FE!AMv;jMQ2KYe(0?6~HUX`hg5`GIzpDC}Bq zAv<4-!X6%6_omu^q!^O;Yr&jN@p9A^C z47?AIr;jU2yM{VbCg2GoFNXdlSN~1s@$u-VKbj%w$izo}S3Is2^K3r_rROe8eOw(; z2JzvQup&@p6DTR*0R%!ZiRgBAsX1Y36@fAZOu;vPZ4Hx1K@K`5hX{?G5b|OWF?asx z%I{k&pCg;AYt&R_JcPi8;SFl=zTTOpb~H2-BW$h?k`YVd2sPhi z0>YIX?cY#_1||wBsg%ujDFvDAv2iRleUkgNt!FMWZ`#%R&FoiFz1LUyl^YlKyN@{e z=HpkvO!uUyLLToKtJfyo^*n+kk|v8EtK%-4nd6pEU-%}bC38JI8Tni5E(BjN54VIz z1yTKAXNS~@?qB0P=k8(lwyz?T9^wK2Xut!`#MBfW4r>48#1yd1sj6ZEEYUxH{76AT z0nj2bfN(>@$OztX0?x++AO5ZY$;(p;3loD9hbI5Q=zcnB!bAysm@3sDyMVS6DX~~Y zes;n0HXZ8>Up$m;A0P@v?-Q;KaQ|`f0Kx2N8j|gL0@is;f$oR)_I!N6S=bkf8$n75 z!av?@;O`u@j;^bnPmlW@+7nQzA+MkY?PSKZQxKL~JHNm}6_pe;6I5TR)6?}m3(pwU zUkdY;1e5Dfd%VC>ya;1%+em@W9^2Z#D^`z+F0LMme;*%0P;w`cp!uOJErB7f_U2(n zV9h}wEv1x2VRmiPT=<1HU>4hHr1xRv>B#dQpX=%{iuwF_YjO(Q6*7p+f+Q5aOKe&n z>yA5s@8j3Kl|RD~M^Oe}NP0*{3Pcag$XJZo?#yJV;Sg@8`5PP+Y}lKM*WS?z>>2|F zt|#_un%9jOLK5sAoG7#{TVHfh{`dVnw?mQQDut0n62{Tw6r&V8CS!lO33;jC{NrACDR7>BqIQHuC<1Yxnd{p7T8#5eY8*OhXY$ zRDy(CE#l`*a(K6f8G9ZYdHJ3ZDT95e(4}w7wJCb!0eI}C2?r{?O37=bT1!%#GhwY3 z-ww!j&&aXDYb(l$ibzn~$cn%m95MdvQ^@bJ*v(>%Fr+c0kEMW+5j53%U4N4AOe5M_ zk&>C|7APhdEC$rlS0!0igXrjK)KSYpRJZXL#Z>w8(D3v`BXlDRi#1uPOE=MyrUD0x zSRb$xa<@fxXU$n@)73IwG>;!d1jDw}C`=;WFR0P83Ym^{_ z?~4-~-x9?Oh7tI@X=^`dayj!vg1{!Mb;85=sI`IrWS?}`$S9j&No)prnCnc8YbEdg zR=m=(B^EOl!^QCSpwr}FgjNE9CVSCO&0TDgQ%Pf51}7PA@IV*x4~rQc?nE*IyatI7vc)utS1a zDdqQ8oYnM}yZI?VU_5( zvR)P+Job?|geL8^Dp!75pgT|UrCCcP24J2se!QC5ocX#*TuEC*$LkR93g1ZK8@AJNUpZA<`@4x%~`50Y; z>K;`~s^(m4&bj7#9;Xp3E%={-w3Gd(umkUSWVE)g4B_pW%*>fsq5J^a%1#x?Ati+S zD5`}MWMAez?04_lc+{QT-!++=xoCeSYO|Kvw=>{e5EA?Vru)0R*`hq+7MSNa%X;xddPIQD%QGuwkRP$#< zyb&cT3koI;DcSpyrOOKR7k>U2lyHe(4TxFqoVgVE%Geh+6?~Nuur}zjN3(3{ng3E) z`2|>KXG-%B#Lxh3)OEH9X&NbRvOy?Xks!qN_23E>)=F{|3TfAh(A(qRNc@c%1_laY zs}ucl=&T*f=rfWb5(Uz3sFL)=k(ivQDSRX{9toCw@H}{kIEZkO%RBOW+BBwabcv~E z2+Rs{(b$p0Gibqbd`e19Ms2zYu?!z!5|$)66L8^lcy^~4jC7shUuW5as6>^sd4JT; zsrX2+Un1vsI)0)do?ABLs}GL6uz~OFe;3q%iDSaSZedL&nsw4&72>&L*i}KaUGKV1 zi5a*3CEv1yK$vH2TgrquF=OO)pkI@X3mqIG-Osj^k5ywv?G8I|GR?+ik_Z)!!-<+z zOjY*br1Xf&Rd`_Aa+Yw`5fg4wm@TN-mw2X=q|Gs>8vCnTI zb(gCK^+9^^eMv|vTVUWjNd7`LkKIlfiG81W(=5bJ4?`pQb$(v{?HgJ4q>QE}en&?K zsmGcVBu&G#ZIwM!Ddzgor`q6K76eqZ*=nw$w)SAKek6aQGP&G13hjE&x6OIg!O%1w zFx7jY*>g9nprG2D7*bP3L|=&?r{^rZ$hAHk6~go&@fo=x&`zkQI8r+C!&o(=5o3S0 zGj+L=Je2E*v>|ZxfJw#Fe#)l@$dhj9DGBJ>vc)f9rK9q}y0_oLbpN=xCN@0tiGzcl zou)FI0J1N(i8WtUV$vg#R)}F_Brb@iQ-DUem)%4)lf@hyFnFv3(ikj}d** z-gLB@byU|_Y$oTVv=+Wd8H74l}P^77U%2iYZ_#R|oB>sh-b2zqJI0rZey5 zu z-Mk?*yVN2s68Lm+(i}&OEaqlUR#vgtJQ_;sy~MQiEi$kH%Cl5sR6YS|jK96e z&GqIM)8V!t-gR!7Geoc$G1WB2gw9HxF^}|)*tT+*fY9XXr?U_pobr{4U5$mFFTvCc zEUmEE^_{^9();g&u`e#4>6e=hlXqhVZCiF4zr(tc4Q^G;u#=W}Pf$`VN4Cw0r_CGH zZ-Aw23Fq1gGx@S5GE277T#044=!%x~hU%)iX~83)n@ioZ6ATWlz>1ND9;js`6(w_o zxgF*hv=~hCXlFkCVq*arF6ZQ3IWE2LErAYTwlI4~zS)ly`0F8iN@B6?3Hxjf{o-^c zCB6wuKS^`$NOyoLKfklRzM7-UF1o?Xz$YXtMy6|-^6f^3wPD7%I`Ta^CTLGHL>B{pAPghhN6Z$6R-YM-K0Vb_&p!BB~rI!h63OQKQ4>2jT`fs@Iy`#HFuBuzo}DC_3Z zaTk9X;E;qg2%fUg%f%qj>!=ihKs6FeBCo1y0kO($mmA^QRwtJ&@+VUmH3-Sc=~!6e zAj{yc`tME{7rc{eJgdo0r>5l>#p4O&_|Y!j6}EcKBcXUTXr~?B0r8 zT#1tv2%Pte@JEQeVlw0pI`df$SMeiIJsNlVpO@6%gaUgB)NW&kfu6f$K+kXSc6Ild z0*rXF5(3*>#cQu~jB7o%j7?`xh6k6e-w}^Bd;0Pz$io;(gH$G?>u@F8!0G<2 zIjk^+f#zraCwO0ZFB#d{xL{~V4+>=|Oh;oJ)IBeqCO~d@Z%gb0&vMA_UBSN4w%%7n zXK!Fb&ni!n?ZelBoy<*>kyPHxnbdbHhDS!UEr?Y#2$G%QvUxd!sTR54?L?oMIUMFB zU90^w!~JeNpi%gBzhzJ~~ua4PC1cU__uKFA{OIoDq-OoK9N}nx&?Z5c^-o8_d zSyXq$S@_*EpS@~eNT0qH_3S{8sdU8?sFIe}Iz((3k($er*5aaD-`Ab(p6lfA-gIz! zZY9bh#>cSn2uV!*KF}`Vv&&IW>|-$$t;0A{l8ORP@$VziRzU*gKEx(4t4k|HL-i7p zx7Ng-TBOp~OpLB{8VUJww+oY~E&Q{=gTJN#=6hX8ad-7qX1rK~GUEsH%?jO;Nnd^_ z5^(q?r4I}MTYoxi-ICLnRpcjRi4E@oa~e?_C*H;IMbRDB`yKOWVz?Sb#bT;=r%cI# z?GqE1Wz)vPizfjpP#4N<0Qf?KkM@oxKpDBfIo6gQBTgwoaT`Qc5~&hM8%UBM<=Nb z_A87|R+N491E(o@5nK=2nA>=AtcQfw5}s{!!bFICUQj zIA~X`BPyq~d*QByGno_1cuN<-SNO(U)WuClt*IrsoV4Z9J-f7#-uyCldK$+!>*`&1rM-~y0y_uFrM#ZcuRBCYJ2&an* zl^ttOf72{JKsO-`J+gy<`JkBz|LVIU#&!7=HBdbx?T$=I@5NI^ZXc)cib@B74`d;L z}QZl;+z-LsfJ0zX(VnJZBK-jmY@;@`3^=2(^m3I(Bh! zF=T`ZEi0qbtObsS&$k}A`8lof(E9W0@i+#A&DS{|GJh+{;C7d?%i*R#WuAB<=o#dU zMRClg68JHkmcCJfxAZu}8gWzCR|?Heq3i(r-5@2-s+nxlG9#|sZxkbIK}}>ic3`!r z7=nD7xA2Z4uZ;fj5`)JC!wo0;C`PaMrEaa$6fJN{W~H+H>D5a>FzdVPSPTkA>%~bG z315cA`5O|NY88Y7oO8lk+S2NQ#KYI0>`sj!j9p?qOSXO2yRHD^G~m_!)pJw7S=g?; zIL=>h+~K?%F8;*A#g6`<&~UDeBd%ve_bKp#B;Ikn4MC3Q`G}ZnDSxvqsLSDq-e6le zh)eXILr=nZ#Lnl*uZN77c5-SpsJV9~@Hj+brx@l2b;z>&P?LD$i5|}iopz%v{MTA@ zVywPz1`YQ8#;YCnPh%d0!5ACAYdAlyW;A$@hAOv^tktmku<#?BnC%*cfe>+3MKlLn z%N?@dZ)^9(=Z#1E3eLH7eMZ&gaoi+Xky3^Bf6gDaJ$RDxEknOEwD?MvfH*F;iaWXz zYYqQI=(b5HGJB)l0S*b;xGqN~GWnD-iHOVq6# zJ^5yr$2HtB;SYaB74Q|2=PXts%hr*aI7y>QUfW#NNWQ2DMRE%7&S+Wf!$UK$WyOa9 zji~69^4L?boJIrLdJ3t+iGdAl-|2FYMoZd1qyK)hz!2#Jd^;c5@P9jh^Hr_mbeX#8 z@og3q?E0ZlWvxROFv&fVfen&~L`qO2Pa1zehK5PMmVk1Py-9dN|Lz?;ID|L~g(V5l zQCZXMYvKMh!9O?7Z=oW3$gI_6W7Q&)RC=q++&eAHi0#Yk1W2OOs@^V&q|<1o9Uugb z-@p;}SmAPF4R4WLzZWFp$EK>~X8o|#-`{{aOPJ1BBkzup%yf;`f=;m$fJM`X@{oMO z55hH!&aXREba(kBhv^@^qwBZwe7Ef&17Q;=amgr7WC6z85WX@P_m2ELjUm6rv5hu$ zWH#*1pAF_2#Y|^pl3P6c2tA61oJyx~PCc7Ojdgb|gLSqHwycUr)Y5*XMHrDmYL;OK zrLihaZ?I!)LQ6X7RJ%kpFu;O5w?APNhpS7W1~Wn5oQX;>bPTApL{;S4ywY;A6b5<3|%5GF~g#I&Ei8Z|`?TJZzn-nq}%u zZ(k|IXi&WQb}yh8$h(~4w%S}PaFNUjJw&RqbqmG~lajcUYwxAj>oEBxijo;xP-2IT zikh%^WT6>E3eD5Mg}GUf8m4lI*X0#zTZt7U zd;8yb^9$haCM9~jQJzQU8#MrFM0{A!d!J8%6CrG?McwDQH8)Z)z@1~#eQ-~g%%^Ah z^%wLWu;sRrc`lvRnfJ6Y`VIYT?&h?pyhwkjEmjPJNB4EU)A$B%=l12gXkr0dS>(MDL5g%wt<`yx0Z) zMMJ!A%wRZ%i=-z@_OUcMe@nkbqw-a2_^xNbfSS=Gmuwi0&vZY27&gA{Wo!q!GI*bO ztdEO3ieS5gFE04_-CtjjFN6?Q-QR`sa6|S!?sTBHqLfvB1n&RW3H+*iM;HfNiwq-+ z3mJ))m6D`Wyv z9BIXncj(C%7kli4HptcA(9bp-wj=EcDa0g$4u0_ph_DJb{u5F_YcTjdvRxKolnp2@ zHaKtre$35MfF+#~M#Cm$2TlkG?jIJjfthY_0%D{DUIB&rW{B+w1HzB_+AWo8K*HSF z%0knFhu2i?PD0_6k1i9bMWlp8)V7ol;iTaHRyzpLLQwkW8chfZHiR{)>N5cUaSK88 zdsV6&ZTtNnQLqDKd&0axUR-^1<1h?8Coq2cbvOI53F8lZUE1{FuB_jlENCkb0n~DL zvOI`EhM`XJreRRi7f5knYPewxp_tSHJUMU3`e@{EaSz*KvzUbhGdl>y^ZQ}H19Mo6&G-8zsBy^tU{Wa2zFcFrY^g`c%d zfdELTe@Ojxd~E9Ih&jgvfmJ$LX@P93PKRu}2ShI{XhK9(8wmX3>Y4&cyoPL1!9=WY zX_1DAY^cP$K(Py%0@Rh2-R|>MQy2uSaK+ip-Iunw(L9n^>BM*2fYBr6X!S=`n-Z!rxa6fjc&knX{*Sf#$no#kWYwy z&BuD1>bU&96B)STqit0Wmy5FXp%d&~dWhcJZZ8%D%e$YG6fzybES7Z(^=q;0@q-rG z1%gF#bQ4F*d&Yos16m$Pw-`>KO(<*c#Bx=Gf+A>FjFYlf*TC4Y(9-OX6?6{w0u+Cj zv}s>2QYO5?)fGC|IS zHT~Q@Mb$5auT$PccY57weoi@c6F(nXyJRYBRGgfh6U!ReQpBIn^p+OM`nfr+Aa9cIXPOLH+J16Ycr-C*wyVcpHk8(b6~X(;5;oDv#)`N@&U61{w_ zlhx1Q3-d>6k|VhsTxn8NWhTTrU=}OX)TD@eyrt>9B!wvT)Ny(m4BSac^W3T8DeQk& zM5Ro{ITd>HA-f8Wx&}DAX8w)TS-01>KviM;>73xU~8!513j;od`@&x%{OzZN!CyNz|A>1=@FNR zd<8A2l$-U%%{21VI_EG&(>!A(@}kvaU7Vr!`>yI_)5>W%AG>nxQTaCuLMH#fnhTaQ z0g)I(yM}hPvgVeNffKIJh7oyrxcf4hD|bTWs=R$MGrViE9E7IRvt3It?UNsp%X`N< zf0xsf?osnR-oj0#H-dT)*9Qk_eSr(r_?ddz1!}urW%4?f-@E_b(rjx$GG26^m}0>p z>COxeuHkYfTBzc-Wew6Yv~V{GC8qoO2?1{Y6qTO9CT7D6;}0`B$uhufaX-o0K6ZHj zGjW`vCpZk0Jl`Qtw`d)rCm6YsJ+Ukz(A9Mm1!fmcui5A4uo0{4mHlr) zf(_D|-V%U}ujHP4SjeH!4&JM)AP8H=(%RY{gkpdNf*gAiE zJtra}GPbpiIXi=($F<@^sL$WtQM_LG?`|jmB}<`RjNpgN>y#MG$~3;SCutSs8g;uX zZs`Mh3pQ^VB8AXEAp+Grs`9y#Iq|D&i}rT2A0BheF*hwH9~`sL!xkPSJ|-E9zN*E1 zX&%{|qt~&Lk_Q_L8V=kBm663|9vkgIBWBt9$*?G_8zysvL?Tq4nZKRcgaX-&cH|n;5^a&G zt6eUi>GNK0bB6qfaWIqZ)v(=(T2zQt5sWc5dvZ%EbZa_&TkAk2tLV^Ryia2E_~+WC z!X+gRJ~lCGqYKhOHZ}KP_^lO|R)jKi>{!_ggyLq}V?a%3iPsb2b6>MA(aO9m2RR)P zn#mIlEe;j{#(hDv;fK{ri(4MQQy{X|yvsG<5C-++H;pCHOm!mbe1nPAz&@NxUM}?} zlu){adz!$<9jfHf7?B)Y#ksB1>jc~f^9)O=hQMMepM_)%os-QI@F_<7kQhbI^g^OgQxHUS=q5H=x@LxxIB0dFaQus_K2J ziiwTAga9Y~WgNjHAk6Aso1~~))?{X7nfUqTS+|VIiavdWOoWen^rDdvn-Aq8ywDP; z`FAb=PRxcMLEd@+unN-(t*K$I&)qXk*3&aG)Fh;VM&3uv$~bV@B&I9g1N$>sb9wkh zN+eHELiQn3d3ohmw%}l^mqN;#S*U2upfAGHN6eGk9_T-;#7Y_l3DOJFrjyq>mRxM; za~q)i+6JC-CSBGB062=|k{)O{&Fui+IYO)Ap->{_Os3yx7>F)gQ+mIu?VW$`$>(ho zwxNw2QN#s9R$FknYwNKG*Zl6G+?&L|kyDb*?S1_$^u!0Ejn&sSGxp155&}+B;$E5( zB46O3!(NW@7<7cy)uj8OOb`>mqaGWp1%s+El->L zVw3E0PevcEXqH0KAx+Slrg}AVM9M>c`g~?mLND=wd}u=2$}Tc-F6!s2>B%{& zWcTP3!BkTMg_(bl#qxy81onxDjKdm2ZFD0w>dm?Fy%__7a2d$?GBOydFO?5wS16+DSG@TlVHXYoAz(G$A6z{Tr|OoWWdW%?Dl{8!B!3xY~jy~u+6_%l+vD~Jd_A-X?Y zWvm~3SVli31*{r9t^vMUV338POsqm?uKu=9wAkCP&*z8p>I#L*!{Koyo-NJLb@W5M zLMxNEh*??B=e_yI5R9YyO$gO$8U&sJUVD9Ws%qad`TWh#Q9oU4yb|qJAC+Rg z_ZgwvI!wHH_TTg3)+EG0QUl;aEU#{&pT};%#bGwo+VoLgD?7Vb!^c#^?e%j%UT{1` zxq77<#6Sy`=q8-L*4EMKZv@!){&9y``25a$h)hgOxX5sj*6Ib=EG`@9ncGnfb{ZEc z2rd<}u(%xC+S?T;kqs$NzMr>715;3c1*Hwl&`m3%JAzlA7^_t;ZaSKz;>r7`@#d|` zz{r^Fxam?f?^4vcvXqI#q(9k+Vck48D8)08wrcTKgbMtM`;b?R8eDg%is1zIl>{D~ zn^Z6jYBe#jWr#7pzFTB&5K1@pG$mPOuf)rK(1TRQ2Et;AJ9Q3?-W33`40VFryTX}r z$lY658xuB*dYL5pwvM@UviS!qTM+MkpKE^Q;^len0QURX;X!fX`fZWJ2<`jd*}H43 zdR}glzesqneMfmjlID)G2n~XD4R$7ygGxnZ67+*dFINjdAY2(jhW&_)N;Yi4WvSV! ztqr8)@CiYG`+x(SwOsj`D*f|k0QJ_raVC$EM$KSfEzQ~jwz>C}Qy#x)V4P+j#d})@ zF|26#!tYJMv169DwR?2+PZ>T>?0I<+Z1^=Mu+$l8&~_%$5*cO&L6S~pS)N_NG%#3c zDl`yFXVY1M(MN=0k*NT0alD*+as2Sw>DymFOtN*5myNnz+lVeI!a@JrBzmc%#EyiR z#}Ig2dQx=LEIJ3Nyu(>bE(n$mVOXG<4v;mO#K_w_8siWSop8Vwun)5~$Db zaYd=37f}pLH!-qSt^p1^3C*~TwZI2jMoeaM?!m01-Zh(C0ctKY>EKz*R5~>KHyfj& zrla0lDOw(AGNCqKbBjyNVlvlQy#$b*(S9wr{P_V`CI4*%?-m|yk(PX`QNlgdt0eb~ zaN0$mplf*#d%DV`i_YCIbvc!^@Ie0R7JGG{EQb?En~Ys)y6|@=ZD84{Q5VYxTv}q3 zqzGyDl<>MzG|sNrI1yf---scg zupO|nC~bX*OOQ@{qB=jCBRbw%mp(Lw%vpsM)Wd_EDSslDrA7;N3^Z@O|3Ri>VlSZD z{cYg!k(;zG-$*YhZVAAd$-%t-o+DG*o%j@}S)&H?vZuMP!m4t9GK%r zNEHpmNFVT^dc`-JUYbbIn#z}1SbXQ+(Oy*q*m=29#EbKT@#Mo+(aiBJtk7-?TFbt7 zIdxLAGh9NWlelQ$c7`!1S$s$6WR563QVQ1=&eF#q+1~<;|6- z+hE%%V%_Qy_VZO5B_CZz18UC_2v^1-m?*QLtc0fiy+t|i9XmnXxK~oSx z79#-_-s-qsTfif5IaPpJpZ%qfr}OEVNP&mnieqXO06;?w;I$_wA+(@q=a9b`joUa+j0)h%w3v58$EuHn|T z`TysKzeKXKu#gG^5@tEL`*#2jGAV78Kn9+?!|m%wh!3t8w+rD>^6RDOAOEUZ1;M;T z9sEbiLDx?D>0aMdAdndsQG|cH?f(uD{nx+tv41Z9yK8^hNT1>Vhpznlj#`fYxaJ>S za!mc_^uM(WHmC~qKkNDXMxfb$T=P#ITOa;MT;cBA! z{vVSFCCFRZg1^q8>)3Z9e_ZW7s1to7qPsGNmyNrwz4xIw|Mm=(7!bMBGP~5?=kVtl z!_@bA`U+*Ug+k-9Olx(*+1@AU^eYMoqI%(hh3(WoC8ECtXA>XH$cetQM{WCPcCe>` zD|qi*0G?i!&F^3A&Hp2r`v7ubn#$X0sN@zZRrASp7fYu*+>04^I-cite1WGqk2EHldWI3CurA`VV{1!H(USDDJ(M#MAu6ba`$dNrFB|h z%|wNf?0pxOh1eH$c+-_lJfdI~z@*xTk*VLFDu&HOf*cam#Q*sy>jZHiQq!I&lpYFx z%Q1ZGr7)^FR`Gi=d?kQ4MRzGDKiKVkaTGIbT&2a^yK=tmqX> zyZ_MYAYR!2>S*BF6SV*9+aImpZ7-x0PoeiS1lNTDvovkahc)4kq502H{%k%0`2son z{(e*w7=I>KQGy4wCcLuq|Tn*TJxc(;CUfbJGnX*Ydc+54*4op^t_p@+csDKk3%O;IKE z`~bn8V(<1#Xfo=FinERrOnH4A4#}4@EZf^4&DHuG*ppvx*6f&IcV<+>VBr(P|{FsG{68STn3(Zba)M>uCuSdo}%Fmx$h ze?oSJM`8wNb^NP{nS2N;$15^+9JObk)4<4Y4w6vfydNRh zR$eQ57n$gxHSZUY>5!VHMpWRYFA{(KDx|k>{MI@lQz$d94Iz|~)#4CiAnkW>%3K6A zCF#>?)$TGXQ?4lZ8xlv;=_wNN9E^&*0a14<4LW~(t0CMGA;uRL5%4zP4#vc{jDtdMf|ex4v7|H%!}qvo3@ThP=T)glIk$v3wI zPqo5|)b-z8LGzQ1d`)N2g#0Ys*pwbuZ|-ZKb_>MQ3a`{834wN^Ho&O?rOBFFB%h24 zq|#|8YVM>kU>DX24~_izU|=1v(H=j3HLhgKWqD3Q`N_cC_?rUdj{3x61GezVZ^T5y z@y5J5BkzLdc+l76xCMqP6i(aE$9&N)O3N&NMxnOrgL!bbW~)!?I~x@WS6~OJ7Z$`V z^hn1m8#;1~!BGWOf+=5>g-(l0o8oJwp#EbXSYxAuOlXcUpXtf>RLg{4C+mFZec36Z z4hfsjy5cGPz~Bc~4F9OLTB@Fl=ib5E^UmyZVvc^d48J{`mw5s`Lf`V?VbwRfw>ytK zhV&oE%;umbcWs&*S;KnfLwy`TPcl zmk4hu!#6c2$7{+|-a(>o7YN$Ei&^09Fe&AcFsltfxy{s!uWv!aX*78=uDx?lD?22oR$`mN05xZ*fBD zi#2)OO}v~*^de=cI;@?U{iVhV3mqN_4h=b*vwAO!RJb`jW7%l?WlpJzj7|o5Ym$--N3L!gAfG#PXWaDD4k%n&{qdEX{ID(cnNOVQ*`_P#u$Wslgh^s8M z$J@pl4lAHkq39-!p&*1`ak!cJeBj|>p};i!YOlo5P&nikcJ$N8M7?oGr&5KV=YtgJ zYOJ=2I;(+$Z8Ie62QQ(@w5uB$a>2?aZhVx z{p?d-ThJUNK-yVQoX8XGzbTUh5DixC=>WA&Y~HZYJsubUlTK6)yG;^9H29^!xZzal z9zpaiKAt!fhHS55nWpy19R&!0li$m{9o=-kvB+VPT!>{gfOE4#%qL??Dk(LT|Kfgz zVdHf6*W=80^l43_N{*C8t zw~({j^a3&M+hBVB3$oJ35o;dmQZ`yNmD==X4YFU-_YUBKoXvHo0(00;4G2Zt#?)l3j z##N6b8DV?(@hrymF2)PidCxNe-_k5NQlsh9Ig!!+3H4W}fYBrV1+ws&0Uh!t_{T%= ziYI5|03-S)k3uvYimGqk96ukIdraV`fS0j3A$PA}>z4F`AD)`Igfyl!B?ItWV-xZD zCe9|YQ-itPv=}d!=;-A9a(Yl#$K{?oN1qi^?E&6dqul9%*(1N~Pzt-@ZCrMN8986- z4+-TJuVrScZJnP`^0%sYh-Z6t*L*8t1J|EN^Myu7HBel+$|^>}_<7bswZ_?bbNhc2 z?ceU+Ii%;~_03By)aw|slQ>TlY_H^mom5s4 zx#Q9xAn3SWApgN+Q=J)1x;7WPlCh1Pewqs4jP~@3Tbq=VMRZ*5et=b-&@U;qi%~?S z>Dy}P`V~(MQVC=Ac@kP(&A$2Brp?6kBop5xcq;3io#SJYrE7-}PaCzEEL)scULrvON>>tpexH$C zRCw6gIA*q=s12ysD{FXjnp|Z%Lnb4)1by!UD3yh$BDg+F-)*tjdNoYs36|aH~|x-51h9D3jAQCq0sUuNN!l$<-Mb3y{M30XW|E9f}aW zsAm?vDj?~tcyk-Lw3T$Qz*QCR;OKt*ws#?ik}v8McAC4IVW?MpEiBq+XE?=?SvTXp z4JdRK^e(2_@WBmqOv2BXcqdYG2{#ZzEuI3+i3zS7-0x99(>QzqLBNw zNqkcIIw?h*jr2V>mm0{DGw8@p-I>&o!Z7bCow7|scwLN}jz-o4Gi|5NbYB0tvc4&I z@RF1mk3b@79Jiod#;G_W^@N^0+K8bqDnX%@BYUzQ_%2kIGL6nMivfe~pYkH+)fAMm(LR5q7d{Kr9{m(mC28twsSodS}J=oD|z@66hmuX&t_s!-YMV zsgNDyfsiGYu2_BJo&%n<$5vPxQ?g(59cjo7_YcC&FZZfW)*WImnVeETBx9PT3kI&O z_n}K$(c?(Fa(fsIT}V8>jZaUKeZ;1rNfEik-PP04Y-8oPB0Jyjn_zXh`ji&eBv>MN zMvLDF(oin|y&Fb>pjm5pP!ie|miLlTuo}T$WWp~NFSxP=e_Zi?ihKmU3$7^jEM}Xa zJlfDlmGVTwZ%~7h>{wbE;(#FxIR4x)`+yeP-_k9E^TNK;^%RNFA;X89N^Qv$5V(UZ zJ;ULYOXr@Oz7=?Ak|ENpF2rh7D2%b_1d7{{4w+kXePCm%8V6XEYgc$e)^JeIHxv!M zQG*u~)qgpn_I8cM9nYXM)LLu0Yv*KxZAb=O`|43qqYpE~0t+LX@ z%nMU?kVQr2LIf5|yLl#uL){coiDS!RkHrVw8?w3qi??hq#DjY_xMe(SDbedtC=jIm ze?kC|wPVJAIWX9==ey|l4v#}L!Zm&-cj9**9xgi# z)0}UWP2GPi{A2AYxN!7Cg7r|>pUC|#j43EJruD&&o04grJ9Tl?zUP%x0_{6s>{a)Mh z$%Z}R56XLA818x6mfcqU2Jjt>-vywPMV=C6*ashDA!~y?lP!ya zH#8x;(3qNGGW2t7uD=CT!!BUG%cr&(BXvdxg* z9|nw_QB5xmDi+8KQ(hgu@%#i;{;~f~h~?TAIaR=%Ql0@;+8`#-IHNrl`!o1`!~u4> zT4AL!z&{{;m!GA_k|a2i>rYn}Uo5o+xp&ZGDJ*jhy9r$#jdWi=HC6^qSF@zQB5toH za9e$hm4mle;cIC;Pkx|=^-Q%F=EnfVn&m!s=7EpHw`yU5`@9u+OiOVnW$z3!oxk;e zM!)?!jz|8efES;xzB7;+eIn(>&*qiTZm7Xj?Q-^0rw40}qn-&Z%MC*U*|Bmydnqs8 z5tsNsB1v1^Mt`}hya`02Q@sz5&+QkR{?YL-%fU)4we>x%6F-;hE@XROqkW2;WA5LY z{H0{Y3MJ*^D1Otu5m4-Sq;E{!ug}K4d%##5k3Um?>4%}%zZ{j;Oh5FzQ1#FCznnfl zMB(HYG6*@_IXmx&W_=lz1pzkq&P=rxMCLt~)ul_d$57rj3JVDeje>Zh&vZvBG8E`5{HaL;P`!8gxFYvnNE1IzXZ?HfbPap{ zWaBRE(XZoDX}k;_ppjdbm`K~yO(wnPuf%fgI&a_(z82ARb<+4DWEGgB?PGCERXRS5 z+SX4+TqbVP__|B1c#i}&WMGDYIg=q}Tg%8WiXx^;lF|r>KQ%R4=;?x>qduox*Fnvp z(}*ymEK;-k3_6kUo3GXHOqZN|JrmA!i}C~uU$eZ+Ym}LlK`|kT#x@c|&)}0C=+#i@ z)ng+3My90(l@;Hp=Cm!O+1<> zO93k~s8@Q!LW{qQ73(lHbgX-K?f;z%&@rSkxzDfA!KGlPkP6TG^T%qe4)A^QXaW=N zl7XI)y(b#$hIAJsiQu0>!+_+!oIe_(tcLqn>bLMm@XIuU=t!M^t;%G>Ojx|*v-s<} z{|nL?d7NYBwuR9NX6T5l<)f_ZGRY2dQ~r7Hn_CJdzz}Sgw{2mbQT5fX5>*#2*WqyV%ySwsD_N)tQ zXC3dL+&6mMqvJpre%B^Uyp(Op3jNQ?b(JScLiHh;U?&=1a1H4PYW2Ld4VNf?xCHb_ zA#!qvo&Tn5BCz-+7yF~v8zc2nXDV-{Po1SUj;-5)w}wg-!lB3}!Z}@eY6h^)l8M+5 zezwA3Nb^kFFK%@V#EB+ebVn)=TbkR9Dv}W*HuEzNo7?4i*8Kx4L~LtrvT%q*N#4!T zzBlGmCgmL2NE9xiR@*5J)euH4ahb^Tk3#FEn653R`F3)COj+Xjhv*$7AKFQ6l7rtR z?Uo;`gbHijN~uY3G2dU zbSydM>*tbIFls>jn7P%>!MjJIJb{LgQm#vFsHQ1JOS1yjevo@od0SJ)*x=10QaIP7 zHI7|qsTx6+Lej1ac z#O!_*n%)hC9Q=8QmlA&+`L8$S{O%`&4z#uUG>~MUAHR_W(UfWPin`jOKXLV;c*bIL z&S|ud8sE8&fZ`5UPO6z3>zh?7?SHJSE30}48jlw=@=x!1C9ijR;Ix7Q;g?Y?7-hni z*VFr8*JLt7mmzEa6n`q&mbhCN8v*7#6_rpmxxzOdEjzquEKcLcIa%5zZk8#pC2*fI zJe^6)E$-i@$$ey(KNEuT-Zq}k6mKQ=3%;sM?zH zceI}C;yzvbwse}gyUcH4u5#{GE-ai&+HVvDMWrJ}72-Vgxpa)bhxF`Qw2@R`B9gK( z(u1!Em!|BcV{C&9?GC$h9)q-T^!?Hhn6AEtGO<2<4@A1EA!fWZ{eJm@5NVYf7ipKX zf2urkm;*Ot@S&RyVh9T5QJ?b}fiB>X5#jz2)=GZhQQ4rNCS{9^S2BW?3<<)IY(N4! zBsekzAT2%OHbkPn%4HNR4vbpJ(_w$PLQeJxh9hcRCEyiC4 zp&K$N1#U(MO41S0CpI!Ne4sO0DM?03IwEC(C-+Z~6Nieo+FIdHiiADNnS5WyD@RK@ z#KGyR&-XnMA7hc(26_%^P?C+buM7=+oUQ0~ zYL!2Th_4`iB)l^w7Gs$N2}PC{ zgw#`rW5WLHV{AekAj*G#44Lr%b#CZU z7GE@Wl!BVGJyQr1;tQYYiqE90sG*PhPfNCKz!HD8DrA$F_5J529ej=pPhrGGsARQS zt^-y^Bd)X|_x_uQ2wXgzTay2%LhxTzy!3c7Af|uBlxYtmBoCU!hJ60rMM#l#!Z`ob z^Y;l|=6@IR?*_AtO8xy@{=OwIzSocA7A5{5-bTh!`bpfz$mJl>lL+9_ydx{p+M> z08o}9-)!|>&mK5@vtpHkzh~({QCRIKkT=98oLh6U{3s+VxXb6$EoMxUeb$w0hE`ev zej-xL#7)=Qnnri|b!!2-tcl)x{KF8z<2aX_&Hq+KCGh`IQ-oEhCGcCOx4nVNXZe(X z#ZvF{>BmP!f=rL+`*wdb=B8+cOilg4n_ShVYcUeW)%~by2_BIbW=(vI`rS(kmy;bt zeguF&k|OW=cukvdOztb|i9qTIMZKKr(D%;n@f%pAaG??n#zuYw2R5l03uEG~y{3{R zd1rLOS4^zl$lU<&RlsgQHhrd2Vv{~ggU>tUAOj8>`iG-<0eK9J#9cCo2C9kmSr`+t zxU)!pNbPt37E=7qxsvwCmVEF=4Kr%1A$dqchjej>fPvCzKQp5Ie?QP#`8+j-8yFGR zImO$gSRv4}LReh)Sm6}v{9F|yYM=Ix@wMUTiktU*H^0&2;zP8%+}WobNlsy$fFd8d zFL-)dYO{(3{rcl1*S|RPLzu{xZ(vU?o+^MXaUG%XM?rMtk5j%6QzvEb51i$fUmUIu zH}xdDuSpw%h}PWSAUor_^X6?9oc~-cgLZvyVwH?)*T*n=yy>G4SWHf1Nwhb(!b$9P zze_FZc*$^x1$dI2?Hh`8rfH+BJ<4KW61B_OY!XTCoy?KS;W0N|1Hb}6yY-`~T`8;v!u%BW9LpwxxMWZ9iY(*oP0n z1}us|p|h5)5Sg!zUgJ({%YCPy3pL| z0&(R1J>HDmvl_HEwZ$BZV^e{~w$>dJ6*WcQ4FiO2s6ySkS=srbaK5uGc;)=e!uyG< zss2N4XgFDRG=17^B*~<667E4}ik*!;iW1lb4-fy4Ec*X2_ZC2HZd?CwTMiV8yE{cn zu;5<2v`B##cXtRHyc7u#oZ{A^rMMJ#_monsXb4h@6I}l}=iYPYw)g#J{@**_H}5-> zXC{OvPxfAW?Y-7-=>>6)?@x|U4lP5&(bCHH;kW46zwmH{51H0`;LqZ-HR|h0p3~|z z{M&cp^mePZs8CixBTW*gE;OCE$@RMC*PguAA~K6uGc zD&gaqA@5uBVN*Y%q_;P76p7827~DD+E1ChY>FT|xIpejt>ik$+ja0^yyvYS`yoy~W zYXou8r{ot`X2w5#GGD6K+t=uhHo99l};(QPuZ53#~BPe=>Y82W>ZcTOeM!4Vj5;-`EjOoh+T#*^_! z_Qj-$)DfV^A#7)E8~u*~##gxGClau^B?|6+Ub)fsJu+?W?l4-&kBocs6q{>9gnPzS z;o%Xf?&|brI0toS{278JCROo2(&bb{2%hAOjWzI?t?den^eb6Cyt3WTGM)U2D-Y$* zTAUfS#mG}}GuAo-N_wEY6Owr3NH`6 zVRysK(m)5h@w0i?*yfEU`NU!$F+s&x&OvYM3Fjo|b~% zF0HxJW$7<$o(zr_QHCnwgBTJ*MtN`A$Pc-epSJI^@8;vl8HKR4jc`QJ7FV|5Ka@$c z?=p>l$=>@4GfY-MK;Un+>7P22$v@;*Wr6BgV&N+Y7fy&bWib0Bld+=<){E)I(S;0N z)_qF#Ia8_zX~7qg^%VnM}kY(b0@52Vwp)lPT)6E{z@*s?bRO8ld; zw%F!sQrpNct7(_a_&wkh*<1G}Y$Kip$3<2kuC1@|)XrG>9!0TRWFg7*DbxzD;?VL>8tH7V6JC>lVM+&>=HpOI4*jOZR}x zBX?;4UIALDmgr)Fc3}^Qdga9lyycy$cV2w1s*3x!GWCyBOyz!lYwNsa$Jd9W-l~3A0<=aArLObpdBQFBnkQdsMHJT2z&)U$v;T2V`2EV}*9o z$Bz~VBq;o;xV@y}cK_q3?Ec(m9@QAD%I(1X`zdPT{9h=1z~6O?Kg<#%e_buqe3Ddt z9SrKUO0)chga7=oZkGQ~bZ7F<4NaLd{>C1Lsza8EsAf2*Hg8i?Q-9uq ziU{1@-5t#mO)5p5L%b3aqo}ZQPfyRk{`VhW=c?ES0)et-W{ib}g{aWxDOBTFRAWOu zG99Ru-X9?{&CSge^z_AzjcKJ_zhjZc3{-n})WA^T?QiTd-le1vLHio@0Yz^}*$c{8 zq9L$uRw0Q&`F_P;Cko#lt#@iQ=IOpdWz9Aj#+LS(KLWJg!Om~L+xaK%$kPW{=yl39 z7DON31dZI?lqUIycBF-W*Yu}L!gr(1^l(gD3I-ijjF9*7szfZF#!zhZ!l_V%J?KGt2+qW{kjd#pM#9Z;Th(9q$*O$@^^N4H1z0?Yz^xZ zBf654c@Wjo&Dl@=-@tWqynO)O3G6vwHu2nx$1Yj4>riuWKw0CyNtL-HThQIw$T7>g z>?;AD)da|X(dJ>tDm`B8K?{0%+ocfQEWzsCmC0QOG#2^`?_$0^2&+zefau9*sVM|s zfP?K`G*y4QkJB26&jsnfFj_E7ti7tQzWG9sN)Zlus?6W85P(hXVm#JVJHNFFg>Sgi z1n$w>jo<==jnWm6$Vq|yIx75&sweZ=iuUf7C1=toR z%-8DM&$oVLy!W@3B+P}tkuKG{7dp0a)PPwJY3P74f_z?o_PDbBg=u55rHo{Z6*|S7 z#?sOSP9or)x!de^*F;)@d;g$>f=-rnY@zXjx-3g0 z+-#QRa=d0YJv(b_?2vnM~`;Ow4-NYVCvr}}-^Wa}#H^kQAY_>zfR z7y0{|`dGswkt^e+SflCD@7ood{o`sm;1+Zj26>rIs2v!SB!#c#%f^r307vN`K!+_w zA%BF}xaSGqn!%uxfishUYnG9V$IIEqh~yU=8e-MCux=B9Y!}C-T&upG)|1Rl$}fexDO((};&weg*cr8i`*2;MtfVx?Ij` z_zH(&4RJ%`kI*xg6bdJF>M@Xp2zr`bE1mNSEA2Oh>@3_q382Ies5Ymfcgka;=o1c- z{Cq){;}4=%5+l|v_T_y*>Ytwh>&$C)<3?`dEcvdAqM%F+bf_c{Kd9zJECp8jKE}Ji z7ksFr?ru4Yfgt&I?JzxRpmy?78vaJXEiV^JuJ$FUDy%EGmZHZ8r}I}{2(7F$JiC9J zo4aGmgs`~9^>&?aZX(geyz?j1VxY}rpo`-jdaOG&n2}kGaDB|${w6M`)=nyRTV&;} zu>RQFrm5&0|7;V96V_sJMc2rqA;Yhcp#G!Qk(gqTgt1Co>WiZ}fygxqynRYG4zton2ixyDl9S7Vypo@zn9|T2`kD%$XHgVJUpr^w69DI3k&~R?W z(NL|X*+ZUweEZ1_|E3MC0O+%XFx>=0uqobjUE!dixf*7UsIdCrpv~70u_?_WNA5-s zbv9~Nn;#2IsaY)EEZdw$7he>!o$*@#s_b;@WLO}YxmQ*ig({#27rH9J{h4$delt_9 zR(L~)W*89jS;GxReT|{suDkxAz)PCk!Oti^Dp5zec6Z8&x2v3=IE?PastBH9W12qk zcbWtBOc|?Sh)`gVub3hhS9y`SerUxhg>_?mjos@%Mxp*`kl{;D|0}qd7s|hS;B_S; z<-%yp-*^5p_ElZnI~QE{C(MRFC89-Lv5PRrC;qs>945=}JkX${TUI)u=xvrRtraJ7-lt&!0Pb$SeG)w8?Ws@|$O2iEW0Q!1Qps zd{kF2rCJ@A%?a(91sSdB=o71@BT|y+TzKae-|!6D(i3N&bH-t{6T&Q^Tv9=dEBa-n zhWWt_+yqZXsZq1cgpGMR;cv}Jj2V0 zy_{|JAf1`V@{M(}IqSuxh{Ra|N=e8I<-mdB^fqAI&{wQ+nKU-8BC1aY{Ae=2o%6~$ zQ_X{vV{!R_2`u|-6!dhx#OiR@$QM$Xrk1v$mfY%PSAVEW6PhB6Jj>eKGkQDk8F4NC z@xoDR)T#`@!XwcVf@-8^d1?s#_Cqi5WSh3GPjf9BrQ4sl6A3S)`pOHNCfbx)2ulYB z3e$o@o}ogZ+$)i%&jcNGV7+y0;O!y$TI4K)sN%&{t`GQl9Jfq=IT|i@OuN`$%d=+z zi2LQ>#xxORCSP7HUu?0Z2VFQ>ERxU81O%T^L{FK(LbfvwGcZg-;nTviwO3ZY{YFwb zUGfN3iB>NRy11^-<-h{VJkL)-2N_iCw<`w$2xUH+gRnMgRJwFJaOW$p0t~rs* zoO}E~;F%&>3A%xLr$^Y>zv$QPt|-&|-hwD9`)RCLu`c<#MvphBwE0Ped0B|}5 znJJOSB7TE4yi-v&%l-6sKKvG25~&5S-Ie7VL}@{d5dZYTurSI!`4d5F{KSTuT zf48OlX-@f1{rVhO`G}w{r&wEl$2ebM(yJ$Oc)XpQlQC!>NkplA^%P&)cep< z4x&iY3|3T)I2?R@i+Ld(C_*Gl)IGekix&ql!bE|isihS%vmNX=2lzV{VBP*tbotAx zg7vRDIr2<|p$>0AVk}Bx;H%`B*?YqvzDK z;5%mj?QvXzcfJ=ax6dT0_x$6P8l<}0yqAW{8(iS<@@05}%TnLl=h7YO@YSY;yFs+k z5}x&>&|&*o4;UHQYch+2xn!P>7#C?*;>n9Y__eD~#Fxzwc|=mm7sKM`JuZt~WlX;h zNZp$6b?ak5yEAd7w$Q3@Bd}*=^mXPeP3}r$>M4H~+?(BWYVoWVR5Y?=r0!bhnO3&V z_)0X@V81a^d_Ri+w1MZ#mg^W+ICh%0UU&S!(_eTKWeCx%Nh882JQ*u9mWqp*Sz}(1 zh-@drtFzXvGry12t}NVIQ&7~$09~-(Wl=Z!1QHepBJo>~wXuCt&v1^B@(jHpFQVAF zA$y0i*5z}fZNh-O&IfVXYqJjOht~$15rvxvyB+>-q9&4}$;Urbutf6*1{;av6Bt$H ztg9x27qL$F!5Pat9i7)ocjV5GVt`qXG$&t1rBiSI*oGl98(@cw;W8iX$*7%|Q1wTF zDrl>VY_m=7Y&MrKnywO*?AW>9U-|_--yI3+;aPfKdl0VbXgm?U3>;*;L?KZxWAX3L z`wuiE+#lmFHbh=V$e6FtKoY|!3eFt?Os|oRcLv^*x;q2Bxc%bzhohBVUw|~5MEtt_ zKU?l1#y(G-<7Q%C;R=scj7Tk9?PCGUQC}ZDvw#`79x@`Vs z_7zlESaJT4?1($b5b_}ac9Crn*;JG)YdZ1T^RcC}iKSob+dQ4Dy4J2}0SOj!dm*YHeRO)l;_fKTBYDQ! zr#hnRjwAzC2U)t%?<2vSO}6H86W3i>>S;id|_iAcx;{eQX0WN9%gY$hhxg?03kZsK-20*_>pc zshlIVv)SH~LLlA@oq@#?ebO83pK7Se^L}FlX&L-g`G0$GV!=lWfIbEB$q=*Zo50!B zTNNz3xE9(#q94vFbAnd&@akgto%*-{Gy#V+V$(=rdj^ev@_~L7`~DU#hEu<1F+V;-2@7%Qic#Tg9yF-jTa~X=4Qfug zz}H5cJiMO@-q$l;tj^AtRBkPo`{Q}XNL1em4FzxGs6ktr)g+hL(w-?x`T%HPYj5uZ zXwjccXLs#nTqv;#{kS0~p9f(xmPfgExm4uGg65Oj73-|HIchpL>~riT^pZTX`X-g< z2n8^7H;2g85%pA3X!MhtsNzfZ!y|Y_Q4sIr)0F@ZYRU-7`sj?_BSvzIZ!fek3PH zGk@X$tcITJ$GYc9tL}a>@cB`J`$`GG@Pd15Ma)np(NZh~8fkYQRl*4I_y;nc2Y@Of z1N10MZXEH;;!`{6ZEO;iWf>MX{5nK|@L5Xsrl~0*Nq<6bbxMU3OM}X2sNC}f zI4+WB1*oqJJMMBTlP53Fr(8RH8TSVPLlw68 zSbY9(>7D2cj2gEH6xYhtuqTEi{E(%MeF_Sym@1vQ}F>jPUm&&}rzgU9-oG<2s2 z>*ma2vm+30=`yxmwAbrxwQ~(F_!Zbi18X4?MZt*1Mifprl9KGX7ji~{Q8dsyR(L=4 ztciPWLa6>`$7MD^Xv_uG{n$J)B$Ix#v+^#o_$-$hade-MtTKqA=h&30l?O7COhTfg zsYPhLVu+34zRQLpEI~dHGjl$a0hdAqRsIm0A{>j=6v^Flw6eC^cmeXd-o>9YnH)_l z#J2rZy3zlzV_-Cj?R>q0x!3JC1<-#tO(ha=b5m@$ozdC#zQ7FKfUY^)=QXK7WpkLH zvw^zmw^d2J>TxnPT%5$s;cysDp)j>sSCr=~a^I#=>g@jJ_1c3?#UE00s{Q~uJ)~ij z$`XRZ=J1#u)qt6`|2!Iyx4ur2^00|TSO1x30AmmNcUKz7NEVc~_rbbKrCmX1kGRUC zV!>*7N@lta*;(mJQb#4!*@ z>XYU4qxOr%0bOIW`xuWuFt7>@p~4MV`=4DER~K-1`l)os)Upo`P(;PVqhG^;!`s~y zZ&12LM^%*L-95EAfO8Y*@x+>)vCkgpc8@-2cu6WLBSZGlgWoB{rKgh6Yf8=2(Ucn= zMg8&7)yIs_Q4epc?z8F3`1Gy<#YMZ}H-5I%Dl13g(K(2|Y$dSch+@5zP>fcwKBc#; zz^K7s>*a&fyX)dCJ9^6>v&t!pwK?T+B(8be ze9~%r2QwIY`;vY>)-%4r#$B)Cj@NJAWu=IEzneOzzt1v$4ob#9In%XFk1+tMmQ)xu zX^&oNICKAbfWG0+P9oOlN&eK{0*;EB}L&W7)9j`IpEb$SMY*`OJvqW(qX`5z-6CyR=y#Uwua z4SyYcUPIM4k~V$P=wmL)Wf*ahu4kaeK)|Aja@+62iDrfpD7FwWPSWruNheDhsqhh9 zn4Vfa4sp3%L6aSZrywafDSO9v6=W>AsDziP>w;}_Nt?0B_1;bqH*A`sQu0L0i*BEJ zU$G?jwSw2m12+AKO$7yk+=X*=YPk*;5)K47sagi>o7Y>lp_w{b7WiMpH-Hw}#9eC& z#=fU|c=a)+7cV|m#(f3t0RlqgJaVMv7KCH;YBN5Ta`(_l^znBrjW-3OBKz(KYg!A8 zy$?0^y0*rb#Ny%4w3*a3Q4J+Ll|YKLjN>J!$GG15v)v9`Z;YkOqBa>`HJx<5`M zI>4T^6KOntYWdN{Iqy>)0W0^>DzUm7m?0*Ka(#3H`Nz)34c@vb* zk7k=aIRym;Rl*#Fpxe!VJh=ah&I$i+GcSZHRP*-jgQ6%6W8*zwAx=~gkLA>n&nr^$ z6=YhYS57ZjnjCL;y|YiFLcyW^49)q39#JeE5QRiwcE4U}=B8h3=1y&@7CM;`8gk)}lzn+K@GH_BX3E=3Cd8 z+Vi|dyg833fC*lcN@77~pVu$82Goq&mjIzJ@NU;2G{1YH2>$!H=D#0s_dn=Xg#TH* z`raAn)AVv zXIQjy>JO9|3eIZgfb)LYb8~k4N`qmzU!xU&-ue6+(@@y_wwF;ZVv*nRw+uo5e?h(R z|5%(*(0_WbqxpYd-9OB-{u4_h{&Qjf{~ucIMJ6xkLUU+jsB^m}A}r+C%M&O<)%B|p z07a~&F~?h0&GgUbkGPvmpGe_Rn_U@syKQEa;~Z`*l+3*&BSqX;05yBjZ@dH-z`eZv1)bfy7Fh58+!P1bNSTjAj#tR2?u zwe+oSk{jMrTf6oBh`pov3Zx&tO0zzmIg&gRn3e?(SU5&5*Df#rrbNCyXg5c4^DopT zJq(-)9eGg0DSDNK9CT=cm`KkJjWEgdck|uW)0kIda)gmu78FhZzYSq{XUZy{e?s zb)srC#G@6Taa#^4wQB&1oDl~p1sd>co81?slrlUt^O6X}_b!uslMvrkX0=&m4wv^& z2^*=$^xRvkH(6F#d`jW%#y7 Qs(xjghLr?@a1rPWau{vOA35b4~l3>omHJRx%u; zCvSU2pDxgq3%89~rES&S5IAIR+o_=`!J-MOMX+gz1J&JA@0w)yhI zrJ#L9MXgi-dJMTPgs)r zp1JU9`qj#uYlJO}n%k88BjWJc2D1pgz7iU3;3hz=6_;y|omu zFhy{mIpK(dvdr()PTO0lB%>wm!lumhrkT}z3aZ;~`T!y%F}|0=nCM>Ydc@%_>Nm&g z>{|y-AAwnbN_yPmDhin2b*`ba@h>}kb$pYJ>c+q$9rwqbFP(2u%zJTYI=1?%+@@la z#d?zIYym6lm#Q1P9DVXmM8FPI>Mk{h-LWnzJS)TbdL1_f`=- zr-H@R2g@6)s?Q$}ZT1089X~?*QCY7}dj*e|RShs(-@w(Zzjos%WRfF>AbU25u0Wcl zrk(}Ynr>NZjP9L6*%mYQ~a9?mxVPw&@?V zk4cb!*6Ew0-;#5y0pS)8`4knVG8&G1Yb0?FG5SV-OdQ>7s?IZ)^rS=$#w z;n);5Fipb})zwbMGnFfe#mk;9n#^R9Mtf`A-B{HV;!%ggZ}nZi#$5=$Lh!@c6dArb zyF{38yN5%xz&g>%aB_v``o2ru2*er?!uTKLsuq>;-%3~EfUR{xx3_-cAJvR*k*vLP zb4B`|9wTcCc4Wz>UnNz$lV|tbk9}d87Q2C=2}Gu~STe5np|6p@j<;3<`4E-(!F_%f z8TYQfsI<}$Pzl<;KP`p)qlfV!&&r{drl=q(|5sT@R~(2iaejV9`ZPT3+o-C|NG#5i zVa)^SmQ;;LBKGeK6TrS=kNMS#JY=z~)`{72cU4{|gP5^Kx&l@}I2 zhirh6%s4%)RNIU_>sAC1T^3mZ8bnolzo4U2dvrRdRCl!20EK-e;$m}pf+qGCB?q-V zTcgIXR)zqDzUGu$kbkX#$c|_xFpQ2jDgT~wD z#eO^K35?C$ACT}|66lnD?6k@tF){3|SQGc*+Oc|Kgs z&$o1aMdePeW_`Ruwby*5tp8R+bJ!ZBHw8!pV`FiUsFyQZfU!Hy7wPSOQ1?8~?N271 z9K08PsrZ8$$dPFfyZMEz|4nF=^4jC4)G`^(z8|calGutsw)^++;`u;)&|%Y&jq>Y~ zoeZtC>~B3APV>628)ACDrW<;#D}7`?=@j~Xi-CmLp1SD+okv!AN8FOqvVu8xdJr|d z_eOU6H{B8vvO-5{L`J*iH0^-{%tWz7kyIpOy2D?eVOeWqX<3gl1T*kr%n^$^&2hXV zd8fH7sHo_JJCq;bQP)7g{b@ad@4Hu-HKGg!=6s)AWdxy8voKvOk5`1mvaMdJ$JT5t z;l2GJLPUlSyl`iGLy1vDI3C2DYss2K-mt{}JMP=Td7Kb&(pROI> zJ;;c8_O04UT~a5@LrO>NLLdg5ZpfcW(sSRu&UsF@XEhzrd~Q!vzxRPBM{Vp{ukmJk zwwiG?_qi*1&G~}Wt7M(3TD$n@8mcFWS9%@Np}*~?gC|04Z+l8b)M#AKwLk;##?X-C z`ittTZW=P@*}-h@mL}5+!y?#-rDG$MeE)(JLY5s)39=2^(GH}(;$>5>H1~>BX8AIN z*02QYzhtaQY&mR*2GU@#Np$jiOUq9PzWB&qp4V<6+=Ew1@zcJrZ*hQvCsN>WwBHC^ z<2Spw@O&188FTt&L1Nezoc6lmI%*b@q9y7{_84w<>Z04XRPdaY&4wuMWO8O>V3C}G zY||>C^PbaK;3bCd+|ltOID7q@N``{5z*FR&w42T%i?vPPvzm|{@UdFfK~t7kTZY~^ zrYg*t^+Zmv7oo8#_ps^aq4Y)_Hq88}fa-Y+1+4-7`d0Duq!7`-;$*n=QjhJs)X$D> zx$`krfYNfyJ&DX`?=uIS{SwJfy?WLev_94Hi2@WSPjLl8p=r_$ zB$f9?WPau@KlMAZBva$;U8@q=70+3zF5kthOn#?&Qt-r3(qL|aWR2GwPU*izzj1WE zC~ZD9Efac1I{*?%L5R?-i{K5o(LgN9^*={_|E{kVnvg(fUQJO_Qu2$M(%rI*id|%8 zX127pcFUVZ)u7Cz+uY0M#=(OUGHOU=SNhezZJzVYYSH^y;LwQ)%t|6CQIvwuXZ8;h z?A->~mp-VzZ>~WfzZWp8NH9Ql!iuey-w4)aw1A&cA$@kz%d?yV}|Isi*&Di zrs=0_P3a-TsDz?b`nH^Yh1ZthK-Z3XJaUZR`Qvxo_&cWJ+KqAC)AM)S>_4 zC(n^0c(X+^!LYCa0m7xE)mS-q1{IM#hHE{V&R&KAha^h_OqbF$#1|?Y#L8bY?4BR| zJB%?@+T_#7YHsnGX3m-nSe+`e|9k<;{4bh-AEy1nv40Yb_y09yJ%8`^ zPr(;DWA?ADcpzi-_li~a{BJ|S2d8%PhZMp~VCFwBX=5FjI2S65yTy1xm6vF9ky&>0 z)X8yvwpW)1-o_>NrB}}T`W>}PYXbf4CElqgB-mJJu{oojpmM4OV&FVKes6&7+c8ap z%l=CC#K81?PLUNj!YIzkI%u2sA3TWvFFGH19U>2O+SS=DfJtn{RaXus8@Fxumy~Z# z;*d#fIZOnRhkJw{Zu-c<@Hx5YKEkCQa~UtGOZIJ{xR;uuge^XT-q%OmVh@~w&ub6$ zTeW9|ZZ|h#jzRwVCasZ~!u1E^u1UrjA;RB=hLjDS3Q@q20NoEw=={;dmi6-Vty}#N^O1 z%#uMm@pmkMck}pWaqjqLmwTY>ZJcyZ&TH=~hcgU}jXUhSN=y3X#zPyL9~u3dd#&lU z?gr-w^7by(Osb(=cl6x5pKNLU)`4EReFGKQkhs)drMc!KC4Xe**%xcb#ksc*QIH$1 zfwND-L-OocjWg_N$!UcxNX2GNma6~$h<6pgaer~(DOyE(_F%-c$>l-gm%TJR0_dG~ALi>`+KqK%!7O-_@nF|-jE%d_i8McE!g{NOj#?F~`7 zQ%vo*`zEFekUmuGRFDZ#(af9;I0f1W$1+I4p!W>p&0kH6=aquZ74 zf2(9-s%#t&>A1jZzj#KSZ->07bh)FC%G}>@P@hitRvmkD`qTu)anh^PtHAN0 zv5lbiE76=KJnYH~Mw1SaB$w-r``zWGTLyu|($d8=e@U99bw{-~!|SQ;eJmjwIJH&r2WMolnFSi#V<$G-jQhUehD zH_=GSnDv|wIf3tn-$DwQa$ARI{j~BnTuFO(oy8)p%aj5_@!{62yIQ1c(TD|hK_r}` z(@EQLj{Dg}cZlaX_4)VyMUv>H5#nlriq=zRM;{QscUi`WPK9pg$;?sIxo&)$s4 zt6;8?W+!4%M#}e+#`-(T4lb79M~|Brbh~-JQz+%!zEW%%aJ<|UxP#t)ve+WJyQ!1V zu*2_PX)A=k*pZTh>$jEb__f8EzgV(8s!+Nyc3&`Oyr*HgQ+?k+ZnM)eT(O>iKZD+_ zaFdYidReNX<#zJer4=$yuQztB9u$!@en)~nsb*TZR5)Kj^aj2O-4EGLWQ-LJdiGhneToTkC(xzXLi`H`!h++_Q;|GsM{ys%IIhbTld# z164@6qa)@PPJqV)V)78CT`_f#=7UJfJ#km-h=;1@s?)g?M%Qw;h4%4B`Md~MSUTo! zJj!rr++enO>1aU1$#Om>t zs~&52i}7%1;lhDQ19Cg}E^T;%9Q@<3#g;DmD5}QtYXEoaP)PwuUP4L{V82qMVjL%K z^JFGl0{>^u%0NTMvsPi1P>WXu@j0eMbZ=~xp5%AQd>tkSSve7ES0&9m-(pbXp(0hu z7{|@FEDOe&1VzJI)1Apw*%|r9LPpR^9;2_3NssiWSX#d-thtz@Re7eK&m^8MpQ;LS zW2$76Xp93ekFUH>^+B1{3XLBQGH~^U{H3kz6SA^B^>@Va_9f(Za40D+FRZUmLFxGR z|NO&?3Vn-PHuzpJA0)8^`(t*R0?X<_Dz6Ime0{YO^?W@qt}&{`31Ggz-pO&KmlrGt zccox!%F1|@2@;~*nq+@o8!@m-W0l65;)II0t9P<6V8wt?rlo{ArKhFDIL*)2$5>Id zcJo#{q8|3=6Ki;JjLp)>PXb;c17J^86&LnM4eIU@ z0lt5V^`-Kk^y`VGF6K&=3h)|!(fxP!M7TV5qRM5T#5;9^qz&dnaEglK(~VVZ{m28x za2LoiT>|>?IpsY!zd^GZ08gaUCk8FZvc+Q_2`&(Hp6VpdqEF-gs#s z`ItN?xYAfMMh<_5&zyoFj?kOJ#u{Nv!B~DOQDLOfH3>T=^z`+q@A_al1knR>LGYMk zdve)0+6YWa$I9?cdN`5^A4q0L<9dIY``%gO&SUAUhDk$?zCL*<>^(8DJTShYc#2k% zZpHGjR`et)j(4OjBuC~UT#}jW$LhFCKuG4^qB6sW2+~v@3w;p5B(D#DiReh9M_48; zkA(b;D+wDNGnu!@lW$FNPHK-fP44IoHSAWFzsvOUg3Jj~i?BMTT;TGY3JU9Zn*`;W zcA;gWb6icBelRX_=kFGkLO#m?@${Yje66%aapFRF& zB+*_S+ojQ2#gQHeyc4-%s`1B?zrs7{mBT+;{Xo~vXWX-8SJ;&-JJmYJvSXx<$-785 zOr}G$V{gMCncQR84zSL3#tG|ut8<`bf&DRS&5HT01YLL(H`bDq6h$d@?i zB@6$X?P7*u8_U`Yozg+)DtIHevki^g-Ht1z{-q}vv zos1zxgO23C5=)S%XFX+^hj8*iUT`<3(fe=ipMQ}`;<^3Fp~fkR<##o6{>_R$-~W*9 zqI+rzd?|DUeaKaDc~0a1W=qw=T!w1-s(;FH%j2-mL`p5@5P<#t79x0(s-bH*WZO)h z+7hpRw=I3JNL>L}DLPtT_Rq{K)W;4?xM2-o3St0wz@v zGm(&MC+;Zcf%j;DZKkAIu;EGpx4!jFnK8YE^Ckt$qFBv$$%8W$SC8mhPM>_r=4}^E zJu>m@=v}_)k=7F05*X)DW44*_^l#g9Hw-#daP7ANd&tmnp{=V_Azk$__no!P@oWMc z!}t9+a~oHb;*E`aqysOgUln<42o%4qTk|M-rNS^}ffySseBa%jO=7XvN@6{luJB{% z;jHh+%%~r9KSS2Dyxb_c89sar7tL(%v$+ctj3mWAzrHi*4c*`@Y24DTA8kqDAbniW zXhbq5of|Z_wK<>LNKw2)bc-lLFL?a__)_O=h)?s4tM1_3Qt_4P#;O- zqmd`<<1hN7!fd4)&Rp+jIyen9|2VqEu84Y1uzCth2sYuU-+xds0-)PrJi=dy@XOM?Y zXl3@tZa+iW&u-6CcPRew9)4AEHQ@fk{!mNGj`b=C=~;iCa+rW^FH#4m^b*8Rd}lQF zbsZV$e1*H*;T>OF(VnPPd^LM>jBs?IWo$Ahw1FwZx0dhXJ@b)`w}@h3>jlX8<90|d zfGpROd+0>{aK0tCQ_GK3r;^?D?s96!9;#{V{s?NlIdSpLX`!YlPD{!Qx9|g6Yn2^- z$e|@GEC=@Po0s!lU%oCWIr30ldlI>$CDBxx-W4o#?Sv+Rxi1roa0Q_3P{Saa*gx%1 z`k(DdKY#w5g^98=^_ugcLbm^MCBrF7#ltV+%AG8I9ot14;y=CnBlN5Q{w!L(+#%0G zp!@S{J^um;;>oAr0KbQF*W3NpfS#4*qV*a99!j{=lnBwrh=KkVp2B%=FMykK-P_``fqd`Xc_fI z*BLJ0xaropp!#ke(gbL%m@>T}%aYMolbUey^EvP?KNh4dVCx-`B~Eqq$w>Xfh0)!9 zgi`DrP@3bL(_SSfryjBMH7@Z%Bh(tmyqpKJz$cu9&P4NlS%zo`CTDcp%nrR!{O)sk zp}KQ$_Ae91KkXaWQz(e@G6)~u?N~HpeRPSV;_xmug{+g&VwGmYe4GR4=3&gXv9*) z5D2OO=2gKTbMg4qoTvP)Q(wO{OdARP>S$=>Y^Y3W3C3-n|M>%7oJI)vRn=$)v-W;% z!5a@3;B@rGn)(K24+~xN>d3vfPX$I_0{-Ra85ofPe>QMwOSZ`kNU0qcgN9*~6M-K8 z19%%|P#Zy-3QuP^5xI zFXD}vX$E&}jS6-uB^ymN;y|%`XZE|wZl2~N>*kNcQPs$}s|ah7BjBXm#Boy7@wNVH zHkSu}+!s9)2waurAupI8qofeoc$sgL1pFfJZW5m`?(5aepXNkOk#e76mKz^q-67-scw_er{K?EiShP;D-qX~d!^dt=0_zqgsP?e zZcYCXY3B?m3o9!?EEW0*M}Pid#G&tk61*omZbIAp4V`#u@5a^ajKpARn&_xO z;Qi#DIOK9eh0SQv^O_m!GQWrqOEMVY-S1R0)V_y%;`Da8k@MBE2j5Ee`~m5zA|4#) zVq9D=mbOpOkVuw2BY@$cM~Jt6=={KSob4v$p0bumeesKyE@cJ`Q+{Ym z2YPlys*#ZShblp-izWC%bNJE7T%fIRSDLb-vf$_17j^~>u^NBdO3$I?Mztv$E%>G@ zu1x3tQmFLw{Q_M;e`n{tppt;AQ^H93h(2@2`TCd;X2QnS2XP}|e^IsjVdPk(X}9YD zqp`mi{(WP0G5*w|#LT3(w0K{$x|kbNW1IRINvoQ3h~ad1cJ5(0Rqf7% zoGB*H%(W?RUoyScDvl?g>w)PEfc`c+bejOWwjWDM;)W3S)BsQ&TC-6pJzHB_*M&qx zL?Y9=BGC{3p&sJXWfGefl*UYy$&zIo4^mSX!8>^&Rw@}V+4mNH@iSCrlW5*6QMn^o zi8!r?QbLj*()eXtEEsi_9gy7E(`x@E7FP9|2%NpmaX)%18v}yD?hfyVxY;qfO;fL| z=LDR0Nklf3s_=xa>nW@P+>CU}!e78zkYYi*PNnj67 zoztrK_Gyi2Y$m4i@s3}p$qn~lR9I4)z|e*i_;;BAh>}TRB)z*>zLX;|S(;8*?g5r3JSUu0K0= z=fac}6wpc8viteDg%&4c3UjIfF(_V-Y>Q^m6WWeED0O6qEG1X zfJ8!|fe?8KN)|tnz(Y*tU_J?BdhYqZOxK}je9YqF^D^K~a7l)6>bIxFEvM&{=$c2x z?tPx*{Ktd)dQDsREHLQ_{@dbtiI(P4tf6MBPfF8GtBbPhdCpMTPaMZ3L(7lQHqTu! zBE+L$%tpBo!c2o9PA&3{kU>%~>|Guc&XL-pZDhD zoO@TbbN4QY)($G49eLT(U?JDJ$~_=V{^sUnsmcFv1a#+8P*Yo2T855}s;H4r!?Q`5 ztIZk7b;c~Zkp{Mu;ZYe1B#QcT3zNze|Ktm>J4TvwP1KWgZLQ}5Rk^g8c-1_usr9FX z3miTw>1lG_lZ?&biK1qo(91 zne~&@Y6Wg^Kw?G%=)JVsm*JuX`!4z+OM3+}+5n8&A~JV*e7P=5|BArSvv;JN+EtcA zL{U)aAy{IE7UvLUR)4Ln;ORpcTrVeZ>aEp~AG~pTf3!fNNPVQ({f;rbpDE2*IQUkk ztk<#Vl{Fu?{77)fPUU`ol2P9`Sz|IfAw!!=E6bO-C={Tsybh7L^oW_%_t2D9hHDopowp7zsQq}@64J@iiAK0$Leht zKZjMw@QN~#*cUj7B$w0z?dF=%??#Cj`kyWH!ru{?nPAlxZLD~<*lyGJX7VT3ok|{) zj9PWu(k|X7pL#7CJ@US6Sk>{4q1*4dh6dq29^zY_F&_Zc{%yq=%d;r8!ZSUvzL~ChEV%Itd`%!=8CFaM(Klvh+@`6bD zm1?d^^FYB(Cjkd>(f?uZEu-pMmbTFZ2n2`V9)i2O27V)EsN1Q6JXX>5?ke?ivl` z1cfOq)2_cX(t9o-j!+@(YxgQLYry}84XYWLd-1WFZ6WD!h|47t*0tN z?z85Y!rm>OBp=Q&^L@T5wY%FKq`bxC1lErtU1YY8l^=(Z*%5J%C@>Y^3yHi-taqsy z!#h+KAfJe~y+@Rde{_GI+hp|9M>{R=g%QT;i%0^$lpDdRM=s1A1nI~gO0{u#^roU~ z5U4Y!CcV3jFZQN{1QErIild!X;R+lGqlCUh0caTrRs7jal$Y7 zUqa|m&2?X6k1y_pf??(^0kK$zn9$SBuPV4+_9)|=UOy4PPF!$Gajpvo=jz@xnC~Y; z;BB-IG$IiSTJ<<8u6n_zstO*Mto1)hIKVWX?Oo1t97Oy$SZT>9Y-E`_T_1jmcm|#m z4gQhay9Iad!C!X^POCxv<%@c+o(y&;N-MhBi zmxR_c3&z2ln_mMBHZKV`E?fmoR@{RBIyLsdTk~i61w`Uz8{jKbTg(w55}MGjY1?FG zifC!V$dPpKk%nLn@HMd1qa^{b9_U=7=%;<>s<=EQ1M7EVa~J*(JPK}G7aJrif3b+7=! zFdjsl+<|pT(OdhiH`gO@v}kbjNl#{;mh_v$*50m;W0xGD6Ws6%$kucTdZ-G_{CQuh zLhlSl?a&3Lazn(BTZo9E4I7#AuLmti7eGep4^qs5D)om}W&w;1341z&Id{PxI`c+| zL<>-G%iJW@9zUFsJZr{qJyquw+)xSVU{|D|$q5HXrJ#}1KfT=lo(=2o?vAjwOv&-Fa+GyMKv$GKq&0zs4Tol+h7Hgo z_GRkQ7A9568R)>)ExBbN64DAd?fXP{VH#)>y>WY4VR)g0X*j;bBYUq5Z}1S7I0RP9M zR1RU%%6!&rpHq${wjpS;>HX-*_K@2ayGugCiFuAtit(^fu`km@-vG1|PTB+%Zui>~ zqq<{5Mzn36BW^)Hr;?p}%FOL?jPBvrzg&`O*kFO4Iew!$)STRJ{#YzDmxlCXqa!E! zntQibHTo=-y2`*1Sbr4thY$Y@w{nu?CKvb8Zq zNRvUD`(eDi(OvD4QsZ^EF-qWoVO}Bh^xeerDnD(sQV5tTI*jv01h}xUM9#f5m*2Fr z2pm(`mtW^^OyQ%Xh0c7$5vLYl!X!uQLq5{%D!h9;0cjp{-{@QaK0`1t{;z`zK)+kF z9fiyvi*xI>9!l&L=-93&EcOpD7YQ}?#ZlTy1X8QdSen*C!O>EVRZ##FXcao(-2xCK zosQ`NKKt+c9Q*Chcdv&g)s$ijF$N+H2Kbc=l>Lwh;!?W8BqpT7>}m?^Tnq{-1+JzA zt_4QTMx{q7XGh6Ugwurn_FwqEMo=ULR$-X8LW|X)E-&2|-rqDlpS9d)*tK8A>Q=yX zE&M=ddwcgZOImnkYH#-P4UndFxWIZj9B_(ps5CfUkO`$|up5`MEcCiGiF|#_P2+5g zAtE?#u)D5raREVJv}aNBxTJa9j^YtyTijWCJ~OcYSa8SFt(CScB_u?1Y}M>{nyM)M z{WHEl0$(d12H&5=XMD~2F&)dMd!rB&!35%!>ujw;Vc8h9^Q_=#;2kuPk)1(aM$klP zg!lv!-|{fx5tMCK)gC(og;Rcb-Y571UlDbsn2(u^=9{WbhJRgh6_(Q#KPbvuxqDmR z8Kuit9dds5=orKi00Y5E$000T@2j*N451VerofR$_(MV^U*KN}zF;LPkwM^ALPjKLViXFp)WRI>hg;uF0>&z!{*%f1L#YHuLW${!runJN0*-#mC=F_)lKlr~mr*F+O&%GH+p(hL5k@Dg+6@20fcM%3)UXK;xTf%P8CVi4Ts{AeXF zr@u-jn0DXnrA$oqhyJ!svFSUwOSYN+&8zXy(C7D;ntFd@Ay~<>U7^Ov6%l_;88s|s z-w+Lvhk2Iq4`M2_Vg4eh7k(}s{7j9({{h0Fwbs{Jh0OZ===&A@H_x&du(=yWL|}=% zW`z8IG56AMm2<9VRFCWPWq*xlR_rZk`kx1_lVJbh?(ftj>VwR1!gYh&`Uq@k^WP2n zTR@t4^qzzu>7YuNk@17TSPt@qd^syNlq6NoOKQ%KPG1xIS0rD`oKznWi*pcL} zkqvg9A>$>%1Hpw&<-})I|AP)->LIeOuy9|?HpBt8Eu`hWG;{A%tuluK)*d#p*0Arx zbzzjXd&wBQ;-W_mj8hvpd6n&Z0ALjVWF z;}pN!!?Q|Xkmo*2Q@dM&hg;NC$E}9@#+ZY+yBprLy5ZA+%gFdUINqY-W^^HZ%>wzO zr)%ZPAcW_k(l5I85xpQogi1HOmkkINpuIJbuXd5JU)acqtH%ZbMY!wb#()%vA= zn~0DRo?fn>d`;&kvQ^NYaG1M`9Q!`Q2Klky8-TT5C5cWR4@$p|kGNw@;YSq9bdJlh zl2_irqTJnlnHs&kwcZC?FiKDNvj-f%R5qvIuiP5;b5OuUQt1^{rq)8-Rb3yjnHs@g zKln+}GSznW@g3;6uL?G3JrkRb^K|h`;e1th;o!L*JH|LDuCaW6e%Rn?j=#B~ZhzY7 z;pE^Me#y|ljhPy(gF`7w(fm3TAMP!P6 zzI_+(vi5vY)Md-0{cd!CRWtm$J<2_aWB$(WJITCxM@FBY)YZiH_=Kt_R>#ezQ*A5K zKc@s|{*hnUqbFN$a%(d$l}vO;BULn+`#(d2M5|yWQKJmd_(eo^2F! z1tnSj7^6DTcYjSR9ggRs)$qH#Q{=OdtPZwii}pu znPTd`ZPXlDiR{>uyXWcM(cP&wl0@XbiCTP1faSccXP?;FA-KOL$@K7z@h>C+eSa}( zs5(8Ov3;*#`{yXV6+ymTBX8?I&+>2*%@qy@Ke5Wv8%V`cduqW_WigHUcA%HJ9bc5` znm_7e%`Q9G7Ai?MRHP^8{3OX7=`qs04)xMw35X8Sr+OPMgxQY3XHe zd?I>u_sd(7NGRVZx#SM)mS}bAGyEnd+x+AjGK3kkSjgHu(Nj>r%ITyF$`#>p!n4v7 zPf=-Len4B=ztPD;5)!RM2;D<@ZfUde09G4DB&I4?$OgQ_-b|@NFL;$Vf}llGwuaEw zavT`7#R8^W8LnmTpR-IbOG&ucS4c80KY|+txv?-J;AxUE3EMq!N!e6wIz@CWElJcb zV-%_465^Aj1T^te46Nc-`F%|xkdXlK50KAJ=NnxTPM2YL&hSer;g3HA@7=OOhi8JzqP9b&0B4_twDds0VExn*}l1_nQ;f60dG<9$s-4zZU-wx5R zZ$nOAcr?A$KS7LOrH_%wo8J=E!gI5JF;~9|rYTKnt3R*D*>I28eq}Byvt!?WSpHI* zx!lWZH0&};;C|)xBa}4Qu%=TnPCUkV(#wb6LVV%3rXa5)wRBql{D!P5alAM{mt*c@L zfBtg8W7O9^emMkzYe+qMP_C$)l$tZkKGLfTVU`#*p#&?=BwEu&*B=OBEY8`jl3-uE zwWQN+M?xxsvDliDT{q+-hxE_MycbKQpXg@+SuQ`QJcYV+@kK%EnSNitfT{qbh@g@Z znhFTMM6iMXnI?El=Y2`_`Q!kr-L}U|FU<>51-o_J(4)Xqg-%7YD`@Vm$IAQ4HwHZ; zw)y6W<+JwKy$ndf%C_iw0I`n~@rkF?veB`N&S|%#)8=?Q#Zr_* zT+KcQmWpjadoEyBt@VqGU4}*!S5L(KO1IlOnzQXX*sGG|-kgOQ>)m72b#FWBfMmuN zHjB+|C#M#-UDDnnz9fzQN_$S26*u#cd3qx0m9HL&;bOz7VGl{#y!F_v@&8xf<-Es-r!p>or#%Behra_FH1sOR5Y` z>A%&tztFTcNKYL8*<%7xNS0GtsDagA=$S**7tVe1TSWp871vx#|)i%8;@eGEI{vJV{1?$4GBQr^iQo&(|Ncv1?# zZ%t%bzd@a<>53kBhlc00`RSTFv?HR!lLx2~-DWyluM_IbqDMTL zWH?{JU|Dtqu^2(tZ*x5KYI#P;!T#4s3QL87g%$R~<#2OzefR7cB9ZH(RK_`y=y^Ww z+`Pv7`O+7i04{8UR|@|tg+F$Ke`;}`@&5-bHUEpm|38ojv{Bha6N)6Ir{^n{{zJ>{-L0e)Yi?<2 zilg^O*!srB#dULY^YuTIZ7-J6Kc>px^PR}-5u5|ZSM48z45gPZ$%fMQIQ<=(nlX}%&bQ9)ON=nL~ z`)q#<0$WV6|KiBjEHKb&HYpT42Zu-u&;?T_0r({XpEj@|j1ogN;Y1^eG$bSRQ;hkl zR8@7x2ID2wNh0RX^+G2>qyxia~0L!&=JrC^(8;iR!N@+UD=RMg<# z3lKU%(|XKV)A$-n71`3+KUo4Y(xtPX@v8{O?e_Nfi{;5FfgGAKWpZ_M3!$4bX9f~v zWNiGGqH~!d0t`j_jV~Clhyd+xBlJLix@1h7))&c>>#GQs&VC!WZYfstC!W*#-b64@Bx<}G@{dS6&2K)aRwMF370vd z`F#1Ze;e%!{w;vN@NEpV{05%`*;20>;o2sdvEPgS+x+@B__wBj2%sD}I6TZQEe%5e zv9q&#sRGjXyFT(i-KmB$IW?7F+%7OE2onFFeI`F~pAGmrvcPvOEj&6wYBZo;_)EBS zHir}ncumEBzZ)f(U~qGL3zWZ0h8-Iu(E2NI#YrL|H^nIMZ&H8^FQ*()27)><)*FFO z9cZWwNBkF^K#Dc7=md4c)g}L*?SuYL3BOZ0RR16n*h=*|$qddRIVmoI(%G&0pZ_TE z_rnIsOzu(Oc8Fv+aNJmEbzuOqGZ*>4rweK^63H}eU=Qy_SFzn0M07k|>WfnS?*Zf2 ziMXU>jnO#na7~_UG4N(spls9F9YT#VTmtd(|BcqadSVR&J$?78)B0!>|9_W)@c+5z z@Y~|g1Lcw)3vqw(5NX6pH=E~WPc*+)mtJlrJzK@OwU6w^hyt57UqV_ze9 zeqb8d?4n|aG?1KRCUc6XYp(Ad>3<$<6D=nsDJ|8k+-@FsHU>pgXgrscF>;7^kQ?(! zvqam{)Yq=wf46ktaO0L#mV>2hM}2Gph2ju8O{oqwnbpPjQ!~&GjWNq;tw}dBnLBL{ zA3;Yy(YH=DRVKkOeegRbz{u# z1WJj8kwsu_0;D5_wTEwjz5@MpXR6rlt`}@H5??X$?y^RAY-NT}V4l@)Reeo&>}3XH zY>ZNNDj^S}&2Kv#PG00;6NRCWjTsFCLqpPvgr=0vC2gAlR`Z+4Ao*%IC0-%PUmdsg zQXdHL{JE6f;_=sy@ACuiDTk3d;2;1gDSEl47@$AL8w!uxF}bYf+O^g*|J$kGa?@FZ z#Ep`3NR4MCjptSrs|)Z1w$wRRP8gmfw_RMv!-3GD`}_N#gcT+}VJr8J!x%OX;If!K zTDY`YIQR37qf%2=&@IIP20&Ib{t)S8$s>)4rNXc%HPPX2LNkM%p8x*pcBQpGD*CK5 zZ0p4)!_yYxdX~i##C5GPz|KcUJQ(xUW*<>h=VDZ&D7=kiBp;vIZTkz?IOzi(b*q*dTW#B&!N9zjgZ{aXD3R3ac1yNTw6-Yqm#m@ww?s!a+*v^4^eeC^# zEx%S~BsmAhrco@hu?q{kk7bWfzB`_<@-B9Ssx_>4I(x;>`{h)3PPUCtG&@rC0Oi@< zG55ka&QqIcKN11sDE|;dGL^zTwDvk~x22wLDg}>Imz1uS#qRtGtFk4qIwMYQ{A@>g z9Q+9a{NP|?${1kxvZ#@E){(4`n@*@sZil_tn7Zh5tjxAFFjA7KGk*4Zqs1_$&63!d z)7P~9{BY;vQlg>22htm3SSv8l301M|&ZPw7bkRdThP`v>M>KDx_D9g3=6D=%fXT&K z5!5dUy*UE4pDEQElc}(jrqll#Y&5ZeBVihcQ=}3wD7ha+WowKW84wKA)r+wlk3BrE z6oF@tIB)atD*#?0KD$e7_}2liuJ<<{uQUm-7n)z5S$#L|VQX)=HU8JCVbGMQWL&EX zK>K2OALeufxJbUJk`o$^iMGud#xy%5^fWA?0i5K@MS<8cUZmHxDi{ymBdV5oeV3TK3sW>!&7($lF9 z1M*l#B18ZVGS0xCu*!WE6Fx(ZMM&-8@p;Q2H7|NHUd_7{Zr@ zTWYKvvv)GXaY@9WAQA7qWVyN$F8g>`$@H-%Qw@~hwA7>kQyS2o)i?o3nYbU+!hC{6 zVx-sXcGpapDq)HgB%}0vU&yY~a>6M^_(T)(33(}PRSgJ5z9vKy<{^mj#Yp$ZpyU6p zO?rpT04#<#A>W_S2c0Yw_=Wn>zl z*TC>|L;`_mxwvTPFF*khPL$C|s})Vpnokakk*j=7_~fFApc3i_)teMWAO_KL z$VAc5Ky()P_@ng-%k2E&vuq9JZVQS_*f+iu!#p5T11b=iH9L}b`4tj+m1!SOVoz9K zut@|HYaP);lMqPB2>i4Iw@#Tyn&hzLVh}Z6FI{-pwZSm>vhnbfUS_?wQ zSRBqN@4%>kF3?)<0KBVxE$6sZ2q`&K6h(`U#P!#5ExqMX|NZti6Yjf#Oud2Y!Xb!6 zo1S!@0jsy8wz8aZbE>NAN}l14NBAh)`|~$5?pCV(H!08Xj;Ets?Bqd=c2kP;w~fk+ z>_)2s3bBW3Q4pM2N$SDFK@dVR;5v!YWR7^@`9U&LiUm?6Z*dqy*@bG}N_9Y6C`@kr zRCnL5kc?jxAO=2Xt~>HXahO|f4zM$x-#%ee(%FV(v7S|?@ku;7ILR+`i_Vo}sz5&^ zfaX1>gpeAa=HM1RHXn;oMg^p~XLyX`?ycy4L@eo{0Nzr-<#%i3Vt5XAtGH1uWT?_& zf^_YMYgCZ}G!si70{v*1lgkvGN2Jz7wQ9$bd)yXpS8P)9dXpwPwlFQ1+))772J;oK zX{Cw0g!WN2HrJ?%QTA8c2{(~+D{WCjO=PI(1wm=kg?k4o)iD|Ugh+*v(Ur@*{hWmb zdxs}l{9d+^uN;nc&N<*et`xr;JcwGhZ%P307RM(Um{Nu;aSbz$8_ct69b=3#)7Zsk(hhZ$Njw@>)Ke}PUB&^A9MZqRH@xw`p|>v7~Vl^yqZm{wsh zUlkdHR&Q$Lauhv~eA?=Z)u1jhaDtz);P6urMjBNl3jxbX zo7!leKh5EoseF$AkZMZ{v?k8@JFE49&5{}&V>(_sF695>BdL~$Z$?JeETbp5{K$RqRY z(0~~o0p%ki%6CMY@83Uv4*_o^rKS_*w%TGka@t0{#hgUw%OV94{h$J-uMA;$oP3tN z2K@4~5i2eIuxLAYprnElWP_!C z7S-UxAi|LQr`yS(;s#4ywU)4eOP#<%r3sXxLrkANOYmkjYlD4f=Y7vmNr?D*kdnrb z?G}DBDwf2-LZ_!n<4frnLnts$?as4R*}+o~f%Z*wZ5j1!d>z$6!yNcL>pyEaU^)N64+@Sl1*F6m zWai2Q?reF~yySBUU%h%CqDgK?N9s?0f?C?K!E8ub0nk3$st4Vhn_Gk((zdGM2?F@4 zAO2ku9zWWcku$!}Rw-L7oHA$VSHjmgIcCV(#20%Iz>vVas78v93SJjj)RmaPMMOp` zS9!+-`riKjMt>OsT(QOm2_f}}lMjNN7U z!DIf)3h2NEJk>#q)=J-hqp`kG@SNcMx~sL_3%>Ap?X?ZT^NLZqur$=4m9D4clOMtl z*e^3jUkP4@d7L!~gp;JjgWA3>qvfpjt+e==kQHz0n2{aL4|oK*DZW%~G5L z?#ym=1_#7~ry83L(KZ+x=xIT4bA-$4mLC{dXQ33%Y8x2(#$!j7#)L#MFgAo&dJw>B zLd>3k5CjQg+%HrY5)Ufp+}!`5o1II-z|qrp4++$!Ag5s>ap2~-EG3X!ZEA|kYnrsS`ke6J+FFE8xi(LW-h}n>>jH>bjns?aA?66)} zVOhl>?@lms4vzI{_queEMaClSw2fho6FxF#l_*(1xE>zX zV`CZ|bYkLQxf(kuCSze4Ty4NuWHTG~_lua}Gh3Dn6vzvYO3-rCb@Qnq$-s@sO|)Pv zeCalHl1bIvsJe2EnJjM08>M5ucR;EsyBh2wtGf_nW?_{k#}K@WqNl^^Tt?~vRA-6W zdItFF6qeT`Dt+@au*odtKP4zWv9t_QM$&V0NtECR$L|YO zCM4tF8O$Q<_GoiSmJeT@fW^jcm(>XwTOMmO@kEptCZ7Ea+W~xs1Lzl@9}#R-ORH;W z+uQT>IKnT#a8`&`pJaJmgPSX&JvOaUB5T-Nn)Vn?0ryC?UEM>yt$tSVZxSg`$f=9T zRD&B^CqHT!f=p6_$V`+PB+w4-lmFmgUqGmoQ>Mh`yxF? z&qFsh5sY$whe|(9aFRJRxx~TSX_E7cb9AtmwsuZ4qppE=*e8T_>vuCY(JY!vZi<%iKi_mz*3aIwx4Gn1~|FeNT~C(xIeTZ_h%VZdJ% ze`p~aR9T!Hojsr)jK~Od^;U#g8sC|G{(F4Wq(uS_bQj-hoiW+YJ-nnwD&JGJUar_HUN+IlNV(Z1SyQ9DUaS_s z)L##<9bmgdqosMFdT_(Gy=jPZUdTAEbObfRTlSsX%0l)U!?f@MUC7<{;u#wlH*1O} zPBu>7n|(4#WDl4)l4r5|zCGkv^roy2?W<|R`FxPy!d6Ub4WseeFJyeb?YLsuH}5l< zELOPEM-nh4W`Rwon$dD30u^IBT1AbtFyt%8V#{6(T|egCw84)ko)jvI3p+<K({2$EQ~Z7pqFx z(uq_eAWI=4+JKJ5p=y&PKk@1eQcfxuAMhhyv^d<<1_=`E(v=U{)DSrmb}b1lVUA3n z;$YUGCN!P6Ba_$S({(TMP&~ONfl!5hcK&48`oY$(V2q9dpAvEe?+3m#-E8mC!%o9l z+GH-UR!Lc;*t{$7O~D9LHYpU^ma@1)MdcCwOHks}pyoYPUCnq|9b9DS?0#a^6g_2V z$GJU{K~k!^Mn+wh!-G56!7gkF(c*IH+Q3X?lbkltQMMQ!Ps7+h&qU3aAYyv2ZFBB#hb<*cE(tg~W z4H#PtVfGN1lZr@`?%_j5BA7`HQG*+gS6K|YqJ$`?+rrpK109tak1SBR# zg^z85**^!WQdCCwAl;~b9l5d60g-8=qnNc-SHs@H+peCSqO9L%J-NKZZ%H19K9JAF-oz3}>V+c3T-Ts7oG71!$65EB#?b&t39 z(Zs@vu*kRlS7JV=KY^m@E{F>Zr6NRqz5%M3z>H}?iA+xwE97&}1E;1f69qF;90kg7 zJbCD-1R-|!F{HFSj&dA`S6rWnnI33$Fwjexa!4N6j4@PZC>BD#5H`QA*Y=x0Elv0z zWeUV+;}Zk^Z>@@1phs@L&&2BeL#6A;GihQ?FFaTOc*P@XV%zQ4%-1s|XHc*3h^)rc zpHkbjv!~e{d;9%O0;WuUxyWS|U)N>`pS|oDfZI%f6PIAMbC2+9;V(ufx2^jcmHT#D z&qkVVw+Yx#CEeAd{0O}O> z7855Ix({uNM6v38G64P3tSfb7_3wC|`(W2{$8`;J!o<*EO9V;E+ak&0k)rv4s~1L- z_;va!4nUlp_HHsVV{B;7MawYC@FIwz6M9&Qt-om>Ruu?S_?+WZP{G1=XF6vfhE-nl zS%qRGN(xC}Dp-?x%A!4HEX9wB!W=|=?R0%Awvielu_4r~p$grSrRuo(q|7wpjNzZ7 z`pP4yHqTeysky`YVJ_seP{mCm^qfPTdsTATqA1eHuo7}Ep*gyAX2wuiq@Y@kv^b`M1Z9Og=hO-uMh6xg~k zb6f0^xwhpv-N#9pW<@pRAkuLxBVpM~i&4HDf>bE7G|eh>hW1$EV`S<&QVCj934LOa z%mfV}Ny5?ejRc9E{zS_GxuB4dwQ=#u=7SmJ8|kQicbScU33*YxinJ;KZ(l>Y+!FGj z;?U3%^qFDy*?P)++xUkSMv>nFTc+A!K&g4ej6`KA=|&7)jO5V{GLPw*+qPSvB5$xw zCMOqCxfDn@=Ip|D)Kh{9Jwz5=>_oZLQFQjzB|d4Wrjg5$-?>ksa+J@|o|1>@;hlE@ zhqlnb>Aer9yBg$zIV3-`1_kcHIU`465KA$TSe|zou-N!(>hJipB%`PIo-h9BH7xwq z-oN!gd1#-qV;ZR5;LA}NTVg0v26sYD0Tm_3cw))pn3d)&iZ4{4nq#Oubcjjrm6T(; zZ~}*|2rE5N=1^y4nbo!bLm?Eqpa|12o?woijl7=zFki)XgtO1QQF?@5Z5*8bxsdf? zVig91*xg|i7?*mhPtZwu26(QX~P1sS?4P;qHFMF#neRiLMglyrtvagHpGd?sN( zS-6;UXvaDW6GhY;<*%g^NEfs|A8-w&DIWOU{ejz&I*ILYmt><+jW!vy&Vnl)-pbtI zt<|o{Zk(`k5iBseP<<0J@O$;IF&S{CHeXFp#w^ubY3gl<78&G#LUg7*+&8lKV`!eQ zSjM@vPTrA;>0>InNbd8rojbZ0C-9fsa_d8S9OqlvTI=Jd?@uW}YDN9C4Z0mbL9 zTIw#{tV*7{7vRJQZKN1(T4de0Cn3!M<@Vm);;FR_NObp{uf>plj?tC$FvRzs@Bk zO$pVc^3(YMC5>PI>&MUu!I$HmUSY0~MtB&kKo34oa7Y^%Snc-!_bNp1{mutvBgXg(w%;zD4lvtbZnVBO-U!*?Chz7V^OvToiK0P>^3Ik zU3a6c^a=RY$RuqE1_qlSTA&OXEY7-&Hi-=6#q?#xb{rU7-IIinUJ`s{09CEO#3w4U zvvQSSc{i&>$pko~bCv0MD`n_D1Q3W)ToaNzmML@=Wh;F|5|TG%_X_W>rK7dS;olEV zv6P;6$S77EqOU&2FSs@Y;f80n3QijBf9a<05c()UH!%W^?%ll&Q_>t+QesT(8FdtH z^Fjfluee31NMBZf1arC;(e6M|u0sMvgAW|PPX*FNqA3-vmnAs+ifZUtx+|g?u_PCU z?PL`wGl(NOED1al`==EcAn(R_+jIOrl5$c@xm$jUCsB4!H@B>$x7Utr3iy1M@KtJJ zvJ_jPLI$)9l|pneOLfwvSyVzuMnOK_DEq=a3B)4pW*EJ=_uJ47W^z2y-3fV>$ucVD zi((`!RA@#Wf8@Qt@0X^WwVQxnfDD)YFq zuAVv5Hp~d)!()XFg$oTA+X_1`2jCGbF*bG`EC`WoVh$CiM)XdG6I>SONVTOI?Cp8` zcv;8~*vI+4%P=C3em|Z=I)rTo&d7hv9j!(3z{G}t!B5PN)XKMQv?iVLZwh{e@`{Ec zg;X<-X-SMoKEt1(hvEKp)|Bf}Y9z&q3&^nubT6n*EBV{;5lgtW_Zu{RpOP`~?8I@3Zpmh$Q*WG4hICz|dHP zUNig_Oakor+hNTUVgfN@kTbkkeyI&rE>)Qo!DyfS){VONt#xI9(OC59ocT-)VpiP( zMf5{o*g^vs#fIoNBXOB67Q3~*Xmp}*CS0gf`XXD}<6%$f$cIFT(G-jIx3Cu)gYsj7 z0vjx!AXug`YGeWXWXLW<3ZkFBh+&xD;+QYh07EbWgEan*??O^GLNtV9mH!<6Ih5ku3wL7*;rJq(fMbLQQv6`qQqdi{*5Y8S07ORXBB|8|0){( z7mIwNGCB@AffD6iNP{V840||+LCOMBbin374xG89W|C@P z@uo*OHS*`V#>r_GVJ#)}+_*w{QrnY~p8#a(QAI|%jKu&H^K9k@^VRM>iPQsQTYias zTBC(9)xnzfhCn#<_%D~29xzh*MKhiH)!T3l6VtQ))$9Yk>M=E4Vcm##I^_Ezm1WX& z&S{d=T;mgGuarA6;b+S|@-@L7?GX{6ulDr;Syhtlb-g~T&noJ2MP_GtUfMq+1zF2e(3@EK-R@{n#=rgDgCA;>fGJa!PBDcRr9-kALvKPtI)|$JWI&NG{_hDjI z9Y8s*G#IZ5BW}8c9_Luv0S|Z7f>uwE9OaV|V0R{*C$1rsgyfRz5JSDvlm^^^V6{WR|e&R(Vta=WqJ!)d9MTO5}A0n>DhK}xclE-+k!<4YMyAORK7;vYmhV|5;{1nmQBreQ=M>g~#eD{+ZH94jV=FHH2yCQ;csMQz(vLhn&eVCRoFpT0Ug4}Ff z2W5O7zcTtKz9I3$%S!&RP9347p~`hrGj33Murp49O>N@SwGEHv`@)rp-xQN0585-~ zVoA&i+r~IfO~*hdS;ZpCUEgcZIbu5#rJ!X*C)2l%-g&%5q$Dg+rS1p}JtjoD!M|G+ zx%YuX5Q(re{?O%%{E zyweLZ{fwJ~Toj0lVwUyT+XMfAD(>Ys2f4m8`ST7(8W2K6@MFVAbSjDG0?avyY)g%kzg7a zGRZboOymw1U*G$!nId@o^!s*xf%dK7;gNW)nZ7T?nFc=cmCh_Q;TK4#(8RSjW=_-y zRxj_I&07|ka}}MF>L6vSvH{tlpTc=EJy?ehzvu*xlh=9rK;h#)yCEwg*$g*)mVjm6 zi^0xTSz%;qtZsGVfe_3!x0?BozVd=_KR@LRFo?M{D0A?r6L^u2H%w3zbg>o+Veg1= zcTJX;%+2pW$s`-NHmm`vv|F*ym&&;ZeT;f?Vr9g~7s{|Pm#m6fT$YuodID69gKHXW z;wc~p@=$E_D(Tj0{@w#3{f4?W27#c@6s$8KArJJ&DApL@DU}hZ>zr~Bj?q#9bSSZu z6}E$hi$JLplI8eIg@ag^eeC-FI zz!z12xl}TX@e1qA**U!PPR!-~xVa0tz{w8U)f7_gdNFM+al0b(70P2fwP`Cc(rkTOm75}hUfkZwWmb5E_Lq^@OC_r0eD?0c$*Pe zC+n}fJ7pt0!_VUj!A*+#HgpnoR#>1^o}AUhOD8sGt6F>YXzl58>^%S;S7;Vs(00?2 zL-(W55!Uv;uxm2kIibnNfZKeSEw6^PhF1Uz5Qe@({I%=`f#;-3M3ex(jH#9 zDP2R%nZ>fpwm&n|Ev);Dl-Zg~Pe@Oe8c=qjF7;R<2M{2akRw13GOlKXCdU$-?Z>3O- zVuy^(u_^pH2ok1^k6Wn=32NJJN%|JEPxm9XPR@6Z5|CM0pBkNl?EH|b0m=HGBx{hAb3&}VD+heUP?);*7bx8SebYL2mWi&s8c7;p< zCzi8p$VszcM9eGJ>fyga9^jtgUK~m_G2CAikuoYac%Id)*?9u@1?m^w z$JrNlf!ajj!JttX*LuEc^**#ngd6`$cI2x??7mRGU)k}&M@DNs-~*9$Z7K3&9F9DK|w9UJy)a`s1axY%|#*d{1x{i+_Pp3Qc3_Dju$YF_h?Gp9^>7??5Mk7h9_(!dVw)s%ZUg1?j z+rhRz;-liM*j>Ahx|7g5&@QHic;)G*tfnfqk3d!KQ);8n+M6aHTzq1hjx-qxse((I zj}AIIxUnG#Y`TZ}Nw|&z6&OEm2@q|3KDG)3Q!4^?ytxI=MbMu<59BD{avmSEk%y)i zoJH6(g3-p?m0>Y|+oY8&4P8AiNKwf*zujKlU;O_43 z9w0z)cXwR~Zb27!4bB3KyX$}Ry?*Pyd0zag9;&*ghMkt_={|kV=hz!!Q@{pIl4gK5 zlYFjr4~C_IkvzX|Xyhk~Z<=Fq!`rl~DBeIcE>~`Ft$CA(u*@u=S(|j_FdtY?lC1Fo z8_DpiGRiPO^y!@ZSh7wfF4lS)$b6SfXKK#rf(4Abb z*#|!R8zo*B|0=}{^n&CkEXi#qX8QhJU-Jg0Yr0_Cf*1`UQ7YJ&Z4hd3?}rrHBkboV z#FZY27ZLt4T6hdr z!1(9Gf60k|-PtGn@6P^+kpHUu_lm@Y|6o!7&Z+)&S*Q62RP-M;BuoECckg+m#Q%Os z{rja$k?wzV_crS5|Cb^EnV&B9ipoldSrs*mXb?u5IUgS%&&%V5L$mgOjRopzi3B42 zOKXPWb#uIZp!dQzJw5$amm?@B=ue!{O0CBKdRHD1MeHc?KXbfJ&<_GxSX)yNqmjE$ z@?^?E&FsVa4F9`}ua&C6r6XeLzl}@Vc~aba9hU6BC-pWvS+1COuH?UJ|26&phm-aH z>+k!Qu|@#Yqaj%2wa&Z;7u4ybZm~wQ^3h_bR}!Srcxz$(-H`nZuQWwgX|B!vkj@WN zv^ZweAjM|E#uQr@KR?6B;6oTKe&Lu6l-n8hP>nzmY$Q?Hh@POWq@>j(cdz7I>_w5r zVYWf_<;W7P6!WbAM!hRsZk(E?@eAYV{pQT=#q0o8QVR3pt|dwyUEBLv@9^e3xLV4r z#8I+*PTXGwbO1lT=51bFDY@T7LFRx=Ia`mU)NR1^CoEY-JVJRj!@UWL^0dtReky?K ze{K)vkE46e&*ac$#JmKDr~OG(6qY%}vH3^UfqJECmNe8i$kW7piz;HUe)!GOGA6h>eZV%lWhXS^NeMBkYpJdD`7@{KnXcce2KdDPIrn zGxqEurrU})>=!54Q4DS*W*JIfe>S%lkSC#m;wDVkE5_87O%5O6We?&Z*}Jy#~C>e|P61v>k=u3N)K12b}(bl~L{{ zo?i+XtvG#5T)3%YK9z4w_IN7wQTJ%U^zyrh6TFvSGcd1@I$}QPs&r5`CCm1LM%C#e zpLVtu2v+U2Sl1gOSKK=(4xFs|A3wigqte2`(?Xl*%2sKX&S+u4~9zOFX*yi=U7=D_` zz9t=GvxTIEk{R43u5fsb5-T#++>DW(1?eqIw_)#Wn{~oLX3%NTpGoktri4vQ0+*`W z)3nG^KUJ>{9mv@l%m;X^K1NBuCBA3waFqjexH~C*YNQ^A_VT;>R1BS+%*3NVVeV$E!9s(Nm9(;VScHhuj z;Sjs%WOxmlWXvUj@0tA)my(NEUi`8Rui8(*nZ*t9gdQeCRXwcbHk z)CE~ONt3%iZXr5?Oq^uyRD}u?jQ%&C^nkvG5Z_Td57t! z4~`)b$MIuZI*^QTK40>eGcZ4X)iMaDuFiWmn>oLoWynUTGj@q=QL?3Avy5n6=cGL$ zet+umv{mh*e2A-B>aXPAfO9lOCNpHN*PJra% zP~hpmYo8N>GI~wygGB2 zo=^!7Z!j$$jRkiUqq z^NmCxHam^I$>=s{2vM8ZUK#Izts`Fh`8pj&IcdRR9)&v78|eal)LyJ|;8Z{eOx<{_ zPuC$ai@VdyzS_T}+nE_7FzokyA|JOem_BgnvkY`YQyV^O1M5JXMBLuitLl5+=sa%j zEO}Ue49?}-^W9$5c0Mk*xYQ`w!8~kxf@u3(1mTb2>Rz-kjy!Ifa%rEPWGt2`X=-g{ z9aQZ=ZqSw3=2`9cd9^S5u@D!^5AP?1E{jE2syvz*SE}$Z0{b=;RQIaJ*RZdpVx`J?%5;iM$)&TF03p6|@@NN8Z4kRI>*w*E z$^}!>?a5RQd78S@^p|yMp=D8s)!MM+>V>EKEuZgWI?h~#N=eb(tXs!`?6lzOB{$i2 zwJy$9l~vV7#~|mBN{}V%U{CZiNXng`>@hup=Y#6T7@2o``LV{vh_uik=w*j-hWRCY z+-F;2vyhHnwDGjC`Wbt8uJIxkUHg;txnLDE*5a%+Vt&II`TQf65GnFmk+e|1nc$hP zE`i1tg5DihD?5f<^A;cqy_e@A?&dt?Xw(>FyG{QB;iD5UyKu|CM_By{9Enu(iBBiX zEVex5aMg7kE6k{A&SYnPeh0CRe7EL@u-1C*%8)(|9+LM9?-Zf9vg!M`B#9{ewznAbWXkRb zCfV!x$ys6N3(>dz%r}^aKD^cZPduKDow4YTM~fVU$k6Cle#u&p746$x`nwr~mvEaM z*vA7rzmXrb{Y0zoR3OQqj5}_!jf`nNg}U;J3LI9;1Hfav-T5>t{Yld!+3kW}cBIA0 z&D{_)`lcm+Y~9#`M8V5!+l!B%xF5c%(icY@*!+~OvGUG=mv*Osc;?%;WUYtVA{X4} z+hl?R%ef@p;52@F8-ZxQ+ul&#az>Df-oxvBE1ay^o<@fU#{^`-TgBB{*^QR*7pxb7 z(uvij?-t`Lhf+|D`asRayxS(pvP^pXX6q-9clG_C_O0c=`0Q>y=xRPhs$l_NC^&4Y2_b zfv2iEx;LP`;DRp{XD4Ja_GG;`;1!bm07*-%-`yPaM7XyVJlm6Z7%XL6ewjY82g52U zPYlYTJFSiKm$-HYMEl%+HeIS`jH3GiPzfF0;bC!gOPvm_+5JNf3e;$do58_2T+&Z=XW~u ztSCW{!j`ds6)&1Ovhk#p$ehmf$k_Czi(JH2j(5VNqmCD{HZubV!d^u%RuJ2jqb!RJ=#>OQ2YP2;fx`Fxpgq6-!t@7<=c zT=Um=F6Xnl$NcW#Zb$$Gp!OTZeP7rUza^{FG#oCsdv($^kT9v)?i+;F5rlDX%B8eJ z;<iJlIpa-&C8L&jcH*Vjqea~Q??o} zVfMsG%gY1Cu5tI5S9-nCTq_?~_}hCwl2&BBvSGY(-*f`dl93sJI$Y#hZ|mB&ze%7S zSfeWo3`8~(Z+ed)u(wo03V`8mXu7! zf@8-Cx6m0hH))iP>@ALY36n`)UG&-B8;|0*Gp`t1g?@>mAAZbBzhyVdco}-~Uvb=ThMlM#hp1%eY(JK2yFawpm*{Ksbc!x1lwIRBALbw0MT zbfsPd&Dib!b= zVVHkajSMPRjY)ebY1Kpz$4Lu1e`D|EXYQaUpNz%M{q zNg_RQL|-ppag=BPSEk!Gh`YdC%!<3lDp(w9)fzme>nTsNy(ry7p;~yB#Odf|#hU%} z=m)t3Z&xhWL?9e{2^|dis4Ho+c6BiOCKZarka^^!ebyCvxav5V@{}#1FGfdS!J5=tgy15)T#v!YJR55V40_bb3+C``u^X zMe6y6hqrJpH$z@uA69gYrUSo>$zrk(Y1DljaOv2WT~BAf z)xlE&f!E1XNIok)H`f<&8fMuxkP@SfPpt%>l$)t`i!uF^3qAiZqvIKG3j%zT^Eu6U zR)^SH682&t0NFD|fPR=oxoU#e!rH{<{oAcn(hRCaw~X;eU$T(0ni1CNkp)VWF)9t% zl0uBf|6GcOe=Wt8U%=FT{;uF-#2ty}Z`t0;8)-0$i}-qezEHakt|*%eOQfi7m83*i zz;cd%o_}|`y}jy6U-9#BwZqcTyXvjp(Az@~B4SFs zIEj9j9JKjc@VBJ})j=i%wpAVSXVz_8DC&i4U5l2lZiMef* z?FignmQS2Oo#{Kjq}R6ApPwGj8asxH^q%w{`{Ao5N;*#BA2)~CA7`;%dP|G+j6g%L zkg9W9N>=WSygip-Q8EByt9V)4VJ}Z#p3lz6AWkpf54wYz8C%WF&p0pk?$~}jQg{1M z6=uDC%?nCVpX{#kiTgKsyQg;o)Vf>2PY8<-sV|{^%QuS8n~g&K>A=@Z8?%nA_tdzh zB^Fyo+4n+^r^;l^f)_jp@{h=!SB1KOtnQPA@a#YMD;66Ov z^<5*ses(<#sD1dj9th4|dg(S;9x$-S>=RZ^Yzg5OoCq00)pUky)%1D1Dvj{2sY(cw z3CtX&w`bJdB1~5v$MW8H6OtTJkuotsEGL|=YjF7X!v@Two$udm#oG&ycCje$7lMq; zH#y`*(2RNgX5#!3%{P%HLXO$VJ;;h_^=s+)ZDtS(m#>7ARnQu z@l#VvtiCIAGW=A9En?0Bl@qxS%J><)a$w(ff)(==D@RXM%eyofSI2M3QOoh`*gL($p zrpygMmT1o5Ce$~emF$*f{OQgQY&Uijx1{hop{}q!HYv%{mGquI%lrC`jnJK7AXDN& zg>~Gj6CRCnmBG=q*WLD!8@un8llOV_7<-D(cOqg#a)`%{bsE|Itk70$B|UW63p^4$ zZ_uWvJJhoBjb(z$R(nG-$@E8~5I$!;e-wJ61`&Ms`JvK=k!b#fvTw&pn@}29$%n$z zJLVU(cBdUE|2X8c6_2F#d;V1P<(b?4l-$SrnfsXt#7FI3z&T|{6O;-`cOnk$%9m%| zT^#g#s>?Q!4I?shNMJ_K?0)0jWxl+IMaV(Ov^2QQM0vAirB;;Xo?AE9mps)5*&vtcnLTqK+IR) z+d|eEfja8quB^NBa{6$4enPr+d-}Be5}@_PV>iFcrxCs?^LpschBJ7P=;wD*?VQ1@8>1WTU9wy8-1mQxmT z=VV>I`1#<0toLC_Z?jHM;ckG$>WdEFul#Yhsrysvg)yrVK1;s7!iW{?3Tw%E98<}A z-uX${{_L$w4&w&=QDv)@{L7RB_;>Eh^ z^C@!l`P8wiZwcj)OL)A|?u;NQ5$_PX-`4AWfiH3$?xgh|$o00ttkFX=_SBg{Hp;pA zY(0HWkOgch)@i>uANkHtOyis_0k54i1wmE2NY>AZaYmQ7Sl&ik9Ar-m52b8z89vt( z5=UJ7`!?AHZ2Z6Ud<>W& z_d?z6T9%(L+j|G$&|A}oJ;?XB$-Q?X5&=KklfvSoFDe5PA0Nkmt{Z8@%#B*z7I!X4 zPQnUxm3@ys$p|bXC>eVL8PCX9tTe2J@W>1JyzPtf&x;}D?Oi+c3A~f^O{m8qBsYTw zZ26hS$!V~=K`8KfDGR~lC14Hgq$dCsGJ<8>zUJ5p@UKLlMXzOSjYU6GQIdXk$_!u1 zkHzO5+>4l7@?gbpv7*&TMr*7rSqEzguKPTHQ=MTvixe z^8+J$i~6Og-e8XlUgKi{QF4?f8{&0lAMQZ%erbo%zA=17yyv}YJln0w^^LUu61oCD z*&j?j`o5e%7q_hq*~=}cw5n?7C}tc2&E?y;G&;J_FnW2(aps;OLnX?WNtGxSD^>Wp z@GBHQg@#`m+;*(cZ71wp8y_9XeCG3#l)j54dw9<&oT%m~Uj9YLJBgq@H4_~$&_99A z?0u0AaC6Mq=RYoDTMe)p)v>nG?kbzBFUg16B*lC2a^o4!k5NtKdo-*xXX@``LQIzr ze*+^8(P=`m6B%ukOD_%|cGNtc)v5QUNFF!+9$7VqVQz<2>nsw}jiUV`+e2F%su#Q9 z>$CT!mONgL(ibPzAtN2!tRBz*umz~T*m{otVfu0TVj{Fe&ecHK#`I_i8kC%c`DkhQ zryC`I!^&9E4rYkF;4qfFfVA7la36-<{QZcO+ImW;6`T zmF{~S#qLC$ChD>COOMdXMozF&lhkDUOqFIB*V^bRhzuN@PonPPb{>{}_l~~D*IO_s zwdx7rbo;UV}7Sf&Y;PPAyKYzl`ZM=&zqqd1Z<}WA`Ga+8|4VCr)sM4_(TPc7f%)C z;R^FDv6fybc0NbtheF(4;jMn*dYY_A_Lsol+$I77

+wr*md|FP}Fm8+0m-yL|QmA+18)RY^$B!Q3Who6asM z>sFR#qO#a}4mjl^xRTj-RPtm1>F%WgQ6Wd}=7@YT?#}4Ubneo)R zZ`5!INr4pN8Y;CcENnm2ER0UeA8+e}-V`1jZ+6s^78PqR1I_5mRk@WcBTsLQ*0#1j zg)nL@;S;9mKRuA;*5`e%Pv_2w$~HQ|THE}!D15ntem%h-}QEhD1~6eu|XfE?T@K@Lr04#c{H z&|7bwyI?A7O$vJeN41O^C>^I^!okZUrJy^bLD;~KK1nK#nv3U6&b1$|}MX%G*ryHB= z0+U(E2zalb2BeARGT|J^v=e{ z25NA#%ESjfa!>7YEC|KL#oq)8+d`G^C_U>o8(aA98xh_D%~{#*?%au0*pRFz!epHT zj}k5zQ#|>{tCwzfXaO0eUc=KZ zrNiv1owdd!a`$nT_;}}=8>hp;_63|9U=j2^Oypbxz-#vaGwt-dTLd@E0Xh41VDV)7 zDro=JzjFg^p?gqz+SM6bbpZWZH<7p|kmwhHlHZHp-67c#Ik-tLcEJ&qy^Or+p{!{q*2 zp&d6+RheLRv`FqRs7->Akx?5C^sb?S3qXo)hxea)%)d_z_y5#fHwphwH9zYARP)>Z zZ_9+B|7n>(^glJ%%Q^q0t^e!I*5UuphCK1_mI>>lyl4&JQ%z`24jQDjtRS#|9?j!> z2Vx@71}Y6rSztH+IM7Bx89P`jBglxKr!6EehYo2C5#;a+PJBaWtjmFz7z)5*D~oSr zLiCq%N|2DClWT7^{U3eEQZgNbt%{2#T%ek4I!Px*&Bi2>bj>hN60L5F39k z>RulUb&X)|(lK9PSL*vbn5wFMn^0Lz$kZxm%!j2TG%sV4*w$fS@7lmao9xsFJ6UO) zrNaIA%z41wFs(o!q2eCO(_7r)vA4jZ<$99W()&lVvOM&MasKDJY@#Js5K(es7Z?21 zFj#21d%zD0)2BSq(;FG>E$UB|<#~MXh*NE7_CN;+8DgW)Hgv8DT6Ho*3~`-SC;cRR z%0+9nEM-(W#73QMU|5h`2Vqil22n}D^4U>kkFxM`$tmtx+Tv{dcD2<~>SaR+)wT=| zSS6LqJS@5ls1(!r3!%8J`m=lgIVO9h7HTmS9jtncDqy&JTBd=uL6&ygCp9&)s@Zf6 ztz&Q+Hi_hT{0z(%6-Yy>doqaH!a_sd2H35B!+TGKnX>zBNVV2LM}Rb#9!bVIv(St! zV2{TA<7|c2nn-US;};h=8d~}u$k*ngA316!2Agk?Q^grz%y%%sLCL#BTE9fdwVT6} zR5;%()9GVo!g5v-3>}k^G*%)884_0nd%p7(Du!O|1VGr5NUM|?J&)FVt;ri>XlU47Lj#eC z9ke|71=l))hER^HzpzTKe}|nWB>6ni%Dj?MSm`x2$XJhCAlZzThj7)LlZcB0b4G=x z03RC3(vL2rvgvJ7r7`%oCZKFk*$=PX0T1VzgPb;Vzy832&Xsl1E9rTVqk&(>Br!XA z5pDHqW^M{qC1k0fOU~4T<&`}E> z&NnbK-P=p;-66DcO<6sIYwJ@H6ZLgY&JGX%3R8t5AejX*KSA+q$~dnXJN!kf3s@GP z`Wfa%27$5$mO0$m5VB(Wn%tt$gWPz(_aVj{A?vG70BXw&EN!AH5lozUWg@QrjwAA3}zg0 zxIY!XH$_^OLQ2IhRSs9C6ZiH_i|yhK#KCcIC-oLg2L>46ZfYd)reY_DB52G6ib)Tj zwxD#2n-Gq-~g195`b@fR`70Eg=48F`20g(G!)hIC+!J2jb+4r>~&j~Jk39`#*{(W14g zmc>o5AFPX7qEDAMblDn44Xu+VtCTJaQ1U3nzX{miUDgrr zW{4nSV@)buW*mI9a8U7Vh!6#}s!xd|B;&=VA8rl|MvN3Q{(-zzbIAYg5bKiN-ZaWs z5DF@Si3&>Pc1Yx`VXeq@&r?oLX0x~OPwlp(`vJh}0rPYbs#en((PHH?Zz!xaVH#D$ zX8|eb8T+FWaE|l` zEiG}k7i{DRR*9DW?_4K_W?CeqsDx?>_UpfBnY499)6?l;Di(p6*XQrBO?l6{d8*DM zmtD-*b>SS9AnuoUzE$C)n8ODg9M$=SeBV|zgHDvmLgM?nK5fjgc$zXR;WhQMv0l9! zC5kxPUr*pv=w_t?JE$fk#D5uMPoL3P#LewKA_usq=Kc1xj6i9*(fg=2GtkhWa*J9* zi~cKYTKh1QT04^>$!T(7PR7WFV1Edy#Zv-!jx`=;`zpc5%>A<^0hnp1A(u9RCH@B+#o%9Q@{zg7pV03{c^BIETfXwcVa9 zi+wR6cs@q{A!AxfqoeQ6Ctl#o{{b$kY3yT?RP5k($|2uSTVBlwkbTQBM$W`sQ}H#( zlh>U__6yDDf{Hj7tMW6V<+uv#4B#+VSv++!i)7w?DuyGK)1;S*qF5J`w+-JQ~}`^0q#VJYl@#;+C)ot^1NB)JFh)nkC@)i37=|GekWZe1OsT6`RL{j z^Xf%RQjK;}Gw}^~H!U7F_{|y76m=~p_zB4Mw8R{**`!y$4tWdKoNJqDNRKcu)E1yP z-Y@VV4a?wSX)w4D<~PT&`9&d{$+7V3im)h)sr2v-bu)Ej*OppHNMbGoi|?53)WE86 zG#(Y?RuT8G4l<0?{Y)sl7;*4auBv52JU5LRR_ZIb5KNwk!fcS`(9 zG#{0p`wK!z?;ptSU}kd#?m6XnXcZ2H%J7n0*g)R{u8%E4!`P_RLjL14o1}g?SfW0c(bsGo9KQzC$kpB@{zxRAU>MDuZ#$@3rATL%mPtx@DvxC$LoR3yNBaC+ zQF*y#gh($^(-X=#AT|$@;cj4;ed+RgpN)NFYXYO&VXo~yn~zV>WfU4a2iwr^NG*%f zh&)HDQ@nWvt$YU^@rAz4!`C5HN#>O_xg`}aPl=tOQOxvq{s#*U?bprTJKO-m0Sdo3tzY(G#c&-Pbcvk@;*@UL{S zOETA`FzYv=)FAP|Vae~&{hpr(yPdI#tEg)?jd1WWsce`;1s$m`M4_f3t0=ms=H|mR zn4Iwyg$0FIm22ncl-fFHmbX!&E<4Dl>`f{%p`x*}afq$ylkmGXD3P4~;jaH{x3!5w z@%&dvH2#D+g{5)NPC6OPO)S!cB^TyxX;Ew7K9k7UsUVI7ca5DbrICIoe{sIl{xj_9 zhW50EqnK2_J|pHIY@GGs=l5?L3B{)!ya4Vxbh;;+$BPyh3r-MThu@<%>FE>Qiq1~N zTC+5CVR@Sh159zBO<4g3;ZIb^d*bJ~yJ?~Z4cQR%OzN5jRBq{lDOwp&bd`UiCPn5T zxH=3*sV#gybiNWZT~2@2ec}~AzqmC9-D@Y>FLcEUEuQ3@vP8g(2;#K|RCo}zx*`c> zg_D&dF);#mFun885IyY??_l9@djVb$N21!M%7OXZ~Y@iFo zKr$R=}DrA(A&7vTDa14c?0Vd^tq!N;|(HOk=wf$2pHJdEvkS?Y;p z(l%f86F3QRFlhi1WeBhCk%gJvaiZV zc5>nL6dY5Shby|GI$qV4)c{63Bd;U?k+SOsCJKj}K3bVq(03D(?QHhh&g|ecAzAc$ z-m}2sRM1=id4|h_%QbXng#Au50{bnyrP4aaUVl}k9X9MSt_p_6) z(z;b*r!YnK-xfz@c3ZBE+}kX%MrUqqO)5H%_Rr;4thJ=>=drjhN$f%+&1|g1MfQqteQ%#S9@EQ5&7gu zbJdg-U@<-(1BLMi#75`PDX1>n(a*E+i-Ku>5;A2i;N@u?NysA>)#3B%V25ZEc{P( zj6d@<8|*A8FrBB{9O!pa*)sraQQ<+V%Alxg38`+v18$qG?nSgHQbLh>o1X(n4Yqpn zspui@j6$;MjkV6#40ur_5{e|w@^7-oR4B{m>iB%qI@l%gCpfH8%F?)|)N#fBCdVX9 zO?^dx0%K!P-Ikgoap(atXe04YxjtF~L7l+e`Jp}}GSd{;NdL60@;BNUOy#OB%CY1N z$;~~jK7{F!ymkzLKVF!V>MgWSG*TSEt09J4vCp;A-?>O>=wTC|WkG>e*+4?5ivj&4 zuKKF1&4gFh&Dh^Xhs{EQlboARv%Joe5)3HHUw3R)a9|mQDKe>(QFK8O6;;7bT1@)V z#o<5suUn3hIXtD7syvzWfuPjeriwbU9!i~T@yxXNgB}bvg{+0N&f;Eu} z_XH>O03A%Hit52ZO0&&5phIeWwe(SsMxVbeJkJv9rdUL$4c6Bc>@flrtNX15aRg&m zIn<26K$}xUU1maEVxvu=ih`v8!n;^V~_qaF{fd0szHK7N*ZMoXSXLJRO96;vM|)-`umSO z>ZY%!zuGsWVpsj`F;tq8v~VMS{E&dMK$EnTqTn8!@NirF-LgN>Tbn8aZV%)_kp?>@ zJ>AIK`bT3ohjF{ZFyDlOv^Vh_Jg0h>LS;@4{E-{#9#UVFCp&t&3>N8CI4XdX@xg z#r_AiSqmnPhsERd+GET5JvyT%J_!}VWoskA8NANn9{Jy78)r8p{+vO9c9Q>aQ2p!n zBrq9gH2r&(cJwe__lY93mQGt2pX;(#V5Ju#T-#q*&FRs9U^b^Z3JwpVO;6AR?Q7Z2 z2kostJLOH_Nyh&fRJi+xyxzpvFtKuS{Mv5DE;`B~6dY2WQBuO_DuhvuY6j9C)*=nH z*{;%ohS&rIqZHfH427!MmfVt7Qz=(t@-`!IjTkkc;u;mE{Tb}=5F6*z@Psi^`utRw zYPIpzkmGV2j)y4-o!GVxIG#FwDb2+@ytim!BR7w=B(KRl)_ttz`qyWYN4?v z77JQ$@ixs`9aHS4Cn{k=%=^R61!{d3TRMrAVaU@zA@=!z)wLk@y>v`-D>( zyy84oR$*&XG|;!y`+F#~cE03Be&a}Ejj2RzP5Q(Fl+;kTZ zo3?3q!ZKLVM{i*(gu_{`u81q5=B{PZ&pcVbo@b$hDh(?jlS28ETg~2y7Jlr;FsZVT zHzBXp?rnW-=8j{KThNF%86MN9{<(y*$hHdyhw{cYBTJnOsw-=0h5k_)n>Jx1OZ@G^ zyC-nnmJ9-Rq`8Lx{lU=Lp1)CiR_4dtdm84j94DvXONL~;oQ9S+I!f=n$4meEg5mMS zFAEXGJd1<7m&9krfnghRQDn>GgB=7i21G29)=_F1G&aA@j6bUn$}c zF$doO2Ms(Q5K_Y_KT}{)C?Oh#5rl+NYYe46eyY3ZEtcFR%A!d69P^p#&(D7y@N)OBZ{sKR3=<Iwx7pYZkwzv%O#E%xE(_ZStNQ=QP1fAV&W zIgU~yKFa9cfM?G(G}ezZx08B}zZj+jvGw6Mw7R9LttxDBb~nKIB> z`3@>3*xfU;_RJDv6aJWLNUQa0gR%C!w;usNMMj~fDL7TeEqk+%ym+@0Aj*)c@9X)U z9fWSbCBq!n7Nj%4Hsz4iwfKf=O)hD0(w6Vu3}}tebrbxD{&-i9+}J-mrlnm!EKGj= z2G0}B;bVs8@F;>{W$9)xJ|3g4*_)bapzEec%r;OIPcbE}&XrTr1gjVxWJ_b55?!bu z0Dsqq(okf;#1NyaX$XZG#A?3BEPdKYD{J5{u>_YH-$(-%Qw_im$7U}6^u*`Q6hIZ+ z*chVn7{d=qAn@FPDH}F1ocw#4h4MIc_4ihR$Jbbq@T|&k#1YnX!q%)2;F4WZp~=s$ zL{*wpnx)egoo+Ot@b-ZQ#m)}U8NDUZfYmHu=Ocd9`pqtlzsf+&59$GIYm{nG# zmBOMYTgxUYhYi{+;xpjgHGgTOeXyu4>lc6x#uSdDv_l*LT()hj=7?cA-?M#!e-d>= z%zHcfCU1lAkAliciwymEpP9BD6j<7X!}DHqtnw?n#q3<)038amtr!f+@p#?00HI@Q z_Txp4NMZOEOQISxxfDiXE8c5ai9Opy)vpD^mR|V zWX9O>r0aSC)9kPFOs)>w+QapK^{_3C#LM-6F_TqL20XKD-*#hyRabySS7sPD&VQS~ zFR6s^4_g3}Z`!z}Vj0RsugXF|WFIk4^XRlqOr{wqzG5>Vh&y3?7ej2}{6*E7LXo81 z?TF2hocx*No0gJ8;tcR9%5Yg&J$yXQ)f+qX$(&AYJMMzwoNqFYB_xrYG@>q{@zd-n zJh|ADZI#0Sl~M1shV|@wd`^ekXU7VdQgvc!wL!BBp0sp(ig3w6HX$-yGz{wyiR}=@ zUq_Y`hi*M`6!FXO>;`M61-r)u!i67yB&O|#eAU=lfcpx>kKn~}=8hU=?ETQ0qW_`! z+mU}>E23onElQ$ERH}*Z2Z>eZPt%Cs!rywh@P% zI~F+^*L?X1G-mdh>Z>slZca|Kj~KbIHlgtp;#Xd{p0PMqoX3r~!pXAMAB4w#Y7{lM ziRCFS6LN2-5bQv}TZ{-CLHayW6Zsc(T}Z#(R)aU)fFonfm=VPTsa)Y@U}W*PciKF` z3}Nm_jO!n=W#St@-Mpbi@*1GP{cI*Xu_6``ac1tV_PA}ceDR5Bc*I*M+HV^`Kffm7 zCCWNnArdmP%n{ZW92B^7juH08;ec*|YEL=v@+;bs;)KBKyp31z)u!pZacB4%l%F(! zWYpOoVn4?*F3^-kapO~MIpx!}wH8n{&j@|ftdA#KY{$$rL34VNbbnSxkEb6}nBhS# z<)A%*nRXJNNqj?NjQ52?X-gJCfyG-m?}LylSO2nioQ$TKr)gGP+`C|9%q23){n=)^ zWMeAD_X^rSD#MDA<7lp7^hY7Mi^9mO#BOn*zsQkeH6m#j@Cbt?mGE8hy`2_C zyoMAsj8>%$jxXo9ImlrQeI(ZtU>*{+V+>ST{TRQvfrjS7;5Z)Y7EjdrYx3gDN8iwj z!kA&E4JJMDO8UXuN6g?6Ob<8OrQ@dfmYTk9b22k#=lGujO#`tGFh8xA#gudq;ZIZu zCx26EN`&yT&y8zqeKQ6*FzxK$#kOjx@995FS(F<)#-3p7P^o5NwL%~Exj zBV`D9#FqXD1o`(V6Oy-t+q816rxKW03-)04TwaTeN(D!N`Yk$7d}a`osb2i9*WU<^ zx3Zi*p1f$ET&&*YYSl7HCs8h)CGxN$Vw=KXYa%fgPSDml^Bwu}$+)F{9da&Wzt}5g zTv|LjZpdWBrXdhvisN|gxCSKQKeN#&rd(_(RD(Q7ojd`?`&}_1Aqm8C=P$nF8CwhM z6iKEcywTg?wLmleOinr_a!Fn`%@3AIGz&<;BL=U^Cx)kKDeFv zA#A6E@A~9qx9ezmmSJ~i?3Y>ihT^O&6>J_GCew2yKj(uLVq;=r-0jj&RzMwy$a)iz zmWYInx}qH11Cs%WL*dAUUw^ppLAoUAR~Q_9 zh#?c5wR1ax5^i9aeE!xhj&aFNk!)_TaNOZ=0E23t15gwKwLsfCZKypbxJMW4PAJSR zv26>$&peXdGQHjVmE{I#Q~LR7Rwu$kIcH)7D0m!0vQO$~&?f z9=YVNz4~!=p{MGI$bLH+pM-BTjQyf%nQB{vl>K*al8si$dG&2HqdY0wl1+Pov6=r!UF)M!&xr27$I;5pooTM8~?=rSW%%^OH#W?ys=|*c>IHnm+eb~G6lEb zi26k28bku8&5oc5xq4oKhsW#QLM~fN*rB-w*{WiZYE(cCSUx*^xM&1e$?htr z*S|NnUh;SIi8BhS+qGp*s5O>m`d!@=SC#apx^Ik;1>l~hl#iTZi?=QET0Sq4QLE+i zeG>)yXsnNXuInSxJf{lJyju!5zkHW8NJ=mox55lEyY$x7l=prM8bU>oE|o1xC2#%K z!Xm1Uja|G$`$L`WblVkSLjyI7yVb%r09=ed$=~pUMVqJ3r8Y@e=_c{eGO~utFT^bB zGQJ+wc?g>UFLI@IQMK|M24rY)NP*x6|1n#*)scz?5AmkEn1|$>M%!5wdfh!W6XW84^co za4}Y10xV9=(xKF&V__+nESzO=y?wVGh3NsMUz~!1=2kD&!B)q{hVfITDsi2HrbE1{oQCB{L)Bc13yrk(O{w2G6S+sXsW;-Ki zlUVAkQ#ARrg^6Pt<`1X z^s)}?XIL2odPP;XnMxpe(H^;~QgkB)Lz7Jk`Rv_3vSn$X&l>{L?w`ed5_Z^(>mHSp zjKFA(z&e2tL-#K?0s|E!xn^p34$<%V*@q^RLN8lbXf*clDR%_@stKwYO>- z*!l-oX#;HpoVlnJr07wydB#ffpJwKq9dvo)yXNac>GK3DW--u+iMK|a(Zfv9?U2a3 z`^RYL<%x^zD8zHhWAWby*0;@CPD=G%qt^hn%gGv%F$^?`G2fIize@mi%{h84wl&E)A%o35_z6tzV z*`_Pvm7$?g_f}CE^4fDfZu*WUCqqc7Z=JL&v#1t_nmhhv}{Gg+0GxmP$t%r<*(ErH-m6 z3NJhLmFgEoqPWcfykC+>?2!%I8$-X@HTu^iSQJV%T|IwNOAsrr6W`94$RdfySlhJ# z?X%B)h19ul94cL9iX%EW29e1z5BA)AHgwc+lZr##q)#Lf9*WX{gbHZ~+!1k#2QW-L zwX)+>>0>bP-BB{3ncB}yZ}gB)F}1I@9wGFV$O8g^dn&|cBe! zF*x}y>K$~XOhSZU^GHTM+$#r9%f0(5g6b14+hbYKHt+vOk0<4wZwsA zoj-9-L<7U}cj!HTHvx>TI>qljbmoa}C^#_B1qGB`&we{Zx1%RR6h@L-xc_1DxQ9nF zR*G7!|E!$F+-NcVTT+N1xsRa~m1O^qOBzTUx0a^4q360)>(sYtJ_E^=Ir{*#6DV!n zzH+#u=c9dx?C+w&$YKMucu8P$hv6~tq5e%0eu9}W>vxERtOTzP;~Zt~6tOdljp1lu zSDPSim)@%(gD1i{8V|$kR%oMp$6Z%h30>PTWqYLt6FEJb{dL!;wS1oFJezExCE1a% zvfgbTDc7i^#F3EM1D#-llkgr8!#sbT8YBuEdCzOnJ`;Oqm2u7tg4NY{He9AK6{H}V0}9%cW2NapH#wz%7EEKxu-a0g!hlxIh5*QSUZY&Y6!oepbvta8c;9DMF6 zH@(o(FY!BgW^3)ub6_DVm)6;^=$#vor(gNimcpcUFar^oibkF()Tu4<1HX<>*au^uhj1qk<&K2_70qi5q(|9v~t z?9>5>7>!-^N|B_2Pp}Tz!B`GcvM#9w#X4hn+=jqM#wC73K=8G|J1jJ*YH*YHONnMr zmY~dNuBC(mCVK6sZh7k)qDqC=4I<}|nzpYLpE2E-zn32ut?7RcH~L1lb7qmBX2G6K zll}X9GETZs`-G3@V98HCYcsR-HdU-%ztmyJT|OTsL@eXn3Vf=snedTNN6$^K5cD!) z=qTF7A#hJO=_suph`-kctmw*`qL$AJ8YZOT`~U9a;X891WM+Ja(d<#t8Hd1xjpS>d z3D2~7t~{wx*kRX%)5a7Km^huo;PUfsY$+sM*%BVa?Cgw{40X7^ftZwanPd%w$fx6L zk}5t&yN;Z-M!<3;bh(zcFR!hp-v3P(7DDF>OgY2?ek1$^72IWp(2Bsgs)y5nQ+_W+ zBMsI`Jy)HW;1q0@_DDj7hlh`PgL5uvfDwWub*(^ciCPNjCts^*|40s=9z?DLtDn2BfEqE`O$|G%@pYpC94f_NQcC zZat8R7@Zf<4Y5iWokxeBDLaG0qI^H~kUvw-&3F|s0v0#(U6b+!WLHcss}dr*8oMYE zzF7==zu-%Lt%#BeZIO>bnx58B52>0`Ox&S}M2K-qVp5h)(;ZW5_ao$H55I#js4Xlp zZ)9^%EB1&y-4(*LxXaRES=~>aXJW11J@g?cAJRAM?^IO8k>j$>B2MqawksK%aB=s9 zH>kGo`I=>wbFcye#~4%^tM>I8g#S3K$x>8(`mE2h$mut?b6c8s0;6vhn}0$raJu`H zSp%}DhAGkATi76y+M(s-9b6;AUD0KlRI2-FeuC17w7Ez^QF_Wxx2ht=CU1|?V z3yVW>dP)PsOxv19_uxO>J72Y*>CIhHQV=7AX;Uf}-T_6Sbl%y8ojN{XysIZreu$Ge z%VxUA>w)vzSD?~@?pvwAP*fL2F@ouJ3L7M{;p2^dSOI7S$7)e!2KNh7tp*0f;>?JW z5@+X+yjF%UcT4tm%gFAu(>@QUXbCD-*IehOGG3HUTB@o@NK5Hi>C`yLXz38KG(?$d1fjWa6>+PRj7rr9(|^JvbCJ(eR+xC~oi1S?q1Zsj2sSD>;-0Hj%Se|s{z)9zR$kJ%|g z?N8{>U^p{e4mBFwzW}6vSFDq-ODi-k9RW)!q1Mh(X<3=LOAbf~JD=XLhz#qqa=19V z%AhA*wao|}0xtdn&l37I-O9HyFdyiefOwfL!OE+In;WMfCe#Ml@E<0^N< zs!(r7{~16Y8>tzwU$s>h#Ez_~MxF}EJFTv%*()Xf2?Cr7p#dM~ZGr08m;m_u^r@Fv zF6W4^u8VJUW0ygJ5zPEi)zFkeZ6h@z$b(CVAP&cZkw<4M?h=evCjJR$&^1SWFu$cM zLPN7fx?~WLR_FIWEuFwjo|2Xd#GW)bfKAD^?AwNUU&6s#mBMwP0C$B!98+ zBY3HPwrH$zDFzg*Wez@}X$d0WaM>v-U zF8-lHi~;iu>x^1!>y2B}M-ki#$w6|g4;L98E`s@X zVx;QEEgSK3XvKdoK#L!v)Q%z19~_`$oEs z*>zP)8ukV6To7u=*mL21swKB6NwMAm~ z=Xj5Zb>)d-@t24D+!~sU%zw{5Qs_8EIeG>%65?}cgoG3pqcpkE7npHyQ8*fpXYh6t zZv`+RW%xLDX6Dpo9NB_*^lh$Z?1)%!A*7CN+j0+?1#RPAfITzV4EEFv)ICr{u;UdV zz78cUE-eYQj$apD8PXbi0fTxLo@T_Ur4g#1Qa+Wc7a$#H=@e-mF7l%sDU*Qqvy_cA zh_&JXr;KM=Cs)NiD)|hW^4L|;kZ@CsT1*ZygI2dqa~)#(g1j7!S5U(*%sDpIO1FPB zBFiNS%2l+{D=VnGxs<){jsNL}gIr*l zCmVr2*d+?+Z2Y;*NEe+jCG1zO?Gt{$Z#+;`!#*~0O=J^^tj#3aZtt&ZD5-_*|9kU% zlVI-vfwGZLIYs^RH6YkAxsGUy*wAdP%}Z^$WuR}4!(Kt-PmV}zAvigf>z6Fz^0}iJ zdWT{L`j%BJ*`$$RnWHXEXN#hzPsV#N<;q{JFfL@~g#j%c=Ye8)db+B!@G- zDvs;5A1)ntlMlVMqUF9y@J5|lAL1U4RsEGqB`KFT`EQXihN*_qguP3;Gm(m*< zVjcA{vWqY{xH!7T#-NQ6G8F@-ueDAR=!P^79Bz9^3AM8ejll2u!3DeUG{@(2bhRP_ ztAj$?)d>nj+SA$lpi!>bHfankPSh|}b# zmUeNJ4@;64)HKnavS+G?13uz^e>m=4{}s}L&q{`#yBh~P(9CK2H~tVoM>a>va}o8~ zTk>)K3bR^WX_7MQN`1NXi{*4X>#*TNX>tT7+x@3v6WJ%=c~G3L1tYqW)V${TZ>h+8j>BW<+P{b>U4J@1cjB?Zs%5(BxL*;3BO{rR1ryK^2*ZU zeMeV4#K$ukY;bTGrx5kY@+0xinhi(zkEei9yf?dQC@I^@ACTT)Wnn;V4GSiG3V2D^ zP=ZhsO&^6{Qm}GZFJa?V-kw3j7FK351w2d;6{|yoH6=J~d~SnZ_=MW}#_e|WE8cJ@ zLpv=cW7q0P^9i5!0KWzsv&O3|R(YDVEN=%eH$GMX+KQyT--iCo@IjCFH8G{g4C#Ug zW#NVh1rTlN&nHYD#PO#3Du_^65)iNyRu44--Ve)qI=j#o+_i(Jwxpp+F;D$~jZ+Sr zh}3!}!u2NIOo5>Q)bqU!@pwciRu@`c5VH~|pHMu3Gr1MSJ6cp;!tbch)(GI9)}XHi zyzcm|1bro9#2L1zW%uw*Nwl&^9-m0aqO^B=1~2#?iTAr`l?&S>78acIes|*J9{y<5 z>;tk*O_1%ia-@K-CG-@L|Bpyilsfet8<6Qb4>&h2-Yr@YRkZ&tA$MTzO>>!dcF9<# zhgz3af{j3GB7rT3*>sgBMGJJkBjasq!m5BBi13+jB|dJCPqrhE%_}~Zq_8-E)6ub- zizWojuN*yDN!OT_W7A4x?xsR}QC;m&mY^|Feb)x=q8+s46ET`woc~k==3#rfa2lyz ze@APjY69hUMB*i#;Y&eV@L>}&RVdBiJvS2z>R_u|RT+B5I`90}lDq{&m_EH~Q!j(% z$@&KxY{a1&B!GbT*Q{Y?k5zS}Hh^#X8U`-Resuw84zT=oQ_N(q5GwR{Ca^@?!CcGZ z*Dr2hbQ`^;+qeD$39>PM>!J9&iE{X*acrjXz6OS>TGqyvuY0J+bO>-@Mx~)9hm=y9 zdPw>;0YN)w#}b}>u*bUmngWa@=HsgfDG;zs<2#?-=Jj?%8H8U_RM% z!NTr;%qFdX2$T$~&oA{c4I?>t!iggF*qg ztCe7~pD*Iits>}$G{*rQiYl2WIRj7ukCxGOdY0tb)Ef&D6o&_$eJs}aLbq&~oe)Hk z>f|!LOHoT#yBSL8nt3H%DZ!&j=k1T^1;#PS+~MC6bf~Y@j1KljM&KW;Y_6k;V$zwQ zWd^H$irt&Gb?(C`yRIZtVju8_sG3t|W@__!g4nGw*d?8SgIRfI>)b_}eEsW+vvT+6 zMfTP?7I%hu1L~4#d5sp$vZ()RjvaBRK?|wS6 zl3dz2DYNF;f842cA3f{b|L}Qja>_AOH6$AM+$_3z7{OW7`oIs$j7Q+n=BAO&ZOZr9 zeEVjp#CYMxnB{v+LQ`2e>xx*kOswB@kM;-_X?C(aNl$v+r=6D+!p7mK1le{JVxh({ zUlgUr-|icEp{q#MbK0divSv0fWBdUlpW1t7u))geN}|p1Mdm^^+y_3{3O$=ISkI(W^)3 zYKKF&t=Iv7$(#Zhi9G;F*zhjY6*tHzE>Gpd>2Rl2)l5J}EObpe!H!sEIt*HKqJz;F z(n6X7ppjm_X|N0UxxcGLZ=9jOF2_o`H`;zhkrQoRTcQVeb=WdiP#wROM=(}_>pr&F z4n<8=9CTU2(VL+Eb>-A=89yG3UQ08xJPr&^V8`asGcmU8)RhGFExC)U@j}v~UiY-( zp{EY+?lMTfS?l{FPzuQ$QxUe=W(3Ak%D_aKEK15|9fB}XF_omzTdi>1X6if9$)QqF zgS$)8bEdT7{}6+wK7$iZrGCd$9Wqn=CS2Y8(KEx)62Ral(J7>XLqxbU^Uh5{ zqHCm2ty8Bqfnde8QG)9S_*R;*>h9jGW?<02P!74FBNU@&Txgp*8-}=ZgMrk>@AgOR zGFuTt6R~I>-~8UdHkSORQKQrNDwhJhjJtezo({^%$ROB4DprSIZvs<&zixzyb+kq3 z_QXt+_8g(2#nak-U6)j`M`lqHV@KKmJM1b0( z0PM18!AJxqu>pHBJ{1s(q^zU_6a~9^{-+9vwi8O@dHLx>>(`9F_V`a>(I+?Hq+T#0 z4GQ8jMDVBJFW*e*q`A&zbhJWdp@4a?;W3(7j>F_SmMkVo#wD?rmJ0vmTUnBaUd65Wq@7@hC77CDQyuku3 z#d$4L0M%i<8%0MlFF}_m>WA_a*gH@4( zWJnY2$%$AF9R_=cjfgZf_*L#D%ax(n-LaK8;Up_oTUs;!mPLK)sAn)2AtuK%42B|H zcvsM`Z z*8|{!n1!m(8@&s=MpGHOX@z|1WT>dI@a~fY-z&X>vZhZV1C->NN_iFF_F)rm3NOHL zVWs9;QpB$>sA40luqj)}pk{k~hJqkMHsMpIINuTKZ48qWO76LW2R)vYWkl?tV8HvT zVQ=9QZ7OjADR$?*xg%)hx%S1Du{8UY3c9l6k$*!4I6*`~IYF^Fr&$VpuYfarN1Odd zDurnqk!kCQm??O6b8K8NOGY9gb{`=(5ONH+(MuOq2Rleyz<@tEgUz+rLnuGjP#(GT z6F2yidq_Q$SuI!xl`n?^2aMU1x^8t310#zIxjY@LEs{}#zsxF$gMZRI zhflKmo7vrqXLnHx42a9;i~0e(dC+%uZ(+V8x2UF*+;gU`78Ws1T3lWj({##K0lKB& z3b|pxFcUR4`P99wLPk^^*e~5Cs1N7o5mvB!`@)8et>vY$z9!%2oL526HZJ3#0PE;M zA3}oKD~Z&7((CvYPOG1^B3}T1=!jh{ESgna!N!j=(D|D!g_IVB{EQ7n*caXos;}$E zCtv%(5nG-A(CgPbW*V}{&?N1|&3I%>QUsUSb?8Y7k$C-E%Kvll#DKRuaN5D(zwNU9 zO2qSWV>Hvu(J@ZG7)($8Mf`u;Zgq1p_l_dAiVf;VMIMWQT z!{9%-f>+L#O{f3)?Z47iCf5J|&j0LTQTuo3|HHt3<^O#b{8xDw1Nwiq=D$l(3jgn( zWcFVxykl0i)6Mf`2b<6<;>WEDDnnw37>X&v&GBWx0SrtD9dyI}73}D6ii2q6=;(1S zcCPTF)dPxWad<-L$%8&e+YTPsO7#L=Ix~KVO{-lFE-2O`RTiAGJTT}7N;KR;Z6Z}o z2{(hn8@gSb#YED}Zx)+H10`7x&woRnc8Z?3$xVDqmF` z(8{J6o;${pG%eWD0fSI$nTm+T0OC;6>^+zU?_6)`V8PllY|<)GtuSp-@f!lqAUMnkBuHqC&E1_}4|R{V6q!p-RO%dkiuIUwE;!TmSp zQknn#6o<|LCECf9fJHN0$EAt`tkl#ni5OOpcS2L5AaNbIYnrOCA4BaRlh>-KXe0qg zwFMC`Rj-<-S&t4DsdCS+M5r~@899gnLxb8NR;U}!pot<=XW2Swz&ul{00;R&h+ZzCsZnSsrEF8%^J2SK}ua$5Um$zGO1SRl5rW9fRiTRSQ@H12E~EV7kQ^F zimAaAJq)-bM+z$)KhX%Ba$%KRqW+~m(P=Rm%jXf&l$28vWtx+#l$(bZn6-g*lK%BY zkt&nfY99W7{_X9?j}z{XKk1%N4-u8CCz5-V+i#Vz&m74skDdq7+Av6XFo(5~F}B|8 zNfbYA{d8Cl|98%Qg|7n_8Yzy|efjz%o&kJRT$gaDpK zA9^AlMyP7dWSpzx3C^9#!^&VO6QzvipxEiaLkTXKjg@fhzYlL|27$kF%NHSR#dv~1 znEsbKB45S|)eTAZfTV%B>!?T?y;K|Zk=xJJCRP@Gla=%}v3DaEqM#f|2}?)W-zLL0 zB9C%S3$SXfMTyzFCEc%Xe-pb0bOMk12at@DEK<`&-_Sm;Oo)$6-dC>x9*#d!UF3@r zwnvvgUU5Wj;sl?(F;hv0%{?ko{hZsgX<=wauzL;WN5MSrXNKLsam>W++K(7g}tGE>d>@&JP@`Vm8;!SBchk>rFLIa*R&UW&V@PIp>2JBX$Cl$ zLmR+1-tTo!9f>p~wdfpRXK}<(%Czhqu6uvAh2IDisC}mvc`UatdpParDX*J(Bn!I1 zeVluP4VrK^2JYUk6eCe4Z1;TJzmb`IY~hyeSe+BWI=`}RxCl(7c0b6tOSV+47BgAE zIvsX@2*1uMbt?%wlMFL(JROVPrg|yewFf=Ho>w4#MtOcV3v@ku^f@2EN{^Pxv!?BS zb{3g_`1=XrD)x3wh?b-~(%@=)dbM1NNsCtZce=^nbDL*SJdQuXRbVl6PRDaQ-|{>8 z^5R`i)0Q7?_lr&A>#{+&BIduLLS6Z@_z}3?a-RdKXd|Ci&Li@0!g`(0eBZ*|OO1kP zL$XKsancQ7z#XzE%c6Bp(j5Nne`6Aq;^^@iYYUYKLBYmnx5l7l?+Yok0_|*l8dvAm)%LLYR*F8uqZD4o_3wz z1dOyuBz?k5uikTCLHYev`@PufBem~KxUD6e;R-v{M1m?GdtR=?DmkVO<+4(4jVEI& zH|Z*u`RuzT4IU_tV;-0j6mHlk52SzLCz& za`PLs!JADrL)h|F%)are!5COclU!w*Ta=(ec&ya8b6{PUvC18(x`dg$9-Ce zY(nRTPF@5A-i8K&awIIU7#ZG&s{K!w1LGKZi`lTZu;R7MFaLzTZhs8ZgKy>Sy|Ic< z7^02N{(FR2F;ZJ)+wZ4A`=YsuchcN{kutP0Yej^52=9xl?_XCv9Qoq=4IKS{_v8wR zOaRlhw$7w=ZjYn;N(I$BaQ_RE7fyq^-K&JE)cD97m|L0AubA7CN_f?Z9z-=%>ia6> zvpgx3R^ky-fz+}>9(uph}19;x2`OTw$^ z2x458;=3s%Y;hc!#QgxNlwsJBv|p^hZaxw|)qcF|izC#s{dtPp2#~zK33|~I@UIl| zKbzd}%uHRl@=qkl9fDR4&X4RKPe7fVdxAI4&9(qVwYTgHt*?sejz{|NN8YE;Z}<-f zOjscr)>Ij0$T)Y0m!_OOzovG}R@V>B!Upv`cKL8#1l?m-=)~W3I=?|VUe(_ICPuD$ z5SZvBJhk}7 zVe&4{duk}k`wkM^IElK;B(>|s>X7vDNc3}lC+|Px>3Su7B(u&rOd-4iJyZw;e&Fi+ z7_)Ou6?$#TU$sc^)(+>_`Nu;(Zccr4oEUR-P2b7$Aa*`pXOoil+x;a=ZCAIBmO$N@ zLgsx7E1w(oX*?)V0W2$_vgvOPAV%BewA@_z*#3?Sx<6o5x2kgaliSt59r*J3{nb9V zWNrV#1{;sx1$XEoa@IIL4=xa6+as@j#zHOCH*$n)xSLm_!=Pkx<1 zzlPWL$j4*n*Z0>-QreokEgj>deTn9qy}+l1t5^2PLx==0zq|k0A>YRvY7jaqhI7Hj zz`XyRv(xZ>u*m_V-`Y)0oZIwdGa$4wfOmsjQYceR$k5*K(U8n2VhPJOF4Qe92Wz}*22|0}_d z&~}QTA!p-s%i9^RRy$Gu!^f_%_@7fde%<}mAKr|!weKjK@VKJ9f-?V}K>etfpdJ@U zQVH?h$+|SwS;5?cH!_VX7YCmH|DPdi<%*Qe7wmQB?DDonBF?ZNB{H;9u7MP%lw`-o zC(tq!KOK~{X|!18iJhy6dJ_jZtlQNUcav}Yt0de1PiVXBGgeug+PPos$I$@(lApQ| zfUCbrB{0~&jrS9?roG1bM`Qc3g2}j+oH5NLXTpQpJy3L~e|?=jEbb+V^EH>3#M=Em zRP&_|&S6d_Xrej6x8<@wJND+U^}`2o(DNiEm4UDAgl>nSRX}hgm>kgc4M!jL5zsnT zt<1ruVZBT%IyM#W%k^37@LuTVX?sS{WPpHg=v#Br$HT0B87kfkn$`0H=IeeGxd_I+ z%kcu+dd&ySveV1QYA9+?#aQU){wuGgPF}w&F;6(ZFky-jjGyEZSb(b~we|~>%EhK7 zHJRT_;RSSX99CEKev#PXWbk3YPPZ%k?$2Uqk`7YkIY&{_pJvf5R$5Pw&c_9%RqB(h zpaB~XBfoz|Yt|?3wmkIci>!9U?>dbRc)Q`mG*J4j7@rqhbwb*v6NJek3}$yHr;4o0 zUo6sl&yJX`cgR_e<5QJlyRny)#$#i?_q1pBjhxSg{BEb}s%SEWbXHtbc(HHb`<2PK zwHDRAcfa`l%V{~W!I8nF!tBuCBaf$=w95~t(RLX=!@V^J2&u2yWX*6jwH?dY1xYD` zL_;|T+Y9X-&;C&epLVi2LPx*%tKyZJY2xuHLdGf2ElV`c{PLZfUPkd1%ug_NtFmSD z9wEG7xAW^FK$y~xyPq8JeEgeeWHT;nEgQWb2JMh~woJmBam?+~?5@r5FRy)Ms-0?` ze4aYqn|F@O3?dO9JCgF|4W)KlSkPh5*eVk%ocRhUe!n(nXw97T zRo=59e_}L`@QRFa%X~frU(ui;{wsI%QfGvVMlO;jd0x|ZsU%u9n*b4gO@>aGggliy z7pU#?fg0%8+?F4q*fiu(#H|+(zybUhpO=fMDt`Yk;-J99wakdwBCt+aa*ZVxi|Hhg z&dL7?N8fk3&S#X%nrTxk+ly=S1$FOq0?S{sM`N(dLm{ox7WzfO-Jf#qSoDu?8#nSW z!|2W)>N5IrJAU|`$Zn%}@}Ely{CVDFVZ?2lwJ{>wxww}rBIPO_NWa65(fx=NOB`0S zn_yFzZNU^4^g6deq!Q*0$#f#cqrm=?W$MsR$TRk%>kV0}cc~4Jy2LdzmcAx}u~pjj zd{lY)VM>T4>^?Uv9hnc7s`8LWWh|CMFINo?NAQOAO872sbZwm_?X6hnk$bl)F6&db|Unz~k= zYL2lE&Mmux7~e%INCCziKSs(TgmT_(31AxomF7Rq*hlLL*|6udEESy_4dQR1|8m6a zwWH3Y<*ea}NB2T1lKYm#7$YY4?)2cj+v26`oDtdbP1lJmAvwrEJWz@^R;kt7i==O+ zW)gt#{k}c=*=K>E7FSsF-sO%F{13j=Ly4sQL#`dHyC;;K zkG!8wyzd#6oZ|G`e3_Km3M6>xQW*+glF-&BGRNNg_VB_qd@V)aC!HxP-P}T;`D#!5 z3OwxT7-|cx(n3G9XDf7`)-%_?%iBpAhKI*pNqT=}^mTWL?A|q-e)wIr_OriOJ^^9B z4Gp#-pR+PABs-aeC0qJ&AL>%R`Dt+%lBjb2IUf&)3y($=y2NVMH0@PaZl%twf5>WY z_w>bhxAxJU{L3Mgmk0mwxG-ObAy*dP7pYKEcwp~{zp^Vvc3LcF?RTMsB$UFyVGYzc zstuU};Hs*;SRgYCsmWIbRKPEUxjjfeDNyoUYzgx4s*diE4+Cob$9FPHejKH1{-|%) zP~yT)xAEHMdNkaTOPl~-6yeCYBB>(Y?SY>Kt=lB6{E2 z(au#{)So4r4rXa{tu=>ru|Dg-z~!p}p!hYF)geZ0lf-YIvG?AeZ6UZrli{fAw}-6 zG*c6(+@wx9+l-J_$^?H@(^mQ|Heh+C1Ri^XD0&wGE-hx_T<-SAGrZFUFq#R<=1y~; z{VHkj$*V2?9Ur~S@cGrE@o6zh#my_%nh!gQlL<9E6X$l8kQ7+SZKJ#WK$P{uk?{7n zS#hnF!(BGWAK~a`8Q?}=$@#=EZZw!;x2Ja@_+u6_69WW7bc#mVSKr6{-VgwjAnMD= z8`qjGs^;%MWlE-x}Z9 zE7n}*F{SI>xcw6;B+`Uze2m5Ng4^w5O(X5?$;!f1 z^!M7@*#9rRH4(h!$RzJ9XYb@fx6qj2nF9N_(`r({T^eDU*Y~@HtIQ$N4<7vAgWR#L zW@z=;h|hZ#SBbRYv+X*#iJ=fD><)a{8R4o%KPLMCdn5N(iXVZS-+}UJ57WawHdfTc zHSJNo7vg$%mz*6!0m9su0-jSr38_QXWO_sv}f+GGoAAYBLha@qUJwHUAF48BGSec6^xXOhZ5|XdpW8z%>V35w`8ge6{ zYek0`-B9MMu8?Do)5jhXa*qb#73j*BM~#GmxF7t4{GFn_+k9CiW8}s`r@2)ljDuC8 zQytAFpzs8*Bz{{y;#XxhuPImg3Xh(GhZ%yLz*jl#C0dU2!&&zzCYx8x?)`qpUlu?NGTV-(Kc|Z^bNOI_62xK|C|jF+;uOHTC@z(#8s4kCRzg+OuvQ^8ymG2>jt0-7yvR%cWpxI~VIryB(t&MrNX`eJ4;I7b87 zVwyQR?G~y%5bUomM%^y*lzfR|F1w<&`sd+ipA4a#w^p*PLKE#s(R(W`+8VuOIiR3# zhJi+-t=*iczKmRLojQz7+rZIHfWI;||3ZSCMSG*GJM3ZXWl!6>;PndA}c+pYHxm#STw_# zHu^qS!P+?|zS0!xjJ4^(HT&hQmuSjLExrAx+<44rn;(+gXMV%cupg@2aY;-`r-iGE z+4ULU9zyL%t*_HB$=P~f<#4hNU7vPuvcQ`0;`acTjM9c5_;&h1|BVHp#@s5U%cr$4 zkTk1yp`?x2q2!ZLH~gu+tg#6CoaH~-_lFB~uopg`@0y`;FRklGU&&6#YSjO9;@5iG zWHL9={q4jst4P3h-5Dv40g*q_YN63%we(y)skK;!;C2i7mDQFTlQw#&DJWTkH2?>a?q;pRAA}mc?eD(a){N5oLOs(*o z7~ju{ z)}Vn;1#gdnCkXa zp-Ht31+%@G;B=vJotSSG+m|C6ATCy}tUxMqL|(itu_5>iISpU4daDQa*Z#*Pk@R;6 zd~?>v?)HQ|k4;-Fyv4}%AruPz&nmO()&Wajwj-U3YzlC7;h*FeI|Kr#TXmYrbz}0b zE*|GP?cH9nY>BUw-gj>HbIS_l$Mzbt5dLm|Fh;mX%QWd@uwd8QHGh`o7Sv`k>-M5a zK)#CWmIxVu>Nq1gdMh74-mIdGdtKieS;_CbMh8!E_Fl}?okwhYKl43>Q0=H&F^pHT zGb5s96ac6A774u*zNWksP)J;>dAZHw_GBv(F)o zat+ZmZqWy3AAeh!MtM&B{b=41&I7&+k$uyN0;X&lbL9yIuCIea_P!aM3n`&A=vmSJP+HzBx{9UNKPdal z^{eD4jKM9ky!hE{sF8aL6^i+-Qq7p+(G6MXeKLtax#4Wsr&0;cjj zE*l4JCJj;Mf&8%+EHo~RvW8d%cHn2JM3-`g>d45&mjKJ4_c5wyVd zh1Ak!c%tYq4d9RPFB+Cv4WGzWt?)@pV?KA9r#JGUN~O4zV^n-A`fh97$&<78_R`{I zhrZf#tZc`H3=VR%e>{LSF28l1annsL#vC z&549Ka!W1&%DLjACN~V_6B8bmRlq2Kkmng_rSSFSc4M5tnn@)jyM|>5Q z3Efe*dS?)|#`5C))zrJ>vpvzCOwSGl83z_AwhJPa6a^O-nsaKX^j6>&pAq)A!PSar zPd2Y-r!|z8MVJQyQ-#wLW=Ei5yY)tHM4HZtq;`z& zaVmutsZ2070|fvdyU+E8!hcEhZ4Q7*Zr?2ihxvYV{(x5C1za$DvGe_96Rdc{UljOU z3Zi1|mdzMosA3gT3?=E~g}=VQmSe5XLfn+L3UTW-#88$?r;h zi%f8FA9gbKw}wpw6}9b(GXylb{A!?#+yAcW4+VbVK5gk4w&h_(73$238&^d8ku_-4ym&J;!0Tge_6f)yRm9_{Jhyg{Ht6?k zI>liQ`w?3M-=#VTot_-2+v3X$FHA~4n#!14?B=k{G1SS9?Ny1svCfK3TS_n4bljWW z?W{1HKka?B7<5-2gva}++(XWntEF;Tk>K+d%1wpRm0Gzzkf%OCUA{J4uqeevc+WN) z7!t$bfpV!U0rbM4RHys_edGd7%Xz#sJi@@=_veQ7X6xstKpz>e2mdD&!Ya>=J6+Pm zSJUY>{)rs^;E_7m<4qJqb`cyY=1DD7?2G%a@P zg^r)lE@2mL;&`q^1}LF1>hbN|qxXXp!o{@8<1sF!>{*m#Rmr+X;LMbXkJPwnJLe12 zG*m~W;KgEAT>cK??(J}~{0 z2oVgtHxQtP0sY9=_n1}P{-UJ>8OCS30ICri&NVfGuVW*qSP?kmHqHt9*~R;jMV_6R zHwvMr229XaH$FN>8#Nv4B|`8MjxMUFh4gIGI4xa)`I-Z3X?KAIsBzWhVqEI1Jvz22vEHw2% ztbC(?7SQ(3`lcIWy0>l-GQXbM?c1_Pn|QjZYiP(Ab;7=C5-he>)=8obMO{>7wx@({ z`K|nkg777lk}9@^j^tYbU~TT*K0ob&!LDjT%iBSu%a8KXd*Y`n4WdxH41 z!KEc>dsYH|LB^AgLB2oV>4RUV(aEULvZP!hKoCqxA&%ou?vuV94Q^>Q@_6^jSjpOr zK_3#o+V>KW;9Wto(S>q9hU`Rk)Sgbq%d3}V30-J|9D&0r+~c}8t3<}eTyaIg-5_3G zj{%j#hC0RUzilv7lTxw{0_624x)I86-^8l^g5*U0y2Ch1gK2BRC6f4*;NbPpk4xgM zf}wR&2I;|B0OkO49OT~uP8t$Gs8T&&VQ%jG1>cPU58{F8nyaYVA^e6@GP-XubA9VK zjkG!BHp?{HEGA>ZfYdWVwSEr&7wkCe7H8Va0v5!zb-Mi0zWiR zwDg~bhi0gHqAKvYj&Vxi(6SU{v&f(HNxUF?*^TgCu5}YT`D5h2=%Psp9^OtFkTy0i z^sx$rJedY@MdtNdm=e$=mgkLNl=5eF$-1(q!Q8h+7F9}LU;lQTwVYDr!WDa`f&J%u zLH#mRoYoREZc9c2o6D*ART$>Iw@X&sg=8|a?1^ely&o+rG2_^B@AhaV$1QB?W|Z!= zWwaqg+eE@ciI^r>_S=D{h7-bENI?BfHU~9s{YC8#@1&pVx@m#$b3P8UX*P{oR$~hC z>U!DK8!@8^Y^;9tZ9>DP7pkkv)8vK_6=u+25tTFS>Tn3&5Qf4{EEF!K!g?X~@M6=`ZBYN}JM= zf+>NpoO?4ZeEk}OQ}^rN z<$}Jgn9`Q&;$8ZlBH=ry;(jJKh68jNE@Wn(pV;=KZ3wJ*xi;ck2!)e`kyxji zixBV~ryVtAo|YZ2u57hANjP5BcOYyR9-fw`CIUtT7b9B7g@RtK&L-$HGV>e&F6p+= z2WuQfAcETUUW0XJn~Q(cz0u>+J4M^ZW&J+j`;CCOt|1YCOC28dUi$EiPVyIV-(Jvo zpM|UZq|?^ZS$omu9U7#57>L=;id{Hsr87+Dog2Gw5czfIK?JHitH2w5o+^4b{P`!( z@sj}8jKb(vmV(R68GH)!X}pughFfhS;a0--7U%bUG}=-rDvyE6K<1a2jR8qJ-4f-U zt^mB9N%&h-GA^2k76IP4U)P^P3jBe>m2^zyRKiSnW|~RO_M+<2g8Doc^0)sFZ(kJ_ z*RnJk2m}r8?(XjHF2RBh?(Q1g-Cct_1b250!F6zV3xPYyK61~$&;M{=?!$a*)|y$} z)z#Hi)m7D1Hx}P-f+K!)JpIri`LHn(=6dzXSOXOwGc-~H`SGVYbXSZ-B%m9Z>%*Vr z%IC$Ch>^l_dCzN(3G7!4iIX~XMr9=XxZOA=YUOS=g;9;;3YVQB4y213>UPEUid~31 zJ$Xl09ucFo2Tii1HJ!gElEsV--_}YVIFyE`bxnfU@kaf#=16K4i6OFW&Zpyb@nEKX zW4Lp%sb4V}sXkzQIJP#9=zAbVu_6-8XXN&W)EROTEJ*WG+Jgb*Fk8ZKQS|ReE@&F+ zfNjW+cX+%GK`bQPxiaQDT*BuzXrI`SPA=X#BQSdA2v&yeXCy%%*V8(D2t~y_1VI&f zpP)*Ej5Q^W^$7!i*Px29sv%^C!3z5O+1_aTTTg^d=*VfG_>=e`(Spf55hOC|L12)y zyaO0iMETYI71KrHk<{7w-d-jTum;BRfpL2#X2BcNnRojh^W88c`&HQ^~#{Athc7UlN6)sr zwLO(A?ATRg7{vtndkl2Qb}r3)6fBD0qM0xVF}$pfZnHgh=UH8h7t#fNc!b{+d6@@I zd_O@Gw2Hb9?4lU_*_STCWg4qi%0 zLNcnp(asRhhnF3g4^0sw7+GYauVe!s8Z6(4Y1YditdxBCa(|(}Hb&m;CDFK6E&J2n zNMt}sd!u$zgr4#devTdMQQU{)@rcyhzWVyg_x7D%jO#H=T-Sq~V9@_m$w5jq2gZ#xFcdNw&%xwtgUf} zS(eDjLt=tdpuIYqg{TW8Ny^wD>xBTYVlWYu$9KxJ`D^PW3A>l?csyF4_5yU}mNcdm z!vp@<894}Cr@SJ|PH2y~X_BA&bSrr$q+gXS$RIiL<;yvpOyx;`EaYxwWXowMNm@47 z1hOr73YhiqkQKbO!UZ`ai5lb-qlkwHE*R~0Q-+#r3C@bMi z^~(7cMZoEVz)l;u;3QybJHV9jQ?1_qSS9I@PNiQo>=xz#zAy)f`~FI~P)aJCL@aV= z19Qd60zwZgpA80jFKUb?01tsAgA5tJPZ|>Up#B?8OwM}dcFAR6Ipes|kD!N9j=9#V zGEQ#o#aSt$u+c*yxf_n63HRKqc}&(0RTPtU3>F`=cY%^6J8Eu8ij*DwYan;_ZEQZ7 z9WJ@Ymt8VGitv_4xwc_B)8_%jOZ*vxy(ICtGWN%fbLl|9((qG&<#dJK&n7NWqm4S>>I|VzWcn5#rLTmN40v&MVz&3c-)! zxLofDd*QJjRHY9F6Pr!G&s!1CYMo z0%u!vuy+zKG2jRpJB)0R$T_>ir!@%@%Fwa;%lGrf8|~ht@I-rrNs>M~!>5wOWkZ+U z_*LHRy0uR!g!x3>Uw{iwm7m!X1I71>ZZ@6BoVUwME#z{!z&5u#Eeh0XO*f2xs1@07 z{t9%+Bq*qrQ0b;ac1Kl21WJvphd?6LDm??ejt?76*X4IypOd`*c&pMnd{3FK9SN z;-Or}@FW7?V;RfDj2jC_hcB+qlh~?H@Sr3gm@Il3R6Jj^KuL*j&l5%j1${HAJc#J~ z*j=mACSqU(Ka>m?NH4X_VI^_mZQ1Km_Mx&j?r=ORPi{P0;yhA(Xtm`uX$gZ%DZ6*r zX5%}prSr|AyGF#{^1ow=)!R%WMo4C;ACN2-?;0ixjHIoh-AU91ywoRSB8=}^v%A=u zUI@x;zB-W6*ng=3r0(0K4BScIu)4$gp}Vuab{k@A_*g9E^+to@maBW>tOpb4 ztws2ErKx=vIBxsKr0da|1_g{P43~v|z6T|Ys%rdkDj#sv*DErDZE?*CftS-_g-2N( zCCh2%D5|HhbUVI&G*yx&aY2dbwhVi_^o!MP`p9&*MmQT@&VI#hC}U+udpN7bU6sm| z%b^=wq=3XAGN;8GRcx%q2;TED(>1R|cTYY1azA)}+xFCojF!^1?x?{`H)U5bz>c63 zS3hNjaPzMKZ7)jZ%j)*Bj7w(5UnUeOI1ugfzTx2Rc1-L|F6qr*xOTkNEycol5a)qp25p}!~T<|?mIxmMFP5@rKouLAL z(S3+%jo)dexS5+zv=AbttD}uw;}4_4OpV^XBz2nMAL(jHZ!BNITU;m zu0Gnp(-+U*tM$UR@fmq>%P~<&hI$*23t8unQji8SuV;KO5xcbx)9`cxbPBgxNKe6E ziPxTnEE67$YB_N&md_faRh;Z4g^b$(a-6r&q}$U+c%5G{`Od+=UWXn@Au#T&BJP`! zt{~?L-Ob|5W%HnKD*WL}p9*+%rkDEdSk5-5uK|JKm(c~ARfuzfq|nmJqo-Jd6?QN9 zt)}o-ABdbnNQuI6&%2^WbLIB#y6MHCDQD^1nf9Mw2ZG(_AKFf^9=7FPuOFR7b~0zH zZ76vL3HIC_AP27X;~SwvHrxb%B#lOS`J$&B5fx}r^h@eXsz@>EiQ4UMJ!11c4CsIt zs+zBM(Pyj1sD9G%-$@kOoaHo<%Ngm;^#Na|3@BEXzoc;nXUU)Zc{$^QVdujZUb2rX zr9e2>7QLU-OMjUtTlahCI3QSt%NflzHK4e;6JBQPy^{e_EpW}Tk0Q6g#~jnHrfBJA zjB#KFg^gu22;7a)&Pa4$$f!0NPgx@X-xAXvhxAL?*$T8x9WF`<+yOvdiYH3u{nu5G zpH(n?osAgFf*Bi}d_8eZ$RpmY{koDC((gAc{6`|TB20cHH2A*fb&6H>*g@4>qI39c zFXD9mb~fL5;=}szUEbijeVS#Q9*CPW76(O8AlaaVwk3A&Xlgf}C;z~8uaK)~>p)Gp z-!Xg-nT5Q1gYkM%-7D6N-&2bEO)le#Mb?rITy=%j^44YGCogwdeUR?mH?m;JE47>i zoqYxEum&r;-)u+?pT1}cIk3BCV%dAA&0%`_#N*-D)r(0E?jUbijL!w|xy$Bq5$$tq zODeMJ25^4ytxv33M9I;qy7f+?jSPy!2nhSh>4iy~^V~pbU#Ob1le@!<-s&c1XFC%; z`BMYQ%-}ca{30!sNVxH(BaSELifaC;q-ed{!w^Ka&YR< z(5hFTQX6ovdZ_PgIdf0POI!?L-}^uuuq-LtM`^Y8t|gvN4ko2&u==OK08T>FLGlzO_t5fn{~kaJ;9`YxFf z*bI{S5f&G|HveFu1oTf@JBD=gLW?ePvP877^>iR}qQxOrN>G6mbeF%$&aAw#xCBbT zP5PlNe!k^MKqAexTZ1c5;4?g)Tv8xmk_nDG9}mWXDjQV&J32Vs2ZJ;WkpyKlY-#Af z8jY+$8i|M?f$IUepW*ulP*5TYh-3q5%mBwcZGeEdICMxYl&s-;PzhCH+L44U+;kpW z-8@T{fE%r1e_KtehybK7M}W72)8@pX7ySMnoRmg;bP{g-KN( z42bfnSio!L-mljxeS!q#iKIhaXNog{l!jJd{fO9RYY5{{M@v;dK#dVnZG=UE0WO`Q zjGlfGXp#XcAOgr>LrDlKpaPj%GE8s|+x#uGr0N4m#-V$qLB^#+o)8P-QP_HjVwie5 zqjUmzSrKJpkew;5&EFPFQ$$1^4NclU!KB~>^gn;35zvdWS;jamF7LG#K$g}zIecAC zCVC;M(sGbtVw8=~-5E|R9y!I|KcHAD3!sG+5(7xngfPlo_npgz*q9f)J1JWt}+RaL8kmHwm+9(+$H{@{rGtADYaQR7Q7!q4OD<(ie=UXzrRrk)T?+rgaN>v4=W*nLiV?F z%**s2AS6^0jk6?d@uip)c=4rSm?2BmM-_}f`+laI?lxiXlN%f(}W{ zRkp85P?1tyaugRYHcsF?zf%!ZnMd6`q==_ zWpPcRSqUndPITBPpqvJ}s}Dp%h@c6GQ$qVqQKmQ`@-omO8ikFZBZb(cA0GfR$E!g~ z8vy6{uLe-r*x2kjh^0poqL46(0fuQ5`YwxhuRXw3#Y?pWB&vW!=kdNqE=g3vXtL#9 z&d3LQ;(EG#So0Zqx+xd4Os=nKPU!gWwY@u)g2uV%PUr-e*!e_`!uYmzc6HoEf~1RgvA>OtxbB{-*|u`me=7V|gSORm z-vv)C3;=@Hn|)ab04291}{8?gSw*SrT2VsfQs^jYj?ZQni_cZI=*n;cDUXC!;m5@8b>_PnD7G&PA}h3H*%V7aCu4|@Q&J1Bq0BeOF{|LK55=K|@rx$RiZf;loe+odFR-Kn z8A@5oDV!XB+uqR`;)PT`u?0t>f{UxQ8Ycu4Uws3$C6fjCfv&1#i2Qwkkho>=jQ5uAF#*q>4 z;NJ|Y(w=~r3{6dRn=Tm5X?^G-CpEPET#V1*vf79d~lP^HmDB zJcxBq1pmRTPTHTTF(D`WB;PmW*2g06JijXl_>5s2`IZJq!_rxXl`U`ayXNWoH2LVV ziq##%m~<8U)#N4cW^$u7ahdl?gkF{7)OhSem(qZZhj>=;0fxPqEm+A}mezzt&D8ot zMV8@b6Tarsr~|Iag0|lNnv&}N4}YRfKrVC)q|F2cYzKEZYn}9S{a_FS&J<={y;arGd%Kj z%sC1n&m@iUe{BW;Y)9D!8o19xTfZnK{x|sl>e*ZRcP#C1JpOxR|Ags^{4X@-u%T50 zR!EY;noV~QKke55`c`ka$27#5Zx5PhdBlJLQJ#*AHQPuI&gW@SvO8#Q)VY({Q{~yA zrODtH+3aCp*$J}{m>O-_4un1u!?O=@$OF>qqL5xc=nY|Z= zPnJ3zA%{CN=30&LGL+@-=awh5S|cr|D4=7G#xCo8!W``ZcamE7qdWo6cO@oafj638 z>2ThV)y^bX8$FOP4ZkPX{%Rnn`)SRomOzND`*W_@h{qQpJP{{ixy4u@l4IY%eR!sB z`dzgpK{VaJM}38=lGERgYUWV6G1!B9k?)Z&IVDsT}gzP82MJSmqbRnb3G{ zs!*W@xY{;af~iKEdSkTqk&h7hiR_xx{H!8(HYuLKfcs@X@vRnlk9i^nz>RPMC97@;r zh1IBGM|qt;-0w}RnEo}2;YbwbF5#)pA@$7cD5=@qs~1^LZEzQ^p7SUhDq$mC^_J(J z-ZS>h!{>2OG%!qwOR;|z|K;a`#&Szd5({bCO`2_tjqZqK#aK9O$|H~3aJ^Q2S(dQd zYfiomX*@kAJ?X|y>;+?tCUb^!b255UNJ*vO z?Lq7{`Yj-qHNZVr3a_uIr77+u1BD?OB{PDQaA{qnr0?Y#g zY#t8l7dNusBmiWYJ1HMoPL&<{(*87fsNjvST4T63G5RUkEl;Du91YA<94xZ9b9k)8dIy3a86&Y z44ASD@v{KfXx5p-Pz@Rm9V$Ad#OFxs$$c@OtMdCigR+ZReE=1)^e4xg!=Un(^cdrg=pAz(igDj9TzR+J{N) z)T_DGlOsGf`++^yo1mj2@y|=cc-BV171*e4NV0U{l@*-ju6^=zcK|t9MfQ&f$d^6N z=7zfdSWtzk-`L-xrDidKI6_$Zyc)F$;jL?3c1kWN7z-`F%`c3oc5}X!f@`}K^5 za}Zl&^gaDRB?ARi?~DufDn_h$$ZYhfuekf()wJfBpBAo=O4YvKU=YagZ*I>aSKREUmFM<;XkwMFeYbqTbg|r^ z#Ew6|I5{^=P~^Lt{9VTR_Q!1SU0k?Oqq zb4pcN$*omtB8JT#(PB%5`*|6y68n9F7+FG3J0ITE?X2cKG~5H4vMz$Z8$ z<17EP9z;gDLup=n^w;UlX<7Dgotw_YuMXI;>tXFtE9AkteJo-VUb_|<** z;HJ8`Iym-J1I%ELv95?gwjR2G$~whl`50vnr7pi$QC?^X<2A9JrJm$34+twLIHAcU zn`w#^pbTw4c9YhxN+5Xz2Y1I8sxVDl)&m zSGtUnkMhmT9Za!NrS_l1{cGi9h9$4&IzIEHi_rW*CkQV!IP6iC$icTsc4l^Vg2?Tb zvkPQ%M!L?9&Q;SriVLO3x0Wr=B{3av>ZczBPJ&WT8R8sn8P(B2N zGR}BcIUk*GpNUssAr3E&O^7(+-P?m}C>YssX&s_lva;N1-#)VL=4}Wr5tRxu3K7hx*% zJh5QFpJJrHIc#zfN_M;=zpR4ajV}Z=^vLOr}5wc(^%IfX>;N=h` z8K=V+DGO(2a81Z>RjRK&JrT(O7-GVouTY7*#M{GdQchq4&nyWZi`EWT%$3(ki>@hB zLS;wSW2v5`lmRlC_K4Q3DyGq*R18VhQ$+7trNR%hJ;nOwSZEDbQ)4>yR?5+}iS+P( z;mt!icQ$C#cE79R^{~oI%dBaQ3lPHvyC4E%JH2Oego|V%&=q~EF3VCvik;P-oU0d@ zE5<;H{1=Cmi5@Y&c8BuQy@Eyc?nKj157gcEkrM$UB$y_mxL}cgv012GUr>;<_@Dri zQhwB7!+rD2-7!v(R#I{3!1pd@=+sEsJj8b+n_Gx*3seGy7gw%%330?X)maf00nk(|5GN%b?T~b>{(X#&CR^ylT>l)naPcZ(V2pE z@GRebtPDbg-1tFiq`b>?44;}fUTMg?T-rdw~uM4n31vz2csqcmATf0232)_rE<2uGEO_nP!P&5)P80U zd7Nbj9d8-5ob7?Gz52TPIR7l>>vtkAcIp{;uvt2oP;zq`*j(s1W8ZaGA&bpn%3)@+ zK6vx?45ux*3$(2)X7=V0`B2eY4}(E&{hf_P!rCP}{t**CJ4%s=6R`o21=A7>zC zXHT9E*t{5Kg16?t&1DyF*ovAa5~ReSeFI=YRBju`Qjzel^yZ_f4IW33Q+9_nnnQ`A zu~h^P+{&?_=$J!XA!LYsHBL;=kn;BCySlm>7#>c{%8;;iy-%!UcR(^^#K=Knq zDU5^*Z*ssAUvL;ORbngnVvZ|{hHmA~*80>RR%hiM7hnrRYN82tq|6~CyihezysMIy z9dPbeNRB{ir-=je2O3mDLPc+H3Q`FZl=Co>5_>Z4aDcn_hfz`rhu5c)ICLAyfAI&k zf>KRg2Q+v_GGaM#xLA_eWFe3z6^(*N8En#!l-Y|+t1N!WnRgaulx%4Dce^5vID?6b z+kKJKc}%n61)w<Ne;<7tQc(Ns+R$Oat?i}l(amaT##C$09C~C; zGBf9o9X3!|QS@_?1w8u~o=j}a5W7I8fh>3EQBCh<1rjvCxq7JC>LV60!q>swLdp~p zE>n+AJvt1y5C9Vwmz2Ifp=6(79VlJWm+BL5^`u8oQWy}A8)TJF6iqK46)v`Lq?%{r z{vH4cm&ODD&kgqX2Mg#WLPq|l4F0DmPQp!D`$BoRE%l4`!q5C^v zflzWR!-?WvdC@R~BEU$+&5aWP0Dw{y^6>DOo11S*1qBB)JMK!Iy7K0tqyK}b|KlS1 z=`Vr*kIw#afmi+?K>T^mB=&#Tq5ey-#mv|w+V!)^$0W_8Nbh%_?@3X&#q2w7$x(s6 zd14h?kRW+`^87;x{`+DdWO!Eh(oA;+^U{pA4noV5>VNN1{4f3QA588as}kwAZtzDh z{MUs`Bsw}et#~wEMqXZA84V)9DDyvxZU5;SFCU7~8!2~sIR5#}&M&h11R7$P>*;TF|$A}cnp)rYCJr;e7UOl8$# zR&GJakN#WkMZb$k{~Hs(#~~?BK9r)6!YU=3OsIcG@tg$hc*<}J0*>_Wz;V&wVDO`c z67*T&V43oQ^~)HCps$Ar+2g7F4S@lL#g|~P;Jpqqf5&2sOqh^8VoqL&ys~n#J6(D1 zMyGFp07L*V?(ZeR@z4Vq^!dHkc%*~Xty>GjnSL%Goef6{+4KFkK6h=(3cZ(k-9xj7 zo=^>!I-c>>7uZlmh`0#;qa^vC`Rkp9gA(vCI5F}`M>B$@^Cj%QG)34`tRcbO%#*Z-_ZwR&q)6rfWOlMh(`an z_4NPJ@b8grL8li>%gTb3pnnnmZ({0|jEsor+qSpon*$L+0BI2w?C9v||D-Z~g?}oe z|M7YVNO1kfgFkmze`=lo&*uhVf2R&mhs6J%8~&}_{wD_fw;EWs5bK*4$G-@Mjix46 z(-cWANjtAR*8joCbRu>^w+kjZE{F2sUN3U)h+0t40Dn(X2+MtY--!0 z*urhc&GoLo)Oo-$>LnG5jKy#OIH9Zor&XH01gwcLNz8 zEhnJt6CaJIxI(j}V&sV#CwVy7vsvw-lx;X~KS0Er}z%*dm35T59;3 z^-b+=OU7__$ziudET0x{2+JIBSAHgj5nDnvI!ejxX{p)7GSV^+=Wr$LDU z4vt{PxbT|x@Q5+_)3sH9fCEylKo$%zGOVdg=!8zQ`)C>p* zUwzjt)op6}t*&#{EDo5-R|kcQyJL^9aT=re992n`xq7gyP2`syU0ohw6vvE?8&!5r zY2GO12;0Xt^E-+`^UXJiTC9?^vlIAos`*M&;o8bAHA^Q}g27Z=6qiKB3p>m;ujsQ! z^TWD|)F9SmU}-eTSd5g5X5Rq!MB{rrj|Up{wmL-aN(4GBbg&3snKEigZGlf5DLP;_ zOkF~H^$nT8sGDSUu`|JP>ZP10Z1r*v-z_2;=R)c z?^=^S%#YY9!Mpv4Gp^Orh@)W()^C{o$m@E!M(TF65~4stLdT|7Ww1SWRuq#h6royZ z_EY5i@u2YZHaOpPvEIFR^^uiAIcayR|MpaO%BNXy?1#X|$mW6!&O5C5wI~%ySo@{`BEPJ&9+2Dwu34xX7{~=P zjAFNq4vsz6e)WtuS}<7WV5D?+ev$trJ%N=Rf@5BCI{MS&vC7igHGBPJ!_eVtsKb0D zj0V79*znc+!07uuFZAO2r<6_!kIv+d9X=AeT9;n@7vSLVg~jT@XA$q(ljrkG_GK^Uz?_-1*8P~u%Ln=6c5u{aj07;!fJI5aH}PS8)JfJ7 z>-KKG-i?+vh3;#)GP&O^GzmR{I{($1`!_ELr=eoNwhC3-#oZYa9w9+XN4stq3|caRcbbd z+2oTbGw1aZAC}K!(4t z+PA8>F-&^Fwgs!Y^vxC~lH;Dmb;Z1o9l;Etes7_@71-(+0AM&=?=4syJB*ny5i^C) z;2zoe>DNDB@ws+6-*f@DvSx9FeqtEC(&&{|c6t7gdQv&aH`!i9rCV@wH}AC0q-mzNTT>w7I~TZpJ=!whZJW<SKj+&yG)b?q$bUDht4`<-@XSv8m+*jB5OP|M9~(=D^AQb%hCg(t?Y!`l7g& zE~bf#?*D!xwnCqkhCcijoV_0S(q;|i(n-)Y0Uk2Cp@--IP^Tyob7<_)W5kGr>}SHl z!z-$(i6sTpX@XK6gJP@%Xn${5E?b#P^=`>B7e>9`pfD5g^O=|U!b4v@7;CDiiA_8~ zG2HqR6H&A+`==;buN)hmxB3%4F6gM;tf8G&8^-cZES$b?GO!6*SUEg;t!EtPepLzW z4DyCoH#bL&ZAqJY0_gaHyn5Z)g6nVKK-)KrWsVI>sV7>+*$ht(!N>b>GE0GS4A2S9 znr;Q@-exM9+6Kj$^^pKd0%JF-C^P}RT02TcetXJkx@BQWePW7G*64S4xEuSyE(=t` z$wtXF?J?pv8c3`^Wx8C{VH6hfY=Ac}r}x8}4%ZN{nVZ{B$+iqHPl{wANowbVW}^mS z!ulFeA@q0)oIPJS!xpfmi^hdV)@9wgp%!o?jEx~&BgaBn0@-Z6vlkB*0YkeEakHJ- z2RV9jr_`WBouiP4GGQ?bu{Xow-;{CA_m_=QL&E(%F9lTLSAkk?sx8=nU(5R`oqIr z^QkGiUKu>)p^zhDH(rI|*M*(s*gVbSayA5BLLz8cLhLJq-OLc14#l!CHqeV|s!+Sh z0}johaJC3fO?qm!>oc7naJc$JTA*H*bdYr(J6D-V=ArN0YP}BBkF(}2F3u^r=Yp$) zF57+{o;@I`u=VwkI&hzvM}*turTTw~uzkdW51^$9vCj+vc~xw&tux-pyhzM$`Y-&V zPobl`I&nP?dl$7bawnfqS!U3fUVJKjx538Rs){Hazp!T|aHp!WjT;coUp9C>HN31R zjx~4lW>N{Pg-uwyrl`$uFb9*t{lxb9MyG2;%&Bb>rXaDA{<4*7hJ*-8@9gy;^ZMBA zrB#3D3|kUL~N=h%Pw12rp0uo=}6r7b#YkJcg5Y&mEQE+6EXGeG*nI| znrKOHO5g9=O|OhA_GGx7E2-P|;{$(lv{v(LvJ+gN5=_0-?D|W0>ok+;fFkxr!xzjS zz2x05-196|@N63PZ)dT!=DNt6S?mcXeowS=2$v7vZ%%#O621E%CnlnSzb0O7o7_TK zZ;q{6opoTL$}n($LO|hY-d&5XJ)QEjJ*%V-34f5c+P~WQVeWg`oQb{t9lEJz4^hx7 zY;ibtC)3DC`SB&_Gw0P3YX!F3+2-SOnpBbyCEWQ7w<2G2#+scXxyZJCM)cZp>Qw{n z(DpkyRZ*B7s3e= z8F71K@Xl`??wk00`b~-VuOxC1xnS%2J=ps~s&d{r!fOPCxhazc$obS%Li$8OoW|Ej z)Ld2ShNm-pTa@N$-pU}|@AxO5H#+G+-n{ZT>E_k$rdb)2wFBmr7($BVl)O6MQ}G1JpJU&Jfg>s zq3=#}qKqb;0~is~Gvr2zA^z&Ll9`MdY&e6?X^YeX#SwU^e-5bAAl z_PCm;qen%m}Hf+oExY_Q$l95H=Tz7^nRDo~m=mc|=R8B$;9?J)})_UQZ zW>^uc;S4G}E>h&ml%39zyU>okT4}R=<`f(vb%k4;{N8#|yb2RVKd?C~bGua0^8DSL zW5O?49M^T{BhBFynT#S&Xw5YB5W#6l+o{WIr#U^pE#gM6@CW)Mo!50jCbOWxNyfDy z!h2IEJuyo3Z+DIGpVMEzjgP>7stE}4h8HWaG+ z=Dj6s_0D29PpmQjKuGHA)QJshtEC95;Hwimswt28+Wd5>6Jsyy#q{IBO91znqn_4+ zFstIV64sh?FHF!q1294)SO1YJ<^ZCArTj9JtUf2<0XzY zr2Y5yiJsF%zLQK2OVL&biw7w+XYvvP@ucG3{(cFbZBYKMpd@`&X+l1Apuw5+Ek94& z`VeEo1f;MJRzxiZQR*AQU21yI?&_U>p@1Kis3e^96@Otb`;ySX zrGCCN+0&?FhzIg*)&iq3m)PP4qf25ahl;O#-*@8Q235i@@>1(7M{Z(BPK6;G50szI zgS@zm8VHWP-!@IAuai3{zsoPQ;cE`%$V$s${al)(U@`gro^Deq(#x7|+gtAsL1WYG zUNhHmS>r6Jq_J4B+`M*%gmjgmNn8uSW(uiGOHa=S+NA+HO!dXpCpz z383gZ)a&1!4^xocFClGs^em3WR#dBR z-kio9gpf-_M7M2bAtp^Rk0v&a!}n@i29zfi4EBB*sw0CtoI@iadL?FK)6UV{y z(dD$j)vJ{%GD&R@MQ4(5Fv0fE?Zlk%{qU}{2lLqJNRwc&_q9|cA|Il&ZQ=VReV#Ju zmT^bPQ)7oatx0pqs&`d<`PgHNULDn^D)yO7IZd4OiISWc8Y1YeHMmTJ2X`IL2R%B5 zFKdl}5;%>>yTgwbM`x7m92k+sj$eXpz8{>OH-|zfq!NrcyrPV9 z!8Ys^)$r%5+k+ZZ`%ATd;>Aw-_|JY7m)n15l$Tylj(BM)O0l@=}a5UYidb_fG&sw%Sq+%K?bY6c{ zs2P{ll5J5>IYM=IxoL}fV+FU@WAQa^%{;i6-tJ~b6)|4-mUAe$iRod1y(r6@c@rlw ze0gzX_3f4Q&DRkh)i$$Nz<+XetIg_h>WR6r#yv@!&^K)}xZ( zMsx^s#lu6E{&jAa>LdH7NA%l`sz#Swc@=BM1?`6fwj0=j%E`wl%i9V4>t8pC_7c>5 zys>Utqd5-P2e#T{^_R4*_iN!|mhybBBlFG}dLz-8v1EOR#tSc)>AZHxM~gXd@N+XK z`;D?)gOwf7r*#Y68?7LV6;B@y$#fvb3E2_OeRw#yw4-7uu*HzU=XJsu9*r;No6j>aA zpZ}6&N7yh4IlMR@6xfk7jSbt5+zp%B&Wdz$kKN91G9-2R38g zrr6L_9WQjmot1X5^^2sw*x~E}+Yq7NKnx9xU=zQM{SPQ#eF|T)Qyp_UUt4%%Io9aC+;H7_n;oP%rIYsZR5bB6JWF?md z%s;yN(qVh@K2&RtdVvQz-@zwnqf8^w!d!*lp4sLzzT3tSpcH=g^1S*=iBdz2w<{$z zz}}KHmGe&QGnd@R@cG`fgXAKM^M(loXiGut@=P5i@8Wn0%RI9f%19jpQ86Uk6l84J zVbbXGM7$EQ6*dsv8kH-@NK|I`(S-3r$o@OYP9hd-+n3%2MV4+SI*lBSiGe7bjg5LJ zK|)5-mC+=s?1-u}>)0q@gfS+26@j{!sCR<$lZCh0X9hEgW2!)b7!#Oyz?!Ea@@QxD znm&JzfjgHO=73Rn&0gDC;2K+dSl@<>`ZxEhpZ#1_RI!=SkHOyK=Mlt=a}4 zi>Oci+jDa1*GIOr*&`dv*JrQ~A_nXboLhh+rt%V*KsW?PeOUNy9AZyfPa zfHw{@_8@%Zw;M3ixVGjTgM2@}gKX{?^UOYne&`v^*0f>$GUMw(%tS&{BRphz~v8c*RhHzJM+B~z2^09gQ83XBh(}p z)ez>?;39T&vjgHhO6W{RDp=eA6iDp!f#v72r4DQj861ExCqDjr zP2jxL!|C?mU9=`JvSmk7aQixOC^02T>a#6Oi2`Hq;+nL6*ss12`Te3qnb9%4p(SR> zrAj8#>rr^Z6p4+}^eCyDnG1C+ZVPx_D_VqL>ipEky+Bls&8anXp}qSD z45A6{gtCUAgyeLItob1@SPl&|Wd;0p+K3h^%KeRRbW0WS>G49f&5_lH9a7o#f4Y0?ptzc?Z!{1H8r&J&Cxa8*o!}0^JtQ~;2AJS5 zIKdr)Lqc#37Th5~f=lqAVQ>g?cb@mWkDPPr*8S&Jef3ehre^5gYp-7Gx4PHrUb}ma zHJv*%M-JEhqM)+Mc1c9^tf#wt>7Wzo(l^uF9q}@sC8w)r@rC%aFGl^xOf3VMJlW-B zA~gLJ)XEm0qEZ~cHN9cJJG5

R`f4^2$KW))bdANXU5){Bko>J~`?nRhBf^ytN9$ zFZ4cH5JkA55^~%Kqpo6OP{>+aCxWnIC{wD-gaQT?cx9+P+iy|glDq|X5>CiToYpSV zN>@2N?qrZubuq3VhF}Ljym8?4O&2>)&*Ebu&g?=VU8kI@mvH-n}-u9E__d zE_Hr%zR5kO&$@3z5Ez!SgIhmNC@HmK;##8rU9S{4T%+TAd{d4JJ~Rc}#Kn zP)a-+9}^!P6+HwQD!RmXP;PlUk4fPJ1GT6G-Qm$?5`M@(rdE~1BEP(mLQ#OwyAyR= zI--j+P~cC)684dvQzBm)4e#(~7cO*H9V>b8eB`Ic&|IM`^moCFE7mS$g;8`TtSj!k zS|5AF`KbS4V#OFvPI>wEYVv4#DHeA=YlI<;4`2qfmbaNhU1~MhGC#kmnKcv?8dvmE zPc0{{lMR6h7xwlb5QtIAZ*}AC_*lBV8tLN%YYtjTy3qK!_2l>B8oIbM9Sv2>3}OQ~ zRw)<8(YOf3F~kQSkUa4jpe5pYko*RlHzXofS6}e+yAE%(afu@Vy=&#GiHmkA+oa?i z+s8|kJ(N7qIC5!CqW$Lq4NFh?UgMe{aTqZga+~w20;f>FtB$j$AS;eEIjP-`Z!!Ft zJz(=pB=ZSHnWICAow-2f6Hne^)}ZsdFWf&Yeol|>*Ayk@fI{U~X9<6lyu4Bty`05#8~cT$pY=RMO@PFt{|L^dx@<*8 zN7NfxT%C)5TGI#<#}!3T$Pd20IX;dM)csLypLXka1M1?ddI8Sh{ye5Cvh$dHzaWgo zvfe-=_>ueo3eh?D2wobAXiO$_t@)d*s%X^MlF^UD1sMfrf!$c}dj{gQEYplk5kzfV zaN!7cf@O9oEDi50>gRgI8A-c$jQl|grm!d9#J8#L^U;NHs!V0+rqK1+3Ja`8V0~F# zlgI(*Nk(x^+_(IKSHRS*{d{YHUCKAE^AUpGMn}tcez;;LCi}CW(qAahy|2jcQ{UJi zJdTK~ddN|n2on% zCL3#+?-BSicK#Zxd3$w5YXSODq75ecv!sc>8oXcff`n*_77w#pU0ngqncXO~xJX0d zyA==GoZ{H(c^9G@sQ|igRy%^Ob z`6jj~Y;aotr`267Ydbf`=Y}E7-HW=3Is1b5ZXy+ktDae%&xP&>!{twFuXaL^9l$i> zYxMpq_~T+7$=A#FvIle0U!sAe*OUX$GZwK652vf0ImRQJoJU)?XI_gZ-;w+7ul8h* zQL#>I9lwcELtn6UkF`ZSQRkGm3uk9eP2?EeWIVxUzCGz*^xhVYymJ#T6Q=q8)7f(K zN!-cL#!SI8c*j;3H+KN#2bs=OA9N%NG_qpgEXmTyF7doru&IfT`XK5|;(^AcMr3lU zq5jcDdO(+7#StRQ(^fc&hSTmMS)0sh6WFyyPnY2bmTT!-`*&p|I?3blK~B1w52=HS zBZxz9u^(#de&I{T{o=4O(i%cz#*2i8hSJh-^7*I#I2EVf(RWqt2|*IJA0&^TZZN&b zXzl0X)*+6n0CKk`bUef`TAc8BR8^*eEuOn>8cuw8fJ-COTef3Blt^oCZkdmt`e1*P zaMVbzv$cw=U5F)PJr+3Iv>hxU&KoXX?lska6aFx*dr^v|Bb$PqjwFq4U0FMR)e&SV zsUd3<-Z2jMN+asa>?94-b@fLRi70XC`$p{c^%@5xF3BV3X)4~2&7MEE>UaoZ#u;LH zGSS}_m4ABF@cN()rYD3RM7ytACb+jP(U}Hr$y@2ntUWFuo1T?K7$B%Ug zW%Bf_j__n#3slp|_`~3eqVfJpZ+e5e_ZzRem8|^(tfNb5sL!?iPHZRb^7NE^<3cgr zlRn=2UzB&3J+Q9Uq8mlO^MXW#5ZiR4r_^g|BiVwsQ*zSeX10U~DWa0;N{8qu%!RkdbhE(c>0B+7`D2X^4Z z5iGP|5P`%zlzHC@myo#RL4})j6|tx@zo^xv{gC$c(^*|V?Kl>hOkqiK^rDK}$jge` z=?m{_fi9AECH*4_QtOF@j))vZBDWjKjEXVU!kWlI1T)*%h}~_%oM;tFxL0+94SQy; zf6!uS@An=^MGHcTzeetH>dgXf)9D62bS@d?03Wwo`5Jl<-*G~LcWBT%zeP4Ke#3Yb zJj^=ca*}RNOAa4tbx0Za3)++{_hL|eW#o;BPzblt<^CubEU8Aez(t$=XrXW24gY;k zpO@dn=j%=9svj{-kepw59mbTaB`Ym&haULKwfYlQj^-%zRmqSW=7c&LLV2;s-B0CU zZN5)Ce0GSRUo%&;z=W8x+|S}zP2I2>mk{XMVPn=BZ?pUQa?%}{lwH{Tjx zVEc|>J}&Ta$zVF!!hglKel(AaoSj~7Z`|`FlT=ge0EFofezYpI{J=YTz4?0ke(Vw! zMOBzMVi-V+j+R;6%UJ_{8dQ063T>7AWJlE~M#XH9DdpdhOP5D&d%ONqGUG6Xn6E)Z z^y#-0}QdVy^`R#F`>yIIbU+P;n=`MN{m5@mx4Hx!MwkN(`maDv;RnXaIAq z()k_BBO4qMu2oy=<}=|OlbzMVN*J85j|jmpU??QD_JRM4m3)F#pi=RV-PpD9;qEQ>F)M_4+&XEb!)=HfuT`W^;8yFs3olO3-Bhn)W&Xk8KtG7bpJiM^-f35{|K3p=<(38|Z#-7aoXrJTd->5nPVAEtxCjW2_d zS}A8HQWQ1AGua6a96fxo2FB0n(0^ri7FrcfI}xdfPs?!JZbF^l@f|C7_+1p=Is|hN_XbXXU%w2RnV96y6_myRVpTUvmPpeXm`3J zz;GL@k|tql4G4sgc0x#*sqIKK5^{T^BSCH;+T@L$r;d=r@MFU@TJ)eQ(zEh6@sxKL zt~ZI}YmvOgTA=SHfs1)bc47m^x}zXHUTm|PR9)xt&+kF<>HTtMQ_@Zy2+r~kqM^>F zg27UT)C~yIB;GpO&YNCJK`U9hx5eJe!{sM&8$s_eYd9}4H~T#k*Vykw!}M|Kjkj&5 zgEl=>g_9galom32>lT`^Imo1!1S)JlaXTbOB8J}hf3H5QHugqvC!t(70W;WJ4h>a< zLm%jRez|Ijx(TQ1(a6sqH7tkToVVP|+%lPM^*onAB1KEF=8Vc|&wO=1F~XMWpT5qb zw_59+NmXe1!+esH-A>mm3n4BMGJ_Z@kHOB-RESNsJpoew;RkCed;gmYuwci1eB?cm!TWIr_H>sSqI);yO$y4O!ni8*4KdsLd`+noSIn2=) zTFw8EW^5AOX2IILGbef>|Kv=2IFN6TWTc2!LnhA`cr)>a$a{)o>j)pNVYu=+sfH_c z>6H1vq7=JIKMk5DH@?`Fz{GtwZo{HQ;4urPDlRIc%5|pqKGD(?%Np!f+pPa- z*8g!F)cK9&+gc;ktSqMTs01z5y+?>tbH#MT~EAtx=-kl z9KF%CNUP^9ep@NpsXfdibfO&WGRxbU5l`f`@_A1=gX}d_$mxmP!ZB&{efpWo?ad&1 zkPF_fA%dW>tvzRGm6TWT#S5d3kgH6`;vBvZwxY z+ih;9{c|B5YLnygYM#$NfB?#L*m>l6C3&X2x>oA;YCd$9Gu{yYe_|({R!n4(?=3fiI0&Go+#9 zFoZb|+|W8MXMv&l4>JjFIHg?HXUg!q~%s}+nf&tdF|Dz2XuJL;;1;mM%n%p(ty zpmX>H!FAGd*j;}|93{6CzI9xweaw}qcjBMY5X900uXaGZM3&jJXRSM$4Lhj!YB(}? z7njJw>)Nb&VQ-U~P=<0b=D2Zb^)k)eXci&+>BatjKe=YN`Xi&d zi}UIxh9SF%U|RI9WlJCL>#bMmQ(2uoDX5FL+j0)t-UEYKT*XP0~}R233e z8sFM&pSvz5!wGoav>sh%i&izk!^)W!z4o3p3|6Ff&h>wD8;Kyi+{wM8k=PNB5Y{Ig z7Zr#L!}6ClNukl7CL0Q?4GtqzUn^T#?)ka)fh>7pa7rCwJ!`M8VU(rxr5dUaeipU@ z7a?JMBBJ=2(E2OB988cTc1C?*Z`-5-8^%G}sX;>S*O1}6*WrG#^{C=*HzEl-SNSP} z#BC+}Vv34vcxVpSz59cMjUm7G_%+x{fvRs}^W<_#-M<$B|5odLSKFE)8uF@G-@KlE zZMtJ|eDVbbaTLvVicMT?G7FqrJJw=1d@zfD-OuUr_Gul>Q}UnJ&7uowqjF}s&wx{4 zg4nd@aw1TL+>ELHuh%Pg%w7`}+w+fe4sYcDjPBh&UmHdoS9ma+Hlq2_^UvBF?0&2k9tm&)BJil$qje&*;~Uh?@mg}1rPBs%k@mX z*K~JF-o(rspHn#Nk@si&fIm35hE|#)xUBkO=@$;|Mvs~5irj=wKCo!5!kPBju>3V2 znl+i@thC=g%K30b*FXLGWWn7Zo769}GTS%~j#RNIosZ?;@BeXWWFirscnAyj=cq~d zK?u6N!8^6B-n~-rF68OKw+TCFu6>s7xyVPd0wt8#b+WuJbPXbQ4+CVUb(W-eV8 zh08!TP_omj;0=92(jSx2W0>n3zUOo!l>H&Lcx+)XinBe$ZZR~A;%EX6_(a)KkEhCD z8@P+7a6Y)seCyFy2@KUw0cG9F7}t)=HVM{`TA^a%(uQ=c{6Pqi{&CFO$6!EaUz*S3W2}sdw&cQwZ^^$0dHg6o&f}OQX&%_f!`X zJCxnK`qZmoGtil@we5YS-L_mi^?O&Fl@;qDgWu{n-$ZPaZo6)Bs=fP+`sU>4cO(B7 zTTeRfm_waK2V!vB!dvbbw?y60+}RtBNp)Pmp01aYd-F_TwU3>3Tr(-CS6JR=^}jf! z8PbJgciTOz<{JIzxRD&)x!p8tT4VLJ(b$)_dXSXI-G}`wG0x)j)y%QKKSE(FOH=h^ zZ=BcH>gmeO5cCFyClHJ)63C{0ODb`Ta8BoL2>(`_W9fJ`c=Ij~h156I?hDFTHbSuD z{wi)3>hSO`rt)}P&Ts5?mG2njnG!i6`0J-JsW-WRIRSn~3a903BMjaSzDMwfsU_A# zb}E(My*hs33dd$XD}7C^KCa}l6&G9d8R+0dx{H@dNSq>h?~Y)Y3kvH%KWyqHV&U1( zB>=K}DXM4^gSLK7fUDrmk-YWY`p3b}>T>ev>0|O?&L_{VJe@>%MJ8J(GtCQ!Rl{s1 zb_|+8Scz&=xFJv@J2PB?k=;U1d1@wp5kuCL>nt&riQVEK8|&dr-XXv38=kPOyF}8k23BwKk7#hI@M7oUZ%{R^v@VUbOf<@7zzUwAgMPt+`;en$W0w;k7MNO=4`TDmP1u& zi~_TxtJ+=J*u2n1h*|!q93H|j8`_;_p#RM23Z41uMvd*NXhv7Mim(;4nP5I5I(LSA0o^pf(}yg=fJqADWvIA>&| zumS zz+Dr=@V>L_WgIqmx_#*9t>X)E#E!q?*v|sX>*988OA1dbs|)uGtjb_~GHK2ApjUD3 zQdM=NYW5SDH64;yD;B4rNaZLNN*Tr9HXUq+Ct0)v>BLYe#P672)iCij6qE7a(pQLD zfuu6q`rYD9__M+=7zWP_>ztkPSZuYm{K-;{U76szb{2;AT_#jb&*Zs%6vJN_;i{dn zGV6uRcBPS-9AJR^WAbA_)dzX@29t|fmGm$Kx;erBL9W`y{%>!&S zV@<2XrytGw*6JNm!c;aVhsTRKX4GvFcwxF5HLkq`diAobnNZ}^oTteKW}LdfUIfoP z39)p!U7>6^&Pt~f1m`?Xi_qg+^GUYrUp|8$$YkXF@Z$Q>I7)Lph`NW}5x!!w1pGyh z6PcEo`pM_jvXroOmx35qY((b?jm}% zV%#_H* zWm>UAm_{h#VyM$St(QS>ngcIS^=-LO6PCm>&W!$2mb2fB(0=-oC)to=5$K9dTruN> zbINoLXC<3Hpn9zu?81r7eFSQR92CPCWV`U`tNDb2DFyAeh(Qvc$WnX#2%TOPx|#$@ z7Q>;8zmfZg&6VZ|#tf}l_NjmZE8hs6q*Mg2aT^~MDciH2SRw5wvP6+o6m5k^G-XUr zCZm6^U6<{~r(Z>ncR^Vn&2Om4xfxN}Wq$g0Ze<|fW)EXpbsdW|Tgjw7^J9YMbK&^` zGMqlqkzYrks2 zV3~4ohAELu!8fWo9e%{P_OID`GO={`D|IWo3JbXQM>@?vxB8ky#M>s`gEh?mW%6+S z%e-Y66Lx3koC*zr`CufoUkCqCS`B^i;~J>Nq0eE2>>%fWh4g^M&Zno4aJ!RcI`7j4 zez)5$){o1y3Q3QYJiJb3!vxn~9kYdB9GCQ=E9jH*3`HE9Rq{_vqgkPCYHTp3)#pd} zrj{I`P^ppehaV@=b-W8QQyD92Y0DHzt*>e4?DUP{z;t!c6{peXqv$>G1bTW5c=ze) zX%)+&lR#)_C~z1(OD~v}l?#Z=@5!xZbGXACqq!Z%$-2*}efPO3C@3x&PtVTcRhThC zrO|y=`;UcMFWo2;$L~R*I9*7rRJNUMaW!SOqP%^NJLU0Er0`HIXr4K&b($@YfAMb) zS`|Of@d)X~m5|b&aEWy)?RBIBo}^IGyhUMIP3F6M{PXUuzuD|i4n@{#-hA>_i#n&3 z`Y1@hbG-PQWweE}Z+&y7_MuOV2?lB_$WS-nLrJzmCVkH&bzjmZ5A#Z%s}W^PM4fxe&kYkiG|e))FOhL$C^MnF@q<8jb?U`;BI{1! zCY|*>C1Pr2JSuYyyK;b80Dr6fu_g}NxJ~JkKC{&NhknI6Hrku&y#7nf%Rf+49l6__ z@NB4;Q`Bl(>!VWn&ugEM#)8*pcy}+GghfO!P>>l3Fy`myU%Y&2sSdnfEFTMs7~ft` z{5|PGyZ=gz{lihTV5BJ8>gS|grvi3{;Lf7ZS)k~5sZsSetR0XSnXc*idmU$%18RZI z;AbunK%OCOl`Nq3Jb?@{JiE>?5)y~+RGOxywehKQbH}Qg2JI1Eg@%)@=}e=o1*sig z=f@Re)YL!rEizSyyBto2B5h4AGSO@G6w-PFbmotG63);Ce?Riq{C9>i8?G*-k!0=8 z({bJ3OaO|JfWoP|d%I`%j17xGt}r&>pIt0N4WDg{3Ma#Yvuy&cj6mfMzqS0gD@0v7 zWhRfA<+GjCmjf;c=RoJs-%9_01a7w1vUlN>{ke|j=V1W_qM8z9j?4g?rp>S1`(=?3 zi9WLP(Brqcm1^%d{I|+&l&;TrU zbf-H~sN#Kq^}I_}4Fp|~qcwK)KRA)PEEH^z8`z_Ty`TFZ(f{p}3-I4iEXCdbnSpBZ+HS9QkVIDkk^EIx<{N{=*iHutI%fmIsg4Af47gmbQ2?nv0sl6Oz z2@xoLp6i~5`w85IT1p9JeC8G{AW(mwC8Z4bH?@Z(9Q~cZv6z40RJe-A2>fy%y!OP1icHs{acH;g;q-t-(B^^qd3fVucd7^WZ`94_^Wm=-dA(X zs^yLFLjx*_=Dm(I%`H290+Sd~*ok8~XT}(lojUUrl1kR}Cg;hF#7b{|j7g8S(@-DP za1m*alw%%l)P==b-B@#?^ZgcE`_Z0};3p85z0dUc_8st!O>Pu_!jU!i;Q+7D-d7oN zqGL-y~i6MSSkvv8urPs5;RjXgaylWu*<5+ipkRQZOl2 zG^ln=JO+_GH@m}{xeN>SJ@kGn^h6-Oj+dPO48=Cb3z>on^zsXkQ>B{-! z`2*MfYRqd+dznr*$@ppTz|G#AcFu0%QA|z*_amuG7Z7mU`FpAurX0>$Dop4vy7Lq-~ahb-Y zvB;4!1VQ{{s&ks&sfZn7P^e=?tfV?8QU@ddMQk;gh~tf6R=@ zc*3bH1+!4=@cCPb(v%yiR`&U!M0e^Qi5;DG=)Mb&gXd)Jkzd8tPc*hN&@&4q2@$1J zuAdQb1M3;r%kT0D`_J#U1Ps3=cUEs_4}^SjfPm4%X1Mk`Qfk;dA&@i~uNM%~+}HHF zLe!-ktM3=V{FgEYf3s30`(-+i#0yLz&#X}P@#Bp?CYZt9j$I;e8Of7n`R#68Leu3Q z_$K*TWfSi$o-$p)c)B0Vm+@RJS3D4V!8Y?UtO!zHTgE!tyTxAdph%Zk z&}46xEW(xRSy6Db@YPGd+uJB|BLeGGm4v4!IU9L$_G%LD6hhPuyS}%Tu+_<7XmP~H zXoTCO?tSM)rC{Ks1C0omeFlldaN?7Ekna$;P6%i@S?%b$?Y^`3{#+rYk$IpjXAIZR z@3F#+tpy^*BO&(OlfVyGjecZE_akFXEp$D8?`pb;^tktf-$=nL$jiuo2skj2JE{BC zc*(L)j>0b%n^_#zI<=!xo}nW6m$eQnh-Inab)O{W7Qae)IaH{RCS{pUQ3 zefxH~ELMwr%sLx2F^Srpg%O0Tr?tB0O;dueUOuI(#EPO~64pEZep{)*zv&^bYinVM zUFA?S9L8%eOH6J`IlM^+Q#zvCL2wlT^uaCnJ9^0+f|LV05> zhLHpk7L;f01KT&{&)KKLv!|=~Am6DV*)NY5MDOa~{5=45gw7x?39-tAjw!;n<{j2I z^jxod^=lPWf>L_Nk5eh{)n48h8haSCi!W;l20FiajcinCM2eVMFthnVzG80a{mA2l zI_x7P1nJEvR(eZcZ_>H~;qh~abxJoBvtP&f`+7CWbB#+X`rmf4B$^2-zYXI+NOghL zqz{R6O83|L&3An*JCMxI)L5#Rg~LaLECz2FdB?9n--Jv{*?O1j0`_R2G2;SRvK)Vo zs$JrnT~=|7FIax?+H2V|P1L0>V$(R0zFm8`v3iW#$D=tZtH?5ARbFuPwdq~xQfc40 zwbkSYt~yICs_)pHW5}QIL*hDryAKk3hG$?-fmD~(co~Et%!65OxD|dmS|69-e%Wm7 z-G@))?Hwoh#MCd^@WiaY{;);#;K87HMh1WF#{CUej9F7O;R)3wg820Oq13WvIN&!# zx(}WT!uZKEw(b)v$`R2N&jX^wNDWUYyxQx+nWuUp?WkzwjHY}nu{wrvDLFTGPhy!Q zepvc4XXm@`B;H*6JM~lea+`^Jfj8-B^a^R5X=M#d-#X;NKW5?C*qCa$&qw@;0NQ-O3e#6PGJxlU%sf_=P++ zar;90eQFH5YwPRa0ogxY*V%2Yo~f9SDVwJ1s&a_2UQ$IY>mW+YgerJGNGa8F)!Z;$ z5;tOw4S4t7qZK!G2I_ zsN($ORGoQG+O&GzvHo5H9*?S-!k#uvawr)dNhdRi(=kY;&S^2S%A#eaBQ5WT@pJbj zOaER$`TMX*W5wQAJowcy`Iq4=2bwmUr#h*dK}u6MeYLH1+yixNgoxMdYq1h5 z3SK*!S;i!GcYE&MM&a|^GGxM@^jG7Hy$2V#YgzUM=#@T!EoOCsyOa#*K%Dk3RP4L6 z&1);aP*Ev>^ha^R$b?RhU^S`eEmKN$zQlv4=&e?@O388Ev2}QAy>o5#jMoBd7@Jn4 zdRBHxR{2qi^yoCHzZY8Whz>1Y`wMu70v^EomIyoQ*2h1~2nEZT;TOrfR&)FM-d3tl z;ww53nKO_Ewo>wb-TQ0j=mW#9k6t&AXTDgKaULH@86V;)yt#|=SbU9p|LeoH;wN4* zp-r7JK_4O{_;~TT?Q$Kq)o~59ayG1a(RGF}gxN1=7a}g4m~TIkxOnv(pP@gxzwL*8 zj>%+D+p4Qy$0a$WK`115=~{L7J?>_Wa`&R&eZkF5^Dh@vEHbY?QlU2vbpI;{^e(@N zsg;59tr@|NeTVk6Fwyq9M>(^mt0yOoEZ!+ z8CuslP`hNeCDbUSE}P1Of{+1m{L1AHI{*UiJ|NNt0`8yH{nQ0{d4cNEz?N8p)(IM{ zJuIV)1tIGkH%J2fKe^cQCs+mjU{1gSbE0g4rVH9pGcd1@9RZC_+WdYdXh4=w%{l=k z=#GV?V&!rM$ON@3g5o#>`faKoCNdJluAZa0IxO2xG7@s|CZdn4jA-Emr!0TtJ-O_b*|@Kv8Xg2o z(IP>UE9}7gy?U2`4|5+`$U`kVJC;Dn09kq|BO^*xRn`3nN!rf?@eGJ77)|bZ;?zW4v zSv|j+i#T8=^Qe82vd;7a3jFAzIXbz6h_=%%xQqIMmRAvpJFCHl9HQ+HsA>K}M<89c z8lDo%C98^PWW-5)nzcmG(O?b!pTWn!zE}jr{L8R#?U&LYJa`M{${$~E~ z*ZG&Dtks?=*Yl)d8d@$NY~LXhP7Uu6UWnNGU;XzVbgJO(+~<@qojaO4 z5t`(V7xXltzayHXe>m=cJc7*pFD3npg{}vb|9n1B@zEcX27fjcn6Su$Q~QL=(s$+v zAJk1%IZ{dqR{hQz`}?(@H)H0XZ2;-Z5#CJF{_k~Q$TxY$k(kFvv-Y1vp3i+<4yJ_( z|2rX=G5H*Jt{)FCxVt<$Nt?`Q@D^80M%kAJSjVvXH~x>fH#bPQ8Y+Tv3@Z_nydN-h zzfNd84=0tnFs`G#ML0jUQQI zhw=3ZqJXK4O;J|02mWWq>rZj5T5%UnStQsuB+@sq$q5J2!Z809xc?y?7=+Dou=kWn z%-S0Sx^O|RAfp0OiT_-;)x5zpCKJCnxwJ z!Kwdi2!9nn{4eDHKSK0hQ)z$E3gY_wZ~6h6i1a(W`samR_5Wg{f7SiJleK?0Z1gUq zl9rZC-4A-zPv++4baZvA8~;lk7K)hT%k{WghdG{x?USpfo8#fK_wOy8tN&e;ArI9w zHSu}v#tQTEgAPpmt~OP&1moKV|3fo4im0flH<$eq#jUMbBO@y9SEJ567F{Uk{D8Cd z_wEo_elZ7d)qeKTx!P^>Szj#GaQlBk1g&my(HNM%p4I>V9Z|n+Wx=&&smYiFGA#WM zUeCpI$KmUO!0Fx!n{LgQjKA&Hp?~j;{x4VoH~&)OBD2P7To?`P`VX=Ep9I*D+i|kL zg2g|~2n`(=EY3Ntu>WaNAW1;#f=6UlP+)5JpURt!0{Wy`iD+&Z*?$zlK~~rcwGb?T zB(fDYO&Bm}{ZpI_@C&X!z?by1jQ(dy$nA(c(0>JUhh6Xh75>gmm$)qh;Cw|AX9T@OyGw2$ssc`Geu#2RT?_n9@a<( z8SWzM299mFz&0Q;Dw=x4WzZ1Qy8vmZQq1IgRE6}=ASIMLj6E7@Va?=G)q{ry0Z70d z$k@)nA`9^4h{Wm-!oW!`_msg_RKpeE2l=|c;sZeOcm$NlT0HGE5j3<%xCTAuh8ZTU z?mt@(cgT&Zrb)oa#PDaRLrfY%;J0y+nY5*{8ah87XdcNf?=;FZ$^?-&j8^>b#a7;2P~op1X}R9qHFmGuU5d~mqtMG@CMhEYNdnBf1iX82 z#k!1vj3AON)j+w~TNzfDxP^Dz(a|b$JaX0MK*dBEV3*_oYTqCSLJ6hb9UouNY|=

auA1);{p2P|#kNojBp3uxMtZYbf&&pk%oAfOdUj+EJ);)o+h2J7Gi zs<(g&O85Je*?GI~31}OkEHt_((Z~d@10SV<)-Xu|0$XM!G6EGCC&`!#(8~kuKt(~` zFnOmDAS-Iinv)MDKqZRI)DfKYlSYMmY6{WI#)i9wziYzi(R>GE1{32ujwninMh
>MzB#ZdzqwVH%nP_kmYic03yip7%x&YS)IW+ z#BZNF<-e;z4U}HoDAsZ<-S@NHl9CD;qC2veaE%TgIy~0mWPL9L{h)bj$+QvyNf#`TZ zyCm_;0#73^M>7Z-O%Wa~QG3A?ZL=GsU2BAw+4kV%rb|1Q-qYQ{7M@CI@ z@XAPr4h~_Hb&`NPNmRi0itX~ zfHBCi3j>^^_b@r(GR11JN#d~rc;J;m5OE@b;gY@8z1%o>K)Zn23aD}cLv&hE&yc!U ziHVVr!JNdTa^_ggWdM?Y8x@^8X0LMZh=%4nWY_&ZZU%76@b91RV1P=UkcnA|ge6hX ze-|i-5(5TM`u%}sp4PiBVGL&A(#%7H=AIvLT@TKHa!oDN%coHz)(vswSYZ8w6GLgXrsGR*M3(O z1Yiu%i0sA>T0)gVkpnCj)jPpc6GMXx4xk%M%n3f`cxNe(4%RXCOtk{6Hx?aP0bR$G zUKWfOo+<5$60H>|s?-AFl@a|dv4Eff%|K+jnwAl4Vz7c-FE=mjV?OX?;5h*9LxAj< z-gN<7FxXTl7CAIhrf5xLE+#!MMP`!~=r59#5s#CN_WSaXHS>a{r9z*DO1q}VudzwC z7aN0ZsdaQ%Q<0NpECgaBt!$yBy>i|O1#1B0bihQBQA5=Lw{YO5A!Ps{BULRo1j$AL zIzS8HL+qaGY2-jxwWcAIB!CL854cGH1h!VRmPQVC%9D``jl`5iNcBb&0oo?+A9;Xu zto4VyiHGct0m$cRA)unAMbH8_9af?hjX4wBut=Ha-<5}oF2exoGTsT8=u||QfI?zB nfStW=PT(|`SKGO49QUX)JUQ~mSiEHqfIlTUHQ5Sj^Pv9+=~yV4 diff --git a/docs/images/quick_2.png b/docs/images/quick_2.png index 75848ca09f7fc7cd9b12135f7d5bb2f0ce6c49cf..936158d723c02207a82b92df92e7710be20bde0a 100644 GIT binary patch literal 105940 zcmeFZ2UrwK^DjIEL6S(8AV?G?NfMEjRgnw=f|6OuNwUBK3yUH-gMgrbBoRqUPLeYM z0+MsiuuEKE*}da=(|Par-tYN;@!sdTr?JQ0nV#yd>M&KmsT%wkeh#>-qM)n*5D)-> z68Hz;=Yjk3a9b+?P*(?T0swFkAPBYvh`|zA1z-U)01$p8{Kpwm=trV|l!-!rU&L<% zcQkFBUOKtiI5}Sv7rq7Dd8n*T{F?%p{wVYQQM~-?YQ%UnP(r-!PxfvL^Ux2!K**yk zCkNAfqM@LyCjUpL2l4L?*Yxxau8}SSgkXQjfa}0TQ*$@x2b!8{ zziIyC|3B#e^KdlsN8f=Vq2IK|PaY+7zTY0R2>K@M26O)Q&uagx?fKIAcR2(s)4PJ50?QwMm#zO+{`$LY_P28DA9Zw|$bogn!185t8&i-5 z!5UZ=H2c$e>pzwM-Y!@LpnN2+rgrVAn55{PN56mnznWfJ!a+U&K+eh8%hlG}=H;~q zpp-1HDLa}A3tkhuB_;|0zh(ZnM)$}5kR<`*#ebc1W&{9fuwz77|2k&|&Xh_}Zp>u= zI>(a>0MvQ_P~q=v>T3Eodf<`Zh}g|aT^K=4QU2Vnw2BH{}qq!-D^DZmQl zmjOZoA|gU!q6-&(PeuY?@H#+DbAgsq^Z^N-mMJNh3%%H@*o=$Z55F}rJQ+aoh?}{- zCL?ELy28xD%Xgjs#!U&y+fsL=@5()rS5Q<^R?*gZ`b-yO&)mY&%G$=(&h4eUhbP?2 zJK#-VP;f|SSX}(ugv6wG$tjsx**TwbKj(cZDJ?6nsI024X>Mt4Ywzgn>K+{WH9Rsp zHa;=Gu(-6mvbwguf!^CcI6OMWoSgpVivS?{8(ZM5fT#<5tIJri-6D* z%tSQA7dS;pXdh^in!3<&iM_f=|1dV=TN4?#_!AU^nd<;KBag&9FZwrYe{l9+W9;?+ zEzbVQ*uVIi1mpn1KN1lkA<+dQBBBc<7r;bvk>q!}NPh8;ME>_g`A53+J5m2L;Xxt< zAPr(-Vp8zmWePHi%m0@Xej41oUg9SJN+JSqG7-@LPyj2J878nr9Zk-k=}#^&RnOqp ziwBtNB_FtZq4rziVG{;7tg2$;W=Kr&fZ(8hSQeCfne>{gv%q^y0?K~Dy7{VlS$c>8 z<*C}2hrtdd4_hM}c?J?AtFB=S?0ayNO=rk_ADc6cq#Xwg>qI$-50$bdG8gEzp4`d|;|KoA|8r$f9NaLM z&9P^R2QK4*9XdRaYp8M-RL_nFUdJH2lCW}kph^*!>cCHg|M}0kdiNj)i1!!MusfTb zr+)Pk4*+&|B@BvU`D$gqnWaRm@KBnoWxp2urt@g(ql}DylTsgt=~~NNzAmEd=}fvn zE&GUSe4X)Q!wCV;`ZEq)zs~e4^#beZ=t zStNAz>o4}R+t2XBEeFbXso_wy(4StTQZc;{i>q^9wp@=2R7JUeI5Y)wH@qLmK2m&Wlx@XOEUmLaX% zKT>pulI>E|&nkS&z8_bgGbNas3*HlChiRnb3PhxZmQ5F~W|cbV3pr(+#JW2=2wRq0 zE2{~18r+_W&{@3M^7&r0_v`vNOfSk_0i{N3m&R7>xkoJGV$=PDE6mJX|50uPGyUuA zI5FzS0xJ{etQW!}?d)8jX_)k27iJSNxSEP`QQyX>JyK`rdy2iz>#$RiBRvd(`QRVo6~_YJrznN$JOHo#xRpK@_jyB(-=TgyY-wwsX3pV6^Vm z8EW-$Qv2QL;6jpo`n+WW)>P3%uF}sb^3OOw>5Xf>OQNJ_V{_b;mwnbxrWJ40#lbQF zQ-hZtNuTFnX%^F|5K`#tuY~8&)De^c%9UY;K5uJoMGgk0kXZyGa!!_}7R?a5fh#%a zhArzebp&J-B0b+gkD=WVl549JvCZkQmn=tBRYfO7`(~G9ibZ^z=?Q%1R8#O^+kEQY z+cOSPE!amWHF7SEgVnKK0vt4__V0YFjVZ|RHdMfqh)FxGP{BgW>b-pj8HKBL4K+K3 z5nqXK6kkQpg6*jx?$n`{lfodRKE!FlHH_X*T2V&d+nS6w>3Hknt{8nXd{KLeA^On^ zUK6T`#g}{u6SB`Uzb`I|Y;~YVk9@-)S7E--OP5jj>Xl5YuNqE2+zH^NtSC_l!>pR$ zG9$S5oXuAs_5SEO#(1H4yuSrmM%|hol9C?mP$%!}{yee9@0D!P6FuH!Ca9HFJmpt$ zMt*u)CTVV-e0hN>?uY7oW7x^ehNp;oc!2l-MH;byy8UggI3y*iFbh7~YVawTK3L&Q zsMY?{*o?76ntChxtCQG3mW6LNQCb%1N-;UEDZSh}T0+2y0VB{DZ9Y7HvMpBP-&S=i{&RZQpt@(q%*&K9*Xv){@tfIc6o zbdn8S;{7@_s!$a-J!2S}{Zvy8DV6+{SUH}$Zaa`j%`kG z6wsJSl9!AyzZ>v^4s zeel=RV)Hq2fdj>(XmA^K!P;U(JqtJM$%l&S%;y{kyJdO`%^62_0(xd;g&@bNahyGO zIW@%|-~PtVoP%qJmK~L%+WbUq(}3^k&2a{-OVzZX^vj7nKO_cexvuH1JktxYXras_ z2^kI0RpWhVJD#<|BCrun;rClOkq81(u%nb1l8ybA(Pt$lO(GLQ=4n3gcC}aoWwEKw zY}fZ(GEEnx2*uerzQML1xX%^JEL_d?-*JRM^P4^yjZd|?*mI`cGt&v`?+LXM6Ia^f z(-dB!WVxT;3w5}(mToTl^YGG|NYJ;?+K9KgZ&n$BYov&gEa{Vc3KL4NEmxX*}>PUJ(xrcIJ24 zkJJ+&yWBBuc)*+w6kblOcG?e4H1`=JiUS+4cuIcoYOrp$5xGYly(Tb7s&XRj=xgCY zzSoa2K@#JEwT~DdJU}v%j`jHdxLy`D8*vcL-m%CtfZ)u6KZa4xHks3)8B*FBoUJ?) z=uC}U*bRb?c~%J(d_1ut(AUTV&EHEez+M9uI=FFQ#Gf9dunLCZ-4J_bM5 z!%8nmA6|Xx$X7WOuV#ukN?dZR*-IV09OD`+^NuL6pFsd&I%I)K&Caqf9fSL=HdnGt zRp>HOuzXPz>2;B8`I;gX`7|MO58cGG`jK=1Bt1iQo+rzO88QfdQ~~y-@|VzR&5+eE z!N@w654XEb6v|vX{OIZSFRVYs(wylcB+x6BAJ=Le5{oVB`NVPprFtiSB z2J#&;t{-EhEEfD_lQXOEzy-{HGpJs@aB>$RH-6vaE4$}I_O6Y`&{3$9R+#7Mb!N6G z>SBZWLl_%;^D(A!ewIxA)3_nr$v|o#yO+DiN8DgC^i3Gk3m3k{N~-VgiLoCnh$i?8 zwoO(wONWGu%HBh_DaNv_f~Gn-X*)@~KPeFNU(ajm$XGX_&O*S@jPFsC9e$GN<%Fym z%~B`q6EM4g(dX;;IUWs|(B)uC7p#NoOy`TKv5F4Dd$Vl| z3;e9=ukAfb>MCCqsR>)AXx){t!NgqbTRy8Ytt`Q$7h^hF{OG<9*D>Ty%*;&6LHc>O z3KCI@liLKPP zncm!T;aeg9NJzF6(;x}>-jGLppKq90oOcS7yh_;KJ*rz7-YUh_EKnQH7&gdyRQSGlb(aTlT_4XF= zS4*}zHeHR6xPj|BnBj%!z$6^&FTpJYl-cDot1UTylvJGZ^{rSXy5mdYD@)Ja?sKSX z8Rfn~ILsHbVIKFl<5+9=5A4@w^m1R?_$iGigz6KP%;jy!Sm#PMW26>6gRl;H6XRpH z2IUEfd2FS&6M@UQ@zzK-ZZ`?)s;Ucq?)x-S{JgIGMB_HaarG1kx8Zu84FeWCzneda zQ_4%Hukule?NMZAkcqZ=3Qe|fw>n#rv0f%3Wcr-#cMsk@BnJz-8&UJfgwesh%z6+} zt&m46G+!#l8X%QuMeNls!b2^B+27YB9n;Ybhc|S{-u)@dh0%UkIV6~gTB-h=dT+-v zcBwWw^24ZMX5{T`l}N^Bqi<%98SYcUIYQG}!4-#4Xh21na1GQ+o95b9NI41Te44SN zRTlbHNxpi!@>a+~q*=7!BG4cx#!s~2Ezk*(M|-NE%1%v!1d9>6g)qJFqu_@XbObMM zeOjgF*9o2E5?v9OlAn@yIZban_teA0&LJt_V$G{(bS5Rf&lV0QvXX@o<~044A?g=3 zq>i^b`?5dlKt%x##Q_stw7|lB6m-16!7OxVN2pk2Ic2%qP=CyJo~rjpEs@eBqE+XS zQSB{etvJYHGY6Zy%(;~e)pL0tyHSrssmCeqE3xjm{hvwCuBA*IW{dIUJlxeKX4L+g zPy3NWeZWKuP1-iYGlYFG-#`ezWzbfDiQjfT|8gXiLSbmgASV%YWri=dN0o(ZS1Xa{ zgYYG25LO$_)as0hZ-&|oZb9_t?bjSuYV4I)zgAaP1qyQ|2(|riZ0?E5;^4l3=N+VG`HA9!1x)`U7m9A#YM3yws{9OdX> z{lyW(R$2NX$%?X1D1JX^{6=c$uIwk9q5Zeo5WI6>&ySWS8xEL3P756~gB z2d)iiyr~oN^rf)fYY~8%Osu<>>AK}~CYy(rvIX(ns=7w?g97o9LrV(=#cH4{Tcas$ z4f>-bq$PZ)DQyhAO%$94^;do^TMC-Wv3(0iq<(%=P_WZN5k)y_B7uoq=%??W7aeDgy~k#TS1;ZG1Ju)gQ2r%9Yi>9@)9#ST?!+U}RR) zAyHv(WjKyPTKHmjme(6+YKSK3sDv)-oD)OW!ZwRxPc3w7`(gWvk3f&Zf)+z>>09SG zpmw0@rNbAMue>@-msnNNqfyJ4z8T_Ci5>we+C`OyhO!&ny>L*8tF5fm=P|9|QGRCD ze5;_*+$|shU>}u745O(Rf};>j3rY@V)jyZw_D&~eHBiU9mcxgvwmC{^%NF|g!>xB& zNl5R1_Phi2tirrs1fQW8jwYNKn$5uI+4{!`Md5UJ_nY6p{~DycjLs4a`Btqt8MZ@u z(}rl&rkD`pfEIs(Hn9Iu&ou~n>XtnjO53A8R8-azxg^eM>ggtGk#t4zDMgwm%L0xT z;f7|4!fI!(?U}6K9kiQ399JHlzMyMz`Pg$gLXb&au!PjgLv-)k?PZ=DCjK0H129R1 zGNx(4>0-;=rcZ|XJS!fcp%|@>e)8H_uTh9bDeBIaRY>=ivETd)$`8)=4XD*{g!}?W zFk?M0>ibC2B9+2HELB@>a_-nIF?zus?y2rH&Hi`Q)`HnU_8j0{&ajeBiZv{RZ_baz zE=1Gt)_F&+rj-hPsd-StycEt~BL0>58suA%434%q%ui|*t4!KbeUK`kifWr5s@!-s zG_CCGNpH*3=q29F8LIY*$V`EU&i~Fe_7izT)1pQ=P7Jk;2Lj84e9O{<_w7Bz`dyF( zj^9h7j;=pFqj#blS+9PZbZJkdXTZ~f3d#=eSR@U?F&JYXm87?zrff!-bdzp*F)^Wx zLtF>$)vdcYn(K2Dgjl+gwJ%1*P-pQMX*bhgdL>qnEVFvBX{Dwt2iZZxJ0oIQ%a(}? zbEVdhAfl_aX+{3zHwz=8B@^mO*lOD&H1e*|!S4AD;Q=OyA>`N|_ZCsY3B`q|o57VIkd z1OahWfKMC4oQcqR7KD1Pm_|KgGsWX;qDK;q!PG`iZs*G$ua(9}J^>L+A1t5xL+DbP z(b3y5R8N!l%=avl3A@n&j77*H#iRgZN;gH&*;mMJQ;{W5}{StuMYdlrdaP^4`KxA=UD3u(IRWEt<&O>lZ@U(r#&Ft-~1VNQ>ans;E!# zdFQ!WBQLW@wrR}bEh;WeKGWix4!Dj{rrO{q8sN{Y!vhP&(L#P#uri}qFHepSHTUvW zV$+D@MagLC#v~%XXgs$e${fW$osTA)6xt1Q55LDU-9^E>-RiNXv_YiQ&HobuUSl+r z+zl)bzh^alk%~8=)$=0R8im3S0Tf9SXmCZR@uH#8xa;sJLqjUs-g2skDEXtHM`D7S z6BCUl5>z}#pQSph z^|@chO4WQIq=o=UxkJ=3mOcQd*08{F-V8n1w1sJ6Z=#orTX%!KNu!0h_jL8j^-V-O zqbzxO@B0MC>Oa`IfcOro@z?M`ya*;34>(7k-<{AT#Si=g5EV3d?BJMDW1=Qp1f9Ds zd73cyzTwh_o1LPDjciu|Z+#9Se^(;2u8qPfoN+LJSu42zx^S*@areaGY%uJK`z-Kf zOhO=U-AqH&VaY6#E`DD^OoU(0f~Heink}Wu>dptr)xh#IX!;4Z|0hDW4|T;zE;pGk zuX`fDR?>I7FvWXfS3c|4`QV02{_ zswMRYs(m37Mw7deBEl2KeIfo+*B5}Uj4LNJNc=@2R*5Q{#1#YH9$)R7nyxWO~*Ge zY)(!Jx5;r8QT3>_g(Sq#u=tC=c4cXRYz{lh3Sm+-f&=EWX}j2CtAZlBmzfzKaQW@Z z$dPpM0NL{uyR7-$`IGZg!Z#!D#v@ddJuK;YsvjBR@hBE%Q!j)bjZib-k!3S=vEv=kzh7Xf}x6)|PY4NBZl<|%zBVJJ zZQiJC?JNGCta_bl-50wnpzq1Y2f(KQHC3;R+x+z7O_jtQ(u(A5D>#cDPdnyA`vZ#? zRNe49OS*-I7jry)H9~VDTuLf}w-sCZC#a7UF#6dDY78$D&3M*U0}YV8YN%U>zM4>! zlcQ+MJ0-<6G1mUBMpU|*OFlB$^`r8hc;;zB<#yYd2%Kbc{%T}ph=jvq4GMS=PW=g# za*ZV@+0@eEv;IuWQLT-m=FT|J`YH`~7vK8Tr=e}gdvBj4l8bnBu)R#|;wWh#@w|Us?o#tb|RwlzxJG7G#II3a8le%jaH|Lbh?EmBau&Pz+ znKW*agUs(LRvcqb+Ir7>QG2uu7IL?!@N>(Nee$Qup>lPoW~Ed(=L0%3pRYS%+nkh3 zoRXLUdF^KT9XwEo{E-k5H;yzw#-28vIrt6W0aC0jO7Ss98P(Hzmm|agMti$T`vGct zB+%ZLI-Z~EMoludC(qzTE!P}&^7Sp&60G#u3w0Foh#AGyQvBLa(t&lsDz#`vHupsZ z$&-{9dnPJSA9u&24mZg4i*4;b!0?H+{FCO61o_iA^ph1=li=;jt?e^gEb9P*73HzH zFd7kI!aoQxUob|ts=Yj`Cm&aP^7QGQM%3)KJIYWQO}dpO zg5{$%TnFc)VhD1cm%`S!WT4oTW-{?<@bU61cXrKqo9lN-8d&)A;CH4QHj7W*ek(13 zjcV*ckywEXD!k-M*Wq>S!wp7o!t&<(CLU!bPX>3@ZlanDzf%r32=xK~#p_3UWh_wA!m zatm9!8($NyrxDm;YQkKa+xU{6Scb*BIAxrkAA9%4l9G|ty6&xWO?*fC}~T7Sc*m<#6H(3oe3_y zs5GcmmTHB(tnU0`IrRB2cl&Qr?;stm4=TP`B))i_zPb>W6c9dKVbI?7BKGV%I`^pf z{2e&d*6rwxG|5V|V{Ngmw{KA_Z&9>oHYQFr+}V?_)a_!_AUrdnt}%>OsNU?0^tDRr z#obBPx&DxPVcd{GZ$rdV4Qe$CODF-o#8_5O=7y*sk$j`@lO&w3N0|NTqaM-efE28L zX2{$^L$SslnYqO&(f|)6pA*7+j=r78zo-|(v>YY>l)cfvAd{bdp=^^2TDfUp@H>8m~xTmIyp3TnlO`gRqZ3jV zr7y(>WCRK9@r4Q4>AYo#4KyfnJR(0V_J{2m`?2AHv92sUux{~msr>(d^nO__<>R1* zesdzg1D&$akHtz?1$wchc;Ffa_xk&_m&Ngkt0Rifi_^>1Wfm1G=|@gSUt^h(42=(N z8rkW(7QE(Caisoq*&|M6HRC$N0TlzctUjpEx-TM6d1tASC@v@5Q{7*`wC;oNng936 zUQ8XZplGzyykGfTxUzXgLyb;iw-?3DHg}>>F0*dtx3sl-{z^}rsl2!82~g7=TX_rj zH0DH6kBh(P8^K3>-LcBFe;2D+o$mWH&nTm88cjASr8%3?D8WlhdA>M7yT3psrWw?lK+ zJU1=rzXknD+Zo)M(1r4h96z(Dpx+x1+rGoL-@)m0CXQ)7)*7N~^W%=zsnED!0yjHC zrzq6fhgV;H6H<_)&#g;NQDt6c;cCB4&QCPoK<3vEWwIWDFgDOgsxy?O2Tk}cN5x1C zxU5b~{ve!YU+T`#ym^zZRQlRspS^ucLumu}Du4vbhB|CpNgIVyOX`ahkMq3E{#F+! zdyVg!e!WY~Jc(PcZ-PAFl6En?!^QTv2Tf46QYTeO<#3+DqlarZymgZ1Bld*ih(U!>Zraz z(!G>L4~{I2Adz#{?qPSDAfoa=Nggldb`uysI;Hn1ak zIq05JpNy`XJm6+RIZ#w5Gj~qWa$FGY@=4ENqI)1tvrhD;=#%?Aw0Am5uLe5Uk@uUr zhERw3yBLyM00FiQ!fXKha{Z{ajMJ5K}ArWlZ>7I<21ljL3jQ^2^AF#$Hx>!h^OfPLk?X zf;l{{ptj}AKixzsGk?6*wSXkrOeZhbcQ=&T#mKFUIv#rdh%GDKKodWB@#^BuH-PHf zIY*+@tZfy3qJ?)*D?Gs3HdjmucPTF0EnBOH!06SQ^(!Rfx&%&a5Fe7iE-qgafu+T5 z9yy`waFl54;Ah8=&mPw zH`D3e?+Kr&hl#Kn!9DCbB}xSpY90or_f1k<5LE0ct3ZZ3c)vwb$zHQTvb)cQrm@xx z@eUv~a^)O68OGlk%eh!n2$FEJY*~m@`mK=KzHlY+3(fSu<8;Z^g(^i(G+NZU4D4@d) zd8;yR=9WWUcG$S+4O@n*P1h~}FINtC{S8(_>rC&p=Vf6WWmJ0~%haa2UeuQiEm-*Q zxZ)G%-k6gag`5x#c6Zw3|Pk|0I{rT$9dF97*eP>zM z2(y94h|!N@7?Cz8Diveies}q>d@1Ewpiizzo!x*$rKh%mYCZqgr{?D^C>ZxmSx2Qq zaAr2uOTbAr(JCEzwntCwoO?gbu1y3M*2T7YjHJ8@Y(4517c_CNi^=~|3v?T%HWIbN zS?3e0i{Ld*SK#lBhF8uH1ACV%GCob}=M49y@iKlA5qg*TNKgD#Y9B*@Iahr_ravTl zK_dX7n!lNL1wH!OWVn2myvLPx=5R*9KHqSy)vb9@p+v$VK~Z6>#T8Jvq#zOup+ZwU zL>)(#A6A1H$vg!8rvK#|$fxFkLE?(?Xs_rWv1X4bdzXujcpS9Cj_vq~dS5z`ZXl`a zuS{a(=2yo*4NLT>Cl1>2J)k^(v=e6^zz7rT)SoKmeK$%~1m|9e3S1PSh3C#^t!q;= z8yXZY*okXUkL@~e-QxWrtMkO`%fL1j>kDc5mbnkUq~I(%u|(~5q+gP7B43}SOjjGu z_$iU~NyF(2(TgEz{^ogMvoKq_m1jaUtXJ4G{3WYVLrt>c7_a$RDvYW1(9ZUVgTkhM z44JuQ;-0dw){7XbT1H#)RLlv<{V5ldevp3uJO>HZb_mA}!_2lN%5wb5`DMkp>6#Ka z``$ch@x$&J zoW||F34UU$8&aY!tjW4I=GBi${0hOzl`P74Ym#!zA<3lz4s?fhMBeLsuc@x8tF48Pa7(uR2vT$iZT{S;W8(aUfM?HD z7{QsRj`nL?&vTl%d+cGrN$>)+1sR5e z1Z$%LH`gk?Z)bSNPi*Y%hHDQAB|eQ&q`X0YiRnTeY=Ui9G`p>iu=tIH;Qw_ zu4GIxgk5W#YGjl)Baqwx8AwH32cQ3Q8o+uzC7rLU?ATD|iZr9t-oq_Q~z&GV$@L{x<@wAJp+_04T(C25Wa zTKCy$3NTKKLghipP8W9K7xkc}fkk=N1qyd=b zn|wP@J#2XMb!z-me*UDL%IkaR^`jE-6&HUHM?&iJkow^h<;Fhg##5CrAZSLMo~CtL z>;9c121$C&3+ngu1%+app6SJ0hF=S003mi(Kh{koZ2^MUKBM1($VQfDs5}%m*q2eb z)w8QJ7M*-)zKo4%I?%@$)F2<{p0U9uZDs^YQ7iclf)f>4R;N0i3Q7w@Wwk!--+grW zOV}-9LkOHfol{xjH|_5F>%-s+yj&1RY6vbj7HF%aF_zhczK~)R10Vxz&$eQJ6u9ch z2wx(7Q91Kcle7iOfV6_KVj9|Vbx;9oDL3>AGRsEhd?xRibO|LevdXth60e0 zMp^#zLkN;=P`9(t z+;h2M*sIyXXPZ4w6Q@b_dCOykX+?LouW0SqPuP5N0HYr} z4_YoX*R3wAu1vmO`A(i&fuko80mRgLA>|S*ZiqYq~Sn?HZG5+0PI98}GE>%ciI|WmVBL`h8`FzksxOv#= z%d#A2DE=3Q3CApl`;?B2#RJ<1$QFL&FYt9l9Q3@tzB#j@0lGkM!ErzZ`pWhHJ{tjD z;9TX{#{+nP7j&KDLVy8Ulo@zHTLY(r2aZmm^KYRWws@e612+pZ)l$Shj0nKJYS;tc zZ0$V&oocD?fEWlKI1z-N&@IklEU8aZkb7XXSS9ES^k+I}~iDuOZKTIsQzk`0M8^9H#;(o%^#V5KuN?$3ucY zAVGO!6FIP<=v}N>@VPqlgc}b`uKhb9-#~c|G`Se{DG4YSki#xg>?6n`4heShE0{H+ zk$)Nw#Xr&6+`7pRHlr1T{3(UgfZ~+?8rT1+bF{M`B_3f-%ARl_`7#Y$j71p!DFtpAtKB@DM=AS7sMy<= z4u>93a=|%4<`>= z?i0Tv=GKb#!hpYy|49Oc4i*G2t9AlTQo248h6&^h_@9ZVo%Qe06ZC2sFVhn=4g7Wd z4-&BYOi&V&Hik2+z6%H}>|*$>vB|rw)MtUn^})gVf#UFnxs3EZ>xRqF9R|>-P*nl& z!}DvIVR%53dVlNtC)-i)_8i;nvFpihJ`&E4zDp@{U(mwD!xW8d8z><{Uku>lkA4ba z-M)15l&5&6p_Go%744Cnk=xJsRnNPmlvE-+NN>veppHsi=6jK9XsVAtPK|#^X$7>K zkX~x|TTW~=TE!m)cR9<|>0SPi^p-(eEw z@BqbxZAJYpltj<@d%;<=@8$YD$qpiK%$~&da48Hv>V7<^ASV#ad$X5)%6(aDGjqti z z-elNG_)wk8$N5X5=V^r41C6hO4*9JhN3j>*Y&0-?1)Ayj2!GzSzz({YL7x!Kc`x z+QaJj$6tLilTzN^ZhMtDA@(6_%beLY(7#W#7pi`qRww^iALHl+k;@o2f|uOM8QK;W z&M{0T9-n-xXUNR$==Qn;NOM`oF`ef~OiMMHa14n^QaP#qLdaN+AgzbYN^g(8zBg;_ zkunE~`P!*}D>_Cj>v}XLI=5A?JbadDqn>SlTLf_0+arR)(iemZu8%aiV z0Wt7G6g-Wy(B$P81De{><@%|SneM)yp1RW;6;XG-F!SEj%%kWXH#V8vo2}0W19pYj zcaig+%~Xo5-a*lMc1JNrV;d}iD=8J?#>$p1f)y3fF5JO-mJ_itA3xB-$X>3{b6Og#wH<;vo49ZD%npK?9zub8j2kThR>9-%Erae;@D{E%XIg1TuBtB-03 zf4P18Xs9KH-aBH@U^_BbYN9ypSnT>PVS<>Fzhw}O_oWLqwj5;@srGZ<$(s(Zf&w|9 z60~g|(g^!8VN`api}=&R^SxH3#<#>T<{XIP;ApQT8n>HeHj~MUik>}%hI|R9u5J>E zN3JTp?xtUvucqEDcj}(`(tWBuYUsYQtm|VkTH`0g(U{KKd#M=w5KB(^z=4wFak4SC z`o#|SzNX?^;;u4TxeLq+tDoQ^jBrIQl*4dfRKB3dy~D^tTxW9lqa+(1rDe+eiu>7& zS}|hMk_RF|NSk|1k5CP-%j^qfR~Gh_QymVIodX@qKGzSkX(~uXwwcwv1)3Bd?s-R3 zVbT{bVtp33o!S!7Ww%F%(n}TQ1bEBtD-wRa;BOhK!$vFqb9B%G{$mE-rQQDErm~Fe zwj{%7;fKTtiD(L`=H-~?cDR0}R=NAQ#O!=FEak-I^0hZqu9ih zcB2AKRgp!^etD3|;Cn5jt7KkQAgHg@b z0wLj4*&R27jiNe;U5s*8TalD{SjC}whAHK1qlfCUzK@A%;+ZQdV$EZc)-GCKxw=MF zmsuskhE+l}v{WOcs6%t&Zlg_#9WusFT#}_srUlCE#?HPxa!7=b@7e&FRT?39fELSy zN@^~}jCLGvf3g|rXLEgAu|E~F_w^CGag8!(*PTwIOIY0yxcJfdd2;mK^d2s7+t)68gf$ar};8#$WbJW?0{dq^df#^7oP5E<Z~(BPb0tR#bPR%t{nS{Z_^WHA?9xMS+0rNop-j4@IV_h9CYoh zC&zd|>9Hy!OiDwmsY0f*M)5rodxX#KWCofjni1T2dWYybq5nN$FfL5_JYSadw>xN^ zDq8NrsKJ>c3f`B`GO5n6oY4owO;|)}D)-46v9ZwwM1L$Onr|q5jhl*R_+igQd&{G$ zv-6Y%D3y1cNDqfdW^Q^zgAN^3DAcuEsl;rL2Jh%Qrc6L@YtbH>h3?Qyf1=J@t~fAaM*C(NPKasM8}k5Tn3WJf--9vSnI%I=S^ zv$fO<;6SsJr2e9vfnzjU2Ik9mtQ)3WXr{{A(;jyxMAzv+&yt8Z`g_<${S|~U#tEI? zlG}!)bRU!47UjKF7fY%T^^_n?nJp}i`*}c+rqZ>8kJO5P6VC!xaxqn^?e1#G0fm6f zrY%;~{c28Y(zAYIaY$^aoMC4R?MF%i%QfE+9-^|p-LeSbL{l`gL#jr%5jT)W>j=?2 z^=v!t?nDu;(+khy1UthxY}`-cT|b~YQh=?sBB=$YXvjTOOOruEG**`<)|Q2bZ(%u8 zjyt4@=w?8itGBMUr6<+-2d1N?cSYv~(dZ)AS07&LzK@xaD2&@A4?4JcwIk>@J2C!i@lA#k>j20H*6WO1_1nf1So{bJDAHw3|ehBJANe<$?kb}xe_xEE;YM`Kesu(3xB44|$5 zuU>)%`oGaZGwe~Hl7ZZSO?L)k<$f&U5XhyUzX1vG`xyUYZhss4#h{_Dfm4R!9zhq_ ztvU7=K^!O;L#Gpvx z!7TUJp#J|lsJhB?vnQ+v&{JFm9vH}nfvfV*gHzidh%6gDItC-WFWSIRktZAzciYn) zR(wl6`tQ!udCc-yGxv#!ZyGV@hUqh?2^l!I+ARzoTMt8bQj22(| z+yl>lff!z^;KE;6l7HSoUra4pg{Bf0pG&n1Or2YY{&o z#Py$8c{kreIBxS%9g6B#lJH(QlCG*e*3 zV{0ky)8$`8&)lZecz;>EoBw27e}6w9itM7W#&*=4eTrf#Pzk?HM1gKmBr@;um;++YC+AvDV0tCED=)XRc~4ah5Kg zTR&nYKIbmIEjoaTeSLm$-Lb)X_Knn6yNc1fB$&hRmu6c$1GaVLdIt+1&=@YM+}@`2 zb8iiarpV>*sOSKR$;o^u`PBIOGZv#bgOdScJEkP@NBc}q z`{Tsw4dNq41wL-D3t;Basz>mu(K=I~`EBhsS%Uhzfw*_*iP`CQOZrzilguxsDTK(B z@XN>rPNvD?&>CBXDC+ir?u^cAo<=Os*;6YDd=@?uzg7)_XmiZPRaB0 z{ko8)E7hND3?Apsi%K>GN$lsWTW=?Y^QPv-D2L?ex8(1zsn-4=5~#Jmd>;yp4uj}> zeZlC?_lF|n-4-^Tz#W(EWubt^aIul9r)2TCqXmtT3b1b(7^>?krrdEKXix8el^wT2 zF^^>h;asb2+(sx6zc1Q)xt1~fGGBw-U{cRW(5mmWePuzK$0H(j?-p?>r*_27`EBqq zU^V;Nj|DXv4B^gzn+XUBG^f;gAEnc~O_4-vhw49BQHJ#H-+9J;Trd|wcq>1m(EjKN zhG2o3)d>4w7|Tn4`uzYsJxy@0+rVh5)stsvydZNzpJ3IkU7067z8CP>emgj&-{b=E zQ!8mN=!%)t4^mmOU}F~(*f(#K6(6eSKuDf{uG9ZwKZLQO&b&*>jp#6E z{-LC!Saw5Gho~3uLn6e{p6%3^5H@Iz7JDl5*3;)B5K`X){kFx;GnsXVL#0o84#lM7 z%Wn-E^Hxk>+_imdVHc^O^}=^tSG)Z(qE-(D3l^rqS5+_Wtwt z+v2J#oEJYj^$oM?0x>V(&?6%Z)0|BH`&lZueOKB_p92Ikn&hcKA@iz2NvM86BHcPN z_)54n+a)mEt$56{BkejSJRbvXG7%mYL2#SSK`GYv*Cr?(QAnq@n_jMVZk;)ZHEv7t zn266j;?^sq8_^SPjf3!>HW-x?;_4!mKnsi(a?jVtXMXaYDPqz=?S#NFQ&fOdhR)te z-s-_mLQiVLfH~@BRqeS@AqfsHSTNc0nplUsYrKBjSI+2OeYqZ1{uZ7e8!fv;@dg|W zeiT?q%xJd3+AOVvKheJZLU3gHRoHyP z_SJc4Xg%3rog`ZE@lQd%Vacm3R%t^|%Ul_!_ZGdcGy;Ne>Id%Mqf$v2YSnf)jX|;^g3eL7m?U^=UqOci9iPd7p^=0cumj*HL~F-S#1P^O9Wbx z)Caql7zYv0ff344&nMF`hqHQzp>FFFR}cR=$`9sznV9gi8(1Pt;g<>fG-=cq7;M6j zHzq|OKVLU-0^$&4?j-2vN=Zp3Wlm3UvhO&-$sn;v zk|j!(AVDOFfaDBH&PXm|Axe-aAUR4#lH{B-MUb3RkwZZh2ozA@t|MaFRco#Nt#7^G?-jFzD%r|U`Dogg&KoM^8(xJo#*>@I4$cXLjS zZpM+DJ&*+4I}|gAj(k|Ufh;OP&O=*2G^yu-EF_2-y_Bmz8_X>sP3B*Cs?t|X@~?Er zFJvr2_-k+)p5Gb(D^M1KItmDa+_?si?oO>R%*OF22$s2ks2K273ypEbF;*zX<^5`Ncqf-iFXx^|yQgJANJ#VyiZL)P)qEDO8P%z>L zfTra7{Q9ZCKmj_zfI=b9i;;Uz(O(PsGD-x#PQx~Z6?9Uq&mOih{;|g_*yS8Xl}WPS z=y^hNbPxtDBhf|OKz3is&n}m}Ym$PNQCJh$P&%Y=e5?t(N(1BOz=x496nXoG9#Zkv zvBYTj)i3*q!-I0yBDkS4u;3)<6VfT@z}ViF|;oDwZx|5-TtetpUda zfxF1GUe2N(B1p?lby&=F4(Ve_&-CxF$T#XRtTABSZt88rMM$*Of-AvYFcqjuh!|QB zeqU{DM0ebc{WDjp+Sv=CM^yxDtO^h496rW-lAEW*k4)2$Tzzb{jJkDI-6Xoo<0zP) zyJ$77E&$uFRC(?7rSj>Kf+)`|)u*Wk;W!UelfMTcpTIY&4sTg==2(oc&g-|l&#=n4 z!Yz4q$uB2vVLnede>rx(>+G3{4*@EvLr;?~pOnF5B)Vyd40l_o5sLbqIn9Tt=Ud}H z05vUWnr>%R)!_Uv^$bk1AQQm;O`OX|9+_|d8qu=c>BreR&h0uvjkHxBXVy&%{3K*; zNAvhws;AwsV*aAtM>*yeB-mf#E+f)jqJ!+4%PM(!1uZ#{gYKi1odP3-eNK>(lFC6pQ%7Z>xyIpeA5x z|7zVeosU~eXRTLvxvD@(Nf!@A%g6(qLQ%W9l6SS~X>vQu-HrjvxYq(Zq{-Y}_AHmO zoFYyx)*LR1_D=SX$d~VbA&s_W zQsf#>YHQl&I*vjss14u`$iyE;7b(=XupxbOeY}tu++hO76iA!-cEJW%)y3qoG-0YHQ zfpaS3Miyt)%W{hONx=Y>L6ymC+N(If8%X2!bOtzH)T()uJi_aX&E-L9PJNTcWf9Z( zfWxCE*bPZC7V`d~C`>;{D@F?t+4V~uH_hfY>vVVl&OUyMaTx1 z-1?Lh1PRfLHNT;R{&t@w=mWs6q zkxDm3A4N^&H~K zkuznP+y9Cgvp?U{ji7w=RLl!*11FX~Z&*~h7Bl++@c#keF^B_@AUsg_`L17_L9wnn zTEFjD8(U3xFT-DsobxS*(4O7<0T|EC1%Q7{|IbBz#&3;}^nan$&vfUYArl6g8=AYV zH(J+ge*yTGhoQ3%V|8CFE4k2+iBgrnm&uNeK8c(mLGu918n&8Tf~` zC-`0Y&3~bc-hZb=>mQN*`-`=~g&H`p1usQ=;?+3n$yy!EKK*8-(3na5s?b~~`-jtt zTc!+^vVcuuAdSxSklTrvF|jyhUT)z6Es2k`yo34MEc4ddBQM&M8$-APJJx?SwuR_I)SDN*e8>i64j?1Jwf;11tT_uR5Ru>gY z52~yk^9e!@RkNZ7ING5&ji2OmU7|zx)~>Q8EL-0t8z@{p-l0St+4w#B)}-odTZRo< zt*-iJ7H`GB%@m}Y)*ml@xkYgV$1`TGsdk)mJavZq)`pF=Ux{{;3DFY!sAk(H&rD@k z%74Fn!v0buIsnp_G-pj|B5D0RirZ=!)^#-C#+@bd>EXPVlW~>Mz7kX-m@=x%mD;y9N>GCKt~9gjb6(4$K;ceUA9kGg$Q*2wm&q3FDem(0-F zhbE=fW;wO71ACZ&5`*%@i;MUafk6umGJ+c}@UR>pZE%J=RjUFN8 zazL5L#x@%3&`8U zU?L%tnjHlSGpW)qAKF=KVR0M-f(TF`kx` zn7-c?JO2%%+s`utfoCdBA$z~DVASma7E4oo?S>(dp|N2@#)A|K1ql1H~jw#!{otUMiMM?$6=zuKx@Jb#r02X8Nxrg>uSqr)K^r0mhQ)i

d<8;FMqa=pV^@JVQAl}9F#kJb4+xMB$o!ONm8T zV5qoS_4!^NdY?gMqa3tv=wt9TQJA%Waf7+6XqIW`%Htc-6Cu}*Mjce0bE7z24PkJW z+BkQWyACPSjIU13nN%2rZ)$&@ZWB3(z;8qE9?3~H!UwSJfn6n3j)2DzM7t`?X0-d+ zusAuXHmZA<7tJ)J!$r_+tw>HLtxxsl_8FQ?>!Y*L7Miv_uH|>xKDxWlp4v{LQC7wp z-k;H(9ODo2x`q4O@X)Ox0|n~;0l+|uy4e1!qOM;NoBy4N;6J?gzc%sw146~J;4Spd z)MV`sKq<=)fKrL^8|_{fkZD)`X~Vm&Y=uS<$@lNP*A-4*fh%WuHrFhd&b?07N_^yi zYM-XCkWM%KC?c`I*1e|(lmI*8y&rn zZhNdZgWRB~p>JX1p4C#dQ4su;Rc~Mz>va$nRv4=5_gb&S^(sw@_922Zmw4D3oTPMJhrOXP%m+vBIaquv{Yebmoy(U=o*YE z%X^lc`RXaUC)A5){;PiDf9t+J7os1{;Tdq+P4<}5T6J`uk3wQaVo;k&^48zjf!CfB z_|J<+N!lM{C`O%Q<)c7(5w+>BSs-P*uz~xYPjj;}=v*qeSyeag#fJ)GrbT-`jbhcP zQthMK|M%;Wl3Qj*FS7?oIPW`SiP9eh(!4CR%Kj)kOZJKNC{R&`>g{k}(`UI)Yyxg|7bx0`hoQ;d-3-L2xwi{{XgOhL;5%U z-O%AT8593|e^pddR%QF21W9`?&-G5BARP4gGPyVXjbDF+aIe8JxC-T4 zq){ILW9jfH|99d$`gIiQ3~cUm5N?VWRNOmxUX6PF}cryp+>d=3yVuBsU` zy`F`ib(G<6dxWi~gz%r$O>@TV9_rR73@;)0ySi^5V;p6vOr!eWuu13xnueI5Tz#Bn78-a^_*YFdBg$brRX<&il8*DnLA6w)kxngn-Zw#4)?s~ zRJB2U5cKSI=wsNRzi7_$kt8bf#Sg{_6#WjioKXa2jXv6!f+(@_X)%{oL;8`fDPEA` zixkx`ohA_TIP)XP%ESexDvrh@)#rV^bZPlFwL3IV!Qm3tCEUG~=w*x&gK>gR9b6)8 zA8IH*J8OhC;h8i=YdOQO^?J;;j#a4~ZxZ))Ym|L^S*zX=yD}YW+(iJgGtl$RTYE}F zr?qq`-e-@y+2PZ4-lEgPQ|kdNKh`$p-XC8hZ{kX4q?`&phGemXn2eLc(7G10@nk}t z=+`_LM-S-&I+^qxRxrfh<)LhlqSV#-AF6?zw%0@jKFmv$JhJhdUGg>dHnk51vz}7C zxd#I}_r?Ike9L;GaHQAl@w=jILH4dASS-OgFQdz^c^Oc>uOTBf}Ynbw%u?qm9T6? zmsLGYM$3A0LHdg0Rn+*ZEp-#s6ngY-I*B>+*TaHj=ItT3=ellMVuen%Qi}25M^s4= zJtwF!p-fLvvn%Fm)u;2WeID$Dpd{AD| zUoExx^(zBiA=Z=NsU=K=aj+kswMbOwyPRFJc&tuw>dG`uma2DZmlhxDvFCc7=pwB? zd^~_)VF!@0MA2UrmAYcAabqLFod-UxcpzJ)ai+k{NT|N&E8Yxd>(^J~&qE-a_nAD* z@IGa;c=J4Go0dyM^ZsDLVd6BP<8B-WV-5J&&eI<9d!LdOFWVgk_NtR2@%!$V4NX5` z|BOi3-&y0nh4fILK5#RnRgaZ?R9a%1zjS_uAw}*|V0WJB z6X62AG~?x3^g4%WILQkF;G5xlahpEYh}yQ~vku!<%$|$kMh`|og}l5<+|*z>D_uPo z-0zk!Vs?L3l4yW*fT)_X9^P}23`SKMlERSx{7}#?^1(u@#6948Y$1~RSfoA>hOPU`RKQDZR zO~CgyJi`H$uPS-JWl{ol4mn4E*6z_$beyv_Y&HhU|8Dg~8ONAxyBgLfQ(-lKUvZ1uS(h3=0LNS0Tj&Kl znYLBtJ!6}UYo86&8rmsl`{%Wv?eeov{2UK|pt)cNvGDwWq$DSt7HX168%unl`Gxft z{lh;(`!MlW>-yib?SA5jmRuYeTvLN*dUL>NkBtq1-DM24sJ$K~@@o$23rqQ%McV1* z--zA+L$)cJVd4LlZITg-z~Ox!T{Rf0LvQLrz*pr!XZD?f&U>QjC06hbn6k9mm_%h| z9gY=}Il(bDXB*9OY5@T`ERG2q#$Q#`NA%yJ@K--N;@hrC7lR92*znFE!<6flsCCQq zMC%OnzHBj@ke|PLwO+$x#0@pZMwV$IGP*;)KenjO(1WK1#;3V!W`8K-DPXc%w?&UqO43f>asZCTRqou5L!PAo%ZZAI}3yKMPkg7d-4 z+xr{anA^=*(8Er&Ws&%DZ@dwGMcA`CwhXJfx=4~pt(Qw0jWc$!H;E#GD5$jJmLvHg zU=ol=2mE1!IAcw7LjaUnD}?Fe`*f)-aH?Q*@9h?WQCO2)mX83^arF$II~j)b<=U?n zt$}}UNZTmcf7JEi)au@9c0Eof?1AB2o#;+e$cdrMK zXLU_GDbzKx#i&~idze1nw)3O2vS24iy^srZ|m@nIyZNCWu?;T$bPgQuibV1vZ@V&eD&^`{X03r~{aTE;r8hT}{ zil~Xp135tD8pr6PR8myF4WTZ=7#;SRT14ZQ`@r?&*bi8E;w=CG-+VV{2F}bL=`~IB z0tuCJzfv+kCfvfY2cXjX5> zb_Mm;hFo)xRu|!iXj5hcf2Wq#m^iU*rqnOul#?U#a+`hoO4{XiPNJZ_L$(u9E71dX zip7`_JfvceI3=ifHoA2Qj(!F;_NB3UE5Cme_IFO;UP)&=QRE=WTK-(2%vTbOYd353_Vb9@v{q zUy$bJE*aXK&hf|BMsui13yd9VX(U2fUe62P)N<^(R6=K#6vphuR95v539Wtr zY~yI{i_iNhK+(l=y4;9!m>?=Q2~@FuT=qSYaZ9xsErXBQJh zxG!`VpDl>3Mw=h0nYb(P2C?~>hNBHQzakqu5-9guNNcBxbEiH7bISD-=XS^_AWwlH z@~vot>0ohAln}x&ugo_K6f|J#d(SfHNZN^9zD96yfBq|j3Es2>`>U1dz7`<7H18uw zx~*sha%l{-N@u$sn{cbCFst!)Yl^WAi!LC+W^H)eR8DR(D|!cmy`U9dU5jF95rHoi zX4-1S&Vh<;rp`nx61gN?7$E}HT?+L49!_0S$FC(1Zu`AJdoa#iXqD&eO%d8OQ14Sl zd|Klfv8Nv*j?#2xQdzN0N1R283Bjj3$;7!38Wp{!cB4I{cg*V2b@QF5&@1`p!$gDZ-s{}_=wW})_ z*1Uq|oO$`)q2c-vacq=HNGsY#YpPXwul6G%~M zPu<7BZCdg%N7Ul&?VXXAf~ho>^e%jblfrpYxc zyuSY2x>vk>!Se+M0F?z^^K*Zw_LWu?$DjIMP-PobJ;8OToEksn%)$P~XDaekmI2}_ zEc5!1hvh1$l0LI-# zWKEXesV(RsILGMD*Z(AEoNqxUUp~cLCyC+M)YY2uk@(YlW|;dCicsw!SQdP7Db|Um z(h};czz^{b6V6U1IK-)R?NeRG1Al^g3T>J>kJVC0>rO#x}S0SfW*Jr-Xn<4F#~J$f<#px!ow z&$FnPhP2YjP+PMUbx;YW7!kkOMrr|$M%>Ny$V{rxbA36uP}dWdr{g|CSvzF$rx`>H z884o2{H#)led@~A+&wDA%f~jS!46Lr1;7e)wvofoN_TOhv3Q)A=cQSBy z+T97_^#fo=IyH07P#+P=JO`=5XtSTdIJL-Mj$8*LiSZjV-5b3qP4R3 zKLGeAVYA|d_&|`umP4@P8YYLwRHD9I{PhEy3f9|?WQp<3((-^nHqV5YIEjulcjad>cq2x6WM76)p|YARb{!tC|Cmx#Y{z}#>> z^eVz(wnOn<#RvtxIi^Pctm@`q?FG)eU8+aM^fd z!{rC>*O{BMr~a~U>m$0Ng1>uB7S$%w+C_VF85uOS+*x~}Gd?veT9hWfzD#tJ^$}xxtPrZ*rXtw@#P0jFR8a zhTW8$k=+^NLZA@c*(e!bsub_V3lc8$ZateO<9Svg6KA+<=x+`w=v(8!a#Ej13+%|e zCYGqGC-*leOA=gbtet=cJuM{!_AB1e#%w;KNPX&r4}O&TyK|fyn<-gVBJ0uOMb_lD z3I~q%>37z3Al>%^nB5|T%;=;2HK48UT$NO$kmZ)FRC zJnE|^*E(Lr%DWev4QvW^lFa*KoCf+Jvi!!ucufk3&(~Pn?iX=WB~%`{W!HVC<6kzg3F=DSG|0g7=^C^Y`da#US0w#0=;Rb``qZQy1i+fxs&) zm{akJ-szaQ88EvRF*~;lUGUv`3E=fC(vlz`Qk<-vuId*Py)!<98Yx)R&2IlNC~i>R zE%x3LzZY-r4VIlQoi0cWpeXD3P*fh(n0aS)8owE;m?T= zghidPAV%nV{WdQY1nSGKP&rXuK->PbLwM1=A=$HKv98AqkdRg{LVBkThGwj zvzkjal+>u3k_0q9w-BT+)iTLlZ7;15@6CQi?Z^))cU;e(vp->Mue5aqma`-1%}qrs zB*g>aBi&J0pvdbjS>LD2DwD4#9IKGUrCjv-DM4S8K6}!Kxu0sW%D3%IHPrj)uIdPE zqc{aC&1tIH4i!o3!VP)XMBg*jMn^i|@`-YRFIoBAg{GN|cRJ$}DXt+3rFiFf8m$rx zf{8!RH}!DV3a%cVd#B_nIaTS^|H2Ul4EW{Az#LcZ2rcg3e~+$`uTu& zbSCmUMhV7gG%WDyF{FnfTyxo9fK#qV1-kmBw2U-$tdpr|*ILA&WJT=}r^+W&+-6zX zZ<)~-+>*5HptI9zxXRlm1?ZD|w0L_h3{`Noj%)gfsv@1uim_*VO`d~CJqN1do;O)H z$b{Dk#1ltPeEGU4Wg?ri7IHV!3Z+Pfst~C;Uk5+0xjpH8c+;B_)rIge5=W}KT}t~1 zssQ%4O3d#_sBbe=IJbbbIhHh|3@j~R2!wToH3;K`9Ae9^5S7LdwqxBSG*Ia>hN;~3 zd4cXZcfJ=jo66OWAAS!zDGxj-<&>HDs6x`wPR=Xh@IHJ0W&zzw_%z>I6eTjdR|5aU z_KKzLIE{;1P9+rci}lf>0mP0;wyxQbDP9=3zlBRg%CVtn4*;~$oa0A`Q9I>ym$jG9 zDC7>$Gc@WKgyXf5&Q;u=Q^t6VmC2l8y+(UydD|J3)oV~*H=4EBaw?jbKI@(;Xo2(c z)k#0Jh5VZjVHq;$;E3Ax=>#{w_cE1LLkL;2YPmuQv>cnWbB^s+RbFyDjYR{zb z8YlkPfNUuz$UqWu6x)9>4Ou4mPKIC!-WO9`$+%-9LK9&P1yy4sB`Q4@mgr}z=+9m2 zZ|g9IC&`A3+8qn7->`&R2Dph*D|UF5CRu)IPFoIlsD7xlncM&hq?NDpALcaOykiz? zb+y~0n6AA^0l0vUcc+s~2oz=D&e#fSi!>)LK1uZ;Y#x-JglchR@3McI@?%GezL?@) zya9PxJ1Kw-Q{!Z_;&h;o(N{2!RKQez8%9T?OU@un-y)1uK|f-rE`kI3l^m!|AYJJc z%}vSPfumVL6N-#3#j)ZA?kS0-LR)u+H$o3ot3Dpy0a>p_t>Oo*AiWE%ml)tdIoLux!Kl0x->;6OTna-Si|r_PKM;drMQybINUSjXk&lT_b?DPKr+cuBCE1RCZ>3M2mJl`u77# z%85VN+0ZZV?g1!RqSTzhe9FZcuoUBi)^&-Vs6QW|3!ndY&OSo3`7Ya+9((KhsOB;4 z*F5MVIsk1%nD-_YG1`?>pGH7Z1#e502Fs6rFTs3KGwP}+Uu?(mQ3i(~Q+`cm`%AVz z8~2}Y@_+ws|F>G<=imKPcl%%KWq-8xWwrg43Rk+k$h6S$>?R=7WvcKc#&2K*^~uxY zCwEFrtk$)J?GWEH@!~`KuV<^T_pYT;R=*yg74HAcv#N0hul*V5j(73o)~zRFuPSY? zrcuFY!?=Gx_;$OR8D)b>s99^Jd|ATv9L}HWyBqu5%K6fw0kqq6n68BJs1-3jk@vQVs z)2miF6#?IS>%Pns%j(i;J5PY z{t(&we9B_Fj1J%Pa~Fe>2=vNihT0)UGyO4zzXi}Eh~;MF!@k5+kHA)oWlChF^!SPG z6K%Pw#;LVrL-rSqO^<9S!(S2!&3E)ilxnTjgj8B$uMolWVb;e#JUl(R~ynJA8 zZEw0BAr9kASK!2nk-xU{e(_!*&~AmsQ*Qts+}nz0bIK_hpk#uU2n znY-A209fBVlIEs^R!dlngrozjCs4$zPo4^E8IoRhdBs8M>!MGgo3;~uLl`3{20jygARvxDd;U>gxJAvynnM!8bBZTrd#KMhvCe~ zVtYvxdQnb*E}s*nJx5Kvt+#>a-1cW(5Xf*H4W9LD(i%I<`SQ_)kmVfOcxW_K7< zwh%uP!(PLu*tzIR=RkU!)Chjn^}Nx6;50&yj?x3h@IseOd#w@Gj7DXp@AozA3+0rq zQq{Yc@q94Kw)J`vvTVf}DBlxF6&a;?a+j_&$B<`LN13A9HPI9r2XhLDHdc<`h!Icg z=(NUUH784F#e%_-?wp=YE!d*2vF@v=0}*#z_AT1=tF_F<-Zv@*An+P@`=YcY(g*!X zI8R1Pk$9(PXuik?SNCP}#`xXy!<&|apF|2odD}1}_M#dlKd*r2gxbA4rvzw8BTPUi zLn^8h0@MI$OK|qmRr#F!q|)##XK3Z?SrSMf*44+d2Xidl_tjI2M0vR;89$1wc&Xc} z6a03c=dZNyM(Rc-aN29djLx!4^6ybs9HCj>WVoYOzJJ@}!;|Pq-~N478TpR?hiI@H zKmYf?0o_}8(3Fbztl(M9-d!(nSJA&wjNN=J(Ig@?);(4VarSSp9dy0q^q=B_FQ9xg z?W&{|{`;!YvNEE90inFCeOarovck~^bvmrs^x>ggr1x0+vgyPA0T=&9cZ)VEgj1o5 zBaV?Se<^-c+SVW~xNi0NI~5p7=n4#~5HmNK-7XB|RQ{>J^jYk&#Hxn$=5v% z*GGPqDAqq8u;*s^3zzNNmL0Nef;*!Ae9)lBiuv=+KU?Ex-}pIFeoh5HC&r(TiJuQm zaL0c;Du}Hcci@Eb)~jWg-aSNW`Ti$_&6)&~RS1=gAJzKrCS;eqEofz4KoGb;9VLkd z+5Y0A$%^^oFLb%zv8(r|oy30{#3rki&6Rtu^R$WT2)JNqDa$sp*2|-dk-QZe++vAWKpo*RNO{O_f zzWu5Fl2;%dv33+p0#4#{E63+$Y>!|xNZ+u*mR<2oS52^9#=~lX_x6W?p7ry$Uf$vU z-jgIp+8^s!uP9(P7+Fa=xX+d;DmHZ)v zXwcK-Ju&Y~ZNCw;EJUD0x|5bb%-k{0mQt!lztORBp`Q&{fSQ?MRPB3ypDy0uiy0PO z^hf+Gt1?2w7VH6BVm1ClCNiw2g{$i8SA>|gVB$7ooo$_=W>aj@^|_uN5T@-YJ>0JiX)~b zMzWScu5X<@P;AdZV{nY^mO`UjA49$*`lc-ar#K0{X-$N2=&Ic#E__wk^aKt$4UX-g z5uM)uzKq$wGH-7)Cq+E+q#JcM)^Sr@GDbn8EhQjJ2vGZCdcp_`G5ubg31kB4@w++s zdaBA$N>R!Mg9&KO))s)t&VGYJ=l0E4{#Ka9SuNgI87&;3C}^e6a~k@ZDl`!3y--g> zZ!pL8KnzQgWq!yD6~S{_-i${F?A0pdsxQXHxvY#RCj6N9(5>&8FCMsRC_8r}XcC4O zQz=cM@0shs#TdOn@-RIUPk9n>K@)n83FSuXNK!g8Zlh%z`t2e)Msx|!Vnj76V%Srx zmG%dWSS=z}i8`AJlMKH-?A5!VUG0rRJwZsfds#-HxSdYZB-Ea2qU!0-zkP~!A1L?% zU=QHcPBn9qr0)I=v*fq@lAp(aVY&R?wzw}Kfkeb!W1_m9HK+cP1}^~owVUIm?BAS7 zf2Dzmj9?($X~~dVoh`)mNhQ#!YM>%nG=L+_g||xwvz<;*)^`5jyUOf{az*fPiV;zm z;`^k{8ge0azV|bllVY9ZYN?5c7^DMMI7(pJPK+CkE@@$!ROF10{W7yStm^S7bjdXu zE}Hh`jp%f9REQCE05vZ*K3Vs@omhT<1=}wRy-vWQY-xd}GZp)d3Q>kTtKlmvgt0mk z&1jWfFAxE2YsGgx#8!kF-vZms_giy?;e6e#*#ktoiBcSh&73bepTdR(31*){0(|Hv z=1L|wy`q@v+<1%a2xs|@Z<&zakbA1(?@bd2^W#Nuz4cg?Ab6V204CV;qo1m}lGVCo zj34-5eQVzl5+kyZsYwNR^!P@TU@2Kmj`H-ki=eZ1+FSZ<=`;wfxO|ZRA~NgrP`M_^ zSF4*L)E3Oj4q?z`+qd{+$@B&T;z2d_`BD_wLE)PC9amc(!Dcg;ybV?2+8?1JE6(k5 z;ma~z*RqGyzsb+gOGYCmu=SAYt3Lq2)rP!EGCrK82D?J==OM4^YtmpNpXyEPD!SU= zu^P##CZEyL`i-Uo5aJ(ltOyK{K~Q#DBK^G^=0Omp_^~n8yolFTf`bDU(#`ky@Dty~ zG-+wa%*;AJ84-bl+lp=>BR(U&yY!}#pq%Vl!CE*M0&CAr(>sQB-4*HyzXfYDuvhoI zkskt`MflO!X5d4|vJo6#r3!80p&L~zgAV&pKhsM!du*V5zH=UQ#rH%_QbG-KsQ{vx z@1-F?_#3L|O`s&M3LFHyQ-JLH=V}L|D>eSTnCH&L%I!((c305sRN4$bPS9Z|n5fis znY`0dAS(3W^K2;~gyVpVz9V;#$=JE+3r4gcThKajjN-ev6J{IIZ+k6lK13C8%g!hO zTAutN<^{^jLBSNz`h+(oSFcO#CcCVm%99P_s zf1sQq#+duDT36aY-MvLPE^yg@?m7(`BKE`^ii-zSEO}&uuo>~vR`!-PB{z%CCgO7h zx^tKx&r?koA}>$h!=pk4()3?eMrqcOwcHh_)GdY9Dy%nMzNtIj(8-x8ufTJ6CWqaV z8`5aPM)bBNlOzP&3+OA(doPknX1oYf3mTg`uodm+PG47YB!QgUwT>N}JYj~s7%%2L5f&&sNaC-lY7(_yWIlojCC_RAd36Xw%iS&o zkW-MZhgpU;4Px$=3Wg3!8jBhGGsWgft3k3EkoAH7#7UB{+gO+FexGWQDXV2s$xF>E z5O!*P)iF12ydC!hrkwJcDU13shxH`px76KLDNXx*{)Ij6c(+ubW#KgBNU1fT3YNa2 zw`so6dwv+RwcXOxncBF+{(%Ei@KFDyXn%#&R#qzR@`Du%-<(5-T<^z@-_DblPpuJE zOLP;i!fy7IOUo+bbW`A}+{V}WBl*Ltq@X;)7o%S@6Q{)Q?CTf7W=3wV}N~ z3h(UEGY)CB1db3R)n8W7P{7MVvg^GYDY8h(Toz3jKlQyEh+eapD+w}Zq0%xiz9(ib zScw}k^4^VX+~g|LK_D)o*y2e0xnN^F{6>mQVZycE4?q!TkL^hw+Up|gD&LP1bf`Tc zVVwZ)4XhSjN1$UQLMwZY{7*Jyf(bgGq&Gev=Y&2I3hdY1duxQ2n1eBFD*e7?RF&2j zA^lcnZ0F>HVf>d#CC2^DQhCD(Jo$gtPkt}zZnlnhv3!FNDe75Lb+}RLL3s4;JK6q?^Y&}JF zEdE(lb5%oKc~c-$&v#vzS*TJ(q()531YNkAc#R<+KJ9z*+$&o(bBelJb`;X z5ze0FLc^N_>vv$s>ZXM!p&m}PTB%^}w&M6mttm!@k&%&6GJnpU0nW209@>kKP**SP zcJA-l?YlJHAalWq$Zgd?Ln`}Qu|Y1q%avO_hrvrM2yOrB5z?J&a-@3xwst}?XV`Lz zl9a=`DU}Wzley+m-;1wuc#=<*d}S_4L89G$WJqslvu(E?$2A2FAKGjbijB*>;nNs~ z4V(&>#wlMvcRV6}IL|;wH9dpwR+|~^QK6KrZOa6x*WY~9;gDX+Fxw0QH<-!rmLeQB zxmY~q!AT@T|NKS}?gq~-=X)QegujS0BE7+-`aZr1-Qsr;T-kZ(Z*a71Upbm=wpu`k zlaBtha~No0>q&-F*q8>1#Pv%lK@@33 znGNr=@NxbAdu$3Ky;U44^z1y34c_%`dvwqe*M?mKw?!6`#d9s1%c8c8$K zHoSrNVFGj&)JGF=9t-UJae%dd6tM1Z@ADr3*&X}W^<%K_*AGd&(Z!pJV|F~A&R+Oi znv5DRj6@NJCM}Mnwx)tnhP1z-&fNHU>1S*F>>GcEF#Q}3KPQKuQ{}HrvY*4@=WzHr z9R9x_4j8|6oY+1*VDeV@h3ZA+GpLRkz1?*kH~immh#&vgjgtRAaV`62kSYGR|HnUu z5hsmKr0wHJ(NZ}p(M#VWi|*6Zq+m=30Nj|H$J<+RT=&xGMVHB~^?K(V+{pLG*zIdJ zS#_oO`j`Tqz9bF+VbUhVfuwbVJJUHveWYE0ilhm?_O9EB3y_;%)^QLqSQKSm)yT9J zQE#1Bu?7gn5t_`}!ObPA3XFFS;BoweD8u@0OVyU3Ph-0-Xnijg(xu}i9&?)q z8tRQGO|FeO;vDZBy0C_Gc6?F_j9kYN!d)qKX7@z{vmPRS&BEZJof^@H0#|%`_3tzY zIo`DAHuh8Jr~vy070a*IaQ54T?UF}ZkPvv3WU!GqM%7eaLXX~5TA_`!s=k}P4u?a; za-8EvUyq#iT?*&h3piVh>l^IbcG-CO0tm}f*GV607bwiJ^CCV0oQw3kkmAa)2U$+o z+2tnYRxZeDm~K35#2Ms^$HWa+2?S%IK@J~i@tQoKcMdF~jNrTaEolznEr$IM#g2RW z%hNLY--f@|^>e5%J?v>E7bj=}6I34tpXz->V096k9y4dr>Fet-r&*+Z>}9BI3Y7gO zOU!cf?XhQT$8uO9e57;w+pG-V?qRVt<)>S39#|ENA^6nM;<`#lLrmBO8z_gv zikn+PEO&Yd}Gm(*v| zl?G@;)?w{B5`#N{aN_oh&E_0yv?=*%K9m}$T3a*8&B^3F^40JOgsEXpEJr`<-R0Is z?b`{Q^JTPW!U*o}Tuqwy>moEfruWjXQzL!g0Q=7C+UlxQ<=0XDkm1$fKErPq5wpSq zJcUB#&iS|MkEXl8!5(0olM$qx8>i)qNyqFZE@Gv$eA2OO601CpkemLI8kW>}_#|7^ z6QE`NY6N!#I>k&llih3oWtU9biNhaDBV8+68lZCmE*)AAsl3GPW{JBng`P-ne>r6|Sdp9oMb?B-$BE;35{&Dg)mSvWO4yZR&zpl1x1%r$3pCHZVMh~xHd+5mRJN$IaUsC z55&Om%R}1i^WVK2#=qS{6TBkStMb9`(f?uZz2ll})3o6rRX_qrZ$Uvosvu3Ih)Nd` zl-{IwBRxVONR=+2pdf_advDTkyqy-7#%gp|Ec6WB>-JNe|cb;dLKSD^k zb0^n*US~g#V_d9F$@v#=s&9lOr>9Co(we8?NlUB-GGpm(>tyX=P0f^bz4=e6H(~G7=3x#>~40Gz$b%f)C=nta4jLqLr*U< z_ni<H>|(kWnf6&VPEsJ6YrAK;s$&a(6(==?v&1|` zf@AlO%aU}^Dmz8)0UV!E#(F0@$z$S_#S&FCi}fqI4ZOA(OA(C|LK&}Hkq-;^MA!6= zVqt0+gHXby`KlqIlQTG7=C4rAGX0~@;f0SR_#;WKATH|_d4pNt)B;elf*RZ2Ln!9d zEzk@vrup9v$a=`K!A z{)KZYBvZ_10H=h4rslBxB&f}8OnSeSnb^Ap9qiB|JgW8Hl2X+C)#~PIEieaG8U!>J zj^~E#ESYv#JmtA!eFji}z$iQ`Gt}{kbc6#x2?9cy{Yk@;-y3aT+I_*jX1j)%Kk45Qm=XVWc^~@`T$#(=IX!v8NT9rTuNx+_w)n_MSam8YU+$yqRGExZ~lj$ zBRa+{)K>H=fKdg}IA$aPC5>VY5g+ayJ}yk?yJo@I|kJ&uQL zL0z##m|fICa?>JbKr2thnqS$TR)NbAT7#x8`@3XT&!cW?vRl_8J*o!zv}cb)Qg;b} zA!?{@!l>+2%c2r3!s!<*US0&8Gf=24JW%}2nTgIUude5+?4VrL2WGdm4Ud6O3`Ad* zPYkh_yhbitx->JjBRApHj%6rTVei|F^)1a~U*F0weX_d}j;t-ImZ&0n>uF{f)9%VN ziW6HTPzqojkPDGwUIb?f8$P2RI%h0WGVCVTx9JXMALMMs@QH+k?olfG1m(5f);Wz) z6xLFTkz(&g4hzEQuvCxG#cFP_qWP5eri_aQJ1Z+c`Q9cgvx7SZp_xLFL_WNb*iOo&#su5WCxdn0ZE?aCx$xkUmsMZ)4ioZc1R^UnZ>L729t zOh-A_!#KC_glx0fImE15kx2fER@uG5_pJJT3$KF*XX3`negW;DA;!qspJ*jUuarJ8 z5$98UTH2}c$d8_gr?Mgnd76sz1Z_P%1Jq=-jq60R-MYNZ<+@-sv|+yw#p}GsmoA+e z&My^65=~&I4+pT8iy%5k^xf(GG6({D_ZOnyWKv{*NjkkCttS?l?DTfv=jD1AfT49xzbVKOP_ljSati`BVf{@zo zT(?d?$b(T}D(JIOVX@k^9mQ)dY(7~9lGUH-1xlA{)J%gbs|@(&4nt{fnI$AeEpWi> z;B+{1Lii%`kW;ho3y^+Yj)V=Iw$fxe$Mn-BThvD1a#rKxV}a#LqeWT?qFoRbfmq5D zm62154P4NeY?Z25yKqrV^5e4!6ANBu<(SKYC79ACw}Wkt4gB9s#k*X!>Ze;D?LJZiM(&KB0HbP)c5IS>I5#vT9a%^DkG zkfbt}-yZzM<@k-Y?9

U(c`5%tgPXul|c%HXVw5({(eqxG~c272yFyUshkDqVDqt z=Y?8B)8eTk-)hm5c z9TU{Cl*Q@NRL+j_4F}^~(9^vcm&S6Vgy#I!+bdtL2v-R=80^wB3$g}>TO~4l@FLFx zKy&hOo`iBghyqKFO6jH2p>f6oF=^{`W`uFQZeJV!ymUuZ?zdsXy*-|NiRwjslrpy- zyxd&cj z*q`uF4$1YjyY(1HV<%CPjX|(XGS;*0KA_ z1l%1TlR;2S_h=%@aCfHbVhuGXgny)XXIu8u_pV6yc)e!%WW$BZ4305!N6lY6eU0-6 z7h`{&^jLoLyVM>FrSnEw(WJa6bOtgGcPl=#OFbn?+yOSlDRYR92X7H=`7v)(N(I<- zG>^QpZAJ&aFT=VROzqZgo=~89#(P^7y3H>Rsjvwp$570CRvmKXru?|LfY?R$$A~5N z)&{_oPXgb2mpVyoAYk9GPPw{{4=0eTsds(6Szl!j%(C71K97r%8;%?XTUK` zdzv-zT4;-pNKWPbfr2*;hi{%MgWWF-Q#u?yPf0L~ExlWg8MbzW)im_OXcw?|p!yT7 z^2xAElAMnsS3ysbI4$;w_lA4vp9Yoie&Hx{(PO|O*4$c7Ku)2tDf)}R9 zgFcFo{K#%Tri2u!Og;mswM+_=mgQ&GuT{wV{Cp8(p?=S<%{-U7MB=s1_`y%H@HQey zyQ)%6RZ~kP82iMBS-QD=;F_ZUa4tK*&69Rse!THG!}$A`n?I+gpZvSf@Sjb!VIqhN z5|6(OtiE*N=!ELjF#S-YVRz)$tlLR?Rm@UGt6|x6vJJ`?cMbj>?ig_bI(0y_=FKgi z0X#Y7zs!lp1r!obtPyzEu|j?AwehiDBXL}ESQk(nlyp)1W>-lby8-6z15t3`f;0b{ zoJ)5n)gv2kKMoqoc%c}w1VZd~f`Hxup526;M7hhah*06J3h$D#%MVXfVn5c+UbU=a zrCt!`wta2XMkL4~bYr)Cg^$(%B>m`7 zq0jZiC!jY$j!>=Bl2)q4l9^rL+M!(gD{$$)_mpUPZMwp4>Hd&usC`4m!PnrzMQ~S1 zeTPv{>$DxZy|MuJJO zG=$QjAC?dTgEK&gB51R^TQLVY$M>ZRrDbm)me&xe8{@hW^CY2DkVfw19ndZ4O)UIG z1ya6?y3`+aphIP{4g%IG+|y{0W>eWkd0cO^2~CIKIzF&2lLmQR>|`t zD=tB-M@+|Y%1f6#&H!D~Ttba$nq#y~6J#{CxsCBN;-Fsn2S}lB%V_wcBEqoa=6GQJoo(feuO@q24)#}|; zPaY-_tZo68_gL>U3}xfES2HMcw|(u0W{c_W2aZ13PvkHp2r+^*jIML2Qm@ly>(b_1 zHLIRD&|&d7vI9=m3? zk7eLd_(MaL=n-)3vAZdG3`j!KSfW2%|3Z-B>c2)F__O~nWMn~_1Kv!RF(kcu(vU*Q z%Io<}u!0wU2CyKPB2bZFV3ATKn@l=mopJ-L~Y7xxN;lbnmoC5ws!UjKM}|P2so^s;{G(~sp|qGmv@0-2)rh-wFEF#7D#tjeKTL0H`T3{ z>b8ax!(aDAJF*^qjtMiX;zqY#-dpHa`mTg(2$y%pi%+SAi~Imz+>j%QT-}iC`1rBO z(fHYJAc8JG&N_CbhtsEm2<|ROv|(}kw2(h-F(t0mbFqxr5RI?MbSx=Z-!6B;KVg3K z0Ol?EpnPq^JuxgW-L_vsm~qixdKi^X7JgpF?n1SzAsaVkXj!U9Z_I)>Y)2 z^m_TdPg#=;xpqOm2B1l(Alse6~zJmxh*D>97uZXsWMRPS-M z++xg(^-9X{-jz42HV2u&!az=(YCSf>3C8`k4n||)lqAf|H z1Khe36&b|fT~Sc@Z|Xkf;69V(-EioYa!8W)c<$uRehGV8qieQo;)eQFCng5|9Q7vx zM^a-3%u&MLA9cF@^3##&Iw$OgP4%g_Z;#VI6e(w=h?A=oA&z4u6^6e3d#89j0Hc1FS z%C~1miPA~Ng3>gRg#D}F!H+jJgG_njtvyhp?o^dbO zC9G+WB9H*=Tc%ir3PVrj*iUUT&Zl|Blt{H!y$77=&Qc4ocM=3Le(Rvap8mo_`5? zwT-`Oie1%ih|?mF%og%A=sF11>d4s)c~g^w2IYQ8WcEj;Y@J|^EL*W9PKzRH-IY$3 z%=;($6Es=Z7Se5e!PfO2*V^A+G`{*?SM{h{EwPg&D3^G~TL?B4GvYy4S9VqGw3oXQ=v7!u^Ndq5&U?Mr)0v27o)@s}n%dog5X ze9?K@FwzgxdjdVrJK5EDRx^*4I}Z3Wk2-Ga$N$v;A(O`RKZ-6gr^y$cz8yIJb@Ge!;m<^e?_5 zc6(WkdRjuA99{VUG@yLycwv&z5b|GdN|^qWC%sP~ui!wZylO{YC8WBGHx6e`WHcLF z>rXgzhu&xR$&X0fKUVdl(J5YnOI74i;W+E3q&{kO<3$4`#!VVhCrqScC9~%*V%{2> zh7F05e^w?hBFlDd5yl4I)EbkvTB6dbE{uzRsRq9D?y;pBo@K%`sX*HSxrMz9FkSd(re zuAec~EMZij^G&PM$xRD7mw?4|HEp0`m_xBz&XAv}ev@T2Z-YcPS@+w86-K>|^s-u7 zQcr~CR>M+&<%4l>S=}6|Xq~w-KW3t|aLCHX?FXh;&FQ%h@br6`2BbbUrdQ zw1~));F+wd<6Y025SAV5{q1j<)J-*q?yCR+C@iC$!dhY6~fw_V9?H&UlGHu(w z|4~)@TjBoic}A)F?!>GXOrIPzGjA>TdML5r8P^GbHRbY3OmGyH_H!3Gx4O+}A0J1A zQNj>(v|z_p_<_h#R7m6X&3P<%$u~?A7GYpP+f@6}We7}#4Y}{2z9i0`Zh(1q<>`Z< zW9J+ipB?h-%9Yj`R7wm~WJybj!osoW1x-PoW4)h27H5|#_nZ~I3dd`?TlO?W!kmWB z$Zlqc%1=*3di-4Th10c>e1mu*p8`K*$ET%HG<)uUo@%hD0aR8zGNIVS5;Kv3NSwaKz~7*#bG zweZZn;cyg;WaF1zOoPd@1Z0{ejf;VyW(g;bax5TU3bFmh?zwIniCnNDG5I&`9{W+(M5w{(2TnidBX_=mT*OB>pl^90LrMG);FgNuIw} zKLwV&N#S!-q4(8e`B#*r;@6+HG&s2TA&!ic9JeLKdVD8%ChDl*w3R9M8k-;wrW(k; zxZPS!&;I%5w#m@TV*(lz!n zf@xY-aWr>IqSFtqYBGIA;Ta)fT9_|YCF{RxYwmG|#}+w@zhl*psrT6ycAj|~-qpQA zOcEFBUEcrT54G81tY5!jgzlKl#DOW<#8$0_=;6k+fpcC@)g9UrG8G5iR{G3ETRIa9 z0;W+XVg4Iyv1>54C1uoV^&)fLaZLH7q0%L~&M|dw*h#+gq~^1gcuN7tG&mFLjkV05 z!y2Nbmcz@TU?k@yuV{ZGTu8o|1_{uO`lq^)W{%DB=;K`*4&o+Fy|$MueTcVp@b7Wv zgi2u8!b>S_3C?@w=;iWk$(Et*)UbmY)t?XCvISJf0FEm zUVzNw`^{5&@E5*r?|*X}bxZEs@OPZ$k<744pp}nJWJnbM8AH3kG%PgxU~kalr?|ar zKktwI;)3A`{U?5OqbiI=ul~6z^w0O_4TS%d=Dgp-vHx{j{d>Z0i~1ku`TRFo^*^ir z{9k(Q?=ki9CvdC3h@bM8^2&ey-2W8D%J1~b+9s+|y9t-l(bD&z0h}6{cmp5jMfr2 z&gZJKtgB;nN)5tifK@_)^ay2%=L?b+%-jCpTY3CI^TzFj>0KV07^M*fAY+Pmsc?Ob znhnQ-qpcN}wHMK=4&F)tz&X(oVO@C1HLknl#;nEx?@w)j+qb!aD=u0cOdD7Bh(@Pt zDymL_Aak~*#p>GHX;r6k=Zr@?;u+-f8-P)+X$5uI?A^mT(9WFZvpMWj*I}F}X zPC>NHJ!%4%2&a_h!s&h3PW-ZQ!@9dWL*+wt2LlU#TkEWb$z)l*>)R{Fba(47KmWwB z7)OzL2_eLPVh=nw-6Xi_{U;#LCji6V1@xnl zuvwu$5LUb)I-B@Y((BLNmjCSIbNA-2JNQJb`r#}W6gFy%P1=n*ez_YIW>W9v&wzBX z{Ew%Bj8eZ}tggQU z4%Y8zSnPM`auN@=i7M8eM&y|~vc;t8FVC0RTo_?h5t6;I=Q%6d~x+vh&fui^L zPY=qHoMakEg?IUD=Y&0HvTIn(WuD%)xy3P+5~0Yv&c!)V>P=L*upyLtAH9w?|6Hx& zz#f}>pOG>0`_BtoJz}h7O7sveG1^V()P1>Le1g9NN&VoVHpLvn=eW++Z@D|s!Qywg z0>~o|;1^+CkFzvN?)h>>x?78sAG~m{zCA%b`Uz9MZccHHj97Y^v}KBOx<`lA1)SEJ}NRFh0}VcHo$ zQUTfekjdZ$-+3PDSUiq~dsskfjiva*q3L3=M|@NCWZ(pj4zJiCjP+@iu|;zeSh8Q4(PX4@l2h&H+w%ug7pok zXWINw%kmfnKl)Vu@L$4*w>;b@A{97{&dCqmL(_kHy=X0yLUG82PzHaDKPHUMPU9g3 zr#}V#1TFxp5d_{920}cc0C)Te_5?lBJh zEzBIho4%c_;#o5jC*KrUcKTi%AWAoDKs~=O&yLPfLKYt-{LmSV)h0ee}B< zRnIIJ)+%#5x39z6mzwH_8wqr-#v;Y?eS7tE2PpUqK)}GV{RCCRP-)dnOTj_qIj}O9 z6uO?Hp%=V+JVTt_MK@|$j(w2&FL|NE8RL%#h@1B?%pI_5c&82lSkcbNsf`vt+k{aV zuB)%`_jLiwRy9N^-K4$AWcKMqgNNj4NF(v?hS1N!g~OQBOq^OS)}f=>psv6PFW!KS zD8e#7nN;jbGBH=Pdp1VZ+aU6@ks35)%)tDBzW5tM9r%?}-FEm(Oe8bPB7Fr%`})hX zzA*5rOI2O<{0l=$)4_%(uiVHOW^y1mIf2nb(2ul@%(hx zReE!&(m>ZP@dayWKcbBZMBk=}v&7Evceh8h-Kaw=#=0*K^Y_@>kGmbPzvFM^WkG2~ zg&T^>)a+8yk^Ce4AaNF z{PIx6{4US+n`RN~O1S~^D!(k-G6KjLY z3r)?a$%lj(89jpvxggo=$G5%Si?UsoltZXOzh~d!jC2eA@zjlbA5T5qb_L3UiP%=b zJUjysz76_yFH$2X)s(z4oTRPAg#t09mUy%6+AGG`x3YGm@sdQps?1iSS}l=q5+|jZ zHjxSIvHdfE9E<#V+{E594mEm*$+_ToPs=pWMI}dA{8WZG`r|_MP92yX3-0mrUY;uF zMTbpAl10c%=dXaO=UIYhyLpJ2)@gKTfS(y&fO{$~eXs^u?AaKYF09B4HJu8iz?q*GDsp3; zr`U(B$GSVEr<|UF_W`VBJ z^bP>{`?)7iy`_Gh0k#RrBVMrP8KA15d*PQkr;k#-Oe{_ry>RG?sfdM&@aj6J+DexZ z>yY@0hg_Q#uV2eIR6RC2U6z$UU?AFWmQ#=BwUwwOq5H(We>LmGyaLhIy~>Zc`y#^t z*>3ON0yb}2Qn+cE-VSP$!r7t4ddnOhPCyCgyVS~R1GfIfRm2~$K;TfhzM``qI#K@h zCL;88=%ciUiXPvPu{`RS`LYujU!B{M5x(DjQHWC|r=dF7$Zj02Yjr5FENdOEJyzBy z-^C(&gA@=L`_J>0=lB1fwBo<8CPeoLT&MA@N{dRL4X>F$D>wsiNx(W_@5(iUD6uVg zO4c&~pFf5ekF@;dRa;T(g4)FjIN;vz_5e@9VC!RJK}u)88w@W2V$`0W znuRzi1=GJ(fj;VWLK@?4 z6*|uoPvBVXPT%KSVtuwu%=!!7QXLvPXc(wBl2hk~50EFp^9nP8sq(i@u0zGqy#0K3@E2?-qDFQk32__MTo-Y<8;uI+wIWP56LNB^uDdQXYJA=#RF@a>A$%y38p* zMj^H)H3w*K0IR8#_RV^H0=FEFG8~3S9tzE^Wdei1?Y03r#@MAom#>?tt6^TZfft=j zXi~}?5ivf^A*XMeb*O`cE|dmK^Y)M}Q-`TnxS`>MAJN?@0lk&fT^2x&6VmX$wH5Lp zk$vC*V!~boHeM3pskH9M;U*_$zoWY!%RuzgH;mv4053q4jO$bGVq_xSaN;(FoO;vU zR7I|%ZaZ4_=_BL{DOYYYRJReI5tw<_8<*CZTz)g%a&`$_D>xi zF*6z5&}=ilA1=EDtW&%%k%D+f%tF>Wj%VRPJ{n&_|AfGMwU8y8stS*_M`5*-wcFCm ze3$JJ%wcVh_e6Qcf_79n8>VL^ACbSo*Lf*r8Caen`P(N~pc3Oa&{8}lq<-%)rj$S_lS(!%C1Sm(#0~MOgI7QX+tOBUONSc{ifH8Xkh9LS4sM7DM1Y7E)6LkVM-tH!*qTz=lL4W6c|w z8PX^zgXd(cPd~p{rS3q_vz~?!rta;sU z#0z^Ab`#-l?O$z$>76D>!fCO$dooBdDLo_XkTea9RH;FYTG56<%`umLk0BK=Xf3LQ zZ!I{nGkmO@l{>ziKuVsfcs1}d7dtk#+{G_%AsVWkYlO{aXqb|0xpbco9zd_Q?BI`T+CnqJUuFuGSd-5> z+B~{<507qejt*u;B^ES(x*8-$Da5Xe0Y%jMpWfD-tkW_TnzL4o8oaowp4Xhf?9VIE zBN$@^^^h%JCeSfAB@f;j z-QlJjQ)OYkbBr+d3wrLJ(U?gksjI&uCL)Zh@s(3vmFr3SCVI@iQ@mQj)#2fIp_g$8 z+t7!nLer#i)hvC4>7ZBO!X+2XoDFr2yT&DID0-44iad&YL_O~CS5+v15i*Mx$Gtf% zrct6VVJ^9MMV}34eT&d2_R2`ueQG1Vk^6M8^vzLMiaRHpY64==;UvI$714(i!a+ z4sMAS{{8{M#LUH>rdQ*kdor(B_D;GcbE$a zY+wQvH!xy-yCGcEQ+(*782JF?DV2T$g_1^PG&q#5v6%aGHF%CqgR@zXd+72O zpxg=?uo$xVE%2&F`FB&SwLfHM`f-L=`1sb(CEIXg%Hon|EWB~qL2?^HpS*SvREyFz z-iMsbD5VG~q3T3+?%K&RQ9j!+HVw*tj+pH_#&?Sx03BtENaP5eW1Hw#$z2A>vu&~c zG82lOvS&PB2CUxR7Lfyq?Rku2b``lv)tftYXLt1Mpy)oO0# zt#_a+<}^(r_)uCI_YI1gcC3v z_yT8wt;vM6FY$N9bKtlp+*%*I=V49ze&x0Ll&vIcr&rd~tKQgL(PF(*O7w|`wPI8g z36(;3BO5}bq6y?+*n3^g7LmoyAd+-fLNmkCWO| zV#QsYlmthm!sEeh0^cljPywPp4{v`GF&2&;*CL_FG^X-T_^}NjBYq}B45vQ*fO{N( z1ct(i)iKAd_G*!>4| zlIabj0c)v)jPT`rB z%Z$B{RNh?qTan(fE?U(GkNe_d7PmVWQ;)}hvA>oJdfKBb_^_D`^DG#hmw1q|BVrcu zK({sfSV|+<+~{I|%JQt5JHu#AY#}IvHucoz<5w>Q9o_M|Sha`sE$w4m8p7q*!gy0B zSMG7dQdNqudR=2FA}Qj=JQ0I3FK%n%38YMtLR2Ml4=0S~Mnp<=M0zu?h0?cNcX&mw zu6oTwhvO^x4}%(v11crF^*W}dtJyH}z(qsS9H>0uI=^ckZQ3@zD_RE>bZe-`Qmyn2 zns9zDR$Qn2R-ucj2o1dIs3?T}h`bGo&JdHfxP*@3x?btf{#fQPCvOTmz$9hks$I$V zjWK$ao9JbH`&J+|4m2(~6AOBXmJV~V^1WcH4Q6`poS;SG@qUb7nZ-cmZOlt`?pwmQ zZj~2rb_FuFHBW#myj+B$)2%0r=7+AeK07jqZlQh9>&bi1AIUDXC&Q5nz~y6KV$@KC z$|}>06C9mr0eK2)NkO3-!xi6>R9tj2I zY*enPSFNx67R-F9cMKvmBF>^Kq`bilcHir}krJd!I>wvEKH<1Ig zclg@2CIf*u{%NQVd8~H@nd3B`yWS3RaB+(}aozleerRdfk*MP14;8KkJ9&#>QWv=& z7W|VqrGf<)TGIh1&aJ})Esx}JGWxsK)hTq1_S8Y!+#lk!0fFjjmLEc2QN@1V%i!m%0g071Gpf^i8@<~IRF7Cv-c-rJbE-P0z zwAQD7fh>-T*l4&tGq!N#G`PWbot2@PqlA$p$I`qMqnB1>azEG^j zCh+9GZkWDq2jDqi-Ru*Z`DH{{{L6?4uTf+*1^)91uaZ?~(lF1~3AD}j7{5Dz+`M-O z^nQWmKqpEZ9&bsv4}66ho;px!U>;Arl zmh+DFk7qXyhaCKclSf-q3L#`@ONRp+>O{P;@#2~=IT+e2l^jveY|J9-5& z`pR89zOyY-(Rz>kRh`iFl|AB=#z_4h8Op3B!%w89_CytpLT}LVFXibLsmRA#FCtHs zy6g4Y-=UqRs!UYBz+RPVX2Lzq&HpOX^dDHLJ?16iwcyGb3&s=I#$EHTTuY{Bc&pT9 zRAObCwVv^Ql{^{iSpZSfbZN1?=yG#ITo%ata1fK@MJ>*TjDOe~*H6rq;_c$*?mqsix|%90bwMyiX*>-Utd6dWID8fld`oA&r7?7qYs>!^&>1jaaSeMyXg}gdDV;|9 z_2UzUEb8>7E-7{!R5pzrm^xMnS6qcS1lt;4O}@@hl9O|xv@|!Ef;%MS4k)=zxE*Wd zjkY=T8;4#NO%v-SYe6TBg@3SiQ&ot(*yaR<&x005~ff>&!S8})3V=vPCl@VmPNHI%k!gCkcL4Ophsw#5J^j# zIn6vSH<^d^H3@GHNV8%!IJ}10BDyTY{OVhjqNFUz)~1_#r{Av3j}*)dc=9K1^7`l5 ziu%Y2iRw@nt8qUiMs8D?PVLVLEgki>vq1=*c4 z0g2l7<~@x#dEy|M%g8-!QlH|LQAJrO|3ux~GdDq{h|8OpUf+uD()oUW&<9QSG|RU< zP1?bxk3OeC%y?c}miyot&~-ik6{Iz2AHkiSE?&4RbxW*+6xB2)?zZ? zg0>X|jI`_qkwtLPp~K^|U%Zwo2?+0+pYd*~@_BXa*cAIASrEdXc=HYj?y>~BjJx%< zje|g4TR*+qs?4B74b?7HW;g;vQ)CXmnrJ^`v5h2o*vs>~X^gZw@&-}2m~p1YzjhlrCO&M zvCOZBT*6L2i3e`IIt+%3Jrv6O>%Mm@?Ybo}J_=Ufj=cU6kJzg@8i&0fw;$mzKyUYn z_X*8!5_ob$)$S1fMC0k<)fI=~!)YfD`8b$Ahr?KqoCyIXNlBu@vUljj5a&vGm!&cK z*&AuB%P^NvUE=Zr{#rlyJFX=yAk(w`cFo)MxTo0nz<8VvsvQB;8^>SWOeL-8hS99c z<;Oc^&$GLT)x+$yygnOJn=m`x5f7*AjC}fVOmQ0{)(HyXx;e#%0mjG^BTs$x8!4_4 z9Ii6aa$bro2FdEaFFu6;TQq<;YCQ)n6KoazI9Lq0zRuME=1l)pmV4#dc2-O`$IYO| zWXZ@VLB%Im1j%Rwi1*lg;dD~)R|Euy*m9gdM(zapgV4Vs9b2+CW#H>>JmY+M!$8d4 z+g(zkgsR6lxO3U#hEEVnh@~rjNV>kEz51w|-SnFtidFtrKA8rCT;)&NpESx94<-W{ zapKd@ywZljU2LjZiXm*>1q1F@U?Jn4R>3>w7b8{47;lgXm5q7D3f&nO@zSOo4oq`cw;~)Grvo3M|v#j9P1C2$RuHOH^#xK%#D+uQ@G7OuX;-R;L+4C z-aYye)_MA<7Tq6Si#?Dnl}yvJom%maK8T|DYCbGTF|W~8xM!ZAZw%)x1fWSDRii;K zYsU&6qIc3=YIL-DUl_d3mt2dyZZBO(U@O-ft`}>Rcz#Qg=op@h-CH6L@`95<@6})e zD^PQmC{VYXe~+OwTXgA^Q?Ze;sl^Ptmg6WBH=7OXeUMz7N59n9z`-^?Y*E0{0;w}w zM6}(MS3J|&<$331L`tMcos&c3MUF^cI|nx2p>rIbpj$1$utN9KEH+<-PQAw*fXT&zZvje>L*Sz>wAe#LK=o$bHa%&&atb#9fQ2z z+TVn*^m{z{9NHJOon*bs$?m)=%L2evOyQi$K*1oEHZDlN{!}@E^B#LQd(vD?Dh*8Q zBa$w^UCZut*UQyo+bmquq+`Xf#H4?e;|Y(JK>k z8&1~^Lkv4JfM~vy)5yhin*Q!zJ}3gA&-iwBUBruxnXCF{A3WSs*B$>&GQXT0RSbD% z8O|jd(I=C*j-YPV-uDZeAo^TsB2QDJ8O%M>M4Qa6EN>W~?WIANlW;F5n1Y8pWP*<1 zVOUb<>0HDvAo;u9Og6g~XuUWs`VRZM=6TUL`|ch&TPo|5{8WMO6FglPBwsx#NgUau z=F&nEOfLj@2pN}T2Qot59>fz^+V#e?^TNwWYm0=L8+NYvmvk|8^2N#zq*J7-bv{Uz z`sIm)hgwp$GM3F^Q#$c9qx%XbXvWl?!7i#7bEc#(A)}@wF$o_o53D4fAc0*04;Kt- zCt8`a=k8UQiOgpwu1|ewNV%9^V@4JeX=^?l0j(TZO}Yb08sAK9@6`#L=V-(n1B4EQkmtQww<6LbR9d!yrRbw~xP zHhWMU#hc8T#@W>ubo0XVVx`b0a?@Q(VT!!1A&)T3A(G~L8;I|v*~yy^8`78Ryu@n- zzl6RTq|YN~*I?1Il&*zvzb`cf*my&9aza4yx?qumuROguT$3OFnDPGWyfN`g z;PzulI2UYIi~wxyQ+LP-eQ^>18_0v~;gPUko^|;z;b*@o;DEoLfy=KU(d(BbmcO9$ z;}$^)Ur3o*#Ba49t?1zS;u|_ozr%Jv!*SZ?*+JI>+JHMlXMjLkLPMV82^2J!g2xfi zzX@^7Nu|G0JHe+(`DxAeXK)|R<{$16mTwBHTV!~e-E{9BQ>zlWs# zekP@VE35uDevgs1mPCpA`XGggoMrOtxSjd#19?h7)wQ`?vE?BN9-8N$i8>GS1KPUr zfjorvp5w zc@qY+mY#0-(Rv61E5E}=T46^W1PZJ#jiKuvQm1cTgtbrigjC|Ch_Mmwv;Hq%UuqVV z&5MIfIU32yk{WqXC~Ro{#kABvvaRAjd+w}%Wy;GN{uOG#9n$|adf{()tn|;i$o@vb z`%S3k3uW8=--XLXyLNus`1U{8CP6io$+_u~IQ^)LbK1Eva^+pjFMxba7-z)6K-t_= zrZKTM-Z!+)0GX*pV0^aHk=oD9i)G%|bDK;}8%e~j)?7E7r(r^?)6 zv=ytGCVh2mNK`XEow`x-UhrXqwc4GeRRZ^7dFuaZLZ|adlK-}ynac~%Sfn&jC3^Yz zlO)l|1|pGDaYOb^y|p{MI5nV(Z2qbogXQw$p*$gn!Dm4Gm8!mo7ZUUBq zkS#tGAhw9=m9x)OT1GAHfsoqxFYTb8^Q#1uC6ptvM^q1++wmD7`ACUCn}~8+(ws2O zBfvIA3*g@nUF+@tUS}%Phtji(KS4DM`F|=`)JJ~2WM#VMDID%(Cw7J1_x3Yx?+)UZ zran`;GwB^WU=u)-%I}j}@$W#Z_`4c%`At}dzir<%iLqkG)zExNTy^Eb`I9?W8${?M z1ngwzyScY5%3?lvk(KM9+@vqLXngHkxBLHgEpf*@PUTw{bG8dkQ{6nzrF%O=^La0Z z7a421`2{kp)?Pi@H+^MvT+FRGz9Pmd{gUSlUM-pW{MrT^=f@deWYaD?Y`>(xwfgq% z%Dkq?4!ekjDQ-Gxv-IBWU{<-np6gmo%-LYKAnYjKMRcHrbb}S+8+6h8&(PFXe{$X* z;9?~3dVcMCThu z-XE=M*~NZjt#aIKzx{hQN%_9rY&&(%_3G3~)#agyLfh`NWWlr}oVPVZP7iuO?>rHu};55<_Lx0|Vv(gxL7_9XF7Pdz!# z9b^Awdf`7q4)7iw;Bg>||4h#Y1{p8=cXR8%*Z-XVWWX5NGl3iY^WV;nf4BdM{jq)+ zV<+$=(^18vp)r~^Msvz&Q7~FCj#d++m8ShMl3N9N--DYT-CfulA$B~b%<}6R&}iO1 UT!YZS69d$$AOB~7-zIPq02SSM64GfdWB;I|PbraCa$Qytov1ceen=-HH~6;>F#icyTWd#qFl&p7VY0 zxi`P|v$OWf&XO_59CNHF6=j+Cs6?nRFfi}sWF^&LU=V-7z`&!SAVKfQyu%ZMfuVqr zlN8hNggxm&4$MXLyT3d?_saG7{j|QGHWsIUyB>ppA}&e+iz@|?1mVp*Hf!3laAuL5dK~?TO{b*@rilo>i$X%lHz=b}shSw(?N~~yX zbg2=B>i!8DtMbodTLvGKAZ}OoG%g1@1^1rKb#_^19Af-Aga3Pw?mZn)v|*`cKtZBd zfd3k3p&s3CZGeeZjwniW5v^Bg!SKlW@214a)ppURhKub z2)W7GXtn6ujvtVSN>(^^-xy1r?NWA2J{a8(Jc|iBcekLB)H&snRuaIZ$J^-+$Lsge z^J-3Ly`EI@?a(vhAl$A4LXM%U;eY*V2X97+91S)tE(SHa2tZ5{ET^fexy~3J99VXt zOhHnL3Kz{rBAFuA84g}TEQ00@920D8qVBzl30L1VR5OK-R2`eai-&JID0K_x_ zTg<+HgQX}%;(RT)yMz8$!{hMm$SEL2@(z5qM<^7gJl}) zWkb+(G$2@Qaxx%*zK9oR86!#=;+H^Uv@$9b2Xa~j1O$?>#^7j2ess-XP{~1%b1+P< zal$*ZEh2o14oS~Mo1)zJf7mrlkN})0s3A!7=0HmBy$6_38ot#eRS>h)MH{I>V%Ff7$V=n5Jqs0#f0CpwKxZ z(U*gK6X&l~z!q@MlPrlheq8hA;5(DBVKP6mM1aeMpja3|c_HXBZf$;|VBcc)h|7UIB)TF-N8csXGIv!GxpcU(to1}!SRZj!OK?~{V#U$R zuf)3EbB3zPVM7es>;D4ZOT%W@Bd=IOq0A)xMf1MFINO!RjOGw?oou&#lP3E$>7hE) z4pqobf+Ag{WIe!mxW1uC-*f9pr+(K~I=-6vYf?;6@D2myd+2*M!K|hx9v=IZalgCu zsUUdd3k^by=t4SIrN%mHvQtjG-?bbn)+#MS)W1F;;2`PbAO*Gb=oR;0+-tB65Pqcu zpDX?rT1{UeIPj6=JIv@%EtH4s)HXJRG(35OOJI~oOc5NhB7dQin?mJpVUjWQ@T zNICrQhi(aJ9tFcCx+4>C$|$HA(EE%Myit1<&?Qt5Z*W7Qt1nz4o@6_2baIjoi27BL z?m<0Rr!^MX9J#;t{m4#4R$E4;pAdnI01p`uB33aVTJ?*;{7pv8ZTht(SwujwEU4 z0dIL;0^@nS@zrPcx2J0ncB>s3(3e}N%wXy=<4Z|NiA1T8|MZCf^Y7?gz%|VKp?jgZZzq}Yjyu1(XyG@uKCKcB zUwDT)-iJJIeEdA|m&x3{hb8w-nHZiSJP&1m9P?@_z>AMtH6lcBHBt*gF?WR$@DSh3 zv2#Uy|BO7f`5;iecW`&zR`KUG+#w)le)GA#$7ot6x-;xxB%F;%%it^yC}8 zfsFLBfzj;h=|X-?syG}%%(l8+kbOh*gh&-0J3!1}TsH^BdPmB<-tAnPnjsaEv@&7C zfg=5ME#$|9Uq@zexuui4RQUTT;i*L2^eqsM*#l^M0OC&MotItCKA}bk6R- z*J+7s^(k=z7+-gDOs{6g%H>XW!rluUj{ZlG99k{=_AT@}jfGZ3BtA(a#g$*e@5S)z z2u*VLr$99A)#B@7nJ5~b)?iA2e3+28TqgilMXuLp0w`@y&w zTsEC>wdA=(w??q@TmecmXv?lhOnKf!7AbsiL@QO$RK;ndUuk5SRB2p8z$Ky0Id^gm zirab@j$Ghg?CTfVA%S1mdo(-=ai0828%rpF^h-jr`x^KS6fJi2%A8&IY7FO=lt|na z=rk3;>2_+$^vB6cK0QA=+OYxw#fN2QhyCiSoLV?675WtxoO^a#2?0vKx2%*Bx~2v? z(mSL5WE>EY!ed|%XvnKWH5aw|cx#1>DnF1BgaraheXB)6!pI-};F+typVMWoMV2NQrHt{$=Wb?cDb#K?5~XiMw6%q4guY^DH~KvYh)ame4Cq3}PV^k9T%(K}Sh?hXqhh+u?7iR-)RX zpb8C!#e%4l3{fXF)xqKSYwc-IXYF}sXZF%sab}Yz%f821pAQNm$o9MJU%3qo46>tB zcwd53crTH~$J_38JFkw=mOY<-_`WR|8ZwAWog!GoqEmN_@3(t84_dbTMMGP@y|%@+ zHvD=fFjz21-OtA$FEts{C7=c-zMv+^*QGAHhJ{}-P^@jjw^ddeM$CE^V`-~ zc4A8iBKafi_#;-=^Fa;U!pbPcRGnP0+7e4CO|s_1HwK);(XqkJUx>ZjzRxaigP)m1 zMn(J%krr?jcWpLaD7zkDnVt@XT(ze5lEwzFmI#FV;0^q;o#qc7&NeDH54;YX`$WjM zQWTx(a>npns4D%hgx#Bf$&yQ~WUl>JpWmrnJbhzWaH8WNDv)drf$Hvc^~wtlw;ZGwsKHD9WdZDM@|b zQDMZ5RvJGDekL(sP)0#;2CFFK6QYLjzvZ$nz_*G2VCA9T7rUoG;LoRkpxlLMnn9-Q3;P!9`+O zj(K8bL!wyWR00B7#3DBb`pzR9!9f)Ck)nb~Lo1Y6k_uy6?~sKq(655x#pU?wT?t9n zlwKiy2{56Q>T{Cv3O)6rc<^k2YScZ4lYYMr(x@sx!ElABC9oz}RC{r%xGIR6%50J- z$hboxl1MaalLMb#@y_$MO|G(Nik0<0Q+UvQkJK2on3u?6Q-YsTA5^N|9=|SC@2ZNLk9VNQ#mGm6!WBgagJfB@ zBYwnPH8tLOL3fr!D5KyHRm@*7$!TTY6mY2F@8G}{Wx>|L#wZn=V6m$>!f(a!S%!v5 z02+O+@D8g$kpZQ~xn-whg1tm_)$qmkc+amQCKmA}wIIpDhA_zv@*jm}1d#4i5pj-` zKqKMS0c!T7r=ue|7sj^L+azH6N8>#NXgWBg5++Xh#m1(*yb3lbJKJ{q65Kan0@O%O zb9uR0ROM}h=sEn1qtEk%BF?Md6CgBmg-9c1t=-*pm8XvZJIZmScnr$*n9f>Wng#z( z_x-)OahiR(Tbs(T%o(8<@e;Dk@q?zt=>=+U^*tsI9&E<~9OqVU8vk~_UiVBVE{t0L*A zQPD;FOkFkfijC>vUwhOHq<}rEq=~YOALv~6`Nc&9Lja8P<;j*iB}B5eN*_3>kpRKe-u%VD;P$cmzw-U7z}YE z7u1aCmdK~DswW0c9g02-kVkTOU!lzA$XN0H9ut_zEj=^G{0}PMq3)S0N3;zf@dZOAm?r;bJiDR3W}j?}Zs z9{f0wUQgw6LoE}|sympp+&4`!N~Tec%?_TvqHM$ol?q)Af`f(00l*X}4D}8zU_%et z-ZmMGCy%!g_(np2AWO$?|M+fj;eA3@V|a>@kUP1jL7BAY4WCffqsuB{Y5MtA{h=;Hel@2@Mwe^XE$j zw+)@o?Jt}-X-5Ohv3lTc8%v3`nwFL{4R(*4L0dZ%f9A6BHUAXykXI^hXHW(!RMWN` z%5!9qOY%*+^jxzIjt}96j0eRjjy64rdVST4m#j-M24I8%*ryYeDAbkh8*x9FnFwG^ zIsEQ_URDQ<6Ulwt(n?@b2@itl9Y@?bJde5l1y5WDhm>o_`F1kGBjQ8YS%ywZ)h!hAD(p{3iG*<^V&5Z(qo~F5Lt@7g3}C z;Hzs1hh=&-40EkZ_|SVA#-3pmvW>%p&{2?=;1p`YmV zF>@LTc~kFs*SqiE?X>PdHm)ut;xF8tNg~3-k+6W8dNDiTlht<2)h0p~&3R2GgSO<- zJl%mcj|~aBq^&(@SUfsE?yF}C?x83ht0PsOhq54@j(c>i>W(yl^R_G~WqO+Ai}+A| zkJWWQz`ERKN6fqNypf!eQg6FFAoz6hB@=^mI}W>VbNgdJ_+6gC;NvO=<(~^(eqycb zq`dV3in+`I%PwD*cCt6pkCh_*W6WcD<8BWfyB)_}{X>(L<_8se+D^wfB)d|2pM-`n zZPOI0U${2XdBYuEJ!wzZ-YI_?DE_?pJr5kC+DbS0+_je`BHa5E+0M{@_NivK4b7tN zCWzJPILWc+N5Fs-&Xx7LM`$!(!!UP>_A#CR({Yw(L6nG^kTL(_nj>|}g7NK-G`8xN zw9jkL#0w50D72<6yMIJ>3RT$`?1=k74JiQ|3xlHJJKj2rPJu;}596(7Ja3mZ#GYYH z@N?L+ml#M=6{WRS;(Rj4i?v0|ed+_~<^UuU63g^R#ygBOCdzzK0l7%6o43r8VH1K&z*Qjw^k8*B|j!bgL7l-RSiljAy2RZe%K z+>hOTT=nGAaS;zfLwi0iX_f;f)&i#&i`&M#O(%V^_c#g-d~Ht1IfTk6X6BsMJ=hfu ze9DC2h2hkLyJ?wF!UHC%*|p&&JLE{t;lfILsyRCR+T$7SMt+ts(-KZgeLPAWS(3en zI03$eG%9WFk=P^kcTH#g)fr;t?4oHnG-GE**PN?|#t8{}`hBsjZX~O<0FPe6SXnZ& z)!UzcmjYA9(i!FmKc_?l4T;*o>ouQKCvrx)?BhEb9cU3j&;v81=36opnaPdjSA=M0 zF?JVgtrieY_PHw^BL72e9wH`)A8I|w#e?dBbc*SsOgw?+I=J}w(`$2aJKzre>#e?B zvjYmMY!-mA4vn*aOt2pwAv>}VM9~op>$w?@OJ5lYGfG@i0Sq?+B2|pOD+#;&os_=1AVCw2 zn?~XBO7yfU(yQZ7x?uVi_WOp^n-k;iZ^Tl1E$~~1SVcAkTmjZXhgW(^iYF=QCKv^a z16_Jwx{QHMu6;U9MNktxXdxXYm_yd9D4AfHg^Bc>qeQYbpk2-#WtIoUv4vIE#fMi% zj1mjo1Q>ka<(V6E+!n}zTOrQ|G#gLnr~8;I31zs@tNMB>#SD&&(NTHHPzVIW{p|Nh z@(7N~Ug%(m(#g8PM~)0hNzXowcR;r0?vh?v>~xp+JyPU-#E1QB?MUyW6MkcNyqY4g z+%x@k+vIW>a+=N7%-$cFWQ$OzqsYe4L`-(&S6aeS?CV8E=wU<2q^mxtwZ3-WjsjO7 zv;7ua>~l*NtB?ngnEky+<*BN&Jo)xTo)Pu%%XuGaddIS}f=rU!fb4+x@GMcVnkZu; z9brR8U2QCcn!O}XRe5>X_OWgD@69*u>qBGrs&fh~o)OkblhF>fFGG8rg>+db9aw&A z)~L}9hqg+yA}Vs@OYz$m6(_K5O>5!=HC4NnBo?W#nUqE58k4!wcPs6YlO5?Qcc?1@ z0#Swf26LFvF6IQUMAIJF-8ndBV??eB8NRJ>SgGoPuT> z?~p<|GND66mt+i1FNrb+b`WV6Uy4$vTC08HMV;eX*6T`h<)1jJXNn0!!-tB-UMH7B zP#L%*JCZG+$M;{V`EAY`szQd43>kwo@TRW8cse>dih07MQ`g#cSHf!LxFIX6t2$l& zP}mnj=U-SJ8BMvnigWOdV^F7?b3MdzPz>FU5t(=@4Yv~fTF}HLE2qOZqMe;pBSJX-jcybI$m6dPn3D^*b{>gYIaW$QxyeGHSyM{prcBtE=Ga2Y092|x{k2!R8H-8X(o!WRoir5$< z;tQYhyQ9y}@rz`A-N0+4yS{%Se@%8UXilai`U@(OeVyb=aXdSQ6-uIpI1vhAJ3I;Aw>Psn&!F$myVi1N z4E<6~`D^bVE}f-Lv->GoRXt}SabAhWQMUQST*-&J2>-L>)ihG>vT^-xy*U4(@bIG+ zra(^54tT`mxnK7x*g){}Mnnu&Z0vR~<^*+}2l4mM;7~hV#n0Y+1fP?KIKT5^bIPZ4 z8d}>35BHE^zS~pN;mu%goe;Ftj?LyulXCg6e0RusqNJ^u?uV4tA<}z(=7%BhA?xao zs>VEC?dyn-SN$5bO|Oo=unU#YlgI4bpI~EE70#TfV#3V+f{yysiXg*3N?36m_LmRi zV>UZSZc|uE`uj|L@wBj>79QjUiD-n2ZpjM^uqRd+MJ0+Bic-M~cF0Sa?jy_$rWyu> z?ag%mQO_{7%FMz{QnqZ_j>b%qFFrav%x`VYXlqktN*9S=S1_`+P_2oUy_Pgqit0H~xBj?^Nwjj|2D9QLU{~fv~ z=%Li`nIZQeh*9G5`I^=7X8x1&#MOYUS7V5#jIMUlu%)>%zPVV)AOts#)Cq%&gr@$9 zQrNHWL6P{=gkAFnUx*Eb(92P)D z`9cG5qw5#U?VFozrUzigh^6sOM2|;$+n&Wahmlya>_p*2tN*BUK-;Zz8ZuLba>#eswlHx_FfZGB!EB zUauqr{P?hTp3Gn5@9Rz@u@dkhuJ748-D#jz3y=tb1Oiq#Bld{N$rAhS{=U&F4Txsw&`kp~}?A2xhJkC}Pi4Rb>YS3yl(Ca~Me zXck(rwVJRdrl3amlTuE%Kr>4$&G3i|YycJ@J67VfKT)#2LHlOoJ5W`#w~?>N>$&s2 zHY^S<);brTR7$^3eBmF!0;&pSwq2d20)5sZiIk-0){$2A+m&Lj&$o;mVh8|*=vVBN zTlZ4+yjp)g-Lh-sdS^#l zaml;6;oaQ$L=Bl{<3>q?SXrTB2dAv2wT*zhjSVplz4B%dGWqteJ$CQyq5H9z-rr8w zU8>JB=Z*l!b(v~&Rdb$?&u3}Bn}1|zW6M`mWC$9Q=+Lne{RsTN&YNM{e%?1(wby&M zcqLPh#z4%ASajpxS>KTPQMohtqasp&zT>uUiR8+1=l8^Q8v3PE%!@cXuWX;0r!C%Q zNB-ZEB7QGUB$%3>rV9X)u+M3e$OV?d6y042MKzA!E2*2@nEhY^?prfRm7=$4F2n-KWhh ztmrUX$-EmV^ zB)r%NSazBaMeh~G2%-qoZh;qSBs(thJh8=>PmnE%3t{g;&{qxu6xOyto?X$1S9Z{y zNwdD;P>2i_o*M~78Ic-Oi>ccLO9rQ<^9{s;SgGb)a{E|}@Yt^Xt{G&xuTRi?4$Tv4 zT~3KcS%j0yH76=J79(A@K+!SLe81+z*yzROtnnv>Eq#6?54$&#I>C+l!JKk=u!g zfa>j$oI|#|cL`BKjZ#R5z1M+arZ2QfK%mk?AV1Sj*reis!X2mC4V&c zuNq6GZ=RV2z;lo+`K7xMD3zLR;E-4;a}^N6TZCd3ZMw(=B ze<#_0S^>fbxs(K_B=B@~As4@0ri=x67T=cLn6%mQpqx)J8w?n#^K!p&0XOwITJbrD zt~$IilfT_-#**laEL5#}?Iy|`9-W%2K(&pVjYmDGm@rjuPM~UNI63#RW;??&7h24^ z_SO)>H3wmk2@VIm9w_Dz{R*jQJ3~^{F}i=hLak5J=_u<*)wR- zb7W0}TW9k9fTxz*0uIoa)%q@?WHuu5LDKgbXtr1pZI~e37|g=^F`O`cb$oXPog-;cef@w zNa2*gk|6VURQb7ux!e3?wA6*kpAM#%Qz@|B0Pu9nsH`HGTk z368`~%sq|R{z8Ir{a9@RrDOjPJcbgBTj$5)yl?G32t_5NW^3pE;x=PcslK<0ndi?U zT<(zk?ZO&%#!t&dWb!C|wbmPTUrpJSQSc!pUw(n0YS)dK8hfP?N*@$gp(1j;|6gj~ zi!01}(;&g|QW17jIY8|)Wa=jm|mDP|KD6-p_{?Pfy zc0I?)RR&zN3w8)Z#9tRa**|(`62`S%lsb9~N8N%M~&9KnH8WpNO4A9^snHlO0#(oHt$(SAVZv!9kwR|LUL?cM{P zV0!?5z;##`)x7%DT|j+i-%vgbO6cf2gqT1U!}*n``==jwuzayum&To!KWiF%y#s6f z&sf|qI9gN24SX@oU$Q)(>quQ`^L?KQJI=EkMhssNh=s7`x>1pX&}(SS_O#d>Ml#IE z*G6`1+}ZLeN9iUG3d0lF@47kiw;m!iefz+F1gS%h+HspMU0xh9^uJ~?@&nW;LmGG>ef z!QZkHcb;G5ub0*o6IJ^JOI3 zOjS=y!PuBGLoxiU+8!MMbyG;H&ruG>k&Zd?t@tOep0|YTjis_Xtdq30@q7I#FLlyS z6QiZ2&GuM-0x>Xbowa|eHSR-FgW5vEpsIc}dy$OG<<*|=Sz-C}P5il8m*h-GqHH~; z-TLVA&e+2<_t+uWIbF}bHLj^hhR-0}2FsfBsOCl#FBIxU(p0IXo)(^xhSeP4KZg{Y)qm+mfTV^TVfRzN)d4CS0fDOHDZ)dW@w%YZ-?Y=$&x`6iQ<5C zldPhZQ0_A1_}7tux+3Q2?8NfwL^V12U~YXS&O8(3KsJZEy|R{_Pe;|;d1~5!*xF7#=ETt{b$TVBz|+JP5s=rN*f z!8@n{!}Uy#Fzjq_0Js7aEf88CTWzpT`o1tx?uwGl7{G}0Dc71BAHJSIC2M1(XHs7@ zQ0XLqN2)&-8^tMjFn+etpBNq|`iN?EA)7I#R#=()E5<0C)Z8^G}t@$#>I_=mks9*#&<&{)by$5W~rY zL<|l}>+0&NK+9X=P&_j;r{&}0gBBhQ6)3Rs%gPcuJB7cF;7XWBZdP1*V@Dy#KpLHI zmo1@oFhXGDrLMeuGy`jm(LnSQn6^aD`e^7Ot1DyV5!rsbdI%!!g+X7OMjcGTZA>N*?jeU#4Nf+>7foeR^w6ET%#}}&D$wB#l zpOU7vHD51j*9-Ha@ADUYChhJD%_)l9q5S%Kjz53?v>aEr>$KaGRv$Z8AizU&NBnq~ z1A{vi1N8|Rf{*4esZqb1?5WlE#5*D`b>up|PY06skREz!>`li~tIE71f!Wo;(n*tq zJoYe-Vp2VU!|XT$g-Z|lgqVxlQjGc8WQZE7Q(>;CHrL*Mk6pLi9U;g7%?@o6X|1hjXd2w7^nGrL(TpU%QZy1*q8MEb z7hpw><$qHtB?tNH6xL*Tfvb_yh{T6IR^ssD&M2=?MywjWvy`7LyhWRPU<^b`ltox{ z0|211K#BD>)J8l?0rj{783D3w({^ZZpdkh$loc;5HK*F(&2Nr+$L#Xb#u~8$)t_VH zrq6+8jFFD~P)xZNL_#bQZ}Vi0Sdot+1ySzM%F;7E#t%f9S)znVpwnwY9dF%kLSF`x z)rAWao`Y>GWYPdA>AU+MB0H=_ z90Z|68XEzFfx*_Osn!We1mBuD2ZD0efF(w{NNzZ2UWd0YlFo4bN$`PJ-oKXs81{aV zo$JD{2|sL@0b_`zX}N<#ht~R|y&svWd5I{$+f#Z`&vhdznkUGW1n_2gR3aT)y9>7E zj@|`C%*v%X-o^@2rS_3?0-NtmPd-0L-TpB5WY(exJ>d7u9a9+wdiW6LhIZRUJZoQ4wzHOaf zhdjk!L5iKU0$u=r|G2ci3cN+V{9wI4KH!(dF1*kF?z!PL=lQx7k>0lJJNdlJZ+KrL zBUiN~`G!1cZ1xQ$CIlR+qt70kxCJC$yt^DyWa@*_{I!KAsHq`_zC{Cg2!I|vFv@ew zxxc)-^YRz);r=eSQX-dCqT~UxV=y1V1uP#hI3BL$cEWb9%&2d@JTAW|5PF?%HDSg1j!d=zSeeN&>Awm`o*Wv$`J7Xu!vAr`Xl9iRG z;9f2%q6_9B0J1+ zBPF@?biiP<*!bAcn>zE`D^Iuv*}8V@>zVy_ChtY@RjPR3%}I=6F_<&HvDcQL*zlY6 z@YaNP8xlK*f%YYATibSrPD@Sv0>;E{Ii5BlfT-!zvTW5NorBRrsi#T6%gYfMCrW{Z zf&dpJ>evW1P3Pwa1_#5fGttt9y*%9|ef`=ST1fTh?!>s#x{8pDtY^=8Zhl@)+=Pnr zgT?gvj2I13PZM)^cm&k;-xXm@1S$sW>e@hq?)}5m`5tkzZ%WPEu^RuXfagQEa=l>{$H2luDrh8t4qbPX zU5Ro|v@x);u@UORWl4)ME#^koK@tf6J+2UCylu3rG!U?7*Z(~gr|py}_OEZYk;qx` z;!Y+Q`4qgA<6~`iS|WTX&M-|vT#T#ryVtc*O0u5)Jzqg0Tk77$Paol)zkEqcEp7Ox;J|&)ckkU2HMz1> zox)W>q74^*h#(0-4F5fjK1jlt$S4!FPsQpzhEW^(S2ZEUJQ2uB1(|pOxal+>*He%M zsZYX+1QNzQJ5XDb5c_N<&7T#iYkmKW{mU*?ylL~uciL=pK)MFpZ1Kddi!_C$_oGMUgr0I%64@Vg!cn8 z^VT1m>O!c>4P|$bu(3TV`vEWdCcD+il^@3~C-q}MGBV=_G*-1AKck^uz}r(BI;frO zLrqIIpUeKmv;^kd%uJ5}d$}8NSLYKR*=N#ihglvU^41(y!QMRPgH>MUh}l;CPEVEt zWj)Dm5mBuXlixouUGB)qEqNWZ`rt7Er6XjBTF&>+whidiJM%k~m%(E#`2JKvl9_jz7H34xb*Z&J3MD|onH>jbr*Yxe9L|m#Hp8$y<*#M5R|Gh+ z7|0p8CQU;-IG`-<#UQ8uzCo6^{TP$2pg#F37+&N2e1Jz0Rbfmnrm>;Z4@Ra>_9Lbp zDLdU%dQ_G7@~ZG~nd3c7T$M|Pu9o6XC(CVw$)>XTY$z`OZPVt(f=K~emD1XH9#>8* zSB%~FHm?sMP%fwE#GfP|ZGBc_NwdAFKrCb5y)P=mdO3{(|MREkD1EOZwcV+4B&m^u zLoDsi(A<1um=!qs|3H|lNgYVnW*uU;dh)w9=4YJJcrJI+zl9Rvq@zPaM_2fxugm}b zGl(Tuk_jK`u!CqbDI1Hy&d+m0SDb)(CSAC(BU9tT$x>Ysa=pBsI%s6i%Z5%#?--CL*G?;&b=ZT4IJC%5n2|A_Bb4a2`nd;qS;QRyGB?U(=`Fx&+JGve zP$Xt6#tkpHKonHWQI(v_bw_#(zeMf`?Q?3X+32f8EdEwZKfB1;f5|o7l(pSTuyaWd zc<;V_2wvVvV#vw6iXNb-Vex`+{^KH2~uEnoevU3`~0DJ{T0jrI!VE6S^aXqqwTyjg5$35cvllWO3 z!A)~zugE9=3*1vu1HqenKPJ8%I{QZx*7%K!uIH}vm+08lnmu9>hT_G8p3m*k0{#~+ zf3m?k@N}G$k4y&!e+KF(1X#mTBm9R0s(hn$DfoP_uNmpb{P`1&vo8LZo(wtbkh;Fl@XcCED@pu=a$7<@R- z3pLI9c28w%&>k&h2YBUY#VJwhcffSYw~pkEo^G*wKS3>l6;lsQj(YvXAj4*ljjrLV zw=)7(PLkQTYgt|c8R7%h=S5a#2r{sfD8yQJ1#WDy`}vOKaaE+r%pls83l2Lmj5;dn z6W={^!I=H}!w$4}I{1gcQy-7H3b9L`O17NSCzOu{ite@1;bQA*zO_*)-_^^qReKN# zQn90KPE=Tx+~&cVAylsK|KfPy?VvZ!;qByuo>n8A2%dl^q;$Aa|4pzc!16Ef|LpC~ z7L(aQ8|Md_CDvbab0Lez#ZcOn!dgQG<_Iq09+J?~!jFo27fTGSH@l!=iaev}?9_vw zDmgD6o^c5XOf%^qI~U3sB~YyetfgziW4AKCxk(|%Lfr5(*L6`tu*76-FvH5HFTL)X#1JLMBmf2(V6eY?64D$JCj(_hw`oZ|NoFP{}MR{2^_FyvoLw)wFZ| z8=gVO)4h7|#BLxd6|&rx z8A|4>BbCCK{NV|?ku50Mm`$D`%v=Wwz9Fx#{`CzFs`4|ds|rx>XR3KYvPrTUk~{+> zO6>%#)pIv6k5#E@`sGye^CMsQ?B*355ee~kYhtYz-SO~wTjIYLwx#5H@bc_h($dma zt6Q7LdBe45hy!BLs*qDC<1m#gfYOVftwC1c@Qrix0CoBsfm>4`Qc;I6Z`R_;VBTim zsawGe2zN-qF`x7U88x#H_L@?A6yJ+vpIoJ{3Aqp{CC|^_(Z-+!!HwYUTG=IWn?rjv zY}CO^M15Po;+)@?QR9Z$3O6elU)4IS@MtUbX22akQJicrSM&poTf77alj>)B_@#SE zOwtW~36oUhI^_6SigLQX1%0`#EZa|*4WQ>OPR?O{4hFpPqzx&2nUoCo$*MCkhG zZ5GC`?4Gzx7CJdld#pcf{Jp(p*V2zogc2VA$D};t0cJ*v@y!wX+}*W=)YQrKpFY-5 zEP^suOz6QgmqnJ&yS3HUv&VSn`G3{?F8#O#BB*X>m$mOV@%tv^&Dg<|TGs{=X5HqP zL6>TNsmyh{I`pZ_9WBkkoxw7tFU>y^ z_*q`K>m+7-5v z;ndz;kQ3eK=QAqX2U4&ItluXf5ri4kNGEbE^5!@uj zWKlqfXIa*qOOUr7F{S~+aJDUEM~mPk;-BtL!sL+<90hUmrOoy-J>D^uGW2HkA0Aun z)v^iY7(-M%>6MG8@yvZrmBa1ULBjp(D^-nL3r36>PE0Y{zs0W^BXV_# zsm2M}H~$6Uji6O771ZfWZKm)l?uws52W0k00;9y%;n2GCUy1j?6{=o<-kT@6<_Z-V za+RGoanq&AHHLnPNph2&$t4)FPZ9Adl;vf%;anV__oD_SHQRI53aN`2>VD4qUN~{`BgybAaIK6;bjT_+5jU7x0ac6*IM(Xs3kCQIA z_hBgRhFz_%>wtCj1%@-(#*^nHgxXv6+|ru1{#+6kX)6T?aFY{&F#(bim~wR_mORNXcs8)P2!8tfxQ7?23*)=+a9v{Z{Z?4pSzFqNG(hXgZDq9-;yr zc;@`SL~dQgV~H%vsxk-D&I;WEdHG_ZW9W^pEp>5-|A*~rH_an>T&Mj7))bi@|@Y(-E-CIXR)xK@RDvgwM44`z!Py-SI0@9_l zboT%flG2Uz00W4WAV^9`=g_HyG($>v!?(TbcR$bbertVyeD7NC%No`im_2)6ab9sA z*LfW0+?(fg>fH(u5FI(r$FFO9@kvRYE6b({2?-mUn>iIl5ic0pVLd$-qxq?tqX!&k zVzpt;aG~DS0Yz@h(PNJGof*$x5rsutE1b#(&UmRDtX7kW5-S&a-O)2U0OyFS@Y}c0 zU+I*ncWUzWL}}t*-_JYP714$t4SBJ^#1kV}9~Do2=Lz`R33&{$EX=Hd1s))6?Ckzn z+}yyV23%wonuUZoA=b9GQg(w|XxDz0!)b#RCdmaX#{m|f@I(3%`1jPHNl?F^TTAl3R%-CgBGrJ})fux%P0FD_d8wbMmm zqoW_c`A%Ycsy-?=znW>V;=)-~C`#m~e2mO!zytmdYB)@m2P0-1tp=(I39MSy06a>##1p z@TLfDtApw$DFNMf@Zqa0D_Ud_aoH$ksel5j{Y*iT4K0S=46p!I7=KJlAYLySU>Q%~ z05{@sp-82CWHhh*HrfAE96u2F2oMgDE;o0aTPn;VGpBxpnENUap!oKX!;o0MeTO!z(yS8)xv2WYRKqe@{;< z@?>OwV#p{Frv7X?Ax@)Vm-lVP+4~*1G+mc z#g<9Sc|uq4xE$wC`ZxD}RzOG-naTLFnIuP`ay%1bi9oEgAqueKsoQ!%-|D5hocccaYkpR2ANg#9;m^V3r zjbcJVq)mEseo~S-8=K|(jKq%p9!$V*1wlsoSn`SQSlXb1DEUxR)`yD~bz#ll=l<-SpaIKHK+Pz*F${7*URGi__zjn;Vwn`tUMi5-ZbEu? zq6We~WWyzAK~Heif%dgjdXGII{&26fjR&+nKNp;C*iC7R#&0(K_{ke4t4wwI-scjb zHXoAO)vztd?WfA)e|$D-tF*7400CVgp8Y%Mo4I-ds1mSQP_l`c4CaVa4n?$CJR8w) zdyb2aWJ*VNKtc9TV<$i#1u%rCYJw&xN6vlxl*YSsUhup2?5lQGnz0avu|9K!0rm({ zLcxv+(h{6o66=INRyc^YFgIU3cx&xsw_e6;K*KatkLj<#HDg#u#+NVv@kJNUMCn_4 zrRN&15vi_cm-L`ik`>K*1iSQuVpC6fu?%}g;%hAv;i{p!6o=sj<#HzpX0SkuOOC8F zxSS&@Cr8qLMs`1zCI5x8G9w6&DB}5_Na-K@`8N3u9%6a5JZGSm4t@frE_p~<0p`vqn9JT*j%rdc9rFH%XviD<&tdcaY(@c{l z{=hcX`&ysKa&J-ae;(0+e@~)g{2VU4?gzj88aGngf<6N8`8@$sw)N7H#_@th10D4L zz=!|xV)o|&O5(d`bmG~mXCg7^Xx3^h-3hpa*4xiVn-Bm)Ej~A{KSV5 z0FBgC$N^Bx!3>+yM{xe?H39%&o#*|HIwURExq1=7T-*Fd#`T7?IERfxIKXht7kT8G z%=fU-tod_^Bh@KK*i>Lzmp@3`i%DXFG~m@qU~(Z7bWPQZ#||fUHggvM+^gbd>$fTu zvuI8j*M$`l*tyL68`A#U@$z(n)*Ap5>oT-Kk`?%1(VX-}uZhcQzD|vDM~W52FYy4? z^+gXr;=*TEgd$&zMF~XG#q>OyCa^f|y=}MF6BTzue=xG26x{4~e+PKxOcUi(npN`; zYua4jF;xU>9EkA3B7&u@Q4!CBwEojtZCG;9_vplTI7tuU5Q>lA2lTH>0cDz}B5v+< zb1CF7oei$9-kXy;icNkl{lq{dAc8i8D&(CsP8hl16MK^T{LFkEZi#$QQpO`L^Oqgn zaun8OhzcZ}86GUHlM;f21jyR`yUvg{)-Fk)TewmmW)H_9kdmw+KzlS! zvUGH&!X$x-?mb+$=b6Vtu`4?)Tz5j6O+?K{_%-qTGwkMPVEHt(6_Q~YyRei@_RXoP z6i-iBVobVGy&64adb&oLgQlaxp?bo)^wEFn`fjl^FG-3u#Du(V)^V0g2x$KO;b#;~ zSVdlWtE!Cj{%nqM*=HZ)->G~%b;xaT5Q-&PHP8Q}0smd>zeX5j;GfR>_n3Qx{;yvB z=Yv@1u8$`Fi<=N!r?FGw(*JYae=b=ZHmK0S0nR)U8URfQ7_S+6V4?~@hTA&%m?Y(_ z0dTcEMFGG3AJ6^wE!w=p9|GSvJi598kVHDjCNMWoZguhCpfWSg)I=bV=RV^9dBX3u z4kQ0p^ZZjENc%r;|5dWK|Cg%q{Og*34B9#PpSORTIB;1YJ=BzyK@;K!-p#lnVS)s{gbM+|vMr0caB)9=i1P z_ZO9vFwoNjOJ?ik$rgaYt*oXR73Alm^;bIozUqHePzA)*4YNr%a2u8JS)r$W0({q) z=Y^RW{Y;GmJtt>O6n8UhY)pg3_gEbek5S>L1bP}?ZXf#etm4f$Q?a`cFNp8O&_M1- z7py`p;w5#s@mtT+D||i!cTzG#9oYx}t-x)g&b-|qL{33LXi5sHf`UTNzyJgAnAur_ z-G)7JfXN4-bU7s@(Lh8aU`SXB`j3iJ7lyfMz0&ZQ%yb{TP2;V4WfBV1-wtUmjLw8;&gi&Fh9C@OkZE$fozHF zh3hpTr7Y33=x7`bO--y2sp$Ie90e)L^b*E|zZ^}p-=G9t)05|!E1A5Np=s9PW}keu z1W7kj7qNPRX5)98psd=#lpZ9LG~Clgwq??lE>d||$YY^E`u0<@w&3JJtrAQ;ZW)So zEF`2>TboSaG>pt>g`;&bz#8R~ZaxdC(MGwJ5NeFUw0OhJU4!TBYMdWC9=v50mPtw` zMaG<|w>UoCY1AiVWyz8~2xbVKv7OeK>GqDL&O7MxuHV^xqja?>cM*rrna?s8w)~j7 z&~9D9?|k9x@aB$kkbe4TRFmhR}|8{ZC3 zFj)Se4x;t5f?{)vX<3CyOgNwjatkL9x(R~3;~%@`nonl}&&u7Yq3YWi%%hHtX&$6U zwsc3}jq;f%!m~n$S5OxT9>(xM_@%zPsP(n3sxpWPVAbY$PEJ8XG2UXOv6|UgldpD4AD`h8 z#?G*eOS{;L$or8Qk1mYLkBl{@ZK^V|Pv^CGU^*#K%KPDC|J$x(L!*$-SY-gEo1mbe z=;-K?A3tP(a5~quf!~p1#l>udgoHhPeWt)fy|iS~t>M>IAK1FZ*^bqr2I_6|nT(5~}&*mtk*o$(tWc07y}49~C? zQv5Q76`fXQYT+@jZuChFp%4B0(P$gexZy?v-P!ZK+IyFr%8gAc$4B)iyWASf>&miI zw?{_%xmNxI#cZ+iL!CuFJ72tEOP`VYQDrNWLG+FDd1!mcnOdKCXp>#tj_>4=_dF+3 zcq_qazl*AG>Y^c|{*XOx-;ijwCCf7HIU z?Hzx%4J}*}*G4Gp;{f+?S1t$`)0MtQTJE*@NR>X* zUjtqP>F}`0d~$*EE8(oNLmoRnnwSisQOZ*uKPkZ`PLnmbE2U-?kvgbg2l7;VewfCL zNs&y&645bCm|h*gsK~Trl_dSCHslwl{?w1zutzU)>rBOyEzmq)k1W(hq)u$-Y+rw7 z8Fu6Ci#;jdt&kXr7tW|69bz1Gx<|V6gS)Djsi$30{F%vvJ#mN8nQw_PgMI+it>{>p zI%%Ep?sTrC?3dYPg61E#D?*b~Z!5NAL&8xEd*0f0u&RSo{iDiDpD2Q)u<8R#TCFKQ z$hd!v?JXTsgkjf<=X`#BN_|YLkDL!u+{^>IGxb01;PRYRYxW);?T!gbXU2TrotIM` z>E1gtmXivuPHl|PEMj4A?QdWZlnYmgRWOS5o8srF z(Nm3_NiQPM{UuquQ#0GI0T%tn#YuIRk(9qHZi~_MbNOjvwxdMPAJ68n{Z6k1zrdSe znj>y|ULjYzx5D~wua!6k87Hb@*R%aXWoU2l9eoyF%Hr<4?(M*@sj__yrqO1LC{8WE z!l9(OqVJeaWD)kp=IVN)RUK03e@fg|#q4}(BW-VRN-k!#qET;7=*N%3nQ^frCSmcz zFe9*+2#V`q-sL)6xtlP&AA&#FNn^VfV&$i}o_EALJYf8?#7U^-t}Ez!3cjX@1EtL^ zYWZc6->oi9Xr)NPl$<$%b`eFK+kWw-j$hzgaii< zZ){|wb!3DuXm)hq)hokr9EBhkYP`H>aU`xdyIP{6eXU|%Tzkwcb;F16f^d*9>rlh=-_;QJ z>PS7`4`W(HD`|eadpnAM{)4t=pVEVn?Di0?`KeHtdghE@|Bvhoyq%`|F4#vG^JO%v z-$jLTMQX)5<(i#^ICp#-89&FR?ls2f(|V?;#5&sxRiz$^6b(E@y80kLTbb^7!t#YA zI@X(gF=#&zslWclU1_Guo;!3^ySINr#hq_HfsrQQK>v#KHQe_C|9q`>?R`5-RIG$p+iRhP z-lH37CE=r>S)YEPw9FloB!Q8hVL;!Z<@`p((Ci%M+wXL79m44OZsS(F?wsCe@l|MT z0KVTPzRzkmn6O&;B=YLwu+~pS|Lk5YLa_~vcI{#I1+mpC<2}LGGqh`05|@msPS%LT z_w8ue-$L#h*F-fM(R1mp3|G>rnw3i%;QEeNg4#*>qZM|{y(E0{u|iNpg6f$6blCZ9 zp}2};YmSv73N?lgd81s&mCe*b8)ouA+V<)Wtd~NGn~m`##Nqp?;c(z|p+@8NWXa4A zc?&vyzPpS*M{A#12M;AnvT0cz2WE=`tE$P$(ZMfBa7zI*xGN2cPCO7_|gM5S=Etw zeQ zAK#Uv*B_6&5{>wl+B-NQTF${t--IPu62`&3gNGyKg+XaZO(;}{V^+pBm8r!Pbe|-r zz9ya8`)9<{4?e6(DZ@DsZnWCbz9|1G92*;*Ht`qm&XUbue(k-FW>XLHUkqh+MpJ0M z@aUSZ_2t~L;nRWm(iz$Xq|7yZa@b{BvrZC<=^gK5rN~yPZi#h5eRC7W(JJ}BFs9Hitng6 zcK2pnJo%o2rKC@^(&;w`Sl(29&L9d<-ipn{@vJI>i)S`6beUi=Fb*>j&vvR=2eYu& z(Mf;TcF4;y12%6;Mobp2>l#UqTIXugBjY7EhR$PF&x2Q|^AaeMWYRG}|Kz^ZAWwW=-tv#C(VooIRMgs`#!=!h4rU1Rx*^`}4W>X#$g z>y|>PU=se~R#{%a`Q9iH5tS}rM@~s;y4sg)KV6AdDv*|zW?d@0nxy$cMa8m&L8U&B z;zGQg)2{BO*=nQanm9{1^LuX*=p&9&Er!wKcOPxWZFHNWEw)YkG~PRJ0A>-seb^E~c%g5Gkb%`Oja! z;Sv^znlv9=t%@isx1VM^1%)R(U6=N`H7elwvZB;{fn0F@Rrjb^i_>g+LM(6STO_R? zQjf`ee z*6%#_Qrvgks{YgCsL6Ata!>bg2`=xfUrj;hOdLx?!+rrAc#1vTc`FYx1o~4dzP<_e zTK7xJp1s~9wqi|e+JMWDx+3a6-Vsfx82rmv6%BRTKjBt&3{oGW0mzb{nz`r zF-7L(RWUZw0zN_p1DJez?~JD7gQ5se->VH8=N>i=i@r{ViLwQmU^@R; zm?cjgQdjFV+U!7hT|GT9z$P7bt3(R-=1NTLDDkNXB?zl0L{UnY2;zu~u=%#17n^IU zNAV5V8g@YOMkQ-n8s9~iunw>dbuPw8aoTx0hJ~S7W%fr0zBDL^*p#=tq#y%xcMTSh zY;7pAnEOGoD>7#w1j|5Jh5KZJW@WgIml!|whO}KqRt?iPk!84e6+stPXT}Q^E%f5D zv{TG7tQvtwhxv)gY0nn22JLO4)zkztG}+cQ6L!pu1}!NYf*{t*@jNDojuqV_7H%53 zy)V=_-&{YzjCvanBRO8^DeK<|x^LqrycS99>zh_mdSm_Rr>v$E z`3NM(FdYLo(sICJew#31+r9Xg2sz1)qvcKCdXa6+f!1tM{mTLU7ZzTR?rR=QMzgaq zwK-ZKYN)^O%y)gErp_Z)eoW8Un!%cy3hg4{Fci(kDiT-0xv4(ue4)_)girney-(F< zm2fS275O!YU%RxhW>Wg#xc=o(hPf(ahVXt=AAF)mgfD|B*8$OjPeLTq?QF3MiFuy*gB-g|w8D2y zWE?j+)&|q1OzV<4U&TD%hBRGmP5%H(8E$fr!Zmsa66{MRC~Z&WMn)xn)jH;}Zdm>H z*|PaEp~X?9GwGNj|4eKEj+-47Wwz*255O%e*GGW9FvoA~rN6&}Ul~x7?N8+{ECyuH zi`ix!g{(5)kP6MY5drdGfD&*7;8~1~j~{r7HeMh015x%BBC9B_JuYEIM$NJDslTTD zkYNy^Gk&hu1_Kk{I6EOqPocJkx41?(r`%I}8L~gQX;(f*R0+FY^WW|l8XYB5x8ysf zc9gi1K_mO`@jY`d%3c|lX#L?@6#+V@zdJ+Dc-lS=qADC%-!5e*-v9;L<3-gRZg4Iv zAq092&SIVdgtdohFP#}__0osn32s0ewOCFt${2kcHt0*`9dV=Z6L*m zx$F<82xykJ}=6eJ83<-mCM2g2BfU9Vl$k zcFxQmvj>=}&2^DVM!^$VCNGwU(a*;0kOlKOQb^jQA73K9NrFtBkTgnhHN#7y?y>Pa!g-qjqr{1qH?gBcjOAn`y zy~obF7Qzv1L#N+%bl8TR?KnRhIW7vQFn@Ug&zXDUCbMqxz%VYVefy=M83T{k&zF^= zEtanxB#y||TNe!tr%6m^H?4SByA}uEQr3*`xRY->8qXE0#FEq522Ro}GkIxKc|bz_A|#vmCS&g*AbrshtQ zD2v&@h!)G&9p90=GQnZx=Zj`Y6j|xR0EqH^HG_?@wPLCnN<>}ui7L^@b#CxHy+RMy z2w^S^M(b~^bD4!l3bh1M#-_uTYoP&dbSj?anCxpjkVniy3#~j}SmQhSpFT#K5u(7U zO&`&*V#G~`g^er5oQ(V=D`+$ksQMfTdnW0QzJ$om@oC3x$ zCUF~S`|ET6eu0z{-W#YVI0RYOwy*K>RewpEg{Ns+^jY;IOQs{sAB!H-x#|f#~Q44|Bhr)0&mmW7=xC&TYQu~S^r>~0`RTQpZT9P5w@Y9*(ho&xw z?!9i{nHrJuj0fMVtSDaL=QB&{_IzQ9jI!UXsUtNT%d-q5#tkIdw$^w&EFU>`Fl}^t zzD1b4Ewt2tYY%<2_)1J-zEp`5i9L&Rh6{caK|n=iWy~NyeHb#pg#}eIaSy)@j|h2; z8TjXGpSip1vpG3BE-vm%ZEfV))#lb#E)XXO=v6>KYZ1&@ivYCT%>&578T87Kz3$we zuHsb&WDl7Dl@gr_Q#4?ogYUN=R5#~H^LFEj{mj~{qn1RE(&E;3>A|Ps9r%LXm?U1U znFvOw#)L9If1x1wVji-3ry*!aJ@zM}Gz@)8$EY{()y1rz9qZmjtx)2gr}Kq@dD!f; zNV9=TgFvqE*xFG-D#o#=y}14tTFpYj-OimRCXbk-YoJ&6G05o%soLBhZ*Ucgzf@|q zBhYh3Tew*e2*|!pc--vCo94wAfM?jp^eBRdBF4WtO!rv^LBKh zaU3Bp881kzBpvl*c8_~x@0XJg7b}%wq0Zzs+>6pim-@xNTt1HYyKtqfR3(W4fAJgf zTi;iV?04H`7t?(nV>g{mv*$(Qhn~_29Ba|0@_{Eg<*W& z`<)(heM=Q_YAMNxw|f=?tiyrUv@KIr*=aW(Wj>ko9n^f)b1si=w-}=%a}z|i#|IVM`?t*bFDMPWj`3PV0$svfJSYRNzXijunh|ThW*j<}RYyg>o>5$uZ+!2;Onvxe-Ab|% z{)fJT4Uhb#}q2V0SSYA?0oX^RJJ#CXCHJChZ3dOY=eYw99Cl zdD1*7fZr5ov+Pn|0qH~LG}a`vHHCSeR&>-gQY5|n!CN&JWzv~O%9L=ZM%mYH=K?3$ zC&IaeQBsGo(dsxpFnrjt$U>BAVVb9zcdtd_xx7sHWGB*z#vhQ)p9SvKd=77X1)>3{t4rL^*@N!E-&`@?PiMH<0G{i#;vIHZcp=38=(+T%{dT zv|b&p?_EIj&-~IjI~wWGzU~>n_PyVU5GFr5%<{jxw`GfMJ#)tHcvDz*G~nIR7vyBO zSa%GG^RuUS(ueX+HX;X;R?u1Nm+5B@V?A7k7h1yZc2`jmOoN*g1tBD;rXZ4biwExM z<$RKKI{_wy{7M|H-JSfLIIVajNab*2|nM^WnXkss;oiqHrTfi^591r}YZUEca_*Eqw@U#xOg$t=wM%tY z?~3zW>E$%gg?E+V;elO|wtcf8i%0tzD=&xEh$e3^?ASR}3WGHHu1QLo%2SMHFW67r zI2jtfBqEk^OG;dzY`f1e{WtB!_v7xmyX({7^B^s)!s&_XINy1d_JN#ck?h$67clMA z(EM()8))0IEnQE_58sNqL;B4e&Z+3M%H1+g9IVvcmYxe4J2i`t0mubY?~HtXKGJZ#?7mLDRWqk;+D{Sa~_-aNUa% z&kp-*ro}u8(PNyjSwD`{472SXaujJKYX30(@NyRj1Nh;&Hv@GBJ*;K#+K;!a5qz_R zZMl5(_F#RD09!S|*&e5(we`uMyeaZHnz3hy{qdsg-{VN~eB9)aO*aKNdd>$+KM+@T8)O>sRk~9JbxFkf&$43 zfG~U>Rf!vZ#T1UEAD>ru-khAjs9c{mI!rF}2`TegWd~jht4oDf@(KO3c0=o4p{o`{ z37^x9u)mkB-?(Bw5$~&dNKGRpkd3q$;P@hr3V4n2FQrVI*&5N<| zjBoO2@a|l6uLT^IIw`kj&6miV(c3)dv#vQteCmS234_s?S^*gr(N0V@856SEn!}0G;}DO{HFmFI7@j<({C1r z&%F|R(MT>thbt%cM}e!T+1cxETD~B2@Al(-Jme&7R3EX=*Tb{{A;KUF?JT+bJa-9a znyI*&(%w^GFWNI|3UniJDsCMWla%c1L(!D7PhC!on-0F{+BLLfHnF0YB#sN2@~7K$ zK1|$pM~jd;aGL+p1!`p_OR0W$1jMaB41OpT1aSgLP)l6NoK4Us+Vy#E@Yvm!55eOS(~?|AYf&K zprU%P+P<<83UJXP3WT|YDNKA!Bco>x428sL@Ap3ji%SI(X>77G6@VHI$qIUD@6+bL zLSqR1X1zzjxVdj9s3|Q65d|^RCjnT_y%yjIXPw+e+mgJ7aFS zPUPZBm=TdBy*>4Yz~jPn$nFz;|JJhCsVB33jkp}bI>BuFn7C<&{@k_l;A##;lo<`C z5MX8NUtuKF#0e{WF&&U->};MCo+(+w-2KdoS>;wq>x(Ypo4{us4b8wAi+iLL?{FSy z1vf82nDZ#?oJ{*qZ=$FF-n2f(^w%?JR4A}(OPtb;+7b-WC3|ufRTt9B1Qh22@FCWceB)`#fK z7^VNSmCGQarMGMEkhet?lR?&?bkhvni4D1PtQA?6N0bko@aZ7I*Hb$()w9bq#D#m zH`LaQh_otRM0p;~l&zVo&<&WyT}Ly!iz<9NV;dlOSa+VT%6GI(%txBC|DYuWcqo}Cg%IA!%?zUk4Q`5)no-rxXyFsD4EeBEyfghgAh7(mn67s`i z9vW{qpPp;2x)?NT*s_6ng+0&|QV*K#=3AMY-I#^Yl*P}p`aks^nK>sV<~f}O`5d+Q zDcE<1Twi=cJ2yaQh7of%y|1Kd>G8#Pi!W}5U0l#)rA|7du{UL}9LVij>E8Sp;KVkD zK!0X6rP9tqX;1v2Vr1lv9DM#ywk*5qk5bP}VHek{?Sv}KIt}k0TFthlAa<=zmNin6 zLU`cBzE5dZ)*5cZw%Kp-?IxWK)L)}y(asJ9#Y*1@?9m71Mp~7#@Fh0v-7&uFBl(_0 zwzy`WENNlElyOG=WS7(2SDp#sbqIT<*(yDAR@nvrSjp4s>GYJk!^V=ksB=YV3Y}=~ z%KFy^5WgLW;~hv(O@Ds4zF5nVWd);nIm~9cHP4h)Rmjp`u^}<5pvn=&n5?^~bm+3& zk-0rT8Cd2{{ZL`jaP4bhF08SvqedDA})cjB|5?q6J6g&_f^w%Ny4V9 zo%n{kihIwPHy!QhxBCmf@Zg_4XKwLFFEEME0)1%uoRJRDHmCX848LbEMlCOqA<=M>EigIvqXwb3(e@mQ_YP>Qg_1$ak5UN8C@$BC}52 zFWY&+hi!?8a`KYz*Iw&Si@#Ftp()%^!%s{8%(_qa&v?JKfgZ2{nHT`vZDViG3K-VF z`Kb%ArvBDD<^^I^uE!;Ah!lb3KQwXUv$IM7>$ao>r)5EvEtw7o$}$6k!Q>!NUT$uX zc45F+BO4tGhL__RUd^0%XEh7{_DpxA)p{k*hMK`#E3dmtf&;!!(!0Q6wM^j`$Qyj; zt%Vz^xM{&*pX1Nw{_D~M`+<`-Yg!UfF4D+_-zeee7|dmN@g@{jNYFe zzDvb7@J6W_R6MSxHm;pS#4qtCTr*|#JZHW-+qSgc#*9rg_c_PBN_S6m+zEXid|Pv~ zeSkm8+ggS@{&O~X^_PXYs6r!>py903dWY%>9;oqx_8fW>Gk~8t{B3~QhI(Vot7zph zC%`a!dx6_=eHfIz+>mQFRnxI^=P8h!)1Z@p?-^EqRC=ely8h*Zs7R{$<$ZDJj||Uk z`s@`k#YjMoiEd}6?xIQvI^Lm=H8Hq8(mV&5cMl!2iN(S{8Jn%fC@hU?u8`C_-7#_g znCL4J170s%Gxnfe36r>fa(7vp9bfZq5Z`sx$e;sp`^(JWC1Iz$tmoR0UHxr0t?P$j zlRNI_7dAf-b@rBcUwXNkp$H^LiN1=MjmuVt8?9{t>rRe$I9avCh3@Soy(*A^#RmXt zqCr2qZ%gI-$h1Z$1lomr_tk4|>3s&XJ3O44!tRdl<~pJ;r-?7ek3DKepE_!Oy|xuU zddFTg=x)L@m^{}SYQZ!qLAdVVWF(a`*>yBKeRR=Nc$e56Ai$Vd)Z<{auK z9q-aJ#>dgU6K+5EmirH)D2i% z37o^nF^Wh@^j68Gl2a+x#U9Z2s_E#jnl(4KSn}jFT%5{tpqhc+6Y4setY9jSTI2ybrr>OUvR|D{Xw=9xDxkjP<{Zr@`r0 z);e*S_@0XN$Ow>`{&-S?Yj5R`cqBVU(DT`1=emC75;q*ZcTffS1`0Q+>mBHQviQc# zIXZ&-y-W(StC~YfcpA0I5hoVRd7U9rvW@3{wWpxF}fB%;UmkGB`DQP zZUae|1;cN6wLFWbnlEQs$FA|-YG4%@Jh9BIjcsift9ApK-ow{DvD4xGO#W|YJ6c^+ z6S>a@Bz-*E^=QdJ!NB8jnipkbW9#YdHSzI*1JOr(K;#iu)3FA?;Q@SZXNZ!O74yx_ z4TpYBw5O;F5BYCGHy}*uyfO0Rw?_f!PFyq<8x2eNZz?u=8wNDA{A{&qj(QA7actNT z1St4`lo=7#mc``h4*yq>3D_n_J7P7|=@2Q`(N&Ytsqt z(k`n5J9<%fq-w-~LB9;ygXT50i9c5R&ic~;wL^8tZ%a9Zd3(UX4` z1k85f&QC=}P)`y-$>?H*S)#Ws2%~}{>a6*!Nl97RM3?mHbjR#qxeLfD;8F=Fu;1QX z0k|a~`e#4GL>R#%BGNZCMV@`$$qWcrubmb9iQQf2b06$_-QT^nMajP43UtP++o_q+ zskVO9PMc}X9|^=@^D75SC;$pa=HQ6ob?JXRKZ0P_@ZQhFcwxY<({l8qOeYe0?I!^w z4NOW(0ss^#Sy}X!-b#+cqaz9s$lT2>=_Q*Cus{KD3E(BF8yOAny=EH%Ht#4wLBWOp zx$B<*P4L3v@~g<_bfV{rK|t;epkgsGF&swC@Mn*o0jLZh1f$8y!eR{E)nTS)bZiXJ zHC}6f$i0H=HJtNPm@|L&9jG~b`-Z+j7}5uKBhQ}nQh-oWo-HBW<; zl8Oo_B?CMA!`hj_B3OaEg99g^R)L`Q7@3_N09w&&IfB^}5KjCb1(o1PoP3MK9TaXnlAj62dPeRAn>HgpG~8|F&smW##<~25TFef1s$p-`tiP zG|B>|&z$X(05~$gP59fJEkJ~T#;+xN96+0X7uDR#%JpieHZd(t9$;=eI5_-|y3wXY z@i7oalmkx$xa*yJznZUh|J7-KHt!G~(_cXNKZy8aCB8?0fn$G{2-PG#;y<7N^`vp7 z|9+Ju%)J4^(a zi|!2<(mDpyc13v0pe;$pHXL0G)wvuuHwbh7Plx_B|2;@R^SQCL<3{EEI$R zFy(-L1pdVfr0a&@S^fO}o%p~L1W+Yx(*LqbL`-ZfA$G|Ag-_!=U=mBtp;nSZ6 z6--va0fgC}uaD2#| zOlL8J8R$$31UL9xj%ODDpyxOs-M0300SG1C+4<92u2~~J9FU=V^OBkfR0PC5@+GB# zBN;mLO4+62?P`$SzAL3ldc~>3w(b@ zeld=lDUZ6!d4yAM$dd%rA2)0b;2W+{>;2Tj@kB5nZkTSg3aAYIG4MG{4r;-f ziQ!)9yDlyg##A9pO_qG5UG**J{V4sFm%Fz_e4ZrF=kRplhjkl)99cex+ST}lo?@N3 zLEB!A%u6{~_WyMLZCzbupH9uUXd*&>Wr2eYEgg@|{Fi@?N_b!tUuUX*U&7HOPluS~ z-aZ3Kh-CE3;CHu@E*i#*1T@=--ps+DuV`p|ZK;~fpen3RO#P%OyrVT~%(Wn6aMoI| ziir0|`|BeT{(;@LUuPQH*@$;?G{_f$$Ok!I@D`D5>8Ca0;sp6_yIve6Lx}aj9yN@H z+PysPWk7y!sO#$!rb2rcY^glvT*xUVkZ$>d6+bTdiuRz%!K|r>AZ@{iI!v?ju&j8V zQ0AcG>8LdQBCF2TE_#-JL&}k$dn&g}&PArI{HTZxD%KLU}o; z+qr$sOY`;{#SD}HbxA;hfzW5i&ZuMW%vA))5j`}d@@FFjkGqaVdk7wk+_dB!j1bJX zPaC*hkidlsiP=SZ=sj*Rh6PU+-TX7ls_j?iXy4MWif_!_V4jH{bL{x$=FO%JNZe^~ z&6Fo3;3V=r5pWN_+N;kfz48wCiq!G#dRn*LXzPQ{U5=FA_S^`Yz1!z92zg^abent?E~r`i5D#KWypUlfdLEmVWNmWfu0Rgh=6+=Vmy%_aplidhk3e3?$b|7kbbh zMuo>6kBLF#wdTAGn*Fe*Z`I-3Y`>x&Vp1+X)Z%qr!vUmj54=TTy3~FXJ75&3VPVJ^J_7v@MyJ=c?9; z(drhBpTpo;i>^ZKaER5Kk3wzjs9@G&muN8f8nTJGBpH_KLQa|9`iiiVnxeoABI`TN zA?01eyKBx@E84TTM2(}~pZ~)P=c`zMR*H$eu_u39@tdko931B4FH~`>Jl4mzA)s!O zp6(*UXK$c*gL0Shp#U zsj~QSu!Qk)QW8=T@xt4c#l5~F=t*P{RPX6W2p-^8u>ns395cYz05%s^8317tNNJd( zhWW>GJp|p{xQ9P`jMjYPnt|_!ygOWDv>?YUOr%*GrJa|rUb4MC?Vw_PNLg-A+I8!6 zNGs8hsyN}3ApGNG#p0sEYQR+GHQJPkNHjKA!y0)0%@T(J=&R>_VRw8XdnXfy#w@U5 zP$!S-?FtQ^McImx_=QRF1JKvVNCUELAitpfYmsb4%d2<%X(wd+9<<)vZ@k2HUdqY- z>i4;0&Zslp%y$#`3WnE)eikx(7CUL)JN7oRiuQ?PSOu2xpA)9DE!WUJZ5vKBaSQ}h zQ)l7XlktSa4kg*cPk89P{SF^60U{~F=!HX(%nYQZtrLu6m}2HE1(OcL(5IRnew8GT*AZ}a9Z{- zZ;gdzval$xm@6{vv#dzT5?65-Gn;|~^}@`QYEwUOKU2S7?QvpU!ofg}xyX&j`*RZI zmVgn>uzQvSH6iF&jvj*9UJqoXiNgodp#NsG0-ZA?2<#()phhs?pYG6-K}xzgg4=27 zG|Ypy_d)Q2qo;(bN1b(@3tGNp5E7W>^mwuZuNs_p zL0SCVsaTYQXc*Y+90jiRLM3a^phkNxP=zEkX8O`~LL6mn=n-nmAZ}Ub)Jjtx#eT^Q9bT$xCxXKz7 zt+?R{!4;UYX?=j;<#~<1SsfG9p(yP3mDC0dCf%16!OJBC^cXcGspyl>gFS!B@bsG=E)vMM9-3?Dz?H!i%)L%7;W&8X zPp$dv{7CAB4MjXo!8Fi)FRyoeqO$F;114ALMJJ^Dc=u7ISJo{|JKLj!nK!9xiZV6h z5PV&+%|S|+mOTCwN|wS*b2AueN{=zu%YhV6Jw2E}{^Y6{fGp0;j8Qui^M@_1!`$4W zgBog~cXJYpZ4Nt~jeepG@{bz{n{g`b!y;awhT-6-ykN@B8>hK83|cT(1+837`X{c7 zZe4`=dtnU{>F7Kl;$#NYcep&AzkTIG2}(+y=~#q1>C-s<{dI81i8ko|~iFL$-x>O6a9 zZ{?{F@qy~yp;q7QpPK-9LKL5|55L01tXBzlRt#cmiJVzvpjnz)Nl9htmuOGVHk(2a zy7215x@u=quCk~JX(W8C=Sj^@iTO@~ObK6L^_ydG8V{x=nL3}h;BAFgjbJUI`#pr^ z|3lkXM^)8*?W#z3cSv`~K}rOqL8PS{Bo81RN_TgcAWC;h9XbW1^N`ZrbvMTQi|_m6 zj&a8w;~*aP+O=Y?wdOPD^Q_v@tp~uara}ZKp7(JSE>KmW4VjD-jWdVu71%P$i(Xop zs=Hgb-Ef>aAO$NmT*GUY)qMn{qyn$5rJo4rf4V}qH@8CL+F+rDUJ!=3(-$X8ct#KA zUz3{}g9bXjBYf-}XK%=aMspi!8jRk(L&^~%At0vpr%3w>(7=maPY+KudkO$3ZKb@a zDWI=h@&M;v+SoMI(;%G(}5@5t!3CJdW~z62&G|PqmWA*Vs`H@MLfE( zH^RSe6IyyDxqCHdy4v2b^E^Wr|F&A^DL7r1Eat#PKgeXJ1V%fbKd0L3UzGaf9>juv z=VJyhs8K?o9E{@vVu2*~7Qr6Z?>C;?jaehE`?p(p9jMnOG=*vsvL38f-u3ZSTsG_< zhudCW9*5#Bk=rcS2#`}cjLd5AHa-evpcsno$~@lPZ*=YVKRvOzjdF zgA_P8z%>(*-sSXY3|Jyq352iMVZroswt=N)dAkYW5-gUdcx!+_s(cH>6u53sM7Z@V z2#pTds$2NP0?@~Yk0!}WpsAX0U%+euK9T3Zj~s|3|LfOKcrup{xJ+txw0;l+z}o^$ z@r$ZJiu0t7Gbw;E+>`*Z6TL%2>*vqnWdKbSMM1IxAn#w%`)W)@Mdep>w%1e6bh7Qu zi;Z7?P0}H1uaH|_0n!RGJ<|#_0J#6q+x0jGSk(yhR|JVu0$dSKK0YeNpRL*4+NTB_}j{{Pin1Jnll%Nho~SIp&J#@itr*G6^Ve^N24gaUVg;; zO_AoGGtRwEN((C^4r2G``(oYU!MH<9K77sT$H$Fz$KFmA%UkE8Q9=XvH%r}ddN;zX z8G2#8q5QqlXx5%VQNR`RgBy4_aZ0G|GLro&f|y0)t>btj*Yr(E1fOoDP0H)_G1F#o)s%h^U` z15up8Cxj3j)G{0h@Q5_u?%FE9dGp89(e-jZ)nzHAw9;q9W2NoB_jDzM{bCGo9Rtmb zh=>TZ_Ylz77Ut#@0R75u{~OdZXM`SL?F9V706DaX#tCwu3$t7nUl;{Kdp7`Q1P2Fu z0IU*NzySE`AOB_GFU*;N-*6vZ(~AR!CG#l&<_nTZ{_*$y-XsYV0nnP?9P*O6S{b%60!W6ky5 zMUV^_ZGijif4f2~pydq6dl>NA%=Nrjb!to&a_6m`vq$dj?fuW<$-gce3uwOA*v!kg zy9>aK14)n%5;OmMQSC>7TM?ki2XHY%0=SKw($dJ&?MXmw;a@2M598be^iZ|PABKTo z;(j)>t}sGi2VQS(W0PM~lK@2d{;Hdcumfs!`+W%dF>DMy<8asS)hpm>^an214%^C( z0<Vf-SyUIJd4CqT-n%UZnp1%&*rwZ6g{*gyvTwoLxac%dWzyvTtt;NU)W+f-De z$60Ru2!r5rzyJ^~#qvi$dP%yW#(+%(ATtzdp#K5gRv0m21#nLTa?T#w01IARoF71C z_1mz&1VEDB!NK2VUeqO1iu$FV9{FuAa4`TDq(I07un{1F8`RVfc))?MUKk|{3xy-(8zQ1AUY86ifEKira$Ki}Diue7MpWNXfnAhD05?M$pl zvnjpdjBI(&xGx~w;Kz4E54;XBFElHy;TlMB$K0lhTDU>SlD1$PrTh66bp&kP{7@UT9Uw@|^Zirn&>wScVdNqq zO~#=^5G~EIEdvjyr0~@95nRPA$5nA!v|-39se)g_?)Fqg%VY2Fv>dq~IAUp)Qud&q z&9nTAxB1Fi3KPRdqfv&Dloz1}t4WM+1ku_ep4ZsSbwJN37%m}&g@muXQWAmEs9Mg^ zajUsD-YkgzMoAj6va=lG)2ZlcU{?bVs1UVjpI#OCulf7&jNiW7?PCaNv7mG%M`4+$0%v3@&v zB8{_|mkE1~vEI$@Sk~@~%e{%tWgHBVaWPg;s*|?@nbR{jJDlSyHfZ(4Z&k3EH0aVE_DM57$@*5>5Z<>iB~D?b;oh z)QTG%CtI($HR1xhMQim~!a}1%TQcL8f&dP&PFl|-o-%=Kf%Io zKg{rGtOQbgr8&$NYz}O_E4pej?ejvsMGBk@}sf z4x~w1nCx~^58UDOXZ93Uc0;!*n9%9OZ*d}qj4|g3$u}L0kyp5O8$-$H1`8M2@zOpm~_{1AOZ?7HelxD$Tqb)-f z?|orYRB2yx7R8(0`(^*|_i{r=6_#Vq`bw^l_fmAjX3I8(4;b&a^%5&fHyJB#TQ(*w zI*cbUCu?cOR zWaZ4PniIWs9FwY1F-amSNdx2E1$w0q?Ye)w^dQ5B@s|)mIQI|gT1oOKi;IySP>6iO z8Z#DSA&jvc2G}sHd2VanzAd475|cVXV?5N2oM3hPz>!W!i>BlU#UP_GP4B3g7j)5} zOoGH&fJlr{xV9Ib%g)_1FOL=q%vldpd?Fw2rX1wyIp47> zE`uDS`R#%*BU~=q@HL0(@t%pA8&<;D4Pa zC|$gixCs`8v3Bamlz5LoaDVfakm)qX%a3F;jg98-kj1#=%*sMNwI)`Gz@*Adqf(Es zDwvX-dU4*JA4aIv6wQfhnG)FwE(U};cfkDMA@+2Af~1wLadhFmVWlv?uD{RC%@?HEQI z;^Z#%cTHC2AsE%JiUUp>%>}*|@q}TFGNxH&KjEt_b=L~lQMA)SLA%$)ASOUTe4drp z%6wIb*cnrHaJgbsDXuKs?)2%G3O9{1W3KK`T&_bZhLXG$Hgs|+LTGyD&wL8ka6ZxL z!vmRb1}yGG?&XZM4ic`E#k-;vG9d@`R?pG)^&t~Nj@=}bZI3>_#+j8i>%&#%Xh;pZ zLL!Vye$iCw3orwem1c*6Xc_oFg%R?lG~u9UN5H+kma|uW*;L^o1fGTFa$<85zSSA* z5#_H-qyJV6uldbXAl+a=ne&XZE6!vo1pf1UO~C2(O!;H3>M&WeFe<7M*h6;t#>sLw zcNp-m{sqL9z?Ul|-0mSxknYi1NtUDH?^X{^5qfr5@nLUxr)!+OtFiJ-S7mdCG8HW! zxgOVBX+QUKN1T%WfwFIMn-j0i?kGa-)PElMMisS$~;!dMxDN?q8ZiJ^Kyt^oNgLZ72o6-cY*p}=n zOuhZy2R0CDCiz(qpgaW(zu8ZXd`0?IoCx1tP+GV_(hnlZscDUFNs|ttIywAq*IshY z6b4_dU%iEf%YNCIavgJGw~CM-OhkpQF0o=@+cB?&fk;@6U}JX?!*Zy%2bA zZe`WpOVZu}#-$F(To0cgMER23=U4J**7+$R&wD)FJ>3~CEsIk~7NRnN?$Yvz*<#lG zkjd41J>$0wKgi5!Eq|H(6JYR z!ZU@+KN?~6B?+;G*L1xaC{hY5{6f81cyG8F?Xf<0z@Lv^0dlOvWO4=>NuqYH2oA)M zJ_9;XQtidT-&g?Ps~@uNSZd_a&ut-X=V6b4wJWJI{oCVW<~4eFJsXF=p*LpOJX=4{WLBv9`+#p>7v#&KXO|<3^WTC!%#A0WME!Rp0BWa%(D~?bKwK zH3W}WQO=jCP6u;iY|ARcW_N2Q)O42!``*c(@MuTb4QGcxO389dcHnGMlq1-lJ&vqQ zjTjt$r0Y?LSKh$e@WFAxK;XowcI*}75>0NaY?&3M`aFn;yqmu!+=$?W*Lu}4V}LVX zk3R?ze>nSFWxP;-=F((~XFP-QJ-k?x$2viNXxxC{(NV0cqSxb2f4Hxix01Xp^Sw9a zV^WZ|2mylpV1olmv>G5u(Gc)u@J1NuFX_*sNfHJGTM#w%KV;Mq0B$3QQh$kcfQ-MTfML(dtfErE84y0 z`jDNIa1EMCj^e4|{uVEv+OwMID1o?bn_Dd97!W+I>nE{#M>m>4-AlsAQo*3zI{M@r z{=H^Xyu^q(X|8>{m(1=bdYT^WNT$y+8}FIq(!3A_7PpMVUE#FwZ-Vgo-eZK}+pn=W zNXam@b?Pqv*{a}IVZ3)g!yx1ih2qlSxLBp2TN_>v6%$YuK*hH&HqV_A&m8_(sAZ>sK&3F#PJxbV?wgn>Qer%G;7D0~tpMkS&EoSxg?s3zQa6jv<4p_O$ws6@~joi>;FiFXhrXKXwh4Vk952 z++7}Hm;d(lfb`C)PrxN|V@pxG`W=n^2&e{L8%5NL1JFuPrd4i1~2lm8JLxpoLv z$tz7Aif}V*qOuvo3|f}0WYy zGfrAXA&lU=Ah9qi{q)^ZG);t4ijAnPlPaO?Ao$);QIyOkF??2`eZxAJPCvGH+SMVD z)7_0@_~z;v8VUpLv|X8w%_|d$U^pL*6*F-6TI1zBwuqs+0&lZRl#@EEdMzP^n>22~+#XMS0EW93e{!T9}c_cUlk*r`;f)slFIU5Fv9;RMb!PV$hFe1V_rx328+HsB(S+ZI}b8ZjnPkD>N7kFcjlVo3+foa zDQQ|sMGP_&*A%wnPR)4}ovPY^~s<_^#6dz>ls!fvkVzBxiy?#Ppuo=bF>&^!0 z>-iX_{lTZTH)<)8`_c0qTwLs7#;grcU3PxS&X?bRO7Z)dE2if|OfyQpBS-)pwz zJp2A^#+|ZGxq^X(iHeWqqs!+s09y!*%VCyVfXWyG;ZYvNjyQqDvq8{jLbe#VpInnX zcqQBfG!HWGTh|>!ojV~y+|CloF5BvL2HKBVTi0_f6v%s>ZIfjs8sOL|FzAQ$Mf2J)5~xq1qKQ@Ic@*wWi{j@f7g+F-?yHZ`uXB5g%p zvhh5vY3liMMY30ak7k;1&7PK>w}`jYn(h6|1?i_KnlmdwL9!wDoQgMzmLdNmav2J~ zF80kg9S|r@I%oE&?_)>~IZjw(8ZjN79j_C!3^fr1ssoqMEe6?=IE03#6^2-7WCT~)M7LcS^!(JU=|_!H&p%X6OV84DFHd|RiHt|>)dpP?yQm|q3BZYQ~3eRAYEO>Uije=<7 z=iE-UFAQNXM(Lf1e*}=S3!x0kQjwU`;T-CN(yiVe3T`jD!7@FI?YC_S45f`QTa57C zLH^@0($*+C!Rm_KcnXpClHdWt?5KUip4`zVFZXfasM`~v`+lW8ARD8!{4Q1DB9chw z-}i-RVNFKLpyy|BQ@2!}od%CTNn)I;+N@?$!quR^okT0YB z{_#?6OPDw)uO1~%b_W3@(?=oHX?6m#neJ)Dhe~r^_tx5KTm+f3OWMZhtaOVT%b?hk zDo4r@H#gr8=yr5bs@gJYF`>CAwW)F9<$Py)4e$P#IQ^s76485tz=UmI<|;}T}3k2MPjW=At0bxV2_%?^G(?YqS^)k~8} z!8UVHCg_HS!I{wL8?wLFIPuLJAhEZ%2T{?kBPWZ-wW2}qD;m5_6 zjMwo}%fWHxT|x}}T?hwka9a3oM|nDku3uihjazQP=mqnF0;mscgKcYyg_!n5h@cHj zGDOWX4#0Iy{jJYT;SnWCb2x|rkGpGR|884))Ng1Z)928{qDWw81K6XJm>Ohs?o?pi zo#VF#cTd01iVx%s2%1S7sS_+OPD+7b_$V{oAVx<`e_ zWd{T$5v6tI=UTv9PN1*!ZK{K>xnbz`%(_Dor!f(*4kjy3s%Zw&1;Vv#ik}&}l~EQK zd93l&aK9WR$ZCwZ=&`(5b~&JKu$Vc63!U7_VVLMz=98`W8Y+%^>Tahke6Vi<9j&!M z`V;YsfFBd4H0N-ySK-=E_VyODiHi1(GFL*oBj3xG<24og!3`Vz{+64VOp)@Cea~C_ z3g(F5S>9-uuoV&?&-s(n%gYm&o*ro$8@?6v7fM*&;y#a`+@by2A6(#svB}f0jmuVw z3~ahR!6ehOkJ_avG_Grl=$uyCD3~B$k*kOmlB1V}aHGH>g?g2pS4&>prIHP)PA#?* zRu`>p+?V@CwS+5oG)z~oItx}!OXk&vE|gG|3ST-**78VyQ7yH zGHaqQDdyRV#Bf8~Wvb@=~n|IYj0222Va&){1HY_t5N4{~%^%dEo88kDP;&6Me zwYlZ+L5;WN&^?9MY{}^mEzmA2^la##4MV#>Eq|>~C|l`^i&a9Kqc3L1ov!^Cdk!tD zorS7%9QBVNi6OP6kqupVlvFY0hL6#9r`oe;5L7+hGQ;U{hn_6nB&ZTcyat0YSq1vq zSeT}&s{KWkfBEd79!z~w`s)XB8D5QuvtUqqi$F)k9tL?AyMAAkW5DVaffTUx{rokV zO6Md~3OyWEC{4PlH1?`lVPSl7{FNR)9pX{RSw00_s6?ZA(MIG<;WqTCz$;rVs&6D4 zwuRlpVC(SHv?UD0ON%e(4k~RldeE=RRtqktxK2OdrJ0#?KMTgLlU|pRrYiG`U)^w@ zUh-qpk+m%@__U;tfq&R`gjWt>UKk-ZwBV`rV{R+5Mn2iu9i8af-a7 zrZfupcisq8P}VMWfIL<r$35Ub_e*k6i@?U_ zl90?&SC{#e<6hR;m3d5~kaNWih$9n+-i%_ZtQHs$0U?n}SoV^f?_4m26Q$eALO-+0 zP|j`BtuvVnsa6K@{D|S*!lG3*HX{0Ef9-^6Ty^41=tc>H?CXSj>e)Xanpn(n<@${l ziQfTfLupdfKEoRjc9bjM=-Aj53J#*92{z-=j`Ovb6t>!BCW_6lwperKMT*EgbNI zp~uGWn4a}mtw;3bQ*(V3Q3c{)fCSpV(Fh3MSab@CT0%nB|CSt?nbeWy@jUT9ihNIK zz(iRxiU2fl#JAASwRfzo5*_(F<}@<}zyfO(SdI5STk%eumF>Oed!PR7r4n)A3Kny$ zk0aZPta9?}gjWh7@U{K=SrvZ(kp~ylzqB6{Q1Lk?dKLx zH+6rT%Bp%DxJwJ&5|Z7uzL9C|pAWGLe?Cy3C>4#)8#YxA@;~dtg#VeHNcWHj^`9R* zpa08Z59qxGkSB{5`e#l^YyT$)YxwCyV89rVc?P7E)mZ<2BtIh{o(cRXblvYy5o#Km z=%gg6e}XnZKp21^89ET`^5?O<&mMN(3P89EplbBzZ~Nuesep*x?Inqm0R8UH`0qUELxJKLV4$D!2aIpO29jaD|7U9G{|gfT@VpmKI}AC1oTBL#Z;={n z2i^dKsLagaG6M*ZBl{oi*IqFE5G6nk1Td87G+vCCdUdW3rUrK!dJv3p_xAU%Z0p@W z=i%XT`S!aD{jCe=*hBE{_edflB9{wjpyoF>9w1E<$OYpjq9F>$3V#J;)B?@qdHq^|?_K2s4wWOw}%WGD)XH-Dn<^}SjRuYJ)&l){7yQrJ-zS*Oac! zf^b-&UdC>*42?gDnmr)KqjRiskR_d=?@$%Rw62*AudYx_v{CGuOA1zJP8zCiPNObv zxD9GUgJto?>BEWb8|WZQ)p;ks-smywa!DiG`rt;U;cf(+a1dQ_6k(+qhDN2|xMT_# z#MfEbGLR?<>w2}AsgF;te-eKK36Gtsy=ZZf((VMz)Z0F>l7L;HGo!aC=?3a)N%xL8U4 z_ucl0>cgwH0+(zXCa6o8^K`^u{)D`-8&|!>Px%XGasW#824B)Q{(NWC;>1Zv7CHvF z&DW$2mYcMKry`I0wqTwbrunD#F-+R8>=-!iOt%8~=FedF9+t0$?M1_}(fY?7{i&Pw zAU42TGrQZxjVdhn)`>Rv6`JXGmx%{SK!5vg4f_Q~0FZd^+s}Q$9?&1&jS3_CaO{e-jy6HP%O=0|k#xq;5_6_9ii?w^#4-l&6Ud6qPo&xso z7=WVS;N4O)IAbW&O+@Whf^zNf17}HOLhO^+O^CG4YsG!_W+s5#lmDUl7h_~Z2Yg0z$lE!4F{D>@eDnx-F@+-~qO&0^QR&Vb)MV#Gx z>+V^*mI+;eQvaF*+s%e* z&aA88wD@Dc>dLBI{>}R6sNq65Dta;!CBgTb{;Mhx8%I#Yi&>&+^;wyj z?}Z-O6(7tpTq0%=_Bo2g-<33{U*?XP^LqqExFF@4dPVdtVkrU?cZj9qz1fdx5fp`mMN1A=N1GOV9k{+PyHv8BZ zJ@JNain;pCNnKq(p5=F@YKco|eC6-jDw`VY(M#GHE3oD5KE2US2(pA1H%jZ0wQ*u} zstQej!BoIUzp_Aof=HwN+!g8aaPa15+u|4&d=Zw--Gq7)$bhr)e9~y?`+H-U7SB~0 zqIUL`X81bVf>_vLk=`!IGnsCJIrm%rm^Y)3U>eT>W!vkgU(c&!lSIO{ZCgU?W8n|` zRhzm~g(K>*?WY@eXcBCMUk-3g^LTJhYJ(At)U8ZbX7~zA1`NAHI*YdLz-x)D?e6A; ziY?{g3!iwcpbFwz+xfe*_r43wx-&EDA ztFG4M%Yi=A_3FeG4s0;YpKa(n}4}mMVg2E(UZn?lNo3|IQiJjR}bvZecXUXrshPMvYtqyhg&cx&q;%y>ODSv+p!- zet1_}_tVZaTuU(%LgbwR=FKmHt=XlnBBjFDAmmAMY9fv@&ZGLBtHwI}E%woGa8A^% zCAGS%=fi9!w$5oyeZ2$p&Ip~qR<}@(uiM2dR@wwhpboH|-3Jg_)K7*-a)7`7^rIUI z3E)Dy(_x}f-f)4aGstGJUQ@Ali8b}NgQ zHUhyT;usr8;}jdsmk38xZq7A?mwYXus6MNn4GvxNDMB&%MV8XD)y>-ldCsUCdac64R22HhNRhxFz7WxW!45v6;@@H;1mirm`fv#~iMX1K<1J zw?|nCY@nBH_%Gf)g`A=8)n$`^+$z3VuPu!cgxWL<=RT=~vZtyQC$@X$kWA+ zT0$wOTtIHMNo@inug&0%B5M{0&M(1~;6a2pI>8Is`pk=)8Xn4w%obWh)c>OA?oiR)k1<-)(Is25ko1~ZB(?9X7vwz^>Tv!a}8COA#)?8;5% zd}l_Ot3;(FD>{*F8>B$FR+-+ee7oq%Iv3jO~H+CND&ey~fOX7}{nc?nA8@Vjiq!OkX1bXq)>(FVmod4->!ltZ) z5>n`g1|5#%`d}W`)X-+}6~GN%RYj6l@-60>CSq8f1*SqlAimzbG_yAWa^k{ZkRWV{$(E&C@C-|mRzr7ylujcD;dJF!8!BC>M^IxklOg94uA(!&Ri ziVr-&!KX>qeR=mtRD?VR)ma8>&zSgxU@Ar^Z=D8OIvZ}nsBKv7gfVXa9isF_aWYE_ z3C;OQVIiWpFWec<0^X$7X2d&ct;btm6<5Ysa2f+oPH=umC1q&f=kq0!MEauX;i%A= zB=^$G;P3yi-C>#IDvdc^ynfvHl2&05Eg8)x?DKWyz3OtlEEuGSA5XA~DFVfW zDW&zD3!;)t0w+LUAkuxUC`Eg*1S!O|BZG(5{ZqAI{-HRM^|^aKIfD5}$i_SK!+!*; zPc)G-;_?Ely6(sJgi} zK3@9i#&q)r0W2RX-E?wm6tSs+#q`fkD#DC?C>9gmQb%*T3ZIic_`TY^E-x8L=bRE( zGPyH&RSwx;OjPaOK#n+Gb}++)WMT4!!8;zRf`tM3-1iGVLt>(o+NuoU-=`{$%|HV4 zVkT_ajcECDGOVaPm*@l5yZGyN1Oqe&Xs)JjNjAwKJ>|+j5%+UAmh&7hK(y$GW$TkI zP*h~Oad`#N$A|^*K=w_^utv0;&5|2d4pALmRLPzzuea2M2bjVOKBKB}T6Te^lwlZP zlHY26z}j3+Z}1Q`lzEI?SVxekY zJkHHqWw7>vuKI)0m<@$X7Ml9RqkNPzd#5oApWfVynT;;{ciT)mUb)0H9GTRTVS9V* z<^P!PHo6qi^GhAU!;b!{WT%iWMg8RuL|TAV#YO|O=#f1AyH2sTSAW*1Qs+~byS(w{W(f`<&bNb=EpS_Wt7a|q zNKcL1TSPf8lv-HJY^W{XZNOJ3P#5tYp7@}%(Zzb4 zc6~UtFXk~=j!D5;8$Q(eFkPg5#+qi1f5mB!%q*+xc1Ncrj7lD3oq#EkrotTRxW_og zVa0kfq+H}-LONcIS@F@1`UBUe5D8h@6-SF7yITD-p5u;zZ#5fR{48Q^3alJ)tXQ9% zx@%PL;xr@tou1q168u{Z+1YN^l_tn+J7ammV39Yt=x4*1s78AiHuTn1&QRyx@2e-_ zBYs!${D4n>Z>zfq4$@Sm(N2wxn|rE_I6ZJZD#FeWX_GKbQ=Zi`l{LdcjCfjsgnf9* zjfo(Zk{a6Hrm$Sj{`-!?rnnZk#;z6SsMSXbI>lc#*5wVoMpSH=lsUp#oqiPSYbtXE zbT;NSjuy(5imR$zA%$;C?OPak%eLfvGli?nkEEp=iI7(+V-9u63a3g{E3-c*d;$gG zP2wGoS32w>lT7cmWf#(au;@U9ed4zQhyPOub;skBA;Q)vw-70z5c}vFp%Z0^d-(P| z^7V=+EC0#OFsQiW;Mz4-y1%~oW|GYb^8}v2Un1}HX=(Ef5vl=k0wxs!0#-2b=W-aq zt7B~OCy&`t-$vMOB&R@J(uk1sx0IfIq~MtM8*-btIV-RN&v4{=dE60{-e!N?wV~sM zl~bm}Lcs`o`y#3#H`OJ_RD?AUDL~=J>oTV2j~AYe_Kuu-XW0xRapN*mDSY3jvpUR2 zK&7iW+U|nXaT6oq9hzgbh=e`nwz&~noXy6F^o5v6mZP+biev{xNW|?jdG$w!vs#59 zC3U=f`~*r*`>&>rF)W`G&$AvFy0fT*@0Tx$zOj%DdW0R*)RW=CuClNeOp!b@p(C`E*Rya=G-fmejILXRi0{ zcQ)-_3wX$vn<$s{j9dy?1+kCzhr|4TZXtkG6BM!hUm`JybI%X(eT(=x*0Vp$7y zPDdEp+=n9Oxu`5eQh7ho^D4{Ozqz_Uur=!|9B{H*@6pZiH9PRX#FlcV&kNJ&80AO3 zISVES+ni%)-w_R|^JNLV`7NM%z~L}OA~%>Rqg5A9DlhO|K7GNF>ZKW|z4~*AjH(kZ z*Y#jWas8&Om2T;|_mNVW-0}!Qwq1}NcYwi20d82zEAL`|DoKU!gc1}Neh-d9z`d@G zUao-xWE#%ydT10E7eC0+0bDzJG3n`ptG%((GBPiJ8QZpkBtmq1`~FbDoBqY}HEljZ zUz^pBY$82#9Jh8;I3@6^ssKD=o)R!`v$0~Ri3eaL9UbC##v!PZZ_vb;`g&!`z^Wl3 z>k9dK_xe|6bUB}idNn2D64zF$3bpe-gRmJ(Ym>q~YQkdl?ls7OBD`Woap3K>Bg% z{&9cB&%p|2R)AS{fw@1YcKoY4m(z|$2|SRE4Di#g93B{u3c1DIUqs$#efs3zY&=zF z;BmXHSCEzU==ZuM^DUfY!hP8N{bBWwT6igv`VY2Vr(Oa01uH7`HdegEZTc7h_hR=7 zbOoTg9UL57U0eIJ7XOa_Ub1m|rY`>U{k8>i?V;DGwuKy1QrBuzuW7Sb(|GAL_m~++nLf*vQjcT=<F<7Pax>85(m9F|{E!)J0`u7qqC@!9k8tsxS)F%e@N+@{?40@FmgFYC% z{YTkxGXFpB@ns(JvtEzZ6OLS4mqwbE`0v}cRa*e3w7mxusLH0TPIy9cfOW_3gTf|O zXO3NLO{s_fx@c+$uzA#`1Eg&MzTS~U^AsS~ok3b!`kxYiu1g;FyC84~$odV8i^I7J z0n{sKA4b)KcFj9X55SQMp^mNn*|`Z|0J8pX*I>J zqWVA2;r?}w{qyt(cn+Yam?+WdU-;(!fElI%P%MxJ4ulOQ#>4=`e`5cA!=>LRJ3LQE zA?6J;fCT(oUHy6kkE753{Ynpiuao=tOL^)5OvmNrn133jtuZM-xPNWgZS-#^1TG<} zw_yhYLUUE=(=j`XkNiG~0B!~A!pJe<*H@~@t}RLP{# z5;?wIJ2nMTH9>7j_K$^wmN9tPf!{e7>uZ^_huScAw^eHF_HfCGg|P83QM22A7clw_}?x558B{f)&#SHnT*+NNhBJ z1b?xM8O!NrFBo{s8Q_aw`!kPhP(o*5-k7}=m*x35#>Q>+j#rGW*EN38f@i?ci=Zo` z?Tb=!uf%S(JqURvJ4AGML8PSwM5Mcv8X6>|yQNDSL>lR?p<8n37#fCdzHR>A?>%R&^9QqLF}t4q+)vzb zU01ymdUejr)U#}%+aoD~Rialy@)K$OH@RMIYZbWv-uU_doKDoA?C^n;)sT9UDnDO# zpq6*J@g4OIU+yt#CGoLD_EaZKm!LzvJWWt~&j^0(q%TEt&k^#vli9FueOqWljV$ft zwm_CM!=RZNic5V2aoKrycRWLWlx8GOpvfHq!t-SqU@)~DnmE~Mj^d76oP%_;r zpGs(Avn_HZ7`?O0#JgmiL@1x_EOfkZGa(!$XE41h`t-i{t9bE^z3W4ahETsw>c)M7 z;30xjx(H=afYNDK+f?M_V8Dq+|L?S6n?@Kf$!2ZWhi{5hrR|QxX5_#1t@^aWir?Uyewl0*OY!2EA;VP9Hn+AziZFq-mNQkO zxif#?ov95Iz1eOVnZIve@oXFnk}e}8AzX>P!{_=U(a}?(9op)eC^mVeQ9zZX=V2j zj}0a2-BHHH#{Gra-C3v{LBf1TXrxG6`V|F!4n|u4;{B>P8mA&`AiyZGQERv^T~(~6=4H*^ zqbu2KwCm2w0eBOT3CUebZAubwT*uzu0&eZ4VEI><}z6+eh0%~sr_Xrn0H zn)iKpRP|juylN<@5JmrqrA=;LKZPJC;bZ$?y7DN9A4>#jk%6Hs3fW-0^BLdA24-4} zo`;$bPyucv#~&3Nmudw@HU^U;V)DDZpH$4?waBeA=ROGu{xMA-QKi?suDR7I;R>FH zd6G6Lz^((uhMgHTw7qF;+yjXMfEjca`Y@&ruOS@w`n622SGXzYY1-&-KZEM`3?veELwF90>t*=negU@pu#%i7d|1jv z7N3a-r* zN{hJS<(tQS)@F;t*@U*6(H$FkN4F>-5)D^~0*yFmXSZgFMa&y5OQzlM34fGaIN$fbEPP_?O`iYK*^Yb&kH;p+5Ab;`A=Bo<2a(hn_E z4z1tNSu02=(pTeBc3RE9`DJz_GPhc##VH_sq$V?DG*Ao6`EI2@=*_}{!iy6MjB0tD zvCvQ%+2gSZtfiQVz9(zui2u3s`Qh=2gBOJ3GJOh5KAkZzcNit_jx)&@LE{k{h~YlL zipl+KM~swi)K9PQE}o{K=z3OT$l;yhRw9qE!6?4q@a?uEW2*) zZe#FF)1_m?6jJ%-AT{pxGegE44&iWKB6xzQHhsp%l4PTKv@6KkIp)?^=xIHE%U#q0 zmim4Lg7qoINQyY19FC0*07FDnhJTjL*6&MZ3GJCg-{oKQ_fKm)dQKZbC|?hTlS)kw zp!)W%DdCd|97`V^q~{{!S?b%Us}6Aa6=vS`CT%r&p|rgjhf}u#TCYO8X;-0eb9k|! z(lT$ROW(_GlN5<8WMh*C{w8~VGs1x!ig%qFD&k`um7np6_cq?v5eY$;vE5vgze}%9=F%qO_gn>qees zpSyK~0exmbcB%!20nB>28e_YN>T_hTcqN1OmpC*7+koCAc!kYIntVgJbC&d%@kBX= znU5ID$;Yi*e3MLE%`xy@Z>8Bkj#?s`iOHp8tgR?RF6&%#=C~@5l=NYw#|#Q176mr6 z#&Sj%Ck?gsZXyMWuks+>IyLgtqer7p9M~L&URo3J``rvk;F>+D@mQ7{_$E&U8GWJJ zBlcsL2IAWPPGt{8_K|NQ7YKXZl}DhV0puEJ2VU7)<9o@){eAU0QNK>8nQG$le@>4( zBEjCA>+ZcJh4YoTJFH~u4K6e`CSFYJewpcFOg+gs)%Y@0%VNB)XCnKee%AcGXGUY^kfoc0`ERAHj5%Hkw|b4(M+h;}M0m?m|; z6q!vX28iEszIRB!;jN2z>5=|mPlbMpUgh9`!_DNzc}jmb^p=R2$YKWro0s|b{>xqB ztR|3+DYS@isX9}Ww%8!Hz+e4o&;@d{u#m*D(}lUJ5SPC|BC zh>s7STc*u-U=vq$)q+g5&m5m#WYdy|Fd}n~^i|{azx&=0jY@Lud*g+v7L%O*N?DBM z%vM-%vSii#113qK(OnS200K@K?%Cg3fVfag#VfTzLqij<_tnB3_9jW4l09nb; zjVa*QYmk16`R-OYnpzLK8SR7I&o#B!@Fbod{DMyV*)$0Ijj>Gi=OQQ}+6Bc1FDTzs z0;=iSN*_nQ)LM|8&+B#_BGBcdnG00}pqK7_p~8j#$Pn6M$7=E@mv3A(iL2hHrw~u* zYK`StfpNsU0Z}hcf-h0rPC^|O28$JOyC259eYqatSK4;IK8hrP`pC>#Og3F{UB6$& zRdk68cB3ji&>kMb*>gE3HEXZsfCT&uD*PVc7ZaQX^Gy-F1IEp6MtHqWas9ZK>32-E z_dI7^s>+UKxa8OBLg1n~%uyn)5hF^Uyo5eN=c9xg1#0mf!~WGpGp%niaOpe|>HJDs zW|*%g>G!Y4;N|x-LQo`Tm5ygaSu0^9gzW*|7X5~6ss0ddBkDrMO}?58wf?siw+>WK zp9gS;j&kB3)y+`BW?hg%Uv#kRx}7mpr@j~GR%3F;#-Qm1*54sCUGhn>EgN;gs#CD6V!OP3pwNS zV^T~d_F`;ARC$83JwYn>-KOcHMsy6qfqh3p9Gut6pSQcoPGeG@QZdhDxpfgs)VAjA z9U2g2cRawPnIWtdmPN||;jcG#?aQ>KtUY6-^k-o*T*iuik&5~dUE2KZ2B*iuJT7K& zesN*EkruD=6y84rz;fTd^bxj{!b}=Jk67Q|{$(Gkq z+BDzZU73Buz<{=~ine}26Z|gCdjzR$c>v~pXJ^E4H=g3MGBdcQa4&SO$PBg==ngPp z9bb6z+h1stZpG)VOTPj60?B=W*Kddw^WpPH2Lnh`3az%gUy3m&zH;@V%Xez-u9m_< z^*?F9%NZgIoZo8A3t*QQ;p-Y~CEl>z@e^}NRpgSS8?*As%9`bJH>-i|NN*Ex849fi zgy$H(!J`U5*ZCRcNBBQKkQkUzd&tFRPr-I^`84^N=UB=(P+3QTMX0VTL6gfUgvb{6da`u90 zu$e#~>Qi!~4B@nS<{#`tk>JYvw!%n_RENDMB-QXoJ>si_OB7Vvgw@xS2YLLd0;(P5 z+fTc{=4*IOwtZ`9NMq=*d+ojZ&EJ7!q3f6tlg+Ht*lmaJyT9Z0Gssy6z+)!Bk z;N^wm+mBI>6m*p`bKkD&pvGj_J6OYcY|C7lc5HM+7UtvxxN0lkKm+^xoj+VtIh!9a zSlmPhZiQ|{?JRawwrKQjl{_x-L3!$R2X_S%(YE-kKA8EuC}~mn>an!eL;4bqryg6n z0T1B35WIS1M}u%JiyGJ;Fc*h0mA)i~n{{(5BjKmsD&DmTF@!6ba0dkU2;%*_;H1Cm|N?aq>#*~>ZA=r6giiTz$_3I&?Z9h2<&!@N=gWx$^?0L0QmZ%$IjBvb!$i;l{GZw)NA9Km@HM#)xG%c z+S@aTpn3U)D(8apT(G4WfQtcCp8M6FNEM4eOP zz+fSN#9#N36Z{+iBkU6MU!J@gawcv}*f#=i);gm37VVN~Hu&03Q~1lJ@>!uwYa6d6 zd|7iPV6)`%<(QXvU&o`?wozuFfnss#kM_zrieDcW&$ah8Qf9oSV!D;-*`}-f*0j8R zr`YIpxp4u?7yGLoWKA%J;XvaIf>Tn}ci~ooEXQWZ_wwh|#v6tW)}Eg5j+|`HHLzB& zNriF2WsDLY!VVvvCs4;n7+LE+DzwJySbFxyg(VNY>s1`O$krb8ek)q3qS8BfY8N`+ z_Js*aQF!P|5y!WPQNa$#^AHkcP-NDxTF!ER>QaT48{0sr0UPl zpbLZHs0XbOh2oN<6FSJ#+GU=)}5nN?jgm@e%241;H? z^Aqx&ISJN)I~51H#7U|##u_ybJL`5MdOu4qKP0IY;;2%Ooez-LF+TH%q;zj#^@MBia`KeJY!0k&qQgMS!4ApH7lN>N1iyVEa_><>UBK}TPI`JUF|)jS zsY#5Aij$=CMW09@Wv9V4mpQ@$#^R)w`F!5@EBy%`(UjQ0l!944>Z3BYt=;`+MEiL~ zm7(ME%;%UAv)Gr6hUh@hM*3>X|QZ@Ew?X4BLA9xjagh>Z$Y8&hv zcnz>PWNBj=&6PIy>!!$ZVRyA>ZLC$gLmQrt${?OBV1~cn2!kNUN&hGNDl^G7)u@EnT3Bt~Pv?tBot%+me(WH*-jFe#~i4{=vu(@nVNpv2w@| z9N7N;!B&w}Uh@(?ti~$=eaDH7&EuJdwf_)K>9HQ`H_FqYE$Y#fa34w>g7_AHv-G)_ zTRKKE_Jt>02di6p&7VB>MZfTqqNVW!LcMC!zRyWln7?Gu1gv&yFE5k(cSgr2@`X2e zXcG-naLxyCii3!01-AwGN2bFSmKB~VcRVxIY>drnEqlY88?12y`yrZAna^ov=NinkJwmH@@_x*=HWF)E;2Y3pks0%y=4U8N{?w)pXgrAN6yq_My=ptk}8-L9M2@yZkArIF#Cd@zZTikMYM7C zqw_owcEap=sE1jf{8HA7*Rj~Mwc}a^yY{lVWd*{T4&@as%4M6A9Z%By3}^pNz=WcZ z!4?c3>bkO(aJd}mq#X0jc4NTxvp1@%xhdgukXt464a}d_GlE zk$98Q6cY_?pyHxQBVN*`NWIi_ce-2~6VN~fXs3aig}-6+M=66+Yy>}56=5zHwE##J z&_i`~b-k$zGyfE@jgu<8_`LKxf#q01RSz#R}9%zGvefXBMOTs$lS-!}$GrBN9HR zfRZoXj8qv{R{8 zHLpH4J+UoteezBG=Nvn6#b+35%Sw3RQn2e=Kn^*Z9%afmCMn5+ax)Y+C3dX}w-F;8 zHSAr`Py~x1t&bAj|Er4Qug6^LGiXXhfCeP`4*^v}ZTaz>y1JysMxj53d6isi0V?1e z;QWzRCAE_H^gvFHqN5O`1$w^9*~It8UsM{X2?5|)>FE(!^xBw2KFxg`Sq?SdN5cPe z$^OLNP86Kf3>*Ri?_?Dxxi9#}@bZMe;*2RcjJH3;eW8j6;fdqQ4-a|6%ezh!y*0L3 z^fMX%zb|r8nZswv#3F0*%K9PnhJBVw;7Ld4L)=yqRawVbdRg0;o$~RipQ)}kLBE%j z0KWInC43lb=dJ%+p3lyqg1p4mp z|2r+i?Pm)85aXu;1?~SRh(LGd=cURhd)rR1zCn{d_YeGbYma|w9@N#< z{nvz_5Y)v8fGXf6FM!!M=?AoN?%V+dK7dajxG7*2W~+P{fG%@YRaKc`7YYCmt2-VN zlvPyxqrdv6MRz+bL3CB?+x=(<)xv$M12A3k(& zDcrq@H(5S)``;Q57cM{B!tx;A1f&9h)6W+^vi|(@dR{Z9@Ka?1I%EPo#$N#M@6G@| zp+9l`%MYXC;v`?aLIG;PZt8$W3p(vC9WHlFOib9G{!otv#^o=te1={5Yn(u*J^h2_ zY_94+b|y@V8Gl?%LaF<7F*Nu;M#-o2(<1Oc0i-0&XWvW9U0{X)!vLT*BC1A@n^j&O z4X8`#y2B*hPPe;&T0lTn_t%LcxdKHzhyB-Nh96zI>5?@!QNXPZ%Sx|kDy5%NK^MC4 zSy?H2foJ`7@R09MnutfOt1vdBZnQl4B%f8j8-9QmfVJra8&mX#&C${En{HFGa<)`- zax$Qk06b9!k7C&DUg*W0l4pOl(H^zgsp<3t0~+hVvq|l?h&@4m+lvRRH%Foc#q}v( zjYhOHaQMI9(#slqLw3V(bcekK$?JeqKgPb`n(xcg09nG)EBW@=1nCvYTm-4G&Q}T@ zhc30Re|Dob1zJbzp3^b($VEs#g(v(c%KqcI_4dxr*5)P!H#gqzCYu-*gWp{ph3@jt zN|d<}>0Mu`*BQS+jBnF2ERJLPcmucp@x+mb)hjqI8`1uBdZ8Bmck})Bacrz|txlh7 z#y3mVR$yDEjPrjYMR`&zDh58Yak;0^{vj<}%{~ zVh5QGj~>DSHC+6?e<&?3-c%HpPWMkLAZ^*H)Lm_`vbp|zjD~M(LV1fWNOQd3jF!FO zEoWUn&14@vcKiDn)=xfIelch>sw&p;o&V7#U19Z14Q%dW2bbxK)YG`9_-#O*h3o13 zm;2qhTI+Yuqs`F*IoZo0)(~s_03)33?RK7dE=!LvBwc}?x+Iqy5vCQthW1W=d{i$V zBtardl3)Glq*!{U7>}x3{$-~9Tvr`ZQ_p|%#q$C1VW)R zLn*X@R#FRt`TBJdSY*}&bR*luZU(<)K<13&{*I2Xrw$F8ZZnu|{o~Y(Ssmp~ zbLL9>()SPwTCe2zY%Tf^d$O=7%f_GY%I7e6xzn(!SVLis_yZ_QS<&p`A@yKg^hsU9 zvZ8~Sx-c`V@XK4&4?)gSE`d0aR13np?vfPf|d+~MDzB{+@Irt6y+Gl`$V$qu8T zUXsX`fQ{K)P}>z7zihZn3GFhx;Jjw%=wJK(YGLCURW9bEBu|otf1)8Q5&@F?BZTgt zO|+pd>*=<79UiP4^++@!>)7F(S@fI5gF9Tk$d?tdOd^|OVajXywU_kKN4YoGn3IP} z_aO2F^svgC%!5kj&635ouguIaQi_bA6v zk}>FD*N^J>ymj!QQe*N`ZbpYSeYU48Kxopi`#NYcj!~wz0N2J74j(1d+SYc<=;iYF5vvjq3^^uwZ?}iDzeKA?hQB2vCy)1$=P4>#Ix#&n|w&+ z4CdulIXGw6mw+o(BU_^GsOTXPcwAH&tO!Y6p>%%D5DUi4miED4d6LXnbG+N`Sk=i% zz-cc4-(>Q+4vFoXI!gR}TZ!6mrKL0}&(bhyS6rp^IMryNMW+RSb%IG~g7WBl-z{3_ z9$ZshNXD1pFIPQI@59_HcBaa0Zz?&rLFHcjsKy!p?Hk-!EkU~sq#B}1grA{s=$+ugALApQW>De=04C>9Q)!+ zXVn8}$DA3D=;c12+vyn`>2Dh%oc#WgZn^K(5H+aX0n&_>nXf*6GWPbRBUR7UOHpiT zTICj!AecyW@~Xo=#Jk2B(D%lF7;62R>z3AVeMhoI+T_$HIE%2_6*OkA zT2ry^l{dVLEmfJpUKM|aXIb)UISVfLRv5@9ZJi!bA6sxHQZSyy5}qR69#7PUL(NQlm0?diJQ%XV)5YZ`*a8 zb;QPkn=BVS9;mpf3b@fuC4$I2OdplsCD=g~j5hfiedm#~&t${FoC zI&;I#yp*8Zd3GHR!~&F*-fFo#5sj`^QDoS-(AVasz*J zYp@?4V`1y5;tDM4^b5aT!H6?qv*tirn>s)JWWYY_ihK>Lay4t-G zMi5-{Ky(=EQN{hKi6ul`ZU8ij?UX2Nf=0zzZR^MDA91{G&stf3gYbJp0Bo@t$gEew z@jW{Z$n;*S6-BaVG(_*1CPZ0MruxpycIB-L-9k|MR(bYc`%PBULM~V9j!Wih>3LsV zEWO;njO4x<_kw^;;9beV5q3+b`K~KJxZl*IT%-~xvPBlEFZ2w=xVy8Y#D^3!X&=&; zCP>Q2%Ti{an;*@+{5|%NaYZ!SF75exWn_@hhN!hQkRyIg%$-&-A2IY-!h}{*gC$?G zJrTsFT&ZOs@nlabG0ysU;WFl%LtV$;48kAeu?d#`I<1Shk-Te+jniP8MfpA=VH|5+ zfu}eesSqVH*GBUbH{vt~xgq)7Q1qF~5$bhp)d#vRD%8!d+TduM;Lmm{iK`|%u7&O5 zY@C5l#E*LVSiaD6K5^E0CZ-r%GYP5+RZSK^PGNy8Aj~p4!TLspsdLY>TqJBNDmk|@ zcEIz?-Fw5CW+Re=8y;`I3#LxYx&(P}@1Y@Td!Y2l$h3<(AwJ3`QRHZX^YNBIL1Buh zdYFmvHJknS76?I%k3n;I7VI@xnyxuee)d4zY_VOqMpsd6WUZ2AY;>Tg<(d^k=mV&J zC5iXA4hEkIqnFF@im3m_@4jbmBE&V-=<52aGb$A0Vm=a~>s(I3_~^v$3J2m{SjJ0r|ST{#LeX zJch`>wE%Dct(g&DDz>wwAq+_7RoWBH)Yhb@pa9ZqHH7KVe9KXu*>L1k!#3e>r`dOlEMGkTO%wpM<}Rwh&>fPAtuvk^rz)Wav3a! zIF8Gss4^j-bx=noOPR(ejMH2+5`2nN>EXUVSzN-5S>q1sTDf>=M9+MM$5=%47#?n@ zdwHo(y1*fPgIHZLm%i(fF`MILuI2QRYP@x^=~9#%kByZfg!%lnCWN^E(>F4aK6R1) ztX~FB;Xala??0HtkC9K1YM7X6Tr>3m!_4sAQ+(`D>k*soK?oaH^kg-QC9r}-eU>n6 z^o`sJ=AvVmZ#+G>2ULj+03Qj&%- zKloi^C29aDgx5c7d7Lm+d+=sa7>E-)p?N)-a)&)dBz8w&^<^dN4bpZw_3|1@?t8lu z5D3z{!BD4$*KwS1o<8Yge)dC4%xIdjE#Z}z6!sf$9UZx8-p>d&&zQrdgJu!EKa-to zC%yX6_0P~LFIm}y7*GgmD_5Du3$^v zNZtu6p^f1!#x0VoMiu&R*Te!c!{4c?$rt-t0jIY~6=a6Q=FTE?`KT5G5aK&7b@Jmd zG7_WtqWVknvT4&={kV4Vn1>VKkEQShhnr1&v4o6bNmrL_e6k_Pr6r$Sc%h@pNEMpT z_EGb>&fP;qW7^j>_m34$zgsS-4k;LI*v_m`AQ5MuYzoK3Oej`=6MAW{PFw4KLdubz z#7v8)0~ZjH5!CRWu3?8b$li&R6p4RYC>(ZcZw@(O!{aeNVzA>olonE0-B$G<-2aS!_lbdyTk^U?b4o9lnm1q}G=^E->TD^s+4q8Lh{` z{$bE(8v`v5hHRE}DL)D|O3_=X&m7NdLg~DYsndR2#?3L6CV z#j{7&$vml_P?vA~w=pXZm#-Vk5DsCfAoN{NGu8VpKPx_`pru*IfyzUubMp*gwCbPWjPhd4Xve-A;gPR!Re)fOYx7RR zs!(n3<<5e+1|R>T2rQ?SU9C%@(3>eXqbs$~iTYj18rLg0OpA_TSV?@NSPFZjxC+fN zqj0R5t8*GBM|NLRUZ%=bHN|FMzfXF-wlb3Ue1UGF&pVuPdTvK0PY*;oY97=b(zGkA z**}tFtkRS!9^~;sy`pYOfL7O$G&l2c<_3pTsi4f(2nCtmvuobw&Z49YqsdN2!rjhI z^XK{1B0k6x;%ZDE?=&Eltm`p#Q`lgNNtL!E|IlfzMM=*;qJ=((gi{)I(R%j>HX7BH z0s0v#)=k%p+M1d#TKx|}odv>W-ZZ7v?0Z^^OJ||i%hmE|M2Yt~a7m3_OO`Rc`%qet z-V@H!mhfhg1w&rajp%$`fanEz%{Xd9SWH12sT!Wem)B9*>siF_K*5V(qwcYrzGta2 ziI^c)8Uw*}KE~P}cx_uWBO21j$C2YzB}615orf+mOr*ZccA(6%O~g5}TMYrtCg)4 zWJxmDwhX@u=)hd_FKRxFu(BVEplLCMa@el$(EYH{Ymand>H>~36UMwXhR-v2fB^_h z>K{@sO=Da`@1xW~F2houH#RA)pfb|H{CjAN>m(Z=;N6oTrF+4VfT^dcg2frh3_9%h zI=x?*QmQW|p>i8!Me|u9@g7evGc&QFDcX@BfvZ8E#OKIY<^)^I|FCN!ELV0@YPV#7 zy>5x0+I*wY?u{M(sXgf{=FOw5)RVviBf*%Q*_oG(%_>{ARI$SzKw0Qv%dy?#Mfklqbffi#go;8#Td6WZXZX-@*im+4N4b?P&;wkuOr+s8Hcx)8 z3V@^xs6N4Hu9Ke=Tn-Rxx3*Q24kfHk^I;s=kr2}jeI!8w$cDSx%1KvJhuW5{s(m}Q ze)bG$gcT0-mL!n(lF)5tPhM7}JBDVb5LrwYhHx~m@Xg$sV*8L53G5VfY!G3z%J*j? zXg7-{T^;G}V$$*YR0c21`Ej&%l{zjv&i8=Fd2S3(mUf*;{Fr4n&XQbJ(Zfe zs857c#yR#gpP%WLmI9TEl>U`+^z~t-y6QtWcrDO{V*Z@z!8a4mcB6YW9Cd7X<7OO= z3(OODAd!}f<))LKv#}X$noJ}vfVmL}chE&JQbsR@CHY~HKhxIj4&=iCUXe7+7>bq3 zj>^a#hZ^jESo5PTV!d<>`jvm6xjD*wqO$_+-pb*$&E^|8hZ8j1!+q*EAc0n`S8aQ;m) zNA-LR3@v_IEiqjMY1bBE^)mf?(h)lx5#QHwe#Nf+lt=(q+~T-oxs%*e`deXeWIq~$ zOo}u29UOe^$h*g0ADHp9G!Ey)FN6W6$<~MffaQ^z1PCfGE3NVUo;2M1x>my{$a!Vu zl%S0?kJuD^hjTi$X%%|bA!v;Z{AlC}9kz-%B%OZRiZDD5M|aBnT6zKz@9`vSQQ=Uj zk;&F~TT zpRisFWkZt;Jq5v|PMA!|tgY7Y*@BiXw^=lzu zr17sRDy4KCqi)B;>m(vB6Q{H#VMCKI#aDm-d|~0Twz8sb{z7)UceuOMxsh5J;4DhZBRjr?kc0+ywsTy?t!&ent5;JUvC67hLH$O$WXJ z9J4M$Lwd1udwJ54WNT39>ha~@?eDK!`bC0&#>H7FC@H<{eu$dO} zLA;`O3&&xXSE8IXHrUXj^(8yc%WFEm*GP&?5*{9?`}zV5o}saBO83}{^&lb7nzl>S z^4s78SW)-P&5fREbZea>2~cyVB;0j1k5}c?RmLpZgcP<~!N_Xlpp0oQ)ZTF}qAa(=pL=EZKc~BBBYLqr;d~&!L=ROuS-wg!9cN^Ye%G(eQ>;Bi zg*j(@+pn-@e`lzmW;v}pR3&F6Vg;l~cTQKF8UT^0ztq+j-#F3Wf2oFdu>H>G{pyfZ zh0n>p4SdGAN0sGO43l*>t`H9} zZW}_ss%z9bFoS0)rtR4h^t?z{L&t(@rpi4!-WuLYML{>XF!sRb6}w&d@0xGvl^MO8 zvsqWW`@ykk0}P5m9gO6wcU?QkxtPm+#HG-ctT$*Ptr)7`t&m$hET1@zN?gSct;846 zjdj?71e{e6{bdPLaZ6BQlcOZwdU_#Z?)Sxi{Yxyt{#VUXYM*Z+4RIbqc_%+uRM@fS_NmA-k*P|Bw<7FO< z+bMRZ`74EeLX(QSb*PK!52u%J5vlTqSL+!%WKNRZv6J$G{|#|3(7z0BYSR2@&*Z6= z>=v)zokzDlXwi+QWIOD!@N7#0=F z{WHC$V@<@)^Fv>oyAW{$BteLji7QRG4!h!%Moog~&9KL}J*K5&y{qRn14Ml=n)Blf zCF^A!V!fqIZsvi;brzPb`GJFG82VE+KbMSmj5b0b(Z%aqw*Xz(qh|MGCZF|T*x$Jt)OpK5U3K^QRZu;Ds zPIs}-8b3w=wZe$5eja~w2NlwbZ-QMwz!gJ0vI3e#ildYLs36o~>6?Ro# z+zsD{&cKiHrIA`0-Bw~At}d43K-Y-tuDc6sokY7^?#|s6qq-A=)yG)#6Z@^3eHlab zFR5CV&r{}-KK85EdJJJj2pz+FHm_m*^&)Iper6_r+qQ<5IH6#q1fiX(R}9YW2|0U6 zmz=Ze54QBMRO67)kL8v>JhuhGF`o%c?}t370gWt(+lFJ?s-yS{LdhYf&LkB5aS zZRui5Yh0w&_dMTP3JeAH+nBkxFnys*SbQMee9R_yrqfGG<(nGm()sKoWZm6y4EHME z<&!t=JL$`7(fDx)9+jq!H=PYaM<8x&!gfAAr?-CtdMbfUa6t0A>-OwM-K1V;)TD9CFQq*pKYt=BS|zy=1iqXYQ?D^?}n03*FQrp&b0+|MRjqm z)@jC1(vURVjmdaV2dMa8OX-SH)*$GoD*G%3j}(*sT0-h2iXa*$hW_7a+sx=M zX?=SV>0+G^iXJNq<3PKe>*fy|K+K5T@YY*1VEu%$)Qi2BwW%{bS@DQNsqEM`EP6cj z@FIqQ14PuB#if=2+TDCoZKJKAGftU2$6xfqO*C&{Tdt}oJUg(DL9i_8ht}Ds>1kG| zY6apc74OnAbIF^4&nN54P>eAw<_+AG$!^W<513`g!S@E|sMdYP#YM##p@M zgK06~&x}W94FUzic0nN_RF;}`zUEBf5m71LxvISySJHRa_w!=b)^JqaN`_YxZ8Yb0yhSWNRb*Aw=;)z+O8Ci=XSmEZw_n9#pP z9;yE=gJxJZ2Dc$XWSxWEC35}Dh`<^x)_KN>M9_o2X5o_N)yGG!k=}GseqER44e#H1%Uui)+%bp#Nj1NaLcxk;(NEyZPu|dsy~PA8txd*oT>w z)nIEOT3atBwC%d9qnN8&AjcS?cMpfU26Gw)l22pvLGn5i6>2c$=)%!y>;RFV$uo_w z=>!D(iw!urkSAnh$ysk)Q{`8DJk^XZ(F1pgdZzoYHn)3um~eRht2RLWT|NM8pG276 zG>5I@@oS&Pc^SW5lu!&-ZMl{7JPVw9r|U|i{xpxnvYmp=!<@9JBMpHwa7<7*dMH4c zNw+b&QNwHad;_fw>QgO@s&RgWGSsug{6kRBbZZ&G`fj79ww4Wh>0w`=`S9%^oLKa zuQ-r-p^zz|;{8AmW0q;RdXdwd+>!P~YAqjLp^=KrI(y{>zo{anQE6i^NL_n)G_GZ3 zu0^*$npNJk4D-)+o9}qYU!}HcF}Y=hZYH#Vs@dmoJzXD_94l9 za}55}el~N|uwZJ4g3y(=h7n z8b-B)4?Edmx4zzIyR!8pX=P8L7j+0ip#q4}=H6A93)41z;&YsPYPUEkVO@8of})Dv zS7S|0GC;-3>Fr#n%FyJ1L9=tP8m&%yM&lob%c?mgvI@m7`qdPip6isfYzf-o?awewlizolPPf5Bx>-MbckxIhD=To9erWsGd$UidR zVAR+)Hb1hH2PA9q3_ckwRcI#$h?PFEuiaP?NWWiIdbcZ=?J_}~H%ubFw?a!~sGmq% zhaYbA0&MDenf<5}2PMf3nvgU}_hcLW=Oy3*Sbck^JP9Mc;FNAQu9?^x5h_FL_7ALc zspr5ytHJu$+qnSBx{akm2v22zsL+|}+Xmk?bfS(m#**Aq%n#=8of8GFc}m`2qAT5A zAyqkxsg$-s4o@2h{+ZgH@q!zZxzbg9pWY<-Ldn474`4!E{7grYw- z+c8KoJPi=dfMsJ}tde_)9joa<&pwpot#6rK4o0eDb;%8T;~%fH4p4XeD;{d=85nQ? zygQp=5Q!ob2oTwl1daoR09(4Q>v&R;WD3qer4<$6q&!CW;}kA{*X>3}N47Zk zh0pzc(hW_W4{jup?7`0e1r))1_4~@Mk`!{tWYka1DpNWF4T2A&^ZX9!wKBJMM{29XzP zM!<&Gy>q9GAQg}?(cxsLaGu}`sl)-;IN2IDNj+p7wS}GEQT-H0owd-7Pw)^&a`=QPgwWQy{J zVQ3ZQ1=ZZqV)beAv>{1tb>a8==+K_C_(^vTYQ&cb8{84!rmJjD@z7`MPRe0<3=DWf z-(a|83Jfo}3cW}DGbQ=&+Y(q|AC{Ue3;kq=qjps{@yc)j(311B7PhPkC+H&1Dhe&7&X*o{P_C95$#8OJ4DvoXBHYk}i16tcih$KUq2Y?fVf>2)stDYe z?Ea)xHmZ#J*Pc_KK0Usoi&n6B9Lgl`qH=HV(F5+knGlL~s$`R0ZC{Nhde^_+=9ihD zxseOL(xI8vd_B2ZIE9@!UekrAAv9F4+!k3?wN>)2qA3FoIV5 zhD@l}v)G`gM$wzg{m%7n(IcqvyTZlZ@@Ot~pMYO;j7r8SHj3uy68ON>5;2Y=$^WXJ zn1Pf#63*O6070%^tcM)$fLaO5{c$#R;60?9I=1bVOllYw|2^^PjYWL(>hr@di}l$b zrPvB9<=}@5^<-GdEz;G{TdKn8@f`A|mCVly+#Sz91w5b8;(mYO;x*-9(!sndFurQ^ z`C|2?q}*tRR>afgmQ0cwApsPp6!r46f2{QPJcR2YoLo`RvS$l2x#!1xr?69Fcy)$G zTcQEE)RUp2x33rN*LQg%GaMRI=DA;7?ISF2jJY1Y^t)PNT1(rAX^UyHR5doqVPz2| z`L{DI%u+D87Km#JiG-gb-Rt|1Cp@8{z6E&yBA>poc==jTRpb!?nqT5f-y|eZV9MWQ zW2eAfy{-1Lt8Sf#OiVm2QL;N9E1j6=y*C<_Zd@uL7Qa-9d~a5bex0_2cqFqeyeh z+#`vKoVA8o1fmwFg94`r5SPCc7Kn_>=YBOMDYK^#b3p1qyZzvXZYQjinP~gfO>0cp zhC}4UyK|N6H!b>$3y7AmJTTe93=imiG2^crQ)`pGJATs9vPRvh;WU_>pwjI_I{8H3 zu}nma2-z+O=BKgYOm48om;wb69X&Eea(bTx(ncbCh^tstt{up)cZ0QC~c+^-XAjzO5!9gg(zIjaTv%^ycAQ``6! zi;EhED5w`?e|CsWh1fZIzEV)>#5tsyLe}hKaxxT7;zYKZBKOJ4x)lO(7(b!zyHdRnreo-7QDh&fFf!xD zhWTsFNofWmsE>3$tL?H|Qig0Y$Bi_z&`>(k8AZtxcqd|kt@MP)x$U7Qwk8;5%O6O5y`q9=atj*OAnTl4cwj;r-t{6HIK zYz&{+WJAvmj|WsH<+k{ey!)2QQ2Ze$4>^&*uk#aYc6%jb{!uNN$>k%M%tw5{m*67n z@nL8x3zW6*i{G@>g82kFfl4MGTeEcZ%R*}=X`iQ?+5E)yj}{w1+s034cR$aF^{a$L z@ZdnTs;Hb=uC%1dFTHF${CkvLyGd2c$ev2hb*dCr0|k6MED-eb)veEs#(yEN%^CA| z)pN$yJDttLV5fs^(S)YA+(Mix`oPQ_^NV4i&*AZt!=^(O(W+Ggy`n&?cS%sTqdSb8 zl`Gq)?6ZBVI5wVF9kC}pKRgHL<<+-w&qdjp!OG)T?bJRUd`Mw>UO5kB>=!Dl74w4` zy!^s}z}X)@E8H)gAvaIydo`|_MIns9)%NwhSpV=Q$RlIjX5{luzSA!DldBGi@_xNc z3#R=^0%cBqL5vLnv1C(kp1z)W#SI^W4pw{s(2=}E>;aS9(~J~>t{6qmb8U_wL@$2X z((jejM$ppbD%jDohaT6F08pJuAFC<>M2j=EEsTlv+dL((z-;0eh;4hMfz$afWM+9f z8kXH8$MCg)+K|{)(IdAz&k2!UHBjf-0K-p{cyWflb^j9wPd?oh3nk`P!XO%0MrfiB zhiYtQ^6cit4WdSv(Y?jRF~-EN87&yFsg&YDA=t-eXlf;|#!IR2c#eN?MV+2*Fp&kX z^U#?l;nC}`IO=>@C=`q~2|Y7G#}9hl9AoxHnn{!9tznp&9#uI@Es2w2_LZGn{(l%Z8$^ zZz$DWS}9O3-VyYwIzBy4QLsh4b>q}J^d-&PEE+Ooqbs(aJ?yA?;TS{$y4ClAJJPs| zJV%+>irYNb2|-w>iKWH8L&?ZJ)N^zKJuJXt)N&|wGp!Ec8~0|K@&Z z;C8q?QWiI0#*nP=pUQ#3TCl|3dTesA>ZsViFI(;gf5l<=_^E};GrUu#vOWctY=8`h z*RMrkp8K1(qWmYmIlF)LpWotr`O9_RKEy{AL%)4DrPl*9dNMLk#7zxuEYG7HSO1{ZQ_?FroGW3U+(L0 zg#yXk*X^$Qe*pum0wyflMpHB7zqKYnfW@hZQ-}+b^_>d;)tY6fv(66o_Mb|bhE{yA z0V+%g5CX)XEY|%+1Pv?*5dIvxvnlW+%!&*)`u&eT{~dUA@V_xmas782Rto69pfOTl zUVZy3IIKaUQLpz@ zSZVU#N3r|>%O9=H;>`T()Rx*$g@wVeD3HIIBIYuo52Vjk9|rS(bn~xg|8tJ#SI}5P zh!6WvBh=6&MvEIsI?<(EPaen>IAHfi!Ku(8?)Bo^xBS5kA}6RM0v{Kv6EvB3k&nED zxip~LIra13jp3jEzBl}1iK}I%zs?QC^FRP{L;e7OE2*or(&wfuKURoRQc}XyyCxI8 z1j)@U6Lzie$zzdjdmg%$Oq>)yL{;8DaU6x8EgTy_r1An1y%f1ML; zStfu=)aZR!>B)L*xrH3HCMFCMSU#gJk1o)FWuT&socl^)x>}3qZ=^E($>?>_8BB^x*#&+w{}&Q0as7h)`j2vqsr1;3eXN0UBx%dXOJdWlWl7;V?&grH$HCbNEB zQlG7O@M)`%AQG{;%Hvgru$N<8CV9KNbJ7SN+07Z$0}{j5`7fEiv&ENj9182_d7@rB>o2zT>0-DRm1wcp0i<3%5h|@C4t`E8uY$2SDGbfypGP z=Y&oU(PTF|Lbz7ND$e6Z+q!Wd4cRZ!_BYkFCxk`k2J5nQ^or!HM#b6|3^F=B6p&Y4 zg4Di~`2<7mwp0=%mNC1w@)|of1*r)-*l-zfUwFK6lF_eD-+3Qwqa;a*a|b`+X+LOd zmkY3V&U%UZngv&f3D2n-zX3D)G`2+!H8;El z_lZ_UV2X`EM^27}ak?r$AYDGrK8j>DsgCmtclIAWfjlf+BA4FN7~Z$2R@aIC83+0o z^K?}F<*jp~&L}!I$=(mi9+FWcM9v6H62%6iW#d(!rs1u)^|4#i&Ckgmh7!@4r%asz#60 zY#nnI@eEBVy_`Om&&(x1#($i3cYM1H?tAFKzs`a@O0}M4kar7vN1xUZ{u2a(b$ekv ze3h^AtKjv<;loqXME2vdHt*>1=paJWMBCCe$ww<$`bm}o3xVsq)6Dd&PHvB(U?TU^ zqZ#bJOVOU)obj&KyDJ0&=mNyKJ4#(9-8|lbkMo2tF#_nv)++I{4$GT z?0)he?3>KqlNVkR#~zbE+;i;nK4n@ZdV9&^xUj)$3U9h}9|1AiAcjC~{FjF!ZhG|X3pEcJX&n?3vduU*M9&n zLyOTjK)Oto$FqK^VKbU=Y+Agf9ea;#{(#I6cfvR!&vPE`@aL9@o|!Tm-VJBn?Q{DN z<(%^-euHIJ)v5E_o3yo(|CiLc#3d^w6$BIS97G_Ojq!)KdeA==2IBo6O{qVc+=0wy zT!ab**yHb$_`5E-&P2V5YHV0bymu-+3naZVftCnW3w%{hG9rb#W^O*=x#NH(@xak! z5ovB}y>D+TYnbJY>vATgDxJXZzBUwQ=HwoB6V&d`j2qz4@y-mrDvxw6`5v*Lro3QgJ4tB=|Ac+&hLiaV9y zJBP{)^4OBj$cq0B5#;MVga3U2BFB|g@*dCqJ=`#>XK=!Z?NVi^bpMKAg}RZ~o-lsj zCL`9T^4i4g7`!|QoFnaYl05er6T6b0{xOEApA^daB|1NJ!t5NAbw@b?T>U-8JTJw> zedRDj_LZfB{w4n(zJ&?MFg-_KpET6ik(mGu4GpGoHJ|@Ks&dO^gN+ic$MpWnBXVv1 zuykbPsdPp#nIMU@@GKyUkJFWl?@Zo10AWkU-x9I`bKP`6>YZ(D2~@? zd-jFRS3A+3SN+?&u|oJ)HigycXXEc?_8M|{h5jbxur8N<|EK6Oe<&WXEDw!(|Jp6g z<8qmVjQhohj`}6m!`94{JmJ5Vh zpYVUbBKI$B|Fy`@|1mn||B0#n0-60y?}qXy= zZh#etV5$AmNgr|cw2`r^ySuNi8(o_K_Pd|!g6QSY(@9jN#a=$NWdib9rz13!MJ}C9 z*n@27I*=P71v%TnXr2#NYJSPORj$YB*f+-LnZ3K>_3IZMHb+jRh>*%k*2LL$Z}ga5(|3F+R*uxO#Vf>kDdr|H60OyGxGKn| z%~xl2^v(y|={omO8tN3s z^uzi?Z;6K{ftu#}BDj5(*prWa$x9isi0f*6eKDquahMoYNCg#)&JpM4hKV<-c)0r& zp`>oMx^luMPFcFcjyLTFD#|ZA5W$QZ$I5|EiMyWU)$*j9<257r?N1(3!gk7_ruk}a z@@(`aGJ=u;qx5X{CGhd&||BZ<-I&yG*Q#jl;vBqfMZ z*-9+TbwTkite2v96#AM-on>_{F^4`-OhO2i5wYH?w`hPijt!rKiK!|v*5b;{iFW1= zv^yM3x;Tr?qIBHZdY?4Y8XGplZ%l;LN$j}YZz}B0*!_dtD=nL#f^uk;{2?ynyIu1k zOwaMKI}HLKbwLZQet2SB+8Idnb2SWJ0xq^79J5~ZhmAlHr{}BtOk!dk0rWY+${43( zy?87^IUVMkor=QV9_c^GJ;|A|u?%zrmhZ*#DwcY}L>q#%-O~EVMf#Nn=#4V;eqL*j zd%d+^>MqnI?GB{tTzt#+V@N~HBSMDl)R(D%E0naoPPsj=5J>HPupsLj&i|?ZJwzgV zx90jP>@qGK0dpBEvY1gNM__nNIz1Ax)#3Gz>PS{y{z*;puDt$@b|-|vk_HDLH4M_2 zhS85Z=UIfv=Oms?)(eQ~jhET?k?03C?*Fn0wi?pIXJj&s5fGHP#l)Y~ke4?S3 z)<0Gg$)77J$UCqni3*0k95xCb{n!+K-mua*j6rJoX%1G!fjKLaP{2N|TqY#`@?KZ+9wqKV`4)g}u zQI;ye+}dQsij{BcI2?9O0so$r<=t?9U%#MAjGS*_x%&fbh0y2oStE=L5NS(R4~ShW}!>Uzjqq^dfhjJG0(;d+VKy@5|I=LNWKq#_Jc%PHBx} zeITnSG+x5d%$y=yt?@=jFv>l&RL_~5i?ckHuUzpHMO2vCh_LgfW$yRq9)D)lV>&0$!&*(h?sbZubKSZP@`LxVN$<6I^#Wma`#wXdGa`lEHS5WUw z6&pINXB;3ZRQ#Qw2elKwxH?T;?fSgz4^IsMeOv8}2z?U*vL9|PcuRFB<%K^`EcJhr zpQ^34HbBG%z3d<2%e$2IB^R(`QI7^pk{k%rQeDJY~;$T9NZs{@uIsg>P9UnZg=S zL;)c#nE~F28P`hBNv54VI+XRG%vNm+JE@T6(@9*B43AT5H-}w1GYX_s07zaXXU~*L zqnn@FrF60Prku#O-06hzCT7pnf=!iSzV}9%_50cqW-xyD1MjUw9~j8jglI;d4VBK~`4p-w3%klD&F%sYEf|N7fsu`L) z>09jKs(u{R;G^a#Tk5aNmQ@OxwM=_6{jJj4TxhnDc>&oQ?ELkR2gl~!dko5Ek@*f9 zQszXx)uf_E$L1VuUlb8$A*3hvlq1&~h=d@6&uf9C%GF+zcyFh)j2qy4!)~^k1eE7_ z!>8(Mtk#K$Oq$hcyFywgQHSB@UBRT?Lp5hPB176=!yUrj9~gi`4=f~80*+=FhqSTIITkkRb7V^_RF4yx7Bmq&bV_no5w7Wc`B*|V5^~9KpsvL^nCRG?lOaGQF76i-<5PtQzg{rB{%EQMyF%d_h zt*Izyi8z4i@Ig*g1iaz$0x{Nvu4khi3;n|Ki@iPN>M-yQ)FSM8gsM^+zZu-I5&;DO zk~&LqC%%DXQFci{AdT|eB*Aq+Ian9vGt&1ZE+^%*0PEwdnk=0hSk}`5#p6^J0rGTl%E|3f?xXjq#$go7~NW$-}~llPI|Q zv3k}iD3-C#&P(~Z1rbITNtWb~Pj6NAsq`QUwT=A|)M1SrpdKJf=KLc;7y@1dzx$XP zxkcU~`~|s?T-_cuQpE`RY_+E*v5H^lHUNXat^aI;MJDAXo<*9nxg!Io7-w&L<5Q+- zWgZ@WOal^Vk1rwtbIA7AULhsAh9$Y2b}V`Q{cFupr8<`oHXxmUx4ro8Fg-r zVM|31m4HiP)wG$1zFvArPbyq``>{}OiOh)#jiCigEl>*01Dq#LSiJ#R6r7zia z))~i>>_>C?-hiCgv6;-z0-h;Fh6J6b3ej1<0hZ|Jt2FPl$KS^0kgfE^iuJh+$YJ_% zw(4%&fr9qL_`-AE=hYOXcV032opl>;Dza%M(*Zo)B43*dp^d-j3d>$;HzJDV?|Y$H zQ(lzyL8x!zTi8*|MrsWzsecT07vtJUmP_r-%4lrmVKKgZBQ0H+s`J){%$^xD^E(?x zmCVfu=jgNt)Qa3TZ94;i<9{`O_;QFdll^69r zysKdH*5hE0D&9lnV=ZT3y!H3*3twWqK3f*1f!{QY0)fLyBMGd4;j9HP@c#4-@3+b6 zcZ!>9^61TTioH^pY=77{>`>QgOXgc^UqM#XF0#+Rd@U|2sLD=XKv-DNMG9W~Ry&1- zJUliwE~UXWbo~lv(ap7{wFwVr>pe>4WnUNW3;0*h-X^`Y;j8i05PGT*ot?36!qyvk z!1mS7kJIlZG354%4G!KiLA026=+41qjim-Fo(>ldgt{ipVs(uNiqAT;1Vm(BLl~2! zatscV${#*=QP(7w#Pk!5%4a{mnqoAdoNCv{*vsW^Hy0hRD_d%_B#r+d@BKVmm}gL% zGL{oCm^a;XOWu%pQ~7aGbBJ4&gaXXYMqp)hdw)5O72vFy(&pzBGm7RtHd|Os1O7=kk#rOhfO{rXIO{=VoEQ>Pja_NH2q~xT}DbzPM#=@$5 z04;Ydd#;p|okF8}C^QOE%+TpEH%Q{WTmsh>GF*{ff#@4~0J7%$@17wYTATNdo^wNI zfsb$)V??Xf%2OnFWOD2=9J7sNsEKAn?+zNF=sR7N)n)0>Y~jHzTMXq03)>(qPNe!$ zXv_o&J8GuMC(eQIDNOcn@*>fM+-kgB>D6%F^31M^Hs-vk3Qm1;_e6QO&gnA6zO%=3 zslaMfz~-u7L^lGS1lv9!sEhk^&AFbin93I*v91=samJWB{@OKEWlShD*GlS{gvY5u zqh~0B#ruUbvJ2by^zUy7tRASrCol*!bn66ah&UhqHLr1H}fEed| z4zP+vlt$u|Iq=v>hXUmeunYh&CdBkT!@3eJXOk%dhO_3m>b?CeE3&#eE!JqPyp->V zcL>s1G`JdiD>!&PGHbdBz1$9sHN4U6En?0Rr1Z7$OJ-egih>pvo=&ue9L;d;&NTy3 zdJ^xOucj#M8DteTP|(g@r}%j_a>6l7?D8nFsPGRyHI>BlY||2H9hltrl6?)t^B5tx zE+Q~|g|+-Gp?BeVhf?_HfgASlNlh{+%aAH<&+NxS?bMqPZ8cpP2f<+I6N%@4+%A6{ zHhP)0>!vfZuAbXYUdqGc{tWb@pSAO6vd=SolqF5Vwe?BNs(-g zw51GFMS_#K^6b{-cLw*;8FZkk)A2uPl7#EfA99(HMFSu6lFyR-JP z#(u8E#vRT#6&L+(SQRWKf*Gx*YwnzkMMLuKNCAFO{K=|5tEyl|-&cKAU~mq{mwoF_ zVr7kC{Eo_qS)SK8n_nFn|Y>)^H_i1TX(41Xi%YeuliF)CQ~ z+|*G9R3J&{;Ftaa`597YhNcsAzytZa^;hI{Jqyg@irhk`_OoGO9yl1!OL$t&)%lH| z-qYA8I|OxV_Gr_CjI2{Ajr6O@^o+iTN_@{eG9KP$8d=7tl@vx?sc*dBu!sbQa)TIm z5~inkzEyjf;&XNx#|$}ZkceppFKF`2%wqAhql8&mN1jeJt_$Z%{&T*|@-Q0dOWsGw0R~B0tjEvI7syto7(68$fyN$Pl9-%Tk^RKk3u^_>9a3rduY(qwxbE9^Rj)XHc-<$SosF+D%XuDfBBp5HCOS1yb}K$HOUMeh+BdEa>l zbQ2Zkee&UHwqXbFdg;RV#A#)Dr4*eGB)(tJ4(TcO(60MvMWkm6OK`HeN=z4Ob;1Qp z=8GmgyUH}w*}L*xynXXU5PwD7z^VskL=Wb2ybnG2{UMeLDp<`)I9fcea`hQojA^{> zV!ISZJr^IB_~RJTIzIcDdwEcpXmer-$@}l7#34YbOme~^KU2h2qn9Oa!r~fG9ylpq zJ(6@IGmp3qSNGEgyZtnTPTekdQl|1)QkAi#_)mo7^%Gs`2UrFIF42i_C0k3_CLmQm z`=s?gRgT2sl^cYP_XH3`2+*+sK>o{bs1N!g$ODQldX~#Z#|AspW2*I3S%SuI>H6-= z62K{OkJ&#My|NGZ=U$?J-|P2qmk09-rro_9tbwZ>@h}G}_MpRK1<)x8Kt%F0^yzX_ zesU%9JV@j2{UKmkH1Lg<4Cf$<*+(Y9uW)*510&o`ZzypisF3OdNGaaDq!mjo-0%oy z!KwJxey5I1)L>`Ai~L1NqxktNni7s)dT`q`zi+W6 zc1OaFj+y|Okwt}DWsN5j?v9NasoV}CH}kc_9pb2bCeeA!)cG_jaQE~@GKr~F5y8Op z=?bFTBsPY)c?`9fUpkh3Z#z$JoPK8k9ts0^IB}h;P034C6`|1nq@Mz3~+go^jce=e=2B9&TE zo8S@biuI&T4!2zHQDJ|>49-z?a!TmPeNkc2m%jaF%@RfWy{#QJk6^cA3*}qB-u~#E zbx9_<(tyQYdwpfOE#a<~KBQ|Anh`4euv>x=nntKEZ}&62z$2XOu8BK=ASs=4M`oFc z#C+6|&KvI;HHQG4SS+rn=nHgAhu8!|0dPUI{Fl%SXMOx7F2yLIk~cze6;^1UhCr@G zdpn2H(%rMk8WbK`4;_I8NlPGm5Ofri@2v(712x_MFN6xWs@Ic->uTjK~<+$GofYZCtr!-4eK=lRV{%D!y^EQ11V~6$H%w2M{|>XZ@UlbdYn=Y zw$;Fy+ZO_Ci14yaq;G^jNI%C-_&`9NJg9z~@P?`QbHY3ITd%{iCcnc|B~}_kTDrL4 zXG}Mb_s=Y2vW<9LCMK~ynnr#XM7^j%Q~a zBR?jsC@5MT1dB$EcP0@%?sCsjN79un6>&aejgkQRb93qNz*~VF3gPjq+;RXvE{<#k zNwS|)5jbOFz1vdYlcjA5Np}1m0tPv-^VJ;TSDbfS2k+PBsl7dRu3wOPhPO1u#6?;3 zo02S_69^%E*YUgOb>XpyAs19GXbef2p$q=-LK5k?1^XG!CYvSe$llyJjqc?kCBmyW zTiK1_i*H#z7ZJYFNSmq6qIWN+XVV{C3#plFxaTDE%8gJch#%mtFn{-$h#K=zZKzo? zIVNVnr8<5jw#>5mI!tG3$X$Ogb2PoyX6|gxg0Iv+1yg}Y(6Y+s9NTN*E7I`|8kt|D zT+HEWa=K;P$L%2@Pdj*hjZNVC2`AZnbuxZ7)0et$<1s+T4o0$bwxNyy+RV`+-WTEMtma>mB&TN_&EJ1OZt2deDVRHwE00`w_c?8JCxXJ zFSEB9M7p=`9Dwg?SRV+WC*K@E+nt+yoon8SOQ(`PxKbrCs-9Z{#3uFR&LGnTchZ)4 zs*TSL-kF!kFV@wVc1Zidw8I*Z6f#^7v7y>}L19^xAFqV$*q&;BSOV89e=<08Z!f6R(iax0_3q=~d3$ zv9P)wMN9#>8K9aoIy@D7nQ(`Sg^8|C+EWkPA$vYj7UXHZO*WmbVdSzmUmbr`PbV$b z9d{R=$8lA;V$VXp1%tBS=~bWE`6{ttR5{&~Sp&wny$MS;Rr3aS<8X@5_txUO;!(Ij z){8WoWep&i*$c|$>h#6J8jlBwy1n_)D&Lo1m9vy~ck^5P#h{@~Ea~ndMWgoS(B3eH zXB!Gav*iu(^Q7_k9%`See6dl=GX`=$3VH5yH|e-^x!Nv2;Knu|p>kTL(~Yk6zMP%F z#Ds`dnZ972U6x(1`?xpX2znPIURVWWiqa}GTmDQJ{*vzE0=r(ggL)A`yWu^ShN`Xk zE_L7g_g^NKg|a}2f*yVzaX?xZ)okh817%wuGn~@z@$-sW&PM^Rwgzo6)~4JVp6p#;p`5{|a8UI^M*=TDr>`Wnr1;kU{s!b6i9vgHx>d%j#2W^jj zGo?CR$s8OU;tCO?qbjX7&V@>}fYSs_lt=;(H71+SsFZl!ag0OVv;hRvfK<#Ek!g%J z5tt|@sCr4<=1c>GCTf%j2AV3g=IFRGG>-BLw2V?}xN!=D&t(cI91T>a2Ble;y4z)n zOXH;^u;Ak4&E*I6a3yK*;y++ksM00VGM?^8e#VAlOx}3?8rBY=#*9<-1rsg~#~jq3 z2`hDmn)@j{U^YQR9t4+6b&{8ZY7h7w4@c{7gywlU(cE{8VmIEXTQHbNAnDPmmC#B? zcM~83G*W0miYC!6G(Bt{&#&q<9G-GNjH2Fgr|*Z~Syl(5Cd zdxl^#n9u}&@IVL>e?LYoT5`)HC9%eXYoJOE_Za63joAIwP`inm+y?*oJ8T6;L z4qZ6B%m=RIjbRx|S(E7J-I^2y2}xY+e{N)6CQh-E*|VzfI0bds>Zt>i24CaCQ(6vI z)8K^5&>*~m9dRapxC}*+QaC)PI1Gk{X(?ZK2V!D6h~($zqoSgYO-@Q71u#>dU0hfM zV?z7^mTalOO_@JGoG&5%Vm33Ss|o}J1QZ3*mX`FANR+{tf4=G6%gV_S5)gbUDhk7V zrX?x;_oH9mT(SPs@{b2oGQWTLXFD}~tbcTdL@BNR|9J;m5=fNgpVs?gxkW`|{#wN0CT{ST!!-^m& zkB#bI8xa0I{2xQYe!KJkckqA99;6~u#sTS|(f>F=tu`3hqjaQ&LHQb&j2efis7l#e zslP!WyaX~fh6!`!(vyGrs|oBI%<)PkOHf8j3m^8!-z|}x&Woq5&n_<$frZ7z5irzJ zfTs*@P>PPfxNr*%4HfXc;i-Lx_8YEXkW(%yqc%1*B@L@*0`nYWE)K+$)T0X)SHt~Q zS2wY=r1W0^SWjOPshfZ?G7#3aI;?At5X@&XYM7?~z`yVkEu?_M&|i`pktEVT=5`EL z{YSH35B@K}^Z)lL%^$m#$xTg74Wps3v;HUb(8_`N?!ijs?CTo(AyIalJCCBG zqpO!p!xAR^3npf-t^_Q>*#-IeK?KxV3b3tZNzP0d_fk}|pf>?ZNQz^4Nmz!D~F$p4?G%D=$BiE>r|XoQa; zdd8qC<{vmU*Tc)d)UWSeY9vCQ9VKGfKI-;#I<;Y&p@2y+g<2a^?;Zt{FjC-MNw zdTa(%I2k^=h~BmS?8dxuyAZuX^f_O_UA}K6H-9SxnOS-0yP9aZZMZI4?pwN${hRD| zXJBK&NH+^w&PEm3KlAaF-!FDV?itG4Qj778c51^rg0^Er(Vl}zUM`3#FS^JvM@BY4 zeqdM4`F$VcBm(RMqK5W*p7sJB&d|x0{5~Xk>Z=@FBhJ{(^=@Msq>fMC&NbMh(98Dz zgS9!l`-Qy;&L8#tAE7=cDBg|h`2HWso1Ey9`#ld=MBFQWK0X-wC93+lQITJRnzn6X znl+}-S`bm^A?eW z6=V49q(bhIAjfYHZ3hwQYm}Q$rza0@+fhEb2bZnefNPx>^YH4#zK7h?%Wh}Xr|UYT zs|q$mofFqN+ctwdnIXM9o|9HzX-3I>0{uPiX!u!nOu?Za_YN`6Mp^?F?R6Biq&B8M zA7fpJ9WxNS9TfPNc19Dwt+5#Jq2rN3FqrF|JzQ)bhr9PVuF2VK7g`4YIBlYcJsIvr zoWI(5ilEL7#49)3-+}*zr>m#t8SbQ?{7a2?ntMw1 zTB8|BHUDw&GBP$cyA?26XLH+*DBAwbYO(>lx3`yJD8agRFhkATSfuT|r%30x^%6Lg zsq!opIbgK-RJ+;&sv<y#ZnJX+o2sw8ui>0$}#{lW& z5u)L8|7st<6OHm`*E6fuqk=Z*d>5kUNkq{XUUw(!OIVm|yWb5So0)|w4!zqWod3nP z_uR=AodLt`mc8)6_uID4l&&lmUCyu+fzcWaNFVn|$#uZb$uH{FKTq364u6~?QI{od zm7rE0w6a%uJ3c$D$dGs4BLMF$^klEE59?Qcn6H6y)|iINJ+3|eG`%dPb_EwkQI|Gd zie{6g_pixJT17gp6<3^&LvA!KqHDQpg(`uQkgD5891^?MP=BXe9}^@Wp|HB8uKYf_ zle)!2GiPQJ?>ht0o2H!A?Qz*cSNpwawr5(K?Nid}1p3yK+q(xtyL{s{7rM6yI_?nYoM+x(^EJxWd(^)nyLtL^ z(ksUCi3v%VB%1q+c}d*gW?PgNfOK%0BAV4d6}QCB@9XjSm|z%QO%;79yTWX{S+D2H zWsJLZ_^Hu$ZFs$vb0^YvD`r8iney8g05Sq^1zG=gw{J-@NYmDI|D^4V zxvf(O|BS=UmeK#r`w?R|N1%VPJgZ%XJ~m&h*2{i(lXE;fgm*e= zv)HgS$L-%m|F8qFzjc2EGCMGlRR zyDiW~Zgd{5T8pm8AJz|#vOUY5a|IoG#2Mb6i~Wq^E;`n*p+CCxe|Y3)53^hKv-SCN?A466zAI>BmHXg4i*#MH} z<843r{j7Omla6qA*r0B^b80;rQ}=s7H2NVTwC(x$*c%4n&Qa)J@>{RjeNSmRXimwcCtE4z)_W5SDsA>- zlEI3oinr2$a+{tgm1oT;Vo{HjWx2T2ad7d;j_OTRWs-aKo%7(u6QHD*w}iN>iuE0! z2>~>mK}2Bt2ZSKBSxyI8YD<;ujg5%Ou`xwMh(Jb;u($%IA^|PXK&>ym=feP$K)5BUFWY+UdqxUpV*y2PV7V1g$9d%0f(NiS+dB=d zOJvEV?;Tw|Za0C|7HD9f8H&y4u(7Syx=nQO-IemA3{7v~&S%Y~+RtW>G5VQA4*8bq z8`oin&?v)O1fgsZ>092z6@345!z#0Pi2vZhP6of|cwoA>SMijalb0F6`%PU!Az)^} zCY_*eI&w3^1i+XJAg25XNJ9wb03q+|6)I>*v42FHf;X_gFe|L%dp$cE9KjokILakF zYK|aqw83E+YqtvagK{YQXaG9ngTerl&Y?-%?A%=a9X0gZVwGXB51R%iNP2M^AMcS75_cW1-(Qp$qA0vNw>nG!_3FCtv@e z8ujWZYQI??>SzfgcsHS}xWWa(0Sk8<`uYi-`d{^dG~O_(=ejmOfqoF5%OMs&Kz|j$hxh^@H%!+v= zg5X!>Gd559wljx=yj<97fhI2RKPdE>dG+#5WOJ|A60r|}lDI)y?iQD`R}MK#?WB*p zkE1)t@rutVaf6@aGEhk9&`~(UE)YPO)SO4k5nLLn++AJNJp3+Wm^R;(3`Y?k@6{iA z#t38f{9zufSMU#O`L_cf9|OpntoV{Y{6~!H7W0P5M-71{Lz2s`?ik(=t0lJnH|GZlUjh!+^+OH;4q7uqdeyo zMmOQv?kSnmtx{*u+d|gNXO@>vg33mNQ^iVIf+6LOf$aCkeT~p~C?j{O*{jUixxr{g z-=j{y&;>7$pevI`k&3yAl$if@#A7Eu|AD2;#Iv{GqK~iERh`^~ff8kpG`F$jYa|ur zjkzl;xj%#(9pZ*Pt-`r2nZDJteQqLMfU=ldm*tQ)WgCCq$P;#pEXo$_beyHybU9JJ z^}GYsn6b%2vn_YaCFoR}Vq$r+Rl?rSM&at#`Yku1k)n08tii5 zH?esB=9l`*&>dKCc*R(^f1@FK8kV!N)pW2CYC{*VN^Uk(=<4Hpuj988BsS8?Nct^l z_7u*B{zS{LHEem%TZgkG8NbdytIY#eB~A>8&z-));4ov*dI!z1aK1CP;K{T&h4=+z zk(OLf!}9cY6S&52mn?&KydT*`FD-LI-64Z}DZtXU%gH^#>CrYl;UFs6uhTD*Yi{Rj zp%fbFn_P!8i#qrHc`Zl09fSyh7L*v^OVL!KV+jqnZjq72lrx9(9VPJQ} zgEYt}HwI(~V(*_4Puu*pci^d{;siSdIo zr*7HFK6824vh~c$f@gUou$2icdiDhTS-#oZn*Sd__{W90lMK9|W$8*#+ti>bhEGU^ z-vh?=TDt(?Z5jsAPFQD^9#i%>Yg|{HD$04va_aOW$i0pG{K?PQjG- zS*{{t7pI5N%|;v~X}VE^!I9b0brcvoBDoqjT*ZepW#(M}G~67@xyQ}9%W5?)}Mq*#yt9J`SfjG{veJYWOG|CUy+DUXOi% z1GoVvG47v{-*sH`AdcEt?5p~OSMOI`FNlz@mvlw+M$9@+#@?wm-`2rTv2JG7s~UGf z{c^x0mFB0~fu0RoAL$RLBrER%pDHZltV14LwoKtzTxCJ;s;17S)a zK)yrY*M0B0@BQ(ux8C~RUF*A>wZchqs&>`hyLMIWUrue{F!)fGMOQ?^IV~6AVDVjN z4R@(0`zXA-(ItNQSo1Yu+gAtgKfQBxSMp@SwTn@4Uj%`eop{7que%Kni)q;EWb9I$ zSh=6mDH-A8W{qE7c&l6o!h(LLw;~b_Y6)H54}=?grz~Eb6IU?U54iN}?WySk$M*n1 zy*cl$w+XE>m-p{JCSVX%la%nQcl)Gx_}K$j#=NzZ%-Sc@e>#4^`RCngZ4u}0J4FH9 zfqNy-dRM0(IH~0QTwL+&)yseP-85C#z`%51hZqnL00gm4Y!6q~D}ZS|PU`YoWk28J zW|B2G!sDKH2s)kSwkQ~wEqDd-Q%U`hsK1nZ%O-=4WpV_ZyQmg6#);txFF5E=aYG1{;Zb<+(erfAUnB*p$=s ziyCJ)8ee`hG;j;%AjY`)+b)`lhF(CV^lHZ4D9eKn+|$0iy?)SD#ZTbZJtP1G9sCmI zT5EAig5Ce}puqc>M>L{VbH|NvDQj)?EQ9$rd#r{yFy`9TZ=EvGcXiCBW8#XOKD7N= z-A{lJbxeUJWnI|AhD}|(kAMzG{@v~KCX=taORw5cA14)BchH%8qvV`Ani-GP(dv-% zOMCh>eYqz!SgLo>Ju4Pz`|J}Q=F0hdO2g!j3Z9#|BC!9Ribr9m616y>(PnW)cWeh*#mQPNc> z_j}>lrPqwm;Iq5mgvcBe@|(7h#*IKjEB9Wub!`Y6Sor3=-!jeBdpuB{ByW55Esc>h2+52UCZ>#3}jB$_lw=;a;N(?XDVM@HXwDk8X%7x|~Y@}bP}07_}#x#J6}8PKY(zx{&LW1l{VsE(N)FHhIixr{C}Ox+uKcpI`xH3lH9M)4;%tWF)@?cMd|@Jv^R z`hoKBuvqoYF)$%i4_8iv1s&vpujK(>~xT*S?j<5P`E_xrcKlHi+Wq0FdktNKUQoZQ0NF)%KRY3Ez2~lF}v9;WyL%qsI2l_ul(p#H|Ok*Dq*$Vbf=IThgr6_%&+Xcy4SE8Uh}3_ z#?0!K@YQoCU5){mU@NdFkP!wr0NCKk6K3tU;!Zy|U%MEdeDX+&20{d)Haqn+{oyhw z;B`wS<6c9TC({}lub^a>8j<+)%ZcjK*LOQfT)Y3%)z*j`Z}KHy9z1yh05ioE0F19i zMozcki-}T&^>rV#dIth4^r1Tbu%>$aHXQom^4st@@nZl;3I_1P`)a@vL{j2--AR{r zi&tQ2U_x)aX!lFvg@-c2!3u|-eR0e8nj1VyPdOVVJ5zbU&sQrk;p9@zHq;DA{g=za zlL@UCh2stjXeGYg|Lmf~zRP1d=?CurYAd3U&O_0QY5>dw~fkkEgC$Hf3tt#DsiWgFQ1volY)TPm?7KD_v- zPj)TCRE%n&!OyW`UJE}8-Kmu!CwqQ5Q?7l3KZtoQ+wvjtMBlGFwc^^ZPH(S*B-&1% z^;U-y z1&2s)BWgD~T_ED0KVNHbk%=vQHg3MZR{_iGi~kr%7W4LyO@4OG*+(`y-7==fOjS71 z(As9$6){yKv8wwyI9mPF0tHkX2aeTm|LkID)e9wNHiSea#9d!X_OY8kiDc+~`SR^d zbUb3LzruLd;Pg*@1{_>|RPdfpgL*lr7aZYwB03q4m^cdTnwzajh z>oM#ta_(n=ndbaZ__P{6wPIi1%a-SwH7;N=1l6eU`?Z!{*YhHMZ!(hi35wh-Jb&KH z{*sS#sCdDrii+-$JL6Byfh1@Jq}x?f%!>UUTiU~2pTM}O^Sl_bv*|~wJO_XM{2X|4 zf@-?Q=;1QauC4P_QEvJ1bB`XY>mZqvV{H<`-+G~GCr%}uDaVH@9BTwjcxd1T8m*%~ zH*0%~sox1%Wsa>0f>Q$@D{?$^+ivQwdBPvHEr|t1v@}$VAZ^rb>K2YES+76PMMe2E zSug`RDu$VZU8|!!88v|^?qLJ+joZamH%YCcmG01sScYKTYDN4=$$CJH$NCfHTNEe48DMbI^yd=DN($1a1l?Q>^eHylMcbWHY# zn;e-MpeO;qQF2{x0e{#7C7*Wgd*wFkI#olgo* z>5)P6{ZUcoKQcSZ{6}jRCOu>J?>9a58Q7a!ClP0zmPf8<4p%2KNLiCP&1#3#Y`VT` z#pf$DqJCB~T#qe54tQFOiY$FYu+T*jT?Ge>qhJPMYTW>v>%EB7oX>O9#V7Yd}XED3p^rH_hs5h zqlAzSZ82Wjh`dGcO^GlmKjmao!%6 zQXdE1FA@u+?fV7E)VZnpal$QR>5dOCa1Y7@KHOs@qi8NG`~IAxD2~sk*ITAh!W_rQ zOIz>7p49IE=kLDwwDyIhfvLj%4mXz!Tldwr`1#EKY0@;mc7{;Dg4Y`8@yc<^XTVw(Iyg3mlff91Wyz)Xt< zvxY&~nBnLyhGA*h@8;`x(FFE*g=%hhO!rquhmCg25$uEHlM+1&3K(-kMf~kpoim#P zh`MzsM6`GaSLWnPwdcvN{jR^7$`FA;{R!okR3B2gdH{X)6Rd6}xr;8h*dz@qIbX{l zw&*W6e}Iho%sz=PpnpW1!O{1T^gKMzBPV(wHAr}Oau#|d&Va8C>C(hbYB_8T*{5aI zO${T0$m`>1=z5d1s%I95fv96JII-5i5usCvTBrHe__{@6(B)+TRKF^u$( z*r2wMutRqIWxHakJ(%TySX%dYYsp;cGn%%;ve0##i@2k;tevphg$u)$l$G{*gig@% z%TXiNdOgY>=}ZdA#Qq3h%dpW4L=%W~-D{xALKuqd#wK#2NbR)f{V&?cm(XSO5{0UmAhx=2m?0L2Jgb zgZjAfr&$A(Dh2}-NJis00qY&GVMzH8Iv(M_VneM!O~Hsm?xx>a%A z#&b~3QVM@6Rf>(EFnrzzE=`#u^|?{+W7K4*pXr?-4E0pQr{qi>qTg`Fimwq1+iWg0 zZOdH{VPROjReC`8rVvtQ%_wixZy{pBkjjrCV;71x9WWylEB=>vC(=!zh=+gOD=1#TCZU=qGoQ6Ey78i?I0|hgtYt2sDV8Z-CTdgSyR0*!rz%)%R3P=e)k| zRQ-g*g8A^gbnSWrNK;>LtV3PT*_D@PN%0$AZNFk4xDYJ%+H9#^Flv#B^&sjogZDu)GSJ`m!9W#0({{ym=uWm$ z-6VG|r(zz2Ug7C4*5W|R?_s)Q<7j=aCE`bQ?39d9vPL=wHt0RQVi{QF`)x@bdY9ar z*HcWORgfu_W&9Vbr0^Rr!nx5hRs-nb8Hd-al;QX30V@JF<~~SuwW0EWk#dK?p}K0V z4&xW_ygY`zeT9o-Pn=SnlP_{`ZV{NQPZzCmb0Hp{o=12@Vh;M&2y4qZOhh9va0%KY zEzNKkbiphXgDa>LB)#dzjRf|iHZi@b2P<3Z{@gl2MWLX@q1X|7!o=4fFuOZx>~HO! zxeO7TX;qfPay@#+7Qgj!a|wTSW67QJc``S)m?s3cEX>q|vl2*AbRBkhi~MKO$Z~6D zw6WgCGIXY>P&PwGS>IG*~l_8`QO4I??YyvCF<_R*K`TbqYmghW+82d#Vr+2Ze~K%GdL{yoVkE2}{l zPeyu{HNmI9Wu`JCaO-+(rtaN`jlodct#X7vFLypOy{We>EDJ;2mGMT^x44;w;b}4WBjL6_ib&Y0X$}Rk|=~Z7lsx)N?*I~0e|ZRSaG#eOfATp!st}h z(F32>4B&^z`;S#sR~Sg^2Xs_?FoLcvT;JED)0|9NpX?POBdCi%-dmoQ^B*bt;312$ zx?^8i;?vIq*)M*p3sJ!M%FfROK8APHlAzYN1Z02E>CA zby}@Z^U)*1yTn|LJkF5_YF@V|M}JAGL5SrylL4vW^$*Z!YIqJ4Ma_lzFSOeGyg<`4 zvoyd2_xt=;ool?Bg)ok_ot*~w!lMV>6|uK?WBoVHm%fA-ZrY4y*3TQwc?U}iPmwZ-LmJ+b#wJzpqq=5$xo#=wKa zGSPDO5rifwgQ5!d#5v%g5GI!5_1?Ps5vZ>K>wKcLV9Nr1zjR1`D z@`%JP8OQUlFMvD!h%3fqclQzGdIm^apF85Gt3kX8WGO9&HL9PjEBACJJ~)47)i`Q@ z4>)hLv3JDN_yuG{n;(o)v-4)tt)$iMywD_-iBfF)YL}{PgJ%aGm3m}tgh&}hl5T2Y z8ApW9UT`DRTVXd+UmuBLc@7@pPkU6ASGvNfj>bJ(lvY}HkRighth%)R%$R`9*on_xB!|_gRbu!PT`!A zDJ~v994pdr45Kd6kT7a&>`AGijf$yN`LSyN9amR&F?5#XwN~>F&Jb9Ahz)5f`2N~h zt$fgigwRH_16?%G8Agq2Vu#Z71Eh;2dUx!QzjgiUC5wg*?Ai=-qnp5?|DJU=C4=e#OhU$4 zHS@&k7e%&pV_%!*;bPRvX$PnN5E%4FEn%aMwK0fY?PP!2YymAVPsFAkiQp1Hg@;oU z*aPo+Sk^Y0V+(U-sUKXyjj0Np7Nj1?mlG1BU(he_ zh~vc$2QGc2Cd4_w?;RF_qd-eW{9%yJnQ*S`dC;Y0&r&MO%O|h{gQZ~kwAZf>mHrq43?Vvs1a+@C>qvCo zjhoR?dyq`e!TK?!W)x0lPs5o-=ZOptox>&NU?hh*K&Rh5&s7ou@4KbDCFr8QnZepv zxKeH!5AjR4~ z)0I;pVW(BQxDoZs#Maik=P73?gKT{|ePB=~?RDmAb%w`qGv=Y2+wg-F4e-3>C&pZ# zY-LONY7zYqA(>g^u+$9U59vVi3Qx1n;z~%R4T|$_(lW89izxGM{H^>`Gq-pP%QBYg zL34;L`p0hUhIt5ySjeS;bSKt1gsI*SsG}5K%DX`=BhXr8gHqhCbSr;@0OV(-x9I8l)F&FV)(Qx9 za|I530VNJ861%hqaFNFWuhqe-q03U&v%%#4sLftG2%-q^KAl}%Tp`auS_YVW)SotA zx3!|bT#;Nyi|ezp8ODSWHA@h&-P0%DlPGL+9EsnGbMS9?mzJi05rD&0nRaDJ=EpP? zYrT{qJClyDoQAlv7k>XyF|TGu~kT>5FYpdPQk;|ePCdqvgFJRXu#U%M~~RZ;O3j+ znq@!|LhAloak=oYxGcn(3m3|eORsP0d(AKV13jC`6SGG*&A&(3K(}71*Dbs%)&uFf5%k{Wv$(sx;VRj|xRMNCjzo$KlF?5p$!!$+oxoW_pm_<54 zbG|~LyYPU?ulQm=kHCT6!X*lydk{rDA}h;l+0$bVt}M=4T}7jzi?df&ySV&;)=Zwf zH|s-e%?b+3>f?-Fm^D!TYbOQn_h8Tcv{nv0Pery}s+i`tAvi z)fyMQHK$&w`-o&u)(fKkn4e!gx*`j%<)4j}*PU8n1vRX%@Cx^%0=eB`8moG9lMk#L zNduI}TMgvEiDvv^JRURL*I_v-$68v^4;UztnCO!-E`MLMSxBh#aW9^i)_crme)QLO zn|wcwQmx`qv=D@q*EZ<)!2V4j%eKz72JP;C@8QGdQ8y5pH+q62Z@XZ=gtvrp$9}i8 z-^@dd1bnGa^?k3;{Z_FSZ?An8v8l*LjOJVs;>ci^EBW&TJKTBSmL?y_Y5jGnQ@K_= zP*7ug3w61Zz}=?=`#4m+@d4H4uC+QDbQ=j7UL`HDtwf}=MFh?)0WV3dg- zvi&NRTwKi=i|WV<9P23n+xu~~BnTsP`c5HE5qe^Z;QJ#Ns-c$U>&(t~K{3io+;Bo5 zz1b-;VM-PUdi21vL5YQ(9U7^%*DSruGta7KFbJD&^I{_nE3lx2A4kPD#?6!7Z+K65 zB*O^fvhyHrDWh|^$o})(bk{3&KHGV9KaJCKrs6K9im{2p+(Ryvt+-`zxqv}3V-A;A z_>X=d>pl8G3t|p}s}|bOo;Q#BQtRv2MfA@e`6!3|)&?U5OeB)!c=1LfTKpuOp4QA{ z+r%`am^+qRki^=I6jWr}l>5=VV1dgYSXdq@gV$}}fkw)BP`y^Z|Axwo z)4uxt5;LhY`evGJng&Dza6m&N1E0WzxygD19Afl*Eh}Gmtb-@T0XeDYZ*csLIjLUJXS8Mul`c@7aN%)v764lW`G+Qu3JY zr5?yO|ga`vFTLN$er>TZmxVS=jDgOqdXSW(;t}` z?WSg54TrP5%7<~CM>VHs>!7P3O?UprK$FjmEue35cS|6n05+(vKVVH@bTua^D#Z+3 z0EZrRBNq?b-vJW$VtQzNj^Ux(qB>_#Kc?!SCS@^bh$$SWNHb*-QB8 zJYZ{-QW5aDEu&*TNfhLJZ{-a-rfBH1f78?{%aJ>2-ivKBBlxUN$0q10A5RB|a!xB? zPy3!{;616IU2AyZ6GF(eBc&C{9tPeghKeEvhZr3lfldv*TT*kVy0(FUNrPWNJbK+vdPAbGku4^XDf zaL7AFG47&emJhXEO_7au(yg0sPPAE-(|+jdpIluHz0Fbo?87K~VOcbosWaK81p0Kw zmrSSdpx%om?^0Eh)$Jdx4XNgiw$v=plH~@W_njF@6$}H8y#50u1{Z{fk5+qit z1GwSc83k&#HT3DBfw>S8cP?Iucr&&U(fQzES3Mf^_%5=8QE_2@UbeDQsG~)mP1{G( zt~uXv#tj2+Z!UaMG$Nz5HES+>hVg9 z9a^l4H6A}rF8b`Z7HOliPdlbRH8UgkVFpzI~w`$#&Jqp5Dcu)N8o+!%!kL zPS@&cWoPHj7cX8I=EgaMbq5OwH7n3?2wi!e)dYJ07Tw~k@3#Ghze3oubl6%<-qaqc z@$c#CiUD%-M27zdaYu2brQ_3`X}53Rjxtmvh=%<64e&qZ@YTOlW8*P}ztbj19f5f9XHo9v1yuza`2i^t`ez9R3_;|tYlZ)A z8~-1Q`MimyS&d<*`Pn7L>g(5KZe+Ekr6r)I4egsil-LNkB9cfz z<=}t3l?lY9080cK{CWE>t>=z^nLiuR^S?Ek{{JwazYOAk-W~m0fPW6i|E53xw*dbw zz-@o}zqg|Jw*~&oSzsfA>Zg0?ujL^@5r`7Q(2ri83iCaB@zWeei!r9Lr32e_8OIok`|iDe>@JT7HTQg$5Cflr!6Dn(dwM zo_*OX>VV1irJNK3TqX;98lmX1TUlIv@vWKeNxB-i!8BMw9p8~n+mG+Ckp(Ya*J}bP ziXW?xQ&Ev}P-f)h91v?tYiH&x=9!svWIy$8?KlLSGzzr($p+p5vlg0#-AWDj@+#EhoIJwck9YSFTH_(xos$~Req3W60% zn#dNrY436r#+;w;2#zfNAnYz7CK&kyVCTI%PJXreb8EE2#qe6`wD+Ggl7Z5wDP? zA}W=D+<8g-TXxcp;2mpk0iDyg+KEDJlQ(j^@C5JKH)X1@OHENVtsseG;%_eR+-cU4 zEwI<%NsUE*K2X1_bfUUsWMqSkvWaN0s!DVAp~StC!Y{8JyBxafEp5U%Uo%o^>e4a$ zx0l1LO)LN-P&UcjW%yJ6%Vgo;U74YYtHC0LcU=;U)Kz9Y^%mA_V0sJhZdFPQmv&7~ zJubgDaYyjzl!@0ph<(moL*bV&z+e}ncLeV;Ql`7u#i)c zV9@p}7zyreA{$01TI`xUanW7{{@hQQvdcm=I5L;^XwP2BNX3g6Psm3Caxej;-rd4dD%4@SnACB!loa@ipl~D0^n<1fkf1GJS)OLHW*_=1 zFebCTE%N0A!DhgivH+~gBEROh8sV#s1r07vh3AW3k{CD?wC>+HIP)y$R59Ih7WscfPV1 zAc-6klkC5ljIyLdeqbTZBv{$R{|`~-YpTS^oHX$-0J;_~9ZGxm=sBPlXZV|N#mD(i zOm=FiNbZ$HI>Yl#v{bfP{2_XP#fBjTNOWcKhx7rhY5|v`p|tDo?9#LUs2%tqCK3|7 zBQbeE1Q-o_Ei6=kTO_x(a0E7i&lTct%*x_9s1UDdU_tE;OTKZRcc==F3U<@2X5Ta(Ifez2157~7WxyUALEIgP5`P%wq8=c+r``o z#v?8X>Zq$j3?CS1>fG1(vr(6rpL%$>{PD1>n~&E6tvi>^%q=dHuK||`?V$uj0E(wJ z-X6CN4e$R^=ih$+tpD#9^u(WS2Sz3SsB8N4Zp!z}y{YFBm6F~NkIKIn`_BQY=eFK9 zgj-lbw6pQ>@*xo7IRO^%_wo1xeocUxy$FpW!0-Qn?f(t_@(29v-{7u4%RGFbPAD@) zfaz@C{mLJ4>PfV#U! zfS04agU{vL1S;8H)^W3u6u&HUT}B!J{-F84z4U16Pg)YOQv64r2P*(55gJCE{~vkJ z2%J(+pc@C}f8+@k0RYGh0P0?PJoS3|Z}kzbL}(IkJ6V}O(?rzu06?0G#~+yj06Br5 zE++B#^Ljk~;v=E{836c(#(x7CD2T6;D-#p(1D6u}U9#Wz|!`~_)bPb_{hYRfS%2BTRVFP zM<;I|Uq63nz>Ba~;SrIqqoR|N-=?Idy-Ux?%P%PWSoEp5q^i26wywURv8ki8tGlQ7 zd*6?d(XsJ~$*Jj?WyH$r+WN-k);8+!==eAK1ao@!2VO(~@xP%(xc(bv|AH3-0k2CW zB*Y}-f8a%Q$)6C38AwR4NRu(%HX?uO$;2-cLcx3|@k3=hrGV@MB+E0eVJcQZxn&{L zAE^C_+5Z}1q5mb!{*Ksx;57@V1DE~`#Fs7+lM)jXlai4V0vQF_AAy33;?F?!?}6se zK=(%g{XO6bl@Jl?AR!?kC;Zb>Q&Q9azYh3$!rF$%^Z^aYrSFV>b)_{M6*2zQR3yrN&4ja3hJ?NCWs^jV!-UcG<+# z&&@E5cW+#&<(t%RWI3b_@;#@(1AP-1wu?P2JfL?XpW+^B9Z{ot<1A+)Equ*U+{SJv z#LCulfe8;JP~cLgCi`{0w^Q3Sm>FSN{%Uj)9)T^-?nY*F!ZOC|%c(g3|9Za$|VL7|`R#CXP zYptGu?qRHH(hr5&l@B888mT(A4BtHxd+tQKH?b4XhV_d*gGh);Jeu7KJhsk}EE-1Q zG%LN=OZG@VX6D?+a$@pPIm=q@@M=t(QRA4e1c~*%FIIE+ICZ(a?!7LwNoZ2M^296r zUQ&VUsp85+b^iQzwm3345jL(AT$75z+QT3Wc?PYWQfmO{|=~#PtuLHGTEn3T%7KuXW zJt=C3+|oJS>+yQ}c)&S|N?nDXaL zZ-hiqU7cl4^<2Tr0M+M}QHi;$S?`1_Rmj-6O$S-dgqc`4$HowFd6X2$^am=r9U(rm zT^F2_Rzf9qkT9i%eu4z;5hFm%&Tw{L>NtURUiQ!*l#&wbmi|Eo4;w85Z`8MVNlx7C z^nBBOle5D1{1R3V#n-hMlTG&PcbtG`6-L*ezO`j?f98j5(R}*zx@O5l?dTl4n87!d zRwd9rMA>Att77HKXO$T}iJ`E~iUvD85DLe-Mj~A6JB!9z!04M_DY3e2QQ#T_tiVA* zZ_Tam&C+hyK5tSmBu9V}#1BeM!AJvx_?Q(u0D|(}8%eCJ3P;@CCAVFewo~|TNW5D`juv=x* z&}!YriL%;R+Pi)E8b#CW-Q8W2QjFK*!OHyRHn#54eyJOj&z{VeD@J5h*ep|tZSp06 ztNr8K?5A76ic#*zWpE9YRrS21MNgzZdR`gDe@%$t-UFF??pBMUeAnAmR*u^`;a}Rm zIp8R<(wLCICHQMqrEwLJoM=F?GlvMKgCzSv7q&cGamjH(7$sbd5UWct2t}h6Z{Jgu(W6_8$=BSH>t)f_YXXQ+uwWhb7c0hr6$=H zL;HiIdUo!Wj0$&NQ^~Fi>&HKC4MmAojJW!vEio*p%%%8ZR03M78O7|<=!LoYiX{GcyJ%=Uksa?dX}!jdv$2~A+ts_l zX2XyIzM2Fq(_%Lspmm4&BN8Gw%hM4Cw3vHnl1%NowsZH_KIhIIq%CQ6!yP59cIu)u zW*=zliK3AUHwm8_4C{deg~~6oLAHZ=#%e*7)9RmgGRSg@W3rs@)uyWqDQwSaGCVS5 z6?*>UA*Tt2aPtLGZ&8*zHIdeR z9NAGYku2;Vpm=pxA+MVI{_Pn*)*Iy2<9J{x6%W+dB_e1=I`>b{ody`tJO>O89z#5IdmZAMRTfU#+9ed8o?34zaW)(;8ier@DCgvrj+gV zo{AB|)w_#v>j<)$5enJ#o}t*c(yM-*7<2WNKoa!H3Xk;ZhK2E`g7VmHV^6=Q5Ub!0 z@S$%jDIN$k!ULm01F)F?KJVHh$ycfmsNOMLHB1B#@G@M8RhHlEfTNcAMw&TK&E0|? z$o_WKAG$koAfKg?E+Kv;31N3Pz)XuV?O8Fq{WIXkBhmylN<@nRj?;Z<N;p^U995T=HFvvN*tA zyMUZO+aAw7XS8yO7c}Muf7Kd%c$$d^Vw50hmkR7ZZGmgFI^x_PRu9+goS!teW!!{5 zIu6nolDXH;EZgVwDK|;x=biY1kbyu~SjuwSYX!wt{*m3$6$5msfiKCkp>yE{a80U* zO4bg&*t)&j(ru1G=N*gx%cQ7QC)7y-?CFww0*-iOzm^6g(nJ4v>%eVT$k*_PnRYZzv>Y98gi>7(X7#$^VUaZ4d6-xD-S_B(dw>w6Gic9*zGoG|O58X$PZXv|)K zc>uB!9EHRKy@Wwi{(xS!OeiO!W}egAQK;iFS~-f z_6n^xBQB^f5{zQLH^tDHGHNBqt_Wm53|naFSDR?PqK;}h>QogOFWZ+Z&0^I(4-~WX z6HwF@lu%EyP+Pd9>*wDb94z)taUbIa#T6Uk z%&EUT-h1$kDfYcxtJi3-5N2NW64c!?%}6(g85(IErm1`SA?|^aWGz_@8!gRK0pS}x zOSyme<+2stq{BJx|l4=t~vQ*9Lxx zLRx@ZGtt)ycVzAq>W~k-_S}4K2$xo4-OPPYlYtTW7JTI*AEyY_HN>Q3wbqW>KO&rv_4s;S<_g`g{4wY$Er_JRHG=f~kOrjGeu@YqqP)iM@98P!~CSRyt z0*%?7Ol+siPSO;#jVpLC`sMMLM?gxCj2D@8{kdu5n2U=F(Nc383)}(fsB%BtEw9#L z4?Cg5Xt>>e|0&(QG@&mgMM}A#FQOH1VbteFQn4`OWmnFV_Q@vUv=wLFm^e-x0t(ra zvgb%zrjcIDe{?zA4h$xDATedh#Z8?}G}|s`>4PGaL#j}Lb2E-fy+Lkxz@aT9{qE)K zdX-6JTM!3GCRMvmE~sI%B1%$KwJ6^B+obH&b}Ppi0#&ui86IAHc&%_w6!yKG`)&sU z^v+iEXX4ls&QXjx>IC7}?28iK=&`152$Z6DJXc`kc&P5}lXSC0fpBh1ySBu86D$GyLv%&b%q6O9YpwsoZN)GXy6MCx=MtcN8oslAKgXkF^TJ zbYj|hCB}Sn$PAXV3=Dql*YYT9?+c9CBt49(ySrv%SY5g&EW@L%A`3VWZ#s#sxkt3# znpht@kQ=bHK=J06XYuOp+){iz>QaM|QgHcMH{|Nd{Hl~boF_~aR%K5Tuox12!^yP- zcB?6~*BKF~)uVXTxy&J0qXs@LqsOH9{YspD*!OqhmiG2XWClh}FID&QS_IDB5iAM4 zdSabtJ*93lbQK@BC+VjwXG7Lvu9iLE5X?Ff_}(w;MqNoxOryE7#w&oyLP*@~wU%g@ z#NI|&QFmG>l#=G@6>XU?Ii^nF0lRt$q3-VPnq>OXmdt2b7xuy7Xy9Bq8ac!OdyHbK zkM^yg!-%wxV)CSD1ZhhKc*gwp6Fk||pVmc<$91(D(V2SDsm(Vas^lsdu(v1S4tiSZ z*$TDDp0L6h{YovL5<%w`FILA7&T#2>@9JJ-#~JqKk66C}XV;=ncMxe^1O9g>_El~z zA0i81aR%z6QN8ke-4kV=LS!N7UAXjPu=RpIX=UO*TPp8BOX2A&=Ti%170oS zuf3(K*VgtWbjWsUbj<;~q1zy$-LA$HN)$1t>;b60%4;!L;w@U?fqyIN!by(1$(z+? zHkK0x{m=#KS0jV0Q$Hl~!m00-ZUoJRT(%zH;4F%671*)nMV)lhg1QC@+n4$W9#BgN zTF&0e>Uwe2HgnHi&@O1S#yyNX@%30S@IHy?cNPn(KCV?1Q{v8UWnc?OSV?u(HL<3B za{X1)EVh*3Gs$QmerPrO!;|~jl}fr7Zz5@&w}V6sag)fb?9%0(<1yTEtJHMZcuY@7S2qOL%LOHkvLh$4hSQ!R>TAroB&`d-g!IPEk^N#f|v z-zBNCNmA0CZ9?}b#8A41c;`%P9TKIS)>T2djh?AZi^dMk5`$D|kW_mXj{!3x6Ff--be0F1InaMY>{H{Ds2f3jEL`o!+-#N_se00LtWH|H05Rcw_H%A zLpk`Hs%UagFx^Oto_s@1Hk!Mi!gYuH`#tTFDXzE5Z^}Xv!=jJeBVH_aDYXh>COdgm z#$A6C6`4CsdZ;_PU8%2iUaXC>iHo;=6TGJC$ewijiXl%f@q$CeTX_%%W~eiVRYh_6 zW&%q;{L>C+SmEBQGnik-T@x}TA&r_wXp)ZG!vH51`?r;I=FFGaDlxSP_t(~($Yec| zGJF5PiWv)w{(Q@erd`L8?}Z7;p3G6`2(_IPbE0eB6EAO_MW1SdqFxN7b+qwqH(7uE z^ca)h$snsk9<}%Rvt5;60xZDNVJSRG<23)-Po9SP*yxyLa2K2fTG3@KIIMEB{IPQ# zx=BURuc*r}acffdlm3V+)3cv3i$r*dKiww|YbBwe*qAem=IQn6b^nreb zzf00%U1ks2NNtv9;db`G_2(4tal46iF9}oTDwec((b<|5xn5n|w_y$0s`F=3y>yUj z_Gtb~EF{#-p2!}ja#$+ z{c3y;!w?A!9G%K5ib)Gx(qrlb-TIhWqFeBIF7y0(OT3vrulp{V-^PUk{FS`&)Qh4J z>A!bP#T`Yr(>|jZ!Sh-CH^*_GTiiwc3?gYR#qs%QACovqETv4~Dvcren2%{cBW{$@ zv5iq~kr)%xVeIJlVzBTc&C%xsUG+G93ZaZGg##bk|_(WB76iXcF zy)2gX$R~#9@q9<(-2LZMKZjjqG#f~YJIMqbue|_*YBgZCEAUx9H6%Pn{;TjUqKab! z8fz_^;>EZ-GaMs!PZH39Y^uH;J-sxh?3d9T6@@imF5yPRfPDvY^DMl@;;u>6$9y?1 zad*-Y_e(R6bFLR@Ej?K^BzMe@@muG!Qr5Geg8>?lbPH3DT?|yoNlu$ZC}>|A1UE$uM)#C`9LYU#6O5HT|OH zk@&MT{K*5oBoAVGZQ@t}u3lpT3imyCXNB5mq4bj8)=#+X9oRhZy7O4GN>pTX{Hq!t ze)w-XU1czZbABi9wGr5z;nUJ0D~+2XCn@Z#+^Fv>*|(gJ@j$!(%zd=%l*M;9SM0Q; zc>FhkjFGk0U*$2E9s0==6K5T-v1m^9KWh_l9%~^-!Mhg#jq1^bn*+BXKUwR=TIIbc zBsfT9cn#(S<~Vt8vfYtb7P6ntDZu!5VQ6UWb!EeRGQUpj9whyd>}IMAuECh9dW>@qiTsE0%;R+5>%pF5XSMGgc$!8*2E%;Kpg&t2eP9 z>ax|IXnJ_Q5nEc&jmgrw7UN%DiXGa5nZ{z3)X8^|@jvvwn2n@_i2^zI2Q zgBWy?J35%R^3GAmw3??j5!C7V4-gIoP}}% zQtQUfc4Xs^s2d|R(Y;(>)C6t(h+OLB`%Hg%;Ha4*bGHy53&onfO2r@=&EpdDzrk;B z?7Y6Zp8l$`rs6;-@T9J8(O}1FXn64zxb7qZZr?VYrjId44t01qTT74qPOk;i_aVdt zn?|1BKAbJ9>h-FU&FdzVRV1=)Pvrb2ZWuzXs)W@<`No%h7=kZ@Xs4h} z3&RqZlL8w!f8Red4~&#$k#RA)VUTnq4D>Dpdun9;7Mb|sx9YQONnNeVbFqBf{9?6! zGNO$J<2BUzVgzIGO%;^WxxhW?w)_a&01^te@`{Ouifgw`{179vz6QRodN` z*cVrsHn)||wEA34F-b$m$6oQ>;@SeMl$T9(#?*N~dg7u)1$=&|eW{HWV{AD2I}=1% zxYPd$6Tzc8v%ZZ$t(?Dc92V5!6nsKFUVS7g)>xw|i<@~7`5H$DW5xI*-5;YAU%6w6 z^As}6TG%9>lCE}$nXNd)m9;w>(L8walQ&7*4!I+9&2cQf4^&-y>$<_RNz z%}_)u8`9@YvuUzPq?6Xa-B@{c=3S=I7_M{m$E&#rg&nl%LbrNsxq!=i$1QOELHjvq zhM=ltD;$wuAFfG=WPqwVm@G{0Ndza?e7tMy`=vdCPuk-YM}>{d%in$@wc`zbdWE1u z427|v*a#|Hy5PY@h8CqVCte`oTOB7|@v5aoP!p|RJHK{?c+6Zl^zFyBqkCU$)vLKu zS`9B=gB>7|;A%*B0wc5@LHe<@*x|)c7v&BwlpBNf4_|hzLeKSqrnK(@RXTab=Z}<>r^>1cW>bh>~pohgoA8!?`00grgEdazSd5 zd9^uFDz*Xn&`wDT@0qOC914l}M2m`x8 z!dv;qTE!QfUf4CtKo1bSk%(E~&1_|K=lzK%V?xoclL9V16w&-2k0oAOnXdDtj2V7; zbMG4}*==K2F}jVguAbiM=tcB)Nrk+Sih}gEv20W-((eq<(8>(BWUl+r-#=u!R?SmT z$HjgvpEi-OK{9KKcKVu$XzNZOh6Z-+p+r{5BjL+@tyN>E1wwDX#to*yXD!!kr_;8fl2>> zEN_$inI>9q>9~RB6ZwJ}Kxi#(UGZ;0OKCFmb`+37NENoEPdRPO1RU*LC z@Oxt|L`w!#o61#NO+}qZ+LYZoqxnu*?cB$w6IBWm=2LD0{y$zY8Q(s?F_-C1S@!|) zJo}uWvP`hfFsu14@vK3*eAu1)x{SCsxu%W4i`6HfPY$`Dcfs!u8m@?J~}FUiKLyM zn~X59UXAS@%J}*H^S8W>#N_vr<8KYgdyCH{oo9YP6CPl~SNc@6Dt0)jp)W>`&Q)(h zcXy}uB~}aCzIx3iS-tpjkhp1xrOTe|!jYqHRlHUyZehvo&&y@aJsl9_zv7vEg|a4Y zmg30^w*!cpks5b>XiWnBrYhU$aRSSZebJi+PfZs^&1_trpDcMagFg97K;E7Q{Q?XX7y0VBwEg= zf;bfbc5XnnS)zG&MwEDUkpuU$Df>hD;trS&<_vz7Y&G&^*w-O@SZZLrtmPTAZ2*sj zlFElK4So9*N`nhw=L%cF>?m>0o|w-nekQFTRi^kEZQ@o=8|fxT^-tnDN@_2kuU_Xi z9XU26|JlU0yKxaC)e!|>zH-#IY&N@n-DWp4c>dx8eT@Iv%Ia>9=h6OFuSNc{)N~Gx z!JP&xi`>8p<~ZSjaNlApCtmj{bWru4l-Ow?Z2hqA^xb^gpfU~Ww+1uOVR-#+J37c# z9*)uRcR_XGWT(+8@N(h{CRKkIcv&xat~^;EBZR1+b;bkq@^CWDbWJmyE8Q(WB&$Yw=~cg~5?+DtyYr77sksKs0ze59)Cp2bGdPpVHn zm%O}p3CtgbabAv$&PT=7$F^RE7OLF#E|@YX+S+orMKf?($=Nv;@|)}aL{2oE0>2QvU!;`XSqz@opsT2TcPP~)xhPtma)MP>>0}f=KBqX zB%>}wjh6wxJsOR`d1j*3Y&nd7xl^`8C1ElO`Tf0yCwO8|PrFPTy zwZA|6Z9!U@G@{U!C3eD}HODoABvvNMr9I;cO`#?Em9ZtpMwVZj))eE|t51s@csVAz z>mmyXrXChE{Z7>~b6qUYngjgB6c{{)pOA?tv1}e2l83;=FvcrGk;#3mrHCCKzaMDF z@>=4HH@~bG=I(ts7VF=7Zz``VY*vg;74)vbPeq{Z!%e2iTco6G?Nb%0P! z+CZ$j;Fpq8HXT3!3T%MY2z;(zx^#=n2onVE40J^++c*-$zh5Hhj$yGNBp!ui5dO* z${liP;2;MJ_wZve5S;_+-DjUEf%&=B(vGWUEXNQ{NXoW!=DYzp=e(8^OYkd_5-W z#P@9C?caz(zR8zZL3FOy-uXbuwX| zg<9Sv2!_s`oLZ482Q>sa!hCinpxG}V`>r$ZUgM2H+1U3lGEi58>7YBhF%OZ8Ay^+D zDQc$^MK%dH@w;gg5NB_p4s)xhH=5kOv6vrf}>66{>yo zv%D_Ub-iY6kM7f(0Hae)UAur4(ZL2uz8BTDultl5y^)ayd~`5bw*H=CrpZaEx99e)c%1iARTsIjo4(veVGgzE z>}={9E{ar9zi1N}qR7zAsyx*pTiU8Ly7Y3&?bY7ucn}C}n%tQ!n!TfqQZa%$1sT<< zKRMGVm~^>C7$y2tr{x0yG`H(+y$x+~Qn;DU@{e*|U(am^am} zk9nX@?G`_^&5+_(|JZ|r+UK-NRyW{W+RB-ai9mclOj!5T0M^73$zm6Qk-fd&j5e;r zvjcDjkT~V2v#|Dd;N%41v+g|O+ja6GOi30ks!}@QSa5lHEsb8GI=UDo@@{x;A&R7` z&Vyw(oZ}S41>-Zuh@pf7-vzTnA$lmDk74cz?cWsQ<;zu^8U_5%p6BdsuA8Sh3SFTp z2trW%UYHSRqcs7xZTyI8{fdyH!(>ZH-9z0%EG}B;7FaCIiLb63yZWlG(7!#rt=W5H zr}x(jsmK<+kC-mo zYx}N5EQ~N(i0N8FM<`B|_`F0D{TqpIh!EeXm;-a;2HQwg1)U^HF})qB^=U)>b9YB9 z`mg*>+*?ra-tWrThLIynmlr#^%q3LFb1gP^6d6_SSvUflyrvQYL{qZ#qQrT~@(|Z$ zFpUF_wdxuAOg`)DTNh1AUFd|oU=VBlr@WV7{K((blT@MvF8!~>vmG=89!M@jN}bOn z;4=4}S1rzF(JYD2E=pQx1vVU9)pG^$z<1S+FYDE1NZnr(k-dDzXTyJ6Ew&tWyr_Ni zB|-cGyg@L~LCH@M%sUA&01q6@{*gYcnma43EYu+Ai~liS4*pN6&$9QT2XK>>xFbBU zSdRw~4*!(+!uim|9sM=oBnNlVM%eh!`UhC!69VfueAU*Sjp_g4`cXhr}xo=r;L>;yh@ zZ>&fx-Km(;!s*w4YdiQe=W{8H@mT&(_kzk65qG{Rft6Z6uzk!(BjoRUP-}&azjdF1 zs=K$erV+=*o5dXDgx~yK)QgA5h9rE1!U=?X#=~m#LD^FGREO9K4K#q;Xy{kba~129 zTg*g%75|Gee=)&dpU3~Hk$?|B_P;+g{wE5Sf6Y=RxL(mZ{7-k}LvtG&>#FKf4-X%P zQZN)GX+8P$I$!)}?@M4cd8DXZ3Sodvu3*LkQ`gS%z};KggCK$z7Mh^&$SYokV0p~p zsF^=hu&>)d1Rt(RjS;NgM{6gxI?9rEciL4MJ;^MQx#qhj;=dF$g@6yc&L}a0Q(@LQ z9mapjG@=F`t4dur-eN^Q8$9bP5}9#Yuoxh#o*BYKK1->w)1!c{D3{0^_Gz0%5AxAX ztGgChz^*>3t~Gn*(d)|7$8eXQohtgJHEpw))hpTttcNn3L(#?D35OA#d4in5d3s=?cnZM!I$h-lt>nnAH~NdI_q-DkAUz6|`qzf2TuvAhn)0{Aa2^$kvmmqtlz7 zboousPgV^Y>MNZzvGE!{k1#psD#62wZPYDSR)!*SXypn}rrieYFpWvFgn~NQC&Xb{ z*n%~S?Nzr-OU&E4iFT^;{&T+N8`wvD5yz{jF0?-<9!UE5`xsLY@#|;6Q>uH1H*Wu8 zeWouPl9hYOodJ{CnGG6tgJ~h=O$c4(f<8U*uP*%BVmZZskb0-2IV&_aTLKgoW*B%{j@d%K;?pe)i9wwx0$=QtD%Yprc|Z3%|G` zD`(E`F%tz=X_XT;rlwxz>s^J}Iy|o*Ju=jlz6{7XwMLaYvBNrG9%4>VQdUU!UikFU%3xfPW2+MKAk;VL0aqDEq3cb<(bZ}n z8BLM?V$DRNzSAi86hC^H!7_O-Xr_MofS^%Nh_(hH6C&M$xuKfh)o*=WJrOZ)t*?LA z?di=r@THrxb*Z`jI+dj}4Ha;anb$zr%#a^x6@)T%;=sc!Nu6B^3MaoxFODr=fhsF| z)&}WrMsK?nI@r+VGFr=T49{J?pask-K$tYtdHW*O?u<)qz3}SgB@|x;6vk@wsT?vj}@}9pj0p0H^eqeLf^|ip45F5G?b2*8z~2o z?HFOo6QFn?q7P$(2h_gW;INi>;824PISr`XWa;GD!z-5iaNEl@R;(YJ6-atgXWRygSR>~q( zr6a4=Tx>Zob<2CE?OGQqhxtKR@r|B7g}onp&$H{2tE=^gEN>;eD(ebZyK&T~Ing#9 zDEPa2c)G}Ss0O}V70iwYeo2`TRHX#Q!6s^96%0E0&PaFoP7nkIXOh?k2oyX>Td|Gf z9Wm|VXR6rLEhAURe5Da<)7P7!2>*PJ^*L{uWLKqjn#krw*_D>h^QfM8q@|`NS-kU) z3Ld|v$@6@!;qeXMQ~l+5O>g}ocF+3)L!z29I}R7`j#S(FD{^#em=NMzKUt zpg`x1U+2^~E|KMgi%Y)~&QXLRt&bVr`5Y^a2Xd_|3945KJaEe2>NNqj)3evWqtyI?6w?ZJ8Ox za}wG8yY*vT{H3{>h9r(hZ)K{!eBad6gevirq7PdqJA<$GuFH0f_F5dznMBXDI^(oP zJ=_AKd+xPJo3x16Sl44iqsvFWG{*yElUg=ADc8tx;U;3g1@;Zz)#8Df&gIES`FY*W z$r-;q19s z-%;>H;>QDU@O1%QqT0kFpO*ILP`&bP9rh! zn`upOv;8uqk0F>rF!sw0)w=}Ry>9+ORr(7yx|F~?uvEc9IJ6;xwCS8FM}}Nb{MYx7 zN;ZOC#63S7_cD$hkB35sgf^yO0p*sK)~y?_Hj5=6Dy*6F?QN~IfV&<(KMEPC%9u2E zxG6yfvqjcN)@=FYa6s?<2olplvFETVGSuj~->TF*e>6X(CGMpq=VTuJImG`uQ35IQ zykn&Yr%Z5b4x6!Y%NV$*&Hr)zzN1`-`AZeX+$3`EZa1t3$|Sk8GaqW-={4k=ze(TJ z>+REO)ji(b4pKSP`r7e4TtjUjT;nvVobS#=Er@pK1^jhQI*I9wRhOxu5>aSBain@! zdnZx%L*q4mJ#Ri!jPrerI*JM5+uQD{*q*pKG5Nr1+Ht1IYF-D<(%qz_v(K=4{d*NV z&Y?q8t=HKz<7gHX0(Ng6-e-EuH>DgIQG^m2%&qGVud*mD>uyV12T8AF-PNLzG2`|t ze}xT1>Zg}&L+D{wmLR)1+)kcDF+F3v&Np>q3dL>>e65V7Klq9+)qX_DPC~=W+qGi{69IRn&q-x3ArIM zh_kFSV_cx_2)NG8h3kHU<=qg&dnm~~gQTMZdm)x!^h2gl4Obhq2XY`XN)Snxxl*ym z$40Aci8)>!0FD@M4_XhTaoWi6aCBvzoI8~3dgoC_x; z^S`)$;#cH*63ZV@#@C=}6QLwwn!ita< z&oZd;s1aw0!=#kcB)^oPN+PU^HZzCovCPHFd;347)^<)EQpV;n6AZE0?*&b=-+ZKBG^Rt!z=web2wna7a1@y?Wyt+ZI<;p4U%D5N9RVoW(@BZn#|YkyBT znqf^g3gX=qn|Ga~?^|C*CBYV{=YTGlAnvlr4N+n05%FP939c8tJ+4eWFXC&L`?}|3 zgg+EEbv>9d<}PFW-1P%UcI*=q7VvWZt<^D!Q$m?`23oc;ImRtjutLmb4BXdm2cWq+ z3KjD;54*3CosJ5wd2kcIxpgTTz`nwSFXzmjtdMt{CroD|*up+b5)M&uf5x^Td+uA{ zcVyn~Q!o9cHr(cNu?F8Y@@s%}giz^1usFeyq>*49L@>cu1;faPmryCaku@a9p5hNf zzC5q$O^g_a`)Ll>SlX3byAmt?N^L98NHu_9TgOFWRTZr6c&iN78%s zA+C0Fw5o&q>VoJK)}>Vc*qgnr!oJgSC0*X<)EYp*Mu!2=iL73Z3~%Lb2hT)C%I!D{ zXL7+NBd%8n-*j_%?>FE#i3hgXa7_ea=d+d%tC9Y9>y0yWh#dC$1wqhQcEu2^sB@5S z;I=^-0}?6~RRQO9I)a(?0hKGNcGB_l0Tr)MT*6%|hrN*z#RRnEm45NZm^9Fj7H+G_oS0o?)i zk-i6!)hbIu^9Nhn(_cp;X*xZx{J2;i3l$q_TaJQnd*gv0 z{&>J=m~c>+E*>z>!5O!me{7R0O+jf`n@x6T8tIfOiFq}Asx8#O78f-Ixqd)Bx~{aQ zd#%A|M9*T!F3DNv6!P_K_D)$s4>;n5d%|vB^;iKXS=h4QaM^)isMY8o=dA{fJ4>!c z{nui{W=W1r!8~#x78rBeFBJpD6vPOcK3%1eeWBCYn>$88x!CYWX1&|xBpFpSIpm)j zm4JB$IeSl7pAqOfFSn8AX=$7=o|i8qI4S4qDA<;>Te#n5xv-K7Qgt#S9Ce{%fjZZb zTHj2~xjENd7(9#v)nUGt;{hk@i|P)x3ethN+1p}Vx}WwK*=+A*JD7G}>G-`$`?(9* z>uwYumUr(X{wat$}wk;(;-O2>SK_ zw^f==9U~MBle4aZGyAM3B|~`o%A^7%q-smdTyUoAdD&(At)u#jtbJuFt#?(KEbTW12pRyNs$FP!kc z4vBK8!w!QKLN^e+r3eWsJ^}rAz1%?Vod=@h ziYz>e^k)9R%Yce$Wega{o`x#|V~gM?dO^z&tL0XW^ry}?(j0NB=8rrByXP15cIWH0 zetEc=#(pqnFy){#Rmk2spPX)8y9!QM%`d;T$aU|0GR zd`o@O9|7jT5UkRd_`+3K90S_SN2T_K8^ZeExWsF{D_(eb<-6~aw919^7MvycI~bV^ z%`roXV1?T3aN{iv&4;b+ZT&~9jtcwdcQ&{x-_x8?5&Xm=gz!Ky!9?il;>9SWe{|_F zF3UwXKT_|}dVE#<;5}38!%2;{xnLeiwt=S)s*5zY{VjX#MseI_X7~!0zC+Z{}7oR%0#N zs08tq0bie!w;AIKb@tobVj-mw-;DFkE$*FI+*(~!Bb=tWGxXuVWH$C+F*W=D`7>br z#NQSt|7DW?XV!TC6=iYswXbVyA9EaUU0}CdU0veDtA4Z##LID|r)Ls9QtK0Wc}>$S zuff$65!tCnwNDGSM7bpr4(RGh)?n$rXnCN+oZ~re)o$y)eEA8we1vkTDdJ|PBdb}V z;rC5-v8x;;1fP8$IURHTJUj0!@4s3vX*DxMr-?W#WdvCq=@`C_TDQhig)W91d*U5$vG%Fqhx4k5J@6Y za+4%CInzKB1j!i`BnuKmqU0n<7Rfn>?iM7|4G2yAPn~nV@12?T&Bd&_nRl(3i{hf1 z^-xq-?Y*D9f5Bo(+(+!b7rTsB$q|c~9$Q=(8C@?Sk{Db0(u%ld+Rruwr;ce=wXIEL zi=+deO-fL4&k3)&xD`xv^BQZ5!97Bv*%HpeDCHw6SA}!GK-TH?8k$NTb+{1{-bYDG zuth5f!i6z4wSJ#qasyYYZ52z76V2bcBetD>?3r@-TN6I_3Wx10vc6Rb+`loyLf8_s z5VHj$({uxZ{z<$WiXxgct8_m%y8SnD-NllVNQfn9s_rZZxm~r7)w7A+z#$-%2p4#XHfx3@as;#A2&feMFw>N_PzDgfq?| zZw(!w@ua{AL(bn__gD%|gOuD#!xz$bFU~KZuJlg{9HJK4ADbjCZn7!sVMUO&eIl_F zK5RhQK?}9y3D=Oc=Yg&$fU(O(85!-0}@3*>?- zd;pi7e<)%-AVL8loezatFS?iqit(4i; z$z68A6lf|L^8l4DE0zxN_q}Xk3S6ZO>ps4%!_C(u(CUgxe?_}kE5<7nu8lj6D~>y_ zXK7n@&tR9*A{FuY*KFMqWEp`kY zHYEFH5`ewu6ZbQwslGScy~?j%y)z5hK0&1AI{h3T#K#FTr1_I0GO!V4$$^oWh`0fx zSUB_-4jD2n6xOdExTk#{N81>Syzg0kdU7$qSPuMR5KjU-nzaE%`edj-K;^!f@5Q8= z{~^z+9XZ zru{g!pc#t2k$3sj67SxmY2s@TZl3!mpLSMJ^;mpVdn`9_EzdS@l3&C;Ip9U~lP zj2iebYpl1N>wC&&`buzxfiAr~RZf@CP2~;NqraUlQJk2K8tZT_uck4JG(D_&GOS?} zYM2GuoFl)F{nKkjaUGO;z^8<(7IXr;6poJ84f18Nl)L{kMeltN0f= z;+bOaJ?YtW1F3PY{*^EV(iZ@(pw+_-2sD%%Lz`c+Zut++m*}HCUd{{<>DPk?3@DCn z7MF5Jg8zHP`ij(=+oI^hcPM{Zl>E*;XWlQR1w{9z6^hcT*#em3=>xV&f>@}NB*r-LU}W<|#{*V$T{JKzlScGK66j5dlUXVHHGcKq zGul&i`a^w6HPQ$eAwAFAF(>Qva2vhTdXV~)0LP?Z)eAvpv5dHE6VQ{ zY}z`t^~g%i-j8Nv`Iq;7A3@ha$xEGIiy*d_yIh&lwt;0{%w7?rqn^}c8k}eM!r4?j z6-cp_=pm%gz*c!Uz{P|mhsd!(Q?$QF?;=S&iDR>4^_6f69uN|I0tA@R<$9QRh2&Kr z+Y)nZ4y|3tGr1jBj0J(|tH&cpy3a-u-vV5);whEDzgYu?QOj8g+t{#jY`1zFM!xM0 zcQ(rzIjkx?^EFP$^?Q=KLB-!dv>}M!1}URF2wF0X_4(<7| z_oQD`m8o%t(|89m8|wf`deq5M3eY{5V0o>E9`7#SySQSiGK>12KWj{7`g1FyYL`ju z#i(8#^F9AOUvVu|7y$%ZMvkXbyNki5?{R?4Y%N}hl*B1fx zgFiOb^r5jOE4_pk&7r}WE^6>XFVn&H2M$R(>|SdaKHLb%xm-s`YzT5ppg-kJioEas zPxnNnR{VFb`tgzQ32kG9QHGZ(!9S3tyDxiA)xtdWk(ogDvK*(c3W?^1k-uYCoO~n*6S6 zM~VLQe{j5>XlOQgO_EF!R_dux&OH(c)slvU_96|1-NcQh{<(q^K z;2F*h@{Qt|>pw3&RpIz)RmF+H27mT&+24=CUv$nVO7fc%z%sB$InlXZa`?$MxqYhi1*Ii^y1_+w>0%emGz6I<6sr7&6*HYw#tS+8`WikkHA1%a}*^DH<@oDIawyAh7ginL>`fxV11>Jyj?CGeM8?Yj3IS?Y1`TQ8?yUSAj9-abMA%VeErLFZw<3szjF^# zxH~owc}(DZCo@@F;6h5k$JLcMIO2s9NAgbGEyDZLAqV?rxHp+JapCgD=Jy6OneO)V z<=<*u+j1I=M2s)UDAn+e!$a-0`k5ntKI!s7yIwS?U8_kJeKB*+(kf+BKn{R9pbn7W z)PsbKsrm=Q!^6jxYvoSuW_N@7rU`bBTB5nz`3Z0fhoKk$;C$bPFa523&D3+LP1Yjh zG_LpIrI%5y zz9>%<6Mo7ljoJ6k8rqUuDV(KD(kljS9#W%iucx536%j~5qbKf-jzrI&y)(BD?s6J5 zliS&l_*8W?Uuz-Osd=pp=S2^;1s0jFt*#H3fwO`o@*6XHoNtNJRZ#4SEFE>;vJnDV zx1T!g+&k$u*(hKl#;K-z>)bQc=4LOa;;MgZ#HsQt@vAevzo6%M(kXTh|AtMG>P;v@ zOi=B}8s&RPEY7(#F+DpScx=PG)*w3m8!wZ7<>Xu14Sbx`5HH)b*oHuI0(9hHb>j5UyVF?z0^YP za(}I{`Ll@-29e~p(7rXK6&n0wmGv6Uu|E%0tf9v3<|!tu|mNd=qYf| z-i^6Xjs$}I(*kYT?P8Iy^X%UCtV}K}u-?Z2dJYPlI}eaLgd0MCGh~vIAA-DvA81h2 z8HTfO{=H9;s#KfRZ67>AqK2G^m&`?Kg*2^HFh9Mg+F;83*!;xMD)hjY)A37n%F?`+ z>@hNB;8wYJ3)0u%ELlvL@{x7(LFl9n#}@myVUCE0X@dfli6^g^|0Gpt=M~Am%&S@6 z%90Xv{rs!?`%K68!Vxl>v2S|Gv{ZTa50ky>b+WxrsEtuWyMTd; zE-r8o{_W&Sbi&TJ+&@@zF3h3tKmlZ+fqi!DfxOpd0e|uSkN|$0X zHHAF>@V19Af?pM0h+elMRYTQ25{YM>a%1bYq*)aqv3Zv35#TtAVi7Y2berC{-}l+c z*_{GO@x(W2h|4*RYLXRM2OPFU+1{%utH}7_@&TF(N!El4@pysFN7bextBrF#`yk}9M7 z)PKp~j(se{s@AU$0vE-)IReU7!BB)2KRM;9-Ec)|B5}~~M9P_Wrw=$HG`#^dVIPaT z2QJS+F2+^|!?Tn!_!+f(PvHICvfGOdrKalByR?R1r=G@%mmh!UoKLJs^#k%(@qZ^M zU@pjjsHV$KnN9G+m&>l1%jOr1ZX0nD#B4ct!5>)l2kh#p^}Y4A4$~sY_qG;#Q=;kC zMH7%T$M1ZzZ}f0qvsQ2=k_MHk&b^V*x_<9T=1qtfzyH^b=jB4hdFV3GF_n*t1ElLo( z@LEIS+_rp^EqVf$vOH^Ew#sQfELbp@1`-TE+z4G(^abYfyl%QPz|935dno8`J0;;m zGx{uUeSJ`-H~Zm}231<2Qi4s

rMx0N!My#G0|;0VCib|IdgOyLf5e{RH)r#yEB+ z8xw>*Z-l65uHv$+O};();)j{XyNw@aVbN#%9;6yfFP826Q>!yL^4c45;q~+<-kk-F zxR-AcVX^03%)onofnKr9eU4d(K1(6gtkA5&f^}M^Iqh5HP{DM;uf^X)v^lM2@zP{B z{Hl!$9Zy`Q8vq0xiZ2GFa>$xI0!p1lcT!<)lg^UlULFf(59p!sgIn|wxptIK#}_uP zHGyt-bQswk>u!q~_$~Zt88&qAvSn4-7E06pYH=l-Rswfi`m$k&MF89fChVeuiCGl( z-#tP*Bo#YfJkHeY5FirzWAD)k`Z)z5y-@}gXyJs?kBLNfBa$TC6pj!6poVUEqWmM- z&Nt2%RQ`-^@HOpdWNbxfFIwz zaIk{Jtq5Y`UAMEx+jgF035XB`F_?wE(_|s|2S-1X!T0j`uz|UOfs>nZ!u^R1VZ|%| zS{(FYa$!a2Xx3An*v1WF*yFpWoy+)x0KUMdOfr{3h#De<@_Qs+)UYkC`&*sy#ZQTA zvFaN>>>H#G$r4PYpOs?Tj|J})k&yW1fpz-)qN-$h#)Ic+hn57BqSCu&0SfogD#uZT z;YUDgatu*K`9plDDRzk{D`)+5+0G^y)B@$^max+Z1PR)ZU>wND}{6{{i@-HrHmb}g@P(RYEt34|39)o}ho4_CfU;rKi9QlhUD}$dDryRR#2n=i zXYheA^*ipNB3tB@e{KaZmID)k@Hdw_xhnQ#c~agmhBg(CA0kOr*0)u|njj!3=+R)+ z{lV8aqFjGVv!Ey|^AN=S>0Lf&8UYje6L{Le7kloEZpR{_hu&{j?o-p>^ zZi&M3+o5@WA6wH^KyP8xm0AK1=I8^0g?pmEMXq!0B@;bIr}vGF*jGZX7!sd!+3j`_ z3anM1<8O7?4LngxKQ@^T8DshxJB5;`}3Ad#NjuRPcfq9lw zh=}q+7nkz}0f^0QvlPRfQx6KVTL?-_&r=Cx`BneZP>Iy_N5)p2?gQ=yuw!@hIvF+& zm@pxyeq)kS&fkZ!G+~6xp~M|TDy#Y%g5I+C?&nHT7S$Kkd8HEF%3*kb-VHAH*Qw4d z+zRns%DiLRYP2EeYMd!ssF3|As?>;{Csf5+T__Pt}6;hYpdPdaSet`A@OG1Qcp~+nA!&tQkD?x>ch8xoCQ7AbwBJb~usuov5^4Se(TR(EzurdU|FS!8C+NFJqCiFn} z!QmGrFN6rS$n;yC8#RhnR2Zp8j`# zp1o>L^ zZBP21NMX^Ib^At)pKt`Ih{qvaPI%}H6k9*34`87202GK!VTBt33JUMOS~0TlV!0u) zQB{(XS^Stw|5shk;wnCWDsZBBuRo@Tw48sKI=JD(sGUdVAKg5zNuhPhtY2jb^^Y{3 zdSMhr;rb+#<2)vV|IG;#t97i5Mdw|^ULk=3`F>78QMC&YMw&>Kjfg~qg~dlq9okNo zEI@x?BMS2>9A47=?@m^uIew}betNy}RsW1{igRqY)V&6D-QE>J zK0?d5mr;H@u7pO?NX!H>?BZ-(orjr9h>>RMRc`02|;&;iN2`&$8c=L-?Q#SisaH2^+1a2?b!vofZz z_YS{%mXSr%px)F$iAawnyEPXckP+?0xH?E2q{5eCNb>iPbS?uifU>(d{mMMz6PUQ9OXknBn> z14-pNMBr7uR;t&1zABGI_IMlehIDYg&L$woxHQ(;A8>R234?P=mbE>Cu?w>4Q-iJhK;Z}xHvz5 zq~dG#>R7?~leHC(7opIGgoh#aHM-9nqt%k^S4*@jb@!x!e{xT3v4?Bb_2Z(E!}HBD zf|&GWf_n~$u7`Kad{cf0*6&oD#Wd(bxjE;JL(R|@)9@uOgFh6BZ;Ikzsn5HJ4 z(#-8($p0xqzbg%6)wA&~6g0a4!3@1U15-E)F!l=ju{@hUdPA9Rct1mRtdW+JWYg|y zL8}<1)?2`UzJh7?i7_{ z($A3aC{2}ynwB#UcbL2308YMH{cOdwVFnH^Uxr}gO=~2ik&V&fT{*vPpu&a*4+<1L z4=x2|foM(}<`@=hwh-k3c9g!Mz?hv5(1r4+o#p%VIw1L6&-UpJia#_aDJxF#uG@v> zJnwkB)XFA=CB_h&p$U7*-zl)8c?MjD;aWTis3|-2s{jfnqnV>4yFksQua}9_Vv&E^ z*aP3{R^r#L-4Y^1db^2To&pmo1=6X;BqMtKssR6>kni7nAzz~?1k;M^6hGm7LgA2D zKdk(GLywh50D%xN=-ad)e$_oqGyMy>@Tw^qVwjfw@%b+O(`ZK<|Frbc;Jol4i+>H} z^=HCWW`yn)4ljjB!obNcucahrEB~_L}P6M{^Pq zYipZuks%kHrg#9E6UOZPiGlnh#OViTvxY*+-`Zqx$VfHuxnb*Eqp8{8x!nGbhF*9Z zj+?g%`?JlHlE_0V2c`RxlS6K9j04m!DJXCZ4fVotm~jY|OgZ{2nHiPP*y(D5%u(~@ zK#JSE2952EEnA0IF@e(88&Z}|cqnoj0HOsIXO#x(wj_xpg<}oOhs-p45-EUt;SwN} zNLlf=k0Z~#K^Xl!_2JubYs^OMjmsucr}1V!mF2Aa!E4|S)pd4!AKNOe`TA~^e|?l` zglvb#&J*4(ilV2#Hq#-6c{{ngl}=Z)GL5|N%x5I}HR8jds*y+2r(VtY=+8F3LFA8O zz|e|$mP$AIADowsZ59@ka#Mm7t?lZA6b`H8AE}vfmT-JOk>@Hfx5Q`ZU=(7IV7mM# zLppQfD0}ht>aL0qx9vbTt`7tK2@`pX6fp!^^_x@!Ar|JT_hSe-S#T`Vg>F*7$csC%S;`_FE|H68z6O0nu7i5um;`>1%Iz_tq`W`CvY%&BMwf7kusV4E6?uCM#Jrr^!f#Q?Mz>#*mO;q zQxtOCV3j%-imrZ7Lr>77r{c2ZUlY3U7YC5v`VLT%NeL`z#9+w>gEC7E`w)} zUo4k=Sth_8dYRF)L2}0om%9c`0sx*QRDt1c^5kyjHp>=2t8mNxScLb1Bg^-56gND+g z$76u$$Y4T=>F&kVY@E>T!1qostpHBzub`uNMpa#P{9$%a0cUkorewS!S@J`G)x?fg zBK#tbdntwByQ5#fo9%8Wzu~2_-%>+pDPIR%iw;E1Bz>eE3I6TPN%iN6&PR1)(1J|y zz_l#=B^CuVB<|g?w@%&&ZPo&a5DQC`SzEXASC{swtROSf5g#!N^XJoS>n}%p?hLhQ zq(aMKascQD9Fwtb0VD1WGLGS%^*Yb`*M1XP=Z*I%g5k!(_EM^PX4NP2F9CnqL z!JnZAO1Dbl#Kn}0m()1*Rh2nDS1DHV;Y5%DQ%6FYFjUI2xeo zmgwStqW%){bG|<09s7F*Zlb6Fct;D>TdVYS)jL2iDvLwev24A>|tC zp|IjtrVJJOxT_hJtRtRiISNbpT6LU)ZMY;B~=xrQ{ zm@}E5j;n7@683^y0{X%T7xy=a75D~hyZf)QQf|b`HDE{M!;X}ztq~GD&W}A=n!`=Q z1+@vls(#;}ELcr>oH1>uxitv{x|KeaNmk5wtjde};L1w$o7K6d8(~F^t&x+4zy7?m zPqDk8x4vAx)fsS)_=`73!Q&^*!>_%T_oEr4LGyocLVfRkT9N>D@Wit1c{186QXG3$ zO{Wp|P}$9m)_<_jrqyQe$bUPx}rE5e; z9s&rNuSHuYv#Ghm%MbAGW$yQ`GjuH+z>R}TOjlik?7UT=AzYmvejdzyl$@_=gyT`s z4;AHi@V2zPH#PyYHq7$I5N6*$=(SQA5VrLsOXatkvjp$mvxPk1NYj3pRA2Ri&gKyT z3GOB8+-A0m1%^VAbzHx|>I?DowL26dc74HQFymJ_E7z7My<<1KBJPK8LXc{_l;YOf zYaJ=Ss{X+#XS|3xW21r6DX?KwbKR<-0pWt@S*fJ}&V|UDl@E^tM-deHlc05i_+NF! zW{9ERF+P?~X;MN$HPHfkWnW!_U_0hb=bftyP6WPgWI;A+Bm*;nh#A<3Q=q_zvptgd z2ZwsY(phif^D29EztVXy3s~pbSBCWUXRKOZ$xIp?aTeynnj)a}aO&I7F@3=vkNT<; z#@P33L(Ws`HHp$W9?!IVv*J|l7+=I$S@^wVReDtz4U(z`oQP_xb|teeuEbM>k_P%I zMkGp-J4`nN^rsp`?pNGTCUu#(Pp7Nl=EFEf@=I^(hrRuqh$?abjYtp<7U+}ytKNyIJdR z5pvB?h0AAO2}k~H%Zvn!;RO8~L-~KBT$}o=v$@#CePh8Y9*kdi+L*!H{WQ?hWCpcr zqIq)ocY@{*n6OjDtJva3rYZQ+7l_h(b=ameY{*{Yi&1xSi&dHUm2W?K;702<|+US@af=&YztQ(M+-WIpSJ zNkd#@iBu+LRki9nktt0$$F3q91I$r>p^Im4onVGA;V&eF!?+F8c&hORh8_;^aIAQd z8O!~;O^@Qo^O8s1{HYJhU@=I^Y8MP`pV_3Jkim)$O2C7>d&Q5Ja$ZG>*u}?#Ae&y$ zkYL+1Pw-t*{|9!q#wxfFBIy|fdbajtd|DIS7)VOm>D6fE;>9)C`}N`FIG#qLhc&TX z{CF1PBD{RQus;Llp_O6u0hM;lI8Nm`K^(_n%V)8bHfO5sO?2NI9|s;*vpb4!1z1Hp-a$@Ee>xmU>Laqrc2^x zBTx7wIMzCwIpsQ*DLQ&SG0EyZjw+A1+@HNj>v#_ z$8d6#fPd#y6K(kFk9v@$bI-F^91e+?9E7Z8aB=z)sizWX+-qbxx!&W4OL`@a2UT0#fckD~-UVC6ou90<_ZTjJwzVtecwjc!1uHZ%!#zfGt0!+%_ThfHAp zORcuK*2hk@`E~Bl>G|#LEQt=wiKoIJkKpZRUcYKS`htCk0~Dvcr@LgBfR-Ex0^f97_($qgOiV<#|!XDJY;a+qgx zI0aO0Qyi2Tbi@L(FSMoH*EVEo%Hp@GbCG)r&L2wFX6Ci_K4fh8HtG~l?v!S}si=JZ zUcbSNv(f!9NUHK^LSgof2CgvQ$pqabg{a0KE7iS7USC)Be@Zks$0vL&VL~QFxT?gS zM`c=o3;3dqAd+Sm?ntI6f1M!wLC2gKBXxO)Mjt+Y%_E5#^c2f8{;Fpd=8pQFh`w8g zrw4d#`@0=2f6onZS36grS{`iqmI5-&*fQ3v_iqP*`vK>=n4U0tmvn531$x!nYPI>Y zfUms8)a0+DdcMIoQ0AbId#M#&#JE!{P&C`9)iR%(>!E z^LvV%l=wkWG~%aucB01*9!?t+V7B44Tfw3DIs$!w3E{8z7AlQZZCMjOeG*|y4Vz`H zS$KNrX}Q(4u@KH;B*GqgBzYMF%kZCRgl8|x@PEoE)JA7Tc0S$Iq6K8EHFZ+7W)NKO zynb(`7&+LOE^gX?%m6?pyH3?l1JC6sG<0h%y(0iqg35~qf3ZG0>`Mt2GjUr6dEw%W z>uG>3(wXtEpZ`{H2LChpKcNUxokCEcQA@Q;%VvEycdfP68D@J5PiZ_(Gaf#Lo!pEb zZrsRBIA4{KM+O*QFutD$xR^EXpkMYxtDs2rdn>WRTL}l3eu8(Z8@xYtjvspL%$+EF z@Un$gEUh{F7kV^AJFUBfi4OdEQAIqoEy4#Nf|wJ38`t%qCL(zkxWl4)^?*v~+uQ23 zFYyl?0;1X40|K9ZND16JKLTZ#jwuu_AYHU-x)(Ob1CZTO4(OQJ>a(RA8J=iHh(Yeb z^oH6OvFNmg(NkL-z^h?p)@>r_lyM(!Uj>7h%_7?YAs1k5!_=T|$hvXpZRzn|jEX0l z_r4tsQxLrE2nxqTLdXeW3okI*mBq{0V%l3a9IW&a?0O|@R9Mmz_Sh*hFs$85qURw) zGIiI}6ca4hH7{JK1KWVE91Nw!_;n8yM(13_m7~u~&+~;txQb^VHwHIGYeIkfr^`IK zr{cM|S&5@6Fj@t6ttG|cSzzG`#Bez)GAyLWEV-g$4QhOqyT@aYYt{^rE6wyFQK1oP zQGT$zNFP7~s}>;-T`nNOdR1U)hY)mqc!&)mtXjVqws>Ew&@BB5r8isFnAS~4;9RMr6m8YsFD~7ZRLP3rmF=*`l!I%aT^hb{o}-N<_7gQ?5%238y3ix%bkx(o1s(k6(`6a`|($Y#;_6ZHOo`o9ke}K3D z0^j~$9=KPjQR7n zQwKo8E|HSys8prOEARkYtO6_AvIu?pe8JrnBv)xZ z6c_--BbtXIa5y@6*ZspqrEI&zbYT}raMB&Gizqf`=&Eyhz0|{vG{;=0arn0)QPKll zDL2B3;lC0&`r8Y(s;H5&KU7v0p}pV}zb^cLaDw&z!I=pegD3p2=V-bf1d;t)Tz%|+ zimSIC|L@X>_HBRF0X}o2Da?!BmlV2>HQU^}{gA)?=R897Xxhg$AWjyA3|1uu56V}!J@pFCv znURKy+8^cHzc&M{zTge3QN8hw7y-`W|I;7h{bSMhAze@J@_#J#pM`yD5Wj|bzuf$1 zssF4iv*5X9_ue4%KTG{*UFQ8(@B052FY$j-UPy)i`;u#Gx~(s5J(fFuyfvrl(A!bF z#jA0VoiL;Nsa?Vhrrm%7)tby2cSZ9<9t6M!h=WoqTDzc(KTEqgh+HtLwD=D~04 z{c@!)QHms*Ju$`X4AaIcLN;BgRk^!iku&o(NTPi&1x%casN~)qlY=)N*J@8Vxa*l; zVjQDXo4Xif$5Q8+-D$0yQVzMiI-UrvV3LD;w<~mj9o9XwRZXZ5QeKh|Ay}_^ky`ng zGlH9>wEp4G=geoeEwwZw%=iIz72o}q4p(4$JX|qn~f^4bj}NI&ebSBH5~@V#p>i* z-)PB}hTdq||IMROA(-h#IIOiaS6Xy!^CXp&Y;JvRLHRZ<{XkS;F z-FNA(BREC%bdGN{L>B&SqhGpc805b#D=oJg@xf1;HKttB*wX&i-l4*c=j!=@f02E2 zb2HLpgj}EhB*BWqWFgXDUuBuM_?yJP%O8Pp1x~o;Oc&$w&Z*^Yz*Ya%;T5#GdD;8< zDWGk`nNTQeVj#=`I@(W`7lH*0&0qCD%@26liT*G!Hndb--4xvvA6&lDeIY<{N{9kE z1`s9!+`>be5i1y~$IRlxd}#$XQm3)8zEv_~Ok!1Uh_pBHqn*Dst)grN`?bMxN+O7> z#AynaVl;o?+L(R*@tCJAUM^t$S%5=#B5Ssa=W;-@tFSOg)z6CKRUP^a)4$+t3F&lCd8oN9t;UTbg`zy< zd=qouD8D5M&Jcetb zliyX`?BE&2^M1Pb?*n<`fnvC!D=RCfi)j*h^jK({JE!>o^Mudl_&lzN2{v>;@4QxG zvI`^F7^a6J(#gv z5?oV-{fypwq?~EQwgE&Rhsm${DUR0Kz_GR*-%V$3AP5V{%Us}J$Q2lSwi1@>N80bL z#Flk&pWsA4HX(!OkMz!eDt5a=3 zsawWU#Q-AY4%`B13KFrm4`3-?XByduJgfF0ZJzRBPk}*~lxfCaIBT{cl*Do8rp!NK zHr}onCOo%#v^%6iS>+{6ZUO=l1{89 z&^gp@YWwWhS0%ex8C}mnfaNtFkAZ+w!pLhSzDxEyKU&!}tBJYoAS?IbhpYs0`?XNC z5ayiaTMcwkqwlh5n`&K*%-jn-4b*&me3(Vsygm?f)V)so2_Tka83hUe z9Z!T7p=ONvf<&c|oObKV*6Ql&010`gz5mYx>fhb~1ZqM9!=%b8GXtU_IvuPiHFw6W zbH>i!sdT0d1`_74J6WsIQ{(O!7_)%;bz;uvpRpo!@(09B&#gvI@#uQA zQ0_;$t$`YG5@!N1+3M!vSndzK0rz!HAX2=!kp9 zK9u9;TJ0MdU#&T?_f)%Y99*^sOsef^%S$WTUXsu=&G zuRF5Q`N4>Mj~8*m!UujYlKCjtUmfPNN~QC}e&FzK>!G&>rhf*-ZE|d^FeZa(_rHfy z{bqja+1Bcv{J|*eD_TK7#e$?;2 zx(Mcv7z)>d0&1akb0C#~(&bBlBX||k^(p6FK<0^>pie_Ghw(FhoNP+b#nnx;6JFc@ zjr&-6r4|o=iX=`L(IPZ%m@S{#7->shM#>Zr%3RtR*yy5RuDS$={~xcm*#L(rZi@JT z%tp*~{JDV+{&3>Se2NhC?DcOl2eA<|P*eGo8oSTa0<)>DYK3?iyzKw+x!zCaQOC^2 zKo_0jWW{Y;P}AoTvk+=;LNSJ|YF4wEozJNNh5vBLpeE!z@v(X{W#y=o%%#x;7?QJvEFWH%T#Pv$xrw#s|;zI2Bf z4qtyg*IY|^w&imD%UeG*h|4b2Zy7Fy;lM5f;EV;=Ci~gk0UrJ6y#=*s^&BzLbh*Mq zHGVys(6itZg&lowfh7%Q|9cbE>W|l@24kx z#Zq!2n{(A0zw#7A#bEhB$y;LN>x@a`6<6u>)W+-xUJt$5Gk2YAx#OB7$Z>)EtKrMyK=)0mrxtMGw1%smp4sq8*dp>D8pniy&p9M)7$oG=YOPShCsIrXaS8mArOO4t;_kd z-S(Dv34ue)y=6c9blSG}u_)^wqAL$S_2AJQ%o-=;dga3{AJX&naIsn#9sL>yQis3C z7{}ZQ%q)|MPSD(LC19qwV|qP)HxSsr2%u8R|5XMnnBP|TCaf82pv9ZdWd{zcW8#F4 z+{dGsPpp*Pn-}<}Ro&jEiJFl1F{g@+c>C+bK`j=k|Kg^4{!UExBHyD(KbPSCf&@ z!^&}64g-_y01FjQ?3d+7=EW3XmaE z>S+_Hk2t9W(u(l>MU-^yWDqajDR&qNbp646=tHypb-Fo;TmGpVLzwSK0#U&UeW~a( z(Ym<&m!Ip>Y#8ib=NDzUxh`UXFpQS`fXu!j4`fXt?AP!c31K$5Y-tW6nbKgL_!e%8 zn;J**Q2M?#^TS*Gj!ALIA;9{<-^%meGt<}3Y^b|LKvIi!%j-N73O`i?0&;hlXV=z) z`xEQ@#kJ_p%%1ui#hD2kjgwW127fvj&LMs6=l@xK^L`>pd8ER*Ojb(5YnQGG{a~o7 zN%;;3XHN^BAQIK79cFKZ3-g82#TYhZ^_2zdGz_1|zN4!Hl_FQZ`}19^sh-s0)pD|IUVrmZ&}ViAm^KSz zp^eor#0Bf!Lt5gGC9gG`I+$(QVI{ZiZNN zG2bX+$Yaiuf-p@k8)V%}R>~FAm+&yJOizm!)y>01T>aTtyM97rMa3V}Gjl7;BHKidv42qV0J( z-OrskOlx9=y!(cHJOX~>0u>!Fic|ya{y!N0zyr7^>ETIWJj(*#dvl>Gv@*;(!g^ey zYUbMF!!p0xlHij}*+??Ks$9AOt<46P6KL?_R}Jo)$8_jev_8ou@=_JveL?a)l43+=j&{OqGgc^p_VY;o z5KtuF-+oj*ciWv}XYayqxU=aR3`ZXBiCxGj>pc4=uKS^`b62GY+_SfTHnZqGRt=nM zLkohw(KXKu&tM7n#r>KPYjhVgN!O7dEee~P-41P97AxJoCWB+OyTqno_EKlRB5Iubf+UuZ9j6}93kW~L9aSh92vmav$!~Cb z%OTGUN_>dfQhKMBMRA4LxFqdhb%JYv^R*8o%b`~iIxiwq&eF57+}0O9`|D)3XZCn< zwO$JES?|~&_s<hW)uoGP#Ib9wj(9c)G`D*Qwu$+N^9et(0g ze82CPs>0HQC7=yG_MR)43{MNmqpfywZq!aIS_ZDMO$8hp%(LXFt9!Jo{9Q zu}swaY0KN$*D5L)ZtGiCs`LG=MRODolcYe3F?OdUw~3!(*_1GMP9^t6Mu9iYvns~( zx5z&7@X^Dm0PK!x1zBAgM!;0%3TlLUl`E$ zyMWp7zPu9IqB!J-D%%G<%%FkIH}OrOTMD$<)$wej3@6*n3=9BQM59O z*5V82=UCt~1V7avB~?GNykfL2$b71)F;VD64KmR$;tGf9i#(bl%@G$+G-Nj!_z&O? z71j|YTlzLYC`cN^nvX7~T5QIyYjo(|`X+=N2_E_tA}V9m_v{^e_iNv64iM*wYOD3d zx6ObaDoa9V@B!aeF*%q*rg6{0;#0Bt2G{S#k6y-?US3Y>(A%rq5?xOg0sU+I8vg&w z_auPHl_-zM7muu_N;nNLvfEG{&<7(~h6(nIy^?oFm*ADhO6xnxN|Y-7j?~kS7*b!i z^FO1bqwJ%9xPP+Y&$7@kM}jYXgtg9K)R#>YiU0NSrMy8-E&s=1(9$nPFz)?~y>%Vw8xZY&WX*Vl?~zxvBI4oFnOXz)1j zB{;$;9g-4tBS66~;x_@y;AdC&jwPAlxg|OL2$b9W?A9{)a-&@q{GB5&55WFNRN=-f z(UW~j4XI6A4%oNs)3_V=viWz;R}eE(JDbRa-liC6%?YgGN#ii@DtH~n5! z8{;m7e_8v$bNb!QUvW~G_{(|&U|7w0RH%~jd0o$l@U zqr|4B`(?LUPg}YwBkf7aS+My*jhaAh8W9PIs0CzfBYr30YG(D$L6W_v(nS)zcxVH_ z?7Or1cBn~ccTpIZ5lV1HM_Fi{>c)gusTG^iS3;o@gr7Yx)Rr@D`$L!STy^`x@MJj7 zd%)0XG9EHV#-%nMDBysax(LIYe*cEFpTM1!dbSzZ!an(<4jbxGkGy|w5w8*Zd?|vq z8o_j7i|-G&y_>I5deSO6?pZ9%EkHz(?tVE25r*Vu6lNwD>v$5#2UH;-Zm_+e!x-yx-{E!O#Yf!Hk@QEKuWnd zV&MaJoWVWPrD>?~vF!mwZzG>8tSUV#g~L8>60Z{b()lpD`4P5*{xH7xGIOK>=}Ew&Mb6hqnCH&O*6O6dnz= zyp#nPJ<*q6ZF53jicx=PfD6AeTnfB`pF3+6t-NADQ|0(^c+^I#ICU_2)q1fG{u5ytmPQm>!LR!;I%cH4??&P)r~z;1VkjdZeo{CQeF9UuG^Pi}EIGN-e5`&I1m!xabqlwh&nB!LS1k=RtzF(My68%-ZM2(KYBrO=Wq z>TElSTTeY2lL!R+63M22=TqAUP|C4m-wILm(A!p*naZ@52m<+uYZL zJU}nd6(7_!<|-+IrEg_V_$y#;VsL4Y1!_qKaNFeD=`L@(y(t;~wq6t{x#l*Bl z)eHA9NB0JH4tiz9A2^BPwvDU|rnbVebo?F*IM}>VW2zC;-rX3#nQ?u;iP_Mc1E|ma zbImru^OWM(2jZ9tNupY?$%y-FeL`!AAJs7{vivLrv9!@hOT=ZKgYr(3`${@00|gKk zD$HZj*yp@f^@x5sAc}^R5hsUHtJ#P@&B7tS`~h0R6+AIv+%#Ui-;qZXHwLITD|7wL zE!+|ieeidEj*cSVw6vf;dp6~K4pJZ9PKFf9)#vXFY*V;w_x@}N3i#T@;vvl5Cza|F zKMM7nJd;p&=xQ!dREeqiymSu23?-rtbO`{dD#-O5)CnY+LOudG;4Ii8n7|Z6RJJZt z58%UIVQ&jCJ%FuSAxVK6WsK=JL(P9URlfh-nHCKwQDd#U zJNaDZ&@=9>D4RXhlB5l^ard_ON%6PS>h0rtThVcuGIh5_cN&`YJA z4K}ifVD%eS$JtW}l5HER)s?%)Yjb}$dH=_aWuT~wqsu;p0CP$@Km`7~8~Zs{{OOyX$mqG*QAU^iY9BxXyhdmQ5DbZ{}?a>49 zP){GJY21LJmX=}Sor~G05BT;65x0y;E1!TJ-V%K!S4954>n`JW6Xx-|Z} zo&Fj%{u&(r$4@AaS+(YNZaVvN53l`EP$U8Dc>jA$@qkR!c=I{P3P_p`w0j%3bP|#D zC!1`<5YhU=|E{IaiysktmotcU>gS+LZBwUz+HwG8)xx;_wJ?a*Ph!GY&*|#c$Ej0O zVzM=CZaO0OmrWEMmJ#(i2c68jWCTr&X`9H-T#G{TY}d12NjJ;2$=YZZZwLn2X+M6| zCnwTJr@#r?Ui(^eG?r`^)sb89@Voh>#=Z_e_YF;Uu^#;|p!i3_Ra`t^{i@N*uEdo- zoIpBqoWIZSU%=V34!4GYu|sLeB@T(`E%=)uL1ersB6Z44Gk95+NGtrI)0EB zj)Jj2pFFsi_)_>ah(qAu3pKN89A1Ahsa2J8W2~Mx_^X!7A(@X@;>ys3a~b5!+S5#hcp__S1g|0(9X~_u6LV#6eTewFA#0{;7yJVr)aOOJ+q9UXHibRBy%q|U@>r^s`o2ei-fgC7#MgbZ?F1Xzk=qHD?x}$9N7bOZ z+pG*WIt-xox15AVDrxNUrsQ2~?B(QO7rl8em^=Nc0mb8gqVdCxJGc2U(No3>+~X(j zM=n;T-yM{BeDwh|60M#oNWvgTC#9)0UYGsteYvCtC^C|p&$bt&5Z*TibQ!&N)7arf zlX3kZov?>dZJLZpHWco!;otmV9QeT1=3D;v_a~LD8p7BGKV=#mvtKEhnPBgL7o+Il5t>++ zx6c<4^p*jok4CTRMEH0q`7L~~vw#FYkNy2Rpn$~bUuhrz20*(Q-ipKuMdWSz+J{QI zC>5(cW#ZBbm){TU%Uo&&Jz;sA%6NN9=3SrwUck|Jw$fAmlrPkUckGN%r|K^KLR4a= zAterUs_;B<-Rj}MZT3o`@`;Wc`EO?j0zG;156T45pLdr=3Cy< zPhR~>V(3&do46KY8I(`@;sy8R1Eo<~Mf0I5N7&V27bEn+aa7X}@Ugwr>vsBD8qv>2 z@w&OyMC*LDLay8Z+`jZ$R(Md$pQp3xuf7l^+OnBf{ne7z1V^hv8_{rknvsF;UZ-%C zhw-<9zFUtWniadyo<(yBJ<5%mW8i8v|YCSfKN}Gtk=-YwCv9>8s5%K!G>|KJVJYP~pnZiLh&Obso(IgBaQX0&?hMGDdX-rHH=&iP*lKTv~eeX*1 z77rc0Yp>7B=hFxNe4~e=P>Eqnk2u`&xA{aru5O5<3ro*Sfx2Yo;1u``P@q1E&}-$A z|MWU*!%DtCfL(H6BMM_#Iwz!irC8G!pfN%}2kcUupMP2$xEN5xdb%JSbTD@j(bCg2 z%BV_nmZ&&a4Ry<-yeratF-fon#4NFuYO7>G0a3SKSI7;i%Qb`eK0yW~X3}^wXjdiq z8x?p!zmd=sQjlI$41c|Oa&tU1xOWLL+3|ap+L^e&g>HNK2;uERona0d&-&hYue24t z34wgxme?ih?P1Myp7#q?Q+!9a@j4;OJ;}W#8shaQb^l)|{Qs3#`Cjj{fSS{Cg!DJ3 ziE+gE6rw?R|6#|Uge!W$N%G&#tN-W0Dr|Z1(*1OX89j$9ng~_pj zuPh0#I(&4>B?P-0<0S94G8wpIj>3Km>q!FvHcJ&DKxP2t;8iK=n2imoDVD4gIum!O-h+M zKCg9c_Se8BhAmvqfqS>G8a>6kw|-D!wQ=KI@J=}Db(iP3sGUM#6ITE-(35&Dt`k4} zm<@nQns)LkV~*aY+roIO5KxCb3At;!S4^JKlYWi6y8|zMhSdy1k{9oWH3E8HmqgdS zs;{lgJsjEE$xkw1Gl(ne@`&$VYIG}KQ8zYv{ULRJJjj7_HWQuZt%v3Pupqj9V}+~1 zoBx&BvsY_cN(}6DqO$~bY$!S;AHZqM2&3C%kvFgcMmghbatPJyf$8G3~h~?NW zo6qjcOna#`ti!e05Y8ID3cGi#rsh|$A88j)d4x}Gpd?#vXd8xzkayNDBE#>spl@#K zbCvCzq^F$ThMS-2R{CYUe>LG$j`tmJXJajQtdO!;kJIFiFG~=B4n6qRZcF|7rpn50 zlFA}3cE~E+zY$#T$qKFr2`<>Q-$LGrAzt>vOd|zjO#LX? zWN}BLP$hk50f}))3V7Fh z#I_5q@qCU&znotRqSIv2M{9P(P)$0rnZqIuYIDj52pk0t3$qs4e7KaEja!!Ng9&mE zmN|lAYoi*@il{bm5yQ~RF$^l0aQ*Bv`lpiO*EJKbQr}@+?)t_su_L!fgD>x!QS9If zct|d^^J2y&R)8G!R{Ym>Bz@zU#9>|bxOA@oM9xcCr@o@N1vBC~Go|+`B+_ zg7gT1C4jdwxogbY$`Cp`rjr{2Phi!`5qr|6BP6BCK^k=909@&k1Y3}lhQC}jz%2L0 zduPv?Z78(&E{~3wMwjeS=W{)OWg7!aV)#KJ5J!bUFe7g`06u#C?;3f~Fq#b7b?eJ) zMHj0!ts7BL_>z~G!{XJNY^D+a-xuz3K$1_wYA1fj%%k2BR2RAkPU%;wP7Q|q#fvsf zP>T6U-Vk<&oMR4esPInplOyt`i;Fnt??JydAMi@O0mMvl&Oy$)Ujm{n#qM8zH4xF; z&gn`?`(21~(M!c?^fYXRq=f7#q-7dt?2?agZ|)ByuKc)YOwudNm50Gnpm8U z7rQ1?k0FLCqai-%7kvWQpR{a9h5af19At*QRaFOQmljO6NUtxKxz;5>Qw;9g>LgMj zJRXO%Byy0~_bTE(V`h{6;QME`PK&JduQO)%n{fYm3 z-n3N6B7KsF?*y;9`3-!+Ub0d9^zU&MAk+ItTy=%{PoGM4(0{jAqZs1-+dx}IPzTp< zZ2$8X$iE2?P>fK+B!#Q6z$4m{V%JZgi6~KBrF=@x6pj(57&kIFps8MU23Zh(Pvj>U z;6;)iKb?S!R~A%Vx*EqH5a>d)G`Vb?>z_} zoyZ7i$68%GZE=t2m3*YDz8D7>ZcX}yN$6agr7)r*r<-{g+bbJji}u00Vj5bj6J^Eo zb#rvYYB+;O4}5($Vsj{6UBrt69ZMg*`$%K@lCCE{{7e-jc`tHbgCZ9cXui(bS5e}B zXV~Si=(9`799k~iNaJJN*7MKTtR|Me82crPMlggWLPCTBc*0YwlLE97h%LsXa()$aV(kH_66*ZCm3I~SiedgwDUr;gp- zW+@U(>-z$g-o9oCbL3ZZm6(xzTrJNe7U?Gy)IDAlVy6&lcHwFG#Yk zS+kF34oWLO{!!!oxc>{+>0!{0{6b8hkh8XOkM{NbNB5pC+0zaM_hsivHtUDi(*;9p zT}e4jQbaeB#e1jgt$IS1XiLe){8Ny{5@R+$2%XnZCU;5Rw1Y1EU_S?K4p6K|N&%Rc@8OS;7s9+g*Z9uvZ+NdMb8C(@o zH+FCg%S`>WI1gbpwMdY##6+Q9HCsXf;9$s+oCy+y5qE2H60yTgXeLpU(Rv z{_e=jhrQ$M9@{eCUZ0JEO`Vm!*`NR(hS_bcp9!vwQzdKlwD&BqoPGZytvrj13-X9k z^2yilHqEa^-$X+Wt>Zz9$^|6*NzKH|LxEE8jMh!;<@`}wScrmdcV_z2F`fL{v&q`yREa)eJs?70k=5#2YA}^y%QXe?16o2TKFwngd3k(ktq`OS>UyqeWe0L$%-Cy zcYtZ7`#GE#HsUrWOJd!*X`Q$8X1kmO-=w%|Gu!Davx63eX+-le0P}MaRo@-5yI9h36z83_ z4aOYRlmsU1&k?SgR|f~6!5}3`aa7yW%FSYfy5mTkQPb%B$(CpT5b`(+IV`+*9euN# zu5fl7&@{W_amwTp=v|$XaW&=D$6HTBt@_5R6ORFOBT10q-keoTKZ<#CY#Xa~s-qtw zu4-C^m6qLkk-lT4o@r)G&ER^`y!Ofs_mK<9DHcPOyR_>F<^vQ>i*wZ)Y@}YpZa+yq zcHUm}Z9Iptgr=f)Kg-b$x%+I=Y|~SCP*!1wv{hDCao^0b9c!s)b(Dn#I54I+<6@nB zov~?rJW8;*mo=?w?#XWMb`7j=(ViJh@9z!-9k@b=6Y04LzPe$u)rrhnIahydmuE(5 z=^RztDhJ8Mru;TlY65eh09^A&w@p_j!hXZ3>3h0?fDN{$@AN1mh4!fa4fyeE=I0s| zRpM?7wYbXMUfnwc{e`i;ch6EBMQbN2kH*F=kg0N{hLJu_GLpgm3g)*zPHNZKiB1$( zA{!*%A$c7WPn1Jj0H&=uR~R5szCj$?P`mwef2yTIliR|o8qF$mi5%3qXu0}Era_y4 z_%gi~K^8kIUiCbH#+1#Q&C){qTJb^r>!>Rp(+!{w`9)~pd!uJ`X`6Y0SN7|N(paZQ z;*M~-y*cCJ$uSDmW6MM11DaOB~Zh9h1zc^5~+xywfT2!|NYKG;c>t5LgCb;a&w7!j9?6xXCtGZ9`64 zVzP8{Rm3^S@HtC2WszIKRFYm&Vw`ymvOaIPss9R|qu^eFzjBqI4%_#d5TAvxN9FoV zU+a$NeeSwbGkTFPd8~4jKFbQnr*^ZJv4#gdv*#ldsE|HSe!4_& z)AObil%H8iJb}Jn+^@r2BMF_slVQ)(KC*QP`}mqY{pp)& zV`WJ5q(=7hMOBW6N4Fp*a}4G!)Sp~lhM}v|;ry}uR6isR6Uy&)EvAzNM$X&cSvtxs z30L7ik8WKY-7y`dfV=>yczob0EAvS zhOTarM^s%K+9-~uXM*b+!&!1ZL!ZU^cNa1<_6Pm8kuUH!Tqx%6Ft2Glb8kHo+JBT} zgt2M9NmF=5`&Wg~Jf+o-$VCbcfT_9Wc?Cp62P(*3;5PpHXJvVTYr1? zSe#FdT)b0^OGS?JvfdQ@+p^Kc{2scy6f%VcYbUT^kC>=r{h;lo@7Nb?Mb|o=z}vPD z5{j)&B815vex}ppSFSA`6r)*fl5BEn)G~~^qw%)#iE-+bexhNV9(GJY=GU$Z^h9ay zV)g2EV7y9fOstW%V;w`W7S5eEcoF)?=bB?3RWI}7uZtDWKSw*5@7n5(wHnK5SCG3m z^vWL3#o*a`lMhrQc}X#vZ?*_B_E0}Zh!cNfU0kPy-=cki<HfX}n7KE}{EU!=pb4BF7Qmp1<;HMhIS#CH3@|AE0vyzG9N|V;LqhvjDlOFEKe3hB= ziUSV{c}B@D@Ur1Vq_hv4Fls(r#9}8jQ|f26?Ok+eKZSOv_sG_5sEk97*I0M}3Awtc z9xx*i@zi|y4T4KbU8)e}4f;&t>+!5u zHDOw2(GS_YX6T|SJP?*pZ2+d`kMZ@ZkNG&soBZqJT82|1eJY;YYxdl4-8#MbglzqL zNa;Wi9kTM7y_dYw%h#$hDiqT*jzoD3VU9@!Mm@8UJgvg|!bbf0m};d2P1d$nbKM6Q z)*iiq3sg!iTI)zge`zdm+XKFgCnmv#5OdF%H(BGbQqD_IJE$J4d?Q|GxpG_bV`i>3 zhb^fDh0^K)CvZ07BT7788C1mEga?=`tKJ3df|X03;``-<<(grQoD8SvSs7>njkWD9 z!`l6R>hM{Xp-MA{W*hy0$;9sCkp`!mX2h>w9%wc>24k!t=L1|8@l&TTfKYQ5`*JK#V*T^T9n6B zvKsZKuBvHaRWkD7{4K<5R`Svrba>Ow4^=a>8LsX=IaNgyKSNdq^9VoJQOrp8 z*7wPca+-wak%ct$SH=(>`ZQ}NY;95AE#|$WF`*ucw4A+4H((Y9u!?2NexX!xi`9Cw zp8_}S`JaJG)B`hz7$`|27h|{Ya<9I9Y#hML!Xp%aB=6LILgKTQ&NDIvc`f1qDOI$HO}#^Q<k z^prGC_E|d8>L8|>c!`RR(|Nrz@m6Z*X+t~?> zC~xZU*QbB=#{cANLv$5@LYrh%vYoNQ2sO?qeEo@R_@UJFLJ3pKgoXqTpGmnrl`d6% zGC96ku0#kl(`SM?EEN+fQss8L1rcf6!|tn+|9k*dw0YnF_`nRcH&YG}JU7=SMMr?N z3l`C(1CS1ZboR1+2Jssjm7Y(eD*`x`8S;ph%Cj%1wF<;?gWWl(eGph@4da3UMrQ9h zC~-UK9K;6J7}PwOiy%G(zsu)i&@zHTNktKhd3}Aw$oP)%tB#9C+ z|J)$pj8w-6;5{RNE6J>S{1j1i2ew*x4tfhLAouM1z?foH2gwt$xnq|J1lHr~YOCXB#>IRa}+5f&Z$f&HC zLlhg~BMPHVPc-4%=;J!xK+JET2%qK=%&p}K5si=SMX<(@ScWnWz%d^j5?dKptNy23{fK(cKQV~M* z&A=;^o`Zy5SyK-9dw*Fsn!uyl9Ty@xKj-Kq4lh<*zifO_Rf28uRdb5#(oW}H(eM8> zB$j5h+t`_B7fF|#3bO1OYD>mxR)uDA8a!F_li04=hgK7vbC)+%_h%(f59SDNF_MM= z&KirD18kBO6THG#Wv~1A#t@Doh-U!*mpl6(#$~Il*MEKXSHJw9x+dhUtXx?A5S0q5 z-9oaNmTpaMy=GMvCFh{h>T{5n{z^jBpImmnf3s=$ufRS2)9;)#-~cv3d^Se&Z`=ma zWzPSwOJR<*iPpj2^-%RI{eM!iOGsP=Y8pV}-V*D9J2o?Xrzkk)%%AUYO!ADz)YIs~ zKNbl88@(qJ0mdRAKi~#r=7_Uf|4^^^@R?kb|Nt15>#j-`4Evd|X~w zwv}OWKUae&JA6t5u}5#W{1NlLw$1f#j}11ew;AGH&OtZy%6_u(3v!EIh>cSK)EWc( z6Ws~7h8n3(x>bsDom3|`zv?)bACIqSddE5NypMgV^(0&VXNPus;M)(jZiu4cr)wpU zJ!QThlNpMR*6aBcFu&OK@|yP5JCFLOdgqspp*>Y#M$5)(mocXy(<7h6hRqOEWs6GC z_)Wa?%;pD(NxfKoKrun?(vWqw5~G^O+wNGfi~Q8v53bQ(E`fpsqql(ZlK*v2w4a}- zuFWsU-D`4#zpy@T*M4qZi6mJnvUlKXoO!Jw1#QLL$64@*%P6~1G=3ldR#gqLlI7Ck zaDGGW68qpsfnwM`&-7}I(>B3$o@?8?W0nJpj4z^(eYB_s& zG@cY-yE&) zw^oQZfwHR6joL7@s6KtKW1*Q~HzZ69C6S2vX%nuDmhBn7DJYw8tO9dt!|4B@E+yGu zbzPr4tW1_jQW~DLKqg#hAnTUWltJ6_=QvMr-M=;O0-~D0$TeH^liv<0Npu9beSp8lb2{KIvb(C7xSoI0~yEU+kTTLd+q@xM&q6Itnh4&p@SG2I5|#~o*ACJ)$Ax;#C0jF zMk9Sew>;$j#R5G|Y@~b8O=zHEe*l^f@C)&N07bN=-wx5g(e={e>5z^9A5`iQI!H|a z1No4}7mkGnNhZS80e{$Hfs*OsRl2gg3CC@FNS@_gUXi>wLxCi&2=0a(UoTWXqswbN zQJt7veb5+cgwYQ}l(~2m>j7pl55Z+&Un<`kMj5RPW=mZVcx1rL@AxS7;wudfJIg5a zb%QpUZGIowMNxSffvjb|fOb`TKfxFIzouf5&-LY=CMP%u|J;hxwGvqL5tQ`WcAq?I zTI`7n8*ozPIGobqqgx+{pY0tEqxeW@%?3M^^2Jmy^2EU#@CdRS2Z)#a#zQ^163u6k`0^lD@;;pVnZ3 zv>GM}?&en)|D?8BJ1YSeA{1Qg{q>vQ{jZ=oAu0KlltKa@)@lgvZdQq{>|V&A@AB<5 ztgX~u{47I5X?rSr@I;*oX&%zDh%)bO75K>fI2@L~dsJg|w#siVvo*eRB0o#$DoV^x z(amc(f{1eT49a79N95`vZJMEp(9mi_WHjv=~$bs%(NH%yVgwR0Xc0?FSTf7pBN} z4=t+aJv-pNZnGTtw1wi^&E|<^;;~rT0W4k7%TjkA+Nh?cMy6VQSs%DDh|GW`EH?t@ zKYpcg=$j2Tf}Up0d=FCX>sWUG=cc0Gb4ZNzZ3(NEv~03+wP#fEkI&MU8Y;iDCLdhe z)g6;jHj%7a-}H8vn{?I@NbDCeb*R`=*`dF~v1m*z)xboKi{;FI&8`esWp85IWS+Lz z^?3@c;h_(QUwKDD8(EeCXI+B5;!ce6t28zohi7D#ehM}{})c}1+Oy+5Fx*d4E8;5PmlSf;CNWe66RPORX!i<6f&9Qv{S#&3XJh|=5vr)WhE=9~A z^q~m4OxVp2jE{RFxCHd%vgvL&D?35?(4(_qj*E9H&m8CLuvUa4tfl5fmkJ-7XqgL^ z^<9;n>31pa{MdX7rXezD&jaS~vo}y9#XGxIPp{T^eE+U1JLN3ayP|ZfAv7UudGFRw z5|(*e;dp&&Go$dxp%~G#MK>?YNx5HDE2HancD$t%Ev?eg*HQ|mzf9L5NfS$xU_R?m z)*5fOwWjKDk^Do zT)5q>cZ&D3w#c+?=I6)D@u~}tCNsi4N-tPfRTp`>qJMgJjyi)`lekv)RNXnI@1y1@ zeC8L^tJFGBv`BuZ$=TvU`0dW99J9^uf=#`_haX0YGk=HN7W`US_CDhJ@i@)5xmz41 z){+wor>!yQh?o$fSW@eeIbEa&$4cTT;JnO@qdUt3irSL z1;CCqBX5i-37-`$i1N|RuJt?STSjn9u5Q)bN!0P-`F*)g+a{m?&4A~m>DK6iKJ^q{ z7SMWITdz-g>rqrJb$v=#JG+kUahQ${lWxoFTa~ijQ0Q+meM7rbm3F&%W$G5%(k@rO ze0a!Qn|J-Y>J96Y9mR~w?$avIr){Cj?Bd@qv~n+zh<2#50~$aV(!tOFK7^TRwYDS)#vy*lLw)6GycBR^^91AAXMYXT@5ERo0eDJcBsAa}eL* zRd-l`e);+OMe4R)pT=sN>co)&KJ2UdS>eLdkXWvXE^cVPvhj;AT4u0Ki&L zFvI?j<=#ukyx%n$xGC%H$nEYpPPtN;&dBf``pYN$Myeimh%!*K^W`(mW&Jp{JDEu9TBTUdxc{b5=!TLh6wXo^aeyEW5Z z&|9FIL}M@`qSac272#jzj+$VJ`9|zkVM*U?W5642D@o=+a3&9(x1h!kIMdT!*;gZ@ zYJz`K*mhQ|aPP2`eI+nrVG)hD3LVoGvC}8frP5=j3IQ`!sb1e5$64uD)TK?Pmgst> zyXuI~TeiPY&MlgTOG7N|ur+T=q{Z(qS_~b$;hP*=m>7QYL8GG8EjIW=oBN7pCBv)R zLA08SF)>)Y6p)3%QKOHmt}9(v2_^rXZ|s*{EN^^?<5e-Q0&6Sp=2(=)JSx)vF*;(C zo(;ZZvF*9*EJIEG^IIIXl}dnoTP%CAQU`6_8!4A~@pReOWY6d)`mII;u|l~E2!I6c zxDhCX9Dvz>a@?eG<)M3gCHl^o?Yr1l*{OwOVQvm2a)xJl{ubGiVezJTh$qb~<<0Km zcO%fHXlliejA~+%l|DO<-KC#_-uSp_v8<1yIJU(R=ogLVwz_<+y$U>czbElIn|({i z+z85O_+eG2Sij2iH9rgqcyP67s0hVuXH)?1Iw~rG3nnJu9AtYcMhi@%Z+mANyi^`} zbZuI8MxyL{{j8B#?sN`pii51R)X%bRpEvwa9D=j%YoJN{{C*6Y7Gstvu4X{=o2c?!)63sGKUjS3|K1*__!Gg^-$*?`{ozhOEbvFz;gYD#n^Oe0CPNmyR){|$`B;U&- zAdbnjK`R4F4kt{N;d>oJEXnd=Tv>NO^o_pZvWNne{UgZwFRd+N8Fkqj*F4F|l9=07h{B&*Pd9*0| zvShi)vd0oD z^6D3n`BcFXad&?hjm&t@$Hc&@eEbc+!0`@~D`mOuxtouL660FoS0$JGnRR8KB^Xa= z_HDi$j`r=p=+8PtMneP0pvrG3Zemr zX7IekM=gXMq=~jpNf)@dY@08Ee;k?YnJZqE^WtaxQO?rBT_#T!6hs?93BI!qsMdPY z(}+hwLqy|R%}8}`AEv;QJA;@n8lQj6q@RN{|JymbQpo>f z*1|$z_v93I^n&R4bmR|Hy$k1q|Kc;LEQ#iQlLb%dyy=+EU*lJ-ELNphcr99uNxr-+ zeVGkTT4H@GK00BSvo}BJdG*spynNAJ*tRpHz`p*jK%_eJyNB28Bc)4fJy%R#u}j=7 zO;_Q>7wKc{BW%V!pW4+{o(g3`uC+YTj@Xnh_LkUDc`UKTTifpItjeUsW*6XCxGtxF zjQ6%9O~iKRn>m_D4E}0(ss+u8>Ak}(o$`tA6NAsUmubCUM#E8@2VnB4X1WHnE?J|( zKBRs&L%C+k2zRqu$6n0O{MulN%Fr)ko)Oy%(W33r7MtR7Pil}O9&1$)2MbBa?QU+j zNL!kLMmKtJ!kd!ucH}io?)bX{FcsWL6wR6zPyp5u$40ozY?;Ic8}hLmLemjV)=*Qj zyd2$4j{u*AD0(z>wC0U;1Ru2{l0Cj-(tK98u&JZa!`ZT5%)Tpho6+ap`_~WboGWav zxZRiz>^2o&Kv1tM1UB6oZqz5wR?zj%?RWIem6VZrt?PSeirRY?)~t!MxE39cpz+2VGKEK z@%A?N4%YjkIayT{R=VPczHAmeAH^$g6DR)5t9(9q@>YC8UbSo^?@E6VlG5QQP!t;) zRC%4NJ!!S2VtPW;)lMkG8&bek1RoU)a{rWL(6%#eYHNR|PsJDi7H!4f9LOKvYHnS_ z+aw-$!&;wIL(!vwZ__5Yo9Tu|prO#nw0m#|3eGUhR7&m=N5Z^pczFH~?qTw-;^ ze9=%%@Rw!8=aR^KxppFVb-#jyDY*Gq@M()tssPj)aQ_s0tFFss&=^LSbv52YPVP$} zZkhi#(lj~n<)fP3_}Vz9FS!BQDZ!O0vG|Eo#Wfs?$zK=^B&-JlGW4+g?fE?^OSP=t zKV!S)-IhN|PAo?|v#vL=lY}GX@sb#nF+M{!c78?=N_`K?r#m8st}bsFlXM|Yum@mU ziW@HIt%_~1T@jIABtOOFjY_yExhhw`i{)eKr1*U_odmtHpeVU@Typg}XQIzDu{thU zY*%x)lJe=p=3EOa$MH$BVRNf>1)=2a=URO=7u**>(Ge=gW+DOAMOMXB}R{RW=0j$XGnkxRai9uO_h z>0E4rv~VVcFWr2qX76q%n~`g|!N&QHt?XN&nAbpR*0r1a)T*SfUCM4xIf{i(%I9Un)VRhXFzl^&kRG@Q}aN(Bc zJRfU{z1^j+t-I1V7jwSnunH3<-R4mCwmFM0d4^>RbCv-!{F08LQw~N=IZGEFa=5s_ zinHsdcq(ZuH5pZV`EUN5dXxBL`v4rl%Z49yUnIkhhD6Y%6>P{V;FL%VrHg5f(IeWIpu z94tJM`rk`BNm-~`)604v+Z;RYyz-u_2uwmd`ahdv@VDa)g`h>Jbo8Y}+3F!>eizb7 zI4K!Iuh)pc!71W<`~CRUdE=NjN=(oNWzDw8$+Xejv?Y^#y)>~M)@@^)g%iguT>*!w z&v!{c^KqBH@v#xWyw?XBH8X`ijKyx&7pG@)$t;cQl6sl!S>%ZA@5UMZnop9Dr=YrU z!Pg=qdh!vN)>9Sapu&~@10tJFB!)#$?(U{!BU>_;8QAeN$b&q3^^U--{KA0mP8{E|GQde~r%qrRNb5%)Bl zMNp+ghM#3m?_=`tX+JWBSLH^?vkkfX-MR4X<^!eNbI!~@-{*awrxh*3ig~S2mi`?ybP7ZBfid*}=*cj71ngbk0L9~Q zg`vAxkrS;|d*>wJ6L!Dvpl<#R1kdT57QkUU4nuk_7^Bq(G9CiUi%RmBve*Uh(0F?0$ z75WSCTDH>qjRZhy34pQnoM5~PeDWOxxSL-DDkhEf#vcj|u<`!BP6HTKSazu00cIZ>I~0W5{lbXJ*hKrOMw$ND~<0e6zvuzQyu zggV9F@a`g~7OCECgp{Em_sI5CNjZPXvD)*;C+X>){P)~iitZ(OZQJiyWM^jQ^1>V) z=MN5P&yQM{)sIp$fu_)d%{yZyyrNIwU;fThQ2u#-0c>atd<%|eV3}7utfl$5O8#MzW=Lu3Lc<}r~>GI}6_~v$6_;T{()G9& zj&3{IG;4hS%qHC?_3t1yz#O>!9DGWZvI3^S(tbgo^}t9Er&C7)XyNW#J^Hl`U$i?J z;xi?l#ZlVuhm2qr`{NqAb*XUfV*HTrGRn}WLtS>_0m5yQrWOS={@wLb3B_ira>~*+ zu`T-tm&PeS+Do&)!wERk!cdI*%jctW_B!7@l6dLzx_ZXZ>!43Q0pqW701U5+{=1(pc39q0|Y!3TS>D&2gpAKE2JR zxpcskgD)*`a}KR^CV@U^!*U?ITbK9;Wvp@mL$PmXANA_a*BF{AdnU3RU1+$YZCuLJ z1pd{uwLXqQ(6KNPc0>^m!zwm?ILrap=Q2Bk(vN{Hok^sw26LXT)QQFl%eq zl#XYr9<5gbg)y?|Dw1z=xa0A;=jqn8gq+PsmEGL>pKnC3Nx$m37Gm=_p9%N=D~1!9 z^&@qZWf^j>Jqx#WTtJ{ZVo{N5gSKi$uHc;AT(WykLp?QPS%rvRg^F7mw34_DtJ-TH zqQJHah|o1$|{`M)*+{dsFR0pPfAk{82qfl9YEMKxvfu>2e|Adptnk zW znWa=WCFWHe0Vnb~3jD(dTEZ8C0<@Q43G3y5U=IVhpr_>hbK`BwayPVtKTdD*~KVraaPXhLAU9z&Bw}3aFeYt~CAmi{iA{2mv zPh_rU(lfA1eQmU@Q1@Y~sJ*4%$I^45Bxn$vqY)y0pPua|Q<5&rItNx9wDpfne1Gqq z18^*hViVc~IcbTGan8$p4E4qFmM7sZ=iJ z=OyO_J`(N2y~B|zZk-?|Y8(3*xqW1ZPFNgt)tNXP7?h)vd)&g4$~jg;qN7@kg$Ibk zk4iQ^9^B!2;H}Yqn>g;(Es#Rdh=Q=c^dv-1A|QX;ZNPPz$Sh`-Plc(lj^-$`Qk%;H zzb(2gTg>s>ki@;kq3)q@RvP;cHAwv0##ab`H*G#T?mC+bt55H#amOo4((<+2B|Kqb zlJL)61W$)9jTyF(BNd|$+?5!Tu6H$MP9x^(j3rgi$MVQbyKnOvUyD1Un0n63 zcT=JDzp6b}+BjC)2jYSdJN14(`dJ%4`^L}V@N;tb zSDOzfEOdXE-0OAG8hfn`tShkYAjw1BL*y@vg{E)rFOLbt(J$Y%_C@5gas1(dZm66_7A1;U=1Xn&zIgN+ZItc)%cM1e z;MxYG%gPN`Q`p&`IZt@F#$c}?F$m^~v_<~>40xRE^hx?_Pm;$^&f049;qsLVH7PnS z;ngRGrS0g<#X&{Y&mN59!;8qHd>_FrbJKQr&bIv=G9RHkN&QOJd&5C)v9^?!QmU>{ zuyGCYJ1Fwotjn2Bgi&o})(yL=dQg`eLw34kuiW)sfN6KBjQ<*All*dGO3^_$GbOl~1KOZ78g1f>SDwqOyAHVUL

WieX_}cX zbtz}~I@ed&A-G1$7I!%;LZqj?4KArm?SA__r&)fHafo;ZO4@F(Iqbn~YT@K}!;`il zeRp4jq!D|*bdFGGq`phGUbK>Itf~8^*<9w8cU%jp0|RTQkeuT}0-egz4217Q{9KQ{ zI%y0s$X5|J+&j3|*+s7s*Sri`+1bQMAq(Jv+~*k*VEU<<_o%?M!dY|v2={fzTD_To z9Qm;3iq-Sn!Cs3_UivO~7IAdbVy0QUcOVgx^(kdX?P=Gk6GO&%#qP(wOIB2R;kWh| zl~+SBJ<{|GyAf?*hFv+re6+h(gof7CH*LcstQ$wY)`0Vw!;XE8?UZtO9OF7?!vil& z^LT-^D^XTjsXqlWJ^{(!j+n?5_k~4EoIm{R>iRaxDE?D7cFqy-Yf)A|t&X?ZcO6)k zUP@84lWLu876i`T(eC-^q_v}uv-6u`oBK-g`BmmcXL7p?8u_ZSw)cWV6BBEzuAUbj z@*LJq+7DlA$iPy(z}$-BfDdS})i_Ik9lvExx5P&QUgW#dzd?EQzM?V%zOIYx`ucgp zG;HqmP@%K_yt%pj?5or;Q^yDWVF!4`#8z&xN#m@|8{pm3ET1BOYUIlpxbI}Rzb8up za89^eKPu!&@3Fbl(&LWH(Igi%qRE>`M|TvQ9OShgpK$j)ZFdR99ag=gXyG{`r%7_& znbsJeDVZtwO2#^(!3Sdd&>C0tMem~h6}*OEbaYU?TZ2ROrZBZIA$*u@|&1R5J*ry{SE`GRS4ZQrez z=>}dzH5zdSPeiZJX|wLIzTIpQ2AJB!T}iP%YVZ8LXg7rGNtQd$?`LXV6 zKvj^xh^Z9tXT9*BapF-QdVf{=4&qi{NI0$`J(v1o)GzVJTZW!3guV>J=;bZgeHLl} zp7e*^-Y!W*mSaMfPmL_V{DMOzD!D-LI>EVT3;;b+&l(Z^z|oh7`(HJs{15&9GJvi9 z7BF4><2S&}(m6l`M4w;@ksp1#uqg%efP0?=V)IzonPZ!nr}wQ!uP2;&TA%dM!mA(v z_CiTSbji+Naj2Msj&_PE3t>b8E0$&WWd>4AEJDkY8c$ zhbB#qaC}dS{%1QBqHMA+xDLPEZ2KZMWcNyF9y_mIFQ1wyiND*@psqRcVtJ&TwvC%#S#|^q$C>==#+2RNtVyF0!7;O!wwzxA zDz_vrMz!4~Y%6Y|N5%ojB*ucE!%kE_Ewt3dU+-c&H;ZRi$dvHlEMjx-;gC(}Bo)rl z;6zQ{v=a?Nd-coXIZJi%;m)ska$aZt6L)=|&NU@GiXM*FSc~HdgLf~8R=x@$y&57D z3TAXXeOSXCj-k)_;@sF$#2t;{F9FYs*Ns*uTh!Eq&%Ur)4HK9lzmC5qZxVGt0`8og z2qZ-cWh}N*zs-1~816r#wVh37G3`2@5;Dkw-q}U0X+6O$%YAmk_LA4)O+~eh(m@KX zlWJ$e8tqnyA2dB6I7MzGsAYJo6#wgjXy_KsQuVJw0e^{tzt}rHEIfSL=|aN~8I8Z0?cqXwhm^0aW%mMTx`xVVDI>LRWzZ}Kh%f##|zG7xj$ zYf?Z-+YFt6qHMQhYetSwVz}|Nk(_N~hI2XG4dhD9$C_koWuGPHg)%}rfSC06KE^dn$I%OH6&D!bvwqPm1 z>=uexGp{zv#zvqIBA?PJx#K4MJa-^jw0fH(?%-Boy;Zkg2`d>!5;;E9xku!GpS-rD zbX)98K)6-QoDI(_6Vn`=RUsij2(wH__}$ltS6NjP%vTPvjr z*Z~D~Rn;mmcHMbsOq{>dZMf_oC+9AUFT#gUERtMchq`7v`3-i#eA`_rQaxURXIB6- zAtsbM;m3)nnxUiCLLcPN77>uv!-}rAJgZ&;dp?>aB;T{c0-A9Ms#p1DTW#F>Jb(BS)i8*3k49*FCyaFF2B(PgSoab6O-0(N&d zS1Vs~-c}?)(`qB|?PG`Jh}vGc^2v zNiYGQ$6x#&|A~)5Yh}&YR~PY_u>--~WoW*6VgtuTV_PycrL8LtxG52l;w1W?mhX;A z@H;n|NhR^Fh1(`c=S7@b%kr|BC`M9T+B1-o0)oIVL;FVn*MUphlBToO!mG`-M&w_v zwTRWEOFp=ANm@Qqo_{q!{bv09LTeWZ5gg$)!>{eTAd!;ir&JNH^{_r%df3C+)AKUH zjc(H0--L0U1DQ|}-2u*BEI17jdc&hP-&Z#_S9$uGxJ)ehT#FR-FXOmgF^7LUidskJ zF2U#jY+d?i?@=!u?ocm%!^s80vCCbLbq}cPjMH6Dfp{~I?;sH~kN@jXI{xqb{hz29 z)jw^F`wvFA+bAc|uVJen^#Vq6Spg%Qs`gP4V1)Yw{-lT%3KCji`A}F4cFM;3uVR(~ zFl6KG{& y6`$HutT~@>+pA;IbN5SalelpSHzqQoB%fRuFtgbs?e|Xy!IuL=s zffoK%8q0|K!TUdUCUrys-~)C8CJm5&G=ds}Uo9tU1;0A}^Bie^cCi2djj0j7;#eM# z=ki?9c`@wdAnhRG7g~}XfHwr740#2^Sb!{YiUNK<`q!w9a@umUAIG?*=1m5f-xhSw zsNWX)e>3Jy{9N4s`9=NT={Nqcxc{E+qF$JKJ3T97d5B=Mwn}t*K2GDyttOb?%1ve* z!Pp1iR!B`j--`Lyh*%MZ9e$(~+6ph7<<}#2Any>&QujmkRO6RmY^iI;MvqAGtlqN9 z6x*d_<5z>1dg5xdh9dSi7z|$uW=BcLmdF-F+}CRIxfU|wGrseVEv$~0JavFia%I0C zbT)>!vBe*Qg2k3$m=&TCPwC%}3(ZY9XP&9P6I{z=4!^ScFlJ!uy0KA=(wRm;NrxLc zb+Y4_~Sh9&7s8DBpmRvL5_NQjzh#Zuf=~gzs24Xu9=f*w?FDl zURg;^_u#Qc`&E`%Rj67%D#8pqsY?pAf#VsBy&dypR**>gW~Q2{z_aSe{87iiTmf{s zQ&N`V*8G*0>vQOJ{SiS=(kluv&P#$q5F_oBU40*pXoQB%OS?F#OE2h$-_pZ*`w)Sq zpkoI99{+%1bo^Nl5CGxx+G%7%Wa|hKcRs$;c=PxgKPejxkLt%KokSnn1a6C5&Lg&G zzqiO8<(W-^*7rhD6}cBV%Eag9>pqSytRpFy)pX%PwDP^A#=Fjh&jBI;;f)sl1H_2G zCr)f$UsCu`;y?!V{Pq$P3VY``;qG5Nvyi1-$^G$)sBA3xR5a+D6aOH_Z8^XOe)=wX zvHbEzN%?kqwYL=Eg*4?)JUeA_6#@yoni?nhaa0#p>Nr5!J}L7%ctzTdz!`xAt)_i@ zZc-`1vgJUSf>3aYLYHb{(M#HA*?|s3$4{nY*Y*YHTN(91PjsM+lZ@E5OhN=k2r(DB zxa>&1v(aqKVX|ivecyP&Va7Mly?aE&lfNnK5kq{3YZQ#EhLpxxfd!>GY;<$HVANw) z&TMADQEF-&ujW9L@{|8G5zeY6OYc!)(vzDCF*(groj~X_Go{(s>rnVSWVl(@&EB!p ziQlUb^03MH)yZSJ13u+swS5tft1Q%!*J>dOygi<-1)s!6T3u0`dl40D>yA~aYhsGC zD^k-3Rk1E-JxoEqgVYC-$*9pTnWJF*yevu!!NXaGqFnd?=bA<=L%JL6E`*+I!8W<}E?pXkn9PiW)Z|wQ zsej%m$p0x&5L$2_=?dR5vK;P&A$V8JX2f$n93{S>W_s9TV~fRuBYS(9RYATI)Js^> zR+LUCiv2FyuY5Zhnl%0`BS0UqwZ#v8XYDTjOqY_#_)ToLhFa`%VuisYze+k{%xH)R3w~2-(FtqiQPL4 z5OOZF9QX>hY0%2%y?S8B=T4j2evptM*t-Bx6j{WG6(Jm2I7Vt>&fL5qR^Au(J09Pv z=(_5qo58+@$Aq$M{|+LQYPYK!7g&nv{5-RLCBdC8wJk!V!K3|pDxJ;L!Mp^eT{{b* z*&KD<@NTnio+&!ZyXN!T<~prun={mmcr=&i!hTVbwA_(I)AFo4oMC+KCGFGFT|ohl z8LH~R>9~TFR86L)7a>K`k8|Ur`7@Eb@6z!<6gWRIG27C#l5>|R8?!d^8KcWY}uH*3Awda^|`cfac5wG)o?J5~J1cI(a; zcQx9|nZcnkj>Y~;(H3`gS>dzFbD!HoSs@f#xV_me{^fhi>)(ca5o0ZqP}w`ko=9M~au2m{gxi;Ac)dL}q!i`%X8%)aOlO-=-az!w9q_-q5O?iNE$96Tk2YlojblH z$o#yw2q%h4t~mPL$(9s|P@)grvz&(EOwaC=*E*y8+KS9-x5A&V-aF%-vJJVj-VYXo zCC6HcG_~B&yElpjX^ysH6kpB^Ai-+Lo`_}#PeIF(1C?k}l6f;{0^L-t3Y+47jr2L; zYVPM#y-8A6jEQk0(hpb@u$Peu@cC%d)-XAGZ^6O1WYDI#9mAZyj@1>D*f&Ow^ZV;w zBHcP`A5imWu5G6Yp^EP@**heR@KfFFqs$q__b)RBr&ozjW?FU^tIh_^b-RXG9p@>V zo!yt61(`PL7?Rj6RgXd+Cp?k%vEd}kc~7*@Jasj*N1FB`|MvN#J6|9LhYg2{z@FZI z568^^9&!q)S~9S>A(2U|iK2OBmy$kyNXj;%$L7uxzINj_{ujyq(qf;xH!Nt63|V;V z8^+>G6r${lx1~-46NnigpYp;4?H#1dhbXNP*U>lkZC{D8P^GT`#vp`^@rZ)dECxJU zoni)2lXYXb4p3d)J9px^ZdcA$gL@&gS5ijx877%+@{e^s!T?0iE>gP?8njcVT z9K-xDK%5W!(LQNzxlecg^%D)(#E7d8C`}MKs&g5(mQ3dE3QJq7%MK+Q zQZ<5U>e4Odr)+DN?kJq3la;6RK4UJRbEQNj1g#)l!WJQF*v-~ze|1NRuc^9(ab;E} zbC)!&d%QDNs{+Om(rX{^G|yjq?l88Qracw-O76PnnOM?QNN+JWMgO`UN^VQMYU&Jq zmCU8ddYxmfK{s}uiW~i5xg>#3K4_#2K_|HLbwc+BWGM`UV70at+l4h=H)A=>^VxgX z{bZ#!BQ=l}YNj$NM}VR=8Qbwcn3JN>mb&Bf-H@ZZuYq>53Kz0!wB%(wK3UfCEA!T+H9q%I_ z#3|f^yBZ$gT+?0{F9C;~DrDN(ACZ`zOPC2-YCi@iyKtr-88aZjq7p$j*^_8^2AH&Z zKISP7S`Mg!pvO_oc;y()VOKde6os?I=e-M$%Xoo~%< zuA3)uV@o!qpdswDe_nN#){DFZo~cCJXRm5k7yLiCt7EAjIhIRB3N~w&%xHQZ8MTJf zKZq7o_7gWAaXh-oY?4B4OV)gQ3pZt^%Qv42XZ188T+U+YTmeQ39r2?c+4Z88@$FB; z$w{=mL+|}gif3qG^sUTjvpTZm7K(Z;zgXgYC;SaeBCf;SOipRXezfkA?Nd$0cN$(- zm}K5dcZ*qOs)BrU!?0uun5QW*BH18wn7BLf0)!3`^kFQ*oWvp{szqk6xd<^R)w;va*Hf4p|fl15t8 z#e0-MBm>m;ZYO=(C7Kd?6no8Ktq#(wo=x>RG3)i4cb}p``+_Qqa)^}LtYb@&iBA>z zAKL52?@#3~!95Bq+muZ&5ko4>Uc}&o!h7i|Y><8(=`PvcZ%s5knP}6QOz{##jAI;P z2iNdzIX-rK3k*KfxiZ&qXv%}PG9Sr`=dZa$V!Ex3Jjj4B8dylU*vce#F=jAFtqnQU z?yU7!bmzz?t0ja{5c2^36|DFkWx59Adyi^%jMU$ws6rt3bShJMcFH-)OYf|vbER`7 zk{jxBX&eU;u5+AGQ{k-muD1^onWHW=4mkOSrDT4sO`dm5-M>|9R5`UnCf@&9JQP>l zki0i|{QmW;_xX`v6z8eBGyjE-+w-yql5>zl7nRTx(VxigSH{kBKyWcsPd zPGCgqo%V%;sO*oIht0YkHEA+7Xgp8)*jd3Xx8U*ZFEsiu9pC{v7jCK^>n~pj1zSW8 zSQXtG?l115!{6wKr`SpJ!Jpl{RtF4PIixqd$@Q|6H}>j>@x5l1(k-%f2q@z<&&o8vff^;IB@{|Et5as8_L9(F4f!j?=-R zh{F#$8m?cD2yL`vi$Y$K+k&%rUg{VGFC4#XrU8Q~xrybaBx|bgxy*jMQDKY_XCJb3 z;&-l1Z512iDW?VEsGp*>zk`%~fJko(AV9qPm(c1z+~Nn~unnpdPcH&6RFD=Xp7bzA z7zi|vge?HR9b$-oG#Mb}A5sYg*!%z?jBBiDD%i)PX#OsM737TqC2ZIdhM@(+eu=RE z{nznFL)c&^*aF3O5I~-5TnR|IT#CS6WId42iAKTD4!Te0ztLs`!P=AazeLCX;rpSP z7*J%>Z1jcJUd0K4R%{5{cTli8X2>C@8H{D1T$DP$xDJGydmaO90YGH;A8wo*egq0j zQ-H-0-v9acf42Vrbm#vDCW2PMu@WP+7RbTL8G)B8kvHNs;?Wt-MJCk7WNKGbpDc^s zabaN@C0EvqpLBBskLTtRWmxeKB*hu`9qDUhi8nTKqW6bx{@&vEd+C&q&dz>uYT)Wy zNjuR3E_bH}6z`19Pd9pFPJz+rFO1e>#XPkK_RD&(4SfS`_f8pt@JFy}jAqe{qoiMo z+;&6Kr8+6qc{aV|Df@I#-FXH;9Z(8B2G}>kSbs>yP_1BD_>x0TS54-V?lz~VL@I9X zRA2}03zzEN&%cU=4C(1KqavPGpBpmc1JwT&wPU+X>eQumA{_g&pmsya2VK$6ZsJL= z1xdiFJoZiwW3Xs2GWSeM2S6u{Tlc~MM~gC+4{x-nt5HspL(D5f5In+XrByx6avJX8 zp8NBBHyF*>LruK0WRYrb9K8#Tp%FR_HKQxWE_U&|wDE)%A;lW~Q)&V?aUwZT6G2jc znoCDSkL9KM3>|=Z`bwq%qA{&fwJ9upQ|ao9n~!=$FZHh{-ltun4S)2_b!Uijizbk@ z)_EX6G* z^t3&xo@lPS>>rg}7i_hGw4Kzx@{s7Q>)b3rxs)u*vv&Bll$=MoU(BD~X?U+?yiy;+ zpNMVQ<$WRG&TC!4@&qJ~v+=vO{V{VZ&E?h$s`7)SEo`Gn&ev!>oaPS)=xX3mQjDAE0P5IX_jbuJ^kf z%mMFan&i0|3v`4|I7CJ=b(3Y6+JRl2ysDoFH<~Go^F7X0&$3xr{n45jI!}47E&dtV zuvw~MnLetRt#^4L^6jtje%IuRzx9QZ>bn<(zs|~H0{}Xk7FrAsc5t+HeM0zb5!UhU z^G&MO-(LNFw`o!Z>HqL#HQm?ZIAuRJ@sC5^fZ4CJg$Y@czNNqofG#m=ZglDU(4A$WSt{``HC?M$ zr2=_Pyb%$v13zsVfwaa@CmfyT4DgU8Ker(o?Lq&SD z;;pgV&o!A|9q_xUVxnf|7OGT(WLry854zN!;PVGh1d46?TIQF`7;l)qttdc0(Zeq( zoK`2AG1029^R)9xa=hcPHR2=E_nyV)0*KV`PkI8s(0bP#Rn>UuH^xDyeiGfgcaJ72 z&AXF76}*WTZGYugDB%Pm)QQA@H^OeT91htu8OHF{j?6;6lH!RhT%wF3;ktGl6j&==4I z-B6v$n`cy27AkZvhw#M(rW~m-rMVoEO7yRPNq&LG2a&GsEAi<9Ai+jMt^WH6q8jIIGgd@HL#DM3O62zu2Oenfm^d zk3E-*Z*T@p~GC4;tbE=bqLkth?-0VI-#tAyJS9PkS7*4FB#@ca_s~e zOF(BSX~g(S3SmE|$ln%>fBi+EIq5@Avt^LD$Nb?Ab`xt|yG;ok0WIfuv)A}t{!Jq9 zZyce2*+}}o`2-E7hbCZePXa#p+wXuQW1gPTbPXyk5|Ab-gU(|cH-WRIDs{jl4xph- z4%Ezd&=H!Iu{*mx7pr()Y5iH1`dGa)>_=x0b^ zaQ055ImW_GTd}E{mQqx(-b&bV#47vrV)+T3Iz|+;Fd zzdl>$cyig-09G;c9prtexO|f#@rQ5>AN9_2O4DpGvLL#8_G0rr*Ea)R+@py*BX_=Z zk_)k)-RmKTzBjr!xtbW+2Qtdv{G4axeK{1*R{fpdheyK%y_P&k(&$@aAF|;y5FKWB zIDf8cfr`8HQ2+x>())p_4ULtXsH=m)3%Vc!c8KG);-#7sfJ2Vy#^|X2_}-3(#v@ne z97c$$-Zt-$>}D>7cudIWJXj42nG`m(89FFiNRTGkLRi1W~nnrl1gKVky^%ZCzEpC~LLZ z7aVuu_xxgQM-2lhg0XHx?F+i}&?nnkle-weT0m%Sc+4Ug_3g97YiM_QaYl)AiWG%_ zs%@B8uav97W6ATw@^?Tcv21A8B|yPuDUuZeGJz7DBYWrqTHT3#E+FqUnV6gkO22J_FuE2wL7@@QzD-N>3bQ^sB8 ziaK@)8{p~rc%6UyG7sMBS^3k`xS^05$Tf>A^g=Z-hMr;zb~;^zIO?7H6WhcXtB7eM zS;E=iI+wlFLiEt$dbCTk){?x~9c0Hmd6JjO+0?a-rOD>$7;xes}`+o)yqN zGAj1x9`)DEhYAsasS#cFg^V!`m({78b!<|-p>v)!A6Xqd+1`p!g!NsW%=-&=Ngv&= zK{4={cmT~dJpF#-V0TLC6j;8RLW!!Jq&Xwo#$1!i2k*6zm}QwN$bWmO5CblEW?@8) z&4HR*5;PO^tI|{(vbDv+n~aH#90Es`gP?ScTJSGaJIze3SN^w z2O{!V>E_XConDf2OcwXB6mcpV&2^KvYI_?^O#~b3dD){5D3$j+*wue&MgPtL$3?|R z!b!Ru~egkIT(vv8vxRu!8h z$lguXiJ#k7k9S-TnbfHZuG4s97G-#6m6BV43YUM{A?MfS=$~GL4$5N`XvXF2kY|R& z9x8NW<|=MZoU%4PJ6u|kpFdrpMqJNZSYQeNKobo@x#CYj%faZlLVpz$O@wfGS@P|= zk+L!&ySnr>bxKXS{f639OleeQQTjyVVzJyf`e1i*ue~?`?D1|K_En^$HFmp8}k=uVRlUO)moTVPcY_iS=|58#uIeZxn{=|HWAHSDw{t+%<^@tt4yz5~*kx6`C8=-~Z&2T*##@ z5|F`PN${^0R3O(z#N$I9Tfko zVd3Y~|5t0HO@+9qck@Jl_@eNS!mrPN)O;;*>ffTj^uRA(m9Mg%IFHOP8dUIi6k|bu zPfqsdLqJpae|IJPBi4z3ZPvH{+P-L#8pYn(m>uemx5P@l(daYo`YnTg*Q&-Gd$^?B zuECVp4SYsl89AG;$9$DjEr#0bzefV+K%pve0YPB~&P=O};?%fyLvy5>49ktKfDc;x zO)ma+vhMuZk{D|A%jH%&@RU=Xf+%t_wprfC_Lw1Zbo8r{uUtFcwQ7h!EGjvLOI3}4 zWHfXdt$UP=q>VW|7gnvR94{^NSK_E}l*!zQX)o9kRVJdYdy%Bcm-p^ad#bH(sd@c2 zni&XarN*d_VBAqOa5$C)BGhxYx*-j|!`m=!iDhMrYfqwUi^CBLN_%jQx`Rw;n~%}c z1ekd`0wu?)?UC7Vvzq3rtWF)_y)1-O^q3Y?H2YJ&7F)z#eJrWzKCdo0BS+SGpv9qu zA2y=Mkctt~lD$5oK3cH@CA7|oSK}`5XIpMkOWR5D5TW<{T z7X`O7tbV&L+(}}uHzxS7Zk8>CzwnLmo#OtNp3utvybx)UA%^0qh{{d$4CdpkfXpl) zcL2u3C&O8*Sws9)m{%I(Qi~R!SC8h6!<}g$4pojQn}AO3G~(A}w^tM{$v$>Ky6#+D z*rRg_RhMxo&(U?W;wrK`FR&gaI^(HmF@xj}EQn@4C1}4ObmN^1X~aA|2C{H&-v%^} zr#wTCR?FvyBhMVR#mfadqsc6KNAvt}6HcrfE@Xk}s6Y8%L(?v`UUa7M1*j{aP7}ws zM%QX;8Y+@beRf@#l?lyds!C#DCK+TrtBm8afEFDbon*>WI+`+~=6Bn(= ztV0>Ys%W~8hWe9E>>7jPokuY;E&RzHM~+KJmAYi&APYuvc9W>yl}=8!e)}q>$w9@- zEd#GQ`K(w&PqGEaqi)(qxDeb%6`2Q5>;LbtN)&5yBQg zB>Xl8S#a4aTS~hXxJ+wrt+9ORjtqroQ(*jFZ_!XXA8+QvsZ&aH>S*<&OogkidL`Ym zBEPfj{jytR(93jvhka2SXJT#}eu_CxJ>o5LG8{Jcsoa#Tq(L0NR+#F$G+3ckJ8VX` z)_m5{iqmq<0ZV`mSkTtlzMV9`cpS|(wpbr+a{OhpLhzGRu1Z+_tTdz2bdV0JEgzkE zCWnpyqch5&2k=x=t{Q@<(z3E$tBJSG5nq!LxoSzBf=U#*Y>dm+&GB@&Zr#P!DdMM_ zqXXa`l`=T-+L`moCYq&gDIp$9fgI0#SL9mYgxJk+E8r?UZnR7OV%lzXRiLVmm#-c{ z1hM!;yT%{yEY{Up?HvqIj40)(6l@{R$3Mlz|psj+c@m~OM=l^cY!xYKCAwtN2I z7)yrcUCP8Ws~9VHDb11vgN&hiPhHt;0-Sb-u6}&r!*LxkJ-ZcBj#x5JuK`-*bU48- zLQm&dM)5NCa|;L+UT2fLS$mqtpR!=0qgk= z-$5+R+$9bO?CaBqeKjk+w}clPlF=L7cib-zU* zV1UUPMq4ia;Wp8C5HZwM1DVoY1#>K&I*(r?v~G+g?30bs<;>Wvpua#~PbMpmca(_K zsXf;F;+WnYjoNhIHq6zl1g32S4b4EKCPdSk6%<+%U3*EoM`E3p6)%tvql|(9>Y^z9 zGquVXOJvw&_$|{L_0MLStMdAU>~9_5dnbw3pWMeAc(sR*{iMJ@TD?6G;Hrfjk6V#p zK1y>2!YRIu4!L;}JmYegCUT_*w?rmU_>zIBuNOt>s=jUd4l6X4xEmi$}T2! z39>J>JVRiE^S8WY9F4Px;*`f`I7j;P>y%PC$K67vK(8PISDZ)s+6LXSNiaA37WN_n zX^`=Gs6OvEn}(8N;v6s1wHzq8d5n4aZoi_y>A5RoNRxszTl9)pzJa~w$iiG(gW~*H zHORTYcfg>PIp@q&s}Rs5=Usj~qiBF}>lCo>iG4~h802?o^yEj=F1y0ZBWWSIjxte| zlYsr&B5zBiCN-D5Tf^Bgq@m|YM%r8z%z-r0_ZZF^AqLHc4|lqdjF(k?`O>{VGX;0! z(G+?cFUp+6d7~pM&#U*HHGF}jCrVB|552r>6v^6^J`p8jVQe!zx2H4x zVza)ks|-6mdurEO5*M_M_;q9G=SI{xpsrKx{S?FR&K2xa;B30~=B;C@Z^}w411!D7 z@o`vOX9lYi-_Ag}UiLG|D{}qwM?R>Da@x*JXT9oSe?h=Z4&TX{*+i&l9Z~Ym%{w=_ zU#1jVLtIvS{PL5wZ)yZGq0`hg(%-7&i@B+|B^z?>Wu94>50Svp-65tpE*p^AotWwn zP0*@3{aW7krk$&W-|>lS(Wu_t3Bao|vvuKwW&6#xscV$D87bMuC{YZF&K!SY`@%y($jwDMn+RUBG4iidDpP4`R zHU%R#d<>aFdnMYuSg&@W6qhWk7Mo54B%o!&-(u#kpk3Bnt8Q&bysTk^6!qRsXtYTZ zEW8AgR3pTHxm=IYq@F0x?^?Q1y6bTcYex^croWHR{P5L)%ze8A3nR;!TrCCNi7+D$4<{=TQsSc058b!Ol+(wlR8ZQy5>p4a&;>M=9X)C7Ww$0Fclc=Ja-*;OWI?b zM8eM4IylcGY8Sx>ZDX`FdeI<#O~O1jKnmbbQVeOW%ji;^Vt^furaWvp69792ybj}n zH0|TG-PlxnvjDs3gpe%-eQ9FS~3kxzmMk2C*3p>D4%IM<89Fa%d~u5SOLV=IDJi zHgik0;yE>p19CpMpaH4x-8Lwf!&|UWUcS@Zid@TdSBNZ5rOqD$ z=Sj2PVk-d}*2%vZDd8YNR8&J9O*mEaEW)mlCdH^Qc<%MO!wn4avEUp;~uAW(MkB>7b&E zKa^X(Jpi94@z-`o0Ud+dl4|^$g%h-8UiL!sNa<2ISEb|=GE1_%QTJb;tUO$d^Bsz5 zA>3dQnY0;q-LnfH*T0lLA?v4mX!w;*m1+miT!rcd?yY!LVj0K)j2cp4i=lep>Lsvm97*so%+&Hs8{VnVDzYN*7Oj6f0{yBfl6e zU*FrQ*Xe!8ejz&}+oct0|5QTz$!O(v0dTL&5GF(U^*AiAqa`CUBEp}$7!1DMLal9A-m%HL zNg;`w54Rk66f$?QEis41%25B)6|I0bki{L`K7rh-QOIixQOWRO^~*eUQEkMD4@jch zZ?ZkULXhu1iV;D(x))4sN-0o{*j0>j_&~1u(J}D*a6ooZY}$n%d))YNxRVD#k71;q zt&Gs}9tHO1@fz1EM+T*eXh9*KvRS!-GMTwGd4(^64g*<5sA-KWp{I<|VaA$XCwYn+lVThe=lGNSa4fsW4mP_m*08hw? z^P+3GM+TSya}&u8Y?HV?b?*<|X}}JQ-q+1z)FkO3WVRq9iZUThe|!CMsV&V(di{Vd z4)n?TA}j&pv&bl~*fGJv=jZu8o-Ez7v{5JfI(d}kb?S>)JFYwS{V6zSHHWnk3kmp8 zd!2?6xf`#i9KED6n$d!goN!)mTt?q(LDLuF)pcYH`$np(fGxf?qqI)u%6azJInxss zo!J&}T3nMPs*bLQt_u_hn`0d0f-#^2=T&0bvnW!RJG2S`>IpTJX_t>b3hWB(>Ko$T z264Y4kKB)LdeUh$qDWjl%L>n?0*VSFFzd2eoRNiGbFDPH*5&M4s3RnI7)5qLA6a^3 z0jG~x&5V2eTlNLCl+tHzwyn9Y9n?z9U= zQoe%E(?rxI_XMY9@DcZyH&lI9eT4rHd+!+*)t0S|79x^!1_33iV46gf$j3`H(-t^$D~Sbnwlz1?l^KD$qMpYHEDcVB~Tf7`b7IYb5)SmJi*xNKtyaD=?8v|dID!7uiZu= zPZm(Z;U^Fu6SBSU(r*R+gYaeA%Hew|ZMwL?um zYY0F+s??r>Ex7~?k>BDPsZS-zDhJ7~Y<7u|%K&rJO}Ku@fKX2?l_JjBiIipizC)wD?06L04`AZ%};<#@K%9*VuZ5m>BmmE0P- zzt79d0{*D^_&rYC-ieU%~7QH2U5=*RF3BR(-m> zgEyoy`ua6>v(Q9=M$I!3Pb;*>XrE0})O}^b-o>RM4?k_6=sa&7#MuhA|H%NFhN8Ty z!Wx>85x*(3eV4zq?S|;9f`>9V1M#&^m?Vk?rQZQoYHxMiE*2)s!?-tsJ1avE^~khI zJBC@v^2pCB?QeO+9;kUD#L>}XW3k3+S;6>IKOQCbB09cNMm?+vLvUW%gsE@D4?##&z!@d2g#iYL^hivFg14al3CF0Sy zbSWVl6vdcuq^X*sw*jp9Z8j*InvW`2#tgt!3c_fpEZ1)pbdc+{z)oLjN+!F3vnZ6v!ih^O zQAI@iR5$X2@D4Pim+9SwXaseeXGSRZ^3-7E+y}*&EH%DCMzWQypGUBpWCOA`^K z=!#A^@eQQ(viT(lw7BDYElE>V?Wsz!Spv(%t3(WD+!g+CuwVPR#APyQ!)im?8hm?G&>(^4P6U(9av5$3OITOImDu*^ zcfh@G_FMK`A`Ant!%26)+-9{(E)>7F$b^I)nqFzowSE)9 z8J6Y8>s{A02>aTPV0zbrKXt_m^9vQLt9?~h4C5scUeEco>X!b+KqjQulq~bU10`I6 zgXV$Mf^i<>Ge^kPhydhOt`!<9Zt&s1EI0iRm9Wx4!%KAB4z76mZaAp1=I82Mnzx>{ z%AzPeFKpkX)4cHUpPWgxJV?MipY#*BMEOc1+8>hY8+Cx4>lpkMdk5&jGN?BfQyFv1 zsJ+&+4K);d(Tq^3hvTNy=@D#`W|dd}D~8HG+|SMA@|fd0fL!4(4S#_MwJc$unSTeg z3iu&}d|HSP(AZR8Gc>!Z%`Y&ZpQrsa3H#4c@^j++bP9hcKWVIFJZjAQ-w~kxvDwZ4 zXT>J}`=BttSA^>*#ex`)U-|azDg2^%hiyHKqn2J4=v_s+GRw7_FAA0;!Zt-7*au2etssVi5{N$0mt~if zHF(^(&3~(U?1Zx8_2R89O5nfGLwlIR4-S&wOwK`7frY&#b(5*6y2fE8=TYP#hSW^ZwiX9(Kz4vq8zrR zk>W|r7x91=z!;H1R-TVYz2Iw#cA*XEq2=;ilIypscHJs%aFFS9HYzh7e4(JX72=xYZL?a73h`d(Bpi>8*%c@;qHa z`logMIh#8D2)7=>HU;xM2H_VAt*LKY-4^-5)|Zn>rf~Oe?L|@u(vKsORgIYaqazx` zWT00fIM;m(A7AZ3!r;&~!te?iTBUj1b^s|dc(uv8Q3ootgty~llXy{=ly78x91c2Q zj@JBW8fDDEAA5^cA@~xhCj#j9EMAH*W~xA_U9pgFpRF@D)>bsM>Sl{*gQtt>9|1Yf zQ?7cCO=%ySVND2LvmF+)pGC`3KbdB!n5@~CLX)2~jUFzehDWD{d!VcG86y$jKHA|c zR*_s!$tCg-KgV#h^Tn_}VoTcYA}) zUkyf;%mQeoiKO^!TKEuLiF6s^65=;wO?K+`Keb!(OqAQX*XhqNUeHGZkG z&jb(H&Q}|Wuin?Ii&`4&IbQz`C|&?>>_#Ax0b#wXyutR6CoPPs(i%~VUQNwXbI(BP zx`vChxCC&yTTYkLeKF>-8ex3etlB(jh%=eV8cAxy3iv@S#}(_lG=cf{YijOf9qb;a zI;lKe&!RuReZbVDbDeBKdU>PJvQ#ezbz2X@n``6V*c_G2bvnc_d6@3j=e!XE?>ZwB zvSQXG`l1_1D{S(XntF*)f*ZaeDXZ2P+*zl6s>z_DY%s$kHFB+kT^F*o7Nj-TK+Arv zbn5{2MiQ*ntSa9zrylm;rAon=mQuD=uKF?`G5wp*o%ipxk$@y78{6S^A&a!|srZ#1 zNQ#D-QHe<2#tK+fG|pe5kJ#?0q!UkEndi*tZ$VS_RZmk2^mj;xBrBW3eTon85L^{;^es2l zF(KBgm%KMr@@KvSEKF=Y6AL#|ytu`L;@qOVqITPMXN>s{F!SPUCj}YLmeUSk2QnBwxgjsI`k*k^n+!ZA)U2XXH#ysrz0&nX zU*R3~vFw5h+uX}Gc7+CuI{-j*^9X>-`x#=cW0A6@CBR7Z*~IIbO5o|hC8tZ`*rG~X z>Gh{Qa=_uT$OO$IS`~pc?N`WFSZi1RpdMi*`OO66&>)ViX)<X6~I7qa@useB~Y8Q)Fn6SCePEA5utDxIsoN84nl0~xD)tt-neo7P^O!7QugjE zjG@;CS1RytgCfofls$9%HoK3WWUU!NB@mx`(M3Sos!hvBioN{;jy<5@<Ic>^+I(i()$8kLV*IQh5P<;3 z<}B|gW8QMB=prHW(3`1kKPn0vTU^7Q_I%DUm?E! zXoXP02vS#1CR?Iv%lx>agc+;0I_S$S+`wBQ?%qXjEUjp0%Xa=X;MqzGE3(o48j|A~ zLNd3Yxv7q{7lq^K*7LeNu`HFx#|3tqKc zv;ZyX+unPdk)L-;FqUJ708We~2+@dTL@hc5{vE*39Jvp*hTFJ-8+{;&l@d;h@B#cz zuJM>ru|fhSf--@kMe-H`MgSGWJSWM(Dj=5$;{(p~b~SS)f9XpCj>{QRf+j4Au>+=5bhDeBZTGt9OJj(bv+gxU(K83= z$lZQyC$mf>uf)j2bC1W_Xd&^Ni?6uvNO8aplR>eKa#-n71i5}9sr=Q%dX86$CIc(;*y8hq^1bu?Gik)( z(YKW(bdI%wiNv>0iZ32T!tMN2729y)N_=I&o_I$@tR%@A9lW$0;O>Uh?Xb-=gmm5u zX!aK-(#@HV^BWh^=oHFYIx!tP@r!l_#rL*ew?H}<*}==Bqw4HQKdA|*RkQ_?Bx#P; zI8hN$M5H&+6!{UiS`%U_AQhGuEDc(xAoeAostweV+T-!ex0=ni*7Nr-C7#i!KNczT zUX&-+e3+#CEfpf0y_0fH<(^D9xi_u8Bngvco*z8AT?_Fytq+{wTR3{z$cW8@%;n&; zMFNRYbFJq?6+?HH)OtPq+9W%TzvZE`T&+YPTeQ+%d}$^^b-qqy%2drB%r;@@p=ppv z+%Sjdk7>th`ZgH`{0;#61v<4;4oyYF=xLanQ6m%P_2W+*twq_w?iu$%a$a{aCFsq( zH~>sB1^Zbng+PrH;Ef|X2+F!*(SX|R6N7A_>WZkTht~}BtFxOz5WOFC3O8wrjCsF( zxbi5!5=E9TUy6U@=k<1lxlQdED_M{#ytW8at&YVtkH-ONxeiO&6i)W&Z@WMjbq<$g zOjYo`M5kNRn%pKX#+8wQD@L|GjhmedV%s^>65^DQ>530IT9G_S-j5ZZvzR;hYC4V= z+(&~Spy9BL?&!dD(v&V~A{W~Sdo@V8Ayl^adK}3(D}N^io>sa7Z(7&B7PdsJ@j^~F z0?XU6Y@k#r4;;wI`7*Zz|4u6z);IKZ@O}5n_V(uHXxoQ9)Xln#m*JSgDA2n2Ea{9xFwL`JsoJFwEjD@YvJnFV`2B+W@+BS#Q%UG#x6z0) zeE;i3e(=|!r<;tf3nhY5qyCML2^=H^Y+qGLNWTf}_h1QPJ5$K_@}%!zp8 z7~6ZTcDGENGqtLAAt3Bd5$m7$>p0OB)EVcb_I`W9#oqzjW(jL2yG+$1S0UimQ@Y9T zfUi}`i27`lTs#`dA$^x|gwXGEhxAI)Ciw#T3Rf}TB>r@v-~Q~KcwRekuSESrE=G{R zgQxON()=b8$fpma$Zg}cefjDx^^pO)q{vuV@p+ZoT@EI8TrYjydY*@<$?1+@Z-xI* z9$AnR_8_KL09!U9i1wz_ec_$;^Nh-OTf?I7_Fb#TEi#fUT`r1qrqd$Zy!2#BjPe;8 zOWjpH&a`c_YsV9NQPOtOyQwBayZROT`(s|S;3Rs}=z}H>&2Mh8$*re2(#^czQ0Q+L zR(|KJN{tq8#iMWAeurN(uC$n5yuCyxyq315kwt(eInB%KOp8gV?*JvV!kfkqCuZXR zJng4RevXo#6X&N>_yhRKTG+|1-w&?;NEtBt=i;&w)jwC90e)HWevrrcQ`y)*`}2=m zX7;~4i&2WZ&wZf$aoq9A>YSus!lQ2=O}_&qTA!hWFtGldkeI5qfVNXMe?@dOJ?|yG zaynVdAeSIDU-hF$vJV)Jn>-tRff}=|?{6|7`;r-H+1vjde6PIy1S4V0{V;NjR|0et+?I>_`$_qM4z9be`|7bE#K zy*akp-0r=xVN(w`F73*J+vm(~9CiR>HD&Nh2Bno0-?(2bMAFF)I3*u?CJ*nsS$LKm zcUQ|Sa$2IefTg}X!Cs096xL}R+?h^T*Z9SyS-RJ8enyqBFd1LvfyDjA*Oet)b8{iT0?hsCe;`>CZ3qr{8N$>2fx-I-|>>-ka(X7w@jB^0G*t?k6dm zIm21QbTFl-16=M@ddrn>r&EpE)qZ<9sI6^7C$Gizkd+tU=vb&Dm5&DYjPZzIw-0q)GTyS=_b^dE%K`$csJY2op^s=Da{S{OWnQD^o4m!k^^0*rE zQes^PN6W~6gqG2jsbj&?WYlgQ5f$s>?3WLsS`a$sNFM9-u3D0~SC(_o97yo6f~}KG zVQCHXpy-&}Yn2!s-y+mM=)l4+&o>2Pjj128^gKK+ag znCqgGerqB-Ne3J0rAsW7oStRBB-=i-_Yx8aPiuLGxyejPpqzRDz=$C(wX=Is4Ng2* z?{a91IOxqPiiqOiaOMdEBiffQe%*9BR&pY~-M zvMn~Rq&=NBGzeLfdOdWL*q&h~=7qq0P7-fQTCG*)`f?H^`zbnR$cR@}H_GEOghxzN zTcW)Lk6cJ?oBaQ9Vf4rcfb74gVu-v+*}z}`X&@%;iM#}sbuVxj`Ic1>xH zl&~gyQ%!?O-_O5&QmeE`IltAkPEbWN+hCL%1--cTuLViNJeF$%REj@R(Y(|=IFKnS zib#9~0uU()bIVDP7(HSu9kdk^ysMyqk&qCCS0O>7aGeeS6S5Co#GJn3EdWg`PcbLS zj6uyCATJPI8J6@puN%wO0Rgm5B{=sKiy6s~{wsd;lJjB&T?6fSa?7dpGnF z+2tOzbI_JGwI}H?@*^t?kr%xx&Eqv77ZoFZvk7jF%4FE)KH2cj?fS>@W_WG(b2Zf! znhvS}ukL@86Y}r112uceftmqU+kXuPddTad(jo z4z(7XwIzJr&vS5aJAr_N^)Y1>RBNs9*1@)knuc+mTIyu`(`PyH8`|#sE|+D6j26}j z&$>JK3ZT`EU@zg#(dxBwq;>|K^yW0$-5~kV7~Zve(kuGjXmoXk3GsKpa?jy+Knyyy z!nugX%Oa$Y515?MZKCPFsi9AhXgNVC^qIg*rG)%O%F(k3Zt9MPSU&gbs6nDjHKkgZ z&;k;XOK9;Xov9iPR1?f^b6#o_T_B8okr9mY@Y%hJj%)p!-q6sRKx;%c%Ewz8T|rLY zijpZWwaWhH-i2n~s`nZB4u~T@ktV)6x|}{|V*A^!G|=q#R<{5M@?qbT!xqR&szVI&j*y6E8?ucbCC`ecb5 z{dHE(q*02}6Ias51BWJS-U)xxk-`9-F- z>!g&zuMC-Sen#AYzLQ6&eDqzVn|OW)u>7OX{LYV${O!*t{%?Nb%}FcN?}SILBbB~J zlL}XxsQGxgq4UG;actDjgg(hC##XkyzFAPNo>ea>b+;=(H(nGBrw84{NoFSgeM#>mX6osX&=aVJnM%w?))5SKd0MIpYzl4{;>K$jGOHT zC;T?wnVg-pA{RW^{_acvADf;Zt>^uVruTp9gN1(6TVs*_<`n)NAQM+M`B#5JI$_fh zS31jck|LX(bs@kqwisjMK1=5U1*?#hnC9k>8W^<}~Q|NUwS4V$!(hDP*^B1?m+{WYFAii;BfQ&2z2cD$) zcB;fHBV4<`MdmigzBlT1wq)ok7(UnwWHYeDl6gzZnJBRuHQ90pblK&50}=&?>Vo1% zo}&weFbEG)obB1Bimypf<4bwZ5Q*T2=;mbB&j8ji_tAZ9OL2J1fW4w`s1kix7J%ZsB4vlu6>H9U z$M>eD`m633Li_!d&Ywrl863nA7u_k3x?)*fqMos#^R1fZQ&6s81Uz z9k_c}qPmHigNBQKz2E;c=T-)`6gzUf=OVB>Kg=aB2)ioaba989S7UrxJ0_N`((9{+ z;H`TDtY+BP(~6@WO1=37(EZPUj+Vka15s)AD{kWl=5?Zsx5sbAbnT_U;ta(^#so_4 z--3I6!7kJ=)>$1PN489Nc~S>J@VH)03WIp<=~gwB%)n9j*O+xmLaSg4+(=aI`trpTxY_`^T`{XbE1x)P^de=jFIiTgZq1=V{XXC#Anbgk zD}w@Jn{RV*ke%I`H?-1Y8PH$tQ?*?-hHDl#n{djU1c1s4Pz-ELRFMD#!omoWSRmP*X6vEvaYm{#ibxi)d^w6 zuPIar3;zNeF|Ro}ap9a(YkK1Q60HzDGMP?m&39OA7gOe&J!+xJ*uhua+0aK+;BvJU zWNk{J>6oy1cf;}1nOj;_$4>;Lrig_io{Qi-5uw1`^W^M-P_IhUA)9(G){CJ~YZSw1 z%ja~%(I>*xi3dISrO`>xqx-ELW@Xf#+|^X8`tq;&C`xS5AVPfoDfPPks9MHa-ksK+F0JOvK=LWus8zrfQYc|KII$nP^WRI_mi$UE_ zRufkAiTdW9n>wG6B2arAZ~ciYto$AK#-M1PP}{{eN(g`ek7K^+Hu4p|u=MgdF15F$!}! zeceL~EGy&McytT8!&Vn33SEAgW%X>}rwV=@QvH^TR%VF%XeGsxTHrkfvJIL=#nGBw*N-gzG8HqFmN7R}OOKsEhL(oj zj7Sex7zN_-#p|F8@dEqj_)Q65YPS_C@#yN%$5K|$wRo$Y?{t#kVthb^r9Pe3cN|F9 zO;OoLU);47l1bBldGeqf2(NCN@^lN&ot;s-F2&enyUVUp9QcvPurhY>B}WQxN;71X zg)fGmh(^cuM>{e>_1&+&$$ba-lq0`j-*tsAwNq!n-}~e%D@X}NzM339yRl8iR9(Qz zHyeV)5}ZGII7%qIvBAeGAj#cqXfLMMiC={%ijUlB&t9* zW4ijfV>d@!9oY|5B7XWZB=CQ!&e;r@umPqS%p8e z{cGE!11UeY{VO|RLGLB~?C}2`HNP=wnQ3=A@fQjEJoZ@Fee=Y{yV><>%rGxKO{=FG zV+QVM`g6~hfxzE2`=+}ewgj$Uj)4`p(qx1(bxe(5q}YbgoxZ<`72nN{uuWbgAV4*E ztS%GjxLk~eL;+p8gBDj+qUAuNYg(w0;K5lOjG%ds@?!!~2{0yu&hI@ZCJE&6sdpVB zy@e+`xqnJjYaoH-MtSaMI9?IV#6W}Su2{lB!H2FP)QV3-bZY5Jree%kImew?8ceC}3&FOs#y%~&3~yD{IT&5 z7x~%Y@0t{o97WQKpC+g~YBoM*z(k@X0fv9pf1GykoGbTj{F^om|2N_o&#!L7FFO`* zTpA#KaUqxA0bFX5lat;1U7J^&aA2~=qqkqbx61wWw7)l{{wRwV z!txJiQS&aFm8R0!W2xb2=Bn(WN&}|5EXjZqPtIXpDef!D z@e?0YS;!=$sg*eI68<8!*A7pCIru$CBJI@knWn0>n9P_qmvkD0#oJYUTwx~Sd7)9+ zyFprUw7~T+EsQWFg8n!b;0uR-BD7j@CQ;PyP!|7^{rmeL&KN{+7@EA~%vYCirBCgX2nLvr&vxb>KV^PxB z7-Fp~$Us*strR^-R>z09OYC!PQz)|sKf#d#O-e|jl4)eNLOhc6gP55(3x;xf+@yzm z?8aCteHQU*ctn}r?=PoDym58WiPraaM_LwbPeX$O3tO^JIcZ%1vHJ5t$v6%SpB6}t z-30H|M?1>jcOYc;c0;-n2f0X-?^0J=$L!Yi9NfvRX%Ofod!fWyP)QZV%7yW&{L?b-ok2s|ChE7#Ra^^_r3S}yF?i?9-_Ii{^y^6pt9$W z%?$Z})tf^Odr|ZV^Ml%zLu%yR*JZAfI1BQ?THIVPIFY_f%ZWAYLD=Vru;=HNEKZ0f z;jP#oiw81^GrT3?;BH2`Cb;~k>!aSH!FI$wmFK6H&rBFwU7}-b#4fy`v)ZRK|ch}3x`?FJc1E$QNJCI|2d2MyO=nm%E zKGU*$7*-N7cb|j`VCZvuXWRrn^l5$7@T74X4 zpdlK1@_ynr*GR?VtO?4bH4e91c7T`T>GgfcaQJwy7%CrDtVXDm&%o5{=|boBiF++B zcQPhKg}l!$lp>U8E9YDznXig)Ry@*&Tn+##St6`uA8WZi1JR~NmqR7mkS@&0x{cE$ z)9k(8uY0;h_1Ne1Kr5!wus9;M)Xw`Jd?ehPJIi<$RCTt0!?rGQY-vA!hrNHMC+yax zdB1`X($$?{2ZsDuf8Doy#U~8mcSG)pS++xI1u!j%8 zD8ZpZmx}A-(kpc15H%0jZT>|2y3gQNE4Me@>mIgMDgm+{!A@`Z@*iBtgaOWXWsr|b ztuKiEYrLAmwd{$k8iPK%A*jW3AYrV8R2uF)YiwYk=_%1X7nliqZ7nxlOG>rYd(olEP8;)y6Sjw8U{F)27u@`+Z*#kc^E#4H0DgP z&pbBcs?^stfWEvM8cG}Gr*#)sWUEs3H2)W~f`-yv88`#0<=1z7X=)p(baXgxCXOe} zR~pWJu5R1QzFzb>Oc9&al5$?-A)~@Fda{h!)0td7MC|C!F81ZO=jqRGANRQuI4ZCV zaSd@*v~{n$sZ+31sylV1danPA_w-u_Xw3VQjq0~H`8WT4!}Rgdj=&*4*5P?MWQMg9h_;yAaRC{loO=|CpWB^0Zux6}Wg%$#3=~uY~N1lAaI{X)UFPauR4YxQAb@m|u`Z>K`#_B!fc`i)JR$Fi_Y1R@Qn(y;NXT6bOQUlUgD3x;sbQ%4xN}NBB{V%XfzDI$sHH@mYA#BD_ zIi|d3O_!vUj$iy|Da)F@#-O_mHs3+ z=%k!v`;@NsV2OFmt01dB$)jUG@&o@oh~{QG7QltcJOky&c9o5$KfUwg<#env@l1Z_ znD)Y$kTvUzqRcp@Hh?SZm;$S_e7wvG8oi+R^bvN`HFV;1YIMTC+=%)!(B+l&pU^MAmRn$d|6|h#{hQx; zep{jP*F^=8V3=0IDR{LN6X_;tjFt(jfQUsTDZO+$Ep@mRNK2Y7Km+KNzKdPw@YKgs z`Wy%SLS%?eKz+$6uEBtpzq>MD4MwK@M2d+LPNl2o zRQDOEilV57K#s` zl5VpremP*9jT&!oy0+{|9qRqavJe^H(@KJ(e7c{lztw1CVx#Z2V(;vXb)EppkHFyU zwB7KC2yD@L&Qa@A2EfF8bJq%UmD{#wAA#L<5t;N(Ld*SIsB_sCm9=Z$eATBQT@Kqo zaDtLinU1Z1AIW|3Bo@J5MzI$7VRxkT!%<;_`rL8XNK6$z<6UxE_L?RytBpAB`!XFc zc0EO};!tpC6=glnk^=zHwv5LonEn+5AsS%jhxf=j3*`Q+(MJJAb)($g%U1I182J6F zX7a2#nK-3hjKba5zHk)$z|J;tR03F-fxkoD!iezAE#4IX@#?l9ZVE1-^#ROnzI2m@ zzakHRp#4mM>uwFD(Tq4I)0cZsFygTF0dVPHsta)Nm51|6-$~QAS5s$?Dkd^6W!&n` znmqENbRxhPwJ}(Ow(m5vt!RZ0YyQQ}Hz=uBRp-f!tl^9mGJ zRY${v4*~A%&2{L#q&-Z!OAQMYblj<90#79rZA-$`Flg3Mp74Gr_6{Gn4ji|0;~YaF z;q-uhR!dQmIJMVxHJaQC?3q{$GctMFIC_O|Ut|!%!#FYE#@2FVSH_~A+0;u$%kr`^ zwDT^s{xau{JkAg|z?%@k*#k|M4mnX(>$<25h0(+-0m*uL^x~jioFPCc%;c^@61%ze zloI_$sD?vl#F@6#Z6t7|I3skC3R%Dc)X|%=tiL4a^jR!67vks6m|49%3X~(g6Ok6j z;xj(vg=&QlbhNNfJ}oZ6Z+IMuIrJI~#De@xkxr_BR6l<%h3o@4Z$7&M`k*Y--JN|4h_DvFndcQz|k(@my$9-O&>hleHPVp z@lc=;w;1APZb!$^<@T*(s5%C{pNr`AzRUhF05Z5o5MJ>gSTl092U{Z^uEzD=`FizAHv9&c>J_wyjC!`_hzH4 zP_mbiLHE6pwc$38H5M7hhuItS(g7`LT3Yna0g_mFMPZZxbuo;wH&p*^D&xONb)c)t zvmnNAgk1iVDBLCgOZxCv^se+DoEhRjyqexHRYuqd>;9#LxNgOLXz4FZlnx zFK|T@r@w4L$7(&K58t9FcWaX#-Xx45Z+~!{ZFP3HXIb8Zv9?s_MC}Cb*TiwG|LND9 zG}9lP58^+(od0HX91XJ%_cmHtqDJzn!qY-0%_rAdBlqdd>w;t{7+=*?Q#gfu;J8bQIF1?+)N*C)|Af z8QQPF!IB3+OUp+$dTz(rcuvK#r&d^d*6xLvlL;(rCEb9Y4gw7Yi}(2}q#@Y1x~0rVsHTPQiIbs_7or~gES7U;LbB`x`UMb zC8Mork&Ag-TaTcn>p{4rp|qL=;tB8J^0-EVrvdSuEjODd%4=z^WM4X?W$(@=>OHb&OvfT`m;TL)`; zlV}t_MNvdYe_*v4RALL=D6o;f@s+bUd~QMT`Gk;HaoTfRbrmc4#xSxCxB;TlH8ZNt z7EO-hET*ymkS7CH?9rNg`y#vmYevb+n(?8!OnWMm+Fg;ln4CH?n%uBOZNk$eH-eXU z{23lvxDYgSB0r&(0fEbYj2nm1+RO4Ki~#&b1k z)f0irpH8*lZ#P1%M|eX{KJdOS<=HcbD6M#<6x&YgaYX4PQsXwnyy#%LBlY}VJtncy zw=@af(rwKx&az!Pw~roq(CwtU)9Qzz9t`h&pOf|L-+0MfKzm2S03eV+lS!Ouh30bm zEraPl`ZwpWFUu~p&L2D2L@DF5=*32wrD%U?3+~z+vv=DnOMN>-NV$9&I--GJhZs%( z=8Y6AY-{Q*w0mI<#lBO$Ikn$_a|}xs5S*Y8!bn(+rr%{`nsmt^1nI~b`58`TH`FV# zHc~k%0Ikqcz=oF z%4NdEa>|v77->}7jNREY#yK|%s!*(6?4>BEY#XgRQQVBod9|bCjyEIB@NS6=b=!uK zYsy;^7N1$>U#yn5-3UTgi;US<7Gc|N#BrpaS5aa1$4SQSv#_omGm%K$K*o2`)lQ|X zY>GfdiW+U^Kf7{-iHeAm52cR+=Y|qYBOiOQM>65D(r9F;Pe~DM>!%kNFV5OR%c5Vx z?p}xHrI~YS%o!`Q=zbg!QWZj1{*|OZi9H064=Vvi5X3!%3Xi8w?%aZ(&AIRr&FevJ z>B1I@==I8`IHU}3Nc6|#j6x!eaIgbk;gqJRg*h%5Y#&DomCs!%d9CbuG(cakH$;q_ zc@Um@@2%3eAbPo`rHHKbH$&K_44zGJl^mMaq+dP($FWs4C-z$T=V^!L(2!|J^AjaI3oyJp zuqVxq)#ckBwPI{*vKBRW?G3zr2GG}z=;hMV+MsCLJY=~+Wy9!R%9BeoGX2t_?x_8~ z>w~~B)E0pP^F8GU;%8)Cm7g z4H;<2(e=$s6W-ccPKFGZPsaC=eL8-?0ghYq#~X}|OR4rht9;{`&e>Gv-rYhy2C-ck z_zu9geL0kv&6K#o!T-7bc0w$=Os!2Iy$cU;uJ#1kx=Jri0D{35k5C&vB805gyHBUP9(SLdF(&OHQBkL6mGRR9gyxuCA02 z9sUQ&iX5SVCF(t=G83uLislAxjQ2J5D}2i3FYx(RGP z<+!o3Z&Xk z!$9==$1j|$KeUbBTM$S2yh5X0sITt)id6ZrvGiZA@++_PkBxu0$j=Ucn)ELt?B}5Q zIS+q&n*R^^(kUagZz*W(dWp$d2O7)r=phX>{{kt!5Px2(UVQQAUw_4W5P$zkHt=s- z$tzFP$o02 z5<+D7uXM|NFJaT2ky3udvZ5icjl7=;w&xEp460EK(Fw-g6d7wUdK2)-?fK;(mUlyz zHzzXCi>iK+1AcM|U9f6&&6>lHWbA+BY6{}Ox2RKE%TfQf>*!$MCh}RGzid|-o*%up ze{~ne?fmHSc_T+C&!KrZ@EzS-T_asp8y=p(Chp>=amsX$$jLruzA*%55*Ko`RMpjk zYZ~gK?jF*{;Vg7ON{WmE?ml=$L3PUoke(2;$OF$ZkoCNnjGLS9IhMQK5W*TOzf0d( zw@0btqNxC=dfP{fun)bcNtgO^n+b1ezlD30NAsecElS0FfKz|)N&LM#U4ow-Q&V3! z8(T zFhJP-1!NY|7_M!#L{PUwGXf=;Se9&(RLB*`rJd1h6jrIY+I4C`tUUMqQlD~CwsvMZ&R zJc+!wD+Jq}DGGQvV&qAxDrpT9eSM#6o4(1J4hQ+Qrwfl=aeJ*OdgGcwhm&5G$lj>+ z;Vwx;lM5}vg8L(CN_aj5)$1h+)3h!0+}|Y*k|yVQE%DObE8AFc+5{dpuV=8-m#ZLj zm~BI5XxLSlzgHK|AE-MJpKy=vDPAV_6z*!;4b~}MSCuB9w8|JA$@I9<+_$tbkfxSX zhm0u`1mNMX;xLeK@5!031Y9yG1z55K3V&KReyBcx)s^IsBhZukyh>4*^A1mDjA9>V zefbRpwf~igW4VnlS9s@80W?3PX2Hm%E^$V39U`VY+ZcJ|A*0A4J|SL|7#{FQI}a(i zVuz;ez?BXbJ|qQs(u?DmBe*wbj4~61rY1J(_~hQbDj6>3Y>0L$79^bMc_@_?WM%hK zCEQ)_Re~(b|6%XFy zS^A%hs^jZ6iyt@!M7wb4tdhS5@<6;b*L#*)b&odQ4dl%b#l`N!|3vl2v2n!=- zauL+r8mk_wTa}{Rkf|vW)?`d-WE(hsvP!|fe0!jtFY4hLx)I#Y?<bbv(4QT3VO&mjQx`b_SDT|&ulg!z%$G{2d)W~g;qT*#bEJ;Ii z0(S4uo>4pvU$*H)X?3RxhBxzhY}TN31GF^TXvNX+4kv2T_{E~Ws#qTFF1qYQS86;v zd=Q0w1I8*7h8jhvJ zYRmpzqs`@S$2vtrgjj`y;oe%y)Man0+b9IX)2mw)dT7s%Qa?A;9axMXm#!J8HIK*) z8r2hj%c7&l-*`>o0*e4sbjM^DC@srLFEigopo}au=Y{Dkc^k|-D&TZ4?Ia1=si{-C zDZ*14$>WzTm*wQiWX$n)b%#7iU)adfjaFIa=p9r>#m(YdHw$*a2#-tHI~VaLW>mvE zwtG9zN;3;9tHB(ng+wH5D)P_mv@3Y)uZqO?c#lx}3t~>Ha4f@`V(9)+C()gI z)tczx=y0!uV*&O90T6yO;oueh_X`KFDJps+n#DcJuqr(zwjPdJKXBN~iG!~V7t<&l zJ1>hgrSA>1q!ln(Goc4E_7=9vWQwmhy+A!*)yapwSM;zQl5h5HE=RkhtLUz4L}D`W z73$q;kj_6KY+y(j>93jNEz#K+9w|0UYRKTA$(O6&6erD7u=X*ATHjUuYTeFoD<9TO zACuCFGE*EQeW~L@)?M%(6}?gYQGi`lQqst_BQf~3F9Qe`BF|=q%{FL7qfX)jkSH6i zFp~VrNqiQF*|QYmb(Okg+gS~j?QimHR-ISkv_2cX^A_*y2H2##pi0ZEZ>y#J#T-%% zT9cr5Zv4$N!jh@#8%aael|AhEUsNRaB(2#L=}%JsVW9r^fIhE44X?m!p_+dS^ZlO@ zzdx$}|F5~d0D#5c3E)@hV+tNLUThAykqOcoiIf6>iQKzwe*%X2{u@YjaLoF1pZ4O+ z=~7Dde|X!IMAR5tg2r0|qe3AGKw2-0jF|TpY9AM5uEnwm67>;#?iH2l2vtNPjTAL&e1F@fZU zK+3=C-v5MkQGi3czf}vDW$4$(IwQ)NXy#_e5vG>3CcZ;ff6C_W5M$x(u3%exvUfF*@?f?I}D<_4X%?*#|+$@eJ zafsu+s#8wR$V`4+QZ5>#_>Xng-!xHzaFqPa35AsWxZcn~9fioi@WPxgYJ`3(wc;-Z zy8-WUvzhaR2IK97I}f4?zj^FETRABC3aHqX-8Mnxs3YT;gekvwer{)C0DVtD;yb)> zw`qsdmlL59MBZSH=<~NTq+F0=Uybd?7(>#|pLe_O@~7(w1*CDyj*fIX&b=9tqsX~P zMO^xDT^w+qi?!hnw9Wx-OlIW3f@=Vu`f>q^--AN@x$d-evLLrw$KCnue3w`|%TcDK z>8XZPv;Nd^eUK_0M`57rWmJ8M7SG7wL9J>VW9^a+vqrjG;3Y-8 z1+UoN>`75=|Ff-_9}-mW?(+vjIiHd{t7->qUM>3@Hj?{l%706%{^9%_{Aph7ZGG&)EN6AUX7Zup?YD&<^Bh+5x9EqLo8QbZO+{EmUe5r^ zH<_MSiBMAOlP0Jyfw-^M{nc*KTWYIlOjhG}WKUHK68S1=t#kYP%`1JRQuN!>24DYs zhTs3aK~(--U0acN(pT?qzWKlXO6%OC8Pw@`3OEv+LCop}s8Z-Y|8i4x5?7FiQOu*~ zeVd(ZN#Xc6`q)eUTMX*ow6=e{G5({DMg#Fz;Xl3N%)k1I4Z5v7$U}t7``F_YbN}%{ z_qPYE?t(eoxWm!OTEknvK#(0@+c;Dw@lApn-sC5P|BB!LMNbD$-v3|FCcT99OOaqc z)yvsBfZNmbAOOCxH!^68XZE}8$e(Yyc~DeW{`wY_2c^V0)GVw_TKQF7ikG1Z^z5~9 zC}AP!RyxFa&!gyg%(hE*n6#L3F zw$~=t&UC7$2w8UXnRnjzAM?gr)1Kw>;ZQf>nlHIsjUmxg$(Q36Hx;SVh zMC;m~fx}Ze&wRIhIm)#xZTat<%YB+2Y9zY-&ap4!?ixLa$A_-?ZQv~&f89)}te(>n zib2kPPyKG{Pxo^d501(njN1Nv&D{~Em?-sr!{^>F<0wYb1(zjIVgblBGM!6~4vr~# z#{AeT4E!0JrOzKJytEMiaQ9{~ukc4a!M#*58I=@3qR2Kg=JZlHVWWMlCL=RztCgWG zVX!sA>7}pOE0a>Pp`Lsng756t?mvLeL}>5v$MGKVDByYDrMpuT4m`RNdFDkt1Ah*i zp5`~!m)f*f5yXV=<)0EV8G8JdQ4`jfQIxaO7#nhwR-*CX(YvP&$vuRr!;-DbbA5fL zoLb21510$^8vpaEH-DeC^UroiMi7SO-G14-bQa1%Dts!VIld>WBo_|J%+ogl=J13YD$9NH!^ycGH;rThoU=|RfGIR ztOz?$uukdT;L>Ad^whS+rkf(HK$Td-zEUOAs*lA1^ZwzYsS{t!(q-A=k1ZC?2?6(X z`ySpM#@8?EYZ{|DBPN$n?$GbwFztP%5H+j`dH_%io(Xo>BC-=_Ah=W>&LoJs0oQ6O zOC!!w9SQh0O4zMH-zx8Yjy#PS@9dQ4Xq@J67V$u~DCpHm2Yw`bU-ngl_`9PcXS(BR zs`slyb{g?z$yfw|7j@q#&JKfzoI35|H-#~3`Z=~yA4T8Y=Iw*}H$la*Klt`D`REeV z)(_hI4CWe~cXtu43%J`@%>oO!z0oD{n1JIEmaV2X)m(aZHH^eHk1pFfDoO2op)x+f z?U!^VWU9!z&K3)(^@;(yHN{8p?Q$fu&U$5#N^2y=vAU}X(MLV3(yE|?Herti2X+dh z^(?l>8At$UYt$6!h5aahcX#627x%vE4+31_Pt6XJXK3%a4i0kgEgCyZlLmm%T0IKr zOdKzI<6W9A)bg0b)dB@6kC?_Wy4 zQOz5nyR3niHy4n!9w`>VVe8)#IxVJe$jlxJ&XK@G&9{kn`8B*Hmp&u-dc)VbRt z#gj$7OE-d~GL&yT3RL?jlKgT4+t#B%dC3tQj-cd|b&?N7wm_Dl-C?mO#{7>|9|sL@ z!-*DhqNBdpmE^|bhh+0@d#=4?NAYVcAwt;EA;2O$?t>tm3(BabKcu ziFbb^U4ZQ(F_2}}2!9`ikXG&;1Y}C^sNvWQ0`Y?oRTpMOZn2#|O)2;>xQhmDmr#%U z%%wR$$Kww$q3vt~DK5{-1`vi-_u;g7b^~-AkhFtMo=futhu7y0F7`gT=b|?{5g>q< z^%uwhof1~PKy#`OJ$0Kk>4frk+DIj>7zntH3~I@a60Xtqws0v>U5f7{Us6^}#Fk-g zo`&}%EL>zbk*WLS89z#qj3({jo(&eCCib$dAM!iy?P-fKVr3EI7kOd>Cfk(fDM#;4 zi_}IyA^f>rl%gpVoI{ho!RCEd!r$V$UV0F}F?k0)|9aRSNIq#f4Lv3!?pPPRfx4Vj zhraM=n`P)~vm?#Xd^@G>G)zUX2`WBB&mrf$GY~yP$N)ZU;!a6OX$}f+)$|z~ zY#e&AQ5+Cl(eU43?G=n+xs7FC(e&!scPh4P?0sWBvt+I!NHIVvVY$p=-V$BY-8C^Q z+@~lJmtypwR7rg7EFvG^#U+njh9DU1SQsW8niUIOJPO)vNWBs_%F?6@;tylIZkZ&r zUT>wc^Gxjy`?%CXGd|O$7?w|E>QSlc&&V0t@%0KfM}kuDr%Y4O&%B-G-mU?sXejy) zBN|?WO<;ZsaFWn0wHOm+i04_9twev$~%s}4RHI?i-?o_ibg#hQ%B-p?X#4(vI@v(!6#E<=?KTh+R8Bc zy}I9TpH{9|jmW;DUXWtLtDTm#AzibHH*Y2ixuzi@7_Uwg zMUMncis@_HyjshKm8E`GJw{e7rAf~$@*;H+n2p)HwCc7bBZwA5h{^R8&(Xmv$L&3C z6S=mN;?tuhbG;ALMgWTlU zw@SL6PfFR3VIq%?!WZYgHP=~IQ8%BqR@ zU3AVhc*8J(3p}%fHRNnM-6;pcLNex53>C0y30n-1hO1>aroS7IW>+zI6(1q6 zJ&PhPWksS3L5+=BTEOn8VIi!cI58y3VtIU9Mq^Iy@s6Ph3o?58WWkPJIZUD^k#o-F zg?N#TcgjkBH=_b0HoVI-gSWc661@>4IzrUFbW;5$udTWnV!XI9@?7B2umFp3?F~vb zv_Z^nKwB|R3hTElXY^#oUlAB9dXqp2fArAMdl4=BD}9uGo2K_E6z(w{Dvxw#hRHBe;{-nB^#ow-VA4G&l_Km|Uiy|}+n~mx6mf!D|2$X&IG`Pz` zwh^85X($iwBh48lo!edC7@rOc5Hfy5v>v!G5cZb8D!?jzh>&8lg)1?o%=v#2C(XHUWW{ka-;VlFAZ6L<29AC*kOv#yE>Bj4b@4#Lu5X2kl0ZTlgU$j} zR`I7#UXVOb{>q_#|62mx<8RQiDU)V?WLhkFT?p4W-`c1JMP+TqG$u{SQ3b*M&1gVq zCA!HaMo4=RE7KvJv^9CmOy0AN0fY;p#^)X#Fw(2p@z+%*t~P+Gec)>NG2D2E2dvSV zOw}+l3#?67BjUt-@$+&3iE0(*+80r&yJ2o7-DS~t6K3y^kGORQ5vgEX7Rkafme>s@cUlKr4Y!`f z`p1$uqzx0KhT{2M^a6{Yg@ILRh-2#lh%nKxGh>0N&7F@%mR$g(cF z_FVRJ7-tx5nVc=&GPB4EVGry^e&_ym$fZyPn(X+vEJ9H5_#^xH{y1(kMMn{zF31OU zC$3xYqi1}71F>-SU17PI6Tt>Q#4HSXu^Or|V#N`X#ikC4P=_;q^0i4lG1o#mb@D@0 zCuC)cy;ap($E%G@`1rZX>k<~nX0jD;v8nZq5IiW&2PE)d6kDU4XJkpe#{T!dj^F^=Z0+~jHk(ne5nKZBtFfg{0+Td?`Q`Q7i)?g!iJ_s+?7@uLE zGW#-lpjrWfY{P0OAbAhkd`_D`QSc4};mZ%lCtDf5Gg+foz|n|m!th{!bu19hjL+Q^ zVx_QWpzhppvkLPNA-Dsc1Y;-VEykglCQ(LlB za`Uw7N#BNUre*GXmmecaW{%XNM)EIY9ga1+QSMPV` zV>NPj?f74qSfZ;>DxV>W*zYck^;^lco)S#u$8vcJEt{&wUh?B)v2LYvwwL$XW{v?G zu)soq%GGj0ej>js#VBU{kpP^;Dq-ilBExV8)gr$AwMnNoXdtznI>vf`g*+GLR%k!; zT(pVqE@eq;y6s@)lep_EM%Jm1$oF|0SUo(;2qD_om*_deaFeXR3Z z$Gwxa#OVZv@5IZ_f9=z+*xzFa?{ErDtqsn{8tJ1wqkXbKPa$l5?-l~#h6^pNx~B29sD~$sj}pwPm0pM4C3w8#&Q5jZXRsYcP!}tQ1haT+ zeY3b(iFSHdA_gWkwTPuyG(&7U_m(^v#J0~=l!Wk58PT1=zXM-2jE$wppVe}b{M2`GyO)9#%iaH%lDh836HAf?W z2{{6jnq#@t&|&dbNR-l@ji+H93;bLoU~$ScRs|HEDT8X~{uS@QIrXa1W{JG5QCmTd zRCmu4=dN1UC(%r?jyE1eq%$W)KCi4GyqyZd@^)tMVSyrl798o6zw(N@g8=&^&MXJP zU(-@;covn)KUnVg${kwbL3Nv2@4$5o>IgYXXdnt?^(OX%$VlPD^3<=T`+CS2!-sFi zfvfXgdfZIvcxv^ME#7&p-(fY;Jm61h2~;}DfBJ2+-D8>mlz(r$L$}^!ReOhC))8)^ zDF0D%q)chO6(3!lK|7gm+pNV>$c(W`z8dVNJ~K$RUO~3g-nTP)V>?BB)WWPE;z5^8 zuV0i+8m8BdZ$yn`4TrM^mkR7vqsNC_dm;V(G4fx;#@ z1)BLi&^9__xLZz0#$@zxNwIo?=|Wgq_W;Dc|Ad{?ws3$_3%N1;(Ox?w4FtZ~7 z_;EEW`Y?7`&&WMCJOeb!C)&dM@V6R$6Wc_wp%9uA=Wgblz2ay-;z)c~;if{XcD4_< zf^6{)-Ik4F`~#MuPbG26D@EW6+g~6mz|o#Wuhu5_yYUz!;C*$i4&LuIosC9%g|0un z8n)^;)9Lh%m2&S=W(29gw3B;;dU0qLu2xr7Ti!hG2tE2q6s70~0hCZ1gM{9F*c&a9 zH@HeEa&&y!F%l-7`F5^{Atl89U5n2kA3S85=D>dR3C-5&1QI8NyN znLIQ4G6k8o2`$Tf6eRbM%up;S(JRseL28#8Q=#%L-|p*!GruuvGuL@j`;Jgsg|zM!blv&;^_2afDzXu(9wTkym5N_s zn(VCb&t9#_w~&V_xj_lmnv?5vA}o_0H{S2FlBHAz`GDWZ@^ghA--8R@>iw$fZ?g6} zYku705YMMr$rtS^eT45j<(zQ?8}Jz`y}Y-hMaDP7=h^(}!^$?h-qyBn#__`FRfz?K z$2k9s53=-Mv3~8Wi~T@!8Mur^ehF4y&9QSGCtXnJ?YRWBBuk%03@ z{x7k)=oiCK_D7hK^@dV!jRy?VS+Gj?56|mGo(;ye?AFmzzMwrAR40p4aW-0`l`85QZ$!Kd)~t=@pd31R+xfKz&mryfU1sAi(EUy~Q+aNz4SJ=Q zHqM?!6rGn?1}JrM+|Vd6Zuj7Dc}(2TP+ic)8>Ga&6y!GjTK?*k6B(j==Nv;nh@kP8 z5csC3AQ7rl8{{n4WW=0}gJ2OD66DA`F$(%*7zMVjJ&>mXVirF3oaWZtgGF5vRfzjp z$k>|n?jtB=+ z9P^SX0erS1dFCeeV~PZ;H)kJWo^{)ce@kS0$ozsyg^n_KZ#jRwnP@41SWbZXRwvvu zlO>#LbZf`Sj`ZO$?mT&P1R(!_b7`C*wrM%`mNzk&6&pnMqIX#^ zxYYjv|DB4+Dbe#Zbrd_vtvvJbM-7}^Q3%QJDDeA$*=Oq?m-er3mgTEw+j@#qWW#Hs zX27^hdG0I0usCQB&e`4}jZWFJm!10w@$y&0J&=+2j z`>O?b95EDHxthwLKz$tfm?-S3_ei@msqU6(UiOL^)B^DcvmfphRR@1_E?V``BaJZ$ zCjE${0SFk58-PMcmCdm}7_MFxhKp4B8+OwyQj>fBipvm4Q~U{pUtk@r_>%ZzmW1^C zlaqK=$3X_8PtC(b20SXNn}NaTHe^|7&0HImw@rud)5K!%28y;e`M#o?fbV`1rHoCh zmYCrNlA^9Y5Z%d*bkZ77cp$prL9E5 z*!(U%Y^sSzfF~iX#&vHR{-^7=ZX8UhBkBTKWxSxzZX2+wZnl!5VPR{LF!D`9hG%i6 zxlvFEgo-pdzDX_ao-+QU(%p8#@AMIP9Fu2)q0K_bLJuOGOpeKx>-G$pm+y2f=G%;f zV5g>#@-=lvrlwaJ>#ya>&^GO?!HaajeEo9x5N#1}#}(>6M6;nWXLG|$-KMfhxcagF zn{{f&p8E@j7Er1aT~g{ zl{FwJfcNr7wBW9hIOUdZ`w6#fq3HSZ4n`HtG}G&yn(G<7c6fe<_Vu47p15Mbu{eVz zvlR?Sghv3EL38=7kM3Js8P`or?>1-(_`S2u*FNWrHnx19{*}(C(0&mQE#E>@NZx=% z$!Nzui=p!%sILyw@nLTDfB#83Q|#ruCt=^SPcOxC0rg>WZ&-(~a$7i#KhJdY%sh*i ztmM-@C;aE1;^KR#jW`m&?L1%<{a5GrHz%0-??f*)YD)dOE*U2LjdBKZwlxmgxL5X7 zRrj+wNtf#0|ny328$v(1f_w)|-hZ)(!78$E#yO8b#aUrh#+m;+=+($28NHM2B}`eE4xFOsKXF zb)OvMXrwEU9*u|+g~XD3H$ASNK1yrJ@JK(pF+Vr3Rv$0X&HV~m`Z-J$|Ed&qQkK6g znlMz4(FuCr1Ky#Qt*kQN%oH5o4bW#}AS)v=iCv5GWAe1v zj3GUB7J{H2t1yy2=$;r#*Uaw=mOf6F#RP74+ci{u$PK}GDFK9z)r)G=vazj)U9>FZ z?^G_Q4JSm4AnFL6M&8ShqE)?8cFQs^G_H%#n-7CIC~DnfLllUPNF*kz1qn7&J&L0p zhNxT9p*KZLzT4Ds>fR>`v$kXB+LPj|f1zoHA?&0GYvrHHp1~O)<+w~Hk?;C7isr5h za2Z#n2dUrqAmG#E@__Zq&RJ|^HHj9ScwZO8b#nFtJJ5;l%-c5f^t!LaJejJ*7sT}z zU3f1>HaaWoec-8}IcIpUR^(0WL2uVhZ0Wce09C&=Vj*?9PNIfiKC8@EPSPd|%lWxo z@h+wRpt{B&5b8fgov?L^5K^^nMr0 ziop3nUN3qFd1N5~W9>$!3(0ETae(TZbO=VG=7L?S274CePoGKyK|$Cxd2WBZNijs| z*+{g%4oyuiPPKk!H)mqHgT$js+8EeS+pb4HirI^U#z_nul4U(lK0j+{Qn-dy(qee= z!`73g^N~+Z`3VV(+oAC0lJX)Kr?V{8%#~wo<_-HchK+mTKx0+c7OveGU*?|!AK`i1E@LbL z1>FaL^_o{2$cw81Poa)bak!?+@(&Yr*U`@(QA)Iwq9Kk#j@Hl+R6iH}-Pa1@%_8TD z{!XnEIIk=dh`PsD_@?yTikw9VX7da}IO)VfbmQjuy4kB+kEr0PUbo+?8CcC1(;ZyO zVUfTxMEfm6q&!#~Q})jP)^bVUu%`T3&*Jy9dtcBO13$e;UUDQK{=6snX#`p~7`vu_ zlN_d6Jrax{5;bF75K%>|guk>Ydv1FC8|!RJ2Eun2hXJCZA=sWnap;@b7m0%<+qjcW3knAbY?UKwHD?5|@XUt}V zQUJAIS?w>7r`CE>w3+n&*}2o{t4llM$=`}Rul(~!b$n@=I?hPBGFd}E4j&&ap3q`Ou6%m=))%Ky_5&w` z?g&8CCUtCD_FV4ZilSW$r{kOme z!878_`PdPlhED3mFVH57f}esn4C#$y0*W7I)P8}!{b<8}Rrt#fa4cV9u_;ezF9@{u z6afXO0pMvLV9saHI{_-V-ldHT@n?->jTgorYi1#p`!5h4(A&#EN#|dDQiS%6R9H5@4?dz+ zXduL?@_{e#diQPqBQF%_>6=?Yz`Y7D`V{_nn}58`Kj!9tWo{PP4|H8JrJajQ*2q`~ z_~fT(>F}(hXoZtQYy_@Sc9l9ym3@1Bk&89C0GNcYYjuoN!mKaC&)EVePSO$3U!Yk1 zgUjC7lv4i%MEiK#eT+;e13ekBSi!X02Gl-b7X0~)F#Phi;V{G?*DxWLM@!{ABeBq5 z1x*nmMs$Rr!iM##H@z<&pq}b@gkt2@ZZ{4N(vLa|)^GbiZ1K)TRdH?`w%?AFQ;|We z$$u7)#l5p!zUCkW$DV>f#2zXjEtvV*g3BiXbLL>Z9ctID-hQOz7#^MJofgxvVvj69 zzRxCvENllZ{YR{60Pz2=ou01XMzDa*`L{zDdD|w8b!XLw6+NPUcB`{vJ? zIhoFsCVny=)cjs|Oio~s7dsT2Q>I5keQ2|{{}&7Gzqjx%{@d~=)%J&-{=-iHVWJN|jx4qK8V`=~Ji2vB%{I-4lW4HRpZuL)K#2?thAK1emNYbBUjQ>N|&3%Pl z*5EYQ>i@zM%N<+Aaku%@Fd2yeOc)Qt7-0PuctV;}hx5k}_-*SvU{?VaiRDaO=WlSE z7ETZx{0np>wWWZir^Su|Am$9PH@+bE)w;Zj^bPpRXp?K zT;t%7vy8tCda@4oDWRap96RlzD?ATvP$PfuqS60*H*F|4RRw=KeL!g<^vQkIy53tO zSW-c*Yk(+m{Suz@Vs(Y8^h>JdnR)IZPg~T-P_ZFdKHX!Rz1Y8Z3r*#~@ZaHtYrn@Q zX8|;F2MxOnm*WEP$veM53<^kJ94i0>kEjDca9>+b8_{1uVS!Ssj$^C>fWrQ3_eQg# zTLLKnjCKPkc3OG`UIB+p{uOu36ZRK($&i=!XOZ)_`Im3v3<7VA{riCGMlEI}xYH{hZ5fg~;cV~M zYbwv&T(#Gx_KWq~^iK9DN4P#(O;%LJ@+KWw=t7QwO4L@hGo^nXVbbj3OF>va4<`H~ zNU-flX=d5}wed9>McGU(--}9Y%P$Z|Q4j0M^;UrAg#sKg))1*22kO2qX&L9TO`!vx zR!{|>@)==MR$&s}E0?FhARxhwn%v7_QBQyu z{awYC$L>;W)7S7_(GM(Mr>=M4hIvC}Q<}%&GgGqxE9v=rHFXDx)*H%evnQnn-X%)s zigvyhbu3N51S1C0WwpFYdP}xnXiPW@rOZyH%}yJXx_>F(p=``0oJhCvb~&4bb5o1 zZ3hZ=dU~Hu7Vq!!K|Q#S=^xd($*NQN8b{>&7^}3aaN@K0GOgf=-&$%dyBh*OfWReE z1m{e9?l!uoX-9T{v`U_}xKDhu_|_ozpyCxsZpw;HcyYB1jLaL{QE7KV)96|EqSfOU z2+mO&Gbe~_D~K^@;}gBcN)Bs zWH$|n;yO8^`XeZrmR89Fz{NJMffI>IZqPmPY@gXSUqpX^rz0G-;m4H6o0zem0t*m; zJlsLH9nNWvx2Cn3VL6wh2|G2HWr|SpDzzH-1o;v?X+3oiZ&t-R8hN;7-ZRNi^GRHJ zmK*-sqs6GV!e3^5t+shGuj%}bI_Uss)qP0;&SycdUnN{~mbQsJreMVR)?-G(=F}%q z9_(IT)bEX~Y9iq6t^IXw-7mgYWDjc7EWR>O5H5Jx*jzf$WlEm5@}XHXe{+N;k>E!L zzpTaMN{;8NcNTgp9}M7xuf#pzU{T!t4zb>+&{;;>CnuGBpF%MP z`DzSMC^>?Ai_C z!8{6aA^LMX#t59-5R0*fe3?HrjLDa!sbveY%*-@a)tya{4@hORr z0|7qMj4}*!GNCt3)|35t9$`H6`I)T|nOk6`z0r*k!L;?3r8;5@VXE7O& zk`}gsLKFU4PL_Pm@Xh?Of~J|_v6tbjImAg9!r9CoAZR@!Jl0fa6OjC_^nA;I_L9Ut*%Uc;tZ-v9nNlvd z1=VtN>R5Z~ydb1DNX#xGHBTq87dDjBfQOITf>nh_A4}b>3Q*Fhn7pOkJdOl-;k`E+ znktpO)Kru8!wd=ND$Ab4CyglP2adV?uun~&*lMiCfZ;#pI8dd-@YIq|vAlS`m@54A z9Qju+92&31msV40+p4+dWL{-+y}g&@WVPcrDOv zNWJh*(CiT%KlSpDInZe|Y%}pI%Qtvs`Z=B>XMM5B56;Ca;pZMxRHo|rYVL9*@NRQk z5Q&_Za;8@O#9j^OWWu^u`CA7oNU5SbdF0bz#Vm(*73>$lr?yCt3 z@}mrW2&UH3-!*Eu;X^5}iL5noyT*La30*GUM9?c#%HARjZmX0i$fVt-355=vhFByCwCIr^ojC4Zy-rGRW)0^(0I^qPs(G&Osz-lA%n%Gv2PG4b{`^x5>g>GERL(rb(OZ=>8CB-_3j|rN4p?aPoAxcC zMg+dl3eI3lR1AJL=D{rSOjyebDps56Y?BH%SGsuk-Sc{-*{Tc&w4}Y;mkg*m!UViT zz6SRj#R`{L+;`H4wd;;PV3XmE>H>7B1FZGZQ4s;qIf?MN~E~nzFt! z1mm%_ssqpG_08UWo}bWs+}E3_ccZI@9VcLzW@o{$nW}5Ap|K#KKWBF2)JWw@r-4l| zoKg3JEExL2M+_7EJfAhgs`evF#mrP$M^-3vTrQbiu}FJc0R%x$sVuM zSEuV~vcgjb&!WDOAW~df zN{X2CeGQqbccyiEUqut8Mz4NoV)FAFJN2OLA~(&YU~G3FSEbgCpNY*gapo1>GwGQk zZZS&CU11w!!NF06`{yb;T`GneNe5z%ju7{zH6WLi#1l5y_@S(zd+fdA!o?0EdFH^V zrS9>i)@M{I;9jLOs!hKfM|7I@#QP^pw{!ucahxul4{B<=$G?BOZdO1%VJx)$Ijy%| zeBTZ$9!~M|reN8;J=}b(uDWuPtD>&nvn6sI+ryreZ9Dl~G_oX&!$uOW#GH?v&9Q?s z=L7k9SI%Q_^?mis!nVmg!I_H}I*7FP0?0I;ICH9$r=2)D3YKK6~nP?=Vy%x0|eM%cW z!6XTPyl1k|Qk+`Xutp8@F?&@5<%o&xMc_O6E38NzrTGoW-I>st+S@(q6C=swv91nM z)p27NMj5*{Q7^plJdf;5J#@N?r7*%=mbm5G=_xw!L9JUFGS)sQlQn~!+}4U?e)NIv z2uW>4r5+*ZanNyj4*Th|vKZWb53Z_7uGBjzdGxTVJZl$Aj(0>h#_3f|pQ!1z>t%h) zfaC=lxVK;zp=m>HW7?^(GUF^*-O>ZE6_k80?*4o>J0zR^s-i`RFTF#RM@qj@?EZUq ztf`ZErg)<%ZhFSt^O-Srt&EXY`Em=$WtXvQT$?b1&rE1a1T?c==hF(6)hQlWP8T*C zx_0m#PF*X7bW5PtU&3p$qsyHM6tlZzZ^SsjW5YjiMSwLk6nJ(y;^w#F8#yZL z5^G-yoCv>;U;PwBAto4pDf7&BkrOjq>afqhi*s_FalM;fP!qjS9psg&Ah7k)A@rv7 zeIewVVZz6UX(K;5B}?2r{nKimZYc;Zn(i2;Xr3qZH)PYxi5WdxVAc?Cz8I?2Jf7oy ziltcgyYSON7sG24UzQjz)Vwz|qE&VjSkphe@`jpj*Mt3Ke}!?}mEi}OKo5hO)tC9N z^;!T1wB*`n8-c9kQwc4q9UCVRW!d3Y9qo9iK(7}M`RE{V3^3~3YObJB7d@W^A_7F@ zzn(=F`wnhfD^WS;&?bCO7;vp9eqOpklK36^6EYwQOiVzmIB1-87x0Fy&{L=3RU0c_ zbFbGcubh2Ow&@jVG(A= z2S-zLX%L6fCY1WQ40Z<92+d78N%wE8nM!YI9@S)58ACig>~?j#SswgK*O9}KfDS5a z%`b7{k4ZZ2o!u*|UJ!ZZn=@WBzxR&H^wE}+3wu&@dD9P+T!*;JYx49I6rPV|&NDlj zGwIN@PG?)5b+<;TER$@r?haB`fn@`tIZrQ4q!y)rU;~D{qR{@7;m$)3R`QY0)h;8U zx6B{gWb9vn@2Z>3=d=3le|nW`6d~0h^>P5Q(7I5z@C&qBR)3M9JTNr4KWFB&xRA24 zYBAPIyy?4^lNVq=qH#_ezUT3%@z#i}L>S{%NyvHNU7xJ5G(4G~=q#Y-TA_PN9vJU?CRg;$}s^hD>OnZ1S&G=I)+u)lcz zZU$zKT89mw+UvuV_RQ~wtVQ0mFUiy`oun0In% zJ8fuTIJVie6BusHJ7n+SKgq~ z#2yPaV4qmmeTIl^9XZiLrR}7RaKS#VU3^hVwB8j!HC22fJ^ohe*t)_OZ55t>oZ>U< z25mF1mRDujl0G~k9&M`2*X)SoE?tTHJvILCk9kkdk=G)!5kRGkUlGE%(6Z@-1Mxy-nTTfOcoG|o;gX!CS`fE>x}Uy^mxkx3=JpId;-tjctIVC z*i5LXjn^c<5#GM3C540Z1m5gk{~>*&!xWQI=ZP($##S8*5@S=yl9wSGxBMPUn}Cs1 zm)CxQP%ghfoRxEz-{N4KKkX46Wg&r-7+4S+M!e-(otKAOF)%g0+TGzY*7>4(>v0() zWu*GzT)E_Ot22hH3wG0A7ZcUWjZu7u0$Wq^d<$h)8>^&NYL z0#-2gO;_9sL1gcA_awL}5o+%1F6JMA4pu_jgw&v3Ahc6Y4%IQVCx8G3n=M)8NV0ZQ z^OQCGhl8_snP%NZYBO)4cd$7c6;|lwB$kn4>0!rlpj^IS;q#iWKI&#@(gTvPjlw0P zG-I4KI<0q^pQ4L}f_9Ne5 z(58xbR^MuP-0tYK@!oMA*mERs%OpKBw9RJW+SyL~nN!3@>%`u1fmqlirL4xPM&5UJ z?T^u~nbK$ZGqT>8s4n3%h>Fv+a;L7#?8MT{**+Pq^OyMb=s0j+-9oj1-7 zb^g@et1&iQEZRodtZ9L~=VaJ~W@X@9&%7A29=}Ez;?R%B79k0jEOi`N*(;UhMJkX% zXjq+`i0r2uqa?0Dqxon)2bq=s_`&Xpw-E}0a&gRKm?Ts9ybPE`I+t>srkDY8-*ZQO?(8}tv9-a|imr1nYWL?oT%&Mw@AT$a%@gCngmLwA}>itacp0I0J182k-a)~Ce7)l<7Q?b zI9UV51)1#b&c~+8#>~pzIs{yJ9PAlvM)lIVH4|HgdAPFdn4AB-7Mj0xrdeKF`)0%M zqtH}_dd|=EIXsRt($&?>4}w4`_1Vg0qr#!sMZghzGBbw80i8)$6ps%Z`e%ztN$A${t}Ymo8*_0=Xdkj>~4|>`$v3C!JPb z$yOsKechb82X^oD4LL5(KDn`9GIxeZ} z{=fFFGpflhTSu`WO+N&cs)F>QRDnoD1q4Jyq?f3Gln^=^q(qP+T}40vK|w@nJ_Hh} zkuC@X2t}G9L5d|Ip@aY-%1D~wdmspd^x3`KQy=BeS`nYFn41`CSiP6VuHSu)`>)qK9i0foWHnhtgk zVe0#)LOjV7LT;?KH=sDT=tvBYmHIp5XzMRtd2Fa&k-<`6zT)81wCWFb?}zP2-3HOA zm-1?Ge%>gtwA6D`yUSV*p4eq{ob6CR3nQB-$g>U6J$I&;lC^x^l}ss~xd)nq-`s7& z|JCQUEcqH}Tlguq*s2Mf(Cn&ubR}{Jque0bx>wa2P6==Dm3P5ZqsREqeRQ3&Q_>O( z&%AR_M`$uwhxU*Kp#fc2u`eo(_^RjoWD)I7QM<&fuEp~XdAL5!wlkp3L+ZVVxV9U8cB5lw#9^>&ufX?)qhoSiGP)4mOaqagQwPD> z=`aFQnpR}oBUh^t^pX*@rr#XO9Yx}jSte`S9hOj_g}h)BJgD7G=@J>{)oA#XzLF@N zm+<63G2dKimt?55Zwur4EnEtQ=NR=|0M%JH_ra!K{V3(V{EK9X#a%keG_*<9|z z4&Pm?0Lfx-bp4l-*dVg7U5*VDY{q0erH%*+8Q7nQo(g4E!XUdfphjq!L+zp2UD z!4w;%JV|H8K)35v4(t9mMp=SA4Fh=s0gk4!mAQkT#^HDv#TQ^i-HOiorzpUWuOSB& zn-sS&lADx1i)?+3HF!2qF@EI3!bKUvBqm5W?i@^R>uzMN&5kvH5aT`DKWMn3sUq{7N+jd5%3V~k4T6W^w@cbc?? zYN_jOQ5J{uBK!M-Y+Vfa4UQfQxzEuq%hgSmFuIa?oBugYzg2#xHNI8{mdk)IwliIV zyaG!hqrp>YZ|`;Xjub>3FrSh*ZS_`XCGeu`g3SSs$@I+hbh}XvorGhqcOaxszBYa6E(DJ=M<0jS8_?(^%gG~%QWuPtxJa$fOFyEr2X-M9uJ%&tUaXIB>q(4 ziK}kV?22yIK-F63!RS82cny@v2bL>m@J-(7>DBQVgMs@HLL165i_a&O9&%Jo3kyT@+pOQ@xKkqb>yD&hN_9@;xbcATYC^J zPqd;`%G#?mWxh;zO}9$bi#6LdT(2%Kcx6ipnEOs|usjv@Mjc9x{Cf_d{aX$Y4?JAX z!>fl=Pp3U$JUIFEwo;(BnPSosT-2(*Ut9RI04Azri7P=#AzwV%@XG@m>l`aE?(w7e zQauw%i8{7MtiU39H?9dYk{z9xM>5&j!X?$`Xf#cr+E=Qr_m!WCr&B_ zbk1JK?YywB^keF$JV^!p;?fm3?s}gu*0v}=Yb)Az>2Y`7L)+mOq%_SwO$TP0K}E^O zuY)296J9%^OhuYWjAXx}X!GiB+jsQKx`~7PQs=cUz0%s+r?pc~4bIVOBoy|BaT?dO zf8^tFL?b;&So#dO1hyxYWSo0M>dv0RZUiSmX6ex64tm%mTMojpMOz34tq|qafwMnq z$F#4V?7S(6CTlHwjK-HLXWgGT1ZuoM&(!HiK$|8Y2mWD>|95- zpLLGkFaDBK4|I^vwmDP{D1`5jbmy8y%K}9``V*hgy_7@8-c5TfsKu{(j-2!{BJl{J zt4a`9a{>FpB!2sMzQG@PaW2z#8Sy&Dcbj9w9&M@eJ*M2o&C8W^DNp7Ba@*yU$>+%l zt0#w4m*?GvpCHm1>H_y8EZY?9MH}%L^YTjcg0a0Y;chK2^|MLz2r8~tR6A$CUR`-v zB4y>CPN4zVbK&mcy<*M1d|XjCS9Xnu>$)Nc6O`x)RFsWpT?i5=YuWYb33XZdUi0f& zo%XhM9W9tEeoFd(wl7@am)W7`o)pKDN7&kx^&Y0qM3q)()G8eoxF)Qpm2u*rxV89w za#v0u>iEI?>F3K@eL*%!%}5bd93Mksp|QFt%Z8P?`1UY5SBw&X_%&iDMO` z7`~(2<_QQyuy(*Ld5K(HptPZaUG>qzYWJaJ161I!f>qJ#Qjs&X&NyGeNViqVv>xSh zJ*)QuZrn0-;YRt@iR{R9$!J20gH*rQQk~L*ZL(om5Eoi*ut)EK@vU~S+3~hVVb@kJ z_@$N@^AcU|l+?>WnG!VP`rLp37o?_IE-YkwUae-Z^t8&nzI;WS>>f9sIS^QCCOPam z)3!$vPpLo(FfIwMYg8i1HLws((|p}+U&A#HjZSrH#fYMpuafGjQ}U*eS3nYC-O;ceQk$MTKRLKKOTiE``e`@thc(8o697CoaBR5-ZX z{oUiQS>aa2Xqn73g+%()%~vLmweo4&=KF7UU|bjM?cB~-)Cuf;_$en)yThd+#2TaOICRK%N(puMBud)Tuf%^sH%z)Tw%eYqH=KV!jr4ww@tju4yK=2`!nI3 zT*wanljnqniV)ev$P`@jy7C(W_sBF1?mlW#g@(~h)KqHGMrm&GhWq5)TOQr^E(|=; zPP17;Gq9z@jC2U^tXhxFd6gi00V|4F;=IO{@DVLHb-4n}A9v^VPHZbZ*Bva^m#98c z_(-2FUOX~2jZdjlBy_yX!$B2_BIuq6Qs_C9w7PdFC7V$UQ&5-?7rMhunbBs_Q!n72 z;FD+al*<=JHb&>XNjd_4cJ=P-(|m?c^^qBw3rX8({C9)3ipN~1SMitDH0?+Vqu8ex@?hQ zediwTf3p&UQe-4g>d-h^)j!Cs?ZS6W+&$0~bKp^~)w{REi^}!m-3|NxFyZ?MY|JWT z^inmoNp}WmUKtriqn3y-LE>nPSsxBr?#zC<5fYREQ>BdBFIt4c>nPq8rn(^o^I`wE zrf{pe;WK%tXUb`tR&;AWMn!g1g+o~V(qQq|fH>LXa3CsvU-R(BOHar3VwIrTIEK%!q&&(Rndt_Nfk-f5r zOU86a#?3kH&*bqM%0hgA>V&{|pQ)FewY(I;m=Tmyst^&mQQ5~jDzch|6}<2eB_qnC zgc<8Bl*+*l@GiXhR!slnk8KY5-ftZlj~&NvfS$C3-X5A+{;I*kwn8ZY2GLEzo`NwT ztzIkAK2Z7$D%9S@FSF7P0j1*n!XocR&wq+pz#;6I#Y_Bn5CrMY^lLtXim8 zbn__bXJVRpc)qx(7MSOm>!+v|hTvK|Wa%a>K4g>0>b{C3CJ2zJCjm5_NGKK0ij)U< zj39a7#>Lom2unf{AZMN`U+Y}~Fq2bb(DiNl6qiY7zBQnkXg8LazrNc>p9Qi7NT&ch z2gPvP0G&>~4-5l-;OziC^Eg6rb$j-QJFG*VimY(4Wo6o{*ma;)>qkovq9qeR`kxt{ z16q~{*|2M9t_4+J?w1f0*ENkq2;(O+Ke<094&bE9 zu@*&XY(B`vy`ojw2@fs5W;1`b@}uXQTPCox{K#88jIh_()m=$rzZ+o)=^7Qf$N{O_ zxf$GlpGf+@aQkL(H)rm*1>$#RZpt{qh;>Rt6(ISag|7OQp$4*lTmlM+3_mLUg9hnN zlYpI@Z@7lJ$J4E6IZOF&n>_L~_YhIq@_D*|TjLS1U|-t+wI{t5CD)m-f{>F%0CDtx zZJ>tcevHE}0%~)DhZQpdVMruxfR2dLb}Eamaza;*udsdh=lLzfU!7o7wd@ zLb{nsrj z$i`<3V*s19h}i&zk9x7x0eD9=g0OhhHJc$1NDNN;BY(!w8Z~8?{aAcFjb5vNRG2XN zN0GE@2=O9Q0styy&jK|k9KcNle~lOJ?Sua| zNfrkj%NYL!K1z+LHkaPg6}p6MeERRd*Z}x6k|@0adMtV7pSE9S_epNs08InXpMv(T zYeURt0G%Zsb^pGoMvWmEmm%!fS;Jlk)fT=1YMuSbarHrW!8XKNLh_oWbV8`n{MtEu zwmdn=PZBolFggsqR)COAjBbcE>Q$yreqW=z;J_HA3cT8khtmXi<_*;NUz@6WCCZDi zlqseibTHnn3`^{PlpZ(y}p+Nw6 z{RjMc?0dLZEC+k@SJC<*o<9SizhxGP<*(2{Kn;5w_m}SeGbz&TWmkKgW@NOQMtd?>(_qt4Xxds94ZU zeN(_Xt{<5odbOH0^BK}wEg-e|otysr*9hAjOPlj(a}C*CKmY4;2YbaA+$-xgFL4EH z*&D#Is vj(mC{62}gq;Q)Q+|F8B`2i{bts$X{6bwK$2%#D-mi!WXO=gxV^#_-<&re)?X literal 156375 zcmZ^L1yEaE*Dfx>A$YK$DK3TJ?hY-5LUAqb?(XjHw57O9Dem4P#odd$-MsJj|M$*( zGj}qRIdde*+Gm};)+0NSYAUi=05Sj^92}OsoRkI}9P%$XI7DcLOgR95tN0 zl=wRj_~UNWfKTqe>yM|eInIxrvyYZvWr`tL>ZeKMO|?5}#xYh#mvT(#eF-czrUWBw zG;|4}<4N>%_LAsEoQ6@cj)|s(qo30%S1n3NV|jdKxZVHeE=>sUJ>|(rq}hzhjMH2Q z=Er|*vCLfW3B@30(`#|q`0ov%e|^+lygVed%*wJYY<{cv=(;FH0`l(}9f?+9#PNI)qU9MuBjysVr5;asF7Y`1 zQ$(X(yh?P_jt!&ZqIz^fKq$qL4*_yQ+Ktktn%90-c8SOih=lTMr?!bd6Gd8zmZ60ck9PVx|Fbw9u^4D$< zEGzv_U>JBbxhk-*96+f5b(=P0r6JPPxYW3qh5z}M<*|-HwEER6BcAyDVqO1<1u@1Z zGh?}s0Ht1bGZK99jcyIjy3aYlXVV68r10$A85e{^gs)j2)Mh(^db58_X3nCv=#aFu zWR@h!;}0YKS5owoCr2bGF*t%~K-k48n=+aTg(;f|u-klh3r1s7E4qspi!6Xsr)IK4 zq7T}{2S;c`%-{mkH2_j#Gyx92zye&h|7@>sVo8QLry$@}WoQ8K#3B0bt70Kd?IHuF zE(yWeej&NT>&Y5`(%6Jw>+1Xg!A#*002V!2ld1eDHNj_gc~`P*Ak&69U7WI$aY)=A zm6>r%cpBm9##JEL3&$P_@1>aA!Lgo0Iob40IsQxcTWP-H1E z9%k=PM^I8E-PSbIuqi&?#x)pR5KM{(l01`sn}P-Cw~xjzd`AFy8zI0!XIR*x1vul! zmTpppCc}q707GOVVHls~P@LfVqk&9Fj1;{paRJFYSxljnOnqoXv`lRHW!lNX&;!Sp zS1FnZhs4c!;HyXo1RQ|S5A2WKoc8f(pax@qP)0Ay{n@>HZG0ER!{UW{(1(P|CmfTE z=Ts6ap%bHmh6@^Yf^?(Uh~X9B=L3@qqB$FbV`7Sce5`mz72Y^t?C+ygrcOQqmappn z#?M`28*R!J7_q(?bh&EB_Vks#- zeQcRxm!MQe0wOa$UG=qD zYKjdE6qiCO1@a|r5#pQi1}oNS8_~QLs0>rGXCVXnW6M%{9Fa;}SP+d8?ao!Im34TB zTjmGZS=q^Lf-$4MF$8@GX&Q#w$Ei;7h{Qz%7fj(5@$U%DJ>)OUdNIcBO2Z?d6&RUd zgXpHup*J3k)OuLrh29d|&B9_epZVhudpyC^(7re!`OSq%F_zV#6f`ELD`>+`h7typ zDWv4PBa@v2_@LVb4o@v=KKbwa?RVODPAoYM`H>>INXi>XlMufI)RPk@0fs9K4{X011uY zkcQymA(3MRNASQNI`I<-i}CW9RLDjiqKA)oR$5p1it3ZFxl_*T5T+gVr`d|CF9S#C zPL;*MYhgV-5;F%p|4F~y+eaoAmZFXhQ2wMMZ?>6(LqcMry2clvCPE+_Wqi0c%RnP8 zp2io<#mfFv4Fx>x@Gq4WhI5r#7nhf$BqZzWg}s$c^Z2bzJO(C>-3}Z9Xm`WFWD(e0)52;>fzgTOp~7 z%Lt@Srk}$q90I)ZbDKi`Au~^CVP$YMg8UBi+_ zf(^=Skl0jws(ezTCut}>W)wpCIl3%En>5(y0%6Nr5&MD_%@2

NSiCi$H}Fl&JeVwY%u>>fV$7muV$v=5S( zC}{UCb(uh`*@zYn*AT)C++@7 zF_<{fq6HBGAGA!TG&yw4jmQ)9cX4eU&f+0H1Z0V`0)<8ttlwJnv&zs=f;>9(+3?xk zo0Q&V#Atj+Kg1DL`JE;Ev?;%`g#-jbWULZU$%FrdEDY>1W z?k0;ov#2;l zjN;+h0ZW14Afx@1l$7<$`wTD`69*?*p%iw2Cp6*IRr`I`=4ukAnIEbo?D0U5CwG(X zd@2zgN^A+FUseP6z^?uwlwHR!kI)VGu?cPGEhNv5%R$Yeavo))&1#?NJisJagGmfY ztwWAeng(0|l_kK%0+M|gJ-Xh!_B6_w2aYipSD0fD-U=zd;gyLS0~_oX_rW0pkfWTX zkYI&46wLx%ArSV4fb%vX#RM4yyUO*#_0^!*II7HFFrET^4+&) z@mkA2Z;W5=&9Z0NtzX;W`)d0H#(A~Dp7g!o_SuMJ?Q7K#FZ=zBo&Rv!CHv`X{InX| zb*S&t(d!HO%XK6tU31PNj$cOjM-2Hl*->Gn+gL)w7C#>1j}O;VCFsX;yJ8N^h-zEF z?DOW6WqSvQqQU5Q2=oo?xxEc~qULi9Bv&)-7Q*dqMHx*p9MuVFJIodRbcK&-OC})U zmS{v_N4B%=_ZkMe%8E;vGn1m-niLiu^rEvLmIp67@aCQlbso44k!Z)dTuM9cee_XPf;@GX1bj2=N$Shp{!tp4~UlS{CaiaOA_5PxVecq|S z9&_W5$K}`7ohsePoR7VeO}Cd{W8hLxFftAD6z7JEkewqkf{)j_@+F1UC(gHLo3402 zmZNWN6@yk=k|`C7TN-kL{_xs;XB5R?%#8z$T1IV`yM@jfzA4Es0Rp`Ft@W+>3P^VuI zy;CDpI(up>UtCNr>G%;jN193kc-In=6G^PyTpXQfnKe36y*0tNrciK;?pODPDh6%# zt3|Do8S?Z7Qg;zpBfP0@M~s+2hfT7)#f{f}Wu>oHmU6E`bEHGs6{#hJv8TaGS}A8? zZ1x+fW!Iw}SxoZNak*}!Gig@%@`#q!*f(i+C7zXFe*T|%wvVJrC3yA`hn|WVy8DbL z3nUvp;7Gf?HSY+5XUCrfEp3T6yPt&sXoSi63xjPZ6qvE)o?ocPo7X0DuWMY#xsL@z zB4Yp3p*wQFVP-XXxbh0>rcVEy#h*+tsLGKkcjlC-6lTd2Wv0_X`St5pOUsNnqkR`Q z%Ds=K^*TzkeS4RVtWM5OcbAhsS()RkHlUH2MzT;;&D)#husU$4{UOs%PG4ucG59awa5H zSW6bLffia8xz;rN23esH4)sjO>a!OPE{N2y#sftGBn7WfD0TMW%ppNv08{`{>+y%e zVN>r%1Qa+RNr}PdbqJV`{Rjnrk59+#b_1#83XK2!2^Y8JAe^G%Aw&rpEUT(Y^K{-@ zzx3o+b=;I9ZofUI&s2gzF_|}40z&UZhfhKw#}1rnncYe%0W%q!?;RA4!pb80o7*HS zsji)AA*EA1SmXBUQ)K)3H$2_MtMubG(fQ2<`-W=o1u7$2iLb zt5}29xPV9ba-|u0f>OwYdQK@^|50g)AFLD0;eO8b$dv$?qK}*hFys_NRmH3TNKzq? zI79^q4B?3*fd&~&e)$9&60BjV$B1gTJJ%RZiTsg_ivEu5iOy{E=qUpe(7zI~qX z1)T~_ScpD)!U{l(I;I`mr}<1icQ&f0DXJt+?x$_W__HOat9{OoAG57;e~$z{qh@4e ztY51c7)(<6o#9keS8ok7<`oqeCwmCVFagV2TO*fFTXNg}e0+F#)M>JTgyT@{e$;6S zwO!Kt5{j{Vc^)qsvUa+)5$@mjf`}^5khHzC$Laqfq_yIWLr6&Ib4k@DZL#NA)z_B) z65%SkV|LNHLqS#E_H+By=UGvx)-@^pjoJOzTW5bw8iDkoX}K?;-Ok&UC#!4L45BWE znEZv_*?8Nce2h2RZhPT(eS?P!XQ^E~Usic={2sb4rcjN2t}jJ*srj_zFA+q}^SsYo zUfe}qB4v$M5qXwdldN*wjYlBs%U?1=yMOy!sJ;@13vu7w?RAah z`)tN9UjItxFYrdcwcGQ5In%+WpZT?Ni}$jDfpZ?YCiv6;;mNrpCH?E7V&*4Gi<3j; zoTxX4#B04#sh4uqt3$&T(w>j_S<(15e&$$=CcM{|YkhT|82jX_V_#Cj|J>{}JcN5b zd{`n)f@4)YH9U_(zPh$C#Ka+tW&M>J?$E%&ZzO7^FRk*n{*5!K$QcVQ>BKo%==LM4 z48HukX&a|Vg7`~7Vb*X7YEq}MpisL?k}II1-#ebDORt!}xvn#aZ5 zoIE9?kV`8AM`FbfC?r%LmHo*OexI<|Vb<9{vwQqtj@%e7^;H*y7gTN$!YQ3N!{eCvW^My;ptX@Q2E+-PmrVquW}X=nMq34a2#q_{>(FybUf`Dg5KLL^B{ z-&fFm?SX;l&(UH-bvUc4@ObHybmOwulCcWa;v|}d=cQFnbCA}Wd{fn;%vwg|NUlum z2gkhpOpLS{ZJvP}WouG0#dQqAB!U@ry)U3FqVdu`R8F=F5&=qD<-uKx#bwd%MnJl% z#osF@m(&de+VVWY=uf(6WVOl)TXoCR+;oXS-C4_kR#{G z?Y`QY!vBR%Hy$%-rxC4|dvM|I7bjb)W>>_clg-Vkh)fK~kHEer;$HF3)I*-d$;6hN z1KH?bz2TWfi@Tj0HkN9fNhzwO8#m4j-t%oB_i0mmt666F@L;|PQ}NUlkN3Rjcm=xD zq33+NorRHgmTR`4oTIwo9~Pp?AyjYBf9R~uSlWL(x#3}SOQ(Kb|Z>b2l;6vi|N+Q81Y^{OA=F{&YvyfEcg4~em7ofI)TPH2!p6FC&#&T zw$@{rd30&WEYQIK78tJ+M_d$3bf1gzqK*~1`598m#KZ)frMwbSnbV4o1gHzhD}KyY zbh3*S-$8n>BnWd(a)8mj0JupGd_sw$^D#r<`N1TTxYGu55V<;d09`WIp({msgx{M3xKmGBlg`W(ey#I?LL`kva zmeE-e__*2{<0~k5f8JHehSvoS%UBue25AHY zP?JznD2p==Ds_ONY_ur!s~YHKA58$V?-)msI~$B$W=-0r-Bd28}-Qa5ErCMWkOaF$gidtF`*qy3d-Wl_tloVtJbJ}$r@ ziotVxmznvjlFO&PkCA9P+@q;qD=XJKVL)ZWA%flSToDBIh`6ZZ%F4=KsQ1QccR$(f zJ12Nr%=ZoY7r)MlK6mwPIaWz~`V!uDwFc(`VJ zY9A8A9i>`+Md3UV|79lD_pl*&4ge6>Uis#4&)%lhU;6vzp4zR-V&2WAiwvTQ-17Nc zI8**O@1&~RYv_$F;EC-4iAoz$4M$4ux5Y8_m!e?*T`v!J2V?IsDNAbpL&x@e25mtj z&HxG!9%V!%yikpGC^i^@iAPxyQdn}OVyAO#cKU(O*#j)SEHY9r$XollU-vFI)qgb5 z!h7N39S*UJsGyjtf6_$fI4pdTns7v46a#}%nRP^43a>6RnY$d~YeBnUlxAX7T$ zKxky$10V~0E|240ZAlr}#C`4+K&J197Ncg=*w!!{kdt+1@#k6@kz_YSV&`q{tC2A` zXGI+}>gA zn56aE_@QuU>1s^Mg&E)oG;l`{jAaH=N6?}XLrT>YIHmiN$Ia0B(IBLQJwX+}! z5ZLe6ft}ldWFpn@@b&JVtOkxx*@tVJhEd$2oZeG<&W+Zb) zJ-_|+;T*rP$M}!=wxiFIN?SNr4mZElEnWJC$-^=Ky9&QN5h1CpdYQHbK}!>7{f@5a zygP)?K{OqJ8ND7fHGXF{`U5+ye2|fXzS-M)YnK}#$CMux!@$h_9l9>9KYhmsG}zUi zVS%+Qni`Sn8tD2k@dU=L<09edzx%}#JqIYPW>*f0cKIYZAL=6G;R+~)05y>ukv8zV zCCGF?p7qPfJ@o1&tW7CJa*xyxKFXQHTPF}OqO_R2xBzZll9ftG_}~%B_JF42ggVV1 zqDh(@Qo6(K^UnYao>{HZ`n_-psrnqVnYG{J$oNOQ?=IW`KJY?fS(hoWdnbmZuD3m2 zTvkPPdS`@v+npaCuVsqc;-Dud-|$!?{qqrH=(b0V zVbt&gF?-al!)Fck1B?wsji-+VNyr#vjt`F!=_vn8TKT1!UG`wKYj^n}CMPGSqyR6Q znmbc1c}9LbKjPuBsWF+4e=el%3a~q&YqE9jOwrK8jg-pZ_U3JOKP~+7C9*%7=we$_ z)j2T2p%kVPIf7sl#OL30^NDhFI=4NXUYA6h!dTXqq4DpSHPCt;<;qz)lE7P7H4pdL zt3Nsto6?9ddxNdA^x~yzLyOV*2Z33|FrebdBL-xG2kxO!Ct#s12H=W0vZiQkZ%Z*T zGxyfo_Bo2p8deZ*)#nEk!lNN2>@=HnU5X3QBiEe22FVuQh>lv?M4TL;^#J?dYXYWu z-WkzsRQqCec7UWZiUJbKF8%zm%h=HndVoeFH&nV z>FMDbQSaV_R$2d=pHEaOtFg^qT@~3pht7X{%Q#BWPnx5K7ufv5@70Siqv}I}hgylb z3H7jh_7~&kXckh69Nf(%lz;ib9NvJ%)PH^9C1hGp)Hc{-osM@VJB3N1Yy~>m@0%(k zn8KI}VuZ)x3!+wnWNZDD;)#T@bP=>;x-)V{@@&x`n$Dt6?O|$!@+{oO^>#E64~^KN z?39F(F}>z1Bd$iJmp%82?BKysoVq$eHxGr&r$Bra%51|DQE?Mp_)s%(H+xEI+^Z=L z=KQEEwAN5R>{GIh)^Zj61Sh4pLF*1WZc%r`$}TwM9a$axr^QBI%#S!Xn)O$bJa2y| zYQ77o-`7flbl1ur@Y|`0a^r{(DPcNYwT4H^$l_u4NPnxyZAI&J<%HpZ6vl7|eNr!T zn(JE3z1aG_B?Hqbu2#_K7%n-#w4W74+kCujSqssZcUcBco9ft9A`G!+YZiM>*L>cv``7ltN~ZN?mCvLkTNS4-KLbA8(xvyR`#== zsuiV7P7@TIAD8f9;bDuN7C6>(m8ErcNvM=9`MEo@NU z`d|$*=4DSdt|($F2araAOf9?j5 zU10u-s=hdLf4o00BuW2xp2&Xo|5&|Y;n?<-gqlXao+5=A?98IT>_B-8X*EtIhXyn)`Ts%Mv zulIP;G&srV;W<(Grd%>Yc@l)m9GzC|2nwZmQo#IZ{ zI(E<#Zr^#y6>DasM=Qi}2o39C4=+IFsrk}Bm`N#Ae5<1Nv?0hP4Xz8*8MC)9D5yyY zps+aaJZ0#D;xJOK+n!n$IA6Nm;KXbc;8kR+e2VL?-P1Sh2|OgU-}#FY%PRVaEW25N zfZV_{QP|jtCND^$z{u;Bz`CT0tC*Ej>^aQ*d4qLS@hnQ5gYV};o^s{dymAkMfgMe~ z)~dAT&mJUW#^!oQ3wF{8jGg%@wUphT1Ah3VXAg>&nM)H_R-Tqw;zQPsrnlz$P_B>@ zv^X?kE0V0Ro&k;4F}<3E#rm=hbkecbEkCjem-Fp`=a~37&eDu@bxJ^#xtif^oq^B0 zcUlE-f)pmELV+h}>0Am&mbvN^>zAVD3)@0V1H0f-Ow3O;6sUOmVB9apFJsiXx?a%a z8gUFwb**6kqU4IMfDwg$AT$5Oln3B!YU*8Z;$ivxSb5zH96a~i?gbA7@+lW8GiBQ` z`%G~;v)b=K;lZ1lTT9Wh?`NqzbslcQw3F3kUAwj{+X$g$E-sEWxNM{3Wkr6R?rf~! zw^mz|Skte|vo^vaQX?p(n4igaXR}0m+V%*ReccI&w~3_GZaCcLHbBE%nIOiUZ)I|P zP}}m%m^>o|n)EH2W=jEPJ8Sw1;9bwQnSkf^@QD-wuM9L_NAUv}?Igy2h_g#vrDx8k z@~v&&c|18;Qhs4rW0EcPIV9t=g}GzEL?X24z9OSJXg7zcy%|()w?m;9NA#xn%NM-w z*5tDIyy*l(*)p|xrlvMIJOw1h15*kVk?#WEE$4`tf_R?3=#3!9z6Q;o)tN~xnV)*yuy4@`8t<&#YKg< z6u|+B#;0wrAhImFY>F)WYT2{qtxZo;xBKn#WTGm-BbI%|5!&_?qlQPE=CPRCX(o*Ghv-OB4Y6N-rnrO z!cSN^hQYv+St}SPSjWP^U<{M>tJ2p_%VDVKPu53F$?}Sdh>i}S4t=Ug+tr>vshB1E zuGpj`eAskZtDQ4EEU$8tV`^#&9CY+xZL*~W>qXC&s}NwKXxBMl$5UN#h*+M}Vy5jP zV&{NrNn^^Oo(#%Or)aS|G>6u24i5Mu&2;pb#cP7F%m4Dq6aJwN*c8H0%PDPvkzrCb zzCna8xX21EpZ`^G;?T0timdikUogK{;C}sD_?5_ZmaMY5GkHw*M^Np}lBI0kj>DT# zQ5y}tfxR4Ar9uojhMqi~t!!+5 zHral*{t97cHkrubPf1HFN-?&viTTv-hKGif0n@t;45*fxP7RI018?zYEYTD(l$_`z z-8WFDMl2FmUNK9ONn#b%Lw`d}I``{WLuS2kmFI`d>>1xoA^%43~o)e5H_NK{iW0 z4wzDk>zAl!9+)mzLX^3}-nX1YRMgpSE$YCiOCUfkz+`EgDCCP3lgWvRV9Uv>=dba~ zg3BqZ4iOxM^R^Lodt)vEDc)o{*+Wh=ZqB^TFI28GT-6ypNz`w-FST8C=|BBjw7CTm zj9sWB|NcIEaQ#D+&Q=xZ+X*XxK9= zmBlZ?$E$-LMzh#Kzs)>SHZ_uB@hA>xt<5}LCAfGcJbNWDRpFMD99cnw8{5N_DtX5n z*M<(UE}vjSz38(YFA_7jWi&eX9Y*VZ)YbKT+uPe46BFYAYa@|@#eGDmsu`@!52#2A z3XKHn&Br%?z91?oDLHh)ya<1n`LAwnoQ}0o!9Aryl>f+KW*>j^#DVyGkZZMLtZ4#h zu|I%;fN7`+%JVZKNXR7vLnU8ioO`m;#KSZMLON?Y7)Me0(G|zmwtk`6U-UzHny%j| z4FDzIRC5>G98U4nQ}AKkvB_3+2ugf)w%`03C41jE!?1ce{5Yj@YCM^Ur*C(3+chy~ z2+!K=AE4T0&CvOSr@Y-ObJdT0cFD6ekn-V8dHq>1Rr#k!5&vd8;rZg*L4$2}nZJAX zue*vK%|1z}ug=6{-T{1TzQoo2#KZKti-Q)T5BdCr20aEM56Um=l2oRn(_M_+(i^&K z6O5D(xWSX6J>{>5RGC9WhQ0@a9j}3@LT#-8 zwc>lpfMCHU$Inp2Yw?b_U|Vy~#LWINQ%4?ZVe$DuTVQ2JC}H%j^BeLaE`zOwIb}?z zQWt>2NCgX#N(TqFHYVlTD8b+@*JTxI1!!^Tp_k^buOa`|k^jhzI2#vutg@PJ{c4)O zkJ%H$@}rL+21c0nV;}oI`RFFje4W#8r@#7#8^e3pj&Y3nyJ_$+eU7tjD=Mk2b=h%H zfry@A(Wj@x^?s$nOXX~zRxkoSboK}xj&y?A%8n7bIGAY(L=6?)r=4q$A8d{_Boiki z{t>Bb-aO^&6tfXBT>9i`)-d-0e6E60L?GoZ(p%umewvxvo98NOc1m5XJb#`FB)GV@ zw7$iUH{%3LD~;AJc9{@7wi&}*66)&e;#G$hss?EvG&R;_eEUj{#-LW&=}b131H@FO zh7Oq7^-q35$oEJ=klhJp2Ex&G){25Zq7!8n`Nh^uO7eEN1{DDVt@Y>T$n4V2nGa?AC<#rC^&+&miMVK7-*83;Q+DHUhE9jlbVLeojf0HHiuc@1>R5T) z==2aPQz=-qciOjDZ)A(LthO@?F2O)w>GVe~==7KtRuu37+)EOK?P~i4g zAuE`2GbdFcQHgH);^GXRbYgTI3jtELJwIWwlk`S_ugd(x9&TJ$gtUEIc0f1tzP_kg zz^7$TxA(!Lwv?cu}H9AwfJv48DL$H~0>jpV?Y~J~2vSi?) z#Sun%t?kB6ko5b8`P3w`wQ+|eXP3@^Oy?ss1kY0G7|`PQIc+%yzoON)vUSmyk-)a9nVN1FPF#pFo%sq{CO58+FX+N4*()?m56O;e%4B!L)Ezi3x|yd>(}o`BHB= z%@q-uU9_ALkLuWsJAmj@{i=ZYT$T-UDJ~)w#Od#Wz zWNA)0i|e3Rx-?0wWxhk<`Rg;BIa|1H(cIxoyjc0k0Q16M-T) zf~o=~3<4xsavHiiK~+a%m6>72Q4TP$e&?ePz`&U1!J6fPym?~`v&e)<&|p&tghZ%m zYW614D=u_+{zHS4gsF&&!@~wsnV;11bsP5e0d@$CD9rVxe* zYM>q+1>@z=UhR!Yqy`yTR1Hnay)Aa1P(ce@RvAs4p&;*pBMw1wa32^I!n35-7It7t zssJ(tFDV*R2SAreq6`Qo)T1h`G}p8Pp*7yRsLn)hxh$qJ1+epsqtFUGR*;;dy72wb}TCaNJ;4sJ`(lv24hoH5p89Y;XbOr_Y>>DO^ zNQfSkNDEluMVFl&A_ok7!${D5eH=5esd=_|O^)10s_rkw;c(jKmv}!OB8hi($ZzR| zXr6}?|4G7g7PTUaFLq(t71Pz8j_^xSYiU5YGJ$DJaO&v~s&W%V0>}iX(+Fhbw*IoV znp22p%$jo+9Wx$BneSiuc40wI(HaBs8S!|b7ICSmRUNZRT}6e>{>R?cZwq~s<=)0`8L zxOqpa-;`XjS)_lZ2;gH9dBk3#2x;s1`lcA$OtmwL=m&W?J`*5wdRxZHMVM>KA|IrgCL6Naivje2yqCFCy#kOQ*NF z&fe;>pQyzm;$p;se*c5XrXc1^D9eAA?HC9fB{?7W3I@9qVX z7+WvG_Z0=JRVCi8kDcW{(LbTgO~?CdEu$(iJUvVTW+RkvuSXi&U5-pjD}>C}|WnKABHFL8`DL z-)FoX*wK~nXU6Hxx5r2qndwizE4BI4^-e|NZU(0@9M^vaBBEli(nm!{%a?0N6|2J> zF#E^H+Y9GgmD;sIMMV-#rwiRV{LXUa+%UWw1&Gc7n3$LdUF;WYrwzaV@LdLyJxJvr zw9to?3^NytdY@UqxVxCJFr63}j!w>Bojl+A4zug&)}|r=iA`Ys=4$_~*S%T)XX2;s zuU5x?p2XhnRNLMI-j3cK+ewZZ!6#E~!ClXtOWKVUd5l!vIp1{+VCx%jgQ83Kkp5Os ze*!EZAc>X(%hBN_3V1?!kkLX<5ReL5m;f+wqSVF)8Js_;?aBH_QME7@Tg(aJ@-Vx$ z?7aj-EGv&`3zF_Fhgk{ICs~zzm9uJ5tgvo6-$#{S%|C+8$$zP0oPDu&O-TTqOkPrD z6Dl|krvyp*CRy)lmt2xXHvDAMieW&`3izuX;FWKv^`4Z`3-#RFmnDNAv9!fhVKL|Z zhYzwwMl|hiCz}hkCQ65=r+Z=Kc80rSnXtwL4gg!BBH>UQf{-2(P$ee!&KM|239l@h z@>{ZiL@h#$rp`qm1EuA;z=nsqU=fW*O^=KQDb3bMhgVBskffrB{fWz@bt^n8j{6gn(}NEikCpRrSLDw(L6UI_Wb@WQD5}YnnaBPwT@3dg&Ii& zcnnq}(YOe-7WEGIJT{7sruInH29nP5%amC0s)IX8m~QmY2Ima11!=B4i-{uqfiP|* zd<``nVhOaVP~yP#?=1>MGd_a`MkFZJI}qES6rM!O?^Ay1OIms2wT;Fv?Z||VLVg|L zEmGL5o!R1_XQRKZEZpm_p5|{$xu?44fM=f6k!U+7)G~SGSH2`J9;GAh87iI9qbw}Q zMSWj=fYlZ{&-(dqc;TEf`4Zd(x3F0+)tbgzq=MS(9qgCGZ?Ap!zE4yKOIKFB+a9S} zwE=Gx_mTylIN1ME^*0zfjO~Wqem9YSGaxl4)8Zl|E{a*suZ$Lg%cQ8T#v8o*zMjO_ zlbdz$afo(5ucqa3fVs8pOLQ#ifO-eNK|v6*bMVy(r7769C|p6i!x#%|Co~1vBB$?4 z%)LSb@QvCz^%jjq99;Nu%V9!_^PaH2;(XAfmV4v~!i2sx=M;lu;gLF$EEk*fuJJCE zZUo|%<}o4#(NCn$MYZRU)21l1x6FA6WxiWS9a(Jk##ssGOd=(~1+rTeNzgC;_Drh8 zLK8PS0afYOhXLHf-pvh=U(q!3uyvDWWs5C>|8_K@`hLpuH6nj0z z3p7hgB`t4s&p;lv;QDqUJ%GZ``i>*}pqtYz4+OS&Gf|2>qSImX_5&|b;;~m`cymM2 zsP0ovf#n>F!VZ!kj(g%BCJmBCvT2y=6la=wQYL6yO*Q3qJX8L)?qO^WpytZ(iZ}yvi`02OnNm({} zPG{sk3kZ;{UeGW5>Jm`Wst(kgKL{II$tm|X+3L0B8^XysX5k{x{H${&t7?sGT}2bT zbq^Mm<5-tXM|gsl5Eor^A?rm+cPGPo%O-Gh#hF+KO*b1ju!`rFce^dN>xkb`bdnl$ zSNo_Qk$6Z)IO&|0I+if9>%cv1csIoRS9NK*{zCX2O9QS}O7C}CJ)I<)Z>j2s?#Txf z#><<833!s?)|=SMM~%CEm#mv8cH zg-gj*7XXumOlpp3;>wk$gn+;GhHd%A2b1^BdkNYPqD-NqP03eHjG1(8j~sKI`Kij$ zktz*p?$;m7bb4(Bc|&$F@G-rnL;8+zZ~+eyjNA)QhkuJ>da>2)NQPi@BGt>jERc|h zlUHQm&s%Gl4%&a>22&V0Fq2_4BYtFz#yjo4ck*=ju-a%66Cdxq70Y)Ph_gHj^EUer zlcOMH2z&FouX+l?v^0C4t!yb7*;?Cd44M!)B;s#7h8SzFFY$LLs?S8P59i-nPiNPR zw-H1rcjB(s8RXn`>xH$zk4-gSwkyVejK(52upQ|X;>q>Gw3Rs zfE0=Tt!?+eQp!ryDLr=lAln~A&)XG5Kx$6Y-t%Gub}{2%ysXR zr^Xagb)=mZii?o4X$<+PWw<~MM}ve0pN3A*8z8X=pb=Nb492B~N*Cg8+XimVEH=d& zX)EDNsOjlTl(uI!<$aIw!Z9>8ESe1;Y41j%$5cW`2rC&F zh{CFwiUI^CBurbT%+rPx6chx|r$)n)4J6_eGCj>%7G%2w>oRJ*u0Ja%C`1fybtJ-8 zdudhc_jd=gUR_^1)|RAb!=?e)6%CEe=Y=?{vSwz;=N>up#a3jo&UJx#%sIAnl z&aGyP_RjOlCO-GkYel8Dpu;l9v^v|(X)2@OKh%i*H+rIn`$}zs$$Hni8VK8p~{E9=0nkmCe8c*kbj9g z2Si&AV!Jh#dkk`I<}8yJ+xW50FE0(nQ5_ev4_KK)BX{JJ6)hYIfm``nXu`A^?O@aN z`^2pDdWAO+2R+n{6~@}pv) zCyL|8567fB6lHhEwB5^foEAB^f%7z$a;g3W;`qc505uXs$(UWGviHh^{)Ra~`zp&_ z3BYVHi0qj0TRNPL zPBTNqSKb35D1Vk)qKn_WK4^JclNxJQ3hmgDMyT5Pm6lTf_Xa2NU~$}pyosQ8R9nqJ z<+B47W0(ztRTVXcR&{9(i7;yO0fLHvSZK+15x{e6sbhiE(Rv7u4Bc}g``*&g@`Foplj#s z6oE~-IXEJ;gyIRm!O=iioHh!>`bRsEZbc@$;e$Uwviqi>(NIl&Jz6Nf?Yw8( zwM=_XgzsxC3&VfV#18C?^EgF)L#rN@ypIXhEA?DxYfrVbMMYsNaH_KJ=%q$Gr@`@%K=T`i6{;ZpPG=oPV zPizcr-RT?{r8;@Y@6qf#_|NZ zkdj>qL?s6Fl3!DFd(Nrc%_EZA9_)z?TP(w@E{>hE zE53i*h!Z^BW*2D*^S@g)XDA6vi#uO+cV*W4UoU_s{nLT(JRUj4=8?fGgYPq<93&GE zN+Gc2tgMmB%LKz&G|2tm1=7gzxovICg1@T)kPwWV z1(8M=JcZh7-)p;~qvUu=T%TGZEhucu!YGdcp)0FoH~- zbW*;<_0!`~hronJ7z)iQQ-@(OaH1GQT3Hzjwwe`Y2(rq8ML`#`ZHx{tuQ-*S$buH6 zLERzV&?dbw%sG}88zF{giqFIyZ+-um@cVK|W*y^D#=yN(v`H$I zJhnRd`IohyXJy-&Au^KWM?kD7uHguiT9hVGBEW2M%%&0yyqQjYoAI`p4E84g%@fQ0 zvj;X5a6tQVn>S(X+I@7uw2T2xq`nr{y)rpgF$KYNt* z`=9!4+dfYpmL-DmMZ3?qvR5^DsvbF5!|5Gg&fqmQc7~!lr42QoiTuhk)ecK1qk@k6 z;lR)pOlggOg&RbHaxQi7BKlnu18WL519>AO%YCz{;;8V@b?}B~M8QL8rPg2lV~r{3 zP#o1!qLJG@d0^bc~E5>IVjL$YxBGUVXfY}$KIi8=WU-ZFGeg0F{5Et!NjR~Fm7zcDtwJaQ`Q z<5@GtdQKexR5>5rrj0KHWb2uH{NS#(G^wV^!p)ezo^2%{o@@C z$dYYh>zs!L)jTjv2O$(pl?`)lOu<0X*4H%DR4A^_(L1#J=qJw1{lyn?0==h5g&w zQf$%)B`pf;j;`4r`;dIV%O=abm=bKyfEXJd5oWrU+!A{?Cj#Hmu4ID`4`{OWrH|_4i$M5;6gwAlyWc?tK=$S7a64D2r`{}2g(U?ck z#bt?zNMIHN#>Z9o;x3SnpS}3AJ<9(1T_>Qt@VbxdowKv;(LAA_sPA`EnN2PLI2r>= z=&I|wVpCGE+uPeg<=OxX6G$ACw`yv?N2YEwP-X_eI!v)MtOSoe*LANr;TeB0rL$7QB>UuZ} zJRMoL*b3@E2`;rMFHO|YUOm*!`DtTtHa_UB= zpV7XyIn?`O6d#i0gAx+cCUmi6%y(XQN>Iu})flwxd1QmjQeFWTBx>mQ;8)<-ct8?V zrCdmsx!x%v$5!vd!^!qz zDHnC{>ly3I{@HY6Ig;@W0+{cnEDrE=mKKgG+_SlIFKyQ9aLn{a^CqL>zq@lY z!r93r*G9^xOgK1Z0T$^}E6h;ui!r3|0YV8F{!!phw=G)vEC53%zdJtRdPYwe+>v#Icv=a47N=j#pHuh)~C zHA9g9t`hMs4MC4Yox)n&xg;$;=K7+b zR1btKy#q#J5u{*FQ+Bl**MdS^yvc%40&-hly4ZmuUYak*BEst94{yf4z-Y_qbh+Zn zSv{4i#cEM-Ooa{Nlp(}kvj~|2(Z==7y z`nZ5qk_E$_<_iM$vYFGJ0V}PVTPpJJkMF**8qN_Z04D$zu3qa~(Ov_evRJ}m>G4#6KmFWY z;94du{o?eYbHIp;DEDJ(bL{83>ExNAn)1P8`AIX9l*b&bg}cL8I5EvsZz4C4E(^IdVyfezB~Aa7noP&a@8h- z{7qi%>fT&fBtOH&HLA9v8hrC8Qb zF+)}Ia10UjU-I1Jo8L+So4;ew=g5uGa4*=3wd5*Azb5XaaM60LQk$m{n%27FRnyvD zU|>Z0ofa3JJUHg15BtTD19B@}K?x5DAlg7KJk~&jhli)NrRH|2#JFAY&!<8!gZ#it z-~pZU_{SsGGwI<^2)=Lsc@Ss#uhiUc_0PJfP(A=b|10eEx6URt0t>*z1zeWGR2JMi zj62CklvPlW;DB0y3y8)O0~+B3fXUSEuLb<>*@kIKS|MpG2NSo~Qb+uC@`(Ul8>97m z3{w+VXsiM%j;t|ncf{_+d%s_h%Ek68`iNv?W^5TQ{D`n0$z^+pm`#-D+(+1yyoG1j zM6>l?xsJtZTDfsmd1{zl zpg5$t*;uzE4<=xP=jV<%!!H^)OyHkVzJ`U!0)C9djzw=W&}&wOJje}YL2zA#YW3q7 z(n}zqR$>Vt)^mVt@+Osr-?$(>XSO?tkL$gDX} zca=uOR-Z+yKw>P|(#tC(;yH_;jUHJ>l48+Ulm+2f^}aTUK7o!LqYH0J4{=NgnTk5Z zC#)oeUJB^WBY@~H@*ERVCP}RaUZCa*Rtm|UDSRb%$X=>P79X4${p z3i-L>GGaP!?eqvVW^8?PryV+Q3X14@Bt2MsMklwDV6o-nX?-RZNfcAMS&Jr8G3dmm zMkwBT?+#(Il^88IXtd_;zWLKV^jgq32`lV#884!H8(EV0z0jd&R8r8f;O+PIXOpK# zkHYNpx_uzjcU*x4;|Z|DjsqJi4ZCV%mS_rT0FI;mRem zgw7HK7gM5V4L2X%+Wj`JlKXo*P92AD`d&%#SdySIBo9%ol|?9;hFd@>U=lf#g`9ME z_({A~q%kjpY*%hH*b+WSHHYZfa=q)(^R?Sgb34|62NGGgiF$V6KHE-8r24X`%)hr= zcRnCXN<9)C(Fu8s)_vYAY&@uYVhqW=+&J-z*9h75LmDh~T!&4HH9u$=Tp8V*6^2|2 zcRl`oMx%W;TDgwobSESh>ALH_8{6i@LiBd*Ec}&SxOHK6M{W!^jCm2BRI}f45~PZ{ zNKj#KzJwZ&t|;&nvGePSZ{3}6{6jYRe%qhI9qSzEy@um9Vos~&)wi)e+nRkWQf~^{ zi83v3m%yCMU%Q<*&C4ly;Nwj@N#DeGKevo+Ir>EFntMO8=ss&{B+?5;$ zo(E4T!FqLq@`iZh(i5bkFpe#S>@VIbFJ%#!+Z+aNYHjiAyUtbR8X_#-;V}wG_7)eQ zQ$hB((DTKV9Re~cu38#Yag*cfH-~bGl=ebjVhm+?9G&1nyQI} zulWp@_t=(|+-@T)-M$!)w;S17R#mN zGpi(bww%4MV_1{omsBeAYyGc%Cgqm|zL18D^iZpl9<5IFVPd-Op4+U51u*a6k~u65 zK=}4`VrLhrWcwjTdA7?O?{oAt*Zb{1KhZv1+9?uGy2i=_1jO5?AMU5de(?Np!e0W0 zN{>l)2@OF}ucANFU#Rko3&~nZ&#wg$a!_%5rZE1Wqnr?e+!Sz+-Hf-;%`oc)G$1-<;j9c zAF3S>m`SIOpAdKDvS_3s(TJ81f zZP}~uZYVqgs_sRvpSvojGi!T%0$G=;PkZ!KUjn)(IBq!4XPC%ROp!0}fB+$X39E3~ zx>T1y&5(&&dg=DTG-66f7N(+wx~v3#zy2{LmO26fAr=IONA;+rk4V)(kDidj)>e{7 z{04f~Y63*A)?xZ6)CZ@hw$|1W0D!_kkAnilusIbl4*)F6ltZQwhGdF_Kb))}c(EkR zwPBVaP-DsTQ5@0#*;p?{O0?=%yqz{rX19{4Nc+(B6ijiyW?ObioAxpc3KUPC3rAqR zYag7|Vk*B~x@k5iA&@~X2zC{H6vtk1CLGc8?R0K-5tgy`4b!CwheL2FG`0vC}wvJwXu@dB?%>KB|zW}cAHAY_}{(_{)N(W7JrzEc_X zJ%mueNeeQ#&6K=~93WKztDjPFo|b-fMp^46s;dliV~2z)DjDQe+*NRI|5~r;-QXWN zI}q;#ppsH&RF+YEp2{4eLOm0brIFrz0rcls@KuM$;s}) zbm`>{CfA$(^S!xPX29zGZhI1mjf=g5C580X_4UdF(l0de=t)IsTPJuJ(Lf&X$Y5wle08<5d9%o5*HWO z)7$&J7md^Xa*>p;#Mwk$2sn?u#GP}8U(9T|^KSd>e@Q-`I3WRz!3T_3yuhDnfjVs4~Cn1zux};d|DV?O;_-4#ONZESx z2*jc#yHnI;56X8edv4pWtA~GwWL07+wz+at@PFBErk6GizNJna5{`}&K61A4@=Ie3 zZj97gV~{C*zxi!tc4vFQP5>$a?)}TG`bRE5G<-C3baxB4`_ApXgM;4|gb@ggt-uWQ z2?-v6WM(;VG{@yVZF_BZ;I6)V_s$%8F$rB;3$-y+Wnp6~YHQ0bSINB{(eF|aOWBY) zXjf;#Q&{k-kYU2xS%2ydDef!*+X;5dLBaBAnGtuDA@};;Cu`wYVJ)dL+{`-fq26X0 zV1k4&_51^}@ZofU@;~%*oPJYH;5h2?vZ}_gXPDh{KC!$6YQX~>T$aHjwY6>7S#2~I zDr=U(4G8W}VXu!n=i$%Do%IW&z!Qgc^FH25_l7cURbKR`n!4P(X*bn!Ddc*mG;L%A zVV@7)Y$Uh%`6QYTl@sPR`T>I`IE_T5qlM$(;9CGrH z_9o1sYPOcial2RkZTkfsZLkVbA^%Gvv%7Qsn4xV|{Dw}yZm#G`ZvF zM@2^>88(2NlfE>RPk9ueMzP31`BX<~L?~<=9nqnD*(`0w8GtgY)A9q4n6se#fPCvJ z1K*;89N%?6XJhp@7^u;N@!q+5nj@LfVbnqZ1 z-!q=a=6_7Aw(v+*R(2}h*-?Y}y%5QUidC~)#eXI|n1sU6YWe!0jo z-L0|qYizQ#+t#m1|2?6AnZ@Dq`gPOisv4SXRkw^ZI4GaJg9~84cdN^5H39XS-) z2xOTD+JJ)tGD~tpL!b)nPJC*FP^lM%laspBdh`uNpI=v1R{xX)-B!dGR~9pp{Fx#%>TYm17~ZE!&!#e=uN+a$>Cr zlhfL2&Zgusz!CU_nbk@kze^vz&2g>PdN}@W$FWCubic`(*FH^qv00`V5cCaP>*qsf zTQ2+h73jEMm&tp)Ok`GWZJ4Iq8Vn$R2t}I8t{l`a5S5n}XUZ5G;^OgT|DRbA&ncFG zcqAchH+q5Id|JBrz9N}bnmHo56B`kk0g&sWMnH{}0^5AFXiFipLhiBo{VPC1&2xh& zKY$Y6#)BCkqB3#TnTv@`oK11*fY@A2zqp>MSEkXVD}y1{uC6_vCU%eaM(@PeMqu-L zWX8>W-!qPWn}UjGA$co39swOnq^SR8l#m%y0Zc7rQx#@Fkn5oeg<4Y?rnllOxas=x zuUk@KAJIqMyNQ+UnemQIL84F*xu*`it5-J4c5V?FAcOVsosujUKJfo)YAT;Y0#-(d|63rN4?{sLb@;=;e-SmhYJ( z@rqQY5Rrc%p_8i8`-(E#@Pe2pD)CT(WJrWLi?8)d>F?-9*&jA~nek#Hgd^V}b)4KXmi(Z|;Fc-cc&?wgJUcr&RHz=a^2Qgea0 zsVL|f6{K7ZQ*+{$RMu7QF6ECy?j`Hv`2ftq@8W%A_-LqhtQ?IxM`qvuwQ2t_ZoWPq zZN@-Z37^{js&X+F@`N@MLfB`Thz%L$@+{>rowThhCh8YGM>?I42r_4AY1*C3$Uwe5 z>cvYY1Q*6~JQoGzo)BUgp3*~G<)fU%n`6OY1zfyEeNDO?tPw<1EL4E!3p}#W7kE^3 z<|!5*xEm5`$Tr+7+r(50yEA^?>3g{Lg@9EOF4QaE=r25xV{p8ZLH1k)ZslS|!hg)~ zKVO`k?s&JMAcP&X$NBJvZWzP$=;Fwn9ntvW%Df0OqYTA*Af%Q;#)8B8uyMiwz!9N; zlNmAq6%rZn@y^WRXt;mM*0+ZJ8l3AN3dkD%YWax>hx|2V&KVT~m5>^8jGU}RD&cm2 ze{hLArynx18BWzQRKj_z*Dal2L&!TzF zlm-_`qmdSWY$5X%E4>3W>dcmc)MKN94Eod>%Ub)45NYF;xA>&P5cN}P$r$6WvxfRA zvougbLnOTyz!tQHT9r`K{y_x?92RD;c-?<<0sh4gcSERTk~(u{jF5uz0SJXe9p>K? z_tnP^b2SuCs6HKNbo5W^Rt@}`34q+$la|?jX^nn@Q+$NlKAYzV$oTr&!VbjvymjWS zR+x;<&j`yrt$81J$E)#ycqGFnb#k9e4Ze?0vpGS#DDA)aP5&HYb~rLr2*}jap5!f~ zk)ykS>+{#E+E`Wg@cV4!DK4L$fIJ>)laqDFcjt%$^!$Kg>@3Mqhxs{R%8L*JlZXYEB{u>Bg2dKf5~W69pcv5Iv%IH?q81(k!qde+UP$}`O5pj_tPK^U z{@pgVC;8M=p#ai@p#HJSo&@+5rCZC2I~rhnfg?Q-q9DKt<*Wbg0uX9ZuS`u6QG)`b zJa;_m!@t{)p+XBHfU!-6@d}Pn?*I1v?_>Pq=LQ$?%m1~7_E`g`{l7JgRBWvOwF#;I ze|iwzzn8Jb7%uue(N|3p^X?%|RoyIDAY1m0t{<9V<5#j$Xa0~?N2{gSwz6fRsF!-H z7kKxbXolK*o7S_n_v}WrO$Lp=EHMOYDHP--<$#W(DLK!Yv9TY8Ck8R<@c6*|2jw76p|ZNOK#V%tD|>9olPghT zk~z0>o7CKtc$4)1ehY40fH?H=@zLPRUv2w_CC5Ob>_e}VLpZ~TqD#A z#+tPbO898v!>%m4M_`3-=#YQ0Gx z8&jnNZS^UG>9)4FQ*5%kx+H-4L=YG0BfA@voR-!P6nrahoYB#BRjhiPXEW2 zqrC`=ermk5`<}Y4JD3IE&-t1c?%4Q>?dA)`9i2R~^&T_3YBBt6o}Q;7bLD^!yy#sZ|k zmB9Uqp{vfog2boRXW4KDJewiiH#T?YyT(CRb9D6d*fDaSYHRlzhA3ngylOkol>r%{ zN!X%QQUgGb1^W2AHpwWkDX2LKV!EJAVexz2!QF#2xxwL;e3RT3h{*{27XK zNYpdyrVs=3<1k20dfeq>mkDo3Q9-5bHAv!TDg` z5WT)SqubX+qmX;7oQjLzLO1PoT$M~QEwS1cE6=b-9|=M;0sN%yD3=Fc`fa;{$-DhT z+25bGqPvYCFt*mqraXS?gp6;Ku>LBRdp|pr0_ff4cpRwZoKwN~DxiBOdz$LGc6P;* z<7wxcwNbm@uV|kOf2Q7)^NKO60b#pMR6f|)m$AK!YB;d+Q0u+@sBfFG>iNT`JiBeHaf+;1#LG}CcS7DEtsvb(_HJogr|nsl+% z3CR;39GH$;dYY9yB=!)98}b*D>M0SaXF`2XhJFV}^kK|2N}7*=hMEI0jwZCCRNji{ z1G-!vbqTiUsW1M^&?L0fDJj-(Ii4LI70kk%OJoXUB#DP_Yt=GBX8j)9-zXP_!5W;oC%xOZKc{b24(Jp3 zSj{hLU@zSTsAqtU)vG(>Lmsv%^J?xc_AdN#aDB08LyA|{}+&`B1`SD~8S3*>%OxQOJSd^1oe2n5>vPS0KJ3AbSEO`lF^ zHb$SxUT)dh8xkesK8QYHs0G>;_RzA930X*kW@;H4f^2N))6>&+_4Ks={<4d@m#9X? z@4&DhSOOmL*rNdn@SIn;4T8(r#B9VEO9HQHSUB605pm-{8;g!^yKF#*|E-70)~o4t zkG?wr-RzJ!R^bHseSPh=K<;uqv}YFK?)WCo){khJW97-1;OLncyM~fRa3pW+Z7T{! zgl}+WVMaY91~uBMvoj!~`9n`g#Cv)U_PB?`(N^EY$PlqJbhY5{kU9MeoXM`QaZj!h zN}p1%F%a$CX)sh!w{6R(zn$XeM3ug4S8$ygP0V4KLA`MO$;LG`R6y=^iOBc$fayxK zW!grqoNYYx!;hg*$U8gox;IYc%avA5%m|(Aj)7kAPQkcb2dgIXZ^kL$q9U-*8a<## z$G?!xEpd)lChimXeZ!Fis(Uq;oduk$DwgX)KSRx)k>1R)-+JNGPW2^@=e|22lG{JY zLO)p=kNP4?9IK3V(NO6RPfCi!EgbflcXqx%F~7t9;$D~mDwl)-t#6AxQ6rB>;D1Ul zmbYJpA&HWRo<){s%FaRgct z9L0-(qfb>o)fbe|bJcnl-YCuuzV{Epj{lfc@r$K-)~k3_AT29t?}T(wbAMpREE~OL zuF>Mc8;O?cF!=horlhJCGyHozwzt#Dj|l=!EyEsk-zC^!_3<&X`T0xU*I|7pj_bEb z-%i5iyi(y+(M|f6JJYMiy)N&=h>!!P=>ECz|NWjgFdKsqjsW*pm2op`A|&lIE{goL z+L{i_@0TF3<{Ny&43Mg-^iiYfa*oPZ)rOI=Zi?Ks*2tL@uhR zh`Is4A1$)%O~>=)>$Sl5nXqPflQ;6qgcH}C(++2#yxe-w2pQw1#Gj{Ovh41LkYBe1ydzkhe53q92Uu12#-c-%MjYxhG|7NX;%|bVOpa zSln@CdQf^7szJ-IZd*9DL9w>DG$_hrg)SgCGqt1`5ngianB>Ywxop}N?cymJD*O`> zLlCjt3Uakbbg~?ESo^~UQ5J1RuOp^edaYMaB-=3C`Eu&TvJXEONxXYC>gsF zjsdu$S-!8^m`!Gu>fAPWh6eaCL6_6c0*(%HK6KhVw&=v_>1V^p8U1o>5%vLIxU!(b zcxLv2;v6}o6*?HV&9ARFuN#`yrS|Ev7}UO(I{tP`F*}yqla$rQ%?g^H>pY(;ylB~1 zzB@*Qg6fsXHY|P_qWL`#c5fT08TLna9JVCx#ztWboEg!~wX`U}1svw+amge&VZyeU0sEWZcNg&n@-D;ldtzo0A9 zuK`V*-&8*e~uz(_a86Q z&vg~zE-(Esu6a-}y)_qRrTf#ggphRXzSc#4G4kF_ATjTU5h$dEiW_~tQ8v(3mq}ZT zBJXTyiup)QJ>ZcgnyI5cx=TenJ9*?_O5GJmGms4$CIeME`G_n0GT2k)9C8Xha23oo zTbEL^N+ul?_v6v>moid};b$jfHF8MVy8^G1?^O zSiTY;Gxs44msU@XbJQ0A>EW{rNDaLs49>?*i?Y%tXha{>WZg`pR?Ww?{AL4AE&LEG zUDTaE(L=1IctKP(D9Sx{5dTUmMd}rO2r28cXTi!o?91uqo=2;ByGXTt1y`{_i#N= zdEsj%XkQtUV^r4>-^Q!m@WQ_+`VH&zuwcl!P&6H$>!?PstW9xlP*m=M>!^OM&3H50 zUcL{(dg}>k8UxEQ@60*sde^p{*mrMH7Zo5}Oj(D0k(7|dRE)J6Bqr2tD>XDxEy}LDS56I>^sSl$YW1Y_;^kJFxl#BYV60k z{(hzJ9oL97gUdNTtb6?)b?%D%nAiRZ*s-Xxu$;*Q4)=oL%>7&cez)JN%sM8j#2|Eq z_kw3<3%@tI@)O3MFZEjwQwg;;^p+Gc{SB{LI@(TZB|SdUM-Hq%p_tVUOCegqe_J`< z<(z@MoG-)SxL-GOgO)ftR!A?`65j2Sckc`cdY5*j3jL{AZvFKjAAhs>0^aQCNbD|C z^87JiVQYlAkKgQjJ}aRvmR-ss<9v7Oa(4h}5+A+W?w9rb85Bsor}x(IXy4{~kJ@N3 zq`%I1gpq)=--;jBa>+GYS8@)sde89vyO#fJ^&IT!!!!E`bk~vNeGZd;L_)fhAImDF zl1aS0O%3q#y*dV6V=}S$j)po0ZqgsoS^t2*GClFUhSmhwmA*o|C1_Jmf_<%Ez3ita9Ag@An^@|i}*m0O;AGztX3`i7Mp5SxT3Fy8(&vjmXL}JDsjpR4PVn)S#Zw9it@vk zI{*oZsL^8Avf+0x);~%A62g4NRJK!L^`*|rmrB_*5nl&$%2;1;@(o<^#Iv>C5R@4%dM`oYC>uS2MT`+|Dv6#S?EW$;!5Gnj62jCt~-7B@juE%c$S! zRuIr%Y!-pjyioVq>wdHhX;?Jm=cKb5E-$Ko*apdrccCj9a=&#!<`2#qZkEKl3J~r> zXJ8jf8jvSwSJ2f~B=!9TdBdk@MK;;L1sB{clpZzD5tVPh^QzL0H!21~m)^)YL9Nd-9=`UVv6HW?fy zBe$^aX|H@h%+#{MpnmxTaKV7u41*@y*Fix+4~H5dG$aW;4)bh4P6#L^Cg!z0Tk)8j zUS1vzpmshLPAmU}z92T58p0)PH9-i;7I>w<*2Fc{BMoNTu8;J+{G)$SRSE8JHW+Kn zm6*Igb>7RnVb)6<5xd$j5xbiakbqs8xlW|r)wJFk&aayc>Ua6ESMHM@KTMy9i4pC# z*R4;0<7$iT_T$8$eJ@cE@#@+VtGkWHn`|d6@4)h&?MkRMf- z%E6v6nl9fy_s%cNpWknONkd>Tdom2}+Q5|)GXHel5_q#9Dx`(MoD?nT!#|ByyL%6i zTmKw8WZ%DqkH#C@X(#2}4}(E;zn{HWzq>0u!$1_mo;#wjP)xqmwsv!NqI7)zYJ!3TR{0CM_XU)FfBskvJJCtOfbqg=T zy@m*-!V;K5m$H@gfe;&O_KGT#OJ?%=>GU{qux)&(VK(-ah=!m>@=lZ`!>b#*y^J1a zL4-~ge`o<)vm!z#ip)t|WNA3sJ_yI5W%W(t#l;y`7WM^ij|l%w4Unsl5hL1=F)y}7 z7jRkAR}?G$;nrnzCXF6OQQsNH(lROk>dB^vdMPkdDasV5BK6wa{O$4LHXB)yd4o{H z`xA6hl58^eio^c;&1P0I9LngqDHOy&Db&XV(RY{uS;i@S!_D4_pym`kp_G10QGvE_ z+r)~y&e`KNSz{O;Dh@wVt`G{v;CNol_-;|1H&&FdQE?Z>`{igzwpc1(Wv@hzfYRfH z2#+Jh447w_gM5^_Vq#bm^LsGs(ss?_>gTjJ;eO^B&weu8yV7vJg_C|7!jxm~ zk2wM}WGvA_+w~SWWLMJa4EQ>OXZL#xh6{&siHI>Gu~}Lfk@=X({Milo&;}7F^yb8= z5epPzL{$}LI`!$rRYR3{L3?WdpGId(XxZZ2uX4%CgKqRla+aA{l?NfwALf+r$C!7r zzO6j}IvTz+D|(pa@xK0sOSHMT<)P_ONrgyBHAAK#v5RJKaq)4^m)k4xOKZf^cH_a? zeMslLU^f1!Eq6JfeF1E}m*n#)ms4U+ZS+IZl#u9=?Vg+SCe|CHAhJ}pR6Z{fD<0X0 z?deREpXaUb&gFPWNr-=KF4jQO_SI<>op%%G=TtpH655vJYlsi&$hH^VM_dafu}l5v z#W^v+{(V=igpc^^h45thJqWgH8byb@gnq(${c~ux+!iY$KYcumW9(t79@%1Af-(W` zQ3Wdpb$E!%7Lxu2=llrye(;Vm*C;2u{xT#h=g?Zx0hO2I9(}$7OFiaNd7jlp%rR44Xm6nsAA&-Xd@kFDzNXp?eVSyA%y&1$Y8 zObJK9lV>6L#Cw9xEtHIw12ho)B(t|COk8`1FX?+oE%4f|WZW;tubOhR;Ejp(#NLZ% z^(WsWe*iV0b(j#^(~!&sUwX89J1Yvc+>4K*=^8G(Zo{dSE*`MG8Z#IF&V5Z2F+lz$)Ur2Z!}rD z-UTeaM2yZaLYR=i@$%rM+sfZ(G2@1x{3%8y|L{)~n!FfZgzDnu=5rwld9>b#Ed}o7 z-kj~mO%!!6#7W{Y1ZWHVA+yq-N@2yoeMNZ{V` zeuyGI3Ld-#H&`?5yDeqgZB%4-KW|YKnBCz7w;|-(p|k`#i`j9Q{P_MH@JiqK@)gc8 zj*Z}B9&4qY)f=h<*UilrIsjXG?$zDN{nfz_jV@7&Xg-;_ItuGmok`xbr38n!*o9`v zeM2b`kPdnkDrz9ronlEs(sioP?C3(x;lqQAxA8m*+HRFPVJvdjtwv4LCL7pb(6 z@}K#wFU5EEnHu8IH#Eg*_8$mPL8QDKg7jqkCjqGeAG&8BxLP^g_s%Xh@>FL21 zI~U5hCM$GFcz=l%)bj!b1>a!y3l{0FLD{ifn%t|mS1gu&yYFbZkFECMlibCw!>q~n znEJZj-oqV6sF4<|!0H1ChqliRw7ycozfC*+5CwLmUbBD?LHHNA82ap;GDW4(uVAmv z65DU#cBon~9K0q-vB|tlyyCa7Yo1K#{5@g0FsrZ54pRpJszfI47<|?YL8B^|XW@(s zJlk=0O#!>I4_(v+b?g4G)YqM#+V6$Ey~?osPAfYt*|z6J>w319$7R>M;V*d2EZhWs zqowXo0Cb{vlMATdu=Mmy11doQkTaE0qXIx>{$hnd2pSRsc*k%0 zhlX^FjpIw0jQw`g%gV~mE()LXfL&fX1ol`aNHXC*#s66mY<>11({}$!ullufOaX%{ zfvE4%VcPRLwz1tMf>Q-Vr@^|zcLC>oH(}^YBU=+>13w=AIjwT!nD#SjBSHl>t?e(O zAaXIk3okK zjX$rbFFbWc=i)*qPyJ4Z5(Ce-js;I@MI^=7m=jEyocH7NJn!(;A{EaM(ozWpRoqLg zrwzcF=*>1&o)9(+{9RF@ZSBAkS`wLHMr4lCuC`It&K>YK+y0ASLgN$5JIEq;WeN6RTXgFMwVl(}@p!#Lmy z>|HJ>Uy3y`C<92iMIP9%4O~Z=t_Y zKR{WiNkz1vJ6L@PbRQEn^%urIms5;Jmh1HuecO;7xl=xH@jLcj;Q-)5HoOyqHTGNZRLSot zRS;MfH5ITg8@cVM*=ie7sK@ocWY98`1ajCjJ04ul_jKnt`NxcqPgP?Qjcl~JBG}Lk ztiB&b|8B991^S@NWxe$Yp-a_34H;T16$W%2r`!R z(sm1V%rq8B$J3d(h3_fdr&EMB4J=0+VR-QZ5YzE;Va$mYFp6EVR{ z$Fo|WP2$`r?Go+HLyUufZp=Z2efui=FHBGUtOU=Gi95UA;~!}CdMErEPZ%5B3o1V6 z5YAV(oyY^ZxpC*`=R2;nrrh580^G!g<&&{I;nZ@KOfVQ6Ho0Hg-k!6!XA7_Zh(*2G z>hN@p!; zF_0|>szkK#KYrqY5*ChLrZE|XoU;bsZlU4j!Nf-fZCs735Hl^ME$ZmdnvF$KDP?0- z{j0)_RU+<*7zci7Lfj1NK>%eakWcQweq}u4Cx#dzZZ;o7wC@{R%INVhO9Ubm>Dchc zF>J~9xx_h&XnO_w^=~oc`O-6S4BW+RucCWj!Z3Cgxi-!5jP;}#Nzkygwp@9~nqTJ< z?YxMtPA2wnuUFPm2oq@){voMspr_Ov@PC+l%jh_kWoyu~WXTpmu3tuA$SWo2b#L}X;d-gd-A@o4i$r>CEs zUI?Pom;+q(Ae3+GDvSJQN_u!>veNg8TH;W{n}{;e5||PQxC7X@1JJ%8Eik_Xg2T4{ z9F~kc+%%-e0vWJpbT>1TA)SxU_d2mD*ELk-Ce4mS(T$L8^kzzZ6I`FS+LUNZ8Lh|M z9Ttlb-LIrB3y_XsM4BcD2Y&(U6(i=v34-dy_(iOM1KN?YCt`y|jO_mq9E(&=m3KQP}BrSj8UlmquRz+;>S zqCm1Z3=%^M-WPsxvq0ND<(#Up*dOqT8Do~_?3E{FXYZdj{Ou97i^<~baf_i2D#8?y zQap4kh{p)2awKLt*%fjMDog{JWRPkw4~Lu0!yc4^4AMwj&HpDn;aA4_IRos;3rx(_ zUM-B~-3?`kVli&xnizq%1E*_Bdb{GAk@`+tJ;V+TqbMVDbXXZ;s-ED%=N@{+FU^lemDQ%GS(F2eGK$-_&V_WtTB*nzV^MM+OO3fxlM#gV|HBBJ? z%d7&qD?oA?wN`UneLeGu3s(X$10!QEuyZO#kMSL_zJE_V@)QngTAWyAy#1CTpNXlr z<`8fPj>9X>=Cd+~LE1JXOF_8j;fUyTQ?Cd|-B;RUr0T6!ileknmmCWWy&T$nN_(f9 zKS2hEjHXeUf%Pn_<}E&$`z?;mjF!}|K`k#49y|r#w!Zd;KX5j`t{Pr4I?A zwa+_#e90j1j*9y%xi}E*DGb%L+(|Tj9*K?L9eH-ONdp^>%`kLCy`7O^UQOyX-Oz&rR_I8>a^*SQU@sx^fZ$AHmz}M2|VuG_=N}j_F7o zO{s+4$2fd-Xwi5>h|^D$(2?gv&CjN%*%=au6>O9dO1~Y5JY&-6M&V%^zE$~Gipq>-{6lK zme|jLy-5OP8qYUjm&5*;?dp(1Uf}zgs)gJ|s!a z1cn*C(TH2{Gcu_j0u=8?y=|irVt~?OL_BAdSBWr}gi?gR5n>=u5!-}#7yA`1gaelcO(52+L*pJ8nGbJZlYf^6>Aa@AV7>H zsIn{W)H2dIe9J^?@xbUh5vl~h#pA36PWJd^>__tcy1 z2s9h52+$(!w}a6cy`P+bd@q?|MaGkGaK2wyIt9@$q#cQe|reERDUfS zq%&^($NBo9xAOif*fwfb=ah(ZAdrVQeR@mx@ktxG`MQjW<2gr^J+z}9zhOaB{Z{Pr zpaU#J*;^}o2YsexHD~HMPkr(^n7l4j?7IsToaTcchM7KGM^|nx--GMH`xfMZHcM)F zwY7n@WL%Tn$==?9qgv3NGo8abGD4k&C`7?BEvaM6Vo{i;#1=d;yjPq=@I_= zgyd?9{eA(*%UeEE+|pRotnNLQ@8cv>oKkoAv>_vjqs_sktTz;DV*Vr!_g!16d1++9 z`lI_>+ymESx;w5cZx@LE3qs??ES~K_qpmBx{fs4Z3|HuvRKA{+QG6Pr!YCnU6i?8k z(1yxf_WjkjyW(^{eF9mYt~fP%F?OUu{G3>#$kmaSMDF@$$mQQm?1&?{P-9dHBMpCB zc8K3^J{ee(%F_raM}E?){F*IOcj*tl5f2$mka`<%xp%B55Af<P$az|(IQ9V3AagAcv>=}%N1|H@GZX4X|moqsu9kmj+5p$rK^^)>QnH=J{ofGp`$U>c%x5s%tuSYG3~|1 zvz(zgSRw79%5(lUyl*``5|imix}C#Qp~$9BaZ2qM4L5S!l5m<)>hMo(A91U|>I?J% znH!@NUFGYA7~WKr4|I7U2^*J2ISQNUEJ*m`4Y937{?ZBk@f(`@h=^PSyEX-Pn!U=* znW~KxmZ9`Sfz)%i=AK2NgJbYgz1c#ZbnXPndjm}62#dr$ydN%BfeW;s1`EC!V$teQ z%+%)eF{Pco)rEP>dWOB~;sZ@~8wMs*4wa(ENIlfDd{ZcY|5o!Ktk3OhQnOi~jG~j&1v*s*TLwP z-g$%B;{Af{?DhGxqXRPj+T`(+?e_R_*Pu~TZ_16-$L#ESW^pTbcjW4m3nc7gqVd&4 zkWG>F#CAB#vHLP_m?ED3Zjlcqt;XOimC40WvTr{tceGG|)p~nKFB9+7tk(uQ<#|Hi zHa?KNJ3p`F;OKaCb~dwqHhNP^(rVS39*9)k8c?{IVB7X2wPfO0Q~xd zv@|gw5xA-<25|8KV0<@MYFOs^t6`q5Z@4zbuO2Z!t>?;Lo>Jp1BWn%W{pY|GVH9s~ zq$`d%a+#rmpfuFpi%m;DS@>Qu5x=7;Q1jQv(#SrMx0PTV>#_O!i5!CSZDlrOvKxy~ zqTNNn{`9=39|cRV5+`hkl0e zbhY2q)!8|%@P0vn?XdV^tcg?jQjU>5G3B^Pw<`9zQ7@yA*~&Mf&9?=)uMIqe%l)+a z9?_l0e~@xRX+rnxmF0L3u?)L?DQc*>oHu@QVgVLxd!cK0NL=6Hr0+Kv_Se0UcvT_ zBNDcbFsahq4LcY*9v6fT3TPHhj_=ji_MPty8m!c6enTcmSfF0Evc#lRiOsWYua{xs zL$b?}5}B<90+(QR(%98W;kwzh91{T`0iCE<4(I!R>dx?Jeg86?&>F@N>DZf)e?}G?6zb=+L&JgUO1c zu;qy^=bt&D)s2r(jN9C)U`E_Y#E2aTQApwwu8_FDpl!skObTzODQL-HBRwyPOUf2i zYHlx~XqmS}IhFw4uRu}7*IZ?ZbQaXM;KGs=qfycr5?&76<*)>K&T!(5(RjVSGGhf^ ziu5K2J@EMl;)o#`A(_SxNMFI#LB*W5~&VRxf=VMF~<}SFFw=K{#%(qgH#hgA2k6^l_iZ_FE z(RZIX7=>cym@apN#=051Cnx#+z6hqp5pW~Bee7qCsb-HM)NIoD^W521US4+0+x67B zfT-?L6G)S#*7$yYe6Xg7Jf4+F+-F%UV0}83@AGv*t~4rIQIcl&LWBBZzZkD?W7A(I znoLGV|EJP6UQu05{fI58YxCeSg2S+;XnAtFgzDpgRf49e+legT>$+^CIYRI5zJ6MG@fE<3C39`EwZSp#Lit8UA_$YOVf3s)`}w?kYc&>(bmNa#|L56yV?9g}9Y&3LR@*maJ zQ`@ZdO-4Ef8I%Cvqb9|Cx$eVj0yl7uy%aio zPTYg|Hjl6EW#97srVe*2Cx24^x#gryjaU#cxSA22OwRbyP(5EHW zl_-kmowRtoe(uATTRG@vMLT(XYl_F&f1UdXF<=nCcue*_AbVwb=o7tI-nZF9mx^oh zjNlWyNOjYBe{ID+3MAjUcksS!BU~S85M3q=9N)GLab0k*{54d-h@B}lS}T?+UAS?t z#Y{{{udJ@xx2?A;S5#JRyR-pBpKI{ht4ftB0DJ)~0)k@s!cv_nLPUWc2Bbf~-|O>z zVHxFONm&Gtu80;XU%sHGt-Zh169|;H09d^MS%nhi0D9-er=QEzAN{vv@_swr2|Zp; z((|R!&^Kh6a%w`M6cDgMLH07YGCN5NY&X;$qpe==O3_v-R$96dTA%R!j4i3EvELS- zv{`{Ld~WuwM=s7Wx*BI+m?Q!nLkDOVHhYB^lP08j;H<6>&dlh~S6*jnPTEInS1XXu zn`5^`7JBP9nocp*Y6&e^HfJI@?(nBYmr{*a7T1EcsT9do%=qkefX;uVbvfk(3xQch z^NZ3!-JSMSC#hw0_Md#i5?!BJl=tz~cxDRl!MihbW7Z^msxgu=}7^RV~LgI0;wuHKJDc25t;ckLlem*sIS-Z6=lk06T$ z!XFZTGxuwfqAUXs{isN&s1sIuzj+FU);Z~tbGar|7kSq1s|YAz470SCz?cJSZ9+os zzcMiE?9DHyd-`8GB@y93n^sZlFT?98jvg`;yn;^d7c@+{Z(=4l!aX&v3o z16wP1!yB0#+`=F!sb#Yb9exCrE!wf+9rXLVXeEhH=Irhn;P$DQbK3a{u2=qE+hRw{ zgegv6gLS<5&D)C$l(dp4kR9FW@I(wbZXS+wmxkXyEb8Bk= zPYJ-c1CX_2Q$&1Y2bmZf2dAfF0tvI&*w}y+0@SF@05NuYdiu5|WnIrB8E-7Fje`#~ zpl1*OQa4-T;drq)qIQ`9VwlXfd21;8aF+)Cy5_7IM1g!z|d@dP32H-4oBGW68 z;#0g~&#ejCScYbwD96gxHMp9qezuLmIJUvA-+l(4K+BATIYGm@gI^ze4sEt^eMB-n zyt=6uF13wi=rniO>v=Z6{lRg+2P0+8+x;6L5s3NVlKgNYJXu`=d*dcXD_xUtZ+Ft; z8OBbMlFCM&T27w+>40s!`xRmC&!T#IHgRtG*C-G1%87Vt4nJ{4SuTCS!`5E-UDXr{ zb8U9``BUi9h*ZJ*by3r})LO<&>cMNR-3X`Yh2PsnD`%v0OqNOk`XD`1pQvO5@F6@9 zLB(OS!yO3bKpVDxr9zgrrm?fzgttF#*DgER)R9M~Qv*uLE84m~q@zS7Jo^FdsNhqH z8&NbffLNE)Ib7w7hrlx(U1y(D-AOOL=MR)E)Ko%KQ88J4r@Z0Ha~Fi&eEr<#XPunh z0NZ^!6r{tszzKO=Iac`E{6 zw_M@q)dR*HjOH54l1xVuNoR-jhy!^s)%-EM%&rcu`#JuiS__;^E~u}gI90?s(!RX+ z?uuJnvj?_udIRNqjx>E&yWrvWGsb80@hq;sQ4b~{!r0%KS%|>HAIUb+LZ++dQRfe4 z8lgA_PloHmx4j0^<=`4oab&{5O}4DKK~q#bP68JILD+KXw1(ah{L47~ZKJ?S)`~0| zMwzC#1xXXPhvSL3pHJ$D6UD2-2Mhp`CaN#N-(cb4kz$7)Z*||DdEXzWm;ivW*Za+> z?(Pqp4y#OF+Xz01vA!>xSC5IGAp&gEzXbpj|JN03nTD~adqUl=Wm@M~PXtgN%bvi3 z62QP~P#bVsLJnOL5m7+?y%EU`Tf@xaqQF5qIs1WCS(lDS9?otK;1TU$Kxq1|o~A=K zTbFxKgUvHN^7}WvG;bC3uInNNs>7PnM6vqP=uCsFUzjohqAb$cZ{j`1X>Nh1kPx%G zw(#nTZS%7Nian79Cyx;LuaO=Uwv&vkb>TnC<`bx*qUq!hGPos?9nIDm*4Fg6+P=?m zpwVXU%$F)kRid}h)(tK#QFAB@H_VP}caD3^Y=^)`DpFzKRHm)TjW2}N*CSmXnPsGg zna7V3m)4G{NEDVfWXEalbC!3noM0eZ3M-*Hgrm~xq!BC9#;Y}t_e&&TQSSF*rP4~p zj}EAZr-m~(?SgSvX1k4Y5i2NHmG+}Ak7O_hNp;E_yCye$mzmN-*GZKbn07++OHcHC-$>b3L5RHMSXJ#Zi9a7}L!-*-MB(VJv%X5bT*uK+VoPy(S?VB`O zrQ#TMDMy=Pzp**d@jmBdt@!9{qzP~7`oWhIYMC7i%i@`1LI4ONBk@Ojr+Ba>wM9xa zo7~p;oNPk5W^=~9gOM(sK+{L|Jv5WBV}`TSFNSu?-=kfs9IPtFKU%uec|#Rw9nOo} z(#Sb9Fsva#aEUIc%p^)?Y2g-Xv4x*5kICs$?OWh*b&V~X4vx&D5aK|wH!cp0&%;)o zg)X-Zfoq?llIrhH&*!hC2M~s4_vOY_hazj_Q}xH zw9D^tPDm?Gk;n5QE7nr{W^o~PDJhtp?#MON<_sy4lQ=%4g0K(8rs1luN$YG!_(m!P(Nvuv&)jL>)Alt-4Tu2%Z< z{L$St%en zs4u6_+RYzH6%#HBgj+OGzEjFG6}*pnm-~BnOOESsNS9s)y7oW00D7fGh}NfuzM2r# zV*vhIOmwok9?yC4IX`$#oJg^CEz={*a+%$=1@ID*CdE^Yms-pBolnzhv&c zH}Aaxyu#Cx7Qx^5@2u6S1uaXlu{k-oI&`r^eQ|}?lCI7d9u1G$C(q%z-S!}z>Nu|c zE{;1}w+<&|(WZ|2w*%X2k5XPdgfhzeHAjSMRs648=peW;d$$hmnBc$sJHQEDO*hW~ z{9iy~Z9g?=zzx|0WrnLiu4A{@;p^(^UUul-2>BiY`G&)h$pB9|Y0S7?hpxKc9t1ed zVx);6E|(~Pfb0@j5fdX3BLM~MJ*y$o4NCY8ZvN>kFh5d?{(NyzB*`>72|g@!1?819 zMQJp-QS>qA_p+5F>Wtc-1R~5GprmnoMjVoLw4d8i%{x+8|M={(CMdRxwo2H=G?Xq` zE_9!h$M*d%zwKv3Jn4K3RF|YblH3F$8)wV=+iuAK`{861!|vsl8c9UR;3e`o+4o|O z`+dcz1P=Bdu^&LAj9B7A+_ zUT0~cnPROEfu-Zn+fgnZc?uBWmC=2x{oy*b0PQqPmok^<$Q^SFCv)vs1tr z&}SltcCgah!c~K`F?`d)`p=BTIbC9v3*Rjs%;=T*Nw6*pZK&L^+Ax+?4ZLQCv-0mu7y}Y9GcfnyK_JK)Q0b|21cITE)oH zAh@Z0KXc@m#&geoc!+X_bv76+>|+|kl={mI#3^7aw#%tpZ2q9b%bvA6TBo~x;+3>Z zV_wvR8X~G;pTGr(hX6Usjtht1PNY58n&0Z_fs0#b2RiQ zv$pg@CaRh+aEdOm%?G2~J%pN>Wr#O7Ws)j`vxqT+3|zRt>J`R_mnkxC2EE#cYey<& z?d&;3zn_ysd460oN@ZQ9Xj`89DgbUJ+;bChWD&c=s*JKG^nx|w6-_GZS9E{@hnsgs zgvOfIM9FTEZLzQ|ER!wel{}FROJb7OD$T2E>TZpl$|tH2K0Svhd`PL?H@)6`jJ18% z$L&Nv8hOH4o_Hv3d|Z;4sO0eFDxDg}vJ!M;o9Foch4Vg8n})&tfY^C1x`(R7L_G~e zR{ja2^$;whQn}N^9DKFxC#z%lt%BbfIGLI2KD+lBv|fQFDs&5FkhGo4h^m&GrMStW z>kKw}a}ss@@cxYH;WCF)7%BRm3LAyHW>|2`r2QhWI94CyWHRvd|fc)<`z<Hz)~fozItrVp#;t_+{6-#YP%{N9gQmZ}n z%}T_zIpIi+gV8JK{a-q3ZIbQu$@#~6c!fF8@N(m*V#Jm~0ldtX;pyoR8+J))d_8Lp z@fUAIH|l7*IT;2W@7ndAR25pCVx08j{EJ^h3zR{BzBj-gMTCk?3)&cFEhnY=cXVKA z&Q&CFgTg}O-9Ig0B^zWVg9ohUm{^2Xrt0cb)AgnwWa;^Zu!)jo z=6iZu{*=M$qVH?cafwe z6hQdjf&#yN>j8jBjQy;!&g4oiGEXZCFDrX7{QZ{=Yw{}wFq*SbOmp3lR$*t5&5Mnj zx<}6#j|@;P^|Ol6$>cuhuJXd4M{i$#Vt@@sr{Ki({V<^f_DS^`|D2b7Vouj5V$2y^ z_L%{6KSKIFo8XT6}O zd1D#xN6C2Ip;v0LWEgdr514ptp{))CuJ~MCU5UAgS1}J zL)Pe(((Kb3J~+9)W|y@tjz+8@EnPL`*gnZR%INd%T5dE5)+0BI}S zRc>J-uR&|LT0NvIM5|HWr!B6_Fa0D3dNDaBo&Wj6gL7-_Ioib8j+ z&u$&$Yk63-0jKNp#kmKU4mh%_J~4O9+e}y-_0^-vFjEZLikB?TwZ5|GM@NU`8p;y8 zWXowEPy$mnyid;ghA(%_C%NC)BMK35ID5A5K4g5%`JlSooTNRg%oeRr2mQai^h97^ zQy-%pnH;F_fCoJ8%G0^rH|9`yoNxGF+XGRcKLH7efE?-m9gw?JF;z#q^}tCT`?r$> zxMO#wtE|ZqpJImsw^-zXu){yE{`ZkxWW>Kbs(+cme}5mK2?YB7*MI&#S}P#@$BOem z-*2Iz{^#RAFV$4?e`pTBSy=erI`p4^dT1#B56ykH`G2@&9okf&FShoL)0UFxQlt=lby$JmH9N(u4&wp!T`~KOT z3jiznPY<@T`|Lpg!~}qpQXOuSK?*!{c)+Km+h>Po1Nhn{|9Syfi&e+}_s&(1)5Ahk zr~jkz|2gCT=TGzh>suBD2Bh;-84xGodrI{>u;ra{8x?)e4}Exx^-4p)jkr8jlAOE} zeLv4F;Jc82JMw&*LD{&XER`lTH4#s>)`@5PCfMPJb86yNgM)4h+oodjI7)Ig;SD>e zC6(P8uskE{tI7mvYyFt~Ws>epj|<*6wmKRr2EL#SdQ*orINMmoks#y=OV2eMW-ju< zcJN_de6Tp`J*Mn`(2eHz=z`YM404WVus&i}4 zeRzjWZ`HYn`6+O3eX`zRxeuqJ%j9Za&+PM+oQ=C%7k7CWq@Da>a^yn8mIkOyd(`NF zr}5X(IT#2K;b?$6bU-RYt0e-CJtlhhqEU`+U(7VJWaI086mU zj_X|qK7Z2MV)yH7N>rw-;p7r+SL->z!d`W;1N}oW*qKfPQz?KgPFwxZvFn-Z1ooZy zeP8$W!z&^>wB3Ch?w&Wo3klzKs!%iV=yXd<`!;DoE!9y=VRDu)rl(b_p=&r46o(i4D4g<6IVnpj zUNqrl1G_R&ko-hkwOUD0Y*7mN?Fsyd)fZ7`L~|=bkX_-99M!(q2{DDzF?ohK`HB$n z@9ll541?3(0(6whjjC*=$2vD2{#wi5X0(H_Gs7RW$^Epc5+sO+QOejOV%X;S6$QTv z?#JY(`?Q82%pq@`*Os}nE;iW*(A2X$%w$ovajgO#)G3d3b2iqpQi4$t|GND2I#`Bg zfg2p$5M^EFl0KVmkG#^FKRL||euxTLZcO6%F~kjJ+aqHMr2imV#w*scQP#Rwa763C zY^#hSKEGrI_}q|lj51+uhhU@9fV`5FZkn1UI49cC1M5+lxEftNrBbc@rRZNFy?ihg zhJnF5+J)A@g5MCCD`v2*a&FfPR2*oB2jb;4ATIkjlj=zpnN8e`;wcAv^Jq6urgb=< zKgl0Ug#xrnLkfSp^~q-tg25Jxuy%^8T<0xDph&QQ7Vwvrz7>j+G&D*OYJZ#QGmG%A zW(XOx-Bv*9&Q}qu>qsoH^52HrE)+tfvzA9MbweUNt$%Q?8au6N)+AcN>eF7)XHxFj zZ=l$-`0a1finVT(xO#Fhss>M9=HHbcKNuI=e?XZofof@)jjBw77B!1oXP=I>sJ0Ip z5lR3?L*BW4TO^~`$Ei(%vr~BEqCoiyWelwZvZqKT<0I`q#QY*;=5H~VbHAhDWm4Jo zeGf*cCENM7O&Wb5xH%BZ^Md}ejU0Q$Q%m~x^8UyjegBGz_j|Uxl<9f=eXGl7WUPAQ z^}~gXJMS!y7@qsVSfLbt_$uq}bQ7L$`aV*xB{HL6xzrmz_B! zkNe9GB585}Vdbvt3fp(FC)`!^AyfAC%i#Nbk2VkOpwFy!ucg%5iF5Y+^Zn4T0 z(bLzd^`6?v8;}Ns1DCeQpE>)?pk28KYKN_8YMPS<9H&0#T(K$|rc$t-7WP%euVfi} zIV89LA<_Faa%fp$D$;C<-}|cX+PxJ0 zT5SwF3d9j5Pp(ikUt14qOXo%;8(P~#4S0SwyW3OG%r>&1@IOE(ScpI=gHG=V#W=f- zSIabXD3=(n%unM?)(UB58}Og$>wm#48ltVusNB8xtZ`7c&@>mh!^@+oxIAzmM|BA! zqsb6$HNW}2_MMzlx`PM=gKqzB55OKoj*><0wSi3jL`00U0e*mtqf|ouP87ec7i! z{yJFgh$<#UdNtu_U6am7&ZI?GQ9=en3CB^o>VgQ(_U1M(r-+0L#zvB!1a~eSYdm4DXA-SKwo*cs!UO7bkF_`M~HjA-<0ba%xpGAdDYc8x~bbaluuAvPvq<|-G!;|wQ|hM zbk{!A+1Wt-m=w=j?&DJKtLw%)-n;4@YXgJ5ar*45c7L%-=^-bEt{S0o(`A+PB};iC z9_O!7&3KCkOiq_L5p?R@XGY{Bq&}j;8)i>&+nmF3qpWYW*-v-GmcrUC?;&sbQl0U8t|$<@4*^`m1CTxUBu+n;$npTxDX4o8iW8b?tuU7}5 zOX_i(Lv!Lnv^BR|ih)3=t9kiJO-|zxw;O^d7F^N_tEp}pS?1U*R!rW7i9?CXEXu{~ zxobXUd(`&T+}fYRjI|XErcCA5d%U71mN0{T^(>H~jYUe1=iNv66y&|(8F1z&sR(Sx zEF4_}Cca)fTf7mc6r8%$nb!%c$>_R4WtrscQ+G6EdawI0oF8qL|uGxGK>R!X3w zfqmg4r$RH%F?3pe)_Z%qnoPg5xaexTqaE(dea1fTzV8W#C3o;OU3KvHE@^JFmUQeujouN+n&jBf+C12rt~>&(5dWkpUzr#;GS*B;iz3{!x;Ck2n@|VnF9t$boHG)vX$oT(F;po|NFs=KQp#xt zx3CxiFkyPpcYqv;m zNVl(7pc2Rpo$=fZp0dunAa(36RjNV#gcEE_T9^LqOXKwXeAp_)_-AWrnhG20S_M#d zU?A7L#M0pqbI*;Z-4WW4ihzSTIn+-maDt2Ss5T|a6ldS%up!N1An2t()ine*CZnR2 zx099~5Kd$CGq+T|FA1L1(v^uH&B`ROk)$MfddP%j;3tfy%E)l24&fneV4E9dWcfk~E zXHWxn%Sc!Jfa$Ra{_u81G1yoKL8sF{H#;Ggh$qork|bGs_!5LoZFY+RJrDSspXGz zr-k_h5j~46%yaqfGt9}pVZEEGWg2E7p>o6#n_D3v{ET0pqy^hr;W`_Z+FaHR5=^hJ)R`ht{UE^HO>k?(_~6xn=VI#(%9PH>J%n zIE0yUT;zL1cWUmrV$plD#(g2<8w!PV>YUj_eaqp0N90|4diFZnBkoW|sy})D5G{Z# z(LTJ3Ftb`8itsiqdu^ASYOCZLvJ5~!jf|-dXGLF4%i{JoH}%uy@{WNoFlP^iTw+4o z>e9kW3tU5(A7kSeTr%ED#IJ6%GEeg`4?US7ShgFhW2UGyksl+zEI?3UYcfBq%A?Ye zYN8yYvF3gJk!@TxK6L5fq%`pzU?!mcbFe>JVJ)~$xw~Z-D44Gx$n@IG9k-iIo>#Ke zfW$I17&z!044qPzk4?InB|`j5u%l|m^-DmB(na(+nr z83Q!V7O0__SIN-u)kXa``&9k6CXi2|Us{|jg0+i=$bg-ajBgzV^3`BisvJs{oX(m< zyHGIyiP``I=7s%Nyr~C!*vum>_Vv2W{p{W+Th{xM`Z!#h@dyQb4Eg2O_Wh%8dvmDo zQypHxT)Ex@w)+-8diwhv-oyH3=tKOyQ$F}v;Q6*F7xc?^?i<1&n~PX+qx+k!ZzA6F zh3Mn)B|B?%_wVfvyq3(aw;SJ`mUR%qMi*Y!=omAOPnck*``Zp3o}mrZz}Y0$l=q)q zuc7-_Ms!A&`d_>oj`Raez7Ip(cphP$jUS%R9(B<8uJKxGP7fdSV70fc$AxV@lN1~x zZ}+XQf(cvZAq{ms1L$%x3M`h_;C)@mHWU&TZjD{8KS8$;*QLr_Ev3rTYL`0}+X39i z1d^%&>xh9uaFx+=bbrs2=s@l3tDeIzPbZDbSQ}rdorkT`^|KFo0CpS#`JNxGBRUJzrrIWe>U z9$ZtN6W21}QSL*+B4y#ck%Jx06k_zKunE zis${@!ltZG?hxbQe{umF!mmx6>`R@?8U+?N1Ts2ovUu90Xr&xUYeOQT_ge+7bG-Yx zYkHJ3L`)6{i&LfkEKrJ+B))o2kSAR{dnu)uv@jbpeErod(6hPJ%kK+B$|SkF_66!F zmF((F&EkIHQ(yggnKPU_znYK*}e|Ac=$JBjdYU>)G9&E3GO`Bzr-im9a*NoRyl(6N3!OVO+yO?2mx1-+J zZUZWoS>J)Q4O0`v4lkvU$=2y}veME4*WOkZI*NKsQ_tLyvOvXw!G@drp`T2nv>wQ& zSHsHJmu!$+C8D!ujb&707`41Sc2dz&LuG75)W6~Fd7{SCG;n$naT% z9;_|v;3`;r{!U+{wO5X+riFdrVGr7UK-Tm9E79wzn{=|}pCjWOFQ2SCK?J7RKafAO z-dC&j;noM@{mD69G0MN0un8}R%_6b3N!3ku-lewjJiGnA=AaKZ| zQ;0??G2#!u2=#lXddP{gSA4eI-{Zkw!h4Ny~?+QcB%YEn@Fl#3I^Re zCQFReJzrKak-fXqK6nY9JrCk%`5Z(9!2`KC4~?0S&fR!Gdg3fkZZs3fK&S)c+Ay}6 zL+@k76c~5}2w}gN&5xWH4KF$sOd!Hml_o4n6+C?l3e-6<2r=S}HRfWEh+v!MXWtT0 zZZ>4UWo#>D{!tdZ>sNsJAP5K10SV1EZ%Oj=2zI~%2L-N)Pi*O2xUZll6JOM?C=5aR z6XhqQHq3c`Y`8PZqjhaci?|d>wjWvON7TyLPU!u`q+!QA?a%!m zT9pAR9L2JglGaE$exlrJ1xW| z7yNWQ-hKV0grj45U2FwKMG*l?0bV9y8#%lob?%>$P?DA9DnMEv%dAn5D2&kVc)D=RA)1pSZbLk#DrLkx%g_T!!dz>5dK=&8`@ zz#`~JAN(vW+IabEhFG;ySHPY#n7yk&UL)r0Ku(llEOoNrJ3aB{=J?&|bKbTAt-4w? zaM{1E9af!0{P!naXZ?uTvM(sCxO)wYTqV@dm{=IZGO|WTOD^=UZvNd@;LY&cf4=#5_euX>PZ%-9SC{JzdJ>;kLQ8}eEbt2H z$&t~};KCR^cx+Y>-k)ca`ccF!P3Y%kl9uyJG3yToe#B!kh1ep!op6c$f0K(~|;gg*Jc@F_*4}X;;*eIx3DF?xDfqbMv*j>xvsnBP8t6EZBIi<>d*k zB7-eiuRha%Chlhh^iGaQWBU~Z-6^|tj&5|qsQa4Jq6qx^HQ&WUBazX+pUuCzcS(|B zw1zTWhrWFexUnoZKK1HnK3e##?s-Bj_aGIdfyng-}hbJ z`qv6mX>02BDo&%p2%0t^6& z`8vd454w)Q$KTQx_{tA*Q~?e3zeM8Wf6nW2M7V!b23QSojNDbfLO=*~tJ{N+hX7o8 z0Q{LW#$EJl5Wf^?C<_$=%yhs$>YqIboBAOnEATCQOk7~ z+%{c9b+!19$EO|p9%3goWN)ciB9P3bIJ~m?WVhLN`IlFz9OUiiMk zBa3RVuPlj+uUmCKi@wKe9>CY^@)Hr+;~d+(N_YASGgRB~37`=5v$p`b>gHx<{QnSb z2KaA-0%au`$*I}{6|(~HXUhX;9iqP{__z0zU!HiH6nUKd4klgT&o%~f09-@L+_8pZ zy6+YEJ#^4%0Fdvt85!AqHsQ|UUTp2Y7~R6{Ev$2HQA?L!I#p5PRz=*ArAeYC=Vba_ zT%>Jh;)1eu`^7Ndbu6E|pEX&c*4=HU`n()a)@5&>;eW@&eIa{){ZL0YjUr=Wa*6`| z@pA+jz59xk1KGxo9P1PS6XQ|Ux^HrT!hAg*$hTj7&okK5^F_<^6|xA0#i532Q`Oi7-;2?tXRCF1_K7FBGuW2*?u|N?}BqF|zT2Q8QG{gz6#>7tzWhGjuO$H|>IyGeLv&+bT zHTnJL#IH4&x>DADM~(|s@eetLq&%}Jd|RTC8o!ipb!*U-3g43m|0*J zf9k`Ze-7|7VinYPk1&Fj=M|)78drbJ2a`En9x}#Gl~-*|m0Fh|9f)q9U6JOs*~-%U zV^kI~Dn{Fu_VlTp=reLqA>xl@4v&~KvNi`q=Y19>>FR>ZZr-bfy0n&>sDpmOh^LsD zl}+iWueqw)+S)Idc<&6cyieo?Yf4H=7Z-g9OrCdE|5cg&hdK%N^esg}DcV!l_k8Yw zFrD|d&HCazUzt|sMyq%{M<$#7BT#H3%{Q)L-*-2_weC)JO>LyXr~96x9ST~&Y11V8 zb|hR*PY8B+0vJ=qD+;PD$vU#VjUaV)r!ORmq_qWR`j4U-&x|)qx$V7$c-e*z@ZR~j z9q5OqBz0>u?!!gRxxIrD3nN?HY*2?%#|^fYuAN-&1wR>Hxij*Rx$g||?JR)zy|e># zqh_FvG~-+JxTLJ)%sz!3p6^#8FDYo443r3tkkMWIZr1Jb0aSD#vf+9N!z=Ty_?eH z$}{2>KszyFCsXAyBgNAPL^aGBnp(JvjD6}U*KW;p`93n!zv-gks@n7c$mC4mW@lN* z7KO3N9AlD8Z!GxZ=g+eqvZpvGgh_7`QcwUKGA@SX-wq1YXm2>P4FUG}e~O-MGc{RF z`CtvfT1ZkY6I416?@kE$mTS=tO-SR z=d1Q#eW`;W{;O!hVr8iWh0G&*fS1gK9ezXb=>8g`qCKMJhW`5KV7*-LK#`Y3Yec`y zemDC?%b;>aqw{EA~LFn@Isd%IuPN-)^l#$*l< z0V%7B`}kl_oY>Qny>tvsd0Yf2uuK>7epeO=^qGB`?R84w#*K#Y4ywY>*u&6{iz;&b zF}c2H=Q*ho=!Z4bnzK5nUw6AR0j=nBxG9Tjj@n%`Ymf=XG;iuYMaJ=QE^ zTv1*E;e%dT4a)=S?6IpzPH2Bcmt?CdPE0n=wNykHh2Y!&v8Wss74>&z*DtE9t4jo7 zfP<49)puqoGf%=kca*pw_5MU>D56M&fL-qEamsbm+f@ zCj?`YdumFwK;>Wv1%U{{6RN;cN{Uf4hx; znVe{(->~BV<>pe3r$Rr5hlfAd*yJ}hrVbCwjfee{Uv}k*2cTV0 zl?@COF<=q9+4G?d&Y&F*p|`h54tk@ets)krL^<(ZiBG&4|94KjVI0|o;Nap?`V|%z zhhJT}{qc+ULlvC^?lU0DwLCLvuRW5{mlXR`ocspQx=#;ZU><_rAz_!w%fi1r!In0a zhTl6TqKv#fgndb){uO5%VSk^D8t*JkrXZ?>o{eg8R`f(B8yzo!m8GZ+@VT`jgZXb# zLAfQIEg2Al*-0K_^5>1JXvy&F53+GNJg^w}l_Arcul$lPW|lb(q%|1ryj@34iguai zO4=+Z2mR6bq4Iz|R>*-&H}?8W8%gb>p4UX@c5b%C&GWH$_Kv79b|S7p@AB#$|Jd`r z2*iD{<;J0px%q*&qoy+C`N{u*hA#IXl)%VIqu%`7*_yh32%F$jzrj$}RLF}Sf`j|U z!B|OzjCLH&@${3lr<(_N@ApO>7162F4CV(6AC5M+&N0c9-j*6OT359sMC~d-SAVQb zdwX8Qr&=Cv>x&Lf5p_%)AzRBz!4Ugga%6jK!se;G^*V=pSm3KP5_3eHq%Ud^?h|J_ zG6BT?CiY_CO0+`&u=kPs&GO|m0X7G8{kpWf2vAri;T-9!OUKpftv?z6=l~`xk{qK2 z*Vl%~hwqkmGVBFy=@C4@{`}(73M#d9c*;^*Mf0IiMzrIl{**l6j=l?!sA^S4^I8e0 zquQ4V1>9f~?rk{I9aI8O^rhOEifiIsw|hoU{fhVzN|WD>qn>C{;AU$wa)1fZ zDBrvznw?y_CAS@dTXNa0_<>5iF~&ky0R)Dt+k#KxViHQXpu0%Hkka>f9>IZBgLWlIv1YbZz zBefm&y{wR5->{2vjz!ZPGpxdu=vww`m?ZvbmXmD7CQJPNs z_*lX7aGD*Fq@fP;vbl?Q#p6jh+qVpR_*(oCT^m@D)UE%J$;^Uausa!+{lyc|HFSm5 z;eHLWiX4$0?+i-o;yCu-6K=>kq`RC!a&q^}U^=*g#rmOO@NG&PuFdgTzDq zXZt3rRin=9j8^ds+W(vMdBA^btbsTLy2bTbm?^b9x1%UAasxK2^{UQ$*N7hh{rgHs z^k*U27rnnL-i{x&7FP}@!By;WvK;PX$>oN?43}4DRjH=LrlDa_g!$+Q`W*gP`Ld$; z-QW2Aq#7m9NDOyQ5vWxV>P%Z-y>XH%=XEXH+X6v)TS+@S)IB z_e97`S@HpGBrJ*(>;Vjjk#V?*@TYFmAOJR|^g|RF5EE|Vr}(zCI&=6u4w~5GoEJQa zj5x=DW>+vVAzUh;QSV+%Oq285aQetW^si^2=K9MGu*JSiD_@sMZS}o!Pcd6+sO1~V8 zb?&f{^7n>Cnlle?#Os!zhenF_u5GfJ3Jl|FHJ`-(&YL+-TR?5kOogN z-}!`>wAr1AdZg(J;WK{+L37r(xeeRNXk?1&Pz-ga2OZh-Deklbx*3RJwm`Am&$ZK{5o)3 zSLvId*j53SB(mGw`N<_4>O(7)VX$leybrg4Q0r4~oTBHBz-P$)iGA&lzzYBP#Kfa1 z0UcLur8r{&@l^;F5+L?Ej2;6BeR+})dhE?aJ@^B zY%>J8Pa9-=yJ7L^`(mb!$#5<#4o;&B4^2K9ajIZX2tT~IJ>Ye?b>(}B3B%%jbEcHD ztHs7VVUBM(-zALea5<|HA09UMc+oC+sev5qh$rG%9euLo0-R+X#O{5WC2hFVF|hiX zDe5;7-xw*ppBoXpoD*@*<=WW&N;W^FChc}!2|F;7dj2(;OD6PX&c|^ow$mGv>**!z zvq`MXa)Y5yOodpquY^zjOsbz%scQ)YThip!*Vl8h6Jp~kZv0y87&A+*I?M69c-;&Q zM97r>FT$Aq9%vhYTc2!baQNTM+-t!aV}eG)-}`V2xoui#czEM~QS{tsjtts#DPe-<#fQs3ILx=Nqw`Nd`TLOIiKS1Hm5+r;Glc#4LuRpmSE z5wAWmE7{x9lKpcqR&Aa{*V*x{%%@p_o5^5Ze2QGZV>q9q{>5UJ4NXdIp(v)xjJ{L_ ztTRhfctyyR?{lI8It?N;dW1#@gWLbOn3xX*4Q7raC}|>oX!kcx5DSM;#IsToz+&L2 zywd_vO8D?YL#W?8VxRZ|i^|>DCvl#h+gF9GdWzbw&-VYUXZ+V`XOg%nN}K6b3gXcj z{|o}e{=bhZchxx4sAEieCwd)0_@S9>6u;d&BE@)-DW|NxJ=+F}Bk}a)q^PVc(yn~? z@6Q(wp+S7b$jG?2zyGDEC=3vD%Lok=U0hu`xVa^uIF!5cSifne5R6?lPfwu5<<-^H z{Jbob^L(H#z#pxQAQXtNup(gL;NX7D&Z;RZLxZJ%1a!T(IH44Oz+%382Q4NTnu%gK zXkfr*D*cPvq_CoIF<`?Xzh_JSC&iQhD`Hw<`3Nz_vN8raXn_6OMEHnO6IeuSFz&E+ zgfLZUk7Pz{=ZlfY;2#3U%alY$c9s$L35GEPBMeUeUJC#jp?MuyTwIV5b{ty=J7CS_`Mxo^!OG!`p_L2g7oA`BjX(DXL{HKSDtOXQx-k!eV*^1H6B zU#e2~3wwtX=PG|3VG2Etaf7e>-MI_7`49o^zb#^vnO|C1ZN5pe>y<=ot{jyk+jTTdCkoqJL1D*Rh2o@)HQ9l?hbgX&2K4j+4Q^mtTj1WJ3>~b>Gp_+ zA!VxCjR_}@GqVJiLUL-lB4h=o-Q2t@UR2v83fxyHr?Lti=cIo33u)Z83z7(AMw9SQ zkJt7^Rvv7vN*53n!pbMKPXA5Hg=@P4PmHr|mkFV3XotaX9)qTwP!8wM`pr8JpXl@v zR-#S(($oY#OE7KAXVh+Uqnb5r32^y_@H*(#j=qx+Q5byl^L*pp_$ftX(`n0MlbtMP ztATH{zVsq~L_(61LGMo@*7XBE6vvZQ5ozb%T}4Mv1~nq9eOcIl5=u;tll2v2(Wvp* zaNj$ekcS`u<@9v#v6?n6YxYpW&(vz)RR%9b7feW0LfuTQk)+Uo;UiLPdwpuW-l#x& z`krj6<18Gs8i>r2B?)Rjm9uN!5F18@!9UupNF`lMJ5q?=*>M^;S&>kmHMlFo*ic~E6`)a|7zDlswC=&|!r6Yx`i;BvJs(UQmPhwpu3vHzL<&Hj$xPGxh@!3=YL z6FK?xY;T#8FcgVFXDI%&$GNIqyC+OtsTxYdv59vf4*pIe5<@w7NMZGXA7Uks7#u#1 zt?Rkfg=a}R>Y$;l)pu^~m*on@Xo!)X>)dwC_CCUBC-|V%5tvcg9FXpURSR>tdX6(; z2b3|S{Y@wkDLNv>`o)z8dn2OKbaZ9jO4#dS3-w^0gj2!mTtfy$9)zx+E}z$KH+yh? zD4Iy>qm~4qW3aogGeSp{>|?gZ(s;^N1?2Fa^?fg;{MiL#3i~JMD(B z)g8Qx;c0w;=c};#9Dq~s6C2#~P^bEvik zwRSkNBu)t(F0^EgKMA9?bwpH1OKCav*)BQu5ig*(^>eY=>?r&5@cCXZ>T#FExhzw5 zIL_LRD!896M6uOWl|@EbC{Ux^NT{kiHy`nJ^i0*Vy4iTeq^>L}8TWg2_+CpDCVk9Q z*AouBYk_Sw^p4<*b*no}w|83ZbEaq1?bTmx3F?knZo62jvk?&PjkN9-_Wb1QR7U0vo_?!Ku! zc%aRSU6^rMg|+}5F4asgTQWMG)}a9*V!?N!J=-JN2KSU1F(l>=N3RsgmgF~;DGWQ` zqot`;)to?gn*#y0Cm387dR?VFkPr6hebK{4*^e0r9+z6^`Bz+e{DG*KKCc)GnGJ>? zTA0}$`;@V?n?Xez@FM*7y;)zENYW?}5{EW9c8ckb0&w zvB$x~AserUXAA~1)j2HShF-}Z@xv2I@ht;2y#vMSo?D-n>i z1#JbZ>UePysy3>{zV+#4J+%osWf@m(;C{S@8*l7+Wq231^qxGkxtKO;QDA%^+Je#* zCd}<5N(I7rT$>kVNtea^qz5on@0Nuy#eLGbFy2d5J)zenbgl87jAVVdWplwId7VWDP$hjd45p<2(Dvrhmqbo!B4!Cm>D zmTqMDMxeJ|L#Pk;i@4A%4v|kBxUr7|8djKu-+F8SXk$DyM+G+M5qcj$7;*dj!{Q=O zm>K96p3sh%VD_>Y=T~AuK76{QO{vliq~=N2;quyUw~YZ{Fh`F{HoB^oj;w+iX1f9l zijyLUcpFH!({5}`PD=~FfxiLfaD;k`SiAh5du-4423aj}>ej5LT^G$u=j~?9K07%# zyLj##apJ$5i4G^R{yuu!qkF*Ql9I&RqouD)ZUe5PWhLhQaGDMWW3%k-jD4+?@JeJR z_TC6UVXehLq5AD%Etg3e5VO3KVr_lQSEd~`VswX{_t?R{1hk`R@}7gOSU;I9>CHa3 z#y*h>f1x(RwxZ8a9@O6d<@Bpa^SkdMKn2H9@rOrxS1uB$GDr3$rV1<#4-|^?%*tH7 z(6cJuhN$8ZZZun4n*R;7^ycXGDoUf$?6w^1E_6JOA$zzjo%=*ay(a%lD719mc*EK8 z?A_T{iiAY|Ikz~De4OjYs@&{bvb=q-6eY$r{g*V;yUJ-~pYu91P~uoCzL;Bayo>{- zvYW{Y&0q264qNKgI$^>s3wUl6`HvVr{d_i6lTX;6E~97or|M5gh)#Us8t}A)xS9Vj zkh689P{TFsKI)quoCO)lsYg2Ju9LdiEq9+a_!$dJ4ESYxIf}UJd%tlnpsU{fCVQ7> zT*#s|eD1k@=y&ROMPmCU@Aw6ioz?_NBTjXdCscjOYjH8WpD<|!aJ}oN!&Vk{MIy4p zFZA+f`ijhIZ0U(GA>Oy^)#9PSBS@F8l3AR^onwL(uz zmcnJfuS(e1+1HUZpyjcjyhNB>eQhej-n? zZjGBGjOYY`pTT{a;+<4f)7H2^3BM3%`LF=Td=cR36Ia+vidL!9-(tTaYPDztk4#(Z zGsY=J*yyw&fpCUHUk)B!IIyGndJnrF3Rj+;VzIl4uuquQg>KiGQrVt3?{|tKy3LIH z&cago`^b0p-${FxoCIA=c8lFFo>b7d4N#bu+fk}2AWtrAD@;GnrG_rT$sSc^)6IE8 zLhtkm540H#i`wJ&4cV-KLM`ieCwzpYaaj|p8}Vh3dIngyXTJvwPtJNJBtA+q8+u|Q z?~dhup&THJeF`aR$T6laHGYEQiV#3MoB6_(sm|w5gX+k1e??O0&HXt)H?X;G7*Ovn z(pRU+#j;nxra2l!6IG1F?|BQ=|4zl9*!2e<;9N8Lt7CVrcolO5CG~hR%;a%7EN0%l z4pTh+(W6wo>q6FZ3;Xe>ns$>1DfY&+;Nwz;i^@SJMCowft%pv3#Aj`vBCG~ z14<2QJom3Om@@{TNSbsyVGN_L)z@RbE*!M8tNlaSaUVxgxvYOt%Q;zi90%Te);R_& zIq*rN@v79lry0RxF}(L7oGaKp@G+25ANMWTexc~vx%x1W{HGM<^BJLv-5hUObyd@81VNondID`5)tbiizQ5 zXMd}pfVxRD)D*SP|3r(PJ*Q~IuOP}vJ#0ZhLO_A{yFG&zN5*Mx%X>Yg!Vm}~e^+fG zH->6XGYBF_406bffPVh-HIVl2Z=k33iA1blBpl0k z9l2mc!4mA9P~m^YgcZg3{T1TQ7v&%M&YZO^D2fd%^Gf<=-tlC4sojkzxJ}r?m=QQn z;K7nOi$j{<(VVEYv{T_ww9U!bms@ni9&BWwX?6UeHi1Rzu4R9ZfZP~@H4P%K0_22v z1f(=5zWVa-d+NTA>=XI;5}Y}>H73i9KcT(LQpn!-DlD+)-)dt0%_i~YaQ66seN(icKWb;o zHtQAN7S)+!;#A5wRG-)T4>CbXwo=6W!~kud>^eEGCLvlH_1+(E`RRqqiWWEfTkGi; z(s1o4p~m@W4a$c8`0S@Qk-+PPSWX5uxUinyYU#PlEur!`k8n)s*IW}VnOvn#)gE0k zcx7a58K|zlnS;H-td^Dr`V?@bI|y4o{zxAoX9b|#Tv-?2vt+i#5K;~~U% zXr$d)BPtrkXw$f?M(atGZ7C5w;ZwB`f)t^eKvJfxK1~C0X!leNBZ|CQEOyFA8~+Uo z()pzqie%V3tj3dVj`>Z@#I!fM?hdnAu}&U<$nBw>BxlyoEu3W1%x@RRh1fe1W?Zl9 zZ1jan@)!Y|4|N1N$pJ6xXUdKGB2nwnKH^`;PdM9^1o{t(OI3yml^y2`Tap${5R2#b zkmK&H-4>Bq9L2x~sSy2>(O}nUXrFQXu56mZa+cZ>QYvXl?^koqcUkU{PSKsrIG=-} z(&B-Gg*+pkF57B0_FUk)8=t~dkD)AI)?EdK0=H!~!3H0mR`_Xv$$oRRJQuJ(IcX@m zjxD!^eY{So(o`Bsv?|O0x$;6kS2hb@NZb8aIT=a{|z)O#f!BL%!15hBa#uKy{$5 zjMSuWVGK-Dh6r#Ch7e2GoaXF!Oh1~Al=ohAAU1nuMA8H>bxDlw zKtuRjOZ#%~Em8L)Hz}Gjw;AUfBCYsI}6dWGs5j-Rhy=s&%%mo1LHTh?`;v*7gmlh*je`JKj`(3euecbE4BQ-6;Ub*uqShvH_0>2skN-sh?IwK+QO=Z#bDY>J8 zQgnkjOFjG!yO|T#qPvBvPNv*FB$f-rJK#bxC)&vuzcghDA`nWSI)^6hX}OjT*CZW7 z@8Ki*jfpWMB_jHn1&1)b4$!}^9?)=g23c+wY~iJ-s(_`+IU?GUFc@Sp2^$-*))6zh z`Uw|aIR}`&=9)UAE#!La!Q)87OGD!$t;-;`=MXcQd2(Q2%I%Fnk`c zdrdcPx|3gngEPLhIUBoq*6J@!f`N`&ZM*3Vh}>};hnp^Gx8;FRc`DSJeR;+0QqyGh z5*q;lKkNI1iK}?rfFVKBXlqc}c(zP=RaD?QCOizx#hvphzlU#OqKZ1%##F&iYS=q` zM3l`FqR&up%06z#bofG}_u$NL-`G8dWUfi=I`?^hnY79MG&L|UHevI>xkblU7mm>8 zVq{>*?0%Dz)qN*mzrC)1^e>Vivef{PUhf)?evKM&OoQ*!#H~3vYn>D1j6j~qkxHMY zKB!fSJKgo3%V`c^Qmp!B$xqKftT11j;|xg*^Q>!=d;=mAiM;#Txs9&$6EmFPhLP=y zl^br{x4nouhKs8jGlUY=7?yIelwEiJm8w`3*m#}SZFJogT zleq>7SH|t_{gZ-)L`Rd5^qxvMRRg|s7~R3ML>A#~SnbiFG)46%Z1Q|C0jC@rL^Bk; zODc3IcEcc&VaXo<@oKKso2@EI*bfaG`+lz79UJ|p4P}YP>r?|*;2V>rYUF?GbU;+% zx76bvTOgpDbBX%C)uBs4TwZHA zN;ujJny^^+qarAo2;5C#mfE$=|3U{^*ejuqzL@?wF9nG0d_7g~i>g}~ZE;*u zq%!Tr=FLy}z|w=@_T6&S;t)3D61@=fVEY&^j`6G1LRtX^Z-PHIiW10uD-@l@i=1Qhf+M}b zzQ}QfdWW?gWnjklqE9tV&>go#e?qO(nyxA-NvDfo7l*@d6mn=sNL}IaS`z8erVN?# zUn~Tbs-gD~&IVf;-YU~YUVB+?=wb^#-zR$M5W2mKti>X!<$ESvu}+~HSP zC?MmAQscax~ZLksdeJ_$i#w}ir zsVXi!TeS(djc@3}_YXT0EPd85&R)14%uH=$NL+5cLW!xiT)v}6w-<~)wL-E?!j0|! zRki-sI*uw67HTA` zF$c7v;6HBU^ce5;N9Z9_Q&WdptxVWY6GGi&4bAKTN<|0iw@c+k1#UuUFx4g@Vic5a z$fyEKU}8)<9AZ%qTXTMO$ix@MFF{4Kv+Slf^l8FGWmRdRp?<$R+|a*6{0X6A6Z4UP z6b-_8#!fZK&W1s`%-pw#Y~8njacrcQn+?8u^*c#) zZHsMqu`N|w9tfs%stqJ_`KK&(FMbpIkN+LQ=Qc>T(TLE3xuRLgGVxxl6tCy4hpqk> z`c9EKadJXw59wwScrvDuKaTuKh!uF;zbCW}=(yN#grDC^qE#4pnr*rW9TDrRZ3rKY z-D1F}3A1%ad_?=QCRCv>^p1;Ce~cK?82{h5Y*0GVHA}axns6xPp+p$dd)vR$Qy(jV zu~IKF(&f6HHae~-;i2n_koRyO!`l+{*CcrmY5%PbPG@fP&^VWIyS)<@qtCF2*2m4ru%!PS&OPA2=-rKw9QYq!G~Mh^(zxQq z^vB!X)^;Zg_sIHvU!_jaB;bTp#mnjn>Nhg21G>F+U8<9d$XYXW?c0+DPW+?qRuI2W z9Eb5v5^VguLqGr4!-H}@6Qa|2>Y%`P68-W@s4eiyNCaW#XkB^WFSoD_9sFN2;%2#C zWB(lY_=ubGb1ZT7ku`P}-h=LvM6WBB%s2CkL$rG=``AJDdK#S-Vn60ek1^Gu_XbCb zNKFnqe8q&wH?rpw%$8U0)8XN(zSvT&-gnixKxXQNuXC0zY+z1MV0ySYLKmA)X0`>L ztI_{rM+9cYSN1xjKnA;&(zUk&M~|Za(Ff}=xmDRDsCmE>1o^bI-L*EjWqeP&jY(3C z=R5eP)n$E z$Am0fwc96{=f0p(Hw|CN{lga9SDdk1{MEZ-X#(u|-Z6Ckn>kl5*)=$~aNpRKVTIMs zM6aSQJbzCzm=TS-OZSO7n-$}w3inzgPc9(Y6E_)4$@+j{`$tg%3;T(zJ4-fv;`wtwek1Oq?+IMgrmjw~bhIiI(>gFoK zCxScPDQ6`oA|RhBO23D(D&;JT)4?N=HEH+Co?1|c&!P@1{Uv0S#c;cSJ8VcXr`u9} zyEh0%&|yULW+>+xOW-K6{H{^n(sv7UrrbZ9ZjlyJ=UrvQ-QB8g8X|PxQZrnt?KSx> z0T{dfS+=w&EZJX6UeYp&%vMkSm#?+<&e%qJ)oQPWs;6jX*PX>}e=Molokhps)}Y<5 zS7yuxT9Me*n^}U;Z7h&F^$gi`_TVc2bwazgU^m0R*8((~2poy?on7R!K?bugw^$d& z#ycOuk(Z6Iziip&Ap4{rPGgToey)!=*-!gc3FB#9-*ZF;etqF0sB>Vp zbU*6F8-2fg{=D&4lmidv4J3oiKhP}0R`IdB!pB_}>)G>U6ZSy!yXqsd9k3PWFQ(gO zRt$2q@XutTYuY;`rR=wM+!z_99NX=Rmfo`x7_FxI#nVNvwD4>t~fSVyKZbeA3~KaLuwf61KPIU>^X$ zgeP2-jg;8;?I1N-n7vtbe%*AC@@(>edVM}JiEz`Om?2wFYLuMTv-@PmK{Q*j8MCdv z^jjT7%~Kbh%M=RDDLxX5zy8D3dN zRz6eghGef3iL|8xJ3B%~Z*#U&&5*U*4;ct^7DSceWtY$2_;~g?nM3i+mg;?ZNBrD{ zo*~}|b-bcJh_s~)lL_|xTuC0iT_HE@Jkz}>zZdfNap6g<63WoF@DIwcF#NW&-MO)| zYYix{5E8Kj_Y3*AMYwi8yvYd2M(y1878p2xwiC>N@xS90(Qk~4y*T;YPuf#u-x;YF z;?AYSFJGBC{$%(NqGTvM$Ut_>8YRLVSf9z>j+zA<)KkxTP;{5-?(lRa0y%y3{{mq| zbg|#r@uqs`zcZo-;K4)4D_(1I7zWxgtd%cqwxkR5N&DUg@?>{*<{ z<##Fau0eQ^(iSlh_?4k@Y2(m;pyt7H_#eHv6ItSeKZOvDk9yvZ4q<3dG+p|`4@kSOy@aghtn_i zXGk7Tl0tJ_M3lFd*Ogv&%ZV57wofRLeRS={fr#mLll@f9&I{n|L#A#@(qCrYJD^V% zb6|CP_-Pq6`cx+YUo9DmsMZw?2h^+kBxO&7ZJWWDGQ0$-e9w5_1kt=!9XVhy+2@hp z_t_ym)@zQMH3JOhxC!S7`ZH@>oBN%U`F{pNu|IpuDaYG35=Vz)AzN(EBBX*Cf7*5^ zw>~*?v2{jlr{0$|KU4VmKP$~=3&uEm2Qw225w;HR#7DX*rK)(2$`M~ z^)tY&OqhIVGYv--(H?qhDUT5sYo> zEklqT@a81X+nGNYn-|-S?(~enbM6cIL{&>zFpCg3*@tc!f&oms*)*IT&wUp8wWT0q zy%oiTDZsE9a8Oa!w5cdjrY-R?!#%))^P?mj{uf@}3cr9)M`*7Q1dKo-F*`6gpVCE3 zq0q5yue=c67gwcybI26R=u!4Vg6p7{HoeDk0Oz$U`yt;cO$&zykNY^RFvmW5tjMKz0}*obUBZ4ZK=flQ zN|kIepWO2bhPN|sqUjsw@j33p@8q#7o?MOs-GNu6j6llZ(fX47g8T%L9tAr~B-t@i zz2G&bVJZO&AQ2j?@$rSJM#ZYQqZH$I8p97jmg#Lmppk)PzBQAS|C@|WyYtF4$xuuf zPFbbD^0(dIAw#n7rA?@}lQ_#-(trK3DlIE3sjMu7Zr}OypCYx)thdb2w!e+^S%Nug z{kvGPXE9L{lxsviTa#!p&T;Yl+1)O5XCB1=aiajTgc@fx2ljb0IlWEDRU5yY=`f=J zGP2jzrC(l3y*k`S`JR&?p)*V$bbp8BQhz!p6MzjQwxg^e{CZq@3Hd+*jNFO5wsOd^?mXw zeSKfONn1t7a?+z$KiD;+FtU%->rtS5O@UUB>b%_f6gQ34I`_3Q4Wr6*ZCr+$y&@?} z81jZSM|ZHRkiP%B-|^DX`ckZ4ekdEN)k^-l*#S6gfHPN!a~ zSQd$zKBR_iXPePgB5y}8{`6V9y`h;T<@+uD2d})SMn>+raa}X7X|Ug&Dy|Y+p9ukY z-H%t$k*%zehl>~<1py9w$IK_hH2+6UJw6QaQQ4Mg;fc z_wQ1#*BBq_dXrsOt&Ay;%CG;B)bpA_QO$%eG{f9qm^i;DGu53YcXTIIQ!*S*tCM#W zu<_Epo-~S~g?!Ij>aapL)u>*(=o@7o`34)R<96O#N;kg@5R=`e(R0!u^yOglL_$L= zH|(NPvNOT>i@(;Fi}&6M>)Zl`z4~pz@TPu;w8tWR2vPQsicli3`vS=*08;lb;)yLw zvS)Jc-qG(Vk~xqD^rUXiG>sXi-EiIn4X;z}W=|;jxiA#XeAD!AjZtGN#D2PSZ*m!0 z#+hhOvG*+^)|`O%gM1E-)#pou{dPgE27V=SxAW1)+WquWcUF1ZD;??nIZ+)UB_WL` zGK3m0r7R%Uf1lCsrWulE0a_#KyeDW*<6xCXI?ZF#lO68Eb6)u&Y`lX%Jc6XmYCvxnENzHzU+3{1Wonk6z!gExb_6n)r9h^<8vr#d;`De& zW^hY9D)Oc)w$gO{T474QtZBWUOeK4!2V&+)?UhJBJVHCU|1CLFPAdE3c7&?%%-D~# zX#xXJ5h7X_cHWkZmbU~^cqtpOOU;|~{#UEb$E&k(%7Pj8K9*=E>8S~4b?H|iJnsVS0eWI`7&iL!-36rB?5|frV zH&`8LpcO75xJI_vwyoE+guFuxlF{iT-_rF(BlpRvI*DUvUp5WVk5F;+4Ln4mJ>Bn* z37*#Yn@uRpLe@Rui~MhKJJj~f#Z`Fskre-QkUV19Yinz5o96=DA6ZDY~s4 z6V|$;p2y@ZEzIZcKU7Fq24v|*JUFo zxk65NIYuJV8=o%7oqJz0a#pI8oZ^4z|OZkK<=A@aWM(H|*3K|9Bq9-5P!g`5#Q zmRl*BiasG;Ki;_?lD^VB2UOm7-(ED37JAsw+@IXIH+bMV^hujf)oyFM@mwZma%JEC zoW`GbBI@*bAI??PkuY7FP^{k=dc-20ACuN&nM|s&e9JV0V1C|wfQ!BGHB`ZM8DADS zDqw=mcqp)kJ{>ws86S)f23?`Z8J=47I!3|nVj5Y zCQVN38D>TJ&Q&F5+#V<_>mR{gH_}W!`dOB3g=rIuU3&b)91eQ7_mnbqKiw?flrYwM zOHG=dN_wCrd0T@ojA%zo2|<|A>DOUm+$~zFwhF=rqbFNyWc@D0*%S*WOIiLu69{ck zD7fyE>%{o@t|4Pb&N;`2cH6^M?>O=*nvapaD79SJ9Get6=!dy8YD4t^dj&6~I@;v# zt=FWap2m1DiHKX#*BNdpHwTjIJXbN?m}Qo)cm%J0Kv4QXItU7t&A)IMr*iUY?%6pE zhIUS4bRNv==vgOBuV?Jq-l)lvSenm%v6G73hT{#cK=617)UmQgTSu4dniRFNL`s}G zci`;BoLnI!)qA4WH+o-{n5Kr8m?H4njcj8G;iSO%SrH00y3Lk7)W?A2fqf33oEReio&Z^2S4&G2p}HJQXlIT@l2EqJnlN z1D=5|Lx(s*jouSF=5_&i{ey+NXDY@(8R~LR)OgrVs8$*$w1)zp@=);dXtTNt@J0uO zen?jNgV#G)Ti`i|*ik@F!|)lJoHC4=LqBITrQM2yV*)XQdLDnc-=&!n;wvK1!WUW^ zz2Jo+`=U*+(J#{e@Q#x{zGIpA8R`M#+gAIprvg|?NlDE$Ia291*cff}Mu7BO&}u?d ziBtZ{esv>6^vSZDuS@Fb8BI>|yiJH)qA^|~tu>4hEdYN8)*n`P#K z{jc66p_Q*$_|M7AC4&+MD3`bfl~2fFp6DgFH)ZFmBf0MW5!!H_=BN@ zBx#V98_eStKuRZH`QCfFU!=@k8gg{W6Wt3^g^Yj&YXV}%bi@BHjCHTtZuTpn2u}ud z4MBm7{y3-7sZ0sWN2nd6SaC7)|ZR!}HX_Pv94 z;5xS?mNj15Vw5Wve{h4~g8q}f61{1aq{M`>5jy}#8IPD^cBTl`)5`R{t4zdbCX2k% zy_=3u0ASL;605N5!TlT5`1>cpL7a+0jY+fS@6$Q>(a3hf9~h8_dOkaQhild#%-^gn zdyE#0G829xKv#uqoiv}6Xyi61Omkg3SUVueo5z|c*2uaPCnMN-hg5um=^e_gtBJX4 z@FH%;2FIIDwp+hZs?<@+5E&ks4+(AgaDWe8;eidg2i9BCeHxm#(OvDsoQ$X7UtGY) zKQ!50KseR5U@vgO{aEv&;(q4*bls`YOuSa0`@KD#ZOzLV{YWo_P|v zL4qJ!OoX&JEERkT3FR*kZP$>lg){+@ltass7TRC=<&}Zx@^WUP^{mY=9;!49R27F+ z)-j1jauP8d|Dw0te`xLm&2{Ja;m%9oZO^Aqc4#+9E2!hFxw7I}K@sYu^cm#*0hWbP+ zH!qG#>TZ?OGVtb6T!=T`-?K9yA>GJD4%HI3sq4E6%+2zH9DfQ%s(Q}iNA})q-AOG- zQ|j^YCNjx!v7?6IE&JUcL0j#me?-RPbU)TJp8x3M?2Uldvu2$PWFA!5$zhWk=+Zwd zwjRg=%Lu{~eB1;pr+@4t+Os$9>3U&yjCQ3nT}+zV&CzwG)8Nj}AWKLA7S5h5Y2!J= z7q9U4sB{A}s1?R#?Q9=?Uqgr3>+0^kmTGDcOq7?O`qNmfzY+U_;n$fH9gU)hi@cs? zmZ1}tszw%Z^^YLiKo!@QcssYl zC27N6`Lhc-ES+oRRWiE0!Ow$}Qu5TFYVxM{fAJNU3t&y-1-6cfK^SK&%N?Df-uDFIcC z7VRq2r6rrlzJ%WHBTO*3@GlcY_5!zCfT{O*+dba(Npck6$}!E3bj^GILwsZ&y0H82 z$lgQz4BEZ$dTOo-Cd`b(YssfpRZ30>zql=lyHRxU ziIq6*O_OfJbh2s$+k#x8Nb32NSZ3RxgVxrY)M;$1KIpzYXp!n#2+BgZht}qOBqA1Y z|EYGJx^oEOD1armlZ^G=asvjtd@s3O(KvC!*@dx}WcR7*wCjKJsrom?J{@BY4|jD% z8FvXUEb{N8yO|J>V!dkX&C&?nJe6v&FYhDd`XB7QWmKI@(>9n8oFKt9xD(tx1l_p1 zySrO};IMIbcMtCF?!n!i;4(WWC+9iO%&hgTS!-s^_v8Ho_r9gNtGc?oy1M$R6g_F8 z1CIM1H_r38f(^3+UHaXQBS+g*|LMU;bs7n4V>rC$&%nx-!EN9W(SlRrDSu*4DJm>1 z%09=;7&P6b5ip;f+Gk$|D)0xC*Z>6q-->u{yMK4L80zk0rHibALFCOGdROp$*r23d zZhJ~yI<#vq$3i#HH3HY*n3R6*6+zkahe{9B1S5UK+ljd;@kES7DY=@<0Ws;G?be9;kvw%&79VArlMDRh;)HL{$H-0-*1YIlQ4}Ep>C6w_kh9H$cu?f zlk@PvX%o=}M>rtc=EbC^ep@oUO73c%VR5+hLH?+>rk!qZa)Yg=ma_Q##OcKPQSa(9 z(8bA`>*VC*yYtU2e9i%R#0mptPp%Vqb(Qtlp15dpp`(Ne6-(VJd1{jXd zeArIJ^$bkV z2smoL*;Tv?#b}hT!D3tOwV?35TcYW- zj{T}FhSnDM#V5ZOREfl)z&_(D&z+0|EKCvvR7ur=BK`M3==WCLC&NZlsq$E73tc{I z_?gayfqi{%NNdO$HI&7dbkIy&1ehhH?Wuzew?xf31_)nqqg_51`t8l15j%p3)E;Y% z+e{LTX{rdew5OdGh7Rb8Vk?fN-d~B{UVb!)W;J6DR#ACU6rfd9w0hdWL?{T{CLwR6 zWEG>dT1KJ5g6*?$9kSulT)=c0<<7^}(vZdXdm*SZjaru^Skm1Hc@$;g?h&~Z3-KkV z#EZb{bl|F@Hr63mNi^IsXiV!VMWU5jm&ht@!L6SU|7$UJX8)hP?s|^2VD>az{XaWgo?gg?jwn*Ls`%ug`9MSEaU-kfpzcY)U_rx16 zB_7dH{Vxrd`8C&J0yn&f>L2g%4_6`;oM4S5!;HA}N8Obw(E6(V{yQRns z(5D>Ev654`&M)jXWWGM^O3C$&k?OBw)twx!qmti|jS@7TaIhZZ`*&|-m2^imGh$)T ziH_!d1p%IYxm|T!UTng%He;}O4ugjV<~J>R7Q&F2+9yQC`v?! z0{?x2Pe;G5K6>9_(lHb06Z|!)|C)pLqsk42M$EXr6DW8;Ee%|hc&;hqOBQOh&j`#F z$I*yuuKVA6Vym40iIv*GkJ46@oE&psY@>!ZeveSrnBX(|wLXZW1~1Hk?dn+nwsIPe zjGxuCyVXFVc(*jE*WpC9`$yA9FyXMad`WH(X}tLQpkPyYLAbrjYv08?u%(8Ro0IQ| z+ctu83mhZZ>fKHRC3b|aB}z7HJWIij2yzYE645>{e(J8*okq*$f&lVSJu@NJzjdW6 z#o*6!2NWP}n68jd$C1%69XfAXy?<#Qlt>#kC=V^9k89W@CRNw4{lFmT8ntu1$*jMA zxoM}e9iB1nnrFKVJn-<6OU#jA3oZ{1W925{=nyU6o-zk8XiDkTiAB%%rbc)qQ`V*A zf+H5Bc{9Aoyss3M67LJkwN=!O#H6FgbwNh9{S)s1@sAoqcc~k(pOcW;hmljFA&QVh z_6#>NazA|nbtMCG5knVOEX00xF{z)nM9#j5Y;<-cd7Dz3?9N0*r-HvNr*G{k|AbWh zi35TDzm1Ro|L3)**W*sO*cE5p_1m*E=d+PzuCs#Gcp5_0XqIRTViFwvuufKRxelLi zu@F)w$)kk1&BoKmtV`*mbt@<7E9omcvDK2|)i5;QPzPc#%|4QGKgSymyQ!T;F}sBs z2{YNBjif!^y_zi!CUWW(9QS z#3bau4xS)oQ~r%swb@%VgR_IGBZXA@LkQWt&{ELK3(azr>el0e$ zG#7!=z{&W~poBbOzz7>-PpI5DvbU>6bVZmHv6kIliQ}*1Di_GZf8#{BY#5orrzfU< z2~v)46z*#)g55OE|Ag`D7=MpA#h)68n1}kX(VcvL(l{_5CHBf)LU)!ON8(c_?AW|f z`b}`~Z-RG|RvHt7;c%QE_?*EuT}#g>E6a^PY=$8HCQ0@WNjT^W0NSsc-&}TrpO|+r z&VO*7>2zoREgxvu40NbJu>gO|bNEYoMhL(7BeK>l(D@2ZU0FtrP4u+l)M!2!tGWjc zSpZmfwktzbBc5%9 zg)gUP5K*u46DVYqFOMYkDvUJGYHfv{W!$-k{I3Q|kJ~HAoM`o(e^qw2eXN^x*v`~H zk!*Qf;jr^|Wa9#+?}-fC&z^r;zK@@;F{;E9?TN))d3~eZev89x>k&e8!HsTv#I@Fs zlYqN^`|_-P8Kc_e{`$B^^t!Ninbh7C5x>E9IVSmO`LU>d2U6@SyxKk#lI z|CMoXn8hn`tYq#Y{kA=!&ieK3>j4h&e`$=1Qu;8>cbe^=XD%4l%he%3+_e~eouB3n zM+sn88TZ`oQpzq`?~d+!oma)i@1)wU%lt-Wuu&i4Yg~uXPun{t{z}@`5|Q zTu&^ks~hnJer=Uk9P-<;79eZD-D<7l@Ts1o<`Qz%L!M9j?x=#^$qRjdgT?w$s671f zZhI|#*Ei96?`Qkd(Q;)=0zFrs^XZHIM_yjIlDZ2YHh6oh+s^^4+T%Lz>m#eym$M@I z3l2*s@99o-POq&_pMS8i+tdvVnFkKmN<=}j+4kofUZVxx&3Y$p$!p)i5*HLS6qYsk z-+HX8jp6Z0zE(TZ9FNFiuZ$|6J+q4R?TTi_2_4nh^nX%UnTRufd4f4@zdbs}{Zw;c zd1n0OUib864uR}Y)57IM-F3RJhv#+2+SSeoI5w-^cyyzC<8dgxpe5BpP?Z*!VzpV9 z5^!KXF3aJa`q=j1=mZS2m)&3Kc=S3-P*M3>gu>PM7`nFW&=~NS{{Q`_C0dBKW-ySi z|Li3P@oHo3wcU07oFup)R_d~m?_ur!-1=3Y_u-d>iNkRB6iCNQ0;~Vnwg3tvWA8 zqd7d&v)e{`l$K<0e<5kEJJWMB0*d!8bcbwwYT00!kKt<0NQ3I%U$TK}f7m_RH?=mL zjjz92Yq^zrpcZl6o=i&(rgoXPe&VdxSP^&S98W%)n_ki+Sz81XHQX7JwH>y^Q9HmzQAP_D}ZN4Cn`gQ(}Gk}aAVuh#Y82t;GEw#`uyb@qc}H~fyXNq8$)3b+tDb$ASp>Q zQhY=@z7P@22ctkq*P=*{^zL-Oe-DYE2{?V2Q1HhOF3LK=^mxYN zPcoDViRoh>0Mj#a!1B|Nb`0m=6$-dD^W%}n(M-+HG9%4p(lK{BdMLgPoc81IaN$^Q zc%HeSST~#=Ei3O(`V0s~VS*(9!a0LOk!#f;DL?m3%K@Hbg7MWip&---<&Dmq+XfO& zm%1%BRVhp4imz?Va(i$BlqXqSSi38YV#m%$FR+j$2?ayaG`}6< z+FvO$rbo#^f)fR+#YbK6P!bO_3WRMp5ah__n6^pri09J$*QtB0!AlLuGQ5@L`%dp6 z%aR{3+<6TBQT-p+eFEG22J~I4-Qt!NE4G|hYm!z4fCY*och|2EP!@E`Am15l`%k|X zeb4ardvD&}n=0h+SlKd#L`ua%L-+xLys*-RLB)ISejklJq%Wz(P9FIM_a2TFmm!`^@PIa96}JMSTuu0E)nx7kz2 zK4!N8mX`!Hx-MaltvpgX;bU#}9862}9jPvAmYBYV2L>J{_(5;B85aU*P@} z_KPVH*YNIUwaW3k?_w<1h9+<06>{CB;c~{^Dpzg?a+O5!6dcmDI>Du&lYe|UjxnHl zqo(4umC~3q(<#^*8eY+w?y}3^gq!^N-+ct6Mo-_j)eer$c2rG6PBOf|buO_nx!uvj zRcH+$GC5xB@;)Yoajm*REv+{Aa0m0J4H8FOC}%u!K9cQ<ctK6sV+=rbi`mRV6$;EHxa}T5e;bcs!2tb$L7iHUW z;ad^jpg2Oe*hYYV>snLFSbY6?{`?D3id~cfmi{BgYpV4zmY*TqSJRu*;>wdx&XZ9Q z3Q{MVHQx8|Qe(eO(7#8|3KLxu6N(EIv!neD;H_7Xq+b*UO}Op22w!hb4ca#(X)#LM zqF7{%yx0Z>K(P3V{Wy=sN>ki$mIBqJ2{Q|ad~M7{R(#GNM>?Bny6Hi0{#^XZsb z`~JHBPRcwTqf}PctlanCRsyuG;h;T7uApPNpd-S6Rch0a_P@+DfuQ{V!aRc}O#WXk zIsS)0Hs$}w;D2QB4Yd6JA3gXV8T^kP{IV7PU(f@P+s7lLsEBfChw)!t6^;99nj94x zI}VTsD@feI$q59n`EO9_3XKo{HN^=q|8aFeU+9GafpR6YpqvodG8FWjoB%*@Ru&;B z=K;IzCO8OY7W9J_6Av%1tSlUKX;zDEJeHcJLc<6G_-xeJU2e1o701ZVJ}@(b8WR(9 z4TAPuS{m-@0iT|p2IaDunw|Ck?VJ)5{Ig_g!Q8P+k}DEy)gsRW(a6`JoirDR}Y z`qHRDHq2N}qi{DnTP#zwTR7|X`MgA(P z#o!++Kns+1QWcPg{L?KB4%9Fp?u2tb1A;*;{>@L00u=)nmpCBUtyQW>LC?wx2=|*C zQ0-|8l}cg(q`P~2jhS*qGE{Mevu1yI`Qz>E36-UaOH4}2QYwj(%LWkxiRzd1dakdT zK^70Fw46yDkVNbnCE^WdjvVvz^Apq3Nb>}L3#ze^gElcE!`RF$bW5xlOrW*;mx-t$qXfw2)B6!BUc{sjCMwF!J$@LzI=3eM2f# z0Ojd3v9u)p#-Arg*e7akPEGlXmBAf55VCQkA=a!}oCd83=r%|;YGuN_83lQH{-BF8 zCE`BOEScYl%34S+|EdY-j|&4Rb!-9^Dy&8^NS}-Jss@Az!9#k9Fm~)X{`z<2`m5z? zzki>Yn)+?fxrhn<(X8K8CSyS@)FurG2DJ*PIY1@H{^j8P+x*ku5P$|L0H_>LhN&LR z##B(@J*2;t4pcL5Z$1X$e-UiBG*A@=HJY7W^y;b>NLLcXLCr!dTQY0HjD3B5jimpl zl|UZ_QOk!#ot|3hh5gb~4p7B5e`y6s1hfhNT3rQEYiG%1s+HOOeuENZ!2b~dsFD|# zQF5Dh{}(s*|9j@D6NNTT+XP&<+=EG1ES!(%)ow0KIPtd^0%0fZ5&dHbHd~VB z2SZmHlsOJJG&>qmot+=;o?NOMVVt;ILYFVYE!vtQI)(-Hfx-4MvAai4&uaUcOyg!kITSVzK2JUu^7Jcz@Amj zA9ieNY981F8=uW|@bWEM?3jrDtmf*PoWj{JSR{3}DdZrSK36!(#NZUrr~n!W-m2M0 z6&Nyw0j1=Ru`(-4I@ai9Q_}RB)O0fIJK~I9o>a#ai9G%MzVErJ4q=myg-J!iw!Xh{ zo=)B}BrH+p@U}{wS&{a3$MM;Vnx&K+k92gHT#4S>Q&jrI#A}-lDEkaRQir|1zL{1X z`7)G<#no%kmI4CfRU=9e@l3zk+DkOYGO+c=Ns#srX4;xjb5mVd>%0QM1NO)VVf`F1 z8msdlYvlFAq?N^({S$L?j3e91S;6?)EOv2o@&vVCk?<_JD-O3!rN$Lpfk*>8aM$qg zlj(ntPnX-E$zZsQwHxL334`}?kj}12HkXHV7cf2Itlb^>((m1=sg91~>IKzyWBW2% zuicHmS{Yx?Jszb<)_6Oeq31BGeplPfD!1J)&*dF9?s1)i3W`s7U$IY&@8ViGYUj31 zKu)$RN@8|}R;&sH>QRFrCzB!GIqccBP{feCcx37s;k?4)fOf^L9A32rwZMe!y7H?H zIRGHlkE5mgCR?U^A`2G_V&$vXY7b3!ghp6A zErDC%aV_P}49dwUmQ>0g>}qHtD6Y^*W(ypd~*m~h!$f>8E`zN3bImnl#vmpRrRd-z#` zX)?~xbjBd%P-=M8N2{jR$W6tByLQWN+*Ds(*dE@o!#Fz0!2a3rQ;)&RI*-pFNASSm{K0^7gM;?_ww#)cCYDNz={u+Zit$_m=IW z-cNBU`*ec4!}=aq5w8%h8`_@9ju}q{7W<|$Wu%v%T=mvXRdb|>AujK6$3+dov1jk! z>j6D@Mc(^7ETcl*ob6fkmnj^=huq^*4x2C^GFO9x>+wqSV!&{d{+esBReg~L_pY|} z)*tg?Wh-W38qqHs`G>GUkE+25cL(;adOF6n)afd)SGz8SA!D%?+f&}{aW>xXn!5_; z){=_E8*)>F>p^pWBCU>Xp!g;V{q!}KQO*|$j1WAX8c9F>aD+Y2E~!nA+{aiVjeFFB zR(e}5IF=N74F!w=T-0%wifLr5Him^!4v-sbVy1gh9Dc0@k>(){Frd6BMG7K(^Xp{{ z2)*}D#}G{cZj>L{M@Jg3Cgz7BYO9!L)Iw3QuTpuJbb0~ltSqeWEYlorog?Lh0nGmV z?|ovaMq-B))Ilc62R(u}l*6JLs6h-ip8;5uw!v>mz#RCO3C=w~&}J1`%?O)Nrif+%EZI6@==kZ&;-j1UqG zgG4S~lx1!vi`b38q zL3I=cQx0He8|?ISXJ=H@*_uniq}U#eN=)PJZcB4tXsifN2}rP&1z@T6P5&b5B1~i$ z+;jU4BoE^}!FXg0e3=nZJFb4j3WI}K`05>s`9>F;esGJ@lkOSI}dF=_(qg9$b+noARPmkw&q^JmbPUqM`*ztgMAaO}oWSy&pHmh-cRaysvs z^n3rF#$ZQ>gfv-z1X5I0cr%+n$ftwppTa&94I_{LB8UN+)uDZmz%X)SMJYe%k>rNa zO#TwK+>mInu|FovU-NNLh78D@Y#7-!CdU-lwD24?nxoMcpgoTjYT5hp`VDm8wpBt( zd1FMoW~cS`>Y~Ey8_uIoX&>9m`t7|m6x)UO%hD_MK;p-h!HkzB?~xm89s$cvVXuwn z_QA(ByLU^k+nyfR6^OF0L!KV>PaoPYvT$71Po=Mt=sf%aZ^H%lb$%XsCwaec;TueM zbxc0Cb@cOHxSp*=H@AY4PdOwlXKh=2mTQ6hYU1mIg z$Y|a>W4CD<%XVb5pND!SY_%K`(u5Op3N}6j<}R0Ya74 z@`;o+H@NvbbsV+AUFRJu9Fz-rU9mjS$1=R1Nm&w)wfGF(Jhr3KZpx%g zcTCi8UyI(fPcdd|k=+yhB6azpFk@@r0ILZdK@P{-=JOD?LWuyj z?lF%L5~5b|xywWNVclJDY3{0w3p>A9n`uF3 zlDzzPIxPVa$LI@r^1_Oi0B%w#$q|`-tTa802SXRR4$xJfxBTnt?D1?GzY zZ1M=X<5N8=rvSTJsWx+%BPA!>!-)@M!;09Av`UsuKywUdMmE=I9(M#AqkLG(ob(-) zyW((-dMnEC=|C5ge!|OV*L?C~)PTA3;zV0nPpj1ak1j!j@uU{34)Fe$&`ZuA^~&E; zlKy~Fu5#VNepmw8RM>eaxEYDY=f&m+MYu+q*5!8MSE`E z&RH!FT>7vAKI%xJ{fPz8?_cU6w$Il_DD{g&bB;`b7DYA#zD+{R4UUFW_3-StO6kmk9W;>Te!^<0i-?rPOd^1?VjBp zmcAx2I9YjzC6&}wTM|lA>N`DwyWgG~TjUE1XGdMqsMPwvQ$Mgb95IZI%mpEOB^=Jq zR~b4Y79gG(-bU5TpcrgpsWqmDmn2N`+NVN&TpaaKhPsL&)f=~~D_0)~evV*ZR3-l= zXm+4C+IOVVSQ+ZEZd}@Vw?ILxuc&0v8nxqjqM<-RtABFNEh22LAP-4$m?!JROD|wr zCqNwfzI3_c=R8f`CuWN9@JjbJ!an08aIjvV9Se>2fX-7T>?=|!ydT{tB7Fk_1e0CS zwJDe%hb~Hm(}GbR7pttl0S!|ywW9TJi6f>}W`>UkZFg(JR4@&m`*fAwJ4Tz;r1Dfp zq(O-kp$rT#HF)sg%+uuLSVliD)ibTwf3j}iyqvgM8M#G_?(0u1nJtpl8HH!7&Ts7* zBL~v+u!-MSwdcsKb^P!dXcPIkb8G==unqgJbg3oS@EzWE1LL;^fV(?+S}$&7;a)T4 z_PfXif|QiaK}{}pR#tKJOxf{$jqf4hICB%)BMhDo6puIf>&q5?k(p)QSo2mY;`ow_ zPt48uO%XAR1UPKgI%3vQ9k;=*u)R*4TyPpQwzFV2Vd*+nR?*!NC;DfY*Zo*z<`Csg zR&W(&X`3A{5PFLJH)Xm-md+dSD!CUL7YDY~a`5pi8 zjJ%o!r<6Tz4*A(_t~$Xo?A-qJhR0lMG=CQED=1Fu(Z%4FM7?f>zq~F13O_lsmQR&I`LzeB_#l~e-#K?YJe=9P4dXU?3+`FqefRreC>y=h)BG) zaGdg`o?^vemC9|Umr?xHq3N3|;t>WrJH9;s_Tl!2yFd>2*S-4%e6QGapVx5bmMh|? zAHWDZtGm`G=dH?1JB-_{k->|0=!S~FU;#{Hx}4l>3CnAMH;v2 z`TE*5U=g2t`|oPTJ>o5KUa^iILp2(yQhC2eY%zT*U2TXzRien?>q~zZ$T7*+Q_Hir z;HeXqj@C_sPa#8{W-uP}h0p)wr3iU8_A2)N7Rou@?p z!QH9jpLDt%LkHOMu{d0Uc%Y|ctFr4qD4Y7=snnuu#S z5Bywe4k_bfm-lUSN!&T`-K>`3noGLMq=?A;n~%i30~7R1&g`Zqr&+ymW$9X&urm6AU9|HRchQdgT}k%>-p7$aRRjvdcuediVTlsM zolC8*8`N=bCST-3L&F@nyjY#r^bY~#?_AuDDs5;Aky5lstT|21&HL~q^^($bxf^}k zzCzBT5_8ovs$Wj=OneimVC1Wg*yxm9Mnin8JIgyZH_n`l&>CfIv5w4=rAt%NN|Gg2 zA3?S_(ydk2HyT%hyxlIeG&8(EM2N2W5*8Nzp#)K9YnFkElq~S(gS@Xpg+)hR97qKE z7A`>#bz7#UX9P(Z9dk?4(&2#%Jz7pcvcjW%^39qk=@EM7mb4XCHuGoh3ISSc{re%> zV<*JMq!WqlREKhl{t6tLuv1+J8}N2(Qr*C=qhd9p44`avfXR)D3j* z%QdN)!veaxzR!Nm7pe~~pWU1idAe)Z58YH2)dSlr2M^iwqvJE1%Cb!TJgEe;7EnEyh3C~P3+e;ACIR<<7+5Pmrl++~= z=r3XV$m^Ii5h^_t@qc@12{*6YH!80@Ly!zdSS!*_FE!W7KV>g%F!mnS`5{Z>J?!r4_f25hxUp00|l# z%x9ft`U5n$Z_db}^=_K?6BagpL#X>{d+<^_pLj2Ju6cK(hU09YimLfuz5PV(p#ZPV z;u040p+d@X7@s{$1W#33C|^r*X1`=!e$<>AVt^ZW##1Q#cW1}V+M6DR631~22nqKXMQ-xIw^%wgMnB!DF!3-_rHe-x#vcxhYQr~*1Py{*&G}J ztcqnRG_B6TBn6nTw>X99Geif|yzSULms|TGm7|@Pm;u=fkd&uBDT7ymgPE$)&y7`T zT2i;&R}}3H1{zYBAz*VVBiFgYG+9u<&O{no*~SXQYfr({I!;fmuYX-&mKkDXYKd{^ z96e-to`E=~@E4*&d)G~G;{pb*@nN}~K(PcTPG694(1Ye}x)Js=w%TmOJn!IB5BG`> zoOOcrUzqH3M2ssW!FVzLFdh^us2l4^{3LO~)o7!NWAw4fF!}*>XoW z`m^~}P`)mo6mQ1pGH0)+mKFmvmFbD!QU!zOFvXB8dm)_9-cJA67J8dsFoYQZ)N}j* z?GEFrDR>F{F>PLZHZ%E?FH2B)VGARKabeOJ5a9^{jZP@r-d$B9&ei}F3;c<>*#a{T znL^uF5z{5cK9Uo8rfzW04l+d%Y6Nj>G#_a9Lnk z0qsA5Tfe@*b(44)J zMnYeSNh17&EKX6SR`+W_CR|*C@pi&O{T5bV85I;}99WScw)GViHki_GK7T#b!M+)3 zt}jDv=B>Ehep;{Ia>WU#ogj%?eSmd#hc9f%dt_1(cZtKvEuvS zjM>h{ujQyK8}g)t&Sp6($vpQk@JXVYF1}YwJ5VY~4y{xRJ%n+lPyXh66m_;LV?E!hz4Y~h2rEMh z?ibwkjY{ML0ia#p$mSgg`&mb?YLPODl8LW)$4xTGGmo z+)Dw0u7^>C5-U=g_E2djZxw(LWP^F6=$jnNCvO2s?+c}t5pYnhawLooq4VuxtJYx~ zWH@>@5s+;$FwVRpjjqTiP}ghGV&eJ(D}JYaPg?Rs0R_eD{{3nQd> z^Rj8sNL~USSc$3qt5USscQs@xcW60!GBk{+*Z1CN)@Rv> zY8qo?Ezk<5yZe4tSnhP=6#+#u*Z%qH9P9U+1;*d?bppt8yd zDJsS_>RwwH-a;~82$g3T?AMoJw() zn2qn@w)}b$HfhKq~S&i|+pUq8S-KtE45M&hKLx(;e2*%sh4or<;9&mUK3f30JRzqs8X!baT z*X&^+)~MGa7lJb7ltC!gPR_23%xz%RakKhoFBX}a<+aWR7Ng#mDZrN$XY0t&6b!}< zDeRIhv}f7{!6@Mb$Yw$>{L|%y|LXG0No1jZ(p8qLT}|-j=gqC`%US|aMh|Q>#7>h! z%a65Y zUvAdg2cL18T_2m&G-{tc?n2pYCwq85dai4+@;spM-Uo}^Ewi-^2@MP2))c!M-B=s; z3qooSL`c@7l-A3RNW3G<-^|qi2{TiGk(3r3Em4rPaR0vOiAu?pU-r{X?e!L|Bx92? zz9mclhf3-vvJf7A%Z_fKzpT--Rsy@ddo)GGs(WnJRxfx*D}M}XhJ@W&SyWWT z-cqHG?SvtcT?`5}k8e2c97T1(jYU}Ko3_zavaJ16sWyj@zMK_Yl0Q*wyFvD=0e2`@ z_4|yjE3g?Ti({!VMty3XpU!$rLxLjCm6q%Q(`WsLqWSj58*G{kj%DdzeoP?XUMfW=*2zasSwYh>59ZRa5?M-f?}3xohQ! zJ&&^UX@sWAARzAok2{63(n^tzm%4(wMvb76S-h@TKV@}n7t8-kvBH&DHWM7U1r;or za4$G20!0Ec4XqJn?0foR1quF*OgRr=Z#AAc8a;d%Y^J?TquIF}EM5M&orFYp4>NnA zS)u~GUqW$wO_=39L=Gid#(4P3E-C5^WgI$)1&OypZ*%nQ62*GyWzo7-2&>ciBa zsp$oswOm`cBhNM$z&KPVMqdn>__gcml4hh&R?_dCgEf>nXYuhe)8Mv5Vp}{xAZ8*O zLWu&y8^!wUH?tv~QB;a3LW;5qS^-@px@8C3+w#~!qvJ^7q@uKvm&-L+N@eO_a@K|z zKFdxz!l8qPGjzDJ4{SLosv-U~p-k>BMic?mNIizrFyF9c0Q#6@Dhlw^%=9H>nQtH) zN05WVLfXwT)p));*UGD5>KV)WJl)phAo}4W=cx<$~GM0e1HNiLD#p z=tLjMoAS3Rj`f|Ydfo7R@iRAUo544<dsQ7hi8xY9wrq8SsG^~EdIYEE|q`P{Yf`Q;_O(tv9_U4Ve3MH;d|)UB=K zh1!f>g{U^ChBGYeYNFE{CPYQ(6DC?1{8 zh}148XBNkcS&sILs-nh?2GLwr;^17fYbY%o(@HgJfVD?+)lMsFbLHw^%31|12mynq zJ^2cvy?P29Bpi*QC+F;)+AE`RGG4wLfdTp55k3)r7xbA7L6&B<$x*!ygBn+kh#{qG{Y}h_MQW;P=QL8^Zgr%3BB8lmP?Ql>diSBt zrp?6Gw$h^d1`(vt_*rA+5B~YS@o*AN>WQ4gg?eV5s z^8?G<9BfFahAyocfp`h3@VAL^VD?{@7CBn@i(s2(gV~m)w4G6eJ)$>?NQcNo{A>CG zwT7aDKFPAWri;-s45#C{Cyn-EF1CgSy9CR~lT#~IWLFTsZ$xeE;(l~GHRJ`kh3AHg zXG;!DFXW^OiellJi?;znA!LcORQGqtLVgKZo8Oyk)Bz)7Bn%uqBSMQc_82H|s=s?N zGQYh#7Dme+>`qI^GIK2yk008NgW!mm9bqmOpexCV|N%)>eLu7-XHsM z*D(Z*?-4d_eK>6>NcmYyn%?q^e!9#ek%!|@(>3wDwoxbIbk*jZ9`tbqbS`b&Zcb!w zvtu1X$(njk>r(0TJ~XA_A{fOr#V{J*bEnQYj@jpzG+B@yKak`nans)MTA;@<1}=7|F*mR>wOpxIUvQ=h2Q)TPplQ>kQN*Ic z6W4bPEU?4t;c>CKawuWM96=trf-Hm zg*3y%#X*HHVf&U8Dko!JbW7dBe>J~h5bIx@09lWwSf*EHwyhuY9zoGI(N*utzQ-P| zB{9*Yuo%G0+BfiBN>lRP&rZ`m=NDf5Z;b7OE()Q7I;57AdBVPkp0vrtu7rui5{E=SKUnk}ax!3^0# zTzKKNf@WZaxO20|dlwrNRfn)@-3U(M7F&%1JX$>c+k&vt@_g``R?&kH!u7mDRlGs* zp$t)Srl%EotVqTtF3A^~<8no|k($ZGhAK9f&C}*z_EDyf7jr7NWkFM3bWt(53xklY zj$){wabFcSWtc`-qe++t1pQ}O5He^_*E$6gp5+V+D^rbqY(WN<3`TX7YN|f3?o?A2 zUOni(2cYNQUn{4O)*K1yh0kUhZQ1K1sgR%E6~)FyYJytAapLOFWcjk;Ny4q#$Wh z2DUG&$a+G4en~9kP>OX?4mb%k7AX#Uwxc#oTLCQsTp$ZT*Hl8$pFfD?9MRIu#reQY9BQ(1iMSjjPUJ_}QLd3jLH9J>CuYBn6 z(XNW@=xB|($lK6-FvvWOfILBe9reVo^J2@zj5rxauUAXd72a7m+Yfv=Da-mB`7vQev@{A~z8d(kBi z?=&I!@Sg{sOo`LwRaHghfKFoQVnVW}gwF$<=k1nL)_QC;z4~tWM!OODdOZgFVG=w7 zykCZ$C$=Z};c6~swnTZ{5$mmg_!ys{X}C(UDR$9l&r6YV`Y3t7wLVne1fHz|%0VpN zYkS3pTWTqU#{Iw!5AOFGYK3$3*w0C*?qcfKtY00|k3j>rpTm~3*8~`F4;<{J58|+u z^QU-*q*%9v7ZiL-wcjzmi@8$LDKfyqk3iOH^O-#%e3C5tuG-hIm&CoNX-ip9`-tUs z-T!pEA{|}M7vBCbEN$cA{ssi-6DPnbqG~(6j06@M8ae)%GJ`BhBC6t0kW+a=XHXV| zfW`8C5_w{vXDfGF^%BkGSWQUV9aEe`qy^f?=ABnT2or85wmLzD%Eg1HilSFuq?k;d zOVA8t6}1ztaJ(-mzqmf}$7VoYDq+fW@XXx0lY3F{OlCq2Ni?OM5Q?-XTgijEjVg_b z0*)|)Bx9VgKD{s9CpJ+o z+Lkkr3#hY$yX*H0~&2Uxc`KREwypso%`I8%ohV-r^kef(KN@uL7a-3{n+1 zC0VsQ*FcA24J{j(wVXp^wiLMGzq7Kef2Y6|v*(~JkXKmIt|f(OO0HXIhBfdwo6i3LDhtZLF!DTdmUXNs&gDN=%=u>;m+8|3ACAK@c z!ZLGDlWPxQ@ghvb8}1i^6%XLt5XJjg0VmEOP8E}1`nbP;+NA}El*kh^zS>WBi?6^X z*qm}%0p+Oj2=FKZ)T(7^(8|*)(<|b@Yf@wGoA#8e6%}2`4UvcyFd2j*S)FFF6v}jo zYqASkdg9I5g{W|gbyiL;Gx0my!>ch!W}6Y?2CvtdH~3nGgfhUg(tVCVk=#(GY6Lgc zm=P9Wx5REA%ZwbAlTDkMo7dTK4UwvBds`4i4C>T_3AW$VRN6`dGlxUy1pjlS;*&`S!&G z;3FYLtbx;KTAx{{1QrFY)jA|cB*m&OX!}j2%?%Cy<1gzA!(W z{?@IF0vb;rm?UF}lkx}QB&N4T^G)q?A5WVj!g1K_m<<@QKylYv6&Sz277=6M?VmGF z&ztcAvp$CXy>Ii2E-+iKN|ZJMgz;$mz3|U~{m1nCwT@$#5c%irr!zSP_~vPUE7?Db zJZNz5HxB>2-uX&fLN`*3A}#cvPkdzl^X^~k!oLc!N&K^U^cQn~Kk)5V_=nc-djItR zH=pQ_59nVtT&G9-*E|0_2rT;N@;}9axcE<%{`0(3_tz5jpSu2efFS?>q{_e8Und2n z+~vFLL<|znfd(dna_rJJ?GtIxV*EFG-yC6CR3PiG7Ig{6<8nHWc)g1Vi-;g*H&ZH8 z>m7=t{*_ht|NM;r$g3l?Ve;1=Tf#!KGhLPpbi%OzdSTU~#qmOw-;W<^|0k)5gqtR_ z<01gbNO2KTK&msJVVaCP$SkR)lT0SU2vV69=QU4@<%Xyn!k9ssB879e)<3-eq14~@ zztoHW&m{W4wGse9`qCmC2k_Uc3ev@F!4;CNR4W6yb-maO^-yBAX|#x<{twpPDlU$1 z+ZGOzK+r&NO>if;LxAA!?hcK6Xdpm<;K7~7CAb9F;O_43+CbCj?d*H^Is1Rl_wYU3 zeuCfXs;;V4Ys@iajY$L`^8Xp$gkv!pwOn{5z=n>}BzVAm!ksxk#I&!Y{UZ-*i`-P! zswvs@xfz)(nN(x5;S;^Zb^?>=ie}Kt6iEawj{+7eLuDsjtZbqabk#bMgu_+kBnJ;> z^wvS54mPpYgR*kv_e%yU}yXH5<&Xo4YE|$zU z?=8@{6Rw19{o9qvnX6)ZmE2~FnVPOYBr>ds zAWbB`*%FEM$LZGuUko0 zqO=s#X{Gk@)pR5_qB++!n^V9d*B}(XExB_d2HIChVX*c(i!9vv2zTDaq@(XS^nB-D zfQ-WMX43c3TmOD8-Wr9sv$(49T&4ZyiM*P1&v-VS#nb}IomJHa67kuWWeTQCcX%d) z2>%8JtG`wf7r{4p7FBeRcOKSi9~$59s{)?ss*>L$pYxYTTo2T)T*i}wBXT5K?P;%) zBG&%A_5H2Q72M6f@&p2HTPcm0t=?M6`(L91w-P>W>ydAbwfK+KQaZayeM_`59C;WM zAnQPn+a7lw9&NZ<$nrkn%FMW&^K>4&B()~#Dr4{1hHLlkrun+&j_Y67p4BbSa7akj za*XHT@5N{{JzNvdx)BFf%H{W>U5AN6EU5Wq*cBGt?d%K7TX{z!oi_&g@HkqO6mL7~ zkNOB-`*9^Uj*Xt^QMCz=AIQF^x}~%;xGMLXNy3 zh=CXT&|Nq1p@WBwuh4bWXv}3%!m8@<1E!wT#bkOb@7+Q#|>m+CBf7tt0KMuI*!L3(iI?t z5^bEOnE#mXFvHVjGu7@wkK=Iz^b%rz4xNb<{_~_)bm2}i-ooOIWVeh^EE#)f5aTj$ z0J)smpwMn*61XYX_vjj&`O2gYW~&!+qg`(SvoV}`jPZGr#(S=c&lk1#rR$*$H4!MS zTnYI-9w?$Dy_w7K{&+6rNlIoHX3z6NIbWpZEVhM3MpU|M;D2wTXb<@d5C1mzUDf}h zO@^@A3A#jCoCpcnwzXcvQf0!U$BTe%_=BNKWweaxImaVltWT$?%mkX| zz90$OTMR=CO-S)S@c};TDJlC8J>1$MeLuMN$>qjd zM$ancU)byJtVWMT#PpE1#2Eh_((nD`8}|YpSPFY=rG#0{myXag40gR_+()G?rOh7s%Cdo-}{CuW0q_`)aZrh%+Dbt zd=QHS(71sB&|Vx{O!muYD8t}rIEH&Ze(+R2ZTV(b*^~y!nCq!0ZXZ!llAvck*?L%S z&NAtLQ*l2s`#34dBl2{lwaA&ZBMojL-x7H7MH*^uaO(P{YnUImb-C=YgQRz*?*yII zcV9f*A%DPlaoOhexI;J3#kfrxek13uY0gts{Of3HV<&cJN9f7Jp%tOr2zy%*uj7QEpV z@g-t;JdxsaOht->12oYKdx~_Jt4_CNxQ+?O7)Hx!p0W4t3??tM4T!EpC=Y<4`s!_`{X5kdf zJ+;0nv#_cZG(N^*!_&O1WR{7U&icj5=lvR#(C{Z{eV9@&KEn@c1g_xWbp7n9_9OI7 zIh9(FId0Tg77PD_-k~FksUvw;z|gR>M`o`Es5w%z2!m0tIMOfU;cT(XhFp96ejxJ^ zI1EplvH01-k2uBw)8&eq^q`eL#H8qE2erv%)cJar_ZwOB>3hKaK3h_a1{3NuU4ZUa zA6Ucq#)DGG#<3?=thh#%`jiy zo>s)nXC$n2dPiRKiJ+}0TFAC$PD4{Pv~8YEAUWIo&F9OilYG?1gTVKf$GC#MgdWdc z8NC295amD#zJEqLmt9yR*Fnk0YO9|}miPnx;exj#yo_re)aQbE4?`RWSIZw-vqZi=z=NLlGEaCNU!lG`;Eng}crriY{4di2^EdAha{n5e z{vTq%5N5ql8$C}vxQJ*=d&{RGezL}Ws}p~S+4=514U7=4tlse&wXuG>-7v*7Gk|#tbIi){m}x54B0MXe)p|TfB#mE> z2-;fM5O{<4Wmot=@@ngF(7I5UON|@F63EPe5 z$08dXTI)&%cI1wNH}ZN6Abb-W&8L+THH2_$E0?vB z!w*9EeKyAMClv4CSe3W+Z}|QdjfXxDhi?NCu}@W}~L-hh56$jSOkyBc@@sRMp5V^E+R^fHrvKIwNXy+fG8G$FBa*vK=$LH0i z7m(21EAT@*s^H7>tpCeP5rk$m(lURtm}l5U$+Spx>*op0jxj#Vh0_YzO10o{H|Yw? zxJ$!Ib(LTzvy`4JGNw)R>4I~EMQKB<D`SjM`*9*spP8J%1$A8RieC?^`1l{C}XbyUbTVIF1y;p)|+$+g8m;JBL?{t@w zp>0Ml@FJ1>^!jdYQN&lyg?Q`<8unXqFK3w0oskc$A`mJmj2GsP``JoxrT;m^yN>yf z-eGg!s9X0J9^gTU#PU;&@@-rHA=%N+RcY;s&^4XKn-_Bj+ob)MqnNv+Fv7rCxhco8K1pZ&sL$QLkA384R?X+UtR0W+{m+}8-e>6o(A^iX{}-f3iSozNO2LufO8@wN z#gnadAb^9PhoIM;#2O5?vhuleGvk|1b-!(xH7W9G?4#tmCxLY<_3wl4RaZ}%n5jwJ z>oY;gicrqelc$zJ&dq@XtrrWtD=i`USr)9}QatO$wZCRgyp)L4e&Ddm~3 zxZQFKaMcMaKM5vpWX-vtp||#M_A6ql)LL+$7qBQn(Br?FbEv2VouIAydN06_>Eru@ zwOPp@K>o>~C8OYyuA>dJ*6S!gKjp|Idz0v!+nVc=QBFjzgm{yh%7|e}(w3#;YV@dv zJQN$2_IsveIr5!`=WhH(W%EsH$-9V%eigbYMm0Cg$|+bM4JF{_`bF3{%hlug3yr@= zJxE^)SGo7umlWXWX!;T+vNgD`H2JIs!LD|zaCLnv>2vSIplj1|t_Em2-yR`(lJnbr z;qR8deZP{eO?tj{7(Xod7|}7Bd^;cHq9goV^w^CW?wRXiho*nhTe{YFi$rGe@B}h> z8ZiQMyg-q=8I6o79i@tj%fjRSM4*jPE$R^?;2f{6Z_U2|gN^QXmBeJt9_B`YXN-E# z6Z}x9W5-o`D@@xz&pzF|G&R;)6NPNH5DrQ`=8mb_dZ>nTn5}WT*FM*=T~L%oy*UGB zt=GC#+OOtx%b*>%gyMJH`hyRnQIuICDxcx}pV+f&f{N_d8h@Y=xDLv2MO${xV!q-stZXYqmPOvw}gyDE6rf zLd|S0Ca_$f0LqTp@DH)-WV5Xop80o31Q!o0MFu%|;j{xf*j zbvpb7MrWT1ITgg|gO8hG<*1lybT}|->mQj;vUiY*mqcp zE4hd?rEYEN6RAl%w839q{e`|>isQQVk+hvlcgXQXaG<^Ad@D@EFZ>0fXCl0YWp+KF zCMZxPd4uuON8Fv&o!L-|SnKX?9V1YRD4!958M>6$KP21M=N%Ic%p}SXFPsEzti+OC&z;UJ64m-;LQCM@(A2Axpm8P5FSPRp(F;7M?}oC3TojiqJ6EW6MN=; zdj+yxr4G&EBFG$v|80vP+e)X>%M6~FK9VOQ4^kpsQbrFk zkGl=y7sn#CnduVT0JDx8M1RN~36jXmaKin^UU#A-o1ZDig)Nn_l0L!j$A!BX5eT{6 zuVSXqqwXHj+MWzEc|Te||qx`dU%=36q59Y}l`5?k~ot_RGJ+uG* z{d)xk3M3l`UqFGa@8{i9VVXp~zJJwm*i)%e<>TaOWN6~4;$<|ov}B5=|0Xt^ct-?u z1RvbV$Qia=jqcGDR5H^SduPisywip)1ww_P&fE_)-e=1(myabfLy%SR*{>qcw@3>u zC0S9Y>2#4Kdjd+ry^F(s9_KbRe5Y0C?N49kSX^x9w$55Gl<{7zVoWZ~DcYDMoeSTh z8w$>iy13DbKoVJ}bly|zR|kxJ<$XM@1Yei7RXO96_7OzvLM{e!3;_0Zyq7!k-baaQ zk-b4HPXqlW`lnsLR=elWZ&DUT4ZV(R@PZUb7CbSR)5Oq7eL`qEASC$v=orBJj_t_@ z?)TN`4wh*&y1)>8kYBiKd->yzl*i_1*5)8BXzQ*M8#fc;WaOJvlE_Z>d?y5`s3sJE z3a)#d=qO?^)n4X_42}!8W*VCWgAK&LH=I~V`M!nK@mz+`?BkCUAi?ot|=`db7{vkG>pv4YGEHNX?J zXA(C`*el*5dUkSeeK>$%J*8@S`ht;9TlE(MLd(_;4xbe~E;G-8>HE=c?^)Z@jqZb` zY*=owP(rki{SzxY=9G$?(+aBLenEiY_P={M>pWGH2sZ@Q?`T*vGxHzMST7?$BA*r9 zg`y@otqlFb``aI|SCct0o_5AB7u%v1Li*5Ovp_s*Rcn(C^f%67Xd4tJEBqqzNCIg{ zGPTyXNWl=Q%qZG@9(RT$*~&+e56M+dta5$xpLX!z#(U3Hk}LdkZ?=ehdvSiO#+>_N zpy>RgyI)|OFZpK0Drth%gV#|QEp!bmMJO|S!SOQ8T2sl^IkZ8~l7H;Aqbv~c`K|p; zcz?$w$+JQWozB^wSaX&X#%~in71LnXR?d@s*O#k)8?@scO({NKzt~%G%xcJIq)YJn z_TYKjz~I{O%=r_657E`ge#ByJdF*9;WoMty;ERT{1 z6sOX|h+@v;=YyU?A?vc>UG)=SVM2^asK>MYiBu7T`T%Cm!@_jxeNQ6X)RU{cAMjAj zioCAE&YkW?-Wg7JKTc^0D~QWm<(w0=1#6PEml6{HXC8Eox#C!r4hQ*7&ie=kaB)AeAg5Te&?J-NLVet_*V3((w!`c}ZR0942*g_DWrv z^N@Y~=0fmwQwds%FZ4WH`7D?&%ltg3o>fpnpHzpvbR;*ynf+;a{(0VictgCS+uy5^ ztc%pb`;PkgqS9!$3-r{m((*BKgMlY`W2%SK!{Ys{1RN($YQXsw_~qfP-{H7l`l`x5 zyyI5`yDs_DHnMVZO&apBkquDSCp1}ESy(=ouHeo@e!8cI$>D8c&Wn%#*!JS;1&yhy;m7}bl18(<_XuxKm;1|q z%F{TLgj#bBA;n0!RD?P{?;2emO~CUpF%WxRE`lB@9(WQ>;?3?Z%z&rW_yl^R8#)qh zYY*GNGqunKWuhEw3)924` zDVH$(Hnse+5YSEB5a+$U_ufQbMmY-0%x}O)pB6bBfArg$BsYVEnEvCj8~?{+|Nnn0 ziTnJGS)r;#%_hbK{6yw=$PiXJO=kNAavm&;e=?ZcV`3p_5ZII7C40Su} zCls2cy&(#0P(>McbvD2(usLWBFgdXPFPj5sm;U}2Nc^{O z;C~FPWdCPQ;NRmLZ~jje$Ul$ZIrzWN`ggqXU%}1)`{Z5de|OLPkLz&wo7wvx15W=h zE*Vy!^4~w_P=3e(yeS~lmq*JiO|Vo}pB}0p&78q`n7G@_*rC90rY&BZlTlb04^=e% z2@`DTZLYaU=hvbW)W`gzQBJZjEhEvh_oQare9kG=?~ z?2M$W9qXd2mqveuFht!VJbp1juh?(vmCgJ|Kqo89{3F>$eJ)P>(CA7_TiWqe33Xp6 z(gD9~Y;Md+5kCOzr}9k|1@>UQI#U00-kgSQLSn-2J(fzF`s9rH>@qTRx&{kV2##n} zX2hh-Ue}Kuv+ExnS}>6J=@OX>rS5E%yw7hS$!=RR^|kz-HOY6C70s&In-pys4y4(@ z29a%UEiifGM;gB^Pj65r{oR;plcq?gsi>im!lRy?*Q>=RATZ2duKvotxLLVr0MG2#vj!E7{`H{kNu`dF~dr3 z8ShsFj+!riinjcwsjB+#0A_plJ5EC_Ry}*7;BxYZfXlI9`zZ!##hnM=tajx zbWct+%VWa5Qa8(k?Y$30O3{_hl~12q!doI{tU^au4=5^i3=%%E_ZDSf8UTcemBQk6v>}Qh1hG^ zf?Ht9>zI)^7P>Oyw+iV?CCs-uA7S`#&hn2X?;x}rJ6-@|xB1ibnyf~9SEgw-&9?G# zQm*p^gsop__FaO*S=l-HqluZQvEQ2Ma8Kr9Q}`t>25^uFFN#kgtC3XyP%_%-O&+V- zqEqDvA`zNZ5PSWjH_&`9l+Wc3jgicgAb8($a1XXqj)3keBB=H53X7$ei$X%>dd6a` z^&VTORwseDX#Y1ppO9dvT`2Mq;hS;t{gqC5=!k_PnP1{ss{4sGPDicecu9B0QD$B? ziRM1d;qjp*w;eWeNYyH@FRjt$^#HpjRY;5A*uVjG4T`e2W&fD#(H3t7E+yVf3OR4@ zCh9aVSpyP!g5HN@nRvw_nIi0wnDMH(>5_`lo?9F9c}qeP!!OQJ4ZLHW*C>`{8PXG> zHc0gJEPose2?g&dXhZNmD1RBzGOmFjN{yk#yWp1k_VY~%MC?brLnR(T&)7Df|3u&W zpy-VO6?>yanDQ;%aBpj{JA=@$6Q7wTzkpE4Cq>JLW!syt5NYgUDGXfJB>YD-b2kjN z6$nfLeC!?QRU;AN`Mw?Tu9}nuGh+uje?QA`Tfw{|=K z`jrXgVCP~)S(CnoG)&FJ&VXr`DLg#SoE6h$OC8_!m<0yC7Y-c%iNPC5u;9wqXu-a3 zV;Y8CqA_z-wG81VQ2KH}r-*PL;8&)0=lIp5#axCd23PV%i~jI{esrHgdhx{^FCTmG zY+d>j^X>r=pe)Me?l;+e-;xbOSxrT_ms7uoY#yAtIH6UD)}Up;hX&>X{jG@Pl&XJ0 zRnH@05bA4Nfn_@P`0JR+>H`gMHG*C>SGG4ow-m?<=#^5F(S4yGUjHD{KgJNjn0Bwo ze%MDyo?&4+x>KP9(-nJHsWy}8m$q%zJu`2Q#7R+regc!KVMmAq8=#G+1BARVNzo;& zo1(6>1VAY|?D^j34&9-k40^v1wOovO zr>%Ejbx_lnB;;dFhyF?-0Xptto1HG_E|5Z++2hcZ9_q<$eDw1dd9Dxp;K(0~5GlNp z)u#^+>BAFT5)U)pSt4^v>$_3vP7>cwF<0bVJHI`mv6AX!m+gLloL2M~b%k_kdEElT z8C!$-wj?7nNd>G+kheIsT&g|EMu*`^cni$O@nOB z<8CVYU&(PLs1v@wNzIU{;oGaKYW$(OCR>sdzr!!VpKi(=OCvNpT@!md}D& z**}L|jt+?Cceiw|Y*S14G6OZm3<`SP6ZLM@WCGi_#f!BOZeUuud0ygNO9vz{A%Kah z$B%(;w3>#8c{fdEBTNZe$MAC->mob(jhDVKGB~${@VQfnO2{N=>u|L+B%2TtHJ}=m zr7@>3)4hE)M47B(W@`LndtZFTvnuFn2?f|KuAerpV%R!x0E~;frJ*3AY8yiwkzY_7pZhZkMu1wE|N0aE#e^DnIlFW7Yr+a`+sE@5HfmhT(l1}f9QBzp z$Wo5TE7i^^k4~vZr`b~#N?2uktJ&zyJB(d?3TizP>+9@ z693W-71XCkmryjpOwP|^ofN$1O*g)cghS%S@y*0q-^@6byZJ28Qf00#6Uj_NJ|=RK zXGX!q`e)7&hOo;x9(>5yY3GGh$uIj{njx;Cf)$<3!>+e1m7WutbDu4$K)X%eNvOkR zmVqa>;TM)H(Z?H$)NW^M3O~}V-{eRq{QMme8u)u85|e|UeQ;m^IbI=svCWRLKzL0p z;7Wl+aN^Yvbpvl_1TD?LjsdK!A00oD)fj)as;M?A_Z5$|@<4^t7)LcaX%vsz5W;HD z1p1nB8bMmfHox(uWpq%H8$fTLUjfxPUEf0TnD4IhX+%khjgz}?;Q*O(8~tRhB|fC) z9noLZJ!AU^>I!A&SWp*#nR$m_cvVJ}NJ1B#>&k(?jDp2}MRoH57dP>M zXv0!RPbo)_9|{Nw#50{9Ygw!VJFMIxj|=;R#7@2@YS40enEG`3s>}Et^3;kqPw{*lu#hR38x$}}xEd(rIEc07$KbBM zVF@kRWWfNPS@2Hw)l+0H$Gu5=BTHW~l?Sk4-iK}qkI{CP+7?ar>!o~0fMc?X1ym(O zVZ}4H0^?@C2bs{R4x)LepbKDfRg2Sasr7pwEg*v3(G%vRyjfxV()iN=Zr6k%*@%VL z?X^xbQ24fC-dNJC_>e@CPE5l49lFHFOsXvB%TUuVr%ZB#Oj+>aYCBfV(6dXnUjk`t zC=opzI=>=(t-sYKeK=*Hm?YS-@``sN=s2rsPUvAo&qD|vfe&NS;ya{zO(|x!50}<( zO54Ice2H&lVqFpNEAE}}zbdEa|N3VCwW6Gc=^Hwa{!qnxSGZ(yCil}yTjQQoOf>zW zt1Muj8J_55!d1xpF3zoWkQTCMp`&Y9kk2Wr;GTl&+lHnS9#v>26%W;kcJfIjZK>f?gInAgSUh39Hn=%WR&pG}wNRw5ujZYh zW$?2r{>9O)vz8ogK>*vEG4w4vmadV?DUNoTg~SxFAiL`imQ2UEPnr#)El~Tq~VE1$$8FJ{lMN`oY`Fs z^)FI{f{O}OZAsawR?e-V&jqg@&dxsgJ<|mR+uj1~Da`4UG*wglM4rEmJ}|=;Jaetw zH5!_-nvSHq#(RyYqQm3!ZYSc)7MVg=gXk=V{T!qQWfiP^E8`S4@}3bAwV-(3XCmDr z+fOV}pC&|p7}-%2vyWh4V)29@mW6d9I^vb&7O)Ql=Ww9CUZShMlGi6Oo_*^q;c^c@ z4xzGMtdF`k6-h0Ss7V9GeZw>mhxvuN5+n0yKWF?OD1w0 z60sY-g^s3ZX>GIp_SFm_-+*mb{zjIZkn)ptbt#((lk+n{E0&DygOuo@mhZMawJo*5 zDdW54C!b$uws5`f_UW|$n$wu>5IQwG|KCf?0giuR;t?P@h zB8Yc5EYD0kB*H4O${phID?Ea3xjv|Oo0iY{1iQXs;8=9Y^DM7$VfH-E+s1zMRTI5D1PbF$c3gTH(-5uX>r;O#2c8ySoisR)vmKdSx(Sxuo_yH z$XLF~R3fQj>M&D8S+I*-N!rXmBKN5Mxj6 zaVCM4mj8S5yF}vCX%m^V6SmOPLQJ&pfOyS?&QTPrFDOcp1SDTC$d1H_wCH1iEK*%J zpO{4a^!f+)2XRsU#(mxhcVMc`zY5kHh#$pQM&VrTyoFYyMJ*?3eTZkyUWNn0y67dO z)58AN69(45bqD~k#2nLeD)iU+-Z2H)30^HSy`C4(F_-1*`W&C~6w4(!r5l3DYIU}R0bz4Nv9!nxwRiVjuHCZ?$$@0;OrKr$2- z=RbGhP?%<8BJQwv$z(7m5PD{M(iDK4+fbU?=QLmag4 ze7Rx71Lv=SInKec4hdU7@1O8Gr-R*YMBXBzFOB0?6RZ^hiN!m#`%YK-9aq?K6B7!= zAp0De)6s~s=~S#~-%+_Kp8tKH2V@D2$Q?7R!G%Wf`oLiKie9V~Jnwf}||L~Z1B z`Kp_<3tR;tFU;A-bJ@AV$Wpa#4zPHrk|#xn!EEElu;GSHu1GG7Sjd?biatzY2{V z4OvKPauRuHV%Ko4WvAZ-(eTv%Jv=((6w?$eJmm)Pf0y2nLwpA>3Iu{2j30s)T1nBT zuS|4>{k|2N-9M%XuZeY60TG$~zS%sm*{>y;K#_HY|MKoO%YX20pAi##zZ5mvdzoNj z-WlEC;LjIb2^L?gB2e>^-*TPh=F}kn%Q*)k|HBkrka5ueTIw;toXvw$;7GY424ent z<$l^ix3;l`iH~(__t$Q1h*8baq1wv9nPGgjhVqk>@6NHF)y@-EVNmuB5>hvO}3sZ*(FnJfK6bk`}%7@g9TDp8cg*aus9b<-X$O{Z;en8X^z(Ue5jQW=dtlX~1XADlNpzJaOennPBkX!tHOV88b&5AHxErG9raDl?{ z0GGtgSeZ0Wa&t#l!n1pdr&XaZm}I#t=i5@def_*XCTR*t+{)NH3N9rYc*)rMn-+6A zGTz@KA})!u5BdFQhp*hS!tcMDekAVVatH~QqeL5_mo?>f#Ou`3=-ie{wDR`IP@(Ot z-&0-7)}Okxz)N=dMgh{M-dyO(HIYZSkQ&%|KVoH82e9jl*&xkljhPZip}>pS;NZH@ zpDLjWj#a^%$E>NW#RK(~C+eAAY`vaB=i*GTE@RadFAO`NV~ z-Bjw-qZ({(4v9?Q_uWidGZTjRTix{ikJqnOr`Z@7WIw^Ccpb&piuI>GQ_kNx{wOTv z`0}pHn#}~%hnp2on@D77?vfZP;*BrC-!QXOU_wJ7d_)945``XXSDalyGbEtGFt)Hk z7iBRzbAXD5p_Hv@*_?S3J4&L8nG_kjm@)ef%VJwT2K}d6 zd7ainlPkSygj$Axds>)AjFZZ{wx1tk$t-Ox1B4(X`7AI1bQn{*Z?qF<7G2Zf7V3h! za@3tL7V!co?xg<4db?UIY`#^5xgU87js!3l5PV~xA{2fgqHSTRDY^Zyc7%Ogg2TjG zZM%#U%Is1?73TJ49zkKcT8u4RoExX6tXIuiI3CC!!H$-(Yk^&YX2&Q;mCN?0KDdwT z_wreh2eDS~114DAF~k5Ns6LmrbN1-qg44oBi=ljSfk{CaUBQL^o_Z zyB(5GCm$rL-xXE_Tn!Xpmlf60xycJiGZFR7aGl|FrI7;f11#6Fx|gnqnoomZ$|han zJig!au;7#6=vIIsUKA2*8VhikT9*iJ0t1Cl^K*m8q5vKV{_|Y7cxBj;;~Sc6b?}r1wDe)5x%8HKmzpR% zppUWAv95v>oM-~lDxBX-NaDhHqj5hE7YyJD!Z;!m-k;$fnS8KUnhdX|{J>n3nD=XJ zyJK?{7V9?wPLa|RVt+}pa0U$UW)mVTe0FMBG^{ zEEqMy4VI7d`t`Z7jsWB$O}(98^BRZWfMFO}fZ0>RN zyr+k;B!`9XxmfV^VT3G}$E%oI_jx~ctk_T|t#Lpl&cX2t)mbB04D@|5a{P%OQU~)U zsu4cs=DZ52fJ328Hx-KQQz3d*Cew0WR(1$52WHB5N#vLz{rS=jhlZ|9*5nrG#Sl@% zDewQ&n)kbfHE%H0Wn<9z+@NJGFImd-yK3ooF2_ujW$}tOOw8rVnmUr58wg^X=F>g# z^e7c#986-dbfTcJzT*|aORo(1wb!ei-Fi z4yKQOJ?Si6-alw+hBLL3Ad}Ax!0|LD$qP7DZpkZI~ z?i8YUO02gjF}bmRxFp6aRN;+pFa%p_e=(InWFqBBH)q!ABo-j9#lKLRFlG7+cPl9@!`Uu*We#r@sI|HY4jMq#6clXUDf2FJUKU88R3N2=9HOd` zpyq=4S(Si7irnY9oTQBJa(K6-wnta=(|&)!?l$_u1lY9v*K3d^#sLtUr2DwMQNL|| zIh>{b?^*yIf{p?8oRFBOJ3|IaZY{4cwk+>^Dg*?s1OSVLw?&jX&B)ahQLHwio~KOD zwmB+`?7wc1;_iloQI4@k&1OzBDNKs-5`T*-5oAsEqXbSd%Zmc8iz2sPk|mxhc4+G> z%IXzX^cQ69H{>*b9mWfmDNgk68fBD;&$VJRb#67Q(c6uDEK1QeNeZzi=iRq|0LC-E7Fh-?V8R=S5J7Tp+XIp=ay126VR{?NZmziQb#GT zOGQBPP{Z}mAvoR!o+Hx(Qw;I?wM6&F>^U`XO|CxXEUJ5+Z zuOMPEYVJ{`z>LQytV>50sJ)IP242yg(PrqFm35?_UA~rIrU4CO(b=2MXBntV%^iw( zr#Kg-U^9!(T}o5cCfF}^Kie~Zv3CfIL(jZpfpAQSNJ1+VVAQ^6l!oA#Phnl1q}xhm zfNJnah5we!RL?o-IlhvuT&kLbkc3sK*?4-T9?QxkE-_M^J+&a$p8UZB@IgMUB*5@1 zQsID}aGe-Nd2vAIvOqB-dhXx!8M$yt7+BELBg?u8lc!W=&F^RVruW!QbWI$bo-52b z<_4_w&K0t|(T)}bu#Clw!dbqa4^MC;1N6u`^=Ikvt16q|Ll7d&12g#pM*7!{|0w4hSg)0BmkG>xL? zIru)i+81AsC&A2{un?)`fGDlE_ECAj6wlrm@~k*?jYzq=PuQH%!>ND*g^x|s^nhi& z%Y#Uf-x-vt#BMTVHJQxl6v_@b{ggNMRyiM`bfi|tlO3blL zLQA$?lVcX|-J%>{T0I`kI0Mp+#N<;v`Cn;HKsZRX?izHpF~9tVB1Rvy-YD%dAtlXg)?6)-4W%ODdnxlpv?UA}U*nR*wcmH|CN|I>|ZW9r;atjLp7af2Y z-(A?5vwJh>deqqYaNM7~Vg;p8%@NSF#1+7nFYfHLx=EFncNl<0A;h%)d|Am(0Vl{) z>f`S}$`HbkJ$CJS8J5fsj)^spiI;vdVCSJP-kYo!p_i-MwIVHr`QqWEm9B5}eee3e zp$Et{RSja_axYSWSg7zbTmjPS9uL1~kXL8H+C<$?Ny11~(GlOgfeNjEn>FO5{GpUK81(9sw zlE5tqG#7`#Ct`bV=J?Kj#Xe)Fq=ig+eod*kCop~H^+k}Iajz`;P1AX`^9~79(fIXU zQz8dyW97=ouOFeo0lp{ve#qtC2*lj96IbE@-#r}h_l(kCtQP$*sLpPOb@WdAtHAOZ z3sUQw@pn(h(qFk)aK#X1B;0B6TyanyapH0GN_VMrUS5iy1ZHaxm6NkH5^@&QLS8%m)S0;#;=?IBq^-?(WK9obnHJY}x*P%D48IRuSn~|LT>1)6kL37fRZC6@LNQKywD08}{`;CM9vjAQjYJ+jUs`OIw_AAf#02EZ4!C5T_iW3GW+jBC-+sthDCrj>r(jBDR8utx? zcxDB4=FFYW$+xz1hN7-LuXhE5kh1obJCZv?SOPI{U>_Elq~dqQOY)xONZmSx=Z9tq zxu*bKtsp%{4z7vu>ULq%!Qo2_MW>A(5{*ndoAM}AD@i`54c;el_c}aG7l!S8*C*Rw zk2Jm4gcu}8`xj1J$-PiEt-7$r`k;~#cz0(WOSa*a6Oy&1*~Tli6G6@&r4{j9Q8H)( z;|G`BBtlLJX}`Fb(8+ff@{M=0&$u$`o#?#BPVo;RQ>8VbNk3tSn`r;prjcw8X$yk~3H>HS6c22l5d!|hZ zzsTsD8P|e)btwaGKdU97U~71AxC(YhNoIq526KKTUmfjs5%&_fS%{9n(Ym0aGG(?8 zWIjRksYTl5q?3B(fMyycx;7rWQCjJc?k|X^xXyLp005;dKT&w*ck$sh_?BCWB-;WX z_UDq3)Z{^Ly2$vnTof*GO%cQXk4B)()`cNTtda)y-Sw&~3Yl@1E&`YYH0&k8)$}cSLQ|42%|ZDGF1YqQVgM!5 z^6Qd4RRKPze$NI@X1H7q?&`D?mXTQz-NjXWuI}EoV!&~{jpJS@;F}qALt{ISG8uXQ zN^RKP6bJp(Y|AZdEZU^%)?`$Xp~=m@*gfi@wP90WED;;BL^w#8Mz_fyyY5HQ`Z32B zk_)S<(imL-J}D_rta1lR;Fpk%HGG5->>{j1O`1Z;f|kJ5Dp0on4olBNsQu~(2_;W8 zn@Ipqd7Zr#a5h-kUH^E;qM*$ibY)&FqE0&H6wuoPG33Ijw$Dwo6kx-v`>^F z_5RsbLqpBAtOiNNob3MROy%%(ck*YuuG1h90Zo=uV`mH2lkY^%&30tox3ep!!As1$ zJa>8t8vSms0W22p%)mk}6_uF6OwI4RPGIax)wO@wRpYZti9&NX967(+;eLDqLN_if zYLFtU{c?LGrTuEgwHSer>6{P=lWt&OYU7eo-Uj7woO?2hX{4E;^Q0?1jBf+Ez@<#V zL#j0I1#1YcbVW5K*I5z!{F|QDL`}W~$iXdFVjxZ~09^;Cmr4A+71o4&QYnVL7p(GJ zsk@;Xy0+j_0u5~Ru2|K^h_lYd^~fnQU}opx72`*LbIpuM;I_iWWCklI>FQ9OpgNCX zYiDX3R#Lo`+FU^0US#-_B!TrmctAZ z!ksGhp^`*0ORoxO4p4B5KaWxq!+IFSzJsN`S2UCp`38#WhNc8Jef(`Wp=??N9Mfb> z94EZy4c@OC90@M(>Gru>TE5>wPgHhcJ9RpA5$!C z2XvM@EiWapnWf3WQ3737INC5H6BGZ)wgZ^Od<3SN?vGPD{V77zCr?#^tYlk z#(mRDT!Kl|L{J7|pCAm2iyE}>J@4AZ@+;bg;;w6^br^OYIGCYT6oGrHw~U!xP)LQF zh(dL3%EwlF*IiZ`US>>_8%u#uI=ms<0$C@M%+$3!v!PLywDPycF%-h$?ut;qx+S7j z4zk8lJS!VnTCgWlWqiKVh(z}K&uc6O*zVN3D&^K0)5^AdJd%X}dL0Xw zD%(Q(x3!Y=?1vM>ffy~U0d|Zpq4fv}2$?AR6U%rt4la@NNrZn?6VDcax}u*VRdvnn zzviYUPutC+LQjoz@+Z(PB=;KkWTG#e0u6)apb)0y9DIl)%HZ~$iP6?0;4(GTxI4tn z`bWB5VJD#6H=+0w+Edn5X*iTf*XnC0Vbt=XxSlK+^$-TWg+ko+6;y-Ja=(lo#866! z>jcMjpUR;H1=30!7im8=17=|%p&_GOD=&Ttb{&P<{4`=BZUeGDDH&@Rh(dq=Ob`r7 z(6T+|PLe2HW~1rW#83O_M0fUOhgZpcAaLnuI3BIKb!Px!AduF+DpC%JE0)Dzx&qa+ z6j7bgDr963HhhYSmNPzwqv2+Qm5-z4U{`KC@9dpzAdm8oUa%?NKChs4%UF@A9jImM zXd5y=@)uOQr#ne3EDBv?o_0G4DW#m7XJ$z`aAO)aX&PwF0agepxlNsMcUO|Q!XllT z0m?{N6Cq@|(x`Nsn_13J;evS+vEur^|eh)jv%2f4u@K>yR7%sh;$Imq}26-(-&75xMoJiQgy{s$8^RK+xuTyGWK1>U z^en=&fhMB@4pMeu)^U(qNLV~Y@%~g%8&%cunThQ^=*uN}3DdZD^U&N2%@;S3?6zROO*+xhF;s9vB=eDiPO6mUm$2I)&0;E`M?_FiP=B#JLf{Mx z+qfmeGG1sgtfl;m4lt#^)P~vO2H3@_7``nfKCt6nm}03jzLu;@D`ZCv96_Niy9Uv_ z{P%If%a8ny>%yY>(WETJ~x~lZ@ zf^zunlt6`a$4=!WDaeVCW4ELRn}5@9K(a_e~9Q&@eFiT+$y0o5T=Q zG8V;6YB=wefPp|6KX=$qgD_lqaxv}d>h+C|T=<)NXqkh7n6@2HO0fkaZc^wUW(A@n zYcbo3E+F&hwM2U$D^Laj211@3c>-nACZ;kp;%}mSLXsMb0C6(+kDtmVEiEITp1ds& z@pKF(6Q`JWT=yAVVrRIyr!fn}zq)2al3g;~Ggl_DqE=`oiI67_n+{y}CvX{`n7JBW z(z*0vmX4$g2kj6STIRACoNf0pR7ecP+xx4mPfes0glMe@t;2@n5Ccnyt+NYlpqMws zp{h7DG8hi$Zxz8Mx_Wwmtg+7kR~#WG>&xHy?xx>@*T~Gq<{~ut=GHc(V)~-ylDIizu!YHm_(3bCVAm>fF*1dMoW zmNJ$EGy)@~>ppFIE-Cvq0-wVI%+yZVG?RI%K zhoBo6^Yh%jx@_QbrcIN89@08#7$;G3KfA9e$f!U&`pbG^{B)!=m!G=@a|zqcV(fOh z#?HP8oe?%$J62O_kq+;SyWE7D83182**PGl*Dph5yY;w!CWKxSTTOWjX5^U|IOUQOUzJO5-R=w^#vPYtor9ESKzK>hQ z2~7=`q7J(TrRZPQJJ|aHTME9!5zFq(e7?eRJxH6rcm$+EO%Q?Qk$w1Q5Gx@86chp1 z=@^`S(tGuMq8k8wt7Yab_e(xK$AcUkZJKNhcsr`5|B|O1 zNRpmQZ)jIj1iIKhIP0hKqx)gd7|;6yx>;(dKEZ&bvmA|AhFB*(%Yrw@()JPHgC-lq zLKHS2VfJA8X{qB;4td@nTM+^Qep>E0$_^5e;1wDwPx4tQZL{Wtl+6`NGn`eJ`FSf( zh}ZhZIxI?40=|-{t25Hj{Od70_--oR))tUkQP?iW>HSE-)!7`nGa&fTPhWTVX*YtX zV38jg8lK?LgMhZZ8r!xj+D@SJ1+Yf6&vWuoPur=a**i9)4TQ&~K_bwZRlid_(zga? z>&w6mv3ttecDotHu9=+hRRA5}%SZuw|k@C$Z9FaQ5Z)z>g8wnC&R&NL1R0~Q6BXeE6 zPebC+D4y}nJld*DnU=O+Q$sTlmG8Pfbkh?PzZfub|10aTh+J(V%^;<=`IAfR6(H57 zbxkOe#YxpeJX9q6$<>7>kJA~MjZ%a_CnPo$EDryJJ)qa>?}0xA%->lKyak{ST&tLAhn4d>UTCo1^Y zpQISTQ}Kr5l*L7fIeYE_kZDsPyZt%$O)N!=zDja84l!RC*xxo8 zu2)u%fMF#i8MF1B;ED!=)tYJ4^(N6t48!U62?qS~u!nRbb<|A9=O_6M4LPZ0S?WWFO}GT)WZ|E&yDGKk z7FXoVltN5}M^2HXVcI&k>TpnyxrqYfSw^ZGM4OlQV@X3HqhF6y62hlAtT@Z+tM z;Zqk=+|)5GZqq5^Ooj?$S>YYKeHZ`syJMX5`*|kH`XJF*0>-Hh8IN3<`l3DSgg6E&?Ebb9neWOZO}{ZnhW}!^!E( zorp^jc@L`5n{^z8ISDXsmt-Sz=@abyMx&uWs3A^d>hs4L?)7Ux?0l)$TN?=jIrAjA zA)AdN2FYTK_y?xbWr4>H2ujt?*H>N?p;Svz8}}lOw5_fYIj(h!aaITqif{ z^B7XYVLOl=%!I&)ddGm4BP`1LONw(x;zKut3D;>Vc1INvj^@ zHS#)?o7R+~XH~41jhr&X6hFArbl95G5J@j==2!IhPqe2Qt*G4V<*cDFSXeAR7HEna z`b=zB`GU|(YHhAHLf`QqG*C#*BiTeldz5;$P*eEpXvk4dZs6=z@NRielg8KlkXBcx z`MVqDjM?D20?G0d3^g>v%RWKgl=@m7W<-7;vH6?y?(+b9F&iK2l4cK-LKbM%=*LtNn(s%pg5y z-k3Ul>MV0qs>Jqp?#Zu18!P4@GJ{h%^C7!oNF0THo5`5BH+|}R%ovgHw zc?liE)9*GxqT%ubZyxykZ|Je4+dJ@dvQk3-49cBOcTGKPD@%rqlrH5eAuEr{3|@CjZD?rN`R|ENOW?r6izHWm1Tlr4I-+vuB~GX6Mlu(wiZ`TdYaX`vMu=NQ7#MeKx)(qW%nl zN3Zq!izSx_NE%b0%R$KBiI$?b7wJtcr?l;NpqykHfn^JUDHayOYGZ4Hjl?vOZ6I8n;)7T#NLqlR72BN2VDBy26pn9Qh zX4oT79ncY6?cF|IJwZz*EjE6sE@|Njn+8=oCw|w5rgZkgZhUZ0x&k4{6t<##ykH;m zf@wi)pO>P?>T^H>5al}a7z#NILgvwjG5h&L&ss#u3V||upeRP3h%6^z-_Ael=KPzt zdEr{R;#j9C>{ohJLlMhW*dAbjY&v1NIoOL)taoqwh>9ka9bvHI`xB~>DS_jQKOd`c zNJRy3=X17mZ)lh#*9IVq|JYt}*Tn~6s1O($%a=`o=}rYBOPt-!J|>9}++fex8bOEs zXh4?8BPSRZivcx{UDVc<7Fz;?h6)P{8>*1h*h>h=_`bvOA%7?86DE4wY`k??xTGI( zSB?^l0UU%@f(gAxA@wv@GN(Fi6-h-~G4t2}vbbFiSYD!TYaUW2ZFTbNACw9G6c`Fv zcOZ;!7anFvbakt6wgqI|qQ_|Bb%2C#*pxKwOrlzD_;#yy&O}6EYbsP^eU$b^7gU=w zl752nfLy?DzNgS6-qsgDqDv>72;fQ@68YM&7B4t)-1Tty^Q8&nh5&+XIV*|PqoG>Y zi(g)BrG;Pq1HM#1RuFT*49T9mbBA$NySj+uP=Z42_^}YedUWF7E?i7((Wn+t7lu2n zzs)E?)#3ctVCH2i%9D6VJz5n}H_oXs+TeoGRh z&BsjSQ(`{$GrTCXet??aqr*cqA)+V_6Cfz{J0QxWa&!+*^Ae2RGNW>E{@TCLR8!Ex z5JW4g2jxj2uQdqu`P!7k=6H%Nwl9VVAcwK1xGOvq2ymODK{7Hj2ZpXzo+XCpP$>Z5jAr!#?N!P{t zu7!C^cx)u%UmD!df2XuC9DJM!1F4_~AnmfVK>@WsA{e}}mz$&XfTPy-`i{`CYAOnS z7Kt*G$fh}8Ka$9DZ-~{lXB-ycD_L7-28zt~j_0CLp^+L3%FkdA9^rlK-BD1*j>ANQ zoG6l>aMnh%VoFq^>?Cx!G1yU$>>-j!qydewxtq#~AMt~blPq+TCoBZX(4Fuw<3?y8 z`h$}Q0}UHp@jk?&FjAHun-IERHv0!gh;IObVgkYX7p3_2k3Ik2T7PJb4s+tf0R$qX zQ};GPHFRQu#`^SHO`TvL%U{MH9?Xb`~M9CKiSwLLhW(08Uv_KhC zZN_9KJaub=u)u0N1b@j@2>&zd2hFf6VoP9mO6uXtA0S}`I;N*X9Po8Nky7Mg^^x2M z2HCk`YAm(nb&k`+Vz*1PuNdbFG%J1xWFH4bL4s(SWtz+Ys3~@KRMNgU z*(}$#=zrgwNt?MnVqdA)W=cr|2^!cCAoPLM)ytF1b$G?+IZ&Vtv0v5zpV|~x>M_L! z*p=Ls{_VDYh7`@cPle_jDJ8Pk`53`ysyew`_TPU?$>`xSHuti7z_8KPjaw8snpy{W z^4sVqPvF`!d<=9XfK+bZ&Jj1&=e6d~+$L4hvlveeJDI_RaEgIJky5dGiK$#jiAg%7 zf4lhPC@JQ#2?TD#5pI-_?KuEVihM*QjhJ;h8hpFx&QHp7qytx2$O@q<4v1oR^4e&y zrq$U;U=W4b6xao&Eq}^zT@k06afhB?z%;N1d@2?%Hl@V04hyt_3|Vd!cf7fm+a2a; z<=^L^GEGd0_=A=pWN~c^Vxjrp)6b@^W5ONBMP}ee8R(tinNH9-;Ix=HJaC_nC`b0+ zDUvD&aOA*2h2|wBC(0N16Vrr<9>d}4-$LSwLt%5nQlKe{rlYf{j0gUlL6n|!&7S{NB~tg_VPv8bE_vMzm24y&|;%Q zWc(+2S7eBOVKQjo*WZin+@CV^i0BcUr;k5Xwk9K6qn6aiV{7j^Jg>L8WX6_N*Itc; z-^Pe_b+JW{1nq31eY*ePlf-Tf20Weld`Aif3&jKH5$3}d($qzb5W~9 zw)x_BMDF#tZ2NZm?j^!uWLgLG4iV^nNqcm9Lh7miH6oB!1PlnXBJav`a$1=>6GvNf zpKZT?UH85Ht2>|BTkbN;G)ZbTk!sjz@J`rn+F2Om^py8e z7Os0d1e0Xwz!(QT_@9Y5@8O=qkOZYM9gRG#ZFPsjkMzR`(n3YBXNP3*T}`xXjH6cC z&9*KqE0TC8c3iZ^N$ly`d#0a?+><|zF6O|HLWS~ad3z@wk#~ms!*7*x2;X)Y8?X(}7OjpIB)QhK{ydhYyM{B2(<$!l9hrk35Eb5hZf08-jahZ@ zo7icuq_P7ML@DrVWZ+?QA_-oaLBP-5p*up~sXI072R0;x8C#`OO*I2&aLOdV8 z?65mbX9i_8OR|qR#_DI>V|3BUCK zF!efd^mRv1W9CXX-Ba4-S;4tsPk&8y4UNE+;Di79>5E-O#8wyWWFLG_--e8$@NGwc zG|#-?9R!K6#q-zoQC?5xkKxK}-$Uh#Ev=o`yjO|#7i9ak2cwIp)bPjmHxpZ5ezrtd zWx+et?uQp(4Ef>RinsEa*LQ)77qqqKN^d?9xq3nGWSEg)H!r51`zP=WPF3w?M91SVkyoZ_krxtT@uc+!a7&V|H*vt#DXBTc%iGf9DG=Cu z={i45q}a!VW2NGJxzsbEc-!$LA~HX5Q~q<^<%rL#u`MI6CfYUD-20h6&i_&juGFxT z)g~!RXsRJ@b7ZOK3DAAPYcyqPehIzab$}2!b=q1c8+CA!Wbd~x@G?QVR<_!hnKtU0 z%O#V05a@HUpC63udx*EWB5(6$<3X#tS=)Jjsz*#>Jlr7L{PGuz@7*Q-0Wn-x z#!m+^3@A98lGx$od)4G5RES9dVY~3xzyo%wk>=Ha}SrR9f7t|bpP{*w`}PLDuJE{t+$;#YVLp?Yyc`N!a=AQ zS?eq7vw_{yI=Nr0s&|kT`3qdY1NdY2)kurlY1lX<1}1og+U@nX&t$-8%^L^bhaIBRHzif5*ENMp!m~uA?-K zNUXp~=+1w0&^fc;KOiCX09DYzZsp7T+#{jm|D^>b9SsJ)wl1xY!`J95=!A!jh7$_k zNX4Q@C;jxSeR!lIcKXYnrTy#21W~%sl(BG6Tqv;~A;^W`rwcW?|0S)uFzN5zrm1#tv$UFUQVlyxMEy!Fz)5H<2i z5pKSFH+QiQme@twiwI6O<-I~_*IZM5J639O{`0uS0ZFSFIs0UDJl&Cr_h4+=k!ruzk4jL529*6V7R(o>*bm* z)rqZc-j4Xb470)6TpArXU5$GWQ}b{j<(x z!s>ik)}1GQ8f*|sK}XJyv|dj8tOU@8_q3(OZ9F6Lo5G32RN)+Nxrm?(x_rwh5Ch|q z?+jc*?!5km3V4b`uY2D|*$cS$d^sfdO<(!xnW$V<;A|~0zp3=%1R-0*3%fq~3!2EX zNMK7z@Tt=aLYDU646JJUPADDO-Yn@~*)&qg(im>W-&*gDum#yFRD7yHONWZNKQa9w$;YtK6TSZ#;D3r_SVC!x8HEJ=b|6VoEmA zM`Rs&GG98IVS*YLV~9QfuzIjZOKj?c?7x`umuQf(Y$-M47X zIXWfRk?1Z>wO(z-34ZIIm(V0+Ocg!2BL z-G%%kx%4N=o<90LO+4-y`{R=e2X5#LEhLB>gbIzcR`ZE1Z>K2oEwXGtAW37#bQ6^y zxYxQDMkLXGXB(&=KIfTl^)tR*?JdopW6$V;?YD$;R~JX_?9>MpaW(}f=96@IPQ4zq zh^>(j`hUTI&mYnDPH4Z;(j8^&bwkG*$Tk*}ct?9a-v5+vUFJw?2- zOtf=e|9355e9OsgwyQoXLl`@hPiimB)5XJ6=qqTT0erE&kYVHhX{iygyp5^KdJ8Va z*P&l~IeYPSl^{*?#(#SGh3RjZ%j;;}n(NpSb=L3Sj3PLN;nZcYva6m@sbf&~uX|vV zlePNq)O+0fTb4G)oVOY%&w^Ed*j<2O8ZDmn(WWXgA6e8*ZGJt-%{r(rG3u`8k6*^q(d#+!a!bSZ90OM17AiKGfdD*w%U8o1!ru*}m`=(#E zr`AjkemPAAL5e91qc|%*m$(#^(IJx)LJ#XgU?x=zgX2=Y{I?$C_xvVBFDH1?n)cX3 zn*;;J(b%Kayw?cht<_-en(0vHdsd&gn2{ucg-fJel$42JMa9<(!TWvcKeiYnHp=P0 z=cqL+IJ~mu?=OU;zq~$6j%DZ5%(SX8%J5y3_OB)935tYavIxm$W0vvvkIAD-EHSJ! z!7#LQ5MC3NjLMN#t$)XZ3r$Iv}y z?-tiwouxNH-aXRvy4K)%X#Ms!EIz)j`{#5u=ueU2SHs1euP*#j6PE0b{o)J5h9}GQ z!e38pjUM;N(+zQ^zU6Draz4X230)H;@#XcRky%_!6nb9|K;!)B3as>L4s)3II`hP4 zKoe_iPxh!Ry_!31aOOKzepsh4%X_4Wd-EXaxTF1Zx|tlamwObGkyYj69}LCJ%@h;P}~QD=VU|%(@d7 zCo+&2>;ZgS;eD6b%xzCT{Gna}BE=yfwFHNRK$QyX# zQ5{dj+_GY}2CCTqPE`XYWZ4zY&%coUv=EH6{BBriM$$ zY^QdR@BBn>Q8fd-&`AOCZhD<^x2yf9oGUq@-gygce7@eDg}AvxP)srme3VqweI2`F zN29%}6;xM@wIo8<2EfE$U9Rs2ONXzo?pVBjM{2egOW!n&MYt<=2Z|xWaou0iaorMC zkJ(B@4?g@xUoz~w?-012RSKPaR-;6|JRTfm;vChYAtP81Y~tfR9Oo9MoagKUFYkB=tMGl*w7&miH&6oM1 zDyJJ0OoSgePWYuZ>>Rn2qmVoD)X_1FC|d=%qR^p{e(*^! zyO@zb%v8;Yw6}`Eqz~SWqKB@Fh_2p-7(Zq6iFPiqH}KV9H2=BRbTdO+{78VDitP(J zRvNnQtaAqsgex@3^d^=WDo0QFBM)6D(ZX1;_Bws#srnD{tE)+RiW3Pi$8pi0x+OW>bW^xKldBZr6 zIpnaE=6Jn)$Mt^Z2XtrdC={9XoB|JK;8-!xvdvGjGe{Jf_BZ?8?4Lnk^8({CCA@pYL}IQunAquX zU;pxEX^XA@`xFJ8K-Sl`JL|bl+!Z-a%dxK|Z2aRX=z?P&=ZR`3ihI%Yxn4HH#0p5G zJEr31_uUy;@a{4R10=Z*f(kD&-e(cvHfIOeLbA$l@FTDAIzTJ_6x5o-9m?|=fc0t_ zM;|_!Y2S-S5Wf!d&_=084-Jv&})3 zueuB$xv@}5p0qG`b1K31I~-u8VE#&mAeS6*=c|3*J#wstyV?HTc|wd( zCp1}1ZU>?C)9^olPiPHpZxi+vlcs})+8QFt;nE;U`1f(+EyKy$Puf&Ne1|d*Eu{)c zI5>!_MUb+&Yq&VtVQS4RJB8yg{x5&S>1Ai~^|>tHuJIK?XgVVqB0GLX_k{#JWgM9> z-gmADFI!H!o#sp|PTvPOOh+$;v-7nN&14Y}5PSpdp*|9WKxF9eRC33xq?Nx^C3xgsZJ}i> zr-bz;=Wx4;vUj%A{1k32(8$JM6b$C{EyroD1~X~UUX8>!+Nx+KX#zuNBQPn(iVj)| zomVD0pI_z&$9YKV5?h}FX(eS1IL)P~`+vG*Vwdc0+iqrdX2Jh-yf`eFFuS^Vn59ezz!1lhJ{>2ZZoe{eSTWEIcp;3te2v79b+_b zgpOO=W|oK&%@3+)Od7>LfQch1t@uIi`BD#GOnk`oRBTlq^8)SZt|+x5Qn=XL0W5A8 zm)_R@Jj2c#YTlAZb2|ZCaTif83~QW2iB%oZ);K0D-%F=N!^?Q&iTdTjss}yM+6v#! zU7R(s`^^S>YN@ni-JF)r?3b@-Mq8GWr4#?~$Sw$p4%*jgyWjY(1^IQO* zv)UUeCvyDym{C|lIt`?hfZNH6+d`Eui)56h?D_?-i)%bxgK}qxvL742Du^$ZDBBF{GqivSZ!-BvYxQK1ruuF*$!njdB%x2<*dcrs@kD z*WHw`CfpE!A{F$?h6W7;g61yXV z(}8B}Rc3}mBV7HBMTU3Ms_#t}Zflh=Pd`6|98NCLX66OSvilVeOr|HT94y^<;tcDj zSB#zSrW+;eFcyP;IA?MB<(>@Kf4wK8BYcZ$$+|htQdnK+GUPA6q02p`Bz}OCQWR~l z8)I`uf1NZiR6O`sbQhZk$3ZB1;|JSa*GrUmR-kA9n1kT0GH4`LB4X6(gT zQ9(8K%Fl3TW4mru>8Q=tBc**6m7jD>OyRsa9kVqmXliiT`&mo zY*27gr_jUJ()iYmEr<8Hd@vl!!BPl;3UP=%Uz?5<8X>stS6gQ+eNs8g?}U+0&N=}e z6CuEYyv;?1hTvD)U7>M#;LEB|ovV%sY%RHno;8&<_H5La;~BUD$FY9lXsfGp=UTN8 zW$!MV#Nv`t^L`~?v?Q@6s(vIuLHzhJP$7752`@o5ywb_aI$Tkc0;jCN>sBV3^ns{O zH{i352AhZ=7SubY{$W>NfAFk<&-;0!;w-S$kx~Os(bDlN(!gpFT+^_NllFi;fo|f0c>-KM@p3Dt&%AbQvnc{?3gT0C~4JS&tOeTqoN?(GS%ZbwP z!Xu$8z5X7!;Lts6Z`$c$_esGhr2l!h^H_yDN;2jY>kftVUy_}&Ru$po>b&6fi#633E@8*jPRxz4hJP%J~ zrgjG1V3`#C0EyX>CaFHsQ%Fj(n;o!N~@#nQU?;T+OAgQDDpt= zp=-pE`T7}273Y?ieOiHW2rp3mzFEz!NdtA|Cg)5N^j8V$r%DN#|; zZGi)fYT?Pwx|-|zOo|r47i9i}pD zio%qJ^os?V#O#l9;t7dlrF_Ky%f}|!SJGaUoV24Q*lNj#{@Tvq_Vz%->lUdz@QeEC z`rVy6mCaPGafdq9K_@N%T%&uP3A3Xok49#4KrNDQy&SXdrKRu7n8r(|_`?epEOd>O zZ>jBa6!)BTI!{qnz{offDZH$l~-T5#KVnNZ8cZmvEVE(;pn~3`pm5$b{Mr#Z#pbcrHB6a z%TLKP``?U^+F;r%ZF}tKZwK0qF&a-b0J(+BuS`8-zbC_D6m)@3L{U~#1yA1+);%Tg zcQ-iGsOd&|ptCnxOZL2&X28P{&4*u|HAGXht-cOG$lVo{UFj z1fp#_$L;}vz6lsjxF#fd#3?GF5@P*<9~O5oSm5LK51S*h!QZ24XQ#B~t$EXB$eqSh zra#K*W4@{MaUo zeqF=be22bWBDJ}!yre*LuCa>Vz~*6_J$+zq;qy$D5mqUn)Y{(Y53!|_i1ONkCM%&b z!Fyk^;KTkYkfr*tdgpcYzT80ZaTHB>`OtZayt>39HWp^^PfzB?>3jdLa~bQT^MP$& zw4I9JN$*pwVW-5lyZrZ^NH`7xB-zZU&pC1qh8Qy-8tX;OF9i`X>fbqktHuBLVTV-8 zkYLU?Hl*kBy`Eq8p28WV9#4ZSw=0OG7d`j{klVclaLeubCIa$LeOc@tNJ9rD9e1-5 z5@yUZZW-#)b>}{cY|+Ujy4WC*Nzk_I!sscH-Ui_*wIT?N1`>q>jfs|yv?<*3>*Yma z^^eA1n0d8ieHplVf7OZ?X1hDq#!+4)N>OnNgn6x*yLzW%V6iGmCwzeKZ&so>a|ZpF zFh_$`xj5QbGWvWmt4QYSivw$;mVKaHCev7#{wgz0KdC5H2aR*Nra(a*8jS#3SrQUA z($k++N$U8tnDnRY@}R1$RWY#(Z7quNSc?tM`Ra1BA5AEB`s5nM^XYq3+{ivQhZ+F~ zXhFqex%&vC?6NmL-SYyYJ=B(=n3FYcbrc+ z?yd%C;3`3%E#X@KLDj5h$xeRr_8jqs%_7U|*$MgBcggS1^sSm;09)Plbt`ZvN5Sje80jtrUdk zliEG7ZYUhK&Ylcs6#A~G9A6k7S+{r7defeYLhwbP;|LNt+A@|`+k)ENP&4qik9Ei$ z#%5#9UwRcWQ}t(z1W;DO&0a?Uo(kxj#7DcmOkeDPntBq2$Fk=372~n$8d>ra0KOZ! zh?`9L3MK3r1npFpA0DPLl=65FX}{R@{wpjyeOk5a=h8VL3*~VxF@->F&gnh zY_ySkhhk2GV8AUd^EeuL(XS+_xXRlTk2{Ct>bz0NFf2~=GnOKhhI@T%6pJqn5DriptT4{3X`a&&wv zANn=ZE=tShhK<`(Z09@H^vzjK?ciIH3Wm@?=~QmF|!>|#ulpdE})%VG%@tb|{_CEjs) zABO4{-FM`5gf;e+-rUh6F`7(%MwkH&p@rAl!>0i#T__vu`I3zMB0>-D9#{6LZF0|c zm{m-Ls1|u}NLflLga}uv_pWk-qF3XA)jEfyhn41ZAa6RXe$m zN6oUs5AQ7^3C7%VfFYzpjD6%4F*0MHM7q_Y*%mpNM3it6=YBLah*6=23D@FjtN1qr zL2rbxYPmTa(+%DAe;+H;ANCUkkfz6Ez zo>HcSW*ye-`g)}CC9{9(59x4l!- z|3%zeM#cFwi-W-(g1Zk8+})i)gN5L3!QI{6-QC?axI4k!A;I0*$?v`Ieedr5f7x^P zoUJe2)AP)9KV98b)m7D1VZY`_CS}NV6!zYKVqV`#i46ea(Gy0vRoMsH5tV)~Q1ScC zBiCe%7pEigxLArMj!_Kc-H(`1b~0y+c1LSz>zX_av1jI+L7yO{EB@5hWD!Um5;QfQ zZ&9Qh=9t*snKpAIX*@*g-E3X`Ha!#dj<$dhtnn-h^p& zXDfDpLBl$4KCv}aHc(=DT-k!>(%mi%k(m2IDgXAlOAX=GL(pLCM_RVvI29>Rbl|b; z6gB#2sbwboep8(LQTK59bI9J2)$Yq4yvha7f%mJ*hHmbjCFz@$=_Yaygo-V>cxcl5Ep z(S4QX$YDK8C%mo@RYNYT`o+)DRu_EcS})4H{DTRNi;@LG!Zxy+NY$Fo;bj8sjf|pQ z!TNwOn<%Q2Fg+Q_Sl}!R>y@n&D)8<-a1{o3L03x;ms|I8pe7c$O}CJOe-ERSTiiE@H&v_1tLGo3WRgN>J=j z06F$hV>-8wV$QXH^!i0z0z{P1(6$;m)Upxie1?Srpxz6 zx~(d;HKX8{P1r*?=?!*f&4Qb%LV8hTwD!)~Vq7DEdq>QT7972z<7r)kg_CEKVT-7b z9D{T1hUnj>Gx}~JGn`ihkgGEK`^b6tpH0Yuek`|}Fn64~-_lM{w5SV={f6q~q)qNV zyJnJEyg5G&VEfIPnpxhX6Bv2@pdaBJ5{r55o=!GWd!T z_s6AxWNCaTt>$`jRl^<81|-+>T7+{?rX))`%QDd0vth~SNTgR_cb89gwrqcv>$oH1 z=#%um;gI!l0^2h+wUg^miMCN62`s|vdM}~3gVky-r>>?h9;=qL*TG;1y5X2`d-P#p zV;gyChn^Ef4d7&PwMcxy(Zk-VO3xk54yVy@2c6)!Fj_Ijq~B77yInA;6fSSqMP1hp zZakw!s*7VkS)v5DwX$q#l+5evu>g56a<;)E776oRbO^HeXub<~|JpIok&)j2J~1%bVa1 z0;Q8ufN9AeU+6E(-u;sSl(O z=-^h{bfn_fDB9xyRChXI^gfg@A1V;*2dl(q==6>#LX6zD$3=k$->vm(vg+=Haohc3 zw3O_4l|}yHNvh^9dEVZVe3d|XIT6CX{DV%r+9MW>o+Y%o=EyI7G&1WMO@?wzPfUqx z+dA0ELMRZCV(o}kUp{ob`7gyG;(YS5*$>~VVLhC#sO{rGHK%XAe$Hc)xgt^Hq~J7X zG5okzE7XnJ7l99p>aahO^B(Gc5Ky4Y_OJFB`N5kS7XS(yS#Bg1Q`O1+ChmPBj1jgX zXEtAqjnp!A=qiph@`?{Is7Jtcv!J->gaw4LB>cVx>js;#s9o41jlaU#k z&s{-RpUq>CD-};)u6^Cg8X2GF4=#|{mb2=Ud(e`s03^xyqwmiovsbP;rtW2AmA@!d71pls&JTj-}0Uu0>f7H!p`}X-;V6V_1Psb5*Du+f8dTkJ%0qRjQ+*; z;gV-F^Hsr{b)1xiC+nX4e7oG_3RY_|`fPn<6*}se#SAA+AEhE?($Lh2PNWbllMG2*4TtFl zQ8jR?#1JE=cE#fqvf-Z6!9kxt*m`AzHxJf&B$T{AmV4+xQd&yN_OJEnojbb!iC}n# zZrF}1XOAn>=KgdA0}Jcgs=(f9S#*h4>A;7s<5Sz%0^ESwI??WuaSN_mcGzDqNBF>?v03W zLTrH^4h(THPUl; z4}j}G6HWr7io}K03nDZU@3+PMg2uH&;^art+#dvh+F+Vd#T~AzlSQY+fEMp(a#T^@ zmEq?2HZ#!JCsiMAjbj!*aKxqm`!^k13WDlk{%LHO`Dp7p9$F4|vn-va$dZ#094yT8 z?Kxz03$D;~rYnO}v#~)irWONc*v$&$8yM62arl6FbQC5VTyP&(bYT7oPJ1w&-AaR~ z@PuAytrY}XcyZnzp6~(OXwfei0G!6!xLHuF5#>05d_zB$Fssx^+>9gPARKBk8=@+i z8FVcsL`t!3v>`g1T;*3BBh3B-ktFWPI?+_HgxE8ZQTX;95oLgUUDYSQ&o~U&;RB{z zf&P@jw1H>7mA^qw?_^*g^~|Dvu#gL`UK&@qUmkVl!Lq);FgmRfIS@WQub=|HWLY0! z9ucisSdraPn-T+-4A+`XiCpl?BkZ5c~RAbmn^)iw);h#ya1xSxzN z*SKBZ_g)fV+DO1?X}e!FwUjSmGilo*bq3IbeLigC}m$HDY;whg%*qQkBZ3J3;kO>il9R~vZ zhN&212O>E21tv!%@#IHgdrNm3pmOrS*Y5E`7+FsV1?tXCxIo!PR*qXk5tO zn0$l6HJ4%mwx>+O^8@f}QuY3SkvoBl+>BYsV79{r!h_*4Ui|rjT9H89S0fvz8Nf zvMi_1n07bTW5r5(kE<8L#g4^rRF_Nw5Db&yG{ZXN&1y@cS4rMjjrlwo%0C8p#_1yc zkntFEb3V~YPLydJGn+u?_fjER?cIoYYn-;~iCJ^~Y6UZle6<8i@6 znS6H8#*YaoIl6v150jE?vD=bP@bGBN4Qk7I!6G~e-PyZma9hW~+;4kuQC=!0{V7|I z!)=OkC5Lw{o7~TM2y!yG`rL)jzQ5y4)NHdn5`YhWTt0Cb9fcEjNE#TJHwsBRj>F@C z@*X>BI@r$Mys;7FlVYC!_~yO4yZhKI1qTO*{MSoD@W03n$clU9KoD{>na!Wk*C*1A z1iJ4UC_>t)!w9~W)=#@>stQE*sxkKiov`Oj)y@ou9+7_iEJu-xC;MwV|1Wq%Tq{2) zFi&94&=KE~n3aNk?IGlMPXwXlMy{x^FxKZx>bwl(dKuO-{g0d zXUQ>eV^8IRj%Z4an8US1P|EJiOPD9pP$vt^jt+zmhIvr*jkkfp!7HnObX}hx6>dz} z@@nlkh~WIj19uBnd}*bkZb16Vf5WA?QBo&X)}@I!If#{#*gh%$Gf>d)iVt?gw^_(M zc1N!M{V=G!G&UL}sq$v{J1PEwuvzHuB!ZAv)>$%1;lBSHdRnBGobcGpdN?t$u_Y_> z=IDkdIg@@bnh_8KTPnG_%LZVxB$%U0;lEwUnb!}soJ}p_@^x7f4_gET5{7(8gahKX zXWJ@B5@eDRV`I1f_W$;G`aw)UJ$lbp0ECsT)0GBlI=cO}ugc0GTGsdHj_iL7_n+z| zF*fW99UYCKnmT>*+2xj00>!lfzvFatf6>@$2Pk#+#REDv_Hc{%Un9f+f^DT8CFAEK z${9gUkO&+V`M=-tZ%a>!5#;|F1d9F+=#m(XPG|HN6pfb+O%H+!u|_Yi;h|mRw6y*u zKj$o;{1G@vIi{Jy?A0qRZDAab-)yRva}SXGk%`4OI>tS(-u*`6%^iH0^)g$+gsr`M zLy(aGv&iCcZ*Z^1^UG#FWK%l>y}(YRwK`Ky=CIaw!<)_p+^YS)#L%0^I*1r!m~XWG z$->TFbEaJ%yxu9kiFm@Ow-_QdQ-QonQ4Jc&TA_0aitfX1aUyG~o=_6jjM?#&J&-n? zqCmzMJ-^coJpzl)#MGQ#@nrI3rGpnKfjo|)J5klcpMv%q+I%ZMy)u8MzOrg#wt53r zU5hkXl^sm==}NFzOz?kCS5FFWy*|xHYF(N;A+&>B%3&+}RNb2~T^4ItOtXZ;8-!Zi z*QNw@Wz*Uke}~3W$aIYRe3ut3Ei}+lGYuLW|B}O$Z6h5OGrUwlz1S=6sr*C4VUgI* z;c{EJy9|^|JRvzb7=#9uzgXxt1S!m(FEw=BoY~g;L{IFM!E^*j$4j$e$+-IMzzH@x z{U)g(w5YkYdN@{Yu6SSH&Wr(A_qpo!zIO;)+i~=gu7`1dLK|Y3j}8Qxy(=VAbb<<1 zEe|$6UI<}mh>ofSR?N__OKG*Ia)^B}D}Yq1M!}DdXGbfV?l3M}GudX~bUC+F@6Dy@ z_uB2TM9Wi;vp&U+zr3_O#2MNLP<%zXa6kdWnO_fVc#j{Xkm>O$Ng$&0_O8WpZG#Q7 z^bSs=GH;voWBYi!dlAS ztV-bSw5N-)qCa#I#FzG`{7V4-1($;~;T%kcLoluK1G8mvqAXs}rm8v1{r=R3X-ysU z8#WgTMo+@uDz#$LHHF%zp_YFoT5nIz%BQ4)RA^h7283epEz-4euX++i&nKTa67ZK zQAC1YWds+Q%7yV@uwY@`^W}^!hM7yZ^qT4B$eJYYqU+UCO2!cvS|@J2ALSzn#RV~6 z_mRA~GOqRQWNU_c&H)w>N<{JaL{I;t})87u>3;tbP7AMZp1fXpPz+&N3x9?2^e_cLr&nC%m8B8eC3(}_Nv1C{w2 zDhhZija^D@*u?z$SjL-t=3?; zhJ5|JJ~eH9+LKZz%1)Oh5LZ6Har+}Knnt*^J~e25xZc%;I~q%TdRSN765AZSitV=IamJ5VvM=1K17JYfY&w zCWQH0m%_guY&FihDI&92vP2j3JRESnH<;2XsWT~oVq#BBbP={(Vp`t1!AJ~%&22VS z*2`ARiQFK=g*t}?I!gLgu_ki}uzRazX>}vXbB{{~c;zP*rk+`ikW2~gk<76>xfQt;;k?)E7oplX3{$ffeEEY!2YTtI-Td zzvruM7^m!H@2j6f&rFcL{&<8&TTgDw3Sjc|?zYj+*2Bc(A`z_jwgNBmInNckOVY=9 zLe>+xrsmjC1mq4z!e}hnLj=?|dNeO&_ft@!CCMVGn&|s$EF}{eu0%hl2!?i~i+#2r zHohR}j@=XN9lkl*nFe{p2HRyG2-ap^U0jft?2)k4ud;rLaImb%R3)w&O!9`P+k{2q zMJ_}&XB?~l*mESR|lwD>N?Gxxz5(kU1Y zpcIUquP>6R+uyhc;T9Az=QBgj|A>Ji>Vh_d9jCRh5d>pXj9gK*Itx(EiPHXgSmilD zD>8W&W{_J%?+kID@}c0!k-X7{o|K_ldrc&=fBZZ50~cxwa-lnp?X(J`T(MojDEiS!lZD*GX%Pl7D!Q z_csetSV_ek^kloD*c411^in_u2t(LIU!5<%m$ri?VSy1iY9G+S!Kp*j5|CkVjb3vr zuNW53eym!%DpVUj!&E%ES|MC3X`mO4zP7M>h5^97ttfV<_J@gPd~YL{T2edvCmP% ztuqFq+)iR?E0=s(^_~dosNb$&^N#{D-DgWnT#Zcf;1s?<<9cAkK8)N=B*A z-NyPaYVMkHy|RXpeHYs9el|s4MWP9d3i8GSUV5G4A&qikVrt@k6(nZ zq1^Zz>z8XV$7$F>`;DZW`g96itReh___H+B@x*+?huhDCnYZi(Zt`UCIq?`sDSaiG zOi68)nuvr>vHoNqZ^m!pm_>JqcOh=%245 zbOFp~0fQKXi)I3klm;#DX(=uEBOrIh{{k%)F|Spp(4DNJ44h?>sKJ|<*}G;>twSP) z&spLRV4^3P=;P|5AVZCsPiF7jR1hL4l>cAFJwj_N8)N+GCuBVc>%?y?r#obwBjI3N zzDUa?9C1oW^c7y^|R zOml=n+H!{#LpAbnv(S#R^QRL3n%)@PT|VSIJcK_0aByr=au5T)^s2Ams(pfuLrRaY zb|B>y%laAYYrGd&uKv=i*~wHDXmQ>V^A44UBma4x4Q>V5dBJ+q?Y)LBWz}Jte-tcYR$oJ?(y}}B`riACqe;P!sIF}JNCNi2 zp(~zZa{~=h%i*2n7^tv%vAv#gXDCl7iFtH=Hm0So*no zV5y1$6J7pA^s^%3NPL$tNqs-8Q7mNr(!t?PC%CwfVr|X0+Mm-@ut=a0Z0&G$AA)0W zf2!&Ywj2}s_WMfZv9*@}aU)4egEpQL3rGyr{UZ^Jp%1w{z1N6IZ-zpj{kJU#+uXL8 z&^CDE-zD*ynTdq4xHwyG#l~`EPrNZSukcFP&pz#-;7;Lob6t8}RBLdE@l7TZV1DP= zaW1fnGD&6g28Ni1yy@eHQUWDWVmrAU)>y6HlPW1_v}d8J|GQy2OJ zo0RobWEAs=d1e5?RHnva{Rmy$^+5N+X@pk(t^VaJxmr|r0AvRg*sr}o3=3ID6ekk9 z>T>pkwu3AY?~hX%$QqB7HeTtl(bdtH1Q&Y^J!!QelO>bHnLu^2YWzu&1hpn$-b657Jbh9^9zEyC%ndzREHrrpRe(IWQ87;p zC(}~kYtLWlQOvOJ`#Xq7^cL{A3w`Y4M$I7PXh_ z^uO9`N6g|A;%jk@*phAQS?6JqW^&m>66QIe8g~1N;0pib&LVb`K~!J?)MZrcJ(S4l zS~D5ggzIkGo;BQCB8CeOe8K@hvs|1wcLl0GBl@;ps`%i$zFf22M6IDld9bStR}^S9EJfYmFB@;oDzX`(Lo#0j!T? zXY*?uz?pDI0&CXmOJF%(6$lb4zyb{iGB_bVTP#;NWY&^&ged%Exh*8s(s^Frs@ADya5nR56>!>iPAx`Wu61$CbWD8(dlP z{s93`bt8Yb^Zx-d|Dn+TIVJsf%{KTya>{>eEB;n`a5es)WBU8qTdckG%dU=R8nN`; z$qb}MZJ^POD3AvsVAuHh_M{)hxJd)`Ggk`MzlG5MQTh?b6)xmQMD7YwoJ_n@RYKQl zRDo#EN5cQm4*q9De*VlQ`X8kK8~Oi~+laXT<8S{*)sNNa|1@S$+f?lT`%dWplJ^g= zlKHUz+r<1cISP^mvwBQ8kKAT||MwBawM*xJ_w1eaFX{>d4gvAMyMHu}|KE-Oy#N2- zFaH1K>_+^rx%<1eiuz9OZgw`|o7H~p-BZ7!;Wo0Iky&bG5QT}C>?jrfbEKmxbj`x> zt_zG2)b887rle!$M(l7#kjd0sVDiwm9i=_G!lQ+Bxm5Vhql~B4 zPrc#rYOwdmZEx5uF?)2cb^@~n1%?9o%eRT0sF-g`F#(O69@&Km3t|L{ZKC@^d@wc5l6LC3ua;oMPy~-4K*C$0k=tvGr z;U6j4V%O}H`AqDFOX?Qc&@w>4XSXXYdOnlpR8E%k5(_Dv>rqCYOJ;7smd6y&F}it- ztyv1L`qNjFbFSM%5blD{e)3j>&=klR%Pi7WS}FXzQ7yP<(;EDX2^|kul;_BaU^1=u zG=+8>ch=^<{@vYFEO$xDGopOEoTtY>ZT|bEINZ<(8+0_$u2 zpW}yN$zxpo?Uzz-lmpGf%lEmj)-TF}{@f|QEe<1MbWMQ9WG%8da)m+T;lCejUGHQM zt)dKfZQS*b*f47vGN(7)!HTAHaN~@3QYJ2}_j`&dWOG^)$&K&6n7|NvL>WBBS`Vy7 zeW_+Ce&18S?IPZOrbU~%-hfZ(8rY|YKJWUY27tOQ_{tx3x)~pXktX48@P#D>KWUX5 z-}SA_*b&zSn;)HyVErCSUXx|TJkIRz!sqH$Sn!*i$mGbflKefxJBP!3_08Q>LT^So zQ)W=zEtJBcxB3yvWQL z+EMAoaAAs${zf;Cwfif5a1kz-XwlAi%;TNI4Xj+oh|T*~A(HU%)sd zm6OQ~d;dw+Jbc2pL({zHGVWI>hvMnoi2fwkTm4&N4;az*;?HKW+>;leT#8hTEaN=c zR4_yKeRY${ZkIvgnR&Z6?w9f7=r)2v#gii|8pt)fWht9QxqU$s)I6e$*`5!eNCSN- z^EvxzPcMpb+`tN|mqV&2(HyhO4(jz=IAv4%1KWh;#Cy>+F8xo;U_1Z@;_@x=ljx+0 z?1bTJ7VKHVXBLwICYFNA5j0apu5K6Hnr^Nj{w5x_`T;l|i!WC0Q3kJ<=YeTWQd-ZR z^s1I0mRJ`I>rxSFn!tfEpNqgq zk0ZK;pl1NebrWLR<0pM2%_rNb47O4i`=uV)-wzB&ZA*uGlWoDeF~CCi2YZvQTgawh z)7Gkjs4fJZy_2Bf*5~IJsYK}~%f$-5Gx+VRurp>EPnn(e2RFA>{r2dtB#VIZ(F}ns zPP~nE&N^a!iL?Y2#-fbks~S*>R@Qv<@G%!<{=~DDp3{dd{mn6wxeL5W&Nd6=>)u}{ zmi%8uB#Bd?e3Wr>x|!3&G4-{4mWbhQFzQ^-9p9hBb4-Z{qnq4Wr$0{UJ^3Z=r%Rc_ z7bg!6es&GLDSV_y_v$!Lc&8sN{lVqECt?Zcs?2_1NXV|*t=`&{Z*EFfXMB&!iuPN7 z?P>aDrP(K0BiHA0M2={Q?jP*n_6LQUCw4iLODK>3dPUK}k7bnwfkIm%_ z?Al~fub(4{NT53;ykiH)gM&`*dVghv+GX!3LnZnGlyF7)sNl>Q!8>U@)W4(KKkJWx z@m!$-?C;x8-tvlXMCCHhb&iW`oek}QSNtm8vz~IEPqPtxUA2i*YPTk*z9fn1{gs-` zNhQw2yM9};xlO7H-(~rDGQh$vj;zW`>W6&Qb|mMOG$-aT8$$7~J?7d7fM$r_@(m4Y zahqJzY1FTtR3_BkiC6v#BCneB0y^(E^K6ay!!<Azosv@M zs>Rl9{HT-p1e49@5$SEdKlbtBhTy{==;MW3bR%U=92c$CoyJ)r(0lbP*=A^lWul%} zcFlw@=#ewZs`mM+!Y`U_6|BBN)6BufTTQsOB+;HQ7>N$MWL|U3TW_K^;s8 zx_Nx7mI`L&?%k4L&#D&zN>qI`yE5%=$(a%UV* zNocX;cTe1reA)_NIFM2}Z1^)w< zQ?91)FX|eeX{<%IVJ=_Zel0Fui{eH}Xn6gi4mDI z4&uhA=37jv<-*TZ^3g-OV5}f1kfA~H{F$Cb%yNaU~sXj z<2Y_JLWt1x%E8k|U@g55*XQWFy{0-TD_Ya7%Rc?quxeP=EJXR%;Nn4!Vm>rF#^KJ( zV>+$+%?_)qZq4U70&$A1MWyDbG>4n8(`7K?6l?`i+;LNZaTraufkN@Q0o(U~ zB#rOxa)4_vdy+4YN#E_VT&@ixU$8#4TX8&_i);OyRuv%t@KJyZsxHG}Z<$%q8f%7# z3dFbg{&Nc`(bDZW319SM8alaZO}GnBs3KAe`c`WAi(Pz!0Ws~8W#O?agg{kPWVG5` zl@&N;R0d&=j~GyjD!<&hwk8A<`jtd{6WiYiT6Tkf9x=D;aDfe#ULj zOrSoa?FD7e(p=PQ5VbIrD6+%k(`YXb8ok{9$D853Dvvq3^^T`Kk4sO37n0;}-gsPq zgMUeR?Bwv-zEOJFHll?m@AKV7RFHCdgF&gsEu5_G{Ltm zOAf;G@k>Pfl!!k1oV`e*0qufkbgSS;k}n=t%FRU8WE5O@VK%U#xuOr7T5;Rk7(sxh zRN&J~ne#)B$zl9qIbYB3Go)LB2mD~i2q^XlDIe~b!>ZguUHC9)$f@O~OYY(c!RQ6l z;BH_2h7doMDqzWwWC5Tol@4*FK?D{%DE))x@m~3i z7Mm!oJvmRj-mm#}S6_PLwbJd9rR}S$pZT+UdrB1lC@K?X;~geN`PjMC2{ zR~0i#+9P$8v|-Hp#{eNiN)C3o$v75iFP=H~t)90QL<%~gaUj|kol9C8xdjRgHJ}k? zkZpJ%e*yhyH5qae`xX79_$hy7Y&=$sIy3+&`WVCCOFo?#oc3$bRzR! zEh%ktT%hRrUP}uSIrFapPSrEb!S)S!HKb$FLg4fP8A&-|i=ZoKmohd;PCcvWmzVlbXhVVm;x~N5Ro+-Gaq+7}%(KeVEB;J#N zKnqL?N4%Yj;DB70@)MRC6s_PZqT2J(AfxGc);s6FYK>UdG! z&-I@RgZx|K=*O{ZyGo&h+&N@3#`dXg8h+;{d}<3jxcE8nl4D&w)z6K0KY7&TlCtw_ z4Q^)q7gp6#+ZO-un7=f})PkcYBkQJTT2c`GGI->v?iIsy7iUhvoxm*?FLVI1j2;tZ zfUNcvgFyBx0V5-~?@F=god@^p(;DB^96|ImM6KfFSlRUDG&ir=H@|jmS+{F9fi>lr zL84HVE2sn=*SI##=-pw_-trY$H)Hr4&SZmMZt5IoZrLBhf^H78CNt*TttQZnQ(7WH zYrywTH%o@{y}-kQwbYw?wq$FDo)ah+ZQP1v`Xk${jB>AW*MlZC_0}qI2zdG^N%byI z?jTipM@krp0OSUGoCw8`n!LoP3%=n*H{RGP4nB&EwCjl*x=l5tY-nfe$3!22H+WvW z#_vru<$sUH+bLhTqA|)>5`X4U60;F+2n(xzQoPc~;ZCWnLq#>%pNZa3VSlz=NKv#c zP2WD2LWHIV%R~Lb9i618=d|ELExd`CL466J|4>Fi=)^|Gw^J!lM0fCl*-v|!Tj znX$SPc4v^A-T&_ z9dJ7hf7tQm+~Clw)qf`X&5Rg3aDvlSo1n&$WPM-%-Qr_-<3V8JWMZnX3HeAV8XFV{I)oM zdIhG@UHo{}ninM6Z25+*wGAyPP`ons zF}V|AH4(hi^nkzXAt#xi$kRkmHc%FcegP|&7`{gizbnw z=ff8^G!WhYIUSQGgbewDJdM+SH`SB_-m1%tI83 z2t=u2*y{~svkgOGnT+NoFfhbLzpsC3Zmcv`{|O?d>h|i9_=1XaYGM`NXc+vfw179}k7(4uRms445l#EU#>qS$khV)=zYhbO-~eVeU=3nwc1d0 z#6qWz@KeSm!*2V}UJW8&G;SCwkfH6yy;|aktRam*Ssn3B%dn=*AI)mzLdl%l615t_ znq&n6fPcj-G^j%6ooFyw?;Cwe*{0lPwm9xX1;i7KpkxPgM=W=gpJ9DVTZh?U^RXDj zq`0DOE<%#vjvBJKUN~;Xq{`V{8EM{dDGY(V=;!3L>wPY>nuf=z82#mw9;H_3!5eDR z+5AajLwBUDo&u)4EFZvX%_lyV+;VS?!cXqeZ6Wt|!cwmUMiaeth0qS+zYD6(=o@+2+V>-6Qp%Y4 zn;q)eQCP>%IB;Li?I$ox2J7u&0BeS=6Gs?pgrNZ4c}Da;T`w|`8n|Dt+A>J3xTe<^ zp(7?J`MX~(XJUY)`)>@Uf?#z zG$=64jnRY9je}+SZbodCH&;5i&pgktDjz5+qhp88%cn8d7a+xc%w*OD2!VTNDLm1=2X9t1amA#o2g+|+@|gz(;Q zv0M=Ya5Sc+;W^|Pf(K_POX%|tYKW7ybDavpaYD(UZ-F#h#a|wu!UvWBQ(W~{s1`9$ z%$}Z{{<19C;_){*sLE3gJv(A6WfatrQ7*Z z@dHmPXWG-d{P1c~XOdFB(b5bG@c=t-wD^G*sdzChCH(8d>20|qZw)Gn*mQwZI7L#N zMyK2 znp<*!BjqU$9_E99Ldw#?kuS^Ztn=%VjAq_0|MO^tj~UYt zR=)ZpS{1{twO1^n3aSNV1{YRZ3NS!NT875cm3(=rClX*Oh4vl%>^p=Pi8)YijO3>t zBRf5HA_i6huO1UUW)p#peLlY{>`(46+mJLK#P1NhaP{V~fFuVg5->DfMNAsx)8gF9 zxF^;v{i5guycnW9>zrUR1qP}C3CMzl5vjj!wLewlLQ-xPJuIYJpHa`xaRP2;I1arz z0%8cj)|Y=Gfw;<{1yA34ONk;jzwg#&wv3PiSj3Un)TzQZ+lwf#QuPni29Cvf(>|Lv z7H;2w46L{oJ*+hJSPzsEmF2VIS{lOgChhN#6Jddrp)T z0QdFPdhFhE|K@LqdEkszED2ViiZ30D9I?pnqVxFEKzN>L}UR9Hlx(yj7CjK(QJr};_}hm+$2u8yv3)MjGUD4{PM>) z_$hvRu6g>~Unix9WT};W3;c;M-`K9KKn8)b$hA5M&aG)9l!L2g;ZDa$p9C{P)QM6y zeqHP3ezqFopYRda2n8{Dc`NLqQL_w z$IR`U4n-9Nj)n=gz_|_4oJpAp(vz5%W@7@tB!moG*^oiP4CDzMgj4?ZGFT) z8REXdGTBNU87($byY^`c1Iv$+dqr8O@094$*xI;MlUdxgorMX+Q&lAtK^ z);*WR+MAF?z;E4=x(l2dKVfy}mzRdagbW6*w?X3W;r6I>Bd5rcsu98S^Wzma^n>0F)Gp10+9v1Cr zzHi9wqSiSh`}dG^vKa>681h%8l9GEHYwNeSGb1STNAY;0IS`f7_S0N$cc4WmVLT-F z!r-s&_OpXnLoH2d%^Wp>v2sT&kaNMT>4@9i$Vla|qRgp`-aQAa;tB!y0dBN^WFR@V zl|x82SL@B5SKwrvZHrhcTH*S9oLO!qPY1K-)SGQe1{!G(o^i$i;w}od?c6S*#o`kZ zL<_~I&8-|l&=PiqelX~7-kMj^mP*9;;9y956`&6u^<5kRfDkOEai79K?gd70v4B41 zc>O=#V=?D@J$_%RJ6rg9O(u^ar2Q<}-8r`AUn4ULhIcF~q2{V!04Qg@C)<|7-8ssv2<{@8a7}79#>&<(d zqqVV^(w?1JCg0Vn$^FB0vE#VJDdB~Y*v~0Qa|$$rvqlX5-gTrP#4-52P5}Lm9d4@T zVM9$NR)wjc)8s5bY^8KZE4kFM-T6;+kn*o}P}-;I>)&HlY$&6BG{9Fn3>5JULE|wG zTDW7TWZvOBz{H5SQrTupwp9}|-!eNi{ZUC%;EYobdnUjxo`NGtHef7zmuCp&WGEFw zXcuEtL}E=go%1YrYy~f>ICE7(u%ztK?$7OU0G(NzmKl?o=P!1S9f(FxaYef)jLL85 zN5;sc4ceydR9swDh7wFMA19*C#CTE~W{7wU+IUsaRSj?@OT$Z^-s|;&3kD#!qLM=K zf?0BzOz{Y2Y5y)Q-$3DHt_;994n2rKXe9bT_VwYUj-uqp8AaA7H=MBTjKss)IsjuS zK=CMLtx-w3eD^KU1mVg*-7I_UF{cWjWD3GK~ zhh|*b3=V;GwNm(MvB6{3kcdIzNijC!tG^<)f{C0+BngZWmnbT$&css}bvDDPZ)UD2WevdfIpJCyMFD*XfRs6*53Me{%w7ATDVn$ZyUKd|W32>6CI4S{uu} z(<3;q#|m4hW7^VRYzT$4a#7X6Z^6VQp_xkxF=|$cvLL@iVGvSNd}+hvpm)WCPkVZ8 zu$*@R*$S#Ds5oHNcMC(8?^{za?JE^Y2WSf1wG-76(LcvU0DqYHcnd&fqG7s2c{TKx zPSzA|9#Hc2K>k1&ghA0a;}2a-Ghe^@iXs1ARqdPKZ_UziPeQdZbtN8jIx$@mhrYhg z(XI#5w4FlNc+L@edalOYUYG1OZpvc3 z!qV<6PeDcIJ06o`2#WvUc1Q%h;l{gRa3&QWIAumo+C^_=fL&YGOjI)$PE5u@p0NY+ z125Q{e*Lqd-uTpiRtRGe_h=oyrJz?q08QzVJ6&7%i(q96-)2J3&KvuDY{z3HwGs<) z60dL7xkT6VGZi(_??0KxRXolo!2}~ZXIhPjpv?8nLas*N@Rp)D%9voGHaw8W z`UjlA`lPjy1ql{Xi4h;>-x<|Z*0w!5qs~ZE5dmp-iWKP}Z4?lt z4njaWh;#{31B5b)fQW#FB7}~#(4_!`Y6p7YSK0i@43kHB~aj z%;(-*_hgu76`zx2nx#=|vgZEuPOj(MUP$rH>92fV;@P|Y5NSE#b8Pb!B{`GA?_A9- z4ww8qtCd?Ux_HZPF*V1< z;yO-=j1->ZglhwY|K7Re*;Zls?%nUhR~B*|D%v8cZreArPw)aiBJ3a)wj^gITJ4(P z$!A0#+&{N^?O71VqRGYmhA9CDRdtcKXn#xb+er`wl8a%(2YUTaV@AwQ%MAnPpv`0es-xlH1krL!dM zP5)U<@B6z?)2e@x-*un9hPbjbj4jJxr*Cx3ZQu2sd;yE~g-U#MI2r)IC7Z`3N>tUbxcr@o*5MiO`c@%)23J1P4c zm%gRD+zqyrGhC1{+X58rHeX`Ds=6v)ZNZzX@|WCdf6@8+ey&!HB5W?R7b_QBcRD-l zP$rrC_)ObhXdX&F^(W;{Zd}fp{2cr~N3~}$U!c<8D4V*e zU`BGk68c1;)%$@eYF@X0SsbH(P$>J&YWIXA644pz$C*{?H_R|6#KgY7Bq{c;0>qOF zGuLvS06#(VTq1bW)W2yDzZN5rmpZOE}NHbu;Sm?EVZwJq)$8=^pLa8l=~&J<7{#N_l)qquWCg%hb})~Jd8w2x$GT-kF)f^)kQktdrtp?|WSZ zZcVpQuS>x7@7Se}?Keq?r2X{thQR%>YYEZoOQ2E2FTRqgn_ns4uStF%nPQ0%z`*{f zPH_ajyB_3z){*JM?c*Q;hKjd3t67d2P?-j$GY^k79M~^s-+J#Ete&8rt;cZi)@7JFP3`P=WRND5} zPy+D4bU(}fYORjWn*rR`gJ_X*{ebMzl*%VKcKu?(-XebAi%p$ z7PqW9?U5yZ{9vrqrY>{2{@VK&nw9y5Rc?$c&Kmc^5cCMK09wa++}JA2PP zJQs2DOu3u=j0;(WeWSaNkT+OjEGKE9HCLcTD~MhYpXn}bdOZE6B~?j>)g3jiN|{2J zJ`mLQfF-mg37I_Yaekt!bS{M@^-aIpBRNOu5(V^-#GQe;u~_BOm)<72BIi~X+;W?0 zUfr#Y=R6>uJZ-jY>>BU#%v!_}_z>JT(`j^Q?`vwwgDT3UFE-sPS*6!%ddfcia{1Ec z2RQB#`E1&%ZeU`PGxXv~FM((qb;;g7+msZ+0EwInYOVLw!YW{zr!|U$6QVJ3eVIR^B6TT!TdQ>vgun=rrd9L!F-VR+D|aXbg;o{kPeEcotONImvmW|WE0oAOybR5Mc{c?uD-#)s3{e%Sm~nL z{T`yKcdC=~>99}3(2w6_m~16iJsp=3TlX)`B%idB3z2a7{`0Kzr3gXI@|>Lr)wz%` zSs6*zvjV&$D2a<+YIdcJH=$_!v;Oc8D5+#+iBf!-YJipEZJSVu$Po?uZC_~Rx3Jk` zzv8qlk*;%-Y)s#ctK}EIHG!`*zZaJy+V$$- z*$0b8$qOY$+qGY=l{c?B(-03__9ruYAE{;l@4oyoDV+JVUS*>aqF`t>p7Q~F1xuSGVPQqc|$$- zXNiwx?76;{Cadrcf-&_D>+?@HW;U1aAV!tsR$sj_>ck(USI)9)@its4yZqx)* zq*?g47Z6+fn5j2dDb6uZ)~hNyTMRj^8nEUtENCOXB^8?Imp)J>;k%C>%6I(<1Tr2v zmpYv=bE;M|8F)En;&T*h7b&K7iV>LG7vJI{2-FVDJUqer0oB4UN%ZJ$=g9ZWIz zxUXwlM(<$NX(2&(`Wsg}jn(ujWn4{MI)7b14@429Xk}pBI_$ z!$LDV$toy?3=L~auX5Ir_F9>2K}43 zA6`5Y74Rl^f_XA||IH2?%Qb6Z-}Ih@^A|6|0b$rAn}Cw4f%nDT;BX%W5J-6@S37of zwn0`Ss-WNouvNDLEbh?j^@qjK=wdRJTJYstpk?IbTaZxP7+0b|MSG$2xpQZ=?`QK` zY&~n*xrAO14f)VI^!huG%Ej}R>tdp!D)Fb(>IOwDBOkrH)^7xIbt}p(Z8}$I*C}|> zAh3un<7){}l^25#LZmt{_a8g}w%+~qX|M!#(QDat!ML}oc#P4Zn8*XsOw=COQsuvHOUB=7TFOy?8fD`GD$eMlyKhnU!SCvK-X%iZ8RYi#^W zs{j3v2*kT~?b=C4$6tqgtY0B1BKf;F9xn;3?%&{Y@l+n`vuwNCKIrkgx8moZepYv1 z9cdBemwGmO{-U_P3y&QMuF5dj;e=rNpU~j9yR9u*d+Dj|<1{ z4EDST^T&FWSy{!_)s5F-$JWhjbH~h#zdWkCKa`vOPX74mb9EozR$lyUbVIgF-`TF8 z9zxD?@Nqu=QS0a5wvi+WA>`C(L3OE(s#C5DN23egvu>^#og*l2QQIntH%rs7X{nE+ z5N5$WHZz1N$lxQBW(y_5bH7G2Pb-PU+M{AW7bI0jrH^jWooQq&U6_i$;6P zhOBld#;2#Oz$}B9h$HjvA<3OE3^FjUT2WlyelJ7FccrD)P{px<0Xc{>!ql46YsAPG zBoP6O;F~GEiiuK9>h?1Rh)ExkZQch7%c}8Z13rn5+irOoi16V$YT<3@?s0V7DF~7= z?&&<9`Is>rv7RO{KHq3Y;X5A<5>=hRWC0E#bwKpa$v$fpsovsGjtD7ZkR(^rAWfUwc{rxUL z;bi^eDzSxX7$!6Y)e!18-GJ}o7W||_Y8vUWrO50dTyd5j$r2BjF3%vm5Kz)F;puws zyja_98(ZoYf#}t@NHsIenJJ|3`?x+8rmY+FG_5!ypcpo*kf>B%m(oYBK|(%#b)iI( zVhiQ5xhK_PX3q=qzWDmMl%>nm==@+#Mjf%C$5t3RW!}>JI_Rcqx+HBEGF`WqJ+Pe8 zvU^i)#_Vu=0K8e?8Bx)^{}8obny((xF5g0jx3_G?BX(o~G&EQbQU?+ebhc+C}__Tb3dMMT%oCMvZ0CC5-Bx zo)KgOlO*Q$w^5bU>jRK3(VmxQ?*4i?q-q*6Q)|0hZl$(YlC@9@VtIL{ZKWpzGPmX= z1x<~1U+RpL4DNyi;m`YPBJQI?;b6RIjpJ!gL*;@#QDKiP3_Mb{{P1}poqqe-{GZLxJQaa|4+py;BTZP@g>dfjk?D3#(>cYsO#ijhuw*IVd?EHB zU7zrTQ0*b?;XCc(?mL|x@R^c2&5d#c^0y3*r$Za$+qmBqbHP-@B9DB3Ddpod6*VHx z2^2O4uQiFzP-yQf{pMrwENbmea8#l zLh%ezj-`^CQ^x_9CA28&qUx+|2=g9_)&kp5Q7xI#xDOIf&)i-l1G}BVYdt3F&a_>v zKxPyn1tu_tqZmU!T#|X?!FG=te*kHYu({9<2=tAAwH?w*P?N*UPhb}qkPSckfE|vZ zgW^_BMe#iYZg5SYbN7Fuqu?KY{bhP!%_BKk98wT91yKvD-h9RTuX>M;9QrT)XM!(K zyp8)?j;GKT(WAhcR_ZI>*@6UbQWQhKs1dztrNS4gGa8;wKcH1Y!HsiJb-xJY9*3Ey zI`)H<+CX>YNc`=l9-X;8^Kul`@RWP8tW-;HidsP4+IDJi4Zc4CiXEGn{+wd4V!Ixb zpfd7aRHsjH64!~JTS+r|>K(}RSy3m-aA#JHH?^iDs@Kd2tM5yPc3TKm2SqMyf`w4D zijY;WI;B;uk@0kjl2KSUHQ7j|YlA7EPJ52>VMxpfq}#5)5LO@4b}5y!S+zhCn}*~k ztnuT>J&SsZxKZHZlBaF$;JhykgBqz-4EZ`dB1qa8W8oj6%u#eiI6ecI5Gl1|(l_fR!8x!J!RLm$+ldOsV;C`S=1Ft`y~B|$N)4NX*^u^6C=xugYT zseh+iDoQ7d( z0gB-RD9U2bIB)vKd_2B+8^uiHTk#6$OlzN`E<^Ue#fE=DsIDB3rDfyic21+V(Tb14 z$4{6EFtbbMn8&0JGD(#5SR-kh7CArqDBYHn629eb#LN*27+>a!@V*p+s!=5plxch${iA=-2Ctb_HaI|P-9tN7OJrdM}&SMqgJ5J^S{CPXK4=R z2g4QgucP-8rfS-f_nI*?bBCa?RV1oa*@K%_O3U`Z?)YfXmkG=*4K?&s33oQ3X$xb! z3#-_94xK=LpRHA=Fd;!JyHeYnSO%+}?%!;bDxHD`Oa^q6sG5=olt$V{GMB#iS8V^$ zr$Rqb-r$hR5xS}=2#~AVZ2YbL`F&J(uasgyt!|!=9VX zySzr^-YpyEMAqzbkj9pm>dFQ#YAc&>Zqqkx3lfH15~OSPTP^=mtsacyDo+h-%nDhH zO|SbCHpg&BQBGC(_fa*7pZ!%1IzS|UO7&qw_;y>Y#58Iwt7Sc<3Fm1J-By;WabK7h zYMg3@1RnC);Z`d4S!M#j&r*$Gi0%)zI|7-T2NpVU(X$k8^Zf)BL$UyN>#2uJ`uBM> zb{*t}!SRKyyQtDPwAVKUNnyJKK?J^Xzpg2)1KlcYqIUP125rD@qtwZSGE14TWn#m7 z_~xK7LbNoNK32m)3T9Vvj<#;M_Zng6LI2D$yE2F=Z`n2qDau+rWTd7%LOkir@toKn zJ3R1U4qUfeR*|%U|E(l}=ZiXe^d-`b1};JfGJ>k#TjA7_vC-d!WQeE~K{XzEeT$kW zF{wFks!w5%9u;QzT9dPZsc95BL}!AN8&v;O?$&mfmf-0VU&GJxoBzjk*; z10U3KAYCG!t{55Ae|v?$JY!|7ySX&lM&SR8Zo|1R{u5R{-&IZ?m!bYMTNP{pI~3R&8#?;gzDfV$sr(>WCZK7`?C{p@Rn)1hf6WTDo3 z%Zfp0;bbaWW7`p2zivRR4oC1I2;`4a$kC`Iy*}1ntY2}V8rZL6recdNMSV=>?~zQ< zt;DEmTP1vSTXyebhK<)D($-~0sJT6GzVcMfjx~!^c(Fsh9vfMpo-!fO#K;qTlf-R9 zgtgGRI;+Z71#B76@cCpF5?31*3=AGwIwmaU!PANizQb*(-S=58mJrEP;Lrq(zyY%q zHM+8)g|y@1AjK$<;KejQ+|#t`weyCrgs&h1*BvJ=@(-QDP;&NYo6p($)%;^>SJXx{ z7?c%D6|ZjNYL%wL4AMxUYHO!cJ=tA@9*bJUC<@qP4QB!p`qiA`mX`bM+;P3CRtTb; zVZ}CR;J}?;9#(cpA){8gZ7F)!$4@J70zNie=2O1cRVStvwg3TQc>=}SThs1Sp3W# zit-Ak@h-z$_hz@eg<5Je8r|ZG+mX$O1GYGG+dav>4IybPuTf}=5Vcr{Jj*h!(luIh zUG^#FbDvUOo--+cF3h3ihYz6tJwykd|9H4Bh1gkSqcF#NFys`76>V(bb0GB`N9Rte zl(d>ZGD7IUj-6m_&veXk?2aJU7pnDqE}MXo@pEqU9p?e)cBI*_3(xTL$B3lSy3)eEEc5*-F8jS@Sl(n+A5ZS zTKkP_Cle;u8~KP;{%Bi_$l<)i+$PV;{2)a@J;laKW1)f>4AVm`bd5`<2h4U$(aliL zXvYdmy+3PY6$us9U38WU`GwEyEw3sAmvsEKCH1U&nr_8@ znMTd(8g>P37gBrzryfmg?f_LF+dQyA6&3WZEK2Q)r2ptak|4BSSEsN4Q8xc=rC3qU zIKfoGf{kWeA9I2)Ag~KX=g15n(jAkcfV_h854VvO-b2b78;iBEp7Fe6aR+V&6o zo)y+Co5FU(tW*PYnxpO3fW^Q;fCgg04;9pbZ(eWr2&J;vEnerdC5DH&TK+SLudLiZ zq72!pryVfR+WZ3423>aAMvXLHEZwpE^m1pZX`sPV0hBOgI=rq>SnE!Uvl z;Mee2;O=?zRa!0Xs*vAEZfxkli5^B7J8|b7Dx?{^-@F&xOAi00mkCY9v2=Scl^ap> z%$sMeG-^jRXjMYHZ(_%(rOQ%Pes`$flL_gl{;vy~&<)jRlWAi;agY{ld{Utbel)53 ztUR`Hq`z`|P6H8oc==8OJ#=R}!7_YqPsItR8V+MXLPv$Hw`S%L{twf zP<4?Vzx9*aK)NQaj`D?lRb4-IWio}Fq%9gWy z-&J3r&2}=>nJMAlQHT9>8;YmK_Ziz~4Ye#ip%g5!55>@EW+of7kfTupvQp|DO|349 zULW)_D~E*-*DYt91#0mW{A`m)&WMqX;}*?7nONUZ2uA9tnC3ajl9Tg z#t3WN#_y2(Yo1Q81guJW?e`rnsHqI>lpSu+LvRfx>8>kSGYT;Sx- z^!lxNHFx)%&$b2?3!Cgq&4MozCL1bxVVTm7A0lNl@Xf|hhnX5Q=O^<}&d~H|XK$)WmW}yZp?r?FXdG&?1;Ngq#CqqUy3*2@Av4!dC zjq7ow%7bF;+OTYj)$G7#TVYd|M>t_+mbOzk{cPXa3rYv}oSNIa4IO*jTF-Ds2-<#+ z!A}66W1%C%`cQ#=ePR1prVGBjxs8id-&&lKu^@%rM9!JSTQ*U>GwD0AO>_`yJrp73 zJ2j4PB0|8=X5CW4fsf+bHy}{U+y@I7qfJ@tq5rr<>b5dlYzAb?d~&}WOYuc1>~EWJ z1nyt7Y#1;hCXv4cDAzOcmdjcO zM?1Lr%*tiw$inPFw*rFcaaIiAMTN=z^hgD9-q6Jg4V>9!)wf4MKOqy|9+bK5Av3(jmcKj5FXKiphry5uY;x)Bc@8I0x%OOy?Hz>2BvG5>Z z*U$<)vsRzfelA)^{Lzv(x2-+qy(nj_5wKwslS8j*AB1%J=zpi_t}KhXFU&Z{H7 zcR*qo88)oP%^fgHFje+(70kJm;Azf*I={}8L(kL|a!pE_tJVrpkoY$(@Gp^=y%B)!x&xB!3|yHwx{Jty`3gJ8P38xpHk4@KlSOHxBF^iA?u^J=XxZQozCX9*`WJf|s0sET{ zJ;@Tl+QQ$X62yCR8HPDn(pb((3-3%-V9xt)&<789e#~5K!`o$qXoJzJrr3i%>_%>^ zHlj2uvqv1 zX%rU1;QySm14yvrhj+Jo0!t(5_U`Vw5MWaNH#`<%H9@JFnUrT~7w-Nl{4q6ET3-J8 zKd%CdbnGT3CKtpX%`eDFygXxgOJ3gTF}5X?Lc)6S>tKMSf8tIj0WQNDk($cO!YwH$ zE4%cM0)K#_9B#?VIzA2vF#9(EC8)RG`k0xi0940wlA{oi%x4-v3edc)?A3p-6an9U z1_MmX(gywZZ&vG#{4ZSpPiu((OIiSszW-06aP(B*UkJVYQYYpPFhY%C{z(wPvV{x{7KV!fYiJQfioGA=Uu**wvR$Vc zKuX)yPsqO~wMQid9gdpNgdkD>ttvj?S`dcNy80hh4pgrJ`2x?c|3|9M?`7e=j>y-7 zR(WvKB;vgsP?GmkkZDp(fR;)wD2b?G3~DU)8C6oSWQ*|8PZe=YxcKu$P(ISYPd1S* zcRKEF9{l|DSxkJ17~vHdNCzcVS%F0T5Hav!jHxo*YRop-$kCnlThL`Rq<)w+~oS)W$z*fQSx0K35EfqoC zT5I4Mq8aE0ynC_?zN#Q?>gI>|$GH|n17@lKh#{Sl7|BiK6F4yU>%Tje4Yc3 zz8_FpdPym*NKBr!)7#HF!IP&uU8Yb?De~T$pn+);U?|ZfA8iaOK5_vK_|FeYT7v5; zV)A0(x!qdI&Km_*ptU3C4CK09ZXX_yB`iK`d1$f$w6kzUff>8W>Y14T`wOh4IWaE)t8ZvO-)<6N`McL|=>P_#+&3 zL2KTh_>IqN0v&b{S9DLg0Dcs98I|~gs5^N;N5lY|u-t&_j@k3;p#2k~95#Rd@mwgq z6lq|}1!}ZP9RUmg97D^)brbU7K3QUo{y+sjKQlxy8Lw0fQjB#1g!=qeBF5_N_sl~&)wH%9v zSH1o8i+!?=kSK!5bJXPmIc*da=Nc~q%>Zeg-s89Q*9vOkGR8b;3r9d3)0F#z^i2&vrVPg4y34n2s5;0B;t0vI6kG@AF_5%8s#+g4Nh~sK-F_2R(5^yW#A35WE^Z{cqz!*7%F?^U09NuW0(PH}<00u$pD62of zH?FvxosA;6zFd@8*{;%UxUMMLUr}51wIEE4GxFj|0L3D5-RS^g0CVE; z-g3qufER?1<9)Xh!SG(U^BB`15wzTCOc~%9@bD7oRO`I+mVmtgh%-u$j8DG{fLET? z&qUj#L#YnvJ|IyS0y9~z0csa1GhmA#DhvTmn+_4`!R{hzbz(kL1 zJ>}dV;RAq2G#<-_16+#?c%P#Sa|Pw|Y3IO>aO%M0r;;l$Q3N7PYQ!2kMq$C$%fCzP9RT!R>{S`N&*l$GFZj<2q~a^s20Fj9`|Rt0a^%n z?jVrqhk_l|bcDfNUi7=%^B|ztY9A(EJ)-1r(c_}ljU(y8u)GH(6wf|5EE@g9w3xMJn$L?<8D9+ejpFf1Bi!d zl8-*bB+?8>jdB<~;$s=ofVF*|*=Y5{UO;z2`NqpNX1X5veu&~VQQ*3OIY7-~AVPn! zt_lzYD1bX45gC>M4_6)P@G{KcqY!mE1?s2^zfIE-nu zO3FJb1$yHMv;>6g<4B_b2=@vQ#3PMJ1-vbL4aj#=j5j~f98m0tvN?cUc6+zq0u<22 z1QhR<_+w}Spz@z^WBBQ^nFmMm z06ya7k89b+K%OH_0z!xcQEPmxk9y>&|BH)7F?ry@ zQijjC6p5j0fI%}}4hUV=oTD7^^?2neWvpp+=RN$6Fi z2}tNcK`D{|hCo7ikH7!Z?z8TPd+)mIS?`B;!#2e%0B&ex&mUqX<%#s zqM`zUjDSDT=_=@|KHSF>1Tr-R$$&s01`t)e4~Q0kfLlNUKnH=SUs3=28G6lEH2)4$ zwf((#x&zX<;~fwh5aJyWD5|V*38Zn&*p&7!2B7@|Oa240e-%!dO9$1`{(8jtVw-p^ z`V>noVSN2M^p1s@f$=T1s{-MDZhSx-G?d(PAe}Jd~dKf`cAO;t= zkU+gVcW(W~`R||q;Qx;UVdfv&K~wU7am^jw$Q~@%`Q{$?Q6U5x`0+n(`@aX6+&x0v zfJLMB z-Obwtz(MsBfaP5O%XzQ=0{`1{fm=W7io91K)#^qMqOVjyY`DjE)|(_Ro5z(Ys%Py2VoRMa%I zbZ6)p7@3%X8yeU_)KoMy)U-5obbp1B>LG9)M9V?Pd0t8H441hJy;#t>%a1c&Gl*aN z*u`xzfs;^n4US=C;^F1v7m&OlC4EswMO95*L-We@8~O%@M#eWS?^;>g0PMNBdw6|FORD^VcSRZ~x#A;gEQA{Fg5(5Y4~X0*?RU?7#8F0q{jl zOG`sb|CcW+>Ik6IaM047S31L~XHM@D#3gq5F~hlQ8LvNfF^Vf&;J96bCzyC7R8}SN ze_8v7v;Sv|#r%Kb>^~U$Prl|s*Fn_(G#YAZ8af&p8oD!dKs&>5=5J$QV)&;q{aa)C zr=9)V*#4uP0!XL;9JI8w^uRwmGb1zm|Lb@6Qw)fOh6)fS8V(QybV7cUBn|qv zPOT}K#H}d%^bVtB8i*zn$#geELO)aOYz>vgSv~sK^phQ8CvOo>tEV7p;#N=V%LmNx zL##q$i7753c7kGDbEm#;dC~t!JZ8~IUOdIHzA+>&xiFVtON_-|W7|o`v|+8wu-2`j zO>ym7+O3N5q8286FRoc^?)t2HpRinxTVN_k5DSqc067QdPQOVF#UkfW#4dfO>wH#~dhjeh_BjU6pSKHWliOP4IX6l_R zlvJx1t72?Dt_5X#3=WFMT!~>2W!De2I*#l;XRZ(WKmC8FF>sbcIAEzrH_9LDPM?A} zl~=95rgrq6g3>=C&hM-5EFYd#`#rzRzy3()ZPJlL-*SERMqC4$%W)2V&B>&zi2f@q z^izZ#Z{oSeL-mWaY4_@svb4j0h4AO4Bp5ijc3|SIPfGvH!@_@?3U)zZxVU0$vLd5d zrAb8x;)DJ+xCWPoFz=7nz!mDrh#3Wq$~QF^105s*4KiJzaJ<>pHA4 zLdmDf_??WSNeFG7o*Z`Y>bFNG0b-T>Pa3JDU(G76py#1oA_{ochC2KCdG9MF&}7-> z2EN|`v=0!?lDNdMQBHc>Hx;Q3O=i=NKEdFR>cmzj~bB$eJUyiFbcebve6p4FIW&?yLf3i9!$&_Q9vx2D2O{2prl z(Ml+`b#S`X#RbVD_;CbW%w=j}*}L(yux*bO^w3Nn(Nsa1^wm|=!&Tmx*e;DQy#hPV zLGk28gq?l$nml@C(zakHnduZ%)mw3$`Vpt>a=pAmuRz+n8hbwhK_BbS!LuBn zRT)@}cCNfZOweWI>!9|9+gdrU3afJCAv@yujRA~QT*t+q5XJ*8)q%&MQq5l*h7>br zNxJLn(>3RL#`!$eFAs>!*(#-8^$<+dK)Fs)MPWVu}uK$#ly>Yc6|<`C>c1xKN`_kJ+V9$Gi4 z5#DsG%J$Y8M}Vw{NT;BPBs^G%G9`dYBn!f-$O==xsp1Yhvv#=*g>vPwtOgLRTVhts zf(dG+Dc{|`&@k8MgGhiR-(DkR&ZvBXSWK#-=<1y{>u0MFIsqkm6%sRt>h{^SOAdpI zw~Tl=&Kzc>IAo37D@doo0La+V#r)@|EFHz0T*po-KIy?FhT6lKm=aT`Df{ z<57kQ%iwSy`<~t%eb9qX=ghByPsjgSm34#{mJxArRmY+C1EyeRO1`vt@9})IL0h5} zZ7fYrrxa*~>(SLKzOsAegf}O(luKj{oK&JbnWlWs4xZeN|AoTJlA1AoFqtiT@4mO$ zZk%x>r-dM+Iy2>U2~c5MX&UzdQV`*UKiuKQ!qivTI_2$md(k)exK{LIZS(i` zlHV~KN#aRT6QQ#(RD?oNoJB3Tjp^n}9ruxu>Q<5HREJP>KZLm&p^Rhe&N&oO#)oYS zD5MLOm9|WWC0EZ&eN{-`bx-10H&a%ovz^XnqJH&2u^amI3bJ#VVbl2`iJ0DRPcFB> zi?juo>|CiS9r>yIonynJoq4bSB~^2sdMBPf0e=4i^;@+pi@G1_FVRC`4FtcC_c#i#647^@yg@@(C|E#V{h`I$kuG3Qb8 z<+Gw0m45`gkHePYU+cYfW|qvpy43pcV}0X-hM?GGBO4IaVfuj(s%JO;I3gz0k0GWC z^D=z9$>xTqN#e${u^{oL5Q#6p7n_uYLL5xd(V$B`RwgW%MKqeN(9S{ z@Qc5%eVL}ifGPwOOfiVqhBb_F!{x+dol{I~vnNGrwW<1BO)$R;p1-{@1m@7yL5QCuqkUK} zKx6k7ezo3i3U7>7z!jFoE?9H&UyDUgjrYp;EohY}JQKQ=LeFCCU)hTmLHnG7QYcb* zoAeHLKi&?;k2u)NrYm0Y7wVgeznFL^*f=amC@n$JLv;CrwR)U+rSl}T&MAl^T7Z0$ z_yjMjN6cO`y}g7{q=grEr@EC5xX`Gm+ozR#$$#fq{NU4l-9zP?we@_^CC~YDLGRhd zm3+#5jzhlLJLto_-){^j(gvl@$z7`f4Siqx6kVQ14lvk%Y0Zt&0E6mych1y*|^%M)WhAi=ZUU#v*Y+0M`SuM>mL)xHltx(8lmQ-0Tn~BS9F3$daqcv88n4+m3^k0iZhq+c%J=Rxi3t0FLzrLMfFHK&=BFTk ziVsO8 zXC*v1RmH(H;?{)9PuL{$PfjdHNR+~lhnduEF;`cFkQ}JyQ;?ro44Ub7{0bzlgB2H^ zT;d01Cp(n>oLIE?A&f8R`|llPUAZFa(Lc?i*FUMZGTUS*@%#ho+z)ru>-`2|9D9Ei zXEJOa>Vyx7@GOv4b=R=-?ynl(EYtChU5p}`r=V=V0F$|kBqx!V$^VfwNS{bm3 z*TgD1t;`>Rwn6XwdUX?|-z0(lPye4A1KB?95S$&L>!P3uS8P%mmKnBqFd;r3^-u0u zICvI}^;GW^&T8R@cd(yol0ZkuJ105Xs+GR#%D%d?vl~-kdE1Jrqbeibjrr5RUuH-| zsJqBpO?+oJ+>meU5h^%i$xlDp?m~hPH+rEwPmhJvuc$vQ#p~u`O=`o&F85=dbZ!0J zGgKzLCG6HS*;A^6ez5)druY~dI&l@uYzX{reQoV*!9x$$FNrIk#QS(fBrbKd5Gpk# zpWXgG@3ZA6*tWyk{^p?krxv&2oLcUvL#)=eUuYQT(F4^7$P$}%So24}W4 z;7upbomFJlxr9suhvWGKuhK02e8u*XB1{=QhV(3Rz>&S;eaDmzZK-l6pDX%r%Tz1p z=e{jQ@+Tq+yi0>)mQ-yr4eqlj-De5=t7@5v4p`XY&ibdYNSbA0fE}(a*(()PJ3~&d zH)>l$E-TK>j<{v>!v^R0{$QkOVrq#IZ)}NQMg}g81_o@aguD?`mdb>syi;4JqKPbc z#uCSzLT*`aAHxQ0ue~PUoC!ODQaDosdffW*uX*QxzH&DwDVGq$*sF(G$3&z)b2L)s zy08-1nRI*scrm12blp-GcG0@qDWdPn^p_{)jy|(NKT~E51|5AoFFc;H^!3M%bfd)j z0+V)LrYO^FJ>ZQixc>Fe=&jAeq>1pi?N34-xIVPGmOkqQbZ$avfr@%#>bf-jm% zTM{d+E5gRLd8Mc!0=Kj-+oshv@-%0C&-;!{!`6}8=;vgi0dKRV{?u{y;QHdZlGT~` zH=C_IxsnGw$LOS;b{D06H&2HxhYE!^vCIb7=4Bn;FyNo%F?BvO90#5e3vVJs{~K}Rs4_nG-^<*#XZ7`tA%mz zxEo8Nsy83}*#f95tEh*CMTX)_9Tu(&nTwSMy%94Mc>g|y<$LkXf9ZY5>BI^7{cKKn zYwXM8hX*D)4WD7>K7JT()xZoJzNSA@1;}^iaw8g8F^LG_%MSaI)NgBwxE8&zg-!#zwzz}K{R;WB<~h=cmTDHSQL{^Zs_%_lgY~1(fEnr-Shd-nUJXsPYY5=Kp>6!$ zOuW9%M7N?+#ZMxI$}@>Z=*9~5ur1q**$GJfbFlA8VaG-Lqh#K~RVSEw3sZ7yze|qP zBl=*!n3JNgc?buBnh2}@cBP%ySzuyNMvp4?`OU2Xerr0TZ9Q%xI_1UCz2GJpR+OJg z+5Bn*X*=up15HF^IqEuKvb;whusvNC*5!&G*HwDD2nXw#THfd=2I7k|wPI<@7A~hC z?ZoQ=%3OOAdHhoJzaEPBHx|j^{Xf3(F7V$*s9=}xw0^N9>1Vr>RNMncS?;5hewIz> zDM)baor3BWo`hl3A4lts*Y8M?!YRAgzGhAiP^FFzH9pg4r%mr5I=5`3`Heu_O(RxK zbGKJl<&`@HsuW#b+e%z{mPf8khUpt@?KTdaUuNlF8T3UTFmOI0W70ieNbFG^BMsn zX_0zgftTY?oBkIi^JKq#GA?@|V3d2fsMc8PSIBGC3oT}~!d>q(2cz)LtU$OfU4#YS z-w=ZPytn{29H_;Ziuu?-cXs-AM_6~l_vn(dL7CkXJ#kz)+=7qoGqBk5KLv3kI8WYa zUnl7huXkH7UB)e@p+vG>fjFFw(c*hUx`D!nQZKTQrBx~Jh0HGtCt)=Hnu^rxnknxm zdwQ0iMs8z_`w6XCzPdh3qZfAH*1VNrughA#PO#X&Jyla0W4CJU=kPeZFuq9KyXS+D z64QeJ7Y2(kTVo55qrbwx@l<+arQEqo}Lvd2O z$0|&{&)%@pEJ5A+_)~j?TCM#r!XZzczVxtm$Pqs7nkI`?u%?gzdHoa=^NehHQYwi< z#gVn}G(F~IZQN{+vr3{_t}*cw1dAfZ6-!tccdhYeX2{>rOhvt-d)ch$2{R zy%sodQfFtl2K|8KrSSed1@Tive9^4!*P|j_-foE)6`4pGZOpU0yL4w#B8keK|Hw=v z8#hMgobQ5uLsI`K<~I=;YB#MMz^u}m+cIr&V_GXiUC;v zo0e&8WC_i#<4Ep(!<`JgAYtqz4ZX^n-V5bAFk@eUm+W|m)la}mye(!&M$mb@=Tlzr zQGTIN5I1DN6L@+Lu5Fi%$=*=LkBf z4B_5s4}X!f!x!%^_n~m;!UEUJSMy^SC~h#Gk__oZk)_8@L42reY%e@n`sfm; zPrkc96=m~bg_>W=vcbcx_J~(IqN^8`|A`G zcLQ=@N0CIX8X$>%;QvNXJbLvqAVMFI<2*<*?=R@8GYWUTABr=7ODtO@L6e<@ecHvt z0>!@C`#JE@Uef5fC9>@IJH0|E?x&%4=oI{-vEkg7`)5hv)pXSYmjFbG$IVEK|ZhkG$6*Myv%> zks)}+)C~<&8q>Cpt<=!~W~28y-yYGo)PM=WxTi&QM2;0Gi=$L-$eC=~S=khhuyEa} zKsI!NxYttaRv<;dHla*VqQLKW(`L{eEW-WLRQvCFWA)aF`8M_^vE z9xiDMt5A$(E@snKGL7&n#YmXSJu9tx>@2i?nIx#wg;HpQ$Ua@r$=>;n{!F-Zu-Cwas!Q^C>;0SAX{nf3;cKkNM*|Z?T z(H4Xm;n_Gk05w;+s_+gU(g)^2$X9&(73|OpyEo}(ydITkEos{{50g{(7tPqK6#5QA zXv6N;v6)YX*3AA{IF_RmTH`jhj&+b)oZmBijr{KCYVwTf0e^Rqf?{Dy>R7AJrq%vj;a#m$5u1D9aR~TtH4&Py!H3RaQk<}~OZ1mlA?e%>3cdF{AkA2KVSwS@bvZvl7o#szd+8;opw(vz=%0<`o2Q`e z1mvF1Lmw<_z%5*V>^pe6rS9#*Qoy%Yx}$3(KR!ql*09px#&2|wosC{9t}lLEzniLi z2A><-PPdBY%zHakUztqb-jamTERygk%cLLa7@f1Mk6~agd=mtBMAI?!S7{ai;b-Mr zXVq0cOKF-H%RnXC8)1MA{4Yc#pMqL8t9Q2}D@X4CDkp~$(_w*na}uq)FO!10q{UM* zislDq(7L*slyA^Rw(UcFyd%*Z%f|P!NUROFFd?yr@p;B&7V41pslDJp&nlF2-!6(OZ1n+;dP}Tt$ z;2fBPYFBV;L(?|tL20x;S%9ccx(a_nh{un0p%a(+wC}YBJlPg&HCa@&YmTj9o$rSk z@3)V?#(K#$4|;UEo-vz);0Af22M*DI_SQcIE&Q29oUy{w_nM`kIj(}ZW?2@c4)t<@v z#k>d=3u#RIM@Rf%!pbE$Lb=+S*Cx90Q|^kxs4|PSjEeNvy4pRPA8o&uz9avyWE}jp z$c{T89`sz%I9UD|Gxq-rMdmPRxrQl{bF2 zIKDyZjF;K>*JvQ_(S61Iz>;NlB}GJo7|X)^uI6=^=Y^*9VZbPJWL!)^yq_<8zSB)^ zry4bI(r)9joa@dmE+%Wf^%0t9G{a0+m+b)3r;=vUC89$-B1uF~fZ50#K#UZ;3FV^j zkuA|qBFfb*&0j+ouNfGv%4#XWUbTvx?Kis=>AQ)7He?gG@I0f@V$t)^y5%131sB{w zetdEbVL$yN>iB$s-o`Z>$d1jn72<8gwL-bOt0o>4!}-A97pm_I$yi5Zq1U^Uq*2m@ zm?Y^})c?~REdl}7$Ma9t0=9SY?`Jz9Y>>J$xeDoi50?EJl#M*|?J_!C#PCn~_)y;- zP++H^m8#;s5CVjtV0#MsNy-Op`lDA~qnpBWU-e;kOuEBy-j;?!Ll4!SgbA1QNz)14 zay)NC%YFWX9-rzV{WcKZqhlx_B;k#TPIzAZrU~+0!fudPkSlaMH-_E*uF~KaywQ7g zw!G{|0d0>2*G&|O7m?lRY?RLehFv>S-T58mSXnT${m;vSgFA1g^1ZinW(=3=hQtS& zIjK@j&YoSsuNC7=#vVJX;UNR2%4=HRO#N2P7!Bdkb7JR`HaD@1^41KtYmw%%TG@k6 zuhf`9SN~>CV;)7%0n1vX=W^l_7?<9I++KBi=e_k%sl7pYXuY9#BJf5M^-fAqS_`TJ z={W}80(=TxTB0HroP`@-e#TLFP;=}iI$zRHq;j zd_j~CY~vIZR+?9WIJlr4rf~>O1~<@5e_wlG5DxAzR7MxBQJ*ifB9dSQ;h$Bu>f%-Z=#gO#r@t>&YpIlG^l&trF>HHV-EB5dkybXB{no>vka0-)e~w zl(*Q`Z}GF!{^sVYJ7eb>>@IhP%{*;-ZJ7xYusi<*o10KojZj^Ep}KQfYu+}Dv3~LF z`VZKfLfox%uI6+ZzL;-6&D@|7$Sw=y_o*;N64NjA@uk^OG|rIih?jB^IXjbe3VP)@ zPTzs-x&iSWODeA8!-0ri1UBM3WJB~!0POQA2){uvQ}h&8qO{b_YKIQ!nv&R&UDwHW z_$;hDnwg?n!0h+Q^HQi^fstve>gRj8P0x4(ZzX^BO3%BwXLl8tA}h7a`$mx8_W<+t zFxr(kva*rf!9&&}y5Ki%wB&^@^YlsfeOkCt`E4^t-9j<9CHj}FzzCO@)CSFO&Z`h# z^vbK{M*6k!pKp+j8zsvU*{cb$T` z+PzeG4j(k822?EHe07~#d7#7KD3Rew*3e1>Y6DE@Iw`3_`as#pa(MUc3}X7N$=_9B z73W}#kwO)Sug(*nn=_(Vo)5|R9jsx~6aPS%kzLZEeyJA>l3<2C`3gl}KD}@KPn5T}E5J3!zU!BNCE3R5|brmI6{rXBnq9@4w- z^fRJKjjLU4#+1t#CVt}wv-3rnn=jpI(q4tWVlz95gZKimV#>4#%WN2g$2+{$;N9v- zg8YvtsSvt>qLkH~i|Q)y`4oPz?wgJus05@NBpF;c)=#sBK8HAid$|0x$Zy0>HMS*A zHngQV13Z$c@v6xC>#9TKor1)n1!GmhACSH_4$~zG1V_(ZdKVF{VP(ZHku^>=gD!*! z0^C_gLz+^;dR9Uw#c)4AH46lfxsIvN(LTNA9-!hee6NY-B_8@745-@VW!yW6CxnCf zBDv6^uX%hM7ud7oyX@_fW-uR?#X6UCoJ3Y8DwE_F$fhT`9r6hOy}}x@*bI;_h0AgG zmGAzM?+gr|zmmPO$AEnVDS^x-uW=$GiGr(UiKrjdlrq?lwT9L`T`7Xmm zRDv8EspALKA52jgnekD>?fCf4#hb{SFsG2)mQ%qyF*|xXgL}4AC$qZs%S2hq$I4&P zQbf;Ij{`!CiO#JLan|YK5>NOs20>TBEfyo^q&@O_y+7HdfsV7xP#e5G$d9}`7x5%Q z)P%0IBb*>_{Y6;k$3(qW-fwq^S8Va2uhHIG_8_c;g;6{tZIO-4IYoGiPfN!?V0(O} zzeM#guH6_GJYzkj8CKU`pRnMc93C#-Cs--A!d3r=UKBh&YOvQQC}y;3e?69V<>S`l ztv;mUGVTt9omezR7{f2-WW8UskIlWwAs5o_TNOHgje#jpQ?q8D@a`lY?Tw5tqHiMt zvtN9G9X@$HV+*}|QNA~8UNDhI6<%Z|xCVGmj|%i?rSAk%orrzza}9T9urT}q9Yb%6 zldLJ@r=W8XUp6M9^1zb%r(0<+%l@|>&s{TqxJbqlYR>HZw;x#!Fk=X!gVc|t$h1KFht`HuhVk~@Y$ zPt+an&~KXUziwoM?_d}D>J-H2 zhED-}abe!xYa4yWB|ck~{QUfsCHZm3-`+#l5Q}p%;aO8jWWfH%)NZvd<&liM)sFpN z2l;Tp?MIXJW&6ugcYc03_xL5%=)*01IUyTZhIm201+2~{*0U|cD|_>KQpHR5s$^4Z`YUF_ZWQFZ^NO zJ6)!o7{kO%9)OUy&<(l$)d2(qQMj6X7xyQ&qx@mnh?js~sN;?9_0{vP;oSZ-nU@~S zK8x-FH>4*+e5;xJ^2?JB9OMPe*c>rE?6sG(GyFNeD^Y^g32MgOgjYqARHZR z@DM`MhZ+5x>{6c@KiwRwea)}y2_f;avW#!Yt&>AZw!dPk2n zJ6+Bi$V=@w+JWAP9tj5`dN5r&qW|*RTwZ(V+e6aT1?|fwqu={kNBwVk9f~-s6%|H~ zMw||ve@Ak<`^Pj}h)``&M6J6{&zldV}kk{Elj%TTJM+tJ5jrma(x@_d& z4k93fv?CL`@`Go~D{bc%`@HKuA}{~mxSPMk`6JCvTtPT4*xX3Cg67XzEOu!|T<2>g zi{Er%Oaknrl*`C2Ju(and~Umqeq`x7N}12E$y<;wLN=ZB2fgQMhhcXs+^^oi&r5%@)12vbq;#(pr%)} zEmS^AfX^!m;jXB&?pOmn~0A;|y0GPZhmKt5olrm%9+yW<>gTP*IA$LSA@a(VWQa04>!87Dtj7@-xLGQJcqGMIFZtc35+ z@~9x6KPe$=RlsZDGbS*#P2NdQ<%|pgm8^#Tkp3I|1k|$G+(3_r77p6Q#sIg$y6-s> zi*MEK1(fEj?f6g1HZs1*S>D^}8oRA^hrZEV{~YLl<4$!t`A?-b&3-Q7?XtBu*EJZ=D;!z|yXmMP$Bp$V0 zS4ri+#MzQ-i^GO;5+f}0O081db9i#)CFE}FT8e0hb;O;NLA;oeX|2)mD0qldaTl7s zomXoK9_8F{XV*_IxzW}$fm}QP1ud6=v|30%nkWHcfc1Bm8+Uuy&XC>fCoFAo{n=Gp zx%oMW4o^W2#g(WFx3%(*+1S~RoYfG?|T~eRSo?M7eQ8?g16}%!}3%2Y%yk0=dtI!j#om- z4{SyuAfy+XK6>X4osx(I(vLdnKnbY+bOtabG}LfU{cys!FKvTp@oR$j5^* zJ%k$E;nRC&u|@27x}0L{Nn|F2S8JY2)0Bm?>7VcEd%YP_yJ7}MKVWTl-f0r0-2(() zXp7iMSwZTtGdYK)Gu8T9ZJ%J3v)3yzdp-FNz)DNRsGGz(lI1rdx<5cJntL{EfEZc6 zjM01oq{dA&Vy7F+Tms9b*DSd7+h}1drE0%*8z=v210%IY;H@c$;oE)?W}>G8-A%8 z6U7O+@&TaNED7z~+2&KS-Wp&OhoWWyt%V zJh5LQikAB+a%zMxkz!x`UbAS>*IsX;8MoQKl$>SmWTqBko}4uY9+%o~T3IGug?I(T zFY&o6jq+sBeI7Po8jUJ=r+3J9k^YzUDafsj4&SunfZxS_O>!2d^(kWikVOoFfeR3i zNVgN~Y0dw9-1Tr^KnfyFy8O2u6K>^DtR3%2Eb=tY;=KOO;J$;&bCTATB~ZaK3~Y*!h5vH8->3jZFpQ+N*GKW+NBhonHf1g9ZH5|OLvasv|f zw|KcO@`?G4`uV(?iuo|CYt5w{w|o4m4)u=0;PEI zS(FjT!8t&xa86Rh*+v8x5cTJzjN890FGZ*}e`+l-daR&ekDXXaHD7jp^f}PB=*c)2 zrWU=LjP`&WUug9@A-9}@)~12%;#pl5gf#K0Div;|$GSt9GQHO^O$g6u-xZI@ZTlQS zYMbX9u($PIJ9$I+qWuXLm784T_zUgAzaQFWO=ijjTHhoA}wMlRm9RI@_y%TQlqc-Iw%)4@^cx|#?RTz@L* zE}0Q$n1FI?coL1O;JQey6c*%52&YS2;X0pvhv2Fc0`)b@Xl+-P_6W4fekA&&i&o^- zA@{_!n_<57f2^Ugry$ELF4OK*Ny(D-j>CXq*wA1y}r7W$mW&@|QL*6c%0HG*4iYRg~!s);gZ;w^)@S|N3%s_A6<#J>>da^VHL< z#sWlZQ6JXn!ASR#$aK5;!>4r<)e4Fp2OJItlTF{>`}LwdgqilQpA;j6FTBgD*a{!3 zc4~XLQEZdFYge$$`S^euD7T!?>VHO5B)H;JdyH$HoJ1!}1}qdk?mlNjiho9xA*7;S z`|5H}pvX!p5dx}xFWEdKZt!6iUhNHM^sBA?n(UY`HsSIkE0LGUY`sK!dLfzcNsCj^ zBoOF;5y9$|F=q+f+XfN>VT*4{QXOHv!bbg5AR=Cla}O)c_i3IvZ%Ik~qL!1pwd}YC zeFaJicnc#4e_Tx&uCfcl+l_pPMI*ym-#hBb`F=h})ts`zQY4!U{boWiwP6G5 zbC1703mIL+3jr=osm-7p+=0uwe)FQd!$agCsS2I)q2Bvi)>G8ET(f;$Aow0*1(!TX zf4U@t|B64C)ocI8!Qit}2dl0Ef5W8B>y`T<>va zNEVbc8i*PRwKL-@3Hij9N@DaMDV`a7)3~&;y-)qcu3n^D6BoPoFxaanRG2{{dW~ZS z^HiqLz#99eqo^^wu6=bR*3YKD;%655;WSZ;oNjp1qU}sMd-6itOPe#&=1TjWU*_Or zO13Ky6co*q-Gw|AktFKlP%9fxwKv*5S0rRFyhJy6|`Ctph6E=WOkqnid! zqM^t={lJYmZiJPUhOwQjU&zF-bcBs$ zZ$f*m{h|}U`>%0B4vC^iEC<{t+VEtjk2q9(7?cYdYoa4C88v69|Btkf(^)#3;-(0G zM4-IDCbt)Sw*9k=(tLsQN*P^)D7ll59g^M{ug9)Lsq$rU z_y9Jx?`^!Bk08xuhvnNDWgav7Ggt5X~QA3IPq$NIq_NZps*CW!eti8)TPW$K`R?y@U7nSO4&=LHws~}+2(QA7iMYM4^ zY3sO*(8phBb1Sze${J0zE4t=Ja9p)F9xVEi_1Psevv)IS9HRQi<5P7*Y`e9Zk$7W+ zev*XwqmTV+6*S`@Iz%6e(sv4~Q%nlmG3-4C-<}DGYw?^{52m`Yt{kEBQC%%fB7H1pef`pKFk_NW zk=j7XMUm+pBxi>N@%pOzlQ6Id9D8*CU{r@$=)=BLF2B)v%aM`ut9KvX21$gt=gEBt zF7%yr5PqrOt2EGm&Rc)?;e$`vRm&kjR@R+lxdv#kg0+C&U<$1K@j; zBc0T1JD0$3l*3Qa)j;qbcN-O_D~Ge`k~bVcvfm$#Pozlu*%V!9Z3>f>V_#yI#|9d2 zxaNU2XRe+E>}z;hG2x82fdcGFJ8X7=vHkDX6z`-TFvn{#U_oB}h2qxBWQ@ zt!9>O$g2PrxSu}GE3Uqn0q#4${lFmJhc5M{;F6il5=QBqamTeU^X6Aal-#~%MxwIG zCM2rp!47%+nG$Sj#40&>^Gem8dL7l^?XI;7XP9>s-JLbc~5^>R$mFO;DSCx3s%C`=%?*sP-sQ5PxX@eMF zI#iC4*q!&a4MQ1a8Z`3a%3MPd`%q!uWt^$m9L?Ph^^e4BQLJB_u#I0)JQpOsr&i0t zL>{X>9=~^oeq}7dKdgA3i%{Fvj_c38j86MgwpUHOtmocliFhhpGTdak#~yeo&V2dlf!z zd@^pMkB6=t_7tay@IdLv=97@ddXMI`pL&GE9rQzX4};|Z3A0YO{_PMT5;+PbO6JDa zg9Rx=6m1+bNuGyfJcH1G=o3bB056-`&W^9p(B>G_PScN8i=Kp1|5=`!$Np}>P1(t- z@QiN!I=rPsXk$#=yRbs_lDdYTTL0b^Pjsc(QwVJbACMmSG5fURoHzR01FN0+WfL}x zfeQ)GRH8k&k{WNt7$h|CY!&8ZiAPTscVLkM0N8*>o-Q*~3Yz z15ePks$1u>&$m#<(aqoL&Zp`97z>i{0#;h(!Y!}ZhL?#l$n72=)G?dz`s58+*r&;4 z*4^Y`xW)kcGDkvSsxR5esgh*Nn-xhynk$DF1C@ zRp-oM_#}_H1nlRaRrcd9YYToOmQr?Bcy2dxj+qbvZ1~E-mv`?n0<(KdF+lQN@y_8a zq~&?#mF?!{Y)u(p25abe)9jNdOg^Mrgy`}HVrq0}$-j;@ukj$gqPWTgo5aXq~P zvx$j#q+Qu|(E<)7+1!QI8gxBhtr^r16=FB61|cMYly=T3C?6;p38AN~ZT$+ETV5H9 z*VP3myP|y&7@t=X_Jygdjm`794Q*-1tFH>Ret`Kwsl#?=EPh+46m(sDS2~871g7vD z#v_~1AEGA+iU)0NP}v=4Xd#fNuG4RecwQW(F~B!8_7DdY2AzVQM)MFMrZ~1o+mPA= z;q$`su=Y96z3IhLm-Sm?;vO%G^U|R=K@|1&$Q2|fLKUC$aYd0AQ=sE^ttE42UHN6e z^~2}S)WuUSs~cD_tjr!{{KXl92m#)55HVskHA@D%*>P@aX|M%@zTdzYFVPFW;OS(3DNVCL z2CstI;LP0^n;5c`4?$Q^zbS2Il66-xo8`K%!jWC!2dz5JsC*E`4&rH+uB&*GqO0!g zOw798>>Y0W*01$e$xsUe-=&M~E*^=U9%C#mj*sT!J1DadE~HCw0x%;`s$($%T0YAh zs;1VmBRTmqn-?*fDe1v>b44t<_l#I)%ehG<_yEFvqA-5qRROm6DJ-nA(qm5ei+fY+ zUCM5;KYsYG$h90%k?H;Ecrw>@f^KcAUtYU-$b-@!t_62Lee38M$>U2uOnk+bjb2HE z?#Up4ss_}eGEhW02Ce(Ev3!%T`wr*$u^`ZW8Uv%{qEIXQFDj&CXEBa;TyKcQH{N#-IUJ@KeAF6eF8d zQBOsu$Plb;sHx!HN2V9@^B1Sv?>W9w$kG$Izo3U%dwC!+yS4s_#2P)~ERAmf_WiIE zrFJOizbZQ1DOz_jF?W$1fxNKoBNWT}@2<>!V0*5xB)fG5M7gOev>7dTk_PczykN%r zsu7LzeBXHd=33vRiRP^PkF#ffL`5>c7H@s&*|85uMUNTlY(g}Tu z;_5^VBZ;z}kIUfJYb%Gb2h#bh>PC~Os-2QZ5lH-Jc;?o>?wz?)H6N$~3i{pe+H0@)tY;e?b7j~QO}1VHPLd*?5XXF;&?1R6 z94{9azG`n`^HC^?{{6C~nkKnbIU553yAgxsO6WJb_eAJ@9L+Mayn1OL;RrZ>%#M2| zF~ZzzPJ4vG^b1#Q5a40G>zfHSqJ7+c)bDfxvisV`s9+cODnkxk5%TqP4wKos1rv5I z3vD;f1$~LXSMr{mL6_Jl>!+ajN?R&TU^09eK56WxqlzhSN|x*ZDPffd&TYZ z?9O1o#pf45a4`jxZjm0<474|~uR~qpb?=7Qj52gzc*YBJj-^L~7lkTOok)p1`L<&0 z` z)#GT>F!D|wwUa2m0qMk@4*ZeHKCd)2>1QH^_8+%wbP`hI5d?0&3R4HZB+bp~=~hO{pTdZ80_T0EWN(6I3o!fln*vKlLAVHwP$%qKS}GfY0$bm$o$$ zg4c+XVn~P}nVpI76VxABsI zn!$a!&jzJO0vTZR!F3Dm`$3r$;=-YsjQh63AtoX97{dXm z__zpn$2I$Or%_G%iPx_^ovtlki}fF`JO9)oicbEqq#oF2cScsGeAwwvHYq+MXkmM+ zw0<&>GmOuPO|Iucp26%YrJ-{NC30gP@K`}F{rfB=C!EEbhltDHdArZ7$T&@OzrTB? zL>ELi+|I9_G zW>0jIb^hKO%e~wdJqGcgz;k}8A1=>oXiC`W6H74Px-jNw_BD#@gSl4aiy-6249wzM zn8Fl{w@YM<%bAE0mjz{ARS?IX^IL6CvUfHrutAB@%dyJQ!GY1V=a;e7c+!P49$0Tz z3OnDp+*Kzb7j97<`Xr!6UlaRsi7EF|zz^zB=Jj%@sp)xuf(_M`J2_imN~hP~P4h~h zOdgz$_||X*ZM(c+^5Tx&yvpdZWE~hlb@jYjY1IR6U z#qFFB-XV-!Mg}f6mU1)<Y-%+9m3b3MGzm+B4DN<5n`Fl_Fy78NYBGC#+i zJm#!MZF!^V64%@kLZubM58r*IrsR15BgN(J$JuF6;ZyAJaj2-1*jt$6`_@=X`(xLy z*AFij_B{a+aT#vZcPV_A#lW#8Sy;nP_B*HwOvby#v`4l%otT+v`2lA=zJBK*7(yqR zsmDgZ-=dLMz6_YRUJu68VI?16;frhvXTpl32CB7nhqjpmG+##B_IbSGtM+==tfz>g zFN1J^xeRi^Pk_n%2?v(Jq&UOz2T|4)wPh1|n_NPRiG5q%o1kZVdK1=;_QCaB~~=Pmra; zck3n3dXFV@m-c0y31L&;!_9hHV^XFAqfXXKUbk1@syD`QC7SQI4}BK}UY5d-rxG>Q z@%+1ERen!vLxO2@RzVPr=J&5kZQ<%Ql0Y}~&;@@v2Z=@s2P}}Z++>A~jqch0kz`nzkLk%afY6u?s_~%h0dgv-=I+T+-GVrbM+M zxHzQpIT2xw4ngvV8GwO&9QhWZaVxGZVk@x_)S z6PVT?XNbk>Nr>gCZN!quKMAh!4+IL0{O9jl*CU?-IQn5 z{G9bEf&*Qy<((HUF)v@W^5_4%-r>@dc|~VwfgP(X8{@hmobKcF;)R1)^J9*8auL*K zp}GVfydd&nlEa5*N;pZZFlyUi>^fEyE$Gi-ypUV*B8pj%Wxz^_rMdFPjYI)|-8@fR zXFOyiWrwH)He+(0h@&cmVT|{LQ)QEaOtbHg8gq9=J^N~!AeZ%lbSqFty00Gv1V1Gh z4U~TYWd&7GPuUA1XQiS!9WP2aB5-V5bP9I_s2=Pe1Hyj+G;^G{Dj}Jr% z$|AM@|M@8S&#`|eibTBBmsh^$4n4@hj{=FdQs?AuF0w12szYkuQ(ly8q zMCJ>Ttz0mb$Ef*g#_7Fd>pXMBR>#PDM(P>mwJ~qASdH!R=jt14fah;-y^gh(>;PXe zrcm#cl(8JO4YF?dRMo}7Y!!4Bbm>Jk48u-Tmh@L6XaA_qNRYLuLePK220W9S{So|j>HD}4nH>-;oxMs(iX8FRM^M}jhM z?it^L?&}0WsxmtuL3ohM#rE78Y~+u}HjR_V+IsZOO%wY8CwoFH@LQR0EAP(gOJ4Q; z`gK9;p^?oje~Rq{zkd5e-6TVA@dfYhV+NtjcA~5 zbc=tjrY?R0&I4>H3!ZEvHCF~O_hj8B{yER^Hii<1rYd4C=PT|ZBAL&j7=}oPna4a8 zISPWp@Wqsu3^{f$3LJr)|Jt01c@Jukyr~ej603T1W2;|<|HieTTT1N}!w^cG`gCyVu>sof78nAYW3hYRzI${ zOc;M@3I$P_xHfsD6K-*H?3mT=kfX6a6>%d6zR#I^{mogh`2-MhZYz(Mu8-A%V}N#w zbD49FeSVnvV>wr~WhiIK4f0Y_V(K6%@zWGlsfjoGiF8HE4-O0!I9j9uL`)#ocyYT( zxP=p~&?Ce4wWM;jvMKEQcX*Qt-<$Yd^Guv97|1uWchQVLL4b8XkMaNT@PT*%y?x(@ z2O(VheEI$sz`#)D(f+B>0yP{3w0NdRfG$?=SldRkuPPwaHlQC0DSWK&_q5meZa4xM;$K zJudA%q2aqGrAeLgoLo`r3-RgXt56^}O7>?$sbz@8k@(u7*CH%Qtz7I1)rqgC6GgB7Z5An_tZxrw5+~ar8>M-A z+J}-wr8m_0lW%^(wL`Exn7U$mxP4qWh#>GURQy!gU-cF@vkL@x1)Qp2V1l?lC^Nqk zlJ~}-?Vt5Q{#oCOdQaSRBA(u^LxI^>J#BfR>joUjIx6gvAT9p9i0h;BAl)-hqF2vL z+K1%aAi&t#N6&~7oX&ZfuR;YjcFWJeoXxq`rzGkfi-Y-Mc&Wg*l?31QxT1>9Ow2G= z9Zj;_yK}fC9MVQnP*pqOtBi?nsXu7BmDeJgd*D0L%SkhDWahL&o49?Vbfy64nM|7# zQT!3#>+f6BVBenfh%5cW!A0+0lT}}pGk!b*z_p1h zkixu{wWk87+D)9>fL(lVqP0*q(f#RtTE-O~H8t|w^a5tfwp+{<)0dA9XV+&Y9z>@` zTD(f^^ro69Y*^xWpLRT^a!7_}YcGe7bm0_#f`Ez~{?kjHP8XoV(mr$vR@@7e-Nj?P zmI}hfs@VHo=q%C(d^Aa9k@W`7<_eiViwoJ#$qgW&xd+ z;AvwR@qWh&IJyPEokl7krAFZE3y%Olp#9eiF;$SF)_Yu7HPMm6gxJG-wW!@(dE!@H zh|sD&Op%jt%+70VriSR91g^Wwjy4%?h)*8Cn1dXI92gJLF3lQgv3+3;%{B9OUD|Te z0^6#Zr@a}-kPIclRw}Y8{LkOr+a+AV5gcg2h<=7Ao%DTBxmA}!jWD}ZCbV8(-V*UoH)j=Oee1>AV*5`#acX9PCL6%+NEaZb(xgc8yFHTDlX3Rk6o$tuI^2)0;pj`yG_zCX;>ikSOxO(gLv>I4mn*Ut$yA^&p;cmO0k0MO(=bmD)Ycd8&) z`2V(#`xZuE7o{8Xn8x_1c|{l0$DIlSv9C4xXf>Oj8C5*XGtPRa!ddji(kC*xQGv+P z-i;%mggvL2BLGC8cj}=FFqshs$X@2leR`Sf_UEe!KDvY>5DH(JYM>9Ni8aJCUc#mn zVae2|xHEL998B(%MHoGMlC&HIooWj5$}mpy?NPN+B?gJ6PBo+LdVr|e5q?ol63u{N zs*O?aHVI29(y^o$JchFR$zx2R!B+%VPFl@e#Vm20Bz+Go7a7m zQ}1_=K~wFjSSXu}i{Mk+*J6Eg8749{8Ta)rkfef6og+_3@GPv|!e_b#9kMQ@jz8pC z{b=~}nj+_?bF{?Q-w1!rUy+l?6CU^+DK3*AjXr4-i#qW`DK&&I6rQy)0XQ{CJA9Wa#N-vmC9pCIFkZ&6PUsGmqN#NE>JpRGK5iSWU{*o5356TvWEnq0*@B% z+D%)+ro~*y`dK+a$jS}>>*pmkUsAU;`w0~d&hHB3O`b?xxv+# zz)+it)ar^R>(Y2*x2H=odSp1{4?53w*J0Z}l@C@3^;JSQN#K%^Z zceDWM7Y7x`V>8_tg|uV_%Wg;S@;aVX^YMfENmF^?&rOo0vx z570XX!-wte177&q0xnIF6_K2Fu^?~GLDcv4Tn9>G{*p0WQIE`XSVf>eneSt?Nk|9Q-z`6v{vx{*kTMu^S6!Y-{IH?Nut+}Y`<-3>nPsukk zv4Vh;OSC^@jBDEv&4(}6x!|>dVQxV&1s3cXCZGjvk%KJRr<}pQLOwC7Y>uumNSR^@ z`Q~JsP7prGa_gz;6UFrc@x$aqfT#la8@ruzlq)z4FHk#)yRhU!THbi4Cr-{&!Xfu7 zN3zOuweOtQT&UPrNDpQCdf+<$zKCyNIx04p>=JL}>Up|=FEk(a${5W*3&DCl9uC1X zOjwwPgl(OOGlm11Y?s{rZM(VGjF!?wGjj3(+AJ6^kF!G66=iMf@^&?s$vDv()r_q* zj(s-2DQj#3eH+Z(7-sa%cdl`<+zn#<@6rdqfl&Cfb?3u4+cPiv^EhPEwbivq78;c} z7)adK$^vTg{e5gJj2*)&7{!H8Peu2MT;JD20J!SYH~>@uo@y1_GMS8%NRy~uo>N;w zK%+cE%XeD)o^m8hdR)}-ItgTUZLYUm7vG0GLH{_v=stvF01%aZHNUTFn8ZBtwprYC zi{DR>QaaGJzEBN6a037SF9cIhq76BJ*BPzL`1ibdziC=w3n34LN~4~<;?}sh5+}zSgBO1COT}9B#47##;QE6aZ0s zA8j|+sB^&bPkDb;a6InZ{2%vTyelCbpSF_g`tR+ed} z-2H6#p0=V>%$jlXYDn0v*vLp=j-1HL4PAmv>1wWEzz&$AygH%d=vW{*Kwm>cLdD-6 z+1g&==sd3_pNd$op|O#^)n9d&u<%hlB*$Hs|_&Nn8XFit%M)ntg<1J3B9WJ@|-%zrz#a+*AdawqxeR zud#x4buuBMLp-*&BsPww_S~_zvff|*aSF8G8Wij*zy-TBVkg(baZJI+aJH^iM*yT8 z61sfQ+Pmt!u*%&IUG((!`NBNQ-{;)+R407W!2dEjwg?GKK>*O2)?3hB#`}{GQVO*T zwx{;*)DGs&eVS_`Q7jyy{n|Vd`I`9!q-b21WDR-#-q&hLCn%F;*S&%y^U`+amD1f^ zu^B-vlctGd>FlxLx7z2cgS)erzwIq?g%`$Lq7el*?`N>17UFz$&XlpxWs-0lx9}^0 z>!_y%`8pI66X{)(sUe8;n)oj5n$e4g*CJ+RY?aeX0TSNhJ9Gpn;clt`Find?v9Hso z5vOZGer|fGjEXGr@yaxxO?M_MZ>uEzcR2>IXzzH%K8m0m@lES4aE)YFneyoro_;qo zSWL@_Q8(fPS|GV8eq3BDyhG`d7TGJp!3X~%{Y9oeDmq7LZ ze*N!~0m;uPAcRo<@oNat+DNu=0z~g6A2=SUkxdlX#8`h!d{zjxOs)U~1)?`+9aOIxgcG_Ea*^iK}T?Am{*U9 zVJB6%aEXH+Z=|mR#o+p@Jpo}>spNx8lSa-$sT`r!B$prF^^G|G31We%^h#RXz|H{~ zFC_TL&wsyLo?+^ludv5b+4cP9T2zOakFyX*qYrFP$Ux18_4}iNXjzYIL`P8?=ee!q z+ZNPll9*i&%&YsT4>2CLjn&O{O;sYx1BYcip+a=>{wk};#&#*G1%U6}s{>5+NUv&q zk45{*z}?oVoT^yT74K&|%$)Iok&;(61)zb+CslL-8n_X(VQdQ(x_D?=GP)Mb4AW{i zoEJvzWoFx->aRx}MU)G1kJ+SdXuD&k^i;s_xPrJ&rbCJAE{Y}_-7e9)0Q(dD zwgn4H=oNs}d{CVwP@f?*T)`t*|3e~{AaZo?{xzfbCC;*S>FpvP0_0b*wCEBg?D`^` za1V=W?O@X2U`$f?Y7M4dHZw;H>`SkJCx0srHy1fQR!;r;j8P&ZxhLNST(p}8si1)!?bJ^n_Q*oWsQD6_c&cNvI3qjGE(4u5$3 z1O+~VqdmCk2-y4D*7vr7tCRr0xwP?J5_diaUwpiPGxr@i(}3luo5Cr5MaFUJ;LvQe z(~S7)*=$)+P)E1#rf2`xW(#AvhLj8yNlR!ds-PFlu6Z8cfJb;uJ><*ePTr(%qIe}8SzoeE*EI>#KM52y3G z>fn~+?|Ko4IpP$ifSx6du@_RrSM5MxonIdItEi?@05H|ueq0Uzz}e=}OmB4))T3cE zUqneHM14T@`l01(hMOarhvbB$#D>Mg|1MSK3}j`P3D zdaFDK=-q7o1es_31dWs!782h1W#2}sTu#oKfPP6)8@4GS5)}6bX|;%ABG7N->9M}< z?Zv&>bJ*f73KrHrTO+;i&fj<&L~fgBJSJ49$Y2t7vuIH1!zNR9BpMGTG`b zzFC{TZI7^tRK!tb*ba$rV8E>YTvVjI-GbH{qsyrGG9{ab-E@hp%b->Xu%oh4+-hPX zHCQA_B{~2kU;f7pe6!aohPz)s(qg_mv|}#ba(i(QAimIPc=a`HcgkQ=^5PBNh_;FA zkJqhZQT-Z>Q&UIx#aY`??O2j(ra6@4^7P-CC0qro1Q5`3?oBdXDrFh z>yo}aApo6v{UX3$k9cl&0Ov2udhFTAINTvDR43+o8|;u){DuhoX%JE@0{`-00p9?) z-4E)D40V4L;MP69f8E?a9i3LG^IV*ji_em|ByzVPstm{NkvnY$3m(ciki9Lwqu5f))}f3WY(n3+VEk==Ag6~iuwp3Fm1NFwv{-#_X)B~SfY&$!Dk_#w<_DdA516&rre z6B}ojc2o%X z5Xx14;p#6!xb~BQNChVRWns%YBlDz3rciS&t;upRKLElo(AYfo4EL5{i9eC8=^d_U z^6tZifanL99#YV0^&e#45EnMj--xaV-f2;RJmdqrx^0;xkLQwwxCs)AkH!^;Xer|BL=AtPI~YZb^@8|2l%@4{uflC&n} z*{83|NYAm|%7G%lSH$ZGZieJ%|=ldO!j_4lk1QW-TtPM zz#DX`{s%d=nl)la&=k(QzO2yf&B3KrG{Nv*@E~8+G)B;mZPteB){QuFYiVa@r4@0v zc~NW>8fe9<2W!>I_$QXzTL_QV#3+48Sng|N6hUY?kHm)=*&5dkvC??15V|RXpZv)O z^)1k}%3-TNGd8_ z#wK}xO1#-mU9%dvV59u{$7XFkgAkDIBQCwW+pkDq?c}1slu_lMkM8y=m8;s$B zL4$38gkePU-<&b30s-B9wU`qAWNdG>GQ;j;gk5N4A2`j2kS0(a~Ty@LfB-+O5J zW|YB9`)ji_7xpbLMs~JVcZEAGO)o_q8Le3#P*gEnUyHSk1KJg5;6G^~({QW&cB*=q zFE^5c#!rOxlVnGe;O_3XsviQ*cFna08cz=B!?}dkllt1N4l=YoqtHW3>XA7Ottj^3 zf_A7CazNXERa~0KIB89iV3$q}G<%<%K^er<_o%@MbUO5>$C}gu*Yf!>^mlg6%Z=xHGL7=DZo}!xEDGymXU$ZFvpdK*nqu=u z_n_CUxw?1uoXRZFPJ8#?Yp{!1t*O_O->(0w5&Fj?_XJhYT@f+*9md(6Av0Q20;Yr^ zv`k^$Ip2OrBv^C|JsT56ULL%57ew`@fj9vm%nXwk`-Bb5hQLy_V2fk6<3h+Wb`i~t z{&^j4Uf&}T8CfBz?=?TNs8oLSHjU7?YqfVjT4+Xo6pie%>kS{m;rJ; z4fywhEHcwygPq^vj(-iBtx&-|rws5h@_&vg_0B+6a$ z>2xD6CvY32g7R;={d**_oO_<*YtEilV$^P(x1Bop{`4L~$7br*v{`wcXI%^}bBeEC zR~L=zVwgcMu|X}69%RO(-k%q(Y#OP#v4*0^eVKk#bf;h48!9F&qUxMyobyIN@1mDL zZfM710!Y5%7b*U3rr+RE+Vi0jiTEDJi`4 zL5@$@g>MpzPpjQCquOZj9MCai-`k@#4M)>8G5m(2TXB-_Vjf7wTar?-qF$}dh4x-s zxk7M+`I94sNe26inqk-VQ3mg_zpGV2Wo*CYmF6Tb6hu8`=;6Li{OhE|n3lJ@g~rI+^L(oWMP6+2`9mN9UtxCm2{HqwX@^xoCVzV2BEGCk zm_G(2+oE0MeOX%%LP`W{x~PGSts+{U45tswr=nyII=e+Qi?^F3-W6=g7`?96EL161 zE3;+dN`6v*AJY{ZsI;#Le(@*ou)ZEBe-l=KbO|ZNQ58E%{J?XKx2TlQ3>Iz0h$1>t z-3rn<1&ouTngdyY2@tLT>7!5JWuWM0%KNHC0qMl6&GN0N&@t7cySsDNF92?*a&>8q zpwbI@^P4f?W0?}zpz5@l@@?66;%}%^{8fs)zSA9dOR$QH5 z!Oqh)j(*y{!;f-x7ci^!o<)qyUO@ZbaH($;=+9#nGxZq!4c|#(-quROtr)#~lQKou z4_^FGSk1R5+WKSibD{@{hBGKs%ULbky91tS8?X{fKaWE8-3CdjE7dBdHJkY5Tvom-iKSyQ+(a!K}{I@w(vzoFQ1g3Uj_ z=zkvZt-NZpKJih2oRRtqJ$&@(qPP9R{4r0(1gj1f`w2)2fT>Vb?tjr2{vmmOtcyO{ zu5-{$by&!zW&%Ma(_{2LM{us0H0$J*H3y%IJ{LP5QQMzXuHucbJV#qMg*x&GK^-qB zDAwl-DORGblGuiQm6(h+EO$=T?uW^1=ED?_@=41>VJwDSI#;$&L;QuO+_A?xySc3m z;nRa|SKi&#EiJj+d2zAT*gVg0Z5`-m5B$j$0GJR6_3|THOzKaXX zUGsZud$VRYO-e5$i`)0tzx}!_1kxiAkNE)5`)0<8pK$>v(#k0-ol1ME!ntDJAepIr z{vmf0mzh}F6Q^hm-Ny-(ia;j*%aQyo6YrHLJI}9;qk*!sH8fO@<&FA2au>|5s)2@$ zml|01kUBFDD{ggNczwe>wEaSW_!Mx--q=zbGpu+)2`!kBmN#pzE<2Eyk};z{h7fwJ z$u?Smco?vzL3crFt7VvE>|Zxtxv$(dG(&43`))$TU#Yu|Ex3E*=6|H_=|Jkf52Wq@ zbY-OM-=YO{TO9S)q{T&c(P0xvJxpSc!d{j3&6(<_A_d3L${$I)hzC??>jq+Ps;-4_ znWXvpB^Oq#{KjB`uQaMF#b77oL~kp#%L?fQY368uR>DK2BU%5P>f^tJRO?!qf%vN^ zd>nL13rx4*AkR7WRdjHPtYmY|xWB2U%^^}VnIzIG>3cv)W(wOTRDsFYUjS#d>_WN8 zKnr`+&X{75@=%AXq|kyw7W^!k0Wzm{W$#icQ|qXvStRJx`cJxI)q?%LEL8y#&AmB@Vxm2 z;~D#&?XQtrqr%e9_Q=u5=#tk7>bQGqov(rD_54rKD{x0o8~t$`*wHKXY0arwg3_}T z3br&I7cU1+iOG7PDoG~AA0MF{@{Bs*MP}CTnc{@j#m3YggoBGvR3D2dxaO;f1~E`Q zgcQp_aKVBK+tKjTOaIkKK-I^A@`K%-&@Vi!ZxOtg@1>VZyr=#mbw}0LQciP;A1J_J zz!$jM-oon4-SE_8QAtng2`9%-M&}71EH^*UJ*!UtCOb@mNc&?La$hf_cWw>J$| zM0s>let02YRxB#hZf7ggRV4FyCcnWTiNm^)^rm^w13sdS-9HKaM8o$5HmU@&;ZKRe zzbfF3*sIlssE<>0jj{53*A(Ww9b)B(V86i`Z~QpmyJc>yNn;rlla6LjDcsJ`Z%q_q zr)xU9u$t`LItRbgRDqZuE@sM6bp?jAj5D2$%2h#rd4G)NA;PmbSM*)*m4!|(2@}sG z>+=s6^2%DX#}IRl64fVvl_5kf)VdwkO-nLUj^>740~h_i>A&SZKxOU2%cO`+ zJQ^u~)qX~AW-cB5o*EbeNWnOFz|0TBHzxkI^ke*TzPbSGrPele>~NPs<{8cG&~-f1 z2faNwhyrB)C%4`KCi1^IZ44XOeK{4mTN{QA&7Xt18F#``wDaZ>Uk=kAvQ*xts?;c{ ztvvj^ptNT}j;`;~VQ@&5j2u%Qow|#@(`;sXQ->Gfj$IASt4ZGW)_(_Hp}7=PwYPYS zIDpFUk4`1B<@%Bfn>kjtopD~GF&3`{^aadJOb0Fuj=L7cF6$Z|wm#>IAh)s($7L81 zgejKKQz0&&xNBe5R?&{UV`V$}pv{HZ&Xm=Ufa}b<;M(d!Kee+IhZ5jZeEwu7w6IQR zccB^lXx^|L@o=#|0h~dsb(9V?rEv_>FWQr$eN({oqW<&{`!LsMP=)w$H%R?gWhio; zx;^L^;q6YGK<{$&u94le{>LZ9l4G@fyu3i$b|qktb^goMq7z!=RL2}i4?AHSTE8c! z$RXo;n%)$lqvvVTtTt)z!^T*igRWN1$B%U%zAhq)H=9g9)5c~#96jP^l^A_idw00z zTuV1o|4ccrdvL`zncuzc(Z^fbtJAL~3oipRdwUdrJWvJ7*xu#L2rq`oHVUyQeod#N zs5_5zS(?8}Xuq_fMyPRIR$|v$Ga$AGb2|ukK_dNmrz+$`7k8;1jEM^pJ*aG&cruai z?KuzO)JT3_Tge~#ST=uvQjN9@nnZ&4oZA$GHHi=APVRdv`hvj<_5n*JutCs~Vc zcak?==rMVJmo^i-u`cNy<(1|K*tB9;ls+mYl5>PR(a?{vYj2X_hwF)}`KIKRnHs)E z>bA6Nma!Z6g7}EG(*G!9{?1IWoLjMd>MmmA)W5dW0bvTsPkb`PfQV+i{3UsVCZnMNZBbKetVHAz>1Feorsqmh z#smN{#4yQs*+1IqL+55Rbcw7Y##b9j!T+EEwH{E;Ti+DBr>TW(4dNSG0)R`0qShH`Rq+MGz zWHSQ!>Hm3;*x3>uk0#eK&nJ!mK2Ec=%@2|(rVLWkR5gPWQNcKi zjCYr%f7S|gN<`}o++DOnr-Qf-xnt^$j3M*UGOm`f_4firFT9==hp~0|c+Ri#Mq1qW zMm*R$qkVuM#t+rur*5k-%SEc>mTio+Y$uv61R04!hR)6J90+-V^s_6 zSfYWE@xWq65Yg_m!YN>D?Qvn{n|WnDsMZm-fC-&ED&eb*qbg5yaCY}`G8Z^zwoQE4 zNZa-F_9el3@z!C&lWK~*gWCbxAV1qbpF&dDcF+8c=23Y;*udwh{Lcf(eJV)Lfm{^V z6*JR0jUFOqR-V=8FJBBf?%Iyy#Sp;d8eA+FTH`#5yODE3J~^_pf^TgaTEEj#$7Q-_ z9qfD-npd&0B-;KpZTc;;Iq_e>#8BItWQKdhNDCqpI}I#2MmY?g`efD!CvjI_wfvHz zebw52MdKBFn})rspu19ZzPuVLRISyYa`;!^|_h#ms2B-@_AQAAOe+r>2 zV^S?}pmv&6)nflJsa_@_huBVP5a#@D$&a=jLMQg!S2_@IVPv&Aw2r8MMplVEc?EbugI|UBTOVj78)u0=sX=)~KYr* zxES?l&vF|3tMBIAEc?+%tV=TfFSf_2zI!OVmEa>bupC=wN^xIh$>=DGsbwG}mcKb% z07Eg@W}D4f6HTwhI2ZRRb+Yn2SA2Tc!<=Vn=)q)dccOOi{Y)LnE7N9!MYDeII8%m{ z(KW(hz6`S=))xchTmC5SR9{ZN<>d1Zdq-`oQ9AO#E!tcz1vS|SeVI#WhQz+?30xACUxB%ppop8?97$=tb?4y@r6WvhJ?$^3_1`hd z2I@|CgM6Ih&~BY6r|Vr5!umENW%BIiGcy(o7TQ-0y3CAKeAZH)5OS2ZYg&)fjyA!` zfvX=GW5lFev=@P^SvH&2;+)O2yluTGx?7&!ZtNUj6)AeyM~spdKYqUYsV)&^@Q>xS5w&+DABOEou_Ly~T&kN?S_nIwkWsNDbJtd0s)yol z;6%_pdGH*VaCej1r$PtsGxJZOM~ot$2@v-qXsh`s+POTI%l!SIsRkA3;9hyY#Kj>1 z0(H@37AG)SBgTI^)2t0)C>)%~caN)^^hI_v>30t}TJ`6>a8 zkD*r{PV(A zx6m$AfFa|h1kfladHnpAy}r>6;j0h6X5DV*vSgQ;$}(K9->G+4`b5>sVi)Qj1Mrj? zpm<`k2B0oQ*x-&q?F|jZIhJZ^Iq-1me-ZP;+66xlcFa89 zR=@{9pH{Z0{{(ShfWbG5YM?i9F+hq+LU_MoT|JHQ$JAl5!qc3caRQ&gsI%>pk+tsg zW88L!I3!XgVLNp)5eX*QV&{(P)GL*V#vIHj)2Kx}z5O{+qfE3|CIR&D8v$?RkpdYu zMFs62U$D(0c9!_CCK4l+NhZ@N;h7RAZK3pN=2|pbkut756@>n@9#_uoQ+98whwTo7 zfm-DJ{0$a84&AI^&N{Pd3D5Fd7x}wzP?WTXaKuY+WQ!==yTEjIOK*!MBzdfzsXF7% zo!#t=c}bPzq!8%MkMTxS?-T_yl`@ySLIrT93a`Y;W}sS9H8bX_9;D7sv>YysA1YRs z+$Qq0;+1XucDvh?B{SUjc6L|dHcNMUi0id@6QA%U17+o~3)6(TuL(92_=q}MASgbw zb|hA;?A{OEaUQg4Yi-Ha!4R=m=a=dXrZw%tcmgt}WT|A4EDvQ#yN0O6)^NdNt*8oW z#}pQPnTLb7Q1!YJ+fv_?OZ7&RQUp^QINo`yaiIM+()t1ch4E_4lQKq9+yn)Uoy-g`4C)ZoT&C zCrI1s+{UCC;|Tinw+3$f{SXiJor$H?l&&G_IL2)}Evi1K9H;0j<&?Z;U%o5Y4lTUf zPi^PUwX1M{m?YORgJQTglgWWybUfd?+?`Lzsy6T)MU(h^JJ*coo`{onz}{yAXAw6G zX>{CTS$*`e7S$Dd8l4NPgKO~@Kd`l+XL_FPRLC8=iV%6DN~F{%-d}a%WjYch0f5TK zZI?FjVhwF^Y9GffG+sOEsY5w*=wj$zH$3fq8ylTV=zW!jKYtu(h>>sfSCj&j;WxL+ z$q#$4++273INKQ}M>Kns1>9s5g}PlEM*@pvs^uS|hedViIWQcWL$qDUORQ|+bi3ph z%BH<6#on==g-@Mb9}LTSUIAYmvK@2-ZR9vgvnUoW;bbaNL*W%IkizZy*6Wk| zWmc7vhHKwVWR2M7I92Pyh*#eyb8FGY`WWV&{H+YDM|03l#Nb!VOSK}X-<@*u%62i= z7N17e2~GZt?}dUbnn=@u@9s0&Py2no*j~}RY<72)tBZ!yhT=8$(LsC$RUXI#{|vGK z`4;!LeR}LasQ$eWG`1RU+G++bPis3Dcl<{^wrP46Zr%O~%CGiM5H~*)2U+~1FGH|u zFT`TU^m#WE+s(rS|?fz7M^`WVrYLT*jwn=g*bfjO1vayg2Nk##b1O{f6M@6Krtp&)q(D+KCQo?ays-TbV1F||*tj5p!)$f#im(e8Z*eNHvuDV63 zrnGG#iu$oA7mu|VdVp@NfU^}lPMM_lgjdhp38@x5N}Q3c>KWZZaK>lT5(|k8q<=oA z_4dhg*x81AQ&h(CHdqNnhFV1j9I$m+(Y+>fXJ+1hT!vFZMm6l`*G}xL#ogyJuQF5X zm@Vu()Nn~8D&Ns13B;-F7_f+d8ABuH>aiYosiQiYl6nK%&lW`kxQq#7u)`EY$95W zm^UP_-b7(ICMRcSLUR`gJghFCb_?S}-VoIPhoMO;wb(>lO!O&-(w3FQW|F7S_Ahu! zW$S(GWe=cLWFz_eGjskirQi6`nU88~tI$Gii+(zqsHC%wqM`x8YB|m4Tk%>i9Aj-G zB?LsC-&#@0YoKAko-sHC{)?piL znJk^GZwj5&`&6SlKwDIAwbnWn6IA)dQh9lS3r!WeG@dh(Hz_ctn}iUnm!yt(H%{2k zIoI&|5*I;yyp}@+_9_0$mVwNt^zQ8bYoh9ME=+GeDltFrau!e|d%qt2G(e7556d7Z zr@sins(SFxu$mLz@myOsG+y*sE_Bp(o_Ei-v0rnjvkAw-5$ktuo?#HM-W*Mi7 zW-lt(z6}BVx8uklk-3?YT5KAr*uE{#ybh1b4OPepT&efU5_Qm zy4*^PdKv*hQtu|Jjh5RsNirHT)-y5+gh8XyipUeb;7zh^5=R~?fxIP2Ew-RGJ&z2lM5^Uiv=n5yOcABs@q*z<4qs^o0@v!SUn&jX81w&1o- ziGHcDwS$=v3~dbxC{T|7WBC42S8q|ACTbkG1x}7Xd$zr#u5POEX>CL7<9Z*yNj1Yu zrpXl1Hq;g=fR7WGej?*EO-XvIZ{a-Au;p)Mf8r={=6;rc6!8;uL~H0jsL1%w@7e!0 zh=u2=an-scsjyc_i_BtdPPReT0S zK^hM7d24t@ZUEfy2Rp0TShR zymls52}X#ql!-m8FuG(mci z5_*;1rT3B`AV^67AwppP-o4K`-#y#?-R-`2{Qk&rjDWn!T63-W%xBK|%!MOubt=MO z*z+9nnqp3s7aH;rxP7m%e(#~kK?kcSvs-5~XX{#7y{AK(5eI@w4%W=96AwAa7#Uyb zWt)1?n~a6;8mP5S?LlT3nfOGtmxBndaqDXbAFo>%2fw)UXCmO^|Gc1qJlk>0>kAA;X%6HV-=^t}Ln7 zB_ak-UqwH?ef`~wyEFs8`(!^=2-~`Gb=2k3&7|cDPU`C~aZ~Rt23mEH#Az^mQ(~!) z^{1P|3?4mh(x!6s5il8f^KlBYn_qu>863DyQcp2bh33#jr&zU6cW{j#`zdOEcFzzb z*CZy*%EUML%)7f6MN1T&H3z2Z4LYQz7X$KnYcW#(z$P$>OGO)iz+k z;|{+c2nKU0R6+duDyb<3Y}Am%tXUYOYR_t*1liP>B}fS)zD^kTwwGLDC_5wz&{PdM zGQvbLS(beB6touoQTpY5(|ft9T)jF&^>u^i?@Dg;nVt#H7Xi=#+K8dgo$DzC;8}(N z>dT3VRz;-vi5NIiSYS8VhSybqiZ4B|Zr)novSB?2XWF5i7{-^+s&1tHBC`1FNfTFb ziUkCo^M;FGG*vWPK3c5CW^G-1)IT0g|7gaaro9JRwh=cVAmnm6+5=lo?Hm$P8%qAT z{tbK<5F;n%<&dZ8RkUbi=2VAT0-KDz??2u>jE`j z!A3BHv(=%E{qW=735|^n{A5NVy|^#gYE3g(S?iL;uZB%P?(>7E7-?B@Z*sp_OukK+ zvmb%mOnxk_Nhsf5#xH?CQ{ZYU6+z!UwTgUUcwE$*O(O433%*P z5{w@eW3fQhV`p8kAm7~|OSRadp7v-hM-ZefwE?OTqH&A*-cDw}pcYz7D%EA8%2_p0 z{vQ77O#62xc!jEERJQ=b0f6$%kzENyXDvPTK zP&n~0E57Cle<@7cULF`GB={B2<-g+Bo%nz4>_LP-E)=i(l6eXyTxx0tulP_lR^FWd zHR(goxQmRq?ITWiuh`c?4@8)2f+Sj0P`;g6ybZHLm39q5X3l~lc;LZAiFfJu2YAY6 z@_BWY9W(}(3PF(GyT@ZShOw{*m{_E6xPv6ng+b&oTAHtXQsr08KV;vTy08C=dMGjY zH&9uk{yXb;$D4sdJt#NzhA0G94XLGP)&`VaSETRn7~vI7Y|&HPXHvPQoS4R#?05zC zuI`1}{UWRsYO~O7oxga0pBC$4NtfwEr`{2F!zz{5{$;om{9a@*A~;IFNkoA0OBN z#h~5{=)`rzZGBq7LG=ZD_+EhT3WC~dCU^bES|3t$7I5|9c8fH?swo!V*1U>w%a}F4 zjv7hLgXEj?AIHCcnkx6>$VpX2^@B@d4*K?r?(qffU2uZWoG|GrJVF)+0&m3F*IImys^%8VB{ zoxG11WzHw+S`qo<0SInClX3T$E9aJ3B9NdLjfMbN0P71#K)h zJ1WU;INobfVew?|?R-4{@H)eCJusBCG?N_{D^!m=?Bo-V<>Cie3&&W(k>K+?X5MW9 z*ux`irR-^&cE-lRC5x+hEMJ727+E%1xn$+DRH<8Iv|ZW^Q7THtZ%ykaK=?`Ipl6wo z3h2wD2N0jJC$-rR&Fgi>n3ZHa5+qmFTS0c){xlPEJctX=KF!$gPo{<7tg4z0U*b1DG|8KYdt zcPpqld#{}gm^u~kN$R55>7ueeU=EA9oUCWA*|75$8Fig#0P21L=r5Z@l ztF)!p85xw1tcUD~xS?i=gsG~;V*=h4 zG68{XtiHtYB=eYT<0@3jVzXxMUAJrV&l5J*)I^vg)?jsClb*ID+`dz(gg%WDgyA5- zm9!S%)+xzqaIbYtVE59a1l+6ZL2KET8bE6#%>qxyi%SN6HDez0b|CEieyhbP(m zCFRF0mLnyWnP!yBy(tgb5zVfJ&0NI&l7;)00c@!!+pN>AO$ClDK&VFXvyfuFiVZ(3mKN=O1pBX@8hWo zafqTmym3lUaTeSo?jmk+@IGk>6Q!jHM(vmTd3@M5`B1*G5!usCF~*Yg>25E#3Qt;= zG;W~WQC9$gxG>4G!}wjKpuxetaoZ$_qFNR-eQ^Gr=&O{-XNlG%%1+#6^c7b@kuXJ+ z3otlygxshJO;=c^f^fY(0dcW?g{-0OwR|pfFIgjE;E8BIhAkh@@cU}SCDQ~P!mRSK zG5AT-)OVYSXK^b?NnZ`Ns|Cxqimeog%dYd-$EORkQU(9xzRBAtd$o$gh@@9C1TASg`jU(lkIsTY|p-Tps z%8WmkgXao&1FH6nE!5B6dGElkxh~Pv#`2+SmB(Px{8JF|7`LHjF_6BH%s`mw^;(r_>&I*gH-;YmOD+GB#EHdd&{bF5*((iK45;B`6AF_laR&YpGc zO}35?hC`9QNkDJs8ky!pj9@;vlc%5+>1W+7IaD}O09NyLqlyN5Pw~GQY0(kWtOfBT z)CAt)O7a=9V8)29cX|2?rkT{hCn{4+q_-L(xrRt~g}~0}&w-D~Y5LH%*;w9m?chD+ zxzhCUV)+Bvd=HTyo)dT3K#a@nWB9l4bR49E#!!V_%s}@+_9Zo`X17du#CQAMM;Z7F z!Qf5i`<(qcDlND7pV{h;p1)bIqDqhJAdjgkb=5Sgd(OD}eq zIj=o8_#UZvOsT}|{}-MN_w*0HS9v=eVHli?DU|B@;{6CNUpP;qsv_BuP-1tr{>7W# zXhz7~9B6T+(P=EwpF4o|+a)tjo!gT4rmjju(6LkMY;uyD0LK4{;^-2>2 z4`~=AaX{$cFYTu)&-ZY;Uch2On6)%gXF)15xK6Ie!}dCL|61Y*=YqVLWqyeXCi0rCs((gH-uTT^w8?J_-cTh2R3s8g8Zqffms_a2tn1OF2fZ@ z&4@kWvM-TaHqEjXbgnOx)o!PBB-ZE3UfOat?`fN&HX~9Gon*>C1esHOWR6^qc%ZEm z+Q0P3SEs}&i{_9+UTUtij??;Qzr?*-vWJ;B_}*>S;{M-F;c_la*#C zC$g|SiW~m2mp5He{5n;{7qd0DRr&o=r1mv?T?UU<$06+m8nb$ON@XvNq+qhH)Xu&N zGX954+nsw>QaXgK_=^?ahdT|uMPjgX=8wesIzBJ42MpxJN**dC7c6TLFU*Tapo2bG8QUC=&s!ha_ywXzNgY&-y86IP3ubnC-VbvqcFZW`S@82p_ zW&|>H29I6eS+f=Y!5gq?dZR3qz@?f8#NU7pDd+2xRuj*XKG_@Zmm#=&d8I_O&U*yr z-g*ukvA;n_@M1U+Luz5#SH3l+g%~+}b}%H=N+nq~0poHU`H$1Awki$N*(~x6zwC?g zCI2H_e8I+!FbhmxZ}MzuxKED7j(cMn%YgDb z2**kw8eJ-S>O)JS)e63$L9PWAEa1k{~3X0Vh|Mpczdk2O*+F{i-3N!L_xx zTAZME+*;;h?(A-V-s$fHv)aiIYtuKOTzX(ZD=na%R=mth{%U7&A#geraJD3l_QOTFJcs#OJJMU+$6BbaPX&{`1|r(xx1kHPL4W|`)Lkay@~CLn8sdlq_Kbj$ zy&kXZKa%Ch(`?*Y`mDWL3#y$n*1A#`7f3O9Ui+GwPu(~X?NW;nx_`VRuSJmYsVtl0 zY=i<(?PJ_Ny)VA!rTZBorvjm^D7CW`z-8{wi67AFwYP;X4t75um!W%%vz7k+pazIf zL3xN+dM#8oFz=AqVNyIbT_T_Eacx{*p;!aVObKMhG zzKD) z_^~Gc4N&tB$H>N}%#zaN4!o$2zo*v((r3eSrzg7(cT$T>)SFYs3t~(~zxPLEe(^;H z%KR(&`QODEAcwm>I*V8)i(ZWfUw;^>ZraqK4SZ!v!u0)mhy+*h=8o$T|G+fOTJpXX92ABrb5YA8SDNTOz3?C4`iHo{OMVjmnC zCTq{a<>~smC1+03Q!-;d3mEDw+&_T^YjBG`dK3NSY0?eL*)~5ZwHcTqlG$4bt603S z^Wt6)o2GHzTiWo)!y$ocxfOR6xS%gjd(PNH#bw_}SeD-}(m2TgM{LB=O=4sF7l~mb zTV(4D+an;|@U`q9>(Ad0$Td7XBnl(kqgm)yMY26M>rG9y- z@RqBE_A;&e?UGgArH(SOu@%{Jr;saKapRlNMT#D(0|nnJ*PjNq;3av8w|aH?EH10P z-t-2+JyfjAE~d;Lh}e6aOwd;_PGvn}<%FOz@U zQ{JDdHfn)5!~%>YP&}StpHcZ=xrxYicH(U|sn4UYN|rQe8yf-FfmhOYE*r6L&G@v7 zU>&Dnz;rFRKM&cCi*e~SNsTRW652R6Xqt-}*cf5mXkOH!85~?p96uvr33QwL{;%pJ zWGXw(F+Z4lyb`8Uwqv4_nKGJfY3u4rm|{yh#f$qm|JH=KtB88c2yf?Hmg8zz8xh90 z>(qyO<=a_mvn9eliSeCcCw4UjwMY}Z(nKN}{p44T$; zLaAw247IxSmCk(!vW;Ujr*x%c_mPAy`#pIbyeI_|L-T!VI(X%-$mW=qU@vL1tI5sp zC#S>GuDsNJ)5TJWVS&Emfb?sckGM;Pqe)yeqBPhee_KlT(@CnEWsqofcg?`Y6;2L; z3JUiDJd$b*Ey9UET2_AsPLEp22)HP8$`$|U7sQ;)@m8ImA~R2e?0e*g8_adL`6oTh ztd%~Otxrj`K3ddFQ)Ix-CpZ{vje8jgEtQ#dAfy`S6m`EZgt zX{ODjJZ39l&LdOPptg2$U-E|`wKmI(w%+Nl+FImO#>A|wr_;8gjPUlxHbD<6|Dmm_A1!PbZ0NmT;2Q`#>>1wmfNf7Eo2|I?aI zcBBPX)HHzET@eI(gi;7C!YuYtMnVpRk7dD?wG7jFvAoSmyc~1)t8e$cBVY~nbG5h% zv&Yyu0dp@yr&?%vIzP5#0|zd&V= zTZQxDy7fBc%nAX*2v=b~$tO=zp>0fRAM6!u(mC85SihNloB==N+8}Fk7;ZCE?L42$ zA&0XgEOxWnsmt4_<3s!zA?DxI8)j0Ud={U4F1S}=1$P&$33%J~MJ2(`!WZ`BV$|2S zn($aPUM$`}irQAR_2WV;vD3n+OQBqS{`6Xpx6?lE_$Y)ISS&iGM?k%*W|)vCYh^P> zLWf73ZHI&8!+<7urBslj9eJ?75Gr;vZH<&zX%>6GNa5`Vv+X8{eCiKCQ4&|Y%#G=F zgD`2D1aCrzwb{?3+e~Sm*Yb+_K!$x5@c1aK^jbV6j2dki4QFuA=9uxizr;pBtyiCLP&SRYFp-0gWG+IH67%w#aVr0am#KyXDZNTV)Px4-(OQ)H z{Slc|b>*ZIyd;e~G&m=}f#MW6zGi`3$b_m$0r@Ia^3F+o|m6t)75clrO9Tit3mV+M>oN*;p~{_=l2Ko`B

4&=@9O%$u4942i?vl{lH4pf`l}RHkkH zAZcU{dCp)1@-G6Q-F`G`tiuG*N;Z2bUE#h;W<2KmA~l!oS=_RwOfM@?%MwU@V@#L7 zyskNHXNCB+tO0`dvm{Oz$?^9de{qd`tit3aJ=VDe$~)EU2RB%IuM%2He8&1Jl8d`Uot;L0w?iFtXX-E7f6Tt>*t@^jZ2)0y z0aJG!_B4CFV|n<{UeUK%6gMA4K6`evWbw5K!Ar5=DL;oPkd6w?A1_0WwSXVEOTh3Y zZ36%Zg?yi#{pp=!oY@(v2tbp;d|L6_Ouz+bX2fkps4;Hnlh5>*oYz5vo2tZ9p^=fj zzGx+uvH6+!nsW2Y0W={N4Is*naA3xOcz_Ki_!0^Xvst?j=)$IhFkQW=Ob`8~(X?9; zSTUp^G1!62Ov)5hW|)+zI;Y%bGIsI;RXbj{!T&PDv#4|NG1U4A<`uxzQ-58N!er;{ zNS(#iAY{Reac^x-A4(8|i#I4A*6j=dQs;vjvesk_&LG2u)p>>Mzk#wO+c6Zt(1U03 zyV(>0W6Ty=CHC}H5|~^Y^(>aZ9QF{UQ0!_)ACy}qR9P(N} zA2AcW@+>p7WFd5V`Xw`jb}M~vjnzx9W)`ba;E)cCP1~BrD!LZUv(jwe*RT7SlJ&K- z@HUnD6Jx1;m7_WcPE^zd=P!)lhZw>?yKoiejy8+}Sb)-H>bq^@?wBOxx^{+k-2Udx z4@Zn#>38NRLTQ3>GMhi@2l;J*$@7o%Q56O;^ip^Zrqj#!FU>442`%x=Q83A&P_#{RL5jn8mJ1^ZD&d%2b9& z#f}wqd#*(ToZyeLNDbZPpdU|Dl?RVU{r7&cxY#310Ai?}Xa%Zfgn5gIRnz&rC#UF} zzo6{C7<+fuSx@(MNo$D;cjVT>j~hueo%(Cjz3TO?T!MpLfo+ zsBTkt4$iJt#^_3^-{u02RHhzhyGFpD?V0`0r!7!{OFJ~pgoK@#zEei}vv7F$cYfmY z{_^seNwASZ*-Rwubvw(ajJ1`OFGvSy8 ziyTK0w0f(9$v?{l5hqO9C)hKZW&Erz#AvxP-R6mlpqFT|b*~XwzF` zNjn)rLV^9 zx0oS<-EB|>;kNdtp7|S>BQJzFLz*tWyRxC$MKc>ad;H3O@!TKZ5PrM|%GrAXlr4;Y z$n#96&J}iem_&V?cUgYa{LKEu_A^^2ZBe~V`h;6l%z}*~7WXcPJ$w8?7&a#-83;c$ zwaLBO%v-&K6N7`?VMJDMr@7*8(^Bf+52OKsa`4Y(X515mPs$sBf-JFsuQ~vg|EJ9> z*~BcU% zIg+>^_mJp{B!fU==8VAng5_)y=xGN~s@)DqdQ~{hO8Gp?PZe!>RAVZTYUor{!jPkB zUsr69Vk}9Ex4CsJ_Gj}O|7n--*4PTy;4UrB`|N4d_`~ovPe9DBzu=smz}T_Y|_C|-_odMVzfeY#|S$~smvvZ z+qh19M*Zg}InkS;RQe*zkRN9S1ch>fkSlvM%MTl}W}c+e4|bCeaq?T|_7ZUcQmW*b zSBTO|y88IlC=N9*v}=7F2u{RqW}9$j&>$p3rbL5Mj( zj75Z%qq)?^-6KEDwrC>!WaJffqDEr$mcal>AM2N|tp zaY=je;Ue^l0_~Ilksm(PN7B37-z-wNcI?J2&*}}XFKItYk86IhA4p;H?K3hsIv!C` zxfX!$C_@?-Td*U!5Hh(fwvFzvsn;Q>sZG4@d9OkWS5Gjo8}65V;!*myUxe)v8*lTFtPYs|C`6n+x=$P{`X$w#zm&n<`@M` zd`iw3J^N&edrG35T-$=o0rg=eL(HU4;L|>;<(XNiaiTHUUixuI*88^HpRS~zu6YoK zx$BZ{_L9ZytD#9&N{OQZ+cmUj)k$=1ZTimH$&g-ELRXQ-My^v5n-0swbw2)n2F_T% zPbT5l2P zA;>pd+1+79B$PI&ahP-aoNs)mqmF;14!l8_PqoS#wuo(AY0!9>|15I3|LFG)C1TjMX@4S9 zs0C2l0yAieOLR4zDP*o6wBd5kFVk|KvnDO^0pb-XYU7dvrr3yDzAPupcyHn8 zGrZuO@?7xQ_?Kz-5zDgjB=JM%y=k7w6-ix&+Wsr}Z{(ehEq-A?M|?f)b_A^d2HIuXY$V#tXpM0Q@T^+TX+X%TuhbfvE=o4!Q^P;A~| z<3`&>3h^~exCqt`RLpGRIZ>JHHpNM*$Cl1^Ai2%4O?94mfZ+7?$2OR6CwEuZS99z!FBY0C+ zw1uX;$t>7>k6zpKomHOHC{bKw+W#U9xCXdS^jN%!P+lokys>J0kGHJ}qNeIfLs?r* z6RihNQeDV-$H_0oHh5JI&wTyQ5B`hU9jWzP@H_>=PYb$l3-Y6l2nde{O;g;;jOIGT z>qTO;ZyG$3+ne@@jn-gDuZ5gVIvQ_H=VV{ir!%Mc(8umWP44PZ!?4m{48?L z_?CE8A5X9CZT!J75<}^}uTp=zPK29jy%6FanRcvzrWXIFQ_r*7diyh&BoqE=vUv}L zM38jdF=;({5sk=Kh#yaf|sGhc;0q zlIH2O7yW~YQ%omge-lmpzKjZ^aNoZ-#@`Q4P3;?+Z3DF#1|ZE%2?h!l{uFta&VN<7 zURxV9I|L8lp%vR|7#)vUAIu^0AFcKKm6Z^Vv{S@?D&U_7wt% z6!QylZ?446aG}Uswcz|-6xr~*($%Y&iNn?VvpnPGt6fZL5eF4~>Ez_2<_CLpI{8e| zUtgxapwjyK;6|r{V~#Z6ao^uBE#A>kComUcbNc%MkwR520HI#4_b^qww``eI=)t#d zay8m?lDl0W1r8yJ)FWF zNI8OhjFZxg!1{zl1U&YUpXQ^6k5Ln{Mm)Gwr$o`*mE2J|%BI%b@7yeme)5$+2cQ1V zP`6Gt&Pf-bhLaS7r_xs6st%7(JE=?If%k?Bj?^M-wVrWDG1q!Wg_7_H5IqlSq`73n z%Ar+;FlrAzLtKs_0ZNm?mJNxu?7Kx}o4$O(HnUr}2g8{uQ+@5OYz|zsXBNNO`aQ%l zo_5ORwXaQ5uOtM;Oi>?pZAW{O3Wq(J{!+uAQ44q&2x z2)t!AUOh^&@w1Ev=C9HI?4Db*f|10U>VmLl!$TcC1>cxGhEcf^Q20N>Xs~1mD)=$l zu3Is3w!UdK0w(6-=Vwzl;ZQGV%koux(Z=ENYmUJN8-YSl#Y)cXn2cAj95~+;`rx$3 zM}8#k?FXp;*8Rs>tjR{|2aw7FW0v{AUd6gkoF4CP8jBiw@*f}l>SDERV_w3zb*<26 z=ug5UJQ!-P?r}K9(w}?IsPi_bLXW?d-l5YF%-46nw#^A@Zx--0aL!vhpzADIarBo) zU5w<*tcKM_J2A_eUDKuzP*7>qIR2g(Em0cz2+>m5DAb*{VE39<*7#-Op(b{E`;vX~nAK^Wz01s^>DKy> z!X;(^vwM@OC^Y>7-3B{V+*wjkoc8Ly^=DQr=MMyU5Z44CK$yIfJm-YdV*H*No%St! z2MZgM%-ST$X?z5wrpQ+sD(IEg_P_tEct{s|tTZ}2w2c>|rh$qLsrM(Il2eu_>yG98 zdK$E_DI0nVhuB5-A&>dJ{8xRqgVsN*Xhg*7&OWxFT!rRVmhV6P3|s!{LT4{b zZKTMp(?juC|D!Famtu?d%;{Gj;5U#9!gqukY1;%1=$f6?8$&h<3qAkN%BdDN&zIS< ztU7D*ic5#_exp^&7jE_&m9(d6zh+r=0FKOK_nOdi!m>US4o~+;jFI7L@9oJ(1xpJ`!Ho<7L)#BZw#ZlfGi3hht{}h!cS*6yOv4U>vKU)YxizSQs>0q>ERCevSS+XkC`guzu~-ATNPM#5^PT7nrIH*xJQSalZ$|dw5zc;W~l#*>gh-zEVM& z-V$^jjbeixdAYE657m2o_&Qpz9I2pZLM`*{XOp%Kd?b2mPfZG|Ca0e)n5KHaEXU|hyL7PEnt!%;P~br=_H|GBtdi1w;{NK z6{q!>za*XXSjwu`w!zQv4QjYY`rgrGtTUcwJ6-VI>}fsSSI(dn$y7s5{X6a z{Cx^Hu`+0vaF_>TyM?-Ns0d@&-F9>QJL`iLd^;J)_tpsw!8szb>shxSb_dd_kZF&q z`29>;{Ip&;KbF};g!_W>FhC11QTZuWU>bd+Ri^I*mZf^ICtRDUd9FEh81rm#V{y|v z`*7%)6CYAB)D$a?NnFcR>y*i=TKCg7ZEmY=3VxWiW(sZ{93klTDrQzKWo_d+Z0W!< ze8SFE{w-`LA3k8h%A@7lu45wKJ4qUznp8$g1iKb!IOY1m3+r5&7Q}t3KABv+Cb{it z!SC>tY-b0nf{<5JMX%(X@rDKz@62X#RJHgrUvO$>Xr>wlvfi6Cs#iYed>^~PodOj9 z>@LhH=KL)oN%K$Q$}3mi;%3+>9xnu3Fn#}>CMtJlrvNLD|L`nNOFY^F7<;1)^kr$C z>tFU3xUt56#fh@ZVpUP%3L=<1gqfnz#Fr^?;#hi%+BO%#82*#gYNGh-460Ad^b%g? zSMv7_uW(@e&_0_$BCCyU#(k*OS)NOIeD`{*R>!w=NI}7=uG;2P;RTcFVch}^PyFKq z!)@}vXAHIuVzFE$(93T<3*nusWaeK}f$0`nGOmKJo77H1=Nsdc>*5&mBHvEnr;7{r>>f3it{oxn4TJKoQeUq}W6JlY4K#`n)D`$`YM_iXaS55Z04T(afI&g#kIznh_> z{vpc!Cm({HZabR{YiOI}sGx`%%@%Br0c7A5DV9wZhsz01vNGHP1d>c75rd{`;jj9y zt5bG%a-$SUVbXlTRw&OZwAv?=xZHw(PaIb=T^J^|+MSrE8xr?9QpY)y)K}c?J3N)h zmP4X0D1NnD=P1Jv2@CvX46`XF15vuTO1@}+1}`svZ>(9Wkw4}Yu(inj@m^9>vRxg; zPCkc)z#66LmA7l!)DEdD^fUV{ibeKWOx4pZ>O&5^QD^z--pvh$wMm*3#G-HRa}Gvz z%D$MdoKpOoaG!jXUprjafsGxtSy^o2WI)V%(&%P+=X2^A^Nq^<=u>(oQ8j^to)=HD zu4kCY?O8PkvV*Z?wbL9J05=;_Y>nE`GI78fZ?Z#zfJtp^0{^7LAU}L^x48W(y&LE#>t%JW0Jacs(s6o3Wqw! zU3}LP@cc^8HM)*|_1;HDhZ>c+-vD*0lZ>WjOfovYi~Xu#3w^-)1zs1E-%Nd_!x^m3 zHCgBb$%20tx6}Mu{Dz(z@8_Os$|@jbJzjM=222qwh_#J2M4Kk|jz3` z9A1)AUXB7k&Am$_lNL2j|GqK?s}90m`Cvf;)Z?H%+fiz3vt5lYW?8i{Urm`qUWqIriI6D4S%W-qNkf|;3s9+ z;8c{pEW~5yLDj`HXeHSB1t*EW2~&>NMDhjgwaJ@f>`WT#WXM9-;;X8bUFp);@_4$X zW)nJImJE>DS`qTz26F%d6tXdP?KnWUyk1W`#sr%D8z}WjBp{6jc9?)ehXDJ*p8gEO(j!Nx6 z`ca=2$wIH`zk6U!v+p?~ZlkKlq0jUK+zS-SWh}#PTzpVqfJU37o(y$0U5PDSG+uxv z9d#>OrW!Pgb@6;3ez8Pr6UwH}B8t2)F$z=$(BBSy#acA?0L0Zb+g9D#=}lN=aa?QJ_Sb9P>)Y*XkiC= zBokkECb-9(ACpn6;(`>s^-b5QR2raSvwSC6!ybCJ&mX=Iw9;R=GSkmZL0+GH;)U^J zgSBe*I5Z@=t}9ZJBu`~y-g@L*j|-}-8r@{FO2Yd zD6)D#}KKEVfXp61PX;h)IZ;(GW;0+kdzEMeVt=a7${ytrPX$$5 zSJ{g8zwYEST90Txpqn?^_K6Q`dCXNmiC?1vVSDSB>#Y6-_r=i5!Rr3OZiF!ubiw(t zI$W!&ZnAtqEow$QlF7j)OScF&=x}g^JNDI3GoAGim4<|K)D6)4f4?g3cl-uABxw8% z^f9aaH&Ef#Znk4c9+Dm#p}+(Dn(fwQbzL&!-Gqd5Wv##V*v= zT{)~ped2{~3q?TK*Wxlz$d!d7RkW#0i@6LqT0W?k^r3nFj$D`DN!N3N;)uSFo&o$c zIQvcQd+h5@{PMm?kL+*(9*iX-)7gya5V5~uW}v?3tdg>f6(NdIcu0M4v8Zqw%h;(0 ze-z(67TZoQC#D!LcNYN^h8|)x5oK&17rm8U@+rLq)^*_K(8-ey^j;kA0(W9mZR0}_ zvEiF<7Q}5r^SAFtisRg#N0884a9({i;V=;>)T z(G+kTBTji1`Nk%XMj*dedM8cSYR)ylA(887B(YfyuMi+Sr^{Dd3vk!W`p} zm`^L5_{oFn>bbg5Qtp_dt83KB=C90E&|4i$l|a>Nh(9pY=aro20(@6w$eMeel6Xi% zcXM_4Iu*qbQ`mJX96rJipqFjLm<%-2DpD}MOE~&W=xu?c%t?wuX@PeBvZfo}MVxwG zGdyiHPLir;t9x+gTK|VQnhNsI-w#(qj}Ic*sT4$^0Z7Bx77o`nKdH_pgw zsZ*b@Ql9pA=!-bx1aKE?-L##pmvCkj&e+uJ|EVYapSsdOOaP#C5G4^G2dITylTf^> z`QAbDOlv03A;8U*TH6QW!c}Xh&S^*4aek_577`CQ{wp7|Jcq+h;HH*gc!B9k;?pJHkt6U;6K^4xI;}#@v3mO0^^>t_RcZp!;pyZER zQokvCbHwhKSC$W(+bBBN)H{@K8hy}DNqIDJpODy`Cl5=H$nT#b!?Ez!F{KnfK`J1h5gHvOg8zQmInf_(o2Z-=6TCi)NqwOyy0C~ZHacExNpp9e zqB#-nB^r~ML@1G`0)nOiH((SS7bdR&`hKg&oZW(C72|8xG{H1%T0VKJlSZ^ugOfC) zKaGSU1^=lSnReZ5B#aRi9?`<*hh~EKPwp01LaR^psuiphGaMvZYaRIdN9qNcNz20+ zzC4XOS@>Utfp5Hz0`Ygi?xF*Fm=ybjuVDq*s8R)qyUD$;&Qu6-K1p?QqV}$v!M96% zZEfB1UhX!>eVnG&vldMxd5D}!1q_TaMEVI$j!w!+d6a4tX>M}`x?@HHNt{fP8R8kIEY-Ks!H#@>^FZ=$rUB-6kcS7 zp)(!@zxBFH5Efjv1eNW(1l5J>py>nQYEzaU8!~p$tBXquBeovSyw}TKitD9DzJaXd zNYbRTkpozSFHX)0ZKsZ&iANn7p~NNg4AP9#3}$lPdJr~$%vxXolDVzT(Zq7myqlCa zh*JiTf#Syp3+%|tOw?kh1p`pc*NSUF3hg?liFP7Y-SAKkYvdN`+E9^bPm}GnBUq=p zd7Y}f$FJVvwK?m>JFFW0r!gXo_NR8-*^X>#aVM=?lIY~+ZHj28Zx%ex&e#!!eW%9L zy@WRlw|VeO*F-9(|GaZY+R<#1voJ!0@SZlYy(lyp+}d0_x%>Qc6Cg$N%*a49l9F5G zF5UY7Veh@8nqIec;UGwn-laT^%Mn>K`=X~aT=X@q#g@e4Vwlf!% zpCAme-K`{w^TZnW+#YjxBK!1dE#7=JMFRYA-{2H>Z`myx6)#ic+g5Q7)z+=Ds&s8F zlkr@9rzg_R_)W=9SOVgC$)>rj1YQ)RV2_Ws@aTg@k|}tRw)@R1A=ALb))+ z(!ZEVghb*ZJ4nD#`65Uu|8TWtH8>ZcAAv$?@g~#hq?6t2-?<`OO1k!>SrC5*GvCE^ z4laXBd29Dk;jx%n?qsCDLWN0Fbc5-&T14$L$uDxH6G^54U-VLziJMKZ0?$EJqb@LrST&B`jwR#9=AkEf5bZJ}@Jnk8RjZFP`h9c?f);Lb;tL2v3HC92sO zxBUAFz3FxzwPw`EwV7xrC#?cuSW;P zx4HPf2u(N3!mm&HvB7Ti8)7D+t55Uat+U>VCGvDpcuF$i#8XAJ$y)h{A}xkM!JvPV zi~LkHGZd-3(c_BpjJ-bUehdv)&<66s}^K_{B}r{x~Z!5*;}LAYR##OePH51K9eG^h44iK$yia9jq}fHNm7qQo6qAQ zm{{G>W=WWTH;_iF-$EiqP7eL*{$a2yz~PZim`JgO&6ubt_sKy1Al-+y6o}q=R}0r+ z0yT!azyFW~;GX{i7Nq=R%o%aU3jD&x%we26^d+XIH;aTPCok?g6q63|Uh3}gx(R!D zBZSc1XAhRr$wdl_G{x0m$~uqC^3(HRj_djfcT)z`cJ8g~#FNHMJ{+o7C*L9NxgR*_ z)#OA4s~a2EEi+rWv&>!?HiaVRB|;slK|$o%rxxQB(FR2=6zEL$yFhXR)^ ztPEE6LII5snUC6!Jr@6rGg};XOZP*fb?AL7#;3t=rtPin2h$O7H!wTK$Z&KPg-W)Y)V}fNuINNWG9-Paef-i5*TyWqqjOhugjOq zA}x~;?~YjqeCMh@pbI!H&c98haJ2>tm654GjrH|HD*^{r8)IEPCYn#ZuLlmN>)kGywKoEyJT>gj&fA9 z%`oVGz#SoaR9R;%ll`0&maBa`T+Sgm6YX#Y*{`c5HJy|C@j$0VqKCcw@IyRr&U-EP zu-0$-6sixMEvh}hAvt}DqWm`vfgI)WEBZX%Kw^t^fB$-nwFK7!q{hX)4X(N zS9bedvtoOq=zy&9v+V)Y)xV&mKl%u@D{;d$u_a}lltL(t_7jQr<4)#Ox1+AlN5wlG zwh$|dcW0c>DZ~noHct=|=y8N!J?A^u%!V7o-f#COZdUq)SM{~h7|%~p@uhz-!euu4 zZT23)exRp~Tilwg%!iTq(d=hytr8U>1n&vHl>u2&VLj-Szy++xVCepxTqjqre0v!s z+f)&EN;%Y~qovW6hG8w1OP|;hpQ-Q#uI}yXY~&p24Rw$=84sC{R&S~rPwqJw+;?C- zG9f-oHN!ib}z~IF~tM)&vS7?cg`E)o#X-p#qh@9(Cd|Fc?!r6H8 z=amUhxoEsIW4k))a07fbS(0hy0i1yMxe#x`r7Jp96NlhM<%rYsn9~HT>q2ZOTo2`Q ztQ;;ElPjH3J0~!*fEZnzL>rb(9+5dFFKIb&Ck?AIWIVJneM1>QJs|`x@;TOYDq#no zZ5NKK{rX{1P-C36+!SyNKSsbsvbiYd)CyLWt!j=r1|)>bb)3GGCGltm6&2;3mxw35 z#%^%^M=1VxSkCE>y^p5K#1x^Ho3P{$YT|b|m_7}Npf6}C4lgXU||6*^?I%xvZd?WA7dErojFpnW#Bw)p}9X%uI(}r=)|& zXA{f9coikwZ#=<&kE7h^Kh@Z@J#9<1J&kE9ZLT91JLIa)JSK)@9?9(k!I58MvSa^Z zOctwcv9O=(iQcs8=H$fgSE780;5+C-B4M8X+sFEkjt5=oc@m zxNe!H(%~vcf9n%solqf6Ib2O~J~Tydw3@%7e)JmGD{EldA-Mtq;^%WltR*g(Uf4}l z7X&@F>K)aYKGu&}(QS%Fhlc4d8iWY>OcSS}9v<9PDK6KL^nbK`Q_bUC$%{8HK@w2< z{d6F<07lK=n5ctg5Jxd@itRo9A(g3$n_kd-$dp_Z0`o$>H_ybC-Qgokh`P{Vqx12d z^)i%L#GDk)%ilpJAq!^FVZ0nFKXX_s*D-roB1o9h-kB+IR+F~`BFD`#kJwvb>F^-wqc$15h=-$@G#KuUQF-_eK zF#g}|R2%3S$M9B>)GJJ(=g@?A*=d{o(09Ijo_*C1gw2eCD=wlIg@(?3(kN`o znPX@eNi;V8tj3R`Go|53{dB7^>x*H?S8=n8MaOsT<(RvM*pTxjjhWO{)YRs)MrK*^ z%M(&*U^DeC#gmDWvg}VJSd9y7lpqgI){~@S_<7;bAh=i2*OuuPyJN9WRsLi%Y5s^O z#~O)N!zs3DU-DnoupvZUt1K0%_jSh+`t#J-oZR4FP=#|3qd5ute|Ivog&80qe(A=Z>^v{zbH5(~! zkhtDv?yf}^ZZ;+5w4WcNkhW*tknYgE+HWETU$2j9q^ z`EM*^$`}*b@1Tn#`(j-F!?w499G55a_%w^xr}WsVPXF*^@9bH;Q!PfI8z#_5gA?5+ zC)ZXYvrOu0DvNV;`Z$cU(BwlHj$rEqFiUgKnydt&oIgRlRumWJ1l>{;C%xw4&!NzQQk+CW#O8K zx~?dN+uEN!nas_Xek)9;DjucAU$@)O!gurT`Q&c0y)B=xdjo!I=wQV}i1cs?M5i?B1} zoDLJL@{Z=02?y>Fk^8q`k0lCU=DzHSw{#MH#1Ql_81 z1D7TC;Dp$12+uV59n@RB>~cEdcQ$mvBmY=My7~|^m$bhv!_DCTM0xrk-UwW+FdHh- z+2yh*^z13Dl)z7>Z9xpEA7`)D2}^A(LOBt~Qk}$uGOW}OCSQqPXQdqC+hsi07~e?+ zkPtBcc>Wr{n+Pm9>G&Sb+(_Doiy6q5Fee}XHpsbc9@1}I6~a=sA$%XcX+Jz9tj+hZ z6qi0S{{BO^=GrMG+Pvv2NpAegcGeJqO0Kmpm$~znhj0FL%}kJAXdI=BVA<{F z(v{6EvieMj?R{MGG=Go@%NCFI70InyBgXSji8XDqsEho%7PpTmRx@!TY2QH!c0Ww3 zjBca_QbvuN>XF;-@F#(}b{cq^bB&9Bi`LPL=;j3s!4=|;Uf~1~Dy%O^|KSz02ecU$ zZCq83ud?xcI9peLnQAO=m*2H=Xykd}Z|f#zcaE;AA39#FF`As5=B0QOp6RFws-#!p zy7h;GY0mx~RGryTj>7GouPSjgbJ4cLxxrvYf@_OM&br)fnu-p|f_z!n$iB=2;8{n}d8y^){cLJ4yE!?DeL z|4e)R^^G5;`LF%>Yd`+T%>H#h{<je@Vk%((sow{BzIvFYWOEjCL5W z7mX}J+8-b%t9{Qd9lS6xDV&*+uIDkS`bb7C)fxX-vHQx^(!#$5)W47ai;+kmY9NL37CIG=&8q^u ziyauTu6;SrZf&-XIRdP2NILemQKY=51?51h*mPdT^35p!_8ZMl?&-Ml7E)hQry7YH z^yNB!u&&Iwu}2+(6lgm!s~U9DpV)q>7A$VzZwoQ-#7+;^vTG;z_I~Yd(oDJcio!$uR%Yc&WG+r< zVLm7|B@hqs^hA|duX)xN`L|2)k8dvTTGm#^OQH$O`K#j|3Y%feO(3i0xy2TEjp0)k z+oz*7dnQ~9tTqkTT#XdL!F;H+$%&I1VnSMxb-Uw0 za%t+ z>eI5s_Gt%49=>V?=n`!h4*{w@W%SZzj4f(D2!9r9eccCpZpYA}(d?ac(`)P}$ec~r zKt-sgl~H1jM-+wq%zP4Mobi!GkAuGj&JzP(9AJH+O}Bq4Lf_>vBB5B5USO-U+h3}} zbQG`77}yr0u1E?jSDW#01?$w~PvbrmI?n-A$k}!t9C4h`E13$ywd3uWO7{N}YPpFfMX&f~7rV z*CB(HFEQ{rY2t#j3Rf0`!ODI#RtT*TwTTzNY~R63Ak<+8vk0`I3j=>lemr;{5AUg~J_ zcMc-6BQGUrIByChwXz5JD*y6EZ5kQ~kRFn|eg{2L1j^Fx&`(+X!@q_}EN;h9#5LM? zk)I(Hm5}^w%gd9g2J-X9CHE|bhADBF!g+CBn0sQ8tOI0m(amq`ot8HZV))%^ zD-$o@TW42$e#S~s(1{Q(hnqi^$K}m;sa~(em7LU}wCOd*mns?%AxD%t>PECCvODd3 zHLq;io_;wHUB0t-M5YFUJE67-;gBid0iPci2XG3N*VK({!1djw-pBJ&r5@!Zhjh>L z9}Eg4u?fd_=YolQ&ppVzh-*5QhO1$lOR)6j??2pqx-e1rF(zOvwmk89_* z9wu2=Cq>@kjq9W3DYCviYmp;$mt8+qiZDx#05vLH@~dT&HoJiKKgjm$#AUs{8|R{ z=@6#5!OBJ;TtU~f#GaWIh#ER1vq{P24(42;F$4>@rYM;Yr-7*(o9S_W+wK13a1NBi zjfp3vRlaOGm8oY1^)HKTj>@XsP2#>42V{Y=8~6if7KvqB&^rLKIit-5Q#w6ZFizo0 zTxFrg8(J*Z&2m9w0p(k}gzCldMVm9V)M>noyW99nTu8#Mc_P0H>&o=saMjci8T_a1 zqYhaS%V(b78g?xCB-b}eI{vw~wjmDuJVmedA%kCrt_oKgP=myb26oL57mkMdpMf?j zlw+&M<&s}1g`$h1&pJ5zeSFdH zANemE%v9ryJy>=q?&1hNg8}8B2UjI^NV6c!d$-#G zy{>Jcj&UFCdIf~9%hsINj?&-q-4-24Art;$>3R9f&D{?l80$cR7Dm6zU;J$@<6nRO z(+A}r@;(0(d&Kp9;y(t46ok-a!3yprxRy4h%NW1#ytPr7((>AoSk3xP= zw0?+9(o%x|OY-syqR!@f$67E+G;`zzTy&12#@F@s=pLDW4>PxURYNiDPN+7~c8p)Q`= zBXWh8#ZlLZ9@kqSRZS)~8uhS#B66oNrXCdK-l*`s3Q|W_pVs(-dosO2S06p!*+bVV z6VBigfP`mXl4FbO%^35zMQZXj5B>K&WjZCuHREELbvXYk0q1>#h&h%h2XwHzchGlGBIBZxEQWbg7d=FqN%u8H=9DJc#JW0l{uVYd8p6XX zPIaLPQ{C4iHgG*&F(=Aa-hhj8-rj<(XXAw*OTSOi#S|!tRKXn}#!*hYc=$wMY;;v> zsA|PCLt5QMUF(?d(H+@y$EAy{&FomQO%AnnXXd0HbsFY7(FzCFn=>c}&>n*hCLky=gee@0F^f2JG9+mybmi&zn z#I$D-*ut{M73l0Yw4X|xkY}f46ocNH@1~b|i!N#G?UguJh!YvqlGT&?|lr7q~6=i*KF2b?RQBlXL<7MR>T5c z67>~MaLfZE-w(nhR~CN4EenY-eL1kNah+M0G}mUhXY|=vN2pBzNfi8`kVFI7*TF#W z70I(A+O>J2AbZw6~XW#1*Z0?cZxG)8qJLRwwAT5J!9N4pVWzNf4!$UYevOK zJ=x^5(DO@fS~Dvy*k{o{LWio-*DIe={#acbW}(g4c<->hgCmEPDv9*o#=xj#%8(@$ z$)wcZQWF0G-~DsQXC(J6UiFQ=Ce)j;lse$+Xv2IV&PW`r8FoI5}qFKjJ%%cx=_dhkU?PQVdYkvnRds^Tafd_DA z_1>11t#yD(Ht#zKy+)1b`Z}M)+lTa(&(!I)iCMh;x9z?AaNn#mAN z&Z=~9ZhCvqKJ?|z#dz49UZik;`_?rgYr^&D1plGgsnfH-c6W_bI2|4Xw9KvVT5$D; zY{poDuE>_~4001io_=jC15&Aualk;9S~>{AIFV(NwDWXFIMBc{J*L@qFgl0*cnAqK zmPw_GNJc)6MGKV|02x4F_!Qlz8JRS%Z_|9&ZLOL2bt>iD0+)nL`_SbEl1Q&#=;DWnUT~8GYy7R^tiz2yVLE67ZtU{y1BB z$3^3+1@S2#E@Yumd1Lj|Fzn?RUN!>lt1A zL~7l^a&&U7d#*P({(;urN*3g8v4L|E!Ca!6d~vC})Dc`%ct(L(JpGlLMpd+;{TFvo z%0XDA*eA#Qnd-*`BrhXAsWgd?MSzYvej*di04%&b1HbsZYmQc1SPFlv}=z-CWy#R%F%(I>H%pt!vi%hzi$tY8-9WB~DdX#MCU3?I0%h)rZ(9~=hYSD zP$i&&ZZun)d=>2}@|Q&B-U?yPA=p;1xfZ z;`XLcp>$1~gP)13oXjx@9%7wXWZuf6`y!Fd@49bxV-7j}QiDixNTLy|G)-m}e3w0$gt2L%4C!QsB`(Qo z+6zf{e}uN|r_9KhqV%^Mq6d>8IUg8WQ&;gOeWHEoi&EjT-uQPY^~2B~%$ZMe_ELb@ zt1`>Gnk;1dopob#-CB!n66yLr&i4DyOIx#%;Vj3c5bXr5XAFdu&31ZCPD@ntRor6h3y3)fD7%r~I0l z=6E9uFcfU34-svwJuoHi4+|0?@t5D<@nu7ibP2_VrHW?DymMeMKx!{e_wwhzbZ}Hr zi@W%I@Epp>;;c#1LvZzD=k#EofbHM@Zjd#f(ad4BIVTJFb-? z_}2V0&;Wnuj*UdOOgB1|LXMQ;L{k>nURl;@#JgwgIg~8cra$5pq8C)|GrVf4!u?e1 zi~Bqhf!*GV1l&TC)lmp5bFxxDY>M#ixM6$32N8UZ-P;RFo%1cOgz3PW$;8j}Uh$x^ zan*RSNHv?gGvhc!(9x`u9$5FP z2l=up{bB=$expl%>y|c?;dBaM2^!TSZY`|lpY2n7u_v3=18>*-~zUvwVQPD!?wNteI4QhK&=h@nD_cZ-dMU!1mz=I#~# zmrEe(BE(O@+WMjs5gd#iPZfI(=U<>y8qA+`sBWmcI8o|YdXwDBkC&+Kak5$W?T7DY z;+|J~q<;qy<6W@(Wn4jxYG^ZRo$_J${Q|A*u+5cZlMkY3qm)7ZMD-B`#MT>)fkM4V z@ed#_y`NdDA#aWDbe+2xfewGh{`~eDPU)7as@iDqlVC@l)Egu5E}G&2l55Z*sn{5Q z7HMtX_ET;h>aQ)9*ywNdX0IQ6O`3Qmd=ygXf#^pS(}_o4`P4d}5J0g1(_7-@*`ds_ zc6zyd(m_E7LRngd+Pq~ZvAz+UI%g1bHP6Uuepj&KRzB4Fd0fI|vpH@SYx!{=Z8rDL z1`TuT##vD2wnNH1IU3e#WL<<)UW84zrP#!8s1WdIUAZ<0^8NC21V9acf)$)MgB{-= zDdQ*+g_Gx+n)I^0$kFom&{u0)bXm-(Ez_6+s*ewWS2ZmZrGQRmM+Q*_)Tdx;DalF9 z{3CzrcCR(@)n`Hu+HD8#uOTTxzS2J*4hUwn&tMNC_eAV`BlNC)dd4W(W#jBtmb&E(Kt0^?h>*>rv4 z&}R92SxKs*3rn`a7hX`8{NR24fc@8((rvrUmDA7~(WvCLISYz56s&Y|mai^CFGAPn zOQVe;>B?gjKIi9EqNAiGViZ7~T6cr(Tbu@}DJh~q7w36vT+C4#;&_p5+fqrbK2ga^ zoVMI6{pGhq!{q>_oS)f_vIHryn0Z=19{Xfg&yG#%JJq{YSNFgP@A@zAIO=bqKbq+6 zXbir(AaC6YI@5LM##AF-9y{9-+erBa*DI1bzBXC@5wqf%#7mt{_|WlOYrswh+0!QX6+FG5TmUvTE9aQgYLN>}4o|0zJ=f8LKh@Tg-RZWj|pNV$k}LDC8n32{w> zTECj%LJ6~;lWf9AwGp~w6WLjmWSCLacr*Tajc&%keCZ<`C^!m%O1KCMO-XqZoNm}*k=az*uho;a0Hs`ow5|DTh2w0ayloEabQn z_&Z0;O)CdE8eNJu?H12yy|}_1ei$|rn7)h~Thu1k&uniN8WZNyub!x4Q%@btJyu9g z{X9hDZlvTMFUb}~cYOgv1tJ+s`>ETo>(ogd?ROC~AGT*f38=a1raGW_Vsi2=YEC*{ z+a4NbGGOy@IOE$>rmOKB`j`bm&ykVR)K z-J)a`xgVMks7t7jO!@qD+@Updp^nzj7?B>o?+#Ia;&Z7iS_uT# zK{#<;XUay0Nf*V2>)&51vt>k^rc2%pF-SBn6r!g)iFsfui&dFdAviGqEyDb5EY;5+ zCxB+f@v34xqxM`Cmskyg6g1rVOg^q=DBhT|Tw00uyL-nubw8~vDnlAWx%U=tI3}!? zQIv1bJd8*jtxDcmMJ%tGc-{Q0Rexc|`d*8~i@Rq48z=Tt8)r_7;^KhoH`ctLg6n1( za&YOF$M5-dJ=KR2?{t={K6N}-oYp329R(_z@W4;3WH6p&DHx!+7$B#N)Njl_<3c{J zfiIRrUoBMCBh(W5GW>XJZ{#+^G&@AafsVeu?mr#af*vo14UO0ac+~(J_1U4B{PC^r zSE4+%bq3-Zk6cC9lf)3VpI>&I$#MftL9^iw*op2mR14SIk#v+fHCAu0`ZZ1W4p8bW zzp7AVv}|>pPtoQ1HsG!-#V09(JE$(YYDFW=5}LQ$LOnj%9Uldk`3T^?X`g ztMa88STaA*c^i1%ZdR^t9L9?;#D+1+F|{}pI>cN@`6MN;Xfk(kj5vI5%Arf(zCrt( zonD1=bzwU|ZgwbC!6Xaj)RRGjr7Vkdn9-SXJqOJmoa&w_>WL3(i> zHW3T)_W940K(k+sCkDTRLKJ)!Hg)vJ93EGDiAkL}8Dw-L^~faDofPgro$Tv7Q?C7r zol^+BpE#^U50gYWMeZqwOnU1-M{7wn81Au1-He#S!ywhcy*1Jw^7C&$YkTpwToIy9V!0zRv4hL-c~$60?=C7%kbUbYZIT8dkMc8967A zEI+8VI9z4}nfo|#!AORSwjwKLj*;Y{Zm{s@rEzcS5Ioe$744;b2vz*0#{+; z`-Y5^Re~YF;OuoKJlioRF0|DXss0AF>C{0f5c)yRe%`MBG?y|`a8n-E-+b=uqfJl|DEvVch%Ydv|PJU z10asvfA<|^UilsLl^6ky$9{`fTd<(*r)O{*b7BErx_Ou`Jd>kUVfUQY$mO!(O#0e_ ziGlf_270s{pp@Io#Vk;BX$3Q1k0K%ZH3u-7nZAcwxdMUZY!veWSyToa*`u))vA4ci z;4~DJzk^g7ERHdmC=EPs1{YA{YF2iF)4^m>%N<|(mJNKhi5Y020Pv9wi%y;51#r3m zGucWgFfnEzlxjL9kBb$Kt?QE=h;hb$`Qe?Fid(6*5jl3eEd&e#xXq7gkh$gcZ# zi(B0&fRHah@hjnq4jrDiauR=QPJCpq3EW{kFu4kGR(g#=qe-V{bL)6~w zP;~Bc#hwl|vyY+2O5yTF_TEa>`Tbtv6OZCAJBeD{8Zz@xL`duWg=Y+B)IVFDr^gcwqf=*eho#fOHzzhl-PwYg zt6>yf(prOqom3U=SOuj=X9>A&kfcwRGK>9D?0~$UoZ0jT(kFwtemE|r2MG(0<0zZM zMWajBQXf~oIVtL&xM!Hx_}UzC9O%3smJ^}N8}LaSMam`<7%^CRl8%?pT}_@us@D`E z?e)FWKbBGD@jnB1)iuTT*M7DR%yvtpGcux9SP6_R`@?~?hFfg?iNEH3?)UUB^fEXK z$rhWjiLs~}EHv>=UeXRrajBsDbp9}W#gQrVE&lB2wnf)nn$w zSyKcVn7GxSf`(KCGwK#f>(Ky1#fYJ>lJK`9l#;*s}-fO_^DInWNWh3u5Uzx^5j zOpeYXzk}w30LD|$ZW*BJl=>8fM*irk{U4d3V3bG4vATPY=38alx1=@y-%3^N4+o>d z>AtVKssU(ojUwC(>T%@av2uv4@sk8x$o?F8Ib&(GyC;SIfkZFGJyX-`ju@3F%m+!K zEH*6#SF@AkB$SByw3yVoJX?{MBtkPgd;}4nPm%Tf8L84Yf_C3TZ(ZQW&D6$QsGeAQ zm08@XiMbrzP!&R?_e%Y?^=r_Ai$SUU1O0zn^7&!ziBnk=?<;WU|bDdBC7>TxVQ? z!WqK=(8^>95b^xk9VnWl?wF@4);qTz*3CA`wskWg-|t5= z9ZSFOj9G&~QL1I@Vko{BSBH_X=!}Vk0h*L6Cs`eFr^SO&&SMc1<4S~InUTr)!rq8% zNH)~7sKB1}rWk2Oecf4d$>i{Sy%$*$Z*LWz^av;${9cHDlx+}v%oXF*;c3n|(a6o; z^{H~w-C$z*DfY0)nIApRsQP7-_Aml;R%@o)b9L zU#~plUPr{HCpXn-krv@V-3zjFWHQW4r9d|wiIWT|YLfK= zGlL4wIZ)Xl@((3p!|x5weNq3yd*Q_EPbaXeH}y*^4|7usu(JnW-IN3TMymph*&|Nv z+=WBs3C3_RbyNJ))`w?i7Pdo0;Foi8H{>HLZFAP%6?J}D+%<-6YvT-*4DWNj^mxI* z@!D}{(BjrV+}Gc2w;vQXwGbPvpz2!;)6pK@-l%FaFk+6BGOkip)iifY1&36Mio$mY zPCneHPW3F6_jO;iAcSPdV`hhf_q_N+CRZ0c?9LSj%nT%mk_%BVsZeuBMsY%CfKGNl zH~I=fU7Vm1yphjai#2T?@Eg;n0VVUn3bKdsbDFtSCH1l~`V_>}w`u2tF)OXnyd7h! z!-%FLdt;f1iqg#7SsgZG6X7tc?t49fM9GuN2tHn#xjGriBTTl;wSBjP3FwT^23#11bG%I>=iKPhvIY?!Bo%B8~m zMwdJ8ge*f{8LLaiiD!q20>Lyvk6tGqKA=;%uveebHhT_OkvSWtx@q%ER0b_m&+YLG zu&yO)DD#VAu=T{lqq(jpi5&!p;=SgQz?CQqXo^u7*l8~iAK{h224Scl{b zGO4}LvE-%BPy4d5BuxXv2_05oTYx%|K*pwX_vVURy`CNz$a;bZ5KK51W3M zq$Ab8Ra?V&9PR@o32>IZgUDG#NQYY|fBXpb2V$2xS8f!tSGj00?bGodI!G}Krnp&^uGif{9;P~??x~G;tY3sMMC$Z$YaWH`D4GX9RLu>mY$=I;PoEka_QJ-o8K0TO-)v@hJs|;9eiNYsHo66_1W6(*LO+i|BmN8jvPXt zOQ&6`7U7xEX|8(x_*#}`ZglI_zWQg>N+EfnnwZ>y1T`lc8=)CQubx$Sl%tA6F=%1o z`SVZDRoI#T>8j!y6;TuS2J?&quJLtng75`|AAw8!D>y6DRHlZjjFmofYtlp_rh{wZ z#G_+vr_=LeLJ%!KsC$rChf&DA9QX5OjFJ0Z7(ulK*b=lZHXxBdK`M;mYc31Zx;?=K zEnY1KP?p^Du?r`FDgTq>+SP7pYoRi+7hiM!Z1&1(Y8gv&c5!CEZWB43aAFS6iqT}5 zZQv4T*L?)Q2XD_1I5*1BQ4~4W=6Tm_xdLyt?AF0ev?^+m^r_Q4RN-mguz&^v2w$&Q}(&JoTs;RqG z0k^U5?e^Fb>l2&DO)Kg_FI&l9Tvr$qZl=IFcHF>hhoWj5WKLNoxC{`FVF!&{yLCpS zQD7Zx#Zm>=eF@Kr`8bC*3411+6#e1xB{HHXdgnirb8ykXNk*h+W*y#4Rx1jDvGT>K z&1trLm!aOMn~nqK7n3CgFKNNw(Zkqx6DBx- zx$_GR5v2G;l7a9!-Y=FCkKFiGIJ}CRT=(5D%axo>c{A(YrEza@Da%b`#nv?uBrjAD zs0k$TOHe^h&$5?w?5kVx`5V+f_+h_yNYkX1gN)U4OSMXYyJR!dhFL&vLETecm5l7^NSMfS^3xTe#=dPyX3I}X z0ZSRr+muMg0M>mo`F7Tu@q@OxJwVZ966DrW&4Atm`XL*}%^!kq|4EJb|A5l|ubScm zYXleail|)I=UMe_K2Nv;`|p zKMz;NvP+o4U=Z)ucM&~47Ci0-Tz$=7mdUxMnNh2yU)&Vdax|f`T#ZNgpIuD&~bA#I8C``8nQ2l7-SWx zhj>!EKEV{Jw@NhZu({BHW~P+y{u$DN`@0^?+aYs5=KGf6(o7~q_ZsY;lr6=>cy~9s z69*jHyUvilb-7jFNiV2#?XiG|f1n?Y)24|JnW)!#XQSRTgFZsv^I#6uw^y{ZV69XhZh3CN0NXzU03`a5;S#cOyvSyT3-0O~cww_Vvg9Sws=09Uru=?=0n1(Skk5k?YyMvkJ8r!ssixt{wL3CigJz`Trrr=m&`Bd9z?P z3d&~@xdmn>IOCn=?K>f!!SRgs&%hqfh0aypj1*${3=Ziw1>PA({cV9}h!9yhQ}QBv z1EA-h*ZzsTda|Z=Qc3G$H^F|I^vR1)x?_a|Uq5{+zL)=pLl5fDlfbV*QQBhdw4cq= z6b1CO%>~4`Qpb9{o!aYm+F#;3ssge$7Crpz_o)95Je~cw@C5(OTKsQ|w3kMN+j4b#&aF zRzte1*lRVI(7;CUn@n~j-g(^JOsu;C`E_(YHzim=`<8lKR!!xr?htr8UtFA3f;{+2 zYi~&z@}3xzHkN)N$d9D1qH>Bl%w4P!T=2ixd(Wt**0xTnfH9|=kC|lCfc8szN>SwV5i?lS^fAqYy9))5AX8e`l%1MxG6kt<7^mausiL_ zbVpDo?83;kh}Z?8v84o^oohqPzC;k#1lcx7KHg@5D?qX5A(j(8yY#8M)Hw^T-;8$c zcu+fDMN%=MAms2c)umNeq1iB~E>fcP$|}HxE#A%zQlj2^vBf2dmhJQ|n(UcHxk5#x ztjIb&c%>vBV1#=zg4&R=8XgLli)$UFTX7If-fO(D1`b8vMyFbgmK;-( zlXPWSTGpgrA+-B06?EGqSw%p_V(QzM{?9JTl>Ul1KCB5zMLIbwtn(E%Tg`*Jz9LbV zLtodhXZVTvgQQ!aTM_e`8F^c=#zB2uB?i3Qcw z?);mzO9X+VOlZwcQSnFESLc%0Y-D1H>!H(vbW+#C6%79^+B)SEeD=lre7c_Ne9}c> z1t!@j*3xPZJvP|=m6J+r;PR$);T>spthbFT#1Qra33EFZj<|ZzHs8dXc!%x9o7f_l zwG-ib$Z{zOc;OP)RiM1uqiA^8o+i-@)=cE!rsS~DIM+|wExqzZNyT@i<`UjSRdSPQN_iUHS1X>3Up}x6eKN2t#*|c|4)c%C z*|wsD)1E5JnwQ*zs|xI0R|U_*Widv2SWzG* z4CkVoP*BN)$s+Gy9#(uYmyLX+9%WqKP@8C`jgrx!G{01HvptZru_;tW|1KlJZVSyX zkV85Z>}5AM9ZnD%hR8A5Mr(?Ar3l6^lkwHnydl*zfm906^AWOX@zVtf?|eVh3zpl< zM8>`Lr^JXA=c;EdY}ZSNjGf%PH>3{cAL6UL&R`#}pOC4?L&bhn|9>)j2){uq;{x3f zyF!JCt@b~-H2?HF=-(5q|coE-%kyhuIED@VY0P217Wh@)e)j-?1A0UApL^g-z zfm0TvXtKSKiaG|dcib`?OxCosS?HEx6Hj;iY&I$pO7|dkj77^qU2(7N}66c=PmwtQLob=?3<6ubLbhci%!BTjuEU{ z3Sg^{loo>2q)RoirBzV?po3-m*^IH2!-TjPv+-vqF%NLV>rk?+wA;ZqGEG)Fa+a6| zRhAYbxmlWJVIw{F(LE8xaE_cs7(++ewV7%SbAShc57;zZl)TNX2B=9(apyAW%J z7mL_em{Md?PXvxM!->7ACfIQd&IU!sJlYf1+UG*9gWo!QHS3XQ)FSHQPvA~Q59izS zmmO0-UJl!B`10u{M=B**Wg(e+{_^N?fjicIG{eEE*zH+WU4$5~hJ*E6QZ@(tp*>WI z&HsmyoO0NayxESf7%in)uVbEe?#}4e=}prIZJ(>pb&JPdDTv*j=V-XzdixtO$U9^p z2iZao8|r#h;Pe7p+|1>|<=K+en&e_vJvMUZ5xD@zYrV?-*9m-Y;uEj29&%GC-2ecN zyxP-)>-RhR{XDn{u)%G&YLijfp*oa1iW0(D9tm*?a)@oMx`>x#chhWo!I7fu5@~43 zuDH8!Ki79%No{;b^^;~jF-3WwZ`KDta>&Fo5^tEFy6M7zkw+ya_>~Db(K7V4Kg*<) zFOb$xIS$S7h4_7G+A01N?A-@(a^wGy)fjQbjOZ%A;N1W_3ec%ls83!rw3)_Pi3eM7 zyKJ)bXuWw%;q?K&>m*>5?A^8ViQ%u&)#MJU^SW?uU`nWin+ksgCT@y}3vrL}YB$wg zPx8ydsiG@;p@qSUKm2u3yRvjYi^SN*ELckf7fPn1X;_Y&n|WXafrc2t4=PK+-lWRd zV;DeC6MXV$e1Lb`7M2O4WxRasC0>LibO3HMd!`Ce3t@lHnOZN zMlIYP6~xv-kxks1LoA&&a*sBBqREMg^eVPBowKP!i3r`XVx&H z6BjF%mKa0)H!DT#xal1DQOxaap3rm06U2;C?@3InZ0Z_x{?O4&y5%D3TsfUGodkgK zb55V?d5WL8iEu&uP(uO57JtuQAbAAN4}a=_O7Ey~?7Pb?wAoVW*xm0J*7-?{viHak z!?%@vkNp}m(Ss-hIrmr(jW^0O?FFglsXg!4Hd6YqRV(6=R&)r*eP*};z@n9t7xo#$ zee%SI(v;_5ceiS6^nJ=^P3lQa!Ywh{-r%+H1GUM~LiFs3traUbM-D1-0T}&8*}6*C zX{30F?48?;;)hT4D&2nK28DserOV@8GD~oL7w>V}U$)tE%eU0`8mX3ESaK}4 zP~Zm^7})uolFr;;!F=}i@FU(H4r#Bv{TNhUC>-?3D|RMmtm!;3Ut#UOq-ca^W(h!S z+5>zkGTeAEfWAoh4RAJwEODQ)0sI`)K!03R_CMGBPb(jVbZvl3<10q*B0`Q7|FOyi z8+GD@&3th#kK-D1|K#YO35+{Sz|z}??))r)Zq%Ps4jJ?WFx-CtIS9hZqfUIrN?BnOQsb2XwvsO0 zEWalgYq++@tGWC!FH_>dC-wY(qZWCs2uiFOQaJ`Cs>DBKwgXufC&J#_G?c{@ouE|^ z&b;-Fk!%EN{0ZGxou8XLC0xD4b8&TsW%pp=8b&!rX*4?q+HH7itG;Te#Xw;EvzD## zfIvGujO2~*Wfp#v`7>-(YfA+pqDYEEc>eyth!vG-acr`8-p-$T{+(1rmRhm5ZS%ZiUM+Zu3j!zTj}U4r7eVTQdvF zCozns+0dtNBxQkK5T?L0zWZpa-N`Jh3oW_xB>NK|ce$ME>Q`S_(jX5<>}o@|k!yL9 zgExBF=|>v&D45C{jhqX&xk)`1FK;Rdk4qQJSQvGPKR<)77#RN$+~cEa{c^wXYW)?a zHxfZaAhss~vrWRmc-L(dVtvgSN z_kKpS_K2dpxx>pwumv(d?&+*Jeso@kC@DSr{jl^7@k|S+85r7~Mds|j3Ov3C!n9;?o2;DVm0+vRZQG9@R{_pDnQ?$Vet~d| zrTeTURK%YX3$?;*1f`6sl&4&`tx8x}L#7=-E56KjpNfmz@8iyV_^E-g?J)*}a8 z>q}E%mzID*q!mYux4{;aqgjz@OOfzKh&TYQIn)R6hPf)^EF|=};fg6TKs>PJ7E4iHwMu*$#uGFbe3$zs ztfC7*^ByHA+3 z{enCPCV%2IQT+bq?(-d-Nm7vuHdmtA@D3ua8W%+03++GlicFh1ha`j9|yhsQRrez?W) zyr1C-dTk&xSLsu+%kWOp3xtd9kFjTUkto4A!MWNL*2QOU(#0rH{FiA7G2-2NQH!+4 z>2$d3jT5_7#U5446vkUV^IZw8iNn||-*C1PQKW}9``pa8XP}Y;OrhO|r&pJ9G&cvH zK8_uBY0#n^BO5MKq;Thp)6DzvkIxy|t1DB?Cnn#t-g9j-l~q@(`tb6xhaU@4k?6!5 zn9XhTm=0B1bW%&nOZ zi&OEVY)?7zIY#z-F_>$MzLSW$_u4L>%RMlMKROEq3cY<)sHn4qPe(TCbf2Uwq@+xg zK_Si_$|sRu6FAiPRI?FP!NivOG!&(93#og5 zK3kh4xG{u!iI>FP?ZHLb=QLs+L7bTThK-ZaQiog$TYwDzSUnZY#u{I7&asUrD90eWT1a6!osVEaw#Bn9g4;_C z=9WKG4AaC)=E?(;#ZpwCSQ9?~!bo6s55~+LY(*pLR)o8Sba&e4w0nE5)+x)L?Pd^I zw!fEW$GUwvz$ZQh+02iXUD}9kqy?W=7;p?qncZN@@+pD#ra9Pwa4 zJ3#{;L9;4f{=p(uH6O&*EQCd%S(f9XaRMt4x%Sz1Oa5w>>Uxjn7-GY?!h6*TI-(yj zYTWyoF?@$cp2TjEZAtvHi7+GNV7D-o3NDE4Kb{cer zxIK+7*LL+J?`#Opj}M3=;;LFoC&Ui`Y__yGffTM%s7FzeD1>?;uT<|#Kg10?&(vUL zK47EKI>$F#3*z9RGD2^3q)(@|`}0*|lp_4>8P8lR^lTu4H6mwzuO~TPmzzxWJ)fu^ zSe|mY_AWq+m3WnCgUnyXf5^wE&5snBTTwD=xy_|Gk%`8q+JZle6y%AVmfx$%d88sR z-WJ3>O|u&PZS(VU9d{iJ^dv4=7o!|1Rp@;TzEIjxPw4&y(jSFX_CEr1ObeK8W|bwv zir4Eyc*QT7Fza_hM8dp=y)3U>eA@Tt6mRz}?7yK$*%QoJFot)U(%C4AAKdWny{dEg zIdXaU>yv1TjbbGZ{~ssKSH>D8&fXW#nKtz|t&Pu5G%0rH2=wPtH+JYJu9XZg_v}U9 zZ@4oX{V~7@tG?0_-h6v>&wYm$sLim4g?V5h$95A^sWFmIq8|9#zt2n=fAAt_+JTy} zw|;)5jq>9p<&s|5q!Jw#gL!)oYvnIB|Ljy1UA#YA1ywdYbX(YXB6g}|wpE(Z@~8 zt61%2zZ;7x=!t^NEjdWmZs^2_uMtN1Ix8{$qd zIcy&Gn@hp{&sG12bM=oT{=?1uzmdd-6{pheDI6UDL3ZB6yz#`((z7NOvIWkakqChI ztKzNZO;uk7?6Q`9d-_cHTcZgQa4ad5u||o8_-p8kn2ku}AS`r9|YPrkXVvJ^!5lG@ww{nGw)w#vBc z@h9TV6!SCxjG5wkQ!9a{Mxzn4K9@t$#+OCBY1dZ`f8Ok>xomtL#I4wcQ9cPClA>jV z^;uvZekw2lP?a6vQa~q3Wr72AS?{GA6oFH0; zkLL;Z^+)-?V#GeDG-q<3q$!+l&ajwxkp~O1FYr}Sl=gjJs`mE#FS?u7L>=(Q7`>D5 z%3<6K42LR4CYdQL6bIoBEpktdxv71ny2hw_f>N||=c>-SrofOr*%OFf>I-PntIifS zC2_d&1Y8rHduUcZQ8h7!yKbAabC?uFE?<}O|(bEX+B5e!%0xdOHx zdU+0(u&>?xw##rrrar>&9<7X`gtmRf^=g|n1&IfH3F>7XmuJhr+$9}IXT;xxL$UW2 zoR>HhJ^MNrc*?(`*9)>;j0|72>buO0IKMKpR3o4(KX#gDqM5Mi)g7OLl_D485RSUf z*2SZI&R`ANV~vUkM6S6&w4sDjMQ&OAE*&pFcV*WNZ-NNd&Y#a-f*VHtx%~T0J>v^D zQ3a+hl*Q)8^H%1II~5Cy=DoQOl9>JN*SdZrk?_PbB`8Z;uR=O*-hx|?D3D@oO|gDl znMg&FBXJ7w@XlnE^qlSs{%Za2qV#kL8Qvq8-X%yxRT8o;5e@iDm3A2`JG5Lj!I_Nh zmI#a=`LVZM9YvYAIZQ%J=-E)Ym+V4{1yi(^+TUH}<8|0F2WSY0x36QicRB!Kg6ny3 z>C{dhs#XgiB8Q`3=QlHvNgc1ebr$+Oy;L;BmQnw>ATt8D|VbYW4d7io=V%HeJb zI^LhRMUkmGW}NqQ@a@T&bnIODastWv&UAJT$o{_Z{u{3R+eW7nR2A;g#%EmF5QDC)GTEf=HK-flWv zcxV&uyzTLjKD+(KLf0or*h6Ht6|#T*&QtvM5|Q``4DDJ5>+41^xu;Xafv{Fx&z-B! z+@h^@wAY!hKAVZ@FR~VGtcZBZEEz%kqTtOXd85f@85g`P`m}wJ4wke$^0^)#1wFy? zxYJX>5<6O&Y9R;j6hZU zL-s|KTrg`6$IIxVOQ>A+cmlPf0&MUSp-ejc)@%3mgFh-fO4Qi*m`8^zo_XNht(kBj z@v2M*GrQZ6$hVfdf*?+b`L3|7MrJ2rJR?C!#3rEX-Q`D%L;!Ul0)JNQV7sNvXpv~6 z08+=s7o2CdIi&7%R;lC6Up_s^cng2VI-)G?Wv#JU#P(9n*(r_kB3VHmJ+ZQMl2Fg+ zPhBrrSr1;&lzK2ppZxJHfAmj!&Tn4!HydO^^l!j~fSh)EfPxy3?Q;k#oSg>?Q<|xH z73_>|SD2YK%c{FgPD*|-_$o_wvIn~KjR?f0vuqWm%v#)d3!S<`8zFtpX{3eC&fls| zVrP_brZ-kE&Q>g|sxy@66)2k$qTi#*(jZizdajRiy~sE!4LYOk(si_`ZW>eYEmm8; zkm;V(GI{*aWZy0VdjE5fPGK;yXgOyuTl}!rQsS9%M>-9{0ZNXI9ql|5IDYqdtGr(^ zq%vi$V(Wy#+>O51(N?s>bXGph=0nya0v>CIPdlYF{sX5iG)c>fnt zt^$V>S{sLrtMLLqN7SEhPTkNT0l_z0n1Ko|R%(#}YkR*Y-LPeJuhK*WdQ2@d`(RwK ztSRAXbEGhvW8?xj{}DafKHi>6W8|BcFH5^PCr%yHo&GA%7#2FHKc*mG+5s&TEc-m? zobuV!c2=aAYT}D+j9A}Ir(E5MeJ(Qd^&(Ta%tbN^ey0Fte=KZPTjiVT+gI37?Bg4A zUnlhehj(bEXrD!PN8CK)d(T%jYg0a|@8sbFvk$e~2=wfmUC7_rAy!7C2H zp+Npq-+pVUZu)*hW%b9%Hm8WvB$&yP6(9X2ywQ^4N z=WK6({5)Kc4AK z&9nT3<{HtYt6SpP_l0BI1iBaOTJ%qDeG6=rx1hbzahvCWmdT}#7He!t@C%fg1h9y< zP9d=M|CS>rD$V-ms{fuX|3?!4Q8yqeEm7^mYoZ+SZ7GiU*rwm==@oFrqAm7~n5u>* z<}+$l;?c@6XVEni{o5(%_ad*OmxAM7+~o=e7z9aw4gHs!KTi?p9j!X_T$(h{3Z9ER zH%>F7mK*J0Pb>qA9DLS$xtuYwXHCx(D+n224WJ=oT1h&@ag1c5k5a0_x;R)c8d)<> z-x;y*MaG%vk{s1y&Bw&H?qEM2^h9~X5kmfraZg;pjd zk{4yksc&ajb>97XnArYOZlDcmgu-L4DCF_jUa3+KqF9$H zqr5rrb6s5!8(~P|wzU2#XJf!q)&MFL4ej)x(WHXcvW282CfX;i<1Hz{ytn_sq)>r5yZ*fr^WGxfZ!vmprX;KQ8zTVkAaQK3b-crbJGJS=0%g99MMF-< zjk&u|F588vvlD&_U8O{qLdSO-^>zF5L?3mb!EEhb<5JaSe z<=)u--fG&+p?*BF;Z3`#a6m%li{}P`&{Rf(W6gFyqt=s!n4S94Ogpv(ukzXXV z&nVLBXv6tS0~)oyKi5e14*mslwG6m3jCs_pPHzejxps>*o!3`8V=> zKqs{S`j478QP!4Ci!q>GOc}3TDy6fzrk*H76H8!7d|72vCC{&LAwW?G&m3)aB({zC zl>Q6EKMZsOXo#P&cqk7H{sJX{`Gc=~B#VCv{%O@um6wSpl{czdE{*^|5Or>J5IFsCW)Xjez+Vjz1(H(3QTUK~ zy+Z&%Y&AuB*$JpLsqW?d1iXm@-6w!bFqr=WrA_<-osI%oCIB;syaX`_Mx1Z`0%0RT zfSv!>WdeahNH)s_WqtYvFmTZ;{TP2)_Z8mP3;@{C`vvMWV(_#a*amAGvjk4F#wI@w zjszWX|AnAr&W)P5T~0G-fexI1y{{DFPm4Z;R{4&-*uMOIpzMChGe`+z(9^Fpw3%(B zTt@_3Eq@^zOUOy7kJ9-5U9Pu({T%CyE~nWjOW<~YCFy@|_s3TKeGmL2yMI&3e>TcL z8|9x9<#$8%k5>Gn75`|(KU(pRL-uEHSNb0h>VHKKDmx!s?-uVeSzo*=W4LwtIZZw0 zlAlR9Bf&OMy#Id=*DURsGZ``bXsvQh9-v>BiRXwXmaO3aMV&CaRP5-%kDfKx6iF;y z`;p%k?e;`wl4$Q!5e@Bao+D|;zm0Q@>#agOEKIvaRcDM1?|4~D-c-JR<5dV>{&6Aj zX#f9(mw#X*2-f`qDFU#b`*}bw&^5xvKSka^TJT>&>~TG;iuB-56!aI!4}i9&)5P(l zY~W6SkhU3!z{4{r5r6TT{wIp6|EGQI%|`hZUD-~r$Ih5G#rLb%_;1b;Q`rh}yaRCG z37>5h^bd6%D2<&?N+_3Y~l$G#@ldYK_{a0-50TM#!EMQdwdRQ znXzh1vH?o0dMk_U{sT(fW#ci&&FS`;Z8@8#US6Z~eOL60^eCjfpRK-msG7~t1;vlw z!rdIhfze41u-QeMo^qVArG9?-DuvtC^)F=d9@GcjVG>HY&QG|mm2|2SE+n)@9LIyH ziBKYSX_i8hhWKB>S)lU*fmsXV{o@a%qQ^~K`<@lgbsH}iH;o+tkg<~zlI54#MxEzq zK=OJ_&dL=uZ_%ag+{}%_0Qs4cz(}L5*n6gW_0L;|2lr+=dYHZbh-=oOhgAgiz@R?6X zX)pazy=2AnVN)NbYhq;HB7aInMVUp3Gh?4|hED3mlH10MRoyn1_(t}PyPD?d@` zS9A>kO8`si^pOo#wqK;(F-3%Wc)R;XVVpC~vh^r5_WBsVmqRFTqE4;Pgl5oqCB@ecL;$DW0 zS&4booyLLQkJ^}o>&WET7j_{BslMcVENd+bdh0TLpUN#(>)>!J<%pB^BymK6G9Rw@ z$Lw+yEycD3wy@LdEHSyz0^4ynOh?NNwG_giCf$^9NH&~k$NbYKfNbC{>6AY=mgM{f zHcxf5QFnPiREf5EaMs}^sO_B*H>@Huubc4M6RrE)Xl99-3AV72e`%j5l zE0EA50ZzxSX~6NAN4`_~Zy#*}WfV0e3ocwq0JzM410V6nJUuo9pctTyKeFn+JQl>{+O)6*!8J4G_r#hEi$haVUw*8fq;E|h%P{YI@dzas54P@?|~ zE7Yh*pN1E$j7iO|PD?RRf7|&O9jAYnCsqX%tL{Bs+D02%-RqOEHM+>LBmYTp;RJth zpoJ2fv>R$a{DGpVN5ix7I5&xY@MDuP;>*mR@~p5Gk83cT@AqFo0xi-n|eA&E||+P~qn= zPKE@k#`4e26+*8>p7^$q!QAlsx=l@GCTk<2B_gPrit?PMkt|&fUPuY2t-e?AXl$Os zMuj<&DMsn`uZ_=`v;wW4U(9u`-VGxXzHAUcdKU!3P(o+&p2qI= zbfrWas4riuyj-(w#Gr{3FW1SJuJ$DkCE^QaBxv|&rv;B7(mw}Z+`nm@AFNF9_c z8nl`AmB`0YQt1rSF6{4`pH^BroOH;@YJ9(~ulGn^{}MOKCKt;%?_AR?kYtPzkMNKe z{v_VMKsPGxP$ld0`SqP<12>F>6iDp6z^bo0PDG{wz9_MSC&vh8FDpkG;A}9KnZ7IB zLH-0oix0+%k2zB9E5~_9_QV~>hVE^xYfF#$nra8LXjOB+|D-pWPN&3%@3&&aIYTfE z-9iQkI#Xky*;?J(=F5(cS#kOYhg*=ZwSmv=(I4+|Qx_#}SFZ#OQofBd@E zJWSsp>V0eP#a!A4B>@;e=u&oo@>X;UP8}HUXdwp1J2ZcRc>00lw?1iKOGO@v1nBol z#&k_~(o$i)@N|eEPeUl!S+JBq*UEUTPdr)VMrk#N@3UeyA(-bkwmX;GFbzHQIA!!= z49=OPWmaG)v!y{xo7WEJRbEKtimBa0^Jc&xIeqAP?SvDv@?vHzEn7@UjUny$hUf7|%�a}RiSz|0VX7m%JUm#0Q^>c@>Jz;C zfOSl_#MRj$A1bf7#=3^I?mM|vHI+3Lw=^jdnC29UWP3k`@dsB@jE*%6Vw2ix zxULQ>NE7RC&uqkAN;o~K_vx4E97dI_eVolSknVW%y&)EK=MH7?elt7PVhKUkpF5ag ztnD@s26%4?hXl%28&^VS!Z*oR9U!+~+YnEIVK3 zBnG8kMy!Y3nH}yo!{VMxT(xrsHar{{J zH#oppKwsm8GGRgwlOCW~X{n4D6gvk+M&@*-4nXYM4#^AJS3*D)w{C;n2?G@QP-#bg z6$^l|kiKv_w3-NzqpkKGJ{k1cz)c=!_ajU}n@ZFBnzM>K>A+v(fQyJS&K;Y$T~A98 zz+V+6k27e3|GCFM^7vnqIJ(02qykXny$8;DVs#+*3TnU{Xn^Rejq5d3tsz)PQf+`WJXb8{_ zLf%1rj#a0wwD+=JvIE=+&vbgk@}=4utOa^ds|g*NQ@C^5^Q1hMtV(aUg7ck`YlCor(I)I&Fc6Jd9R?Eu-Fly$j zHQt23H}Y5VFg+$$qQuGgktoP>y&F&a=cYZ$EDv)(~{X3MA6&SsuNN{cqHX*H#f0TJ5rCvM{LiVwFqxg!heB08y0Ec{8;a0`6zG; ztux^W*lhZkbxKb@z1n=1QacxNAhVvpL{Q@>B!1faZ16Ii4^0~>-4KH?8C^|SKe{J2 zBH!=(Pm{jQHnw!bz(FP8pqbEAY4mk%_;GG#ou|u*+M!b z8K2p$$&j?8UwW6Q?zE;cG>zhPa?GcoRhU4}@yGq|{4#po5{th*e|)pM@VvZ9@-#w7 zM;pBmmAUPcSe7&U!R;8qhtgZuTE_RYMv4gEdO8!#eT{AVrFRBxC5{m9__~OD67W8U zi!EYci$U;L%P~S7%%=W1zH8=P>rg8(jqWYBup6J<`I4C{u<1&C`lcU(Ycp+F}S?XQvl zT24#B4B69l&K+X97-T&WQ&k;@ls}*K-I*+b4;q{-nhf_bStMU<<=LvS1` z7QV~f$tx(XeG339qhQ{JJ?5hvb7|pcv~);+kqb?#g z!__b(`(1u&%MMNUV9xIM%T2<@X~^mM_!4M|*ixbt4Z*?WZfk+^t@&@o+@v^_28?Vc zG@OepWtb(| z6jHdK4&q_qVCHA&gc3Sv?5dg4*dm={Ym*+Pjc(zTmS5y_S)f6C#*}PYl1;cB)rrz% zE0z?xSHJ0!tR%=tn5(0Y$=?fz59<3kIjBU9@ruMTA)$2NTWE``nq)070{HTp8p|sf zAFD2sBfP_)IG(P6vC(!fPKwYsw(4bHPYWYXIv=hwqQGRYl%9i~v}hdX881<|zQPf; zb8VkDlp%^~m@i^A?)zL_-E4y78!{g5%rr7gXat6`ZAK`zQI=092NpCyx#Qq5gMZpi z&M0wM6>!^TYSrkmr4`!jKDs3g&jhpW@TZK#i?ZA+k8GbkfB1dlVWWK|m$rBD9x~{u z;|=4t?4bi|KeeQ{89@f>|JRQZ?-JrC;oL9Kl%C-)(CF*q$;sb7IU)c?5LV0#o6v4e zcCUrajV(0DHz_+igbqtm%6{Xz3{V0h>(`n++($IkK(BZUeo_;6Rp~asO6gI9C;0_B zsXv{$M%3D}Oo!>YAS!M_i~j;u4Fe^DxLD;)P9O^xh`1g!0Sc6OI_@ioA-UOKAVPq;(t;Zuj$OeAPApt0`gVYE~#mCst%^))G3uzq1Jp}wKLmJ5c@ve8Wzg{b_7;BI;Zq(~3|O#g z5CF(NUM#ig_RZK530pQHUnLf6y>$nrk&Q%9J00+wHWxcTJMi{3^j-%S=7Ob-cG7d`Q2|`*ZtXVC9xt!HMd=Cu0`-BxaA@KJU%+lg z;(Wcr7nvm*$KUR!p&+{QSXfo+9I{PBcal9^PKv8Ikv^g}h{LOTaUXYcUr^i!X7B&J zA^%O;ONFuN7_bV~;38N#d?^_v-LP0zi2)f---Gb!^6ac-6^Loho7cF2*8b*Xk=W=q z^u)`IG}7}r3yc?u9TR^t_O=n_51~j~tw|%}i>0#A@=53B@-|-Qg}n2xZKtj< z7#&+Nz@Vy?p}#<8;}EHHa02snH=)>Tt0EGF6>rG-a<38fApWGq|8_#6IQi8RWPoO6 z;wwS4s?>6Sdl=@9DtIG(TP|z3;cj_BmK%P?Q(U<0JO{{5KObM_d``lkdfQg z&5{N`prGKdM+uwIaMx9u`Y-_aPWKhATmo8-{a^o^exQ_Bkf--A zZY6-=6$CZ>R7br`{a~xhMU|y=PCnGdWn&r*Hu;MkQWClCiG-JCKEKK9XNW1TC5U}U z<3#lOZ#KnIgE*&?P=}8)Fte90#W-OTsg>PS7ecf3?q>Aum$85aro3`eY+=>K#Wutj z1o1Vh7=5&g6s)U_rQ=n6(?C7%t4|sSxc>iY27f=LeMlxdTS%v|+oQ|#-I?@Ymc%&E zNW183WSFbnFWVZY6^6cLK3t>nrE1W^24>A+A5L~P^TBY^&{zENsFu1Ki{dMBte5V^ zX9-lhf$l6V9gzLK)jHt-P(ZO@oyy2xpvU9IqN2Ic{Z}gVuDobebF(n{K$!jV;P$Fd zjfn{uuHB;+sF&|KH4$`@GQ#Q1vWjbjH)aT^^rH znkl#FbB@vIb1gIZfSc@nOBpA9lY@98Hnb_Iqf?bH`aCaJ`HF zOP8$1wBem@b&xmhUt6dWM7zbZ9_m^p6jdmWq73zjT}0jbIWSyiM3GuUUqr@3L&h^U zX(@nSi!Fp^T5eY-p1q&?Aycl$u6Ip{O{?#cn;!`Xw*S{(@}DxwRo$>Y%SrbrFqoTi z3tF&!_Vz>ZDPNSF+PSyi1z$+qTGG1uUD5{(miE#YgwPe?7WGWM%LR6zBd6 zR5FCv9M7P={5SjhbY@+_7K!3$ZRTjgJZ#>p_Pk$bJk~1`yGwt2MWs(6w(W@*zdq@^ z-`4p1*0rj}HWW>{n3{7bqU}#ZjxOAg@;YX{ZlWc&W)ZcK!Xbgvna;G|Kkr#|Uboiu z*%}A9q7yNt*34)vn*_D1yV`2I*ZMzRqCHqTOp_ZTjrg0d^X0%Mb;N_OFZX(w-5dXG zvE$$vZ7#8Lht6$5(L83M3BPXICz ztEHb417NIuUD|@L$x>jvxd=whci}R-1R) z9tsBDx>8`C$j6OezGnv5&1p#yYyj!Ke+>)%%>=Y~BY{%*gn^4`Xqh9}SwX{TJrOAt zgS4pg@U*6g=}Vxw!Zk#ecsp^*#t-(JIP`lyeOzn??-IV@OoWhHDfYEjyM&4^3-xB*Hz z;@}RqWc8zbFj2L04k1652m8iDy*&u!DC6Ie1iyEEbN7ap>lOD5TKT`x(32=#FSVAI zx{)Fe4&T&S1#h&bghH9yRuh3p9qP>uPRY*ZD-$IpMU@c45j?WDdQKgx@Ul^Kl>P?& zlk+S3q_KaeOMf35SU@9j&d6eMBd{Jd%w@%}Vr1BH%>Kr3QVf>P#H}^Wp7e*cl0Hk{ z;D-+A=b1TywIbOU!dVX^xA`??^lXm>w{>{1c{9_4dd zT*WDq&~FFs$xr1~A)txO9Kfg^%KlGnWaUl-hZ=gntp+ciHlVBG8ciXJwAYN!OXk&Q zLuao)_eKG{RKMxVZ>GwlOuQ3u#kt-srwA6-%3|tK^2zm-qovZ-PC|n2ye#6bdCyjhj*Igs}_BXq!Nolcb zqQ7`&#xRp$6jjMs9bYPxNJmxpJi0UBNq>DZ|HNLsH&`b48SsIB!6?^|Lj8TG>tbp`w6~xF2pI0yzNdr z726)P_j8TUu*BQNGS#h6Xmjoi;O<}=tN?`YR3rgfbgel?AZ320_hjelDaY!)Cln(k z9iie$JT_YAorDC+zrDEM!q-yeNSrux4p)p=nM2C!lHr`P+UN%Z8O#P;sICB>HC83NH#pf-t3j9bGsKH>zY{W2Z~uiUnpa ziJM*QIwlG3G;#%Y{B}NuO@ANLPNrc8I~Q(@ib0OjFFA=9E`AGL*GzJki6GWo*Hehe z^?p$iT?Q)fzwgr+oal=2G~Lm;wVk}t#kGUyW_a&2BFo9l?cS0t=K6BeITODe zSynNmlTh^0jm_4}D%@&g^kjlvi#WImV$v4q{z(JSBmVl8{b#)H$itOJjH!#iz$l!f zVB3;m*i8(wcYND&_PLCkZJJiGsewp5pYSEuOG|G)8)a;!Gb%74xmjGe$VaMJbH^$- zMK|o}H6uuRQam+ep^nk_9z<0_sOolFNlE`kWwy&;i)~4CR(g<9;|D3m{nN4Af(N8f zS?UAFiy}kuQ2cc)OXmV*1?IT980z@hlHb*?KJ8j#QoI^juKf4U0d((0(9B)k6ip985w!U zQ|{+}n&EFA8xKJjNBg2gWDJK^4bi=7pV4SO!j_GUOo?B`;`p=$hpPZESkF(8vb^d4 z2$7Wm3=TnxVAHahyZtfeTd-^E9_AXkEtOgcSCWZ+1XC3VxE&)>9Zlcl0ZT1ZlIOG= zri|o?WJ%^46&-|N;}2>g=yD%d+`jCbD#_#%LN0n?_nzv#FM(fkKEa#c_|$!g zmM~L-5jpNc-1N3&?W|QaYnhmlv9gGDD9Opq;H#F;Zs)7<@1i~);d;4VEgmn-2_EoU7n#{IulBGwa^c%1c&RUCt7xkKFd)hQ1Fl*!Z)J#P7VdWsiVAPcAe(+DC{%j?u~)&G&j}3X z5P2}y`HJlw^<1rLyvoAXw+n|DAlP_Vav7k}@<~-^qg6h091Km5|5Y}&=T`op(R$7t zg8>>X_V+ZJ=Ptaa9Q&kG8`p1yaR`)Saz)>)Vl1(&PvGx|SaQ77gHPPwOS*G!SCPr* z-{L!yj9_m89KR`M+?2PmCHh`uGJM_taRNVdceUpITwiQ~(*<>vQ6Tjaxx+KNEQii}7$=X==d)4|dU+GI|0&vJ_b`N5(cJvPrc86% z4*mVmRUk`EP2GGyqWJC#>jv!w;930JGcz5@+HT+rfq5@v+S30>`}74X`P?s7{~v_e zF3B5RKS9hk>9@x4yueU;q4oMn_OAvNTl7lS8j4IyNeWu;{0N>MMpbXQDF}}m8_aCG zOU^#3bgw4Q5F8hjdGwj*wr{cfr^giisS#9dnG}M*F&xKiQl00_lMX^IyBvB|$i~W_ zATv^$9)TFQlL>=(ykarTtY?vVtg6KONB8;M;|9!bN!P$YHUm{v@3D)J&N#Jps>M10 z$^kn1-%~;N6{N;2G1ZA7HTm`9;!C@X0*J4Z9=dmVUuf?5zP(HsdQ~QtP-dtnohk0& zSXs5#-6^%Yaju%Uomzcuf%4{22em=gB9T>~-l1()ZQWOH>fhSo?HLI(D!7l5-HRy+ zSrh3pj(%U`hi{Y2qT{7v>JIKOm9S{^Yn-|^mZu4EB6b1)8%#`0DhhBZr^Cd7|33BL zIxE$ACTob({t2SLR*@hp{;9s@RF>}04MHqX&mr`vBf7*8W7g?O!EE~@C06Ilz2@O?KQeycsVaTHE>Lu#*$` z41w?7()z1@yY3!mV6mP*K_oBp{Z+_3#_KFs9gIUw-wFrL=rp7sE8n2A=1{6{YN6$~wVMdFfLi-RJ!O!QO)>e(| zS*SvVa>InVvbEc#qYer$meYQFWnRw%IqLUUgN%FMRhYGy5NeQHQ}p?0-p zLBX8sl_%?=aZoT;s-vMCE73L4$?TnmYMkC>KDG;>KL~|o)*If7rPB><2Np=D9s(<* z|Gnat4WTngjlOQ8SxQU?*={LbU1;vKvv2*@nN!2MGFiQAUs48JX{<;R;!L{76Z6T% zQvT|v&)oCc!~kReZ%ybw%6XdV6yOd!kJCBd&cdJXZFIQNkZ2Ue&}P_rn4Lo)kQGD? zmX+A@QqAtjnocdq8T=Q=yFIEvgR?d($37h_DC#0>fXs-?a=+{QV$Y*OymtfS!6&y7 zV^U%d*FV4eI{3^HNJBQX5cvrEb2Qv{P+HiA&L`@%;L& znY!KN?ykTHzNS9jg5`O*&u69?cPWxsa*%v(7|NR+uQcL)9ge^S->+#YS6h#rwSzqx zv?HnG4QCluvwr$aMQMn0!~fxw_ca@D4s1&xg3yEZzzo+cyf|twL3&yIYpfw+Qi#!B zpG6WFuzzPuQFiv`g-b-loQKKEmagOhqP0NV2rX!f|IMg3w~f(Ir@S+&B=yNp5Y&dA zo;l)>POGv>CVVjF^0f5x);T0b`$Bx1+)8CjQ@B=nT}vtrTa^Pp;cC31!OESDi@t&* z#?}BVV6y?}LttSc;{J$5@|-4?EwcHn!Gusfmj4}jIH!s$SrEwgi`a1T&e#9D4d=hv zj#KPF^%L~DxzJ~-_$TPcg_7jKNx}3(_2#M&-@`T@Tdju`Z^ZzrNDH8#kxNNgKz75c zMGOL-Ngl#ufIh~S{hy%j{PQo9J{y58XFowc=MsvH(;26`C&?nrp+XwuAx26;@%<^S`4)B}(O9y-v9*i~J^TITCFG|;7iK2^)r?+PVj7t9) z0cyOIy|&Bcr=~R%gRiGUEvr{iDh=}Yu(URGCuYL$>0|XW@;8m@HEoxBiIUEm?d)AO zR-BJDig=>b31np6ToZLF4y>p9Ap$^w;}$^DN__q$XkZlk)KAbxeA$2KD98IyR14$W zBF@JIn;4uo`jQtjt|vKJ^MkcvM>1Abym1#fxUMcZYcnf(0oSwFkBHxFx#CSV09`S; zy4`Ig{@q_0;}uy_+z>&+>8FBhi0fSliqIrgk>E0TXDa3&_uU=OJqsxOLG^#y2*9U% z0>w`0dM)39d%#VAoDKdo69>Dht&0K;Q`z+Cd`-2O-ESYaw&peYW)uDhuJpb$Ezwe^ znw?M7eSaA@@T>rqDIRDmxzS+o0($D~ax@h^i?Xv2GoSdt!!XK+5-DD63?+Vwg2 z0A||%X?C7o#95d@l7!?M&C;kuHoz<-tQ&p%4ZgP>|1?=EK0r5=XDO~9g;uxV{g9`e zZx+6^Q1vNW44SCped_mtwN5L=&p`M|*blzHURd{++w%p3&Dwr~s1Y11kR`IYuix3Z zhMT}Bn7tsm)97w8OK$D$&9Mh{`~oVxk(XZ$v5($XlGu=9XU3X*PhR905uT;TxQB>u zcC`fOjtWj1!YDf|nLp=xgv7jR?7z75s$770OZB(sj4;c>vtkx%JIyZDV^aC^W^hL9 zHLZ!+kH3oc-=D`DJUQmP3>6({$}u7#T-6~!rKZQ9Ne%xP zs~>wKP50A1)|W+}XsXDXizcvtp8n|XuNknpNQAX4ZDwr5%CGeVzqFLnt67ICjE}b9 zE9&Y7Po9MwgsBTQzD&|(X;%H0$1)NA?KwG4wmZn|llx3YZJc}L77*YDU# zYAPGec4o@bS*ltZLg#j{J={C>2h%+_Od8b)KjF_Ke-X#>IMmJvF35o{Co*%wUeKb# z<9tj=vq|XS8tmX0b07jV1FkM#1>wJ9f)diHu))6iXo_H=J@X38d8N2y#)~-ZjE6y9 z_a$m@*9>{i=E*<3Ep)SQ16@=#SuP{O)%5EO+job1nCECPMDkWt+pv)Z=HMrXpbpnA zgcvC@#d7;e8(XSzxJY%(WOVA_faOBuMPx-YJOD&Wytb%n$Rv5<>EZGu)jFzf0QSqR z1Db@F`q&aA)WlYZSHV=Dm*o1iljUqCU`taZGhxEdIX3l1=XjjNA3tCs+IUKAuEI?h zS85|uIAoEdzT6*44FvrC8X7mAXl8Uu3GsNSf0BAa<6z6h;(qex^-oaoY!=7IC7^TS zC+M(m3P37=?PLa$`v0rd(7k|(?#LjuWmag$lo?}HgG(a@Q1|jQWewB?5@!9>t;|o= z4f^lu-I*4Mg^~D)@~2`Pf+aXe29`4@fz`%qN5_HIdE0v7m&Z1mW82SH9z*q@f{&%* z?_{$vU*9@X0C&=CThw?{+d`idL1<2PL$%K)`1!L1MgkMUK=&gGK zLx(Dk_0y^3KofOI#^3zUO$WBsSE{UI;<2}U2U6{yNrNLb|Di>YF8&inII#;4Ba{p0 zAd5Kbr*V%asy0O1>gQ*NVuHffuitSo@7}S$cgQuUbfopmmjlHPe|7qx!)TJPMp18a z1k=uooQq7=1_ZuKk#{zg^}l==A0{$lVSFq4NvvyGz~Y0op&!$Y87E(>r++oSnRS@% z7Dq-cp@~;tPYzShy2+xCOt4{~wttzssd$71WtI@+{z`^bRKcM07=Z5cS!1S$BCjE_B1pe= zHj07y3_BVncyj&8wWu4ugc?NJq~?ztO%ng#+`<2fTmFx4!R#aMPc*gpTMN>MH_OC@ zKdq^4#tatmrWHX4E&ZMAt24OC=)OF7^|d2%dOmplUaG%)`w#l8OsIynPvQL5RIHeW z{6I&!F#o|E&GqLIE(TLqv&L65RDXSd9KYDmOmZ2V=4z?&Pf$rX(CnKYuK%yFu-Oq6 z>5aafZc>OEIoyywjbP!}YFwfF-+I-lB!p>AO9Mt?p#koSFtI&o6&9a7Nw+hTK#X0D zy^tVsnE6>YG0M?&?N=rE8}cQyVYV+R+}=kGU7CaY`4BE_CKluy!DCrsV1%ttPp=tC zW+FXao%)xqM_h>UBXrc8)K5TC_D3Y|U2mXjfV#)3%T&NX zejyV!1EC4?0qkWoTWK0=}jn-^4S)S;NtHF{aNxV_6^!$kR5hv%92 zEhY_dMe01Jh_{(jHE`mNgmYG0|4OIHRSl|wv!M0vJy~9e)f>U*VJ{$oT7qT+m)`di zfjuw1m>U1B8#`p^*kv)PzcymB@c_={QJ~qQo2==n< z+oqg!-h7vL*$1|3H=d%X&XVmCdS+*G_^<6Oiss%`@i?RSy%5^{3k9nD8QL|N{nl_| zfz3a%T|d$br%c(f5`&4*f?I9*_f4{IGCN(bT!^{!g{Yf*t;3=Fh^3i)a#lmbroV^&4YHG4u@kfZj3E7kSmW!3U)7{#b7*>4&8mg!8U zKeN&p*M=b(I!YJt?h{8GWYuTqc~H{VE~5}8p7$NuJ(WAz1p<2BVD~``@l%i&keK?- zSA*<;Zb#S82q-vAJk1ndW{!C`aJ`stkF+M=$nY_fg-iVb2a$oB2sbxNxq&1*`J^1g zH5WRY5j3Yg%+M|4x?=hh^uieGcYf#F_8Xq>l0c(V`DgCPoas;@MgHv{vcJ87JHQ6$ z-KC%=M=X2akd~LQO8xlD(h|CBx92_|7~B=&2?z08DR;!5b2<;&^#PPI16B(!vXBu3 zuGj}xPpMZ&OrMB)Wy^dCC=-DX$GYnS)Kz9$0PulEhd zo3!kYE6(C)w;%?y{S*X*ix!}PL$02sjS?eVJ!Yzt#B(K5wpE5T;fKk2JnAIRuW#<9 ze#Q%AofMUd{|hHI!S?mrPW{ztCxKo7$@H5;XGBn8j<$dh1poAT?OW*RX2PVCc!S9B zhjjR=wPwu@-6~wSg6Zm;psQDUIG#TD*{@x<%lXF&XUq_emq716KrcPTcsKLLc`>ti zP(L0sY7*s0!pFLWejLQjTR5_C+<+eo|5hncRVKV9G!JST|gjExfAbUiRi`|bkAoa<&SEjzy5YV2U5F(UV;<1tX?PFVMc0^5O~6;I{815 z%s&m>zzRoST}dc#YGvYP* z#^!C3_af22I$GOAB(>C!^+r^NNg0`RfqH>Mxz?k+f5S zauOKi9*p|*;TT>T>S|HQZ6mr^jm&@UOOjZ5gZYJ}o^oBJF+r=nrgz#eq1WI2*MJW| zVg^fSk;BPtg!Hy1Nd{Doft{!8^M-P_wYZ)Y*=C?h9%IVdPu>aRd(E|EJQj zSA7Q`4s890-OUBCuE1MxqODw_FkpB4Dpt<)PYnxx)nY}~ z$v^FM#GioOrR5jU^S8heJJkjb7=_@ZG{%n_TPUH)5$=hV5=rCxyU&2YqBlk9RyWlY zslF8}YwDgfy2>H=byNwKAjoTYNSc_FsZRtX(M`$#uiawA2 zlG*RR`ee#0)9#%8EtH_cY3{2el7BLJ2~&kd`G%sy22KN~Lg((8Y0b_G7B zgk2~LmazH8vQO5Vn=P5FIh;=N*}fBLTvJ3DD=g&Ih{yid589seKu4wAV9?&mu4R9&!=+73a+U zDIJ7aecc>% zAH7lsUFb=)O{X8vXqVaWg>*2CB2=5Oq_qjNpUU5Lewk{AOI*kg5b2Ils(z3}5Sc6A zghp=vgF84}LPrL?2EREMhI(^ha=NPvw~XSCzR5H>_DEE@t?88D`)gjdALn26ASPA2 zKpQ;p+6i%MLPA~)eO{giCE}p7ZG}MHos}!}lt(j_PvuI9hDXIiObp28X%4WvGQfIY zzjL0+bu3Y(>xX&c1r6H6p0mbVM)Mk=i&E(mSKIh+Xp4TG2T?pY;`saV9R37#APJf# zlSM4Ewvdl9P$iGLe~b#!+K_HKU(GB(TxG0#S4SQ|tjz7`kV2&4aCv?!*({_<4#_ql zQF7s%Ec``-CfhenL*~UjUJcEtPzN~n22}LZVO6dgf};hD;sVAXo{=&$oRf|L<508C zsW|{FKiu}gXL$e&to}F|TGp&7rwD1%;hQ$t98IF+cG?l0tc|D#Q*M3P+PLkAL~T!0qOl8jN-z0raT~;9(6(5_5H2(RVgh?-;4oG7VJ)~Xf2Bwav5%A}9RD6=Zd`*B#Gq{5p|M=0lNx@;N zg51W+)uQMlvz_(-$UlhxQ;PaevU;TC^&QiCn@h$3R} zxFn?3%c@TTTUHJ4$zZ@c^98k39eF5!+;os(9BB2|2)R@PtU-3ESAow3G4k>EO#d3b z|IL$~d_aN8akKkfrc`n5v#yKrJTeLemU87bVm6a*G+LJLpNW1U9Suk$9^C1#yU5~9 zWx^i!5n2gU*CRx-@N(m@CasS+PwndFdOO*0;Ywo(|E^o1_iYmgoJCl4NQ&$aofxm4 zk^UB6QVf+{0#X%7^08_S4*Jchbk1@76I5h^OrcTViwa6Z-`V@+r6~KtC2`{$@%=#2wfcmGg0N55qL-%JlB`Lv);(Y*Cy3KpJey(>#?Mso1}=q?R%_)1*|r=3$yyZf-)%rt9GAs zrZeKW3K6s!9k08F!d7a=#ZzARwWD@QmHR)xofn)ot1O1_=s6wSG*tqH9YqH&(oe}r zUy^6g&@5^G&{fbRgEMjUL{5z;I@TcI~{0A?n@hklP z#a98Hl6i2Ju7rkf&-XjOC$h7ZzgsYxOUgcd@&Fhk+J|0|UGwAayUA0tTVbhff5iF^ zp;iy%O4D|e^iPnHY{toYyz{@nCuA6KH(}~a{nZN2!9A&X^QVsIMQ5qwe`78($^W}l z!3pt7RQ&a!XgPvHpuBXa;c1vH8trvOcot2caOaXzdMlHz^d(o&@`kTnFApX<#7qlY zj!BO$?a$n*MY7aI9)(!L9-$AG9*4|}o@^i2`rV;}hdo*=qp>4`0rD+kT_^2-*A*94XF0 zkGpo->CH2VSDC7z$z~v7oh;)~8Pq9M34|p~fJjN-#6f5C+$a`(e)rf78Ys!{kMX@<_iy>bn3hV~`Z`5E^T zc2VP+RqAYGhO|T|qA8tMVJP*_JR2OjzyI|eD6`A9?2O}b66i7-o8*Kn6z%xY%;Zc} zEt#btT9LE<{7ax^?2XhO$*!3OF^)1&CWZ#9H)u3ibOY_sVXOBkiX={T*bvCqxLe3CN(>@!JEK%%zjmA(SV#*m>JJsMn zKzcBn4ZBG}zN$}^YESIjnlbWgqT|0HthYz-*Nbfu6_~MlEvUvNoYHR7y)v_VIV9@# z*T;7Evve%Qrd6W0=sE{Jafx`eE>k*oX)R|^)fDlCy6jWfG7AS}VqK3;jry~C-TK+O zGcH0*Bk$x}N4_|`bJd9YC&)#V68!l03Sm3d>P|mPmfPqFEt#36pfp`FG*-?chGg^P)8%WfzN<)g`k>_$#%OyGRaYhe9a-hn*(mfqB%G;siA#)Qt5h&zB+C z#*3DBj~L`YmkGZ98VLXW4A%dI7XR3R|EH}nHOa@ZbR`;n^Be2@o^Z~Fd}m?;3C%uz z+?R3Cx40sU_IuKw&sn-qjxyf;?(kc2_}5)XjN@+KX&HBq)rxnRI@`1b|MEG#rQoW_ zdyncH>yT8!=`q(r(lX8roz$=EpSJtTB;WYx3;sF&4~z0}kmI27bF!~X^e?N^An3G> zc8U6$j5)F7uZ&DVn!?-L)U` z*z;oqb_SOM`BXzldt(L$^`xc3WTu6PI`s&sb8@63IdS1=a3}{!6LB%#quxsu<)R>t zET42uu~V-)HL+9in^np+TYz+Hs6ypOE-H1Z+NKBE6*OS@c6zZCXzO;Zt9a4!@`)xw z`JbS$#r4TP^_EJ@#;Eo$4c!En7dh(dxxZYmy-4CfcdUMvj=!Wttcx`#KF`yl_BrT_ z0b6H)R6cNz;|&V4Kom^4w;+X#S=|dKODiT|oe^((J8z=V_3(!{M^stZRI>NSL~Wp7 zVbX|Wh8}&x{$p*zO-;Xrz7OMSg_-#d3x52Pd%*Tij`nDJRa6GcrqO-}kawnsL-Cr> z7?zXu)Ca0fyy4OSEKGENbe1O`J(E5-3L`hWuz8p9TgWtIz$If= zlP{)~S_E{_;*OV3Fj*M_HM-!cJN6Bk8H$mApcD_@MxnPmbLm%t-$W&kC zv#aj88hk8%drUt(z}C^h*pEHzdFW1MMi6|c(-$Or;Vefjqq3(1*FAkCOn-wX^^cQs zdZhjn^fu91kt1=)j%+{poNx~RXuJuWivWYkzU}WEV`1@K>tVCKKqvQGL`spt=vRn` zsZM-!e@lOdRkKe!m<-Rol4gAjH2Q|%?|XCNoa9-$PlW|%54snRFAEDKFg~ctH55cK zMiMj@hcinWs`2gJ1Y7$AAAE>1(ddfG%U2M?auH+BM*;ZXA&(Ce z6viXsCc+)Bg)&eJiaB=YE&2Fq6i1fm4z<|AU75Hd%jW35-)FrZZy>nH|aRIOzJ(ceRQ|4;Ceh#ZZ3pvSOCkb zvJq0p#CpJAfJ-Y93dU@AHhe&iRn2gX?cLXBZVIJOk883tKh;osdP(K(hXW5{w(SGI zv91_6P^EP*nSxq5fBvf}16Htl$K8~7SSTkJDh?$e13w()bAedegr4Z$*3Ryx(*RM7 zMUx|ZaMI|xl{~m-mt<`m3$%>8HRiFWr(;rfZOl6!v>g6i2iDRR7S|4DM^eJ~4XTEL zUE`iGl|${wwJKfeTP6bcT>KVQpGH|l_!Jw=B;9tu;2-TLfY42s8u6w@+-mfHh#4%$ zY^cEQu01NrZSEQ^3(gI?At3Of-g6;N`A)i)!sVL#^(S?_iEk8cz-{;2mq6!ixc1Y# zn436wvO=|wPyY6-?lm=r2d=eIvva#XFR4wg_sJmn)Ocr>OJlWwR?dnQgRwKvgIjqZKL6-x!2t zPuPZflxW(*2rlm#rkPkcb<4MclsMl~bfze#-txNLT{rKok4&VDwvCNbJ%jF_VFgm;;Sn@RL+rit;-FvuPmkmMHh-FpDS`YXiH4c8?+r{;IV%C4ITH8J)WAO)PzIP zkUaQt+BQwOF|}e_p*Qnc<~b?6!?YSR6x3vB?UTRQ6w}7cKqBzTZ}fhwXaH14w^D@# zPH^3E06~BPhba&@0y`~){eh|vgy08dOvmFf`+e6HG>l$@<$V==QQVF;3|smzF^Tv;tjcw`AGns#<;YB(>%rMQh^PKc-Oz0CZs)5sT% zO9EdQBdFCfDaJHac6Y>2Q@-x5HQB+_l`espEN^57D0b;qT9CvNy^fhgKBzv+ZBpg% z$D0)4g4?fT!~NB<$}NH47H&bteI(JxB!-!Gjqf{pD3$Alr8T#ixnV+zd}9p9AI?%t z{G!9W887|Vrhn#bWrWj5lUHIAyNt}D?Rl^*{;sMVlvPV~n?GPfq2gWP2IZBX%4^e! z9IEId`U>Q1D?vHFi*hvsATJAwU6L}JbP(1jqDd!r(`eq5{Zzn>!JBnF_Mk_%q4^!8 z%ZAC<(<1?urA`oZ!Ni!RFKVcbgBs!1LZ#I{6xN`ZW?r-i)hyop&iu4P=I#e%sq9p?{g`-m)waj!y8fba?gCFXy z{6)~RH089_kWcSn_pf$8jcMwBmH$*7^rNzkqe@QU4lWLzLAKc-2T&~Zu?_JF`Pq$I z=0d!+X}f3wUMe1em|A?*Ekmb>?89_+92ToEr;AbefaPy@li_`Xr5&uHRC1L|u1W8; z7Q{dQafM5nr-S6m{TIMEsnnyg%b40|Jl}Fg#3;P+oI%T9BANnqMbL+f*C@aN%Q~>udULS%|hKLCuJazC3T?! zn^`}xY58#&b^D^*aA~=;h*+5r0hOKRS(cmz8cdsyDNXrkX7kvNdh?Z`dSNm1*;;eX zNM++VY}e$jxX6Q8D)!wWW_nV_05BuYet$VA3tdJ8m73l2N2P!-Z)3OJ8ARdezVC_= zhK-ukHp!m0G;%~}bEapQuGOCPYvp7?Lvc@Vnz=Zu)8+2?e82<=$>~ajYq3&L+kD`x zJ|AW1m9_e6CO8jF+sP`1GTl{OI_Q5vut!1dJ>t!4#$Y$A?@D#o4-{A z>|Bs4&vr6V&F$P?tBvKkoHIxLfJxMsNC>On(?U0k(3rxLd{~6vQwvr}?;GSs%j-c{ ziW&j5P#SiRT4nB7;PV~n-c2()+=dAbY)1hkn=fuMD3D-vTWBR%QqIIWqtTjl++&*G z)Gjw1?gfrp^{sqK7Ou>GQ+rXe-gaMs6Jdzcx#TT}X>VHwvudqq@O#R7z2=~_N!S&; z9t*LbBwIial8W-n$XB%VR^2L2eM|94K@h&bV%?EJVds6PwgS@#_f#4iSlxB^KqQim z+-26iAGI*P_m(cizTeX6!ktpFC%9d#`O!_he4foB`-rOp1>9M4@+kjkK*i2dieZym zqB-|-%6IMehNdVS>D`^>$3w8+TglwfwPA8fE1j60u3ZNG^2*v_L<`N=kOs9V z(c0VM?o-1ayoQxG>jGpc@20*4`FeSJV4t9oC7At(xE7SAnBo|M-y^fFWS-1vZ;z!& z7p^dJ!;!El< zx~y>%)YjmpKBF3dr7DlcFYQJuf==^V7r7F7iE2!^JSd&2Nw1ncw>t&ynra z@?UvtR9&w6u$#59*vyl+nFhts=|hd{{t4<@q`}stLCw(|gtk-$3kqps^9J*RT~+R{ zdTG=`zAhSP%Dudz#{u^vLvZEeJ^rAP+K}8>3V(z-st)l4E8NB5Fm7Ijm3(u3FElHs zSg&ABVq!-xcHw==+gJO{ashZ0(S8xw(uWlv>YR!PzrXdD;icsXlK?(bUy zl^uqj&XQ@K@GpBEq=%xhuV`T)5JlNba5Uz;x(lI*tHx}gjNL>Lic`- z`>DN+$zSAU!>%#T`3TT^4pL#TYO)V#l!&9)>IksGWJfwuJp`O_8;b#hMM_`>x zSu2=c9LBez+QCNCy3vV2=rP2qq;cW>rxPfh$$z*s0JR@ifaF zLE*iUn6;#5VVhD{YD3!Sm;K~6o@$Ge zDmk;Br;=JXdOkneK>$L)69%OAF5r2w3Q1@R_ATbUsBrd16bJd&wMk}KE`{{&e^N07TjvlyFl&?25H`N+ml|Y8(bZyv5`GXQ{-gW2KourLhA!EzlR-5^?PhPr7)e(teRIkc@264XieOv$X88u5j!Vy43aD6pw@i}jG zqAHd^6goP$U+-C~_+^t+yQrT#Q)RI*Aa|7UA{pZPG8s2Iqn#p>odkm{KxvdQg5Tiz z(PX@z=jLa;8V>J8Kdz)_h`F*6M0KsxBFJDhR_es$9aWW-J`B^p(h~GiTpk!^b0kCnYIS&)l!ThTB9W)4}Qu2)t^9iJ_Cj%O(RjM~ZW$E6m2yWaEre7D+)0w8#W;60_n1kLVa zFGZ54iQiRE-!teZj$*XXOIj3-V7qI1HLdm8lov{i9%t7hLzowa;3~LR1eI!wkdoYC zW3i((-|1^Dy4j|8iSqKb7I|0UHym`|C>W;kcgsgzA>k%srr4xYqCg)DSbj-VvlrV$ z^FpQZ=T9(mChE??So2^eQvPj@nX%U7mN#-&JlI>z72*Nbq58Z`K4$`LW_*7tcH{fl z{Y#qszC7=g-%*PZT5*VgR`A?dj$>b3h{V~Gb{TLG=V(>4yTJ(R)OBrZ{?2HbWz`|u z^Ebl`2j8EGblch8$IX;%H55xy8$vNWNQrTTU{Wbhm}*ep6)Qt79acZ}>HF$;b=>@% zjaA!+W*{h}oGO|l%!fkUbBNlO{npgcjnci6l2Uv2$&DxJp3-s?iz?!fJpc ztq$iOFycujo$Wc+_`H|#8aYEEw|qPDeGU4Ril$RWB`}ZukDg=#i3~E_u)#o?7Wz)k zXp^#~yR+T(?r2a$x~E2@!t*xeir}uU3TYujOYKqqo1in2|7?%`SY6bO=MnbqZZAQZq_K997-66l&xg8Hc&CY>IQN5Vu7H-~g(w5w zDCzGWk3F2PNk?gxl(*iRzQbE_F0*MWHKjlU4}XAjBy64nR6xm&?aQkNk)hhX)k1H4 z``3a*JhJ+!VZsONO*G3bZ@hWiu#4XTj@!k{L2I}aA#s)*iwk~I(LBz+DNJvUWMN`w z)GiWHj(ks%B75bkqQb;4zX4Rnhug$b=-{+4aX=E5$|dE}gs?cjnF7Bd>&J(u6L%As zxMhnb`GNo0EM3M0_hejZGDLA)HS%iI8{bWkg(Et{J+J75EOFl=zp&NrE9wBek?bnP z49Z)6W>&7(F;7(?nwKuhEA>)v+9G&}RZlVSv)%!t| zw?f9uW^8c(dIS7}i(af|Id@%m9r{bhp6*7>QAzEveWVt6G<&ma)SAt_r`HSSp*iO^s`56UA z$IEYwo8XrcjD1@~IGs&1KH9xj4Hfqk2G%Sb2&z6DH`fTad(Wy)JN%h|J4;AOkC-zI zw}1Q^M+LN)CP!n6JE?_ziDLZ)_e^s>nZd9j+6jX2qK<9d7F*1iwKr{^3(#6(xG_x=DG#H*f{2mR1 zNzA*4%Br)r*hIUTct2(oY3q@uLwR31%TVC^I*ju~fkVV$PJ104=hvAFCg$|ZH3@f@ z$62()w)kK0p6)*Ev=7Z9Sx>Owx(6&e%(&WOD;#_-`DyGWLcM5a)0=zTRsP;X$o)gP z7)PKjt?4$?G-CqVU`3xwEEu+Q>#ee9bmY;Qdxtkiq7G&#cmu1DYsl@_cjY>@i|y|F@{GM`!RjU#;rw@Nge@E_&Kg{`_N>H zzE^?t*q^VH#uZ>uOD~eh)|AAfN_FB}q}$%6gt|G;fVls`${}P z+tlOQvJui_sGx1li6$x6vK}wilv)o4z2R%QqQz0QEVTsF7XF3`6xhiEGzAOd-sHaU zlidK;?Oc z1Fg;K4(4c0y_?rU(f-=;r4o(HJlvjxMwYtTJuaE?vy8svEyu1ntPi%dA0l%cfRM_c zmM>W=DN=9V7^zmf|HF0e#l@0a!TM$>__P%=rtIyv3_svxkZgEsOhzXgM`v<$WYuZo z%_iBB>p!4wK|`9#b}2W}8nSjGoI$9Emaq2cpKF6yz^2+4lxhFV7jY#1CC(qM6`am7 zVvo3P0nN8<*sIO68P54up==-NBb>M{d%qNCT6fjiR2mJe+V~?`IE7`8#4>cqJZwmd z4mbjabyq@V0uI|mzxFc4lH}*shN(%37t$*ZmV8D9@s9vzG9&2w5^a+-%7+{RU7AgJ z1@lx7mXrw??~rN zooCWpTvuCRh7I1ZORe7 zuzvJ)tAkTgZbiQWw}#bfTcpA%qlH1na9FVf8TpFoq;>PB#7A%!yZ81>JN$&5i9%D< zkbET`ksZAgj0*V{f(Xmo@&T^|31Vl*a>EDrJAt~$4xZdaa|ipkkJ=?I)-)>AtV2JU z^15=qU8M7J)q^^ z^E*e3!BjTzp3EaK2p8A&nyoa33=MZ%YJbArRGj++t!b($si;WI{vh|LXOWMXAo+me z4VXZIYj$9H0@sWRhr=j>5SDp%3EglZ*rb=cNzkz@f+y;QI;r@*5 zYHS1NkJ>Z`=z-t|PnPY3$<5@yTwCAJ7?GH4i5EijUN|h?UAj)FZDeE?ghKi<5w!)>JiboU1RU+ukTP*mNvF5G|wiIQ_rl1fr? zj!F;_MRGbb@oGmJ5x=NUd(xEr=wcnURQC>6vYEUzbrs9TPLKj02mqe5KfMr?9|ke7l4H1502o;(DPp-X-uoFUd#_N9dE{+ZeI*CPB%jp# z?P>N8e3A7PqN}PbVBN0QjOi5YeAVV08^B%I^uf1K<0h0w-K@!A=p$Bo*d9f7B0ej? zO`+3GEGa>2p?Z;wy+BPCX=?AaD$4G3hR*~i>RI&FH7CrPE&O88vEg01D`OlA-*zd%K%;b)zmHVGic}-|;u>gO4L4p&s(G zleH9NkCP49Pot1Ean;h~g=i_nNm_2}hD76{#-nMYmNdL9-+Vz8cf3oh)X~p%&^3z; zuxmDB31=9g2aT(Cf2yIK!Cc?X6+xUu zNi&uN5G8%WFxMlUaw;6!0w*UE8n-GImj;DPc^J!Q(e+y&ctC?LP&&N9Qa}Rs)Eki= zv9{>Qy_OO7W0_`q2Y^Y=2&8I85($VdKAJUeO;M3!bLP6qj+HX_IzKfGs|cZT8ke+@ zlpt-pMiQ)s>7GSPRqVaC{~Czg%XG@|=Zsr^pvo%kuB>Z!QTlDMD0V}d@lr8B7nLnd z1YUE*e;9V$FH0lK(TbUBYgH}4-r_^Q9lb9l#!sqG`eubtVO7bSRG2ts#f6Dd|&IO2YQ73@7F(=9uA$iu@X8TG+!&E(o zaBfGj`Ax}HXoQpH(IIA3Nfl*oO?H%`N5?%MK(@Xboy(8vT&05q7jVC-@~C2Pq*a?* z+2i>(P~G@wQ?JlD8aENCFBMIT`2xX3h3AGP79>Yu&fjkXZToJu$%Dy zUR|8N6;!6VoyjlNOjB<-NXoVmc?$UmXjZem!@}Igj8XYVHEmj);Dyk_qvn#DSPi#j zXH}r@amp?&PKtn!T9X=|omPN$F$;lqz(>`VaP7c~5XoTRO4)@#@Y_orR)>7jy#L+J zk$=DU0)L(OkUzDuKX19aTyUT$qyWB_Tr0EuHK~=W!v^BN260Ou(rxkVF?QgeY_=h~ zc0WZnF}`ex+_YDoRTMKR$5PWtXlR$W;75;SzB)EBT`{VRU|4-HwQ=0Y*d3xS{K!(< zIADBF$mA@xQ!Ic7VGMPwiAF1tbPUwLbmp~qkm6gtM)$d!>GmNTC`=JsajPyw6}8`g z(G`k*h=dKY+(NDnj0vrO*wvjxSw5*t(n0)4ohbWm6!(?iqAT%xk4^|v0^s!#y5Bgb2j3EC|Z zxIwxD8fSwfw9jTb8Jk0@9X-d`2PMb0)A&Z}ZE)<%R!r^@bsOj!Suhs8=Sd;BrnhtU z3avpFCGCvZ-$;MkrV?vzG!Y|vYo_u?j+thw2FuF%cUvDVWHrTSI1OSnMYGK#2|T0| zeWyLRyHgJ35@5f67|qa?h~_NeFF_LNARn&e{RCa-e|=c8U~DI}`}(|}{~brBfueL6 z-fO&!a>6lrR;O=W4fx^TJ6V9@6RhNo8gh?1kkW3#G#3H&d)Mzr=WH+W@(90=3uix< z=L)inr}({3c9ISw<_D%wS-qxQd8&w#!zdAE{=vQrij}XLGT*&jH7S=?dmTBPD6QJ9ms%9D zutFB*G2%XM(90h0Q_LMibxx_8MFr!sRJZ9(#CD#i^Lx`(zP@Kmre-GWMD8rxIdCIE zV<)%Dl82zcybUsF+3JUw;R@C0$huHzAW7Pkc%4?S8$}+VhkJ z9DMgDDA;<=$vat4+`5RaJ|EytM?W(nRLvl5s@*AuYcl=7QLfLsUZ!!YUc;9OPdmx} zXfuuRbJ~0qqbmM4k$E^S!t`^`MKG{!GP0&gs4y9}aF*aZUg@izc=_DvB%5Ht_i@1) zNrCqbvXlUeE1+=U8W}c#rqB$h5upAi*&Gd3FRioy67_=E`l1h=BNJ@&?vgREGx zjq2k*Ip$6ZpXa~?j)($dIGY=0)AiX>1Uan2j4FMvOIMWMHq_`fPn&5{mW)L=wsat> z%h#OgfRb~lthFmGSWl**k6t$qFbqWBnC`37PA(1m^T>SB-%eTK0eg{-d~LRzQpf<# zJ~M1#fl|O(w6HUi-TV~zj6%IoG!LjHzw1ZdaCDQvDGI84>c4@km34c2_OdT|KoBJN zoWD~%fV&#{|0^t4pU_U)YrTbKPNE|(p4B4e@8U2g!{VD{=@XuiI%SE8fS(Lp8gnuv0}opTgn?xTAcgj=+H7q{wQQr zcbjt(nisEQqe`s5P3Id@If1$H!eFq}xlW?Zz97#t74CcU=;PA^_FDi@$x+UpEc7gh zWd@?%DNrB&c(SB1_S7g(@M~4(Nsm#(avOz=IdpSZ*WI z);p*94dx^u<+qkBUnE4NJ*#`|;uJ1KfWN0GN)_u)kTY6|}$e<^GB=RZ{(G>^bh9%sTR#@wB_-r<{W%HnY z`rt)-vyD>ZGmgh2k8%kn@T))`BFDDzK|BE(t;p4A2st9S7wS|e5cZv~##3eHV5{l4 ztiymKY-D~UR&zzzHQkUn!@z^xT@b{HiqF(u1|A_Dk#HyUp2m!V<@RxYSt zW8LEY#%c*;ciRD*!`L?3C69iB!h*?mP&$aD9y!wL1Kp$j-V4W$%404sbxF2k6Zi1O zJXho6c$*Hz|IE7lNA~5P{FeJU3jmlI`^3^O1Y&*!`fCNVYpNS98gM(GxT;Al4n-<| zi+#W=#zeekn`|=Xw!$%U$j34_-jB{`p^#Dw>ErcRrecau=}jS{-w3(+6BH4##(F{T zF@YwD`U#3F!qCu$1b?%hmP3fLV)$!=R;vK+nwInB1{p>~W$~ozf=~mf(yQ=G9}~y_ z&le}5^i8nnWqCw{=; z;UNwqNV-P;Wv0s+sDStFj@T6v9}v09Cpr9UW90tw^!@l*X?^I;dWOO1pgvE=R}D?a zLF2ieefe)~Fc;n9O%tX|;)n&X{pf9QI400rQPKFR$M+|5HV0;E1#6*R8^ z2r$s+(%`k1a%dCtt1dtVG{1QDtdA9t1#R9B{y~+6L4^Zai%;>+{&F!Zq85S!;8gkF z0X0=?0pU!~@Y5@2Ojv60>WyGzh3r3?UY>FFuU7*V7<&+8SHx|$7(Bos`{&61+&z~; zfRd?#TGwH;9H2|+{FkeN^v<7OMPKeu7A*QVdiAmm+K`YDRMHdotC{wgg-cvrjy!WA zO~ixe+);hg4-i@HANEVss6Ip=RF`DRjQBpU8&KF}A#n%NUJPrS4sLmsF0dc*!IJPF zQsf5B^15xW3T)E+-UcTGKU2jrGc#GFz1h5>(2XB<*Y7rVq5UcO0tp{qkr8LSA{#;B%^N1|*p^Pq z5KFp2s(l_F8PRbPsV8AlM(X-=w=H56RhaBVD{fU|SA>8-=A^g=f6ur6eZ*5bTo4-m z{V@0+BZ9yG`x+ZdVFumqf7S)YDzHEYN1ap=PSj%LdN1Z8Kvd5j{`p65sEY;djtuV& z5^8Edyg8#`p>CcsJ=i{eH2q?(#-j6BpRN6)AW^7|rIz<%-}oqiGOgkTqjd`yMZmuoa&T_`*Z@<64+ z1_v!I@e>7(f$Ti#;5uCBxjmoo%sel=@p^&1v&EeBJBCvtk8@(TUK(l0kTJ~BgM|B@bw{Ovt{aW}_9v>} zVK%BBa7mt4D{|&?kNcpzSP49y*7bby=50=8Z{6F}`nWe0#{8H6SNA-XG<^%r1UuSapRtUnn%f-cbnE%f zTts2%qBv5iC(eNrAFB85&;Ieh^7H>>h4|0ws@?lLX{M#Y>-$~AU_s+S%fMQVQ}uU_ zTvjT2x4A-79_lSN_S7y#yg_Jy9D*HogI{2#F0JIO##g0A%><;HEQ4HgRs~EUo*z;^ zGm%54#S_K5ZEVB|@i|ibu*}kE+r?1N9yBmhdn8aQI18zFCc!gA zzpN~4GTnQ-NugRUB+?e0|M-LYkY&m%4D8R3~P%(l72)!4_W_gDmHgDA~{#@cNA#Ub~Ld_Efmat*3Z znx6I$z2g^=7=aoZ-N_Za44UU2mBeaov-*M$)fGFH%&GK-I`(;cO>Gv}i6=7xr5%dH zhx!d`gn)z9pCApC1mZ()7n#Q=oABOr@=s$70n;NN+U&8h&Pra_uaQ=T@C-mWFO`=7 zP7ggGi+4?9z3WDQNw0N5V}7aGM&VP6yvhxsI(JFBxQPERbJkuASxkAqCf(Th(f(kA$kh)V8j{HsfE9P2`K5 z=kSLYh<)#UAK;Ka&WbXyhe^~Zf7l{}Q_pN5@6;tOs3qM4Y9M@3$?1GHu*9UtiL~s6 z9>#dALUH<8SP?d<8?2=Kb*`tYlxA-Sx=MlErI8Z#;ps~iS(n-)gDP)hqo8B9JD1wSpjb2ioWRxe%d_U|2cX?jH1}o zGB>t;XelsC1aU2h9_5YHWrcG0=Snw)Yv*>~Y^;T}$IJ9brjkZD`rC^@;&FN&PpiAO zislJzkIMsr44yEb0L(U_@j7stE_!iw(%gH9f36FC!*f@$t+8XYExxWYobdQbD2FFj zmsgQA2qdRuSCa#%KrQP%p9{7Xz*^mF)>?h!fn@Tgq_E0ep=aUoHO^J@(VEGIneVZ+8j;gsS!6Nn^^)rypQ%Mj$NW0;gJnp|V zjpq|m+WT1#6gYlZGU(+yNl|B_C1Gf&<5&!sRbh9FT4)bp6P?TrtC{@E>iyT%{QuzZ z;Gc6qv_RBmwVmqVmraFjvhf?x=GIY!;=4`DA`kzt&h<3Q2@>UafTC+p^ zkqX~b=N;4~jHY89IJi(ZFvIPai>zwA+}&u5J=#z^$nIl{;a6gdAxswt8w?$AHyMn< z;QPq+G0ZhqEckqQ_T?s0vT*Oz&vWVi;tJd(;;ppP(0Crt+ppfFkQA_zDN153NAXx? zHsF4>v5$(@m+yEP&h8}>g!TITgo{J<92MV+os8oncZ9LlF1nIL4(YzJ;4~pERa`4k zmK&a(37+oPg0%HCt+<~F4yM+&ecV#`^b&>>S|HG`8GR!_t!oP62qVhF!K%t~g|Ynq%^zkm|4Lspf|B%Ti4~oAvi%bTp1L$q)XewfKEmq|>-nPY zT<|INX2QFYz{-1y!|alqUgGSrpl|=djnqzKTHVXLnnu6JA^qR%`?3GU&q#>=KwaDE z21X$@_{IwLu3cGz@cztwUbBkS{G zaX?E+w7@EWCYL&{uSj2wIC}E4hxs)@>&Mpe=JMkjAUo8DQ+JdSj4?@0cZxd{NFQjl zg!Mk{n69e+Zmz3eRbBOcJC-LpS-+yP;?o_TGc_tW7{Fzy)a*~8!k!W^aFLd z&YCVfm*(Oax;e*Q32|1!8TvvWCyNp0@8B@c1Nn6nWB0t)S3umMX{sb#vt_pUQt8|w zT{MZw{mE$1)AvUMW!YPixIPn!T@JwVCQAj3zoh63fN*^Z^RZ#Gs@2Y1SuagyUQ;o+ zwMsF{?wIoNB^$-{d@}ht3?BL(5-%&T)PX_0Vs&qVTsq?ED?ycD!Ou8~5E{AK8a%b)YbLsVbfxS(ZtW zW%s@|ld|M;pA=Qjt_=U??9(rmbtZpwXxj^+-v1%KXq|JHP z>8RW42t;SHRp=BFc!Rz!37OIh!0}-C_sr-wNh$M$;wBIg!CB3P|e;*(VECFPZVupz&>qb)H@ArB{+B&*A+h%Gw6PeDR zWyO89$@r`>=zvEEP&YZ|stXYF-G(KHF(-_00JR8XmSyJAaTOkM;)Tz7Cd#FKO0sLs z*iHg7FV*$Hc1&y0h4W5BL3oR-sV2;7F()&5H*I03L=HAyDzO4v$F4c8s z5%S~f*24j!GCXT%{auu36{J8y11lv;2hAxEalos^Xn(-o(q>oV7}3*!QSeE5%<_!6 z=f{!;$S-Fi93tVn_BDug2}&<~{X3(&za_^zuxQ-S{qroyxFyUBzI>ePkY0p1?!JJS zGK4>yMx!9??$j&kkp^B0(l;RF5>HXD3#D+6#mv#Y+gMTd4?)!7_Uj3xg8-a8uFCO& zUsH`U*xplin@ja&rcJ0n8MGWQXWcHrT4+k58 zCX>LBy}S1WG@`OjmE7*7U0v*d7 zC~xD8w%3JuH04Bt?xzSrTrNMX-p(hZL)LshM`Ub}wyoSkK8ZaxnIIOO2jgzdR&`u; zwDM7j&7B?%k}TQj!__7~U8@#NA(>!Blc4O8f~WIZNZl+kDBLl3NW;?KyG)HUJIWn8 zdg|6=9g-VRPQ<15iC*HQ?KXB|1|av!m1SRKcRKl1U;n{tqi(`iv9}AKzK{M8Qc(c{ zx!{`qH*V+uJ41zk_UxbYSLnk3;tcX0SYNe&f)4Vwv~4Fppm&x4P20MtG8rWxWj zY6lZ6e2B)*`Uy&T{|h)ZD0h5b7Q9BaicXVfX@BTT3+S%_>PvZJfZQ$!h$S~G0GT)B z1R7(%iozB93F3T;Up;C15SR=A%J`3n`cBjllxa=EK@p`@b9Y z?`f%~g<#h8E3hKumH?H{0TaA^`bpqR8=$A#vM_*54{Pwm3XrA`5ehys0r@9TudgC9 zP?wmX9}{S-UtR;8RlMpNTs=Mm4g!c*(HZhAu!lb0Xif~EsFv3OoM?dlc$VC{0P+ld z1sosX{PLPF_$`yB%z)PfCw1){{yH|H?>gsCLaCzz0k841#el{DukpY-5Y~$d?lFTM zo?iliG@dg?9dPBpG|Ej@a#GrRT_8!fiw=Nns^>A-&v!CPUTbO+f-3o1OM6G|aH_3vVa#Nv`UpV~qM5 z2^oWAttm#S*zeoYjZglF`seR8E-M7^_@c&ziwLQhPn>@PaY9azqu zwszIF*m*B3J(^VBby|6_9!gr0t#afv*oM+X)(ixJP_64$(;`NE0(~sBCSuCXNRyQ; z-%8PZ=_@4BzJ#r%eORdMtWeXb4X$?f6)?8`nF){e;vpnRFU{D@uKo{Xgg%+-!Svt! zJhSYfPpXQcEB0qzBre3XrM|iN&2Jtvsa;EPlsdaPRP*pE7lRKDCJJ!mJ1s9&!ObZP zLrWDJjyGd(a-7z$s{1-J)=|>Ij@hu5sllNN~2h zNCViBd5Hl;ur2}s*x22+6%`K&EHE)JLp2*grkvQ1a|j|=hyrC#*Wqx>ahEDbtaMve*Pd^c6gJ>`#fjb}A9y?ba1UH=cw+mT)0F;)D<6N0in%{}X z#8@%nF**Y$e*a`j1!!JyIT&6M5_@8UY)~%JaAt+K#kU{~*~rYG!Pq;ON%;hIpawKUCz`D(d6_L{9evD)uczsIg8@=KNUEmPRsp!vm8IqJu zd_vDOWA=Nl_!m~Tf7220Z@35eZK?m^v+8f3`qwtQza9OzqyOIyp#C58sU<{SoV5CT zpEO|;dpK7lSnYtiYAuCEH&XREe!x^1!JAXMevn@k)iZh4pI`p#X_#uI4O(=xEay*9 zt@REj#&3WBH7epK=oVQ_@OOIOpP)k15NTVua9}Bf5hg6lIVa2H0M!gX(V6m$>%Hdk z$cdyNU_4TlVhztO*QfVEV*bZ^Jf_%lv3@ZImBrQ@CVm4XcFWSJ8>FVTfZ=H0bsa~Fw-hf@P;~B!ipFwxNJcKDfhTegMX)2EOYGr z^j5n1+bs21dXA>-5o{@lTNtxq^_nt4uWrX+e_q@Lqum^aZPKHhA8ldl1J%||84sFx z-)(o6Y`9E|GipFhAFyz`9w~Z5A)l;H_K)vbl!InHZkcNsF00pv2zjqJ&hB&yM8=+1 zZFXF;kRkahDcZg1SWbqAk$D5AtPM!B1lununv^O2$*i;&1gx{ zQ7NIxfcMRzr^?WMX(1%-28S8-2FeoVFC&uTND*i%1(zu$9Ah56=gW1+P0#tQn&Y7N z=Xkf3O|_o{Ose@As1zxr_Yo)^SgB`wiso>R|qmg&;`%0&hAui zLd76;FAYDU{0kL|W!tTbl`8i1nz`1XxUK1`RRu9r=aDMpLaN^vj9nPTS>7RfHz)&? z5IPyL+zpuVw&Vl8>Z<1RFvcv26{v^ar=>#GMX4?A4 z+!x5IgTw_e2Fz@eTtcg=#W7rou;2_PhvWxQI0UiyOq-jzML=%j2E@@5iG^W8OE*np z<}FJ5%U727Y@QQV^?iJJjeL-luKjJ!yU!LNru*U`R@FbgWXG!mCdV3JHQUM9k8Nn5 zsax(|QL>$VCYGGnw5!e`R6MmPt*n}&Z4;L-$0K>ls)tsdt_gU!&I6a6FoJv;81%KN zwMtxvX}_>EVAda?gYmz0!5`}o^QarZA_Vkt>7`}X>_v!L00<53|5S8V zmoA^I?T-PsL-xvys*#K(!IW9?P`y(UtVk1he^cpl_Jw8c6zuHuh-)ROH#|B1hD;Vb z_XpO@)h{pM1Efr4zaGjnYeMZVHQZKTo)!K6&)-J)?Fqk!!#|x9?2$vv-uIz*=SU-k8LQ%zF@wqIog=CDfdAOUK@W~S?hK>R=FvT9U!Fy@|(JNRHv7E*w%*O9S z6rOsR2|82~$n4F=W$g>pQ2Sf+UI1(~KcG+#;ZKl|F=@Tbn_TX6K1HIe_x;%qb(d7V zQaRpne`cqL0r1+oe|(|jg+0uzkC$8-*H}e+Z!05FoHy8f zZ_H)g!me`b9>h!{Q!xts(cBTlw8o;!{1D`K@O%|>A#4r6A`Xj8y`**A^bLH~PY;^| zGi=4bV(Lc+jeU);91^o?Od+~Zr}UW980dkr{qnMwcNigW%__QrnTD`_C4@K15NIt$sAuUfhTb#w2b=q!R~4%zEbplh zNwMD1{*=@LA6`s!kpz0Ey5~L}KJ;Ed zWZCa@5fJ;oZ}iaLt5D>{Vy39>oi6(UI^x%oE;4pFvmDmf2UIjSWE;8uFlNF|ojl#~ zfa|_7!1CeWj8n*gxmB{TD0FXYGrz}a9r6DOT2+oO>$>^@&Yo$ct*gz2W7j_@3GpUq z1uEGmf&4OIRcPCQ?Tc%%Jm+i8mk$&d#7V^*d5(#*UECxSv%lb1{H^a)AHl*DE;5gZ zeuAhRn*H#~Ijj1+-Z|1a3vzjQUSkSNh*F`W;|@pZ6)bAHZ*ngZ`mXkG?KrRJ`aQ)f zT(-gndv{Zu=gKynP74J8@GS9NXdk4n?p<$KaZW1S5uX;2`{7vy3SDXmAV;1VgwFs#A0*fJ2{xSatWq(Zl~!C~ic}SjZlnz@s#SYF;ucBw!k20AvP;m$ z3#vN7k!J{CBWPf!t|(Xc^&FKRx2mtTvm(7UkECZ->a*{Jl8*C@8bo~_2xzis=N6d< z#9H*C0yN%(Gt;j6m)yWG6@xxaIvobi>E*@mqc4?r+&G3MjB%m9HR`}~gEzc;)di@b z`xi?z!O*tqg5v%%&g|(de{Y&*5>Il2X8}W{6imgTwUtz`@Lc?L%@DMaGg9ktLk=J@ zs$WY|megVfKMlJ$^-A~Ieu4V|LauZp8W)H7`}a|yu|S!JAopB;#6TWQok&JXJ4L_n zV^`hG)1Hw^o7xuj=;UUQ9WTNNB)Go9%Z)4e}*R<}VpXWIU#z$r5B&cv!pQO9)x>+p9~+Ru8Oe9 zp3<}^GElIje4+A3t3FE`}emup`LY1wXMFdy}R?<*atvqx4|xZkMi+WzTH@b^ca=TLAU1A%yFfgtOqO7L2Fic^i@Ru z)J-$|o&=?-R|2>&seq0V>_{djd_sIio0R=fo>7lQ}qC zn;nGii4VA^%j^rlo9V}ugWhyGwO|kcGWrQvb-Sy1hW@SE%a6B06N=TFC17$r_s6W{ zqVnyxeGbZi>PKdh`s8nv4_~W|FMqnvdXE^ZTz1*@YW@7#>3-57*+>X)u$4i=%l%a` z=wnk|@vrlTmh{VsLb++$sGwyEvX4s|&LZ`XhwoDH#VoX!6PG;NL?#d5O2Ow@S6I_n z|JDv{px~6F7J=4^Z$a*LQw#CC*7h9NBpgSy5Ac(lneRXAUT>+QyrMYWz61Hnb?nW4 z-(H@S!1}JjC+S48M|FQ|N7CO6fmKv$@UX;?NZoC;;9Z<7k&py$-x}7iOvnd}rRxvs ziK{=%T?C)UZvo!_5s^6ju)6V@ji5<%Z+Jq{=WE2Sx#(lvzJzVod`x4+-yWU%cdKEQ zQy06MEjbX5TD7W9h7NpTUygUhOiqYIuto?E@ok1pmNYNBo)%k~6Kh#I4@L?Q9!<>h7R^>#)#j`Sr`q$rB_^!3h!2xY zdNe^hkCUA=@*j3#9tfFFKQc6P(9^py0=kQN3JN1&e`&sXg8;sFrwfs&c&y>>?d$3) zjd*MDHBHr|JM?L6fhs#YYodempL~qqSB2~5G$+imtH@Hu>#LmNz|i#jDcIk2n4h>Q zY&Tc+y_SCssgb%paJJam*{+y0RTuO}yR{e>!wC51p&@kBEgX9J0bH-Uk|O=>GlDN9 zp^%jh)^%)pX;0qrV_pW2z0Kh3DRzWmfvULW|~E{lzkG1i)ofa_EPiz_NK0i15w;DmtHi9tszrqLZuq zzUa3JetW_1f$)1S_&pu|UJ-sTiof>-zqdxecZa_>$iHKR-(k{!Q>19rW|L5CS0~IM d6Hh~ae^~#P{+r~bS0LRqs$a#buzr61{{R(0EO7t; literal 240317 zcmZ^~WmH>H*S3on2_77ZLsBd_6nA$hQYh{YE$;4?06|(DS_%~R;uc(5yimLpm*R5r zyyJVvH_kK8`IoViy|VXOb4|JDwc@okmGE(?aZylE@Ku!Mbx}|-f1seC<6>hWuOt}^ z!BJ2cP*mh)^!-o|dvU_v*#`gFSPz`My>h<_)?(vd%0{Rvk4$!f6SQ@ti3Z3&e_z;n z_qDod?B~~}@5?4D?Z21b)Zr$C5s@@=!ZQyMEIEk7`*GomqrLp=(3}C~Mot9F=jjV^ zr?U*(h!=z}#r;ly_eTErhYcY5|KC@={bPXt@0RCBELoxA;!$R)|K8C)$oak6#{%XOLEekOrAECs9 z_Bd>&|9;lfdlhiBE^fcT>FvW!aESWcfPlvT3|!4g1ZJrC9FxCE&4ejyL`g$^SL~r}Kp{CL(O2 z_-zYH-2Y6YA4eQepU?G{E6Ph%r2dE<;6umwk+Qd`AFe-nMPipVTvaFgnZ_%Eb^8&ODyhK|6_8o!sx_t!%8X@bN_94 z?yjhzVGf!EF_s=h;4;bzm{oop1Jgq@HX1Gr6haRg!r8(Yj4qV-V=V?mnOA5dU)>`F zxD}NRMm{iupPw4ZMim&7sSxPisw~6Wun-)U!pt*WA1JX*?qx$Q%9mUSv#KPDz>3lX z;zpBGyd8}D+Mo2rt{xCUvi1X3TpNVTpY=Vl?rBS)`?P=|cis2@Ja44_YYisJn7IGm zCY?=SV6S$ZgTQEXAxu{O^!`!b&_GAuRGxvT$CK8oj`G(pi2*BK`6%HGp=kZ|A$;ug zm!xxVkgr3e4GPHvL?Kl5TxAPlte1PpiOS&Em{DfguZPPaIT$n8xacD{%#HdJj9G87 zK+=@#AOOb4Km939;dyqlu<%U1WT{XRrm#&`TzP33pFbeC&ct=0GN3&PvA8LbB(v=F zxPmicHeq+rSU*yi&!My$CX_RbA>IuT*$EuEKNiV+2c>BeEJ zhGX;S6bexN97f4graJ1}2P?c&p~K_uNX+_`I=9l9v+hey0k@I5?3|EV-wnsY4jA(lLW`M; zrwERokI?NbChGg}CQIs6Z2MlFBxQ`*7#iAuo)jOZn@>tCYtmq419!ISSEr$pNmN9^ zqnza0i@iABUK$iGN=8u)FC-!caxbl+4_rT^RM@|WC=>~d=mTu)cK#7!63qVXG~K7d z#NePvDBidD=^L|K%sN}YcG#vswIEX;&7d@Pj%@IvrAkPa3Nb}=ng_sVD5z89uRVkF zZwG1c8U#0@E%)Jdp2a7{ipZq=+Bm{?hip71rz~Bvk81M>q)nnmY}!WXhJ_WPY_JeF zTFyx}e+2SQD#*y7hIALKt-X7fXnFVfV?&C*_|OcURKEur5P|Cjk0C+DWJ2px`OjMN zg9p`sGT@5*uYcyHSgQl1aokOq2JW<>2lD8V^Vi%vOLpC0nBPBi*Ly-@cK=azIj)`) zhECCIkbk=0EI5yUUe#FM`2a4%j+MRPq_?lO!R5%MM&M!Bb&0%TiJ+0{ZCD}Dn}y_+ zgUK#w{WHFEb=Ekn$#TE@G}UuVv9ur*$YK7t!0B>#%ana<*-Xqg_V-ltnp7*k(r6SKS%N&QoP}i7f zP}?H)-^`M*?EH>l(5-#)SH#5bPgPpMc6L@|D2e`xr zu2{Fj!5Bk;$!uF=eM7_af&xM&SE@AB{i@&uPR(1=`yhddh`45h((#HDlF!1SK#4(Sl4NjYbi;SmRNcT=kERRtJtN^pdVdD~E zs40Ll1us9dkBp$Gp6#;sDVuY!6Nsw?&l%AAUA=Af$&KgJKyFw&i0`rK>({Rr>uouj zUA>Fo1_pwmy_iJU7)C8_T)w{^LtKXK4(2It%;5a?LS&g_EKn7acntT_+l#!`t(0re ze}aa3Iy#yH)V6Sh;f{&;1znEfW3PABo;Sl)!5R)Pn@!qrp`u8JQ*`p-}<+b z{z*|en!r~sL4+u8OHDI!qmSO?$3@HQ!Y*Eb#&C4y&DF& zwf>WIbE34{yCV2&D3sx_*0=F8d%-%Ft-*exxTPg^txNKqp(C?l@M-A9Uduc6iy5KP z#`x=XUwY?*hM<#*w?=+*-yhbWA5mq$n_Kt0y)$&}XTKR0UekE`WG248s+rZ0Ikn2_ zduCtIm8rsv{`oFv)$1N#y}tSEa#BaLId?Y`O9dy39oN#app>XP6rA%WVcC5YHo z)Oc1i?hBqB^eYtbZ@4yGdCexP`kk~&&w${=j|r#ZQR?zQ55U_zp#F~rn|~w;eBeW6 z;^bs}*iDS+Oty__ZGCzoPkUT$4+v~FQiZ99)2f2?LeW!HkK23x-RP!M_!wr|@WU8)$rpVGiFp;(iw=7QN7L4brVBg= z=Q+*}JHed{MoZ_Q4K(Tst%~-6;)W)Q*lgIuPD~S?XW+Y1Zx}yDPWyW@3N3u^_n|a7 zn~tWzx36|Jst4%yivuC?yNwX!gG_en=2=fjEW+xJ%Q=k2tI0XmwCv3RhBuji6TjIR zd@ML7bsZ=R`+3rtrRp1XbrrIzP0(OyKDMl&2NuR!ntMGk7Njr8xjSoXODi@n^JL@Hiu-{@uf`s0`>T2PbXf}(@dG~i~b5P^*^2+1#5AS3) zh|zd=wr$9+N)tL~m-cx&?hEDK9j2I;GKKSy6Wst|x*K}GjylXX$J7hqb$r|N%QW!l zXVtr=jGtfM)Cc@iee=x(@z;RE=(`iETCnlC*p+qH{mZ0GLup4GyFJEs>k^l-8*N;S z!6jYkl&#tThqMeAtPLP0c7%DyqE&r^9aV5TuFDI)Wrl-SE-zp_*k_vcAKywxd6#H+ zF>EaF7|pzrmre|?#y53E`j&k>-iGDj(dd8UpZK9m4feeDSDbbexrlLNeoA2xk*GL` z4pW3>5C2trqfdTAv#a>-@xXKBG)M z&vAyOI$~O23Kj+E9LM{DjyC*ziKMh=5oJuct+CMzXLN&={LvDy9+r{sSY_gePaa|= z0M(EVNV*{B!;3R1Yu(f}cq_|7%_N!IryV9OKwVr@L|X=o_oMEIpdc$sVA7I=NQNSK z(nN%Is8M~>+VkDVVxME%QmE%^0(ZtgS$dot#0xzTlrn8x7+RS$cu<6R+JuM;`Tu-C zlgRl9AHfig(bmSlim%GGoFyr(3Gm~v>nulwLN!9ZJs2RpLs z-F7d6?!2Wers`O_-AGPudMeuf7OMh?Ls*H5MR-#CTAsDKIxPT(-Wj^n*P8)dS7r1b zFg=*l_EF2Pv5&$Sx0AsI7LV}$3HnDhw~hlz#p7qOPd_6(HhyS+?5)n`VQjd5`O?n< zkFz!1c^ywjbLp`pbSGsmRJtl#SUAprk(BP z69~Djs2Y+~>Y=_7<|^DAz#h(!bh76vcRFMsJXuy#f1AW<(*Jjm!qeu2`8{+vN%R8l zLbumD(V=1!-1t|`f8oJD>7UQE_`q0#Gu(u>>>&CVVEN(Aw(z9g(|hPZ{P)WN(NI>X z0|i`}fgTa=sQXLZ0`YZW#3# zaG^jfuHd)()Xn||CLX3z(7nlBqXQH!W|c_=wyg}Qe@wgdlA|?l8$Y97Rn&V8%I2HV zEY8kP8I9WJ*FS7;%X{b=|6j=^H4Y*mATR#1;`8UU%1YIQ#6*>{*?a^9h(|Hb&Q5K4 z()AdJO`!{+w7EJozIl}V`}gnXnws*Gl1OjXVozi(Cj%aAGx3_SdNn$ByR0RAwopO_ z6x_KuMjNamVh8I0jj(#9!%mJM`yw|Yq0hq$Ubj2h7mLV!HC@!q!W5pg@?iDwN)JZ&yseVz zNkX)mu=xntKWfy6l0~8lsC}Zu(jqF(pefSwY{v|t&PBJhY0HEIo{c+OSq7kVJOh(M z_1SHZ`2Feyt<;+At4vvWICdG3b#PEXaAKpUw;u)rS{sw6Y(X@^iJO~DOz9A$Kt$a8 z!E7=Q+h20RRZyioTDO&qALsiP^iLfs-+Cv#EoIz%To}2AvcSm~6;F?GE5mU@5O#zu6cNf zP4U~ukpJXkYoXWoOLf*S%%6t8A?wO#FOU z$(EXU%3^soNSpl|5F|l((WUi}=BY|H4Jk>Q`Q#H2OOn68tQC|a<}`}qt-5rW*mgV_ z*jCjF{r0?3aqDR)we_@5dOiD8H|2eAD5df5x@#y@a$Vu|fk&6D!JHAKA$&8A(7Y6` zaIrUSoH3jOvqUBW<-IC9&~n}M17^`ICL`^zDHL`=`M1t^ayU*@eyiYT+=|M9{- zC@DrbiWwMKNT|?6{crGzg!})DK;n~zYJ;}P$SqlvMsB?!<}uZ+J3yfPy^V(VX0d%1$pF{kr>Q&sfLt~HlF|G>s?+P7LB4a7g( z?k)MgSmTi^Vyqb3@P8FA1}NfmfK;nKc$gqaQtE$Fh5(8smzd!Dt4*hg9KIZZbY);a z1S!H@kl{vat8QH1gSolwj#t|r9`+85Yb_+RYu=7&h1_8`yKadPAXW4gA^Uz$m&SQ` zIil?q7|fVCS=;>~tj>eWn(Oqj_SBvqh{M%L*gpc-V9FWjwSBJBOliL6Yw02wt4HQw z>M+W_-}*GwQ6o5*pcuQ?4w2a)Z8VUkZvJ+ZfR7RB8#dA+7_yK`&&x}6ka5P(LcRCd zx}Uq35m02oeOu7`IACriz%{gZ4>+F+O_~eN4~_?Du@z%)0r%uFVBwTl+Z4o)4|}{{ zE7R1mSkcgkF@^6QK;49fJwDPzz}hbgi%mH4e({BH?C5GsyRtF5F;h(dRa7J-U;pqx zpLU8NbBu&@P9DrQ4gnxz9$eU+0Lre#L3T>8sff2>$0NQk!wBt7O6fBelTMj)vm3Aj45C`>T8e|`G#WkwJp+gB^08ddp2OW z-8p;E@ku?>$_S5D|5Py{kyLb%22aLf8BOcc}54y7Q zKP_*g58OTujacM1?^5dRkz2xXk8qoh&Ns8(nGg`nK^oHPbFtIuBym;c1jacF8(+(U z4!3T4Enhy(TlQ(vIX1lL-66BZ+Bbdne7$f{srn`aerwB0^tsIKHrBW?h&tdisrNkd zc=Cq~b)Pwj#gR)e_t}gkiNwm{<-czB2@xsG%#&g!_JTuJUtVS&!a*LgkbI8aGqwK1 zkl)|_c)GlcKUf^NGXZ}qNaZ0K(j3l-(|Br>TspIK(xW0AGrcoM`F_E?mQ;Wli8M( zo98Zhru5&!28M>-!yiwWN#Z@$?(XhI4TrW_JFh>9#os-4x~jZPm3&d+7}8I~;}!t3{17!) z&e;(?6&ge(^>l83J8vB*I#*YH;4kt+OUl5sF@RV2*9ZP0IjPOL=h|>|v&j7Y3iBt9 z6tjP^4`*VMifgcB@yDOf+bDbNe71~gyHhMv?iO!2x9Xb&#lG0&(dZ;G$h9XDHvOW7Ko>4d7>$5wo3#Kxdlo+n$ z$CZ9}YuZxv$vThTW~)mlh5o3-(hbvq)l%`E?p4K43&Ja22s~_)3P$O;Jd9N*%9=Nx z5r=1B&?a&(wtw`u*wagbY=+!KSEh1QEj`@2ao8V0#Pt#VhwwaK+aDsO7r{S-vaASw zW(CuB8>K^dGj|k{UYcqVZgHyq{hQNdA$eQ!5nv|+OKc0obDaFmV>Ma-VH}5UyMTVR z*KzKR`Bc>*>bz82{L7LAg+x0&SRNy}_jHQJ?;D-*Vy8-6S;~;2#`c%|iz+7;5!y0f z?<}4o7SW+vXYDun{n{&TpVrn?HEkuF7pi-l#3l>#7^r9%a+K5R8Zhw2KEE2X|MQzX zz+a^m{$-8^oCr@!%+g{Cb&)y{J}#C*C)4=a=HwJtleLxPKwnIzkeOMU;=a*l#y-7) z#@&AR5td3@&R?Q3+;DR~_9Z##>|t)oqmeziyESG>1MdqI6|26m3h1&Vqm>I)4~12J z$xNbKS6p{MGw2E3NW{P~$_-Bkkfy(++SQ)Y;WiV`Lnyr!{Q=!E7z|umC$9+l#hL4r zav^&UvgaA>HFv4F%r#Ii&_VqEM4-!Uuf4sStz&5Gb+{MVTJygmDY8RK_V4QIY%?*i z_3tQ0j<`SH<72Qq6EQUnji|?O229M`q@)357LrnaD%xSB?tVKuHD&+wz*hOS5(0?q z>=b88$Iekl294{jc5Xwn517%>(JUmM83^E)_qlQ}2tEItTG9@0G&vYDGpP?COG51E z2-CU%oP$ut_keQ0bBizzHGDMH89|#lsoZi`hLpe{u`xRXUi-f+1c$$pw-(i542z5! z>rAol!|~8S(kK%=U`>8`M%8Ia53~r7cAqFxGK})9!N#JNw{-gzXVORs!!h{qnTuwc zxx~ipiE?+FiqcGJQQYR@-`cx&S7`=oRZi>uqwCGzi4oKpBd|-GJ2M_h((DPNa$rXQVQ{f`3)6y zen#onE(DH3PzL4Wu#2?QkWNSE?OP2tXi;e^906tLN9vQsGswoL?iV1k+-4;@MNU?^ zV@7ttaJj}?N@{95tEgJmal{AgKb&@h`tPkM?eFC%Z!_T>ZVF^S3u9mRo|mj#7s{7l zvMHGUWfl^%4w)<{Y*Hi|Xs%6`iC~mMEuioBz_|%-Y^1e@tc8EnHU;4hB86IjToKB` zPhi%ru4&ekp46{NvJ<1x?R|s8Up@e42V@KL^J>O?NY%jqvH&V9F@e)Ha@wQ3r_eb2 z+cRMTU4B+Vv66kqRPTMU1JT-&f|?S~S)5ppl!E*|l-{(|zu-3Y9Ajl2<1~*ST4P1T z^;9p$1aKBJLFyRUhP&+gooV&B3fJj$Hhfg?49b(&wx%F_->55g6l9-3XU!i0&Gc+< z_;dW$bzYkg6orGcSJmw-Xk4o$fEw;BKOgB2*WO1|Chn2&lG%Ha{%WLK)10mI=gN@> zghF1-om=yJB5V@o=Se$>IoU?)Ip0)uMyL`OhrpuiWJO@8|*ekgwP++^LfVV;evtW?3#@;v$GL#9_7*u($?1KNV~TG z*oh=^WHn{ z<7xJE%Wd5W_RUy(L^+^wvnKY~q7P1TI1qaIFr(NmTAj)z_3Ybxs`hWpf9lr(}vV0sg|-&w`d|zY7gceXWb7Z_jwcPs3z?-oNlE zPc@IXM=WzozB8|Pl{z4;X2H@6lN;`;J{&Rbj~M%AqdLyYp1TuW;WHy)C1*)A_MuLX zDrC^p!{r4pidb*FL|)u>jrCIdS9sZDRKugoN+;BEh}H2|LuPeRld0FBfjYe)IAzGs z>ybNP^C3e)$CE4*lj=%bc(heog$pmcnql?|sMR?=3P*pXtDf#L`D%~T$}YLzJGFXm zBrTCgUwVMry405)@$mX@a4vJlr#)#Y%3hEjIakU2@$Y>Nu|E^kunChgp;b@1EVc*N{V0K7!c zDM!lv6??!{`sUB%542YH2R2#U`5Mmab^-kPf^_coQ=dY(bA*zMNzvZ}9N*ayjeaec z-aKsNclf`!MQ1{9uME=+Hox8Y?`VpU-cR2HK_JlY*$)S;$L-G87-%9Xi3hL$lPS&2 z@N#1eH=o@=F{acGBvXHVKHZ0ga5pWO<2V^SKEK-7rOCbBHicgCf~o9QDN zuon!sx)@OI2N5OZHK1+DOQ9!(;FKObAcS^p(W56!d&c`bQVzbB3 zw_-cUkdgJzvs>PYi`3&CVz`M!Y-ev&b>XXt9R~gG5O5EP!;{5;p%JrWDh_Xfi0j#x zsaD!!iw|nha56?61h78#bMyFsVY-DVma-Vgk~hka999IHoHNz0*r$o?;-$|^UApw` z9`WGQ@b_elv8?s5V@~kwlL!5W!9nU=I;urYhRR^GFSqs#_wzK9V>bG zJOga8v_XC4+J6_qHiP1@%N?1RT5IH-riVQ|dCZC`z~0%UwP=6dXG{(VZTirk)nFH6 zYUc}Ms-E$c)iw|$&wZ_H;0|o2vqp>>bDF-+{JxUPtBIlAX(=np%A$BrwP%RwV94(s zf|CjLT59i#^y5?;+aR{%23bzw1s|@j#qRLKUsuGje*KEVNZE|76b$%RMvR|wlmXpi z&m7roIY*5&++)}6N^w-@GCnvSdWmC)Mw*6U2M(Zd!+D|BvL_d^m2 z)~OPczFol)7q&TUADTYs>%%TgV7l;f6#?p^;?De{UdG1xdvs-XDziopto2v6n())*pqtXuVG~R*#t$OsYI7kM-Qc?(pSfMIILqi~L z?%hKw3|TzCsb9WzJ&%S@3!yB2!|bAtbJkM3!Sr{|PrJn=PiJB@qV?Yj?lKL%4kk_t zAt-Mcl;5d> z*)7Nxn7-cj_5bbOJ)!*yukmbP$i3d_U+nE3QksL)?5>1fgQOf^+Oie?5-_ocW7{_t z%+G{q@;wT0z8ysNuzpRsZ7&iDn0n!>@w8!Iv?bGG%45uMe;dFCLV|l$cgflOuFk42 z)d=`0IIZ@VGXamM;JXh|qH5Xv9Ri4}e^b*L*t{1RWL2h3Nv;P4yGG-vp>f6Sh(Gg> zjRdLT=X&o9)NDP%*nNEYc0Dp!uqaUyM1`Lmu)8x^b4i&*a+y9-+eI0YPvnf|zfRoc zGAn(w^mzgE|soh`gE@(a$*~%Y)bP)Q~ z+1TRS-p1CVg;W(Xu~S;c0hiKIV?-xPU50E*yCYMYB~3N)W?EhFZSh>I(oz>UzCWBQ z#ZuPgK!inmc!a(y|7_CW{Pc6M9NqX`Ch6W{xqXh;R_1u#+?cmms&})zsY%LVX{i<0 z*xvjpbZ_}ausnX2AjLtG<5O3hZ37!AjuD0&_luCOa929?|Ldin?kg)RnR1k8S69{b z=1gbzfv&Eu+q=6E$n;bR(%LWg=l1^|+MKKf|M>aSlW~9)N62LrxrKJy4{FMaEOE6Y zn!;<2-Q=TOgvLHe1R5h-oyd+Ltz;1VB;;g$|L90T-cN!_-3VFS7Ch}2++GyRK6&f| zt`3Pp9#-WZ7(%#S!D2NBJle1H1cLIXLjDr4Wp1DL2*3PI(S3utH#QE62u_gyg2weg zO|0pS23h@6{1}1lK>t6kR7RetnwS_inP%pRITY@C<0BPI$(BQNJix^ntxnEmaT6KG z*J=SkS0`;~&mDQZjy#CKp5Ns5Zct%yZP@WVxwNNbBQ_EeK+XxWcfM3>iC$e@ z4JxCgr2Kt%@U5b&>(hTWvtzYPRJ?B37X}M7RT-h^%a(}!{?BlNCrBKYIZ=rd)?p9^ zCuT9Pc9eRuED?~j42l13Ejw_f3Gv=n@S4mHY41XGLaD&MaCxt&AMs2i!B~Z@X~1rl zPWd{&9d(o34hONdv6P8eBY#^`|6AgWc!6?925fhjwkC01ujUXHi8NSYrB0`^Vq1rw zHT!Yp{_GY2D}ty;TfPdCjO_1t;dkC^y-oHaeweSS%Z+cpG?EWEsvT7HO1Qd&u>d^jFo0b zH@EU{-zds}-{ipx$R29bho;MG!s+Dzv}(aUjh+u?vPDFa1%vJ+#C$oNTY&zalWXf0 zp_&6eofuS;i}v~@PG)9aCuiqcPUsxx91|tLGMBAFq0iFG+k5}yWX}at7O(YRO-Ih# zy5#QLK1-4fC(Q#l$K5UZ2guiN-@^$XZYlmek9MC<>{bU|t_}C%LhhGWZsx?YF8;(y z^>)8J+HTCa%|0LO?n*6LzsEJdSt7T3Xh#h}oRg}FA>q)vB2+A_8$sTX*v7;I8&M5Y zR9ln;=(zYBvq0hi55j`G+3Y3gX0|DT#3Mi|bl2%=`)&j&(j-D%%IM+R z_(+-#6>m&n#I9YGqW7;Trz2;{McYYzg#@BCa!NwuwqtFmY6z>n!)2iURH;uo0;*2h zFW@mpaBGV{o=3bhJtaIRDG)J+z2Xz~r7Z+=jNG8^%Qu)YzU_O7Gl{EzWnma%^X^4| z9~XFo-@N$smQcoxtKlz@YwDUr3-5+$n*v7M+(A_}j2OIT>?fE4euCt3K&4H_}g~%8<;sDz|f&2&WcN1?HYbit(3f4?tz#H@L z87ghl>uZS5P?nu29aYa}ix2UG3MiZ+A@X&4j6b&u8JX%q8m%XS5PUH!nKb(cA=FNKy9%8-fF!`fp0~hI~ zTArFLNjBvRB&Ped;?Jd7UT_rny3J`#o$bR_#>U~kpVTmcAiw{F!nlOEGEW7Lk4qR# zM`N*8I}8Oqw)p}ow-TTAvu-z6@J565f06Il=i~E>i?;0yTbIRFVmT^!d2joN1Ec~w zP|wLZrW6_~HKCN6j<9P>b6MHF-4o`0{d!bv{g%M3ZHMq?lj1qN%C!V$JYeEleAxtJ0KucyS)R4)`WpK0X z!>QH-O2lOj2fqF;-;yJX0|ywlj6j3_#KfHFK8@5O)Vm9~){cH29SWRfkm*vEzQiK} zMNwkd zi9m}o_u~8W$!>h847*T%`s0gUdRYa$^cfj}gi>b)&}24}lP4dd%3?A(%J@n#8*r!) z6D(l^O<0VlRWE~RXeZ#Q`o*Iyd;oJ#Ran}QMF#U8KaKnYX7A`|!~08zaIcQUV=O6= z+5>P{)ACj}iPUsx(AxOr^OFv>9mVDaO&rR9cn1{BQEkfdzCHf9)aCarsp%@OZyvD9 zGKsHT5Bh5XjUvwl5;rL0UpomU9s;LC*Js!2V_3)JGFtGBgx*>3PpDHah0n50XygWx z2CQ5*JHC-yX4A-R6u^m!ips>m4qOaFjdbmFHLe$QF|N;QD$}~V650En6)xL@F-(rp z@w|sBaKg6MomWlxn5Od#a?jKW7S*foBRVL2Ec-V1q)SyMnE1uyfe|fvig#M7rxnpK ziO%%i-9PHdl0hOV^u(R=YkPQ7KCY@AebK>A$NuIKU&5Y+lF%mMQYUHu`y#sJ`CKSj z>hQjU``&cX8xw+;EI?e|bWY8TEs-Bb)ms!&-B6yVlWwc|Dr1+mOI&<>e55D-7Ts^z zw*eV0ffak`GPbp~0SNKuYP-XcjD>uyYhKNyCi3TUnws4G{mh-vzCKyWpov(&jav#w zC#OvtP>(hnfy$Z&h;rLvOy*6E>15*!%q_MH_kDo_dQJhADdpvg- zdfv@v^iofB5|J{XEw1*()9vcUf7QGF!a?EPI9=n{ZjW5(XSb%_k4><$E0UkN!sk+C?i5}CE;%p*+MJ{n#1>mb^i5^D>1biK+aFF?wN2U7 zeNem|!B_JrMPGh9oT?ofWoWRS_V$xPZ42T04^>=p%~%O87U4@`v-D_7e}A*3Qoy-x zVV5gmXwrBUaMg__)EvIR`jJQu6*PLj=|;_DquIOTgU&c)EGhiOkvx2oq2|)Klnoug zTD;gfBsFTpBqQ^N_*RBTQY-2SHFm*KqQ_!H!Px1QBpzR-{RZ7ZS%-TVhSHKsMjfbm zMJYj7UQ`hADd1BeBjQH_cs!>CuC}Xgz30tS-_exXGP^+(ZxeHIhY}qRjm|68u`iz*^8Z#!nmW_ln=54$SAuPbY|O1GJX88p=2luG+dJ`$jQCS0 z`|~JwKRe_{p=D2@5BNpWq>jlB%gKv&r7R!OetQ3spF<7XSUO^Bj)3`v@0@;TZd*fn z{ephJNM87rolgs_{<9sy`*Ugy5uqM!FK&L^Ig#lt_Af}d9SvzvM4ca3j;wCacho<= zPEJoJMs~SIrlyd-!jzlB-ydmsgz4&R{yZ$MdCT8FaGEv4fabW(uF!();XjL%Df32I zh;og*IC-WAUVv>OY|}=U_6;343Ek423yvdmDLu;l$4 zb9Vx+ZjzAgAfDnwE6q>rvSRoCJ#bFoOH~yMEdjCvij-V>w$6TE->a}eku6Y>=*?0v zHX9k8CDebvQ1J6*kSQLI+;HDOK^qx!0}8WT^v1(sxP;A(`mex3iK%t01 z^HE_Y6RWVyz=GDVaOP=ftgC&Rx=7UUL+f_}FxXL@65qr-35`~Xzq%45d(7%O3thgC zx1Hdczq&oDQWWa=*%e*jq|9g}Rk$S0Q^*pAcI5DCZ>RoYmM^6KJAL*_kEY0gwNPDL zvM%nC1FaG?!%0W?lcrN;q5~-2@>Rm<>{843f+SAD$o>$4Ts_|TOW=pjC6ZT$rW!6M z{YlY1(A+^9a^1F7h5!p5^l(Fmr94^NpZKMk5~==k%IVq`D+ko=}9V>)?C4 zkd{kbx2I7lPq)uG!y)&Oef!Y#RDD+eE2)sa$zz?6bri980V!D7y}ad#e^(p6aEtV0 zWzTy$q%An~p)*P@W?Z_oB^D3_$35STPpq!eZqi(fE3x0^ci$wMN=i%$cm3m%ysxG| zuhUwnE!iR}+70~Ukd|Zlm zP=^#<+%my#_Lr>WiN)Ua%7#SKVY%|2DX8CROQr!$$M&rq$yx21+~3`ZZwHOx)xj8w z%we$$gR-udLw0((P?@<6)IOnMP}#JJcMudV_)~7ZODF(cXqb{){3U(-h9Y(DvhwRf z+;fDFanc!9HhXf_0}wj~H8QEu<-DHc{ULV5o3O;gK6O*>$j?V%=e*S{c@BPYU7s*i z!R>^H72m#R4l#$!bLI`hf(Y;L8sL&`X(Q=JjadFlK7LWb$<_Mr8q>U|9C!9VDorzg zyd2!sGkT|Br$$$jYpwt0<+-?~l7wQ|jSV$A{H$48G0V-sH;>n-Qb;ucRt;0M#DuugX>G9p>@|9Dlu$dx0z zQJ7b`I(Tma&~GtUKH-2*VOpCoOdpzYEvp!SMYT_bRA0R(MfW^5qQ6oh>7wB)x|_!m z8+-c_n5^?eJfAvx8IWDgSwnz&XJ+_iMSFV!$E4Y4*KYsnojlh?1)2ASlA)CdZ^$`x z<;vO{{cH5Ifcj4nhu(wm^yxM8E4IpvfePqVqIN^ z9Qn3ipfYWfEl|(V(0dw~U9M??YY{4@i_=`yLZPVt@{EI03Ec}=pQrQ|3a*kjOty&X zgO`z{D_i8#%fD|48fpVxH@lVyl9qq}{_*_W%Wg7X#m$WyN!%9{XhA4To0`a~1;5QX zXntaHkvsaF_!E;?e6h8jeWS81>!v|!s3sV973?4ueqHjMB)EsR8ThGBHO3Lwh!)ql z5Ool&qe`qbprWD0>YiI}@0a8k%B#qS5Sioi@Hbsl$@4UEhUwt&)-*4Y;Rt{2bfg|1d|HGl|>Lk6)WJ6Su#rZppND%j}0`vYr3Qf$*UU3)d}d zLYI-cFSuol{O z*6tmUx#mVJeed3d=dY#(k2wutJ_uyw*Y)Zc@?;kK=3Yw>w7|PqU}H@vH_(5AAvPDQ z=u3Kcpc~d0`T77w*9Ce|dGYN`sI4orNT)&ARpi8j@L001h$^RTSaW2uz@638u!DPX}(DMz+b*e~bUeeOuS$dhC=;df+XY04DY^Mfe(R(!V=$1LU=m&p} z5o!m!iwz%HpCk$A7SD}b(R@;aMtmi3fjQ#z_=uL4lC{cYvAsGhGjVUW#B5;mUHQ~U zthzTo$WWPDU>3+P8i*JiPj#^;0!ao^*{#Ur64Lg;yCPIxjZUJw4$!^ZnqHZfcGZ04 zyVE3>1)+~Usst23ToR?eH$Rza=RV+YXO5;gKQU#-S=^R#ld!I0O#D*lQn zEjOBfQA5f8g-*}UxrslUn6Pp9;OD=aBd6H^RtW1iSEclR=s=P%J3TzD0Al2YnQRePE7syuu599pod82l&+ z=$8i%M$PH}Mss&A) zT`(_wGun52DN{6TN?-6pk}4{2n2DMAo`ZCKB@!#184#|iB;2<&NU#D1!d#XSB?z#I z`fb=1;X|9>Ec9M{QUu)rr|kUe9$h09=CBM$^n7l39G+abZrGc!-2ul&ACE&wN^>tn zn_8RJ76thl4%4boNc~6wbW_Ice^~$l=#IDA;=D&#CD;}OtWXEj4);O;NSbmHsk~n7 zO;xnE!aaZe`gOS6l!z=40R7U=6C#6OW~iH!{rrlgdNZL+i*)zc>vz1$1KL?HGCY&= zb(3a|2U=FO9+uwX47ncfkF(g&6vBy`kJ8e!5VKz>PG{#S_Ei9po1j6Feh3a0nqM%u z)s&kpLY9{sm_(91Ffy*n+QN`n&2z~7QY|fGchKvti@&FoVbFW3=)tA;bjL5s{JsHv zfAZV^)UY8}XIxy7Tcs*#Y=m_k6HPQlnm!>^P2Hf8|Kv*T1B^V5Sptw7Lh!y0N>B&a z;IzB=Jw*+FtJgDY0Hvaf&xjL)zsZLjIk>FlB+sL2V}fi?H9P0_(xA&~;yt`RjRnmb zE#3oDZ>#JkbJ5mD3Gn%ZE<5U-ijBsasJ1`Pw!#6?%xdskAQ-QAM7`iT+ibLed9U& zI9L&fD=h`upn*WfbmdyNBmP!S3PV$WOz++CV#`ow&qp1Y8Rwiid+)Wbb*;6o zy+t_n!IJdBbo&6ce?VJQMs}+$vH^AQt+jr#&gz>Cp*((?8%A8zHmMih1Uu%}sq*Sy zoe;>>pVrCKcl_cuUaVzY?lHTTr#c%qW;Lz9Lnip*Ix>f^aWpIte8JGhMp<^WI)4yN z9nNqKP_*~%jT~w?p*v|j+INqwne((iguEq9$wT&nTYERe8lt-6iZIpTz}d5!S#Q?y zaSZ;>{>Iy}$L~~xY;Fujn8#Z7{9+Lw8uPvCVaC-O&NDfb+gmh??gw22n}5P6sA72E z*_=kVQ5eWtX`u)6ZRz9f=bMZpO|;dGS??`V zl2ellkY2x!Y~*Ytd1e$NBVtNVQepmF4t*nl`uRe}sC!J4i$Psg`Zq=1m%(j)(37lb z-fdG=IrFS%B)#0(yLPpg+Gm(dvf`>+)7~0Xe6$7Z-wD_%?2eSm? zx3p^R?y~ZhkmVsODP7{OK;a+Zo=Ip5nWwdV56ha`vT>H&zB*W%L21M&7TjxV{Rjj+ zmLHerLnta|B~`R>i?H|4VQf;S*A4^^ymWN*^A)~dUp?acV`ml<*6(nCG+Q;bB!E^V zZ&vCxW9aw3d&TI4Q#>T=qZW(ina{VdMM;K9iG^wvKyd!>NzAD}Je1TQk4a%RvO8<3c$(0so<%tc5dN2Uq4M z1_-JPjufx{rYvm0<-2SVHwGrAGXzA{pvXa~V8WIR=aDrRN2>W$J&)A$v_O%3MQ{Eg zWDm(jO6`0hJLMM;vVO@292RdxuwC;Xx|Wj`dGb7AM6*O~^3LKe9qHts2ug}5$e_T1 zCt!U>(AZ=z!k{LW`p3c>qQA;NGjiuJLQ!M;xA>(f+mw$Ga9cNe#qx7!3sg+_5K@U` zFb0`IhJk*&a%Zm-t28LnT@~l3%!o%=6*R8t*j9gz3)YHqF16zk5a^$-HdInmvoc~N zp`(tsq`&|vHqaOUL!eBSwOin-R3ucoMn>%ko&_C*j0zo^U7&8!asE2FBP&5Mv9)DV zh{%Qav$ywauj8&pDCz{HsL|SpovM!upMBV3$+lry)aMk#9zmD!-fJZnACC}Y-_zJu zVfmHe_WV@8SzcbjjV53IM4R6Bv+RW2`#3{uTt7Ne-%pJt^n7>w7;7=R-}&drtlnFNRa!}+Ee;P zT0|0mSwz7nLeeGOuoU^RFD2>bbA^z2%GDhG_U2@cbhXW51BzM`l5C22&fuybmzD>n z{k5doRSgX~W5O@!>TSm985nZPa=oK=cNtX48q2LKtb-`Ry0V*H2L{Kid&1Jrw4r%g zasIvI-@Z`*)d3ZgnlvF->w5ch9?*0S&;=0YAmF2UcKK&#`#p6ej~EW2B08 zDO8;@Q$8oFeET+?{{=H8WybO8>D0tGMHv}XF|kkCb#>3huQhJFtuSw6$FCRe^%u6u zv8MT436Gh=rIqPrd)mqEHkBrR?c+Q~g;;vEjih@JsBpHtajjRQZryre4;X%LZ}bX| zVA4Wm_@e95wngVIjoaJ2|2guVO(O?g)p0KY>;*J%1 zT-SJQIE6^`Cm+3AEv$oaiXEb12DQa5qLnokK*-k4HK+8x@z?SY%?5((&`B7`9)lS&!zb z$++TLl`z2uDL(!@PYGJ4S^1il-b}(xqkZb4q?Jbj%xmza2uv6U5OWOK}GaLbj=## z<4wqVu76;Xm93f|PnRjphQ*%=+7HW4G1In^$eJ3$ob1gBm^Ek2P%a22NLs)pS0<*= zJm!((@NIAOslY-cbF|V&KXa*(&N8SH7mUSCU44_X(GOoZIsXQT_uCUw2_}-r{KmET z)mN<<{BJWezNVy?_Q6Uzo%tNCDKMVC0QqdY7b2=JUp`z^4ACy-FA>jr8%deeaYdTYWE2FlRe%-_x^3L#f-UBqa1vJ{1bc?0RzWk0G!-;k%r2L% zD-!xEE;djpTUT3;U*si00!gmvRC91(ApKrOhOiywzPnO<{N%%B3{On z=h&vK5Z2WVvR5)P{EA&VYam7Vdp-z+Dx|y-~HnqLYtOER&Gle^(Kv(9JQa8Savb`7Lsc3y!L$K7wt35rDVh|f=XHGu#? zU?;_gmq);+qo&>A`EMUUfDb@y2-jmDJ3G&9qM`z{r$_hiv)Wahi88D6$U`aU9bAn# z9C7Sesh7HlrJb2$hF6!0Xqt0dTD&(mw}{(n@P&{0IM28!&z?UNjMqV#ZiO4ycch&&qW{I1lXS5TOFCB|{M$zGM- zRl%i}l=3L;s6xXp=y(J#83iKA3dadWeOi=)Of`}DP~?@jYA>`Jti8-) zhU45c?_n1q_id2{e@{C9#dO>GDEb|8H5VT&pu(Qy8XxUcLbA$+kSca*#cp&K+9Rht z)@S5$pv)mAfm<^ZT0j<7;MDxf-t&NI_iEJ`SW93S@IKC}IXDn-q~QWKT)Y}N6tpGd z6B0+tW2g4w<3KBf#u_lIsN?$0EVumgQEg%=Iez{C9j zP}zbd3HF0dM1(MJ93eaKN*A6jkS5C$rWN&3I{<^MW->tf1?Ee}-+0^f7V(8-EbM}U z|h_^-tyoB=vZ!8@H5Ho+QXYK))X{=YHReXB)>fJAel}<2LLHaYN1v^IUtid*Ec@dmfsJ{WlKE?5 z>{}}b&TKI}ew=d(+S<>2#Fq|W-~`UftaBPq?UptOB6VUP*)CvHs1~zom%hnM3!bJb z!IKQ0&X=@AqAKB6nk}d}M@j(3q$D8uS>j+VROo=hHtutPH1SdKq$XeV=NM_ed_F+S zeol}gDa!f%EcrBQCave#uB3?_)ygSZ8t4^ej>V~Cgogp*_Qw=e@{;Ph7OUq7Uc(3E zzQ`$C+UJ+fiu(vf6^))c_A3UzIkhL6_fppVl1r=Oc_yYMOHCBxOdf7Sm6RhSB;tl(M&;R>n zXdk!oK|9v@Yhb@DKYv9-Tc6v__RLHZQi9x< zF$Txr_y|ly`0S26800aiazdiPz{>Zxg9CX{{`olxb22~)j}$jPAtKaio6!a(Z8m)M zPeNjbhRKGiOY~bOueM{zH)*{djE1BYx59s|KE1f2+8st zspz@B3_5|v`%jN$`!7EKSad7;>vUsiPk|-+Uw73T#0~1KpVJR! zx3#!?aD0zq+4Day<3F!?#x`2?;ruJz`@dgrldm>zeKMXUNcewzpsca+!_?WG(xxVg z|Law4ZBmT8>;3x`&YAt=DRAlq{Qp1R?2O?8^1#lZBgs=#ohPGr>jX4CJC-!|e|QPZ z<6i?FH@nlnb+Y8r4BY)cukI?|!a0oc!Gofjns`ZC&=KCyz{kVO3lRL>EJ;#jzIzQM zCxDt+Sa>)#S`eU5Kqbb%2y)Pos$yWk%4@BYn3@`9mkBNT`*#Pkh;noZuduNc5Q>it zKXAcdikh0l(f1$IKYD}<{2dHZ3eufxYt~c`vc9$Zid0v>l%+R6Jn}vBQ8MCC)D`yPQ32(*)8n+4nO$Vf5c)Sju z0Y<;oW6v4{BW;I6k3c{YZpAt@d?a*xRY7*oD}$RMXWKr!fiGB*QOd(3AT^cvsiBD1 zLf|Y=$vU}HMf&;{l$Trc|6_2u3i^MXodjd*AJ*!ss=2#*Q(3Lo_^H^RsIfnC<)ypA zY&x2Gt$$Uqm62n8zDxOOo0yDUogaZ(@51WpSWq)o@9;~Q0AkB5A(8a;?YQ|^ zcEw26dP$p6LS|-fUE!SxoTfqm-KO5(840&h02%)X2`h8E) z|Kr}5p|Z#?yLud|zM!M&IH1DOJ~P8UJvgjytvH)-DJ^X|_w8vrtC3h4te{!Nq{-;P z)JqTMtYQ$|oBHeWgB0{TUNQ8|7!7aZ1)oOuQG(zNjI+G6GdIwf+n)QBb-CM`7W=@I zSwtlEMa`#ltKsMRP4A;(VjS{`v5Ejl0A5SkPO_je60yu6GOMDiEAUp2{9wNa)AM4Z zta&vR^{3~uTGi9WOwdjTXw<(i6Q2b;`Wq8(@5dwrWf_ns*By@aI9->$OL&QjjG$

o&2A?+rE8ur>#A7HJfBEpe+ra`4JZ2qByPLCK2}ScQ4CJzW_hd{t(yK1y3iaR`8k-j#`;&k~l>hE+UOi>n%D^1?HzM|B$fx)d z_iI%{zkB4PkNe;Ox3nFvUxnaGXvy5?YP;pmrOEC2`lxi#I{6o&xXe%}O+lt!EcINM zBh6e77rgdLczWc~g1Z{$&wgArg5|ZEbG~7=Ng-Vfn>WCWXlZK;x&0>cm^)VyaGdKc zNqe`jmL<`1zMiM@G?P#fi91;th2W`3K(H3o1Q?r!S4f!E{0sSw%$%IDk7}~V$U9&u zF);+YEn5V;&FdxvG~zMS|HEPa%hWh!Mx8xqT5j!F%GZUW?GW$X_|jLh?|)gD^pWBDyDnVTD{#U)Kb?PxoO)JwT9b7CHd zHRS>|h^@A7c^cpftXgtud_5jsL+0Vb)-6>ofal0a6&xzMy`iz+iQsWKE&8~xm^9e1 z`Mk1JgXw2oAcJPy#@qpshu8)CC-XSY>EVl6{{b0H{;yxZ>gZ|MXho&zF>U-fXZ+}Y zx@Au>1QOqwiP<`2HA;W#HgXNW`-K#ReZPoqZMrUtpSq7B{j#dngW(sp$2R){*5>1WOPM-Bm&EE=W@P`WN-nT)j=Kt`LG#+8+=(yf~ zxw|~blD86s`@HF(e+#gGa$9ImU6UiVnDgO7QZM2rZL2kvQK_o=^FRNPV#T-ZlwwtT z^$I7=Vmm%Ch_L;I|9*%P`#YpSLYRl+6ym)fNLW zud<=8o1s3(b@#|x$h>pAhW!sn9$M;YzUi9PF+Di-`7WUhP3&YHEPB2!_9Ek%zVu* z)5rr+ewyX5VZEhBvJMsx@wa1CMK=>zZQw>^wXOHy_=X%JIJXe)N8x5CWUK`Gz&lKwYiU#nbF(FByNu>$IchSS@~dg z_sfJ3&R7`1e0R^}A{051!uX76jLHNNUTGsIZdnmiih52wUOi$uHJIMzwXvMQh` zgNF?^J#$IU4G6buZf?4#05lunkQ7xet}lJ>TCQcKFKa!&7FW~{HuCk{P*-g~pZw&Y z9<|Z8tQDhO8XDRE8KRIKTvN4LYx8mPWmZ!TW3^aj%2QN3_Kub3o75Ng@;)75@z%7& z5k5BP7E-ZEunGhTl1kUHJ&9B6vHKe znzk2K0+B?eD&c$j0zAA&#;ksrT*r+UJTd}FWj_FlO#|(|GNMPS4<&BB1Rp;9<+DUe zQ&d++0@#b3=}b`WS|Ae+hI=HP`>q9@eOUVMKJzNFq%18F#wSmpr&jDj$KhF5eS9B1WSXT|+agJ)|?uO6(YHJ|sW`iK*XbF#cVEPTXz)4J~I zw#v@YWQLKHL(KC*w7jwRSv-Fz8DUxP0$)jE>Bo<0 znADyDhK`?7YHOt&n#|EnuQ%mQl2?9l$wXf-sqiFg9D1U zSBf+%=7jCe0!%4!kmA4-@*ep?b&_ok8gL?9YQf3LZa6qD02h&nA0|-xtUkKOZn$Ri zt7YD4#@J&DG&)ab&Y+`6?Y8cTxBqFZIf%bKb8o*nsX6#v6c0pd)a+=D zquY2yQahO7pq~8-!p_BkxW=kIJG#LG*>7GmdS~5f zehQp5E~{!Rj}Oo=!x2!9A-WoSjy?lFYKcPVu*RaX)%dbynwRHNIN=mEuqfXKuL-AJ zclqD4*1?$Wmto&ogG3N^&?n@B-~kkJWrx%Fg;cR~`roX-HTWcq`SEo`V4!(sRyZXj zhD(^L#~E^X_;Vy9Nyg7lm;=x=zo4r~0#8L?acchl0X^Xg6c6eR>KF;rhJj0w2Y#jD z)x-~=cLVM0+&ba*aG(>MqhO#A9r@8n_$DiBR88<2q_Ko#i!`adR?MkA7t|-GW)0qV znjzxkg+*qaTE&dqKFa}~=?G=+XOn!5LvDvKa6_ZeFqF?bqbl?m0julMVWFy@y# z=Iwq7%K%4)tkvLZa*GkPC*Nt&-RCk($Y&ifzsl)I$m=j9muJ;HE3gh~2Oyq+}-T%?6K`NK(~ceY&GH&sdg73c9X=ep`Ra;IY@eXZme927@*ow+8M0 z!)fO#ou8IC&o&Y3!~=&&cFDsBC@v!~q=d1Z$bZj+%V_-j@Vk-^XKtIy5Ti={ z23yu(_j@s%aW1k=$&{|8aB8wcQ8tzlc?%Ah`I`TYy14&v>)&&}(bvEPFnORw&rTYQrr-s1ENqX_ybCt9G@JHAf zg!`SSH3yxjvl~0Hyv`OO8NAVUw$Q?Yg6vw$;YfL0Cu4G5B27t6{)Wbr!!6Uj?xVVF zKW}{PvQ%O$SO5wJF*;r)hAt}93@Nx0xegD{?cjJyq-mp+V{)U}$p5}5 zSV2F+gtW5(g&@g*>&s=)gAgv?3#+RmiCbKEcX$1N9&Lnn%!6H@vhrVvyKCC|pUn82 z=HeS&g!-F*{)S|X`81LRmr|r4#(s}u_iS$T{eJz`1`v(Ky6^9q>vO0iI#=d1Uj{-+ z{uk5l&3iCDIQ&x9(<3)FHa`CIha?`6A>0_L;eWxy(cYBM-EE!g<0AO+*B_D>GatCT zT#_HvK*lp{o@nXnDmgo2$1NkHK=T7ol!bFvKPzN`+Tf&kpW=j#DU}0JOkH$5`hR_Y zRA{BR84))uPD9{ra!T_^?qkujg%?-bHWD@`oBX2J$GnV5BF(!kTouV$n#60}xMCn? zeby7!2-J($_G^YbS)h{(eL(FmU)~v}%cAH#-p+dP;c7sVtcXzP@I2}EKS5zem z4%BSm*`)oiAEV*oI{j4!t4;5~h~v>mRmO9R=SKuoO7$xAB8l(p1Xme-9~??Cwx6F< zGc`?o$FZ8AKwVf!-yMQ2u9AlFduKFJ5e>E+L&C;3Ig&9o{Gz7u5R_akU5Y0tqFLhY z=|IT}L8HAsdsv_0=Eg=rSy?z}Y;9d^;Q++;(2%k$8t&hYXbBwkrjY3Nrfk~~nlj_Saq}f2JaQuJ7X2%!BTpKjoeLzKkFJi!?_>zx+l)I*ZqRLR zy%k!|W&&bRp>=g^pav2k{wEXPV5u6u&nZ>X#!9Q~Y1-$^#Y)&)}mm?rCkrtBx3j8Xee^YW;81#Ep&+!HweS)~d zr;{htzQ>xP9*t^?o#jn_uq;(HygojpVWl)*7EY5tgP^<0Y7-JbP$6U`x< z_O`}ET37c`+?1SXZgF*WRwh+=B&j81(!@7?^}E}JN)a_Y|>ZD}Dp#yDRmgVK?fm>{sr;aRS zBpj_(+4bWbiwj-sq|8k*G&Ch21f;osqj2czN-3+DFZciA6BOiCMdBnYZ7R$R>b3w| zWAHqWI6dL{_DkKsfO64^BKl*pa>=(gFJ29a>pPQa9=8nFeP?Yrl++`P2>9vK_{=}?aehK2{!2L?de{`!hLtftN!VjG=v_#oIB>(75n8Y2g_BD zn~X&L7`v&`MK9|6|2$~Ss96HIG3HMJI&*vVO6$p!J8n%(OsvZAV;s4OXCTnjaRIa^ zTt_E_JM8Ah?+&p#wC=Fem-^*X~;e{Vf7>x)8wkX;0}o{jEv zKAczD@;$=KZ!+wh**-6LhS;H0z_&?e*zl zFDYn^r2v~qXuu%Sx}ZnK{A;D|)e4IQ9feNn<4X#aG!vkL1hz*)!g&V72dWH?=sV8( zdPcGYlg52_lT@?y!v{F>QMsP+*cE=}D}aXt1xXuy41hy}x4Z)qmoLRdUL9*|C0Z}_ zQV>agRnSG2({+Y)oz}a#9RpMR^2-dl%XcdkP50*WHA70lF)mkY?0Oq=O;Z}l5Qd8j zEzzS9y38z6PZJCT@qJDXIttaqvT~P@N)!xTiPNQSj5O7 zMQ8|35?^Dty;&tWE2icC*q$R%_Ba(?yO%FPH*YY6m=57Aro8olwW7S&{UMih91>0h zYC8MlEEC4xl2gS>AwA~o|2)JOMCXb?vK`b(c^&kT+u7R}1McZA9S-8C_b+Fkx3n4t zN_%^^D3(k<<1sMtTtvS!-58j!&o}k{BKp*Hby?$zB&ifFR55|+x9E#XIcpy$Z{KGb z$rg%*m!p2QrVWl*x@5KBW+Gu_CS&V#7lYhQD@|P;Uty7O#S#pqk}x=aQDp=L0A>MQ zr@oy|b6)7G>2N8T9{WGLJ8Oq`*1jqtl5`mQD0-oVLx^7d@(&Xw)uF+A+U|=J@z_{7 zBX`}HXLYFKu7lb-_+|r|x-Ld~<>gLT7@-L$1zh)H)k5esB|RA=-X43LPR`B^C382% z&`3~^AH9ch%%1|maG=QbR^*M^rfnud35Rv~A)#>|m zdbKG47zPn`iwFur>5rl8Sy-TzIO`g1_~5WmbMu_};~b#9QaPrpU#HfO35UzlkHz~P zGkkEI@9((TG6jUTT(5<>Y+Wbqd%|y{3uX`E#Xba98v?<)N{#oP2lTQn0mcz-iD`1~ z5s4)6OG_k1f)Nq4OGr_b$Yfjr=JV~y7Z(>5Zy(9_BKv@q9Th=>g>`Zw_hkl?m30Fc z>AXZ`{C%2R?o_8H3SFs`9x`2~q`@VaxSE=#4n9A>!F`SxkgF*zpi}koLed2eA6j{p zTKF7^iNx8-SUxfYx13rq#SDrArN#1!N>s8D`Z6~-QQ$x|u0L>guzVm&7;UMg3wRMQ z3z;%hc~oVwLZo#x(xv&H1eS9pB7LvYaMU9VC)l+yQey(Llb}cYpL3A?uSp21I&g4t zDJA?w!TWytT|V6Yb;;v@QASF^%Pg&-ksTLD_UV%|2F6A6(U^#x!`vuB!jGHW$Vt4f z&s=s%^1=IiU;TU}`aZnex!Tlw4UhyKxv399ToTe#_%DsP2)Zn_#YMc|GkX)3D( z^9cys&xrySI-;e8Xs}p=mR6RY;|@?pRTfqPQ0>g^naedo3~E+1#jV**Xz|!qpdC!o zJaZ~~!wx9O@jre%1*tXqpgwL&%Ww5Igd!fhj7fuUiYj4CL^ulNe^cTqZQY`bl{v*1%g{bqp z_BQc3kYiSd4A)tyRFp$|oxKCC^C`c=7N%yi7AngAEM10Xd(2s|u_*^qn-S7GcPe~k z$x>TtAy{-mUI3`cDILtU2nt!e&*c-J6G}e{#yzKU0(T3U4ESmgClE!Vr3*Kv7Z#%E zpu=U!bkWIlu`rC;*n&KBwe)`z5rjv|8VQTk_gWC7q)-c0(iWDpRVv3-TEuiwpeN5e z8!K=1!}YZyma(3b5rsvfMg}^xWO~5KVzu=25EBzb?R!VzLr0>(ZM@q$l<%iV9Apd* z4h9h+@XM7$A_=3`UO+b;sCV?A`nt%X$wu@mLn|tKanUvNdXhzv1MLO60v~glU*TC< zSy5QtDXKiv(gmLefFr`$lR=j$YZS<>9{AD_F9vf^;DI(wAUyDo!%sqVj)AzK9N1zh z-9hVmS$Z?oC_U4c3{=%Ci7MjquU|g^%l`4RdOMJyDpJkhp`KAgG7P+LhDFTPZ!_Oc zP3_|Z#wV}4qHK(gjHi=&fws~>lAAfYk_0d8pp6L3At_m8RN$WDmbFc3%xo1mwque8 z@j%NmlH6A$jQv5))d({#Rj%@grhvPth65F%k6OoWuSrh!Y(T_7WFWkucCh#W)Q znP2&rGYPX7P!V<5_w_>@p?n!l*MMrFu?6r%b$1qUwEWql&r+^)T~`P7h=CFN>XBH8 zqRK0AEC57go!RAqQD|&z)X~vdfD!y_Ke1r2Rl)?&^wH2DjFd&O#G~(?O&PWR-s1k~ z@^nu@cTiKeH}J5ZIqK01Vl*a&0b)rxC8p&D`yY}5DPS*`G~KsPh@<-`N-MMRK;TnX z#{&|!;0$EsIKe|<**g(=xUTwr;>GH6HkPDi7$}`mR#AzRFVcB( z7sbkf4bi&)bu7il9!wP{QsObds2AZxGsIyPRFi{rbzz#>k`Yc5GG|EIOiAjAV^s3a zC}grhSpNZNj(G@dF6$8r*$}YxT((fXeys)n+b27#wTPz!)_*0b-|9&W1Nd-0C1Kp( z9f}KOKMug}bfAr9i+~0PD}AuHTc4m?EQ}s2sj5OtP}dQpPsAYOD!=f|4PQqeAU@Ds z27Q+zR{4#Ko1i1RMs4{WE_kd6s$Qz6R!Ee zU2(63pZpBFdKc)atru>tKQT-D1k`sN!BEf-pb%R=9gbN}C{Cd!6%|#}cHJ=8eTV=EM~a?&(Y}MWK&THyiv1LEa5}Z4+J5>7Le)X!oVtm4k)A%7EHBNccRhKW7<2>o;7rFXM ztde_`Ss%wO2nY3H7jAo72$^0)XMO9#QcF8n&T1d~zC28k`uj)kNAV+KzxGf~Yrl)x zpU&3`tT;kGaI4FwO(zoK;~)2ajX_;+vcF2)Qhh}x|98w7{c(61EkeEs&#>+pHW>5K zT(KH${on8HrtT*WTFWj0K{lt2l6Br!_ITf7pYy(vlSM{P)?t&eu?Yed2rplv2TG!R zL&YoDL<$4X@?xxs@eR&SBnvy*GW!wDI}#5--IzXm^v6i)&A_C>zSh$qhQkh4*^hsM5Wyi!@N zRB<_4^1Rj|nnY6J$7Q6*8^4&nqJT=0xck7nK|UwCbSFRqCEt)rBZb-v$hbh&{q-Kq zzS=e$_vlao+YJKl7(_H6Xe_6XlPQME+x^nVQDMUhWk3s}gU^W|qt|Va$wb0@A7aAX zoM<+lk?6_=m_b#ix{t6FaEOTE>QPmBOdxwS@wKZx_0v!H8RNG7ZrrOWiEGmJ+}9j{ zU;1n6Kon9x#J=@>cEWt2-S*f)iuG#>(-(eK%fn=69WqadUS&Lao`1~O8X7aknFqrq zdFRkr;y8xBo0IXQl^-)3%TNTa%Vx=pi?>XAQ60o{6`R_iHMWb-QHy&Ak3-^;3`d3l zk!Wu){d*Hgz3n{+i)(mKjr<2`IOwLIB$xBt@qkm_ZhQAel8(HJzxW>D%(jOv>S*Ftd8LBuo@-TbePH zEu2yknTd>~va%ZpA)9zvJp#-S5za%}aWGYX%wAGb3~-rCb1%GMut1{GMFp54znrR* zD6$C+g(IAR2}F|lnNqy~H~<6&raXc6ovUK&0ZRc=dS~TW^U;EX*eHr1C1CbLN`bKM z(gEM=Q)^19B`m~AL~5$$Q^VLTuY3se3YHg`dDvy_h1Qd*G2Cq;(!!K1LK6y?xsTf} z&Vsi+7Y#0iMTm$i5o${^d6y6R)pPsyLZt<=N-XzYnRxE6+xx9)5;Qh0T#8plEw_-R zr1CCT;bt`Kvs2mR$XC_j5HEfr3Q=r`fWxHY-Z0<|TWh<0+-1t3kk!Q|m#whN9P9PM zwG%$UQ{F;o!*!Iov^pb`m6#(>PbWScxqqlJ?km-ep8$y{ZVccni6ZZ7ir5J!sT+?p zHRA%=cgZF$c`g$4WKt%vWZJpe(ZGi_wJ%S!+uApYNl>q8r8Si()2>YCiyI1Yq+kX- zV4a|Tauaa0oZ^kId$}{!_Q9D!@TU_hX!!vlYrGj(#2wk%$9)F*c?OfXx#6^>TzKh> zLL7y|K$bQQHx@}9Zg&#X<8WYPZ!>5~JJI;rbC~yOKrwLN!rJ6olZpYNkmvG0< z__jGAw~MbwnR%q>Dp@7{$-R{(tsGJB%h;>4_VsU{e=5(zsa8i-p9k#5A-9P9sZX%! z!q*hD79}kkTaMB&MOfkX6T85;-QJ9LG>7lapAi`=S@9LDWRvlhshMdRP((|>?Os)T z2hhxa2znx)*-3u8wP&DD{9mx~>klP%T?kv_6_eLFlb$TtG?#ecOxU5=&Zx;EkVv)Y@ z)83x3UUP#p<8RIyTT9!0snWJ8)P7>>!F}4#$<4gvRCg@sxGzzm_xfP;SLXYau%>YQ zanYVv@|%_hlTM;b-pNk{C%!zRu(C(b{V#ys3qfEr(`P&PQ{oZypN~3HuQsm}0KSIK zXWsJW>N^>ES%84-UjXvT-IGNF*|YvW(zT9h$URo zQ{hX2gApkhZ|w$qzffO8lM?T%XE;$�cc%&ml=NQlWK~RO-!VmfL;@1|+_plAnC7 zI~#goaA;#nOTvZ{!T>(y5Dk3Liy_5s#UZ$;`b@jAtw2u|F2{N|=gmWq)>~Faig60* zc>twtBS(QQD7`t{d^*ki2n1ni#NQ-Eik|25s z3?MDF-u47Qqm&_Hi`RVLM znpj+!V9d(%k7(87FqFxEo*TFHA%ZDpTm@p^W zE*1$P0$sqTpkmx5M(zj}K$L#Xe-7~P{xy4hUe}RNCurx@)Dap%8sY*`5V59946-E| zF_Q*mX(gVvuB4_FnQopey%fOPCtKQXZWJ!P&R7b#6&bL53ZgnvFlEY8Cw3(A~h_=E>uaeu&tlHNmr4V|#eC0UW6~JP71tEc%6zL}kD(rW#!`ZDy4&2qVJ@wBtw)lSQrg=rjh2(X7(xv6SNyivnn&3s^c| zo@h?-g?Z6qic$dzQSJ{Qq-tjtHUF(#zMo3vjh^7 zXu%jnl3_trp@Wv2?J1hI+vbJ~B2(X{`UyjvKdoBtKtp^!LMLDwHLMMhYnmi_dh*&v}C$kU` ztS;;*oY9N%yjCIekKkTLw(f zU-n@y=0UOFSTlR;rW^fA1yx-IY@I!`U+{Kq-e9&UCiz=q*vwOH)!0zaTW&Xd!@ge< zrUMN%CQ6pI!-G8EuE=hWdcre~M&DN8VeQ4p*pKjWG(D$M=p`hYuaCO-)`Rycx{ZnR zz4f%CTs*#-p+_ejmZS}b@h6vh#N*?y_&8e6(A9&Wgy+f9H zGw4WI^AaF43}gh$&r<_;pG-^Kyx2897j2zg4#9bz?z~2lYT`|IgwP{74iVIzlyROE zXTUPyl7;>@=fM_!)zQsBIEP?(EAZLIXlRqAZvW?S(?t|-ITULVXD=5}QD}B~nr&w3 z@Cg~10Kb9Ofys|k`^C`pJe$?9YPZHl?qqaMcy9Xcph~R~#FM9McQ4o;2q3aA5Cl&LE7kSd*^dQF}o=xH5A5XMx+{^Xe6s($W zNsHTM6#ASwsG7x{(`S=AT1dGJWs6x5>aHyxOjKqbq>{%DDGD)S%7NscoHYKtj?O#y z@hAAWly;gcJ1wDCj^#K{+eG008GK0HiDKATgOG9x5s;2BtP2AGF{N zU6N7}$jQ?w=-#1bxX(P00+FKjF3%ZELRHP7Vz}?>Y#c6%o@`#JoLe|V4D!xg{ci^G zKF`l1hB;52_dY58**yUmhm)w#M-`Px*D8V`0;h;IDkw{iLTNzz5J2U7IuGHPi)T7r z<6xi}Yk$bo*Gk7+ob|8E5+fs=ot%)6LoBrh7=5)O0mc*LezoZqvCbZyCGo;0I9efxlNCJzUNn=3Ug^W%i7Il+$ z1uH**E0K*t_ewQ0vGS?)__jd1YfdvNjx?D;O+ zM(tO(*CFT8etqqeTB3^~w!Y_dxqf1Oik}aX(la;LR#AUZtYS&z=G^EczGZJxBFnzLRzDK<@)se4BU9oNjJ&71bm zqfug}5%_r~ov{z_6m;v3hi|)0O}P_ncCR@7JWr<@%3#m8n5V@KkUW0f(6wJ^_#;+v zS%~nl_T*b?QnGIT7^O<|LCJk~tZ&v`X$m`+@$qDgUmF~q`Ccug8hSZ9qV#0F{C-99 zXt(*Hv$Sf<@2596H8T0IalP)tZOSF4T08Rh-)g;DU817cTp%Cr=%4XF6D=~j zaI`=*ispDddF;Fz5Jnw=RxGW)9@G9^C5{VRDYK> zv=BjFnVc&BZ9hNn*_Wct2egzu+-+mg<6XZb+K}7m`#GEA_O35&GM-TO{hOCJ-aiv_ zhDb5u79-Pu=s+g!F1vwFhQKC@PPt=Y|-zA4Y zYNJxw)ivnmhOe)W2v1sI3l+oZjxz=MOSoht!uQX1WRf}yD|G+&hLWZpIS|{lh*`LV z6yQ`s3MwO@^vby?VXDl73uZJm9dpAw9|`MRZ7}B7v54saA^jZ|Y1V`!7qzOobZ}X(d~`N{oTWN*Su_>X`%@p!m*=<~EOakOyBv^k zkAh7QOCBEv_&!aq8=I_6QSuD0D_BGcOq}3Xf98vi9Mba&30E9s9%}Rn13W+GWYlHo zd(61O_5iYQJT*KNFE6Wwnvm9!+lw?pJNXW*;1bJJNo^^D_&_G6Ftn~<69twm25v48 zTC>M@az%b%^nZj-gL&psbG>FEkJY2DE$+cuPEt_si2046SX0*kodq<1^C53|31del?Sqtr^0e}3?tnOV6RX&2aQSmWRj^Lbjs@;DQh zC@EUG$BOiM&Z=3Mr^m@2WsUpZg410vwDYVd zJop_t)nbCYF#Ti5)Vg-b$;NOtd%fv?6A$poSC2eS-8+tF0%8D(W$+m_q!*dG*$3Gu26Yc6vmWUm;x7<$f-nhr& z+Lc>a`Sms>e&`hj>wbN}y@4;z7pwWKokw6GSCMc$O5T54C(RfXZH$}FWDBE@^x7Ha z{$*53>e`^P6fVvM?Kwtw$i#KpNapr(MeCq?I5oja14USDO%NX=du!EzIS0V9_wnSs zxa5lqS_l$23m4&{AmqD#Xir6k0|kDkNeXK@lg05mo!Di+M3M5gSw|pEkrJAw;VvG; zz2xad>nlvq)ItY`GU$O;PZQqW28mqTP@XSjFHT!%(%}#6m*8`RYw-KBWRW0f6H0t1 zpxU;uoU2MgZ9Tq!e~Kj+PPJOuk=~Vf4B5tE*u4;k_o}XS=)jmgzdp$i&9+c#X@9ZebKG3ke3i;vy@Q05!ry zK{Z^c!by$UFpH7*0flQzGm2P-c`8djBxjXLg7bSGtJvUFDX+vMGbDP8N<07DR?#v> z6?zR_|L`D|w!&33SuE5Hx0=0qVaxgYfccytPc@$GZ}Gb=w$3F)B$<9ZU%B>4Hg@6Kn!CHEsV0ZVExYV*c&+B2U))az z?YB2>9WSmdY-BdMd@HXNS*6z#$nv*s4^Daop$#T_{`|Q@O?r|1_{p&TPeEG%de`~M zmphI12JfX5)ecH0&-j%4V#J&2neDd*hI2k6`yT9r)pWeCqdy+>-Fo+e0kN~&`Q~@8 zRea7T*o4m|C71BU({lW&KqEY+r=LDy$kHHklHk=-vuJLPmM_;H2V(azV@uT04(de& z|6aXPSYMq?v_0RoHCUbt*N7og6HhHb-_%O~8^GXQD+7Q&OZKTEM?iWiq5@^ipqrWN z{p6m)RWUO-E2Dm?!O=$7E+y8K63Zvo?e^?@TZJFz{UM`;22-<3&5Og1BBi5RZEoMw?Hawa2*nc2!`fGRiVb(YL^nI1&c}z53hg$!ywA&>;M<&>y_U-aRl5P%ow-3yNUE(BaNYYO!}3uQI$5=0#?z1V?G$TBzmd$K2)DQXU99D`s4U90 zJpFP3z5_NMf_4!e=x&;k;bE(<;es!?i5=zGH^?$W`W{b16lqEZO@NxcUM2GGx6XGV zp|W-U#ZzPnp~ywJ-pkr|%ZK`H!;5#%*bIL?S9jpbH-5D94G$in=3M=b_)vVC& z0%_EW2oU)s!Nd|s};hTnE?4nrFk2wei5z(vzSXuO29|ROPw+}Mj68+B7#{Kt*+ge?wSye z#a-ioCId#HEm`)iLU*>z>+8an^Jr9r)(Am_@39IL>>kZBk&W2>7r2~%d@#1NRk&g< zQoYtLKQo;jyP8}o3Z0j2bA1`UITRYBfjCV3WO!iRFbG${psCmYD zxNDIQj~;hUl=JgtjY^(8F5Vbjv6Z;V^tkMkx1Ak4r?jE5hFZL~xjUS^IGoY`<19BFt{hK2*zr+%b?6^(K{l<>cYAW4OI&Zd<4rhU7FB``*)}h$Gu(X^_ zSEH#DzTK4U0p=v%*d#`^?rMzk^t!*Q0QM z&AP$a82-BL;ED}>fXrj^bn>|B+~~OrUHC4YrRUwdyUkAvriyN`lrBEy*=KG?-(yl7 zyFTzRFf%6hSM3Ju+t2zmL0yt)yA|KaKA!a3!!W4vQ zOH+U&c7Iflw;0Na?UmgUw126v%lyQBM~gf@LCgEN~dAq zKS|7mo!_ONC*|N6D*x89;ojdwv?lf@}J}HE6qHqCS}tbz8~5ykFE6tSOXWV zbITQ3hzqUuokW`NjIyT3RetSDJZ$a%_NLg3umm9R3z6(zk^i}gRK`IQ>VVFa6U_@k zgt88(6wfBl(=Gfs3t+5pZ}A2Fub?s>74F{mgJ#p;ti$^#qN-iCxpsHan&)8?69YmU z?OPL~EG#zC%@?-_Y>Ll|6KrPqhI(U=KkI3Bf7X+@@x@6-_H9$sUNd!v1HnbIQZF^} zV3mhca*7c?4PJ&`jSV`wh?b5QGe<3mS>oj6iw$Mvu)4Y=`L<{?zL=fu$nvBSyY9~` zbEHUV15MXpQGq8zNc?M?IIJ>%E^QuBX}dv+$jC@g=2on~IT~bUW0RJKnno|(UGG}} z>34yKe4>MFlJ3`(lfPWA0~$drl;i)&%JXcO3XlAC^h1I5H8lmfVz5+j5R#S$(GnE` zVWI<#f1qPH6?_`=9mnHNlTNMJguMNB3K3t*!U-aPOj0Yt3;6YyOISceVZU|mv#uvP za3YYPg@KM;S!9_b(B+K({w)h&+}gdm$;uK1nIE=TnngLwlxPvlpE~c&qi<<&dtn>d z;ew4ZK#8H_E=BP8DF3{=(ieg9W)T&@dJzCI6|GExC21LG$<1%CP9NM8NZLIPu-9W0 z8iyuYL>-8U4(K#g*tWt((hkfS{N%F)prE)$74XY4(?{R<%?1HZmCYu;=tTz(kKdiQ zwZ)ZRT8=ePRjYrz)fB)GAM4 zn5m6h{mWGIiRMA|cz)gg@8Me+cBpZF=M~s=*F?mLTG}CxIbOYD<$L-gD!N{2Dc5Gt z!+Uwl3GLc;UKGi00$iS^OMN-*?` zJ0^bS`8XNXRvq|Ab2W7OatLE*>q+lHZyC8x)q0`_k_b%D?ShEH`}4=le3;2)>D%VS z-j|%uxh;%NR1_%NqNo{|u8n7g6xpA#m3{77p8Azlo!(Y|Ss|1kO4}*3aa7uNIzD+O zzxRiH0}^w*>ZHGUd*DHM{kXYRW2E6s^Un< zL?0~gYQk?_Q6J%gG2bDaGTcdAghoQFepYT>-_R*{bS!v$amOZ?N`ZaSO-izQ5_iLT z?n~G6Tln45dfOdE1=ob+yVl(l1A*~T{dh$gkE;N$=?rcqqAWjAe6Q;NG|COw`Uf*=WA~2Zx2JsMq84}?dr2IMs%O+eGilGo4TDEy%=b_ zOI5bzPDE4SsY=U|D5+R_KbEU<<*Co5JG1C;1VTKS3939T9n>_T@zIuZ z*Xmai;V#={kdT{t4pe3uSYm#@X)r3q-<72>FH1X05D#_rkj%2_F6QTT=H-Y(kZh!F z5Cd?H>>kaYRlbt&Uor`kqugB%0RiYU;UVF9_3xw>ga}5x-7DDobw%>USOjbsWMsiJ zFWq;zDGx7f5D~;;6FRoP*&9;hGQ9$_@l4kl8e&c(X)`lqAS1<@YI5T&8$hbEWicPk z&#A`+np4tzC57m25F$v+$RIz+pFZhSFaW|mb;j@!5cQcvOFU=@lep<8c$t~X#j%)7 z6H7Jv*>1mF_tVLdnno-hUo(ICI2_$e6s$N{1>AZOfn^%wm8n0sgi4mQrGr;3Bi+Ad ze~%UlM#sR#(-L|v)ETJt+W(gGnN<4*7-y=T>9yCOq^2z^C+D$K)yLJ#l{6(#_{}Bp zg|?|4VyM+GriRZ_s^c6tOEf@4R>_9J5BgHN?;~-3G}`jMH2@=-`X_XZm2GX=HA`85 zE$Lw@S;gca5?)8OZ6u2|GgAZ=O%6<^e+|}n-Wn|i3Va%o^u!PQUVl6jYHh2(EgER& zuL4{_UQtnKsaDe1m^$!5Mow;?$LM;J%U9mG#R3Mq699c8DmH8pApT9F%Xp}@fQHAwe+|I}E}fRMT1}d_ zd;Q%^LrX`806VOTE$Tl<;C!Y*;76SM_ia9@3ldCeK4gNoJ;e={dh8TSw!P-PTh{fP zM;jMIaRTzI(o_Y@+;e1bL2|GJotRP-c8jpgwxBP|wXt?~U^F1GyTHj00}3 z33~6vBa7)X2eXLKk{Zsl( zSZNWK$nQ?)7s%F}P01K3FAmcKi86g|H^SMbB+ETfD0)2xiS5@WFyNi=g4gOZ!ZUw& z{uZ?qhvmjgG^U@AB`<1+F3bC7S9f_$T}XRL@U;4SBS{IVA+Rz7#`7Zi6(Vp1Y;Q~xK=o9J<$KYLHJwORS z(tLJ0UU9f~#?K~fi_A(!mMUD6uGVfItxLhOHF>b)`Tv_iU1yj4Ivg87CNVKFt8MiO ztbZrsM?fB|izbiR1*x6}^viFb1;%0!wqU7@lF8bx6=Phuc+sp=;d7vRl>L!y^0x#98>1zDuc|vkQ zhWh(EsH*7ZO4%othUG{{78bpe)%*+61QwfVHu=p`$^QUw^=VyZCDB%ZOv{QBY%z+4{wy$d_ zL{YD54}CowaIJXFS1^LXqh6;(WSrywsvZfmLV|Q_k2*t1v+hVq#1*2AE1vcynpyzY zw4^3UZz2iI*3JYAac}QkdV2cvLXB{UsTR>}EpxU|O5V{+X(Sm0lRyWoe(B`g5doHU z0bCZgb^cPGry@$KgA^bU!60U%f?_)IX%~L_6b0~LU_sJ!f|$T`R%<9-PF;_mN{Jp) z?jYZdkLgyY*<^{_5%^k*2$unp!*tkz;F<2GVW1)D6WBkrv7!t1XmX#>E4Fet)OK#9EQl+DZYoA+5)c zA3HlcN8VU6{!a@4jBcc~G_X8{>CX*>Njfw%Oyg+pRSN&w&4u+B=>fSgXq1FQ{*z5* zy1V+%XzLuD8(%s79;%>PC7Mt6Hp@JZk2~B8%i35PfepZ)?5@x6-)4qTep#132xfub zA8otuTs{-2EX2r%CBN!=axXm{62(0*8;Hc}z7ZxmgpBiUz4f;VLw$rOg~7d|xiLkD z`h2Px?Q2?C4^&AY1Y_z3;my_b(T<|?c|qEMu_FS>N26G-^7_2^@fU&zQ*#xsQs2M1 z_ED|6PuS0+eslaHyiaj>2j%TlyKm8R=W)Tzpfp&@D=R>wczZY|mn?5?!N{ExoRDvN zPJqetaVWKKKwFXNOuw0;sw=a4{RwOsu7^K%E8;o2`{({I!%XOhf^jPzFzHc=xNkUq z7|!J9o;G4`L}vChO@8H%5Yi2?T!KB(AaN4jpGRqIa?wRE;Xe}IXzPUwzMgJ=nUi^W z9C_eLm9*v(GHA|S>?+#IKHJYCNB7crRYz>Lx9glx1Bwpp56lrm=F^uO_B~A;1tu?o z{YQiTd(&6r`woxn@k?2}&iBkF%fYZpDWJSb`c(c>*{YQ4H4tCZz}`G~n1Q^MBZrXK zQ)s{WEh54i@Hf_PWN~EqHpbacMM1h>>!F}%B&IYOczqc@TBCkz{TL}o;^q$5*AJ3C z8rQcX(T#|Nhf%4f-QHR2GnBeiSjF9Ub&4(QMiv^jBAp+4-co_o#~qD^#MW#S)JOYS zZ>6yDGQW6iv3Jk(Llwf(9_rnYk zwRInq8A<@m2^eW+YtsNzDhl}Uz*7hcx!{!CKGk?3oT28x%v51j*%UO^fxs4*a;0tI zbQdNnf4jM%Mi}u*4{-p#^o2JEy-bPGzwo4PuWQ(>Bx^F<_F5uM7Rhm%&*vKFy*xu{ z+2;pY+<%tt+3u{IK!#ah_~Nlg>E{eplLE46D#!p`K1Yz@E?rm{sBAFm>JC{uJn8x_ z9-Uk6N>fAmYwlF%T^-#?LEQbCyMK-Vnhgy#jewFIB@0-}k|pJ)duCY! z1{xvZ@ypE2L3_W%r7RAXfkC}`dul+#&L=em(jt@5tY@o^LZGr`5-o;HrSU8QY&}}= z^zC)%)+R%WlBQ5Go^cK&QOBWLFVySmIVQH6m>Y3SObpm{i?+75@=8n5`W#XRh;!fW zNuty8mn=aL5`);Fq%elx<1q`{56(|ue*Dj202kUCSsWF;XCATpyt-SS^Om3T{!f_4 z+PoO*1g^3Z#P9aHyb+n$kI4vYvSCnSgr@%U~qi$lh5~7yuAkk)E8yHkJ2~| zhLi8Fjh&-G!NFs(?&dF}Omq=!{=JxzZ9%F(mP3ks8rxwqg{%^7A#t09Li48x3xT%h zzs%50*1b5)90IsHG{FW{oZa&$%*ezzoi?*pxWQ>-La6R%O1Onh$qwjg2MXjy(P7N144ZOTg4#D$3C-^mIRABfvQs49lo)d6{ltTH*khiO@hq9P7y!>V~^(AtaveOCh3pW9n>* z%J&{B)FgwEyz1)o zo~Y5~sy?IF?)9M9!nWFcj6T&mM-O!_{Y*&ALFl?ud2mfq4EJ-aLJZT!83#gU> z0SH=2Xf(t!N#6)%Xr^jW2?>Fy2ah@2oOVh z5`Swg;#{hSbp@j>2i*N;;3=j1hE;q}E_TOD{6&b;FXs^^w2QA3ZkEZVg0M6Nj$*S9 zk<>Hun8`8@W<(UTL5~KYOSl0xPv!UTbfB6qj8r^4gb1{V9z+U{!4x#pXb^SM&>*2l zGL@EhQ=$iApo7jnr2bv#ypM2y5rMS%a0dw_2;DT;0UfRq8*eFA8V{}67N^e2n1O^pij+oJc}KmwY7-r3F*P!7sA3=I!lQe<@6!Er!GbNg%YZ zCJnvzlxLJ>uE>x8oKwK_2P^RJ?rvdmF{&vhfpoAqTuEM=^zPLoEhw!&BF*AcD^LBi zSHILWw1XxorEz1El^IF5c6qS82w?;D+WroaE5lTu=If0?zIY}YzOXI9`tNlN+N96b!{Tx7U!#DkNF z*e<3q(;i(`;IrvBX<%WEklE&I^Wl<$JT9b7qb!e; zcdo97XQuj7wIK&Dk!Fc;RhZ)JF`I|{`cA#?YOohB_C}}v@I)`gSP66_U*9Egy5Ssu z^Si{fHYfT3O0@s#v@ld0#sKb6;HLsq00H0APp3m!AI;|)HGD4zO=@jtRZUHwtgZP< z!=EDrg-c94q_zIQcUD#+Nwo+#6prg7?29+Yilk&@y$;nEx5V+`+4k`8N%HP(tXH!} z(rnS{y(KI;py^JP7p=|@>hDSnF)PrgPcb8c6{!|iB3b9Vij?_9ZVu*vXnj~{14*yC zAl&A6Za!`QN?cQP>~Dvam`VHJdKOEE?(Vd`&_&p_pf*co z=^ulS)%pY@`hqk!8XT%1j(v#8Z-GL+2w#!u%a{BDPZUI^NGhfkX>o4_yK&cM3Y?AR z`vkU^P6&76kVIOr5||Qz-uDqg;({`?7^ZBmex+~RL4v8FWfVSzh>pmV@UtI2r!0JU z*$kY5_~;?2EGdKB07uJ4+y(7I(k`s`M4?Zbn4*Casu7w*=dC(gHD(ygPSEq@qh~L~ zkzuQ26VRu~|CpSKP8e2GSC>#$#-fM|%Qxd8%^PC^Bjih1Kq2hImR{nE->)Ka2KW%` z-$m}V=;O}jeQktQjH03s(BXYAT9KYTUs|tDOi9r+G>qw6S+WFEw}BA~(0V~muU+`H zHul5TOSW0bC;#4XgKl%C23fUT)j@{Tr~>t&uIKzV*isIow>~i`383M@^yAXd(9pm8 zz-Ebu*BcHAA(|l6c>!*?8VQnGFB+;|XqHdYcZTJm^0EQ|E}pIZU)AgwDYfoQ;cj)s z0#|yvjYV=N>b{-VmLmc4$>>6V-8lTKiP&t-9c!U;au z4)y$Y74PY5js$c)&&{yQ8`OoK&!pOkgkj7F*UxLk`VL~7rl)%-6T?w)7P)%tBUhi3 zby@dw<2%_K77>WUFA>*1w=S=E+um)8T>rAWTv!n4#PIU8k3MQt3jM9@R&|#)sONOc z^ZyCt$}5M!QWoHE^xoEASpKcwHad{hxgpk;sK}Al|AC%@!s89)@bInwV`#ffx5NZk zhv57JcoN{leg8T-5A@h1wzkT2TfIeDbp$Fc$G|)*b4^`cWiH)7TU0e6&uQw)dL6@ZfgBxN8p*YprEGG{lE&jIn%rj|pD$s4K>H8;li53iFs zjbr;(Ov*q~3GDmwg&O}(k=g1}TV2TlPtkfr&^lHlG;@Y4f3C|&i%PDx+Sr) zMfdFbOheRy8o8%1yhmbyFZqKK=Fb*4a?vMdRj9%m;V zKbr6LvEM4Wn}{5ng?o~CctbooOR!b6)$7@LiNr~B&;*tCf|@^_Mb{L}4_~1%D*J6P zvQaMfQ8Lb+Cuz9py{h*zdf83qZ2Hu%@@D^xasxS+@9V=Xy7kGdB)=TLYV|Sr9v;t5 z&j}6eYkIz-ZEO8R8dhg+`HC`5tR<;nsXoo-Lkp)}UeabkZET|kWuF}Bj@6R5J}<|3 zh^nEukR~&u#%bpGA-7@em!Os=?Du@lqAt*#%^W|v7{scg4fi6}VZKv~!>Twx{~8@m zvyo^8g-+^bBOID$c6aObs-Z#Wcc{v?h6ObPV|7bz($8DPpV46gg>Wt=f@cT83JN?T;cM z;GlyQKvzBt7jZL$l_3hyh}sDI+oD$5QqmBcM)$|7FoIm>K~XA`6a;)QfR&>ScwIr< zLXC~E&@jW_<=;xb3|E>Bn%m5-R}XTmuo_>I3C>?2b2J~UG;ci1UgNx5w)S4a@cWne zfsn5cq$d$)XRaXm1M8c{e{>_i$d_Bx51|Bc01vU(0pvK;8R44hMk8QQG$_85UR_^L z*V$2mPD!bVC}sT2G$_{sO%^F5EH5;7L?zQXd0J6NQ|o09)GXSO&lEH5glY3lv@?$^ z3n}f#=y-+R=k%rp3EV&Xq5t{zQ}AsVHTa2g5BIxR9S~L9Z_d?`2xjEJRiPC%rUusm zM?hdqF{>|*>;6GCp{>m)Ah3MunN|iI`G!rdfUoe+p4?N-M#ImGCdVS-LBtkbv1)nGN4>S=&8r9nx_C)Va_4?q^%92Z6p(eABBy#zhBtQ))`5}3W(`S z6LWLQPENejHpBo_q^@l=0gFzq2Ghl5xhAAgF&VI?|I_JQ3C2}T zP1-JBkxKc}yI++$@Z@Q`-Mf1{oOgTe+)5kl=JTB%T8T4CYmGt_eg_CMaxfzqtlR6_ zgFlL#39p_unLlAQNADZvSRXp~i&Z376iVyXZy#{;^FPnz$7nmKWtis~Y2A|gENqKn zm+mHz;(LU?9-ON38mp5@7tSCciW)pkFY++{Md zm+$&&^Oe>L_jZzoFi_zX?-+X587a+Y=hcf?S3d5;L#Q;-7&SIOIO!Py=_vx=y z=ysE`Vx+xG-N^Qg)Vb1B%E!JHHd^&N-(~D~raw&?S7Xi;;h3AHTsLmJ(eb`7@3VZO zM4L?FNA(=8(etO!!oGika%tV~n%eE|gxtVo-6nQDHu^pFf1Alia1IKUN-dV2`4+F| zbXN^K)ghma3?(E2aAvWcwn}9Z6EUkzq{KEdHI+?FkU+cUA+H433q&T7F9WQS!%J`7{5KZm%TZT~k9bX=s!tfQ!D>=lX#xX%=m!hlGyKT>iP2lI2yVeXu__ z$L%gVlt1zJQ0F#AK;WZDkQK@}#0-k7q5=`ed->SBUi%?w2uTu`#ioM7Cv*RoO8C7{ zmrbkb7<57%?UlnTwO$BF%xqXB{ALjhCe=V534+pETi%KG_D8n1w)N!?9hB{3^_P0e zB_&AOMm;56YN{YLVj^-my|_WnX^wO-486XR?+z_tLDvLiGuzqbtMo?mpCN*aW@d{@bfyQ8PRvE3Pif?Cq5WyPjp40-!O1 zLKZx>3yNo=ltZQDAVYNb3W5vvsAz{dSXax(`UW=bW8!6?#Q9ft!hHPX$(wQJfo*fO ze;1%MF*`d5m|gaUDW%|~Fypn*{I{s_86dCJLa)8#F>&zFQA+JBK?ywJUn10pW=b@= z5*g?7A*FVIvHS}+19Om9_L^`Vn4fHS$rh#Pt#qVbcf&_!p;Kc~Nzi4|YjRI3w!U4+ zA!y` z^af6+Nj6iLQfDXh9>O&&DF+d8#)AZ-ejjULb_zX_))(Dg6ziYdY%CH>mu;>l{2ZOG zIYtbZg)g8_4# z$Bs@oQ3DpsOQ&YMq@kwre+F=C3*RJk@!8$6*ZJOii7fOuhaA_heqJL)e%8z#HWqIp ztBGx9gaZv5>v)j`c=g`b8*x9Jn(E#4gcuVH8>7#STBC=>7E~h9LT^WJ)BUcR z=qT@22RM}%1#k=f3UFr8;^caKa-+P|A-%OLB@f)-YVMGeO zm>0+TH2Ja5Fk?S|axFYy80X;B?!}?s-A)i&nqIW3!OJ?)A=TDnV;kFuJSFX(#`Cry z^r(cP!t1rZhK;m2k@JhR&jzkfEi%7_u3_;qV|xg7%kx5aO_`EcEwN&bArGph|D?nu zIE=bfxB!a{RNvm(x|OIEI!mjF%kqGJp^Qq}ee`;I0A9 z9mQaKE7JIMbZ-PsIIKWvEuHOl_u4*Mv7m^!La?3AOp+rbui{&p4TCZq402HhEF*wx z(_C{D0yzdL6e zONj$BD=RI21gH{#q_XM{`YkU8laXXM6 z_e3G!Ake6CrMR0GoVsYYB}DSpMSKhUd1!Kl`g%1{{x!kND?LHRRbFZ?|=j0YJ7pYQ z_a7p+FV2l`xOw450@|NMyBX)tnf$!21z>Bb5rUmPl=n2(D0j)mBJc@|_*?m}rHwmvWP#zDHpfHVE1VjLK{?eQG~VSXjBDtmpkmvEr#16Q)c?T9#@I8m}wruM2$* z%rZ$$`a-=f!sbRD6(w-v*mFW{o^~Akz|Aj(B_`)bvwjv1(UcbZ|(jq z9Ca{5Ttb}@Nf3kme!v0K23=8+iRDmIHMX>N;oi>MdHhRov%uCHDE9DjT$?+?1{>45 zZUkfdz;VS!UU99{^|&ouR;U}ZW^;c3u$c3YOu#}Ay@IFn`+xwbRhWEoN~7YtWvODB z%8Oh3m1j2ICls|d?!m%-M$+)VeWv;#h*ok84){xzfTTLjQZ7~z@c)%og?*x9T?s3|K%%%ZoM}M8fT523{B7n-}^H5XSO_Jbe6X ze|Pu{+&*Sk^JM5M9e9p|QK=sW*W6j7xdD3=PQ) zEI^qd5rc>hF_jMPelFG|Akw+FkwwMLg9-FkP~gCJXD=IoW-q8pSP6xgeS5`L1Qwq& zPgi?@ISUC}UgcN9DmKu>0DrGcjG{os(qSJZBUy|FpqGYoF#shh0ECH9$Z3PdX$cFF z6IuAL5d>tjrxMQ@$1$BLsB{gK3Tg>wX%`*eb4fhXPWkm67i`NQ#BV)HssIJIfN-!` zZ|?7JRP|BNf_c55Mou5diCb{^goHQWgnr90_0d4Efq?u;J4GjbA)!vx9PY?}=A^>-_pT%2mUW=?-QPy9mU?0$@gyb9a1 z#$M05^xg0h8QmD=isJQW%28Y2I9;|DCcoU<9j(s)Mp78}VRJXvN(aYI`VW^G!N%b< zxmeHFGVgh8BO~Nvj<)m-BfU}DPcsKXqM4RLB)&Jtf6j}yJsjn5>{R4k>Z(jvWCf07 zJ*}py<8LEZ*b6+Y=@PqyC|k2gQhb1otainzAh|1p0sYhEqwTkJPOn1 zJjF{gPs^oYcs&h7jv36aiF+m^r7sxNgRYo*lT}LS(TtGmKH!n5e`f8wkkiayxe_0u zc+p*0%v-=0)pe+emhSp4H$re(jr-I<^sdypgq%#mofjcupDuE=9(1AGfjEb` z%kl4L!7GYPpVNAZ+YUT<7VepG*k@&)s-3B4kF3u*3pnXYwgtvJ)vu-udwIiIlH zCTh81%r@#nm85YIwC95vl~Bq5rv*^sc}rHTnmKe#p2fI#OeQuV(^bo1q$xzWW{D=G zJ$`wzVMiB#DY)n3iqTrF`aWX(s}pP3itI7eXFcq)mxQyJi!D>FtiY?uYICIluE*NcGU0OJoX)k7$*E=*-(T4EN@j6XiCPKpC!&^EJ4=#(? zz9Qvke|*kWBBN@c0&J*wXAgNfU|B*SfqYGFZpS+{yF2r9vM-X~e|&rnVF?F){>|0E(bQ zAan>GdG@4xR+yQTKUSBGL|qMgEhHb4oHsV`r^(md`QDFi3t-~4~T%1w+QU80v zxXqvD5PjcER$QY56^p-HTQmerxz7plsPPAj96S4!M9&{5zKBI|fl{4}J;I-pPp`A1 zUqzoV?R}q+@bK_JV=>Uy><1cjoO+IQ8a8u@PA-}m#^(m|jPHdBWbX)4Mk$nk1=;4Cdz+aBg?C{HzEIj{a&4T+S#7bF-dH^2Jx zmf~XH6A|TtmE_R*_L~yZ@Ung0v=C;5tfj3j^%odRqUzkvrT11%_C{voYgnvwc;`>a zpN6~6^dFij>T=Sf#2lXhK=-4G5NZ_&8u#lNnVVa&B#py`y5q1A)v5;Oh$9rb?n;LQ zs9@eB68jnqp^dV({09Cmb{vl47QwD4IxAP~kNgN8>V!sQU;XI!)liXN4JRx_T%;fcN(8Tln_p^Mj+oPoRJ)nDFt5 z#Ym=s2xq|C%mfw3R6aBckAnEiV?_{s;G=thHS>T>gM5GU;cH!=$5yFPGN%qrJnnl2 z2oE^m@FNIYak3aON%2=Qov0v$4~Sc>B;Uxlq3+EY<>eGFDpZbsDq5|N#;Fa{d)Q` zlh2FeneCUag74&}6@zK>6u=vd6)S?$4IRXUm@yczH2X z3dgI_kq{!O5#nBz1d_yZ`=05Ye63TvztN;?6rPrYi(^800w<-b9f2}tsGw%l>|zS7 zUIn7HS?urcU#~EDHvHy;{6L>_Cze@|Bjoh)4Pjz*2~BYc60S5{obY>LHVfcRE6ByH ztvZcZ3^5nuVfTp(N#kBP*M9vuRv)DcANo6%-#;?a`|w=!DSvim)A7A|`l9X1g<^iH z)8GI1mrJHzhx+cey+B==f&DYA^TRlPck;|Th5-!AJJ94_HFoRz@E7JS7%x!NNc zC@^LX-rrs^nRECCg3(BUNvxBVg6wE>9je&l(xhWN`R-QAKFxl?j#fT`Toojapdw{K-|V z&6U2QxjB(&yRO>f*46XIUo^Mbv5eI^6>f^IS*%p>?vJ}H7AN)c!y|+ewDjeD2nNX( zxECJ6r7R%*|Km8w{>OQ|jUdi$BZ$(u;h+=1`Z*5W+aKf5xi2Ct^@(MD&lDmG?xL1P z1mT1l^2DPB)gM196BcgF3C`zqM!-x~N!wlc%a<>8n_SubZdOcf7Fv^-61blo*#BQx{l$8hKd=sTia>F2B>#qOUO9jr% zU|jsJYXx&0==)gWpC1HK-uP$VofPx&@#TQ>Ko&{SRU)&&?H5WlJz*?w6E26GFCM7p z_kU*}n2mR5SE@%slu5!f{()Y{RyOxx~AD@2aG z*1a#rGq19a51Mfinb0Aq8umPIZZLgiC8iYNY}y7o2b~;w6kEbOjQ&4Wkuq^bj-n`Q zt|H_-_8IT89LCL;B7?%gRt6JC)ba+MTZwifbVyj?qUU0Wf%LOqZt(Yrfe<*}+8w2D zMvNkf)BMI=8aF{PE#x+_hOgSg?Q!JGhf0w$XC&QD+CzbB2}A?&f%Ie%rDv zQAZc^=dFF9$K(J*eFzzvOQ{lQl16Zu9!uLx;n8GunOKRlq|p&Dh1rXMW2GEYp?b(m z>MceEK}6I(!|TB7aCl}g+2aySi}yx@Zk$xhBL+?p{~Rjk{#bz_jcyGU*6sjYOzyE< zy6@%3G6w7)P12HKFNNFdmc9AOtU;RnSV;P{y1^2EKsG~?5`q1yw4}5tbufYigA7X& z^((G8%NNiv!91{?{`B?d5r>7T!6P^9gA?0v4sJ1haY2?{odV{>Jk*oQ0g9+oAMe6S zqm(be@sS{mepv_>(LVZKp$Gzh?{4ziBu&j}foqmXP9@DMn)*`(ddPR z!_WvA?Drb>O+n-UG`9#76e?mZ?{U0yf<2BSp-=fjHzb%=sXFg9w5TCJyH^aIoxdhL zKdH7<|KM%_RPYcXA&RmGGt#0qRj8r+X+XmPJ02QW*G7`7(6H zKaT1|Q=AYyAnd{dLGa*C6KNzuBoi?SaMDANh9Oii0_3GvlDJ58o*%yM&B!w)OEKXC z1C2QRJ=IP=Vrvu%M-X!mbKnoTOJnprn1VQRx?lz1PT~)bbU7FXKSsMv6b-b5sEQb< z3gOcPUUd3)J~9!b|JYyYG|XD?0j;wN5(}YemWAf*6k=mzQ};>9{QG~BlqTTeR`(=m z;j__IhNmmL8ezvm%BW88u9$8O`a0s7V)y*HQOU` z4rMt0!5#aIYnQYV65(kQo3$vD=iaOhc;E1H#qgCu2Pv=;=g|RQ*W3ff>R`}`mr|Dm zDJSVYg};p@r%5MTR*t}pe1o?<1)eWQ14-)G^l(L{W&CA(gU)!iPS_I%14)w-Q%Dm*KHjQX;VE?(XhpkrD!eAPthz-6-9i(jXuq zNGdJeARyA+(%tZ%xc5HiclPsqeZRfyi>{k>&l+RQ7;}v4;;Lr1ejSmXj!3bS59}VK zmnz%Z;$~!g2It$(zC@9_x+fqsDn345^6gv35{=JB7YFZ#y$bxas=|7Ek(TQHia7v) zBbn^T*qGd_R|vwwNP2EL$+-Fo0Ih=E*)rvY>?A7zwxuOt4l@PE+~JN;OmL#i!-jhd z?zOJ7Z)PNk0K7m%YJtd^ioBP zFCq8S6_?fYz%T56q?`f%6_g-U;;=twkw3nNhkZk<-Hkhu8DPH}$P&%csxqwrG;ZY11;38{`XMW`!ZJ-rNLNlE zLxq(e4f9~LOS=VdJ5v?0QNoec`?C_Bj<6<_-OK!jFThN|(Pz-+149=~zJvXv-dK>> z^TVc%ZGMvl@0JrXFUAuS)+K|s_X&^N(m*W2;{c!crsmf3L!7QRv<*CEz?YFS^h@}{ z=R*vx^cE$Ng4^{I@t~xFU8d$+Q-KdIYC6vR6+7gg91sn`Zs3s=Iy_kUWgq|fVxL{( zP76>nL_YA8Z+wjJVh7h6AWQj=<@`S~ixQ1qQGkX(DWDV(@aIpZrEdWBPKOcs`Vetj z3^_S~*FHl&xTVU-@B~5j6d6%fHo9@5H{$%4tEnKEY#cBztndLwjzooYzSzRd;*Cz+ z2L7#&Ixlsv;KV;bbJG-Pq`>_6aI_>LAoK(|e^{=wvAGo^8j80srt_zV4e*gC zmD_%>in4ycE>Bb;{9md^H>KU_> zP86h}jv2O^UHtBcGCtY6%B)f4E!q<`@aq$`1X#oif**5J9m|5g{r%DN%33NSzUQvK zGJs9|uW9{%EfF{aeMx9zibqFB+raFE3A{W>ZxCrZggYi-I@#!i4 zYi&oq^VuR0?z#-<*Caq{D5u#F0NlYn{;4)Nke(`~u8vwPFEkH~m2k@m15uU+w$@6{ zeppye&zn1q-ME0w#VUmR?FX$23MGLIm;?a&gAod)XJs{;Z*)v$GZcziv10<45D&yq z4Gn~lzmiO6M-9J?u!Ta2yw)C}R)6KH@|=@*s4`MW6G$vAWrFVql!SmR1a zqF(v*dFVQJ?4}(0M{l^CtX#2ao(w@WLM#U+M^99#wD z0g34;zj7s1oT-i5*`h&&(ev3k56EMuX*Ltdr|gDAnXZI9>RSTK@WWF4xb4vs2!~G? z@!MCUV9S-77cVkWJa7&;XYe)b!V zA0v>(dX=DOjI>0kloTgmpA86R0*llN0f&d{;Sb4%ZzrP?M(hvdk}nzF7b^>q09xe_ z4f{$J*4mFb&6)vp)3*xUYTjMbjp@XM^)(ev7bcL)-wGnVCjkd5IAZP; z7N5u!###BSQG0kUn|E!9$undatvr9ewY!(AEaOPYDq<%nI8mDIF-;VdpaHhuEa@3wce7P3=9>23Z)4P~Ysp=bEJln*)5ZU!4a69^y z^`kmJGd8{;HX~V?Dk+k$ZH5{VX0)k^YGsbgMr*j9`l-cab<}(&$xB#ynen*9R{*Rx z`1h8T*{fTE`k9#*3C7qLBB6UMF4mfZNo&t1t3GGOTXxxPQR6x;1>&l{ehr%Yw>TT; z7}kAK^&@|aB~Cz(k&9SXnR}K-%V+x{SWLuHmLYk)$N$XZjht@j2JJ!5jHjzDz!o~n z*2fW{ym3@cSCI(-iRZY!zlHk*b@p0pmr7Km&|x zysupzU@=H1obj!;D?4}GERZM~%fzr*v5eX_wGt8%z=^=0o144B=x+?((oZB|m6F(>9!8eF9lo3SyaiNaG zM#_$y+rK^oWE_758vo+aio#G_pL^{jVK>4uTneH{^5p{$=p%?2`h0q>IeN@*#h5+C znBEIG0+7FB52Y*rOet6%gdPv0hisntr&_4T3QI`J({s=@^J{ zW4!ED{s58>;T;@dfB%;FRtD~s!+@QROuz1<6{&c)UDgmaI;si{5GmVsXpF??3T8jo7yxn7)>t9f_WFmgr4*|IlLaKN{vkhPpqqKcL`aY&gW|p{{}4Mi^))C2 zf?7~a#9FqE5#HI;-=7CKXu-Q`fKuy$>hGwR*8Oz+)f6!JqL;|vR+B&B(L+6SSQG@2 z6cj}C-S!U~urwZPig*PIik*!*mVC-Kif9`bFO=x<_ zuNt)i5X0%|=(39~sm2NvR|YtTACeLNI&=Qd;R8!`97!2K1&GUL`IAm(V$Hza-F+a<6}7>2f4LQ5e=WIX zcJODBMVp~!Wk+ESdDii;XmmJBeaP}MU`54TTa8yPXlzUeI3pDx&FuUWX!i=0!h{2Du{-bHVts& z6)0uM(_t;2f`Grt3gZWum&?mbCY=j9G||RH5o+r8x|^dZ9hWJS2i%l;7Ft!1bPAmI zjo5(KiI0d$P%!QHZ%dSYO$t9ix4W}Z&jMiEfDqKw)dPzixIlvGk~zrc1p|_y#F@g! z(y7k9iiOf@4|j?WMJG$$DKKnjsLVB-=X7PMG8?AB2(@E^!U2%mznn?`m!RDQXqU6oC#WWhVzbA@jD!l#RGl%e8NV@e<^K8F|WQc z0SSCETm+rRLjN9JSCQxzv0{LB4bt)VX6ptr1abcmZ9#z}GCTe(4TU)z^bPkkM*SEML^WG`RI(qIHEeCgeaP6w&*kC_B8<(^IRN z+Gh*rQ_4iab{aTR?SSMcFPlgV96`sbone8G3rOS1lPCYRuy){#1Fl+7$j*FITC2`P zUbYpURJ**rBntuXQ3Bn{PPS&uq$O|w)&?fhzs}JAdlI-gJpgOUK5)@c#Aa7i#Dbha zct@e=#!C5{?>Tuj*v}GLJFq?K1t;q~_@7L%( zdX;m_>4OM`MAr9ni)qziW8V81bEc>2Yus&m5)4!QnZ(+7;0EfYx;n^Y{vXr+t3~bl zXlP(~_wtG#=v_hB7MNKP{T@LKTbw*OwKMetDBD`Gs1Xx(G$X>okU#@|MfGzkbg`2&lc@5Q5DS6Kd}I)|LXCdH{k!-0sNop z@D~47{-4?m-~IPW{{IF4J+eWM|9>_VB(^Fv@#L|xF}-hIsG|g7Dnspvf@P^Bzy$Ii zqMApJ{+bPvAV3!W50FN{M+68a|Ge@7Bpd+-!@j=0|EoRVTafXZeFSjd&wRBl90n70 z@8Dpk!NoD)0k5v9sc=6wb#QR#>Fr$_F81UWy*(4gYThScNfQo35Be&(k9v0ye!sky zTL++GxN`rGCV`^6{oC2Yx>o?27vMz(OU6UFVNePbVu1v(xPcAb;Nizqstjvw_f-T^ zvpp@x2TDE=kF?~sFC?oi!}{1W|AX6Eg;Zl>{JowKR^N-r6|_P(7Z zUS7SP^ziZx*XOk{OBY*nbwSsp4Oue#I!cWH()sqoWxEKlSb=43c2?&u?u4s!y%$%{ zP&qn!NmmB?>C>l&fm@;O>Cn0p6?re2s@JmTd-6s4f`52>#AQPb~ z2hPyyBhy^$m2U=n?&;o#y;%Pi9}z1_x2*c1cNQi?yutZ8w$Sm3Di61dI9Ie`UKhLL zFN+9kV#^KW7ki^(%1C~zb-N9hmZKgiem%O zggOJxbrD^e*3F+Z5evD`xPGfSZkoMGe2dTCvGPnx#;$eaF81laH7SZV+BM{5w-QO+ zDZt8e?qBOM)hXmG|rU_0LsOx&EDs=X2lYjQ=Q!rIn#rm>c!-&5%g z4$bXp+DkQ8>Q_Uh>t}*Ej@4XaNmSOfjR6D>w^Za2`|m+)R3miZRUhg@3(&ikqb+34)Kgxy_EIeCr~eneDHr z-OdW49nU;P+F_N2J5G~_jI4GOnJm|y@vSNggqL(V9ab#}$$cGu%QeR%;SSog*0COfv)_Kgy^G3IrSH5TJj`MJX`P zhAXD?p03--YWY5>U76zC34=@-V;Nm#GgYNfp4x{0v@F+koFl36M`!i(A%lw_vFSRg z(UNh6KIh!chvBdAxv!CN8hK%Qd6Go#V4EkiZ?0%xsXh9{P?V{W>3$@3= zbvIV+IC);kvD)u?+)?YU!^WVH74^B~VucZj#@7L(;@~yvBJE`5g_6j9jc+&qE(}hB z%;wn9@BMFQK{(C!W03<)qN$854Jm8GLVZ^TTxQByVx2uBFIOMgQ}>*FvFVjKqHMqT z&K>`WBn^!#=RF2opO5aT@{uy{4XNHxdfO}`ZVWDb3uot$^{LZ%#2T9D4TY%Jua(m0 zAN zT47((hU4Okp6hkBSlnfUCGz|Tk~R@Vl!9t2kN5hwR!C1RY1xYD(r#;6n^Ut`eOS5w z2%Zr47D50196_teo^baNC1*#1K{|6Va^Z@|eR!_l`su458wLBU1;d}*vlc(r%dNh7 zebLU^($7ljm3d`7NJR}*>BKYD^?_Zu5;cQ32iCsY-zZ#I==kaEYt%m$uGLiZ-tRXW zyXhWEm?Yjjy~oyUB9$zzYRaJf&w=7j=7{D+d6DD%XifoyQC;imj`&p%0FBnK|7Nnh zWw;)<)?s{Mu1Bl=Xwqy$qS>*VLt6KmyV?B|KVc(~e7tvgrll*{=YsBSL%FFb>NS<0 zE|oO%;&$?IAzL4WUf|Nm6xBP1*5@9>(I!Pa6W&U@E<|xjsH}`2EKR51poy8V?tVx+ zB;A8v|88_6m5)my>z8xhcU8ZuL)R0>i<=G0OWk3I=~1)eEmocW5OR`huE&R1 zHpS^^vlqPSLP7|J<$E-!)xq7qUYLxd$CtAcqTQGA-Eo*OErl7sxB_azBg@P39gYew zXIOob*q@;KD04P=2Bf3^dx14{OBah;c}QshT=7Gr`OiTZIDj?jiX`lAbUy}ef1={x z&x9Z7da|Sx$I$~;3~AABHkAbAS(mR}btIhiLYy{Ol2~3VQms1nCF-v%)HnCLCM6%X z``)l6UP2npQJ+W({hB?+Y(D?NGViCSwWwy7x;ByGOC%nQ{mKh_J_0p1) z&{c6`y8PC!K1_!q#|pWUy)hoz!}xT?&2w@LTMVp)>|w{{SzXEd7tfv!d4Je$KE`Ij zdnNBP@`3_2c)a$G|M%B@!*gc+rK5X?sp0+Ik2Qjw1-fjGOxI9jyRL{T`Ur>BHtclw zA-S6s(tShg_Ll`XyGzl&ZtsyCTa>KjLRL}XRq9{6R{D6uGHUNHUEz{ST0emHvD~GQMCJw z46hSEu*7`z>F;#D#KK;M*#FzOP}^6S(IHV9z&HO&-koR>ng06-|^}F4b;|^LCK`bo9FJ?IA%~I$NMfMS8E-dWrt>fDuY}a)_N*2OhT# z-11x}=J1}u*^|U?L`^UIy3t_18)iAh1;C>mxDA{6^bEh;aUk%;Y>yH+Edbyh1 z((PAe8S36xHk8Ed7K^7|6OSj-0epXkZan2k?(ukfzwMofiZ^s6cYVDRX*0D4Lt=*y zmTL{YoTlN7P^tJ|FwD!na>S98nNZbchYM~+E+W(o=uSfn?k%RH)#fXm2fms2!yQK$ z5^dxik@o8~3(HA4=MFRN&)xQ9NY@t5uc)mKPuisuPp&aJI=I1*5l@H>Z~v}EuXDQSyyI=f-EtE=hjr3V#myc0KCgh?*o&N< zMkM2FP}nF7>NU<$G$GEB{jGNT#YGH$6@FyD_R*pRmR3^b`fhTHj++8$PV>@C)5m;Y5TUqjS zYKnA!uGCn!I|K27Q zu32pk3CCeJ0*NUPOO(tWQB-6V^vAaoXuxxt+86}V=G2hE@+0MiTO?MiA}J(N{=3%r zn;p?#OMoiW^r@WpQ@dTQuF(t`6&R<4!v@jD?>T2SL&avjqGcZgIdiLmILXSM#ABee z@u#QT1zyBhFPcjWY`d>*(Q0Z>`z=TO(j7ir!Tn{&o z|MYc%`mEjS^Od9OinG9pPiGFV6|yAY&v-_#aDcP$Umuo%o(*suxhPej!L+xx*JQ%E zi%XFEYM+sxIdOMEsIkK*9j$NwA>h#UtD{@HU}KFXChd(^#QUBOtT@WFqB3jBPmi#3 zOFqBP8+L|6hJr%Lo7-jv3FHWAO8Bps$-EshrTi!) zwgpW^VRmn5{DZI_pd$}sD6Y<*Q_^t2U~aHw+i&5cz_Sn&-WK=?_T*j5MR32VJWtRC;4p+JXcuO>2|p%RC9C$ zr`mjjO-+&+-Uz|64|1lgj0V)s2B=e0-4|2ov8z#;{kE@?F{_gg&|2St;BFF`>D*;iS!LI%-}$tmR`jhT zXD>Y6SP5M-$#O}-b`50?;bJ&TB-q%Fe5+SQ69w69P!H9Giql#Q-t;-#VUucTxsN|> zrh=)H21-S$e=0{oKrioHYo)JNPr2{N;Li#vDrr2VOHcFSdF8RiU^gQl2;L)DJT?O$ z%2ZCxo-_xv7Gk!zSMwgxK`yMMeGSVjHUz0%77*`L_w8pLsGEVQYA942%B$A^r*8B5 zjwM}0c1tvUs!ICB`tMx*av`GhxS_e|TgH)c>)gnACYLLHXIfqqrBxKpEy|69@8{U= zGM0$Y0@ZZkzVv`iH{w8B(iY87 z#PBtZG+eG(STbAZLj6@ADmO^MjQnNYaYk{0OjI!FSQCRTlONYphec!P%jEU(N^+** zWHGf>iJgPI!`=&1eeML!;N`9-(dZ-R!UX)i1d23P3Xem zRbS1{XOZ``Z*pu$;L4h}aBNb!gHALFGm|)ygl<8cV$Eswg(8;TIeN?8;oKV;clwzl z)bM%Df$7hl7Iu@Di%Y=Aci58P@H z{Fje!NzK~{Bh4z)NQDf6#1?gJZNlzoQnQ1_Rxd-Uh=uK(hHgo+kKfJPOdZj&*S=fw zILP;Gw{1a;jEyT&pQTeQJrdTnsrMpRNJ7lb&0~KeV0UfZK@Bx#7wg#vm(2+JWC0kk zNr(t4BP!68$n+dUw|aYLcSIyj;*kz%DI*@gPw35=HXe;4euiuxn4y!Dp@U#Bo z)3w$b@*fZXJm}Jr{!h*M^JIwQ7(aE;6rd+)fnaC?0(}+$1o0me3V7h9VJeabE<79@ zIQ$XNq$OAS+y8X*{eP*k%ghA&f7j@LzO68E0sBf|URMM{H3i6A*31RVvw;`L{9 zFGFYmppPH7lv=?Aub5Wh=Z%DPx&P0@+@tH;Nh_Y_A-`5Kns}tVk)8>4J|bx-TiT$x z7mH5|Y1&}dM8^M=_W6mwIH*86FSfiM6gy(x@rEPkunydPLE);VBD%~lP~dxedcH5& z9cZNhfnRL%5up;HA`svPj%`t~u%P~_9Y7a{L`Tc4V1gKfFRwp?`ngy_!3fOk(ujzV zM6xOXU+k_6CMjuj7p;?v3kEiJ;FOj0iV(QN`#PiOlG-8F0bLz#<;ONRy5rGdC)yziAR$a&W&!A=_t z>e>KUI4bfGeEiTZvt^ftOQ8rasZd0}{}cvz(zVdnv&OKKS683z?RoAnOS~*(jd0Pu zKd^3Q`^wJZ@{_PrK)sUaf3Z}cN%+z%YOQA+ICnG3mw*)gAge!WX=6Xb$Qu;s4(oCuOk8L3PK1YPAr>?ud1~J{QsC zBb`1Yl*x3&eJ3g| z0Re7g+r+qBa7nnB?2n6*5aiLht&deR;TT*)IetzcFmUq(U3$fq2Ciw&$;D|AiGfQb zkmARfRPb2n*jC4u)6%n`%La_rg_X9s6>27(^|p&ymULnkq4#WA@>tCzyh?>n^<^&v zkPw=_WhkZViUd4GytES#v~#^id`e(O=ZmIfhM+6#lW;&E5VPQ4(k@p>8`34y_86^9 ziMAO%iy~rpw{09C!#!^k_n6J#17#@DX6%vp87?_cFOdn`(CW^UwRIzj+oENZk+ik( z!}&%(E&S8Asi{z2I4zPqEucY5!Iv z0W$R0$f<60i?Wl!joOVN3qR83mV+}GCKgviUtW;kB@q^_;x?DzbnS=e`;2^Xt6*$R z+}`37Xp|XkbIh{D%N`lCLPGr)hf?t|xPMpV^w2g}Po$^5+}H;{wj7qw94`^C?VVkg zLAlgp8Lsn}zLix~m)h<^B)|}5<>Z#a*DXZ5(pVd17A(H9nJ7T!1obw)@TRG>JA6rd zG>ilSoZ5?Bc}L`!yhlgnT)s_xxv*|bRihsY73c527>_13(9(qipjaQf@c1fisAoRx zMBq(j$@)Is*6INL8Luk=Ogysg?y)BB!6 z-!5C3dw34`q%3T03zf4(bQA!RDVS_Jjh3EbS{im5x0o&=tU(LWs3_TCVD%qxPQmF0 z1s)bxZQ@Xv>E@KO^b-f0xq+yW6%7k7Ni;%mduu0-`SdWSvhdJoP2_3>p>%FwzY-bx z_ct7d;P}`U*f4p0R&{e8ZaV^bHIRKm7vXgpf(RzRhPj~y3Zl*2vmPZEo7vW(CIo&} zJ^8I1g4c7>svL2lYy{uvZde!?e#dOO5t+jJh6dd3Xx<>&k|4)rCCelvvQk4@_s@;L z&eVP3`E`57PNLs{*;2#mj*pb`ES+|CZlIz3F0)&IQhw(L?E14< z20}-jg-$osN}0|95V}}GXHU0#U(#W8i$|2P5@VgeLkNfC6ea{r{w@s}!+mud*msw_ z;YD`zl@3>sx10_nYk3YU7?~F<619QHDI8>}<*?D3*&URq3e12V_ z$&{i%!$S2#rGIpT)I?p4BJb+F)r#vNHYRP3QO|T~S|coYW0?)Vb$j(ktg4HxUi;Z? zenN7Rs-Am7)R)AM0dN#d(#;AIIq35Ar5cP`!G%7wPqVIIg9jz?GV$BpVFnwo2eLFZ z)(kpgxR}y^jtxZVdQLTYV>X~AscO8ImvUw%QRAXNtVmIW|3}1TIF=@-E73E{NbD+_iqg&U)BUdw)696#Gt|XAyx_8=rYe0mrqJqp z@CDv!KK13DGuc1b%$ldFG+@|b*NIu!?>4G1;7GkF^?KHDITnAfs6EOD|8eqLO51xD z%aY%SQ$``xA1;Kzf(+K&s;PlMqJ=vS<CA{LAoq)Rw{WOfB)* zcXA)494!JpKb3s?Zv32L+H{fg?R+mQsb^HC9wlCk(TW66xY*d(heNN%`1p8A9ST1q zuif{w9#?pi#V?b1JY^xS6FVR4orRFRcf8CuofkGACAAbBb^bQl%H88 zjP(wmnq_~te?K!^XKHRcK+fBcB-GU+(&R^6GnNwHbI(MPF@-iixXycS)Vz^(*%{#k zcK0m4YWyZjf=DVpmTw91V=)1KbJzWVmUoJaBt`Xv+T0R3b5~fGocM=r%PlqmS(FL8 z%};5Kxp-W6maRiHeS|_(R8nJ_9&j5WFXNca)0gv{c2#*2ujxPV`xo(BOe%8DfeWo^Z(p|wYe=n(AZGb6%3vGe1T z%~y0X2Gy&pgF?a_4r;Jcp$nOmjoJdCm+h*2%GnvZf`nURg7A^bc9Wms(!2R}<|MLm zMN&|_sD6w~OKEeuI8upo4ME~FWm-1fqV^c_-}cDn@|QF;`}MIHcEl)HrLPv7c8=!% zluBUR=u7LB4{bxd5cpkWPoH3?RoO`uGsT;{>&ak|rY7-WY9JV9&&879!=@Tv6kZji zad+o+OZquMZVI~v3+Hhh8?2!(TI93w04#m2*dqKlv>z*-Cm=eQV0RLJtdZ$-8`TuE7eBr%L(0(2Ojsx!)oNjm1LIrX&p8Da zL|>J9H%h1&YGO*t`=}U4hRhNL-q3JfQDfMOuN-gNh~{)U)3r{PfF0HN%@CZRNJit%Wwko}{f z+IGH%oEl5kks@P!;c7IV_{Z_gNo|ICyyDXsJoY7(uboKI#vb($2b1)#3i1Pv zk^aP(fxb{6`qKCn&PcH>N7a@qCo1p3dP84a(qRTuyjhN8(HVX<6fB2_EWNow| zbl?a1=u1qHhXy}%Q4$@D2|0xhgUW~nG%)=ocP-{R5yi`$Ia$6s(Oz~OFgRc+=mQbW z0Dn*jBE_FUviNHxmjD#b!w4iGZv2UTO#lof6n==Llp%<1o#--m*>ILT#0440&KmoM zW->6Hb?jb#e+y#+0|FKMQ?fXp6uJs%6GTMr;a||0Hut|}fDTDOY`X+smQ*>Vw0k3J zRCM)Z$f(lu%}b_~9hI^0L*?);@gRS?1R0{12hq0=?SWgu0$rmI#8`@lQvvOjN1{lV zc?it~4FyAYU=K!wg;_=bv-{21*td)iE<;N>wT+o9&qbuXt>2}=?l*VO*DBbga3=ZC ztKjidQix>ul{RZC!i!M-={Gq=`tyh4KwT7G(7%WF@^78_X3oTZLi$pyc!@2T04AOK z+w|{GwxNawF32>-Azh=yrBh2LM<+!ItJH?;DDlj35$O_f6pCK5f4v@RNAgE9^noD< z3zbryS6$X8uP z%P4Q1kn5(VpsyqUDHzI^4B`?WGz92LNcO|Z{kL7*#^J_s;{DbkhEApuo?ex?dq$B|{LH8fD?ytr?*d>%t@uBN~83fti+EMtIDu*0E zJSA}QWN=~$-mXbL*a)?s9gFyb+CVQ}7`4Pk;KfN+r&d89boikN>JYlWwJPwy{?DkX z{!zyVt%pD_9R4;TxFzF1uK+>PNPx}%8a5P39!Z}j_aD6+4wL@9Nvtt7Kb^bqR~5j*0oM;gj`dxSA(lH zJ)JBV#%wj`jm7FCv}-Zoz3Bm8&|v^S9liHv_e$}lb3nxu1-|R)F4lQ?*T7o0e#LwB zK@FN&j?~#cS9g-&g5**(gz%`J2I8f_PX_2q>nN(*)YOfJGbxF1p5vy0xv{vQw3J!D>1F{ zE*uoB>!sSh!LKh)L_@>+EW{VS#lMERX7o(t|CeV9Z&qPfZk8{y=DwWNbrc>)S zq*fx^gs|o2l%cXaf?}t%U~oz*nLKW}cf#)UhX~O^Rr)K9-H&x8Y2(8;QMX#KAcO&a z3c^xrt=FCe`z#&f#F>5hDmXlCh7_~siiVxrHf9%2X|c@L|i3OQ}HQO?|WtHvy_-Ee>3C*^3H7! zZ~f5Qk!sBTvSqkSl~=?y1~4kv%|(n*LGj+R=?Z3_s*rFM){6aZDoHMV7x(cKS-8Y@?wxF=*vnaX6+gomr`&WHzHK&AiJZ}=cRXG#kfOx#>Qz3Ja)Uyd?_tFLaeZ)A@Q))k1A z^-P6mUc2qTimswILYMNMB`R^|aM(LfNw+TeY=ib3$F`F?ESpyL_m5B=$2D^dN(HzE zZTf|8eqYhP8p4hu7do7t{IybG|r8ASaVb@SX4mM|+^KY;fy~^(} zK5$mi@_6JqLIr;sy5R>M8TSf^B(=(~9ty;351|Z=V}HfrVn^UomF(ZIx3RV>UZ5Ca zdY&U)DKqfqZ1TOV4FB8t+jmFsyfBlPRo{&Etd8P|WWNLlHIaVtlEEiwL0>2o9W&LD z)5v`Ws}ILXFGc;P*)v44{vUQQ_nJEU99G z(?yI{osh@=V@}3S2UkRTP8*4(i;eMZlwvg;(d14=fQm>7vO#p*%`RFF&u{3pb>)ALH7oX9f;+<(0Z! zIuoG8rrg+|JmIW+yDp1COw-IqLvb9$$@mtQ@zs9rVi^RAS+F~4_^Z^*-%=l?;04<1 z;QC$(d_10ocwi%u=(kAMuY0*Q-S2uKG9w*Y3iy)$0}=q$S1{SRa8g3scWu+BtiqCpVk?*P_OlkjmH?i=if3k`qtUYjP2ep{tLRpL(kGCJ+nz##@{WhW*Z#5EQlPW9AI6DKmBHYlNF1{}VkLh_?U#?4X5Rj!#ITf< zhJewj1w1C2Ri00~>o?dEI6Y={FR&+33}+*QwQ8E<=ASrjVn3Tt$Cxd*els?-r6Zcl zniLc6FZfk;F17XNiMc}3quLOXvn0}XOaf@NTM?_ESZa5Fox;+iIoj7-NM`635|S&q zh$E`{;Z=-LThv-qQe82xz65&bD-IB)Aj97=DLi-NSHC0twl*^XQRKb;)!bIBz8@3E z6@6nq^l4IQD|_E=!>l!^bsyqH|B&0s$ZfyMw)21m6l6rVCJ-x}} z_F|Il(1fa5owx%(t39H0{-jos?EZG+Q;}^hAL9o^O4383DY^dahKpZ*X(2Cm8}B=g zne;0|-H!`mj25X_*JRFOZzBaY?G*L>h@Bhn&m0>Mm53J7d67JiS@e8eh@ZIL2H2X& zylYAA)973q8uRv>wj|9fAD|T}LzASTsw&M1QgVfi*qe%AA#zomo1?kTUr=%^GyU9u z-IjZ5`-OSkh2|`Jqk3h{ia)*I)#t<*14YD=_w)MNL=r}<9WAD+#ExEVjCD?ln_a72 z?z^)vd9QbgtVMe`eII5g5olHxB?2;Q<>$Q*b<_Fd-Oa_>A@52oiCLQ)(Ko8EQ}_#b z;gxGggA$7#Wg4cnKJh(6ZYlpZSk!8rlZ1W>^z-)^sTk1RAw|! z%OchiD#u%lr|P*+o;O78$cjjzaj$jbiQH>5ZQ_O=AH2?Vv!G8qMqp#)4a;0g9AC;6tQ9@pn1?1%L;G`18>>s>4w! znX9zSGSl-6e|{dXH*yT#LH_HElp0-ZT1(xdw0%6=y9rXavm#@Kx-}9UoeXKSOX<6l zWrklHICgW{a6*H@nuntfZ+e7CQnH>?*)4x((DS+0Y8=})m|nzE5*+!?-p;q@yNF{& zBdY!ur^>G1rFfQlwbL#x>w~0B=sxfJ^m!E>qNX0!us6H0l1?XCvbl&&l|J->1Zv;bXLx_GB1c)h9?;$|%YHQ~RtuNOXjwr7>0|6&#USq=%tg3B>WId(qR2Q$7{%?c6&kH6m9H_)!!EPss48OHS zsN1MzlwFR<{0*<#;z;W_mW${#F&woj1G}Z@&2%5&3 zAl%seCa6z!yS5&Pd{(ucVdj3D*iBFD@bsKjQ_M;k)>gPB1~)nXxMdGt^39Ll=J9Z? zk<6})^V^V8xTnoSV6FF{FNgCQ$P$L(EOb(_|K59xCM4@pjpW;JL%@JrW#-GMMH$Zy zJ4Y*Tk-%iS{?VAz!9vQElS{Ya-u%fBWQ$50oA~oHt_Dhc{%kaOtf3@n9by;ccqW|~ z1JUHV&(qFNlZFjg6u^DX+P5X6m5@>-a~zk#u`Fkb;id7Uc8G9Z%a}z6la*> z*)1q&NV%ay3>GY+drNC}p%9lla52XcHBm2%M;zWc@Z`x1gS+FQ>cA7nZ|8(k&@@;g z(Tm5drJ4~79-LLkd3>Ln(xtC&Z|uZ*z^ZGzH23w@NDN1ktuSANxxiH81-}OCf_z)! zq3oc`yry=0fcCyA)KsXsmZv_F1dB((OPpv6Wzhah{S7)T%?TwlO#PhV&#ZSjoSYR} zDBlE?e#{$@(T)rSA`KbOxbHUUzKN)d4GtS)FR@|ZEnlshSonUD;YqjGY0>Y%dX&W zg2)Wk_h}Wj4_{B~uq%)zDN5+zy2J81WZ&>{hJ)CB0>1njEAH^*Uz|6;_U$|kk!f0z zUCaa_b&hqkf*@Z0+(ml94(-)5g-2d5kHEyz#Bn4yTOIkEhj%Kp7wF_*r%(}oK(k(0=zJ0@}H z>}sd2hXHSk(6Hc{`W1(n_IDX!Z`e|`-CKTUi0GpB(=u^x>!p^v6>O679>cwxOCh`G zM3I6E%SDO%_GFZ)DpRrP>%~iU+i=Za4Ia~&(U%KfQuumViei@Elbo~20=iIyXrbl^ z?k0>n7c2M;<}} zI72J8hI7lj{5y2(XQZ>T^3{-6?LPA!XT*BRsb2pdZGYX?=JWKA!)>9sySuv;g1ftw zQrvi=W_M;^ZTF<$EXDN9sUk57$hsV|@DRoK$Ju1dGkEHk?b}~!H+r6bWJvVOjPCKfq z|5~qz?0V`qNBup7`&n%}HJ{xJKt=3Zv-LL%dJ8b?na5W9r&&X${%&`vc71pi3#?lB znt>h7kQ+RVgr)LTx64zsq2XhGWV;D@mh~q%*)Z{Ww{im;p=2Ve-{ozlOD$RN<&b00 zpe1wZ4M#WmhwUB`9=o51m^p<*jz<)qS_;O1jOT9Ovatd~myf@}YL7oy{rp6!GBLuc zzhdX>&G{Ip=%IG~1L>!+UOVU=i7-!LN<+bbI4<6iiHuTRgg*3xU0u+TB$+QJuI5PJ zZ^J1{Ua6sy>&=eKzw~4;EVM;Xlf3z!!GWWijniIhW+dG`35lRKcSS3L!^}m$)8&DS zBH`rM@_|T_R3=>zBNCqVxZ~yis+H7M!#bPMX#tF#C74K@9v^A|p#8{j#oq?~x5*BF z)d|Lthr-F6XFNOP8n5N-Glam)u*K=t$##v3ru`dUd_$72MNov@@vz76ub~@fpR==? zZ?;KYVU$YtHW!GCk(UfXVR+ejsqom!!s``jm7e+4og$jxoPX$!|L+wNbb@o|nX?^e za{K)^>2tyTaaZ#F(Y+Mn;zMO!-61?dlrct7wxM`dkbjRY`FWKV^n881c`xmkVjZ)sPj%D&hJ<^ zNht7ErRRsKfEf#B|0N8|23gy^Ly3FvB62Zk>!yi3Q|=h z)O73DCTctpX>|riOS0;ro^zv|xr6QD<`8Sm5#U|B^}<-2-u?zwdS(Fn9P$2QbeyDqS$JuJ%1m zV?dT)P7)geQXNYLnvhUzUJV&#SvrWTeRRg{XUE^d%IzKOmx5TwPFbGi5G)OU)tx~% zR`vt&mD1nnI<5P@a z-X_b?diNLyWn|K_S3+4wHVq<4sy6q8edJA#FhQ2tzSNHY>NmA&nm`8^|v)V{qRp|Oz{Tz7+V#@^z{ z1qX|b1xFA38a!%4{i>%O3FAWg`XIE*?3t4EX);qlC>thgl>3)2KYqC~^#j+4oULi6 zHyQZd9RYftaHC-^Oq-qZC+^gD%c9(v)hYIen*lpuGfuz~>nP3UaXKb)J1| z^#gDV@Z^^C$;W&N<1QgJMhYM-c$33_%N`or-imcMNa^_D`~~7<;%LiKnT;1GAs!3A2$2NFoZ)ViC%CL2PmHVcow_Qv+Zt!OFPDss}@{> z-k2^)7a#M7%_I6>kyi>Sh8gk2m4wM63f~t;CKJCfVN)w<=`mHlv%VJqUGN3ew6!$! z!J`QE0KZ5`PHQ8sMX#r1_ z=4-313ps01_GXcY`yopQr&7!lYU%O&z#29h@l3>jvPy+PAfHQ+F!lYF{_d5B5)l$C zfkAS5ko~K91lh{e*bm7-2Tb?(Hn&zK9+;ylack4@pH@>$#k77-FC8h$1v{zvZmLhR z@-pM-AV0j63sYfWOfXdetgVmKYpSU64~fP?fU!2;%lK_Olbb$D?d&p;i5trTEAKepu#dusqY0cYnURq zDgTn~Xf~y{2L!MPvt0aiv-I_)GMdVbTdgh;{kTQ4Blyl}__WUoB_L^rzr;r05P1y? zS>@zjwVuOlUsLG8JJy;-ZC4;lL|~1m1YPd;fx^r;RW0>) zurekunI>^n71iO|Qy%1}NH0ACrE1ZiV7r&==6NQlBQ+4sngwmM%eU{-;T_!q)Re44 zw*0kpSy`Qjt=OljeB-imsj`3HHdG?ue=zNV>&Qe#Vl#0cg=oZNhApKpe=Rni)mtk$ zQuw@X^jA^S=(9hz-gOpF>6l@MV}5#BGc)yb46)m|Zran~t#m+7&lTY&rd(5DF9ED5 zIvv@@oo)PxR5N}5QgrOa$AQv!(9vwTVapllMS!W;P<7LdwR(=QBogYU*KxTUw02eY zp=kE$K#^gb5&;;Pzo&S51;f68}~VEAd@fl_GngI+Z{0}ad36td?Zmd z$JU8a zB;R;pQFy(XCHzg~l81r@ee%-Va523)Xwt|tRy7YtEuK?VgELvWl~+p+IzFkIc74Lb zWt}Hv=Zv#yJjYv(=ygZh7ZytOzBD9a@yCnP^6=qT1ri44cj;0|d7QJP{`t|H@cnmC zOhvaa&KuwJ?m;G9{+4i|B5E((TyjSytT#U-EZ0;gOW0$}+Yc&}!HwMcInmMlzJoNTq@z`h z{TG`DuC(X%h1yzBq)0ivbs->2-oa;SB|pE=kK!WuahyOOT>`?TEwhhr{J$#s7>V#{ z@CIhNS@V4qBW6~edoi?AVf$BuAVEo(#$mUh^(Y_{=FaYEuhnmZ)Fl!YUc!_#OH<}) z83X?I3+o0W*Ca2N;e~x@E-`a^ff#8%jJWLiog^wat>Y2wK?RveH5*zM^t_e7G{38~ zTTh;{m(@af3^ZzLW7Zhka=u-W4vwAomm-omB|W=5eirX|zQrsIw2@|gY3;e!o-c-|}$eHs`x?rKX4d!W*Z zVTi+Z{L~VbN3B_AV2DR^Q0hF`Pl>iB@U{iyM%p3x3Jt~vpr(=Fp{$%c&=OkScxOUO z)J{f9NcVPEXlMsUO#}R7|H4KMM_*6LZxdI#ta?w0=AD78DTFq=-d(Y&U+8FFaR3xg zE+V?O|MwO0UdJi)>lbjuFfc7;(<&Uix+^KyKGn5{^@6HK`Na;6vrk6wARBSbPh=6BfNt#9^_2IHFs$YukfD+W$BwprWeQ%3d`h~e z$kqjrk{Sx1%7UvSDF(1DWKp>e4s*;SY;ffds7t5cB^tvZoa?@t4eI$w(YDeU^U(T2 zsuE@g-=?46`NVBa_cH37um8g9tIg}PSkN3mo|Tz1(fyuo9gWb7Iv*YeBMYl>ta_i* zNKbBqzM>@Xvm?6lY5Q78bjc5b5%KL~WC2J)gYY{bJ85xGMi+Tg2s>Jb3|rZ*)rw5}kcvOT8-(dP`O176@gN>lt^6d* zaFQZz!=&Wdwu6#VkCzK!W>hmA?ZqSUV^N!`Y)q?$6O-SfM=o#YJHDlk`+LTE)_Qq> zf2@qG{3fIdd?G#YQ?yF3L)??vE8`5;w_eghCKOr*mvFR8Xc@4ubz)O8Ch*@S{a}zB zSBn{-2TY`9C^dJ+_E|;f|9pq*!-6)U_1={oMA_!i*ivN6Kb8u^Z>~8zSs>=YCbrlU z!g#*mT$%eJ=j1~P-$%*0?X_KU;!aVrLl;CuJyNuUjY&}cZV~OP6=Q@R-wdh^)7+KqF#DEkNq+lU2;;vq0Vv^ngY9t3cHA5^xrW_--Q`t*7y zH5+J?P;nm<+s3DX1b>CR%RkrzrdgUCMYcB7n)z17;-3-?=J!hVz3(Lr1H^AVi!Bz=zEM3Or3Tq_b5+%2BwwB-zf zVR;kW$ZGcKM3wxoy|eYRTm`IA@us@qq&JbVoT{*WJ-`IMvQJGSA}1CW3tA; zc5tpMzI7Mu`B+DyiC_?)Lbff_dXHziML=btGmrM+Ty|&a&fe6Jtal$&f*~akv8;Kp z;uBd`ZI{FN=oQ5+ar$y;he!D6@RPfiK*=HAuVs9UUHc6^|5c$7j!e7TgT>C|`h9ix zHtJ3HkWk~9v>!@=j!yeqH|wPjTNXaE6gs=*F}GTPAOr%X>6enKfa@~_MQ~>3ulbr;IC%)Bi3rUMJj9ex{GEAO$6ca@+6Av z%~4SZFUOnNLrVTSh}pMvfEEo9gNW*tVdBu9B}8TzxC=Glj_7BnH#fWT^bz6NX^@sg zY7jwbGaD)r$?c7wI@`j1iefD4jEJ(!y|I?~6W-!QD;{a5PA}Rw=@=IuSw?2M?xPe+ z#xfWHfmAHItHhEEr^tW!NxNUNcZ2!hu*#Xx!c_c2SHg}0{6pO&UU6gTg;w-h#@r2& zfj2EJyG>-$5f93%+%kZH79j?xntaYgDV8j4yoBSfXk{?Ddv@qE?y8)k6M=aVQWPBG zG;riz!k(aSNNvcE9?e`S3T^l9WAB_uVwWdl`97~Q5$h@e51t77#bLwc$R}2!`-yGj zhwE3~kU}G2;ZH*Wc&YSYC9Dhg$m^jB8{Y+8az-6*Y(;GRq<5c&bxrdoDG`_BL#7xw z#(`g{XLCJ#X>g<@mMJaoHpFvt+2=2|c;{w+%Km~lI=n)|2GxBoC{@NeAH;P+m`7W` z9I_CSxFLY%Oib)$iGb`6Au1f!&(xBUa*SSIu2XcVdR_Gan(8c^zpQuH&{B(iC;~Jg z=`*p0qMFyYs#jv70%RKxUBd+jyt!uZ8%kGLJ& zloBUX7YZY^86i=MQQIf#w%)!kP_Gov%+g}E;!k&4fX%;k zyS~7L^$#x`DnHj8bh7!W#@M75jZ(bJ4zR8Vs)W3)-tNM7XF zts(-@{DqmJs5IsPA72!>f^Bg7t^4_Bdf|f=`NNs#u}MS!C68GoQ~k7r3in*{?cU!X z9DHaQm4#0#88uQ$0T0wJtG?m4{-;nvM`oUtCIyN5aZT1Cej{SMFRw1um6qL0s(o3|3y6 zifAigvVKHF$zTSFj8%W>whPIrBf`svD~s85z%>F+$&(+Qhevx7&`h{=u%fTN``|+G zW|A4XriMY2(9sy0iO;WNC{i;P$iOuT^lHAU-USfl5A?sD6=mM!2n-kKMMO_=L|rkT3b1qct2?ZLY;9pDn)`TDr^)eD^LZVa*zEa7(ka_Y}F- zd{ITsO}yN5caxN_-vsTonQjbwPo~?d$%2wvj1H{BdAp z?Ku=`*4`3*t?oe4nV~YE@Cw6CK(jw|xdIf75jHLtcmB1}^BA3l)Z5))u#u1PWum%@ zrp*EtFRwwv&gEulWZDitqyDxFMos;v`Lp)phHizzNznuy>ErC0Jvs(56gMlYJQlr{ zV$7E$yW3;=6XQ7)NmPz>UHdPhH(6Ww;PGqI?n4!+`Mh%hRZ-OetkE%>cTg`#IOwRI z>7iT8=jhlfpd|ksUrv68*=lC6`9)v8Mna5C)1r=OHr^{k&s6bqC7O1B5`M8=73dC& z)UVp6?sd8bsVm91L=&5sxWQnkA0=r*jFp}pOMcryIjE<9GRsQOtx!Y83Ig;@O*ydq zXd@^;a}H`Jnk+Bbf29uQXZm!;5{uu|>ngwdAT%Ls8bEXXaXt793gPSy&v(Oavbv^m zDta>VsUO??-9@|hS9R0@BRgRtSH-G2nm93f58#)vfQTex*{rkadC zUa!D^V&T$e7FL)EMIR2fQCr@sqXLF@e>g4~%kL)G5)ct%qfw&J#<-{GzLT*$#BfT& zPFbD~&rQ{=pM6;oUSH+Mx;of)R20Q_CJN?4>t`qF=^=GVb|Pg>7&v|$E!cQz`C z7Cnm{%C!{VNE~+H+|}@Kb+Bl*w&Vp}3&y4&r)HatF}+55TO%KKZ0r0Fe_R9X{2jXI zJk*>1eiwsD5*io98D{z!h4-PB5G#eo#v2v%*O~|<7?cV-5ID+F0WS~9P*;VmCl@uE zNcc#i7gPF{^ev0UkfcR>R2s5KcAis9DFq7H@fC-?0wQ$4AV2Aq3eO9(fjxA->-wIp ziZrs7`VYtOx_7HG8dp?O)TSz5KNtO)k{Er+Vrxx3S& zDGu#j;S+9JKYc4IKey*?4p1lYH8TrWM-ws}GHH%!SPdc8GFv~qOZMPlN2f%osVG5v zZK%(uNeJKI#$o`=oc^t<0b}>+{3pG%w3v*!L*Yqn#kOju?aA$8UTzlEZ7!fg3S(ZJ zBZbN+d4()le7xo+hJb{GhOaLI@xW}BA1i{bnu7gl`K<(or0Bo@m-qeWhIVEfS5KP0%wV11~oz8$U$5(v!wt3_HwZ8bul28yc{dBG!iY3&ty@vOCR-#uvUf_p+x@hlYcG1S3+5KL}7b0oG z@zTQ!Bq`WCtX7d9+lj2~zvKJa@JhNGirf{#D_-Egr_BGZKw->|1M@p?SYJ&!Ij9&t%hp7k*Q)u5^Ev$4%SaXibHAKm3`6S3A-$bdA8t^slCWNMe|y z+9)XpI7pK)I0Ih^n^~C}(MmGX_l~xA+fe;!K^xZ?p2&HDNpnX<%&lBye9ZxOGSXlD zmV->1&5j#&UYUL~lV~ujwF|CR{R2$|GD@)%4<`|NVMVK4{na2V?qR?O zYwOZ^Q!t|x=x|7!vEunz~K92q}t>;T7|=# zYuo{CauV6U6LY`e9Mw8?kDDkOsylmCxX_WtxE-S9%C-C5i?-U>RoDk&>0nF1lilCG zVng*edTE@<1?$wO#+8+yJCXbIAy%S7%n$ELV)@%cU;Vko{-ar=W*0b#(|b2o-W3=k zxX36eWV_bIbWwfZA@n|OQ&h1jkPTw0cv;(3?O46Fz#zTrkz$;p_57D*p|hWfb%9J| zTHPj)7hNf0QWPMXf zrt2B{#SP2a|BQX?0u;BSZncm>;NxaG6xLBrGz7w-H?`dwZmMm`?3RrnD}uF$3I7$7 zx1E^m)rxE*l%wb8OzxKaaDfx4;x#9l9lf#d%Iv#><@6Rh8TZ*_?$^)%7KB7y3Lk2Q ziW8@!ju3uTA1%r{|E!Rk0dt%-GkG~{j3IswWgKd|Dghc1dwnX`rHAW-*!c{W{DbO~>efULMD%n)$q2|Rq!C2DpbeH2$IjWgb~je;Oy)tR#ZgEAH7=>9js2nPZ^>thFN=s2lVrmtLw3Za(Z6ScfOh;UkPq`OsOZZ(hzTzGcHPBMLi9EoV?%riU*MC|;k`U9U ztmd7XLq!KPL7I?9366%5A$EIiPJPrNZ={mJC4o5-&#rOs{Q0l;n4Cqc@*kG5FbF+# zAg)WMVv}WD*7w4m7k;E2px7#Cb!Bdad#JI$>K~aHOggFfS06L3vV@h?t=-+frY3Dh zHmkK?uk4RrRgrxhxWKuxnG@yBUyiDAg1TKXjj5;I{L1=Lcb=T0*PQaM3l_C=Vkc;0 z&N9{A1YQWo0+w~CYHEZga(j`p>~;3vMq)>NEeuvUfgyMx+g+MSv_Q;15N+2a$3naWHL@64-BYt!!`zbSAgB z4by|k822RK58Nd#{-McA@S?Su>CF67Zlqgyu+uupEz#v7Ew9IbihFv7xfsiy56^&* z4X(Y+GNM&hqa0<{A65SzbGox}4cbm&8Q%BR;nZW-h?ggZum~l#_6t-=_V#T`zDq!# zf!eXV9ULpDsro*mWPK3r&DeKkflnZl7gqz{P&&OwS64L}AbER8X1f5XMmEJz&Uc#k zY)54`+cMN-o+ngyl9TA1v|~Z6JIg{>3v5Mc;7OAz8jrJ<6RcxPv*j0+7b5jGdz8v# zsRX6Iyu3W(8cIW0XKBl5FaUAWs>uN;7HhRptJd+z5Q>8!cOMQm4I&&=(d>Z|4 z7F@2NHhPM*Z>&2zaU10*>K-Z|E#71xGS6(sbo;Oc9Arc$H4IXH&0bnswyVj&I5{C| zfXlpXH%ReCqN^^6o;_7>+S8Ab_i%43qA680kY+o85?;8d)|R2d?znPA&=rkjpvev^ zKT9%rlf^#M;yC{WNqfb%d5MBFLXtnT9~rguBl>twesYB8Jk(f&H`xl_>HYJ|m6clT zf%DGx#Fq;k${@T>!PFY(F-MaxNs`K4eC4)VgHqTBrkO3uFhKGk8b^c%M zu?mg;Qjbwl>*f(H&+oKF@bPek$>Gd5ne%%XCg%Q0cRaBfY3nPc2k%;9C;4b&3IVy4 zKs%kT9e*EcY9%$*5M^pNC}dB;a^w=x1Wo+_^)h zr?xc$`psh|@B8#+e9F^PO3pV;-&8he3dOumD&b$P58}@wI~DXXUM=3f)CJ#4=RP_T zWt45U?51SafQtP$&e(X}n?pj780Tl$E?JnO} z$E+oLy|M2H?EMwhk*u1C{|sE!XE-g7EVZ3yp>KR;ouAlq!@VyH=({hAa;dPh!k1pm-Eevb9>7vpwQWqUkoWddmI0xtOrz|H zu8cNjyn{rOOx5}8i+0;E-bZUp>k)oO{?y4{XsXnUz=vo!uu)Mf%eWA6*@#*$Fg3_h z_{YPE4z6?2BYi__`|Wr^5yJpn73T~%MfcTSn$x1XSX-^k(f#4XNBG!r#4Ntq z`Ax0xFMhJrk#(LVA*vFaxq+YgJB&WA6#+(qrFsNMAM76QF6Y$hOSmj|O9{|vZQ%Q; z~q_Z)GhZ;-^pn&TnP`zNZgdmaIt!~SGm`zW~ zLA*UZF;HV?oO7rcZKWW8+POkduPZmvgc4!N$jL2Nlc*1z|GvzRSQ^cAi2mCAxfn@b z?T(_Yr8z)kz-qMIRuy4G*b(-ktUOJiJgz2^k29nv%;(<>HoM1gPFkqS(^aj*(ag6f zx6&zrl?i@k!fcz>)U&1%`_zSmY%5)GD4C!(I<%wCQBg8D)M>Q(sl%6yow>QTa84?kr8z z1%80Eq;QgLbU|muTZFZO8i_f2)-!x>j);(Vjq}w1LChoZOG!&tbZ=72e90i08M2j~ zkyb{#-ccS%_K%lR?q3ou-wN~@;Hw0Vp&?MlU=8Z|kJa;tC6 z7ZxHB<-6vuhK)q=Me_aA33Wlb0q5R+S8i4V74lfBs(7S7{sV%IrG1b^WaoY0D{x!`I zA2Z_zZ!VLrh$sB?cumGQwvzUsR%&O4+E1Z18(s!0F!Nr+c%+{>AFyyTUrVT%<5v~n-kx1 zc_sD2;yI%;(pCz!I5>Aw@Q*iLIKsw9HS*tTYfsS$SRz3UOnoJJd14N}S$F}kslFL6YN-Mw@vAlk|9vYlaJGvobFC+^J7p{+iA0&5 zJHkheM1r2aD=4bF?Gio*6#!~D6_;)!mm1^12FKgON{W9y*s9{9MO~OkO-W{|CdBf= z1~QyU@%6hBn4JS(;BL1)yqV1^FMbIcdj0eHDw)gi*h=GQDe&fD+|;067i z#4vJl(V<$S$U*^r#3I(&hdO80F`idA3bx=`PY&g^V!4*~K-$BR0h4D77IamDZ-4bj z$F*{e6i&4D2oV;^zn;#a>veH zyFbZGnKai|DEIxHCfxiVEcs_GLa2?TFLo~;|BwfFY+=(CnMj4;7H%t_hzj9hbVqVc zPXN4#iQrmt?{Qpv5ZiZIP)bUnBGV)~8{n*AyK8k~y}U?b*hZW@Tj*Ho$>lAT%>h5V zZFz+Hrlzof-ZGcj6$7utD)yjW@TGl6fD%bKrs{A}p{i1E#2B*Ck}b*VOrN_Ry;N5} z+0Z)k8^MBiT^MLo^Pab~&lWtqJpO4#M|P&;7vg?fc`t0d+z94#eU`)siHP0k61li> z%!m{84+bmkN`~jiOS{fH&X2nXu1IOOjnsb&!1jwqHGbm16s=#YGS@UIaM=ofV zG{&MWE>l~w&ly5AvRkoFO1M8Aqd>(}0o=S27oFH)1`-Wb4q4MO1>v&=8 ze9Tf#`+Zhv*8_Z{V`b*M&*Bwq0iY+Q6QRbE&!hDew9RZZv#A^Y5!%&Hvr~3IbzD(s z7Nqj?f;rZ|$m$%M#G%cGvU3^!6DeVa5m!|O&{?mSL(T{oBm)gR!AJ^$whQ&L+ z6ICW}#N`@%y=8}jID{uF{%FOKNc68xI2wvk`#?yuy0=|3K+LH;w84Q&Dz`fv|kDO1zQvCMGX@Zuu1Z-WyV3zW$O`2Zm;-B|T0>%n71yKpDZw2Z=}RXZeP9E;F#7gQcIyYO)F6s;WDygJJEcNr}rBg_Y6D@#e|1A0el%g#rcx%Pmjqu(rcE7IP< z+L4TH^G@f8WxV;13mekgx4pBTcZRW$k|McCq|We|RefMpKDgE7qJENM_H(L}6H4*k zpI)4%yY4uzj#v&&OP{1ChH`_OJ4BlJ;K{GbG>u2Tf!8u;Lxyp)WUZ};F5oIR<{ zDM<^guhS5h$Wn>58*N_`M#?$eepqT^E28Ya`p|6j5V^`TKT2lExnArR60g9*L}b@; zamxJgQb(x3x#!^nM;h7g6$0}AH%3;&BT}*(_hxs)vJ3-hvURe}q#17N*FD3mr(qi` z$^>}Nj(N9!W-J3)WfhMeBFD(}JidmG3yq$D2Pn{6W{6saMSLcT2QWdCuIAI(Tke@Q zR+cY4HLL&obm9EDFF-3!b2o2DB$dV)^;p8SD>x%Fb!z*qR?MS7U!6_K?cs9f4AJ7Fp+E-oqg$nZ+)SCE*#FTvrhPY zfzs_`!(e0J8=A(h&(03FzW0<~AF?`Ow)cFF@J*$Jx6dsZ2i?M}p%*{(vRo*wH6DIa z-sT6YeJAQ8b31d9dS14D%X^O`ZPu@J`rrtrT8#B| zHr3d;oV92p;dbk;bo?{`3@#K*FL8%u;=%?6Kdbmu?HWLYbD}IId`GBSiqMRX7GM zDF$4fUL+T>u0OeX$k#{s+T^hyxW9P}mYV0f%lmqMLF*%UnIxW*dZ>J|{txI94f%#? ze}8{q+oO12tjGY7B#{gpHoQq6hg;s#k}gBn%UbaNqk`~rLjx`X>u&NUALCqW-D(=+^rzDI*u84%tYQ!x9R0^ks#=uCh zWFTJk&{p{k-v8A>R5Yzu5^V-yBh8;zkGm5N5zB6>&73gIhCHp3#qw_QcHg!I`6#i} ztd-#lKR>wLfMzu%o(l3$PGloRn|xXK59bjFO&w(!WZ=GSId9n7`5gXqFCQS=%F5IK z6q6Ec<=)Z6h44u05JjmEWvpG?@gH(jSXsV@S&;=1al!lfuz1)BsaD9W zAsmQ0QWdT4eh=-36w^X{?H6SkITf*1WTrksBK%DxH9~w?lgyftM-0DN@v{CsPtrhm zE4i=+=;G9s5PLs@0lu3#4Br;JA!nC*H|FOB)b_7JFBK`s(tPcVNx)VoAmJcuRBh>w zI2|R?qHb><8Y@dODCdmOe}vwY{av}{nEG5ZKq3;6e_Y3;^o0*A0vG~O}t76shbg=hm&6yb)F!eeKq1;a97X#7+dsaCix59M#h^Sx+urLY@gur7ZH zlJzqnW+4hYbA_Wv$B4xP@fo>|bsj`L(dv#n1m@K%q{$U7Cw26A6s%dhRB+#QY$$Pw;B8 zU-tNY1d!{oVs*QdsL$1v&07tli%X`bN7gmfkwZfOALE6yt)wE}csTw&6OcoLq0z@| zQ2pO0G!jn%)Q653K7?{~k%-ZR7KSJ*C4?wT7b$}#OH}0WU&)(xLODJq#CO^_yjSpM zJUP4}Ww}i?CIbTaj=RZY?Zbm~_Ib52E+H#mbRz-nzUY26=rUS;?+-Z>*XyJ5hL+D) z@P|u_6}3)Gv3;X+M1B!g^Z8#e%Xikkf!_xh-{oqgAh`toExIY{d8g@YPiv*WX4;vy z0PER2e)4$v@=ZTw&mGS5{wTHAB@TWQ(NY6c_ECOyW53TiW+rxHQxQSg|E!LwAA)N? z<^RWeNz4WVz8u0PktQMdD#`iBqa30zY2-gP1u%@*th9H)fkzCFbLkH|m%IO5r|QM72-fZ|F?ZR)L!PzQf>eLEGmAosoVGchF6X@u7Tc0PI|onA z?qsi9C#afSwmi%Tb{U^cSTtGf2LHV}+?s!WBo}-8oc87Zt|{S>EAWhOru&Ms=K8?? zx$scpzSPabyBIf<4MC|H`u{ZZKlcnVTM24sCMEy1ga$+x_1?Y><-N%UP{aIelc( zc%jZ}5-4Q6!r_><3Rqv))FqxoO3c9Hq!)9 z0Pk9|aok>dVr+EGMdBqOX0JIGbuGfRDy*f-UZ8GV6Q7*%LwPyr{T(5zAmie2IRTA} z{c}AEG^%`yDF6>rg!k$nK|C3S;14e1KL^Nh^gLK~Ge~82r=32VsdE}Dku!P8#0w^z)M{=n$PZ<+Y`)b}`qli74WV56@D6QK93I)aXP^<$rSt7mkJP z;Wx#S7XZMej3pn^;Lz%aMUj#_SUGD_I;M|}#CalO4-h+n$hF7DaXfDCMZkBladN4% z%B-}4HQ5so-p0+!H!7M$8znno;T36mWLl;6kJ2WTTxyk>c(vzU;=R#W-7kEqf5c07T*;mIk_nVpp7lWi4mg-F3P*ZUl-QFNJuZ>-7T%RkU;s zt&v1UiMVPAVi8g}p1`}@Z<`S!*dUtNW9+F*OL&)YR;)iod4Ks#q?N0hmXFJ||FFK(Qww|VE5JoWU8ZeIM&<8BZ)CqlZm3~&$sJt1Y-K(x9hv2;GEa-w&_H# z~4Er)MhZlF-}ig?12iUy5k95^jisfA?p&~wm?&om?H_- zCt2>Hg*&wuX~WZFavP>gxobibA`OCT?CVc|C2Suwa>;2U^OFVCZBJV)ik)VQ8t+dA zxZ=;U)((&r5b+or+^%iOgG9J9E$NYRFDo8TiR{Yphs9&pG@{Vo6zzyV-wH`H*htmN z(U)WmGz8si$E*e>!0o?!y3eSme2KKUJ##zf#5E*vdm!j}H;$zW&Hpqr+MRtZpl_;V z2ogSWPgyS{d7rjvtOEhxfPfC7pcp}85SK+-a&{N9+Un&16}g!TX#17kM|@buAnpmidZ?hW_JFvz%x4!ciTv@UalGetpQR1Uq{?~%jR;@&aNiq#ST>3rM>M2FWx z%fsu8${jzBt+u~qGPy6F{QB^7p`o$l`|y;8dj_ZS^-Kpe<_)pq!#gW!Zc^>l@)<8hpF`6R#P{wxo+D3t5sjf8>s4YRLaP1HC=}7107PC?=h0CtjgpyLB ze^PXfCy1OjZ&JNWl-M7Vc1z6Uf75}Vs1%zzxGeBHD63RcF@p<&I18Z_82y42ofF&~ zZl^f9WELFYG}YWt8-zPT%4j^b8I+*RoBit+~c1mFAGA-Ka)B2EXT_UR&~d zlU#2ur)L_|2i`vj{t~7f{OF%$8*uyQeCFex(7p0blePD2sbG(^8@h*YUk;=R=tJfS z1fUNoj_;GGH>S@Rj(D}Vk1+UudPO}}esbBC(A!)J!UKYtjv^(@rv=VqS!CRS8JMV7 z_@4uE3^C77*66wCqnQSi2|wL$;3rLGoDQMxo(7L&{cL&G5n1Wyz?cbIVi#moA~>F6 zn#H9mYr2u5>1l`rST?Q$cwH$sZ!2eYbE&74%>Y~9gHB(}rqkG)`}3dDtca*WL+I}s z3M}uNaMa#Tq!iqvpAq^m`f=Xl6w+6py*x|runBvjM0g1l^?q1m=xyFO#{W+kRcO(b z9RZDC#(`0+u5Q@w?As0eHpPA{;et&EH+(x{L#DtEL0Rd;SuGDsC;s?aHI*2`oE| zKeQsm-|h@V0f78?Kijze_E-1c>3$^vWmj-J|k5yY~KIrHJ&r zC=w~m*g^+)iREH!r)&wlYMIfveN=!hZP1s}rQRs_3_6G07{<+PfJS2?>v~S^=RgXqxvHrD29ad)3h+ zr|oRo>J1{#~vOO6JFoztrY|ZufQ=~en{qzdj1>2M&?vsIYa8Z zw8n;|lF5C?$7A@h1Xt4kH7YnT&X>kkZ$5t3Zx91EPN5#Q2)d@*r6fJK^(K(U2BMIG zt$?KWDN;RV99^AVlokRP)Mh>ypbnE*(uc4no|`7s1G_+2r1su=!WSpgr$NBFQ1bliaC zWTmlpuV;)esBkTFK&V)PHc8|_@ag1u0s;%km0u7#N^Zbys%0mp(4Wa9o1L_Pa&DQSJbhXT5i$n94s5&LKz0 zJ_oT{Au2(o!MX7n5Vb5W3k3;0HZ2?LO{K}7M!FEgM5CaxTyo%&^$0?zB7&Qbf>{j_^DkGIEd&L(Phj3$JkrIMe(%(qasL4cgND*UDBPRNJ%3|w{&;slF}du zN`rKFOG_;wA+Ri+clf{ke((M6J-;9BIx{Y zuxurSyzIBz>DD(;aRLo?F5o2~$bc!1fFI{Q!`dAwa@M39vX2O3ux^=-Yotmaw3Y}? zu9XoR!b?w2PT%SAoIz(J+$Tbn55S9Jksjrur56;FaT!2=fp1|>7twVgj993*)fYQ6 z%_9XuEx%6HA+X#ea6q1RyGIq9!8T2TDMwQN2|jd=f@wQKrHCcE@#WQYQMFa2M-i#^ zF(zkRV;76KmC(3Ph_G_=E1FZWjj2bJrjb=IwAZFpKRNMrwF@JbbZ$enTHXukK}W&F zd!N4xC@9*C+L4PPNMkFEn<-jgqX5Nx^8&|p`*EW5-eoIU*TmnXQA|hgVE^vuTE}aG z_@s;)%jt?bpr~U`ElhXyenb{k+_JbLqNHRfwV`W9X=UJnw8gj?Zk@$4V)}pJyC#Mb zIRNFwy!2BRq2llC-MPou>?wtcf2LU6M4*htO-)V5iB#jA^G)%P!)cYjGm_GBjF~IZ z?T24iW-a*S1Sd{Kf-<$NGvfM+X|YO*f*&eA+ne0mDG-nAvwpks0slBa(-M8h&8wy3Nc-cU_k$NM9& zPvqLk8hH4Tv-26G4NNS7-%r-;Q)EG(PfDvIR|t|N`=qbGSNl;{*Bn~1F*kB1?_AC} zF+iGAw5LS68e~9Ic#p0&j&lyuHZW2%%EW}fGyBC`ADf_O)a$BG+vArY0{&uwrs%?B z)moa^WlgcYecc<%VEyF}RXb<+qynXMGKQu)Hy!@=R>=+LDz7x`S~!dr+q6bW8O03~ z$M~I;rseZ~Br8iF&lvwfiA8l-tb55bhfl=OyW8tK4d6r!lB}B(cnSyM)ac#B&7GGf z{ocaX2PD1mS+kYALA^wjD1f|zP)2j|El6n}JcH|e64u+#f&E@G-8`-a9j}zvi1HUP z#`Av2$Qcn^6caNF%E5{e`6VF_~uqK*AE?`ucsp#QPP?Rwgo{;yq_tNC4edt*Igxg*}?u zr4!^662|*9$)8blMc$s$DSL_b@3KB+bSWr|)AZU|_9Iz5iCLmYN#4=7#-v|!dJy0| zd{F7|VS`Gp*r)L`eqYlr@)e)w5{&1bCmHIdS!H*j{bIVbAYMC@mKn#TM9 zwCxa)1Gjy2v^otN@u&XM2$^SGRxQ(-W^SVq&HI*_`HpU1|G4_zpTOZ}o(epJX2=#~42>HSt%tzldsC_SL z33wrda>og=!+yleu~R*ZM|ws$q!*f!zP*9#kae`{V~CV!ji-fHhAev({yh1#JA;sB*2I6<)^IUk50UXhxXiAE4E>O)UEj?8E_K{J07vuY_hM2LJGZ&P7)$)D6 z{#_57!3T=X2G8Mib{dh@KK*VCF>*H7mui6`ag{n8$L3AvG-JWkz*}heEBc9jSwZ8R zWA$Sy&#H=O(%6_gd%Ssgj+o)mPmjDar{hTC@|xwYc`6j+aJl*=@GLCr(&zfwOPc#| z)s6Nq&Sr9-kJL5CW``7!a&Hz^Pb_sbidG;&KHVQOBIv+l*0$083K^m_HFY8I2HER<9>E%BZ8)F$-DCY?uTM zD1VZX@bc%kYXib0Wb|3hTBxkiua+!QF6xMVG9k~*d_C&msSB5nSF32PHGg=0 zac2b0#Igmy#lBJJqeTgoYx?HFqR)Ot6)E}XX0&!fQ*F1b>^p%F2bpxdL;5J;nr0%e zK*P&7xY))~`>2IBQ;VSuZ%4M=7WlNnbJ~Bv7DW;85&i~dU?l~|{d6?feR#X2khH%* z4;-VJ^^a8*xp1nr!WnQP`qR<1n(pCL@0$`K^S&~=aJhX1jpO+92XjQLFL`LUjj-T1 zew__%;d^9Gj@d>S9Zg^}!XGEfGSV3Jw@;V0_r}%6j*bhfT=3QS56EH6W7494owBG= zm?R-n9D7p273^8GBG9@ zD2j|E=02u4>YwN%*$Gi&UH<%5^K)z^e!7BY#` z4=%5Im(0Zy>L-S^;1OH}3GO`_g!IICKY_ z=KpQ-oIIxgVwrzRgTG@A@Saru;vsYh0S~C_Bg1XOpMok|fMqg>?2GFq0zd`Q0t;_z zqYn(9=&lET?A^A8FE{L3fy z{2Mr*81}!uT4n^A{56qY&lwb(3$g;=g)S#D#CH?Wo3vow3w1cnSAG)tjtDf=WCQA) zcwM|hvG?;tu4H5bU3<;lEY-&aPK*&Cb3~$llX(Al_TQ2DNYr3 zY`i)U1Vn-F2P8gk4o)(hq?7m+^V|L$1M5Sxey~N;kI0HQncz);F8&)?0I2~*!v4q6 zzYxmnayc?vEG^#`{t@s2DEx$NXyiaR9`}WsuIvwH01z|2kZ54#kZv1PJ`` z9DeptuwPPA!Swe&fBPj+oBr|arCbR){eZV%ix=812_mvGATyVSAHG&QHq1z&^0>m# z`AFj?4+I61#j;mxa}4Y60oY5sc`wwMVX{eM03g=(j~cPl4gl!vKVSxg2>+8F%`@QG zDT_yRog6lik<@wD(-T6_IXQ8YBEsmKzSV{$r!RvY$pA~a*>%gUA@4&cN+2kRcIjLkmU z5LkW6XgCrtx!3%!Y4wc$7w7!Ptp7L7{qv{^Vn3VHVx`}y6nECRW2=3_w-9rp_lTCS zf`))Hoerx5V_jSTB^KNwdNrEF|5Ld97nuM30@HtE zqMjHb0^`x1`0CDP$EXTJTp2nL;8Ott(j?42*W|X}>wAyU^uWI6sp>3^VT^P?mY0BU1~3Ba?LMkcWFd)_lOfTC%| z<`?fz0ksi_otp6cY&O?)1Py5~7vMfcVUE0=caN-=?*V(`gXqvs+rS=Crm3ZK2-+>b zE{LGY+4;=8gu>hZ(u4mJ+!23STmQuRe^3Zd&okIIY@LCk z)9qT2FjGEKVGtet3C`GZ`W&^kJ=DVVrA}nw&I{^X|w8w8w>@~Xb+^+G5!I2HJ)nZH)L|5l4wp7;tj}ln4Y1vCR;PfGY#wx)g!!g^<86<||Wr zsBP7?Zu6Q^XtjC??=!kGM1uP+MP>BDvJzRZ4hTIa6Hdt1u10&dJPUQaZ>}rMO7-f@ z;fSw|MmwMJaP*BFTV1~uAsi50x$O*2vFgW77FzDzxS=MfG@4j?Wgj{1eU2x;-y?c; zh+E03NG-b9b(Ah}y5BDRaJb0Ko-5iJdVYd|DC@>3ItZ+fc#6;1^# zasoP>I4q2ClgiLozJd2c+M+s%2iYgtCa|SbF(*WHL-tX@ix$}`;O8m%1uf|3p1;9<;gU;F0G&agr)cc) z%!w8^WyRB|_A~1C+H+A=WrAewG==%pw!>qk&BY17ApAZi8R=SHii6VjdQ6_47!QlT z!qOsH1wY)fiz*G0aDkAwrfW&+S5kK_gLCveZ&R*FCz?ZaXJ;GQ>SGc4sdS0~E5s`l z5dODMe}uJGsFe(Y@BUmU%j90!iMRQ2iF*WGRrybJMH9Yl{g&d>2uB@83uS!^<*Z=g zjzF|z6u&d%KU|S|z3re$2M<5W8C+hXeJJ>AY`_oF_|`BfL_{ZM?RhLV?c?uH z!KHBm>VQFuL0)|tiV??;_i*gnyQdoUS;_hroLRRB6FxWMHEry7g={eb^I8%XPf08@ z0LLE5V6K7AQBM1441AA4pBDq)fMi^tQoW>1(Ov-1e2SsY*wt-MSEB(fh4kO-Nw4p9K9 z2p&$lUx2I}K+pxsv<*I567tQhonPn#;zN%Yvsy{5rr!j6(H;+;rNdxvv*t&t%)X}h z-&BaUw>%5&xgtl3@B00-jm#c*!soh{bv%}VS66Wz=`s4=%4Nvcl+F@Sn;rGbo0CD` z?Iv2#KoM8)6&5H3z&k_ODF3YqyAxuE(eL`j3U9Qbb%3N4$3Vh zDRx_>o@~$iXZf29Ui;dYNyH3yw5sR=2KjW)0TM~(;Som$Oju&eJFK3jJ|I>0`3Q@F z9TPTs>>ugPZHSd?(tX@Vw7YraYY@%4nU&|AFzZS9;H`B{>EE7=+kEYF+V0A3zb5Cg zMUnfsA{>RYiL8F6?A@MJ~i!ghS*0%Hb619d*)D za1|c4ShL|~B#}6DEM6PbS8-f;->BA{;AfA<>F|dc$65OXtB-i7^xi?hx!w(* zPA@crYv6D9h1_$E7RVH0KlM}j3_atodI9rvH{vo%$0pm`TFBRyN%zmTn&SYmMEo+`Av?6Lhe2+)mFV!mWb0ypm21)MWTv9+amP7u1G|1>>=hM*K-l%Ox%JB5 z!Q9yXg|UHgAhC?7+(W^wUL%OqG8}o1q9DI87`#naFJhgsXpBJ zX~<{5cq5ESC|54#lyNUze0Q+slay_aiBQ_iYc$d))dL#+e13*ebp!LdSd7Fd7;dj97Hw285OmJyY~?tayeaPe(LE|iy}4-sWLd+XvT#6R4*`+Lvo;+D4Yyr% zydWB3mXQCDWMUa$Vu?^1E>!P=`-?apx_E9oc-kCsRoc{W>EP97iP>lTC|X?i{l^Yx zGR!~JAG&3lv>C#)&^tM%9g}H7{|fReThxc_v3tPN{T^o%?GNXd6<>_jySIf22&(AD{!&ddVgd_4 zV9$);}VSiNyR*?tU%N17hGzZ2#zIasKYjV)O{9Oqn&$injsle=Cr+1Ln3Km}`Q%^kfp{Ekxg&mjW##rl zUo2Ub9axo71>{N94@7M8pyp#D{egn41=n`VFVY9?~D< zYMty&nJMo?{cBpt8sFdDe28;JYI-yi|Fo19;Y4r1DA>I|iI9&&MV*s${d0@ni<62@ zEGs1hGA)xK&9F6(O@+RTXLd{f)7@T|_6Q_z(W6t@xEJOCSZ0|?_==4rZ70pQ%Ts%^t7{${mZSNE%bHT3O6*l^_%WF8i9OFM}h^a74#_fD$;O=OMDIHtTu3WAUldE9r-LZrXPAWDd(1 zv|=L41YWr+?M^TVEC$0d?Ja^BR3=4!hckDg4?=RizbKwLMn@NIn&Mb35tC(w(NIc^ zHqlY%Oall8c||CyPS6Ya#jNH(!$WNLss&H&r&f-tz3TN;sAgrzQ@(5EvFBM0gc8UW zsEoN!IoBp1%6gfD=#qX1Mv__y&F+1(Q56;YquP7LVA%AE7X78#`o!g za4Qh7YoneM-6}5q8kw=EC~z9j$;&(R(T8+be$(@6&xRYa((B1{$~d98>)ZjYH6_Xj zgy!PCV3?KH_eGM8b{PF^ACzq&R-M3lKU!CtD$eJ`xheOfH*X}DtQSsV3MH%<9Zb5@LK zTZ{pZDbRd2Tvo)|?l-(;`B$$ETm*-33W?YfW%=wSTs=S)7w{Q*<-~;>jvZFjYvrmyI1$_I322s`y0c z4cw7>gkx=S==G(jl5SGCP_E3Eo!6VqE5q52XyKRj5dFrWbxG1#_|8#uxr~w{>bEJ< zyuy9&)k)ddyhba=si{}TB*??Co1^I(U7aIZOZ%c5jv zm~D!Qgx#nFEIMZ$G8e{W`n(E(ULI+WGK-px#G-#q`e01$lyEySHB+)C3p0<{LCb)o zBF`^v=lD*nni5g69sG~I02~lB%8?ya#25kNo&JbC^-2R3Q!YU*v6L@djV|JkUK2vskV{cZi>IZX`GuaNK`#2LFWWu+wMKhDJ)~u~t)i#uFA)7) z{>V|>oc&q3U*B!|ab&jA;)eAC4_!kKi;;?%`Aykil4Kxp2@Ib%C?Fz=TOCVBhoCIL zIv{HJIX*s^p{$HmgP@!dcB-NpqJppHyd?=1*yF|l}U04_@6Z{%o$Fovy2;+yaVrSCBmm{wwk^_P?Pf}jr78R!^aPl)T0Tb z=IMLB*upA>^_ewF#(66{>8sId! z5@ZpdzG)JavtkA6l`uex!fHUKG|JB{0;5jzicgnn>fA~_oo#|l^+<49OEwQd_05R; z&IPTkO58d0iAjzz8>PgbAP!}+sGwUj;gjx(Ib2zU{ z4zl3%9R8YBSLP2R(3I44s#40&C>7-Co9aPQ?0=qC)a*3jj#%R&s>dWPFGjo_^;T3_ zn1MNvrdrh^aQG+9B=5H*g%HrqAJN{!MIW7s=dhW3yS|mmWPua)0U+9+t z*f~PODy&L|^r6E^2#a9|QyBO|OE*3`omM2VLd` zh*L7(XM@Q2_KXsildRbPrn<1gmK31-cEYP@#m~`= z?_ochj&G5OTP@fkBk*9_A*mnKD1;=1x>#-+k4%fIBV~LUe(=}BitqDwAOjfRJ$jte zs@KLHxhG`myGIuxKnI;bmY_NO!y{sPT+9rMKmg;`Mh#$mws_bXPwC=9_GTv!$8?($ z<=8NIaU>*_$w<&Sz%=^}y8ZIdTL;?2JqB>>z+ka<8LMJ~Z47QRGV|$9c`G@9@ofhz ze5}+t3lY6#@01^pr}2UjA#1D)642_Xhk4Ne?lh2$XE1UEJ7 zgoTD44sEJ3j{0~39J51eRMhZtag8LMhPhPa-cSOSHGLLNGmRR3Iw-9Sn~|AYT~kj3 z=!L!fc=xz7RL-ya)j)A%U9O*=5?)po%)kuXVfcBffLsiSsP%&O?jl@wy?o%R*LZat zJ|6v4Swvn@zfdQG&NySe_j`{@Q2}e@H{St_{&%>%7X05VwyE2%$|bgb|9&@DWiZyz zW~-PMof3^}Ab!n?q-51hBi0_Q+9p<%q6j<*RR-q_qx${H4`awWm^jRoPR(F&=0|&$ zCxTL+9_qC+nZ&0Iv>{|=)5E3a}mgU?a$ z0#{=jC+~X+J+G#0fA1&1U?#8vf0nDHme=x<<(4%Q?FzvOqZ@uvK0=4`Jdtjyj~iE*bKmU8NB}00 zwh|r}iHsUP&;$gH`BM0zkDW+f>oK)E`k}0B-@8~Xr!*gSX?6l>fU^z7UK+fjlWibtZmZj_(Ole+6CvfZoo_g8%KN>hEGxE#b__D$%c5{~{x@N+w z%qDKQbSvLU%d-#vEU1y(w$Z@W)Kuhmt2KdO`9mjrTAS|LSg`}0{W2F-GF52QJ{w@q z{+tQ1mh`*9ktzazluMz5BDf9lMrH~M$bT19cDkUMpb+W>dxp>{RxP*X9OLn(VQaL2+IH-R_b{Z@f!@(P$Fn`e^DeEf;_6lo zVe?7Dyzfs~Yimu(IV!Z+378nT%?q%RU+94fMscz5ED~8Y6VsWfh6`T#q1y26*M87R zi-3aMz6+L@IIX%IP-wNj%OJ!8ml8!Lsg)>}>@S|LVooN41I`FilY0; zscwQul2JN1QQ~|)_&k+rY52l;X=G?(t*bGQXoK=2DLpfwWDKaOK0mKfkVybBUspmFdCYp;}3c#2QHi*(!?E3OemK^3Bd*Ej<;eS*Tt~ zXo(6czJ8k1^1uTMDW+(b;iYd@huDjVySNMEh*UIh+Kijw}kL(5IHu zl0<)V2A+Pn$!B=m{unf8ivhhCIocWL5F1N`S@@m%vRKFTegK?CZwGRQF7@))QW}7q zDUcDVcvA;04_1dwJCY(PwXJ+Bc#*2pkZVoT|K(R>O_Czr8nmQYJ{fR@Y63^3 zq7i{s1{-ck<{%N(=TTgIMj%vl>D=nb)^Qr|VD6^QL5#NId&4zO%lw>M{cram(gT$h zwamMWxP@PJi(xuCHszeReMl;2-IGBgy=)JEog-xQHS0j)#I`PuYy%0?C#TwfkJyM& z_Bvc&3zVcN{N~Tb+#lao0xxg@t5Sn2_e$bspp{w;kA8uUow=K_$dp2?IkPnL)KD6! zgNc`ISRU^{bTd)n?-!P+rkS!XF0Z4mi4~qloraaSI@L(-Fg=IfNL?Hmb)|P)`o@(W zlz*30vG^b}t}>>ji_=#y#J|GvZP5CSMz5A<#PsQx{YwNW>D^^>o+ zd4Ss%EmHs{Yc>U&epgXN@mxGaPlq3}&?@TwgdWGu!!q~X9I)~dN|CK4X>w@E9n9`? zv-H;VdqEGe!`v&DwCL&4MZ z6=;ssJ5N0j6?haV>|(-SS>P<|W|^r)qsC0kpgk``q^7|b1yIhgN~*HXi$X5$NIa;k zh_}Rvavgm^>k<}a^Pk4IPV2<&=VK11sSKE$;9=-SPB7?dQKyQ;BC?G2_q*o!m+Fsn zYyMj({mKDBfKd6&03M5WYpql&e_N0}niMIkSX5JF$)12&QO3%|DW)eaNvA!pA}Y|+ zf8b!@hE`2IbQFIlEWI4(xaNO{V=Q{Z$f1IC7u`)PoxwGo%SPr!@RxK+E%Vkro#G zZSX10B;!>S6lhh1rR;Fm@e5!@%MS53O!0~gfk(VFY*JxlVvdPg*fKfFg-&dTmZgvuXi2ej0!m^7p}t zrTH{I5E*o7sQI1rVnYkqOLOn`;l0Fm8BCLejP(Wh*9tEf>X=ATN5KGXfFmkE`3aG8(#QUJI&wnSF z|NX(L$xAg~8}kQpPj+>eLzcm8LZj^MOa8sKR8YSvUyF@m8_ZA<5HN@XIE1OPQBkbe znwr>f*qYc_*qQ{9fEzX_+GkE{*5g}*DKXZyh{`<2tU~6Q89a*nL<;{u#{R(Ev=bCB zUd$zEnygHN&FQ9mpgG}3+Mfy%O%$}+q919sdpG*myN^Boef2{$gzWg`q(oMA)Wfv# zlqE!S8u|S4<(#U542g42bMu6r59?0sZ)_QL#yfhGIdUyq=>LJM|L@Ny51Lo$xW;YP zLJihCQ(8V6IlMAT7bpHmV>c33uHMI$Tfmh}r&p!%8n0VPp{Ok!nrMr^nZsCG?6bRoxy7>94VuGPQ#?C`kbvE4t%N}5OP^?Bi4_0g zC(OrfiAP`iJ$1(&gFx2-DziT??|wt!w^;aebcU}Lhh0o(kh5CMvicvrrzley4`ro5QLAH-dZjFzW*iLXH zIG(Ij7>Tj}lU`$a?qW|otXn2D^Oap(?F8j^w+g9k+Rx;VAI+Mo!=)$4-ZwmFu2;&1 zj~9D|qQS-A+w$5aTf`@UhliSxTb|3fNb75tLFyNjn*m*?`|1aA|C6a-Mf40zDHZ<$ zoI(EomL&i4C=FFX0gfVHt;M`94ad=LevbtK6S$wNYOi_J1$eRYi;)94_nTFYr_+ zS0!Zp?_T7JyXSr>&|;ZzJ{hvIJ}^>#MILC5M+*iUDt2+KwTmA<|!(O~5a*Xc>H_9cYaRzn0H2mFU_-+(f* zv*{}%{|`DF#{YLv{p~yIbKrAIg<<|zUCvNxN59#?NWlA${&CGP(s9S%gG@?%8s)E1 z$$aiKNksUk)+ma-t`V1PXGGnuyt*@yRdT%ocZ~(5@VOMGg*YYCflg)6Csk}~QDQYb zn_|Nh|I+v4f)j2BY{D-OGsaxd!#(YPKr2jU~|KwiGnH6MBA?S%Ut zCVarnfDz2-d@}@@-3+*U1a&W5WD$Gb9jrW_XrHw@kVEwC*Y_kIF>X7E%_@b4w$pPh zgfUNb2PZ%dDp>hIi1MT0l`KV*gG74RQW-uw^lr!{1*M6kVA0)u^$z>$!cY8u4+wNj zj^L**&HyiUbnjPj`RxV&xEj5`y>`Cly=QZZ;$aYSuZksI^I&^fj_kEHmh=KULvYxm z{hBHraepvRs6SBq^~J-|nCpwXlz?$Jfp32n3`Nh%C%Bq@iHQNTQ&AwU#@<3?$~vg3 z3$FdXhY~vJVZ&-%o0#9lcev^lh}3cQBHK`%7HxP`_ja+_ETHOK;5&4ku<7!mn%l)r zfNRHdZ)j%L^zK*U8zOU}U*95aF&^kHSI#+qQLgk{jgzzbL#wYw(=!*R^9mgI^w^8P zuqJq&a(mfsF${`&;a_t>&?{Gm8*q)+dGgw$7y0ZovbTF{t%hnO{c?**t2e|bg@*_! z{1R?q99ns?n$4a`rN*tj{g?OLzclqYz0Q7!x_{3#KxJ%Kuyf_h(!ER^rEiArsQNr~ zRYMmYaYvRnIp35JbWZO7axKbQbQ;lpILlhw&Gok2vx`a26CLX<#+l+hraGMos)}7{ zIMzGzJd5@@KSgY*O_1=9!0obGt#x|=&X7#1N)&(Baf#<4u`*m!Mctz7Q1A#|ZiXFU zr17PK{q{iVE_@_Sul#Zc)w!of!+e)Lw{VR&vM>#L!ab_DO0on_uaS<|Yd7T58v9JD zRM-OYfByK86x3hgNJ66`Nay-!sa@Q1akLFfSP$g*Ny!{YgxP%HE4~|f0Mi$ zuW2V!-YTVgH~-z~%GL)8jc(nY{}h2()49TUd+~KUdhJ%YF@1t_i6q=6@*V5&;1}{b zy5$8w7~;cWVE96Ze^k{CQ7tY97oR5m^x#^K0_E|oyvWto{hHaM;qLE`YB*{+yMrzh zJT=Gn=)KPgR60&*{mu%CX0RD^ENa{$amGG`37`cDMg-(DH6;M6Yjmc z6-V4ZSnz=v`CV;OIu4M(?ib-nff0H_T?2`Z(WQ>n9yW4&R&EE=n_-(aOVcg};=UPZ z=lWNj{Za*RFz&P1%jDa;!j=0r^bzmy-^)!oI}<*sKQ63@Y#5#QdYvq5CVMCH-Wsxy z^4sr}tKTQw=21ZIZRHmlylF3eAQ(qc4*B30`1ELh3>rO%x3!V3rl7<)dk8x0PU16N z7i04Ys&(J|2Kd=mTsZu1<`o-W7QNMA$u6+?6;5os<)*Ma{I#<6c*f_l9QB996?F}_ z`hH9Lce8mWpwK|Y$~Jwbp2mjr_hhm8*rda^T}U3}n=F@pVGbz`lO*BYiww;g(aGky zcVE$*4G&xFN&U|sSJmvQBI)1?1y=iScH8?sIv!rD#k=CN?Ff0@3HIFDTgqw^6za`= z{MGMbe21RbToL-y4lad3&`}(5FbTQBzBSb>&o{$i)r*kUmrVxP-0?98dJb{zMI-xk z=B9WHE-O=^* z<>F%}Yuh5;Hmp?8M|{NFZyRw5o=RX?Rp9G}%i0aEu@j}35ToIzG(s40)nMJrZ*CL1 zM-MlKE2*tA`k-gb2ALmBj${Fw2Y%D{km&tdkIFyScxAXpa z>b?-68e1@fn=ecA#*O09{my(7%Yp)rhNxFcVPxuxD@=~$d0BlWB#mutCkxXb>}^?H z%tgU{JzZC>aquToO}(ZsFej4o%PzM%;!Od_1(;o#UFm^NH{|E2O;7LqtOI zsy)3HodNQ_jg7Fz{k!<9M*l0TmD?*9b790i8wDD)VY1sDhU-J^8XG+%&DE$8pIgjF zkB7#>Sf9fd&DErCucKRFpW3B)RDJMQpltO0@0mv)%15IVyC1rml-q2Aea3f3{u9B5 z^G+jtv3CVbsY(W|o{3de>{TVN@+Irv(NV<1mI`>!V{zJivSBmSCPYcrzb$ribI$mC z-(rDkT;1KKu<{BuP$T)yUH&Y+K0C+co?Mn<#=)B4<*f7LVfj?3qBg35<{Wdp)KfpQ z9KkgaVBQc}wJIf?#f|V>Dn&(_R2u%i&b)w?6Q{JQJh*)ogBu#~mRs5O63Z9ZutS?P z$UiwZ^=unej@0^~l7XP2u(2X=&>)mEv$&!@@ncow;M!*0KlTCyPHA#Wfqn8-h?GWt zqo$qn;&M^_npHIRz%}7gZ^lVFFdS=e8~L#J+w5CovTMwI>clP}?8;jCBV{17jc(Ht z3$pwa+x1(*={!rX<(=HqIMKP%Ow7;umv}y7bGxyf#hy)ND^mesrjq=e)poddHCzg_ zJi-ggqO2O_I>=^!J~rFa2CLfrb)70!B)zrup6R#h;E&u&qN#;#RuM-Hh^BC`=ceVt z!Z)|S|CBM{j7^GgOE&0!)uCTHtuav`I@!L}^Vke2|rsLzgvndh^#0Zf}3@ zmC<%5$$Uv{ctE82PWCHjjF-QDYH|75Hg~E`Ox09srIFC^dw&Tr)ekYCGTS}N&mDO; z2+`=KTO>=rDfjcKaT!RIr)THJ-aC-)d4{}@VpRmXcER5p78PRSEYg3vz1_1{GU1gn zR2B%n_noDCYYvjD`kwlbPeH=%gtI62=|&{-wKCoa_0$c!?uEIa>lruX$HipwhFhu< zUBMLNlyyoTU7DSVxeuKfL6Zg}P8dKi+>U?5)9U;65^RXySC%H8&f*nwf|6{^dR7f% z>PxXXap-0y<<$Kp#ufp3czQNw@O|6XCTmo$HgEm;^zxKStM29kCS}O?tVMB+30La) zA%eNXro{UDm%tA~%&WldfqpF{I9OlJ9DPzsJaZRcDG-I3FAeC4tjfiBNrkkOU@UQK zRQ*3>eFan;&C+%tKyVN4?(XjH?!nz%7I$}dcPB`&;O-tA0t><2;omFod+&e0IcLwD z+3oG=p028{>3X^zwXv4ld6o2H8`G4nG91+f@_we0tW&d*Wn$FH=p|;Zd3h2#6-3dh zC3B8mk1qrGAX>5e`GiQ)dOLgjIF(8z0NQ!wj4@3|eN`_^;NUALHe3@e&#>(=f zu&^VyZyE}kHpEEE-H?@6!+l|nQ^dpP^A(XySx@MDqCl1l388e(Gln$5M5%JBGwCO^^%-saJD&FUp8jOdaDtGX;A8Iu(vlUNvY zb91Ut;_selHmw(C14?LtbTVyi%7Aw=RU;{>tnWVe%v*Ve!mTdcCYyKY78WoA$VJi; z%cljTpQ$>Q4WM+BQliQGJD}}IZPwbYL-KkIo;z_O9o^7jAyFeC5~V__rO6X3L5&Vr z4nqeM)L8LF_*`pMR$c}(kRWEl$m?-Y&_+C*6d>n;A0EL~H9yLkeR66_nGOS_pk3HBk#hX_-J-5-izogl=P0 zn!kV48MUxV5(~;=g=LJgG>ww^vLH2c`3XOk8XQAxl>PhcMk*hwFYgg2_3!f()MGi{Bf(UY2zfJF{V z1VzG3&tO2?Fhi|enBG{@1}4~^ngMJR=5Ha4us~*Ev)aM1g}XmUibT|91^I1riVWP3 zE~}`LT3YtOStIT~Ij)f`sl?YXqi~ukoe~bDR#+Ft>uD(SfkG)|@5;Yah)bP8T7LV@ zhxJ5~_vjSAq?%ijZ3iLqUj}nnnk}=m)*)c44$AmdtEeY+x=i7Rqy+Z)1qE6#kRbkE^6o>JU4w4+s&4=GPR$$>VN6r7Bw4DrcH^D zWy10>8;T#@0<(~&FRkbbU2%DBq9+;A!u-^z<>$~!?5KkTouXWMpE?UKT~zI)6FmjR z?i3l2?VK0r$)=}$IXXHouCEQWWD}OXKql>`E@K2%h7cRo?cx7x}p8?LilW8Orn>)hX(yyt@CS@L+1&CqG zQ+)Fh#{MBVuYRVGNn{^~EqO%>!$7E(TZVFOcF9I`m1r_7{m$U8q}{i7cMD4yK~hf(8|rEfXoHUXWR8fMLenRttwu}=*b7L#MHwc8>7GS*aT%O)ktt+QDY@Ktb?dB zpE;nuJX=jkc!?(zbbL=ut22<0lsK}QLI#_!6^(IUQO(LKLJt#TJ+tkyrzEwrGBrpj z_}&>__e{n5L}HX->bOs_4DT_=%_QZa8%H(E(-5^)1`!GfWP*=e@@RU^N?oKR${a_j z)cd`smdI@aU$Z^++{j_9$Db+_s#7aLlDK2d6ag}%Wfl$jhk8i?gw+=~Mb(wby9Xw& z?7#>JScLvmctA3J4CW{$-Vo8?hXg zBhe@?dtT0~&o(>LIZygS;c|G~qL!*PK_1=_aLs$Ymsmq^G7RZM^?j-So1#gOmsii* zg@3re&#s2RORrx9p2LuM&pYEgBQk?&A&BXOKc0Y z!!q1|GQX=O-2bNDKemU=q0i;DGy8*@@pYQ;Mv@TcH|EshBD!Q8p3y)Uf}VUjgMN`3 zP2!lTC#L4m(+$wd-kt&){C%r(3T0ZD4Yi?U_;&2#y8Pn`g=71E-Ij2gxc=gFt@$6W z>4UnReH=F#>S&No?PxRTrRn2VD{lKGIDQsq*G<0%kSRtY%#!&VsQjI4h*RLb-m~XH zv*YZuu4Tkk99kH0I^(ypj*d(Z;aUx|I*LRhLuI?Tv0#r&f8Qrafmhs*5O#d!x-jeS zbBVy}wF=brBD?Ch1K6I5c&HqDc7 znz0Jx$(E`V*44#%T(#!#`=s69yZgy~)!vNtg>*mHDDD)4)fE~uQW(%am#6+!H z4TY7-Ox8Ko&)WS(N4`#+hg0uSkU+LFn{FuusTk>X@2>6pGEAScdw z6e@?qmQ2S${92vTp<{6-m&lEBj|r=;d&=G|?J5b{m{jaQ#q|or#wo+E{?;VtoUKn$ zTXJH0u$5OvM_=P+U%6!38jV*ts@}Z&_f@1K5k@Lhxrla30VC$H9w~c>M=+5xB&_T> z-Tg3LG_tE*9TC^ooK@5zt05r7qb-&i1bCm|)OQo?e1_@*D;c&ba4t?(yI(pga^7gCX*0VDru7?k_ybSsH_b(O}i1Qn^+Lz>v zG5+b9a4SlO^NF(!_?nw+L4NqGFe==>btDx0FGTwmbfzBP377C>x|j7ktXHRUw#lII zU%b|O+!5;qxr4m~r;AtZC|KLRaK^P~5^yt9fqgmg%b3h|=p9&uv{^l2O$%L0g`# z)XbRuwvbRdp{8kEo3g56u(%pm=UN>>AM*k(1M?zJw<>nc8mY!(Rvc%P;Wlez7o%7hQ@%9 zGtcaqBgS(Nqi9kf&HsVmLO?KYcJ-y5h%yH4)M8s3QIhEA}ravC-^hC{j&cD%7tQ% zaP+CHG*e@Ic++O*?03SS-9RLf>eYTuLcl01n#J7J*n9iR)Zr55b^+aqhy+5{*J_oa z!+Zhb{s}ZowGRR3e(vFOazbV!4Q@;XpQsus{JQRS?FRENL!H$`V^KRP>KYn*j$FvE zqu3QCI z9(;?ch)E5xPpM|)YAQFDSz#+Dr6na0eZpa$*VlX8{le0b!Tw2QR-WAkGBPsAcq98) zJ2y{C=#YKPZ1MyDtU73i@Zipynwq+hwO^b(sReZ3fRd z62;z*ElKRnnY!ph5ilN320<|DbC$}2fmNOMW?MyWt`)cw52_q7Z=*1`f5!qO(dQ#75HzSu+v+iebkJToc|+F%o8b7rF> z3Z!@^6~mUiX|bofhDcDPZ;ff2Kzwz5cMJ~)K13I2n*{6Wl~;|6+X%l01MZYv`(1w= zSy;C6tdz7`hW{q*YlAr^w;gvHA;9~qt{h8dW}#MXhwG8`pdlLxmzmGQ`H&$~LQYQ1 zzjXZHst!7F4be&^My0EgI0(wv#O|y zQYaVwFAe;sF4T~RZkI1ZbR$yzmLQsJ_4knPFJkC#G zEA4*k%`Z{>MbVAjPiW+#Qp=X-mOHnjT#=WKCv;%ah;1}dbmrK(=^PkZgJ?aqP%N2{ ziySsoQT_`2aK5^+;pP_fk@47D4o5ayP_|-2*=(Urf3OXns&LLAPx00^5`r0F8gxD$ zWp@3gBD9-E!cmhoVzHk$fX$Zmk&_$+_MZXyhY9}26`pu$nCe|$OGkm={nR$I*OfT{ z=Zo0V7E47M-xp`UbO36wI$qr;8cNJdFUJnOc0C3?226%@xcCv- zpO`|`hy_T9BIRCd-B!f;e(c#MPZb-gljoa0jy(9MI=tDDgbD*I?c|xl94db-kjE9O zPVbK0g}*)DpUGa_{7Odu-G8F-*gduIZExXeok?P)P*p@jK%&6F(GzIoN=+b^t7P@hLYLfG@44vbb+OR7310G9e8}Y6o3vk>1 zN`VoJGqZDJ&|lq#lAkB0)Pb4$rbgnzX#Bf0+v{j)7w{+qvt%#GBZ~#o0d(GvNPA5* zbJ&nc=H!aX+)zNexvT}wI`1~?E%z`JEQy!`SYpwp@Fph8APJpJ^4qBBJqnsb5>_jC zJvO(#Hdj$r+6mUtZ;ZJxA|0IhcW=Q_ac6L!Uf%RcN~izsQ4 z#qta{OeET-fy$H)Km8`<8RkiW9_+2z8daq`^^U%>^h!!Fx|T5njyL?2Np$a3R?;XQjU zj1aTNT(cBcRSaL}u&qqknv2pDwphKq2YDsU1Y4>`HGa_huHC{E`p)8AB$Xk?yW`9X zOr zj=}!b0nD2?{?+Znx!&eLNl6+00a?E?f%y&4w^*r)@=Ds&+}yDjMGK9D1hfUge*1p7 zos3RR9ptMiDFswjm6Vk^i|}GPUaWeoYM<|Labq$)kDrTClLMaLvjMCQXp&@3W23r9 zDfW_%Iph8L8JG9jbX%}}SDEl#-dwfGQ_zT)+!LA4V{nhSdcrg2Fh>;!;~LkevS`{J zR-7YamNRyu3}t>Xp#!f3X^Q@_z(Y$t|2H!>MOblRN1PmTo2T3A%Ee=*h~?v5D(Lqw zdZoBefO#_1n=i@fD3Rm&PK}Rm<(@{v7>HJGPgHeQm@|H09~cIm_ULe2Bt615?=A;R zi{DjAEd@B?1n>u{x4$;!)AJ+>xABSHSR&HuUO_S{yhin_5LIGpB4Ep9qvQHlxpJEr zWJtMff^E22xzN2DuzwS9! zVLk1+#zB|c&Gf;aE)sPoy>NZcFZl@?SzhlI-LVIst z66v$n=EU;(L}$DXivpTaZ_9|hH`ub^{^+9Qa7h+vKrVbvT_&qVQF++x$=&UrrRbY!UZg%jk@^*r?>Z>2tVLOlD{tB$2Q9o^mh9Y|PuxzU(Q(a_HJwQw z&cLR}hJ?KgGVJ%1!;!}Flvu$Si&G!LX6x+8zJ~|OPav5QX>K=vlmfhe3WwnH$zVJ5 z^J5=E#r~{)>HEm)ij{YPe>?5`Q4L@VoJ%>8CCbPROVz_PY3xP?`(a%db+yW%E zCTC@^xi*1*dQ3K<3p(KX$^|fw@|DjG__ZVyn|pyjTD?*3GsS-)R*L;CddQK#h zNBoDmHY@n)v)(34ANJ4o*s>{wN#6u~B9%!cSgkksTO!(=ShDI4%0+;8=lWz+&7Q>HSdSg1YQ#}{PnTqm9CJAG1vpaTVK3Pl~gBbMx4@W zEX*p$yOH8Z1ee6TcKHmvZ^7~q_N! zW>Gk5Tykf8+#|$%TMc~~xhL~p$Fs*{N3Z`NuRF;IP!@NDeO0H^ZouNys`|#s8#dqSAU9FJA6n0N{D}N`K*I8JNDiL6<yM5$>POIJjV&R4i=TNR(QJtyC?ILYESX$I}s?y z)R|O^^e)qky6on|!E|)| zkQsEyrnLVTOp^Z?`|=D~dB_ zCRXHDMkqPQHOHHtwumAX*9_Iop1Ll1_`A@ceHN?bZR| zXir?qk|%2IE41y}b+Lq{sQxRwAr~)`SC*vt0d7^VC`{I?WOB9+ceYp&QDU}yl+WfY zj<+|H0v3FmNeDR+MV!c{&1g8*#4UaMnQzreZtpEu>*J0v#^t3vh$^Kg3!;oSx#w{m+d@X5sQ%p3<<&7boj!r9B6 zoMP~Gd6M?@*)ji7vxZZ=LZ$F)%E+-HC|9ntoraKVu8GH*$&6&uYlJj+pLvi`4;g5TjdT*zIuA;-7A<)Da25r30Z8uJ2-)- zS6ydT24ZS0aYvb|6~OT7%qD#9YW|Nxp(F8_qerlK1wi;Rf&(d~NcfI}@;}~zgZ{Y5 zdHW+?=}(s{gn;!~(HNaC#*%TINHxWNo-CXzp_FX)T%zde`Sd-FAp;7iFVQ<4+jxZX z3T8p9YVd?*^DtqF=kObvF;vj^r!MV4ho6+93UUsjD1;r9-Uws|4D`DP{BFx14+LMG za>uIWDM0AKG@iFxm108N0`gD&EJ*O-ll z`qbrvz|u+WSR7jkb(GU$EGetC@OR&7;*FRcWXR67kWFK;n-?m*XIdRoQWX;s<<$`^ z@$N8Xtz`z&+~DSbWcHgh6@tx3jw{%3G7fvm0$2W%HcgVpX!LtocmxS`bZaESb+fmX+4#_Jic zuE_J(AQ$YuxP5%xO*r0w98I-U=R1S`983eet>4R zOv(kyS7kVxjEz5R6cuH#ROsFP$PN)Ci0o*;8{Jm6a{<85HF3k+tK68xbm);)*We6m zs~yOL6eLB?3B9*B^8FsfO&&;uc-={}+GPYe^!2oul#L}fW|B(nzH)1zCPJ!4r4^WVUhiUswRmO77Pb2ZO2XVh2O*Ju|+zJ$J!8%NzW2U@ARX_wQ3@nhYF@Lt*|bUEXh(hJa! zz9kZahOF4!a^)ixRuEkR;5n;#vC*FIfrVWr%2%DYB`XfnauHEwV6fnQeqe;v7BN5& zhu0?oormtQJ`kZY`d2oDd_`hDD!l5*0atG{Dtmx{bx#1H|4&5)0NKba_UspMh*v*0 zi1I39G>6`M9G5h7qxjhJ1@_NYDW;Nz7;{8B4~aO~kyEC>nR3B`p|VyaKMC9IJnV`V zI|0bTsbQ-W;{V+2B3b6JgZlxpB=$q3;>!NOAH52^27U@sMT6dfgBX>L*t3k94%oV& z+<7P_e#}<0rn)$1@48#`-EduCfAzTb-guqZ;C;ORAlN-@{H8j!t;q#cKgZ<^><1;1 zwaXe9@Ywhn66%ApXz1ro_ne0vNCgF?bZRLGG1q&T6{Aqd86AXiU^)*2V8{akKc@BK zceo7v1LmzMM;h0dXhF|r41!9(E#qPW{Fa|GnRH|F?dWS+eRTQ75I%JCA7tohl|w_@ zsQ$Uva~W988H@MVC&1-xE#KX|-u-2EffmoV8>V_i&`sL5e=i$k0_1A07p*~5CH zz6;9j9d|nXHu)wPrtgKl#w>xzl8`1CLHR9fbTB=CufN1D#_N??;MELw1Xjs_EB>mu zm!xRsE@zu4V*gZoM3+%e*6D5Hc!3-j&Z37U94>xlhS`p|! zm2F6EmT)Q!-p7i?nlg8VR!HqeT#3P&8jM84@_q5A%714=61+#@?eeqEW(mh_+Xmg5 z=6T2P7TP*-Pv*-AOq22Q3D-KhCMad_P|#X|BEj}OF_x_n3IUeKZj2GG1pe71H!?S!0Q+V zuk}oFxa<8Z`42S%g71MVbB-xr8=Zg#2Hr>Re&_5liNU(Y!bBBUpaeRU5E;F;Xg)sW z#$RIYA*Q<@HKrlWxq8iwVJOQuReR+zz5V#3#cACg#j{PPJ;9JB%Dcz;6}uEI-;wjW z&5>(WMY)JYnYe)lLLUx(hspVH?L3T`0~dTCzqzYdsv@+74HzX!5E`H`XthMWF6Un=W|;9=)l{7SrH=7*YAI zo1vBfwRkYA1^kjR>AIg-C}E&?s5XBB>-i>Lw&~3~X*0rVf4J;Vt<1A5jfOGi1J60j z8r)6{UZH63ei@9;;?@_6>+{Ctk2pOQCs#UJ>G61=@|Ymo)#FTL7#J7;?wO~(3|X)f z0}joAS@~Z3q3MtAyF59EWZnoJnZ&-ok}NrcZ5Wf^d6SNN^mvBtfw>-(jhlqRo$sO@ z`%w&o5XUg>wAvpx>3nyu)pWEq!#PUvd2iv4ZXeTPS;((9IlP>#4qoi?4AXrzG_Soi}8`M-pLV^Apl^? z4xnKH9Up%xQUUup?0Jq2F`M_@I&V8+EF#e(DcNv8rVuMqN`^#2Fg;gY0msNp@Zyix zt0lCtr2XkT zL~5MzE_A3$W|f5(!+BQLRsB3$S)I|oB`v<_3190JwOX!Y1+vNRq`qqO7lbs7siOLo!PRWHCx3kvZwD9pIS?l5AtSW2Kt77^C%$Mz^BGopa+`A0@)WezC~^%~Aj$twXY#>e0yYAOZ7KKG56 zHqU`CpI0!JpS~h$@ExA$O|!`jjfU-}Xn-#`L6rYnBsDLpQ60+L3eztn%e(KIEY|hUUoVbI`YaTqFDN6oB zf%Vs*_tSIqw+cf-3(&jQ+t;hu57BD$RMp*Dn!}Mb)ypu81bj0wqScv+)>2mLik*h3 zXhDlz4aQiOq(-beQ#RiC41U~RPwoMR)v2S{U1|4Z2KBu;D{9in_x%zT}J&5hx3-wKjzwY(@ndizb}ruD4? ztrDaQw~IohHoX?1yg?@?lqPeyR%t5{4rOosW%_3kQVq^h&Cy}kznAnQ5IOehe+}b4 zZ2pWFd=nY-vJz{G=TCHD>`g|&WT)&eYVsI(;B)%GGN)%K(sak61R9+LRmvG)496A| zmU=9()%^IBTB^IMVLucR>AhKJ;`t1I`0*pmjyHdu03`E#VabOtC|o>nQ0)5J45`f&skt z_;M#Lj(~#+FQ-8%KP`>H2*kwGUq3==*n>|9Jfq5paRfBU#Y1i8B#9p}Ed^*i>_nqw z$?0j(LyI9-JO20>0PK05!=y8u_AjZ}dJU<9Z%~o(C`&hmo@@Q{X7aJeS|Xs);{!27 z>nH_jVZR3Mm~NP5SGEo&<>5qw1s1fjKvK?HJlc`TJtb)7`R3HyNg!60D0v91+&aX> zZgY9CUNonBS~_u6Z3hpWl&lSTI62vagWNvYwDwhP?rcnnTHzUG7G7tYZ9vN(`UdAOW9$*o)q0|V6vq;cQ#i4!m-bikbysdkM!aVtaBlp zvkaeeb0btrq0V?hhT`?Xs4V>GQlmq8YD~{}+i<>C0wO3FaIR9;<;ggdinlYX<_+JO zd0(5v{3#LJOOLZ5VUAF6<`BD&3mqTEj?qjhoNXGTY*5c0$B9`{(4CPK(-c|lSUBC~ ztLB4)0(6jS>C@WxLmz>cQ&pPHox3m4apOj-Z|O<4i^|zY0}Dq!2cHZG`STJV-a|Hz z2XK=|VmUoXQHAMIH+BGZ-Ik<;-AqXxFjrVr;A9V-p?P*H%+BtO=I;Ap6R;VN*HRe1 zxe=B9l8d2j3&4#Z87MhWE;6`bTY8iJ`90x2{x#9x`(rB)G=H9GLnda8MQfTr=Vc; zkni!_CBwvcj`UGpOhZwEe?7BdmFPV~MkQK+??#8Se~(lNIR_*3F~=ar>kUt2zgP+7 zHD1iZV}FeE1|~9E_#nO_q?E1Tgxy1>-jb3bK3)_aMY=>%hg;}dcxTUJf=nC}R46ho zZn(H20`fFue2au8vNJO|(!mEmU*Tv5 z#ao2LI5G|%w4BE^7V~(?o-)c6vq%Azq^ma?kbyU@k7w90l8A#FH0}_?f#Z%hnXJuQ z>F(8sE@x{?=iVIG93J>c(cKO__D{45Ghg-RuLMdK^2 zWS=Ffx|3fiN62tJRiPVO7uv62akGz~VWSkl@{xL>yhA^;O~n&JimNMefr*RnBBvHi z8pQb2v%~qn<~>FG8TA1bua~PMHy)n3evoIQybhpG-dm!clAwV0QH6bLX!-MjQLYnAAEjH`~4r}hiX63HXFJ!4gQ<$TtWXd9(Gp1%cS7|El zA_L!Q+0;~ol0$;#5i#{aeDg%n>fr%nVQXl7Rx{6U)phWI4{y0d}1x5LDBBspu zuO}GG%_h=v&SYd|JI-3yxiZ_^x$3&EWty6rkaLypwBBxv^5{I+zy84GS1PQYFzXOKj31`OGEN263>buh#8kivbGxKnUWOo&tS5Y^U zh((MJCop+$Bdf~4%wLqCEYGFq$H^_z;Bp6(kSU#x7K24`uR z!TJ4%hnBzMuA0P@IFqYnVI4yKUOpIxN z2n@7!w{S2Nv*j$2bub7^G2iC!4Z>g7z<|KZ%M1El-P=0p4?Td>m;4yGJ(fFg&Eyo~ zv)_ik@l2PDc~+#Yium|6rL6a+kHteCPz}HpS9j~!4@z5g+cC>;k_X&v=+^)QvBYm0 zEp)-^7W3d8@V-29ag$ovbl2eT3%g2=lyu25QsyNxbSH5 znn3(>-;sQ1WkjWRJ)&p(c#bwaqZLR^1Ca}!)frQ(!}8Wl@?Y@4e@F5AU@8}hzVdQ3 zjpfU2pT{yg(WzQ9r2n>P-d|Z#b5wG-$1QU8Syk2x<1YSUV&ES|;1;a>6w99zlOq@8 zdvHqUqSCgal!wip@;s*@G=xJSE)#W4;1nd{+Px#aw!##G#$Z|{ab+~ii;F{^EZ359a;E#M2n&N%RV8I*f$~TG z|E=H}0F{qZ%q9S^M+Ct18W~Rfp*hI?r#9 zzgnRbZ1u>`pttoY{3ynJ5(@u|m8U);pC6*C8oop_On23PHMX}LXp26V@MXlepO6`f zbNAR`OZeYED-}a-sH5BK^((3hm@JR0yyAQ@b?Zri`_)FqOAe3kgYAFEEk`9sR;&Qw zO&_3WQJ~$6qPJN|;NFb*H?D+;D~sT-ivo%HSA*qAKz^Hr14;7>LKhFo=iG5HpXF)u z!#LO`mHyer{MbYM@at95O=u)fZT>g2bI&u#CRM^BpI}S$|A1C@pg}25|JBF>3FnKf zkwFhXqUsR@c>DZ7g@cCK8khD8H&`eIA$H{atpHnpVw<)L@rC~kP>@N*-?GA0~ zFDfjh_@hmBu)=w~B_Lj39P%`>N^yVD5gqcc8-ZYXVgKpL;T8JjevI`b<}$g6`lE|| ztKDlRqhQs~J+}tOKawR6o&Hrco`tHxO1un6L*Y+xWPXHL=+^azj zB4tlN*uU%Cxm_iRWW}28@?%4s*X)2&)1^;|+MlP~3Ky2)sbqD;eIV$f&RG~2TBiNwzrDLs zA{?TS3idxUlq1{MzAN<8EBPb!?f&00y`_^Uz+0?}l^XUs_KCIAodEq4Or>%kAGs}& zd$p&Z`eQ$0OU1~S$ybXc|E+$GzeG+v_Q$uTGVfL$4f0x1zXfcc8c*pVQI>j9XFPn( zXIwg3A^-CKX|v=4DHm~fth;{a^}_9!46b@k&NfZ1yy!jo6Kzeq9pqy2jH7nX8f^Np zd;YParLSz(D&vaD;I7@>ywzp$W&?FBNtLD(t=37rLna)5%dbLq6;Z<-lf_uRQPMX*nqgb5O0U!C{tPec`9pL2td6$Az3^GfDOTR=Fck{4@w> zC8n@DoB8yujk|XZIZm_xG@eokgjq`fp1$xB$M8I-uMU6HPx&RxCS#88>gUt>#ByUSLyzaL$MpF(X@F}p-e zO#}U(s{ZOnj%$h0--)j@b;XgJfb?MXwPMD$<2GwO`T_CE$h>>v9m6Gv zrvxouCwu!#G2_%gF?cIBQVDqx4+I%5ZEwf=-*!2MRoB%WUd)m{vI?WParZ8CyuR+k zwR7t*Z+Bhs;o9;bHjb3fb@(_W*&>@plk@ZBJlvBdVcYEnjPkh3=*}D+Nb9-Fas}a# z($T@EPl#)4WB*5v|0Rq&@D`w@E1lX`b#I7X%*}h}^Lku4tX}1*MLU1$_}&dv<3COL z+~n~-k|V0Sm(FZ1Qd}IQUn-WRe&|0jVNWBgoSGvxB@4b152t=f%rq2JG6@L{MZv)M z%Lx5#-zp$)p%FotbP(-nG-xYG_(v&d4jqog4gb5u-;Z+|+8ZML=*~L~iE#RIci{i} zL&CMCAW3;2i2|}*Us*gln_+{1U8++-jCuRV4`t2O<1^H7?bI1B*ou5_0yZ9+s@YuQ z9M`+7#ZQmnckST+Gi=tJph?H^54gb>$Y=_ehaCpj? zTBw#adUgi;M;|^YU7o;~O??wD<^4&$065)h2Q6AnSy+^2`%AJ%Z~Mnhbh_@&6W2En z)q{AR=hgXKw%Kow+`v>}pES?t9~0vZ@<9x;b;HNv|F#GJJ}Gq`qrB$10Cf)QLC|y+ zC~T{_47K~K2Ae4znc$=`g{J-bUt>*6xU0Hn2bLH<5Sf{kDfPOsf{o1B$`toEJ`WAg z=?D-@wId(RUkC$~m%n@t_rEyXFw?(RfF`S98jxA_0R?IXgLE{jYxk?&mNabeYg^-Z z9BDM?L``)tRPkrY%Y+2~$=@MXg6f|&B&nD5Wql9N{A z+L`zTYZAL@IZag6vn*YqSZ=%%v1|9$wyOATd4Chpbn@b(4Os3~^2&`h5@C1PMX*?L zsc-J;?0;Ge;bmbKiY z)gSpr=$#b5dR*u}(HqTJ&OglhEyX3M-uttE_W@E@+wCX6EGo4N zgVAr_{PK(IJ_(LGnA}ZjdU`)&f)WLSG6@b2LpoeO+7LQ5aEz`5xz%Joa}xOchRMrp zjFr&gimL%0SbQ@aFruU>SL&wod|4dz{R<1Q&nTy>8nv!Ge2?Jbb&jNXc|6Xq#txC7 z{}sS;U`61$W00rs6@j*>guOrr$YGZRu)#g8cKRX4_udm!bJ`dGv+;J5`B$in-R~4W z_%AyfA_D5ei`I-_?-ODY@9JiLKzrf|8oT9bK-+a!=kk-GjW0vD^=|@cG4tRNtnOoO^SWXPLJ5%Sy_^rr0oxFdb zm?J7_+EcQWB8-`nZZnP+ekZ(|fn3_S`b~J8kE4(ru#YU;M)1=PNhBwa;(gp0!>-Z! zm>qxhYDh!|FVwYF#8wBhiB5K{wLS;T&V&iHb@GRMW4Bh>#czAl7Iu~~9El+w&R|Ld zaZm>DpzrKLu$ZyZ76&z6=t$3kzjPndo<%1-fm5ZXdcK@12)3f_!f+vu1e+uGclk~~ zVa{rD9{$+K2$M2j7EO#rF3gDpvsd#5Hz9L**xiIT-yPrC(s$3*}lc~$OA_0e8;dB9D-`I zBeD;zZm_6L`Q*BggJ-RyKJL-|ERyNuk=Ve3ej2$%XDY|A!JO55)RCu+c|q<@waEr; zYRyz8qZW`1hMti|(u-J<@G?Nxd1Hb^#Oq znVt@a@<~nzE!Hz}>m&xV{@K^)%%NRPnMgra3FQFGj zJ9X4%yR;q&>FJW77x^dOXI3{+eiq8mF2sA^cTOnG!{qezkx^X&qbV@T@@w9>%j4(= zCN{?St}Z@rB>V|DxQG2)2yWt|y2HyR;L&0w8fY!xAL}O*(XtKPzB$M(gnI|u@ds5S z0{zG8T*B)3d>(e+?0sDkM_vKt3FUE9H~9xiU3{PoR7V@7v`ACGq}98TQukjK4!b>Z zy*NKo?{tCAHlQU2G(uDH$mk2SWcCDReYG??9mU#jyCt87WHt8WJ5#1JjihoEV7#gI zAHdr5*&{&ByS|3)*bQ7+^PUVXdR}D*<&2)doq`KTY<4*#g`b)rb?8@Bdj#Ma+#n{0 z8w{UJv|c{H?T=1$W|U)|4jJ?$ei=am#xL}g5R1VLNy;A^8T8WT|Fhcbqs<2}j^!_( zAU))%2?hKR{?+BV^6k&I6?S6(M?`k`%0+i#$VlzrB<6w5A0~h1Nnf9wX}2{-e#{6I z5bKP<>^gz$DR01Fih?VQPkQYOKDwAjvy~a@XM&5qVWdOnieJJuu^`H9#@c6cpBv@5Dhq_ODH znK`eBo*-DuM58QrDjMWD>7+qJV5{bqLek(vG6^sU&oXTPa3L*2s zom69>O$rBJhXH=qi?rSMR)fIj@T1)ZSQ>-JmLXeDLnKkXtClxT-*vaz-E~)*=w~vF zV)02ltwOIW1_^G#-Q8V-1b250 zuEAYHu;A|Q?hsspJ3QRo-6h!Gd^7W_t#4-ko7&nvRS(|fUf%1|-KV=xpO*Tf>dno) zpuR$hY4Wc+y&>Ux2v6n)6^QkZMFHhSmmyTxNa%wDEy>@lWwk3>C)i-Q$%XHN*IfGD(EPl7ne6sMz2P)bnR+-KTz?!}nH0G@7F^z8%fmX~QNG zU-DnE`Oadv@;LTun3tW8eqID*;x|T9_M=LPs+(WW9kqA;Va|7K-KN$biPM8!eb*#4 z?@iR67+o3LEQ;=#_)|2bP=CLPk) zu}H9?T=gMXo5_#WgZ6j_LA#&tsGV9qQoI6b(G?lL+vw13eIKPL5o{$AkX|NG*S)s& zi>lUtGvv2!S1$GqI%yamJ3uIWena_|$HO?ZzSr`yw?$RDHwdSs)sVtver7mZ{oYN` zg)P|*rrJMRhvnC6uuyr>{g_K$IJNzi;twCuR`kNd?{2NB6pOV&t`?%ty$9)|hw)j^ zkc(Z;jL>y|9PNr=!?^W!eAlzDKS@vOc-&Ue9mZh$7r65@KYlxZ8-=c5c;;9*Kt5=C z`UbLD9=}d-SbbicBL|5N$j3`_K0;mI?>xj}NY($jR+gt&NM^eTNUexCAKf5q58vXS z20`S;#OLp6+uNFsf1z^dxY$;R*874$3{8+gfME1*WhL~FENVNFJQfUPskhtBr&u{G zyGCTW#pTZmS3&en_m}7YQ)hAm=4!JVLT3KKA=tukC3W#@d4crO7daCu*7&gu0=7zM zB#QEym{=^iZh{ZhK75mFYytf(ugwmUoE*>#TeoRa;VfHacI2N0@T_+{=g+Eh-8{(y zO<8R$aGf>`#E*9Akt$aujFXjaBep9J$6yigc>@zK7|j=A95Mb+xB~b)&Zw4Q@GoQd`}AvL{2<+UI%Pf>s*VSR&=IsAF{Wi!ye=35~601N+RObDxRXHHS5RW>tHN?Gec0gNO-kW@{6b=Mf?9D{TqmeTBgH z@}iU@g}0<71&}Zz=&&N6^9(yG2f4H%F#nmgW5^Y@LcK5b3AvTEUx+`ZAQU#eesP}3>H`Qi&9t~RwV7pE6NNYuOt75v7eTy$r09LBJl zVkuv12m@b+uHz^Xc5HT36sGn!cCFO!tD&Jj^-rV4lTLda*3D1%1}t?dyf|kShbaFZ z+>=KrkNUwTjBTFi4;{})U>Y!u1m+AaI$)Jaeq&hW&JnHc1h^5vQGoQ4)3erwClM$W zLb68krH0tWMV}rMo>B!H)^dsbgTFf&fs;XDRZplaS^9ppiMc8Ede^!P{Kw>uz_>ZS zr}3%zLr@%_J^g%Bw%ObCx+z(%w*qCM3@|6_Z!lFp zAA0hN;xLO*#t@`_kp8S)*%u@}jT0&-O~rYnuFilvu@e3A@Rb)x6-$v5&{ zs;>#;o$=}coMS(e(8wD+VWmXEzvH#7gqMYqkny3h6^{hp&Zjiex@r*ET7Rag#gwP8 zbjFSlzjR20@g?01WQWoMt=K@^KgN)4UM*(;!T+~$izu@%~;qoD!vi@B+m z-Gk-ue~)dvWFDng8>)_D;g?(tjPn~!U4A2!Kop2f`dNUWO%do&rrt8eJKv2wN({B&*nHf5NbbGrRwlx#<bwkw!ra(Q%HFeSSE!(-bt*moOcM!~w5)*vN9buiL|e7kQfT z&Y~k-hJAhFb(B?-I}D0sTRfNfYEvv5!GV|3!862^?GbX)Dn3Qp2ij9<-V0({y1-G8 zV2eqZ?5_lG|1&UJP}F4VZAO1UP8#X3Gf;D7$41>F%cSj$?}PWvt$k!h#mIeE11lDX zKpf4$7@{oCo`g-bZ|avG80hBSp=Ls&D?*cWhgGxXU!ntgcCmFn5#U`CAdo2Hwbay7W#qE6VHcZ-(pHtcfq}&z3~~ z-~TnbpJW2Mky*q|S-lUxqE^`c$S4a;K5zAxil=0;oF=D~OY6HJe1&d1V~2;2reGk+ zT25fEGwQLxjFr#6SDRg0o@tCstVfey0OS8zr8D8vl{Ho%7611T5qVOh<)||RdtM+Z z%%$Ez95E+H50c7K>WCx7@K``!#~}d|beqkLdiKjF1+6W{h{Nf}w{gLigSW05v0?SB zj(3Hzb@kyUKyI@p4L@nyWSGcL9Nc?d7}eO4MXx$Q0B06P(*;timu4T`1Yf8>O6j7M zb$Ula8>A|bQSpqp)qI`T#y_eYie$@;lqY7Q!1I{XdAZv9I6FmpSZdlkSZ;6si}Y}+ zn4iQ0y83m;BkZgpM$hKS9n&f4yua8&O43@y)Asw4UsrU0AuWTA_wAcZ{LX~Q^sm! zpcLzzY3m(;fteu=7T%DoU$w{b6|Ltv~7)SR@yUDzg4Do4;Zgp7)jKNap zu2V2GI$UtI5Spqytz|X8?VwSL+xu7a6&@L0I9rc2Z>=pzsr~0Yh;-7S&S@r@!A@OyRA>V9LV+Cb3#lJ3^elYD7dcE*=&he#g5A5|75< z5wb)7k)O)KnKy5z)&Ay;Yl@!&&Y2eLb)QRhL7?|m?1A|*-YfP)Bxh2t#!IMR)B(Mv zB(68wL-fYP#)14^U9^D+8JISIEQ^P(mmRbaX}g+&ZHhO;uLJ$PD3FH*MK`*fyeF7X zNRv5}0n30Aw-ICjr_b<;TclwbxnV?v>84!Q&M_n3(x6*xjVsQ0X1wd=BOmn~6ht!)NCPZ)iQZ_;-6IXuk*4$yE2BRPs8m`BR3HzSE$PCF7&CM(B?8&EB-5f&wN(dK0y>+U4fJ5`Kg?g z@uNh(kyp|&e^&QvW}20eXAW2+I-gKVMco)b5}AH4Z)!$qGgwLS@a4cw699z zW~>2U;Az+Re%S(ZrHV;Ik2OOyKM4~PYWQ!P&BkdSX~Ky`-uiw%WnLjHdnHGi+UHAa(} zBm({=#{cuOTMGBTvA}=)L4KO#@*CWJ?}{e3`NzbDpOV&-L!Zco8jdelkgxiCBFXRZ zq!tk9P#M7Q*XZ>dvL~>jsmQ?Z>3y?4wwcI>70GmMAslA<`x9fVEYw;onbi%f4{c^x z$Xk>wkN4ofU%3*F>_N2UxFptigNo5txF;mVSCB-XX_r*2MD`KJz9TDg=M!Qn^msad zl5thNk?UY=OME*bab?sQE6Zbj$(Vhu)>G>fH2M>Rgh5q%o#0m2rhO(xnBmf-F zCpa&42E+1kN&Snz575xl7u3|m$d-edV6|zCnYt7P_4NF6V+`(UHrscy9`O1DZLa>? z*z741*>pyvPrG)bH3M?lFJ#PRj>LQXs$tdfYWU-aNx8s<)vuw(lC{3Y%x>HS4m1Q`K@i5ger=1 ziQQpa?}`WN8>i17^qg4k$Q5Pr<|E=WPjp2T8Mc0Fa|lnPndRwxH?Uey`Gb5d6KyXz zI5>-<`|V8q6OWf)U6*mrf=!V}KcxOAw(R~W70`Wj8;AqZT#mebDcuk=b-aDFOoJG% zHQFWq`>Z5a{#OjD@s~IY%KitV(`I_W&lOGvbB9-4uj<9pvC>a^B`7+L8owG@n{2B5 zvPWr!_usW0z~qdofWkA+n@Jh1S$*O3bG{(^jgYiR&e-)t$UEo|wR)<}l^AlN_2bh= z4O7^OX8&8O_(0L`Xsn(FVz=-_^^R_z*Leo9**w6H()9y9eT5l+G5MxTNMjk!UN(|& zy~eU=Wu(e0c;<=(JQ@^{W{sA;i|Ll$~$vT z77d&>5V3L(Y*H=PED^X~e+h!{u}TcrLfD!go)p|KqVsuv$E`*xd6a8?cHuQmn`lqK zlDnexHNtfCLOdw4e*RQ>U>ys)^*#Jz2Np6GQ(mtlKeJBa1kx)kWp~RsL+XMl#8q zAQR>piY@}83E!Zscdk(Kj$k8SEA255Y=Nm6YWab)z=SL;=(hq$Pp&5}L6B12))n(p+mSG*(EdL^{PpfNq) zgyk9_EZ`9=Ry37DemX9LD<++AlDJWi6L)y>oizJ~*`VUH@bnOwc$tCc;j!q{U%FB#v| zKuN&J-RGxuC{Kew_)A+EscjhwSoC6NEGg~wh|t_pcv+HNJ)&tJUtcC2u&%ZP{EErH z(IX=b_ZMuL4_mMsai;$REKQg&GdB6>&-i~S=(vw*IMyvV&AdB|@`GD46`4LciU_S+|z6 z)=)^(kE%h2ZUU3zudJYHQeKMCu)S)>QH<&tVG6gQxz7(kIN2-|^B{R_5^8`vnFFqr zjHowmDlQ#wWMG>OnO(;Vm-=I43DY}~-@0I^!5VvpD8!&8kRFp?Ndg7z@Pf65Cx*#| z2c_bWtR2Zs!;9@tjx~7Z9tGP68VPRu3QVS(Up%TbZ@?K;2iEz5+OBt|EIIzY@%wwN zJ5&G!Czgr?BW&r;&CdgsU%nhnWVQh9(|Z2go$z=+GJ|i>v9ZO!y*}S`4edGB`n~&l z-X8w$iU+bDcCNnuA9VhhIU=W|g!;duQ&d#+FDqO+9oTaN!F;0SbEN-V6CF6VNJ>g} z+i*HnB{3O9A^EGM%2JY&k^=q$Lqo&=5zG>2n2a2d1rRq)B`8Sr_gnrw{IeMd{xX*! z#**0Zy7DSX5aoWK`YyTHwMe^52u)?8q;;lPbCh2?6i~FgopO3+n)&4E3s`a_+1_1W z({R64%ww&I9sghy;qRk81cW}y;`j`dC`@2Ihmr1-&=yuK%c1#ba$9OuhO%FFm}Cb` z$MKR*&-{#LC%L-F*)Ff%w`fU!*a=hYo= zH5s;2&!oaR?c;=vzfnk7V@&H5e+31XR8kL8N=&oRA%(t!xA==dN9W27k+Ka0e*fO> z_p@whF`A%YwF>n=VB5gV>ue7syQ96b)G83gd0L=x&z!@ryp5lwO_Yp}y7_UMU$#xn z3HXjYv;wUu;@i@~XA#^ZRws)cDFnlK3E|`i|2fOA#a!&5=HyDe!JakoL zc#rnW&!=udk8X-4f)m<*|K(bwpFZu5aFh$nY0RwGtUR|96dm@xJE+L}#2b@dZ}QC% zk5h-j6#79?*Besd=f$4G=|R2Qe$8DCM37%H%6+d+>dn0_5_d;Nrgl>2E92-d4OGY% z7qNE19phgFE;sS8>@`kvRI=G!QPbDb0GQBC^at_Lg1b2Z*&#V0WwZbgh%0Y$?OevR zC-=)WOE!x(oVeg{04r_`_ufwyU++hx(r+{)M~^iPRl%uG_#|r>_(W#Tu>;9k2WzPp z^^+4?ur1b_Tzvzyzvj#MUStrT zl4@eM?vUK@JckYc?xyuR>wX7ce4*DFz&tO|rC*4|=md<8XD!>CT#0DSQA!M7sV-Pd zkDn_TN)HBMm3+RE5shi~8HNB2F}u}%k0@+>9Z6_fQX<$?t!oS8$W}1l`sFB03wau% z#eP7be(H3Yy;;fMm*%2VaxIr4Ph*Qm{4!LETE_Rekd_qb^H*UFZJnY^_r*cYqqLULT(MP55vJKq{i zzT(lWTk`1WP_wge6?f}1`On@I=JeH{S&>$@m)bd91TZqB=&D+G)LY#J_K@u=4$`ly z+v5!)PlpDOvo`N2B)DnPVcph7#5&lfNQf`8?)CE=W_HDLjd2v*QxKP)_rG*AZu=> z)a^FMSGgroO7Uxz`hs8Pmo;Fb*c^~+1ez>6E|?WO-Q&r-U$?=rQh!ew;;@&m4(;=Y z94T_g`5*mV${2G(1?$2~v3d!7P8N9P`{b#|)31=;;s}Oj@--&Nn*(^)p^-Zkhoq=D z+;5maiXd-sC*b`DprQy%DH(gEfo!(fs0os&cf{YGc?H=S%8osrAh-19(#d zeSp@#B*G#UVEj=gm%b=yeks8IU$Fo=Y~Ok*n+iERTcYJrA=2&xoCBCB4Xgh~6QI-Ir`tqWW`AT*^Jvd7t* zVbP*^d5f~(0~9=Lt-7oyA{FFXqv|q0J1Ar@&oQnH(qXYoiGf8kzh}x`cSu1`kD!hv zxQ`mmJP&ehk@jqe9fX!17br#r>N~0E#$h&^FKu2UY)}d%7sKJPt_vx{g>Ee&HyYR0 zE_qyaJ;tTjHp++Pj^2vR3uS*Sb@EhHoeCCn$9iuQ@o8JWF`DZAWe_7tTVTPiubHKelJtV2M7>st!Ub9 zMo!rm{KUj&)3j9MxKQ{7ryr+o#?_5G@c!!N_nv^%bg4uP@-{bHn2>^D4g4v;-R zBf|%Bww&Xj0zlv8McArmg4ryHr%N^~9#E8jV-w2RIFs`t8o;y9esyxUEQ;>7y535X z>l@Hyv3UCO=hE4+zE`U&KOtDgkr^a>&!MPs&`ks`Wlqe?=?{}1Y?k{0$%_DSoj7hn z$k_(`lQ`n|EHPtS<{xoE+8mA`G%Vx(lK2l8U8C}uG8BvqNDvIW;(Wh?QCMngQ(RE1 zmL0v9VG!z~q5NQ+WWj4O`AcZ4mh-tg*x3fj(XgtDL(4|EY3b}o|MU7lcevBDi~x+( z*a(Kp%93P8Q7>Ev?TdE;l(`;(vOD;I5QA%zESirh>9CS`XVFMAfh)c(5iP2?3_1%- z>>&jp0Nu(&b#*D#^$2v3=cYIr$IysSiq(?B(9_D|`EwFd7T9ok(nU>)NEON+tKTOz0Nvz?#Jn9L zbN}#K>FJ}Zg;UUCS|`9FZj4*5^mj_b>%rGA)c`FNnr?T(UbxvaI)p?JIYW{$J-3zP zS*&@)V4{2d^M8xLz~)n=EvCRYqYiaK9UtRAihM8VMJ14t4g5qgB_yWx(FIZe$35*x zRO-)~0b>rUDLe!Vc2ug8+z)s6xVq~o#D*~84?1qFb#Q{!YFV?#WI9l-tM+f@g?>XJKB^HN zL`5?cP2`_dUj&Kve~o0%PZYXuDXHgU(Nw9iXuwABY(UKkwy0>@fx-UC6;e{yW@+>@ zf{I2tdQ>UAp5=Q12o)z;G(if!=v6S zDh-G7!8I7_CkGl$M6fdJvqx|eb||*mT9&{g4sR@u-Qo%hyu0~dw{n#x{b?UhN4kRK z)e#SR^xV+B@}<$F(Bzf15j^E{Axo>vd-o7?^7>H;!a1e`p+5SF@Vi7onMBRtghqu@ zOc(U(>4>*fB2^DYpK0>!3*px~fF!l~p*oNv`!rO+39(!n?;v^P^T|jl!OFEH(A4e? ztII_=1`TgDwl1gWNf|N9wZgk6!+^b}ie(9{X*&Op-(Bq)hgjL>cUd~aM}I?}UO!}y zX^YRNc(oMzvGO908@1~7%2Zlkdb-SFb@Yy@soQ$Ja<&%YSelKSU9Uwe+AE6Dm9odO zYXj9;Jv)-v{^MOv@nj#KWY!fmU6}8eXMx)9Kje$W;@A(JJ%PuH)pwfs5FOh zVOe%o1@8TxzfM;IjO6-$iYnU*tnoF9tC1X5njI{gVs{9E2L3@E2d4d-5_*=38Pb06|Enq!kuP}pQr z23u*-pPC?n-YXHOE$?tz06y}Rs%(^5T(r-TGAu98F{3Px8V+udBZ=+(%+lR!bx5w4 z6$DmxYCIW+tfd$`;G%(iuqmNX+un($w<3miIW$B_l}L!F2OVW>F$kFkNg$#5&`rFiY?ej7=K!8b~F*3&cCCs3mA98s@+Kms0qaGxy8Ko&u{*bQwAs9Jv=XVaZ{Vm z@7Li0<5?*hA24cQDz8OFx$!wGg85_Oj_OA-rAyUsTXPzm%FI{E5f((fZ+mtuPED~@ z1kY?%{qMf-GN}y=h@9Jy|LYbm&`&3k`e+8pPnZ1brZm z8cOK2yPZbyQn)Qy^-vpa9xrL;J^7|tXpok2$msk~I_hr|Y?ps8N1ng{ee|IEOm;!_ z5)yfL`cXD7aeH|8UnPHvY$$&5aL1~`pUaN0;NOr^Qc_dI_$Zc}T5iP0R%DyEg3 z3W}6^Ov-Q)Nm1>i7}$fqY5OgRk+E}6;Z{DlZqDv{IoY%Ipy|uO&CU(SW{pU+%-~GE z8hzOfwv3>A7hVj#<9 zD~t0)MDE*^m422>pi)AL{Kn)vS5it53ms~N=u;v7q$rgy+l6w=NZ_^Cl#)>mz=<%iR#_f0}1bc~(DRjl%) z9aVHSpn0zn22h($O-)fvRZJ|_1i#R0so9fk@Yam}dsh*hh`zxew-aHj1f*G%#=;fN zb|WqlXY3w?l*-iIhhy113d!MprX>x_L9%jo-tsF^*S#iil2EsFR^16bAHl%FLLpPDHJg7|2zj>GSyV% zw<4OsNM&n)qA>SmHUktvG!?{M-xtedek1RI&L!U-F<%SSt#I^bGNlvJ*5lZ7x*dbw z%MLfakq2QYGG`qm1ywC<;hh@e=am`)Lbl! z`v?A&149mf+*`*6cf?W7#C)#QsS{bz!iM4;2|U?c?fZ)LFQucQ2C=1$3tug{BIy3O zu;b;?($KA2EhGW1L*bYQ(Mzef?)_czvGF` zr2QuI1X>70>AD3zU3AE$7E)#ItTw-jN$UP!iHNM;eQOxa6nezGvFI*`si7`=brF(x0cHhD*RfO@hNyfSRBO$xw;IUqOIb z84_7 z%~+CIWJ+s*N-R4US5>$!f<8o1RWSht0LCc;Usz2oK@mnPDOLoImX<_IJc%D)yD>p5 zq__F43{)U?a-~{nek^mNX&P$eo-ArY5jc22@|iE2Br^_VEtF(BA&N9iT_#OUPBzMl zIv7NuvPhF?$)|2#^4w@5Av~obAtglsB%GsI#=(@b&P8eeBZS7fc*>u)#)=ubHA zXKK#l5#W9V=?4Vj_jG|KI9f@tG_+;7!bzs&YI#zUV5&he^kkzXs!=M!Lv^x}>OdE8 zVb(f&G-dRRNCD?zJ)AWLv|*KrZDez1zct&|qC4T8QltLuN$NOWmZOoazI2j9YSVcg zy5BjZAtolNGs7h4Y9BcQ=|_jOgO0a3Dpqj{ZV+2iOmJ|x+uu|;@uLiPnW_rpB5($O z-+4bieV*4QJ4)X&TxyR@`4fGR_hbIhRM0j$Ifj1cb6u8}%&+*BcSop$LZg@;vuE#d zmQDx$b0cB|WHA^YRTBZoXN1*Ng-FOphw7vy5x1=nVyB%xJZS;JY0v(NW0`IK>dE?( zIgGH_VAX5l%;YL=CJZ+M8xjX6RYh*%?Ai((L<)YXL3E_e-X>gmw;5X0qVbB7;z^cf zb-(g_u035biE?zxEl3e@R|}!3OPiRRCE#!kSjx(j^|gCrwlt=BNDw~HVGDSNOeW)` z*n@=f3iIh-Z5SMjKSeMN^-DsmV2&B}ug;On;>w4KMR#}6qr(AtcYoKA0Wlglx*92E zJ;qgda_yUabUpJ=(MuA?q2;H1r1eOqAEq=^7435 zoj#w}Cbk`dJ{dDOxzP71L{!vPiJ7$L)0WBeqvwu?(OU#?ScG4{B{U}=I2IFEfzdU7o$J%`}7~ZZFLGM9M=k30EE~jrbW#A|90opXuDEK-- zLo4V?SlbaWOEhXj6lj+MJseBtA1on*6t@}z?Are)Uaqan5ia%9YI=Rn|cso<`4 z{Ks{_74c(pq|0t|!tI{4Z{Cl;tSzdUn0Z`dwR5F`pd5(uW|vXYv@rXv%lcKH2VesL7+g!5st+xwamy#GP&9Lrx=9boc3 z%tZL5y7;!a=6w}QwZyzSS~mgpMtX2BFc>=xzRuI0@GB_1<<|vWd!03vIVXKiIb@=A zqPARx8h{K{IyEclO7&dj)UDfds>c^|eUpAB-xMA7J4Nt`Grx#Q-#_qU6Acb4ckrpa zw+Ij3XJVKeF9Pee9yBifvq8jAJ!tdKZQsUJ!~~-g5t-#qo;TpzALlQ!*U$MygGr!ed#!BX_UfAJu`;p7k%-r|* z+AOrZ$--OHG~WmCN;__e+oI&aSlK*cqb7pr&#lN_#3*^3oTDOj)IojGZkzt^A<;g! z2Q4R6ZO#ArdL%kunM5vU@v&f1$<(va15Fy(M)^#>(7fN~yE!cA{h!wrlfW}1n$wHq zH@!986!%azV*m&hrT?`7np$j(;}g7(eu`{jG#f-izLt33M{a)a7a3FSJI2xX^`>IO zv)hU+Z>6RIm&@d5#}F;Z^`J@TtjuenFNo^Q6_d^6A6@KYM>BW3x|P{Ms51DXVj5N( zKEv)tpJRE>zJi>6&=cx&y3RMS%&;&sLrq(FPho?t5A!o@#1zOr3mZejNc13`8GN{U zlnK#tI)hq5 zt*|U(<%S-5bZyo9Ogl~bi-?L2ogAX6sLi?Qc%$_ZP{MoU_V!^$`SVDu3Py&&Z?+B3 z&?B{ln%0i)zC~=~L?|dPTpm|mC(Q6V7c z2>7L{h@vXGUs?il%ZP8%dx4{fA9|7I+1oygyo7ALcRen=AHQ$LD`f19cio@Z;Ms`wU+H~*87I57SrSz!?Ri|p z=5z+Bb#}jjW@IO2S$!SjQbS<}ip-=+qvn6P9xEGfI7Kt7=})vKLL|T&Njq2S_Udj!ey&g zhti_J(Tt5F)Nrv+zis!$W@cmpd5wR^#}-i~B^pjw=5|k-9X?kF8*f}|ES{PZW<)x> zuP)Y~KKe)6`NE%g;VRbFhMZLE4By5EWZPpKfPv|+;#hV3=KfZyfn6~KY$wFmu3povYy?b-|Q7Y{I zz9{4uU>DsYs;G)CT?kgS^jTKr3w>j~O-Xdt8%2eFYm9PEOJ(}U*SiRXF=ds%JlHU@ zq>;5_k*3vQ#X?c&bLWD9FE}>3^1!BrmPjWmCbl2iXc=|p8Kg1sQ0s9+j2I?eY8-<% zk6dT>Fp4iI*xTPRSv$jNyIkl0s}&qBvXX(FYhH<(eG0TDZ)y|)xKFdZ9tmBev$h_W z;8g*4o9QKe1*cJpKqkysv9gFqInW=$F%CsWav+Qpta z(^px&#oi;9|26si|Ah}?{3Pw8+vNcZE$(XX+Bz`S=b+ES`g^K%u3;g>gOrwrb87uR z@dy9;VTk_}hrc(p9xJ6B;}BbWbe_WHv+Kr=_j5Y3eXcbbmz}OO4XFloNluLsfeuzv zzVb#}%I{!I-ekSevY>HM%VT|UgByC488%fQdJABqJyt+H$l-C-=av&D?*9hdLU!g$ zWplTUNx6y=ZttYHnMSS3fB8tP|M;VL)|#Bkf_O00@szl_L0epamx0+sEPTmN7(lk` z7IAy|uQ8o%3SCJ5Vm!$!DQRF>9$ThRb3Mn|>M@BVTgf*3<%76TZ6Mt8LA#)%Bk0c` z@WNpAvl1ZIn^+uy!qm1tEt5Y5J+vBkvZJXn3h^fnkhHKV(GC(mNHzj@>R zdcv_ukL-6%g6VtPwKJcK>es?L+T#K965Lq}SGc}@VrkWQP}YDEqz)98P8j`&$_N`| z6gz9Cl+xy&redzZ{C23p{UV#H!Qp~j+nhf%+s7Thb9U>>5fiB4LVp<2O9CqelMES$ z!YBn36G%>OsE=AiHR0s24Jpxir#8{0xbNku!7{rc&O;)Ut5~E;8*^zKgu)0*PNJ)H zs@)qI-Ze7H=jP;Vj74Rz{>5}N65&1p7Dc#4Riadte5+2mR5q8GYBJNjj(4E1Pb%wa z8EZ+8@w%+_u3T$&PLza1x8FbswQ9Rqy#kPXbBNx~bBOp}`}N+}oz>UFld?A;PQR)miOR; zW>&Kg_fyI+8VMG*mbtY!alq^Xf!COfFsiU5c%X-rr0{ogauV1=qgdryN$>(-dcflMzh1=~Z143N!=QhX`L9@j3hIfvgFbc-26fyxO#JPB{F@h!>ai+D2$eV{MhGTII8?b*2tZ%0Vt3k@ z3^*i=6bx9{Y+^XTO1Y|eG2kwRsIW1L^q4L5B!7|K)6UEUU>%m6;&?<1m5~lpCL!!& zSU)K&`NueBCRo^NE~kh=q9v)y)|65eKKCQ^+5IBRW1jKz055LA+8;&p_F@2lJ?$~t zL093%vd*_PIaQ*l`g5ym5<1^9(hQv^T}R{=Kp=I>Ef=%N(*q~Jd)Q8wg1)fdzPQs8 zIyYMAaYDj96eO_86oE3Rto#F4M?A-WJS`alH89e(4{dF?Eegd^C37?-S~Q7aZ^HUf zl|@8))LzyAz5y1bAX;3Ey%e#Sr$%+*?HF3SqdkAGInj=yLkt(w7nlqG)* z2VlEelYVBYy`>uq+Ox!LzU_%^AQkbTEU+?YT* z00YcSTfj;RKaf{+0pr)BKeB$C4V_-QiQ z?w|)cX?jAT%5w|y`QgHRecIsZoA1sqm%uLm;wYjVucb_Sn;qJ!Z9BylW0sZmEB%R# zN8|f5-l4HNO6_n8c{%x;cYgJ+p(ELe-8=KeO3n8!ez^+YUZ~ESO<#G!!_6ap4D~Ma6inZSrButZIXH$$xrBpb z>cqC@rvu;AN}qAvezlkP(e8cZOaD^Kx82a447~KB+|tyRaI~>;u)CMLgi-cn>4vI~ z$mrKqj+dAgk%ofq(omZH@2odgW8)LbsXG^v{*C}uugO(rIDA<@E@M`0-LFJlg**rj zIpHNa?esudSQJ!thEl8u^o3OGbR)-GUp!JAL;t$DZt5B)dIP~%_|AAe1@Uo$7U>g@ z$(=BA`rO7@IFy3+#g89h69E85W#uO>MWjvBMqL^z)&~a9Tcf&Q<$Odc2#$ASlOI^L z1$#Y~n>}*Eg&$+fRN_@fD2_)daMPA8PIS8Y{H~I*ZU17r>uXSFqwdNbHWUBVhrOMd z3eQ0(-lq^#BWEJ70XhUb!ms))0cqxk7q+{4Vh`vJ1P7 zw7{p2njTAw!O1bG(qg4%Lg2b*<}_c}FO1&r>W8vYF5HW_(aKO;GYAR?kj!A@wz8-;BaN3?VgsOh7#d+;GvdSU#y^1snauEl#kEn9XMQAZb5s^W|9`b_rRi|-P5?H zCZF|r*p!o{xhm)t8i~!LS9NIO~q+;jzf`ter8tLw2D=%5In&> z($e+w^#^m@>VwRSl>xc(WM(%bKERh&eWRhaF@=!v)e6^Qs|}#^m{O^AK`u&(BLKS| zMb-A5{AD*0zxWFD8!VB}C}~wm-7mqHt__bY;~lpe*I}#RXTg4~t|vIRs*6Hh1PxUl zr+pYKrhA81KHII+i}msRHoF~uI=rXOna>aHHrg87)px3tWw*EVXXPYj;_SaEJ=`X1$z;kCl7*RJ1#0SlWPI0`OhVPi1z z^%(+R&?9(Bjh`LUBN8y@6`p_axDs*?{xZ$fW%=!|7xFzaMv=N zUvRYVk%yR^#M@xzhyEYF^rhMd?u(7ZaRa~rKtc7bAV+~Af!%suhMJ!;XiL8EJ^@3T zuiVa*(UE$r;+crd+}{=rTK~h!JU9#@la-@4n^Q!CXQVHv+*vtY6&%3(qthIg{d1(9 zT(^ItW@>6$T4DzQ(Zy6-y~5D-fJQze{cBBQ(1J9v5NaICeh{EyRH#Owc|G3xeKIB} zJs-hbAS(3rce`^tGE()1h4>XSw{C%ziUUtxpeiBbL$-IbBl{r});<`-k5K}R3g}cx z;S?-jI=Zoefil9)q%_lW^)bIDsFpl7!MPZ$cEVtz2tGr|(MbvSoKOI!kd)-{(TY1i zSJx$dwD*%zlK1Ty{i$xxvMV>R&&A?$n;Aid8;6H#c}o(j(CY9vjb86vh{E4}xy1}t zN4!e3Bif&B;qml3X|_3K=TnHTKP@6TR*_J#=I@_oVU!PIcUUUT_xkP&sI6vlFkx_Z z*Sd2DR9ErHd3S7M!Fp~??; z2+kaFLOrHl;tFwi&+c=z9o19wjX~!V;>#fmzvG$MJgqV?&b|Q%4*QGc02Jata&$O? ziY3;enQX5x_-}YizY2oLg;J6O1214n`~ydYCBJ{z0zg6l^b`127@(ZsHONW8P>BKi zsM(?DH@f%`K7eHoK| zkZh)V(3P><&B)e`jXtZ;---AK%76!ndqV#?% zXf(ffyuB4mYHGaJ-Z_}8&3RY1U29E>0@wZQ5am=nKwSxjI^u#jJIBQW0kEUJ7OL{X zWSQDsyYp8pDs%PvX8%#j$?W~vC2+d+RViR5kbsK&OZoE0=h#318Iyq@z`)6KS67Fx z6GuJn>7pH*>teE>`+Y}XH{xRguWrnb2p0!_RazfWclV}w%a5}1552qlM<6oyHrKmklp4GI!n;M;^yk97dY2R&ONt<+cBA85sc zsZ|ZJfslZW0~k#1YN*1GS%Kv36a6!%+>U8UjdnW|P5LsS7miWA@Vg3K?%zSfmdnzB z;SN|PTBy@gBXRzOVaNoVqGyyz32Vw>$dB1EFi8ah})i}=)M@Qjroi5_TW6oar zIO$@S*IBHouQwagAgZ=7`vyA|_e=*Ch>AFoxgfFuX30$tg+B@}l*W`6w&6i8o6XVD ze% aa4A9XaDW|LTn2&6BuBQp%6o&0#;IlWRF6ApIZC;DP8nsafsRDxZj(AEjB`R znih7J@9mEFQz4p7vk$}jqnmH|``N=AKhZ1J!DIG1-&lJnfx*_hw^7>06V*Gv4A(nm z*ZawP#>VT$I}X+QyOWLHI@Hc!^F^l@cm2Lbi`kjgzFao{$Pi%ewAT)m<3g*~|>ceq}^So=-UUP74ttJiw(dGyAhxuDX{xLZYji@Ndk zswMBdo|5;wKD4p>scF<-fU)o@ceDs z)%IoFAio%0A7ghLFbxku0#@n4C0R@V7j<75Raeuj2|&uG!=m^99 z9L<6I_7z!q%Q+$4v3XACoy$4jCG~mR#c0BY13$cGxHcEN@WHgOCOQmAdcfg=eu2SK zH`X2%a$aTT>Ft~gx~DW*;}?QA+gUIUrgL1)H|Ugi?db4bg=}o*e#X8WKzvgM67R3X zboG*VkdySQ&g*uiCcX%anszde@yrwtm3xlJ8-00e#QGZO8Ea2q8{8^+Y#85RV5^c*K7s69zh|Pxj-3j=+bhPK zOIF&aP0(g*l-FiIh|w#)ivNq2>VpDkVL2GK4-V$6%|9Za;PZ%ENA#YBRmE3tvS8;Y zysNi*0<-0P{ml0~$*DgIy&R6cbn~i|LC~D(TVE29KlIzu#^lD)cK}=P4P}p4b<)m; z@!=xHkA*z<7fK}+hR7!jtwpz$6)wA1U4b{-uI|PM+d_yG9&V|fXHJK4e4BmVJiWb{ zk0X+GpA|T~pEXLJxYnzGRu-JE$S`}2L57l|QBPrd%N2gtps2+2)28)_8Euez5acH% z-t&W-4&z1y4%V6%;Z6?+)TXWrdOCCJ?`6K|wmLV2MBa}+X{NS0RSkc}QaduJZflI+ zc<`jP`{n;GW}sQj5A&Claird(RR+bDymJ+t)%gW}kW_(S0Lh$ROpHOlgi^5Met}W{ zC1v1W1_z4#HKq-8N&LV-45tR&0+^kwcjH%jrh)tjI30WRY-1UlOKOd*Cg{0BXdo{I zK(e_U1thevzl44UVu-l52w$ z9L7(uTm`DBHUnDQHQ%Jqf=^IsK1Zspei3b*!||-5t52Pr(<(>}dn;}aNupj`v=851 zN23s&5^#BWg|$9NgE`v$QmfjB948Q~3!9>KVtlgJu=RCBNOBjafeOCF50O>}mm|>2 zJf4SNv{B=D<~tvfH9i;x`$R$E=TWxm!hK`pWE(z#t(HwS&gRI&7=SMzAp{8^NU4M9 z_V$nIY$^f=1@r4yAjYu5UnEx&+J$U~gP)OmOIZ$d7(6=%YJ+=Fs!q2D7v={|b&4ryvRPNaRocIZj{iO^;a{ zJP^18yJ$8fjOxxUI^VLoz;|1SqBW=M-i#6)q#qOn@;`V-xo)(_&Og%(%B$h(attk7 zCfi+9Qo5Pvr>?iM^4&xYavQLMwn>Ml7ex!nZU#nzvVHIxlWW3gpp|vjn~>P!KtSN> z%=0nW)<8v%f2YXh@=r9Ht@}rm6Gvi+Ot2jQ4g;+cv}$7fU1J~=U+47R=01!;>D;f* zvT|}P0s?Y+dc+@Jad}&kQ?H}i->l0(dHzz?nYWk+Tq4L?YWE7~ngeGS&56&p>k|Ph zdL~9jfqm$sPnjEUwo9{0{(*joO?D^SXk;<`y{u$pk*B;9rPjn-oa=3=!TumajoJQ~ ziud8v2=rQPEULJ;IM+;rAALl6YGy`(y%`}V662;siONB7(z1G+4?TR^iJOYEOGZ4% zl(CwCApT4Fybzk$u;tx`S*79iy6el^WDaNpRKkC$*W9yYU&oK zS}L7(^RF&|l9JNc__!iNG62Lo5Z6Jh{Og_hZ@L_bqSIaMS?ok@yjUKLol*p1B{>|t zYycH3!^rb}z%8qEgn_3&Cf*gJ2?d2$(l~2M%OdA+txiqd`q4C{NMPspb2=loj3dmf zt@L}gG0U}KLR>EPHe(gV8E0#r3S|-MXX6tN)mg@b`o!+fmnIGmsrE3N>{piMq|PF6 zh_oe@T7*{oeOl8@arvE6@`?tJa66;fM?H=-nlp^~#}zX3R4ojTN->>gG0o1k`U{QW z7hgt@#3dwf2?>p_@CC-w&OZT)$7Pj{ag3IxeefdwlUk4^%AyaOEYHjM)PIRc9hf>3 z!tGWWqII$O&D1*#w8~97Zs=P~aoFHc)wxdCMC%pTdY!a2JI|nv1pP`l#M7K8&HB`) zyEF_Mv_y<--4(E@rkMG^mmmM;^Nop#iJ6rZ`F|(x|J~RB6tCP$%TUaMJihB$7Z8QH zJ+}GaH*S5_TisFj0!3Q<`f2BDnG8?Y>te|Ail`pw4BU#d9ZHqU5a~3TzsYMs0CH3f z81CqFJCoLvNX^fUXBzsHAEK*Gof4*|6lvg_t{d;xKvREp@G?$Zj+n1|Yf6!#G+@>1 zfT9&|8ArP#A^I6@uMs@SIho7;5k*2=BK4>}!lH0gduJz)uOs8N&Azc!9h z&SZ%$9TIYAUXsVbDO+O5JerI!sg49YyI-2H{G#6QNr1EG4t%oX__QkU zd~N)FN>O}bf~DRC{scOWIyrr4RXEGtfKnocC~wsj-PZdAdO0Fj-NwjlPkcmaaH+wQ zIB!sHs`?Rc{U+(e+>+{7V8V(PuT~^gou;hkn#|L6&JmF4Irnka(-xl*8ccwi$jci;S{=&7 z_xMlMP%~2ROh>3Ire^rhy@VFGUFzs2>%D0F2+#9Nd#sse%zS69l{69N#*UTEozz@XRfKOkfs z-SVAxG(>H33dO>42165@wcYDy;Sb3i)d~fxBjY&*!Kgij5|M*$IA~AJFqKksef=dk z53OU#PjlQR(q{F{NEbiddI~{N<3VsDHud)@~_tO!B)F?MOi&|oICkd#NzwBq?So<{bkwKu3{-0!Lk4QPrv_u&A-=?|At1AOs0%G`#K^!jBF^CUH;HbV2qF z101USZF%{@9jyIt`H}u>KiV!UTCdynSgB>PuAe4UEzqIj}wgl92wfb** zJZ&OG=HZe*A-E`sL4|~AXvK@;>lt1IL3@D(x5h%*swwRLFpnJqv%;us)_4)Z6(Zld z4+hGRDcQf`M(CoTeuG36{`UX!k2V9gM+Z@&R8Z&_(oh&u|Njqves1jaU`z#4)Sw>D z`VUxwuiF}D0o#{(k1V5LW>l8c&H!1l{<9w=vaC5ghXN324iU-_NFj?}eaQaU?&Cx*r1a2E6!40%9NM6~VW(NYQL{Xqbf?}vQh+sTOe`2tjDDCb1^q~5 z=ugN?uAt(2N`r3QZW5^@!?B}6Fs6&7>V;!JeEu)TbjSq*+@twV+8(4 zHOt9vP4y`^*Dr4Wo_x}Q^ao9sC^AVh$ic=Ojr?%Ywg)>Z!ECsuurWUfb97Vjul|s) zZV)uRJJXUdoFt^7ppSzHWCu`C)8I7x|0zp?RwnwR{-cbs5nqWb}6iqF$>kX9^MjqM%?!1buL*sHmK-G{wke@?2@r(9-5L zHe%&Xg?@ti(CmEvt}2@@X(Q=c+1S*Vk1D5mzvXR&|Jm_P#XD>^cT=zL;z3D61E8*s z1BdZSJ6oqeMoS=tGs2$S*k3NvQ>9#!93znS(|Ym^!Q6bCyvUGNFB)H7A^WA7ffWww z<6_l@)20c(ZYlB;6zKgbP>kFcE-st~&8WZssUCgRbaZ5ul`$yd!{8g-u70plaRDk+ zS?xB3K(JwtdZui6uyAl9J@S7A5j0#||2d0(i-zAqJ*j_nr=Pzs zsw;Im$m!(Ukp%1ujwi}HaEhh22Fxrm6+@w7yjuRFwEm6-V4HVfM{hYeVf4e9pE9uF zV1ld-m7MC)`MB&_{dydYpysUEP?9oIb4wXdiE4O!hB*))71Tl(`afD4G!(e^g|NQ@ z@Xxs5-Z#QvJ`gnO%@lL=x_q>YZB8e-Qkyd@e6Xh!PI~wc&H{$q$Sn z89t!r=RbsR)(3&&Wlx026^Z_<3+_xgP@3akBLBmd1DCe2n3zE$7W-T*M%_qg(A^9z zSy|BZA2<2u4GgM(5eoG5zb-zuK=@Su0`2obM1+%bbsHel=J4Q#mHo|B+FkK|>UP@G z?QjI4cif}YG5LB`)X?1f0$zKA9u_mo!lCH;knE=U`L4z(!tW@i;b7}1Z1u4_FS5On zl_F7S+(oy=8e=O4xI5p-gEWrUAH`VlrDQ4F?Ko3QlB2l$Zz@8zNlg>u!$xyWDe1ps z0qmKf$~rfr76$6}?f9=!Q?;*(LRu;u39BC3{3e{JU$2Pe?M*i*%pi{)u^cu6Mw|D~ zUXIWW!C#fR9WeUhlWo&8A%w*wPO>*N*-h7a;VkOC|7E0YPv) zC)Me2(hbWlG0$`#R^ixq$&yFrq%`$`o>$1Uqk8lgscEw}7bd z&>M~SZA-&y)c-H&^_PM@^2G{v43*2@!$)aC;gQ9Pw)~67hY`AR$s{Oc_1$Y?>QEc9 zIQUTIec{I6qEn_|c}C8&ULz3D&kwr=Q%_P;&vQiMnS|7%~wc$AfW=#7fq_p+?n+p~+k2DC zxL$2Pi0^cL^rJ)f;%IFP&5Pqt&54;GLQ{3`g`F&k!qR5kXcues8KfKcr(POA4_9C1 zwTfT?)5nAPq1eJnPQrzq%t23GKz%yZy8o&)zk8Dh)4IqAs=3amsfXyPI%fKw1}6H! zkwP9H9d8kS&+@VI719-7WP_u|EYR`VvrPSZuC3hF&Y!!SK6!&B6cVC2rgCeJG5MN| z5R_2MYNu~nuGy_IFe0|0NU#*&1Ih|3e$@SW?7udq)tNr;eM(*&&2*A?L;#6^+t<9X z{x7XrrBIX2*qJKr13;w24bWmfMHU6HT%#MW6tSF1!Yv?M#bm{=5@@z1a7%M20`2$J zpS4*+D`m+Pu2$^#Y_}uKN2y92roI&3pj4i$z-4Ma#%xg=zm;DUTe7E=?;HG`nq9G7 zS*y)}Qb~dWLYZy{fJf=ypYNM{s->53u8T=t@}6M4!CM z?o-Jc+*ZMt9BWQdWyT7;fO^F^*X^x8I-!OTA@guNJL8A>yXLTi6dfRly^yTW=AF#9 z{Q&Tuq%yEO(MjGEhwcOoFWfVg0aVkKVKBv>_X2yH9#LrB}%s-kB;b zsr5lk#w~tewKl7=fRcT+*`|Xv0b%6^FVS9j=AqWM2LoCkLvzV@243P zmhRr7KRe(4!5zP+u<|x1SSfwtEHnK0z2sHr#%Y5Gg#?ATOJnQRCa%*LJTC`o!SIT z%?VNcB)3OXQGp7G%KJ>l({3j8+M}GTFM4 zt^dHY+ceD-c~ZoRM?2Pe_8U?7awfXhC4JYNz~%cIlJa&|6!~;W=8|+3upDQl+rEV? zv#KpVxc(PN-CMIaaN&ICOibc2Hd_?mq}oZInZ9kQX60pup)xp(k#9VK9x7)#&-tLn zZQhY?QE3$UlYoP2^ubQzi{goFaiAlO<^##Y`<@HON!ve|?PvA!SJjW17k16vO3guG zH>!qK%lK_;a;hed4kqgpJBFr;-%lW1q0*Bk^tbNw9zVT)v$jXNIRG9A0SI{<*>Laa z(D>R?TAsfAT{bTD@}5yx3w@%xTK(Czx4Dp3x-%G9edCI6!njPgQ>(G+b<}>CbrR8Z>iGv+9$@D+kbDC7B(K

i23%av`Lsr45s;uibJ6{YHml#$Mn4i3Cp=}wW1%Jw((a*i7V z!(92zx#+GMoJJ&!b@G%Hq0h5FI3oJox!i9IuwOv4R4W#J4~5PT6l+{KH3W&`f zzy4Z~w*MWTxuU%lA4VaUZG5zMma>p%f|$Ibiua3}_gs>ZenZ$94{W~u$!#BtA$eX^>4%9) z;EJQoMR$L;qBd$)uT7d{PN9L#5N)yo-&LL~+lN#McE=yj-)trZw(;rc>ZE0?gSpDM zYXW8ZROCmpV%Sc3aR%x2uLt$_caMbs#oMkQx=<`K8jxVBdYoH`HQ<)Jrt1(r4XT+J zJTRQBQW?Xf?=Xe)+M?xbxMkHq&FCbAy^3VTS#(6g49v;uxSg)r#|L$4b)EYWq`TO> z$P?id10Z_4aAevuZ6PF?6|SUx@g>V2J<#$g?P}~sG{p>SK03>ce>m_Ss6XyiH*-_6 z?d&%?NwdR?Ulj%GqTI!w^`C~1J8T=ogSHmY=4Ir{gYA&-iSwrG&)-iHeR}P7sx`0u z_4ih}tR=sJV8er|1SLXkg17zIa=RT_ymb2G@pD+*!b?A9s&RAy=qhgVIwZF#_{#<96lt{trl)gPEtUkv@3isgXCYiyMyC-$^hloy~PkjZR~(m+NVR zCY_==JUFLvQq)Fvjn0Sp@2R;Iu;~q{wurnoR8kd%In+v3i|ySSe%#r!M$6D3eh(hj zbP0;#xNv^=h3f~i-LCcNsZV{m6+#5Dyc`xv%0kG>r@TJGSG0^t=N4>jXPEiw4{jiK zd%b6$q$;f+s^5Q3ijD?XA0VcOu^N4~iEpzcV=<=5r_F~RIdR}na4HX&WPCN^o6hZB zrr$Zb!H{=zP32iO6gY${`kx#e-F%yM35w*6x1kJ9tPaxCUrg}qAd)sA5Ea+}a22ga zRGl$HaRlD_EOzFq=&ZUu$qh9LoifLaw%T(jmKh#davH>!Z(pJ@m)$(`a=GPGw0BSF z5)^>R%Tj3Au_3wHab(?`Zxy)8u{)pGy1wdyoWwDCc?T!)YGmBf$`+Y|_dhuuM``*K zo5!EVc?tBAM+o*$D=yN{4z+A~^qmTqG~nY{7WWr zNbdUDtbX<@scbz{q=28XXY@8|VI*u}sr{OxWAfJ2l#z+stz+M8)6pkdmWhBq*Kg&T z_a%*{gY-`A?7I6vTgc<8uuT(F>%lvG>bHk;~fE=VVBj8V!Csl-(%%?gs*ve+tzToiZf3_5W!~M%49rnI`!T7Mg!JSGkRoHzs zj3X$i1;X>$j6ODgLJ3C(w~k|AXjzJZQiP{~9tM@DhMVUj21W*~ZjYZRES;0menKtM zPH1%(X&f09{e%?Pp()t$OUI(md+4wj!z54F-}9j68pWigOG{p=ZZ0{~fo0J)-U78l zT73y^+Zu_Dx~PG&_rV3v5^znH#Xab<17*ST3QU#_Q4VRI7FAaoe{%tTSUgG^?%&s` zarb|5Kk`JKAfmL8(rdjq>t6n30tyJzX=>m1;YJRg2 zF9!V=!?5KpwEzn7D2UP=gVaTrOs*+mb4-hUN=42_m(4){mJ;^6T{uzOv1qtmzk$Kl z%V5W4wC9c4`@28v*1pO|Go)^a<9DW^dE!I}_&(lVqz1Js7=@0?>-6!kjE)r-&3S}f zMkE{u;1zRFo9^1!Xh|mCCaVvI_yo$J%z$NiY)#C(wL%^l`9XH72)eiLU-R^Y7vDYOGpBROTOF;S>9?1wFcaXr zbjqYNVl@JhLD(>oocFpCcP%$=jJa$J>XJHfsHe1af)+NLq1&^BZwVsozR@M2lWH}G zpLLF$_JVI6e7)a)6Z|OSyc;f^F(NFxf#d4&TArIJE*jnRT)EhAt5U$e9cwgD_VuFa z76a|)0pZ@^cSQ$v(~ZWspR|e1FQIei7+(4{PFg09i|APk_G?XGmAA+Ts&o=O#v2IBD@UH_I1?6bnKkqF0ab zFACssBd)AtFed2d@EBSRBO_qgUuPpa7Q#^Og2ytvIx7h2j>S;0RCpupE!@0WF_x({L31g+F%}^>W(2NDj(vF&G^VXVQdBKj%b`i7$+xEb3t`0;oEaWK z>Rbu44So_H2ORBp{`c1Xii0C36BhaxgBXJD3;agooMnymzVI(XISz8lt)MCtr1#Zl z!kF|TTV2vpi%j%at_iD!eg-uLrC~YoqY8-juR`Z!K;ge-erxQ&(UP)tyaE?6jL#C2 zxZJF##A0gH|3E!cX-HTTx_LaD5Ndj1_O)hp1Y28w*hv>IC2)h)xFzp;^_uGT+}Ul>|LvQX^8xemgfb;%0T4H_3lwo}ypIMSXSc3@c`zuKkf#Ni2l43q$13QS( z@||oyixRmBUm}gFx`cJ+l7wXO9*LVFC`9`z#*$_`-6LZRygXq)=_U0784fn^v)A~A zJsR_9&UE|DlNG%Ytp%N)Gm&hpmkymx9|zaBNSk>=6oP57&O0!E1=LJ%VQu&q zpqkC2Vkj}6<8`?`XNT28rWms{GRQ*lZvmG2x0!nk*$K+bQvz}^a|akLNr9mY(iXGl zdcx`j<~#Z7AytCdn8w@olc}2wE_~dTcH~yf$3IgI%=;u3zvjoC5kws2o^U4|kCPbb zGkg!dA)KzP^!6Sno&^zGnF!STD=!Yu+4#w%To*ortExOTnjqRr#pL6%eeGqAGN)nA zofjJ7p^?A*?qd9u;OgV%K$3s)C1w2gnLhl%D5o+Sj4w@9acp{-mc34vS*cP#N#}(5 zteEv)&zhl*DgO_nsFK*Kfe6&{xU6Se_R=TamIJ$_bW+*2;DaKy3UA3d6!)zcT#;sM zB0Ch%nGOwh`U-m}j|pEqh`&zTBC$>({dle%~GWF!a-$yRFM2|S=Sozu;9aToj;9V|6K z#)lJ8aLj9`N#O5-2nwJ2hBfp=7GY?)l9Qmll0aDh6>DgtZ>c(|ywyNDq-7xEj#H$w zc&lbZ_H2#B_dKh|{i*5DU^UYDRhR&6j88v>;@dI`n+0mG=AjTkkVcKKd+5Nc0by%& zLf14gffRc3&8aK_Lmq2xMH-jvQl3{x7D@VQ*HLg1)2g9T3V(b=aYBjs4kDzqy__*SW7TvGBqlwH882+X<-vS^_h3N+=SHsvWFYB3r z{1Ja3MUB6WxjGVzr)e@#p!BPqeIbAYih<~F%}w;zWwzLEzzW3EOJ^*&iBKQCMjX+emnoEhse@{!*RNwycO_^T=wT|1V1ps3=#`6_Y`CTxyrfP`Z%XRnm>WzPs zqoAhNXHEKh8_joP?9tKD|Ncrwgo+6dU7gGh%zydKs6$$V+tuma(o=d!2qGZQY1VZM z?wf2G;Lb4oU)dFZ9&VLIgJ@KN|CfD~06@2y0uZ0^Df|Y2w=O+iRsr?Y&)b-t^J;5r zo$pV&0Q>@^6>Gdd|M2&1ne>AQFu{^v_0;8teZc>U$zcJY4b?ZBDP?m9njipa{pxVx z7q~c3WIy>`M;3tO<>BG^OOn6CdBXcm1bch?Qm-dZ02&+sOW;2r76IU1ztA8i=;OgE z%ni)joj!^IUWdHDN z`C)K4*iXc-6s5kVpaG$iX{KfKx88h<>nnEwHhsY4wb$9`*9jW5TUrdDBO z*A{m2l~?*-q929*5=-RgK%u?+<5>m(YaZR$C(0t=0Oy+}`8wRVESK#meL%T>IQOF3$Q@U@k*D}67UIB0{ zM&{@Hfs#`|QvxYbzbR9HglC^N-AkJ8>5UkXc2XtS{7?m=qHYlzpK2NT^!>OCm#fUC zhg94;?<`nD%9`fy5K0n)~vsZ5cLREyILER2wsL|E^a`G48OVWhmbC zd-AAD&S3{eFdkkMe3`gkVwcFb63F8MpAFC$sv4h7jvfb=Fud@U{y8$$7WFX**r4zQ ze)7$@{&)e%ZuKm?Oq&JgAGfQ=z?Z6;V?q$nS1R~jV6Q)x0?`@{qtW`98}~=0 zfv5u5&JEPAKhpB{)%6{I6mS-p+rB@`2)evDh2XtCtlCkeJoN20-3j_WrXHIXFn!wv4Vn_<#K`duu5^@I}OGZ81PoG$F=ttw%kB!~P$ zgacf~ZKFQ$xpl50N<*_(wLkXzj0~L>Wm42y*T`Y9!9Nqo!pKmQp7}~vBXCeO;e9!TB6!;1c)3m3&*HOBTtR@90&WEr;`o`NUGRY?LAy?sKruoL8u z!Ay&S7)V|zm{N`Xq>!clsYbSlKNP4pZm-vNRcVy`Ru&a%$|EA-wY|qZIbuu;bn$QL@(2Vp zd0>nsIeAQjRizPKDc))*5T`BDY^kLoU_tPL`}D}x55Csa#66a_yls7WXs1^TW0G3# zt^^IrbI6}k#aJMqqjqtB=tzncQw|_Z$@R&^ymXM$m=c>VjfzhZiQa9=rdoAW`xr(e zz~I2{0K@!jAnBd3C4plF0&~!{tErhqD|%*> zZT)NeN5|K)ga>JYdcPHkvn&EvYOdFiC9q(Pw60~jZM8(J9#cE1uP~lQGGg2putsr8 zzCKQ+;jt`!*}Fz1rf}ZHDly=qLG<+o{GX>6BMYCbbNCyXReF;-lIL(4zH)bjb&*oC zvfBJ|ik zz$ZsDqY4bInM0x!c_TEwlS)O?iVv5ru-|ak9$+i)n^6=ViwWB6K!KYw-s%WVz;TUC zPKrSP-B_2^Uxo(o^jWmDZ_H?(q)sf!GUwVAkJ39zQZea*lR}%~a&%E6+&xgY{C+2f zSflJMRUnZQ8NC|J+@Wz5TJaS;Kg4^KiKCEiKvfWXP16OHhJ+sl|8RG?oXU;RVW7u6 zT#Y#T0hBww&^qWb0gWfl-w~7hk(VEvVRR^pIOSVa*cS^?Y>nPI=dN-4-2U{wZ|xxM z$1%y>2VR?e!}+I7!ojzxuj=9yXN9YcN@ABFY%yd9qf{ zW?Wp+HxnwXbBzB?llowzpSH?0?-ZxdHMOe)$y1b%?>iTMI0cqcZ7?6?qLrFV!tW#c znJb?R2oZ0j%eqJ9yJ(W_?&&SnQqs@a`w0qopCBEsLuN3=9loU+`k72OLGt2@awU3< z76s5E|Cs5^4KyFsdNva23AVxZ0MVaE2H9A+bs`)KyVFi!3pk-2ZDPD2^yf$;O_43?!g1W-K}wVx5nLFr<0SM@18re z=GXkVvsS}uimvLaS0CGZ;|4<5CHyn@cW6`&{a_^NECLf9PJPkO_99U7bPL7=Uyl#|lfy-NH=~=X4@njJnZyx? zjaoGXBLt)&cI9qHCT)1XBOmZMeg9-VIh-ZG*rSs#blI-#w3okbE?fcWR&=lctDv6WLJ zfmi~aed&?1?0iI}-$v7nh?;u64i#borjZFWNV^O&UwGsW*0o8!1`t1LS$WYxXz)LZ6U+lnU=5Q+uAl+Gif~AbF=aRwmh9h z4{!A9TxE#1@@{+Ssyd|rBWu=dMRmU3i-czt6M+t0F^Q=n-`o8|T7&UKZt0+r-a#QT zcmW|v+vo&wF=yB#5gaSYzC_-^#4ntt$zJ)TX=UxInG(Jz7M`x}B!fX>p^4~1TmZV+ ziH@jdhN`;`?M-^3|Ew%Ii887BJ56!#w3rOLdce#{b)Q8a+=}kcBLF+hR+*>Qu1O7{!Z9 zX}k6~;oFy!*?jWW=#Ajsd)`~qd*sxx%9pHnteO0g& zlw=Ixwv#y6P(p-Sxq^X~cs$kjCW#IvjH(|^H@a`V}T_CI}1XRaNs(#L~LI+vcZQ2& zgo1YRY8q15LL*-$fCjw!4X}9@xR<~>-QQoZ$8_Kox=_p{aC58v9lP_s#^+#6PaNra zr_=uGJAR=gG!_;IaTkp92o!{mRwOLMka?^QAvz#VF?C|(G=>4PjHIQacOmsj3WwYR z$_?T2hd?y6652ygRLc}!DJv~0a^L$L2-Zh=k|$g^(mmaoJFCxsH9lts9+)Crm1I%s zXv;!)Z(kay)!KnCJ&mm*wN!+`nFeYd24`Yg@+Hgt=WfJ08tJ9_OtRepb?(lL;lp1J zj#&doHVIRRu1DXf@LpHgteC6u@k}4E=JA@*XfFH5G}-0C7IG<5HQhhL{Hh zFryV`RrHOdW|fEb^N5NnhfY>0N>!wa{CMO1fLnZitRpifQ7A|9jjJ$<%^06+h%xJ- z-q7fuC8W+ENrobSJ2La#d7pt=HmsxpELvW3E}lZ4yktiqWV{z7Crzf3`J+0pppoxo zJI@R}T$9Xm@&7+rol$q*tc|{60;vcL#Uyi{OsjtS~|zCAmJc%G*6IYD|u~QSY>(xz>!9 zvW!at&M22_K7prPV3ln^U$0bds{xZbO58+tMEB?$77zMscWd0`#2EV8;V2I20;8ku z)w74kCBjm8%auX){+UDa!I#2NJo2BwT7VEZVw!K{IZ?RBZKj_T9jG_HS;D@Q2LcxD_$K=_wH=oTTX;AJHV* z;6`Tz3}unxddh7F4LzJ~k934RcQO&zd5Pq|{&afY(ORO9Xwf5TP#)G%g?gUxY=xsr zHOwkZTxh)ah6Po620QiYuVrm4XaxYd6!X{LUOay$w5K6rXr2w(3kAIWFZ<`_ zMu&6BCXZAHY@$#QmgKPbh2?cA;od%^qJLlRVjFSB<)FX}P|e)iL3uk6F*-}|B6+q( z2hUjx&RCdJ@^c)t8As~(@9mhILr7`Eg@;8~u)7z&EZ)FEhoY^`BLIH5BR6P9v1@Az zy+NISayLnMq zIX&65JU#Z+3RYPN^347P`hq`ZjBQ6b>LOpU zA4y{R+g~UxPYCUdPLTgz;~&!1u(0dAW(d=hkU6LH%H@P)PX7B4=pFss+^l$lch*lU z>7|*tL)jLMa>UU(B#4u$L(0Q{DL(fZQhZrm;UG=7t#lF(ag3|)XO#x_c?crP>Z4{} z&hO1meeU62QX-y}qr@r954V|*ILh%ytS&RrMckeuT9zfUn4ax5Sxd6IcGI&Z(Hruw zcJ$0ir47x%$t>r3M*&;z8IXMTWir+#=%|L=)cj}Q>DR7MCzgb1oy1cedI@Lzeenh- z{cZ1ROl@4$SEvg#yZiXFANe!jPhVt+1!Klu!he4Sn-i7%Ny7Qa8Z!H6xk(bGa&lAS z=p4q^Lwg91$kAYN60)P&4s{3BZuN4}9YEf<6OI=V z@+Z^#Jqgu`{f5HS)u_@f0q}jz#jVnZ4DHOy%$`(t~@eEl1d~O?->I z{#kHfeA@#Q92f^?Bi<-i-;O-4vFZ*?5$KAr3BFlbVFJ%A&|}HB`jiTMHMFsLVes3? z$fuzStJ|GHHwMVA+*RX(nU$qjk}PG@#5z*zX$;tWf})d?jY)6B6*=HUF%6knxu8M{ zF?RTdqCjv3$4EM)b^Z{!(*Jv2L!%&DJmReR*ewFW+X>;X>UodKFCr}b;`tSXT2q|F zEtWyOMnO%jpR45*8A56M_YgU8e{HINsg?fVanKIu8Xo}MK3zF)8fnxrgJ++E3m*~V;-y*$RTlfWVrj@YxD#l?b!5>5EU z;px$}+wd%`mo6{GhgUVvD@>)OD}61=Jo63N0D3`g(cIXG#KEvyt)563YLB-8ARz|_ zF5(EdjFFz(Xc;P_uV24?`}Wu0SSv)?rs}wO!}P56!`Bpzjoiw4BjVbNjujS97@t3L zdIUp4@rH)fs;a8E{z4XIXiwwOr=(LRn%U0C5Z=#t5i%jNs9$J7-w{N4;!5*ds@H7B z1>DY|;o5`&o0M#=fzeNXIZnUfhrWV{bmTro1m>Mz6onU}l>?`zb*M32=_fir%_dyT z*}RfT4vkB1_v`%y^SpU(4hk+TL6{!v`cu*#c| zHh*s4xFQMt4QkZnenFcfml5P~0O)9R7^^pD7@JqaNF$o-!CF{YcyjCduZwlm|HQ)WQ2GNEK745Reu?O~lNs>U*f=XY+ov^l zaZ%%_>0HhqnZVyD`TumDs9b&wU1?J^i*9$B?XDuH>FhfS46y6j{bMK*Z+yMYc63aP z4-ogio%$0)xTBwEq^O}W_!D;u`T1K7{x`~y)U_e;U#I*$KKc*P@(*qb&6oIp$Aez{ z?{vMH0e|ad|GbK2CfgrsS8&rG3>V~rjJEOmQ8HA&L0#?;rBju7QHwut;GZARgYeX! zIzpO~>K{eoRP2zhSAN2f%!YH!A7YkQ85BZG4ckrzpODu}>O%^3s67g2gN)PNfSCFT zL-`#GgFl$e^0epK#ja)7*F_7yG>76u;bDg!2oyyk0FF&!5{cRz>};sU5QJAyf&yc< z<~P5`cG@qA@rA?gO=}=7Y;)HiFrIIOVZgp>r5@V?ijoaIvtN{h3~ASfd^A$EmG-rO zVt$Y0c>j?sujFSX)&>iF5+*qlGmc}^fwem)2u#VQMuJYA%e040pHtVG+t42*?rcEP zAs1-JQRaRq_u~CqJk`h!NvH?{A3-JWE$9@LiSdAx?Xb=vYDtC02jZtwFXWAQl&q?6 zsF~}$ju+cMR2=$Vw78qS_L6{esXd^mHAPyNs${-#9P?k|JDpvSd9p_rbw|yhgfK#1 z-v@Q+P+-9S;Z^{j53|eC4|MF>>LgP&|3rHKy+TFAy|F)?@R4q=h6YRZ^A=z4?X#H^sPzh%_ z#Q?=>-8|Z-x|YcqK0U3lvM4%k8=d@d(9^x%G0l}!$PnaijGrRu$hs@|Ep$-{rxhfJ zVanZ~?iK(~m6mf!tD`MROdQaj^g=W{$z$THTrlC1$aa)D!q!r!cmIoJ%;T-zcA`nH zYV>M`hg~P6w@h!ePZ?9f@1PHOty8r-gw*V{VB=n|Sy|TTC1y`G0?fBq9N$4{!ts-F z(5u!ZLn%L!uRM03HNk3&b5~iv5p=#mwwEL5gJCk1?%pxw>29Cr;n77Izr*_Yc@d5a zXnbX!S#!#4L@*Nr{KP`PuzC~qp0Ln@3h3Osr!K52L=|szVg$(uLfYCA@VhmKAyD|{ zq5rfk`mw?DFppgi5P(fzGMOFmY&ppCa zuBC(yL%j5^=9Ek6ScE~0d;7`oNcG;Wt^106VU3PA2_Xd!$ImfN1*zDmoRxdI0ZwM> zh)}H=mg@XH-sT32H?ro5u|qE5MeAd5sNjo+uNG$Q{X{*=tya)(+#kI$r|6uvZuJR+ z8YHbczaC03@2u_D8%;Ae!ERH{wU;QTZqi!sVvWK5Z{-$*@Q8w`2Wbyt7*N(#w7oI|QKw@1gtX#fd8VsQ)ILmUw z*1V-8P$L7;)Opql|GH!=$xr$_m+KAu#H5je{TPl4bFn8moc+Y)Sldx?N*^t;SU(t< zufmZ-QXbzRCv7V+1ERo$pMW&HG11SA^n1Jt^8N$-6;P|n4kA)V{lIF^$*t;!$ITa$ z?{M(VXeeTew`h7N2{eK8E+yOzPU27I4S0)o#Eq&x>b17EvcY-i@#Caxl=^i zH=|>9zNb|ay^;E~{d{CHZ7WU(Eymk5td{G24=@QQ5^%5+UZxmVwAGs|PgAi|vK%uN zd!MI0Saf7Ok|pisj9hjEuhikZ=)?AeJgf7=HLk}PUwpAgKuAerQi%0BpTi3FU}nNu z9Cx;5cX;Jm?RA;yx{v|VaOZCV15#JQ|Frjf(0h~F#pD4Fa_Lj2CM05BN>lksx*(_rHT-jm&_S~&84&%LSI z{*SpYHi_>HhzOR=*WC^~!)MA&?>|tz#?n2zufGr7ND(APne8M43S0RUc?;|Oh!`ib zk4}$)EoIyclXb)W7%e?LS$dL@kiS|(#{O5DIG4@w(iHCN+Vd%H-B&8EFw|WWt%DvY zY2AC915PHpY)y|BJ`{^{GWBa>B`tN{SkAW92l@->TrsTzeo(h~xNW^jm5Fy}&s(`+ zm!Wvzt_lrDH-V;QK)sPyLl<5www}UeoX1uL78AAQ8!&)7%X$YJUBv7;0BQRP#j)PL9**H zJ;aJ8fQ?nVXvwE|FL> z|D_WH4~CqIoN9S640)u&;8wS(x(rhbjFvZ47n}4^iksH7~0s zG?0u7e(%_MwE8a?t6&;jx5xI4(eF9aRVLKqHE1pR zl#wL&5mBYyl^f1HRmSOCWFd>{gDD6Fg@QmbXX8VBid<1;)TTUt=ez>^#~x&$!e<`5 zOvq}^JxFaNaKahE|G;CUTZqeJil(|toIE35ury?p21b)y*ovK=<{>P2JXLXRYqRFD zQoEIDzCvMS7}<)La?>M^SiZ*UL!ZDEta|0)3`>jb-f}(YzPvtQc!lgSv@zH7omWx( zaCwUF`6N3B%Ok~>RJU_;@iGn}{&A0sufdj%yoDVP&d_A3(buoEtNzR+;p6IGNN`@M zWQ7J=-Qb~xxla?Y`iyccvT!VHQp$i}WZWh?>^wSja=|IYstHh=7QmZnkuwnQ@^X^6 zSbt=%i|?w94vFYgKREUG80+Zy$2J6mq6rr+-oh0D7p=FotoFUt>+RZ$7aLYNpZ2D2 zFW73z9L>ljHDbe40v+!0)4zO@-|{Tq$W`;@A((ZmN@PEry#4Du6-`j{` zpE1oHj_VO++h#Qzhmc#}7@2>~=It<|<}@?dFaJz7M$_J1+|*R$RbfQl$8zH-X`o5_ zSi!4ucg5KtjGi_2YU$GCW_QIs1INDB%6u&qpdo8lg_b)RXeB`(d=>phR#v+rdA*5CnGprCAle$}vg@w9a_#=O`<^eul2~N_QvvojcyPMi+ z!*7Va#$lWY3H{(`W40aoz{r2C1G1Ay(`8ar`Vbz_d29zw1)OYL=&N+WP=(RA?-=Lv{<9vw_-8#FHh@Dt@0M2Y%W%cf4Qa01 z1rzfH-Ga2X+Ho}3dpBNQ&=qN`wcgiVYiqUGy{8qx)G4?{qO&?*t3*Xx|f9V`ABmOT|Q^SY!%NWEbO-Hc13YC-?vxe22lc1jS(N_!S;FOJ)E^HBg|K-x=JXXI0nB`69M{he zn%&5~S(Wwa-S*T*K`>Fi5pT$)oSvSvPO_o1Y2^H}1MFkm$Kqs&6WgQgnno-mdF@eZ z!@rDJkZ4HkI+zRp47fRnAg5=M%kkxw~>?=H;S*#gb;zI6S%I3%WM!1JV5Hq^dc~jP&poleZ2#h%s zVvKARDvxQhTP>-f|CVrp=s{3h-aI}XlfUZ7Utp+R6b1s{G{3Ou_I;`$3dVdbl(cAZ3jOblmY2wPio&#B|n!#0RsvB1&|h+C~}17CgA;7RmWf*Z^6 zNv$Z?;Iv?TE$Sx8v?sq}i<%xw-wzufx#_EROd*+J;)BT(z&m>J9&weSM2T^K5`eoK@o4XN_>3MJGCro-#D z*Ju82jjff)6-8g$*TW)%nIP!%1TD1_v(^z8-?Ivp>lj8ON#{2nysN=)f&bEr9zp8< zqY|qxFn10R4IiBs5j~b*Ikfp{qYhanAfTuN3W51B{Wk$lx7xZhP?9$73lpHwh8GZ9urt>B`T_7~c z>ZS4suGfl@jL4Ef1u8>RQwbpoA5UixwlWzATUjtfo2akGY`99V4+Bycwu_vWmL@`m?kYg?w{P2_ zN{9jnQEPzsjQ=@i7~hu>0;Zs}Kyq-sx2Ut$fM62-jvvv(6oFUibz}+M1DY7%O-H?n5Mc+b*+&2#B4CzpcH2iVQwQ9{Zc40=L_v zS<$?!r=YAS5K~N>b4tN&6}RsliPV`J`_Yd~AH>iF`4u7Zo#U^1kHczY2We<&w^+x9SE_7F|O$aukMC#@#$u}3y5+|_}-kt!!;T?*_9$M;0783`;l zrTTK4f4Pdli>^C9>{9R;t6cA$)6Z#$^Yu|n=$_O`C#sf3d`dw*$?wxX(^u9pQym0D z_ec;3rVL0os#KNwqcAe3E5XXuhdQO(9n30=Q^aqq*RdfcvR@-CTFjIme*Kqf%FlUV zgxx$C#gn*y+~ftU7ohh+O}wd)N_1K>6Y+lT+HX`VzlC|EW_HoR>zNT8$>RC}Pia;Z z>sq+w?qy_SSmwShoQy%GSu06{OBPiHg_$rD5`(++MnHS@^rfXL)hv8RJL#A&ay&t6 zk0dL057-)r9~tNeFCGKnC)Pm0xt~T=XeL9;gF|KO01oY3b|O0gFT^heKS87+|DLjVaRcMhBiCe9o$YW9f~KBmu~J{hTMr&JMkrTO2liAgt%GgS}0zJIEI#vMwpdmUi<|(F!** zR2<%qjKUN$D1RGSfBUv0Y$2N1JIb%Z?q}zcr9N0`u|CQL1_0C?eDi8ZpB7!48r9&> z;G3W{I=`CFN=v)&vJmX-=u8xHokTE^P1r}XT4UF0BoCAxyM<;`^LQaKHawxcB_g<; zxi94IlVVugD)mUo!one@>#orsDc?vttcsQ0wlGf>zHY&9>gu|A6wtX$aWau+gYFK^ zWoGl_wU5;w5gb_HrQhxYdQMY+H>l>M8IRS!DbV?1VJVG&NbTOm6jG7tWJ4=kb3ZsJob?H zbn%piTVqV9+a?4bDCr{L`Uv;~m_|qjqe&hJ_8Vg!++kZnpOl7@24deh&DMB{MlStW zUXA18!d$0_nLOiOfVG7ZlZh1s@uO{lnZ%;69 zOoz|K!v$k6JFRnnHWe7;E?62K?q)&CaE1;)ct1&E@vUZywCAd5J!onFFC6Hb$IuH6 z3=6@Lk9KJ{2;DZfUFW-B`P-$@nTd!MCpwx=J7R1T^#@yxU@Ki+R4dOYKi5FGWR62* zaM@zN1YB85Erm}N5dvc8RFRvm78UW;8g7{<5~S3u1uYm*|TnMXob_P^Jye+>M3eXBPJ#0Gsa5u*y+agytKEYr|SX=YU%#A|6Y%i8B)^I_4A z9Rq9zEol?NS%*+B%$E0ao^SLHVR)my8Jvd{EzQeCGg4`BBxw(sPZ!Pg!!P)8Yx6+g zHCQ%qGc4Mt<_$AjS@2#45WG5V3{~Ofttnnl8*&N6Gje1TXSTIQH-BdIS4Q#;KNNu< zrRM!>jCqNL3G^1WLT8t8UL%3I7xUW`4aLgqHOCYm0sULtfR2rep>O&l`!EolIq=yA ztFayw@_xGhW?^wvml+B%d{mlN=gH~orb}aTw5`$p8P)nsd;*_j6%>#~Qg5Kp+`-bQ z@4p+NV>_yAXm}ekKYHO6!jd?vqJn4^L^CCfOG}%1l~ensiN^1{{af zT7P2^F;aU=+rrGr3D)M7l#|k4kjxsR83U|(iZF{s3ks77Ik1Zj!kVV&v3r3nb&}j@ zdoLo_nDnazBPfp9Fb=D_HAd$<+o?h_vnS^49RbV3#irb;{W~q>c(G2N(SxrRw2VPD z+|(%KaqM&*%klO+>zve--V9YW4r_0RDMQ3aCDxx%Oyw%zKXHvG+pdoE!zTZ0Wq}d8 zX{y+HnYWqs%@ymBbrE8g<86xAB+h?Px2l3EYmAuC%V&Yqvbf5{Tz4+aRa@aOl*1UF zE2s#>X1(KTEov$XN7%nRmmp#_%aq*@R zwX(DkGldu>KcoU(*NS*$q@@8<#7T>cxV=V~!fMqHQ0579%$ll#g2;ru* zJ=Tx+>RgV#b1qsBsc@sPpPjX%Re!exHj7gm(Dy0_Y$Ix9S|h1ug^{iloQ8c5rED(X zG*^Hto4PebB2#cIivrs|h&SBJ7lK)h%-(Zv+0unMbRbA<*x#nZI&=Ijo5LHOMtgkn z?U~PllR=ljUib_;o}h1m9IBlO{&zvN98lum>RYZM-uUq);x(DP2~ZiHvZC{iK0XfD z*sg8V;WZlHm>BzXZxp1^CHFSRY^mq;h~lW}1;RP&Zi$v0FZJy%SK(hhCD{~~W35@C zwrqppAnN~9d*m5moH&&eHjRKu8=n;lgZm}tJ%Ovt$Yg221J3{z_TWuhaM!AX=v?Ii zvOP*moDr9YPjEaYB=2ATpt@m@8wIF{&V_4$-=6$1!x@xRt06w0PE{c#MI&Ps1!y<& zFD2n}-7EzttO&`eavL2^(W_VbXfMk=skE28@(OO3X)TH#j#~gMqi<THj6ly=sV3GBs*l%cHnpOKr=BTdP^GnYf7BarDmIeO^<3h;c z(TJoGXzRRsL7JObR$to*-i$YWJq`3Pc23AvAuvMd-uit15R^(x0-_%N)+Ub%YGWsY zAe9L7f`quHq-&#kWpnekNNak!#^Uw?SYfBYJ{3fln!`LI9Jv9Ap+F&AOt^SKHPTd( zjJaf$bqLX_t6rDvBV(JA99aHj$}=gPQ=GmOpe8AaipF0NZ?;r$F@%6-G~e@MgkHzc z2CIqo;uL?9WLE48-LZ9aQi_<%k`T%i2Ra+lq8Ghu5bPC^X-7L z&fXpwM$qb|^Y1?X8RBl&M#~rpnA!g$I5`Me3r{5ii!GXa83X#brKtkD*C1&puX6ej zJ=Whw{RuC`j|dWZ5diw_NuMMcjM14!pW;RVyh&Z>>otC=Uz(Yj{g<4DP#e1}Bu5G1 zXI1$l0suJ+!u)9oJHaj~Ee$%fhd{Ujd)EKnZ(W!1AYzBAj?J2j=3IG3e;z#VXR^>W z5Cm@9zg)9#EI0dYCPIOjnPo92*#G!7=Reb^0^Yw8LI2vqT%Z1gp#Jt}e*XS{lbQST zHT*LF*jInv>;V5^>HPC4VbuTbk_e$W|9^ICklYCPeRLlM1l#mesrD4`kN)Uyn+XR6 zqAds@gmfYVZxpXTmh!V#h9hODb|C{XNd{7A5d5b-e)R@KyXF)CaX54G^42T?I}AUw zqlSTE*^puMM-uRlHD(E{q(_`!hllmKpn|ldFI(rj6-E#Y1jhI06TjYMpp1RRTBzCm z`%-0phRbi?;>H#t3%RWKKe%&#Ns~sOf)+hyQ0F=WxF!0Sw#Q9M^9%Xl^4o7GDnc4P z9&uH1o*m5b6Z&qgLIaK(QY5DA79HJQLb+!Es(++!c?}UiUG1Msvi#1v{5>drTX7+= zFJDc)g*V{`fMAcKp+(C}i3bR1m2Y^f7W*{h_z@Mw^)jPQUkuMY=DfSc46>L!1S!;3 zCR-Z8!4Hc@YwxpMP2WMvng2x7el`3_!cV>W(f!mMLmalRz1oTZeP$nV6mqMBQq^h$ z4x9+4Opn4orzq~&#+G}gfA!VhseFJu5?fr>6J8aG`5P=bv|2T%{W)q0q?YL2^%A)G9tiF(@`>iYrd)F?P zJ9dJWqGV0LH6>{BP^Bwc-^A$EEgrWURqj|?lT%hAlZO=s@ss7L0$t-YiN8M(VR_aU|fG>|L$NS zAlNkHUy+hglb_4SFdVNW{bWZwC~U)EGiI++Ld=J7z9t@8oL9n9%J-u5sL-LFFiR>k@4KqjU&yNrASGCk$6b$0_WsYFChnw=(IjG6KJMMDRf{&O_ z*Xt&dCnc7M>Rd&6caFFcL;W#L+RuH$ODrxIe9w66@~9gF0_Ol)?;6*3ZwIE9rBdwa)-5lp)~pfW!IYvvV-{ z;RiVr#}KWD2Pf~j*D^HMWoxGyry#9XA>B!TJ-U&H9TG=p^}zP^d_~nQgH33`^%?o% z91fj-@s3~M{~Dx_+A@0rx)r4LP3Tk704{p)&?-zxEAVYSl#d?4c4vv8@ECF{IJ}Th zM%$4j<#@R=#3w8bN#gW{P`LObeccnM0Wt*7Mg2j9SJ26ht>>sHi>Y-0Fxp*~@j$YL zie#x+kRL?CxFq5>(4_|(el~wTK%01P)?#HwSr9+e3G8}!5Es<9OBL)7GLt{Qcw4u1 z+|srAx?Y8%fG~MP-s6EZq9mo)!SMB2`ZQWh%elOUAYWKacLKu(ub|f8Uf*ktIR_?_ zYwquNN3t3$RB&wEerHd)MF3Nbh_3cO-1dKO_B*eCa&R{@N__;m*O(r3Bn2wAT41^m zEQO|UBv?ESX^6!d0HHn%hk-swb)68(EN>6flJZu>X#;}85@M|Ha3%z~pko~H1gsI7 z0ojKzx~>#-cI?^xFH)nUm1Rh+BNf=Cs;qbk)GO(o@8i3eJ1 zgPi&N*znjWa6)2a#_v%LmE5sM%uA?Pe?(qy$mbbbzRic}r()$ic>#AlgN~ml?p2z! zDGy1^dZC9Ba3n)zZ&@!m{@L|@K9TS8FLmYphuDag*aeFp6x!4~#d7)B3d|Fn25w#%cLhx#6TZy9uy~$t?!M50g!5!iXUkwSY$V}Q^K}( zbjvn+?G%!568AiCCHc(d;Y)K^4Al6HugS%)L9Cd@xU)mm4LAPrRbr|^vb&bd)p?VwG$!6@d$UnO)UtHmG_13jI^zP$WQGkEAlc84G83K5NW<|IM(1N1 zD1@mD!1v>@6A&9v1}&_Idng?3C^%Z?1f56a`UJK9oFpSFjV%#2Z=s{pYsLg!+BdLs zCn^wAlB(^nV*?tD%i>HCBgE6_2J?b85&SvY2&8h-taUax5plMDyx6WVB1HUftyOua z!1inJ{QU&Nr9X>;0H`l0KP@_aqC7mN--vU#!j$?e4?18VuedQpCDmchcW-}oX;~_@ zmC&%AS*%WNuCmr&+xyyAezKyVAzdLfVwId6y5rL2yOF%LV#Ogq)}4zsx8Cs-vVg`O ztrN_-eNTg-p0k}fHtad23JypBuR~X>&sno~(jIc`&n=}3eR$?0>{$j=KX2z3@lE<2 z%4B+UVYOHexx%Ww5iW<=i2rGC;nJU(KmZ96sy>^!{#M0vHyslb@M2W>t@XnN9fLvF zeu!gk+tA4*=jWD7PULcC|30lyI)#pl-CVwt#rJ$2aT-cG<)@dTMYHPirZ6P!Szjk% z?gUQhh+DSN_rzcTb>?_>yU=wy5^t~-n78n$nC_wE`7Ztna;n_f<{1u7z^+)}2(?hY zT4!G;Mf7lIy5gwpm7Y9#h+*pZ{v(l(J;2lE1^EO{#m-oO)fetCVP#&6){D38Yv79! zsozHP@43tIYwq?-8p~lbwj%mea=*CMu-YG*e0*?P*6=_FF_Ls`+-gx7^AE34KZU$D z5|wGbU@aL5Jhg7?lSFaw43!~(RWeWw|a&9M6A=Xw1Wp)ch4nD@ib^$rHMF^UmTK8ZRUroTT;4`(a_A zP15Fex(E-c|LR{B)dfB|HFb{|nr|!LLJfBeB*6fY(faA1ura7s`1bcpIlqV0O+pmy zAw`$lpIDcNTZTAb`e79#cR_zex8q|!?MJA7a))#9Fe3b~1E=F9Id%woNEd|Cpo1C7 z+2>~gn`NFzl}|0nGag)s0jtL_#o3}zp>p%^zN~vmOS#~`&NfW{bpq8u%*f(`#fG0K z)A>G0tK1}K?+#b|5A*xCY_R(OdJNaCpT5T{LsprP7IK5rOZ0z~1b$hQ0tr7i?tj1j z$JqV!koyv#z(PO|6Z@BY?~AC#n$)>|I{M2p7ob3U6&W4mU~XvL3{) zJ9(35q@((eOvJBQYzda8z_{T2|KFYY?Qj355C1RanYYge?S^H!*T9wQ<7e7!?#IRD z<7F3z9^kyWA-bklv>|!`i0PxSr()|93D5q}9ZjT0 zrjJ2EL^ad)NQm4UEv&|A@i-Ih@)4r$FtW zk>?BhPg47zWB5M@z`tMrQ!e&XW=cR8Slv5#K@rl0{{bmJC9kLg!XM`@)}7GC?9x$4O4&Ai@u7LIY#*-7yiLN}&yNp;-)!gJe9 zm9!VdhTkZ=s6QGW9T9!T_=0r3;iRqtG;j@_i+js*Of6*7u5b}^fs7LAfbJ_^T+>!f z5Hp#T2pb}j7j(2Nhaj2IiXdbcS0CqDGi^}xt#e}*036_fRhz*5S@e_E)a5vq=kQ4S zG65CL=5)?Er3AO{tZsQ@8Q0~qnEriqaYbJFBW#F&0Rc{dUgPe*Ii(jVU&&q^8RoTE zrdgz2F&;e#E|7*{gjL{6p35W88GHb$vZ4D0BO);U@_ky}6GpAMtU{qUJMN;ASIIJC z@;E$FhwfnMu=VyGJI8Xw8`ZD*rL`edkf3H+19RVvch`vmD*TM_jd0+4;MClQJnY~b zA*A^CCAlH#!v=f{f1yuJIbrW}H>iKa70@`qNlN}PKD{6ZX){p(WEQ*nd!HdFi)(JK zEs4so`e6nr6$BHLLP6Y^V~{Psg+^_*dQ8((!~ ze@+807JO8~#H*kUfD>*N&NYuZ#^37Gg+%F8`PA&Ts7tN4!h6_y5GdZ#Ew%uJ&1~5A z)y8psNL^n@XD0yIwxSM*a#qjY7%w*3#&=Eg+g{hKo`9unX1Fi!aChq;UC#@M^fXce z))LD@@)M7_Wn3KS;9&dHP? zpqvAEtW3@|h`EP^Q^_*y9Z3t7X=g}h>eE5uS7R~qq2O{^-QE#&BUvy6-P28e4pm$o5|BGRzM}wIby?xXOiqFCqK?Rs&jMy+kszS!t%OWg@ z)p}f1<-7sGCnleC7N}5ObkM-+u2b@BEUb9}E((;~vTzSX-VvTB)W7t<=Ia|R!ToNz zy+)8)7qj+5YeD`k_%GN!N`Kc7jkl2JSnJ`0$5cQhW5(8G4SGFQaBG(}yGjx52u}d{ zdXc44#UJ8)+Kt4T9bK%iI(X-Z-P9v2P1UJUJFPnBC+)&B03#-bYBzUd$P^v;acUxD zF&$u8p=uc-I3}a@gdZaD(7?1;DM~y+K`)G}@qwN1<6}s7>(M@|sH)~E^?!I9C%G{{ zcRx52sQ8i|x)WV!DSr#D)3f|bwC0NiR^LnfDd*7b&dG;J+%m!lrSUG(hiKL&b;6gE zg&f{Aut?zvoi=7j7QyR*YPlk~-M9no($0W*@FOl}x$`w&zB8<&wKOK@F*VhR$>biC z)u01i7E2vVxhyNf0i5fZjN$90?6`+O4pfhei9>-!(|y zC$Ik7jjfq|XqGZ!py<}7=2L_Zi0HE3!R6^yzPdPhM<^bHL~4Xa zEMKq)n|`8o-3`kWm1p^Tpkrpc4xYAXdQ2a9*LBy2%p`t6EPqvAxn#P~ zm~cs?O6Wt{)P-xh&~?tY98Eai{p4rl?!1qX-GyvBlD-*0jRQDaSjoxY3Tr+k#2I3i z7hD=nB5CS1tq7}XmFYSHOKYvjp6gJ7&f4sM{{S>bZ4A& zcwfWpcBXHflCk+iec?NOxRuey3x{~)SrK13FECN#9RFhX;!ODvA z#o0!s{6s<}x;t%G(|aZ0*}T)RMuYUrlwQ={OAFoSBU#(6dL{$2E6yIRI;mU?v`8J$ zkp&1@&JSFct#kWphGI}7cF32-+J@jkBL-nyWp{|xA8BlE96T&c*Rjn$#F(b z)=)Jm$0lW}o5XUrFb!R1%QJ&^)g4snmA`l4)e!RaKV=%crK?oSZ_X$#LqMRobbjY|l(*TT+$?7My6_+8A!=%Q4+{0!1ox13>{- z6G_##fZwfru+qDni>Fy<8Mi>kJ3H-r;>g-f1)k&6CcqiMHSK})#3GHh(;003@E2eK zzAjmY@lvM4K2-Q29;sT|+Cu@`qH57vVq)8HIdBH*s&r%w?ZcP!)eRHFM7ukutAZup z7#wb=Cv20bmq3XMyGXYH`-zluGyy>&B0*o<%&^pi&g#06Y>%&WUGqd)&TWgGq9yVr zYwG!q)h+!k)cz_hj=>=g34shgT0eyI5_Crh-!wa?&^hGM#4%{zCk>^$`N%Zer&fEF zo?k8v@vSPGLjoErI_DEwKmgl_Nomut(o0iJjCqCw9$VkAW+y_Jc8Ku`X4CDhK^fgm z+ZylPr?;(|m>-ifG@UjIF)X4z9keo zX68j9CMOy?n;%H1bE?g#c|ypS*X+b!94KF=TKpO|_G255dp}yBA)zJ~V!@_Y7yKX; z6zFRNIQiG2PIN$j+S=AdIpP;FnbtX*=VsKX>%2-!>s(g`xami=ECtMsrW@F;!5a4b zaC}(X>w1}m8~|vb6FS{8(&YAtZvhV-@K>a1?^02Vn=>!3$d>dNZcM&>s3|6#n+U>* zt9Lj@z!}lZJ1l6f%J~8V2G!X*)Szj#4(ut0nG0wTcKK$4xlu`y)JG76ELJe98&0yJ`& zOGm7CGxw$2wy11&K)hy52OAW$rIWjSJ=OV~Z((_NK+9UZWNKE4?kmsqJ+9vUiapz% z*9C>-8|4SXyXn`A>D;0pduIBqGPH7b4mL+)n(Lkjm4{QeC%aHuob}KmJ7ePyzMr(;XycMGS?PkA5U5RE(ou)D;zzs3$PKJLKEueW?cNDzbDnOhEmRG?7s z;vQM7b%w<>sqN$QZjnqKAzxF~?v;Cd`xCUPLp(lo%fB}j%< zTl<;qqF(hZ38s@p(jifuiwS%$z;aDFesO_ZR`LTg1pkr}bwTpO*SD*x4?2sSH3y6D zN)6G)CsLbOWba1>{GBmz=XZuRwTnLkja^gy3PG0LS!!yz2#(wlYsGkb^Y~;#`A*W%#xeYjxnvn1Z~ve* z{J`Oxm{ZF|ELmBnCaI)aCAyfbOTHAlA`jl*A;buziPw*|-;}*qml7@3cN5 zNU%9PVsl{|ejgP=04Jyg8IO_H|EUEyb-`3(^Ep^)NiQ05BU0J?pxMOJH*QRa8xVgK zb}(BN9*X=D6}#J`e|C}8$4pftq_uO{zT9wyd8E}f?uYK-0m;!uni>}|*aSpeuK}Bq zn2O2bW*Dl-Pr4>RCwje#Mlfv`^PM7bc#ggyWsZGPzCQ@wpu5gRH!QB2gDNjx!rlS+ zfI(+Rn$!^zyhxCyZ=%lSeX=`c{4sR66NzZC!_tGd#w%W?^Wea$WaW#V2Z}^ZrS7%? z#07pwceMt?wAj*Tsvr4;l@S~#x2Ed_8P5peeJx68O zB6Yq>(k7FZU(GG@BTaP5@M9{FP`mUQTiv~P>(Zd*kF&Y9(Tn08pH+{u5`e^d&!4tMcZ1t(TmQ?iy5Lb zl>f!vdxtgIY>%QU2q>W=MXGd>9(t7`h=3>{y;td-&=Ki|A|NGH6#)?lz4sP62uSZ; zr1t=UoEN{}-urW(d+%TOK4+hEe|eI?lXuFR)n;bRTC?6dWd;GRw^m)+_!~L2m2|&M zEzaCRVI^@Jt+Ea#d7e+QZY-u#i;tW7JHOqha_#HqE-vO7bm_=q{Yv7T$vcr4 z)nRmVnt65aJ+Ep2q!3CDm{{m|zMzEKK}-q80k#ypi#{i%`Z&3DO@Ys3bQ zT5lbWlyPttz2GXBp+1!U{8etYC$87L7(_66#xWV(fAs6_mmAX$fOU_#i4NA=_c)75 z8LHW9%YyFjE4=hd3-9EMD$N!JTEzCObx(+|Fvsz)+c}VL$`KjwOM+A+>O_jIGnTCe zhhPD5kY`N#n&f0P?z^J6xUR~BoW*KK8|)$NM8-Ny)KtBDEM~HtsGd`>=TuC7jFXtcF=7HKKu_eiN$uh*79m4H3xMtQur0M|Nd~Y%VB7JkM zTc05HC%eTjQ%^>ktfkr?7 z`Lohoxe(P>i{*@l$OgXFN%E#nTLHEsd*kuw?Ue(ZR=?}S+?lb$(aAOLFqhiY#vM&! zVCPkKOtr41VfL3> zLoX`|`mhM0! z>EeU!rm*BH%mNO$i5>Z%(jM2N_?e7{!)4nKGlYGUgRoEdK7{FZnYCwUF~{T$f<+CN zIBnI*Jy(%9Wsd}ah4TBoF8+I8m7~hk2t4VvVWy@vtyT|br=!&kEj=uwMG52z`es%t zl2rK&zcJv@vE=Yc^>oyMne@Vb4hGrl5hk&5?dP0RFCPj>b`AgJeAdn7LF4Ji0-M`(BH+%ASVvoD-)HGqR=pm$t&*g;0mX|Q;VFHGo(376t z?{TUmTytineS=T>=3%You1xBR;TZ3FT{%WQziOnJnvQ-deBWA$Bn&nG1l`sfZ<}F8 zXd-Kv9p(&~=9_}DRH+xg2L=`F_7a{EWoiuf4#nY7&=@d*%xJ$UgI!P-1M_;o@>{o| zwmkvNI0G|yz{K*EJn#0MQfusS>s;d^zKUWQ&ZWlWgBb=TTgiSJdvT2nE4S)skdMEmjGWpbnS z_G2$K`^||GD=F+Myx6jL90dsR4>q5xveOs0sc?YRnY~Lw&lFQ`Or~)G#R#}O!4Lbi zq@;*4!1`t;#p0oQ+R_Y+M88^6mUC&{)Mk}0!RE1O`kKL8c4++R*VQl)r#+l7d|Kmi zW2(^PQ>(>^&|km!Bpr2W(sSDVhVG85(pjyHv2#kuZcVtpxvU#EVYle9eT8}Rw^@S` zQTlV%D>-7>(Fvsx(DM`y(;t4J^GNL*rklMa!Z;PLg6X-&>Q2W9zy0$##ad-U?+}t~ zuC!qeGcYa^!Sc?`bPiCY;-6RR^sCd%HwS3Sy-^9hAP1A>ZACaOn&?s1<;wOe#7=)a z&CClBWwp%?)~qu!k3H$X0n5~tD|)48`htNrCPozE*)Hzs@(oOy_`lT(rwpbY2uqUnZWmjk%2?MlTo*^ zEu>d5)cbBdGtAcFa7^;Kp>G0tBGzlyN6OJEFCW^OoQlR&h~YGC@f{s`I5}_KmqN}_ z>*V(gyTITEZRPjLD;=_|mV}EW(9{hL_HxkpNUmwSiH5++w0;BAcV$&xm)XPyx1mEm zW=yqJ;2)#!`W1M%`enr?g~WQH^+ikk*}S*W@d>p zj2c@pJ{ehBcttgDE<{*W#R=jVDM!b%X0kbN59_>L6|~7$S03b0i=Y=I17bkM4(}e0 z_#Lfqtg$FOWW}c3Db?0nRY82Na7>PhSC32+m)#xrkloP+19&D%1eU1)}P@+7j0kN=^mDXdOJ;ytoY zb!&>`)ml!j4x6TPeK^HlJ?799CL^$iGxMXnY$()k;%|i(alAB6h^P^WgJZL(AU=q= zpWi0S#kN*bP>z~s9u<-E@u8KzAr{|_k_1jJO8nNv=08B~Uitw6{Vvu#)frF+AoIUk~qHoS{P^xEZ4HFEPt+CDDv(rL z4VE-gScctJ62V7d#)AMBj1)nYFhm2~Fv}@C zqs6TrspbZO#2?Z}GukAu*PL+ax=>DeW4-p+EVwuR6*F`!uyAX>CN}3w6g^5`BTE}! zk%>Bzm%_}6m~skR&=vWeOB8&EmM7wt#J2S}+b7RP-`kyPp5R&H#P_>ZTI7-Y@!8M1 z3FLY;+y2U#JbYM~@Ku9R7pt<=qBzyuZs^Cn%7fU}Xm(O1i#Qd(F9o6-C$zP9by9Ri zU-ya@zzbfSaHfpgFYg6%k`j}dG{{Rl|2{hXWU3^PqDjA{2f+1YaL^vw2dk(i{^Y|a z-9ui7MEK$?_0$m5wz=Bt{9@{pnt8T1gVTZCB;><(5N9>ZXWbL)o!Q4$pZjEna%&*C zJ`l$&0gHXZPewWNi90(U!*`(Akprv@~_8fJIfVnb_}==O{cW*5uV@s#iF7bAXI2?P);7Fq4em(Hz4(g z;R&_m(|~~|gpw(%p8Qo6i%$el-x)?KqYdgH-$&vQeE1<_&!=4RI}+|a6L|e=zULE$ zp7rx7;7H(DPD~8Dl!LR7kmau3jB7B>tWM;j1cS}$U6v7fRV|X|8o3(yPWn`4AOuC4 z+`Gyv6>8z))VUl|tVLXgXjyjVOc^Vo!ZZE|xzNN!YLkj~-I}-Yo0gr?Q7huI-*}os zukcgULe-oTDr{(qyxw4hK}KRL^DYE##b-C=;*Be*cQM^TS-d?W)C==@ih1u{2XRGL zw(GJ3zQe`?6RP0BnAPE_s*LFJ2k*>ii)Tr6g|~%>an>-sZ0OO=eqQ!{@*! zQ8?Nw_xiJ*;S((zhQH573VrgK?ehz^zxfmatEreK-9q^i?JpG-_sl>sZ~u9_(LbFV z@c#5-xUS8w{oA48?+tx72D~2r{sVCL)o)LeKY#mQ(bxh2!XD0T={J9X;_DKr3s5GE9LoMD=%HHIURf4T`_m001j8nA9bR}{(Et0PgJnU(+XYY zglCIhvOZ`WabgI}fSqn^xA{|+_=(E?td(m~ zqph@$f0n%`9wQw6rw>(U3@_Go`TegVfUMO6b^kpMqRIC^*Z<}}7qV*|G^arNMDC>= zb?2X2Y`K#n$U;Xuu_J5kH)*5^UvMSmjoM)Tn>G~px0_~(efs`+t|Ak$__J%<%PwK) zgH;!paCEmh-@i5b2qX)f1OoRX2;;}R17?g?F=V4EMQAnrPc0}uXEzPRL-efj#l-zC zg(>}*BJ|I;$YrTJfy&gK;_Uz1%`@UQ&T{VVp}5>vwpaU7s0$Q!(?uO>7QJ_hyhNX& z7g5N&{_X1peYohYQzUi(zf3|*ZlhpFzLZx&w`qu~V^lnP9No2xB9NkwG?k2wMPKY5 z&|Q+F!*OwaIiN{b9|HF1&{b(VY=2CxzBbp?iP2E@I$@u`l)8E|z7|h?_G^AY3q2sJ z-Pks0=o^jRpfg;rLYFmlCRLR$-(@Vn(2hUfe+fGR;)()QDeK8tm0*P zw|kxD2~&j}+s}8eY#T2c)+c`)R#9?XdrST6-MZX)eKbk*?wb%2x;vkR7D7v6hw2rB z9Lq`@d^b{!ERW9kI@F@77YN6dA~P-@&R~GR!TB+Z+UTMwP; z18Vez)FoF{qVLsyfI!P)0J>>qmu8#kNXv6u6@43hf_z1m#UU1ha=$v6_YFPy5Os6O zJ7U_(zZHT;G0Q)C*my+M^hBNF5E{P8yc3D~uq$KOXJEbkjK}zTFnR3FRDV z8NYL=6f%g;^?Z)qJ~d-u7d7Bm`l-9+#?=(a5w)j^T+G+f`8r`Ij3k{uUo*1OgOHj1 zoi>Ec1Lk~9G4wb_cJpTgke}Pzt1tKbGI>DG21%+}9>-o+-GeM%YvWgB*gnJcP~LLT z)?%;GoPrgXv_7}X(mV=$l@#C=eZ_i()pQos#$@??FDw8_y|{3*^4MIvVV~p*d6rb; zUe}eA8xU2r)qOA$a2#-8B_%ef%-KKWh`LLa=7Tz;YkJC4oWAYnrMZ*77&eb4G4%FJ zQ8BuNoQ;)8vdkgDS7$LSlOAq#G~Ms@Is>};e@Hi$R*_c;b~d3{(INqE4_ij&uk?I3 z_vnR;P$d4IF42MZdK=b`EX$-dB_9^6{7)gsgEh&x3O&@2)VU+Ftd26{@_2r2(`TyY z@*wkIc3BCuJNZnnXKCsX{Rn-DKEd)cI!{H@`Rv^^KQ)fS%q(F@h9oOtyJ?wih3R2R>dRKRcPclSo{5R-+Kc zgBK(OhOBXmH2!D}qxFldd>Z6xgd#_^WTt0t|I9^BveF`&Jl_VO$>%?&QJI?Sz3W-B z$nd(djy8g2Z#NDesK*efRU=6QPC}q=G$JxqUwVYlqv)>8xu<)OtDSXE;7%cy%nR2> zZ_|iIKkP2~KPD>4*ZSif?gl{KwNMi$?g>C5XJl6^EP(4ZU{1+G>bkW(>Z(Q-<5p;F z|HTMDIlCYUvD!O{{;5*vRv}Nf`N2bCvx=@OqYDMp(Uq{%o6M%8SN^*)!3qH=SL*Gk z?^}2HM9v3c$p+aQ?yd?C`x=3?NB$clZOP3~9YZ_j&t)6;XILqa8dODzO-D~MYBLts zuaeNws`$)2dq1tAdkO)2x#E5&4?O&R!{Ko1Pp_jg8VyC&c+OX<&JcDfM(Ft~UV`$A zK%!9ZY2<9cd6DNi66461qovG9^!Yj~ny6_no*=H+M8W6n@Q+=mSjg+c`vGU;R+Go- zXp*KQ62^ABiP`z+qWue07wY zP#JS2`l!=u<-6fqvhpfsul*zTmvs@?6#vp>Z|TEDa(+ynO+<^`IA7-^DcEaWbPOT}7ekOE`2I_IY$)kv*?KJ>XyKs$MpYofOs zIo;(^mYouiOW|KjN@+reGGIwp9kw)!A!kbtGnj8?3~sR%0DQEB9wE){q&*7+!o0e6 zt)S98q#cLPG^l1w?x&Ztc-^C#jy9RBFR27^O4I4x!fd0Ba!If4&9~zVD3tMEWkO#O zAoln9lH?nJaPGy${reSs;1IlLLXLnUa-t*&O7ip+KEB~smoI$Ahnw?+pQ3f4jwAKs zcH7#qfbEm9@ker{Ix=MQoez_&by-^8#Bh>beT=fu$P@9^djEcY;a;dT(@Iutjr?rH z{zMTP{MqP26?I&ty5Ausf-vIeJdeKOXhPM*49DXe|83HGOD_n_VW;m$8#A$q(Sdo7 z%nIo27`KHcmiQoxy{LV(Pe=@^N6J`LSn;c%YKdA0V|a^_IFKYKG?t!pp$?=RRAuF# zP@x;-FEjHh`$h4c?6_b%LuQ%HH<@<(tjKLCb87xN~(tEyyZ00{GL&dlcS%$ z-RfdQ_0C^WG9-2@>F4&WtdeNN*C~b;4Ncm`aVI)O7`voyzJT24_woxF$~{Y;4|u0~ zxzn^(q8VSuVfQF{5X*@F-3I@|5BZPZ*HM34#ZnQH=^BK*dN?j&xF}KpCyNM~z9Xij zpO*abqSHTQ{-V2S5HZXd9dHaqbtE-5k3&EQ&=`V6p}&(J8EXYZc{mZkuJCESOod$M z&xi*N(>nJ(L-b(FdPb9kx&y9 zqh>!*{lga^>*xq#qDs5or=O7Y%If)n=WV{@ z!kvQy%PvV}zKP`U8n@jE=^5Q6+kgYdv;5_eA9zyTmD17vTV|kj>R-?Q2;*492g2)v zeY%S&{I?E$T@L0CGSR1Km@VBIrhRzs;v;ddu|ZXId`#SNi}>#Q3Pct9$cXUaCADnJ?To7ft?peUC{uJe zH0G`zXqYh|X>jL~^9qGV`Uz$rh1*J08c#COX?aIPR~MJlB_OCV+f%1Y;(4zt>XI=0 z;fp<1&!v^ySGxgMgUMI>sw113%c({v;?eDY~7` zQqc*H`i399lE*Eob{Eydbdgz77c|vg6t=&pI~^%}SLo(c$FPxDXEW=xYXs=!%!7cj zHDni@3@(LcZgiPjDr>sJaW+m!`XI6&BDFIA%0E2_#;pZY zk-n0155+#1j>lrxIP&!LKgtz3K|O?XYy1>JSzj$WqWth3peB^Y$^KK=r)5Q~J^i?Y zhkYBHp7vqL5F@k&EzMJt`!D9xOe`f@J}I0MuEGy`NNqJ)8GU@hwcqHOSv%9#*(b+l zfI4a(iA$v}X$2hLNsN)|cGVi+@ZPgJ>ufqQGY?`}DCGr93zWsk260YkhNmmgT4}jO zfw{$dyOqEzJ7|uk^HcxOPx?4K;W;i}E}5UxO0_(D{XTZ~GE)42Vmf)XV?AT_$MF+g z0>4lrbVa5Gy@`qO`=1jnEe$k%2BL^ER)vTt-w&!=8y(7Shd-|zl-m{r`UoYk>Aqx3 z_MYKtbxqIuo{alLGeu@XyOoqnFBz#RB6)olDF!oYm9w_}ujYzS3^BV79hhmud}Y1q zzB_cjajki|$N^aD7kuB)c~r&VcS^yQ;@9(+gg)_J0}XrPovH>MRfo?0$4OUIClCeR zKQzLEvjO?I`<<{hAkRUAv3Z#e0k9AC$yndF0X}?#hn@n!r-C)pHx9Vt-ie#^?rp%E zdC@7EActjywXu2E-{h^)S^vDTkuB-pEfI^374liqSwr;k-NqDHr*HoV`~J-lddk3@ zbd^{V#Nf4(P7{|H~Zp6Xl4{t)lq zH7b+;z6kjJe{skE918XO0)@YCecsaB{q6es|LWQV`X|uW&FR?z4;#bQy*x9*{$Tui zv{y!FgC$^k*^>Z_Fwx6@JjK}!fNv55X^4}4*H;jy1wsV>y+nm>l+35*hX45nhyOJr z{SPL>|7M_FGXNtC* z)y>v=m^FhWwsw2>?&fwC4s9xPCpmdZJeVrQvQ5AVYOCwQ1m?a%zRo-_$%j-fZZI(O zh%4+id+-51gr|mCJge82u?Qvd2h2W5xhItVN5eI{MOz)VGNp9tm2&ZPXu+OD(15XFcb4BUbQI{Uw^OW#W)1T&?x5VzfnzHT*Uf zfL$$HEihqH5#iWh)~NGYgZ<^V50ly%>tlU<_;2@i@{5H92HLmSv(+Nh^5lTyQlJ2P zLY|PpoMhN2Owoap%nNTY`(@D!_5`*BK^>iBQ#aGZe$&|Bg|o^9I}2e?0Q26S(5J?h z_6dp+=;nuK%+InJsPbKBs*wyvXasKGJ?fnXQKd4i8c)AK?~yip4eb<|sYzMr`U4zC zb>|H-GZL_!peG3)nPU{Vp*XNDHGj>DvcV;AIT!x&tDsj1cCU~lPeZ*lhP0=Y>o;ag zgdM!zT;sZ%((}T{QhkRgQn)uNd{U+2QSp!^&%B@1ww!A>#1V*eiEwZ zo_x}+nbQe{$w}JIh01gG+{S%=Auhulu?Fbc!QIqV_SjXgv89J`CUmRE_XHo|=@Wo@ z6*yv9$W8*aEWJ!fM%LT%UcJ6g+uOVOLFob68aZ#3J7Gml8%T~q93>nyGiJwgxfHNi zW6Di;{#{y^VP;KjpeVLTH`pnU?4UpkFYXF!Gvq$pnnW=tLPub1TrjVOF#O9?);8@o z>w;_rMQUG#Jwp}QQkY;kz`Ze!3N9BnGd#bBjW3Jf3676D1xcqQ5RcYEt+B&%)#NY) zxve#E3*e4w5WKg%bSg2s>rikpnhoQkrmzorw(jN6(Y_ob^_b20Fizeiq_R|zZ{kaN zqkIbN``l}NO4zywu(V63mH5zv}SD>$`$J=-~ zTT7VsU*ohDmR6QVNd3&!e5h{wR*IVO9_|Z6qbzL*&lV4mBlx5JEw1BX7TYt1Xj|j6 zwlBkX-U2I-8H~RGEs+_yEk^hN_Ku|7>q%BQ!nY+W;t5*@0}Za}9vKB$&6TV#>IFG> zK3rvg$#u{fHNjk(DX!cn=NWLf<6QV$j*i>;7u0vTs8P}F^uZ3@4g%Y=Uo-?o?3y47 z#}1{T&rl}9omvoh=;%=VvS7g*OOsxiJ%Rr9p{_%ajDLFnH$^4*?Myx9rjqA1W0)fh z6e{l{LrOrwr!V|cT*ucHk0iRx2#$#OZ&U#)l0@8RjkpSH>Zd(dS?SBqTSlfP51%U8 z5@)Fy`(-_C*m!HQfXf0P*QqC!qX@2^ht$^BMm+0yZRhYEWZBj3Ti|l z33_MSYo~d?IwO!K^8Itx{ZyP*``^OF&;~BH!)*h5g-EpxTwdHE#n4W`W62@U`@znk ztk0M*g|P`u6Lt?NC1QCks58$I_iJ+S65AtouYVskZul|wDalDqwIxnjy>x||`B7+$ zu^#rz)T*xoTR_-b8s6bXW^HCvH&oomK4>So(;EZu2MDy|Hi*3iee~-7sLHn%E?Un$ zoM|x2;rO8I&N=5VOPpy=^+StES7WJ5num6ni-xZ?5A-EYILS~%k)dH5BBamoH?A&_ z6S4j?OYA~i87r-Yw-&W=z!a?<8V?4hHIs$=g*xq{{ZS8k$YWf+CqILKeEeBW`c6Q% zl7P?_USe3F&bh){Cg$~XPtxTjbame~twvC#<8blxmqtGY#>K7~MC<;w@H_~5;ULC% z8SlG2g;{nc;RZKchQ;TL@404vNv$K}5+7tXd!sRgpZ%!I$>%+bII8JO^9-H(Zibo|0G)O44rJE!sI@Ve zi+=bexE*xi3Vq9oS8-TmKjNg0=q(|}8vz%q$0mjj1I?G@#_;8eS3hQKl zp^8d&=Ap zq!9m7lU3FA*RNZK^x-dyzJt~#fLx^K@g~8i(N@NkSQ*U{H7_I*YeY4n_RlHvo(2~t~%u_cWzb&tK z(^81e6X4qNEWr-kd0MNVv7H+gxn7GQuB4HKH_N2+)~xoLFSj-ZmlO!E*ZFe-EG3DO zwWgglPFvPFd?Hh#VjB*Q%#*zBh8RvX9=kIGEBaCP7tzI8g@{qd*^h}6-9xdoGKsyv zW^@dNA@QRb+e|JCqO`GgLSZL%tj7F0jb5P#g>2Xa+n<=*33AJC`bhP(Z9hnZL47Z` z@Ybq!m-oWFajKQXPgfKj29Ac-$V6PvSh^Rpwsyiz>t3zF(-#gpUdHXqmdmSPE*4H! zdxaLQ6&>7fB=w2e%IaUJ6g_)JMdxqK-Cf+$S^-Y0caF+toaOOJR}#wK^4?>U*dWZV z*`(I9TUZWT*JN+U7jF$LSFt`4HfrFLpWnk#IH_8;i#$ZR3c-vfpNqa&+&WVPQdf%1 zWOl5-f>GABk3zLJ`lxf50R;IIrY71E!wX(ceBHAYA8b2doxE7!xicM8;T%+QcE7=g z&io_iv$FE8Y;bN3Z%zB2tRD!U<7G;O>-;APxo&WqLDaKYs`Y^~TFRAA5`OJlEWOJ> zq~94o$@El&tDaXpVu6u*Tgi-{Oyn{>UO!dalzt>|kJ0gR?bX1%cFsjt7RVyVvw|Th z15HL=^hA-RGH88C*Tnmj)>N;@(teH4irc`lmbv4F&#m}flhB_tBABa=uE(vfbZJH9 zxcY4SVhl_Z__3pR7^!%=2Ip6P2(`zU@1luxjE{a*G0OHnvKP0A}xiG-Gb&hL#~ zbqjJbUcW}LyJe2P3a)QF_iOD_m`}vT$5Qd0L#qMjhODGZd$PBC0|MWe9n!=)b&F%L z@d@7V3FyG8`?~Y3C^j8f+w7z6mQydqexgn+8%Qu?6^1y!one#>_GIN4-oUIB>0i^O z=oZW=SRASh= z$bnTeUqAZ~Zh{J`r-Hg*P7r?84ccf4xp~yxk-!G(`_@@JzzqS+P`^{U98RJyF9ExW zYi^BXO;59*1judP+#NC4(&nRxeOLKV{+qH<-7`t?W(EAyWua6Kea}SPWuc9oo;W=E za3Bv_YaJWm=1L&lh45j#P_?LFf7D@_zJx(1PL(tA6ZT}{9iqGECCbppY%%*`4aXL$bCq9mSU3O08Rn%Q)>l{jRN z3b6_FwGH-c4cV)FX)2z#4LO}-CN=eskZ!jfzCy0_grs>+mhzzN+#6{)q-PYXi$G!^?MZsc8K{9v(!{!A9I;3CUJK*P3qRqbHqOyU zR9s^E3g&D82?`dox~L9`aL)1uRCr4!WE{DqF9PYzQl*|YhtbdcHYPdnfj9yp;h!#k z-WLtmD=IGK=|EQWQv2)(kD0|3M`?k-k|!po=$SA!wIC2I*)FlZ{h!C&oND@SwDBOR zfV>7+5K16vxi$-gGMS}nlugUF+VjV=+xmuNRu?C1pFUono3-i@ZBBGB5*+V&r^yiU z@FS|>wz1pKqCuW7$YmuG9{7gUi{yl z+{K5$Zi@QUM%)*CySJ7m6U%NqDfxPD4TKSNy}%!xe5g5gP_S}Hdf@dgy9s0QL&jxfqZPv$c?sU zIuH?0b(__+TpL;MC`(2s)!l3?ib;viLm)n7GH!KrW(un+hkSBk!q^o&h_H+>{q(GI zs@?S_1|TZNP!kvhRuyG(&P^=g{#&D>p2zGRpL=`xroF>ieTCl;NPFgr7d;;V0d_jx z>Rucy93;p61Y!s>sGTFe4ivY~lcD#~5aV+_W4{{g+os8dFAkluVZ4ifY)!|jvGK8q ziJ33j9BN;^G#=eNuM3;hYDD~4q0A81YRP<^of{DDSEpNJ<{A<;29kGfeSY} zMYi~1J2}p*9PAezn$CXCE{kbNucNesGSugSMidyv2V?c6JXN0ALY1Qp7JE9(JQsx( z>b&r(3{9;4qem+Q62}idC_TA*cB`@SuD~2V)V_^ztH97?G&^pag(k;~Z%RQZP< z#V^M!q3Mto(ImsPh08XCmEOb^Q=0-^_N)xy9BycYwcd|>e0V2BL>Q0!ZN2R+e>xAg zmW8f1kfk!_;;5GomWFfeJW5)A#Z5#yjvaiGuIC3OVoqd4T5!Gz~3NBz+}Gdikw2A638U zHgG76wz}Zqq!N$G8EL+0_i@rKZ_EHCjoY?>N12#e8EE27p1Ir#dsMzxdOsdmjBmv~ zaSR|xhfz5RY|2)7-3>LlZNd-=85bsw)m2kA3W^YhxTUfXU9iOG@i(D`aRjQJ#RX(Ls*pccw zKLboNlJXF5z`I`Mn`=1Z!7xs&l3I3w#S~Tb_0g%Q_2aAvKQ7R zH`*BDHjGufj9s@W$;jS)r&+w`ke;1N?zPy;M-z1EX7AWSzL(tOlqfQ%o%M3v!Z7<; z`tIE-*PpUF@2K(%9PCPY1A=g^28hGOd!ok(TVuqpZytt3EP8o{cmWgs;L42XacI!8 zF#mmvL`%(PIwop-r=+k=!F@rpWSvJNf}Y{CVptoyv(I}t)!aN~v8ued;G0YP4#n1k z*(E9Y%8^CO{Hgq@`1vHA$ewW{V~2)!^!4r;)9udG)SWH=^_2qeBp}5h?ERA0uTrWH z|86Ik$2Dte)6iG`4a8%#Fj4kjF1;MM;A(IoYeAZ^TypJ)0^JoBM3^y4JseqD3Oa>i!clmFf}7fb zI3@v+cB0W$f-T5U*|Nzm8$V!ml0gYi3GuO8?Bk4JR(U^xZYQDcY7@w`J8-p zCPULzLFuwVfW@>!kotafm=-s$^Dpa0FU$Mf$)DveTz9+L5#BCC zF<(zep1`wIb@9U=BlVp3oTZZ5;9V2?*UI=< zH#t>12`@0Ih!1uYzUty0SY)}FkSLE2yaKjIWC`9uT8HAmG1a~}ZqwkG(h`}x$uE;` z+OXuTKKp)mE{O}$`I_R#Y(p}qCz9tEuG+_<$KDvyH?GffAyDg~610iOs~G3C)PR2L zfDf;mKkBkCPw;v~`i5sj-l?E4$wK_xSjXA!j#E)c7QcM;4Oe8VN7VKn_Q?8>lYxHB zt_cy29Is%fu-BBu;3D=j`PsuJ`A$7H0ATF=RNm;q1I>*;s znEKhFn@@%@!mRPIh=R4O$C7KkBe`dfZqhz)kg35MnjxgCX(GtH*;wgNtCv%KvK3%a z$Sj>*9dQr;P+o02eL{UX%@eMy)W_0PDP6N>I*E5*8x!MYUli~2G!M$H%@vEl*Y9R8 zAT9?-Q1J@>=RNR}dR>O^?(bu{QY(0a>D*0feJXiHAsO80QEKV3=cD^2%Uy(&ZNak_ z6xv>aWkzUjWlQzyO{D@V>l0dP1}Xs<&&R~fAQs0~o(K;- zSM(mADsAQ>oGG2=gM8p>*CS~8jT;n7igMB#$9M61+a0PsDe7No@;{m1 zb9`^f!<8*3g97fr+oJlD-kP2;4HR> z^U{xp8M+xv+xcbny7b?j-^4mdr3&55Y#Hv(^|Ijz(bCQS9_c*4>=Qv*FEVaoy@hF% za=0I7nI)6=3ph1cj}7x!024|kqpqm8BYO44T_Y=Cq)`W=9cl?Vi;D}4iHa{pozsdq zBjchd3X0&1*&b#2ZN>tF2BC_jg<9%SlD6cERyUEs5K?J6UH;czRJ!d3jF$*w1EASQ}WV;_49LY#qx5KjE3f_ zOata+$Ksc4YL5KQsrb>@v9Br0_*-zf4cD6CG)Kn}u=6htRS5yrzJm8fRPCrY2_HGB zmZv6n_p)mQhabeOuHI8?@cG%*L4;Y%5ZmPlWTL~KUYA09iHeEUkpDUs%~&S_VV$KK zV}D)&a!E6rwtwbb+GpY{Ob3BtigQ_`I_y1yJtxAptM0JWQEXYWu2k($`?ux^dhE_B zShXq$kIL{7xAV!G>$$eKN;WeQh7;a7E9OXj{VhuS&8`)NF$E@xj>`sxBvcHfEEB|SK7eLoSjy=zq$=S*2y zOioFQ&W}cCcv4e$NqR2%30vZ4fz)O`JVQ%`k@CtfI67pD=D3A0VKQSUJvK!>LD>lc z3alLK_`bF9FbpQwbn|6tyWUpv=#j}Ps6>!~9E#=b9=m3^o9GjU*lRF)%1iBySs8)7 zJC@x}an_11F_MGIh)2$2n(O4~3r;(qdE84o)EAe+FJ?Clmtt05D`En1wI1}!3TOw9 z6OUL>nBt0V_-#hHR!QY)mP9*7+Wq*x%DCvza27su@Eu?gN**~fSC<;QxE*Xyf)W3i z?Rlku?03GK7#Pr}qDq1^9WqFw1*kws*jEf%EytKo@Ycu7RSLH_rPwh9+hcc^TSkOoP*51318xuIB+|-3VyIZm z!b#A+d8j_ilkb8hmk);RCAKIjS*S`0gN3ZLC6{2-5l-)&_@&|bnTrNN*X zAch)9b`T-=)?8ExGEaDMy8V%PtMf6knTH2Nx1s+i;^~b{Zn;`!m5JT9a=-SnY05c? zr!17rv36Ny6(P!e?*>bYFQtdK{kGVvog1pB$ zD^^Z@%=F|mdKTp9*dmA*eER+g$M_RH(P#W*jI^rnm8~{*<~zEa=psJi<){c#KOikG zLA?~MBKV$h*G!M)%IPgU(8R;T3g^l47qH^e5KhY*y6F=jlU%ol+k4P?v7e+&gr>*# z>=Qw1hS^~mB?at_aDwT|5{w7BwOA8k^+D#%z?cA6m>-+&|K;jQF=;m@OBj5bvF=r$ z0(|-o#SQy`&vrL!=1NyA;|y+y%uOI1go*eMw}#dWqe9KB7i{ml&3QmUJ}v*RIyn5N zoULEmeIzsrH39#XKb%24eE{U`lK=Ml`~K?}dxb9EzdiT%@!iqBAk&*B^5_ML#S^Ljnqs}jnMpO+A4N;sl zL_|i&eny)jZ(Kf~z!_onBs#^#4vuS*|o#P2qt=XU*1 z#iTBe;$8oRL8Hvs|e`4$Z7>4}s5rzM^cDmLw|1Ge;b?tvM!+$fwZ$`NO`hR2w)PaZAKPEpp z)H7#peI^LQBp6_^o$K=~v$gDt+^kIoD(%}8JJcB#&bklmn_RYVff4pw zTw^;al#!G`<8AP1SjmJzovr1JCBh!!Cv2rZ?hPa)6)`KSNN({Kn(K=+aI~z3_lk7d zLtwmf2IH%O;5F)T_p{=0Q7dtT+2t$CiH?Z|pf)hr0DiNg-o9i?02%`DaxE8lxvg)) z;?yNoBvNR3wxSmfE2*k*bpV_W;N?SiFOg0z+b9zX-$t$~n6st+_-fmZ=(={Klw$+c?id{h zc#Z?B&XyO9(B2fB+lIqTLJjE1z-C2I%H=j=gE|K*M4fZ$Vl5XGARs_b+@KCm4)B&~ zf%||-gKphi9|*jq3-TCGGbA+~NM=UsYf4Q=5K*XVL(-vF+FxZp<6x^BG0-rVofdRlE4KQ8@vr|6s zSpI5S$;9RXkYi#3peN8}7|)09|vP$9P}$He4j^Hw$`-qrRpy-R7-kAc1*JpYy2qkmrl(yUM zUSP9Gf@`sJsO$9vBhr8X-D!$VfFZ8O+`zRCpqdXX2~gGpkQiHT$Ha;O5kNw<9m&8P zh<1M?ZA|!8jl$*G$?E^pHXc5YtZ^z+oS9s$$b)z$A&%16DF_k>rX19@cGg1FkUu zrve*}E%@|KZ9fQWvu!6u#Dg2?paGDHN{U@FD}*T|3IgQ?8qu}|1zA8i2O?)Hd|)EK zrO>)?x|iB^77!J_<^KQTSAfmt07wBcI9x*$bewHq$pcV-OE=Ov)#RGHI+DR#wH~ur zKo1iJ%cGGk6}G6B$ZI*Q0*FCTMOpM45Z8%{-bg86v$LIM_7_K|xK0Rw8!G`MQeXl2 z2V~4!2oU-0wB^ccowYg21-=IBFNI{AeyzYPA=~h2k46fO+u@a~X5Eh=4 zL=Lc?Uoxc;z|3Zl>tXz*_#EnN*Ng!vxn^=R2(LDv9sqg)-~v5fbHOwh7{HiFCtM3? z#v9yrjVm9K^^P4=UZ5(Vp(bo8PjstXqX{-EIV=H8{G|^p_3JD{4FLJ9gZJ87QWS+= z^HM!f5?~pzbzmU&p@3!rV?kR?A?eq;3APW2B+v{%oI(p2rK2Tk2+ z{MAWCfe-*?p#UJDzE%D8Et0{TE{ebnke3iFiyJ z7^P6s0cz{IBgrlJ@29obhyx^*909cbd!)YBK!?}$eXq6lx$AEl6>s(fiWoq~v_T!f zo4}u{S@%@RfC12Wg(V`h4A2&U@Xxx7{id4x!1X8qjSVP}-x$WSP5SHEZ`w2ffJ0`l zmBs(qM{o)=1(v0!PZ!3%_BL+b3XJW4M}UlP^f>d%5mkafT?np+GBle_>|KK!N}O diff --git a/docs/images/quick_6.png b/docs/images/quick_6.png index 56854b4219557f16fe0390c4d0c1ec8bd7a1ae90..e4142672387a7546c477af37596bf1dad4314e5e 100644 GIT binary patch literal 171128 zcmeFZcU+XqvMBnIBukK-1_cBqNzM#{WC1~P&M+W3%K(a!lYoE{C5cGRIcJfaBxlJm zLmbk~owfGed!MuKJ@1`&&i&{9-fHNf=&!rFtG}x5>guXN{X(q(q$&!^3IGNM04Sk< z0MsfVBkv2b1pqZQ;4uIIcmPHi1i(f^=qdmmIs*Xa7tFtx;EI32`a6t;ZjZ)r0EoV@ zcky)buy=7~7T|dTh{`IfVgG>vo&JQ`{sc)U8KY<8fJ*G0H~63S5wd}(HB44zIXSZz zS{e$<&*lFd6efj*tLv*j8ap|Ax@#%QGV2={GUILnnCS7~1MC2vg_Vcv(-$wE|AF)G zfB(e)kB{@|KgSMCaQ}g8_VQU$f7-z>>(B}w4>Q+_e^mS50|eGK9#-f_H|T6@qR;;L|Nm(6wDCpL0RVC?u72(iJ9|&&r|4O- zVODmw;^AWEf5OiP0DsK+-#R_{_2*n-P~rWv%#{iN#Lxr7%Kc}VC7M&}(DO!%|Iaek z0stV<2Y}i)t`_bVf5V6V#5l+Huod9{^M4G&Ism{)Mxjph008$90Nl=?P}g-R)a_R^ z{y6|>KS#9#WO!IgxZ+qC4**Ou3@kDXR5t)Z!#l+>F$tx%-DXYBH*3s2NlV@dZV{2y*aq#f;^7irdd;RWx zNN8AiL}WtZ$E4(xPoGnBa`W=P7JMuGURhOLQ(ITx(Ae44-P7CGKk#FGVsdJF=GW}p z>e~9o=I^cTon83x$?4ho1>*AR54tb_tiO?k{`?zd|3ViTnl4OiY%FZtKj^~1^g(AV zGHjgte0Rv7YT{bB-Fv|Q7LP(UKD(j=pG80mMrr9jMnJ_XxXK3qgS0;>`_B;;{9mH% zAB6pru6aNX!2B~|VPaz8U}0h5+`&PoJ9u~gNO%NzedTjIgX$u4A}IR&RZm8WAo^(;>FjoYe=qd*cL~*H3&3?~QZI zGkx|m%~qB&l0mo^#0mKRNZ=eS$XMAj+8ryQetGGabh@l|Dp8PZ$)YhY?vm1y5z7&wl^t`#+PfanErxO4=MXFn)&N7-`*l?!}vd^k&S;<8Crs-UcmA2AwW@80aF{ zwjJfx;|gybd~Ll`e+XM8zBSQtTe^n=;_#43zh(wiJ$656%2TrZr#k)54D7Eu|G5|V zAIuXNHTn-e%$(w}ZiwKTT5|uvf`{+E@g`elORQIFyQVhv_&2AjTcg3J2XWq_lvir7 zxUSc{ok8Rpxu)CWj@-F<)b5Mlb8=NW&S_bbawEv(sI+<0KLlF7DO0ECEenTM>zpYs z=N}<8ZpMx_#8`SYmQuL%OwOc>5dv@&utwb0j<7+rSWq?bygUEK9;|9)o3iii{L&|R z?BGMTRsnXvl7g$z#_}ACPU`KAGHgc(e4zH&fZHqT(I<(PTvMarW7(7VO+^}WR{Bqt zv;?KDz$r;M^8R%5s|k6pwGk48>a(U;d%W1+5=8|uxqN~u**VX{VNKM2MErzq#=nbJ zvciy$#^~i?r78YgGtp(=_T~yd+crEebNm;BmP=Kuru_u%7!Qi6Jrh5;!v_z*gWntR6Ems1Izuorx~7TMq@0cu`KAQ7M0^!5U6hI@l!i z;Ii&}5|g3SkAK2PIAl~pG3R16q-{M8afX)22k@hSijf0eP>&6Q6^Y%%#P84YEeb5Y z(09)4ofcQ)*JgYC?k4SK!R1n*ZL?4a{A=PoZCsjV2aI1PiR%s3C^f7($Y_3Pq(M~M z?|OXu{gL6UraBhYM<=c?>+YM^`WMudSysOva~`39ur`>6S@hLN3{(epvZkBjx2hJT z49mX=ul~lO#G9y{I<2%8J13U?sVBt`vqH~SR*&<9;F!_>oaeUSMhH$QhcGBY+&Pd` zT@$sgFRzPqgsi7>sIz&v5tU)$9g7^rdpm3WB5F`+rmyMUR^$A2`8x8fEgFO=%?-tx zacUNUdvqoG240d6iE8AE|=>a zxZu7om2^kHIcIwL+6ubF=PAiNy;Hd-@A6b#Z9eR8(=))!*p0COg5t|i)f~WtP^V~w~iDT?mSue;+|k)JY0V8LY^GX$i2$2_5vRyc2TABgn1RWOzL(Xgx6-`MZQ`}C*yk+tT7nvp)s#8toJL+ zg2#s58Twg1U~3%({95l!F~d7ViUbask>U1m?u=eoH0!sVP<&U>0H%|A+YR(i^Jkg-{VvUt2d zRMAF$=?j&OS#vwN(8|@jY&4xJk=Ht4^N~5iE@>f{bO_v~%fnYs4j$WLl^q0;m&*LY zz1b4~!bKIbG1sB|n3c0W+D>TFw+o5C1#jvA-Sa%4Hyh_EX$9>ran-;U>%!Z{7NeiJXF4G6uUBF2q*~fUic*EwT zR%*jc9xgtwXu?}hR<-|~V4swJK(tGdP!Hzc3WulcTU~2F!HCy^j}Tq$)Qv8o;~c+} zeG(Rb2aCDY6x1jiu8|6y_{En_beT^nRMV;PsYFjea(~lt?#2Y2v)vX-^1_aTB2zEk zI2YfDIohV2{FtHox<;=@dty^%5HpU$(7OFr2LBb{R#k{>U#?k-^ZGiw3%cu^;AD9# zIIm+=^hk0rrOy8ge_}aVxRp1=h^$bnxT8LIUapbE3cP98H>tM(W(=}4qd|_C0nHNd zmur&8JFV=~LUrS%f(lADs=hac`LQ;Q!4Pt_JyKE4w@tJ^;>}22zMo85d%jcOq{O*b z_Dd0|Uf14s(V{)}Vm*X1!ECvP)Z5Q6DFro_a^O-lowC z1H#S=i*7PZm-0~m)v~!5>&lsC#kHtA4HM3iPu+5SZAX-969N>qjLgmwoFD}CVhk~n z^~@XbCML@VCa}iA&SWO8`mmS!M_$T;5N_$-z|>X@1E zQs4ygZdKzub!kR8!_3+^=lr}9x!N+2S@ z5jTqLH!Mi1iRz}dDxpY)_WQc}#eJcTBAt%tPyH`mD!SKBH_-3NIcu93? zOyazvkL?@kgoT9XZ~Zoi7#N%0d?Cr^D7EUy%NOhes-tNtyW1UC7Lf zUu?1xbP?sbOQ-8)*)xg4rN3ao5YLye!x%tZ947s!iEtkr>7^R-}To4>kZ$)4EVe5 zBiKuMc*|l^eqY4de1CP5?*npqTNfT)we(kV0{st+U4lNPV`9**U~TVm_TF%=mC-|x z;Gi%jaB|m+`lbmp?TVI(gNK)V8M6s z%zezAfot!`%fa5D>Y3mheE+*~#Sprlr9!GKs{BGD$a$87)8i$1B18DQ$6t(Zez9XE zc3^j`#myD0mW7sabCn!;nfR5R>Ywd-kM9rl`_nR}xY2*imMI=-e*FDi@?sAV@JJpC zUm2P6Sgng~*7)jcrWmCaayAIz@!V@FVy~-iIVNr1!H#|E6NL>PtR|JQGIsuaZ4Z4s z_40WtF=sd7-Rg%%kM4dtOrd^KVOH5`Wp~1E75&&oVUy^GGJv(0%DG8cHb){452$;4 z=7Q&tGDq<`C6Z-~xA;(Sd!rtbgr6|HYWnm%s5!4c=d6Ctv1JwfG92dp~w6<{- z7v|Y^M_Hm}aynsO|Ni49c7Qy^Oc?qBe-3|y0^=-(a?r$gqD$A5*d?4=FPIyLD>TQM zRzE10=4SL7k0hZWq<0@44L5!10bd$7!}BdLdiQklR83BH%-Ji-S18Sl*+h=wRsT+< z!jD6MpUU9Q>IZ%ftmW;p)Id6@SsPry!F?H#D-Xv`b^bInQ~-HxvluhB!au-}BJz@J z6gJQ(IHzB@B+dXPxN(5TyM#4fMDbKRmawcD7ZKOwzjx1Hd^bd>Mzv#VXV95M_WYR} zr8fIo0C`juG=lh60Ex(oNwDY5s*-Teh5 zYpKKuOtp$TR6vzxZXQE*zWBZ#RGH#F-DYW<;N`JR{8<~UHNMc}LqX<}tQae`D^S;P z=Z$ntPP0c(7ClsCRqge)403SFD?-rMGD|y6z54o@xaUjae&@TJuRdO$_m*BuJJlCE zNvtVGdM(GMm||B>Y}LXJ;|e-FxPIg18GYutK><8+B3YkSZ%4@vB0hf-J}3&7kvfiq z^G1Lvf5T#Pg`@JjXcB%*FC8B5`fg_2H`mL6sAqhOs#B5tb|NsHa}6Z&^>E zU>p3@jQ9qNJ0@hRHC4!H{y=s~wSN4Ot-X1usAVdqp+cNG^HDtmCUrimz`)7`%m9M#fa#CpKxh31t;9NS(2T0*;&To?~&kKrYOc>$6 zXouE2vp%Y)8L*p+4cy`SgYm~AHWsRd0f1ip?vYMQx zQK41TQ0%G=K0}rOf6qoM8NMy3v#~5TkbN&TE}91k;z^-OS@}FSmfaQ#v7Ic*Al8THO5JNe_wbR?RpH@W*gVvkN@yV{%4sOW zMPyF&n$k=(Yg*mwpvuNe{E@Nmi7gfH^tV69$&XO8|L*8LULKPAtPTB@dlM0ThxRUvHGgp!dHSZ~I&r zw(NDeW&gUYbP#un9bi=*IFy{06@I@^f^D9HLED&t0wT@sL6uj6@HL91@=M=I?0L`G z)b!>A&N!b}DZMTI>3yt#0zCZm>xwoNUPl5K4xC%u0&Bx{BAp|Yh^^<7p38I&bOb6| z%jy-46VdL^-lj@~^E}KnZ-*iB_o@R=r$kOZMoq3#n|+*CGi0(@3LgO+u@ zp^Rd@*$ST3CoaDndxOgqohs-W%b= zoK|yqP%jAudINXCF3ryI<#|?7(^!pMR%WG3p9noM54GP4Wj)q^%y&nD%bm1TpbXB^ z6GSV?owL2fJYhFl+?(x{w{tSPV6dsQm8XnY z-G(aONnr@&Y^rXVeCl*OYc$}9L3`=Okf`4Nc7~rJ(|5;EWL#2MkG$7)hrV3;J*GTh zm=k%ac%Fo~UgLc`*Y|ctW#ZtBSj4ElQT$Va7?Wpp)v>90Z56#W=fcQ-UIwiX@o2vEY4Q zT>{o^id@zTcwnN=NfjV0^;FcJ$=v4=Xz5W^>%NET7v$&?A)GggzVk{tt7t){ZhYsR zwh^^Bq$NyZ^`wK9Y&J4PjK%@Dc+J_4mJIt5tig3oFz&+6Tu=C9nUQ+cq;vJ<{fWcU z$@ooq0e5AvXA*%wQ~icB3U~z$5w1%iW6P{?vthj2iFY9Tm1yDNX=bHJv>BYx<{LVv z5?yMKwcE(K1*Skwg6>MPt(nNlOmf$#Mww8p(VJ-K)K)<}hot@b2^qUrZVFhjLd@~| zOgX2k<3Na)CUEbX6(=6pP%+}Q1jNPScta<~dE#ey1i|}&!I2+*jF(^T{)pu42!d!t zy}Y48s1+ib)ODP6^;9K*b$7zgC!rP#4>+YI4mwswaSMH;`j`k{VJRp8-z8k|_TCV! zMx9BIQ@v;ZiGc?~_iJ3Kee3-S6Akq6&Ll)c4N7d3b6CCazJGT)b25|d`wA*MA*l>^ zTq}z>JmBIwEIG()(3f8J_Y3fANfsU33~`Oml}K180CdFGbH{Jg)?H@7>#XNXOjDt` zPPWrW9C7Y*ySmhTdGO@uEu1x1#h3J+?6l0FQEtkV>^&;!`(vKBLC-DEUuQJyy{K6E z^i^hCW0K79BL-^h?*>wFv^j+8HEy)0lbxo?d2))TGXJ_*_>(2R<;|hMbc8VXLkarD zhhi2Vewj*se}Ge0yxbDH^>~1VDk7Qnvhyu%hg;-LdYK=>1HNtq)26zzEp!m_Co~Rs z|1GvEH%0z5{j00@6PCBp=9cTJmaxJ>Lc4$ousUpAYS8s8gJ^n}c=@x*ysF?qNoMT% z>mtM9y(2Ti?I+Jl){HjqIf{S3Of;^4D6hDAUv~3HA5A;KU4DLdbqBvPdZqefFDV_X zr(v;<8Q~0kT?+HwJbJ--<|ok%3^)5=M-5_gJJ}}C`MCIM>wPr0GQs;o4<#j2P0qeA zJQ^^tc+u^iT^Z}Wg?;c?oiE*5Z>z?A-^8IbU%zQ!cVt&Qlf!kI5NqIVcTYVx3xvmR zt3a>F|7^x?5;^`_FP0sqlM*Gl(Fn_5`trcQqDJ2e;tfuWAc{TS)YAB5p}z1oTCdw{ z*OUuUx0XfitEd9A%@xRycz!%<#ozSBHz{6)lbqs}Xdkmjl|w?!X3eRjSXhr%SG^U5 zc0)#ki?h*=|5@$UF>koRE!PNtGJkZMma6~u!FS`KCQzXn(B{B9ivrHwZa=QU2FAj= z!SskOQ=?lCW&TBonMH6%*RQPFVxmev_ooc~@fBSvmn<}3KS&!X3b1RNqi)ZtX7AA; zM0DJ6QNYJ#Ezi&;T%=|=@J@WLs`NWiz#>awah(QU-f(hq&=53-A zRBjaO74c^tU;yd=nxhn zZJV%@3we-c%#k;^Otlhh-b?~NIuAlHr7pj};e{u5Gd&SDBR)G|>e6hDgh9p}^(yDn z98Dfh^z+a09`YY6^|FM+)}3C9bdbzDu8_RDc?5ge705f&J+Wg-1g8!Y`dl#Ke7HXT zyCq8_c~pa`M}$<|u#&&s81^NJRtVuD|FTp5a3Gdn)jBXba1jMCNi*E2j|S4+NF#iT zA>y}f!P%40ol~j1>#`HfA}*w={`x$&N$20YQu_MU$MZJ6Uj=oefcc|tCISQrjJE?^ zr4g6er90|LQsJA4J@L*@?@5=#hp*RrmNC^4p%#`>8jEEEk&lz(Q}kfPf!{V+dF;*BKjhcY2|f#se+T20L} z(bNF%fVf54wpV7eDSd5K1)(UwCWziQ`!s{}*HPfkT7Mw)__4tmsg*vX4 zP``jbtaGvQ&$)f~e8R=)5Y(?b>&iM$hQCn-{<+Uc^B^l%ngu!2R=Jq<25OXzuS;O ze}K(8{`v$CIgTdSGW9*Ec6z&s)vNqBUOe)o?*}+{E!o`Mj6=2Ze2w zB0gz@SXSWeru5z03Ta}w!W-MPZlpl`3S0#J^6&b-Q#j$~%_HBT^R9zPR(&G{@eWqi z8M7RG_>QB)9a7fwgF~Mq-|=1VK+~_=ak$HY-Zei|stK_PH|q!@WkV!=ztOKB4=D(< zMbl073xWdBw#J3tZ)5LW7H_`Gev_fuDCO;ro6|eDjM7A)EyZ0yFE-W}73xPZFFZ%t!;ioUk-HvX9l$eAya*$<6yHmpT<+-kN zg|O9`9uxLXY2k@dP!E!zVO%gIfZZU=EOljekX)|Np-$F{`2m|>UGp8@x2+%iQF#GNYejZ`113@|GPCer>=wd=_q1-H^`v%AD6t0t6&5yZpS1BT3jbx zNKx}}@}+uX0F&s~HvRPZyu-212^VO(zO%zdk*t(?`>=nT6<)Xb{=iUT_==s(P(NwxUCq`nAb-vtquFB z{d`=CpFryQt0P{KgRtx;?oaW!Jp2FT<@SGYH~N3w9rY!8r;9|Wt$`w2?@h44)NbhX zA|=7!I==19_z(>+xG?E-@v}x_yi9juuKXrx+xjNw4cZsZ=>ml=F~IhAX2jdx zK>L!HE(=r%x0*4^FcL`FjQd3tehrH<>y(I`L*PRB1@CN>s=0-?6ye6E%x%G42dXX` z3Qe-lCh(aILZweG&9AN44%Y zgQNpx)sVhL17J>r!9!b8=b8tR)=Pe)P&ak9%^KA+ryB-w^!BOgXd0&^RCQx&&$fvd zo7;C)URi7Xg&sOp+ihkmAL4ku8~F$kS7`6HchO26L<}k4`MP2l;(bDNLBi_a!DSSp z1xSf*jTa+{5Pcf(tgax6ITOe=lzO$WI$|pGW$RAKw4i-^v~xKBu_~w4m!SZ#JmB7n z7uQRQ&{L1<8?v0c%BEISyUQ)&ECH=xABy=j?7zieSwZsNivdvUH#$?RWS6qWeeK=o&9QBqh{pXtl~4?HPg|8U#mH&b^hY~ zreUYYH~5Bj#u~wH0lO?aIB0&FyI7iQYFI8cUni(b1Vu8D zO*i|&=CxoM>B+EN!}qSl;{wGP6fy89+feq#*OLP# zPs8XY+!WXFG$MK*_hO^_z8%Z~it>P8U5q5wqHH_0H)EZ1kyRtX$z@EUgyz_MuH<47J_U z4JV6W_pNAu6AFwKCI*;BcgD*#s*Lp@5L(NCbah>sOlLrtl31&Rx)!Syl=B4SX<{g3 zh2_IKSd-T?={vJ9go=Xb?B}MNauANnVG2!7vTN6h5$nxUr8$ry&>tQc5JO{mHiW`Y_Uz_*(J9M@MHjf2FQpZh|9pE1^RFW|7EU04gKOqr4 zUJlQ?8_4Nf+1Dd|r#wwQh}gH$+`9R*A^kiDqm{u6If{o8DVNHzit+l^&ckD7&(SYc zY8gSjRuG0(PMlm;YnBqGO?i#Ty-#)KXT_o{x4&+rZ*==7Qom3;T%iNN;h3*4bLs%ES*Gsg); z*t6RDhPug|3CRI2Y;J#xfL!ANFNkNnvACk7({Di!#U{5GO-~0cq$zIrT5D!(;H4Es zyFLysJY~7nlf_@E-Gcmx+%#MzD~&AVAJH+q!gi6j8cW zKP`l_O!sB-kf`p!;mJ^orFSf2kKf!+MU|@uk-$dj>5!T1Xmtto>TOFjnZ7c_Wz&R~ zisKX!nO8)ejDJFPKfL48$I6FtrSYQEECTKRV1|LvJyO#k@I5~Medym5@zCf%%8L}| z)J>+M+QUU+j^8>C;!Z<)C$J#rjU-7=ASoi-N;=yH1{XAHVbPgjo_KHg!I!OykEQRo zODHYzPaB%RL)m(kO>=z8DPN5)=_fGhuv}jWMSuE5knE^sRj5&D5|7QK)09i|x)Bxe@E*3fs@L$Er<`fEY&E92K*>Bl@M;8Dq%cCf+=g@fYAZfiS6$E`3JNleXR zpiq6H0W5#Q5x*plbar+bQtn`Z%|FZo9Xd8rYMQljO~AYk@1K*N$ay0+g6Hcn(Xm7` zA<1XMIYO0&DffFIj$aq+cl`Mq5eb_#fGF{;@lwbtv8pN}V{MtvSd~zJZ`x3;8n3ul z#~;hnnuj0+xZJBFQts$yup2{j;xM}rMv(>{WE@`?c%gMGnr@F5N9*j+V zO47sk^;h{H;q#GKejKJ|k5hz$-b39zi^5>Pn<$ZA8#^7m$96}bP48f?1Vk8$4)(sW z&TAL^-dD(=S2QLAdbDeX)9kShB7w>^!P$m0^7oKpP9ionRjM8~6o#62L<`~**i%L> z6D&!g?>VrEbZ0`5w20#qv|TmYYjQ%8R&tIZmAm;hQRr$^|7y= z6{|s^K7mvSHtNoPUh>^z#)poe>L+~N6F*}KonEN)G7bZ{gl{9y2{^D)VYV?|tIN3C zrl1yU{}}$x>1o~Qgpkz;&Bz1{=J;M2(S6R&eJB~ctwY>7cf}>F&QRo>acF9CE->Hw zgm8$TbHLC}qY5vRGc;ncImh~i0#+$5_&It~q!DQOqde1{to9f61kv~gy{rr6I5-GAE`B8a&K%RhD&=!u6UhX9*`}Dn4#dk=?S%*hH9;-ze1j^3NsvOBuJilpe z=4LYjdf zn1t9=N`1-OjTN@5o3p^B^6kgsg|kijPG-^(dXMXpro2t`I>J*#Z;D3mbX&)oRqqr9 z__$juvzAxYT?>^o>+20f3ITT>=d$C{P{w2j_9O3)Ng4_AI;}3Mb`2HR7BJraY6;L} zwp-r7(@a=?eQKZz(khTpTt+Xo-E?95tLxWo(PX|Ss`X>dQgn`---ZfzT>M=E{A|k_ zN}RbdgiDXDFB7(Y53nh*Qr6w3LxVuQszD$Fja=W?ON{!<3rjovrX612=3KbX9n<*L zN_Y67vlxSRqC)3q6d0F#1;{}VRokjKY%8MX)=p(gJ!Fle#-N7wuxw+Jms{85nwHTJ z?=uCC{yK(+k1=!!=bJqfcg;jG5)+|&R>LTv0oT^kZS=6eDBp#EGF7F9y5AUaH~UAN zP*hkA%i4j;(R8#gvE3#%EBdV;Q8>JIstc=8yVsK%b{;iS6i`vRpQ*bn&PvKJ|2TO- zZMgnuBCG6_5X@kJDAJG87WcV%P)MBjT4g>$9%lVq<2+&cK7H%&rA5DC|L+_?8T!h| zDGP8|0wbt>8(Gyne7=};eaCx2eym8KOpz_Kt1D&zUv^4$;+ppAs{8;!{>kxTQe3nT=X%1S0K#l+1r21bzVqNW>lc9jb z5*Y9GTpTj(&~d}?YWbYPGw!?yCeYtOA)K9{j(hX&&{2FDyM@>u>UIV$A;CuheeZ)* z_8s5CGII-}@J_r)*RS`GLcdS|1hRZ`MtytUw*IFJk`cB2ce6P|pPO5G6cA8`7`x3c z2pX9*yO=|tXOsr3b$AcBC6IwRD4=!b#vVC7(YCeAdwmep`-W#syREwV=<+HGd2I&I zy;fC60a@SLj?f-3bU>55*1xG6_iu4v9#E?OyUO5yV);V&SBHO1>E9UaFQ)kiC;TNA zf2}lsUw8i!i@(I;uMPMw72_`@>c66wq9*?P3iR4!gJ7r`+FAz)tJ$R>iW? zHVe?D_%bF6X!%=NWeH4mbULgL^zy0$mhtgnr?iBg)j3T7j)&<#?vMTm5{%c_e{w}2 z-9_@H=%ZHsk06V%QO!T98Rta3{Wp&^T_4?sf}Se=vN)x5kL1Atmv?P*ToXwLmukwl z7yQuxgl?(xgQK1TiN=-t2hs+4YRoKj1Bw82OFfR9m6v}sh(ou0yh&k9@K;xVP2gW_ z@t0ivU$k0)QIr4u_4|KgQ2E!Qf7()tm;Yf)iG^j|_$r>3_O|L8WoZXTaz-H%Q2^^v z5BQv*6$OluXbF^`SR${Gp0FS47_@r`y%oj%9@q>LAHGKN7RfM$6y3cFD2pk$wwqti z(rc)R_g?&2Zp*Q$W8BWdf)zjV(#+icN+k5=(X}rmkO{URHYynOzQS;}o7ENt$ko-) zl1I>_fSX>3kSp({kK*p+Y4u22>N8p^Z2x$Q%iu-H7N`Mza$5|&2LqqmAq+}DGOr*T+GAAVwYCZ`q*!qS^@}RUFX$s?-qFN#EMxjDAWgh zw`R!whnk!c?>G5xY?GRtO`J5j@(TF$bws}&>jbA;V;z)twC)UX2TBAEEm!k~1!6(j z5wYX5s%yNIh<9tIb&%|B;h3lvh0UBQ8e`6nF{;GJPR!N-A?3@fu~ms9vfV)Xi5d_! z#LYgn=O-uIP_zj6l`4cji%Vm(oAvP%e7rjz!3y}mHa-ErIu18f0|nqvvIh2|08%AU z6!1Y0nc?&}B?1gFDhs)y?)>vYm8AK;8#*L2+?0K^_^_n@9h4pq2X*2gKx9QrEa^`P| z{*9XCUYV*6HyWMzr`C%h?MwBx1AXtI#J8nTe^}GGCCnw9E0EMde5V96b=wdB3+lzAM%9T{$HYBH9^3hrVeQztxIMiZw7CDi>=Uu)uHyG^_ zx{4jHB>66wpM)81By!J>EUb6Gk+)#kyx~Sb=AK@v@8CLaS6SFqo}oxojZ3+y8o>f< zkX@#6_s@QMK85w~PkKH2)RT6+E(@-$QNVyY`i6O*garA~Sg+4*hID$6|01$0ZzI)k zY3@o8dPMR2?1`wNX@c4!xA)9g?w$q$QWgw4X5*>E8MiuCf8WL%>Ki|-g$(-4LYne!i&p=CXpbNiwZaimf1@8F!d|E5r998TgAVz zDWtKetC<2RB`;Lbzi)_96_zncQyAw5i5PID?mW^!8mW4WJS2YEDd!?&N<}g#r zCf(P+nYHM9KAsi9AWxBAUcPs3Gv)!V8*|(yUzc2aeE9zLX#nN&+(F^hQxrff1d~IC zYTP6?qr*7`gSyn9Xz$7FKi2>-VhBq74LO387;k0X$-3)eIK?aU-jCb4uHT83ELUj- zoYKZw9V~23EF?Vm?4TyeV55RT?Hl=(w?H)EzO$Vz!DO2cFk`jm~kY~jT5G=b zW%HbDNbN#?tMrwG-2_B4?&kH^vhlcTNh6)xY$S<&YX0w-!abzg=KFD@5zs78nso)A zqH*yv2DI=qp9{(Td;W4fFZ_Tq9j#xPovL_At231ea`hl3)l ztF_zG`j78_91zrU;?jN2o622zpbwMl=>tEC z3RHNWyl#>bs%?c|#jIpSwBjLR&X*B#Uc3h+`zU~SWMa-QcGQL z-ih8cAQ=6PCk0aF=*tJ-ul(Ph11!|k-}MdgEF530lV0TGiDquA$3_D|4<`ga&r&$# zZ^Tz5zL&QZ-NK5z!k}L{-aS}#xr30N?-A}@WLJ}2JHE8D&mGj@9*9{{eMwiurxU6< zYJ*_!l4d}BUGB&fm}r&@AY3r$vZB6?GNUIGshIUbt&={gD4WKGRh%Up&0IMqKl1k3?P(~t zTFK#zTkmdO<|3-pb^J5v4hj3Gs%lbk?^0F_Peg+pzAb-@F#2>QqJr>%z3dGXZ9>Sc z;tPEeni3H{r>mKSNo7oJR#uU5S3YCnu;wd|Nmg)cP{rymWcDUT=Y4p>c9 zv#hZ-@hhL(bCI*h|NP0qRU9fn59Hyt58rj1+7D!hlEA{Ey-1>#2y3RZx+kJO?|Tl` zAN0f|9n4iWHSgT_vyA&`MfYA;6eE1kI5P=erapnZV|*j!D24)7A7XB)*R;wMQM{Jg zhbQ-iu6OgB?bu$8sF|3qD;VY87U8@D zmOMYan)`D9?qHj%7QaA`gCn>~BuY?ubHWaOH^i-2zF?@3n4hpPuf_iUy8~>zn`jsN zV;yDemCR4=E+?U8iw9(%QGj1e{^ zYktX2kqytsc>S+w45d*NhdfIDVG%QKWJ$#J@hu8ZL?pJQ^^H6_YuaY7*Qnm+BbxME zBJb3}Y;Neh#$pyzA+*t z{(5#{KLIKoTOHOy33ZtbP)Wes9ORTKS?vsz#iMA4Iz=2x=dZZNS1aOwAR`B1@aPSd z$>Y>-ckfZcfeFm44Fwstz#q1t!+W&um!7G9F8#`qo#4YS3?_HjX%IOg)0Brkl&8=% z{1if=DTDJLX4b*k?zKy~ukT`HqT<7AQ*C!WAsr8GN$(0oKdIK(} zplZnx=}lSzBL6_AL(KK`nYlKqoBL}es^{wn%C!Rt`Ig4f(-^R+>O7((&tFwHb=WJ$ z_Oo`L&Fq^O-PRv)AGU6xeFE^C@vr%BEMOURQD6rWjP1FMxbC<+hSuZ8$X~vqjy@cZAYN!c#X7T3R*_YCgu> zYj!xjPQe%SAy?)gc&1X^^TW&hCWL7QF4xgU+KLHm{PT$@VCh35+(I!I$N3W4Y13j}3wNh)gpoM?+|1 zK*rk63M6lnEMm7ZcVDDxzK zJU+fUnhxDa*MdohV5O|&) zsp5@}ZZr1{I?Y*YQ{~*yh+77rR`8d-!h!c;XPY4{H7)PGi|e#hVolhR1~4ADMT;J# zZ$+X*!MmHQ$fAF2DiY&9^57;rHg_r{6 z7ULz7EaD`-!>qZfqG~Dmraf5Bxqba%bBa=E*BX153+avIBvdnh7LmH#?creRHPo43 zvf?#&toS&XRzJm}oar4imJWC|rj(x=igDivt=^e1L;)j8D+!x)%v2(*oWi6cKQxTJ?ld^5u4TQ4y1-P8CSBNT%i&)8 zjn;ZFT5^#lf1^(oZgkdEcck^-9v^gEYTxTaE6nszv=7CXuV^QgphNH_Qyjv|C1UxM zVhjZw-J+kfOih+_s|$QB*GN`zSYC^abTnB@O5F1a_>u8^vG>JQUE^d@i%!ZAZ|nX= z8Ymix-+~w^+-72(EG`(|XNx>}HhFV;%$!u5AQ9fYGB4b^)vGSD!B9IYIj;hD39*8W zy#JDDP-)=Q?0(^%-4HG4-Loa_K|D)#EbVc8EcUw|1^nxP&?6&qf=eKSQ^B_=09|3@ z^>`^~_;=x5^R#J=%}4U{!wa>hx+a7f&^4!qRdm@AWPa0;msjo=`m7k&F?U|GF2PG- zHrrDOVb`r2uc7`-##6l)5yho?UxJ}fd-mOrbr(hUgo%f+ux_V&t4r0Xy%C_B2gwh+ zv?gi>nrjFu@@%!EVwQc}(?h@%f^SQ+l47x@E zZYZGtjK8Ftl01|JbEEOpWl2jf(74dAWWY-Rl27-`)aG(GwJks4#3)k)o~8x+S_r4t zt%p5#Q_Rhd<#^6n!_Y@pR2e6`vT{KqwRbMtCWzJ?{yle@u=`Ffb;Kg;m zX5;_}01r}xm{@1sN)PnM*kC#NA`x*#os({&9j}V}Dl8O8UT5W81CJ$TH3WC~?&|Ax8 zItn;sL;-a|7a`DzYl#~=IJi>-p#)xKZLDfQPF=Gnoh!zlUotr#ELpp)OPlu!EawMs zo3;1`q)c~ioN`#Q3^=?}^eJ5PWhVApZ57pedZzYLw_gI*ARScxf^&2w^q< zp39o}1~^oET-g{VORH~9*3UeOIVS(2eoAmqbamz@C@Gztei!MN3M{VI7@1o&%4nS~`bVTmKS3^QCV&~whz=DHCMw3j zW(X#cB?iPJBWYLbHTdzV3f0|Lgk2}n3Z21 zu~4!THEb}+n|;BI%$fW8DK*mm$cJEgfHukLXXHcsjtTN6xo7hg6%J z3*lN3*9A4}4tAvtJ^S?N5Lf)iZofuxb{~swM%pYBh8f2viP7R>yKOtDpc~~1$g)vw z&I~b^Kac5A#9lAuz*Ko0ZTR)G)XkMPwd;Vt!}kf9%16mUf!d8&4RYSB9(vz^ocPb0 zstp+0)`{)FOREI2TmwtI#yWHKZ8P|3Th+Mm71iYo9?tdX?Ulsx+mEfE1A?Qbk&T2uKY90cjGd(t8(_-lRrq1VV2D z0#c;cARs*ffe0bK_y5`Z+4Ii3*36nUYd-8*?}z(KSU~P}U1vFt-+9t_V`xyAxLx@_ zFrBvxGh!&!?!5Ax+fX80&jqd?a5rTo_LcH`vg0L9r5W_>6kjdzPvFo)m#_S| zN?5rekp|PhkJ;A=96D@wJ^`3F5VuYNnUPVi%zgUO{C!K@@& z-4bUwYcoa|`Su5n<2Q38H=t>J0wT`O=xp^ogWuGolt(=)D_1c2Q_NCZyUErf^BZ1) zbSgJ1QU=FVj*ieoJ$sd)v!JJ_=rvT5x0hW~+w?0oXK9tEJOq;&A+a?oP1S+On>;3czjmW*2O{IY`xm4=6Rcokj+5@v z*)U*rv4m6kDVM(fJa3^c#C=Um1h5>F-67)q!2YLh@5N_;B6D1B#u~M06u2}5Z zpzj}-!Hgkcfe(JV8Eri7caUdxxk|F`j!VYzAImlZ58PdbcdRuUGd{FxP31uP=t!Z# z94*+$lp3&&^NbGvxc7}n?M%UyOYCrljW!>bcOo>i1$TsbGkG{Uu-(4^_qHkV(xfit z6g2#0-Ky1M&VWLY$zt=vZ?;eEcRIIA9O*YrWXseXo6mJ-LrKaOw!H>aL0$m>jht=G ztZ)u4iDAD=eu;vy{6Of3NmQH_Q*tIXMO(fixR4%`FU}sq+vO3@Zgl-|DQiy;D2;ynmNRE0JN^ zRB=(oIHYbW4PE~Z%(vOp6v6y?S}VP)`g6p$Hnp{Qo@`;~!K9@1r#y_55iX-M=}sbF zO24)^h?LA+9Lp{ECYU47f6 znSuXKX=_<<`fLK>yRy)%5;rEK&qe^VJW)SgT^!mgRay7wWFRVvy8KR`m*@{4y=VFp zuP+n8W-YpkqK~EYWU84NSgYl&6_>zWo9Vs{ylJ4 z`{Dh*gd2}BAqNZ?q!GrUM8UVT`r$!4-n6ogqK5ef%`HCAnsl=EwliUE@W=CYF0PVK zEjp$Zlo|X`FF_vPqKn;vQ$2wN(>M8w1~HyS`jzoP9|bAh&$r#JD)-6~bv0`z-G5Ev z9I&&xQ~OE~A1t^J1Libf++~rw6%-BFOhdJr82GKf(Ids*#vem>MyVhZN(QC9H8|f3 zZq1&x8(bLq6izadmwsg73U9j+@tuzO*Kzz-SgpP*d$Z3?(@oAzmsMU!hLV-%4chyq z&$}15cR0BGGEj^OmSW}I}SC(J-5yqf*oEQg?z*(wT7-h%UH?=^kxaT%id*`K3P2ew!% z{7a0(SOz);4HO1P`a>{D(Nn!;gF5Cf(+4<4j5F|;*+ohZ7B^GnkMd_+|9mjGO6+vJ z6784!r?bJy-GQ*J5JJH2Sg(*E+cj5sMBWiBZ+m1HKWC^T_R6garON?`mQkjytOp(# z#le%^*^9hFG#!K0<2tJ`%b9@%D)XLhD0$C7^`De+v2OwLAk*-oeONs1|s4x_qTV(=>;`sy2r>C^X}kW*77-_1P-iL z%o79>>0;n}ge^I@biK6f= z9huA5?Tv|niLtEh^D-KxE=)d8m4;-0(7us0-?o@RC@S_k9R6DhNA{K8wK+cZJ-5KcHI0rJ2vbPW2>?ou=Gm zb#oPmSj7E)vc8O_P#U^rb@bwqgm2jI{Qdk=iT1V5xR7h;*u5@}!Kl;7x^@`dvz)`p z>7`ou=pEhUaaeME+k-jZW@5(T-G|UK2OKK`FFQqGfZ(3Y!$nJ;2-i17l!hIYQ-eoi z{g)jrC>+HWkLMNZDQXexknW=>UQ#6m0t?QiSSZO>X4La8oxtR%^qNfSh^SKH9LVAQ z+N-lY{Z+?O)-$q&AZ{VY%cWi5m0r`@(6`cB3Uy-+S~` zj$aHx(i-l4^&aJib?Wz3z6XQoV6S&NF(;(^dRq#vqzeW*-MeRDLaLFd7rXfO?MGrr zC18eE#LV_&c8GtTkdfbL!u~i|DX!@iOI^|m^oHMiHI}AWSsj{ebgfLDQuG^~;*vXf z6ezLa^-6Gq-8?jU_btEwY{?w^X=9^}ZJKtxt;VQ7yz@Ev(^@5CzpA61s*1F1XLLSi zhlIb%8gGA6^or=@DG+@Cobum8)xW`DfpFNp{n{<$bBiD6Pa^z07;ifWseJqyizX<{ z?-InZ_Xm@rJGdscu(SyU{*jXl5VmpeP1j&9s7{J|!4xd`J5cT&o965P!s9(1T)hJT z@9*!om*5h4SzPi+!)HyJr`Lb1Wr|icntO<^!eLC%yN^|u-e)|9MJ$!vL9YV!Vf&Ez z!9g1#6xJge#g_`%eHK^sBl2)}VddBdhSrVqchGi#%?s3X_vOwH70`V+MZ~RgcHG7r zVR^%YDFMD2oVC!OwU@&N$$eI1&-`^p@2cLaBunZW-Tc9sRE$7^?#-dzm!M@Q^y&dS zRt23D0|a8{gB{$MTmEFd3XXMtK4<=?+=|cJ+L7QWhqhdGtru<0e|~+)2)3)%CI-Um z=+*k!E`vCZ<)~VHs@ojCL#10~dDA4*XzL*HHdnz*(GyW38&S;ahFQ{KFg;;R zg-n?iH?i+Q;0tjBulT2#Ht4ohNtdMYFAjQ0qukX(?mYY!tdYdFJh;kxBgDZYjFAxXO-^-aK+4;r57I!_mBfTsnFwZeB`_kD zxiA|PGIKXmp#kCoLxSi$VV$?q++%*5R3Q7N4b>>m!c zWKs=l7vbqcv}-JU9@`_zH}GK0{g}TXDC8Z46K{t8DuH^o!CCXlHFE-0uU_2m13lB@ z)HC`_o$ynd=v(4q5@*uhc2Cm$$l7Z616j`eC}kcnRjvh#X!cfK;Lr*iP8+qDE$GP+ z3Q~Q9k)mL)t$L0#z?^`w$f%B6yOv^J{!u+uuXA9-sFY|@bbxp80lR(BNS8#lS>@V^ zDsMsuJMIIvbM2@%Bp$+wB0}d%O!clX;!-}f)dsuG=Ja)qw^?>DQ?;#wOLQ4bIGPWL5Bo z6TvJ?xF>9Q#r6587q1ORJ*m$c7%fT@w;r%;1m4K9zbO`c=3hV;WpKcGX|$`r+S?hi zkUrrrJ&-4f)h{h-?hGmGZ>e^>6>h#QC%C|r=b>VoYlZFv`#|Qk&fqg0-1GQ5(@Na# zD;%F&?|)ZKv(ma_E$63qmY!_I&MEIG#n=n_3A%F>2?X?#p#%mDA{x)I4sdwE;9tt( z3^c|=B{}BqN*QxeZuN2lhD1vr^pElt39caA<(buFAx`g7caN9+6hdZ>Q!)9WbdLq$ zK^JujZ95P=F!hNJD&kC5nVgOf7-r?;A%FD`lP}7&lRCsbiPQg$IM{IZunc2e`eUoG zV$5Blw$`C737$9@!3FwMaOV?msWcaN^5dKGTQ|%yt)U%PoBB z7Omqc5MNjOxy>VjF^ES2<6Hpsl#+tm>x@X9r?%Biw;|&#IsCo3 z9TxnFf6)KD(j+)hjUey_U3mJQZwMyYQV)SU8wbapn_{lITFfB^P4zKFM0U?O-bq+I z&X>X$M*X7A|7VB3FpGsLIhFg%DF)f}8td6Cn6#lNJl-Url#D3wjmE3enw9)^*Ebc- znDJ_tY=qG~5iFnsq?hfy-mhHc_gb=+-pZ=9wEw%npUDM*A~HK2@$_Cs-D^4pbg?Lh z3cO+3SsiY(3eJtq%U!?ih7v8=V_ddg(DI+XYq20&qFw)4dn4#C2$f#N%=zkp>@NtD zWBXNEr!AYZq<;_LL^p4$*D|l%v)tuGwkh}E3gw=H+NIr&5QrX5VOh-Nj3Whn*}$$R zynldo>a&cBQ|9wmlhv~~VcGNaIcxj!(xJ&fLKY(QlDy6TMVR^;%Md6DjKX=W#T`X8 z!0&v}Nl?1|Fl=(6hsEEX1;(3hXPS^P?Y42JyKP(PJK5J}0xL#10oeGH-YMU}u9Tz* zp1oZ^tDAf|y)jul>rpth0S%JVv67bY?jXKJvrN8YBX}umi32~1JBt9aQ(mLAbus!S z^SIGlG*4uvev69EJt@hwjIaI0vHgw59u{9A!mQQ>=|nxl(5(w61ar9Llf(7ae;uJj zi_~E>2~{I>Pi6x90qo+t)jTcO1Tt#gNT15BVsejThKzbUvv zuq_owuX<9NyYh(D^Ui%xN$MAY91sq?#v9St`7Xa%BkElN<^r!gDVtBN#}eaIzRSDd z+rh_joOS=)ILa_B;C74I(01C^3(#^#es2|o2pGMgj*u)wdE^)qI9_~XI{l>2!JbM2dRzJ-_A|Dp8{Zy&V;(f#5CK-dAj!YiR*lV>0gdOt=wvDrMJP^LHXmp{Qj!^z2i;o)YR}0 z33qSi)fXJBYG;KH=eHi@r#&tBquPf?Vi-*|5*AZypULEqsW!jF{Y{Y(I;iA%$s0Vj z%AlkdfIecv={ZdMJYWQGc=-9aFz*d`m2I&JYH#wzZ>FbufBZQM6#V5Ztus14lyzk` zXQq_g?PYwID8E7UoLP&=1O(zDQf2uBl*LP-gpLzJYtj`wIhGMR$;Uxa62Wjs1T&<)3#81ZJNAItup_kPGS73Q>`z`2d#2dK`blU)yahJml`s&=X|MFMuE%t;T%tl#I zw+AGjFZ)t;c4fvX{Qb11d}wa3Qo?i?=3ue$KQTB(3q;x1)D+y^u;jmAs^Q`Qb!hd#%cyqON4 z)oZ%rjz8CU9QPto{yp8tJ7<4{e;dvWERV_c!zbK zFQDKBP8CD7A&+oQYf03ljcS;Po?BG&@O6UsfcZZ8$lLmt9 z!Bgm&yc5GI*<~ScW^4NtUjwqPgEEO6c~_?u>POv6m-WZw=uaI)%@VJGarme}gW_=g zW5ZFGrZ+Y5Gi63p$39UwI?_&nI)zV2Z!i({aoyi(t+O|TTt}vrHXYMAGiiay_VHFp zf!!uuN8WfscXfpnJZ$TlftResPs7JnE|v2r@sPO+Ed6yze@-n2#2L;R@@|MeH7)!1t~#HkeLK(5WphF8XJ_o~b8$u*VribjMT79eDvvdEQw{%9}E$IBHw1gt)Q0uZ}$j&|yzk=sziJuCRNtqR={Oa=mEqfdjvC4)IacqhRWwz(6J~d|D+FVBM71}Q*;tDj$+t9YJGaCI!j;8ePxKYG6ABnEc7S_ z8A4qHP^RkXZXRwVS$&!MA)~{wz?xxh_2=DwdckAG07s*+9DHQ`)V|n1sV-IUkJg}{ z$9Px=_LGZ`iUOo*X{l)``2(x79(GCirKsf6$#MV((>DvTu4uJcmgr~3eGuSaLSI0$C7pGRq0>0SYN)qi`#sgJyd zf8wC5wZAiBzlNL!UdCil-$iQ?AF-gLd_>xCIvArY30=Kjfb(ti=%SZ{zfh%$1+oG4B5yWCv_>&br1tgJ13dOYxM z4L+!{>UNPdkWaJ6%i8%uj|_c3$*29P%?=MeNBm5RzU^CMRHg{^{riQgQtx}R8FVNu zAN6q&L2U6_SLwk-a0{#*Zgnt^Damk#vM=pP7Awo%>tuR+@cXqC##g0gJa$M**27TI z8T|7#0}9+!H*%E@u2n8L7}dn-Q8((usYDLLkY(0T@$-Q|aw&<^M5B1xzo1l6%wG_p z|3LIFNQWO0Et?-@+Cljj6#fn`Kg(%4Et{*}&!Q}V$^-9wt0Dml+_hXRr(&Y#hiMQC zXWHo?CPwKrrj+g%xe&M~Y%i7bOBeg0FqW+l2{w8yjzcf=byapdokN{kdy{`2Q0QQ* zaNm`I-U&)?{i}p{wFG`AYbf#;EE`_xdBdHos*oS($K#qV-{2t_kN$QrS~9{Xi9!6G z3bpCN;G>C?TO$=vujR+cb2lc~y+!b!BPX-H(I?y7D zR;}ipQ$)aemLzeK-uyZ|CX1mKUS^8=^0ajI@W(idHM-aprfE3C#sSYm1I-^#HMi)G zRVUdTbkfd0%Xr0W#|E3R574UdomLe6!nRjeInZT0aHk>T<@wTT3ro6DX*IAdfKfi9kxd>PLj^bqQsIvF+5H~wum2`o5_1zGYtGOMl#g8AkcYTuXUbQ(N(>vHg8BWOy0Ml}xdt6i*yUx6c#UJ$ z(GY3F*YA%9<6&w@V)uKhyqv)^`4m9J+=iAbUK_uz5wszhsH~Ja5U5k$c8iX}<;*R+ z_1U$8=66Ir%{v0VUx)ba@A@VM5v~yUemxgZr7+cm%&vsY8vO+oP1a5u_0^#gOSVjx zy!RR@pEuXpU!@?1ep>f6XtmpPU0>*m57nnL%DK=) zT@K_^p;)okP7DAYE7VhbCNoV7Cf;f5r;#>mdKq~W;G^gWiaAon3a!x@EKAV2>0u6Aiw4=ai`NO}$AsQ#U* zfRvG@WFctvU5(pkln$0{x_@ZFNToeYQ%>b!m0S})K&hiD!5!?H|hg48qF z>Qo(p>ivn9F`Yk=%=HBzqK%Lnc$bY%(m}bH5?s%@@ejO8aYwPQ_-k!piTx=VFUS*L zHEI$~6D^*hSJju%m6;s*u}W%KVKc1Q!v2O{8%8LG5&L5SDn7rdqfl4L_e0VUf0I84_rKHXq zQSi1$L9^v&P5OO-l@+S8m&u&X!Z5u(9!kd*9y3J42VI1fV_9vyN_^p60Wiln0-(*(KJfgvu{zYw>=o}D%{EAzV8&z041xt zmvr8bd0>DZVg9?UIRC-P4#d4VhpySOUa%keJ_0_Mqw(78>7ns(7d-`ur}+X|sFL7% zT$|znZQr7vZ5jC2cU}cE3VTwZaviGztbIMsJ0cVA_+j+4I*sF+OLXVC$SL5!w!bX%H)?sbrZ#AsFET6czYFHt5ibvVYlQrsW;wZigM?MI%1RS}?KOJb zi+#+m3jU6P{2rqyY6(}2wtkuToVvRF&Tuc^+a;a>6yQ0iAMb)h+em!D4#wecGZSip zetu}a)cE$whVuo9(C*3uQ}%B!Kpn1f_iPB2p9<+opR<8pf&OO=-G5X3`>{bxJ}I0T zYpkny;NvdVWUa{SIPLcGSORUROG>MLt>9y*$=c6aiU+5yqDaYa(Dgf&IF~O&K9Jd! zL@W2}i9#QWZSEKav7aFRseB^HU6lD7uud^P36KGf%pA2wnB(5Ty{=)e>bOQPI(|<+ zdU8-G)NG&=gpvgo1~y*q`8|csNS}%#a|w5uz|Q)nK3C8+^`=DC=Unu6ud8(pwocff zzVu4_WZs;G4|)CZYX8yPblQZR_5Nlh9ct1ZTh!t)x*dGJawQqmIEP81Ba9JS-n{loD#$DW zgipxUj7?1YbxJR^IRx|L9UB?FP2Xg5>eYKNXdFz$sFu6{W}ZMFXHQ{a@6 zj?Yt)@VE!_Ctu#ra~?URq!zpaG0(e(5GzrItnwNgpLXczAcE+6%fzDVl!={nYomR3u|nc;?yBvTv~Kx5ay3KfiC$G|zF>g9R_6 zX4Af(NXGC|!1=N226YXynNZz?I|gzC$HNA6Wt`Q$bVkDt3-+s?jc;q zGIiX`H=f4pqRHaEg0F1051wf-tW`U?hLUdx3$uelwUy^J?pSFhMzXK1TU5GVK#G6V zo&}kXS8_j1B2#+Q|0LDhbavG3%1oys(%*@ecwgTndBeLC^sys^YqN34_kfoPhEWsesKFL;;+j8===h zAYX3pUoxgGQA}HGl{UQ$7Q+h8_~y>Q*9IEj$4PtV#2y;JG*jc7W@ke>5wD73MY|p* z1Fp`0Y{SHVzyDinBil#?1|Rt8*SAM*p;ypbY@4y8>dOj>p(EEmzP(9CPDfg2^ShHu zN!Zgn7}iL~9h4`!_|?{^xSzkoa7l;Ds$Qcqr_Z9e8eIKM0F7DdHhoxl*+~fYt|>930|BpW`Hbi7UgW|@CWE) zvzOT|eYs>xzpllO`Tc_GFB{Nf&!gPn($(s=<@38UMnl(zpIJ1qWRb?zZyH&IB~gOh zPN@~m=xm~}l92BxjoR+O^($TQ5An%)dl#RbRA|BaQikTEE=?j5+O{}TvrrHG%~IU# z<7s6+m()FFj_^a4>FTxQUYVH>Q8{(3_4JosW?RI*QmUiIZx{q>gXl0PXAT{>I$*xe z!ggrid9WBg>61^YQhC#Rr1vIwD{1l*^U_DdAb=Rh}~0 zIOeEFh1v~nREg8=U6IW1zi6MSp-FX>R5XA1j^*rgw@SJZKm6E_DhMrqz zjsDkxyevYql4!q5ol3vnb(-rzWEPmC_|E3j#SrS@R@yR2+T3buPv)APmkIV_el)u_ zcTF}U&HQqJ)1#<@b0p3{f%$`e{YK+HKKA#%do1i!A0AM~*nUYgFHGUw`eS}uv%41< zY^`B~e{=N#bz^M+W6<3u$)*8CCo#L^oV9re$kvn|B;;it39u}@{I1m5sUskPq-DZa zrKBC(5`L&OLV6ML1iJ4sdhAUhOt=qv(-IPbSh>#fodw?_ne7Bqng^W?+5KNNM;a%S z7f%{tUZg!f5{SBw>joK%-s_bjMXia3|M}eiUs>jfXM(BWjJP=LcJ?MRU%K0q`Yjz-U21jt5`#@U zX74N-mk&HCmaE~x+^d6ygJ`N~cL~J7#{8$0>vBCv(4l$#eJ0|xm6pFCKxV-TfXv$l z$OePubPW!&Ve7~|s5S|&^WB6Z7gYh7YURuGe#e%Urn!mZ=Ce`zw zg3)WSXt+1fj8~x*a|2jI%R?@-#a_+6Xi;cfh2WRhPA29D#VN9L(r zC~t>{e?d`}n}x`<;gBW^YYVyk>+2FlG4EfTf)eLPK|YQD@>f`J)OHp-d)V?I0=SD^ zvu-=N3do<-a$C&Ep4xrtqq7Hw>zcA;Ak?c|IcjscOPAYRq{So!`{uDAlUB@n@dPz@|nld>TIu#{S3v{~Dciu@j}KdZMju}|2@?yI0;1cy z5e;F7+|-Bkij~()Wyh??DIXNjEc};k=bw}VaSIC>ghVP+&A7j&@is|2XeLj!5pOCT zU6>!_8*Um|?7!7R{?mhR&%m>GQL5>vN#}@N0PDRI@|mV(pN~^OTsANuKSwobzIm=s6y;N z-$j)291=NbGbKaT(3YuZB)I<`#ADfcuY? z>PKbry+n!+2B@TYUl7R-F|6fSUk)u&g4a=crTIf&Li}N#}%AeNs?~h!CIdARg zobX?u|IMrWazyd(c>p7s+x6kGzy@zdgH_a;MyX*o`Z-wQsUWT#4nOHd_}o*qEuMS5 zaTjJk$T@VbjTaI!aqVgJi!(U%yVv*qxyx{Rb1L*mnHJEPNRr{U*EF*+X+5^3*r0s{ z-w!OQH&Z0U4q-NuGQ*Tf;o7x){V`av{9jPn`!fWR#50DvP}LX2GAKZsEpu^MY2`^NG53Z>^CYWXwCRDotiT`B zlvfEEwWs=)JJ?fsO&_0{=iZk0Cgt=QjP+!{ou+=QJ#r|`%po{hF6^4+Gf6s3pF*vr z2fq6kMEjK$3YE+h8EtBCXuQ|^&^z#CpXxmEo=weq(mM{}B=`fYnc?v=Uc;|;uQ}69 zT;}c9dsZ=p=Ci*2kO%7bh=pg_1{dG6FDJfn%xv`}2+UTnUD?nwTD6|QhL0Sa%SXym zV2fj0Y|6yN6e=)5nLn)|QcWv}6~gh8?QPSV8dqM%U`_z##!|}JbxcR+JoBlmSQCa2 zcWm$a4b5f;_+oaSQ7#^ptN5qb7c;Vf*8Kfgs3#4JMJ7Q{L+o65UL)P~PXyZMtLNuq z^G-XJrxkalIJpvmEdE!IfBL6kn{hhlz2WrC*y+i{Iv2$94$QL;!&#a25ZW%W zU5mi!51+l)-%w~`TcEmGKn{PL*~ z5S42O!!*iI#K(i(b#|dhEI&wm!9Zx4?sKdl|DMgZhJU~yJIdy|yvoa>S@F#v>#X)5 z<>}PS3(G-72ByB(ZZqaAp*&D?L9|^Qq2aUrPo9aUtNtgLQP#bxkzAYny&ClViP@`#&tCL`zcfC0{dS95?g%X-4rW?t#^xI5w{@cU0+eKiYK~-3C!!`_#n+5+Oe6?BijZ zK#2f+KmyONoW>4j%#ZXu_KvvCme%HQ-S(|@2Zs9}n@`@GJn0_zt|RQhe5g`?SYaQy zenda#V(HMHk(vK#KQTydH?Zc`uSOAzeZ4Lk4!Txst%;nruuw-q2}p#p81$M6^dAA1 z2FMPVMbH>EBeV+x6cWb{aX)`Re0CJ2$H}bh7pbW>c>Q8k_S^xu&Dm&4l3y2zz#urz z5%DF*)ar68x6cy6kyEv?*7pzUZ&jSO`lnFI)pd2;&Wft9evEOo~N4 zI16*@OSkYym)H;Mg{fo&g@O#3H9`BgorzC0w|f-yfJt!Av35dVfQK|+8nDV^ zQX_^k^k7N?o5`17dhH>%gg=IYx_*c^S!suPXe+GvrIVrvz3Q4 zCsdz`I|)268)g_`S0IK|H2%k$70UAO&AmeqfN6AI2kNq{{VKG0ixv)t!$QOglqJti zV$k?=dxBENTgu0!a_?hruVGXlAet+Zn2ecS=#*n6^7kC?$W59OS;6?ZwrXt z&iyO1p|)HPSdC!&rK((<7ni^nUAOA%koNdmFP)l=Ik>L)`P-?rzMPLUfD-BH=XS>= z^DHeej=FB4;kKx>=Cv+szW{&2!?I4jXzIJY{41HEDl#a&3c~n8QlAIeNUfEO3UxU~ zm(wRvH@*3Ube$ZjthW3m7{hyadyM!2+*B?}IZNU#W=L6O{Eae~j#Eug3LW^XP##M*S`-{ zkwuAm*EHq#rUi*FTF7N@jPLjQLk?)l?)08X3umAF1yv15gzfVfSdA$UV1@cmomE;S3nAL-(tr}x4CIr zk}t--1+eeD)REq{(3Q8KsY-=^)?(xHjJ*Jbs*$|p>G<9FZ)ZHw@%dm|_qsteFI9&S z)Tj>dse1UZ`P_aa6X!a6)*~92MM+_zMZ(HY)mC@wi$QD%VA<-y;8yJKl89PH+yRq#%vAR)B)-RVOv+BwP-wiCKNjyZt=Mjcjph!fC0}t>> zU~IBI_?<6~dPhc$FDS}0JsMqESZ{AyD{Ai#Kt70h4-r2+xgcRSvDG9)wu^&;;XAE> z?xJxuTn<9Fb}3XGu(lk$OQII!54>41G2LAw^?Cr#G--_11JZ zg31pn6DmTSMmi68ov-HeM^AiV+m7DH2Z-qaJDnnb2hLR0=TV=O7;H4wSHo?XFlX)M z{t-t_dNx7yPg-4F%_f)o?S!n8epa2Nc1mnZbiE=6W{uAF9&!IV&$-`~6y@SAi5Cdw zoyp-g{gJ8>XmP8#O)u>pr<#}7TNSBrEzl(H*<%dy9nd%-!d^Ri3e>6L>b@|^!7=^Q7v3_T3KM}Ny5enCeq@A0pz-WK=oM8 zw@DiLRlV(|mND-$81A9xQS22nK=Tq+(oqS&HK8nA-UW_bu>{{-Yq6x1YYa^*R_)WQ+E5`w5j(0;5(`SV-7^QMG9UQwKw`x1Aq!U$^$@H zp+&fXV0X!5g{C;sEVdaaFRwgDkx%LI*=*Wt^1ZyKCE!yl(FTdK5eHh%Q@U8CvsE`_ zqc1p`7$&timO%ZB&TgU&^ycM>PJ(+7jeQVLT|qPOs&{jcv_55hwxC|-%2IT0+Lkye z|1to6{^w_MjGxRaDXI8(Q#*3{j+}0b*xHxAYgI3*hdA7JyxDEL$zBeR4I6bE=x}_v12Z6k_EpPA6|} zOcu>#pBs)>2V#T1BL#Y+Yi_^hOdmnaWKZOMb`mP@en~&dSpNnJj$af82 z`beI*l{&W>%;O(C-ru2u8ISN`O;yx$dRAYbUBstdxAZin?ooH>3%0dn%{Ol$TDZYA zNO%ogc%$7(?K74o%5&|*mo=SJ-&nA?n$7n9Tud=CEMrVP-h*xJ^y{DA_3daeydZ!T zqsCPk=?FAVG%1SCXYx@eaHEF!NvKU&Zt4^9IaM$pDvRR&&?UidZ$K#h2ewT*6OYUZ z^O&ELU$XaC11pdpl~9jtUoabfPB?z_aVq1epcv{J(2T(N^>fHoZNPYT(Gz7DE;}$f zar32VjyzfN(-(Eid@6Tsq*p;_$`7w{;P#kH?OC6P>Xs8e?0A{)ANH@YVzE_u{e_5l^nroXM3FWGf^Qq zEqNhEeK4fKERMkIV!2gOv8NzWDj8`>cOc41HJEz5BAP|Pgy4K!Aew3XV2SveiMKT* z8d$Nc^|$+!`BC-Lza`0>-`Lb$C011IrnZVtL_Wxr$*svKp++P>xdwvuQhiy+6>n%o z%678iEwH3DHRwg$;a9MERf+9}=R;J>M+$PIW+C3&!P*-YECd;V#fIqfpo3oM$G8NN zv?5{`t)rI&T#M%BN^1w^`BQ{Wg+a`Vcnfsu7TxQu-{bSW1AdYS?!f00_=`;eU&s<$2DD_)t()uCAq&Q{19At2yR)r^Sn4^E4gt)Cm9>ACA@(T;X?uM=Imd$IF531e>( zlh6qAW<=GFvJmF53N0$mNPw}NmR~A0JS{9tPftk6z7KOC@tl6?R53|V0ES|}$m5wJ zFwO1y&B!SWc+gwZ7}>?l$TdSXnxgv!KE(F(6R>qBFV+%C^%T+P-Qys&;$MG3wBoXD zab#HK54F2qyE^WIx-bPj>>{|ti`nVt=6sd_(f7^gKV^!%2ufEU;Ccm1u6}u})34iu z7e=1zDL@}8|F*I**YUX>yP()~r!s#0vR?fzExGX9L{7A1_W^jly}=5y*Q#72z?xv| zb3NDBF>i8*sV>oZncw!@+}&^_ScbBZv%`8g)E(+f*K0srH+YX% z^3kANqR(BD)wisl%X$A{`Z?P_#s>r&OOZ4`1sl`?h@`Ly3p zZAalc>S1)mrHzY<+GzU+*IIwu%OgNeamVv-5{bo3$l{Cl%V*jKhR>?RG8}=u>fh8G zCnXRmbZDjKxA>C*^@_RjkZj$ZLu1uZvO_u}a3te^T^}xJId?feP1XPx`%!If^@2uH zf`Qe}&c)8|RD`Z_a~_mUUv7N-%)r3F)!=P zuT&Yd`aRb#B`(N#u3ox2UkDWM0Nj5v+E)tmKR~EJi9h}(?VMrxi39K%PCk&tnf3$J zgSjWj zcYOgs3lm84L~1mh+O94jCz{IkFY&>r%x(asKF5&gEmYW2RagrgHP5m)6W2(f7W5XP0xQ&687Fi)O9LK8baA5`Iw5hpi@j z^lEvW-7e^SPVsJW)aCQ0KaznxLD$EcgZhT{r$k-P5PG(_Pf*kb#VP(H7ty>QeMH_+a(}RHWv)p{_X%%Dha-_8cq#pR@qXXAHkwr==Yr`g(~6S6|i* zacSKtQ7@DQRuw^=4rkL@}_?uk4(j$%O(Kkh*H za=RX--b(r9hHAe)NiR}iDI_!6o9S9#pACpMQgH*D1rQ@ux$IVbaUZwo8l7Lw4`mGdK3I-l{Uy= zVdd-uIr3wu{FwzUg0un{ZVbfCc?eI-Jj-Z>Tko^eFVhC?q5+{d*n<_QmLGMv6!pAg z`{kMPztn{+I@_hLmAcqZzrB(=vp${^!q>1A&Urniua}Rn@>+ojj-G4Hy)-d$v*Y_>{Y) z+;#~(5WyGjmr(|&?B9i_MjbJ#VmJLVY5cEGJjJ51%g)U%EH0)17H2tr1_UrOlVixTrqjSSwJ1xK%P}WeB_YPi~tpQ@o)~&4}Wp4k?e9 z`Y;}lpe>On!A1aZeSca-2AA?Lx z9&5emVhp{`-(?LgqL!_SW_eKAd9yk20oC-k4I6o!cX$4$8!it*&o~ShuY?sE%A3~u zj0tM3$%OH)*;leoUS^46=>M;*jdYFlhxK(}gj|<3<5~>mG*ayDsZw>YPj3b;6GDZI zsFnq6vf04VhHX8Tf_1Ygf7){88l?ZdRb53?A*5E`&#I}sqAsJX@{l>GFMJ8!oyf#9 z9`}wxuhwDpF{_k8qkr63@SD~9#z7K)Y5;Ry*Hl$okl#?3xn*Tja6_)csZyAhs5w`0 zPvkSQW@SmTl{)1ohA3adrQZG3nUFW4`ZTqa2pjM7? zyz7e2=Mai~k&>IK&Mb)h!D4f*S~k?(fpjQA_#Fxy)S7JNZK7($wWzY!xOaY9Lr~eD z{ThwRSf{GvFAF-)Vg)BzOJzB&=vd^b0Fe@Gg~wPnvOLq!#?fO50ny83v)F3bRHF6F zB-OGQpXBX7Z95r56t~Zko10tHZNs}Cg`6KuwT+k&+S)WTNd+JD*+}Tk{ztR|RY5avon{&VKqOLCTtF@A?>5{2 z!zn~=0%1WU>J!Vu7nsNs*)rrwG|O;fnJKqP6=c+2%A)Ah8{%ad?ka(JtlPHjX>1%cl{`rl*zFc)YyscP=DEuV@>QOM z=IPCPi7iVnUeqgzpqW`7`kL#)2Xs}tzWB+z40l={!UB@-AloK{alPuv_+xA3ecvR~ zOlfq$VBCoUZJSi9Od>=^_=iwc{V1RJdZtD}0TX z5&1Cx)!t`7 z2o#IrdodZ(F90S=gJ=MVV%`?S3UjScM5GgwP2a%uv9hlydB1D8i3q!W)qfI#kN|XI ztSj6LA-R0h7S7&}tjf>pVrCEG7C$PoKT)ELHSYpF+M}6Os|fCHG99FCUW#+<9CL`u z)!$6lj+LSCL=gy1btSs(Xi^1%IUaZI)(|6v}5WnOtB2(Oc z=lmTub$HHpAte-Oy(`eb@1mqJ@jXOFv4(eQ(DqH&2VC|KpW*_$eB=oD44PUb{5`P7 zMa3LHqXT%Dz6^+|86Z<1eF%3cmK%cVh|@3f&gHQ+r27~jJvuEvGIfp8kv+hY~mV)|;sO=Vdl zi@xm+gXR3vH}0OxaLl@L#uD2wvspW4pE zzTZJT37^x6b<(AAcpdmG4@V+Mlp?&4q(AByf@<&d=j2boS5c@3nC8WoHI%J6S{y!> zl|y5RMR&YnyX}ZN`f{0>sqT0@fZrVvr$*oUs+XC0SybqWz|0fq-}ULPDNW+a40!#Z zM6+YuuIT3b>f7>UTa%vUejgmU8~`@hMiim{g@Ak77PZ*wc$<^fiLKc8h%S?gst?!4 z*t{P+iLBg-l7K&l#p2?;_{3G^e8dzRSd5bc)e@{Di~ghQtHP`(-z` zvn~@v?QwlgCK=R5o@c_cLJSS?fZUX}c@et0i-{_>@;AJ+MVDV~0sVHpMq2cHq+9bm z!8lCx@$0;H#2l1ot(0iuzzhU z-q?A%)7MJCWOxteKh9$cfeA0|rhb!Bmw{r(is;j_6?4Qcn)&b6)8GbG;2X)8#_T_b z>MbZGANG#jFHx6IqBRk%oFy$Sj%#@L=q1W$X^EtF;Hfrs`xrgKuEaA9t(oR)2< zoG}^Qvwl@!uBFJ($D4-}ad2(*oykRPpd@5@l;CluEF>%<&?!Y{K%ww|#? z!jpdJ)^?{WDgf8iE=5Y_1L(ir;KFW6pvG){UG#h>~PTiD!3N&>E`OO`St0@0~mu7rkQ-AMQq6uMm@*ll4{d$9q~# zR^RFXqog5l3U7E;ic5nuo6b&?w_V(HMiA$Mg=Vx)1xkZAV_(p8Tobug9|?|lZ}b%q zw}TyluV3JdCbS*A+^)^aLRps3L{9Rt+^quURU_ zx2V-hx4RyLm>6L-v2kUrWqrhA9J(IWFFd(4*?P{Uc{C$|w;67!@~8%qQRZXD=VL;A z>Lmoj_+~v|(>XSTy9Tg^CB0E0ypTY8G^M(1?JeuYwyR_``8JNyOm$m0=O~m(PrSC% zO}7_{pJH~x=$LiKg$elqdJ!bY?@)AY5^wyTlnn2wLzG-Hk{3*qnD@PCtqw7$3>LU; z>w+goofz=1=q>az8>=$WV5PD#C|L~6C#%#@^XP6)qH#>tyPp#20qb5BD>)RouGXL| zIAty1_@p6R+G{UHfYsZy!}w-K4t^p5E`anpUE*Ot&okUcXSZn4v!*-?U2lE?xorTc zEvHWTwvW|Ks!gnP-F!DX=4H8&T@faBCoX2Y$GtJ|;$%A^F688GL2W}(M7CkQQ0+(R z_H@pwJMeS;t}934FWrHXX_D&i@Fq970j$B&z8{e4_zG>LGz$M_Qsr5uD@CwjdM6`z zN9v}Q zQJ!INkEVRh4f3X5@kkcauAwLGgsRrD(jEzf+Zm+&$8m zjPJqB?YyPWH5>vz^7urv?RD2y>+y;p1l`Sn$Z&#gF<`$v>!MM;hIxT#C*)jO1 zMuNV(KBn7$pvb&)&vs&%snG7RaivjdUYN?uH$QN=Oaivclv{ScYy} z;%60>QcTW0s)s8%Y+?1h<7Yw?e&yF$e9awZ%vY)^&uO21No9j{?vZ-6!3o3R^#qR! zl#2Z%2#Y_DzG^^r#EQP<{ZOLInQBs@HvAgf zW{cc^Hom6JzBqjvA3WNZ?8->%(96E;L`qU&;|j31R@Na@Fwly7j{R@w?0Bvhl{g*O zMps5a!uxO6#_M84y54HDznS~ol|hi>JE01)*ynqkQOeWdot+HdaZa_gs!&0Fg>K_2 z^}Oo=ig#7vsgaTe31I-K8zJ}7D0Hu$C$pbCSnyl=lw_}EW4QOHU^i}7OoRxJ|J&@4 z3K(9Bzvpf13w$#M?*f5*SE#j7snVP@w8hn!mreJBo7BMrdAQMIL^+Q?193e#?SdD1 zoK|Ehz3>8gbet{FMqO(`ArxCO&#wdN*6j5+YIT*N8qjC2F0QPHARXbdg5mQt<4nk- z$`IVC-CemjGiT^EZ^3RTZTESADjvt3nP8LW*j5`&IE{3h$=E|z^jmrYl?|C2L_Ss2 zi9`mKSA}ecq&d!Sl*mB#??Aw?*GIxrh{Ty(5|-9|FEp3+dtVFr!uDZnL&YCuJtnG9 zt+!5xy=#ow4Mwq&dI)ax_X~B5`K_lipSd@c>bJ_a6_;vW;&!Y0g6ADR8#J4>_e_l7@+=I4wHk5J^R%iJ6zhE z{Xt*nZ@zEiL2U5ji;LMw%76z$9eBRrgKdt$okin8(|zkRLHR*N(Nor4=zo6 zh-TIaDVmbgu{aA9Na_Hs9H0`NJBv70a_S|D%qPyYTat85T(5-b7_=J(_UC44s%pn4 zMZJRTi6fG2JSxsy^Eb}qnzB{Wwlyzk0MyzqNQbeuT*0stJ-bbRUxL=dG(3u6(a9KZnXg%HG2a(CrN-g7U$&~0w+7~^q-`5Mxq=L*m-z)c2k2#KZ4=iQg= zVHVEE4^DiHyL1KXmEFy=Q;^bN`MS=jrTi~&iGKG7z7^1>eE$c;a3-!|ttssmY7{8maLm94qV)W5{&Ql`{8Fp~)nYsPqYsz|nzd>3h2 zL)cb~b-`JIn)F{9<>4iR9S4#o*^j_`QJ| zOkmMSHOl3XyP{}Z9ymZ+<;whOW0%q(Gx{TMB=2=BJ8TEuO=^v^MC!2#)$Sa<4wQ`D zaBlpgRmNJD57wG*@~bi5dnJ3n6Tg8rD@O0Nj4ttnIWACH*4EFn)sH#MJ)K@^)oo}f z5RxWWAaaR@`@C6_`M&ZT?ZIci-KOpaYqqI++8I%UG6(^ zoP~BVvIc_hD2c!gY~+Dc9dns*Sbx zcniXzf`%fu6&r=KqL-lT1ITWcM!2A=JWm`uzn%Dzq_Z#|2knZt)FMDJ&U6eQXFNey z)EClG+G7_M2zPr!ZfJ ziW!*I?_8Nt#OPDG37w&lbCpgd+YKBX^I&kfr5&c~r-__TaEUL^6ofL2SC7%xe>C!FKF=2&n>JB0J+v9Cz%WTH&mzSf-lleKxp19p>w@ zDM5}37f|TAlycg9dzQiWKIT?|kv?DQ^ZeJhO%6q8GyF>ryqt;c2UIpPhFb0MVorzv zZRtz{N*paMO2+riSdL^-_kAx!xP3+WxTDR_9)-L)Cy}D6JeUkmRx?PD!@+sLvHi%% z-+YlwRy`I-;WDf|p*xK{e5-fHVKtb5z>6t*IBJvI+r7`lvk$0*u2eKsI4?AHO+T*^ zY(RvOxJ0!z5fzI*&NokHg$fQT>6V#8nIniPlfS;s5@J``$Jjn(qfw?On^>K!xRh^C zW3FiHSn3yMZ&d5jGqk+fS z9hCA^tfQ7FMco%86ZM*i%gH+H{a&|I)urI47PSG>$_-|P$5G{tcZWH6#0n3{vr*h< zgJ0*FMpscRU$#*$t2=4IvtMfUVY^sNcc~UuT_1x8W3A<%b5Q73quTqMf<#}NEgQV) zSXV~1ZcJ372B=3wbh84P@VE;oy|^_5rFUs0yYKIu7lY8`=PzRQ8j)*BwmB+gebtd8 z6mb`oc$ta0MitQt>)seU?5D^oUuvq|1ckB(a|Z5}8iJ_Z$er8sZXwkccB%Ukf|8FH zwUy}wo7KxQWwbeZws<`B+bybUC;K{zZy@pvGzvd!Y1!%9>oz!#OGKbz@(~OqXo{C8 zTwjK^vyg$Xr6v-z!SvqoX0ppp1xDlAzKH~wA5jU`$6Eg+NQRA5oMh%J*|X1 z7cs7gdxx)l=50)g7NfDgoT6r59dPW@G~IFmlAq3>7;cg~Gt68QXgK6rhE-hmWhOCE z`E=W+uFPwsY*(BCO{h{4+0oe=4{a##AYrkcYROrf3zF8AUi|beGVVNM<<&^5Z6yJ3 zh{ce{Fso&a)wDxmX60@R7)?>wbHNrE8O=KWGAzMuBR=PI1974!l0gY>f=CI_>ti4? znh+H$uq?zgWcQPc=61a0vP-3|Eofpl9mxjrU=A z5km9hZToOZF(0rv8M4(HbA4k~w0Px`{CBHp5bdSl;R*>jwIFX_pa<;bdHo&3C=h2e=Ze}zh3MX;x@ z-ceP+6~hMgQRaC2j&@AMm-t^ zcr!2W0huHqMg!8QIGyJ$VAo3A%$Kbk*-037xQIAL>)H9Rwpj17B<%eFG4ug0p*5k0 zZ~%qHw3hz^WC*~EH_E{A^jOu6qx=$H)Ey1(pDneBz&gzquQSuCCe-ymC6h)v^RUG~ zsxmcJyDdsGN_G~)sn$=^%6o%3oD9fV_==Rm%@Jrfm%wYJanU+C2RPkHMjYJgjRwj} zs;Z*s6P9Hq3PErc+;%RT3pHe8#nL52yZX!Ig{EKx=iAV9t5-2odgd7l<<-6$>i5-3 zwP)@uyNLKWdR`Zp2_shgd?AUFXsesyA>GnD+KnFcTS-(h#j;l>{>J6s91z2=i`B^~ zen^b(Zkj^S$RL{!_+~C1W)|ggZrh!_=^?Ti&2s87o~o{0St^T&>esD8N_ZEF%J2jg zI^c`b*}+d^HzCAx;&KF=zuJC)`Oj=yLy9Y`$ZmlxH8%{4GmL4?!{$o949l?HHhDd+;&In z8))gil(PQftH_)6;8_OIAl34B+^gG7xbDdVqvtrSU88#;N9CR-LBiUW#;1|0Gu4jgw!9h6x4&!3pt;sbNt>x?&h&8iqNrktHXPSMH`?)p_p5&X%p zI1OAMU1o%6(A$170+-6$6n4)a(Cg{wJ(s;>fyF{MNX$p-#3Gjp0^tfdG-xBNqd%kf zl}78K1PPYYvYGu!10}^XTV=Wiz2Yrvt1nK}UUt~{cxgsOgKWW0GiXn#vhw;b<)hHF z$}yIB$frnBo%NW-HSGw=*76ULt6p3S0#O+*m|tsgVja^q4CzM8cZJWH5YzJ`lxd%6 zR}`sPwYEb6$*Djn+yIe#L5CQ;VIh8}2_T!z+!B_IU(lT>Tz^3}#WJI_eZ5F?FPdPK zX-{qu8~55oG>RLLyGIr%7i^Q?$aFcNbeDG9SZAPCv$bK_E$q3FhSJ^iq>hHp@hPB zBcE-WVU3uZ6)f4@VsVW$U3%D2&YNc?a!Q@-5$1BT}fhzRGR5Xy3rsTetk;u zW?sfCa-7D!G9<1SKRVsJ6)-R5HiP&hoLV-Y3)@RS_^Ogg5)!}5X^jSfB(t}#u#qZ( zd%Lg;HNb(QT?2S|!(3rpZ>n8t2D?T$$kvlZ4kbxk1P?XG;CFzK!Dc`OM?Q^(bub<^ ziAY=Foe)24A(HaYbg~gu5azXiaR;M#rj@P~?uSY25ZMFF0@@u{_p8U((XP23y?di% z$Q7aKjKib_2|p-TBqL zbXVdP}WYz^~T*qX$hQ&&-YNrptds(=gU zoY_rrlF_65LgGOgvtGoYg*awI6ex+j{ zBp`09TQpGHn7QcBy+S^jq5E=<1;r9@v$zY{#QSN|yVKp>A|Eb_#AzZ=!Kqm)#`7ZBLSiM$q8WSxewGB3UYs>u5G^(Zp zv~|Hy)Q{k~z2)%`P=ueE!!;~HSxsx`-A|?-+(Er`Rh>$}@ZJ@LpEkMOw-Kei3}ERw#7B5h@5K9th?#vLfCUd?AZIOLE`^}1-NB@SS#7TRPw2vc2=*tXtm9cKqsPbo4-6p8mXrdJgfYf<1l& zxstYz9Q6}CnVk7mW%tpS6~#ad*QmH3QOAX;QZ=#1w+5Dn5F3c^tUEM2C$1+ZLvsI1 z1cwCnKyU~Qz1m6X=b)~1TsS&j*z3L+DarX4hV03o;qff*htIo;E%?5gv!*iz2Rh%v zCN)-zkZoy}r2_u3`JyHV{PU8NR4qbImg0ZlBv{v@RpP+W5a$QTP4x$8)V&`b7xD8T zu0(~lSJV8M#P3d4&8CWC?tV+xfkywuS*92<#G<$aEZE>9wDcNe6xx?xkZ8 zfQcrX)4s)UhL+ea0H0A7ZDsu2V*E3H@H!wqVNcC~dc`Hun7lqV`N&!Bcwt8Z7e z${=RHtbG1LrY7+>L6CnyS^j5{nP+v!i~!k9I#T{$$Zj}RssG`S@B&r?zWGmif;_^% zu35U!|BB5lj*eL$PXp}Q-%>*IeSU@8{gF+b^e>5w{|v{l0ZY2VEOBWEEV-m3i^l)> zbqF{5;t8s;crnDjdEm>}GlGg-Pg8K$)|z=njWm}5|F_fF{917?HPmFV#Q~0iWg~-j6;5pwbi0o`|Fsg z9_M7BXWRjo?TOC?BM{L}^=lIx6e&kCDPs`i+)5397qQpZ`B+sOv=X>n{sZJiy(6fn zF+8;~EjHH_;L!gDL36*x<*NG?rPW~;-bnvpB0Q&|Y`)<|2R$B2Fbzf1;vt0Go+=mC zaMp50&WZZ>L!91lA}Pnqn06lFzjL9YgM6;Q3r#Y7suU;iLYs?3^+YQkt}jxzS*-2D zK2^VYIObI58z@o)bKNd(tu5W&D0P5^&EroxCMffEbgGlbCywe{oPCUG=08987c%8= z4La!4^p3e%AO%`0zhFZZx6t#AI}{R12kt4$s0ME7?oJEn@&m3BwySTMhg7`qK%AWv zYe;(|J;2E|>0JS3zZHak^SD#GHgXMLDtTv%K<7GgNkqTZ%seQTd!(n@|{m*Jq1^PBXfsvExbav(>LL>L{xg zSN7YWmBd8UWS_%=7x=a)m3yY8_^rm#b$YU~qsI;jkWDkzY3sb8wPwBolL>`cK4$#p zDw8my!I)PEx=U;?iuv>MyZ54Mqtp6}HJ_hx7`{E8dC<1@rEM(tDdAL|tfS?CP%L*6 z?QT(JXws$J`tsn1J3|g3hSQxd8X70L?^jfgysDVXbv@3$yhBfJa`W}pN+pPK<-Oh& zJ-T@bM!aB7Z4yioKIh^E>yZ^@x?dIboV~2wCbNp=-IuQzUfYRAqR(pp(H%tp<=}^U zyZD%BR|G(SrtD$ttr^25t^_%{K9v5sJhd&ajI~X`lO8iUMdIs+M_w5jLRC^6JJi9F&Y`R;T-^B6J$r6Mv zZig+sS-e3YygtsU>k%Wj{h`DCqDD+VMAD<4eXtu4h!%KKUL{K@ zAr5_R`KWKp@FLwLE3r`(rhjA&xtbF>hXosw)gmX}BXXAN$UT_GW92GeP`i4s7O_;} zEqh#p1t)qxf7szdo9!#kv%upE6EFb8rY4G?*W_AMPE2cNCveDH@a0)kX{hGZmzBPM z$<@^C8R*p7dPN#~q5ydGCj-(a8NU_oQwZ@7ZHm!>a3wanD?T|N&tEBF2DB_o4D|U5L4&-xmExWjBjAcwakIULW zDvE*HW#>&gRq58Z3Pxw|Oj`=RHDBGEJF0!YYyN#oEJd-{W0>4&2o1o%TMg;Q9Nl2I z=ld;qIWepmBNyIwpG%>c6()=M<8}qutoVD|NB13F3Y9Ek zVy&c4U#QgGiT=cKJA;jpq^tPxum=trxT!ck@-imMK2xa>?bfDE-DbWotgD2z&E)Fl z&F|j~_VUG~oSieDyu!VZZK^=YzvI7@Q39-|)0gSO{ISSBZj7ggUQz zn&>wSK22F(_yDaa^i)hx8OIW*1orhXpB#vYClim*+Q-M;QG1`?=~Ss=U>yP48h$3r z?`>&?<+9Uzts|b1s8t?_e#^qjh$t;f#|)L*?d@3Ph#9`3+nzR!LlKN8x46P}bLI^7 zGF!yjn_>m(ZLjYc-Qm34Zq^M@llk^Ld_{r!_|HJP=p&QrmruMRS4rTyA{wk!4#iY zeB&`#N!MTo{G?9I$zA+Cz-_;zQv7p)B(9DQKxMZJGqfwE_)UA{r=rz4^jWzJO;Oi+ ztF&U?G=KKih+*-UlhnFzAH(Rw9C!>PL|?+{WsMe+AX>90sxr zDC+X~eJc8J$!9)vo&kkZX!1F;GB?}cvTl=m508Ll-x zK;uY&EXBzihV}#)a5rLWP1>~WUY=6$INIFBpkta1BsFt=xkuv^-tMRN8p+UJ30TnY zbk`2mN>dC}$Y;>$8qx5sU8T#iUA|6hK|Y%%c56&qs=M}IabM?Nm_{wl;=e}S>)BVW z8I5KcJ5pwT7B6;h?MW8!|bv;yC6O>-hAMpS z6Y^>)&5rLOdww$!Q<6@|M_f34yB44ttd}m%v0Ok#&u`RfMt^u}22nCkjmQe;WA{fK z-iXv<#PI6=Dqe@;>#&83A<1i&*_cVhK3WWGB%oZ|aM7q8LKJot;S z)?`{pW(9J!@pDyewPL34md@^Q{Pnbl#Z-y;d@$)cDRmW6j6oNPWt%FrK|Z<@(Pp!$ z*W|-v%|B;&>#lRKK_{zD%p7G|RbuHARbo4E(PIjx%g>@z1YU&@55`ePPeSFi%Y^95w;`7y?Hu7>Ezs=YkR9g@IujaOr%-(yRfDs-&VV>O^a{?Vz= zvuE}(S}d=Ejo%cZtCpL*>nS7xUQIAH7%l1YSMTFJxMAn$#DNhYV+}W2ryM+yq2~wz zD2{)CLd3<81-l3X8QHQ)D*-V+jS(*BlR+cAT8Gc-_mhTAJRhw;k|cS@*DdNZ2_*dn zUa{1poazt>{nTPIZPR|fFk%{AZahJhc^O3v?;yb%=cjMqPCq4npiCvG^qemHTB(#9 zA}@);Pn^c{7tqNZRd5%ik~A;kwprDBEY$<$T)Fo29iL%>D+S5DuUi7pprH7MtZ;-tRT@Hwrhb7TCxR}~w zWI_-39b6va&fKb(0B9~IPuyzYJ|^>q*g7A|^3NE&3z!tqr7HbKxgAXRVo{<^kyBK< z_Fe__wPWr^BkkyC^X%#mb(?r=YixG|$Q2Mt`mbCDE=a0!#3ad2a}5K2j@CQ4!C zE^55!W6T3xB}Y-gGK&r^Cf)l|%=u+(+`P~AS1Cd7+=TprtuPoLZRx_vEh<)QxFy$S zsU4#g!({P&Te|&8>?_3zz6yU$Q!Bh>%KiZ0S_$l?fO3_a=7Pd0{yd^m*?LUmKHj%!*Qry{^yhy{nHUuajNN`HWi%>~2f<_6vGg>bFN>x4QwhHzUkTv7|EYtBm! z==#cuxEo1g1M27hOdr~{x~s;S%PEO?ErEXoFb@BlhN%c(RmI6hMJ<*o`RPm%5PfEd zry~a-Hc)3(uvXiG+p=0;SbsK|Z&|Covkb99%qzQ7Lmf-zPzS9vRV41D3+ZViO^@(}n(Rwz5d|1BA=<{>O^0TZ$GqRABedCbEp#|gM(!91l zr_!jephV1WiJg7KsX!9N^N|WDvqs6}tNI#0jfz^+FAC+Tgsd3(5CeFs+JZC4Y>F|F2S4EHYrw^6plAn|Yrl8A z1b>ai)kN_rEvf9j;4UQ9(sIsH8V~AmNecW#?9l12W&Yl2}e_Omnrql;`TI@x4BYq1n3{_w1QhsXQG=m#Fh%i2TxCEnJ{r zUhk!)YFn5iS%xpHv~QUU#<6>2t#i1ALe*t1T=MLr$EYFA5ZugtS>@);oM6>`->i|J zxR=uMs+bdXx121cpdfQ_w}ysv98F^IH3AHVM{teF3kt*_X;Yvm)oa^K?tJQ5ux<}U zxA*C3qCRgOxlh8HL9;?PjU?!1)~dTgd`P*N{j&DX4jdna3nZ8zMMt*;lR0(nRJg^+ zIX0MT#@sg_m%OzmcRKx`NO+n52gn~>w&#`Ua&^)=e;cJ7lKkEzTL&OEHs6j7Ant)_ zvoakRaW2&`TbWs2u5Bhys$S+?@NwrZi>Z%3t9J`Z$6xTxig=^Q$upx&4ecvM$elg~ zZQ$(3k%kj;q%48Mu0W&rUH4v(mG>SEmDBMqXjGJz*49-;J^Rq^6j9jwQD^Vb%|+iu za>G{sEuG;!XVH$%V|!HabavqPIc=@s?+O>x$H4e>jCwV#2o% zj@s{`@e#KqNVk(IChMirHvFW>JCu57mgx!*D6DWAyPOdPI)a@KEny9nbrOLRv~WP# zN&K#2mL6CdZYJcHS#dZanmPiVW@~GIdC(`NoVsG?^u4H8@-oeXerkT+u$w$LeW5#h zBPPPq$;nAuJFwQVILiK7P_}XCV;vlAl~0Q>N({@IY}j^JvW_R1uX|U*jDh;H4qS<% z&(3gpGt)X5v3W3WS!Fsu0m*&Qr?6-f`m+3q;t=S=!_G%Zz=o%ZS{HJ59Du;rmmf++ zE|rDut2Wh{Th*U$Xt|_@O40Cjmy~|x9&f%|Ql65j-PTBTnYqwW>wL_eZ~wWsKnFrT ztn*T*IZvMV+ue`JR@&;WPz)mX?{8EWzce!?EfY@Dv?i0Hy_WZHW@Wkp?{#)FRb(2A9Tlx_bZcq4`qokyQLnyNd=}F9g_I{0=hPHvv3e5pu^GQCU%+@KnDE z9Vd%DzrZz3?4^R2=`F(ReFLPej3Qa4k62d5ZLfVp_Y|?NKjE;w?Ydv@8$P2m-3>iI z{vh{RZZ?BmP6IEbEjT&W06n~DAhX1Bh3!1y*gf4daqE_W9?fH>ks>Bmovw- z*{I1>NmgH11r9&nkWn2w6$g-n1AsOd>pv*)7Uws;Av_h9!18R!@!V`gG^+jmk+MRh z($IuoC&P=g)PDmKz6|s6Tw!1ssogr&wRYE|`z6F(t@8sk+57`!@CM)uZ+F)Gr=|C^ zJzEckGcSof@odp7(%D$B3=mDJoNb4#S((n2^_k37*$fC9n-v-Z3g8Gd5FM0h1Q2=e zIbc_h^`_U~*E1vff0j)9wk=`uv*e7d?dtx|l3Fu$<^#fuW|==rCjBfaEeDkJo&iEl zhR3(gd8T*@+@5>X*H@l`4Z;}6F4J}yzO1ZndO)tIaWA-eK6ri?+3-a30-LhskDh=3 zZ21ee1>g!6b5Ata&np}}|Na>UU?kuMSMfZMNx35P++4VN#(=DDBm&;Xej32|-ZS$9 z6siXJt%H^rE(y*UuJ8(*(5no{Y>%IpMEty@61XG;2m{=71}?cl&2V)K3Iz5U{`Iz! z023DY%ozwZ{7=+Y{(=d}vskzYmsrxi0>=Le&~^U4YX45{|GsMfPVKKZeFt!6EQDxH zfNS3K^ufQ=l>gA!fA_lnoyO+56a!f5F@Lx0|M8a93XMSBUz%vjkuU8;iw1Yh2U_p(O9|XjZHzJ@O9%M9sPDDYwYKf z8d%1}1ccJECg5}CNWK3q0t_oUvm2M z)2i6h?0?2lBS~SN2mXc6<_{EbAcN09nCh?0Yrm4>kYfCP;GfX`{BF|Uo#x-=Yy3S9 z|JN-IS<*2L!|6%b6`Em5k_hrwz9Y>9G^4i*{L_S3&*?O1kYiKxUsF6s{>_2#JLloQ zo=fq^oQL{W1gv8a6V5yVzmwrItqa8s5x~sSd+DQ6)@oa3cbOLV@pDZJ&dqOcgrG0K zdbVDy8V^jhg1<;PL+)cK87rhRsg+S_FAo2XUQu2gE^d}MWh%rn`&TRY+ zK1uor3zU8f3B-eM7zU`&#(s^Qmd%-45LMM9tkl2TXBAm?ITQ(MZ&|9QqP#D?jqq$$ zq9dMcn{4D7;ri~j;>xve+*GAx5h>9)+ygdZU$JHk7ny9Xg*8h|Gqlpvq5*YwavOBz z!3lUsIplj#_De&s`I$kzYEaJSCN8Vm4~o?AN%lNbEk`Wqy^|L3)OIHlaz$8;4qHl* zcs6pOhA7lBKe|!A9YxldYaRq)iI~Q!;uXz2;e3d#-3!IUlN3J~sdrs;ce+|u8Wxq! zwxp=Ek#`g&Ke6fM)H=v9Uv-e<%-NFCV#c~Hl?&%tR>5_dAAta3OWPCFmxcAdQ~<-v z=h~3FhB^Ht5Aa01?ZJu1O{LjwG@ic61ju4Pei2Xnt=`2HO7r-_Huaf%GyQ6X@jYzG z_e`UW+3x8IeV}6TmUG`e2BQrSFdTq!5hW484+TpnXq1d{#?w9Dmkn32g73K6n8$o+ z$-f<*_w;~2lCAm4^%~GQ@&DWO;c6BCWT6_g=h7kKx~M>&OPbnoFDxU-G?}$Uq`Z&2 zh->uCZu4||Y3ofng}rT(wn}GeKR*HeTl|OjYs!;ZCmF4sgY$wDF(TcDa6et#M=yMa zYmf3@s?$-ZAzL6$B?}g65H1tav*`5CU1pVm(_2eZLp(q z(b8kF=@%%#C+T-{W|&s2Gf^((je4kch+~&}F@|xbSm3hVOiw?~KSu&a5hP`83HL%v zDr@R|srEjh{F$Bb?;NW_@``&UC_H*oNLR-S975#i^xGz4uSk$<0e9CD+|u>9Dkj3| zOb^$_3a$Si_TD?J$u8X&4pOB^k={EbFERr@FmDhBh-9}DfZfOZh zyxYmsGyPRl;NgJ~L}bzhpOb7&S)S;}eH&lTo;I{>a9JZA)cGmuTQxF^uwB-(<5hPDk1OUr z^YB=M1XN<~l7TnSgZzU-o#Xdp-jRLX6u~gpk7k1|mYHe-hQhU?J(}Gtr9eiE-ePuV zFIzx^Y&H5N+&T|)g0fGf38z!GsgLv~Z{4NJ+^7^fX+gf7q!Z6v5a_WvRNBK!B~`lV zWj!i=C1^BxWL_n_jKS_?Z{EJDSl5=lncWz?QVLqtQRu0l@=o`cP+@gG^x7f~$2mPt z2vZc!75Q}TR%+`~GX@d%3q{J(Ikn-q2Y}hc4Jk~%-N$>Ghk0)C5kR?e614C`=qh_G*Mw_hI&?4_$hrK}({YvQ$ zMW(Y@0EX1}RDPR!Tmm^kzt|1s5_oeeJ(Li5cRlpT()_oS!Pb*RuCN&_9@P%)Aqy{k zkcm*+Wj29`fIt8CDJnNS^%KB*^4|nO{%Q1_scWvO3r@s$SYg2L@%eG$#y@}jlYA4_ zqErQ&FyA4E|C(L@yR&P-gpF|PD~YS*VP#iZcMgZk!BNt3*@WL=!_db4Q5d=hP^+qc zaLjhLPj_{WjS#8-`t4V}07`~h4!8?K2U*>U0da$bgcYC}I=4%*sSeNOk?$Vw;FqUi z*H^MX0n8b_SpG~M@N^R!iTi!u|LeD3_42Dv{J-)({HRI&!~5^FfQp5UR$eRr1Wcrn zu^3?8Zm_h-i5x>dgX`gBW!KfSS96D7;!arOqyfJ>62Y)uPX9L13z;D-ffhVj;i688 z!6eMDET@i_)HDk->IhM=RzwZdt+!oGMZU(znjzu%w(-N7DM>V!h5!809we3 z@x)3(5km`R%*Pa$UPsNf^uNZ{N(EKM05%w+sIppZuo;oiQ%7>-zVDGjKEdIIahvge zdfkYG4j#J|!!5%{3t&j~B_APcT%6j8TG_~4iq7%^(FTGw_$xS3H9q4TRtKrygJkBF zE?nU1<3p8*Vin(c&|<)-{5ozgSo_I@tG9LZF?e&|Oza`UDu_5>k}0I))q3Dw8Hp(6 zW~G%tiMNV+B(#6EPS++zL-O$bGsB>nv=*DUMVKb z5{*9*Q>^%*q8GnD)16o4U=LqKL4HoA65B@>vR@vmSj%?dC4VoZb_kW}nTHG)ci?^N&c3>U(8hEx2Mh>>bzDBz^K&@X(Yrq)dMvoBS&kpX^trJt zKQLwLH{BjmvGrW0Ss2k;0mgXQy#*FDg3)_`vaKk`g8j^nYK|{YhiCFUNy-ihtcChF%p%S^oW#ZgVGHl(;ib5A1 zEq~m9M!aVcZWo=L&$oDzTHrHS18=ufT6;MuRj;eDS~mNQcQoAuFXk)WLW8^rMX_iZ zFLdAN5{SlKa)h_m8#l9&&yPK7Fwr`?6+bv*}t#XPg*wD$|Bc#ZYxIh@_i(()-_(`@avLWt7n8DM4J+m4}Jn5(0AZn zUUHh5S8qVO%s@LG)`9jP&w@cwaA`W5Gw8uoKzwt-%JsMsUeIz&gp*StI3jJ!xH8tl zVtQ&uQr+2F`nx)*(a3jx^=Msk$Ig4y>e9OH2cvslb9tAz=vzsyY3g=`8)oJ353l*k z;g3rj=WHO-CBbnbe9>Hn0>N)O+{tTWeQje;-l0D>u%qUKvD|s9O2Sap?$R-#0@EG; zY3H{uqK6sq1P_%A(wW?;@Lkw$UW;Ownj090yp)msyxvcw7AWSnM)`vnG7hpwCnk7R zKUb6$YvK7>x}lUDj!rI=;fdstUQSqud+RxPVLSB&+ajM+4$85eBp``DU_2uL!&06$AdhENy!o~Q>Nk>1WF`$k@dF-?%gb~3Kh&{{()5^ z!TVc&Y&6I2I}d&W_|zBSr{&~Vva`lL(tmx($hG0-JH{c*@)pdH^*@T_0tJN!^Zsw- za1&#*x?O6p^HsX*KpxBQfBfcQ+~zq*uY(*U!Wgt5mWG~Bz{<#GHz8zgwZdPG^@MZ_ z{3=xFFL)BGOniJhDkQp9T z)s5RAA$goDxa?Aku1Noi(OL&iUeDxV;4S9^Sto5JWszVRuqQ30S>;Dl1EKa3&9^?U zX>Kf9K^KtmaZ8OvQ1_O1S;t3tONJ9>mm<;E(Ol02ETnfTx`Y=kZZdML(bx(tt`rf@ z9vQt1^Ki=9m|zT?)y6wDLThus+UCrtA+Ws_80X6e$Ynpq3myG$aQ+3ml*CH1vfqF1 zdaS6OxspP7z6Cfw7i@3f0@qy5^P?65E|rlqs5Ztj#=tQ|Xgl7HS(i%59DWnD*-22N zt-wvqExMakn5|7Obe?~BeZr-YqBHX+z!s{+;uNnSXB&Bfep!dj>DQ6=Lhd#(y^3nq z+O<_E!8wVOsu9HJzA;DOA$L?5cIkn(>4@q?!YMaX1S?VNC%7OnvrVdIi0BO6(X>Lu!^Q!fc&$)kGm)kni>*swHB(HV&sj{3pv?l&e{-Sz0TXcq}9>V)@Y8G3FaeZRPcwR(ZnWG!nzmo>qYeEOIA>VFf}NsNiPu zkaPFom6V==L11;LslF*IbZ6e&wKlf@r34*CS-3^T&T7b@^rjWw@TKe$F#LQn=$dWj zLH34Y8tBy6EnXuR#4W@Kt@P!~>X_|UmIjr{BX4vBEV_HBe<`v$JM4pvZV zVJ~$}@O81&!tw6)=JoT6jsKwoGXC$Y+h1X<|85v7SvoMVeXjoUWVqWIRHq`y`I+;O zfX#+7ZWcgDm~w1cKt-lzxiqlx^&)3*IJIEN$-ujTe6RN>zy}LuLfYe|jQuV$XB0HB z@VhFb2kn=Ao)R59(lFdEcLg;DqK9oO(cvk2(q9!tm(ua#TQ$lk`d1)&ocHoiEF098 zDhYcK4OTvT1hdbKf@c=5cN$OE^saf~mCaYvu>HQ&OC5lIHEJOk!dd5ks9m0NPt9b# z#*|&nbnQKRq~-%Qg|twyAJgodiZlOmM3 z00(go-`V+690PIAp4yaEq8_m=9_99MWcQ?J6!hSB+(UEd_PfF$0T)ShUJG@}wk!ncq{B(s zh&I`#wq+6c2~)z?eY{)*dV(+eG4ylc%i)unKLJGh!96{V)A3b_;|`guTGqR_-KLl# zpEJ!6`8=sYcjx`xzO&LGg5gVpc3Q%pHrB^8J=Nd@o9yl?kDC`+QyY|#Ig#o=yGbQa zEw))Hr04z}O5SunU+IXnv}?uU;R{~^f(|@QvZ})Gbdhk~vG&)bSF{UVxbR||OSrzo z%6zsB|Dk)Ok$V0F5I|K7CxsRDayQ&$-?jVRm_Gqynep#$sV4~DaqZ9*Q#XTOQ^4?H z5ZO>z5f&f4n=R1qePYruX8M5m(Tw{FuCMb**0@1EROeFtK&nmVEXF`HzVlJ^YMI~8GyD9N=*rr*K$#3l zmbM%`?9eiP^~V9}4-YkTV`lS&Hi1soEq++#-nQT%Cud&IhQBn3p0t?gd)k+!`z(OG zIPE{YgzA4Df_z1W4IBRO^8V%h@6!A#_E#h5|Mn_jppDHkZ+FF`ebdQuSeHOtoMW2u zAQ%Lof8O*gHpX)T5_HM2q(?B_osCgfD;*ZLb>?cfg|O5}rpVGn7~YPQ+-!gKce}># z2)sWx_8;dPY^eP&+WLQ|}%AJ#@8U7`6OZfi2;F+pU~b$#QUa-Y%Mb5P^<%+=^+ zyW`NmJFNZ6fc}%)4C=>!i7)yqO_i#BN=)KUfCf4>uuSeol8yy12_9AUsp7pxlK8+| z4|@6C`z*&a$bNKht5N*jthMT_SBXN2FK8>3?dmHv)>D|K8wPRJ{M9-(-vqwk@6wp3 zc7D7CqdOL~G;W@tou?Q#zNWHNL$*NosED#E#LTso)Lu@rE6Bc%NG2vkw6|mcJiTf9 zx90^$73DoMre9ePf!#$x)M?~*cr{LnE3}zBXQHWf9J6P) zm=r&VSlV*SOYkl;)`+YMHBs2(PTCg$J5oeOE^jV{?bCuaC8t3>`TTy0-5Pd=Oa|=5 zy|)OJd{YFYMy^FS%-(m~czk=sXq4tkFV$IA8!DYBFY+9e9$lfaxmt6FJ&%eU?*Y44A~ry@xX4gPhPBM` zQ66)2kji6uI4mtlH|Ahh=c_`qTBgVRwhSom)p3yaY%6nxX0~pKF#m~_@p39P^eZ_v z2G(Rn0ycNlHV-;E(C(dLU1I{9v)wT{6Ys@UsEh5$yJ!cz=92?#c}b#Oe*(hmBqkB< zb#66TfvbDTp3J5P4a3$7Gqviq!o??$$C~V7cqy5bP)Tx6F z#&=xK6o)z~EgB{T%M(ZmAHKc!Ann#(=;9@4`9hRHJ9`RCDFTAu#phzNI>L9QV%#EU zafO2WVx)+YK2n);m-yafH@=y3J>)}P62z2uPxk4!@X841q;34jw5j6`H;5+llj4V- z;ZI3k&*-kVDwqJQ`>N3FP#*=Tc}wNwkoLE&cMBQQJ7dk;@T1CZ4`y8%a-nBudB>esIN{S{3sJY6dH0;)^M#r(T z!CmttZIkrRJG*pD*-;_Is};`l`Xdt9^`i;^XaA$|myOihQmK?sEbX<0Q6|;f`RY?q zdHZ>Xe>;BLO(!BM23&EMsAmFHL}x|Bu^VIm6#nUn>YD{BFH2cJuiLqpl#wy3xf!D6 z3csZ1-OkPe%{NL(qrRy>ZX;cmOS`oedZ0%HP3m|9MPU8nu$-6qj+0#}jI@IJoYK6u zNs2r-)HZM_9@J%@Y#2lyWX$lCeSP9jF^B)ak@{P_ssHR4c9ya)q{AlfWUoA7HLy?l zHEpA?$;ylDv(TA8MFPoxPX5oR#!(?D#<|5T)z{0kaO>2#_|>`kv{ENCrP|5vdY9qZ zj}bxVaIcKj?L%+GilU@UGz6(5u-r!ZxvZ*S5nKj{nb>BRD8Rv6m5`I(ePZs#Up`u% z&pFx&VJWCUOZjfZ(uS*WMBku%16WyHedp5t+lmFJ*RlCL&4|PC{_Rah){J*Gjs+PW z5J&0J#UGUYoEciv?AAC!Z`F0bR(fG~2Sg6;?`*UE3m8x4C)zoytL+)52H$c=B^*pA zv2mzw4@mA?><2@nJjpzD>(ThZC*MBfq3g$UqQ6SE&_f<@QuLU<#l>x2OFp70S4zO- zp=?^L$l?$0R}9ywrf=nXl6_0?$oW*@GWXOr0puhl*1_Qz3?&}%$^ZkVrB}r-IVP9J zynv1#_e#|lwY!9+St7BaN{M+A#paJpL5E)|77|kGjEk%;pJSYtfWe5Z+zYsL+$*LO_q6I4aUH_Odyw$INhp9xcdiZCc{s;F~m zSWZX_RJDj($B0`bzL|ShRZ)z400qngi}+JI3mos z!N+2m3H6PN>))g4Dpu$Q+AMaQnG#;C{138ZO>$IuD-XUlVwKSGOk>=T2P zZ@EH<%n(-jt0#U--gj7*!++x&W?^ZpacW1Z#_T zB&@dn7*$dIeG{R2nH9V1ChP8_U8mb3_8DqR^>Bu0i%8+^taQT85|ZWZ_2GpA7ML2Q zd2HS9@Ha1yaYc`-WM|LCUFQqg zT{lTi61b#J5F6(bwT3}|5VZ0e5MPrJ4?PS8pJbwr3*IM6&t2E1i6rH{l8s^!nD4Oh z0K?u+a>_>xD?Q2?xc_3dijN~wJb8=TGPeTi7H}bZsfng7&ai*y<%V20D{H9H6Q85S z&3<%pO3{B{YinxA3$X1i?DuzCyL%eos(cFrYi-~d;|=ab3rWtJxBQrYr;684aAuf8 z%o1Jktm6J4%@J$J1vbWDgW-ByaUnf+Eqr$+F)(5!GCXQ}c5Vh}@o=w}L@E}M>!3y* z1gSmMOTzk9!oObXqL(JTgzZF9)20yS(Yj>0&6rD4gq~Vi3EH(H6>_hBGJYcd;e^Kemv+%lBU;?iN|rrhHwWZs5SY~E)Ryg;R7U7U>j`5G zw6)VrWz$%ul2!f0u@IpzRUIQw(^uHyvm+7CykZ03y90cX(}5^y#M)X;M?%QAQ+TUU zBDvr6Y}2PZb7W#B=U?WtB(^eP(+mboPs5Z&lixda5v~wlfZG>Xe8_;%u=WKiMQPiD zRBBADx;LksNX)d*SBaUdJY7?6ccc^^2OjO992cam-0(~OK5Aeu zb7S+rF!%(Vfm<3&-(S;Nnt{E6+S(C%I>9>D1i_?lw2J!IK0`QTKstS~q(9m}YvX?8 zzb`$s(TI1(!=%AKsVXka^>%o3_%Qv4I}-cVz>|5gTBk7YM$bWIGcSZ;5JrDUz|^jc0Yim`K- zRXk?UPzHrbn<9aF`Pz&kOJfJ3xo|r6@9Gb9t~Rqfml-!cxzDP)3=FhuKaZ@;<^ArE zIbPPwMD!EjwB{i(WvT7X+$mDq><-B8ZMoWmE|-h?${6dQ`4KPLn#v#ud#`f22%PEB5x$2D6PpKN(;k{>X zO``7eu@}QNG2rh|UUDL!{UO=w^_IJ;^HJ>2gjKuhV#fPsdgb7D(+z7(Zu@oOgH_m5 z%jZMQJ6MGqMihbe&#)SXib|8><_w)D$z>CjAN=iCBV{WcXd@UjqG(x+o)uwQKWBuMp72&lh>1NeK#Xqqm6 zL-}Mc3-h*2s*NE>AZ2q8Cm#oab|YrZcY$w=Lu}ti&7qLB*qUxHSH{w|DUKP+3Mt_< znUC7W&7|3Grh(JwdSw1(*i>h)&s$N^WVtl$IW2tS( zO#MCGp~{>#&x~~wuX^=MegFVoKghbN3h=VQ0&)`Ii2bK1ctHHcg=iaa+RP4%@yY1rJ2+{rgl2slY`r~>vPRc z++gNGJLh%I_OLRXzu*wO`=8+M^-LL$1rlvFGp$GSey z4ENgG!n^PPMjIf$FZGx(Y?F=f>!=HyXyb4z2zn`+Wx46ld#m+LrIR|P(_v-2A+OV+ z-96}fz<8tG9oHn~TMgMc_BAQFUpH;kW0s!`aW;f%n&M502^iU?n!Pye{;i`y zYQlQrxVKkB>IiJER#F)>F*kpomN)UOm3mZJIFOK-8c=s0_>br5{w58;O4MrDO7uSC zUC8;#Pk_r$fVjD92o^(E2sza|q^bOqQV`p}x^OT}_p1~o6mN((@8^6dT&Jimm}#l1 zF6fT&8&X-WA})4+33)>4a#Crn!uXFCsWH`Yudqy)@0vVoJFaEMgjg*BgQegrvUBBY zQeo_yQ=`1}%1aDucHOA{6VM#k_!B^937e*GJUbM<#45E%_g`fP{RE^IUkuMbv0%Ft z)(c`glHI@-6syE$K(QYJb5X6=L04?pT`%oVz@;VIp9@F9KueA5J3dG5AOA}iRzFb_ zz4pRpS8=dX8VD%l?1HVHWt9f2f6y|9vBv66_`tDx6C|)}5je&yl~j2HGW-*OvW8%7 ztcS0nvBa2E?B2TxYe8|6!gTG^xv%b_`UZw7SE7dvEnxM|7h4v70>Uz`X|r3&&zX*3 zm-Oa80caQ43#=~3dS4@o?)t&6hyB&E|6n)XDoeK0nEWyMWN1Ha%#k_HG(XEUgGZN$ z^nG~1WMW6dje7n=!*3&pI*h{|!#QW5e%X~#shqE&D|a9MC)7pKj!MmA2KNM}=V5!b zd`3RC8vQyaM9TG|SJ&9wdFWT=Gorzt0E@3!fs)L;G92w43fMc~#T7P}PNP1>etpz< zdaidZP%wO-x}llyGz**cAAz9_z5U7ewSNL&ShN%J`!4n0EjIBwA-6rxTzX7w@J42- zN(GY6uRBOByvP*5LtpTuI7pwKiYV=-&)+?>NBd+ugaCnw-)~VUCrQ?7v3K0VFNn>t zR3Rtl%HYrN?Ycu zZer<5BLV)QQI!7NBc8g20 zCq2Sd6DfEelI~l?j#Rg}L;8kD3TNYw4gX)fhQaJ?vmHcuuJ^-|AHy76^(h1Q6~4NS zCn(p{d`pHZWVawg^Lez-cE_Sb(FyySFT>xvSNk;Lf94T?75jah`ioZoC#wXkU_5@* zkE>NXah&(Dw5TX#Ed``I7Ae zo+YYhk94hTBJap(Q15ux>JkyM`yqY}GXLV^u5pZUnQ_?Af0h~lpVFQGlAuZ;HKt%- zU>`Xd;WLTn>=56|G$tcyOY0O31RHl)as33ulrO{1x3hl&HmRSUdg1ftJ!ix&r(qXf zR2ye0(X{txAE$dGq5Oqi zdSicomD}AY!Ssn!lTrE(hk~xe0}Ghs)5~1gIudxXZIbYnJZ1OfwZHmN1Ec0yBdjloNpr_B_Mh{ zqNXDDb>p_iTdsMVbvDR;1D)!6v~ChX(7EH-ojdJwb^LZQL!{xQ{w0nBhK{f03QPIZ z$YWlj+#{1Z7zf`Ug@N+R8p)G^mcxbIWGC*8<7t8rs^XSdyhAHVer3*2dJOajY`44= zC;3oT#WrAD81=jN@9^j*-fXxdG{GN^<;*~j66l<}6*uWV*``oW8I!&^yU+F&8+PeT zkwjTm2K(^ZLzaNI`aNl~PhQYgz@{gaUseW+m}bWJ>X%2UzkDq3`C*;@69CK7YpF$z zWSVJvX`?!_3!qjC`}E=^l_k|RF7;Ittmb#-=V>HpJYkYu2~3ZR93r+__lWs%7i$ZH zl=#NI1d+me?TI?a`ypj807&IgELFF}7K-;PCM1?qyKAP~$hOzEp1XNZ3Jv?_1w`A< z=X)dTgHs24-bzdT< zu;~42DU)9aMQ}K?YX-K00@^66`PKWPh;S0#THo*cyaiTnpC=4{g!akG2Bpw9+?{n# zyo=F285nW4$u?;zy7_pJZ7=P&Xgg5!wAa`(fP=3R0&9jKKjFhTt+}+o0(0z-t@HYVpF?qtFQ>OmoCW?vU0nqFhYsY$@cn^qNzJ zF6Dw-LnKV+b&Tjhk$j)lLjVBqF8?1_D>06|w3ijLpJy9OE2(XS>rR%<(Tn}!B2ax)^S#aj! z_%WQ`({55Cnvm{N)cL+4W1N^@9Ny_HM0Ak*elL8FJ;sKQUr)fp!r1aY8$IT<)xDBc zVSF{I6%jZX`m$k!sr7C|e%*oTl-X}ZH=}0RDIbOi42%lK9X{~|tl(t7@Ot2gBx7qX zOy7!dvRjVOHcZ*xiS-U~xu~p9WKbPgxZ`&iO&r9d>ka@6PP#lMY`wmPQJY|8KzWzg z)7^A0gxVRDPtqdg$&)12ac4T+xamnFiQCxl)t*AT3IyAfZq_%VI{km-muV+UJ`8Bh z2h&tua=D+DPH%Jk1dOme^Y=7xRHz^|rn>Py=-HCF@VOv1aE%Q?O*ZvI(1h(NqG;)l zrQ0`oQ}N4p+iYdoM}_s>1Ksb6oT&%}$-FiO*;F+6e*GWj!T%`GN3d5(dgE?oh!Xv_ znN^4br0{-amG-myvNtA>3PJDfa3y}&5@;|V(`;cMI{+iGwcMdhE3M3Cvguyq2EP>o z{w9;VR7}(s)neU@c-Pu?(M|fD;TGQ%7oZ#tuQrj;x;tg8gJ9K6aky;OKcuYM_U ziS*2(egceP%l>=UWRP7zUtj~Q7OU4FoopbibGPxzIpHkJqatMa7)GhHxIk-$&AEd) zW)TGPm5sh^@!B9c(ix`)$;0#=b?M3A=BIfKT8WJUwfnD6=CUScT;=nSo1-6Xp3Eev zBCZO{7j&=4RVu-O^+`J$@QbR$>Yn9<&A?ZUYp;|KTSPC;7cO7tVfB+ccRR0~!_Ml# z#vCKR+$ev!bpCQE{pHU3%bE6kbDFyig1bxG$KaX%3>vShKijdFG();ba47|-5s%;e3B6H%N*g8{H5%8FFB37Pl zYnAu`Vp(vK`LF}Ly9JD!abCVN^V#A+IjE7!M*x%XWOxN_9O!a9q^M$lYJ`Z7$YGkw zXE40m>fRvNVZlroUb08OA3cV#TOMq|ZW_K|WLvHHDjLSSKdMs^l86U?#-l3vu!bd{ z_P%VwJ7OsObV-8>Xg3B@I!dRKL2m)yzcWm9ONSJ6vRq0cwy0)Xp8Gv3uZ)ZGkYE$j zESID2)GA&q3~Hn+_=9K6hsB&vm4D;=)BS?Z``wTl2 zzTmFtw@n%uL9BAmJOo|mT<&RAy%Vljs>_QIF(vq63+cX1TeZ)CKs zyWnn1h>)+K#HoCv^y*_b;Dw#sS_ubMW%k>0Tje!HSleS57^Jph_0UL_wxwUYZ&bsO z`_=cGvwhTyyL0^{SGPV=_0AF54K55lQ?X;shkPfdiYSGK*x{u{zK5oK?-$8zNw> zUZ;dXS-pJT#);Rdy*4{KT$aP-SDXuNj66rO2*+6;2V=1+Xo~Va0rT^^1r5w0Di-Lu zaRi}Mg-+0p%qu5Qw`oEM@M4Pt*(&^IT2(?Qo6NDG&yWtdx=PMwN65 z(e@d_`T7DMN<@Bs+M!_F2P9C$MeDU#$&KVM4exu&F7}d`=j@eTX&Y3G=P$=qoRMT{ zq|K?*%g^%&5Th$OEA!00zIkH4pK+bg*OE*xtz=h$DBQfn!nvCl%Gp}>DK4c(8+f7q zm(W)|=gT{<6COU~tP$i6qx-xX=+Ma|vw8*#R;VgSFuwZgL%Kn+8N1EO6FTl#1`kwr3ozu(r4+^(tEFRs8j1u4UyD}MRDGPC(Ijhs ze`7ar<(Xy`U?lCJ>kwx}gy7YFyYl-K6Lt}Nvjw5)CMpZR$lD>B#atp&6l;2mX#UQ6BzQJ37H*5qx&qC-%rQ z$mh9JzuhY!o}#<+g$RbOX@XYS7A^XIPij}oQAzMq8fdjVI>khPRr|LXnS-7)!MYoI z?`ZibU!J)cPoDtSt{m8m(_Qyn_i7sttP+&MMy_Qp1+duXTV(`o1`KrqP?Y>7iHQ2ZoSOkWA3m= zq!pNRG4y-2leYpxPg0oC!J%GvzP`Gv7vj1d!DHzvY&QCMY;R}B zpwifbA|af|3!Dck6afk?FhWId@9bJkKr3Tbd~OER?#W2LZR4clK8I&mk~JV`guaB9 zu<6-IjYCUBcB}!j5!UyGZ1VTfAa^Op?6k5Urnlk^Tht3>#l{&ff<`zI^kZep+s``u*E} z6wbk-P1ZY?RV9;Z;de(r0g?k>toylBlJ6$71$cQcHhwTOnW&@8H`_L{WAFswe&kCo z5uI1VD(UztjrUyIVZQj>#)w{1nK&3fuBomqFRF+%Rjs=LuEu+HIG{rG)k<5Ob=Hon z<0utN@k86S2}>de;on9txpF~=KQ?WbK8g>%p$YJK7kGgW4wOS|6n~HI&@($J443iY zZ$Q0>Z=gf1B_olC?I-mwdUmIWPGAd)^F*Gg<<`wPTHsjm#8?j_RoJ6`t7Cm0vSTc+^ zh}%uRvN`d?7%UM4>!4FZ939FZfWb~uAnKiT5o19C-|*km@!}QPI}Riy4_PBncAbt( z`=M?Pl&HW?f3JH;QF3jYY>eCTdV-9kMjwpf~iFodDxPdmzcB0b!_@ zYwp%v{j=ZG2l>3?2o3YTD*u}!NM=Hw`!DZn1M)j=Hg^i&9X~3$7(F~X*l`)lP&aR^ znYU`N8SWSI6?(lIUhU<^UsbzYm!`V`rcb6-J>|i6dYK zkrJ0VZ8qCp|KeaZh(^t_%T$-B!!UZo$tun&UtX3Pq1Y3xrNdrma+LyGQLX<8D4Hnz z3CM~uj-CABGpKyPIgcGDz{c2UXt!WD5`IRjclqWr^WCiUPXHGY*7OkgMb8C`{6A1R zLj5)N|KGmj@?KmkJ7VKwUmE)bo6C0vO)>4Ryb-SUF0h-R$@@v4ZP;GdC(eE7FId#6 zvktyZs`(?$TGx1bv>z3#EwOvXa6SZ* zet+;{mIT5vxk2Q^2xXloxv-ZikG!X61h=dv;ST5#DkFTFsP-mj;8()Y~1E+wT5I zoK2h+(}B3>%vmDl+nw^5WGZP#ySeSXzMYY31!@I+dD-w0iw`wl3_!v#+WjB1^(9m# z*Y?)CVxSc2%m?OYDUV_~`EM9KT*h5nw2@WE2%;8L#)XePg=Ii|rOA7aBWve-dn6t! zLjBxg7CPMGGNvE+eZ$eAfaVfRM&7LXrhYfXp%8}LRd3x9n>C;Ag?`vRg#8eFoq5ds zg`d9jcD9!pCPKCqyIig{N>VRrK&x=a$D>SM-F=@&_;H}wF<-l1A%^-3{R9u6(^~>> z*fk9s)=w9sMc1J`U3itvGTfkYl9$RM@>xr&8?xph$d$|SiCuU>T6n!B^~Qp|nR3v4 zhlxWO{>xK&ZlsnMZGG*e)87p$Dy1MVS{v^Zx*GYr_)px;O zpF3N0=dO)`GkVqWQ?tv0Eo@NrLV1n*5enS;kGGSqGLNJhM=)+ z4OC@V=wnfx2*T^OiWe`Pb1#gnZgqHgoQ6yLBaT*abO4~Y4YlR-eK6dCAH4F=taVP>H%N zweQTpsWwaO4H^>*Au91F<=pXf&=RfKCZO7+54qPpG;g_c;VXm-+{t_Oszh#Y&$^;c z98jE;XP;?eCy>T3!Sx292mlb`AoW76q({rW-Jc9EjlM)JynlUkaqoexX~VS3BnYoW z)1l{rQeWXc{}1?@m@TQCbh#~B-MFf9Br9fq8k-ksN5F%ptg9+Zn>8w(s4PY0tC9%V7Us614Riyn*WqoCbS1fX^3kcI7*;MTjZ)4!`{ zPF03Uk*Pcxok!f6&gU!DnDS^|y+U@y&@yR#WWOUiQ7Wn=;q_=pQF+Ocb%dzA{vC|G z{l}a(Yjp0s@L-}_^Nsw3MC<62B9F}H=|vB%iyF12yp&NvNsbw&)0<1yL&905C2BG1 zl;#89V>o6zp|+|F-kp);W@~mB9u#s}d*sNjxQ4byN@B*`VGN@Gz)_s^WM5!_I|kPm zPhXN9!2dXn7?8Et$bgw_pO>8!wwf5#L+wSJ{C2PwYMoP8=_Y3B;8m09W1t<>L-o#D zE58#YQro7U!AmiZde|`9>ZrJgdiV4@KkjO>h%$34Dg!AKm$vPG{;fLuRbpU$ z)_C96X?PCj^JoS92o^tU3Y;hO;Q#_b(I&c@lq_T+9p<)b-Q*D&u_JBI!Oikq+l0qu zRq>|C6qAVMZV9gMWVHiB8v_9>K2b0{jG2z0>CoEfh&ee$w_+u#t8ZIjmf!T|tgb9T z&H`ub*`%x^%Ckx4nr)&^YQmk4Hia|d`4}tmde1ZILzJTlVoNj8J2dlq&Km$X&LOie zG_D9Oz26FBSis!c*RLBo7OdqjqpcptzI?ygVl`J6>#Gy`HNNqL<8{XntSk)~=XkN! z#@zH$S+dk^+1Htn3RwbwC~(cocV!g(aH_&BOJ@f)!tv9|3o>Z#;>#FqxDDmf8J1=v z%cM(q_d1u^Ne6!~%iAlRbD){BIz@$2-}zI|mq7WR0vRu^MXW_QD@j!uku4RlhqpRq zZ4KP}d6r*QLhsFi#*85#tpc!t4{$xg%3uFIqSt@rI1Tc6mj-!+^SdzhzmaN868}uL zars@M0O!y0+W(qZx&KxQcajWC0WZFkL*1%f&dnz~w~sG;`^@;D&rkWiI9d4l;-s^c z@P+E8*|%zxWCubm%uXiV+x?pkv&;#)tfXhd#Ln(7^KV^X0jp6<7QkSJNNuO_4JdhffSg-|)x^>a%l$b{2RGVn@B8 zYsN9Jo8P=tQSkOul2@#vwc|ob`bkD*CGp0SLT19L){DEL$Xhm%Q+LTSC2)G#zPZkE zX^me~0PSVY?`?q+YPjQjzcp8otomy|_n*UyDCZ@11N8gO-VW^u;?$2oOcs|rDB1|} zWxqW>ZQBQ!LfuW!THq7ecRm43^~*D}t)d@|G4y$h3T$zPQQ8!D$Gu6Aovrn-76dTd z@%Tc}QNg&9j-tH1Ph9MVblh;Wmtflu#r6S%dM^lSuz8+mOxVkO5mP71Gan)w_@bnI z%+(U#MfEnZ-M)ofb3mzsrbppaQKkJ0JR$q^ zsl{(=oZmd+e#<?PaNP;SgpK6w6ilEK(sonS_1=HR{9Ua16-^_@6J(~t2V9~JLWt_I)}elZ$fZ<=x? zr!vFXX`(hqq;}nx+UGxeC>nL}tDeQ<5+%0;O6I+GYFwA&UQZm|E`PntwzW%neAj3K zI3BBnNCFP!7l`bNNDJC$k1k#4^g$#9n+f^)qcwZ@2-h5Lkt-TmqdNoe`)Ij?<)?J{2OpkK;PpC+-7VzY4Ssz=Hm8D@lxg3?yEI7!97 zueRfwiNL)ar&}f;#<*VK(x+1OcJ<3+^6Bykt zmpIB77(GqXm$o0$40b)xCJA?<1DD;PQ&nu&l1>u=zFlbz=n@zu0JX7MIyxc0I&EIn z`6c^dR^Q|DO8z^@rvy2Am7510o-qCfVDUs6X8GOy;aJa=TkLJgF>3*nWa`tC~ zKSlaq3ieZJ{yesR4vU{N&7bDo|D0hV_$A?YoW1_o;P3xhsLFp%_^gXBJwiKV#oxTH zu{%j9SIj`aEu*2l?+fT-KzzhnwEXS3>tKbzE#>>k_D!#cV7lYs?-!EZjr|Y^YYwUJ zHX=XZf>kbs?o-V;S{%|Vtt7(r%x852!!SIlad3HYgV0&8RqjAnPgha3otKL>(sW30 zXHwmIh_~@_ZdvU_1*03Wfi1BL#;sOJSn-2cMP5Ck*oUbh9JS8;dQ|vGC&Uvu1nQlM z3rT6WM*-A!5KLl`qi1*AziMfvGfR&Dc8+p*xJ!m5f63b$sgfsWNECcLw`xO0;xvuT)9nW)pO29vMR<_&5 z^<2~ywSKEuBSvPiAtMz4I2_^`kyYF6V45P0IWSqWsWNArL#l)8cTlw@!)*tN))7qh zTv_qfwv=;d7hUjd&Us|xhgw?Kj}F|do{WiX5oCd!hn<|S^EsVvp;y@T<@l)yw$6Fe zPT6_OfxR3(a6iw2_1(_*% zJ}8s83xl2TlH%A4CAhHPa8E5Z^X!Z0e&c78)Ee&Oxuy?EBfxsLr68%l+!0Rs%08TF zyNN9)#VX-;8P8qny83JP^6oTHt0M0rFlUE5!~z=QY<0e&=qBuT*C#9Sm4!E|jjbW; zjr9?Op)$)kkozx)(YZ=}4pa;pkdaXM4S03kn47(3yKYQxk!>us%`T(=L#_yX8dxiV zLkUS4Dfmy`CQ}XCQjkCIkYZ{+oaD$(nHd5 zyk%Et9BfK1nO8!o40g3wFPtgow-yaAj|M5&r#OzTCzzU?M%ZIvMBLr-Wo)AmsWTD#QFiU7f=)8X%lYcq}!zjh& zyFh!cy2NC*bvA}LrHNaJ>?HM@GOa~m=`uNARxGvxtKt2lM{##otCaM?D*d{%GQxK` zqsf?_Va|nt&KcF3%O>r(DpiFnYAYHdS%RO6$0>b5OL+4Q;NmWH3s?cuSg9OL#eYDC zUD_tYU`4!!3xS`xSs&ms)hnv*+~{s}{KpNt&q5+302eQIj{bvmiwm98;yjx?Gv8sJ zUA6Lo3s`ESJ0e}f(8sf+lK>wX-SJW(3AndFU*)5Xer{q$>Ea-B10Pr6o;jZ)$w1Dj zI6lZ8fTwFBIiP8*91i25`fMbwh$D|N&DYacpJJ!Z6h~OVH$aXf%A_M$=HbvhMvOQ5~J+Do6$=po^9F5If>qCDzm&!(MVL z3`@MFHhdQkD2cbBXFhm%x%ozMZ8pScXU1s25s%zu_!d(}MS3JOqm4%~EJYz%#*T+IX5OIF-VS~vk>9~&&Y@_Z^Y!zyxA}e3)p>zZ z97vs~2(A!+ma%kY1oLrSgpNnq0JrN*vDSkX`!Uw26~MR6&k5m_Gsla$xvhmPT;?L~ za9<>ubW1r}KBB`+o67YkB{sIOo3v{9dDa)=&& zQivsi@^X!2@+q#>^MVf8#i7B*gs=wXyjex=cYrobrNX_2iEc!c`&7v{UdUBo@^JJU z#yz`5Js|!DEl9Kn%h(37w31esXKYJF_TfDGbCJcwHF>Zzl{G_)dx)#t;OBt?xMU~A zN{c`!Vs2$ZigM&qt;lHbgIw_(bxl^{*q8p>%Kl7oSsnT!F`qE4a49kF+2;@sSsP;u z!lY#ngrYs1Y+);d>PNC zj+sXsVXM;st^55L-orfYa;3EATgq`aNEI0#ymz3ddob%QcMzNjpNmHSyhI&oJE*lD zEMjR9Y9#m$w~x3_eWRsV*Wn9ZjLa^_z$Mx;I3055W& zT@77gTJM>3M>=-@Y6-@cFGXb?nT`&}?Zokf`O;@|J>@Ms*%$>qVM(WC@E{Rtqsc6XcD=GriBlH(4cPpmK>$?*YGW`Q%>qy=1MrkFyc94{6h;Ch^K;$d} zQFs2U6Q$*i@oE8&D$B>$(J8)BzTDrJvjjp~2=B({Xnh;QxfbRu0qfCKQ$NlE9r zP4*ylP_l?Cj^dJp#P!Kc0GFyH09Drs;A(~K6$V8El%*$AJXd%+ZkqpBV6)%Deg6M? zZ%juI83rPoQuDl35z&4P*O$GFiR72-#lds$m?wPuIb)yQFN+BqElZ?iq+F-XBb~M} zuGDFx^rZn<%;-WZWi#dBkX#5ClQK6qx`EQ@<7+ZeYi4fGjjbst;SKfrS1E$gujwiX zItYl7#oOAW4R?!XY)0mKbGPo=Rexo>C7M+eiL2C;tJGc)(2FtAOwvSr`Dmn);0Drc zx$7Mtl1^pZvRtUnArmvjjdZ-6G|sDbOgZfuHFxRSOf~Pk5?1?Z&D*14H}|;8^NpRN zT}NsDnNwXG=XO~9%efDA^@U44-&CQep-8S3A6B{P%4Sjn**r)bM`*=Gkg`SC3K!_; z(`YuRPstD#%ysBz5SCqp8aGClki1H|Duveo5lU5DTqUq0ydC+<*^pr6S6$vQTP@MT zCST0)ru(6|(pCV6Q8K&l@CY)yeJd71>v&705C}?n$fddytfJ7?w`gKH+upxWakZlz zy^DX)eHw&l4)eCjLqgSRfI#9RU0Je3&PondvFqj$o|`_@ACuSKwGv%h!J?%18C`Xf z8XS%{6evd-G&LSy4`@I7=;AT8-I(-vHm!OU8h!p96oO)aG`G!i!6Lt=WI}NEI}_tm z_r{;Ke2%0{XnU*s{4#@gXR_Jk;vocG771P+ru2R?xqECV*ZIs~*G6rx5JQakPg9^Nx@-YEpO2hHvA%_TgYHgq6?k7m z)@Lnx3nSc9vyHbjH;ss@4p7-7(n!mYDw&XdxMP@W@7F2Ix?sxVLU1eTcsqZFE>~{Ee5n>lx;JvMgZN9+ zZRCwr9(*wLn!%%yptRKGGYMIte5$2x=PFjs$VW;{@%^42QZ8$mI4*M-2rB6o>DRMr zEovcYbdwScp3TM5SH(KMC7S2X3Zg zH3@Htd9;xu)^hZ7_k$Z$gtC1fRduO=xh3MoUxiEbDX1t=q>05QuQz}%q`V(ZN|AY6 zjR=(7%-(8ZNBB<)4?jnV_o{)3x>1#^Mc4EeQzu1qxYvnrr^^{AcC2}SS}gPFS#|>Hg93VK_I;-3POU9#fa4+>6A}+6XxS%0inozL9VbuN zU{i|(d?M9vYQq%H*34ytG(AnurhV^d(NNY{PwwV98gG|&-WQnE&EL&E+26eKkPMfb zpw=06z0w*{n65%RR=*v@M{j4<#IK(avXUa6D&784H3;tMGN&&~5+37Tr%=$&NWUC4 zuf68hZQ+t=P=B37mIPLASQ^(u+hDcm`!Wir+f9$NH|F+{SJmG^#lT@pplCr&d0a5XtzyzX zyDo@HS<76#)jTqQ5%1W7kv>ZP<(Z`g)xCTL@IrDqSeZw0(j$D#NR_2wz!z=u4SVaw zh==l4dcpn-OeeYSrM5KU!U@O^fTJAsCqo>G`@MvE(>N9B7I0kpPv-HBFF3!0j?Q@2 z0K`WkBiYR)>jGnobQF~NG8;gZWBal6S5~*5Wk2QegU0;SiJxQT=REP38T5}o9>AU3 zQ>cFFOO(nP05RO5^Y^q3An(kzX=j#bfFENIz@pz+X@GV@_EPisk*Ox`t7~CX$pcZU4#d8dk`VP8K zE}mAa_0nV2HBpe~Q;;jB*z1e&vcmX+K4)DLS|oLM5$Pfyb<~x{kWA#U628Hin|PmU zU%j+GjnE7oZSS2#c@TBV$J@6pZH1*cCk0ZUblAWLjrz79sohYt`(#+wmZwRESP<`U zj;jb~kZ!<3_in-LrKht*MtAy>9r%RV`f^Qi9;=9&6Xr}$6U{vs`@~5oQIt`NqeMA` zCK+io<`&6n^eJ3xWdHeuT`}gydR44tXm>%~tsEbGZ^t5@FfMQTJfxb!M2dQfEYhE* zDV4jNX*%UjO1q{Zp9;&qV2e%8Tw+fTWlOQtx&eYEe1(bpJhCa==_9gPLyg#Of_21h zr&#+ltO3^p&x23akuT>7XF5;vK>a)KwhFX!;-`X=Pt-<>SSAGx0xK%2DdY5RPN+ZT zQ=6Dx-;V}z4hj_xFyHV<$kI>4T&e4~RccPc!#TiJ%DD#a{`hf&ALn1M-beI>-q2q9 zX5eG(ZHeI2*@q(~RN>$Se%r6Rh zm6lc@$P$>whu|XF@=1yArXciM0kqu;oqi}0>JEgXLpY794j`*owrRqsp$c2rs?*N? z!Z1Ovt6JHRq%PcpBT-WQVzIA4cEkItc*$)@ihTV^9I*lPzAaRt_&f*LNW8<8&79T?NsyF zChG)pJdt5HGdNxFB2_Iib;U94mC&SHs-;)-rvc%Gt*R$-w`Q)Qq?_5EYTSC=J99^P zL1F}J`dFt+N&+Fj8y!{oni)ZZN`iwqc4>L=H}YM#@vn;d(cU!H^vyfQ=0#aO?_!o| z4d!#*X<~YmjEY_kPj#?FU?aM6X*$!ByDY|L_L-}W$VJJ%@)IPbzgdyFXaM;*@5N>- zGhWxih+5sImUxA|h3_r7q9RSS2vKW;+5^-*g(|D*xLeUa zx2MacEW_B${WIFpi9ebKW?paY-*W)Jp1f4`v04VQBsZI=5&AZD#7L{bil$n`^@YSM zJzdVx{iwy=V)JTx_E}N%M41o=67*w-urq*|4AxX%bSaCltb|J%9qPH}1FY0Vns9Rf z_jR1hA~g~7M(UU-J$O z9!c3jkme@@rhkFnye*YwuDu7?@=#u4WFStF1R%O7QnX;=U`1S$rP<00zBFD@qtsW1 zuFzQa8&m~dRwwqs#RGM^+{+aP&|0yuO0jaiOFnK8&0UfsGczGxkLN6GN}gesRYY40 z-RrfRChVrV8R<}*DYGupx@i=&tm@dFb`LoWgJib|-`f(SvL+)-(n)kE0dvg))F&F4 zY6&^PWa2Ho8h7%$AA-*B{2c)CYrnUw!Kg8-!N^a*^-nDG+r+AW=AQpmy3&7eMg?O= zWPcl!5RQz68-f)ND!WQd3Iwy;1%!n(0@e%^sjrpKD=Y_VO6~eM$$RP1dOktZbYr;A zZP7PT0ydP26kp)Zi+^uH*meKj`}dxPd$5_}{BKhCP#s7BPW3+NHx%Y(^JS9aDQl$&$ zM}mOhME^Pfk&AhH=2!#OH2EEr@e+C_{T;+Yx245%b|rlY058YXml1k_XZ8{1NUT?^Gf(i->TrmNXAc*)M{WSX9+M0&5 zqJQjnfP3R-xBq*(-EB`A!&@2-c&XV9q;Gu|O7U{~xJ7Ox*Gp0qJO4}~G%;v_zcTnm zSL4gDqaz2d!tnM6@m%Z=<8@z~lJ;5ZPQ}ZHD072gyQ`X-?qDMWu*0a~D*)8?ja1BE zd#8T~~oTndIU4YgrBbx+N{`qsbBZ<>_8JnKK}29n%HuFs=SR|0hEfU*z2 zZ+;I2_)~1tQQm8ptG}?TT-#*;MbOGWxp)d37}t5$0h1fK;w+%@>7q~(AW^jrRPTSL z`P@o!EB(h`rG9Jxp!)t<_FvNakCaOr8U>?3<=@f}^oVH`B}AuVHhh8VE|?QZjbUAX@foch1lkYFf8KGef8-lwLHZ9W_M>Xakl*8f?|k>j3-??Hwz*#z zrLUaBfx|#t@#Js*ZJ359b*v#B$>LOV8wLv>9TZJZhv&^a&=7_jBr1~{qIPU`@aD;k zD$ur$b%xW}kaIBP%WX}v#amwWs#tR6)0LvC$4XPD)6&*URP3CgoTcIJ^c*`cZr(QW zFrJnY`fN={c)X8cY%o(KmN3|u!*i8~*6dLyGul)4GJrL8!I(P$`qTW^KIpmCe?=1ZQ}ISo~x>c~|j#tijx#wcb%VOkbS01C&U0f_Wj*cTpOv27F! z(oH(7K{T(^-pMh4gDA)SWgvN6O~b_VFfd0Td!)U~T3j4s@o4i}Ow0?klU&l}K}57( z*zBrudOL%+A#EvbB<DGB8jeEvrx zFxd6I?2ntXgN>bE&8w3;aqoqWk=^2MVU|1Gzy36^mmYm%i(|DO8lEOm-F)Di;sRks zbvM;lARa~H$>tjGqUFp=k%A(W8L#r#NbKb+c<5i6r~eOWfye6 zILa;3))&x$V~gm%?fZ(Qs4@9DMwj3%$cI_pyL81k+?Q2VM-6=88sApBK|Wi#u&^-O zxP_}cqu1B%NF_9X9XEmcG2FSvp=5=;+nyhuR_K|PBc%OJU0)7LHDmi?OQw~Bgo8I! zr!Sem+S}>{bueXZ3s%#gT+RR3U(ewVA200@`h)FF{xeL#WjkK<^-Qa}RU*Y)?d@%I z4+_Ro610s8ofPD1*Lioyi6Lvd5$ULo@wA}zp2Mvh+(GX(=JY*H^=LKYVvkg)*lzSX zD1nlXIGSg%G?T}7QPc+!$q9|c0kn-iUyYb<8FVnE-G#O7yg2F_xd*zZP5-LjzXp9% zpJP3mO^ThGRJgF#{=iHAWh)X zEfj+-E$0*fkQAR$>|dz%VY3cWU7a}QSXzJu{u;;n{;Z4)y)RBOc9)Uqi=Pj$>Fmg;<2tdOR;Oo%&tXk2$fdslsS(7jajaGCe((;$|}FQywZ5*&o`cC+43qPCXQ z-rMB*F2Pln2+RhD#;v#pETQ#1j^*oC6bs*?L_sIX9AE8OSM$Q$cCX|R#83|?*!1$~tgZTYYCl|badKloP*U@@%r30}S6@4v80M5(2wa}2i5G&_pB z0Bq;38Mse%`bUu$-HWtHomImpS`#aim$ErvJ{$LHF1r|+NQqCf#1w#Gt1$G#La?Q2 zHr_hSLyDjo@i+j%sE+h1*^YfHylNt>9ubVj*H`wq9$dMyZH(7b zc&A+6%!a1JslslfdvMPA%yRxF`P%Z=_kacG_(p?*+wqj3$aDLL(>h@6FMj%sCs|#g z9Pu~a`Rl$R;oG^Lpk+t2E1-@wEPd1WI#1AUh3BOlY{_Uihq{+*C(%tvyPCYov|j-4 zj`Fv+U!Dsq-FLM2GdFXeTlNiPs;!?qfvM>>)h0c3adSOIy@*z*Cr=!;WMjP9^!#k0 z{b8Td()(X)PXagJI97F7e=_kzy=nY}MoztIlCiD|5?Ga_UDa*PcV96AClz7mY`Y{|&15;wT zm);W_MBeE!@sFni{GyZ3tbTz;SWRW0Q%%>XAImAPe|3tR!ib$L3Y&8#TJIDvxBAEi zE(b`&!_|>$Lb>*NjnffsXZk0($*fh+@F?^!6pYtazPm9SGK{s5n^LY2NHk~Gkn-c1V+n-u4gd}O?n5EAMZ`N&|_Hg5agW|pq} z8;|j#a85r@DVwFQqU`4Myv#DaC8mH>`X3=RR$%;U2VdXD%?~=RgyTkv%`0tQeCCd2 zgbEaJ8`$EJ&Wa&6gXmff48Zub)2axgYBQJY8KHNlJxrAbDd9A8aW~%1kzS2ZL_K)= z#@SEHmb?Pr*g?z@xq15y#d8 zMc+8N-ym1CMc8W3e41s#&sACKLc&&skfZu&Llb(V_Fbqcq&su;A6)4E#=;0`^pb}( zf~+^2x96_LFFWn0SW-}p%?r{4+6%ve;XuFtvGKkC&1(jxnSsH)E#FlZQ<9yAnR9TL z-nr!m)ZlpY`V`N_3bgI@j|ZIm?IuP5TsUawW+B}E@w8GI9^tU3m_Y|+9sVKdv$S>r zG?xqE$AAG-w|PGZfaj7fcilfb2}dD~4~i84lc$yc@jkINim-NyD>%PIzt_`(B0<`R z#DauWyq1srYEcR`J(mg#29oU5f~4+PyNfjiLDALREbP6w($k6OXLBdOcQ`j99^MxA zm1ILN{oysEcNWA^KBfR}1NkKf=6H7+X7l*^W}b({Mf|$%LOZDr zrwqph@AxLB)&FJ#;Oao2qjIx6(gI^ z;LnOv;Oi5M)#~ox0hNwxd8>RNE-L4GO%80ya`{RX2mvg*1_pmkauG%vYxJzY^+7Dg z6cB|ajhZpmsU5TC3dK0Ed+lr0TM7kxLqXMSTCSE%&oMP2jKatEF7gPnbXwTT)%~R> zrAgwfl?g1NrPDV9^SetkZUCSof9{X$-*nV3e7k?6arR%Bgnx7M554*;h3^ldXZxX& zU+9UOe>V8vQ$np35oG^NwMApuMVvq9B-E3#V(Y8e7( zupf1h?g!=6qVeizx_wZYyalkl*$ad#I8h`YLUvcV7f1wx6x2-KPf99wE#i+cSBGd% z7jfp)6A5rgPH0qQnYQuyzje5&RkgbRd^IxzzvORpYw3wJB~tpqMD}Q9s(V|zdLloc z!81d7l0m{PxbX8%MQ2P7|0!So&6h^Gg>$+a`X5U3kj@&Ig&$`Wap&|@N-f+hJmYL0 zS#OPb^7ekB^~3-XX#b!!Kgx-Nd%bg!qwB+{ zf^Keuq*Fqw_ZI@I39JJj1_};X1hk2Mm1k?oIP-rJ+xqu(e$5!J(~dCvxJ8JJeq0lx zTz#Q69q)k`O!06}=S7iwFvaX+w4P3R3xwRt+Fn2o=VkDxUtO z4$@yyXMQ&Le-kMfy;F*1OoDo4L9UZ(Czu0Yh$T_8z_5pg^r`beH%@^llfMkr-&_rx zj0llIj_^qBi}vbS`EMbWsZtc&Y5nSio-li7L$h~Oi+b4>T-1f2i-aUROj)U*pHt*t zwi-VU{l7m&{z4r8H>RimuUDOhxc@yGXW|c?=J;>k&HmN`(vS|13Wcc^BMmym9Q+p9 zt4EpY^^PaJT%8@Vu#gG1=yJSTU5kC)OHra5jk?eO9W>3>e0-OE7v$Z#A|2Ai zc`&%%UU-qp^o;wR1#D(f9`6&hqsF$H@Y!4jcMmOx#q0Kq#=!_e?A~HW0F(jlWqb`O zE;CldIC3H)L0qs=7u2CfcyDR375qt5fkkbJDYDiBBr8gZaTNlPOUmbZ-$p#n%z0dw z82YhrejaIxn_~ai=XNJoaZ*0h(N6f|V739L`{uJ_qOlXZL=hk_H6kR{IbB%Y^lOU) z?ebz>!`3{m_p=HxvUEJlnP*C6ZH~LR&toI9@yrPxo__#N)=7p)o%}5+Qe={;fbNGT zgmzhccVf8PU~;&;_}gk63}^2M*=0n}kQxB=7GlKQG})l@0SRr3-5P<^E0vEIxm}N& zWggA(8@OQ}J0^SEp1vlct+?|Lo9=E&5x|W5-qDw0enHp!iI2*C)MyPtC>tZOOLV2W zwyyG3Ti-oRQ%W<7(AGXJck>_Z4 zWLf__eM_EJW&<~3x==%gP7GNfO};(b9en8~?8n2CJ!Uphv^MHsHU4O&cnY0#mad|) z79Btz=i9?&By4KnPC8IAik6w!5p1Md+*`TCr;W75?bs$A@AqWXe=s16OBhF@Lw^_N z=H5-Ijx0ZE314=mWSvW9Ba%shD{1X1Z#fo!fhTGnpvI~=k=+afSHG{Gy1m2=EQCG! zii_>|=^4ixzAWBS6PJ2fNUyeTqi~~1)`{aego`775eq`Smu9OWT;wnpPc}?cm_c1A zM&_Id21|3Tg~Wb{AdGrSrvB3JWyH7q4fHi0DhO7WFbW7LjjRqK2f|DOv(?9VxK2j7 z%D58~RUC?Ol;GYSBMqNkkG2a0^3w70%|g`Rq$RHbiesLT)QU#wF7or^dX1L7}P51{&E5|Yc3lCBUt?jmpLg3a?34*5g1moYJA zBxYYR6L4?U`!R~%wu(St0cR#Uu$Wq~dF_asQnoQ(D(2DSvXlo)cpINRJb5(X(s!0Y zDlOhEq{^yGY!^{sAVf*F(N?0_&;-+Xc6CGSYO?2Y*{}^C4ysh6^p={epdsuMBqyBL4}JW3(MvfFF5nUn{pA7_?T=a!jKbZRrRvZr6EnFoX2_^f3bAmw#q1ua_ZdXjDXb0d@gjAQCZ3V{J?NXp!gQcf9HYIu5_Jm3 z^2_(voHT*`5s$`qP#OZd1Vm`AzVUn+efcdMbp^zF-v0H>{flniRBE5q$q+!|D8+w+VlB&Mb#B?}z6iLIN&%RwC^-)NOaYr{kF$`1?lZ9z+$= zy)Q24jN*G5LEQC1G5<$*`QcmqDM1qfT57ShYBLF8x*4#&{mpl${?sk`PFI;+Er(ABB585T`9K9lpOc|o z(R4F5DAq9eDSTyIe*8pSyz${|8!@P{R%q9mzAX8X0sFmon#*Sy40r|-J|0-FF6qF< z^T@2}`=udzjfU}yxfLod?5h->_xDZwuvjw+k^ux4(8h}e1M4QLaRFpW>}rjQmx!re z++#y?96!G57tZ(~Z#_5%W)s3O9Ekc^uqB|7sY`0Vu=$#dO{k1l_kLjXnIa)!DZ`MA zIu0oYah#W=m1Zf@0+^2|^e^uwatSMpSNL#$J)fP?D6%X5(tOy@2^DaXNUR?SyLS08 zO5p`~Q@4rSn?FD~T{fgm|GrJqeZk}jU4z%mg!Rf4kW>5TQ9J#y?YxX}V6#UD(7*~q zTH~7==gg782_h!UIb@^q4l7+00!o=T6|op&WoXfWmg$g9-#TYYf*Kf0N3;_ZF*sp&EwspuSI*fH*`~^OSTVT+_6svaqOI%;^3aK1lso=Mz zm5CpW%`wR5#(Bi@#2%-kcDCPd0AOGA##*+=#NKt4zPUN;daoxg%V1XSR#B|O=jA(o zlpaFk6XvV=c}7U`RAVHo%{BXm#4*-dT9u+$xVGl1SwY#+VGQ;c9MG9Z zo5#!3*bg=Y`L@o;L@NZU(w6KLSC9F%K~-Kf)9W|gOev)P6tffs_7S@UQ*lp$5j zY`%O~<11!wk6l1~+6O<0i}&x?*z5n3Hj z!b8XYDCfc0XvYHH&15@a=q%yvgCrx(xwAs5P>DCpiIOc=&zWKx(F9-qb2)&A1QDG1 zHY+U7Bm}oMMOpn(b&bBImHRg#UY+_bHnePGyKdp+7A}sS*Ph+i^VHOo>5u)q8U{N2 z4thPi*DM?+Aq-?U5+H-Tc`;#_nQlT^rLy8=WrRD`vM*@wJ$pFD%aIzJn&Ye(u81H8 z7*qv3Pf~e0csHeIj0i%WSwsy{nl(1)l@Lnrqw~!TeZrj0xu;pL_ju=ua|zd%QYcvB zo;PC-)U0U9n#}ll^~Jrn5tI_J6yaqvcm61TL)J^iCOQrvw#-r=4%RA*@w^*Sp*r{6 zLG-nw9mAQzYB!6YlsrI%8BHZe7bcrxl!|a=oG1&hS39r@rYJ`d#J^ z5cuc&0Yr;=?``xTUMnS`#l9@aaCl)`a#J`=jeKtXc~Y|0d-aYfg`xYnZ8${iZ|Ec< z=6Or>5FV|P=v`tvFJV_ZpJoeWiS@f3ZBPXZ0oYlo8Q|BZ@gSwxp3D%{T2rsaCoR5M zQz!L~x3;Bd$M;XX?qLYW&vtTkm^_xzqg27WrPR7Ap^oK;tnXeFD4o8PFR)ppH{eja zv1CK$WPNBLWBgQQt=hxgLxa8HgJl>U=SKp{c>;RyjJN7EO@#qR>O)>HuT`HX3UOSq zAz*5&242N#N{TR%C{)h#_#>CyX%yZ;aDFksJNsm0$KfO#biHwBYE;<;`*t+9w&r%k z`{z-WbeHrf9MqyC$#>9moA02x$6Co3JEcA!%U+SZtJ?3u0=9Ylm0DGTaYp-JkSA%4R9`hUbm-ZF79~4Gru~+2D z5F%)CoM@%GY#7{VEyu+#p~2Gld#)EWEbDE_kE+DXqq%Yj%+Tc>nIan(neJ9nzTIP% zVLJ58xJj_GSH0>;id^sTNUnmN+A}H%@yh3RlGf+FbIUWB?=f1F;}7z@IT#%RB!DCN zu9&3~KTnM?apBd&XudP+)=>2fe!Nn8wD{PmM*WQ1s;UMc4D^sG?Z!k7pgpmZ!8m2_ zUtuo?$k2i2Uo znVXgrB+uM?waS8qTkSF2)VNsdh=-(ZmBPJHyJT3T9k1Z2vO7Yx+TfR@tgTyq2i?_i zUl@#7rxe)&=l^r&tKXQien-mh9~+0`-@N(5G<(KG6;a=wIx7E7BRyN%Fa!SK!kca( z33b^EK%k8NCPNUEdy15<=earq@>oN&cMqWM#urDd-$BH`I&U2VF3P@xmflGtlckoh zzF7e0Gd?KdS5}apkNp(MPc8X5aDGk-0JYK|XrBDVG*eUuIvV`5X{NaVS`u(IBb_{n z^DCJ09~_Y5-@H2im1(9KF5IEAkB*FQOLi+xcPi*EmfTSVTM;I1>Aa%j#qe4na_TkR zAxHmMqW**bpJTYeXW7uBld3tSL4F11CZMX=9`~%5C~7^g=P$XluxUQMXV2!@%>#&|vt3sY6g1k8EtPV7L{cR-pv` z`ef_4Wd*e|_%p68^gu$E8!u0da1VmY)qF>` zl042AVO4A#gQ#Rmc)+#uK70p#Of*ntQ=SFOMw}Csbl# zJi#y>Dym_oh{Uz9$nN~GP!)}iuivftP5^cbUu^6_*+nUZrlKWJoJud_a(aRcyZ{n# z>M?$za<^hmyq?(15hhgmVaArq)L2iyv4e+ZvLQy`BXh7$?3y~BrLZW*G7f0hLF(Sd z_DTv5sm&})N&cK6SO25LsXp`a3z9UgzIpOkB3f3w5BGL>$>wN*q*>JjEU2k>QH?l37+<)nvh_~KsE|A@*9p7 z9q%g9h^TUw%^1MShKW5l7;h=OE@L1_c*b#8+z)Hw>A!-g)do50_=X}6xgXaik|UN@ zVwTtYHCQBU?X=dgsOGsCaqJephg-|#g9G2ZRW6pcX@v>+m}AAl{b}`Jo88EHgI(hn zI-wSUF$)K9KbloFx)}(cp;Y$;--W?MrJBIpjLm3He1-cmrMZ-OY{rB6jJO%*Hz4I} zHL8lI0S7*B?k(tT))nVFvc!_rb5ku7-5IEoen~VbS2of;LrJZt0@OnIhGTeVsp3j4XcWW z1nF+Fyc(0BZ%(G&(&L`HiN+_{&npAM@Y#ca%Hiozi)06|ljB>Pb!piax#5mg#xn*v zRf&g2*RAik^Km%1x!M&ZsQ2k5-6RAS|1_t>Ib*=^=*JIVjj;bZw%9}omi{X z629)BZ65X6(K1SMRJkwF4&8t4JBUz<5^`01VvLOxb!;xXl)p-xejKas6Q{SUNx}5) zL!5~K&K-Y=&mEcq{xXqegV9EHNJIEGmyR(IO~KQdEjQYr;q2(hAY|k-Pku|Gbl}uO z%9+#1TBh4u)84ymRRlyeyVS*nE{nk3AlG-u=5&vnmG$w#V>@*xHnu-BtAXCT$NG|t zEvkT0h43cWbO~-8vS(7BEK)4A1lG~Vu^_**H@Y6BAT4}eX1|0Lry!*fAI0Axja4aW ztUoE$3cp@N|_Ztnz)RdMtoO+_$_?@ld0Phzsr!c->3@s-kt7+C9(aosw=VrJjM{mg;hj;zHES(UVrVmw( zF*f850k*B!?pkF_9E~Yh?-p~A2_YBky^ufX(OnF=`n9L%;xa9R9#8){0te>R1*rN# z-YdaT$et8Ww!O34v!UW{*CaV zNUgrx!B}7mU32_Ch)E1Y0ByyH6uAlmrx7C-8*_&9Ag$h9dEK61&pbK7gvyRXIZl}E z=pg4175!@!%r7Doc-Y0&%gvn}C9kJn?tYp8(2odpH^G*pwl6Ijrec(t$Lf0>Ow;sI zyxBjhV7A^wL-Qxq|F--g9}YGq>0LD@ly}O9B&k-7T`q`q@4S$^o)f1+O<|tA+}HU< z{9DslGrjtaVUy*hYv5$*FhOZ5NQqAMq@caR&}<7X2fvv^;%PKTCU3=fdC#LfPGiuE z0!(MLErEYw=KFKtOOk#fJ%l~#~{e&_27xq=Z-7^(WnHl7q zN7VO6mdbxKj4zZn%(47%fkC&Jg!&Fr25^nkuLH0{pXpJ@&EL+|Q0&p4WUn@%`^V{* zS|G!$e4keY`zqf-&>SG)=i|@Eev0I$mi!zzKPQDh-aOf#k7~vGmJ8Ud$DjzU>mW1R z^liXt>aO;+@_xzIU#|ZfX7net!~e4tI^P(4ZkbwSlT11h5WiPObuS z^*`$x%`*v8y@%nK@qX32CHyYIr^9;i7_;fxL36OToJ$_Epl6dRF+68WUnp&?I5l(0 zAl3{2>n0Jrp~llOm2kHv_Mu>h!K)CFZi8Hi=9rfn2%P*K^eRvv8gi-Xo|!T3FnT@b zlpot6u5EQ7Z3x}fY{1+;GDf-WasureKIzeZ7ryI)wL2|8IHv~nuKX4x#f-raGsN5$R^B2{-<)>0n4KJ}G7ATZdjeJ4&NJ z2(AV6-;J8XbBAln=awTuKY7SLYuvQjD-nqvvh7{%13h}TH_zlGOu~3qLZ7M=RCvpk zY^ITq?g=iK%8{m44ek`^M(z;45S91@D?m7g%~mDAqeEt!B3Dv++cCZ)%hHg`eo#NRlK5Q;%U}8IgugXYfEr5uZ-@@TSuQ(ci>Hu{*l>0 zkE*LNXM@A~yaFZDTpKn-myqIplvLvNVU)cQ5rFr~{n?#- zG=ECad2krAySs_Hu?DeN!nsI0!R`G-@HA$Ta>CwzJ#MINjM?u|@Cqo5L|NPY-LsP9 z2oM%MzBnTs*OAZ~h3`Y~6dW#n+g3m4o3VMyj!ND_eV^0K?sfG}mG zDr;+B4J{hfClR$@%4O{9ob@GvLid_{bYGR z(B5Pv9czxo|3%WvZ^3TjhE5KQF=yohFC-g0OlAFgmWyN$o?7Tcguxr+mQ&`wgYY2i zeRB_L#|79V#HLCOpl!lXFUhnrx+{h$Y}QJl@%rbTPV{U_mN!YuTIjMWbcVl!@V)V- z022TZng{rs^;D!j#n~Jf7|km@amMGHWQFLsWbMD<>oK!jb59fpZ~T|@;YY07-*e3& zt}}T&i2ksC>ZI^Z`6mF!J28Bz`5TGIt}f>--qkE3gU! z{RNi;V~w@K3x4)4b#wn=4H;_GEu{Ynlv;9L3~xL<1&vzG(F5#c@gnpd3+ufFMUoUI z9CxXOJPh<-NH|+BDczh_>B6bfql;YY7sM4q9#oE#soo#+qAce29p&81q7GCRk}1Nx zs#g)e&WbVJ@Xu$G{WwbX-17BY(IWriuK6FHo1ai5Kg#PLy#60n2*I{3AQgw`I|#_d z5q;)8JdQId{O*@r9PH-(Z72f3M^@@9%GEWe1#;*Md@X?pcZoZ|ZWIP!vTYm*(w(i4 zUtKw(gk=b4t?Q)HdD2Zxu2%%wFOVoj2F$Ueb=pkw@c)t|Lj+v{hRLd{XL9uAK{yHp z(5_Jf5UGUNtp!GZx6)+6z$NR_Pu10Lt_gSFVtjr=38EVRd(V(t|MR7U&#aTV7Fz4- z$9KQJU*(ff8ggp+5`oZ9P#FM{Is%QqoirmCXn*^@*LF#NeC(ZE zM?){&z6Q11d5yC@LZ#g7hw+pfo|cKm?=- zh=SD6luist2MI(_dIt##(wlVY5UTXvrG%Q$OCSM((8A4E&iVY#+;e8`+?jjNz4QAk zne5EYe&3zF%Cpvbp2?qt?X&@hK=XgS{U=%4{~CIUbkkPDX8_PZG~gS>F57WQ^B-=^ zh3J<5rLbM_BxltDxZf&#dP4L}(SQ1E_rEdO{B`iJO<4Nh@K4B$TO~+FGGDLYhcG4} zFoHup>-^91O<|=KL?Pe23xIMX;JKXQev|DPW#Fu-_Cplrg#m3W@S-mwhWno1v91HJ0u zR)5jO6{NpO39wYBRcDcT!ef1fh_bbNFtx1B*A4*NnX9I-yU2$7?l$h9zm^o=wC=X{B6KC0T7ed%HQSs?NPT}AMNw2x0 z=iF9MpdSwTe>wGr{tj;j&Wd@ z6)Szq)fq(GL-wXG7l5E#oGg%D_2VaFo5IY^c<=;V*Km6CHD~PzB(LtWsUljcXvZk9 zo~I17PmZf=5s;4tUMA2=2FY?St_^eh^0p91T#-ClV*Q5(5|jv1buh@4WzHCCzD z9j}f3h%WG|0Q5!E!)oklNgnOd(XF%^voC_n<3z&#wySl?J)fResF_+SUyV9*)Z-*s zDFz-K+;gVgkB4u-vUdr(z+mVwO%lj{YFqd>BxUo5;Z?8;vzFZxSVa<5r4wCiLq- z_u|jF9tk!WP!w^&y*_rZVD1|EGs;GXRF#IvOJQqTF91v%W|zm$6j0i9cJI3an`W}{ z2-E?w3hl`~ca8M?K1~8`U;zZ4U%JMVV$wT1zuqcUI2fR8d?hU^;ciB)17Zra zWZ9H=n68HKutA&_8Cs*MCBs1aBW4Oq(vZAgbizWJ>W%`3-VD^)E(L8e-u9+s=L1SzL)37<6T*IH*R2-rs&P1Y#09+1%oHuE_pmQwHi*- z$%~CqA0>m`&gs#k^AL^G*SW#EpTZm zJGcnSj{@siyQ|_^hYp>4oM$%F%jo zOni_jI8HqL`Ricr_Tse~7^=Q-OY6zP`im_48ZB$DKIcND5|6V9cc-QOOQh`X%OGh^ zo2H}^U>C86qqKeZ#vGo%vl`-?wIEBQL8C2Rky!mzh|3y2x0G;0gZamUw!B;-F(nIpJ&8I>Q3>PJ^Vwo`P)Gi^{T!T2=3_HZMlRe>HZuUgfpokst{^ zDpop#Vh1A^fec>eDCk3b=F*5_<%FG#!sJHCivLHIR!djcwP!Tpb-hIOI_5NwD&}{`Me}VmuYD zt1Imm)H>fw11%SohMF@bR~Nbflo=xlW4cv_Yq!o%6C{p?Z?z;Rd+Jf>dEEM0_zOe0 zawbrX8*ir07?}d=%vNpLY_1f`PvO2KEgLaueNTx=5Dlq0U5?lBVp;TTqY7s3|62n| z%wf#{2~uTmZgx(vswj%bOHc=GL^DtCPGR|pD~^ue+u)M#gHmjC_@}JD85g-|ZVw^o2Y0SpCY@Zm0N6W)8UUG{Wbb6) zUmpll*bz9Zv6zjT5qIw$Pj%XRKD#v?EQW2gG@6enKGlV4CV#Ld-7s}s9BR|1nZ`G@ znT+V~P_Z>OAbKmuqAPpAwC@xbi$1iEWQ$-#XgR+-H4fNnVOdb7U6GS$)|02v#mCi0 zLY;@(Gkvah0a8MDtM>hdcyO5?Dr6y?Esx7DrBmIPzZY6q>qXtJnHp7KlBGNWAmmcy#M91)s8{{gGX2qQMqLvB4mJE77bx ziA~aUw3y4qlBo5YbYjkpxjE9!##;#&wuER_dAs2w4S8s58%#exc#9%hH*a9(?XBwma>!=#h{KmS?TR?x#qamitPe$((j= zUX{hxR27%Sc}V<#1CHM@PI5Z8w2+cH<90jg)2Mmy8s{lCP{k&Jpw@>pHcF_PA2RV~ zv8#@vA){)KQgSWibyKRA=dYfZHF1JvRPvXIRe3LYds$8>uyAXl9AtM=RXPoVHph65rXQKt z*1d?%kP2lGF8>e=tb`|66K$8|`^+kn_F0&# zX}Odh?|v$K#)a1<)>vCr7a=qnf|^EbVkR2C-X7>~gqq;bx+D{oNapRAtu73glI6$# zAp+tZ9OWj=0j_~(wIxm?s$m6JVr>L|`C2@HlSUMQslM^zZ z5N-uOZTF@>#0gv6l#Kd7@VyG~O!6M5{0u}E(#&Dl?E=G9>+5t{SlMg2qrM#!Cg^nb zEl~*GqeIs5q~-I*Y3n%50igV+23}zhUUYDqsr;ciy?Tar4nB4BaH!RwMk3Kw z{L2r!Y-fn<)d|ks5B*POy_P4xtfy5*cfzR7Wbob@>xT>27_Z=V8KE5X9e2+>2j=Zj z`*05it(sUj>o-DHL9cED)H>v7xr(y&Sx^TPBoV#5FGO4VnARrrcB zY{fUj1#ki^Ncu7Il4q8)TTZ z$Rj{%aN}O2ziWkE+r#cjnB79Chy2}1sA}GZu@Y1s+Z_J3f!mvc7(DimnaWiPyOZ;K zKZxydOMYV|uwFDXZYe*97M|`}Y!0iEdrTy&6^j)5YJc^UbDLa8RRi3k7(wn6!Wqj| z8^+zs(~a`Ek@oUDm2k_U31}W#a{@>-1AP(JIL}>yr0z|7D7oHE>J?1&8V8hDDtBHY z<-B!iA3+#4XWglMT>NJ2Id&U1BQH{ibqbzPuIJRcdUm+FgD?6*9g=Rrt;rBWAvJ%? zg`Evwg{yCUuif6LZz@A+VED1CO|v<5V*b>yY_up|t;z-w);SkA({81AkLPWt^_%_~ zb*FN1U;qdCLG;ymsC@H?=eohkCuC7fc=)GSr@T6FS8bGgjMnYrZBtFhde^{KoXK3? zb`RD-d)OMgnw&9Y4Uxz>FCq5&m7@O{EI5uXng~SG1jZD7^wW@Ewc#NN!YAT5j3zGt zmgaO&jgcxJuPP&`xUV}MA<$W-HIce=slOVo9Om|*ZzRE#kxn*Za=?!XKaJ)HI!dFF zRYFb4YP+W6CEJo*P3+Bt>QfJKRddhXLDG;m4R^QdXcoyNg$HV_d=Gw|+$8F#oL%B= z^KPQP09bK(Jwp#_kO}ChzYuR|IS#WjpjkPh8EW?CiJO)#Funl%EDcKK8{e&p^m(dL zU#zVPxx8LkaV1Slc=ftfqzO0V?PnL+_8w##FDHi}kqDb{X0dwEe3=ye2Kf6 zm|<6Qgj6Zb+>kU!IydC%*NoE%saN0XFCjKwNbg|{`Y$6mLV;}Z525m74g4tZoUkU0 z$t&mr@Y28J?OSmlPpSSnhj730)N8szPJtv=sUi><<&+%xR%xZAo;C zKgPx7wjg;d1B#m(Aj+}lGLVpuCLsFFXrYD&q;)1Bh5=JC9? z41eWklcMBoXl){o$Q2CF3(}ws4t9bhz8%lqYQ~10NK3~z8&yq|9*CgHO%;2mjMtAu zKzFJF^uiW>fAq)QiMdU4yUrxSAlI+^{X`ZTINjWWTrHMOl=Q5Q)8L8gwRJJ}mg!a( z^ly<5s5yX5*DQ9{WvrXAH)Bu9fs~E?@OvzMwY(~g(uf6T+CH(&#r=l~Q)=Es&2-_P zsT4$6s09jpBDs=#$e0^MkiuhPb7za0Uf`8jM8AsR0WTXVWBSchBJ~Uvj|@lqE?yIe z;y$%boCf|nj~`65jZID4Y`OsWPHka2-p_g4p7Be;Pyz(xsoTPI+LnA*WA$-Uiu20R z%vRU40~rASUigxE2!YoKs)nB|#d|Duh$t+co--}}dV5f##qzX@&{vVVu{KKCb@B`_ z`fP)(VyEXgpS9*@6xQh&M#~YQ6znWBRBX#N{K5C8 zmh{@5Ds(>A7Q3{@oQ9!97do-cK}E|de$-AnHK=D9Z%|g7Kky~dtpZOwF+ENGiVB;+ zx!lNb(~Mw;dMe{jafLsDRPK>V?TPeecP7Q1LS*uS#A<@I#)U?QK_+uwzZOH7L7|yy znC_Y&St-Vwn4s4OR_53AcbB%HE|Vn~+jz}USXm|5P*76`)W%JAFAnVydp1p;XxN(V z5LjoYO}evNR@j6HTtX4pJy};$S`^ib`md8Klb)IE`)K7NSq z*qft67d+56!@h|vUL}&~iWyZ)^6ZTUfiZ8PYX+iz@LH8TJ940ku&ReceI4Cz@tflfac%_ac zzpvJ4J1FQ{PKIG#aD(36P6W~o(H?ojM8VbJNKz6$dL_11WxMm~-thf*{AFW6J75%I zuJn_vq&=#S?NK{{3wJXoGdaav^OsnkbaBz|wdSak`O)<*x9^GxA{7!wOiB^K?8mmJ zl2Dck?0gN%GNZ>6jFzbORrD#`>Py}Vu7Di&PgZD7-br_S!_T=lbU$ovSpNFiBW!=t z-Ol)I#4P^(;o8f}+N7OTG_2YTRF%CyATU}#xJ&-|-Wxn)=Z$;-hu}JlQC7CwM3n{3bWW9rVX{K@NQlQssa%UEvgzKvd%8bz`vu<*{Joa{A9<+mI3>P3`Bcyn z+?dyUTg33lL*SK2%hm_!jV|N;A(C}7`Qg%%W)yc*b#rBd&1X!U5|2yjI`cz`)Ib-& zV$|Zv-GhXtEL*|IVL{rysvl~=7pk05>62F%n*ox6yFyN-Rb=i=Mqxi!A5pWKEq-&7 zBRfMR0e*+6oaSw4D6EA6&H~Sq{rK=X!t#vsLK#ae;a{P0-?!%ZV7E4Jn{AvV1Qyrx z#cM^adYk4l2-3?YXx_Q|ld;qSTZ#$CJtEvPzjgKmm-eyGKGw_j6s85ID!k=p>R0>cM-!{3@FCiIn2UJ;_5 zCf&xb+@!2NBdRZPKe*nC-QAwSf?5xma}#p#@8sZR*7uUd8-M-!wXFX7{_Nq8yZz;Y z4Oi>-P}l1K{=|4b*{cwp*98lfr=@Qa9&3>HMW=ubMt(!seV81%#u_kYy42uWYnv3B z*#MW))1Awoc>LRkP8H9bG-jNF=}&BfvvK|M57`+IbzJNA+7t+l&Dgo80ZO8x=`XMq z%@1Bl%`*A!^c~vc`7Kd`20zJBhqa>z)ivM>S08UzcW$F^Gt{19LoPuSerr}A`v|PA z#sn5TB{r|s1F2OVocSdxv_PX6)OYt+|5$a7EEVK#|Ck29`r}GbQNQDFmm)4zy}N|l zZHI-YG0Q%;NV=0f<)y2R4^7A0*i^@DF1(6coqSo%pi5B}F*9&)eS$`kF=}tSL9zx< zz66UFZ;pH)-4@L}ZRq7e1=NXZZC#xp%oA4UiJVrho)C}C34?qw^M*L7Ffqc9*|q$1sxL6RsRm!`C5&zd z0|PtvxGZ_@dfDv@!1bUKfdk5GIAdG*A-YAKVdmu$-wTl#Km^58)+E1s=Y@_NYqN%D-y!6K3|So6*pbG zTK+`r5>ij@$wK)3=JE-NdL%UC5nZ(Bn+_$`o3$jV-`t70)=BTe%L;hsI(?Wma*BuJ zJ9mug3p*>%;^uvJMU>9d@9YaEe16Ir<3)GU?~Sfap+XG>G)ONp$}`x@ z7^|YdKKI7povPk3NSV)eUvq3cuBwx|(kN^_;UMS5e8dS+-Nd&M?!GrGEM7m9xt5jN z4VTINYU#{gksm3C=6+$gnsB?w^X2y_%{99RJ*K6d%}8!^+D}snp7FFcWOYTh-ENV? zgogxkM}J)Bz1Ua0F>$Jvr0_2c20G;p{=x7tQMo+L^F(>l4f>;r3B;9Ap&Yt|#y0JI z^I0|QMh(BGJ^OJ9TWi~c{3q90o@m$YO-!~sP4A<4J0ev~4Z4!J7iMRTV7AXK8|y1U z;ZOSeXg45Y8hk1qc4sXej5~@fNkjoveVj*)a7XxIuug-91U?Ui*S{n1g}xkbea}wu zKDVXx{EK&>Z?^~DO#6*p0BHSqp|ZHZgUMjqAK1e>6cx3DjQe?ReBpG8s0P>r{3`85 zopM#JeU?Dq-3mUV+jfI`(X+velWmp-DR0*=02{>x$SoV~N^PQaFblte<&_!eK=a*N zvI-Wed27J|*Y{3DyKAW)Z|Ea=xX zH#f4+bZ}ikzlWF|*vYZ`V8q1NT)eQ)1t52!F6?CToNHx!RLAO6#t|koZRrgQDO+}E zJ5@L{!M!rOJR;B9gCY(i;a>T1OG=h^eb33J%oJlc1RKdFf3bco)Kk2CH_J7a`|2?v zycti8E!C+Qe_P^aLtY^CtwPq*^QGp^n0kTK`#O4YOn1x?IMF%+m%~D^-;FadZ1#!Z zd~i0SquZtL=W(`mwMnMP3hwzHfw=Ag*?2JrWA*SR)hkxZB&|(XpuYG7-0gP1+cmhR z_#DFxKRImsJlB|kw)eb`Q~3adHfB~f?`?8NFXFS_OhB;bJkZMEo0lz$$ahIPfSQTE zX>V>7-}>GJA@K!zynHJ6Y2bDPupzc)B|T!-cTfV`#w#EFcO-@OfW%KAm(XX3+_m5P z)UPi$7fwggK?mai>fP%zIiB<=fe)(fJqYa)?+m!kXBnCD_eoJ7BmuV}ttQDb>{tgiw6$ z7CZG=gVy&Rzna^9w26|sGa62gW$KI;nntQn2w;aYhBTefYrNa;!~|t!Zf>8`r_?!I z)0u#vQ-ARUy`*f)aCQo|i7kKC`PAA26TvBMVR(`l(DXNq$v&4B=9wOajwv8cVBC!bNHN5KEb;T@);6L?E_f!_zNKm2y+kA7 zXDh`BI4O~Z1dOY^=UTXh`^OE(Pru^*FbjAzWmfVNX`}I zngK|%qGQX+<3(UQ6~mb_RufqoRjFR7W!6bp?P-TIAgm0W+ZDn#$$b@QP~30{-Jrn< ziu$NAuLyheppiQx z?kvn05d9O*&O7?sZ4CNSj6oNKvP@#|uzCLTs9;aV_FGg#L@J_=NNr;+E1aDZKX~LR z9F%1HrekK;D|i4&h#=cZ^W6!+ArESXwxNz2(67Tmo-*3Mg7u7 zTEl?dwdyR(_Kd+H37l32bL{e1sz7;Oyw3hld5H?}Is0+C_iOdl>NkgDciT0~XBu4c zjV$%A8oK@NF5zz@YSOO#B(8d$aA)}U0+m2Df&T+?v(($!)z{Vgl49zs8cChL$DI0i z4Ez{)BG~Ony5)rl!+tVY?T`k0)WUUp1mt}1L+UMh4G(KQxD)lJuF+XJ)3oCThtVGuL2nd87tZIs8-1Sq&Qpt#KD<3`wU-I8#^} z3Oz)ccafdhdRrH-&vW2BLSN@0RS)~rIYFUr)i3S^pjcJ%rQW1Rj_&e`NWvfc0FjT- zujU_AOy-SN`V$KqFK|WkWMOnf5U*MvEuCEj>UuN(yfgLD7wC^ICd!6=a|MSB0Er*N z79uaiJyKWI+fnw_gs8)7$DORORV{IJ98x9pu3>w~aMyl8koAiqBNrn1cf~?0xl;%! z+D!&v3Z0X~!zyQ!ts7%yYY?DE!qb6m_c?pH=3Sm|6KzbJJdNKl2mP5|DgK5N%Z9NS zI|n{gsLI}|n!}C(K4#DE8hTIku}Y8ZJCPgdv{HSnG0hNzWIJL6-^6Zr9~JUDg}2+B zRKr{Tv+VKA_NaML=(`26V84TePM+CoekK-{qjrGIFuqH1zGEf_3%aHhX>OBpp5w!$ z>gFVyeiYT%T?|VJXlO~8CO0$#|4jIHxX%#nX4w?6?mkOO+!g4iVD;0CF-y)TeoDjV zIFsl3*tMU=6ZPNm@%p2qI~M@`(dWS)qmRqAaudBNDCu{FuVzD2>O-V>8EG%!(t?^D zU|}bN$xB-DZEf%-Ufj*_Ec>L1B!owD!Bm?w<-s_7|4Ps~4H{$dN;LZiALzr4bc4lr z3%8mb1eKl+`>{jyCU!C>5})AcYrrsYhO)1f?Zfp%VO9gUL2rdILT!C7J6AapknGJS zUq5;QuwL*#QO?b?2B&Lqf$8CglEU&?FlT-cQezF2*(cuC!x4uU5fv@BxXKA2RmeQM zf`6GFDK%{ur0&+YmGVrw#IeLN7rwXmt0*rCZM5Q9Yg^;Wv>*PVwuR9IK$MwY+Ko8R%J;&p5pW^4z6OV#Gbd$f{ zjmzI#SOJe;Tpm%T{SOwcOM~G5Y#q`ctS2 zKLDIfVTW*{_Ac-xYynIVN2M_|FDH1l5q>f(oZQ*M2{jvH*IE}20s@T}3=g4^{4V=u zn-_qA+xw2vIPz2aVi~ixw6BslV00{ zqAviwu=8Oc`Y~*rgFuYRksSZikI}-~YhdFrg6;)i$De4mW(esQfIS#tm=H>=2vqJK zpM(=m&2d?$8qbMZ?%$htt1bY{1P{I}!rgj87V)cT7XUXZeAL1L=>;JE0&p1oTl+kR zsEIyqenebpBnQ&i^X#11N#vGhu%$ z$L^11Hm8K~5ec&$iC?_{oDqBH=ST?LHRlRMHTny{dB*u|VpaP;_fPeYeHB03eyRG$ zR&wCjnm4H~0K69f4Apwtzby&)ueFx0y$J)NLCu>$A|3k$;OGgl1r9^Bv9M#}2X?Th z#4Y{TnnwOyEbK8Yaax6Dj#vP>02~wd9zpuo&HL--{nKRpm&g3CC+{D|WS|Nzp{NwV z1J`Ld-%W{TZ!p;ZWnPg+@s@q5)tpRNHP~_-KBnF?o@Avurh(*&jI9~qO*^`|fViej zcRUm_Pk#X@{C!{lq^oib0!oBW7!d3^TwCVvz#w}08dEMV*#!Otb8OZ^u@FXZ1);vG zHr0ER>4VhX*hI;nhQs_busWsa2*y~>reQ__D<=*`_}D^>gU~>h(w07aq+LL=(V$Hngts z-`km*)7wP5n+n7erDh^IrjpXma70D6#4~W#`AZbl$$jhso=uow;%$y&+PVM?lMp*t zh^CSgrfB}(S|GXRf1SpT&^~?vSc@jeHZQ53WVLeq)3F0m5Bq7mo5O&!uXLtwj%mK^%HPCuMM{7)_nX9)^1x$Y`Y=a*2}On z|8ufG9NNDd5`YQ_PMb40Y-N@ zS)=}C!FHo&0e(h@dFb+F(ck?a&mWHAzxuzXe{)JAf0^ySeC}TX(_aD8AEEVMKKCC! z_pfO8uTl>Gxnd`Orc1O6wBnDXdfFycb7`L>oefmrZa>LZBn z&nlp*&1+D)dP|);Z<5lFU53Hg$4rBDF>Z9e|_vi}n z{^=nrainoc_{fcz&^w!_i{pVz*o8wCYXhOfZ)fm{iAHoz)q;EJiQjb}K1Yuau16=TVfN=9~rAf3y$>ELbOj12kbf?xh zAA-6;1aO<$_X3c~-h4aDTAgl=q#)*5+!IcZ?zJRB*@?^rpzcj`do%ZOeHX`30}yY% z>XX>s{K=TNZK!5L$14DB>Z9fL3OTzm1g1pyz8kWZ{6cW(R2y8@YP;(ip)W^7QeffC z)~hM6ebURd3;H4H#7a19p~%lZOtGB$_V(54p2|gbDe+B)W@cHIY%iUCW$Zv_MUm>= z)ilrCDKI}R9H}~$XY`dTc_2=ef}sx}U}?!VEj_UACrPA;oeTxmobiJT=3!ubtoVSk za2(wp7nzKh#Hp3<5Fuu*|IOD+ua3lzzCdT>^Hh4{zcpg4SzN^#XB_pedI}zwjJ-@K z8j320FYgz6>cVC0nckN2?+)~1Aa!q(AZF`Wb4sWHZa0Gd=b3a)a@8;SamY{#g--{3 zNcL=}-orcesO%f=8>@NrVpn^9WjN+N8}~rj*!&7ltdQUgZ0Z6qS^0oRP*n9}9r|Vi zyO;Q;^VuuBJ!+R6i8p8?D>UhVxwNUwZZ35iO!Yro33NO2vY;u$4lyb zu101wE>&wblQMIBf|aZFW6EnV8|yNYk}Q{%`pP(iN}4&WGCN z&dq~^5^GXtt`V-^T+NPgBuh@gXV$3XEs(d9ps;(l*pw55Ys_s5GCqB5qjG9oXsVf$ zb&cmuvJCx$%ahUT?+N~>=1~24u2G=_waW0v3o=Z@tx0hmsfyqZN#2R6UGVgXGc7tm9@?k) zg`Wbw_H_}z-tJy4{*>hL1^Mmm5xkhL{f)N3^;M$?D@O{ZBRa*0%k(CTU07YTReq+Z z{S6q3JJRNdc@>BhPXB4}?gQ)D*pQ!hXCIV2RFgHx8^D>Pijr1PAeJwX0~nR*26Iq` zecp`GfZRZP>(km(3f*IFJ3amO0`vGzK+!qXl6+t;eo}M7wx$*$4izR1E~x?ibQX9E z4=3-Tee_6iYxVmbs}?JshfG`zUN&|6x^a0NK^s!y1t8(~Cyj3psw40HKHyEXI{M_B zH9wHgTR(SNJ!0>kb5O5gc1<|I{C4idPJ=w+A*hh=b!|fG>YQM^0nc?q6R`mLeJZl5 zo;TN-IMG_Xi?$u6F7rD}8pCFA?VeD-=LUp7|7 z`R$LG4GJA;tL#WW*NppIsNn-jlB`H5C8~!x9O;0>3(TI$lbq?xESs@v-IDt~QEtWw zDzu6CNfB_LWsv6Boco*1&TL}~9kEJy!xJqsEDU{-q!zFF<5WBOfeENdXXLDIIg(y+ zuY={?s0x!5ug4|v1w4${=n?Fe77U-KNu9^eD`kQauiXsKnk8SQGQHvadTEMYf~r%2 z`%)bjAsLsZ@Da6>lGS~r|9&Dd$IMNXc72y`L} zY`$(HKeOfaVf$VY4Ga%vz;4D5G&ctsqG;yz8IL=1!l|m-ogzBSCKC=DD=QXy4x9ox zZnAp+<)vi4$E+Cc@ol-BNI=6n^-$Z08bql{BDkkT#kvW7XS-565y=>6Sg3B}gI z`RmUUA82r?yWjr0qghm>)Z1VM$P%!xZf0#_#y{Vj-))sRY25EY=ish(k8}C!S;Ubo z!ubvd?peK*(3_&{Dv^jdK6o@saBg^aXkYJBIry3dP0W3c{bL+5)(e!rzxUdC$l};nsHWck&-TSofemI|(^+K&D z-Kq>c#XZ|WQbrH^(%F`~gHxSG?`AL<;?>7Z5p^Su$};T4_9cD{5&vW>+t=1vPC)a@`v!%`p5LL1;TflpxEZHgHlolRsCTz#vNmc@JBNWZuN!% z?-I$lNLtpIj+Q6KmcR6m7Tpzi#g$~tlS4bVVBnDuHwNEmoZg?VW)RBb>!Oktn=ip7 zBqS=7nM@0CC2vV?~i^xyjNV!4|^AUr_c#$l(G~$ zr0&jiTI>}*8=K4wAMr_?I6FE@sD#&y&Iks;h!hmgBs&uIHz`c7xt8hogQd$ByxVw_nTWkQx7Klu~hsoU^S z91}ag>8J3L?eUuilatlAhV12PzS>@GoPf?!R?L{7r8798>qgp&jMPIJ@TGm##9sIG z1Z%c$rllnq$pN?cBON~j4->;Pdb6^!$on3~ zQ|)+Ai6dbG>G;*v+ubIni-Ni`Td}$*koA(^=O0Exl3BY1&^Oh-kU&G65@uta1FCS+>)grI{|hdv<#@ z`dac%Rdf@wr6`Y)`jRbG$gxjzK=bV)Cy3keZ>2S}dC=6)axWeuUPSc3C;sffXEK*t zHMbX6YNZ@wvF2!-I%l?$1Jsa@i64-3@78ebUm2e+UKp>2M#Y?K^EI zDQ}Q1fYthw(5dUVW2Y)NchVBL6#o&;;zW<%Z8y7LP9s=d{N2=4vBE;FOp9NJA7FTy z_QMmNn_YC>2D8?2r2=x&Zyy_%}ua)zud1vNdpO7McDg%H^iDsgT16R`e;i#awY@)BByi`OyIkjAs#q}Q z#>15<&W=uEjS>?CIxtMYL~hvSPS590o&DFONoDR^MhZ0gSzRyrM_Yk@Nn7880kEsc7Dm_(CC8Ta2! zyuocQ3d_6KZ9<+2(H)hn{Jd&F@{V(7wb9rdsoQ!2uR!_w=#1__cEsx0p0O-fM+pn= zIDLZ28pW^`_4d#FP=STVo1h89)yAcP4J>(HsS{`js4WQt&v18T&LS@aD_c`&h?afJ zzv21LWBJG6)uF3x`7Zf$g8_0FoCNGTN?lFlc(A-9Rm?;HOm7f9wrO+7 z*)sts5JgvUs*-W2;tAL29vKz$H?8Z#@H%fh_jFF5X9`q)r`NtYWAZh*-8d*m_ECX= zB(v|9yan_zwEbcCgI5o)-1gV65{@#vijxe1f-Ossn-B5e4oKbrMT`K~4$_AYZQ8-~ z^+tutqaa?AJl>mz&)Li0fs?gZ5(SU%9e6f=tD~(4sl;2XTQf^l`HDeiQ1SDp(k^+( zne6FBKY46wI7&-p0(cW2G}E5s{+rC1uANa;YtAx>t$##{uw}9Mo0JoJ1S1aUOmZBI zTmZh7miS3Qb;|Hx3pSM0H9lHrA&<%B(494sG$MUHTpfE9ixzw&`7MK_-Nd#M+_^K6 zdW+*YiO_gLJ=WG1QelT;Lv5qLW()My^^TJ>2Ah7byB=mS!_#+JHjcb&CI=FThUEhA zZ0PUuU^g&L99|-4o@m40kGzr~Mel~(Jw3S+K|UZuq>&ZhL&HEb25uhz7-?3m_q=UX zjP?X_oI=wCW~F{q7~@!Zicja%oT!Nv(_!p$ZuGnch#_O_9#l<9?nSiz@^Xq!n1Nt& zN^?%lHFh)R;40zmx`xpKn}jq$$UO4!F0RJ^cQL8ba03XHqN8P?JV_Xht;CXpzt~o~ z8*#(M0Y<$1hT*opERBd^AOXF93K|-F9UTP5w9BcF?6u4MsCG7PrV% zNLX)@mE7*~yN@j%3WPphG{1UY0H&ng6|$|b4;5TS)HPULIlj-6p)ZA2zEa1VN=k=$ z2z`%@kdaSholH7F+Y?yBRpS+7xtQfTT0xAO6geK;RDFgoYNPumAjkCj z_*-2b%-ys@eb3Zo+Dw9AE=lieLrw*kIneVl9;q(rnOUf4s)nKWv^Y3Ml<7`Huo?e@ zA_r-%*W(tXd?{)G)afViD`(`lsSwyl8SL}g-PzgQwYJE)bM$HLQx(sbWJR5_-=aYK3a)5G?b1dan>v$}G?UbT8}U9F5D{n;#e^S+7phcn zTet*M#X`gh;=DN#H3pF%m3d;X8F9swC`A@s`pMKO+xc_)7hy72kgU}n^^pBJQ_P6V!n(5< zHvAHq;{4ciV!xMD(w0?X9Yp59vnGwW4+o?}SN|lA`>*ePjsly=?UOKhLEGZO)NGI2 z{X~Mif7b&mykp)*Ae0Zk_7_;5Ds}cIbH_9GY=vQ&Pt03kbh0EisgyWiXt7f+j>BJY zrMfzRZwUw6hL1b~T>rTU?V|nF*&1PX<7m<APuo{PlFMEXqFZwme5^8%A5RGU!la%h$M7P?1}%!f2N3*-r9Z|F69d`LyRb z1FlPn*I;4jYsrhg;CAMqD$nmm#3{zvsV>jlg2wUNLjuq5nvS-OyQl)ssA^?a+cm~-Dl_TyU%jAp%rOnir{547tP6mS08v3KF-Q58W@ zaKP82x-e^M5mw<{c{sVV+ceQwJqfmN{4MJj##j17-sHcFEFSoP?RSOVR8}eRj3GM1 zWGX~jNH!gA)V%g_&NZKL(2uP4QKvB+%If_2neZkTI$N6Pk^wG(L_6Y~e$H_yj>$fE zGp8m}du$TOeB6E&50u8=w24t%MVgBs6)4)dccXy2H>FCim1dp2#tEY^bHS1B!eq#W z+#1}aMevs$9r2)_&=my2Fh`v!ci-k^&3*RUrQCG;fzUPbU!BiCrHURQYDCq~L{>+5 zit_s7PfLmx>f6PzWrcKsRYkitNPG@tw^W!P7MU_iT;hpt%MHHFe^|LcUk^AKppSoG zhd#Oq(-{%Tylk{|yfGzrsv{c;jWVK`JjlNI-=7iPL1yTcr&m8RGd2CS8@eqd&$1y; zrMALQEt7MwRY$(UxbwO?^76NxSqj0JR}|C%-d%~&dgsLqD%y$3^ZLdm28{>$9>@ic z=;5CT`N4vMsv5+>$Q)#T^K@ZX%9t;+QEBM^^y-LLDo313SR}-|W}439dD|`98IaaH zX|G)ZW|bg3CsB(5qptkkGo~KZ%g^Os?~Tv%;ia?flbk|wiF)K0fGS}qi?@??VVC%i zsz7#c3U(`+uO>arnzGdbA+V5l%3!Z#v~XLW7DL+v2jz}CSPWZloC`BG1ARwS9aHvrr zAO?uiq=Qtcp@b?WK}rHC_w(L+?mg<9|HM(5RaiOl?m!!5!_n@c`OpNcVWz7lO4-HOziJ#{OGm;Lq~ z4lg)21wohsxnW#7iS3qgr``u{b*HQ$vv&u~+!GgsQl$}O+qNq?OhW4`N~Kje4w9$o&=@DTuM$gL zoJ;PvPv4wD=|AsfN#+SL?@KAjC(xW==P(UKMea zlvq1nRvwZSxpkK^Jig$x0`tgD-xIH!`(<khX{?E>uUyKa_|VbIaq4l?EAa_KQoN+dk@qBMoNtj$7B>*`^furWsw^cBYs7jv zYn*2rk&bcmkaqe&ksw^w?sZX-x6ZzKe`Z*a7$MV^(ML^gzH5iw?bxWo z@&vG4t=vIWe=rYVLKMJ6X~`YlF(<9|D4`zYgtYh>e8XYfeViLCf`3a{N@XEZUWf$g~7;n9O^)ivgWO5oLy@1FHpr#fMR znz?q`%|$Xf3aN6j4y)L)M`K3Y_VzhiaS;nb17o#($1eL{7`?$}V70w|;Xu9CCMKgp zSNPqZ>~IPSsquIbn*I_Vje1sfu@;_J^`d{R;yv<(g@XQg@P{vc{!J@KS|T;`JVpOUnsrDc!|WS$NLIjcilh<%KUJ4^>Lt>nt=sxf+gHbU=EyaZyA*crCJAtv*A;dgtm9QOWlnC`5^>$LjPy-A9% zIw!9iNo-iw8)?#g&J?C_5@7$7aOt^1w z!5l)E$tWD}TwqY6qo}%MS$tBHA=(+6yLGpQySQgo$!?uW+^Hd4H!ie!j+pyQ5|daI zRPLEIg6;ckP86-tad`WtCxbt7%UzFS3$EowO0qSDzTU)Ie!QeT{CETJsb;?`?=wl` zqxR^h8lwoFnS_4H1c&^}d|Upsl9l-Wn;nXC;e%Sc+3&aKm0kCYo3Shj-_ZUjurKq~ z7{q=?ErnVkiLN~AuwkS-az0qBg1UeWfI?`#M%G~!9#n*N&ig!_z0 zH#ycr(-_@R)X93UVK7^%RZQLzSwLyDw23VhFLXf^XW7BW0te1Ea9?CTc}-hH7t zo>=te>MbW{hZ(J{BN>ZsGXZyIl z>BQT&qC68fh5}9#RYSvN!w;U(eP}vf(y=@svMYq$uBiCowypi9F2{niTQAg|QZb1O zyC*#Md~UMq0P@zJm^TxCxj26bOjhTI>ef5eO{qs)H08Qy(Qk%_(6}Nc1XqASk2CEj z90g7Y26GQ(le#)W`YeZQrf?tQsdMgxhU88RTl(ZRgJI+`#L@ z{*LFq#mN384{dv!+0KlxD9qg006I(hLk64#f2?E`v#XtetK}8ZGD>S+jmnEh5w#$APYZ zYZZ0Zp-$JBm)l$c_~GEC;xo)Sc9lHXsZ!~W+9jvzo~HUTL%cYsUA&Ecu) zSVse$T{HWIQu#3hE_M{uc@sk8X25}azv+Y|0dOA9R^dSr`Ev;YX}2c>6F8$OsKZSkDgR0 z0z1@6ho#g82K5r42;7q47i1TRKRmyVSp{TS>~n(pMt>+Rc|m+)|97X9aLg4PR|DKv;5nRZbXdI_JmW@pJ;{W2 znn)EdK*hyRb$?e%Vi~wd3fC#@DHf*+7)G@gRfdctN}B|DL>818Zmk!x+j)y4;4|nl z?<1JdLfRT6$jeBq7f8zdm35Z2UK!_?#6r0Byl?I=P8;F=piJ| z!pE!&5#$vUI*ch4$Ppki7a_=Gy+y$E9}2u^fXIb^EmWN%Ax1a}d~JjRu;k54Nnun_QuVFK(PTBC>Gm>U8Xt(_7e#vPoItyE z2^7qLda8y{%IRahw7`oS6Tlh*kGf1|+rE?em1eu3-)aFqw#^>^5G7P!NziHB_(fm)T4RN?v6rR9wg>vAGb)<5Ky`5ar|bT+XWR+ zKP%vI?WIAa7?vAR%^#yft*oDYN0LdpnEWW`V!xK}jc`Q6WT2s$Rg?A6q@6F~20q(G zIH=xV&esDYo3Ld9c&HAm`-AX^x;^$#n6RN#Tkh=?*O+e{Gx^BYUm}61v?D55aY$jQ zs^Vk?@=Kt8j3Y$jAP;!`mp;?~9tJ-7F^;LusYn2v69fPNK218%*zPIZG8gARBVZ&Q zw(a2Bnjj^L1lBOW9`?@s2o(G|w?osQDuA#ROb}=XxTKc`wVze{64fW=DMCKD!Kfwp zxcYyN5LrJ!y035Y4DGNQC>}k%^ItIa+o!i~64ViI{>bw2)qCyOBr3qQ9{^w_HdtZc zwg+2me(XfY^P{;Mbtk$doawNe)`)j=vpHzZE?n!M1_Ole*N`le%s_*`VH9Aj3K*+H z#_EufB`~rCMz+Yv78$GogEe5VM-29e!6N=&v4}N}P~rQxPRUy{ldJhsO6{m61TPz9 zJ#zs~PR`myz@-#?q4M;42QZA+GSpiCU z{SPT+nGcAf?_^3xy_H($!7ll&0ZJKU`F}tu6H~*ekG~f=`Y4bIY6ts1vAtMI;2$!| z5iK3R)GMO;Arg`^NR)etx-nLxk2gg$jZGbsIgN>!eWwea;VRVqg)&|!CO-tB{&CqC!TEjG3A&a^lUtc3?& zTHD`F@KMZw9L$rjPrlIR0<>Mqiwo*x0b-tiZUQEyKnNx?t}yeqHn1K7yV9EgSpv3c zL{Sil`i(9vRD^2>S)y%_;f>M_2H8hmmz_U>33o(ObP8h0JcxIc+4PBQ^JJ^SFB=4Z z%9>QpuLyDnsjukyh-H(x_mpq#$NrqKofh(|Vt=2WcSy(D!@t&36bUI1gjg1b@GD?{ zLEsPB`EgaAjT3;K|61C&3i<X{DpbWCoO;HYkh}^Q4Nv3`QkI*knVJfbCvnlw=fM6~n;a$5t4p8pn+$#a$HN%D#P}aEVhAIC{_U`1j~C{n zMBY#LTpB-N1a0~A^sLPV`n{oH*9|aGbkwv{bRH7Yc7h6nuG)&?F(r_&$5i-w86y-E_D!m(2~jdhj(#RfXbB@f&MZJUM%?ft`tc!XpvtjlOmOZ%vfdki9mLoi>*D&~A+6 zg{x=8Ru>=U#=}CB^9DOFxaP$C7ad!>oe!mc*SDs|jbVzT$-YUJM_upx;U%+~u9%Dt zN=YWgiIF3%W%a&}w3)ts7KEjnGHgy(?~^5Yr~mNwf4lwiLPi4Y=qW_y|4Gx<|zdc6QyNrv+c1dYd0O+*|_OEVamvLb>9;s53pI-mWPfh zj%)P?C-5zto)6fZoDkkQ4YWIg7S(Dkg7E+6Tm_1`>05ev;q-1PktJ^LNdp1$l<10O z=uNAWLZbimB;-JWJGFJUK@&U4$Ts+<&_M(YrgosC0*f()OnzH*U zjv~>jBK$=%4C3ti?{I;#{xA??6W0>jo zG%X3;R9MiD|JV9@sT3A)aBze*G~nRjSu*7P2u?Z`Nl@~8d-J^b<2D?mfB-?5l=ukx ziS`YL%F4?6wQ~R(tjGCp2@;gPqniS2yf0uwDRZ6*UjNCEx3%Fx`tpr`+_cLS!@L6F zIqc;0d{Etu*0AHs#f3*S9-;m`RLJF{lv?xSDAuePF9C;eYHMJeDsMlf^Rr!)I5Qqu z-5&x*-Ve9yKr{Z4kt5<_URH5-Ug>$&{*DPIGevJaxPNZ=pZ4)RsGp>fJ}D_F7$D=P zCj%6hm4%BcoLycHU$q-CV!jk9_xq~Hh;5ncO^HsW2oMk;3?CHv_6;j1C+GI=?rI2K zNm*ZC0f0uDAx{Q|fQAxuf5M`=x0$rjQyu2MACQ#3AG1@){b9-It=ZKc8*DW1O=}{`1F%$oGr|3|mojwIl)p zIa0*_$(n}#DNfumKYsLq5^|;M0|L0>?cnQ~icuAhtq90IXJx(dH%&}7TH{VKHJhja zHY-jJx}F9O!xI5WKn(~v$9JJrrElD+D=QWBxjJzZ)j^hUd-O_;Jst4}Cj^W4BT06( z4rV(?KL-ZJ?Es2DdXwDO9$3E5O-_aD0(hOlBtI_g0#|{OPV^VfNa&IqY4o zy5>@vbWTcAR(B+8g*qFS;A#+CZt$I+4K!^#?s0*b-*|>4(A_0Lls^|_iFA11T zz-tM%1bL|_{DoVkvMZ=F^}9&}4pn7gbMo=@c{cT`8KtVW?3ObYU}IHPQI4J$67`L|hj!hZox zAb9cbZbgC@#Ik$00h43*6;w191K4wB+7?4V3c1H&rtc!2o?PDE-rzU+^}7&pXAs=9 zzy*J4X$DUD+GtaJO>t}hR7!VaiJsdD20{PZ`h68h8UTDHpZ@twH%aSq?cn@J^DXzT zr>y=AIL7_u&hI0@&}hBcmD_Smt2wS_-FwTBC-({4KW>4JzqvXf^hndu{Ktc(=j2f( zQ%)SHj}M{gFfS-Y9^mC5$E27QSR#Ri5hp!GIP(e# z-+)W8$v7!wvU$Me9B9hd>+1rkTNZy5)8Md4$ugOXST}c)Ib1j{kCe3Tp|0Ie77p{( zgn)r4&Y0EAl=VmLOXQH394oIpztxZpCwLAooiUNUt?g~hnF*lBgmn~IytR=K&>ZJs z5bwdzE5TfAtT>flPm-qIG!S8CmLB&D%O{+iq@vjCntECRR4yGO{q===b~a3?(zu>` zjEMm-WwQM?Bx<08uZsgmrA&`>9W5)}R@JRm{P5`x0uFL`n0tyq3e8Z7j_@=7UX;LS z<4}b1k)RD<7>_9BaFU#g3YLW=o&3!v6hg+qerGHJ2K@ZyyzsJeGPyeIh6*s1bJ0NBLOs8xH#0>%GSCVpnx3rJUNCidX4_ZV3(B1UZKWo&^ZC2_y8j* z?Vyk3vrf>?JCY9H3oOW8Jvw+j$dA9m0-%_yRh;(5!4SeN+Te|&maSA@lgViqoqD8Y>S<2w?-x;AWH5Z>-`sl7L^r?mI)3_l)b6>}lU?4i z>N;jUZoD@Vn_Yf0Bb1l-OUynm552|ZNj>-bv$bOKu!-^0^#T9e?Lu~WR1{n_{Pf=H zH)5(k^JrYFSeW|D{|;#$+lsGIl{GNIgvw1YV$;%SyUYltuna+eUb$3y60)!nDdBzPU0Iz|avH}NIVeMYaBp;7;Cu!^uU4Sd__obqOEcS=uwZQ{ znNC^Q@D_PBvZ|23o!QzJ3Gs(}n+l*Dms+eLWkS$qIM8jDZZzLPX|>nGbq#eL%LsLG z{Z!N)l9$Yk4J^g;ff|(0(UA#LIAj+11&XupItIE*PcV#FAHJR71qNDp*sqQKuqQi^ zATmQbvE>t1^F=Eg3>t#L9Lo(HIX)gEvG?_;f9oSD`DNm=At?zM_{@ukcRWA25!$C6cW0VY_%&Zj6-WJ5A_Evl|2?q>mtnx!K7 z!ob?ept_@1&V2-|k|gg}zW?yS->sQLu0N4F8)m!+w_F#cHJ0hCO|)b6_&nOXo8Rua zT1=!BhRa_@w%r$(S&KFv0rsh{NkcMSSp+e_H|BzeFbk2nw$Lwh$* ziYj9^K?5=pIet7`7zn_S5J@%vhvmwYsJOV&Vqy>okf1Y58!V>qR8>_+$HxcESm$PE zg|)SD@9ysW($!6|l0b%H4RYjoDNct<1ulPRx>eiam>nCrY3La#2TM`z5%Y_En8y+j zkmpp@l!P3Cn5yt})DK#TxCN-k4rLWBSg7sf`xvTBw5!NjtQB@52y6)Ytf8Kr-tC?bd@QQTSN`9`zapW5MrQvyX01+UL zd>_OU+P$mVs1_5_?K`VBV#@U+7hz#lSqy}7VM0%&lmd#w!Zo{iw+PzdzCc4()?F56 zCNLMb*ulB3QaqU8+LTdR=6YP4W^PSpN!H|MVHTz0omEg1!yo)HmnnLc-oP!-N`C8= zaCk`h3?)V|mCToVVv1=d4DY+D3Oqq@iA2pq{Zn`v&}_}H#L+b-nQa4f*Gsz*f)-5rzQRp4;tJbOOgSg$>#~?TwnhaIMdM3fEiLg0^IbE2Ditvhsqd$H`W{${LS_guJ~>?5de7K2XKuhDwnr)RuM8}MgnG+j@R z>P&XB5fFq~XUDd3JM%6!?+_dtu$>n41Gb}7smP&_y9X57^kz;uRwwmc@V1`cA8<~; z;jk=T;%&XY>v!KiJ8Y`Cq!S{)5^wozLw53Hrr=sSVPLb(bcbF9_@YMs5n_2bopBv& z3z){XJecjeKppK3vZDhOy;XnT?(NvyT=vR>b$xpE3>XT~;MKYr@b}xhe}M_x>yCC5 z+IoIgP+x9iGZ`<;>e!=NR!bL1goRA0uK4x18?ZIPk(Od&E)FaZ!1vnzcuNraYrmUp z>u`8Dqc0MvW-iXUG8G;Fk$UlVO|xkty*K&s1ZkL)8vHrpVfi{G%b_q$ET&eK@%isX zyR{6Fg?>@Tv-#iGekScMQ{pjoY|HN_x_V7#c9b1cKf>y)36XAMg%3S7JGhD-(+&y$=bL&xX@S#Uxj~hd5NjX@-Dhf1@`NsC_^hG~@ehDAI0ScX> zWh0{)j-B|KxB1RZp)Xp>GkmY_VLMy4Oh$~g5MW0|ev)^AC=wWY$9L3d) z#5*e`pG0l=G}r)OMZI4>CoLT-_Hh}zD$aKz@B!}p^zduXU>=lMD}{=ijQ~MM%0TVD z5iZ^Xlw&wq8=IV06{Lr7&rXUhrAy2C)13O^g=~!VT0_O?_YX>&OuN!Zg^F=wg+ua% z#OKY)c99FlXeEnKBjz?%s`N$4x|KDlqYACxP3)Lv)r3!}QlD zL{sd098*q3ncirSY>w0951MZv^oN?l*gd%xbC*+Z)ASuelbq$D=4_ZPosB+x0n#|o z;k4OK2ez^UVg7<%$b3EUNb`IMd%u1C$&}l1^+~J99$=5#Z0{3mU0B>C#QjUz>Wh>KsqjC?eeZTozj5q{tvgyJ{RJ8c|33U7xV@caXdoVz z7@$r2L86_i-Dn1zl9B?(!D;S21a*yJn-(;<(Brc+v#Csu+xz?L71MZBn6K$Pu9R5O zp-$Xju?3S(v$nXcEe$!?{clH%0^9alTTt1>61V-Y*!{_L1)F~O?el$R8I^i%bpETG z)74|G5yEAL-EQ6YB*Zs?bRl=b2@vk%Y2NoK!)0~Xeg;pc``b`GzY{pl*YEUP0jJUQ zv~#Pj%Sv8u)FUm1Z=6Sry>Yn$wpw!&q!kqMUT+h60aCCNwLL(#(wPHb3H*u2zRSXL zuVlgEsHx{$%u=rT>*c`0FE{?0zAZtC>Jr~Bx+j{rA!Pl9b_*pg-Gj@;eYv#INhNDNN zUFEV0M`v_t*Q;_(*52==MBjJZZ>xIyk&mW#f|wNdhSwU~U;oh2u((~F@?Dg52#p+l zyg<`;+{$JW8&0^s=JW?k`6e?N_Qp%3`Gwsm;)Bu{d9t9Fhj5KucvTpfE*DM%gs(V0 zrc#<_wtv+@9={$c;l3_vK22LrgLvCxFE*d4%q&TlH=W2eKP&|W5H(dUUdga?SyBQC zxAqQw_rPR9ft|ybfnWQ|-WAXshzcYlh1Th~cqUdO#La(e)P3CXZc=$^y*@YO4-`1! zukA7>f)ez0Q?TSt08B&Y+kPd-gGE%XNuK77R^5yb>^?b40Fi2=V>kMdK_4jS_Z<*0 zq|jg+Okzm3Yx(pa5EvJGy@z7l5CO<}QsCPQT|Qq70}TcWZ{)nNrEswg^hP4ChD}jX z9+wk<8Y3a}o4|;tW2-yk3EDoR81yJQttl*d0y2CJdCH8<0+Q6IFC$%E(O5)65tf*?)%)nP*kj|P}LE1iD# ziW)W;vyI+0z28AX#tj`#oMs#!^ti)twsFa8?~#(erS!YqI@#>^4-&$=mRl`3=ObSy zixgJU=r(}KU~}73y*Y*vv-T)4S76(*OjwxD@eTeMuH*JTEc0>ncKqEug_KIt_vd?# zIW7m*r16`EidX%J)J}hKyC;l;GSg%dJi_7z_aC>rk;w-gG6TsF zn}dI##!vcMZd)WZ%Id|0)50k1#~onlsyr4-xGZ4}UXMUabj#K~89Mn?liTd3{z`;O0ax+vjEu z4X(C|Xt2u88rJ6IM27sgNXkumB}}Y~hnMa&otBU(E81p;0ryC`zKAN%BrSWvX9@_GN7}j$dbU92ZA>XkC_wnEvHSbDBOe zPcIVuRrkoIRJRdt)7o&H>F9Kb15b>Z67XZgfr^PKJ`^5%_s*xHvU1?W4O{{O8GAoi7GQny|S6@7|2<+h6Fh;G+sJC)sAUA6}k7#ZGhHYEHBw z_qF%H`QG5pX(nZL_M|!k_5OPS{SY2;rID4(zH41UHdi17>^<@OI~#9v-+bxP;; zlLiX#-=1rFnZxieOk{AmvzTiixtMFRH5QjDZ3#_as)@cp)VE zjWY6cVz^nOjjh&?KO>i)QNJ6fY;gj8=FX zNMjG<1C*o3-33$ig|rHSEBrVTC+nBcP@RFc0z&&6K6=CWpxx6HMEC0=T$s!6dIg%@ z=o{B3v-zidGhtD1l?*T+DHXQTWewjmLQtBFVWjqE)fq!pHA+B9N}4E7$*n`iFE4hk zi08*iIm0kq4w~hFl}OIxcW=lT?P=Q~4Si!=&^xHH<`VFN@@*c^r z{;LYCbq9Eq-JC*a!f1h<>ysO5*%T4E*=BDF_m}%U7qAe+c=Zm5;e?jl3`X+N8_{4+ zt%Nn_W`Jwx_h_}-=#T4J&E}!V+8tLLC1x^CF3?meoPbi7*=1<_+8I|m9O<=YWrb|y zW}dXex}y+=YiD1`v=+Q%Z+d+qHwS=(kXitbt$eLH!$6EvqXcqAJ+kc>_O(ahY<=}d zi507PE`}l=LyUuqs-Di+(7c;E5TW_ZOzqBqcb+h2Oj_a#(Eps^7%_>bBHOKL5=MqK zAFZ^k4q2%aqKz%wy6n)HITM^oVih+N0Q_MD8e`3>KM~S3`a-!m?s!#GoKWxhBS`uj zpWsO{&J1Mc!sieYM4NFHVU!ej=TQ*B=DR(rj8l=WaShCpfK2L3cUg6>?&=EIQu~Dg z(_bG5@-y~W4$F+y3Kx;jOF>zp%mv%DxiBqVJh@KmGgXM)GNg_!wBED`a%hxsVywn3 z2PR9ec2^^bcLpXStdw}?>@4CyQ{`-KH9ZsuQ%l2in&3g^h6owpBqw{E3@6Y$`4~^W zo+;T#N3U$rtTgD8@4xj|?$3#rpCZcx$GZ{KBgySnC(}nndUS~nY3#AuHhCd08?OT# z_M`$b#+WR;R&TD~*csj(9eVBvG7Wsw4f(smPPlRvHsIqgyL8Au+!vM`p6;&~tC*B^ zzjH)tzNak@z(0-Gq#0da95l4IWsm)<7a;Eg$Duuxt?^O1+)p^Fu->Tu3mhC=#IOkh z0s=}rsG_Qh90#G%_rZaekB^3qZVxEOXnZ)CQCw2O!pz)vIF-ecHIyzqA|h)3n$|l=Zg|LmS!N&^YAmqEBfh+ga9a(?+*Tlc|HU>G zAUkX}O>H9iy71adoD<8&ITXg#IwzQ|iH;Tc+<6#~`!B%|wOR$pgQ+mGgWQb{`ewvDfulIn7*THcUbJrB|QxT4Jt zza`!4@)qFrVx$3vhcCSW5xO(k{mJ7>H)%)HMFFRHL;;U1->O%oLwey~tdPZj0Av{A zkfebjlccbDX`6_~Kw<7nE+79kwZzo%FMHY;T~jD`jmB09qzcdsL$jBk8j8hbzwu;P zT3+>J34yBFQ&I)$RJP8HOeCPB7MiIye=ZGi4;9j)AkVX%QPo-le)}W*LO{>Uo1luV zmM-p6pJZSm%O=qtu})&41~*=0WB35Km7#q-)EBDC*u4K>TxUzVZTz6O1|-3<3tNQ5 z)t*vAL4*&#LM&cv$_^%kUM*0n?KhH2>`FH(k`*}rbAoh~xL0|2YCj4dImVgww*-o2 z@fbk9Q$%fGweZK!9QG#)*@*eod3m!wCHl&Rh0UM_68PceNTv}Hph;g!V))?XOXK0U zdS*)kAiz07)ItM|(Ju5oElgf4&KRA#@8D|WN!X&V=BHhrA+pyZTzXcQ&DBhoy zm+C5+UN2Vc^Tl>kpjzbLd|LD1G?}h5Qv94mV`Lh&KN^sb$1Lw8E#~f635zSLg^p1 z!NouN2d&hLwPxKB2@%~0wK&)Hjtz zVoo2gn!o0}a5hz~N&y6aIV>& z#)lSu82sJPi}l{5x8br6GQ*g{vV1x8Im`t~i5@^SuG&pzazq1xXq{bMtBvOHoLRAj zg=F9g=W!i63^wk&1`9(U`V0v!BpXinV@dg$P92%!|T0;m~m4G<+dNz|p|3L{JGHucp9eUPv-pW(pW4 zWa<^(v~F{@^@akK;7O|>P?n$JJ z%YAYBFR+=e<(1jOwF{##uxy-SKw)-qjQUSH^os16L#f7)ZUbWaXB#a^tzLL)7BN&* z0+69x6|9xo6yW;$vReWLi8M*}kQE22L!If-G-qKyCClu)-PuiO7m`-x zFkumPWRK(EgNF~-mhr?pj**O{_(blA{@2m$W!fjG)UJ|Jfn>&dpIHF4psOK_(A~d% zf|9MEIs${7Dy=RjXUc#-6n)HnJZAO|(8*Xdjs`6ej*CMulFSjx9!hQo@%Qy=a+2D_ z$>~SG2vZINqTQA)nd>*)N_yu;iak`}BZf|>F_NM=qDYpdQC)V1z|;t5ejQbEqFCcY z7y+y=3}aWfwy#)1Aj}2n zdx<$&Zn6xLVp{@*W%o^UZjtpoVNVxvGD(=6J-9#O963c49I?#*)|siKHRwZtBF}!L zqtZJOFnoqSUnVAJ66S8W-d{}+#UP*>DUZ-&DdwKKCe>AA;H2az(ZrX&wql^OO2N9% zE&_*GDTMn*MV{2Km3b>Zk>B}Ys0)&SyWPae(x)t19X7@r%_qQBtip@`<4hYz}dGdNXNP~q6R@cx_(bNP%la8@z>v0kl z5t$wA1#Ev1eNPd))@ppUsO|7W*b{W88UNs97X2nKFLX{jt#e0c;2IEiTFE*Gq5tas z!F-d%T4}d0ynq7nQ(}@{?x24Ex6W@z?zG0&4!ukRQ7QKE1I>UqS9r$46=#MA$32|i zR)4I`nYnShB*@wyoZl-M_bj;##3jA>HtuGWCu9;379Rib=KegQVEy|Jf25c`!?SSI zSmg|1kmKnZcW?v_(VYMz5y>(9*935z{<;+oR;IagF~vVN=M|;4v>eLmx0#omXacT~%9<`Vfv$ zZ<7V=me`U+Rt-oPi{MIRju`h(pWBr$Q|EDq>a_(v(#X*$Iwb(;w9W{EgD?mSTMF>S zJc#xLBxg;ViH}E)srzzmp!CHSM1W?1NR=eF*!H@d-$aOQ@DKN`ebD?UoXe7}|x8cpQBLZNxzZm)=>OR~iYO_Q}Qt+f>A}T=0022X< zNNPcwVNhYtJBm|-@<8DLcnI$FOcRMj5xjpa0{gc`owZ2jmDm^iudEVruHHYn z{1yj@kptI){;mZR;qQBe6_JMRs<^}^B`%T1KtTk~Qc4c;p^#MKGiygBoKqseAt3!? z#iQYb=b|T=$jFfF-XPTL?Tk$&0i{Kt-yorBu;NseH<#1kpo!cAOaMIKtZtJ61Heuo zKHjWGiO-BjKe|I|Gs{zJ71aOLtb+9k7^9Gg8%MDc{clAUmX)uu-05hxy((AuO8uoFk9aIa0N#pR8jn1iX>g!@LXw%rSkhlf`8#c&sKGtY$CC>H3BmS{OF}Sg<%m zC*G8riZWEuo!DA3siZM(_^^nwq~D8<{yqakn3LQs`~7~=z>XFmYS1e1_DR=<2+&QL zRz>CguRg z#P0Ei_jdk^pzo^o>)BP$D?-Fem)AY=i}mkmnLy1nJPQG8(M_y7|K$)w^-4?9rLC(r?*X1-?+I1*ceR+n4Tum9=Ja`yX9)ZU z!@d;wneBe6X^~w89|Ffci`?gX_-$G<_S=xetLUF&@XH@Kn1qH8M7ggUI+=e8ha6wN zAoE-pk2ObYHCXWEqZ8VgZVVdk;P+aYUKes%5%h+PEx)G->nC=A9xQn=nqRcqGdt`#Z8>u5d6?^b2z8-&WbmLMmc*&tcrQ2^t4hG3uOM074jHG zblo}5-*GrQR~mjs561j8u~1Llq0fo1+~efz{!30SVUiz!Iz45ywM+)Qn%}@3**&e6~`h^-D;fNXLV?Q8w3}Hzg+J-Xv3@j z?aQX%ezVJnrKJ%IEdhf=LLvoh@K}h*XR%6rx+01lM%tog%ZN98gVLF#urp#K7-;m$ z5uihA1c$@+ssKyrjJ|Y3mP(R3g!Ufo$!2$w`)SyiGcL~ALm>~yMb;V)rP%n0vXzTA z+r9rw_B4WQ)XhqVCm~(e0?TkX4nkE-mH<&&e}wxP4pi$>#UXQ1Rr(3~`q}z7CFQ2sHLZ!?hdM~EY176Q z8No>ddPDop=iMs3u`R(2#iOJ?Ju=rLN|b}<5N-pBFa_Fv6=CeEB?5L^?Wv_P z?8&1j5T9e`#~`U2Cqbewl)DmaDho=(vW#CnfmSfAc>S?mR!q55F}tvn^v?6TZMlD+ zxj_enWSurtS5>lbPjpOwzuaFx}b_|8h$ zWGn@9@6j-ZMd6K1=As=%;=!kn5K18i+k=+zozNwR8lf< z{`lXPO|rUg@?5d7;yW|=Dneu-VoV#Nt95hcqxkk(BkwM2=U#&Tm$!yVDqy5ZKqB zP`X~iOtz>$=t%TE^5Oc~H6G=+MSjEKr_Y--ikdezM9==n75z|d0@9}E(XUs3->f#e z`j*EIEs4Xz*7>Rh(o=Df|1uP>26q9yx+D_pK)o`Y)`QC4=vup4%-?u2v_Z z^DNW-+?;UjWvkVmR4Rkt<01ltejiy}jn-GI>Yei7Qu>p7Za_z-6#}Q@J7(LKijqbv zp}_9JT`Lh>)5K`v^ChLO*2`k>BP)hYrMH~pOVR7LtuN1MHJfvW+W>@tUm(Cb6!&)V z>>x>sw1+y@mP{rHKfFBJNj^@UZNkEWnS26FcYQLbpzU!>Q3`wvTG z!EJV-vjN3`kGpgC!<btC*HBc4~xDV_QUpA*s``ri@dtzI)B8f%C@9wdLK{89@{hbvZ1|EY}Erc1lr2sqL{NBf)X*O z=j#-=K_x~J1HwJkqmB+qM4+7hGKtF1Tp^=Ni^@}h>5YC!PBjU;ffzTAg>|i9sRLd% zl+ryTWZ1f~iIYtf==f|vedqM|u@OY}g_UJ-Ie!F=9fn=FlAD02)z!EGj9;%?$_IBc zR5Mp4GnywC(fJ1caiJ44jpL_@C0>U&fGGw)dw2EsL#5GJgQ4BKmF<(K-T|W!JcWa? z45b^-VM2pE38I%Osj32;j%m?r>Q`%2ngr2Da^Dx`HU}HE1Tm8 zhELaNCw#Lpo~KoQ#xB`47t;MkOY%CWzT?$f z-#T#BW&9E-Oo9UQHAojt(242erq^zpE#M8KOewA1Vmmw_6)1k%KAHGvEjEH{zp;Ikx0Tr^hbh{n}vI z+t!an;@g141#CsOx{T)Dmp#7+!YM0>=$Cg=fV=};NmJ9p0Tw$K*UjZ1GMk{FqP{-i zh5v;wP1*zo93&V6d0h5ksFZTWH8kwjN>wWoz!lBhTnjU^uY=6W7@dMM? zQr;_I)h^YsD)`~XG2ndpu|Py9;3^O4DmoxU_!8;zL@6M1FdW%cmj??z+M1}RdmZ@v{u%xYB}mh9jWgAJ z=gP~As@^`!rX_0m^x)e{S^%`=8}26cM~)CE;C<4ySe0h8Jm5;`?W?cQhikWrBw7D2 zrhw!vqBr}^@-F;~QtN`qqg>xOu0O%VuOz+9h?<^_-74|^kJJJ0x*w5T<3~4FJIDd) zLhpM*hp~SyEFYev> zcz0>(>kpsa?LNG8VDh$=>(r>KsSWlALpRuGfhnYR&YiG#Yi4#9rLxi`w$%F3Pf$lk zr=C7KB?U7qJbaEl7JSmc4s(TiloXM7h#hCNFo_qPvO5w%`3^{@=+%gFW?wb6A_kFq zQQRNFQ>y&t#0wEKv~CAWJF|UgNEFVoFbf@oyt|iDeK(Qa*&arvC+De_kP^xP=YhGb zJ2AV>Jk?@=@sp*iv46FU5f$g1C=B0A!^1edqH2$xN-p1)-Gj*VQW!)3K!fK6p5N6c z6F7mmgY@%sNQ7NuX9?q;v)ylpfM!kH{S9F~0Ekr=I8fy*Jsr!!+a|lTq&!=3LUcGA z$bGV;!WieU8L0E&OZR(c|F-Pu+8&wK?vYqhg`G~*a=CrwVGwS7E( zv2+Q_!sYk*F_{uQdA8p%O`B9@LxA%$x=7!-mWEQZgTF^o_kuZ=L$G>)(=@HNF{9$g zNlDEMbx(~kdy1xIN=EM``(76kltwp$r^~YN7Z*Ccc4mc6JJS%f+uPV#wB<2}8K<2A zb_$N^{vtwv)J&6Q7IkYpn5PPtY>ny+cIDDIFNg7Pi3o}XZJZQroMYG~IJev{U@|L7 zyri5L_^jLo7NNn-Cja+U{}Q;3INu?0;+8o#r}DSsi0J2aN}Y0rx-Xuy=)N@6hE=3 zAnW7w7F5MwlH6mb+nr{b*xCkDC^j}lU0v`!>87TZS%r{cO!Cy9CYtC_>`Ll}2gi56 z zO^<{Jn|WsZx|kzb&;Vjdj(O@~>#R-I@##5vIkozAmR^oNsV>a6Q|$2gGm*V#cRV>F z99O%#DqRfInKG9bGC3lp?!-;;jrN})W0x@r3#?Qv-L;~FPh1^Ht2O3WslUcRnO-U! zS!VuR(dX1&MI4I~Vi;|j>H1>}{;%J42J!Hl7WU^k38gMdl5A;V zh|i08Z|KWr6m|$5j7&W+8P@xS9qF!&GGY%nqEbb9QVO*0b~|AFkt$1ReBY2;r^ zte;`FLn1{slq%ON2&{&KQN{Z(!?OM3{RNy9gF&k3l(&u&Epm2YVQ^^ZYwrzTuNXDP z?jZ3y^Tx(T@q+cwb55$Kb;mbf!Ej-c$5v+oe2}}7J${v{h5b7%K4y*)IW| zc6)nmY|W$9UtH2}N6>Dsno|TX8tO8FY zehkt38>Ljn6qQw+=1RIgxvqeyoIdj4PrM+?l_l;{UZs*`V|SatRYEI=2<6R8OA{W@ zSQixaKEwD4?sT^HSEZPrxgM{X!)GhztU5omX5}oLgGx3&D3N}<|D&Bm6f>YQ}70U2`y_v*Q;C5P5&T_ zAfr*#=8)bpRV2QdbzNJ|#xtp~F0lXiCo9@J=;sRPY}t%k3$&(&xXc`OCQA+0YP++- z@IoBgWsQfnjBegw!kT_Z%yh%GQWq~Adw*fR_>9QoIgu^hQyHA;l<`GWn$^IRZ%kx?ra-f86ff`J~uMw5G%9|2?@tu zoDXPE{qxRAmzp;Hi#xLt6)t2JegQCHU?S;$sw<0 zYo!}OSMYtiVo3?A+)W})T|-^->}KM&gagOKy&}-45Pfj3q{K0Vb0NBd?#xG3Pj6DC zvF{XPWqlIG(V@7pagd|r9I~Zew za~Qf7bc>?4HSwZ8p=JgpHH^~((}K$>a2T?Xe&}u<3^7}u2+3DbO@xl|IJnT*PbRV8TgBBRP22>ruHnlS3m_7l*nq-A9k0pQ zn%0_A?Af`7Q!I(dHB`&3Pt3jYA?pf1QMCTjq=rPERXIK4H6#xnDf z{?=?x#q%~~O8pLV{zfkTM(eRALPluNO+;8WW_}f^!P>!E5*4|Xdq`K-TG7diI0(Y6 zy+gVGrlmJAkHa!%aW7qR9U|)9{Oeqqae3hvhQtj~39EZbU3RAA&64Y`H1U!N2PedAsN>=e!SX8?;DIJ|depvx|74Z`x+sv8))Cch~**FM5V4C}{wd<5)jDry^;U`G!2)$&3x$Lf@k)m62 z!{9^^9vKNM_I0qb2fU*<~BA_cX#e_L;|vuNjpQ4=x2Zb zHum(yf@7=j1~wYnFyvlCsn7g@%tS_cnl4q^ha=OhhUI(HKcU<($CuD_Z}#M=9vbAl ztA(kInz~qJHn&Ecdd4}qFs>n^e13#fe4^bHj!5G+XSLPzzPE|X9RyX};A61idT{+} zZ|7E8ZKdH6ArZg0f<8V7OW!hih*T+^7jUzp@UW<3ZAlxZIj9dd&YxVmIVx|>+3)W< z9aG3Vd}=Z!=q{|5e!H+C^)=5qt)>mN%iRCXIdjgKPpx-U!_`OIU}?MVeXC4~Epe!ynn-_#cWKC0lx0RvuWIPeBP8eXZtlZoXimoV*^^v4C{EQ;U8bg{zP(kp zzm#*kKwc^p>ZRq+(q0_O?qmfdL`5;+#7VE{(I!x5lN!6c=FYWhvDL#N z6sZ|Xr-OnBCRB7iN|C9p`sdGFT@6FSnHsoL-hU-qjxyW?7M`Bib<5GqrxF?(o0aqv zWI2kivDTUEv^i@=zP?#Xe!J*?JMf18d@rAx`hTc<>!7%|=6x_gAh^3j@ZjzmAcH%> z-Q8V6aCaEo-Q5Z9Zo%DMg3C_s`@Q$g-QB8RZPjkoZ~yqf)XbSPN4uYXx=$ZWshOR6 zB^nIPh24~qBnHKD6*Vm_BM%R}Zy>`a*HsF=)!*uR3AS}qMn^Ayx;sYXLolRcNdQ?~ z61@U^ox?#!4|Z{>W-|EjvXqonHv2;|?nt#NtwF^Y%CFX<22mVFSD<;!X?|&VInNC~ zx0uU!tU(r!O*Iek;fU61Eix<&X>U zZa~bDqDSeK@7r2pu?DEpR5LTl2I9{5_XBKFrYI126fbSd=js)Gr&+& z;beO;xyt(a*7Z@+LHFQ5-8XT~xvh;@T|E7b=BX9(z-ni3NUVaIGKtg6C?dGPVTK-< z1MsqffI_xBCu8);xUmKzbC{+C?k&Y@C0UdtMr=3zSMh;N?t3Vy88UptT95$H!!9Zr z)=02g2BB{x6MD1^OglN{T*6W^f#J2f%@BQWq+gpHfnIMhpqrYeb|hE z3}m0mmWk&bs<1hgNTMT>tzGVAYI$4FIv1B1MhVT7uh~U}@93n~r*gkmz!h1V;TjW5 zNyjDK_han2Sc>6VHSAHS)gB$(R4_M~Px0IOkNXGVUF}HFhjf9do7aw}Yvk^{Gf!Bp z$59X!xI>z+YHjH10A|h5vYb609-Wa^>^5`OiviA|%%MGK3~w8XDUJrqX%!eher`jQ z#E2O0(#8$-&{SK_(I>YZfmA6prV8-831V|*Z@8Fk&`ZW;m?kAyvCA7oz+oMuK)y1F z)pt9hNic9+t$xElJ=30CKyO3Az@n2SY7q_@|c%>pt zBKuBhLJFAi@GJ~R>i<3a6yV?91?uUlVMcCWs2HVC6qx0h;jY6HfL2D+Dmp^cL{LF7 z_8PAHhZdaHN7|^!Vps6`8^UstO4@BzR#>1{!|hf4NNt+8dB*YRd&;x3{Mtp`=h1B` zbDHWs64F*`OYz)};t;&HlTj=Ec?xK8t+Te;UY5eljaj1)%Ni3yB0}>6?1TI01q{4H zW#FblErC$jhn4$7*M(nI*|t_AW(%CDu)N?`oq{DN!tFpj$9D1N;GmGXh;5c$`sF^6 ziRMBuJ0h~g>9!IElg4!9Ez*S6U=_0 zbRhyDGos-imDpdd%xIST3QYCKysGe~bR*!d2_lQ`Lqcu366jSF=j_Fu>>q#m4BnLO z9eB0h{5-p@>C_c>uxaZP9DGOIzDr%f`_ve(nf`U=-N#jgn1yF`aK8b_NDEfgNA+)jt7Ly!-2k{(dL~fH zj%9Jugv7KS+CYD!96tKVMMzM~DEtc^;ckONOQHaa2#Q7KOrzT0raL`IgArPUb~Nj95opiASyH6=CzFFg)>Vw5EOFkXzVm7hyLT z(jP0-b>{cN?XudfaoQbuJ|=n+di~|o-aGi=6{qw5l4$#@ZKTdyh|j33*A`PlDEm5f zg+a{vMD(ZSrZyI;l)!=gp8e}h7IcZH);viX{!FCIiJ%D!UFARTI`UVFvdX51t?{7i5_H{>}m%tc@SM}<|ozu4~MNo#Wpxdb}CblKl0{xnNW-1{jdl9Qe2tD37&ytQ$a zZisPRHa=RFz_9%;(g1{l?X>>cFQBQM&-zXLG{z_{Zq&Xl_~<_T=y`r+cxL3YLxR0H zyW{V@>)7wSjC*@dy6@?iz1$X8;Psjsaq&8!VoJbjGUTYVKf5A)p_El1@*QgqOXK6{ ziagCdp9CNx)S=Ci@cG1;6ntWZy2a6vo>`d>PyH@(M@We>*<4WLmC)(N+^2PZynPl# zD9k7KeHg9`{F5RUmxFU?nY`p%wE@U^le2^6diJ2Ubk-DAflHKdLQ$CMje7UvWo2RA zwWQCgq4kdr0F}^g8uX=fm=T*GP_Bp38nD4q2ujI(Evcyeum3 zo0}A=IP8e@;pl0YE9>|;uP%OInGvLQO)m`IF)bYjEh#+2f?|^>*3+CBY{bU z+hLowWQx369xH7fqug$)5+hs!jcQ8;*TPdWfUeh-c&}fD8`&V zA)AS}#OlE!5sU%nKQbVk9^za1^3h2s{n#FmID=XJ2-Vg{mU;+VFp(Wyn)3mzuD9J# zaXHVy$nwLaqu&(nmY#rzhJ~R7M%gKXic@;O9Kop-BBA0oZfTl<4Torl<+q&8s98BQ zm@FO)<{_)A7NqjnLb)127U;QIMLgC&bX}KK?q7>s-t5-y^+W-f`Pg%w%X22QDT<3A zE-)Cyf1&nzG5fJi0A)9>^AS2`+o$VEoz2$DxZ~>W%A=6?dHJJ@Jm($2`{gb}%*K** zr~Kjolve6Si`=Mi+^I^ZK3k&EC_K5n`J0rCXN3W`dd?e5E49vToL2jEYaop zYfVfiH_i@&&)XeS$N9Q#-ds;U!wX^OOY?}oJ_5E%BE#!1tfsetS8^g8sjFsz7isQa z^xlu(WTr<7A69+t;>gzDD0#0WiGI*lTs+7=gITO!m;PS)8X6(HLrhoIggNzkiG}?? ztDf(ju%zpU_AGNkaH+6tl>>yizZZ5S^?cWQUk5*vL|>a&fAP@%Gk_Lk^smKu<9fj| z*st=|s#a_qa(`yUHMV)P{ipx?lvNDf9jG%3q2vVR^aO7pZ zBh=60@+-MWEc}UaJ71tylE>w?K-@9Sj+U!RHG4BPsUNeHwtLizs^&B6qKge8pV2PwcKAWk7qxjHnXUjrKAlu^GEka==5D4?gejFUu) zUAaO~$mIXH+95Y)zjF2SUQx(k_EvvHn$pRQNh31hE!8#W=8uxl6M)NBi6l6tXw1x( zv#XkeUj`8LR8+O7k^a<-l9yo-@D6A`MSacs(0U}u7DT`+2s*C2wZn;DmQ7#epkK&bn;4NQ6LT{F zy_6cRRc=p;+GQ*G+jZbv8_aRgL%mmA1hxc)I78(0GXu}0AWOG|TU(JNb1Ykoo6~Iq zndA>KZ<~7XmrAPx8J^*9+E6^XMQyP;?gxbU#9r=!N<9-78C~nU46+kff))%V%3CLW z`$e61s%A)1*E+8g0@=Ib-f>ZUKLf-|nXgw&?jt%u$-ma8MA3{&>;%@YKN9T|!S|6g z4!^GR^)$mtdfNDaw>S0A=wxhs3*r|J>yl)~GkK_;;2YYwdxI5lROKj8)P0uB+|YSB z^x18Fz3#y;v(mjfg3W(g`XW=Vn}3X>@s-Sq=U%1Itzo)5=-hPU^vl%q4L(7^JcJ}x zhQ2+A?D{0*sgpPJF%}m`y=cdze=T2^{o2mP1qyBtw+wa9^A7e53jh_OzKvo*=<@Rc zK28>WR`gQaXx}Bm6+r|xEmOUER<%p=?IUBTqM?CzEbvfm2@er%nr#iXq>ryZ(<>Y& zK^2;Py(^>xn?P&zBqEZ99hg>W1PwGChIFPBE6p}3EenofaBw8{Me^@Z{<*o-s5sN=DhcW4zo zBA-Cz)>1u4c3RCq&?&CAaR1Bcxgt?W_Jy&1vC=g9#SFn!7EPPqCd||%`c}%yBle61 zxjJaEEMk8<;)g>s7?mBLv|AARk+1&&r`aI5B@5u~l?b=}$#UldC=y}Fnp<+3_eXXY zH~W6S+AgGZz|K5A+%h8HK;#VWr2>H7VRM))apX9+rB^+QX?9Cw4k3{~Jg0w|Yh_vJ zOVGK)cP0C5TO=-}1ro^uEi z5@<-gu0#{Bebk9-KGE|lX5*Q+*X=|E-7GKr&r_chf+~3XDU%4dQ|!$nn>$mto_i6U z=vmn)4vsA;vEM(_@!FF1cM7-!z7TPSx7dbIhPHd|bkL(Gx=pS9G|TYu4ik|syx&O} zXewx_@9kGrX{x%LNy9GLJlTcIyNzP(YyxZxcqYeOTS)DN&W2|WSXuAbT17V!LHgun z>;)r1Lc76B`6Qc4<5zwtiBWGm;^s6`X;k>B%#b>QM%LV4@U^JM#!Ajvzl}olBX)!$ z+TC_nA5pjgXP99bP5&MRSE+=(LC6Xs$e}7)0;D}Q3ewivE*G5R`+S5Og5BoFaU=v9 zieGg2yj3F)Lk?0$^6pWN@_6i2#?C*-aO!Smkzw5BzcqB^*8LK4Kf?f-bQTkKsxEwrbWjQ5+naSqjQE%^{HuW9?IuuVr zh>J<@s*sRfx9L{1Hs7@;y2DHQT1d%!T!MDO`oYc-<4a zJN)L0a*JV$xC`{MXjbIPqu19&i@Cl?vidD3U9dM!w>1n42P}xrwS7jFZ_}esf!$im|^=)u~cF39m=0#)8jv4b9T}(O~UEhOB)1Af~u{S0$%CY>~~C+LE~uW z`M;)Q?VJwvW)H=aP2WB|3wuPbEE3pynWlst+ot0j?m!@AZhSUDcS=)L5V=`(AX}dCvmOs?fcZoF>CnQl+qJ1N@v${EI2Gf?gcX}2RgM{mWQW*4+^|>u>UrzDHyu+ zZmRsfdq8M_u`6^S>3eu8iOm24cal-8gF=u2EFb5as`NE$*S5Q&0ey8^E4 z=$R5Z{HZw~Hu8)FIUM!IR)kZyjW9>M>Q;!u%QUnb0oGe_pqfTWvJDDL^TXUl?k5_C z%hZF4?*f2m#R}j=GoJuYpT{Rzmmt z{RiTj_>EiPd=_Ei`0nWr*i9W{N}4MW|MdfTXMV zf*{i5qp@K>vb` zc&iiGjcrJJ<>h{7);2Kfh@31lIcg=)B+2=r=9lzIBhH16gxT-H(pTmWj_xwC&*8Rk zdq8`zm3amNFJU@xlohK+*qt#ZUB5}r)l?G@*u1m+W0PGFWc?OB z-o*K@6GklanGNq|EGx$@cGq9sUTs?fh+ckvFuVMOovnU`XZi7^u(QhsbhMwyy^4U~ zBBCuQiuLSTJoAB9aoOXfFfL8t7v6I(l23dm;d=k3axww$J%sqfO6Ci_Ptwx<>K!8L zv}-_bwhPNi`ZmMco{icHAZ~Ga=V9Xc-1FkeCq@!Tqb@?Z&Q>=t<8wisao3p&5upbq z(~;RM@b*5**>h!a#=qhMaW6ZUTFlNSEFysbwvY7T22)kmCqP_QR=yhFTC}PiM~7r+ zsUH?79HYryktxx;CLw>j?2CozgaU0=ToMvJx~2@AMPJL9%P z)zc26aIIKoeQtmYV=kgRuj2CM6pLz4v8;@~7T6G6--z6B`}ONOaXNzn?MW@GccNlp zhH?FYeMD_%*NtJVF=iBF;eZYoZr8uocH!S9{k1q;+-k(dn(DJN@*{!Z8Zf zg6!=>H$UMMY(EmjAYUeO$G=#yFK`TEh3&8`msyu7-Y!gS$Twqz4v18dd~0p|aitw9 zTrKEq0E|!~T!3^$&~h1~Y#hQ0MX%8qwJPY^Wy||H!u?s_X?CFJy%)vYFnx2)SQxry znLW_HZ)FjQmYbiiMinPV{t+3OM=1O}wsE z+7%Q-iOf+OI|J0&o*6n{bzI+U(F3rG__@#V^^cMG!OmZZuwg!Fn%rKAyd4by-#XuX zeV!Ny^GUL5ffCbCYnfYT5HO&SJq%c@fj5*O@3SX0i)W+9r5w+4+ZTE7o6YlWf!E9P zD;s(rV6wKJ$HGgf1C!Tiw9aRuqM5E4hxo35_1^Y3lY_GBD;cbl zgI&Q#W=XXDMRi0$@~C{#Lws|w1tseM%Gs_i2(yC33VUMFvMIs{zqpyD~!HJUPo zz{M~fBl%5>_lI0-V)ot`xz$k>OT_N^sqB1csautX-?i96C zj2&wvWIu011<2)m%cy`ybHH-5UX3zNpyDFr_CT6OmjAj_4~Z5rTS>0gbU~ZyO=Qg$ zmCrG`tRO45KUVl)c6N}fKT&1-0Dr74z`vRwJ&_xfigR>Iip%=LA$n~de5O|QOT|N$ zLRLxO!O=NIQOSw@CwOzz4+mwP-6sLHWgpr(RWM4A?6ru64J1Df729R-)%tDzOx7mQ z+GLxZTsbf?Hu7WfD8*K)!iVgcts2HGIqM;sR!ax!m&s+b9W9b4{v8W2E(spOkH{Xu z%;E+-#XyoN4lWNm#5o}G1v&2jVX`|u#(yT=1PP%v&x+$Kd(cC=;N->ydYT!$FT;KZ z%{8L7#u|A!hnBL;kcB~2KSIF6|FR=S+1Xv$qMyk0Mr=KNe|JOS8e^gQGX-yVdJggy z7OETq0{)i^+O@f1BK_6e11jZtOITA<61DxW5OKgf{{GIFkAy#{&+INoF=iW35S@DE zO1M<^`?1^G1Z7~RmNOzsueip$_@l&LW0i4HtQg5O!XDI(EY%e&CbG0by{FO-UgY^kCG@)Z(fvyPqRP*7XMe(hN(J{+K50)kjAsXO^@G z=$Sqx zlOitbjDQyZVTptDkYbCX{32peR`$=bvJ5?o(4c_TU1cTJ_~=lF8y6QwY$c}R0)NK$ zVjnn{=Op#!l*}bigrQ0Nk%gg|nEW0eAK#PoJ|vOB*u;~;R03lCIV3=j`$2Zl*Lrgd z;oztUc^RU$;Sup^O^4lpctcp^eO2CmU~Vhdiqn2CZyJSm~2(Q@MIXX(;rM|ryc zshfmfdHSzk@mvakj8~bD=C|`jU~raG#%Jbb{G#;p$Fv&Te_BD}e?JTO2x?C-l19q# z(jhIh2<$Xkms&*si4;RH9xuPMJs|OnWXm5(T1XyY#)N}$);+XgjYR+V zVE@wwPrrPi97(<+mfcti9FWfhQ(BpNY2sFqt2olr<&!uff$nvn374yLvcHR}5_?2F zhxSMbjl!eI4s0b0)18qR%-MK5yn`6EVV#N!j{a4r=`Jbnb4+# zL*FjUzCg^s=v9%FTI^|G~dL23ZcNVJ9UzDeHD1q z;hjTqkXc<}xbo1k*@+=~2~H;<*SrNTtF=}o`VkW^c!g7*btch?^XQ$XKX0}aJ32+m zO??%u$W8K#B|Lk&j9PSegr?H%!Z?1oYBA1uPAxBcyq&}EcnW&fOtUa{xjGPxx<)ocq~EjYZJv^XeaIuSEXU+P zpCT6OAY4!m6tUkUBOm}P6}VHTw>a5-$Xp-L5@^oUGLdY4yv{U{SA-Nw&S}dDug-vY zKFN4;+bK5T)L={iA}CV95>r)}4A(8%x>J5|VA7)zMShSwCz2Z3H#F~`0Wov`Nf_GFTAyQyZ5?4B-#5R`S#wa4MMLt$rV#;Id&k6W*B|=}u zpfiJo;w4twYa{5{eOy?v$7@`3g#3l09fN0Wa2}X+Q1H!SVST)uLq|r0BG0*tCUG9~ zO2ueaDN&epR1V9gv@{Oc^9U^(k-SuD9UcbvG=+8SG=+C@Nr2p7h6wPxp+v9 z|3V&!#O~f>B`Bx6F(Da5nsRPHz{RYstM<+Pz*B5lQJ9zGyT!%w1?d-aT7`kLJPI2| z98jHkaAY9p3+lf&H)}rd9S{<(7)}i5U_xPr~qT5V*7ClL^Zi_hu>$*fKw3{lgggxhc(Ayqsys)D!@aT{&jlU_X$Q_Dh zFUgLgK6>k0ZW_S>xlnLaNk#_Y>jlzVG$<_`GLs(Y%9#5hS?}&qAj==Zl*rAD;zxW7 zv$_lKxWMZHIsN_ox!rXvKv1}$Go8=lHa7hgrv&ysz<+)f>H38Tf1}36^E=nCF+n05 z_TkHU`X*g=MNHKy3ljk*8|3(ES7QVSEuMqpBNInkhz|oQj8>1Z%m^$)4W~xum%MHWfsKUThf)fZ-lNC`6Dj2<8Dp_!`t`}?s835`il z7Hfk*j?0wofvArZQlKN~NW2&C)?aTM|Ia>Rk||_|FsiRrYi|hNOF_-KC0QE>u4-_(_bDWnHwSu^N7=PG1pDhihXOmE%~d!8 z@I8#N%G;4Y@SQBJ513$<8@U_76cR3qQAS%I<_W0&s1v=W=4ndmdLR0!p!TQ+^5%T0 zUvJI4t~>&*FtEvd7njDu0I|a}JOhQQbjRtk8tulULT@R4F&=-d#}-{-jx`7aYc$JX zjW(3@>VX$6yPzQO-21LI zJTj7$LJB)@G*ckclRxUed;ES-k_D_q#zoQn!G4~Nj#w*tFU>VCOk_Zog_BiSb_B&; zoFSJTO<}Gr8INL&G9l9K5_Exw%DEn4$2nKWZGGweUV7YOt&4z_2uZ}uUaf8yt$mQs z_yh;O9wS(6aHN;Vm-!Vb1N(T7yIWhC4yO-zNaedDrPkY}khNU1K!d%lFrd~{7sL|> z$|bvfIzA_eAWE#rsc>PKAh01HNZnm(tiAl1py@KUPgR!OeYuIc$Dq+adc*YU?g0K# z%Q3?$+qKU;R#A3l8V9ufQ>#Y@aR=%qm+zc4+WqAh0zMb(ZBryQd7(GeuP#3e}CqaB+X8 zlEe9pYy{Z8%o&$5S_Yf}uJMLs-62Pq%X#%1xk^lT(tuu9H|o!r%C;fj5A3md|0S3p!9Cm98mzJ1TS5`~)&$;33=s{q%85 zKbfxI;WOZ$QXI(?rD{vRO(iY1J&^T4GfkZ}Z6g*)0sa0v>&W`x??JZ|z;qU5RVBMw z*yxXq!V0g2C3o~NT>R+&Xm|}H9IOM%M)n(2E|3Ab@yJ0B-}6OJnR-3!AEVg&{tF9O z>GGLa3*zjUoXHWul>#B#9R z|BeNcKmc})IBA55m-lbT`nz95L;(Q-=vY`R@*o7=ix(ad5i@2CN@ykY-%bZdb!gHL zTwE~r_4P@`vkF6BN`h`mp{An~{a2alY9^=;AAn!}cj&y?za}Kj2E?9y`vxW^4${BD z*W?KjD55C)lHBP9b#?KeZ1#S{8=ij|1^@JC%M+J?U>BrxK&O<<&CNM=wm1vw>l60& zjK_nBp|SDsKnLDZUthgbQd7Bf{$qOl{awyXP$Do;?!erNib!H;PMsVoQg{&wNl7jp zP`0q5va+4oVntADXa!~Ez6a!gEu1Y+kVO;|gF-_?1MT-t2s?0gc9s#8cJLwA7~CI{ zTuC_|`O3Q&{?{+YSlK^zg>bNJEmbHL4UKSHMN?C9WF+Ev5-XbS<>UG$UMVQ^!e;{r z`0&qyU*rKWrwp$I>#c9^Y+~pOT59~2kT1jlsBuJi#=55QoSWa#-lMB3DWXS!jz$W{ zUwr$5WlZ(y6AA8-kuL3CdH{Mtyg~}h!TA~GACee(3ymCr+N%NDZoiwECn-TiN4Kkb z$bCsGYjX@Gk=(e)Y|d=9HlJ7joGQ=+tg$5W>Tw%~=|%@i1zHYxpLOnQH)I~{e@Dec zw|lU8p3t0q!uENZp)@1df|#mv#n5>hc2_@tM0#GGw_a0&k*s_sZPbw;e!J^@;~Loh zQ#gXwax$pjRgIZ3ngnKRT0^sQnyQ%}gqiz9@z+@fc>wl3_^y}U89x`txP+1m>bwaT zG+iJ$bbLbNd>+uXJQ-uDCSFav#qJKMA8Y=e&A-O^-Vwk8Cd2zA`@A>~4|XozZ99Rp zCXYI`?iIK_FmchcO{X-PG9q1T%p5W%)%++%pE*5t_j(D?p8K)i;q|GxJ?eV0-~ZQo zOGfB8AN)M6k&FoSOe?CW@?d znnO9?cqwJ}nqJ2>V0a>#RrrG-<0OWV(u`&lqiTNrP_xe~O6>@@sAvs0n-hi4Pm#tk;h+!4=~#^pI zk_h?U|H=ct3lI%xQ_DEpep;;BeFxE&j*6-2jDZjk&=yCm+*>DV(cEDO(%woazhskPrj%3Q%2obuj8`AkS!xk2j~ zmmS@yRa;`jJ+&0mNKZ8wd|Cs!E3QJK&ZsK~OHN#K^?5Z*J7KX1U z+%30OTi;sQudZHadX+9p$>Hx@rV@5B?9LhGTY{BZf*r1MMd0C`Q7+-^MmN5GmIs6~ z?u^9-e|rUk5=SK_8AghQ3oC+GM3*TytlC9BHF*U1RG!=!!uBhE+ z98zwkC67bU-R!6+vbfPue#|;2Yvm@@Msca#HbhdTFUdD{WND4MwY_H3$v5v(YiV8j z+35wxKwwJ#7a#ODJru&>u8;QiV)GW-vFTsGD3QpJ@pqpi`5#1H9UVx{7fgNd&dG)% zY;CW z@T$qxc;cJN?ey(mmDm0|k&-XzejqWCc;n)IQ;amlVl z5hmWx`zNU_;d4pagK3A7&1Rdhn3iopt%OVcx_@GTtvK2cdolp{PRJIoo!?(8x3=$Ag``0y&&n+=hJ+SY{Nh@ygz7ME0_quurjh;?_xV5-8i-M zh{fy9D2YGt-bh&rX2TF1vVO*tO4=?KX#zO!J&eV!z)|d&DJ zANgD>1x7A<2y`vt;4&y3L?ix2bHoSiO}2d6mMwE!I5@Y;oG$tXfji#o!u8?&%;rE5 zt2_IK)+A437}au(k+Zevy4?qN@Ub;n?KLOx5X<&5M+`Pl=F->= zN?c|pPP4bg>9|OMt9u7>gqp~Fb8-WkpvPm;!yPh7+vlVFrSqerGOfB3ijl~`&+5*w zP)T1YmIbd@F`8Q^x!ERv$%6bvk9GDXW|pRF2M)B+)ZT&~tDUf;WJN{6^J*5yh(7_u zeI5=^Z3NNk6TH=n+BnF1)nmnS7{A0K+VuDyL@jd-*GJoA%*RogCd9o1!8Bh3^l7@| zYJqZSYemH@f8yoP;Y?uM_{cE!9vhF4B5CCE7PX|1ek-K}_!$&$0uY5XiHf{xh)wq7 zm&Ko3W$)YFcaxj97rYsGeqZgdE7s`{MExlspzJ|HL@KY`Zw zo9+Gm%``IP{h{5DTmC*V$l`8S04eZZ`3z6=I5HS{esSbJvNjrkKF9>Li=Ff_=RtaD zOVO}y^4Bn?Gs$aB&AvtKmq1&i&ECHu%TkPlK91DyoBJmE#eADaSQslLJ zo7gBgZgPIH5*(deY=6E+BHXrRppE)?b=u0MSr550#|aIr#SQVlpHSBishM3k(FuVlPt20Is>epCBUdWaUy7CKM2?2I1D>VTu2u z+TIWJ08~Um-ofXGYq!cJj#tRHSAn&@%d=$Fo!ap5U4U;@R&VO(;*TwOc{B~1bkM0lFDZ!t)4?4d3 zJAbB*k)>rg=zPDqg$0nxthX}Pe7^pbd=#Dme)G!z)5-hBVIHayde zjJ3z#Zq7$=#<7?U4g=z`i2tu;0uos6M;~#5VbK2E!XUrI|L4ZzmZwDF3>m=BRV$Xv zUvKaM^$zVVUe17NX=>X->0p(ve zwY1Eus*1iQkNEdEQzuBsD=Ui{8Igmwg&Ed+FgcKW9*mDoG_}{6j-{T)t%N43#$m$>juRWb^V+hRcm&x_b5iX z7N#T=SQ82>5jy=|5Oo*(=-fz`OuRdYGJzgMp~YvvY6prtw3RYSo2gL`4I=k6^PZ~; zxgoFtlh-1(`(3es_PC`IICuv+E>r`p)Krz(4V+n(tzj{||Jn#^S|xRLH&MyNhb7dS zo~u|7dsay-^)2z;eypI}K}s#YTcPfIbUH$cEkTL%weaD6ZK>U{fU>qnZtIRima=wJ z94y+-;QFL8eHD?wjCnN2sL!5XOK|D8IDGaU!s`ik9iQHvgU&0^aCpDgDmmBNmbgUU zSq!tSZ@Ac`^&Mh4PY<w>+X|ySHc0_88M7pp}Sy&zFY&T*`_h_8t_P;l5*^cB% zW;TLFBS{^LC4&JU*VR)lO4U840K5E|`gYM5^=J(IbBp>@^%E2K(?^~`o zqy#j+5SGHV9uU$Q?5%)Kf$EO1L{-_{nXS=gW!W=hX?gQmU`ZD+1|M);GDq?K;w^ll zyaXl!aC9^s|Lge}cz8<{5H866kN(Civ`ch>LaI{A+=! zu)(*B#bYsp0aDD>M;Lav!0tL}o~1K8936#Vfc^T%M8@vPmkbBLZkj~x7iIcX{`4Hz z`1Vx7tzs-Bmgw(*u*%KaV--2o*uax_9vpn#aSQ~;j7h}*ReS!(nPMbpB!iQjX4Esd zEbn`7Tr#q4SobghqjJmM-nU3T1M6>%0{fRAd(hd(b&U)NAi-=t>aJ~Z#rN@@thdC) zXpSCl(NbpKgT39oJtbLMFytI=r9RxACdH&J_Bc#f8Tx1p8a2MLA2V3zL<`d#Q&yYy zkt;b7>2DVhpiAlWq<6mH(N^N=Uc;qwrb_Mir!ttB8w73*18Cb_$sKTOEmy0)<%(dG zwVr;`Y^aK1+;A%H#;tf9wy}TN(vP(C`L@q~PGYV;MxfL3Z3w6h^1uP^bOk;%TI)Ye z$tP+$@#wT$qr8Z7T;E(X*zxiJHGGa`=*?-#6gz_%yk5d+OHNEi;P5zkXMf~7P@+J! zF+eKUHlOmWZ*`{#-L~D(GMsP<54Ab1xD7h*m`1vwAutvlKHWQ{^~^gl zwvWSpCSQ_@kh?oqpgjrUiKj@*lPe4l_FMGQ9;w2zFK=g_e@duA>@zXMXbULnr zyE&Hq1~FKce8FzpCm}+Q4ef?>+$`I5v+l~(A@Y&J>g!_P1$0(gld*8{NSl?uY{m4? zfDayUBS6nxNKLsrc)Un_UE48Une*kop=?s?W?TA2G`yc~MF}yyH<6f3R!4K1ZN`M@ zbhSWjzF{<76qMnV_GR9Q12ay+3WhnQele_PTv9i4U5Ca)J(#>T0Hbyg*zN2Y$Otw5RnZW$W zS5_Ek-{7@GOzv^`=ywKu5OwKCdUMY>{(7G(VN9g*E9zov*fy2hrb%x7pRbp&qQl>-U+v6}9>q}E|Hsemt@pmzr=@Arp4&e34M;_rsv{4X?V*k3D_7d3Y zvJxL;rVo+)+IA~R1`v+DXvy~MU8bfj&T1hQd0TWAbCnj3{oFmF`{+7pQbaM6b_%7?+Ds9 zfeQJK9*3npe5%v-ZE-F^O2e64=e`?Zw1kdyBh7y^(ct9l07SM~EO(|x7WwsI=?IIc zr7Yx09;-QQf)Dq|b0ak}&SvI%-{KELXeX2yPsQ6qc|2jCYlX6c~r}pB+SD7;quZ8z86OR>v&pU7+13wrH2c?!6vzC zH9w8<`H+ygiO!GwPZYJkh+f0RW$`y`7wjvvd{}^ELD|my7FcX3H}#hz+@Po*>`$Aq z469;&hw`}lv;|b4Q%2h&X)>pK8zn}Xmwg%lE51=VJqWBaHL=?pb{YJ0QXn`G1k&t# z+!c7D;(#^2Y%{3N0-nAZEAZaOlE()tbG`Err|%P3fEwxBqu=<&#Qp@+!Ls>EW@d*( zOekCUUM4b=DYG@Y?tB4|qDu2~@@i=yB`alZ>3bUWhIYqSoB7SMnKh z^G8h|Tr!6Vrc(ffrTd09;j(l5Ca1X1GC(FGzQ%se9Frlrr>@1n9BH+$M?;nof3C`Q z7vFCj!ji`}(Sd?aeT;Ry#$-asy&rtIBn2o?7z@#QE63}BF{1zhPl-XR-jE#V$FF_= z@M6h1DMK?em_r`N*=&PEcWPCaW=^8a5w$yl$zn6^C{bJJdh(iCd#%KfYnKum2n4xM zeOq?Wqxh=>Ivfc=X?09CpBGtG_ZA<1w)3`Zvwr-}hcB=y`qGwA)}~7`W3j0uo+vdt zl#z5h+1K1xPZaPx(}Jogqc>WJq_nqz=`*f32)5d(HeImVR134^ zOKc>*_U{!_(3jPA0|YLNM#Jb7!k~=?KxGEP|A8hwZ$#xkv!~|g<@tx8&MquaN)`Do zsg*VzUF}L$&=e5Ef7uF->0rOLcJdSNyn-l6d*5R}ha(~} zhyJPWpfBxW#nP^)*I$`5z6AeE>b_HTVdx6UGpM70Fm;^e&?;G@r3oFZqF6Dr{pP;ZaQY%;}@H z3|xV@mP2Ae&oWVfFuTkkIn0*o(U4fUC`^E!FDFC)RloC3wI zEmolvD=x)?1ZkmI(GUnO!JU)J`~JT7{mvif-gWL-_pZBES|(Y~Gka#wo^7*d@9iS- z?m)dfyoR=SsfH^GG5Ak4MMk9A0w@NQej;;8{^cKsHFEym-EwDk8&d^zWR=j1_=Zgb z{H0(l%!XH`S)6R?OftH?7&TI(qa#m#`$Op7{2w-)yKhElgKpUx0#E5~WJV-wdUIQe zmE|y*Z+GgJ((kl>_W7~d;25ay`nmPSpJvj3Zd}jE@xxL%8Yv;^-AB_*VUf2OUOYSw zN%LI^E^Id^q#hrI=e3V05!{8{C4NY7_n#jPKEi+f^1tUX*xXplndWTJ!O+itL5}05 z^NtRb-zS1l# zp6<={-d+q1sY@}{7+MPhot~BI%~Q>_9M-G|;o?;u!N}i<3&qf5hSOux-C@1acj6&R z)mBzjWn$r{EyMjY)6+i5KH?j9szy&9cr~jjfi3o^k>DBBvtb;4+54(dR${Jqxs zLKy1s!{4=AtM=_3&C2+d=si`T9^^R28j->EG)QYkR$uV zKgcfNA?T(Qn1G3c-kUQ}$ju=8 zJ)KW5#LX!X(iFPdE>W>O|H=B`YaU%%@JRcUgFe@HbM+C6ExBit-<^Y2L$4n=j@*?w z0M9hsFNm(!8}?OCFL;9sly>Z2S)B%!^IUPYhq#cI%;2Tm?A|1oue!kN${aCE z23L1Ok>115gM(9F*CnuKJ^%9C#SFR|@_R$R2dZ4crtb58;qnndnpn<+X(*3Nd&>Le6-4s{Z9(%S8=dI! zBL#{j!4j+L32@DkDUQZt$Dku@a#`_<gDg)A8C;Q>XQumN8Pb;m z9Y0vnKd4X5blhtX%BTrFHVhFKiY?paMTjQ1KG=OCYylF)k!?=+;GE3AxggPaIRK@Y zJrW8vs7i`5Ys-tu49t{97m6sAe62tHT&Mv3(BTd7`y%~dxpykkW zk+!A=f?up#=GIorNJ8!RnkAUatPh&hJ+5aSBKFRjZa3#HNmzJ9fT*)}GCc6(J`QGa zjslV0TfHTyI^a?KoqxDv=fWGOe)3Z!k*qV7DDsrJeW}h_wIPJP?{HAft&u~`Lr`ji ziKxpchKWZ48;WfydZB#4ywo?i#F^dYbZK#?VCsd|+oEU|^E~(+zSnGEDUr`Cf(doQMf%6R%S* z0>M@y>$K#-;x5C3{oSw)#`{TDTOw)2K+(4Okn%hy6%TCs)dRJWPu)QAzOIvbjX-VT zcLfIK32ha&qdFgAJsbOviruXV6kA)n6L@WnHALVtXC_1gmal12EoZ0HiDALKB7v-h z*Jas}tskX=CwKkr;av%ZUbjE>Co;@C_tFq6NqwrGwsM!?4tmW$AIJ1LkJmVWA$O7s zej)3C&=CzT@X|;nR}F_U3t6*C2!yp3RykLhJ5Vt)&70-tNm1b&H8rG@ACpuC7hH@e^=na-A)uYI?; z)zPR0m#lMS)jG`uFHY+3pbt+;4WFILK1oS`K4&Xu9htLAyOhlh#+tIsZaSpQ(6wFU z`$D;Gc~pd7*=YYFZ91=iuz5egmh}iZNDQ6!_XEbKDeU2$@Cjr*^wtwN`iPcb_?$c0 zkaP9jn;(aPx?hwhLx_@2&wtVh-q=4g{6jiz*aR0tT{GijV$R&r0tRL1b#H(;2I zkIaRb=C)l&buYgKf{|<_TR9w>WN+X%<5TT4t@0z-YRZ^nP-fQ>p+4L}OAG58o-?pX zTlYKDD){mBQP8;`nVFXi@t74~rQL_Xu=Ek9QY&WVb(X@_mOFb%k5Y07dkR)}O86aE zK|?^z@;P&-;xul@0$9%vvP%n`-`Wjq&fV=E`OSgeHEgD^Sg1KB?mbqd6|r(Zba6Vf z-u*~Ml(+8Ojcf144lVem3jLUSXSKtE_offaM+xAgL@4P{F9_>%xjvd#YxByEZ#7}4 z;N`VC8ROl1Bb?M<@)&^kFsOwJE#?&bcBVOY@4Mrf9?W5<6_s~JTQb@;=9_sW9AL!y z4F?QN;JwF`ulK-n0X8qjoOssl#&(Z3ms+}I$Z+Qpy)o;2YxiG$I z;Q_uZRDr!|(9~KNaf0~wg<6N?4eWV3=R+NyS{jh4 z3uJS6Amw_|Yq- z=-5fy{Uv?Zk}F3ul&bxZAQB&Fq^35=o#*qg>6wFnl|@3_D7)SbLaE$tjiH+(PSr|V zfd%U-clwF!-6Rq+<6tz;8pWw<7>fg5Enxzu_3%r^Xd@rOJU0%y!pxqw%*mS16;V~+ z1%}@a7S~jeh3b)x9n4byXfFIN_{q{fHvWi{&gS4cgv<1k+lG_}B*0z<_{L2xCUFLz zIL_daW>(MWduvKj?=+F&p$g~E-5L!q9m#D*Ti`5pTR(a$vmc!eIuIpqS?%(v<$aFK zE^()xuFvQG46ZCousS}JIxhdf5;7W8(O`~#b(HIGZ$x8K{D@5j7l$7}BKt^i+tyXmVwFP}Tl}l3+^^J^(>-6wE6VtM8!9$zUo(@P?x2Eaf z8!V1PnjgcZC_8}h9+#m&ScNpzbZt$F+mhdKR;s*eS4dOq?MVCZR254NusL6h zjbX!_%P*Tnte?U41-V3YY+Wwba9l6E+H)W^F5BD;IuhcK|3UBL(wE}gQ^d3aGs$4_ zo9)MK$PdEBLt@FA#{9U+z5QR~nhivpRxi8dR5H~%c4!#!r!NC%mQ*4g;W5~|D9SV>B&4l>)9cht$u&O6GoUecR#p9RHt!ocMCM6U*tQ{u zol3OsINvpQvh^ycKdQmRo-0c>{(Z{l%eDDc`sSPx;y8}{<`=OO4k7+C^54lq6I{)V zJEqKet=#=zs&Rd>fxo=RM5b8_SfYvN8z&xGF17=p!9E_{g&2&BMA`!fAIF!_PxP2U zy&x|#F(mb`a+>K3dXhv8{LWQj{OsODUZbH>|y% zo)Y=Is6NuOHLq@xn?IX(SM3XJp`zM>=jG+`#aa>+@uR&t?u{K=;AHEg;pOS)`2+jf z`#2E6iuNdsotAj%FzjoX*)SU?chd4I9lVZ4T`eo4zZ*&1mnaPMUKOxXxgS=cx1=({ z*tKQZ_w-Bc(j7VJh;BC0C&E=R7Bi-)wuQaRBy3kS&c&#WX_2ZBI)-T&f4`slHU0Y#&=AR z2W@x^D1zP!>$UydL z`d)W($Z&-P+tXDuXk(+Z)MuIW51-T*lk0h{=u2PRQIf^QO$6Nil5N;&L)V;=@MFQV zmyB*TWS>)Q%cO&M*GzN4E;28B71<93Rj-2=Y?_N=QjOovO@DDS2Wfw5fPxyssP2esbWfNH!_lP`AK z=xl?NHa#T$7k8Sc-K`VyC<+m~JPDvN9jIZ=Hi4osE3t~e+H4It!-B5!BHYF>Z?Kvx zsfH^)%`l^C}jHbnu7v`1~8F}~5mdBy#nyrRo(q`uG>Vb5Kb`?(_P~t^3 z+ILw<1To0yq;`5M(5j(mc^v=zYy&~&6x}hDHkw^X2;QrZSjOE?+;;Z5kPmt<7WDxq z?UyM6!$oB&HvL4N+}pKoxhfQ^q5*PP@%ypq`%Q8Up4AOxP4E5|Se!obFez3;qwIZk zTm%OGS3!XR4y?nf&|cLkD;@TVte%Dzr)r#m*cuA4S$?6Nx32>cD|rW1xmn_T%DxAt z6BQ=%7Y`z-zfhzNblY=Gp<%2^2|b9c7gP+?p`vNaR(ls?J6WI`hn>1Mb-^nvtHKIHhnSDu;fls*0ZEqiG0t;6daL|7$tt6fYOR@Pwt5+V7NXciq@VKR$A9#*qJd%i7lJ1aGBvfpO#3H(Rx-kzyEeI>!lAqcDyd`4 zN9VaQ!D|nXONZg-HLQz#&Lk_O+SQl6v)4gXp&#E~VyWe34Rw5OM+u~DkdPCBEK=p# zDgc{6!NpKmeCM{0lUWzq%p^g+NM>k@KgasIp%5B_nJidDO4I0}&>1djy`M-R64lTdr5N z>`C$%(N)hQTkhYldEMs1>-thxBENd=_6C*6e%X0qH>sZc(b1%UMN95V^WUyVF>%Z>-} zrdMi4*7Gr!RrcI<){ZpHTI6J0B$ELlnx+NI_*g1mS~W-hv&TXV^m1x2U7mt`xrY;q zct_}u2R|_WkbR)3kMUaVdT<1m&s*90K7k^htg8OXVxvj!l4U-cvME83g9n_MSueN| z|DxO7qSGR6>bulH+0!)ISS!nM5>~%F0PXU&Az}F*uBXixOv zt4(_0KzvXCqbOraw$LW*|BNVsF?)Xr!QfOSPhNwk`A8?G9-T2(~__=ow7o zO0vqans-pcF;kT-&E3UJrkhGE+*c8C`wx}DiZ}@{QH8!yQW~=tbz2E`6!7ldyVvjD z#U{KvibR-2blJ-Pa`M88x`)@zkpUU^3|dl9+%?rL<{K(&xMHu^K&YK$vg|f2u16<3 z)Q6)Z#dKprIw|y6?hQa}{}N4Vy)9a>bkJ6c6=qK7NY|Em6nLSmvM<1AM}!%%vS~X6 z9}&Dx13l5wmQy{UjpjCm(@|LS2@L>0VNx%zH0f!NO4!>;%6+K@;-GHQm=j^+BQLM; z3B9LLH{g89Ql4i*&hwqNklI@+7>=kR-Huj@c*K>2g>ePI1@LWbm-$@j6!+#06;H=! ziHWH6w$Vtnk1IWW6LEKjpt<0T*yW;xD^DUbbq~|I`FJe*X`GD$ssy+Djq5IK7CLOR zk*5GOIHaBLljofLDgZ3DwvGvd5y}xJiC4oP(l(aND$PS2<4O|SvXxMGLKvn4IhCotHh2w{# zCJBjG5QxX?gCjBWe)=@~=x2~)k)|tD_K3XB@u>da6GRVta0^wjqZbbIs`8|y{;>!2 zb@sKmLq>R~h$(Uit#;efLhu%6!on___2<#rvBZ8Y!aHqRk@dC*d^Q~{Y98Y?=RN4r zLwB)DhrY~D4#2w5{s@astRi{vbaOMVthvPO@=G-qwlH6k=LrC<(L*65cc_@!Zc_cNgiYHEPS#zrpV zh8M;A?=Ig!AFy}cbsc!>{z`W zcr;9Hfan=ENX3*zc4-soM+)fc%7dj}t>6t`*62;yqPsm0mnkVN&8wo`huWs-{ohc989oW{ zRaW{*ckxwn;dJqdu~IMh?ckP6Z!MSm(^?kM{H6#7TP43x?9s{E?1d{k!9VhU&j*lH z%@wcAEXQp5@KxbCH-_E%o@F+_T8{1mRe2hwlt-gucWOT;675my76_)nAwj`Tp+3ai zAu%e2ke7E%2^YT(Q_hz3*S0rcv@>OoHjYm@6xy7;pbBNq2!DG!>8>Sv;S12YmN_2A7rDV96u579n4o&GLoT+1uJ*ov3iMSG~xcXHTeZo3dh*Kc&)!^ z7`v!OSz1aM%J+~K`_zV-91HhJy)zZYGq5|GwU!C3;)!f&fc6^f#^!`>wtW?(`MUr2f2*s z9SeSykufL$xOM_qN>LQEu@Kr!@6A|olxM{i*<)O%MyXNFLJ6?cJ`OHM3JC~A12^DZ zj)O`w9TJq1iedv4g>vrpGF`YwLimXu34-adD_FH#Vqf?^#f^M;^`1WEfhLnAo>$rn z9SuVb>g%L{jq|i@-HfC({d>_((+f{V>?|n+!J)RtzdFH+yT8(a<5;x=$_4O+gn z7sREx?N+KK>`bi}Du^36vv+_!rmpx93+_Qwg?vZ^nVC^n{R-|PRdTVQ)v9vg0s*8^ zg|XxwZk2-fqsI6ppRKP^mA1E%wkJ>CMF+3rL@E#U_X3?ts>H)3zYS?wWjyrC1LrM^ z5fyPk6qndPR(x?XMpb{zR@34Y{{{G`XCyC@WlgRqH9pT4l& zr}WjPE$Nx=Rywy6n*|TSzR-}OJxW`wvS>2S3c4GmD6ZQaUkc;Himd|B&8s`>5PS1} zDiME~&9D#MfDb3*<>a`bWwG&R(T|bVTrmB-I%u!Xscz2vw`rNG! z>n8R`Yd#n@Hf0cY0%~_`<_46eN4HQJYYMhMgOPPx=Ht;_S+w{m#~;go1iWZt^8UWH z3!zx;NXt_$p9$;0NKE_w21|D&umA9UtThmgdTXDEiZrncn;1c=LGA}$TchbFq)8^l za~D8E%N2~vYVaHF)qj#l*yBFjz#JDBr>tJ3u|0I;kJJ2DvyHw5^P(PM5 zkq_;XH-H#C7B#mlKd9lPjlI6UE(W z2M%pWJ4X-@XAUyI52X7jO*r3&eQ!<5>+Z;Rhkd!k?9Esasx_^%PSkT4DgS=CRl@>Jk@E+~B-!J`_f#)X;vPUMz3@5q#l$9T{21DwYDD z_?HB-ByOZi_ia}Wc#HGW{Keis*MDjHg;^uD({W(`y}$Yor=)DdA^t`i>ZgO7)3a3x zg~mPL8FsLb^BZ`N2xQdO!^V*8GjB)R(8%H^(U^h!&ROmFFAvyeCS~P|IF&uL?%Dcc zY%?)R&1IWB&};xEG??);4j)RA?rfy!!Lzx9ku z!81W+vjt40bnd2TPY4CIutaUd+n2_rCT_Sk1tEIZJIPMCBv$dNow&@b-J=U*@w^$6 zSpG&pY}UotBvyX5C`2hMI@)~FFA6sUseikGQVH;h_LMT%iMQL{5|G{X3 zeV*?Qb!yQc39XS&^V>2*{SR}s0J4@^ybuWyDVf1D{kQ!%&W~EDt)MZrp+o9$m=`T6c#htaY)4jSrb{GX%0gLK3IBo z3%VoZP_A7_*st@D@IC$N-}>gyuX)w%_?|q?N$}w&q$+id8^5d2jx1;|(^|v$i#M@2 zBI91muh8*ay+ngE8ssUdKeasi2}r^lI@eftb583kG+a?SHLfXLBDa2f}XDN~a`ef+qteowVUMyy`cXZi6u)ej)|R2zr1aM){{~Bk<)aag*(E)+HSdi{=gN|VOxKA?_Oyt;655^aguQ+I z(ui!AD^`%_3aImTwV+V8FJxYWUq9$vyJ9UJ8@BH1o#Obga@)z`qJGwIZUVF&n=i7JJqgTMx(4DjmXct7UOYc z@0@8YxjJ(n1xVPr_Xt>=TkFwGsbKO^Yj}Xt2{yJZ*%_jXoQCfXta4c(DQvnWx2BxY znYM*zeMda-a7(IBH#pyE*p+@0DTrOKYH+gp{wX*{X>tON+uUDa(EW7vY1*O?|F*~- z6I`jQKCADw0kI84@3KPqSBLL+I@hl`e<&O4V_Vt7zTOL>(cAY!@JQrR^l)Su06tGt zZZwX67rx-g^Agw3OH(!z%h@uJybzJs4>ZNo zcU!Ode&BQFA=;1QtE&L@eOrtIXk@x?9eIHtG%SFe| z!8#Ylx_4glqzXc_gsr6f8z~ZaRXeBQ-4xplEEBH+M`W6=$dqvL>#EwJsx0ORP#SgG z^FW9piTY=;fl6_=nfziG zvMI!ZmF#wo-SbGXh&wu~zX;)K*b3q@65>FYNYTyR@+^Q^kAMeCm8Ur6gmpW1CGVSJ zw)!<#ZF07PUq?ToWTunX5HHn$A!n zU3+9g6>SqZ=u|1%ZilvHR){apM-2cNi@t5p(s>Kz1Av4-*bSjII+hgK!QTtuiTm*H zRny>=EbBN`(Dz)qi(xV+AL-ADN2EQr$K1eQPGB;memJA;Mjez@D68GODOR2B`^}pK zuY8g1#EEq+feB*6^i{O`$qk?UJYOTKiqlMO6by5Udi^D+5jeGL<8>DR#?f@ZLmulz zcd|atX0dK=aRO4ZSR%<2g|>=AcMNrhRoT9UX649Ryj^^{Bjy49@R(|@C*N1P|JsGl zHi%*}$X2Q!efNv{`b!m$JQJzo+?nYKX_f8-%Wve@X&$1BuJFDmSy+uM#Y%Lqg1ZDjAJX;GIidf+`eI;ZKqqV3X9ajTs(K6ag7X#0IxXk})m z&(3O%>hQ`2bj)rI@ItyRu3rnS+;K`Zl%8CTTp=B&fs1J5^*7*skOtG#DI;!v>!VYs zqGFEyn!CG?y~Orr!7IT#9G8aOj6~R$nmpzkwd=)av&$JzwNvsRyvp_^uyx`}SU-Fs z6DV?Q8lTWf^d86Rb-un=jlGGq&Zqnyo6MZ=b*B_&!nvu;l1jHiFMR512Q7Bq;8gY6 zj4s8$uC0!9ujra>v|{^@!2GSu*n?Ju_V95np>!ji2RJ{ss}EQ~Qwby5CPmVQVR~w9 zo?=tG*EnPy+bMi*o5;LM+ z?z&UECH2kmFjQvFNg2iJO{e=G5Kn-#D}D2shTo=#Uh1a#>0|Cb;57kwh9}Pt^PnrW zY?u5ET*m2-kL!)lU;2K3sGl3^qR69 zoE$nhxyeYmrO59dYzMPC))$Wgu)tl+ z%*=nOjlUORH}t=FWtmzgN8b-4Ql)r#TN~2r2qWIai7zWk7=D{k_&2fs)L8vo?@e7D zqFEaU+#Acx>jj_!yuJ?NXiD`|AZ|1MTNm6Fkn9W`n)Hof3prsb_~j1$FQo5B_qP+~ zZ}IkHy1vnD4FnJUe?7j08dMdl3>_zCz3s5^z2vu&$K z`_qcq<{nF|7f?v#7YqyJ7kII*m$9X>;h?TcM6wdhm{SA%KB|{I(=8coGc$Rt?`t+J zepB`Stu{qVnNw!hN!#ChwG*A{Z!G=Q5SjY-GKW`o{97?0h{P57yO+NjPL+RoI{)hH zu0=%RKWP*7ES1xLA(G$nV#@r#Dg6KQp|#}X>CJnM%x$?O|F?c!0eGLVaoCz-C3l(& z>HNlSHRgXbFgq1kMeLtOXLeWLxyxiBcbf`Y%?IiIsQ31}aRN+XDw2^_t%_%HQds> zKG5fF!75IIUIe<%C`#S(Zr)k!(sXAJKV_u6fX2wb?*tE-1p@fkosKBxS& z*FQwX1ao^tTb|?1pLfu~c7_QeM z;1i11JE~?(`5bp16nrq6e?(rWFEg!%8b?YRB*OjTElHK85DSdSO{b*)6yeySBq^e{4|S zpEfMGJZAz7kx!rOBv*d$d)Xf9^@5bQV@I?#zJ6O!>ad?zFqKoQeg$*4Jy3L2=4zFi z7idZ(sd&?7zP#kUBnmyPXX`s)ECm^si#g4E zV@L){+B@1alX+1DoC;pLwqFP4vFpXM=B)2-(o+N63|~Ya(*}qNdni<1u?HU!ezUU_XN7gSM4SC@aAtOjO=hw z$3_ckAXRBc&$I*{1UrZvNGqk;_5|_F=T%r3no`4sIv5KCp(uD(wyw%3ZGtSAqd zJv5a$PVxP=GvdB6$oBy@G<%aq2Tz(0h>OCM#NR1xlTmraqp=W%w)MkP5G=uHK3)|( zBjZzlrk3evzeOri-AEl~9YxFf?kq+i)ty!gE0A(HCZ6k+H^I zkX;_-l4@b-^J#~eo^$7*%GM;iyblw$!-A;;9x;X*(!tSB+cH;3#jd8%E{D3`AX3#(lx$$Keo8hp-O)0D7| zC(HGjuo|Ke@pfSpJ-5k7gq2__=8Czkh78E#qlh{rz9f}FNUR?mde}94jQE&$t%v)z z%Ii?|5(z5k?p>M+N-PY%x$QE2?%n25@%*1$t@DHvOqDv)jr_Df9Xsab;=r2XWW6~Y z(Xx%<0#m<8M!rK`y~%tdBsypp3sors@J=+ezDq~Aky=SjjOu(h(C_gz@o;opjzcHr zd#S(QF!a0!s>7K!fMEBN>gpZJsABZK8?o&)|MFFVe9OtT$!WRgTgL<|L$s_}dcl(B zsTnGelRE}-0F!Bqze&RP>#UGEsJZ8Kp@cs=lNo4NT2ld8(~bL_I?}W;ddKtK(>iab zqOh6=Vf`KUK@$`i!|G}N3lDReB^GBD{>2X4%*2iHGZff~n6{>jEdjz9_e&U~mFS~G-Y$)MS%>CRgnt1>eG00F5BkO(tlwIEu$Rh~3B_b02eiwc~ z1L-;m=9+<6Q2Wh`` z{C6pQDwMQLp7;hAQ$q{$ zt>;n5F!NMaC?asB!R_+y2aFwf{>>J6t%Yw$<+U9BXy2D4>DorCAXIE z;&-q;sVnsjU}lNzClwOu-&>>tww z;5&w%ix=dnSoyg_J_sC7>7S}Bo6H7owr+`SS!82dDbX$M7SMuB+&#jgo@O4Oq6#`0 zzUu2aqXt{uE6rr(peX41lp%ICgt6w>mOQiDnEc#5o8}zdOj71yl`&$rC{&5@)Mck_ zGnl%TTg@hl7(b{WRIK@~$;SstFi)QPn=e=0cSO>AYJ!e&lKFJx{)rRVMb7w-9}cHV z*ptG0!^ziN7e?HsF-Y7gG9Vi3P{hs3h}bi(gIO-Fna-tW=nJC87KUUf%PhOY+`?in zIKpPyAOqEO&qif57^&oPf_4JwtQxxOocYiffD?upi8d5L3+6K-zzA1>xy20GW&)Bw z+RI;(%)b;Gs_O1RC1GlSA=Q9&A+Cdyl!6Lw*g}qron`=V`zYN4J{pSlKL*_vd77?iL zd*KPYR(>uJU;=*-zSUl$MLs|J-=@Ov2JAj~vjh0PgnFFlA1XFVHtnH<{uix;fAK}k z{}&lB|34lYHA)P(slI;l#oOc`0vQHb^`{Mc@O-l$MFKF&DuV>n#f^XHA}#KB@7%6n z!$6&+IZoB7i&6i>>>y|h75@)fd3y&2unSgG)L605Bg!i(EE8V0wG}*||FXX?fC6GI z(BG7vP8!}p-)~Jca?ocj5Gluy+RvP>5`{|3$4tbKqWpx5tbg^O8}7b@6gvwlu&h!R z$Y!QY{Nw#-Vf{Ie-=D}#D|`D`l@d92l1_JOR1JZ)UEt99 z^7kR)LqBz~|1viv9UbC=RaDniT=A&%>IwX)!8^^AqF#PL{~Rt>?&#{uhPr%!c^4JP zS6g3C{OdtGA(g;DNfcVWk$!o{$38@{cQb5Kl$k4?a}BtneE8%LWt*R$H|>P+p%8&0l)HDjE%~{iZ>p+LXpv(k`m?@% zL;t|Nygb^ZgPq;tGj(-!;}4CE*_H9`pAFbJ=})S`898W3`RNHV{;76C|J7A+hE3J* zNw5p^PbUyc3ZNkW7jE4mlE6P}_rK)j5v^4z=^Py;`b7HQ98nFgX(WAqW>U?_%=G8} zIh22VI&t4U`?!A!PfpV);~W1pME3buFUi;nqr#e0j-- zf3*!dTPCNo^?R?Zx{Eh|{5l!wNf*M&9ZNvQyR|NRZFt>9EQ&M>x#^{dL2^O3-H#oU z`O;s=qb?P--!*!}jpZaBVdhsvvLBNv$m zfjW7st2)N{Hr7p@6WeIrOt%X@c7C6JILtdW8*pNg+CoT?p{OAsmTfgd<66o)z_MZ- zm!>JzpK8ByEV}T>lewFvM62yM5lg~{*3vhWm~5oS#@itaDMux`?EqJ-@z`-s;<+~+ zcwiv`Z($hUw+h;%0wHtiJx;&)P`(jyz57`%+Ot<<(Pd`jI?0y0pwO(J-#rB2YtA`C z?0(H5;F7+zF9ZU!^*uFCaINh0K6IMzM^vze`z6R6VR|B+XKMqWKYFpr)IK<^)BAXQ zZqVph)WW@$i~scF!~K4m*{XIh`7(k++LNqiovg0T)9PSgupl(2qe{5axdQvH{q6#^ zfIBxvjO>%H_ZGWZ;o)@P9$6EW|3gPmTYnK8C5RGxd=1Dnio<9y@W0Ig8UEPMdz45i z{GX*RxuA?B*zVFlWs@;ze2_i#+UO>_ozfM7Mt62Eq)EK{a*E#U?0fslYUwr2&C%p; z5%(a@`={8}2R2e&CwpuAd8`%s)3 zE;2@^h0zWhhMc@xyt%vDAIJ9qyaT=_4xh0dRXIsKNK+FnD&BIbrSOdJb@H_FxA0{; z`D8?`0CN*zfz#coiM9SBc&@r+>PK2{xnCH*|2j!!Ya%S$^=!KLKu`@Nox^0TZ8-P@ z!jSs5cj?SJJGT(TJvErv=f^Xi6E#AWKJsIk+J6KxFj$?nT`ZH+y@dM(*zyc_^2DG2 zb5K%G6v?-Oq06-84CTZEMEBZ#-26EwXBDDS$iOI<4k%ftd<*1i##CTbga`n3d0X!} z2pf;Ao@0-FomCVggDa3CvhzkN-z8EN@}E}3lPiB+m{IJj>NwTKwl7uEu#~K?4pk+- zZOSlGqIZA{GJWap*bupnO87?MX#kqN&xtdZC*vOX6;+)yX66s`_!K)n{?yji)=%8B z3K)2@vl8HeV#*D})$ih^#C2d8q1TgVjrFcQReq4b29RrH^7TGB@Dk~ZOzPtZf!|QU zl|o?hfbQFH{>#Lh4pHs7OHEn7NY(e^(R61*p=W(*ckAZ+1Lsd%j(x`6_jf}hMuHEF zUPkNp3ttP|99_$oYXFc(7<9`iI%AQ<(z9N>!9k0|`Q7p&pVRqi&jvF6PW!dWE2LZC z(kJed1sVlIFY;-ZJqv@fqMKzqF_MM&1JC)duRGdTzn4Q1OuNg{y8(hmxkUyo!+F4M z>0En|Nz%6g0o8G4NuRAnfy~20F?p!JLtC#%-QJ{`A-jg97K8dOa(??Zz7;>6L z$Hu9<5YM77FM;Hl_iKGl#=ensA*QL>6#J}|NpWh0qp>P0x@Uu;A22?AHSpAwrMGhs zpgJu8!A`Z?eg)DYE>{=i&wj1l=ed)hFp0Q5`<=v8NLfbbxm%v!Ehm(fRQ*h(=k!qO zEDpRKX+QN{Gt`J$2P!BktS(^$>1EYk7=GYNMc9itBC~T~B&05@tcz3?#e0!!)MM}R zrb>@dupQl5>y@H|Fqqi+Rn3a zy^G6O4f`i68IKMDhhIr>EgkNW*@7P1VEf5TX+&legfvkV=}u;3Wtp<-x~4oPcPF9| zngCEa3?}BWZ9~@9w_nfq3yMgnS1eg_fvr_VgH6=$a=TE^-zSduPaVpAd!Vg0TweK+ zJJ!ZM7HKj%Ry6pD%sn_>xjK7i-X)a*r#-_ez$tayl5_$4h0{)HeKUou^w5-c`tY2Q z-D36O8iiix$3q{s$N6+cL~2CG0_DpercxA}0)OlY9t@Uvo`$hI&q>#cLO3+>1=lCW zCdVGm-X0LQEiL5`Jsu-pF}zd>>uI#lm{4gzzk`}jg$&ApUCP#Lz?s=!DBG(%saJO- zr2Wjq+(Sg6YQqYnsMhObYz`RKiEWj3kk!QAlP-LzEI5N9qGe2N*5X>=EUba;qz z0af{5QDA2)D6(ol3Ow<2)5Ir?SCx=>7E>TT+_xnaHO!pZa1X&2O~L)L@(y86bHJL@ zdnWTRvwi_dX?Y3#+cU2#8so!~6!V_n1FAqh6NY%S+;M%_2TJFMoT~jZ9C(J7zjL{? zq+Qn$;G}upeU?VDn#tZoP~rHxy*r%&)QBV!Gc6UJ;B~X0WNgOHj8B z%3CSt&;`AvQTV9LY?7OUhSFcNXPA|3$*MnqTU6(VzfSoTu_MF6;gO?x1b6-8H-Y7d zMxY?Oj=bGDQ5Q^&+OjZ8Pyt%s*}h3?ZW@>7RTwn0D#t78bj6TBi@Q8!z^IWGx59UF zuI#SK2hI)FryJx^F}Lasj4}P1Q-VdFK|V(qP4Efvjwm(8Hd5ftuQeyi72azdviP`S_I$q9#0r?zQ4R$2coe0Cq){!*YI(B>Wa)B={+%?(C zRZT zhp4o7!oq5%kC_9-6JiAvQ33$bImhKQ6hFsC@qGfkrKJ@3p&u@z=wIiwb@&Bkg4 zhRRrV+AQ$H3Ko*lYF~rDp*z*ER@iFkg#J3_2xnIJa2Hfvf{w#*2M094e}~6dFG>9zx^cF=}8d^}5%aVb=QT)I-u|vZ)om#z-O+ z$FPBc=IQ#S2Jm?KVTao>YosP&Rkx?BMDj)S7aN_YL!neoC+V# z=G%I?V72Sn`dng;6$GYe6CZx*c%yg7%?I>PGj5a2&Zwi0M0(ZMdT$z=nxO>Ny4MX%$K&ds~!Lc>Q;@)|SrBIzfnKy;DUcVjG z66-B__J~J?gDgK+>0mF{G>Hn$Zb7umsw3HZv=S$vGV#;Iih{$9Y>Zm`rI?iD*0$yG z`V+jK5CZuqrYf>8T0>1hT@wx9~@#AnF|gqtHX-l`MczFdSSEq5!o?`1RN81 z*4XsC4;_iYz|1FQv|BLS#r;!H6S3O;HfQMP`dT^z#S9W2~U}ei$#W5a##Fsk$Vq)3*%-E2^1h(HdQUIarLx z%-?xD<$E(^Gpdn8ey|4?&}Ekv#~a>Z#6;fInW!Kpkqpk6*GbGdu~?vqBkzF$(zaMJzd-r=4;t}j6bt0HTz zj)M*Qq6Te6i5j&}#LPjnY_!uj*O*nZ3W*PS{c}k~<5CX2}*C~ehbhT24@+iN( zt$M-z_6?&papY&G6Uz4cNIy*2#$U&ihLBp(;O(6If&Qz$;m1LSxDmwmwT~EIi{ZX! z2zxrFEbP$P9zU5G|CAmIhnc92X z14#JO>BfSTgKbvh3TE#0=15xw+(|67((ibOpen;BC4^G3$?xWa^9Ro_X|{fA2H&qv z_r~i0bW50ri9xx^znIQ~pr&{OWZHIWJOZu#O0DTiE5)85&eJ0Y26L&PW6UHh)yBL~ z(I2E_LVYo-?TiNPf%kN`Z}gaE-GY3QGrmTNw6v1e@W?v0zO~xSr}(8MKMo-g^Gj(m zDv?g%5rX3eT@K>8bmVWGi4MkbepfB9@o#!N_5)GRA`+U)E^xg$yuF_57P5BfFw3cL zKuK!j*r~KR<7%ZIfC|nAJPAyt(=iTV@Dpv{V_#Eks5=c?LVkyf_Y&;PCFlPB5c3)4 z(Eo-YWz-Oo0Ug9QKi4DwUJZ!;y3YBDW9azOFEKxlE@9~H4IDI1wgi@C>WxbW@R*L> zhuL?%HR>HtNL7Uh+yIcVa0Mi);j#yf6hCr^@JGiFFZuj{nUd9$Xnfc%JA*?WhWif`2- zCRcg%Y#D6v{{^(~N!d-A6y^7Xl$zC8We zu{d+m%(mD0A^0u*$zv-Tm88hXuov~i!|uQR-mJgq;FYcXir>xH?a-62=P6`N#NRd= zx_L`s_N~cb01^_}g0ww{P9?v~Mib)| z755gj_G4-vk$wH6o1x$Q?w7CTn1DV3u|DfE+jM+^jeD5#ni5*OwL&^(LSQ@Gd>2o~ z2MEU!dU>mO=sl$XvNIQ2Thrlzo}02tH%1!Fd%0{9_%T*hbH3mIR^pWlX6O&w!J*W& zIHaaK%phUKCSH|c0X&@P&xVtq-Ida+th9|j5d}zo=vD57(O5g*RyC-MTs77((o7;} zE=d=}o+MLA4*U|%B&m-#YGq0D{Ql|{vx@M6TRJ5}cJ!98`4UH7h$_Q^^iOm+!EJNI zRW7{fAC3F{7J>D0w9W5|4N4mFMy^cO&^NbnRX=rR7)|p?bjFgO?kSid5P_lv=H63V zSf8>m$9-Vfbten(SzOkM8cHZ`YMZ=pEHD^zIrfv&##i;WI5-#`zoevT%PF;nZ)t1W zbu%on<8Hs~n9d6#!&g}NLA1k{Tkx%Y0l`{66DXA-%Rt0T)o4KM1&>&YRO+i>zZ7wu*KB*l!U^a)N1hOCukmI z@>{dz8W1PIkT^qaBmCzrflBp`7C|Nj2BVd4-PyqLohv&S;lTVA5%)AxjshM!k|D8& zG$Li8$82d^mr}W&#l;}0ErRh*CdPL*L1ieRV<#WvT5ZsvwmHRwLUaM?PC#Z+L#=Sv zxJvHfno@w#%E0(_NqJv5&k08gWx~6t!wGHF4K0dN*w8u!S(iHU_U}@;`+Ok`y|Y z_n+P(MMfgPa;rtCu42!fb*EvFE=0gPWfM*o46r>AD3kWshrRoVAP91!^KuqbK+&Mg z9$>f2%A3_}K3-|>%%=f?bok=)B}d9WydrADuc+&cT7O!#kCUcq2-n1;U;~8UCqxDrsfzZ*beTiR;clNVUSvjp#NpF(GN33^YfIq4hk#<~eO*zFP$uui% z`RfU2(xEUk5N&Zw_$Nthx_GBn!h6S0rR;*4OJBY_5_qt84F7v8#gz81S)?DK+F}zT zc22&PcmquHoQls6hz`D~4@&A22Zop(Z_6U0B3mL-a(RX^lxWuG4UiO+cX)+@o(F;t zFp~1eT-R0X0^*H}M~xd5cvgm@g3gv#F7|fCFRxw(%pNO2kf_i{B!3JP$Otue2*ZdJ49&8cU@r_KpHo{no>4lAx>7D3 zb7$w!&M z+OB1u0;3@-*M>K5sNSh~;+nN~+8wgPP;*CTvurL#;_Z=2x57A6U4D{%@TEV zN}`1#A_Tb573A32I%dQz+T{HfSU2)%_-KxA5w~oZ(CVDiDoyAbl;v8V07n(!9Kol| z!9n=Du8YKK_Y+6KSGgr`T3lLZNUWjp)ZpL_UtH#luVDcIH13sSSrz7lNd&uV4;k(b zn^ivv2D5W04CzM9ha0_tNc75^9Qq{wyyBPz@rd>Ie?nX6s7f*sYoDjU0e7JzqB4~l z9Kq{u(yN7FMXp^?l%`I#IB!fd5^YYA#Gl|rw=eW{<^r>7U{nX*NKN9r&sapDs`ps) zn}pWx?gdMAdZJ_{#zZtEtsQKqTLxR5y77&ukrZ`=9`LulN#{@rj;Vu@6@i=cZ8zuS zh-?@NTzU7h5{@I>jV?OB+vhnead;h2xvZjU%a^)wq>78mNgzd_qZ8=ry|iRad#xwg0S7^8ha83(y6 z|6Cxt{7x4cB#-3a76vrmr0N@m3?^5FIy)XSrBVwCE6jU67SqtKqm%(FTmBRsS;ST| zY2)tkpg$XA*>5->ybJh!>}(gTEmlXxP~Vb4n#!yJ@TLmgFclU1wX(PpYx$0)w%Sg~ zJLG0|#GU`AQbjL$hrz*Q8UqoTW&`?a3iB^BHMdXn7`I8>R8a*gcpTXiviek05oWhcGTj-F zR@bMTG1JbFD!VjDa{93q=xiMmh@TaF%1{Cz%yZF`Y)%h1W=D|wg!YENDoX%sS*gM1 zitW8v!1xq`?v1uq06~NveJ)v!{cjE%B}KUOKIsGvz0WdEp5!x6nkw{3VdikYdF`L1 zgVtzKBRyEA`8B`|{0`Ic%5U`*z*I+oOSh2epEPYoA?mvQlon%MSn(r{S$g@(0`(~? z5$PR1k!S0xKR13!kCm}i4+j1gm?>geR`*nAzP68CW5u5nQLvM?Nc-_|{qi?@&qM>QKvmTK>#E-xuF2UXwtW@8N?PY{78@Or=g(cCWQgt>dD&4o<)Vn>wRgJXQ z=Z>`624i8JHi!ZWzJiTDV}Ol>*R#cz*O+-3ssaKZO5dE@Pa<%&m89zH$-Db;)}B`lLslf2rPE#6kLaaC*+Qp-S9)1W+r?)OJWXJswIA?l5a2^El?AOiE2 zU~=;I+&yGC_7S&HdRvwd`OP+6XLVv;Ud)1ACJF3!a3D*D^ib4b$&sPipTZd^K}h_m zw8@RYN|mm{$ovNlrITziUwGQWLHk2fLD$VGv8w8@x*@OzS>^OlCF_ej);CnM_gJTnM@)ckcnRSgT0=ypTr43HS^bwf7tJ1X_uUHd@!OvhMr*{Qw%?Jh zqgvhi1&imlU^Vvep~p^TABe_N{2c765R61eA7@L1B$s`;6+i+(!9DIdWInvV5BRJ$ z0zG8Nd_1!LaQD5#iyo5!eq=f-L$aXk5yT3MpBP+OD2vC%da!YOsdH@bc^my#;$ zT@X6>+8#OCfJj8-)kd7gc+w6d_{fle2uTBOOwB4s5Flw#Azj;$*R!BhAxb3i(AEHB z{oYUQ?A>8{C^Kv60A4RE&G$MXh`qC9@AMdlHYQn8JXpLzY|_j^>CMn!Vo z*jNu>0_-787Lg*8{Ywdi1&83k)WIzcD_i;EVe-c@RpWp^Uyc1|8+8aH+~oosOxr(Q zee}Hj@m(waG{}}z%r9u<{V~8$&hiB+-e*CbeAGibu~ES&{;VPa!{XkK#U(-BDQEFZ zzQONPms#SwylR;j(HH=4p%SISxL;N>CZ_=or0QoYQ@4mo^KJUKcS|=KUXwP)k1?$F zF;u_g37K^EcXnwA3qkTd)`ErKZViIm&cD@)R79pkZeC2;jdp_B%?^Gy;eIq(w}$s4 zfRX0P3L*+nS8lP(ddym-UmmU@tmQH42Ego0fBiDM-}bxynL>#-eaFDX==3a5Wacib?IUmycpCp73jSFD@0E`HD?DmCzI(z zs0#^+D0P=tNz$xl(oR>y9E zvQF9Aw}c{Kp`fP8YJS!9@n%Fzmmi<>nZM4xZ1sN4JvR~GwwS7e2EJLOd(~6->fXP5 zeu3>NAB9A%bJ$M|q@3ceYsvPrih=$I3#Eh!13#BYkJ2kycLnT(&usCj`dKm$z28pH z!O7IT3Q0aq_1v<0-e~Sidw&+ZP6M+KAS_qf)zS9^quQhdg^6eU8n+Eq!yf3Y>}Mra z?&PYDoOj7I0FY%zU;Z~r)(dv0iR+Viu>Kda?LjJeCdJZHWq7;Jl)9U0rHCD*d#$3dCSqpe`^RP+7gz$smuSXf1NO+7rQr zo~>u3^cG4o_0t}JP*@Dr+%`nJfACizVcXjNrFiX~h#yP*8av?fJ$6oU>3deZQljv0 zoR|!hVIg`N(m&3=dbRzbEh8$)k!fkgw%p4sxs%f*AeOz-=5yeq%;#e4gHLIsw)6{k;$I`;uNnbv2Eq;oP?TuvuTujLH&fTVm+|Uej$q=<^h$FELF{m-a61Xamd~i6Can|4fTE86FZq;t1t7B7U8$cZ2D-ATYi}ucpN{&5%ky| zcoT|QM@3`mjs9||J+@;#dOmqh-!uZWXkcS- z^@cT99D3Clcd@M^`Qz7!VJ3B&FYeaQ@Ud3NABO-QR=k*mS`?P=l&b2gY_@n-cMosLoXyfN&hQ^+ zzB;ciB&h2`qDtR|2dPXk$i6m&kUZS9G_=mc=#Cj(RjDhz-?C?Nf6eI<%cE)Pnkff~ zzio=U;l!OS?-5Xg_A&|jlCe}L*?*@gdKfPo8yM@>8jSd^Hf<%d`05w>NmVshMWX|G zZkLx1Xpr9{+t^G^yg$n2%SY-8$*2htcdzszmc7i(-E)`X0OQiOx`OFCCLiZBd0mdQ zx35#3rY2?uzqD%$2)=3RR7Y54G7ygW)das2DvTfZ;NZA2J?cX#o1~RTZXMlc_Qjgc z=9}2ca!b0}x;zaD10Gfp*r!WVVgHj)&T0EN^XJvjTwgS|`b%hJR{FRz?bAaL1soGj<4(ik&R0GDVxL@f=%Y{X9 z=1RFk;yD~Ezo$@xcU8X>1J=juV435f4nN!E^&kw#Y;mZ3{Q*SNv|^}~0YasYS84j{ zmOwjtBO&iyZ3zyLx%3}_i(M%ITi!byv38syx6e_BZ+YtD?|<>_gu)T(GGYp}G&xvt zM5!{s_e;SRO4jQ8Uq{lG7)5<@LxC==^lG`fFv=WV|I7 zmfY-x#Ei;&(b3*{V{CTj<1GQPJeKBr{)in7uErcTAHnvfkJ>nZr4e0el`MgnHG3kZ z8Jh0vGaG3CD*@2Jl9ftr4IM?amZ{N*h84M(&{e4u;Bf$c&?SdnQ^uX`_*mh5Z6rgL z9CBhVMizm8QpFPeZyj{<{o%c|Q^EA@ng0pUhGNls3R*h!0;=y2@3qMu77v8Koj*I8Geb1kYuud}kKP;p+CxrrK3+Buez4x&b=sNXa%1GUP$W?Ssl9gH<<0nZ4+MIb2~iSaVxM(3UOcD{^!s=Eja+ zS?aDe4=U1=JsU7F!Xd{X5hXU(g@i>7nVfn+1CEbLksqNt=`GKvI+3gx2GX%z8r?-S z#PC}kw*|!%B1CCgFGBHUx@?~LN2$k1oDpw@TtyDXx57oGEarA9TBMm845)fJY1so; zFn1GEW@y%c#Xmaa;ImTseAsJvKtwnNztn;r>L=;nY#nMj)^y=Ss5>{EWq1x0pj{f( z8+^=LO+tWI_8y6YI*ir4rR_()+^WE+W&R13lq@5g;e!Y;qyhyM8B3=h>O5_YJ_Uh8 zlE9_2x2M>%)+$rMAlCR8^jNKmoNwMQFPhx1`-B*d%(j&Un`G+0lYw6MA(?OS_W*7r zy58up$|{iMcNca+_#p=FrIcv)AQT1lH(6aelZ~w#?5SeO+-w_=&JBIS952((B1Ks` zWG1-SHFVIjC0j-yz@o=xD5}{qV9;hFc`Y3|cp|<~g-ALzfW(`aO`s7%% zh41>o9t8kCl%}g9BnOI})NeVuQ+BzlPjs_RA*lDYaR0|D%1QqY{goR=!3k|PL8pdQ^CU{O! z9#u(h(;ZBvVXKG2JQz0FUp^uVOJ0wma9tH!o!AcivZMd_0hVeVpgQ2`tim6$52Pr& ze?&wG5`anEs7iSQ1&-_z_(-(w13oHpzjCSyY%o$@g_9@7d=)X-jUoz~Z{P=TyDI*r zd%*5|0s`+~Ed3^q`-cYySJ%Vy*XR*hf*;6WAptnCDTRgSw795%z=A_}VGtCUdgw-G-D!b*(g6&}Ua$iCAa}{>#yVX*M1i85tO( z8pb8>>nnsY)1Rq|HKH9BD_)}_-WGlN=hgo*ss8mPJRJ28sOE1Qw}s{MO;x$ga6%+f z|JItgfta`ddBsaJObv(&QU4R(fn5XUEvEa=zyGr%xPxTUqV(dWJs8>b>#X-D(`$`8Q*>+e=GWHZ zKde5QECh6(@gG7W&P_pWrnyO#0_4=)nS|YVkKc?vqcbxl1|ln5j5qpJmi2-lf6hG~ z;!uDu#*sP87g3c$snZ>gs@JGS8EeDM4}Q9_SH3~H+>fWMqkg`6AR?koC`V~ewVlM~ z-6pZ;@&PY^WVa`9?2YkdDT<;Y*ZJ~XUF98QnYht@o^Dp`hC@wkpFm9v>O5=I5-l4? z1mSKMpU{}xKeuXffZ%jK_EFi!`|_={+@j=p-yc0rVX>3A0mYiL3(Y(P8`=eDbEkitNRMZW20y!@o&B)j^(Xo}|Oh*7qJ&5nfaQZ`mphn}j+Z9l4j16fh+6^f4sE1$#)Xf%UfSI)3RUZSoR z2IuXQUvCl2?zQYrt}K0@)nqP^YS~39_<5#vd~O5nr9I%hr>k5-JlbfBMaitQ7*F7L z*r>|$+|jWNN(k3l7xVL)*cG@TR}rzGy_$tO>&7JV)?32{E6XMnT~~mYa}P? z%zVA#oAXI$==9-py7J*6e}^juOuTKKPI&xTlB$>s1`MH2;0J<*aL!uew)o6kf3!Va z&p~F8b&x8+^^n)c93$!3k29jubdXg0!Y_9k70W5J*=yYgZ#X;f(t$&~t>F8Ht~lhB z{gfxSV?x<;WtqD~i_(0mny}WOFZ{I;bGMi1p(&6EKLJo+aDm9}jn%N*cA^0iO(mOQ zNN)oJ(+ktrjzj(7?W>3_zpTm10 zu0zQF$&oLe&a8M0hI*lliav>0Lqa)-!%kj4rMBrP&u5fXg*~Czjlto)Y2YV2Gv|I- zdRoJ{Sxd*)0$&4jDp8ogAYzWU%LLeDPW}Y}W?CKKg;drHC>g!!bc^2zMcUDX1@t0; z8{y7@KaVnMRO)`-8fO-gM=g{y1NnduW@f4LLXQ@m-1&|e#Qq!pv%cTeg# z4%lawhx|EE@`P`(<1(z1g5Zm2BwFax>DuB3(8^%?!5VAs0ql4;;IASFQj3FN5Ygue zYnU0iD8Tz)KUUk^DWYyBo!3XTXB^bTZC*9osxmKv(Z#yu{qmKo?-YaXo4bGRrt`K= zf^t?}_kKhzH`I6%EH|&*g1wDjw&dW`7qhp{_9_Y=V;}U+izF@he{@C!oZVAS8F5rB zXQWbkt-y-Udn|FP@6_asttqM>@n~3O66Ne!M)A za(f|W@xIuHt+RKW{W9`%zrp2-T4c!)j@t?2sIHu`(~^oF_{M#4V&rP|WG^^I{U*uM zMiR#~fH29KdYIHfALe!*k4o|u2qhS%(+WxkN-T4( zOxUk7JwLQ2Ef~1D374PUs)| zj0#3svqtCnndZ7L)Sge4Ye*o#@F=H*#;2oz+nHTem2Amyj_~E%A^a?{^4RM?3Go#Y z*i8X94VJ-Z%v8FevnP48#JBa-PZ_qApE~71hU zicPJUehKb8mfhIGtExL31R2lH+XRi`^bBIvyw3_a(iaX^Lt}IEGNF>}xDV1R;7y;q z7;``^9Oe#k6Y-0T_0+#f*u%y@Zq8R2*ZYh_q?+VD=iUsxy9o}CZKDleOv;gw zz2#*~5)F0OM(bK8Jho!+Mr*|cG-jZuoy&d%`Ds=}?)h{_b|wrnj}43HJ;V1&0yO_jOt zIYFYB3sI;JO3d6lu7@9_d0MGsQkCDo1RnjO^s)wwtT|li$jRtX1;A@FmjnG6>zk2R zzIjteEOH%$MU1`1`ZCRi0om(kaopgr{ho>Uhizwg_u%zOP=ruEByXnGr;Q5(&265C2;`nji$0^;-QowPzC-LW} za97WB{O;SkyD?t&t~|T8c%G|UR*@5uBef2n}%Qx=AKBANjsNOpOOuynTz&C&5DL)8ar z3)(5--%bf*XP@n0nTWG*moPo`*9LrZu;I86#PlJRSX6~~&&ye<#BhlgQr=nT&$hfq zhrv5ngPCsD>^nbnFKT}mBMb#%lQ~WG9fiYFXXW*I3!G z;lWq^Hf8k*v){TMKAzsR)PkX8PSMZXADtb3%3Jux=5+E;TvSn%?W-S_iO^E*=srs) z_MMW$z^v_FAOEhdpWpo0zh#Vk&OZ2q>|oqpc$PMjFXSw@tLqZE@uYg0!>?#VQ4*Qd zR|FTUHZdJxEI2fx6hGJ1o;oYBy0XguZgpIEAP>EQkVV}6CU~*YfzJm2B+K)jQH3Mk z(ic)KW$L%LeCe|ic#JZpo$&!o{K#9O#%gtkfa9V4!d1#=qF_v@Lb}?Me(}4_+SLiHvZq*bbr)$~&_rd{=WZO^FAoC38o<7Lc`m#ftbh9br4-)j&xO^a>{7}d9IkOtV#}rQa z@ca0z$YJOe$0|eeLRFz)(t-`Ftocdz7f$XSIVo*mQjoGVY) z&mO|?=1Hds&-S@F!QuEk8m#WIEFD0UTT!;qKBzY>IOoVR5o=+7d1m8aM&MHT-2zhv z0^-nK8X^80V7eR)8PhO{;uKT+H32|-*Md>r!Szpt)eSZo&gDEA2|aH{#aM#d;~pMn zfLkRZ;#pg=Ho?V?6QMw8`3MV;0R@}qXM4DvZ%{J#ft`XVujOlMSFJRfZIXBd2Tx4h z8G4c}EcD*Dh{5sn;D!a#h3)rh5-0G-7-2m}C7|kIOHpMrY+P&<4T23^8a(}h9bK+( z zms7V_BO_8;8ADkP+i|Mq+lPI!f9mA)dcX|oYn^HG3FutU&%tgkT=isD!q;d3F&i8w zQ(&U{bVuGXM?(9&vd$T4J^$f3eLwXJ4jQ94qdF&nnA^#3Z4*@GuPDr47!4cxG9E`{mNzp93(93KdEbRNa)2_pHB=wOMkm~xh7r?>71FVeR1Z~qL!yAtcF^;FS($W08c#4vEwGE#@ZaaRYl3kQ9W*9=8@GE zQ%)iP%}8c)>Asc|Y%NE72fs&@Va3K3v^ofdLn0s~c-Hd&lFm%LIm@5BI)UiP$`QL` z?L4Z2vFtO!5;Pb@86kuiM(~Af%aVzswxrY8ZJ_GiBbjU0PgG4Es)K&^e`~eB$_>oB zF9Q=Q0KE8E31p79V7(BQGz<~Rlp$L_SNdNJL1H7h7wrM}u*ggh^y zfEy`mrs0dpIlg+gB6=B{>{kU-3hLzRCKRF4!t8N*=E#igdD+d(MewG!Y1fHTbcUj` z5ceN{J}~e8!23?n@UZf2EDYVP`-kVy434(n-5X;+50S{~&-MY4=XC*TeFVj|v!Dvz z^aY1zmN`N<&d7&uQvERd>Or$zLo`7^Yu1}^!oZW9mm(%1xNHP*aH-&XNW-*yB!3VS zr8nDRk{A}~)OH2yWn^9b+n_@sJL04WG|W@DuPf=S#`Q3HfAQ`RYKQ-wU}JBER8CIo z-PEDmz>8=kw7$jOCJbS-;ssALG`OJ0L!=-Kb+z9J|d5Xis6X@=1D`WvyE zJ#7#YJTzqm9LrL8|BCoo+T8&SfGtM4oE*10l;__!mSiU(Fpt(PMc~hdTG^w4B>=n} z6sA2@u#?SPX!E7}le+2qgvo$?-nm=xcFBf>(%x%&u59i2lK$IBF~j)|q6Sh*im)}Z z+52fsqM-Z2@yoh{hZip~(7AYy;amFyzsGZSHLGxcdZCRc)UoHiLKaMqXzpCnu^*l{ z=YoyyTCcL_9UZ4UG=2eP!j}3TNS%6`I}b)ZvvGekzyh;`I=K6pvtD0so>*q6({zR`7^uWEe(U`I>RF%BR(?=SV z&Z<|~mu1oSAOD*eLdyTbpXcEvEWm=Oldc}}F?S8!^u7_^Rl^$(vBe~j)ZL_lk<0AG zY=Eceq8%0JE|Y5H=b4>fx!7#>nEL+y7!h=>!uHpVP}A}JgW>9=6a3Pco29#ClH?e!mpf8 zWW<|J39s9I7j7?_AMfv-I)lCW9maM)5MYg{%&+`Vd@(@gPakzRj*LP%`m;8rtX#O? ziI87LnDp+D=&S?s{w%*c{53?ikK>qpH;Ll&V|EDZ$?xG z7jtSOFyVLz#p_hE9}FZ*TJ}2vn0A~a2l+i%2~hkltE@?v;9S4;V4uJUtG?q;4JNxCXIncm!yap~*-H_EPa-Ng@A|?&WW{4$E^c;awt~1U)~{)wi9q3A zd@=hh@rjrV%j=Vo`yhzQV$djIgqsUvGxI3W??}66ZBBaiO!e`uvGO+=8N#DVmSEu# zgox^KH`RG%Rc&=6pQGC0u~)Re2Wvev z(|sd7)AL*+$!!so`FfJIh^ISj!6I>-n0-558IW|bL_}#=6Vrtj-&ex z@w~m5Be{8eM$SO7B+%96a}QIaHYWh>@e_mg+i}*$6YV@ErdTa^$_y#=!5me|?#}2{ zW*YrF4?!_?f%#L3eIe|#iWi5Ku69s-GJ6ZAS)*|n?ThiZws0PdXW7e9_(0W_;Bvki zxLd}*;$r^~p(DM`=k@2nJnj`Ig^RVwQejL5tEVGX>yWc}@f z@A`K+hj=9h9}#l|nRBMT{PAicG~LD6_c$djEW0YMyuI?S(a;3j>3TpiK~`bgV_#~R z?;oGm{5^3pgRhikC0sCq18)U=xTfg38^V4m%4>f+LPD0qhZS@XkyqV|+%$^|zNM?C zFfILrF0y^v?9g|k=yEsG479&3NjHY4=e1@ZF9{8B z#3xREf3rBKePY#m{->WN5Si>?po#!i=&S>TZ>mcB)1t6SQ8CnzTY6)Gvn{2PQ zu{X9DV4giptLwg9pOHi%77vb3?Yrm`DQ25I^X_5P5KrYc_Q%}(_MrAd6tBMA3>Q^5_HqAa_L10kW6ov;WrSf(swyh`@KvG65 z!wrOA8hG|#Rw|fd$$|vir1}lpE;U}5aF-h$ZImFS$EBshDXpQ(wnkrUYn&!Nuna4;-m3JbVSy3Mp5)*hbRCXPZlNAWoRe5qCshRl5tJZ;D zX`dXzEAV4SW00`i_~S8?N+Yb^AprVm-|$Um{b3do3LCXBj^bFOGiOF2Rd}nXh8=`u z%sv2i%hB)=d^Ka^3=DB!l+(nbYh=kuWw?R`G1UJQT@)Vx_1!%5rg~bd(J^Y_F|rY+ zvw3-E6=biE3|?>i>^A5pK4E_t% zRg%RI+7{WHI0`UPVyx;G)oJfi)_N(d+-4vEYmAr86`)GhBOnP>BQ@in6!9Ptai-xxW`Ri&{RhadX3MU!8%K z)rP%aeaKTy_6`oTL~+Hf2ZykVcpd95L8Mdov`5=Dtl{Q8q{pj|TftA89qX5%JKvX6 z0;%Td$v@Q1I=>G#y@oYbG@mnHjwC%U`tGRjHq0?~e_CJuK$HWu#~>LSP!XfyaBCxT znSCzf9#W|-iCJ{S2{liRczmtZOb)V*a;oZGf893X+*Zy zZ}YAFpD|}8K-jw+7;-ehAyvAmt}tkAe((9ltF;34WddDCF3}vGT;o^{_wYj4h3nY+ zFnHUz3#GZc-)HFtX6u383n#pF9XZuGUs^<(T% zIzPmoD2JEE_(=0SqX?}rV|Jzjz=3%RSxRcwe0ELQI2;MqS$6Kd6m5DsMYa$56mGKW zI`J|(#10}7mzld!GdsId!M$R@&29JuehLX+oR;%*VW#9Ak)l~K`n_#DIi_&xA;eV* zFL{a2=f%J8I;nr$4L&dYeo< z2=eyf#qssZXSC^&`~5n8 z!yLWOBgret=}F*&>hG5jpa&ZFK0<@-?hE2`C*L!v4*(L4{o9<@vy$W*=QG z7L#C-Qx-~F&>fv9>@wc#6dh5{aA{f=+bS_<|NojFHt0s^)`z64 zJ*oh0sO4y+sev-xjj+Zk5-Wl8d4Lva^mWLGXRVRT4>nwaZOH#-HF@tH^N*?cw>iai z;@^AlZ#zr31MJ^l|2>Z^mVe(B|9cZ&D)4|fVLnHfL6lHbfygr~&{(|Z!ei%@+%8^$GP3D%wZ%Gg#7a7hr)gh|3f6WXbH_);{T4n=UB)N(wsL#)i$K$S7K$!`#0`^h(n+j6H#U?t7RPwR7zOHnE{yf=aM&)qiq!Su8=8wdP)#`i6l`-GJS3 zFLTiQrod@@Ou5BfB&kmsiRNrH)N%8S8syfPfBN?EpMAS9Q-V2c9Pa9G2k}8q(b2WM z^W|rf2Ivax`t~o`v9(hle%mQ!#ZR>7KuF>YX(fpK3D#Y>MY~HkMBQ_(b`QM!$rsR6 zJvz6xQjW=xCxFu;9sDF^)^XE*B`h7#6jV!?s*zvgX6R!Ucr=@_SosF3Ow`yrx?%lF zi+ehV&I;hX&35!I!L(1xo3A$@%An2Xf+D(ytjR_?j@47REnM0>IOxW^5`|=N?2Os# z4#kLtpMscd@f^)zHw~XiFMorI*txnkx8w`H@Pj|t$Qg3VyJDmt_u(c0)8W%fX6H3A zk3s@_v^IHbMy?*m#k-f1M>2=)I>PTOq0Vlu1=SpbeJ=>w3<{F08Sd@Ej9>HgaMD^| zXzymfcy_=4mZ@*82Ts_Zv@d5EnHZC?W+2{z52`VTE7)HEJxp*RISSX@!>Xf51KIE^ zSP%QrD&=haxI%xHSM%{ry8m=z%sSx5na69j3yI1LFzpli)5LM_4yTK^qNk)yHnI-7 z_o1#>X>;_5M=xz27oZ*I)bCu9{nr>0f$*Euzqk$c4HHk|L)4}W-FkC$EI#ZbIwz;K zHl?mrphvweAu34yy0ZCv-s&iZ{72z&GWsKp{k?E^R|&5;?Nz-{Co>SwI>Trw$hJGY z?OgFQKkBV>%x>4p68m&7xh{9jqgX8WY9<&+TpZ1n?_D^sn9p03J6%k`=J?)E<-DE)Y@Dk5-3q*&yS-n)Ocy;Vb$HA-0X>{`2^RarK;$2qNC7q>qQZ~%tTOL|yD%5WQ=R7X56 z(mZUuQJu{G_)7)TIl0;Ch6N+mSI-{n-=x73QpB;;PHQYr6B-1JZPGjA!sBztKxyR^lx$kMiz1V zr{LFld@@6&5`Jpz%%8vyEG*+8ykQl&Ja%Ke0qQAibPpjn(#d!}#RS7=$ggK^gk+yx zsktYFXu3sfz4B5gFh+`|6Yvp}f`_NDqH-al!G9s)pF1UbAyS^zOMQr2xdLN#U0wgq z0d9Afkp{F~x<3oO4e?aAV}fi&!WD3K)4>2!8mW+pAwsmhZ~l~&^^B?xLme|lnI`em z4m%qjC)&p@dh!I8YnP_2JIdy3u)*6^Dw}T_&Ow^lCwg=llFV{7#vd$>wvNv~h-o}m zdt{I6M@^mLV>}=e*L-pt*+sHwb@N~0AS0p<8*B$bnOJ=OX?`NIV(~oQA#xSu91n$q z@SdI8EZQSgXDPS@hL9JXl#j6(CSRT`VHoWfuh`7?7MjU|!ty|}>G`{_p`ofZu4jxY zHgKE1F-jz!F|m_DTjLuDO;am)y?W45oGrb;j5>Bam-fMjSL5%k2HdGX4ezRBtQQav zX%lYORs55~axq7km}JHu2Z(=YfKNj17WdI)XjRElRtnWlDY4now9@oezs(TdV(`5{ zd3k?M?mm-ds2hv2XiXg{(i&6xmeIG;%E9g#eo95+?vY#}7&SFPx834ITr=u6Qs|}P zf_U$8+4o%*Eym3bdZLk+*YYS|;p?Ui3yQ|$viLoi=Htw=u+rH0>Ij=q0752j{9o-& z`dpJej&X`db!c}HTK7a_l|Rb8hlpmk!rjL14>9vCo^6kX`0!613ICb>Ri6?6T@dqT z9r-xT(kBlmx3t5y*L?ecd)FjYPjZ7HC90;iLiLk`ti3_5xezX8G*4kes!_XOx%&KF zk8qwhQdX#_rtLfZdwuU<#0ZYV?+3#k-RLCS3NdV?Ym}MoG-T;@ip+Ek^f|O?vTCL2LxR9% z&?q{JHdCgit2z<;+|AnI&|lBl93G$FJ(pQnPDBB|G{UnxG1)$2ifIJkw!N;#$;xIP z_a5!QxAyhLyrw@LSzJ4_VpyMav%AGeKgQZRCB<_|DmnP3KxkL+A-dJHDJHd?h(WzN z?=vw@jYEop6oF;sq^(zNAoX%joS#6n3#DD!v#i>t>_j%n$Afvy*-z4A)Ze zw5ExYi^&1}AlbP|C55dDuq%0be|Xraz3%1guSGRA6rAv4dW=y${S-Rij1=InMW?70 zgUd{~CPX^V?v2H@uJBnr2 z&@$HE;V@0Jef6I8L?i{nJG4>rIf;U;E4ZLF>B(s*$hL8G<&v7_ChkEN`DtUvrMr4H zkQ@ynB&h)-<-R;jgW*rzytD|9yX-xfkf>|siD9|CxbquCSH9e?BBNMf{ld>gpgCsd zWfq}K>&ua?&=AGozX;L*YcxdnZo}ZW_0=*lh%kQLZ$aM;7rh7-sBQ_#6($ga3r`$% zGq%<;a3Ub zixJdh*@-&|LX7lj`Zc?4(K+1xT--g&C)NzSZ{BdZ1(H{ym!X=dtxPHmkyLYfg#3Qq z_v!6(4vU@urH!-B&$D1&a?GowsA-LEBR0&=ep^&i*EKqe72J#+@5h?IqpNpezhdzu z)kxpk!8xl*g;;n(Dm!4ZU8dX+v~tpO=%hy{Pt$=GoN!R1$LQFqo<_9)`bhfy z@cmje*{2AHaua*ZOStl})jdeKW_IbbFQzszwVfRUkD|eXudzRpM3Uzb?|V-i*)8T% zNNQ_c#7TXI270mPmA-I(Prt|OV5*Q|Mj?s*uSR;fXC?^-h=p6+-FIidVBP(zktc&k zKz`c4BqQU0&N<-~WR*caW0&+n%T`GuR0bHKYq5521Pr7Vu8o+Hnz+zc0WncOb^To|Y=4~HmFDKZ_V7)Lzg)2XQBw#gmGG_x*MHCr__ zm}SB-S-I2)0lA2Tt7(SUk9?ahw3iR~-4mp;4Nn`f*;_^JF1~)NxJ`TyfxZg`T6Hzj z#8*@+9P=%seNjG!pEkt4AS#y|%Mu+yLc)mBSjspsJiQ4J!&jzrf*v0PIn-z5^y5fF z`>FWvapQf(0qga%#Hq#h%yBn~g0+Z#ZCly|r#%on8}fu2?Ux=f4NekMet19a0~}uL z(e|#z>XS+GT-!P*)p_QH`$-4~E}p3Fc*1jVxHAg_8=tSfaiJoy-_CA?2NCQ@v0ZeK zKIZWzp3%35b>ylq}&JwY(SG4s1i| zDvF&OkPlM1p5&H>5R}xX~#BqoyFwioSx1^i{R$ti;<0bNFqZj-I*~Yyq z11q}iAHoL6T)|X1rP!!TKlyUBnwYhr^lRSh6hnO%;PBz+%*dagN@e7mR(o4DTc9%> zw>-KT(GTyQ$*uI+XlZFwu?`ER3bjmS;Ce7DWOWf@%<_0rKEh`@58mn{(~;J3aE{V) zPjSp0kV1-(FAuAp^xJ|5uk1~jWY?^ChWR)b5f!?QF`6tbk1Mix0jcvrC9x8Pw95FV zMjPi zmeJeHtPhsq(v^`FSjj7LK>UFtAe2m@>(J{fRMPhW$EUDCGFL*_1K3@ygJ6-n>PkOBvbuB7dZs2o<4?Lh$0+t^*;(fG~{Q+&FchR4t5dY<#qC%1{-dM2C-pT)Nsbt8j~(PdBNb zZxgo|kKlbgAP)`+al^WN;J$GOL+)e#Z7n*#S#*vvFYH@hWG#{eKX*6DZ7M;eCgco_ z7C~o|IIg>Sko&H7G}ZGTMQVc}W!UiCy?xG{6~bt8badoNq*Ijwp|e=EX0HnWhg@Cd zOO@i?1(u^;EN$huS!eb{@Y`sx@0v4NXvrQyw!8m-_BugGH+V7 zj_W^S+=JUFnp#J!D+g2+5sC*xw0}@!Y`bbc zhgL2O9a}3p51NMGGh%xrGajvmh^!sy0cv8O^osiz z6h~?sJ$ye;i_DE`4i%+J)TE3nG(j0{Aoqp#`-71AE$AMEBu50g&iZ$7ER-(|5k4S4 zF(CzQG)?@~ITpVxCsvIty`!|WH>;7%e*h-<30_Pl9r9R1!E(%8fS`SF1d=GUr?#_q zQe5k(BltJUEAHpv%5ggzjzMzg2L4LRZTPpgbX~x6(R@N{Rku4XRWk?m=k-rk_IHm~ z_Q5z+N*A?E`zsmn>9Akrf`>C)-h-cbYkad!%pWij>k25bPisx9hL79z2%{bVf}`AM0Ifsi5tx+>o63x_2b(ypP?gQblO4H z3~?u|%?)PZqD%|n$@X%*)MoO0xB&~@%j7}#oe-2wy zCX#%*W?1Z&iYnyIO;sfx_hU&xCDX)A&ee?|scEl;!>+4^&ysv&BdHWq6|beecf;ds z6Vk7*w6$#P1O8r_XWFKlCo1|5dYNgS>^`n72~85#W{KI-h8K@*Aenx0=XSw<=NX4z zWc#wt8wzazXO6+%n$|fvojqx#fG{rd`fQKv*va9^;$agK`#b9}7s%B#TL=G15>B_f z7b;KL4!0;WO{pvHMee?VA`M2x^0Yu`YfOaqc64~u$MyFbwPmXv95M?(81r=20c>y< zSCNsE)!?#iTaCpAvgNd#yB#yx+s3VFtob5Qagis`?9Z^_&x%GiR>2+_0=#YZNGZ7= zp6AseD^|D7=q7>zOBsl3@}k|3iZHl`@#IR^J~6C@!##eVSxN-O-fc`2M$-bRa7rV; zVNXg@4`Lj#)XYy5v=}G9u}^;KrL#gNs6bBKk{YM3I&aTsi`gMr-=apP%XGi<;%w(G zLMl-K`o$7VEjEhM2F<4rCickduB20A2IsnpS`bvYN$TN-vx5QJj#;iLb!$30##?q8 zIjoKIH19O75__01Cru%-uVQEmALuGg;c(VvrmFo`*<8-nNf#W$P~`&M#_Febd2Ju; z@}?U_KLyR;kCP)9(M+*vT%{_)N?rucFKS#x{`oc!?*!g%6_OyHWqY{B-zMK)%{)IB zONriW#B7|$4en;qSTU_&LYji)0x3BZfRy;>)l5M=ghJK^{b1=z(R{rCG{gJWpF(cX zAXmY^>Yj@fw1pOBWs#_8THz)LX^HAUWt>iyNn=G)gxD5tcvV3|)V!wzEllB?ikgQ1 z0f4A^2oxjoG=DaaJ6}kN`Ix52G(owT8W#2Q+`00>f)|mZEBNC|5a9a_Gz=`pStRjj z-UY8rcI}?c99?e%fQ=OjBmA^kQ}~(|iF!G2Dy-`F;0u``9rEXC1IhdcW78GUt4s(6 zW|o=|ED#DIm{vf9pGW`_rraNN8Uc#<5Y%WPW9xg5z-?^2_Fy9`1dDZIm7Fw|v&?*k ztsOKgcJWE={l`^`i#?UcZK^*7rX(uT6uyC=&JUF?mZ;}D55XP+tz!7se+{TVono|3 zj`QGMCh)P#;Lj^ckTC(#yVx3G3;8XGl}7k;YZ~^L|DeRa_1@0$005@8%IUkjet*aI z;(32E3n4ybvUSs5Y=jinBOgu;0U1&ROyRvc1k@-9lz#J&5k_f*OSAr=kY~@>SX^_> zb05{iE&~>5e$3t6zcy^PHT{vuTZSuXa;^g ztWgf67osXyN0{V&|K$2-(EC^Q|FueMhI5237qk>6m<~nd`s(@`G|&|;VLtT>HHul+ z+NtRsZIi@tT@b52X0CR!_gp{nnwI>qgkBT!^th*Z?G1VE<%`lE?8R)nRe&Oi(vSA6 z99i>u4!sOX0sSEC-+&KUiRm53u%;d{-MD~*e|!)OXm40~z8B4M>2yfS`98_#-6lzK zO5WNjm#acO=bbrwf?_;iGL&eJ+rbw6vt8U}PT){C2F}1h`ixLNhY6-4lgB4LtLP)F z4O5}5`}5lXZ*P@=_9ThQD9f1yp!({Vd<83$Ah_-6m2%l1#k}8(IC@RrkRxFCa+))? zK;ObUy*BgyS(bP<1SA7F8pl9_b=!)y;CoE9UwOAqaO>~{_gk24PAI2&K&Mi=H^K1W zT0!#2f!@ZX)Erf8LTg9*D+`LYp@m-h#S-P^m#N@{MuTFIS6f8R?3R^2XZza#fh_po zHhG?%IxZG^R44RF-i_HB;F*_gknkdMVzbW+M$tF1`>F}JmWL(eTOKG$F1%1j`(1EC z$r_%u9HY`aLaRH;rFK4PN$21MGW7d(=%O)tV>TfiqbYY zy(LMW6~V5JzU={_m}al?mC|#sSVSNO6bg%3coII{+{Af5qT^sYjkiN3f2uG|S1*@~ zsg!5PL{37s%BN#y?<)uFJ33#PLZ}KD96a71GP+MFKJH!#MbWq|CsvY4_?r{6NI)~} zJsk1e+MYLf2 zV}mx;oJSgJ^mnyXA`@#T5={$fpZJ90$=L<`9!0jL8@GW0lSNvfyd;`A(Z+MjcAu70 zBTf-+wl>+@HIJD|UT>j>$! zyUEm|AT2%J(2fgEV>`Uj;ny7A_l)Nf_ptFNgH^Cju77~M1-e)bimZ3>v3XpnocECZ zT99^`z9y-xcC&POE?`! zz1W7N#c^>jGUQIB`l&GLa6sawi`hA4#!#bx?z)*m5*F0#vbH;2s|fnZu|)YEEu6_s z+|E3?f^Ik}W5yI}kS%)!7P`1H56`;xl@n3fa9^G|N8y~L#bPJ6NiB7JzAG5mn8mjy zpbmWz_>`cw@DV5YF%*RLaE-9NolsslO7m{L4{aioxPZJ!Ev=<{EhOC8Cab|Q%Y1xb z);a!6`xm*FPvk=blF_yU?>FM@9>{GEZO{CAZR&Qw_|QFfS~zTBXnpohPnv-+4^`nc z9R@~1`7xG&AN73y;cW3iIM8cH^%dD_LzuHL>M`My%Nc8|?bpt9AOB1g+3 z1{6=)7jSe%ghw>pKZVGyd;J#TTveyC_Ud_d#wr9hlVx_ermgRNYzd2bc!ERL&_JFc zz@uYLX!Il1AbJvKfNB2Q7hvpjdEBRk^QNqg8Lj||HtbF|qpizFh}Al5aF;(#*5B_V z%4m+x^8RbZWU3Ai|0kJ373z71tkM0au;yO{DO3;Fr(WN14>l>q(v8;D)XK8RnH{lXSk=_1u)g9rs- zR{59{RqQqF5DbHVNKoJs@tktzRLABLbMbVf{7b%S!LBdb*6p4?>loKSDP!lD==D^L zcH%2&;PHULe6Vx8DLT;2(jbtpvt_X5%qRQy5th8^tTw$OuRaRT(&YY<%TZ+Yy|)qe z9|3d){UnFwKDOad6SsvTPCiwQUbN%|m!zIVXa(65hI(=I^9ODZzwW&^D~RM{r>4=_ zt&%fkpjCBUOXJMaTVBS82f-lA_0jVr4A)HA>rJo9s6r0+TQaZdkFcE`?z~c*=(~ME zpVV4dsZ6LIJgdT#ia#@(8mG0e_aCe>o|wFhMXwvJMcZv&e&GFL>|z!jr+Ih&yZZ^) z;^wf#s#sPRi34uy!`L^p!HB`SRbwRCAKMim{6SQ7HV$C>DYnwWbW~@sx>Aov*daOg z#l*raTB}_k{%36Cu#qA#8#Jnrz1Mx`HG(7Tt--x^#BrhJ7Ivp_(i)~#`!gd{49-`l06D68 zcp|RJprSWXTfBAo24g(p=zO!iE!$kHsa1B*kYMPqu$tV3i1d-`8+yEbVTi?{97pB6 znMUr0NKJR>5{URgFf(y9)DzmIQ{v2AJfnR9R?Fl(AnAUavCG*d7wEI&{m3YW58v(m zV*_v!)wA11^-24t3Jn6mY^5ja17~tI@#0MHl|Q*56dKFwG)Jnd8SR|XxS)OqE%S}p z0mfvdnOm1dm2#rS(^|)l)338ygzMyLI$vX#Jn~)rNoGh~{sazzu;ZZ8e7n+WL+>SU ztmA&gyF=82a@w+u7%F8fPP4@ur=@ASEV5~WJnY_gDj2_Pa&H?XyyXRj zkl*-Dr!few2@zV{B`m3Jd9#I}?GeaRDxKRsj;U)v!#cZ#5!_)zMCCn4hg5g(n2_1+ zRr_Gq-_oh$r5Tadrm~r>!mGs~H82F+z*q<0wF^y4&SegtLM%SY+xfi~rx#*X-ezgv zVs1T+czDR(G`zSbjJ#o+7Udt%m>{F9Lj7Q=A5+;KQ6I>ojN+V{-7(G+60*sQsf(t*htkPll%B@>gWA+K@#!iK?s)J?uv2KQ}md^P6Dvz-eiY)S1 z%B|quYnW46>FQX^t!K2UDA*tP*I%3ce;yWh07s51_lD9&>)%S(5}>BF@QuDqurUTc zn13HyUCSt69M2>srZD`msP%pFdJ)ud_p`fKb(DTp&-v$MX~bX{7lGwO3^Nb25Xcja zTAq$r{s&BEr>FncJn)XnCc*4XZ{rz@umzcl`mC;cO|qUktn{9tq}IVFzLa5n06JF}tXVwgV(OBmT#^|mZ`j=>&$t?S|O<32?FXZYjwMrAR z=Rrew+~km^@GmwH1MSjKJZot5Itj`Qd_OVhq;EqZy_13OU!Fk;@g7~J-0N@6R_zD2r8;}x{tO85ctN+d!sVz#s;3^BDCF|%A#au)oXq~n(rm{A&0 z<3S?dDvk98*rPOvLOx3m)pgIzV5h`TGE4WbjO{vZ*zymSdzCNO~8k&sWqrN_62jGaB=zBPKQE=Py`eXs#& zeqUW?|NSB5ZK&PF4qkL0kuG&8#WKyek1P2LCEwG--mL-6od`RaNY7xUq%_-R*ZWz5 zH;dbU^xj*id)jb8wf3CKW*OQL*0pu(e-lvhL29ZsGlhd(V7aMqQn`Z*wP933bg?t9 z&Td{P;Mk%_SvN|t@t3G>$XGvh_BW1$Gf*Nt_~<3@`Mb)XV~}`Y=2$ro#oF}vuL<6` ztTWs1e11^%)MSCPMxaFl7Hv{@I{yqIQT@SNBr)Ht<1v zO24|LdE!+E+a-o1X4J12gA5zxoD;&1E!iO1%XAU6EgY}VQ^RI{`mtdQ_(|QABy?jA z*>Y95M__(0MST?c!@GK}fvMqbN8z?9{o2;X?)i1z%2yiR7>p1i3T))d#?pP4W|t%C zRTq5Ufy$KO0+i63uZ3K{^7K$HbwX11gx|aaHZ`lIKQJdL!jI!1MVLx`epv&n@lSij zm&xep|K{^Jhr%`S!hQbz#65XZ$OQ>hK5I6r(P+6%dgcd{s#kRKhEyJ!l(Nul=Hm6q z=C{?)`tg+lvl)KDctJr$Xy7rjaX?hmoldRM@bb1J^3P#*v_=4R;98)*oucw_ z&rzcidIAlTc)^>5H5n@Ld+}-hHcBdbjhcK0aH8i@U z8!n_!lu#@@A;Cih@um;->RHYD~E}xCjN<&ZMw7QVj#U$YcN^3QKTcKwcj7*f7Nzqa0&4b&66A zh$pdG(zbOqa-V+az2|$_w~)ju+Sv0^=3W1s%y|AD0O`)jwz(pp;j@UE`zye+EzGcM z@A@O<77~QOnF(Fb5+rO&CZa&sO%O@NVv~C{Egz{C1#2i9OF?d&0Ux#?0gD*`1)t=| zCcqs>Htp>zo-$_lJ=hQsCrAC}Q?^Qcx8Thji&<&I6?VM;^bmYFT88@G+jjwE0qN99 z!|{@zFq1grIJyrQ`;6ir0wmDGm%`PzK*D1196^ibKGZ) zSs$r0Y;juTLEZX|O1cY&_>=i5Z^F?cxsx4ju?OIZ24#neuOfi*Z&IsaA1w`)I3jt(( z#6AgazR!N980nCDllrsPv%(jH+`!Uyo(uwQbsxx9Vacn!Wiy2Y;*#p zJd#?Zs3Cl;5fbb>M_y4_%L!<43S5wTXs&JG^bk`(7-6?%dS-EWDeH?RS8bBXG8?%j z_nb_cWc$J!yD8Z+YLsKtmDk%6>C_sq6> zZww+BI$n0_QZrZ-D-#Ecx2m}W5lfN9ZGn#5$Omno!eT=E;7BC^wLj!WYekgZh&<#|3=)? zo&pAIRuk9VjpqrqVSB`!u$by_z0PkOY}U38O)rYEK4K4m`#v@8#*SW`_c0@LZc?LX zjFS)Wr@h$@wf_3GvrK1##ww|{Lv>`Ou+^m&4vQ5v6(q3m0E~@f`&RUsQjJSgPL3TW zSI_;n+~i}U?}I2_U9tZ|1digVf@>L2JKvY$!i<#GcqAQ1NsC9;$A;ZYs27RQHLCjp z8=I3t`dwi?y^dtmLYJn|Az#maa`$OIgi)8S-FRI7zOiDD-i-k!;#n#jbq zDrNKY!zLtnOgVwxe4YrVJ^mB`iFkQrAOH=)Ge`|dbai*vkSaPSKfo~8{TAmvoAANO zsgWsgWctJc4=;PtA3fF(rqIcOW;&+U<_^kSs_otqU@t%^HGzyybe!Aj6MC@tkw6FD zVIXavX~|~qx+ozXSI-p~(8TS#W;kF9FsPBq8`{=e2*x=$j+vG>+@t_6@_#yNA{MQ4 zvt6cN^y)t_PxjW{p99XdO%PnVVG$9<+9t8raWjQ=XQ4a_kw*rhT-uu^8tpR{_W5_g zk&}P?`0<=M?53gCZX0`9$WD1ZCvKu>XfH~wye%j_l6h(+rqmK40utVOIPXFF^PT^g znec!)V;`*;2PdJ>EQc)7nZexz(RJkBLmNPiVUY;XtKGZ9L5hX1OOby$T=RhjfI$@Y zA!--Im~EKj^?60(X$lCQthLk+gi!ARq5&gdQ3?rx35u}2E5dxr0lWyxX;2cO5LjD! zh_arZCxKEbxmx!Ylioe}Zr6ExsxA@)23UFq%4P+!E%t8+ zr*ChOE_?K1{$SV`V@auXo%Sqxce10*3sh(9v6y7SBqojf!67OYJ<|v=VZ7M=B}2Gy z>#Qn9580APL~R2u&@^053&;rvRTM+9BhVq$f#Sg^cKEI0OenE+l^Pl?8aiN4m?8(5 z>PWGw@G424$e>lB*2#{jS$F1yWw{RzXqNtn^I&Ia5GdAT=%iDeKuDx}cDr^>dAW#4 zMA1XuhoK%lEE98bg_k$YZIdDxrSdZS(D|zpfBmp$V5z5-pp?pV$*An z#ET%NYAHp=ai5h=#eB-9Kqn@vxZniOxrV-QIZv}DZ_NRa2-kZ%^7BD3tkO2>UH%+V zY5?Y~3lL|q-wq8=djPJaN%ddZ>vSEPqkbE(I}}H4bvY$Bzf@S^P6&!fW*TN-W@Z*u zga`{4iL!Z>mC^dNeG>s*1r3-mCr?Xj*xgDRy?6|WYPC6NN{s#YYMHayvhMZS+8ONC zt>n_O`*-#az+ogcdW85siSHApCIk|iu}g@__IvR`W7g#Z)^DPeVks#kD2Zk4GP8_m z0MSK<2Zwf?sK;AV)uLwOm>PSZjW>5b8j9jYk)C(s{Utf6(O|N(VLkVp2-jSirqX@);-ig}t(&V4JErY+pOMu*8ojlsibqPlu+5 zPtUFg4{Auntwr_?O7!gAD(Rf+pJm;YN5~cRr5oq8Mr^jl)D_WqrXQL|ci)07BXM21 zeix)+o}W@!ps^+iKv*ZYl91sgwdvZ$R>Jll`VNNf3DT=OYZJzgzWn+YF&yPI`-7mD z&w*4NJ)5$oTZPT~*H*Ju`lfhbfqAi;ERS`9U-`I%AM3C z*JWoE1nL%|q(kKrEpw`Cvx4PAPpn9zS@4+&kdd6v3_3YNY4M6$n^Hg|S^AFhKh#8j zSFV+h_LY8O#yCAGpf)eqEgyalNuH-_(7=Ue*PC+`Hp9F`4@}tBW%zx>G}ThwD>ivE zk2D@l_PA6`VS%QK~J@jB77Aq340^VC_geI#BCyU_-fIURH(Wa*pCxZQ=UjsM9!# z39?s^qpUprTYs$kR_E%6BqyYW(fPRH z_@y?LJ;}d}&MH;a+1cQ;d4x*J63|K$yD%|qsM9=^-5lARV&vJ%@XGRhj|;xnM&`f3 zx!x42mNMSOlN=T2+eFqLCaJA@r+@yiCuMGjAosf=y(YoR69BmAdxVsS0=y}G2U7bCuG z#z+e4wV8ZAf$3FVc*|VaQ-9YFF;p6SwpUV}>3#(hNXX4?viBXF9viO$!Zg(N{qO!n zsxHlSj=o-k&;u-D@{%CIN>h>7bif=t6jmFH9-JKMX~Ci9Tsc!&8Xw>tZ^?gZ4xNTX z9spHtZLLYn1JcXrEjgN#kV_T(C%9k0;7IwNcnO4Jp>(z`@Z!UGcL$K>DfK z-^L!EQ790Df<6nOXu;*`*l&#xp1zQVLr~M=Q;jK1KVwi_x3h8?Vs@wRY__z38D)bX z@$ISFptuLrJ4R|zRv8*}(tOu64Enwj-|krI)fJJDUcW(es#$h^P0de`OP0BnvX5kT zj;IGiG_`)KrtD(h5liSkcbzy-J2`&Rthh;4B6Wuy;CJzdLN^FF<^v763b7JXiumlm zffd7oLWL&D)Nfk(jRytgC)ZR~)se2tG$clvF`j3JZ(NtgmuTrIucH(uBNZ=BB~hff zLxDlbI;#e1MJrYps7aL{!XQFPP;3LrRQAQ<8_;0tk$qC)bc6k(0&eRLTd*J{meRXZ zRoYSni2}Ye8IuKy4a$RgCACy2!d&DNwuK$2pmzl_MWaWJw)^5mB?J0Xkrp2RUy_M73uZ{ zEp2G?TWQ~CJlgk(+YWdfl&rn3^nTsi0_t&T7|K*+2Dpe)G#477V*07vg>T!Pqlcx1 z#qoF&lU-}*Vke72DS_cIl&<07@X5`?(Woq#(Xn$C$}(1)k-zkj0NKb7z-hI3Pevim zq(2p#umHV$8M96kvg!(>m%z1T^t2`@t3nM(mpjT6&(sZirxyAi<+C?Ll|n~qdySi3AuRG6eume6yEbNuJ$Lnj++IC zz~T{0ZO~X5T8{$5i9~^{0>E#Zh_E!(8E=x5$z&y665O9%^eCsJ1SBQ;$7@%zTORS*Nv6O)(Fo<~N9Nz>!DJ zO!%tUX3bSw85b@W%ZiOg@2qryxZ#3_7lVX`k0(}#$zmXj_(g{c{V#37N`q4rSh!tQ z(r2~h1gMOs^PHzA-~6ySjSH*F!J=WTpkvytTdSmq-xV7@#Rrl`piVHE$gLR*=vNS~ z?vrum9+L+ND%L5oRs^wpi(t-)PYor*O8h`$uOCYhDU0#oiV)D0Jp3hy#+QPC7D0If zQ1m`TFfzKh=AMeVsSvNKH+1^6sJ_{E)iv~09<#J0Rj~;%gNwiA=LEIL0x*6%|DaMI zdL23vVvf)eb~7VQQ%jYn5hNquw#XdZDy77wz-#S{&mk8xwe>w$LocfeW5JQ@Hml#L zD(TA|z$Hdu24o33tJdmH6F#fpv9ihn4RV^>kW$bnBzjC!#EUF}K;^B*jM!KvtVDS% z+YiX7=qNF9V`(K>Eul*dK{WCHW%c+~+yfi@p>YM$Ii>hb)&r^<{ulL&!lvB6S3|sUN|Sabrs*ba#5dA^zw^ zaio@!|lEnqyR17k^YU9|Ret=_p z|4sSO=`f^5esV-(xuPz*1eyL&ld;4U90N8w{;Wkcq_xxh+P;W2inwr*_i_bx*h!m! ztF+jNm=8a|iAmVo)iiAZ(*j`&B2;WWwjgQxVtFpX8lAZc~gnThM>M7 zHFN_#RtAPyc+Avc6N5f3c?!s1nS58Qp;_g)i39x6CD+&t!(x0z>4?cIxJnrB$TA4;+ZpHj zXmU51Y9gi0ln@gvC2Eb?J1!-FiHmpy#AN7rdfcT7>)hFCQIx6nc4QaaL4`kDPXKJ4 zBs~Grrc@);>=;Z%c)0i!0DMXddi@DVykczJdtfXkVt>*XR~a|`i?_qd@ya^+ADO~h z9E=rIOa)XhBe8>ADaMB0&P`_dX_8PABCw!KDakJhd*ytn zJg16HYLt(JBqH%53;(w-fVdQKkxYeen}1bavcxX0^b!qBfKX&4HIw@-@e$ zIw8NQJZ;){c4xqZdtV~Xn=L2C&%Y5Px8Src4f*68aoQ9?KDWnp zlK_(mLSBRm*Gx~|n9*H9{{KuH+w5#?!Q;l{DWhm%!Uso3+r8CGmJp%|gBrE_lA%AM z(*KY$DgNkz|3k8kkM#W?Qv^o8$tOW<_5%p*N{@c7w~bDLzrV+nwF1|@ zSp_9o`;+bg8;H&fgpb`CR+xgnffVuu(8GNmT{2}DqNc4={`nLx*YDHx<%P4u*5Cih zBqZ^BP1*5kZ%BDTP)JyqE3xIguIq7Wlft#Bny8}G^6Pkt;M0y1<{wPl{MKamgGF_d z_XR>|`jfs_vnuBVS6@r>oh z|9fFQ}S4t8IE+Li;QiXk{Hip);wQJYGC7k ze`K1ImFie|oC9=qfys*r@`}c7b3He}lLcy#i{zln^22XUj%K}oN3&911Jd_LI(+x? z@vfLUlNz(-lOPL)mw6vkyer^6udA*cZ_02ehOx%dwwNqza+S*G;GlPZJB$yKQ@~%b zC~dOEz%%yXR-afK#VjD|$S!i8sS>;$bSI7{tTe96ok(Y;IpY5O`3gfeT)+MOu(i(e zYl=r&fYISu?<0nBwkjm1KkW0s64coq)i}*x;8zgg4H+!F&q2?->3WY7r#JP4mcg`d zBLh1FW;vGOl*<-4YPleDT(YX%^w3ewG~f48iBx}cT9)6=sN8LQF<`egV@Iozc>SIe zUAQ}c!!=LzhNGtY`s26(9F>O|^t10vzx1*nRa~)|)O9?gdqj3nv-%e68AY+z=*lf}Z+{VFndw{Qyfr>l z>d^a+M(T6N#;0gmeXbdFJ-^AO!xkm-`bW_~aDJdslEcS;zL4sf~{AO>kz&uMw zLal*d%8|UtZd4*M)h~Csb^VLo!`>AsljF2Cmm$o7(~BK#^lEF)VvxUIet#bN^{Kxx zCAQ*^Ytma{;Q4CXX-k3Y`;C9}zGe_BPhY>GEMbPh(jbrsE0&VI z#c-$ZRZEVqW!*RVn%-!8ZH2kF6V^5ACvtF1yW)F5b1^&G1jh&M z&q2e#dVM1bC2;dLRi$m$7b3ie+pG1uV>GL@>rdulbyW_wJW7Wa`ZWdID7<;an zonCgn&~R@I_ilE_gSw<%j*wrD4y0$ZwXW}oYJRY{f4>ZTXZcI3Asl-R!Jd5$d!K?{ zrOoWxGhpeq;cG543VJ5n+?~ENJPLrbkY2vNRX-CV+-_9TZj6hN&Xn1QST9~fOA5i~ zM(3MbldrM0V1^6@z|CV~@5ns-h>EX3e006)q^5hj%^ic*FQc&t>lxtR=a28#uboGV zd*HR%n_H*g?A41P+zEmVw_QzR(bXFSk=_UD;W>tnWCX=iBh{?cW{W49{~0DDtA$cp zEU^*8i&nz#8~P_Sr>%s7x3AJI_sP@txiPqxTRrcOX!iW^0o&su`^wW!CJ(P7TN$h# zNprkHM})^m9|ZEBxi()xDISvQHo2kwQoy*O{|eCVPEL#Ir}TtP8!deOF#&%;Z~Wy( zW6Dozv{3kn^>-OCY?LYo)$lh`lX=$hl_Qk$iz@3ZU~(8(=&J}xHk&8&C38uk!5HYQ6`Ur ze3LY}B2#pQtNvq{0~{KNSKbJ(7be=S=PZ5VXH`zf37^>g|N7a2R+igE2F~Pr``eKj z*zKBi)XdlF9r3Vk!aQ0pXw*3hh*(52)XJewLh<2xrq*-N8yh+@?{eU5^jw9iGPpkg zDqBV#pN)mbwr_H>jQUmwm%-!QUB+$EixcIhDH<`URPT3#-2A{sItKTT{Pi`=f>&~^ z%>a8@&_+>I{=vOK{^x2wdb$P>F0lKS@3if{3up5Bnm)D!X!ii_K!JMKzWzYOnBRyBM+UR<6B zzPQW>6&)(>j9vt)_(N$MH=xvNmi=X(zX_vVX7>?3)Liwc!ghpErdwQ7ztpnk$4iIM zFOq)QY)^CiWvwbi`}Jc9kVNoqfF&Ts{^)F9i2AAZX4Ls|d(jAwI*oLLg0PU;(faKg zS=Y#X=e8W)qxn>;%n9lF#}|tZmnG z)Y{uMT8*BTB2-c$YJ=mA75~$2W=v=!hz;$c|{o$3ASp0LLfdoT;g1BQJ#_~CiVyFTjf?ak$rT=orGXnd% z)Z&6}uBveekO~y!Rxft%u&HD}gFhVlVcu%0@UuzjLmw`8Gb$T@L*$5Mp02vv8R@JI z8%pE3Ow3h!~1RzX`0zM7Q3>^gSaxJ zvlykBX-|&>c{=vH#+yEJ++gUN1EmoHfx%d!`y@7Smaw;QihLebBwBsFd#;^)@;SV8 zMP|>}74^`)FTWf%nqvE5u+w?!P2NzDrie45@9W$rl=!2H_biAs6&thon&HYZW^dx| zi7ifBUL~47mMPLp?n-!NqrZdlju8fT-S(M>DW-ocN=DzUrv4f-*}Z&}gxio~SO3c!9Ktz^1M+l%&@M4|E zV?Bh)^LbagSlO{S)-*N?szj9e4H{QpyIa1A6BtKW_R+2{7VveP1vYQKA|HTJ`aI-_ zWrMtC2osvt2Ae3LgB^~4d*ugjWeB!!N=SAmdRsQB11w3o)OnSZ(QXC;1+Hd{rwVcb zh~l^8_C_d;x7oGs;5BlN*-y7>UG4Dn@PWeLsFmdT?|st`d;ko|oqSDwuRdd@m9V0|xG4nQ{wxF`_$6tLYq5hV)5W%vDos z;AMXX3=ca#%am{ge6k7~h%=)VE0*BS*&Yj_B1?#6w8Bm%BHR65yc=NBA~nt8G+1gA zCwbq%{8utpIeo-*yQqwuT1sio&0&*glpA~6k!~mh(eVz_>KH`C&+A;m_P!XYo}c1k((Y z7}EEsoSH`}?k+NSJ3w14!RZ{SE-!41{Gr3g&YG~&Pq1rF0&`^7ohUzUHhdvuM({YM+K{F%N$`B+9N}Z zI%_7>^*AVX*2rS6s*Z7K-PM|tTxz_nzf|+%ul4SfnfQ7f%++3jc+Zm$* z>fa<@sgH(VCaU&0Wc@my=~;!e2C!P@^Xd*-mACD_jA)KPB5NW;u3=mmAX9tx@|a02QYB z?Yq{3J;msXk@ej(H}k7OLm}UW%hN2}x5h1_X;UirY4O)5+xF~V&NB|pp_-0NNxRNz z*BUSAewNHZfYBf@{UX--GO(CU-B+Be9avqT3kb|^)LG}a(!-8SFS>5f-VVp&<7iV# zZZHeQPuq8CTR{UMpVYVqhNUmJ63SaJsai5I%^8jD9OZ%iVuL-Q!5jS46v z2jKViv;x7lye6iUnFCqxo&o08m*go!S&QVezze0I`%~*sXQ7xFJUY9@ruEqL6M#bDviM^M{I#mVBw}w_<1l7-3vL;i-*g`=ciA z1_v+ZhQU)c6f}#hAW#9QSyLX7-&gxs1A9?d%J=&KQEmu%+}#{Oj646RLR<=q&0ZBx zV4iz+cL2Fxeh+jl=;d?t%}kzr$jACOEq3B(P zBa!)PaW}u9A#r4b=h_(0kr;OI`ixl0c@1K7bfjFS)!aHj&rmLxm60^?T3>26b7lTW z7JndV;G%^b5tP;!XIU+1%}=Ib>Wh()k^P$)DRvyWy<~6z{(yClb=rQ%7;n+VwmBBP zCfRboice%#M$Oq|Xpd3H?%^|N?0T6wz}Jf_1KtBIyvRZJn3G5QW+qnN^)FL(uI^NP zilYv;k+qePC*NkSp&YbSSsENT7Tg@*Zbd($0T*?wXq`;#gU84?^}i%dj^=QG|Q>9m#I)$1EDD(JHTG5~IHbe7=MWR`Y4v5z= zm)x|moqb=5AUi2j&nPSh%Y#eW=2DMTAayU7sN8haf#Q<2@01hd&&_jSke@ILV{*vc zfT)4}QIciDgkz;{T=$E)?7}VnYC&Lbekg$fpf<9FtE=|9(P_tTOti-t8Ob@l_~A&5 z29w?oGo#@p6v?YLFjT*1b&PBCh2NdQVKmpt;;hFmJ0SN*)gWBU&0v)&p|+8Dy=!PL z3&t(Z6q+3JG@D&BV%ucWzRwUl_M8$)MRzxge-;U!LanK9FQ zXx)t;n24U=l^o6WPpPzWdqW|#g6=cDugJjQo8Y#Rm5703sev2&4k`Y>(_!c} zv}~7tgk4g(YFtj&9?rg@0?xsy;=58>FXtj^yUC_DWHZg#;2jZznE?&~wKpcxw_5)r zWa+FkW{I8o5lo$d43@Q-?$#HQ8Sq3Fz#c9BZZS0-TV4BbxMXMvBY}XbtfD>H!6GVv?*JmgLI#GF5)-P4jy*clFCX`~GTiK7u;>`xv;kw+L9jGQpm7XOBi zdhvFjSQn~(xo_ny-;jTiVc!J**3+ZJn;Sjd5l#<&CDc-2ks3$68toN{oO2V})G>;s z5WioGSbGJM-K^cy2h>-mh@EQGbb}ZmJa&F8l2f&BS=tZYF4J8|dP{5a6(2WL*`q zT@I;gL<`&_oO+|U8l@l?XO{(zklv3I&R;W>Lu>CdX=zSMq0F$M(Ml{@V@|R zETl@|A<1e2O1`G7&6lo%ENkOQ$-8yIyGLbk#+yC3^AvX%q`@`@jU~P0{_OmgbAFC^ z!un=BS?fgs(rOq}^bE`4u_*3#FnA{P+YLBb_u^OQz}2SbZX86U$z`FODv{Hbo&vT*4tWqaUSkxw*H;C3J${jfn9U z+^yz4vI~~SgEkT;-xOxXjAx#( z((+$Bwj{!3;l2>wFd*s76CR|LVA=uSQ8mT3sNvDU8lWg3=6(q=($xuc@>u3Z_OS|?NRhs9y|jU<5L4rwr-u zSt=E(=#+nzrg`48oY!BeWV^CpjyjYsdop|+`^J5>nFh^Mz~mM~qwKyLN<$b1qiM#D z2g#+LI#j&of2l5>Am=|#cZ14c4!1OT18oT+S6e`{>ksDn{?vHPHNf1zQlv>1(L6G_ zKW?Y}iYiJ3Z>NI8>WOUP9%!ojMIzFJPc( zx!o7MaT!H@S7sbDvAh^}Rug$g=xn_-F@846-vA31+kHmXHkcmQ_4@lnp^)u~3)M*= z2`32xcyDWv2ABX|u^`$ic16P06$1{I18mCgccT$lk5Rk+^%Bd@B95p7tc-E!^YRb80e1_3{zj~23gq4vi+>E9=v!uy*C`W{X172V(jAx4a1gk zJ~JQ{dBB!DaQEHbm(6s&y}g^0BUDsKpfL2#STEf;h%Ae3tk!}SyyC^`H>6(RbC%~Z z%@)34I`uvqgUP*|X(DRatl_+suHp46SAQ=ON+{u!x9Fw!6~;iCnvhN@am zhS)2rz>0rc{OGv0&E^kwy1$0<+KK}He|NBu_T!>7c+$+r{QIgjW!b={%xv%Jclzp7<{Cj1)8GBA!^Oo;7V9S@hF}*mM}&=6Ax~ z(99KbyrjLuj@?J$q?fs_DiwGrJxP!QrC?%HlTz?z_W6{RY$6U5&%{)@c4xx)*`K9m zWgNiz$}`h(w&ps7u*OeDx3Su{b9h+)v->O2MXo}Oeld+xAsqu5r!4NSbfXSj94v8vG|}WtU-rqK6Rl-+C=O-%nIDkvKl_pN(nt%uagyR!A{)f6#&OA5thvxurxfmGNelK9aUWk&2X1d-NH;52X z{I&mwk6@^6hl~pyvE@+3W?d3|fv)698&Ugy46@4!gQ=qQDcUSqG^oZ0MpIpe^P@w+ zF}>EgE%`V`kD8TXavWhRK0-K@K$rrf(&U1y+iip0#YgT)4uq(;SncR}qt2*E8KK_O zlidTpA%~zHO*?{;6m^a?Rzsd#7^M-hw{ZKh)(;WI9{xzwmcqErJec3Uj*MBS5EPnD zU>*o}W15dT^mxq=S@3?X2yxQ8MgQMT3D#`4tg2x5#mwtArk5WIDT+TST)bRkBdHM& z`f9%`2jmYbq$5UNa;P_@bS5pdScpItIij2LmCu``yg`=uq%(<$tR}jN?n>#kg60U zE4q@E>Id;)%)hjGPN%*`2}jSE#qa5 zBF3Prs$}P*+qfG$1qoB270W651E#hZfcUn_8*2suvz7M?l!j7a#n%B3o<`@QYKJ)NvsHwV&pFaW=S^~SdYu)j(BmZUbiVP#+ zbT}fg-aU>l2>(9#6)(MdW29-^?kpD`8x;M)K>4zQ;9 zxD%GRznqGWRV+3 zr%_x{^>72zTL*T7`!#r}@>qfxP3n>2#mZ1f|LZCk0rY8NN!NgbB@I-AkEW!#>G$nz z0n>@#Cx$D->;!xa%+q5nDV_+^jD_}c$QNQv&Rj$WEzWW7A9S>7`iyh3JlNkNeC?#M z?De5@l|B9eMX6gvcKROfi(EgB8(D3*;>}Cpm@e^-A509EIs7Q?Bx78K6pl@|ScO)O zS+sLiKYoDMnmi0SY-M{&zjSx>#6X1DL>6U!(#4W~vQWVU4*k>0#dCu^vvS)huRk>_ zlHVvBiVQgS8RoiO(ONuA(yWV3Acq>1T8q@Ok`?|m8`IK~9?}7%6iecJOrU_$*gxP` zSK6&m^$9G@Wh{6B_(~tkF{&-!YDr5h=e*nP-B=h*hABO~(YHY0y5j(DT~|ULmaU~= z1L>999PoA1?-E|k1h?{(EEq1u^USpN3embv<#JngK`y8?d}e`+n~@Qt&-~Z?7wvCm z0W5i99oyt9Im1rDwA=X0o^1XvXXuf16S8HUwgg0fs{3a!@&(l*wZ5lZn7IZb)7Vyg zCAchZFGe=@vdEO2M(uc7}cD{ak)CQXW{B%H&PVgls9cs@^ok*nFW=P{NuDhey6aAeARrt^gO z$?k4wNYZrN#7?jPLK@Q8?XjO`!*{?ej>dwG4KyR)M-iP42Ra;BEZMooT*L;@@%vWu zs6vnTs!lQ|tRz52fz@K|kzKI$F|jGiLr^LyGQnvF*9J*i2gSoZ`hEAXw0|^Ts9y*lVY_^%iu7tAc3u`=KZdMKU6p0 zvZF27sI{Bkd7zSiGFT?za|*`Qx)~!tJGby#a#Bu8o1B>RM{jSeL|=MR=*)^ZI8#=EPC*oE zQYTzxaPT4=!k8KA-Y4Y-;rrDlvW*uDna031M2!y@gr)N36h>RzrdhG1To4h>y$N%o zN$-cTZ7O_UG(n|h+snVj zg2|Cspdq~K=y&4VHkg#3DSoCOP(imk8wrwA#fO*S1;#aAR0i53%G+~7c5k~b}*wVc7;X{F)wT2o6y zp0X+V$sI}(!6q{EBN?j_+oPDFu@}QUK~K880%avCG0lp@u%TU5qfM8u_BHs!u!u+? z6*7;Jql>u0z16Ir19rjaHy_31Sn-F0(Pb?Hy9=6sK~2;0{{yM1Wan%0BSmKx94%Sx zmD7wA^LK|1JKNAm#+;}eZ(Oc(L$&OQl|fl-2-r8`h%%H+clurD%Kk8;FZOok>~8x# z?Bd?418V)4Qi!<#w9jyp(K8l7FfXI=w0t{fW+n-!WEi77rxmgOqs$v&|AqS|y=S`Y zB@3JXV^?DUSK%PiIKZ8*g03V+BsNO+XhTtb27x7?KN%bPZ6;J6yqpV(Ij;JEhwyZA z0x89lJ+r-g`&XmZ6tcNfC7}HXAAS=n6;-g2bdFnpl?|T447yv*xSD*V$w-{0SpNM5 z+Jsl=F<`~O#}x{_yC)M>Qd{*Tp}`i+;g@Hr0jUnR+ZfwL7gCa5;aX#_PYs#Ih2w`> zSCq=m8Q+fRx*L|`zMMAmNuPexWUNq?a@kMPU>JV=DiUxEpIZ%Zr9~$5OS3c&h?85i zU4J87eE}P2Ys%T?r>H66FJBx+kcbT%dVL17JYJ~JTp0?}P==fdk~zBLfu2SKSQum| zW?Ld6|XPGFpRs`T_6z_WvUFHLqVs_^dK{ufKIG68C_U0 z5M)AMuPeldbIL%KR8_%A>zP)1f*Z)pm75_MuO8<-5PJAnYTTYWAQ9G*-Rg6(vWU{V z-)XVH^=Tqk#!HWav{P1nN~`63BVNrDu^=^HcRa3;Pq4`aigK=|XwbU6)lxuNOzI>3&|Vqsj_VicT_bLX)yQN3jEwn$B(;OV6NCLttvQ60xn{Ap(brmlZ%IrI zq!YQic`&u4o#cAwlid8lU#ojm{ZzB(eZCfg7(0y^5f>ASfAT`j%Dge;xp+LQ58PRN zrt7&SWk;Q{gKo{-j9g3CZ1i)QWFsltnDMM{58RD6E{wXC#WFW#$=Dlx^0Bl@{M6`AXxHU< zOR74rLUQVw!Z$?__KSc|^V$loKXZ!zo-UI$x!c-u6Vg)Q##xxHoOWHWono&GMq}$I zImQ~5wcDVy?w6%tE0(;r(5WPqe z=OrT*ORHGrV#msVa?dI|(n*ET=A2zB}BhT55@<~qY6)M zv>1^JECaj61CNL?POV%O85@z^+MF~#>hs*G6a67Dub|5<_>bc?k7;i@^kUg`zoPRz zN4C=N&0Ev(mH8r5ZTis`P*L#CpSa$Om!{t!1#1RiiyX*8&iEY4-n6V|m&miF$5g!pH{nQ`HQ%yu= zN#|U=ow+DkzKrUG2ka1_E}2_?gh++RnT`nAwBw4tp3=k{2YoJ)r){Q&^IU%rDR<8k z&9lI>jPCQt2gfZ#U-L4z)4N8xd>^4ju9@o{@rT`}5rjcN+gW7g-ZKhXcf(hl=S~o; z@nK>1ei!4j+1qwh;~1GIeadP6_C76{RRlC#krRz!u+R`992r&niCmAz_HNhP(QCEW z@$EyEfe-K;!ldTrMX;C9lS@R@smuES#Z5(vi4F&fZ;#BO!6}3ee8#;gNQ42e8ozxt zMa=cr$`sG}Q}5~wD4fI*yqe%5KAd9MiLBjJT*TpVu{@mby4rg6S>qlkq_flxOk*70 zhwv{MaCqUxlD!KNVdd&N?Q92o~?Rsu(n*xwti} z>2&8FLKFBS3219KhpH6>!*96Z!$6B+%GYcSGe+)wv0R?4s`RC}1Ld>7jc@7*mo{`qB_7suI(Iz`&S62y43+%Xx8iHhNlzJD+gWjc>@9gFO1 zITkw!2#Xs#)Uac~a_WxHOB9+|gZD~0Yru*Lr_b~u!9%6*yJ$}?+GO3@e)dZGe%4>N z;$nqjO8$w45CE3YATzJ+{w|~}JkbRoBp)*(NwuIXbr)~I-+}LZ;SuTVn=bNl|7nHZ zh%<+T(jMksC0vG6YHTm3tqnwPl=Uo--sva-4HZYl8kQ?7Y2doBKzn zb;tDjj8#4RMFqo9$spy8WF559awQb?P8pc4!&=;X9NUtADn!vv;9e zI1a7d-?8p3LMQy3DPQqMbv2=Q*XEhVEzmRQ2p8I?J>U;#gCRGJ^~#6yDCogP4%wYa z81!Fz+P!|!Iq#Q(R(pr8`%_C-qm4Ih&Qb7O3yF;;{H+FjaWD}xJt0!K)18Tc?k;y6h?^xB0NMAY( zPq?@bW6eRwvAGP#53kjaL5-ZliNV)cZhBfb*Q<&W^Vs-4@lJz?pOZ#FcmQVga*=@a2j`a-uc;^M*cCnSY7R}*h3N4Anzl}$!_DP3~_Zz_}kK(W=xy8rl~nJlXt$5G@8qS z=TE4=e1h)UqdGS?#{`K#wOi}r(l#>1?lMj6SfpFJHY^$W$Df?oO-H`H1)0_3LqO~2 zqos;@iP0{4o}DhbAB>bHh_|j1pgo3ny0q1l5F?JGE3{ILf&q6^d4mx29o_bl=d}tD z8?&ds?L`8}=sxue19I|jHbk#xmx`V=1TRd08@i*b{baSl{@RI#dv0yYa#;aR0vEb*}{%zp_KXM9z4dFB|KnCh!^)p)+} ztX^4{y3=eNZE?TQHT{c5T2^WOAw|7C1u!}@)nAzf+C)PN2jCrpBTAOFnm5UQ_eEhmE(Z5*vGOYmSeB^r)<* zCuBnu#)DQDJXAvcR~7scBmL2%rFCv&QJ&7OXm!2B$|B`^?$xPu_I85szi8V$DWfy8{bkuO{Q8xrY-6mG1C|w^&WjI{ z3QArCF^dcrn$eQ7GGa$++zl!^BRYn);0AD8V90!7eSOXubX+^K?QLx?X=?bdN-a~& z(C7XV&;nSATA{Yz&Zp^{Rb*mTw_bN3a4KTv&bhnTn!xW#3R+$? z(JasrH_@_ftzl2YR!L0dFl=R*;?MSn@ZovA@OMz=LGft~h zz?M?0amY@$u{606NeA8sQ`vMy4Snqmc{)G~EfX`7LW>}ftw53<0(NsQF3tbO6LY6d z&d?T3mN%ajo#)1_T^&fGH!jq$HF@sm-k6d9J^Lj9jRZj0x49syZv6pxPANc}L)98G zfMEq$zF5tWRA=7t2~^ZW)| zY41Rd8SmNv(G)L0n#P9G#FXQoU$jC=YRf()4lU2)?RQ`NRM@4hb=Wu2{`%m`Pmr=h zPK?md2pIHd&I9BcJM=(cf9B{EWq>t$Fwj$+FrCpZp)(h|8Cb!xv)B5|P%FVA?@M&U`lg8GSGWUrpe4aloqf*^= zF=vf0lS`OU43Qq$T$goU6N{LU^!X8cZiw5K7YE{-9WcGWA0CtS+D?dCnZ+rk8d8kJ z-2HVB)c1pFU9n@T2js9)r0JDv zM@NTCyMaWp>PwIaDk^F&V#b7|rX~(rR9K^=th{_91nBcH-%>YsvVkc9qG3~X(}%{4 z{Li_5R#ov4Jv~2151B!@Nt9>_<9A$m+nz5v+aEUkF>>qEf6421P5>==;(=8Zk7pd6 zt=D^DcM7j#ixzf}l)?YvXe)pR8Exo3Vy@$4033B_3d-`IP`lb2vRs#@C|0AlV(m}= z{9@8;`UcQ6G>i{kw$N(1buIh9{Uv~Ak7<{ndNAp9>i;1109SQOLXjMe!DEQxM*(gX zdw6$oA|xc_#bU{b{rQstA_?o*gnz^!Z+|_?GsY_^uYMv3ukHlW?6(?x zDU1C7QK$G~#BbqqY`Z^8Eu?~j$3uK5LA;k^sJ@vl5@nEfV6$_xVi!y<94 zTM>ch3m)+7%sok%DDkXmWo*=}(f`v6KjktHMWbD`v!VANX2<+jO8+-j;!v!AyNv(m z7h9Z@AwiP~Xw#_&prQ_@Y8f(XInUDh4H9zPSl6<-`FT}r*pRK1-~Z&=o2;Xan&WM+ zp-4A4Mjre1{gZynSs&R}c(thkdl|mCI&Yd}-76bW>qVJ>9px_@gkrhIx>E+}p|Ayq z&Ay&U76pBZA)pk1YGeYQ-5mPN_Zbzmtk$3 zomrfoqiRtA(UP|+yCofILUcsuO0ySrls|8OhV)$p1BVb?dBeBcmU`<$r4#-yK8+I< zKba_2O5HV>L~jba&v<8L6dsa|*;B7`u~8JaZ<#{Rqn2lD*n2q0#5M6Pve@;|9V{RE zt>9eLb@)PKLa|X`|F?MheQ8K{uJWfGs%*<$CQIW+?_3&uj2UxU9EQk74|WK|qZiLD z=BFVvUA`IXIa01eY%zdt8zjhVYP%{^ycNDjux4;Q@#w-E{t zj|kQVpV^s?jDo%Ga?Q7g)IuL#ngw-Wk%*vDpcKIvOMGzyWom|Y@-aKFZb^Ao>5R~1 z1CRlnrKG1_W3t<}vgW(!lN?9xul*kFd$UQAt|<{BF#K2Z6#sc&Ad`zCZywQmv0-lo zK1;v_!(IM{%>l_|u5BCk?KB!kr;-}GjIP?aF`d2f&}ARCeHr_|v-R-1jmWCj)FID| zV^M$--22_4s|~UBoU03ovdD|S-A8S8uHxLj8p{_@G^z#mzPUCw`Lt+xfy+3ZQe<3m&>s(#zaWCTbEJ z48|7y_l8UXFYHFX{$y`kFiN;iwX~Vs$pSs^&W7&jwOB=rM z#Ww~`=LXY2<*sP~N`}nT<>4IprDQ>{*MS0Af_|(@=Lux9dacQV?+3rae`z}BZi9S7 z+EakjseeqQKGhk@)pb3nwuW`NBLHBOomI5k%$iY;Jy6f}$D`zM{I8mt7Ko6P^r&5% zEYu%SwcE|q5rOyS2|MVsa{1!LwfPPrPS%JJ+s>!Se_yZFX>{VGL}DGNg6K}+xRdK2 zM@TxZlOZ3{Z9(;P3^Ct7Ael(j7m005?1WWjC#*Y3Ym$@3e~N)HlM9nDK$OxIeax+Q z-OfJk(Rl9odz(84ZDSFJ-YZ6v1=;%4^GfsK2j*^7VlQY(D8K>`f^t%6i~i}@*4kf@ zEDbO2^ZH@`v69(xcr>;YcRg4I8ym+EwB>TLOsquA4bMVEr-~Ip`SF*-a#z|x4{ung zqGwYZ16xu2*?M!V_>>S`B(%6X+b+A_svd~O_sH?@%!LqF(?ibk8Mj6RYZFQA#crMn zf%lp#GoLMs(x1%*3F|NHFefXG_B_pwuN9FbhU#@Y7Oww=73HzgGPznSal zy1h3xSu2}Q$7g)M+z+3o;YO!o1MeWq+;zCN3GQhtlBW@Rk=T9yuNB-9OclQV!s6k) z^@~JEhZyU`$x_*x496VVb(K%7F=Y$1bLR*0>!yNyXnbcmQ6UQMYMUcg=6$o5>hkbi z=I#C2x(iS1%C(X}-No?Qxs4UfhnTSzb^rp^<9o%>C>+*Y2BxiEJ_#Fl+Y*i$(L$;V z#HArdMOZSM@_TFzm_5T<+r;Tnh&ER0L7a9?{Uli@W?5wjRh%}0Ka907_|E&cOIJQwd_?5P< zljpeLErBn52UpYXfW6^EW{$~fx;A=U|jq8sRD_f%*ctf?(#4qH@s}P zt?gw56Khxzl!)OooQo#>H|T#>;GdBWy;1xEwYW>@3p&r6Bozvgfc48ppXEs!|CUUC z$_F9G$q%^2AV!jocwV)gi*$JnM_PmYbPEWna{MaLuY7oUSTyx0T*)&@-sXgkA)IX# zL}X{e)!!XepZLa*Foy`k(;GU5h;(^)^@Oh*2z4+G%TzJ8Tg8KVxHAsTWYnAZ>0|#& z*+18(fMV)^s?`HO>?N0h^t4nYyDzCerTsp+khhq`LFX3xra))#Dn@`_hWW7N@)gU@ zjRy4QRis3u6qPVgP|1&kWY~|O!``t&`|n9i`9= z6wHwwdWt63cg7wM_7$ayRS=xAuuuUPpRWSrX%SDV*dlcXz~2l*sx7`5Ip1^}S-yg) zE8;~$z*xh!)Epn%=0FJU=p-&hz)Vov&>e%|o)@0GYhXTDPgE(0B}1Er{W!9K%|L$G zJ{IdsfBKQ=;S4q;*nf;9aaEGaCOE%UDorNYdoGWiiCVqs}bFAW|CAE3WuP;y0> zW6}{A!bu!&2?eAXu>b8RBW*|D&1R%iJ~P{omABOh-21On#p`T~9Y-oZ0IaGw#h9MROyBEW=U1j&8UYxzhH8}c8fgNhXF5Yh7v za1dB56*@wyY-&#S7~gMa6h^&1-~zNnCTD`**;d|f&814p*qS&Xhg=jTr%@IbnA{T-|AS-V$K6#|k8g_*@>$y62Go@+<~b&YkN8!R~a@;jIv8dXsp zpVRD&+CjH7?fjXjt%M_fq03DTijlOx<_i<{$R00pUEL|6-WN3O;9Y487TrGN8Np!z zaB-H8aV%dH*Gu( zkT6hf>9Drimn0G4c)E%YjxTIOW8T4D*#-9}_C_j096d?s>R?c-;Pnop2KL5Sk_Rnh z-9TFwbqbmc*CUt?iJQm|w0-i4N>mB8Q-`FdXW`Q$c9bMQP=*0?ic>?DfM{JB8t6gA zu)!|^EmNYbANzIxvrZ0WDvBfH1^DR*xgyUb!o+^o6jrbo;g-*$(6)%?Ce(aLhouw! zk%Pu4C2951@a`9i1-hwE2Z^R8L8`C^%!<2l-zG_<2JK~xSDoV>?!q`?+^8FejH-qP z$&Oa;w_sN1%X71ZWl#yNOKK4TU1W@{TTo0>qwS*CnkE*0^0rA*(a(x($w>~rw9tvG z*`*y?eE2B1aA=PR=v18YPsm>N*4iYtDNeiZhPzTwMW|Ii!s&hcwrJBbO*w%?tx5b~ zNcOIoKkO-+r_t*FBJM4q;%b&g(I5c^XK=ScLtt=s9o!|jLy%y>gS)$1a0?nN!DR*~ zK(LUIL4qeRI02G7`OZ1_{zv}1-dk_Idu!I3*?aeNcXjpd>gwvME(Rr&dUCw~DgCN; z1yB~0!V-$_oUX1~%u5F^%&2!6HLKWb>tm`pXsUX~pFD-N2MU{q6T*=(F@wvn3?*hc zrr8a}>E*Ge-ii!G7?TVKgA%bkGc%1xf2vQjBj*hnMdY}dO+S^vQ7a88%6?B3|16Ah z>vY51XBD~YD}8$gLkTXMgM@}(c0on?h0#F{&>Fnao`8ObI+%Mr0LbG2TtSCoy1YZj1=edGNVVc$D~S8N;`$E-R~2 zuD0;9nFrpH{Bl%%5Vyr|Td5#ZSv$MVQz?rfC`V%G+~p|FQ~f+e@U3w`mK|FK(iHM9 ziB-hXS;*^&8rHSD)weBsJI&rS+{je2)P*&D<>p2Ch%ZQXU$iW1^FV7QGOwc8Q6<-1 zxNj{Lqmf4>(}5PvUV!&xHIfw0Jht_oGVV~s053V1CNQi23Y^aHhlqbB-&rE9pW3WO zHK!kYmE-&@l%82htL2`+k=9*`;V_1Uwn%0A01Delt(%`wTu@D#p+nd%g8EG%POxNS zh^?S=?yKTW3%obafs2WqXQw=&Q{Qqe!+1RM#+nB-4oIaP{NG{VP2CJNTRJXxA%oU@ z%Qy3VAj$bEo%_{vCTXq9P{JVx9fZHMj<%(Fu6^0-Vdumi^g^^NRaN4*Ppn6PXa3gGZFm(KpYnK^EH z{%9Z+-5~{Hzmg#AJV!U@#!|Om!cMOL?R0+_mPbDj5jE?^^{OGf&O1b<8pY=rPJS&Z zxA1mgyO5awMmrE=n}>F)v?L%dV18;Mo)%tZe@F4^A(bM-UH%`T8xxAb^h?# z{kjjysOP%YUknr%717MbsDL7z3s%4TYQh9GLay3Nc|-2~tJU*|VWdabi*z$qANOf! z!pg-bnrURi8)Dy33oR0!fp32{gOAGjiV7nBC}>;rqlV}89ReZaT(;@~7_q|5_C|V9 zZBy=>F8s<{9}P%Mf%4E477pVPX!3gP_m{js3dFnZRVjT*v0k>cH|3sflBX3q!6)l& zeBzU1Xbac8T9UWog}v7Ca?`)->IjM}jzD#--nQD0uojeaO!pr6I_T%ZKak8~B6ZTc z?&8eQ+)alxIh$WtF?$U(pE~2ecbMZA28s*M6sWw@8{@0+5ujz-;jl6G#NVj&iNWW9 zxo(X8q`d?g%-GH5)@0<`VU8&|Ob|{k+Vu84r=3YBss7@HX_PdnXcX>5k9lNzN}Bwd zZ*gK`_@UURt}f~6oT3q_3?pMG$w(%etL1O}yZ5x}1DEU5RJpMle|XBDQRh*@YkT4I zvaC8w*B~!`#2!YLL^wMT#*_SHSevB$;f+t!BBfvx$(ydsCQWI#H^n}_J_~E}kdMW% zY11D&@1|n51(EB`q0+I#=YSV$(-Vykg>fud+mk%<-#=@fSA;UrC3x^DdjhU#GX3Wm zH632MV_X|wLjWq%WYh=nW$RN+naU2HYB9$#jZ-DZd08_vpD5?m?Hg?u!vq$tKWH{<3DL4Zqz+QXpj*ql@AUNhrf9dB`YEB9s~pg zJXp=R$qkUE>NNPXWTgM1H~LFtWvl(KHU7OL7(`_M(I@|S@~4M}0RIql|1R(T=`*U% zNmby$CGBihdOn0}4OL6jk@J{yt8EGhh#e{7&{#yuN}~Tkb%yu!Decpz#RE|jL?ukK z`ZVku2u8vp@jna6|61o?cbeKrQR&|jJR<(HhD`t8|BB?+Wq-o;{%aX<^#4qh{Xea> z9B<=NU;Gm;`oHG$m!4|7>0S6snWim3Wp8Em=wC2~Kj4lGI8Xv#z)gy@KEMy%Fap;pQ_eDg$C1kj*=@m`Sf~xO}u3v5rN8^xlZ=`$t?ZT}4 z-!6KY|J$T)f91cj|A$=N9*C4EWXb*Gf@ZTj3kBI~Sn_R=uiGHB}T;LxxPE`IK z`F}k5-!`xU{wKRM<2seMXMycCyRx^H7v!9+$M=XVznefYY>lJcZ^7}`4>87H>M4-B zi@0I-v6*iaZE*1l-|RQdrP{QQQX<{fMdai%&{4ISd4nUR3=feZ-wKDK6GLNdb;U!V znZ=x)AV&1B%tWfq9w(b(vcG7^k{8`N+kn^fpBaubZw+7V>GezW!0}+vHrglf?+U;DeQwtba2z*;_6e zzMX$!QIZ#YL6C|CiMY<${;>M+bU^%&0Roge&DpXKJizEOte&?E`B4y0Rc1GD&Wq96Rt%KI_Tsapro!tncec&=l4Xb&4(0@1oIc z-HMS?CI?-bbjlTR@zsLRBxtZ>4Qw>itpmSJe1+O2z3HpE*IWb54Be2ke27~=m)cNU zGXI5`Aj)B81Z-rG3?C>8E$7DH8iXVXKKJ(VQF679t~A=NrxwPcDb~mv=#fin7=4`` zG)ZnsN~5R*j8IFljcnu({Z8l7t|MS_x=7h}eM|n9S@?EeDWSB$?P`MFem>*KZi{Y% z0KfPUU#ow*oJ#_}T}Y{)|3buP5>}Mb$HnFN;b~-ha8B!sgj*ky{NG|DPyi-rvZ~>_2%q!7M1Q7fr(^_FaC!;IxzLnn8CIc?z0`= zPSLuTd8-T@bh_S0jZYo14*1kDvIxA@`Odkb(e6bOH3LeI8D{!AD7Zs>N&dXGTQysG zZ`JN3);PD~t4s>6n5TLB5;+S?L!tX4mTe^q2V|)nox^rF(pLDjC1iDVL>`ZZ{7AmQ zE{3sX(OkV1o`%Xvr`br6$`Uc$Nf^%vDQUx^6MsL#5o!t3xzXR5pn0fMHvvdIXw z!-_9WNOn%*RWEE96qpm0YHBAj-|pL7eS2n)(HUtnyYQ@- z>6>2n&MJP#XU3%uFLzNEDfA{w%9>ejnhiyz%4V@wZVp7Rq7bWtZL}fd)_io<>RdFx zP#^eWqsfl=ySpP;X?+KN77dGxt2>~SR$EwELu*1tHNx|(`G7wM@0? zS6Iv)9GlCC_`@OZ^&0_SprT41;RGjN8Md-}c9IH%ZwtTbNWMvv@z6a0AEvIZOYQD1 z(1h5li{{p>3AfOOA0}qxm$K~YQ5d9%^mu4TK?bCibZHNhQ6vxIzW4D?QRATdtIeZL zb6f6Z2p(H4sfr*>3|AcI?JcOa=?D%uf#C9txkWDn5C;|XD)=jmC>CGlO34PsKK((zD$n1W4Vgn0j>Sz=VJJd<;awiYN+NOLgz$NNw zkcvZ2Ui@9!qp5w*oH?Bn%JB^>lw#@0Tr+rJM%N^ouS=Rm_pEBs3JnIi!5K5L2@p5P z{;Z0ZXh6MUZB&Laqr_1N^>&0Fr7IV(sD(Gx+Mge9F0A)QizNdRqPqd*gh7r!(L4^r@wI`&|IFJV;^MMo+uVCIA-a7<{O%2_lESx2 zZN_f@;{32-7TAyUFEjk}jy3$ehEB%_a%8?LAoE&Z+J^apSV<-6Tmo=jg>oMnY01&fCUUUgPV}f4rb0 z4Q$rn77^v%8eKNlv`xNqT!A0d>Aia6!)j`a7maiS7?Y3nt0056S|Wr)M4G zZUmy-J~Wl+*1zR-Jw*pj&(r_{KtJDLM<1rONNn}G01FE2rD$!UQ-_0&AIrhGD4|@nSExGBOU`~v@j1ypL6DuFA2`>&ElqoW zX|gDbTAe0*H?_4pi?M7nOk=>1Dd>}xO?V?_zPTCbhu#g^a@wL&djI!lxt;9vb%;xae?Sc6{GXFaVpO$4BWw;=)YD4aN70MpR?JyNMgjy{76Y% zpYpzW8Qq|u+dB&6s2=A0s#>${GZ#8CrkIE@!)msjughgG66EFjAP?E)<41o9@Su1} zot04I17poNbmZh$<2+RM;=*`?J+?@NHlnmVqCRJ@f-af2E!1-?TEMz^aGCzVL6+m7 z9aKZJ$B9cD?IAWh>U)zizq?;6yUTJUE+04F z8kcWSGusi&ABgfH7!m~Gy~brH$xMFF1@4GRUSy4Z!}VoWe!Vc3^|G4WC?6m4cJlG0 zJOO`Ts?D2J<|Fl|jbPdw=Nz1@0{Nes>BX-Ij&S(Vee$nmy9~?xH>?CawS%p9cgf>_ z+Hso~(P`IS<%^WvGEMV51r`XbN=w-a{LonqjqYq!VFk(zj~UF1?ltbE^<2gUB}#!T z+S!U|-OgMG)Llf9)5WVrDLzQ9+K2i?y0nLgE%7Jpy@Xarrs?0;1uQRGP}Mx$`U;Bs`%o_DFUTv88+YeZ&)ufn>zOs%C7Qn=iaG2@tlAPu(Lyk&WKKTyL<3qGU zW#Qb`LY7WH)0+y2F3Mw507I0F3b2if`FxK;hi9)29o4Kf*SR=%wb=o28ytno`Nk{0 zJrGZtSYqgrQpL2hOV|aRk^dLjoOGz_;sv2v6u2;;#OToVbN`gCSX@+MDL*F7I<(Vm zg9wX}aJcf7p;ebP5#kOj#z$=L@oa(o4kke!KH#jdwAH*L9<;^&g{!JH{OQc;o+rh1 zA&KVfsRQ4*L*(aKK=q0mh$?bObgZ4ze#T}io~TT|)7XanwydqH1M_P z2Zb``sVY*eBxGURW2A=6B-ioIw4=se5(kIyq?gSq_9;|J%H6_=oav1sx8Wi}op>T- zea|vC&~1=Qu#e?0ODZ_$z~0PImiOjcW!l!h_IM^hEXLR4d>IrvtM>)p&rx3c1DluZ ze%nVUZo0WQYw%CHpSdkFAz;HbC`b9dv`6p>+q~9=j-;j1}+B%#c3BE2`-)QqW3>73RHayzxR2693_#R zXB)6AU-$$kC*Z#@C2eCPb|9-rq6*L~CPbvsr{ZoqL?7#f=UX;;diqXJ<;LRndg_MA zK1C`z8^?G;UZtkJ92#0?z59kT?~@>Vn|9b8$$aZSASyBpZIykLBD&@Jo011xQVxM` z@qPqh82|i4irzwPzS(~5J}|cg$U>?kBE44qndZEUo@-J3i>-G<7_laBi8TvfHS774 z`@<1Ir9OYTK&PFJNp@!(%u6v3W08&Y>XbNR-E+ky?ViEKPh36CafOMeCA zcs0a&pD^aPj9%kWv-sk3V?l`(UgAwr>3n`QWaz8{N|)g;q{&g!ze`2x9mgtPO)gJq zOrj_S zG5L>(iAc_3n_%yvx{Q!viWk!8vsG3DMaW5g)mXUQA0mRzW#84nBtmaTusR#s7yd9@=Y~7#KJ(O)UWqoR-rN{RKd8|)D^QfspMF~=ilJhr%&5)2s(btHbAW%TOwA<18ql%D0#7YV zF?G#-uI@ixt|gZ)iv@t>$exCzak=6|!IRysGw1xC)P_^&G|M~xc0qC}0{1st9Syzr zXB-BLYJ8M7{(&lW%iHrKrcuXybN!#}%c4~jBm}(Iv+AGHNpwSuFWBkb7jxv)HIxxr zT+_bcZ*omPCOtanjvcnh_6$wpL0P56x9EqLS*NRxdIatrmlWGd-3K*Q+4aBh_Z>aY zg^3KI6Qz!sbgH-So!%g@1E0Whd9L$a+RTOr^|*d`f-ats{U30(g4xT-E8sU|%C{2Vzp)dcv~e-2F+@diJsEF{ z0s%k}xg7gx*}}*-HFkx-Iv@1+uO=JLIEsTpN503&ZeK)IBr^NF$g2HX?e(nEkcSb< z`l}^%nZIz!btwDOhGkB@3LLb9*+MAu0p{rO@f8r!@L zM?9o#Y=y!SpFsXTxwOUBMdRfJHTxshe~(Dev0Dv8=QU#XE4p&Yqa4)`0jVuG@AtYzymB%QOoW9 z>S^+{rAms&@0Cz_x$vuvn)`Fv(lOY;F7)lJUD~(O-`7Os#W-18LZtlD52-(ZMkp@X z3k`Fkli3iZW}-7JvwK&KXC@(c1Ls&n;u(|E9R#$BYgCX{a2~AYyZku2f30^3&p~ilYjd=0Qz!Yv=y!UcX7|*|_rs#^}UH^uNB# z>spW=qrkMJY+pF$`q;#)$;F?-jp&Jt!iIQ%-)LceJd#{EM`?2ZCt~Yx35*A785@Q`4p$M)b8~&`EBm~ z7%2oraDuDfK1B2x-ErMGa?tHr{owum{R+1&?AHOKX|&L{0y)Hf(P=W~=K=upqxO*T zn!@)qON&tfY74JM9k+J8Nn;wEB{CpYhK~ECwh(I1w*jM|m&K{e4j=5SJZ2Z0EzUUK zLu0z+ez^`FDu+%bXOTDJ_)7mSI!?Pzko=+3yrOMC#4@#?@nr)V$CClT;QblktNC;K zYww(@Slr-d`P;dbAn~l4SGWY%lu+TD%RbWkdMjg|wpYfRjbCGf*E45d6415N zTr)I2wvdu^;QO&nsBkLZCOv!kK6FG*zxwOyo?kStTGvT?9xg+RGpX3ORf*QE8xJ>`6QvvY4*Dq>H(UQ3p$CsE26& zuo=`hhPevXb>i`LE&4oHrZHl1mkd@H+ehJgemA%F>5H{ct0qfs#ooZ|@0%SePB8lm zM4@lWb+O=K+rz={nI@#(uuXfI^2h90xkN3O#};L;`&N@}d{8|WBl{2^N;&<#01 zjt_UZsEe8lrJlvDy}zIFLWMkf0G7V$fmwu6JP}j9xLTI!kz}AVg2-n z?Q1tI%{%@$zcIwt@T@)VD-$^e7Z}sf*z?M*+q(tCfX*6HmmWrj`@eQ{|jVVtkzjZxGasJfK5wetHktcA(CQ}gD#qiOHFv!}%g zulKpp=Mq5q%*6OL1NL9baE-)YN|_x@~LC{L?YYgoU3-q1LQ;k4JkRr!5; zHqsO%&xw2sUk%MJ(>0WGp$T>Rg=>YpBrdf{3qnNAd`=7cy*{%Z`F(Srq;!Nq zC$kDXC-|Lm*^R21+x+{$a>5}|qJLmm-LsUDzp>fj!|j2|gIHnd15YQqw3UfaJeJk6 zf5*$Htlk+OdY&@0%ppwQI^C- zbwX;<{-!NutHc1hO5Df!d{27bH|6n>@qMu^!&r}JP~>eLp>&y+=kE%VtI@I*=Y=ER zx(=9e^Js_7cfP-GfD1uZwz7h)D0yi#61qn)} zEkQ+AKFR2iQA^Epq7}(;R4ztFK(YtZICVzKTm_SQ;gE;S3ouk{ecQx*^9?Vvt=vg@ zyr6L8^b!js;y1hE(uZCPZb(7um^eLrm25kt(04$qD)n6N(oCdl zu#px#UB&F`ChX*w+YNeLK1#ZA7BKHx7|)A=YWDaUcx-Zk1_jT<6*_{W>5VDzbtxd& zG2KoqMW0L0Su+m(h~&_gseuwzCiR7h08b!AjO^v9%d=kle95HO(^evk>Db=g19OWr zQi!xt57+1*9BT=_P!$z_AEa#2LW{9PW-IUU_Bx|?lv=4dQB;8Aa@!}CbJ1Bk1CIB5 znu>mA_3fA?K$wo^|607Z}WPIy>Mt!DD5XyO4`$klnWZBXou*76UNLfH! zUP-AW_kkZh7gvm#xI?F!`ozj#cREIYBW+~58r2S`*JwHcF0^g(K~;#NSQyHvwM1$!zQRU>3|8Ip%KNwNAa7$v?0mRZqXXTht*kPP)n1?Fybx1Vc!KE9)Gxl+db%KM;SGkNJ$JP46!OEmKO{=m&WjUP zyA2-n8Z6&%f-CIBsR3VYP$pxVT|g;ji7dO$dj%c9LOP$^?iYz7u7oUKJ!BjhV}&wI zL;DmX=t5L3i4$WuihAKa#f(%J#7WIw(nLyc6E1!-L(2!x9czLc_<2t+#Bc@DKm{N@@&OSnE)k z@L;J0GXum-x12*4C7ZSRb{I5Fhu>9dEnI{3^FhJrBn|hDFt}7zfKK*_!|j=2rFC%8K6Dm!t@D5x6$1_@3Ax(X9#JOg~YBZbK8O0&j5E4zY!OQ;Rhwb^ZF6AJ* z^lqudSKS1DnLVkG(q;K!HbZG*=?Na0Hq znRU`g< zLvLj=aJft$uWw41!efy;M7{Plc|y-bMkaM5{BmRwz>lP6_l=Me2L5)OztsBU)DhCp zZJw!Q#FQiO!ns`7GsY~{u?T}x;RI{OSLkXtd6Uzb7eP163mGy53WR2E#T!_PI%%Z8 zB#?Wf_faFqlzp36BVK9Ik=7=8)gz@lA|fe~^%Gb_egXziPgnKCuBcIi>Ki8YIqvk+ z^Bh}Xm-u~?S>=YMom2^9^*5*G0<)w;Gxgrjljt{$IE#F;c)VS2qjLQWqvgO<3~9YH zwkM+5Uh$cITsc$>k^|67m&7RjUm#dGl4=B1`Q~{;HK)leQE5b)spS=A5a7UEC!RXX zpqWFsip;jD8rDTxs%QR*KQj@Kap=~$u;+krCLC%rjH5|ROwd|~oA3W9{smWf`B74# z290PK7y8!)L4p%%L5Zn{yTsaznQnufBcsPO`lzxzKsW_4NKz%xH$N}+r14RPgO}9= zI*2m<>ozicOcK{ruyT4j3jii?iB+=&)q&__N3^TZcb*n21AL% z!%vKJh;#(KHO2S_c+q=Gp*Ps^@rOQbA=i&_QE4|QFiP7kMxks*Vgd$4GK=;lc(@)x zJEqJ@wCB;F?p9R2;#4-83t;DqoIn>$v+`w()cPhamw`~hcT*d#PzzRGI316;E>07) zSybabhBn^opzZ<6zL~~u zw+@lP+uu<`HPK1GeI+CW%s^WF#DEw%_)?rv@ujWSMRJfY5yMRkW-2ElhRosHvOMaN zp6V@A5F4|8Xx`XmkeO*5!}(KLaNlsGvNdNF>R1I6vocGG1sp1ezTS4qwreu!AK(aB z+h(h!>SaVf86fI-Ry;I5C>98<39x8K`f#gLkrou2@L6X|kD)+gqUj23{?Kxh_BA2V zhi`&I1OUQb<{P@D$XGuKy2tARt?ErQuMa^B-dq|7y^DKHK?;Q{uGBChC%X6-02u?8 z#@0yTo=MEq2>n^LtdvAtT7vj|7WdD_Y$=HinphWB8DmyJnnFa2a(nRb@(>-Y4wpKl zA7;HOwv8OchA+m)-=^E;Q3$)A0Ye0CTKbDdJa@S6C77rK4XQS0cxmTnnhwmFD>*rhfv0AI>mr!#pZgybS7%ZPx@i0ibpd3{P)Ym1u< zero~X^liHRfUQh$Iwiq?@=!xS+A+%W;DRJ>&j}$|5-z4jW*!A_{(`A4|FIBPp)W$z zl4b{@QH90w>YhU`Y^BE9yjEe$IRk~#1Dwno!{GtNmkLmKlo&!mw_3u#U3mjU+&W8q zB-*6lKLDe8Gy*CR=q7p;=aSE(d^qJ83oB~#j9&}r4A2MgjXqgl4NR-crxZXDVXa6< zX;bg2V5T&WsZO3_j0NZ`ktIfmO{6;1LN z;9>PosgGP2t?Z1kU<`2HgJf79{y6HfZ#a(y&6atdd@BrBnc1~hB}|frzaY;ah+;KJ z;6x`t8AvXFT*m-}t8?$C4icNaq>HJL-r;GX4i`V;PSi&O;d`mh{hfdxoVa?Z_rcnsk!HAfG;Zmy`q zmmgMl+)^=Xqr_7xbIK#{U5U~@{dtd$%6aqfS&ydQzip&_FGMQ1Md`zlJSi&VoVPfh z2_?mikB$;cgBcih?=(1$k%~UsGPW^A?z2T{XKj5El&6WM7{;Furc2U!&|^TCE@Nk> zi$UX|>+)YiYJ*1{Y{Gdmc9pye z1Q%_*JVytDEy`$4%j!9+pJtX!FyJ6%xfppah@t^*2zR4q;khcBXwT@Rdfbb#3MtL2 zIgXV=kUYmP5=L~30nGPRgeg^!$M%Cm0#FLpeVg&pDay-(b$G7-O!+HbrR%B>ec&z0 ziu&3%`D>df;%lZy>z(^}T)ili>z=1$j@>1*ExuDWAH=p2o6m~a^c0x7yX#qtzDw*a z^$bU9{UFk_k4ihek*mz$MP&=XV8*-;hEZN~(7Wp$v)#fsU&*~o0=r23z7R$GMG`sx zAw2Do>$^S__~W1m4xehWCSLx52>bk^{(Irh^T)b+919_4W|G){OtcWhIwaER;N$Nn zuZ`t_veIoWyh(IDa{0y<2P|2lDuk2JBbVjJg zgG(y|u`mRYk=+E9&9WxwX}3)CmFPMq=!$uDb8ed%v7&}SwtQ>FwhL!`Av4Hb#Lc5V zRyBgqq06_a4TG(?+|OQ0;6Z-1|ka86+W>K->eyT=!FDtqn7PJp1a0CngtG1*<&BA!0Xo{ND?l#J_b@7N< zq|6o>FTI(^rHN9hVMW51JwZ(G1r|%EO`JD`c&7z}!s4+74+U-C91%Le_%0A*3mcl) z7hZ&2hDX!%{FH6)6NrGb9RI+Gh5X*H5&Ht3Qt$d%Prp|f=AIOJWC`Wv?0nLy836m; z`6R6K(SelsglnE)j#wreDai3PS*0*h5g{GtRc15XMU9nO-8g#bDl2j1wZh2Q`v+a1B_}HhExoS_) zd->yEOZO}Dk?t6@s;o5=UyFbAQhdhDlQebZucG|tF)na&@)DUz(l}P2?bpocovLaAkWp}EN;kwZY^h&9F*efcs zL#VXhjOI;Fpu`GB6a2bj+_b|E5?0>)s4TW+Z7livBVBv1Sh7<3n@r+_cd8PV;Q!UC zmmxCyp1YYLQd0v=G%D>>*h~`<+RtND+)Mjh>71>x*ADfhR;xu+&*ouApiCmwo>vLH z(P_IMBqkOF)FXPo(yUjmH0i$dAL&%m$EmqEW?$I`=6j-T6f$?a}nj=qZxHV60k?TMk3Ch^j3V^PjjoV}W@ zotRy8d>ef9*&@CIkckv$b$xmfSb7s!rekhx@zkJRPG}3X!4Wg6DReMlIu!`agNA83 zsxXu>t3^CfQ`w(jaepIt^JSJO_Lib6x7s9$#zR*G4r);ye&S?=O=CxG_d~+L!69L! zPY;`ocNa+aE?c51XXO4&VB=P=bFPg>Sztjn-=agI$33l|Oc{t4B6E(6%r3enwY9I$ zMypL(M6dHB1yE(Rz>QckdhfsTqqX3a*~FGrh`k_%6OqDS-g}qyy?SMD=nt2mws*fkoma)3b#OT7Nj=OKR{Z&3`h{+U+FlbRlRc z)eHTkANIlo`sjTCRHs$-1#1e3CP}tb^M$%PDj?B%K=}m?tB81NSFoH=XoNtaLW$1= z{jsM*_3ch1ZYcNrph@?uqHj@{?V(U>+6>_u0OtoHFy+<31+3PBplx)CETKIOxAo&E3C%dvhQktM)e=vOOA`ALq zaEc|IFk%88@dKQsb-iVPp6pdp%@!f)?x9b~)C<3u3%I&u)!ijp;jTkL0gmo z%zE0s#3*I;K(sqta=H$3{)_L*XF;k@K}Gu7rs-N4R~ zt3@1%HW?fZqI01eTLH=8`{1PSpXuf}#o9odIJN#SlMx<|5>lN+R2T=was-r(H>t7H zKk`vG5dYD=lo+(K5?`#+bT!RwR>AHDol0BV*JQ|g9c-V$lV?1RpnTTR!unbnb~5F> zxV*&6H-_wFiW79LJ$0Urw%N&rD{FxrbBhl61_f3?o6B7>gh$k12WdfJki@HJQ0BEH z+U#4GE=ZKby@)KsyK(5xurJ5lTAD9GQ;#eosD*Q}{3WKFUVuMuY>%B&)wh6`z(>L> zkQ<-3N5Xl>$LGfo+UDoHLXvjNLV3v?9~+)FG%`)l(!o3rUhSl66A(vU+8Y4n@60M}}v(=|6 z_v|x;JD;F@>vy94A%Q8-sCb~2^EMSEHKy3YzG7fzXqHvcz^Z^R{+lV6#%)V5^+gh` zuTW|E5ns$EVMz}(kQF@{L%K4}Oud5cN5${o$3vJ!*_Yw_CnxK(LC8;jK7ZdFxcGI0 zfDVbyaUUD&@+gI0JL~?;k1t;5zwNFaPQJKy*1XdpF(qjoO+bxcH#h|I4}znu1En8& zzt73AU*(N9Y@GzC@&PEYx9=$x)f*ERTDszsU(we30!;~YOf3~vu8FvsSxP=l%xX|0 zm3b-3O3J&5vyica)xLE1Xn8Xlm@Fc?HcuiQwRGCr3!_RjG-hf?zsSzoYIH25|;S3$zw?nwyH0O;qZpndoY{=}>ubLf<>vkr`^! zYqbU4iPN02i?k(Fk0+g1?&;`>PG3o$wA3vn9_NV!l3iYYtK^_zyB+6guyS%`GZi9u z?ZO9HWEyb5uIgb-(tg^3_;S3~8qaMGgH)aF8u3<15R^(xE9>kKGBPVFpBt*b6yl#; zNWZ!FY$S>Ng4wc|$gXnk$)tFmre~~hH0vJ7OuA#|nC9wk8_U6a{n1PAfX=r|2q-M}C6WC0WFw+c!`8-{H6dznFEU*`oFnsT!S%7@UG*!Kgoq=t+F$CdAGNAQG~_jJ zRkT)C7vCmUFT`i%|8m?m}z^w9aPF{kN4T7If>3sZkak}#h>*ZzkN5kB< zR%>x;Ogw!PHh#8ZfAmTj?vAA@SGU#aYHptT!ePU<+G4(NTprX?EnQ8`ltg5(bjP`u z=9)Y2xRraFrsB<3zA5kw-1D_N%xq~f0^^bA3OhSH2U9ExAdevU>^&!W566^-r*7L6 zLbed-)!)7H8sYr+{({k-O_@x<6q_DSr9qj_Ero4XZiWWB-gl~;B?{+v^NkcldQkAa z6L#!c9s0aK4JIiw#VGT+RFiFHoKlNEeMmE=$rfRuzeqxzKZKtII8OmFVwQy47ZoP~ zA@6QbQ6%`fn-m3TVf`DFzhw(xFrK7-P0ljmN*b?nX7L1f9{!eL)49^p?IXH_0C&NQRp*ZC(I z505?uDUO`pA=rVXn7&NY#-F#$L6jQTo7VwpH?%8^M0lBCS~H03^{vMYdVzac4?u~S zJ4QN8d6STQT^*Rn(n3QHF+>CkHrfG=&WSHLj3-3q{edI|1=tRgENOHyC}vL-1;|E_ zEw=L+N!`>zDO3c>v<>6*~|vHd~R5dllAK8VG+kr_*Abt2H3rT8f*lT9BFqh0ESUj7C znT(+X1qzx520syVjt`&`6X~*^luXoNcmL~A!SG3U-h@c2KM?j9*;uNO-hyfCCPfJ< z7Hct;wghEQURu)7g7C?`qNB7VWrzrsa|lnysAr0X1R0}sLuD1^c#}LI9dmk{M|zqd z?ZBpDHp}3cDr?*r@oH04};jfZX-rzBQ7QG}qSGNExh@M&_DjC#su81%-_H?wX(<0zjO zX(Jl_fXuF!lnmAB#fUMI-@;lEiig>PG%(Y&edtk2Q@A|yiW7+9Sv`^WcoT26du4IW zcX1|?_<<0(7ZYHt!uF=Owrx~rV)?SSZSLF0o^}rGahly{ARFE-4$K*92Dl!Wn7153 z!p$&2B{e{Ap8bd%N%8`f&39qQ@rdd#n1Aj0)Fzz*GUZUzc!-XQkuYlQ%clfF;)((p z+9QB3*%j$0d3+wGR+rBe7!`KydRmFYadnuS>*6BzSVMh~QE z2ZhE}f|A0*o5m}FI$IE8*khCVQ!0#iocyz$8KbtI>g$TS5)?7v%1Bhivl*zgY{a=U z$~XCOeGH(j8s+IC^>c$qI_VEDEu<(f6wk5l#Z(&(Y_rNameTsgvIdps|rM zQwe(-(7!GtNA}H_O(hp52w;F5v4`VL8cbx2EY#?Dcz|4e3DwV8Sk<@bqNAF8WJs2G zbn#rqm=c!^^||F*Bip5eV927}2Uiqr8BHj*EQL+P9v0Ld(f4MpBz5X!ZyR zVp~agLgbY(8tsytB;vOt{=7cYbc!}+d_qbR&fBONja%oK9}EB<@v+JfEc;YpViQG&D3HU*9tb(vfs$A`{8)@9R`VMMb%h0nbV$S6+&yAHAdb_Kus)%RHC3 z@BVCRxZEsRpRfVutUmQKU^qP~rK8>Wme}saB@XS07J}NeeW!Mk%yNP0f z#DZY~J7h$9^k2r@hMcloRMjtGWKg~uen(XlDr7a5GLTPxt~ScKvk^EBz`w$Bi8QYy zRyzk5ZEjj4-6{n%FfcIokB@gU!MKqVnQYsafwy`(M94_ze}fxB0RE3PP47L1jfnUi z$H(b^l5X@`D8GN5OL4o4^Y;=($_;pba6>})90BRyuKwH*yv{VWh~)V?q%sH6$o*5^ z&7w9|R6<=Tk=&Vm8f_M`tefG5@u$s>4*f~wf3jr8CWQS124l4WQC?Vo#Z#o~yiV>OMq5Bs;e?#@3zC|>Z{FMY{{ktQ z|BD{~|5c**7iF_RK!EG|zu^wAtgPpJ*W}Ho`?r7PRmVFQAc^tMQ1sTve9x|u2}zk# z4gf~vKd1h=3r39p`@_GU(6If9^7|jaqbTVgJ@U`LgPwmP_x&e;P5fW2q~;F+{uMlc zYM=0V-r^#T%q4vkGm07$BE`BiX7`2@)O{3{;Hl+*QO(3lJ?6cs^Tx`o3hKf+8@RQ) z2)A_7+Au3+7o&gxx1#!+j$~_2c^W}HGf@7!>(`##&Aen`{n3xkmv%qxpgk#yC;hYn zo!8dRl=|u=s`i~jv2DZ`9Vb<-RXB^8vH=>j-);|TfhSw6u+_y6sE-XklAx#GIJv(~ zbX&LiINdNjN95od2K5v21W$YB+fU~|lOL}+3-WY}5jQo$hZ=|U?!_h)`9%) z!|^Hu-g{p!5AdvKz_f)(vN}PMuRu%3ErVyG?fp`FzGZfN*dW`Em;$Ew+IvU(zD<%1 zZK9Vi1$U|sRKi@K`|d9)>s~%mp0kqst}yAfsLg^d6(=9uUASBA5IBOLKW6aJAP4x- zeq)A3XmzNm_&sHkchVM~u0C)2rv)y3KsXQk3hU>^$&_lc@|4twNwMc@UZ?CpRMb!^ z_r_Q^Q&??@k$jlEE}Cn|Hto-+g?{J!)}MTy<17v)Ei5glc*7WOylb1@RIsnh>`G2! z1yT7eczHw}n(XOhS9vkHTrd>+38ZH{$rF8n>VEAjB;b<{7F8l};D22olhNy=kL1ym zM~+7F;wDRa>@y#3*(X}vpS@S&U~gvp^2A%;raQA3PXkxU{9d$5z0gdfK-r(FggD>iL=`Ik^NjW3f)3+QQMSQ*g4uF3)LAViJmRz02%{gT$4!QU3>fZy6L< z6ZQLsBtUS7!QI{6ox$DR-GWQ-;O-jSJ-GXz0fM``yWe@9SL!|Ip3k@HR-LNeUxs34 zckf^=th*Kc^ITBnowEDah+~o^Qd`>jatQ_s%wB7S2ez5aE>9`o%ed8iYVZ ze2JnvoDXIrMBBZQMJdZf6Yu7$q^DIl8xBKxy`iNmrHAxMers;9_Qpz-&jwb6CO+WRmPXo|JK zR9F738{8H4ncio+iQpY@nDp!1c!og;MA{aC+nH^?KU)Ca$S=QRbl6$`j#U1R+-CRT zbT%7QVgyHgsAc#1);hScB{)#e1YhSS4b15@EQUIV?!9S64f_*4bs z%kTQWHkj@kzL0|g$&bBNd>yya+GtwnA?VLM21ai&;y!0KScs1Oj2C)u3i5fpS zS$ZOxn_YJ3o0$rP4t%r7@O2D!ruQ6-i!e5&?`yZCVXIs3wB>0*Lfh`TndhO&s~?Ot zO1>`-rISb`(mqtY^J8rmgvLja>91!UG_QCl=SzdE-P$4s^4wPX>HRWt#}%|NPpW=| z;p=*@`&dY5WvziLz?;hvJdTbRPru);oaBQ`rBR6^j0h}N^4J3F6Zoqj|BnMyuSVT^ zUl{dj&EYH?hn<=9kW%}=JB2sgDf`GcJ>|~@;KCQ4G@PrB$d9J^U*!1iHezczF~|gG4QpT2 z)a~?o>=x+r*(QqYW`TUKeVC1vK1rHLHJx?Y72SUHCCMRi$R4qehPaIU&qTP;eCWlr zuAT2V-w*tfZ6@JMKLz}83FW|!$(KJ>3$j(DLu z>-*uue!!9p@f39PK^q&*%Q*+<>0Mp$n#04bGWTwZaW7y`;N4yy0RoX3dg=L|M_Yv= ztd-$v%U+uYzWQUux!gtVzREkLi1A`zgt`Y?XK?h6wdKerRWbGCKo2#j<7tMJ@7kL5 zrc0+EiJ?qvpDJ@bH2L*ekdPR=Ejq&!tu|AC_dS<0PDLZ47bUz-__Xmz*Zt=Ac-f@H zmZ@2T!=19<>xw}2M}v$EtE)hiNp!8-5~F7kWh(D^DcKq=E5rKli(a* zQCiy^&o9jzjyP7qOKq&36R9?P{i8nSCvNy8Nq-y|*Ud-1&Qe$VX0#!&rR(a0$f=JLhX z|4DeArY#ectVQJ${OG z?P&R4Cxvb$DpM~w<VXE z^nIQE66(1>ZT3sCJV{}|i6&c>UjlhWC^faTq?v9Uc7RB^Ov5y~Kr_!!f~%ym54Nkg zifd@%x7TUae-=~%q;5XJ2(Km)q=&Oa%t96hX&|iiUV6BHs-8iEok5|l_PXX!j0b)5swa0cHjWAYMeT0y zUge@d{0Sv*rH3cG7ArV@3XZZslGo$C7uv(f4s8~=p9eC`q@W)AM`n=gFDREnHIT3F z51_B9wPMP&ad^V{&i$upODm=09#-zoEK5CU=*4uwrQfcZnOv}kixR_V?#o9er~cZ= z_%jgnKy(R-Mrgb zlouV>ACsK!Vk%r?4w>#s4A!P{Y%-@<)Zmm!uUgVy%61~F>> zHlvvxZGf@N^3e?wThUGtScp18xZ2V+UX$Z{Sgi5fpOEOd!*Yt^A5X#3FO(KY_P!8E zb;URK*0?G#V6=ExVofXHxEeTFNprdpbR7K4M&^=M;&;8!QSXe&&M8OP(>@5zQ+KVf zmC}A*NHg%yy!Zq2V7nWVid)*Y%fXPX7qhZ4Fau3sJUCDNH^*S?P>3)-OXX=^Zb2z+ zf9qRx9?6Z_PI0J}?i<-hn<*qT^mTe*&#;YuoL%F%ygJg{v)rTKW!u7@(>xes#G6eO60m{fZ@AWKl!x&x|>G9Mz~*=G1GnRfnC?-o75?5i;! z+|od~UldNc0S+jhJ$K^6j?z6?Zxm>^cWia5MMk}KM^B(W19jJF{CU|9$`d-VkW>PM)*)>Ch% ziH!dNFQefmgg~Ka!+rvjdRk|-cs#Xw{q(rEo#7?m4j9<(S zNw_8gym3e-DmP`e+48Z5on?j(>ez;n9Z3NIU`<-7ovpFz4ew3sg9P{l@N3Gqgg-A3 z&#aAH8@#1;KfU{|x^H}JhJCaJPgIGvFc`>G+thonaIo0yD`^n)5q?wR^kpa-W#V0y zcoK>T6Vm4S6@?6L>A&c%U0CAKu|eXSk(4|$gXi~Lv+|h|e(G^TUJwmaS+P!rxb0B? zxFS2>{oEzwvh5Td21>@+Vdex0CjKBriJyuRIG_}a%B9@NWDUkbvQpIZ)uCQXh6eM$ zi5Qn?G*u?&jmkK<8R094goPgGTC%cLrYGy)7wKt~(2o?0CDfHd4;|49XHny_w<*$;t~* zShy;sp+Vgle`KO>$2r0dH{oR{R~qs>$dNNs`UaM*G#-e6uqHB*ZObURd+o50)0;77lO{^x@3N(fbDlN!aMBUg{(5@Ew(1AFQ5x!EvC!~V{lw?0 zkIhY>xa&N>>Przj&XDiZ_f`L6DE;*WI|R(KC2h_ErQjAj%PE?y##UzzKcAd%apIDJ zHb;+&_`Kif2)bQqko35-lMT{`nr%7(G@o+`{t?##rhXDo?a$P#jxyUvRCh)<7F~0* zevR)WHHD(R^i+*e1Tj$cdDTS<+!B}4 z=`aN~-tX}-gm6#(dg)E zE&oW0V&?zD@t0IqXIQ|d#1skU+|;kPWN`vvU!91-lfB=e=MeU|$YagyS0p|p)myT{0vqxo^xIQ2l7iqYx*>Sqa1Ykh~ zDD(}l(sobi#yDf2*_kPDCQVEbrSS*QcJp9=Yj{gH>u4{4Pd9zH~XpzhwQ!&PU8 zRWOhLn9A*qa8|r8-`y$B=+v7H1=ceFi;#c}Z-io^#wl2yit~qCZKB^W@z3#>LJd~m zQFRXcwXwc7?8mwz8wXK!xYF@&wHZ4J;S%7gu)%CGQZaYac5VqxO_rOoh zQBv?a!mFl+Oz5oi_QJ@_%s5P@bXxnXY)UJ=p(to~7F6k1kU8B&0A&o`qUsLQM^gL( zp&C#G)IY7>XVaG5hH2INl71!})8Pqo@Ux5a|9Fy^?7HC{uR*###q}v2Y0_IgEa4TE`_=nE(RY#-Mya&!@6#}aEnO}1aGb7!y58v+9-mzU2>>=W zhDWi41?QO$PIpIpvfYR(2Qp*3)xI&upuQfU|gJ9$e1L5y4akqK|DoS7v_!et?i?M=EBOo|$WC0W^njXH-dGLj)vx+x>i=!})ld!Jio6gG{MI4Cy1cuyr0Dotsf z78{%Nv`~rHg+ODRMiLnF=W?{h<8h0U`t6;wbdo>$!^we|+e{z}1#SgIytxCAPMDTW zlwu1B67|SSTsI=*7DpyLEo3z^)Mg-xS7Q!y@bxF*F2Zh6PpWfFI6Xef@?H8I;X`qj zslTl;CD^xuMw^a~-+yby`ql`U#ao8#(wMzK_)H$lch7;bp(hj?{G*3(B*yLLI#B_Y zo-c?M3|*AMGotv}4x3?$a=tRvRjst2Y-&72I3 z{I#4T%ipn0*CTyr>Z*#Ia)1 zZ9dZ1kp>nQz@w1N2#(g4J9mCKV-+M11*xb;NRFZ$<(a6$|;OvSF zK2hFW;Fd`sQxzw%qHw|!f=Zw^W}vE=``EvAKJ}@6tV*@216h> z7#%Y`vpRX0e>74@Wl|h_ZN=K^x%b5huKPYrPMcyzbB8$vEf=g=%aU zG8r;wIF{acQIn4jCzu{bI=57n=2EN!`z{oAB0$7BXFH^z@}a;{pW}>BZid8@Vl%^g zq|3f_rDrZOtLw-8w{Jej{~^uTxFx*5ojlET^9S(U@Yr`dBSO#zD z9w60K?ckJ`m@=C=V}^;FnsUdOIYmWAl7dFc2QEV;J$)RA{qIxVon&&LIh|k}`0rFw z7!8Dv@HG+$$BXP`7{Xd{uvV|!=@L}g=y@b&jIYjZR(v`5Wmxh&lr zqxxO{nH}fdOT1)im)sMA`!>wOALbc3E*y*6 zmrwj50@th^uYP1?S|atdTqhk*s_3RH3_IJAX-%gewbeyo`JIf{hw9wLrrwLhAquu8U{MiCuL;iBT{(eD$FJ*C3aq6&`rrV52@p1a$gc)+sKU45s6qQ zA@Q)#wS>3bUt7nuJy2Mu^m|t4Vy|+i_=o>KlbEB2mS*8?ch*}^Z4YHUrLE~afj-`- zk!mvgec?`??l<0%rSI~nQR8}7BN=ypYxoU4zXkoZFb_CEJzpMlQQ#ZO+@d*unf$I= z40P>&V-wVYGk8-c-dacO(+f%~WA@@fRMyRq^ExAcz3JACD)v?+BQl{m7+;?3uCz8K z;z)(BiDhI6|FMzBpc69VN)yCjc0n9Yl=Wie+H%8Oddt;=iZK z@U3M`FPz=uU>10Z<+Pn{^C4S($B%-%msj|xm;@=U&Dl?3@mu;yx7zjaJ#y2tmzYTW z&ur#*gFYAKkm!%>Lr3qjTz~q7VC(tt%tN8Cz9Kjm$`icievAO2!?-HXwcp<|pU4uJ zB}kk`SU#mnnf^@T%#5>$y8qeNbHCu}_;O-6$o zujHw7WLkbGp927Eh>9JZNW(}5jDKX3@DlibBz(_7m`6D;mFbR|ith2#;Xseflub@_ z$Gwqm^n^yr---9lDX|=$kTO!t9qH=S7019kYiOZ|qrcG|Z&&D%n67J(6I1=_CH@jJ z5Fq?TE*boO#4r^imUi&w1eL3#r|bytA&dXA9W|XRX<3Utjj0iWmkw};VxS$6 z18>zn+;0_8G&bI_3hn%p_7tkkkKL=m)I1{-{;E?#HfmR1&4$*XXCTG)9pI}kY!z7B zN$uwvmS@6LK!r8S8$_)4b0?gewTRnG7P?Q=P?t9^Pk*{KdAK}B>jQ{1{tGI{?=e7ZOG<-BOg!G3(XZJgVEEtl47=9*oy?`8I5&kqz+0n1OnQTr?i^`5Lz2(b|k+Zg6RQmL%&XGy4=rhUl^^?qAEUiudcD!CpS2lubC+Fd)8X@y69Z> z{`bPq;%!GqA|xML9`*;_I9!mPf2yA7>Fvb(Y06Ya7Cb=6Ft;%<8%t*-cLI>)y0z6& zaL7vtC%5VVYEw7Ryqzw$rKXJ;O@HC#0>CIGq{`{b!%Uw`cuC+PHpoJGw<)>`wv{`Q zP+}X6(Z*ix;JET*Tv!o;WSX}GyQE!A13rmSo1|eY>-@P^@O$t+0uxDU#ub~ z__&^yM-`fAQwNy_?QtYQCdU2Vp7*`Oi+@;xCy3<}&c9>_|3>DP3AUrnv6!faU@0$l zUP5qp9R3ambL-~Fv~_JpCJmLGOb-A1+=%P=Ga~8Qi&f-f85QF7OT_oz)1;m-j|iq; zy9{5gJd+AVfi{fA2rU$P6TF94^UBGQKYB_=O>1vHG5e#;EkRMjV#<>(;1_m!f5Ua$ z5k0jNkTQ3|7S`JDm-50%{@!}~-OhM66PG&81$K#sa7vzOac)?ONGwBg&lI&(f8?LX z(BZ)xiN&}-b@2+Ta!lcPu5rG-%x(7iP8O&oo@AP7yX5sLlQh3du8|s|{Kf?|0ur*a z`9(U)9E5Kdbu0{DGul4r@d7fq8qb^!F=zPh*!p8omhWK5Zgbbr^&LK$ zUJv%(#GV2gJy)(^`;P2==PTeun{*RovV3cM?_h5@vR^yUqpuB(5dn1hlW<1&G8k03 z#2Y+DEFaNZ{RRH99>m6%HNtLks`>O)p`$C=amGy7IWKen5`cEi${e=ram0yqB_k>b zogOOMbk167& z`|<~&?GF3n)K8lD4mXiuIfZqMATCN2GY1v8u0R;mXVQ5pK+_<5gfAX5+sIJw*AQ&K z{b0j`uAtYwV}8EX>;aKYmEwORldz><|H-^COb+S1+}e1_@s+<{M-xFs711;z{l?`O z%TumsaZ+BT*WCQzwz}f&5LPv(ee(I?leoCzeNrc9Ffz*s&k{E!_rY$|!y6!3D%V*$3+N=g&2^S~SUcS+GA&D?I2JqTKu(UxWJM2jrTDEtxFJgUBl8OTT&LRW;K24k4kOpk{>4XX=(l!aW*Pm!Y*g{W%| z05s^zgR^cynac>!K;xOwdjbM{>v=bI47d{JG}Rcf7)GI-)^1wq1=jQV^hd44Ly00miYZ^jPb1Bkyq~W z7OLua{gOy#!#0deKqn>4jfu8F;1#;w$Wv=wFlCJ4pA5?Mr)CRBJ)XcEQ5&t>oK&Dc zTR0c#=ic-`VC~C}Vg;83FkKTrchS}Rk`Os%qUwcW23g_p5|aLv0Z&qF0Roe8W9rYC z11aY!_%K0czoQ|!ocbV@`rQD%?$a-XFi;(&~jVsGi$~&W~1_< z8Z}8op&^@%Gy@S zuhkysA~E^toYu249>~pFCf}0ATbyFZd|4Nz#4VenIY4;82P#-rOCoFayjTla9Nrj8 zi(;x|PQ>oqui<+wp=`PAAUVZ{_H?--hOK!dZe@n*2;0A@;Kmqld4Yf$DrA1z7Jfa7 zKJXCmASP$lPnqzC2KK7KgXD;co1i#!=9pt0=2>7VNtqw*(d{x@IG%xY#j4fV1|)s zXo=$j85U>dLv78kRN6$BG-K|8Vp-lYBUotKwnaqU5$Wn2hIhx(c%!oP+k@OEt@?_1 zrzNG;EZf~gw!hq-M}~N!p+hbSsamu;E!X>|_S8}ZZaQddDCs_e!xAIlxZ{&=T})76 z{Z}?%9L&Os9Os&twKYvHboWMVv6Utrra(-w6*zjM+iZ6vKFfXG^Y!DpiO~E&ow7}< zprHucA`$K1mbfdt1Com>sBKL1Fkq)$urJ`H%RRAg5C2uR^|%~cUsDh3njK?KIZ5$v zQ^#uh>jTTOElo{Dy0SRlgrHM;J-~7{-`?87g_w+_0*^U!1C@X~-JkOPr0gu(?D&q8 zwUzBl;^+~ZAU(aFSm8r~fieI#SN5h+X~D?ia>S2|OP;)I1NnN#d#T%uv^~~M#d*uM zI#*qyQcWD`xpcifv-%P~!Wosh%m*I9-J7Uc#ZXMM|7HB*<%V}e;<7ro>bJ*b zd?o=O^ECnp=j58)V7(^KL$9Ulv~nhUOc+!SX8D8*@n>q_5gT%DS~>JC?GF4$%ixIH z&XRq`z(|z+f;(@Yzl=8YSW)aUn8sIW4rkT+RLH}3y3~=A9flK5moa*_kgE{5Ho%z} z$WM`--{;DMXlLHPPN6KJKPi{0K*T*hA_(WfV`WuZ^s5fzKOQ4ucTl|*=S}trMVzG? zvdzFwvur=il!dtsr_9c#sC6>D0rXE>NSNHUvEOZ&8!L2981M~3t$bs^DU zPuIlBqRNVBxAcP3uvx*++EO2tqh9@GZC$a0TT)U1c#K}kKVH06*3%>O*kU4p#=i?& zBLt&AJm9hsep`kI0g~`TnxClFP$1#QB8R_O{?gaZ&<_7616E2d{t-q!v8yoG0$(6IA;PqRlkh=}*f0}tQ zVpI&?H5?lA4Me9No)t5bho*oM1(T);6@hldbZaiyYIYDA_d<_2nQ0iIg8H=VG=NAp z$8C>PC>*R;9Hba`^py@U&Up;M)~v`D8dvSH*SqSB(q&4r)Uw+%UummDd5UXniGIf4 zTEabgYq}I>lUDgfsiQse*>Ww1sx+h2Sf7^K80l1mnGt^}8bWSA;a%LFA8D*e;{0CQ z$||Ox7EqibufEXAtyzIWXp{H5e{i{tfkqFXj%QWgO`j(flZ=IWR#rihvRlhLSWcdi z(9Bo@eeeo#5=x=~C8)Qj1YEK`O_qX9u7Brh)47!kjUq8TIQVOn1Zgf7xqyH^n1bkY z2hIQp0$;vu)!;(a;42c7-gw|AiVs0sdpkHKKuk>R(&=-+C-sVCLU88c>+?dl#O2e> zn`*Vb+;8&aIRVL&v!7Vw!n1*o!~!%foetG zez}Ps>x%uHgZ|N=?a;ullThjv7>S^PlB2`J9{XmrZ(l9#?24+ZV_1!vSEy-d`mBFn zd;Y^72a|QhZdxo6zW$Vx=`H+kn3#`}kKSK59Vt1ffx_z$*rwAGVp}q;|G;`(vK#$N z53@jy*&`X4tN#U8qaHYuruPzq~61v~~+1C^L@z`(G;n&9#u`9G>%CRH#L zF}{BA{EH~dQbUIf>iq%se*Z}$^ZA7i5A~(I4@-gFkd5k}`_mu?8Bt7{Z)Xxsnv?Vz zNnA3j{J-#Gfi8=G1{|xWz#tRzZQT?@PA5^}&qXjs-`>q6{wL+^-(T6p!?a^})^S4B zxQ`{xJIbmUFBgfL`?+Q+bjis2-y`s!_XDQ0bA^F^^qUG1J=*?vH^GDc-*+c>iAy1@k|1|ASCjMWs%=s$~s~8U0@- zbs!Qe03Pi4x?-aKF-vTgblJc2^Z&cOWM%&UZ!Z7SpZ}$+U_#~QDMwS_V97EtE9L>tk)GP1n&TcVUiUW!n**Dlj zvZ52188zTgbISV-Ew{jpC9WAY4oDj6>}L&c1aM2;iIoQCpi8_r^#9dB|pgDeUN9SK)nUl|b5 z$Bd)L?(OL?CeQ{$YPYEsviXT4P9APEBDjeDj+&MVQC{C#4nV=9oedD@U&&ka__Tsi1ZGy4i{Y@r8(pdm&53sGr$OY%g6pzPgT8DnlLUjy{3Rj z&U?PLiZZc`)P?y;5086BT-%Sw9BdbyvOiO<;I<#7+CtllLiKpvlor;3uIj|1k#IlZSg(oS8QknvM%GEI;CVi@BNE4b6l*&p zFix7bSkYOIw)yy=IWE`FwS9!;odztFHTiz7>O(MT|5q%)?l*X!-^(;jDrfR}mQpyF z@6vghUwAod-xOIt=y?3N(2DoBKIz`QEH?TQ(yt*WD~+_&lk`PrAs@QcTa01SWa|Zw zGbcsi0rj1^phA#lgIw9w>s$^9aC9MzStGI4S$+W81CU^Exwvb)6Ivk53dI0i!bmep z4YA)>?q}ryTH>NqXQ_dMd6&sEL9TPj4OTDu%dhGzu+SjYI?yJn!96&KIlv_JmLv_sT5J!%$e`NU0PM6FdLJ zVuJsugLL!k5_@$&kBNtv{M>n>S-+25CK{S}%JfI8D_D}iH${|ywNb6s^-3lX+YFsz z>5)p0{lw$?m&MY`FT+dK2HR;bA7<`bh1lLwN0}NITBMI_6vV@Hc*2CpwVvUbS9>=G z&j4_6Zb&7ynW=;(cUgF<-!<)gqm73iJy&Knm$$333oIz&x3fBIBhyD6bb7-tg2c7J zc570HDsCxLa}+HnbuXOe>%Tz?^$lvnyEP_Dyf%k1BgBlfm_Zv8&zyui>F)Q)jf14&qY!YTu;+Y9UBhbC)xO@8b!uoS3?{ zdTnW)RK=?ns;XWq7QBOt=k2v91`tB-p zG?pX_Dd0|mMS>{MrXk8gXtxkQah`Xs@9k_T=hC@75Xd4K~u4c zzqTIPVH^CTPl;h@=88|6ddcHH+e$cSL##Z!2~)t6dn`}a*2w7sD8sFj($u#b!o9LJ z6kR^QPix)>*O%?zqX<-jA;Bwc1hm4Z9tw5)Wof$tG>ZmsOI)MfFqUix`c)vAu&XG% za9F-RGTzNU-e(A-fh50O0G>$OdxHD}uly5>rxfMxm8L+uxc4`gZ_NNMIR!L_&Rali zH&bp(BaAK+6G6xWWZB9ql)u-bQxm?t*^C+_8@*?S;M7u}!OisY>A=!zPY%ME=J&jy zzpH%81(j4j>|WU`k2f`hMSZCgwejhiPm}@nh_(j(%lKiA9K258Hji&Q*A6VLgEtv{ zqzg=@^|;(Rpt@i)p4VP#^g!9mVS~aTH-Tva<h$UE{Pby-Hdl$fy7WaYUxoX<&mSZi@}lbcA#vV6vwcuBe5 zMPqa+9saTI6}JcEM3$lb#l#&|6C?4m^_}l#x8c-sj;XC&v-Ol;m-!2oe5{1h;}4>_ z6Z4zYgVbq!*gx~z;Es;;R#j-#737qTLzoV(C~@f=xvD#C4N~SBi;8}#XpUhYQ=i@+ zDWal91PiJKPb>E~tGR?mYM$_9IOi|@J zx`gCXnFXS1Z8Rpanz1goP@Ryxt4Z?kzuMU#WK421=2;bn>ZE7k4|k6q+!(Q>|6xOj z9S$1X+m;2de^60TnQ)?g^#5H2VM*-#sZU)jirsvNt|sT4`B`Fwq#*c=4#$z~c(7=x z&Y&y3dzjgJcdd-)M>&@38`TQO!Sb4OQ?}F*jBp3z^YE@SqJT)cTV}rg@k*%B)ezRb z9ybuNfH{-ykKf(uNk{RD=XF3#;B$-+D&WeyWd1Jw)9%KgnAr+Pu*I82dPx7CAp>-C z(E*fR$495oG}_wv7z%`>r5@>sDu-lL#^6sTT=R#Z-bdaW zvtYhi^MoJm4x`&J$7FCY+{PE&&+i452Dou_UAyOGG$+7>Xec(=qqKK~tUkQqj>Ir4 zckdziJ>24+;^|0fbtKPP{jK&(FQSKhUHOnN>DKGj`@dwp2negP(eB_)hl%Y5|i=Bu{{#qYA*7Z_W7G)W{N1NGF#Urr33Ua}D2KbBNk zBn_tgu|%Szi8q~Q(G=^_*>uaDZ$zOy57 z4S3DdEwNCoZ+G{9G+pA#%`NRtdeE_B=I5k)tf6ypbM47_C_Wbu5VkJA<1) zNhrnx!OP%fmPF=jSU;v)RB&(so7c(0Rr|-W;mBU4cc z*fa^FyN~)FLgnemUl+(l`JEHra6b2AX!*OT*lR% zR(Z^s)=nGP{{9dKAI=%no#{^a!3~F5nNPF>n_!!jALe80X~8jJrP5;YjIMmj^PLxq z#b%4cngJ_nb&jqQjb*y%)vSd^*fATUnK*wO!( zpcAg~By-q>$ib*52HZ9!NOK}~!eo5G5*Q4s8Mm1E=`GD#;Ul#S@VkxV{i%czbMl@2 zdX(@Z^7qLDYvqSWepFVnt3PMEdKXY`k*dpoCApC26;Jz$&)?5Qz$DC!nP?_(lg;Z7 z@GJ(B?{0kMly6mJ+3;&Fl8q*rdckKeYl?N23~ z!NnC~nsULJt~{#{=#lAEk%0>fz;=USV-6G(=07VZ1v;FlChQ_qVu_TW06)K{F&z=} zc(M1{Qpjrmq%inNl?f9D%izvvPc&{#(O0^8f47Yom@ib>AECDQRd2z`)7uI930W`f zN3lya%Rf6!gEaldpLi7dY&okQyqGlNKv~Pd&*mXma(Q2PxHmFU0DppNf+7b;lqOpX;Aruq!iWWN)(=J2n{-h_&it`bNG@T_EHt@kguaATSF|ekk6R7SSn6sET)goUQHsjmiE1^Wp2+C$ z{qkFufo=%%xX8MnWW8E?XN7a%deg=yf4&pw>3fDU*?k0kd#(k7hFs31dN^R3SRXi_ zkb4}T-o-?ZGt%JZwYoL__3>C1MJv@~&iuvP`TCmqu5w`S4h>CxTzms(@&^^J*B?m> zrXO0rP1wuwycZiD$@R56zPJmCB4LAao$)zkae;w)TPIQ{-=|F);}TM2J<{-#P4K>? zKUbBx+A1Fs3*9_xJk6mdOTIhuPDJMHgs|i9?rBmB>PiE{V$%Xto~C^l5@iVa|HwU! zYIq#}Uek4bU~KJROW5!9=EjREuyur@5TFv1-1ExuKe9t)Z~w!_M2^9{6Qd(#?-|<~ z`7ooZF=&u`xL1sws`zx`Z=rV2l)#HPZArTW{h-j_@Ezqw!>-_01J)Xze78|ztMxKh zef~tFhOO~==0$x?*O%NWQ3a|%Vyul3{8WVAP6y@>UT^#$)>-b3k56)I!l+@oeS72( z^Nbs^X1-s#4trN$-W+DzHkDGWcb@L>v5?IVP2Vavt~PZpU)(>0;xRff9Bv%^x%%Aw zSwh@Do>{!q;qfIie-iS=d|t5DKS>{Ts0rV#64!B3&+yot{GyY(@-=ZB&M6NwIMoKoJVL68MqUMOq+YrxHZ&sr#q}A z2*17Cv9$(?6~$~m__uvd+&-}Uj$XikD=)1LP>*4jOvp|jFMpeO{QM(YMPyy{baBsY zmp|x=aB0$8PuO@y;`+uwH@v?vusgYGEVZ_iG;zh>M}RkHqwl3{<%DZ7Pl<;)_mGK} zMm;nNTXHe*>C`SDJlu!8Q9?U(Gz=iSc!mfz z+#Ojz)}~7)Sxi}=ia>ZS30qz!(fz@aw!5*&uv>5!t z0%+~YzofNZJ!MeY!)VjHUa4~B$kNLeoE6*z&bp9Mak?y`di;iiwc)1(N!z*JQI6M8X8KR)*nsyC5L^E*AwzWFnp>T9}&b z*C(6TPx30cG9@L7>da?hb57*g`9g0 zRazfNsz5sf11EQCY`2;x1nluR<8pU1p2qs*UaN+N7FVVG1eyqY=NU47Z0|a9Tr7`W zJhc2j<-KK8oXyfU8eD_R;O@>4Jh&4af;)i_+#$HTySoO01$PG50Ks)|g1ZFyhOO_j z_qWb^f1a}r^n!b)@2;+{uCA`W+CHCMM4P-$6Lhk3hZs20iF@QKN}Iy{`SPw#s$X1* zC89^s%M3tQE+Ii@bjUlTQk^b~{hcaTsDovva&1;@BFXwH$I%N9IIM!F7_Laku(20b z5A#4{PpCwsFGK~iRmdmX^Cgy&vPfn9x9H5*^5w>)aOyJd@z$DErs>eS!nq4rZMQU2D9(@l#-hjx!ymq zw_vD7N6YmGLD(Kwhi!&{CqZ0B_aF6HIWIFD1VWv2Sc-)oiN)X0ft$U$LmaFK?Oq9I zm9;AuPYh;KL1`UsG^_1Or5!DJN=k5tA1ld+4>$Lx3WGM~TaMQR=p!`v{s6*`c~X}D zOwnzd{Q3PBHX$^T5La;*ZazT?f;L4nf3+eI7_k~^e_)kUW%=0{Z((s-bl2#aw!(&d zIyEkMeid2C1n9$+jHmZ{(wZJgOKVJMBJ&L!F%AT%aE4W0nPDLqf}V`gNc%yZJe4_KvziiV7QRHLTFEZMio$ZMkkB`0{^X2Rl^g9It z+0UW;xw*%@lLzi1TV7IBtveK$y-(1`4EW+-+P?~FD(Fx6AH%J9?#)t5e=URkeJJL2 zxRx7-N#I`yIdpJxS$sGdm}U?c)nqJixvtauc&25WIUUO~y1t7TJl5=sC0k*}m)O8DcDKZCb-d}r^_cg*sDX|rMbl2nOv^4J ztM41#yyU=R)}JwsH3LdhZ=ylEO+DS^W}!|WXp&b0@0!Tg<)ZWpH^&=(7Gfp74;Wz%QO1E1myTuIzg9U|tI$5L1{g7%{6rUxv4?0&i7Y69zA;Dw%J?%R}h z@(rCa*@2gkf_YJmx0PB$+U2opT6pRLZx5~>^qiNi`=l0Cc!zOFlrr+e3!{Z(eq3lE zKC~UEj&Bcym~K<6nV}NQ;e8=$kHee@<+25bhdxJXXz+e-YD+dBHL@i&lu3)P|46@} zU?D?8MS+D)M?h5|Q}3cW{}Uz{6H(C=S!wgp?j;~B$>#^lTJVNbJkT2YsZzhv-3Ie! zN)veBj`{1Pq99&LGX+mFt0|Pu>Pb(KJmgUA`0*(&@}rP;+Ia7$!pjyqwh&a5U-t0C zPw82lX?ce<2110x*?}bvomMbyU%pM<*B$&KsRD;G>Fje!U!Eo+IO=LM_FsOvb7rJ@ zs&C1wgJB~}!5{*~qL|#xeiVxDF)|)$c3#u}*n^O4UV65CIa9XS5&E+7Q}Hc(EqOF* ze^`GvPfP@4e$Y!+?ETv@&l7@H$Ub)md%Q0-_~T~X)|_WTfnUT_zMen02+?nRF&h5H zR=3KZbNXQ@$SBUJ87_3Vo1p?pVaDMPw3keGtf5BD(uZ~>4OGA{3?bjRlifgOA3Joo zM79F@;Ry@ObiS}7UM8~L;=)B~)dSe?oe#?^KBwsSQyrRMgZG)ihfL=OS=Y|KbVL@y zeV*T;Svw%B`(V$QkhwF%Zy6$>0xQ6m2vaVKQw`*vshvXtxM<(oKXZYlvJDC`tYXsn0gw+HJ(2iJ;9>QiHFV-Dc6|;nz`Xj+#nWh76PlO>OglVzA-Vl9(QE61L1Nr*hC(OA zikvCC(_B`E0#Hj6cl|;qStEMdGkIa7*lhlAI8l9w2`Hr zFmnk2e#K#s3!7WRpp)3;G+y$+F)vLtOjF7#OszCd1v(mW6L?VZ*-3DbRIy??K5Tt< zbzH#BEA8-qID@5q%TwFZi_}T$div$l&oh5sUe;jn9^@J!bI$N{jx-HLcAX;$=_Ph; zkW5(Yz{z*>=Up9D_-*W%&=@rI#sx1EXxyVkcL2K4TPJ01ALy1v?IH5}xa;XCQ3MNePXsVAGn9!0ppU=?V``M3bR=#_9wHSEQfWe zf@t5`PU5XRxf&Q)qhg|Qa1m_ZCdhUF&Sacc_0IWYWcYcl`L23{+VVFi+5{5gqQwu+ z&FE8r%02sT3Aw2BYQ5NyCS8ybMTZPK(2g z?iHp7e;A6U(zvcOk`{QE_2WW>Wb)Xqv?9}vnCf-Khvdvx%WrEVk+FgbF4H?+FDFbx z;mVdA=^PLPIEvHuJQ^1dYQAPwO-W?J_SlP4=KPtc0MEcXF~ZV) z3LMEE6U3jYPH81EwqnGl;nYmX%&m&uWYDO@rHA|(-%U8~w6Dk37nTGP4t8B{j%_w& z#F_M0-yJRMk8cDZ>M(tAaLSU?G9yf1m6iY?V^Fgu)ZNMU7-My#v2ICQb9gIH%XVf$~LUw-5N9e}F6%2qg&yWE-R=R$Y>A z#lNGl?Y#9V!J)(a%^x8-fA&Z0^JC!m!a`FX|F#`^>BHqSr>H48u_Us{3am4Pdvb*t zjtau$%a?)5j&LtKF7=b;;8S~;HCy4(8+8mIJCvczTN{e#Ez}U+2mDgwovpOtY%hG^ zBm$0N1pQ8gyxcvQh>3SeTgr^fqRL1K+iLx5&f;ih`eUJ?(-#HQj}`I z@NFtUVjAxjJeu$`=ZnK_CTtZFD#82s$Wnb7LNbL09S+B=%vP!@E)wm26yNm_x!kf3 zR2?E|vl1@Xq^@K(1?A`LZMR28rz+UkdW3WfRT5fniA(8`={FFY^6j%QL-nC}c&8tL zj1M&2Cts*%eUe~3G53_hca*H$)F693q;B8XGDg7~zBlUm!hL4hPBQ+(t?jdy8{Ru6 z)T?2Wns*}Ph!q7!9Ix}&dpOZ0BQ{56O~B(;uS%)l7cvGK&9mID2-ArbIKe;DAEgxA zGKh#7W9JE#@7!yjq%d*06ZMm$&9;9i*RBu#{PSSE*lTGGQ*t(AxFW+#XW8kRGVvg6 z-q{*P@l=wCXS{n?Y z7h^3GtbM}O2^eN~TfE3`oA+QlUop!pf8UZasoqU&fRHh&N4Sazj`(1d#c)OHqUNc& zj3Y0ZGzWl#5lSqdy!10a#cM~cp@K^LA%r|%8G{6Z#Lgof@S7*b$0iA7Ao%#8*Fr{z zS9z(?^}drJr!^Dht~Lh6Mkj7nbt+oZIA!)QGvll10fh#ua&?1+DQOwfopmvX~gzaw}PD*Q(S9J_rh zD0+Nh3#*xGP%cdryan6FuPp^wXHYaMHQx@>LODmmb5;^?^1?80N+7cAuYk@Nh*v~s zStDPkeqAsHS%Q9{)?|fLZ=fZYxBlsIdyYxA)+wNeRJQvxe|8oo9PB66^ex)J3UwHh zabI+asp(>*eo~F#T@KlbD~%)r8cw&_+rgE^z?q#r98D&D+GKEi2ea>QCBaTJI?@=~ z-0qjh3svR;3~5~f3Z9fLFCqNZ*7&hBawtR2^Jh7{&W}i6!{(MizYvzv$!f42oT!1P zk8d;iCd7uDbI-bbTcg~Cr1GLkM(3v!dE#sNFtbeFi19xP-)z9tj9mPDxx4jPzm3XD z?0|vXJ1Ww<7J$t9i%~<@;g?FooW7N=!0GWzW7C$7RqE=DhIGHqt%ziWi-ttVQ3pj( zP~0{(AK#YuGwc_b?*X{^4}ZK+Lo06-oSL6X+O&4jH!7-gM~B2x^?S7?poOf6GV)8} zZ1uwdHVoqHk0125X4YyF4t!gqOS#SMcT&HbPj~APyE?*{Enr+PyT)Bn?7zvbvR3FL zP)`|7AZWT+%@zFcUSLPv@nkkt1@QJ_sYHrA)^^UdwxcRJtAhUg1{xmUdLzB-G2OPD z#Om{E%I5H&9NI1;u1SMSNz%`;Gb{=3l67;R8F2}uv3dWw7N9-a8R6Sr`>;34J2l%2?UT>)*POc<7QyzP;6hD9&^EYZk!E&aTAOW3l|KjcU@VqV|1vfNIk0z87xHur6rN_63uu$v2uwiA=5!Ec_wCrCP9GHN@d+!BNbQC`y%M z6IO^wq4w-$wJ+d-soeMlHTMBd2iP7vbYgf2up~Fu413 zfj#sIkJWhSOqDddD-7(tcxD}P{+usPfn3Ya10<`(n)*SR2>P?lb6f^ z*|KW4@C6l$-k5U?P;9q4N7{HLFWWs}j}i=7bo#JTq3EwgFwlP@hYth^Yol=cZSl3g z7iUirE$YZFnM3osu=i6xL~FcTjA96PtZ_GhpOVPhJ+>)g=!KcC{m4zj$ydOf=xyBo zrlaD^P)=>!?gTFeSb4jd!0cXDr?DZfD2Y5Srhj2NSyVwAFW}5aCPD7cQNL2cXEU-p zYOk9FMePJ6w6dbp#JZICG&7k{qdjrS$)NIv;w(<;$X|`uSvS*Ch-S5HyMkRQ`ve-q zD%RnxN5u|*2%qP&#x}QUD9?Z-hm~skM_(^;W zqGCcRJJB3jgtP2?+%`GU>kLH4Mi<2hf+=%ZMR`0a8AOWC94S_#Q^f=4Qr^G28um!T z{no@@%O9!S8a<1mX~YOWEwGKX{KHT9>G|W_%{{##?+fh1HoS*^Q5pKXe%W1?mA*j) zTqvQ;z0R?RD2lB4E_A*<1KHDImZC+5!Z^{VQA;wwMzP1JB>~W%m`-vlLz0WH<%;RI z(&}DUx5HeWd~2lk?Ni7>6{zJRV2dI0htt7Z9((vzVn^gJL%SqtI79|_DCjJR9Q!C_ z6zhTC8{duTuSoSL5!Q(lXP7Xx9M1F9(jR{Q-nwrFpJaBfl;4R;l+|;^C;ypLI9_mm zy4{vScV_Ve<9NTFoyo3#*G~2Fx0a2FdcJnX)cF=Lee#nh&~6-gi9ZB;+}YBgJ`0Wo zE{i`eD5Dj4UtJgOT&qSd=HY^?pAB@ko)c{Jstn;oR~?tvYKDi;<%$D+k>=n`@2%4v z?6vd)$2zb+u6Up_g!J0^rh;aKR`sg!``cl*W_}!b*m_a~wo(}88U6}ZhhYOX?;%5+G>Epx(&*TKnw{60K+u>dgKZWUE zHNj>TJok%7P0uh7KqE8U!4ARDu9X*5Z1`+h(QfeZUDct-CYTN>`hB7)LUf_ss#Z*S zo#5>j4c8BGwl&$bj8^}ZhXs_%irntBI`$$H=8Sv#$%Y&7Rwx2w@3jqT*AbB(S8ak{ zqS^8KNw;g?qOWm9C_2sgnRO23@K0GfYzF3H!-2Pg*Jc5&i35oQR|{nRn~ovdbROUq z$t><|`vUk!Gi;?5S75oZg_8+I0J#^&y-$XVm1e^Hsl-LS-dfedz~bZm6f1H?wXsaT z_~`D=n0B_&kT$M+4EQ)r8Sh=7=lK;A+eL4Mq&l%D3>;~nyG7%-M3zi$jL8G}kKJ2w z__~wR*UADeZ&X0S07p*^we%O>$*y+)Ld9wjoDSZkvG(kdSQYXT7eF6dANdFnMT& z5pV3X_l3K|&X=VaLJ?7Vg{R-amGwJW(@bIH?PT0DON#-+Nb#bqU=dboQ!cwtWX^T( z;re65J!AObp6ql@9fyOVL@&6g#`!nxJ%8Q<*jG^Z|c8%04olZZa$HoL<5fF8gy!Sn;Z2G2A+2xNwYe_Gh}t$A3B9O z6IYQdq&Yk`VWpJ#lBKJ=f{Bj#rBWM#TT`hAPSeGxC)u)~N`ZTj=_mt9^5nOfmp(m^ zZA&1Z#cxd19DC}?=ijv7|9B83A~cU>+za|6S#WkTCsrDPnI&yXTe8bzxL`S zSJbC)Hc#zI*lW41hT%0?;Fi#a~xBCyS@7cs`~{#N%!y4)e+>8hH~dqP#s9E zaogE&z(}O32Z7r$z{GiB=vl3k;zmpEQ^bp<{#b1wiZ;)w?5^$UWHDVb#=aOv5IW@{ zWCFuHH^6~Lg5)M$!R5S3w{s-0wpP#czyk+eD8fHhe{$TnIk@nUGyU>v>xX9cz&7TS z{ad?9*aHD0`o-IqILkwX^Cvc3@R|Af?Br#e;X7>eremwUizUWXQ&yqWf-&;Yq?Fmm zn^fU^m+e`v`yzkm%ck!Z;Vj!j7uGj8^n@eYWMrVGlXm}8o~#f01~5v(=g@%@542mJ zrS3LMEP`AXDDM)vFRu)rhd#J6RTA_&NGp<`|F*lEy8`5P-N^si!R7GyEqyn2`qHhW zGtXX96xj9to^y84JENzVuN|AGlqZokX2YWCIfASqcj(O=@N%*L1Mgnf0rqyD1Lb{G zCU1_Wj~kKj(Nf<~C2fv}IcuQs10N$$Z*4k-Z{;e}5bIrtt&J`o&`kxQaI!fnqJTb> zkuIq}jEeSrnyvIaaGl#;LRMgMEK9P>wA#qX03hlv?0mM_>JoJ*9M`G#fPJDXav7q{ zY-CvEUPjZi=LCIK6*bL+7@4tIxzYW5VXgaji4^T_j;I1rQl?FNYi)KQ!`jps)nwI1 z+Hiy3*XcJtZl}lwx(frp9Fq0o=7*oA=%2oNT34=*V0Wh=}8m~uZ{4OP&PzlXK<4dH0h{fE1S(zLIS>~-%! zYs}d;EDJ|Ms1=M9+h;M>oQh7F`CsndL~ug;Q7CJT8iSA*I=@gv&T&t99$D%+PBC1K zI`7Z5I=~$+1)!2>OE(Voq}m$K1}OA_Fx%DXj%xw6e=7oeQk13D%AhCmCQQ|LhNhta5iNfINZ9HK;42W;COsWf znwcY{YO{Nr{AsZn(0=4NK+FyBy`nhd{!+YO8%J-5c z3ac;kqY29ig#cKIJcC(Cr8QJWkt`8_7DqexGfMYe7DtCTSA(mn4ln|$A2R@MM_pP# ztA$o{rv}+t9Oih@n}DYh9qML7q71;DZ8=HYSN2;> zkS8;$o13y@z^QM+-wYm=VvtiA+6OwVlv9;|xRHFAW35XC6 zPg$!g`g*jj3?Lly`;pyl9LvsMge#uZ7J63vT(&+arCQI9jiOT%#Qw-ahCo|5Fgg&D8FArgohflBb-Vj;wA?(As@5?>JM- zZEfoQhoS|w)3uHR`jf#{t$_Fa%bU~z1gMC7Tn&BA9{K_)9nlO?7uyk4T!V@2vg9Hj zZ$4VD_8f&MuI&eaY1*}3a6D@jn2%RPeWVCxr?^Jv1P8E}=P^p9LsENWap}%amwwIl z=i=GyR>2CUw%hx;=w`YbTj3LBOj}3lTK}XGs7-IfAN<@XDuqsOrm6zl<-j!mY(ur2 z7)OG(v84Fi`Bi{P7pn*>POd+_-k3;U0NSjj#%q+@;LE;=0jFf+wf1sNU@GAztaylx z0*j#9x~%a+aba;Z-CRRWF^9smh_0uA?|Mx(ZY@BGW%+}YB}1sj4zSiPEQQ2oMaYfh zw(<+O3YBSiH|RO|v2F95*I(q#`YmKPXI?`?@)HDarDtUwxD;GtDk&=)g7gVJ7FC~= z=5!4H{_lp%`@wl2nrP6I0xvRNt{8HDg%qz*V|Cj2C7X@uV+)7-t}@#-^)3AUE3$kX@`7>hc+UIGKPPRm zPw#A3cnJ}dHtnu$j;fiyMBYsoUTUwd^gL&r59dD`oun=rdgKfGeEp?VJ9lxK*3c5H zRyw{5+I-CKxCd|RJnN5-k4sQuCJa+YM@L()G{-+-hD!iA%@D!%X}1x9Rg@or@CN0d;655+5^q8N~E#_%6YC(MN z^4F~Q%~5LbIG3V3e`oez8JuxDIynuGkMFGhP9@l0P5Z@@>t>}sMUR$JvuFR+hJo&e zjq>1+mokBA7`$F@4hSJR^j~TJ{%~e+PAsmtAbuCS`yVmnh+^ok@XT%GwQO?pOI&?M{{#@dSgfg77XzX+$prRwGn%}249DW-Wzl}+T2^_K7VA$o)0zE z81i)^-m)od{Sj_YZy2Wk4zk^1+x>2(L;>8cD>bubs#eS$sEY0UV8tjil^3q(exfdQ zJqne;Rqd6m>+^zOZu1#*=~({$x2c}j9MaW@qer3HppK8v0bTl9mNc)?La~WpZrcNp zSQGoxI0^*Od|B%$k-?caPW4G-+wm3l=7hscMd!?nsQ+{lsCV0;py#>XaAnbSGcbR5 z@Gf^}agJCl$2c{^^v-u)Z=|_19U_Q6t(#5Iipx6cN4wa>MNjI|9Bp$9137!i0Xz+Hf%@GhG*eq;K`HzN=Sbb|D7mfaKev-7w(o+qYcY+;v^k z2V4PXri0Uu-=km*>z8I7@-7?#E+(CYRvGodnW0M z{CF{|^edrfZMs&0%k_Hi=-YQAnc7iSu--$e6nVC}^bY|)6JV6Cl90bI3zAm z_^6W>2Zm%D`2r`!Kdu}@Bs1NQVuUYl8=v(a`;}A(R(ep--*?-cj%@4Z3g=jZGO3q; zdz|zvH$QdKX#vkvpBY-B(|w!c32G^_+{TMhhYmOULr->N-Dq1brpvF!|6)emc?4gL zR-U>iHqiD3&y9`Q_*ans*t%G(dVozdiG&rp#pB&M1mo28u_mhDvo3lZD)-v1p@7w= zM}G%ABtmCz@?jioQDC+~cG~V&1L)$zRq#5T96y@hAe-ct(FArBH(KX)8u9Ka2xj&0 z9re=+d|s#ve9T}V_USV=6Y%=X>~Zwf)+;Z4WJJfYDRfb--3KTjC(Zsm1S%iiJifP_ z8}|)v+$@AR8Pd~iGsT86pYE@_c6WB7uTIcD*l!cHa?C z7GbUBg?Bw)Nj*1L1V{M0a7iB2dh;o&2|Fr_E^81`z{ z{~ql9%yOOHra#q@H|B&0oX|B;4Ro@DDFYhtx6IvtJ`<8uzaQ%yIaLFJvC#_2VTn>E z&38xTF@s=0ug3jN-CVIz_qo?K(aSjB7So~{BW7u*F%iTuIBT=>PEj&&=UabX6|emK zc@SDZbJre}I$+#kG3<<%Ae=tn z&1|YJs<)x&ddi)&?+MmCd(zD@Q&1ybzwqCcsA##fnm=!u8H^{o**7jXYznxMKu}t_ z64tvIR=Nm2wB6mARiO4h6KK5r8LfFSyDl`=h?aZ(&48-9IpDEOzR5_GArM8ZQm+K1?bA08$JQk%DDw38t}H&ND`PId#H6+bZR$*F`S1hBjyYRtd)Yp)uKVKGvSL#i)pa%?D@s#x?`q2{o_bMmtqlkqcM zi#MNQb!?&vkH0r~8;|2|I~GXa8nDM*O5nSKM9lXC9WB6~{MCXGi};Uup$Xr6goxQ3n1$o~E4_FH6-)7qG#6XWHtDfNtSBA-}P zpITtwwzWV7*v2I<>x7%(s~Kme-`^Q(x7(szAEum-p|GivSq(PfiM9NCOCqVElC~+cdK-x{-Oc>M~5^Mh<_c_aF3Li4;_IZ0+g6ZIW*7ly;HPpeRBBTaj}(zAG;elA7$Re`k>SvRZYSZSL;Fpi<*Ql zZP{`nJqFq4r9yYR6*T%Bxzajy#WvBs@Vq@U;pX!~Ioe1j=|G){P8Qljrw7ukM*!B9 zg!%a!c#tl4-Flj{=uC!Ph(-xqBM(6NFix zQ;6KHVdoDbQ%n~*Ekn2}Ag_z&Gt8a1cu1r_B`cBkn-<+-aDjFY(eIst{+b->UT#Tb z2@J>&B?-YtHCh;k9XebM)jQQ_JdzRNBNO5lxvv2oY{8$Qwn@}SGB8|gPyPC^gh1>H znMaJfIHt&fgcr|_VRYta)V>JgBNTaOhT2w@h z)cd+0r5tzO>)8+QrPWe~Oe%8AZtc4K$k1a%7W1NaVcrdioP0IgaX*f*tjaCq3#DHS zB;8qG%%a5CpFsDE;Kq!iqM|RZA<34Yy9c?{6EoR60`XtH5;-x(b#hb;h0 zK&G=-NGmwE#z>DGC90>f$t3eWb^2L~9}6wV!G5UEDWPx9HU6_C2HG10hz}h|U2hEl zjqtnw_76?(aAslJ6CUh9Z{=>9u*YH@yi$K?*7cJmh&E4;3;4mTkw$9)x4W+c%|&XT7cC_El& za@C%q!f)CS8~Pu7JvfNwShvCvPRu1+BWgkQ#w~+y-e6kGN{Fg64VM#7VtUvJy1xXD zX?MopUgqp{hVCT@pCH0-OWr9*eB?Md<23NJCj=VkkA%!ddR%pO)$kR@)Vg#ysF5&x zfJJlC8j@mu$ghK`!hZ>mQUMK2u7A@AJ`GVZD``4t*+Au2dOf7Jj%5gC&!<2(l^>P1 zY&|a~k$Ivvq>KD9&-uDNDcDsJ?aNS@1QwTbKaclx_c&+#$l{*kd6F)AQdS6968QVb z?bRZJ-zEB#P1T{^YhAU%xN=Sz`i6Y>{B*ngj)BKN7eHC2%8}i`Z}`Z_`9%^%Jn%aV+`kZ(kO4?@ki1>$G0u(NO77%N#*#t{Sf>d zns(&ZD684hkI?&m7oa`WA}TRQdZeO2z){U;s2aR!i9t_EAIZwg=;^rykBsro=ZVfU z=ihP#3lma!LVqlB!o8wY#H|=A(udMZtxcB-n=Y~u5`!YY@SSIsqq2t)%Q52|7y zcO6|8K#F-Hu(lEoW+C$|-WR7bEWkeTU6(oH<@t zae|-#R`?p92zLuQSSmAEz1I73*z9bF>A4L!kXaBk0AxE?%Csh`$@zAG1_K!z?JK}l zj_&iJ3`T?mQfr9qfmzmsd%%xi5%v$fIDH&P=H6KP*?rgy?4__K5^JXUtQ%xJC@bPZ((M_NYcI;qC&m*wj>zq((Ec5YT1#NB_B)Sx{hej4_Wq z#KQ#U^bw{i^L}98FoD?OskNza?1%(R`voPk8F!LbxD;XD;>BZ#kO{}pYavXb( ze|o#*h^O&$=}I$anT8iqv{cxGz>QK_H`f5WMYV}->><)j%Gwjl8xvy41?;BXw6(*$ zb~pr(IrguU-(Ck_bc5H3@PWMW)8(-G;W&0z$<5}&6f}%l z!?T5fD2*88fxYE!CQ-kTxCIw(4(B01K#>$LOCQMN6lnGdZZVMvF_8fAG9^l!otOr& zD?Z#T%31FY6NrzRNPv1tJAg4ZL@k591wRx9KeSN=Xg7k_?m=DaI}bBAKmWKQk~a3n zZZnv32>H{$bc(#a+S($D>onlHb{KFPlBZY3|K?`>If)6*x@`{RRe?~6_VZrNr|CNZ6N(zgBBf*VH*_-fm zfWQJ(90QFMVy^$0&p#f%{*STnKgK9OU&k7M4MC)6|A_o2=KC5GdHgjCLRQLyn0)IEq5sFyo-Z=YTzrz_}B3NBY8ul zQiEWMIso{K2*6KZUbF$gT3K;%@a#9u%a0H!u)MNzTjU%fss#v+_7_?9{u#U!gvdA= zV!dWcl`IaXi;ozH(VT{9QkAu}hy6Dpa<2&$As1l>Nc=_AYh+-1z~rTSVL}2HYs)qk zq*0CLygm%Xkc|3o|BbgnhnhL8$o~EPeez^E<0{F&5)c>wK#nwwmkElBLWOWpg@cQG zK5pleXyn>1Fulx)?XC}A%-KwTSZw}L{t>&6#w z$m{=RV*W}7qL%;b_V?dP-fF}?)V9;gdP?&OK>sE1?n4S1iv0iXri~_K#q!sNNlXcm z8E7;;8*Jt=c)(vidie@LFW>n+wRFjheYSw*O3ZCUjBRa5M~nSVjcD zh7|Ar)S#fip{=*CiR|P4iz+!}um}EWRJ9ZR-&9|tHtPQ`8QS}Kyy_M&toDC0NzP6) z3sVIFts~*0H~^+KbI|ZL0Mj$?NDiZ@{LD{uql$Q@sB=241g153QO5Y|O@955%?+!l ztEipVwSF*K-GIP`6x+D~(>pp8KsO|@9|t^)mTO(gJS4@D8Os2mvH?P`svi&tRoUPS z95AYaGhOJ{iE`LCuYv=uRes`ufsf+QV%Camz%wQ8HmQO`9pHdWLgKOeeWJ5EFpq*6 zF!CldMQtPKu8Vw)7yz)tAO0~yZPQ{8$woCgFs~}kY``0IaL?W$tAsOUcLr)%LA??~ zU5(bi0mRrc`0AL=q2p&BiY*F#j(W-3V z9q2Ksox_TCt8MAbv7M_++=-6jaISyN1v-SMe(vH0C9sfK8l&h<0j8xQ4AyNIGMQ(5 zJU>in$4SL+f<8}RU^m`+IDMb4q+~aULL`3^7O@P`8dC`_!f&XtEXywxtEP=Oxv*Wm zJzTDhTq%Hk#MTR++>Q}fdS{+BmH?VjJ*NA_QQdhVB{eT49sGh#Zg(!}Q^Ctfk9$!p zCRx0RP^_=+qejmdGZN_t+nA?Zu3QcW3%%3y#veM^#GF2dGG-`I?3@;p@wmWGysaz? zF&>)(trRb=i5KdEf)N+fW=uweh>@{$vy_*yRE$O>Mj|XLA|0&$NIs-1h>Hsagz*nFnel_J4CdSGNM=rL5c=Wuv zpI67iCfXkb8}?`jz0D$$CjtizU6fJ?Bhwe(gp8XUL6+pIa<7nI#ycV64`ICwP%Ksq zeb2)Ojmd~ZjvG6ujHz^9Ac!R4Clf3O7a_yKSSsSdkQN_+=n;uQFJiDQo4=4}LQJ_N zDY>Jbr|ebg$f*9B&l;n7h}wA|r5$eRCNG5femBNa${b`|XsL$=UNH=IIf=G*dP+oC zTtrGln>=DDiv|4{5kD`LaQ8&l^jCokE$x(*S)b#<0af$7kT8&w$e>{vv-~#;H_Nj&ap+-@b=9N9Sr~D<7`kv7ap`em2^baIhap18%$p7g7&$3` zk${j#DvLX%7NZssDK6M7Q~y;EI8rW>e`lM-o3H9euAJAdK8AX(?gd*$>>p9zpH4t8 zLayj1$WR(UDY6sxt3cI{T~4w;bVRyeg%6jEk+E%5rCfyXqDa{bk`5P&62l@A7%_5k zp~6lDH?po?)|V*)ha8NCL8hCfq9};Vc&=Wy@kS+2IZrwNLSK}{!u(E6vVTn30XfN+ zTGe0q++DD&>`wV!Em*E!rCgaIM{MVnE-pkPAeADvQ&=G4QLYThdR-$cJ>MOfgfT`g zc%(dyfU)zNG9P1u-w%{tUN3y5U}R{~{&L7S5J3!L2CTw~!b;{*iouGSxv3y2140cU zx*ma!dyl2eK?RG;NY*x*PJpq*d#=t46FegA2jRItQp^EbDx8<{HA)e}45BhbTW}EL zS3jADVesMN{)_e3+LjNJ36kmG5++qw_bSVR4vr{%RYFtqvK`2$WVQTu9N)qVyfz1Q zaz9bdN$Vs4?Ieks+DmLccs2z6`%S9vVJg&l}>fy`ZpL$TRvtwMoB${K!fw KN_-bH3iv;Dt$|qp diff --git a/docs/images/quick_7.png b/docs/images/quick_7.png index 24e6c72ffd0518a802f834e032cf390d664461bc..f2b611bd35c9a36dc22bc2b48d8be8d87a2ce70a 100644 GIT binary patch literal 160656 zcmeFZcT`i&_b+?^kuFU@ihxv=rbtnQL`Av?sDRX{fHdg>K|+EcMYJC<0$hf=2D$(p z9RTQqFMx&vuIa#C9|M4~F>oFL0A_$L$`v>UQeYE62&@1=|DOKek1$<*&+zXw-TvQ` zv@PJWsf*WBFFzMAZ}Cft7l6w*42+Nc?Ex(R(9-{qT%)JsCKG{*V{4%-Z#GFckhBGQ zDFbb7nCTr8U4vUX{|t)$jjX3_#*nts+zkN*}+)VAx-1_MlS+3{@^Dh67X8&`7)zQh%0X#(p z>thFR->2Y2ID)if;8X9vbTmkx^aTe6((nG#&i|sn{G}iLi*Em?&AmI?V4I&H&E??o z5bT5Q7f8!L`mc^V|5xF61bF&<}P zW?^Lm8&q=v^mGgi^v4(&8UG3+T?lv{IL6J$bLQgp^ISJ$dg#L|q5PcryNfZ9E z_77+O_ZSQNf5q8gr53#ga;l_lsXgqG?OchIxc1DN72sUfj)y7ja0 zci$N0+ksEu2W70(UMe4{@)(K>>sY`lcp5Y04I_v-*7&u66aL+NOyiK_Lidv zm(BMnPA!(Qz>z;}9M-)YCWdt}y93L|UKz6$ztWHIcJ!CJpcQ?&42>fx;XNnxQJm9o z&TCOdKGh|>MpZVxjrd30n_aSTmi2AnN}Dy^+-5% z<^-o?2C!c*w=5PTb_#OVb?zb|FbXL+`KyyNs-B~!sj%jY^L#L|tg0$tyi?%%37)aO z+Y37Ey=o75?SmIhrfe4CSEDrD7cr66p0AwWlQ`RIG-kg)`)+y3Zr%v>kn;XZt9Tk; zgyhTATX9k|N3R*DjDZlG4#IK~oj?QlH%k%z3u4SxzrMhoam2!cU;7ukUNO~Co**WYUKNuKN1qOI!tdD70QNO|DH`y?_x{4L|2a>q zhgk~SWdrZSHw{D-t;oswaqbP@*>9LKP3>8(lF>X=d4x7WpcO4V(~Qm~DcvWG&BkX% zeEmutfj`+&z0X(n`1eiu3YFv9K|K# z`L#q3m-zLw?mVpLQZp=lPyWVa{(x@lo@FPLk*YzqB7~Eb$H*v$9GyJ!oe@vP5{VK1 zC0nb0FHgm5sVe~n{QCRepFXMqo=K51z5rQ47;#aF0e zTO-Eum$zcl?p`|+Z+)#yez5A2xM}-BC?qzgnQ4p$xWS_R;LvYKjj9oOgdC0r@WP)d z*#EW|t)l4N8^JdDGFTZYJU1!NiNj1eGpeB*Qr(U&8KXOSNongw+->jGZcrr1Fv&T7A|m<)Cd_Otzz{rym); zRb=bs7cjq{xFguQ>Q|3l%9`u|LgsbgCY|$f4%i~NSG&;YMKQK{6O>n>t=f*-?E8^z zC6C?*5$*9$hD5^Co-#0%_1Isge4vVxHn8Y+4mQ(~hxqyEMlEa!a-;p6hg4X|)q9Zv zHRbvuYUVNt=T_N$r*Alj{9szM<0AyVY=)3cNj99Zd-Gyz8IZ!FwP%pGpIuRt@ji@| zkV3?iug$FH^3UjxyOSSHKO~j!j+Q|R@m*b1Xd~(Q0xSZ?n=>LaUUik@s(H-5o1aRxnz6EgiF59kCMEj3KXs}E{4B;I zKD?p7CR^Cjl@Hv{!JMONB8M=-dr)Ab-YZh|{!vVUr~aQC?Ik2)($J#l7t~_w4gab3$w#KzYY;1Xj;n$igEv0%YQtk0zdPr zB>sylo2GTvk?vqM)rGr)0WjR9JWE$u>@QZ|t;Z~v_49S_L8?jdM-B)Td|<36Lf)OUo0ETpS*oWjKd7anTFxvdQI^-xuMt5<#>gqr%cR%>&$hIhtNYFkk3>jFaC@BL|j2r0*` z5ME%H^w{IZkr(Sbj~@kGLmDLVRPw@hxrZFlt7I_ z>~wN0M7kEg8Ux`W8~=bXQTyxF-tFt+FgOK5XOr7hWSL)x7G~PnDWs>~Oym{++-B%Z z=jR58)S|t>*2BlOIcu)&W$u+56HWFD@Dkiog87}fv#*stUWa_FK%BwJK{>4q&Rz3T zQc|pDdH1}rKS)ylTXunqM|d^>cmrh$q~OKiev`$rj(LW8L#w9@A8e=3QH6syB$IMS z24b{d&16tb2jK}dTBBuh_N-ffQkQarx2uz@yd10a_g&4*Dz*d01DiF+s1h6WTNC>D z#He74hje~jLqvHF;1Ai{mRNOthy42fD$Q<#Z1gUHR0bh!fK93V>1|DSTZZRhQ)~M%Lc|Hc*BwJLG zm9U)O zS6+DBGx%L^1Lmi6(tY^~2YZ78y7{0k|NFe$zV-6R&FyK!b;^?^w9;s*VW}TPf}}tL z80ARPIFl&rz;LK5^px6Lf>N8Z?V84LNZ9s6g^m|Hz&nFux8qlGu5DduvqN@kE}{=qdij(S1i#A~13kNTeD%hdV`9h?;HU2fQ*RPOG4 zIH=`u-k`Hkd3_}lW8<+Koc#xHM2sMr9o_tu5f$Ccy*})&5s>ajGA|7$uOUHj?N{)=Exvai@L(zECbPIxY4*2)^ehJvES$};q-c+vfM#!EqjdqAVKG( zGoh@J2H>ReI}}C_=nfgRc)tOr{mXo-uv+I4=ujKs<_}q?Q1c`j(3wawKwPB(*KJI| zTPsEy7W}_#^5ggzAKJCZ^;vjTfI!krMcgZqm?ci{yiuXSXce&&)3F#XH;XApc5kWr zw`XqXU1ncm1t#*O&Pu(bn*)bM<)3IRI~3W2kxxyd2)%gJR5&;Vfmrn9{DN%so)^Ds ztFxlZ2n?c&i$N%)ol{QgJ-!qC+AzN96mI)A?F?)d}*joWzu4d`(rGlR#SN%HVq;_F!0*TnBpt%pv_1bJo6$s1E2 z{pR7zNHD3e5k>%WV(KAN9-*toVkB**Kf$Z z6m+lB+c0~4RWtxY0M!bdX+-#*g%Vk+`7{l9Us6D!0f}U{KiMB1ev;FQ_-=VViv5wl zH)r#(3;i;m?(e%JPj_>u+rr=t5A1(+_wwG1t{eKl{jm#!TTI7)e;F}+GAWQ6R)gNB zLJ99VKFbMEd0;1Kzz>TNDh=qOs-XTG`3Y6~R1EuXWoU>7uqZ{Le#yDg0MsH4gw3XM zaw6fLkF2(qtRDHKI-Yr*XsUNS|2=)o+5c1j|Cocui3NdH6MC{~y5UzsMDv8M&?MxK~jbkZ1KL;6-dM{jmbaATbd<<~#+mezFVcQrdrcE`!MLkvU$nKdlcE1-? zy1A5E;BGUS@vA40>CL8vT;!{hJfG>4IWrM~l+>Wu5=zNkCXH$}z(_-W3`Hr(;Ii>DC&&Uv;nh3QDlA)H~?|bAx7wZ^@O*9=x$bud`ms>r5ZPC=hezQ=0@^6!r;Ib6Wn%${} zEeqCzNLPUqr#L1VypKJ-XUdf2f?}bHkvs=m)r1sNiEvF8w&`Des{^1n7^9CI=u7u? z{Ekv|ln-9j97Bnc#qnb8nG|&kf^kVt^Ki1ugRd2shLM$BMOMN0$wpTmx}HhujL7vh zO=aR+t1@K@v!i~WXY$+&k!y>()fBEbtTH+*SrKkI`o22TB(B`X{gXr8z4|z>CokKU zcv!?=+2=mQqS;-MXB!FGXs0E&ePX~kuVID3muYL2cy{*_JLlBi!cwo@_BH_M#>}pX zS`rwrwQZP}hiT{{Qgi2m&8*tFlEsW+>UPW2obSi`((VR1%B{=ys(d}~CcJyMPO`+I zo+EhG-umF;FKym&In=6o?uJ;)+ufViu0M5AvSzD*G_4q1UP_Gtb~7^6yVvhNxpnVUZ?@mN-}d=BKoAXxrt-rXIwq|1S6@Db{}EKpl6RSS zWSNv#!zO7cajwCot>J<8joxE*WHS!|Mf6liJSaH#$e66hfajsLK5!|?js zXk$cVNFd%6IRyxGnC*_jA)b3v(; zZ-Bq*!6Xehc0kZgS%6OJwVJxBy9cOTsn$}nu#Hnu|6Sqzjk{hTo{$g^OCe#R(*$IT ztW#Y3JC!Uecc*QviA_~?^Xa8-FE8a%N}j^_Y+R6>q%Lf~=EhWdYrL;k$e!OIG2%_6 zQShgQZgE@rJC)ylN?rbTN1tyBp+Q>3%_n4}g3eCsQTZp5+tk1m4KN*53q!j!#TgyX zZn~DO@=)Q&z%P`)AY-1F(oE?qnc8Fl)Ae=T@ykq-A0+dU-pF1UKjb8tf#iZ=peniX zLpjKSxW3q)YIhIo`%fNW($<4yj{M*4RLH%NzC(Xf`jzP!;Cu?Xntixb>3p{61;5mD z!G&&HSAVOemFRR>?fH#uCVIWn4FpG0@P{OpyK4j!X!EDMJ@R)Ovbn(R4mmbry+lU? z6dsH!Sx9t^7K%H6Ar{-Y6RRdPLwpaRn{7B4$!Nq+cTRH5+J=MfG@J(LwF+nM7PEeN z&d0>`U3_f*1&j@H)*4|lPE{n^jC%N=4z!QyWxsFTl$Dj5A7-#TJ>8`;VyNA7se5x( zDtlIL>HM9-@LbqNve~;_C$^)BmDn-4-AS1(h~>xFdrxKZCe9Isn;vx)txeRbt68Kc z4Qgc`%f7s134as!Tq4YnM}r~eAlQ!b83D!Dc3FG14JMT9;2PrH7F<;N9}DvBWn!c{ zS>9YaAey{2y^w0n{qA6iq4u{k@fGD0 z5b!&~xRPb{$dE-yR`*rpZSRX^C*UCo{pW*)&dFd``RD7-c1)SPwX;)8tyg#T{b}yw z6V|W?_4XW|!yU7{Sp2CpnehShJTH@O%XaC9=bP8Gu9XJ2&z%_|tA2oy!Y%8Ju4+L9 zWz$@5kKT+ox^z*C?^=&msBASN1YfzRv=u?P+On}g12_@FLz>1Pjs1e%Jqmxq44yiC%__PE668qvSm+~r>?LrkpVZ4rIEoE1}%drrSiX}~x4{j2SBBEy<6+^kV+ zJy~Y)*4Wl^bzQvOq7r+r*#yGL&NnZvkZBGh3&o}&S$Cg`iIR1)A)ciQDn&sTr%t-A znj$N1n*6YwgjQ*jRy#4gNKP_6DR7}^o@t&mf48}=c~oWXmgPm)DW~#e+Xr!x?D}_G z97R*9@4UxF6Kr1V=t7pvlXop*V7_`-TWz5?Gt)muo|P!=t~|a^2}L+7tvj4rRBP7CF4`k5{W!L=LQP z^lGzqNTnFELUxi~h~dv~VX*JSc$?)BcSz0)9+nG_tJRt~B0r{$3#jXb3?3%^&g~BJ z7tWPykaDW)1f~_bu;r@e(osfBnC`|2#qVEU+qo{-7#$5m+$qJF2Q+|*q-etAj$xK* zC=YKJ`V$%MSJ3ah&nks^RBpu9Z4}3MCiQJ)()Cl~l+pPwu+5xP5%_E3qMXFET+lIB z{k=~9L&>QkyZPpF#m_(q%cnX|SVF%zZw`w)y#FPD6de`dV5Fiy!=%TmVtaeIlY@AD zvvrpqZo$xKR;a^{DDnsBFKhY-6LvZ?N-;TbVe?F#461v2#_xMytEHHStujm$c(jUF zS0sHu|B=Uh4NMQn2&vlNPh7U1mh6)h!^P7*hKBO}MsL1#eu`vqMV=qc-TlKUG&@g9 zSVGl)p6SszFg}467Qg}8y?^7%OYO3va{7zjIg1kQI&P@ zMJ*U%P0Ix(S>xJ<#RR+wX4B;_+xKj?Gim4wtM3c11qC_nl-h`?I* z&e!UNX&Muz+tKVO=R}sKWs?-q;GO$)7o`m0wQ{Uef=ptPUCL9?<&%t&k;4 zk01w-tYeLVpJz8TR?6KDU+k+_YW`NX>)pM^c14z+C-ek5>mFW4)BMPke2ZYi0g8>` z5c!hZZ0v2ZyA5P*10M88PQ!ioyAiiE&*^G@4j6LvEZc>+#k;*OHSE~%%pr0w|T2_#7y5b+wOCPb>QD;%jrD)B!uCDl9NvNYV=V|3(1Mn1KhCf5p@$w%Ckyc+1t zbPA&8knruHTd>%JOnak0TI4diw~Ob3{i9u>J4Smd(Em|Ay(ODdq7ii1#N!)cTqtLX zwmB(eQHf)61~xf+-z_lMcDj97^R`p!Y~bfq_Jjt5o@0pW-6%5e&f?A%9YJ7m3U1SZ zRKyVYL#I5J`!_XSl=v+d8y+*^g&3-5be}lEEX6${_6c3B))4f8%s>cRa|p;So+^!Z zd5^6+Y<&6X&Aw;=Zm;w+cguozWs%T1uOZm&ei3-a+eltPR-V$O+4`F5kh@TIP_35z zbR;|Amuou*r{t_g|ELjU5ACwjr}C@{A_n_>0gcesPXo|gkHBIp`ZFC(Wl zL>ma7mEtl6+e|o`D}xJuWmv>}evWMY@ws@nS6cVgZ0zxw6N0yy@9_Dw54T{?w%p6h zn@>hIWMVf~1hayA<1J5>sU>e#sbl_!3BTtH>M|(Eje(@2C|9yF>ZPi&p7)_=o#gN1 z7?4X36CQE^CAC?(tQ7-#WCW!cL>6Rfcg1R3sx~;NK>nt*g(b()m&EML2!}ZI#Li?{ zJLDq4>l=&#j_w+X1rewbiMqNsXH8V}jczXA0|aw^+5jArMZM*hEz7cN zYBh$+P39muEEo;4x}&(Ni3uUPEz#N2CRK@)Gx*|&u$S6fGYn5U^{lHST5-Sbm1wx; z&8eyOZ++c$3tc z&mpWy1z4!SJlSY`Tl2d6$94W6_h)9&%HCVNn!lW&mp&brQ0I+%=k1=Z`2E8Dn9s>= z`6y|so~LX2(fT+ow!7NaUn8q@cxD<%+7t(kOw zjfkw4^8!3+Ugig}eDY<2y0=m9m#@B7W9}Z*awnt^OTLJ4A%gL;Sd-VDX9$V0*v97g zXPF$*+B43Stco-m5IV8BjF{&(%QxXWd&j~$WOdv-|Jj(gPnN6Q>}1mx=;?Xq#K^Tl zlFsJD0ROA2+^!^*H5i1)76n?29dKJ^t@FGyU@p#@!(jUb8*fkv+w=>m?l`1rf84W) zjV2^7IQI^qu6#hUp#k>wwbUq&YoeF1J0ObNVoDCd8i9#NmeD^=&TZ-Lm)iV1<)vI1 z8xv1vKInSauAz*TxqX=_=8xCURVu3-A^a|>`~%`F3Fjw@DIg_msr>S?XWsHsP+WPC z6t$bmb5rrg1t)ekhKBfp9wa@g#jx5d6h*J%6<#kE`fzloN~dwRS}tBEO(wXgu=Kk7 zz1jiZ{^{ozX5*He8QxLCQO@Yo$RA>-pFl3T`4rm=;+w1LOLO6dpE6A&#<#D!OW6s3Twg^-}U_aMvS=R4J{`IDv;vRZ1N zpE>_H*(w}juXg2(9&&7-sT~!A;!*+4TUJ3w(*C!`+OM@qmQy2}M z_G^%~e&6^GK$n*h1_JImH8i07SsM+|ow58i-%)m;>-rh;<+L;e@TZX&LLeA0-R zibp4^m~g?nI%<(JKS~r&&by9C(Rq4gF2A~df3zq<>cap7-MhH(BNH&=<~jBh_4Dv( zQK`j?OEoWjeln0>DB{bVIi0cZhH&q1JaOjF1)0RLF=TF%;L(+NvJXj{0P7GHG>|38 zh6)%q*VNvso$`_ALrUt#GgF_`?xEn8mNL94@xV3%af|mWj@UevXCl8lAB;RHcO;j-SJ&k$ITBw_vtHfW!o|BA z`zd_DV` z|6b_olsEXcYd8F1O2k{Sw*k^sr*Gc_+6|rmD+S9BfBMtdmXeiq&nTCZuP>cX6n?|6kF=)Fat@cH=FA-j1Jet zD(CTcmEzI7kP`?9-l7^`(C+7QcRl0vkg@8Qc=FE|!lQ`wMZqO*`-!7tWIY19gL5+T z2BEfH%ih@G{@1!ue)qywIc06%UvGc%d4C4hV1ujeuqi_tu#mnvghjuAoDY&6Co>gZ zuBr1xP%dK6u~wpx^qS2+uP5V@Ix!Z|$0atVU`T^agPa|ETHvz1wTI??GkaT*C1?T; z`FuieI1~~>0i9SZnAhPZYcC38=&Ajcim5p^w>=!j9G)wB`3o|5%+hz4mbZ6B<0ExX zdGuxFrVNt>LG|ENHzLrfmTrHRtMAln9Z2Dm6c5Qja**-mB=sX1@!4{F&{`ks#n(oW zO^z&GD%=TrSB8qcRN3^GgkR6LelGuVT3y^vV)cWmp12U@H03=8Ti{D(BegAI|Qbo)%~?uD=i|EyUkrPGhSWuT9>x2@uFHFP*;;F&V{H11Pq_K*rPlTx9?F@ zx0_^B&^>pnNHE6fy3nG#^GYJhw@dY~8kPxav5Bn@In*&2Z}rhb{MHq`Le>2 z1$*0m`hggQNopV43-+ULbGpHfD;WK=K-mhIw4yXqHEqehgcnd&vWH(T=}zmxX0dp! zzO`Dy4`%BJ{lYvl{;tPObQ$6vVZ2aJpt1A#JdCr~Dfsxfm+ZaZyAMAmdCz+^QNG`* zTYog;Bss)^@Wk@V9n<@Q?wzEYf{4)&34nGy)P~EPNWdQ1tB&# z7_*H|@IwFNKAqYu^4F#2x`fHogsjiQ6=B{i-&Xm^^Pp5;q{{c<`-mxtEgDsk^S24# zIrDoHcR#~!+^Q-5_PfCwe(g{C?ol6=l?JBqq%2!d#0xZFD{qwU(DD<)mUK5C9)hqc zBn|3R4M_#Wd*zQR3{U2nG{&9_iHP!NQY`^7Trs%70{^=yOlYGvXr6mcjA!1ik6BlB zAj=P3VuW-Z$Dqq5C5gZY%C#(<2XSoz;S-d1vZ21RyfSk3aU@fstE(^~+&kc5s%aL4 zFes{YBm*z&gjObQrRR~P%UGibbokP8cF%d|ubjS5VlOO8T+x5UBEtRa_BH6oL@u%p zDFi1sI)boV2vO2$NWOT!EjXC5Dqfi9L#zI}VR9ZroTHdw2SKkB9X`jxAHOIj;*udd zy=dS;o&Gd^?QzWQJ2y`(`d(PNuqd+1_fKlu4>X6VKcJiUKy@=by}quBBFTK7rG9^f z@T(&9kLNGINP6NE%{{91u>sFL__5YSZw1@*n0JR>P8mM0zJ72Girfov)+3zAJYM-8 zcD-6APn#U>ubwDsr%+}K<|IUjtYaniO;m2UQU46kL&*$boV^+MBC?7Qi2oB$} zWvG+*niL`-*z42TghO5*2d&1(YA+FW&x4L{M)xM}1OOB^f@SAUIE1k=M7=Xx^GB6j zH}y}`c7cZ%__);g>I?6ELdX)PvFL=I#bJcNBDfw5lfG?ZwQezDt3E6R#*6t#2yz4i zKpke876W4&9sc^#jLW4(3v%O4cAVN(oPlo1pU$UTHhRy(AYQMRq)XSWdJsiO>%u04 zFEm`C0nTVHck9yRPg1U`aGtppwiizd4J4*SxWhQpBJ_8DG902Q`4HK}7Z}DyyM>L@ z_9KQ93MvgfPs^$`9r_z%WZ=V>moBGPFm=-nf|&zqt5~f`J7=sYguow5*zc&?E8@CG9%I!F zH#-rD2QCgXhS;SHat|dM>a@d79i6*EV2aQ*DYkd{;d1ERWFSAT7U-v}SKaun?bH*( zHL+?MP|F!XzD~%*?nJJV3SnKDr_!dy)i!CeddxrorQ(&qwvsEqJIc5@=0DC ze+pibC`Jmz=V~{j@09;Q+$lQU(QPJ2$>A`>ZZp&4hq*3j(Aja4dG{AM$>-5O<5vob?b}Yic##G#_PAwME)u%yaW^68J?EH-5W#PlF_v=*0<*n;At3Xt{YAY!Bj1A^quh9zT-L$(4PtkbycV|)E|MwF^pB~wco z7jhOVruq_cmXxvA!B<@Wu@F%#Ri7lte;NCtTToIs??AUTI>y}ceUVso`Zom_QIs@@ zJKBkulg04w*DRV2@4QXBIvw=LPo!L*z^K>7bjI8Y=B{+?3)9|4DJijSj{l2B*~@Ap zJBe=P581mq&Tl!id7`D8{%Yo8c_nS7KK-(v4^ z#8*DR!o+tFe&mcWOkZRo^X9gpc(xr9giDYf62UKGA=Wd;pT4!!+~Sb~IK_QnjMPp{ zxz_5I4R$uz0z+tT78%;Rx+8H_e=Gh+^18lx+_d%mA>^IU`iz;Ux&c$BJusgdgJEx$ zicKV0Hp>q;8hTuKEab5%;K3ye)VFWm6P@e~wsa2)axSp$3;ASny1KePJ=yBMgoKjx z+f0BCTvKL-b;_A3R{~Dz zog=>>72&Tg%7#*}AlOK<3ra6z7yB7(2~`G@|JtkGcm#H4*oW2B7amMj-=%hRI45# z+?+HS#N%dvieysg`b=Rt?sKJ$o7sl)!LzMOx#PmgY(?IaD2p>ZLJfNehsoF=9O1{Ci8S3QvV=(gSs|3HyN=l*!fB4R(nUJ-9V$Kzf}`;3?pIjZ{BxJ7biI$e9O%mi+L%%+wO@*&vUEJq)W(&?phQ-pqe$_KYzGQ5ng{jv?~sG8 z;gm5Mb1}oCIZ3w!*0j?>NH+3d|7EifhYLk7kzPB`kOEM3D7FU1b-RFI`3=gBeK6!A zbn7tgtCq~@=8OwmO5mEhM!sSIiqS<)3Z&qRXV#D}Dk z>8Ed2RDX$Y><~1%pC+z9D9Dzvh+S7N(n!py1>-LEK*EZ|sOzwu9d#L$H~c)uT7c*! zN)f_oQMjAm4}YXL@>E#x6n(kvI-B&LMcKUUdr6(&udsHQ*jnstoj%edL5Xho`1BZr zrL%t_q`;qOV`FozVYW3rz}c6N?_O}SJ|L=pFi%tgcM6EXS8Yij@@Ch++}}7Xi1k;n zm|}Zrpru>1`~AcVok?3x_SdF-!nq(c|`NP`Uxz`T!KRs9wu8Lf^u#qI} z_&i-wmPv4FV(h`nI&S8ZfuGu+0~+v(aH`k7kN4Cg@u?WY>8ES!f1t`J+$M}2^Gjz8 zlqHF|ra@v5r5>Shq(iNwIL7o@=cG5KXOIZO1$3nUU zCekLNpm>-QxF~kRMsJ8JObS@u(Y*Tc*P(HHw1exDt*w&Tq$I<%m-kXZmntqa_?~H- z!|RbkbJYoLTU|MN?3f0JQSORoS0-{aV{0WG&(P_Sk0!lYuW`_?V%SmbERneX-u^%| zK=;a`R?#H+5^h6@%|*lH#U8F=P_L%8{KYdJ&sZMR;%7Nr;3E#ursvra=lZOHjr`!zv7%f!1^ z|1crgMRS++S0vY2i+1Nh#BLRz9ovnbEXcJs-|28 z3_LA#Xybu!!jmm&hwAq)rUf)YU}mNp!!R zX?SaNPIBeWDa$cK@5Ars=3Q#N>QHY9KKankw@a#gLCj1B+|F3`^*2ZGLRlM>3B*9O zHKmKl4+tkk1%8m(a90~L1-$}`FRcruG_rF=Jh18UEMT~}cr}y{MrD0?H&nAjW) z#Lwe6Idh=J#XMJ4MBY z|4dXfK5{?OSIoFDlN4$G=zaEkOC>P+Ho`hnm?5CDXr4V%&)ET|n%o73rltt3#^;(|1YP3LLKseAFZ0}_w=`Yc#2HTS{D(b%7R$Tl zp{Jn>Iiz^JAFjJkdOuovqV(j`3vlG*swoZF%zSwU`v+>ce>KafWsdQOW?w@lC#Oq> zqn%YwRdwyl5XgeW?~&&j{FFL6je@(nbu33@>I&Ni&?Pp zlhLexqqp2?!&p;#O{wY`q&JNxLQ_7kbn0LukgAt*zPkKp#@FtaN!i`uzSW5x$TsDJ zrdc2s^Bf5w=!E+DKg^j|k*1SRZoCyuM)|zCxW}N{hwQ{u>upBi{fG}qL3kQ)hlI|v zT2~Wt>kDQt-H@_O_nf%Yx#&D+AmzBX7#?Sd=_aasYbw844RboX{?pO_VOX9>7&A7! zrA!%mtoe9K3RdX^2-%osLaC}on%%3h@0ksnn}&{sdJJu1i|a=EHX;=FIV(b*?C{sY!ph-mswaM08Peq90Q| zzomsOdjaEAO=MfMerFA$m)ht!LCMfrfQ9}Jz>g; zg*!}XaRf2$bj-iV1|=*k*44VJ1ATG|?qx`F#~C1dCp<0G$5ge$asl%E<9F)hV?sxg zxcvwTyliNrIyTzd^4iFgO=S&U57y2u!g*uI4Hz4e7oK<*ZB5{O@znUPS>1W3y84kV z=ZqVl`2u^Sp7cim?AZWA{?kR@`qVG4-sd~#(}1$Oi3$VfaDNz-=OpmIx?19(WPT`B zmoozMXQK%9D`4!X+*UTOnV+moij^*&=R!#4lCn0nOB&qL`1e0@%dka%|8=D8A=r88 zdG>3%0|z`C(F(7(_1CA~!WTqsKiLAqY~Cwlo+=cphi00A-s_g`r_ywU=VV~B_3y}y z&+G0b_U}wKEEcqL{!Cn~rlu@3$VET>Q%4;j$Ry7PC98=Wa}A70St;={MBTaOQC5%n zv;(d*XtU=S@*u0@IAlh&hdtB})B-N(d3XrCS<#z3Y51ujD^}!$fz+h^7qR_9oF~8q%auQ>GViR& zU1fuBc`Pbl4E$&4Z_+M{BpWPEq>>7Vj+AVK`=|b5crI=r9QDZ-ld0P0iZ9-+TOZbs zDPG!_HwI9uPyuq_B4#>;m_!``H_FXjrFc)f9tNO=1>>8lVB$gftbnOi-m7lb(CjwV zqsk*&_#H7i>bGnuyjBBKY}?fF^a4qx8#T$(H<$-Yrw#{O#MqERh)a_{Q<74<0G&WpW7wk++wBNL)sL{mC?*>jgXc!n3* zzf~LH^L4SXM2Vjy*@A?O2;3x$KQAC9-ZtQOsu?YL)cC^ieT0ulZ{OGB;v$apAubHN zUQ}ifZk5_t5TitRK=-bfg;qfi4C2tldx|t*Z$}zsNId~!gy3GT+3JpTFdk-KLkiY5 z_(%<{1(;e%mFCiAJ-Eq!4a++qhfc(zT3isf5bgq@nvm=>>c%0Iyn6HZO-D9YUKJc& zOZ@)ZD7jTxf%dEPr!SoIGyqR}gp-~&> z6=g3nRBa<+&DNkGF)+6vBVKq;-bb$Ry#8g$FdYDi1TU?Fa!0X|c-yG5l+4m9xxNyc z_9+oxy=RF+(>E0zo#>ul9nh?RpYA~31%aok)hLuxS+wZy zO7^)xu?x@R&WP5jQdHoT*ufSWAi5rIOPF~6REc}tNT^lOUqN|%*N2D4T(c+l@f3>` zJJxXm^z_+#jzGk0NEV+?WiE!2`T+#!7&6`)|3Hx4Ovj?QZ90E+e1<39s=xV!-yD6b z$-6+rks;g$gE#UJFaX;mmXd<)s}qilNB{KSo?4kRtH1IvV|?=TsX+F;a9A$*pCSI< zw9aRz%OVjEaq&uaLz{L|wH26(u=Oc{m+US>9p{zL-Fs{^{5=A5@$xwijsUj9oly9H!cE3;0ch9}uZ@li;qhJ3~W7HVCYVWnz znrqJAn&CI2X>=DJ`D+Njkm1HXm--W}w;P{2q5DhFlq=G6(m(c66(?d&PARJo553#0 z>DIT7cv$`jzxoPTxmRer$Ed>HdsZt~FGU2?0F~nOZ6RxWbYVnP2j|04eU$Iuno!VB z5Q!{1;C6Y0OpZf%RFMiq7hAwyTTt#TZ=7#lN;sL$yL=!%rm0_zxq&wRBq?pijI@oi z0kRWm?e|140hKZKA=$E8p~SzIQiFgiK@R+7B&>}JHF!#jMp;yfmCl>lr@-!{6^UxN zUFpmx*$@*F$M?+JMBa5FV(f)sYj zk4zS-qF;z-<8E*elYW>vI0$VMTCKdb6_exDs9HCDyhSMF>#{2|JDHI{f|~$3%oXO1!u>QS5mq5U-cOubkm>Xq#eO?U zDgAa4S3CiIkVSFjH?Zl2@r%Ty^Au5pt=t42m*0U3z#5XXRf%>e|&$T0f#nS`W>{jmt<{ zTdvORUS{wchu7~F_RJ5mpE5}q!>M;99!Cn~zB6IW3zbkm)R23|B_%Jxs_qx>_tCr- zFt*y4Vn)Uod$haFt5Rdfv+t0-^m~eP3QLA`A}q12{ZCeAwi<)lz^f44EuJj2kYdfK z+qZ8ubKi{np@deG=PtYz8u%K*_PFOju3JDekSP9T6MV{R2;Xu#@yq~HKLe7`_uw|( zkO03M$fW}5Crb>C$dlg1A8|~1EJgxlM+#QSsvJ`C)Wl%}*N;B|m=y0BkOsRu7_(Ui;ElCd|>cMgE8gj z*YLZ@-y~d%qPEbbk-M4o0z^mieE=!4^1+M~NDdl?ge);@lKU1d!|kK)jG6&SvG44T zFqI-hLPxO0R?i1RAU~ic_md4|=FfUCPcy6s&DIhQ8d8n{qK*N&n;E#sHmG3J>5N1& zMe`g-#)w`jjGyYG1BHFo27 z$9aZ%Mog_=?_wkUoXDu&Pf$pk#OgBP=JXO1G9v|=vn6ivmexZ{*$GDOA^3?clvmI& z;xNc>F`B!0ZXwOv)MiG%o?4yW0xtfkCmo|3{zHV@^Td^lmdGUVkt|tNS_SZae z!p(rB*72ET)3a4j=eGy}{Duyh({ za5Cvzd0wx-D}FngHS8tc?Ob%w^^6;NG>L;L_Ih;Cy+nMKt8;LgGj`xU$RwrLBFkEP zJ)*b*{wjXf{tV}VzDpB=NYgq`+02DkILe64a82tmd4Eb6*1gVBsCGwydwG96oLqkeE_x*I)lwqbh1 zaWnO7h3n&xn%YF2y5{O~2u$>f$&|nR$G8(`_^b^}QxG#7)hkfLsIVNDD9#~4S0<@r z2jF1{u<2e+zlq;6SucdQtu_BB&~;FRszm+w*?{q)oXonD4c=r?lO z!zPtElv@IL!(a))qcHZlyvC4Cr;`u=9B>5L<-@gGm;qdT0vUtv|=hl0M;tA0&_)f%+R?fOimAgEcuTA<; z<(Q(OY6-)2ikofN|xZa48K)O@i z;_LRD3g&w4h#r6B`x621KPnQ4j0Noi5Fp_yUiQI~Uk`BLJ88o5qWvzUUhUb8*Bjp- zQ{1SFU{i_DQTbdM@a}R&SbT#ka^b3`)86&c4r7Vji0Nv-cC}k~3E#aEpEmf^-rAJ$ z=&&<|N`?A1x9$K7k1hLs^Ua<9WjsF$pQm0w&`~RUd_7OQGE7{TW_v|Z*$?L8YtF+( zfK1BNh>sB54pI#N`wA?Rmyjy*WTG(fjMl?N`2{K=z5K5Ey%hHQ`~ci@@cXt*Qqe&U z|8(Ey9&%pGe(CH{no96yk1Kt%?JKT14dD(eanc93rz+9x2N=L^--ds4ha4wMUOh$W z2ez@Vm2=jC8SOn}StsbH<)KCMik>)bmt|J(4PQ##w$_CZU}TZEx(6Q|AAr1|Pmn|9 zlNrxPw~}AXDs%YbE$v)mvHgh4VRu`L;OHJo0;?kc)biNi&j9IrYn9`j^O>jdXysMS z2=UD8o%UqGDD=#y@h!cJM18$+x_T+8u#HrSyVYK3J_cls!s~thy9ArPBOgZsJUiH7 zbW1=+-SNex2~lm4YTlgKYO~(b9g%l!BX{o-gFSYSK1}yV`Q7pnL&s5TVICqYQV@!P zp+7-&&sSUS{RB1H=<%T~{UOOQbJW;=eFoj2v)u)0O{dbK7qU~((opq8y;epkMgfVm zPFqH8etkG0<7;$y+*6nycnoUN$ z5aJsb^YT}T016r*+2$#=UnV{#6k>DXc$r3MZNKo6UZT{A<#-ZPV5ZaH4?(ijFw0K=msnV_iYDRhMM3n_=&-B)105;AA#@|Yli&s zRX^f}9~4ze;UxTljD`^@-F-QA*sFwX0KC1zKXi1^=_7*&RDM0n^cXTklrv;DuuoDb z1VfL2h6#byrpq=yY^I3B{ZP$6hp;L^guA&9Cr=;T>G*i{0b-|#haKtwg>=a>aA{9f zjLeCpJFbG-rz#CB`7ahPjDChYO?xH z=fffZLd2lmHFp7n_b6x?OCkh8Z}^rYYggd`SYj`t$3Tk7>1V+9h1Uqwk+6UH!WmcK z(;EL(t@U=oQagS){_Oc)31Iet1FTgP>{;7%y*ZJY$yA9Dk_ zuq)$700)i!W;^w@=a|iZy`l~LzxXP=b5jeLgHQQ?{=ahUHFwqd`}47XyLF5?p(*^| zZu4J$9D@1s>nN_ZLL?!78C@d0y991oc|i@l7}DVtz=XkfY!=QhYwPFsk1IetMEJi> zmn;O$3EMR%at`V8jB6~}ZuG~-&ur#X->5wr!7{mA0`Ab!TKG>;p!YIXN*N=$cKYBa zDBJeWOh^2)PK-yVJa?gs0a{1_FrapReMSNp$pkb_{R4poX{c|J*@6PShLWX2!`%~N zLYZDYR3V0n@l+Jm&?w#s2_TJgdC{8}^Q1LneDU3x0Zm&|e@l02R>T4*siC_Mb;HaFib0xX6dO^EHB1DW{1$$@eK-jM-mTbIJfjei_>c|_I@&E_F zURlO5F)|8AEK^5#zglDR_UgQT$IWWFU}cW;_GA9OkF{rS#=>!!Ki;*gi}^wa8xa4&9kU5ADX53}M|ndzO# z?aWyM>>GwoEJ+2tuU&Z+j-k3d)k3RR8ly|QJ8ne(1R2&!WwlJ%rD@q4kn83~d)g;W z`s;Q%KmE+sF_hEca{*N3{NE&=|10I*{QqezxIZWV&1U_dwfNsg7IA-0U17?iphk70 z<+R&dX7cQ1+KQC>@rK|WmNkx*3>7+>N|TVV_rcYxTBNv+I=K6?pFHeAFe)fcC^OCE zJ5q!iF?e`C0ZZv?hYga^#9|HLho1VmFLHA=zhQ2vG^qq;cCM@`|Y?6G`S+>iag$O}FT@Oy$h2$z-i z8*TX!;T{vxakl~OzA{4Uc=q~E#$bMTiP`qkt-Xx}lP?nk8~r$mG*J}tu{SuqJ`+rJ zA`^BX$2{BcvqQi;9S@k}x0!~3Ez^7okX9vwF)8LELg9YmUiygTPp}XyWTp&OFq6`2 zJ{MD9{EGGr^-Z4Y+vl;tc0`JdU!0 zWGBsrQDxTi;P-Wbyz$KzNk0O6L9b&l92X;3H(zI7VXA;anHI<@*ED?d>Eut)!A1x` zkuVd3o;XcFo(1_pOT7pbkcFWb;LB@9hZCDoeybK!8Ry9X(7ORq#S8_!DK^S`W-fDW zNbE!ho8}zv*P2rU;svR zX##x1qlLvkLCb?=Shd{WSJ>wY5i^gwIuk4=Xz9w8}c*=K_cJXA{dpw?P+9Fr@f`YlDuzaiyRO=9IhDrF2c(VB2~BX(L|>7 zfZt{k!K+|C7OyHD01u{@fqkRI(}6q1cyBL~a)un-U4le-n_ooszRz{4(BQ^BDBtVzW%v`AxTGNOLlkQRl8)I)Kq{>XZazNRsxvF881KP>mRzpO7^&WkNMWh zl}~d};;krBvXD%CX*@uNa+kc|>L`AOw=Jm-)9^ngiS~p=y4m?+sRnh4&0VRgSh?D- z%H2($%3r1WaX4NGtk3yg6-2mmNYEaNK2_Lm6P6_ZqHIZOkGmPpw+0RX5VE%d&FHpXB*5q*+TBJhYF9&&6R2k;3Ij4q@p0kT; zBa;XycZ&&a5e&H4NNSS==rC_dy4VJ*y@TOZu^Btu>$v4l`PicB{B}y|sa8_gM{`I?DvZZTMU^7y_Omy00#un{?se3;@I92=E~d(0e;?(umF zzv*K<<)e<$Th>PIEkz)aJ4OYWu+p?Eh9z#;cVdeM@*a*BJ}}ifLC~;l|NkM{ z`yWBH_uqI=vY*pemFZiOfF(a`j&N9&+HstMbHD1JQ3P+5VQckyfJR zeP?87UhHG8?nL_qwn(bZEJ%Lle?AgY>}Vzt#v2h6>c=(>BzQ|qA0N;RIS?}_N4k+w zekxFuQoi=CukiVi(%0B0hL7Lb2+`rF4BrC05pS*5VE%shx4@L(Zsq>C9a-{mADL2< ztuws_7_4}9VUU034qsu6`2HP!aaxy1r!DiR@Gh}(_~`7P2%wb=tC0KqS^%z4@NADP#~yhONHRF-VtYVpgrLnKe|R|wSK$gm)}s!NHZ%Mh41ji;ySBgCuCti)1c z?xU370!0BDDBIlpD9FQ^#K3#m^=6(@#ry=R zpC)$xE*rdsfdb#_I#4q8y^|?5%EfoK3J5uTzRkrNC=`9beN(Rtvg6|L_k58pGx?`KTt}<m; z%*-Wf5p?^pqQx%d9T*CGVWd8H+mVM}R*0mp7yg}-HqFL+g4_GUd{XJjs4mnnVVIGy zwOru=Zlw5~!b@o%{g98c)W|zP9ft%$FCov%N_lK3y0c@!AvG$fG0w%+#*-#7VSu=Y zr~CXRk4d&k5*R)v2c~hz1Yw1|o+_ZZUgbkm;bA^%$S;gLrVmdX_ASEqNgh*BXFMCY zedz4ZTV_1-L;HwE>^sBEt}nuu%K()h^45nD$ZY8G@DrL65>r$6?(w+cQnEQzBz~TA z$S`c(s$x<*AM&0+%EXDja{Mtj?vKt8F#ZV{2=jE>N0)dyD%%7NeK@+u9xT?FWUd0O zyqCH}m)jUFZkS-%8$`###%%%+#JAy;VVXyDZ9@SglDO)Q-xL()pvt zy06CRoX$F+{%i<%Qb;%1?6Uquy%6nADsY+ruTam3I=C6jK)RC2MU+zLMNCYV)1(a7 zZKbYtHHG34%gf#@^q0l=(VwtF(~w2lj8XWJ)h~W1w{UgkX{*NeI*HHe8al1bOR3D~ zN2Z78^6vT>T|NTHSYZge-Sbx{2m&18?xc6h2qW_<{8+iFbT1kD+1a`^8Gc{s=^%G0 z3;T}&i-4x@#ag`>bfb6OqsFyBKn7h@a)c9dU{*y(vq>7 zw}Hj7q~HD@6VmrN-eM^(82{If6#P$5Rs8RWV*I}`Hk_X`0F;pL={#ikCzNnM`RtW~ zl{cdT2tPh#Ucg%a%W0@h1fbRqs>DE$#OBB#x|j$bwPJ(K*KETX)^O*}L>5mo;i?{v z_aSFi4-oCc9UNCV(o$J~BNVWItmyETOSEz~gYV+8wio;fiuz30@yxlZm)&&pG&-*A zE=Lx36Q!-@dvS6Ve8I*`dGsM_@Xlt%&ETA7oru-O@Et8 z+9uMGx{6Y8L7ec!SL(DJNexBxpq66F&^LEJU0Ckui_dM;xu}b$?rJ*=^tRQ8;l#f3 zetBEv(cX}cAwr127$}hMF(<%qMf8A?G3c7h?v}ks$(dk`jf<#c$9v!FGkvczA2%hf zH3u&V-)G3|*95W`3Uh zooD0G=f@#YvrKBlk=D^|J}_kMtdC@Sz?Tac+7LxwS2TUUv53JZHP-Nm`(_{am#F;U zLwt{okUj?eFllAft^SBLSagd=etyOK)kS5w8X&ICi5ScjS2rHKrTT#_wLRK7o&mB(d1;nKkn+;EBE}^|J=FGbdlUxldIkWE4OV91!*ery1 zST;Y>clOJvj0D1@GV(o=H>R+(4e580!zr#Mj*_<7o_RZ|rjujal97#;JQF$-E!=YI^h?vx{8iwG7|V_=s)c>&I}NM#6d7P zrxmIe8nw>#d>v!u$cb0g9zXzU%4RF`Xp8LX(j1wq_7fCqQ+VY9nE8h^ul>1yyLc(| zUo1%5CGoT$ul{6+F6j)BS@n}#Ds{^B7MVI?uZ-K9zr6Ij+p$*?wzDqny7ywf_^IPLAkPtdga=h&_ocEqBgtnPh5 zvL~r15+a0I^{No=5}M#+3V}xRSevFrp~JXi^3&d0hzJ0*n~<90k@Mpa>{TUNzaQW~k>$c@qAkGTZFlFU8tRL_ecQY! z$Sg9M>j#Cw+{LO>i4sHK^2NNrjmJVez;R6y@Cmgd4&I!Un}5=qYflLmGwviCDs`we z?WT!IaL4P>+ar2w6WP!8fmrby)0)-2VL5_{<=NMEmVO+22FOO6LkTT0)=fxTqHo)-&r2WgKzsys+;%`@V z%bn^NHx-8x^=2sUu}C{c=M{gVr@u}9@T?LbFJ6OI;$TFk>P`7vp|{>V7oChUQe`+r zo@-Mp5D#l5+?l!_TL?-$m+D@65V+DA2(+_-d=&De=}5}&F5iqP{$^@a?QXDFnb`m8 zwvGQgj!tgA_c0If{bhm91^=9`zYOsFDZsse&O~+&H%`@G!pK$2&`^`fy*#!1E3^`G zx;4Z3*Q$>Mmi<5puGjEQ*25cZ*5Ki?w=+NKw=A_v8lP0D@7kTD2?7?_W~)#^y-+}l`3$&5^aZilnf*T5#o zdt@az=f2pgiZ|2}{DAno*3HI8au><}*~$sHEj}o0BY{#7apQ8gkqM9zFc2 zvO@43z~MTks$X<1=5NJXcbglRW!|1`)8Jis&>(B9V!cEn0baw&M=tuU)Y_|*0{BQC zhvF$hV=7bG7ZpPMVHC(~Dlt@@IB?tnN9l-%`Cb+u^Vtziz7`T@UsS zKS@{X`w60{6!7ErYE(uEq^HMpU(RPcSWdfqZqzn1>USG(;rV0rw)mtq@52GkU=Zk{ z`?rY$|FRMIHi*obd~{I5^=qvor0DSZ?hDVLJ<<7jDMKf=^un`h840HrdM+Rhh4G9^O9_S*I=T!x{n1ihr+?gV^gE)abS~nYNj{2cC(TGIy0Gz-oMOkzI(yepv*gC z1AhmV`-5Uro4wi}XFDRxB zvkca?)($!V;!75=1&I2V!6)Ss{j{;$ zT4=c9#P{*%6;n2WSAxcG++S8T@tTOb^mo#0f9!+4pMhZ>AS<@dx*!BF)`=5MO0Ph* zi3)*mf~v8C&Y*=Mk|x48>*173dKiFO6?C5ktn>Gan1omk&7_g0cdhiIWU`&nn-;J^ zVj36{r$gVx=yzPQce@iS+<8{aQqZN{N>1XsOeZE3dWt1LXReTV_W;epG{T^zir!3# zFVw`!$C|^ThBO*e=c{<8Ag4kCu2j&a@o!sQ@sx?~t5e4a?0XRDQ%mPL4Uy=k&pYPA z=%e0D@?+CyUJxae$;|8I_XQEf!&q^n7fd@n``7T|xg)y+B32Z)rOgFvE+ zUWCA-nO-U;oD--LZ6@IMuWN0rwg1xB(}q)_+oyry!E2onDZ`keJ6HA?djj^o+aG3+ zexDS8i3zZZ&IBCZuzVCjZdvnahm|F+nGjSx4EXLxcH2d;zN~;B4vMAzW%4UiFos1K zS7ctQM!HMIKuNz_$}P_`Bc&uK7f<_-3e}2C2?OnW@~_^0F6Ag>)wnOVzf97m4*30J z-iVr0GVVC3A2>~Ad>^xpzG1DS{UtBkRMzH!68+O8pdz8IV-eBctdU|^F*C{?-cN6RPF$bcK>T9q|Agkm;ozo|GWrW@<@YC`Q zPPCZA%l=-StAPIQ!wyfeG_8pZ=B0=8k&L*ar=q@ zPA*cy&np3B?S{X+lzAqE3}cRf6MD&i$|P)~5Zg$+R~hPD&{j7w^R(68Ni1v+5yfUn zZr!kTJ1`~@B$9dzv`9Q#62G-39JQQM}} zaqjXAZ{LpRjzy*tkrf-aBctE%D9_e`>m$f`=l3|JKhaJeiQ_hUEVa7!y|(QTmyB8l zB2f;@v|f7!d^x5|4h*zKd6+n&dAg`{Ad6Tnh~l2i_1Rr$Pzr*yJO`tIQr;u1PV1eN z<|!S>HCD~!cR1BQ)jN&u4xSec+Gix#Z~DXp@P>)}rh0xu83^b>g!wHDPd@aeqr|P( z{<7dncS7gNI=2bG0bkqjZ8p;$b=+WfVj>=*MmBSHs(@UQF69T zT~HX<1j1d@W30aAM55vInQrfNaIWuzz}mlT_}t&d;FG`78D>u5cULpp{zA5#Aczw< z-h0d&bTmaz30h}tzuIV=rFNw(y;jHn^%d;B{);{h&tN}xj4?X96m2Ir=ANcWTECQ3 zV&bTOcaC$eZq7D~-*f(kF^-KzD@hX`$4fa;oRhP^g@F2BQ@MH}4BL{^VaOJ>P#_*G zST|ECa1!Q|bki*j@7A>yrTdz7y<#=2eG*buNo(7wpxj@8VX=Wji|fbUJxips48#CErj<2>3uT}K$#p=3+I>L$NZuMX@G71>%B^ijqN3h#%Ft-B>9ACqqO~P z?%Qy95(G%;wfo>ABl^jG7(#klQo?L)M}5?VLdCr0RIWQpnqrV9M*Y~yQGNls{V(V& zvKH2c(F)E~Lbir@wU@Mtx{p5hSmWcys{F{9vifc2{ak^zni(=^4Ig9s{c*3_-VXLL zi`I9>X$^HvQ~V4UYhN0Pub1B%qE<1$eUpLDH=c&b>=S*udVk++=~NxAT-7ioCUeiJ zz+3(CgRdPc1LirhH>QW=@lMaFClVu)N;~Ja zt~|w-y2RJ0I%`ak6s;9%NpwWqpiDCc?T|vhFER8{M)ik?wVHBipuOEaok-scyBR;6PD7YSE>rp_u1O$+Tk!)6?Hf)h9U3Za zs1Ju+eG3%bH#GS(Jw)T>Ufyz9P?9rYE?uqcE!RX;hP#dI$|PSX28S>$)ucUin-W~; zOMVY~|AhHTFKW9vi_H7)71du3h|h8-+}4VQgQo>@xI-Clfszxr3jY+Ee;S*_Uo}dR zGyk!g3P*EGz!IG$UizZ8?lX6e3=mDRVd`{@1%ip;CrEC}>Zq)qo3kOE45cWX=z=a5 z#kqYtM^avEW~%5!(&FEz{OlR%jNhwQdr}Ys&RdXoC{eetlNspI{yL^~NYzJUjMsVd zoJe`H5(CZhVqf`3eea~3uMyM~9vM=FoM7)8CX;fGFuI-5I=^Fj&=qsd>sCLO2}Ksx z=AAnWP2DdnbDX|NFjq*SG>$NCW(}TI3$RqlR{2u0M!z`u-Cs84@2BS++MRluUFNvN zjMhe`^j5;M4D1DpE2>>?nwu^vHNqT-@11`M+vwA7lu^r~Vzqwps{DEUq?}QoNcH|W zT}@u&FC6jlaCF6G+wtJ{;R-;2vOa7p+0~yYMA%utO-iMw@x}b3pd@+8oT#raa=Dth zK%SztbDJL7Bwxix!%VHzw@AN}_z37UIrRL?Q}wwm5{Hi2@Iu<|B#q$_vA#-PfEvdn6_NF}lnN`Dh8v2_vG2 z(r3+SI?1P_ru;KXros0coCqv)X_q{SowCRPL;VX{`(@X6Xd+{JD#J3V+;;@z)h2UB zMY*LP?@Cp6AZ?$Q)KO?U}GcwC(Fo=xri5d^(@ex zES#AKITL!kDP|12{>}cvHp0Q1VTh*vZE|*o{n`tCpPFF(1jd-;TlYT~6?BYYh`eFQ z{*{KJWHJPpQWN@OoF!vIG1f3`FjRn@)k^OV|1v{s<+vuW+kc?uQP@tD2x2&@c-cp+ zdTLKdtQAbY+2Z!Pt|1h6c86Sz=cs%v;8oSTf%BUorftL@rf0CM7~^`BU+anhpJ7{5 z=BH_oa?Xv45T?~h!AgoBPhXaNsd`jI1B%4AWNIX~^p_p@dv%7MFh#eboc1hG&%XLR z$h{L|_|f;G99OfTqf4H*C(dM4`@C1oVH#-}-L!alQ7lRwH(rt$o34>A;D0tJ79b|Nz` zZj?^qDF1fxSMbMsq9~1=f5_?FQZ&Gk;eQh}WbA~uZ#KZz01Zu)>Z2hl zX?&z#L_mOF_N(w$j#T1%opzChzE_FG>6WWnXxuT^y=vy zKs{}4gsU1JcP^95u>X-QyUEK~{yORaz*YC-oMlYi@dxUb)+uZvb<6^!hM&j?>vXZsFwU5;k zr@rw$c;OFw9#wWOmm6sAHi`%S1r*T$;STkQcsw2`Wu`bq7x&8NtmyP!rzs|7tjq+q z^&!x~K+m7-mp}Y^YA&7BBDlwS7Rc5iY2C>1+6=d@%Zyb}^Zx9IAIwv~x-L6hkjmW$ zIOxHiPHJluEX3`~QWLE*N5Q#R_Ee=Bs6#X)FWJveyw(#*5%BUsB?IyYP`k%qY-kTA zm1c+gp5i&Zv=>u@3hj9h91oHxmh+E`#qQ15MUCHuCF$R8u*W@s{LK(66WZ>f_z)>E zVq|3K>x+fK?sgG+835xDltXxgHqC{pjg8U4aGWb;h>W)Zk)}eo*9t=bgx{H|Z~>mm={V04yvs&!F)#p3bPKhz&JxV(9}<-!<( zw{he7EZL!r|GdBI77I$Yvlwc|Fy>?CBo-y#jg?GOzJwaHiN*;g2rRhrzGO@Jp=

&k)f5lUdecwLVc;G(Gb9M3oa>|9I-Kpj>ZG?<*RJ+N#nL;%`^Cs9V<>}~cdHg% zh@R`6@$%7FknEJH#AA2TO28?4XH`i5XMfcGyt1Gzlw z;hk0)Ajvef8aX01TyjXKVYxIHHQ@C22VcgAPZk0vImx``aJN_HB&YlKwU-KP#_!`7nVCVrmf5k)rm3?0nATEVgG!T+a^Aj0d0p zxLMP)yXnOdPvRT5QYxd<-PK}KQCpO*x5uh;hR^Y~$@$rH%|s`iThUDe>!Uz_r@sqn z{yzfWzd6N=SnI|F<JHYjS@pFeY|XrsBmsq8XW5B=naw;-3iOirtO)5*uQ{8jV) z*%rlEI~n0f#!Y+NS*JZwwI;pqINUA1~ zZ~HGo_L~Ytk!1F3hRNPb=r+Pn=j==JxS*e`CrTGmWB5>OQohh&tjEP_72-y`yHLGQ za`OqL-W#Q`LZIi!cr`=%}uvM<(6#t#b+m74#ET*mcTOqGH z$k*EG7{^QWh^eZ@{%ejv@prmUH%)|0vtn=b2|F+>%c(W(GE3RCBrzz2zNHjq#d#;B zLtPUf_wn;dWq6wuVwnP?-Z3uy^t3t;DuKN=B|U%$Gu1QpeN4A*6Ey&WB)<^$*k6m& zl_tvZ!fvmvY$V4Z-0NFfkv3mDpL^e5kDnV#Q0F7oj;B~Owm;$^;5CT*K#p^!|F=S; zKQJfMcR=N3yR0(f$7$MqC){ert#x)a$Wpsx)Aj2B?+2t{`Dy~i@ewuNPT;Xnw^Svq zBQx%pJLEPE!^ZKo?;8}?EvX&|^Ghjv#Uhf?IXe6Mp(Q1` z9PwTt(!6QEbeB~o({$1I&tXP)i3k~ZZU#E?vN|a_zt6%ajrTP zSy%JE>}^9YTHMlEc+cS%V~0C&m=Heuo#&yef+JtPu4ip3sj%SgSN=M9qv2&apwq=+ zmlX6J$M7s0;pVsd_iaAkXQx5If@CEyPf@^XyPrE$)>mNYD~1`VA2NHlsvM8ZEXDhq~)dtdI59_)O0jB^c4FcE1W2lF4iI1;?PCghZBoDBe_UEq+iwvv8aeUZ63~;(B zZY>M+dHr?meuZb1K&28by741c7+sDMX404tB5`q zU8u4gkf|LExj4xLGI+!NWk`V(bF!bK|Lr+n9lBTfo%RrN3_&~6dlSoSPHwP0zBE+g zzgs)+(BkGGGl}HBF4ceImG}p1K?1Q8T_11MOs_fFv3Ydo&3zvmL=>)a-I(R{#-zoR zyD{Sf+GVX;aiqlTzO5>UsGv=*XHI27s`4KBdl3|fWUgQelyG@sLI~TkPIBOvHMkJ0lMHoH~4hlN%Dd)m5+$N zf@0S>UnT1uea_9V!E*Wq!VK*a@>ltOMCgP*axbtF3MW&$wJCLY?x~$evV^|g%KJYW zJG}I!9msjU^z$*33P0@sF#;rTqv5@N01aS1I{B;%OFS;iIPXaNItqyj4wUxJ8!MPt zg{u6~>OidcC02#F8P@?+weCj-UB+)?kj7g9GdP~6)@9t7EVqr(2UC_AcZZ!y$eLmp zytkDUo$@kkuM$B$6&_4Le;oJKw|^ML^t*YaNi?> zud-<4MJ**ht7QSMixg`-IQuOE|;;Lde1#K`3hAg!t3^6}12EOG`D z+y;~eS^({KkCz?fn0%>$X7mCDCS5H~Imzpaber%N-h;U;?Aaz%M7?3L(5SIY^(QDR zy!F)F5tO#41@}G{^(`Sg@C$BZ?=_P}>0lIK!G1UD#%FYby96HQPwlV2u~W5nqnPid zc0Q!oNIw`8zAr;&xC*9ksD>4M^QzQ0t9K~&LR3$<;Bhcu9FA^DkxEui?z&t?R zIZ&T^LVnf3SS};P@&3DR+LQX0Ta*v0h4W*Es>9bQ^EbYdax zQ_UGh2k@pG;oqw&dX9=j$=$AulqEnLYa=!4S1U;hU}Bf$!fJ!*8sE1sR9C87!>yw`A|sH#bk+%x@|7p*Kh}cpo05Y7lT;LwCfcLc$b{%JouO z;_*?V#;a&kl#L*-i+2c+zn&wJs>6D29eKM>{(&?f^9!8qB`PF1(C3!7XYuBME9c3v zOSiz#-p17NcRDq?HA+)eK3*QzARU4S52R<@Pm3V!7YxX9+_jW9Vh40y30ka71&dM<3PFQ|O7k72+Q{vSWW^3JJ#YV#H8R9og>L(CNe8{7rcqf4X&U2KR?SN)3|&FUb-RyI|g1ZXIB!8cZhgFdu1WWgSQwaciLQNsp0l5eNr8R zf=HutW;I?F@^>3YV6uSn#tmtPGohMGt?LZwUj+&li=3b6kcM0qR9yW-5^KfyoLbx` zUMu_r0sfy(o71ct9a98no$|{65M)Pi&82=Ax)4Y+An3_N>o4I|jeoa44n0w|oh-;; zJz9{)p?(x|^>uxepm_V@R*2Zn71z_Ll}q)$kQXz0)^%^J(sS+GhWvwU33qS3kaCR+ z9C|g=*BQ*9*{gZe)KH=NPAf_IP)DY#$d@p(xvsM8o>vQHCBpOu*KlxIDB`kw>_>A) z#rO_FgUQIuH|OeFID53doP~+4tHhJl>_)0`TfdlB@J@;Cj+{~UM1`!HCPx+QoH4UC z&`N$vdcNq{s)3B&muO3u2cp-?ZWa@A@kHpE6X<yl}u7MeD1%x;@$-`w2N81Ud3|9}(m z{@BNOD?BS*Mn4lLL>(787f|@!w0`$|JqX6#>3t#`tjO4h?sCxOhSB6X;OKq9Te$lY zpX3DtSKmW~)Md=BW|hfSchB?%2UXpn0+6Ul-QuA|ai&r6wZrh6tPWiR5|85DY&HrQHM*Z49&ry&|)NJ%YqE3^xFZvyCy>L5*nL43T7Do@5mCn7`Y~<|FGI{jD_9Yi6W19x{+{*{#s>Yg(6Nz0;2^Xd(Nf<; z`_fv)>Xu@_`8BNq@kFX4r82`;3;3iU8Q8DtEBjKY9t?;Ip@5(T)e_~R4MFx#P=NlZG)9Bm|BJS-4vT7U+eImnlI|`g1?g6#B^1daRJt1m8Cp6W z1q1{{KtN*18A`fSTACS3KwyS)kl1JKz2EOUufE^ed!KV%=eqopS!?-=`?;UGpL#ou zc+16Z7vk!xZ6X;tSGLT36qPW!Q4qk8YNIUEKXkHwB0ZY^o@u)6mAu6Qm6u;T7 zmBV5h8H&lb3S7}<4+~~!lt2&KCFfmHHXmOA1X)bhp%7z#s?GaG}!XsQZWFC!Nob_es4|S1-XO!Jci$s0J(2^CwtS)WtmokZJzrU zqw6&{jc>~MJ~CB`xsJ15fMr1jdH3Yp!SKc7>Nuk`%jN6Kh4*%)=GyH^o7#OU#kKkA z665mS^~qg_1S$Z+%@!5R+Yp$j)?;#idTdsYCVO$}wUlIKI9C+7ydaV1KFPgkgRoxR zTP<|h26Sgn=37~*$mYB5A%VUWX?PafyoMITzkh-aThQY)J=E2P#XGC+;pOt8@12&U zrd(W~CoYv(EOgk{a5QS^0hPY;OGUtmd(;T; zIMtS)M=VUCxOAHr|_0LQ8}CiT@d&60}*BjB^nVAt~?eVT)fT z_Pu!=PzF(T6@Sjsw76Kk-!ZS|amdlT|BNsZvuBQZ`D=p*P>k0ctYnk(oMVoK8n& zv8WxyGu$Gd5&Y$VDHOBIbS=eoHzBjs7319Yd>vlS7AXOiU{TSj&OGMme3yCi_2mI) z`~51nHW3EfHSfN%Qujnl{PFW!$RPV(#sWs&JQNH6yn$pN_v@Zl_bc`cX=XW%o=LUP zB{cxRN7bQ7frf>%H$^KaJ><>1owFU7vDs8#|C+OhQL}I#iIrR5#fO!ebBu7CM zVDn4ce8aPhZp2crE^GQ;hHAn?=+cJ6px}4p`ac=noa5G{B~r!~K)+ArA$+Rh*usfa z>~8GU09{i%DA8?71120IwI@WIb%Eb!gNm=Z|8eZ5gv2&zCS0izoHu(_F2)Ql(&tJX z;q$dSM#r&AM-tViFMy+_Ks=UzMXK{H|Bi~e?Yh4$&jlgAXi z7pu5r9j!K@AHQHaHJ9X`jXJRaCgS6*W}(LTA_N)6`PC9)y0R{USr(&c5$-NrE=| z&UVsGOW{ZSJqT)mG=X=yNsE_X`nprq6c`6&UGNxYSMn~m z&wmG(3ZbKs&W<|oS@)|qXsRc>57k1V3 zH5f+lO+I1U(KWg=Yv}IpG36_hdG)k_E^%9&U(e*rlW)m>%8!O{nIpL)^e}AQtx<}U zQ*O0Z6wYYV#V=~r=24VlplFLe-j{|@NzVa+O1r)}GF2RRa829g4NT#1_DL;*V#!1> zmUK9KaX7Ig@Q+eoJn`{W1rNkLZw(3m~AzFL>MU!0t0COG#r0|^Q4mPN&dH%$t!>i zGEmZ{J6|t)UzT;4$KuZmXs7N9Xx}r;jNfcMy2Zc$`|bKrAeoU(Gy;MMX|6fyLbWc7uvm^5VO)fv0MGgsF4P8E%ruuyH(kZ0+%uwLW zMY-rQv-3w^?UkZGErjA9oIxJ|!erp5gmk5V046~bd92WJkc8Vaj8|9V`l>$uQA;S|Z+7F#b;1$f54XDR zGcTmpE`GZAbV|_bLHe#SB3%$J^1U1xQ0NzRrPok!m z7T#kLlQrwYXV{w6{Z62|m(!hbYsT6zj;Y=y;aYV!|1ft$G`YZ93FsLey zPdXbx3*F$;LdWToJ70L?ap%nX?E?dxJ>cW;?~z4Ok5gCe)=H}w)}GAhy$AEPEipAu z>>mlq$}#wq5=;m+zejS_2GANM@cO-N6rg2?2OfA2?O4$((gV^(GIfc5a-Kmhh?lU` zJJN9PF!7jTvzz^RZYekWxa5tZVo@a$xiO5(ksKNYl@qm#VwXg#&w8v7p9XTgYm0A` zJW|lB!+8XxN+IW(Hh}@zba5$aHA=hPtC$R)JWR;b8x*O@=EJ1Yn8n);~Rv z;K5z>f49tv8#G~I?x)V8UjzH5lxBlIrbfb_SOXb%y9;h7oQZ!o{>4B4cINHkxiHo- zs9>v+W)J%6%NFA+XWFW!p$@stj7>)`kRSenqjs0?%wc(UNgf_z zLq!i7KFp8Jv*qHFOO5jID$pl%EkBpQ`lC32G7A^S_kJ@Q@P?)kNgKbMVOnXP#MH3Z zRYM$cRXy*12?K4cA$nCa{N}rAXaMIV=af!&_tbFrPquzu|S+Xgv_SzGI418 zuA5MId44?4dTw9BjRZ#{**cY|o$1V$YMyexaw6agr1l3&b(h|Vt}~T7teM!);ouIb z_2Dpw;;Yd0utj=(rDMIR$cc#7sjW|%4zOD$0t6mWT2TIG+s5#LE6hiwOKjL|faNh- zmXqq@{p zqa#Uq&`n{RRAp7FBa#1dMrRVD?vD4}6Eu<+v703Kj*&LwWH)*UkLFN|lFSLwr4)Zj zh^A)zlsH|2Gb6rpOE_&dqFUg!T^OGa{X!5_ zR=Tm{tyK!`64UvGYGJ=p9mWP+K50A*2&lOE?*Pu@{D$Z=Ex*O~IF8RJj{Ltv;?3Pw zl(p;+Rl3Va9vsJicaHws{th@^UPr5?t_m5T$a0Gc6xpZq;)t6nZG;3pBCj=R-mJgg z!2j+o$>!CDWQubptQ9Kz(RY(6LA#_?gX8BxUZ%E9lI$?U0}ausNyQImgngubb#Y^1G9MrOU{B`|$f z1vz@o4G5w}{f6VF?CY-dEysk+U{`wd<;Lzo^ZpO}){r`NcMJ$wxEz-pUVeP$KaIER zyiPh#+CfrFEG(bT|2;wG2^gsKpay1rUecR`lk})Hi~58=H0j(kG1^#nS#Q2~eL>Tq zn%M944YE}Rxfbg~??=FVPg3<5!dS&_sX5y3XBsLeO|YtXRQGUS?V;l6#-1GFG+^=2 zccXrk?c4|L9_0!XtR+>O*PjvG|Mrb>@kP$Vw7}xLOpcpZB{Od>zU`?n^~>(WnP~w` zTPW}Uh+D+kZ|;A8fiS)MHDoi>aQP44WZJeIZr{ubO}?N__5YoHN&OkUPD1yFH&d(( zs{m3^cZv!Vmt*!4_wqP1=tt!!wOQ68*JJ1E&7q!9=Tg8|@m}JM=+^u|weq#BaMHhm z_F_GWCAWJ>Yd2R{RGFFPd)w#>v&8f72wCknZWW4C3ed$x-dyoh!XaqC4ed8g2AO0M zFB2+|B|jE~KJ0lO;gq>pFz78&Uz=U-wpSK8T-kqPL8$g+Y(9guF$sYNGo?Tb)_IjY z_ENb1)1h89;_u(@8MWREH(NV|s61!}=P=<2uG-n6KjSX!%Q+qs(lM@)%U8>v9 z#A0p};o@X&%CljX+gN$9LADsehfR#CpBCOPbh5u(qa%e@_TWAJ;Fa&kWwig*S?BVm z!;kXK@a6Wy&X0~Qe3N@Vr*@}PD9Mv_s>(0(rsnb-rsg(R!9Zbvac3)0Fca9=7?>LZ zx|N@+OBsK2t*J3*Y3RZAKB4mUm+zxJxLIPPfk`6*lRxvFf5qWGngtCB;9do&lJ)2B zFF*c4m55#1dM`7Fe`rkp&s6DOKdn$QL|cXXw$eH*$xMkK<(i{^45b-!_mODJrh3+C zsIRN-dFKsMYR$wdtXDCd-O`xWjKF<+ty7r(_NnGOvd!*gGI^p^ z*L%yR)4PSq26eITk`6{TP(5Go^?Jq(yI`8s^yKJM@t_2{P9?yI=)p@0xe&bGf-+gd z^0uZIs@!9vEaTZ7NH@bSl*_9O|9u&DC=re2CTgaW$%0T1Bd~+keEPBZE*>3RiH?uE()eeY}8Dh2{JsK!uFDmTF0-stp zj@d*o-MQ>|xzMibH%k@Q;@;ALi3M{VPo8rvL%z%amV(sqw>aMhlSD&rkJ3Q|vHq)x z8as>^1zbo8ec#UQsnmy7Q#YIpe!lEK_h zO7GB~CU&t$3Q=}dSm7Pua6f_GCX;0fVa5vYYm7_mxEkFOl$bB9u6pg(^<8l2JrDiJ zxBRQu=;yo>7;!phTFB3DW2k#TJ48=Rg`QPq3dl8;-gFaU$G869kwik=q_@V!okH2o zns_+s%uY0h#@9a*;NXzq&};w^>2maF_j3$-G`j9AaPj_W6PHfA-9gB70Of5W$x+n7 zY^9hUP4`X+OEaTH5TOb>D&Mu#V!LsY(I$ct@8E6qFgN8B?hleUlnHHAqfR(KsQ$+Y zvxM9mkWQ2S!kYpX^6xtTU$|6nucA2zN?&3sbuD?R^bxNND zF-jtS6AfXVmB|NIPK{?AE%ZuGk~#J%c723H2j6K=Ynn2V`PgwpE|FdrlZCj-$5~60 ztshZYcT%Pi#&jMat%S~EVyO%!YqCPU=ldd;{1C|rc_4XsE%mYqPxeikoZmg>Ugm>U zn@10*yP%S4llCNDw`HiW*z03$(ffvt({B4^Hhvh~`9Vnd#yUqP$SC%GN=NmzY2~NM z!XFvJk+FCYbs~WDkyHW_8!Z$k732DCUN7le`T~6D@~Lve?DyDH_#RFQ84=B<6_w&0 ztUw#1drWz2?e20Hqhzkc_?&b~5ATiI6jkvUNwtX^R|UjsY+MOBVN2)Fu%7Qi>`lQi zbHs!%(;z4?gHvwy2`E&ucH~_dzh;{w!`LSl4VY#hligR{b_`Tj@{#w$i$UykP|yKe!JlWLM5)s%{~iE2+$Uc z`yie9AQ7NQHr6sQwci6~I)8e2g~jF_pKp@>h9mS;f!vYykFaqoyz6?%R+&;+{Qn`? z7NA>&1|i)(l{lGI6xsT6H+Hr7t~nMjce{J?b|=)PVMZ$94-It_^qYTUqnVrqwOQB*&Zkr-Y5XF&&gb-6_<>FoIs`ZO; z%W8=puiha?2AYgzZ>m>sq1vz7g7K3dRF94K(mwYq@C8#1XRWn3F0Bk^&nkU-ndQh6 z&~qE7!gy6ASwbmFk-mDSHj$%lFsoc`yw8k_271?{i-OI(m0lmrJucNXETunr$p#3< zt#32xCFShcj-W^!@!zJiM_!(+eHr+|??uiu(^9!@G^kv@6XR_#+@>^+_vc`M=Qaa!f)k455Rq{S={h+DS(OuOIncbe zudhUELxYiK0xeYIv-&5CurConIT>%>L<=!QXnZw?-o=eC&Y9t;J89#`$-2-tx5t8& zpt9=9F#4NpjdRNMcsTeL2ml?&_q)~GjecA-?OgS%JBZqiLK2war?Wmx4eVjAl&8o{ z*8uc303lGibIwM!2GWraqk@i%(X#^Wxe}bOi<8un9T$p+j0vT?aa3UEJXm*>6dZav z4z{VQv5*S_n0D5n%_pG^=K%d-`pa*TRj-nY6ckgDdW2t}zjtO-h%;SOeM*oPCA1xD z>okJysEVk%cL#pW{4m@DhlTB5Vdem#nRby#DPUq!f*IsQy8tAS+MMQ(NUI?>rEgL` zRN?REwvCmucti5mnzz>1>cujtuftK_T#q(x{#;P_?9o|i77^`e!hFqKKa;#V?%Ff8 zfNf2f=D&_Ui4qakzhjBuvk)-vOxtkcsOiqFD17089G^sil?LnZy)Kd!dt96=7Igoe zYyKk;7q0y#r1qWb!nHyca=)biN>p>hf5VAQAl{*F{Xy;f?`-ug#y%QyXEpJ92q`#| zwsB&Qk$=4$;l%_|8S=2wF- zR*G98Z0SZ=mNJM=yf!GvU%5`x-KXJgve5c{DDPszn-8(m8uQE%N$*y0C(Bx>AhOj- z2Lw?kKbG|o&}p#9vBJGeB93&Xu(<62DtV7~f3hwEA7Df+N$nfoo2oAmzv0YH+w0Z` z-P_*2RI%~COsY2T=i)ZMm}IxXoy66f`Mm%D-BQ2AG;{n_a36K>Vqks%bov|4cJry> z1CRZ_`3qnqL4t$%<_f0Zto^@yl625{f1|@?7L2cG(Cq?kfUJ7s$+?FNW&z z%Ee5@x3u+0M^NUiae#MHoIlL82te+jhmXws5GdlnfWkPLYGO@hPo2vOO&jHYN+onn zb;Os?ghfC3o@{p1zZV|-UuBbF{1~dkrrL=K?H9eD9?9LDe}1njVm-9{ zbL6PYYTO8VITS*OB<}MiC;JivMJ-KhbGu7AinjD)xj$cpuir3ahaYEMNH*`G3Uy5KFxdFvF;fxUhtsN9;7%&g$=#$X9rm=Iw zqHWpmrRa__BWJg-1J7xE-#~Rm<8W-3+mj;^MrN+Bji_cxIr!q{KyvdU7^9P30oWcCb+T*in2O_b_{<;*O1b_1F=03+?P0I{3`H@-2kpDjgv=&x zf1U`33D=d8Dw-2rJ-y1XxPkDZ;UVUBicG0a{KChrPnd`GNA?1q$fqo9ptiPNrRiu< zTi&{lHYlTIOc2D9q2Wldr>4G$CqkV=kM&$(Bh)Ec00)noSZN`0#y|J_`g*V<>K!sQ zQgpaNGolnq$2=q3_+)-40a>}{zv0A;zBmdBUfO)T$j|LuF9;(PD|nQ`m6~$Z^x>OO zA_Y<%Bn)7FWX(@YS;#xh)}&@4S?@n4qqRvu&e7QtiIqrOGu>wSikpd9?yw;5nj=eJ zx!@;u5Lba!)RgqfK#fx&n~fRI7T8F|hj1YSw2~g5A_GHyhM+&IwIs9+8X{}w>qz_K z?tcCa#|MyPw>|l2mI_~#-TK5Ca)I}mAoq{o{VO`70;CZX0X9l4v8beq5snW*E;=dI zZv7~1h9g96Cm?r#^hBQuNeLHwxEy0%8Q8sJ?j*~)zzM$OTal$uH&q?|lJQpAg1m6W z3(1GC{WtUaxQ%{KCH_uu0umAJzu`pD26O6B&2JqD4f(qkb1OMENNR#lhday|6i2F| znhRVb!A}ppAJ7G;2I()TcV<}PxkV|Uu{V$v_wlXwT&NTuAU-{?YW1q?(bhr0ghk># zZBK76xR_ z;O^-F6{Hf|Wo~3%v6rTC3W;4ND?b-!VleqxA8T!@qWWd8j@G2O`Nk*+`jME4L2J zFW|$qn{+c$Qb-r8eGq#i)FLJ4Yds;ZOk19!(79*`SxART<2j+uncf$~Gd-zY@aXJ$ zVvB!v668}>)rjb@ZwuqhxzZ{NkHjd+QsKqxgslx;HP$+y zHd1+kvM=Cht=fhl6|LBJHEG*F4!+GRZ|is8$T2odc5KtXEqmPk9QHpR;8JA2b$ZvW z%1zpu-*CQq8u0;H3hPeMfM5>2K-1PE>0;dR;`eblzm5H0ROk9)kI&SwAuITskyKro zI#Um1S_ST|@xPjDiGyVXi(SzugG#sqnmaM8ff|Rhq#$ zJ@&gqZqIV0YoV1D3tf97(Aq|-)7i_~@L|qs(W3k6!i}no;kO;rh9tpH)6S;0jpt~q zFwQ-hIZbGRcG;p(&MItR#aFKdCq|o0w(JrsZWZ0yL~fJ8EMqlm&hXe2TraUJ$Eq{C zBIup2yc8Vs$&FHHb`P=&9Y08jQ3$8M-$(9!bS1c zx!4rM!gGc6PIsf~a_p{uz?`|%XU!ni_OwN}3LAWN!Ru(*na$SDJ_8Gk`?ph|!9goF+NAjp$f;Whmy&4Ni1Pk;O{;IUf-x>aLsGvt-t=5IHb%V55?JN3A%ulNyGy)qPV|4H$18Hw508WN451%z-R94O~ou63l++XmZ~R84p09bwW`3t#4}?yfJI ztAq}OnHLX!lT&7A9b8SGqiSa)?+~(DB@Ej?0-my!`2ha8YGNM+)*khwCSe4>_*fQh z(M6FX+T(4Rj8uwcBbR^veGJRg^}{{}r0vP_Wn!PBQF6~aOyu=$2_s!fa)=8YdD9&s0@gbd<#pGD>J+rD`DLt{GKxdpfr6I$tL^F~S6ig9cUR__S}Czw{?Wan zJ2|wUj|Hf!TG4eCFN;jMPwZrbHkuJta8RfPT|dR(P5Udo_?h(>q>g_F>vgPEg=3?C z?2%OrqkreFl%@ky$D^GehNy~DrJ4%s*$p#u!FQ)+qhu86o=*yH67J{}E|y=ENloA3 z8Lk@;e5uVus7cXdm>PBW5!qd43L}AdtoubZ5IM5Gq!i*@?)*`7Vw*cxJFkrY>Gh>n zKpFeOJDZRrhuOaSdpr4IKvNp%vq9u*EL1%+%>o4$`)+ z2!zB0Q|8QxRQEE}CY|XuKkB$gI+?Q0jepH2MeGK+SE3CY*p^EJVd9Hu2KnbNIJbU5 z$bM6ry%j|L9d73KVN9az-p2N0%{g*z0Xx0MFL=8!gc6cvs`V=e;U%urZ#b-g!ulfU zGWQ!B9N(fyq7Vlpq}*fqEx2p6 zR<9*Jv?8XldPw)gen-<7Q2aFQPa5vILzK#2635}KhEdyE4K;Ol_V6oo?!KA3IsXGZ0( z79S7#nm)zfB`9*0sb6fp(){MjN(z>~M}y$gey2OjEY@uUHKXP{Kd;0#h`L$MRUc;4 zP|h^~P!!MOP4*o{U0Gd+hmD>SHSv8`88Wo^1mY5B~iO3?4jcd5+O*pSNOyIJr}Tv-C1u2}Hh$~ENhE>ha(lLJ39@POX!a}csk1chDAj#-U$i?60%&v z2NlcrKVskEc5h%Ai}920-%y}J!n(AF^>-SRqk3MRw8q@VhLvAVxb3p$0ON*u3ca9W z8S*3ChJyZU6>FVrfQ$W?_iFhr_91Hb_lhP+<*NmgE;jZp#SOdC_Qf(HcN7XeLbEC2C5=fMDY*IDPLD~>32l1rJ)G$;( zFT*12YO8MRPw%dH0O3Hd08t&DRh_Y(rnf-jP>5~O++kwFA!2_-dS@aI=gYyP-7tAo+@>=l0dWz!BQ(Znc{Vb7IXqy zvAC35dA`&UaeMSo;OTcA&ZvVyH$M4<%mJ(@8rFSDJAFeTYYb7G?d#Iys^a zoWB-6W@3ru<*{z?s$Y5gmaQt}r``SKy;$PTB^HcchaMH!>anZX+}Y#I-Y!3Z0_`Bu zf)ZEWikZp<(7;;904KPsDw-_raipB7dRg$Gv%+ZSqOn0k+_lzPd6jbATw8CRDx)S3 zNOZ@3!fg&FTz31bEiW)^ounc?w6RZ7^R9P5@K{8FaiiD5`&ONHo0p-A$K_8~b`tpq zSfn20jE{`Z;(4l&W+X|Efv-BcOP~Lnq04h`5Y{HtPMnfgc$lLM21! zzPSYUw)Gj>;mH!qkf5(Up*K%&Jw9d;o;R4-^O=8pD&;A!7cnwRUDCVa{O*@M|Bs8l zW zp!8lyh_HgXksmNM*p2B)^$=<}{Hw5a6-dVVQ<9_gEwVK+k=L+;26KCcGp3)=^5--0 zjyxXFi3fFO5=KJ;kZ19aTcH4yW$H%TN<(i8@40+)dy=Ybhff#yfh6lLo|lK`=(#PL z_EX`xRQ054X!wLOHl*AUF$AMO_y86>FRo+~yhHMa|9XNi1H)P8Pwy=W0hZoqAV}(Q z&iKLj7NgA5%%Ohfm`;+4#9Jf<9(U=*uVB_^UNncSmTPxREPxH0<;WwBTy+;0_k?of z2D9cloS7wa9#?dbTps!iG9a($-|Z!JN2jCgUsN3J8VQM*Oe!uSU72*!NSPj1*FmSh z>xTBA&6(1^d~R@w<+N*GOl2=ode_;R`7yLK1P3=1zXsqBzZyB*)ocNzRJ^>X=gP3+ zs2joz!llQaMyMKK*jCIk*UGs^ANcGT)X3A+R zH0&5e(85X(=N`9F@BNWa$TIHy6O zDVqrw(CDn{ll!y5dn~y`JGtwMf+s(GQFgPK2=0PZBM1pnriQ%E|7VX!=f<*KcWq%m z*GBtZHURQb4&V^2s;Tky=2g0MPmk#wY+H3g|jk7_?6?yo8D}1=i9Q8l~DVogh9qe2C^YI?v)feV>Dk^F6+(`|4g0L zHF&yh=@M4u8yq1eS^eN7)A0P2rFe%-Hccusysg(Og-)bFeZlJICHPNP7mZOu+Q-Rj zua+A>oDeT;96zQ(ze~yBGxLlqTfE@{9OXUN_j zSH^iQ<0*OLj~0uL@f#hCs5`DMm68W5yxfWKX#U3um=wL&kOwVO5ITU>UeZ?APSgbo zS)vY30=Hn}M1q+=2ITJw)xa_j67#?H-0nL|7kt5Z!SKrq8x#zX9A-FEz0QuoOZ8KIa!*w+L1A^R3g5F;v?nO{W z)eAxOb0DwbFuDBm7V7`QH$Vyt?_vayeub+-A{uDl-neq#VWSpr$3Uk1&a^UMz%YPx%p zetS7TYG-j!_%eolT}@5#@Rue2k4t&jcg}}JyxaEw@#!+F*lC89=krmi<9jqv3zX}M zzn)qZt{oV&CkdPFmrKpVHx|C?^}Y#4e6EP;?a;os$0FCUj~dkRS04BI+C`_&Z@REm zsY!fSrV+|3Io#^uWAya!Y6fS^#NT~IcMBob*aRKYp>!Q=)#V2SYbTEe>5=vE&-<%Y zx1~62mGm`Wh6j)8el{Dwm~?=E*Bnc~2Jv0^1GI9i#DeeV`#nW+NjtfBL+wWbJE1yi zwlSZ8?O=XVAD@lq=0A9Jy+~XpK|V?;iuE@71Za7bU@=gi6mosPP_m(;&{W{u(DyKn z-8Kfd;p!*#pG?lr-~5g|3y}R6==z?+fM(5mzcu*L3M=)nYLagvp(@{8(CFYUXV2as zTNYbJi`Wo#YLD7om;^v4a$|s4!6g?K&PFbo8i2aV7Gg6@cjbOlID^OD;`u| zi~aE06*bhI-ub_U6+;YprxmTd4vN&t*#h@byx#OtmPXl1Wz$4inp`+E{>4;c1vY|MfvRgI3a9CPh1H-6xeV*rZzg4&A_6{Q zH4z&(JW8RSb+8w$wC8$7OAGU6+f}ApW=ZkIUqDVXex&8*S%g`mk?v0%srh4R-bo3s zY1~Q0+dLBa%P$DG#wx6`-LM)&F%6SHRBf7(mp)J}Tb^66)VBU$E?zSz*0!Yek&xqt zToVbVkLD{Iqar1?raMP;qC%dJ{<0UNT5F~hJ3nlaA$}J4xC2Oi(A}2q^iCxc)DgbOQ^8YEDiNp9ub7Ax%C14~B@?oX3 z=Ch}7Mn-GzkL~qB+=e0Df$t%vnB>lE;+2i^l~bRFcR^yy2Ggv2W8Z>@iw2pY+AdWC z6HN~|I`Z#l`9r#_B~H?RxnVyolQJ8nlM;Dzh~sL*8ag++;%F2Oj#9XXvTdqsKwnIi zWLs)yxC+}iqME(I;Iw1L`L#+GzBd(Zq3JsRffcXk!xesLq0YzS|E_ z<6tJvg4K$$(+`9-r-{_geD(k9PrzM*C}6Xl(c?2Ki-?G2{Z~3O`JWj38*4$PvYM>&u^Br{^U-+BWeA6vvHcN1*FL2mLqd1AXE^7j6hVPdRL+&mu%yy z?CV2`=EU{1j_P`zU^F2fZnyPUMHb9VoV+o5Bgnc#2&rv#Ma9GILmF!CagM3W@cVW< zq4k#ue10d6G}4CjixWG)xP|}8-)JWwbQo_{Adjm@!W$Og0uk}|b}i19b(N@NSUD)T zG5Sr?*Uh*l4-tUN$X+e4VLy^%I0e0cB) zTpkIap&b280V`|FV4G!J`2Zy7f}xg0_Tor{z?pH}5$)&?AG2Z+FpOj10&~{~73QUA zQk4x=41oDcE=c~4+N?CWzfk0v5Dygc+UfXFHZ#N6Q&b}NWg9KrT9j3fisd#BJG-Gc$3uwI{ zj-X(ZCO!1h`}(W#{N-x1KsW)}JvJgN;B_w56b5B3O{h#6>3kPms4L&gEAI8;T#!t& zX1y4Rt&t0r=92h!c?}D| zk|Gd$|7)l3aHs7z90K_e!C*3UdQ9!Em9E}(fmm44)*iy^Ae0Qk^kt&U0@oNvr50#CvT*y?)DPMQG)*7P{NCNyAk>) ztKX-je!-gGpB?ZUPIof7lOhk+0tHkyeDw$_dN?H>e+F_u@CUro!F?~h2{h6x58fd3 zjywL_3o8Qj{)2iLvvX3*Ac~QSi^VLgGhMU~zQ%hJph>I2pyX%B z09s%7=<;eDZQqai9Y5n;o{syvn$7gk!s8hxjs@gEs-Cti1@Ir?Ly+Ov%9Jr8v1lAO z*yq;2^E^o~Rvj?5uYgz^EAD4^x`#O|vh*S=pWx~0E!rj=kxAvYL0pvmRiO^g1h=Wf z=^4+!_v)D%6n2b%LXMwJ^6!W(pkxrMS{*VI2&gQlj{kIW^hbeBN8-5`ci{L03uS%0 z{S>H0{axJs??b--#1sI|$=^ke$*Hh)NFDpG=@y#gd+;-D#}=+}*dD==PTKT@Tk55s z`7ZN{ZSoW6u4Yp-SNAFko_0x@qP~)Sw=TXZg#~v@Hu5rCp|t;cjexj8KWBf{m3el* zQx)TRrJp_2|8RC8rth&_C?K_dE{5hUMf#)Hf^l+|oEIOP&uij!^h|5pfOz86G9ueO zhBpcV>Dbq+%-54c6?uzZC^DYke$}q3gv6gY8f^MxuxpfSoi;tNvJq%|*nG$x)4gaZ z!S0SL_?Yd}&(`x7&+Qz8PH$(6VSH@H*0~GPo7yY)arJc1dg!V(z;CM4K(YCUmCl=t2qR3qat;D2%_X?8;* zF;Xi^$r2(F&-u>Trw7^MSJB65k)@8ig3c_v)m>pP-`2+u!(Y=SeqVJv9wfNh0;J#n z8pu(LA?;Q-j+n{#E`g@fcp-iB`CeDI#>hrch43aR5|w+pCYlr;V!z?s z>f^Jw#NJ&^CUwl61^SwSQ6W{IU3{=2M@?r&)j2=|kW$SyyfHpzBZ{j-wqs~a)ROGV z+1X!#2${cjT?5d)7_EOWJn7L>{pPMobK5(~Zu!U&54r1~{?@Pk|Hw8m$+4`oQ^&i` zZN|@a5V>UWu<~8xQt3JOa*R`T`hG`S;f2pBSOj;Qf{3%{X6?SW)Neb5EuvK&0yTC%J4KU4a~n z5GrZocA2S}OujAuytc6VqD!tITCcM;K8Lai{k=5MX zYAPU@$T?p(Kz}~-kayx!IQ;n`<8p_4oFe;_Z!QZg+D$xzJkC|(jrhFtj>bN3t&Y;S zB+LM-2IacRKF7SHTzC~S`K$3SOzkJ!S7syR>(HarDca7tH7PGrYH7YLJ6!lB%GHv3 zCNFAaY(=}zKHOv%>S5*n+J@sjz1?D{!p=%g?5d7HXJaN9b&Df&^-+$qsPBmZ#AQ}r zZ3bq4mi{(n_KD^^!jn&84`@mK>rxHC-BJHXk?K@Hku!%Qnq}7>$d@2Xe~IcD%1H|! zEz^COwRn)u=2+F7BBo|Nzpm|ajlE`5?xHH`bOra*`?#C9zm5O%62WG=)##`Y`wOMc zAIaSziHf9SzF%s!ZPXiE#$D@(VFE+*)D#TX{;uPiFO&o9R$o~jThRtjAcAjWU|r}-APpa4{AW-S2A4tuFnq`5l7ZP zYNvReLa5w>ND#WDM+=0)~vC>t7z$>qNXo-q6mUtRCm6!Crw4$D9;LVJ*l9vaQAbNH=H%}}b ziYA_+ld9$Iy`KtB%=PW1ut-~IbJzxsN{z_aiR~^nC9HBG3vxtsaNDfVq3Mb|Y&h=o z+C<-5mUybp6hDI(H(YxHKNBulpV(ex_OnB7qF-_qUv^Zk0s|bUgA32(F9+vz(*w$u z&IQrXITKax3E8ibq;;H+Mm#DhHE_@}u-J3rir@i7Cah2~HW^uwNEp|`0|G0a)|HEE zdPz9<9#A`pZrAg4W@x*Mk+$s}IFGhaI#^xZY0v3C-80`> zv)0Tv=f_^^aAJ6ry~ys!_AH|d1Z1Fz0zmc}MH>M=xtw?oJb_6fGfF2k z$mOypAEi#i56FrT8%z=dR6Tpfkv{03W7*987T2)~C4xyMC&|7?G$ux0b(l@@^Hw}V zOHVJYUOjyc;y(LOfP-6M?RpgiW{WYiy%hRl6%t%4XP!%@5bD>;oEM;Ou-wfHkgvyv zixta*DEhbb4j{sxnmy+%nB&koKUYeep(bLYuI!!xndWVpr6rC}MxoL7KIB$&uTZLs z*Xlc2A2?O4?`KnB^L=0X9}bd0Uiz|9@n#?ti2>Tvf4N?=0lHF&voYWuUiS0MUB0SH z*78R7zh`~@Dy;XhcFp@>juf@43|~I_ zkG*VE?uH&5y=iNlK9sG(>3DX8E-}os-g%Ce7dbYprZ3jdIQ1JMa+>UYgHWP>%VcXP zdgt+F(fMsKak~)CWP5;#zFW5K<{?$&N7IVF{xbPqpI0hKqk&>A4{|+aPVjcC^bRtAAgak=OJIHSg=LKoHcZK! z2$|R(Q6PoMw*ceO`8}3YLVM|YO%nnSM6{S*$9M?8Oe`5PvJ$A_@V!z>CBEee(o?zd zKQ2Qsz@|Qj19vx%#~^409P*6KmLJjYG3@3$7?B@$XME?o-isuTh{j8o;w<%v<{)$_ zb7SN3=5Q+qVm-5O-{9l@av8~~ zZQ~(B9QOY8V5w%}cF?}zoEP>uq@5aO)zX1;eUH~^p2+FxxcW{9;#EBA&a+YL@c|WDS|H&=7sZ?^;@G>wrye0NYLV+- z1bTzka`c*$XSE>}Yt9dMC%B0(g>WfI(gX%sJ~T{*&Ucy&L@C^YNn^8OKl7xA3VKOb zPO3iVtJsZAn}`g2ToCch)I*TINaqH@w&#DK9}kt>rN`R3hFpNCP%}Grb+uo_{Z+@+ z5zQS#wRL_Cz3ERrSo5c0%9CG{lkHr+GNu+hcRpbuP>nH+`8Lt2SpeO8T-Pz-8OLP# zalygI-BD(77|H2qKtt<(!)crUrCOa9gAIetLO>)Ai|NtB<)Z{7=6ep1^V>^IGh(}N zP02xvi9NBo(3l%2rROYV%_75Ow{}-(8Wrd(&&T066mBizItH?(nkVHX(Tf`M$`KD5 zrmVAnO-=%RVTD3h@h_7qP6uMO_5EZ_J27EA2_V*@)GC;DHTJv=qil&akDICb@Eub5 z{nySb-GpIka~{Obb>owD?yJ_3{oN{3iQz-KCYIqcK$&YUKpL9f8E^}$pA+rLPVPWg zW1&U)y4kO<)RBfh7}V|C;8Y(w#IbtG=nRsJp#rK%%P`$mTnE-fKuG&b-o)(vsk+@; z8-8D31QpXuiVnHU(XF*kazEF7aEX(=mQL#XIbhv0tpunnRZIGDS08lMb;tDPu`2m8 z+0I!>oygUwf(ITfbelJ>7(Y#oWg$lg%Eto)<(K|+QAGlUsK^UY9erq}Czu77x2-+3 z3sXiniphdW#NYT0d7j2hIi^%>Y4=q=04?1kT`zxDMi8tvKaB+kpEI0)ZU*eIV%r_m zb;dhC4s)(=vQyy!Oq{@26jj3H!Rb$IaSi(V{i7r637od2& zCV(fqdjQ1}ywcmmQ zLtAKgn4H6VC{@q6pO^D{6Gszh3;o30DEB4Pk?^L_Vx-9}>xYWM1K~VlDt?lkLg5)5 z-8m8NI4g_h`o=GULO)qG?058&`=@_0mk7oo^4b}cNK;-|UvkeX%gP^^MX~o;Ufz_< zp=^~$Et5h|yTGo^9h3Q3+TNonhoT?Gy!VD^<}B3QlH@8m-K_jWZ?e#Fu%t&RfT3M- zIxB6=eM{mPz$FsRz=UpEM8%9r-#Np& zqV$hLgj(4hP*peJ&u}h#D$Ox^%}tYNVgD9~BmhBjwWRYNAA0VRxsV;e>*0Y9T%6(v1c#x!H;@=KqfFl6o!@LP1kSnyr8ld^ zS-IQUp__KC6ST7N9}(fGemZpt?$FS&6^VP?AXQOIK`6-zN$hyA*Iro1HBT*Goe8|L zOONTyVOzk_1<2ydk2jUyn$otl8_2oMhdZ0}mDlr#q(H3T9`?9%0bByM`9M8*jM*~c zID-j_|0yi~Eh8U3SvY%3`h~{7I#|w}&`RUe9LXtd3%_a?R;IgLPEV7WBCE577Ekow_e_ z;9}DULc0P#UdvApt3!N+()g!525-ec?k1%=*%eNA*o{ZF%V959pm(i$+G#KoJTEM!%@KOHNJZV2A< zJcX0P1lw8==1hLK(wt4?*~HTyd$-VpkO_;ZfgU_7BygMmDBrQ_+Uu;@wOo4~qi_c& zowD2P5}{E1VyftH;pPFC;M+CN!BOb9n06ApdD<*mvIlK?D4ya+u4g@q zZM859u;OeXX0Z;l+Oa46)AXMzE*hpbQ|v0L&#_lha%o*<;*fqr;@;go?+QTkV?2FOKDZ z&^ppRsaMVDA9^0|JkKNTKaplezR}+P@0`g$oC_5tE?u%;UbF9&TDSt6`8?F>{weMB zQ3;x?%({JlEh_(^0B6v8?Bzt!!To!8KfGb!e%etx-pdL{!@JvL``Wqb_Lqh{;EyU)=@50OrH!}pz8imQUIMJG! zx8Ez*^Tew=?`ATCPGGt8xr2_7}La3K%580!zCb@v}{`lZ_mcox+Np zM(~NXYu(?)&!$Oy!um=aZ8Et^E%ocX#v=}Q0l|(4luv&SEB;B%40;MCi%z)o{p0;# z0a0a+vuV^>>PEGTi(SNFUwn8J^e(mBNFYAis&#rF*9{H|V8D_@92({003ute zI1>sH|8tdKH7DR_782U)k4^(YsBG1ELB=R{RhY!AEB*;UT)A{>S|fAOQlq*x=NYoy zxtEk{mo%rWl24(1vTX<*z4 zl)w9@1^fTBlN81$e=<8;KB}88kAnDpwaONO@^I^p@)y0 zA91fxT)L%T(Ctf$|2?_;{ZIY&v<))8sGZzX{a*JLHOiH2t3ZvS)$Sj1)|k7~FVDr> z6SOgDRX*Iu^|_`GgPmn2A*YQQO4Zik$R%l;q|b?6`piWS`|SzW#z|pMZ~)@k3=`2) z#egeT#NKK{6KA_p<2)WA<~&dzRPxORgtF6VuG0*@2z~pR@liB(g~f2*>!cc$@Fjz)28_BwL0lGiw>NzSnGo^&(HgFZ+j0QNEV-|HZtHUXV9HYn|neJWN5KT1! zou>WC!2TS`UTS4tC5e->dXCm1!i(VG9n2Rox30vZ$+q8=_*Uc!S@^HCb|PF1dYJ*HfJ3ax`=PYL77yD6>wpZ6?zAVH z7wY7Cc$mykiPEtRc!LtevaR$*BXyIaRU8 zwve^-C^=U|EnBpDzo~OfG-$GosWp8?H7plry0RAv6G9cWrD#uh;>xX3^s1YtT-GIe zhJASuE%A5qL}Y3M2MLI1hZA)w{=F&CKzrVqEJ_lTELiQ7Wm2Sn9{MTgt%WGC;bgU-0}qHlx#d{y2*K3CgiKJUuN0jN8l1h*I@)0p=i$py6>~c87m?E-s#?krO2iC9z^a- z^(R`0!i58*=|`~Jz||eIyui?sBjjzu!Jj$77UZ*tJ6@*qd(`veBK_s{3cNFi+weuz z>frhrpO2lPrNQ%D=A&uSFVshXnP5K*`x&vEDZtH890fkHFEP)P>po+4rR^(r~ zl5aD8i>#x#C4YZWzaU7Q^**!Rv&t)A{)k}ds%xJE9t2APX*>N{Y7Xa<`1k1Tq)UR5ZEen2f+=5yxUOgqj68F~Q0lc=O z=ewb`NqY3eOKtl80B%Cwl&QyimI5#)cods7Mli=MBd6ET9BzpjX*EeJN4g$~SS}i^ z3v%t(3KEsGu~Z%VkXt`xV>-*n+n|A=#=F=M$IDY)F$E53*|)=9->s~woL{=JpnXN6 zfljYq?M#lU5ae$H#6E%4)#f^Vfz?9;z|#!<1Y<=CfF<$4`EU3xBQ5w2gd2SDw*t7w zWLCjQ^E*KHnub8bX<@9NVG#88TNl9XAH_W<$|a;I%9l&0a|6ln8;mni6+sqm69Tn% zR5qRj7|hh61ODoO);TwRLbZd%oMl-%YxafBN`WzNaVcE#Zt%LYh3~mnGyGr6*o4(* zIZd`2RC7O^(&XMxRk=zw!Y7t2K_a+goWQTnJ>C3U8KH@a0zL@o%Q?E90PqPPrOwXY zL{*(oKx@yt0U+{b`{Y0ET1pGtF22V;11}f~i^g_*^ceK!4pl*z|S{>u72w zb$j{vVal31d9sT%D(~KN>x70Ew}VIIok-EvI~FH;kC*O>}CBr_L-^>j$)eW z`0fPE2JJ0IT!1+LfjyZ4x~R&w#TTG}9q( zYObH*oX)2)zVBkw(0Z-!TCm>L+Df@z=R$yp>0oaFjARFDG+;Ylyn8W}ZMEl|5-a~B zxmmS}uC!Q>dZ)^s1>`(@ROkHcD}0{^ zKy!tkiw>Y!xzOIF%}OEXwC!c^!W!Sx_ODJ(E3F3vU*+1+&Mj_V`(Jtl9b{)>mvw+0 zo@Im0?@a(0yvhLrj1H zliDX2pbl?#e3YdGROcuQYj%DU6PDgvMZgP4+p%lcXQ-^n^vc9eENWgKbtOKlQDF?) zcp&R7_uleK?@Nf)(W|rERxj+)(sG|JOiqCvV|@XF0Pym-$I`aRTa!=7u2dvbo3IX# zY>1XD6gYJbfsQ-=D+B3p&S8M515;qGp=_KuFF&BBMk9{BT>HV{qsBH{u~m_ZgTN;$ zro_r+JCPH=pdMhlyW#>g&xhV9^S~$!DC4Bd>P*Ia$;h%a#&r?iY9T696_okD-tnQd zi!=pyUs7(S6S3%$e6{@J9{CTJB6xR*4BwF zi5e{yq-l+kNS(+Hs-sZXriQhXND#Zne{LSIP1n;0c#W9=K}Ucx7$$@na{Vbpu3TWY zZRlD)K#^7O>Kz~7UEu~H{z#wF+lt&*jB4BvgHW~A0=6p{^S<_wVhEc6P~XG*RN2dD zXO%eS`QB=(Z+@e(41vBjHCADHgpzn@Wp;Z>+n#W{>c99znRv}pU1$PoAQWa?4#eg@ zK6#D-bmybUkLHE_sj%jE=q2f!Ukc(GCO8w%FF>Oz!h)0x*DUfe#b~=Y7{uSV2s++< z?YkzuJRJKQh*1-SeK5zbY zn=tX;89ac&GMPQ2$jCW61)xxC8r$DnslaS+@J);7K+E zPeZI3naua-79C5i&i ze9Gn`YYy(LTv*Y=9LFNEAV-cf8<@)c4~_s7tP@O~IpT4zF?jFsq~~0~^i`c3s9VoH z3Wq3^ddv#voOVUIi&Q^(IMeL_`*0ZczILi-9YG_uz&gO`ym3m1Ju~!v%F0~AZU*68r zvlioTF#=iP<|DCJL1#vJ9fB#VwBrefLJZpqlvvu0^jzp3nm*h!y(8L+b}XydQAYLM z?6uxCfJQ=N()KdfWohmg_g1;S3z2;XWTX^W@II4rjJY_EKBmiMaANel#fZNcQsF}y zrXa;PMqsU9EdhFW?`k`Mlp4^b`uH6{Ch{d>X7S;z#uuRHQUEe`*}>fZPmtQ%gjGPd zgm5hg<%*0!{ir~jOLmIdvx#GVG<-1IqN6|y+&@AdJUX@VO*UyrCGH+|X1=}_I5=GF zFnFnTyW6ssq<1`IbY+0`x3g(7&I;X($_B#7LYMSr^Y^)Di}|7Dh%NDTj2T0Ot!tUy}jCH*5;V!f~T zh+6*L!^%%|w?ss#%hyGi-_649mXRLDIIg0AF+yF;Am%tCM?0Frrc#J_{VY7)gQfnn zy1l@#=^za^jZ}s{j~hL?#(B(yBA|R5u`;_$QQ(+E+bWIeKJFO;Kp}>rTbC|CdTHyf zbuDErQsqCqO$W#Zq&Ru(hmtPkv{K&tl6#i)6F}V5v(aE(itSh<@ON8;E}4zkPpaQ_ zl=xKc!f6;sc)w8j&WrM|T`Jp_P~bO#@Zx80@TP^(maf_Cn;6NSDsp3->VxdWsakpt`Q?>>A3n;gBmP1q~vg&8rz8uq(}FH<|OBhH5~VeKsxhoW=`h38*_ z8S?gDb8&&JWRFEKs**4HzOfUY4+ijIDpRnMkG_bHXlctUD}8AdSSj#McS+5+K=h)S z!-c>3WQy5;3c3J!kweF?Dx3iwIC(jxQ=aV^pwb-C3Ms}Jb^%iouAG?qx2Q9nM_4)Q z`2mljqOW`Wl@x^|ASLT$i71~BPa3ZU%{1)HuU7D(sKV-WC~_!}xQ?EsyCHpX&JNZY zs^RPfQo7 z&K-V7>{N);t^eWb$)nH{YwOhY;A}nsQjE%I=d%B0kck!|-!d`5jdY(&&f@io3iiNX zU7Qe2#WW^n537BXWKBndzb6y92ePotE*$ET4DGFSApQE7)S?lt)X)f(&p*vIcdl*9 zaFd=D7AUzw1W!Ng@iaE5^GJSpq&>uRqCxQJ-3)tYLQsG>2He-?fmSWfgZg!vNE8+d z1OX_+a2MX}$*n_e9+^18oeI&}H`d~Q|kZylols%`)G|Nn_@7)S zs*CCW^>~OE*Rm34Hy|UC=}_n>WALt7`#q#IecH*TD)0K{e#L0*2Smrz@SAlNkG4ey zit^Cwmh$lK(%Du`be0|Z3!4mv^+#2S0|jbV`EkQ%FV=+O5e_S&Y|AJjr5F&=8=7at z6oJIAI(8*jMyS~Fk|Ev4*&I`Kq;}Juo4bZa%!iBj(UZ?#rqR3mCcS;%xIbL?yv99| znE^}>K-iyttj6&63lVN*6JfrEw;JZz{}L?w;K+6ZF4MiOqq#rw?Ho-KZS>%AlR6=5 zgqw36!H;Z`vkaI!ronj1MefdO8_n<*l0JA0xiQ|G?si2_f>u+rgI1IDN~qPbogI-? z^hhm^Kr&j6ca68gfEJ!{6d;6Fzx+Lsq?8_M^T_tVit1Y&V>!&KSmu z>X<7-D-_$HTx$`dVWFh?y-s))_gmFYM~=rLS%4UHw9vf~cLvg|_UBGF4M1m9)l=;< z14W{n02>OO*qOA+E&Em>=!;Ord&8gP6z}g4ChD+m4+U`Id9jwgahl5Q*ofZoD}80A z557HWuAUP27u*|c=4(_DNRLAKawXYswcScUr z10542F;JbNHdB?K%4PP0>uG)b(scbeH!R!AjF^NzHcq zd>>8$!8tVJGi}`be=GsFtE{odkUbM%)%4%LWr z8f8YCgDhpD1tp|9h~RAzo*nma8BrDKX^YVg_r$@j<(u&|O*yxwW@za=R!ihfuamud zoB#4dT`JJia|)a$rji|5JTT)96;;vnqoRb`ZXm^{DuR_|7qv|qhQ*C4Wh6{8avma;{cNd zj7~S076*8Xo?kkl3LH@JpH#$iOPjBpbP6128|cy+bgnNbMd7vhYQh_0Z^KSBg04HR zLq_1Xh!YRA;A_n1WM>3s5$^g54Q>F#a=__>}ocd0NvJ& zruyI|`vQX03BA;y?*K2or5xVmGY~!S;(}07hij8wS%wem+TMDyLf~^ho~7m)G)7Hx zKh5C06g<(`lTBaYn-;Q4h+5P;XM`zCF?YT?0uYQbx`;PDK6XWyKT4`<5+^u4FymH! z_B?PsN~trKs`gvMXMw_CvNyYV$(x_t%0dm2y3_9l?hiPo1dS2M6Px_D_?GNYBs{Wq zJNqi;u$^;opFiuFksmdrB!n=~lBcex~16tPb)8Ptj=#O7T0WMoFQ z&Q`#ysHjJZ;1E-XZL$6#eya}hwCD%6D^U_~H;}1AE)@b?b2xr-7;%19h0`~xI3)eS zTNf_tz*|4R(f^)opw-6b_WDa{#x1@z0a5g~KKc8m>=CuY$bgCu+u_{;w)!&nx z;@hGobix$_!&YYaQ0gu|dNr(TES?#w(?^${5i1k6xEIwbT4Zj1#Q-cZgC4t2$iegY zzC;OMpw3N24qcLYdWm1eiE&k2^?{n4z2PbkfgoN@|MTBt+(IYBzhrAK>3M1Z{+036x8#5u&BKt&PYMZkD#V;hvO~ z`htqH)o&#sQh^&<-*3r;>o5sl)o#qxMbuBdN5pdkBeCz*3*OM|;NsHl3EY&p+_=95 zd}d?7gul;o_%*s0vBUdCP<;3D9T)3{sEuTaj$jGZ#F7`~IX8(`s#v9RsdAm@dtQ~Z zJ5CW7u1WK?vkR&WZizH8mUYa%vcvyU1~4tNc)o8sbkGAG93%R6*VL}qGVvCZ{ti~j zfyf&rZb~a!vrelddC&T`v}|7IcJw-47A+Ld#)H*>@;$~1w1@dYP`lu#E%rOl)Aam& z_)&RJJsO4;-?Ik~IHF!xa}#d=z6`&g3svF3kfuF**S`v+mC1~-euAW0ijZ1x zcB#L3f8=sio$uZCBt^EEv-BePcmeo$7v7!l+lRQ3^DoCx{%-NP?|B{Q_jS1an?etW@`e=d%H-34eIJQGNND)i_Al=Be2hXlYw?7vygoqyg|ZC(d|aAr>c z|9|Lqgfb2=K4&;(s@svC#lj`4pjZfB(3ev7H|Lk?ep^<D*Y%xaj* zw?&%H;)kSeze1LcV-SGY7(|V(JNNa~zJ_|}ydt}#`$sQ}dtvT@ww4+1+hd1nT6~;(F?n;!OD&v%?ibl+-bBR)wD-r; z6Jq)+gQcS_-ii#8aw^ynv`bSNF72R8)5FRE)#yBmC^s=`58tvwXcp0@6Tj|`fGVx; z-VAuDqX9a7p#^>zE|F$xe~p^X)>;|Bb8j`u9vpH$bxel>rue8Qa%p_+t>!;B z@&1@e@3Lgy7KR$Esx&2wRb#$|v`YFoG~V5D*s*mme{;6}Npm(RWKWS@yAkunL(7$; zPTe_)JC!PpNh~ojwx=}wt|D(pNPMc_A1pf{!10{FZB~vIy+;R0n?c5H6tCovEH3c$ z@^RB>i8Y8w3twws&gAfx8qhctqJ}-hA?)n1*9Q!kM$2ER1&JGT#ArdpJiJ`Vbv5s* zORNlg55j61Z`QVOfk6HtzwNL5v>vr!-fTZ8SNuKWw@LR)gOb&`Km?ZjxO_B8Z<``a z1Ir#EKH^L{(#hs@&uGLk<*a0Pmu5@dh5IF|{_$?ib6w$-mPEg08ffsb#o0TAmYp6! zHY~%B6s|>_K2aTy6yRy_3+@#hd@C@)O$LQ~uYb*5Jh=3*GF=w@rC7mr(CMWzc>uq)h7Y@CzUC@yJNdV+bLzv zhph%WZ(w%sxv|*D2h!K30!8^Qlak0x$K;e9a%B#E4uE5;Pbew}6AzU{w-=Y0m^%ar zec9RNo_M@SE0FuecK^k`3&DYfeNADHwFCpPnQfp#%Wo(7pJL3D2u)lEx-22wlB!kH zLk^6z;GEs{oX@XTgcdZX-MMCnGvf4OiM?(Xo`scMy@aD^_4ltpK142+BIz<6osym* z91!5@!nOU&l)Fr2_`p(@O5Y)X{R$}U|Br|_{|{yb%fu3*C3}~{n*F|~Y}W_e{30H+ zC^FXVg?RD->5o`9wodD(DG@=-nI=B*d}~0$DE|-nJyz(l{>)E6p9(+cOsE>e*W0IXZS?%WS;jM zI#6NnuRk}sodwtP6zdw~38sMbE~lXYg;$dHs>cSI#QMy!ih?2lhYd zt?w*4G`}3TrJLbkdQ5rEm(BDYo4ktoZwvm^rHVBB)6Qv4%Ix{u#tWbEXUMVj5 zgjAY(OGay<-)Jh<)=#%kNm_uEuma@Y|D-2a{XOSrb(g`KsvrAtwP^UVa|Pp^c1iqzPAF(fLH-JCp^=5S16{2Dx{IKLm)6~hAZ_hXHZlfY86wVXKV z)#tI6%Y?*7APavpElZ{@^fG-{nK=&LIXP!f&$O4|#m5{IR;MmXW5l?ms!jMw!&-Vv zaWN^;n7$Vt3(Ue*;_?jvk;DoZJ(|JOxFW5KOD457otkTu`yU+!iIJo_)k~M&g)erJ z%>7m9Zzk*7%Ft#DB4EtFyjF%m?je z<(aT{!bDgXYkloqQU(ywBk2vNU_Pr-?QDz8xZPICqVW>nX8)|kz0?Y`k{E3fvWErM z`-Y#=H~L>T0zr|rnHp+DJ$Y%<`DWUKWEHh*Wg;pG!hv#$nc=ISt?eqbT)Q$KxJumoX`zWTf}ohWk^j^nYVL?*E9B`e|j|qO@Ht8HAeR4#}0) z1c5Osw!F%J&8JJuEw0PIojbPStvVBR|2ql)M}_8pZ^ePHus($6Q<{69(=2tEUF=>#4d)mP?Op+(>h&TEV}aGB3Z{V;&rgyArwglZlGl)&~agxIkA=NV>t-1GtxQ z4LX>LqgoGsi#wA-#fY|+iODX3%3Xt7#ufRAQ9OZBl#gvFj1k@7-jj_|P*xafDF($m(`SL`1*0b!B|U zt97LnzeWJJ3y`>7(p!clZW7q}$maOQD;0 zucIw{N91`s*{^wxzL?bmJiWgSL@as?~jv5aL(VSx_?zgrt(ZQeHH;oQ?&XW@2c7Z6dPf2Kfu z+@{dT3 zPtp`gq-5+mQG0OC##c9Ueh2@^k{s89Udky&&O2D#__8PP3~^^XjCyVn5IyYv*R~Z%=#O9Pw>|v$Ur+Ref9YF&24FpZRRiGtXM*|nE&sTdUooQ< z6%CP3c0L!FR@jd9k-d{M8slUBXEsxddW+wqE*@#1tMwVj_9?QZ{)3a?V&;DxTX7fH zhMCm>sOm!ALqm)*H74I%$qyG6rTdZ(N5f~;O6M$kX%#Scqtz; zDTTPlqkDNV{G*IomxbLTNx?U5^in!Ut#R}K2WTxgqSeC{(3`~dumQM;sO;e-b5S*n zf7uB5?hMwfFZZt8+u=78BX+TdHPDV#p>A-j0>MlG8=ej;9$vkwWvDo9HNjEM{Da?x zsoz>P#c^XO+&kcSgd^+C_>CNrcG>f>11NhZoRVLhPgdUXcHNnR5XBJb`DAr_Am^Kn zny)rt?-OYzo5Rdpm*edT<1j@Hn0EA$5C|%I6fxb~knYi$M(54O{BUW@H>=pZBsWxu z`_qOpUfZaFJ7d)eFMcWpkvR+NYv@nSF~xZ0S7z@PKp&VJnQRiT-LW=Rq%3h%{`e3+ zy|2hJqqbAFx<@s?+0KTk%V8C8s*0~I`G?R;ib~PRr|}ZAYpzZCrv^48Zn6!q1(p}h zmUH7Mcs$_Qc~MK%Xn8{|1STN_rd+lyK{r!hlcnMJVbArwO2&q|TwN@q`CF?5!Am92 zL=!36$oe`+VM?n~{XwT{j>i^vzJ%2S+#)?aaHmuzRfwN=$|;e;xQKzZs;rVNSBbbs zck^+N&<$wUlWuUGcC8;;%Sp{{WBx1Db(fyc#o8Y2pc+cXZ?kr{@MB>n!_asoOyB~P z5{6s60GY#pG1j836VoFZ>@hHhj-tcxhIOD`Av5tK^mtASgSX#Y4s5`7i|c1)9I&~? zmgO4+uHBZD9X2|&xF5r=q<9lVBQ|V$uB9$j%<^&{TzpMugYj9?A#u`%A@lJ~!7GdJ zVrSco0DKyscCYrO6-t3*R@%Uw%1#GUM6$@M*3 z9tU4t{yKgi$Fp+!6{dz2gg&Ma)vi7}+sG;@T$Pi2SraT^@b&!X`pdxET-=FN+fL8B zZzna}BjhH(VQ~q>4zAK5;*1Aea5h`(y z-3&|oInd#R)8iq{I3Q`_qzdoYVYIsd$MtCEEBkso~j4wJ+w0#bcbvu$W>NU{mHe`jOPN@`%h5==>9uS*i+U#ki9&IaLoKo}9n?lgG;`nJ!0itAXl zm{I{j8|KA2PVf=u=R&xNqfw!A_)P#l5pyoS36NmGB=ffc2Ctu5+DlXVvq-P0UF#$R zNKt*l=$i+j>-f5pOjeRxb)LIVv}ObX<_i{k{9HFQ>E9B?aQY^;sN8jVth$zD%;04} z#6qnC^5y-D-?S^f`m1V}5TmJ#6~%!KvGu7AWqneJWJW=~2Zj`}O8lWa@lTQ>7elhU z1Nr`wmoYL?i>ecNb!s@}8PnU`^w}jWYhTcVxJQ|_Gp>e2;pvA3>*o=#KnLJ|Tf_X{ zzYx+fysXs7KEr$iQTj=sy!>V|+SgRpDw;=d$2}!5Hv@Fi(|1k^Kx6XlfO+&LK~MB9 z#DjPqxiJ40M6o_jSP;`f$RR;_S*h=0l}U+Z7Hn-t5_*>Vu@ znMbb^E-E$n{b5yE-KfTZ@szW;zsPfUJO3LVJ)GaDZj>@I*pU!XJI19)c?zgme8CIb zK$EvJ(G+MN1zCy&Nnts-$l9X^6`q{S210ZK^ent zs!$Wr5%H|%x?}pZSD{l{AIx<48@$OamW;WRZEdY>d6ShrpDBVA=25)e5FP5`sEQT+ zruf~eN%n?EPLtT$%+NZKvn(5FUbh74lZeq#e;*YP>=?Dk+ih`U((3ae=}kmUw0fZt z{tV|AW#T~j&DQnpVuFQeP*9L&-txw4PejdPHodKK;T3GLj|}d zCWYTlmA0b)Gk&(z)u-sJosM45`I-jl0X-<~5oOy+l~-P%oat{H)&7q!(y@90ieAQq zLd$vr$Ub8FOVDwqy%UyZ%5)hs_BTTS=Hv`@ll_f3H;UnBH*Mvx1M&csoej#8ZnVr4 z3V9l$m#zd5RCPQw6E!vOC4gG9Dua5WK?mc1v6Ob^xM_e)`U&I+xfjjvw1KIeQz_*d zN~=wIyfMK;$YbKV!m4HCz+NZRiXFR!x>0t1KRLi2tf_w79W<0a~<>FNYQNiQchE<&wky;#I8FaC4X(OH5_esy|nsgrtmM zU^z0bdnlor$b9!GR0o@u#pl}n)cG^XP)+%ap;0;SdYW9?n8t!y2aXd<<|9)v#w)S%{ zmH6Z)GAz5I7`fF9%l4{RQx_8?AXrUO=6kjh8%)I|MzCi2oBvY#H-|F&WOY2cgR-5W z&jDcjQG(<%AKBE$+kKjP%?Q8CL&3GnQa>%VDm7}Qg}sK@oF19DPDDdT z{Bv#*GuGuMVi8yuqNc!_S_$;|GW_4}{s+hCp^7`fSoO*%R8`rxi%hxl*HKq|5YaBI zpdGwArFJ}Flt^%wVAmp|0vDr7^MUgAgr9!j zgTHz9vhz#0stXWlRZOeq3P8z8?wHLm-6~8G_o;eIz;;`{J@5DqQ6eI1bg)aJ##&*9 z#eJL&p~%h#Fsxi|r`moMcU0)-at&eV;QBm7x3QtVB`KV-xFOs+`aV3ehIP3wqY2F( z2EHn%<#RNuaU(8cj(=TKbF&=Xh)9sVnJw#UrEX=GSh^A@wV7mT3itt$h7bidpw%EC zSLPMbCmiW@5J9>)L1#1nEleLAugJn$kgOB7!1>j!G9pF9AZ4-VqQGP(T4e z>AiPC5s(fNYC;Er1P}v+_&x7;#&`C5-!;}=YwfeoIA@RHkNjbbCz;G=&bwUqbzis1 ztdYa?0`x*Khwl@$_hjkyAN>x^>nbrWDAH!%qJ{yck36 zu3)nLJ2q4t3BupyfHt4REBNa;Lv-oA00<hcPz~7MsEycTsp3so7vk4iHzWgT zmG*Jo`kYzVITCu>V+vMBzb7&rx1?#NV}N9^cTj;|)}y5pz)+^nHo`?9VFKf!M}hjZR!3qRNa@)VkK&!Ec= z%o1J4T-P#9eh3l9S4Prov=-{gi3J`3)9iAsd6S*C04bKkLUdne^DhtwF4C-Z;yq?z zQf{F|pMP!ju+zitk+Mj0Cq<2Hff8QeSpj=o>V>v&K*c|t7h_{d(%cxcqhq1yIm`I* zl1;GsYK2JLC6U%n5^w3Fcm|?F?*ddJ@pA%KEv6QTz6CODpr)4)gS;Bk50X`k)ZMs+ z`U~Z&+(KVH@Kwd6poFljSh=w+NP)z_fY?~Q6?W+_mLJKdyf(C$L z-?_BewXM8eN<4DD)0XU^1r~?^=-ENG#SnK`P*Q|j17?=2LAo2Sg|W=R5yn1!xHI1UBFTQ`!dKx3EuC)?((GcLVgFuWv ze`8^tkpDBh`k#aBf4*i+aaOC;u!GSy4@v%8trl9*6NS^Mzj4Rqj{kW;aGWVbjQB@M z)BmGq^X&{D4Y`u>Wd{Rk)%ib4t1KwtPjIGfbnVy;JinYTW-s0KIGq`dqOLO6KnuhU zkx`xX>lY^*2jA4y$;X@E9Lr!bhiIk@v>7X0w$QiTsZH|wofl7jfsCt}kg9Uy-mM&sijjf+leJn`3-g#2->E0>HLR- z&9;)&BxMUNkf}+1vLUv}mu(!|^?vVu;PnTUDn-L)bdS@mZt6<(q!m(fQ?6BzQ3fxt zyXI$gEYUbi2?M+)9R|^l+vy`k^N8bg_PzWDW?g}ON!OtcnNhXl-bLYUigmJhI3L^! z2))m}{Ee`J-{$n?TvK(wAcafp?BY1TWG$!&vI+u$59%@ujGC%@&s9v~S~_X^8rjs2 zG)JGJ>0S65Zd$KL$L#iwO$sSYh`}i_=>7XsNB&Xo{T01Wyo4vu3=(B_*#zlqv{Tm^ z-k*u z1Qn&S$#5Pr=|v9Xn&j0}UOK`iMEsnk4k-9Pl=4WvM86 zQ6GJ*XvwRO1)mWv7l+yc;r+Yt?n_AHqPg|<>qNIVZm}e4jlse1t)4e!r(qCYW#*K{ z1NYZ8(hTFD8Km1G8Lt=`y5?~ZrR2N^Z0zf0928SGr=RmUVv(V-AwWa(sgHo{^A8d* zwUaE!@WHEBAW#V8;+wa_1}NGt3nNNOUu&7cLSxN>UZ}Ki7R3Kj1QXIe#Ktj?xF)V@^Ij|N{bDDqS@b5^quH4kk(uZ zyu5wWtfp>&#OU~yF^g#V$GZB=@j0c(lHF9FIQp{^NMh#cXfzpr{9qtDD6yhxW1T~w zil7fg`@)z9$A=&j8l##Ab~3iFrHU3!_q272pE#3#T7V1VAYOS}e%CcbdB*s%%~+u> z8}&_f{%LkugN-qUZ^$V*2iqN&uLMi{s0F@vJ$e}&(Z-psm7@`|h}5N!Z`frqZ@wA3 zuF)W7GLX9JFj+m&SS6YLz^6tL@JZ|m$D)j=It#Tu|E<1ks{q8H3oxugk#3bnEp0KN8hHxwP4ZaXv@BCsDy^ ze9IuYWTyctVxVT25BjcUiH z_dbJ(Q@YHvV1mFrNWJ3KBr&+l-1vj&@IjOcy*~jv~o z&n89DSyK;oY(7$4+;z#f?gt9h=_uX}0N~!B3u@3Q$#q+aE5fgl{u>ID=>!^_ z|L6OHlh5iKT)qz+!4+0NRWu6OhLxS?%ciwUus1Bny*-x%V6dky7|G=la=&t?)nN&kUWY48n83KI^autw0Ypd!J(tf@QwAkYn(<#BugL%OpG5XW zz{$&DMtUUAKS5=pMaZ3@VRH=XTr$_IZdUCgWm36Ah{&Xm!T}?BUI@79ZwsA4P!6Di zS~UNBtg*?P=XhHn!ZFbCl6=xP= z`Er!3=S*8lNkrxnB|a?EkrGM&m;%1o=pCjSWt3X3i56x_72|G5-eb`{8n(YUG#8el z?l7JOTZQ(CRE7@ll4K55;jUqxcM%5l>+3EaHK#UL#s6T>;I7WF8X9h)mWa99o?Mj6 zu^it5nFTVFFQKTuZpN0@R^w46VASdMgy5XrLHv)B5Qh!pmL*Yk7}XgA8Se)5V`n1$5b}%y538FgL?d@sQs?ge;M%qpYZ7P zQuxQ(tRCTG$7FNU;dgQD;6JLh6UDDu0kP;ajbEUOKSYxAN{aq3DUn|@|9++R3hppq z*@C^xsA(y&VCKKW{LW`nh z!gU=QVfV7YFAc3ss@kyPv7+L;OzKPQ(k(OFLsjBKag9cx<<-lIR>!~y!)OHv=IqWm zT&UhD2taxu5e4+KDq?mu38Ih4l6MAel4Lq^`w^45l7kN<0CGM7_?; z_8D<~y-hahO;*uJP+C)IZA+jhVz&!Lt5?h{LAp+NutDDu1>1p~Gdw9O7*0e)f%viFp_z%r zlIfNsl`JzQd5qwKlaQA;sbb#HT%$YDr+lZNYNdJrkhZ??U?D1wz6 zX05X`@{HTQoh)PHWn{^cNAx9g2^*M!fpi55rgR!j;MK4K?a&Jr=CclM{8+=Iw2FIL z@q4V&S2!%~$kpv>mgzpL!;7)vD6*b&_G7O}g{R9wRL`0H2*u*DaOE-wEOPjGa9H@! zjLSwvlhy@R&eDP;el_Ox`b;A-XX=34pB4deWbq#Hn2PN*)7|i#9`-p=Gi|I`c-~;S zlqh3rYkpnIT=&Ipjm=T8(7=`WGv zDf`XcN93#a(z~z!V}9emSh>CWmr#6&J~unha5yiqnpp(-JfsjyFFn- zefXp-ru^or~8h##^qw-|{*3dkZo0mKpt$GYc86(?T0QO$`nW zq2*38=DLYl>I6sCd(RM*F(Q_Hjn(W=(;s4{5c5f89<`=s8dp8OO2!@M(x$EgqF=Rx zE|f6dW$|M|2C6t#(Z?jF?_N{v-CBQ}NfNz4WcXx+@$UT*UbF5CPgCowN>o)>+gO0{ z7eR=*_Lrx$QTTwMf>~V7JuDpZ$nmEeLabaY`je*9bM+wyW!SU$f}3|LDriG%qm-Rt zIqEp+6^EK{3bxDnGuC76D+A&!vTp8mEktKY{KX_J54%=!j~F{}u_&L;eR^EA=CrgG zO~w(m`28u*j@ZhYjK(P20{*b0 z!FSJ3`{&*O`g*@`@HV_xh*c1axPZd{@$v9qJM5 zm3As?!F>8;>SuD>*}_=O?o?bT`YdD7g>}(AO~^ zFO&3hsy$9Y(b0vmv@|V`;l}3=JdVuUpFiIKU04DUbuez6mWLBacNX~)drg;ub0?)Y zA3P$gbW09;=6mwk*N7*5+u*GQ!AI=TgZrF<2r%mo_=>b%L>htENvI~)>cqNmRw)DO z%Q&^0LOm1j#`pE3?6V$${x9vr=8@4y?aZ^L1c1%JgulHa3c1^g3CwRQh%d;Z^w$(DFmQt8DKk|OEqNCaN8B(eR3-t)4;CCQLOK@ywH;XF825QJyc=y=T(mR zQOpC#_s@_k(m(cUQA;6#d^6t_AZRdg-=`f@eN{()4^(k<*^?`8a}yx7BJXL1Ll3ew zsJZxn&WuhymYuXk0j$qU^z;vZse|_|t$^J@^tc@!lq8!^pPlp3VSui0@}E)G*<0WWw8Ot;=SbG6@}nmlRNCyu5L1y>iQtNGD~wHf{&+%(VLV zOCwB$=s9o%6I)sVN0Pc56`#pfZZ|cj40)&g zOnyZPU8sejR5*)=k2q~MPDce z;yZ#>mmvVhhrAak*(o7&2ox^AKqp+l=o4X}<{q}94v=u5dcQyyWq*P2iJEvpHgWvf z^B~-{c0k47545vcc85S8&rArYRmL+c{Q?~*f-!F$+7EYu5hTkWe}RAqcM}L}8)vhxBG&#%AZ7-NvinLb1hWj?7(_!dUU>qgD%uXtluv zf7<^2BuGvsT3Kl>SXS$arFMC(f=4b+xG$S~6mH<-hPDuXj9K+@GadOjyDc;Hto265 zN(xh-du50cZGqmEi{mR zRgoe(c>xS{aS4;wV;w9uzXM%Kys`tPly1G}RO`FDJXOo-clsZgzWryo`q{0*NMMGua;OtJi7`8#D-`V{ zPE*^7XfZ9C*305}(dW?MuYN%pEmU=Jgs1J4Ehl&eQir{;2_b%5Rp)d|tT){EII8q* zvD7ue*?N%|{rS1_+WKxCCGl#5D6vvh=<+**F7OM?)-#|ltP1D7_Wh@12Bk0cbL#pHt>L zERojGQY^rj?_eGo!`x3SoTa`g$a-+TMXT2KREQSE|D<7ey}fTCR$K}bwuo3KKq##-XN)~T5XZ?ufGT1M|_ToG`{TkB=_AeH_R881Sh$7@!^ z>HwNZ0$Bch8k-nV&F&e9BYRRvKYuN>G~wor=fmlEVNUsb<420hUHj0Fo9_9i0X3H3 zCM}UQl}?)w=~_zqB0Hj27jC*uj*|5))i8I1V|*5=aPio5G+(Ca!Z(;kfY_Yw=p)}d zW~KA86C-X4Xqlu@)u>SoMk0^5wf@R|ON=o89`E)`FODd2wX7}bD~hCh2ewn2+rQG1`9`Op zYVyTi$&=*0OUXCr_!F?J4j_p{p{IezP`m*a9HT_be$Lxrby?bq0m%K7un(%`mn=%Z z*ZHtnl(?Hb_dfWX$Zwmhp_?1^W>+ZpQ+Z87@v=r+qydy-b$Oe)%}ze@en*Azb3(+yvv z8~t@9v1e6gq64fuO4l6-EV#VoZwkcjUYyEz{pZNbpXkz@Os{|a;`w&=5k@B7 z31|s4zW?OsegfoGLNkgL?%Sa4)tL>U+Q|ll9;&2 zgA3~oBHHq?ypp{I&B$<;p1XMS4dd0@X-3LV<3FrcLp4@j{(vq4p8Jl*BwjuM<iL9undT4swco7bsM(;HNMgD+cq*qW9Geg*Izw*$Gr7rGXiG-#wgVurho zU8q}RXNL>Zn|+4gHI4w>&2A$gUiN;g;B=-X9x)!oyd!6sljqRJR{7LlvEVI)PRfTZ zMMW<|$UhaOF%DR`q2?Z${MeXP2MuVtI*^x2Vm8rKZp4 z5w8HhJ+-c6;<}q!>f~zzOgG4;vT6QE0xq*)psJ}BO`yAl$vu*T*S!LT)O$sU&Dw?|L)RUx} zKmCmCQDs+dbJeTZ%TH#_Kb?3y#%?6Eqae7Pl?}WAGQK_T$c4##D{YaJo_k^n`opE7 zGru)qBwHuTwS3)vc!}G~HIMN}eWb!QjBqD`I|~s=hoNEjeq41&b6V-)Rw>SW1BbP81 zj7X$~E92I%ZGF9RrHgqeQ4h#Ojjp^Or(ExtsgSAfli9QN;CqMSZxH$#Ekw;;tawdq zWLGH~O011-K~k6g3@dC)_}Sx!_*ioc`G96A-5=L+nD2JE*}u8=y0^4)2wj;a95YJs5S3ld|o~q z{azbvbWABuZEl8e&%aVfMAvoQS$Dq?$vn&`_=Bdq^@$MKAmk0W6ICpDIOW~VCI{ChO`*C z6+LHi;$Ws$NJknw6{fxp#<;pHBJ4YCl@e@8RhFc*NPH zH2R=Gs9<8RZtRt-5b;J%gcS?nsuf(N3K-e3e-+n_60lF$LCARb=@GtuyG8%C_&g`D zCrVtfQYjf|icIygPtq&V00G@J&IQ{9u-R^jixpEK^gMV$mE!4o{?8 z+^+D@eGmU>Y6-Q^$NvCn)ZsnpvJcv|cjY@oR1o;z^Lo04z zoN(A({h;k?Z2x3WCiNWU=3P5+u~l}LCBd|NP<34b@pnk@9Wh|cV zy4Y4XjPF_}4;s{`KK2zBNxlgsx!dQceo@yGnf3{tI6I?)VeYkJhpiX`;*MpbX1K=k zbrUEWsxeF5y3lex>J^`lTfQm;EV9A(Hf4Vij4A9Wr!d|=wTe(6CQY#|?;YK~VT!YZ zJ!^^XG)nN)QK`CW^u+L`Ii1A7=Nv| zIu0IdB)^$r4Zd|h_;x=CTjZt3uoZM+A8!vN0yB{G%)&7EowDbFbn&@n89XguN8{yJ zpRy&aiA#%wC`h5hUnSQqDaKo%)+^${OzjLUck*C^8ckJg72~3w#$oix8sy{8&SD(Q z%-%aM;)!dVoMZON^p`2lngXmlnN4d%`oLJw`~Q_o9>fn#WlgAIxI=SL*E=9N;UJy#3L6ph`Goj(!0y z-=u3PoI^8w9Tr^t{N5>We#R?Ht~;yNoo`+Fkh}EudiW-&gj%*6C@e??kuSg~?_m^- zvf0{1X4q^j*&1}N#Xd>lZ>Tw#f0rl!`iLteSfLv=l+GoGdr7-{5uN34MGEs*9W6WB zWqy!toP6Z?^X46enWlv4S?BlAi7%lZzV}S>uWvj*mq1hARxn%M)#&a%tu`JgG5(U4 z8J`z2T7yku?0L{g^6BbiTxj1zOBN-Tz>&E$cUf|`D6tCtbp8vkxEoU`ol zAtK<9#y|!*G0>MuT0e#OSZ^Ir>?qjHgftWGpNZzl(k|(tDX^KTb4%PxUWQsqbU6r$99zI*Y$~Jy?k{rzeO7etl73R4RvkI2XADyXU{T57#U) zb<(r-6T2F-c+OWBKO}P~l8|6>|M?u3RM(AiZ47f1Uf0ooMLd#A0Gn31tZVnRPjj;8 zw013`5$eQa(Q;)TaECyT(<`7|B6zh?1`%`29~-Hsb%?#KNAzfyqc{>DrHqj>roW&U%)`CrB6 zfAm`dMdkCnpl^~u?RaeZ@!#S3C36;!L_K`=LDC8g4r>{}>)-t6Hh zq4@rgrIC1zWEnED#E$nplfn5f*8ttQ%;h*sf(5`PsU?TLXwl8 z(cN01s%#{S>1n7te`}?J3qXr9AHy)~c~C5wVU}iake#E1bnB$Xa-t;MyqLeFJasFW zmJh-Ygt9DnF6??UFuP74u8YYI{k$X?DQ!i$sEZc8hFmfpznT&JO5GEuzrQ++7iVvH z)UnIzP_6yb3ky9PtB`$2WTR z!`c9_BwCvGfUpcMfyxPO$T*BHe7{k8m zQR4QNI+Zx5rOqMuLlJ>o8nrD={=CG_jgwCkRO?ttt-bpKKeRrB7mYDb-DfjKDGw%Uz71Vwj~1OTzPHOO6<1v8bDi|Xg}|==7*N(o zOC8kjgvdFjPy7YaJx-mO`3u();JgR{-9A19rif24>fXc;2+^#RHATdn%HF%2XPsii zUaT+Bi8$j$!C#%FP101{5cZ1nDj zXNyhKcN5sKh7oy1_e!vL7`9ElRMQ;f?|mlX>oul1B07@Ai@4DL0*z7(0-f+TgAVw? zn93HQPN^a+j%OJF%v$Q2A6ZOI5W{9&AP?poxwwT4bCRML@E&0IeEN!p`b9wbVzB(k zUl*o~vWDdlf{mzD%s@1eY8QNA~MO(c~$LC-9)`i=i1IP@3$^<1l5au0S zr~)9{xEy@6=3kejY=Zfkh#Dl9wz`aqS<3ovwZD5a%Y>PIP2~9|F<^y%FZBPu`+s>C|NDyl-qGJ@^glOq zzc16j?5}=brr($8_htG&A>40Y|EI|Nw;la$M}Nnof18W`jz|ACzx*R-nz@Zm_Nh%i(MaU_BZS+AEiB0{Gzr@{` zP4sSndQhwK_6mzN{Y%1DX=3f!C>g*t|7W=|1Q(5s*`IZ)p}OAhOlQI_rJ zB<%;Zz)&(xyzmTgc+&~0K-<y#Oj0$5bXxk zBL5luVf|+~;rGA&_itO+S_S)KI{D|HI^DTM)}7{on)kc zIsh|S4hAP}Qkd}#jKd~#zhZT{wm7|^e=e;+=l0~p2FI|ew-Wg)mhUbn6E)YsyV zjxosB!!GuhCY|%@Bz3Qc81gQ)Cfowe0OyCYyC)lr(^;X7ZsTvj%tR?y@){rES|p3K z{W*t^-e^H>lg7NYMaI=)NcBdN{ea=yQfl1WU%bs z1K$#~^|J~oHhoa(NV;&N?b9_O^NnCywdV8p@Mp?)?ey>`p}_nLEF|PXAx(Auh(ZKQ znP;?x7?1WtAI|MqzPoXZUV^25J`unWYa{ z9PAAW4)Gl*4^@EV=@r1N(EBe5tmVCM9DsN&|yHUa$Ii8evUB?h=K3Xu4)!_<~V2ig0-hkap)Sf~8t%$l32OrvER z7YSn4n!A1CKs*gp4esI!?*Q{ftRmSg-!BhS%ymyAJ<%@o&&Q2*-Cf2tl^$KcVVgME z%>h)COLK5$%Iqz7RuV76AgIMiROvHiD#ugB)qS^5=5wVh+ph+$nVIC(zw-|K{@uq? z^^yO~@Qji)wza)y_IVT{c*v}ZcaYdu8)W9{cR=Z%;y@_9=}{b#Jdug|B&?I!8HXzBHWqWd%H2K=o@mzC4T@2NqlKIdM2_C?o! z7sLP*jsDMvu^;9{xMK;h7c1Tdox0SNwfzNoehu`9r~?hSW9SplYf98kOoVGX(NZQ- z*YeuT9ySG4)mECcRLIqJx$Rq0GBqkkCU%nofOsCArx~L{sLSO(JjjSN5wqXH1)j>EsR2_vW;z$j_yD!w zZieBc(2Og@?z9SrD&T8yFu>;LMeTP6F(7d%58K&xd;FkZvqWN#>7CphJ@~G1TvgGG z{-6eyf3r8Qb&GA06PptQI0`D9FIKn3A|MD&%d}%;1UWU(qR!%9T^BAkF7;02FPRW+ zdD{2-eq2Z9X&aaZ=C?SaH&J@jthFnZZX)vDnAdpMg5h(`b>=InMo%b$Rrq|QGwhee zi5&^NS-nhLToa|WbMF>y@!m=N6MJWK1r!`<`y+~*#B1z9u>^%tFHxC6<7qp8t8mDp zPV_7vd*%W1^zgg4f!~x^euxiDgZfFzJ6Q3K@2Z`N!VU`y$kRC7@Tn4u#5MJgmy?6=BmFGf&zAYv1 z-+ITGnJu5`n(|(ccillB#apt=d$$bj7>{Wa>~Z8+)sh^vRq%JXDbnGU={qZ(s`eY? z-Y3}*r0n;lN*`#icia(UqQ?iq(9dkPi56Bb%%E0!_vzhTFWW+y_$M!RojH=nyGa!q z7NYqx9@D2}*AO+HaSVSp4;&1l5|~;v5r9%m3_AYwErr&fv()XmKb@U!&Pm{6z@@Ax zMFN3b@;mnm-fu3Tgku$V1;kF7`0M3AQ*-gaup&z7_;O~u&bOKFQFwg+B z*qB=6qU%U{BG>*7stQ zqFC*v8>@cLh1|>JoSVexeOf?9<{qXk=>yITaa#6HrWFf*JmF%?OeqrWM0J|ie_5_V zTi_Dwo6vM54&9U(l`Vly?9Nc)-U?)!D%&6oPvH5XjBAshq!Crmnd@Jf&r4Os7xcio z+YGg+G>Aqf#$b}2nCAvgHyn(vY%?>H`|y83_6YdD8C%iLRB4WjNtTNgbfxOxpf{!8 z^y43@pP1j89v}GBmG<)mCQrIYVsDlsKdwpaXIGud{5Oub!SHZ5w0I2LO+e?IEhu_P zGrB~oad&mvOUtE9w5&}0E`Q(@l{<@dYvMI_tr9xbMi5z|D^3t?q5Z>!0jF%1_1NN$ zy8ms<34?6mL+A8|braM_cQf){^%{%3x4H8+zQFC<7MNCn7^QFN0 zH03Sp5O)n_U$!F1O5z)?Tg3!6OnNw+)duMkCgrb4H#O(4ZtKF?BpKuz_Ec4ABNKEa zUdyI52@E9k8*6NJAP(BqNUUY+m|v`Mb}DT~Rn7H%D`S6a3>2!1+EKx5-8%rJD_Q_gyDP)@=Q{Xf&!r(-otd~n3 zmyT|y-X?I5J*}&##d_!YqUIFMtVRnYGzD_JM6ld$UTyO71wvz=lJb#N#m>t_RoS<( z^$}3~Lljg8GG!g2GfX$5mmuG)t7jatqo10r<5JUToM)2Cn6&wPt2Q5~LS$T|YtIprzVjA78rPLnYG%NljE+%&Wr@_~qv&s5{ zacPb7Z)EFaF~_>{9b6>iJPFi--I>tWZY-=!?6<5brpv}vR35WP$X9Cwg=a){@=Y%C zV)Q~4o*Y-<>QyVIvk|dowYwz{QXaSD2z6*<$5+LRAo#=whtu*N<*3ZoAMNo!E``D* zyAOVW92}!Or`z7Gw%=Hga%LU7Wq(jzTYqrrLTmN%er#|84gGE zzWN0E1=32%5RV{xOZ@iZ{E0OOd}>pF%haNEzA3!*?G+0>23PItJ+ZvqDi%AsEBg@HZ^v=~OH#n%^yT&(yju za^X_2V6;y#9D)t)`dN2M7A`PRklwY$Uo}5J&tJCzRoj%Lj#XNyRPE-0AHX-Bd4p@L zB7Rsg!}I`Z)e`NtCHG~gOrt^(Q}?YsZ$}?DOO{m&jZ=io@Y;zl#JzDqjg)r85j7H% z&8~E7B9IlYh*b(j?X#JE>DF$F>`t$13M_DU^YC)2?(tsZeMc_ttYjs15;K^l0Xhnb zP-YCg`n)NpQ&G65{`mAI_GehFXwai%`G;hucdZ|!k;IWgi`f>MxSA`-JJGF{U=FA9 zGqnZ0j9uax8<^5;`-@EA;qD_VUFW`_pWfUnoWAAprEtzOb%$wX$_*gv^J>=hK-It_zdSLM|&m|g} zH+8OkD5p}lHD+%|g}6hrx7TQTE-Ij#SFu1FDhFI2n-~r!Tj7+|s>YP%yq{MQuQ6)U z8SrAUO&V%;ev-^DaP=TGmC8TnKv-A#xNJKOfh7%rLmWSvS!tZwN0hB=OKq;k z>A3LX-dXDsoH)xFf?btfa0Ca~Vsy}oKM}N`ZZ!4tUPW=!s+P9#T6tsh0S3sa?ND0! z1;Hnj8Y8SW{a>e9H>7tASHMIHO4)QcA19OpUbY10>UenH>df1pJ!@xn&wSq8QXB9t zURNAHCe2oTxno1}c+PS}olDOsw*EK` zAEqp*s~Kx`PEyjo-;;xPZ!H6KyY+C8Kw$uDv9)Rf84}pJnOwD@~p-xI#4$Hm2H{6 zC*f1j22h>(5wYuX}4f2aNwK{cVH)3QGSj?f@$q9Axf))hk)BQ+5h( z0*tjxGvCg{^qU*n5u#w*Xap6>XK67(3Jh~F7oImCX~K!x47v4!^Hk?_7BIx^ z&E4Rml{cd_$)ggu1ARx*qgaJAzZ&$^n$u&;d_;u04vCb!*<&>y$SO^&-p6` zxE6aEIazt6=z7ZeG5&DKyMK~6xt8157q%kvAqHakV;t{dVUMq1A7GqIF}?$RCh|w@ z^R2JqNz3#6xid}QlSN+xq`f`^fhSp4c0$!vC=xr26{$Bps}caFt*zmpCm zeMqTri5a$Hd}r6c=6SvQ9l`HL^*5RGR1VWOzFo19#yF}cGvnEt+Oe=ub{|-Sk!`Cn zy5%(}-(0uWpy#m?#AVdJZ}v_qksPTYBku1BIO6yWhbL4xK&Q#CrX7VL>=Tl=$krJ1 zD!+CHf6+ol!LjKq``i6+n}R?TH@sw@*ryw1ZWV;&;-Gj|+sy-r}}U47J`x z;5?iR7XGsd886SlV|Cc0Brw+B@}6lmj+3?UoDPK>Qol&h{;rD0+~jyv0ys)$CA#f& zrjkb~PZI@*>tXRdTE7P2R`;mNd_;!ov6OK1OeKJ+IyBCv{@V8@RWA677Wd`tMv z2dewLv-n{l(r!+^q0n&>$QNHB7(ACDh!Jzklr;cOhk`j<#J0i2pr?C!+G8ozjnp~Z zkNG-4UaJ=x>aK5wZ)SLr%rQ03NW%x=8XaarZ7z=%GgCWSjt$CgT20H2>H0Iy?XQ|s zi$)#M5~Odmk{K;gIQ~RNtRHh!2tltd;e4{W@G;XEardb76B#C(guJF!e0;ck_)Eh` zah8tqTLZ%coLbt~)cSEAv2*;O1wjJ4f}!_4@cb^no)1q`BkAUS(yMTOSp1u)pYCS> z{mVmoKi1jLLj}M6r+F?dsG}G91y=5S1i13X8`OPP*7oWABSm1{?u)I&IU_(~C zrRXMvb$t2K=F{A$yr%ZrR~AdxwmDteaSkBhb zHvGnt5}p+4C1bkWGlLP1`z*S=8WUUs4{owh>P~lxLS-ivv^uG#eFh)RkW5%e23V!Y zeeO;W;4!-1f6xpzd2m+)ALIqQPzmDs_V5v>Lu2etKLq>TvGi_V%dHp-LOob6nVEf5 zg*L<|Oiee67qZN`R$w0joFZmmVFj~Fi;p&A?zHfl>X(8lDM$WxhVXl-N!6n7JX9Di zeq;+$EDp#SuzY1oiAZva#RY3kmir2JH_{^942=0J@SvU>dn|TwaVHrW^nOeVbJna* zqN4U8ITOWR$9}$+{botmKi(BLq)vF5M-Bj@UqT`KZws!n)kJ^k2L{41?sQVOPC*sv zv0NUeOI9D6=;K?NC%DM@{pExUcn3K=H6*M%_~3eYeFa-=R|C!2clQF=AHHyVsPxFW zQy)h`BW{4>0q!d%5c1am{kxwQg7w-kDltNOjhZH)GS%Gye*VnbZuwTm2wO%E;?}U_ zB+uCc%U&)|_g=1N`eK5yL9CFA|@)7Cy4#uyaHnSprH7^6TbvXVHX zK)=SnBenlGAYmP_F4xIvX}P!+gde&xjVD9xLLY9;((jpGPL+G@r54V{yUUC`+E5VA z(63InAoZP+Kr~Wg`ter@Fhl6wd>yaTv76Qn`77Jz5y`Ip70!Fb)*oJG&TMGkxhu<| zU{_AIt;tQ9M5a>X2i&Hmv-t+}GHjOqqsVXr6T2RpOEACHxFDT}Esu;if905EmL3Y4<}gcL=G^i8RQr48RQr2y*#J8r*_JRkuAvE{*f~>jYz!%k~wW zZWfS(9+{`cu~il>9vz8M#R;TJ`pjT>O1Zco`c+b#hR0;EEcL>%(iOmWWd-v92 zAvgjFRz;Qfz8E!_d366XwE#%r1r8T?nOLy=2-%6jv}CNsh^-!2@Yry8wwm+yBad#J zbskec138YkU{4TX%KJ5T8aSHEMcV&mW86(X{o?Uh{H%rkGGn51TwzxYN!loIXmEf;=2xWitDabn zMJY)^3K0OQka`(b54C%5q;RoMF?%#yc=Dr*?zL1(?nJUpJA+)=C+{vXb5rJgHN-5V z+SKdnI;X!)N8G?zBY6%ToE5^nm~W#6>9u+<5x0h~SAu{GQ-3sDq-)rs7`Y%P&pM!Zkw-(WbRj@n6Y6jNBwlNChc$q9LZ(@eO&nzNyrH;_ES2 zahF6d8h3YUaqhwQKVpB}Po$U%e;xK4=rOsd{Ez0pTi9PQ8X@ zq3Ekgr3a=x-S@8y2&InuaX)oyKvyFoBYavEB#_D!i>5A(VSNfZ#Zh70p=8{6y9N%m+=|zdyH%{16TWbwJGR@~md;ap$XI{`= zvS)n4YPVj|7!6wd_Pw0p-&WV8KKAIAw<|18vhlsLJ# zJk98{N8@GYyev}vK^xbzx=SjodiX=;^u%M(W+-t;_7LfdKNoaX`Ig)ykDQzh2d#7W zc>P}8QcMNRgZCOd39D>Q!??Kka4pZGYaHt|r^?Jg zwd<&o-VhDW3BTgn^aEURiMTE4mPuCoSQp_4|95CT1Bst~1}+EPuiokR_%7&mdK1fF z(ITAN=Nm1oVFX^B@q4G0a_y7qOAY0jP|G-qBFgLyObcom%O{pV(BUR&>ToJCyqK20 z&f(9?S#VQ~>mtwK8^+lfjNcl2ks^?sO%Ekm$lFyb8rl}+eR}<K_$%+PnJvx@fXYY}fXbmc7pe=p5(`c=wDJ4=ms}c|mWqzp`AiyK?>0 zJ~uA)@Ax>45=95j!``kYk;u{Fvh%2~bJnexT3Cc*!be=0}hkS&{+uu8&6nl&v+Cp(*Yj%aUIhiLXaEp0SlSSB3cr!%F|27_X(@;(vK1liZo%k(N zD40`r5{oqV3k7(9NjXnX?_hj6Cl|hZ-MpT*?ikK!t+-jcd(}?xeyHaY*SlZVOMCWi zXiR>qG_lYzZGhxBQS?dNHsCbYyj~IXsNfn$8^5(gCaCbw%eC$*-KLXoi0>>bUefe) z^kW(#nYaeoi0}Vo|1in$S(2uC%Y{WHiy)y6OwmFcJE8OOPU)6b-L=+a8JDWnrz@P7 zui+c$y;!5>xR)sAp_J+>nL+np~pl^?+ ze*qlg_OLSG5Q}jCa)=B6K?Mg0hDq67XC46gy=uZgafnk^jfFCKyueub3xrSu@Z^oH zD<^uhlSBHgWO|QVrE8NcTPsKATXg)1L>lf`kc4;zTQDBP%++EL2+|w3>ZcXF%I2Lj z{Jy$eb||&7Gt%pxK$Sf1pc$#Z+uBk^^O`;6R(V5w>Z`vN`UAbJ<8Q>OQ{)jl_(nr%h93NNbm))hgWjQDGIehW2+ z^8+j)Ds&f=Pz4i#!m(3sWF}$oH)WPBKiEQ9$!L1zV=5=!(_6XXMKS1uSb=ns_o0N0 zveo+I?XCzI)d!SQMi0_-FTdUHR@?nhrB*v&PCv;@&w?rWO^)3 zFO$O+!Z}@8*byB%_|AMNJN34aUK%rninDZFLc?~)<0T?o4;@cx%+w_hu z!bE$yBRb5-^gAa%HJY()@w7~3N@eU0Cr3V~6t)w6x9&^yA#?X2NCGQ1)c^rX<`g$c zH*z%8gtIthI2eqbrN2L^3%ybiSK%qUp-;em0o3%i=D~CWv>#0Mt?W_XHifH23bh># zSkv)R_ZhQkQvh-Gu#~x7e-#r2b-+CG84@GatnTG4dmZZ=x=g2;oU%oyWf^|Ifzj;l z=bpG=EdbooU+_eC2*795^Akkt;^oXl7*$zS^-$Y>doV@wGCYwcftKgiIdA+6vLa-v zCT0ZHWo@w~IYwzA?>svaAo0GHmaZ&#s{HmFHV!RHHujxOKb@%{Q`FT)5w>`oOzcb zZ)htj?&QN_e(;4k)S_1VSWWb%u=gQlobw;g3Va|X$?MVU1lKm~iENv$+cq`L&vMfRJ+nLGQ zaJPqUn18qR?e#1a0cE&9x3@+|tjJSlf<9k>^Tv7)VxO8sw+jIsB5;YD4=W(WRaCR1 zBBPVc{;Z}2@gogJH+_`_zodu@Dc`{ytPLrXZDl4V_*jTY+?!6{*4o=+b8Seu;TjqC zA}ZPn-6VR+KFA$mFpZ;pfeQP`SNFL`W=h8w@0_%5klI$k2A-)fA*$ZWP}4jqZwqIs zxuObi{|kBba(bb=Q%TCY0N>D5L!X9PtE+Z?h>)W91(Dl=L~Zu7vxQw{g`m ztSo#Jxn;|hq_#h1iB@6i`mN^n|8y8;TvA#FHMXJ9F5AkyC6;eGEOP#C5-uiI2k>uz9$5JuEG&Y)Jq4lAKxLu+z)gx-aq!K+36(907W?7USt_7HzvSEN|WBI5Rq6lT&ixlW&e?#Y3Ab{=1ka8qc1a$_!gJC~syj4*3A z$%9mDO<9gh5Wrw%N!eh8ou9>7~uAJPc88v^fF1?+nWTz@?94t1+z0s5>O=+ zT$&QoWE!d0(OMs|uNC^9FO{V?65tsAq+!9K5U$DUstI4E3u1-HpR-E9Gk59yBveJE z5gYTkESK%CmlGB81@9Z_0j0&^KkJCmkWDO;Zb{9$H(GI|F}nz2zTdyXoC5#eb2!Ny z@^19AnCP@2quaI1718IyF44$%8?~6(kq1-X&bYKSThg^y5!2wS&Azy{`!CH^DT!1( zhH$4Z0aV|=oEQHMSy2F$W@nEoUBsZ&rrk7$b0L5ERpXR-vyymj;+l;(A-Y?SdI>pO z5{b^omC3O>h-}wBlcutwc_+VAi@Q_bgCCo1L=fT^>W(Ol^hlAs%P!5yqy9}V{$>nn zg#GLh8A?noXz0@;T%Fn%7I$`8xf6WfnVQqx<4>-YE*Tl%0?pd=0{qqsFt`{%u&zG7 z5wm`0JZ~yT?z#MJySA^`kDoMgl9S*{2Kf>Gto)aNX(WGK=zVskf;!Ax0IJ-)=2Rpj z&8hKe!uqF_eSG*>3ln1q?gz+=%X9{2 zDsffo$-$mN(UYBu7-juA>|KZHJup@VF`}&3nsR7`8ceks@?cgM_QVJ70P#-qvt+Qa zRDu9TvOh-3${oG9ToQMvvL6zna1%7O>!9O56q#W9IhC2q$;w#&Ai2TUz75w(IUH(> zsAk@(E{9lu7pp8t%S#F#MBfgkOVC;pDhz&?W>1L6;-$jo^PhnA0<8RDF$X~c(p3Z9 z3xTa$KMaJ2MT>M7%9~c2qC)TB7^Z%f#T%rt9a_>N;QLLOLggs-5T5{c#;ubd#aJql z7`-QDB*MiBy#&M(uSj%rB8&Z`>wxPwBwtl#C$(ZmsMEw4xp++T?RlQER%qsPWqur2 z-guaJ-r)!wncj;Lr zPwH5@7Pjv@-mSeyF9~l1vjhh+s(S!qH^SH0>7A${ri?FOfM}N}3iXi== zgLG*-tHRXOcN^1el)#v!btHb;=i)UzT4!s{`l&_m@Pf}S{k?Zj?s7Y2hoUYs_G>Eg z<2Ds^LMC9&Hj!+?nCge9xo)QZ45W_pQM{&M+qC!DUFNVl*UOa!hxS@cA)hY~xrsIt zLF5LHo!YGsa(j{i7@u+hk#^O4-tUb+OR%C-q+jCK5y1NjjXmDJdMzxAH%_Rg4sj1X zzk$)S#P|iE`T8@YgGeWL8^HH(`@0|5YT-K1O_>S3ET-g_U}(bIzPvze%hwT$)az|I z5>f7>z?9w&WH$CGzix0@WQv8$S*VysBBhTBk3b3{(D$tE=UU=Ax* zwy?uwKYPHtJ!;n+svCE&QJc{oTv2?N9FM}EpphR0=)(W{`MqVx=i6^ruKpS&$Dc=j zd1~+d+eW1tKH%u}%Y3|>1;$UmDfEVSM^wda8fsj}E7!*kT-Q zH~aK@;ckDc*ss4n@15@Ya>#y)TE+SX%dOTiDGj?8{QEBxfBpP_@z)%##&pumVzq4% z&E7h&`i{sb@2sutJ z_*8qN2t%jdbrOxYv(l_OF)6;1?zglZb{Z#u`Yz4O?qz;s|8-CPeY^e}?%{uiL|wDR z7Y?t}E(ypB6^8qe;P!VXzxDd?7LN^nNv`bt0+mT_Y2tLfxdZe+O}Lx<^)@!Z6l5|htH&ob#cQPI0u?Pd*Ydk;H=e28XhrUT@xdMB%3Fy6!ehnWO^QD+nrp^vrLOR zT{__VIw5<;v()Y$p9k<#!s2hewk#y%aN}^?2YPtbeZ+ncsR83?NB%MsfGyqx(4D+6 zWemw*CZQ}(4kU)L?|t$hL&o*fZZ>yYW;vyg>6ro>>sYgQZjI{-#~4qVjiuk}Go{1# z=a%w?k#~|qQnk>2VG6o$oDEddzd};ZH!YXdeH1JEOTg)@`(Xqrw-Qu8bKEn1JI0aQ zqKu%O?%KRcevbPu-94wY(m4;M0H#bz{H46>LBp;i0F!sQE}HkvLZ-E)bI*5UQUf7} zkFD{wo;4pdh&Y1V=00CfPn3ONvij%mLR5Nx6s|0&tBZaRif6hWPw&FX-}5Hq9f$-Z z$k(Lo9`>|P>^v-O^%qI#=Mrv{%3HFkiVRld1!mVoaO8e>Js->?)7$ND&+2C*nUNZc z4!SaHFydApD7|VL@%>fF486y#>S=D-Rv9NbQv>H?tb^rux~kFwSV`VvW8t%2dv zMbbaglj$Sb^-vob5dw{bE!43Bw%Jj(M4~m4F}91JBKSyzh~coyL4uHyb$G_x&DOfU z*AVN2JHb2h@o`NC%N@)0A3CktbMEjP*|;&Yu!MhzI0K;PFOp6CP4Kv%=TM?~tBh4p zv$I^r$=qIn65;v8m+Fgy59hN_eJP%T_g>A7Fp>AL#{`k;%AZ?_nREt>kvV5V-#RnJ zr?scg`4VkDc(rt0BfKe;O&VP4YlyZfRz}5a+Z<5R#cc@^SoK=A@=FHsJ&Qw>3_NMgz z6@A?1d*ZxK7qb4Y#xBJ_mKIHyEEg;qHqaUeELT3n3%%yimq^O1S)=11`$D?{AoqWB z0h&N_OvV`=5W(I7{buX+I4DL84w?ok&6I>@oj`{!0Uy^s%c>pNg)L@vj@y2PR?9GV z`LvBKCP>u9^i6T1N85_A4^f@*Yx80b=)2`U4tz4w<*ctx=;+XcP4>m4wG@cyo?5pX z!QI`;1*^Y`YUYOCCBvdMeJAvODzNAZ%D+>K~*B@s5DFIMODR!uI?v;E5 zIKzyS!&)BEs)l+6gEj@f{JuYETZqNLIgr=PMF>N$l1Eo!AHM0#2M74^v^fvaPJ3ve zSJk-7C5WV1gq^zQK(ulL*-#CP2RaqrcrH9%Ss$=f)GA{;`wMNxteCYPBbmt%WhtV5&Zg*!9H_p@4$)W{QBlaAgFZ9TmC0Zh+pO zpQ(s(?a5rp1xf@f-0e+=N6Xf8nhCR3+t7gqi;!yzm+cojf>4ZOjNR4Q3NmZ4)9A!R z?~`O+)L8FpAK4@On{Yq*jZ52*Gl|D>wl?Y+tX?+x0E_1@E@Kwl)%$Uevn_O{B0u3= zGaX?t8bY=gyDO4&WP?xRKQ22_OAC%Zb~b$#S9r_n0(z^5a86| zOStNmJm?^<`}rB)OD6nY&s71bAT}r~`cqiW6;ySUcZUH#^yWBMGEh1D zudh8$$b4+8IUnz9^Xu>b<^esr9HhD-U5IHm{^q6`;p#6FKBg4i$BU|TCJjmF#{^?| z6xyo(@u;0aj#JnlrNQP^fbqJ6+D`))I5qCL+B;=&#d`(Ir)Y1aXmh^;el})evpjk0 z*=lpo=)V0HU4u<)X&DMAhzW6DMKQd@|_f#i)L3fEI}lkbmmV$bd(S0ck4_UPe5OyKg?Wi(0@ z9Xc;XH>SZhxBK2SC+KKB0&b9rNbk}n8w%FBaj&sA1E{g~qGiDqY*5?eG?o_I=_X${ zTPuhjoPzU-g%$>30l1E#MBMob{TQDOil^E1y_t@Y-sNCw^(G4op~dRT0O#Ts`k4JVJba~5h9Ca2JPo3 zZ@A z*%^A#iogEQ3k#B*QIJ6s>Y}k7^~K#nSfqN z;KXcV?F)eB6Z&Z0=>pV#sk3dFbCMcqN7_o36M89{hpMXBlTAC52j2{jK)qX;_fS2s z<05tLS0)Hz`0EndLZg&38x5? zNT?=asG8nc{_~6DaZA^X1%T^2i@MK?^r%uW+|Y|tGLS`{e{5M~ROzod?f(wO|E9>u z%}z$l2+Y3iAW|3D(LQzQZ2wE<-)p>tYX=8A0ni|p^zMrmlW!ZkwOAjb6n(OAZ{~UO{u+4 z%=-Ja)-_w|r}Nu%9*LV0BuHQB^A%9Y$vaXeZl^@LV@zeNWzhs#BamFGH0&5{#t6W>@O zRTUY1Qv{>};oONpDlk#@EBMSKlL0aV_jR$}#YA>ce})rjY%8OK`4 zmsT(BxFGn?!Hj&0@^h2FS<2r+1hg2L_>74zwb(g*l~h5-u~f_FjO-X?USxb)xhqCx z;>-QoT|_2Hj*pw48`+|>3Q&_PlPK8a&u@kJ1e)}vZqIJk!YjG?j1v&Nt6fg+S7kt2 z4?vcVF{ieFG<5O^DVqo-el4lkp)Rszn{+gIzszj`eB_(?N_xsd*xAdOFPkMG_EI%X zg^Yl*sLDtp2zvvQpzshOMu{2F8#cew$>kyZdE8o!7TsuZ-{;tXD+Q^p?bogn7r369 zdi)dgXJxy36ZN$(84KKQL)g6C>zb>A-v05J?J*=pQRM>|wb;nvxrT_@8yiV*Ekx%6 zjVW(1?(fo>0;{u>;wzj^Ok6wYKSWHQf3F?J1s_?aeq{RL`sd|F)pv!R3r4={yfm?2 z|NPV;7P;l)S}(2&E%);A=NxJCAo0~^#L<5E%9dlDeV~h|-T$@g!dTRitK5o^JI#X2 z@$qrSHH$GXH_7Hoo)m(l=h`V&@XNmY-Cxr94s0$+E0oMJ5^LFx>TPJ{=!Ui%JD0IH zy>Lvw-FRJ9&341|mMeGv>UA#ZC$wf!MwYKoievvIQvu3ZUu|fhf>TiKH;pA38B9+) zjh-3C%r}Y_K7LiE%P zChrLd7jz|hZ)_34Rs-~PA7ejE!1)B7oEy&luM(` z<5|x>Ngm<#bpG{buSNu(xHzLN<^B~v@9GBF9}`OKk7o#Xc((5sBOMnw75*^-@h#2=7%(yqfF;!eQD&64&UnIX9{%>`C1H>L*ak|Phej%EAOaBWU#aMyGS_zy}%vSAgx@e@R;Wjj$ah3L8$5adwL7C4kBxS;jk!PLxB zS|;HdIU$x0Q&}{RF(fvc@(~fVq3y~Rf%|zZyf2fGO05}mHhK{ud*|EAo4Y~>T^E?6 z5^*DnJ>G{kptrT;CF%@Hk92h%()NjWnB89x5_-Y)U`}Xb@wQzvb$qw6X^3U0DdotS zMRym?(VlYO597#!8)tQWl&+Lk@LBQqQel;pFU&=z(h~NsMLNHv)iiwcabT(IR{;07 zuoUDk)?*X>v?LaN=Y6rYx>&SueMreb**4vYjMw`|UnQf3;Gboe;+vUcKto@jbO&l0 zG+`2pFvyesR`zG5(}dC9Iyw?=C&tMg@q0y0Bn2Da7w@P$4oUK=N+;2z2<=?tkd$S) zF!#{RH&F?xuB8~)dy#Mf#E`Cnm9xq5BMF}?TBMz&?tU`!pF?ymu~MlTy$lrEb0^>W zkyiDrWYLvPt1D)KrukHUH*(Yv2ivflRafuJP_2ED!2g>)in#|cc66zlo_EvZsYtqXuk?xYrJ?j*wzfnrC2$pl5XqVw*~tl& zoNB**R*@q+?e4bc8BvhBYujhPe%N?0Zwz&{kIU#7tP(Q|b=S}0c!y?;WoYX0)y&wJAs9d;wc4tHN z!u+;*c!@$*Rxat%h@{0`jrXj?qU2nJ&qjP~P|q=b=b?u{nK~1|G5omedIV4|^u`#% ziXG0xmd}JS9)%R8-k%1SPBH1#N)>22lQueBzr+5S3U8jx@KcR}GdoG=)gdr%IAM6z z!|{UgYT?h5-*$FfhivNQ;hc?AZu#jtgZbbhy5t#^%ZCy75^I1O@&)3cM~Foz_?2LK z&JXU_Ojkf=IeDGV%Ol#G=9v=>Z({9|^NSCuDnVSG0El`#HMWnxPb1|Sa zRfUS4|_h8Otb70@IQ@15O(R<9k!h@D;DDVeBPN8{I7Z85b|7f8<$Oy2T- zj2o$$Y<8DOmgC9I%ByQ|@J!TEu?Y8AtQ69QblaM93cualExLSR@Uv;r;l5~t zQFFA4rfoXF3dex^iJSVuWMM2`s5giYV^fY1GRV+tY@4A~(6MJrmD-R#WDxbC_M@4d zY`K+!n#Wd=ITr`2@f2J3wwD@%kZ6o1xYlLTaN-NGNoW}AoAv%TZv ze9ll}WkLRFZzJ{R{A};^3E!JrRl{>5o+ic~jqvqIU~(4CXGDFhq_-D7Pco!c)0!b# z%s$6Y*@Yy0Ga)H@VE3p1V};7-Va$$2$ZZ-{ba-gOeN;ZJ>2CfRQFZY0>Alh59RTY#0lz>?``e@zHg-LtPr zPcBRS0KW3i6~r&VNs>vxc7FGpBJbqnz^VLp|A&u*jU~WAYkjdVy?9!3 zl)C=l{g7wPvl$TztY|UB)+_@=1(Rb#&; zHEqVzwm%5r+Imzk*3RNP(9W3T^#c}w+xP&nqfA+nBD|hy%H0gB8IUw!3~x_)(`dtm~-J8^R<7!p~U2EPgd_7)c(n{$DdxKgF+?k)_L-OGJ@x+WzghzaTR)bI< zn+O<1!=B@POd(q+*v$scBw9ugDTxX!2?!Lt%;DfqrO{-z@zpO+9L#1;@c8IUyCbR& z>6h59Q;k8+^`>s>9OshTn44#{nI5oJUsjS5wi?>3`mk)@gWFQrXG2P1x(JOxaG0)O|WB z)7;!J+b-qqCp8=I8A5YCP%(U3)##B7XKi+~$7E-DwJh4Ndo0Xad3wqm{Uzi`jLPLY z{H@4naf$iLof-iLQ+TSzE)CrydRe^GrE$V9;DfMpaY!42KHmEq&`XXS116A0VHIIT zTRD8H3TkhhrSA_YRX?5>tr_9IW9UyTyRu#6m7HsY-e9A^4CWB)5kjxchzJW~?GbY` zcMB0M-bXo;MAfn>0tF@zF!96XsSj!#R|fdlZ^L@UgGRcJw6Q!tL7_tA=MtJ&3BKLM zpP*a0JvJE5x4_x6B+emyKP-$*Zng+>Rsf;C+151G;D7=jT{Bzc)-L18s&l!w@V)%l zzvLP`X3$TF;`KV|vtHMqATl3ic8D4>{3ob)7F|+_47}eXR9ydbEb<|0x#AeLta-gQ zPTkSYzQw8iW5$(j$`AY9`awWpFv9LE3$~ex(E+feF+0{F5#yjY-*g_W9M*+WfA|yB z?}YnVnTS;e$u}TEaEdtT9dYgMWUECfH5DuTNDmiH2D6R~A8QD?XVZ%?XztZxSIizO z$&rJ$t@&rXyXLDASr~fwEK^2r)~RX3Vpb(Dl9^beZeo9T?1v7&Hl5~!d+W<=L`xrr z->ljL@BR-q2qj&NT0}}k!~!)-lUe!mtU`!mU#DEondCCBqKZXfc)L2%iclX@cGmEQFWQ^C=03ACJu%NqJKS2e02r(?BDGZ&djCiT+u=<1hoC-+RXd?j{h&=L# zlSh?ew`vmaOgu)st&8-_z;Ab@(Qb*Ao>l=3QQalvPBqz~?#*FDtv;)_a3_Ok-mV`% z%2GU6&yJ&9v{3JeZ5%AP65H~z<{cw<>U*iC|IPOG7xULYWc>2_hLHE$ySz4{3D0y3 zYxS*mx35?oDgNj(^|9U@JhRK@uxXo?aFF%EgSi$jDKR;VOKI8Phc>?Gq)w>bcXRSu zj7+-fYjM>921nm@;6`{3h_{O$D}hL)+d03EQZPTOJX#~C|zu@~c2OK$Pv(>TW&5kEmL?vZSK^16H3={4Z|9Y0{)h4Dh)$g{pu z={!3p^_J%;H=PaP)lDD7p&=(8Iz{Kxxnw^694#>erq6C=8OjT}Qk%*<*SK8rdi=J1 zevnj}LW>5@7+M!G$j*k9c{Y&4fq97fy4K(2_zB{0YT7ijXjnhx6Wf-^N$Q~Ux{~`; zI+rKD?IrTc*P*9bSI^e_B~sOT0nYp`jGiqj(c*2I!qsvNaPno8YcO>L_PXB@Gr9_w zp%-&``_Z|HuG2f^od?&l5VspcUk^R#$JAd;{^cDGlLPwGdwY{RNP;p1T%%6Bel|V5 zZj426i<9UTkq)D&l1H4ERVlD~r!k=C#$T5J`oQg3-a4nF5!KJM$uBux zC4){O~*OO!HT4YzOaRVQ(NM`kj<$6|aYZwtg70qdH>CG@RFqy$bvca@xICTBE$%ILy=T7TWo6-P9yUu#n_lSU%u-t~y}xnYQOnLL zwm>*PNJEbHa}d*6p@RB4J3(WIXs%eeC?~o|iQP=h`%5}=6z?qiT`hg$;9I9g#*U*O zi}Wweo}h zwhVdXZ>oNRob@6bpLWbtr2oirYLtI5QRVZg&bH|0YeqV9o_&c|!o!@-stoCRX|E_e z?z%5TB8Hjv%Ze0ps=(eN54}Zn?prCTm71--=Tv9$9F@gcxb^)xt6wvCh@jF6nM76lj(3tIau#`gb&*o61tY3O8o#zq}1nRxd~DS{#RBa@T~q z;^*eFDx$38&frv9^ojQ-TE~*icj+G3l4WwB<_!^f6 z8zf49JAuopknLQaXAcV<=!+*?(3}|Rk`Tv4k49$!kN=sOi^_Km;&uPRgU3> zF{aTznYw9|gcf zC^^{-Ig}Wq#oi4YXI6Erzl4h9V;?*-tDk^U+xmm?XCnb86xepJt+ggl&YeNKNJDt| zi7pqu$&2q5L{HwxfC34Nv7TtrtQ--*+!ELT5N^M9k?!PDL`Gt%!T#>Y+Su#W`&t_oKA{YhlKMG1 zL@~pR&-WIaUd@_3gF2&`wJ=#QG<&}_wL6u|TvN&bL7{Zw`c>YHqV`;ohVXpUd<+%m zPmt-Ykdh#~RE7ndB}gQoAXx>N8-48TRSkQbV?9AMXrBH=nTU-#-cNfXQsi&7tiO(u zWTcRcF%dQ5G-z>T&;@Y@O)>Q~#Y3G-oUA1$pnUYtL<^_(Dy1wg?-xvqw!iq4uCw#n zl$hDzFpBGWuuNf%NPOBx&<)gn$oc_;t579GSTUmOb!j(a?>u;?}9pvrPNP-jVI|NGl}PeuOiK9`y+|;%_N3T-YgBGb<)=_E8+}vg zCu>qRm?9hBB`pib;ox#CoY>KNd%(_V zfY#idCwQYPs$8_h#4>U?cMsq&<{MCxsURT?eXo3Ew9;6?`-0Awm6-c|bk1&Uup4`H zb$x|a4|BuPNNha|5E0i@6iwb)b6^J07Kpnc>x(w@Tj#XGO)G)j6C-^-* zYS}-ou0T|W|7cYxlLSP6SP_J@J00vlcr#@MzL zie2sLB|5cYMBp1$P!^ zyOm$;+uEP(6A?FleG4rFkhvc!7M{a3(SFaK56(`u0)gEP6diO9qDA9fS99eOl7FLJ ztVPoZ$E&(*s1KpR*|(`Xh=bl+Uk#HdjA?f{K6K51x#%=DsId^}1e84-c7Z1~-;8@T zs|i1#hpf0qS<_%NQHSTN8q-pXx@txp)1{544;yNxhAF`@RGTH*4lQv6o<>F%pSWYt z?(;ws$PdEq+(_dUiaEWwZ;O37+m5yfJ&GX`RkMv`W6mFz!Z^nG-S@wyclQf)QFU@x zp2S1N&^Cz1#A=-?j4$z-e4w%+sx*49^5C%)InS~>oXMzHo!hgUIsdvWmuHeV1Lz(= z1)Tcf zgVh|a0ehl>xJcU0tk;mpzS-8}Y9y(iP6sM6CV#Rd*Er&a58c#Hkex24wj+rEi+DTX z6Vclx`@$9$OWLK|o!3tZiVGp5Hnj7YYP~5Dg{#QY3Gk=7It#*(Pv`Fwj9p)o7VxKko)sI{X1q4 zpG(i(f)1Q`_H<|>aucx+?a`5NWn6`TDl-glHVM96zv{Sv<=QdEbtcQ-9WZtf4#^TIl;Lkbt`r4YJ>|Ue?dhKlRR_()7o*SX>U; zik}XtJgx#+)j9;<&^cIe>;H-*?*$Wf%S#pXS9r0r`stz zk!2OG*~$&f_#D!Qcj7>%5J>3N)ZUENAOP_A%siMG*+`ie-l}@fa^kfRRgg={@}{5P zQm{6uM7`hjf3)}B;c)idzW4}fOt!34oR`S|$niU%l-{pkVv~ zJK&VU*5&1>>Vw{j48jUO!bRlcQah6*%(o_EF4iPa+pirZWl>J<4-%!DJ-sja4!THx zA7O!A?I`*QqVrV5xnaHx6s@BuYsd2%L%q6|=n~aoZ1D{YFM?j`I~ZJ3P^V76hjVa8 z;Cj&AEtw=LlIoXjbR-fm%-y5%VoC01SvP{^Aj$Ur+q`0aj!Gb3cSV0-|2xM{P6`(o zGK|u|`h&wOu4Be~rNxnRYph!V+9lu+C)OgY5xPWlGg~VnZg8)kLgi4Y$2}wV^DWpH&wPcVb zKDYmhGyc^b*J^qTQbIFwaqLe}RFyG_ry9-$U0@!_!rci2)M_d9(cHAG+{XWAF9FCT zQ6^ywuGWFM)zk6CIVxDE+8$RhqXLyaj5o}*x&wefatav~O+px@XcTVA3HkP{obfHp z_pq_r$S_7N<{n0i>|C)#<|Q9CabesMq8@2#6SnEw&l6F=Vj}r1OC+Pw^j+mu&<|Cr z!}q)5AuD4qKUYjRZyHl6-XC_6Jf*AoJYhti@)ErCN=cAWBql^#nEk>t7l(`sE!JSZ z?&ixls9*%xg-KPV+f_T7s|5VEW^~8J$<8bt$soK}XBPK*>A7Q$!a$el8sLLD7qLFk z=|BXXOC`y@>u4e641@#V6?fFWPGb9@zNGXa0?O>F5+c>>rz)oO?Y*X|IUa(ez&6jf z&0lS%#>@JUccB5VPK};jwm^2?oI$^f4_Bi9%uGpTbr~mbdBQ%|{`)V58Q>@31q4Ut zzq|{c9-lltu+aQ1V%B#=Vm;{Gyi6Ok%1pWcU^7Q^wa1q)agv_7mq=dSJ+FT$920u7 zy);`=S@OEzh=9Jk{O%*csCZ#e=Y==XR|v}x0b_^Y>T#Lt#+Cq#B-rI9mRyI*dP=7n zIhj|lgykG%s;bu~7lk74l~z;FfnF8l`2HZ+M9yz1?;bszkWx35Ni8uz!nroTBeyeQ z`Hj_jurK8dx1yQLdBs-u5~iu`T(sIGQ1{QW8mSgkn$#qlBy}ciotZ3QE{1ZdWge+u z8Kkoif5qd=vqW-VO|HizTz&;&Z+^BLYRJgZT#V1(#(%fRWN4xO4+ZDH* zbs%h?zwVY|UNvSg_vAfkpkhMc+%|>sOy%ohE4C}%lAdDCSP>sm0g>BT*xc$5V@B3g zgR$$35$PGU8Jxqd)uSa#Pd;;oF!f$zx`EG+%-df`3f&e#M7n}~tQ;J$5+mU6T=C_f zU53$PDs%q9yss^%vDfXdFM+%@2wE1|kUHq2cP?qhj}ynVCKh(bDGWB2@$cXQD=7)6!BQ2GN#zQLFqUQG14G4T)~ z#V>o$jFWIG2G#sv3Su_TqGyfbkYOA9LFPK&Eq3_OCgR!CacHVaePAEc?P+L9_TBl1 z{g<5;;cChXlJ0DJcsU@4AAu*ta<>6wugYYNCqaSRKP0(7RN8Z>yq+9kxXM@lk<$)Y zEW1Yj7nr2!v29H#3dQdr!wwZGlMe*+GT)2Iq!u$Ku zZDNKm--otXEEZDr5tMl4i3cN<^HLQ zX|}v?;yeMGmO1^l6f@eylib4)hrFJJ0E|PwWwA=Rw=?b$WYV9MgWCqcI)mYYJ+SG0;QT>Tl|ehVp5&^b2aoEIunWwx zdL8-zoL6vEUO~SIEqGHwfY zumhBK**tbsf$KT=27E60VR`g(!ueliO#fVQ|N6ZMFsMc_p$`Mjno1O%>|@o*ifIq+ zr3VJOx;la}p*-0{6~b&ULHwOOoTdQEH8aao@%&qLS$XrQs5<# z%=sB6Y`x8i4+VW89*_S^smfm^D{1!Cw{)Vxb7B5)==PI;1he>z^Y$sc7FnyOWClg*j= z!kf<@$*-T~_*l8)$xfy9JfGkcmo`#{LZKH|f^QjO&r(%5=7C9kksulG3c3x30is0R|0p_hYn+~ z8J$BwFyjh9J4hZVmf~EK!jv_h!LLrGs>^Pb3ntg=P<}B(g0!x+ zc_-j{7S7Z$bnOr(9I<>Ny$dP-W(C^oUFLd%TI}>4LSt8%L zi)Cl7BLljnW>PK9JxR$1-$oLTPga?)xoWPb2xR09YUNhegmUr4=4Q+0g!?+g^w1Rk zOAN&`02{YB6(&RNX)`QR$e-1)d%?1NL*_>OZEi%w7MFu7GulE= z4jbK`;$R+yc$+B9#hqz85!&6Z$pGof)D&IL+J;c4_kGlN=|>XEr+d3cg)A@B_>E0S;gWVIJ1`c3o`Rpmin8#s z=k2@OA?!-?&=Kc*sAU2CkqaTM>yZ$+g4p$@DcL*Y2LKZbH+N^|0_37R?|S-$-Q|Vm zaqyNvDgTa{&1Kj1=ow{LRSf(4kmSySzE|M_j*oo}&N4jR7X@i?FrD2aU{RS+dozm7$okDHfsAIIWS@|#Ni^T zkn5J|VqqP6LH-Lt#-k-qw3jwh>UHP7%_<0R2hVm8%-4$(JPOvOUSP{TzvamXxh~}c zys7x`3)I>yjDPNP012+HO&2!qXkPzyOGJ z)zbUUn21$YBvL5D0?QiYIc0CeWI6KcHgB+|w)VA0FRwdZ1|3F)qV7#eu_5pxW0g+y zL0v}Ns$553RXbi) z>fAz6n8THo2TY0Kk`JV)j>WHh_RERBq9sDmj-xv)zz$x>LNRZMx5s{Eqs(akGC6Nz975Yn#}Kx9KA6-I z^xWjbPf+ox?CJd6u>-2IndW>@C!MuwLIe_Al)uR+P___7g40_;a>aC(uMHb^xhlcO zj?@nW_TIDLDYY&$prr^-1w2)D6AnKhS3{sK2V*ifx-ichYTe~$k@D4nCyyuwf~%_* zj>SK0*^?yAx(;7TY9mJ~cC!lZ4uy!!2fI!lvflEW(U}mrjnM=jfQ9-nHeP)>KS8C{ ziFe|<$>%KRvp=XTw1lzIdJDa0SNc z+3SIDW{cd#xdWRB7CfuKyHtf&QXL7EquGe?} zW~T#3-G?52d3)vC10up{nXB+5%(V)OQp=y9>)Vx0uU6$7UO+|DZZ7IW)h3k^Cbg?{ zS3RtrhAFZOzf73*FI=3lS!MGD=5?2yAmmVZqZWn?A~$S-wVhN;Q4g9JNciBY@jhtB zh|h_mx50_xD#+{2)W|qUCla^Xja&^nzwIfq8<}p(=o4Bhs)Uf^TII8Pbwu&q7T+Qf ze((j;6xs+`Wa|Jtr~rVb?OY7!;bcV@;z?1^?+&~4aa7U5{o+JrVh!zHFQpRJPFbAO zPaN4NlrIHxx7eb<)^r1Wa0d_?J-B&E=NpKx3$O)Sm0w}2nVyrQHm_q)Ep==fdSgDc zMT~Kmp+#;8>$T;;H|4Q1Z_z^(R@W}x=LC#yGxc%3g1Wz1(;S2(zcThU7jFH3+}?7|%E zFbelJ50~H7OoO!xyE57+k62ICQ$38_A^p4##8=BWlN<~c*(-`og@~XztESEfnTE6c z9Vs0Qgam@;2UDR1LJ;uBGx{6`kw_hzV}a$Glxfb|+L~v}?q)1?+zhP)Z^Q!{iBDDG zTs@uVRJ$K{bJGH=5SuBvr3mTegNcboy{aEmO8V-wF^@(&%g^Skp!^as&-uLxJs;r= z0XK`W=k2jdHcBf-i2s(S#3o{cPE3bPv}wGC+~G8gsmkmAy+5_^S$$IWi zX*!H_P^hPhTQ16YU8D78D36Fo(FjQFh^io7nc7a-_PZ^#?NWUMEXaq#^Vwp{1>96m zirU+p9<=dRL0-ieg1b3V>EJSSl2{>SaAuX9X-D%TL1o}8C8M66c`{g|BBmusV)EVL zF9+Es;2>LqzDFvpj0y7lhCID&u`h5vtxI88%@K5i4UE5i_DRYe<1^jR<;{t$swz#z zvOV05mgy!SRJM9H}$n4?&;g5S~GER4a$PP?5o$rNz)TxU668%kj4k6V7NXS8i=sU^_1I9 z_dSuX&ZknEw5!q18$CoS+znwwVuT)i^+=e!e_>meT-R#1W6{_T?^6oTXn!LZ&?s+O zYW0RCnXxqA%|b+IT|=2VRU0g^uFKpT?RI-HW(Y(fkJ$-r=Ed#=HWDxRgfObkC9DS8 zRn>_$c=7Um6UAUAGOU_iWp!2p&lwB7J$B6AotO5 zttQ1YrR#_52m?$LbN@!NNc$*ROhn&^Vj$-Q+1?)+0g!8rTIe~SE(nDTyR}%-IWN{j zPf_H}8wp)qU8{{v8=EWf$(%^@M8yQZB=>6+&k1B%XMoC$1b=>v1#o#w`H@lzSJoXs ztR3I&nZ1OR@irU0$J`iYC{;JIYBD*pW(0nzU zp3FBd8({0|9|fRoGu9wG8^3Dp?8dCK?Vk~!of%_4UR&G>nv4Rj1YBcij#0?!3Hj+s z%c3Hl^Nk`NI3xcA`RL#+32RSH4zhtOYkhD^KcbG$-<_W$Z~zYQ3TwoEE=ORAsia9<*KiG~Ae7b!le>L`hO00Qr%E2+4<{KBw zi=Z!MhZqE%;!ds}N$mRgO-_-ERg?heq(=>rqdYln$gL~M?+e#juh7NBe2DzGh~gn^ z1)QZ2R)F)}n|TC(f45Q}64_{4(3Q7-Ve3h>f|lOOhP%e~$MR6>paQ1Qcaxth=aFvh zgy>a(laOwcR(#J|MrNR-InBg&N}QyyjfpAs;YWN4bxbeA>zmID`L!N!49WXlp%~4-dGwH$vSH7L3p!+cfUuWuS@a|om5*v?oHA)<- z80-vTL+XoVEW#-9N+Sql_o?zEzevB02g|i{6zZG7hBmF^ECc1G{rpI(Ws$JG>jiAD z&-b4J;5!8Ts4K-4k-K?vSg~+CrCtfZN4BA0=OL1f3kUJD)nOz94r0e;@1H*SR!8Wh zK|Fzk^`uG40d*szsOFS0WD1L2dUAcCV!f`gB*Sd+(`*RfcsVKIu? zQy!Zak%ii{DYd3*E1E^k;~eKZ&hLymPurST-_b|1C`>&%ecn%$(E<}C;CL z>*gW^SvsXQEG1e3hjd6(XLIFaUR07POzN`>*xpj6vG4=f{kl5%2rrXj!4%xqwh2@X7`+I~G@Uy0vqGk<6sFb6y*rIJHsek9BD2tQAv?`OF<%4n^Lp$~E>pAen;Z zvqBlBwRF}gmmw;>4dkt%Ks)D6sG@3pna|c`NR{~uDChM~vwhQqkgLNLOGP}#@g?q4 z-Q-A9y&yDOxQn3wiTPkMmarq~4Nd*}n}e9QJ%^K-G$jq?w8)XKd9uZbz4GX9=Io7v z2K5u(H#bud01@bVN@-l_hB%5+$r*4I;Yv3qX8^vDPolTo=ETBCR2}ZRi$&dfY9H1> z^m(mWz~CG1s5|{8mNf#3kb@Q>aT1pkeTZ?nX5{ECS3|Y6TGd;=y)nOMAf?Wi~qCON4ImTbdNAOujy=joEad_%763 zd18$-RUUyRX$(t2FXn~0ncMBo&dTM_tY6?86Bep(EPuFYX%K+%m06qgu>v*(=J>-D zm*zsm%=(E}Vo`zNx~rfI%+$1iLm4X7X%gCKX-rY{+2{qqi+b1=%N1P1;ye*9OtF5n zQFLKWp#Le<=!IBoOl_07>xAAvUWi8w_6> zAx6>P)|upVU5X78e86NgAT3BEz)_=mqeJMi-Mf22-8xpFJ5nGo){Ls44!B7FmcG%L zOAC1$EPHNqGW*$B`g9eGM3>fT;_%($WSWt1Sh;F3(`zBc68t?N4(X{j!q%xLlTf%` z?;&xPqN+AhJTNN|x-P}D>K@WzZ}=QI%^N2G9{D<~VxUV2ht6)!3GK&p#tG*zrXtui z+`NXk&PhbebU?>hzS2_*km^i!#Ov4KqtCwL8$G^_CRy+XrTI+chIY&g+qX$iM`1q# zzdVf+kba1on{&V`7NgHT=B^Xybi$%YB>r#C!*f!^0Pi zSSS=m%B3{S!1mR`m79c>+Pp?^M09_r6LJCns0x>wi#Nt;u_hN-JFGujdlQ&eWH4Ys zbchg|1z4J{Kd|rRq$KWpAM6}cr+Ghk^=$ z%|>?wP~jVyaefu{6NKv$#B(zW;!mx8aJPZfzNI(I)vCZ=#LXB-B@V363cW3>+nYZ@ z7xDlP+!`TpyZ@p*unB^y{{&qGF2F}A{=@xC7SYJITVIcAl$jqJMjVUAg&aB3pP$A{xUuEaB+pP(-XtLZ?Yx2MIWOHerv>k(%D^(h#4`hzSRR1Y1%`b6fS=9eQ90>Z}h|lWGGvM$Dyw^_< z$JBdxPglLTZ=<9O20^u<t5_Z+IpexdF-2UF?XgxH}XJ~Hf{338$ z*Dx{gfd4;P`)yOty|-I-d12=Q5x;N#-%;4p|G`Wl?a^};)+bGK4M;M*qb-Uqp3X4F z2W0h0g!jP9c5O=+2-?0d=lyA|fUh01S+UzRH!zUasiVoDH)%@Q;Du?R=~j0XHvhHDw1U&f*rksC7_0^CpQ3W^%R|=>O9e^E>1B_n>+Y4 z_o#wj{3)_8M5xG*u)r1h?aQE|2*FGSE!@hJSIYk6!Q<&-kj!%Fv_zKv#RMoJ8pmQy+nh&tEM9V?(Kz1-9(ntS zfieB?0ZO*|J%#(unz}y0R{^N$WRXL@_h>W06khA2&CVi(#`65e`E)~uO3LNVilJQo zyZS-Pwzf~7n6w1RnBw>9q_m1zCC0oNJlSx@A7$PatetR2iqwiUz3$u78F!&FEvT5a z`|JTvL0V=~Rqcx)j#yk{Zt@bG#9@G(4xCP_1U2dwHofptRG#dY2#S!&emNX<1wE`}aYVV00- zQ=LB4MmT}GTl8*w|5&WZF z#ZE67uV<~uiRLnx&22v?!sUhcAe2FYJ+^uD+7xKvT{z=+V2*zKCcO%Vjl%f&9Smp= zbtUN;FoHW%&YC`mC26Mm*J_nF(c(gut6y|*@cMDYUm^^xIhFNP#-%Ns0%akm;fn1# zHHmJF=P`svZC>VA4f*#lD#il3wDR*uDnbD}A2f)o;X4>NUm1F&o)PZI5U9%65|dR! zRWl@dW11dL< zd=H?<_x&I^k)5Y+rWN#E#AN`poLV}H=6!&wyoL7L9E&}FYX#-Fug6lK#kY?g(s{e0 zx-tS&qTX(M-_pe;cHsrhIK>&?CTm}Q0H&=wkJ5DS?~!)5#ti_)r;VCdNq^K67xH1 zFJUPRet~<-3Irq7pKb3D`H?^5t3o7sGKuEyY*}HiZFUTn>&l^{$lE@NJfa`W6A(n* za)c~@LY7-d`J1g>9vXiRdMZ!-U+qKia}wCAiFd=6XRLO@F`M~kWr+>ED^fbeNKRi zx6)BS_HMLLXEYsCd9EC}VyDzWa{G4>LpvMJy z`!9I_HS=1j;#Zo`!k6R^pIZz&BQiWqoYl9ucef6mM_rnQHl2!qo`lQ)5h2H-ZYs zr-5ZHL2n7RBVFDh_1EblOmB78WFl3PIUm$@dT3q9h-P*8u-e~vA4}NXLa}wufFsn$ z-kOVN>_qMvS-)}4Fs~n0yz=CI2V&s1k-k+M1ER>&W*64la$S^1G=tKzW}x5! zi9Sgxta|$Ef(iPJ-cI1Cj_dtAno;mZx5OJ*F8@&S-u~RHBzej zn(d|D5H|m8uMWD_f!P|06v<$LM|C#e$>>sEc%NA#;WZy{KolUd9C`OJvA&8(IQrOe zo0(`Ul6X{X{-O+v9!_WL*A~2I;e!v8n0)m)4&C-EFcIXO%xeLlD48nsK~( zGM@g|2LF27|DupKWxJ-wiO6@04E5_jLAI|-RsJ#>8ltZ18ccs1Ji7Zn;%m2-0^0BU za|V`>nGYVr#OnX4_kX-3Mfx@~R<=9r%3G@nkxEM%Yio<7#rb{!-^#AMx{(#u^cgNJ z52s?$-Xz1cLt#9J)a|_6RY?dfLB=rIn>AYaA3Tkx1uSHm{fWcH)`S-3SLzfO0ie7Q&Hq0T?G}DHAMMVr{qcoMZPCiHx|ronIZV zKVUAzU=~g;rgT+_{(7n0h3CQ`FJ-Z)90TiWx9{^i-C zn-L@hH}1+0x`sPRz&Zp2PavV&c=sH*&f{|Itz8p`!ZvdvxALI-(EU$Wm>f>0r4}qc zB2q>>90*Pyb)QRMTLSTlf}PLIT+-+|^-AOCr@yS*a#X~#6YXDdZ9V5|lU8EkAsj6| zXMzyfj zPTW22JJPGtwT#ZLRP&a`iv_coIDI$d>(s(xzCt~DS8%RMu>VN0-~Fun%5?UUhU&Z> z(QmXicnD*$QN940gi0++5qlH_j9|(a9XFD1j6+{Fgm8jpzlnCu$e=zW+QRWc{hZ#b$7-Bgm7JkT83y$N;)42x5|R6CDRlu zo4tyY9B!pn(AsmmP<#LWrlC1CK|fBhjTOkR%JF##r+ZkodziwK3*_iRS&F$7x>s*g zeoDN@aId$^<=^nLT>MY`KenG!XZXL?VELc?n=m7*kwjslQMYjIP$C}1Uau^$5UTw= zaLCe%`Yxf^(t<^J8~|0Q#ATcZ>JObBSO9(*z{UGqC-f{I0Q3A1Lix-WLCzJjw`8~h z*2u*D^E2I_pgmELhZYT{#R>~KBdy6e+8Fu?iYYi%5BUjNGj9YUTG~GN2wBz%c{l>; z2{@DFz^i8`6@UZET4oI6bHV_F>`VarV-5}Cs;5SLhZ$5r!J&TlpCE4l-FM6w^;??% zXx(g?p9AB+;Y{9IXXuJMreJUh3to_hj8FUQiK?9C14r5uh8@VxK++$A4L{czYC(Dn z0tS)3e~$lk5!>JY1bJPQ_dNZF8=wBeZw=z#5Bja6zc-xUcJzO3tZ6Mln14V*QMYg| z9v|^c<4}=|>&D8YP!~6;#AdBbdpoJ*EI)L1a`r_IpWD^6v-h7G%pYa|9{e?nUsJl6 zF+M`^AAZZSnpo^|^Z>KMW!O2Lj|tqVbs&HC#YR1NGN}Ny@%5xCVAP0zYGq|`RGBpP zQI@WFF9(3gxj{AZV1Yb}v9tvP3zNt{ycD$*J-)s5rwfy%C`Fn=^eI)qXF-dES$^iSwh-ZX< z`=D;81d*S3`&A;J`9*D{)q_hsrZ44g>_j$d5Z41i5`v$@|GvZhQU~-#Zt8P!psTQRq=#3sTirs76R~I{j!7=dpkTc=d4i8-#tb_rxM z-26VW6`DMbxY2!WK{5ZSvA89QbyS9J!-rfw49}8h<=1yO*{_1-4p>$iwX$QkG!5Nj z;86D0k^I~$wP8@mFLMC$I<+X8-ViG@wmPDioq3X(oyc`9GCxz>z)!kM_f)6J+YwKJ z73}mO!r#Y13O6La(lxVT3Ay4=3|vD?h}5A{)q6cMiySKNb-ofW-wf8GCpxI-?(zvi zpxKD{jp7@xoD`ce(cOSr;eMLkx+d1bp@lNQ?c&n zgzTaz&sO6;T<$T@7Q}-EY1NrLRT}(8TvReH;1I@RTsF`5OSZD=YvR;`>zG211tQop z8hg^?2~NO=h&M75JH}TxGH7LPd)_Grh$T%6khm0So%WOJibG52#e(YQu^WL7)OMv` zhb6xBsrW7UPPZY&oPiLr=QAKtDYBVvoe`+tZF|tq#!NCT$gXm4!4ipT+kUQBewT~o z)X~^!x8*LWEa+5VQM0Dl1VVFo9oq2#6T|U)zmH7{gh*>L&WZO6vp`NIe*hc z3eUyqGpSay?nxrri+w%fZ>Btt~K+d^UW!?r+ht8LCL0T8nVk%W51YQ^iwW> zSD~$aQ)nA z&PIlzv9>PJeK~D9GV{vwXB=W*7KyDD-fJ-vOx6ztH{ZeP!&log$Qo}K^kgX`+6s(j zt;w$rvCF-@$@-k4_q2kevQPPCo0fW?G7-p&$)RutDGSs-;JniU@M{ zT)3AyAYO<+v|K(mVEhAtugqxuuLgZ-7PyGq3i27Ws6`UiW!SZ9umYixLCF$#@1}P=R<8j4`FB5LO`pfvaAb=}O5< zLT6BK zA`-r9M>yG;)b5ig+kS1<-7%i@)!YzBW@o}3@}&CcW}L`OkuIq8`TmV9eiHu?F;V>NIW4Eb{^p! zpQKl^m4M1-eIGreHl=-s@H0gSg~GMF(ZW4yJcLs<@@EtHf;qY)ojnBi_<`~D_3}zb zBjGk8{egDba*`T(rVVIG|};M-t;x2ptoB$;;ajJ`&LvxiW?Aqkp@b| z{_q<)?%yjFtARPFrfm1kvwHNfZl?wtEN<$)c3443NA1&e!y`3L4&z%)7^AlBTAW=? ziw*ENW)0%poa((B@I#?=|CjG@V@JOvD(dbzM8%$d=Ll0_EcyPFlaP{MiQQ`}pnYkJ zm~afimEndV^Ol^MA#q=F9ea9dsrnjMOq{n_8xvi$jImStIKg+xtn-cY=Odh?N*eiA zZQb`omnq>D&y@12UlzK<6Ng^s5Jpvkipdd)t#MlqH>T`S23Ci+Zlvr#wR|!|eDyPD zK=7Ku4}j@e_Yd7}BMj96V4C)_4uaN~3PKX$4w}|) z4$B1*-zz7{4&AB^KzXvUe8S&JWi1x2liLmx7ubcJ79Br#sDwj!52TC2cefGXgW`6K zkHia39ybM(7zKh0kEEH>XDLQ&677kNR(wSpzlk5iRlg118{%3<6R(42d#k?bPOz5t@pqE1l^Tox}X+(D-+I{YC){Zj;7DJgxO`Fl zYvjkXQHeWk%m-bsQJZ>+E$GxVExU_3P${0z7fHVW3>3^CSUS_b{pczDNN(X8W$IZp ze3jT8mc0o+-k&x+nO>Q)124b3*|4K1>F3$wa@r ztaan4HTRF+DSbS60R3@KfMJ42yRWmXJ)-bK-1?@0S5!!qiI)v^1yk@&W6&{amhBZsVDYsvyRw(|5!F}0sDX~U#mbEn4cqEskwJU=RBwogm6!NV5y-DkM zXUHAEnJgul@4n9v_JB%|z_n1m9XrL^v_^QKH@zxOa?ze{a;sqf;hK%2scuAvV0*wNB(`s$ORI5@D5%amOuX} ziMB>8gt|0dhL@ zA;;e|$%cD$9H#gr<>Rz1SI1uMo69_k&hCz=t+QzV(sAtt^U{83$C&puy!5E(H_r#A z`ATtmL$@ZH8VqNf5~{v7)C^oq?So=sYS@xFh|FSu$t`0>4}$+&R^~Ij%+p`{kHnP z_;KeE_B5{5hsbgj#t5;U{{4>x%pj&d_4l5eQ3-)+Z`ig<*#4HnSqP+XLV>LMKXOLZ zu^&mH&)SdzIku(#SCi^ha|hDG5%uzZk_D6Z2_jO>eEkiAi1X_Z>F2EYE0dKF7PwoN z-e78$)fIGb{)-uvlnsnZ-Rppoo_@zr@Qp`qCPYs*Ya_iiY}V)hX^WTf%9&P10>h#J zv0os0)p|;fAk697k-E_BOAKxte58>J?{@TWPCvh~cIobxuJJ$D66+fo5>JV!2bH_a zB@taEd8zhCVya|kOHR{Dn`HsTn|Bf~_hFY6`dRPI^p?)Fqv(elX>z!N&_%}OdVGz= zB#q_D{3UfuvAPU7v5MUM+g%|W)~kn;QGZK<9S4$NA=Q;Wf8@vJn-aHr$2^4dhq)_vd=;1(2#UN2Ff~w}=F7p_oEz9}+xy zv9(@zQNatPecaodDhK*S?=iKfcf&iku0H12r|XmDyvk<11+HPQ|L2BuwLU7a60EOV zHFdxzU)x+>U0)G0_r9Q}>2C6Hp6eZ-+ro9q^sU`$@F;RT9nJxr8kC_r#u?3IWt$zk zvAwd#7y>Sq?}i}|J(h?cy4ntveHsw_SfsXRbVvPlSwcZUG0b%d3}*}%h#Cy!eeWmzJc z8pRMI(Eh5c^om z+AZ?4b7O2a;P{a|_tW?Zng>pyjG+1E5$7|7XVRlGN9y216;6+_j0C?sv-hQ~9rC72 zW_5ZQlrzI07%-!M*c$WsP8&V5^T9eYGedE# zlq2!}t1l@}C9i*{e+<&tXkAtUUAugy>NNkSbO5F5AN)ywC!Spdi3+Xbi}zuQ!XeC+ zk!BY3H^bsuI0%WQp5I27FmNOgfs>P2X8UhO*IU@Yz_oXJmzXC3s;0ll^;d~ySyTfe zs#BSIEQ_VOQQrp#zjhKNzUGX3yQx!CebE8nf%==K{*;CFzp3gMNfgK>4@n3Yz0k6L zLHN3DMy=ATyJH1E2Gk5|B0bpF4UGTwMgit1Sr&(yfYMPMVrPW$&a@QAb){~H2}SE{ zfw0`KN3kll#Zzc)Z}mZ{ZEX7G1A0*aT>sB?T%OJ{s(t$=JVh%$(?>{`xhB!^lgiy& z56)TBCtiDTsMU!pSh{&t@}!cs$_&-pUv6UwUd@-%kp!|+|Ekx&d(mNHbIM?o7aC~= znF=fX_R=bp>f;|qoEQz|z?F;6&-geNbS(eb~__hq{Ry&xZz|P} zMImpNS>>fDotw#9-U>Dk0wp@Uo%EGy29hxde~$jUoNu82$wZFd*M;)@N9%y_=h*K) y{V!&pgum*?{ug_v|LAe?|5|&5zX+6m_vwH5{P=&<4hVk{E&T4&|6+YQ{r>?M4ddDX literal 265959 zcmZ^~WmH>H*S3on2_77ZLsBd_6nA$hQYh{YE$;4?06|(DS_%~R;uc(5yimLpm*R5r zyyJVvH_kK8`IoViy|VXOb4|JDwc@okmGE(?aZylE@Ku!Mbx}|-f1seC<6>hWuOt}^ z!BJ2cP*mh)^!-o|dvU_v*#`gFSPz`My>h<_)?(vd%0{Rvk4$!f6SQ@ti3Z3&e_z;n z_qDod?B~~}@5?4D?Z21b)Zr$C5s@@=!ZQyMEIEk7`*GomqrLp=(3}C~Mot9F=jjV^ zr?U*(h!=z}#r;ly_eTErhYcY5|KC@={bPXt@0RCBELoxA;!$R)|K8C)$oak6#{%XOLEekOrAECs9 z_Bd>&|9;lfdlhiBE^fcT>FvW!aESWcfPlvT3|!4g1ZJrC9FxCE&4ejyL`g$^SL~r}Kp{CL(O2 z_-zYH-2Y6YA4eQepU?G{E6Ph%r2dE<;6umwk+Qd`AFe-nMPipVTvaFgnZ_%Eb^8&ODyhK|6_8o!sx_t!%8X@bN_94 z?yjhzVGf!EF_s=h;4;bzm{oop1Jgq@HX1Gr6haRg!r8(Yj4qV-V=V?mnOA5dU)>`F zxD}NRMm{iupPw4ZMim&7sSxPisw~6Wun-)U!pt*WA1JX*?qx$Q%9mUSv#KPDz>3lX z;zpBGyd8}D+Mo2rt{xCUvi1X3TpNVTpY=Vl?rBS)`?P=|cis2@Ja44_YYisJn7IGm zCY?=SV6S$ZgTQEXAxu{O^!`!b&_GAuRGxvT$CK8oj`G(pi2*BK`6%HGp=kZ|A$;ug zm!xxVkgr3e4GPHvL?Kl5TxAPlte1PpiOS&Em{DfguZPPaIT$n8xacD{%#HdJj9G87 zK+=@#AOOb4Km939;dyqlu<%U1WT{XRrm#&`TzP33pFbeC&ct=0GN3&PvA8LbB(v=F zxPmicHeq+rSU*yi&!My$CX_RbA>IuT*$EuEKNiV+2c>BeEJ zhGX;S6bexN97f4graJ1}2P?c&p~K_uNX+_`I=9l9v+hey0k@I5?3|EV-wnsY4jA(lLW`M; zrwERokI?NbChGg}CQIs6Z2MlFBxQ`*7#iAuo)jOZn@>tCYtmq419!ISSEr$pNmN9^ zqnza0i@iABUK$iGN=8u)FC-!caxbl+4_rT^RM@|WC=>~d=mTu)cK#7!63qVXG~K7d z#NePvDBidD=^L|K%sN}YcG#vswIEX;&7d@Pj%@IvrAkPa3Nb}=ng_sVD5z89uRVkF zZwG1c8U#0@E%)Jdp2a7{ipZq=+Bm{?hip71rz~Bvk81M>q)nnmY}!WXhJ_WPY_JeF zTFyx}e+2SQD#*y7hIALKt-X7fXnFVfV?&C*_|OcURKEur5P|Cjk0C+DWJ2px`OjMN zg9p`sGT@5*uYcyHSgQl1aokOq2JW<>2lD8V^Vi%vOLpC0nBPBi*Ly-@cK=azIj)`) zhECCIkbk=0EI5yUUe#FM`2a4%j+MRPq_?lO!R5%MM&M!Bb&0%TiJ+0{ZCD}Dn}y_+ zgUK#w{WHFEb=Ekn$#TE@G}UuVv9ur*$YK7t!0B>#%ana<*-Xqg_V-ltnp7*k(r6SKS%N&QoP}i7f zP}?H)-^`M*?EH>l(5-#)SH#5bPgPpMc6L@|D2e`xr zu2{Fj!5Bk;$!uF=eM7_af&xM&SE@AB{i@&uPR(1=`yhddh`45h((#HDlF!1SK#4(Sl4NjYbi;SmRNcT=kERRtJtN^pdVdD~E zs40Ll1us9dkBp$Gp6#;sDVuY!6Nsw?&l%AAUA=Af$&KgJKyFw&i0`rK>({Rr>uouj zUA>Fo1_pwmy_iJU7)C8_T)w{^LtKXK4(2It%;5a?LS&g_EKn7acntT_+l#!`t(0re ze}aa3Iy#yH)V6Sh;f{&;1znEfW3PABo;Sl)!5R)Pn@!qrp`u8JQ*`p-}<+b z{z*|en!r~sL4+u8OHDI!qmSO?$3@HQ!Y*Eb#&C4y&DF& zwf>WIbE34{yCV2&D3sx_*0=F8d%-%Ft-*exxTPg^txNKqp(C?l@M-A9Uduc6iy5KP z#`x=XUwY?*hM<#*w?=+*-yhbWA5mq$n_Kt0y)$&}XTKR0UekE`WG248s+rZ0Ikn2_ zduCtIm8rsv{`oFv)$1N#y}tSEa#BaLId?Y`O9dy39oN#app>XP6rA%WVcC5YHo z)Oc1i?hBqB^eYtbZ@4yGdCexP`kk~&&w${=j|r#ZQR?zQ55U_zp#F~rn|~w;eBeW6 z;^bs}*iDS+Oty__ZGCzoPkUT$4+v~FQiZ99)2f2?LeW!HkK23x-RP!M_!wr|@WU8)$rpVGiFp;(iw=7QN7L4brVBg= z=Q+*}JHed{MoZ_Q4K(Tst%~-6;)W)Q*lgIuPD~S?XW+Y1Zx}yDPWyW@3N3u^_n|a7 zn~tWzx36|Jst4%yivuC?yNwX!gG_en=2=fjEW+xJ%Q=k2tI0XmwCv3RhBuji6TjIR zd@ML7bsZ=R`+3rtrRp1XbrrIzP0(OyKDMl&2NuR!ntMGk7Njr8xjSoXODi@n^JL@Hiu-{@uf`s0`>T2PbXf}(@dG~i~b5P^*^2+1#5AS3) zh|zd=wr$9+N)tL~m-cx&?hEDK9j2I;GKKSy6Wst|x*K}GjylXX$J7hqb$r|N%QW!l zXVtr=jGtfM)Cc@iee=x(@z;RE=(`iETCnlC*p+qH{mZ0GLup4GyFJEs>k^l-8*N;S z!6jYkl&#tThqMeAtPLP0c7%DyqE&r^9aV5TuFDI)Wrl-SE-zp_*k_vcAKywxd6#H+ zF>EaF7|pzrmre|?#y53E`j&k>-iGDj(dd8UpZK9m4feeDSDbbexrlLNeoA2xk*GL` z4pW3>5C2trqfdTAv#a>-@xXKBG)M z&vAyOI$~O23Kj+E9LM{DjyC*ziKMh=5oJuct+CMzXLN&={LvDy9+r{sSY_gePaa|= z0M(EVNV*{B!;3R1Yu(f}cq_|7%_N!IryV9OKwVr@L|X=o_oMEIpdc$sVA7I=NQNSK z(nN%Is8M~>+VkDVVxME%QmE%^0(ZtgS$dot#0xzTlrn8x7+RS$cu<6R+JuM;`Tu-C zlgRl9AHfig(bmSlim%GGoFyr(3Gm~v>nulwLN!9ZJs2RpLs z-F7d6?!2Wers`O_-AGPudMeuf7OMh?Ls*H5MR-#CTAsDKIxPT(-Wj^n*P8)dS7r1b zFg=*l_EF2Pv5&$Sx0AsI7LV}$3HnDhw~hlz#p7qOPd_6(HhyS+?5)n`VQjd5`O?n< zkFz!1c^ywjbLp`pbSGsmRJtl#SUAprk(BP z69~Djs2Y+~>Y=_7<|^DAz#h(!bh76vcRFMsJXuy#f1AW<(*Jjm!qeu2`8{+vN%R8l zLbumD(V=1!-1t|`f8oJD>7UQE_`q0#Gu(u>>>&CVVEN(Aw(z9g(|hPZ{P)WN(NI>X z0|i`}fgTa=sQXLZ0`YZW#3# zaG^jfuHd)()Xn||CLX3z(7nlBqXQH!W|c_=wyg}Qe@wgdlA|?l8$Y97Rn&V8%I2HV zEY8kP8I9WJ*FS7;%X{b=|6j=^H4Y*mATR#1;`8UU%1YIQ#6*>{*?a^9h(|Hb&Q5K4 z()AdJO`!{+w7EJozIl}V`}gnXnws*Gl1OjXVozi(Cj%aAGx3_SdNn$ByR0RAwopO_ z6x_KuMjNamVh8I0jj(#9!%mJM`yw|Yq0hq$Ubj2h7mLV!HC@!q!W5pg@?iDwN)JZ&yseVz zNkX)mu=xntKWfy6l0~8lsC}Zu(jqF(pefSwY{v|t&PBJhY0HEIo{c+OSq7kVJOh(M z_1SHZ`2Feyt<;+At4vvWICdG3b#PEXaAKpUw;u)rS{sw6Y(X@^iJO~DOz9A$Kt$a8 z!E7=Q+h20RRZyioTDO&qALsiP^iLfs-+Cv#EoIz%To}2AvcSm~6;F?GE5mU@5O#zu6cNf zP4U~ukpJXkYoXWoOLf*S%%6t8A?wO#FOU z$(EXU%3^soNSpl|5F|l((WUi}=BY|H4Jk>Q`Q#H2OOn68tQC|a<}`}qt-5rW*mgV_ z*jCjF{r0?3aqDR)we_@5dOiD8H|2eAD5df5x@#y@a$Vu|fk&6D!JHAKA$&8A(7Y6` zaIrUSoH3jOvqUBW<-IC9&~n}M17^`ICL`^zDHL`=`M1t^ayU*@eyiYT+=|M9{- zC@DrbiWwMKNT|?6{crGzg!})DK;n~zYJ;}P$SqlvMsB?!<}uZ+J3yfPy^V(VX0d%1$pF{kr>Q&sfLt~HlF|G>s?+P7LB4a7g( z?k)MgSmTi^Vyqb3@P8FA1}NfmfK;nKc$gqaQtE$Fh5(8smzd!Dt4*hg9KIZZbY);a z1S!H@kl{vat8QH1gSolwj#t|r9`+85Yb_+RYu=7&h1_8`yKadPAXW4gA^Uz$m&SQ` zIil?q7|fVCS=;>~tj>eWn(Oqj_SBvqh{M%L*gpc-V9FWjwSBJBOliL6Yw02wt4HQw z>M+W_-}*GwQ6o5*pcuQ?4w2a)Z8VUkZvJ+ZfR7RB8#dA+7_yK`&&x}6ka5P(LcRCd zx}Uq35m02oeOu7`IACriz%{gZ4>+F+O_~eN4~_?Du@z%)0r%uFVBwTl+Z4o)4|}{{ zE7R1mSkcgkF@^6QK;49fJwDPzz}hbgi%mH4e({BH?C5GsyRtF5F;h(dRa7J-U;pqx zpLU8NbBu&@P9DrQ4gnxz9$eU+0Lre#L3T>8sff2>$0NQk!wBt7O6fBelTMj)vm3Aj45C`>T8e|`G#WkwJp+gB^08ddp2OW z-8p;E@ku?>$_S5D|5Py{kyLb%22aLf8BOcc}54y7Q zKP_*g58OTujacM1?^5dRkz2xXk8qoh&Ns8(nGg`nK^oHPbFtIuBym;c1jacF8(+(U z4!3T4Enhy(TlQ(vIX1lL-66BZ+Bbdne7$f{srn`aerwB0^tsIKHrBW?h&tdisrNkd zc=Cq~b)Pwj#gR)e_t}gkiNwm{<-czB2@xsG%#&g!_JTuJUtVS&!a*LgkbI8aGqwK1 zkl)|_c)GlcKUf^NGXZ}qNaZ0K(j3l-(|Br>TspIK(xW0AGrcoM`F_E?mQ;Wli8M( zo98Zhru5&!28M>-!yiwWN#Z@$?(XhI4TrW_JFh>9#os-4x~jZPm3&d+7}8I~;}!t3{17!) z&e;(?6&ge(^>l83J8vB*I#*YH;4kt+OUl5sF@RV2*9ZP0IjPOL=h|>|v&j7Y3iBt9 z6tjP^4`*VMifgcB@yDOf+bDbNe71~gyHhMv?iO!2x9Xb&#lG0&(dZ;G$h9XDHvOW7Ko>4d7>$5wo3#Kxdlo+n$ z$CZ9}YuZxv$vThTW~)mlh5o3-(hbvq)l%`E?p4K43&Ja22s~_)3P$O;Jd9N*%9=Nx z5r=1B&?a&(wtw`u*wagbY=+!KSEh1QEj`@2ao8V0#Pt#VhwwaK+aDsO7r{S-vaASw zW(CuB8>K^dGj|k{UYcqVZgHyq{hQNdA$eQ!5nv|+OKc0obDaFmV>Ma-VH}5UyMTVR z*KzKR`Bc>*>bz82{L7LAg+x0&SRNy}_jHQJ?;D-*Vy8-6S;~;2#`c%|iz+7;5!y0f z?<}4o7SW+vXYDun{n{&TpVrn?HEkuF7pi-l#3l>#7^r9%a+K5R8Zhw2KEE2X|MQzX zz+a^m{$-8^oCr@!%+g{Cb&)y{J}#C*C)4=a=HwJtleLxPKwnIzkeOMU;=a*l#y-7) z#@&AR5td3@&R?Q3+;DR~_9Z##>|t)oqmeziyESG>1MdqI6|26m3h1&Vqm>I)4~12J z$xNbKS6p{MGw2E3NW{P~$_-Bkkfy(++SQ)Y;WiV`Lnyr!{Q=!E7z|umC$9+l#hL4r zav^&UvgaA>HFv4F%r#Ii&_VqEM4-!Uuf4sStz&5Gb+{MVTJygmDY8RK_V4QIY%?*i z_3tQ0j<`SH<72Qq6EQUnji|?O229M`q@)357LrnaD%xSB?tVKuHD&+wz*hOS5(0?q z>=b88$Iekl294{jc5Xwn517%>(JUmM83^E)_qlQ}2tEItTG9@0G&vYDGpP?COG51E z2-CU%oP$ut_keQ0bBizzHGDMH89|#lsoZi`hLpe{u`xRXUi-f+1c$$pw-(i542z5! z>rAol!|~8S(kK%=U`>8`M%8Ia53~r7cAqFxGK})9!N#JNw{-gzXVORs!!h{qnTuwc zxx~ipiE?+FiqcGJQQYR@-`cx&S7`=oRZi>uqwCGzi4oKpBd|-GJ2M_h((DPNa$rXQVQ{f`3)6y zen#onE(DH3PzL4Wu#2?QkWNSE?OP2tXi;e^906tLN9vQsGswoL?iV1k+-4;@MNU?^ zV@7ttaJj}?N@{95tEgJmal{AgKb&@h`tPkM?eFC%Z!_T>ZVF^S3u9mRo|mj#7s{7l zvMHGUWfl^%4w)<{Y*Hi|Xs%6`iC~mMEuioBz_|%-Y^1e@tc8EnHU;4hB86IjToKB` zPhi%ru4&ekp46{NvJ<1x?R|s8Up@e42V@KL^J>O?NY%jqvH&V9F@e)Ha@wQ3r_eb2 z+cRMTU4B+Vv66kqRPTMU1JT-&f|?S~S)5ppl!E*|l-{(|zu-3Y9Ajl2<1~*ST4P1T z^;9p$1aKBJLFyRUhP&+gooV&B3fJj$Hhfg?49b(&wx%F_->55g6l9-3XU!i0&Gc+< z_;dW$bzYkg6orGcSJmw-Xk4o$fEw;BKOgB2*WO1|Chn2&lG%Ha{%WLK)10mI=gN@> zghF1-om=yJB5V@o=Se$>IoU?)Ip0)uMyL`OhrpuiWJO@8|*ekgwP++^LfVV;evtW?3#@;v$GL#9_7*u($?1KNV~TG z*oh=^WHn{ z<7xJE%Wd5W_RUy(L^+^wvnKY~q7P1TI1qaIFr(NmTAj)z_3Ybxs`hWpf9lr(}vV0sg|-&w`d|zY7gceXWb7Z_jwcPs3z?-oNlE zPc@IXM=WzozB8|Pl{z4;X2H@6lN;`;J{&Rbj~M%AqdLyYp1TuW;WHy)C1*)A_MuLX zDrC^p!{r4pidb*FL|)u>jrCIdS9sZDRKugoN+;BEh}H2|LuPeRld0FBfjYe)IAzGs z>ybNP^C3e)$CE4*lj=%bc(heog$pmcnql?|sMR?=3P*pXtDf#L`D%~T$}YLzJGFXm zBrTCgUwVMry405)@$mX@a4vJlr#)#Y%3hEjIakU2@$Y>Nu|E^kunChgp;b@1EVc*N{V0K7!c zDM!lv6??!{`sUB%542YH2R2#U`5Mmab^-kPf^_coQ=dY(bA*zMNzvZ}9N*ayjeaec z-aKsNclf`!MQ1{9uME=+Hox8Y?`VpU-cR2HK_JlY*$)S;$L-G87-%9Xi3hL$lPS&2 z@N#1eH=o@=F{acGBvXHVKHZ0ga5pWO<2V^SKEK-7rOCbBHicgCf~o9QDN zuon!sx)@OI2N5OZHK1+DOQ9!(;FKObAcS^p(W56!d&c`bQVzbB3 zw_-cUkdgJzvs>PYi`3&CVz`M!Y-ev&b>XXt9R~gG5O5EP!;{5;p%JrWDh_Xfi0j#x zsaD!!iw|nha56?61h78#bMyFsVY-DVma-Vgk~hka999IHoHNz0*r$o?;-$|^UApw` z9`WGQ@b_elv8?s5V@~kwlL!5W!9nU=I;urYhRR^GFSqs#_wzK9V>bG zJOga8v_XC4+J6_qHiP1@%N?1RT5IH-riVQ|dCZC`z~0%UwP=6dXG{(VZTirk)nFH6 zYUc}Ms-E$c)iw|$&wZ_H;0|o2vqp>>bDF-+{JxUPtBIlAX(=np%A$BrwP%RwV94(s zf|CjLT59i#^y5?;+aR{%23bzw1s|@j#qRLKUsuGje*KEVNZE|76b$%RMvR|wlmXpi z&m7roIY*5&++)}6N^w-@GCnvSdWmC)Mw*6U2M(Zd!+D|BvL_d^m2 z)~OPczFol)7q&TUADTYs>%%TgV7l;f6#?p^;?De{UdG1xdvs-XDziopto2v6n())*pqtXuVG~R*#t$OsYI7kM-Qc?(pSfMIILqi~L z?%hKw3|TzCsb9WzJ&%S@3!yB2!|bAtbJkM3!Sr{|PrJn=PiJB@qV?Yj?lKL%4kk_t zAt-Mcl;5d> z*)7Nxn7-cj_5bbOJ)!*yukmbP$i3d_U+nE3QksL)?5>1fgQOf^+Oie?5-_ocW7{_t z%+G{q@;wT0z8ysNuzpRsZ7&iDn0n!>@w8!Iv?bGG%45uMe;dFCLV|l$cgflOuFk42 z)d=`0IIZ@VGXamM;JXh|qH5Xv9Ri4}e^b*L*t{1RWL2h3Nv;P4yGG-vp>f6Sh(Gg> zjRdLT=X&o9)NDP%*nNEYc0Dp!uqaUyM1`Lmu)8x^b4i&*a+y9-+eI0YPvnf|zfRoc zGAn(w^mzgE|soh`gE@(a$*~%Y)bP)Q~ z+1TRS-p1CVg;W(Xu~S;c0hiKIV?-xPU50E*yCYMYB~3N)W?EhFZSh>I(oz>UzCWBQ z#ZuPgK!inmc!a(y|7_CW{Pc6M9NqX`Ch6W{xqXh;R_1u#+?cmms&})zsY%LVX{i<0 z*xvjpbZ_}ausnX2AjLtG<5O3hZ37!AjuD0&_luCOa929?|Ldin?kg)RnR1k8S69{b z=1gbzfv&Eu+q=6E$n;bR(%LWg=l1^|+MKKf|M>aSlW~9)N62LrxrKJy4{FMaEOE6Y zn!;<2-Q=TOgvLHe1R5h-oyd+Ltz;1VB;;g$|L90T-cN!_-3VFS7Ch}2++GyRK6&f| zt`3Pp9#-WZ7(%#S!D2NBJle1H1cLIXLjDr4Wp1DL2*3PI(S3utH#QE62u_gyg2weg zO|0pS23h@6{1}1lK>t6kR7RetnwS_inP%pRITY@C<0BPI$(BQNJix^ntxnEmaT6KG z*J=SkS0`;~&mDQZjy#CKp5Ns5Zct%yZP@WVxwNNbBQ_EeK+XxWcfM3>iC$e@ z4JxCgr2Kt%@U5b&>(hTWvtzYPRJ?B37X}M7RT-h^%a(}!{?BlNCrBKYIZ=rd)?p9^ zCuT9Pc9eRuED?~j42l13Ejw_f3Gv=n@S4mHY41XGLaD&MaCxt&AMs2i!B~Z@X~1rl zPWd{&9d(o34hONdv6P8eBY#^`|6AgWc!6?925fhjwkC01ujUXHi8NSYrB0`^Vq1rw zHT!Yp{_GY2D}ty;TfPdCjO_1t;dkC^y-oHaeweSS%Z+cpG?EWEsvT7HO1Qd&u>d^jFo0b zH@EU{-zds}-{ipx$R29bho;MG!s+Dzv}(aUjh+u?vPDFa1%vJ+#C$oNTY&zalWXf0 zp_&6eofuS;i}v~@PG)9aCuiqcPUsxx91|tLGMBAFq0iFG+k5}yWX}at7O(YRO-Ih# zy5#QLK1-4fC(Q#l$K5UZ2guiN-@^$XZYlmek9MC<>{bU|t_}C%LhhGWZsx?YF8;(y z^>)8J+HTCa%|0LO?n*6LzsEJdSt7T3Xh#h}oRg}FA>q)vB2+A_8$sTX*v7;I8&M5Y zR9ln;=(zYBvq0hi55j`G+3Y3gX0|DT#3Mi|bl2%=`)&j&(j-D%%IM+R z_(+-#6>m&n#I9YGqW7;Trz2;{McYYzg#@BCa!NwuwqtFmY6z>n!)2iURH;uo0;*2h zFW@mpaBGV{o=3bhJtaIRDG)J+z2Xz~r7Z+=jNG8^%Qu)YzU_O7Gl{EzWnma%^X^4| z9~XFo-@N$smQcoxtKlz@YwDUr3-5+$n*v7M+(A_}j2OIT>?fE4euCt3K&4H_}g~%8<;sDz|f&2&WcN1?HYbit(3f4?tz#H@L z87ghl>uZS5P?nu29aYa}ix2UG3MiZ+A@X&4j6b&u8JX%q8m%XS5PUH!nKb(cA=FNKy9%8-fF!`fp0~hI~ zTArFLNjBvRB&Ped;?Jd7UT_rny3J`#o$bR_#>U~kpVTmcAiw{F!nlOEGEW7Lk4qR# zM`N*8I}8Oqw)p}ow-TTAvu-z6@J565f06Il=i~E>i?;0yTbIRFVmT^!d2joN1Ec~w zP|wLZrW6_~HKCN6j<9P>b6MHF-4o`0{d!bv{g%M3ZHMq?lj1qN%C!V$JYeEleAxtJ0KucyS)R4)`WpK0X z!>QH-O2lOj2fqF;-;yJX0|ywlj6j3_#KfHFK8@5O)Vm9~){cH29SWRfkm*vEzQiK} zMNwkd zi9m}o_u~8W$!>h847*T%`s0gUdRYa$^cfj}gi>b)&}24}lP4dd%3?A(%J@n#8*r!) z6D(l^O<0VlRWE~RXeZ#Q`o*Iyd;oJ#Ran}QMF#U8KaKnYX7A`|!~08zaIcQUV=O6= z+5>P{)ACj}iPUsx(AxOr^OFv>9mVDaO&rR9cn1{BQEkfdzCHf9)aCarsp%@OZyvD9 zGKsHT5Bh5XjUvwl5;rL0UpomU9s;LC*Js!2V_3)JGFtGBgx*>3PpDHah0n50XygWx z2CQ5*JHC-yX4A-R6u^m!ips>m4qOaFjdbmFHLe$QF|N;QD$}~V650En6)xL@F-(rp z@w|sBaKg6MomWlxn5Od#a?jKW7S*foBRVL2Ec-V1q)SyMnE1uyfe|fvig#M7rxnpK ziO%%i-9PHdl0hOV^u(R=YkPQ7KCY@AebK>A$NuIKU&5Y+lF%mMQYUHu`y#sJ`CKSj z>hQjU``&cX8xw+;EI?e|bWY8TEs-Bb)ms!&-B6yVlWwc|Dr1+mOI&<>e55D-7Ts^z zw*eV0ffak`GPbp~0SNKuYP-XcjD>uyYhKNyCi3TUnws4G{mh-vzCKyWpov(&jav#w zC#OvtP>(hnfy$Z&h;rLvOy*6E>15*!%q_MH_kDo_dQJhADdpvg- zdfv@v^iofB5|J{XEw1*()9vcUf7QGF!a?EPI9=n{ZjW5(XSb%_k4><$E0UkN!sk+C?i5}CE;%p*+MJ{n#1>mb^i5^D>1biK+aFF?wN2U7 zeNem|!B_JrMPGh9oT?ofWoWRS_V$xPZ42T04^>=p%~%O87U4@`v-D_7e}A*3Qoy-x zVV5gmXwrBUaMg__)EvIR`jJQu6*PLj=|;_DquIOTgU&c)EGhiOkvx2oq2|)Klnoug zTD;gfBsFTpBqQ^N_*RBTQY-2SHFm*KqQ_!H!Px1QBpzR-{RZ7ZS%-TVhSHKsMjfbm zMJYj7UQ`hADd1BeBjQH_cs!>CuC}Xgz30tS-_exXGP^+(ZxeHIhY}qRjm|68u`iz*^8Z#!nmW_ln=54$SAuPbY|O1GJX88p=2luG+dJ`$jQCS0 z`|~JwKRe_{p=D2@5BNpWq>jlB%gKv&r7R!OetQ3spF<7XSUO^Bj)3`v@0@;TZd*fn z{ephJNM87rolgs_{<9sy`*Ugy5uqM!FK&L^Ig#lt_Af}d9SvzvM4ca3j;wCacho<= zPEJoJMs~SIrlyd-!jzlB-ydmsgz4&R{yZ$MdCT8FaGEv4fabW(uF!();XjL%Df32I zh;og*IC-WAUVv>OY|}=U_6;343Ek423yvdmDLu;l$4 zb9Vx+ZjzAgAfDnwE6q>rvSRoCJ#bFoOH~yMEdjCvij-V>w$6TE->a}eku6Y>=*?0v zHX9k8CDebvQ1J6*kSQLI+;HDOK^qx!0}8WT^v1(sxP;A(`mex3iK%t01 z^HE_Y6RWVyz=GDVaOP=ftgC&Rx=7UUL+f_}FxXL@65qr-35`~Xzq%45d(7%O3thgC zx1Hdczq&oDQWWa=*%e*jq|9g}Rk$S0Q^*pAcI5DCZ>RoYmM^6KJAL*_kEY0gwNPDL zvM%nC1FaG?!%0W?lcrN;q5~-2@>Rm<>{843f+SAD$o>$4Ts_|TOW=pjC6ZT$rW!6M z{YlY1(A+^9a^1F7h5!p5^l(Fmr94^NpZKMk5~==k%IVq`D+ko=}9V>)?C4 zkd{kbx2I7lPq)uG!y)&Oef!Y#RDD+eE2)sa$zz?6bri980V!D7y}ad#e^(p6aEtV0 zWzTy$q%An~p)*P@W?Z_oB^D3_$35STPpq!eZqi(fE3x0^ci$wMN=i%$cm3m%ysxG| zuhUwnE!iR}+70~Ukd|Zlm zP=^#<+%my#_Lr>WiN)Ua%7#SKVY%|2DX8CROQr!$$M&rq$yx21+~3`ZZwHOx)xj8w z%we$$gR-udLw0((P?@<6)IOnMP}#JJcMudV_)~7ZODF(cXqb{){3U(-h9Y(DvhwRf z+;fDFanc!9HhXf_0}wj~H8QEu<-DHc{ULV5o3O;gK6O*>$j?V%=e*S{c@BPYU7s*i z!R>^H72m#R4l#$!bLI`hf(Y;L8sL&`X(Q=JjadFlK7LWb$<_Mr8q>U|9C!9VDorzg zyd2!sGkT|Br$$$jYpwt0<+-?~l7wQ|jSV$A{H$48G0V-sH;>n-Qb;ucRt;0M#DuugX>G9p>@|9Dlu$dx0z zQJ7b`I(Tma&~GtUKH-2*VOpCoOdpzYEvp!SMYT_bRA0R(MfW^5qQ6oh>7wB)x|_!m z8+-c_n5^?eJfAvx8IWDgSwnz&XJ+_iMSFV!$E4Y4*KYsnojlh?1)2ASlA)CdZ^$`x z<;vO{{cH5Ifcj4nhu(wm^yxM8E4IpvfePqVqIN^ z9Qn3ipfYWfEl|(V(0dw~U9M??YY{4@i_=`yLZPVt@{EI03Ec}=pQrQ|3a*kjOty&X zgO`z{D_i8#%fD|48fpVxH@lVyl9qq}{_*_W%Wg7X#m$WyN!%9{XhA4To0`a~1;5QX zXntaHkvsaF_!E;?e6h8jeWS81>!v|!s3sV973?4ueqHjMB)EsR8ThGBHO3Lwh!)ql z5Ool&qe`qbprWD0>YiI}@0a8k%B#qS5Sioi@Hbsl$@4UEhUwt&)-*4Y;Rt{2bfg|1d|HGl|>Lk6)WJ6Su#rZppND%j}0`vYr3Qf$*UU3)d}d zLYI-cFSuol{O z*6tmUx#mVJeed3d=dY#(k2wutJ_uyw*Y)Zc@?;kK=3Yw>w7|PqU}H@vH_(5AAvPDQ z=u3Kcpc~d0`T77w*9Ce|dGYN`sI4orNT)&ARpi8j@L001h$^RTSaW2uz@638u!DPX}(DMz+b*e~bUeeOuS$dhC=;df+XY04DY^Mfe(R(!V=$1LU=m&p} z5o!m!iwz%HpCk$A7SD}b(R@;aMtmi3fjQ#z_=uL4lC{cYvAsGhGjVUW#B5;mUHQ~U zthzTo$WWPDU>3+P8i*JiPj#^;0!ao^*{#Ur64Lg;yCPIxjZUJw4$!^ZnqHZfcGZ04 zyVE3>1)+~Usst23ToR?eH$Rza=RV+YXO5;gKQU#-S=^R#ld!I0O#D*lQn zEjOBfQA5f8g-*}UxrslUn6Pp9;OD=aBd6H^RtW1iSEclR=s=P%J3TzD0Al2YnQRePE7syuu599pod82l&+ z=$8i%M$PH}Mss&A) zT`(_wGun52DN{6TN?-6pk}4{2n2DMAo`ZCKB@!#184#|iB;2<&NU#D1!d#XSB?z#I z`fb=1;X|9>Ec9M{QUu)rr|kUe9$h09=CBM$^n7l39G+abZrGc!-2ul&ACE&wN^>tn zn_8RJ76thl4%4boNc~6wbW_Ice^~$l=#IDA;=D&#CD;}OtWXEj4);O;NSbmHsk~n7 zO;xnE!aaZe`gOS6l!z=40R7U=6C#6OW~iH!{rrlgdNZL+i*)zc>vz1$1KL?HGCY&= zb(3a|2U=FO9+uwX47ncfkF(g&6vBy`kJ8e!5VKz>PG{#S_Ei9po1j6Feh3a0nqM%u z)s&kpLY9{sm_(91Ffy*n+QN`n&2z~7QY|fGchKvti@&FoVbFW3=)tA;bjL5s{JsHv zfAZV^)UY8}XIxy7Tcs*#Y=m_k6HPQlnm!>^P2Hf8|Kv*T1B^V5Sptw7Lh!y0N>B&a z;IzB=Jw*+FtJgDY0Hvaf&xjL)zsZLjIk>FlB+sL2V}fi?H9P0_(xA&~;yt`RjRnmb zE#3oDZ>#JkbJ5mD3Gn%ZE<5U-ijBsasJ1`Pw!#6?%xdskAQ-QAM7`iT+ibLed9U& zI9L&fD=h`upn*WfbmdyNBmP!S3PV$WOz++CV#`ow&qp1Y8Rwiid+)Wbb*;6o zy+t_n!IJdBbo&6ce?VJQMs}+$vH^AQt+jr#&gz>Cp*((?8%A8zHmMih1Uu%}sq*Sy zoe;>>pVrCKcl_cuUaVzY?lHTTr#c%qW;Lz9Lnip*Ix>f^aWpIte8JGhMp<^WI)4yN z9nNqKP_*~%jT~w?p*v|j+INqwne((iguEq9$wT&nTYERe8lt-6iZIpTz}d5!S#Q?y zaSZ;>{>Iy}$L~~xY;Fujn8#Z7{9+Lw8uPvCVaC-O&NDfb+gmh??gw22n}5P6sA72E z*_=kVQ5eWtX`u)6ZRz9f=bMZpO|;dGS??`V zl2ellkY2x!Y~*Ytd1e$NBVtNVQepmF4t*nl`uRe}sC!J4i$Psg`Zq=1m%(j)(37lb z-fdG=IrFS%B)#0(yLPpg+Gm(dvf`>+)7~0Xe6$7Z-wD_%?2eSm? zx3p^R?y~ZhkmVsODP7{OK;a+Zo=Ip5nWwdV56ha`vT>H&zB*W%L21M&7TjxV{Rjj+ zmLHerLnta|B~`R>i?H|4VQf;S*A4^^ymWN*^A)~dUp?acV`ml<*6(nCG+Q;bB!E^V zZ&vCxW9aw3d&TI4Q#>T=qZW(ina{VdMM;K9iG^wvKyd!>NzAD}Je1TQk4a%RvO8<3c$(0so<%tc5dN2Uq4M z1_-JPjufx{rYvm0<-2SVHwGrAGXzA{pvXa~V8WIR=aDrRN2>W$J&)A$v_O%3MQ{Eg zWDm(jO6`0hJLMM;vVO@292RdxuwC;Xx|Wj`dGb7AM6*O~^3LKe9qHts2ug}5$e_T1 zCt!U>(AZ=z!k{LW`p3c>qQA;NGjiuJLQ!M;xA>(f+mw$Ga9cNe#qx7!3sg+_5K@U` zFb0`IhJk*&a%Zm-t28LnT@~l3%!o%=6*R8t*j9gz3)YHqF16zk5a^$-HdInmvoc~N zp`(tsq`&|vHqaOUL!eBSwOin-R3ucoMn>%ko&_C*j0zo^U7&8!asE2FBP&5Mv9)DV zh{%Qav$ywauj8&pDCz{HsL|SpovM!upMBV3$+lry)aMk#9zmD!-fJZnACC}Y-_zJu zVfmHe_WV@8SzcbjjV53IM4R6Bv+RW2`#3{uTt7Ne-%pJt^n7>w7;7=R-}&drtlnFNRa!}+Ee;P zT0|0mSwz7nLeeGOuoU^RFD2>bbA^z2%GDhG_U2@cbhXW51BzM`l5C22&fuybmzD>n z{k5doRSgX~W5O@!>TSm985nZPa=oK=cNtX48q2LKtb-`Ry0V*H2L{Kid&1Jrw4r%g zasIvI-@Z`*)d3ZgnlvF->w5ch9?*0S&;=0YAmF2UcKK&#`#p6ej~EW2B08 zDO8;@Q$8oFeET+?{{=H8WybO8>D0tGMHv}XF|kkCb#>3huQhJFtuSw6$FCRe^%u6u zv8MT436Gh=rIqPrd)mqEHkBrR?c+Q~g;;vEjih@JsBpHtajjRQZryre4;X%LZ}bX| zVA4Wm_@e95wngVIjoaJ2|2guVO(O?g)p0KY>;*J%1 zT-SJQIE6^`Cm+3AEv$oaiXEb12DQa5qLnokK*-k4HK+8x@z?SY%?5((&`B7`9)lS&!zb z$++TLl`z2uDL(!@PYGJ4S^1il-b}(xqkZb4q?Jbj%xmza2uv6U5OWOK}GaLbj=## z<4wqVu76;Xm93f|PnRjphQ*%=+7HW4G1In^$eJ3$ob1gBm^Ek2P%a22NLs)pS0<*= zJm!((@NIAOslY-cbF|V&KXa*(&N8SH7mUSCU44_X(GOoZIsXQT_uCUw2_}-r{KmET z)mN<<{BJWezNVy?_Q6Uzo%tNCDKMVC0QqdY7b2=JUp`z^4ACy-FA>jr8%deeaYdTYWE2FlRe%-_x^3L#f-UBqa1vJ{1bc?0RzWk0G!-;k%r2L% zD-!xEE;djpTUT3;U*si00!gmvRC91(ApKrOhOiywzPnO<{N%%B3{On z=h&vK5Z2WVvR5)P{EA&VYam7Vdp-z+Dx|y-~HnqLYtOER&Gle^(Kv(9JQa8Savb`7Lsc3y!L$K7wt35rDVh|f=XHGu#? zU?;_gmq);+qo&>A`EMUUfDb@y2-jmDJ3G&9qM`z{r$_hiv)Wahi88D6$U`aU9bAn# z9C7Sesh7HlrJb2$hF6!0Xqt0dTD&(mw}{(n@P&{0IM28!&z?UNjMqV#ZiO4ycch&&qW{I1lXS5TOFCB|{M$zGM- zRl%i}l=3L;s6xXp=y(J#83iKA3dadWeOi=)Of`}DP~?@jYA>`Jti8-) zhU45c?_n1q_id2{e@{C9#dO>GDEb|8H5VT&pu(Qy8XxUcLbA$+kSca*#cp&K+9Rht z)@S5$pv)mAfm<^ZT0j<7;MDxf-t&NI_iEJ`SW93S@IKC}IXDn-q~QWKT)Y}N6tpGd z6B0+tW2g4w<3KBf#u_lIsN?$0EVumgQEg%=Iez{C9j zP}zbd3HF0dM1(MJ93eaKN*A6jkS5C$rWN&3I{<^MW->tf1?Ee}-+0^f7V(8-EbM}U z|h_^-tyoB=vZ!8@H5Ho+QXYK))X{=YHReXB)>fJAel}<2LLHaYN1v^IUtid*Ec@dmfsJ{WlKE?5 z>{}}b&TKI}ew=d(+S<>2#Fq|W-~`UftaBPq?UptOB6VUP*)CvHs1~zom%hnM3!bJb z!IKQ0&X=@AqAKB6nk}d}M@j(3q$D8uS>j+VROo=hHtutPH1SdKq$XeV=NM_ed_F+S zeol}gDa!f%EcrBQCave#uB3?_)ygSZ8t4^ej>V~Cgogp*_Qw=e@{;Ph7OUq7Uc(3E zzQ`$C+UJ+fiu(vf6^))c_A3UzIkhL6_fppVl1r=Oc_yYMOHCBxOdf7Sm6RhSB;tl(M&;R>n zXdk!oK|9v@Yhb@DKYv9-Tc6v__RLHZQi9x< zF$Txr_y|ly`0S26800aiazdiPz{>Zxg9CX{{`olxb22~)j}$jPAtKaio6!a(Z8m)M zPeNjbhRKGiOY~bOueM{zH)*{djE1BYx59s|KE1f2+8st zspz@B3_5|v`%jN$`!7EKSad7;>vUsiPk|-+Uw73T#0~1KpVJR! zx3#!?aD0zq+4Day<3F!?#x`2?;ruJz`@dgrldm>zeKMXUNcewzpsca+!_?WG(xxVg z|Law4ZBmT8>;3x`&YAt=DRAlq{Qp1R?2O?8^1#lZBgs=#ohPGr>jX4CJC-!|e|QPZ z<6i?FH@nlnb+Y8r4BY)cukI?|!a0oc!Gofjns`ZC&=KCyz{kVO3lRL>EJ;#jzIzQM zCxDt+Sa>)#S`eU5Kqbb%2y)Pos$yWk%4@BYn3@`9mkBNT`*#Pkh;noZuduNc5Q>it zKXAcdikh0l(f1$IKYD}<{2dHZ3eufxYt~c`vc9$Zid0v>l%+R6Jn}vBQ8MCC)D`yPQ32(*)8n+4nO$Vf5c)Sju z0Y<;oW6v4{BW;I6k3c{YZpAt@d?a*xRY7*oD}$RMXWKr!fiGB*QOd(3AT^cvsiBD1 zLf|Y=$vU}HMf&;{l$Trc|6_2u3i^MXodjd*AJ*!ss=2#*Q(3Lo_^H^RsIfnC<)ypA zY&x2Gt$$Uqm62n8zDxOOo0yDUogaZ(@51WpSWq)o@9;~Q0AkB5A(8a;?YQ|^ zcEw26dP$p6LS|-fUE!SxoTfqm-KO5(840&h02%)X2`h8E) z|Kr}5p|Z#?yLud|zM!M&IH1DOJ~P8UJvgjytvH)-DJ^X|_w8vrtC3h4te{!Nq{-;P z)JqTMtYQ$|oBHeWgB0{TUNQ8|7!7aZ1)oOuQG(zNjI+G6GdIwf+n)QBb-CM`7W=@I zSwtlEMa`#ltKsMRP4A;(VjS{`v5Ejl0A5SkPO_je60yu6GOMDiEAUp2{9wNa)AM4Z zta&vR^{3~uTGi9WOwdjTXw<(i6Q2b;`Wq8(@5dwrWf_ns*By@aI9->$OL&QjjG$

o&2A?+rE8ur>#A7HJfBEpe+ra`4JZ2qByPLCK2}ScQ4CJzW_hd{t(yK1y3iaR`8k-j#`;&k~l>hE+UOi>n%D^1?HzM|B$fx)d z_iI%{zkB4PkNe;Ox3nFvUxnaGXvy5?YP;pmrOEC2`lxi#I{6o&xXe%}O+lt!EcINM zBh6e77rgdLczWc~g1Z{$&wgArg5|ZEbG~7=Ng-Vfn>WCWXlZK;x&0>cm^)VyaGdKc zNqe`jmL<`1zMiM@G?P#fi91;th2W`3K(H3o1Q?r!S4f!E{0sSw%$%IDk7}~V$U9&u zF);+YEn5V;&FdxvG~zMS|HEPa%hWh!Mx8xqT5j!F%GZUW?GW$X_|jLh?|)gD^pWBDyDnVTD{#U)Kb?PxoO)JwT9b7CHd zHRS>|h^@A7c^cpftXgtud_5jsL+0Vb)-6>ofal0a6&xzMy`iz+iQsWKE&8~xm^9e1 z`Mk1JgXw2oAcJPy#@qpshu8)CC-XSY>EVl6{{b0H{;yxZ>gZ|MXho&zF>U-fXZ+}Y zx@Au>1QOqwiP<`2HA;W#HgXNW`-K#ReZPoqZMrUtpSq7B{j#dngW(sp$2R){*5>1WOPM-Bm&EE=W@P`WN-nT)j=Kt`LG#+8+=(yf~ zxw|~blD86s`@HF(e+#gGa$9ImU6UiVnDgO7QZM2rZL2kvQK_o=^FRNPV#T-ZlwwtT z^$I7=Vmm%Ch_L;I|9*%P`#YpSLYRl+6ym)fNLW zud<=8o1s3(b@#|x$h>pAhW!sn9$M;YzUi9PF+Di-`7WUhP3&YHEPB2!_9Ek%zVu* z)5rr+ewyX5VZEhBvJMsx@wa1CMK=>zZQw>^wXOHy_=X%JIJXe)N8x5CWUK`Gz&lKwYiU#nbF(FByNu>$IchSS@~dg z_sfJ3&R7`1e0R^}A{051!uX76jLHNNUTGsIZdnmiih52wUOi$uHJIMzwXvMQh` zgNF?^J#$IU4G6buZf?4#05lunkQ7xet}lJ>TCQcKFKa!&7FW~{HuCk{P*-g~pZw&Y z9<|Z8tQDhO8XDRE8KRIKTvN4LYx8mPWmZ!TW3^aj%2QN3_Kub3o75Ng@;)75@z%7& z5k5BP7E-ZEunGhTl1kUHJ&9B6vHKe znzk2K0+B?eD&c$j0zAA&#;ksrT*r+UJTd}FWj_FlO#|(|GNMPS4<&BB1Rp;9<+DUe zQ&d++0@#b3=}b`WS|Ae+hI=HP`>q9@eOUVMKJzNFq%18F#wSmpr&jDj$KhF5eS9B1WSXT|+agJ)|?uO6(YHJ|sW`iK*XbF#cVEPTXz)4J~I zw#v@YWQLKHL(KC*w7jwRSv-Fz8DUxP0$)jE>Bo<0 znADyDhK`?7YHOt&n#|EnuQ%mQl2?9l$wXf-sqiFg9D1U zSBf+%=7jCe0!%4!kmA4-@*ep?b&_ok8gL?9YQf3LZa6qD02h&nA0|-xtUkKOZn$Ri zt7YD4#@J&DG&)ab&Y+`6?Y8cTxBqFZIf%bKb8o*nsX6#v6c0pd)a+=D zquY2yQahO7pq~8-!p_BkxW=kIJG#LG*>7GmdS~5f zehQp5E~{!Rj}Oo=!x2!9A-WoSjy?lFYKcPVu*RaX)%dbynwRHNIN=mEuqfXKuL-AJ zclqD4*1?$Wmto&ogG3N^&?n@B-~kkJWrx%Fg;cR~`roX-HTWcq`SEo`V4!(sRyZXj zhD(^L#~E^X_;Vy9Nyg7lm;=x=zo4r~0#8L?acchl0X^Xg6c6eR>KF;rhJj0w2Y#jD z)x-~=cLVM0+&ba*aG(>MqhO#A9r@8n_$DiBR88<2q_Ko#i!`adR?MkA7t|-GW)0qV znjzxkg+*qaTE&dqKFa}~=?G=+XOn!5LvDvKa6_ZeFqF?bqbl?m0julMVWFy@y# z=Iwq7%K%4)tkvLZa*GkPC*Nt&-RCk($Y&ifzsl)I$m=j9muJ;HE3gh~2Oyq+}-T%?6K`NK(~ceY&GH&sdg73c9X=ep`Ra;IY@eXZme927@*ow+8M0 z!)fO#ou8IC&o&Y3!~=&&cFDsBC@v!~q=d1Z$bZj+%V_-j@Vk-^XKtIy5Ti={ z23yu(_j@s%aW1k=$&{|8aB8wcQ8tzlc?%Ah`I`TYy14&v>)&&}(bvEPFnORw&rTYQrr-s1ENqX_ybCt9G@JHAf zg!`SSH3yxjvl~0Hyv`OO8NAVUw$Q?Yg6vw$;YfL0Cu4G5B27t6{)Wbr!!6Uj?xVVF zKW}{PvQ%O$SO5wJF*;r)hAt}93@Nx0xegD{?cjJyq-mp+V{)U}$p5}5 zSV2F+gtW5(g&@g*>&s=)gAgv?3#+RmiCbKEcX$1N9&Lnn%!6H@vhrVvyKCC|pUn82 z=HeS&g!-F*{)S|X`81LRmr|r4#(s}u_iS$T{eJz`1`v(Ky6^9q>vO0iI#=d1Uj{-+ z{uk5l&3iCDIQ&x9(<3)FHa`CIha?`6A>0_L;eWxy(cYBM-EE!g<0AO+*B_D>GatCT zT#_HvK*lp{o@nXnDmgo2$1NkHK=T7ol!bFvKPzN`+Tf&kpW=j#DU}0JOkH$5`hR_Y zRA{BR84))uPD9{ra!T_^?qkujg%?-bHWD@`oBX2J$GnV5BF(!kTouV$n#60}xMCn? zeby7!2-J($_G^YbS)h{(eL(FmU)~v}%cAH#-p+dP;c7sVtcXzP@I2}EKS5zem z4%BSm*`)oiAEV*oI{j4!t4;5~h~v>mRmO9R=SKuoO7$xAB8l(p1Xme-9~??Cwx6F< zGc`?o$FZ8AKwVf!-yMQ2u9AlFduKFJ5e>E+L&C;3Ig&9o{Gz7u5R_akU5Y0tqFLhY z=|IT}L8HAsdsv_0=Eg=rSy?z}Y;9d^;Q++;(2%k$8t&hYXbBwkrjY3Nrfk~~nlj_Saq}f2JaQuJ7X2%!BTpKjoeLzKkFJi!?_>zx+l)I*ZqRLR zy%k!|W&&bRp>=g^pav2k{wEXPV5u6u&nZ>X#!9Q~Y1-$^#Y)&)}mm?rCkrtBx3j8Xee^YW;81#Ep&+!HweS)~d zr;{htzQ>xP9*t^?o#jn_uq;(HygojpVWl)*7EY5tgP^<0Y7-JbP$6U`x< z_O`}ET37c`+?1SXZgF*WRwh+=B&j81(!@7?^}E}JN)a_Y|>ZD}Dp#yDRmgVK?fm>{sr;aRS zBpj_(+4bWbiwj-sq|8k*G&Ch21f;osqj2czN-3+DFZciA6BOiCMdBnYZ7R$R>b3w| zWAHqWI6dL{_DkKsfO64^BKl*pa>=(gFJ29a>pPQa9=8nFeP?Yrl++`P2>9vK_{=}?aehK2{!2L?de{`!hLtftN!VjG=v_#oIB>(75n8Y2g_BD zn~X&L7`v&`MK9|6|2$~Ss96HIG3HMJI&*vVO6$p!J8n%(OsvZAV;s4OXCTnjaRIa^ zTt_E_JM8Ah?+&p#wC=Fem-^*X~;e{Vf7>x)8wkX;0}o{jEv zKAczD@;$=KZ!+wh**-6LhS;H0z_&?e*zl zFDYn^r2v~qXuu%Sx}ZnK{A;D|)e4IQ9feNn<4X#aG!vkL1hz*)!g&V72dWH?=sV8( zdPcGYlg52_lT@?y!v{F>QMsP+*cE=}D}aXt1xXuy41hy}x4Z)qmoLRdUL9*|C0Z}_ zQV>agRnSG2({+Y)oz}a#9RpMR^2-dl%XcdkP50*WHA70lF)mkY?0Oq=O;Z}l5Qd8j zEzzS9y38z6PZJCT@qJDXIttaqvT~P@N)!xTiPNQSj5O7 zMQ8|35?^Dty;&tWE2icC*q$R%_Ba(?yO%FPH*YY6m=57Aro8olwW7S&{UMih91>0h zYC8MlEEC4xl2gS>AwA~o|2)JOMCXb?vK`b(c^&kT+u7R}1McZA9S-8C_b+Fkx3n4t zN_%^^D3(k<<1sMtTtvS!-58j!&o}k{BKp*Hby?$zB&ifFR55|+x9E#XIcpy$Z{KGb z$rg%*m!p2QrVWl*x@5KBW+Gu_CS&V#7lYhQD@|P;Uty7O#S#pqk}x=aQDp=L0A>MQ zr@oy|b6)7G>2N8T9{WGLJ8Oq`*1jqtl5`mQD0-oVLx^7d@(&Xw)uF+A+U|=J@z_{7 zBX`}HXLYFKu7lb-_+|r|x-Ld~<>gLT7@-L$1zh)H)k5esB|RA=-X43LPR`B^C382% z&`3~^AH9ch%%1|maG=QbR^*M^rfnud35Rv~A)#>|m zdbKG47zPn`iwFur>5rl8Sy-TzIO`g1_~5WmbMu_};~b#9QaPrpU#HfO35UzlkHz~P zGkkEI@9((TG6jUTT(5<>Y+Wbqd%|y{3uX`E#Xba98v?<)N{#oP2lTQn0mcz-iD`1~ z5s4)6OG_k1f)Nq4OGr_b$Yfjr=JV~y7Z(>5Zy(9_BKv@q9Th=>g>`Zw_hkl?m30Fc z>AXZ`{C%2R?o_8H3SFs`9x`2~q`@VaxSE=#4n9A>!F`SxkgF*zpi}koLed2eA6j{p zTKF7^iNx8-SUxfYx13rq#SDrArN#1!N>s8D`Z6~-QQ$x|u0L>guzVm&7;UMg3wRMQ z3z;%hc~oVwLZo#x(xv&H1eS9pB7LvYaMU9VC)l+yQey(Llb}cYpL3A?uSp21I&g4t zDJA?w!TWytT|V6Yb;;v@QASF^%Pg&-ksTLD_UV%|2F6A6(U^#x!`vuB!jGHW$Vt4f z&s=s%^1=IiU;TU}`aZnex!Tlw4UhyKxv399ToTe#_%DsP2)Zn_#YMc|GkX)3D( z^9cys&xrySI-;e8Xs}p=mR6RY;|@?pRTfqPQ0>g^naedo3~E+1#jV**Xz|!qpdC!o zJaZ~~!wx9O@jre%1*tXqpgwL&%Ww5Igd!fhj7fuUiYj4CL^ulNe^cTqZQY`bl{v*1%g{bqp z_BQc3kYiSd4A)tyRFp$|oxKCC^C`c=7N%yi7AngAEM10Xd(2s|u_*^qn-S7GcPe~k z$x>TtAy{-mUI3`cDILtU2nt!e&*c-J6G}e{#yzKU0(T3U4ESmgClE!Vr3*Kv7Z#%E zpu=U!bkWIlu`rC;*n&KBwe)`z5rjv|8VQTk_gWC7q)-c0(iWDpRVv3-TEuiwpeN5e z8!K=1!}YZyma(3b5rsvfMg}^xWO~5KVzu=25EBzb?R!VzLr0>(ZM@q$l<%iV9Apd* z4h9h+@XM7$A_=3`UO+b;sCV?A`nt%X$wu@mLn|tKanUvNdXhzv1MLO60v~glU*TC< zSy5QtDXKiv(gmLefFr`$lR=j$YZS<>9{AD_F9vf^;DI(wAUyDo!%sqVj)AzK9N1zh z-9hVmS$Z?oC_U4c3{=%Ci7MjquU|g^%l`4RdOMJyDpJkhp`KAgG7P+LhDFTPZ!_Oc zP3_|Z#wV}4qHK(gjHi=&fws~>lAAfYk_0d8pp6L3At_m8RN$WDmbFc3%xo1mwque8 z@j%NmlH6A$jQv5))d({#Rj%@grhvPth65F%k6OoWuSrh!Y(T_7WFWkucCh#W)Q znP2&rGYPX7P!V<5_w_>@p?n!l*MMrFu?6r%b$1qUwEWql&r+^)T~`P7h=CFN>XBH8 zqRK0AEC57go!RAqQD|&z)X~vdfD!y_Ke1r2Rl)?&^wH2DjFd&O#G~(?O&PWR-s1k~ z@^nu@cTiKeH}J5ZIqK01Vl*a&0b)rxC8p&D`yY}5DPS*`G~KsPh@<-`N-MMRK;TnX z#{&|!;0$EsIKe|<**g(=xUTwr;>GH6HkPDi7$}`mR#AzRFVcB( z7sbkf4bi&)bu7il9!wP{QsObds2AZxGsIyPRFi{rbzz#>k`Yc5GG|EIOiAjAV^s3a zC}grhSpNZNj(G@dF6$8r*$}YxT((fXeys)n+b27#wTPz!)_*0b-|9&W1Nd-0C1Kp( z9f}KOKMug}bfAr9i+~0PD}AuHTc4m?EQ}s2sj5OtP}dQpPsAYOD!=f|4PQqeAU@Ds z27Q+zR{4#Ko1i1RMs4{WE_kd6s$Qz6R!Ee zU2(63pZpBFdKc)atru>tKQT-D1k`sN!BEf-pb%R=9gbN}C{Cd!6%|#}cHJ=8eTV=EM~a?&(Y}MWK&THyiv1LEa5}Z4+J5>7Le)X!oVtm4k)A%7EHBNccRhKW7<2>o;7rFXM ztde_`Ss%wO2nY3H7jAo72$^0)XMO9#QcF8n&T1d~zC28k`uj)kNAV+KzxGf~Yrl)x zpU&3`tT;kGaI4FwO(zoK;~)2ajX_;+vcF2)Qhh}x|98w7{c(61EkeEs&#>+pHW>5K zT(KH${on8HrtT*WTFWj0K{lt2l6Br!_ITf7pYy(vlSM{P)?t&eu?Yed2rplv2TG!R zL&YoDL<$4X@?xxs@eR&SBnvy*GW!wDI}#5--IzXm^v6i)&A_C>zSh$qhQkh4*^hsM5Wyi!@N zRB<_4^1Rj|nnY6J$7Q6*8^4&nqJT=0xck7nK|UwCbSFRqCEt)rBZb-v$hbh&{q-Kq zzS=e$_vlao+YJKl7(_H6Xe_6XlPQME+x^nVQDMUhWk3s}gU^W|qt|Va$wb0@A7aAX zoM<+lk?6_=m_b#ix{t6FaEOTE>QPmBOdxwS@wKZx_0v!H8RNG7ZrrOWiEGmJ+}9j{ zU;1n6Kon9x#J=@>cEWt2-S*f)iuG#>(-(eK%fn=69WqadUS&Lao`1~O8X7aknFqrq zdFRkr;y8xBo0IXQl^-)3%TNTa%Vx=pi?>XAQ60o{6`R_iHMWb-QHy&Ak3-^;3`d3l zk!Wu){d*Hgz3n{+i)(mKjr<2`IOwLIB$xBt@qkm_ZhQAel8(HJzxW>D%(jOv>S*Ftd8LBuo@-TbePH zEu2yknTd>~va%ZpA)9zvJp#-S5za%}aWGYX%wAGb3~-rCb1%GMut1{GMFp54znrR* zD6$C+g(IAR2}F|lnNqy~H~<6&raXc6ovUK&0ZRc=dS~TW^U;EX*eHr1C1CbLN`bKM z(gEM=Q)^19B`m~AL~5$$Q^VLTuY3se3YHg`dDvy_h1Qd*G2Cq;(!!K1LK6y?xsTf} z&Vsi+7Y#0iMTm$i5o${^d6y6R)pPsyLZt<=N-XzYnRxE6+xx9)5;Qh0T#8plEw_-R zr1CCT;bt`Kvs2mR$XC_j5HEfr3Q=r`fWxHY-Z0<|TWh<0+-1t3kk!Q|m#whN9P9PM zwG%$UQ{F;o!*!Iov^pb`m6#(>PbWScxqqlJ?km-ep8$y{ZVccni6ZZ7ir5J!sT+?p zHRA%=cgZF$c`g$4WKt%vWZJpe(ZGi_wJ%S!+uApYNl>q8r8Si()2>YCiyI1Yq+kX- zV4a|Tauaa0oZ^kId$}{!_Q9D!@TU_hX!!vlYrGj(#2wk%$9)F*c?OfXx#6^>TzKh> zLL7y|K$bQQHx@}9Zg&#X<8WYPZ!>5~JJI;rbC~yOKrwLN!rJ6olZpYNkmvG0< z__jGAw~MbwnR%q>Dp@7{$-R{(tsGJB%h;>4_VsU{e=5(zsa8i-p9k#5A-9P9sZX%! z!q*hD79}kkTaMB&MOfkX6T85;-QJ9LG>7lapAi`=S@9LDWRvlhshMdRP((|>?Os)T z2hhxa2znx)*-3u8wP&DD{9mx~>klP%T?kv_6_eLFlb$TtG?#ecOxU5=&Zx;EkVv)Y@ z)83x3UUP#p<8RIyTT9!0snWJ8)P7>>!F}4#$<4gvRCg@sxGzzm_xfP;SLXYau%>YQ zanYVv@|%_hlTM;b-pNk{C%!zRu(C(b{V#ys3qfEr(`P&PQ{oZypN~3HuQsm}0KSIK zXWsJW>N^>ES%84-UjXvT-IGNF*|YvW(zT9h$URo zQ{hX2gApkhZ|w$qzffO8lM?T%XE;$�cc%&ml=NQlWK~RO-!VmfL;@1|+_plAnC7 zI~#goaA;#nOTvZ{!T>(y5Dk3Liy_5s#UZ$;`b@jAtw2u|F2{N|=gmWq)>~Faig60* zc>twtBS(QQD7`t{d^*ki2n1ni#NQ-Eik|25s z3?MDF-u47Qqm&_Hi`RVLM znpj+!V9d(%k7(87FqFxEo*TFHA%ZDpTm@p^W zE*1$P0$sqTpkmx5M(zj}K$L#Xe-7~P{xy4hUe}RNCurx@)Dap%8sY*`5V59946-E| zF_Q*mX(gVvuB4_FnQopey%fOPCtKQXZWJ!P&R7b#6&bL53ZgnvFlEY8Cw3(A~h_=E>uaeu&tlHNmr4V|#eC0UW6~JP71tEc%6zL}kD(rW#!`ZDy4&2qVJ@wBtw)lSQrg=rjh2(X7(xv6SNyivnn&3s^c| zo@h?-g?Z6qic$dzQSJ{Qq-tjtHUF(#zMo3vjh^7 zXu%jnl3_trp@Wv2?J1hI+vbJ~B2(X{`UyjvKdoBtKtp^!LMLDwHLMMhYnmi_dh*&v}C$kU` ztS;;*oY9N%yjCIekKkTLw(f zU-n@y=0UOFSTlR;rW^fA1yx-IY@I!`U+{Kq-e9&UCiz=q*vwOH)!0zaTW&Xd!@ge< zrUMN%CQ6pI!-G8EuE=hWdcre~M&DN8VeQ4p*pKjWG(D$M=p`hYuaCO-)`Rycx{ZnR zz4f%CTs*#-p+_ejmZS}b@h6vh#N*?y_&8e6(A9&Wgy+f9H zGw4WI^AaF43}gh$&r<_;pG-^Kyx2897j2zg4#9bz?z~2lYT`|IgwP{74iVIzlyROE zXTUPyl7;>@=fM_!)zQsBIEP?(EAZLIXlRqAZvW?S(?t|-ITULVXD=5}QD}B~nr&w3 z@Cg~10Kb9Ofys|k`^C`pJe$?9YPZHl?qqaMcy9Xcph~R~#FM9McQ4o;2q3aA5Cl&LE7kSd*^dQF}o=xH5A5XMx+{^Xe6s($W zNsHTM6#ASwsG7x{(`S=AT1dGJWs6x5>aHyxOjKqbq>{%DDGD)S%7NscoHYKtj?O#y z@hAAWly;gcJ1wDCj^#K{+eG008GK0HiDKATgOG9x5s;2BtP2AGF{N zU6N7}$jQ?w=-#1bxX(P00+FKjF3%ZELRHP7Vz}?>Y#c6%o@`#JoLe|V4D!xg{ci^G zKF`l1hB;52_dY58**yUmhm)w#M-`Px*D8V`0;h;IDkw{iLTNzz5J2U7IuGHPi)T7r z<6xi}Yk$bo*Gk7+ob|8E5+fs=ot%)6LoBrh7=5)O0mc*LezoZqvCbZyCGo;0I9efxlNCJzUNn=3Ug^W%i7Il+$ z1uH**E0K*t_ewQ0vGS?)__jd1YfdvNjx?D;O+ zM(tO(*CFT8etqqeTB3^~w!Y_dxqf1Oik}aX(la;LR#AUZtYS&z=G^EczGZJxBFnzLRzDK<@)se4BU9oNjJ&71bm zqfug}5%_r~ov{z_6m;v3hi|)0O}P_ncCR@7JWr<@%3#m8n5V@KkUW0f(6wJ^_#;+v zS%~nl_T*b?QnGIT7^O<|LCJk~tZ&v`X$m`+@$qDgUmF~q`Ccug8hSZ9qV#0F{C-99 zXt(*Hv$Sf<@2596H8T0IalP)tZOSF4T08Rh-)g;DU817cTp%Cr=%4XF6D=~j zaI`=*ispDddF;Fz5Jnw=RxGW)9@G9^C5{VRDYK> zv=BjFnVc&BZ9hNn*_Wct2egzu+-+mg<6XZb+K}7m`#GEA_O35&GM-TO{hOCJ-aiv_ zhDb5u79-Pu=s+g!F1vwFhQKC@PPt=Y|-zA4Y zYNJxw)ivnmhOe)W2v1sI3l+oZjxz=MOSoht!uQX1WRf}yD|G+&hLWZpIS|{lh*`LV z6yQ`s3MwO@^vby?VXDl73uZJm9dpAw9|`MRZ7}B7v54saA^jZ|Y1V`!7qzOobZ}X(d~`N{oTWN*Su_>X`%@p!m*=<~EOakOyBv^k zkAh7QOCBEv_&!aq8=I_6QSuD0D_BGcOq}3Xf98vi9Mba&30E9s9%}Rn13W+GWYlHo zd(61O_5iYQJT*KNFE6Wwnvm9!+lw?pJNXW*;1bJJNo^^D_&_G6Ftn~<69twm25v48 zTC>M@az%b%^nZj-gL&psbG>FEkJY2DE$+cuPEt_si2046SX0*kodq<1^C53|31del?Sqtr^0e}3?tnOV6RX&2aQSmWRj^Lbjs@;DQh zC@EUG$BOiM&Z=3Mr^m@2WsUpZg410vwDYVd zJop_t)nbCYF#Ti5)Vg-b$;NOtd%fv?6A$poSC2eS-8+tF0%8D(W$+m_q!*dG*$3Gu26Yc6vmWUm;x7<$f-nhr& z+Lc>a`Sms>e&`hj>wbN}y@4;z7pwWKokw6GSCMc$O5T54C(RfXZH$}FWDBE@^x7Ha z{$*53>e`^P6fVvM?Kwtw$i#KpNapr(MeCq?I5oja14USDO%NX=du!EzIS0V9_wnSs zxa5lqS_l$23m4&{AmqD#Xir6k0|kDkNeXK@lg05mo!Di+M3M5gSw|pEkrJAw;VvG; zz2xad>nlvq)ItY`GU$O;PZQqW28mqTP@XSjFHT!%(%}#6m*8`RYw-KBWRW0f6H0t1 zpxU;uoU2MgZ9Tq!e~Kj+PPJOuk=~Vf4B5tE*u4;k_o}XS=)jmgzdp$i&9+c#X@9ZebKG3ke3i;vy@Q05!ry zK{Z^c!by$UFpH7*0flQzGm2P-c`8djBxjXLg7bSGtJvUFDX+vMGbDP8N<07DR?#v> z6?zR_|L`D|w!&33SuE5Hx0=0qVaxgYfccytPc@$GZ}Gb=w$3F)B$<9ZU%B>4Hg@6Kn!CHEsV0ZVExYV*c&+B2U))az z?YB2>9WSmdY-BdMd@HXNS*6z#$nv*s4^Daop$#T_{`|Q@O?r|1_{p&TPeEG%de`~M zmphI12JfX5)ecH0&-j%4V#J&2neDd*hI2k6`yT9r)pWeCqdy+>-Fo+e0kN~&`Q~@8 zRea7T*o4m|C71BU({lW&KqEY+r=LDy$kHHklHk=-vuJLPmM_;H2V(azV@uT04(de& z|6aXPSYMq?v_0RoHCUbt*N7og6HhHb-_%O~8^GXQD+7Q&OZKTEM?iWiq5@^ipqrWN z{p6m)RWUO-E2Dm?!O=$7E+y8K63Zvo?e^?@TZJFz{UM`;22-<3&5Og1BBi5RZEoMw?Hawa2*nc2!`fGRiVb(YL^nI1&c}z53hg$!ywA&>;M<&>y_U-aRl5P%ow-3yNUE(BaNYYO!}3uQI$5=0#?z1V?G$TBzmd$K2)DQXU99D`s4U90 zJpFP3z5_NMf_4!e=x&;k;bE(<;es!?i5=zGH^?$W`W{b16lqEZO@NxcUM2GGx6XGV zp|W-U#ZzPnp~ywJ-pkr|%ZK`H!;5#%*bIL?S9jpbH-5D94G$in=3M=b_)vVC& z0%_EW2oU)s!Nd|s};hTnE?4nrFk2wei5z(vzSXuO29|ROPw+}Mj68+B7#{Kt*+ge?wSye z#a-ioCId#HEm`)iLU*>z>+8an^Jr9r)(Am_@39IL>>kZBk&W2>7r2~%d@#1NRk&g< zQoYtLKQo;jyP8}o3Z0j2bA1`UITRYBfjCV3WO!iRFbG${psCmYD zxNDIQj~;hUl=JgtjY^(8F5Vbjv6Z;V^tkMkx1Ak4r?jE5hFZL~xjUS^IGoY`<19BFt{hK2*zr+%b?6^(K{l<>cYAW4OI&Zd<4rhU7FB``*)}h$Gu(X^_ zSEH#DzTK4U0p=v%*d#`^?rMzk^t!*Q0QM z&AP$a82-BL;ED}>fXrj^bn>|B+~~OrUHC4YrRUwdyUkAvriyN`lrBEy*=KG?-(yl7 zyFTzRFf%6hSM3Ju+t2zmL0yt)yA|KaKA!a3!!W4vQ zOH+U&c7Iflw;0Na?UmgUw126v%lyQBM~gf@LCgEN~dAq zKS|7mo!_ONC*|N6D*x89;ojdwv?lf@}J}HE6qHqCS}tbz8~5ykFE6tSOXWV zbITQ3hzqUuokW`NjIyT3RetSDJZ$a%_NLg3umm9R3z6(zk^i}gRK`IQ>VVFa6U_@k zgt88(6wfBl(=Gfs3t+5pZ}A2Fub?s>74F{mgJ#p;ti$^#qN-iCxpsHan&)8?69YmU z?OPL~EG#zC%@?-_Y>Ll|6KrPqhI(U=KkI3Bf7X+@@x@6-_H9$sUNd!v1HnbIQZF^} zV3mhca*7c?4PJ&`jSV`wh?b5QGe<3mS>oj6iw$Mvu)4Y=`L<{?zL=fu$nvBSyY9~` zbEHUV15MXpQGq8zNc?M?IIJ>%E^QuBX}dv+$jC@g=2on~IT~bUW0RJKnno|(UGG}} z>34yKe4>MFlJ3`(lfPWA0~$drl;i)&%JXcO3XlAC^h1I5H8lmfVz5+j5R#S$(GnE` zVWI<#f1qPH6?_`=9mnHNlTNMJguMNB3K3t*!U-aPOj0Yt3;6YyOISceVZU|mv#uvP za3YYPg@KM;S!9_b(B+K({w)h&+}gdm$;uK1nIE=TnngLwlxPvlpE~c&qi<<&dtn>d z;ew4ZK#8H_E=BP8DF3{=(ieg9W)T&@dJzCI6|GExC21LG$<1%CP9NM8NZLIPu-9W0 z8iyuYL>-8U4(K#g*tWt((hkfS{N%F)prE)$74XY4(?{R<%?1HZmCYu;=tTz(kKdiQ zwZ)ZRT8=ePRjYrz)fB)GAM4 zn5m6h{mWGIiRMA|cz)gg@8Me+cBpZF=M~s=*F?mLTG}CxIbOYD<$L-gD!N{2Dc5Gt z!+Uwl3GLc;UKGi00$iS^OMN-*?` zJ0^bS`8XNXRvq|Ab2W7OatLE*>q+lHZyC8x)q0`_k_b%D?ShEH`}4=le3;2)>D%VS z-j|%uxh;%NR1_%NqNo{|u8n7g6xpA#m3{77p8Azlo!(Y|Ss|1kO4}*3aa7uNIzD+O zzxRiH0}^w*>ZHGUd*DHM{kXYRW2E6s^Un< zL?0~gYQk?_Q6J%gG2bDaGTcdAghoQFepYT>-_R*{bS!v$amOZ?N`ZaSO-izQ5_iLT z?n~G6Tln45dfOdE1=ob+yVl(l1A*~T{dh$gkE;N$=?rcqqAWjAe6Q;NG|COw`Uf*=WA~2Zx2JsMq84}?dr2IMs%O+eGilGo4TDEy%=b_ zOI5bzPDE4SsY=U|D5+R_KbEU<<*Co5JG1C;1VTKS3939T9n>_T@zIuZ z*Xmai;V#={kdT{t4pe3uSYm#@X)r3q-<72>FH1X05D#_rkj%2_F6QTT=H-Y(kZh!F z5Cd?H>>kaYRlbt&Uor`kqugB%0RiYU;UVF9_3xw>ga}5x-7DDobw%>USOjbsWMsiJ zFWq;zDGx7f5D~;;6FRoP*&9;hGQ9$_@l4kl8e&c(X)`lqAS1<@YI5T&8$hbEWicPk z&#A`+np4tzC57m25F$v+$RIz+pFZhSFaW|mb;j@!5cQcvOFU=@lep<8c$t~X#j%)7 z6H7Jv*>1mF_tVLdnno-hUo(ICI2_$e6s$N{1>AZOfn^%wm8n0sgi4mQrGr;3Bi+Ad ze~%UlM#sR#(-L|v)ETJt+W(gGnN<4*7-y=T>9yCOq^2z^C+D$K)yLJ#l{6(#_{}Bp zg|?|4VyM+GriRZ_s^c6tOEf@4R>_9J5BgHN?;~-3G}`jMH2@=-`X_XZm2GX=HA`85 zE$Lw@S;gca5?)8OZ6u2|GgAZ=O%6<^e+|}n-Wn|i3Va%o^u!PQUVl6jYHh2(EgER& zuL4{_UQtnKsaDe1m^$!5Mow;?$LM;J%U9mG#R3Mq699c8DmH8pApT9F%Xp}@fQHAwe+|I}E}fRMT1}d_ zd;Q%^LrX`806VOTE$Tl<;C!Y*;76SM_ia9@3ldCeK4gNoJ;e={dh8TSw!P-PTh{fP zM;jMIaRTzI(o_Y@+;e1bL2|GJotRP-c8jpgwxBP|wXt?~U^F1GyTHj00}3 z33~6vBa7)X2eXLKk{Zsl( zSZNWK$nQ?)7s%F}P01K3FAmcKi86g|H^SMbB+ETfD0)2xiS5@WFyNi=g4gOZ!ZUw& z{uZ?qhvmjgG^U@AB`<1+F3bC7S9f_$T}XRL@U;4SBS{IVA+Rz7#`7Zi6(Vp1Y;Q~xK=o9J<$KYLHJwORS z(tLJ0UU9f~#?K~fi_A(!mMUD6uGVfItxLhOHF>b)`Tv_iU1yj4Ivg87CNVKFt8MiO ztbZrsM?fB|izbiR1*x6}^viFb1;%0!wqU7@lF8bx6=Phuc+sp=;d7vRl>L!y^0x#98>1zDuc|vkQ zhWh(EsH*7ZO4%othUG{{78bpe)%*+61QwfVHu=p`$^QUw^=VyZCDB%ZOv{QBY%z+4{wy$d_ zL{YD54}CowaIJXFS1^LXqh6;(WSrywsvZfmLV|Q_k2*t1v+hVq#1*2AE1vcynpyzY zw4^3UZz2iI*3JYAac}QkdV2cvLXB{UsTR>}EpxU|O5V{+X(Sm0lRyWoe(B`g5doHU z0bCZgb^cPGry@$KgA^bU!60U%f?_)IX%~L_6b0~LU_sJ!f|$T`R%<9-PF;_mN{Jp) z?jYZdkLgyY*<^{_5%^k*2$unp!*tkz;F<2GVW1)D6WBkrv7!t1XmX#>E4Fet)OK#9EQl+DZYoA+5)c zA3HlcN8VU6{!a@4jBcc~G_X8{>CX*>Njfw%Oyg+pRSN&w&4u+B=>fSgXq1FQ{*z5* zy1V+%XzLuD8(%s79;%>PC7Mt6Hp@JZk2~B8%i35PfepZ)?5@x6-)4qTep#132xfub zA8otuTs{-2EX2r%CBN!=axXm{62(0*8;Hc}z7ZxmgpBiUz4f;VLw$rOg~7d|xiLkD z`h2Px?Q2?C4^&AY1Y_z3;my_b(T<|?c|qEMu_FS>N26G-^7_2^@fU&zQ*#xsQs2M1 z_ED|6PuS0+eslaHyiaj>2j%TlyKm8R=W)Tzpfp&@D=R>wczZY|mn?5?!N{ExoRDvN zPJqetaVWKKKwFXNOuw0;sw=a4{RwOsu7^K%E8;o2`{({I!%XOhf^jPzFzHc=xNkUq z7|!J9o;G4`L}vChO@8H%5Yi2?T!KB(AaN4jpGRqIa?wRE;Xe}IXzPUwzMgJ=nUi^W z9C_eLm9*v(GHA|S>?+#IKHJYCNB7crRYz>Lx9glx1Bwpp56lrm=F^uO_B~A;1tu?o z{YQiTd(&6r`woxn@k?2}&iBkF%fYZpDWJSb`c(c>*{YQ4H4tCZz}`G~n1Q^MBZrXK zQ)s{WEh54i@Hf_PWN~EqHpbacMM1h>>!F}%B&IYOczqc@TBCkz{TL}o;^q$5*AJ3C z8rQcX(T#|Nhf%4f-QHR2GnBeiSjF9Ub&4(QMiv^jBAp+4-co_o#~qD^#MW#S)JOYS zZ>6yDGQW6iv3Jk(Llwf(9_rnYk zwRInq8A<@m2^eW+YtsNzDhl}Uz*7hcx!{!CKGk?3oT28x%v51j*%UO^fxs4*a;0tI zbQdNnf4jM%Mi}u*4{-p#^o2JEy-bPGzwo4PuWQ(>Bx^F<_F5uM7Rhm%&*vKFy*xu{ z+2;pY+<%tt+3u{IK!#ah_~Nlg>E{eplLE46D#!p`K1Yz@E?rm{sBAFm>JC{uJn8x_ z9-Uk6N>fAmYwlF%T^-#?LEQbCyMK-Vnhgy#jewFIB@0-}k|pJ)duCY! z1{xvZ@ypE2L3_W%r7RAXfkC}`dul+#&L=em(jt@5tY@o^LZGr`5-o;HrSU8QY&}}= z^zC)%)+R%WlBQ5Go^cK&QOBWLFVySmIVQH6m>Y3SObpm{i?+75@=8n5`W#XRh;!fW zNuty8mn=aL5`);Fq%elx<1q`{56(|ue*Dj202kUCSsWF;XCATpyt-SS^Om3T{!f_4 z+PoO*1g^3Z#P9aHyb+n$kI4vYvSCnSgr@%U~qi$lh5~7yuAkk)E8yHkJ2~| zhLi8Fjh&-G!NFs(?&dF}Omq=!{=JxzZ9%F(mP3ks8rxwqg{%^7A#t09Li48x3xT%h zzs%50*1b5)90IsHG{FW{oZa&$%*ezzoi?*pxWQ>-La6R%O1Onh$qwjg2MXjy(P7N144ZOTg4#D$3C-^mIRABfvQs49lo)d6{ltTH*khiO@hq9P7y!>V~^(AtaveOCh3pW9n>* z%J&{B)FgwEyz1)o zo~Y5~sy?IF?)9M9!nWFcj6T&mM-O!_{Y*&ALFl?ud2mfq4EJ-aLJZT!83#gU> z0SH=2Xf(t!N#6)%Xr^jW2?>Fy2ah@2oOVh z5`Swg;#{hSbp@j>2i*N;;3=j1hE;q}E_TOD{6&b;FXs^^w2QA3ZkEZVg0M6Nj$*S9 zk<>Hun8`8@W<(UTL5~KYOSl0xPv!UTbfB6qj8r^4gb1{V9z+U{!4x#pXb^SM&>*2l zGL@EhQ=$iApo7jnr2bv#ypM2y5rMS%a0dw_2;DT;0UfRq8*eFA8V{}67N^e2n1O^pij+oJc}KmwY7-r3F*P!7sA3=I!lQe<@6!Er!GbNg%YZ zCJnvzlxLJ>uE>x8oKwK_2P^RJ?rvdmF{&vhfpoAqTuEM=^zPLoEhw!&BF*AcD^LBi zSHILWw1XxorEz1El^IF5c6qS82w?;D+WroaE5lTu=If0?zIY}YzOXI9`tNlN+N96b!{Tx7U!#DkNF z*e<3q(;i(`;IrvBX<%WEklE&I^Wl<$JT9b7qb!e; zcdo97XQuj7wIK&Dk!Fc;RhZ)JF`I|{`cA#?YOohB_C}}v@I)`gSP66_U*9Egy5Ssu z^Si{fHYfT3O0@s#v@ld0#sKb6;HLsq00H0APp3m!AI;|)HGD4zO=@jtRZUHwtgZP< z!=EDrg-c94q_zIQcUD#+Nwo+#6prg7?29+Yilk&@y$;nEx5V+`+4k`8N%HP(tXH!} z(rnS{y(KI;py^JP7p=|@>hDSnF)PrgPcb8c6{!|iB3b9Vij?_9ZVu*vXnj~{14*yC zAl&A6Za!`QN?cQP>~Dvam`VHJdKOEE?(Vd`&_&p_pf*co z=^ulS)%pY@`hqk!8XT%1j(v#8Z-GL+2w#!u%a{BDPZUI^NGhfkX>o4_yK&cM3Y?AR z`vkU^P6&76kVIOr5||Qz-uDqg;({`?7^ZBmex+~RL4v8FWfVSzh>pmV@UtI2r!0JU z*$kY5_~;?2EGdKB07uJ4+y(7I(k`s`M4?Zbn4*Casu7w*=dC(gHD(ygPSEq@qh~L~ zkzuQ26VRu~|CpSKP8e2GSC>#$#-fM|%Qxd8%^PC^Bjih1Kq2hImR{nE->)Ka2KW%` z-$m}V=;O}jeQktQjH03s(BXYAT9KYTUs|tDOi9r+G>qw6S+WFEw}BA~(0V~muU+`H zHul5TOSW0bC;#4XgKl%C23fUT)j@{Tr~>t&uIKzV*isIow>~i`383M@^yAXd(9pm8 zz-Ebu*BcHAA(|l6c>!*?8VQnGFB+;|XqHdYcZTJm^0EQ|E}pIZU)AgwDYfoQ;cj)s z0#|yvjYV=N>b{-VmLmc4$>>6V-8lTKiP&t-9c!U;au z4)y$Y74PY5js$c)&&{yQ8`OoK&!pOkgkj7F*UxLk`VL~7rl)%-6T?w)7P)%tBUhi3 zby@dw<2%_K77>WUFA>*1w=S=E+um)8T>rAWTv!n4#PIU8k3MQt3jM9@R&|#)sONOc z^ZyCt$}5M!QWoHE^xoEASpKcwHad{hxgpk;sK}Al|AC%@!s89)@bInwV`#ffx5NZk zhv57JcoN{leg8T-5A@h1wzkT2TfIeDbp$Fc$G|)*b4^`cWiH)7TU0e6&uQw)dL6@ZfgBxN8p*YprEGG{lE&jIn%rj|pD$s4K>H8;li53iFs zjbr;(Ov*q~3GDmwg&O}(k=g1}TV2TlPtkfr&^lHlG;@Y4f3C|&i%PDx+Sr) zMfdFbOheRy8o8%1yhmbyFZqKK=Fb*4a?vMdRj9%m;V zKbr6LvEM4Wn}{5ng?o~CctbooOR!b6)$7@LiNr~B&;*tCf|@^_Mb{L}4_~1%D*J6P zvQaMfQ8Lb+Cuz9py{h*zdf83qZ2Hu%@@D^xasxS+@9V=Xy7kGdB)=TLYV|Sr9v;t5 z&j}6eYkIz-ZEO8R8dhg+`HC`5tR<;nsXoo-Lkp)}UeabkZET|kWuF}Bj@6R5J}<|3 zh^nEukR~&u#%bpGA-7@em!Os=?Du@lqAt*#%^W|v7{scg4fi6}VZKv~!>Twx{~8@m zvyo^8g-+^bBOID$c6aObs-Z#Wcc{v?h6ObPV|7bz($8DPpV46gg>Wt=f@cT83JN?T;cM z;GlyQKvzBt7jZL$l_3hyh}sDI+oD$5QqmBcM)$|7FoIm>K~XA`6a;)QfR&>ScwIr< zLXC~E&@jW_<=;xb3|E>Bn%m5-R}XTmuo_>I3C>?2b2J~UG;ci1UgNx5w)S4a@cWne zfsn5cq$d$)XRaXm1M8c{e{>_i$d_Bx51|Bc01vU(0pvK;8R44hMk8QQG$_85UR_^L z*V$2mPD!bVC}sT2G$_{sO%^F5EH5;7L?zQXd0J6NQ|o09)GXSO&lEH5glY3lv@?$^ z3n}f#=y-+R=k%rp3EV&Xq5t{zQ}AsVHTa2g5BIxR9S~L9Z_d?`2xjEJRiPC%rUusm zM?hdqF{>|*>;6GCp{>m)Ah3MunN|iI`G!rdfUoe+p4?N-M#ImGCdVS-LBtkbv1)nGN4>S=&8r9nx_C)Va_4?q^%92Z6p(eABBy#zhBtQ))`5}3W(`S z6LWLQPENejHpBo_q^@l=0gFzq2Ghl5xhAAgF&VI?|I_JQ3C2}T zP1-JBkxKc}yI++$@Z@Q`-Mf1{oOgTe+)5kl=JTB%T8T4CYmGt_eg_CMaxfzqtlR6_ zgFlL#39p_unLlAQNADZvSRXp~i&Z376iVyXZy#{;^FPnz$7nmKWtis~Y2A|gENqKn zm+mHz;(LU?9-ON38mp5@7tSCciW)pkFY++{Md zm+$&&^Oe>L_jZzoFi_zX?-+X587a+Y=hcf?S3d5;L#Q;-7&SIOIO!Py=_vx=y z=ysE`Vx+xG-N^Qg)Vb1B%E!JHHd^&N-(~D~raw&?S7Xi;;h3AHTsLmJ(eb`7@3VZO zM4L?FNA(=8(etO!!oGika%tV~n%eE|gxtVo-6nQDHu^pFf1Alia1IKUN-dV2`4+F| zbXN^K)ghma3?(E2aAvWcwn}9Z6EUkzq{KEdHI+?FkU+cUA+H433q&T7F9WQS!%J`7{5KZm%TZT~k9bX=s!tfQ!D>=lX#xX%=m!hlGyKT>iP2lI2yVeXu__ z$L%gVlt1zJQ0F#AK;WZDkQK@}#0-k7q5=`ed->SBUi%?w2uTu`#ioM7Cv*RoO8C7{ zmrbkb7<57%?UlnTwO$BF%xqXB{ALjhCe=V534+pETi%KG_D8n1w)N!?9hB{3^_P0e zB_&AOMm;56YN{YLVj^-my|_WnX^wO-486XR?+z_tLDvLiGuzqbtMo?mpCN*aW@d{@bfyQ8PRvE3Pif?Cq5WyPjp40-!O1 zLKZx>3yNo=ltZQDAVYNb3W5vvsAz{dSXax(`UW=bW8!6?#Q9ft!hHPX$(wQJfo*fO ze;1%MF*`d5m|gaUDW%|~Fypn*{I{s_86dCJLa)8#F>&zFQA+JBK?ywJUn10pW=b@= z5*g?7A*FVIvHS}+19Om9_L^`Vn4fHS$rh#Pt#qVbcf&_!p;Kc~Nzi4|YjRI3w!U4+ zA!y` z^af6+Nj6iLQfDXh9>O&&DF+d8#)AZ-ejjULb_zX_))(Dg6ziYdY%CH>mu;>l{2ZOG zIYtbZg)g8_4# z$Bs@oQ3DpsOQ&YMq@kwre+F=C3*RJk@!8$6*ZJOii7fOuhaA_heqJL)e%8z#HWqIp ztBGx9gaZv5>v)j`c=g`b8*x9Jn(E#4gcuVH8>7#STBC=>7E~h9LT^WJ)BUcR z=qT@22RM}%1#k=f3UFr8;^caKa-+P|A-%OLB@f)-YVMGeO zm>0+TH2Ja5Fk?S|axFYy80X;B?!}?s-A)i&nqIW3!OJ?)A=TDnV;kFuJSFX(#`Cry z^r(cP!t1rZhK;m2k@JhR&jzkfEi%7_u3_;qV|xg7%kx5aO_`EcEwN&bArGph|D?nu zIE=bfxB!a{RNvm(x|OIEI!mjF%kqGJp^Qq}ee`;I0A9 z9mQaKE7JIMbZ-PsIIKWvEuHOl_u4*Mv7m^!La?3AOp+rbui{&p4TCZq402HhEF*wx z(_C{D0yzdL6e zONj$BD=RI21gH{#q_XM{`YkU8laXXM6 z_e3G!Ake6CrMR0GoVsYYB}DSpMSKhUd1!Kl`g%1{{x!kND?LHRRbFZ?|=j0YJ7pYQ z_a7p+FV2l`xOw450@|NMyBX)tnf$!21z>Bb5rUmPl=n2(D0j)mBJc@|_*?m}rHwmvWP#zDHpfHVE1VjLK{?eQG~VSXjBDtmpkmvEr#16Q)c?T9#@I8m}wruM2$* z%rZ$$`a-=f!sbRD6(w-v*mFW{o^~Akz|Aj(B_`)bvwjv1(UcbZ|(jq z9Ca{5Ttb}@Nf3kme!v0K23=8+iRDmIHMX>N;oi>MdHhRov%uCHDE9DjT$?+?1{>45 zZUkfdz;VS!UU99{^|&ouR;U}ZW^;c3u$c3YOu#}Ay@IFn`+xwbRhWEoN~7YtWvODB z%8Oh3m1j2ICls|d?!m%-M$+)VeWv;#h*ok84){xzfTTLjQZ7~z@c)%og?*x9T?s3|K%%%ZoM}M8fT523{B7n-}^H5XSO_Jbe6X ze|Pu{+&*Sk^JM5M9e9p|QK=sW*W6j7xdD3=PQ) zEI^qd5rc>hF_jMPelFG|Akw+FkwwMLg9-FkP~gCJXD=IoW-q8pSP6xgeS5`L1Qwq& zPgi?@ISUC}UgcN9DmKu>0DrGcjG{os(qSJZBUy|FpqGYoF#shh0ECH9$Z3PdX$cFF z6IuAL5d>tjrxMQ@$1$BLsB{gK3Tg>wX%`*eb4fhXPWkm67i`NQ#BV)HssIJIfN-!` zZ|?7JRP|BNf_c55Mou5diCb{^goHQWgnr90_0d4Efq?u;J4GjbA)!vx9PY?}=A^>-_pT%2mUW=?-QPy9mU?0$@gyb9a1 z#$M05^xg0h8QmD=isJQW%28Y2I9;|DCcoU<9j(s)Mp78}VRJXvN(aYI`VW^G!N%b< zxmeHFGVgh8BO~Nvj<)m-BfU}DPcsKXqM4RLB)&Jtf6j}yJsjn5>{R4k>Z(jvWCf07 zJ*}py<8LEZ*b6+Y=@PqyC|k2gQhb1otainzAh|1p0sYhEqwTkJPOn1 zJjF{gPs^oYcs&h7jv36aiF+m^r7sxNgRYo*lT}LS(TtGmKH!n5e`f8wkkiayxe_0u zc+p*0%v-=0)pe+emhSp4H$re(jr-I<^sdypgq%#mofjcupDuE=9(1AGfjEb` z%kl4L!7GYPpVNAZ+YUT<7VepG*k@&)s-3B4kF3u*3pnXYwgtvJ)vu-udwIiIlH zCTh81%r@#nm85YIwC95vl~Bq5rv*^sc}rHTnmKe#p2fI#OeQuV(^bo1q$xzWW{D=G zJ$`wzVMiB#DY)n3iqTrF`aWX(s}pP3itI7eXFcq)mxQyJi!D>FtiY?uYICIluE*NcGU0OJoX)k7$*E=*-(T4EN@j6XiCPKpC!&^EJ4=#(? zz9Qvke|*kWBBN@c0&J*wXAgNfU|B*SfqYGFZpS+{yF2r9vM-X~e|&rnVF?F){>|0E(bQ zAan>GdG@4xR+yQTKUSBGL|qMgEhHb4oHsV`r^(md`QDFi3t-~4~T%1w+QU80v zxXqvD5PjcER$QY56^p-HTQmerxz7plsPPAj96S4!M9&{5zKBI|fl{4}J;I-pPp`A1 zUqzoV?R}q+@bK_JV=>Uy><1cjoO+IQ8a8u@PA-}m#^(m|jPHdBWbX)4Mk$nk1=;4Cdz+aBg?C{HzEIj{a&4T+S#7bF-dH^2Jx zmf~XH6A|TtmE_R*_L~yZ@Ung0v=C;5tfj3j^%odRqUzkvrT11%_C{voYgnvwc;`>a zpN6~6^dFij>T=Sf#2lXhK=-4G5NZ_&8u#lNnVVa&B#py`y5q1A)v5;Oh$9rb?n;LQ zs9@eB68jnqp^dV({09Cmb{vl47QwD4IxAP~kNgN8>V!sQU;XI!)liXN4JRx_T%;fcN(8Tln_p^Mj+oPoRJ)nDFt5 z#Ym=s2xq|C%mfw3R6aBckAnEiV?_{s;G=thHS>T>gM5GU;cH!=$5yFPGN%qrJnnl2 z2oE^m@FNIYak3aON%2=Qov0v$4~Sc>B;Uxlq3+EY<>eGFDpZbsDq5|N#;Fa{d)Q` zlh2FeneCUag74&}6@zK>6u=vd6)S?$4IRXUm@yczH2X z3dgI_kq{!O5#nBz1d_yZ`=05Ye63TvztN;?6rPrYi(^800w<-b9f2}tsGw%l>|zS7 zUIn7HS?urcU#~EDHvHy;{6L>_Cze@|Bjoh)4Pjz*2~BYc60S5{obY>LHVfcRE6ByH ztvZcZ3^5nuVfTp(N#kBP*M9vuRv)DcANo6%-#;?a`|w=!DSvim)A7A|`l9X1g<^iH z)8GI1mrJHzhx+cey+B==f&DYA^TRlPck;|Th5-!AJJ94_HFoRz@E7JS7%x!NNc zC@^LX-rrs^nRECCg3(BUNvxBVg6wE>9je&l(xhWN`R-QAKFxl?j#fT`Toojapdw{K-|V z&6U2QxjB(&yRO>f*46XIUo^Mbv5eI^6>f^IS*%p>?vJ}H7AN)c!y|+ewDjeD2nNX( zxECJ6r7R%*|Km8w{>OQ|jUdi$BZ$(u;h+=1`Z*5W+aKf5xi2Ct^@(MD&lDmG?xL1P z1mT1l^2DPB)gM196BcgF3C`zqM!-x~N!wlc%a<>8n_SubZdOcf7Fv^-61blo*#BQx{l$8hKd=sTia>F2B>#qOUO9jr% zU|jsJYXx&0==)gWpC1HK-uP$VofPx&@#TQ>Ko&{SRU)&&?H5WlJz*?w6E26GFCM7p z_kU*}n2mR5SE@%slu5!f{()Y{RyOxx~AD@2aG z*1a#rGq19a51Mfinb0Aq8umPIZZLgiC8iYNY}y7o2b~;w6kEbOjQ&4Wkuq^bj-n`Q zt|H_-_8IT89LCL;B7?%gRt6JC)ba+MTZwifbVyj?qUU0Wf%LOqZt(Yrfe<*}+8w2D zMvNkf)BMI=8aF{PE#x+_hOgSg?Q!JGhf0w$XC&QD+CzbB2}A?&f%Ie%rDv zQAZc^=dFF9$K(J*eFzzvOQ{lQl16Zu9!uLx;n8GunOKRlq|p&Dh1rXMW2GEYp?b(m z>MceEK}6I(!|TB7aCl}g+2aySi}yx@Zk$xhBL+?p{~Rjk{#bz_jcyGU*6sjYOzyE< zy6@%3G6w7)P12HKFNNFdmc9AOtU;RnSV;P{y1^2EKsG~?5`q1yw4}5tbufYigA7X& z^((G8%NNiv!91{?{`B?d5r>7T!6P^9gA?0v4sJ1haY2?{odV{>Jk*oQ0g9+oAMe6S zqm(be@sS{mepv_>(LVZKp$Gzh?{4ziBu&j}foqmXP9@DMn)*`(ddPR z!_WvA?Drb>O+n-UG`9#76e?mZ?{U0yf<2BSp-=fjHzb%=sXFg9w5TCJyH^aIoxdhL zKdH7<|KM%_RPYcXA&RmGGt#0qRj8r+X+XmPJ02QW*G7`7(6H zKaT1|Q=AYyAnd{dLGa*C6KNzuBoi?SaMDANh9Oii0_3GvlDJ58o*%yM&B!w)OEKXC z1C2QRJ=IP=Vrvu%M-X!mbKnoTOJnprn1VQRx?lz1PT~)bbU7FXKSsMv6b-b5sEQb< z3gOcPUUd3)J~9!b|JYyYG|XD?0j;wN5(}YemWAf*6k=mzQ};>9{QG~BlqTTeR`(=m z;j__IhNmmL8ezvm%BW88u9$8O`a0s7V)y*HQOU` z4rMt0!5#aIYnQYV65(kQo3$vD=iaOhc;E1H#qgCu2Pv=;=g|RQ*W3ff>R`}`mr|Dm zDJSVYg};p@r%5MTR*t}pe1o?<1)eWQ14-)G^l(L{W&CA(gU)!iPS_I%14)w-Q%Dm*KHjQX;VE?(XhpkrD!eAPthz-6-9i(jXuq zNGdJeARyA+(%tZ%xc5HiclPsqeZRfyi>{k>&l+RQ7;}v4;;Lr1ejSmXj!3bS59}VK zmnz%Z;$~!g2It$(zC@9_x+fqsDn345^6gv35{=JB7YFZ#y$bxas=|7Ek(TQHia7v) zBbn^T*qGd_R|vwwNP2EL$+-Fo0Ih=E*)rvY>?A7zwxuOt4l@PE+~JN;OmL#i!-jhd z?zOJ7Z)PNk0K7m%YJtd^ioBP zFCq8S6_?fYz%T56q?`f%6_g-U;;=twkw3nNhkZk<-Hkhu8DPH}$P&%csxqwrG;ZY11;38{`XMW`!ZJ-rNLNlE zLxq(e4f9~LOS=VdJ5v?0QNoec`?C_Bj<6<_-OK!jFThN|(Pz-+149=~zJvXv-dK>> z^TVc%ZGMvl@0JrXFUAuS)+K|s_X&^N(m*W2;{c!crsmf3L!7QRv<*CEz?YFS^h@}{ z=R*vx^cE$Ng4^{I@t~xFU8d$+Q-KdIYC6vR6+7gg91sn`Zs3s=Iy_kUWgq|fVxL{( zP76>nL_YA8Z+wjJVh7h6AWQj=<@`S~ixQ1qQGkX(DWDV(@aIpZrEdWBPKOcs`Vetj z3^_S~*FHl&xTVU-@B~5j6d6%fHo9@5H{$%4tEnKEY#cBztndLwjzooYzSzRd;*Cz+ z2L7#&Ixlsv;KV;bbJG-Pq`>_6aI_>LAoK(|e^{=wvAGo^8j80srt_zV4e*gC zmD_%>in4ycE>Bb;{9md^H>KU_> zP86h}jv2O^UHtBcGCtY6%B)f4E!q<`@aq$`1X#oif**5J9m|5g{r%DN%33NSzUQvK zGJs9|uW9{%EfF{aeMx9zibqFB+raFE3A{W>ZxCrZggYi-I@#!i4 zYi&oq^VuR0?z#-<*Caq{D5u#F0NlYn{;4)Nke(`~u8vwPFEkH~m2k@m15uU+w$@6{ zeppye&zn1q-ME0w#VUmR?FX$23MGLIm;?a&gAod)XJs{;Z*)v$GZcziv10<45D&yq z4Gn~lzmiO6M-9J?u!Ta2yw)C}R)6KH@|=@*s4`MW6G$vAWrFVql!SmR1a zqF(v*dFVQJ?4}(0M{l^CtX#2ao(w@WLM#U+M^99#wD z0g34;zj7s1oT-i5*`h&&(ev3k56EMuX*Ltdr|gDAnXZI9>RSTK@WWF4xb4vs2!~G? z@!MCUV9S-77cVkWJa7&;XYe)b!V zA0v>(dX=DOjI>0kloTgmpA86R0*llN0f&d{;Sb4%ZzrP?M(hvdk}nzF7b^>q09xe_ z4f{$J*4mFb&6)vp)3*xUYTjMbjp@XM^)(ev7bcL)-wGnVCjkd5IAZP; z7N5u!###BSQG0kUn|E!9$undatvr9ewY!(AEaOPYDq<%nI8mDIF-;VdpaHhuEa@3wce7P3=9>23Z)4P~Ysp=bEJln*)5ZU!4a69^y z^`kmJGd8{;HX~V?Dk+k$ZH5{VX0)k^YGsbgMr*j9`l-cab<}(&$xB#ynen*9R{*Rx z`1h8T*{fTE`k9#*3C7qLBB6UMF4mfZNo&t1t3GGOTXxxPQR6x;1>&l{ehr%Yw>TT; z7}kAK^&@|aB~Cz(k&9SXnR}K-%V+x{SWLuHmLYk)$N$XZjht@j2JJ!5jHjzDz!o~n z*2fW{ym3@cSCI(-iRZY!zlHk*b@p0pmr7Km&|x zysupzU@=H1obj!;D?4}GERZM~%fzr*v5eX_wGt8%z=^=0o144B=x+?((oZB|m6F(>9!8eF9lo3SyaiNaG zM#_$y+rK^oWE_758vo+aio#G_pL^{jVK>4uTneH{^5p{$=p%?2`h0q>IeN@*#h5+C znBEIG0+7FB52Y*rOet6%gdPv0hisntr&_4T3QI`J({s=@^J{ zW4!ED{s58>;T;@dfB%;FRtD~s!+@QROuz1<6{&c)UDgmaI;si{5GmVsXpF??3T8jo7yxn7)>t9f_WFmgr4*|IlLaKN{vkhPpqqKcL`aY&gW|p{{}4Mi^))C2 zf?7~a#9FqE5#HI;-=7CKXu-Q`fKuy$>hGwR*8Oz+)f6!JqL;|vR+B&B(L+6SSQG@2 z6cj}C-S!U~urwZPig*PIik*!*mVC-Kif9`bFO=x<_ zuNt)i5X0%|=(39~sm2NvR|YtTACeLNI&=Qd;R8!`97!2K1&GUL`IAm(V$Hza-F+a<6}7>2f4LQ5e=WIX zcJODBMVp~!Wk+ESdDii;XmmJBeaP}MU`54TTa8yPXlzUeI3pDx&FuUWX!i=0!h{2Du{-bHVts& z6)0uM(_t;2f`Grt3gZWum&?mbCY=j9G||RH5o+r8x|^dZ9hWJS2i%l;7Ft!1bPAmI zjo5(KiI0d$P%!QHZ%dSYO$t9ix4W}Z&jMiEfDqKw)dPzixIlvGk~zrc1p|_y#F@g! z(y7k9iiOf@4|j?WMJG$$DKKnjsLVB-=X7PMG8?AB2(@E^!U2%mznn?`m!RDQXqU6oC#WWhVzbA@jD!l#RGl%e8NV@e<^K8F|WQc z0SSCETm+rRLjN9JSCQxzv0{LB4bt)VX6ptr1abcmZ9#z}GCTe(4TU)z^bPkkM*SEML^WG`RI(qIHEeCgeaP6w&*kC_B8<(^IRN z+Gh*rQ_4iab{aTR?SSMcFPlgV96`sbone8G3rOS1lPCYRuy){#1Fl+7$j*FITC2`P zUbYpURJ**rBntuXQ3Bn{PPS&uq$O|w)&?fhzs}JAdlI-gJpgOUK5)@c#Aa7i#Dbha zct@e=#!C5{?>Tuj*v}GLJFq?K1t;q~_@7L%( zdX;m_>4OM`MAr9ni)qziW8V81bEc>2Yus&m5)4!QnZ(+7;0EfYx;n^Y{vXr+t3~bl zXlP(~_wtG#=v_hB7MNKP{T@LKTbw*OwKMetDBD`Gs1Xx(G$X>okU#@|MfGzkbg`2&lc@5Q5DS6Kd}I)|LXCdH{k!-0sNop z@D~47{-4?m-~IPW{{IF4J+eWM|9>_VB(^Fv@#L|xF}-hIsG|g7Dnspvf@P^Bzy$Ii zqMApJ{+bPvAV3!W50FN{M+68a|Ge@7Bpd+-!@j=0|EoRVTafXZeFSjd&wRBl90n70 z@8Dpk!NoD)0k5v9sc=6wb#QR#>Fr$_F81UWy*(4gYThScNfQo35Be&(k9v0ye!sky zTL++GxN`rGCV`^6{oC2Yx>o?27vMz(OU6UFVNePbVu1v(xPcAb;Nizqstjvw_f-T^ zvpp@x2TDE=kF?~sFC?oi!}{1W|AX6Eg;Zl>{JowKR^N-r6|_P(7Z zUS7SP^ziZx*XOk{OBY*nbwSsp4Oue#I!cWH()sqoWxEKlSb=43c2?&u?u4s!y%$%{ zP&qn!NmmB?>C>l&fm@;O>Cn0p6?re2s@JmTd-6s4f`52>#AQPb~ z2hPyyBhy^$m2U=n?&;o#y;%Pi9}z1_x2*c1cNQi?yutZ8w$Sm3Di61dI9Ie`UKhLL zFN+9kV#^KW7ki^(%1C~zb-N9hmZKgiem%O zggOJxbrD^e*3F+Z5evD`xPGfSZkoMGe2dTCvGPnx#;$eaF81laH7SZV+BM{5w-QO+ zDZt8e?qBOM)hXmG|rU_0LsOx&EDs=X2lYjQ=Q!rIn#rm>c!-&5%g z4$bXp+DkQ8>Q_Uh>t}*Ej@4XaNmSOfjR6D>w^Za2`|m+)R3miZRUhg@3(&ikqb+34)Kgxy_EIeCr~eneDHr z-OdW49nU;P+F_N2J5G~_jI4GOnJm|y@vSNggqL(V9ab#}$$cGu%QeR%;SSog*0COfv)_Kgy^G3IrSH5TJj`MJX`P zhAXD?p03--YWY5>U76zC34=@-V;Nm#GgYNfp4x{0v@F+koFl36M`!i(A%lw_vFSRg z(UNh6KIh!chvBdAxv!CN8hK%Qd6Go#V4EkiZ?0%xsXh9{P?V{W>3$@3= zbvIV+IC);kvD)u?+)?YU!^WVH74^B~VucZj#@7L(;@~yvBJE`5g_6j9jc+&qE(}hB z%;wn9@BMFQK{(C!W03<)qN$854Jm8GLVZ^TTxQByVx2uBFIOMgQ}>*FvFVjKqHMqT z&K>`WBn^!#=RF2opO5aT@{uy{4XNHxdfO}`ZVWDb3uot$^{LZ%#2T9D4TY%Jua(m0 zAN zT47((hU4Okp6hkBSlnfUCGz|Tk~R@Vl!9t2kN5hwR!C1RY1xYD(r#;6n^Ut`eOS5w z2%Zr47D50196_teo^baNC1*#1K{|6Va^Z@|eR!_l`su458wLBU1;d}*vlc(r%dNh7 zebLU^($7ljm3d`7NJR}*>BKYD^?_Zu5;cQ32iCsY-zZ#I==kaEYt%m$uGLiZ-tRXW zyXhWEm?Yjjy~oyUB9$zzYRaJf&w=7j=7{D+d6DD%XifoyQC;imj`&p%0FBnK|7Nnh zWw;)<)?s{Mu1Bl=Xwqy$qS>*VLt6KmyV?B|KVc(~e7tvgrll*{=YsBSL%FFb>NS<0 zE|oO%;&$?IAzL4WUf|Nm6xBP1*5@9>(I!Pa6W&U@E<|xjsH}`2EKR51poy8V?tVx+ zB;A8v|88_6m5)my>z8xhcU8ZuL)R0>i<=G0OWk3I=~1)eEmocW5OR`huE&R1 zHpS^^vlqPSLP7|J<$E-!)xq7qUYLxd$CtAcqTQGA-Eo*OErl7sxB_azBg@P39gYew zXIOob*q@;KD04P=2Bf3^dx14{OBah;c}QshT=7Gr`OiTZIDj?jiX`lAbUy}ef1={x z&x9Z7da|Sx$I$~;3~AABHkAbAS(mR}btIhiLYy{Ol2~3VQms1nCF-v%)HnCLCM6%X z``)l6UP2npQJ+W({hB?+Y(D?NGViCSwWwy7x;ByGOC%nQ{mKh_J_0p1) z&{c6`y8PC!K1_!q#|pWUy)hoz!}xT?&2w@LTMVp)>|w{{SzXEd7tfv!d4Je$KE`Ij zdnNBP@`3_2c)a$G|M%B@!*gc+rK5X?sp0+Ik2Qjw1-fjGOxI9jyRL{T`Ur>BHtclw zA-S6s(tShg_Ll`XyGzl&ZtsyCTa>KjLRL}XRq9{6R{D6uGHUNHUEz{ST0emHvD~GQMCJw z46hSEu*7`z>F;#D#KK;M*#FzOP}^6S(IHV9z&HO&-koR>ng06-|^}F4b;|^LCK`bo9FJ?IA%~I$NMfMS8E-dWrt>fDuY}a)_N*2OhT# z-11x}=J1}u*^|U?L`^UIy3t_18)iAh1;C>mxDA{6^bEh;aUk%;Y>yH+Edbyh1 z((PAe8S36xHk8Ed7K^7|6OSj-0epXkZan2k?(ukfzwMofiZ^s6cYVDRX*0D4Lt=*y zmTL{YoTlN7P^tJ|FwD!na>S98nNZbchYM~+E+W(o=uSfn?k%RH)#fXm2fms2!yQK$ z5^dxik@o8~3(HA4=MFRN&)xQ9NY@t5uc)mKPuisuPp&aJI=I1*5l@H>Z~v}EuXDQSyyI=f-EtE=hjr3V#myc0KCgh?*o&N< zMkM2FP}nF7>NU<$G$GEB{jGNT#YGH$6@FyD_R*pRmR3^b`fhTHj++8$PV>@C)5m;Y5TUqjS zYKnA!uGCn!I|K27Q zu32pk3CCeJ0*NUPOO(tWQB-6V^vAaoXuxxt+86}V=G2hE@+0MiTO?MiA}J(N{=3%r zn;p?#OMoiW^r@WpQ@dTQuF(t`6&R<4!v@jD?>T2SL&avjqGcZgIdiLmILXSM#ABee z@u#QT1zyBhFPcjWY`d>*(Q0Z>`z=TO(j7ir!Tn{&o z|MYc%`mEjS^Od9OinG9pPiGFV6|yAY&v-_#aDcP$Umuo%o(*suxhPej!L+xx*JQ%E zi%XFEYM+sxIdOMEsIkK*9j$NwA>h#UtD{@HU}KFXChd(^#QUBOtT@WFqB3jBPmi#3 zOFqBP8+L|6hJr%Lo7-jv3FHWAO8Bps$-EshrTi!) zwgpW^VRmn5{DZI_pd$}sD6Y<*Q_^t2U~aHw+i&5cz_Sn&-WK=?_T*j5MR32VJWtRC;4p+JXcuO>2|p%RC9C$ zr`mjjO-+&+-Uz|64|1lgj0V)s2B=e0-4|2ov8z#;{kE@?F{_gg&|2St;BFF`>D*;iS!LI%-}$tmR`jhT zXD>Y6SP5M-$#O}-b`50?;bJ&TB-q%Fe5+SQ69w69P!H9Giql#Q-t;-#VUucTxsN|> zrh=)H21-S$e=0{oKrioHYo)JNPr2{N;Li#vDrr2VOHcFSdF8RiU^gQl2;L)DJT?O$ z%2ZCxo-_xv7Gk!zSMwgxK`yMMeGSVjHUz0%77*`L_w8pLsGEVQYA942%B$A^r*8B5 zjwM}0c1tvUs!ICB`tMx*av`GhxS_e|TgH)c>)gnACYLLHXIfqqrBxKpEy|69@8{U= zGM0$Y0@ZZkzVv`iH{w8B(iY87 z#PBtZG+eG(STbAZLj6@ADmO^MjQnNYaYk{0OjI!FSQCRTlONYphec!P%jEU(N^+** zWHGf>iJgPI!`=&1eeML!;N`9-(dZ-R!UX)i1d23P3Xem zRbS1{XOZ``Z*pu$;L4h}aBNb!gHALFGm|)ygl<8cV$Eswg(8;TIeN?8;oKV;clwzl z)bM%Df$7hl7Iu@Di%Y=Aci58P@H z{Fje!NzK~{Bh4z)NQDf6#1?gJZNlzoQnQ1_Rxd-Uh=uK(hHgo+kKfJPOdZj&*S=fw zILP;Gw{1a;jEyT&pQTeQJrdTnsrMpRNJ7lb&0~KeV0UfZK@Bx#7wg#vm(2+JWC0kk zNr(t4BP!68$n+dUw|aYLcSIyj;*kz%DI*@gPw35=HXe;4euiuxn4y!Dp@U#Bo z)3w$b@*fZXJm}Jr{!h*M^JIwQ7(aE;6rd+)fnaC?0(}+$1o0me3V7h9VJeabE<79@ zIQ$XNq$OAS+y8X*{eP*k%ghA&f7j@LzO68E0sBf|URMM{H3i6A*31RVvw;`L{9 zFGFYmppPH7lv=?Aub5Wh=Z%DPx&P0@+@tH;Nh_Y_A-`5Kns}tVk)8>4J|bx-TiT$x z7mH5|Y1&}dM8^M=_W6mwIH*86FSfiM6gy(x@rEPkunydPLE);VBD%~lP~dxedcH5& z9cZNhfnRL%5up;HA`svPj%`t~u%P~_9Y7a{L`Tc4V1gKfFRwp?`ngy_!3fOk(ujzV zM6xOXU+k_6CMjuj7p;?v3kEiJ;FOj0iV(QN`#PiOlG-8F0bLz#<;ONRy5rGdC)yziAR$a&W&!A=_t z>e>KUI4bfGeEiTZvt^ftOQ8rasZd0}{}cvz(zVdnv&OKKS683z?RoAnOS~*(jd0Pu zKd^3Q`^wJZ@{_PrK)sUaf3Z}cN%+z%YOQA+ICnG3mw*)gAge!WX=6Xb$Qu;s4(oCuOk8L3PK1YPAr>?ud1~J{QsC zBb`1Yl*x3&eJ3g| z0Re7g+r+qBa7nnB?2n6*5aiLht&deR;TT*)IetzcFmUq(U3$fq2Ciw&$;D|AiGfQb zkmARfRPb2n*jC4u)6%n`%La_rg_X9s6>27(^|p&ymULnkq4#WA@>tCzyh?>n^<^&v zkPw=_WhkZViUd4GytES#v~#^id`e(O=ZmIfhM+6#lW;&E5VPQ4(k@p>8`34y_86^9 ziMAO%iy~rpw{09C!#!^k_n6J#17#@DX6%vp87?_cFOdn`(CW^UwRIzj+oENZk+ik( z!}&%(E&S8Asi{z2I4zPqEucY5!Iv z0W$R0$f<60i?Wl!joOVN3qR83mV+}GCKgviUtW;kB@q^_;x?DzbnS=e`;2^Xt6*$R z+}`37Xp|XkbIh{D%N`lCLPGr)hf?t|xPMpV^w2g}Po$^5+}H;{wj7qw94`^C?VVkg zLAlgp8Lsn}zLix~m)h<^B)|}5<>Z#a*DXZ5(pVd17A(H9nJ7T!1obw)@TRG>JA6rd zG>ilSoZ5?Bc}L`!yhlgnT)s_xxv*|bRihsY73c527>_13(9(qipjaQf@c1fisAoRx zMBq(j$@)Is*6INL8Luk=Ogysg?y)BB!6 z-!5C3dw34`q%3T03zf4(bQA!RDVS_Jjh3EbS{im5x0o&=tU(LWs3_TCVD%qxPQmF0 z1s)bxZQ@Xv>E@KO^b-f0xq+yW6%7k7Ni;%mduu0-`SdWSvhdJoP2_3>p>%FwzY-bx z_ct7d;P}`U*f4p0R&{e8ZaV^bHIRKm7vXgpf(RzRhPj~y3Zl*2vmPZEo7vW(CIo&} zJ^8I1g4c7>svL2lYy{uvZde!?e#dOO5t+jJh6dd3Xx<>&k|4)rCCelvvQk4@_s@;L z&eVP3`E`57PNLs{*;2#mj*pb`ES+|CZlIz3F0)&IQhw(L?E14< z20}-jg-$osN}0|95V}}GXHU0#U(#W8i$|2P5@VgeLkNfC6ea{r{w@s}!+mud*msw_ z;YD`zl@3>sx10_nYk3YU7?~F<619QHDI8>}<*?D3*&URq3e12V_ z$&{i%!$S2#rGIpT)I?p4BJb+F)r#vNHYRP3QO|T~S|coYW0?)Vb$j(ktg4HxUi;Z? zenN7Rs-Am7)R)AM0dN#d(#;AIIq35Ar5cP`!G%7wPqVIIg9jz?GV$BpVFnwo2eLFZ z)(kpgxR}y^jtxZVdQLTYV>X~AscO8ImvUw%QRAXNtVmIW|3}1TIF=@-E73E{NbD+_iqg&U)BUdw)696#Gt|XAyx_8=rYe0mrqJqp z@CDv!KK13DGuc1b%$ldFG+@|b*NIu!?>4G1;7GkF^?KHDITnAfs6EOD|8eqLO51xD z%aY%SQ$``xA1;Kzf(+K&s;PlMqJ=vS<CA{LAoq)Rw{WOfB)* zcXA)494!JpKb3s?Zv32L+H{fg?R+mQsb^HC9wlCk(TW66xY*d(heNN%`1p8A9ST1q zuif{w9#?pi#V?b1JY^xS6FVR4orRFRcf8CuofkGACAAbBb^bQl%H88 zjP(wmnq_~te?K!^XKHRcK+fBcB-GU+(&R^6GnNwHbI(MPF@-iixXycS)Vz^(*%{#k zcK0m4YWyZjf=DVpmTw91V=)1KbJzWVmUoJaBt`Xv+T0R3b5~fGocM=r%PlqmS(FL8 z%};5Kxp-W6maRiHeS|_(R8nJ_9&j5WFXNca)0gv{c2#*2ujxPV`xo(BOe%8DfeWo^Z(p|wYe=n(AZGb6%3vGe1T z%~y0X2Gy&pgF?a_4r;Jcp$nOmjoJdCm+h*2%GnvZf`nURg7A^bc9Wms(!2R}<|MLm zMN&|_sD6w~OKEeuI8upo4ME~FWm-1fqV^c_-}cDn@|QF;`}MIHcEl)HrLPv7c8=!% zluBUR=u7LB4{bxd5cpkWPoH3?RoO`uGsT;{>&ak|rY7-WY9JV9&&879!=@Tv6kZji zad+o+OZquMZVI~v3+Hhh8?2!(TI93w04#m2*dqKlv>z*-Cm=eQV0RLJtdZ$-8`TuE7eBr%L(0(2Ojsx!)oNjm1LIrX&p8Da zL|>J9H%h1&YGO*t`=}U4hRhNL-q3JfQDfMOuN-gNh~{)U)3r{PfF0HN%@CZRNJit%Wwko}{f z+IGH%oEl5kks@P!;c7IV_{Z_gNo|ICyyDXsJoY7(uboKI#vb($2b1)#3i1Pv zk^aP(fxb{6`qKCn&PcH>N7a@qCo1p3dP84a(qRTuyjhN8(HVX<6fB2_EWNow| zbl?a1=u1qHhXy}%Q4$@D2|0xhgUW~nG%)=ocP-{R5yi`$Ia$6s(Oz~OFgRc+=mQbW z0Dn*jBE_FUviNHxmjD#b!w4iGZv2UTO#lof6n==Llp%<1o#--m*>ILT#0440&KmoM zW->6Hb?jb#e+y#+0|FKMQ?fXp6uJs%6GTMr;a||0Hut|}fDTDOY`X+smQ*>Vw0k3J zRCM)Z$f(lu%}b_~9hI^0L*?);@gRS?1R0{12hq0=?SWgu0$rmI#8`@lQvvOjN1{lV zc?it~4FyAYU=K!wg;_=bv-{21*td)iE<;N>wT+o9&qbuXt>2}=?l*VO*DBbga3=ZC ztKjidQix>ul{RZC!i!M-={Gq=`tyh4KwT7G(7%WF@^78_X3oTZLi$pyc!@2T04AOK z+w|{GwxNawF32>-Azh=yrBh2LM<+!ItJH?;DDlj35$O_f6pCK5f4v@RNAgE9^noD< z3zbryS6$X8uP z%P4Q1kn5(VpsyqUDHzI^4B`?WGz92LNcO|Z{kL7*#^J_s;{DbkhEApuo?ex?dq$B|{LH8fD?ytr?*d>%t@uBN~83fti+EMtIDu*0E zJSA}QWN=~$-mXbL*a)?s9gFyb+CVQ}7`4Pk;KfN+r&d89boikN>JYlWwJPwy{?DkX z{!zyVt%pD_9R4;TxFzF1uK+>PNPx}%8a5P39!Z}j_aD6+4wL@9Nvtt7Kb^bqR~5j*0oM;gj`dxSA(lH zJ)JBV#%wj`jm7FCv}-Zoz3Bm8&|v^S9liHv_e$}lb3nxu1-|R)F4lQ?*T7o0e#LwB zK@FN&j?~#cS9g-&g5**(gz%`J2I8f_PX_2q>nN(*)YOfJGbxF1p5vy0xv{vQw3J!D>1F{ zE*uoB>!sSh!LKh)L_@>+EW{VS#lMERX7o(t|CeV9Z&qPfZk8{y=DwWNbrc>)S zq*fx^gs|o2l%cXaf?}t%U~oz*nLKW}cf#)UhX~O^Rr)K9-H&x8Y2(8;QMX#KAcO&a z3c^xrt=FCe`z#&f#F>5hDmXlCh7_~siiVxrHf9%2X|c@L|i3OQ}HQO?|WtHvy_-Ee>3C*^3H7! zZ~f5Qk!sBTvSqkSl~=?y1~4kv%|(n*LGj+R=?Z3_s*rFM){6aZDoHMV7x(cKS-8Y@?wxF=*vnaX6+gomr`&WHzHK&AiJZ}=cRXG#kfOx#>Qz3Ja)Uyd?_tFLaeZ)A@Q))k1A z^-P6mUc2qTimswILYMNMB`R^|aM(LfNw+TeY=ib3$F`F?ESpyL_m5B=$2D^dN(HzE zZTf|8eqYhP8p4hu7do7t{IybG|r8ASaVb@SX4mM|+^KY;fy~^(} zK5$mi@_6JqLIr;sy5R>M8TSf^B(=(~9ty;351|Z=V}HfrVn^UomF(ZIx3RV>UZ5Ca zdY&U)DKqfqZ1TOV4FB8t+jmFsyfBlPRo{&Etd8P|WWNLlHIaVtlEEiwL0>2o9W&LD z)5v`Ws}ILXFGc;P*)v44{vUQQ_nJEU99G z(?yI{osh@=V@}3S2UkRTP8*4(i;eMZlwvg;(d14=fQm>7vO#p*%`RFF&u{3pb>)ALH7oX9f;+<(0Z! zIuoG8rrg+|JmIW+yDp1COw-IqLvb9$$@mtQ@zs9rVi^RAS+F~4_^Z^*-%=l?;04<1 z;QC$(d_10ocwi%u=(kAMuY0*Q-S2uKG9w*Y3iy)$0}=q$S1{SRa8g3scWu+BtiqCpVk?*P_OlkjmH?i=if3k`qtUYjP2ep{tLRpL(kGCJ+nz##@{WhW*Z#5EQlPW9AI6DKmBHYlNF1{}VkLh_?U#?4X5Rj!#ITf< zhJewj1w1C2Ri00~>o?dEI6Y={FR&+33}+*QwQ8E<=ASrjVn3Tt$Cxd*els?-r6Zcl zniLc6FZfk;F17XNiMc}3quLOXvn0}XOaf@NTM?_ESZa5Fox;+iIoj7-NM`635|S&q zh$E`{;Z=-LThv-qQe82xz65&bD-IB)Aj97=DLi-NSHC0twl*^XQRKb;)!bIBz8@3E z6@6nq^l4IQD|_E=!>l!^bsyqH|B&0s$ZfyMw)21m6l6rVCJ-x}} z_F|Il(1fa5owx%(t39H0{-jos?EZG+Q;}^hAL9o^O4383DY^dahKpZ*X(2Cm8}B=g zne;0|-H!`mj25X_*JRFOZzBaY?G*L>h@Bhn&m0>Mm53J7d67JiS@e8eh@ZIL2H2X& zylYAA)973q8uRv>wj|9fAD|T}LzASTsw&M1QgVfi*qe%AA#zomo1?kTUr=%^GyU9u z-IjZ5`-OSkh2|`Jqk3h{ia)*I)#t<*14YD=_w)MNL=r}<9WAD+#ExEVjCD?ln_a72 z?z^)vd9QbgtVMe`eII5g5olHxB?2;Q<>$Q*b<_Fd-Oa_>A@52oiCLQ)(Ko8EQ}_#b z;gxGggA$7#Wg4cnKJh(6ZYlpZSk!8rlZ1W>^z-)^sTk1RAw|! z%OchiD#u%lr|P*+o;O78$cjjzaj$jbiQH>5ZQ_O=AH2?Vv!G8qMqp#)4a;0g9AC;6tQ9@pn1?1%L;G`18>>s>4w! znX9zSGSl-6e|{dXH*yT#LH_HElp0-ZT1(xdw0%6=y9rXavm#@Kx-}9UoeXKSOX<6l zWrklHICgW{a6*H@nuntfZ+e7CQnH>?*)4x((DS+0Y8=})m|nzE5*+!?-p;q@yNF{& zBdY!ur^>G1rFfQlwbL#x>w~0B=sxfJ^m!E>qNX0!us6H0l1?XCvbl&&l|J->1Zv;bXLx_GB1c)h9?;$|%YHQ~RtuNOXjwr7>0|6&#USq=%tg3B>WId(qR2Q$7{%?c6&kH6m9H_)!!EPss48OHS zsN1MzlwFR<{0*<#;z;W_mW${#F&woj1G}Z@&2%5&3 zAl%seCa6z!yS5&Pd{(ucVdj3D*iBFD@bsKjQ_M;k)>gPB1~)nXxMdGt^39Ll=J9Z? zk<6})^V^V8xTnoSV6FF{FNgCQ$P$L(EOb(_|K59xCM4@pjpW;JL%@JrW#-GMMH$Zy zJ4Y*Tk-%iS{?VAz!9vQElS{Ya-u%fBWQ$50oA~oHt_Dhc{%kaOtf3@n9by;ccqW|~ z1JUHV&(qFNlZFjg6u^DX+P5X6m5@>-a~zk#u`Fkb;id7Uc8G9Z%a}z6la*> z*)1q&NV%ay3>GY+drNC}p%9lla52XcHBm2%M;zWc@Z`x1gS+FQ>cA7nZ|8(k&@@;g z(Tm5drJ4~79-LLkd3>Ln(xtC&Z|uZ*z^ZGzH23w@NDN1ktuSANxxiH81-}OCf_z)! zq3oc`yry=0fcCyA)KsXsmZv_F1dB((OPpv6Wzhah{S7)T%?TwlO#PhV&#ZSjoSYR} zDBlE?e#{$@(T)rSA`KbOxbHUUzKN)d4GtS)FR@|ZEnlshSonUD;YqjGY0>Y%dX&W zg2)Wk_h}Wj4_{B~uq%)zDN5+zy2J81WZ&>{hJ)CB0>1njEAH^*Uz|6;_U$|kk!f0z zUCaa_b&hqkf*@Z0+(ml94(-)5g-2d5kHEyz#Bn4yTOIkEhj%Kp7wF_*r%(}oK(k(0=zJ0@}H z>}sd2hXHSk(6Hc{`W1(n_IDX!Z`e|`-CKTUi0GpB(=u^x>!p^v6>O679>cwxOCh`G zM3I6E%SDO%_GFZ)DpRrP>%~iU+i=Za4Ia~&(U%KfQuumViei@Elbo~20=iIyXrbl^ z?k0>n7c2M;<}} zI72J8hI7lj{5y2(XQZ>T^3{-6?LPA!XT*BRsb2pdZGYX?=JWKA!)>9sySuv;g1ftw zQrvi=W_M;^ZTF<$EXDN9sUk57$hsV|@DRoK$Ju1dGkEHk?b}~!H+r6bWJvVOjPCKfq z|5~qz?0V`qNBup7`&n%}HJ{xJKt=3Zv-LL%dJ8b?na5W9r&&X${%&`vc71pi3#?lB znt>h7kQ+RVgr)LTx64zsq2XhGWV;D@mh~q%*)Z{Ww{im;p=2Ve-{ozlOD$RN<&b00 zpe1wZ4M#WmhwUB`9=o51m^p<*jz<)qS_;O1jOT9Ovatd~myf@}YL7oy{rp6!GBLuc zzhdX>&G{Ip=%IG~1L>!+UOVU=i7-!LN<+bbI4<6iiHuTRgg*3xU0u+TB$+QJuI5PJ zZ^J1{Ua6sy>&=eKzw~4;EVM;Xlf3z!!GWWijniIhW+dG`35lRKcSS3L!^}m$)8&DS zBH`rM@_|T_R3=>zBNCqVxZ~yis+H7M!#bPMX#tF#C74K@9v^A|p#8{j#oq?~x5*BF z)d|Lthr-F6XFNOP8n5N-Glam)u*K=t$##v3ru`dUd_$72MNov@@vz76ub~@fpR==? zZ?;KYVU$YtHW!GCk(UfXVR+ejsqom!!s``jm7e+4og$jxoPX$!|L+wNbb@o|nX?^e za{K)^>2tyTaaZ#F(Y+Mn;zMO!-61?dlrct7wxM`dkbjRY`FWKV^n881c`xmkVjZ)sPj%D&hJ<^ zNht7ErRRsKfEf#B|0N8|23gy^Ly3FvB62Zk>!yi3Q|=h z)O73DCTctpX>|riOS0;ro^zv|xr6QD<`8Sm5#U|B^}<-2-u?zwdS(Fn9P$2QbeyDqS$JuJ%1m zV?dT)P7)geQXNYLnvhUzUJV&#SvrWTeRRg{XUE^d%IzKOmx5TwPFbGi5G)OU)tx~% zR`vt&mD1nnI<5P@a z-X_b?diNLyWn|K_S3+4wHVq<4sy6q8edJA#FhQ2tzSNHY>NmA&nm`8^|v)V{qRp|Oz{Tz7+V#@^z{ z1qX|b1xFA38a!%4{i>%O3FAWg`XIE*?3t4EX);qlC>thgl>3)2KYqC~^#j+4oULi6 zHyQZd9RYftaHC-^Oq-qZC+^gD%c9(v)hYIen*lpuGfuz~>nP3UaXKb)J1| z^#gDV@Z^^C$;W&N<1QgJMhYM-c$33_%N`or-imcMNa^_D`~~7<;%LiKnT;1GAs!3A2$2NFoZ)ViC%CL2PmHVcow_Qv+Zt!OFPDss}@{> z-k2^)7a#M7%_I6>kyi>Sh8gk2m4wM63f~t;CKJCfVN)w<=`mHlv%VJqUGN3ew6!$! z!J`QE0KZ5`PHQ8sMX#r1_ z=4-313ps01_GXcY`yopQr&7!lYU%O&z#29h@l3>jvPy+PAfHQ+F!lYF{_d5B5)l$C zfkAS5ko~K91lh{e*bm7-2Tb?(Hn&zK9+;ylack4@pH@>$#k77-FC8h$1v{zvZmLhR z@-pM-AV0j63sYfWOfXdetgVmKYpSU64~fP?fU!2;%lK_Olbb$D?d&p;i5trTEAKepu#dusqY0cYnURq zDgTn~Xf~y{2L!MPvt0aiv-I_)GMdVbTdgh;{kTQ4Blyl}__WUoB_L^rzr;r05P1y? zS>@zjwVuOlUsLG8JJy;-ZC4;lL|~1m1YPd;fx^r;RW0>) zurekunI>^n71iO|Qy%1}NH0ACrE1ZiV7r&==6NQlBQ+4sngwmM%eU{-;T_!q)Re44 zw*0kpSy`Qjt=OljeB-imsj`3HHdG?ue=zNV>&Qe#Vl#0cg=oZNhApKpe=Rni)mtk$ zQuw@X^jA^S=(9hz-gOpF>6l@MV}5#BGc)yb46)m|Zran~t#m+7&lTY&rd(5DF9ED5 zIvv@@oo)PxR5N}5QgrOa$AQv!(9vwTVapllMS!W;P<7LdwR(=QBogYU*KxTUw02eY zp=kE$K#^gb5&;;Pzo&S51;f68}~VEAd@fl_GngI+Z{0}ad36td?Zmd z$JU8a zB;R;pQFy(XCHzg~l81r@ee%-Va523)Xwt|tRy7YtEuK?VgELvWl~+p+IzFkIc74Lb zWt}Hv=Zv#yJjYv(=ygZh7ZytOzBD9a@yCnP^6=qT1ri44cj;0|d7QJP{`t|H@cnmC zOhvaa&KuwJ?m;G9{+4i|B5E((TyjSytT#U-EZ0;gOW0$}+Yc&}!HwMcInmMlzJoNTq@z`h z{TG`DuC(X%h1yzBq)0ivbs->2-oa;SB|pE=kK!WuahyOOT>`?TEwhhr{J$#s7>V#{ z@CIhNS@V4qBW6~edoi?AVf$BuAVEo(#$mUh^(Y_{=FaYEuhnmZ)Fl!YUc!_#OH<}) z83X?I3+o0W*Ca2N;e~x@E-`a^ff#8%jJWLiog^wat>Y2wK?RveH5*zM^t_e7G{38~ zTTh;{m(@af3^ZzLW7Zhka=u-W4vwAomm-omB|W=5eirX|zQrsIw2@|gY3;e!o-c-|}$eHs`x?rKX4d!W*Z zVTi+Z{L~VbN3B_AV2DR^Q0hF`Pl>iB@U{iyM%p3x3Jt~vpr(=Fp{$%c&=OkScxOUO z)J{f9NcVPEXlMsUO#}R7|H4KMM_*6LZxdI#ta?w0=AD78DTFq=-d(Y&U+8FFaR3xg zE+V?O|MwO0UdJi)>lbjuFfc7;(<&Uix+^KyKGn5{^@6HK`Na;6vrk6wARBSbPh=6BfNt#9^_2IHFs$YukfD+W$BwprWeQ%3d`h~e z$kqjrk{Sx1%7UvSDF(1DWKp>e4s*;SY;ffds7t5cB^tvZoa?@t4eI$w(YDeU^U(T2 zsuE@g-=?46`NVBa_cH37um8g9tIg}PSkN3mo|Tz1(fyuo9gWb7Iv*YeBMYl>ta_i* zNKbBqzM>@Xvm?6lY5Q78bjc5b5%KL~WC2J)gYY{bJ85xGMi+Tg2s>Jb3|rZ*)rw5}kcvOT8-(dP`O176@gN>lt^6d* zaFQZz!=&Wdwu6#VkCzK!W>hmA?ZqSUV^N!`Y)q?$6O-SfM=o#YJHDlk`+LTE)_Qq> zf2@qG{3fIdd?G#YQ?yF3L)??vE8`5;w_eghCKOr*mvFR8Xc@4ubz)O8Ch*@S{a}zB zSBn{-2TY`9C^dJ+_E|;f|9pq*!-6)U_1={oMA_!i*ivN6Kb8u^Z>~8zSs>=YCbrlU z!g#*mT$%eJ=j1~P-$%*0?X_KU;!aVrLl;CuJyNuUjY&}cZV~OP6=Q@R-wdh^)7+KqF#DEkNq+lU2;;vq0Vv^ngY9t3cHA5^xrW_--Q`t*7y zH5+J?P;nm<+s3DX1b>CR%RkrzrdgUCMYcB7n)z17;-3-?=J!hVz3(Lr1H^AVi!Bz=zEM3Or3Tq_b5+%2BwwB-zf zVR;kW$ZGcKM3wxoy|eYRTm`IA@us@qq&JbVoT{*WJ-`IMvQJGSA}1CW3tA; zc5tpMzI7Mu`B+DyiC_?)Lbff_dXHziML=btGmrM+Ty|&a&fe6Jtal$&f*~akv8;Kp z;uBd`ZI{FN=oQ5+ar$y;he!D6@RPfiK*=HAuVs9UUHc6^|5c$7j!e7TgT>C|`h9ix zHtJ3HkWk~9v>!@=j!yeqH|wPjTNXaE6gs=*F}GTPAOr%X>6enKfa@~_MQ~>3ulbr;IC%)Bi3rUMJj9ex{GEAO$6ca@+6Av z%~4SZFUOnNLrVTSh}pMvfEEo9gNW*tVdBu9B}8TzxC=Glj_7BnH#fWT^bz6NX^@sg zY7jwbGaD)r$?c7wI@`j1iefD4jEJ(!y|I?~6W-!QD;{a5PA}Rw=@=IuSw?2M?xPe+ z#xfWHfmAHItHhEEr^tW!NxNUNcZ2!hu*#Xx!c_c2SHg}0{6pO&UU6gTg;w-h#@r2& zfj2EJyG>-$5f93%+%kZH79j?xntaYgDV8j4yoBSfXk{?Ddv@qE?y8)k6M=aVQWPBG zG;riz!k(aSNNvcE9?e`S3T^l9WAB_uVwWdl`97~Q5$h@e51t77#bLwc$R}2!`-yGj zhwE3~kU}G2;ZH*Wc&YSYC9Dhg$m^jB8{Y+8az-6*Y(;GRq<5c&bxrdoDG`_BL#7xw z#(`g{XLCJ#X>g<@mMJaoHpFvt+2=2|c;{w+%Km~lI=n)|2GxBoC{@NeAH;P+m`7W` z9I_CSxFLY%Oib)$iGb`6Au1f!&(xBUa*SSIu2XcVdR_Gan(8c^zpQuH&{B(iC;~Jg z=`*p0qMFyYs#jv70%RKxUBd+jyt!uZ8%kGLJ& zloBUX7YZY^86i=MQQIf#w%)!kP_Gov%+g}E;!k&4fX%;k zyS~7L^$#x`DnHj8bh7!W#@M75jZ(bJ4zR8Vs)W3)-tNM7XF zts(-@{DqmJs5IsPA72!>f^Bg7t^4_Bdf|f=`NNs#u}MS!C68GoQ~k7r3in*{?cU!X z9DHaQm4#0#88uQ$0T0wJtG?m4{-;nvM`oUtCIyN5aZT1Cej{SMFRw1um6qL0s(o3|3y6 zifAigvVKHF$zTSFj8%W>whPIrBf`svD~s85z%>F+$&(+Qhevx7&`h{=u%fTN``|+G zW|A4XriMY2(9sy0iO;WNC{i;P$iOuT^lHAU-USfl5A?sD6=mM!2n-kKMMO_=L|rkT3b1qct2?ZLY;9pDn)`TDr^)eD^LZVa*zEa7(ka_Y}F- zd{ITsO}yN5caxN_-vsTonQjbwPo~?d$%2wvj1H{BdAp z?Ku=`*4`3*t?oe4nV~YE@Cw6CK(jw|xdIf75jHLtcmB1}^BA3l)Z5))u#u1PWum%@ zrp*EtFRwwv&gEulWZDitqyDxFMos;v`Lp)phHizzNznuy>ErC0Jvs(56gMlYJQlr{ zV$7E$yW3;=6XQ7)NmPz>UHdPhH(6Ww;PGqI?n4!+`Mh%hRZ-OetkE%>cTg`#IOwRI z>7iT8=jhlfpd|ksUrv68*=lC6`9)v8Mna5C)1r=OHr^{k&s6bqC7O1B5`M8=73dC& z)UVp6?sd8bsVm91L=&5sxWQnkA0=r*jFp}pOMcryIjE<9GRsQOtx!Y83Ig;@O*ydq zXd@^;a}H`Jnk+Bbf29uQXZm!;5{uu|>ngwdAT%Ls8bEXXaXt793gPSy&v(Oavbv^m zDta>VsUO??-9@|hS9R0@BRgRtSH-G2nm93f58#)vfQTex*{rkadC zUa!D^V&T$e7FL)EMIR2fQCr@sqXLF@e>g4~%kL)G5)ct%qfw&J#<-{GzLT*$#BfT& zPFbD~&rQ{=pM6;oUSH+Mx;of)R20Q_CJN?4>t`qF=^=GVb|Pg>7&v|$E!cQz`C z7Cnm{%C!{VNE~+H+|}@Kb+Bl*w&Vp}3&y4&r)HatF}+55TO%KKZ0r0Fe_R9X{2jXI zJk*>1eiwsD5*io98D{z!h4-PB5G#eo#v2v%*O~|<7?cV-5ID+F0WS~9P*;VmCl@uE zNcc#i7gPF{^ev0UkfcR>R2s5KcAis9DFq7H@fC-?0wQ$4AV2Aq3eO9(fjxA->-wIp ziZrs7`VYtOx_7HG8dp?O)TSz5KNtO)k{Er+Vrxx3S& zDGu#j;S+9JKYc4IKey*?4p1lYH8TrWM-ws}GHH%!SPdc8GFv~qOZMPlN2f%osVG5v zZK%(uNeJKI#$o`=oc^t<0b}>+{3pG%w3v*!L*Yqn#kOju?aA$8UTzlEZ7!fg3S(ZJ zBZbN+d4()le7xo+hJb{GhOaLI@xW}BA1i{bnu7gl`K<(or0Bo@m-qeWhIVEfS5KP0%wV11~oz8$U$5(v!wt3_HwZ8bul28yc{dBG!iY3&ty@vOCR-#uvUf_p+x@hlYcG1S3+5KL}7b0oG z@zTQ!Bq`WCtX7d9+lj2~zvKJa@JhNGirf{#D_-Egr_BGZKw->|1M@p?SYJ&!Ij9&t%hp7k*Q)u5^Ev$4%SaXibHAKm3`6S3A-$bdA8t^slCWNMe|y z+9)XpI7pK)I0Ih^n^~C}(MmGX_l~xA+fe;!K^xZ?p2&HDNpnX<%&lBye9ZxOGSXlD zmV->1&5j#&UYUL~lV~ujwF|CR{R2$|GD@)%4<`|NVMVK4{na2V?qR?O zYwOZ^Q!t|x=x|7!vEunz~K92q}t>;T7|=# zYuo{CauV6U6LY`e9Mw8?kDDkOsylmCxX_WtxE-S9%C-C5i?-U>RoDk&>0nF1lilCG zVng*edTE@<1?$wO#+8+yJCXbIAy%S7%n$ELV)@%cU;Vko{-ar=W*0b#(|b2o-W3=k zxX36eWV_bIbWwfZA@n|OQ&h1jkPTw0cv;(3?O46Fz#zTrkz$;p_57D*p|hWfb%9J| zTHPj)7hNf0QWPMXf zrt2B{#SP2a|BQX?0u;BSZncm>;NxaG6xLBrGz7w-H?`dwZmMm`?3RrnD}uF$3I7$7 zx1E^m)rxE*l%wb8OzxKaaDfx4;x#9l9lf#d%Iv#><@6Rh8TZ*_?$^)%7KB7y3Lk2Q ziW8@!ju3uTA1%r{|E!Rk0dt%-GkG~{j3IswWgKd|Dghc1dwnX`rHAW-*!c{W{DbO~>efULMD%n)$q2|Rq!C2DpbeH2$IjWgb~je;Oy)tR#ZgEAH7=>9js2nPZ^>thFN=s2lVrmtLw3Za(Z6ScfOh;UkPq`OsOZZ(hzTzGcHPBMLi9EoV?%riU*MC|;k`U9U ztmd7XLq!KPL7I?9366%5A$EIiPJPrNZ={mJC4o5-&#rOs{Q0l;n4Cqc@*kG5FbF+# zAg)WMVv}WD*7w4m7k;E2px7#Cb!Bdad#JI$>K~aHOggFfS06L3vV@h?t=-+frY3Dh zHmkK?uk4RrRgrxhxWKuxnG@yBUyiDAg1TKXjj5;I{L1=Lcb=T0*PQaM3l_C=Vkc;0 z&N9{A1YQWo0+w~CYHEZga(j`p>~;3vMq)>NEeuvUfgyMx+g+MSv_Q;15N+2a$3naWHL@64-BYt!!`zbSAgB z4by|k822RK58Nd#{-McA@S?Su>CF67Zlqgyu+uupEz#v7Ew9IbihFv7xfsiy56^&* z4X(Y+GNM&hqa0<{A65SzbGox}4cbm&8Q%BR;nZW-h?ggZum~l#_6t-=_V#T`zDq!# zf!eXV9ULpDsro*mWPK3r&DeKkflnZl7gqz{P&&OwS64L}AbER8X1f5XMmEJz&Uc#k zY)54`+cMN-o+ngyl9TA1v|~Z6JIg{>3v5Mc;7OAz8jrJ<6RcxPv*j0+7b5jGdz8v# zsRX6Iyu3W(8cIW0XKBl5FaUAWs>uN;7HhRptJd+z5Q>8!cOMQm4I&&=(d>Z|4 z7F@2NHhPM*Z>&2zaU10*>K-Z|E#71xGS6(sbo;Oc9Arc$H4IXH&0bnswyVj&I5{C| zfXlpXH%ReCqN^^6o;_7>+S8Ab_i%43qA680kY+o85?;8d)|R2d?znPA&=rkjpvev^ zKT9%rlf^#M;yC{WNqfb%d5MBFLXtnT9~rguBl>twesYB8Jk(f&H`xl_>HYJ|m6clT zf%DGx#Fq;k${@T>!PFY(F-MaxNs`K4eC4)VgHqTBrkO3uFhKGk8b^c%M zu?mg;Qjbwl>*f(H&+oKF@bPek$>Gd5ne%%XCg%Q0cRaBfY3nPc2k%;9C;4b&3IVy4 zKs%kT9e*EcY9%$*5M^pNC}dB;a^w=x1Wo+_^)h zr?xc$`psh|@B8#+e9F^PO3pV;-&8he3dOumD&b$P58}@wI~DXXUM=3f)CJ#4=RP_T zWt45U?51SafQtP$&e(X}n?pj780Tl$E?JnO} z$E+oLy|M2H?EMwhk*u1C{|sE!XE-g7EVZ3yp>KR;ouAlq!@VyH=({hAa;dPh!k1pm-Eevb9>7vpwQWqUkoWddmI0xtOrz|H zu8cNjyn{rOOx5}8i+0;E-bZUp>k)oO{?y4{XsXnUz=vo!uu)Mf%eWA6*@#*$Fg3_h z_{YPE4z6?2BYi__`|Wr^5yJpn73T~%MfcTSn$x1XSX-^k(f#4XNBG!r#4Ntq z`Ax0xFMhJrk#(LVA*vFaxq+YgJB&WA6#+(qrFsNMAM76QF6Y$hOSmj|O9{|vZQ%Q; z~q_Z)GhZ;-^pn&TnP`zNZgdmaIt!~SGm`zW~ zLA*UZF;HV?oO7rcZKWW8+POkduPZmvgc4!N$jL2Nlc*1z|GvzRSQ^cAi2mCAxfn@b z?T(_Yr8z)kz-qMIRuy4G*b(-ktUOJiJgz2^k29nv%;(<>HoM1gPFkqS(^aj*(ag6f zx6&zrl?i@k!fcz>)U&1%`_zSmY%5)GD4C!(I<%wCQBg8D)M>Q(sl%6yow>QTa84?kr8z z1%80Eq;QgLbU|muTZFZO8i_f2)-!x>j);(Vjq}w1LChoZOG!&tbZ=72e90i08M2j~ zkyb{#-ccS%_K%lR?q3ou-wN~@;Hw0Vp&?MlU=8Z|kJa;tC6 z7ZxHB<-6vuhK)q=Me_aA33Wlb0q5R+S8i4V74lfBs(7S7{sV%IrG1b^WaoY0D{x!`I zA2Z_zZ!VLrh$sB?cumGQwvzUsR%&O4+E1Z18(s!0F!Nr+c%+{>AFyyTUrVT%<5v~n-kx1 zc_sD2;yI%;(pCz!I5>Aw@Q*iLIKsw9HS*tTYfsS$SRz3UOnoJJd14N}S$F}kslFL6YN-Mw@vAlk|9vYlaJGvobFC+^J7p{+iA0&5 zJHkheM1r2aD=4bF?Gio*6#!~D6_;)!mm1^12FKgON{W9y*s9{9MO~OkO-W{|CdBf= z1~QyU@%6hBn4JS(;BL1)yqV1^FMbIcdj0eHDw)gi*h=GQDe&fD+|;067i z#4vJl(V<$S$U*^r#3I(&hdO80F`idA3bx=`PY&g^V!4*~K-$BR0h4D77IamDZ-4bj z$F*{e6i&4D2oV;^zn;#a>veH zyFbZGnKai|DEIxHCfxiVEcs_GLa2?TFLo~;|BwfFY+=(CnMj4;7H%t_hzj9hbVqVc zPXN4#iQrmt?{Qpv5ZiZIP)bUnBGV)~8{n*AyK8k~y}U?b*hZW@Tj*Ho$>lAT%>h5V zZFz+Hrlzof-ZGcj6$7utD)yjW@TGl6fD%bKrs{A}p{i1E#2B*Ck}b*VOrN_Ry;N5} z+0Z)k8^MBiT^MLo^Pab~&lWtqJpO4#M|P&;7vg?fc`t0d+z94#eU`)siHP0k61li> z%!m{84+bmkN`~jiOS{fH&X2nXu1IOOjnsb&!1jwqHGbm16s=#YGS@UIaM=ofV zG{&MWE>l~w&ly5AvRkoFO1M8Aqd>(}0o=S27oFH)1`-Wb4q4MO1>v&=8 ze9Tf#`+Zhv*8_Z{V`b*M&*Bwq0iY+Q6QRbE&!hDew9RZZv#A^Y5!%&Hvr~3IbzD(s z7Nqj?f;rZ|$m$%M#G%cGvU3^!6DeVa5m!|O&{?mSL(T{oBm)gR!AJ^$whQ&L+ z6ICW}#N`@%y=8}jID{uF{%FOKNc68xI2wvk`#?yuy0=|3K+LH;w84Q&Dz`fv|kDO1zQvCMGX@Zuu1Z-WyV3zW$O`2Zm;-B|T0>%n71yKpDZw2Z=}RXZeP9E;F#7gQcIyYO)F6s;WDygJJEcNr}rBg_Y6D@#e|1A0el%g#rcx%Pmjqu(rcE7IP< z+L4TH^G@f8WxV;13mekgx4pBTcZRW$k|McCq|We|RefMpKDgE7qJENM_H(L}6H4*k zpI)4%yY4uzj#v&&OP{1ChH`_OJ4BlJ;K{GbG>u2Tf!8u;Lxyp)WUZ};F5oIR<{ zDM<^guhS5h$Wn>58*N_`M#?$eepqT^E28Ya`p|6j5V^`TKT2lExnArR60g9*L}b@; zamxJgQb(x3x#!^nM;h7g6$0}AH%3;&BT}*(_hxs)vJ3-hvURe}q#17N*FD3mr(qi` z$^>}Nj(N9!W-J3)WfhMeBFD(}JidmG3yq$D2Pn{6W{6saMSLcT2QWdCuIAI(Tke@Q zR+cY4HLL&obm9EDFF-3!b2o2DB$dV)^;p8SD>x%Fb!z*qR?MS7U!6_K?cs9f4AJ7Fp+E-oqg$nZ+)SCE*#FTvrhPY zfzs_`!(e0J8=A(h&(03FzW0<~AF?`Ow)cFF@J*$Jx6dsZ2i?M}p%*{(vRo*wH6DIa z-sT6YeJAQ8b31d9dS14D%X^O`ZPu@J`rrtrT8#B| zHr3d;oV92p;dbk;bo?{`3@#K*FL8%u;=%?6Kdbmu?HWLYbD}IId`GBSiqMRX7GM zDF$4fUL+T>u0OeX$k#{s+T^hyxW9P}mYV0f%lmqMLF*%UnIxW*dZ>J|{txI94f%#? ze}8{q+oO12tjGY7B#{gpHoQq6hg;s#k}gBn%UbaNqk`~rLjx`X>u&NUALCqW-D(=+^rzDI*u84%tYQ!x9R0^ks#=uCh zWFTJk&{p{k-v8A>R5Yzu5^V-yBh8;zkGm5N5zB6>&73gIhCHp3#qw_QcHg!I`6#i} ztd-#lKR>wLfMzu%o(l3$PGloRn|xXK59bjFO&w(!WZ=GSId9n7`5gXqFCQS=%F5IK z6q6Ec<=)Z6h44u05JjmEWvpG?@gH(jSXsV@S&;=1al!lfuz1)BsaD9W zAsmQ0QWdT4eh=-36w^X{?H6SkITf*1WTrksBK%DxH9~w?lgyftM-0DN@v{CsPtrhm zE4i=+=;G9s5PLs@0lu3#4Br;JA!nC*H|FOB)b_7JFBK`s(tPcVNx)VoAmJcuRBh>w zI2|R?qHb><8Y@dODCdmOe}vwY{av}{nEG5ZKq3;6e_Y3;^o0*A0vG~O}t76shbg=hm&6yb)F!eeKq1;a97X#7+dsaCix59M#h^Sx+urLY@gur7ZH zlJzqnW+4hYbA_Wv$B4xP@fo>|bsj`L(dv#n1m@K%q{$U7Cw26A6s%dhRB+#QY$$Pw;B8 zU-tNY1d!{oVs*QdsL$1v&07tli%X`bN7gmfkwZfOALE6yt)wE}csTw&6OcoLq0z@| zQ2pO0G!jn%)Q653K7?{~k%-ZR7KSJ*C4?wT7b$}#OH}0WU&)(xLODJq#CO^_yjSpM zJUP4}Ww}i?CIbTaj=RZY?Zbm~_Ib52E+H#mbRz-nzUY26=rUS;?+-Z>*XyJ5hL+D) z@P|u_6}3)Gv3;X+M1B!g^Z8#e%Xikkf!_xh-{oqgAh`toExIY{d8g@YPiv*WX4;vy z0PER2e)4$v@=ZTw&mGS5{wTHAB@TWQ(NY6c_ECOyW53TiW+rxHQxQSg|E!LwAA)N? z<^RWeNz4WVz8u0PktQMdD#`iBqa30zY2-gP1u%@*th9H)fkzCFbLkH|m%IO5r|QM72-fZ|F?ZR)L!PzQf>eLEGmAosoVGchF6X@u7Tc0PI|onA z?qsi9C#afSwmi%Tb{U^cSTtGf2LHV}+?s!WBo}-8oc87Zt|{S>EAWhOru&Ms=K8?? zx$scpzSPabyBIf<4MC|H`u{ZZKlcnVTM24sCMEy1ga$+x_1?Y><-N%UP{aIelc( zc%jZ}5-4Q6!r_><3Rqv))FqxoO3c9Hq!)9 z0Pk9|aok>dVr+EGMdBqOX0JIGbuGfRDy*f-UZ8GV6Q7*%LwPyr{T(5zAmie2IRTA} z{c}AEG^%`yDF6>rg!k$nK|C3S;14e1KL^Nh^gLK~Ge~82r=32VsdE}Dku!P8#0w^z)M{=n$PZ<+Y`)b}`qli74WV56@D6QK93I)aXP^<$rSt7mkJP z;Wx#S7XZMej3pn^;Lz%aMUj#_SUGD_I;M|}#CalO4-h+n$hF7DaXfDCMZkBladN4% z%B-}4HQ5so-p0+!H!7M$8znno;T36mWLl;6kJ2WTTxyk>c(vzU;=R#W-7kEqf5c07T*;mIk_nVpp7lWi4mg-F3P*ZUl-QFNJuZ>-7T%RkU;s zt&v1UiMVPAVi8g}p1`}@Z<`S!*dUtNW9+F*OL&)YR;)iod4Ks#q?N0hmXFJ||FFK(Qww|VE5JoWU8ZeIM&<8BZ)CqlZm3~&$sJt1Y-K(x9hv2;GEa-w&_H# z~4Er)MhZlF-}ig?12iUy5k95^jisfA?p&~wm?&om?H_- zCt2>Hg*&wuX~WZFavP>gxobibA`OCT?CVc|C2Suwa>;2U^OFVCZBJV)ik)VQ8t+dA zxZ=;U)((&r5b+or+^%iOgG9J9E$NYRFDo8TiR{Yphs9&pG@{Vo6zzyV-wH`H*htmN z(U)WmGz8si$E*e>!0o?!y3eSme2KKUJ##zf#5E*vdm!j}H;$zW&Hpqr+MRtZpl_;V z2ogSWPgyS{d7rjvtOEhxfPfC7pcp}85SK+-a&{N9+Un&16}g!TX#17kM|@buAnpmidZ?hW_JFvz%x4!ciTv@UalGetpQR1Uq{?~%jR;@&aNiq#ST>3rM>M2FWx z%fsu8${jzBt+u~qGPy6F{QB^7p`o$l`|y;8dj_ZS^-Kpe<_)pq!#gW!Zc^>l@)<8hpF`6R#P{wxo+D3t5sjf8>s4YRLaP1HC=}7107PC?=h0CtjgpyLB ze^PXfCy1OjZ&JNWl-M7Vc1z6Uf75}Vs1%zzxGeBHD63RcF@p<&I18Z_82y42ofF&~ zZl^f9WELFYG}YWt8-zPT%4j^b8I+*RoBit+~c1mFAGA-Ka)B2EXT_UR&~d zlU#2ur)L_|2i`vj{t~7f{OF%$8*uyQeCFex(7p0blePD2sbG(^8@h*YUk;=R=tJfS z1fUNoj_;GGH>S@Rj(D}Vk1+UudPO}}esbBC(A!)J!UKYtjv^(@rv=VqS!CRS8JMV7 z_@4uE3^C77*66wCqnQSi2|wL$;3rLGoDQMxo(7L&{cL&G5n1Wyz?cbIVi#moA~>F6 zn#H9mYr2u5>1l`rST?Q$cwH$sZ!2eYbE&74%>Y~9gHB(}rqkG)`}3dDtca*WL+I}s z3M}uNaMa#Tq!iqvpAq^m`f=Xl6w+6py*x|runBvjM0g1l^?q1m=xyFO#{W+kRcO(b z9RZDC#(`0+u5Q@w?As0eHpPA{;et&EH+(x{L#DtEL0Rd;SuGDsC;s?aHI*2`oE| zKeQsm-|h@V0f78?Kijze_E-1c>3$^vWmj-J|k5yY~KIrHJ&r zC=w~m*g^+)iREH!r)&wlYMIfveN=!hZP1s}rQRs_3_6G07{<+PfJS2?>v~S^=RgXqxvHrD29ad)3h+ zr|oRo>J1{#~vOO6JFoztrY|ZufQ=~en{qzdj1>2M&?vsIYa8Z zw8n;|lF5C?$7A@h1Xt4kH7YnT&X>kkZ$5t3Zx91EPN5#Q2)d@*r6fJK^(K(U2BMIG zt$?KWDN;RV99^AVlokRP)Mh>ypbnE*(uc4no|`7s1G_+2r1su=!WSpgr$NBFQ1bliaC zWTmlpuV;)esBkTFK&V)PHc8|_@ag1u0s;%km0u7#N^Zbys%0mp(4Wa9o1L_Pa&DQSJbhXT5i$n94s5&LKz0 zJ_oT{Au2(o!MX7n5Vb5W3k3;0HZ2?LO{K}7M!FEgM5CaxTyo%&^$0?zB7&Qbf>{j_^DkGIEd&L(Phj3$JkrIMe(%(qasL4cgND*UDBPRNJ%3|w{&;slF}du zN`rKFOG_;wA+Ri+clf{ke((M6J-;9BIx{Y zuxurSyzIBz>DD(;aRLo?F5o2~$bc!1fFI{Q!`dAwa@M39vX2O3ux^=-Yotmaw3Y}? zu9XoR!b?w2PT%SAoIz(J+$Tbn55S9Jksjrur56;FaT!2=fp1|>7twVgj993*)fYQ6 z%_9XuEx%6HA+X#ea6q1RyGIq9!8T2TDMwQN2|jd=f@wQKrHCcE@#WQYQMFa2M-i#^ zF(zkRV;76KmC(3Ph_G_=E1FZWjj2bJrjb=IwAZFpKRNMrwF@JbbZ$enTHXukK}W&F zd!N4xC@9*C+L4PPNMkFEn<-jgqX5Nx^8&|p`*EW5-eoIU*TmnXQA|hgVE^vuTE}aG z_@s;)%jt?bpr~U`ElhXyenb{k+_JbLqNHRfwV`W9X=UJnw8gj?Zk@$4V)}pJyC#Mb zIRNFwy!2BRq2llC-MPou>?wtcf2LU6M4*htO-)V5iB#jA^G)%P!)cYjGm_GBjF~IZ z?T24iW-a*S1Sd{Kf-<$NGvfM+X|YO*f*&eA+ne0mDG-nAvwpks0slBa(-M8h&8wy3Nc-cU_k$NM9& zPvqLk8hH4Tv-26G4NNS7-%r-;Q)EG(PfDvIR|t|N`=qbGSNl;{*Bn~1F*kB1?_AC} zF+iGAw5LS68e~9Ic#p0&j&lyuHZW2%%EW}fGyBC`ADf_O)a$BG+vArY0{&uwrs%?B z)moa^WlgcYecc<%VEyF}RXb<+qynXMGKQu)Hy!@=R>=+LDz7x`S~!dr+q6bW8O03~ z$M~I;rseZ~Br8iF&lvwfiA8l-tb55bhfl=OyW8tK4d6r!lB}B(cnSyM)ac#B&7GGf z{ocaX2PD1mS+kYALA^wjD1f|zP)2j|El6n}JcH|e64u+#f&E@G-8`-a9j}zvi1HUP z#`Av2$Qcn^6caNF%E5{e`6VF_~uqK*AE?`ucsp#QPP?Rwgo{;yq_tNC4edt*Igxg*}?u zr4!^662|*9$)8blMc$s$DSL_b@3KB+bSWr|)AZU|_9Iz5iCLmYN#4=7#-v|!dJy0| zd{F7|VS`Gp*r)L`eqYlr@)e)w5{&1bCmHIdS!H*j{bIVbAYMC@mKn#TM9 zwCxa)1Gjy2v^otN@u&XM2$^SGRxQ(-W^SVq&HI*_`HpU1|G4_zpTOZ}o(epJX2=#~42>HSt%tzldsC_SL z33wrda>og=!+yleu~R*ZM|ws$q!*f!zP*9#kae`{V~CV!ji-fHhAev({yh1#JA;sB*2I6<)^IUk50UXhxXiAE4E>O)UEj?8E_K{J07vuY_hM2LJGZ&P7)$)D6 z{#_57!3T=X2G8Mib{dh@KK*VCF>*H7mui6`ag{n8$L3AvG-JWkz*}heEBc9jSwZ8R zWA$Sy&#H=O(%6_gd%Ssgj+o)mPmjDar{hTC@|xwYc`6j+aJl*=@GLCr(&zfwOPc#| z)s6Nq&Sr9-kJL5CW``7!a&Hz^Pb_sbidG;&KHVQOBIv+l*0$083K^m_HFY8I2HER<9>E%BZ8)F$-DCY?uTM zD1VZX@bc%kYXib0Wb|3hTBxkiua+!QF6xMVG9k~*d_C&msSB5nSF32PHGg=0 zac2b0#Igmy#lBJJqeTgoYx?HFqR)Ot6)E}XX0&!fQ*F1b>^p%F2bpxdL;5J;nr0%e zK*P&7xY))~`>2IBQ;VSuZ%4M=7WlNnbJ~Bv7DW;85&i~dU?l~|{d6?feR#X2khH%* z4;-VJ^^a8*xp1nr!WnQP`qR<1n(pCL@0$`K^S&~=aJhX1jpO+92XjQLFL`LUjj-T1 zew__%;d^9Gj@d>S9Zg^}!XGEfGSV3Jw@;V0_r}%6j*bhfT=3QS56EH6W7494owBG= zm?R-n9D7p273^8GBG9@ zD2j|E=02u4>YwN%*$Gi&UH<%5^K)z^e!7BY#` z4=%5Im(0Zy>L-S^;1OH}3GO`_g!IICKY_ z=KpQ-oIIxgVwrzRgTG@A@Saru;vsYh0S~C_Bg1XOpMok|fMqg>?2GFq0zd`Q0t;_z zqYn(9=&lET?A^A8FE{L3fy z{2Mr*81}!uT4n^A{56qY&lwb(3$g;=g)S#D#CH?Wo3vow3w1cnSAG)tjtDf=WCQA) zcwM|hvG?;tu4H5bU3<;lEY-&aPK*&Cb3~$llX(Al_TQ2DNYr3 zY`i)U1Vn-F2P8gk4o)(hq?7m+^V|L$1M5Sxey~N;kI0HQncz);F8&)?0I2~*!v4q6 zzYxmnayc?vEG^#`{t@s2DEx$NXyiaR9`}WsuIvwH01z|2kZ54#kZv1PJ`` z9DeptuwPPA!Swe&fBPj+oBr|arCbR){eZV%ix=812_mvGATyVSAHG&QHq1z&^0>m# z`AFj?4+I61#j;mxa}4Y60oY5sc`wwMVX{eM03g=(j~cPl4gl!vKVSxg2>+8F%`@QG zDT_yRog6lik<@wD(-T6_IXQ8YBEsmKzSV{$r!RvY$pA~a*>%gUA@4&cN+2kRcIjLkmU z5LkW6XgCrtx!3%!Y4wc$7w7!Ptp7L7{qv{^Vn3VHVx`}y6nECRW2=3_w-9rp_lTCS zf`))Hoerx5V_jSTB^KNwdNrEF|5Ld97nuM30@HtE zqMjHb0^`x1`0CDP$EXTJTp2nL;8Ott(j?42*W|X}>wAyU^uWI6sp>3^VT^P?mY0BU1~3Ba?LMkcWFd)_lOfTC%| z<`?fz0ksi_otp6cY&O?)1Py5~7vMfcVUE0=caN-=?*V(`gXqvs+rS=Crm3ZK2-+>b zE{LGY+4;=8gu>hZ(u4mJ+!23STmQuRe^3Zd&okIIY@LCk z)9qT2FjGEKVGtet3C`GZ`W&^kJ=DVVrA}nw&I{^X|w8w8w>@~Xb+^+G5!I2HJ)nZH)L|5l4wp7;tj}ln4Y1vCR;PfGY#wx)g!!g^<86<||Wr zsBP7?Zu6Q^XtjC??=!kGM1uP+MP>BDvJzRZ4hTIa6Hdt1u10&dJPUQaZ>}rMO7-f@ z;fSw|MmwMJaP*BFTV1~uAsi50x$O*2vFgW77FzDzxS=MfG@4j?Wgj{1eU2x;-y?c; zh+E03NG-b9b(Ah}y5BDRaJb0Ko-5iJdVYd|DC@>3ItZ+fc#6;1^# zasoP>I4q2ClgiLozJd2c+M+s%2iYgtCa|SbF(*WHL-tX@ix$}`;O8m%1uf|3p1;9<;gU;F0G&agr)cc) z%!w8^WyRB|_A~1C+H+A=WrAewG==%pw!>qk&BY17ApAZi8R=SHii6VjdQ6_47!QlT z!qOsH1wY)fiz*G0aDkAwrfW&+S5kK_gLCveZ&R*FCz?ZaXJ;GQ>SGc4sdS0~E5s`l z5dODMe}uJGsFe(Y@BUmU%j90!iMRQ2iF*WGRrybJMH9Yl{g&d>2uB@83uS!^<*Z=g zjzF|z6u&d%KU|S|z3re$2M<5W8C+hXeJJ>AY`_oF_|`BfL_{ZM?RhLV?c?uH z!KHBm>VQFuL0)|tiV??;_i*gnyQdoUS;_hroLRRB6FxWMHEry7g={eb^I8%XPf08@ z0LLE5V6K7AQBM1441AA4pBDq)fMi^tQoW>1(Ov-1e2SsY*wt-MSEB(fh4kO-Nw4p9K9 z2p&$lUx2I}K+pxsv<*I567tQhonPn#;zN%Yvsy{5rr!j6(H;+;rNdxvv*t&t%)X}h z-&BaUw>%5&xgtl3@B00-jm#c*!soh{bv%}VS66Wz=`s4=%4Nvcl+F@Sn;rGbo0CD` z?Iv2#KoM8)6&5H3z&k_ODF3YqyAxuE(eL`j3U9Qbb%3N4$3Vh zDRx_>o@~$iXZf29Ui;dYNyH3yw5sR=2KjW)0TM~(;Som$Oju&eJFK3jJ|I>0`3Q@F z9TPTs>>ugPZHSd?(tX@Vw7YraYY@%4nU&|AFzZS9;H`B{>EE7=+kEYF+V0A3zb5Cg zMUnfsA{>RYiL8F6?A@MJ~i!ghS*0%Hb619d*)D za1|c4ShL|~B#}6DEM6PbS8-f;->BA{;AfA<>F|dc$65OXtB-i7^xi?hx!w(* zPA@crYv6D9h1_$E7RVH0KlM}j3_atodI9rvH{vo%$0pm`TFBRyN%zmTn&SYmMEo+`Av?6Lhe2+)mFV!mWb0ypm21)MWTv9+amP7u1G|1>>=hM*K-l%Ox%JB5 z!Q9yXg|UHgAhC?7+(W^wUL%OqG8}o1q9DI87`#naFJhgsXpBJ zX~<{5cq5ESC|54#lyNUze0Q+slay_aiBQ_iYc$d))dL#+e13*ebp!LdSd7Fd7;dj97Hw285OmJyY~?tayeaPe(LE|iy}4-sWLd+XvT#6R4*`+Lvo;+D4Yyr% zydWB3mXQCDWMUa$Vu?^1E>!P=`-?apx_E9oc-kCsRoc{W>EP97iP>lTC|X?i{l^Yx zGR!~JAG&3lv>C#)&^tM%9g}H7{|fReThxc_v3tPN{T^o%?GNXd6<>_jySIf22&(AD{!&ddVgd_4 zV9$);}VSiNyR*?tU%N17hGzZ2#zIasKYjV)O{9Oqn&$injsle=Cr+1Ln3Km}`Q%^kfp{Ekxg&mjW##rl zUo2Ub9axo71>{N94@7M8pyp#D{egn41=n`VFVY9?~D< zYMty&nJMo?{cBpt8sFdDe28;JYI-yi|Fo19;Y4r1DA>I|iI9&&MV*s${d0@ni<62@ zEGs1hGA)xK&9F6(O@+RTXLd{f)7@T|_6Q_z(W6t@xEJOCSZ0|?_==4rZ70pQ%Ts%^t7{${mZSNE%bHT3O6*l^_%WF8i9OFM}h^a74#_fD$;O=OMDIHtTu3WAUldE9r-LZrXPAWDd(1 zv|=L41YWr+?M^TVEC$0d?Ja^BR3=4!hckDg4?=RizbKwLMn@NIn&Mb35tC(w(NIc^ zHqlY%Oall8c||CyPS6Ya#jNH(!$WNLss&H&r&f-tz3TN;sAgrzQ@(5EvFBM0gc8UW zsEoN!IoBp1%6gfD=#qX1Mv__y&F+1(Q56;YquP7LVA%AE7X78#`o!g za4Qh7YoneM-6}5q8kw=EC~z9j$;&(R(T8+be$(@6&xRYa((B1{$~d98>)ZjYH6_Xj zgy!PCV3?KH_eGM8b{PF^ACzq&R-M3lKU!CtD$eJ`xheOfH*X}DtQSsV3MH%<9Zb5@LK zTZ{pZDbRd2Tvo)|?l-(;`B$$ETm*-33W?YfW%=wSTs=S)7w{Q*<-~;>jvZFjYvrmyI1$_I322s`y0c z4cw7>gkx=S==G(jl5SGCP_E3Eo!6VqE5q52XyKRj5dFrWbxG1#_|8#uxr~w{>bEJ< zyuy9&)k)ddyhba=si{}TB*??Co1^I(U7aIZOZ%c5jv zm~D!Qgx#nFEIMZ$G8e{W`n(E(ULI+WGK-px#G-#q`e01$lyEySHB+)C3p0<{LCb)o zBF`^v=lD*nni5g69sG~I02~lB%8?ya#25kNo&JbC^-2R3Q!YU*v6L@djV|JkUK2vskV{cZi>IZX`GuaNK`#2LFWWu+wMKhDJ)~u~t)i#uFA)7) z{>V|>oc&q3U*B!|ab&jA;)eAC4_!kKi;;?%`Aykil4Kxp2@Ib%C?Fz=TOCVBhoCIL zIv{HJIX*s^p{$HmgP@!dcB-NpqJppHyd?=1*yF|l}U04_@6Z{%o$Fovy2;+yaVrSCBmm{wwk^_P?Pf}jr78R!^aPl)T0Tb z=IMLB*upA>^_ewF#(66{>8sId! z5@ZpdzG)JavtkA6l`uex!fHUKG|JB{0;5jzicgnn>fA~_oo#|l^+<49OEwQd_05R; z&IPTkO58d0iAjzz8>PgbAP!}+sGwUj;gjx(Ib2zU{ z4zl3%9R8YBSLP2R(3I44s#40&C>7-Co9aPQ?0=qC)a*3jj#%R&s>dWPFGjo_^;T3_ zn1MNvrdrh^aQG+9B=5H*g%HrqAJN{!MIW7s=dhW3yS|mmWPua)0U+9+t z*f~PODy&L|^r6E^2#a9|QyBO|OE*3`omM2VLd` zh*L7(XM@Q2_KXsildRbPrn<1gmK31-cEYP@#m~`= z?_ochj&G5OTP@fkBk*9_A*mnKD1;=1x>#-+k4%fIBV~LUe(=}BitqDwAOjfRJ$jte zs@KLHxhG`myGIuxKnI;bmY_NO!y{sPT+9rMKmg;`Mh#$mws_bXPwC=9_GTv!$8?($ z<=8NIaU>*_$w<&Sz%=^}y8ZIdTL;?2JqB>>z+ka<8LMJ~Z47QRGV|$9c`G@9@ofhz ze5}+t3lY6#@01^pr}2UjA#1D)642_Xhk4Ne?lh2$XE1UEJ7 zgoTD44sEJ3j{0~39J51eRMhZtag8LMhPhPa-cSOSHGLLNGmRR3Iw-9Sn~|AYT~kj3 z=!L!fc=xz7RL-ya)j)A%U9O*=5?)po%)kuXVfcBffLsiSsP%&O?jl@wy?o%R*LZat zJ|6v4Swvn@zfdQG&NySe_j`{@Q2}e@H{St_{&%>%7X05VwyE2%$|bgb|9&@DWiZyz zW~-PMof3^}Ab!n?q-51hBi0_Q+9p<%q6j<*RR-q_qx${H4`awWm^jRoPR(F&=0|&$ zCxTL+9_qC+nZ&0Iv>{|=)5E3a}mgU?a$ z0#{=jC+~X+J+G#0fA1&1U?#8vf0nDHme=x<<(4%Q?FzvOqZ@uvK0=4`Jdtjyj~iE*bKmU8NB}00 zwh|r}iHsUP&;$gH`BM0zkDW+f>oK)E`k}0B-@8~Xr!*gSX?6l>fU^z7UK+fjlWibtZmZj_(Ole+6CvfZoo_g8%KN>hEGxE#b__D$%c5{~{x@N+w z%qDKQbSvLU%d-#vEU1y(w$Z@W)Kuhmt2KdO`9mjrTAS|LSg`}0{W2F-GF52QJ{w@q z{+tQ1mh`*9ktzazluMz5BDf9lMrH~M$bT19cDkUMpb+W>dxp>{RxP*X9OLn(VQaL2+IH-R_b{Z@f!@(P$Fn`e^DeEf;_6lo zVe?7Dyzfs~Yimu(IV!Z+378nT%?q%RU+94fMscz5ED~8Y6VsWfh6`T#q1y26*M87R zi-3aMz6+L@IIX%IP-wNj%OJ!8ml8!Lsg)>}>@S|LVooN41I`FilY0; zscwQul2JN1QQ~|)_&k+rY52l;X=G?(t*bGQXoK=2DLpfwWDKaOK0mKfkVybBUspmFdCYp;}3c#2QHi*(!?E3OemK^3Bd*Ej<;eS*Tt~ zXo(6czJ8k1^1uTMDW+(b;iYd@huDjVySNMEh*UIh+Kijw}kL(5IHu zl0<)V2A+Pn$!B=m{unf8ivhhCIocWL5F1N`S@@m%vRKFTegK?CZwGRQF7@))QW}7q zDUcDVcvA;04_1dwJCY(PwXJ+Bc#*2pkZVoT|K(R>O_Czr8nmQYJ{fR@Y63^3 zq7i{s1{-ck<{%N(=TTgIMj%vl>D=nb)^Qr|VD6^QL5#NId&4zO%lw>M{cram(gT$h zwamMWxP@PJi(xuCHszeReMl;2-IGBgy=)JEog-xQHS0j)#I`PuYy%0?C#TwfkJyM& z_Bvc&3zVcN{N~Tb+#lao0xxg@t5Sn2_e$bspp{w;kA8uUow=K_$dp2?IkPnL)KD6! zgNc`ISRU^{bTd)n?-!P+rkS!XF0Z4mi4~qloraaSI@L(-Fg=IfNL?Hmb)|P)`o@(W zlz*30vG^b}t}>>ji_=#y#J|GvZP5CSMz5A<#PsQx{YwNW>D^^>o+ zd4Ss%EmHs{Yc>U&epgXN@mxGaPlq3}&?@TwgdWGu!!q~X9I)~dN|CK4X>w@E9n9`? zv-H;VdqEGe!`v&DwCL&4MZ z6=;ssJ5N0j6?haV>|(-SS>P<|W|^r)qsC0kpgk``q^7|b1yIhgN~*HXi$X5$NIa;k zh_}Rvavgm^>k<}a^Pk4IPV2<&=VK11sSKE$;9=-SPB7?dQKyQ;BC?G2_q*o!m+Fsn zYyMj({mKDBfKd6&03M5WYpql&e_N0}niMIkSX5JF$)12&QO3%|DW)eaNvA!pA}Y|+ zf8b!@hE`2IbQFIlEWI4(xaNO{V=Q{Z$f1IC7u`)PoxwGo%SPr!@RxK+E%Vkro#G zZSX10B;!>S6lhh1rR;Fm@e5!@%MS53O!0~gfk(VFY*JxlVvdPg*fKfFg-&dTmZgvuXi2ej0!m^7p}t zrTH{I5E*o7sQI1rVnYkqOLOn`;l0Fm8BCLejP(Wh*9tEf>X=ATN5KGXfFmkE`3aG8(#QUJI&wnSF z|NX(L$xAg~8}kQpPj+>eLzcm8LZj^MOa8sKR8YSvUyF@m8_ZA<5HN@XIE1OPQBkbe znwr>f*qYc_*qQ{9fEzX_+GkE{*5g}*DKXZyh{`<2tU~6Q89a*nL<;{u#{R(Ev=bCB zUd$zEnygHN&FQ9mpgG}3+Mfy%O%$}+q919sdpG*myN^Boef2{$gzWg`q(oMA)Wfv# zlqE!S8u|S4<(#U542g42bMu6r59?0sZ)_QL#yfhGIdUyq=>LJM|L@Ny51Lo$xW;YP zLJihCQ(8V6IlMAT7bpHmV>c33uHMI$Tfmh}r&p!%8n0VPp{Ok!nrMr^nZsCG?6bRoxy7>94VuGPQ#?C`kbvE4t%N}5OP^?Bi4_0g zC(OrfiAP`iJ$1(&gFx2-DziT??|wt!w^;aebcU}Lhh0o(kh5CMvicvrrzley4`ro5QLAH-dZjFzW*iLXH zIG(Ij7>Tj}lU`$a?qW|otXn2D^Oap(?F8j^w+g9k+Rx;VAI+Mo!=)$4-ZwmFu2;&1 zj~9D|qQS-A+w$5aTf`@UhliSxTb|3fNb75tLFyNjn*m*?`|1aA|C6a-Mf40zDHZ<$ zoI(EomL&i4C=FFX0gfVHt;M`94ad=LevbtK6S$wNYOi_J1$eRYi;)94_nTFYr_+ zS0!Zp?_T7JyXSr>&|;ZzJ{hvIJ}^>#MILC5M+*iUDt2+KwTmA<|!(O~5a*Xc>H_9cYaRzn0H2mFU_-+(f* zv*{}%{|`DF#{YLv{p~yIbKrAIg<<|zUCvNxN59#?NWlA${&CGP(s9S%gG@?%8s)E1 z$$aiKNksUk)+ma-t`V1PXGGnuyt*@yRdT%ocZ~(5@VOMGg*YYCflg)6Csk}~QDQYb zn_|Nh|I+v4f)j2BY{D-OGsaxd!#(YPKr2jU~|KwiGnH6MBA?S%Ut zCVarnfDz2-d@}@@-3+*U1a&W5WD$Gb9jrW_XrHw@kVEwC*Y_kIF>X7E%_@b4w$pPh zgfUNb2PZ%dDp>hIi1MT0l`KV*gG74RQW-uw^lr!{1*M6kVA0)u^$z>$!cY8u4+wNj zj^L**&HyiUbnjPj`RxV&xEj5`y>`Cly=QZZ;$aYSuZksI^I&^fj_kEHmh=KULvYxm z{hBHraepvRs6SBq^~J-|nCpwXlz?$Jfp32n3`Nh%C%Bq@iHQNTQ&AwU#@<3?$~vg3 z3$FdXhY~vJVZ&-%o0#9lcev^lh}3cQBHK`%7HxP`_ja+_ETHOK;5&4ku<7!mn%l)r zfNRHdZ)j%L^zK*U8zOU}U*95aF&^kHSI#+qQLgk{jgzzbL#wYw(=!*R^9mgI^w^8P zuqJq&a(mfsF${`&;a_t>&?{Gm8*q)+dGgw$7y0ZovbTF{t%hnO{c?**t2e|bg@*_! z{1R?q99ns?n$4a`rN*tj{g?OLzclqYz0Q7!x_{3#KxJ%Kuyf_h(!ER^rEiArsQNr~ zRYMmYaYvRnIp35JbWZO7axKbQbQ;lpILlhw&Gok2vx`a26CLX<#+l+hraGMos)}7{ zIMzGzJd5@@KSgY*O_1=9!0obGt#x|=&X7#1N)&(Baf#<4u`*m!Mctz7Q1A#|ZiXFU zr17PK{q{iVE_@_Sul#Zc)w!of!+e)Lw{VR&vM>#L!ab_DO0on_uaS<|Yd7T58v9JD zRM-OYfByK86x3hgNJ66`Nay-!sa@Q1akLFfSP$g*Ny!{YgxP%HE4~|f0Mi$ zuW2V!-YTVgH~-z~%GL)8jc(nY{}h2()49TUd+~KUdhJ%YF@1t_i6q=6@*V5&;1}{b zy5$8w7~;cWVE96Ze^k{CQ7tY97oR5m^x#^K0_E|oyvWto{hHaM;qLE`YB*{+yMrzh zJT=Gn=)KPgR60&*{mu%CX0RD^ENa{$amGG`37`cDMg-(DH6;M6Yjmc z6-V4ZSnz=v`CV;OIu4M(?ib-nff0H_T?2`Z(WQ>n9yW4&R&EE=n_-(aOVcg};=UPZ z=lWNj{Za*RFz&P1%jDa;!j=0r^bzmy-^)!oI}<*sKQ63@Y#5#QdYvq5CVMCH-Wsxy z^4sr}tKTQw=21ZIZRHmlylF3eAQ(qc4*B30`1ELh3>rO%x3!V3rl7<)dk8x0PU16N z7i04Ys&(J|2Kd=mTsZu1<`o-W7QNMA$u6+?6;5os<)*Ma{I#<6c*f_l9QB996?F}_ z`hH9Lce8mWpwK|Y$~Jwbp2mjr_hhm8*rda^T}U3}n=F@pVGbz`lO*BYiww;g(aGky zcVE$*4G&xFN&U|sSJmvQBI)1?1y=iScH8?sIv!rD#k=CN?Ff0@3HIFDTgqw^6za`= z{MGMbe21RbToL-y4lad3&`}(5FbTQBzBSb>&o{$i)r*kUmrVxP-0?98dJb{zMI-xk z=B9WHE-O=^* z<>F%}Yuh5;Hmp?8M|{NFZyRw5o=RX?Rp9G}%i0aEu@j}35ToIzG(s40)nMJrZ*CL1 zM-MlKE2*tA`k-gb2ALmBj${Fw2Y%D{km&tdkIFyScxAXpa z>b?-68e1@fn=ecA#*O09{my(7%Yp)rhNxFcVPxuxD@=~$d0BlWB#mutCkxXb>}^?H z%tgU{JzZC>aquToO}(ZsFej4o%PzM%;!Od_1(;o#UFm^NH{|E2O;7LqtOI zsy)3HodNQ_jg7Fz{k!<9M*l0TmD?*9b790i8wDD)VY1sDhU-J^8XG+%&DE$8pIgjF zkB7#>Sf9fd&DErCucKRFpW3B)RDJMQpltO0@0mv)%15IVyC1rml-q2Aea3f3{u9B5 z^G+jtv3CVbsY(W|o{3de>{TVN@+Irv(NV<1mI`>!V{zJivSBmSCPYcrzb$ribI$mC z-(rDkT;1KKu<{BuP$T)yUH&Y+K0C+co?Mn<#=)B4<*f7LVfj?3qBg35<{Wdp)KfpQ z9KkgaVBQc}wJIf?#f|V>Dn&(_R2u%i&b)w?6Q{JQJh*)ogBu#~mRs5O63Z9ZutS?P z$UiwZ^=unej@0^~l7XP2u(2X=&>)mEv$&!@@ncow;M!*0KlTCyPHA#Wfqn8-h?GWt zqo$qn;&M^_npHIRz%}7gZ^lVFFdS=e8~L#J+w5CovTMwI>clP}?8;jCBV{17jc(Ht z3$pwa+x1(*={!rX<(=HqIMKP%Ow7;umv}y7bGxyf#hy)ND^mesrjq=e)poddHCzg_ zJi-ggqO2O_I>=^!J~rFa2CLfrb)70!B)zrup6R#h;E&u&qN#;#RuM-Hh^BC`=ceVt z!Z)|S|CBM{j7^GgOE&0!)uCTHtuav`I@!L}^Vke2|rsLzgvndh^#0Zf}3@ zmC<%5$$Uv{ctE82PWCHjjF-QDYH|75Hg~E`Ox09srIFC^dw&Tr)ekYCGTS}N&mDO; z2+`=KTO>=rDfjcKaT!RIr)THJ-aC-)d4{}@VpRmXcER5p78PRSEYg3vz1_1{GU1gn zR2B%n_noDCYYvjD`kwlbPeH=%gtI62=|&{-wKCoa_0$c!?uEIa>lruX$HipwhFhu< zUBMLNlyyoTU7DSVxeuKfL6Zg}P8dKi+>U?5)9U;65^RXySC%H8&f*nwf|6{^dR7f% z>PxXXap-0y<<$Kp#ufp3czQNw@O|6XCTmo$HgEm;^zxKStM29kCS}O?tVMB+30La) zA%eNXro{UDm%tA~%&WldfqpF{I9OlJ9DPzsJaZRcDG-I3FAeC4tjfiBNrkkOU@UQK zRQ*3>ePvi&%hGic2=4Cg?(XjH?(XjH?gSXz0tEM9!F_NG?hxD^zPWPlJ>PwQ%(I7? zrgux#s+!uXtJFrDZf8|8i*3!)Hp*~R=g9k*O0rMQhZl)aCSn(uxn|`_=u{9yD;F#{ zdORWf@j+$9?q`#tNo(yL91>J2#CUQKY@#V<**pOJY z!k*elJ2?!jfl>ArR<3DzQ63pOkeGOeowxYx2#3k}!@iOFvzenpfu6fqTb0*324&d6 zq1?qG+@ZkUsL>D;)Bjjs8Lo&-oNEXm9p`NvS=Fpspu&hPnX|6RCXz8-A~KDKwy?0E z3MX#%LbYwVFz;7F4W^T6ZB_PrCsQ?+lFDxOy=UIYGZJob$37P=>8-zQ$}9Ti-K0-p_CvwS6usQ>#?UKvzFxhUgU_?#xxl5zB-BF zKm2yu8#2BZNN1DSMafw1^7wG@ZmRh)F6G3j*Yl!-+DOQ58<~fJA#}@xo$w%z>nVYGGF7ZbF+OT_%i7yX6joHB5g}dpkF1j~(TN zSmvarEs5|MP27DUQY50TOGt0)lVo24=(39{ zsHJ5eTr?8yQxodRQcC=dz7+zgGAUs}YK3)SvYLTB8!VJo_OAR}g}BrOq~*7UzN{yb zyho?_B~{##Y+LYI{{qY*X|}A=YR90B8VHkDt)lL9V41>CNeS%p3kuXwBAwx_F<})s z)>Om_6icIhg_a-ZfF8v;Q0oFQw63*)xdP&*PL?ak36E68M- zJP~d(w!usGrRk+?VjRV!IEgmDZItjoc<%b9cI!w<6FmjRt~41C zcFqcP=g`x>937n(*H#Bxu?fpwAdz-amoXkzgb^G6$d%U}8W9IWfxiN!tcq;24x?rm2_#>~4S7WhqhN@pTZVFOcFB4)l~^(?{fY1kGlQgp=8njk zXDZew660@XPP-I~aGo>VOj4e@2~^WObuk-d;NgD3OmNW)o(-?r>GPCCS!2i*dV4!+ z$=s%JKQp4vH_|s*=b*d#ulDBM_qCh}eW?7eipqCOvSapF@R8^6>y>IHqei#L= z{D=1i8()FN+c`sqJWP#|*I{kk$Ulr0$G4`CU$s~bred)qaMEFs{-n6PH9s$dPC~^E zE`EBGLRD3~kBLmqt6tQnj%=b^#5fMCD!U#M(SkL9=+jY@gh z{c>J)w%(D+dD0gS3*d2&S*X$kd3Z;MD?Y0|#2SJV5r`kE?+dktil!mn-ra8(fsui} z+ZqBdJpoa8j)UUe?~LzXKO}cV{Z0%w`*dBNus#k!XQQr^P8@Gm0$UQb#5O=DEF%La z@;jR%18-^r8UU5Id*zuKXB5az+1_r?C z>FGOTHQU{eQQGwqwCIL99-%=l1~<2~tSs!sP9=};J>A*a*(*s0sKtt1h8G`@u@dCT zmZ}ui)FgObwE*~iGw$y_0%p{!)F~(^tgNlK^yDmsBdWL(@p(Ozwa)wP zRH>*u3$wU{Z&cbmA$cDTb}y;5`gf(}6MDa2KRoK=jjW8GcZ%Q<2*zy4Y006C))OD0 zD@^;P9bQ;WzOSb}-G&X3C8OcTD3?2hZHjJh;G11c4(~6a;w1~+?MdCjSne^5nhq!6 z!HnyDFEc9IMTvrGG*Lo^0{`GR+)(%HsQmUUo%|hUYq@tK3s>51E>%Lkno7+iJKrBS zv(6+n@v({N_K|wIJz+B=bFS1;k;DxO95X3YH_+l8L2zNWNQrCjmrl2KL+11PiwdVs zq7V@h6!@3me_R-Bd}v4(q*Ou|zi~|L&Rxk`-AQqNc_bBVAE#h_zVp zy)%t9uVE#h)@p@f}#=dZPsu@&>FoZKe>8g z7b=v@$Ni2;81?rB^zosg@|S|i^mOb^Q{L5ba;N-B=2}_g-udxcOXE0)`8M1)ipf{| zt>4n``C039Cec}2f=YI%r_UGm+S<2YbeafiZ>z2QYEuZWS7uew>mT->k1Eil^(#Z z$!aDA^>uH5-=01vbaGcj1rwrbl^p>p4Z_HX*xNI< zdzF-#DQSLix|{>{@g8+_(?_!i;SKf6?3vHq*Qa4D#rQ^)%E%CSMdKl@MYqop8%VD|FIhg8bE{o2+M;R7jhe?#pbi+AnfAL%+(Mm(@opwf? zVFrd(TCr7POCF0pG_-Fyd(d4+dei&TR)! zmzv0dHX_2L93tj-e_FK~1R{r=sgJk?vrPsF&_^oCZ@zW8VXwN}r{?$uUGmhQ(gt&( zMuI18G?8+y`_Zdc^IZCV$4MGbI>c-oT|r+e61QXL+})u4jY4Q6Axuhbu$`&9u{{j` zGHL~^@Y`Cb(ke!&Fj!X*5~8r@!DYsZhXStjhC(u9!I4o=ko-r@|I&6R+ohVpZ6_oo zq+gqb$56)QuzaJrG) zmaj2a8mFJYy209ff)gt%nf>Xm`McRctO>oheTEHQU47;lgmC`?u)i)LFhX-|?+pCt z0PcGBOwRY1i!%Ar>X||2h)=jwy32;(3#WM7cRWb@kc{n$mD(dX{Z;6A=?D$>!wuF7LE ze&fc$s-Xvn11A%DaCWp&4Hg{?$(>{vhxt!K^Z!lyH8>AOk(i~KI+G)(Kh>1NSo#pk z|COB2ZCK^pd!L)(&58;kUq;H!o#(DA)xh2c$>@6a?3q`K!;cP(r#oKKk-SCbOK>=W z8iFeHRA-9u(&<)(=f%!AB{LquWuCDh22KcU& zgn~XnF0By)za;J=?6I!3INzlX*GHErYxj>|L%H~ous~t{7ENdSIX1fgmCG-|TZp%6 zY=y^qtF??39tuq1v!~a#SSHea$w?8({cdU}&WyF<3kSoSElC%Ps^rD!_Sxo0HU~)j zqw>#}N3JsyFYc%au5ekCxF(|pgDR%%2})loR$qwUGk?eBXZ)}XjD}U!Qpahvld0ZQ z0*;`Gq~3QCm2sD|AQ?78T(R3+j?@|10H4xT^r6^UbmvC8Bd=s{qG9)+TqiVFATKwH?JN(4swDrmN{*;)M zH1q?JATbgem4eB12oC%s=JwXjp=}#NL{zl>8yY5NqxBpG3oGjrr~PURNZ+Lu6b|C< z>9=QQ^6P5r0CQ$~Ls7!|(NHW-^8^B&ch(p_{z=o>!{^7M4U=(OY~(Aj*bi5cbems= z>lcna=k(d+*;H5B@97TL5e3Z$?@173ZI?2IH9g2CwJj-a_D+aBax~)s@C-Pp}FZf$t=az>QI6`5HptmPS3^!Nb{Jmu^I37^;aa`~FX1 zPUT67@B(jrx+f2`i}$2SarXK z@kZC;jhZ5ypX?Q9+kHM^l0Nj}Q3PJ1ZPtG6qE2$~jN59+{tN}oIk;hZ=vsX|)l9Gw zIO2w2PZV()s8p|tijODf;{*KLxkvKp$|K+Zdshgd~C7TWAMv}9E@0D3eA-1&|aw)Ywd10FUr-0;Z&q!msG2{Z`R9> zJNVJ}&Qoi+Zd)va{Cv~ba}tVpAHb%Za@gu5;0K)}axJ`ne7kbybX5;*Z@cq8i|K)e z%|lPD19C9hkLQ>}iF@;@H&v|PJSe?^{ayLmf_B7rm8-{S_N7jfUItkH@Z1|$d}{&; ztk*9in@stfn=jvH<8ZP0c|uJ+=%3@1QGYuvp*K&z$ZB=i;U+%_^z&yfh+I&aJ-;O0 zjKB&Q_+3J%h{Wj~jV;OVSFtyHo89JyeYWlPPw_n`=+j5!lc;@G=DL3!YT!4&?XqM; zjz9@`JGYNO#P7XaA}Cb;k={7d`84r-DLNF$l9gopdwVyr^EB%jCozyKXV0Vl_TFjx zs$KizO)BqcWSI0Ko`N_Ko-qz*Fg1_TyDZRuf~MSg&)&kmd#qRyq1?P7y6<$b^X(M1 zb0;w5#dmSIgyhwMqvkcfVOfEX*@q|(EC-A84+8;hKwNUG+pvSmcR0Q$*{o)Uybj64 zAt*Y(4#vc_o_J$L0qK4!{O$&QIj>!E3>+S&*Kp#!+PtwUQo~>9x8RyfiMvHx^oN;= zj;5!x3VZ@Td6PRZon9_^uO2rJGFqIm)WHvbX^zE^SIt=7?qzi+bGc+(1HTjV8C;_< z>TjYHCyezUTJ+yyV!sDRN^TI!i@fVdBoGXyI>1gDEFm?<6u3+SzWTB3ICzjF?pMsf zQ@`wk5qO;wzaqkwt5zeq!C)eKLT`3@(b9%q>_yMWD{BOf+E2mtK5WJ6X)wa&hHT+^ zk3C+k_xd#)!$icNi+O=6_4+((bmq0T6}rQLfJU&mJVXiYYwTdipouV1+ouYNkkEL5 z-CYRw=^RU2y(|BhyDf%gu|>*V>7E^Ahl{kh%1`$Sa3W3+IhI|MCfQ6yvZh5pj82v_ zpKl2F8{4Z!u%_Ws6UIi-WHL``McH$vU8B0NLu+h?6A=|Hm1o?W5R0!>?!dJ}xwX%) z9iQxV`7}3IGo(`coX^2t`5f?ac2c13NVpKotauEOCz%5hs#WvvI3#-IASD&Jc5Fj~!{vxjS=~ zIt~AVl#Yeu>|-u_eAykN@mPc17=YlEf;2stxD%QaZm&&Zy>>S=@}R-n5gEQ%WVy?S zfR*E6agvt znsk7TIL9QCxK(=`@_nh2bolE+skIMjo`#5 zSh(rx$ke*d>n6`e^c@Wbv!}OQ*OujZ=6+zccXbD+=9@d1p<w8XfMq zFKvoffF35w*_NXpJeY;tR?gMjs9aZe7!i6eH}&cze(`We8*3Hf!fDW(Ohn6Dh-ITd zq4n9z2-F<$qof-6m)^hvJJeJGLm|6z*$NzsPp_w&O)=G_^=wtCQI$lGtlo6l_34ig z_R>u*VJigE=>t+3wW|rkC*`xV$4uGsN8G(AXwe%+!cH`xqDh-i*+Df+Dx8ZW&;$*b z0lkv8qV4WRo+(JLqA*EVCgPVLlG{-ONVJxX)3M07s< z=)ovxLE)%wKB&RHGl2LaDk>Q!;$R-&(8VIG&wWXG-75JzK0g7e%)N?dwy{ zN@qEE?`DaURwqA{y7G5E&^>?VonE75YbNe|i^7?Vy1lR%9r6QD7bA|Pc$cx-&6|hX zhXPr_v_I@48!hpDDx8>!p|33N(7|^Mb7$g2T!uK}v}KKTe5n}Kl}0;AwV(>{0mik| zaiM|<1&;X}x99HUo?RTtZ>0v{dAUtcvu&uc#-b_%-;vmSDZ(rHr^H;;?whN4= ze7axUIGQ^QWX`;3(Up&T_3c(wgi0o3golv>@*&ITgsnFa%H;79XPfZlcSX5KC#Qhi zgslAK5Sn@`c}lT)s@^O2CvHQGi2J{T^`lhK8KeR@y*pYPr_e&yP4?QN$Y%X;UOdl+ z2D7XK1RXL_CB3n6F%*nuGn}419VLQ<#X*6s;a!hyC$16g57NyyC8_fB$$H463b|JU zPJ~FXNK;tD1b~cGWFDI|wAhXBO1m+23_Dt^^gb27va|XO?qW)=?BF2-iSMw0OmR^i zEzB@haaD>_-M7w8zfLG%`oZuSsQ0u6_fb2Fx3bxCNeYQvkEt+4%8Bvc=AX+bH{a( z6vmU2yF%97I3cdvG)B4}1i)`rb^xP<8X385;cFO}E8@q$>eV6zi)j`||KS3V z`jvuGL~-X7l=zf{;UKGNbQFZJ11zQ}x)?8WeoY|T=u(J3gP;f6~d{vGm4ySUiW_u*f< zKT)ua#4a~^>QXQb-lOQbzlxPEZvWQ(iHd4=5QlJX=9(44O`RZ=aLpaNQg065K8RP; zPhb!}{|EUyo=Qt%JE7pz{Y9wKt3e#`Op{E!$j!5Gt%~y_)mnWXqU738<)f{4p*j2n zOr+Ox{gLdDA5SR118s%8?`*BI{SOJ?um0flRP4769UuM|j;v(Jm5~Y=kD9I2AGM>M zc4+joeOf%Bu&HI+58l!Uv3)iyE&zo@<+)2xa%S$vqUv$4w`ks{fubGPKNTvC%*wqq z*$`Av2-K_nt=bnXcZc@i5fQT%mv{5B@RFq{SH9YUzI2R!4d9ovO-n`)C6p#gYSV0; z2I14cl)lK}mCxk}Id9*oTs`?ez6!9I^mB&;gElUPFK!Bnxikk?pB}p&wCYG8b=~OB zARmeqLg2$?#K3x{WL>PSZze8XP?9o)q*C*M`Q#(^Ohra)5vo)a@Ii81&cr4< z595bEZGy^(KPI-{v%JMyw)x(&jXfSVV0Mn1*2a@|eA@-AVqV^YwK6*#3xlJ7OY9my zU|l8b4a$JV163=grX8a!@t~}+H^j~TQnn0NbG(tl&njr{O&p@??H6!8jt3aB8T>(| z<^#kl>w3UZ?B*u$Z!Cm6R)+1pc-?c<0lt0_>|&?@LNj3a-yt+|49=cZa)f~&A-(b^ zXYsNAfp2IKIOyHT`tfLnMLPUCAq@Ue8U?|}y~EznFm!dvKB@ikwA*<(xSeSjupOIuzCYNCsXTD;rTCcX(RyQQjO1h&vAtE&z1#Z@dWdmTG<=)Z zd^a)G>v((h>Gx_5acB)W|JaB!go@6+<=(2cS-f{LEs6*0pQppe>!o$Tn4qG&GEi zJtz5?yL#_eSUwpK3h?0_^lp1>T{**uM^jJ*#Y#9xbC)yYFlSF5ApvN0)gch^g*6tr zB-R4)!y65Djt{#{YKJAwRh7>B_HUeFa$^Ljp!h>%$XZJPbKnq0izjW%dl;FnII&+F zg%d-z_&|R+%8Xwz1^?34ZmIn{oSjhdk@Q z4Sv1JMA>S#>{@G5Z6m7W>Nbe~VEwyqap{B;Ok%Q^4Js^a?86n_MhoscOV6fi3xs!2m{sR}IA zAb>Z`QgtAexac-mt)Lql=4#BwB_v0`n z50>`H)8WnShiTxfFSg@U+-yvxy}*~=eBRbWQp3;qg#_4F&@&(`S5m)SB!YWoFXj(M z(D!9Fey3Z+Xp@F!A>s-rCksEJ$3&&NBExX9u5{A zKdVgrorf-%xNQutty4guu=41KFcy*;#Al8^H;6@$fMov?0y%q_@Qvw8q0wtQEDESd zK`TZ=P7ZMxc4b1>G2>kEqhT|465^ML#}z?rmdv!6^dkAQaROb{bUlKxUuk_+tG(f9 zBGG>AuWa{I*YF4VgtI=H{6TSPCVSp^`NZjN9X0u5he3a|xGSHu&Cs#0F>ou^nwFWI zv+Tp(bGAy|{mUId=Fn_?iSW8?--W`0>{GzlO)JxTuIA%wBMgmQk$&jW!!}E_2cWXM%+qBBEp0 zHC`qkbJo0*&`q6V|CHb5!K$3$Pv}VQU;?lU3!|xnfz^pEzL65pZ>$W?)Rr0F$Gth} z)2r~A%BfsFCsE3xFL#gnVV|d+cyo{F;-#X5oKQshj_=&)R?y@z2dzwO`czOTsoij{v zZbE2Y`F^v3!pQzf@57b;LNK4;4^$8I=Z$00IWC8pb>Q10n3 z%Fp7bwC+lgpAc?-7)~fg2%S$z?X6KA-ATCdz~?K9I!XA1e$GYc6o?;_QFgHso6nFtCKc&OAe*aN5j2K)ZgD&oIzKu=##4=crn>&)-5fof zAmRFjiji^uC2`Z}z=Pj9es#JsBbu5KVK-Ns*9i>AV63&)_)odvrEYgRg}sjILR)C7 zMCZubWHSmTd-IwqpN)150c_YWD-TcLeq3!6fHL%6C96d(al0($pJgNf-8T$W{4WCu27oHL%bGnJPBMobd71 z+2iDwb`nyA(3mZ?QpiT_#=^M00W3Nng=uVPgcjq*mAVzXWjl@^!4|ibc|uj`M6sX0 z_U6x0t@otE>$dD%x`dg2;7};!i2CJHkm(}7VzP}P*je@;Os4sA_jw!Pr}pH_t5r*s zXmNvuSL!<+Hhg#K_P>M2uxH#0L%H^N;-<~du^-1|Ayq6jIMZVABhXNI!}E-NUvroB z+zGt}h}_hB4)ZLY()3Lc2XMCc(3|}+V9aGn{VW*HtHI)Wt>=Pg#0F{d*EsDsXfe7F za;bxiY>_?(3xh+h{%ibp9Js+2pa5fXff24n=vcTYd4mmDwz%&%j&02t?t$dthq*E& z-3Oj*vI+pKqBVoyWKIkvTly^&=@-9q6OLLvvAj^y&ICV<$$SxOC6wl3urcYiO=hsS zjc+#8;5Ct6O?_wzUgK}HcVTuHv!k_cC2*R)_dl=(n!mhjgbjXmk_)B1Fx!*Qje=v@ zpnv>9siMS_5e((ei$m=Hz-V(|>EPMYcDuh**(9v7439b7qYzKiwliqv-}Zu>xk z8!xYf@Q!@17#(B0^wfTkd`@Kb*@(jCd2K6aQ?4+3$v$3HDR*8D%4K-ki_29aLzPI3 zck?-2e9A5h+o9K+)D?}q+O5QkXn_V*D9=r+tXR7qh@D4VyyTG`8Qq8y00o0UG4+N( zP8~TGYGe;Ig)*d+C-I3Po|HowCGXae5{oFoJhal#UtLnM)^j2kL8x3m$ ztl{TX4XzbUmPF1-N1-X{%GUi21NxaUmn;E)l>CFt0viW5am7@Fhv`;xk}Yz}V=)Q- zo)ll)7BT^$eG?O?YLQypm#g<1Q|$Yr#AQ4KKIQ7R-E)O7SC2v7C6@C*MOIp^$nyDc z8-=~HVF)GiUr}2~f?roNnF40f5O{h|mY7yj+pV<>ZIrC45iX^8({B=c(+$+)B3i<+HQKJ-g&!L45ynSV-$_6 zKNJsg1BiS75qL2Dh^RK5JbpNWEeQO z+=;g7J@fP=#uFw-Q9Ds_=ysts?Bb3Scdo~hc_X{;jy#`CVRmtb{=N~uTC*;(_|2Fu z2P3g!S#|*ZVXjf1eF6#?%3Xo&xsf~mVc-)c8CWSHR1d4V%iC6HsUL#$3JZEhnw^90 z+B1x{nz-lW#I+u#X)82fY1m*gZt!PVLwr~nTD=n0oZaQy29D%-#PMJ;&(~08|g8nwod`4>z(Zfk5 zOZL;!c%0hx+V~09lHEb{E@y|a<#rCjW)DLhveg;as9u>#3VW%iM|h7zv@Q8~h=~DB z>2e=GtjaG#NmMC6h~;QK!7M5{?WJ=sHg;xkpOcijDB^LNO6^}H$aIijA<@9pbq$AM z;n7J*x0YgYg!jZV=#UAGYcm_;5eNoYf!#`9i z!ZoZXRWCHl0dv|#r%+I`C%{Wh9wZd}k$hEYtNGN-%@YwRJ1m%xo)RRqbRc0SmS7PQmU1o!KSWBlX4o(y)|cpn=spW(Cl4B8>E^VW@Gzy<(EXMl-6P!(3JxFJ{-yw%LM>!FA=Z=5$z?N@$)2 z6%bgzu0$s!^!?aHil0Yly;jhN+(!0Mz!ZK+uAS+_lK5n~xRssUdBA~cy8ESsv{MJx zA&@wjKb+xL5I;|ZR8Cf7w0!{VX2MB7v`wO zcnFvo%g=Crepnr_`az$4)!>jh1VJIa)X}iYN)R{Bp%u4wNm1fKSZrxiJ{s5U*~lc- z6l!|Rx#JEiUV;IFAQ3WgSXSrdeJy=4!i4j?4N<-N4s;1#rflS>naE{krvalwYvko` z{kFqnH5dS%2?Y=N=bRe6a@9dImO>E4sYhP%>+Y&xKI8C(k@Ki3C2&NkG0neTZ!DE2#=rB$s_)AR0w!|5(@|wS>GxkYG3rZi zU2vS#thKQT2(i=%AwB6W$4(~)=6e9&iGF?d>S`Oq_j<$O>B+&D#bk2S#h&aSzdD={ zD-lC^T<7c7ozDuGoP(Nup&nbGXz3*C8;8B}DYp8}k{WjT>2)7+rOt$$HyAFU{CgT6 zU0Gcl96cRUxF8H!y8?JOW4GnO&oVce#9w>#CQq+fcCBJFqyljKfrFzp?7CDe5vr~J zj4Zr&ea-$5ts87HgSZIO?Y zheg~FF^h`-&40d?009_~Te#=9zP(d2==?zFPCyX3aIG^vG^o7uONyW-rb@d!1lT=o z;|>80La9lxa=&C(1?lUM_|Nu%*tP%VU;UF0+lyVHMl|T#+OVD}Ra@iE>`bHTLZR6z zXL;dxLVfW4cEUpJC}ufFH;B6#w|FBk*=_!{!Z9eV~xHs&I=2n4DBCogiWA1z$ z+$Xg!BBW9=4TVnM@(M<THhmcKXT($JN^7x*W9E+$)ID5cD6%tLL$QX#RJEXUnRMk)k zd~NBL*rQA67$fy0KXpZP_eJUQqM{doiAy?p!6_F8AE5=5_v4_;nrMA8DwfW#I$Ea( zGgL<4enBjXwVzCNOe|9^!v4El{A*1B$t=G2cGdbF9)MafNQ64eDFPN2mdB5+nJ%l}*Q~6gf zZveSDw$}hb$~m_9)Tg%q(A)Ir%?rsnw*TGJU?C?Qe)KSf@?wx&&sS+Cf=DzFsVgBx znjj-q=D6AYn%v>`_23gM=!r{ChW%gUB{d@)fmYP=;VEXGV(doYVPzZwOKu0kVd_c* zP-N0T{jYXV;30yuuFNo1lc}}-{rSug3%jKkg2Ml%S1T1h>6e;0>i70lY%+V=3kbxM z&hqH>n^)te=AnfogvNCpqYRl3r|Pi7bsY>>u}&8}C^^^|g5_vO{+D13koyl^TZ!0O z01@}V4>)|&qS|$0-^t5{o;!vtj(f;#>Bwv?O?<>PA=BUDdA-J}4|V6OC1?wW*!e(l zJgPX;tU(6zUvYN|0cE89*Bq3H+wU1EPErY_PdsqJbjpiVxq*2e(3%|beZR23Lzq;} zldNY>4)H_I{B3Qw&}jbZO!(`b@ros5BYhjMzV4u@8nPq*4tgx#O4ICPl}BmkbTJ#Wuii@D>>~Jj$_hrARBZxsTw8X z@1rOnc_i&wlD4+SM##jBH)ZQofK+Hr>}TtN1U>jGEiFyU$WYPIk+HU>Hs43^3m~ky zcF{8cwVRA={DIfHG{u}f6%@1Y~>ISl;>o zgsrMTW{RCkj>BsK|D(AF8;8z!oW5r7xU2@0d_&QH?*ZtdP)71t45&~{%~}c1kql7q zsLw3E&bk^vBAC<}HQqldXR5PAe1`81B^HffG8;jPfWz*atBo_?-LMnruHsBoSQ%vA ze+k^X&*<{<-qF#+D@*RURdwioM-G^z|4Sa70_Lr3IAJLQpXpjUxI|`hpb}-t+1b+{ znMf_mjS{M5PHaZ))*EpDXJnF2Jp9mC?uBmjg0)lR70!H>eb{Q-i!+XMq`;91J}%sq zZ=F`~gPLS0aA|hBg9wchw~O>>9eKJiQN}^|GcPm(`9@o4=3Ywwr4RxTjZB279KYxqaq~6K6Zvd1pB*6L=p3e|+_`yYGg@aqhOB zTQXEeq1B|BmSRdQ3!}i?uXt>Gbya%Bs?Az)f1&L=g}UXhl(m#YLx<57?e^zzXubHq z=_@XONF%FC*W9N+ayVlH`Kv1Ff-(i0p;ObKeLtp~|85Zf^@O8=nmFgQ$*m7eq(XmT zJPgo}Se?z6yb zQm7+A-(n9UKgQ$ENUBlJ*8ELP{(^T*H2x65^6S9UUPz;~@VCpWi?eb4MRU{!D6$`Q zp3A=9qj2lD7BzH-doVnk7iVlvzJs-rP{l-nfoC-&cYQJbDg`xazOK)(dw@&q98hs@Ovwd@oXifT3fG4}(E0AE`T%>u&`Op8)< zne1){OC*BWNunphumqe+lX8(HirX^a?CTTDJxW|A^_ii7*c>_V|2jtTU!?fg&0Z5@ zV`BPD>Mt*5Ngunz)@;4o)VVmiOWEgaeu3xtHB*wc6a{m4F(L}2$8`w&ik?Vj7Q?bG zX&zwy6Va2jA5F$TCYMU}ynaRX;i5C@;2`D%{8y;&|IDQ}h@h9b*CI z;|BxCQEvMvSXL(=_fvyznp`quea5~0tPp=fx} zNg3-gRE-&^>iBED$wGM4+X&aNG*Dq&V=_LG_hw?>(atwV3tjM6)_IfhYRZEv7aIcm z4M=`_kWhF}sEUvE%=Y^FdSrau^zD_?d*@DZ*${h+H<(64 z{*5BJ?6^Ta9#kLj{2S8pyx?*Lswm@x-qKy! z5&Ea>qum(mTHuFSo^C*p>AVv19D!GedSoXXjQ2{H2Zv(k`)ArEN6>Mm#LiCoiJW&9 z#-1yTj$8ny9uL3E&w(P}%QbH2-J&k&mD73sy4h|nXW19818!MJiy1Fv+~4qo zSFsBj;;wGD){whb^A;V?;d+lp=XFh-=jHHKL17AKDI$l!H{wLHiB#@yUgW=3BylV9 z)~w`#4CotW&YLjpqeQ_2U!F8FVjDOdJ{b?76B|z0?(DH5Mb~U0BXBz1kTWg{P7BFx z`_qABaiyw$O2pERK1oLrpc4dECAe*PGv2;i5qZ4H2Y?7T{EXqaAz4lO$r%FYQRYUw zZggl+v(`bqd%;g5)pEh?(LaoUdx z^UE;W<6LBkf`%_r2O-QAbotzwufiXay0n5y{zlLj-Ob;cs?HC72UkOJ{s}^=>Gxg! zc@KNvUJxJeYTzzTY{d@Oxy913bIUekGJZ5x00(Ya>)yI;7hpl36U#u_Y>!6if*#P{ zje-gm*=lhZ!4BYOWcT=*@*mgo-{J>M!RW2{hWeX1z=_3B-Z$s%{F9}mWJds|@G-}| zCQVR&-x)3z-#g3?2;OuDMw{H&GvCT6$GHrs*u1%`HX7&fg7(QiW(i%2;z+o1Q(D+~ zHNvmF98M+fx)F%&Ne9f4TFqTiNiYXt7;qQsG5t9$D!cHrVufyZdiTpUMT#xagyobq z!wY(uEg6R!I$^XU50Bb+8?tI-`V)_Z)zF-Otcpl^n(@IHdm@>!0hlIp9ms|3R|^fz z%T<4rADLmA(Zl`j{^0NJ?IoqAineaNW&H$Ys2C3CQ@Hng@`O0__+bIA`gK=#Rozv@ zjDago@N}Lz@VH!mtC5p~ zZ|?XF**-SL_;vWJ-t(+Rjn5mh4qm9S%KV?(#k>85VKe7};W+4d^*Ue}bDgArZo%bt zJ;+aArL24?a7vmSTuZehK0N5IbGyZ{oA^U$-UmuU?QpAX#Pp4+ByqF=+`9aZ66cWI zf7*$M5`C7D7Z#3O83Ye{(PzL`9i9l->#QuDn(j|HHhjmzGCj9(zd&hcvD=-%U>p5+ z3Snob5zJ2(QaySI_0F-P{M4JF1lbP9O%5j~y1n%0c1M)j!$9e<+bi!)1>}N=o#KKk z0{gZ3n5--uCnu+)`A5G6_1CYVE0IEcemhpcpGN-t$#?I5)l}CE|95eNQ!4Htg@s)G z8eQ~={y98182&V7%H~%V$*S)oBKteXxNkROw&C&%QX=;h<(V(rrRxlNeHPqUBF2@7 z`+`49IOx8dmQ^;6qnE`rJz`?)EW=cXFT_gc3)rOmu|%#3E!+w$hIHzI8VeQ{B_0HK zGfG0DCnl6*xr=PnLDarAqwz(Y< zmwH?c(pzwES=HO(?3-_P413g#2v@>b7F2+}>2(AWP?0M^dGJ2H392M3SJImw-<5i9 z#hklfGRMk^DX7wG>vyH|qx1LZ`*{;eazTRoc<6XPy*0jz#9$A%%+ZmEVmwi{{i0~Q zvzZvT99#0&q;o8fXSrHczrGe?7yg(>*Pf8+vj2>v( zS17#uUH^Xot3>>&u%Mx3`=Oyrb5HC6UwtI>T4~9tA_C>~1efzpbocfn4Gl}jT z9lv`LroudcW5_xD2<9}Bv<5~sSL5*Ov>QCPSfWh~2~i>O`kbXt%MC$cJ=gBI{&)c{ z)Y_v5k$GPV?~i=ig9eL1EU)5HXQE&^X`7-@v5LMxR%Va;TU!c<(*oLd1{ z;fov@4Id`^@sEIEe64|unBe2hOD4MZB2cwh=$56sL0CRL!`+8c{GAy?-3ZnIB-kA; zrx1oN)l<<*6Q+*~>*EY8*{RN41pM~gLw?=fnhfkCbksb9XO|8A5{Fx2dEnxVU2wPM zjC$3Zn3&$?gxwr{5?yKnnhVg3Xnnosc^(k{9eGgpr9IW`c2PX1AR(7p#BxJuBcnA+ zvD^D6w;%IQ58|@Oa8mmBobGDPu`cPbgd~~7_hO6-pQP~G)0lYXJl5HIW%_!4uM~QA zv}tZuh^q$oYn%tuJ5#4N%o`5Pbv?eU8z`wbZO^lsG0RkpK(Xsz{@dJZ?~8F7x_8_) zr)*jLF4i$_{chU7M1n^i%^#JYw_nem0XbCJ;babZz16M|8{*2z6qpr92 zz;=ucUR2QU(5)OgRiKMzYUl2Vi>GuSZ%_LJUwQ))3*Nu}kMrBV#Fe8L7{8pn(q}|; zx+`%m&5DqD@?}K+;gNL9uhw{crkQ+ieKgy(;ih|U2QxSV0EVm^HuLFOaT4Wf*^e^; zuY^!$f<0k!57 zR&VN~WGpDdV_58rzAS`P1rEw!(6L8_MN1Uk0uEv?X%NtGc*Eyc>19R}oZ(|gOCg|( zO&(H&SRrX^zk}au6#@$F))0Aj-Z1MwcnJbvh;K@E7oMk$m2L3GK_zaQoJLNdJmc*E z#=H%jQ0lyH-HhnFVxWlI4CbdLDi@Ixa{%r(jxH@bLTyE2h2YWglrBdrDMy3tWw+&d zZOe^L_!p5rCA{|#X1ubnWa#lh(D_q4~( zHyJ(nLsSJ#mmk5(pMC7iE-Xr_Z})PeeQip|HG-1xg^D(4!LR1Vd*kT|4VXXEdv5h> zdKlTlk0NP1?p67O?XzU+5l=YMwc4^hU)MY$Hx6QB@biizb}9!i%659Tt&R-#OXp`g zF9%ITZirPX;4h74+0fC~ZwQFniVI33WR=77Jcm%Fa7+pm!E(2Pt|bEu05*d+U+Qe&U7gz z*?-N-f4vr)rG&^gYjD zJ2^8Q03x`nhrXjFK}=z9c1*g+jVv4;eq7p}P|@c~MiPHljY0DZ5~{PetUQ8P-L4ifG(08%2*yF)RC@0+EG;h{M!mi#YlTK3*C z<|PAHNA~mtyj;=WHrlXMM7Mp+qJQMOX7Y(luJ?aX(ZA0aMMO&*KgascQMZa%HNI)f zuqkDTBUv&#zs}mvu!ku_BFR|DRhfVF43!gWmfXVVUIYZ%?*9}d&en@#9G6m5gklXO zs5`Py(I4mq6`+@swI`&b41|;hQ(t?;*Gq-Q`r$B$f{qHN{8&D-H=_>)*0?e+jB z?rJ-!+-0mauIWGa^151JrN)s`=1)_-BuiFM4h0~3U8i1Ta#bAYrg(w3n@x|67ou=buuR<{Q$=lv-K9=8(r9i+QVt?8gpP9GFcq)r<+vU z7vgw@@<@lF_4ay)f$E1qU;}n+$8%e;%&lPRcgK$kkVhl|BrQ9(czz~)s4P%(>h%}1 zs2rTih`&$QQxqJqYd>$nZ*WK+b_~+fOZfK}Eu{ ze^Y^RG#?}46jo3HHTH8&9a-U*b2}rG!;;9XY^kq$Y2aMM&(F_!e|7@9`NIpGkYdze z9(-=bIHZbCiyxV#r+4VNCHydBlxp6X>Geu_x0a;4f|-j-Cut|AXv-X;#jJaZR5_G> z#;x3_<#?)yUm5T{XG!#rbn{3_s0;h~@xjeK!FaeWNc{E!RSkFB&5OX|{js^eo=b*x zCr17{W_UEQ7{oU4@#YgNiE+zeA2Vk#g%+PX*$nG~eUHrr50`&K8v~#CC!qcA%YxYi zBn-^1P6^lv#m9-HSJxAH?DWaK$gu0q!D?-(wG+ybD&5ojC1tHoiY3g2;g)&g^cO8% z8e#fRW(55{%$Wq1C$fun6Y9@2=45GpUtrd>Da;?yf3`FJoPUAaruA}oFS7o{|B2-u z4G=8-IzDYu{E&Zn)pT~CUBclc9&IocX76!W!zeRv!5})sbXCIdq*{7@wmod|%N1)P z8@c&x9#>;`} z_4A2oa>El*{u?U`gH~b0r7teKlMIcm7|U+8y1s$VW9syt&R9XYyF9@eqZOzTzy+mR>@4DHJp{V$M%Eky930CE1zpw~ zS$b`oY4@*^(CYgNhc-gKG%RD0UTsGM9gl5$^!A-OW*>7ql7Y{DGH#SL+3 zU-b-8D~=(I)QPkxx?)w7&6%G|&r~_S^ih+1uSx4}Sc6T+W69`nnT)dH*r3qlOn#Hin2!%ytdbZ`fndQQR*PT7skdJ(8`u7#{9Yue zajLk+O9L*kJ@4vUA^31Ftq-d= zW-&{Igyq->BC$mC@iiy>)xG+uFhR1c9a_SALR{y5c8 z^gl^;*8fFje^*gL=kDSI=rVW@U`-Mn<}nvRpR^Pls0 zX}jFNpN=#3%sJeG<{hM*T7MAbXWs93T^^r)0-{DNfnxJbyR(r; zLm|?Ebf)i#X|j)ezCXVD92?raypT5GIx!Z=f&rBxxWRtI);pm}f8=eS{j%`jWLb{xu3avE2F(}jZWbz>R^?Gyka7-1iU||T`p6H(ju}h` z&2xY%boyaMxuVZh%F?;fpxu74+YDpFivBXw8F?b4>>I>f0OqL=}pA@`gWG{iuKrCpVn#oIxA@2WAN+Bw_-9l z6WOv(WFRpkM-CQlCIVY~r|51 zS!6_XNe_=7zzi&Cc+w^vIXhD970?zID%w72qrYg3h&W;-3TUumXQye*3D|k##ctWs zEm-+J&^>h_p1qz46U1W4=yZl_Wo2dI&ec(JeepUzgd9>;8czyJ4ZkbXOGDCez2OK1N&O9Xv#i7GyE0w;gU=si7u-<V9`?Wz9Wz8{*IM5U%PfUkia~UioRdm`dbHn$43@O*;yigW)@ra zZ`dsg5_tEhaz4-ZMNZx{j1nXyV+CvH9gt#Te>|bgG_u`k!8MAWdcG!iH;k7G`|f`} zjcr7{J(O&dU~x)`izq=?$1qwv5$dMJj90l3-Qko@8(X7EPWMr+wcW7Fx%>&Pb(ZA@ zW&CqIN9!9nrrs%moLrY!>Y(1tamV$aa>#*imPWH)1eeJK(!8?19>@?RF1NFhXCXzD zg&N&4U_>^=w<2JSqnF3a$xou@)rpNx@8gY5?oUbh@ZgshFCemjtmP@CfTl~0N=w7| zI}IHceE9JFx^sX0_=nMpfMXXVOw0(m(tix%pRRVznIR!yz&p*Yk7tUH5VJ;X*pLws z1GmUSLqq@Nobo@-lZVv2LB_weXxH0Z3$>f={?_?FZTdAa$PU^KT?t#s z!a&kSP)&)Xbhn&K!mv0jDFrj+^f|7)?y_{in2ePaJSuSZru`g;s)npfm5r@1ESLJI z{9t5TxP>PxD1OPToJAT^JY2v*PcevN44RUvwm6WGCMzjAMFi_vQm{6$FNlgrN%=<{ zOW`)Z;GniHV0%YvN4pr=cE;HYFY*Rn(1?oidb=!#>H*gYDKSAzC?sBAWJF%n5|LEZ z6~~-}uMqr?qM08mPx9V5Jz-H7TEgmg_Q!{bztayC+u zEQp6G=TuXiT9gPIO_|hme~cy=o%Kqrm=EM{e+C=CK|J6Krv4Q4iU#`8UFjE43F3%9 zD4!`rOj-y!C8gs-E9&a> z;`6%{erOYZBc~9=uaV+*YtP;ReWpo!{wmFW;5=YAY*(At1_+N7aOk4k;E8*VTC8tg zie1-}n*Pw;JH#*G8<4MXzu1p*H*a{(Lk|5+`W_pcnf?yId1&RyN|!?GzU5=13XJ=g zFMxDY3ep!JRZky|cjYFWntk-$TEo}K$uu^5xX{63p!39oz1FCCuHU#Hc{hBZ*XZaO z@V7H>_N(5z@~UOqVY_DI=GF3RY9Ouns*?-!r8f(Am^8ZG;V%UI-(6CF`2b!!9#&3m z=t~uSx3xBMuL-?zvf>qN!L1KujW-U6f}jt>3~<}K(}%H@ORlz8gyarNN$gRXH}oSr z>AncEbE}VUPt-OQmQi)LDA0mus@FXSBwl(%roDc^qFuTH*WZt*xq#FT(_9AE6K|hi zUIdY8Qthy(>&= zzML_&O(@7py_Xy#LCcr*HgSoc-{kkds#PzGB(sk&qOS7ozNhOe?*QX-NCVY+C04J) zJdX!9ApIYso5M{z;jvdku8$3K=y`$DNWJ&|p4ozwK4ZfhqPbT?tIiZ9l9~Ct&*vdg zk}M0qJTFu>UMyJQN;MIIL=_quV3aolN_jMTW(^t zk5xW75xQDaE>GPCv)|lIychiMNbG621{~9)yHC~Vj*`Rh0=LXz7V6FWofhyS4o;}W zrg)^}L2GXrHW>UGqYpx<4qmPJ@Fg^azf{jA~n?!zdWyRrBk<-bDsj%fP$H^AO`NRg9cVny36R-_G zaGCf$J(AFC%B~kjM+X0Q9-|ZS(FN7RVJ~s#5m)!KW{tQSP_I@qn(Xc!&L%FE&?nm6 zAY)r^ujMJ|dZz8HFldR*)ftv%cTLPZZ*G;^T(WiP?UZP?KMZ8tvq+q=pYxq|ykvIrH{pqgz5sNcJ&%h|pRH!!dECwocB;9+ zf^vKHY|GO)<>0`}Yp8n9_bK2I9sexRaAlN%@z{;((D^_gMVlGz zAVJRFW(#BoJDJlGf-`7I{%AC0v*09C(3UJO%=s2}rJ)k5ocu-3o3n!To3;vuRxI>$ zSw#hrMI<5P=y*9rpw?M2d=Lx6_spyi)!O{PVb!29`@RD8^6F}F^kfO8qzGoTs?w^y z2tKL^ji^{Mstqj&RadMXqr&U4lWd&ws{Dx1 zqTd`q+sdcyMg6;q=dOemUZ|opap$ts?{wslGG`8l1uumugVj4Wfx}v5NCw05eKf!7TLyEW~tcJ&c0-w6RhU8{d!a$ljob%LT2n;j;*tDIMc2 zSXge_q#R%Rf5b!$mF=5g(TXJ^cBY8orQ)jhNEAu-s_M#1fGc~j)J?}9HTn$eD+jJ0 z=f@L*WG+kFPRZCr!xhHPGKG{TWMcwWY(wXCl0wtT1nq%w66Jgdr;cJohRMDg&dgBk z4Ih^k3?T~?vLoSU3WP{X6SD@aZvS1~p^wJ52L_iIj#&dDC(rXntC3QX!sE$$xgZ`g zzBAUqQLCY5GT2V_!Rn?*onc-QBQ>>tx9oVK*VSYcrZLwfpc^`Ub<7Y;ZqPiZE=Z(T4*0xRSIBbjv85s@e!{a zHeo54of=WMEgdx`=&LAhY6}Y}=(UI-VIzYMMw_yj#*WULogvt(1O~Y^{?U}rL0tyt zqyn^b)E)UWa#ZL%p)h3F4v>oELHyHMK452|fSvt*r=omZMoz>CRjDVOR}%@X{u442 zHEuSL9n4D@<$8h$QdM2?_#v;1t4gtMsbJjV*^%xQ{vk>0I4ko{TjQx_TRDEJdzA24 z`t72MfxDN55kZY7GYyyA_kxY%0ZnCjNH{WUmO4hnBi8SC3ks_H1Lz&iu$qK5wgj*DaM?cabfjXW9MQAql;-Xsa3-lP6x<=mJyy* z;Bvf+?a&gA;xw`_hz%v8SoWM(4X7T97tSMMrivtKLX>%|C#!h6{tHcux}p8mgmk26 zg*(HVilK`bl0_x4RL$s9-w{)zCyq)xW&#)Sg{*&n$Ho>;6C8_E6y}S(X;QGYmfbuk zrqO22#a*|{50)KUF01B>E|e<{6vLKNl7jsvmj_l>Q_**RU0BeMg&RdX6O9J0FUzV5 zzzV`CW=F(DmM+qiw>3o^e4ygF=Q|URtb5u8PA{=SW&@TZpv@*FvU@{T)s*#X?955T z56K?0=~)3dFGm&Ce61aZ=TV1+Sqh>W;OP&(w>A>57omJE3AKI@v3Cobj`*b*U8D6> zt)44PAZp9WX8QGE=(C zW8vL$ZNY3A%anyH2JAQ3Z@8re(Xq_7Y zy#`w%Sk>^Iy#c*Fk|=O7EL^<6@B_hLk+^jGjO9(8E^{xo_kWmNWG}@sfNcU5;E@Fc z9K6->M6(;Q6eucRUM@U12Y#;L{U){RYU>fkJ?BP zXfX$^RI??j@fXh!#pH}Y$~&7fL!+AzH)b(nz=Vpk6@1j=3@N92TE|(ckJqXcB2bV& z7KA{|{>J1dgI=Wtr!X1lYOUGihfortamOmofixXTE%rdnx)OloQtt==X~K0xO8G9F z)*IRDkKz7BMOE;+eW_zhn9sFo-<}z;B#g%O?NI(l3ia&D=lbLu_CB|VlLHo zr%`=z*1Qic+5)?Oqu8F=xHJNAt~3n>;$e-`0^;QrgCjqQrpQv(d}+47LqGCZJWGS) z;|KTkQ0?r1P6o>#ygWWjMo~c(RrT*+if&UN07ePgmLy~%gFwK1QEk$ilh#!m)B3AD zD$kD^!Wh|_apS`XNIfexQq+SASPj@hYK`s}kU%h|tbJQLsi0`R zn5y)xt6si@Zu<0k!=Tvt9e62ODkZb$9;PlZ819g8k)CKgUe8uN{4VuX!VxUYiM9(q zN;b@M=x0-_)eq*c;WQMG(Sqk~YbIaW+Mar`*J!gt9}3}sWhF%kci)Lre3y)3yDCIM zRRvd7R6xOMm2P74J^9wA^MU=XSzcgFal@#~>r$6$Rj+?O+xppL1ww`Vfq3;`a91(! z0Mav~Nd}AiN7WO90xAv!JA?l|T@V(t*SvT(dRuH;8l5GZ3l=a03UFXlfpGi&=ARc- z5yTp3XRB&fJ=VX@o9vJ=mwdkei$7?s%EZkUR8>_0rEI-9B$a=;H?;XJU2h`KVqd%C zBPOtBT`T|b<;u(2fh+HIFmK+sriR+wxbL_WrcPU@g9J$ga^>8sgZ7PCRa$==(&u3MU_0&jl2L#1z{wkdl^i?PZsLJ`8 znA3HaAxwW5HQ50Q*UrgZ46rL%;kDdP#Z^=gnM=k8xK7mUj%H!6>g?wiZTKvP?V z!|q%I<6t>MzOR%RGJ0_SGQ-2Cp)7XJwGry_N(zYT6y*xj^@QTp8Y~dM`V6e7D65Dp zpnwEGMOFksR}6_A<|B@$HAMH}sm~Hc7phc9LsLGGPL-lUIYJfk!v0$)HRXI= zfSvq>z?i|LHu3f|C`mLe@m=B=>==!ayiX!^8 zBgtlS@MOaJI&{$Ri&d;jJe$WC-jxFb=3uNV2M&K!X*LvNhD^qz2@%rfV);k4G~whk zt=1T!W|&;0q8Y`&#t8htlR!b31XXMcLgL*K3(JXQJw= zt=g5z9aZe|3P{7~sond<0t;;>4-xm7oqi}aT%!$$O9hj2!QmDyD_^n`DR)E(+=0Qu zZ;2+N4i^fl$+^ODTM%>KAA)E-svS+tAA9GXdClmxCPdH0knB|`=$CriW+-YdrlGS_ zkS!u2Wk{%I27~nZBBzJ4V)pRgjP!6+zL=%**DL0*hI9?wm+%R3u& zGfDdnT;EWFdI^GNXEQ8knNxF~@oxvmk8!BZ#JbnIibSzFhFaYj_33Ey##9%7u9Bez zS127;2|$5Eg{goDvVXOf7DKQOR#l&T(X6d^dH2#PjVVem&pD^=GAdXP5kn|n+oCJY5 zxG)f~NYIEtrEM6Bii%J#L9YQXgc^-mZ!belyAuoL^tY3-1*ZG`)~a7(j2+5Ef}|py z%{f=&cCg1&u)%0Rno5yi!4*}K6(}SaFhu$l4JSs4T0lS3Lsz<<}cYm2g7dfd4D6+jVBMd!p#`qX7@|4Fh)f3b&)Z0>eFrJiH5 zQ=$Dd9#d&S*MHP{E08Qz9GQ5CS%#M%5v+2#0Y zxA4AWzQocTytCx^v4`LnWzRP~N^Yk={Jhu&C|m`mEk5MyZ}U87@Qh5oSQR&|7i(0? zS1MPij!-LCp&DSQ)&i|ktX3&gsTdJ$qyna{vMLUfsMh-DFY_GST!Q8PxcEB!anZOt z`SkvQueqLyuFh!Q1OdX{f!sf=K5&)px}Y>6x7UTyI+h_>H+ z1cAeU0>k&)BpOYY1jNNCSF3Qee`;QQELhXL_ms9+-rJlo4VGt5UPC-?KxAk8c&Jh* zYr!R16$^!qQLzSM%}a$!iB`xl@@SbZu(JPh^n(E3x~my(ecU+IysFbT8C%gi3A~;a zPp?qpX#BLicpZYTc?r!myA^;KtlM3?4gZXVz_;F4cCRY!97j;EW~$bzRw#Fn6g~t| zyW<~Q=+&*#pAyv84ZMYLt2@+TX4mD7>Kh=S3Wv|yI#}hKQ0M1n z-Nrs15ybm=nN?X>1CVbtCwYl^+y}OMo^6mHzpvSjThTd#^EX*FN>!^BDwR8c`$JHv z1ZL!0Z-98=y|7JQ`0}&Ev49}k_#beJX>E$jOMi{mY7{C4Nw$Iu0TK=TpcNAVLyXd{ z?({koI;YFgTj>7D7sva`73H-lFWo35`%A5uNi|=oI{%Q z6ruj@Aa~oGzb%Cfa8>(* zt7gkSR|=T#OM$`xoFCCD8CrEvE9Ryu`R?#>x$T_c{nVRMZD_gm9+eC$b$VGttVj(A zC6zMOqGEqacDpQDWkf*dJGB1GO`^?ea~gKa zeV^#XL%;#(7+3j%h_wneDpfLwLcUgI;NR(84xFO%#KbQCFa2v=g@Fy+Cr^vruKe;k zuFU+e#|Vs5N`|Y%?zi{DxyJVedT1_eSN&Pbw^ZR$%uiDVp1&tD`rEo7Jgg7l<*IW8 z)uE`8sTwmhDwTl5sa4D+X;ljqGyVCaJ9$&8zcIR#`wq)abDj^Dwfsn;{|vNXF)R;F zs81Tbou{@Q=2wrmei-{W^z)oVsseIwOErLiOFq(V%h|teqtM=-I^Ep6AlEm!ylJiK zI3Ql*@WU4e$ky-o;~N@!gDPy>$o>{<)XI1IVAyOs)ceNGcH25>VE~D^ft_{tY58!44=a@T+08#Jbp>V>{pihdDR|w^AyDDb5a3cvqdget+rcR(1YsJ zt!0|fthXXLmL)rWVP!P`%BJYOi|B@j31Q-N#)F&PkTyFn_GCE3;p!2|ik&($kZb$d z|4E&Y+jtE25H!7AI%<Vs%M11K{e4A$f??g*|J3>Vi0$QmLrY98ntmUNN%lzX}CGS(ybc z=wemr@!i*^3r;!NbdI{r~1&ua3jG0OVuC!G;jVUj6x@Vkv`{RM^ z9@`U83wAnk>vIhK{l+M-5El5HeZ8RMt$F{d`+xmiMj1%h|JE}7EoI#Yz5C@kZst@G z@1?JMfAe!A5eKhZK$WxM6%{_Iz143WIzviOSXrA|7S2?}=EC&34fT#CnZeY{3{X_; zktR$|#o-v8n(G-HQLJ89)!B1&#n7Mp?VA@Cr;`J@IRM}IboZ5lcD3iG)fOe$NS7Nl z3Vmi8mWqmqfa-4{qN0GLBAQiE0rN74`{gyQBLz;u*34X485i*$j#)F$;cY_B{TSQu zl2LhVhmx!`{G}4@4y533Au4b%TP=JW-hA_aTKv4xzYaB@yb9ERBW;3ZDcBj2!}EUm z&WpV`qg=tj#1jpt>ykWKXMaS3oM;@4gKzBQlC*sdQ=}qpt(M11O&6tw1$~hi=u!n_-v7>-LvEw86&9I>h}%ggW}-lY&}`W)$R7dF3=noGCAf< zM^M?W^U8Gj1{fPB{;#9z|8flk1Sst(&i!pyD=_wqO35e7x*c8iL9UBc`Naf46sQO& zsF3`2r!%}CeP%%^R#+6-%+dqoa11^Tpiuoq%+#L@YqeQDBpBK8O!laQ6g9PK9Z_dV zB?~h^fgKr{czOmBA{dZi`$U3@iWJE&tycX+8PG=M+2FUYN9mlYrnhE_&@9ca%*JNC z#D(Z(lz&;HwST8w>%=iZqsZD z5C|?%I>Fe=Pz60(zb6x)lymZ%tUUV_bIMjzQe6`uWNZyH5d%yj4d>$0tc;0k=HZd1 zj^7@xD_qLq^@5XJK|z5Y0bFPUA{&uAu8-Wv4)rD~z|T=op-B&D^o0F)J@uCSFi=s= zXm&H;oGlb+ntmDd$_!1OAME}5zs#)vqs>vELlyJYY%rw02d{GSDjGZ8QP|^RnSP9# z{$32X1cnu3o7z~HGj&PZ-gfSIdE!08;~K1+Jxr@WJxc2ins7e#xalhO_&5&dTx<+G zT^uh&i628zQ2?fbm~dWEEWga|7Z8TYhFPd{QDaujXhWY`^a;Fb+sInD1Qn0}?nwIt3ovIPDutfR7DQ8@(x@PSrBAGZ59e@@$EBq_qw!ys2|M z^kHA8(~52%#3KIT0qx`zCJ9T+@{*2`k7xWC8N%Ztz7%7{JrbGNcrIy71lVS)i0NLB znjWV|3LG}u)Vzj3@fufR^67z-;VK(^v|K?@z0tYZfoqC_Wm;nZXGmgrEK@$PE8;2s z?Emrw_&>_K|5o{%YH~}^CA+I10{sOipkBW<@&N=TwIjDx_A^2K+oSVDk5Y+lnHq3w z`=&WmO7-S>u=yhWcKf;CaL0n~S&J-_$#aFZ`Jx8LF=4Flk))aXir?z!7s0Qe9*WIB z*8Rl($vc+e8MpFe>GxZKf<=J_xb>xgl57d;&LfJk#ijB3l)vq+<{Owr@Q;D$e#(^b zd&s>}c8Pzn(@M5?eVww<3VLJW69kh7ThlA=E*U5Ne?y6hKP&8Dc4!hQT}bJZ+5Qbm z+d1xjPiOiu&Nd456*`ouSSc`sUm0?{Kme)Fbg*XMCgWCiW34W#!Mcs(uo62=Hts_?OOl zBVA-)>Dp~);59O)zYX)~n@I2D88qp8VixAsDXBXKVPQ=he}zZ*lc?AzUPvgwC|^RU zPz@#>WA|u;VpyJkSp9Ni=+ULYmW56DFLB@hAy7gH zQZj&n7D^KGX9x*Nz6y;J6`EuK3Iz&u5mm|P(+DJTVfLNde0+#IN+c;k$j>WK5SN;+ z2>++@)2lhRu4~}BIdid3qrD{EV)2CXDlSIZef5Pnn6#_X4)4}$#lkARH)Fze?X~Cf zYOU$?xtM!#mf4!`{X8*?ODn&;IDDZ5WScvZ_U>+B_YV~r$#*&_$V((hLa6UjgGhr$ zfnXSX{TIrm7U{FXzKk8idlI-QVW7UAV4qNubrh=9nA^O89S+m~+=bw9R>JFkl=sLV zai7rOh>eF+Ig z@)ZOMDnx-Ii6R-AL;xBE2E6LeZxVq3;Lzqt=(b8Am7q$ILQ^WJQ^kLQ8Z?TRMiNR( z2_cbznH2&yA)r)HqeHTtU{ohku92rflOUi(05zNN^GYvK+~edLcdo)2gTF4DT~5H( zDySqU@r>(NWN~L(cq;?3cpxb^Th}uNl3I%0GkeRP!u8ylxFxU zqv>OE&vuTUR|qeP1AsgC{UlYtYOt^9G;%p>oQ34<5F_^T4l`747rR z-gqSfo+I(=yz6sA8`>mW!w;NduzrS&0U5_YyIt@zlbXu&DxmOvD0UIdE4DJ@`GtWT&>py#_Pkm@k&ScW93_LraIEc zK^8|_?gh9zQ)LV_kS${Sv79!~TjHFqXOGM6w_g%rGAw9QI2hS_;|divMU-b{YPxo3 z*M4AUHqbUB; z&`GDK6v@LFxn_qD`hxLEPVL?~ zPhOj^4{qgexw~Jv#x_0`{SFoyS=gkDxrS80DXwvqqvO=_TjRO>WcHaRfH?q-6W;=q z(hr<)X~&wbp4%607xT;Q`KS-vZ=N!}F)MQSXl!*hH$@5GB<8OfsrHfX_mSXaZGYcS zYzgzfI@4I~Lf33{8-;eER-Jk!u;=)u-AqHJ_&KmSkSkNQ_-wUP3~Rq;+SW+ z0JO(e+rv4W+(I;~;oc-dL0nz$=VD|T6cgPmqvI55Q_HDQDOnm7%CippP^NmkaW-BE z>~l_y$A<0qQyQZzbGUkg-9H(L-@ubkEhXa%{~4~XLZ-o`IiSTFPNx@Dr+4$`LG0I* zJN{nn+U}7$WC%=+&&Ol?Z3rCC_skB@nmAhg_GIk^x)RS54up+&QZWzl_=iZD7#hocUGET>?PgH6ZXD?9;r}G{2vUZShWYj_^W-cV+TyGHrW0*gZ9_x zs+&`=IL1C87TerbGO}`t<<>>{-^-l9)vs_#YqmOLudgooBv;8l7<1b#*-}+C8JXXD zrE48c_SDqIb%zSl_4iKYy9cu?y(rQ9O;zQxxdxtZeRbY@u{qvP-ERkKGM`5h#JUbi zYP-m+)l;6LEZR=s{hm|px9ta~kl%*bADnLamxbKZsh_qJfIEns6f!PgT3;@0ON;E@ z`t<6v)vi?LX8MAtN2wMX!tDYFNdl;t^IHrH|Fptkc}GJ_gc#8cQWuxWBHtkt?uT5% zu^>4*Y|~Fbb0F(a1&ZHOD|~^S};BT4XG2&wP!_dUJ2*}0yc*MTAY9PPgFt(N3dEF}Oockj(2bmN7IVemR% zI64kscnn+@vob}shi}i8*vA{0wSi(OYVYtEm%*0t+Mr%;LvxFiEbkGz+DhVA+`1u9udkJa&+>F@i0Ld=$|3lnc2gTKW>B2y8cXti$ zuE7Z&!QC2nr-9(^!7V_r;1Jy1>EP}bf(Q3|cz-i?tk>5&py5PT4&i_ z`*{dT=y20FF3*hzAa4R_gB=wldwgiAps^(pse*$&h(MMQs?fUl+~#c!i+O@3rGdEi z{-K0cufq*(zOiG5*JwbZMcR;M6A%^Yk0x>hLvZ&) z%f<|v({Czr|s{q6wfRuw{1NvR@;TO-Z8TqbVT@>5` zapdBR*Gat(4~IUbr9c7i3nC^A60lg3w3}7=eRyrlA&unwacem{sV|FzO%{}Grp_9k z=Mn3B&-}49_bGUf1%?lJ&ye+74Dd;A?xz3KZ|b5Ld;c#l4~s~C#}qHBW6NjEk9+J- z+GN|^$_Y2Ezm8T3I}K6o9RNv%AE#W_8s$nZez zs~l}wPV3m~PWJ!M6(6hXys#tpHG@CZ6-^ph3=(;n(T$!|#QtDw-akvCReQq>zFUsE z-55ciIQCDeprKKtKt@ssczwjH!Ncy6K~i{)>H{ROM|>5V%rB5T?iWNwCZCY-dAbkC zFfOJdhzswiLV)}N=~Y_TC8x%t2oBqm>m9j+1_nLFmHI4o!2}VqAVn-S87?r z=7Z;0d+a*BD@EIFk9|(wnm6hUT6RXeQ|6bhGt;%_I4gc8b9qTBDm{(2!LV-|s`vJm7yT~{ zIxYbTrNlXWdqXHLCcXXR#%Z>G{$lDE|J{_o-&bLK1En_{U=s|f7@KBOa{}=XRC++3 z#3o}=-kzhp{)|R0%9EtJ+Lj2|01bIcj*h2y&p#W-1_CT(Jw$u>;~;w}ajShRDb)4% zH!jP6Q$4q85Ue^J(R?fD=v9w5wesR($~305wvhe6|4MOAB`Ec96XS6YbUpDdS{u|? z!DueJ#PVMxfgBkY8OTBq|1nKS@i4C|iHH;*FZuS>ssRZfHliBv83wo%+;B3h*ZM3& z?!TwtJxXGgCNjY6U}w5OFA3=D>+p^WR1!cgMHv8v#26fWnjzp9uL)#RO!sm8M=D*a zx$@%GY^^}gV`3*nD+AqhXg}Sqf4ysBp>-?2{SB%6V+c-{JN5mlJkM5S>>S;_Pf={Y ziq8i*q#|MfR2)VDFuXwe2Bab+fVISXl$Otff(8;?Nzpn2*21bMydR48ASsUsjtU7x z;0uAg^IYdZsmrTAcURQy2>e%dkr%ZLQ6*<@mx0_^*N&>Q$zII2-<|LHLA2xPchsft zqfVj(C9p=0+Ej3fP_LC4%y034rwx%zNLQ@d&X|9}i4(noxzc1e`g=>ce1py@Bm8Q& zX@TsOiV8{LEiI6w!@g$qr=^aTH+=3CB{k=JUSDxKnBB&*8p9X$Ar}>(o4UtP4Qxs( zB%ltXE{Uwn(t1KK=C}jo&;4O{8x_n}uxoysG>hMGn zt&-6Kb9ujdcy}3g{WoO)baQ#FZ>qyvR>gLI@tDs3{+3=SbL7(iP>q5^MoJA)#6|`j z%rlAlV~F8x$ct}xAl##QOiTI1SuH6PrSK8p@V=G*|h#hCV3qW0N!r6{QYPv5U*AwIpvl;;%hL~Jq_g2{Eq z9MRSrP97d?>e?!HtGbeA{)6j>`V&qOBfI zmwL$|nc`zHHv1pgg1<}8KYnI8W~{!bAVWn%bGjXZwK-|aUX1zQ6UKiQavGnYpn|C> z&40f3zpoB%`WM=ILSx(Pwx+=H_6|;P9X?U)H=xxqGT@G&jcK<9Y7<$^pR|(_3%zRw zu|@4czGpGbTED3yfD{*31kkFLaU^^1vFN>XP)y*tsdZna{^0|^C<X>$906}9G zvP$EOgS)Vp()x;G`m4^9%B3ZfQEM|$c}$llHz!5N`XkO)p}lHHd-0gvcnP~7?@Qa` z3Zotvi>3G5`Y-VzwV+ez(9qBlbX#N7X}@dLtGbgP=09g&3yMh4OtVbN+hdsPcVBhMBiyYyUcUs4U%yEl8~ z-rndB+BH@bwG&3}%I>jiOL;xM74|qb|2>36yrq1a3A5LeEiKLLzd*2bVENF@@osdC4)xMQT@!E@`Ur~iRr!9N2(8!XuW!t1{-M_D zvP6cDcB3C>y=c%8`9JE9@|4-+;)d~mRYW7bI@T*HWonl2AG2XTLE#-xV3*nKHZ4gU zMTInbF}T>`Mp9@0D43GvA7u9sD!{2e7(;Q#atNAD3YM&iZC0G0+$3^8T^s}ih7(h# zj3GBPq?zuja1Nu1(;{?Y|VKy|Lit=h}|Z=2Bn zyZryXl@6#{i(;nArXNnwqC1LJlkCBkAGVYP2ftBxOZbh=h=#XaaFQzU3qciK;AOND zH5v6rMK5lOlB_q3N?knV#*MMNm*pBy4rbZe9TgK*(#!$}?&WrBQ1DMB5jK*sXVu~Yvy+9Jl5a6Z+8b!EF?{oXiLYrB5E_c)+M^g zmA&hDdRQc}p&iIl!p){W;yCGq;PxG2U=z)MozWtA)t@%E&oeVZmb0azNC{N`*q+7E zY5Q*aa0LjZo}UtxJy!*9Ups&OFq__@}t1G5HF2ZM~I#9#Sr^j8YW+;pohi* zb1ZCFu-{l7Cyz}^c(1J__ASZTV!^PyD$NSpzx#y$nezVs)cODBwyUcv;0&@7{cCdf z-~9XkeLHpARC%^oiS=2YHI^3RP z{KpQchHy_EzKDJkKB|B!u`kSmF_(RgW7wEor~Fv+-x+H*(QPLjw@gH52u?w#0pLGU zBCeM6DZfj!xOC`oBr&TrgBgkvi1sD1--b7;^qE`4Z@8NzBHsJLRv7arC=Qs{Q6edX zL-Bsb^X#1-wqAx!9S?k6=BU-!Rog+c2+9g89#~2y#-tfS0_raW+)kzA3ei7lYy^mo zm$3~RLX?Fu;sMZaDO-mL(E{cK4dBjR8Se|e1#6TApoO`ujcH~0fJY&sFFQV8oF^) z<)hWw%|wY)xB><)N?;9MIS6*UZj6#G70KkD3q#Rj5@dPkC(?+oWF z^@S(IU%&ut$DPARZSCRf<7EI-0;Ti&)YOL`Wn};pR`-_xe^!4xq$u&b_?2)2y|iiQ zVi{8Im-@~c`W+>n?KoU46(B1OQ(~HO8}gV|AGIA!4ZwNbVi1 zeC?v?d>H6;@f>GWudeg;Yd{q^*QJ7mOf&+c%vzB82aP}BWRjZk#(rtVO7BYCUSjS- zfK8YiRUcC<$VoPFE}u2_Hzuv|#!L3AHvF?Xh7A9V_0Q6ECiRI+)qx=B=%fPiNuTxf zlmH03mAu_`YmUQOsuShKY!vz^#=m?`RxKH5ddQ-K#Km)+`1Gl z9-aVf7k~{GSbmPP&dSP4^w!nbf6ct>r8P7ZJ~1iDNhmw3T+n6DsQU}_8}XB~&0hH) z0I{YE`1E?)fBqxj#ZlPfAZ-5fwa6(r?OV55{^G&B%i*22 zbnDTl-HGjPUBkR_Gz=Fj2z{EsAIB3|Ae9s%5W7Ma#WC8-CTq4cVM<*GBhav&L{8HR zv;_=SbtL>2ra#ZT=DSwuNEoHimrvU3++W+Tq8>MfaUgOrXpIx>fL?JuE+y4@WK$Iv z%z)g?Ca$y#?>|2CRi5bdetg5eZKo^L`7qf2=p^xxC2%Rb?sl2=qWO1o=I>lgbl8$H zi22%J)y3a(y(|mL-wZQJV+WcmL-7*|!ON_Kx9VcK7=Xr3AN2h<^M`)(0tXWSC7z1( zW*rN6xME@#pzGzdUlCYqKD8z;YU^vga95+)2$$$m5 zWO`VU$-x3hFPJ-u-B1l0qVncM7x)xiqw3uf@e-8sYJ56be}&2YZ{y7R(DrIHb9Vkz zk`t%fxeI~l>zszc*P7mf)P%Ia{ix1;n@_nHBda$x6Yj`Wz8tYm+XI~TM+hrmCyW%O zjInDey#Fe?{}5(?1Bda(U~qq9q3=f$xDY(gH(9Uhr;bO@Egz1$6K9j8?f=&#TLV8O z@70>k@f0a%4t&R8P@lfi3&pSv%1$z;Zn<$}QxWSrG)+<8-4D)UtPm_yW#Sgg7|2_K zVxU&(+M%`iwfM6teI)Y&`TV^8+j`vmKK(4th&ph%vclqe%t|SfFUBfrPW{vBf2$|2 z_6{BezJocuu_G|B?TuPDH9@DlK zmavYj;NN_Y2ajTUPn*m;KuLT3ld?=qYC+(PCVrB?5e1yj9JgIv#4<%kCN$9`;cW!tFrU?saskMh5rb4{Z?e1(B?+!hr!76Dh?YTw3x9{-@DO5 zU&KwhnUlSOMi5q_FFELEb*ei*;;8D(w8csQRvLMrI@Cm2LH)e>qO{9)e$!c86~$XaiC)J!uiWY`w>?6#-Wi zebJsSt~NQOihWP{kd^0ZgB6v|4i{vb`$E1g4%xQhbR8AU_R}8LdJ0=?afIg-Bh}n@Y4wvuw%1fs@C4U*J;qU4of^h=TNigZl_Pq_oA48~LGJLv|re83-nj?P~Y%UYyEi;7@}D;4?~a)8qCtH>XnvFJ(3&zHItT5fJ} zC4L(FA7fpm))C|(o*`>+ZXAn*WHY|lNv4v7*^%ek3oEvmJ5eYlWU`+If6bOXIMZDA zb{nNq`aYNMEBUwy$N6X}M7O^+cdWRnHb)joKg?b9(H7SXNyzj}lx7Iij;v6tr#&d8!@mz(o@V3XC!N2SUGH z@7**c2&6J9;YKybh**QUzv zu|>`2*mZq>4F7R9^Ofb(1l?{dJ~@Fr<8yFkXL-1u3d@gvBY2AxF8?{}k@l>3-Xc*= zgRJ4iMuzGq4uGl zIZN%){u}N&Yxu7m7yw6xnViFRcRIqYx|`P+p>XV7E7Kia@-Ew6(6Ec3+U7RHIewk< zUjZ=#&2Yd0einVuoG>+4hkf$7p{1LG!4oDDcjrk0v6>HLwBO$HQ6TWM8(-p7UECJ~ zSQUTaC=lky0eE4UrM-X1JAfxX`M5Fg1+(*1n{NsX`0@oDB<-tz81?Xsy5ZX0ZuGpn znqT}hqWL3g{%pv+bGLW4(Dfws^_M>BcBe7C47k+YT+~DPXo5%DDrA2CNtaTYpcJAd z16W~CO&^z}jA~8o7J9RWF}Pf8cny*67n?1499WhD7Ek(U{XT8h&NppB=Bt<1qW+B~ zBkja|;vzS}5MwQ{9xULCL^Xb&0I-9&-{2o*x#D=VbER)ti}QS~v+gb)TXKZ*k9R0x zTN;kG;$HZy7c`Eis-eQixHhh&(@yd)%~!8aRb>E6e3N-yS%9&Gg-)_aT?g@`MUt`_6@0_T#S`YU z6DH3ekFq>*QuVInWIKd{vSnsTtMVKw=)5_NvS`1xjQNdObj{wJfZt`}kyct16H z)P4OMs4zJL2sfnAaddrxlCwu@dh9(u`&yUwY9^*V{!D6}_pZhkH>6kwX5APm!uZUq znDb?`k#R*=8lJ&~<~VTOW#mEg{NyK;?*ws|Dt_Vw9)bBqT(Mk&;ar$3y{3>9`|r{G z1l4mD5C+J@R<QHpLKAP4wYa#HXl7B|?zp~vj{hH#sh(rf+4RbHKy+PYXbFZVTdMCd!H!YV zAQ?V^(zQ@Bad;3<#Io8WG<4%cAz&X|Y-L8e0tfXI%Pp6)ObI|&;l*WCi#lM9y5DS% z%{h_cpw86zb=x9-$IMjg*ZhNiwp79IQax%X;%wu;f-Ps}Rs_<(Ve8!V5Yc{t_a{n> zPA){iFTjk^#wkd|F{5okQ5Uh_KkRq?+UN<0FNI}iIfKqy&# zXpg6;InUp~h+Tu9*=|Hd;{qDR#_6(gFb7?WAV%@uh^qzkzSsYe*T_`sQjM$&)O8?% z^n7e_a>o{r;O$ecD_8ki`zSykykedEX4<<;iT; z+=Pi3`4u`HKzZlzrY@rT-4D??8wcO;xCh@MFXhAZhk@y_NWzhlvinUKA&^e<^^vgG z$u|CquRFIj2HQ?QzdOh8Jdi+p5{dQ4x659_kao^=^cy{ew$UEQZk1qHnop&+PGk)t~XYCpu&Nw^3|@J@(y|w2R1h|L?C@GT1xv_q%~45YQZ2+{XceKr!UTDzmxo{kWfZ zxSfs|K3z}b`7o^~tQ|^*j;Hr>=YDmHE;UEVKYp*LN2ID<)jnj;N=Pl^@K5dwLvT}HfI710XRp8}GV^t{5^w8<9g z?2&w;tSLouD%&5X`PPQj#n3BrAuDbVB{$GmR;a`a*9-H2K z*EvTahLAH}_Rnx(rJ|Pokn`Za`9K47d+mvQ^JT&4h_qmGRFMj}JaB14Fa%U(e%SHB zQF^M#M!c4*fb*hoL=?=m?K1M?4;02K{7s&lGwjw|hi-3KW$k_}<<{v{ik-A?odM%0 z11>|e**V$MP8raKaGCh57D)xX1eca05wPGZTLXa%JRgCc)hngY#+bZ>Mliy(#VKWt z3>Uz6^`XM`9^)Wj5D4NeNEoy4KQ!129$w=nt#P6|eR5ml(y)!rS3PySF5mht&F~XA z`9?9VvEyuUC=h$DK~QHe#a1KQvN^Fl1}q>O(dbuS<)(*C+m$T=Nyay~5Aipi8`BY} zkY{cpsw@e_1V)weR^$>GYN-saokTyS+d)ry7STBI>$qaG1L3iX)1ivu8``-6jKA}^ zoTqgMX)XFar4?J4gE-jhy-LBtp+iGD$V}jWl6SwW$JwK$gfJSu^gc_5Tc(9JOHL9y z*57y?ap}z0ze;4T?=B;&%qR2vj|9|0d$Z-^I*Q}>8qShOR+ZAO1@UIDBuGV74y8te z9i>@Y@!3q2eMv<#W~yB$@tFdC+xHUA^^6U`MQPPFA{u&%=zCuk>jx%P%}|(h_ciZm z%ufzYKjVhFs>p1GKhEN4X(vif{~~U%T@I;Ipr5c7Cj-KWal8GJUfVlcd|}f%!WogF zt&m((E3vdAe0xenL?@HTqk$Mz*Eh(E3?02DXuT%8HrHzTq{dn{cUaKxJ*td1Ymr-W zXX8A>LS(xoc$tfp^2BNgR;O;x|2!NLfk$mzmE6zJu(xcUf9NZb@#j*`y>t=Q-lDwwed-fSKd&~p4O@^ zO_S9RHBb?b3keNWRxHBS2~lH0loSjSy}0dTy! zW6{-BEk3bY_+m6}P^4s_s29ffX|K}M;T@fdDvtNnfn1^o^Qn@~yMnYiU%`4sc)mIT zYQl|;6gVZ_hD_PKO)|mb6_wyO(1x5S18jYZrD%J;x(jKB?iw;ZLl{I_Iv@&Q^j(2CTg=MMHQzM0rER+IHP;EooVghNqqu8n0VPuPgwfl@L(92C z*A-oQE^A%d#!fESxLbGy(khdE(jt~s#q3me+F$!)9XwG;yW#iXJtYJ9bh~OY9k~-K zf?_>Sp<(oIUqfJ5GJJos>>gs>SZqa$0k=y->Lq>0ErLw1Bf{+seBKJXfXY!R_-6 zE5R=$FZY`Fg@OV_hl?JL+Uu_6U`E=MtSH&DooNUc$E;*QS~+vmU9GL;tNo3_7%6IV=b$MIi+vh5($8*SjJP}!BfT28Rl=!L=czKQIX5rSLFw@K3o=ttS z##4m+r#~@7-$3}aRmNyv(NUN zo0H@s>K4wR*saQ&RO}r^IA+4Tu3n`D6!JKPxWR!G8Isa%4w8YmtwN--Qf_U>kXEPG z@-$YJd=rr-eRS(f;x4FN1v30>*WLgk^*K_;I*;p1=5Z+wBAt zW&N?*%6=xfjs;vyxnYa+XBf-8p6jFy~^_Vk1vHiF%jjxE-bl zuSu82zn_=Mlekg|j`!Sg&hEBqH?{=C+Ua7Fn`{-MsHy8nG}C1Lv=n|Vt|XU@W_O2o zu5~HL#hGm9y$=L;>-N;ft#59&p6-mh)dSLo5f(Suso>|>$|2D99g_Pg94LBWuwUar zpP;!}pf1&IvF3dOy?2DMao$wA%pJyX>K*56Ag(rE6hU$Jdnlnzj2$C&FsRIv@n-ef zL-z3sY45azo=Ye&aCW()u&Ow=zBFp<3?zWAp zsrS~Ll+Dz~oRjm!(NF1{nXu`aXK=c3nc#qCLyl&SU((&f4@SK^qE=hwgEdAx=JhK3 z8yIr0GrxvgQDD*{d30_5FK^no*$LP&&t~gvAKO>FmT|_0M5Bd;GB{d^BInMAXyVSi zN>SnlrHuA1$$gHb+-Dv8Di7k$r%LP=fFZ~+WE2TLz|5TL4veyk^ggcEKjH=~aiv82 zr1(D`d)wz$zsAW@v&kF}1t21)rx+12KRh>5ypp96-J9s01_E&thwL{VB@uY0r#nR? z&m(B;6N9_wn46=fV^KG+$yaQt7Jn>4j(?a*6!0ST-=C|VEtSlEg7z~M+dd^3wu|c= zO+ns7ud6X%Qe1hwIcz@Piz~hHJkuIYI~y<+5fDu^eXr?1^aj$Ym@xZl1N0g)Kqvkw z&AP})P;Q0IN`4RI_78&>wwko$9)yL5;=6@%;)P$T$aF4{*j3Y1)Z5di(2_j=%&OZm zSwe}RwloQQG4Iv5zZiqB=vcJdGRD^0Es0I{9lJFe=B2UlZ~{WuW%{QlutuI&+HuL2 z`YX5Kv2l9ubP$!fOc^Vd6lC03b?0pR@uo;U2n}m-vI%8^U!Z(ONpAh1-#4&tkM9rG z7M^ho;#QK6E3*5(H*h05)U^G^LlLotflcTa*(q< zRNP?Dp!(Q;W@Lc3J9|zK%8h?thYq`V5swG2k9Dtt0^%lNrfA}=8IBiSSji1c$9ujp z(*Ab-=NKRIjCY)`7GImgF&eVJ-A`a>_SzB_pK7?Dr&5*hE^HvG6Aw?MRa`#QJQwm(Mof$Rh0K0s_W`aJH(kE;W+2R{PK(z5}u1sNb5A z?=DLfzg4zC3nzp7@l_5dd^zmGcY??cF2&s5lJ#sjj(v8e@IkoP zt1Nyh)OY0jTH~}Uo3DeOFO`+#P#gbiTkoXFo|nJlmcFyukt+3Z6Oxs>8oQuI5(7i0 z==o>(psCU?-e$r!q?a1p0srX7Jp(1iTzW1l_!4MLD>$}O47+d^H(9>mO+SW2XSmRHx z!B90C-4(SWG3n{Fp=Tds-q;FT-qsS1aVTSX_=>e((KDfbAmy-^v+hb4Y&zCgf2<;I zZ#vV)+{`Ep4>yu_w;86j^b}+ge1Hzz%+p$nWvax9 z_N0B4ao{vQ%sH>kXJSxxwvF>FLWddA}sGHW>3gh{FixMm7)g5hR^45xg3r%!3{&h4hDAP7~EK^ZEr+BvP9?BrzsVzh6$42@_RlXurcFn zk0M6`cgx8qDZyv+Mo$c@oDpzdJvVBZh$eH+XDg?z20KV#3#)V{^XoS3#d7zTXzr_V zJzBAmF5#nWR8$VYmh{3!6LWu!59jbhR`jG!eY#Q)JQ=rwnX#R>(}b5)#LuYE@4evrVgjPpTdvB(W7*pU z0gw<i&x>oN9>HxhwI&Rc|d~U3$rnDk`oq zg>vTLDlOaKv@uv4>ynGPjhhzv=vqe+pYSO7b>lWac3uOV*u9!20ojFee3m&O>pwwY z_N45~hHUwl4LE!|=5KP=00xR*3H|^IXn#A_|ArGmXZ)H@xj2Led14p;?~w&=?eqhMWA||nF47IC7~i1 z){rxOY~e`b4~aDrER-CJpnr4vNtlb*@t=n#>q!BX_Dkdm;u&~VC0utLzz@9F0{lR~ ziE&>zRy5&}7={18_X2Rjyn$CqT#QtvH{+x6`ywcRRWw`vwt{xSF>3>CD%ru*V80;!&Wnuq&W7mWS>Srd&BRbUp8oS; zdaa9i@v|M`Fcp3**CwJHRPrMs$V{l+yUEMv>3!tHQE>4+6!T5mZ;yBUJ*4y<%5#`Y z*)Oy>@a6lf!nqcIX#CAaq7|DjPCE;Atr5KxK{bo>5rZmrF}*!l_)%z4a3#<^`h6f! z!cH|nA2=mtV#euJ7*@)5QERjZ4m?P$b=xI%w)&*Wf+brqAmf;y7<=d15$A>Re2>U6NJu;5X_ znGrkWy+mK{L1O9t@hL$xTD*tcEZLQNmlpW$mAk?{1)=9hO}Yx(4~(^MH1%@29q3gy z^6lrbv-yMI4ttw7LUWpkEza@k;L3)BZ~fjGzOjf(#{_r@|ML4yjK-y!+~9rC!W^iF zh)k3o=L;A}-b6LI9__Y0-oxuW?GnKc>fSP%)fn?4B`H0oE#>=y|1Qz6U^IRa%_?Um z-7XfEp1_va zV+Eu}%D~`3IxM_Ka(_w*q363Z3-qc`e#ByjOL5QUrJaEhSce%=? z7`kNj^gtQKGA7uVdwD*C=-|hDlZF(u!p6c32m$H$ei?$Pkr4vYCR0*UoOKXv!F}`w zkS7<3KOoK<-cfqQG^ch)@E`#GN#yH^W+w+z`o-|hEVSRbXW3r!OpBa zP-##+BS%5AsDja7h-)C0Ux~)^S z(0hK7=bBzlz&(`i+FapP((VkyUli5i0SU>?s-6D$Zae+%2jqRP@zuv_XLy%o-Gj9v zqRqTKQe&6R^QPNjC49Yvnx+=LuBOReZSe;(12R$I0C3!X&VxNwu^kLPAdeUPOMXYA z-N9(^0S{_ee8_@f6*CjB; zTM3lSCJYEtZ~3`F0~P>V@#jr;*8HX>Qh#pB6=jRCxxTOrEWVZDb7`-ZNnvIr6hjWV z_(>kv=|K3$+r@f!wlL&OXZh12^eDPAeIGGz9~!SP8_RbExGiS&J?7k#Ln(_< zC?5=VwhyA3N**{&4(B*P<|0A4L=w?Rh!M16!396AFM34htA*`g<6`{+%^r-4M0^Bq z*4mRgSDkQbB!a}h%u{SE@3@-eCB(rS#yixS)HhY91prMD6$Ikzdz5|j9l7D}?G(de zC&9IrqBQXQy?WkXltZ@Kumqb}cNfMjB**AG4C4b?qqIQEdt>eg0!sM_i&&~J?|i*> zHY04sJvkNFx_0}LvSkv-r3*%ameU&@$tK8;wdyVD>4}ro3_a4evLDVZgb>qi>TKuq zbl_U1D}7<|&||Axsn41OkkEK;p-2Xuj2c6YxS+Xkuute|SH`oa2WmjWJhE|qN#f!k zd($yi2Sc7RupX!uRM9=PjouzxRa86u zZCkTuV$PBlBGXbLWgzt+7VlEgnP(9SIX0av`e2RXUQ@&*Up&G1_|}KI*uVveb^P$w z%$?1^4_Q1!cmHBUivRCrBBp)w4c4=(yFc~uBdoj59x4NCb&BBO=r>!Oq!byhZw|~y z3kvp$f#OZcL51KrC|=(BnVDdmV<1q75)C|)?$GqLBXX;k`GijppPQ=+baLI@?fw+q zSIzSGR+P*>Kz&$E=eUBi?>NUlX+C{dWLLVAOTUTel0>*afL~ttJwKKbaK^f@kDoMC zJ|5R&9!D^Y=qkVIgk?`oM)M@GemnLmwO27Swbf)@r856cCyI$k+AYk_hwfx2Zj9R2 zLX1G}CmO~G&w~`UbJ%4;Ql!vutJ20Aes;L3xSZMbGaGEzt!4}CZ=uMey-8yI+5uXZ zFd23(s+f?OkSDhO^R}vDe|d~^Qd$7$Lw&;4yt|`EP?3Y7az{uo{#D(XrPIM0n8v{^ z{+1^pt7G-7Qg@`mT0BIUzovUJ#r}ZJQ8pfnTxxdh&CFlXK(&wBk&s==72|EsJav0N zc_}Hh&&dW%ipc-cK42c+I{NOsnt;fYCUa53(2drZJ$|3zgAb*3E>BJ@jI~rGsIFGMkxHWr)c=Nh}$ zO8j11)Qw*8pAtg;FtW!l_d`8aA6nn(?^I0Tv zRgTy=yRT8o70#AbNk#CHCz9DU@Wa^i`AQy2&!!UD|EGt{u6M?XGF5Lj%#nLtEvXel&XQNg+>NxV zvOL`PQ3;(u-JN0Lx4YiuGBe_CD>{KmS65tZqSz%hmtjXbfk9RNuc_!!oq@lq0E+Kp zhy6hN1w0D!liB)YGGYg7bc5(Jrq&r!*Ou}LY_0w>@OBq zAJ^8GMz`8@Fr(CA(tt^XrRWN9co_5fF@dz0D!6R48og+e7t^Je@4Rf$u_b#KwY<}e zI1s}NU}ipRsA-b~j2g@Rvn3xW$_SKMJ=qs+#P@d_*Y}o63x4}`v#vO+%aJ~U`D_T) z?S&^Mf|`$}PJ%P;QcP4*X7OdsKJgZY*_^|5S!^kq;~v)=$3^yyL+(93#0vuL7UpAKu_%-`&<-hVSnW8}_-;7#jO{ z#gl^!`du^|^7Suh%K_~=rL;k0MbPWsPJa^o=}oG+e#an|{VVrSR!`AAN=xZckB_?G zXn`!@QuL+q5_w;^&Fw)zkS{;~D2+7)S5wkL!0@lS{s$SiUooTB0p2Qa*$zdxq#TTo zf85z-c0?YzyhtR*oC3hxF|$)&f6J4|MPPzpHxuefnS`U;?U*7GxJye*slJfl7`|PC zBxbz#u7BT+_xZyAs2joP>(y%4`KimzO2>-+fH^{~2MiL;5mwO<_6l-7VueP9#-ZnN z^@MeO(~WdRCKGie3&C4tSxhq=`7|n@DxYU|DO^|(R_T+km_Ym0QKkK+%~knoQ_R^> zEUa#YtNi=C((i#S^x=-xa@@jEqb1==R1?PAt?3aXCz1vaEckv6*(zo3iIWv1HSgSi zv6{KYe&3x4DKE*E2ng%7+PDreE-fo+DTS+clJM=fFQ5C=W*<9b;j(?T=yXHa-N)Iy zphRlX_uVG^SSV#tb!BKfj&(Ol$MfzKw6Pb4600QLIFXl6jA)TtROH<7iGQ*th$1XK z$aZ@(!hdJ>gUmdR>edtjeNs+xw45B}Z88VdZBfb(Yyh$CEfZ6INl7?>`xqHHS6xt> z(2`vSp!fajn%k#(rsxzMzAD51NC?0B{DkIG1{33=nxuYIz@&kylR6fFs)mb)S7Twr z3w@2kqgmbXZ$m|DHG8x2vL2cKoUJ*Cy81fX=u|E7#5PyC_zvZPH??ZN4)R|ftx@YG zhjxt2)Bu)M|MyYVe>$@Y+hnLC#u9TM6hBj#21ee^>#JV$pLpXU{8VeOXLf_WRzOmv z6%<8Jol{!OzZfCq7vB#epi3LojiQ?0{mL1xaWw;+;|G1xBm#12qh$k&d z>gk{-XNrSBpHTqRPQwSmC_cNQ1R4&RUoNgfRAuWH*ME_h7OLv=5^Gn{2oYh$zn(A} z&3wbhpD>Vc&Yh5kU%Z0|{`NL;AQlIw&#s{&PlHJ(q<1z@nq9v|fvTEwA?|a(Di?|? zI-=2BPIpdogPpp0aU$v+1;&-+-5n-QQijXL*CrB|pv0vWuQW6qRwSiJ!fHGVhsh+z7AGPkb30HnvSH9q6qk?>8Sm!m%`aSkcrbO_&IOgjE8gB*l?4p!Q#`DV=!L2bW|?ngqIC zb7j~p{B)H`h-7;My)gwuJ5Xr@Un<5~?`uV?iZTzM0NAqk<1mw0NM~x&b}Ov?gIpzFQU>B)1NH= zVjlqz-S&z!*0Y;>hZ$RRri@@~R4z(!IGhz-W>0=x{{Xvfhad=WTkPhu8s|gaHcZ7KR=N$^b$fd z1WV?fEB|2|0eowPQ~_;S@fO&~|7?c#=bc%6_FO13pZ$uGvWu{s$+d^cN`(3tc*|=~ zgExMf_f4wI8`s1^gQ3Dz9{Zo0m%_%l7V1^7VkH625yJv zHwF-Q9w8Bh8Z1CEYC@ z(!zIr<9nZ)Kd=k;&Y3f3PR)r&mIdqWkOR^K$fSa}4p;*`qtmu zUPq4CIr5C*oBp91HI4%tgl%58W>jye?es>IpPb?1#`?ygY}p=BhRDv$*7dO6oyC|v zah-cY3#+@(*4~M3feuvC_oeLBwX&zPT?h6$UhdW8+pXYX5slJ0Ztb@!T4sHIMC_uI z^{u2$?&c01N3i6Vcp@j*5cd@jmj8KX%&pK4U7MTO*N6%y{>?Y&8ry4RnLjyYyLvql z#jn{woYs~QIYYQSu6bMMW;i!jjwAHI;MeIu`=>$7pL|=DZ?-!w&F-rBmUi5MXHxJ6 zfe#s(U>#^{Nh-cR{8HTMOOo|{U3iiTUh1dfK&cB(m7Y?HDjlr)Y2nu{_#|dRX?(AK zT*tL!>eah_foC5*(vbp9`CGJIJZDO-;Bzo);ZtqHV)H2zsr$W%Cp8STEAioOt}|BG zrUhKKD^lL;HBEK*vX0=duG4!^^Oz^*e2g>qL+GnfQf3U}#` zJWY&GXp3!o?r9J&WC2tLVFM}{J3|w9b@7W%40!T3I|EN=hF%98*1{{XaoWY6EPMU7!CF9{ z-PatB?2H|chk=4h-7d^eU!(HI_KWT=8kU;=*q(V@7%`+6-`TG+#`g;4K|)($lC>1u2q= z%vG1HdjBp$>ADnDoV;+PGSY?+Y`$=z63PcB$ptJ`ge=fZoOS-G0Y=S0j3F%z@ zDI0i2i=CA*)-Q8l?0IXT>-GuT!#Cd&``DF+8n6Q<^&Rx zBSOO1v^??(TEFjy>Ss9)%q_O(GKKJ;KYVInbmvZzyzMm)`W!1iTV;B86m`%>geY)+ zN2TS0ws^VjlYKYr`r`N`7Cnal!zC&8Vr&K7LCFEDx74+{<{|qA5&JL+EB5RszT=~N z^Xel5x;4Q!kL*=O4PM@lvbz^Mywm@t zsIUZA5h!K9bfjl*j}cSD^9P00hjU(Ob;#y!i7vbeT2YrKhp8-AUVH#4UYDcBaKC)zb1v@RW(qM9!MI9tDLqS7h~z8S;c`P6@-V;0rx@qz8J#q7ki_&^}B$$|(&&zX_dc%D-1iC(> zDbZxBalU^Zr)v!GwkZP<&f=94+fx&B3bOrv+YO^MP3bpfv|axI0RdZU)KH#xBYj0; zkh728H@={wAf%&1bV_#SfGR852vVMusIW4V=`Y%k_noU;yM>Vxg#4x<}kKqaK8MF&mk7``#Qc z0C0K6W0Hl=!+hK47jybC@VwDiHPrpqWc(c?r^N>fms6n-T;clnFVi& z__l)=VnjH_&+}gEnJJy->*Y#D9DTtQ{VmQ|`5`_wkvARU zfoqD{A{52Dzs;uN1#x!4@fc@tNikDs<-2>O(-ezn-_z!*AE+db(;017ZMhOg^p56v z;7C}s1)Sx_i`BsV6pAFhuU~OFnSJA!)v+L6)J8KLwQ$p{)L9+T;-cB$TgQ-r*^gW~ zjlr?pjD&`L`w`>|GwF_g(!|WeK?)F!L*t#Cz+Lq7|{>Unxq?;qMx3hfZCwXMjp#(L5n% zi258|#g?s*?cw1clpq?EtPCG#C8`X|a`E8D@W_Y(Gb42D=6n{_dUhG5h+a$91_R3T z&S$EW3~Zhf^0zsZbr`9MG9q`17FCRYDaV$}Xj=P*AU9|Mg+>Zt#FEEZ_Sk<2ydCb$ zb(wP|gFbJ;#pBSv1~UY9dMw`=iKxA7a&5*NHxV&m!}cuu+nHJ?-8xuG3hSg9fPX~R zUj3~{8v#`?d|h73MXMWv+!IuN&ns zPS3vm_N|19{q_jTprWA?hZk$oB(P}p;} zb72RX?u%guDLh&WQ43e#YM)B)OB{#SR$D>j+%8)Gm!sY>g6obe z5dHQydv;ZXqMuq)rRj@cP3mKYw<@zYC%cq!bL)khJY|_16zs_KqBAypIs@8v!_!o! z7+v}}_v&Bll3Ynn@loR>!QIy4TwU`TeX;2UXcaBiWEfO7TND)4$&nPX`PG!oiEoyI zA3@KAEKDe++M|uT@eM#REh`G(yvt&^EaAHuR&kYe!%f$A#7W{YP|-EQ496H68RC)0 zt~m|%FAgXFk!>m(P}XX+`m-%Pyag`2LmdgCn+~u)$XbCivR6e7d8O?UawEk~n(vL6 zkPZ9J5UHv8l5UdmvvJuC`v|b4Ji)w(nELXE_es}B$yiYXmB88&5#!3NHkg7x3(s1M z)0QR7@}$_uk)WbRv8M-DGO*3?VVTY@jsA+Dq2>$QMWfJqqoj++TQ+E<`WMjic*rm- zr8HtwecH$`st>G5m17)z0-PLOKXp4p&}DxS@Phd>RBSnJ8(<{jQ|CiglL6wO=}T%) zAU+kHt%ULLE_M+Lm=0>kFy?|HoQT(AYcQ!xXW;HoaXuCbgyN~wl0P{>p%tPfX$Nn* zdL1d-zc0I0>?^E`1J6WA57SWeMGY83$kHaJ62xQ>8$|LmQqCug7`3yxzhaBv#!xzj zBqdBGJm>zAjzJzY-nROw%=`ODQ8XBO-1IL$L5qFwCk$1tLdubpQhSebr3yj72WK;J z8OaE6Tdrbk>w<4tc_L(gkTdgT43pD8W!OLL9A)RljaJ%o>$R>KnW7&mMNS5e;|1~= zPFc@5c>LMD;S1<{jyG$ z?YKS-gZMLycreo+5=SK6jfuB<6**u6cb2O4+7XbD`i`8hOy1$)6^-W!8N9>Ay)sEj zOKb3Xa&>fadT0W9oGLxv4zSqz9jdaed5ihSob!)BGpn;a*rg?6+Ll<(^67f~UO{OC zbL0u!D6JN~^o3B?muvs|Fl(#dpPpr#Z!X2BUUP6uL|W)`fm2PK8!Gc{Gv?dNFvqb> z$7PGkqWno2$};q+1r7c9yQq7}(cR46K3~RXj9#cFInwRQ?vIXlC!^ADehN5!=zhIJ zAoVcAsv|xN5xOyUI7EFuT^6}tRBF<=*xx9-XgpOVZL24<^(FaYWu@I>OQP5Ax@WNr z6osHkinzab_f?!fU;Po`XVB+@K4~Zo?@J&_DVNtBbBr5uxhsH&2^J;CAg%~?v+WKz z_|krrh76BMOc**_cqufxINW&o3HvqffCscR#yoCeSLk`iB@$_Z{=cyR`g=D*0;By|dFIyj6U?IX*ISfsEL>AbqmyKjqk+H^#7-5R5g|%? zkR%=UwwHQPzZXZWl%&%~W+6x9l8B^McX2pPRKRVrVLcG}PFzX~8y9yluF0_XtusuR z+MI^AZH!_ovPF9oaRuBltU*}l#a|7A3O_CO)ncskX zW|V;UF5WHGjO`vhYD(pU1ylol2k}&+5#xB(G;w9uBl3ENKo$mCEoMrhf~G2rRH3%$ z(Na6I+geLz{jG1S3rrz+w2a|%ed0h~d;0T=8X`N8v5jUrCK- zeeyi{)Z**m|o*g8mdkH;)ll2 zFz!VD?n1qi_icc%eRNa(&hP9Wn0f8?}f#3*s6a!md_)(b>YfF98Ck(tqu z&lgFFY|=kB2%nXKjNS|8bNM$0lYhu5l!kO@Ki4&f&T{xH_ujw*zSpXGXCriN{F?u^1e_2duI|Z!K_%SU1OrKLZ*IvwINa_Ar~m+lztAgzqvB|DWmCN` zn1H>x=pL6>*F~}ZlMJf^EIiPpw5~1z&@FvGko2mUp%qSstEIEv#qmF`k63ua7iCm< z29c*hw{r!~bs*eUT&Q0T_8py6R}NfUG$3*$Uk&O=6w7ByO0fr?r>3V%8ymj|=J>C9 z|M$nHpFd)+%E9E8n>i|YFB%WD&&Od0q;5`L%-ytETW!e!?V`BDzBqj#DGFk}ytXK3 z0I%Pyq_42~%OUOjUBLgo=zo^;$r(myFqF$~dyqD*w!AT=-fHWvAUWuu#BmP91 zwjz!QMaY8C!8D*(k@KFqP3goZju3h<>U&E)Mz0+Yf;k<8ooonx zL-7H@)=@uY$Im`|UQ}{>L_n-Bz|8OJNDTGwg-Xp=7R~1M0u`lX{(!3_Wmfza0j;hB zozjG4@Rf*tYF%AM!iFe}&B)CI!Ewc+d`ZHF-2e#P_u_B0=Jn#qQS8uX&Mdv$sBUgf zbvgby-DLIYUr`&dIrO>K5Q;Yt8mI-qZa!fOR9<Dd1 z^Ydq@D$+mi3vlYQSu*6c7;u4!IJ072gJU<}=q+S$JG{rHRTL-Z*45S~pr)p7Y-%#9 z(_%q^{*RA0HaBlO^G3wQ4FNWe+7UpX=L>Ul{i%#PwM*8)p`nGxfBDNl(--&b^0KUk z#yjBUK&=@DTx6k;SYh&eU!c=bV@nIbc0-q}B|(2#*FO;x_r!n0+U|N<2vm2;n;cRq zpu8_#Mm@jY=Ck5aN62~lP0c(X*H)5gM%F1c?TKeXgW6w{wK!^F2cGAftTknSY7Jt? z=dR%DP03?F4st}2M57v<022194F@}v%I$0q3=1wZZjNV>-F67qvoE3cycroNqPjZ0 zu)avypZvu{!xDfqx{`l}4EQtz`ZgaqV)`Bf;R3)ySGniB4*zdi)PrX*Rh?@WPVj=6 zgqVuP=l+ICfu5@Ys6hA|Bk%f{PLAo^n;|7PzK6kn1Yz4E16gJD34w-wysf6#*3D3?yj`%^iyN`6N z>%o<&vpNLcP5`eP*wBB)Q7kD*ZycsYmAS_>xI-I{m5n}Hdtt)#0We<>fAY|9uutT} zD0ZEwgO$FAv{_B=k1-$znuvlz{+)0y=*s|3EwZ$K=YHSJE6R9cR3hd|Q6%XKQTYRF zkjy!IeS#ypqqp12Y<#wj#ha5JKA|)_)0%32)*1lI*@cxc9iu`Sy0fVRROyCi|JIT! zril}1GRT7RvI5jLa@IR=FG~h`vi&pqx>cFdki#IrvAJn#X;IVD8}Ch4eWnGJgINS5 zIg_67X)5kTNHiZeRvLjTvtOR7nje$AV<2_F`dXpKW0BbjVYYO1OrCg)*%WkZGw_c54{hqTuxh2%bU~$#gt*6Q*w(P?>at>eqhBIo`Z`e73 zJo5A#g>3%N?`tBJ^H?fC$dTMpH>q8Kr4X; z`f3kw>t5Fy`Z~?fVFZ;-Tw;pt>FWeWz+J4Gq!ne&pB>%QCqIZp5)xv=RngzG=u0XK zN>(M72Mzd%N?nM}S7(j3=RHZsJ_dSRcNsDlNkEJ%zz^RUZxkm6ZMX3)Od-b&4=kne z5#>xUL(!^8B{*R$pBHbBh)Mm9kslSi5=T|NPc)hyQJcB&?@0({p^%bQ=>{AM%8gO4 z*FAGSu0X}**+Pa`^NYg%EH5=V3bNx>=YL<41K(|-{Mvxx6Jq|0q}XO*h~mDG7wy2M zxx+}hHv>3Uz5yd)IR=vyk16V3__43i@0Ik2IUAfOTBo(DwM(6Xrn<=Ok0l(9Va84N z@&vL4i(&x)r1vVxs=(zVSqTojn(SrX6y`IF3qxL0dSa!@s?+a`)bMl6 z7G4LpqPlq=;%|%C-hdS52Dz@qN_qPnbAm7nd%KxNf|wxZ@C=w zItGQy?>Eur7SFhGE@X2iDtCsqE?apx4ibktXG({!jm4<|HO_2o7-d0d7Z2V5TZ*o2#G=g}nK?Lg!S*lYgK=F!|1r8Oj@r&g8e&&nLSe85&Z z2`+Nwx!bts&O2i$vzZms{cQ>FnaiJ&ggxqUGbpvVZpjV$NY-H8nblB}8E!d(%->8C z{mqX(^DBAI?Y`&FaUcJo4dx(W|H1S#UB`X88OpW%<*zmRlh+GEy@U;c!~CV$PFR+{ zwEssDz+RI;^?W%o+`}tz{t(`0^YL0C;3x0T4xy+8TwhT$?8WEHZ2K#5Pnt&-D=0$O zD&xoNng$1}camQ02@kX^MyvLPtKbixItM%Bm?@lY$bW#lMoMo)tGRtxe0BJ!$+{<^ zt_6+VItlE;tOs))KM)k;vsl>D?Ie9VGu^6J{hUb@@ryUpX%L;FZ@%w<{bgXMhV!yL zjG26?+KB~_s=bp^pn?Bk{_w!Tx!N0DTF(A)FH#GBIj-EY0(B|{rlxefJj$Y(l}x6; ztnDk|AX=k}8G3M~#-wq*F)NF1xrBhWYH;;1In&kk8e_TAqM>_bcZQ$5O|ELw*LDd|O&nAS<)sko{5Lfwkq824VSFYk5vs zp)<+9faPF`#K(jr_}SQJz2`uBlYXG-TtjEI!DUY z(NE3`%70K+dR2^-PJWE6r8*w`@hh<2k%^{1k@2*~2V+0cy)zu|Gk3q#%SmGT9;>*; zJ9;44%RVU(Wqw8FD5j%bM--~qnq92S7qcR`72%a@<|@jW^yJ!aVO7`+hCT1zNTIj= zp2N*YR}_ z%|>`L{PZAff{%?{=WFa)abY*chN`@X>q^?4Z3&HYuAJ42qK-8IFe%l>r_G*)+sBKI z&AG+wNtfxUC_!Y_3(ErxAQ#rdMh*sbwG~s!_rn1he@+yIeliZIHxWE*0d~njEZL! zq+fGM>ImUz{CKq;m}?76iDIe{xlu#j0>T;6V~AeTV@=(4Y>2e2iysOF$ZQSAbRRnpWH^Yhm@4wuIv>E2K&DA}0I+?kDJ{9IXMC6b3J(^QIHPEdce)*jf1 zu-W6Wvy)1_!3n*+MxCE(bH)c_UzIsZl)V)|0|}t$=B8DlEyXdhYD4uJv_qG9@1*g4 zu?4+noVvQYXkwm3cvAN_VUXUGPx=uH$9pSuCHI*k^VK%VC(F_g4YOLe^%K5T$sb!1 zW@p6P9rHO*t)y8L8$?0E>_&A*94(xf2y`eV%5=&|VxZ)=$pLJZTJMAEKq1EHCCW%R zNM?;{Z9bGb3ykI~K`j@}^J9WlF6uvz#=wEj$W1W{|E0Y00IDJl3E=2~02FA1?@ctmPx;aBO?gDZ>ceEa$M11! z=e%WPGdksbKoJ7eCjbM<0R_Tp`6$p3mZ^Ky9$_eOQm6>`TBrn@`A>5OCFh0}6KaaN zFv<_PA@mFN&0jv2el;qV3?HX3Y7iXP*5O-WRM=P3j(RYxg0ouRoM_#LplJezdnE>Jpj9a zxb8)P2!n({D3BfiaA6RvEWkcDBmjTI#WlbdgFq$sO6dT3NM$f*G8Nd4G12noFDik$ z3YF=EK_O>H!X4j2uTI-7kS6k{W2a4?xBAS{!&;c({zi7*w4Vob(wkBVf}XsrE2-?Z}O!bOI9 z-Eyl;T3UK@JBoQfnth@NaH~KvF5sRnLw`2-S}|?_fmH((@?dswREiIdfhw^is0;jl zm`qOm7_C|Jrx9K3Z~|}g@ynw*Q!amOwZq2_!S-;Pf6%wd?|B}uW@H);xvO|fYsFO+ zP?K$!Wau~YQ3zerrZ$f!9)Qx%T^n%YG07jfruxzVhET%YL!2reS)MzSsMqL72hcX? zR{5mt9#DaR*cAb1n2otcU$mCVFJ?2aMd1w%N_v-1@_zdUQ`hLxajShdPx=Mv0@l4a z(K%z<(20xBgE1Y=CGpqHCn0fSRaq{S>~hFJRsU_1DB#QKRN@Ou`Yv_V zd-KiRg~tF{W+z(4e3q6P1v}6Lu=6OEe}O*Ny`^Lb>WJ-4_g_{ZtFwTYibrVz9N;uY>wb}YIBFqQ^3Lz}MmU-PWtA?Yad-2}azpp43(!O@9%99VNgC%Q$6 z!ofAu@0&7jYos1t=Qv$D<0l*8e-Ng7jWiE*hW;zP!tkp9VMCM9-!`#zW6)j{!G z{niS+*XPFioFfzIoWHbvwDd9f=zO9~M-#tonzV5+r=znpaH1TTWwLU zKplI4jA64%qAI%08fRLQ{cUeuL9*!o*O)ivG&RP;@5R3ZPBu6uHjZeTD zg=f#*``)x-0gYn!-o(!h@j*s}u0+@ki;ysKtynqIhht6AJ{>)50EUJ>Lu|rK_K-o} z{TCR_kx8*xc5ql8eUclg0K)Nl7(SYh82BcJfPQBTN4(BbC`PHZ21+@>09k1H>Jk665%G8oZcHySA2(XvAJ|DvSzVE}aUOEoa(f}PA3xs^{`RU)pF$1O-YH|r z8V3p``yv2Z|CGQD_YoFvf5))iYilUp*a_x%IxScA^2?;VzC?I;r96p(`u-~>+E zNQo6N6x4jY*^?B7puVYU8UINZx@B0fT?%Jl8%=}A$J|O zgygj-5g#P?1!F*k0XqQqebH+$vY^}PliiqH(fC&deXn1d`ZgbTVn&T`WOnMlY+l@7 zwwnrN(lNuQrJ!>K3G6+ju zRDE`>HGJd{bco%pkSW8!>vw=8@W(eib&WxsQXMomwqPi^B>@3<(ZKpI9{X-pSrANM z9^iiY>j|V(VKifKel%_ZeQ`k;K_l)AzjJj0w!HTxi9 zUv>0lC9q-mFrJ_P^89}k(SL6Sfg1=&Q-kGc)|mCNbpBlb>RwU!VV-RK#ALsu1rP)_^Bc@TNDjU#clqv29M&`)3 zzls=dR~uSLa?|(%MGN<@#@=H?Y=1yv{?UmFtjfOQTV-W&T^(k?<{>i7TVfmD!1MjW zh2W?~AZK@B?GIINTAtJaKGFQjsD-UfvGI`W2Y!`cn58!oa%$8ova%Krez%*@f}>9? zH-F9v&T~DRN`Fqnld#!`tKqUfQIxXyuQq~_jCMEjtS={?CF)m-9_DCu+@Z;d85FfoPENy9q z`PA82Nzr*#0x1O=Ik>QQKfLpbhHvjTGYWm=Ynxk*es?I;5@ep|7a+6o2>Y$R3gO>I zv|oTQaG~z+-*YM=vDt}I*T#^IJy^mf*<4n8Rd>yF7ESD+Alx|1|PgEP*c3kA)IZ_dYIe zZh!BN92{8v7&sJ6PM4dq`8^U>MmjvM(_f71yZ$#8K*&$9qT+qe<|es|i_6~v*-ENn zV$kE`<3L4vn13IGZR_j$va+(_QBga0O(sKV>M+%bW*G$RU>1jMaa&pk2M4;yz#ena z1($z+7-o=1Z{b1gD}pYy43>TkUd=+3e@_f>t5u*J9vk}~r{hT)3GOxA+)>5Ez__;u z!X*%~Y4R&R0IP4wP^Jqmq@#=MJEg;l$Uo+V3kqOnQ<$CNKrNaM?;%44y4?&Ii){Kl z{P6{*c$>$qw}1#2ls*oI5p*h0MvzcZ#(@opQ&?~TD)56F2m&0a@llLf@Z-&ZUulWa zC}pq!#E{!J02@AHDw;?I^I{P!N`V3gO_ml3kbsm4XI9NYn&|qy0%r04k8Q@^auuf*S<#*;!>1yLueFi%^@awK10~C z9yw+71|evT#2Iefw8UN6G$Kxp=xEQ_i@7?P*m!KUe`8kssW!ya?k$){`sqQl@Ju|> zxN6w@F%a5Dm(ahsSln7N<5E^q_`ky4^-lFdwL;3qOxKOx%XvydJ{^2+bA5r2#?4FqvXa(vkqCwl@eALsx-vGw5fnqu~ zK2B1){J9ksd~c{SK9oTT|MmRpI_!|QDZ-$7X*gt^Whdi*(As0x#o2tm7?IeW4eMcB zJ$-1yJrTPDJ=SGH3UZ}vffqWsI|WaU!09%Sna}%My#E0!r8sYSYB?J z#+KMUce@8sN&$oB8apiVx!vZoAm@oq!E-@ahT)#T+w7ID)T$^<+` zJa-Q2DjAT2qWyc!f|^#eIy+;DT4&R{?J;35gRFf$X`8j9GW3$ee~{5l^`$PjVP!kh z-pR+BsJpkOL39ju#p%;VsEQ}|V&5g%|0TYQA%&(kP;N^$>=I{TfiGG$a3%R?ez&+E z+iuedYpAcrC&j}0gQTEVad$2%k*nH;Fe%R^?#p8N|B!?(F6sVu2rr6^Tx)`&VSdu% zqhcw;OskJLzJ<;9Nd|j5JeOMfB@4C*inMM4s+&{wK7Sh~jA3g{M5d<@>@1Dbv;Dgj zG|r?p7LpJZUZd`~UQCj((P8Oxatg_EZWDp@cZ1^3&?qoHV4#oV31#6?ex^MyaXKm+ zSAzP7&im%7id@-IFHDW8Atp8A58@D;xyx7^}!f3NAtlr4IRDXq3*oh67VJ@>$TP zia3b9f}C=tMq&cuYl0N4^!xcwDRAG>7F}H)76_u*x@W*W@ssLm`A-uLRSw*-F=B-z zFf~_s1?=CO@i()O1))Ut2FoJ<$r0E@FLLsVL}^ypc(?n)EF@`Jk-?f6IAR@m!2Wvpq71p%J+tYVRR>Cn)>~9$j+V$x$wA7_s;m4%<~a;LmhOY zaXhQgQK`#G(w-1U-q$t% zD%)^JL8=uWg9rr$6>zDfDc@>r?c9QffnsU63H}_v{zCEobFzwL7INBnaK8p`K!G6K zA52M*CRd#+C07L8(J=wl7nUt&ULfU&T@Y=}SbRHD+9J5Xm&{0*l_%UGBLW3|W^H<+ zM;hP2k1HqV!+mtRd9<=69`?cs)A9=fURGit$Aic8{Q>ni(Hdhj2 zp^$Trg6YZJnB;K)VaUjhQVVGoo4iaS`ID@eg(S8H1MRyU)@Lf;9;CtEz*@daw>WWo z;5)R{y7xQ_Q42umL^WS(up+`MV&zt6@j%w%Z_kFY@^Ah{7)w%Bm^CWN%TtDXu@-jL zKkpYR0FORPdRdgq)04U$HTK<{M}GsP|GY*-wR-2HiAyVH@BbaR8M5Y!J5vpKb1H~F ziZ@PK!$QD5)p%!_%a3pT`4XnDi{cp~hGK=pe&!uafNiWyHHbRCF<8yz%9dQ`f9~zU zi@R>r(%`&@ZLM|9P;)M`t>cDgxQT3f|9<5zo2uYhDLujRpkf;~&QXKTJ_6ZXayXZ8 z4;_<$w78@s;iXbLIsr!>Oi#m<6v|47qh3UiH6E+7MUugP)grX%N(({)N1?W_^`YXx zBa=RL-yiBG3qGUDK8VSDrJ;AlP6MT*eyIP_4{k9zAW1b%6}bBJSB59P8`I=-<>OLi zbKkXl%~p~MF4ePh=v5iJ!^Pd+ip(C_(|w0{Z`rP;q%Q-p{rnYlOAT$&fd>gzs3AMkz;(GyVm&2 z_T+;W8%-Q!u6N4NS!OuvRBr)E`&S!$ z(dKVjSl1csUP!XD4z6Up5zH(-ZAI-D3kUWGMabcCC^Pof^v8-1;Z6QEPQmHVKyy@9 zk~9WPQi}q|b2!eNplEz(_nGkXV>n}ibM-F!&~o1$)aDOr_oH4kmoi^gAK06l#g8#gWXWFEhQNo534P8e^_zL(M8uL(DUJVLVD(nn8L zwM=cRj>iD!UlY4*qQF?^tK!U9w54EuYQu|j{VLSNoUS32F?E9lubG-LPq&SLMh*pe zz*RdqiNs{DB4J|Idc!r4LRix{!)t@YeLR}oCtF~BB)AW0W+zcbPkH#~$Ah*hUimMn zF>a{yvvcfOL*329aTKC=Z36sHb3`XCLcNJ2h|z?<$cp8n_P%{f-D3U1p=rCdq<74n z`Ao8JdL0m`*JSN}>d^=?IJgY)XjKnK(Jj`|KSYH1mULB4jm*%M&0*R5#|()# zI8X$x&*&{Blx<13FFMhNprTi8T7nM|ex+Q|_6y%-CdHKMfR5b)MzlB-p9C^48}&f% z*B7VuB^@F&QAHGm6mxuO#i4cg(9BdL$xzn@@I61zEG{ei=m`Au`<#wHk4Dxd4e6OA za`~N^r$gUCKa`3~A32~Q6qO+C3@)D!UxqN(c^7-W-fv4V;jkAmY1o)fW(Z2|jd#Xb9&!GaSb~gO1^ThJJ}E!N zJ{0t#@dj%1E9b_IzX@(js4iNX)vl{RX_ujxatOrpeS&*BHox4Q;|?S-cR&99@#}2Y z$j&<+ZVd!<-NH`?yo8;oDIE`ktZZpO1R<=NJ&~dR-GYNLG@sI8<@3Y>U8DwLjPnec zstlcePX!Eg6W*7MMv=4B{9^At`@9!t zZS{077n#98uomV0XRYfoMe@{3#r@`@opUSjtlI0aTi4Z_yUNxuc(PZf*ZT%MZuO0z z(rb#h9AwMm%ajV_R>j|+Nn7;qgGFJs}jJ~*4?mSdrH0+NR z6~0F|kRlWDg8{y6?i`6~_sn23Cw-^ZV8PcMn}*Jb4ciw~rB3RR`AIv z^rLJ%>0`zWN{WqFEIZCrYs%Jj`;t(+yo3boyLYn@$2Gf6c<3|V*h`OBS2}(VenxBQ z+0Aeoi6SyxxfVK*dcAv<3-bB!?wwix)3Mfd18U=uZYj2nKp%77hCH_qArN+IDyz}) zyh_6qdl*eSojOZQRF{~L79$3w^O+SJXe*anZN}2~k#&IKGY4#3z^MAR;{of^ikzAa zvTwCLtw%W>GsR$Gpz-@oLpF(hOvR747rmm5^Vx-*x;^y|0K*&n%ETi3B3$x#wW*1n?1^b$C95D`J$kX$_h>R3LC}}BKAgbcx_wda2->2adC?-CUY-!ED`2-k2 z`@jBVcTI`VA$i*SQ|cI)>^!ij{WxH&BF}xyVQ$!QrdKIMapO|)*u>EbU^yrJ-v0s_ zYURK%QnzRg8K3NuSEI_gibO&(@v-(b5h(lY?G0WXNj4jtt-t2cUsVZXE-Y}0VQdRO zG7IU)CpN`NW;M4(tb;>n&I7xdgeP9{sH=}rjm}MkmT^s&vV&VsIog};42QNjXUsKO z>>Qz}_o&@X6#Buqtma@Ij|g1BXwsvDo0p=w7SsZ$fESur>(8ThEei?S963iJ`n*7~ zAf=bnJdaz$c*}S1vm~yeFQXatW6d<@j`d;A}!mLGm`JDtqykM(GvIT-vAl0jw~%?32Awy*{2l zVJ5OXnY%0fo4U_maHjv!KuAnY6a)~7I!^iA9J*slwOj4+_&B~J{nxK_VmAC^kH6aw z5v~dfMaWRg&{fvv#Hf%83eXhE*7~BlwdFXv10%2a8&m&0ZyWgbpL2e~)KTxv*4EZ_ zKi1$ptuzT%F=Em@-QD}$)Q_iFX8cIiSn9*yT@@)TF12}EaxC6Q0r*=Al+&ff#o%ua z-;~{v6?EBPpdz4gIJwXA`aTV^^0%p#r|SpLIQr++@I?y|!+K}0sVh(w#YaXWeTY=E zs*QgU`;XZKK}-LVrI{>I!bV`3NlwfdS#j6n^YoIqelF_<+pw$uQUZ5Da;(C1~9 zey9ZYPhEMJ!1O8e)7R^xbQ5+EBQv!~Q+&F{Sa#%5F;5H(+zOA!*T_^~p>x8c&EaW? z?ja%+Zxy`n5PiFte8ty%Ab@r>B*~fLucdmV!ieTrItB#Q2Es<2lMmTPqw1i~d?enY zk>tZp++mR+r0`=l|E4%IwXUGx#D!73H-<<^2Oe1^&i-J6!o(cGx&Cq0AvcF;gPhwE zbCjNDhx4MqTaPh%Q!2_Icb@e#wzNG_Ys!>887=xJ>@=>N6L-o2Wf--@G*U@>lz!8d5zJn4$pHXYlupGs2rw2pWSEP>~kVobZ8Ful1ubp*17dQ8{Lnb!YGi@ zwqC1?Lq46oq@s_mFELeC)s%D%#OyZXbfJw?CTIu`DZ#EKFf+z$bUZvlozs2BLPShk zYy{JWZo_MT1zt;LE3x8yVQxuQV zc+*{)%MTFFan5a4EcxbUKQ+VVQ8!?5_y40XcDq0Ckl#mF= zkcwe^^$^y?UvDbDDWuL)IpYRg$ zjA6GCT67J6XXeP$d&Z;V72A$lWxL4l9lMQD_abC^QB?gFG9T?_s6PV@*(Tr5ftX5# z?&-SL(-zzRq&?T{Bx$!a)8~!wy=S zC}UjoNvN{&9iT`W`+A>^mCB?qT-iTP1+R9l+tav;h;LRaZ?`#jZRgZ{p!j2_RpisS zQSAJ0dmu@^qfU2!VDuvvQb-h;k!C=Q%1`Q&XA8ZL3a7I&W3`u0Nzy>Um!l>9p3xZ| zjclz2VU-a!Sz6DjBAK!5hNN*+^CQ4Ng=1kn8B1x(=$TEB)Pesd>*qv`X7@zE%zNC@ z49r%>hwsdFM!)|bTVEYkbr*CiAzg31q2l7?hfg0 zk?w}OdB5*I_n+(YnCJM59W!gz%w97xW4FSYvavwIY@i*Rr5C^cvLanpS6mQ@W^m`V z%HlxJ*Or#-J|v;%9>ZH`IsUZ$nwJz8FN8bB$FFN^bE7TRUU(k{h!`%or$1iZCJp-S z*%xgRt+-{0Rf2^QJWy)v>?tQN-(|8MnDSJO-X>0N2Sf38eGOOO^&89keLsUm}fJP$I0vY9eE z^Ib!1TqQbK_PidR^J>=MB;^DcM118<~s~}PT}%JL1};EPqy!U*pastsJ`k> zbY7s(h@&a+gcSL(o#Z_psSFy_MQ3<^U!ViF#H})b7m*h&ha77V&|+K;XAZjGD^fHSJ?&^AmO3#gi0- zy%Tatf88@Yy8k9ZamdV2T7Xfxe6+(gN3{PzlEM1LcpsB)Z{Q9dyJGYht?%3SOaeMy zp{%0N&7n{Ejo~`q`nB511#9dV6(__H-#D|59Z(&f@KqlFlJq269@&7>76dlM?9EqF zD)zk??00v1HRp(oTywZ$Yr|GAkdSyCm61!(k)Ep|T>UFm_jHTP+n!DK->n>r!En{M zImUmO%v%OOiKKXFxJ1LDdVsLVd|h zBsbXCGp#9!dFTk*6;2O^l*3i6>o2^;dHfMbJryk;g7X^(i|f1e4_%lym2}s4Qq|CV zL^!VLiN7M%VM?-eTZ;{eC*!)Xa31yZaqL-+PFb2)Oh^Tt#_Yl6E;-L0K1ILh&37bA zu!xqylMxYP568~Ytd59V8;=M&3tcY}s>zzd(znlK(-d1UL0=t>xPsc?kizm&yqQ5#2PB<9 zvPery1pDHO`{4ctlpT#={>0%s<&QY#=U`XpGUYO1zu@Bk9)u98_f5cgi{{~njL`&1 zY@U4^F<&^Pqf>RXF7~mN?siG3VEgB3Z5LrHI3q)mIN^#YJ+jX`M^3{R82PeDC1v09 zn??Qm_~1K_Dv5171u-?X4lQj={jHNC!>bulqHR_k(;fMv!SS$E3q-{7F}EC$E-T*! zIVz~U>0MH#&})9Pe2my&xDbyWlea3ACVt;!cJe>!cpsW3DmCrhj~G=)4d4@0zZW~W z#6yHFz)}f{hFXG(>XY__4SNa*u0g1XZFZD;8Y2r&UM@ILS;nSXSM8s&?3=DTTiKZQ zJ)J|B9F|!fsddU=npzuBv4c}e+C~}f{F-IOD)?AslNS#Ki|Q2`g^Dn}mT9srJ6@W0 z?;3acC_?u9F~;Kw2%YpNBH5VD;^a*arXM5xZYSVno@2cpFv=refpyT`AP7c;ApgC+ z_1jvtKJFc3=h4gCkQMdH!5b>W2!ZO$ z(MPS-qGasSUu|ifTNW8uXy1e>b8<3`e$0(wyG!NQBjIAG^<*o%g{s`&j(FPd!p+@; zjbSR&+b;E8`dw9gPk$Fuj(6A7@|X-7e2?k6q=y>jL>>}U{op1+_%AV=2kA;}-J5mc zKfe30 zE4aqsl0&WRu+(ak6=#;m9!94)a2+|_F&E?Jt-q@JOlcxQUZHE3!&&QQJ;{mTi>ak> z_~@i(z{Laq!R^Cqm{W;<;Wu64mTQCM5Mr%3!-Yt~kqdsS-p>gL9Lt#a_^0P*$<+f>#Z74m`) z*4Cp`xgIl)o>cv1BxP8jsDOX=k1~H^oV*%wAvWLmi~}v-YY5(N$fW=LX28TEMreEI zTY{y5<>{{YEhxuur!3?Q+oWnMKKWmlopJPZmi)0WgM6eEB_+*9PgpP=J*0v!^;@H^ zSGuYC4^lc2jYxFJa*a+(#U`GeY;&lLXA84!>);`#?@Co(u4P9IPW zSegru-ud3iymFE^Y@|RiYc#B)AEBpS3~oJSA&ueW=I%UR?|C}sd3=2QwX}bqX<@$i zBBrAHUW(ybvAKUiz$0PLJIMgk`x%L>I_v2Hj6sMcYC6|1!45eGTK=`Q@1yiz{-(TN z4vo^)xx@%_je!(bdC9LBUX6)0Gquf=&!ms#}%yH4S)&KolsWV`!q&q-fcrB2MEoPsd z@qcRpX#3EtyQ1X!7(bhDF)n#KbMd$>!bW1HYXqEJV*2*z1s1zkr`+M&^am|7c6DDz zK)ZAVJ~qFIT(2$Nzhjh1Z+&w5eVMn2T)vl*zE?wGjOn~lUK+*mM)6AC-?M6UunB`Y zjqgQ#A9mA-`yZLu?#~pVy4*Kx+Ew>oB$7QaqE%0t{MPff%}vqyPSy|a>4cW}+ttUCP&k*5B-}l2F z@>;H`Nhr`|sxi^QEn6M`>o6LdmN@-nB69W=yEw#>DO(*=ATCTJ?#t0-dtaQf)nizFS! zlyCvJlph;6@GrXSee9=NmvrITE>HXy|5pC?O`L5G(kz!dzV3RV;#;3qTK18@1&SOo z@KgChxFsaLVuUu=H@1A%Kes-FgAuS=kWD!L-`Q2IEEs4jP(>=mTys|5S&x(<2=|Lq_7XGh)aA3Y;j9+}_ z92a8ILeKI7UlwsODiS^x34usBRKB-r&}u-!I})Kzo0hjJTq14os`xY^BO7LTq4Z$! zmdwGyDdgL^EfbDNEnw5VzISw*vUgpmd8;*QP9 zV-|6c>60u8#hxUeV5KgGoHyba4uX#)u|cJdrU*}F(b&Xt|eK33#;1BIM@*%Dld^rutY%d z<-rv*ZTw*6kW$Zo(NboN)p_pg_<$B?B$`@zVmuv*?$l_$;2Y_7>QT3HRmY;?m5+~t z3Fs*wXMUJqH%#9J26sO84dRP1_zv%TN2T#;fBr~LY>r9;_a{Q}dTv~OK^c$J2nLYD z)RZZP0HK?!z)jp05i*kvb+brDU-51-ro0^nq))?m(pR)}3zav4rU>DIJbvW42 z@$#~btyn6W@8Cb|{PuFyW;&d7yVJ`3a7qKpC;dzkgH-x(^n8ys1f3~2K6eSE*A5r@ z+)bQ@)!jz49pXZK(C`}{HbCpoglnetO$ zK*#X|=C0ovIgcLF%Aa(4de7fy?8S(bLAW@e!1;=B^iA^4_5Bhr@U( zdk{C-j3%m?DW_LmNxJ`6#spW7@S5sh++9S8QqSx7||hyRS!H;jq2W!qa#H z!xITgqceKoda#|upBv8k?DQqL<>4S`qqvrMGgh~sFRJc)nA(<5^N7(BU&^bZDo-^e zQK987N1QW_+mb|m;dj{1|5og~ukbNeNtwt()6HxPZ}gEpe0D2pUETxorFIbp*wW$)2C zCO{)pq8U@yO1#MZ+}Oetnv$(LKf!mfxRHOS{N(snfHf-Monv_J=Vw?R!gyQ4tsa`_ zBlds0zmv+rMi?~a2LtdhVRG5g^6)bCUk`ZL`*tmbe{^qfnf-Herw~=jTOKAZKn;7} z8WwG;o^bb?e9#mg&JMdPYu9XPX#(ZY) zG-uh0$l!9+s~z-bpiSePjD(=#lUZMP1@BYH^u4DcSbm^DZOC06EmzV+m~L4>9cK@2RyX?m9<*G#4(V)~Dha@q(z}VA#P;KOS*d?pAnLVE>h| z;-FRHE($|jUg67vY|m=6HsAO4G@!C4zaKY`(Jnaj-r?w^7Z^y*-+V2avAd};F*epVTWy7k2%nsqIu0}# z-Q0M_-X))%xti;4UC@}YXL8iKwg#(>9lPaj9#yi^%puwney*rYg{X*>D2J#){P?!@ z;Gr8uCVT<;Bu-tOT$~#W`O5Z|l)!fz)J07UaXe9OjhR0{%N;H%O$+PyQr=1B$Fche&yOd2U`_b#U5QULF-l zG{`4j36qq8qT~I?VT0&@+bNXyWDg`pTOgO5ewF+e68Hc)lS|g)sME;MG43uoVH`)j zy+`aI6+DE*u&-g`L~zPXOGIBG1oPymWG){TH#g^oh9Yrta;j>j01slWOAk7eWDSKv z<)e`kX;a1{cFuBVhl;rLEvf2`Y=qSlf>H*!X^(gCGPc4yK7HjsDXP3HXAdjAs78II zn`Cv=`udo=ejz$>5=!WM`4M{Xc4*8XiyxJrfz85#9Fdk0#+W0Mhzad#`_e#Z>gY(j z&6>Mqnn{-Cfu@lnMb4Bq+spcjI{BXzTw;Hb#_b(FuT$6eiU(AUB+O`a45z`5#v~0F zlT|hv{z4QPD(YkKxksKeKemapGI?5M9vYv++)Aj#)n}O~QlK^3_NI6%!+AYv z5ju6uySY;4<9WPrv3)#njUwM&g?+HD)9HBN%iAh366yLLNx4djis#ye>%eiOmVv=r zjuwkCy6Bpm_qeXM^oHa+2XYCryl#f zi|$=3)%?;-*Qo&mhk`#>?%WYcn#=tWV|hx(Pyw6$mRn1(R*Ec?Nph6AD=isKC|)Nj4l~ zXg~WmcSi>0p)3sC6%=ei|^TkZqZ+0xkCs^Bw^oS=}qakuu1ae3+Bn5 zL=LzFG0OAeQdFYFkq%b8H~hUu_1O_44~abA!j9@SHM2ry@JH*~GXzs^iQJF0HwkmU zRhwx-_w4y*HMSGcbnqCeYV#8qewVM;rAlE`Juk*ZcXiQvKij0~y1z9uH<=ybR`60L zJHjhj$?Z5}%b6djEB= z^GD-aDhlH%MnSz^oW0H|u`q^r@RPUZ}3{E8%C~ zM^g85DH6N`>EY&}5R2Ive6gDNjHEad4Z4cvarxoF!epg9@u7(Q#w!VIti^CzZ8}}G zx^9W3pESx23+;-ML>WHYxM*fggE@RTMoo1=+dfttjP!ev1iCzqA+dlmV%s|qpEfav z4h|N+>Wwh*cCx7QwkEOi)$0lw?t&JQUFQP>?ds}k%$vo?%GzyN{rM#t&~bW_T5{xn zs_!P_ueNmdBQ!!mKtOPHcTddDhKY;ARWZ{JaK3*18f+lb|KQ58SY%!&9$0fejN0_c zET-^pbbFbNq$~xd89$}lKy^RA8^gnQqSY9O9vJXM*qEl@r^u!c3E>{BK9n@lhR$M_ ztr>(u311&0{)YB-T8(V9?PchXWa|%>hDCXaG8Vq_%1_>ft*k~we`NZd@eUdt8eRIb zw=|H2K3DiRa#&05xzO)1ywQkiQjVWIqceOk`)mSv_}8V7h``d5&&=ggoM?w@w|=L^ z_?<81a6j$7%ann=WpegIE|o9%Wkt+_R)tYdNS(jasK|jE@$L)*^Rt=kxZ6*;;CIP- z>M?|asjk|0T}DS`ba))SxCLkL@Vjhj=?MRV$|8`ii<6wL%e6gH75Zkn2W91FpdVQht)O$ssKwNp;{6~c zrp+?z&EZU&VWv!Y`I7z>2M5QDiq>Ze0SP$B_)lM?fAynvgMSpp3(>y{(M5|>E_2mZ zHlDv4q>qS`Kn;tKDFEE0uP7vJm>RA@M!mtd<|>GpA}5%7?%y#+M)NQn=@4=1nq+82;K#?4dE#S4LY#{F z>_hL5B4bj)w1IyNoyeB67``xfL8vX4u33so_~EL&HHRBQ7wWY`yE*-huW6y`PTrAR ztoj}-e#f%B0;r;tR#&w84}~qQHO}G0E2)F40f&6JFnXL=l+>Yh?8l$E#)*=qgN?i2 z{`{q)ZtX3zI9NlJw9fi2#w^WA3VkJT|KVX{gb*+4=(F5Tq43oTj3$D(=c=S>OXLBH zN7cz;`7_?8`RcXptJsle-#n2nADo_R3wNol*4tc7IIuxe*HF|nS!>fAhJVQgKeJF% zcwj+Te(BVpzh+-Kq+`3Ln8xv(8L>GN@Xw;bO_u(^= zw~ocOG8R&KeTk;$fgM0r=r!i1@1~1MO$X`@CW_bVS;1+P4)&KUEOcya*kwFL)|>?8 zyC=GIZ|&C~OP%GQSBMrVPKQP!;Sd#LxZ_A^@9p{1@GU6;Cc}>c%U4VK?TqLh3Y@n$ zw~UnJbOmihkq{1|%6ILuk&+TA3+Fw{;$a&8@})&7UVXyvWq5S+@sAr#X$WP*rF}aj zn$<=-Yxi-(d%|Th>-OinFqFPHRff|S=yP~bD?|I#2wVN7B)8NaUR%WYZOI*Y6(I1N8+zPM0I$JaLOWAPZF{~VkI zK?_Oeu_2_9U?+xtD=&{dIdO(ffWv%VU7ILram35V6fM3tN=$^zCZ5+Wwp`%c<*Vs< zhr{iZG$Zq&8bkdzNqRzIbkjzXYMKd!IT-hLBOom@uLiGa*!|nVz<*>DfpsOn$NjY5 zY^3V=lj9;$i?qqe>D^Y!YyJBn2X#sk^GRBj9}8uh-?+U0FuCWjf)jw>lh zZmFYTQz*yuJ-3Ly^u0jHB_3QIC*hLs@=iyizZLS@j40k2(G_#C4{qP8`2dmDM~1!qE4rY~&54u5 zR{JyTkX2yR{;m^XH?OIsH8wXFxqQfw z)W=Q`yS~DYRb$7cwQ3G$5^H5P?=5#{qV8oPvSF!O(X0U4WMJjMz_`gD@AUHvh+sJx zNChWtB0FEX`cietrW~mWkgp@-dTaWtFs&EdiDBmDU^^NR(Y4nB^#|u(wl}A@`0yUi zy{Wvyu!xjCV=RpL<~!$Zy|<%9eI+QJ=qR1ns^V=!o%gMxIn@r<}4| zKT|W*#}U501=%LHn|~Jzc_mjx)oU`>ErZgP6b!y~hCn($wGx$}03RE(?-;GK0X^mK zud$0olfgv{fOT&XOEa^{Yj@X(KbNNq^6#$U(UoDBU_!B{4;q3rzr&?mnc>BfMa*Y} zSBrJBwGCsOcw6wRyT(pAl7xn>j~c6H|H!G$lR{j~$`(gdpw73I&TO8%=nksx-fFTv)7c=X3 z<_pBQieBJ|67Q`}ky)x~#VfYrSDBtqpNd(Nh(2a9Fg6|!EO^V45c0m(n(%WyozUZ^=PE%$50Y zNJ{aT#^5>E7}T((=?Efx`q8(vG&-r$^hqY(-sELGB6--5RVnfwahn*s`E1Tt3ih~; zcD7k@YMm#E<`%YOn*N%c&)(VUxuip&325+O`1}c|Q;1g6srsmDYhq$(GGPf8^Srv+ z(6~U9p2|wQSTkJ34cv8^gd#IE8sA#fDv!bW`73s~)<^#DzT8l9Ep^AE6`+xsnkQEr6Y#R>p zZT4JFt1E#9*4%nMUNd;P<>bw`oKmHTM@1|io{l>Bc3$W%Z(x^yrCEtRSz3SW9gv({ zxNP+(Q6}BfKSq2!ySlxyRHC(;@9jkKxxM^RbkErQu;@2(dCLF&w;5xcXnl4m-R1Ej z$@8-^zYL?-=j`2OmyYsveWlx!YfZ=W%_l4S&_T8O3k-I4c4f;$o4cF77tGCvao6hB z@edEL&+HBv)FJEU{IMNZSNt@A)|ufbJFc0>$4<-xKr=Jkkp}GNde#~{7fXxL+FC)$ z?iw|64gvRgpO5Yn_frbZH=BnpHdrgLA0;*nB?2Bt_jI$G9d3$O3mQet$5 zacm4dgVW{_7EW^xHD17fm_B`YkSE1N-Vfa6kB-|j>zYuh&PTAnNkUF1{5`uoCOfip z9}SzEd637txENhp66e36Yd*QpI>NSJqx3z)iOaZj>PeY6Dn{0MnD<+cdHvjbCktC0 zLR6pDH0ZFT-{X1DJzf*j)nno4I5tOZHa|nou3iOl@A-jXxFT=UPxxB#!%bMmABjd! z(lQsK<0j-5DU05Aj9({{4!+2~_w0{5BhB6Z`^8`DBlI5rzW2^fXA{h+c%0rcsyiNL8*ADwzDpB*M9L5oQA*Pnw}^>bB-d(J!uAc= z6#bCiayKQ4TVcC3zQq)I!h2kznXLPJsV8V1`!p)=p5?KzI*bF{u_h(!@P^O{9eX=> zxBgw(v7_MxYi)Vd6lodH+XnE5Z#-{pg3W$@b6SL8#VW7ylOkeo1AUPYP$*ne1CqH? zfWrfbsEkKrv!3;>&CjW9dDc&-rzH1DqSORLzFmLyelvAo>g2MbMd9|W zLS3{jl8-l*V>kC(35@t15!&uHm&2UpR6Ur1d;@)=$St3Mtz;P+EE3E8SBjNn6_)ls zaG{}^*0kr91kCfJN*ek8i;FL=O1Di=-Vqcva>J&!%Iq+ZPEf8{euiden>ss-*$nj& zzj$HM5sDcd7dHWjS19i1hK8)g1`iE2HF1y-3N>-0c^YLIo1u(CCj&==m?_dB$Xgq+ zuX;a%`_pA?#JvAZjnF9C3|+}l(k6|LHKp~rvaH|E>DJk$PtpC>AQ^p!2_>_BPJe0B z+U3S+>$T0Oz4uf1s*8>DwF$Ux!^+y)(;JneCnRc@>*&j>+AKKolucrNyEs@`@ni{T zW@-Fwk`7iV%;`_qlq#V7w;pE9RTq|?vefTi^=cD>#|%AMzJ&=l*++WVNP z%!!hgmb{@XzV*}VuN4)tjMB2Q$U#9tf2OB3)YSul8G3rD?ha`%VWZ=%uoWwSdJnhuF(^^{JxwyYI{I zPegta6n-Q7RQ!Ky0cxu<|IXy>k+RT*(UO*5^FIE%V{$)N!hsQ}>Ef@8c@RBRsY_u* z>N#vxxb&1nBKN6RGTGnEUB1(Cr2UADmFTXt=GCQ{oR)Y5XYuCP!*4>)fP!N91*EOR zO9fF%3{i?7`~nf@m%#~UPl(5+3q+w9RuRdI1C`k1J#B}V1<7mX;V8N^?HOB0K{G5Y zGimzmJDdUn5mTh7NY}!O#S`Y7B$_XS!=k!pF7|6jM9|Bl!l`IHuMoj4tQly`Bftd( zA=HXeSGM`w`fe<>TC%qdKSuaSzFzR&ZTdqmh9iapkwSzU_nG&)A6g!(^MkiC zO$T5VzGaFAE`GP_XQPu`jIx;rJ zm!_r{WjsP22U3jEAkSN5O+y1q313-lqjSH)ESwo%|F zTJzE*RT1@vW?Cse^qSdohJHp|4VSv+Zr|}Bp%`MHsg6OV6_<9xhhTAGq7M*JDLz}p z3CP#PqG>l-|2RA2TpPp`tOD_Y-ZN z{Fd+;L;^qUq!1^#1**F=Er2mxZ?0IqSc%2Y12rp6?}JBwsEFETT1fnoRM`divM}g`eQ39H*mri?s_F) z8oWdH`xA5c>tJS0Mo3w0mQ`k>RPbQB z*~-#8+@-}~K)!-LjkSv%g9ZU1xziN?a5--IaC?102jk;~tt(TP8jVD1_)1#`gq|oI zI8WL*;r0m?(U`Cf=7F_gp~H>bj|XiXy3{nD{D-Krs8s$(CHi3jx%=cl`ZMy^c?A%fh3TwxFOA%MWhUkJnZeCrfkt$p*SZ zJ$35{jh8nLUM_Z&C0-s!`!>ZKOHI*__gOqt*3XWIERrJ(VrH+su8yB5QSRZ5b*Yw;?dI`Es0;V4$AmJdfL2UWZhzDWfOgHR*l z7d_K8FIxIc%9FZ@e_L}X!9bv@EU5Is1$oTEH*p8mnSrN`pu&_iGS1PqoS00K%ojG@ z?Q6sH*`HDpRd@A__;-vDB5zrndL{{N0W8TvD)j~e!mpw@Y>mxws=IENm1vjkT1n$` zb?>4z6s3XQm**!G3BT`AD7QHsT#5CjaL}%CVx9J#4v0Na%Jia2T7UJ_SqV!z4xxD$ zIo8t|5bg8x$4aJ}5Cc8J!Yh6f${t46trcuvf8|2{+^78xVjDMuWOeXzNqx`Tw@;lO z?=M}z&-Jz-Wb8+)_8!#ssR@Wuvoad8of zi>h7%RJfEgP$)#irH&kB!htzsgsi&aTHt<7A0C_`izcH-`jmogsRH64X61;UGgY1{xOVbZljMYFN7!EQ6feCI0)Qb^Dw z`DFF~ELA?e|B)XRxhx7pRPTuE55IT##oUo?kb(w567;R4JdF5HqM!SXDjO5VfuQZ$263o3sAoIf8ZujM_5ng0KrrPRaax+5L7wexXQzRO zQ=_Y3Ot`Twq16=eK<)>mh6@EkZUSTH&W+}C;oEMef8}2OBwpmaH7zkjOLI!61aCN5 z*q#j&>uEO+H6VB`d>q0Uzn%Bm&9T!oaN<#V4vDqG-Z$MjTXB?tAfD4MJ`*5miyO^mRMch79Fd`Rp zUN&!_y=B59Cg=3>FJx&GfZ_5eJ{) zlH81cpTR3Hb6sq7LZhCa!jx7L-A=@R4|~QkHJL&zGQCM3xkikC{i1%mzS_9WkA;jy zZft8Cy|!iw`c_yaMF3Y*BqOD&u)5kv-)=$^s2+$b{tP0{81hUq=kR#0;S~vUeJB}g zmbkLm>S{hcO*o1Iu>U}3cX#&>X7s#~uyDt!WOd+{r`?)R1&vT>)~K1>eZDyT%CL`{s5 z{~sCw!yTznfV0Z324n~GsDPvRluYwVTUB!4r`C5*Hv%O$K6TO`>3g%Ciif|rd>$w@ z2;U$|RKFYY%fUQs=@}gV)`!mTTH%-FmeVBnjS5T-{L}oz%5kn&y`O4ojKmxw7|crz z%OJ7P>#@J9qr(q2xY++kuRSZJyf0x-P62v!hDL-1Vz`%?^Shy@M7&R3U_Pr@R)P{R ze8P`RVxYmt#|Ok6P%#IZjQkKdNcO+?xxp!Mno^aZi^j?szX) zdZm;u`FTJ97Ow@<*D1Z7ndy*R4}#O};69%n6T>CpL7l}MeXTrW&q;Q>*FsGnGVH_c5wFedvq@?Z_MqF z#IXw8tw*#Yn|2!$p|pmG#QgH&`uXc=l>_7aDS2(L|A*hH!jj!CjZR(stL>E4b$$@Y zujzd&(RUO&msehF)z0<)i0*b;{<(y`HAKut$`F-`xDXL^ zJZuVKSk9BdFM?5uk?9z-tp#d=fwF~(ZLYbSJd&rkY2yfbrgoMOovEFTCYfIh0AZBD+Qi`7hxE>V{{egOC&~_<< z*Nzkj?CGRFJ(Vu_D9Li3*TBF4WJ{k4VU(4zfMW^;*UPRF!1nQi#~alfYlj*VwHCc*V`l9iNmG6z<6RG}M1kLakR%>t2H4R>46S*zVxO2gr_~ ze=||9Sz^s2?)dNH&Eco#Im+zYsTI!FlBnWv-i+n7_pe~W=QxjdNteeNNL~|GTur6H z?vUh{QjEZvc4&s9$fk@mBsDDj_BzeTJ7yK>Bq}yp($d$(>*TcMyf*-?$Jp4RtN}WS zT^@@{c81}oe69Uqe!XS|R7jeJj|Amey3us);Ng?i!SEi^J_QFf?Z@DO#ijU@5JRl5 zD`6qH;`|aM^nkOOz>(r-&K8P!#kxGKIC7U*GYx3bm_NU`M;n%#Xaz*fjV}JkDz?neFD|OAMkwCCeS5UBdgpYsB3@%V z+rCieB9L=)drR`-1yFNa1DG3qu(uZlj3vP28TdEHamS)JFKI;|I4^eXqb*fKaIb0e7yJGAu9#)vOy?xP_FNhuN>5+YX z^0xEzxDk$q8N@3?#lFLWkDK1>TVBB|B1AO~%fm%VS#tDAuso5RfB-lUX@Nq8!Q9tb zI?Uag7`C$#UCR$FwX9E1BFL8(6|!4@c79%IzbN?O97;(=Mf5l(x6gFWAEL)l0?%$jXZ2o-`Lo%4@B)&;u8s<-d{%C&E_RE9|(3otovAyuFTT9oW%ep#QC zz22;TjX!DOXqM)3xQO0+N=9DeKp=YcNU@%X{J`}_%kiK4L+s;GHDBGD;_`NPD9U!j z(}2#+5tES>)YWAev-g4)T7M4@i$R?mU=NbjvwM$@*q=Xt4%GUTr)(;kLZ48RuqRk2 zX9MP(doz_mEP6G-R6G@AKMfY3nm|$Z$;k;0q)0-lR#hOj1GJ!=sj@&%kP8NtbHJ>2 zcgK){F>h~|0GAIBcQnL{twf&vVag);yi8v^sRP2=@p@jT;h5@|OIG~is~N%6{N~m# zcI&+KwwMvltXZ>S%A*@!&s2#HSr>a>>Z(~9xd?>#4%g{0%V=oe0U-5`TLFp-dKR(+ zhGS>Y4~D=Y_Cmd<+>MD5sNfap3}^=IY@=OY>)P6128Ea_s-`;lm8*kN4IFbSCeL zeIW!k3e~rwA|%kR1_CG$`K;`0bD#!kJ6{t6@{Zug-+%w|L9qA)+2o-Az3S0X24G2) zvjC)favg8p5T&G~fVru!uYU)40Wi3p!m;6wCS;Za6e>2OIU@C5a zTC|o=<^Sx-KdgO`ogJ$Sag;~Dh9K;z32L)ig=F=-=8CNQ8t#~mLdigr?IV%;6&hu4 zySC5prn06W^;;I28ad!G+QSwG5e>+e%q9-h#84QR-sMco#rAO&eDwO*g@hGbNNw2O z_+_YPw!p{s?JY$E65J~}-F3Rbm#8kmLjh%3IxTKQ>!Er1bGkqOKsoq@zyb&Q1E6aC zHlF)upC&~fx88W0EP>}5eINDq0jv9{yrT!))V6h zJY1^97$^W;YWfJ=Gaw6;rKEb*6`ZK6?6QD+1`rk#6BF!8=7Iczf*_yU3pE{`aP>$$ z$bZ;%n*jKvfzc@@BdfqiTp^Gu^`Gxoit{ffOXN91QHYE7@+|hj>4;d^+sq{PN623_9Fz_%^a`Ja!5*5w6 zyYr6a#0UZhu#_YI2Su_H-rwJUnXvx%@88$*a!XUwK&33+vH5vLAgBxUHUINDo>$=r z$XHY~+&nyLZ{I3#lUfX$8?B_geog#Zpw8umdf&BS#fB55tD|{B-I1mZ<4GzwAk6yx+qmfoC`stOgyy`~KkN zT`l7aTfCBAv@u!!)J|M#2;I*0g*zRl<0e4*Yi-o;qWJNv_4_aE@MdNoXGMt%pgZff z;ZihGG?3UdIVj+&voRp6)$GTBLHYx{A7FqeSjC=@olFEsyo95GnYcdPPO>0Iq5-Ew z!BQm%M+r4y2O|VtNk1GcUl0n@4m?F?U{3n``+leYa8xT~6eE#}1 zJTMR*1cOg61GdH!UeidGgX+{eQqzDA3+9u6`F`^zEZmqFi9r%@9Q{(k%unQPLRQwm zW|~p#(+9c}#VVk#nBwVnz&Jml=TnvVswppg-k&SA`nO>PJ(CC zpLO;vlr9AUehXW45N&H%FZ4dgw$B$fUD4d)`gvO1c&3a>$2a_Wf5?g?c3EL!FJt2d z#$<*2W%P0=m2EX zr$C>SUepI*ucV}8dXYR+3UFC7b>)g6QYqg}gwX?>1fU5LGBW7Uu&|*2-_V0Wivy;* zT`f>;so(Yy-PL}_+wbA4LT^CJ0(F({m z7iWn~f6cC|JortsQC;9~uo98(vW1R4vxkS3dF>a%ck9Xd9sP52Df$Nnj&JX?(ctu1 zDQIbFqobp7qPkGRI^LC$kdvD`J0}8uV;LmW7|F=VJ8SILj`;xz4WsV>wS^h5P+-qM zmx&IN1m$89Vq#!I0R5%AzP=6u8Bj$N-m@N#!axJ&m_h^(K;95C zLcCCP+o_e=^mxw-6f24Z?P547ypd`d~l&Itz`klbpEQ;`8o>ctL$C96Hf79v{YV5jeo~5RC z1zf=4RagqB9ipuKcxmr`umDaXpiUGinT_PW{-0ryD|`xwB0v%sc;)tvWF;8vhs6LP znAmrAc69h?m2SJ*7yI+FK%4s5le`cgjNlW|{r@~rTwJx+spa?%Hwz03B0Sua?RvPs zwSNDehA;~Bij;Z^u>jlTA|cg_3x?P``@j2aXZd_E~^ zwnMg??w-+QlCPSI0VlC~X!OM?=qpyH@1 z1XL(LxeuRnZ)@RVBfo1rv(y$j;a7QGpR_OZ*7|*I^7Aw;ytO^BF+k!kX>w^P)0)7YFu~k-hJRSM!-zimB{)C)nIe^$SD;ZNNTnzgQ(+rHw2aVW_!b zakteIzbu8bTzs8Ip41`L>eklvJeb+THJw68QjJN5iktxl=l@XmR$*~{O}=mv65QQ_ zySqyuxVyVcaBXNHxVw9TySoN=NN{)P;1&Xe2F}j^{oXTk&dkksGtYCb_U`U2tJkVp zRrM=#Wq>$8R8c{QfG9!+hJG$ZARI~zSi9TXTR^-1iy(qOgha~%ej`9RaQE;?6c~+rVCa={#HTgAUL*tBiZ|7Am!q z6=+sEy;uQ}2)MMrrkF7R@`7Yqv|cBsqJja?_jMYrP=E{^V5_mSv!kP@7yAb_`Rh~k zC&d6}2t5v6L{@?i?=3puFJT>{<~34Y`dC|U=JuZ8!&~@}vD-%%-$w!$atdP8Qhq)G zIX;5E>xam}V0RRNdtW6Ftbcv|yE97w1Ab_!p9bgSX;+z<@WD_-jGGVRGplwemr7>TV7^hiQ~ek#re`)9p4RF!zKNLodwW*T!x*ZoXUYs?5r?R7GAg z^|pZ@fyav0i9to(R>oEfIDmnQE@(pMUJT+qaR4yo6d`;72sBAjDvgw&_!pE7IP%r8 zkzoPb|I=1aE%6%F4)6<^H-o_x^0WVVjzLU zVzUbb2!0h!%~0E{e^eKKQ`qwC&Wt!k(b7iWoI+Q9$NiG&CGVG^@a4|>n9nC14Ylb} zl};8`k}Ps!pgHlHb0{^d=pD9Tlv#!dmN3CiS@dNW%y_?XJ_IZU(f}I()Bv3PHymsTlBgC16rBOcGR+c@8wK0Z95)rdWm>X9 zG^CXHC4s8K*%Nk{V1uf%vx6pAp((igt{P2dEW8`4;{D891}1&90DLLw!CrpzvLt2Zm)T zJ(ze$Ii>#RqLWHXYrZ>dl}uE_KK1<6doo>OstNdXmyK?(2I~=)FozGBgEv6-08*hd zy}fS8;BR(R{2r+&2D`DVn|AYSg15n96%`eLB{1S)2r!w~Uylkv9DlbhaA8XS#oZ04 z^zXq@i5$2iHtgVWVaw!X&7=*}R1~t;Eia0!^5HhIy>k|(T=L9zW z$0u(-T0qrfL;g!OnI->VGdaRhDvk(qt`FEWR|T%SlHd>yKWq&i^=}uDcsx#Kqsgmd z^Y#}}U*|U_w{4d$(8nBp>oiVzPKUV~KjZH-%ZijFF|6Tg5w^Pr3G~Yq*!60>cVb^>lJ2UI50YCDum;C#A zP~modR6rP5SX=M8wgcn%d;fMV1~M#=-v@-nVBz6~vxi}g;eYh^7Xg5NU?AJQG0kZJ z5D>!-O&rI=#e9#Pl%4!h){2HUii%1V*oI%MlM?>6)pUCe{b$;JkUjaN@4; z{YuB1Oz}*b4A}T&^?-nSv$(sB5Ae_AqyjON9I)>)Gc)<_=XC7>ow5%Pk=BU6yYwzoErjK=`3$VibMmrIM9kYr0gvMk2jl;HRfyBq4d9U| znONf5U{Qb>qSYvS_4Vs~nJyL~7x1)v$gUUS+c$u~1>S^83>C&V2gXobRtEofz5l%w z|8a8Py$)4MN(f;Odm(rxPGi;iCZH@2>+TQ}{31tAP7a_=|JPl?z4^kvzZ3HElz?eZ za&|U|VGjupS8uk{tkABF>-)RpO2oVY_77G5kIYugD<3dKSzDw&{xR~xnqqJTSH(Zs z3NB-MV*c0B(8{slTL2CZU?#ENzgKK`J#cpJ|BK)N;XKE1!6E;UC2d%CFE^S?^RZh4 zD=}&K+tHPt$a!MkTYVn##R1dJYt0Chf&Y{2{*sXsJtJfE)8n0bxf-8r^9NfEknDHxq&{%!^u{NxGj|C;gD7gjK|Ra`eB=4a<4ur} ztG+Da{&ki(o3SbL@b6!xn8=_*x|2SxZ%L`($aDaI=*D<)XLOfLB5BW8* zk;^u;$TtGIy3ck~4J6AQ&N3rlW4 zY&{U%2%dW+>Z(%IKoQ7pH{qi?!_}0uNgcH@Y^^(W-)&ZZcPFp0)C_j%lkPTJ!QgY2 z32#CdVmdrH@+wJpdZrg1;-UOtb`OqZFW%kIPF8ZB{;4hF&xTuF1NWuI&0m9K=)nc6!Q@jTceUS`!4hs%TRPvpmxw{-Ik=AJ4Asju))*|4Oj$ocL-$c4&vvu&0iX&DUS#{yN2MHejcwBs2#hCIP z9VSGjc(Wb3`_3+FFl>ioTV8*(bya8QONR_F7U!Gy-+OJUJ+<4H5uzY3Gok^z5N@(k_|)oMwDf`kAARnuDy$HFfgI*c~r{x>#%4xDl___v;`dzaDE@ z`ZqrwDI+0%6KJA?B2 zY*7J|znmvR9&K=P$4>Q0c5iTD@ZHRPDfmeGFK#f{9Q!|R;D6z|btCb2<(LZZ8q};d z<702Q|M?ui!B3rE6D;jA!O!hN5chTVlL*)kIoS%2&3WDCPfOY0Rh{KfiVoVQGg?^h zXQq{GJdT4Qh)%s=$R*;!Pr*|RAFg=H!O-9rMaSni0LTeoSjmZrVt~V;T;=N-s1b!QB}VZ45bUO>Y3f-X7(4S4Yd4 z)c2ubbTrvDM8oiZl1COT#0(FWP+{-kVBaB!5+KfMT@5WUJ{)() zKbXlQY8w&mhX+|oTAcrO8@@&3?9w+E3NwEj-NI(MT=+@AkB;@F5T*#+Z-}-bb3Wz7 zQTCCT{6pWgX4iec{M%K7fkAyG+5#Ke6VEfV09m0_${_4Riv-O^q&;ADwLZ~}z~d%! zum*;2ZuL1}mCV)Kp&AQa;orq5So{*LDRoqU58PE)^skR#?aiIBCUv6N@2}B`!|XY= zz@!zD_L(@z6gbr2Sn3)nq#RMltoo5`(=ggt^YN2Sd5k2YEku}Pl zi+I;k`z5kD21kW)F9vheDIkxv^8T+-py;OGcl`wK>mqkeF2 z_4aFAMdCib8k6S;7AQGX@^qy;dF~=_#1As)TE%re>zS-o4DXV_7=x&}>*BisWk6KR zNbZY^@ebPU4FZEc8FZu+Iq>gywSG2DZz>847mcKw+M1~mE*Ny;C$|`zk)y0}a2k$l z3p`E)v+5|w8Q}`f4FDzfXx0@K9=sVOq)qNUf+TfMm%GC$L{Xm~b9`=r9yVD6Jl&r3 zlp@DdgS-Hy9Poz7Vre`S+{hEYp_`vL`@*D`h6WjI+%KkXw0hz&^7lif9}6{D?xNkL zeT%5W^FVI2kuyn;a($^E7tc&7$QcWG7q4cKBAXr|ua1X9oI@^gEb8r@s9rzk<~ty5 z?QEPG4RggDmLF!@{j5v zHh3oI_2eS3&U_guzf-4-x=6t}Z17d)DfXYhu(Y6IJ}if`Qhk7)fGih3rtBrC0m3Uh zG(T*vQQw|o9V#M5`pD_2o^-LN({G$3{by4_4x09H9IKNG%Xet*PHtd`Ioc(muZ8~4+# z&G!2TJuL)CJQ;$lOn)D#I3{fn^YXw_^irL#6YArpzQoqtAC;9U1PJRW-~dvQ3nIoA z*C_U8NJScP27kKp-z2$1XJowH`x8L{o}N65djI)% zbhy8l@@!kLjZ2<>!8Lp|82N5G&}RG7q~`cX;P(xA-8zBfFA|$+thx=9AEH^(*_}rf zk%@R>UO%xGsOrEX^{ju8C5xbnLr+rKG+Hc$V9Yf!@@{hF>OT$efBN*rxQ){6NW`6q_qR}}4ev-XlVEa65X{^Ayvp^lbruZD;V?EcZ$R#fd4FjWI)t?SX zcrNrr^>r)J3Sh6h(7M=FwrKo*BPKdT4bq^)Q_K(y3BVg&=@%sZv_Id|I#++QU!_ZF+7#e5v-4$JQOy<^ z^OGmUEHjzF{=|Vd7kgn7dKm9Qq_m1W?&cy6B5*g(OEe`>qszdE@3VcEI|ZW)Ucu=$ zB#Z8~Q&S_i?TM{w@6?sy`tg;*ijrN=X|Y@DvT)lET_#_vWv1PuCRQs`$GtG=b3bl$@dnWXtmGjnY!xx||jyqK8FH z#;w!4YC2j*W>qOa0_CWSJ-IcNF3+QSo|Bh@Y0k&MYs*fb_b12CRj6_kn)JkXk7DrE zNZ@x*XcR&Aa!y+4{rls*G0;>)CeN**=fS&s>aQ4y?v7g-MY=$r8^QRk3ZqUXcjkwb zQe%BZ5z$(k8Qv}~JJ@>OjiCtg_jDtwJ6y+})aiG?=(9ytkvejh5|oYna~FK0ADFui zh}cZ_8{MNfM%|Jji36m?EO@_<`YY&@?``x+Xw}@oZm^da^*Che@N|^nQ)zWz zon~r{_QIb6l zDL49X%Niw#`yAQkNPPZ+(;xVlyH=o5tQhMeuO@#DZFGSMr8y=pB(ZsF1H37t_(F*6AyXypUk#dAsE-LDg9_~rqk|0fc$oRb9%5f zc0uW3h-tbYsM;+~6%JyUEt6#;H?Fbv#gMS5It*U$z~?LA`3fcMzi1uwrbO_7348-& zqKJLUZQ-ifK7i;Lzn(apwloIQvN$C44GuxG#kFSg#vZ8+L6>F|8Lhvs^05y}8$27s z4#*NbWGQ; zO-J?KJaY(L+H#p+%N~$Dqg%d-%|B_C zYwnbBl-KGa=HnAcw(dhF;v&8YnT&i@HT(^oVL zS?`S3gD9i!_cOzFi98X4!`#0R!@WbjJnK}?6!3cY5TaH)MU6~6pKE6dWqOH|?8@3N zeN=?Z9S2u;rQg4M>{$595w63*!?Itz#Pj{2#BGJc6EiwWK9KNNJH(U4PA~Ce6wjp9 ze$#>b@oeNR>9mP3B#{PM2d%%`{qED42ni-lqR{ieiV#!{#eJAZkP3_eZA<{yg2Yw&Ghijx0|!p42^gIMmTqcZ z`rSb^xznYrnl+2lF#vV}ScP$*8V`%{Xi@@$!cR{Udt;)Z>vu%$b+T{;d}c3M1Pb0B zM8b|I%H!FE5AImr6y`^MH+#Qqb`KJMX*zMbO#8E(EoC|7`KBI#hCX1yLiePmEA#_N z*Jhy7!#9eB6K$PaVx!JK zvS6f~5uS8WBXpoT2Yf`f*!937s?p?++baA-ZuuH%zGw@^Yqr~zSekkY6hylzyPMz^ z9)GZVe5Bxegka^cDmy%FP0+Qc-eKTD`P9ip-hW{In9d3EAAZ9T^+kM+v;FFiy{+0x zu&6H)r;%@7zm~Sj!wFEtcrL>p&TwYsbMzKNFg58p)$@8bAAQn*kgCgkwPx;$zNeZR zE|tK1Z14*|P%YxWDE;D?;rQkeF)>skA|$PAvH!>KsDEt(6cRFC&K%c7myqxvDypyF zoeN^4P0G=FqeeF%17{WD5BR~rP1^7~d2B+(B4ErU^fw_?*jl+rkH*us&B*jyEZbAs z`Fr)MXq+y?W}XMBrquu0u=u}hkpEfL1bExM{Hfu1vc5eh(ASR_0t>N$4WAZk4JM`y zNW)LJS|WaU#eHq^4;ThUeg9)1LKc*`M(P{f9r~K=gOVM`mcsjq#tRSuhYJN3X(z_b z4n7IkmOeCXk4#SPdhFKw63?Q};Y6ziXl78$HLA)!)j@J-|N7st0Gy`{Enps+z}gl^ z%d>TfO*#Hl(_ldinEhJge%^-#x@ET9$1tNloTo>ZvoAysbo6k&@1T|zB+JTQ=`HT! zuv;7zsr>?8zjn#sZJ^lMl#!Ez0ncDE)xV8;((%`2+oojpc%L_lFXRZs)bCdE4!14V z`x+C_pJ*lF5-Cmhr?)*tF}CeS_55nzR!zH@qWy-^*Ff?6e0>~}z7{rCVoHkIFk zw8E26@=oxhDs`0cPBO*oYD~4>$sTTMH{|EFz-myY=}p_4uxS zDpc$b^Ylxu_G@M$AU8r|r}c4O((cLQwzywHcNSg4D#faBEEj&5PCfkTw?xVJyL)I4 z;|nRK>0A)+cdlO7W9$41#!ad6=ZYrlbu}SrC7gFZeB1oAxX?szm=F*UV!AlK-S-KD zp0b7B5bq0qJ1q@kcL~iGGspb7P>IJQ7+ad>6WXeMPSc>K?%3unKT zAHqWwBl?-l8la@)WLYI8D|~ixbn!VDlu+`1cGi%L*i_(|_H*59QAy5uT&)Tu;0J))M+U&OG zm2~QXbwbl0MUu8#TvsLEq>zxedBpCpjijA<^!cUgR(xGkbABdX<-vus>X!?i#@0?dSma00#7>p3-nHU9%-Mt%>2T^GR+(q^=KL+q zx~qthzu>#*2hYvQDij{eZOQi$t}gux_5lQjCr*~Df~1YTgV$~vT7%c2jk59f@@oQD z#F{NfoL1*(uFs#J-Fq(8Iu_zBR>SSD_O`W?m@U^(b`{Xq~Ql433^_P zSU<$ovc>1pk5SRmCg*g;3hO?+9h`9+`SCFBAHwVxeb*y5%<8XIM;Gld_OEMlCa2fc zo|7Yz^;E-!M7G-CCP+X22UhhzEhk1QtEjSZkkY?jo!;22mT~f;mJcizy(_YM$3LFl zn7c56)&BTwO}Aw(1Q5{rCU+d3)YkZ;eFFkwRHIj&-KvFSw=Jr}IfQS*ivmTkYTtVxz{av^`8gf$(n3}5bIL*gEcU-0Ap<;kC zsi-a|9ddQ=C*e2sa-%EqruPa-_WKX(n=XW7>Sq*+ywq9ycA0pp9?}fq-h8GTGn@#S zsM#~Wh>cPHhbxgPx~7?f&&`hF)R+p`>;)OIXk8{I%GYlO-(r&V=EB-KxvrSI(eL>V zzgLW&%Svp$VC1pbcaiuk>}Cw1T%yjIV+cd<(e8KxifAobuFuURC0ugxdZDnnEb}Fe zztSH+POsg2UeWaCBM$^vZ`G=1`$KA)f$v@6FDISkEbEQYkPzwiYe4M|XEF#VoF` zKpK^|5Xpac73x2CZo8(IiZ7_SUf-!3F#NsjOHl%`y^Aymfm2nZj4)2LE!?ji_wSqF z@97zshw%=E?12x+uxx+d8`>!@%+hm+>)3l{1#95T56LsiU)PL&aFs+X=J;<0_FqQ& z|Kt||krkykGBaA<-9UFGQV}mdg=s=9C&6#3iJC;5bpghDiwyCtU=)^)CQ=A4^I`4* z0hqRh1?Idb?GUHiyBmk!YZU(YRDZCbwnc52b*=juOJs|9%y8+~7pWoqu1)-`7AM); zDE5@f&4_8?=|U@h)*aKGHU)(9T%zR!|5w@(;g^>x;s&tFI&U{G+16PG>JfE*ohBD8 zoJiK5SJPeGVyZWj`~CSv;SO|*%1;?kB~KOL*zZ9q+qvDXUe6Wc zZ2wX}PWXKWwxa5p6wSw<${;XK&@>n}Ee^e}X#T2o*3Fc`6YGU{QDk=_aDN@eS5(9n zT~+0z(?J=uvsZF6Jg?nWpo`(8;1gKnN4N&w#4jDH&ud-Fh|?iINbIs8v6^?j5OJip~&iNP=G@I z^sl2CH^b=nF%KgeEeG%Tinv5`sYw;gSDS3Xnd-T}&-C_wn2Ase46*0P>E1Muld}2FutCim~kB9cIRO;oJnAi2aJ%+VYb}5*Ih?QmF zhu?=S5s8)2z8#39m6+}4l~qunVrK_B_I7`?Bk5I=(46ip6V26MpkQdhbXfH$TG-^S zQl8lhziwqUg;u&bdl8O(OmC%KKihSpb)vrMK`FHz@zjWRR&96CKE^Jtw|JnD3xa+? zX0G-5%($jCd~&PSz8AahUlOfsHDhIs5i-Q}WeEnNI(0I1;<1P14ZoE=e)!(^Ds{i^ z=aU!w*?Rtj01pkrY;>y~pAFNJ9f_AmbL020F zTjV6QOY%=FPqxQMmkU|gZ@{${TY$m-iK=tWxKH@*w$wj{XuxFs9>*Xuj#k=WFiRJ6nr0$uK6jrY8c+L z_T=Gec6$#9;rn;SG&g4Un2=f<`Lo|LhF(iwrc_S9OE;FaxHiLe8sCT9M{(*rDH4`8 zIZA{1(WF!Ao*RdjUC;OnLHq_kP>xocNgfZoM>;-4}$n zOlNz^cDQFAxqL(%ow!nEZxJGAiMzCJ^m&E@N`oY4a$0i)STb0OT|p2Fe`E^(TwqCW zwPqsE-x|e_X};&+px=Qv-g({Q&6_wlD1nJp%3=LX*zg3V<#k}Z*_8WBhl7KyI$f@C zLt_4_Ljv#HyNnfkG9NL@;lNO0z}+zUFJc95KBPIgBqaz&fW5LjFH8{)YsR%_Uev2< z*0HZ&G^-xci~fJ;i?lt>t~>pnRn=5=JzjF2dC85r4)h*hPDY+m40KVIJirq%`fiF5 zh>5aYci{80KQrAdHBLC&i&&O20Btol)o20j0P~cBYYMF=6orI25k%n#{IynU-gR9X zk>DY>g>n)N@7zkMJr|HG$z%(P3B!!w+QX)E++@Y`I=(e(UU|0`}+VHy; zvRtl?JT6%tqNYURFti`BmExxNr>M(`wQ~A@*%H~VzkI6bIK@`2KhZ$`R2%I1^VN0hK1i z`BHRz%J>`pM{jRF6hf{qTGe_Xp`lm4ndq#|MA|zisV6%1s5?T2kOZ%_! zG>S?TNh)?cRl&L)DL4~as@*qZ8rNZUUK4s+G-v&5MS?iu;$)-j;|;BiKGx*43)S)m zuBsa4KVCV>kTBvpP^%n{?0)OJYtZG}u@sq@h6`XJO0O}gEFT>U9r^%5r8g!|6X4!M z*nwA8Am%PGyLY#yQ3)Kd%MpuaV2N`L*D8^kdZTt{rH;9e=kR!lC4T(7$f%hQc?1#H z^6Fvzh)FL$^cJtt@&L?pRy{%>`Px5q7hB5-mE6(9oFV)=_op37nek?WU^V*ckV{0l zZZ@G&57v94_*$l9HPVqjbi-zXCr8I&Q!&!Dw63wy5ImVGy|IgDV?8YQjmu6nxDSnv;(Vi3j?eHYCR3JP@Og z*>3zYDf^BGXCu;_2Z_A4&_9avQ59p57kKLPTBa7lA1qy} zYUss?!iQYGPp`y1pm@Tze1uPFD!Pb0i#$j`%yBVCvqH`viYnUV`l*sT8in$c6KeYB z6}7L{^*rZHy}Ld+8d^sLT>$-1s4myJE2s5h>`ub6)W(t9Vz$uLCRmNA*}Cs0nNA4A z_i?5yI6R;=Kc1#XMs1lO=I3T!2HWyTZMw1EaNw*OSq>@3!5%y<8XZ|H+qZgg1hi^# z?~M3cH=5*oQ~oez?~pnB;oS+HsYdhsgB5X7G~=br$zsk-K3_^aG*CufsRpI93DJcT z$ygC>AwKUB1A_~gVnArV7IVau=iBByRYT=siBho+vUKR&{`}#7t&CYf9suy$N5t^u zhBf!!$F0fV5VBKQofgRSPvj5a4-hN~eFNfjuCs@{BWnyp;Za>MNJ!$A^kp1XU%?;L zq1<@8cZI_!*6c=32&itJ>|TsWUqkO?BFjcnOLIctoQJpz&DAGzYSSB9<<*ae!<;2> z1e(kG(1w;A6{b75P_ggpO)bIvZrC?Gt||nw`7_k?mF&6K^7PXyV-5Q%N@p${2Llhb z&3om<2>l+AN@<*RwoT6!e18NlQ}oi}|hH1KWr^&Rh7wrenW zRfLl+0Q;L;Zy^pMxT$0I&h`Ox&rjjx#84}Ig;rpfKh4}~D3uHUjC*}oV|tKxZQ!kl zRUpK7Z*Bn=&h}_02I`y%$bc^?xN;*!uR0&y?< zwrrVSSFEkSs)4dohZt+C$4k;09YC+Vkw%W~*Pi@Ttb%YE2C`9U7V6bL`pug8>vTe+ z)0tyKPm?-CR1Q9sD1Yjp6#Q+DwM()R;p091*)1)!L6jB_DI;kO-bHtofuEKc-Y_v zM5D&%@EvZ81s-+Ng}9nY-!d)3{$^tMG8hbzU{(WNy$lF@g#E z_LY-?<)4i!I=uNqT20&au&;N*auZR+g(=49go>Rkc=h|qkqUo)R=t>`DZOgWx*3V@ z6}9%u>%7o^?6Fb*Q6M8}5zAy=iZig>+GoNP{D8tq$Z3QBI2}TfLQPMd5}}paqQBAX zdvTL56&;{SX$Inix2chZROn=qrIA2Ib-g zUyr|((H(yn(R#@EpjGI1lyd`r0cK0zyK|E0u&Is>PW zmoY=uE+x56@|YeQ`~fM}z4M&_bBH)Lxh1!YCmpgX!#DZ+w^6yPDjm2Ja{XKnaUiGX z=qWbD5=W8fT`a)}%gx^4+tY|>G2t}D3R6{ccu~f=V#%dVWIAxHwaA|+ z`9}#5WZ)>{EdKD4hksty_S=p$+IA+=$klmJGC|T7?vjzRqaJ29yrU~a`84b}&mcm% za)eb&(bg9~0i6{P(tZ;mdWAzEenVsO5uIVmZ6d5xJ@&FBEEdjTG zjKr&qPuQ4mUz;5y5l3$G$PFtpQ!r{SPs&Y@bK;hERfHutNJa~|o(AT&?*tgD8Gz_? z9T9CaW5<-0`#6w!WGPC&hQ9oe6Un>gQ7^?&zaH{~?sV7BM^)>O9?vcq@IviTMfuW~ ztiVde-9bdG8J}O(Uu#w|{cb;rO6+nS_gLp{f>|@S#7-c-*BJ3@QHhc$bDmX~mG_iU zQndWYWKx+xhEB|j#oP1Qh1E}PHqq;Qqbe*c`jpee=RAk|o13}&<59oNm+1+KtQ`4x z&(MrFR#r%;@A*ELI)amqCY1KdhEGLIBOpWQBmF!m$BmS#5(*M;?Ch|^BwUz}F>)+` z22MM?MY=TS5P6viPFba9gx5Pu)a!K_k$aH3uSHp%tkCx_JfGQ$Tn*_flOSvXng~-I zt}7VD*!~n$iSRmqhBpj6?z}mC^9BzFl=dh0(Do7qM^xm8WSfkN#%BiV1mBz zkD!st%1@Xe&5?7`qDAK$N14At&7a>-Bs$QhR5g^7Yz}qMhbQ!v=_<+j1&u@Ix5su+ zFQb)n4F1%^T%|kI)GAWYWrp+8eu|yPRgT4|KSpTDVcmz=fz4y}D4v3dpn=S4!b@j; zh*ZnA*T^vGv2^oeZl2yyYr4ak2`@(=JF(ShK-HmfnYoJVx8UBLyYLy!c|KRd7JW}~8tLqWOw zxfk-sPHH#jR-qcw-=ukGn>jy%S9cJ}-V~^iZ((AF0kt^lglXCz^M^@)XOA|%_~dGp zGI}!jkxJzVC-@rq4kTW?)e?C0amk34@@2~-ulffDU zgtTe%sC<*}&kps=jT8?Xd^b5gIleQTUA-^5rzGT&)NUYSL3nIt_cBHg6V6(7(U=H4RN$mSmfJ{ElXi@l z@7v*=^sBAB?=RXT*;z>?UTUp=l+3s^$u)mTsqU965q&1R6S|}@O;&F|f01IZa7Fbp z_DAzL6@oPeag!ngY;gBNnw-~_4=h;VyCt^o_lgbokrE-K;|)we2L1gG(=1~4tiXAa9RkHC zsv@4BgWug;ZFtNH^O=pJIn-^Idx+rNofJFF)kc&`mk=OHOFoF4B8GC7#2iE|#nKl~ zHc2>Q%cC&~Yi0Znr$9lLA{s+JTN_06(tyc|Bsol-BD|LEZ^5WVrX8WfwMh}^VdSEf z%M2oUf2hx|TCe9N;nYeiLI_QXxbu`P6^Wh852}raAeP*xymhbjHPX~~!Y#7JnR|Ws zqqVG5mTNy-Waru~cW_#ALKFYi+;MES(^jVcynrF}m zrfjCP=te2yvXltfNQKG1ciLE*tPU}n;tWj4SI2+rEf$nwfrkvU)TF{qA~uRU9*va| zWS(i!^7BL19g8tDqZIep2-#q?raH5vJG0CHc8}!F2#f|-Bvmpq1p$g}npm#H*drJU zl8+Oq#OR+UXi*eve6twPkr7q#*06tl1Mgd8mP(XT#d1e zp-_M`7us*Y)I*IdOC=;e2Q(d1QYx+!bE9F^m2A!@Td4nxe=ziV45fqZvWb%KsOhHc zB6`Gsgu2!pGGoqKY>pU1=z*e7kK5zIizlozITX;7^_qEZB~eCCU5_VHwW}h-JKew_ z@P4*kK@V^5=Vo<;1&(@#;Z)a<{my5p`77<~=NQQ*A$M#^*`O2n3L2$XI-jr^pdn~2 z!Il}zaq<*9SY_zwWP^7?{cKV8^}d<7?_%{ZGwX6vEs1Q>(}!pf(x!|Rr2Lf;Ig*@L zdB6Bv3-#zFe{l(ve2-%dB=4>}R*#~;o*U*?4Cyv-Wo79OfcG*IO3OYEI$;%x2`Zba z4L5u0((US8heVAS@9p!b@>Cj$y z&gi+$yok)7J+W6l3|+fU;(MItEH<^s>2`vt7laXIIPPB)M-k#B<@~`o(4x%Z^?KlD zYs_Xl-Iag1+^OV0V`Ygq^bm;UVDh*9(Y45hL{sN;ptqar>ijO2bd*cBg{i`B_tTEH znih#TrU&BM?;*LacvAWT&<(sGo?88>{N{@*^T@^nzBb8f(iHUiAg0?9pA>x0Kk~~) zA~(y9p;8Fjq(RjTCEqWPtl5+ZF=+*5_Q?0>tP4cHhVP|_mKTQxSdd4o0`!?v3zVxcpSU@v#?ei*Ld*&Bi9Boy3*9X%xya3saz z|A`Pu8@ZrNk|AgqmB51cXhavHbWlP|Qlx$(I$P@jn^+|j#UY<5=*1uTgjlys6{Fc` zH>J1QB#G5d`o9C$o!%U6J2eWA$aBx1WgqBJ>QP~nn5*6a|)M&rX{-{wb*K| zm@J(a!c>&CNmU<-S7~v}Aero_7%2cXky28g7AgPCdC?KgN=VDTJMP(5vaPBt$v)SU zq4?*h^=1nRZZ|{R>;UskuoFI&@C0mNC7)?sw^nOvldoGgr#O0ri?EeO+ zSoF85@&AuQdP#TuPLn!np{d3{clJ%$e7N`+&SCbeJ%-RlK$`e_9`e#Z<*?q)$rjxD z2lJifB@%P5H)f5@_wW(Qt4I99OsWsk2ifJowPBD`*_p!0lWDW~f%P!6vTRVfMQ zXDI@nf69*ityA_-;f?<{4p53JjcY_T8QqpJHJ6ZI$LguGW)hHbr)qG>kc@k_nxI(8owSKo{u^UFLbh3n+yDE zC?>!DGn+;VQ&wM5D6TZ=YAh95Tu>H|%Vutw2@wz?^#8 zcD-zsHw%_gzWiKGE}e`Ud6Mjf5nN)&|Dx`#qUu_jXyM=lcXuba1U4Q#xDyEO?(V_e z-5ml+a1U-9cLD_W;1=Be#W^SEd}G}E`aj%$*z{npy}G)(YIawzUNtv-Rk~2?t<#QO=tY$-RW%7ZdwB; z>r7VB3BU>!$e)fp&Dt+*{^yde3*izQsc**LsF$O&*W*ts|Gu% zR*K|QL+u;&gDO&%+)0lS^KHEaO5k>)wC0#yuv{@ae3{zSHgrx-YffP36|mBc@I*0t zUP4VfJNf<@1h7UaYArdOu3@Xq&F#(*>!ca>*bIkD|DMBpu**LhNvb~4_&mue z>qKv<{4ScYxHTc#aK!z#qxG7{C2I=@iNA#!F~`fF{$dWjMrl^MT|-HhvFm!sb}koc z5$th03D#7VCl*p4S}Nnv*gcDMAaLD%5XzF-g}HV*2;9`r-nfvF8MRV)_H7>VJclu5 zt}K$@RuB(I;}KnG!m44BMiS6&Eu=Dx6^*kI_{_5Z7O5d z+8cB$lIwe)Nb+_box2!Y67{Gz;d(`X;SP__sAOx(Pf<@=$RiVVnBg`*;U~)0NIMS8 z(}`Tf0;9;m6WCtfZ@t}b1uVmnx>*|@0oKrJiFxBoK^saEdfjuo=0TEE105?jwA0ZO zc*11X;SZ;rbX-Fjl04X1h*D^H42{FgU}jb>vvaYe_cX{?EQjZNB!zYE6h5^%??k74 z38?~P_E%79-r>k;JJK!rQD-tUD@vz7js^rUV)VhvK)!cr`Xpt#VnylvB$59|{-Gop zL|VLWrR!luqgz|Jr#rwkmFZHVlHZe5TB0;$8i{xPYmPr;APJz&jg^8MX zfKHjtz9U${f1<-aow&DD&oN5aw?I)3Lrb!dsPx?CZ{T#b>1Kvp>c_iiK6%1F#^M*U z`*Z=P+sci7M#*V87dV+pB(+JEDIimAyS}(|aZNwLO004VS_sYpoxg#tABR99xZ=$q z)NZ;iiS*#<`;?ckqZgWSy6|A~gT2HFjr+eTu$e`_e{sgc=;rM2gPtAN{ zuT;rmd})pErO?g2S|M9ERgISG?jd5x6rCjm&L}ERI^lz#(`9j?d6KPS&T6_03;tkx zFiQPLV)g{|?bwTM>fJNrmOwiM7=f{k1Wd1Zl4_~NXK3aMgeJGc%T|mJL2QWuwt-F7 z^ZM)y%Rx!qFq#1@8ldL=IbOp*(LWW{kOHQd=B${<3hX#+VbTY&MNqr`FwZ2A@$O zV1M>4k@u}l)>0#kQr< zVWX&Kb2>Q=9bbBNWt32)?kJj*q>xB5IafO1EvU8r_-IGL9Wv>#6 zF{g_!$R($DA5qgdSo+5nuv~?rlyW<|`f{1_@r6`jea3iG{d1AFc~A@^fnW->iBMza z(>eUipIfR1OOwY>UaN^Jk>!C#A9K2?iRH|bV)gW~SJa4k?GUbp6+%>P@@z)P$ud`6 z49Qg%DdP$h(WPQ~I{3BmvjX)-t25Z#R7hWC*q)J>)E=i0^+>F?0BL`*$`!_lStziy zz&hdQr>@c%Q}Yv=LzGDx0UnzH{T-cqs#W#dGt4Zn{zVPz1oereTITQJ$L|VC5_X^2 zGw|_o>POxORrrjpJ&5v7)saODP-BEEYHEgWg3J%mUoxbL-LMU%jBEw+8up$-oUd&mhEWwL|AxHUAOdQ*hCIcTM?z zMzxmrX97B|-k7i)g?HshgSPuEGXK>LrZ(tYye5J1JQKxWrZ05bdCgot%@yNA41eH~ zd4TI<%;u*Y#X%YjDYK4tfpqtiHf@k4BpBFTljNN-QffLudvcdVM_u-UFk8(Hf1QTh z!hW}5;B1)^E*wDG!?ZstMf=;-Vi(x)+B+q-MG2JW^^dK*2RI`y30yrzJrC#7Q0S#* z*b)cd_=d0Vwo2xSSQZ2$eZO;TF|Cf5TQnLkS%7MEoLsm7?SDnjn(9CXYdd4VH6qz; zQ0gID5Jp&JU7J^NU@~wUK5)b`L)Gl_JK@d6J#9#JBqKl?MI# zeGs|6J|E7|b(gF8QeK!#qV4trE|>kI-Z4d~=j++ow_yewAg%_J$pUedlZPX9+pP5Q zY1CW=L!Z|`<}m$vF`~p#)$Sft7;pnR*?J#5GTb8jJQdZt2bJo3rfJ72CMbBR8C33>oz$Tp*Y$p9`4=S=Y z*szxM^c1qo)f&lF7;nu(Fod{^eVzgd1K4zplj~)5AR^2c492}sv^La0i7`>kmA<(N z|AfZM2nYU4hCV5}c5)5i_6S<^4^5J2kcI%2a!3_1x&LlF8sQw*yu{~Iep2iZGMWY> zBZNi8*QhM{pH1eOW9Y4MUZ)xp^K)2*8FTVt4A31I_Yu`To!rt{zAib^&+NuA__TZb zIW8J{x~7^bu&9WN0LUw5fM*?Rq6wsf@KCu;x$k=W9hidjbn#$s za^Ok=jncif@NV3FA*aKYx$dPZT6fqyA8#SX=4lf@eT0?o?ZawEPdGKafpB322M58W z1lkbHAV86hE98lEvN~83PNS)IgAqF&tda8N@!WfT?6u)UKV#lFv<4N#q%~Cpl3Po@ z@4kiyE=4rZdb7R`6R!HsJO6j5@_hpnf=K-Y%VyjX+^bhOQ9F(MXL2OY%x z_PEa2O==O#^oD43DtMr$vO&?^aX}~Ra1kF3(?BtRI1ETDh3raoYDiUOPlUKm z3y$SGl7mYRP90@M&~0l$a<_=T@I zMm3Yz#AiwBFpt-a;6C=C(e6(01!fp=7h5oIes^Tw>v^%>ZU-BrbwYCglTC&$7n`6v z5y;gZnNOHNDo2rIvSaG1W%!b+Xx7uPdkDZm*kIt#HXTHI`WxhefS#i3Z_ML}2pGKY zj$IGMsW38v*9!j2cow=@ZHT}k^TZ+#?omO8nCL{@0rb}Skfh_N4liRyD|RcWOVQ~} zIUn*44=vIwh+HdNpN~0(dN)9SHY?HpLni$D|5Kvw#bjhYZzQ|`t zeir{dkVc4NE&|3)PAuS2Ca(aeTZf6b6GO6|>!TvWxrP^Q!02+-W)z)I1hRRjbK8;p z&6Z5?T@Ei~#!~I0^qom+7rJEUx5k<;(myIqZqF;(4&fW<0sC}^&KLgf!&`OtO$*=Q z2&~P1EOmAEqpGC|ueaBQ7|EYTE~9FrjVF%UF4&3C$ z15!4EUn2o1umBuq7XZvN#b6#(5&X)Zv_~drhTR5N)PoLSw8n%G!Ct#3@rN7^*N>z) zzQv1)w1LkIx;evp@laxIu-Lp944|l9S8G{x%NIjLt=7R+5y5bX4bpf4$AF+v48%pw z&zP|8x{y7tRY(w6zDtz(B4|*HTuQ!>4%*hDfFGu%=Kbv+-juyx!qLgn8Ul^*-@RYE z2LLTD8VM8Jz=rIvxb8)gzqh6V8)BfSq49oy^%+HhAHoPMaixxXb2K{ z@W%jy1=;L7L77x$liH*tEao<6P$qs!PmRg2`%(Se#Kgor3~+jVeJfD`;G#J-3bVju zTVS4c4gYxFvPl4WmhRJQHoQp>jV^x!G z!Ix%DT_V?8cl#Ed&0&d-kD?^n>ZkQ^XSl?IV3%Wo#*C4~X9Zcpxw>1fo|UqH)$1n}=o=#T^YjW5aLIbS;;_#~UG%oZ+CNbG1^CUe?Hew1T zPS>$2Z3>)4@-)|=&!g%o!QuiG8ZnW5*Jxd?g@<8yH1bpkbPcSsK$_fcw{V~@Hm=EG z;hP!Wq|@R0HH2aQM@`nLc=NW^?((-bpi8YoJU#Au9*eIiC)NRjV* zHx_&A9ON=>!u&C#Ym5S3`WLJJlUXT~skbUjjIluykY7q5;7DR#y2=_i@(Asvlh&bl zpm;j382#@`)~+M0HWW0A}^d%I_q_Y}8UIIxGBSw4c(oToIsH&>wokaq?V3_hC zf7b_hLyabM#OP-`jR$BT-#@@&15hFhqvhqF4WpxL&jIVWQ>TNKHa>ta*~|=w#WP@Z zjQ4=C{Y%!J=kH&_u_OY3$^2bzE*+=#-rf*-<^!eL3gEv1h%A7F-5d9U1mA;g7K=(s z5>irNWm3b7~FiC!KrX9A=s7dD6;`jHG3O_0dQ(QQe63dWi^Wo2dN=H`cD zg`&)x{y7m@)~;m9kK^r8p?+rZ1_&MEZxL74LcDTJZ(A()hb@;A*C{2ZiwA&H8{e$g zt(JTrDH*r5GLSba{rH0ms0fjx^*TnbBprvL;4nr9g+)b;Vp+{6T;Sb=yzwkN7T^h9 z$|@kA#K(^W1YX}Batb}x%J97^&;BQ;;7@^F<#K~@2tpqQI7 zN8cPpH1Gy3K05$U@xMi-ggPu@h5{OJ<5*TjASfvSs&EUFoDl@YFN(>}KU*}WHkr(0 z<1?W(nPh$a(&KFZkpFAxSBL#eu>bQz>qU&e%3*n4$gIo<44J`F2VDf%Raz?b_L$@Y z9|VH!MB_~0l0#SW;xD+Eu~_lB=;E&`TasoOKB#P8Fu_jF5Ftd?I3PEyt5q0G&QV*N zBqXFO2>bFjR0Zo*X9P1ymnCP+g%y0oLR#kwD6eOH@Dz{^o*AOEls9M@#xZ!OHb|h3 zow`XWmWXIR0fHg~NNoNt(Rxsbf2ulaZ)5%R?CEoff73pBi3=s%3X&kw2=uF=dXKtX z>q(AF;J*kdp{6&!@eOr{ZErjDJ>dg)1y3~xa|Ey(n|Fe7tej%~#K7U-`!izTdwVBD3fSsV7@@4TuEH^`@A(5H$GwU@>zE#*y^i(+ZoeLuBzD zvYzWi+Nf6UT!xYwZInAIL-;LwOGn%}oS(?&`wcr=s(pJ3qCN3S@GwD%r^g8vz&{)z}LlL6`vPWHZ9+5l0UFVCB?jXjo&IFTwOogjpx;@z58a1l zZ+`LkH#AR=mi;T&km7#4p+1go~AHB^9H@JrZph#q&mMYX=;c761zr6o#5Nmf$*GqvxYTJ zLf8it-H*~hCR!8hWOw0##$agG

Fq|Hr}tMi&jR_cJ4WE^%ViodHaR-TA`%>9}l_ zWWDp=)m-Z5Er2bZ6^ssgbs-|JOjZsAr-yQ)H{_ z=wOvBJXATis-pUl5^Wm<9O^IG7#ro`G21Tkwia})_N*G{AZpT!{-Szkd{7)LJxxfm z$S%aAl{$f!kKrkM7-8K}F7OO{0+vY+VlI=#2eV(YncYw~^vO}Li>W6SVf~v7{rdT0x zZ=PY?;w`yNbPU>N$H{Uejm@2k_=v4Hx)!{vw-(4lO_hZvoA1+s{$OiQi!c?$OQ_Ca zH3Yrl`KXgJ4?!*S2DBDn#j_j6Y^?DzT=uZ>Gu9NJ5Y(%jUZ===MW4wKI`qgfRhOyt zZSl#kS303?*`ewPm#T)%sT!s;Hw?PDs~y~)7S3*6;|=^AFF6xeBYIzGvg^t@BrhQ? zxckGZ!ccpRPOxp;oE19o4#WC$m;nL0HR2CBzr<$et(gW*_&tgVeO@8ywknNM`+i!d?bkkv41!yF&)(@XJ!2*# zN;>evg4Cb9!+@9|t2$~LCE}d?{I?s}t-)nECJ787fuxg%a zKQ!;fcr|@sUW*mwQ!?9jUddql^!@4aRWzmGuo?eYZp8z`_|8*strvH%$P0GAF8{&q zbcexyMDmbN)VTwC$FJUfUMaMF zP(CQew}p}6km7Sn%~Eu|7QrW9=%6eZO5{Z&m+-S%VEaOEv-*Qjk;@4OeMDzQLM5xY zsB{pMHFt0k)=f)60;?E6lk<1k#5K+P+r+9WaLsVHrJ5-h5g3iXWQ8`3aU;XrMejmd!(#D=-#cOc;ak z^@5F;gIFqmN!vNe!N%)-&cmJs%Ge@@plEA?BR`Bl-fvqy+=8VTz`P+qXS}?D>TwZ> zfU>`wL$#=n8H(ehu$8pwnXs!^x@h6%w3s`#5eJ2mCrOyN`|J*7l$$Pxjn+F%m)V#h z%^m4qin7^y!MwIF1E<3KteKOpi!{lP!uud5K{OL#L$B_4N8q;Nj#vo?$FcugzNjkP zqA1}I@b7=h8H33`wd(8P!O>o0Og5wfDSAdt9T5*Xz=_(rv#!k?ZmZ_(gLn<%$S8No zkho-MnEDKA^Ne}tq8}J87O1}K({ab}q(kn1F+B_D5Ed@~a~5D$LjpGgH^bV1X1dBm za4ZY?Mp}uMJ~T@;^jkrZiD%L`Jz)!EV~I&%N~RR)3)rhvob=VZbGDA4KU7qoKK^Yz zL{rY-x>q0Nxv%u|a51RDHcXjmZ=nzuxrWHH*zIsk2|2-iYZeu6)(*bRTc$OtZ1Z`P zxUv2HD0!j#T|%Nox`EPgP2o+KbOb}51HmO{J}Vz4+dn7*{J9DKGXp>L2SsFPybZ~W zvB?liKs0wdEnD(jw$GrsAhx<;=J-O}D0of8XBMAijHWCe4F~cTUAH-v(ijAVRQZso z5kfCUZKMI7e@LJI#$-ne(cI&gdBjwZj;RJ#-QoMpsm8L;75!s$c?MoDkvx6pD=}7a zGM?9&c4so_7Y_p3-q+43LMN6IS?-L$iiD!l0arw@gcr6*iB?xdDwy3Ofu!%pP;XbUX28<(N%0-{Wd`A;l3J`XQ$wx`SiDuMOrsk z!MBzP_Vy@E#5omO*(%R7$*;-YS2iiJ^e@xd-{qnE-$~7tyq2Ub*r8TkQCYeB(82Aj zV&c&PkmR?u)uj4o)==(oO-jjgagBOvf=$S@JE`;xPBNhe@{udHuDyM_oWLcrUsB}dmhO@r zGU=?QRj#t<-!vU7E!wgw;TOtLK#W;`5DZqlqxSo53my2(_4-t^*&JQ3xPE5>DuW{V zuNx^Vi%4n0yloGt^Wm{GcB2uxLZunkkvrQDZsqjlwSgm+3=J9lqaI)yIcNNP(&xN# zq{!lQgtqEm9#H#2o{=&->(?Ag0y86J^)vF|>5lA~cbY#;^wF>BX2$wG7lniyn7rVYDqi#E+;~^bo!FnrEeLjn%^ugZ9hSHgSA*> zohliG1iH=?&RHkZTy1ogDUr)h^BBuqsa^6^$Kpq{*oRHeD(x$tZIq{pQnde>%HYyD zEtKaw{QLmTxKhAFdanZPPev?2r~&Z8fR{ZP_|M^^h!K76PMsE5@9E{+Zct)6t|`aN zhjSWD5emnfV|WiQmMGoMX4 zg!h{1e(h#I8^{44>FKn{v^&KQ1P=d{XuY!2%enY-M^4UmhOD=U!lFhWUe*#X<*=>b z8|t>*HSti+564q%er~)!zXF!CwMkg>W~9dJB56FB$k8VlT>BFc%E|zu^88u7|9deI zr07kT*!Li#MFQ~*W(8t20r1xe=l_gMygG`g%$Es(Uq&x?{f9jNLeNXah~b|GqE~6` zX03BPw{>_)ydds(UKWqf^+P7DH|JtF-Z}{M;zXcd_E&cC(Ov3@G_mV_s9&B*z3d@0 zOdh$tZ=4Kif5;)>Svxn9WgDU)ASf457@~2(BQ#Kue@o&ORz_E)l5`8LieJ=c{o&zy z{?vvf*89Y86LxalLZ`f05!zJi$>v+c^M0u#=i~=51+G9J<)WkA_0JmzxvR5TMqQsz zjD*n1l?q^2Cc0y@cVuV)A+7&AvZl9t0A(srJ6+w96&S+35L$l^>zB(A%M@wGe);8b zs~PyZF{xWMWjwRDoJt@%gNt05Z}hj@jy)B1ECUhI(+2U%BWwB)lh#yC1UK7eNbQ=w z;0KdzjlQUKhH6!7*;<-eo`xG@L0#uK$ch1hws;Gp)HP-`AV1uj#8w~X6Xu^G%isQy z1`Xa;&hsz1(BPjw6_Q2zd=XloxwT$G@PB82pqRDrJ?oYb*}3BxB4c$W2+q^3wB=fU z<-r+_loUP~MLazA-!*E;${hnpUJYOjw=EiZ)jR>Um4&Wkudf>lYbS<+zMPw)vxsgWL58+uep?3SaRx;qMDJtn7_CWW^bckquu8592PgPq zO~RxUDK#s5x0!O6vD8!cT7Zun4V;=L9Kz*eEA8N!-2`2^LE|=NW+I?#fe-kRF)~O& z;(GS#tecaaAJD0Nf9X{*|0&^pChu6vjT zjdXi$DIHY9uX}TQYj8GYRCBXNiuS2b1`ZUicS*iUW79*7_dbs&@~z6M+0TYZZBxF! z2e(zj-d+WsxC%}ubm_||8saFIQtkyZ*yT-=`&B;ikOG&uNgBq250CpX&B<@X^-wWdkw zN)tIWLqKDr_C}IIhp3G1OP&vk9scbqfF(3iQ2wn6TeXO#9tL-}EB23VHCcj}P;7g` zPCG_(fv)~SQc6@pkFPGwj*1=-E2l3j%H1t&vuwE~5`$fBJ~~)#OKOxa^%J`_QVht% zyIH@*6cT)wxGOQ)Cj2@oBe48#@^|h7tjn-mNj!|b7dpObEb}rirU5%(h))IxxLkoS zxl_Vv%|oC14By#R88|@-`*WV05#X!9vK3tuFy?4e@FAJJ%S_OCC27DVA!;J>coW~H z-#z@b=h!}v$NJ{hQ)Of>lN4W00~xkz!5#wVA9sTe(27R?+uj@L6i7lFY{=OP-~#eYYq|#I3`7j41ipcwtgGC+UQDpR-xy{ zVW@u5h4Q0Zo1Dn9#hRNo=(C&}konntKQ2qX8FKCvwk0(WOqeYPN8`bof6})k~{)#|1cBJ{PzZt}P?VZ$uZd#$7k<6}SGXEh> zCLa@0Ry#qmd)?Z4_CRrHb4|>XtHIFZH5WmXG_G<&m787X1tRba0ArLtyS1O^u^}5r?w}-85>s^aLHqWb*za+W-%+tG-|m(Te!2O* zu!Cj1y*AM~Y)oBeY=S`S$|>j1{K-=8i;|!n?dq{?{~-2I@^EkxKPLd#9Rc!{rIr^d zX=66)6N5;c#dNX{IhC<`TNUz+MsH!1wi{ZKgc=f&2N$&0sa%$&udo=vHuXcBI>j*8 zunZ}E#}woF`J8ZPd~<8?U*!USVcfupJ9%B&quBoJ35Mi$Cn8#G4l#BUJ+{FM`gg#r z{tVa;?3F4w(K|Z9%p1sXIRLp~Mc==AOXxx#aUjm=6IAgTIu=6=IB4KS8xG~)Jk?B+I`R|f5{6{UsF%U*j#LMh^GDpFrnBf4j5 z)5@=l?GoR2zHgQw_GtCP0XBG~bPE&Saee=xx3x|JWv;q{kW{c+rQ6Eugkr>wf{uN- zWgftieeX>DW(EGj@}Kh7I?EoaQavwxQotj_fTt3VJ#kk4A!due{vULWBB z=Lg3j9(;90T4+^6&^=HlRD1F|@+KJ=wYFeO8XC*Pc5)`+&H+`}E z@?+Wa<4>4ndDEC53wfd`ADj9`chW@`w}5V<1UF&e8Er;{~hQmTvxF;)&$Qe$;}q813p*Oc>kY8E){Fq|;nus$WiFs7X` z7h~d{Uxiu?{@dv`;#TpNncXDcp)EsPF~ zy0%T`6fpC)eMq3C|Lx(8Ck%b^_lf2VRPi@%zr=gSNzKjz0y#ZP*aVBU&ZiVPOe^S? zKlAB)u%O=vPd<7ibH0{sz)1h%lOQ2#*i@6_NH;&+*1s@LRX{ z-%$f#Y){{#{s|=@j$07ByI)}F3%g0@IBM5w0gn@?$_vWk3hhmZmr|T#%uJnX`Hfa{ z|GTnZLt9iN6dbrinoaT_yVYGuq)Ml3`l<;yr^a*0x-^jNYQw)E8Bl-THZgP)LEvQm z*QZWbP5$&$@jKT{MKqv-hs~sH;z$~Hhe@vT-=3LoeAN%>^Wcj#KH<(ypUK#ck%pPv z!JXJwY>4tN7x9}`oL;{*dh2zj!u!X=0-j0n{_PQTT#Ck|gWSANCF@aFhFHUKm-~3=Z@&UsbO>HY8SeG^)SMz)2TXhaX|F>n z8t;)jT|L7!T1!{M5fxNL<|fW;SX45nc{u*3H4%F_rQMi&_Yz5?k^e@}r3E1QE(4OO z0*LDy3f=^W`N)BgRal}By>RkhjQR_JyK>P92_s4VXy=cczI-?L>p$WD19bbB)%Phl zN!gwMCD=urI>43&ZvL;y_^&;4^Ah`||8*4t;aDbKN_#o9D0v zBpOV=^#6PF|6}NXHUEIgto?ptqtE#8N(7oasL!xiqZ&Q}mnB-hRP?0@tpD)-uWkfn z4sJRm#@S~D1Xu?ILRD#Wqc5RJ(>pp%^Ej1!8W=#pLwd3FH3`;#2kdY0n$p=hrh$Wf zH2nNnZMw0_2wljVCnpNFwj}uYWkI2-FK%rM{eQGZPfLsWWmXnlZW?XLUZT+d@DQoH zN9N_;+yQ_#><>Uyzr=BojAQKuY}?%+1HvrUC-g zjt6)b8R%Ce2({xtmg-4s`5*oaVLOj^)78GzUnMFONv!c5zUFj<$(;>01q@F#EE+AT z6hgyh2Vz?7UQCuURNvm~PaM!GH;lduaV`&4suXQT0DU)uu=8#pj_eXO=(LG)Z)Y)q zYMXQj-f@k~O&v|YG)l|$A}zdo+ckum1n2dW97jGEyz-k-J4P|Xq8}V=12o0ZyFpsAzq;;F1f<5zx5iGP>S1^M#y>9YoxuMQayl!B7$F zbkTCC$J@SAI}ltsal=)ooI3-Joz8xCn=i8KU5k%JI~sLK0{551hO!;WQjH6-eS$51 zxDm+s9v8$^o7QGX+RSwt^Hm|1#HUPW2sIl;k-0|xo5|FDq!jm_JxjGr9`Z-m%{#i{;JHTd;a znRUhiiVq%``&R2E?-#n`KJ>>WS@n*8-DOWCb@|Q@#=H)yGCW1>{ox*PO=1<`c1)s6 z%8pVgXoR=Y!s$1V+*LB(&v9w3Cg@qs=dVur?8^*LLXO>plNuaxrn`Cm5FHm1bFz1y z#ahvV8y9+pejDL+;bw6~q&`M8ROdnueRHcn;UK=8BYU_S<8)BUVOpDxK8~8#x<`?# z;FGSh4SPBT#dOAX9?IS*uRpPrh=ebY9MXrj^bv&_ud@j|9m`>HYOq1;HNsWbE9j7A zaF~Mteg9~|d%pGzDZ6$F%g6O7LU-AJA(-72l0+2LOud$vS-#G2;Ao}-Q$tr6xLM(+j zJFuN-b|0s`?g1+>iKHiq^2HRzr28BaNJ>l-sP$os0JoIeXRN8R*8MIRLiP3a4g$Bw z?s$#KC3qxKAR>oCc*g3C4Ar?KdqWV%nfEE<(7i;~6V9t}oqx3*W6mAqr%#WW;;k@V z8lqep#9hiYw)0rVhBZUb-J9BiX}y(Ew;bhtA@D|sJ#?l1Vg13A#Lt7#!j%IU0&&?V z^172-NE@pps);$mwwU4btFfXJ!i%%>*ZqcT5mgaO*QZX-WhCAC!Q-sf$LNrdkf&2l z;(ZORzcuuv=Ns5}k}I2z77U!rwIW7tW@SBh-@1O}WAj=9^0-`SRh;7x`zRyA~Tp6GafUAcO~ zD^dneV(#F@qi*w=!Xy!U74wXc#cwx@s^pUG=>usz!(~Fh8DQ(-CBe~dwxh$XEuFn_ zs1ho}Cy;t5p8m*hbGMekw&2VTVGFFC-Hi#&)9dQIIpOua`c^Je$XVF2QUx17+bC7_ zR{j8s(hKv8UyDeVguzN+7gYm-RqyjRiK#fJ^G(%DdX!lmJvPH+i$`){9oWRr{Kfp* z{13>vy(2sKs+cq#`n>$-yxCT`J;oE{HwrJ&uPw2>7T1EuZPVL{y*BvRuOT=jVM^fD zjX^<)5iL7|Bf(%_WD)uT4p=m`W{kc^*#}I=@nii(|HNLKh0c^FcV3t28~WH`3Hgwm zk~WD`#mERe7j1B&Ujj^J^zI=*Ltv|+1Nt6*-H%fGPAIV*S(Npahm==3pC6m6!&B2P|&L`)%cmnmxSZ90}jquEw%8~vW$25%0e{I~4EBDp>O%R5HPP}5qsF)G0u zeL>M2A0U+&<#E6T(NarzPpNCI(~&Sg)3Jm@a(52?_(tq*f>t9c>5@udR8z7nF<&H$ zs9v5z8#@*~s3SO?l&RFYR?_2n?~OaQse_>%5sZ?{DU>qiwQ|QQqbICLSF>Cz-@c2X zGo^cz4>F@?XMA7#$xiiecZlWHH1TM(s_dH>%Jd1cEUP+^jSr*S!$zU3N#*`BXNVqP zTp(AX{1gM+x-)}s>JEj&HRR6Mk&P8{%req_8rMM$Q=bPO78IYL_kK`3rE{zn=h6fn zpyaDbSozBA-uVxun;s|l1fgD=L%buQn58Nu7Y9*og^@s-jVe;PsW3^@Tx4dtaZ1*7 zIkV3em`0-Q`fyPZr9B;7O?sUV{rqaQr#Ubalv4Dbd1+{G<>)iM^?LtBT(Wyu8yKZ# zBU7!HRI1SBk%Z1Sm@hQDcGcLzk-gs0#vRKq)-f?RGrVVAf6^>>;`8CVAvnBdk@Ytt z)Rc>X{o~D``aP>3rdYe*B7C(?#$(K`uSCspIToDh(*MFzE;sVqq3%K!pYFi=#|dq& zAEUo|>ZG_rkbGkH*Kv}-Qu69JT&bZ4^xDB5#>U@I>0r!3h>oEb8nb`S0u0)N=rll) zZyVeCvatgvoPu{65|h(|dkX0aw7F{7S>m=d!E>1eXbbtw>XbsKak=sGG0Q#J1TdNw zTRVIcO=9$i9{5;TRObs1<9phEobgW&14ndZ41B1mk`BRv7jPfgOu7k%>JNs0r_G%e zh6PS%wwtb2xc{vC9NgEbSJ*r_JlLuGYiD;!o=|gv2P6nCv#4_uay$9D+-QUMWo_M9HgNu zLzpLw`9Ao*T^7ULwB-<#Kz1`ra&6j2G4h4IbSz#0tg|M>!Lb2w{5?IJp4GjtI(#Z$fp0?}+D~y!Y+y!q|~%f0RP_A-liXPl?3VCn()q z9)et2z1RZk&Qiwg5nXs*+D>vBPS-o{ ze72N&$8uZ{rI_wt34Hv~14cx8LRT%*NYqDMb1-5Vq3oQ4OYG_#Z?2WEaeRDTzIusQ zFWO8M=h=3rDCmWJa7DWbn*@$oB-5V)f^^?Q7m@7dqSjhsYmUpA=Gxds=TD6qh9&V~a|V5iau4qE`(v}qAtNkE z7+Nl;D8=$0WuIW+BWxvl-VsX1(i+ZnZaQu-)~%gv2FioK^~85A^b%z;KM}VwOz!rf z|Ne^E@26&9iy(=jlc!$0qcP=Hb*RYu+`)Q-Q|eQR&Og(Fg98QGC0Ar|OSIyUrd*$) zB7qC;))`zPLik|7d}YYwadhj?8Mejk-sEl!gBTm6(h=cLSSCB2GcyE`8*FQSdVNQgC7> z(H!l^Tv+n%;`H(aDXgj?bNIy$TToKo(g~ToQA!1ew2OZ?C$;6Y>u|(FHUphL@+@he z++4vF!PJ!)8k70y;h3S|6mUL`Q(33?IrzwAER#gz)9KvX62R!{_x*UB*GA~_OLz%A zDe2BPQZLSvDw}2Ld4-tP&g{p5GsPuA%>&dn5I;izS`M1!9$U*WU7{6H!!! zcwuu`JAys9`nnS@V>L(gB63-nBa`2;LRqnYNLWqN|~zJLqUI1YIZV+b(b^=vi~VwACt^> z2Bl*0MzR?c@0=kkDFDf|0?KASKR-gV74}$a%5oji)&1r2@E#nU>Ss$Q{Lq;_+-|pfx`(SZ0Qo>^HMv{8 zYoXCE{nQsYLM#-peV;G@I%oDl^%Y+vf$~_Zu++E_O7IH{FEle~wF4q0VF(b`3Q zeBZ}REVcXu4*S#_TWuP0s!uUm~UfnL19>^(|x?yIGcg?1=8^1N;jEq*CQ`@)~rgd(^e7DVhppDFzSsUJCw3#XOr)2mQf||j}F{hI*CJk5p<6a+#%tA661hu>P;SuVKL*lUd-Myq-j}! z`t8?>iH<|u(dXj%mJR&&*keQ9(#I>S632SWFw{IkNsMxPK-$#X@e`NAQeSxWW*8G4 zwDj~*@w#(4j(mJT&jmh@8ywTmhV_9)wk)RE$g}a)NK_#cs#ZtDV~BGMj9aFPC>6$M zM-K9Do+?bXs0=zwpgzlqT@*|sp{EEax$-IYq1@Gyc#kXid#A@=5Bh3=J=^EhJ2kAS z9SZ)em)$uH!fT*LDNrdw>B=qY88+`fbXNRjUwFe1G1*{8%N&zAw&TM@H`~Im`@ow$ zDf`V8Re#tw)NYQn#9_e{&-uY3#ZYZ;?wX6Hl`Z?oO?@}U+|IWvyJvkqIV;+HWT3_5 z#_crUAoR&SJG;9@PV%$wv&{{`%>`qAUCXX77jjO&TPWZ3_YfBDfQ?edr2mh)w~URm zS=t6;h-1c>DTbIaW{#PeIc8>NW@ct)W{PplG-hUqnc3dSIp;a=x6=D(rIl9uXXf^; zQM(Jes=B)RDvL+xd2KYcHb;o45a+zsG+cm2+2omm$=f_p@Lh11GI}$MaMJs#F zfE_+e8tv3GpP$UcY6DGbxk^4Mu@GS2G1HEPTuGKy;@z=wmEY>A#ki}r4jAknwaccl zxZocjg;l;%2*uGN!7W`o^a^gF44~skHW88CSJEPzOt87RO~!U~ zH-^H8K)UV~Z77|yKz&fTD+oQHGCULYO6`ak6OiEkEqSpaYnO$X7&3sL!)WBh+jT)T zahnELyR2RMN88oo_gf+&vqKUKLwqs1-c9woAxk9b`08(TljR=HFwS)-+h13hS_Q*= z%sF|4o;T1lfG!K&aG`16j+$kiT!z@> zq=>vjAE)dy>I9gJOL(Ksw(&9Y*m@KcPbVsrjTzYc)sgr@Dr+c1FsE3<4(X}v5L6UF zGlUMmzs%S`VbN4)bZ=W6ojF`Uk5s7A8kD0zfQfy$BYr>HU-di5$-j-z`)8$#CRr4= zM67b0>)zMqPHtNrQcCJnQPC;lrL8U3Q>h`(Ys&9BSVCtS+HQKT3^g%sIi;dQGd=ti z!i|wys504o^qazQw`kO9jCR=a>)8gUxz|`X@lVe{Hdm)LGZFYd!s5C}Zj>frF9*~I z<#F6hZW#QZMX^6l#Wwh-BD~VKN)G82zG582Tg{=x+XWkl+W+1W6%sP@uM`P?R5G9P ztrksVG@=^JIrW`7ruB5_CW6qh4n~T_Yzgpk+y2VM9^+zSb0|>#C6e>}^n=xk3bTLD zIX*pwznVNE0nkG}I2P(-DlUNmH&6;uVZ*LMNC*q7t+m^iR+Fc`X8Ot-vpO>9AbLgR z!lomFuX+SSs{?Z@$&s@?Kz;rR^KJVzox>BmBS9*2uS99A@#uF}j_}$@1bg4?lks!IDYv38(*w8O1fjujfHriUZCFTy$6>>H zApLqsBfw-eMN8C8jJMXG{$c;}G4~zv`^4dtt#RgdK0pA15YF9u3s=A_f}SfrUTk6} zAuUT7Mwe+3i$aAeea#CnOz9gx?{T9PfN!bwOj*ntWe4!$*K(yMuu(^gK!iPA-aFJ9 z$V?ijyU04~^rRq=4@cHz;(RBNegE8SqahhB%Usi47ij;!)%g8tzO~LXO*H|@IGT8= zl-%i1NCf`;=Wg%jbcq?X&OI}RH$!Mt`$8wP5lmdSjEq(0NU2^x+hH1tqm5hyxrp>{ zPJOHhNgnZW2dmaJ1zEy9-yC{jcoDQBs=05&<9BOMr1&xY)T_Nl=aw>7&(R2JI*~X$ z;k|Jfth88zp)CNk=Y@zlgk7A8!8BS%sq~K2OBqE_M)>Oouk*d<3wn|{Vv*T;GBbW> zkRMZu?YBCfDiUc>s`iH%dtKxpeU9FvheH_lsbKH+6!xRDY-Y?vq3)O%1^Yyt^gDdC zx&X4R`MO6Z^9_D}z-F$QIyE`ll>Tc7ZmZW3mQGg{eEX?ubBS^61y!$+M~}Xe8tKge z4e7TBtcxCwNb;c+kQvBg^nhi5-^YzyD`m7?L;VwsUTgfi8;P}NxYY4hdUP^aGa}0e znCbUuGu=jI$}1XE_#rnXOW6`ISICY^%V7%3Yyvg{19&u__!ca!CNZASU1;fPfYO z_|RJ_diiB7NK1@!5?hgA(N#>ATg0CQjxD#*Q$-ec(q!4m)vC{PG!>}+y0IIXae5F` z!l~G|I!w>8)U(6nPkfPQP?7vKhJBE&0HcAj8uLMwRdGN)cIddNLEYsR&t@W9ya&}i zx+ZNva|&UQ=~$3c^%jtj(^pMfXQ+{nb+>|=5B?~b$d!{tZ4BxWIhKov@bu2krnS7H zZKitXhC%n1ZZ3QOZG5ywVbb%yQd7!Y&cQqd=8y_7!$LDL58NFjV2mo#=4AS9+=M)M!SIwfirm?hB<7C^0~pzPc{F;XPD*jLjXPWa_Z_-$wVD zj?q3nm??ZISTNA^(q*WxqP~3Ga#!#b6(=8?dmVM7V;VhW$Py>o}!U5*nGxsw7h~!D$tyL2n&x~ z?Fj~!MU`S(7lu+_aFg#~Fh0}bl8AbXNF^o>G+kWHruwyhi7HTB*I#m=xAlAiL@wnF zKBgYS!(P{3G=z1&OjFldg3}6CX-$bgm$}M(KG@ktb)l;RhA=!yxruaf#jJdnE-cR4 zTFY}3Fy~zUL7lY0DCvr5oAvB}PLE_HKP^R2x|$AiAw zzvA)xOz^Ydn-?39kWlbEfFun?@e5q7!z4r}yDb`oj(g1NSQ_QH44L<_fX<{>cnrCZ zNb4JO0sy{bYE?uf`q(ds+7U{(oV9e)B}B#HQ9x5eG~|;^eDZ{GR1w#1`IEgVBVRqa z^+85vx>U~7HytSQtFcWGr8e^S5m!n#^>rqb{hcnka6o+#G$_(4ZE@>Fu#K3*6If;^R(T60eufXv0I$`4gW1DAyaB%e`>_Y zVqB`8%CXQY;W`y8Ko|1t{513;O}R2`YL7~=j_ekH$^RSORhdM@>YF%MT~5U3R|L*V zzc^+yIed>Yi@Md1@BAb?s3qsMpWen1QzftytO7j7Tqa>gGLz3P+5M(U$Ei1 z`q)SwDn27=T6A`H+tUgQ=hv5qX;%%8`aiCz*VsXgAY>{HT|{`Yv)B{D(8y<7^q`o? z>9-uCGl;AE;CCLZew{2JWI84jw@YE;JuRF19@y8jc^8FxsR~ znykj#LAZt0(df9xj%vFrG_{81G?WnPEw`+~Sk8p4p!B}dgVPxMS7LZ;e(LqsdBvE> z7RYNt?A5_!Z*+hVRn$s^p$jYchWXj`DJj?cM8wr-sLtxfygQXE3Y)2$h;nf<>K6>9 zaD#`t;v}J*b^Jrs&RD;U`~8Vnth{X83iM=XYwc-?;O}@B@r_*_ad& z+`0wP&3jVpC>D@IXhPOH&HtE0_g$_5#}95M4kbpMWrAfWCUbbUghdpE(p;$gR%=Cg zUabvT8!`WUU1T5+TgTp&E0Ul>~yPnog zzm;}J0asV>L#in7`WVlIV3?R)+1z?;xaCUCoWvT#?QjAzK~AK?;%xk&K2R9j(hp1cA7;d-$4i){Siw&)Y~$mw$ZU$q(-vfjEJ z`HF|LNa0PykWcjBL_M`VziCIdI%#891Zd!JW~r;I2#eC5I4{(C{~Gf}RfH4K8c`ZJ z&5ZE>f%PpR0H!l%k>A2Pw!z}WqR-PI8He#&=%e7XC|rlRkqzwT+34zV;{cl>*I*F`!Z)x!XF}uCkS*_BaY?Txel*x)-4Gp9{ADuwWvPEm-@7O393kklN3Kv#Q!rPP%*kp>3 zHqAcYl7yk%pjRG7OkvJ!ufdt6p~6ICaXOs~kKfCt)fI*I|A2}*XmkwF9W_5H6b>SJ z%+coum7L+gKK@BdNmZ5u&->wAsV-WLA_-@M9ug!P9G8h4AwwHM?(eFK#pHnWiE%E} zMlt{^svq709tOtflv_E13L7xa@`WKl`E-LhC+o8l zVFIN~Z(9iY9e8d?H<-)V> zLb|-|7T^`E7bQBYhM95Lw;YEe&!4&jEU=jlK5s0Y9Nxo0p<>+1@6(c20Ph;bujFWH zy2wo&J!4FYWlTih7RKFgSOl!B_2n|Y(s-`&7k52!a`m^{X~!(MR>n@ns2uEMoISDpUY#O6jyZ;8@f-hy~6RVa8E0{{e<5!S7y4L(f(Yd>iSp>Z5@cG&>&l11fNjxw^-kf)eo^dGtED zftWu0U0uDE7TO#4CB=Ju#h>T$RC~P6HZ-io6tnG1e%WQnWA!I43@IaFaZ*)BQ{O1T z%)6sm-fTdM=SVXDnOTBqo`|6rPT_#w%##Rd8G;kl;YDsP?9Qi<2?2L4Qa;uBdu;BP zYxW<(-2yctFj%iD@cMsT4l5TDT>s3sj99aWqmAgYgrWB`S6i+_rtl+tyf=B)x#EW5 z)06f060!bq`4e?ApOV^PNlq(M6NBChV;US3+G-e|pm+RYQ%`b-A<}5!57@%26Jh(c z6x*!zehVjMvF?AKDaF2gs4KH;QFHhvyXr784DB6$xo7$8B6;w{1^^b#S1qxR>N{=2 zN){#>9I+DbB#1TtsZ8h0Zu96nPGA4jA-vl&nk2N@Yx3hFna$OR-^6-st7OJ8vefkS zC$g>3z>lrj4tAa>wD5`XgU*VJd^31J*$!zXfrb0v|_XCF;{hteFLh5bRatu7(}R)wRpoL{a63}Cg`@QktX5>GSbyJMq*C_Z6(e+#f;#GtrLpu_nr*3|9UvSpaIwu#hfyjc#Zj|aEeRR#al0$^I=&B8u_P@J2K9duVoDOP&z z%AFaXlGq;2Uw9LHQbEzxtp1boX zW7+a_pQV1*Q&Dsv$royJMSCW>g7MI*1Kx?C81T0f^AzNdL6JEW+#@fWnD2lZ$3DCe{4S`E(3Lh>1Mh0jI zDFRk&G8NNk4&4(AOLQiMCKhLMn8SBb3o13RXL<4*z&RI^?{4im_DdLQk@5DV`~un6 z0a7gJTy=x1CA=jC?j@eOKn@l};JUmkI2e_ew^X!AjK$B3W?D5yBizsE-uY*Gfucod z$IiJYoK1K_AnT)wXSiGc-Ev7uSnphNjq3uvTp8W^@Yjep0d7}FHvVo}3rUOw+4a$S z!VJSz()r;}e&3W2J2YT$6o8IM_bA-Cu6hJcVfi2GD2PD3`s-om0&?CBtikPs4PIC%sOAddrV4ejR7 zr_g5CN0v0m05IkVzq}K?vD%esX54m+K0<~WW?YOc15-6isP+l}$s92iD)H48`Pv3} zV7?wWjt`CTFW$LKN7brT4-9 z*R!Ue!n;=A3_8w?*54{aywA|{O1Y8Oun_CDxmSvy3BW3{9PF%qNI*mkp|pQaA;+Ws z;2~@3)0QD7-ceIF1PCD4UWp=Kr^s)!Fo57zdTmIL9#E6!qTy`X(}wi$`bE@~*f)(; z&h_3;jg%m-v0Wa-hAifwmG%_zzt4*30%k} zH}XX6tYJPMP)By#f5+T>@VT7|KTy?kHRZTg;hbm>N$EKqX$#-|e3QoGgmJYl3Z$Em zK>$uwH^Hej)Wp`8eeBe`shFpFQwg9~YPP1nX`jUH%prS*=sir=O53=%+{OZ#gMD;f zm@i~&#^&ENwm`S(wFQ9!fdXUzMro;{S}aZ+a((3aKLXZgKeAeMo&((vEiw~Jh*ES| z0}pv$eU-@%EGT$mYxn0Lm&Wd_j>q;fBRER$3T2wTwQiTsziuwsRd(jllJNzH`K(G9KS@Ki zSOoUkjA5*X!_l3LRmt}BS1zRJU0X0**{F{$s$nk|6G8PL0GS!3(RvBvsk9~!fz1l# z;cj~e4h+n{;pKJIM&I*4@99iuKzBr37zBUqjqOt+BQJ)2l9saZMJ6vOrJx35LaFF& zN&QOa8P}2X1}oZ6pk+5mQIE#V`*hfLx+SVWRIUWwG0|7YEZxs=derV$e1(Qh+IS>E zeZj;1F@x8>Pg`J?ax?6x$r!;dk#u?2n&Rrz=h%m%)pyjBBd)b~gT&<6L$q}J9C;l_2R31(l|JUvx?ZCT^2vPbIv8ZtE#7`)A`*&X`Zk|7J z&A$qJEGsAoJ3nyZ$!my(ZIX`W$gsjvE)__=hq*1i(I z;sDsXIeUeCv>^*OZS7wMk8r1;t3DVzntFP7xp$fYa@-#hQfMC>aAO z^`0^An=|U0Gt7XAg`uVLr8iu{zZk|;2~H&$r4g(3RHZK-7i~LdqI&JEbWX@o>U9H_ zs!TBA(bh4_21Y>{<=YWFz+fM`f+-w}oCW;zjapX$kx6Ds;Lyl%aMI%(Wcb2)?l>;# zkg=Z}V?|0;-}(+bPT=xo(R?p{bNob`cm0o3CAcI^1#E17Pfr?*ZsD{?Pjm{9$*@q< zB0xoDN^NkU-}s3fGY{@cxq;c2lqsE(=Y_>^iyGZ@*FW~SdaJVbK1_krw*3mn9y}7g zJZ1mk@ulsWzGa4kB=f?+p(Hl@l-y?%%v0K+&SGm!8}OR{t)!_ef+yd%kb)4`g&@$G zat?a0&T(Jnhr70TrzT)`R(=p2@)K7`DpC2Xjx9&sAI?pbTSbD~T^J<5mTIx4nAk|) z(-Z4GM#>BW9aNM#n6uS}fRMfkDv6dFZqasDTQdGjKp~lWDGG(&OY&Q*!U}S=*{`~7 z2AZSHol~pUojM|Jmy0p(=gy#xYFx4Su_3|`VjrMA##s<ydwzaG5Jqhs(8&t@I9b9F9z;)V z9+wBC+p|wFmUH6Lz5D8Y8>3>J^9?s1v!z9%#SA;6n(7Z5S!=h6Std3eneFl_-Zqo1 zG=F}Cd3hjhSzv@{3tpmOV{l_o?{82=Lw{9NEBq}U69#Qf zAzhz4;h9`RoIOpue12GWATpEjRu1kWBC#PHnvmt`yL}D|v~^J1Z%1lfY{D5#<)Z(#V6|NvrOQ3=(bbiD=J6#q6FWM1%fl=X&~kV>rf~_nBun1W z+jTn}K17i8(_V?vV&Rl@niYn+NlN##$HO5o*V~~lQzbY2f&p(Y(}U2#`u)pp32Q#p zfcTMNBUgq@1gNz}&g8dI!kv=ZK(>QZ3sKuK-RwgkuR7j#c%I-1D62n&hjGRk<&9wQ zG~Z@mn*4Ptu;_%6uY!Z1JI@nusfm!dKGcPagJ?p+{KDd>V~c7dM~y!$hwy9p+Snl) z-|tc;f9pHi*8^raMvg;U=@yR8%ReuqU!Dd9_S>Y=FTt|M;~dACAFEP}!?xHA(#iOA zL%YUI8g!=iYHGR#7hR(%RxgQM)<0ky*wAMh3nDaxza~Ett%x-F_GEURd5BA5`+{Ig z5(OowN=r)xz&Jyc4(j!#zhAxKpun8&zOav&#!3vY4KH+o{fMu^+cyG^24PXAT+J~p z#zL3=;`Jkro@Rll04<;W=ptc#Zr!R4A&#soxiJu+d-q#-BP%w0)vKX~A?_qEQxbKK zSFlxCoYqVW7%PM)UIBN`eg`Wu&kk62S344`Aj#3;0+F7}vlPyZkMK|EuG7_t%cq2^9c8Jgti{tlGXo6k3@XGEV$ zM?DXw-ocrgFwcN%ufDzSZ4PP9oNashG^h@4}arDq|`qa1?H`rHA=$-_@hVWbfEJ!l_ORg{~1Rx zG&D&-Hipn7wOVoh{<tHt1fG!9q{!(}XG0_?TF#B6Y# zv248EXq0sF=L*}<=4x=6fy6j12M0gZxn#76a=taoh@Ov=nCi{CocKt5>KH9Opj{hU zRAh_aC}$BzaSnH>FDtAI=wB?ukL8@w@tHqfxfbp2ozIQ~C!`{bh#c{|KZ(-8D;~7sNCjpn!Y(gotb#ZB=U?)xH?T?!lx&DJ{$r>=}rRQx|V;gSY z5u*JL^^}g7@^5?;_5AehYzT}`RjNx`Qc=DwzRC^<_R~`rL&;0=5jEX$3vD)+kEvId zw9uCxHv=riqAi9holM!n9TKJT?MX>(p97^J`9RFPmj+VkzO#-Y*@x!fI{=s*6u*ZA zvEckHDLs>V(*NOB#1BHesxf|A`JVWNU8gqwj{2~E)Kccfy$)~ePjsPP1M6uPV~o{j zqdAJD-v9$%lNfqvrP`|-7%APXao?L&wBE8Y#U3`njGzYB1Mu+jM-h$+6I%WXzS#4- zOgxo(8~%5tmMbquNm8C?gx?)@u&WLc9BHHePMsf@2{hNN2eX|Rxv%PIeV)d%R`1*HF~eJwY` z;gfxmDIV7e@7egJG31b0duypd%AUsOO%R0FoHaGnHJXfSZu#*nd_Wjl>`^i)v{IJT z5s$=|Hpy@E$zQlK^Oce5jz%PV0g{UxGsO#x=Lg%(mB;R~0`e!R@Cb+H<_SrCd)U#X zfp-DKt2(>q2q~Qr6pW07qXENvOuXNRpoT!S(V5;|BP)zab`33CB@4yNHT4TA{3)3< zcu;(j_8q+g=+;BYYkos!b*7#X{elu|G#i~=O6g=ud)LbY;*}J1CBZiI4Jp>-g_VUa zSmWO@9Z6S!nR{4>VWeGYSwGJauPaKoLnn$ongeRcWQ{?_$2{@k09bTFQ3iW`Vn2J` zh|;3Ay+-TJv|wyFP~9g^wmba!V%*H``(){S6_)qLzGKw%v=hE`frxpBWLw`03}MU% zTu%3a09^a9Vhj0Gf)ly2cYovLVb8C#TYg8E;oZ; z3&siC?byn(4D<8RK+EY=)*^Q2^?>eUA7UU?n6(p_OV)A}DS1dB@NVa%?D@zlyKR^1 z)tTCE;>F?kdx<>2;JcjKc>Rbh6W4>67Hd@Q_soN=$%FQ8d^>EMTr_M=P%Eg-UQlOQ zC08=9mz+WDv)S5f?eyw`6CBawz>=t@)z*v(u2*bWW) z0G{s^*St=?usiXMaHQC;PEQVSg7HKJzOuC%&1pFP)@M0pKecvm5+hryhTyE5aCbOy z!ZFBhi;*|@3MPvaP~D!)n}(7ZTU=lP0*Mv`GCJZ*?-nvgBp7#1+K_oLuk9Y4{@`3U5#wFr$Ou|rQr?gPI)EtN+-{k zpAgJo3W;}nW72Y&v7cmrz?tMNphWbR#%a$hFw67}E94 zr8W{^?jf~j7s_3pHo!TH>FMZso7?z{Zd~v0jvPa=6UWn^K);rm?d=F`uX*mZ1!Qd~ z-Jh!%1~beE>pS`Dp&qvE+1vY~^t)%d5b>G}*ZCHvsgUW#c_%oa899RH2mBZU`M^~H8 z{c8k>KBgD@eS|NMFB7u%9&WzmB2lCyqmvhdwgb~|`8}uv@ANvaC&HbTm9aC~)O2m& z_`NNERo?-RBoafg`mj%W(e47FKdhM==N?N;#(=G@U4&&_JB9kW#nt<{F`5$fh?`$zmhySeT=UB=b;$s>_8cE@Nmlnil|T{Ruu*5I+{#-Z)zZ|o94 z26%jFU}@2|#JKTN?0u!{T~4IMX7bnyAb#Ue|2>X5i_t}QG*OIe{au%jKh-Qy3iX!ZeWwEIP;+^0V=84}gWyHs=3AQ5Ct zPMILD8Zx+WCr>VwSZ~#<_D_RV2+v&2`Z%a$D4lY$FiVRrI!%M<5Vz*>^`=d!b-q(# zMg;!+{CHa@skX;hYHZhjH2pwUeU>b&FNGy)ij(x&jhoG#eb;ghR&qBIPK2@3K>I(V zfFZ0;&oG;tnu3TcHp3f}FF_y)Ml{&n+z?s>4eFRRif+($pFe^%7cMrs7r#o77ge0+ ziSBt@6OnepT55bJG~IS_r?trvF+hRTgC5c^ZfA$hngL4d2`nObej9hIG z#c*ZUDhgkh<1K8krC&Im%}qH$OIBVIq}y2g7kw>(h>_*Q$1Mfl=}GN!bqjA{b&ASR)@+OCpO=47LPl0&CQ=beBu*1*5MViy@f`tO?RKZOAq z0c$pA=5eC$vqO3y=aA<|o4i3HQh(rQ()K@yj%rCOR`vC=KDQ`SX*| z4Vtl(7XQ`#D~nL8m7ECC)BQWlnuGqsbs-@`roCXUBRM`;{N9{IR^B7+I9_s)rhCj#aeVYg$HGTsuaDpdH_tngp5tWxB0Rh(DJ9lft#Uf959rtB zn&J5CMk&Ao=fow)`pyt{OE_IGKee@Wuwz%uOEefSpJu%wiSV>$%3pNJ?rwSCpDs_- z)z{FAEiS@zpG$1>640)vSL_LR!^*NpKMEYbMDp~^j!t0X*B|W|1Wk-WXRqHwdF$oC@ zv^O3c=-;Q9%@%Ti+K(K9Anx`<`2!|*IwUw$Kzv}fNdAji^A{6iD`pHsrhikcca$RFApJy;I}FtTg!_SmM;WG-2uV)ay}(K4Cmbb{AywF(v8dOdmq zl2!?Y)${GKag9zFXh?E-6%}EAehxm>VO5#5sev3`v#JQHlF15+OKQ0p><0_lqgOa$+9 zIH)r9a;qr})o;f{8Te#A2MmyphVM%SSLKgUI^UMZXn^e$T&U35|8}AWn?L+mxcbP!l86bo^SN}TS$$~`RWT0Gc2yUa^o(&;5 z$i%};{~tstuD{2He?@y~ptgJDxj{QO6aUvbuqFm+g1Z+kp$v_{P`g==x@d) zgeddkp2*ICL8XTFg0Lvtf)iF^G+=2+hQZU=4TsuJQF2ylp(`FI3rN}ZyeaQ(k$US< zsRD=?GKGXix9hP%HX}!}C2qkOnnVMK{gf;9={LI!nNoea-N%#f?G0hBgFl>{YXp)U zjx91eT9`CaiT6pgDagZgzTK+uQh(`0HTAbYwKe2_D7qCXgiN?BokjMcMul zYWMxtU60J3srIP1ze@H)?-}fQ!-3OD>mTib^xw<@9en)dK}q$mD;W$a?5PvU>(CZK ztpFK#9t&tE5=+PkmmfZ(aGeY_;kTE_PnzXNwlhtcOtt9NRLiU`I0k$ffx2)Ie`a4^ z9gf$TvwvM7cE4M48>C!(-+;PO*JgKE`ZZhS=6V14apljLiT#iR139wUnMzOQIh;TH z&H7vH#rNHDIINX22++n=;(q*~29sAn?ZdAS5;0a)T}p>P)VM@@psAm8n-ydgdZ+UB zoLpxTWF{`)IoYo%!4h3d^W~LpZfAp&A1a2A+8Z(ye zn`|!HTrDkbuW>(HCP%0B%XG_k#(i+RlIIL+O}u)15uA=;`${}(b{L{u``A^NXt*u} z5ZN~Uw{C#g;{*90o@JJITE1Uf5u~Y2qJaQYc)mO$S?=Zv2mC}C)7+8Ov$>3jBvX& z0RZZus>brMcWNA3#WIH3MHkAp+XI(%i zRqyWFHH4OO`Om45&r?0nn!I-0m~q{bt&sm#h~6$2+|*l~9iB9&{f=Y>7&p5$}r zFI}*ocDl8|nQD2mKV))q`>Or15dB{k`GdlX|L77G;TPwN0`nF^AS$V0`qB=QxU(kZ znD)@#@#+z`DJ1s!CqG|tM_{yJuyeiTlQR|ULp|_t&^kR(05-uEhdC7LZ_XY`x0*2W z8xe3ISuVyfC1_W<-f08xfMDEkI2XgUbYI3ww8t5nMvFb6bLSIFKJb{clsrvdhC!>) zgyp-4=2$Q{2x54P&BWezmDre$y2RmSh`oxlsj{#cd)ODKfo8Q&p@=e?F}D7&Hnyhl zMG%B5J7a`g&OUs5HPt}bhELMLy3K{hxaWvJe3{ukXKVc9HM_)ZYe*8C+m#(Z;Trp) zroz=ln34=YWjyjFr;f6j)`S`4cK)#^yS4v*+)_dpZ`i0ZZM?J#eo&X%jCk)`Ue}x< zCdrQPgV;OM1=N464Q4X3MvGNRR$!55k2k}cXO568KKzoOx@kK)Winl_N2!~nNYB{_ ze~!p8Tsn?m-_2jeT4+LFc0#MOh4?q>Iw$4g27%KGZ4_de6 zTg*)(;HDFNLuUSKX;~EOoMy;fGxnzQ>AzB#yftFF>5qFTwN7pIGClx_BL)2{ zpKtjIa`3nEB{O1sjIdwAUFvH}!*_Tx)S5VhpQAL8&$cMm`L*1=b{<6%M5n{)4z5bF z3y%{XX1M)vdb@BRJ^NXoGhIkvp#E{by@a9vH8iQohjupzoFhz_^;=Q6ZLdIU?%Nih znMvb*>zc94)$F~#OeLAUwN+6E25;hu6e<1NCPJp8bf2pfi+0pi(8+mQ?>DCdVvI=a z!9Cgbo2f5^MY?37*ea*24K?g?2RxGxY)?)YuWL3PG|A)MYf5`iRo3>83lid8ayjDjU)5#vqkr95h z`aSFME8iIF`Tk$pgko&Qq}kYv7MtIt+}?6LIgJlO!*L^90c79zjw!!BSxi94#78J8n-EmV^dUrFc_!&87zBww@VvfZSnkPb6N zj@jA6!tXp8?%^^^)(3e=JmT@JcPYM9eTm9v5e1z~9JU2WE-!mxrRXl9h4dm_)_dJJ zb5|ph3dxV@P7WbdP)M5#;KzXQRsW;CF~a{wsq8H)kK6|u*O@+XB7SwZYuyctu@XHz zfKl7c`0;pKo#@4xzSy%N42op!9a?@^jaO9Cf4Nwoe>^%d+B@e8J-0{ALlwhbOEVl* zd)}AAsL`78%cnX$XW!Z7oGnx51~kMjyjhAEhulnhO*JsPV1lB2HJ)P_GW0oV1zh`knQGID} z4H3R@^T>1PFtn>HH_>q?Y3;FX#cI7O1Da^E*CVtI9sPm7W({#ae^hRbe^-a3>0Bwy zwl($tBG_^Uji%`RAT|Kg)lbUU(cie|xsocR^BVB&0VRLf_6T>g4VmT@usHvK8l-)a zb*W$A8@tIl%hSl3RK0fX!GmXBX^h%lX508q-vckUJDzcotJuW={U|ndE_%D*`|1n0dJm)`B{70MrDs}XKV$iLrk(8Fe7BU$UkEN`}u)nKF zmU?ya3!$t&@ErWIaJt|l&CwJ&WKuq(_&ov%LsgE}W2*_?G;RQ86cGJg@zHVRhs2Ed zDAw=V8++)((j9~Z*Tb9dUzG)vK(W)=Vn`taXwUZVWz<_H{-0HpqgeZkNs+u~Dp}u~ z!-ukA()8;E^M`YL-@SFPk}x~KqOqJC2}_2HA4p5gFLRP)rNO4kow0-sgU z6urYAm6F=L-uJ8<8XErMRj+pmu4h%tbH}b^f?po=Hs^eP9Aph9rs3%;bUsOD`y4R! zY2F&C^_;sdb^{77CJKN2`Y(uLX8J!O@%HVhPjzzGB&o^GME)yE{d$8ZAu>8AM$b#+ z`V3>>!?>QIXysm5OAi@zX=hRA{RoYQmIK5(4r`uXizN0s@Yk#ZVLUFlznkaf4mH*~ zy}_?36=t)QI0w4p-dEY7`dzqA!qlbl7?y!WCji@iW`NE7ks3~APhcqqowR<8)#O>s)4*eOWw0Dcr zi}VQTj?;ys%GW(P%;=IUF|a(6>&|{bdQIX>mH+h}7SyXf$@|o>rb}&1Fa!Z4QiIJ! zy4&Skp^>g^EPbeES#fC5R?X=Ae39 z=77?RVmB=P4Ly&QkLHb(7XLL!nxUPjJ^g8Ls;af^- zeR<-Zdz4s=b6k%FZIA|}6Vo5m-DYHr3<2Y};DL7d-+%z0 zA89QqILcg)w?P26$H6*FgdzDJZyf8td^lmEoX{L>6bCM=V(h6EQyJW(^9Z>s*m6Av zXAHn2-PfO;(tWWIpq~7b-uovEZAk%UkAC;g4ENai{#(3NY>yl7moO6El}SB17aB%8 zk9=mRR{53fM@S%zYtuFg73$ zhB6)&{Tc8Z%sc+t-^i2Qni9e0gR9Gfb;y{?KM%QGugFWrV?n0Az{n6-Cy7K%8y?T1 z_&C*}dE^0rfH%bpuy!T^j_;3RUKB{LV;&!YCuTu8wKJM-9S>lGOvYBvIH_3DgdZ0R z4p+$W(kWE?K@rub)-^+<))=O1o9&GX6Fr2axS1D>;5{LO&*0q(H125n{Tu>|Rce+v zygfOwJ>j-35s)z%B=FSE>%Qew0{^HsWC`(qY=Vq@p27l+&Omo!14ls^T7X8n9smmR z(affe#iqc1d(ER|giO=#bNx=EqP-N?-p{UU z1I-p9nRz7dIXvuOwH?(LT_5jnA0G{q_LFI}2~rr)q+t3>4_U*};Nc3Y8WjpHyjC^q z))o~HW;IqhD=V5bP3BE%If{EO##6_uc}1RnyGzL+cV$6v7}U`I}~P z6>1u&Q>=;lelcLdLk0ByTEhJNp9j%-GVCR05z2Y-^c1=*u22Vhu4rkOS2&-$GQM3O zUAyls)gw#=?n(M?;SvQRZ3@L&bc%E-W{!kYACJ>t*c;jWdVjeFjVzWak0y>8QO)BZ z=$2?uf__q_QsL_7Kv{oqMBIsSr!+?6&l}lDPlG48KO_Hv%VVQeIvWU zJ+6VmV`~X;wC45F7A}VSjN^y`j-vzUQgoAHHw61Dh`;3YcT@iU_XF>!lkJkasO{G8 zGtY+&lQi>MWiNLo5+mn0{oeS`q<5q-+*tsK^mCK_(Tt^PoeX_q!r7Kfv5hxXHrv4v z**ulX=}-A;>D3&P6v@SA%#hVt0nxfceuFQqSCb}`4N>I=m75GQ5FmP2kRkCLcd!dt zH+uzT5oTje+}EkRAIGr{vs}2Z)S?5TxzdF#&W-k;8Jo+dKXxw~%1uxJbX>7}`HF3& zeD>Q8m({_)aw;bcp$rhFe2VD`llzYkGD@ z6Q{Z3MvN-ucCVP!+kubpp9_SbLR0YvBC-+RulYRO89p_zp#8En;d(89!1qX_#ESI@FD1~#~AU(L}E`oHb|*8qel zb9Si6J1$9k%m?4Z{GRG3g%Fp%IsX@m+i}B)2qJ3_G*?YIbVv$)^LHd@WtnDK^b+r88VI-^~5_ zs_w2XKe6UIkAA~3HUF}Ah0TITB7h4%LPdJ1%kcKtW*^^EyIx)AtGKu0!OYu~Z5y?d zsif@Hd4k7sO6dD6bGPu)xZcuHX>^_>AZ1s(mSuI`V)dn#lxFL|B42;! z_gRO*551qhW>r;ZQVCmn=QG163*{)6WJS;}Ht&C4mCI?%V1<%*p^{;w- z!NKQ<|IkY=;kUAl`8YCMYQF5bJx*Wb*!rsddXmS-PLVfCHPN&y|AyMHHY=HUH2NhK z9^P4t{`7+2;5by%J(nF2zv)$F^!(e!utPOc_%IjYp%%h3Z=B#+mdifj_^vaFX!`W! zFB&SfvdjZZ(kbUwZ45u(<^Q?9Kc#fTFDvJT+lL+J6*LPtuFm0$N%HynT9)@3w(o6v zEZb0Q|CCFONy}W>A+94EgJkTTZ!4b?NvTE$L|eFwwQ$Ygua*51`!@fq+JslInoxl2 ztufS3)7OmL_zjYd;*&>B<>v}+a7`6l>)xFi*GuZ?*fy5U=)CPqa+Pmmzo~`>UEas~ zszYxyO=L9j!qMrFo$@u04`U>FO^FAjgU_VNu-NhZ3d=5@JAUJG@pw$hVp-MDRIhhn zMy$c>yEd4j#|6k*Myis6n?HeX%6yl7WE`g(Iech)Gr zQu6$kJ(?Q)l46x`^y*<<>r)|Pn}#}a1%D|R;P(duPNqygUYsAbSV>8hKUp21gclsz zA1A=}*NKcSTx&w3#FeK7KAHMCk!pmfkh?ddyB7wPhH`Ta>#VAlSLt7ZwbBLmKaati z5r@RHBrarAH<+B5*Yms2K3#Z7{3pH)dVdqKrm{|VvU?@FWvb)zUrW^%Ieq;;dTJ^g zyXp#hNFz_Df=@0lsXHkUdW#=y){+V9#Bg~JGE3SHQX8hjb(2w{C0@PGy7;xjJgY0?7R#7b8l4Nq3*wXoxKAQlEp+|&o zEXtALOJCeeWXBF4(Ri2 zi9cRhC|K%Stn#ySv>*Lz;#2w_K2@Hd2JM?6XJ5-d-4rXO2

yR*yiwNQuIcwkShWM|ZyLnPoC|Zc=j6ulS-IMd$QZ{b)58wYyji3I;{=?0f^f1}`VPVWa zZvHwWTdE&6K5qMa(RCiwSa&bXjL@l5CONqSc|E2-BM|;_ zS7nd-`QmTT;9=M=`mJ5=ixFhRsGVldJxFy=!pjUDzjcstLcy48(jmfkHsv@|r>@I{ zJ-IK5R`?RFdYP3$X2_*b1+-BC!+y=XHtFawj2e8C=&R!cg^vw#IT$u7vl?|gZM9_e z=SQadMimS~rh7l)^y+vmy208HRX*!7!FcaZ1%FVi0PJK`;eVfzLHQIb1J_DdlrJg? z&tuY`+AO4I7~PLi`%e`Z+hJaz15gH8D1OG8t5KFwwN1+GT^JL?1B{sR ztFEvW`e_vX(*Ll7otP7t{z7$jG8$#=oRQ1qKWluC8~PXl`Gk8(K{ z?c1k|YDERb`k+w!L7sl71QcBl1uL>*%6F|>%H&*z#aw-reoafmW6K1>S7PZ>Qid<{ zn-*&IP?aZ3X5~=0%=1Luy03|=kvF!4_eT+u>YvHvmgE~Q#f|W4I5c)?J(MF0=SV4# zvMJsVQlH_1?_6I@t?v?(3q~^+@wM4OMI=Q#IKJX3Mt$w%i8k%g9DUtT0QFIeW zhvW<^?`67)_0KAKvBx`Z!i-+*dlJ)YKE<|&9z^@Rkz;B4jW^AEW#uMywvh#|V^J&o zP(D9F%xfi!RP}APbU1?9IZNBvK|m{CPnG=-9=bF~_@Z$A6`9vWqSR2sBO|7|q33+u zDm0@}fG#d>?uR-@#VyD@X@^a;P~y=y3)L$OEU4wKk1icK8nZo7%ErXaEx~o_GaJK3 zu|t>>dw1R}+)H%_$VU6(nK2|4_O`aRkdTn$Xf)==jeSua`vK)9LR9zrbpp}R(ZDcy z-b0+h`Pf1|b2qoW6ADo-N1lE-x=-5nQ9(hphEUBos9zFbp>g3tYkRvt%0QGxqlJWr z7Z_Co+sz25zKRBP!;Z+>^K1}zzlQ-^cs|Ip^t#LWby_%(AL7-7HYa>eofQAPIV$4R z9Tdu7cYyvJhWQ~zt$dS;)$`QFc1;JWyAoOZ7+>E4KR;to&&QGl+czU|{Odwpk)>>O9`Lf?L-kl4G<50dSr zGDQ>5le8u~Y=leFoexyu$r4GM-|U;KE);If-{VkSjZTnKnml8x$3aViOm-|X*J1Yc zC9>iKmn==NdpcK|RcV~--9FyF6^kuaFG&5qgBEzw#$<-2cO7DL{WHZ9*ryw(WTrxU zy6307G|{u!BFVcgbiai8B@sTKPVnClbj-JQYInf3(*A~S*F{#^nOSzK}65%@fF-b|ho!EeRWRgT_86$Pg;G``qy*Eg<_OE_m~Z@A9L ziG$32&%uF)DC6@1?9*n=f0 zP6=rCR3&Gt*K|9NguC9JVSW21D*XYz{>To@JDvV7UAE?fK3VaaIP~=2c(jd`82_J6 zAs*B3r)HuX6&Dxxk6!k>wl<&Hl-5@5SI1S@>r5iaWX*S9PWYX^8x+KtAQ8`W>{$P$ z^E-aN7vLBZ3yUNg9fXp))p1HH@nCEd-9*jA!k9}S=v=yT_El3eGhav93;EY--L-Ue z_XMfjGs%D5(ZMz2mV8a*-1;k#k!%UFva;Lm-FXZb=jLR3dU|w@?tYw^`33wvXK<@d z)(0*wu76rB-~Qb+_}WYYoQkDopo_ewRAL<(z8%8o)~%4^OkakF|9JxbOG4swdh0*g z*`fNPs;mr4Wod`gii`J5D2Qc(AOev_CMNeZgwBX?niAAA*wfR~Ax(fFg2iG{?2ME1 z@|Q1Pc64@*(h#Ckqg#ex!bfdZ{{H^|>Kt7URu|TVj#qnyZqPvG%f+bs!3Mw)lHvRK zM8?L(fMrKHgF!EXL9mXyyPTk)Aows~*0m}x1sfY1M@L6ihK)qQAYDBq0hEi1sKwWI z$qYxBJ1#dA&yR@J&)05^jlS?WaFf^BdpDoFWT|rB9IHQ#6Ga~(FH@EoDf%OptJ#F- z|6E|ou1vPht%P@PAj>9%b?geKPAl-15ftn^C(i26 zJur3sPTg1fIoR+JeYHHc23LGZ-&+YP?2~`TRwfd7C3n=f-=3l54o3DdY)!h&A240p z!I2+|LZygB_62fK)jS47+m%Xur43xfw>lG8FIlub-R-e#oh36oW(S`#arJ2;-(Z>b z%7a1+ucRi;aY4uOJ6qD1Tb{eh))?DhB3R`(W~#4vj6WT*y1u|HiWb=s^1R=0#i#0q zlhDNCu$AL?k-;Urx<+=V#(3Odl6U`#WAXN&p6JX4lBxXOy3*ra_fXCm?+y!BE%J3s zRYnjhz8quV(7N45s3e(v;}=!%Qj7`ODep2XUGk+`wPf0pZ!k*Vuc2!8Y|c{1Pz_(z z-=)D?pOW0}Hc+}-9%f=+#s}ltG?PD!qzq*C&M}5-lm|D7mHOVYe&9&k(@khl;Bg%8 zOnPp(uTs8xjDl^%emGMyd?A+gf_ybgrPs-r+C`bw*Ssl}VMT9f`YL>XTcr_$L zQ$sC>BGp*WvH=5_n3y7(ahS|D0B&1vZ%Yyy|BWiCsHl1V{EkL5cFX{L>&oNDkg`E9 z99YtbD}eQKPEhV2+DNV}`$cQD%qy47dkWP-gwKCN^+iG}iRy&HE1@ z^2^G}VNmQqct|lC&7!av6FZ1oNJ6avsWp;SL|MKuli0QWFaLmmcm;Q<3vkUB1vv$* zHsFBP)6?VPwRf)o)0r|N@kneFu&}MQRrzR4T6(&Vdd6vpnH3eu zhwn$!=AinL&ae}sRH9L+e0WFfxrY}IL^nbOf$W}{nf)x=7srIgMhsAYcr9^!GbjO0 z3mBbo$r!SrloVd@8j!x{3uDy!KOmj3wR}XMZE-(=x2m-C&FuJ+DBofQpFQ<#Z2nTM z&u~+Q{hjjL`6u9e-uGng|9JLt$e8%&w=dFV8R4R&Q zDK#4|Enev^7!^Z*jW8!+R_+{H%pxV;pam`JwmJ+On>E-CHLlRmwwiZU`SyRhGxgwI zjPHd_8>1p-UuU3)|87H%4V?KBsH~|vRp6f6zmlnX{lT%RHa^c(!xu^9%oc@^TN1vN zieAM^98vEb^Pa7;!KHWoO5424)Ch~pzEFdK7WGt3BCUSzw@`cjkc95vMFk* z^%gCPab~+VcrMjX#LiD2%I$khWch?W)NFoLYtNIz=WY}wb{1H0sH%~k8JM&s5a+8_ zJ&qMTSduysXsiWIjZu++v@4BhPUf4#uD$mCF$=w3 zdb;&Jq0gam<;lM5GbC3L{g2hpazyeEE4%B8d3vHRGV%u=`*d%IX5Y(7siFkd3#Zjg zj6M{X&d`}f=Y6CLUdS=eND1jGk*cV-A=?^5Knn>oa8H@_Ku~_Zc-mos7Q+R6kt?36MRJ zfNcIm}*dZAWKSn6I77$c(azx9@%D8xWVQEQmab<~k)Ec4kuQ3hrvR{)1uH-qn{R$Q#Z98$&YMeCgi>^@Ku8 z?B|Fby^R|ij?r!?mG}hM&PnrLcRE}(f42$!NV4{P?W;R=1#;i0ZMz;zzStG~xB9_1 z^M%iNU+3TQ5_|HJrf_4BQzTt-3B3=#H)iFND^C)6s$groy}D>Ut>rPAcCUcDbopre zO!#dbhs|1{nKIT?kxdhk4D3-2=OZ*CliMzwAExuBT@wz{38kM7a(MV=;X)$%Tb^S7 z!y0$3%W2ukLXA*}YLzpv8}Lqa9uq5Kh2IZP6g|jnD3h_Wm9`(Er)0}=@?zQz&#xLW z_QvDR4Qh{95dM-jnAyA(H1RpvB1KL06-}WvzS(g;M9v@-uxUYRHH!ABm_;0w> zr(9Qc#fwE#qo+1Co2(>VEuXy=LwYt(dhQ=2=Jx=d#FMXXo_yAL zibA5!Q9l%ve~02V1cDwzMf3jsE;4#yanb7LiM_xcKoCI8OrjBj5Tsb|Z=OW0Rgef7 z;SabBwf+fA+7Un#!$#YdRVs15x2F&*bXYQej#YHFh?r*84{E2qn!J;Ez^zwS1Be=PfvvfBsvgRt`sDehIP zyDJ@QwuF@pESJxOa17f(sh;z-aQlbtFHrq=B0AJ-F`331n;7|vK zuvZj(=au^DrxGUzEo~-D>F+XL(9fS|p-t-?s)*;w&a4VuOf9>=uX5(E6pyc>VG3WG zlf1v1T>7J%*k>;EBgKe`e(HIjs=yD(xwB;n!(@?3)CgX^ZCkR9y*tTgsImx`5r7gk##y-03DN2 z5_AxecAgMM4Q3#tKX!EiCLsTaf~3k!VtRTS@-bpmEj3~-Pmx<&TLCDPgf}a=E@j)T z_VD6M_wY4$9-(qHMj%b#noniJ@u*SG>mw~*aKPT6_u0njrEk0>Uh7HQGs`a@SbZx)rW zs(rsRAcfq;TorA1;R5ZTFY5tDunqqyc4ZuM!Sb$XK~o)%uoBa(kKfLvT)ijg_D(Ht z;BQH~(z2pbr8#$e&gNko8$T)!LD7Rn$@MFv8+k$SbG>iv)lH!5RxtFbZ5!EZ*|QxT zAG?}a#_4&A)8o>g=p~p412cmFmh8mcDvV$l<-f;Fua0R>nsVrP`p3a~pWIRy+mgED zp`ADAy6`<~o-#31I=q>WR?e{}YCHHPjC}IJ*x+k&GU_!GTC90j(q&(LHc9C2<`EBi z?#p^w?eysBlPTBdT`OG{w=jECQm48aTKKo~s0GG5%^th!5lUTt@cfbBr&1M0*fZvq zV1jQ1gR%<4HhxCdO}@Y+0OmP0tIs4mLDaEK?KQ0S_o*?;e{Q-bVaq&?dd>g&VzA#v!-hyd`xK%|!>cOJzKwXuYh)VZTE zfFJE$Y-ib^OWxjIuL0>XQhdNwX#PQhlGxmzWGaXBrNM=w@+C3gDxog9R&pE3gY>fN zp$X7+ke!iP=F=)Dv_gu6~?SeBR6Q#oaNGcc9vdT9UFq?&ivZw)RyuxCh@ ze~wFX4c-%>fg0|#W2D!&E#1139Q}Y?ogZ2&wJFGk!DL^M4eHeco~?Y2tb*aajhAZ< zCVb`CH%HeRXYlPfzbW4$xqu~2Z=dPYYnex4I#bib)aU~?N)|LTzsGLIlEpnWKcgFj(-x{x>hvl?L;h&lVk-oE{VA3K63Po~=;1 zQ!8bHYuh*f#dF`V%*?XBmkFQtCl~&L*pyR6-*(L^mien=PsI4^PLbZqupgkyZ_4*m zx!j?cn3AV)#PE@3^rn*LtFpW{}&qE4U64HVqApi`iE(8JA>+I<7!bnUUe=p{P}; zm+_?4@;H=@;0t5aA(x9$*Is*Dlp4}V1_!$yHzCwR;Ws}u1i2_<4V+|LgETW5Ww0G_ zkgC?!!rHqcTZTGC!Q+1Xs1Dh-zNjET7SaP~U9Y!)*1-A@YCAgJ9$Lx>H2wJt^+NZY zfioFE?v8sKhIlSS#U&)b%6oxV`l7UoOb`>g*^wGTj5QIDfe^F26AHPz*AD_-xwqu# zo8H^%?XtC(ule%Uqg3&U_tNyoXNb@?t-U(w(t;fN_y$Muc#-WcRnZr_zon)M@(R)G z?i9_d(!UduP7!Ba5*$Qi4%f_Hr8JV?u<`3j_Bg(`?(Qk_-7M!j+pmj4g~I_3{{;OG zgx_zXjVldF%{QdH!>p?u4i*>JS(&-X94Jt-+y-e=CkJ#h%NyLCg$sKbc|I;s^k@Bq zU6uy$u1$8aU`S4!NoKK&{JX5;xY!~mg8#51 znn`m{oX59wE3Pzk=RnEXe>l*zY3Ya{m$0Hz)v2%@Qur}!@8|nlFV&tIPHJm@48V$na%YANC&X`N}5=18EJDvpE{J0Efb1z6rDYHqEwJZ~5{w+OS zngSth=#f)lvNMO>bS%u8H+MTN@6-tRr1SZ8uvq^;*N?q+|9!F#{y_+;a9BzRF~@95 zGMF35=@O9Fh|o|3e;EZ;z*?+6%??#PG%}GYN;Fn5wkcnZ>W)KNB8a(>01ukZArce& z`izl|=B~3)iU0tC8OY%zlF7mms+FDHq5A^mvnPw_nw}aRb6Rltl`WNQeiS32UKt ztus+R3(Z7G4dHMXQbs5z88L{5m6s>^@7x!d7Iv*ayEFpeXAin7xJB!s#9U<<- zxU_boU@n_dk?bd2Vf&Bzv7Kw;FTLfjBeKEJR3g>lDO+Z7R`MO2qtRz}eYJXCdR0Kz z!ow)!9|cu+S^v%N|CK{)Owp%sUxDj)UcQRD?StcP~}%(Y2rFTRo^cBlG={ zJi1m#@PbsA`as8KafQlw561p%v~1TUnXRD?DXl?PUsI0to5>cuKndUmghO_QYJWG; zrML4Wj~(HW{I>ht)`ox8P8{qa4qJv6B-`o4^E7c3nOR+ExGf+1b+?d^(BUilubo<2=&TBq*5kB1Xc7IJ1&dB5le-+?&$$ zlFUp@5@e5*4Vkq%-VI87?Nr3EgXhM)gka5I{qf8nqxmn~)QL}uGb+2ZbrbusF|)2V zU(NUGmo-B6{JYUuMDuZauvy?A#g~B>IKbdKD4HSIlhJUVr#s`ee?o!o!h?anU=kkM zb<>w->6f5pTN9Fu($4x|c^{F$5-Pnvmxb||S`%?#%a1tp z;|63L9IcURA-8rOTo)qwthUOovoIP$%>{Umz-sYURpFZPFUGOCi>sc(G!eNrtFoI? zf2*bY{oX#__)SvmbOWucR{O~ZhC1zXx#oa=v3wnp*XB3MkrYh&# zC9NOZf3#x1DqA)35i3cfHH7DkOb*5xmgO{&=0*!zlJFjrG``giCoCDQ=(S{tTY3>2 zyGLn-9zK?nIuxL^^l-v^b+K};f5+-muWvqlalH%Ky&rp*ntOM#ttxp95xg2EmZ=l- zWI`paS6*yoZgu6M_evk$aA~ZnkI8!>U^R1f(0gfOIah48aCN+Dez8(mVEy66&!^-y z-D#0J|M(*o4j6^0qd9QuKGSpyH=1LY@P`<)-m9~otBbv}b$blwbZ30{mR_o|(VJ|S zhf`N7y{EmGT2?ZZRyZh=Eh~+ygM+>EElawq-zFAw0)IXXrTEU)iV?#r=U7$+SKDVR z$8=Xyl_sChX|3M%o>iKgUCOo#{(1A`f8A5XLmU|#tQ(gdniB%; zkKfPLrlpbLJ8bN;YtZfXNuw9 zaQjMG9E^UdVVew zr8ousx@wmvR<-nwH21B~XZ@Dkm9M!A-;1lpv{nYR^eG2_m`y;S@-9fUG_0yLiP&Rl zs%1G1_L({{)uJ@JuZq6UYnI?m;ahIyLl1Ree@RE*DGs6&surmVHdFL~EI*?;}sf5ET@1mwqm!SG)&{QnPz zr9G>IO5IiS(p46OPV;(n!}!9RkP1;rQ`|FMqGCKZr4U!X_gQ>?)A*p2B#CQc3ni!L zq@**trZBiVf!pvplXJW=g=Cy(t$_14&xQmWS8os-Z(KiK)*RAvvYvQS(v~>#y?3Rj zsexYLFW&A}z%|iMq+grZGSb52gT>`>r7jJpF6XnPde%$K@p&!c#V89_lBVJvSfX_- zev7GpkJF71UCAv(Yf4sK934k!4(Spd3h8pfa+;EkQ^#_oC2{rRgFPk5czw$ArVxU4 zNL`XKC6Sxb6wJb>2-A>UPBKxP@=3SJP-tj5Q<7*>mA2v?GdTa5n-zM+@%CNj{yl|F zq&VX}wB}DAPfFGs^|WW1<3yR2!ol%>^v8J zR+lKNHpN&b&YJNAZoBqGZfDPCrjZXO9z6?X)oul#5F5hsw^dJi)iTX{ylCXZf?0(e zXvG=_lh?Tv6Uk^$(V(gw>sWEw=RLyfu0^<%r@#U*pBJ)gUyhpvIh zCgnH6=}tszd))E%vQPr7PYUUoX*9u&I>mDvf)xu#oN?vx_WSfHdLyI`tfcAeD%xlfm8wo@r(8`^Uiq8cL8fB~-+N7rEBvVaMLMcy4%4;Oui5{D1B`q^F7G!qg6X{%jO)9N$;gu4hpX1|&Ak zNz$aQS;H>Gohbn<4dZ&bVAI?!*5d1zL&RZUv?9c;vG{sXqIG>@;T9s7h@`>J_QKH| z$%gTd6{HXV6_%;p=Ko-5YaHx*q=;yXXAZ9Z)6ODmhA6I!&u=$~m6oiZGb3=^mirV!~L#&$BDm5}81aT+U2#&?+e*X2>5g#a{#3e!}+9c&JO=&J!F!74`n*wj%d zo5dt+5Pv``;+o*Js5c>m5`sZ7q-(&*(T`ZL5n;>#2a4~)ZW(EIn>DU411}J4oQJGB zl4MHB0*Eoy7!i7Mu*H&FgR9Gq)n{=R_n1>maNYGCpen2#87lIRPW!ht>m#-pABN2Z4T4j_{um+3xsDR> zBew3@UFZfay(S4s#ZD|Dk1gm8@g<3pA`~a_5aW6y5GdZbe6KqUH}Q{0HV;%Y$AvVQ z5RxTDU}X;nY%I)TL+ELLPU3zLPjA#S9ToKf-cd!!U^v9_!gWxg6l~YTU)+}1WCRY} z3F#*why{O@M9Un@0sbK-M*Pp35QugH!hh_(WK49++5|Q@eTwW*M@2URbi}-p)X@d7 zN>5piQzG}Xo|AY|UK2tX>pW_*q;rK{NU9XA*(rP7=K?DbJ@|w3>~WGCa(RZegE#`u zVICSH9}_?fapK^#q-uagIFIK*sY62{BYLX>u!_mL1&ZV8wpa?l71RtnhbilIouT)$ zr@rbn^n^M9qdBnY@v=2@7Qz%Hqp^hq*kXq{F3%a1jc4YhMfVhfmqljqT(iKs@&W2w ziLqG2cmyFeIjP_+5oUoOy~er5@bbRRdm z0RaGL;{N~~0=TEGH>qUERDr^)()F8=IJNldc1VcznnJ0pO~owb#r0 zPoF;iqs@Q*|Fivnzno0{i90YZ@<&^IKyG8}WsSdu z#b-O~m!95uM%dtCUO(@bf51_An86c|6dwNY2W-(?f(V-w|?}efY=s|EbB_)*nv?0I0gZ4DfWc zckt%EkLQvtx0ai=s4%z09SLy&_=D&F==5mjPhJu*UHz-i)kG0 zydH7NY~5E2nzAtAj= zMoxjRP)h?45)cs)5))my^2cB#2*Te7h-t6T-4MS|La%2@%HzQx@jB__Ro(}c?Tq@P zC_YIm&rmXQCg$rbto%0xZrv7?l9rLZD<`k|P)%J!Q|r+)14APao;+(CTRVFPM<*|D zA74NJfLCE}!XqN1qGOU%-le9czt70b%P%PWR8(B@xvILRwywV6Yhy=eS9ecuU;n_^ z_{8MY^vvuW0=c@j{&QnycV|(DO*VzRK_*>0@O(8Ly;1iqXn*l$?oA3c-*5gS0;> z`>zoe`hSVCe<$oOy5<2@fbh>mL`X<5#R?C5iOtsU{&*Cgm$SD$#3O_ zkPH3NT@k?2kN3csAlKVyAkWI+k-xMzO|_}39{4C^M6pn!A1n(7x-IFlsoz&tLzkZD zOt)UwDqZ$>Hx27%eG-N4^`ECoT`x(}3u&l*zaJOB%FCCa^^FCdTG}&fBh*VJKa~P0 z)v!DFGD|xh5BjxmT8tW&DU>zJRw$?#bK3H&!vQGjWQV+%v;6iyQThX||4Kd{384h7 z_ICPmXheH^;eeDRP||ccgb)WL@1Qm=dvv&PfHqG_&QJqH^RlOzidPxNVm;SF;|-TZ zabd$&&&Yqh9&3GK1~cIZjk#5FT~Cbx_&fi<&I1D6kAEMbTM)Sw96*Puvk&X6qke^c z-Q)-m4p$Zr(nt(%=lbx6Fpjhnr%U#_UL8vQqQ)U-{Tf`rbxn!|%Pf;E5IT2DL@Mjy zun-FGI+*`FU}Rq$bDNuGL+@I|8mzs zR9%kpWSNgE3;o_%YZx<@kG5;yl~qrEL9O-f=-~TRy6Onqi;0-Rcc?%x^Nu_?-ly-^ z!S(0A|2TkK>`2v)(A ztfJGd6NIOpR@3to%X!d?y*g9qZA0lU5_PxLwnewHVR$Vf+bzhOQT5oHnbH$qTGn{_ zn(s6VTNuwW+67!g4jJ9~N-8}n{McI52vYNM>|H?8Zy~kd$ZhbKgmAwgRIfRTy1PZv z9OH++d9w3Fm(IdOyO2M1F|Y4CWm-L5x_$b9&uhQi0VLyyFMEpu$kz6xA$YUE!ESBk2RFigiu z^WgQHkm$1sF*}vdPo_0~iZGr+_teg*w^gcpLu%mBa{@8^Xiy})R$xq&^+3JpM@9Jo zqxm?~v~8y5bMw6~Z9|Rss8k+=wep)_xyM?0Fuev7?ucgjoO+LZ z_-x+S2DP`EO+TuBd*t2=<4zmC->+z3V9*Bdx)Gj|E_B(tiW-i&_>BVX)}p9;sZe8Q zM~H|l#CKIQ^hmU~lCG}y`QW*2qUQMV{k7;fqbxMK4pl-G3rUMwr0ed4&~`aRp;a9& z#ASXtAUlZVFq2=PISV^WfP%#llXcN@@Hdm7NUFF-Ii=d@&MTRry z3uwEF&?bE5oEjsrACANUv+!@1+)WKtD4o-Y3W({FEL!sS^lOn{W;!+_64AGiXewEe37&HiAWy^TFaq1Ma_M+<3I_b2E6MTjU0pYxf zsYFtT<%oY7b;Aoly)SPqOC54%GM%>8{AX(Fg%x(N9O_og$D*y}>xvo(9Dva*Q@p|drc2AEJ;ORkV~AFG#%LkO zguB$*^LBC+a%FN~&hln~kfw|)VGNzWo?c zILPyy&yRV%&}8Q7Zu#@UHv5A|4CDo&;Hi?OX?{y5ck82TF&DPz)7_$T6^Oz`LaVTU zZ->RLI!r1uksK2-@|bnVH(lO&>xln*4YN!hX{P4-V+F-N8(3X8CQyXH5?c z3lKIPEzT><*mQ8D%87&n4mdMm3=nxW{Y~t*XxlaDuFWV8_*ErXvfCfomAGs64SGnO zxG6M7`sfHsfCKuJRdV5)971dBUyqY!gudYbM1T|i2am9}I3OfV1)F8@8!WK3jHwHB<1ehX)iZZQz&dc zVJ=WVkb3dPiv_sx;w4y$;MkDy_jD;F04)}d-=1L1FB-C*jisB_pb{f@1#;YX*BYZG zeLP2RdP-!~^X$8%l4iDu?Rl&&s(e!xn}kc5%b%N|x~DSVJ~!Xsd7RH0 zAljD~LUNG>kHQ$4jZsjnunwtK-Iy}3u?5%xUNk8A-TlPjN=M;L%DU_J;DGf(Agw_w zNAMVo$!GSGeJ5A~b(!RffJK2MXYQSN4!iR-L}x!tNPI`zBuW_C%fcT?7^1wB*Klyf zSy^~$SIkY(5`jj{zZbKQxT-c_9xzq;mIvnj;>r<8uxej#OMrc}2v(yI>y6SGYUORo z>@;@OSJ_>+qgdq@zOn!VG>^l78*9yva!!2u>IokEC8bZ=($6g{X}G0@FCXD&rJ&o>zLH^x!OTvcWk}wo$ub%HAhD zOr7zK+0)WhcZx(UOy{Q^`N)u3ztQ9Qa?)MjbP%1EZLKIj?Y-T~vM#15cOdPtG6`e4 zTrTy~yoR=jQxIsGT2U-8P;%kc+2K1^eK^jh_f#5JQ=hdnmgR_naxmUaCc}Y2`1pZ`POjC^i zMkBKaoX^Eze&PURdiSD$0Sej&!1#B+g1u3+ zM^_!3q@z#6E7r#kAWHdWsuCt-{2H2t(R1T{{4&utt~tR+l>xIkw}^I~grZOf(LS?^ zWXv(mK6x-|Cvh`vkAN4|g`**A4J(@OjrbX4IV8jS;7S;KykHS~DOf++Y=#sOdZbI` zA2YPn$(S;DGS8PvT!*wvt$Sjl8gV!(Q_0TQ4{#o-K{QZlN8QRylg%{Cj8g8jGuz7C z)UF(R8&PiVA~3x>TD)q?AM72FAVV+eff>PqH!s9Hoh>BM8gKUYeJ=Ff=@O%+SkGeL z!}ee^@t?GUdEyP92t%SpG7QNte+_tL&q@)T38_Jj(X7;TjxdH`E~imU!m*3Al4Dt0 zwUs4RQOZTOkI?Ur%8s*z@y5(np8@jR^+Xo5{&?>=R&DCoEQr)%607vdk?AJmxX*b! z6PGyO*W|6F8>x!FW29aX%8OD^zoVh9xCPIgM!Lsf4U0D|m@(oUvE3?>l*D6=0qwRl0JE{v2ZNx^=Mjon8=S{h280}PMAQ;a;3U-d!{6>(&t-w+L07K?$QOG zKQ(VA$i=XT6L{YuugHPu5e>&9se9c=I z1EQ=97<;1FFOwWqbVZ9&ak-Q8CVC{I*-|v6n8medTv~*)=W=Mfo^`Tz;#FS?T{r)n z@Q;21^0nJn4eko6PXSzZqZah&2gm~x%t41NqIo||M#|RC>g)`Skzu1s6|Yc!2y#?`xeqHBZ_ zGKaIknQ08hj>8(6-gQ8Yl|fUrW_;ec^W1?s1#TGG0?*8R}cNZJpAO(7pd2fyuHt|_tU>9#-uQS^FgxEEvsH`N{<-e zSEkUq?C7x$FS0kXhKmZ5Wi-{S?p8PfQ4^llmwqF!UDo3#)#Y;FV~Osy!l}SGOtohenB_%lwdseAzepR=W=pe47456`@<1 zur8J8%-+yua0Ph29bwM)D9L*4Kwvz0G%#r6_w?hF5w1^)qo&LHUFY^ z9Kh_z5V=XAmmn1gNaN?Q|AkU5JeoI8`hPnyT`(qlaEwbd@^Ey zIw|oLUVP*0due!)cdzo>d;Kl;xKE&kU9Gj-?uus z(sDxC=H=L3lE!Rb4vl1QKUvK37bpf!J`1g@kT@@+Vw{3+p$?+TYu(A6X&kDye~f(I zNtZv)t%`aVFg={C7LV{$?4n4%+Ifek83(Muqp(Ix(x}JsKc>AcD;z<(0XM4ylosp0 z`X}X=&UW9hbTc&kIcG$2cnKB`c}Z|;%Xqla6(oi^!P^1sX;usb z$$=o2Jbz#J*c@JTYd%uP&!}35!Z=-Pe6MEl0kT8Epe~g8C}zHq%r>b&B*QPH=^kJg zqMVV}YJdsHD=E>YHaV=WraI0&OC007{9*d$Bs=Fqdda`!Yy4c^ey4aWbNdpWo2{W7sDU9t#U0bwzXJ5iH!x6M{$`u4nSUsW82}i zTD);e3kT%J{MJ4hoPapmqmDY$EC?p|Ys|qQG=KOTFVp8*F5ge0em;#uzx8FB(^PzU z7a5}5nAaw(EFL_{DS_U2(@Y1UC^2`amv=F$zvOq;DyT~qDsfW_ovn;3r5w2F#rD!b z$hz9`~)vj zgUF%XPYTguonTh*?7bYW)-mtW(!hW+{hvrzlQApCf}4CYC`P$1PmZoYl(YZ%NgAfo z4hPUSYplSoZOJM#Vgh{UY&$K!ODqj~$MFWxKD%nIp;#jL=$Dm0z~!q4Ccxl;lUj5_ zB3$M*LPwr5V%AM5`^^z~m-qD#^^aWHqdlDs=j%0jO67L3HEi!;&gGVjoCW6_p3}|qicf@Rv zy;G%G!w!4BK7XU8(r5vtQa9j%0~m0CGc-(v%tycnSxQ1dBK&Ik1Xg&n(JcdQg2TydJ8RSm08Iu6`wY(fdUjvx+Kuh`Rjv z3A2F|h}&(eN1`fn(#2=Bg^TND>+8kmA0E{_POfnnV}4mt7zNakk%+~fapBzoiKPBb z?Y^#g_mM%Y%g>x2rl~X4J4)3Z{-zR=%9B7H#8p@RZg3&E$2Drph+M9QC-`thb6ruq zy~<%;zieSz1G@IAU!*M2?U2jQVUWwqV*DN#ZuFl@Ph+&^{8XvMd%(4szD^7hPrio1{z!V~H7vn6Q8$O}tcdNj5J7W=9kkUzWT7H) z8w7P+so&%}O9sB2a8C@h@?^HMLf|O-1(X=Wk3kRF?(vO6A#)*f=|JhAQ$G;s;N#g~ zzGF|H`NHF(J=uDR??TM>B5CtnL+e=-*Pjw7T2pj@9@$oka5_ z%aKwy*l^zOC219{(26b5ncv zxgl(m%C}~m$y`M~yk&FN+P6*K_V_9M@G52D4Pr@6l7&>A-UHCL<9d~-B!c2tg@~0Ff_|c#l|r+9RdX0)gYZ|)Tm21NZpK>Gp*sfqE6f{cYT1!A6V~zYw>(|Q4!FXKTJ8)2A%+uC zO$bGK<`C=^(R^?3j9L>h(F)PyTLtl-v}a!|vZ-xoG7Fxk#$@M)G=ZQf1EFjY2$}l| zjL4#wD^xoAm?X;#&2ZK|Tb#lZ6zwJe!ggZE8qH7cd*W=7UK5FmDrMn z3tkj_uOF3#fF@#@k+3ivKqz%Uxz)5nsh*)MX{9PFjqqc_@Rz|>etgsW<2rj=79s3}Ds&3M@&RZk)l zZxy@uknfI0%a@n}_S2zmR@Ip=%4#4ul>~X-GXQo-kms98XRjwwW zXXg}bcc?xTRCEwfp55k`qimM6JTz3}-xL%IcwA3Ef1*Pc<4W8TW&BJ_$T-mB>rAFqP&SlLov}O2TQU8Q? zq?jAKT#omiz)tc6kI%)=RnqH~_!lp~2SY~nQK*)^3+Us*=7w8?i_lZRK2EC1Vq%bpSvT3VhUS?;zIe$YA zkK1%0SAOu_i{@q#fe@?@f;yDzs&Fv-GWKz@E=JQwWAf%?g(%{~*BoiOPNsH&_uJ}^ z1e^BmbxEA{CgM?jR%7!m4QXTTdyj7^2knzg zy$zwbW_+ox_w3j-DE4ge$;P%uOxsHSEA@O-dN&ufA}?x5=NgoG>txQgOTzh?`DP>J zr9j%qr!6>*nKcjnVb1_#hujg%lYROwSbY(ePlYa62J!&^RN3&40@ctgh6AoH&$iW^ z;sAT-$MT{%#gVE+mHmUmliq2zrs<7ThACDG8-bXt?cgd*^(8qNA!6NS!G|eCB!-oI zggl-k>~hOy^CufS@Xkbx zpeSrbg!J3gDc3dkV-elcyfuQFX~;H8M-)VT{Bfx@^Jz=_hMb zdy4ib0lajD>hhI^U6C@Dx|h7fGX5-_n`ykVOZ+#9aXmnq@#Tx$-3- z#G}%Bk<#5Q3#~V%yjiS5wM|s2_TPvv+?QR}vP$BMZ$i{Xui7ozNda^J07(Is^@${aaqtu4#G zIfoqA4%2HkSsAzds-H|@Lt8Ok8xz{y(EF0g-4R;d8(Nc1`Yo8Vu4RIC9CAJmX)r zy}y0N7RqA0l$&5Y?VwLnyuzNeW4_2huxi185i7(v8`Vs?OJA6Q8^+4{D#x$oOX)-H z=LZ<>Mh?bEjNLJO#@Dn^Or+$sV1`jAMroncPVxhbJ5F@1`PU~(`PYg!^$G%H!X2;5 z4KLW&`qVrndGb2n6yx_`yjjI&_~gvCMy68g4fAbsV*M_FjN2=OB#^wtkN&x-4j~Qe zLtB=uWO1WgxrD7w=(gQ91T{0>Y=KtKhVSGs!MZdnw|{J&hAvSY>(xv0^2Q~#_YhpE zk{Hq*t9eQ~+{%6N{*r4ujgzeob^Nmk!3HOvS4vI&8Sirv_|aALaZ+;}yg^*JYt&&>uM)_an>5Z9pnioP70|?! zmncOGu2JmW`nkl}8ZaF-!-F1UNKXsM?{MMw-8<_e1ArS$r9^fYu@X>2<=j~wR2=K- z*Ja;L|0RuM=hblH`p3hUHwLKRh0ctOfbZ$u<5Ko-I&neC#vOBA-+onilF!g8ldhfl zOI28~vhMB27c~3fF%d)?3uY>vb&X~tVe(gcyZSE$O`I}wGCnWOW(hU8iZUsf70}jr zG#5Xu^zS5$yKb5KM1EtsIQN=%2PaJ(D*cTMmUH?vbAc&IQOBcX@FXiEbvF74OysvT zcNcMwqcjTT-mXL4?q>wEovIDe&>81Rqf&X&{MAH682R4yHtT1h27}-9!vW-QikOo} zw}?8ivMApBsQv1^ZL0F^x$MXYzieKe<{UW@7n2{|UuAl#`t)UNb!9YW$a<{=(pM5< zJt{PU{3@KLSrtt~CZDM#`_Jn#hw4g$Il&+FY%Y*pwuha7?}}B+rx;)zqKcrB?P^Sq zJU07%GXYGs%y#a4thdQ7R0vlq$Q5zjlG zlt_IR!`CY%tHE3l8q{+5u>)4M7z5R;pV)m+UNA+t_T^iW_JDm-eFI1^?!Jw_fz0^p z1B0ieq^t1AGxvo2al5pW9nDE8voJ5qV&f{je=;sYcK#<{Qk7<$;pk}U>#es}yT>nS zHyhnUeGW+4KRYSgN5^+%CKTJM7LP~|M6IO^9b#);LwLgD1aA@C>v5jMj}d*mI3OeG zFWpR^zp${$|R`*4-SsfWaE2wKVY5`f&?dW9>(C;ORRnhxk${g30O?zRbU&B7?h)AUD%8EYiG64B&gf_$GV)HT;O z7B+LKna~>T{t&HI7{>UHf7A2PTK-xlJ&}~z_m+`2K0K^vs%DV5bEVIvj>I3J2@d?n{(^2zQKzHp@I^=hF!US3I8Zgvn{?;iU>grd+# zf0a%nPTC)?Z&)p|uZG@=7CXGw6zkcYpiSS;mjYHYzZ`z6_{I47+~y_ESUES7Op!t8 z>R#Fj8BCBg`F7p&0EPDt_G31${*EQ=-F|h8h`zIxeN<5(zdhIAcU0I#r2bvBgu0E@ zepe-bMnuUN?Y{UB z`clIXzg-)BMbY1%c00cQY1+rZo5KmU?8GXlLtELH@m^tnnf`G!FJ(XLqt6Q{39kiA z7@{alQOh1lVYa7anbL*6&v?=O_0Ga05sfp2fY?x0&f-AU1N=tKHl|FzX>_#X*LrKx zhl^=(r!mlxUZVjtZ%ww2$@2IgG{6oeoKVM|&N4 zSN{CCc8+1i%^}Z%N_n}bnl_4Yq14h7{?y;*R63WbD_-aKEA<6@YQ)F3dhmCD|2~*; zGt-{nzMqv=$Lks3Zp8q8IcjD`aL9hkVXZ;cCGl08nq)kZS~peQcr;e`=yL)~AG>$& zRD%jFmbH4a`F@FiV&^6Ql;UlV3SsAxo9R+Q9JA5B!r^uiV9}=`Vi_#z+>~n}&a)1= z)bx!Q7R0&t7+ZU0T_J)Zn_1Ufl%;0Y`lhzDE8?bh)X(@nt0DG^i=ij`IxZThunsij z&O|Un!8CNXu|`(U+?<6GZkK$Vv5~!IBWUaE+4Zi+p9mm7esqf(QUcrT=?u@5K>a<3?R_J;WZnBntswAkh(Vu1ttP?uW7_#?R zAfe7D|3!X_=$mxnY!M=%vdaa~ku9QpmpvSC}=lp38X>1OYgqb_Q)&Tz-rqBz96)cRrl{@v1BV2$UI|>0e-u8gR4ND2TT?_)w*RnnL+7M zaFBlEmzt&o1<6;}#XqQzDw|Y>UavZOOF*s>4Y9_HoPNjyM*mh73WwWQiehbIRDC#` zMCv}+Jg4^vD2j^JmU!#CXO-3JDcOEOiHv)5Sjc&_!&FYE89+N!p!h`NWtCth$2a)B zybzV_y9MZj-Mv*M=gu5~(Hlu{SKIMg3W{I8wJkO8!czU`WOzMGdxhPua=$ocm|aGa z?y;;a3XmZR52ioBGz0yEfhmozYC=b@4=WTgZT8*N+9|qYrY}vffZjWUP*!(R{WsKrN)i@(@6MV3JCsh1q~m=L%@gT z!FadGOn*KO*s?KPtKH!uFmvx!iPQ=8Kq_&s_^50p8I0IHSLTCSa}wczw-D%ki?{Ni zB!6audCTkP!bPtyMlJ0Z*!-t1zq~*WB{2 z9ob8~F4?VLss7)+r}sj-xD2-Ee&&;f1Mr9;i!brU+ai-TG`^Quf^s#RKTN*k|JIYn zN&SWVcN~xaMH?7XISC1zn_Xt(O+39Sn9nM;ZCg|BSiGVdllRS8&qU=f1oZv|FD4?> znErq1Bk^DR{=zz8r28*@K>zAmEo3UM3i?+c`Tx?jFv;Ka{r&v#GvV*?`S<+$dwuz9 zjsAQ6{Bvvidw=+QfB#cb`g?!)|NQ$CN>D19&GxZ@jV2z9G2dLm$>d(u6 zu?qALdhPIj(ZhRomG4ZdtmNVYFl6FO7B|n*6XHO=l3K+57wIK3vpiN65_MhdH0t+x z$6CxfVKvL`VoYtJy!hC-sQ7V+`uWQr&>#Ebd(hd6ey>vmSZL73@_aF?@^HFOipqUd_VLgp4?QLViW5}zqd}M2qvNxvf4wcwBj}M>>K3^eP=lYie z^4K|%kx0IY8`D2sZN6e3@w{Z$d9)M_?^dZL?ZF$PAf%Z7)gUggrVoq?q6O2aOm%VT8Kl`( z{B)mWmW-W+T`KZ?a(jszJ-z3O;zgcz#E!&RNPj_tBZF`5&}>dOs?}vt6_}cL&*!Os zc`PXvs&eNoKAJ&hjOEFFfGY^|3<(dx+vS%T{n;%kcIEgBLU3J{& z^ZZOFlc|!<-E@at^e1@g;LRl$8V5u`sEQoh7!;pE%#lH@wtJ_uE48sh5WBn9^)dX1 z<9TW3d;NF&41pa*ZKMcS2PZq;IEfTWta3EvFF2j{aKF3NV}BqF*@ZNqeAy`MRQ^U;RQ=Y;04&RfzO%Qf}bti*2cIivK* z1$Snfk|GB>3<(Zzvj@+1uE5(%V}m5Sm+25W43naHW>LS+wPi)5O6k`v9xH=9s}R?% z`89zHKe!^s5)JwWWlfccNL%hN6Lsor2&P*=^ zeZYu$r^MAj^E7%McbiiH;dn zj~?(fMq*{AFSKC4^>DzxlM14)ZTm3}7{JGMc!6LT-j$Kg3-GsKI<#mLXPZwT(z&Eu zbDUU=;fjEN>Vl$LDq*p(XCTa}3(eIoLebf1-<1J-+R)g5a;c+m7|^p$5+hIMH!nyt zbb*w$-4VEsY-7&}>V6dGeC{(aE%ReOX<~Zu(qMTd*hhF;* z3udk2b4wr370et=+u~ho17p*_j7d4in`wTHek;<^LuUBy-HS&bNGWPguT$P0AGup) zkZ&rT5?j~&WeoKUbn1O-`g`-PT>v)svtXuzccO8L(AMn58D}&Opu)!(>)T%%K+iK1 zF|P2l6qQ4~@pNIbeW$HDKD970RCQD>DF>f&Ew#=ztxj=ZWf`ZVC0~SyM0%0V9yrWHUqg_A0h~#sfPj}7l4?Fy} z&e)3c22?btKhl#Kpb5ekc#^aF1MKM-BKl z^?EvGhB`GC_GEuD-pVm)unpDXS-y#q6?YgWT*bWF?^t|1AL}@*S8A@2S~LH&svdt_ zbLp3UJ;<3+u~X$uAp5wsEEjwWxB%jh>D7eU-QLuNZtT<{$}(( z|6?~k%~KkXLh8K@j{ERQcsT6vG2{jgcoC$61GL6kt2--rhS0_f>TGvD7xtwM$h4f@ zE%aJm!LG7_=}jX=H~@|?+neNIr!@g@6u@+w)FI<43XD@zi@qo@ndx^@Tel5 zW0v(%?pR)xIj-4D-Gj^db@wAnt>BHfm>z7<>IeI|29N49u`tRJEg3CCx{}TjQQv94Ue3=9`IdWva_jE>s;>*9Xyt z!9Vw^*6((w!R~X|i+R2Tr6x!*8YU?>;`U*BZ1Q)}IUBV9$)w6Iwe00Pv*|6{U^rsf zE4p_*mNqtBYJYM!Z7G`fz+XY4Jf>Z6ay`KOm-ZpvtI)c8_BJo(7Vvlezs&<8+~hwc zHhF3?2aRq{!O>=^ssI6_min4&9*04BYKeY|9!=86@dsThw8ogX{sM^bRezxeqkDFe zdrgT0+L|b%-!1@uU6gNX*q(?yQfTl#kM{dEQmiT^p+z;(PpDoZ?qEwP6LEVzjX3+@D8BhY zz-sshb-ovcS^E`FB!a?Uam7I_KY}+ip>Gr!ixwO+8Q)czH!kULbWfam;?L{xC)|Da zx#x57M}UZLE!p||=iIXBtR`gynlwrZN(S+*$M6>UD`;eO7)_sKKCC;Hhe#|f3ZL@^ zMm7lw9L*2e#ZOL%4ZiNZ>lr+p!!!YTR$%O48a}tB=bvIW>eK(zAMMEWZco11^$TEA zr9b=nC?|wa*S7ZD{Ez%LX@Iile_I#D&c`H`98Gx3&wRV4nC{vsmcw~$!{9jUIOy;5ZUDV?v>F1mZl-BqIZidw6i>1Sevu0H=-qusDjcBCK}#{2MdZXGF*x$*)7 zLMiB^BZk9}fr~yqFm_YpMIAmm)55~MB~4UVdh~}x@c@o@bXp^v&V>-4Dg3)KD7SJr zClpokcDx*34!ReCw`C%JZcnlJam1$6-i?3%zALFg^@la^QdtL_T0j;pb-*G&Bq8^He8&OrZ0?Z!&tAX_V_3_9jM}IimR);s$(a!Pr0FXi9qi zZW>=*4srFEt6hw}_=2iny7SwY^P^-gfaMxn|gC z8EJUak%<1Z_q`EeR(JQa%rb{h4QdLOyB*9m>J!_-`Tf^9$CI-kBLS{X^L2Ua*iVP` zTtX1n>JL5z2&H}LyDqQl($8fv?xy2nwWSW#ob4eD{*T1lAcD z59*#91=l+>ODsO3zrk2&HUCuHiqYq$RU2JIh%^Tx@xIg?vmxsg7Nbfmh;#2;T=wTx8>e`;+Wfq2a-_ znAe5QFlwxaho{a)lt6RCzCWDn{eI1HKW6u6Um&S4wPS71eCR>khg(2$031BQnA5RN!(x%P00&zI0I9@~nkl>WN0YY{0p!+YxWd-NPR7 zgbU~OyUVxA69;xI@8BKzUtNP_lA+0poC1T}R$rv={0bIuZ!bFIqQQurM8>vrUPI|d zde<~k=b0}k7iZD;H`L{ru({X=Be&s6Z*%W#YnYyCbh}54?gT51Ne3!DaVPg*T+Q;^ zznpcKsPuek{w`$yQTEV1!+{T(XOfrK=Mjk)SELiqaezDi9U;G9Czd)kb#Y$f6>? zx_fy!mwn2U%3yrj?g{{kC7Js0o3_Mt@f8YoD# z=4WhdegDN-@}VNSz5Q@@8SXQ%cZl^-4Ai(Ns@da34!0roTmxjor4d9~cE*>Z3@Ec+ zn$%}Qx09baQ>_YxJ~0sSIuqzoA#l3nOzgOnO*Hso&T9mFHU_V$rfEq0o~Gs7didar zy<)}v)lUnsTqq*G^*Rpt@yr+pw4LA&`Xp*##p^q{&d8m)PA`{nK+=xxu~&>41e>hW zKU?b_R3g3Qow+-B**gO69++I)UCKFXvPcJ+C&nwwKpxj(V93%S-N&KMK?=IH31bV- zWwx@PRKOTbQ<(+_d{KtsfPyUx{3#GhS5)t1XlCyP4yec5 z#a^8S-Wx54$bEEd<*c+1bQPUrGtPWK@$NO@Z7YEIt=u;vrSKpPLe!Lfc(0XsMm$_k z1ZH-kG|bfRQ6p|xd415rIbUR) zf)BJQcspg3gr-pYf)S{I$YxSROdKnzEA*`y>T;`x3P)HW!h&~y>C1b81O0bS98dyX z)QlFGCX=>_kIij|&0Y0rj8Or4WvUMT&wt(jP`RP9(nx`75oqA1LU|{N*Fx61)Z$(Do~DsZCp`^O6-pfb#2T z@x^a9uMRJfKasQLCT<1MKX@DExcM}_-f^Jg&=v^rXMbptmBj8>%gWnc<}Dnj$_H<% z8XFW@^UaV=e<&d{XD|L%K>kk@kP_2vQ=g-Lj`#@>6>VqK;`5SUcLliO%|E|mH_!Z% z&o?=a=VXDU8pBfpHHLov1O|!62-;U^cZtwaVJPmn<;w}tCvfkoBF4Q z^FKEr?Z4+7b0_%g{ZHTjgKKO`13zX2)fqrZ3s^6Vy{l`%V#P-v@67d@df(7-oz|uR zkrfa2WQI^u8$&xFonivyj!?OdvF?@EoWcqecbAwPIApB$-4+bmTvI1V>QOfg;pI(m zAS(QGB>pLf^e(>v6SzoSXd`>0tbbTe+WNcRj|j}*cbN9tOPG@bJB7-Mo9CtyuZKwB zg`z!F{mOAps^99K{qFz9-g^f%^}qjuLAr=Y??rkQkt$u0CL+>1sDOYNdI=B`1nC_F z6cmu&5lQI17wJtpNhs12YLvid&-Yh$?#$kud++|S_s-q@;|v*QGUp^Y@AEwG*Yhd{ z)zShHi2DY90ltPBYUv*CsZYCz*t6Yd2T8)T)K&arajbAxny@ei2H>)Y=!-6$PxLr$PkN)(_)U7S_8%mD*y+tc4NkpcP z_~&B?_pVA+DF}}9TiA|JsQP?RrCIlx#G%B!cdNlh{xR#!QrcSsIXhqclwXi{OxlZD zz?RhN%!(>^f=D}J4pT3!S|a-=1t*a%OxD>^lpL=%5hkRc-lwW_>JkegT0Jgb!N%dY z{n`wAR;6)_V_$;{hBj^Cy}2eC!b?eiz>F2WJt@OYT&F~*ql_D?-1zx*Nd|&?5~@1J zN9J$qA_Es+{3px1A&Q%G3d_nksmAR(E2~&UcT!Ym(*aL8>3~TZPoGirT60xt)sR zy;aP1ZVB`~KcZ#DEmy9pr;NJf^xp7Z>VSJ4Xulk5sq-7+gD*{>RQM7M36>^I`H}2a z`}w$keVuD_L^cbbg7b=>2i-b-V(a;iI`?BkJ1-18HisS`QjIt0$>B|PY{dcAPUr49 zeDTf5{uR78&2Zz99k}veWX!sYC;|Z0-Q zN;aMR3%CKN7|h4vbbzBm(tCS~)~CvMF&%VQNoG1`Xb2Cosa5W~s@#w#I6wJu2_O5Q z0uP2<_ORKktavZWsTf&cs~z$RPG{huEY2nJI$eXW;GTw<2nlT$0}(7%Z1n6Xww(^U z)(fYCip^|%_B&$H43-V6bjo-9o-~>rynXZW=~*6)PJW|(bS}Lz6*jxa1HBd<)M_P{ z@q?e&l5wRl>)C#y0>&&=`yGd<$ZH~nPT4`c*m(qA1xNg79Hmd3T%BuD6GGm-wiv#i)Nc#(pn?$hk{y{Q{<0Hj>nj>f z8-M$JQkmN|R!vU%;u)q4U5BVy#GHrB5ki=Iz)F33f^|NSzfh7To1(&t+OJg}j7v`> zR@FU0mbZ05j>*ErJ2%C`@!2@`9(zh*G}^GNjo7x zhAp{1@IhJR8)V)LC-YzjCJfMrb?XwX7iz|ccvq=>Q_cB!wsKaf|dun1kXaKb; zm|%^)w2CvELZu&UC5!c9_3zxm=`)n4KG|iVt1egP!H~Ql|7K|S5L*DOnr`ky>AhsA zDDI})0cwah_D?|=nm?iV0mXUiWd0!gFjU4VllWm5t3KcPd;Fssg6WJZrOPUB;z@s& zk#eS)RF~%yJr*-h=VLCWlYpra?!H$+1rOl18LXo%1$S)FG?-(!;?aG+xSGN?b zy*2)Q9*|-X=ypq;>tg=_*)hYORhqEtvA3Y*5_*3uYKrk<@Sli|)H!V# zH-pIg4M_3T!)u1vpFzb1ZhjJ#Uc=2=-x%ICB-B6D7$tc-Nz-~PA{uuh1&Td;3KGtb z1&($Pz)j8TQWY=Re*#s*3B_hin<8_>tTQfUM;5J?pFK!1@K0gk^Hz-XYQWd3bL$w| z(csL_KjFo&=4K5Ou)Fxsg}3HR2Z85q!PeOe`3Q~3#&ijXj{|pj2;-riUSBv|0gI7v zyv8~=`R`Rl0P1r9bN9ZHo!USEH-lt+T$^UA*$#iw@v)B0?Tr`Yb%u7v>qm*)H>?gm=SrsJG&@yH_0D zvTP$ZGU%UwHvn0GO5u0B0l~Z>a@1&gWSm3U)k$75X)_+*>V)!$YG;{IcEZ>`<>b?x z!koCfmni7W-_?GW`mrQkJ;V0s=OwGKEr+BBE<&8^sQC}3Cg(%RE@Tp=dY={ zBC}jlAJ!iTZ8#)TvVyYyO04i{rE1-brR4ttA~(_pDaQU8@3Kec?qA`k`aSVf-F9M$ zCuS|KQ`2%6Qr33J26~CApDC5nA5RmO_|gZrn!g5|hvFZiI${(Jx~Z*u&v!7Ku3^@$ zwfuD>O_7^)pA4tyMuz`Xe&&de1)VF3=6ox)aAjgK;$X_={ZYJ_wZkEu)wF!*+;6S}R8G!3{)hUWyJY4XYF1w>%p2 z{8pFi348LWzTelxHPM)~0i$nT#f9`uFTNwYXdd*x)S?v$sp5I(kf;@6Hexn~6l~U4 zJZCTdk)GE`t5f|dU0s4BTX(UOHeG=?R7SXl7APp%1bTd#7djYD1SzI|0a0+DbviDI zTwRdnS=%0Ed$My`qJ(*4b8DM~J_fsYUX7>3S@-F#caT87*u9;n63?g|ar1fG5JLv- zN2Wpwbi3EnHd8&R99#rLRb%p-TE-PO4ZK()X92~$MV;zF z=}=0e6iGleGwNcX<^!C8z+ORkj8%(9wPUx!mCrBcr?h1&@2enmWm*!3wU7DTD{XFk zQs%l1QVELvbA&26FvJ30Juzx1TC7lv=2kgQ8LJ^%HCnRxEk60!TDI4)a=7u~FI^CQ z{H^(kkAS7RrgDYHt+Yk~2crQ_PcoMqR{ti{sNtSzSBBo3JY&C-H=Y%0P&DmAw0uQe zuGMxahFWqR4PGiHv(P>J1lkH!35$(fMN>8MH6v5rL^|D0{P3Xp`=AqF7{vzdTrtp^ zi=`W|x@5{NI-O@Y%^dhy*ViSwDcC`#bO@aXVm!PGvjvXM??As;w?Y-m|Lx@EY}FOT zzk_U7k(D+B&aYnG%4DV=?&6CT{#e&iyDnPFpZgTlp$Rb$hr1@d2h7djRZtV`Kx}?) zcMfOzw~me+syLnX6k3il>rVy#3Mo>^=jbji>)3#9VC=p=-#@B#q>T@(VPDZ2W^7KK zLocF-eco-5e&uUQl!L_ZZNE?-90S(Ih2norvhu&$|L`a1pFs_MM+s~JosCFJ#1XZH z>$b@pZ0wT1$p~e{>j``D)D!Y2?#|PykSKdLsMf4@SIwf30luoL6VtsaG!6ezb8nKh zuk!cs@E8LZg${1HnsCFM{7~V>14wgXY@5?__gwRFuz2&3SmU3?#azCAaER5-3Z#$% zu-Oj#yEfD!v>E){1wU|ZitW!Vs&2sQ=%e7wt`Sa%ulKebKZQwiw7V(v+&b+XzjbrN z{2P7*Ba?gEcbwXl?J&u!3Nmf`-m7RQ{HKJB%dLHO#H+gSFr96dTUr509j@@mYo-a( z?MOd8dxkxEC%zxkm4ftoGSyv=8!|Xb?9r?$ApZAuV4OMHJf07C6t2t|TwzI2U~5l5 zUs^GPpquWGEnO^Xi+36Nf9!f~uJrzt`t&o5SoWta%hu`gdRxJB_orYj`L{by3-i8y z{b<*qTLnfrcm?{Cng%+~IR2r3q1z`fCgfHsA{%0DWr;Xd2}jlRU(#^xWtDu8wY`1W zqsl*Zms#3kwRGgM4*O)}10ttgGL0J#q?4p-uc}_nD=VxDVh2Lcs_dqIN_C_%kg@9* zZm@!se(})VC%#!psp7Q_r*uE4251$d#``&<+5N6GQE%dRjmk+=G9bsbGH|~9w9p>D zhpY-UR#zz!>+prk=UNF*$0aU4Q4GHOxi?-z%>8tKtT9+@1Z7UItfqK1S}>|mG|rZ2 zy=ZnJvo))7+6-MV3;5*VWI%Isx#prFQ`{PwcvHzv{sLK_RO0uW`5zEhD^UqSq7IJ~ zpO#B21=UY{;C9Ve&oI7G?6pOvMZUEtm-fZSrY5SxrLnrzun2EIz(qR?sD)$P1%q5rhaT{FwmosM8`kB`?%^UuE=EyS|v0-mIhx%Ud@iwSuGpT zUr*4wUW?8^x!03!?%F+QpOgc39&KSsf(zN#mYEM%c)I?m?^2N84L!+CfYU)$aC+yK zID;5vl`X9@fiKKVC)Gm-+j>hOlY@E56eL`?Dfo8m^w;B@CZMi3@}3+4{AVxhrBR$! zQ;XEYhFGyiXQhI#b{?%iA1>0Y@VuWZf4c~eGza`Ds%r4oW;4i<5OWiRYmMZ(4ig$B z`Railg+_+CWU7uGq?0H1;=8g?CQ1roIcbo^c>mz(G@_}9eFgE&Id~v_zw)RFTL{p}91$X{6I{-Ka0L~^g7HX#!)o^qYAx(`L{tGZY z0T>CZ^{1?Cl2kiW`K*@WsJqOpulYD`;nM6M;G@n{@lNQYa67R!=Dng_52xTaGA#o- zNAbs6lRUI!0`J8inBNT?yYF^vlPTEu+AFr2D?2qor^G4P&dPpn+%{>d&SsiM zR@_Ja+0(X+jx1(WRBy*^sQA@q!d)~sEvqA4?}YVIXUlfu*UjWguNq?h8{Rs^d!a!c zx34EARv-oX*pkFnkuv;)Z5t)uKOQrI@-)dL!I5%<-ek)hbVNKUx!GrWR2^hc89XIM zjdn>##^bgfC6Xq&A-N&(M~|2=X}db*&dmFn&=4A+El(iu;U0M0fH0#DE5OxznN|ly zXxaEUwT1Nv`=2QbloVR)ea~+*q$<=*)!N90_o6pN41G*XY=`sRXZ|>efe(MDrfs!N zo7G}1pqGpqG%V$@_dbdUkfm>rvaE&pf&0Ao=NcZ!B*hSS!V%1U)?%EuFMEU9Ywl&A zTZ>U#NLfhy8sblcnP{%&rKM%Ckbeo`!!}^Jl5qDzjJ3dudynQ%wM^TlJu)39qb{pE zR*k0>NW46}-2Qx3T<_`;=-sVgS`cBG?`nNB^3e%(FL52~ z7}1U@ryHAn68xrxjylKqH-*N=iEQlwaf2`$%L zY``%;Z}Ni3KIU3j?Uwts3e=DrAppQ0V3?paBz5WfPVF}tV7y~17IgJ-XRxbs_(#%O zV7Zn&ZPZZB(oI)WagO(mj>OBvbD?T?)W9LqXaWE$uqub(Xix{s&C*>ZE;-zD)MuX& z(=vld3eL?-#lbhyc0ZYYNJt-1T@cl%U*)?aN7NEJ$lJvU&u9HSKD0B=5Tw0|R^Kw+ zP^w6{hyUz!|0-W$TPMfQWk#n|_eJMRGN-HerJJ!L7#C;F6!x%6BJy2o5R3!<4!>uH zr^YU!j}oe?{c6bY?C7A~2XCnzUp2O2PoL#Hukg90!#W7wwJ4J~jvVJd`+JnMudIQF zFWR5SVad|VAmN<>idu0+{C^$mc1Fuqv|h#rzWyTgsmwh)!%S0Xnj{g62MBFR|ygbEV8)FuVf)X!*EK- zDVdUs1^LqP^Vc}B+85P-b`n+0&yDbX*a6J=0Z3KI&1$pwE81zgWD+Wb60;68zXbZ` z75-F%`!TOvH%$3ORcAC1j>%;i)6qONySZC5_(tbdQ{#kuZbCa?wh@DvyX~@^$*D3i8g%4 z?r}y3ZKW#r^~f6i^*wyJ1-ANqQFV($0Gc9S+RioEu0V$_j>;bi$4(2nPh>4;?ml-PwSMLjQlwCzIW}E{y$~GQH*^Vm-gR?aVV*raQ=G|&%6Y3^*o~S?|C$x*5j0T z=I)L<=l!8_J59J#vD@n1jDJA3nFblVdc|?Bv0FMH-c4zdw!-g+-e90B@EY2ap;qh3 zxr-;i)N)n5(>TA0=gR7jyB8r_l)dYd|44BZHixy>Csr2Tq4hpWH~1WWXZEs42Sf~d z`GS1IzUPRMI}Ug^5__qI>qp`oQ1p0}qmmiqORP$LBZp4L#3+MVB}3jl2h*EUh^y(A zA!ViYD|*zx{?Vm0)Y(?fAKRaTHkng4;AK5DWB%MLWv}em-uvOPLc-%i9-^kEtC2_w ziRqDsgxAl~B8apMjNcpGVPSlg4B?p|-)gPs!Ik9dokmC6xBgR zkf{1<-20%XL?@GrcdlylE;>OX4_q5Ve{nJt!{52+6{&^*Vghd?5JdQ$OT%47U_+Fk z#wh7~#4aCUkG78jV`jvW)O32NvgoRWa_z$9{!QpSce7Kvx|Tq+6IH`cbnN5eC%0c5 z+QhH_P0}#n;hb3jExWz2IV*8k5QE?a}|8Z~VGK0l9+o z30-sE^*^z1sw~I2g?64RPnQy78EeimuhOTegk@>)>g0VAtL!-H4S92E2>kJfEeS2* znaU5^nZz(-uN0kCbM8m5F)nE=*VoGHNjqWO0UBzEReba+uoU0EXYjhYyoXG+^StJh zSHPaZ3&t{Zcbg_IEjm_%|sCW(*sKv%pxO!T5i*VN}vK za|bdG)_VV0*?ppW|?+T}#b+0BkaDC}Rz+kAAJN9=ZK<92R{{}6X0 zqBAeyHjLcmq7-@j})IPh_o z+P{HsAI7Yk@5c1kEQTw$L@(H7r+C%!S9Y*iAcbe@XT$ieaB#1aZTNlJQ3BQU^<9GO zRmn9g#NY)U98$EGi%ob-Y@BuS0+Hjka8%toGHPWWv*sNGIUZq^i!e+kh0O!>omjOW z-|&_`7)gp1Jo>T^j-4DqQ*F$r{`zUt&-JW0zX7yFMZ;4jsI1`82w}OZE{=H7*U6qp zq%z1UG6OP|(7S##Z+|ufS91}iI)p)fa=tCQ+pxo88Lt+x=19lOLI493 zCPp1NwThtNF*$NLyDHS0P;XFXI9QfSOqgS7b^L*LkYqu_XH};Z;}LVmky4`2b@POt zieG{eIQi$;!oP2Kn7K%snp0#f-QUz*+PIrb$aYG#K)XE35!oH`9P$QX6ff7q>l}3JhOSCr&BP)=+{zx1ql3H|=vAee`sr^+HU$N{ zaqi1gKdf!|k*gc&#c{2B!Zfj>nDOFqm=?^)Z5Bqmgx7nHJ^L6`62k-*^k5{;6U>;O zw4@4mhfm%O82|dtLI<;T|4`r&hVlF^D4tCn#KKuhtp zUV(4G7{q3~QTIF@Y?~=z~u|9dprAvk`VPV(pN9Bk;!siyk26Jn- zyTy{CVQk12e6DX+8Fp&dKd(Kbp+SPbPFG4poVHTB+bMw}<$?ki+6RB_7%So*^Fnos z#T~Ss_`$hNSDl+0HVF6wi=b=56#FCc_v+W^=FMj}XP^cF5``(&wtMnqVliGMIPFX1 z64&e!;(h7^ooa(~*{I$a;36>_T%>ZIZWTEC-jR}!Ci{I?Pu>RFz!B#0&B*@sR!k8r zdJgUdVPFux}8IsdP~s%}H#2Y&TFQ zvC^y0`8j=C)$B&Ym^r>SQEZ=0m>7G~%_T7bEK-AWJ3RL~`$G)wC={yg8dzh+?wSu6 z&rNN3)VFo{n%c`oL+M9eJ!e6;w_Ftxq)-Pm=ZROjNLxPru(1m(oc=-IDNd_G;%9{` z!`rDwR`1ep=0c(O#ZGBo$2hv0f&%)tK#rJV1E|SIqO|JYR%$V)!Q(WL$T10xXEVGm z?%?7VIDPu~2Z}kY7M_pR6RBY^*cjcO9p`+HSki#>RjgYy*kY-Z zQna)ZWnK#6>h91279SA476k7zh*Cp%bXrTYK;=8a`z{*n_}gZy5B;cE1!oVDI*cBY z#E{pWm z{^wLm`4!JX^TQU5F59s$)gA#S>>m&PE3jIHb3Xro*ABp946(%#rFdhVsp|cM!vlBs z)4^|3IV`mPatm9{+V>)K**oSq(tuGx{$8&=EldYa5 zG9#v@X_xw&Z*I6gal4t%*~C#NU8dGD%hpGyG#JfT$Wu#98qLD0g3qJLxn{*ncQ>EE z$G~aa;8gk8jL(2S{*b%)VQU#`&XVgPv!SC+^}=V9{g>)XiJH)aOC^Z09uD5=vFO&# z7H@9yT!!2-R@N~sWDq??(h+bVeqPJ<}pj&jAjw{ ziqEwZIsI=JiX^BIE&w&~{-|BxcaGppews;IZ|1Rfkydf=NMR}kqmh#Dt?{ipCSLJp z3Zo$rnjok=_QW`9J+WvVuPSWA$)msC;->H+`xE=%y&SE&XU@LEsrlu%>Rw_LD&b*V z^pmYl6$GWUzB?+rt~r!-Es>P`aA8}Cy?5yocUUo9i`EK53S$(P`=LRJ9}z@<+E5|E zT5~W7h?&0P)VM*@yllwxar3%M&6U;Sm#M4!pZw%AX{)#Naed(knw8XYeGjNgEw()d zH6GP0ABl<_H~xXUT4}E!(QvFCPI=pIG}fkl7zQ3E^@_c^(aFl=SJX-PO?(oL+P~%k z8=weT$`=y2h_iZohP#0Y^9vop=Qv2bKb0T5@|T3Bh0QbBNxXcwqFe%M4FuJx)n)F@ z%(>r$$ifcqAGvyk1iGJZ(3ubaA_Bt4BM+O#*C4DCi3UMdAQX&S59Nvo?#Ct&78CMj zJTfeWGAu3KGXm)QZ>`-oNfXV3=G}hquqGS+?tlv25DygE4q`lccY~b>kSS8e-!2$7 ze7rh_wpE;|44ZWPIzie5Fz&L& zwSVDh&+0#3gz&owAQPrXFC|l}?rKpCrLZoO+TqnIAOjf51HNMX)o={1hx1N?H#LjJ%d7$T8ac z^5w3Jk^@Qxw_NgRm~=ym>|~FkeJc{ogszqj?GHvI)LG_sK0p5U_$DFGNgp;k{kS~U z1Z1&+Y1$c8{O^ke{+~HMDJgYbgjG4+`&z!F8R)98#4pV|#9mV%c7x(?Y?G#mBYGzP z?D0oqvtSO@D4^S(j|8~8NT>=bL0-R&wRi43MVt3T9TU#4)jvmc z@<0~y3XCgSrc}^Ud3r=na?$52V{UUI{-j)Vs!T|ruBBtGX_SMSqHF5KQ^}(Ac1+GR zfe)o3xXZpGrupMXQ$xcgzfljE!#$%f?vLnjUi1as646qQS5`~G?;y!2#OahTwJPBC z7GG;2k+1XGXh7nFI)mevoQJYynoh=yX0BzUEf*f+gtgVtyHO>a4h-$~^Nz1AAZ|Tb zX34AEQ>PVrDiYykh)4=d1MjmVGJzML-hIK12aQaqa^vZ{GrjHc4n57VG#_En7w zvxL&_kubJa<(q*@6ff_-TUrd1aBz0MzI23+UQZ5P0PcJ7a&kzfO)Ync-}{Rl%Z%8M zFHVI`P*!}I==AVZg{#UTox%>`EsXUU-Phf|H|B`B&tM2WLN}QS6?x)aYjRg z=QficEr$d@ZvpEU1lLIvK6mH3VkfrK3g_*YxU) zEHvEcG*}#F4%EpEd}ij_7)_;+MKsPL4P5j7la{VKCExCMPsKN3NOFV4E(qPR!qvuk z|4y2S1^pLM%$N6UMW)YWc1 zm;I@oogFt3M7RAHPA$W?W~FcQ5d-^cL!n{z;S=_1#{ds>pR>$r|HqAtO?cd*UwiIV z+J;($_0N+Nz~%nu@#yr(KcJ)}sccSoK)mzz_*-1ikun%lgV%@k0k!`;Deo%jC&0AP zFIj-DNqhWV``iFtIzH_;sPf7P$W#p3U_Liusg@|Tbnu$By}ik&`oUD|eC9J^;k*X0 z7thtj*J14kayTv}3WE;I&>h6T&)L45?}*Qdl`pS!i6jiJmad7Um_}mrxWd}pX%?2w zgIi|(Lg4GgWoY(_@eg*A6EZ77lC%=pi@RnQf0JwJEvX#M)P*!Ye&Xow-1awt2qpdl zdbbv>Dv3k@HlLgm%G8jotD0E%!&7!$iLLgzcMXBe^f7PJDIWLrzIx0`>HWa-9^hQL z2nf|tY8jlzz*0>_P(I%xDuTU>yH^927_BsGzngLjC<|`y-O;0Ua#M$mP8O_;h4{PX*RPmA*vRVHR$?eJqEfV@#C7z zCUY3G?|XIc1@JPEX!#aS2bu27TI)Fw>bhpdz5&?i*-CBTQ+8IcyK-uzGrvWscKKrspv4bcx~CrYFD((kHM(bYb@5gLbQyiQ#rH~9k=3PHM75uxO#ex zKAeAUre{%nNACweh|Gz8KiP|4svn|HmBU|CuH7 zf5UUD{act(MaYjeV9fv|o13Z&9)MC}0-!dBcp#H}Hrl?lFu9bky=wqeFu&p5N%hu% zNas`yrHxGTJonzW%>S>xmupU?XJMrFO>EWq2>j?eiXeRH)B(hQXk0S{TU^jM-|Nc~ zZCC;lb2|QVV+#%duzc-`25)%au&na`pYOl&Tm0wu`F~l2knR8c7Hh2W#D2uk0KDh7 zCRP5)0id@~O9?g^D|~k*%CPwp0i>i8{ogS1GXS%y|6t@B>ga${JxM_lj3-tEZ=Cx# z)JR8kJ2%#_9c%h6y>G-`+r)dRtrfdHI6kn|zTb~K{+8(KIo%`lAWQ-h}5 zY~*jUh+=T|%iAopTOZfM&CYTY)NT>f&{X3$cbg~T%%&#`b5>sFm?zK?d0YJP`o6Sz z$7dqlIhk-IxmU-Zai`q_|Fm<9Od$)e>go&DMfQ3Lj9g%{Sb<1;Whf-<(PI*B>n zZ^TXM%4m$LExGWxzf73Bm0$Z4THLcr_}N9*V3A%FZkCVvdHmNS$NVM#)!`*E8N&1< zW5w=ANX1$+kVsQ!29O*d;slI{KyaeZ_hqoMir~<-J^+a=fNIkhSU)3C-++g z3Ku^ax`WSHWd$Up_pg3i>p+W0N>{yGPZAMCdo$*r_uwq;Y!Gjiw_rEh$VD|8J<_&4 z#*~bQhvra25mR1W*1!d>E#gu}J7CsJ#gBcLqh^2Sv5Z2s>frQ)r$4B^yx(BCN!hxp zeR%FZpa7)P6)|q;4S4!gEu?pWaGn=1uJz{KqP&wqqSDv2wkk(42T#vBsPE3`-bBE@ea19Rv zUz@L2y_*zJ_G<|e)T4!%CG5Z^segLeC>Q+v?36(3a97o^1@`Dx0q^Gm0vup9yRZ$! zAUbFV?&YHJb8cHM&WSp_@TR*F7|W;f*dJx&wK7EeCg$l7I3}RGhT&eFU<_I*umDR? zb8QQtdz@muo~xi-UXFXo{gq0xtfl>2Af-nV{bPG1B#l=} zd(pp<-I{}5opa?sCCrTDko=3@W?1TlD*aU6oZp>crd9O*f|g0{r_WD}U%sDs_wbDI zWG(9YAweBS69(151lQbj&EcMw(K+Ln4^Tuj=rkw>Bi`}HiV47le|-6o`s;Tqmgr1^ z5#8KyK#I-ozKAJ+)b(b}u@!nPB5)tRrm7)%n;*g@sc4r~{k!8sd+B3WvD^J>k*X5c zzre)3au*_b1KX=D?lHXKB!89)P0iwN8hk1&;dCLM7OtU$FLI;CO4g4U@C98~VQ$lS zd;MJF)QK1{mqmWU*QcKgNcuND1x%f`J2YnS1p{e2YLthnyh|ae!=}>(?S|0fY?t`x zsu(MZ#m5ybtR_RJ3`9E!^EiAs0G&NTaazG?ASJ!Y7P}exH*9(~pQD)?6Zp0KEe0(X z9Bp)ZB^CHN-d6!S<^+4HR?^D8JV-F9_j)i#@|x}H@r1XSROv|nkW47#;&2qEtKP%RPU#ML@12K^pfHJ^a13F z!o)MBHDZ~-y;TZjQlktJ_iO|T^Hb1FI^7aRDNCh{xwVfW`P0!tHfY+56d;hOThi>{lgNx&TE#CndqCE!lcwJmlPuA*JRES6CVtsB%$zIlS(6b~@ciJ9=w^DC( zEtB5+?|14*Y@3EyZAJ>A|5{{_$m#B8gBae%S;^vzRhHJWk1|*$^XL{WT1R$Wil3~J z6eM?a$ZHMlC{f&tYN(RfePzRPBxc9o4&b=1Zn+H>Mam^SEq_{+9C8k>V7=N3s^L@3 zTwmjc=nb3)Ft5^gFxM{XZH!fMB#+$6)*ww4N#}oT5+*M^xTDVbf7%5wZs*d&?phJ# zup2==zedW}>}{8J{)WVhmkF#KOuvidIXKP}68MB=pbzt;Q9&g|b1A9v0g*)s>zKU4 z`X9DEr3)+UR|?`5YYZnWuZ|r-8}95j#NqwI#0$bTyx5L`+z&A6*7on%`Ix(Hxdvf+ zWhUC9?hVn&$Gx@;&tkQxG#i6w!Q{Y=u7nPbsgj|g9S9bed+=7kh$<;-QDBWv`_sUu zLDgeH+vks7JYpc)pD`c023_SJsq*Xu*++wUr+xQ0?R#|#A5r(%h2(Xb`E)U%r9H2o zpS^%0Z7LeQVozh#T;Sof_(=Pk4YA;{62?UfmawN?)j68ck5fs64_QNBR{Ed41L=c+ z{IIT$-lH1>1-6j)XQpf+8Tr1Si;g$D?J4_ey#|SDH|_0am1UGQ zC}w;gdCw0HiB^>_-gqOe^Fr&rx`KG8-0kJ{3G#|bJNDqqCj^P4u2o)KMp{{Y2~Ffd z`=l~$myFMM&(-HMDU9_`M~Ty~X~uXshAACgW!!!IdtR>AmF@a-Fl^7a;m)cE zLW=Ac+g;|2h(UjVU{b6FT&*qR29Ohef=%iV$eiP1!|}`4?O=8NM58zL=Wujf7IFD8 zDLP#)4KzV%U27cI48XwJDz0ah7Dr;0Kmev$Zxdx>9qArRH5LrU#+xhnk^J})2?F#% zb^osE{Fkot|C0wfkb@v(!cqceC@7K$78MHH=s#cpGwg~S=WRLo$I8X}F?j#St6dob zQpiYe{jn|H=TXP(%P6G{1Tw?d#&3DDjF{`}y${&O*-M~k!+-=QWuB-hy4zDK>iLK( zGSP(NXBEN_VO&~}PA<*<|H>=#D%kj5QWBgRzW-*j6AOoefB97%hPB zyfb}Z#Ym}Ca4GR$Pu@FrYC0JBHx(=+M=d{2`9XYPtZ7s!A6a33mj4W!s^T=lAi<6i z?-q#F*2k8=Rfyd-sGC<^BJt04Xc8xx>NWUM!=`$Z3DIsH%0$ihp!%o|yWEOkd%b zzq4pjNk<0C=nU!ECRR?q@3!Pp4GmyJ<5=Jo`eNRt^-a}No~{J5jze^HYI`#n{2eM1 zSzy1^*K+dET2HKO1soiFp=inH_ zrLEp~UEh+nmqNebMqdN5IjTGhZ|5^k^Bvz$ z{3YN2v+lu+2SlbG*F2pmZhzy?u}!*6`g6@82($YAr_#5+Yx#`9+|^}I85y@78s}M~ zyOlZhyb)>s0saB2S0OW(r5g2Zzf>8zCwePQBz>3aGiI$D^sP=lw)_VCJ&%Q^{#BH? zm@jo8YIeBoZcghSF7|1)dXMURGJi-*vN=?JJ?%8aE85MTaJOjf_AKRyjhr5l?OHj9 zLY?y=V(W#f%pLP!K7@sH+xLvs{gHN+=|JgP+NFxR@0UMs-=TwC(g7p}dL6t7POY<@ zbxMX>S~ITq0iaJ)-1ARkMTPFwUr6cji`)JxT4_<8n2IneZ_(}ZSEFqu#Yv-SqI}00 z`cM|9D%QFQ%j}uykYJQaF(TsVpL48H&ZFnRP3&Z67b6P3M2gPd^8d4x z1z9EGCU(Ael-z1IDdyFTc&qHWo=YXzFMGsR>ssu?o0G>jOODx|Esw>kM|0F!Wl#GU z04dYIALc`Iz!n)yv%iNKzoA=!&`LZOYd%^P^fj2>HA&t*5kV{9${AF-Frz<=2n^* z7d=;!<6SFg?YI|Rf7pG#FX<~`8gdK{X{loSR>H&^-~g0CTuqg#GKFt?DxB@tLRS$z zKhKIvd&YP05IpVOW$1}qJfSDiZ49c1HAeHpL~t^gCo_Hr^8L+kK9j^aRQ8l;Fw|$f zp0kh)KXm0&5U-(YC*mXdGQV~TWIcu#3&wrhS(FJ}dIhF#9AQ?7xk2+O)VjooENBk- zALZWu{_bD-UXRE%W~c#<7(kBUEH2lw$XaH(^9xUF8>`hF+$Y0Y&%r8=l$6w}WNoof z>C-a?qGJN^=&wa5F%v{_QAsi_3WvVdqi?of#ExmYRPr_>AXR(p3` z&u2iA)=PT5?;#H)^}lXTxB8X4c`huhX}J3Ml&p2u##C*I8$Hz63>!2_X;I*JJ_3FK zRJ7_H{L$mOiPn%j{A<@UuypS{+w5FINQI`4UjXN5!-CbD#=9@A(?JUQ`>pUwq1ZMY zA3#A%Y?pPt=0wgR_^~ztR_#SzFyEzU>d`jDx#TUQ<>v2}Md{~-69{axxOd}8P{hN_PMp2Jt{Ye!XESnB zdBXj8cl3OPieYZ4d~brjIwpNgB>pLfy(;T#IdXb5s!XhcX*Yk;V?MJRBs- zQnhcnzQkV20XbY`_EC0H*S+|%^#k)RjLr9X#OoF8rLti^damP7c5-y>)BgblHbaRw4_W^tam>#Gme>Pa9C%aA5=~?W zWvg01(S8SZJdWFVhVKNWs`)MT>O*#TI!rKW^YQ43?>c5XqiN0;P7TpsUw*NpnBE?P zsqkScAp6EKpZQFdIX!bzgX@TbH&#LKzD|d%Ysz_M*On5wgAU(lJ4h`0CSsf~vO(?*Z|bNC9nq zIJO5ZnfR_%+O^!}0JR;})w~^bxT^O_i0Q)Iu!}_EvgGxSP{x>Cd<3e#f$3yuu@P3v-rw)Tjuy zMe?13nR*s9VU_NAb2mq5{z{J*2MWN?(N_SJ&)9#6ClzF)s3oH(`qDWfA*WJD))Sx+6*opV>Q~{I>lrcJ8~3;LO1iWJY|E zJVVg+zA}a~{=Eo$tYV}PF}*ERA0LQe@uON}x>tu`8(ghdDB%A1#|C?4tXEFVXbnY>)A5M zaZ=LRH&-D1sPSyU|G`ilu|n$l%`^Pz4uKMEX z_1y5?i&X2c%JyW+GS{+o*oBtXOWA?&foRJH?N(b(nbmjXBLnGMw;b5(aV=`H{?u4K z6iG8UDXPkk)9spG#h_aXpF!8ZyfY&I>w0GHr*f^x3UcsWXkMv6^g|Q?FD=^lHN)QD zPHw&(C%d624M-Kdt)F^iKbK<3;N81#lc#)N{WmzpP!-iLOc;GW6NGoe+I+=R`LG>m$5qA&Pq`0z#a@Fw)4MbtDJ*z zU~4{9(@DsR=cSL8(GP6G%S`UF+~YjHe6(zXZukH1y%gi!HTG8XN{AW2@@F=y*K;o2 zX_m|6EA*y(qR0gs8-Dh||_ZR zl=Ohly~>}<&I&T7uRIgm{1}3_)1uf8-{|0I=f>-yu4_?9tm*8*Y9?CZ=D1}-4WU*s zZXomR{b*8lLvuc%t+Cruc3-hQn4+SU*{w_**@eM0Vt(Rug-gNxw8>9JZd0Zpnr)XM zS%0fkARTj8!|{N_oaj;$sxtu-fQ&zD!LD-kJedN^6n>WI`(n)Qy+2O{8%vchb-9LaYn{J)pXig)3n@5{ z#yQ8xh*b%$J)M@(t1@y^O3(r_5hHc()i{0MB1`_v%Ix-Ptkff)weZ(nN^#PE>jgMS6*X^d=ynAiaq+siF6d^d4&HorD@7#QnTqTi^Sg zwb$AEtbNW|``Z4I>$yV8gqe9}?z!(@VYr7Ayv)REFEd=#8)lLpKKg=HdTnolYaP$1 zYkKg&_0Xuh_v)jY7ZmPdrOb8DksgSyp(c7iDl8Wo+zt*!xCir#ND5GUK*5PdpM(sH zKLWZ4~nf-3y(mo=*(}W*Bxf1ty=%@jZmsofWoVSbEqfQKk6kZ)NTGBoL zrXZ<8K}^)^FK%zSoLc_f1FVJ0NeiwprnVh6kk2>XJSn+$YHjgUvF2G2IY*+)=Oxw7 z*oX@H90u^6%rG9!+kcf+`8R>H5&^_e~)w%1$NZ-QMrd71ti#PM*0@ z?N8tmHkkR9{ZpI`&KS2XsK%EYR|*hZQSe3y*f8TSP+!m>kjWng$PM1hd6#7S*lN39 zAk3UMKE+YfHo;w7;8Ei_piEeef|WYS9?@y;0g6Yx(M&+`y8%6l3pvFZiNdfB(HO%8 z2HA~h*lL?4fL7n0;?BB!)e0MGOk98fa{q)U{9L#I8{e^}Gxh(W#>ruY)ZlZfk&A6W zRdNy6(GJwx?ZLH)fSA4%3`cp|1wLn6ICBIr9_U(GB-H2aFHoj5&=3irX-HS{Dl_^x zfan+K$vMOqhBIQCs!ls6$v*`Qu2&VKfdye6v-u13%=;IJpH3DhrikU+k4igIY&t-e z17LL59ddQtkJqvr5S9HBpnn1`sO|z9mz6SG5bQSq!SgTw=?@gR9U6j|FW}3IlwpDg z*~L=6tufX>VCpkIUB_0plEw4Rl)r~luhkLr+|+Rm%2#crzV8VA?&{DW{=M=v*FLzn zpw-5?LrwXuO-fo(e%`6@KaX(q?v(8Mho(Qgwy2CQDGs|T{`;?z;|;+aT7H3~0G27> zA6hF5fAtHL3a}*s_flgQAi$KntRuvtfKL3wmUII*7l88s=-zcrGwoH8tpxyNxl)6b_ysDBR{8}R z>Hf}D>Nv+Vx$ypPrhFMu@$0FC00Y1WCT9{Pp&P?C8`UDu!XjtdOb!I?Fd#C$ynvzd zsS)?)jedcEp!EVsgCi9aWr64xciDt_dDSY<6NqDh(b%5>jw+1*^sdJwnJKHlndgrv zxe1CR|L6CCM8n+w@g?g2{4UV%rvBc|zdP{XCeQC_@sGRj|J!L1@tm3^$hiFuVZvGS z_RRdB?&#lx^8eGI{Am<^udDwUh2I;~zpa?Rr^WAS0mN$m>vxOvU?b0%TFZC(G;*>U z_16&t*XS(TQ5r1d#{#U!^aA7|H ziRWS<-M*ndj`)|SAnsz)CE0d{hsd-bU>C0&v96 z6F6~{>i9&Kx$P2(jBO+Xqsr9C9;LTd^nOJzonYXeA;`Vh9l&Hz`_W?RT2+@LdD4?ho%An<)*yonfeJUSukl6mZE2=A;t?mxXH0dP0JiSsP=ll+2>#Kfv| z`I9e&vsR^Y?k?oc^QsOpHCDHOSP=dMfrz>RIsq+EUTW0@DxVN6PYH0Kvb&4}EX6Dj z-tI1#A4-*o$vVtLVok+Q(n&MF6_Uty_)(4@^Q6!{pWxkhy&-UXGGwI4KJ0e6V$+d` zzEZj`!-9D({m{5+a@GF^|di)S!vo{06HxhCrUuvIAvv|?N1gCRbVO*2(?$im#_1Zqx|R%#Rh zX2&<}q5BeH?jP~hEuHjvW_>v_ualGzt2HcQ@7=a5QbEhHa-wVPmQUBgQH}pO(4CI> zYk_61i~n^BpR4G)X2@+H;$I-LoxOpW4374RHDA@tN}3OgrJPSP48QddGCWQvHO{Q5 z!J02#7}6PQtQ-9Z8aR?P95_|y)#SA?{^?+7i?7W)#Ra*S8-th)@j7c{H`UNw9h(of?$EhgUjhrupBfSj;*K-g_0y^}b)& zqpy`vwRS?G#`H4Hw8F^fGFO0;p|`lvEV-&!J3dIMoB4?Koy2pV&y1p|*CY0o=n_2! z{^Kw?_mgk+p?W-mT5)a@Eb38sQum}I6{I{DE#lMyedP0KL|k>t)ZK46#58U{5%WVa zcDN*gwl@A^;Q-h;6!UbYN{#@isvoxJD-V0j`BsaP>xm;*l6!Q>V^w!PM;Zq$2`PAJ z>P-NcH!O=2T0jS*#U3M)LIqY;Lw@Fd_oT!5s>58v)2s*g38-&_AAydxc>1t{=z!)% zc2rg1Qe86Uh}uUfNQKT${#K(H~YgD@8uAMV0l@ z8Uv)r7r^ucTb*ok(w`dBX< z$`&*9lHihJ)lUl3wwQL+!&`sCY^=zYu7Q$f%~$5eH`US0WJa6orou~)BKns~=!rLS@aQu4v-jY0|vzP2WR=Whw4NjqS~fZbFrRa*mi5H~W}BB&ISd%e6~f#XecetedB z1isxzm-qdyizt!OJB}GtoN-=m&v&sg{b8LNSrqrnltmL7X37+EhK+DVOM}E4VhISh zy25GeN!{%0EH%|pG#u=UcDk<&Qy&o~i3m_)bmhnFDag;-l`OA(TT5lHFs0o|GWwVTIhbdcdE4xTP~(9E;b$CxmB5 zyiZ`4Hn)0fKbhWTq`bJ^34rG1ls2LX(xA?}w7eR|?NJ8ibF_iHMEv#N_Qc7b5*ipN zkYLk}$9<7f+T2(TmGw(Ei7=ioU|J6e72hXY(rzv3BZl@X{#&dG{a@dh){%Wh2X$zm z)1ri-1RF_J`L3)+Mw>c!ZTKu=nX6;3dzIyj&(y8XMVKvB3M2b8fz zPGd9~6dt(Jk0;ath4;9NsDZH6yPY`7$hY0cd^)4x`thYGLvP#gkmV#f=I-ke&G0OXspkAlciV59N8x5_w%;F<2_Fb6D zrnAXwxkq^-ZabCRwlfomsl9K-P&IVo2a!+#mJyLYx@qA>)!r5t@?22EiwwP;tMq#v z+K>>&s{jB*&PO>n*--EFGNl3nKAtJ(|6C;_iriKz{=57w$pV5Z++b^&Z&+O zqTm$Y?OXugFsZK13*B*5QK}>nU6QQR@7!1oY>%luFSy4Xol)T0$6c{=D#6A+J_Mli z*@(GIQr@N^G}~xDiF)O91&j1cK51%_Fq5ckr(F9vdzp?1G$m<|&HN(wvY(nU`z7J9 zmd>?#UQ}fjoGGYSB$>N1G-##FJ7i+D>wCMQCw-KERy8{KgT2Ux51(t%R9le|$?#Oe zPnEZehw&e(4g@#2uV!V|54JXuW5t&wjSUXAo((ULip(^C=xVYez^>K;@fT%E*SZzE zn>@~SQ;qyaWhlyUV@+dH;>9yowETH8@t%d14Rup(()IIC%RSvmA4L1fKzXXW$?=YS zDF$GzuaQi@s0x>yyuoLnG{dw#?CPORW#QZ1Q7DNnB0fms@BjQ`ii^QPKjY6I# zau1tUmCw*hTNP{GB=Quq>d2d_&Q8}I_3VWhZ!p-lWiwVfL$m#k{MtGm___On#_B0gUps?LKPnc1 zyjCez8!0hei?Vlpq(}TXXKlzfX!*ZSi#4mmG)wcSHxcP zMUXYDd{VvAU*|&d%sI%~OglnhfhVz9vt>POW53=KDb0G)_k7|wnL$i_>`Ah>VfeSK zB#l`-+K1+6%c}#bptoZ>_=90s0n4JN$hw9%10)734q^R!Y=&P5|CoVHa)w9$U>N+J zY49Ih@qz(Qhi`Boj&8bO)0H(QsKaG0;A($?b|Q@~LFDT4AQ)g*_^+p%{}dcEzf#}F z2~FCr+ccfjjamr6vbb{OO~3k*Zp;R*n%B|1&RqB7iP&nYa)^d=FHZt`?S~9ARxqGT zB1_q+z^b--IIAJes-}K)<^5W~E^ocEFr&UEmj*on7bv7smd@uMdUXt7C06DZW!O&k z5~RPChkHw|UlaV)!PEKLlbXQsMlrgco9k*6dN5k}(&hQ64;_+Ec$1NOI7yi$OyQ2v z?Cf(;?An$*F|7g!AA~R8TH@@YG+f@ln@L4Q_IRL_h(@crT73JMJ{ z@FkMZjX%8Q-ydB(+Dln3H+;9n5&yN*edk`pa&&`1>b1)>9F?t`3=yyfgo#MM8SO|F zH8yyJnB3VnwHE#26`?zE*@x{F@}TnEWlTYno4U zwk9X~MKjFlJ3{qV4zEnNf5qAw{NP|4i` z72e$CJO1s$MR02q!LJ7|Dp@~edW!`$5&$j-xYEVCrt4c?{dvLl@-mg?9#GtX!(!Tt zaxwMpR1G`s#e8$-k3tz`&XUpo3nVg>TZ{q#zdkw`nr~z^Sg=LQx0_m9b@1Gh)@)Mn zdFtb-m+32_&wnh&^m{;BNF#%r4HklmTuaF%WP>dy1eW4iJ-Nhm&%I|@f6Qv-9!Fob z>QrWriVi(@q#D=Hz5PcNcr|ym-b^StT9n%S-_dh$5zj6kV zKk9qksXlum5WG;P8;n(1EFpz#!wo)31FC2@zk zk(;uNBK@*$w3flHq@-3!D+v0BhMSY;@n33cD}wC1-mJ;p4e7f}9V>R->!vCBk!+M7 zKWZ-qNh0~Ysw`-`g-X+Ca}uX{`AyMrL~3G$;ewHd_Gz^p$mF>UPePgf~C%aFIu?h2KgS5u}W_ zXTLUk2BmvQ72kpnVR%HGemjetgFk(h!Or;d^N}wb1hb7)Wiu`dQ2$YrKBRwhT$I_rL+WoGhb`!#LHxOE<2niF#+^x9BYv(=C0MB{D!m#Ys&l#@v*{QSI zzd%H|5g7qgLTL8pkBxJgtZyD(_tHly4QF(caGm$y45hN4Z-3x@;XPcnP9JrzN5(%q zvQql{LNknLzmWmUj2dhY9_JG&vhX!j@mJ-rudiOEE&O-}SAD?8%2SI^)cvwc_ANb_ z1lR9(CrxfbEzhX{P~ji03a#gp zuxWcbTq$r{&eh$Mpw!i70UG8#@}kh>9-FeKh54PVU9PAHS5`lsWqRxUoCl!=$P-)qzZj~*OgMcd!tX!V{Wz)W!BChoFH=3(M zmfO96Ms)C^K%4Xvf!i3?QRCstNZFUDhK3c14=z@2CvLKhftg9-CDmg24@FmI)>g3n zQI(uA*_hzAG$J2CxkmuZ-|!HfRW7{nRHAQ<%zqf#+in&Q}li6xdmeKGw-Nz&c@iw zFT%c%wqKwtQcPL4RhpZVCsoDehO?c+Ds-L$j@^q79x)&A%x(O2D3exO1tSPv-NJix z?PUF{<{zBWOU^#yQ7Pl}K2&)dBM)F9=TI3kqd#nZryp5rb+ERj6F)YC{CFpth>$(- zrBmSKF4p`t5=dfT`5}b1V!D$u6126(E#~YATx!W$VVf3rzqK&?6bA5C^D2z9`vw1@ z={zrhavg7HqVkaxYm?U0LW_m_Jbh^81z1vMYF!RRfh&IyTwk&utp8X@{Z5UJNw*~f z6FYMf8f}=+Yg=Gdf18hXC#$GFG2FMlW8KusjY)Jq-6cYISnDO>8obT-Mwb}mt{=m~ zoX7J+#h_u4Z?1;ksy8lJdw^j|2kBwqpe*i!eFR2fOYZ6-{p z@ymf|C^FJu*mAsrW3Mk{g0Iga=4LQiYPE1* z$tQ=`PCkMOX3ktSYa%{(3wHUxqmO+5*)IlApU3YU+j@o_To}5MFVE`w5mGZbWhnA z%xfx3&)h?WzLz26waf7O{rrLd<16G=BONAOGr1W7jgS)X3;jL-t39^^q_CcO{aJL^d1U(?21-@+9p3DU$js`} zA9PQ%eh&D{-waJ?O}yHBA(}{gi=5ZhiHqoF*p;`1*u7X7TYUa6MLh( zX_)`b4|avVn-PO$yx&)LyD0)~k)zy-b1xgH8Yop}o(g@Ztx9i(X3wJ8>MSNzwoI1P12Z;yQB75ap zL}UZ8T*wmqO_dCcu3GSmQeA7QZ@PHde)?0#77jb@aey{&OH~t5n)Zm4^o9)n zrrUfXl6A&UpQ~_xATk$HakR@EjC0tp5KGFhzFcj)x#RO1<$Ds1zN*uRq6&nH(9J#J z6B5jw-H2JRs`W=KDzMZ?6j-m3#m;;oKvwfO+V1yqEj<6Ot0L#dfXK1Cj}yu=gtw^zdjnW)i29@>O9Gik-`3Uf zUAJ+k%H>{pdJS&(9KrjHhxaIE60!`=a`jy6|%y+WFG88^y6Su9rk0M>v0EQ^?Qf*+1atJxP`y$TZ za}rgpafz3Sk@8)x&&qUBb6qk4R4o^eaAa7Lp}Vbg`mQOIheq%VvUwWb;+rET#Xphe zKC(#aM*u|>n85=gHWO({}D{%$bV{7wgJ71VlFSk6trO^CS zRtZ?S{as-}RA9jRXds6wFek3PcD}UWhPX^;zo&~$AJr#jN%wj?mEGxhE@{v234Wl7 zxBFLpo?xuecb`m%%;~BpjB44#R4NWR$6p<0Sp{)$0#c~_6BPqDAD>w{D_`PezLT-O z{JM-55(qOi8lgn=?=6ZZ#aVMbs~HQ9y%sGUBg>=k`0<6-qm^4cn>K$76&fkfU|=r_ zXB20Y!UW{*jBEf*sW?r zXvS4-opa{c%Ixy_)DPa3=-yn;qSlU6hFIDg*qk1yUaq(4q7+@0Nhv*oyB0`XW$)xg zzMbPc>Q3m%vGuKLu`G{O08Q8SA5xxqLAMAj$Ll3+vpq&TCvajIC)m=altLBZi`eU` zV{fPzmHKPyxkT5*j1qZsO6x9aF1*4{m%GYU4dN5;M}AtXi00xeyj*ziVyJKXePsUO z;zq^3#{QQ+y$-C=WWz)gUFFHBFBCXLv!m4l-O(;>C=XqoaWiyEN`dvLzG06G-n7|| z7wn(jTJ?UtdMFFJ`SF%X18QJjuM$(~`we4a*thwX?~XLTGVkn_&1nsaGnS=Z?6TleqETu4Ng z$geH5nn+>Pzb7}q6j8dtsRI)i&5qA>=4U%@PEMSL;nbP&Z3AdO;+=JGcSv@ens8vE zmq;mW5tbiZ>uy*};d|#j6W}!uU2U-a@Ue|{F~N?Wduz%*{e9f2G5Ll^pw9hUIUM&C zEoK_);Z5ELC93ke{`qhY;wZs{O^(r|iCqWFEk9=;;~^jDclly#%6amdyP{umIld-% zF(K9(57HGr_IXV|5WY$LL0(r1FV^$8LANz&2z?s0yHg?gVa&%E`8jLUm3ucjj$zAi zmOaz(V|9?Og^BVt^e*I;x;*&y_u2=U`RtiV3ovBe<4tdez*e zjL4blbb(!J@hKLEIt>yKQE4JlL_)4?G14Tr$uNxbxz<>giu&(a7h*pn6+v{l4+EH` zj!DKoN_Z~>C7HyaG@GyR5xyKbwv{cNn^no%bXrg)S)ZQqNu1E&Kl?=W%1@(vs*^zO z!X`|Rk@XGNqerd1{3mM;=lxrA`-OAGNtJP`CR@SR)FaGtc&duZbTB%h1_(Kme#yBx zMrjnkw6j3)kYb5dsg>r9I`riX7B;_@4!o}3aHgSq1jPK+YClT)NT^!;*daGI6KDy8eGGiOb8Qdi5p6K& z+eU_P94DALeq_TpUbox6CCeE1dbUYSnOdbB`!UVTkreOQGFe*%W+q^Kw{+ma1 zyF^pCq4tmQtSV^vWt4Kvt_5y{`aq^T>vMpaDe>1aw5$vK4&ELWJOq_P@P-mttP+v>dXNPzA{a5Nek?Xl=57$ zhE}O>euj}0I>*see|^A5{@LoV@6lY?B{i@ui2KMcUP81`C4e%2z6V5m%tshOky8ti z&*;kB&Q;>VxBH-`!P7zQd1+QH^&Oe)U~9Q7&=#H|wv>ig9NKF_-mOQWM(swGhcd=9{h<ZHu)9Sx!^NQL7XfU2pG!S%bS7B@py0ZbZ|zmz z_6v0xRHSkrsTY9FU0pl|qB%Z%`$=ilsLuREk*4`|+Oy$CMwP`WKjN)Z8M<*1RZ02U z@yr_KqoX9;2%nspXr>k4&kR6xu$zXJS;0|(XWs^pO|e}FoK(kBVdgyTk=EeQ<4Y*S zidw&>u@)Y?{oo8D@EI+H?Y?{8SVMABXis=?xv1Rfw+HNggy_5=k$#?~@skXsEXl%FVr_0g&a9FCme-T{ z%~4MtV!AqGVUnq{@R;=^yvS{<0>g&hNKP!4Dw(n3dMf4B<`2; zO_#2K(+_RU`Cm-$0UJ#q`D-H&Popo{y0>7Ne9`o+#OaA~bgHv)w^e@Qpx7x)U2%F1 zx4(72hGR-R(V6x>w7mavVNCtRv#IVpO#JsB|F4-600!}wj?}`CU29uTY|GSrza-6k z%yf80@eIjhr^Q}eDO&=&`Ztl_rwlxUQUJh*8H4k0xcRn(RNzW1F-Y5u{b~$7ftS)06W7b9z_2Fg%ni0{QcVRe)->KPEa7v;mw)_Ht^bf zOyn}pQ*YroPfQ*Xk~TCT&+5PQs4Lc4o*v#3eB5Kz;9GM;|#&+tWW+ zFQUs?n_LU+v^G?oE&^8H*iOm>nFu(TO>nH8?iLZ!nps z;b0utEPw=mf0tr$*{RZr!q{fuz`Lm}W9PMF#fGL(|TAu zY?LRbvQrMQi&mOG!Lagw`Z{vF-!J`J=l<@Ef1YapVQ=(r-|nG*`EcMMT8#SDJKxUr z)t9!6|6bfT^q;p+hrEXz7&ZdpA5$IKzd)Nt+N;`IO;?&$;68wcMrpV*z{#urZduOH zbVs%r{ND?9;$n_troxslpQQm=Rg(Y{-+#cA^Qr#>T*Lni2I7D9d-0xw&#t!s%VQ28 zCz4fA`RAtZ)` z@)%s^>&OeK+z&y}A3{8!ElC_7qb9^n9tnu>*$-z}Ur8x0ebTMO9M9Lq+1OGI_unJp zb9kw8vsZmM=qb<5+lml$^7+f7=0IYtTtqj_a53m)} z2j~h*smVFiynEtsdPY=0Tts0Z{KKs0@m>~X*1G6kp-+%^ys6EEbE{fUR{=s@V`a45 zUThUVt7fU|Bo-YX9OPr@>S>v~We^l-ZR>68Vo9;>f4SF z7khI>onZI7TVdA?rDv#y(Z~`;pJSXGqYK;d%JdHINy6mk~ znhB|gO9FRnlg7Np9G|kEx?3G|FjV+!k#UDS%Px!DG3A}>cQcX3z;ZFTQMI6x^TQW4 zBMu`c6I-3nqC9ft%@%8wvMJtNzwcs_eK?y+vACfgrknWCX>E8k;?0P)_25s0IvD=U zbGYYFd?dC=dw#e~4UrG$L7X2PWba`w(nM{VDkhrjmo7CJWIB88?@ifhzw8N1{jtIXBa8txTAgC@L9_L_F_mLDzxFqSfT zXg&$+1*9h6c~zaj>J7JAJ7rOn9QZ!-9v$1YTedw?Txhsn1qPD0=|%CAkqa$>d51BX zKiIPH_MH1Nf1@Y+iShCSkTRi*sP|f9{A4{*CLp1PAPp0-NYM9@Jc+Z1#*6+uRbsP@ zA&&j|{XyD;tL)1nZJ6jq0snf%d>cds;FNhvTah3}Qy$N9Au|7!=2b16g-F@IRKr=0 zsV&tun!VpS(~1j0ljEf$nUIsNqsk{{GOS@OJHt`<(Mqx-annSz}b zE*tH~jj@2-+Vm3{6&|Y_uYYh5^=M;H!08N=34SC$<4I|W=8EZ*yk{{@B1Y^G_Q`h!PUT)m zo-|qLi8!?0w{_9#a|z7{BdYu6%WA7eoHiF?ytqbAkw63(fnyy3!0TKcj?)O~!2s9b z(IWK(Fuzjb&Z2O&ql({IL;HO1JMNaBDgJlg*mLlS*_sV6f_g_stmNwy#z( zcBJan+Ui~4v8Qq4vxAAiy-ej&Rssv2`&E8MyW-4~fE#OA6Xh$w=GYurB&m=h!!%_u zYb)S9N$jwmu&^tX^YMw>{dATWby1Ges&0Z<5GoJLhGQAxcQw}8!6dBBAFS=!ez2{m zsk*PBmrVvLs=;f~Ae6^*yynLY46yVPU0?X7>!rH-rizLfWvoFlbf)(56H-F!T8|;1e3oKhQ7d#=*!Jz2`GZcfDjL0;Jfaki?hf58 zD3@iXP(NndgH01=TyI7ED^K&$tP9n~JX6-{E8>|oSNn~al9ZZ>r>NIOwrQ!_(r$UU z=#Ps;a;uP@X%Q4DeW-QGL06E#02c%POluFMwDKE%%+fb+M!s3e% zo3h>Dnx_VPX&)W;8-2AQIq`ny{R?r7r~m*7;*87Rss&Pf!Nq$5yT{(%A8;eK$<3`> z%I2|RiVq1`4nB8+Kp^g`?e)6or%QyPSXHF5lqA$9DK%|5()$TKa5aHWKcn2=Dl2x~ z+RZ-eG~&BBPar}~0o)D%di&ZTB!28$e(YmVDamD6&eNjv)~d8qsd&2u>PO7$tP!iN4k$w#jg>Vk zp%*e9PXvN-oGASHto#L1>gS_dN%8HQoEcI@aFR7@><@~U_rFD3F_bB%lS-hw!jEO& ze>B`^q@Lv6Qys5J3Xi_CB@vVld&o@80uNQ`p?`Mo5@{dSwHNnw-6QSf6Pm9G9w?$J zI{IQrxP;8g4e=df`(}H`c@8|$;lK>xr&9Mf`?P$`r*fB36vF}3e(7uhA-kBKf~_R! zS%mwE>cWX@>l6MR_wqBLy~ysfFO62cqC_->w)@lDa56Sf zK8u7c3)lO^-(v!!NqOvV%tjacxGrr&gK^g;d@Kvgx|8@m!THJU?nqh3%U}Bn^|>FB zs@(@sRM;xp39TssczOGYaxUbw=o&;GKDx8rb>njMJBJ7J`6 zo0vSD`&#GO>T8n*5NiWl_PHFkZRV`K`Hv1xKDoa%V$F3rh)*Yt#$CrcL2q~%cD?jS z>tO2=^P-F=42{*QGfa^2t-df$s^%qoRb%{JHqgvO(4@@ozRw#J#J>_60=fH^fzy|f zgrQOrwv(Tu&sr_%Q>XG?{LM^TL?1OGo9Pn@{LLe8aoxY(xI}*&@k!vW;nae4xg0_F zVtx78ZT1)=lE4s+jG?T1I_q3HF0@=bhe&BjwT2Oqeuy4gI&va~Gmc&&XDXhrFsIY> zgF+0l@#VG1+Q%Tg(#Q0cA$}ZKPZVj}0^k_gS<4^HT{<#?8!)8$+(Ex4uTxnarFcgi zG4mnjWBqNXPIusETc8Qwi(GmukMizXMn6aSE`(%R2PD`gdj|%Z+81Z_v6MLl4SG$C z014c*8#7kt#o5{a!e%(Zcc)hGh$L^R=ca>0WHQ5sAd$;cDBj(j`0xn@ZK!C7y`Z6P}{VqoRE#wvl zw)T!kEYXWe1?r-w)bkUMAB?C$sAJriY1jpuOP*ky(8=oP)375F_Ng{o(;|;2GwrH< z!lkpSPAe61~7T@!y9nhm!0whj~DK}2WhWk9*JCk&wA^<$>f#H0`5Ol|Yk z_h6Q?L`7*qMMp)pdLvw@f#l1mL>*nhHYeA-*uo*B2jTcC+;-^-gz?Nngi)A>OM@Xk zkR=EK_UTyTpf$(7zev#&LsLmZJ)&3jA9$4ZAaCBuSlaDgOZ}>853>T(V-ngr8L3C1 z!pg_VU=kzSDv6-4i54Rq@7rAzUQ7wq1`*tQxnt_D1(Nq>2;P}Qo%hNAKppn`x|qJs zn8)~P=$rN-AiZVXKQ%;TY|0`dszI(Ao&_=tmuWsC8QboREMt}ZrtM;h2E2%C!5Y@d20U+b~i@8QRNg`EQXoL=9+bix$p<6Hwo@ z5wt>m@19zUZ~8n$Nw)Zr)uB?>A~_IEA6Ums;fEY|=z_$KS3Tk8dL_mmT<-Psrc?s_d)f))v~{ zWicptNtO&|w@tfO>XLWa$e?+_u@}FCKUbp@IVNmj_Oh3*Pyd-25WRY<{srv+2jKs| z%H#ER(-z=Mxm|+m9v_^P`x8&$FCy7Ye{swVctaQ$pnA3O3CtFA}`4}S2k2KU2 zue|-Qv-$H39S_oPBsNmip&A2_=p4=rUt5^dL*qW@y_1+(h472!2c!t2xUr8s{Wgn* zA<14At6PE?=#%i5XoZq*)^JS&54Tps0Ak-1wgXl@tuMO$_8CeRb~o24 zvX%v=OIo}r@`8>%i-DMW^E{!6AlePFlHe@SCI!7e8(n^zWl4xw z({Cx{tYN=B-Ho1|zZOG3P~OMy-HLweD<{Dh7awK4+DCz1tF7| zwSAkxWWg}HXQHvk3BR*n2Fj`8oeZ6=wr?(DG?jE#VxH_y))YYZ>wyp0wUJu{OYiD&}9SA{6Y9 zOuWCu|AgWFJE=G|viQB8ubJa9$5icu9{@WvKPn%QdK5w8O&PXu@bJ=D7CiKg7*CNy z*u;_!hyYsg3|&qSAEh_l!M3+7K7i`#0^jwp1oWX_=WZgRl1+0>tV!)%=?fD@6DK+M)bA;4?cHID`B)5M z@Z|L5)U9~GbDp<@25y?RRPnJLDOJw`11lXrGQclAtyi%!7AIzy4Y#$3xhlsZ`$)&= zW<1wBxy~{S)1oxA13`vrH(?zIsBR?q$3RlZwMxRjpL`d}uBq0-w-l=HY+>HT0+6~) zoR&zVQ}3hk=jPfry^JH=O@Q#^gPEM|&C)iKchwiq&uq-?lnSr-E7N)9LzBJ$F3wA*PSPlGqhdlZPKgbGzc72Br-~Hq2zkcVvOH1=!UDO+e{~g>CcFcGOgW#MXDrJ@a_$!P${q{ zSM<#Dp||-JhQ6$lf;lSQS@mu%H1;D)!{$QK0^Xibx+@<@KOKI?vH1<_1a$r|@(@Dw z84hKkE*k^9ZQbA4A8oe>S#uKbXt{crhE(6i&I}2#udLZiP8E#KPEWur(}HB~qDe!z z&~*9E=<2f4a{t*?e!b>sU7YyaQ`+oRb@!V=ZccOS8|F zu8b^~BY$A{A}&ArmVUP7i@IJji)#@_ozcO5V%v;VLs_H3(WD#0HjdLq*66LJ@d>Qq z?A|RRWlAo`qDtdsUczfh#riYO;z-|U3P8&`ef>{Vg1;-i9seJjFZ!D5T7+bXgv#YS&VcC&6{$rwwpTKu*_GVuOG$H^5F8Lfij%RFVKwb2GKv4%_nQ# zbuH$Pu9gI^C;GWHUGseOlkvXGS`H-MZ553EqZBv{Iey^-lmh)BxEV#PuJ3X$_uVAa=$7yHBcL79_xim~dK&XM_L~0q5Uz z@a&JO90|oQkX9E^`Qrd8e-*GcdDyrhP?RHrUD*Folta{5Ug802*~+kwu-}aVwb=g` zyU~Ca+*SI0tI9whK2HLV!uLbP`K}_LDa25 zEHDLFGZL5MRF@o@n!_N_fQvG~9sG+dCP`+2@YEd8H57uOMcl&9;^y!9Hj?i3{Q~&` zvE-7~g%8tQA`aUFlr+Z?+*cB37I8aGr)yvwXEeO!avHY(GYzY1ocZqer+)Xw@44~& zPWi93ZX{pk*gRqdUmSD&0^QZ!7RO!qs)EdIPCw2nUQujfeSz)nm05Owg>N>ITYxJj zJqJu4$|}u={_vIk!(;U+5Z?eN(tkI=`Fo)AA6@^?4TAnuCodts%*NgKXhRadJKDeN zL8TDhPFuNFnQPtintySU$n!yDR%k%(w4M{4sFvWhf?GUuz}E9Ig6X&ya!D+)=*Q*r zM@}Hx61n~K8w|kyyxK;X!0UPseHM0R0@Hdrk7m|P9dK^P2(Oof@gCkMkw4Ii;d|^T zEXjAwGY`?3_8~e+{bCViu7tg%c@?Ld=HU`&EK(VaX#Pd;YhD051@^aln_Sng`FOIh zQiZ{!?^&`^?1<=?&b~(U&x&@ZpBp_7kosni#Bbt}L_u^fKUP@RYoZ^vn^2=U27{59 zm>j(znv9cNr_e7r2Wyd$Cj;OIu&V)PVRip85Vy%v|^FuHxtSUXXx z7R>m?oX)C;-S5RqMaPS0EeFP5K0jdJn`q zRGmy9Dr1#-C+;e7yj~AkvcSLE>E)SRmX(2deIbt#SSoH}e4n|ADE>M*A!Ora-kD+d zL+-ntJnOSl%7`*aass6RcD>nx0(^vvUaa4M&MDLDsC~n`a<%8?Otb>p?#WVvN}9rYus_W>1P;Cx?8Wq zjZmtaa^j$XVE`ab?Zb)LMKdn9ohwE(3WBp`;+CZS*4rqpGdCx1OE+M352rf#e!|5@ zNjWV8tZpza9x(UNh)i!SR6^cYTx|34s1CXAx?O0Gi4+O7l1L1Q&mZ{WEk~g`zI^h0 zo#I~?uesW&bms0NBY_bs@qnCIPBv0`Xv{Bn)E@61U!HtE>A}wkcv(M|Gq%eKt9U_X zT9NXxj)1pCC_k>k#Vc>;49)Xw3`b<$bgMFZBoD1XRp~8#AC)od=3Bg`#an%oo)(Y;lq!q$?+QgvtHbKeq6BIipT~X zFcB|FF<EEe|G z{wV}~^Bz>Xzau{P3vri}D5`z__2-4|AFT1tZq>wwd-s^Vc@mY& z#o)1Ix}Dhke`D{f!=n23eg{xM>F!Xv8ui)n5A)mI@;HjK#kEM zEQQZMK!riq_0XMSL-%CzY%fnq=NXkoSvE?H4-B$KZ+@G$&;Vzcf8RxuBE;h7zO;r5S{llI~+c*GLher}!C_{lc}H1HyDg55HD z9!dmxJqTfo@AN+VRC(NJv~M=iIK5_yJI^` z@Y`>pPkLMk7VI)YW&SqY8PtDwI z2rp^$w4(Rzy(N?y;*-zzmG)x)Ua{~s#>7#Bl%|MfjSb)3de{Yj!i-$z8?VnUU#!_> z2VA&%VHy{5s6x|_o5Pi9e69L8pXG}+q?0k)Sq3tlGB^!AyYDWl%$9)~4!(C6nlfHx z!?cUrnLWhGBQt4s73XzE%`~bLSVtWl%=%(?S#qC0kr}M=xNNI-QL|X)nwflreY78G zhSx-j3Id~Yofmv|)n!OgT3g#%4H2u1CgKKabG>1^ZBLccW#evgD?7kI`1%wlz6hn@ zOHagnXFX=Qwc}J%o^*y~&fKGX=9)&x-lt$ay_@mY38djIR|{XMmtgi6N|3^i6!nHY z>CkMOQ+hVYK%P(D6ii=Vvu$yh&GYS7<5vwJ@~<-*{TU)a0#$|(r7a5^a?_^e^7MO_ z9~oV|-tL@7h07|`U_%v`_*M!Nd!0khxEkx;q47ERAR=l2(F9Yf=VkJYa?Ju@~h|hdTcG++h4!g+iU8Z7NYQKq-ipiWB z^7=PfX8g5R?E09H!G&FMi3W)JLEF?HbjrXtb$}SbO_ZmQ~Wk)RGV-6yzbq>XOhP+NM)+ufu2pJ1-4oF*4 zQnEP?tK{|y))k~SaiH!as+R^$Vi#A}8hG|!yI)8#Xj3>hJFj*J=go+B1 zi@qph%DhMC83DC)V9{>HYV9rD&?vS?*nCkcmdY7E({4M&d8?`=*;q(p#eu)GVBP5Z z1C;IShzxm^4++v4c~%*--sWAEULip_bEYpUMr_=6mj*!yd(H3AR)VR-e4GrgRIe#g z$+cZp8G-TTO>Li?Uyj&}5Aga=(jO;0I|^`;>3ZY&INTc$LLn!&3r-r z;={*KlpcUdCeO~@7T1e`R^6<7ou;@k9sV_#ECSJXhqIM-{T`iu~$`XdN6;F+@rNO#IkNWtHgWID};wBl?qmkzGkckW8`H*li(D#E3 z(wBz&$H9e#o`)3uo^L13tyaecQ^T1*q`WKmva%3kzH}1)TY8+HABN}*V9-wCp|{=T zT^mgywyX_Qj7}j7)_(!zqxsTV(D~?K>N^s@#(l{Is*R*OxNE#sBV^K39!My;XYur7 zf?Ob^n~kpDeB!Zgd@ox*FbcN)Eiew*e__!S_7sl!--LEMqJW@`IXSQh&vn)rv@(`v zh8q9uaFzaNmtQkBrfsHjvd;5xnOT{hMoifGe{yWJ`2Rur_&-81|DU`D{Y}OJQP6dr z>JLz+lD^%~N2Z3z*jBdoYI>?~E0rk+{Z-R!4a)M%J3Y+vYosd=A^L+iXzB4lfZy-| z{o6b{2K=fkagA&CT=d{yM%qx>8#-vdb?N$x`@fPDs|zReX(h@t`Q%BS?4D+vKYB%u(#Pd zkv3Y!6^|osV^ZB4ki}TXQ(c_PT->19iZLP^cPDX8(fJpFEx?lI{?gw)6I_p?*ayde6(zF-u`K zs)&{#nH23r;X02zgXfJ_tK_G1XY_Z^yyOHF{f6J%P<&_X^nzEJs++W4ulUop2S>LB9d|>*Rh8%n{!1!l>7^z>`ll7TGt1T@wk((C&ZFAynP_*Qf*Z|c#6CJsBE9(b zm7Xp(J_)|SY2BeuDVjzeilQ~;W?m>*V_3M~-oV6eT!Ub4OF;EyU(3M4lu@_iDgT+l zEpDraTH5t=9~m-9^)oIxN8EWB76(I*`c3U4BU9~lbcx(9>om$aR z+Wf<-4bnfl*9GY*HkR5-Qx5aG@47%M3?}n!Ev5!5iAjC)ap;vdxJ+Sr%fV^K05CI_ zXlk>%!gQvw{!PdnTZ;Ac-2`Ico9`CfKq;VOJFd0Tu=DE-B*udgYAF(G&;KM&4cEn3G4V{}u43syDO33Zj1Mu+N3zvKQidc5Eya3$$B9JM+T4${ zi)WWLePSoxIQ!szTP&{D)6c;m^ba7vTvMMVc!Vhq!s?@Nn%&ys{tX7Z1R4ph!u61u za!uiuaSqA+ZY>Pyp_k4dXnpNVHjrl=G{l)h6taQBTB(y_3+z(h zVO4zirtQww*22|MNk{_)k6m$&7^zq%CQ>V0*&FD$7%_Iv2ZyU1#K7fhsw%bMLPMO- zU7?(hUucFd^vIr)9fTwzW_Lv*WytZUSL`YLMaIUrtYWnKbBwzosm4UHU+UjD$Cs7k z-@h;I;J(0d1PBWX;e zDPM_kbT{Hmbuqd1{d9*nYu?g@M596_AFgfrq=^|AM#D?Kg^21H9w>CCW##u&K45oL z=2q|n{|UBu^AAM2X`Mo@??8mwPsJNcL#hm_Fo$x8OeLSe@@HS&BaS2bIwi~AhjxP) z`)^>HWh7@uht$K`di*_<&eak!yDZ{ zW{8+#eQ!7ACO)2QYx#^X>3NWIMH1ia2Zy#7U+Lf;b@MmX@=+D-^8`6TtrJt65I!YX zsFwY-O0gDA_)_4_%2&BhDxx|DOz&=15$>G@d?d)v{|~E$|Hda*w+P=E_AGQlA0Sai zd(V2Y0@0>BG^OXj;_IB~Bj$I&e0rEjGN8x|xP{!_(;HRin2!u-k|;`0vX-V9~e{puAN76+PfHKNjeA z-kP1vVGG~I`TV70tPr3ikxm4i2SCKjZofsGe--|!gkL@3|FhQtmZ$~5JfbJA+@V+4 z6M0@Kv6qO*>+2eTrB})~Dn@gWuz-1J#z-5rCx!Zm-gzyZh`d&;r(Q3DE^o>(tk`G% zNrsvL@U-}a)A@t)8GpS-_)FN1I4aZA-5%JmCQn*HeD?{zD_&5R966lnb?QnrF9k?F zia7J=UG|J{gqe6{8W>J(m(NUX9c&g@+xt1jI_EuYCzvkLblCI#c2M^kjRbi}kWwbX z5I#@rzJE)eHTm=Alt-1*65j3cCDD=6;=B;9i`s1d86)F*k%}{|rxqh#oO$^B3Fg>jERjV$1_oc%&osS0 z4x+0rBx4pfa~Kux6r0 z1rjrRvC&hzCTY~fOVplJ)P>th=$=Xr&Gcy$oa46UgE@LWLSHLX;$psADZ%E6=63y9 zb+A+69Hnry`i^N;Z&%rL`-#Rqumm<(u&*yTtYz6_h@dpT#NKRzb;Osh0FpFJ%u-dB zvUhJ?D@X95g>F3LSywAQc4D;U)e`ILeKE>uW3mgFtaVVl+;!#x|S@g-gqe~s7)y>pJ)XXXs z@HxuDIXsGpRr3xPP1yq&H4cq4JMwaT5cR4LQbR#srd^vk4 z#4CG9^AU7z6oD*_602E3NwkVijM!GEvrarOYfPWh4|J+1u4?a&xS!35tSr6VS*QH; z7AR2mlP_ysW7W8%^TI?^hUN%Nq)_2zRE@TZtDwBIrIX8$@3f{~?9;KKm;Cg{oVo@GUS2&q6XPNmBoLB^ghaY|U_9|^RwV9QG~5hC*h z%~`ho6u3Lfq*yCq&QRhaSb1HBn^`&Q*dCuJ9iuIaVb|NDh;V*G$sti+7vyP zYN}(Nnyt=s7Bp6|Db4A2BvvM}P~zeo5WC}>O)N(DnjdS#O}{5}S*eWq=}1+urj^|m$OHhMO5Fk)ibUl@{SK+igMj3eg|Kv^VK$=DA!?me4_h3L}HArkR-ioFwX1ZsA5cE^PR?1_7YV z#`gF3$42cq*DMqQzAiFZUkFDX_-^FnPY1P6`U~0#+W2!$a2G#SKJt@)SPg(#S^<4j zQNi*zjtuw29#c@#f@JS^VGX^}@2_`i+Yg%%U(q(1wzd5@2Z_~0?A#$XcPoD3NKTW? zsurv&N3;wgNm%Thr4@aTv|iE+UU0$%Z&8SzmozMK&FWi@dMjr7#!Y=!?vbB1l&nx* z?FoL`^*PN&Ckb@H8*#SZJWtfb1nF!SmrihNFu|X=ARp8jyW1GJ>O^c?!pjJG9^I#a z#=oZ+?}DZ4(CRewrzFuoyS)LfcbU6C&gUz&kU?mF{^B{fUr3{&J{Y^p_D*mbro>7B zJ4QiPlL^AYbB+uNZT(&>DGx2mi#qb@K4xibNZDme!7g3{cWLQa24KczDhIMok~*B} zGiW~f4gLAw{xel60GuVZuj}F0ukFXT|6&IW68-(jY~nvBvlG5xC(FFvr49Gu+aucX z!dx{9O)4tE6Slao=mm>p7wcmenb@QyOwas~KFw3}XcVq^SLzQ=6Gn)Yhvxg*S5pP)|LP03Un zD(o{Mn#bRbEB(!lOVaS>Iwb^F&k~3NFdV+lrrUig1dNPgb!x_W6I)Yw0HT+y&577l ze^q7toa&7T^2Y?iZ3)U~_*(&S5Kw>}^Hq+oP7_1zw^X@r=_iK@4(Q7|_Fih22HO$OO-H)0E7tRF49bS{EC@7U^ViQ|2f z0N(7q+s@zl!AK$aeVQ_c3qo0>ZxfEubW7?}^rvm3l17K3d59JtnXwvnb;2+^xr``8 z_cnLbs+L-)2gT~Zo`Tm3yz(^4qAZ+gs8ZS8-bBDH(!@x@5Hr)k<}P78?I$e^vhEq3 zIK%1%LM-whAZlN&Q;(WY_4pZHFL{~6#}#U8qL*F(B-14^s?@y-LS6k2hxPg$`(zu}WHpH3 zs@z0FfEAw}R)XxZ6rCzsTC!qR#(G!6AL=+HUPy1Bip*%^QR)=;3@xex>*n37nQWN zT~DLvq0MgP(B2kRf=Tvf+40t(wKI^3%+?w8Rd8K0!BU)f;j7|Bo+%Q%LLyZ}5!Z3o z@FR|3SJac6FY0DsZX&da8@(rSTd_|vPRtQ$d0{chGGQz}bWCQoU&cz}vpG|q+s;op ztG|;kzkKtSw)%K38zYK=3kAUXS{F^j=uf~&V|?zjTEkruY;ReUpOq7PpvikBG^`L# zd&Imi@o!!Bj67;^qM6*RJ|T~T9Ee1m+~a8zXr^|0Qr%6Z8}%YS`##=VHW{&ggA5{R zc2xF~&xQ-+eHPD0+o8>}Gue)#Q089NwHG}rmplCUSm-4JX~!0io>WI_=b?ylHoZH; zV6vFSfR@)wl&ev?NV-V8mO(LHcDy_~MmBCCzU+#A;f4&_HPx0j#C&mguCS1R9#x}Y zVmfX!j+Xmi@Iq+HFCX8Pa5Cpc5}9^+-_x^u-UHD9bh4I z;+B&X`Z7)tA|d}`wkB#lWf*{r*%DB08qsGw!%xiVEwVZHvDFiwN$PW#0E_B9i0@*? zzRg6c$gZrVBoWT}NNx6pF>!^|p$^Iw5ehi6;K0a=jJLeaA}$pcc_emR4Lco$Q7Z?l z0*yUohZeSU2BO2To9hWmUtJZLNhFD*OKHfSZOAs4=C}<`1$9vCWn{HGEMDEJ(!jSZ z{A637J*9j)URLC9VCgELg<_7%e4^44(8iGRxuCtcT~B2PSccd=|Le8LuT0%dDLcHBe1D$8e1A>tIShWQJn|AIRD_tS_u~t+$$lAqw6h39- zd3mJ(06t4mLs+R(m&7PbYbb9vecT{({Hz=?xbXe62$}`|Z>0R)TnfZ0AHH;a78($W zNS6tM6J&p@!ow@FPQlaMom(<2Em>=@s&Hm02N$r~j&whJSKTGb&=8e!#IVaFaqXY= z1JpEeX>&b10$<6FPbX)ld0_N|VElciQrx6r3aAwloS{Lx-P-BCrKcmU>G3@#pasyqQl+1MMC{K$4kIR`MSS+C zhCC&yX@Z((D1n3fW|3w=O&poPEfRc?cJ$YveLbgZI*5=?jc9#HN=3_*^YEj}1oNGT zg%PG}3!31zTngL|6sujpy*N&W&y&9A zge@!-a-r8Q9GYMS!&p9|gdU4)gf7@TOVCf(OE6?hvCX;fWtn2-#q6;+(6)f~0nG>y zwEc5;`^!c$$tr-_+Rh&yQ(T?Emz7SE(f}D*P3XM+weW8-$e$aO;isDn-JIe~{`umc zZt^Gm=TF!EfA_#54PuC^64Ei$YY|TPz|rmE=wL0{0usq9ySQxWexFYF)oJ1U?^cl4 z=_SVx5Jpq1bEdIUzKPL{D(P9>SQSlpRtoxK7g@;(0&Z?WukZ<}LohW#Ybh4QV$&gO z^ICr6+id5HHCv09d}+*eyu%Vy^!f@`UO1vY2d<;QO!GT!e0Umr4KjeFOlS;WN}S*U z@{cVhQhIwhsmU=MU=MkgK2w*V!M8H+g3D%}hV1b*;h+d~f*z~MO|1-mGHCx4zPr!+ zVl{o3C^%$cs+%H>-o-;&hxjF?x*ywLyg@&`S|9N-+GMbr=pb1%I1b=Pcj-jG8!FH> zww=VM%0_{^st*)dLRT3Y;skwMY~ANyGZ6Ty7X1J@*;AYMx~Vnrs}_bmi{Hhzp;u;# zf9@Ifs{H*ei~#3$&)=_5S0sv}rL6OY=J!R1Dh_u@XDVB-pM>&(>ST3s6#t#$(VYe?*-JAt8xjP+I20#9Ob%ti5 zxLU4h>Q(#k+Z|Y9dhnzkHN>CTdnPE4;I>%^m#CV%k{pt>yNjs)ZuQ_)>hk?OxcxLt z1n_>-f&cCx1>~xJy7n&yNc3H@Ul07-lOgYj3GqXll_x!41F^dpk8kK|{9fhX1AGR# z5MrL6^{VqI_ zU}ttfR1a`>N`d7e_0j%MJ<7iK7*ii!=XU39t_$`Gp}`OuAVR^Qg8BhsGQo2lmq8w7 z#2$zY{X5(CUt6TwODx)-Z(|M{dBdNkZrGhieNR=V;WvK_3f<`0WvWBgsC9O4T(`A` zmsEZ)9$%$l&+fS_3yhW_PTuBPc=wYXo?Z%z0;U;q>2Sab?_~Zm5s>`&%M+%Z=b7(- ze12&=^!+Ndn3CY^0wr@G{R)T`1Jd{*-v9v+?Fm5rkbWAF`M3F-r*woE&W=%~hXlvj zfR=|06zf`PNLYVx9x*f}b9LMM2k07O8hW6{@Yly%AwbJY!^y6nT~`7Y0Q&%!tLq#y z;_Tv@nnnre7aC?iKwo{Fu2r)C%ZaNeebDuZC3v6Acof#<)g{9j!-H>TGSF6e;9orA z*!}@R5jX($0>Xde%x9|D%kxY9v(WQz0l*3t9t8BP4Qj6SeUyy@!*#F}^aluS7kwfqMHdr>^s@ zWDd8HW@`-FiWt}a*94cbA8krA;ES1+&~s7%Uv`OA4Oknb$o!WRW4N8ye*RpStyeaH z$-;c|4^aFO-apd1O8?oo?|!xI|5V#fe1u>uNxwzn)2SRy)AUQLaasmKN;QdEVfO4_ zV^lCj%cSg6sahdZq_cLFyJQXZM;jMul*SjgjIwec<*uCDdE=34no@25J)<7doS(St z4LUy`^&w><+!!!74z}l*8^}F01@AbNG6{7x6A>}t?(MYQ)YVeiV8{5F1?xCmW@t^t zyFJ+Ey47B^zm|lpCt)A%NNpgrmRTJmnr0-G*M}QqjeMRxE-m6KFqBoE`ecHlceH3k zv&QCG%7Rql2fu9oj*I6EZM(rwWS^svqHXEuQ~m%E!f4~>W<+y>g)gQ}#9uHa)aW;< z)I_msf%{0>bl;HOsrY~qrR*>0D_UDWR${n#W&*aCB>{oOT$O1%vyd48$q}MUw;!WcJsBe=+nI}%O?aHS+LKi%TBF0 zcrvP^YvVP|#q!Qld5KQWLcMf_MXnjV?31HyeK#V$3NA*1$jZYDef1_-kv53=SCx%i zdR5{IF(rIXRi0%r%B8EdF#^Q7X#>bGG4h#v&pN$9r?iMjUVAPN$j7} zg=`+&T~Z9ERUqm*Tj8u*zQ0tqFU?_pK3?aZb;&#Q6iGPpqL{BFuc6UF`fg3nuq-Y^ z?K@a8_5Pg}&Jg2jbr4pyrYX1(5T}(1c_Ca_N@=W?Ib-LMwKJ{ZB-HP~fHt-AFy_uf z^Qs(|f{dk-l=V(&(#lw!q~#~NYUE3E!#l5C?xyTMx6u!ls0m-b4-ifg_+O%pYns~H zBZC&lPQ#cHjh~7vN3+t`Mmu=+L{n4w*KEi&_{<*_(ugHd)@jzs8dEoa^Or>7Xe0b; zqDoEo)vN0pQhL}eztr1@v%W6BdryyqIp}#3cSKM2oiNU?ARjfd^|(tt#Le;292-5O z{D!6phi8oyG1yk`RPFCSO>~T=(i40V*!Ms?`g_-4=AYI@|9l>bEUn0{+K4>fuf$Gc zb;@=~)mg|~!aP{DuCoj?K-i{~3;}KD@iCYXR%2^Vr&Gh1A^onJZ>vE!S#PEM@C9O) zFOe_=%B%bG^YES4=rgm_sr>&D*ncVWD13#<9k8bZ)&g=O+wLnV$(OGk;oF74xHzx`)U>~D7NtbqS5IQ0KrAr}4a3{v>Ym(r^MM!-X*V0&BqOlT?jGD#OeCEQ>* z*Ol_iz<@StToN2fYezw6v#(js-;Dx7<$=D0e+2WP?}xy-mh+bc?*6v)Ic!b4pb|Aq9?gNt)0b!%Dz@-&c&A_CCP;oub?ben z4dLd8>B`{w(vJ&s>C~iejjzHNDHD}E&qitL*1U2!bJf3#Nt0T{foLG5nSdj5n%iKI z_gG$}FGrzFk0wDv=EQnUp?rqpZ+&$=~(Ot{OUB5=FY<61Xk;M!R2 z>OEOH%QJepaH6oO;)6Kn5BX+c^ytX3y}HkKO*gOm;w2TbSi?Q)-z0pjQcky0$uwAs zQ)zDv344PpciEb9n!=9J`V6J-WBF{EPMqb-=+g=A7u>tqO@}7TUq^M`wE=9rx*y<0tcco{YKl??@_bMDTO}fSR^BN09Zy$tA-NF#r za{K2~`^H%9e7JV$f(Xp2#WbQ!)eDrIoFi2N-PTsC&)o*J&L?@BCRe0rizz3;NO0Q; zK(q15v#Ayz45_+Tg+s|S>gdSh_$tueOUsLTfHiFA-MdNiL|imR&%%ZI5PH^Om}WMr z!7Qa~Hha59>>F4zn$~q|cv{%j(S-EN-3Rg&H@MrX=ij~BuY7{l0+5$FqogTq3bHq| zxEdP4Eu9K=I%Q2wvF@;BJif(-(+-w5TzdV}0K=azj&r{i**q)@LkeEtcmgo#Rb}kQ0p0D4&)ZYPS^n8rQ7Z zerF%ymOZAN+#>Zz>b9+GJEH=8i<>Ghg5s1$-JW0tPhRqk7cY#&=T^BCS;LIYWNdv*K}i&V!n^2?Zzvn9_i<+40U`zZPgnCUX65o zgjky*k|Lx^hM_+%Op+>Z(9-UORgykD-93iMU9gw*Ijs^EI*YBewNnPjdT!$%n8W^& zx$i(t^;%BM?05{=J2RQq{0u^|L970ch|7f3PqXd6E3Eu0i=F*z%&~o=^;5u13J?gh z=CTUhd8^^aK(b?_rj&7_3QCD0F>IJ9F;6r@yzM4_6!&R+mOlVP{O+M77R`I5eUbzW zK8zZSS4nr(KTxl2q$m3&tDdM&OxjzV3B0RPKML zT2*#wP`T*t?87c*_{qeTEHZ1$+rQ$32mGj3wNRqhDv8SczLJu_+oyimCFyl+Smetr zRx$6oRX7wjdd&XpXUBfQ{R4D(#<$7tDB%^<>C&G-rgZ7-1J_n>h_ew3Nhzb#o7tcLaQ}d>@99I5UL)lPhudPd=T0(zX4wVewxSm1wu6OCXejOsDt& zK&r+!dPF?>+^mA5B~wtDa_Tv%y7W^OYIDcoj_=BBMhj2TEr=!^UQEJfC@AjLvp-ZQQL6UwShs8$kR^4Rv=QS$!?xeeDv_y=5`hHAEF5)!G zmu2aji!WF4q`k~@XIcOfUKg|6EC|PpkR=HNp5+P&tLEm!byGzP zzL{Ygx3=v1b(^Xv#`Vd0xZ`lb=7gAe&Ev%a#hCDQLtL;oyh?|t5H={BMX@#l@-2EE z#BOZl_1!HanHa~CcXrp*6Gul?U(#vIY>w6po;aDkY3miD=w!me8ehH2vxrzSBP75w z5er}h*7M~#NUW($%8bDafj36bBK})3vlkUH9_tZF^9u~I?k4grCvd`RMh4;yX@b#b zqdUtEQqM*K=>ZvQF|l-4^2v;MpE1~wx*Sj|DRsrvmh6s%*icXE&Xe8^=(DIki^KMB zH593**to48l8xV&J-v|}BW)d`5zydkPbc-0iLMh)4FV2RHGi4lgo&!7LkTtTa(~I& zh5c}PUI(fbvrx}N14(HJ=0-cMpp)Kr59@&rY{xXO#V?mmW6Y@cMGy>CL{74vDQ!r{ zzRsA@!M%@FRj{yLSVxb^tXwsbYc0opQ?zAY=!GyUs6DjfTs>$xZ^&0-xjSj^yz@ee zm(Fqg-cY@|r6N(+bFIy&4^R1TOqAF_ad_O|RFL_0iW;~>4KlYR)s`nS()vvjp$djS z;{N&&)w- zo2gii99yMxy;|uA9#^e3?*(&5NXx0Bzr007wG5*GV*b~>NYPh$O~EHT1nr($lbZk> zhkV3XIG?ifB$ZU!p}x25HFM3q8B#*~5|k%GyMrE^+VoXE>SS3T5femPW@wvX9y6!( zYADBfc7`cYuq|)(cKk{JSc;2j5x%qTh1k~~VGL^XIQ6F=T1x;+IPFAQG;vNxInS6S z?B8tzb80e~2jDz?o+V>+3Z;X=@h*FzdZ)n04-g$J))$c%<_J^GQ-ykF&)LM1^_?u? ztm?cSr(kD!64h!E;HaIhwB~EpJkOAa?2P7v-Oof>=9Y~d<1~tTUC%q5bg@=)e^S{I zTI2A&pa$VSq#%G2GB{fjeQ3|7RyT2Bu;rXE7ZawXq5v82B=M~ z!9=%D{6?!{!>Ki5VDd{grRGCbtfLON-p1tT?gO3@SkT7?8~&DDG= z6D_wgO`OYD-1qJplY>v-Ly}&Ll)gH5zxi4gwU;Q67=$MEZ12X+858h7Zn9JU_DmM1 z;(o6=*@&+uiiI!6*DMEh(RyB7omjkC{blpOuR%Rlr>eH{ZCffe6}?nRj2Ks;Cu1O< z3;S~&H>W(uJwUL!_}W@KovK%-{&c4pHGfUn6?+ow;Iw!e3WF}d@v5^Oio-AGtLLX} z%M7Io-{r&@t}zQG>k+-yqM>4u#jMcS<#qRySUesKxMXu(!hwe+oxiEBUfbews_4$$ zA>kgrXEBAtKX>Z_&i?$Xw)_5M^|luD*8C(SVg5q3)uTo)LDesNYPOjl=ANLERi#ZT zdWc@Duj`bI7UY`75DKsOf&v&2wHkyX2bWTj#$k?C-zUrs<5UtJ=Jg7gV};C5tZwQw zRnDeIs=4izI=#I}xGs8ozCAxMAKgHiPI;B*R5j@<=2HM6KUE&Olm&F)p3-bOevE4^ zv>fRoCB8c3&1|m5qXO!pDsM@IdshEfM85db& z69#`W)B5wzMII{)Wm8osv=b*Blus*kdKzBlYv8N7c9l%;==iDbLUdki^#=%C)7QP3 z_F=(B0YRc%i_%{z4K~`#I!!WVZya?pg)9-mc>G>EK4IrxcxN0%(rO+j6yY4hIn|U7 zo_11wog1=If41v)^}_WU<0?BgQY1~IKfTU2SGwHZ-aLP5>-pY|r_qE-cME72 zdkUfl?hou{3(w-5yc^D`N3v!tn=smV`qO-yH;Lyd{W#h>+15^{DjSek`0TND1Fit= zs3^4JQ?b`#1|iC)eU2k=W7}JMQ-$??_Oz|PQTJ(HT()FF*LSq!K7*c>7waPNi}RCu zTn-xgO-Q{)2S2AGlp_>R)&A3_rtOo?8(+8sZN~32qsvyVzkHE^FPCrh8tLs{eF`JbfDOriZX}a$&8QfR+=Uq~z=x(F0Yfu4%3ZJTvf*r3A4hm7F{mckyjmDypx$s#TYYYdh)M&v*~G6~5#GGDe_k zOitTQ(hRWJ_-r=HsXA9DiK--7b+F;mV_v=lC4lrXMZv9E8v!0_R6YHx3)Dhm$^nO3 z+sR=iZvip;%OKmcX$F2y(yQeSeKVpa`tPV{Upk+qXx@1<&lr)BmBv?mI6WAOS?Ei0 z#{8V3D}mVChBCLL8K5mi2+XtvCjjJOdljf!C;e^JOX7OWm8V10$=Bgf^lJHob<~2p zD+JssM*VSuS2*&%JDH`k%Q&)fF%Fd1$JM|I09wCfrf*=MJlsbHRg6fmj?!mP9RkPR*-SjJenj&DI{XO4Nkbv=WES1 z(=An;p2vG7-c1#*y!hooSz3hLe!R(mpqjT#G)%;GY5umIG)?)~-Vsh@S4+F!^EHV` zPd(hC*F+r;aO-H2IUyoMy2ci{2m99wD#Kclv4b^d8SK8~$nr6mib!;FTfeGtf|RJO zhzU-csJ-Rq>6rlRqzS_ID$-R$*?4`XH8#p2gmtHh)pV;V2mufB`+tnwD;HvK*zj6WF?XEM`4K?9FhW9(pJ!7TWP%R8ZW;UW?9^x)jw!L|u zX~u2|j}UQ(=yCFs1ZyW?+O4;i0@xlzfL%KND7;2`{x>#q==xB#Uuw^VQkt-<*W@AC zN)q;>)%yO7K*Un+qxvt?ht+A*hx_MrZ|#OWeSUy6zL+XyUGkw`nIq+!#i7yMJeFnP zR$&*gMi4UzxChZoL)1LAp{S%8WOC&-*a8m)o>z2vw~6cmdXhBVyA7AN*GJ446a@eF+fAg<1Xa2rced)XoiC1m&8vlbX zvJC`4c9c0^WxIa;w!+vUO@#ry367LkLqKEPq%5BEiElJ76$Lw(11u?~y!;Ok_? z3QompX(f9OR~ZsmdRCrQJ@ZSpXMDWc4%34 zzDIT{jYaFWsrrC{8j`(4pSLK#GZlA7;iu7>rxx(2#gHdArbQyT=85YnmELkc z*VkUhi<#n+0{X#S)hcgsU#X8h z6hc?W(r8KCuX9Y+1oA@r`|Y_8|hN~*?C@{(9o6$744LhBOMoyMBRgvmV-96y!p2)6giRswP| zu4v3ET~p1V(mDizksR8V&;y2Bc1G|vE;tSAk;O9Q z12Su@<;&6fE&lZMoMZE?i8bK>W*ukpS>5Q1Vzw%24pH}PYdzIEnB;k93~n=%wg~jFP7p|P5>Ov$F3pt$ zBlcUHMB-G6rM56%?dGb2w^JXc&kh)I=-na|&+vc1jE=^5B6^ZkoisE#Ef+8Q;&c4n z05UR6QuZCt0${Prg9=)rV^E2h8aDCKMX-k3FWT_>3OV4lB~@tn8)>e4NtF+r> zJexq1{f<8Iw*9l}Vn%2>_0UnYSJ6>(@wVRi=tL7^!-#*c4-R~CHy38GXIX;}hfTiL zcgRUxqnpoidyO$_EsS=M(aMa}2$!_ZVjzXY9BkP71Gd4E4XUYO#e8f@TL&VwZ%rjU z<0>^v7QTJ1TPZ+uK6dzn+-t+B#_yw z#O>nO5Au~o#WeyVF3aHser=uKDuzdCj!-{824=YGy3tPbvubalGE=&f=@A zHN?I?A9!Ev`*gh2&E$D++c@z?HFA1pGcJth=sRQDW#O&2hRq+sRW0A~-&RTr+V^{$ zAEBZPD)0Wf<+myKdEEwsP11V^{hPuc*G~KmPNM&paPH&!zep%Pz$*vbI&&_e@g*U3 z+YG34jm|Ub4V&NaK^T9Qc>dplaegn6`L6}i8H-)wsyhOvwJpz+;%a+)!(2JVmtDcH znimeZo`0#DrgE1ec{1YAQ&i(|&=UNGR5(nBBtTuzYpLlPEWvP#Zfp<}C5CJQgcJ#a z-5eD1#xB*|Gr@NCu<^3e$sO26Q0dg|x7dfYXU=yk;s=Ic6IBq6c5kt0Uop$2{#z>! z5>Mor)r2)Lc5!v~VjdL1f<|5L)4G}oT@s*fF7v(eq}Gw9ZY)oJG!+g@O5O`D}SazLiV1ggw**wYccVr3PqPGO3Nh)a(ntN2_xPr&84Fd|_k|S9~4; zW2j4H!&>S0l4&;!pudA;Ynz6LNz+z*t9n-zEl$IgYGP$ydlOc@3aeYPf8yKD`5lt# z=-e8GRO*0NOvH#VEnbPWo>J=$gyz}U=T@h=iX7)g=dLaF`#Cu{4zNgoZ6d4}&<>A9 zv!B-pcHQ4F`nKg_Z0KC98KaOYsz}DB&2~?q17b&6zI6#jKwQz=ADFrc}|dG6bT(Ti@8f)#06v6a84V!X5#hl1^7u|JL!+JcrJ%*PHY6)N5M-jv0@pQGZOOt{J59icIBtU?~e? z_tV>~?Uc9c5x@+$t4d=JCgLu(U#Ik}C4Mf@aaoRGOS^Z+zSZg~;nLG~A%oa9g}&+; zK^Tc9Mr$(eA@;5?%T84U5~+(_pfo|_fMBWYo3 zb(T+}KwGYh(0SV6;pJ}(Ve@y5O{N{r(Vz`%gLwOI4{+wo-2 z+--`3P0=_ryIo5QYqh&L!E@2x@_3<7xpl?Lo`5N-N@d_EoMqQVDZLT3#u0e6cIOvV zuR1Yo-U`w_KXo^7F-dBp9pV@9mHzc{5g+aKfHe1nKO-clV_X~>di-jcMpdLSQZMCL zuVrrv4MP}xR%z;vI|j!c`S@l~O8+1U*9)B7u4H;CeDmD+q+41NrKmwSbt(0& z@=6;#>X=zxPhwznHeO)s@2~ovb%yM&Y~X6yOU0p1MjrZ-ERyBXey8mJ^gEH{%^-* z^Smt=!Fh|eOTvP2e1zmIy}0`nh%DB);Oe* z1JV|o;5TThy8y6b@B{n<^wNaEF5}vF^>P{TU)|az{Z;gh1{dnx5g5L&6KH-4$+E##Ef@=`8ca&tb1ztq#QkrWV4c#6<$R&d1QO^PcTOEH8os>m@X>RMmZY;> zc3_nf+}c~$NSRvVYOrrm1VqAWU#~Jpb(Or|!nS;>6Q?B7_iZSa8U+QQ?w;v)VBcW+ zd6E?38IinhHXntUiX*@_<@0nU`I4ia(LITk@r-Pg(?-O;b`$%@sytXN=_J=q$?z%@p9xHH(xS6%+}5xW(P6|Q9ZmFo(+qXJEac%w~jU2|xp zh}RsSXqxso^vOG|#aYw0FO|3!G~B0n)~_zzUN42AxyN=zFN*bw`*DV!R4nLW+_7Q2fjkB@`4(YEP=*VehHn18ZvZUV*2hnHE+qcBtNPbDiinK{Ib0^ z!&%O^@c61-_m$qB1sBQS#c?^yRAOChvRw6~vS2GdL6UQUIb15pM5}f@d-g-%!>iCY zuzMW#W@wADd!vi)JfL}nMICh-TkV9N0VY{2i5Fx!=9$m@rL90t;rK!o#)6h6xYKzGMWRYIZuyR1U}oP;`;p=Zv1Kq29&2u=VmAv&P(> za0g{VYjwvnCigQZ?i8_K5W_&ciqxD`0-0ZER{U`BVwtm5ctq17h(Y=4agj^j`+H9$ z?JFHiltA1oPEL+8gN01L+za^R4Zw#1_|<>_c&V(Atu{l^w&Sulj62HBEPD&~!DV3a zWlOWJ5PM)(j@1j>z$qYz?Pg?vHLlS1N=d2ZaG_Z2ga!jDaUKHo12GGGp zAS-6>u3yWLT@HWZ^~0E2d%(+`Y6ss2=3=-jTAn*UPu|+i%{p(v+7U=z*4R`|5y&0) zM9Cn6TVSAqx21b7_v1ECXEIohM9%+>q2;r&N-vkZ;>o5c*KN_=&$Sx2$*3M7_Y-3| zqsnGO*Zj8t8%-P~gK&rIdbX9|UYltfu(=M!e7AQOemE2eq-dMVGFb}svwJ@}eS4{t zz}g5GvBS(TnxrSN9PJwN`FY>L+`!?$H`5H#F{zvE1) z3ypfdwP~*&GQ2v^aMn=K&B8v{kZ*D%ssbTEsOd@TO~5vYjs z9jE?DHsgGDRZaZi4I1_b&$`BG{h!s|s(Z4xN>rMK^6hD*MJwjFr$ha{2&`dBpR&~{ z^%ZTM_^02+;;$wSeHMC2Ka#Dz!9RykGK;dQehKjdidze zUeFu%nwTNKC*fXiOd~B~Z{Ft^QEs=j@@#j>L`7(hh|(Vos`8#gMAPIw@RH&J`zkx{ zDjN>fsGQW1D8T`3c*f*tHDrWe%|;aTtsvz#!<{JF{1VaXk?P=EU;BJ_w%PARty)=_ zhXn)Z@D7yTaG`olU^;=-A&gW`kaVNXw^3yYR`seHd_aw72cOavzpi}T-4gDwUA!tS zQ#cEQZNh4D(;w0%CCRjvr`&bmCyy@sa;VG~O)R%hUX&{3J~!^X|GwqswCIh!nCBB! zvyA=xJoV$doPGk9$*T7*6X&D_S$WIZBTEX}mE>6&07K&#p>3CjgBjqm0|ziwzjcJQ z#deZw-wwSS`hvpZMNB_mt0V6>MxwA@|LV(h=Vsit2HN5kHv#Th@btdJat%04kKgVy zty=qPG~7z5hk)Y4^VeO4=2sLhKCqA+(X&_k@znEK7th)D$3ICfOh{6Gc;Hol|FZ4k zD=xq5u(T|R_0);jRD*h;v!NJ760>*Gu})hptXGT_kg4*tXv}Q;+~pT%ZsO>FrUSU) zNsbzKUYWOjRE8+=zQZpnz&oqm&=ifNftaOOr&;-A$R%wdBE=$$gk0|6@4J@p$s@!* zrP<2Ojv6f(8Hs&@zBxsyc?oLNw+0=?r z9~HlSV((c$9K?JF%)qV1klVF1qmyofAlnPzj-Z)d@!qq{Xt5hQ%8Ys{N32UD1jC-> zQ%!Ce(9HI?Mm$Jjs1F&#W`a0X4`Q~f#w8xvEpc(%N5l+A=MB4d)O$jg)tdcGzWydg zL;w)sS!h`(R+_i1o~754nw^b~lC(aFrWT~;mdJc5u$Ga?Wf^1hzOlsP<$qqx`$zeM zACLboZ}Be`Lj9H5Y=~H*7vjSyZWNsQz#p)SsPf=a~QU z_CY#l{tD}K12lI@$!~=v6lU4&Kp|6;X`O3LmW>q)oY&*(vft$!;5lRp#B*69<3Hcq zeU1@E>ub2@<%E5^jRjJ?Jn+*r{sOXI@3qc1- zKIKXFLNgv}rDxh4JAE)xGgCVV*d8^@`YVEv$yyS@4x`A6C8! zRQrW^Z8sUS9_nDQ3zRjdm?Ik_BF^weGTQa}&N|5P{_NH7fP2g)+7i3dxA`qO96aO_ z*4vwSBtZ~2bkw&pv}Fq_0O`BLT)!OBI3(4H5v-hEZy1MZfj(UPd-gFQC!HO@VC4CjcHM}-L**{#d@DkXp9o+&)s#sBWUl!ZxuX1cYj!Zta-4l zwUP|YrXSFOE!al^J!h~}+yJRQQ>)nlX8tmd^TzPU1)YTTH)p42DBI}T%dnMk3^R(a zQ_>+g&Ru>9R5%+V^B5v_e|HUZCO$+mr7pz6hqggonZa&oSXKiTy{ zU6rh}H!?~%++XN;g_$=W@nUniypt0H~Ch3Qzr`=aN>bknD*qg zHJ)8iM)MEdnN>Z&3^wiP=p-}|y#_vB6gw)RS>wXFl1V0f^(<=}Z)P_y9a znOFNo_+jTFb#aqwG0IDAtV%#lRjNy$Q-4mfGz#{f(-HtW+cIw zx~PSDi|#vsL{!R<7EEm0I5<1Aa+njo03(Rys$hxi=eT7!+)<)(R<55ygmQwn;?F@f zdRSr-MgNEoM9oiiTPzhZ+ zs&4sPlI~6hPulRKX78I)u=|Uelp&T12z=w;W#Rr)NSM)E*Y5y!v@({uc5LaL9?!)w ziw-t>p>i%6e5pY)aSSVwxiFJ0SiJ2wOU-?RcQmY-0%V+RpzD7NT9=hi?6G95vgKZj#2D(|$XJuH?7Bt9rybb2g=MC}1$~Z1h`9@* zp8iJ0`a|O%Zt@!$>kn<8UH)(rz|SxI!5TkJ`Zv$}=`=t4;m?ug-v-|Qf5%eW-)W@0 zbNOLPpjkWjDKMVP02EZz0bT(}tN{N6i1v2{l^-{R|H|KgEg;(ed1;Zs$s0O5MZEyf z#J@2^xyv1ndffx7GpLC5r3a+n$ioa`DL_2{s$&p>FhI>@n%XNG`oe=T9sO z)faE;J@8W@pIRVYg+!Kk<@J3x+ITidr3S+VlqeWJeAf7ed!v)+T!H>VcRpWpVW*@@EiucY zRdDppJ4%ngF>Om8_NEYw$In-jQCLrP?D%~cbRoK?_c85Sy+7RWaj0da@-rg}I^QPC zWirZMco<7(L_Tb0`(0Q0SJ*Iw+J9QppQ%#$8|bEqYhU+7MeS$ZD^TgMfc9a!1*jTk zv5T4I{qg-Kmi)~qw)krb##l9Q-mY+PSbOBY1e1{bRRCE=lOht-k;LM^j|SzcNk2;R zn6Lg2e{L{!_~lN?9!GUJN6Ygt@aH%e0_V2=lk5>sZmV|%+Z5Z{hbYtS2P)YuzFJo7 zR&1#*aeP4PiVic6Fn>ylIMyW}zD{>Ngm0HlNa5xJ%NVBD!t2FSA5L{V-&@K}{i>J+ zp~$wvPJ1n1mo`E@e?D+R-vjHGTj;?uxwf7;Hv~m7a!WB6-4_ zc$Pu8_Ku77$7V`0U8;gXftk@e8T+$Ao9#cdt@#{R-E_q`!^%W}I znXte!$q^3*UnOJjpvht_GmaI$wdEFN$mCku!$hTvWa(6OQ$$?mw@lnIxk67fCd|QC z2qER*z48FVjl1A26$aKF6Css_fDK!GRm*Du{fMlp z2kOBhGqQ{*3z1a1%?8NoX$lc$01Qz$hf(ET#_JvQ-HhS4=L)63Kf_h>7UYX~i7-mW zf~x{_Y&B&H6U61&Bi!`Z>NDg#7^CZ`Ce|0 zex`ICtY0DJ*;5|u!<93_a!EDIxXUklh%3ezhIEO;@HQu9#~rAxL^gKP>nXFM`EwKm zKsgh+Ut{CGF=J#iGm+H!<6AkR!-N_JUZ+kqTFEi_{q(L8b5Gmh7>I4(- z-uF-X=GVF`QjO&LpfB|0WXMT$;+awm2gO>WFhpC*(i-fsUv_TNEUoyrplp#i(sIkcn2w1OGIqdn$|^c@ zfmzbiPZt`i=&7U9oTNbrux}3udJgDW%=2kpyirga^ZhlD8NP5;lprF0(9f`|0BNL;W5F z_T1OLFIFZ6h18#V;!ZjjgW(q0@XPq#?go$QxayuYJ8U3zkR+^HZMm#&3*SvgbZM8d z_xxM1G@~25l`NI4ed&|f`hqL7t!wQ;X>iEc{wB@T4_gz6zvtKJjQ zNsoUShj5lDVVZ%#vW`4eMI6lzwj>HXzf?3dpV}f-h1ca!&oFXWN_1H7cfeIF$$0;S zS4RZ$JK*yAlpKo73!_$=d#lvt(&ck&iX7@rDmFL-{SMgj$2~K~5XfOv&LS|>URd<= z##`(PxSj7i;J}#$k%{*D4tQ7{fw^Y<9S~EvIe-oI7#<3Ki@*@)VwuAMfWQEKhjllBL1V2r{{y%I2qX=v$aw>i4>QvUG zGxRv5*k)ZS09veCz>`3vK+P6!3VqZU;aGG*)bUV>&O zK_^s_hQ`Lxeb9D5J-Y@{{0kPYewf?vAK}&ic1$A%qj1-2`nK26t*zacb3tL~D+JFS zc#ZTxB=%};#h|qt7{tnwpg{ZrD($Czho={MpZwKaKD&dIEP647Qx< z3I{T9&-ukt7H|=NmjG?ZB+w$7`DI$HkS3u>Ysw*1s}2|DR#l{}-;o zd8~d$h}}RMV9_ljnT6HA`&cTAs?3L$svnYE}Fmoz-CF6cqDCNoGw zI5_>TDR&Ct4bUuCFGd*Mb0FL>5*jPn!|P&T4lA(Hjz5_KnOE5G%hOTKf3BoSu%^f^ zpi>a)KA;A83_Xd~T@K4h>`?Nq6ut={z{=C`ty-XhR8yLXy z4PbD7dWZnG^rDz(>_FCC!>CW~^&IiY(lBX5*JYL<2A+^(@hE4q>!@g3dQ<-~h$K`% z{*fD~(DE?0E4)y|g;%%Hf{;iaa1kGTrfi7J?Q*~=4Xk2%XcVFO$xZ}R=ynD%4SI7} zU<@>SwMKpmYNAD%gDwLEyu2Ci)s>VGR;q{Qb6l}a4EKc>)6O{CLiyp*k6NCaT^(Sz zdJfAPnN|JG$j7C9yytm^IRng%o*9!^-6k3 zE3s~i8rfbF(b;esByH&R0v>^9iw$g|pCIwhE_7>yUHw00f2|@XjCX#6m-4kQE1#u_ zhd?5YtYRhpwu`G;X?bwX*mZhm0w%(ve7=2Ap#N_lInj_W)j z1pQg{mmn8sjiRo-9u+q47f#D`T@FDQUZj1hYiY0KMgx57zR&9BGi95TF;mfM+EX%D zwg)6m*D^Wpkse&Xg}eRuk9PO}^mr*7>ckNd(3OP@r1}m}$_BhUrde&JZc;AP)kk>& z5qo(Jk5X7CGsf2@6coj0IILbKzb;D8T;zCE&V9^R-+OgQCu#%3k4k_STgoFK3Blc^ z`)Fkcr-F7%yWMw522=f4%QFifg_GpmzuO4Z>4=EUBbb_)dh^7A)qDxdPdV0dyAf5r zQVi}y*t|xSOm{eKdQwGzU!NeP^*?R8!lNAT@|X3PXlS#SCO)@s18ktc>WI{?eL7nm z7W(Dt$?O`$iJ!A2b6Oso4{U3GwZ7p!n3%D-%{lrJAny^QcUu}+`9^89J8*^6ermEv znrh`?=ElNJL>9wj_S()pgO(!t&xPNlH|V>9h#btXaol*mx0!q%&^=5cp9yU^+1eBt z9F)=Q9GL+RM4d%F%-CX+Z65F3-+%QcMXfhk{zhrsQ7a8v#lvP+w+Ts?oV~@& zKPfF+Lu`8`u-Mcs%m3M(j&-XAuC2#HuLi*UP*Rp|7K$plD=4YXbMEp2o9Ps`iRvh> zxqF+cE<`gRR@%|a>f+sl&%~cD=J1>i0Ak)A!;0?xh>Nc807|q4e4b%iZ;DE_rgDVe znfL;%yR%2Tw*@N9qYqjU&gj`DG%Z(o^@_%yQBL(wO37fUI8<)vMiO#CWqVrmMvuS| zc#CL@fc9NS*?i_yh%-s$z$qP-QFZL+k}qdmA@98d&$eN zo`5b{=BBzyW&pAAgCINPykS|tYtt#4`D@*TNK35JMyb^wsEYiN+Q@%&obUJ9!}))K z>GZEmeg7}aMu0!j_V+Vb0c^oH9hIP(pp0ZrbtuqPj}The;!NUc@0$xJ`pB$I!}YnJ z6iu=mc<-^S)=N28saxFH7w#{g`v9$%E{eM|Fok)t9_jHLLfKM4EHi+j)X8{fGKJzs`}ME zSVL-U`{j(+{^wM*DvN{fMa6f(Vxk->2>^fN;Fm%;T0NUeeJ9SrrP#o5&an8ni@bS4 z;PvZG>RH@^>#v084u@>Im&mR>bS#*ExI84vJ^gHD;-+2)!@+&o^EsyaI55h0mHBw& zvW0AK5W5{W$JQoV{q9+8iwPdKxFrf!ZQJ@c)zVfbvMCGA`gAF*`ZnA(4AkuX3|*ZP zkt1uub6D7L&n$JI|d`^D0Ey&VNvW?!|q;YUxwbH^x8wc-cjQQw!`eO~?x= z0+<}Ds+v$Mi+@#op-{8Fq?BQHrQ9O4Mcj@ANc`~%u*ar2H9%S>w=;KE*ZOVEsI^o} zPt=3Jn%USw?2JLN1|+Z-y2eCm@WZbjJweGe3SOSYd%OS3*rs$k^o%-6q~S_`0)Y+S0YKkBLYzGU+@UhLtSU1WN@Syms%Wc_^zc z9WyLi`oldlcM)E1jV}&)u(IdB0dM}`(SAjM{k$yc$G82bO_FDmq@7S=eYhz(`BBm@ zY%Dfu0WKPUoBCKs8Bf=ZLhX-U|8=GDWrt2aQr=UAGDLV3Bxg&Jbh1dBcJu1G?9)Mp zG@aQLaoq69)?BD%=etL5?^#P~Wk+LcnMgA_uR4Z6vw6)`eBPHE6vPvIzY-qN`VRPm z?s(A3=*}?GWD+knVkj17Yj!mr*=cKQzDWt1+(o0f?t&%=LL3MpRWu}vA1;@6UYZU+ zE}NE?dQs4wp%kU(Q2~>Re@~8w!=Pm<@hru)ZOHkL`U_4-F^Z?Xh1-?~$?W^0peAXy zSaq^U^B2P2dk&hFX%}@HGPfI2-=k@YZANWjJ@0cDZC>3{*~_gC)*F4khhuKR58&V+ zz(OBA{;4t>Gp%=j~k-Fo==0iYf)qWDegA9W4Vl#tBLLxFC;542-h6tKc> zN)BQu@Lm^@w|f!!1{?~kBOT2(Fcb$z|Wz< z#5bc|_kdffc7HtKz<3j8moPi6r|aq%?ufs?zsmH-J**inn9~U%Mzv_*F6@v7qH*h0W?A8wm5s zf|fsD^0lS; zJpd0jT)eyLZVKi10a!Gj8OeMH5WA>WOk^X_Q!ekiza(UL=Wo%S&~C(EBZ&m^_L0QI zU(uu@9HBE&>7Pxw5wOw|4YU8`WxMS0Zp%>nngeTf_!QM^J2r~=Mss2BC>BwLLJGc{ zexY!0nX%vlDr9cQm<`f8$s_wD-5#deQqOfnUZ^jhJI_CKvqDO;mYvE^i~sfAEPTE( z6ie^h1<4Ax3qzi#<7_k3qoO-1`O?BZX3@9VT{%o1^c%L1bG$22dLWhbPlFx*Ol_@% z3T_#+kG`>FXxw#f;0SVl!O&!T_ho4GR_k4D&5;v!Z>a{pC}e6VHinQhyJ!fT+oD>S z)ZSsj8S4{Qg0fegndsfg~QOQBw1Sy^MWhc*lxU!VFca8B92@G8<( zp8cy~f8<)5)>1Ui{*g|Y?IQgPgkSYXW{wfj8`aOp6Er3TjwZr)-)!91iPl+2j(JyS z%r9J~#3muy=!8`7q&;a^+n^e{{@%sRQc9`dVM|*4fx@VJ=W9rP1m{xttLyj=WY~j| zje=n*EUieKkr!jmikjHEc8UdQisGUMzsT6;hWiOppG-|dpR+B42nIss$lRSF%I)X& z%>_^$O;t_p#>VQJaY+1v-Z!ukJ9(eS=x3$eVWf!K+yh`#)JkTBJ_79p6HaSTBTw zCj}g`VCpWnjIvon``Rp9XuZWYO7Rg9;j(PKa6yvI6qyipsI*1yu&W*ai+G;0y7y+W zvyaC>tRLxDC8D(D5*b%JNm~y8T#YXEv)^C$`)Ecf`1dn}2w?VaWSq7se@90DW(2vv zG%@G@r&BDLJjvpG^Ib`dE9qs}c<0H7V5qO)FzxC>1Y^V9>$xPa6PoixLCPKYNDlB=b~{h`ceg`8!|^6tV==EdD#gwkrf1mxbSz=}YxuUDsb z#8p>}7A0#Q(gCrI@TJivy-9+rO)-5v7%`7h#~w25AqL#o;{H|Z&~q?vF(;6Kzf{p7E4<%?UUs4_mzvv{rbx!Z?!;CxJ-D zUS@LX-HXFY9uO@Od(sRd_D=OCELy7Ks^8l)%&L}ShANbNKY#9+0|}iS>|bu@Kb_TC zqc1i6XJh++a&ASI4XS2*6D6pH_$cnH9-|ScR7*bbU!J<`ag4%gMWTjEwuO{=Xy`Z2mFtMq4z~z0 z+a#SJXHs(;$++5+rrHYP7ew<0He(OgMjwT(af0mg@)Nim|!r_ajBlK%T-9X4j+Z|25;SrIi1iq+%WwO9hvc%<5 z+LlyDugvZYn*_SIJ|}KR2QR2jXJL&sy&M|4Z z0@Z8-gPCMOSxxR}R+SGBJq8{asZihvKR6r)C_p za}N&lq9*(JghZ6rp6cw|pPORuym-}Js1~7Jl=P{3;z>xBzP0BgT1UMu2AEqA?0Gn8 zdD`2L-odGNg!F<$x1RsQVET`P>mPl#+&VXmug_Z8NQ!o?N%}wu{+!sev8E9&ymzgb z=oW6b^bJI*;i&Tvqc+kqaN1CvSQnIQeVZwFMvpytP$y}vY3Ay9Y>?*bA{+dbN`c<$ zQN`c1R{w_l`LqAi{7yN&*}f@81511vHht;rbxAmaYv;AHDaW#En_lv+7b6>k#3(f| z0Dg^_|E2lD{--B`WG5n3uPgN?S2a@O64e*fw&e3Y+X+swiQCpOFXVIJuQXh8+p(?Z z)Xs0cZ+rNV?SsApys$D!@VtwwEDH%(h7D|+?%&i$zp`SET2If_F}x9F@|T-l`$ws0 zt=AYHl;5&C{*Z_)N0XS(qFTdbmIE)4#%?ExGZCX``Z!T@4NyJ|fM+0vyHW{Iwa*fi z#_r@qX!r}t6bstz+1+^}Rg*}|SeFqMc@I#L4~V?B`jh~CLMYcPc!|LVbb~W@$i3bf zUtD5?uQWPVX+*{*z87QIp!lX7szCPd&!QFM1TSXB$s=S2%~TZz3fQMt2kmhOgWof< zTCL!fSD3$*=t!S?g;Ns#r=IgSNzH%VJU=h{e|XG)Vv@e;>Rp~r2E{MUCF&=(jH0Cc z$$BR~3K4NQX8lZf0R11`^4Sk=QdspAPw?-4$?}7jG#X*y=)p$Bh?i+1pKR8b_ph?p zL^De96@K%dD}N@^F?CxyuU`XKcI1`g$D4q+tv_7)gM&TaCcY5R=vmYpw*)1F>c*@r zLj8dpk@(sg60AOiqW1sPt%k9T+T!4HRS>WaT&PD}&=|Z{l{LR)eQy1vC((SyfDGe(=k=) z$mTA5FgOIdSG;1Ooiqi^d{h@GoF_;nba4HZyjYc7kP#k|ZCbAE_!s*X#0sP}&V}$r zM?BAv35Us*8hEHeq&%z7E!@ICdhaFTSl|s+6XQx@qFs}ev#^0O(Olpj>{BD zr@Pl<5txFTv$i?RA#?fDeg2#L;7_{Gf6rxDK~y42%$m|GF2{b(%ayqU+v(re90SDu zeZ};x=x@uY-(+vdzW+gF53jEA<>OT+?$Pzx<6ChI>*@Njvz^;3KEv=8clpV z(l06}N^fnkilArw6oTD9w$v`A?(Dk2El-sF*F!X>oEOQq$PH#^fHm;(q4BUUGoxbB zmA%r@IlGH!+ZA-_@zB?^nQmtRC4&G3% z)qSBYz<5W6QMYy`_9;PMD?WzB04nL_e_mV?NJ6@2T@6ypI(Z=;qiJ-iT^U2hv8Mbou{_StN;LF zg8^rSaM{*e^n($Ro`^?NrS4L>TT}bRGpbp}v#L(!BcHLPpbu{*P-I~=`=Scbg?&3v zL1MIMNY{DkhaKsM%iHbCgy62RnXikALd0&s-0BNXom=P}M~+VFUwG3W0CHKZpFLnn zdAAqj!s1tC4Cj7du&wbGCGs#kR!{m;3`0;yIV2^_2X5?LHHE8YQq&22^u-oVkyV3J z0~kKSrW!MB-udMQzY(|slKLds<91?gw@V_V|Cm+h9=S>sN%GUPu)|ey*+?XdcN5LZ zwX`~m%GT%JiH=6NkQgmnFAvX9NfVxI>XPwwE7}w$h@8r}Xz>f@LOp3Y zx4dqFPGMpxUu#>g#<|Wl6VCEP3ZUA?Tg(TnQE$5op`4_k!;dD=#+EL5HGa!sV%Rn1 zwu`St$Gj26FEG7HdyWr1*Eg9JN&E2vOAD?3lQ7O7h5p-+FB6@MQDD8%B6A9Z#Tl%K z*Xk1AAS3(73o7u@Q-a=BToa^dKwy(p?t>H#-QqX*S&`KGV+P6=_~~?!Wdf1E5Z19j z3o+<0g{eycj|RO6v!_~-QYuYjoRVUtxnE(5)K+ZaaKQ=?b(F%;hb6IvIt?X9qg`yA z&KD5@MB0}k1yzvTxvm8Zx+t#)_2Y)eGF(+Y(Q~Tyqiqzu`>R@hp<_Ky9_`G8CY_)E z$?k%!U;X!P_WLNY=(h9kA_v}#*m6G>AQt+{VMyvU^|vhIZ&*Qs?|)?$jsNNSF8TFK zIs`q1VN*WzCGWj@DepOcJwmc=VyF!z}ST`hT8yv>@{& zgpzsG^L(JG#r11aI@EgZPo|`4F~Ti~^1c~nvN5Zj=!(jzcLOs8_r$c=;fS4fra%t+Hma?XALXcG&)yOle$ zy(k~_O|#{NRu#&kVbu8z-AIU6)wN=g=2W}LVD|8WPhm0BV5Bx{l zQn91<&Z>`$i#rGoO1P@YXAiW?p-Zldys#F3yrMvgKJ2 zV|WMSxYx2eis{HLj0i5Hur&qO%^mtuoHwURJwj;~eQmwY^~H;r&uaDiNnb}inV?dg zJRk_2`T>hl_@0!dFo@&5`zBZa+o-R72Mv6}eR`FPF_C0MatyMp+vn2yxgT~!8i68< zq>6A-HGv18JCtL;_$38yiDu`- zc{t?;u1`Qxj<*DmI?jd;%V<`E2_i4Kq&Q_##P#)MI@o12~YE-6qZRleqOyO?X9HSol zVdfXL-T@yqKz|>>wNmII96!D^11fX~np*6XWeLz}ZEe+gmRFbh26~dY zOCleM(ZXtCA4x*WDuSilf_6E?eCcbWAgiGuvH>|4R`Zk9vF%nuS!Ps%9%f-mq&I#e z`K?KU4tXaBjC@2oQklJLp90rRh>`%X5)3#D6NkHW*?xL`yJWYjYQ#ZWIS=)gJm_A7 zc8xeO>0k_ZM|L_P()(KftChWQ#wVysB%cC8++X0BIN3wT5J|IFSg|F;@w7z+V#jI0 zwp)~l2==h<< z-y|e41;ap0u?+9xTsU^7_Bo?+ZUpvO8^rgl^Q`j(Z~b`!taU@ar$`;2Z>U<)o@!S( z-wMWB& z%1k1aC6d#j*D2H}%_Vg_qy9Xon>sYdinPen`Sa;*2Rq&SKJz?LQ-mPE6GuEF0(va~ zZhH%!yArbiDz&tEKV-Msy1JQBZ6ae_;FxgFr0i{7lTMAl#C$A}gxLNL(XmtAl2GvZ zNg1T+UfRs|Df6xBO5JKuqm|uiB9Jky)|E3%?3-2RlW(k7oP1OOq+7oMC;p>y!~g0r zzWbO94^^uvr^}P`BtYM!*CaOWWGHu{UDPn~Vj;-m!>85Wn=U{{UdGu#_haA*Adkb5 zR$}fNHO`SyKAed|v7%-|R1Y)c;3899W!Hxua$)Z#BFBH;6kd2}az)9}pdFRFghO3P zYAUWGSSm6j#u=`?ukH`rGh=)HvG=B0*U<_f`hoOxsJ*iw6Us>wVbK!gk-H`Stahek zkDbNL{{VM=wr`A&?p{$}7I`8^ndmnL zPpD6}w@1UFyFJSJMV2)~OJyD?m%Gy8ByY?X6p2)z=L1Tf>G`nsGMlH9ACrdLvVc+G zLYtEXrV4N}lzj{>mA^3_shcWx&S(|2Vwgs4%oWtHIgVQ*{jDqYs=F=xww);BSAP3D z_9D05$SBM>H*0?x#j709(4w`y)&Qi`K4gt$WuiHI$;$E+8t@!K^cH{vBRX*H55<+hfNtARQ35}q(XAV`QUoOM6$As;tf0$ZyU z){~oQ45jRSG+|=BYdH>YNaEM`F|(S4J>+QVoaPiZo0v}6@*+UH+ldMiZ^l@cW} zpHxrmai=&pRM|>sESTAZQ1QQCd0;k&v~#b$pGIcILWL%B*IyuDNW*;9yNWNzYkW)1 z?C_DS=P=LE0mebl6YW$}ZwOu)?i^$xg$8uJbDnERLmbD}?vsd(`={J$F+>O}(drH^ zKgrV<`C{nf^XMJRhZa_pTUUc$Sis5ilp9z#I@WZOQsp{l^fp}96(c<0x`<9}d4A?z z>8%{OnTjLJX*^h%N^dh_0Lq8z=pfJukbGm~ewi;BCfy@G%e0KTimjXI6I@Jal1DS1DLTMcQlKA~uKgTM+n7V`!mRDOsCILG$sQ>f0ip80a&@ zt#fKGJR|ae?&)8<8AT{5gFOxBY6si6L4y9CVw2K9+ZAXJt_xRs63Hs5)#b zi!@lOaf)S9P`9zgg~Q1^zB4Iw^VzGq`Rej%8B*utHg^p~PyUJ^z}D!-CjzHm zDXH!i;=M(VLP0F~i^{CazWlD(*?Wp+(MJ>*A z9Y!Z)<#HUh_GjzB7H{`B{WavO_O!6-?MjmbBDQd`Sm`HU-&l~;afb3yyf3vfrfE{Q z&9<3tcpG9?gtFBPd0OrxqqGj6+MwsIjpKt>J( zsXi+TgUL2JdC*k$jrPhG3sfq_uZ@D(Ij>6HRrm~ooCN}y?5mNf%LxI+Rf9vC28y+l zyLgvP<%%N6WCa{LN)Z+Kt#IXGHm%_s!!!Y|NRaVV^^emK>uV5YY%6c`E6?? zO^bFkp~+=e_nOtT>vf9fLTeSTq=~y0=#c@Pq%CwH~=;aJr z{{>o74&r$H8}8Y5eD)pH?@9xtNs$lR4iyWbl1ySd+xkjz5HH(&gF%NA(5@^Q$Ct1) z{};R!5S`GS`wt#2eFVItEk=g4VQ!R-l{CF@6bo?7hlNx^WSWLWtK11}qbMc(D8i?r z26DWD8*Ec=qBO>2G{9lOH&6vdU$AX;P`VNI-U3hP+H3qhmG<69XzucU*I90T z>qyB_Gqza7c!X`_7+8(V)zE_!89MoeJ^1!R+>;>TYqzdb2~JGORwI0S%b0XJ7ij9n zvqp{^Khf^lhRo#9IWlnxn7+}S;?X(Om{f6M{(INUI5UibuYb`gzzI%x4` zno{+e>7!Rg+(7(pem$BvrHABN-uae@R{dIEHKq7zkKB;brU$FShY%;OL3ZEDODnxt za!(^d5+&jGMKm30q=3 z%FJ!)(`y;|?xMH(>3W+Px47zJvk-Ih2B7K^eISGIw>)iYV#@1UN_cKsf2zQT=2MYP zjXrIZM~RL48q9hG3t)tP1*UaCB@L(B`tko$Yx(Rx)d#?~2jW}FNw zY?o$QatI8A3zFJ+$|b9lLxt5-1}44^@t1l@Z{JkfvWUOcB@r%k_3@bdmql5?aPyz% z4pYSPhyCrJ-tg({maS4{>VECS2)90vp?AWmdFC0=zRx#J1`!=4vFck`Mf|=iT)n-RxGeU z`-fTlKbq1f&<`_$`F6-6i2ewk{huBGA5i5p%$>2s=6CwNumUKkVECTg1Q4@{(b^E@ zxa|?|b%stjz)GzIX8tfgnu>=@Q#UB(!)cNDDI(33djE^P_l|0^+t$T{0v14u^xhGv z(h&%v(xvyVAT$(ltToq~b3U^?ODzq;Z?4jYA!gO?n~AQyX-pl3A2bmTMt{^S{h|?MfBmLE zB1wM@{VVvptYY_7sR{e#?G>8{M?78J$>m|uuv1AADZB~askX787|PTmfx?k zsSS7pj2@c~)D8iIgHNzjOP z;74lz?;<0H)T)?%zcHr%R>=N;bL`GtoJXxRN*d!QR5aR?_Rce78@GCOp2drZh+kQO zeqFf#dg3)*Bvh0}ooAaY^%}xDRLVj(Yiny`OoPZx)km14)*fc&v3&Xfq7YXFyx4zu z^Eh4;pyCiaD|hE4{k=7E+~c9Q@6+&I;IvUl+&t&LbUH9HpI=XXkG*~2zL$Ze*%(3i zObkmcp>^O9$TJoTB>T(F9bEx1K}p751FTI*64x+L&hN&0+!I%7+7GaHgs!QP(Gkv2 z9PNfkmZ0IEAk`XJZGtF8j1gn}l>r2T|FIo^3*`T1a)mGz)d^eYLY?M` zTt$d;_oh=Ig$VU!GDzij4}`V(n~UE1AnU3B`wa+QS)@jBv{E{I_C%qja#42e*3Sv& zY;@ktv>)+RhD>gltFLQ?s5<$pJb}ncA@99iN#&+->Yz+5Da|#+?Q*i@GnAq~(PQGw zMF?Qn|Dwl!IqeVs$X)o|kl8FM_<6%Lk2gQmi#WL?6_<>Rw0|byft_Lz_rQ!{=dy}V z?wCKGVI-d zG#8aM5J(JxTz{F}gI|x-KRfN3_GtMGYtQb)<*TO`B7!uwgPdQ?f0 z#q8cLvsCeRBEI$fE%nm1Ru(uX7d}7x-O!NPjm)bt@B4E4%Ym0}UB*EBj^7^Xk&J-6~VpWpD3XWaqMyZJRH6w3Ue@>A)qBkl=VSBfzzyCHlGl3|!zZlQ&L{zPh zy1b~I*ZB>Fp0d=f4^lKzd9NBPYty`R*`nl&6^eR`aAXEEK8jH1Ka`@_zGrnq@^_yE zqps+}R1x22isIEv3b|W=EzgVq5n~wqW4Yr<23q^u2QL8asu04VRU-t|Y1c-*;|C1& zye-*j8baeep1XPfbvMBSlIUy2b6LQ7*Gv>naP`%JZ${yRrNbn4$-E=a)SM=xL~h1u zTmAN_zqMxhRX%?C#Q*)VEbN@zxdlV()3ZA)Ltg+?z2Lr_c{nFgeLy5rq7o$f`&{`rzF8D*> z_lNrT=e?w<)M|lj)ZSZmg`>CXieD*ov#vI&x`$YfZ+O=eB+3d?o07g4{IHe4N9xPn z^xue};ZIO_qtB8_@aczbS}`PP>hdfk_;4jrts!ib26e8LV~%`X8@D$)uG%=RRB7)ScB>hYv3@38Mg$JR)9(uU z-5*$rWb005ISBcZpvyc?KLO2dl7GppNM%qI^rTk`H^(uX`uNMe+5hEfaimNEkvgW1 z8lCFL`!x_Vhumin2glf6lFXn}Gkd#Tu{##(Azo!&uD{)Dj!)mfj3}xu9amQ{_0%4W z;sl2&-=K)6aloSDoYlNX^9XEsF>U=@k^3jo_|JwW$pyK<8;7eZHMR2GP@(hk`~g;E zeD7Y&6P#Z}{a-di2G}}Vg8AI5P*~y`IW?jFoX+K57|pE|l0Pod!?z^eas{VmjrA6a zqAkUpiqyrG&0e;>j@WY+8KzLdq?OuvLiii?WK=M9mf!~EgR4+?FFYg8Ab%9~1mnSX zab8Sn+Bed_H<>p0ynRvkNlO;!5?3UH#x>=K3iWAB2MuxjXK`hCg#D5l)@LI&BP<$z zg6^RbrI2=?1WWNur1QQfS^6v8nM;FK6=M#hYsOHh)Cyqu{V$8fKM~~r-5~}BDGQOC z`9OD;-yCU~^t|ue)rDSPhZA|hPwT1k+uAmFbbf!Es@r&`rDSf^cn(*(kp!i`xG zJ@$%=bK?Pz{^5RPNrRjyVfg_f)We4sFJ4uz?hBu3YHK-D9$bX`UF-lMB0;C)zo$`6 z{=Wzz8NW03ak$P@Ia4_-mCC0;B_TQFkUIXpxM(M!y<&*v1X^rWlF+?F9*GxQeNbV@ zo$o*sl}l8ovN z&@8Angvzq<5oahxt<}eVHFLRaoE8y$N?tk-&9!^6zw{&cS4xBamG=|?9L3vY*YzDg zL0P0FQxa=OFNW+fZM>71IQ7B~b4);35_va2pgPPlw$tyR&2vy`VtvkD^NbfbljEy= z3slxGofCVdF{|F*dH97ojn`9W!%!IM-qR*Y1g1neX^w*#q@ajZby8y%Adx27IO_&H zQ0PJ?v+Pk|n#Q|z?rw8El9jt_Iy*ka$I=_l8nv#XuWqGbhzT|mWb8v#|5z5cWjf+vs>cN1an|RbGF;+k7$ZJ;gNGhGFrtJ zF${7jpT>$^_u)<5F|-P@qf4K*p+Rv&RMxY#=3^lUui6SEta2Uih8S^HezZ8EU- zDNf|3WZ`$(RCb>qH;)Xu>8N@-xnx)Na}XmdY^Q#~y8_X-Q=nB?6fFebH)zIH`ecwIL#D%AJ6CzR5jsUo3=gu zS(-EKwFtwfCCZB@Anb3v4?q}iTZutcYX8~P(s6+=S}|QJZBcK6W!&~F14{xCalB-^ zZs99~jIFe!&07g5r(Abg1xSO*9SHySBN2RN<$h$AGwvhGwyH#gvRQS?2Oy&M7-d}`}?&RKt>aDe&4st=BkTkxQa)fMTQKM3kO*%fez>bf7htN=t>=(oK(?cn|@g9Kz zfwBI6DgIkpk*yIN<#}k!6?;Oo-&J}e1R2*UT;H6S3Eu`2)zp*)t$11O4aYm#Tnu$c zQ&1f4bF-;;Qd7yTk$?Z%q7C!mtdbs_J zy$zF4j}{kU8PrURGVN$vAgCavR89_L2odX*+2GzCVX$;_GacUTN(yC;mH&c2BuR}r zTe*RDeykFNb|MKZj%cR%33_kJ(d4B%Teub1b901o>$IBenTGHS^Ch4dsk91nfjP$OZlP$(~ zi|mziIO65jVHSgm0fh;iE#;c;Z%8pQJAc6+nP=)o8ee~-Whg`TM7y-$qnbxLPpkDM zgP_Lpwsy|X>@noQ-2~Y&%H7%tyG!jd!t{p1aR3R_Tco8HDz{*u5Y_zP#C=~}<6dP( zb!DEBioLxpH@IuvI-u1+_qJ=hY3-%l{R~Ndu|T>)cZ&HaPm2hU;@^HO7pJ%-Qqw9O zxA8t%2u!}DS=8O2UCGkh&ceFLu84?TaLY7KG;+Pnfm;f*m!dLOzG<0`##?Jb(z-AO zXOTn-l5c=2RIf~YQafe8X6-Dp^3ecubY*-&7pgtY~ z`O12eAx+&2S7qC9<_)^*TSCOmfjYP~BskuOL(x9lhOvFwJ7m?o>+51F5wl1PnXky4 zpJ6^Pw!=cg-@BS0E{4F}-orE?kd=80pT{g%tDzL?!ghgCX0~Np({I09rmGjx5xU#U zA(en5Q;wfP@N!$KgX7+|mnWSKD0iv1s#f4O*Rj!4BZYnwmvjOelNJ0BXFN+KM0L2w zl9-|cOX&28=-4wcmDl;IRQjSWI1SIJOk5VQ=#<1Ypp|#6I>?y|s}W4wDQs zd5OA0Vo!Ho%u(hBa&y`i3+x!=LL?sJMDnX7s@4Y0VG}75DRRcV`%qG+N|6+uYz?L# z8&^fm2>$3ogD1ZR7 znaHobYb7NO{|T}z2*-PBX_lZHHk_BIHKSMmh2-)c z=;lp-cB`An%~q59zC5ysnjT|%#B`)Vr*bnzd2OQs6e!L%o^T#G=Z+O?_)SqZers!cwCi#erngLh1wshd6sdWpyW9I6|gXDpIBD2qB90 z;%yY@41ur`D7pt!!rSuoCB|o5%VebjGOL@ksd+b#WLqEibea&)y5DS@rhS~S-_t4F z{HPo@QPre;ihw0=trzPq&Kc-6&B6zV)|K=6T{-UH*!3_uAk9`y!kP(DBP*Pg@%xE9 z?~Bor%QVk%LW{EkuHoDKD-uumr1?1*`^(RF?pMiGJPy!&chUIA&wlU)zhBIc#yxF% zxjULJ?3!u79p7BRK!lxA;+L?mJAjMJ4X z4GO2@d%tpl2`cH%xy*BB5?jq=9iU=bfU zz*3^JL0|6Az4XC)xou4NO%Q52V$=fD_Da~JMNpQWV&D%V*utdg~g5{U-51M_ASadp&-B7?$Ic4vb>m1|je*_PqA7O(0F zMhE$_Ughk26PDQUqQ1&IEA}$hPS{Z-%GL$7i|wG|6U?q7aLbDZsHz@hXc80z9~%R{ z+;pIe3axbuk;?8PyJ@0@T^t&rw?LET$F9wGiw57AxxE^E)MkU%!tv+-p5 zZp8TWu!74h+s>PsX>>^Y=o5S_qwhcH2p7(zC%p}pW~mIWc#K=7_KPK}N->22*T&jd z7tRq%kmYf6kWgsdMYSJxF`O5g8b-z(_ckJKV$7fTl?)chSHt^hxz%w>6J7T;&oo`f zBHlIo)@fn>NP!_s*ViPsSI{@c8(X@f1d5_}_)~kS4qQqn1KU(J*Ua8crT(Ys^+zvp zDaWU?3f(dxm}X7b8^(|QpWTps_3l`31xyBrqMm!QgzPgw6sK~<%3kwvoX^(1T zmA@HPM?U0L?4)o1-WiWgw{hB*OIxgAv3WlDtU7vm&coJ40AmcuH^w-}vov3^m_9y7 zKdsk__J55UDyph45;_EzT6b5j|WGRK?y|>PZ6xaV=z$djnN9E5Z^EdTeF!2eL(8C6}HM;H^ zE8udV`yo^30wc-<@PWZRsQ1kTWc6K<@TSA9Mcx1u+@jWfFY-5cO__3k;>}a8+|RP>Q{P1WrLq7-LE| z>U6Zp>^JPAWE@`*(!bHkJHu1#NuG+l)cF3R2F%_#`K5Wnc}Ee-q&$&RRXe^M+a=!S zlr>`ava!NqaKwu^WtFRnZ||Ysa|D0dN(AmE^~JYaohC3`0DlE|2qe3vY5A%UJfQ-6 zlImgo2S(#v`)|E4_wduL#VTdjl@%DCQHQN7;)X*S3dqTxch}UMwdKTD-fabws2t1UPfwU7tI^CU|>An8e@3oKvd* z-LvyG@_sVy%YK!z65WY2Lb+Y3XVb0 z?$5E?{{WQ7EoWe@+dB%1aQiYh(4%_$rQ9yBZ%}Tgzc@$xg>1(T&wbgvAHFON^%I+} zxIxl>Vs4z&9(wdA;>c+xic*&nh!P^|ZT1dX@?Ky;kDJ%oY}xhcC|WFu2)F@%l?)})lM_;lQxV|g*mbHBoc{;0YBP}k;2eoZD#p^Hea!>r|VXSdw!kZ%&u z1T=EWga&QyQG_a4?MP#uB%)DlyB2Ir!u&n% zu6=fh>*g5urdcfw(@)%!04Sf_OK=oX;(B>ADqn_*O!^Oh$Wv!74vc1v!xvZ%8yns? zK>~4SbY8W*yb{Z99o}^s7r;1|Lez9FaFwAdmA)g$;|q?V+^MowPXOpUAwL0c}{wuHvC0xKii1 zbgoS#u0|uLCIzX{5N`JSms+mZ`F|KXQeqfZvmh>>UPzG*zo)QqeNYh;tC z&hoV5S{rxHJE5ycduf+vxuSFVcq&S|+0xUaD7Q5(b2e1= zbiuyqUW>)v01b%oV^mlg-nUY;L+RF=x$oX`4YhG96D&1pZeu5tLA~Ps37vPJyl_^l z+$Ytzh>-^Q%JW2l`B6^GyPnr?SLRKOro7D&)%e6lJ{3#7Ts+C!eQdQA&qKRvqu?9r zL?R(9B-IIhl4{qzj}JlwQ+UY2(@6~6OFIzr@ah!7=5n8vn|6m%DW8;cJj|^(BHO4R zQsA1;P+1Vu0mKG(d9y&CsyBYEfzjbweQ;cNOtE*8`qW6M-;_x26@$D6nkB57q@Ffu zEA5SEa=`W!{1k2=Ry$?IsvZ@=2ZRO)pU_$(HkA=kAE#ZLiKZ+kOyAJTrN*zy{2(_K z?4V8cpozkkP$=^jTW(@%pQ4+o7YA!fhnKT)4tkXz{E#U0^cR;s{qz$Q=YT5w2{Qd! z_Y;)OuhabovVez5MBZOE`U%=SMcQ~y2a8JBy~SK@8(p41#y8W5o6|6mug(I5Z^pfHg{+g~fMA=_Eez!9cMnEB)7q7M7beIb^8$gR|l8wdfjb`#8P?xdJzbX$rZg<(}KTyUk zSNh!>C!jODx9;@QHQ8;n(uKP zd zVLN-PrQ`Yrw9>$bu@vzWbdvvH8o%7c2jo!A9_TvacPdljTaV{&#SxV zT>WFHx>B!hqxfn`9wbL(xrcDsMUm6Ipb zenxV+^!kS1H%}Av4d4P;bwLeC(;<08dC*tDrSSF{P?C&~imWR|U*?`E0(XU^MazGE zz}c_E1P!DbeK~GHVLAN-b?+b(!X#IW9Zk+J0i62=kex?Qhoo{cIX_d`2WHOdJM`x# ze*6RlwV>|&1W9fAA?|JhZDglcxrqHb8`oNnPD`(`@UNdpt_W?k9POvi^IX5UO8#%} z2mF;kuXVRRkQDpNO8EPALWle|=Oyb+&`Fnq|LuXSYf50B*sg8?>(t7Tel2VgQKj}1 zM06Aj6qmqTP{fkw06|B{Eb!$c^?uGR=lrLTKR+Ui20ph{H>EpYZ}}Zs?oR*tiReFXvM=ALELw3G_(Vw{=(eQ#6SQ{o zn#m6G69nG60z_-=UlfdJRO08vYu`y=f*C(Sy|?}ni@zO^zr^A%u>fx7|DWC#O}C^` zsY_RW&nu9wF%E*mNu?$%so9~>M_l}LY@4bnS*kx;8~n5KMHpM|aGV&N`c2$RMNqZA z5SOJ3mhP_lcAt}7c(!pM-@n3W&DJ^!P)vSX6$p1*8t60?o~RFp8AAe~27p-A6x z*tGP-<;Xqs^X?7Q%yKzzK-jy)`eZj{j-s^Ww9D1f@M{c|h7T05{<|)}{{-2{_>NRl zSj?}{rGxdWA2oTNjF5Ev`R3o*G?`7F-b-!?;Ht4*q|F%SAMcakSWZXD_-z4tfchtB zXL|D(Ifs?_ig$5`^_6<0XSvgQggMK=9vkK~7dspsEz7axrI&S&19?lRZ9}lERzdY6 z161E_>QUkm_f_;X{-5vD|E@Hnfyk8Qb($Y?+Dy6wGbz1_$Sr;-!<)_4F#Zn@>o0%lKRv9!#Nscp_{)v{3tjmOUHJ=i`wMjYZ~paP zSnOX|?CO694*n;x7)*x)STn2S+JA%=x*M)ad@)5=jQ|KAv=0Oyd?jh)73^U5FC^CT z{Yd|g(&xEPC%2rt{Tt3d{Or=i`>w8G~+J#8S!aWlw=}ihZHt2NO`~+2lPA*uHw4 zzMF6ds^jw%R3l4iwjTiWFXsOS=yP1Zs{kfW^GCGh?|{C})h%-X@)r98(2rgG72PpX z=(z;D1p6h`J+ma=mOBB*AAKeXMy!UE4{-GzJ%n)xEb?$>WzH`+o3`9HlnvKge{eLB zrNkrq2b@vn_XnDBqveMf5PJcjAY}a?U=5bvH&+w%&A-P{W~6<){)A=({|U{I`~#eN zYjRBU?~D33aO&4jMcZcnQ{cYhUzx{0jpqRIqW?N`=d9Bf6yf53ZnWV46o8oZ!$Z5F zERoODDQ~?K6mD#;(XMxfcx2vSKEE^1n0GmxMKL1yL-5@1lH9LPs|BDrfBqA60ls_n zq;%7ss8$C*&LSEs@Z)Dqjk9EXljn&`zKgBi(k21}Q#=H*aZqb7>+Qu2AVtg@ z5b`|Mz5oiNYh3Fi3X61Y?tU5K>dKBHnB`c16GNA{iN8>02gaCrPA3=Kxx*7$wKi@( zalBYAa>=RxK1N1CX@ijCsb%yal_qX_$~(h6)Oew6)|_5jhPCQ;qNm>I=A=4>^jd<& zBSKv9MYch!?y0ze1zGPSl}WIudzSiW_V~2Q_?{qDUeT-i$yZ@gevZiwGX8RVOsvmj z0>d&B9@Z_HwrFQ0S~@RsiB{FsII4T~_}baK&P}bWLJxnyS~dmCXI@Nh%N!(TIv7Y4 z+k2v0?#C)zG;uY2@y7CTGim%5R7-!!78)Jw-CV4!_ETJUcYaXgrNvp|fBQv=rf#M%PY9T#9DPNWV9bY3;ceYmE1F zUhsjjN#(pPVWLv33^`vxkSuPlRqtwA$hVQ~9MPeklI=0&a>JL#v@*f&jyDc zuJ7|+6mezS7p2&jT(B*`p%5D%iCo{2wELrAHfvuzq;a{Q1J;^nlvY(A6f@cyBZfIZ zgFQ;uAVEi#EBiXC_+^i1#m^VZQ zQ$VK58sc`MipZnRgv1N3snDz~Po?T$7Z(qj)JCTiJ(i~E{!e_0{P9ndZipn2IKJT@ za(d*gHNAVngKML&1S|RM&BT20ikeaVEnA&eLX--htmTu}>}M4kLCovX@19|#&nRd$ z0c^0aY1sUO8KZi)^@a*a%Cnimh(7!Ne6O0VAumbGN_ztqF%!kAz5H#;p4~OLOhcxg ze1Q#Xhe1tK0-v*3MqZL_SWu{~W+hGAX#C75ukuRv%Vv|AMa3}}BbkvdUk)EyIDAK; zx;o}~+Qag3l?tcV5?|rd(OpJqQ-`2UdhF$*X8w6=&xjg*S%5toJiOh#sZv)NKrtp3 zVmJRfaqy$cX@DE!Zs!+za#ak`@J>HUwET*!ikR|vQD)fYi@GQd-E6tl$1l{M*@b=8 zZnyooPvcJhkUy4OnGyO_tEjjTCbpO=1|MP@a6XIHD(OUuGAb)1)WDBmbx_E>n3G;{ zy6o5M;9I-%=03&Wt>E_IS+&E3ZkaZGDQwbJO_kMNLANPi3{iX^Xj$7z%~o4Fm}#Ev zagDV}qnSQn8!stkt{At&j^<}RAeI~9>XRqe@7Rz`DCr7qGyQVo7T#C12huk_&&Wpm z)Rl{C0p?axYqkE6+nzpl;W_z7I!?|a&DUl0<{`Zzv=pdRx8(!;p5=xn9L4G4V!Q0d zs>dIRUO>Mlk&52MiXn^%V1LwG66nvYrg5{=Fc#(mrLw*?rK~)qurqIvGvGrFyKQ<1 zc3+4}ben>l+t%2hr+dw6(cEk9Su)zlLTq1lPfaJ99w=2NoVqiQvjq(@^PM|BuNfne zP151kQjNMB)GIhAj}Ys);0bCp&uS~|DM9rpzmtDk)K%GWD%P+w`?`~#jDzRiXdS$k zEmz!Q%-apgh`$c}lnOskB&yOKL4X+=k}eF1c_ zNacA^)jXx%X}ql}{m^)XHJ#Z}ibM3{mp5hEtCz8L8!MQzm3(NCn1fhxeoVz-gM%QNhB0e`8QY5K*R<62xK{4RI z)86~a57<-`qV%udeshaOI|Cv90Jzb#4j+RIU*(HIf_+YP zA!$xG0_8_5Bztd8$%Mn~>GTicS+b!+9!4-K!SS91_i&8eRPV>l|cRA)mB^4;! zqPT?VsWasfD8A0PFNraY#aVkA&%_r*a0sKW{?x3dP8bIuOYa^B=|+KO8=P^2|{bLX~_ZoJHQ%bG6L0?AHXyvyjHyJm!dw4r?#Z zf^=KntobJeZ3VJ_v16T=!$iIUUF2OQ4O zc^|)gv@im;7*fTwta8tq=!Z@-eN|=oWRbn)1k`xJ)mAFsK>iK${g;IVUl}I(BPb73 zRNi^AW-QO*NqMW5?Ob#x`V{J0KOO3?a^(Ua3)9*WR{%8U8nL%UTDgpT!8@+P@@~tE z_|}wEefb79HKt?i?Z$@sV&aO7jeFJfg9fj?usz62xW`v4n#_H&YzU=nUw=CuXkrER zU+UwuIWk*zMATkt>@k5}Io`YHtB4VAskv2EATU|%nmwPz#WFP1SZCiqm0V&_)to~0 zXZ4!8S*dcTuFXU5iA}#+zSvRB^l-`nWPCC@4&0?mOs_P(du7 zv@jCgF(r=qJT-LCZ9-&IFNX5dt0eR^ZyOaag(aI$7J`>tYQYO^*P_d#zjv zch)y&##&QS+VFbRYgQ)`^iHdFUcV+9{1QKx@w6eUDCJ#1ih^}Zt(TwIB3 zBjYNLwlc2^`_o}lPGgxgAG1xxhKF8`^xX`Z+on?eQQ*~8*E%O(%M_XMbd~F##{^{# zjtCqk-p_b@qMvI&VB?yvQonT2Bc&sCx#S^HT1l*|tQYpQ)mIwI&S$8utqOySy^O+q zd#KH9s?{^h$176GO-_B5;CMPD-@UU0N$z8@GXhT}!%rs*7e^C#*cx>zluS$_#f)mE zjl-KtGhHR5{Ve)2k?|ver6)>Krk7mt>xHvF5_N?6q&*e()8@%>Uft)oBi_f~(@(y3 z_kwH}X_g20L#8&@T+vEXc7mxY&ns&#sfMV}C~fjBt)Jz~eJ{h@WpNa-O>J~dLRPAd z8DDdl;Qn~0+yr4K?KBvyMk?k85%dbZDCIEkbK>u7yxRBIlnj@jv0p9wOu{EB9r-n~ z!>LZb*N}43WF|*8%oMTK%pPdgI7!()wGKVwM6iAqB&`D}o;__;^jG0YoqKt{`QBFq zHPH6x2ccCA9Ao)=>GHe5$k{<7srbdA4u6Hmg|anvcO&+5s!*G&V)Bg)kD|t$mI}R6 zAPmg*!HS>IU-_YL^b&XrvZ1Y7z2X}MOQ=W%hrrjX2Zz;Jey~12SGZ&Mg-@N|!>74n zy~*!$<#YGvjkULi>^(0Qv;m)Re`ad|3uT7b>C_Jnt{O;{Tl|rDmb~U*ENgl?oycWw zZZE_#G=LSUSpI}el1*|4KQZQu?=YD!E5ZS39FuP}Xm2{fE%2i|kv+_cQ*Y9D^i2i% zUQWsYG{h}p$?iPR!z~cZjtU2O*uK-vywUF!mvHlPyPZVQ8R>`n=0&Xk>H~FNkWu#Q zqj9p}uFzrsG`f@yeZfv`QzR!#+8vA7ZmKCu_sqpdI+~VTgh{VoGk=a7(q`fuURJv@ zZm5Ll^d76{omS{Cr0zlQSNGkQu= zgg3~{0om()(-oy4gX0w7PAHuW;PA@OgZ~z<^ge8*te1X_6!#ndC? zp$C0}nV#Fefy?QHY6DI@*7{ueJ#mV5>!dXhCH${HLD2Mba&H_Uo8yFT9nWF$hB#KH zdrLC`9^Nk4YA-XvL?{g3C>ij>o!@4AZp3@)iu8NQsZlwlb{R z7|Q!~w`OkvNr{5Ku@o;Hxfut_k|J~a0^KTQaQ0!L2XthPs&fKMB;EF1`93j z;m`Cnu*M&M&8#|0*uGLf0!JXH+gpMkv!r+C3n)4^W3u**Xj%ZQI{e)g$yPdZL4C3QQVm21Y?Ur6r;f`K)G(Ng{i=oD> z*Ev1PS7@wC<=zi0Xek$($Frsy`X*MuKl3rCa{tb!3DMS8CVwi>-J!S@po7#`#7KxY zlW+`c%8FQ<*F(NKUd7Ejv%O-?S{7;_W^1Wm#wrUsHHMD93B&>G8A!XZDP^Z@*QH&~ zv%B%-C#~D1L|8*Cp`_6X{RagNi(c6yd8ObR-j62*RYGX##+Qv4H`+JaXXTSNKV@qe z>i5t`)-+%f^}3?lLgoWEZ`73|EZg8hrFe@W$Dvrw@0Py1M9Q>T#u#s^4i?f|&JDhM z;8YffdGFB&Z`n>isz##~EAM9{bEFBIor$Dw8jUy)#ntr~nSJH?gcbaUoA@eu5Z8Tb3j*ZkJ{@dLSjWkdtYt*+S{&nOl@9 zpP)rms0h_U1;_o?xCC&c+bVP1QOPflY?Wq2G2!b(IXJbWCAAWtmt5uGRK?13oO7Q_ z3a%DRSRSatcQW-4cQWOu{ z{TH+ym*snLH3=GN9okCn!UQBN)e#$&W~D!l$a$)^R%-*EV9YYVpisugeC>#TE2HE^ zd%WJa{JTU|aBJ`|Tr%$Q8!h(sDi$qc_QDdt;I2JYA(?S!>Ri0X(+F9P#xD)pcaF3Z z!+*%08!NiIpBb~vZO8l}Y4s67*WCar7EhfDZyuBeP~X-`OMsvmhUHj0N`~mDkyRSn zxRJu+ssICl?0fMPG`3b1d-OiG2xi1avokWcOX68Kr3eB|DU+UIJ!T@GKY|yaP$;n* zUe>>1%|ib$mXM$z&#dPK1m(19QMf$B)B!g|dC5+&A_`wzxXjP#;EUr6bUS|6n__*q z`S67$In8;g|6qmf*jEhoj()X7G1pBy1#TjFQ74YeJw-QjC{2;i*P8+Z%MmUdGJMew zO>W2!By+*2#Juv0<1Y*6kbRl43)b}Ul){l=6=m>#sWwbkrrNxM$-TX)9508wMLCPQ z`k3_uneF{er$=YZ8wJx_1Wk!To9znIuHOMF)6A!T)&l?bKfWRARR+oz%G1@%k7V(m zENC?A5i@W{Gpke+`*Bn;HbmOxfV0y$*;;DIQsB=hZ zcICqU2g5t&0csQkA zOHqVKI8I2R%%hG)*#h--s9qsEYq7P!*(7UYv?WugTCP_8)U3=jpYo1N@p1i$o0tg} z;B#$VWpRTkA%u8}autn}(9V-$z)+;9FY{vlEFPJmZWM zsA%BTx_ig&7RgFuC7)gt;26v?k>`62NmmuHgAGpXxV)N#wIxjUaBodO)KwMi)E!2R zSeI5x?Ly@xJa_j!=84rz9KA#S#A@+JQT@NaFiwgW-!{{Aj&)JyGO=?COjRw)J7zd#X_eG$~ush*p~L3oUEMU3UDyEYq_BI%80vx zLr*e@r@_d4x&i29-u&f!Df1t6I4ZTe?nlrZv_baF6vx?22(0yk(n*Gnk7DWeLLK!V zWEe7bXOb|aaLbFoeQV5PM3WnOu#4u(cb9pvBIKDfR6J4g%;W7Isn(NRk-N`_8tBuo zZ*PK)tpoG5c~8}?yVX2}O5zOrW?cq+XoE8nO zlJXSAyLZ$k?k(1EJb!KV!Y1Xjt=8eSsywH(xqISs1aUf+vkEcF@w<_`rR9H^>ZC^K#;vA01=moHTrViNeRA&g_i+GE2ImqvN6Z$cZd zHJnH7odzmp6z|8yML0yS%*&%m$ll^Q_*r)3$>-nd@K|t;VP=t^ud$fG#H$Y?(CHLJ9V@<{8O2dJR`;p1~r8m*<;fmeNCx z07wcJITK9a-^-7b#tN>$CE|Zs;aF& zH4F+XZ3uKl?d@)UxDE1`5kc1oxJO$L4HloiUP^Be9I&(2s}lkH=vPkau~p;KB^FSB zZUn*>*6gqYsUze5Eophjy^`U*;%GW=-ug*#OS9$D;AUEm;*uDwh0>fE@wlA$*+)mJ z6nb-@VnTE7S&Fc=)$j|RKs5W3B<>GkUkOCd>2UrEp5u;6U68e9WKpp#<%12msRg+a zIK?_M=^RM)#-TjYo)neFL$YlG56{-y;;UYLPz6ai%C_Kq{b6oMjEwp48SaH;@-{#= z1)y@S@%O@6V{&&~t91tQXSgmDx~GtuJV_K`a(&u&8)D8F9^T{sJnvok$o;YNc?zs9 ze$L|9&p!1>_PCL4(d-Vf1BxUpAFNqgS@BW3aM!WeHhIs@Ip)WOr2jR|)&dJEaVb^1 zdsB0wiVEs*d>NEJ5QI6N=^7-XTb;zpnf1n$0DqYodDI-@tTkCo)u|}974XE4bRsV` zi{>d=e}L!fcTu$x9GTd0h*QPpU5+N(K z$)|jc4TMS)THC@yN?3o5K!>FHaV~UEJ?%x#w(8 zfYu&w6kEEKMw~|HS=amKg3pcJce!#ZqP9155;S7V^UeAwL~c^t$VSzC@qREWE|snF z<-3VBoV#=x)(n%&KCH&+$uotss5H)FGo+~c&1Y3IPRl_GwEK4=`22vlN2IjCEd4)|i z4`>bK?8ld8{&3wwEVg^{v=zIlPIJNiC{l2^jF~p-ivv@Kb_t$Fgt=D5bssP+rskQo zV#WdI3{8O;`Uwg+N!g?87&Z8jKuZ;^J?{OSdw!qCo+(V!GE5FP2iwm*5c6fXhUmFJ z_$Cy&xRMQ~o)qT+w>Y_(7wG3?+#h19BIICar2B-mP`F+#)lVe0@g;p_~)5D#Dc-CW0BR@e8&Sn;+IG_8|O% zIE(jT#C7ITkdz+PLTjpLshJ*!#E1a`3ueOPeSyV0-h5?OSuS)KJIMlP^)Hh9x2lC4 z3>($x6C6mla{4}nd`nO*!r`TO$)CK|~PK9G0Loh*2a6CLM9uL_`F;0||7DK@vIw0U9{6R7RPIfFd-YLZB6pMV6K} z$fmRiqA1W12)is{?UtpRrT41J4bB{yIfvwB9-MP{sZ&)Cb?YwQz2E)s|NZ}Ox^@cedY{0Tq#CUkdb4sPOq0HSl^Cg2Sf7k3|YDW37N z`o&|`sIo0{)M3`ZBR8qb&iYp^n@iAcO^Dpw=fmSxr{>)hwtJfvmp%WoQu$Vj{=2=p zCvtO6zo(#dp-yAkiu2Sf;-r9`4Pn3YXoepis=s5#a&xdH1$n)fYNP6zDqW)vLqxt3d`UM3w)|`Rw zw+ACf1>5Ta!__IyYTJv7GTRJ_%}fkCO7+f|tUtInjhpbJeTZ&?MUaX!O^`M-vxu0v z@EhY#=k$6v6*T3WKDg)k(#uIRxn(cM;OVIDX!(x}1Kb%dz*Hl8Bn}P$|vw9AII5@xNJ^0`5q@nHzce>T$BC>xSR5 zZw&98)uVO~(~d1M#ix4hKHL4S!;V(M3!oh$u@8A?oWh{@Er4*kg?N=Y%*?O3?~(?j zOHDA>s6DVB8RKNv?hyx!2s1|TO5yZJ+L1yFX&*&$Ihf#R{@vyq7%2G4 z<60*HUJe@}!T9EG7B$P_*T_R73|#n$`JE>^>Yak_w`ku@deU(T-)tCbsh^-cT1st{ z1BV!V6R>_gUDz)sw(D!!v1847ZE!7;AN|^|)hQy_aesr`i;uxj; zg}-;e>~qn?e2*N+!^G;cfF4vvr)NwrP|^qey2`}9w$T%=lmfT`nAbX0P>G$T^Oe zxM5J$Mfs-;DbTz%5oN*r);OdY_#558OMk$i=M1Eo_?r?a%NKRTpnk@WA>>&fAj^JJ zJm{58MraOk-US_y@5^Qf@616zut#qa=kqZr7$^P*aS15jl40TGNT!DD1`w9-B`zso zP%2S&9c}=E?*fjTjzJZGiv;P#>Nwl zge9)9K*=5xFhYcIV`2pb<}@%wy%?m8)B%*lOYD(#-y#AG8b1VZ zV3&`m5aj@N6~sR&4UF+vkf^dPjWL8r-SMjxR`ei|pqUh3#&c|94x|$*6h_ljR6lZ4 zO?j;5*PW1^S>xr_Z(}$dayz$-y#m_glK;Rq0Mi;g*XE?PiFj@GSzB_~c*a{vt`^@J z;fMvWbPB#&B3+fa!q$uA#Y3y}$4n_fs=GRKZACPYO|GjE$jR`)o^OC28i~toc^Npt6cD_Z&JRm z-TzP8LWQ~l;E(u|@^Cg1LVQL-Rz0!&?~_a)9-B9UL8S~Cf#l`qx8h;>6EN2MJ9Cc> A$p8QV literal 190881 zcmZ^~1yGw&*DZ__+&#F{;O-XOrMSBnYjG(O9E!VJaVys1?oM&{Lh<5q^M3c<`Tu!m z?qrfV$s=d>&OUqZv(|c|)K%p$P)Si?U|=v5?bDuxGuy$6 zNPEN4h}ewUTzC5Zd*BuY`F|bS*+s?pe_XZGH%NfQwmAVtz@fH17_s46Zi$Nw-2?pB z9D`)Zny4!CGoBa;Cxd%e1u0U@|1*C_syzgG9AFjxKAG(#t zYDpdZh8>t0s>Z@e7uGkRsp0Mb!SA>Fp@9&=FI;-^AU(LyQd8XJOwQO65K-~9xI>>9 zYMBCkGF=kNe+5NMD+3O7rt+?Ul!*z2BC`u`DZU~z8y*fg%#=`4;w$bT*;iai*dht_ zbyP)!!H;0~}&0YXV?;aaoPKiXM0TJw7%o( zzvnX{;MA&%s9lXyPvFAhGube^5NO?4IKYB|CfxK^e-SYG>WKQu3ffOQOYm^wo&AUz zY4??c=^Jr#si*{!_8dYmmV^7$Fie=9t7p4KP9}erwVm4sI%6ah1ECkJKz@?sX^y4;p^7p$c>O9zgM`5j~7;V>f^z zd(0?_Bq5#Bkb-Xwi3Gnc6F`Bpy2rUm5yKH=gM@=DSx6gpQ|0Zs^FG)c?zL-2fln0- zL&~?3Xi4`oC`5D~9#9uvFf&Od^A`*EyK&53uxk_mfH|8i&gk}JDhb*gO93!3agwjz zG)-IyGliAZxQW=f^4TYShQU#fUP?aK*w;1mzG2PzkudU2mBHMj3L(y&1>Sl841_v3 zj>o29!~iIAM`w)x)?&#t_!%u+iWt&6F@atOW{#^o{OBD@5A;1;6ClZ!(pM}5$qa{@ zc|aEfO&_RK0*vj%gDy1}LQO5(tfB|@;l{%mdXBg7g`Y5e#zqdj<}lknBmPem2ZN6pS>KclnffKF5%#n+ItrztZ(d3)Xv6k8So%R_)f3XbHV}E1 z(0)gATox8O{L2CuMj~@}A=Rl}Qi{6#q9Xsf`FU`oU?9%ME#4&=Q%{2j9iBaqO^dh; zIe$8|tfzr>t^z-;9}T^ZvU^y9+0~sS=>!yi2{(yHHC@iFbOvewL(+?VUJ=mc8MXBK z$kH_*;3QL0sdGoEJGzvEC`_yqRZAOEB036?KP1!SHn*pynD5>{epAlTR(-IZMW+m8 zx6-r+MI70C!v~5$AiF+r?aCqAzs=L;csjmPQDA_nvZE6l+R|)r0S_EJJq_|@b9ybo zHrqFkm&DLDtV|<0L}re=H-CWh=FQgT3hn>snH;s5Af7!W-DLT2&hxGM|vxul!k>{(T@Ewh{n@DaCBpCM-?y4kW1%?!agI4 z@M_|T;R;Hs;7Mcckho7Gx-D?~YMCrwQZM6;GAYerW?`LV<=zL{*c&be&1C|AvIp;2 z$=f=cOyG_5yEg6vG3u7+(xFCsf6O|k#^x9;owEXEla=pWJzbLWLH&JN_zZ_2D%5nQ zoCF>^#Kkb>2$s{GaCf)gtpKu*UnXAgk66!_a1-_sK-lGbiaEy>%jKZ0bZp8Su$WLf<_^kpjD!$RWVVQb_xkQ5V#u!vW zS+Pp`ad26QPD$Mzr5U_ThsO+(gLV^fN=u&ukG#YbEG^;F1>^gZ)kG)DR1&CZUK$Z! z0ym#~>Kgv8*q)pS_B$R3@Qqp}sWVZQ;`3K_b;TzHy%d8CyWl=?M9%TfrG0A(vr=V} z`kpDAzv!rQ<9Ol2|Nb)l-uK&iZN#c^9W}%ZF|6=2$u0iC2UaKeIY(gMBxaX=IDZii z=?qHzsh!~9d>q^CrVt1 zl+p@xLI%4f!l3%6j>}^?4JyQ6xr4(Dq3uYd8L`m8!GU+DiA@w>Z*Q+e(;j+2g&N-n zcwQy%_Q?mLm>*jB6%P2d1mn4%qDvgOZ8gz@>%xp7^>b3x*585bdScF)HYBEV6sCP(kDpAuEw8`UPW8jn4r4H})?)~dQuVYfzQ?P%< zodWvDYI|n+>w^g7Tt*ENndwagHeueyFDwU^8LU;)jF~Ji4a<)I5 z@Ro6MLL&Cii{*MeL;dCN>SSC!Ym5^g24G2g=`$hw`L=ZT8+LB4Qskfo)OS*aA{{2| zY9d#3;txCBb4yDJdjduYwZa$X{kw?A(T@ANzZ<_dho3DsiFmP$6JRDb89rWQo7@!fNu>6-6IZiH}~eUUKo~S#Jz4HGMutf1Wa2cP~U;6~FxD z|96XhZ~tp3dc3cm*O70m2bRayaJS-8L|{T);4Vep!;@my1FOkL>G5>$2h;S+v!kw_ z{1+!=&RX4I!MKa<`}foCVou%xtNP%wB??+=RiC728rp@UkhoENbgnZj3Y)a&m?0% zHJtGkBYaSCadT*yjd(>yxHRP4cukOUO0qr&QAsEx`4bF-41vkd{LP`L9k(PtA|^OQ zdLEEruYW!beSz(;K*t3W{49uFT?n!H?|Qo%uZJe`L-u=TwqqtfAM25^7pRwFb&^(G zSRzChx1i9OT1hBtB%Jnm@T2^_mXoNE1Rcgt607q$)l~RoyRpeCFce-)OQ7e(H73^xwuxYc!0Gc0LO%a)YBC5gpQR zwugLGMP6y|{*_8{e>dawe*FLw>Kp)sZ?G!Po}TA9*tM3*PQwQ)O^mu6JT9 z=YJ(%d-nPGI7r^{9B?D}IG}YV&K*4&H`^ftYJXJ~dMSSxmc!Zu)Jc>VbR?w_RDJir zrQ_t#v|a8vz<2XR9Xr)q-&CF{4z2r!>YQPlx1sj#9pd=-U9T}QLPyul{obbKOjrq` z&k%k<05V#Xv1d~t#ITvVI`h~VU`-yHT5IcUif-7%X*5a#ijMsb<<_1ymRB-H5z@`* zMRgc=#uUHd@l%caw$})A!`G4RXNYkp_R8a@hGFRz&eg#KYL64vnPO~w4!R0GD)Z*G zF{gLr-;`w{p+??MP3LL~`tUrB#hqN=&ipgKaSV0r=*Rpb`Y(!ilyU|ilz!8R#ENtJtN#~pk%Fzv1T@x%cSqw@kqVl-wQy*)2UaXH@ z?Jx=MFd*6)%}njy`MU2g`D)-q^-nimOi(4FAKQF8v{uSAHIC0!wKzP$6YowYmxIXL z0Ba)%E3Hf{StrrfIO4?z)e) zlXdlEWi%k<;!m8&a=0D1v~?L-GLG#5s_|E05lOhoc}I7y+Y3~JMs423mDS&^&es{> zzQX8cZ>^54aN9;P;&$jUBvS)1E%gcOB34%ova*jXh=oK&ZQF3#jpCj7VwO%1WbyZp zZjenML~5MTmbuc3*^w!k;1bGqc+F>8d|qNFPi{EVo`!r}WeDzx&wt@7l-%k7%;?inn0_3^Mt%Pl6;)<+ zR78Q`B}FZ9@06YaK(Y0W`Ysi!Ix2n0aC!m`2)R_f2xjMr8Nn4sQVszjoR$g>l>?0| zv(+2EbQOCEyY~z5=^2@rgrwIE=mOD1o#$}_W)BS^rF~)RJ^kkzwR^VW0Tl3#wBica zj5e$o*tR_=@#Gp>aK)l153|q@cUPVXi@{F^FsXbEj^r43lVA?lq@sxo-dje9 zV?eb^B5B6z-JMhhMr;qo&e6t%H(jY>5;)&uKXECpiznG+Cq{O5YHj`P4IRb7WWY#N$EzTaDIv~>_Td^B^ ze=&9}@fYKGpr|ZYVoCk?Z`R{!?pvMr>c+;C{BL*Jx3@Vvz9O&v7uUB~K8ZAZ9rxS| z1h$X)rq5jBYa~Yo1l64v{URLQ2)R}JjwRY+N z?KgY=cUo~yZ)2v7_8gPX;_vwnw{k@FsS1FyFrz`g@ML~MlfDpQB&dPt0;EAK@bjPKhrg1N{X4;4Fu<8$9&nX0U z*`rxr!TfW)u-N-)E%I+!>vp(}raNAS7crJ{V+;)8|Di68oO*{K93eM)sgy=)wseQT zGfT`bO+rDMzA>e_M=ZX<)$fVw&Rq;#X@d6yjh{{?>yC8DD2=q@Avi=|%Evl-i+H1l zW_qlx{AkZlXL7J`)@8&)=DWI`bvsLv-F%F1ZJ>MQ5l%o);`G;Ut}&BA%vo*D+$*-9 zc;RB{6tu}Ki?8|BC9c@{p6bgF)qIi zkjVB6#uf=`pG3!aCM$25@L|ZpV#T#~X8+;`F+L3>d#deoz(0Zv+Ovw*0n=iiBcrO- z^)#Z*?Tu9UJ130W!YaSPGMn@PQs)}#Ged%VdFrNgMe`Rw9L=`=tEfG9 z{Z}Aw`;KHDSqpRiS}%m2`pf&va88yhle2Muz^RV8T+moz1G+9k=8{e*>b?6Hx=Bze zv*HJ-668HACL7g;(S=bC_QN`3a3}v%;>|0%E#8@G0vC5bdm@dt=1`6zmguht>>UEc zvBR0>mSpV<4GQsP6=3r9kRp!_GkSu`bRKK$fCu(>o4r@BCUff?xghsoVqNcXQvIgy zNUm*CI4l*G|NcCJBNDEcyC}FGzE`sLNx~vg%!6g7ub0c5t=e=ht^#f;j!0D;DSPrs z3jJI|fVvNrYkG^C=wKhgE(r`YBK~AsiLg^VzyY_r9hvY{{5M3!tTc~{q1FblUv#b< zWW?1HJ&0nGe4IIr&PIjPq|8TOvAm=pHji4|?Kr+B>MG0h>@H?Lgc5c}d~w}fN8c`X4{q4d2>ClXiGQzLB%plsa!z=yP+T*Nx-14=QJz@1UWTf9*@H zu9>cXykb1T@~dz%?#=!1OCHQ9$d@w|9>d?I2a;t;We^unfIxg#S67M2$qyV9?Nd@y zU3d2trt(B1B3(cDC@24ZWZgeFn4O!GvbSeWPEPL7iQhBR7ZjnuQe-x%BKVNjWF|?L z;3PUgnLvPjX1DW~S|&L?CIlkezSS=#^>T9P;YsNi!XCKKcHaFi2Hi@0IhoX|A-$Lc=4l`6q#k*+&C#;dY<*WU08qC!o&Ui zN=hn0?u&UXCGbI~_)nzc^x|~*k zPelcQR1`A<4*fXL<1$C_X|PAQQ6S1dxeTCE;*mW=;Z&k|tm()^`Hxpg(yEY-q={p( z__M7^dXUG4i6^rv;yrpbRV=!JEo z{MX}05)u+g=D`p|XV8h!#HQ%8e5vNTqKu48$9*rdolD+$e&EUU@%cGJ_N4pDPJhHo zvm<}dzoj6B(-$32<0IK9K)X}?=|?TC$-4I^>HP!5$GP;biCN3xippxMz%_qN3W1TY zugC&^hoJ6zzx;!x%^R1l$M?6O0SitS@JB6Htd7pii=byRkuJtjC4$+mKQ1~1e|MC& z{Iitsvudv%uqcLbb{tN{t|{BQ6}``AozdZ0>QWXdpMd$@te+KxZ8`36ddWzE zA3xH#7CoY8Gx6h@AkWhr^Nm?)ja2=6LW2j^0?Bew%Y1rnkRgV8gCj0Rtg;%K$`~}) zB+=NxcB;|sOwJ#VpHtEvjuT4T2c%y+grZ;FT5>a}BBYqvuq6spEYc!F0n=JcgS_Cd zm=r0kXe9(UgdMn2^$A>X8<-hi7e}&9iWw0=BT^R|AAT^v7|+dIM`UN&X0qJ?e!;hY%hT zda(8n%y?EiLDG|TjitL>Jia*Opcl}D-pcYCC@KOnLhat%30v{G>x0l3hvz27WEhFJ zaT5xQ$)tp6JHlEc%gPKE_F+{qhBFArga#{4N(HfxuImLBC7YNAOd@?G;Y3QTmJ|fjU?dHATwpD}9nJ?SYDQ*bQ2Hn~2u_#+LOIJUH1o;l?|3#b>GwG^R4 zBf1dUzUtkdiZ7~ZmjgV6+p+&Roa)L-+R)jx7Fu`{vqdI*v=OCovAyVQG@?)1Y1TLE9?w39nUe#1Ze!!)-uJ(@WCVbRmx;8IfOKa37(&*9w(l7?2JFx+ zTocqNLey}*MsH4JOrX`<%VYJKLGLub=#Vu{-@ir>+OXTq=iH^r!iq#N&FK7eu1&ow zVjiq_qx=InmnNKpBGhxNkCvH>LOkK(AKkL3he#gR+P z0;E)^DKOD72-qW!;W?(4#ceoHLhnXeTSsSinQvF1z@CFI@@jCpZyE6LIi~doo3e2jior)TVBCw8=+Anh8w__4&to_WWvo7`Q%= z`R{YFht)vr9XQFWRi+)da8`8+T(OqTzJ*?%ZDC>bcuZ#B{y>`|^UASTJJLu%JS`N! zM@2)O;fc$}0X*1eE-CvxOTEB~2X<4c%rk*NU}hPXRH-Fi=+0PyL2J2Xa(f%QK`%DD znL~VdZH!BxSM%lK3I6u*pd{#y7YwZDirh2XybB7o`L=PzaZon-T$%hb+3r8bSAfqy zZke_3PWaP)r{TlZvy>eqzKlop^Szm!$JObe-B**IeI4vD{ZxU+gxXX_yT)Us*b#lLd_SSL1m2GAy3p1EtQAR=;y%BxSMhtVj;@XpJHIX z(}Gjflo>yvz5&;ev^xt$7I@&YSctgUT*vpwKTp_*^kS-K`P(Zhoe!na)maj*-uhpn z^)C3P63AX?!j;6pH|ya*K|bZzKMCAHfwYgMg965_iOMMPq6%9ip=H0N5UBA;4i=iQ z@G#YAi1$-B<)j^np88Tj|NJy;#C*b2k248)WVnsE`g?;OaEo=)(k-pI!HiGLm3H9! zhx#UF7cfiep`5sH7#^dl`-|`;W76vi+4Pa-UI_W|c@JY^^zQ0kFLek;(%je65ZD)7%)>;$lkxepYC=q8`_;@|bcy~4LTpr2b zfzQ(!hqYmsh^NqX5jcCw25St7x;|-QG1t!eDIDylOpWo z24_xi#8DsY<^u7skBF{x;6c=38SIK2TN zn9%@j-J{XQygBO7b-l9;(nag1kj`!+ z!?^@V=rpL*nD_hdk#WthKp;gzD=wck7@kYlkv!$^*O4ZWZrU#DVRI?0c zD{+bv93cM8UtJYaEJ>%#Ro-Q;Fn^3bSV+Uy$f;xc(~*scNWbr>pwn>*dw4H4+#=mc z=YhJFBa}6Ea~$A{HP5HbzBMAM^kL<_Gib*Z;qq{Py*|UbsuIdTWnF)Zp|Bi}r=V%T zC$dL+9Axe23?P8*AE3NW7I+1!or@~)FnUYLOc=iM38CSWP>H&KEB}&1Pj6)PgqFFb zDSwd|syu1MF=1gq#9mg~fFN!E5FqqA+|sl+LSLf{zXtb~+B zlHpaP=Xrr^yC3ZlLB?|s62Z|z4`v{nhHP-!6lT_|rPC&yE8SlF}xbZ;GJOCdouY1os-krp#c(vVL3h&#%30{n(oQfx)3wR z<6q{wyOQtmUHmIOX_uMUWYzB(6Wl&rGDg9Pw@&1fYq6K_BkD z^clU)H(Sf2RW5M?LS-q1#qjaY9#IKUL`Q#B5Q{b$=~M7>=A@IL0JyRsQUnMRj^I#| zr<)(ZMM||s%7Kca9)DDK=Eh60>z5{f0q?9CgbYA01_;Ix?b=u+@Q6h7FEb!g6iNxP)esx0s$UtZEb$m?dVIuL92}*gyATvAI6<<_6?0CJ;)l1u z&?k#tcdGr+!i;eCVqUs#>GMHd~ofH899(aL$H239)?mk3&%i z2nd);GxG9m-rrsyUIT5IOF4OX25L9_qQFJ#*f?;I`kw+yU+J!@9{>i6BJOf|A)dI! z*1VM^n6|#y0kg6>y_R-AWua0zeT{gZUf{3=L*FD04$!UM8LgiaoAv9`(J&8&Cx)`t z(RN|K^04ekDY*+l_%3cZ9RBt)AL5|T0>AF{ez9Ns>t-oWi_W}E=HpFN z#PEo9(!LTNk`LHM)9J|t3oR{8LzAMQtNB~)nxlZ11ElbVlR-{XIi7A-l$0W0^?3}{ zU`>gs_Hqx4I7ghktdcFEqXQ;N4_@Q1KuG_%J21x+X)T$aPcYV6zazr|b?@P;vbc2l z-Aw)WsL}kOqz@p63{p#ZUQz&(Uga0zDbwy>S`%K5mgnZSKc z%}{jX9hE^*Lr-jSf3c9_5|aFQgNi<%kgMWG3mh(NZB5wR6f?m>;+J3IO#|sqFb1CT zbE+hh3wv2)F+j!L`$Qy^l(rTYA=#xVG}wUcf4+}0?km84T3R0dY3tb{8K9_piacH4 zBo}(<)D%5BCZ_w7CqGT!qB2y?)s!sQ>>=QDO_wfHtl<&{6AzW9m~qIDcle-VCAgkF zpsD&un%@_RGazuJY9HYz1WP&gM6<-tZm5R$cgD zJsp#L`71Vrp8oG@MEr$0;j$z8QRKG$c^2u;<8mK?z2!#O)Y7&f1JTPrY&xhQRq|8T zSC7z%ziqkgZA3{r*?U?_b}RS>7%aKV6bhz7q?pzG9%%j{N#7>VG#+Y3n(S_3)Da z#+qov`8)kR5lg%+Pp&1u{4o#9&GwP`?TY_BCpZ6%D3Eb2A~ohRB$MYPHhi>dT=mXl z-!-IS6!dN9wyRV7sIK(J*^!BSRAcSzXnE84Ho>-lIe6HuzDdyJ+hpS8zAFzE`Of;R zF@D`v&(B)x+oO;LeEXcu_Bah28G$d`Ffyk)P3L#1Ih6dFYt>D*7cNtimm;~l*NP5f zVNKJMuFZ`@k2(H-6uw}BXU7eEmezC|3b`i_14NDXKNQfcO%|&GX|h@_(_bj>wuL*U zo;l85@9(>8*?~D8_4jnJuEA*QG|f!BKK!9s8{e?`*Rk(G9q z1XXHq&zY(KNl+*koE|*d|L3<5s8nAzw3@?*g~@nxdXHPbNAdYDR^zWv+)X>WnLDV- zajSpa<2!Fj>)7esbUYi5@-WU;GxjSixvT4S8h-|ar04mEgm;AhTwH5&yvViskuJbX zvKQ)|<=5`vi0tuNSO4z*tz(yJ`$NAC(z-Qxs;lQ}5YEhUyu2HH|KxIrnX|n}B<<93 zq49fv>T0*Q33Sl2d3?mma?o7WwwL!AoX|Ca6>-P}he393brIlY9`M(CN`(=Yz^gQW zl;31GH3sKw(RkhG9`noX#P!3Kpi9PL^rBVYj-%zxA%g_1(3i|8y*z?mG+>wdRN7tc zS)*zao8zG9+Vm-H@_z8@Pxmv-xT(dB0W{o26WP+GzJVxRLC(Yh2R?I7!%Loj_3e!? z29Szbg}xIJD-WwiL2X+>9R_CCkNTPLk=N#7uDg6>R-Z9K9ivOo=sl$oBgl32Np!Tx zv-wd_E8u=hz<(YSsp|}kt#D1W?)u&Q;|XN;Ps0+;?QDUH8>QA&^VDOTi+{$tjai=D z)`x(Nej@61I8*tk*F^O`-UQSuIfV()Hy&Nnhre|?mB|;I9Px~eGkkE>LE~#eVNE|` zECH+8GP{S*$6?6h$Gn8;6s3$8HdPPouFJ$@q%{?PqAj9PktxmJUqE>(R|@XEErnJ6 zB54o*m{rRj>l&C{Uuo$67&7g;ftsz0D*=7`vc^ILouA<2XDl(ZQ(3t!JrCT|) zG(Ji)^9(pfuz%C6(F5_-YuEeSFr*!XSfWo3&SGp1ndmD8gvYH|T3WTsZrA=TsahFy zeTTIel-(RX|75iUaj=={8k#tMVIvkymrXZ|bILx~Tk+2gQ`iv%HG@m|p<_w^2IZ+7Z4?N5zDyXkNU z8B+tXn#%9z#j|>JW=A)T{|-c8-TXQ60e;oFJ6ju^+N7JNBrhKMtn5O%R8wR$%FTyo znb~R&|EuLU*X1>uQAU)7{nOX7! zq_#QmYUJjd_48c^e0S%Yk`5>Ogx0*-7t5jS{?WE3WOVj5W;f6ZP_g#=rtd9~#_~u& z|HNhzBD`Y47xVp7v1>(sjO9&A&S^cILBoie9lhcsP)L`F0dA6P`Pb(3NCVMv{>YA_ z!L+g0T?0MA!M1!D)?tzJDMgD;xzE3FTVgEh`sFdegE7-SHUnjquc0fp=*0iNfPl^X^`^cxWqn*|2uIWdH;SPY%GHrmfK{153ED)Rmk17aQI zKo7;D!GLF`=ieQON{)?X_ zhewg`muN7QJnTIQ-Ns+mMwj?qBm4Xk1ZFMqIrx>}cD<0$Mqh%!oEU{L_*!Q+F-2WM zM--m~s_Od}`U!7Z&)c#jj195j$COoHEDeHGaLPJ13ZTa_;-(2TFx_g2C6}*z|C5Ps zJBRJzK#3$9r!q%kX?rd1$B+7p+6iWv`?3J)dIMHyvxzTLs0YFYAS^dpqp6lB_x4Jb zr10?aHaM^GVN!}s?C#PqBu7sl^`CeqDVKfIyly~otw@sn6>q_(CG<%5v<~yMFIav5 z%6B;V4tr)j;ve0&7ZmyB&L>P=9lK3A$sv0xyo3aSUq%DJgnAYu*MCBprc-`SG`PEs zu#g2m4!3s#CTb3pi6}g`{TsVOp-~E;D3kkp%Af?cJOOilZAsyiDRMvjJSZ6#+us71 zhuS#GF%IfWC?w`XK%Z<91yH;BMp7~71Zi3 zdjs?Daq1ic$jpb_8527Aivw|*_~BDisz3dH>Svf1bazu4FSUPEUZ+YZkM%q6j5s{B zi5{f6-|(XR&zlmeW;EUL+jxvSeH4c^|K5j$>_dojB7wZ^iO4pi&c)YO!sNMBn~Q)7L9Zj5fy zF>-bW8pj=_n?1(;_wTd62*&!=Ox@MNZ`zlwt*()~t+r6i0zIb7gP?!UXDhc`kMkGg z^u(cy0e%k?q4&GZuBokgo3*~Tu4!^X2d&;I_XYBhVY$citrx+g8*3+Y0lmTuyMW`R zhuENR!%&U=39;@37w-$ucAn2bS?FCGHA=zcdA2Eb0`^YQM3i#pX6?7R2l5E9(P995 zx5t_boL1e{#rs2DhKU}WyU5K;^4UY^G_=>%U<#_wgnxf{zs$Xo$@;3v8Kz^fE4xoL zd}0bb${M?~5vi2*|B~hnmlKXu>G($vfT)j9;!;2m-NCr1AY;u0`YBc?GhHR_P|KJJ zW1&H*wT)`yX2{R6FSjcmIk^B0&_EmU#uQ}L25ySv@P?)9P^hs<8oaBYra$(ojhGXL z6pwvW`SQX+wNeW9G>-7jN9>Wk$;mapAv?!ki({M2C^I1S_gUmtj4)PmkMC(U{AS5YAPF zT)sA;3Bj_to0)5lX0gm8f*dlrFZLa9E7!aUu4bPS>&wF-jXt_T7>ev5bOZ@FCM@+W zU2yT!Zv9Cfx0Jpp%JtaLG&XBX3-Q@S1T(jQSr#J zc7Ck6wOxaWWCg#t;Y^6}t$*P^ZN1|X3CQqth47mM68u8pXty_BRPvJ}kH`}l@n;@d zu!ua56LmVz+8HSYOr@E$wD_7q0hk$P2U4L68?GunCx)CZMVr7&{C*`w*N6fJgSQ0%966Oua_1f^1V_0Ees zOWYO-eLpk;U#_rG#=M33tN3mk#@YwI>dHl>zxV@7o$~H!OG~stmz(%w1V+>skrV0Ya(NU*T9lVgs<8<4 zty`p7x8I5s+v{XZP#9#r=yism#^i?W3^;h~@`Vey@6zpb6jVENV{ObTUEE^V#aY9| zDRaRxq~?suc5caO8PYewQ};{{wTOWo&!2IsoeGV5nZ*44y*^JEsA&mm}U=TfSO;qJ{a$s27j*+TBf zaX9Vn#t^^bwC~HHE4UGmeMJxPK6eMO7i~U4|Ee%RuuRSP49)c2m`Bhlr#^1T zZ>4=n3PmO{VNyRoGLE%?cpJe8<{W(1*5TLU=N|_Hb1;SZBBb2yqxq*M3#wdxpDzy| zvXYUQo((U}$9>@JJxvb*jmt`b!seX*ob`Bl#>#A3xy}9zjbqI+HC$$P~t5P3ap5~Y;!^&Y+5X-4;+w`;C)#TE9+IP%tfYI&69oI z`>bzLJG(}S?-#*fMVG7;S92Y6wQ|*#?@42*==pR>gx}%Hc2y0|LRwZ)A-06B%V)Pg z_-N_*RTM)VNy*5rd^>ON?>$cD=+h=AC*7c3!2If}A`?Cw9mJd}Y+Duu#Rf5l0Yyr( zjD~oW<2wUb22CI!7%b4q^p#;o(abHFM;*4@fhtVG-6IV97!qGgc1(g-`e2@;&a-6ePkgkZq~!QF!ecL;95;dRcr=gN8a&wKTKGtetxa4Z7aE=@6&C0h94O5Vo1d2CcXeU@_O$2ZFgTB*|Wf0HhLh`1>Vj<(9415 zlnuv7dn!Ng*}cIr5(Z+34lyGDGqz{|LA5`F5&z&GYV!&kj_d|e#)xb{fT&kno;`~- zs&-|XGTn%L>1WSn*B9sWod@j41D!Wq`kyR?b@Bp_2Xck*b~o4qUN*W5qf0FC5~7R?#8eP z59TQ2CFgXEem4)988Egm4Q>(~5GR!$Nm662l?oZPGp(b*!A1LVw3q+YpNQHk|K%{_ z1v^rjuzJ{acVOc3=w#(F=IbN(=0N_&`=E2KA(4-Sx_P9!$8kS+n6=<#kR9J}JYEeF*>5wNlWMcbz2wZ4Tzhsc9tvjRJ9b7zk_x(VkLX`@=^ikKqz% zn@$x{5jH+?ywqF0%DAfg*=(lYb6ZuUeGOlr_(iV8{>*?y?e&{0>6#a;kZBcovQhGF^F!u6SDl`ce~Spwox! zV`4idwex8UXAYB}iDkZMH`2=;rIDpRFmK-u%7u+_|2@NMqldE#^ zXIX};XGW*_xU6}3t?~73eT}bkdG^d7Jl@_I^VaB8mEYGGe-pm%xyQJD2SSK=*Kby9 zj-beb%^*cl@;X{V1E=m1pD+PeuZ-4)FmuGkS%rGGLlUBtsGm#K$CajD6@w_&77XDU zuZ@9u%ZQn8V#=A@^U5trc6V>n!Q@-?+yRl-pSN7Dn*!%W=m`VR*(xe6W->LwSyubS z#;%D$)UV0C!2?^pMNFTpiJcN+Abj!~GrstYNmxfaWE3MPl%QxBNJw>Pk*Ry@2OQ=w z2?iKObgj0WamTh#6ZuP+Iy8fwt!Ypmsbhaoz<}*TqvwU+I-Uy<< zSL5nP>^VLC*q5<2TFcQ&=}WJcg&L`15lGo%fL6Ci;razTvZ$_`A!#^$5`9&R!r*6( zuBgu}EnX<45dW7QrG$F{_X`Hv$vUia`{ex75NX3-ZOHYINmh zp1(BGu&gOTpBNM)4hnrYee9C{It>5|O>#ADlSz70ty77K7iX+KyABDI? zlu|IK4pH2%zTaPJk31XIM5EO+b4}Bj-=_C@=1rE3!+l3+aIzx(1?jT?2TAvC_i;c~i{Dofz6kc4R>RJvM_=(oxASBAfh~UMe($C$}vWs|RPKGgE zT<|d}iO7mLV!CfNaaEO|8!d3aO$&jYwcHE~wq|{*(+kOx-#lglnK>P$UYYjnjP`-q zZk$b&?PJ)xujX6hMpgFb>R_uRZF=n#3TM9HbhG`1grT9fMb`lvy)U6;#MU&ar9wGW zBNJ!bk35xCRi@X!-#$LwbjY_f5~wKw0(m$H2DJR? za&P=H@6 zu+%t3B*IKIC>BY}Dsd5{Sr08k5D@-=$2^Xcot%e>B% zupi^{rQjr*IvlWY?jwLhbLe#`X~9rsIz}?_cx4(8#5}k2gJ8ky*Gi0`TU$gCpxAI8 z6k7PMxMY?Th`ILqkHqND-Hv$(wCKg%kp!XqKYF#6b_<-X)BXtSG5hMhgBqtQJ?AkaQp|<6y(ML&51JkH>2I0kjNB zu0&h;IJ#G4W5JBdjY)RfjWPV>1oPIINdGece@ZhjQQVe*D?-(E>5^CsG8Haw`Pnwn z7dR?VB3^?Nv4H6Hb&CU3mnTw&OFXUY3%wejX7>rt#<_lGjxtrL3|I_{Rcy3Hr$d1$ zX*;j|d&0CvdI%({uRci0<40UuVd(7p$6ogKAKl9&a*UB$M$N-&E+^0}%k)>y=yPYU zBbrewXTy7zCJs6~KZLCNO*3fxRMu4vDWdZ=k$MeB(V_uSY=A%=_>^kZe@vn~(PHz1 zXLvA31ft$?ou_?$gb^!9pbxsGsjqhjGv3iCFhcvxfm?#r7*I%E9oO7^6(GFA$squy z!P?OV&{TDX-gr`BKyZjAMPW5z{?KY5{y10IJAZLe&ndNtVZU`eQlS(Q8ykPNpV0vZ zXaj&W#7)s>rb=yZh>=m>qWj)0m`KNcZP#RD=0@f&=4(LACAHF|?hx-Z2e@DLBCk_q z=0H&glb*ig|ix0?S{yn9m_`FBcgVQHJY5c|u)ao?!!~ z1^*?XN3S&8XlT0J8#WVR1Lq}qqg?)j-v0P0@$8LqZN{PA3IW^|^rO2+rucvxo_bep z+kBo@UM0$$0mI!Mjf3z5kbY3+qcvjmyDg%iZ#iz|-TOaZi^WT%y=j^;bz=#!uA$06 z(4HYm?Ehkmt>6k^5%gDlk?ukh1V-a5f@Pp5ht}fk~czq)G~rvvMRxk3j#* z+%qYIq2d-^l38HN_boxL^G${?jM5=d`yxCW%V<)DKZO||<&_9TaaERxpfz4=EE8@X z6DcM7r$>p|2M{<#szSZjCDS!Sk22Rv&+nY2r$XKq`Bu-ZadSLUKp0;ZhAuMlSED{8 zBLPc!PuHx{>Spy?07Y`VfY8*`1*O=UJ|XaP_D%UmCZ_Jm$yeN`eX)HsoM^B;>rV$l zkPk(f>aX$+1@xQUo#_?dWpfXVyW6>8I?q3e)FdVJb8vXT&RTBz!Ej_r%%GKQ2Vq0C zb8VO)B}02%$EfPZ?9wq9Tsl8>3vt0JV}&In%K{=$KH!6Q4vyS?ZFbtMS~R~cG?=aKh{wYcT|a~u&p_D ziiFanYd;adBpL!hGprrBq)T|;k{g@FvT3VviIZEN2icesgz^z~`UeMESRwTz9u>FQ z(d-w8N+ihKjb9#2!ivZcbaQlmkrt@4}90nOoZsk2%w<>hLUR$-(bW zR&O0iIwz!V{S%AEQNB0#%6tOb`@V7}EV1M^dW(QPP_8!&rkEK0A!!{!)NV^#Gxt&~MfDqnTCo{>UDs(Ik5RL?Fe zPt9n!Ca1+a!#?f$wEDdaFtH;N4lsk#ROnE%{q~f?(k!&Tefz3`Ereu4N|%gTO?xF* zYtDFUm$aoAMr=ykPYu`CQwW)(I}2}xuk-9{e800m;kY$Eoi0lR&@=Gy2S!F$*ohkG z(WGWn9am8D0(B+97KZr~GrkkW4xTP)^&wxp*W{b%$JT@Q1UWmvUF%Hlyuzo8A6U63 zBwiH8<02WIm%9SIeRSzOk)uP382m}BH{^y!F?*FJ${bnh)8Cr%XW1Pc_hb_7b+0o( zv7D1rQ84?ZWg7+Z7Fa=0I!v*Dnq-dz)s&*?KGAtCce~5BNG;5%`Q=>wWqzA&6Pw%B zEVdyLDr8foHJFVS72jmzz`|@*Gcal9puGj-ffchQpfmj5eJPOEsI1KxiI?-D6@Q-8~jN?K?_j~qdGIR^(eP;@WMT4eN zfRI9Ss7m2_fGb$W2X85O05c0mN<)1MHEIMtbr1&()mV}7n(rZSVxLZ278ye6`CyJ2 z;1GL7{)BnIGlB$s?5B#sq=teeIvbm9Dq1iTji@J+4PA?5h!s|%ghr&zc(%_*hz!33 zzrnHmPCRd3Qf#|tA|)_FnIZKK;BShBLgrW*<~A7AYw$ zE7CcIlmWIfJ(4+Q9r#=jkY#eeW3ovcbgcr!YIo&YyQ~&t*<@AP%r+DG)Fdd>rQ6y- z1xmfmxZA_=`^6S_L`_XiK%?$`%VD*!@BRiNGD7I+__!N&80b@UboHTsCZwYG?h@Z; z{rTbct{^D&>+?`SGMEmaD+00y(LN&Mb^Q?tVo!tAu?Y#>e0=@#u4{W@6MH!icfaN_ z+%7h$$hVidUs7H~M0=`#0U6PdWM;0r3-8bE#h-rQ;bVYXxV9s}X)szBY56NO3R) zS0NJ2wV0a{{*cYZ`u1@Hphp*0^(|j6vfEU{CQBVI_?~MziPMb`J*P76dc_ltoz$Y% zj#@NeWwhNt*%ni=1*WTlRw#Q(&-p8zU~9o04s-4fYl9EBp$IA6SJ$+GxPf<8&!dv} zh}C%wdp+!`{48%A3S$ujADoI5B+1G|KSGKWVvcOaJ0@X4B}`+2xN z%T^3Yv}je$4s0yp6qP_5SEwnK zin}bVaak0s)Ph}AT^&yzOg_Ut}e<;P_ag%UWt$h31r^$7VNu;t7Rt*F;IlSZtDtSD=05#{*?k30zCnsn`9dmTVKHR67zG>aP)@YSg77e?VgYszVKi+`v%`WTs0C+ZW%Sn2t3(+)!95L>%HL!$Hkv$G zmekHD)|V8jX*ltAb;O2fj?3#r;7R(@wmt7lSzg;o2kOHPb-Npvg8QkeL$A-%I%Zi6 z?T^qkN0>ZjmY<%^g(EqeKb5|6a$uMJ@Fc-at?;EACoc@*L5N0n_Z8&xNKfnLE@54{ z%_UQ|oVvBG8*;_?&M|(?G=H?N+15rN_K5PN5$au!eU`~6rf`}SHuGiKC^YkwF`$)SAw*iwsTlE!(ez8Bl9ZcZ*cmuU}`$T?*wkrC(22WCN+ zW&&4g{i3pEu5byUVvJLBy-uNo$6aGzPJh$*E=B8>^So?zb%rtJ#^a(OBO z>hdP6Pt1|v*f(2fK0tZ(RdPRyJz*N|Y^Fs|Sjjd$4t)Ju0$RM=_77L7h~A?HGjCmW zNISiO3VmE7TzP-`KbSm$0Fb`AdVTe$de6_REslTHClpd4V`4%ciSd1$zRS9P<#s*#XscpDt-KBIs3bPrvhEd^!0mx0WUV4dQ&uii&Pul5gA=;c^TGmej z&Rx0M-~>BMCQtKYBHGAc4md^R#ATeG$thgik?mJhd}m!9l1J2_RG~bXxUN_TPp(J7$96!afvhy4=?Q38KtY zEa0{z^^;W5p&_LPJQ$V+CYYxBLC;`CLL|$~jZ*o&1!+VzB`v>vNGcX1Eg;tOmx<%F zA~ZZa47j_92n2vH0w_fgL9!hsAolkl*Jl!55 z*$CEU_T5jO7FDmti3Pl#E#b=7CLGCZ#tj3dM$AJ|57bqZpH)U4(sJvO|-#l&b&5ziWfc6AHsDG z@9(+t;Umo^5+jG6R^zJ1#QqN~-Z&=*FKnwoVqSDri0hjOHoaU0>epXvIXnebPyf6_ zn26BJR5J6POH>&PN)oq7F9_NVyzT`owuDaj z@Y%wfMs^5teM1)@&^P9uK9h>R9^ZS%qGYIKs4&HlQTP?+J&o_rl^%O;UV`)UodPeL z0%6-$#>H_nM*3Pk?OlcPa>av!T^M3hlMQ4QhtAHB=+$z!lhnjVkDE>bf>IcW1zYMP z14`3cN5(|p$(}&aiHNjxMhhWbeOAKn?(o;feg{^K&P)ByFMph;7Z!d1g59_GCXck5 znlo2d9+&SsZ=5SWA)WQm7tj%)oonPibv50FzFff3v$I_kme9EE;^zCEfKZdrT1G+a z&u7SF50~P5F9Dm3Y!jXrUtjh+&);``i@e)=$$UBAq%d2x%b(AGpFBVo6!MQp{w*b< zoCOU&;=;=^w#c8`e?3x22*4LJ&S5s4&ykOdGK6oQPl7hR=f0M8l;*EJ;um!LZM2Ep z^_1U|zXas2B9iSWIer;UAm4acOQ<_-3JQ;W$7HM96oGiMCbgaR^`bYdE7G;mar5uM z?DEksE=rE8(%TuX$){*aYTz)o<=~U!2J$>_tnAZR`V=aE5=;Ok7p2kMaEM!dX zkL|6PkwqVWI-09By}{N|SfQv-?frde@oh;XY~AsShdQ(vF6Qmgui0=-4RxIGxi4)M zgJYareDabP&~rX-_lI~@2RvNa+NtTbBvo;=0)YVtTmPaa^?p9y+_hE_0}7rJY)+FF zh^&CFJ(U)hWP7?7sp=E=x`2l8Z{N< z7J;>HYpiCHgD9(^o_l??(1v~3jf{X_=TA_Pkx48l-`f&>T*=H?;YF0QWaj|~C?X+< z_AHcY?7F{hbgq!sz*#;%+`l-8%_XF$POCqUUphGd4CG9UnswhV@hwC3%E|y1_~l*# z$c<-aMk{p8vaq!*&nqaIdT;M;dzKE8Mt84j+Bgy61p$H% za{KX@g&9n|-R@K$=hA&R`pCUA?~{*Bec?q6x{fMWfIudP$nYj49ydB~Zj9K@U04GL zpVWSCTHk1MxsbAM4u#J_H*U#}GDF`o&=p#QY2jqtb@~QWQ`#cX+G{X93b^zK|J3(5 zRq?<(5BqGa!_%SMd>%=VC%-vu0Sz4fJfmYRLW_2>)L#AB?$wp=%_Mj@vXT}1mJ<nyaM3*=BE4ohc&%TW2<(X(K@6hC$e^7Q&a%V4;V9b zoM7+-He>eLy*;DmbI_l4Sh31Wj;ohb!M%2+_rVA-P)g*)SDd|E1h5DWu#KGY(?YIW z*2_Exl%`M?y~+&Og-9SB^b_#Y@03Te}We64w-_}>FT=*uWTu$ zzy!o^dEywAv^DlIrhn;1&W8+|lBPLwwAmuqDX5AtR*W*`%bivTA@uuSebeS9(HYE^ z_1$lcD6_L1Gw7;2a9vlRqGpXVwZ~Oi2?P}SW_bQS!BO#nce0$4-FIthEQy>{q}}_I z_`k+>FVlcRmf@Kh2Pc6q4XX;T%FU6o-_q-sq6>5uANxXSU8UQ7CBUCuR!F;D`)-U1 zi{s#UzsRQ7^x4tM@*Poj*#@|@roGA)S^F8~*AoMS z;Q&A%EwckM4T{jeQ3yn_ADI%MP9H6Gl1G}Ydk8e+(OFq0g2*SX4rcCp=>rRrlUp1X zT!eRaZnoT7GV==(%Ipl0dnPfZTzE@T{}$_ql0PCfQ7$x8MOm7#+wYcX^4T!8<*)Q9 zv6IRmp%>aAQD!1ZN%?=3J9&*?pMS>TULHv};S+T_Rg0eE60PzsG&Gl5brN0*u4J}+{=`~(e;azfXLvX)KB|m?te8=T$^2!=#BRg9-WS?e z4LgzH&P4(WhEYA}kw|&aXTasQ+KfuOqubyyy^swgicBPkB9DrNsp*GkKtwE19H?^~ z&Ek%99f&6_E-yz^nMnkY{Yon(0lwcw<2E1dVcI&=1}%cZ-(JG#a+WmVZz#yI8$xFT z($8jIqLAV)JiekjE5pKJck!@J&rG)j*a@C@Q8l>0Pyq36n8dt?R2PLcOTP@JVaXv; zL}5xM&cNtm%g+KQ>h(8;#6nxDq;?d}+IT7Tn$2@^LKBIZ>C%P}1k9;O+MaDywQg)@ zbq&(p%4D>8l6|9bI&HB-8o*U+U&BBGBp6|3l#V&x*if0< z4dA0zi^~Bw+S*@@WgU+R<>cfB$MU*?IXpN%-Z5Ca_cr!%eUonGD%dLA=5vacJX4mE zfj_}Y{zT2E26butgRdo12LM%e9my-B+zFyR=k8-3kyO2OIB?8L| zi-7*v=HR9$O-CUa(b7YUe%F-bJ2mZ1TNL-;phWk#^n}MJmbp#kX@Ky>^sf-KW z%u7g9S5NGhY2|M?x)1eyA`b;M z1i=Ory_F8AGqGliSmu=Pg1*7N43DvVwd_rXX9}H=2L0$X5S(jS_h_@W4iP!H6L{-TX1%q(8jihr$0%I`>!ja}&n)xzh?`FV za@um+=!ZcDd6Q+8fHr&=cgasNJvKCzjm}#kF8FF9eqo%9g!m-CizfsXdtsEtMOOlR zo;dsC{%D`|ZmqBwx8|zcuzb+{;La;AD9Y7+Vnn~#Z2dZ?iT#5*|9h_jQ{zGf8tYKp z;`$8^`jm{?cTYxWd>!3|ClsOIMKfQcCWax}FW#*kS1r^fwKYyy8QOjdHWAa11%ae2 zsycI?HN-h{Vk~28Hv(IG#MpFvN(#vA!)k^qOBy)2c5AQmeGNf`mZ$Lp`&Z!v_`j>y z&rM?84oNE)#qbmP0^|#JNPP~NQ_nNacp>t4^ReB+rAG$$$R6cQG<_e{mbWi_| zNYlikKi>GjW4@~TN5ng-dV^q#O4?K8dd6(du`~R5CLP8?OYEE`E|>$f*}Ez<=iC|D zl5J}EX-`0Wab=YDIbSY+&9Tno+`JL*cjj{-#?xPmVLrfP~AKwLG z(>ebPdM7R}+6QmL{KqYQ-c{A&Zdn11w^Jr#ewVc*+z!Soth#ujuDCMx8#A9q4O|Is z@pW3B79Tu8`M~4VxBSvCeG{<;{OU(@ltQD=FTUC7Qy|dh5tz@O-+^wNl0s;-%S1#W z2Mn-^9b_gHTBR_JEnL0UQkr$XSnQNJE=+M8zikuT$r;e)L1_3QB3*LFD`b7Yn}maj*2 zs%_F?Lqzu$t_^uZC0$x+%WP2W`L|s)5+m@*id)|!#VBn~^|#s{lG50o$Ft0+nRts^ zbTK41)fH1l-21uQsYq-(3UxgTykW+7vVJf>`N!AIKE>NS~KgsBZW3^MzH z^meA-q$O?Qavc56IH>Vc^D8mz{l1v`s$}%8$Bqu{*DjLHIRinDR2C37$%cSqa}2rUdE zqfq?7q5Vy()YXYDBmu>Nwo9*?83!IHn z*Oh#=0hr*WGP{=v!C#>DkbI#(-%aZ+hqxNYBiGf1-+SpyV(*5G|H|+k`ZTG!2Uyf(Bjx>IG?p1v4E`TKBdTM3a6y-_pChk2=K zVylRQ-s?G*yBIHnpquLGu(Y!$A+=pD+k;ofjlmVUuhO}^H-2(zT56}_)le4_sJPN9 zb*EChhIKYBDo!I{-*?&XwMdO3#)vltKFBDhL#o+A3t)9 z+;n0!Aa=$Bosx$R;-$)`pfCgn6No`7)Vmshn3P*NFxQC z2>{_TbaHj7YkwaJgIi+xOc3knF4H=;-ev;Og_z<&r-Z{+00)z$6PTgjMD;h@=E{!D zA;o{BxF(_T_-)G>%uV6q3omt%4`xhij0^u^D-a>P8(wiIFis{(ZacuB|AFsUWM^A& zGxmfTi+)`BGi`U%asq4CE36Wk`~I_l>q;aEF!U7-jLM8H2K~jgC?6zZsx%yj2u!S& zOHAp(mTZX0SZfgo6W2+FM<9!Y4g&WhCw{<-+#Q7BaX|M&RKa5zr4hH`Vk-yA|8!K& zm?+^K9UTFFc1m(`w?M%}fjtZkZeEpUyH9vQ()t-jzw!?@4yi{I z-Pz^ooVlE1$HTWRumq9>3>J^HDFM-*>YFKuc(fVPFXHc4-uGE;wpfS-j`U((&>;fJ zsrcSa{!8pDa!RJa$XGj67eH^$i)&Ab*b^}AwY&ogDu01fW=5(PHz#bY;O-6FwptK(w3GiSVZGp zWg}7@$6pR)kHc&3Iz8g&2a#_$@pe_F^_u1OLFGZ$)EIoO!OUV% zgVQ?>`}Fbtfa-M8iN{2Y8<@GT#;Vx{dv)NP_4@Vm$m(c$hs?Aw{Jxomw4hw+kVday2K#<%l!&eKAfV<;mJ_DQiAW@1*DH#-d!iUClN zIsj&We{bjD&|SyUUrIw??{NUZ1FHF#)}Ape#K<#xIMA@LQ-F9InwXjSL;N`?NX=TC zkNrp(d=TPmSw?@}6!h?hr1FytQr9<{q*zp|1)fLM%qyn6ZR^maZUWdCM=sTO%7Xjd zNcQdc*=0#7d3^-|&uT+z*NKluofNN5hY=^GjFaO`!Tgub{2_w9vEbd0?sHReVU~lo2z$MSs^6^zsJS z$PJ=cow{^f*5E+~UvEtn_6n$^wbBC*RigDsC=SUcS64d$?6vrbQyZ#O8o9fbWeU_} za~cAyE?O*Z>nTjni{%fIHh!XTEOAgAq}xPD(V-AT5hyHStYtwRX&|`tg~r6#OqVE! zfegQB$&00|xI?qd%sEjh;aUsL$t5z$(l%6of?yw-vSy!`LFB2yKw0n&JrtnJ$^i zvGN$13h>DF*&*~%mPB2|8@6nZevcbfa79|pZ;^@zozMVy=p0T zH&RC^a4+&&VqcYO*Rh8{Kv7MHIsTSb*`zTF4z@dBK+yeOAcNZeUJ7NnQN8UnGOS&+ zUB{^@=izj1ykCV_z`ZQq`Q;&q14x|9aAr!B*gG|Q3n@EjHgq!Q#9nG_At|$S+n+Rx*CC_lWyLL@ zk!2vTwDiE4;mHEQ<{iE4u4(Q$(-7=9$`BCl7pnN!eO^)PSwuj9ZI`6s=toWPFNlWa zW*E-t1%vK?sa+1X%rGpUu>`#tFjG z<<%Wzf4eG$1n^NK%s4bcx|EOh3hYNA7)`;7# zCknk(kyCU~j#SbAB57R;D_V7Kya8_efPlaC6h@`g-X-7|ZXq--K5 zGTV;iwCx_uKFPL@j-{<->6K=%uch~7+)z?rUSCL2((r}Egn`idO1YBTUiDxj+S+0{ zAZJ^fq4Qc`{QS_6w1BiqZAay}+MWIcc2C3|v8j0)=b4^a+0T|h3SK&vQ$z-@Rsxk= zS=`q?-^FrO5g{S#B3%YxMoc7@rLwlR90`seGJ;#~XK`gRyLA1!i8z2LH&ipU@RQtF zW+=8XJt<{g6lUPT8?!Wd%SG#2)Vs38mE?^1P*SiWE(vU^f5Hz z6%8b$44K4TatEau;xn-ZLI|D z-XCqZQftYmuRIL;)Q#pyJx@WZ^+WQiYY3^-C7 zb%YUJyJOz2jn6FCHWNiID`(1CV~))D*h1$74>?rrqPC=l{tp+pc8{snue!q%TG1!=n!@TO>>Gt5!slT+p)P(Vb1 zk<~~-)e8^u>jMHzA{N{pav~OxU67UCaRoej62y&{hfyJ}FZ1vwi4-|B6^KP(Bstj9 zeuo$wDbaOhhOw$hxZKfT zK_`y-8hU`tt|4OuqX4Pu{lkMVy%Z9c4XefP_PE@~*&8Gb-=V~&L>e2|5Ld_r5ZV0& zHD5zQK#$o!Xe63^lB4z!yU3iJjn=qrs8{ z+D4P2?KWF`TJjLvD$7szXxQ=xU|Ixb7>zkHmahchzFNrdzY~!*au=sV`HwdLCbCD; z+?=Y_$)!FS3>TtAgB5IA^!J1zigd~AB8^6iLZ3a7YqV z&FLkn?bwuXN?>UyXuxa$-L4ewHCV#-gKU&j zL@#BxK0m+E$_AD5$rt6kw&S0O$nEwq&Osll}k_SERXUUa3I^iu#x^=+8XMax~&`Lb9E0c^i~?)`>y>6 ztqL_Zi3~Pd>5BwZ7&S>?HY_9_2RkfW>RTDoz$tL71}K?0t4x@IVF^%6=;JBy$=(|GhPf zAyWb!pPWR8E$<2L*#L?bae(He-P$wO;MeR{GD^dFEgc05{Kg07;U+ z)*dQ>gdQpU+S?o{t$iUFv>98aR+>wRk1UoJvQG%rY>*QQ6lORBHV}=KZEygVS?={NbJQ4JK znVOzn^!I<@U_`nKknFd{hjc$MV?jS23IQS;+);AHNR&WG^Iyif?C-EdhXq|f_fcDnO{xnw4*jCLI;Q`s&jdZe3e%m_h{@Q5yl>tl`h-N>MkBu4fC#c+N~Vepsgzax>=v zxiP7=X0B`q8O~LQJnS7nE(!ou{Cc1b+!#bUHk+k*~wf>SYp+;y>|C5IW>1FMs&9_|2GhL+q54GA(0sAxC%Bua7 zEi7IhsXd*KloJIf15q*5N5rxPb+K=P_En?li6z3uv%ib{{vyL~;RDNAR^6PDu<}-} zPD%dX>~15p7Z_XugvG^0z$Y3bQ-Z-@i5VUsxB$dO0ax_56BMkhal$9fp}v>9Nx8Yo zz!U)Q2EZ9+Czi>7aRGp_7%WZ@8?MkS2lAD==|F_u?Wu%?9`Q+hZ%0BcRWb&Zq_w~M zKw(*kI!UIy{~L(!A&PFTN4{x-ew<{ph5V72om{MEvnbq;K^hWdwWM^jw(3(Df z{tRrZhY0LP4v3InEKUTblL~vCZN{(ze7L_;hp|{o;CU_jcqQ{8B2i@&A!nY%`R2S= zfqyAo+4?xvXVbH5iICU_`qlq}hAqxS$#kPE;R2dawUOtuX~gn~~5Pq9p+fsMSFNB)p)q-h#CJIr6;DPg_{uoE0p>ET6chGa5X%y;ElkIg-w@l;jk(b_~ftAcD!Rc7=jqpmE#n zQ3d&P)@9}W0-Aa~3tzYuc$;D9f)nHH<%E!d8=zo%!Pk+3GJ|!m@S!I%h6wmk&2V(@ zH$R=-{nHiuRN7=CAj_~;|1^TH&_dUCDK(6U>7yzzRYBJoPD<;OkfM5edNQnNZx;dV z76~ELutLy%S6xSEAPtZXZEv~!M1HsUEGMkwsI3w{SRGT4c%>bd59T_zr{x~$e*2+XExtCJpIJi6{L=9iiR{jtoc?1 zp_2!iVls9Jb=;~6GaCE7xpBygne|N#x*bHiRSpM(wG`TaYCB%x?y7b@&e}u>uOu`Kb6y z)YfC8rRzw)8(;NUS=)tc$Ab?oKQ3W$t?P#ZEqKDTcIJ^fpT@d^R5(O~@`1ujISg++ zRPE|Ao+Rj&vg!Ce;_cD9m{R})F(EM-rq8e{K9@5p&);-$e(k-=8R^63CghPD0b4F@ zj)55U$~Cqg8cx+4D#*=)J)1F6BO=}GX5E*zr=7FQXBhZ_|UU-rAvPkt|~ z*kEq9JUw^or0sS7#4z~>Tf5_4;zk!bYYx#Mriydb)xmDzNmci+V@KU0B5a}Fc!f>p zB*qhH(Ph1t+L-#E?V+XHKjIQcGx7xg{9vRO?rFJO1M1xJ?Q9#Fx#OGNN6|HzgFYe z%?pLG2ZstIz21>zzZ!iddhF0Tk<4ArMQwwnDt)$O86kII>sT1X5`u$JvFN+K zXSqHNZ@PJWT+&jU5CuQptDs!KnWmKxdN?Ic;#4|=zoD3>!n{k5F212?<-?`>X>DNe zO~*i*qW<>??mKP)|62~7->dIWclM`>udc3KO3PI9hG*2o$I>o*M3xOW3JMzHT}$!G z1~gbyP4k}Mb%r%A-jj%lfR&U`b(r}1@tc<7{-1LvM=K*m;ZGyGXSUy2OUf=~~^^yzTU@Z&6*>RRG*HpvVn~S!n#mCzKi%=HVVvZroUJOT1fgAXRy-rpXnn>XvSPM-Km-f+6otWQiB z=<$VStGY1`-ZY85?S}$l)gwM9ap|`~Q;$;KNl9sn)8|e8@07(}d!KK=b442?9r}H? z`fEe%?Q0qt8uNDHcEu=lhHtp)J2`D=df$&!^pFFUE=FQ=_~-lys>iJ$Fn)SP%^TrJrcDjZHo z5-thmhu5{IuG(!rME=n?qGzg`d=)=px^v&A=HEv|sG!J7LQRB%gNA?{5dz-{D55pb zi4#a(+CaN+ZG`+Hk-|dYr6ofOI3XGF@StzH31n5p&WvFL&fCs4MY}${aF$%(SUKm1 zuW0bWm=SJm09MWLvw)Fe2S1WcQZo<3X93@|f~ekFxW*i@VTk7+g#f>*+QfztO!xLW z%5}E_S$RRfCq@QC#Jb`kJ8zm){E-uqcncyvxGzg&BJUHD=>BiZverC>SOrc6DlDKV z4J*N*v!+;<7HmNt?wYseTOMIVWn z)>?Tv;iSRjH6@8R;M|l z3~w@1K1<#*hy2&vs$0Uhf>$+1)7c|zOc?KaQEqRPtVL_wp?PgVsj_24n%wg%D!+ z@n4&NH7&QIWlQc4ffb3M`V>Eep9C!x6EE{i9?d3B9DbDX)OdF! zum9(I>vC}6aqOcTdGdx(`aWxWXh`l$zj$C9WGou{?PG4$%jkzvT6+OK@lnyT=e(Ea zmB9V`91;I741$DXN5P3TprIldcm3R@r~97nfk|*~#Hk$4^2SU=2pCwm(&KFP8hjmw z>bKU%*`H8^k$>75#8z9$E28a)s=lT7SM*$K?8GXgCUzDZQf8>MW6v7j-(i2-5sFu} zHo5_3*t1zCH#5C4XfLz*#LvrJVO{62sX32c+ZliU%~0LYmha|T3#&7sk?5qNMDMV6 zH%*hCjs*U9mf(lRh~EX}DXNP5usjle-xb?KFwht-Mib)R82ZKy2T?}!GY{}?(@*95N=(3 z^~3l=@ZvJAV)jr>y@h5jR-RyL`>8JM948+Jm>q(aS2>iOG&5duqj+bx8f>ISCMxs2 z4VxR4_w733SVjx91R;~Ce|Y`v)C7b2f|w(^;5kk`>Fx?B1so1e-*R^cz*5%kU?Y z!LHZaU&$luLnlCJKN~H=3W9kf1Q*dXJp{?}e7GAtqCAoMyQ#nWMf}Gja8DKc@Mu>N z5MD+puHr|FrW^FLE&F~n@}k5H8hbZ%Z2tTsG12{{8T7@VSIVm%9tUnwTU@ri& zqFkPbD)aO6z^bVitP0p2BDDFnwXwgZMp>DL3`5JyVeEfW!myG7xL zp1>RH=5(u1o_TR|e!H^IdlN5%mhqx0-QRKjAG9!Q+nsPQdQpB}_fJ#al+=h1SK{0} zJgxMoKK!&Aj_14ByD`0|eN;PeJKH3E*fkA+glOlwe@hE}T3UI|EW67;bT$xEdpQ?= z8cGE6dsbqLoJ4Dq`=MTelX~6$!Zu4ZRZrsTo?9=ELV>lck$uDYG@j#QF>oy0gUaHM zI4_rf2muw2mg~%(-z0%x{$Nqzm*gHhryHJh1^H+5RAH-g41EmNF1s80t3TpggXEwzU(i`n-HV627ZdGr! zbNZCj_NHh4(DiuFe8($meU)9;JRtILLwV!<(w1MmAtm-4*fSpLs2uHyG(3dpcz?R} zL?`a3wAaV+Lf&HtO=zh^ooTf9@n7e3*S0;Ry|X2SMq1;ibEqI+VC%CIysA- zxAh!4kCHvgP#lf$iAObBE7|aN4Ps?*Wf2TS&R+ST@IHvkUhKfzHYxj6D{5Qdy6B|B zA*di(t22#Fkf~U}IHjs9F=s6EIA~|w{EUHe-z^eO*e2BQ2*>ufh}fTS@m)7-ayavN z^dS~~zCa-gm{x{{gN2vGr}3gi?}ViKdx{?hZ-?V1Wx&G2lh%e0rV|XwSdK-d?iq&7 zHI_qALD~+lzm^ZSg&}#3j)EIU6lr<^EeGRu7CsGL!ZtYuL^mU(6jdS#nkfmbOzrE& z5-cyD%rNXvu?kh*a3P$>Fq4%dCIMU`9JHI;ux!-ntm?cbJ?ThHt-kS%bTl7neC9pe zM+Auo^PwsZ^=hWBuS#iQxgy>=+WiE^?j%WN`3!H1hxX&!woIL@$cuA9P^Si~fXc^@ zuctQzRlbY$6Q#3AnYv=u-5V*P?449y96Bo|&*j;Wd??!IU;<5TcM^p zOi2wzdV}&S;99-QQDbPwr=tgxsqxEA&bSH4g)D%G&@3D%cD;g$i3-XO@!~_VH(Y>7 zS`33F{X`l8j%1phi~(&2`w12safM7R~}QLMfoYli~gyU{a+#i-uUfzXvinN zi#dO&g&)_EFMs6FO^iJ)N{`NYMlBjib#zZ}|8QX+sjBDQ@S1S?0Eg|dXSQOwRn{Eu zec(1y=kv!VO_j$Enmme`w{vCteqnB=Luqu>=6jl7=E-T%0uQS98LG&yT}8P+ot2aO z0u-zY1NgCKLb>m5F)kHs@szKtWp2OwDE*L%c$^hUj@AxLw)4il7JCPB_ThTg)od0+@-Zz+awr61D^W*PTgy6e%iVPsFtO3rm4EZSR#Ue5b3W41-Q(p*+v%`( z{l*I;Cc9@$SH$m;Z&TkXE02cnIzv~yT#q6u*=_p_hF`zz&Xms%KTPN5AMWgC-%lE{ z9|gXQq2Jql5D*xufu6tnb}c66H%19LOqsn*e^@1a6wtCMh;iL=7tTD}8uF{&9P_OK z$Zh!~dj`ClosXd{ZhsIlUk6^HKVLO$4H3q8(to(~j~tP6EiPD>E!&(}3iL;B=_-#J zd%QUO@p#w!@|!Wgtu~wgd5k{b=?dqY$As#l0;>Oy=^Cz+iOc82jj^-mV`r5>SAOyR zbUcpg#TMU{@6tvO%ih-OusIEi*E!x(>+a@4re@E z=2mrIx!!lQn1l-YWMBsFW=H)X`F7LSv+wM!T`M9qm~^Aw`n<~yEB~CA7{EJ#u+kmc z(p{eGz4e6nL98v&D}=W9E4=B$vHXc*&2Hf1%T9jHiQHSkQPY=+yV~^4-p6MrLX;Y@ ztP1S-qcIN*aH@|IT^TG$ohQZXs`TukiHz5WPQBfFMy2%KEiHzpwi|&jM%T04lxa2}6 zL##ty;F}yL+QE8`sJ;f|&iqekU>oONgCyFrhfyhO zw-H6P@vdc~Jwi6$Fu!(A%?$4;}Blz$~C?wp%#$1N9KtQE%K7fphH zPOnc)X>0!_cj|k*z#!Rkx#TcD1PbDiZTf8))#jB4SZyiTa_-3dMHR;o+?m{YmK`;!X^;558`GKVF5E zYihxa^gX+I$BRgjJ<-$7EM~}#MpZ3ZFJ#D9Jn)C1IVYC<`RR6V+S~UkbSI{{kG@mR&Y@Y(|sw30He%NP@7qS6Ev8c0a7UZWV7qyzK zsm*os9jr$R49~SMdh?r0eZ4A_WtPqFX4g#A?Jiy}2W7}fzip~nPGNa=6Px5gcI#cT zD$nEm?;?Nt#(=@GaaN9SH}R6$q3YVzgnC|rVa z*Y{%ks@`T*T}e;7@V+K_eP&kchw*mO;pPzfr1bW&qUN$uPk7o;{BZ1`2cb&IZ z76A!rs`CNA^6T3c#~08{nm6xF&~O#2a2V(WeP(huJUM%rvhOPcdvfPzIky6wvTW`^ zTaWITZ)Q{}{e=nn@IhERqb=Q2cbMqV;w03qd);b3W8Bi8B;9wZQ$McU6dDlfrS641j z$PI@n(u>Z&5j18cm1g~;<3bEzvi1-Yx_-1IWapaR&J*~y>Moc59QO6p`RV(V^9Hz9 z0;<*MAHfj#Wc~aruWLUn)c@nU&9j9c4gbpG^$O)9mg}<`j6-R%_n+@0ZH0X>76i({ zaVPD{ZMEa$UuOyLfBx_jsMl3;IdXX~dQIMadkHEza#w21cMY)O@U!$|TwfcOH8fS1 zzG~WNpHu!=o?0?6^6>l(zrNpmn7upcJNjm0ledRkMWiV;Bi>@mtnpSqPa682xwe31 z*W-`F8^60k@+zB^1h#iXRgCnc>d8({ZKn>+NT9K`F8>Qku@MW#iA+=+RU1hx4S0z3 z)*t!jV>@Dx?ikWu_N_^uth z+&NX3;`uKmn43M>pI3wCyzm2kM-Z!9f8+f4U8a6{OZtM^q4?=xMr`7X;RKgL5~r}6 zdrX?Lr`cA^{q?u5{oueqtB8|Q8KI{41bzWOu?ptt$P7UNs#SwSTtV208Q?x zPe~pQC|?~%IdHmQlpPI!gAJUdEix?9r^6c&d75-It~u|J@+hgt>iE_nI&D?y6ze>7 zED0SQa4Mn3m^}!}2)b5=MG6VZnt{Z#^N4_h+X`YMU`Y#z%IFw%jnveXuB3b|P`l6& zRT`Zghb*B6yG{xj&KS!>3r}yOFKKV6;eM^ha#u|+9g*5A5~V_l(pn!PYC3{eO%i!_?N6P=1uz?n&AAj#Mp zgitTm^9ddWAIH6i2(T;a{7avOPMEh(hXz*AM? z5pBg0D6d&(X1Ig}rM+~p<<^3r=v-VqA?4k$%I-Aj+D^SzDLy4p_sgpTiimK5IyOVJ zc2OatCkipxWt*z>zo?##(DXzimX;DE1<_7;O%(PJ&`KB)C>k$m>kg2nm$0PtSl&~O z+Y_-|zC}puH$Q}}l)(`^HCkGn9RYKwH!p8%YLXc$98vt%%Z5C!&4l+Yo;#IEQzb zkKAUJPNB}TXu>;L64TXM+4d4}AH|yf*WMh@US%b#RTq}VgLUyh$RZ<} zDz>lHZ^)l9#JNp|bG=;fpKbm_X6OHCEQzjOC`>khp+#`U& zY8pbTc>c8s;4?ALpMZlUg0z{30Cp81(7dx_4%nRB+}w9wUUk+J1V9uO2_BYA#0M*T zN)CWKLuuX5zO?KDGH~a%;>z^NJ2||Y!)ISRgH+bZ14lN!FRLO@T}q{WL!@&G%MF%c z6O$)AH6!Go%%7zgdmBp9P#e!4CIbD*tKUV9yQ{@96Iy9uLxxktyFbt_P zfac(2X3=FH#)ZQsChZl+#Z+8@V4<{06ZS;0B!Vwr>GL{mgRpEX$Pzc<9r9AL%E}+f z{r4*`N>yTbnl4_cPZ7H7IK!NIJjmYzsKnR}{PmM>KRiP2Xp!5V9x%fC#x4)Fl8SiVn)iOM-Hh0!| z0~j5gWR+=aRWyNid&@(~JKN|*>ZOJa1r<^yjak9a%*=w5Xd(MwLBR^J@mg`ynpTo@ zkeTGycxv;1OMX*i{CPqr!@v$Gx|wW^DJn=2H@J#qrP+bECk}5#g(ws-J z6QE!)d?*_;tWZF-$v~3egwWNsPcBwv{^sV0VZF}QK#rUF)LwbnP8KC1Es~q_Gf|Rwfwm z4(m&6DjVsk8Uh~5*-EUXp_y$67$Ra}4z*gECeq4=xCr#Y zdT0%q3va9#3eC0~nWq#?;0D29wfR@c=9;h2!P;P%xNt-?m)^v`6Oz9(lM%;+ zhzLmt7bAC%+FY~uRWzI6;p@Ep<3SP_!<2U>2llsLjOL8vmrg%b7f>ufUO{}q=1RcA z9Bt4T3qn8u=i62*?&Vd}KxovFGXu?v@PBdvEMEpNrNyMjQ;P161vSEdS3qwM)>NpJ7hhiDnl&)ZvWLBX-FUrE^5*o@qV@PV&g zKZpRjQ%>JHGXVw*;_KIfdPhCw(V*l(Wv&#by~7D>40QJxD)YJ~xIa~A9{NG7S9KUJ z?O#ZheZn4lUM~C(tED$?hfHVR9`8TI9g@(lA?QYbS=pMlz8!g(xj?(T<}UU`zGh}c z6YVGb;TvXaz4!B1+$D5TfCls1MDCVQt~4tZm){t=iLE*6G* zuQAkNvR&Iky+1neOv&H!KWdh3Wf=F zl1-yxi*@R{MtAyUBk){fx}8t?2Z1;JBlNHtM@?DRO#uam0iThGMOC2_MG-U0{N}dBceU-0VF}<-DlCi03`HDuasEZ*wXI3t_JmpkW&0xQbR9yjTcSd9)C{ zy04p66S4#X0ab*7)3WQ!7!@LXc{ZFD+E!9?SZ0>FrPfijI+G?m(L>l6nCwYfzHc=~ zbhJrjVaxH>L^5cpa;a0N&6+zP`vODZ#83#;Sd^Sl9##GX4jN*TH7+EYCJuq8O3Ntq zT%hJvw&E_GHE&gTNz@uRI2C_+sDU}thv>y2R~0m3y?ZB5*Da*}awX<(fxX_+q_xjS18dJK4`SPW|2D}w z*b6!pv1vt>5*4heAS?ZP=zteqZzU>HkBIc7Z0qf4^&zYYFqBi zsH7T8K{@MT!{TL*kyv!;fdHE+32zQu^pO?xb58IO-bv^1&4CB;d`=!;K4{{`ju|*! z!ouk1{g3w7ULp+nGzH}%`K9(xF^zKcUkWmP5-M=^R0ij?Qwc{)fw_hK+Ypt6BaN}@ zgHhSJ{|xays!(elWVQrxoJfn;Gm0MP3|j-8d%1WV;=L^a8`9w2;`e@Rg^ZU~AYne) z{f9-PgvOi!3x3eGNVKmF0hp{XiV?OjdKuMJUeyp?F5!JPW66`Rld6Y^=i#-rsr{N;(@L?yv%IRq3#272y;`x&R|5$q(9BooI% z-5;&m+0&3^J34OI`e=JmEW$M){<2Q!d#n#@{y4Sq#!z)&Id-Q!U^;AQeK3`ObG+`w zC+JZ8BPJ;sN#%eak>UD)-35I5a-;jE^NRb+Hr`uFww29V7|(#z9cfIEOBWgps?v$Z zFxryQJ4KrJ$RST3wn#INPJe%B?mGICay#=z&kJtldxgx7jsg}F2cWt@X6Yd&pTI+d z1dCq6_f>f^>YpyI#N656J7=QnjDEhVpSS=@o;i30vu4%*y@?ZiL}RQG^vMqXFg@Ca zZ84Q(7Ki00xp<$&SLwUycpp{7M*oByDynF9rLOljM>YmZX_{6+=clw2Ot@ZZgsI`{ zSIgFO;5>R#wn5vjI2FX`h-Cd7?m3AhGol#7b3ES;#?K1^bn0#{Q3FNDUxkI-0lBw{ z>O#7jI<+LcNgKGHw63s(YAJ9YPpfS>uexY9pSC5G4$Q00UPC>B6GG*-3<{8{fC#0( z4hQsGk32qSMOBTCt|2vONutq}G!l?VDD&_%hqQenb~NWLsWvZ7FEc3j*yTl*SvkBc zQCa*tsSqM3mM$$TlST_-DI)uxIkY_6H~brgwO)z6EQE@>LU8w)*q@kCR+k5HcXE;g zq8(Fy9re_0wI_*~NH$!j64Z%GTBQwrbkN$HBngrDFQXxW z&5_T&PLA2F1l5TW- zVU!{?$wLssjXt5EbXNh2F6kh{**eGl=QER<^3n&S681p{?Kk_n=Mf@vZaFe2|4srT z6IKA8c*=WM1RD9*29@6uCa8ka7(}0uOrPC@FpX;EwnF_zwYxICjoYzniZ35M$grsN>vFrU05&}iOi7W#bJnc>($7L_GOpDUBBI;!!{YGBQY9t-L6VJ0vy z+oc`3a!DyLP<&}=>96%56Mi4(#5a-s?I%5;DXGMrIq1-2(exImSJlyh&>{m#m252= zQbM_h)g3=}H!@ZO1`v!|n{u42GJ+#7EG$Ayv&?4RpsvVde_ne+{O0+mzkcNwJTD_kUd7ieEZtDZg0Vba-C{CvFVAer6vwat)iLC#27g+&{&lj}*6thQXBQ{N#7vl#alAXNn(ZbPuEIK7UHR z8_wV;ZC75;-zPpO?i&-S3{m-~ZZcHwEKhh}=ieDc^eVoM=pEI1x)T>eLg3Ab_SY>o zKZ62E4vM~HL><&d*h zRF$0jveT!E$zV{h#V$fhJwZhphWllCmyaWC#K;Zn`-D2;b8Q!r+tJSSK3mYRZHc$O9=mdC5|xg4^M#1! zN{YR;g4ER+*NY{RSp1$bpoX+x(jo#!h~&(jR8?R2?G-uE;dUMtMVZPhW$CBtpx(kThJT&Lolsm+S^=j2D~VZy|B!E75VZM7sm@dqZeT9eJjaFAcqO zh)2GFZeaT!e#a_rft2Cf4N>#*g!@Jvaox1UQ}3OIAa3YSjQ1Y)o@}|Uif^o<-F=P~ z>vK$)y`l5$VU) zb@h(Ps@h!hK?-_FxK#IkY8De>7J>PSoZ67a*pqWt)e{z!tt~~kL?+Nlfz*h`9fe09 zD;pB=|5;Q3?#>$3kiMZqV?(0nnwJsd7hD_L5^_z?^T1;wz0M=olXpsFEbMgMwY`gB zy>$!;msa_SPRe0;9IBkcYxQj*lw!^43@VlSH|7}R$g^I1yheX+o&9f~T`i2(L$cvV z62;`cTVEWVI{a&1hR=R!>gy|5Thl2kD+37Q;o)HbT?x3c#P0hrHTCr5nN$0ZZm?sA zfJ9HgA&T?Q#^9bifSduSmiya%p*f`C+pC;g4_^tfR%})~(g_9Jttd^OFh7HBP7E77 zxjr3e0&<1z(m@;k{dl06 zDx!f%R>99}50$rhYXxOK+jF=gQ!S(7FZr16$MX}b+kCLhkW9Fz^SXd}0|AfYm+NwM z^f?WSaP6o+sA9~qXN>r;RmG*NA1_w)uawZpj|`r}AMY2pFg&-R%5KEG&`N4hod|+X z`2Fig-(T2Q+Ju)QnIcyosS%-_tmzAb@$0dxZ;L-kM<$4aVTn*GPL5}NxFp@Fsm)j` z3LPsUZ3jBs6%vZ^=@r*fRFP{2Dq0=rsf5rt-ANQPcFQwg6dq5OVvjfa%L>-cbbQh2 zR<*u%o1NrGg^BudLxx`!e0{kHLI+W!^^NJk{5D*~v}!Xd_8)l=ba{Hz+>Ke(Vo@`_ zK)RK$)nSl2MMB49KWdS?WJ8W7)|FOg-^PuM$1>vS^8ZDe2`IjAFn8onSd%S05eex? zYQ83k$wgcyMj;V!jf@XZ#VXH` z9>MyZ6NFJ813GSxgu#Jejh`Y!Yr?d6CHNx;S6}b@`!`<>kr=clZ4&+ImRNy3u;l{5FV&v-Q{YckCpUm7p)BWNP0o;#=kDBfBhJlT^|@O{@mdF zcw7@he0`+9;&t}A%_=n8E8Dd@b@fjow}1c=UAFH`zVps~2Kv;&Zj3Q~%ooIErtm4F zKzlmfF_|5N6_1ZDn2*DeW7}ryOM10m`eKc)MmC%RGidfg?`6VEl`pohtGSD{^OUz( zwVd`_ls;EhE(p8^(pxtI9=`*jY@iyc^|S&g$Hk$=K^`6%S?l!W+1%WWDvYYAu;L~T z>7yDL7yw`mW?&Knet!NDKtr2_g=NEQ#TY0y2q?>9KP<|B2DdoXx^4BmyS_7(k4(@D z_WRKmSzXN@P}xZ4$zx7&ihOyjnUDcw5kY&0Nf=gKf7aHmhJgx!R=yRf%?!{Jfi-=? zR+frSW*$V=A&dXbF}QG#u4KWURuPEmn4oiKqJ>ZaoYkY<5LZfbv8+R<@7>OD-V~3d z(`hT`2l>UM`oGV6+?&Y=O$Ph2gl;dkX4cxDf1e5ZTv7q{HRsFULJ*FRC(1c&4QrFX zfZ+d=@)ozPWAaG~wAtgIBd-wp+?CvdR=xw~`C)XG&HEEtzJG&o+-HS@hoFl#4 zuYwz+tbW#8%QH|EO)fNwK-$c2F~8LPn!mXmB^e8s%sV!Pm6YZ6v2)#02-h1H=TX?2 zB{R^FlPIVy2q&Q%o!gPsj!*U&q)|LS`0li|76LcB$U1>5`jaDjweAZ(;QUBojg`OS zuF|P>cEo*el9i>%z0#52ZAL&>W>J1(=4Xh27~GJS$hxp49#K%5d&tcXnn9kVEVVT% zwLGIoVM9I~ek7(qFN`MMm`ZObUyhMNeAy1TRW0LW5!7}RC3JP|tGa|KsAmx!HZ7+1XW}Acz5wi106m&@-U5{BE|^`)>e`(H{+mO9*~8W;fBVtWeYLg zWQ>)5{7FFu1g(om&cR@B4O1Iwy!)o>HUSL%#JZ!t;?vQFF)E=pYs2Y_8VNMxF7ISXG*-ii^K|li%GU3&R zuF6uu z8nN5h+7gqK54p57w6=~uJ&6Ep>+kC=JVHX_7ur4$OU$iA5i?+X2W*7^xgxZ-l^ozN zuTC0_M`Y&yO4j9-=r_>}){Ef+i-opX@DPIWu?%y~xpJs-lZnpGq`$0AT2Ga*lK&R` zBOQBwT+#o^YFCoJtjs`pbv8(4a3s18U&(OB*0L$Mw;%j0L&8>xlUVxMZcEbA8U{4O zxEE7_u}mkucfKVq>-!NMLrpO(w%a|B_<+xp4@m0+|(g=@TRECj>l873fhm3CeRV37|Fs!H+ z3X-RIdE!>ag+r@RYSeS=l-K~ISkM$p5YZK0j}T={&exrJFsIN_taarZ+zq9aMhLbu z);q7HrU+Z8V5)4#N;Z+!V^exQBl+s|p0gLX_Nti`rVkkr3$r^8<>ViYK1&{o9%&eU zPC}dHqEu>_Hqn-krt3ehGv5tYHEJtQ3v*Fe%{=nYxDQC3*6+@{l z<5i;t{sO#i8dBM{LQHu7wZ}P1amY>FG!VrTOEl7o9A;{>3-Wy-0lbbmCNW*?{xl={ zDW|Q+F>;G1WvhW!cq`KwSuMsAJPZfPtY~Bfu1l)UB?y|>1iG-a+`baS8k#D>iq?&R zuf90(r1PaiZqOGNc*5+pUQ;B8CFmhzm$eX-5&o~{_?6Bh0WhD5qPIkVdf8MqKY&-F z0Id2MKnMpw!s&nni@k#bFxR{)SN|CEc+r!emPWwdc_{^`%|7HB5+C%{Y>df^|91be z_16*x*m66n{G_x6eFnua7lgYf+{IzXJ?g6pm$Q4)MB}Krl7IOnn}G*`rm??ej^%un zg$PUaDl2P5oL^eb&4M>ZBEiirJi|!IxekGIo6WVb=_|5B-aGK%#vs+}yNjd2SyuNv z22$(yRv6`l;q4ZO=AL2>CW8hBjy zv~){-V8Yfg(fKB>{F!S%i(4fJM5=tex+n*ch5<{w!eO2KV#a7+veBo==fL+)^LvDo zPk7%mmxV7r7pb+4T~wPsilI9XsbOU;sU+SeQ3*Ix8;4C`D7w!8r!Xo5nR8V*N<4=? zRp)a#lU44J%TNYNJ>Cq~tvV-oho&gTYBwMvk{+4ix|l$oC_^8_7K1cK_xo`e$y(wS zfCPMbFvG=hdhsBQY)S@E6U@#$G419;eFWY`&{2>-a(9^e+F@ zE6kTmOknu}jJWE4e!_r@B&u*!85%S*P~88;*UExg(1QPDe>hG*-bA5nqfk;QKTi&$w{-^QAn;*rE1tw@UqA8=s+9BxR(Z9}6z5}LPPi6vEoHu#-_&62nzclHoJuiEf( z?{L~%6)8G=d_lY->_<653qC0y``{zHLBgX9!dUwoIl=BE%iMFu(te1Q%RN$2axCM4 zA(Qt$v2rYtG0XYKV@71Q#1aou;kV>ftW#Kg1PeL_(Tbx*`yYr%@3dOlP=p!meJ%Rj zm)*2XsJOoI#JsIfE>*&XzKtUh?6vBRm?-7qIV@|RkXWR&V55*uD9JF@?>}`eX&#SH ztK(W09-sUQ`%ou{#(Sh?>LN4wA&OGCzaJX;d-_I!e0@e5Gx)r!u%U!ZCmxy}-)aR1 zV#tikpo_9S0+9Qf?SQr=MtB?`mwrhbY{795PyUX?V^WR)*;nJUW03)7H{NoSO^|5ji+sb*T{ru)96Z~Y1oW2Nr9F#d+LdL)TRLlUeYS&uw-rpmD&Tema78H zk5MgM-<@;3zxe`_W6g3rg4pnM0G__JhZLsF|eM=mUP4odVl0GE#leTiuwoGzL5) zsdu^-yrlZ+s;nLsBJ%6gB!g`*YI9MKga&Fv_D;8jsz@Tz3-yDr6O&UMSZY$BQjLBN zduGQ5nxFA(Z&sJ%R~VBLY@@1Gc@|sZm~D_5=*EMa6kxT&(Hlk;5m|NN)RU>w4}8J( zT$Fpt_NUp7Njx;2NtxqmU$mQhFL5zFEUGAI1VOQa(;e4J8%*MncR%)${9$D4I4ff% zF(Qw!5b|u;xi1_2vzmq^G0^17FV0H`Nv!#D`YFZ6gAvrQRq+frz$^NScX9QuTVVQh z{GzdF+6o!z0N4VuKHp3&6>TD%$?D;#vP7)RL%boiCal7jZN|1Na-%h~A^!Iv((;dO zX3D&FB+Ml0|1i2^!;s`c2Gz(c)OPKmOR*YwSm-Dq!MtRK%!H6SqEOmUB)QmdTLaXS z)c~$9nXwC-sseyz(aazMRAd>*wn~43M~qo9Z}bU z#S+f52hEs2)%HeF{S&$1tsXbuHNwx171Z;vXFBRf(f^`59X@BbZ{#aNF|8A=c(}2O zjVFB6zw@_k{JZL70w#JuRU)u#u;Qjx#HrCmG0 zFe8b3O3E7xZ>ktGPap)#3Yoai%t+X&t|;q;g?@QO$FJt*5jD2Ns&gZXCIdH!i(EQF z;FgUpz!JmAl_h~#HX$9rtsrD?Vz0U_sN54s7^9L)gBCPS5xc10ljYevjVM$!FAB!#O!sL?x8-i+!#{P z=M#JjjrdGoN<_ynor0j#7+9;`Jg@OX-zpg+%Jju9C4q<&VO-|x=s;Py zD3RpYqYMLa=(ovSt*fm?q=xU}{?Jrb@d$Q0ZMY+!;W#dkiRSg1f&UXZ1#b60&8YJO zF&ZMUYz*k3G)Tjih3J8Rlnk9zFN{by4Lm`h5z%F;L|k3D1B-b+Ebe=Hq+Dt#)> zXM2ZYFr~Fw07PaudTO^XjnsyVY}lF;*U%8;>PD@oi=#YZpp_$V+7(t-Hkx6K2VQ)m zoe8Rud^=rBrs?7&^!-FTBbbg@^eddTY6guKd1?d_v+o`06uysbojhqaLa7`$cB4!tJ3-c+h>V^x8j3`V7 zd1|r6xWWg1lQayD_i9_Bs%XQC)v?IUnu3fYrGWW0EtEL{q%$VPsm-oambvW7p?v97$eZ027*Gb~e*dK(2bzhBBe&uFUQ*OWU@0_u z#ej)D8EsZmICX40w%?b9%F|Pe2o39H3Xd1IZ-hT4C8LI>x^VgmVMt_({J&087TUdHC9ANNdH4l$M0&GIL z;49Ocn=!^k<)GA*a9#4q#WJ*^;q3`G5i|9tR2&$W{>{~qsu zjtV@7S=0Z!@BjHCD-ZU6KmT{bJTvOW}L3K z+s+QKfr(a|M{M$`4IfA{?F{{uEm@_?J;v-JP%DgOWX>i@s|H9p`Ti6fN!q@5A*p)_rM zsr*SNRkij^|LyVl?rQn_m(H|t)T8^!mpj4pkCcZ+EnoV$eKuL>OOM6<60G`mtP%DW zbsGe{jGd012S&!-#u4Py$Qc$^rClo%gW<|9n`_KfZD}Y>wz!MR1i8VHd+nVvpTj?P zeBe?IT}hLE?-Ew`F7B6TQ>{SX9FKHZyK8YvG=H~4L|-t^B&!vZAVwPbC;O$h?duL# zL{DKx+&lw06l64dbx5lO*04@%9jO`ims#%PtSWI3l%+WXQhHJEwC6tV?B*&Ol56V! zwk9L@za4;O-1_?X&y(w2cly@D2#-;*?4@OsAh87C8~EXam(q6iHWWFEEks7lvsD+RuGI#lvDlF>O7A!r|{id zNBi83j0peR>vm4fXpll`LvnWEO`jLU&0PjY?rUCl;Hsh@IfEQsTf_M;Iug~NI$lNt z|1av^GAOQXYZu-~a1HK(Ai<@P#zPX^f(8h|-JNbUcyM=1La;z^4Frb<8VP~mE=};p z9lo`X>~nVB`{Vw&x4ybnO;r~)m(8AQ%rTxZ#`Emx^}@WlkYB8C=P&OhC$c19EJDy= zG>e8~e% z)igU=X`Joir~r4@ch1Ud9`acC6>ajSc1=u9KpI>AHrt+~$@t>8U7$sNMutAn?d`t3 zv!$c|TTQzpKx_8Ne%9Pfdo$U1KgbQKfq~l6Wd54wRg#D%Y*y9NFewjcT9CCW__jN( z0XtS2ii5IWcj>KX(B&7dN3O7QC$j7EePg7~+Ab+V=qYTtsxlm2Vc0PRs2C1j3Ld-T zS}5z_!IjG^f*^t&(NJ3($L=@`es+G4uD2A65~fxeSUvJU-^K zRBj4a<#aIZ!hZXBo^LhBx(mS*U=pS^qzAMv>->`W*X$a+I4G3>jTgE>Dbb+n^jLs{<%ajj5|3im&x)$6p- za__^`Eo?c}K8Yl zxmdlw*4cKxkzf2_8}xc(-u%8J@ZKunerMKyz5ST#E?-?*#zsd(s1tSRwy)cvmbyecL-}Oqgw~^j2fkyi1wV%xn6QTcBrQXI;N9FwNbGH-QySamV zYIkq{IDeX@ND+~d+x$D)aqll1agL6L3pg>kdu%ku%8uqKhwjCh9GF-TJPF-Y?XM~x zHRJ^;uqy7FXvNl8;O%G)Pe`>`p2jW`Am~iUS8|HPEjTTo`A?{hVQm;~NQwrmL zk?2xj)UP6XYBZ|8(NhqgL_ArXS*p8*GKTU_L+uCgo|_BkRk3o<4SD!fQHIe+YLzO> zIRb-ja8u7RN>0brX7q`aGArw6(Zo)FR#*3{}t|(`i8uXDe@2 zg24U=twH|Tz!dj!=uP4yC&uTU*x%bsX=rvF$-^;!DF=-n5#i}sq-!><$a)>s8o);SpqGjhpu@sdJo-Du?r7+(l`MoQtyl=R_yF5LdZTf~U@ZT14d} zk{tT}D$zwHKhxkgt+i=lX(_mR6SrEcVAog>Hb>qg8D0zWWblfv*Y2zeWAdk$t}Z{= z$;U-M?h6)Q!(XhrN~q1n`7yaO? z?pe%kcqsdqN2AC1=Vs1nQ5XA7 zxpJkvVnT-Z8)?$-Ij2V%?#7m?ugvoiYmF6M_PUgQSJ5|zOMz+DmHNt^*HuDo5 zoNk?XJ^C$qFQC`B|*V_Pt&aqT(9w9dC8TXdY*%1(kjX7|9Wre+5_jNFQA$ zBih~pJWVH6G6i;{aknSUYB@j-)^jMOb~-htNoz8&G*a<%FQXUl@n%Hn#Fx#hQ#$lr zj@oW(u!v|UL^-@OktJ)=!A!>;pWMpN9mXO~I|K{L?Z{9>^1u!}GT-HuZIITUL`UHi0 zmPhG>H#F$)&!FVwp?Qc38IHrRnWttee1-e<4%;PcAjx6eALGTM z(re#qJt^Mcn?5N9FL?*!-fsSCcEVqN>9w}oW`*n8uIlCY)>>ViFLPe04&yJk#?*2e zu=nNfJw6h4Pe)rGX@<)@(p7#hAd_h2G!ytZKJYXrwt&brRY)7eZ$a6>A;U1klzL_So4YeHe*KyPeg7wM@)}6$^LG z-J0U{c|IqNrYfcvvxDnfnYx)tqe&_c*Lr&y$~)Kal0T;$s*EOlNy*5iWD@FT*)WV# z;uy9VVBD^5qR#Mo`!$Wg%*d#R;@p%Hqq9%gr-cGM5tve&2j21a(4wdw;2lov_pv>Q zCBgRL2U-jF>dv3JExZ0G#Oa&Bnz_{A`HUFX#(@@5yeH@D;x2cNTs@v$b}T5kLxmwW z6~6MTba(J^JItBsCwe&L<>tb4f*z)MV~dgy_hZuc^z>C0drYyQ6PeUO!98d`7&GZWKSSFu%bL9Q}-Nl<+WE13!M=E1tw`Ws}LX9{^qP zDK!Mpb)}-SEa@pTGmv6B)o3;1(XYK$E%)L`!Fk}8$#m&QHSW^CNjvm5)|Qg8A=WX| zkViDNBL=9wcojX03)!)8w6XB>iK)J!#xuM*)U^wiX%&Wa%Sh;E`r(xPwd7>-{3HEE*W5)1o0FxpGdDufXb7wJF5EwK|5}*MO7lfL-Q({Qy@+tW^~)eXvCfoH zSnL>OBGD+WF}(fuJV1Ct&Ss`Hv8Mn=sh}dz>=BOdeT%UZaqk?LpYiGPf{7d7nUxxe z{A%&G(-M!z9NCH*h6A}}w6otqBK_-Q>t4%n!PoQ6k(!s958xz^$SWW3u0ZQO1sk(_ z@T1GZHn4xx_&hFSYDML(ALFC%E?hq(#zN;ufemZ*v6{?ev}U~KTA2;_Snb0U&{i4O zC;SfMeGcjzO@|;}o;A_QEcoZc4b;kx2UDGOA=BZVgy^l)713H>kfS^6J<;p)%VqxaMvTs4#TF;ZvC`Yuc_TA(lW8GZ5)IpG7eLIeM#_@0(wK zkfXj{(euK7mT=P}-W4&Pp8}8qBfRltXn-F*jna+Y(Q$!cM#kghfh1mX_S7;`xK8;) z>7=zLJrrDc_c;~Bfn?mHgtQYaBm6%A22#uN9|4hF!%|PtJeuDQ^3ha7MK4^SIBT;% zD5R$)AaYVnI&oTevTpKlw;F9o(j~z@;4ZLufiU$Y_hMByxAeXJ{I^L&#A*I23zNOq z(IldMJzM>3+}|#*DAxId)OMJJG!Yi;my_Nwpv}W_c8-2&s^g88U9(hpzs{aU@bj=J z-m%x+KX1tgL3FL8DCXt-Xq_LjcC=EBSN=35Ew8?Z+vtyn_-r?g1uW(9JwnF@`lZhJy-R+1(!Vi)VmQdB+8w zHs=FF!@SKT?zHMwQ$G}iV>diLl^nOcnjNB!)g*_+C|gtC)*Bo0!%3YNGzPy$D4vl0 zIAIX&TLw5Y^fIc3$t;$SCx{R6`1!MoTX>=@kNkO>Pr#d_cpA0e5;5TK4azmSQBd;M zG%IvV#mMifI^%ltUd-z4z2sSc>XJ+ zU@;Oab!X0oPMVDP{RXik+;Nqdp@ad10voI7;Y5Lio5@?v-W3?}{`R8`V-p9n&|Z}^ z2cIB!SpNEoGrdWX;fCGHS2CO#(o*dxZ4OzERHjs=L?ughQOhFvaLOF^TB}W-h6-n^ zt*$=`K+2GBv4Sr+W8z1EaFZol-^9ZqrCD&|%QmyFDj%3%hAEdNTT#H{$QGd5ZHd`@ z!{er@g{Q*OZ&AxNoU_)Yk_K_d#3wg-tHqPRo8}4$iqBF25qk%RdCWmnZVz_|F)qlJ_`fAp}Rwl;5Qng6P9Q^B%k-AzC3VKQEh zgnVgH?0y!XXYR0GDHSV>O<~jR6GbsqZq9p3yN|v0$ThxJDJJe%%1FlWO$s2r&7~T{ zm)XDk^)0OLjM4&8q-b4hkyZqju1DvoG)c7RW1%i9kfQF|6zbo!8X*1L+QDbYafupc z76$+6CHUMeYhZv~pqX~7S5c0&xb%gA&C!1_##Uhm0Mye>Fr78L4q957%2hwy7`?@o zr+;TcEnMi36*DvBeB0@aMBr=i@ZWkI%nr>P2jUz!GFY2uY(giIclStTLx7Ot*!TWU zgBq2dyvA0zeS>6vdc4CynzzjVd~QbQh`vZXFy51nKS1x3DGkFN^7^8n{gx)+d|l)n z^fQ;0iqrLXw&{n}L13D5#}5YwQr({CH`w^(kSPaq_;kKW>!p>h&;_Ib*YmW-XA8Z+ zI!-{GJXDhaMpEqUF5X~yw1$)u+uBofv+{WRytV+6^H6q5d1sb;;x+lJXeGwp%K}VF z{!2#FNyG8ZoA0D;lcy&W7}ZRY9xBrC%OJ)qK(BB!OUU9XLe$L$BVWI89rDX?O4L z{oYPqzr_^a6mF5n5D_sO?&OmN$`SW^xAmT3>Wk(VWUEQc^wpg@Mr|~(a=IWb9*IdB^0nD} z0}Ef&>G2~kJxrDWX<7gUm=hOGDnUVP43CMI2KHlBAt=QO>+53yLk;*2`{hAyX{8@f zuCC+7a7LY z3E?(v$80owl&X@%t@8(oI1wv-^m0931;&O)@QgOW)P^MlYwEI}mmF)dClbS-QhT5< ziOX{E1x+?qCQ!@3qM3H{ks<6>sPQ*ylj1nIeR9PaYe8x@w=XPVQ6}4nM13n`XIUI5^rt) z=*6pzW^Ww>9NYiwtDN07Ha0HA=qrQ3(uOxzz;XYX7IABip!azv>3jAFt7}It=%15c zdymLPzoif5gtH3>L6T~ZWf}mU?{!dV5u2#nx?tVj#ua}3f6gtf^{2fBJ^_=N^j(KO ze&*I-M1vUIzIuwD^&vl`B+G?D*xW|{x9=^I3G z$60uZv(twP)^{*L*Uv090}qOdHV0D6S`C2Ji)bL&{usc8_PJ1EU`s=869nd|q^fL> z%UdKDFz;63v2ytfkNMY^#SRDE6vZZc+(zTZ=)63hEIT|a*7Oov+b%^lkRT+jnBkrP zO-s};K+yP8boPF0uDy>qUsSY36li4~;EIyhLxhkS*OK?){~xqCPpVLYUt7 zS3usc?2kj7hqarz(xcpxWO-Kz3Q@O!G_#)_G!E-ojjbVbi+4t!FGrEzKu{ha8DeB& zLiu)Q6*b)SEg(o=u><5k(kU@8MnQOJhUw9UQ@tD4cxc^kPrBIKkX!sXxW_5bJX1;_ zuP6JDvB1$1uwKvR9zK)iojI+46|yaM6RyREU})KX9y_pcmj1`RuJbT@jrA7vXa z-xXshZXS{1Kx?+3BY7DgZ~OcmVlu(isyC~!K>F*6HA8^nK)Ze4{{~UHSIh^^3mxlm zJh**w-geI8)Z!_9?e!(B=Lu+l>#QvvdCkz0o%_0CC9ZL4AojJl%e7;+eio&PT{RwL z26EhT`7?e&ZL1ZY+kQgNRNHNsq4sclzAN|O^3i$gHP4@IAfDL;NRX7!IR zdMK=%9V8`|Q?b`u*w&)%+vsf9BemM7#ydiO$VG3hY#I|c(>M^Dy36*7sM}b5>m`Ej zMJqY4<@fQ0V0}ZA^^+$A(w}^SLN?P&9GzGj!JQVf%I70ITs#6>U%iA<4;ifxDrarS z{=1Q*N7Ga7L-#k=l`fDTM4xwSa;zgi9$v7FbY`=b-v+~7Vno|1<(d`e0{$KZpD*~d z&dW=SN~&+hEtO(=(dumJR&5f(20bY=UR6Gq?9X(NbZ$HLnng3u(XZOFD|w7S33QND z)d*2Irv6AT-F8LkSjMDshQA8t`O5? z$$WfiupX!|c?x-WUx0ltP=9`g<4QQrO^?|nXXG*-_#m@b9^xKl+ZMH`s5?_yI|w}$(nBGyvAt@D zjFVSr^Pqv(6#6?~pLc0BTrA|+FQ})hI$>+cP^0n}rP>%OTr?%fhj}(wbL^{P`f-vA zqp$b%RnNDH+=CqJXp|*<5VVPt_%Ch(wuKF0d-hF$2G_0|w(&&BUa%Io2Kz>HNfw;X zXIoHYu{yQSg`u$TOQk7|>vU;^E6R>X9X?(2aZ%ymYs>gb2cHk^w z3JA&nZQFqq{i%cQQ~NjXN=cShjsx%U<^m7Xo9*2}%bF@`?yK4i8#>;^$4J`r0j>C> zDPzjzK0^P5dQZovNfev{5~752Q$HF?!%1MtV8gz5Rn6jR9IiQlaB|@g$_T=HY^FN- zsTM`=mXun&u4^{k=Tt6r|3kvdt4ti}p+GpjQ4hC(uq(h~$a+>03#x057z#V6s>%qb zgn!mi8X%^>TeXImJsKs%Z6E;Y&ci1Wdlxjc;d#*T`B}L$$N3i#Gd=TR-Jdkc&`m9Y zs*Rc%F3t}C?&QSlc_p znz?vJ`B6O%T2m#tS)a$#Sum^m^?27S@xtupElBX$U?6ANtgidXJbR5! zGQ^j+#ruTs`b65*kAHYwFVDVmE__LE_4_mt?%@MJn#Srf&DD!eMNLa|)3D|WAhBH@ z+Sey1bvdA}$wf}A8)HgYZrn}}bN$#QGP`Py)ul)RZwSXTIz ziLG6zMdybR=ZE%l9eb+ZGQ7e<2-zU|$`p_bhWkju`PzI!|4*@Z<7g_@s#z_QV=WCK zHyt4o7ZVR^c$RfDccjDI7v^j|Fq;oNlA<}rVNkbaF{!G}h4~$0_~oU*0*&_=RmS=b z+mSYuY6FT94x4WG;#nsd zQjI5>ZlIpE`T-6!MHEj8V3 zV90p6C(=D7c-W(S`(r^fiZe8*d+CPB_-ay7fF*sidA$r8^OA1=d$M0~VA5x_CYGWr zLXZqvE}dhY(3F-c9x}EvAhy7qDke+!YY4CjZdnTtpEp||I40^|DC9`?bS7)1pu~&F zx!Y%Cx?59b=bB$QFyK0I14VXqf}sPk3~QXmm#h>B_orboD}_ANW{8tyT1tZ+eV0%^g{UI*XKf$bR}5&Q4w=Ir>>@w~*QTKvPl(jPQc9%Xi92j+SM?s%+D zsPVZ8m>AlBR_3&J0Pet_!I$rPqWpBN^}p+!{&l|cKeEY|(~fciw;AOKXDC~#c~Uha z-G2V=IS^Iw2y^z7rm?TX|8^h1J$3B>dbYp+^Z#&R)BmEh=Kp%l72q0&!c&x3+?r|s zd**|YS1ZSt06kzi>Ay#Cq~dg%5dFB{DkY;3Mb*&w~eEmAwl+qA8ZjDb$O3m7u?UhogwwbGIB7oBY4Mu>l zBFmq7slR<*{pjjK1)AHgu6*#imz97kW)uOccFRiG=%_$b?Vr>N1_mFCio!WNf1k`I zO~a|emrnpLZ+<*96yR>Zd;$7=-3zD7{}?Cv#7f?^YWz`2xLKiM^|#79WNo>@m(}A- zq5we6g%4=xO*N}U6>i^obrEL&n(tz4FpODw2FGNTP(CB@%K75B5+86Ch)(NG9lDog z4I|tbyUjQv){e%_DMMHB(Ld`aZE=xBAVE|a()LtNoukY)sC$)lOFf`+PdUIuoQN9_sDS60w zIEDjTcP1PC`igZpyBNmZH)*Qh#N>9t! zU`5k%=0&0`6>WNY zLSDA#*)JYe9%kpSw%uVY-gkPB)j?v4<5kQ>oYuqV_B+{c*zb`O62HYJtt+BGIpXY1 z^jX7$kjcOV(dVaAlvRPf;;adB8HeH6HFx3Jwn!f+OZcAkPlW$Gm8yH9?PbZ%Zeb%2 zEcu+l0@Y?y9%L9XgA2GVoJ$rT@NUOF{f-t{yABOH-XGo9Vluq+A`Aak8R(nue>If3 zC@A|Q#%e+iv`rTJhY*qM+s+S^4y)5@%>yk^#l;qVFC!7 zcHwt9A%(?FfcU(!+S;gNq;S!GgHTpL++*%Ju)GBs=KUhKfm6fA_o(@}E7GH@0`KGt z05%$)8thcvYNnPL(u3Zf{5&~zt(hMUz0rPV|B%vl zO)oIYd-^%8EsfdFT?emNr!p$(Uy~#ceeIu3`R{mn?N`=IDIP z8-ELWye14EFqct%AVtv?nAdCh{i1K}W>|_a-Oj8o;(U!6R_g0ZqQ1A^$G4DT?8|#z znc1CFYYo;|>K)C*AWvAQrQk_6UIs5C187}nrEn@Yx+3Sm!uKPct`4*j_Z=O3_=!{k zR8;gVc?zz1A-Sk%XtXpm5&5-rw2Qsf)y+>*YXEWc2?*~T!~JVD(oxd^q2o+l&EBhP zyUhAcs(_=efUe*6P`@QcUY`AzrqG%2_%Q%%F>T#-q0z==Clz>4sdUOchi`>NAPawASlp&pkTBxy4?@IKRVOp91{j^jmVw`DU$t z!ml#1${IeY4M- zWPC2GfAC6p{ZZ~Q8E|?Ia7cB2S_a4oe=nAol8A

>aOowLsOqg7Zc|yVL&RPbknc z>W}1bB%P$nC#D}ocRTI_Y`i!y?|KO!6?VH(h`|85R^r zTVFQ^EMNr7XC10mSRR(6&L`>yXhbR0#me|H+E0Edc+ai0azg#gd~YH?c3e6jD%tT^ zzwORycbQS{P3Dk6-MhMeeS%zSW_QtLcB{Qyntq9yh{~!g%R+MCtx<2Z#xY*3LMf|x z`|Wgr*&nLC@s&zyPVpZ7@0+OKWRKX{@u>PYnRA60_lN$)k*p3QwthP-cUDMdspx!c zKKtR^`8Ehg+5j1`{SsXI=0x?130q3S*VBCzlnxMS-ah; zYE?IPgh_KNOwBww7*G@72Qrnm?un{R$4_`2csFT8hD`>7Ir5fnF)GQds`s-_RpR2u zecfmxB&ST-G~Yh=GehM}Tt$TeM-o3ytzo2MqXsF?=4^tQ*GN>r?6jCkhuCI^%njxw z`R~lwlE69EbBbmpB+QNxV`ts17tb&n%U}0>eQiZT_shDjEMmwb+jyOCA|QCGk<@i3 z7Pr|dO4i|1?5q5Rp&6HyriGET&3wlC0Q{+ncCxBS54ue@^){I`tO?dt6%V>qW>C{% zk!(|rYuvXh%1YXOvmvVWNb7OvuSt7-@yB5+lz2G{BU-%QhD}ia=4um3*@sg%9uF1; z7C<`?(%bgnE>3)T@Tjbo4S@)EMzAHdAvPMm^U;r@;w{>SOA{W08VVQqjiyR^CtGe} zdiv(08*Aus(M$4}v7bLxnGq&F(M^XVn9Z!m+;=cSuLKoQtgZ;4C*%MDCZGVvb+?ED ze~tj@Z*Me(yC~p$0xg?BE&1PEn^!q&;6D}o{I5Ku$`>V<*5cc4wX!g@S|3xH`!PkL zWjOIl#fO<9LF7W`{?X3TeslrwYC9-KeK2b#-ugC`dQoRdy|| zDEj-wymTkEbt`fQAFO=4y;+`-R58)g6HU){YEG7rX29ia$w?q|ycxt>QrOCyiFItmIXnfzZ zzdfE(ZFCn3ik#P{gU&1O)o~^=KpFt zA7uJIx=Ot>{i=vtxDq4(u7|{sLtTy@8ju~dkHUKK#~mK4Wic_$p5>#QgfYMJDlGBUy= z$L1jQjtv|sD^k9OIJMW)6px;9N7Q=EFmd37#CQmT`0`fQk}})hR{HG;vwj`@u}Fgm zL3fAidBeCvujXuN;#^8u-`xLUY%HF7_C|hd6K%+#n?b^lLVt{e)}G*4Cd%3G-BPn9 zxj)<4xM(Xe&mK zoM`JFm6776 zj+g70Kmbf^F<2+@j=;s_+RBU0c7uWnQmdrrw_uPq%+=#Kjn_zvs2B16E|xMneD0T6 zljCv3a6=vTnqgyL!IUGt-0U_f6I}x1aLmigDInD!e}UPRI72W(+{ySkv?05CKbbbB z7)668Hd4f#qjAxlMh7aR(rxt~D$Og=k~&fi-lSaQvU?R6nQCcyk%O(a2aTLs`fScj z&|7ccfi3tD=gIc(WQX1jlSZQT#_I6l(YRp|J85XC_ zRyb!jx;ShH`1IOFX;d{Npq4|Uqn(aOdOQy6G!g{@4R0hEn9%%M5)TLe;P$(0dC`fT z0*f(>wi;J#lHmx*aqJ$gqiT)CTBZBWM>B~gMfgae-3Th`0^4-8x%9{wE#sM{O(5hmiokJ4s#=!jJ#6|0fbXSLomS$(R8*5 z9b>rrzcR0Me%pn6DwbxJUs;7m7V;&<3ZwNBV=_e7u~wjyjw`<0=5m^DBYa)FUp@6 zo#JO3Ma!^Ahq%(AC*5G8)I68T(GGRtJ>dQFuG!scb$fAyME%=H34W7W0@uYGJL}Ox zqzy^9`M9qa?`UgNSSLDIM|mdU=TO{~5yD13xWNS_)thYf3O?VOK1|*;DAAAt@d4b zf1{y!{9R*P?*PJxHR-!cOa@;>&3X6zE(erid z!T2$W!}Wf0>li;uM$XnDQrvs=z1jAhAWHR$uvZ&WaBd{d>?)2}{r-*aa8_Sj1?kp4 zI98H7M@kpg|3S;cQ=4Qb+U#%U23Wb57OtX{CK)DwIK|5to6{7F<+D=lbxgUuYoxgz zC~b>t5veeBE22>_VRN8eRE~MU6d8hov5Z{&frbu}IH42oE*XZFqQuolGom%-l5n(Sug<}YBZeCb z8~>&=7bNPk?6?@+Jqjiu*c2rI`+?)umZx(zkpwU`R03&h${x(oqj~j!j{Pfiz$+JS z-ncY0F){lTK7Bl7-dfe3M|rUNNyB}w5`}<(LUs?pEwa-T)r#^H&v^QpIy$Sh4408k z1O&^=%YVe7#dXWZh2$zNGh;&b6Q#|qw^YcvGhs0UsSY9R44ZOMY(;nn>8Wrl!qQDrd)TQI}45iL2#kyT#Tz$*S9g*|n4-Avb;1q3&q+dI|BV6W}!TON!IOI|CeaGXa zYsQwZT&(INOuyv+zF)4m53CAcp94MP=Q2g>Umo!;7h0UT?c<~*4zhn>h7UjR^2m66 zeg1udrfu@ z6SnMxrBzy(A~6Gl>3x|x^5#s82|ME48Z>~k%9cSw`RrJ$!t4R zQB>%ls8TNV;{6_WRI+GxkQ}8dMuvuDG_CpDJ>G8}d0&lGjHSk{*T;RXm~;KVdDPXd zTIjsgl8m<9D1^YWp&OU!qKo`Wf*W1gZw7~Ebk@b_TCc98Fa|#PsNsV+iZwIHH@|a^ zBDbpvSo^|l?Y175neiOke||EcGB!`wo4vc))0T|qrXet|46hm(nl%_yj>cm3h1_Nq z&rFdsQqg@}Q4tpfk-rqTXC7~!`;`0IE9(pq9EA$rI%N>p@oGcDj=0_F$E0{!<7 z5~|(?B&#Io*@&=Qnkk`BC_#Fv%-KX(xuMZ>jUt9dc#mS?ySOtKyV303&kW1SHd~eV z>^tA&p}W<^r3^Ae6{s**hzQKJsl%0aaTxe>JVp$28MS7>86QR_C>8ei_y64imAV8v z-V+r4D!baj(>>^lzd@Zqyc=5Eq?ton{fmBQ5Fm9ADAgPB82v1y(g&WWigJzM|08+#_hWy!f@$Is5yIF8JfUSDMY0}s zuQ6>Pn?Y$RuEMmB$Jc<%6W;-J;p{4CAOAIbQfz=T>0&Xo8{U|Cl|b=wFd=7py=4=w`NN^ef+%GpBqvYRSh2 zkB12o@hCFel6bWIV=b+?_x*3q-Tw>2Le-hFD+)%D#s~T%M`WBAi;JdGQUsCHeCEUz zQIQ|$&)UvN=`=#i9+Mp)eB0}DHsI;U^#^T3dKMWW*G8LrXV?K_Pv+XsQTP17YFr}_ ziyZV<2I9N{ftV|LIo18(e}%0)xqq#Vdg=Hd9(PV`>q46EalY9+TubSvvw%5p_O-+5F zT$7IxlP?I8LEIye@m{V+3TP{U5WE_G^{w+AP50D2l|9@XukI{a*b--LA@(GWwIr)C z$Im{6qss^zz9oP_H4cG-(W;A%&E1k6n(5ApR*tLbdkn4*AT!REa4*LpBPhzlt(0^ zX$<1vM4pB!a>BcDrz2erb_l-aI)F zAXmi5_?Lv-;Z~t>+l9-AybQ`N=Tnmj5+1>7&KP8=<4+W#$9-C{#WS7{OgwDm`KKic zi2SeLv52i>-S($M%c<+#QhFEHw`=bwF@#l)Ax&~_OSM<;d-}jOnbYFz8Feu=h5|ej z{^}k=Re5tLEnq0#H?v}>^o?DNW)_N64@^!M`PO4%%x#P#=5RMffem9D`*C18e1qcE ze#lhp6JAvu5bInKSG868%7%V`sgWx;t0LDSU{}=u)#-@h%Uat{TZv2G_@v{G`guVp zwTwttYNDt)UFUEma?knvWh(^D8`N2G>zt4w5RIt3RX;OIe=fJ^(027*?%^0p-2M#; zcXgOVl|XLi6@k*=5fU{q2qbZSF8#i@HX-&wFF4m4(gF=8Yf(d<7h^V?cH;QKmJlZA z=t0|3Ij<{vJv}h98>aYwjcn%e`bYK`1onM+$ots3rUZFyLrJwhiz!^yXqm#xdv@66 zO```j#*WWXUE@t-g{;1=&k%Bvkd}<=+YC)ew^C;f%E&5MU#T z4x!@Ms4zAjp8RsnZrqeN)EJoGfPzfpSwE-OlbeGF4jj*AJsEIZ?(bx(@IMm!xF(E` z|HZ-*Z_I;3>9ViV#W*pVuCi6>pgJAew!)0U)|-7x5RGtpRN%+bDCL0(#d?+C`@<~Y zz(+(pKJ;DQ9gnlc{SriU^xLx6-3{-nYJ0d0Yu`sa56Ql0^q zu4m`Wr6GVu)xXizanFGMY*QoenCsa2=D9Gs$l(`Q<9CT1)w=qyIx5^tHw^++)#17a zD_43kg5Hcf2e;hR`Ae!ZNjrMux0KFbkDf^X7@&HE**RzZ-PHwj%;D5&2`$m{Boz6= zy}Z=GP~2qpfUqSQ-Jq@em>WAvTyC0nWVLUwS;fGGlSa7WU)Xl&xz)eX+b%YBv-0e{ z(c1@VKMz2S4WZ$;@>ng;J`v`9SaFbiyzq_r*g6@Z{A0`|ZGYb=3dcIxoEp6qMF6*D zM$1^;zdU*}NbA$6-tbgfS~3lm2DmU-P<-g6phDWxdd~M0n#&)7`7&z*-|F!3G*a9s zsp1k4=!&>ahw8boKFqRAf#P>BODK=O6rAW9!$rt>3h(qV8}Q|f=&9%xlnE2|PnXX( zUDd*;LN;pp^K{|<#`;}JRF@Y5IJTkzDdqb1m$$Ox(9=KA-5s)gW^J;nC@F(ikG9HN zcI47@EmsCO%zQLLze+F|g>shfhw;|(-m2?Mi9~-ehn2)rtsfs8`9YeWRPIXAXNZQ4 zXA|&k1W`+v^S5T&>b+(AlHhrM&sr~8JgvPnJP2j)A}LT|cK9J6c`+WX*MNY}8;L~^ zl@ldDcf2{Sc+`G6*WR1)|bW zEkeC-W3MB3t8-PNTY;aPuJe`NFa`bBMEk)vkYhG2((R5xwUvL?X|3G;g_|P>)(8*+ z1vheD{+ro;$Le>X3t`5Vy(dcah77BR?bqUGo>tW!P;O;E@e}-ww=3Rqje3|zJhxVq8DN!MPf9)D8F z-MOPyP+pSZ&>nq~WDwOG5*Y4M@O^u!pn+2Fjbx&8p>wuAKhsqMpmEf^4QRnZ0A0-njs+BIp8?^ zX7ucVLq@8ij474uR;Knu+Op7XidxnSFCXe;3NEXkbjp;Yv_%E&@mhfI-vwQ}Er#sR zEv+O5iVa8grdn7Qj@jmaf=k98aZ%i{<7Pf#L$KiP?(P!Y-8b&s-JS05bf0to+#hee z_r|zC));%QRclpEtywi^%{lJ}kNLodF{g&&cn?%!2^(SK>}-M0-!WnOGKznZ#As}H zib2y}6L(?B4dw5VptikEnsKqHbV1>95W|U5*p%k~>m~CKILuwsi49GdP1OW2e_Q2e z;{(IPPGF1u&s1jm-wPb&ukr@+)+_2P^36zHyx zVdSbFNbPL=MH#fHq1BH6%#!Y;@c1P|HX^d^Nt?J($ z|6$bz($e5YIUdh5#|@I#+5WZf`&6Kzg9@b6av^fFPnZAdM2dg^i_+a_iA3Vw;M9QP z-97gmef%v3QC?+fa>9uNb&F<@9K0r?b1f_G9;7Lz0tMp=DbI}Jcn2(8NEYkp8*$}u zxcMz-=W7;1+5fCu&ZQ@(rn-n;K}6u%kd5HNAk0Z6f5liem|&_0LZq~$4Thl80S8Cx zOY_)F{D$S3uzM0=S#jk-(QN@wJGc78xU!Da7nsXE_B!!{`DeZ%y1HK9J@5#(5iXq$ zR@=iQeVc-!Lr33bKE3nWar2G(F<|*}0~6zv@ABsFrINtArluzMhY^OU6KoEEEP;?JfrC2!aFxVd&`&mc9)ipGFfC&p@MkW4;T>3v+xgZ7t>}>n|5e?2} zX;%i!CbFif6wzA_V?8}krupxr%k{DxDZ}~dnMg;UcPq~rn1HmiYkO2##STTr*2q&&t4!J3a5^A~()sBe; zik6_3Zg@;4-&s$!QTlFsU_&H1fn@zz>yz-rx&$2Iz0=)K`q}d7_xi*?Kk8_vXpbIPz_` z-1#-UG>`|~(-RK*tbsc6K|w*wKHd1?Pj&*s3h7*((FD8$BO@jX@<^*6I4gR0;T@$Y zjC!^>u#xkZV#aMRgpa|mni}RRH7wg9j#Wxi*h(#|UmJi7xpUC~Xc~9Rf0@qU^*=3X zR~Wyd*g!5%u;*l3^GF|^u#cqgXdWOBzn6+U}I14F~kabU*x>g1Dd&Fp1F+GZx|Z7xnjf?5SabP#g5TtZz@*>WHqrw=rT)tEv~6XRa=ayM zN4B7WYGAPLq{Y<;FKqt=I=!N^D9W+FFnLQBU}9#<#9Tmg^;9SG4BvS8K%gugN|%7r(pnZxu0E_$2FOaEF}VR- zCPYDEAGfhVesDmML-u*Z0J`jk;$dj${Khq2_uH^Lmx7i8ZU8@I!$Y5e!n*^qfJklu z$7((?=&l}lNcEvUIFq;ymx{`B&y{%)9FGD)HfyVYuks}m(r@u+K!_Ob0mtmMh|5xq zUmBjD`S+THIBHhBL|{aL&hUTmyR_84QhB4M$r_Pc^3)`@WnANP^Y(D}r@kF|EWZSW zunZu+V^(H{=1Jw6ofwTxBIKcX+?V&mV_FubNnVvPEKJKt%QD*=^s!l$^h!y(0SxWV zvG7+!iRSne39zSg&NXN4W#Z?zwS~oUx$}g;CGkmp_QT6zhnq*7J2RLr z=EaFzbmp>@YqX|jtwOqS)U@n>L{6Q()^^b%f|!8>%tTfS?A{?(NhBjQ?`#6K6_aj+&{@(1%cEs zjW-;k1H$@G`gT};Yp;fMItFxKSAp8ucuQ~!&X-uKvjJz;Rz4v&CrBi+7P1uGfu}>< z!^5Nfh)5@CBT+U&!LOYB$-v99>=qleUS%^hN53V!Tu_H$6Xt4f>Z5N!-UgGmt@(W0 zwabHqEd@lT0U_(|u0byzjtx=oZb0t7)!}z1FKQ?2X)Z6mhC8NC_9LA#Hyie3)VISkjl`-3{paP;?5Z^xx(r1o64LOF( z4g>X4#fv^h{rD95Ny|SeAcLH%->_zfO?OMy&soN)3ItQFS#gRmtJI>g6vn=z-EvOaN~-OvJLnnTdydMsWEu=J~3#JtFQ$}YBI~wC4>Ds z^L3EHewIy4NCF-@gjJ5LR5UXErZ`u19L%Y5-JvlkoKAydqhxbVUk%lv30cKydObi? zIm{8x?L@=UMq4Bw6xZF~Zf^%#SRUoJGX*lLh*9y&Xh|>xlty9e3Ho%rp#$e*)xu2* z_)`eFOicke4Xp4$B{i`DYZT;|iMdbX*d&hT30w0Vl~RJGF5ZE2^L?h0ug}Yj9?)@@vw?rc&W5B*NLn<^flhg)z z$Y~}fxz*kdrn^RqS2)7wVxq~E&E%Px3xqCy%KPNAr;qV8dll}6?8>r|s;%sQ#{%p* zuGx>7{)&0c+>3Vk%b4{FZajw(xqAWziugzcf$qGp!(hpz86nz<>ic^x-Z3#r^%yq< z_`FRDs{fwSl8w#v*@iXiTYm&+LNs~|x;**0sac%eyD%8uMd;OJtmvB6c_J-M+f=X|H8aUhX5kFQx1FEIVUxp^yhr7z03qSK&* zC8FC(trV7CN~9=Gp-O88t?2b=H?DOviCLj5Z`BL+WvV&@m)E%E1DY^LA zB&bUaMt6@I+@*odZTFBxzr3Qyb=X3@1@9?ZY1_v>vU=#=W{2EwkcsMeZn=z-2?vsjuFT5fnY>?e}n1=QaM|)bJ3T*z%zG9;BomT@|!3aw<590 zIlQ7SRisjiA(>KDT~b~}x{5-y>8;XW=+L1|6@S4%5Kbzt6`UWFt~9%JAc_bq359YG zc}nBG?9&T7f^-nL)cF@o#^)4geJKdpPHYBOP1#mRlxB_5@u>wmDj!f_np!G zg%+zL?GKmMmbO3%7^bA7oM)i#riS?kC%dMEVzuL))e~I|55ewZpL?obt5uP*YBwlu zCmSkufu5^uK}gAT=2z?SPg@!9r(+3n%*L$Ro@y_Dt|t%7uzb1aNeyv3FZFH666N>k zqTT~(`?h?g?)bFEz;>C&6!8hUdirU`3q*mUctu4Khf1AXl}W$syTRn@=&#$v374 zh<#KxXJ@swLMXjX#E&TgGS^3oRSJpoj1Ejyf%GCj!Y!){SZZs7lA7Sh1f;c6&B0^)$hyn0 zt9$rss)tvF!Hni?Vyf#XseFfiwk1CaFnfu4t^k|^(uu*61fm!@ip%*g25U`y_6L0a zmY-h-1c$fF(FngGpRz29r%pY}{}xW#_pu968Q03s*h?eN2&lyU-J?^B0}-6iO3r*R zRaNEFn{X4e(#`6$R3QHeck~s;&#czqAsxY$f+i}tEvyUYWB16?;$+8-9rcRxO9edv zZg5RRA5+Q#|4}*~#@&N2&Dev9#%mVcYCOKwvlC}vm|(!E7!d6;5YEo`!5R`+$ql-+CZIE@xQ+$%9tGZkEVzl#c% z2r!+!zA^!`+en0eIeDn*>qxFmvejOXvT?G1iiPSzM53n`TOfpAK(0GoQdD;nvsaPc zdAgJW1aAcJfu2yIrwa~7f<|nA<``Phc6LZ3Ut?;bLJ0}!MC0Hcm{S!6G0EE+v(iUJ zsr-q5kx)g69+OayG_GjB6u0oLl$cx%gP&(?-V3B#5-~PAnUB`@o2q-&2jTY^!HU(7 zlSC};ojbXHW0GzCF-rPomlxL~$RaX1{Cu8!x9*<$MAx+eS64!1uz(k~n3YS|OCNjm zVJx~ITQkFmy)7R(=Vk3_hOx_W_(!H039{REFdn1bSaACzKLqa|_=y0%Nw(ecD{<=g z4n;8J@ulvp5077}?7hWQtwBkK8MwKg$RfzYzigV?hxO*?H>qwt_SzJ{r7bhud63mm z!Oyo|tF}2fGM5+irw$`=wY`TpIKR|s^CEMS?Z#iKj6qjUMoS(}@Vvcjwf9N0Zg=3P z<#+#f>&ARt#xGN<+7xV}7@b?KZnV z1u+oij8g8?%DhB(S5!%PNP!-|yfsVzFmLA8I;~5IPIGvXO47=>fN0~7u&7}gu9=3| z^0kIUGtAcSP>T*z zH7Aq?x7=O{8ZD52`Q!y}g*YCs+|a?$;AvDhrXpamlm0mt0asxsC#R8e%duVU23K)b zi1IL5{Dm7R7zy+0`z{X^C?Su)07d>W(BcQ3Dl2G+EhC=#mGRbb0?`(2Q~t}{5sJgL ze%cI=zK`1Fn`DbAJpWRJy{Rc)b12gZC({=kTN@B*OH?(Hx8Si{u?AZNB5d4DKIr0g}M=IzOh3PR-8` zn1rQ3os2K{pDZauMbgHPM}0!p-?{TEkcJY{F>-cJsJ!Enbj^puQ8#1D5^brk@Jmb+ z93C3#eVU$r#D=#5etD7j(UybHJr(oEr1KXAMM7ibd&om&-vx&tw8%FKm1(S>3`HWw zkoYv(z{EUXQO@{mXqB}BxJ}=}KbJe+65KjgFiQN_>#!s}{S+eU8GZZqYeTP3#ijpz z6bLQ`{ICBgMk1mA>>NDz1^z$Z;J1*&k_g}!@1%%uOL+Yl0z|s&X5*6yR4H-)3C!O2 zzlPD%Z?8QL>-%(t>i-5bNYl@x7R~+Z`DbeuOboKn*E@?%YzbNBLb0!ZKru%0m#zwp zQ6z;;QOywXI>enq-TUE8YNF->53-gGf%pT{Tp|A%fq*h?+*8(Y0jYq?ftZjkO=lzd zn>kKZmUfD$yX$uoDbas?w}+ebF9Lx;h0?)v<>f-5ijvx<;dL^9Qmsc4{WsJ9hlX$@ zf4%S*SyNE{f7t*v{`(If+*0K!FO_>g^z{94vcDnxlRD}Aq!gWTcs?;}0*t+d)1nFP zjM7$VsfFdEC`tYMr#yWQKQb@cc<&V5T1Q}Hr+LgVl?SFgaw7}{r0AWn>RA?-#LaI} zh^Sx-t{6{+e=yxwLRQfZ;()8fV zisS%DuL#&)6}PyEXh=O?>AhrGt}#}{wVPbL)rDtojP&LHbZvPMLSW#n-OEZE_jS+h z{b^G2osh8qfwlc+jGdO4^Mm+HR(nfEP7tYmkyU1|OZ_t{UVgI2_Up6T@}&&K|L7V8 zOID?R%vki9&S+Iuh$S(fjYlRd)T5Y#l`J;fjW(;PvXc47;(bYK#`?Y=slC*MMtuSh zNqT4p|6e>YMvWT3-a+PeuAeCj`W%Ivxt;@0d&vDdl-9WWY zhBr`fX6bII`ug`C`Ew=%S?<6PLicYO1kaI)9?2I63X96@beiqqA;^5-lI`)9X1Wcb zT?g@qb84&%GQTRlE^BpKEt%OrizUDgW-RxT?p{ucdKh^Q{_~Q z&bHh@-B~AQKUF$M*wq0?*GDOFh_E@Km8Z{qcims+SDB@_b;na_?RT4^E~O}qJb7;& zS<>aFHy-apLiB_4dv!c_*h};$q+k&V)_O(X3wE8};5#Mz0vYx774-}KEfeANJRT^Q zZwFkV{}R>TVh*AWPl1Xkd&aG22tDAM>&8C&D1SMkDQ8_oN*6UX`$F^lnYB>8ecZRr z`aO}>r_nCiwvJJ}wE629QG71l?=Gjf3{T6p3H@!|-AiAo#V>$Hpa#mzNqbSCd>r9F7>ApuVZOz2vl;0abh`CVO!$wfk9ctP^a&{q?GRp@H zV%ZtOo$xj8SUzmwCY9-q@7Xtnzu5DOp-8AGW{Dr%_cz+B!+3j3;{1zwddf+RSb9vW zT6K@toI?4|OyMJnaV;&u*ov9DACko%pfk4x#^f6^;Hiz|xsBg8o@FP4Q-D@JG65TX z`=&l3cmU2&0Mz511TmnF97ZM;_yc^z`;~G#BS1H6O_qQ}m|Hk^-Q}BwcX%n)6=5q& z3d)u4NG^>zEzWmr)mWg^l+H2ryMq-Nu8K3Jg-KF!1tm^<2HL}dSd&md@2dCmnS%x$?%~heF$mN2Zu%C(44Gvme%hzfAW^_GFc53V){efenUO{yV4BwF~T(`xN!2 z;KDSJwhF|-L9r$)f9J*?&;IZS>V8Ok!PQ=W1obM?I5-g)`xN#o0SA|FEKDaDM;@mO@>94+a)UYVmoJ>m=BK2%Zjlw9 zvvdzJHX9QL2w2Qz1484dH<2?FEMMMXF{H{aU2%_OElJ&5u)DNmr{irq`*9*-VIzqA zpptLfgME~Eut=>-AHBq6a_UprWwaxo%^=mB6*E@tCqe_dp(pPZ=P*9gBEGsb-(?v?WM4LelnX9UAZy~!8tw3|MESokC=EH%P7>N;Oimq z=zj5RRcd%&S^;@`;Qj68VIduD*H>a7F*z|J*;`_8EYZ#S<9bL7Yi zS4+&Yf151ZVDZ0XYz|_ikLFt77mB@&`%#t`l;0%N<>-elsh7uSMe1@F@-({g#XZdg zFU`Io$^%kxLvZZ{KS%{>78oirPAu9QfM^miqScTe7;M zVQ|fN4+1%J40VnHybne9gLysqrvd2jgPdaKcINAio6QknHols}4yRf*DbjY#B)5jq z;E1?fDUZ;egPh9_+itCjC`>|(Gn3a2g<0*WX@BwKK%&TDKBK{G6;8SrfcE$L`R_SZ zt}1j8rOV)#Po|I8`|g%xPc>an?ooN=Q21{AmF7fmAiGY&_KG7V*W$`>lWNw6wSI=A z0Bh8xRuNZ?_j~UEe!kY{H?#sy1nt*(YBw49rhWdibi-GmyBF^(Of%sWv(=*Edr}vC zoHCQyKK4jLuefJQPpZq8s&0Z~-i+PFkEQs&$9f8h*Vz3WFK+oKnq+mbw?q^*XTHu|Z0b(Lu$G zCZD}~0K{vxQG2``fWbu0*i@E`8y$98wNev1=M?+ahD4NqO|J2DMrhj)j|P2j8D5w9 zR<(t&lej#NvA6Qe+afKY(ifSBntKBH+{Ojt;OxzwOBMom_#O%Gk=^(grJ17-+&3Se zbYHZ}RvH2kb^|d&cp4wi&WyDm-xSE-!+FOC~ChVMn#$|G)<`?qa_6Cbk-pEoaon0<&YwtGu4d}AA{ zpUv2Ar(BEzyENebjVMwJJG*GqtV~2FH9B*dviN5hUYvJU`I-IdFL%R;f_MCzG{^L(;`){@lRn!Q)bB?Nd?Bum*JHVXN z^+B?RqrR%U+z0P`?M?a&{hs3gg5WMtJXBG;?TkwB{ze~!{`W_I+mr7^{`DW0ndEOn z{jdAz|F^-P6#IV#58GQ9wGNZ)>eV(@(EsAcO-|9Ceo|u1g}&W1yNxEK>OlreQo#80 zu>gk{_pT_MKh(jH40tn*_-liXS;;ptK^_V2-DIh6WhLYUm^dT5ch}@UQAJS>p{FCWq=NN!Gj4xX&Po9enC$D!rlhH&@{rrNIjZg3b?JR>i^{*-2>84g%4f}Pz zgGj)=`}eQU1y}E5Kt4t03>$N6jZs(euZgc08ZiHmDOi$KH0D8n6WZkEEAcW7&X^*o z_HW+-ot}0tA|3k#Zk2mtneS&B62^UAaIL<$g4WStaByj33$uEQT#g#XmAMsQij#P&=Q1}N28?En0f{>PPb3*n{i6E>reY?%M z8JGKCpFnVZL7zV+dmX&j;4O|e$W1FaeT87eC-C#YyI)Kk=6=2qPE{|>E@WBXUhVg| z>xEF$^Z77bpdQN$Vu=yBz=ly!Wx8$IA&-OXk*96++(A5~O}!r?qHK6>3)$yl?ns;Z zaMW;Zy$5*Mhr>0a%dX`SuDdiP^tqBocH0XLz)(&u**c5>*8)NZ7mbQECw#_f<_XdC z=sv5kxqEl&YsA~yi<_kD%&%S+x^R&}bUE}X`tTiI!+p`tapSXZg$oy$_h^C57a|Pp z4&2v?a2L;#_xIg`#_E#q?cooTK(pn(eshS3_)ERd@A(G9HkkjZeKvJ4Sq2~U8sdGQ z+r0zqpY&g&%=J&mcG(xUmhVen)9jxPvOt`I%Q8v&KeniJ??dm08BG_KfpXTCxiqRK zw>fKDYtOyDCPz|!axRdp<~4CL6z-y8woZ7{+EyOQjt}=XRx}m9zw8u}7Vo;Uo~V;Y z%LBuC%tp1ldev=BN}L40EMC$zUsyk#>9+^nhcURGSQePwt9rd$MAhU^*Nml)&IIE= z@E}*VTYrsx**acrO6$zVH;3=mGGo;5H$D3GjjJ|G7pA#T-<4hee#FX(R(V9*jjY{w ziJfgb=6thGHyzRZvRCi{k|9&M|JfqArWH?!g)hh+7{_LZ6ZRpy|15u^N4aVv>Lr`zpz!jmvueL1}hNjf= z*UoFUf`6K-p-x&oloY$7M{AwW?>PXy?~3oHh-~uNmPH?9Me$F#hWz*+xfa#8yQ!{D zcg9`;ydf7tyKj6iuzUyNKF}_BzS(pH28=(GxjokeFz+%u&FUP+9B;o6Jtr6kVXR!s z-WfT40Me|o%zJzlT@g(ua;^lYPJQsG|DLduXI|%Uo5PEPe7H}`4j3L<$hAhUaG%CS z>#x7OieQ{hi{YMQm9Twg*Kz2SYkOW6s7m`arS@6We!k;4^;>7`*SX$Ws0`*(+fqb= zmaJ8--EJvUTn)gaqlBy7hVRNPMu8Lac#Cu7)YK-vbL~DgZ_g2udS}VBCt^#m`ht+C>7koz@RK8;=?t(Zs1SMJD`#N z-Asl)jA`8k5WCT7*teNdk07nC7Au{AeR{GjWKRi2!g#jar%cPRTg<6DXj<&sH;&Lk zLA)(bVu<39-=IRiE6;3fY#M4zb=kxf?V-=yUX|j8Ace|2vRXj6-kEh~NWM!CXcXem zmsOWc8crI|$+tw#D@%5-%t^vgr`z-omPy@;=UZhJ8wn(aE&N$VLbW{*g-LZ8%*6l+ zLBedj!(!?~!l2BM4A=DyYHKK=FNH}fS0g%X*z~_?3T8MZRpo}=+n4m=Jyo>xRHf){ z-pri^EW&JKNwG!^VrD1S*TXs08B?^DvWZT&*JoeTBZtiTVw2hH_I`QU{)XGM`s!L< zS!xF5fjyj|T^!ov?}z}Me4WpuBUGDI-+tQVH(*UY9bR{3TW~Al(n@m}MDn?HF6yg1gv6I1GG%-?PS!03& zjP`Fvpx|L(n^$0^5u%}(sp-c-CSp0L9#djXQE!wBJIx#W6^XMVy!#PbeX|Ez$pbvX zk0n3NPk)%!U?77gI{6^wTM-j*=iNaP5;tzs4em%zX+?0*9 zdZ9Q?EU)0=YSk-DYq0vj?TY7}_p$hwJ;RkO?xll9U1Y7)@{!QsNSATSd6tGp-#c-R zyR;?CkBssBnRiIX`+3%;#RvzL8AAL4`D{It*SD{2_4DSjU8cJyLst70OIuAW`txJt+M>=_J8i;1)eC)D+RiMn-)Z z%dH{V2xS`=jCzW%^=F$4(bGM0pq-1)3IvMVp?Ii$c1k6 z{mV)WfG^~q^Gf=(|L*Q?ioSAFQBtw}V{(ZN{QbnHUER~@i4EQ$S)jW-*h@qmZM+Iz z3z`417P4Ka3PQr;V8Q|Cjm}dFTU^xohdgsVMny-*jhb27*qBs_$@x#l3KKq@zV<5@ zhQ2RNFegry{5QFp#pHsy^1lr$4g8b)QAD1C5&pHuNzxPe@Aa;T9PIyX@CVNRKY<6Z zA1u^uV8^is5cv<@GH*AkltO`$%Lmh(OOzs>1X1)Kta3LBK8YLpBXkf042C)Q2UYk# z-rZdO@Ha?r*435A5CKC`3`9 zOjvMIQ&WEy6&dM>i@yW6RZ&$%g@RC3Rb}Jkj2|@l-A`7^f5eSQ977yRSO_~fGU7)l zS2UdeS7O*dkq+*aD_WW&UhH#E0}~?n2e{Y1AVVn@AbG4569b|s_^bgqB|ZRrnXpJ> z2mkrSP(``FeV?H*n^y0AJRaMEJ@erGo~r^6X)_@K97?4eUCOsuSo&d+1_8pVR~ z>gnl8l_(P#9X&NWE3U6k1i)uyVfiCu|M~ej5|5BDzqmM}vs3st^1!Hb?Oo%`ZTiid#IM@KLJMnwa;5ORYcwpoa!~QHO$t^973J-_( zC;r~llnQQBR8#~89<0E?jSrl0|D*^g3@$Y-%{UMxtL?XkyalZL{@4dHU#v1kz~itH z`V)RHQt%jcP@p0swo(AXC`=5|#(nREI~us6aj>8uw*JRMf8up{Ft7};Z~v5l9?CzQ zqN}2n8o8gSKGFPhPul1Le8^zxfb9>I8yg!x6?HNFK?hB+VYo>E-ekhT#Rcb_{D+y? z)1RZm5I?;0v#{`wGYsGi531m_O=w|I@gKo`jD4cR1n*0!sHw#%6u|kv`J3&)k0|u< z`?npr!D_s=wl-Z}1l$24n76@v#*;zu*Fb;E>zzB8bHHr272Zo~9uHQ;T%wISe}MlV z?h=6of#ChW0>Ct{mj0VFem?~jhHa%(%m4f4|8SL?*MemV%n?}$|JL)HC3V)n7+k^8 zH-9|*Z{)%0HPqAEEjC7q-v@N8S@odLkKNIkq%~n4RK1KXu z=FHe6<97K{P}8Ih zBcr`<>@)x!kMrgAxDzfV<5qY6@$A)m?Xt~W-;B4K5qC6_jVy|87+^$+tSR-~v{@*- z#*p^*ZJqo!L@&cC&NKG+gOzg{kn>!6EEVMQ%yj;way{-8TS9Hn9 zNYSI2qy)RPantkLz;T9kSHg`M5Lr(^csZ0pYtPh9F_&aM@Kieca>%QB>}Cl1W*t`Y zhv7@|b|ae%&hG9nD)vS9wir|yCTElI4QIp^Cyn-VsD%k^O3IKh=myrqcbMh68%#XI zc+r%fm#IT^e0OnY_dgS+pYEwPBGn~f0-6<=7kgVrZDVyC2Wq2O+LV?zbDSft<`MgY zd70RHtxfH9HbJ&Ag}~`6N)OeYpaKuiYDB7pP0T#lKqa}cu_-+EPvDIBenQ|p=`WNZ zZ=&(bJ0+^ThG|6b&2{8zS?i|_#wY!!F^^usyItRrU%qL*V^7=ycL=Y%iu$jrFKceT z?1gX2i}dDAjm^K-_3r_!V7$&_UQ?b~1$(>;JVw0=23K!5Ivvro%88rATXJB2SwY&u zKYrQ9#Y89LUp7LUW@|HXjj*bf8Qwss!ecL*Whq^OEb@L9i>Da3JS(l%{<#Siz*W{? zTuqHRw=fcjgAcm*g72<@4Ps+!Q=*PT4NyDz*4@ehDN4avV}*y0amdyJ$x*~mt@ah~ z_!y!$NmifAZ!L6j26=a_v|h!K!_uTJuOeUWW#xMvV@wA~YhBZ@ z&-Pt5I`Ji|waSb-G*ml}7=(VeJb$NJuc@_`!`az7ep`{|a#+_X&6S+mxGUX1#;(ecYBzcjobx}Q!KWG%YQ-G`S5 z4Os{bPp8{QCCB+?ZEwAw7AIoPFZEgFJxUyy)Y2IFo;qTZP3T? zbvUquZZaT<;b5nke8r%XqXk{WVSAFF(4h2=W_Py{6cP?FA)u?uPKX0iPxNu6Q9GHRiivgs4t?HCbU*E_{? zg>;3=ZgW%5!BtWVnuP6b!p7e3QkmeIH!0pX)I`26ib^~dDOQIKrcbSfcY!IfpnT>x z3ck5KkIe9^3dh4r?M0bs7?v&Vry-J+$9-N8T%LQ{K!_A6E-Q< zoTyCdxu`e+{7gC2o{v*vknrP$u`7$&GtTFeq4}N%vXmfroVp7Re#ww9Cs{)$;Ex4i zU;DuBT6!C@@B=lfIole8^wm{`mLY-1d)!RS-0J$6GONFr2Le3Z(7+XXno`WD+7EMS#*sI@mWhtX#{^T(d^eDMTd8lg@n8^W1cns7p6yg1O9BuRR7`X~Rv~Z_&?lUkkKpGJ-swju z5)6GepD4<~3R8nml$Vz434I{{&hJgE1{#@#NKVhqDTEK7pEh5v;TMCzeLS3x2Z;-d zgOe3Y_%;j`me-S6@So=w*DL1wEKJh(THi?=u{#(pEDua2dCi=k*NBQHfX9 zBw37pxHFsf;s`iK`i&9S-_39^oDUX&OvzPICftV1ZIjc74a<0>BY2RJGneJ6YhIGMFU}GPCG_v@6 zHfp1T25UTr5F9`3Y@x48m62!n-)V_E&eMFqu6fxfy&xmim4n*WyyG6*pZ`>5!w(CR z_tiU^t50+BLJ>jLzGL8`wvc>G3&X20Yu#&nK0*3-3q*aNQ0Z7h2@ugIU1*lD+|SCN z(6Cym39-Yf1#b;~?imGoR?AOB(+-fmPs&DcNj+1I&e zD)X;*w_g@N)$gGC+z>z@s}pYh@;Rxe$~OZ_J$OB|KfR67dlv9{mTXTftLSVUE@)}( z={>S;Q1efDwZl=iEVe1M=E>KQXZy=n?*d$L_hq{EFO=qd9bhclJ1lfuHV?=gyslu314BQ zUu7hF_TrpBL?TDJS<~FtC0b4!Rlyl8qOOpG)+7S@AV(B?hrdj}bx3WHBs0|TmyC_g zS$Du~&e^z=sU;C@W|zZZ!Y@uLjmT7^=!-gvho-QoYaNw$ zq=Y_TX6vAA_XL_JSD{3rB5+~MaZs}|J;yRe$gYK|wSR9nJ|v0hM#pG}Y7& z1EZ1A_maBiSkfkQoNsVV<}ep*am7S2h0!m9TwLposm6P2-p)k4tGhs2by&l2b#6tE_L6G5s0Q##Z0R(RgjGqr=}5*#~FEktY&XD+m15Zim)yxA`DYH2K}I zneDP!r`a>AHCUPk^}8BPbz@@Qa}y`ial*V7uwWe~RLm$4%@7x8IyPP^OeHKxSQ-Fx*6={`!sTYQNz=fXxRMwfQ+01dN)L&(Ps)r+WJrbnr*Oui+IyMvb2 z3{zu9WRP1eVdf2V>@t~|n%C1MFGuv32>oWLOf@se`QbLI-*OST?aZ(quf{g!VizB) z>pc zqwE7&B0M@L={83-9lgyYGDRq!txO)$4FbDDRwnTel)uzqi|cPK*c$>as&n>W`SzsC}MmTmDSl`y_Vd%abBLi2r!<-D|S zJ7~TuipM+7@W2v0pM$&Dd*QvA;Js@+dmuEM!qoGKy?ws)?JO8)ygY%l6=-T3#A`dl zx_>E8?I~fnd|7)Eb>%o@8kED!=gCA$uBWe+1}pqHSMf`J{SXVhi4N3-%sB z#E(?_4n7JriRUlm)7m(k6~%({LkPsWTY(_!G$y&qKb3N|O&w1^2h zC2)8-Fg?6x5y;wlWV_VX;Y-P}LAvaNi9wyhV3Y(JqUlQ0$fl{J>+v{8J^4hZCK(k; z?k!SqvR@+5nGr~sof`k7^}oPAjw-|>0P)>%#1tM$pt=n%*8<1XaD?Q z{e{iKg7OOAd6TDk;)4-^FMed{>Bj{bnnsr1S*PeROtYqM1Zp#A({;5ASZBo%Cq}4@ z`w2HK7IRld&S%)O);plgqhnC4uz-TE>h&E zWV+a?il&#iin>c1-=Vsf*U|7i`QOX;ewK-CuY=E&1z2b1jUpM5nRBtu229SAR_o_) z4Wu>L(?w=4&0w~@_3H(7nf|6{nV2DSzg1md$FVl>p#D?a@M3SlKypNni(bZiKMSn>v#dPIl7Y-e z?D1_cv+?C|DYsA&v&>7Z+re51>GNKIZgRI$eCD8Le(Qu1^> zn@7j9xEyLHH&*HA%$|pev!BxRXX_SjgfvTOK{MtX25oAPoTqbe{`~a*HiKK(jZX+* z4!lgBTPTW|Uehq=K4Jn~uss(tt$)*O8QGk|_ipW{wk@qPce|XcA4i*GTi&D?A3Je$ zpWlRDv##{HXNl+~W-U-;nfcJoF!KgRn`rYSW&}k=pm$c5ua#}wN)eyGc|?J|4{z<| zjkN9tuLpzA4PCLdb*_@M-bin#+{HwIu?D=%r6On{Ll+r*aF!>|jCK0^?Eu{tKhkd2 zvZx}#hD$*V3JY7f>>jXu)`E=$dpmO-L*4xt9nV6>?`*4yW1}q|A15^D>#||2!7XaO z0{%x!Dy><3tW2(9y;2E9gS4PhiU7(n2b%`;QOk;Iuf!9bca-Y^XG+5i7c-|co#5jF zK%6eHWDVCA{-Q~U#9ZH+d9T^j7DRYJF^iK2fK4Sd+su~4uEAiNyX(;k8yi%XL#CK; zpgyGsm;!)0_HkWf0&1Rd3kS?bslzW9ALoYU(_O=&2lZvR zmOI^kUz9p1-`RP^VEg)Pb4_61kdwxQ6$hDPcY$$EgEj${B0nWe>#5hiJ>+IeRE{WX zjxRYoFKxq{mSjqAT7~S*u!gXuV>TZ};M+OH zul(Gc;j5z0%2oO*GL6yT6|KO9+Q;O~mgl#b>7SJTy&v(rCA^4ty3fz|$GCWx z%`==>4Q05_6l7vk%6uu|*-rMbxxAH0brv&LD%Kw92I*onQuB&vbFNg!$u-Te_0EDK%p~x8p0PW0el|K{UKln-DkiZ>(4;B>KZfz4UkFGy#IsbaaUu7! z?dBtcLVPWU%@ZuXZ$A0-eI z&uy@4q0niC2o=9{$lE3!X>t0SI(4e|8RY4VpRd`1Jz*-HdyxnvPxs^{@$1Snd#AUq zkOL<^WaTjX?4tJhUC3mr=V4@9q`S`u{V(d?Dz1)ZX&+63I|O%kf(LhkyGw9)cMTrg z-Ccvb`@%I?a9uboAh?{Bckg$9@BYp?7w6)CaW2-6nPFzSx~IFk>v^iHAOcH-NwP@r zV)Cuw>Ej03Z$ZAfOeuEI+^_AN+ie_^ULihR^5x0^K)%|Yk~9RAw55k_rgXWKO_zLn zbiQIIPqYJ>7!85QJh(|}5mU8Siq$4&U2K{)u$SlCMt^A3;^V<|eEPM|6W@*g#RZ~R z=mcjU$(|s@Du$kShNp8l*(>vA53);_g+0f3j$c4lbEz%gI4|+?N?<=5QN2l=#y2YF z+mxIcwnTeBpQB=dw$zpKwyGehE-=)D;`@neT_(IK9Y!8IQgNV5iBVNmmD5p%C=V{> zug`rDqmU_m{Gtf1%p6(6@D^Jgjd-8yc@cE9m|N|=lDdsiW==X9@m;0&p_857qY#%K z&5i8UiLxY=Gk$3Qz~6s4ae@o|O9ppMJz(KCt8MaLWek#$WrW7g=j#c5zfiFuAPxHf z1;K@x)hU-RnkJMBn+R)B;rq!`($7o6rs{1;yHqXfgLdqri>o(oUzRttfbI21a>L(m z2!OI=p_Z*Kl!z(KsjWK;NAh37W2T(+`9y^1ub`j;U9_>@%PH9`&@8OBKBzIObF>zV zb*vXCPi->?|Al1!!7e|ug7r#Z4>sQR&GvE1!{5FD82Z=Mv>Oi!HdYS|OF1v7J(m~$ zA@_q6Z{|$;-vTVZut2PRgpGp@1JNUQPe-y}TR$%lKQsDYPmo1xW|$0`+xHA={|HUT z+vfG#qdxS#QQ>>M&jr;9dJ1|OrTE`o3y!~Wi~#qv%BQYkdxp10QT)>umflDb389Y? z-{Od0mPiF2%BQbDf{#eAUkA)Dqc=yVd!BMx(_x)1uF8SUnj0g^)eDwQHa8%*{eQ5(jj`5+`BVH(NgPiTK4sjKKb;5RW zwhCuGNP_z_|Ceta3Y^}Okm}{^$h9c+kj8~B&<|NeJpI>ubiAipn4OZ;*%=aDt<%Gg zSRvBUUaNh8!gUuoUJlef4w%&@P!mZPL{M;>t;ZnAB3fU~RXs*ts~qjnCwRrRy%mgl ze4EofnkB2{7?>Qtrp#$crIq101QWw zcg!U=^=BG5UW1MyDya02GM12#3UxiF6QxE)$-i>7nTB$)yDrDNU0I`@Vc{E;iAKT)St+$&7ERR8+i*>yZ7*=XK3}l$IU@=1BdU2#vOIVz_}DclIsS6UfnbU54cB6x*d}uqe%-zCklBH0rXGOG;djWDDyFp+$s9QeKFP zv6|f0PuTt_2xOTPMVF1eV_|j95HgFkaf>{`my2>p=&qxs3WM+g@#)lr>}I;$-pmm; zdrWP8Bg2u&|9c+y!TpOokiXKcejcj1+?YOoUY_pEJUV|TjevTFw=<#P%MYFnjlFxm zuU1~JGZa^#5W;V|bMq~~Giw_vSs9zRc`%@T7SEQ=|03ZIudBj0-O%=!|2FtUJbtY! zc*~ZBdqQP`^^o~u%J?n^CjW+g^M0M$1K83XIroZ3>^D7qP5gSA*m(^o{YDD5eEh!? z`?iksJb~^m17ZaR75ttoWldS#4y?H!dwK6;`bpERiMLC^u3tb=`v<&@m*)59T0cb9 zQ?Zt9U7e19z6^fAixAE6yn*MuCkj@!jBzW%CA8mdtkm&+#9eGgcX0_c)4WmOlhm+| ziIll!Ktg7oB@_a|-{3%GOK~hUYkeBDIHWD<7cK*Vzj?0))#%jwP##3POC6Xc2>z|Q zY3$J%mOv`Eu3+HZJNRchglWB^kKFwevvyu#REI}|L(TXm8ek_HeJVDdL5H*s`K6$n z5=3<;+=yZ-uWw?!t{@8s|iUM^3oE^ z8XW`EZJu@An!@QT7BX6KDjpny?7`-*HhP{JzkloU@U|a?J85|SpbA-MI5VJz7)5=r z>nOS;l(9HQ1mmss*zt*df2M*bj=42oTW9O42@h=&rc`VQ6pjynNjo7p*(8oVKy|N& z-+>(bezWjbJ?WVB*6uIH90<)BCxVRT0hyJv56&8?7Oq5wk3SK&zlon2B+@Ubd(sxu zx~!L~IbQk-z3-O!VQCE6uSu%c4;U!&zJy(e7?Ayiy#E5lsH0Clzaq?+C9=ynCo3|` z7;MMv^wkG;r=n`fd(9Qu~|l29gk2R)L3ngHY~UKknhNTnO#4)Q|g7~#dSVxQ;@5a&Ncwsnt7(x zfhV}qcr<#UhViF|RO`5Vk!baZiV+T-vD*UZQ1>~JyIvNdF&AHs2($CdEf71_jqG>n zoKLKao<0^0(1s4HAVv`suUFJ{xP%WaAccB`Ds@Y&y4D7~BZ^T0xOw8@wPOt`3ecsS z6*2Y6bw{3gShNSk{Gkf}Ot9m5q~HVhgt~)>9uAFxD=Vw}&(yCOZx>$Q z=25bT5f3dgMC)o5sx_$_+M?0ZO)P3-JtXV=ANYt@HA@S8t}0_e^MY4Q57We6DG@&m zA?0crc4krWI3d3Wi@L=2g!|vdo&!OG&-^)hzkpZz&#Zzs=lTaKkzx0tA;mDvfh=Sx z!~8#1N@gvL#XI1R;HNEBHhvZFIjQ)X$0+#isOU_qTW}Lq*UY#e=`Jb3E5ffM&U121 zNlKglepGWpO4TU&j#2{G0mYH(1lhq02Zf|Z!^-!i(RF0;mJVi!b-$k|GBsy!H#QqM zQa;bUU{#-hXEV4jJHa3Dx>rlG_>0yW+r;t} zxqjd+go>J*V}lekqv`4Q;3>z7Kddh0?ilBKg=QNiGKHNjmNjA2dEkhuw4UMdhKXr6 zk5|sIgL9YF^xH;@mpsHH`FOo`*tXKV1DQ{*EMt^8Du z(d`~X1GRZN|HHdP%b=YNB1(RF4aK?^Zh&9im$oGHr$-)F)7hm>g*evCYp{-DEe3~O ziss9(M3d37@Jn2ik5O(L+&*`8qwl2B9vrO193eiSNxeEXYZ+P6#b7${q_ncl{z4ID zO0(t`GSL6tb#<2t3O-i7IP?c%`yA~tFTKqLkVZp#k2SLDd@0=Zcl?ZJIJ7mkzyNh_ zr%3f;|8E^1IngB-7rEW@c}dq>)->dL=c5nwD+EJMA=2f&=Lf8jCzQ$q5{HeldhsRb zNBAU*0tSZ8VLs&|<*m~@57SrS(>hx-u!*O#a}$iU&YF2Tj0svg<~4H^IML2Qc4=P{ z`J-a*lvi~OK56WxaCnbfp8{o)BElG$&gbA=;soYoB;2XbU->GZJjeDwKD?M7+#L_{ zwtttn-=T%-mZ(%OS>fAzREY&0Im;yB+wv^|IKk42&byprcCQwxP9#nPM}a7#XZCD8 zx>Oi4s1dC{k8LLuqgsoDtW{`j<7?X13~qj-yN{W;Esms)Vcgy=>rHNH877q!CdCka zXJz%xLhczEyMj+{24ou`A zn*oP)D0Nzqwp`P4}hx3*z{GPXl&L?}ALj|fM|Mev_`z!YX7hKu{frqE8C!rjX zS%ZVjH`>j|8C0{p;Xzp#|KGri?aeg7;pxt^6K*e1&sC(LZ$M7IU%RLw^sw?E9p#4? z`+a;Ozd)4d6>sh*!4LP{vFQ8*J&&)PXeNn3Tr_#V^%l^9!VCSzwM6d0+k@c7=mc92 z;Nm4Lx0jW{2Ait;Hun;#l&7s!EaB$vw}b0%)@fBTobE8Js6@oa<(SBY&%Kfw0Solg z4ycWn*;I6Vy`XLO_(!^F1Pa_-0@J5*%0&*ZaJd__pPrt5>|J&5+7jf=$zsZdgH|UF zQB@?yCu(XOUJWw8nH!%NhL%ztvdfOCe;C)=|FwWM0QMy^ea|yzESrSSpzD_sh1*-l zZ}hS&8*?`UpG?E}31nl3F6wOA@%|AIn~3ZbI|Hkoo%V|dvO^LEV|uoVeRSq?1)Y2< z+8{Ik9E9)bryma6KB3<}Og6%1yeB9XOGE`}CN(XXGEQ>K%0;ud+GJHOVRLD$_5RfP zRXuvShqxY&8qH9n#r$(r$UZo^n47KxYfM8^Sge=1cGZ>HLyeB7ckd7jTb*F3$3Y=VAGTeh|`m^deSQqAaY1A2y_t7^as4_r~l}f_xk)KK#2$ec~ z?m|LMbxp`rW}%0N9O?De3}8_Gv^%6e^RQgWEOf3JE*MN^AM!g732gKE7T6|>uA z`<49WOxOJ3(bJ6qLYEU(i~)nm3~~aS?W`PZ7#Q{^m(e zYrb@81~yL6rVMpH+XN@V2yCC8{l|b1cHmI&fyIM2MatYiX||vRWWwV{>q+M-HrCx5LPu^6UUPR&CZ~5&xvNFW28uD8Qmw#{@{f=w)Z2_C%`L!gv`^mR*}E<94XiH?cu z=^kuz?$B&kXh7#E*o7MQ^U{L|GQr)vd+cEiYs@xZ?2(&eF{G_!;CjN2ipqWw(-nW~3$47Gh!B|8mThBxUV0~If4U4lL|$;vZDGkZiQ(ZUqZ8Nc}yDgj~V69rKb=$RMIWG}qLthT8Qxhw;ocvD4E1(MD*KqJC{ zOPJlLpvj6rJrlAp`wN2rpd^WYCj)J#4(y%e8SNC%ZjLH5%pBlGsW7?-&LIp5fhXS~ zQhxeTZm{+PPtJUa`j44LT_224Oe!om^clf?K^tg+_n|_A8kKLYp@PF z`)n5KP0TBjCe*=(BIm3ZY5$n3=bv$@QpB0Go9QQuj}mH-E}!dNk~0E9rkpF%rdaSW zcGt{IlGJ%%sHiMAkrk(qWk{m_?7P-Zx>Ad>h_aaZ;)(p*DZB)W61QOnqwxfj4k`AF zIYEzBp|b6khd_qiM|hZ8e59mg5-YBx%Fj`nFf1y&&>h-EKh+06!Z8r)bcok2lb}YU z!QHD$tJy3KNT9)re+Tp*Ms!r`jVawR?0|4erXpMABc1A^;D^G)J~UQPT780g_F6vO*bh)=LOWT??CP{jZX zkEpf{-gPYWwj@vtM2I(uG+7pAqJ0cU^=puCQwKQuCJ~WfM58yk)Mh5pslmF%!CJ5F zC^2D|x9IR~O0db2)p6y28}1skUZPby=E@1Xpl|#p#g0dgGt+xlmU_kh8x~{%D;lt5 zr(CT%OGdjROgL?+RH~yA9pK{-QI$+|H@jetN>5$bSMx+-Z!}{~ontHTyam?-m%_Ip%BzhBKo?NkF9}8Jg zpkjq8>9R0cm7_{Szwjg}o(vi*|8X%{Q%WRS3>pfIMN%ZY4fZC?WbOB%UCq#)>TCXt zrjN<((bKllmTB8GVBdWt@T^&^47K`O*itheHdLc@J4|x8-=bUY2N-VV{U*=+k^Cb^ zz8=|TumyuK^SkQ#BAu!{5yu@v1UV32sdnDX+;@Pn=KJVr;A~k^`6^y`3>S3EeU50C za><6Nzi!sML<#u_1&xt+c z-LB|MM{F^9w5( z+_1U)_7K^>S!FOVTJ!D?+4ApdT%_8C9v%~a`PV7D)L_t6{Wg{#>2+Niyjs>Hrg5sir_Qzh5y?VE&;*+G{s+XUGf!U+~GlhCHDxY z&hZFbU)R>B(dn%KW$z$-H6>sB2u%y^mxHUp4af zL}J<)H_-nWy);Z)b-ms5%pM>1N_xSyIAp!RXyfFlvHSDqyOewnweHE+=iO7+?&L&% z+Z5uLnrFEqjLo;nJM=v@cBibK+G$W&DO_^OLlDX{<$(V6K&LCiU~0|i92nH-;XB-w z=s9~cN0EQ`Om_$o>%Zy!_LTEB;eY;=GhsrNC`AP)MG}lE1j|Vl@!h@h)@gO!_ROyM z$LeBXWfS}dWnnaW4+mIbWKz<*&r(#QCamdC-9Z1`3)_#ukit;X^p5_y)6xtX{i|q1 zYeTVqk2ve-o%wUqW@h7VERw z?o-cn_VhXV5bY>x7j+CP(bVOXB+I;W@Q*O)(L-qaR3oPy6bvO*hf3U_TC^9okWiv4m#}%(6LB!GO0S=TKN`j?$JFHB$&# zRmV<|^CL11uGBRI?_;y?s1!t@+|2T_3hOM@dS8`Um-W?%ar@oCt{byWoc)sTQkI(q z?=My)L!mD{l%j(1I>g1JE?*vDtSG2v5b`n%o+77C!?~?PRlx}@}AqIRk+C?Ov ze6}D4cuRc8GxK@aXQ-eXtPplG3`FPek_}c~=J!-p*N8i*E4UP&n?NE0lhb(XXn7^w zDLUA)cez_%eu_R7kg5Wc=qlS92Gs%%YLU612 zibvz}^nB-M%3UY#giELp7lO!e21M*}O{I$w`_4rNP~ftXWDcI<_z zfg9!P2kg=#wrsx3-`-ppb%`hS27H-f42_Wq|0N5L^?uIg%%c z$MUaxC~v`8g@XI`=SL-NZHO&mh1{h?t21_f_aoP`T5DXj%-On6$n~Dg1?{i)^ zdO#K$-+g+pZwS-KnON1{xV!H``wRs4EN_on@)-K>ym>}?dtoW9?e`0pm`ZxCxpqOY z+vJ2d9BPU?NT5^qb8~QM2+HTOOGA;c)>%Dc`@Aqp4hoKte*~2BSv6h4 z<2fH@SDo)%I2_^H z1V4&BrR|-e6ch7D)fI;)b1h(cUa5V;q04s&nAmvx=qvlpd)7$7Yh8g~R>n75^FDWd z0NjNu5ZFzA;!wxuDc4cdHY1n+#P-4pTK6G!*uE5o`}T&M@ABC( zYej%Hp7!xcq;c&M;B)#olqPX^b=KiGn07p9EYBuT-&-J;yEAY>yRRl55l%jRHCZDl z{o3R&F6zrzy2AIHf$_ZLZbYg-(ixQ23yWM^JTTUdP_c5i%rXU=vlURYJ`^66X_MN2 zdP(BEp)PO>`l_X5SNpIr?cCd);uR~xCji(QL7dd^4P4!5AdqqSJ}KB5b`}IJP#V3p z_ijF&&+d6ZaQ{l2c%E8s$9vN^37r-j%I!X664j0d=WkyCgPURqoyP_0wtUu!aBN~= zAN5@#=&_bURCYL9$8)uGW!VHc*l>5%0z*(W=AKZex>a3OG-LZ`HvD^C8|W^G4ql92 za^Qpt(=q_sG*Fi!8ZrWU&uOpvFV^ZJz;iEe4Dhx_vnR zoLgg3y9fbdxwY}bGF>UXFC4F^t&|eKeTie@zeJ>7tTFv9r%?}_6;0Z>dB50lkLwT8 zU-n_`4ST)k!E+)Q9FD!Z42r$li?I^$Gi7s&)~~NSy~1DNHHEVI2_@`RD+glytL>`LC<+PdZNC~YUXj|y=in^%~<{Xg%?}DHg!gopSaVN zJZG5(#g;qJe=>T_4MZlJP6IK$IkYM8NEWo@n-a;W1?UF2E8f1M)aAG+BPh|0?nM0o#$RI_wo%mUxBJEQn6FyTAeIsv9KLHDBE0b37UnX4bw&60 z8-QN}qk87uqx=fpUy|((+LVI6+C5QBiZKT2PCEg7dAS9g;VQ3wbWYU@@8q9hwDnD!8L7f+AeZ?`Zi)*8EBaT9SDZvrzu-(3g$9*wh}kDS66Rq9p%{0^L1Si(^nj>CA74* zskoh^F8p8Q^_B%khUXxs`ER*=Uz?s+KAns#VlDqZY-y!OJ7_$d@#}sXF~k$NWMR}* z{mxBT;6EPZbH*!Jd~n_3kl{Hu8uIos&v3iq&`$WDu>&4A4t~skY@E0Ep@s(FffHkh z)2ZbJHm_EZxAdGaf7{W6g9ut%>8!mhV{LRCzjy$>JdNI9Q=j|vXLa^=+3$)u9UrcF zr69JAd6zM3ww*Pq3uXYcb{yS6X*d3var!HSYs<3o-jjJ;!PbzeZeb?~>Xq9GNcd6; ztQjizI#YMfjUWJR&mo7pI+l?#sr7FvI}Z>X*yHjmTITDOT{WwVayZv)SjbS=lA=TKxhUYfjm6;kj7NIlf*rt!!_b)v zyiG>6x+}X!A3~f*iUNz&_&UM9q2At&s9{|qMPNF`yjjtv=UeBg_Uj>W&-W!m($5?p zC~zK!bon~$)9_y(D|LQl1&!4(YmJU8<1XJKxFtPWR!o?#*uF)M`yk6&VPq}@t{4q$ zHyXDrUT*VtzYCRT#iGoZa|B`;74g7{f38BN$5IV=jj`%*0)=={ssPMyBtaFtED;+YkcPkCMEU0Sxli{+omhoCf(o?)p+* z&*Ko?v?ccVuRdsR&rQ~j+H#%of#cO%V=3@9RG2>4&O?-&Kz;w_uyp^{xI6Y=ZEq_< zI%*ABnt(_?%&%kmEe}c_&J1nQKF5)7knQrUM5Gx%K?j(LF-RX{GH4J@3|~NOe9c{P zm(Z4QT>%wC#rwG@?5Ot2rPM(zI5~qXUYADm$kQent=5VRNvvNE;GT^`~ zZT?O5Y8D<4FOL>2F`AaW$CE{nmThXtBLYmD8Y0$~GZp5bThZ~`SL3_&!A3Q|obi+Y zf<+_X`3*Z1*;$(hgPNl@djLX1=^dB$s-|{M5W+u!fGKJ!@IrryDALS|GqH{<-qM%1W zAaA4{7rV~Ob?U!x^ZGs85~a(b(o_Ni={Lqp)RH?{vNgFxCXV{d*_=}X3%=EKW@l~^ z?*PBvUY#+$r3F{Ug&s7mJ-^b^x?4Y2*t8%(g=p+SR!9O0DO`y^oqQejxu*0IR0K@q z1l_N4D%k=j=h34GR|jq(_+{l4(B!G$q6w(e)nNf@@TonitJGDUt!IX>``g^Io@zRc zcIc`0HhYb#o9_?sDrRABXZhPRI_ZdkPLL6nSH;{eE@|WNoi3B&BR#c{l^i3LMu|3_ z2cBNy2jOBt6Jgj~i~Q1Ju2E6QG^M8@=9cJ95;?#SYLtdoaW+QVi1ap!q*d-oW}}H|=g@^?l0;le3~0VA zqqwNh?^D;%7!w=_&1+5^ofm~RWs~CD4CL%1FVn#|87Rg$;7z=a2WGAg5PfZ-FVAylnx9p&=s9MGV!=Myq1V4wX6gyTro;UG z1j8<#xqG7dTLLQWYYfls!Hn0y&G`fF=F61mPsT@rl}oW6PLauIEIFW z;s~@wUTpdkPgT6h(Qeta22Q`W$Sk`}_gt-0yVo7FrGo1_o+9e(OeDDB6ue#X4iuN2 zkwEa%(vNm1-27s@JJ+=JrefjeBVlW{n^VH3LE7f9fyvwM+M6T{k~Me;lYV0w&*Sat zMa{EPp8(My%tG?yIZ3k*I#9{s(s8ul%E*&*`X8f|J7|0S!0Gs(^(t5!`4s=*>FpB@ zad1NW?2T}s?L%|m>)gkYBQ!P!!vsM}c5fqRYp?OEZ7<}x+vpe8dH#1O+!&MQx=oS# zHhuy521#G|`dyYHZMX+<&)ZFZ;sZP!zjF3z25MS_l6yBhQw-uOg0L5rv-zhGR*~88 zV-mkhr1L&vZ>|m?NXIR4@Z8Rt6q?3R=q72^ba#k?68}Ch+yxTF=bsf2}f9MIX z7DzMfiV`T@Q$o#1QS214t}y88PyRg5t-ILc1APe`Q6&~^yGHJQzF38&M$(gcuS<=l z5GOf!bior)?ty#4*p{XygHOv*%uNKBAY@)*j_`J)@HQ6H?fF5hsj;!fK&dTEk}Gh^ z68Y+bfON*9qiJSlsoxDnVIL$=8cM?RM!#pIC6_1ae6-LH#H!9%-A{mmz96hbk}Vx4 zNtvE~SZLihHun2Ov%!mUazP9d5St)X$|b6#`0WcADk{8&=U<*C*Y64H$UPcPP?KEW z;aFrS7Y>IBFMcN$qMD$|DkskAkuIX{he>&k+kAdZ33l^{h33AGlvr6{0UY*8+&^m& z<+&rn(__yz7erg-4vWzA;ioURCB7J65AHaM4Dfmyjq^2Oe~vGFcZ)GZoFjS@06nIeKV-|)IWcaPSck047e0senI%+||(-(nWy=6y1 z0(M9_tHkq)hM{`@7@wnmiukfnRG2ku{EOwaQ$#4QhsU)NR>Uvbt4uEI81iX?0nPS+ zN=B{mh-S*Yj()`4w7nJ<#GrWNuyFHFhU=lfb!t1)ojSxX`WXx{_+FJOd}y1`c=}zK z#rg1xoTt^Hu@2thn?^^{L|vF(pBswQL56Dv*XeHVHu(RZCcshnC+6MvF#a!Sf+{h5 zKCq7jIV^}XVbtWicp(NB*465l$6Qkx{r49STk!BYSQt|kU;W94Q2f(vpGqHYU<5}} znAQidyB*TRpv=T!Bt|^AL_rMuFD)(Czr%X^1zj1)0PKD7)sCkXxtvDRKeor@@Y^l| zyzif9^p16C9~EK@>^tIbBWwsZtTmV1vET09^^1zfruzqJR0bVCGqSJ%`i5{`^&U~x7?%fzy?!>yU{oi8NoA$o|ywX+aJvWD; z39v$(({1doWTBZ}?}c6VAsDRfc0e#3&cJNZB=d6FSb5aZq8TH{Q|<*--rOB;_g8(l z1y!R7lzF!Ffls-11LA@;inj07bv&VX-?lS-fJbhxJ0)1B@aAdRdKFeqp0Be{M;BF8 zn*^q{?n%S=0+K=N6GDzeeZbuP4nf%lPyv-TeD0paoloNIu=2-zTB<;}wRUOtc*x4! zIU_-ZHGj^PCH*KhAmRdHUR`rshu=L?_ncPbCPrb8ciMiYfVj-Nod7GRP-0*v%7LS8 zWUNM-lT|CI9m(gU|sbcE0POng+zwV%Kkr%JsRE?{2M4fOn4o1sIkP zuH1iQVa4_PB&I~8w-)LNXjprhq^-EJnTU;HA(Aq3l^8Kec33Hq!EL=DeKj;;T9|^| z9pGo6koy)`YWA79wLR0`ABg* zZ)I)mhy|;CPN~p9uCcG04UJf-=}U@d+!!#Ydsu49I`dz=Rk#_E*xli*H=Id_LD`xa z9CLc-0S0ADd*Eo~SFw?NiYAB(z765l{)SB8(W3FDK)=9rnTTlOz1Gr2RA5L zS?#^z+Sb4|aKOLuYsYkqPEXUMRMnZ~Md|SAn0$-9rKSeUc}(`yhZ&DglSb4nd8nmY zeF@-1UW=4JQ-KD`7^f@yRoyqz9Mxr|G#%f@+O3XH@MuQs6J_b|BaYi{GORV#_9fKh zmb+F|RW;&(bw;2)#gyEj3NMB zFrW!&Dv~3HDm521C8}k}jhz&mJ&=~{eo5MrURbN`ejiq0aE+y?d?=>DfRUuGtgQ)-+s@y{NZ-BVm2ncO zm=J__7RKdpFXBWK)c!#W}^~NHMiq4+Kkne@u-kJy}aP z0Zh!0`|B~wGTPz50Ja`**xX9485XIhlFITcZQ3g9lKjqhY`8YB%D*`E0G}%>>uBMr zB7@fEw5=lR|BHa$S_%j3rcIi;=u+cFe?)}#qz6+@ZEbC7vQ!(i(j=(N;1x}ecc`HQ z;{dZqh#^kaI5Xl@ljpVBuhmsg>fixGaL_ZN*s#qnORiw%yw4!cg=Z392?GkuWG$VD zsj0mVC>!&+g0m?7vfu6+cjEfQ(j3zBkh{4X_y)qPMkd&$2UAsZD82~?@8YDKwj|z0 zf+P^`Z*qN;;A_S*Q?v}GcViueY`4iZJ`ESqq4JDryzL=3#%_!f|F|W;?_zC#f?0b= zbhLXtHMf9IL(Pg$JKGH>77TI!{aa@(E;e-P{ubu`R3q#m^Wx>RE7m_%ga-^*0c<;P zfq{82Ys|*SH}MsGW?^O3Z`J^kXZbT-+6PrsfLS;`B9u4N3GSb&MfU(+K?Q$rUdlx% zWSiA6-!lnKl@sWuterOR`$z0yOZ!UEbndY!V@D33d)5ekI{Wa0g0hjZfjhtGp0l2B zVglLcFCDnF8-s*revmz!9}5Yew@PEL`(=C?o-Xy3u$f6ck{6CB0X);b`Y)^ijpm&Q zlsb2Rlu6=y<70+@0z8-Avd%o&xe_F(S(|oUzttlmB0Q)m5~W;p0ai^{=Y`ufdeWjX z{WpV<4N~}%C-ytO8ZC*Y&=}O8Wxih0Bgo3i#k;B3AEkV4D{4b+=>J-@!&qf1<;r-y zD=I6FA)Tr>3}e3{nJ+4O`9w}=Xkn4DgPGat=8MjV)WtE!W22gGI8fUCO{E?x*3B)kFNup(<`#|U0=xJP`1g#`Nv^j+lqt4Ha9 z!CfFLVvw`#MrS%W{Ne^2*xSQ}CmOK(UxX=GuisB|a&m?ZY?CBQv+(mzUS2ZWhYW1z zm6yY1P7d>m{S!g{+r;bBqWk;szaRwVe{wZ{oC*GP4u_P8zYqTjMmMPbv*-NlR&dMz zwc!8XFW+Lo{AW3TKWJY5&(HVw%eiF!kBR)psQr&1m#bnWM??hFIdh)L72B5wifP!r zhDs07HyF-r&Si{DdL7Ac_BgmADnW=*VKbheCXXp#KBR_Xk`>bsw>((-EgzFV z9QNP!2;>w~`b_t$+Q5>9Nd|y(xHb$LTn`b387Xbyv5-S`cgz3GIAr8Vqsshs0V#4Jbg0Owo3#VM!p?T+g4HoJ zA3dFa>Ux^MIIrycw-(d}!#Me&aO!y+=rSqRBnXT!(sKoekMAS&Lr}8p>ELUMk0>Xz?aA?x1ESb4v6wsoFMh~+J z44PN!(mzjOSMOjgGFA54y+>pY)a840;S?=gz)r|u4#j%>hcl9RUU;L5wT)4WW|#IE;NS}H#TwN@LS;Mp9_U_dO2$ovykhv zZ(%?+Oa@F={V~%o%>Jtjdd#yoU{;2(xllr7z(tk>#^?KA%)3z0w;l#knyjix!fL4}Em92Izam#gHS< zR)S69Y;dxqbW_{zEGo0efJ+=5q(99O={y*qn4@JCvTBbNm1xJwiOx4P(DePu-mI*@ z6ZVsj8pYkA${${Le#C_R$~r@9Q8ScQKy0Z`G%F1@pDvEI3S;S$nUpR|d7GY|4~W^U zGQAZ!r{7Ea#5tX>IWlGgYJ!7)>iI)dV@!}r6xS|4%!*$vu`Je&3A}_9IejWa|AitK zF--+rAk=!u5?;@BOx zmQ{M%VlMZv*#H>WVzrrhE~iSZIc0^MrKx^JYHySr1JWP)IJB}k%G#?Lw70y94@?DR z7F0<8sWdsooRkY9*?bql@u`ssV?bz(;dnuFU2^*lgXGk#b8C<>#5_)hxHzYh;`~It zSaBIhs+|XUmJL^24N*BXJK=GNp`4zKWqlx$i4feA&=xR##C|s}?lZf&V23>hrrE0N zKtY+z@z}|;7)7}gn*g@oGi%MD)9UO;TH2Qb>@~n&KG2>153~56vD?Wl9vW}Fx*{-$ z>qR&0!z}75`k65dv#=4-^#bF1o5rsrQq>`+ybXg=W~}_@kBM_$@r@l0D};$Cx3gmJ z4QbI;s6B4Aew8i)F0XI9E1pRupRGsE1Tng3FkUma6kE5X=(S-&9=h~l({YK})nMiHaY|0%CkoxNqjluaEw zuaDZh))I#9jBJ%!UC>+1%H|PR2tamN%M>fd{4ODh)mU3l4V$S&&gbww=9L18eo1jl zF^%}x$X*A~N85+SgNzc_Wr2z0iOd?WbbHDp?Vk4?^RocP#!EpW^>T;z$3%D>+=7P0 z<#B8!wZj2RI?^S-gz{}iWlzeqfVMtTc7N{ApVYYAH}V}0__P@rZ#(lW4SfbIPe z+!3^J4tnEBo$xf2a!*Lmi45<~<-IC0%W0XJ28$yFEaNfc+rtHxT>N@f(iz*^QnFf~Ey`)G5x$K^ zMAJaLpkrVM#v`)j&XnKm#qGOo)5{sSmSxo>uu*$%udv8ge89m0b2RB>rPbLP1q?s8 z9OdOpN2{9?!$URSu{R)5vI@P)Vt}_c#yqQAos3^v!E7*^y*%cp%$NH7LO6T`}a!E>v zFubDz*Pc%jT>GUhoT+|r?WI8$@Z18#o|r5c)^WB$ELWMo%IKK}tG;cP*E2tT-4<8x(PrHGMF7;;*_+$y*_$ z0nglh9jS2tPz10W7-F+!I&PF_l${eJ%xn(df2^X+Dd|jPd}Q*3vg3IimP}esCnD<8 zn~55?83XXJbN(lle5sJ=*h~sGGakR&2smbQygmwhU4m)I(4uVlgoyw>VzgIq%z zef1?;Zp0URKx+5I#WhuS_BqY7{&(P!t}`UFTw9M97Lsa;wfuU&OTg`xby`)tr>c?1 z@11ZoimEjXSAxNGHso2BML(v)EYFi}QdWu(twD3IgQc9N*=Yk8ALM)L@d%)Bp@&lY z{c}lu+v!qY+!?5}su(bzWoB2EnlsC_FwSV?Xo`#l#X8{pWe)D=tiaBE=h&%ZVM+f) zu-Y0Wqs^b1twaFKDHC%@^S_R|dot5!<@4BZ5vcFcd2E(Ea- z?Es85Bps#IDaY~5mz{S|6>^uwe(jZ5%^G2;wK2bg0)Imfch>*HH&+WSJl3>4&MdJG zTxePnu}212nTHKV0Lxj@hHWU7a64JWG-m9H*>eY&3^ktyCr%cctBb3wLi6L54uDig zg00bME8}8K0#gzh3!;XOv@3S#NeA2fjgh)fZU*36>wdzsECs_;i3=9Vev>0?+nG% zQBogYKIEFeMu;NV95~0Rva9JUc6y!d#DJ+noB&8s^aAPf2VJ9%+LWRmXl=M%_q^{h z@u(6b2K4(f##cd=ojENbzXPmFsE2WKGYqX7BeSn@s@G79|f~T}_U9$CAm1bHK=;;n{h|>+c{x9O*GB9p$*B5QtFmoDc zn3-|H%*@Q(Ff%hVGc$AA2{Sj$%-B#vZ~EW&ocs2<`@OGMAIA3B%eJJo>?P@E$uM`P zR`)y>w^~mvM)@B=Ys$Y_OYT|DeF?|~>7nCQpQv-YKdrPvQ^ejQv%idv-WJ?wVKycC zFBv$Sgo+3qML)6TbhS3NnP;Eukl3vGWJmuJcM)c>wME$PU^_Gye$`|2qO{YIDxn!_ z5YCC7CL=2I2Nln#7-cfzF8eBCF23C5!m}}t4ckaJF*$)6d+IOf>>Eh(!hM$T=%+7V_gHSpZDf1`y*KxX13;br;h&e?C-5Rr85ZApP= zr7sUnrjLq&J4_GXH{`ZtLc8azw6wGjrGqIAoSu zY9vDs$Kefjmq#Raa*I9So87U@Cu4DHekCyDwlaSv=3lgrKOW)p-PwSRAXZ+_xSR(qg@&gLikuQ80SPV z14R&!5KJ*l);I@)Im+N1BM=eL^dCg{n_1 z#l%SV+3EXj9*tvu47^{=N26oyF!#a7PQ`0)hRz%AxQVuy&vzF_R2(5X9=m_CvnG>1 zeLLsuj5;zCr?M~p`VE`pprT;)yKt4=P|pRjc55)zWGf>ZKx9FGJo5b0$)acL)#Y}` zH7NvsNJIe&Ce38gVL^?lu_&o5PcNTR+St=3+FpzTtDb9e;0V&=B*Q}r zwo$Qg9veU)LBZZsJzTd z=G(?>lU)?z26QudzL@b<%Ey)ZPH%7!t`p6WI01aQv0QsqPWNYQp?xls1jU|-3?J-W zBgh`rD>Z0*zFwURE{w+*Da|j>wn>u$;1H!UU+=5WuPVvMjz|AGpXZt6991-BuT^>N~2TL!^Um-`38;$3dv{0`0%K* z0d?h(N_pSp^Zf+~;}QV`qZa8Ps5vZ634$$cEuw*%yBmwM(kQqkt4wBSsg*@0b{NOc z!o?KUWKTsdS)dVVM$T8Vgtl%hM%k8>|7By)WgI~>Kg~40|@Y~Llw|p zJGl0-Qg_^h{&KB!jkz$hswCbzAN}h9#I_n+p-jaxQ3t{b3sL-N0z2~lB%S-S-pmY9f8?bOgtjOh67>h zC282laMmm;CxFF&XT0VEdk!EUZ)Yu9j2H}LP&bBIQ^xwbjtn{46*?I*H0U##z%AZ> zM$R-4WULS5>@@|s=2p8sSS~Iuy3s&BUQo>qkaPFH%%^KtlXR}`kRvFY>Mk1$Fc#39 zO2*sF|6&bk$68m~x&aB|=3_Z%`mw!s;OG1gr^}jf*iWz_*)X67p@C9>WHsz}4h!&=WbWC=XE-R7pw3`7EMEv#6 z+_IoAw;b^PCT(9|-(Q0c>j#ySKmqx%b#37Pjr6|A@-T}01rPX#fK|(1|Bbpa$6Q9RXa>6K?)3E8P`?Cm$X3J)i)MCse~%YQ ziTPiRR{pKc*KWu_i^4D@zm1>(U>5bz4Ac(;=r4aM$S-Q(=YOPaY1Ds?_&7S-Xkrezh~kBaOSYY~sE|>Kp4?_AOHhP}!SxGL!pd0n z#i3y=ikp!?3tIyY3fSrh>uunGZ6!B=ka$d`YqFZ_WFc!kSqV}cI?`11uihz(lqCvU zcg#fKN?OoBH<6-bG3W?noM~lfq6DSupgGv3c?l}e8uNjN1yOzRKfG>>ixmMvFj1vp z^$22=U}FXn1KSeS+@(n9;DFY}7`by2mPFwEvRVbr0#ZP2d75NQ2`cw2i-R4i$sgpw zB3ht$T&ZI2XR3tH0C}Qf5m+F{tX2WK1Qp6^|NLhvGNEh88A|zr=EGm$bB*zVcH%UE z5|lY1*i9)ds$?~bNh%_`VsvP}q81^Nc%f^%Ih+zfsQ}<(-w;R*+dM@hmo<#f&e=dQckxyKLu+D+RtM$vZJ z=y{G(Pga&322F8{-956ML0$-!wN#U+d9?DIelqB422b*??BhoJ^A`^HM`&|ec2ms)_V^xn8NAJCGUAm*M5xBc{%1aTFbkZm-gj*I<`A5-w9KfV{-k# zXTilr$C-AqPZ_vsLxv`ty8d`~ni{NnQgfwv&TA_@Zolz!Y)NU0F7b71X>aTJI8aQDJ@_lEt|+FXnHz>(I^H-!%E@N}nreEA^}QkD|xW*AM?xdk}L4X4<`Q+)8!5RVl}FF-tKjFYcG%8x7UzP7A$uP z&7F|B2ij?N8fX1>0UzXGWDVaeojEd`TSI#(4PTtDsrh;2?W(|q7aKh+eHW6T{-qN~ zbo9B2>1oAkuIa&8YB>lJjbR5B7X)@leUdrWkpmV7IS<5>{%`E$mkDm02mJ-?-l^Lz zl_nf30DB7sLX+@Ipp(YvS4_K4bt6N&wBxdpQ*6V?|2&64X>IlWOLGMa6Rd0kfcn#U z7RL63k~IWDOH{*)hximbpt=$eu5J$_<=}QCG}-xeb!j3CyATJ-Tsk>BCqWb{Z?7b) zg^3&y{4103rf5jFX7wb^k9wr5N@B`}b}j)WD)H87oei{L;F6)Azj}|3Ha`6ZLkk1* zWY8hQX;e%Bc%4N7JV6O!ci37mxLOjdDbUbV-H5#1zBab|IXuEPHHfFkSZ*BjlP zC^^$Ph9`8Fb5XdMU(e7m|1mRmv(NJ2s2O{?0p#(DsWRWgtDa)&X_OtEm7o!@ee|ex zrXcK^JRX2`WgePcMbx7V&*CLaqhWTqV&C+_It{$A({r;_nD zh&{CQa8n;a%G2&%n-o61D=w?0vgGF2O3|mUWb%-fp~Y9wg@2glMnxS3(oGtJGiod8 zwqrB%WYgd`9U7sd4z1)v28{yFnr{wrrtB-)`oXKNQx6Nb8K0GL;Z$BFSuM%0WR)sT z2-7tUX_Sc-=Er6XutH+64>^>Os}ClA656^Ll13VNhIM3kJUAh$)o0fB@^RzrYIN%c z{+i6busn3qQ-?!2L!34{T0XNSOkFc`JPCF>dWlef4uH*?->dG3SwDth;o%kM!X#^I zl_i@SDJk$Zx9;rGMQf$ogA88|~qm+D8%%>40Nh>+m*NC-4?R_Y){&a;y(dj|yGfGILrsHH4 z745((O5dj8wVLHN)Vs*6C`;&vFK$ZzZXK#eJZ$GmE9s?@vog*Jq_I5bmY1J1Hnd9Z z$(AlIh;r)D9SA~g9=};+R*pK`r$&zEnB`v<_F?t^``+BXP+QgA9@WXr6dRoKOjQ_% zsC^{o+{`rgw9HC`+cDw!(qk^EqS?KD7^jh$Wk#YZpe=aH^XK@?hR}j4^5om0+2P7G zmj=+P^Ut4Si*S-;aUw4nwMFHXEHje|iK`h#x?hP{w@|n;Q{QKuJ~?8x!~AY9pQ@xSBPYlFV?gk!zu6gSW@SLBwJcyekLV zZO3`tbfY_RFfHmYlP*dp%(jXIFHuPo)%Xgx z=o);PUi2(8nlY0o#%`sQ;;7|CvT_oR7e#{WF&<`W50u=nhESHrr)58S@JBhW$Ifdf znY$XrMI{Rna^SNs3b{!=GN^OnrK#oz1tjHj1j1|=tC-|Z}8Eo_ZR!taD>WbPOA9l`8(?MxI^?SRCY&%ga zF2*bfY+w(VxeiA+@N$%rUQS;G879wr_-}2*1xtPc zFJbrgs-CRREG1ZH<`iNs;1rh@gmfwWx9dtO(C5$3@su0^IuzrJ>)=RbY1hQ}13sc= z_*?t>?+40rjj2Oc1YvMUUo)f%Ht?K^NY-)mDbg(pxlqWv*vGE;a!9Xk_Db1Ogi=98 z06{e$lSbo5O-A-3Eov&zo6hND9ITzucxFX(MH*O~!n~R^XYgN|>S+l|-!x}AP@pnf+2ASOrR3X7kiTlm z8Vk@S?)fn^FW#guFQ8nAv$nFE#J_1?wqI7z5o3~K;ByNjPo72@Y-J%Mb^}paZf8|6 z1O>IWR@^9xPS3NKXMZ;et5+Vo_=oc(#eBW{E_eeCYMR&3%QsS7If`?~6Q?k+XC-Vs zZ(~rk8EG!JBj>K zqslKf#S5DH>tp)8v#@=o@n~=6J}scQvJ;%@CVrf_So*5Wo(Fx#o9#mrgi1Azxp_!U z&lf=MJQ?C6rslYiimHa^tGF0>TZ1)@rBtv49bh*pjKyRe|M|`3GH%_N&U$`ELDtnL zlZsxyj9Myr%>W9Y3te?RldI z+mTsH*az>*KB21(-%$Zz`2L>P&eIh3qctN{B>a4c(yyIMZU}Oinx(PTjhaamNvYfNq=GWdIpe?E_jfRq9_IvRPdyO`~x&O4E-_iW?6HhKF$^^@e zeFCu^A4Q(oE|6C(4&r`ta$o=k?eUVSEe}XjlF|J^flr5zBFjFH+_aqA-A|BLy=x`9 zLpr#+K!RV5ha%68*5pXb`x@NvMX;G&(1p{Ve^EeMn2solAX|Nm$;U5vFqwFEL8?X9 z;1K9j+7@R|LovbkiuP#!2t$@>QGL_I;grYEt5tmLk$fv|Bg4P5`AXkl0_UKgkTRUd z+QQBRu$zJhAQiGgG{<+eb_TwuY0g){cnAwT59;5i3H@#@TuB4;`%w^;dH%i*A1NkCt#%W5*e_MxBz zsPp6?P=!*Zi@JI)zP=HgOaWHGLhfh=9cgJN5=F&b3%@iK-~n6-JP!{PdN#4NF%q_} z1WJuvK?S*h)eaa(-uD8=C(Qq4O{nJRba@Xntzf_ZzES?1poN2zRW40hZYO~X_grpm zLB^d{*jg3U+8(!&^ocI$iUlweO$YH?<*sXM{pgqy4gpXmne7qx7Kd}zf4-qi2v<(|5v$c}Bd%*+-eDci`zZ4P1_<)qHH%*VL&D{yP z1;V!Eyfq`~785}q`xd5r08Ji(6MaIb4!_=cA=2sUE`Y~A2^j{N-&tlt`VM&=;oiW> ziaQ~Ya13j%TETZ1YuAdgS{;1pCH#||CDZ!^PZ(*EGc zw%#QplXA^|mloDJmL(3zWnuEYlb@#CzqR0p<`FsSz}4Z!9j4)kfB*3jZgFz3G0q#j zLEC$KmBB-+Xh~Dg+#^(f55W4P1tVn<8Rq`mJ$m#WFD9tvugTSY5&V0W>6N9nWm#cm zw~d6#s;kzqpbSx3fEefSq{vcDNf&${Z=6Cgl>aGyvy?h`UUvQ#099*d=b61{0H;2M zq=@?o$4Fx@E+b>D=*p8MugP0-6_*qaDr~>=G{W69wlz+oS+D=-g1JSh*yWh}i>(7` z=4pQ1u<>2BxmJYTWPIb=^L4*9>)@+O>{+{)b+EB7bYNvFSS{T9E(tXB z^$K~>P0UE7*Fy9ijPo(!EO0*P@+R(voU-x4JS?y|h5hW{e9KY>`iNsw_vgkZu7svb z;GBP{6IqZS28GQl+vr;DLXcH;{MiwA9t={bEx5gb?po8N*ZrPPj!8=Jy`-xqfFt1S zfQwU+^#N4vY)ej}B}oUA(tvbug1ae;J_n25p_M8!Kc!v&*1|D#5{@nyjLcZa_k-k5 zqN;*q{=0ov&xLXA4{iBE?ui7eLm6=rJKVZ#V=tcwsZ}V8b)(FZ6;tB-FRT=xs$v+r zN8jyFbXYK_5y+oE-d~`flh9bFVX`_~1#5MAr4@bx`%rSQ>gLzx{m)Z4#r~Q0{uD*h)%b36RPH4UJ1AvY2+?@wY$FuU)3J{aW^^ZnruJs`2bHnW~y+*%tTB7FXfn}jWACiR5bHX}YK_@4LnBbT&9YW01i}sGp1|)3#LkASsb|D{(L@+9Aa7y1`chLMz4PV~e4~Vs__3}PV zzNH&*G{tDcV7ljb7fh&cNP?efwv9zO$ z#$C`0u|o*aMTPvRG#-?zPXiCT6mJq0{M|%?Y%5_@tx>&3=;4^;)+C`(Q;6b6 zfcxMYE+AZGc9{x&Q{Ez}2R(Yxl$`@L@=rU-SZN5tsjQ)+sJX=7EKEF7m-adC z)u}_Re4b~8$(UlJ=y&Zb@+;ct-TBlIuqDJV;*oN>aAFm3%h|UtH`?qeN+{vC?8*o5 z*3v9C+5n1c2&09Z4hi!|&BTCDzx9eWxzJ;gKs{v???UMsU?hi)g28Tq+o6$xSc34RGw=U(3{*C}lW2)Zg?2fB`%NOL_XBj&htD;4&9^Rf&K zBk{1>Q?xqM{)%TohKh^_#^;xA&-#;UT<9rKB`oj>+(0|{GngFuRso@``4lJvAZ*4L zr4!@WxQ7ugeBZKYT)W~Lv)C1iHZILNBGj6K8uReOOhzN9=A-(K>*z?d-PIJxSEWyU zI5@b_bn^CRt|6W7$r^? zccF*}KCB>NVm@2qry6@|^AVc~hoezambEwdFpmgvGwT4*+t5TBv9=Xjj5Znapu?LN zct#{}jS5tULB=W3RmqQy>Q5_=gL)1`ehO0bT?i&snitWw4*s!!Bb&TPN!H*K$RNQF z7PTY{-79qs0nFp#93!7y9X=>vf9)3e1EZPPrdyy=s?$&&C8dx(=Ph}6@9dE`OoFsL z!D&%5(~C?*cLRu-^X3HwSd7*ZrzNTCX|DnAGS@xVON{m&;XzqwGBs*DcTX&`W(QOx zd)CRkA#;fW193C3@*Wm?ve3PR;C_d@TB^ipP44f^Xa9@!p4SQADP*IYd6Lm!U(?mMH=U^RZ}C~+|n$kBt(>(-tXs1 zWDvvVz|4X4g7{OUfl8wvI5aRWUNH}25W0iX9XBbhTZKQ zQ~FgOCH#p??@RD@*+?uaqxT#2BpaUqAsw#Y&$#JCfATJ-(Vhm=TMVocvH%I>fFWgk zoGiZ!6SvH;xTBF|-e%+{A734WxnKU$BB2 zHH`F)QD>c1RafWjoVFMSb`w5H2VTV;nplTI8N-=K(@RNzP9V%z920)F=K6SNu-M!( zaPZCem4mRjsK({>u1hZ_Kmw!AF;A;JAyYG}?uVa8R&I59WcS`0agoA>siwZ7qi<$w zn>5a`Sw=_fK5toaoje39VrdgKSI(AuWMAr&)gb)YpA8eIj^ zn~!v1nS6BH(tHS)$qzT<{s8-<_R7fPdt^!r*J3J`#^%@EBBao8DfoBpf4yZRU~%R7 zcHT?#aDTk}r&1FLNq7e!H&~?mliBn33LV91W@c_v&P;E;?c&Y)i(sfd_uF0W+{m(S z5lH^TBAH;<+$pPT#RMzAg$WymIFH{i8^gLrgZ0F_*jJDRy?@M<@zO95y!lxTtw<7s z93il4OW7#}JN6qb9dpHQUGK7KwRv#&tI2B@SHiypP-2aG(%<{LI-1;(B~Q{&ww4ZQ z0B#-L?EGZiS_lg84*B22#b~^nDjE4ycb<;HRdGc>mpBH<=qYTNk&tdxY4{+eO?_iz z<_girxEb_+g)G{0D&O~d-QdXyrYp~;y}kK(Nw<>@=XCfX4g3YZ#}-T%;3`!d7kA@~ z{x=Gnm#vYFoT)TOq&6Q;_tPm7kHA!sNVc0VmL;bs-ih#7*o$Ie>p znMhD|LY^mcml4kcWq{5*R*YM>lp#lO_uTcXuS@f#8vto581 zgo+b1jC+Xa`x3vb;{2S10@es>X&rkX)lqPd3W`2s;I?VvD$B;x(QtZtc|3+BJsX4o z6ikQ^dLwqW7_Rs(Y|tBn;EF+?*ex0-Vl7=eid-=@c%vZ<6+w7!3dYTEBRdhYsImrsG{ zY#g-IZOp5BDUj z^<9`}oe=jIWCB-&0Clt9Z6716huq)HZHi6j*?$xH$T-|haU!MixYw0M8nH_&hMjc1 zg88HYrR8rvUd1w31W6_+3B^@zAEmm5KoWOFJAD zT64$ZDBhVlSP6&dwm24X!~+6x5maPPP2$?BFR@%IqA1Acwn3}%m3@m&P_;Us^p9pL zIeGXAU*#KOMGO~tE1>pcSA8Jx)txGC0>~|`=I=+BYO|L$QfZ}?d8VO_XUttH5sc1r z=Txj42iZ&TIcl*fX9A#O?AZ-l#v72fVdU!yQRb1Z|4XDkc-vNS>b$;LXFvJK-nqSh zNG!JW{chpeJNeD9k9l}*4o6jvt|Q0PHP)$|TTgJu1E;w#o^LxJJYxY71Vfg%#-_TF z@`y!5EQf>ZRoEC>0oAy0DjGP)w|b+;9cMYO(h5B@(%6l^O2No)u#UgXS(CB#T)4?D zyKWZi5>uiDh2ofzV70$M;A?jip{v(K~pG=VYt+2NuM2SI zNh-zgh?@Qmf6tPTmk$Dxnpj2*k^=7UW&O+&RLQydr9m^pq?B3n0jREUNFC;^rf6H( z+V^vk(`Raw?*(k0d83oEr++qytTYV!@e{{72CR&N&yt!UE*Uv^BT70W z63vYbLl11S_21uAdox6&1i6}q1{cRuMk2I( zyC)zY?PSr;2ujQN0%?yp>7^@AjlfLUsB?+i|1YFyM6qU1fg$-ee-3eb{{ zmMB^9fPgN$ljbPW?6Vp@1N(2{$(=Y6p@AA>L@w=vLz4b1Z;Cc>#lJa087U-|3UrVP zs7;!}&UrF_FH&23X?8yA7 znNCr`$jR|pHhE?Aw=^H}ChYx8(DJ6mU5w!-8;l)80HQYUY>K~xge@`$&E1H8gDr8N z#s^*{f2%dzo>N^oX2}J71u|fa)gG)LR5hz>zouK(^-b+khS9u+Ou6Bl+}+%pgdc7i z*0v*I`?y0Hu05fuij?fXpxff^TIn(Iu}tN#!POL$mgLdA)%*qM?6;P6xDj%eurce5 zuM|9jLV)4Zi5e}F@V_swEj+W=&U}S_$&y-Ett4~@$|`RmU%a?scX0?Y8tg@pT3%Y3 znYMvSC|(b0NiNe&L~r)P>F>i4_pOXyDR@65kBYkq&_GjQce_SBWHQRdUgdm~WMp0L zKiuIgBaYGe3h2KWE?QrKGg{+At+{9xFQZZ!8445;*_jg0s(9oz@W} zjOuWaHyI9>)!G&aY;399BD%qrWX``{!rm}<*$49H@)!cE~E_x5j%{Xq>dm(_pbrFQDBc7mjH zL~IEPoR+9MkkUP!vFHHMtd4I(qBv(lD*9efk3f>#^{u`}*0ETpoj+d9zaXgyZZU+O zEHAB0Jo$yCjq%n{{o_QAJT_fek;n-e$$YjgE4vsRJ@P~1w+_z_+^7JZISRAF;4{3$0g+>u;5nWI#Z<(C$nJ<^Y zr!TKSlGyPKt<@etfUelLRUzw#E)j?N3UPkax+kHwdDIC>!FTPrOs zNt!xE2RUgP5wBGRr}S`kk8fy^#My6Nd{mZ}6w{)elBYla>P{>#{8;7TG&Yt2&CIet zUDTBx?J|C2BXWf54$M1rS~><`8}7b;D~8YtJ?XWNnKHo$@$q#p0PlG-m*mafxg4S9 z8!zoF37X0jmKzh5aCUN`9N!ofKc8N)m?l{TjQk34?d#C&C9FE;^a}{N&zhLJa=btH z@RY;9egZkgNkrCGuoUoWAJnkX5AqPYF@AP`;+ zTi(AG3@=Z6aUH&I+;zhAY5gv`Mh#!9?^wY@f~{&hK#f8ISds7z3cy_L)Z4gil2X7iTw7MUmeShkq}Td$tr<9Np)(9-X{^1?O|P~3 zL^}#M=5igs+zM)OYtI{Fsx3h@GuD3(Xg=f$Xr`r=CU%#$2s!JdoKy#0Ra1iMZxE>V ztN@INjubXzz#K+rNmQDEPx=}=22=n}>Mk%IGj*XIA9Ox&oh3ob_*R5kxYQV1Nj-(! zs52@(WF;d_b|wEs{Mw@U))+E~e8t?bs-IU)&q&Jj81nxJL0W5vlU@Q2WHGPyI{;W2 zK=Z=S)1s2N+LU;ivIzw z*p{V)olzH0hmk$8#M}aOn%ryy;d)XzJKZE7CW9mtgPuL7wF8px7aY?52g5M|*DAVs>I`jHx8}v}Mdu$G?Q! zYDyF==7C@PInR~^^uNxqC>&fK-R$2}HUD$3uiMRP?zK^K*{FZT|38|7Qy%PkZa3Nf z?WsrYl!iM^WS|SxhGBkQdmw_agSg=0lrjL_@3ocIPCLEUUADj~W}lN{UtcRk?p`2B zM5Az8G=L>hOF7B$&ov4Xuc-qry7jcQV6|sNbRfBc6oR!hpAG8pt?@$nAWc|;xS@m+ zhp_=M1*#V*Zz%^80P4d;Gr3VkKU-~!Kh^N+h!fd@n*J3y=tAa_dPslu2H~4G`B!CF zB_48wbZ`Ya$@TTNa0LLoCE+`4%a!F+@PnOp{J95X$=vz&*dR>IR=QbGPqXABsrLbm zC~*-`*H}{amV=G}4JeTEb#&AV5}v#<$Np*+0@R6dD;uj3~tWqV`nIVWajkU@DkmPA3x zAV7XIAdNM(PAV=)YxGEkphxT{_7t8$MXM&6EP4c|ADqi|LBaGx-SK0vJ1|(h?ODMxvRMj5uygONa>x39m)xNZ^})DRlTy+;8Yr z-*iM{^JGSA&@c~+c(Z=1HW{C?dL`!&EgDix#0XP73^ve5TWkBiXMgu;sN6&>I#0l` z@r&l}*O=Z_-RjjwC?q-N&^Ti$MdnD_n}lY@4TP8FMq7d$!7p6>6wH;J2w8m<&5SN7 zXIz!$jNet*)k5TpU%1OP)WvUucZ#_|pXAgB9E`Umd2Al+i#n;nA+$DF-h!q8=@*U;ozR!#(cI?Ub=iX_8aG<(1*|q18E==>EWphrZvu z9{cwV*_U`BBUh@Vp_Fab>k^EIM`i7nC|;uUvubk zg#4qxG|hoE*@Hvs>JU@(5)mRILPC4BITAfYTTNM0S&B~Hgxk^ik+?Sd&QD%xJtCsY z;g6%0Fm9eG2St$*_L=cdu(KfD;hBxMN@+o2%teYZwUT0mNs4Vp!#;8}g7UtduJ&Xn z558uV?xm82yH{-V&4U92swR1QqCfJd`NHDjP=RpR zQ@4_`@(sC((h+>wEX46?`!9W~kA zv&bCceO&KE-B|a=VU@7t*)BDSeb2-CA0-}>-Yd3oiGj= zKeE+ZoxpZi7*okLUn~AOWxpOF$SdPR6>VOaI(1uQ`I;iB*9oV(`z7Gv;`JHJuVgBI zupC8G1A{LW^LM?|N)zDsvRtQ)GD9SXZ?7L29?W~dgm`+jd(IU<)mwx>pqJ4R2d%q`0XB@m3g_uB-Ay6 zQ#_rwDpiT?tGt5)tXV`-XzcAR!N){n5NuId+<@seTFa$x6#vEtVnQ=^%AvQ&`f1}WuBl6!Y|ymS_XU(C zf0>F(ZkYJ4q$ng*(&Z|tuus|YGeGLoSQE{65~MfF_6F8-KV zuJ64Q#BY<`cXN*2qtEE2{~B_bunyN3{I_XsNX9PWOe$Ro2U)n1RJfvns)s6ki+~53 zR26YX>^}D}$de13V#j_f_U7~q05?_>@y!=VVVwWM0{qIK->MHf!uQG6eZxeB_p#Aw zNW?Ve9?ZBYXZ~y+Tjt9$-Tj;Vn_!6j&++XU!XRO_Py-3FR7D=7 zM&}dQo}`MJtNMn@HD za=EhmZ#%!l@r`72q->(6c4TQZw9O4(kMU%b(mUDY?YER^uEk4sq1hXaPGA8yBvf(@ zBsaMu25>a`p3!A`7e>2*4kC9bzo932whjx*)OK11jc@VH1{*Kp<7L&1(BWI_e-DPa zTj_p@7`^$eZg;SOfKZIcKs;-fe>KC6?|Ch0Mcgou(Q~fZ6g{a+PaY>ByfXpq3d&r@ z50DX62JB6Z4}wOpin4NlrD_!@j#k{zkT5hPhJ%Z1`Tw~A8k-B^WlfDf;imVc6nRuL4w> zJ<2Wa{%wbP+R)9y4pjNgg2G-!!H}|WOj#(7JOW!Tjun7y!QJZLcDVZSV43I%P@?Z% zIW%ugf7saOn=+21X==<{;=pe=mPboEUpuI%iYlV>#cW*-8!+Zu;^@_-1q6WcAq4>i zyuAUHm5!z5&>-?QWJRp;4!~LSSby%3mS~ZB>KgO-GmTW=Yy>`vR;1D84L91XsI>TJ|8YgX8n_HB~EV>gEv{s(tJN?olOffL$C zhXE@pidpq5+{fKAvDn9-to`(7Tq4)*p74Nfn&(ZrB>Qng@@kLF;nW|}n!d1uFqpA$ z9;ceG=S|M(g%Jly5MGG4V?{`f?p`!+@7~UGCE|e#w)&%Mst8|QLYlI~mSUGQbdmb zsans^Py?`gwSEP(4( zz0ht&bcfE(P#;>ome}`j?35;b&{x+b)BIy_)~8>8mloal5pURd*fL#Iq&ZAqG=-~? z5pR0pw;#E6_t0gXk&_cagdKyj`OFZAv1`~9eTlV(T_ z7CCpfut3_b2MT;352cGF7RrEk<9B*_UWfd&i346Z8mXDikUT+po}6M621oJ^g^XK@ zr#!1@jfmU1VZ8|pniQp2q-9Z7jVZ{({c@RXKDlRQAKr^yJia;QB;()6g+_G4qYn$I zZws5M<$Fbei6RNraAOtAK->4TalK}LmE6M zrC$soT9>yLXCPXMreTkztO}^F%afBw?!qbPV(_OUM|5k(nB0;#H%IDaOs`fP80=yk z&CUmj>hjcVhCoOdh>sCPGZZTb>73)=7FS!#_qh|S3`g)txS4S?0H50nDnbGhkLkM( zB28rV9>HZpX3hy&a{pd$nf7ci#;De8oMh( zryUJE<}&70s!We>Zn}Db3C7Mbv11(VM{!jv8@y2!5#+tFp!r76*=ETkZ;oGwC4R&r zqjD?l?xuRGy@@k>$#=NV<`UtX53b&lNgZd4){@RR?s36|62A#nhqAO+Gu|$`(v`P` z=DsBV@Zh&o#WS!1TG(V4yPWSf9I-Rr9_l;nm0QdN*|_h&boZ18Ch+qiVSRX`_;ILO z>+SEPwax!3V+xEb@LKutuDkefc$<_*OFC#qsM40|_co|(W5u1vnBq^E3qn4#x(j)mS)ogsQ9yzauX>$kl_lt(_sb!W(5ZVF}3Hy>$`KKQMnXmz$P^ z4_!(~)6oTp_}Ze|F`_Iiu1I)>fFoSEmnL?iBo2FZg!6^=*<20LyU|Bnp2gUH4y>u+c}3dljkP>k zGz<|(y-O~9l2-|`)m-NXiMT(;e@^|%CZ7RjE@2u|W==Q%%mU|nX}Qvo5!L?;_k$^D z?4avOz}|ncR#3`3Wt*8Ak7A9Pa!Nt-M_J~31LA2K->`L-^{h#T1fhu0GlUNB6#&&1 z^@vy(t`J)#TMoCflGrJlB_{%Fy1*|k=aa#$OTEva(dM%ES5{~-R@BJShV8Nchq<>5 zi?iq2M(^TIv7&|IUSP1n-QC^Y-L1Im;0#c_Kyiu}cc&D0_acK9`=S~Sit2=-vP@?ChJ(~Q^_M>#_s&Y}zS!Iq7MVrKSo~O( z%Q6JVVIP`o{K^f(mICn34n!XHyt#Gq&f%UvwPGzS3ir^@0zE8ys2#4fr^QER6WSV% zpNO!7V(onHbH#8avyLv7>GItO*goUb1%=J@eAEHaKoBT=}|ESdW_Y|TNwYwo*X10i%qVh%nTYQ${EncWt+Iz4iq7+9;idjK)BT~;j&tC{B=>d%mE=!sSbJq1k z0Xg|pQsgcbwWr2E8!PEaNkb{bre_s0qz3ov&2~>&7yXkI{Eq3{@3~ZY(`zFW3uG2m zBV8UWvZ!@<`q1!uw!`5x^L*@uZlB3WEihY^-zzl1r@8U}x`V#%mhzP&2Kc z&yx=snN$j=>nICt8$Vej0}cUE@d8f2Nkb6>Af+c}=IhSoeR8nyv>ed9+fn1voy)b` z&g?i&9OFz|#*L+A7V@waG~RE!dH(Ly$x2EB9vP==kbg(>-I>HDn)K-Y4|pwX1;iNg zm{HDHo#yY^Qxr7wBVO#M-juvDF_T4+yi43~_u1?8vQp9qYDx#x0*bSVHMy>qT88Cx zy1Cf_Hl#60Zw31+5e-WyfWuj(kW2789JPj>C=DJNR5gvYfk1N_^Fb_})2#_!vf!xX zdR%;JHlm!{8{;T1l|1v$<*jMo42VWtX5?jSXO?Hi3z9-*=v9W-MGULXV84zJC(`h8 z^heyjm9UxV4~y`wqV8U7BtRJ_O7mMOhlx=&Biu?|nqBM>#H6e5kL$%%MFYRZKwMgy z7DbEAi$zNS*-u1qTZU6@t zZ@IO40?DRB-`iJy>Ep3yn7Q7%97`P>xDYHFiB=1_{U!31a?r(wh71YQv(9Qm#6Ft6 zwvY{%uosUwUSwZDlXgUsvNEUb6HpP0q;Lp_ikm+DP9_Dpd3K~H!O`CYsri@XO)0{l zzBlaA4~Fg`S{HN0eY=fJnL_GeyLL_X{>a17)1U*wTiroORk_dC4Rn#o!0K(edrKYu z%N|{fubWhJVRPojPG;OA0FSR!&;;>&r>(Xn!PfBW-BbaN#(xbA*7feqkmzZ!8#i* zebW!j3Kwt^8Y;x!FuCVMZ8=(j94$R_1^K(cNA4$3q?sMJC)H?opQ*Xl5%iMy-jFwW z$7tWjpGZ3kOF6*TbpA$bDkfv=Yl`)n=U{s}F|tyXHDyuKg*m-)sVbbY9GrE0Pb#=g+mu6|#>g+vnsxG~MgaE56eWFqr z-(7tx^kz!8M*Fj;>go0V%`Xm!*h;_u#5^BvQ=9(M^A?B(rr;QUNt?QAr<@rkDGEO% zic#jeGet2)^VblkF#JskxV%&$@JTdvOR~P5{=tP*!iEIK@?{m)^Fn;YY}|6+rSMzU z{%*=cOsn6Qo3T5+J}085fT(mcmqcJzROZ1vmhNP_W#9$i?dgsrG=pUNX^>_&M=Cgd z?;Ul}Xe=*T^$lR-n#7n7GtsYSC2rPL9H%*Svq*QsYy4RMLa@jZMEDo$BqLlvqCUjODf$6-(%B@AUd`NOwmaDX1o7HML%G#NQCsBx8@j!CQBQM854ZHx> za11)z9r;hn=}z7L7IA-ZG0lDVD^pimR0;O(>~7No*iu22Y-gt7xXwMiK#QmPvnvuW zUJwQgl^wOAwdg~C?jwSWrR8M9szm;SeeD^OI{hPAdbbFUCTWICNScyQ=z@nj0keaA_%ss;5diGDD+S1pg)QPG_E&hCRrWUNN-4 zRf3lP=j5{0OSfVGF8&nrkWA($IGTphM{^m;bPt#+aEBfAaFe7|H1yj!&UYSbKK_3BRc z*hdeBRwcH^G2e{yGp?n6$nCM~H4NfH*5DpHoKXR_M}m@2J7pR^;hUiJ}=b9|MnRJ<|IEsSz7#q3GBUoM*c zttGUkCw^P3iXY;z@oP=8lChxXdF3yEvvKm3nUQpBCxWw@qQo3T$vh8%|q2cxYbDb z=lHDmC@hqyZ7CucM5Pl^D+ztg$V(VHkT8oU7RN}Gqd~r?(l^MNV{?dksZnr(C7K;$ z%SwJ3WR4t4g@qLM7HCGX#Y#Rhd6)%h*{(QgIm+4CxvdVM`9;A)j|^gf=!|H~ zkZDK;x`x44X4G}@#gSglG&-RowxOHjrYmRX3fG>oKHBjj+k)SMG=;$id+ew-+6^cK zI^g)#uKajBL;@u1Fa*A(b}yCyU$k(_v0-;8R|BuG*wots;Ur-NzJk+{Rqr*0RsWUu zt6bc%xiuZ#)sDGc8K(6qNh!f=RFuvU#YykqNA91Lm9=KQtiLn&BRN9J4^G1diFXNw zSWV062~$Fk2Iwx!<0?j38wT?+yj1X~@A*}$Jb#Qce20~?L2h=TDo#4AznChRU04`9 zLDciH(L>Zbq1eTzVU3*%Wsi_SH43J3xjdwD zWRAAzn-qAOdeJmo-WaWlZGx~6^YLtNF-rY?LK^$-a@O#B8a|nkxY3}+YP~ZTy=T_P zxlh7&n2kEmjI3b~QEOYKJ;(Wi5wO}lB^q=((c2QzCXV>0| zwPtcCW|HEwO*IjL2_Tn#^JF7}#N@<6HfLwNApZx}60L;Y!)gbT8`6@slfD)^zQ_>@ z#yP0}c3Lb&U|qhQi*IOZr^T@Gu;8SOTh#q2Mkdnu0}LHW*$Gh_jfpxgRkWGA|91uZ zfD@Oz6|24FnvMQ@ef;H5n4ov$1V}}YAa`_yg*rm&-+Fdm%1=IYvsHu_dlMEh;ao&M zuqw@U<>_*|@1dLm3MP@s6+CV=*Sed3q)FthYPsdIvgG{GHb;E}w^jNXa%+>z<^SG~-G zz?f!V_2R|k)kbF&rOsy$E*-0;* zONeRx7MF;V11jS0He^Tdj3PMVUA#E#~X;@0fLZ1cU++=y_vj z_SCXC{Q!!@W$a(R0BG-jw_m`8kgPs&ogpa3(0`aG5B<7rvp%sNqv-3$XJhYJYP;YF z21#Pg10_W?n1^HfU9hoJM&d+NfQp(LR63FiK8YeCY19c2Ki|uH|Lb$@1goxIM3MnC zH~PbrkQ&He8fjoK z&8bjQ*!`(MrRk1;UE1-19K24lvp7%&-2SChbGqAm{+b#KeICT^){FfD(ZXyH zJm}&Pni@Q?jG_oS1zrs$Segeib@^En@(tl9d7IPo`9YvyJTYAqci-tqYKJpfkdH65 zP)mQ*_N3XCu`)H4Uk?mG5c#3MifpXegc-x-577~9Br}ATXVC!;aCGMNXCYf zq=rIo`^Wrv5&Zo%%K!K{{>cIkAq$x7#Vmb#Ye*U?vDiU-;PPm5X2A=3R2{*2jnabvlWo)Ms(aWHYXFA(NYh@ z2RFQ+m4nlZnqy!88luoN$X(4#mpDDku(T;lE=#~#Yq<%3h_ zN$IEC{>eIF_U3~CENS7@>7g$}5xXw1)`D^<-0y_o7_-4{j1&Oq`}+J4)UbGR0E~Uc zez_XFlRhXqgyW3fy*7V({${_fjhaku+b#fyd1{`)9@D?7ZjW+6K!(1;RlA`&>hQ{& zyW_q##5hJAJ=%ekQ}I?Pfc%x`x9G7QhTE6Y2k6uO8UwK8_gaUnu~7O9BC4Pld1Wnf^L?G3wd7A{6oizUe2;yz*_m=}d> zOQEUcewsvdXO5#^&5v9I5YN^YGDy>Io56|6by`u;iPzC$j%Zz2W3I9TFULu^M|QH>{@o;^ zZ<)tM%<{t{LnOD#dK$4{P3aUBlUf^Y7XAn9HTVoj?iekWm6lIFHCK2?%Ll((Znu$I zZQ&w|JFaYI>Rrnc?1l@^t9^9rEdn*%tSlEe)HwUqfXq&no#^yFcPM|LlxH$lj>rpl zXAf@foj8nr5JV*%GqwJr?WkYxY4x^!V`~dYsHe`L;!l!x-#elY;>0jG5D0y|cXMEb z*V^cQSfYW3l`rd|8IlrK-X{=FU-Uy zi|6_|ag0w^Mxi{4&OKGeG}rT}4oZs~hx44%*ttwMZD73qNoMjY%#^V^kgVm8E;Ny= zkYuLL?m|D}9eSeOy9XBXflKm-s9V8_tj5&IImv+V=o*}w0 z-Db$@F>Fgp=h(eFVm;g;IjjyQ?IcyGVD~SFM;97H=zJ*{ zA)i@Ym5owI6Ta&)Ac3KAbpIwvj|xE)>48>XpmapcS3gzx5ydWi=F8&U*+y+6xgHKhv5*jy-g%wwIA9Ew^iv} zj_jAU2P6h?>SR8{w`*VBQ-+OIe<*PjV6FqT9_-7>07k8Q0u48uS?Kyc0bNnXnbuV# ztb9-|Y2x#mY;7x7qTo-<`=m+ygP%DkQmHIWQr&$qs`s*}(Z?SK@&0V~)##}9Qa5iq zM%M?CkdIKTNpH1&^`u;KZlC(o=qfQHE+aap4w>y&_87aT7%*Lq+#XRR30?bZP3pF;j5`K(?1zkl>`-l8iols$mQ7-aGk!Z` z1>2sHWJt7{bSQJj<6tG-KvC<5m2MYAIMen4+&&k+j@oLik6bg5>H**U>TY4%BT?9$dM((*8 zEKldo6?t`i-1Ptt_glYcc0KYRN^4u1Fy-OJ?P@YHw{9;bRm2D zA!r9oU=jUJP?rjOx9a`bIA7PN84P3?(!)g7pJXDkT!1%=+`DFaX56iPsP_g5@&;Z) z?QQ=4XKT~rzaHTf^KZry?ORqOp~m`?ivSvdRbPUZkmS^DKgCFYfJpSNiT;ujzV z+xfi9A}8ia#`p2|NJK)LjY0NA&8yDC4oK^5uc+lX@zmtdOaByK8K|hSzT}Dy#{~8+R=4=tIW2p?FGkkl@-_HFw_rTEKgADCGKk zHqzTjKaj_*RLOD|KBMk<{h;;rr4M)Laj|iWRU%`xx7M8aEGKWbT*HGZ4XE?&5KB|0 zgVqwrtuK&Ph&=mc@s*(Kw-`Q7gZZufO03&&&Qx0Mx-XtF+duAEpF0@o4DYN0AIG4# zu^mWO>(N`EbW8CDOaY&wK~$BM?Mx zw`JS?P_X|NfxE$xo>Fp;q&P|7fa_y*TTa*0HHM+$6!nBdX;)R=pdy)t(c_yvN7`i6 zeHD*umtQaWf|HEVWzWj+k1K2X5T^sI$}g!CgcF5QucPPRbnDS_?S+oXx#JUyGr@6u zuN1d}R}?{4KUGg}8|)y40LxW*mw8que9VL3gY&XxOTdE)DFfr33ElV9Q#zcUglGPz z$PHHDOXSl^hbh%MtyzOafv5d;;@x}W-b~Z$NlN=QTK022O14Ur8I>6lZ3qTq5xL8*l>JowTt`XiTD>zbFJVnh+Ife9G^j9V8wjlRV?-p5Z!x2^ z&Ys*5t1ouD^uVusR!W2zb*)Z>`QuMKax2ruZ=SU3P|<}GXm$1#B_a@H6)QIcAwDK# z*ILZ~SEFfjYs-$luiPD-#hUma3*mF!G&{D{xBui=jh zSoJX1#j}N8kG?NsEl%7yK03KSsd5M`m(VzKAF|e?8fMGcqdd8C`#xU1bf1elj*ybNRVT?{dFNIOz3q>5C-ShcC`*q zT$VsiZcsq>>$3q%Q6!J`l17h-1(A@Ba?DE7K=0Ax{YieHh}wfSHoCy+YFoxNYUjXD zea5nB9)-STp{p?stj?f=TwL)zmk~T_{`=AVi=SyAQN$I`dA8#pIfQv6v8Fj<4=;$` z7oC2Dd~F0(-(t?B+^e4_A6g7N_w$6JBl82J{CpFXcQ2IdzQ>p>toJMuAO|-j<==45 z#Z3|_PNs!trkGk+uJ=0%#v;{R!`++2B|q_J$rnu~V30*Vm_D@Y0jK9iA7)VLFKwyp z6zpv;pg8T>t0a+gd5VBq+vliqza|@hbZu8+SkR9oK@=bvU$@_`6dQbp9|u_fSJ>0p^TZba8pu(9oShknPXtueW+oH(wcT2ss8Mwlg3b4>5qhuziI42XvH+~@9Q;J(* zP_9|*0|j<>tQ8nraR#xScE1+cVZoTBrFJlFpV1_M>__4V@+7}}&(=%~4k_2FzX(zN z3NUt8(@l>AEK2J6Aqt97;2U_s;z0E2un(w~Hj9ma!l9_@)9EG&i}$1%_O3GG1A^?| zRWOn3;n2NT31glX4Nzm*Z=BKpc0nXbo%06SPZrCE4Uk*q0C(my-et;lM3Z$NC%U?i zlou7Bl7g*oDV*XmXr;=}lqReNJMqjg&``adLO%eUK-gxFr62X2URajTSPcP<4Lx-}e6(!dr`Ra}*|N24=l5*-@NgH>M#vjB0cH4QF zkEHpLJ)0DKjD(!qeFPa2ftDmRgcJoI+IZ4U;B6M;P!3HZL>Y9#d0No1C}KbzPxgi5 z3?EDqSUz{qMLP#ERv8!;y3x2!ctA$H2y3bFRSE2h+rMZ1`V<@@li@lyjxC=8iuW_PfqnD>)Q18#%lRhtiz)7OG9e3!`U`;P_$1aoH>W=zqnC$mEwS zeXqU}+yR}W`--I6HdR8+%;5PI@v+az_0U@>L9T2VMRDC(l;tg|t_10OrOhS5zdom7 zHPo2*Tw}io-EP7uPqhhNJm0-Cop*wt!C+>qu@To4FcpxhcN{^~<%cIw_ zWw}TC-`7fW37woya+@?As^;Foi^P10`ox4MDtf!Sd-~Y<{qc9lR1gwY%3oL^ACMmc z1O-=wmrQuM>_7A9nwr`ZRm?ljK5mSa;=RHBoJ&G~W;O9!K^>9*XVsdd9_z$zKb1^F zUggF^;CfRlsh-O`{eJzut*(s|L*F)Go49>NqWT<7Q#Df`j)O7eH-@ZSQQU;p7ac$N zuG>zdLVEMyE7fL4)Z9I48rs2Vyiog&{k4wsY!5gzMGc0e8I^gDn*J%5fom8XIUU}4 zrxIRc)uoPvN#2}?Xk^ohIJu1%((1L&n9t~%pF=oL=GDq^&=kf^)ZLxe+Bf>B&(a7Y zWOR0Hxjea&z@#SRWS<|G@2;xl_-Yjf>_(W)>EVfWW_98bdl)(R_~e3uglK1H`}*Be zQH)Zz^~t=1T5|knw#?Dd(ami&0oe@B^`Zc|(Tg9}45g+vj=94ZqI#7fPqjhmd-}=v zYzm`|eaS}zkN9pK(~)g<4Bu zG2y}a5X7DPr>jQMcAWr@KTA$gX+;RLN^D-8$u0h9OZB&u-6uLD9&O`6Vq+<>Pp14m z0U-D$cXbs5>8nIao%J7veF%(0M(5eaZ18L7dq0;X0)GgN-2IlbPTX$eoN&*ifszUN z2ig(3LB^i!rufxtBJp*N@OGIe_P_}iI+>*-V}WWv-X;fITiaTu z*oByi>gstYa|(Cfd^X6Y$|vFPgl8$>Zf)rHY{iGUSe2e+h#WiGnRmbxD}QQef_q~e zzw-hf&dT1#m|KY-Z>RT8Uo0(rlSwmkyZZH9gvZ zB@LCb9t=r+tXV2+g2)_9;G7H}@6Hd_{m%E~9X2|1!>bv+-DyU$ephiqBai1jqcK|n z)r5X;BD@@|>jBmF)?GWjkMKv#0Z;sSc?y@E?THtvqR=~Ugr3a4ORKmg%;trkRLO_Lc3igML+2|1ZIzqKuN1JDwF zJgH|t?%3qvD_sin54gQUXyCVc9VU77QOpxCb35G%OR4nf2qJp(?hmQby@XxcAx-vM zFlXyd+>EgQZtcI>bw0f9 zcvnaodkYXc2>t9(-^SmalNSD;AOE570!vCsA(Gv9BLdZurNd0SP37o%vnuXdo@yBN zMtDD3Rm|VILfUe8)Y46o$M)`hI3vD4-In^kIBwtuj1fYsS6wu_>g43qU(oFN-^*tu zpdvuKveocongm;lX(VY?QvT?vguYA_Vo5A-sjdpK(;J=X@S*QkMiIL#wfMi8`2HnQ z$U+PZlNdR$jPO5b$NtSf1a5$z?b#ArPd>-h1cOtxn29?)eE~b{gdb zb7*qBrbqh^FgGE(bdxxa!Ri%ZLfb>Tgl*Ccd4-#{iWb|b2$z1$xU@5WLOa=4NA2^_ zyf_ndIXK6i?2Z&S;OfU)JR%|ZXGUxfaD>Z;s8{zx&~ThS$WNO&%d_RvfS8qkpd_*g zcOUd9|68m{_GRjP1<)kkWq2VRKMn0{2?dleY3SzdM@SJw=f@HWb;bc*s@)`Q#gT07%t_j2(TNA~lV&GyAr+@4MKl4}mk^;;HxXJat92$|b{>;1Vu zB7UQmiJIx{;m)e(kN`>6sHv|#(frw_*18L4rEMrdN9&PkM4f;aWFd%s>OAm-z40Q> zoy_Rj$;i|QFyt1~N+ma@61XSy`i88v0!Q!iPt=I^pjWNc%WE|9{P^um-@rXLZnxRr zL@0GKo-cJ%_k-%p=IX3jWw z4<1{e=bbLZr?9nSocrE6`W`6gPa8&8{jmjqbPgACxioPjlku4j;-)x^-u*C{4@VA+FVDN&@S%- zHg;CL1Y^od;t#@zK5mBQ+Z7MHDEcNup8pn9OXE9aYW6AU-^R5uBJR4JIkCCcAt^!= znh5+L1m!Y&!Qqq3x$iFfSTq__clXQb>L$o9EMjV1IyVCZL=$>p{rk%$UXE8?=Sf2&B9De+@D}m}C)txgwZ6?I zQ(vjS1TG!x#k7#y>phC#lO9sDB&1%?>pk9hs>e#H`o7 zkG5zDZf;Kb{UJ8= zP?u-9+H8ISpQAUqUg8<>3k*U|jnVDq7cQ920KY-7OoZ_rk6Jaw^H#1E6m<_g^^D6c z-hGmz9W=CZXy`l1Sne4EvYt)zY<=Hg3--~3UlZ=lk~}(rCaDI`a{A8CkFEY{ zeCZ*0b7V!oLV$CmhW0k;>#8{CZO-h(72d%d4eXGg_5oxW4)#N^+8JLh4!RN=!0y)Z z-v2QHyfhtGqmBbYLUGRbk1auF;A?=C)%MRS2jlX59GJPUC(FX&<-ox+Ym=ktkGE4S z$t)t>(24WJG?V!9CV3u-N-Fc=xJyM}_DYt{h2{TkjR-pq>Lq>!Mnq*hbz^>+@~?=87}Z%SlaO?CaBByUwt!Dk=y#E0eJPs^otQ%HIU#KjL&kq%osI#9Vzs z>cT>UF5d>-Z8+`Yki9CrY&!iipNVz5-<5WJ@jd{rYm zD9V3f@qe5tqCzD?s`p)?Zs9xMULV4k_VdOU^56VxkC>&feDvQudz&5Roc5MO12JUJ zyR1AD4cKn5O#MtvUv0bXoJBpOdpZq${rry3&MeMBn00*bjAzs5!W^I;#+2#H@wuvp z3TIYsb26%Z$&t`^RFKx->XQ$DfeZ?cP;fRevV9BoKsg7CW%|O8fIdDOFE6*d0s{O~ zZ(^sV)sSf63*TC(N`)*WzmKXvx;o*p6mcP;g|YuIT+dn9d*ta*7Bg5Hr->JyFV{k8Tq9E7tFc;S|6HPmvR;B>jS zLh)qO0 z{PbajwwA_{cPpC;RnJ#mZt|s(v(7raSFCqv;`h^JPDSIJI-^%V+H4TLbuV(V9|n9% zQK#2^k-58Wpv0w5QBE5#Q-_;YEvvFg%U(%{kiFIGUrJZE!r8Rv?P)>xL>w^8JV#z9+gJaKCa(Krnr#sBgrH7G}3MCNP~|wLBl# z$C)C0hQ8(g>HmM3wgW~?yJq{H-s(Gf92z=3hcm#tT_fqeo6 zUmwzT2=DRO_s?&jKAyMDo;We_bREyt2H8LbiIhQG_J^d+y|3Lk(c6HbzjO7ZuVkYhuF;@G3xz2@XrcRZUMajm>#={Kfcwvbru_`|2^GV#xs_K zO6Y<149)s)#;zW=<&pY5hjWXxuHZTJJpz<$Vwkv>7o&^NXSI$wcmv!0Y|JMB{`~Jx zo(vN_(IM=eNH+i)0JkVjPQ+PU?kI-}1MTRYI8U%Q<}#uHcru=N=b_*|PD7 zHwLYipBkGvu^RChi3_PDJBRP``9}vbLi8%OLr$DFnep9?>8+8hj-H>my-ly5JM0v2 zxI0h8NVEzq`_TQ{My3WmCcJSQo2!a}u9-nW{K(5j2uwu>k8fT} z#^bNH`0R>A=b2GDlz&#B9v`|z&o?(OEVJxQ@=_Tin+1VHo*fz1r)-LKrhHw`Sw_^h zbiyE=%uS&{Z|;rXhCCi*-;*Pv;-RA@!>nE4>Rjrq)62{hTK1+LmSEGu-E8rb-d}?= zB^0A%67azsQ?nhJHlEmxa^tPkvg1i!_JtmX{BJ&l3SSVh|CwUksWgJMv1^d{j=dk6 z{4p7&Orw=K*5%R0ba@sLw%W7C4IDUqdPhAg%hGB;w|ROhh+q_B^e6Xlm&rj38ws;||PGu#{#SKWaLZOk2e$FVl$m8KG7lAfn6fB7Z?vWHy+$0ieNooa`h z#<(`M!09ZiVs>Wl1NB_uwon`D*<=GvNVzv@e`<<|c@T(QGCAq+>}dv@xAoI=co#Ar z+L!kg0Tq30?5il#h-kT%r^NbboZ63c=Q$QsHQG}gw4Sh;eZlLUIx+v@7OUk6QyRpl zU7wRt2zq>4+o@b?Y$FN9zeuaAFNtrO=UUcX(doKTt^RI{I%$l#Q^x^aN*b9`O`K6I z3VS6ub{WN;%EgtL&~YGD7Y|K4I2ggif`T;?p>7||7JyQaq$ zUbnk0FfnhH-F-CKxJI|I6d)CzMA^s4C4+}KW3FHtHUWh}EfJsoKKiR2|3@XKExW`C z)m>m&5=|EK_MMKlAxkNU62$V^C&H6{*9|u{xCHlrC30ZIWMAN${}$mp5<<${d# z!;ULBr?kc^E@t_gx}?Jsu1qn+I>9ZMX^BERJ^Igd{bMZ>u%0)Y`p|gCFS) zU3tz#2FBdz_W`&HIAg5PTxFc$AjMB6CU1qa-q7{a{o2Dj9AdUrey1FviolB_HRP?6 zz$aU3Y|IE7N|&xe#hG4|DgMFxz4o6i1#x+%xh;i^1N3=^;$A{isYZcTv@a;3%xgFPCy^UlFNc0Qdg*2d-|tdv*Lq6v zXo_iC?xYg258Hy@S28SmyKs92?M1Hs?jx!`Wio3zASXRUcippUQr5s+aIA0HnRmht z7)ooD}u_~ z-rJL)_h*5TPCS=^Vg(xs3H+U@JEYU^{;qj<>f(+ffl&f0)5^o67aN z++%rpyBA+;J2vNk_`CAUtsb~pR^pvmLu%oTw?izbb$|)BJ!m)xj$gmAHMW$Q`+`^| zW98Q?Z;)>W&5y+ikMl>5k4JQ3Ix!9l0e@Bf+MEn}VEpCY@$+kYs*?YG#+SO_JsncN z3sMF#zQsBQ(X;(#NI7OVx8oxpDT_8-rEcplv#yB*&NKoV{ogNBNe{ocThE+c5ee;& z`B=8s$N~|GwZfGD!`PSwAN$V3iScXBPp11~ve@|I2feY&;KcfqaQUX-8ZPSd6J^kZGtM!A-WRp^m%JxdU)NUnZI*?5~YXAGE$!? zAiE8Haw(Cz?V}43ow2K?uAeAi{r!QZ5O-=e8z05~dsf%W568*q~TmddiVd}Q~AHfQ6g+fK?4O3KO)aX;UuTw?e z{O6*rNIx(F%_B;WD^!Au60Z6O)l;xDgVku=n2m!&Nkv5rdr7M3KPb&VUc|qh2^;VE>b?fg z_calH?8lzYr6}E;3yl^0&2-eeloo=1=%<#a)<|+CXCl9#H%lJnys)LzWq9-2W!Y=79amzP`nO5#pa> z{=b*yzqRe}c7)*k|GNACUu*Ke*8EZGd~I`o*KF@nq|t61y0Y?bS%8`zHr{+=-;ZKG zYVQLlQ{_Gi4vCV>2WY$%yX+980DRzNn4GOo$R$r`8uJZY2f~z$B1wr8*MuBE-ai;ga;Bi^dpJs0Bsb_( zUPy`t2P~h(wKWRmX=AKS+y16S(EyxCsVJ!Hr4r=pS@e%isMdUvb@}z9n8xoYR|djg z!pcOJcEe~>jdJDcE;xCwpVwAO#1FQU?YE*O$niQnwC$_Gtw~#5eG4ncRb`ir;T@SpfsEH&e5F3t@Amf zBIM#E6H$EHBEUAZ2(s91H%E#y=jcrhAQC;s!vSIBLHrET4fE1MKhNsMhmlX zW;{BlFFiv`;vho;OJjG!NHs!IArFPKpJ_fS9my#2H<(GtT+l<1t@zS43OPsvL67dJ z7}r6JzHLmw3*q>&CofT9W=#H%-JMt_e0he-u{w|F{vR+tjN%p?Q9UdPARSZQ((=lh z{{8qpDq0@i>{3SyMh%l392EKMThPtIk7fPvfy&}oCV2&u)S6<7Pn)MjYy7A?Me;0V zbm<%sXGMig$#`=r{0N)xq9;BoGgW?GKHsyN-Nj)ax9sCshPG=RSgY25zw6aAz402nnpW;8NkLR0XV;~RMXBgcQy}cEdfIR zFYceUG=hb39uQK7p_nw=af{*-@#sgO$Mo07BPfPo#*WKzz=ggP&K5CLsvSoy4XxT# zPCg!^*6BcrK>J4rcwolxx8y#Z>qpm-{(ZvtIVGPtYf4P*BW9v`T*0vJ$@foily3BB zvx7|z_A3#lF?F5Kr|TPIuFG5T#cL#LW@b#(5OBO~E7k0d8IvW6p&ntW)KI1p5ZnNW zg$lshp82-`{o0wN2N&yQi|mc!G7xr)^n*hIUg7#f=Z{h7WLt;#Fg}xlt@j+1AXMB& zXc%4S`C>ULll*nV!BH8Z9@E7jLp=xQ;7`+thF~@D{&0^;`YiP%p)M$!pxDVds_(nl zE*KlSBrkQsqD>6a!6*t2O(%ni_)>$HA0DyQ!{lqm;%-7sK#k;h>Jcda=j+H0B zd+Q)6Z?ij@+>{5-s)zyyG0*hi#cOkaE$MQ_lyJvhq~-je$E{3Q%QL7D4;fCpW8+ts((wx`(N3|Sw6I}Q zWF;EpB2m4h;wcN8H(nMDDR?Wm(mm2Iz1ySwaljP0#H}Bj&~+aWI~F*~NV!OIfUZ@) z)msTHO;vE-9g))cYB(ifM|_#Z-$#xor_35E94${Ff#0(FV8~e% z*(3luy(cQ>4L;CoqQ;p?TFFrZ^r_uf2+@*HnX+#_hRer@^G3$@Ha?3Ni7=G_^)iEH z=be@$p%!I_A@Re!-V33B+MV{B6FfXnIIna`1CtEze8{IM*!gFB@^D8b085{~XA&hiu)xS= z2H@)?iLD{_H#JL^C6oBQnbX*N4I2{RuLjHXovf9hPmv?xn(HqmNHJ)LAx4^F1WTH_ zuNNDDe!WaRz@~M5gfSwEgdJ%96e1?N6a;w|r$8lTbYljfkIn{i{W4ZlX0P1erp4~k zByY+XL5Hol~979z_cQT>xbBq$-} zSXC=Zcc{^Y5Dj*p(X+XG;}xRV63uex)uM7`jOL_3%aC|b(M&+D!#;M592%p!f?0L& zMwc;BYP4R{)Et>zHgixaA4PY=8+cCzUr}DlRLnS+>yJ>UnSmW-RHSEWV*_$l7%kcM zcQh(G%-Mc)TQA(**)G=$(w4OnM&Lj0z{S`m(?Kdhy3!Dd_` z#+MZ$l)|i^W5eK@G|3ZX<3@O?`}*QpvBJkSgh4^*>LvPTj(qZ+K#s zmvc`_vDWxtzeoQ!up>t)S#Qwq-y^26fb@hv|R8-TTjGsW!)FQL`gLaRPQBSj7 zO3#Qeca0hvRD%d(7PgFWrHrLY2N)z#}$|j2) z-)_-M;mECH^`uO2pxHM6Y&%V!JgVr>a}NU_4Ars>EptV_Trn^U@)IHtu*S`hlhx(6 zF8^H;`Im2KV(N0(i#IhRwXmrUK6?SfK`QQuk3!YZ4JtrHn6C(uy-*~|h*oaKzNY~` zKm@v-10gbgEJoIpCP@o0Cc});d%!YN$QSAg+oz0yAP!hj;KJ*rE(oP!ndOl#4& zX$NJg@k!0t3*)-_p^`x$hhfiR%)+Ck@>QweBn1Q6!-jU%ZCGUZV)N8yspr7`+Sk=*LDYj$r^1Q?AGo6(HfT5ByZD9^>@eNO^ z^b>!Acp1j}gz*>}3N&Dz5%9Gk( z+g>7)q%|{8(=L`r>cl9_s?_}hs|T2nBN;10A;n86Tn?nw6r&3lBA1@dA1;ie9|G+H zYg#f2DA9U3Fc17hIHXuJ^LH%(D>BpsJ*G5KBgIKlCWKmuMQ0QQp_}&g^h^x#llxNi z3~>MnI5%qyzj|@2gs?ta2w#IEx@^mpE*Zsgn1~Jl+LY$sS%H6QQRqV00ZuOLLZw{n{WFkBj;%I*(gEYA&1{W0*M3ldGT`E<L}zWGf7(v5Z6vr70;E%k%@Pwy8+^3 z;nrzO5Ro$6rV{a5IL3=v<|?|*q6e$vfILbh-|NH9($j(C z6E!Kc-)WIpPbbi8la6qy9ro^$Q5cI-aQH@NYQ+K3pNnJq5aws+*CxX3(qjACaOFSK zu(9oG4;Qcn;4Rh~dwoVlHYGlb8^fk<-F$eVEuX$eob`nr zNGqSHI0n|KkkQvXzm#-{-SD*Q zAf(@I8sEp}PB*+#E{o{QWQmO*Qi9LG-VLo0lv|Peg9Euv4`y;xJ^T99Ru8V$7?eDb zCuW64b8KRw!1a~-GZLX5ITA#6VDqZvf<64@{x+BR^#Gbc7qPS(rH2cPg_GU#zB6Q# zpGd@dN1Iw$MIN7k97ndPgWab&#(b8xejx=WCG*QvMz^7YBI9o-s+@WSScig6F*C04 z)s@40+O%m4b>?2^0`e+k0{s2&eWJSn&bitXwNJyt3U6 zG{r?K)SVc<-oPf`s)O$-ybPc;X^)Nt&q5HM>?NU5Vy{#-R&==}f-sJLgN6I_NE&(7b z*Ks`Zwx-z!c5Fu!+aW>Zo0E%e5u=q|oq=ai-)AJ2_#dtea+IGVGuJ9kN_xQYf9C0- zO=ZaH#k`m)^wJZfiDM4h*D-?i*7%m{xKHZRHlbpJg`JLHPw?digAazW=47K0w?czL zpZhY-5M;j6C%if}HN+D0ckf=3FF(t_3RyU(q#8wOnWTBIh+2Cmsa5^r=JvZzP(1p= z!apl|F%0>)X=VVDR!z=y(LJr*TmtFbs26dQ28-s1Y}rpr)RN8JKFqaKj50pIqBVR1 z|3n>!XSkCl7}FOrG*6f&nW`qwzpAm4%MpE?(!pp=JxD?%V4(r^EOzMoVQ== zcfb^=;0t3T-D2JEj@a+^Sx{{cHENfejqJnTZrZR9GX}tg_Jnl&~sNH9~Px zu5fCMBIDd|IiJQPf6n3kuHXtoyO}fWzu}IECFPTV?$Ru{nAWFM!t1IWE}CoU!L(ZW z?K`^)2p=IxJ;m`ige|w+JSu)M?|kLq`N~JfX&;iQd9axTs89063WX_g{;*f~lCc_7X5 z;*m8prTz-soEVdmt@Uw1SW)JiswuJ182f9&G?Z`+LuScQTQyFJj;}ZMZjXhBM_mnc z*gYiA3i|2P{KIe;{g!xXB98Xp;K%6R^n|sQmU=Kg{=w!J(esxxtp+zbqK@%c{?E7s zx&;h}#s$sEd)?Aa)@r^})UlbwMlQO@J#~10X#tZuq-A5)lptIOZXK1GZzWnYBW0MR z3FfR`MhQ+izUPxBek!4RSKdgsh^+NhnTc|ArJ354N*!5035BsFlei~IIXGpf>@o%s z)6Hm+L8A;a%^xI^?sPOO8j~yovPogq>E>{MF_NC$wl|sQ5aq-yIx-L)#&%9OFoz7U z0g>@?O0>3yN{$%O#)_KfhOQQAT9!D(F&SwEht+w;)>LkWzPq4+syjllI=0hO+`*Nm zz`I6Ldd{idL;Yv>VEEvL7#spJbPN5J-x5}P%wDe8`8!H~7Q|@kxqpm^!}K;-pB9X> z+Q;<~5XjZ$8GeoYCcG-*7yFg}_Ex^ez>os{jei_8wqVWhXq{ZR)ki>tQ`FR}w}1F4 z^;)H=@z1s*(#VLw{T*R$`@R7ns@@Bt%mRuaRlBIA4t8ayuDwBav?PaD!=R`**t|Io zQBwu&PQ=DM!z(zNRc%cy{hOri2F`n8y4r?v5Eya$wwo-?F;4SJkC(JKgxgt0(N5vN^b&{=i=X=>{2dj}!dubE?XaTTw-C)Jl>>u5E+6h#B?^K`(rw*JBnS#&jDCdqo_eq?bm zG3>I3q5N#e8KOV;9?OpRTsAr$nh8ycJWSZXj>M&$UtQefa&<=Y@6ng7=pC&}O=hR( zKMtdus503#?%#Cxy{_n`T}M=lV>VueroIUXCMwsWhJ)`k4uo{0NQe< zc-1)%-(|M|T2oG|QxtZUj=rO3h;a1E?;lwE`*4mPZg#Z6xXmr@*Wy(=+J-qxeb;;S z^z4r9Cq1ZfO*v~C2fJTPTaski5c=)$kH#;nj`?VekU)wFbq*?A`)pV7`tV!d95 zfIO5gyqJyZT@GoD)5Z#{k#hbwrjttb!Cbf9pQO;87pm;HM%|JeX~Nn}TU)SdO!t}n z#gmNIBWZhDY(r zNB9Yrc*7QHYcvJJ>P>@=VhQdtzhq>WgNOvXzb{L|{Met&+Bto4dq&3k6{3Miy3(Y$TZ%XyN<=VUUK7@v{lVz0=1Ci$2M+xq7ty$Fc# za$U^hBl|Hv7rED-+_aa<2aY9I#p1XHyon-_S=rqWV5jJw7gH@pM-FNai;0Sg?wpa4 zd%HzkxG7`>w#sN}Y8Y9%WwF^usHW{s`?Az!`5UGI`+9E(0P}sK7cZm%?7vc|F&+ci z^JYoIL~QI{=?6fm${Iq=VTRIxF^a8aU?;ysQFLl2Bw9=K7gvk0P z6o-jd347KkUM3MHx|5GFi>BnSFD(LSSNA`4w~tv22haf)!R-X z=yq@oNiYl{nlxn}bwb9SfI3=qH4D{kg_gH$NCJF~SC%MIBm)SA9p7lNu{>IR=%@5_ zePxQ1)hgv`s9O6H2ed&QO1I=z3-nO>YOYy*lf9H})`y5q>e_$zgKP@SF02$Qo0u7- zJiHR;*C3%<$^9QXg7xPHIo!UQ$4w?uAFy%R4__Bj9M{UVDe9Z+<9~PBN!dS!yxCz) zUm<;aKvbkPiV?E-_O?mVFRUj?JI{#l84t-&skw4vJQ;d!Z zM-|z7bh@1p-@~X+(P(@#$64I;1mC!$C$H5P@K_4vT(LH%uAAMd;M0CYWv`2ycsg?=l{#b|ljL!X z$yi1pUbFfpkA5HT-S&)J28FGSBJpODd%O@>+|Zpmk^a~)rYW%(Ri)8L6jxbvI90aT z9U}U-^`)u&TAE9HI#ehrNs(utq-P>|py%ZnwbTok1f?id$ruz^yXK2BLew_2iZf4* zx%6kMR)FT4u0;05z0WFy^?wW;f5P|tEc+rHKcC2O-5ENwy!}@q@#>xLecF4yd8pUk zm7%Y@PWCu`ap_oLI$oajz3Rg7uX9aVd4;VCYA7J_&X{%;MRe83($PPQ063R#I|bJo zu^H6aFfw2gFT~AN$Gb;#g2xz!aI>d*mA9Wnk+`T_FZr}_i?RE(+v95M- z)(3B+Lz9hmC>0aP96UU&tNGD$oN3%p96~{Q7CqNhBb|5xKwnX|VqmIG$D>H6w?FK- z7;+|nc41|vM^D2aw?(QcrV=$d)o%A09Sn8hYEmRkqA0^J|I;E&Dkfy?=>H5vmIlra z2;V2v!ByeYJJ{%llY|TxTNlJ|RRtSLpF7W-Biv}p)$R1{(!=*bkuwpQy6LC(biCm- z1zQ6FE2LZ8YOX0D18$-$hGjL4JiK%xycK$e(@$YG1an&IdR{)h_^;PoZ-tOar!n&G zz!mVJ&_Ib#P!y-FbWTvrpS~_nb(VUAz-lggtF`@SpXY2w$4Zo}_DT!izwK8H4tUVP z9|Gor&7w7iea`aRac?Gnn9mU8Yw#9dZs;q13{zP34pLg;8>4loU+)CFZ6rh*Y66W( z1ATWWYvkC^S%Zd_-Y!U+Ii?cJztHQh8Ejpv*MLq_m&f{rh{o($3Vrw7U8{oG>{GT| zx)V?&P$dMyZSfm47-UxlSt){!wp%&aT^);jjiosj_LmqV6feQhvkxyU*7#o{+Y&;9 zf`SHDfN-Ii<7}N5q=OH%zt8aS%Vo5F+hrxo5AwNw0IErI8tW(b{61}WI#7XJT<|GCitZXd?N3SStUae`NXnT zBg&~fZS@_Iw>@uqLy|)u1%lG0S|dL&fxBQA9}RqH_9m8iv$nCJVeqj{;F^ZNMu%|6 zJn5ohd#M^~FpJG>PmG@?w0AFQl%*4-O)OWbjO?Ou0}v8JL%6gDc7SyuxE@4l;rwNZ0Y8%t2~~&i_}uCZ3X5;%k{lgPfty(R zDN7Qv3>MSym@4H`-;V9Vg%!X-FwP)GCUm1~ZiJC?`bu`{O5(EpP&_aj6lzU-K+g<@qzfmT26 zh}h&eW?hSCQZe{D9ZK*2btfVkF}AKnt_T-5J8ytbRe^7;G^%&%fmPs}c=RcnljwGi zaO2-C%+T3Sp$*|7WxvF5{8Zsc!Ff|mF(Z)Zb340y!ufUm^MME>zPR@aciYlA%j6+n zBjm#^8VBe=g#vN%z)E=M!M@m4@__yZYZCGZimK#plvQ$B^s&a!CJiA!`~Z|y3~@2x z097ZjXc|FIv6MJy=k?#-VwmnHaoxP4vm1sRe4r?y@7x;kLs!S#0;1Vwf)ZQN`P|C< zw+?U#vkl16XAyr>&*!Q{x&@Bk&c)xew93N;=##moqU5y5+uOn~t7#Yi4WkZf(@I}~ zfs|b?tZVAu2c|lRT$&_+*zp)bw9+E+a(A(KN*qmAzx;5f~X33WdgV;Wp>NfTg4X$chluwKewk&eLh}-IJMY+p-w^+HNXY zF|ag@Upr<@w1`yckndO+MCSQ5!Z|_B%FiojXC8aQBp_Z-M{|qF#=#=3yFMhe!)TP3 zo11&Z0q;-ih`n$BEP+E^5F?`JX8tTW*HZZ{OQUqw%zuo~f{sAYIm5;xqz4o$+nHR` z+TB_n))wq*3|t*YdEn;@4G$XR9Aib?ZV~1D#AgFi1Qw9{ut5hk#LVW}UU-dTd7d+T z<8U@WgCU-R`R7vz$NZ@fvNxof4k1m8{|)Wtj9v5%#@&%0(A>=JwS{0S1x9dn@`o#Q z{Q;gC&Gy&jjPV0oz}pP-B00`DVwhM#HLc&(jjDc7$==Rn@Bbt((kGG2F8xegBzgVw zan+>-Xy}rDW|5iMNSDFug|x^!t~a?~sa6!4iL1BqR==y&lZmH!LmwI=PjFTCjAy#f z$xQI2?-qC2BZeP_7;K^Fvv)X{Bozb(MiRw2&KTv|x@F0HBk)s##GS2z{J-+2tkUT1 zUBi+cqtFb{{nm%WS(O3siYqAKu3+3IUbkF}M`iK_nRkSTb^>}Em{$Qi^$%=XHLXh1 zs|Jf9aN8j`%q>+vwq{pZX^qLYL#P3Yzczm}lSLQ;#vsz#gY-#Ypfll^TcuzENSWA< z(s{(}OD6B{ksALpE;45BXVkR%xTC0E@6&&oyGdDnV1#wRasi#;9qf|}qvRS0IcUcZ zd~sAZ_%c^B95vx!R}e5_`E#M#AzipindIjcdvY><&3qnnTpv4PfWH1C#Xa#~Nk)CU zVl$US#IHqjVQxPN&;}9wU2yc{q@TG|l1)T~S)_burV?+)=uc^z{K*Xip+)RI`=Ve4 zZGBH$pL$=X_v6INJ`_?HR_n|jl#%hu(>_<;LgG}x4IP}8{t;jC@ojcuBz{^mZFC& z!w0b2-Nf^GmW<<9-pJ?vZL_knLFG38Gs!x}yGpJ^lUIp=pQs%cN^d(*CksxQdOMHtYwLVNvU6YW>rhS*piKpf2@BA#{>4o z?ynqA`?o?2X_`)!fo3Lgg`VR2w&80PAp*K>o8W=DsmnjLbbokR@Gh*RbGK8tqC$86 z_eF(zJ#9{@-$r#=ZJ8_rQ*m0~(mYUQd;>+8bt_&9dVK0f$PNzXP=BLamfJSmLDKQ6 z+_oC(5DU0bl7uQ#*(NkK)c3cs?6R^@ekPgj|1>o+(YOp%S9Q9yt`if4M`UbZkip*v z)q*9a*e~aoo&W;p$|f4e4i~~NBrn^zgd~z7S&lPY=(W1l2|Pf9`!LXNFH11gujs!Q z$y0Q;MbR=z(p4O6W|>7N^Me)2-*r_gH$j**eb6NN$@0fG7iE;;Fp>Ou^Hq43E$`J~fOD^fH^V;86+vu}%Q~hr> zd9~W@0;M~`)#>9AoYgC>W}*y~3F_*de2a;ElIEktNg0;ZHgeaZzT5Xt4zeaQRidN%&VkBJAt&iKZ3j?_Fu`^=^fDg}+t|TKTB|aLdv`A;hlE(GxP z9nIXi)zdzf`Ugq;Ey}M=*gz1%lw916C{~p@<*;(Nce{UV!In1@L!?aH_=S6{CLzJoY|#PGsjvU!^b5X(d+rVfuo02`g0 zbRcwNCWEs}8#U%!#g7&T{V0osycES#tE#%SC`H;L?Q+HCl33bZ6woDJRCu=3j7F*{ zCjhKddX`|2-(dW|nm2qOyGmM?>P9kefM|K_rOMZ*$NE%)vZ^@Xvf=e$jA`eFc8APn(WL{Wu5xbEU)9QzF=g> z$4h;1*4Fsj!V*{ zlT@t0Ncu|PCE9n2do~mbIS6_dYE|jtJz;-gB z{1VDdAUka+wNkbl>ywp_U~4i>sWY67BC|oyF-QRE;Q&!V8`l3Qhq9r4uP7 zy~GEbOK3xr{8N(_L~ZpG8kOO(yS5?S;~2Pv#D&iHmY!GD*wl|NX0STn|4g93R>HT@w^4&SiC<&53YZzJEs|_I2?`J z^Q3c&70Xx7lYft+2J|#sX$1$4G1(8|^n|vGb0vJH#9b5U`Jx~oRT@*Iy_QGxSC%k~ zf=-j(@!>?d>S}uqKPB#jjHi+#_ZDZ3Q#5-_PTnHG@2BrrB@xoCdwZ{ptJu6-r!!ek zXDw+_a5MMMYkMMX>R8rvE1RFxkr!X%&w9qk9dA$Btf~`$D=g`W2(3Np2(9F{QaDB) zMXA&s-AAU`dWEb0R@U`%%C{FD0U6okSvZCdf9&6X{}vlGtN&rGnBo~zD}rF_kfLN7 zeZ4oKgJ2_62QbG7Zt>msh&gH=v6)6^Ab-7O!{Ay zz;62MZoQ-geKvu*fz#Ivfv-t=GQ8AkQL%S7$VecV%Oe6itK|D|amdNaX&G9* zR|h#@9Ypr6e|!m+N6P*B5YVb%WPyI0X!im#iWiM2?Z-UX7uPWM--9Yds(` zt#2mgUCISrfU-P83|RdIIeP9#<~;ApF|P$qPtVaL!aTJ9nA;r{N~*KB%A-23G_?8s zxfCdxh2?}Rj^5ff&q}pr_O1cWqOR&Kb1;6J%;1va`qp>Z&(&5ee^uC}lS{?0wDuRG zEBopl?cM~@ZI4~o%_`GiFt4*-d=oPjkH(H_i$aQuL7vbfW7Gguw`uurnGzmPw-Aj2 zL1ewxHbJzHu|J4I*vrG8d1mN63vF_N)pp>D|L>QZ48DDPGn14CQ+rNylek*Z&uTBD zY+77CS>s!8rz@P}^;DDgMlOj4(sdo`!uvyRB$?!?{J@WeZ~~8WxG$~HcN0IiU!h;F z-B$lVIA`%r+$dPbDy*DV*`gH&yDgd#`6KFxwg3Rpip$(1O3?zsEGOSL)(V$Hhasvi zc2*#$qWMNaq`|mczPx;L>WNLS@b z(sOM|PW*CXC40N|gvA@K%Ed^J6nP-EgzA_)nYRsfdXZ_8_X=s6?GZQcfGlAdd}bnh zaoSg)=vls9GSXU1Y)liM)6o_wednm>fFmiydIainNZm0dK#D-#Y=d?Amfzj!AJ1oh zOQ0aoDMm`OyjroDAd!lTL)tpGJ6mhWr;|!`k*?g_gi?s=9?lJ@4*Maby*|U8E{LC6qN&hD!0=q4BfkX2^bpd5H(j!d{BDF$8HJTI|waCD>~@wIQ5EI zUI=NUW$4IjVh6S`(SvIvK+1Sxaqz0i`n{vzgChU{xXeU>hOj5t;R5@;QHI`Ui{6ov zL~a2WAMY0jT^Zd{Ce&Crk0a#pOv0>*x!=!CEliSn{KgPXSVO;(2bssyBC{w<>jBaG zYA$8`Wfdah9O>hp=;6nN+;PE)pHc8|=!;i&^uEkiuU1HlnO}2wqDmp9f_T?$LxE0{i;1 zEUz%_-p-3a=f%I5cxmI|Jyx!uZkYobqj;QEGUgGiF9m$~UEe8BaC6Aze+O^PKp=Pr zc`^nX!*vRXTfHP8xbcN+V&JIwiLeA)0t>(ho5S0GVRO1`fCu45eDU}RXk4ZAfe{L< zswCOc20&(pWUDFWxq5UX>z*}XQtfQnze5jRo`T|4$|kO!L8-B1MVpko+9U_5z`kzj zUEIv9yo3fRQ%Bb(2>CFOK)9tZJHu=%K^6t-Z1-3KPBuY)ihS&)(k@5}tt#9c0kATl zP$)$S?mMm(uT*CYl|DQM?1)w9Oln627N5=cQIgn6rE$s@@!SNtCS{U6_4Z0=PTVR5 zo`0E;O{RIMZ)A{E+5_;@EC4yt(kY|AjRO*E<$4fp$b8ME0ezI2lQX8Rlt+G6M0x=Ngik?Cy6lnTO$k|shd$v_~IK^OEfFQ$#wDojz`vHNL(+As)7xl%BPf zuhWt&pt}Jb<9RS8lc*b0@5#;GK}xnyC=JWIb9{Fk&_RR#9k^KheS8OZH4j4`oOK%C zFZ0ADVt~VOq=mQ0+TuYC;1p%a9^{~?Mv;iCu=TYA8hh3S3P`IXkD&njNEdl~!E)7F z5U5WH{Rz8C}EFrP2==Ag#pU%ywUcZqHH;9JqpeGNElKB_~}_MZhWArJlrW z^=uegzD{fjU(D%c3~;bQonmW`)PoQhA-|83XEMdyS=%V$NW^)vNcLVyxuek_R)|Hl z+d6WBYV+ofW6lYSs16LlAw{&YD?0`#PSu>0DLt7`{Ur={hk{q=pcpt_DO-iCe@3(( z!gt3M2PfU6PB5JGlX(mfo4SA?;8S6@lo2)7EY51q_gMN>CJO#JHRlN5l4g+5t}C(v4IodcmJ4&w_qpQZy?v1$nn41<2GXQGt>Igs8z(4_B^D@dLjWS68GO}UC zntqHd$}nw$a^51*Hqco8_V!jUjdFt~Pnnj6ot=H%Tbky)AYmKxKZU&R>o&IAVQ%oF ze97WTfIg5XbAQ6AOP9mIljrAv(+cgq z-c-7ZZy4yvQL*gcn*XI1%anf}_z$(iLya|EeSwNVQkaQn{pbGUBv|AJuWn9p*ZYi? zMRIa;?W0*Q!C_Z;baXJVWTXqyKgEA2l5=aCw=-G?%qI5EIl$}y>N#E1KW|JS3~Rdo zHy4171|X-1Bl1uG3B~`3CI1g(pZ-q{#Q#7$9RE*G|J&n#%Ry_j|2E_QEF)L`4Qc%o z*80DwS?ZsL{tsjm|6hLmU*#(0KLheVP|L)>IU8-{0MP`{|MilYS^HNh8Ww=R3)#OM zyNFXM7e|5qAD(Khl3`8V&HPXPM_d05pVlJSr};l=hfqp-gg5cusi{k-dD{Q08nac8 z|1UZ$_y6n2rT)|H_DT%JB?`FTVhDh~UT1m$X^Yf=lRuI$rvwgc)0rNgpK3{oOdC4N z00|=c&`)M)CZz!3;2iwC0N*{XrJ&sjnqC_rlmKe*dfDf(1UMbZ0qD$6viK_ro}%`^ zHgrP|jyO(`q7ZLLtv0skB}3?Mh}bx_rqEB@%sF%LAsE%91-@s$)j*dTESVDVRog#$ zne7R%l^~^2oq3_x-~ziF2(8A1F7K$fYhyi!w7v3DJxMsDopV6gb6kF{9k&J&g7HTu4qZ+tX2luDg0cDxQ=I! zp!*U2**nQluzB)5AQLImCS;pqOl~s;Z97J#O%#JphqNH@gd4~21Y=D;og_jChpQ@_ z1_tbPi};a{gkjFs)L4HJMX&*}tmTNJNRR;NS9(|}`cZ}3#5wm7={Z9U5#GxS41Np4 z1%(<*vv)#><2W5Ui6%7Eh(Q+x8;Uf~k)a_)S(CVau}_xZ?YHNb9cdO5*XGIm1@vXN zCKwXAW-WwkBlNQgKtmgIu8|wz%-ar;pB`u+kOeQMksZT2`7ufZWmkGw5Os3%7|-N| z3c8u;35FyPbPDqrGjn7xXadZ)xHy6~vHn{RB)A?~sGcIEUt5YBh7dEj4_w?>~+>l!!AexX;$_$^XdY3D)t=!@)~Ld?zQC*D06nD4ng zB{+5!$RknL?RD{jBKYR}GNKk&(>ds<`%a$ye!|A?2ZzT0Ivwm2MeJ$hmjHja7w@?p zEigD^iAe^M?YbCx@*sZH8SU)BF9|uGHSc^Z1;<&(@nX+duDU-zUPgB%$42y(E6d&; zJH85b-I-qtJ}EpsUEqr%tS20Er1)$pz20BIzK&kJp$EBUwoC*vy&-+PkFC7EcD-(J zEAOUbEhIgS4hakzU+8<^o34v4AgDk8J$PK(_`pytZ>KG>o}C!K7JM)F0iSEpwkq{= z@(N$@M$nP(NB_508eZoVpW81VP{E(AI2ied4g^1v3EJw!bI74yd}5+K58q^y;(_M{ zX0}s~yI->RzeqC8o=b<-@zICw!CHM%}r;Z)T z38#q}CHQPEaI++1_08b5r^EA7lz7X-#J!f>B}9$;%FX+R^W*D{BJ5(qFA+8P`}Pyb z$JA)gU&{_?zJvUdGgVqUrBgm}U9RttzIBJGZk&R5F7F&K&z|5m@Ur()*h8N%#H2 zRI!vDt66OX@`dzO^9-yuyyGZt51INJM=%GI?jg2An_ zr8jotN$!0ob_{N68}s5RaAPyOvpL!9O6q-}>s8?t&Ub)e`P$>DF#4U0Z6sxM&-5w& z<5fz&av`LW9QpNf>S~L)Jv-y8Yln?tX1?w_fv)#GIesIhvvVBr<8-jwN3NpQvhmUF z&g(MOdz|Bmwd|tlaS!rkJn^f}b*9Yg*2mq2t9zhoU?uHJ8wByY;CqU$C)bdV7smWR z^;PYLTf+5cpD<#N2O?Sab&(~lgL*J(;B}zPx6|4SQ{98|%f7BjrjC^ZnXbEXKK@IY z*OTlH;^b!L2k!T)YQfR!_h;LibDd5%{lt8qD;9y9M|j%I6y*!A*Z!$@cI3!|TIkbB zpU5fiYjC=Pad_43(#vJ^75c}-SHJ-uw5A|I5p>x8vs>r& zh{9zd`r`MXq3#~y&5MuQXx5C{5WEwDx8vfc$KJF=Y!K2b3h~qQ3l0-S%xXte z)t|>k)BRQz>l=Azx8ecM#);}KmxRwAN@kv#vj4av>y-Azli@yh@L4TM`%oW;J_fXe z4$s5IPd9?^j$qI`$D)@0;*ZB0t7sp=k8RV3nf7N4pX*}cow59UPowOy0m+XDLHFN1 z3U7V7CUrwbXRG<1rynocVp~3YRFsfekNF#W1vg%=E?w7*XpP(gcT~g&QyHv+A7K0h zzr_%Z6@sU51&2KTNx{cNA7Gh#+LZV9m?y2~cDC20fb58z;`goPu2*!O->chi?Tt2G ziOEDS1jHXVOspG^5bMBN$0Mot%bTgWTB@g_(vat4n9k=^!94{*pRm-f8|G#l464rH z?6->{Gx62g<;^R>S1*+ZuU{P<9M#1O*lLr+yIpUAT|;JJ=>Voay>+ploR8bYuMm!2 zg1v(8doNOr2T!X*f)l316IbEHZpgSYmk9IL_QzLKmjm@5f*zL;rpEe|o-ZE_b5p77 z_-uXS-QCl?)P5ka(}k&7QmXT7d#YE#|JYhR5` zN-+oPPUwLvOIONgSd*_Po+43lC#iiEnmo}5CY8OqD&sKLrbx^_gM4N6L6_YVP{0tP zy~N}s7A54%%&U%>A22=Q1o9VJlkJJ0sfis0p}w#;b@BuUGIfgWbK2yy5A+#>MpHfe zphrP(zvMLEq*TVU`y6j(AK&>^<4e9i#)SqicX}c*5&ubb+UVOop0CK@4KLm=x{0^d z3CH%`oMYCsT>5PQ!Dhmsulpv%crfj1MySBxnu*iePDfN&)RcNFjNa;~rs~Oa(4vGe ztsM9epOiMZ@I$kdm@m#K@uq_LWuvUWX@~k(ubYi7Q;`6lFJRJ=uPHM*seSl0ft?6v z4!wFXv9Ff9-5&L3y~JR8*LIe!&tPs=ecbEHxUS3o%Yc*(){_YZdS#{<4Qw4@zq&~_L@gdDnEv!4eeg7; z&29Qh@3jS!rK2(27eB_;=*$?+#64I-vyo$~&w3ZUz;g1`wJrZn)W_q>eYX$D(p(-c zDMgqd6u&Wol?ukAOy72y9uF9 zgdG{dUU^O(_vlO!oEMuvoB$>v7^ddYub=GODZhq7ac(@WvSNg$FJJ@6(Q)S=tuYVWtn<~HBFJ@B9FjmlAGWGljUlH3&X)UlK(@P%Efjd3@;5HE8I^}L}Jwn zcg`vPC({L0)0s9OZU(KKO}WuLdYW*@3~EUgE0eBrY~*#;2&=A9k8hcLBJ{b~Z?N|y zk~?UXZHqOzRiBYPZ2{WxTGaALF8nESmAKvIv|8g4jSi=3-;Di>m`fz#1?B_F&MMqn zPaJ@k#6KXyzbF+d&2&!=d4$D-^Q|DIQNOii!U}=o#(HslFIm% z8w=UagY0ilHzn;It|SyLhi~OFaOUD68w0g1b@gY;})9xHuw7cmaToKp5AjRDp zh?WM&?^bY@snP-1*xY^li%p8``yLU5Y7v@&75EoLITyHbaa#T9>7|YBx5TU|j03nO z+&1pEYC~y;rr#qcov1H*+&Jz&tK2x`C!=yI6gh_VsLHKX!Mt8wwX#z^6w5r}LKBys zO-p;<9pIi?l;m10?AWbjQX^f|l$5Vd#r~NDkdB@Q`k4GL_TDl$uHe@fj2%PF%*@O* zW@e6=nQ6>y#~d@p%*>9N?UrP4;U#6t z7?I*4$1avcRe6C)fBWrDRC}`!bnTGqZR*DI`K9JM84+knEB=rQr{D06NEN#RM<5zr zluY-zy4f&kCua%+c7z*2&E5Ci6gkb&Epx87Gq_8_7-^6?7?)IaR!jwyA=f?4_DGNW zL3&cExhIhP(Ka^Ei)&sR=QEaJA@*af3s`vcD$e{X6?;>yY$*@RayK&F~@SavwQQA;MQ2Oa8ngW>o> z%p-GjJ!c+{v{$3sLxC2}V=qAKip}ZCw4K5Yq@7Z=q(0seqoba@Df9c*z_CMklo6EQ zl(~x`CGl@5QyX!6r>qHjPFV_)q6yf} z_j!7^Mn?m^5==@eVm?405E>)4LVcH?eYJ1a9bNd8DVJD*Wmu}IE!kQ)HI%OAIE6p5 zyeZI473%z;f|aNHNIN&Y0=&+f+ggREYqh-797a^#iQ*?j_8g*uCs=>lAp;K;(uW;b zt`F_X+I$Y78kR7+I(C&`Yfl3LI7>*v$F<@Y1{23QC=E13MpXb4X-Sf*rWhk}Tq2K% z%oqr+9k;X*MG({MSxVPu+m?JNmy#VgeQC;eu2^LUd~dEvb6DG@N$M(cWYGYx{?bBq zNz$jhyW)q#pg??S*#-7@XDue2M)Kya`g^SPhQDkT%IU&&6^Fsv#gUmU2TeI%IXkZ> z!n}*Sh5KhW3WOX9*%CPLX|7nv#hrJYcCEyPk0OmV*FcO)_iu2$@JUOb&4)*~X}J;nuYHZezTwbA-U=1pSr6(nBz(7}WnCCJ zkU7f-ZlXWjlnjXc;Jucd18DPD-pe*PnEcw8@S z`PJjmv&~rqqH=Ax=jcv*b2$r^29woy*m1hirczKemL@3i6k>WohN*(I@u-fls)Kl; z>E=by8veL~&yj07<3PHce|>cPOd*`k*9){0IFkCg`H3HSFIA5w;c8*^ZduN>eSo*; z%qCZ_Ck!9*p2VBspl(cB>v$hC-je5qFE0Cy4*w_gQXSGJRqeTaHT6`J!p&QU; zk1{dchO3*||M(XRjrCxV3%Yl!KX3W>QJ9%?A$Rah#J?(teyW&Q14sM8GFv98Jzelo z;d3lALvs$!!DKAqwa!5!M8DYWY#h~QLmZh|^(xn{_Yof`|LkA+fchSAG0jocz0ypd zzO+i&SVuz$F@pxp+60;X3~_MU5LxhJ7bBSZ75Ot>=9M~>GNoD%X$hXK4Qe|>8vBvp z(;9exI>R|}m$5qQ^90A%-O9Fc@H;zDqdJ+8T+P&zUwFGB(fDC)E;12>c|JAZasod= z%klyKWD5{VM>i*JIKgYhTbms0dkefpe1KXaN=-GTE7p5Lr^brfZ{Dv8vkhnN+XF3N z7TT;7Q2b=u?};%f7!E~MG2I5BwbdQ7hm6uL%7loGXA{iTu`rGB1$|R*K=*0S6Lano zcIVD%NJZoe(IVGl?ZI^S?I4(Gz7ksf$u!-V72Sb}2`$vNQiAQ4WP5)z*gsR$9s>QR z$<0r99SdbzD7Q&IiD}uB9egOr$obbA2gYbazj9E2?Gj~%E2~vFeYxuaTiD;cz=###=a*pAsL!-tdK3ulC&({v_8!|*y&~jk$^nbqO`k-efV|G?U)KQ>tB^TxH|AyM~H( zzq4X}t5fwu*_J&nCM`M2(Gj-7Qr7H=By#*UN+9MP03X%g+5f}N+o3L6b?02ZQfse@ zwt0e8>XYS`2^2{*@pxgA8MAE90H(*Kl8H~u55J#>P45crkCwNBvsIKqMZ(Vruu8cF zp6tfbyv+`H6U&Z;6N)>#sX^`K-&9#+Hl(VpmgPBqaxXSAn6U2TtfomEGwru1T7Lb8 zpe{LNRvxN1klqrta}=CGr^i zl=|+{)7N=*=(>WR77N^$>$pKN`ne=p{SI2kz_V|$E3-d2!|pIQ!^2lp=Tv|;_>n?9 z1AIpu$A_YBibGcAxb!ELQ8NxeMQ6SRXI)eKmDXXde&Sy+;ZRokm+8;SEjcVFq=K}7k1D~byw&wuPQ)ExZou~$^A^!4=KDqksU<~f_$HOLIF>zBGGiWk9D{s2?hG^h zFAw*aV+plC(3PirQ}^ROBy1MWC-wclIr3~xmFGfB*r7D+4`m2&w0@5${X_Fj$iZRfGt@mR|={8EP;$$tJP29z~0zNpN zrX-tA&9T%qlLYvkE6-%)EKJ~WQ(p`vAl4C&7+j!sw@ACJmDP2Otz+lGU2a}rFy0l; z=_?zL-ALsQgKB5ir}ci zYeZY<#!9qrPasD|_KK!&g1dYghyR1iZLjy|jDQdZRhJclXOFpN!nKC%<5Zy;JMZ?* zRf-%B{4vS5jK9W4a{O3mZgwdiqeGsSEWJieL2z$U9KctHdqsvKSItkfC9E?e+gES7 zSe?>r3Ch3ff9gYY=nJOSYyH^OB%F7QbA9!6U2`rDuf9eDF>i|PU2ouV<$u9PzFrTe zQZQ(2Pi_UuXg)ETi$Tp(<(@al0FHvx!%d#0OM?*;9yJ#$sJZj+8!|PN|np{ zHj>A-8dTavicD7}TCSd%?zpLLcZK6iNI!8h_%YUU!fd+1Zq@OR{grG?f9yxUeZRzx zQsZZL)Rmr&;wCmdy8Ux_-y3Gw4yUc*=NFZyw4!8gYfgRlam#Ngt3Xrslj7Mxj@Jzy zD57fS)(zrzY@;U-4{F+AQhGm=^|jF{EIIPJI>?(PM zz-Wb50^Do3sf1XFZ1yPV=ml>V+IZ^1L%Pz$Skp%jZ_QH|uM5<-hRTx^a(8w_I-xEv z(n2-VMzY`Bvkx5-aRc?63Hpweq`_r`dt~Ait37ZQNPL5TODUIb@kCDR1gLjnO#$Z? z7v(0KKE*b!lTTSo;ggX#xqj!;YJ94Tzm?NqZr$Yg5mYip`)$u-AW=7yP(~dv64#1) z-7f1AcReWl&Qn#14=A(d6vnN>h?TIhi55+YZY-?VR6w7jc+qY(7W5V9^s-xPr1eP2 zJmd4S6Q)^FG{66b4H9lnPtN`^eNE@darAfE%Uc-Dqegc$b?b%qwTk4X5MHw>sIn3p z`Stq40t*Hte19g`Hc`gSh%Z(TF z4b7psWRxvT&xLwkj0h^S=kulmpn_+<7vddi>QAijBNuYvk6c5kD%Bh=Q7wf+I^O=( zpAp7%GutVh>T2+SR<1D|?aEbqUTc*?o5kv}#pK)^v~jeiR>FpHYGl+^$SN-WbTvUl z*_!`7=>iqUoJM->=C1vsNMP)I>t}ORFUCOTl$Zx@>Y$LfWA;Sal`V z8bMSo7u!ScMjz?W_^b8QN_n>XyYYejPq3F3evh0{6X-Wj`q@QoGE-ASRy|4@(c%^H zkKhD}(qFBa-7m`Xya6en-KjR$$sqoqrp?%WW02}oY&&Q zC=jgB+xKID4BKX4PQWWT68xh?Y7h4(X#MFhL+5yrsKgKLWXhc*j&GPY%v%}183(;u zrR+L<|5bkxZ|nRBl8G&1-Yt4O69zLa{HvNg?iXM#UgVyaz*9Ff^PYTR$cVUy5{zS5 zE1j2LB&8QtrtbdYqQgN`9fw-@9|~d7ut{|(E)vT(vkidzF)N=fpVEMkrej6fag%6q zC+Zg=nkabqtC@K*x9$h27{My19nmq8aa|Ot2m(8kDBMk=k_Gi=+X9(fmmLS1i;{k@ zSg+P=+Mb=Qw!Ms0=e)A+OwP?_Wq}Ju>hDUJHI?(plA%Ki6 ze*|w3k*AdJvm?Uaal1vx$i) zPBUjF(i6X#-+%cQYcFL5Pr{6kI=aE>rCj)-2j&w{ZMgIS`?=73K~?|%>N zl}eD0q<1n!qWZiuqnok`*ST{FsR_wAc44a$-}sTyPdpH8GR5yFg+M~@04G|1uj?!7 zS>CM)Xu5VR%7zqw8q>q$j!zMp{=U{q)09kfiG%lu4O}G0#o=E{%TO0v)pQJ$<1Y}^ zJ7X~-kj=2oRa19z;y!M_pl#2&B(<}2g|0bpwjy4HdfRfGuB~OKiHZFR8;5nCGkIEh z4)iAnWUu)6aVbx&j{WrQmD-6LqxJKl@$8o6>_#DSeWUgJ_V>5IZISd?=eg%52LOxc zdziAS&X$zs4rA13gP+U9`(ExTlL|cA)UilbGIG zV<}ymIJ!^)KHcbWym*XO2a(T&rirw)e3ii>*Ld(+4tqBr`ZCW`fg_{Lc(j)3Qw$HN zEaPRf7GGUop;QB@--frwe>tlGnr)~*9<}UN-B;Z2QMD}zpvmDHc&D6wasLTrbM|u{j0dM2 zz_!TOjTsH)?r-)E>x(rRO5i#6Ee7Jxe!@*(ROJVtSo68peVQxn)B0n3l@qbm&^fZp zZc|$Ss~>!mm>&W%ux+h?y!;zRnafo`^&$i%EGBbelJxPnu>y*bNH@BXBHpIv#W~ca z)07j*s)fE7bRH%V=n-n-Xfqr7%tD~%Ktat#E5ECMtGPN@J;lW2*u{r8C2UwC0^i;m z8BcX5)2GQeX~rMYtQwS0hj<)qE%($o=-%c8Q)nz^ODG2?4I-@1H;C#W>p>Xm%hF>L z$^(GzIFffiIG)$t57wp-oT2*2mS%K!Z&tV1XsREvAR}VDL1+gq-;8E!I!SRSE9-bj z^unp);q6_ZMtZJ}=&wSL59G?%tx@IJ2D|sGXEh5RVt;EyV(9FaBKtM(sBDv=VvFxPic_#%erSNq>>e5m`@XtPJEMN>$n-G%!C|LW@bbcbVN6}d`6D&TjP9e zMYJN4;i6B65PaPpQH~|vekn45@dhr%nBJa*ITPqFcg+05StAc-<{n3vs(dwmKMkz# z)A1f|7MM%aN8GJp6ksQuXEA}F4dm{gG|MpI=h06JIbF99_*Zps{>Q{+x|+kWTyxD_ z#i=ps6pv4)rwEU_?)NW?u~OA49ZjeEEpP`q=6tM~?pn(c-#Xd-=J+(3yn@y`i5xRE z6*eNb?1dm``DF3JFwH$pv`+~QGkLQH+5F)+NkBtbuykD@e)AH~{F6Nr_DBy+J0ycU zBu!>RiBdb6W@CPb!42E^Y+u5Zer7F@rS z-ql*Vqre#ouZ0oGAbcaTF&`lyu$HaMcPdjr3{$4<9~)1GLZD7M1bvN#P5*P|F%A=3 zMBW{G^A`2=J6c(77f<@Q_=^*B?f_gMib0^tKOdB;74@%tZlqxa>TlVY?$ie-> z*N#B_nU{gM)^x&-Wg&etGN}t}0!Hk$Dlk|%3*b+fWVM_s?QE4!y)_YBH?;~O7_d~o z9BRG+uK1?*`!Oq7eEbD6kqnBMys9wYX`tzLzt(l!Wsqib$n!+p+|9b)*qnw?!WmUv zamr}8HzVJk>HZ8~zANuSwaIbn)FRN(ZYp4zdF%VO1ynl(iWa)$AGP_4&ra2bgA91g zK9+PBI?d@@mfXqoE~LwYEj(Ik1Q)}z%sMHlkNvIhSeDFVR`dA+XC8GH(>@VZ#3zb| z;o|WwEYYCp;!)v5fj@{&T>=s7^HuG2lo?PZ!v6C{Ce!yNfPtt2TZCxtB{jBYoH57m z8V+XL!}Ksn1hR3$X`&SH&of{RPy%4TqzB%}OW@^V7C5x6&@E#9%` zDD-PW-ZP2Mm@y+0ey~EwFfvC&E)FGV7oj9+-JSBl*I_@bJ^CbtD=el1Ytm3da>uun ztwTPBF+0Kw>I$ypXXvzLZ;^>&7~9st(B)hv}Wj68^m(nLX6 zY|gCfb{?k7pth8gNks2W)zGLXr>RH$Bq>kInKc$c*Kdk4+|L|@osb8TjKrR=Dw^JJ zM;LQPp{f6kDvx7dcv9(Yyi3U8>t5fq?<9kDWPmbH-MbU!}wMK=albG&yZ zFe)t=F3z5ZNmJDHEwljA@Rps8k!kXrkxQ@gG>4S+W%Y|u! z*FcR70Uv)@pTGKuP{_L6?k%y=C}#DYe=*@k)Y?iyvb-4506JTfdQ5*1FY=V|sb8Kx zIXG1F&Q4Avm)xV!FLHk7Lb9)kcD4)76D9MqSwXwx^@<{>`#lwSNb%%%`VX)Gr?|wD zr><^Yy2d5ym}Dnbu2WX{orA~yY_xb|n8Ozo{rY#uPz<4X!MC4CUn^DS{1mBrqa!w) zgB!flA=o~W{dd~z`jO6F+jAgfTdI5q{r9I62HGCZzev3q&Yuz31nB*12?i?K z5z=o~S-$Jx!|@&|w7U{|8^+dAhfUV5u%C8dgBZQDxlAHL`yCb9PJQ?&eY}jUJlM{A z;|`Y9r@tcdKpTWRUoEnwl(eDwJ>x=j7{>4H{py&prS`hA=oNNqvqb-$x(>HS#4LbJ*FOxT2LNVeGBx0RdRIh2JuNJ8$^ zjo5R14M*O15NVdGTqRXG$0K&8pH2v|1wVYTrLmIW>2Z17;r$WNNqv%XoER;PJm5nn z?qsg~xE363%!acwUeDp9MNFCB96O>3I!cOgS^aLeSbbrX-yV$S?my8{y*-1PoDX5V z^oTJoyYp~zy>o?y)hsoDDc=SDaFUWwF;J z@pqFo(Y@Cz>1ZPP+CC$G%Zi@D$8yi5FI{%!_9LM8o5mJDd+h|^5^ra`ODF!uc9D#& zl_&W!Q^q+dsL-5kZ@W|HX+mj!u#^ssFATYF?}nC|FSpomXm0f z4v#H%Lod&2>@Lt)7BT7eh3%*o&P~CiNOipCQNdy^IPH0X1*hmK=|t$&au|trxX2m$ zC$CROl6U9Z{Ym0PD-5U0$TBUg?LmypL=no1H^+=(hta`d1}lHiT(;KE;$Yo_)*KB( zn%_3VI0L^jME| zx9eI`TvEdOZ)1>q0MMUwCS(Q@Mnv<*4Gg6+>WK9Qygf074wxt`P-4R0i0apXN-cumGhft%W6&k=idV#evCjWJAYiJQ-k>6Z;>w&lH1H=A!a@`9lg7dv7n0x zr09aXn+DkXb|ajibY#E{5AWwEgq>h#V94 zc!;RZauOM+@y_%`Wobu+%WI*1I{;t7h$<}t2O3e1wFeKlGfnd8ku3#~GlOdBwQq1+ z3c-|G@CGDCN~*FC0O^*5oSzNqBZrv~2g1OD4Z({)D5qcLwuT8W1*RTor#mA^^2};L zkF>BVVr=~5$sH*S;opom*{fi3DoQV?WBzispKOb)B(SdRry?X+Xj0)MQq*Ys)uOOp z3QcR>5m#HG|KFPTt?&#iKr#LiZm>%Hr#DXka;gQmaw08#M=f~Rf#$j@$6Jr1%DtIIE zW&|lB!?he@V~<^!XfvoHXW?0I_r{AAF(m%0kx~52F$HnFnx5Xo&B4esB^UtCpwnaM z;O1IKb$))H@ZWcQ25iNqZ9@N+cf(o1hV z_v-|gjOYKkIWP>47#>653lvFw)c=P5_K)*m5Bh(c{7?7) zJ?HEH=ODqYtjoZFfB%un|Gbu;^Z$4J-!u6CPsRUxTJc{gOTw7(BQ`cRqW_yhw`JF! z#{%b2jQ%np083yrxw~jXU|IZi) z1=yVP7+^V;rCilG=GtP87J0(zXPsfiDF1HJ`k_X66^E6Uva;z7fuW7=eP=P5j{p+=NKL%N zVxlaW{Sl33oLmyX2*@_~9~0VWXJ?r=lqGu$&siXCwoC#7o7jweTqq2Tn|621X%|)s zCQk0K$Tj$F5lGQqSj9cU5pEj?NAgYVAyqfOJ}5{-oXUP)kLuXrQfhMQwk}chU_&>J zy^z(LFR`()5(=;bM5JtN-)ez)4D%BH(PCmQsN#DAyxa~`CX8FrJ68GE#2(q8a4A9~ zaKnz) zQ-cSO`QRnACkdQjwDhN@NzK_;oRZo{6@s6*puIh#r9ATNHZcLUzlNcUgn^iBb5TNX zQtT1xOBiGuXV}Cl^^_t~=*B)&GRK!T4RF@UM!B7Jf4{O^CP`IE&|lD(mVr*rfe>Nb z!Q*AsC(|LlENcz}C^2I7?K}r5Yg_wbCQ!cbD`hw&Zf3tXcOpm%Uypp%+UqkJIwUdX z_CS=15e`i{w#0D}Kq8ai`-v!eH{UVuV80?KXRGPlG?4-s?=IvB`TV?sSsRix*EKX4 z(TQ0-j3qq;&!(V&PGM8^=5Gkpn;~b|Bk+4Gx9ZMjl*JvhhUWoF__ePa=KC^l`kZEF zW@e-=J#k<{!cKH4`4do+Q^B=LigheHeAg^$$h4`c=~}jZ%h7&7e6ZoUI5=F&Ko4=j z6(?OQ>ngPMEM;m$ACZDJK#NKoip+Lv@51D`8 zO(E%;8-Z?>g8{{BS2;2tF<|Tb{Fm8zV1~ zi$vyQF{g|>9>L|NXyd0d#Tztpt*m$kdj^&Zs=05^z4gM6)IA@6(26^rM5aGxi1j~d zYS6}!eB`=1?rga~4o89YweQ>9mccwJ{i^A^|KEGOt5dbSHw8=;5tsWDC6iw#xG_k} zTh5+Wf`JeSqG3B2a5KkdT*MtcHZ%-uCZ7PRylFsIAO; zT+++n;^~!Jm{}H#NSb~aFsv@dT4I}SoGE!`zc+QN`GIkKYf(8`t}wC0QWK~+>dQX! zYMPC}XG?auv&0%m<^m=>cGFMw0m#}_cWaHd#0WejCByVZ9>^uf3265*chXcYRPuPf zsgss_r6KCN<`Ug3-6K2VtpE}?NMRdVi>nKsXJe;NEU#AVdr|>vlFJZ1K2CHVL|3Nf zJ2FkLG4>^l{;*f?=R|~3fxi}+-#5?9v%-Q8vYrleoNV4#yYFrG$DmUi5H%-1GO>1Y zMHkl(PWBZ$--}N0ui_04<0v-p0gb-A2wILu5|$9|ngq5*{Bt$F>~1PJ!8y>=_J4zM zR-(E-hQ+R3_e8|%;uvZ}WgT~_* zvYefvdku@2oOu*qOj?!ew3O~W_u(#2Ol7OjY2%ogASp6Ye?nE4Jq z^X%_e{K0)W`@rSs*X-t=;E;1rauah{k`F>*U)o@g_~{BYda~vryN1qWz?DUhzIP}@ z9W%&j%OCYAD7G*}-urAxxx6WB@TDTV+g~5dwM^L853mWPclthCgRJ&Ji4{zI z1H(givO+-YDve(!D=B@Rv6O6sctLq~qz(gCUDsoLth`10kOaZQ64v{lq);y9(mNr7 z^nNH9*ih@GO`8pEbEGrnKgOB=x>jw921=u=0Ns}}AzNl%LVt{8-7FILJB#;m@wuI# z%F4>>*M){cMHQ|$a@8fBb5Zrj(r6J=b~lBeHwV~rIuUzCogWpu?{_0#kxj6FM7v@& zcXj;cta!%S9J{SL_qNmGd%(w{DC}y(s8Gol2?{1S*=&uLSyvTpg^QnMpUv$WIDhte z7Fh1^Wn-KkY_UK9*hZS~W8^qj^7hTIF=X^kM_YxDr~$1hTs3n@d#=3r2EA0>g~Fam zrp!u>pk_SWEk7D`sz+T+ZVxbkW$6wG!Uv6YAznunj(}+Wy+aUhwV;Xl?3*&0qgEdX zx>2VKDA@CeGCWhfPu`wtX4h-{Cx|@Q@7ZD5HqJhM=6+3n&6h7>`!z{HW7!16Y#?p& zBSX(%YK=)Dt2v@v^%3aV#3OcUW2yWR+^%ayOq?OJ{otgXSZTU@NE7K&ZnjZar^6nI zC4>_DtNRil>#8aCs#5A<8(d1Cv($Keb7ZRZ&Q-z@_5wD3X=Dc}0U7x~tcYc1iUp&Pn9p!%?fMn z>Y7Ztzsh`+4GFNKrOj!mSA%5H+wr0nxEuL-!f!Tf7onPn`=9Uwn?&~3axMT@w#DWW6dSDOE2G=<&0Pp0<_bSfYIlZ;M75dn#+F|Lw@lVLgY_r%jn`zjB2N3!<}h!$YSf*QkSt&8GIfCv}?$Wno6p zzIz~=H>fxX@JpXxM}2Sv1nafpoS+j#=_Z_R*D*-aW=f8pBFLf9=t>-*PR@ z?UW_5kO)X4$$KeR(ta|X+NC=3%luf6O;-|6pKZlElGeOi>vG~v^G`oE95;Ct9H7g& zVLPqQK~+lvSUXHpC?k(lfe5RTLG^5Z2zB!;?+Mt`TM`YeC4Oq))H~g=r#f!RD}^DI zeX~Tw6Z)Q7P@9syJ|6Qs!!7oP(tDYlU6N%Tg;=_mHfV`M$VyEY@4ZI)`g#`kfhl9m zSq+x@!8Tk;V`^`RBcon*WD(_ZIPD7aaxWI;VcuWC>MQouud;Ry{?uAYoQ)hPcl-@} z(oN-^^uWr>>Nmf)oAZm>bCHi+ z?AvFoW!;9xnFG{?MB+$Qz$vAl2Gg}+Q^wsTTVx9=?DCJIDgM@x+H*rr?5 zd7mOQY_j3*02lcU6FTjB?eWSOCV0|m>H=hCqdo2&yP!FQ*;dpOjmYQAEf5t8gA+c9 zVxmFQ3i;=%E%|iBT!}{y6Mz3UlECiq=D_{p;=3@EmDR7WG1)h8Y`yrGc%2hTl4kv6 zhbJPo?l(w#%^2N)vewwG$XT6}Iq4VlvJ}MO)yDXH|JO4I=a!=MvX%8w3pWWk#FjilQv+ zNmo0LZWkNkoxhXBhkeP%ev11ZqDSh8gj9U2VGLgFv)B5vv5rqU{Al-TeFEZ*V-AC~ zg!kL$9Ijc*+hX)*5|Qozss#P{#BcyxN=FCX{=Y&qlPglYbk2$PHAj9v+zn$`1`gO| zuI8DcPX*xazl~9SLa{eLOgUJz=lt+Kx5)IZYWrnbLH?pI)UXawUl4w=>FVX{Ycl&q zKmXyHYMK&A$&|*hi0SgGfX9XwI}4{zX|rW>c%(2rMp$WY%q%$3VL{*=e=7MRLlO|5 ze^>O9!sGQ=AsWhS)2{!yL>jx%IBPOr^$pHrmK=DFac>gie<^O*U53%)fgeC_))=J3 zLrdvX`8eYqFKW=cr|SH9Yv8>td}VBcyEl ztG2Few}$xSwbnCepwoFo%D`L5YWiq_bfcV!iPT~bQE1&z>Z%@vy)7*Ak5l(xQWIYQ zOTIqPD7^Vzq%ggiMbTn);jZEk#~$k<{hi*3IOSB(nLZhIIJy0h%1~J- z-XF-|(-J?(RLO8Kf?Sf=s9+<-C));-_%HrMk z!}9WhbMlYIh4vE!2~A$v^RJh5pBW70V<;LuqkEx{NHG>-VPa!7Z+^0%xDp zfNOe|?mt3YdvEES)kxlBb$81kQ$lyhm#@fQZ5%~v=?`}cC zIdCi|<8h@TnN}#+I*GttXGV8-=RV~IAfyK0v)$t1>PQPv-RRZ zb9823*sk{TW_reQ1vnPdl9C!?4j1|U#^6g{wHbqbJYZI`{4Lgj9U) zsBo{oqLi$liw=zM{5UZ6F8yuz!|e8nZS!%G|AOUHh+)XDqy87c>4_QS0WD^b^;s(= zQ?_)j7N%l|K3Y4Uy9fdLf$eJ3G1MP+QoS*GVX5w2g}YB$5&@<)qaYPq`ihE*;?mLv zivioXF~g=PBq^GkGK@uLO2mNh`1dzf=S!b}ALh!@eahG4;%}^La)2IR1|vlyg$yTQ zU06sTWnH(JBlBB@)GHZ_oj47@W)S1lUpLCxQOSpQ1Fwfat={=sh#vTnJB2RSr8D9z zcf$tm^zJgfTpBllJsBcHz>EDm_OMJ93l4NsZf-%BD?6Km_O4IWcjuF zroL(!h$z*>#P<^5CtiJDec>k8ca2iZu&k}*%cH1(l~Gir;^O*Ykfqg7wU`^4ng}Rw zx5zk;0vKt=TP5K5L|%Pa^@c{b1dL=%>SF54vZJqg!+&_c;dMlG7VM&!?AYKMP+S07 z0-3BzR4xhD;6ATVb1sHF@NvkyRH^W;^%>(PX&uh0K0uLv((W-krfZ+>^OItj*2^N0TV_W%0M7-X2@`dQFtPI(cvZY~{$} zorNcdUjeSQkC%%TZIl8LV;mEIRaE3WS2jl?taBky9R(?MZ4hQkJ~dj%;XG_F;W`oR z+?f7Nf4IO!l?zV8$%{GvwcJOxyWcNQv7UGXB*=ABaqSZ{Fyjp{f_!|rBSM-Jnr0;4 z$yFQx)5o64_0_$Ri-E9W3go2+YgzuTGI`d-T8}N7RoEBmL6|~%zVEta!}%-MpPS_( zC^f%oI(*g&Ft1`c-d(4vW_mOhJ06xh;+j?=@Iqt-_sgQ%`#4rQ^VfZ^&Up zS9-V=K}N_je?r^=xL|IK0ZsV(&#MTgMmZG&7MJ^~IamSKq?UvJR4XtQ`hS2$3e5<^ zBt`QXeFtNy94c~h-nQ1bah7xg=)rl50a2qXspyXd3fGp;cB7)@&4Nx0yR(Pkm? zF7rro(G8c$XN-ukh3yy_o^Pw?>7W+1CR{FB;3O>)-+d@@jx)RO_SbH)MasK0j2T;l zH8LWBB;U7QVTY$7(kgd}0gb&#Eo^Q;DH4rt1bpwNfyZ4BkDin)OYilE4hsR! zD*SW;XS)$53i`zP8e9^*d8gq8#a&`!4qy(a?fSiRQH!Cv*13OWAr?3t^3&Kz;m|qA z_q;ylWDY*$(+IZzFr9%O@#z+_u;5eseHrx?LXsz8p=UMDL(t9UMj`JMfo~DE`RhGb zwwF7Kw>vhza}Y~+*{1jvK>3iA;Jo!p6zdW5;6;KQJ^E-=SBPIN)UHf~Gb;|*(*a@T zz+A=!lNq`UB<)T&+{TXkJwsMK1<F>-V+UApe6m*VSq^Jf)hw!vpf2?y*mAYDtx4Do^28?kwL?X z#PUxRQnr9zsF@o%y{Z%pRIQY>n}XWI?^S8X85zhXOZEj06<^O^8bA7HmP(wC8&Q<= z!C@S#ux9`#QB08`Bp&LM8lIrI^$}kGpYkc*{-_}aK!#edbhRKiN50wt@jUJfxI6`S zFCmQ`Sy`2!7wP*drkaBn0p^0@*Oo!t0^NR1nCAmLF`%(Y*zwmf4hzB{9U%$uPRsC`>l4^>IA%Pf zO~v$#K+M83R~V59b`qw#^jZgdXZ;+pL|#XqdS#Q`+|Ex%Zjnx3^+V%PuI?Yo4iDij z*D_~hjaB?8#JH#h+_+K}@6rmzz%qw_M??y<&wHaoco;`aXAj0D@g0wbg38#T+F&G9 zc+XZ|IhqSt6>NfMRq-Z-5&`MLeR=J`*uZwq7mDfR=Nc-nzwP~rMQrvrCMOixJ0e_$;S6#ixczE_Bpq zcZ-pvXKG2)MGC>|`n2-B52f01tAPeZb*cNkTxv(->BAP9^zWC#i8r7O#^YtC4 z)iwx)Sz+~6$iV0V&kH|L*B?lJ$X7U}Dm?l0~fz)o}tEV9VY?ZTBvYsZbF(=3Pj z>j9bp&9T2}r0yPH)?J`cP&Se9ndfm?Wo>WitDgI6q4U1#RTS<}^Z_>P+6G0mTi-*S zv21xV-R-9{S%i=*nye-E=7rIh2+pvSo~7lt*0za0i<~Uw`4@NDKCk=%);vHlR;4 zJtw%@Q86buVCL45vXf_>$*9PZPd`Zg^I9TQyoYX97lNL+BKvhdhjR8CD;+eq>jAK~ zAggH6W@}kO_$7-l`g~7tr%SrU&k_-J!vH3z?ZMl<(2oMB;?6$sMpL?z)v*u)Je0)E zqZ;i}oC9rU&}FXUH#c-N6oL`IAWT+Eobmj%d2;=lP(1WW)e>Ekn&s2Lpz$n*J*_m; zK9PDkePz5Dc~OP*o+OPs=|gGfX5RD#;cFae#&+&3GoB7HbDE;-572&p*3#Mp?`Z3Z zZQpSrZm6nJ@=R_l)v+@Ya>pCeKC-r|4@4$xdUzLNFDZh~xiS$h!M#?&gwP33% zQ|ZcMm2WxUexAPLzXwzmh_=j$d~W8(>(=ZyL|MGe8n3c_1}>=H66t4b^`)KfRm|_x z!yR&ss;^h4teR)ylB0$$j5bUvs@aruOh%2%D`TztXL&JmOqs^G?HS%nj|KlvRup60XHiP^_QN>UlWzM3 z$9YuK8{g!hRPFM6YU6Gz`G8B#q6Yp{*nM!GzGw+H<;m8V!k}f~u7pn1o*yH}f%>YgHW7lav7)GART@xt) znd-vFi6`P5Dpy%2=$nGO4aCv1fk4v>K>4+sO*Z~5;~R|B`(;{Dv`oHd>Lh~Rr)$&~$#nO0wfUQq_?^(x(=k+(Vos$%OwF#K@Mka# zo;u{Ligavq&=)N(SnS>b{=C|PZ&#}?9A2-i63tN;!R)^Vp&RT=DqE0!aWaY+dX6sY z|F7o0GpwnmYgbW<6hWj{6$PXt(wiWV2aqNN5D-Lq@4YD~y{Hfn=^!n15JE?~^xg?o zI)oB>0y#VS`h3s(em~B4u5*4IF4FewJ+sz*-)rsJlgS=|`U_Kz%qvyXpSNMmn|WC%hsk!2dYkdZfy8Uq8W1G7EF|%HEr?JFb>hd*M*Nr$RDZg?Ti=MPVJM33vp-7 ze_f*)8mESQx9d=_^Wq=-?C5@YzilY?1M8#sqi)>Mr#$(O1^hL^Wah(8hwGwWMsr>23m={cm4`g$VWCUDK}_7z zc7jv?AzXETI=uDgglo$ShlTk2`uX{V-?=uO_`W!fB{{lnDD3hePY1p{O+GAK)Q#dL zJ+zlek6?0a^;&o`&W4GFTPTMeIj+UrQxcn%+$kM_R{u4+gxC$Km5OSyzG`vO5$U`1 zKw;XJfP?!S`n3MB{?pQ|)FgQd@cO1rUSVEGxB&T`i3!cuw{EyqtU+CGp5o&z-n(Hy zu^!X7%7w~hUNo>$Lo=pSQ-);{`aNX?uRwGipW^;FGFRfpNwq%)hW{TbgQprN_tp7?JE&UAL7{6^9E> zAIozb%GV%z!v!qHlLKw;FbX3C#xK8zKLTCPK zg6;CXg2ZcG0+YJ5?eNyx#oZ5Tp0YUBJnDPPLfN8Sj4JCd!5Rd7DE|^7}hg<7%rKk}*fiW}9aP zo9#X^?j43#Y|i+ftH$3F;UIhY0xx9P+LB`a)7@L#v=aI8xFyfbcT-IcEq5Xc!e=5A zpCXMGE-J$6PY)es87m0_-za5uyn37ov$}P^SbvL$(Te${d`I)i=Q?X<<5;m*ao7E$ z42(K9Zs%&8Jw9MZ3CjM+43U5eMy`Lh(bZWEPZ3fHPd!IH3PRc}UZ|1V5@ZoxsnJ2saZDTSRI|R4i-c{WqD&;ba&^wlx^zJJ3{PLYyay-^f<4lyT2rF zJ%&r=*}YdRm^S{+l!iSUir^{bp6M*MJWEJu1L3AkL>DJ)X>%!5;pZb1bwhVx@=Xaz zw>xv*S8e~x7`Q=zn6R@j+RrXbrj)#-!O$gSmi@3-SFMZKV(eSPDM~V>SiUo zV-J+qWM=#1g@s+${O(RzkoI+D+_U#2d9#dVBR~Ec=TIu(T4<4XcEbg0f4ou|E*)vR zGP*k8<-;g((96>1y>+V){59>br|u5#bXs3DdIyMg6rb=F`w?*Tnj%EM)4t?dMvu*M ziDlt89&_|ttZFdthRBpZ|K#-vnBa7{|FN1?i2i5{@)=Aw`sqiJQkF~eeBYDlDO+pu zpfa5)_8zg;L~mxq!C2;#=Cq8^?ph8$rM#gGa<@Tsc`Le!^wj4>pRZrNaU<$VWu-Pp zL3IWpx4BZ0_9(~Ilsm842;wFXpm?)OUmSacp~Q^T2cc#>F8LRS^Pd$PrlWyA`DD!{ z!*s?~*U=x75eBSH>``WvDmCKGVRS`*nbGQi=hb@(K+w-q(;@HhZYUN9v1_(lqIQ_RqtsRVl0Y*f8uZ zaGcNxzB1Q+-}=XXjzwZWGAk{@w#9DfboJ)%^qhKYvhw?dnDmK#uj1kvu6sB9xHrbV z@1A%W8%=Yxm8Eb~NE< zF!6ZOz|THbMjKFrB0sTRZBDJWmPp)l)+~;CGuKQhjj!Xcr4gK^9aI-^$B@S$uFQqs zha+%fuJU0thy7J;=VP9wh_6AX$8CNZQ~cb!J8$Xtk$8lpi`e zsK};Y^Y3^o@loVMTEVc}lM`8Tfdrck^31i#ptiV{H~~CsD~J;KNPS$emG#Thx&jL`KjYtj+hLgsdYBdB5?Pn#MNnj29uE$EI% ztM-l=5DV^*3E!u|%G3&vk7S^;-Ls^t3!-8oL{GqeKuJ2&G5i{dz zUc1s~_d7t#YlYeOFkJXm4o@OC)e7k;TrLian-39VU)CBw51ilJCnlg4ZFHs-@0E$9 zrHF}6E7Nx5^J1MT+>fM-3oX!UUVPcffBpV~8IFcs`XbsY+LVH9$86b|Jj3pMxpqU- zywBO#a!sTV7o}U~$oZvB<9c0EdPeYU;p5=}x-v>8^Vae)Q!2f)CP%`ER$&zPF&+Bz zD4a4%FDgV$P%%II8QnsJ0(t4dj2whXlr81y##x&u?of#z@c?g92Sckm`BR1;qP5ZQ zyK*-UUYW>lW4j3;+e5?W~-5Avt!(ZZvA)7izJ@uo>*A^Tn$2fkfUmnj4go| zx>_y$_$cTi!j1o^ER588swJ$?Wy*;8*+6vIb$MW|a=AZq0bqeCoVukgiH2o{raNlp z;2gPKe*U}%Z;c*o5)j0EuXnqtn5fXv!2Bkh^F{c`RE-}!9?!!8iSM<7YD8Dp#VO3>dFi#L+`~}?DrRVwMai5e{h3?fJBMYV>-PhiCy=okM`0PPcK9yB zz&I(CyYaaA-I?WIdbRHea<9@Tk(P-(gzPb$+jizuypLU6ewbRA9gMEV~u$a zlJv|;yp>|Ryk@=GTPT&HtcH0a3s&zc`Cz7+Yf&LK8#Tm4?k4!eOt2Hl$yZ0ezKh3f zL3t2UC@VYDHo~LYWAyQN4@YBDV6uFPy}g}ky#MT7)IDiyQN*r<=$+K#pia#uOCFbb zuSBtw+fFIL{Fye7W&Lj8pV+b1<;8W&r&Z{apT6~2%SY!rvAsT5HNEF@dfuv&cWe8G zrC?}&7CB#-0oXF6EV3lIGm{o$943G7vGElXkZXJEoqgp%_Irx=CgjM}^X~(v-#1^M z-z7w|V@`RhR=U4!Oj3X-g^ixxeYo|!i<>jUafLX#comef6SDWi?=3K4q+Rj>QkM_K z7(YhzIDB_tQbTh8krKl!EV;GjfhA|`k6G@6tc6Bj^uUc%p{*|Xn0ce>c``Niq@%;^ zTc2IpH!(7OW0UU>f9`i3`3hnL40&Kn>+^wOnrvhn&_%+`P^Aa=97(Q|J-DGkQ7v{= z1F#K8ns1HSY$uq59M7X!zo!U9_Cbq0^ZUv5A98e$1QS6PZ-^mLoc%BxyG#m83R_2w!P%J<=1bV*u$L5D~DZObmpUipP*=*)j z$HymW_}Q5lrEeY19Wn@{9`LU1Z)|mq{t%OB5!-`>;4c;6+bs9DSxGMxok*Dt#~Fum zP;zK2;OC^fONeBDc~2DU<$}U7jiu(ECVl-^iMhzk6pe)c9ix7u@0|KniK(Elm2Nm6h|5u3gS>)xEDg8*Z%n-AO5YM#m zRC0U-r0>5qBKvTk@)P?Noa4rfHcLS+VoT-Zknjk=D@f=Sg3NahOo*84Yw$lRyY^px zLkPXS+MsVT91aYPu-{e0!I=lXHrPvzikn>87#StP#9w$Rsq>p0TNI`^w2!fUnmdtJ znkQh>%9IRZ{v7UqOe_^(BS^x5@0;YhumQrqwRGQ!Rz@zH{MMEv zBH8ml$ThMp&2lz4QM#3i^z@NB*R+EcMt6uAD1WDm4(ANrzK`Q_e5jJhKp z?>-K%Lshl5<}JlPuqLn)59JqB>@_^(DX`Hx72axT0IoGMA?kZc*EM86$z7*F$2fZw zFFEct+&^Z1XPIyAG<;L3S_oet!s^y$%Ou&L?&j%4jdgd;&GN{04wJGjgNoW^uAXT{ zOG{=15$tvUiykQ2z|=lB(HI+{sQ%0v;}mPhXH;I_2$JHGbIR%qQ4UU2Ij?bh~CG z)1qWK6)K6B6dom5l$jvCXJ=IIbkqL2Xf%KL@ZtRj51bmv$;sKntp~>Im5L|g2!vA& zygnQb(-lvgmkrf>Ej%nvh#e0*5vvd3>Ou1!|BM?6FeKDj-ZAYQJzlhQRzuESIzzb> zf(408bwi_5LXP(lH}DZQ^iR~7N$$NW=33dXTl~El8usyCg--G-`G#Qsva6AV z{H0qz{Iw^%_Z*XX+iCI7Ku}zq1~1|9+jbgPcbQAF9RiuJ3d&2krUxo=2d-xpn~rX} ze*cJ?L5{^OD%P|wYA-#@3STI+$LQ!2KwzAMNl)a zGSW49d3jSaGbc%6ylX079?Rs8?1MBiegSo9yYd2*KmKkxU7@+C=&lW~MLW%x4WJkD z8^vn=O=Xe9tPbAcA5q$xzQOgiS2B07b3Jizg+Z!Tlxp1YPBrL@QVcuMF|xr~3iwuO z;4~X=oJqQWdfB%mekGqxH?{lZz)M6RjoIwyp02`8X^}?gXJ+5B#XClOL64;7^g24x zZMJ0x!v_=;>A!~LDbQ2NPrrd1d00Y>4orRuoQ%xA>@n+BL$36mG4ye* z;&Z5o+)itXAz+y=b9P@Nrz(LfyEjDtsJccw#Yq+P17fS~98+kbaX|H``~DYKhVJh? zWg^g;AI~-Ax@+6-?~1*jT34J&eY!r&MHbX%6j*f1SJBw}6{YF7|5JP{_w(g0Ekl0s znlsNgM*c62+&Jn)=&!|}RXA76oF`~a{TCDmx;#d>dvQxO+|5Z96}uR?%a?ACyNtj)-NwR}^QoiRH1%UPn=;zIDgBYPNzl z)D5lr79iw09PUVIGbw?jl$t8Z)=6XX9Ahw_3UT-#xq7$5BTSCXx${$zxzYf$dGoiJ z7u92FRcc2o_qyI#nBE^1bPRj8xFNYMGWQTC_vD)euM*}N`2YNCHF>~_2FLHA$D_jxQhP3I3NHe2cjXZ3sJ^~8OntiLND za!9?FkSzC)H`R;==*D?GLxcr7`Hv7h)mGR4sWc!yF}=^VvlDqJSK7;n=F=l?dE=9TzrTe$DK)^zlZGaO)Wh%a| z90_Hy9mA7`tt*8TYZlK}Cf6?w^>CSJx(0 zkd)SJPR-^VVpaXw>||F)eRsAjdcp$n)7BWDAw^ll8yb4E)$q0L3EAUS&uzfE zVvO>ZER_fi8JQO|oNmzI5(viacMH#SaoNTs#o+yW>^inuCb4F^jRw{vLq}OAt@{ja zLvBHQT;w33w*6UmcSRtW(U)7V7XYpD<1{B@Dzkn1tD@p z={;R2b<8TXO5HYOH|u?d?oQJyUQxEfE6{n6Wi1w&s)46e-VIz(J3AlQt|s%eKY0As z@TLBOF-4cSOQb6-CsH8QtU00LMP_w4fUAg1*92)cI&u zJVqF!|9x!C#TuTl&Ee2{8~$Y}9WSXS{2D#=ha4-Vv?{03_%-8#G4u=pZy* zbM5$@S7IJV*Q#kzywF|ms{4K|87?HJ3NYhIxW}UH6^=)`rLKY<<{m@h-5;kjZOd0) z(e;-sRlEn~SCZ!F&!@_H(_G>1pJUQW6H~Oe;<|g2Bw0M^ea9TNPtd!6&!5#!uf-Om ztgcsOYRTo&95%Xi8I`8k+{KHMW2OnsuBC&d&<6smO3YTEd4j?e7*T_q>ETaTQqh>> zh3jZ!P%$Zb3wd#YL8hU8m^!N<+MeOOeG_(m9HiZl``-6K zcdleYV0?RTRk&%a4~8IXX>}lzC14vwe~Fbuo=LCmN}>pJi&9V2-2)C39L`<6Is%UT zR9{F6yqIl(gXS@+!f^TKxpbz0kEkUX&Tts&iJObJ^7BY?Q6bUIW?9VR(wpMuk z^L3j1N_J60TPYRB=Jv)}HfgAnBFy3L8Ho2%kry4(^s|-cWVXo-;)YylGEFc%k@nsv znW)~26T~x`&(Oz=Hk}0}8C5qQ5{<$`kA;!jk!)@0kz09MI^kTnm z)^)}{ve9?OVy6+KR*wqu5cAryI`%u~_CKP=knYy)qzmox4_sPJ6rZODpogiwn$;TH zn$Bav#R1JE{cyh9vIKYE0bnm)r!8zc$k&eam^w;ws6Yvziu|TWijMne3PrZ^XHkmy z(g!yqB9aOx&n3+p%iL_nQK`!PWm{^}8!QfIE<$N*iWYX2+5~JZ$z-x3Z>@zac7kKk zkel#jMhEW-l7yzw*s3By-EzlOO8-Tp`k}8Uvvxb<5fZy*YRD|Ld2|1TTAR5oi-6;R zlj##91XMDsofqNOE`5nR(BN6rHRf<;h8e+J=;;`qyELNETlE`m6Qcb~y&F1GBg*}L zg&vnf1sBat{!yBjsD#?3pQ!pBk`kAmZKqVsu*GEQ@<9DY-|h)$w{dW4z$ZM46AzQE zS7{v4Wlw7C1dFLhNmtulR2uq^vAV?fYC9aa@&;5gyX(fd7mH4vT(l`01i%GVjWmnv zj=TI(X_ub@jz9^hGd2e2g{o`)XCDI=m@iV;^gizw?p{htqasT7qixcbrYwvW)0#F< zxTXfeDJFvO4w|9^AYc;AXKf&sS{D1*9!R}iw^lENrxE7u$ zZkUMY3LTODfZK*ginaf0sLGnrb_>{H?@avI@z}eEDjD^S`AAxV(yXhr>vnPdp#D5Y zai*!Xe{0@5$pYiiv>MDQMC&+bl~jL_*3f~n_x|LP>YnQL3sB&R$=!ySQ(L5e*195M zUuOP0o*?H=-gBY?-3Uge!Nzg*t=<9Aj@}k4q`6J0CIM-M48dKA%$v14vCPO7?gX#R zvjWMu29JP_t$^i;ixBCik<1dhlPOqTUkTh4NxM}@cPv&q$>g~H!AYpX+P~lF7`{nr zc+})I(}!XdY;w+uqVJh&@Y%t1->jRkciQZ-iEFi<3yy{9)>*`$NO{it1*e_XxTOLd zgLeI9X_=H_Lh4T{OZEaT7LBT#xHp1sZi~LsZ5TgyW1I3EzKVa$JltsQG}LH*(T&?F z>IgeQ*Uca53;VZ}9dE{zz#9xH`vm`f$xsV7QS4pn{^tvI7Uowte!N&17_XIx6)k0Tws1N$Z`LU zUASj#$!yaMBE{ou8FmRm!KB(nbNgHz6Z9RLS}#x_D(!V%zZUxWJ@>TvvYt7~I^Rp+ zerWsR+GoW`HP}XrU4Jd+_S{K^+GLqcjHLLk>qv<7Ca1nC%&jkQSJ!XwnAgAg?fZ&k zk72z|T55Q}0ApkQvcE|Jm?#U@v^%)R zXUxoQ;{f8NxbL=fQ%|2;e&@^5?-_oGTBOj)wp1IMZfgy3lqy_56yne{fsm}*souFZ zcV1j)a2S^>f(mqbS-HmD(+u)MW;NyNaoIAI6`=q%1XuEIq{lB7G7fUzV94eA^x~+2}CWA;?*%dvSeh zjdZfnC8i1P8N)h&+1N_1AQx`x zi!i#xliruc*kF##ay`c*{AW82&%7~96H=SoQ9=4mo(U0`<48Zu?f!Cik@oq9#ixcR z_?TgCW{FN#ak%%uf~YY1rG`Mj`AZpC(<+Jo(S(E3c&Wr8=On9ursY=t{wtysJn)!Hppkcbee_oQ=sFNGI>PxK6>8mE) z_uEzHONYy>g&Z#81zx%iVLx_HYxcWsnDAB^8=Loat<`RR*^*f6qSoc-wC-12Gu#{| z8u;eE5~yNfwE1G<xG(oLytFVP2?K&RW3 z_!p9J<{9@}tr@K@Rlsy>UN`3mz|I;iq&s7oeR3cI^S6R)Aga^eIYhi7V}T{s)+^Q) z2A*X|Rp~DD!c{?H?IP)^8fwF(SxvQa^9w3!$<_V-*>BV6FMT|I2c*@AFFh&c?o2_I z`he}(2_jZU^@40^TU`US{grAGGqdWwt3~t9Im%UbF6Y6N)?-vLUTBl#M&ss|>AtF$ zPEpL>{hBF_wa>;=-NSmSQp|R`A=(aNaIEbXe!C`i27eNIOREWn8#Z~7%j}#i?%=e5 zc-i4|EVj3{3tLWM9uJ{nv{^Z$)}0j7oAruV-bTj(QMTRCv$TWhdOIV%)4V!R-Hm^_ zbkpZ}d;d)kYUfv|GaN?G_-zkd_F6QR!h>G=9gS%h8$3iul{?H!A(6-qr*9SmOeb)U zG>3LHx?WRNw0(9qIYilRsyzEW8Iu=$M;E@6hj1?h{^=d+haftJ8;hkj6Fea|8O!}I zy#JUkOK{SFWJGua-5qT^&(+HDfPuUCje9rep<_Dry_`lE;~AL4Ns;$ikCFXyM^wu^ zySV#ysrQjJJ}J{TKa@Lc&bz>&eRHhfrT0cQVByB%qu(8eKMk~$1E()+fA7yzJjs8( z$G$7IRsFjR_$=|xP0gpaQmwWd23=hnjigLy{0@Vf6ZxO#mJGui!9+wvvZok_|2XDL zlBSExWvzA9LWzm7u_{4K^8dEgf0%?d!z}3EH*nGa(`Z=&e2af-Hi~rq(q(6P--z`aS*v{2YDYB*f{D|#G{ciSGqn=uM9N?9Y|L-mM{{V#l za})jpo6)Hnm)rqAgiB*83B8aA28~Q%clif+0D_ORA%s)Mdnm&2 zCmK26k@&MN>_3jL7Iz)*{WX%A{fMJ+Es!rH;r)Kn{+P(C6TP)O>o3rDbcX z+$=(;eu=VQ0{Lufya-vI(9lb%W3VWP<+t;5pZUKXl9-kzOdJ&QD5($KJAjT%a%h~o zopvfWe-p5Ikp>8T^NPfOaNj6NMMc%y*LPNc6~}5{l3yM6ylg_;nqIdvUD=Z=>BGgt zvuXbiu}b(tyMbgeR-mI_X$^rMSUv&}i`D=G6${#o>()4DJ2Cv}DgfZ@?a>D}+X@K~ z2mNbTrmxUmbAeWDXY7w5i*5NFF$!1?jI17HAb!0&XhrwfUDwl`}s%& z)j#U;yx!;a0odaC&K33#Yg<@s$z(lU*X>Ei|E=YDbD(9Gk~%+uTK#`AbUg`cH7Mgh z>K@Ifw%_9Lwv7U$TTB058{AO_)&R5al)CE{aKP<=jrq=PB0_p3LFHecbx&q>yiove z*vHxjAG}e>(6H2k$ch)vXn?aB$BUq@Fz6%q0*?0xp?nUw3WTK=Ou`CR*ytrdyl_Fks%!Rj@Y$7y=!>x#g|}rtQ=K zi$bBUBa&aVD8Z~|U6U#>=rA0rw8&8tQY8l6Y zasZ^AP>v1-4Z!*Gulde$!rh+~hE&d=<%MpV0HQmol36G7wG&@wjCY!$-Cyg#5QQ^6 ztdH-@X3CWEGr2eH;YIufGFjmWAu9@~lJWNVPHB!(MqC*PY&P6y%O2&#prWLqC7M~s z{Schp`O#TdgBW1RDvpKDOcBckv|J?Gyfu9Y4ph$uffHR} z%aw}Q7w9m1>@dJO0;mk&OjRS^RlLv**1<>o80;#ZS@|4`h?{s}{_~p!!^?TlXLL^? z3Ncv^vb(R<&U}Tc*MgLlD8XjiIr*6y@qlfBu$HSWY0PV_im)bETRs=e<5(6EgVvEe zwTe4X2N1~`Q~_rUVQC(FGd%s3Yr`I#4JdP2Yr{zk%%m<411d32PvVGe5vdvv<^TY3 z6<#Um{yL}!9*P=_JhIF*{uA#W;H{7MsU}@S@%27=Xblt`)jI z4rc{4;uZtN1EdM3n05;0up5Rd35UxHgQM5&Tlv=Ab;Y4T`T$Y@ZJkQ6aXKI)?2*rE zlkB6Qj~-_GXn|`1`PS-th(nFXySg+exOg#g(fxr-aAi-qUsr-&)fE2(lY|Gd*gOx8( z6%gV8toBd_Ai#cU+r$ml#Yc<{f4pA_^4G4$G_Ek90Qv~XEi6hgEbZz+CY7zVV9NJ7 z0QbL=79b$32r_9qKW0*Z0m3Su$F(W}@id(GI&Jpn zAj+eGQ-LMC$XZ=q7#lJ$fFpk5>>1;zgIB+)9|14}Qhq*1EJsHfpmtLlydFIuAv0(L z86Ci}8(=vT7Hz@{yYRz4taPiTVU9|IW|fB}ee&9E1i8AA1(TW^W)Xi$r5NX9XO|0i=p`!!HWR=20dE&bmPCKxHW5zr;Wo6R>UK kd%l{)DB}18#&RiL-6}9|=4exV1^82xSCuQ4eHHk>09z#yu>b%7 diff --git a/docs/offline/reader/carbondatareader.md b/docs/offline/reader/carbondatareader.md index 7c3ab76cc7..d8a1d8f34b 100644 --- a/docs/offline/reader/carbondatareader.md +++ b/docs/offline/reader/carbondatareader.md @@ -2,7 +2,7 @@ ## 一、插件名称 -名称:**carbondatareader**
** +名称:**carbondatareader**
## 二、支持的数据源版本 **Carbondata 1.5及以上**
@@ -13,30 +13,34 @@ - **path** - 描述:carbondata表的存储路径 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **table** - 描述:carbondata表名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **database** - 描述:carbondata库名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **filter** - 描述:简单过滤器,目前只支持单条件的简单过滤,形式为 col op value,col为列名;op为关系运算符,包括=,>,>=,<,<=; value为字面值,如1234, "ssss" - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** @@ -56,22 +60,25 @@ value为字面值,如1234, "ssss" - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **hadoopConfig** - 描述:集群HA模式时需要填写的namespace配置及其它配置 - 必选:是 + - 字段类型:Map - 默认值:无 - +
- **defaultFS** - 描述:Hadoop hdfs文件系统namenode节点地址。 - 必选:是 + - 字段类型:String - 默认值:无 - +
diff --git a/docs/offline/reader/cassandrareader.md b/docs/offline/reader/cassandrareader.md index 4291d6f435..0881796853 100644 --- a/docs/offline/reader/cassandrareader.md +++ b/docs/offline/reader/cassandrareader.md @@ -15,91 +15,103 @@ - 描述:数据库地址 - 必选:是 - 默认值:无 - + - 字段类型:String +
- **port** - 描述:端口 - 必选:否 - 默认值:9042 - + - 字段类型:Integer +
- **username** - 描述:用户名 - 必选:否 - 默认值:无 - - + - 字段类型:String +
- **password** - 描述:密码 - 必选:否 - 默认值:无 - + - 字段类型:String +
- **useSSL** - 描述:数字证书 - 必选:否 - 默认值:false - + - 字段类型:boolean +
- **column** - 描述:查询结果中被select出来的属性集合,为空则select * - 必选:否 - 默认值:无 - + - 字段类型:List +
- **keyspace** - 描述:需要同步的表所在的keyspace - 必选:是 - 默认值:无 - + - 字段类型:String +
- **table** - 描述:要查询的表 - 必选:是 - 默认值:无 - + - 字段类型:String +
- **where** - 描述:过滤条件where之后的表达式 - 必选:否 - 默认值:无 - + - 字段类型:String +
- **allowFiltering** - 描述:是否在服务端过滤数据 - 必选:否 - 默认值:false - + - 字段类型:boolean +
- **connecttionsPerHost** - 描述:分配给每个host的连接数 - 必选:否 - 默认值:8 - + - 字段类型:Integer +
- **maxPendingPerConnection** - 描述:最多能建立的连接数 - 必选:否 - 默认值:128 - + - 字段类型:Integer +
- **consistancyLevel** - 描述:数据一致性级别。可选`ONE`、`QUORUM`、`LOCAL_QUORUM`、`EACH_QUORUM`、`ALL`、`ANY`、`TWO`、`THREE`、`LOCAL_ONE` - 必选:否 - 默认值:无 - + - 字段类型:String +
diff --git a/docs/offline/reader/clickhousereader.md b/docs/offline/reader/clickhousereader.md index 67218c00dd..fadd2fa69c 100644 --- a/docs/offline/reader/clickhousereader.md +++ b/docs/offline/reader/clickhousereader.md @@ -9,62 +9,101 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:clickhouse://localhost:8123/database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 -
jdbcUrl参考文档:[clickhouse-jdbc官方文档](https://github.com/ClickHouse/clickhouse-jdbc) +
jdbcUrl参考文档:[clickhouse-jdbc官方文档](https://github.com/ClickHouse/clickhouse-jdbc) - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ + - **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000 +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - 注意: - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,flinkx将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **fetchSize** - - 描述:读取时每批次读取的数据条数。 - - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - - 必选:否 - - 默认值:1000 - - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -74,14 +113,16 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 - 格式:支持3种格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: +
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + ```bash "column":["*"] ``` @@ -99,35 +140,63 @@ }] ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果数据库里不存在指定的字段,则会报错。如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - - 必选:是 - - 默认值:无 - + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会报错。如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false + +
+- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 - + ## 四、配置示例 #### 1、基础配置 @@ -139,21 +208,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -194,21 +260,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -249,21 +312,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -304,21 +364,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -361,21 +418,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/dtstack" ], + "jdbcUrl" : [ "jdbc:clickhouse://0.0.0.1:8123/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", diff --git a/docs/offline/reader/db2reader.md b/docs/offline/reader/db2reader.md index d04953cf60..dcd9c69b9a 100644 --- a/docs/offline/reader/db2reader.md +++ b/docs/offline/reader/db2reader.md @@ -10,54 +10,101 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:db2://localhost:50000/database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[db2官方文档](https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.1.0/com.ibm.db2.luw.apdv.java.doc/src/tpc/imjcc_rjv00004.htmlId=t14:12:14) - 必选:是 + - 字段类型:List - 默认值:无 + +
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ - **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000 +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - 注意: - 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 - - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! + - 目前splitPk仅支持整形数据切分,不支持浮点、字符串、日期等其他类型。如果用户指定其他非支持类型,flinkx将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:空 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -67,9 +114,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:空 - +
- **column** - 描述:需要读取的字段。 @@ -98,27 +146,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false +
- +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ - **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ** @@ -134,18 +210,16 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" },{ "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], - "username" : "user", + "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], - "table" : [ "staff" ] + "jdbcUrl" : [ "jdbc:db2://localhost:50000/database" ], + "table" : [ "table" ] } ], "where": "id > 1", "splitPk": "id", @@ -192,18 +266,16 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" },{ "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], - "username" : "user", + "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], - "table" : [ "staff" ] + "jdbcUrl" : [ "jdbc:db2://localhost:50000/database" ], + "table" : [ "table" ] } ], "where": "id > 1", "splitPk": "id", @@ -250,24 +322,22 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" },{ "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], - "username" : "user", + "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], - "table" : [ "staff" ] + "jdbcUrl" : [ "jdbc:db2://localhost:50000/database" ], + "table" : [ "table" ] } ], "where": "id > 1", "splitPk": "id", "fetchSize": 1000, "queryTimeOut": 1000, - "customSql":"select id, name from staff where id > 300", + "customSql":"select id, name from table where id > 300", "requestAccumulatorInterval": 2 }, "name" : "db2reader" @@ -308,18 +378,16 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" },{ "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], - "username" : "user", + "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], - "table" : [ "staff" ] + "jdbcUrl" : [ "jdbc:db2://localhost:50000/database" ], + "table" : [ "table" ] } ], "where": "id > 1", "splitPk": "id", @@ -368,18 +436,16 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" },{ "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], - "username" : "user", + "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:db2://localhost:50000/sample" ], - "table" : [ "staff" ] + "jdbcUrl" : [ "jdbc:db2://localhost:50000/database" ], + "table" : [ "table" ] } ], "where": "id > 1", "splitPk": "id", diff --git a/docs/offline/reader/dmreader.md b/docs/offline/reader/dmreader.md index 79639771f2..6220b4ce0b 100644 --- a/docs/offline/reader/dmreader.md +++ b/docs/offline/reader/dmreader.md @@ -10,35 +10,80 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 -
jdbcUrl参考文档:[达梦官方文档](http://www.dameng.com/down.aspx?TypeId=12&FId=t14:12:14) +
jdbcUrl参考文档:[达梦官方文档](http://www.dameng.com/down.aspx?TypeId=12&FId=t14:12:14) - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000 +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -47,17 +92,19 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:3000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -67,9 +114,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -98,27 +146,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false +
+- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ## 四、配置示例 @@ -144,16 +220,12 @@ ], "increColumn": "", "startLocation": "", - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": [ - "jdbc:dm://localhost:5236" - ], - "table": [ - "PERSON.STUDENT" - ] + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] } ], "where": "" @@ -208,16 +280,12 @@ "splitPk": "ID", "increColumn": "", "startLocation": "", - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": [ - "jdbc:dm://localhost:5236" - ], - "table": [ - "PERSON.STUDENT" - ] + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] } ], "where": "" @@ -271,17 +339,13 @@ ], "increColumn": "", "startLocation": "", - "customSql": "SELECT * FROM PERSON.STUDENT WHERE ID>30", - "username": "SYSDBA", - "password": "SYSDBA", + "customSql": "SELECT * FROM TABLE WHERE ID>30", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": [ - "jdbc:dm://localhost:5236" - ], - "table": [ - "PERSON.STUDENT" - ] + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] } ], "where": "" @@ -335,16 +399,12 @@ ], "increColumn": "ID", "startLocation": "20", - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": [ - "jdbc:dm://localhost:5236" - ], - "table": [ - "PERSON.STUDENT" - ] + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] } ], "where": "" @@ -398,18 +458,14 @@ ], "increColumn": "", "startLocation": "", - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "polling": true, "pollingInterval": 3000, "connection": [ { - "jdbcUrl": [ - "jdbc:dm://localhost:5236" - ], - "table": [ - "PERSON.STUDENT" - ] + "jdbcUrl": ["jdbc:dm://localhost:5236"], + "table": ["TABLE"] } ], "where": "" diff --git a/docs/offline/reader/esreader.md b/docs/offline/reader/esreader.md index c9916083d5..bd36c06a9d 100644 --- a/docs/offline/reader/esreader.md +++ b/docs/offline/reader/esreader.md @@ -12,6 +12,7 @@ - **address** - 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 - 必选:是 + - 字段类型:String - 默认值:无 @@ -19,6 +20,7 @@ - **username** - 描述:Elasticsearch认证用户名 - 必选:否 + - 字段类型:String - 默认值:无 @@ -26,6 +28,7 @@ - **password** - 描述:Elasticsearch认证密码 - 必选:否 + - 字段类型:String - 默认值:无 @@ -33,6 +36,7 @@ - **query** - 描述:Elasticsearch查询表达式,[查询表达式](https://www.elastic.co/guide/cn/elasticsearch/guide/current/query-dsl-intro.html) - 必选:否 + - 字段类型:json结构体 - 默认值:无,默认为全查询 @@ -40,6 +44,7 @@ - **batchSize** - 描述:每次读取数据条数 - 必选:否 + - 字段类型:int - 默认值:10 @@ -47,29 +52,46 @@ - **timeout** - 描述:连接超时时间 - 必选:否 + - 字段类型:int - 默认值:无 - **index** - - 描述:要查询的索引名称 - - 必选:否 + - 描述:要查询的索引名称,支持String和String[]两种类型 + - 必选:是 + - 字段类型:可以为String或者String[] - 默认值:无 - **type** - - 描述:要查询的类型 - - 必选:否 + - 描述:要查询的类型,支持String和String[]两种类型 + - 必选:是 + - 字段类型:可以为String或者String[] - 默认值:无 - **column** - 描述:读取elasticsearch的查询结果的若干个列,每列形式如下 - - name:字段名称,可使用多级格式查找 + - name:字段名称,可使用多级格式查找,多级查询时采用'.'作为间隔 - type:字段类型,当name没有指定时,则返回常量列,值为value指定 - - value:常量列的值 + - value:常量列的值 + 示例: + ```json + "column": [ + { + "name": "id", + "type": "integer" + },{ + "name": "user_id", + "type": "integer" + },{ + "name": "name", + "type": "string" + } +``` - 必选:是 - 默认值:无 @@ -84,7 +106,7 @@ "reader": { "name": "esreader", "parameter": { - "address": "kudu4:9200", + "address": "localhost:9200", "query": { "match_all": {} }, diff --git a/docs/offline/reader/ftpreader.md b/docs/offline/reader/ftpreader.md index e687c6e84d..3bb168fc71 100644 --- a/docs/offline/reader/ftpreader.md +++ b/docs/offline/reader/ftpreader.md @@ -1,10 +1,9 @@ # FTP Reader - ## 一、插件名称 -名称:**ftpreader**
+名称:**ftpreader** + - ## 二、数据源版本 | 协议 | 是否支持 | | --- | --- | @@ -13,94 +12,102 @@ - ## 三、数据源配置 -FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
+FTP服务搭建 +windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z) +linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf) +sftp服务搭建 +windows:[地址](http://www.freesshd.com/) +linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7) + - ## 四、参数说明 - **protocol** - 描述:ftp服务器协议,目前支持传输协议有`ftp`、`sftp` - 必选:是 + - 字段类型:string - 默认值:无 - +
- **host** - 描述:ftp服务器地址 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **port** - 描述:ftp服务器端口 - 必选:否 + - 字段类型:int - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 - - -- **connectPattern** - - 描述:协议为ftp时的连接模式,可选`pasv`,`port`,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) - - 必选:否 - - 默认值:`PASV` - - +
- **username** - 描述:ftp服务器访问用户名 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **password** - 描述:ftp服务器访问密码 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **path** - - 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径 + - 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径,多个路径以`,`隔开 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **fieldDelimiter** - 描述:读取的字段分隔符 - 必选:是 + - 字段类型:string - 默认值:`,` - +
- **encoding** - 描述:读取文件的编码配置 - 必选:否 + - 字段类型:string - 默认值:`UTF-8` - +
- **isFirstLineHeader** - 描述:首行是否为标题行,如果是则不读取第一行 - 必选:否 + - 字段类型:boolean - 默认值:false - +
- **timeout** - 描述:连接超时时间,单位毫秒 - 必选:否 + - 字段类型:long - 默认值:5000 - +
- **column** - 描述:需要读取的字段 - - 格式:支持2中格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + - 格式:支持2种格式 + +1.读取全部字段,如果字段数量很多,可以使用下面的写法: ``` "column":["*"] ``` @@ -115,20 +122,34 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ }] ``` - - 属性说明: - - index:字段索引 - - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果没有指定index,则会把value的值作为常量列返回,如果指定了index,当读取的字段的值为null时,会以此value值作为默认值返回 - - 必选:是 - - 默认值:无 +- 属性说明: + - index:字段索引 + - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果没有指定index,则会把value的值作为常量列返回,如果指定了index,当读取的字段的值为null时,会以此value值作为默认值返回 +- 必选:是 +- 字段类型:数组 +- 默认值:无 + +
+- **connectPattern** + - 描述:协议为ftp时的连接模式,可选`pasv`,`port`,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) + - 必选:否 + - 字段类型:string + - 默认值:`PASV` +
+ +- **privateKeyPath** + - 描述:私钥文件路径 + - 必选:否 + - 字段类型:string + - 默认值:无 - + ## 五、使用示例 - -#### 1、读取单个文件 +#### 1、sftp读取单个文件 ```json { "job": { @@ -190,8 +211,7 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ } } ``` - -#### 2、读取单个目录下的所有文件 +#### 2、sftp读取单个目录下的所有文件 ```json { "job": { @@ -236,25 +256,14 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ } ], "setting": { - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "errorLimit": { - "record": 100 - }, "speed": { - "bytes": 0, "channel": 1 } } } } ``` - -#### 3、读取多个路径下的文件 +#### 3、sftp读取多个路径下的文件 ```json { "job": { @@ -266,7 +275,7 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ "protocol": "sftp", "port": 22, "isFirstLineHeader": true, - "host": "localhost", + "host": "host", "column": [ { "index": 0, @@ -299,22 +308,59 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ } ], "setting": { - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "errorLimit": { - "record": 100 - }, "speed": { - "bytes": 0, "channel": 1 } } } } ``` - - +#### 4、ftp读取单个文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "isFirstLineHeader": false, + "column": [ + { + "index": 0, + "type": "STRING", + "key": 0 + }, + { + "index": 1, + "type": "STRING", + "key": 1 + } + ], + "fieldDelimiter": ",", + "encoding": "utf-8", + "path": "/data/a.csv", + "protocol": "ftp", + "password": "passwd", + "connectMode": "PORT", + "port": 21, + "host": "host", + "username": "usname" + }, + "name": "ftpreader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "channel": 1 + } + } + } +} +``` diff --git a/docs/offline/reader/gbasereader.md b/docs/offline/reader/gbasereader.md index 605a762f0e..efd08e70bd 100644 --- a/docs/offline/reader/gbasereader.md +++ b/docs/offline/reader/gbasereader.md @@ -10,34 +10,70 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:gbase://0.0.0.1:5258/database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串,需要注意gbase有A、S、T三种发行版,jdbcUrl端口和驱动都有区别。
jdbcUrl参考文档:[gbase官方文档](https://help.finereport.com/doc-view-2569.html) - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -46,25 +82,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - - 描述:读取时每批次读取的数据条数。 + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 - - 默认值:1000 - + - 字段类型:int + - 默认值:Integer.MIN_VALUE +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -74,9 +113,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -105,27 +145,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false +
- +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ - **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ## 四、配置示例 @@ -139,21 +207,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -165,10 +230,6 @@ }, "name" : "gbasereader" }, - "writer": { - - } - }], "writer": { "name": "streamwriter", "parameter": { @@ -198,21 +259,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -230,13 +288,6 @@ "print": true } } - }], - "writer": { - "name": "streamwriter", - "parameter": { - "print": true - } - } }], "setting": { "speed": { @@ -260,21 +311,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -292,13 +340,6 @@ "print": true } } - }], - "writer": { - "name": "streamwriter", - "parameter": { - "print": true - } - } }], "setting": { "speed": { @@ -322,21 +363,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -350,9 +388,6 @@ }, "name" : "gbasereader" }, - "writer": { - } - }], "writer": { "name": "streamwriter", "parameter": { @@ -382,21 +417,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/dtstack" ], + "jdbcUrl" : [ "jdbc:gbase://0.0.0.1:5258/database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -410,9 +442,6 @@ }, "name" : "gbasereader" }, - "writer": { - } - }], "writer": { "name": "streamwriter", "parameter": { diff --git a/docs/offline/reader/greenplumreader.md b/docs/offline/reader/greenplumreader.md index a8d89386e7..b8bdd841f2 100644 --- a/docs/offline/reader/greenplumreader.md +++ b/docs/offline/reader/greenplumreader.md @@ -10,34 +10,79 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[greenplum官方文档](https://gpdb.docs.pivotal.io/590/datadirect/datadirect_jdbc.html) - 必选:是 + - 字段类型:List - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:0(不开启) +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -46,25 +91,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - 描述:读取时每批次读取的数据条数。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -74,9 +122,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -104,27 +153,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false +
+- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ** @@ -139,11 +216,11 @@ "reader": { "parameter" : { "column" : [ {"name" : "id", "type": "int"}], - "username" : "gpadmin", - "password" : "gpadmin", + "username" : "username", + "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], - "table" : [ "performance" ] + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database" ], + "table" : [ "table" ] } ], "where": "", "customSql": "", @@ -179,13 +256,14 @@ "reader": { "parameter" : { "column" : [ {"name" : "id", "type": "int"}], - "username" : "gpadmin", - "password" : "gpadmin", + "username" : "username", + "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], - "table" : [ "performance" ] + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database" ], + "table" : [ "table" ] } ], "where": "", + "splitPk":"id", "customSql": "", "requestAccumulatorInterval": 2 }, @@ -219,14 +297,14 @@ "reader": { "parameter" : { "column" : [ {"name" : "id", "type": "int"}], - "username" : "gpadmin", - "password" : "gpadmin", + "username" : "username", + "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], - "table" : [ "performance" ] + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database" ], + "table" : [ "table" ] } ], "where": "", - "customSql": "select id from performance", + "customSql": "select id from table", "requestAccumulatorInterval": 2 }, "name" : "greenplumreader" @@ -259,11 +337,11 @@ "reader": { "parameter" : { "column" : [ {"name" : "id", "type": "int"}], - "username" : "gpadmin", - "password" : "gpadmin", + "username" : "username", + "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], - "table" : [ "performance" ] + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database" ], + "table" : [ "table" ] } ], "increColumn": "id", "startLocation": "20", @@ -301,10 +379,10 @@ "reader": { "parameter" : { "column" : [ {"name" : "id", "type": "int"}], - "username" : "gpadmin", - "password" : "gpadmin", + "username" : "username", + "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb" ], + "jdbcUrl" : [ "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database" ], "table" : [ "performance" ] } ], "polling": true, diff --git a/docs/offline/reader/hbasereader.md b/docs/offline/reader/hbasereader.md index 5d4688aecc..d88dd4b133 100644 --- a/docs/offline/reader/hbasereader.md +++ b/docs/offline/reader/hbasereader.md @@ -5,82 +5,103 @@ 名称:**hbasereader** ## 二、支持的数据源版本 -**HBase 1.3及以上** +**HBase 1.2及以上** ## 三、参数说明 - **table** - 描述:hbase表名 - 必选:是 + - 字段类型:String - 默认值:无 - - +
- **hbaseConfig** - - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: - -Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal - + - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml) + - 基础配置 + ``` + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase" + ``` + + - kerberos配置 + 在hbaseConfig中加入以下三条中的任一条即表明开启Kerberos配置: + ``` + "hbase.security.authentication" :"Kerberos", + "hbase.security.authorization" : "Kerberos", + "hbase.security.auth.enable" : true + ``` + 在开启kerberos后,需要根据自己的集群指定以下两个principal的value值 + ``` + "hbase.regionserver.kerberos.principal":"hbase/_HOST@DTSTACK.COM", + "hbase.master.kerberos.principal":"hbase/_HOST@DTSTACK.COM" + ``` + 还需要指定Kerberos相关文件的位置 + ``` + "principalFile": "path of keytab", + "java.security.krb5.conf": "path of krb5.conf" + ``` - 必选:是 + - 字段类型:Map - 默认值:无 - - +
+ - **range** - 描述:指定hbasereader读取的rowkey范围。 - startRowkey:指定开始rowkey; - endRowkey:指定结束rowkey; - - - - isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false,若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey),配置格式如下: -``` -"range": { - "startRowkey": "aaa", - "endRowkey": "ccc", - "isBinaryRowkey":false -} -``` - - + - isBinaryRowkey:指定配置的startRowkey和endRowkey转换为byte[]时的方式,默认值为false。 + 若为true,则调用Bytes.toBytesBinary(rowkey)方法进行转换;若为false:则调用Bytes.toBytes(rowkey) + 配置格式如下: + ``` + "range": { + "startRowkey": "aaa", + "endRowkey": "ccc", + "isBinaryRowkey":false + } + ``` - 注意:如果用户配置了 startRowkey 和 endRowkey,需要确保:startRowkey <= endRowkey - 必选:否 + - 字段类型:Map - 默认值:无 - - + +
- **encoding** - 描述:字符编码 - 必选:无 - - 默认值:无 - - + - 字段类型:String + - 默认值:UTF-8 + +
- **scanCacheSize** - - 描述:一次RPC请求批量读取的Results数量 + - 描述:一次RPC请求批量读取的Results数量。cache值得设置并不是越大越好,需要做一个平衡。cache的值越大,则查询的性能就越高,但是与此同时,每一次调用next()操作都需要花费更长的时间,因为获取的数据更多并且数据量大了传输到客户端需要的时间就越长,一旦超过了maximum heap the client process 拥有的值,就会报outofmemoryException异常。当传输rows数据到客户端的时候,如果花费时间过长,则会抛出ScannerTimeOutException异常。 - 必选:无 + - 字段类型:String - 默认值:256 - -
- -- **scanBatchSize** - - 描述:每一个result中的列的数量 - - 必选:无 - - 默认值:100 - -
+ +
- **column** - - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; + - 描述:要读取的hbase字段 + - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式,注意rowkey区分大小写; - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 - 必选:是 + - 字段类型:List - 默认值:无 - + +

## 四、配置示例 +未开启Kerberos的情况 ```json { "job": { @@ -148,5 +169,80 @@ Kerberos;
hbase.security.authentication;
hbase.security.authorizat } } ``` + +开启kerberos的情况 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "hbasereader", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase", + "hbase.security.auth.enable": true, + "hbase.regionserver.kerberos.principal":"hbase/host@DTSTACK.COM", + "hbase.master.kerberos.principal":"hbase/host@DTSTACK.COM", + "principalFile": "path of keytab", + "useLocalFile": "true", + "java.security.krb5.conf": "path of krb5.conf" + }, + "table": "sb5", + "encodig": "utf-8", + "column": [ + { + "name": "rowkey", + "type": "string" + }, + { + "name": "cf1:id", + "type": "string" + } + ], + "range": { + "startRowkey": "", + "endRowkey": "", + "isBinaryRowkey": false + } + } + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream": false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} +``` # diff --git a/docs/offline/reader/hdfsreader.md b/docs/offline/reader/hdfsreader.md index 5b4e92ffa6..88ae4b9772 100644 --- a/docs/offline/reader/hdfsreader.md +++ b/docs/offline/reader/hdfsreader.md @@ -1,10 +1,9 @@ # HDFS Reader - ## 一、插件名称 -名称:**hdfsreader**
+名称:**hdfsreader** + - ## 二、支持的数据源版本 | 协议 | 是否支持 | | --- | --- | @@ -13,40 +12,54 @@ - ## 三、数据源配置 -单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html)
集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html)
+单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html) +集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html) + - ## 四、参数说明 - **defaultFS** - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 - 必选:是 + - 参数类型:string - 默认值:无 - +
- **hadoopConfig** - 描述:集群HA模式时需要填写的namespace配置及其它配置 - 必选:否 + - 参数类型:map - 默认值:无 - +
- **path** - 描述:数据文件的路径 + - 注意:真正读取的文件路径是 path+fileName - 必选:是 + - 参数类型:string - 默认值:无 +
+ +- **fileName** + - 描述:数据文件目录名称 + - 注意:不为空,则hdfs读取的路径为 path+filename + - 必选:否 + - 参数类型:string + - 默认值:无 +
- **filterRegex** - - 描述:文件过滤正则表达式 + - 描述:文件正则表达式,读取匹配到的文件 - 必选:否 + - 参数类型:string - 默认值:无 - +
- **fileType** - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` @@ -54,138 +67,142 @@ - orc:orcfile文件格式 - parquet:parquet文件格式 - 必选:否 + - 参数类型:string - 默认值:text - +
- **fieldDelimiter** - 描述:`fileType`为`text`时字段的分隔符 - 必选:否 + - 参数类型:string - 默认值:`\001` +
+ +- **column** + - 描述:需要读取的字段 + - 注意:不支持*格式 + - 格式: +```json +"column": [{ + "name": "col", + "type": "datetime", + "index":1, + "isPart":false, + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` +- 属性说明: + - name:必选,字段名称 + - type:必选,字段类型,可以和文件里的字段类型不一样,程序会做一次类型转换 + - index:非必选,字段在所有字段里的位置 从0开始计算,默认为-1 + - isPart:非必选,是否是分区字段,如果是分区字段,会自动从path上截取分区赋值,默认为fale + - format:非必选,如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:非必选,如果文件里不存在指定的字段,则会把value的值作为常量列返回 +- 必选:是 +- 参数类型:数组 +- 默认值:无 - ## 五、使用示例 - #### 1、读取text文件 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "path": "hdfs://ns1/flinkx/text", - "defaultFS": "hdfs://ns1", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - }, - { - "name": "col3", - "index": 2, - "type": "int" - }, - { - "name": "col4", - "index": 3, - "type": "int" - } - ], - "fieldDelimiter": ",", - "fileType": "text" - }, - "name": "hdfsreader" - }, - "writer": { - "parameter": {}, - "name": "streamwriter" - } - } - ], - "setting": { - "speed": { - "bytes": 0, - "channel": 1 - } + "job": { + "content": [ + { "reader" : { + "parameter" : { + "path" : "/user/hive/warehouse/dev.db/merge_text", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "host1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "host2:9000", + "dfs.nameservices" : "ns1" + }, + "column" : [ { + "name": "col1", + "index" : 0, + "type" : "STRING" + }, { + "name": "col2", + "index" : 1, + "type" : "STRING" + } ], + "defaultFS" : "hdfs://ns1", + "fieldDelimiter" : "\u0001", + "fileType" : "text" + }, + "name" : "hdfsreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } } + } + ], + "setting": { + "speed": { + "channel": 1 + }, + "restore": { + "isRestore": false + } } + } } ``` - #### 2、过滤文件名称 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "path": "hdfs://ns1/flinkx/text", - "filterRegex" : ".*\\.csv", - "defaultFS": "hdfs://ns1", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - }, - { - "name": "col3", - "index": 2, - "type": "int" - }, - { - "name": "col4", - "index": 3, - "type": "int" - } - ], - "fieldDelimiter": ",", - "fileType": "text" - }, - "name": "hdfsreader" - }, - "writer": { - "parameter": {}, - "name": "streamwriter" - } - } - ], - "setting": { - "speed": { - "bytes": 1048576, - "channel": 1 - } + "job": { + "content": [ + { "reader" : { + "parameter" : { + "path" : "/user/hive/warehouse/dev.db/merge_orc", + "filterRegex" : "..*\\.snappy", + "hadoopConfig" : { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "host1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "host2:9000", + "dfs.nameservices" : "ns1" + }, + "column" : [ { + "name": "col1", + "index" : 0, + "type" : "STRING" + }, { + "name": "col2", + "index" : 1, + "type" : "STRING" + } ], + "defaultFS" : "hdfs://ns1", + "fileType" : "orc" + }, + "name" : "hdfsreader" + }, + "writer": { + "name": "streamwriter", + "parameter": { + "print": true + } } + } + ], + "setting": { + "speed": { + "channel": 1 + }, + "restore": { + "isRestore": false + } } + } } ``` diff --git a/docs/offline/reader/kingbasereader.md b/docs/offline/reader/kingbasereader.md index 2ed22425dd..44c517ae8e 100644 --- a/docs/offline/reader/kingbasereader.md +++ b/docs/offline/reader/kingbasereader.md @@ -7,15 +7,49 @@ **KingBase 8.2、8.3** ## 三、参数说明 -- jdbcUrl + +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ +- **jdbcUrl** - 描述:针对KingBase数据库的jdbc连接字符串 - 必选:是 - - 字段类型:String + - 字段类型:List - 默认值:无
-- username +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- **username** - 描述:数据源的用户名 - 必选:是 - 字段类型:String @@ -23,7 +57,7 @@
-- password +- **password** - 描述:数据源指定用户名的密码 - 必选:是 - 字段类型:String @@ -31,15 +65,16 @@
-- schema - - 描述:查询数据库所在schema - - 必选:是 - - 字段类型:String - - 默认值:无 +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000
-- where +- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 @@ -48,7 +83,7 @@
-- splitPk +- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 - 注意: 推荐splitPk使用表主键,因为表主键通常情况下比较均匀,因此切分出来的分片也不容易出现数据热点。 @@ -60,7 +95,7 @@
-- queryTimeOut +- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 @@ -69,7 +104,7 @@
-- customSql +- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 - 注意: 只能是查询语句,否则会导致任务失败; @@ -82,7 +117,7 @@
-- column +- **column** - 描述:需要读取的字段。 - 格式:支持3种格式 1.读取全部字段,如果字段数量很多,可以使用下面的写法: @@ -98,55 +133,80 @@ "value": "value" }] ``` -   属性说明: -   name:字段名称 -   type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 -   format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 -   value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 + - 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 - 字段类型:List - 默认值:无
-- polling +- **polling** - 描述:是否开启间隔轮询,开启后会根据pollingInterval轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数pollingInterval,increColumn,可以选择配置参数startLocation。若不配置参数startLocation,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false
+ +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
-- pollingInterval +- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 - - 字段类型:int + - 字段类型:long - 默认值:5000
-- requestAccumulatorInterval +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
+ +- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 - 字段类型:int - 默认值:2
- + ## 四、配置示例 1、基础配置 -``` +```json { "job": { "content": [ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": ["*"], "customSql": "", @@ -190,19 +250,19 @@ } ``` 2、多通道 -``` +```json { "job": { "content": [ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": ["*"], "customSql": "", @@ -246,22 +306,22 @@ } ``` 3、指定customSql -``` +```json { "job": { "content": [ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": ["id","user_id","name"], - "customSql": "select * from kudu where id > 20", + "customSql": "select * from table where id > 20", "where": "id < 100", "splitPk": "", "queryTimeOut": 1000, @@ -302,19 +362,19 @@ } ``` 4、增量同步指定startLocation -``` +```json { "job": { "content": [ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": [{ "name": "id", @@ -369,19 +429,19 @@ } ``` 5、间隔轮询 -``` +```json { "job": { "content": [ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:kingbase8://localhost:54321/test"], - "table": ["kudu"], - "schema":"test" + "jdbcUrl": ["jdbc:kingbase8://localhost:54321/database"], + "table": ["table"], + "schema":"schema" }], "column": [{ "name": "id", @@ -436,4 +496,4 @@ } } } -``` \ No newline at end of file +``` diff --git a/docs/offline/reader/kudureader.md b/docs/offline/reader/kudureader.md index bc8271299c..cf9ba2aae7 100644 --- a/docs/offline/reader/kudureader.md +++ b/docs/offline/reader/kudureader.md @@ -1,130 +1,144 @@ # Kudu Reader - ## 一、插件名称 -名称:**kudureader**
** - +名称:**kudureader** + ## 二、支持的数据源版本 -**kudu 1.10及以上**
+**kudu 1.10及以上** + - ## 三、参数说明 - **column** - 描述:需要生成的字段 - - 属性说明: - - name:字段名称; - - type:字段类型; - - 必选:是 - - 默认值:无 + - 格式 +```json +"column": [{ + "name": "col", + "type": "string", + "value": "value" +}] +``` +- 属性说明: + - name:字段名称 + - type:字段类型 + - value:如果此字段不为空,会以此value值作为默认值返回 +- 必选:是 +- 字段类型:数组 +- 默认值:无 +
- **masterAddresses** - 描述: master节点地址:端口,多个以,隔开 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **table** - 描述: kudu表名。 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **readMode** - 描述: kudu读取模式: - - 1、read_latest
-默认的读取模式。
-该模式下,服务器将始终在收到请求时返回已提交的写操作。
-这种类型的读取不会返回快照时间戳,并且不可重复。
-用ACID术语表示,它对应于隔离模式:“读已提交”。 - - 2、read_at_snapshot
-该模式下,服务器将尝试在提供的时间戳上执行读取。
-如果未提供时间戳,则服务器将当前时间作为快照时间戳。
-在这种模式下,读取是可重复的,即将来所有在相同时间戳记下的读取将产生相同的数据。
-执行此操作的代价是等待时间戳小于快照的时间戳的正在进行的正在进行的事务,因此可能会导致延迟损失。用ACID术语,这本身就相当于隔离模式“可重复读取”。
-如果对已扫描tablet的所有写入均在外部保持一致,则这对应于隔离模式“严格可序列化”。
-注意:当前存在“空洞”,在罕见的边缘条件下会发生,通过这种空洞有时即使在采取措施使写入如此时,它们在外部也不一致。
-在这些情况下,隔离可能会退化为“读取已提交”模式。 + - 1、read_latest + 默认的读取模式。 + 该模式下,服务器将始终在收到请求时返回已提交的写操作。 + 这种类型的读取不会返回快照时间戳,并且不可重复。 + 用ACID术语表示,它对应于隔离模式:“读已提交”。 + - 2、read_at_snapshot + 该模式下,服务器将尝试在提供的时间戳上执行读取。 + 如果未提供时间戳,则服务器将当前时间作为快照时间戳。 + 在这种模式下,读取是可重复的,即将来所有在相同时间戳记下的读取将产生相同的数据。 + 执行此操作的代价是等待时间戳小于快照的时间戳的正在进行的正在进行的事务,因此可能会导致延迟损失。用ACID术语,这本身就相当于隔离模式“可重复读取”。 + 如果对已扫描tablet的所有写入均在外部保持一致,则这对应于隔离模式“严格可序列化”。 + 注意:当前存在“空洞”,在罕见的边缘条件下会发生,通过这种空洞有时即使在采取措施使写入如此时,它们在外部也不一致。 + 在这些情况下,隔离可能会退化为“读取已提交”模式。 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **authentication** - - 描述:认证方式,如:Kerberos - - 必选:否 - - 默认值:无 - - - -- **principal** - - 描述: 用户名 - - 必选:否 - - 默认值:无 - - - -- **keytabFile** - - 描述:keytab文件路径 + - 描述:认证方式,kudu开启kerberos时需要配置authentication为Kerberos - 必选:否 + - 字段类型:string - 默认值:无 - +
- **workerCount** - 描述:worker线程数 - 必选:否 - - 默认值:默认为cpu*2 - + - 字段类型:int + - 默认值:默认为cpu核心数*2 +
- **bossCount** - 描述:boss线程数 - 必选:否 + - 字段类型:int - 默认值:1 - +
- **operationTimeout** - - 描述:普通操作超时时间 + - 描述:普通操作超时时间,单位毫秒 - 必选:否 + - 字段类型:long - 默认值:30000 - +
- **adminOperationTimeout** - - 描述: 管理员操作(建表,删表)超时时间 + - 描述: 管理员操作(建表,删表)超时时间,单位毫秒 - 必选:否 - - 默认值:30000 - + - 字段类型:long + - 默认值:15000 +
- **queryTimeout** - - 描述:连接scan token的超时时间 + - 描述:连接scan token的超时时间,单位毫秒 - 必选:否 - - 默认值:与operationTimeout一致 - + - 字段类型:long + - 默认值:30000 +
- **where** - 描述:过滤条件字符串,多个以and连接 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **batchSizeBytes** - 描述: kudu scan一次性最大读取字节数 - 必选:否 + - 字段类型:int - 默认值:1048576 +
+ +- **hadoopConfig** + - 描述: kudu开启kerberos,需要配置kerberos相关参数 + - 必选:否 + - 字段类型:map + - 默认值:无 + - ## 四、配置示例 ```json { @@ -136,22 +150,27 @@ "column": [ { "name": "id", - "type": "long" + "type": "string" + }, { + "name": "name", + "type": "string" + }, { + "name": "age", + "type": "int" + }, { + "name": "sex", + "type": "int" } ], - "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", - "table": "kudu", + "masterAddresses": "host:7051", + "table": "table", "readMode": "read_latest", - "authentication": "", - "principal": "", - "keytabFile": "", "workerCount": 2, "bossCount": 1, "operationTimeout": 30000, "adminOperationTimeout": 30000, "queryTimeout": 30000, - "where": " id >= 1 ", - "batchSizeBytes": 1048576 + "where": " id >= 1 " } }, "writer" : { @@ -163,23 +182,10 @@ } ], "setting" : { "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 + "isRestore" : false }, "speed" : { - "bytes" : 0, "channel" : 1 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" } } } diff --git a/docs/offline/reader/mongodbreader.md b/docs/offline/reader/mongodbreader.md index af8cb1a7ad..89c2aafbfa 100644 --- a/docs/offline/reader/mongodbreader.md +++ b/docs/offline/reader/mongodbreader.md @@ -12,60 +12,73 @@ - **url** - 描述:MongoDB数据库连接的URL字符串,详细请参考[MongoDB官方文档](https://docs.mongodb.com/manual/reference/connection-string/) - 必选:否 + - 字段类型:String - 默认值:无 +
- **hostPorts** - 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:否 + - 字段类型:String - 默认值:无 +
- **password** - 描述:数据源指定用户名的密码 - 必选:否 + - 字段类型:String - 默认值:无 +
- **database** - 描述:数据库名称 - 必选:否 + - 字段类型:String - 默认值:无 +
- **collectionName** - 描述:集合名称 - 必选:是 + - 字段类型:String - 默认值:无 - -
+
+ - **fetchSize** - 描述:每次读取的数据条数,通过调整此参数来优化读取速率 - 必选:否 + - 字段类型:int - 默认值:100 - +
- **filter** - - 描述:过滤条件,通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) + - 描述:过滤条件,为json格式, 通过该配置型来限制返回 MongoDB 数据范围,语法请参考[MongoDB查询语法](https://docs.mongodb.com/manual/crud/#read-operations) - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 - - 格式:支持3中格式 + - 格式:支持三种格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: ```json {"column":["*"]} @@ -92,7 +105,9 @@ - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - splitter:因为 MongoDB 支持数组类型,所以 MongoDB 读出来的数组类型要通过这个分隔符合并成字符串 - 必选:是 + - 字段类型: List - 默认值:无 +
diff --git a/docs/offline/reader/mysqlreader.md b/docs/offline/reader/mysqlreader.md index 04ca11f99e..a9cc074c37 100644 --- a/docs/offline/reader/mysqlreader.md +++ b/docs/offline/reader/mysqlreader.md @@ -2,7 +2,7 @@ ## 一、插件名称 -名称:**mysqlreader**
** +名称:**mysqlreader**
## 二、支持的数据源版本 **MySQL 5.X**
@@ -10,35 +10,80 @@ ## 三、参数说明
+- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[MySQL官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - 必选:是 + - 字段类型:List - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据;开启fetchSize需要满足:数据库版本要高于5.0.2、连接参数useCursorFetch=true。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:Integer.MIN_VALUE + +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -47,17 +92,19 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -67,9 +114,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -98,27 +146,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false +
- +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ - **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 @@ -134,11 +210,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] }], "column": ["*"], "customSql": "", @@ -190,11 +266,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] }], "column": ["*"], "customSql": "", @@ -246,14 +322,14 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] }], "column": ["id","user_id","name"], - "customSql": "select * from kudu where id > 20", + "customSql": "select * from table where id > 20", "where": "id < 100", "splitPk": "", "queryTimeOut": 1000, @@ -302,11 +378,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] }], "column": [{ "name": "id", @@ -369,11 +445,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:mysql://0.0.0.1:3306/database?useUnicode=true&characterEncoding=utf8"], + "table": ["table"] }], "column": [{ "name": "id", diff --git a/docs/offline/reader/odpsreader.md b/docs/offline/reader/odpsreader.md index 57a0625031..f02f476d5d 100644 --- a/docs/offline/reader/odpsreader.md +++ b/docs/offline/reader/odpsreader.md @@ -1,62 +1,40 @@ # ODPS Reader - ## 一、插件名称 -名称:**odpsreader**
- -## 二、参数说明 - -- **accessId** - - 描述:ODPS系统登录ID - - 必选:是 - - 默认值:无 - - - -- **accessKey** - - 描述:ODPS系统登录Key - - 必选:是 - - 默认值:无 - - - -- **project** - - 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感) - - 必选:是 - - 默认值:无 - +名称:**odpsreader** +## 二、参数说明 - **table** - 描述:读取数据表的表名称(大小写不敏感) - 必选:是 + - 字段类型:string - 默认值:无 - -
+
- **partition** - - 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou   pt=1,ds=shanghai   pt=2,ds=hangzhou   pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":["pt=1,ds=shanghai"]`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":["pt=1,ds=* "]`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":["pt=*,ds=*"]` - - 必选:如果表为分区表,则必填。如果表为非分区表,则不能填写 + - 描述:读取数据所在的分区信息,支持linux shell通配符,包括 * 表示0个或多个字符,?代表任意一个字符。例如现在有分区表 test,其存在 pt=1,ds=hangzhou   pt=1,ds=shanghai   pt=2,ds=hangzhou   pt=2,ds=beijing 四个分区,如果你想读取 pt=1,ds=shanghai 这个分区的数据,那么你应该配置为: `"partition":"pt=1,ds=shanghai"`; 如果你想读取 pt=1下的所有分区,那么你应该配置为: `"partition":"pt=1,ds=* "`;如果你想读取整个 test 表的所有分区的数据,那么你应该配置为: `"partition":"pt=*,ds=*"` + - 注意:如果表为分区表,则必填。如果表为非分区表,则不能填写 + - 必选:否 + - 字段类型:string - 默认值:无 - -
+
- **column** - 描述:需要读取的字段。 - - 格式:支持3中格式 -
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + - 格式:支持3种格式 + +1.读取全部字段,如果字段数量很多,可以使用下面的写法: ``` "column":[*] ``` - - -
2.只指定字段名称: +2.只指定字段名称: ``` "column":["id","name"] ``` - - -
3.指定具体信息: + 3.指定具体信息: ``` "column": [{ "name": "col", @@ -66,17 +44,60 @@ }] ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 +- 属性说明: + - name:字段名称 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 + - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 + - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 +- 必选:是 +- 字段类型:数组 +- 默认值:无 + +
+ +- **odpsConfig** + - 描述:ODPS的配置信息 - 必选:是 + - 字段类型 map - 默认值:无 + - 可选配置: + - **odpsServer** + - 描述:odps服务地址 + - 必选:否 + - 字段类型 string + - 默认值:[http://service.odps.aliyun.com/api](http://service.odps.aliyun.com/api) + - **accessId** + - 描述:ODPS系统登录ID + - 必选:是 + - 字段类型 string + - 默认值:无 + - **accessKey** + - 描述:ODPS系统登录Key + - 必选:是 + - 字段类型 string + - 默认值:无 + - **project** + - 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感) + - 必选:是 + - 字段类型 string + - 默认值:无 + - **packageAuthorizedProject** + - 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感) + - 注意:当 **packageAuthorizedProject **不为空时,当前project取packageAuthorizedProject对应值 而不是 project 对应的值 + - 必选:否 + - 字段类型 string + - 默认值:无 + - **accountType** + - 描述:account类型 + - 注意:目前只支持 aliyun 类型 + - 必选:否 + - 字段类型 string + - 默认值:aliyun + + - ## 三、配置示例 ```json { @@ -110,23 +131,10 @@ } ], "setting" : { "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 + "isRestore" : false }, "speed" : { - "bytes" : 0, "channel" : 1 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" } } } diff --git a/docs/offline/reader/oraclereader.md b/docs/offline/reader/oraclereader.md index df2f749531..7ade70a875 100644 --- a/docs/offline/reader/oraclereader.md +++ b/docs/offline/reader/oraclereader.md @@ -2,42 +2,78 @@ ## 一、插件名称 -名称:**oraclereader**
** +名称:**oraclereader**
## 二、支持的数据源版本 -**Oracle 9 及以上**
** +**Oracle 9 及以上**
## 三、参数说明
+- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[Oracle官方文档](http://www.oracle.com/technetwork/database/enterprise-edition/documentation/index.html) - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -46,25 +82,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - - 描述:读取时每批次读取的数据条数。 + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:3000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -74,9 +113,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定次参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -107,30 +147,58 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 - +
## 四、配置示例 @@ -143,11 +211,11 @@ { "reader": { "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], - "table": ["TUDOU.KUDU"] + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["TABLE"] }], "column": ["*"], "customSql": "", @@ -200,11 +268,11 @@ { "reader": { "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], - "table": ["TUDOU.KUDU"] + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["TABLE"] }], "column": ["*"], "customSql": "", @@ -257,14 +325,14 @@ { "reader": { "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], - "table": ["TUDOU.KUDU"] + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["table"] }], "column": ["ID","USER_ID","NAME"], - "customSql": "select * from kudu where ID > 20", + "customSql": "select * from table where ID > 20", "where": "ID < 10000", "splitPk": "ID", "fetchSize": 1024, @@ -314,11 +382,11 @@ { "reader": { "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], - "table": ["TUDOU.KUDU"] + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["TABLE"] }], "column": [{ "name": "ID", @@ -382,11 +450,11 @@ { "reader": { "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:oracle:thin:@kudu5:1521:helowin"], - "table": ["TUDOU.KUDU"] + "jdbcUrl": ["jdbc:oracle:thin:@0.0.0.1:1521:oracle"], + "table": ["TABLE"] }], "column": [{ "name": "ID", diff --git a/docs/offline/reader/phoenixreader.md b/docs/offline/reader/phoenixreader.md index 14ceca3099..1285561746 100644 --- a/docs/offline/reader/phoenixreader.md +++ b/docs/offline/reader/phoenixreader.md @@ -9,35 +9,72 @@ phoenix4.12.0-HBase-1.3及以上 ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:phoenix:node01,node02,node03:2181"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ + - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[phoniex官方文档](https://phoenix.apache.org/#) - 必选:是 + - 字段类型:List - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -46,17 +83,28 @@ phoenix4.12.0-HBase-1.3及以上 - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 +
+ +- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000 +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -66,9 +114,10 @@ phoenix4.12.0-HBase-1.3及以上 - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -97,27 +146,55 @@ phoenix4.12.0-HBase-1.3及以上 - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ## 四、配置示例 @@ -131,16 +208,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", @@ -199,16 +273,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", @@ -267,16 +338,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", @@ -333,16 +401,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", @@ -401,16 +466,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", @@ -471,16 +533,13 @@ phoenix4.12.0-HBase-1.3及以上 "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "", "password" : "", diff --git a/docs/offline/reader/polardbreader.md b/docs/offline/reader/polardbreader.md index fd9eed5962..f4a022cee7 100644 --- a/docs/offline/reader/polardbreader.md +++ b/docs/offline/reader/polardbreader.md @@ -10,35 +10,71 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 -
jdbcUrl参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) +
jdbcUrl参考文档:[PolarDB官方文档](https://help.aliyun.com/document_detail/147247.html?spm=a2c4g.11186623.2.8.51346c98uYvnUU) - 必选:是 + - 字段类型:List - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -47,17 +83,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 +
+- **fetchSize** + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 + - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 + - 必选:否 + - 字段类型:int + - 默认值:1000 + +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -67,9 +114,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -98,31 +146,57 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 - - ## 四、配置示例 @@ -134,11 +208,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] }], "column": ["*"], "customSql": "", @@ -190,11 +264,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] }], "column": ["*"], "customSql": "", @@ -246,14 +320,14 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] }], "column": ["id","user_id","name"], - "customSql": "select * from kudu where id > 20", + "customSql": "select * from table where id > 20", "where": "id < 100", "splitPk": "", "fetchSize": 0, @@ -302,11 +376,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] }], "column": [{ "name": "id", @@ -369,11 +443,11 @@ { "reader": { "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [{ - "jdbcUrl": ["jdbc:mysql://kudu3:3306/tudou?useUnicode=true&characterEncoding=utf8"], - "table": ["kudu"] + "jdbcUrl": ["jdbc:polardb://0.0.0.1:3306/database"], + "table": ["table"] }], "column": [{ "name": "id", diff --git a/docs/offline/reader/postgresqlreader.md b/docs/offline/reader/postgresqlreader.md index 302eb54d36..ea1234e44b 100644 --- a/docs/offline/reader/postgresqlreader.md +++ b/docs/offline/reader/postgresqlreader.md @@ -10,35 +10,71 @@ ## 三、参数说明
+- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:postgresql://0.0.0.1:5432/postgres"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串
jdbcUrl参考文档:[postgresql官方文档](https://jdbc.postgresql.org/documentation/head/connect.html#connection-parameters) - 必选:是 + - 字段类型:List - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -47,25 +83,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - - 描述:读取时每批次读取的数据条数。 + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -75,9 +114,10 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -106,27 +146,55 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ** @@ -142,16 +210,13 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", @@ -197,16 +262,13 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", @@ -252,16 +314,13 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", @@ -307,16 +366,13 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", @@ -364,16 +420,13 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", diff --git a/docs/offline/reader/saphanareader.md b/docs/offline/reader/saphanareader.md index aea38f3a40..3194d14bad 100644 --- a/docs/offline/reader/saphanareader.md +++ b/docs/offline/reader/saphanareader.md @@ -9,34 +9,70 @@ SAP HANA 2.0及以上
## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:sap://0.0.0.1:39017"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:jdbc连接字符串 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 +
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -45,25 +81,28 @@ SAP HANA 2.0及以上
- 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - - 描述:读取时每批次读取的数据条数。 + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 - - 默认值:3000 - + - 字段类型:int + - 默认值:1000 +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -73,9 +112,10 @@ SAP HANA 2.0及以上
- 当指定了此参数时,connection里指定的table无效; - 当指定次参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:需要读取的字段。 @@ -106,31 +146,57 @@ SAP HANA 2.0及以上
- format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 - - ## 四、配置示例 ```json @@ -145,7 +211,7 @@ SAP HANA 2.0及以上
"connection": [ { "jdbcUrl": [ - "jdbc:sap://kudu3:39017" + "jdbc:sap://0.0.0.1:39017" ], "table": [ "SYS.P_DPAPI_KEY_" diff --git a/docs/offline/reader/sqlserverreader.md b/docs/offline/reader/sqlserverreader.md index e5cebc2803..f10f7aabb9 100644 --- a/docs/offline/reader/sqlserverreader.md +++ b/docs/offline/reader/sqlserverreader.md @@ -10,34 +10,70 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": ["jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database"], + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:使用开源的jtds驱动连接 而非Microsoft的官方驱动
jdbcUrl参考文档:[jtds驱动官方文档](http://jtds.sourceforge.net/faq.html) - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **where** - 描述:筛选条件,reader插件根据指定的column、table、where条件拼接SQL,并根据这个SQL进行数据抽取。在实际业务场景中,往往会选择当天的数据进行同步,可以将where条件指定为gmt_create > time。 - 注意:不可以将where条件指定为limit 10,limit不是SQL的合法where子句。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **splitPk** - 描述:当speed配置中的channel大于1时指定此参数,Reader插件根据并发数和此参数指定的字段拼接sql,使每个并发读取不同的数据,提升读取速率。 @@ -46,25 +82,28 @@ - 目前splitPk仅支持整形数据切分,`不支持浮点、字符串、日期等其他类型`。如果用户指定其他非支持类型,FlinkX将报错! - 如果channel大于1但是没有配置此参数,任务将置为失败。 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **fetchSize** - - 描述:读取时每批次读取的数据条数。 + - 描述:一次性从数据库中读取多少条数据,jdbc默认一次将所有结果都读取到内存中,在数据量很大时可能会造成OOM,设置这个参数可以控制每次读取fetchSize条数据。 - 注意:此参数的值不可设置过大,否则会读取超时,导致任务失败。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **queryTimeOut** - 描述:查询超时时间,单位秒。 - 注意:当数据量很大,或者从视图查询,或者自定义sql查询时,可通过此参数指定超时时间。 - 必选:否 + - 字段类型:int - 默认值:1000 - +
- **customSql** - 描述:自定义的查询语句,如果只指定字段不能满足需求时,可通过此参数指定查询的sql,可以是任意复杂的查询语句。 @@ -74,9 +113,18 @@ - 当指定了此参数时,connection里指定的table无效; - 当指定此参数时,column必须指定具体字段信息,不能以*号代替; - 必选:否 + - 字段类型:String - 默认值:无 +
+ +- **withNoLock** + - 描述:是否在sql语句后面添加 with(nolock) + - 必选:否 + - 字段类型:Boolean + - 默认值:false +
- **column** - 描述:需要读取的字段。 @@ -107,27 +155,56 @@ - format:如果字段是时间字符串,可以指定时间的格式,将字段类型转为日期格式返回 - value:如果数据库里不存在指定的字段,则会把value的值作为常量列返回,如果指定的字段存在,当指定字段的值为null时,会以此value值作为默认值返回 - 必选:是 + - 字段类型:List - 默认值:无 - +
- **polling** - 描述:是否开启间隔轮询,开启后会根据`pollingInterval`轮询间隔时间周期性的从数据库拉取数据。开启间隔轮询还需配置参数`pollingInterval`,`increColumn`,可以选择配置参数`startLocation`。若不配置参数`startLocation`,任务启动时将会从数据库中查询增量字段最大值作为轮询的开始位置。 - 必选:否 + - 字段类型:Boolean - 默认值:false - +
- **pollingInterval** - 描述:轮询间隔时间,从数据库中拉取数据的间隔时间,默认为5000毫秒。 - 必选:否 + - 字段类型:long - 默认值:5000 +
+ +- **increColumn** + - 增量字段,可以是对应的增量字段名,也可以是纯数字,表示增量字段在column中的顺序位置(从0开始) + - 必选:否 + - 字段类型:String或int + - 默认值:无 + +
+ +- **startLocation** + - 描述:增量查询起始位置 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **useMaxFunc** + - 描述:用于标记是否保存endLocation位置的一条或多条数据,true:不保存,false(默认):保存, 某些情况下可能出现最后几条数据被重复记录的情况,可以将此参数配置为true + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
- **requestAccumulatorInterval** - 描述:发送查询累加器请求的间隔时间。 - 必选:否 + - 字段类型:int - 默认值:2 ** @@ -143,21 +220,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -209,21 +283,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -275,28 +346,25 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", "splitPk": "id", "fetchSize": 1000, "queryTimeOut": 1000, - "customSql": "select * from kudu where id > 20", + "customSql": "select * from table where id > 20", "requestAccumulatorInterval": 2 }, "name" : "sqlserverreader" @@ -330,6 +398,7 @@ } } } +} ``` #### 4、增量同步指定startLocation @@ -341,21 +410,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", @@ -410,21 +476,18 @@ "parameter" : { "column" : [ { "name" : "id", - "type" : "bigint", - "key" : "id" + "type" : "bigint" }, { "name" : "user_id", - "type" : "bigint", - "key" : "user_id" + "type" : "bigint" }, { "name" : "name", - "type" : "varchar", - "key" : "name" + "type" : "varchar" } ], "username" : "username", "password" : "password", "connection" : [ { - "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack" ], + "jdbcUrl" : [ "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=database" ], "table" : [ "tableTest" ] } ], "where": "id > 1", diff --git a/docs/offline/reader/streamreader.md b/docs/offline/reader/streamreader.md index 697571098a..e7e29ac74b 100644 --- a/docs/offline/reader/streamreader.md +++ b/docs/offline/reader/streamreader.md @@ -12,7 +12,7 @@ ### 三、参数说明 - **sliceRecordCount** - - 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止 + - 描述:每个通道生成的数据条数,不配置此参数或者配置为0,程序会持续生成数据,不会停止。例如当配置通道数为5时,就需要填写5个数字。不同通道数写入数量可以不同 - 必选:否 - 默认值:0 @@ -42,10 +42,10 @@ - float - double - date - - timestamp + - timestamp: 以当前时间戳作为模拟值,因此也是递增的 - bigdecimal - biginteger - - int[] + - int[]: 数组的长度也是随机的 - byte[] - boolean[] - char[],character[] diff --git a/docs/offline/writer/carbondatawriter.md b/docs/offline/writer/carbondatawriter.md index 2ad1b8e41e..ba4d954b46 100644 --- a/docs/offline/writer/carbondatawriter.md +++ b/docs/offline/writer/carbondatawriter.md @@ -2,7 +2,7 @@ ## 一、插件名称 -名称:**carbondatawriter**
** +名称:**carbondatawriter**
## 二、支持的数据源版本 **Carbondata 1.5及以上**
@@ -13,34 +13,50 @@ - **path** - 描述:carbondata表的存储路径 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **table** - 描述:carbondata表名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **database** - 描述:carbondata库名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:所配置的表中需要同步的字段名列表 - - 必选:是 + - 必选:是 + 字段包括表字段和常量字段, + 表字段的格式: + - name:字段名称 + - type:字段类型 + ``` +{ + "name": "col1", + "type": "string" +} +``` + - 必选:是 + - 字段类型:List - 默认值:无 - +
- **hadoopConfig** - 描述:集群HA模式时需要填写的namespace配置及其它配置 - 必选:是 + - 字段类型:Map - 默认值:无
@@ -48,29 +64,33 @@ - **defaultFS** - 描述:Hadoop hdfs文件系统namenode节点地址 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **writeMode** - 描述:写入模式,支持append和overwrite - 必填:否 + - 字段类型:String - 默认值:append - +
- **partition** - 描述:carbondata分区 - 必填:否 - - 默认值:append - + - 字段类型:String + - 默认值:无 +
- **batchSize** - 描述:批量提交条数 - 必填:否 + - 字段类型:int - 默认值:204800 - +
diff --git a/docs/offline/writer/cassandrawriter.md b/docs/offline/writer/cassandrawriter.md index 2e999d0242..e04786faca 100644 --- a/docs/offline/writer/cassandrawriter.md +++ b/docs/offline/writer/cassandrawriter.md @@ -13,91 +13,104 @@ - 描述:数据库地址 - 必选:是 - 默认值:无 - + - 字段类型:String +
- **port** - 描述:端口 - 必选:否 - 默认值:9042 - + - 字段类型:Integer +
- **username** - 描述:用户名 - 必选:否 - 默认值:无 - + - 字段类型:String +
- **password** - 描述:密码 - 必选:否 - 默认值:无 - + - 字段类型:String +
- **useSSL** - 描述:数字证书 - 必选:否 - 默认值:false - + - 字段类型:Boolean +
- **column** - 描述:查询结果中被select出来的属性集合,为空则select * - 必选:否 - 默认值:无 - + - 字段类型:List +
- **keyspace** - 描述:需要同步的表所在的keyspace - 必选:是 - 默认值:无 - + - 字段类型:String +
- **table** - 描述:要查询的表 - 必选:是 - 默认值:无 - + - 字段类型:String +
- **batchSize** - 描述:异步写入的批次大小 - 必选:否 - 默认值:1 - + - 字段类型:Integer +
- **asyncWrite** - 描述:是否异步写入 - 必选:否 - 默认值:false - + - 字段类型:Boolean +
- **connecttionsPerHost** - 描述:分配给每个host的连接数 - 必选:否 - 默认值:8 - + - 字段类型:Integer +
- **maxPendingPerConnection** - 描述:最多能建立的连接数 - 必选:否 - 默认值:128 - + - 字段类型:Integer +
- **consistancyLevel** - 描述:数据一致性级别。可选`ONE`、`QUORUM`、`LOCAL_QUORUM`、`EACH_QUORUM`、`ALL`、`ANY`、`TWO`、`THREE`、`LOCAL_ONE` - 必选:否 - 默认值:无 - + - 字段类型:String +
diff --git a/docs/offline/writer/clickhousewriter.md b/docs/offline/writer/clickhousewriter.md index b6fc22df1c..f985c243c9 100644 --- a/docs/offline/writer/clickhousewriter.md +++ b/docs/offline/writer/clickhousewriter.md @@ -10,71 +10,113 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:clickhouse://localhost:8123/database", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String - 默认值:无 + +
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - 必选:是 - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 语句,只支持insert操作 - 必选:是 - 所有选项:insert + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 + +
-
** ## 四、配置示例 ```json @@ -90,8 +132,16 @@ "type": "int" }, { - "name": "age", + "name": "user_id", "type": "int" + }, + { + "name" : "name", + "type" : "string" + }, + { + "name" : "eventDate", + "type" : "date" } ] }, @@ -101,29 +151,12 @@ "name": "clickhousewriter", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:clickhouse://0.0.0.1:8123/dtstack", - "table": [ - "tableTest" - ] + "jdbcUrl": "jdbc:clickhouse://localhost:8123/database", + "table": ["test"] }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "BIGINT", - "key": "id" - }, - { - "name": "user_id", - "type": "BIGINT", - "key": "user_id" - }, - { - "name": "name", - "type": "varchar", - "key": "name" - }], + "column": ["id","user_id","name","eventDate"], "writeMode": "insert", "batchSize": 1024, "preSql": [], diff --git a/docs/offline/writer/db2writer.md b/docs/offline/writer/db2writer.md index e7b66dd319..f7d8ecf731 100644 --- a/docs/offline/writer/db2writer.md +++ b/docs/offline/writer/db2writer.md @@ -2,83 +2,126 @@ ## 一、插件名称 -名称:**db2writer**
** +名称:**db2writer**
## 二、支持的数据源版本 -**DB2 9、10**
** +**DB2 9、10**
## 三、参数说明 -- **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:db2://localhost:50000/database", + "table": ["table"], + "schema":"public" + }] + ``` - 默认值:无 +
+- **jdbcUrl** + - 描述:针对关系型数据库的jdbc连接字符串 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
+ - **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into`  语句 - 必选:是 - - 所有选项:insert/update + - 所有选项:insert、update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 描述:当写入模式为update时,需要指定此参数的值为`唯一索引字段` - 注意: - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 ** @@ -120,33 +163,12 @@ "name": "db2writer", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:db2://localhost:50000/sample", - "table": [ - "staff" - ] + "jdbcUrl": "jdbc:db2://localhost:50000/database", + "table": ["table"] }], - "username": "user", + "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "SMALLINT", - "key": "id" - }, - { - "name": "name", - "type": "VARCHAR", - "key": "user_id" - }, - { - "name": "dept", - "type": "SMALLINT", - "key": "name" - },{ - "name": "job", - "type": "VARCHAR" - } - ], + "column": ["id","name","dept","job"], "writeMode": "insert", "batchSize": 1024, "preSql": [], @@ -210,39 +232,17 @@ "name": "db2writer", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:db2://localhost:50000/sample", - "table": [ - "staff" - ] + "jdbcUrl": "jdbc:db2://localhost:50000/database", + "table": ["table"] }], - "username": "user", + "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "SMALLINT", - "key": "id" - }, - { - "name": "name", - "type": "VARCHAR", - "key": "user_id" - }, - { - "name": "dept", - "type": "SMALLINT", - "key": "name" - },{ - "name": "job", - "type": "VARCHAR" - } - ], + "column": ["id","name","dept","job"], "writeMode": "update", "updateKey": {"key": ["id"]}, "batchSize": 1024, "preSql": [], - "postSql": [], - "updateKey": {} + "postSql": [] } } } diff --git a/docs/offline/writer/dmwriter.md b/docs/offline/writer/dmwriter.md index ec37a9f433..15d2893281 100644 --- a/docs/offline/writer/dmwriter.md +++ b/docs/offline/writer/dmwriter.md @@ -9,80 +9,125 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:dm://localhost:5236", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - - 描述:针对关系型数据库的jdbc连接字符串 + - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - 默认值:否 + - 字段类型:List - 默认值:无 +
+ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 - 注意: - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 -** + + ## 四、配置示例 #### 1、insert @@ -110,30 +155,18 @@ "writer": { "name": "dmwriter", "parameter": { - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "connection": [ { "jdbcUrl": "jdbc:dm://localhost:5236", - "table": [ - "PERSON.STUDENT" - ] + "table": ["table"] } ], - "session": [], "preSql": [], "postSql": [], - "writeMode": "insert", - "column": [ - { - "name": "ID", - "type": "int" - }, - { - "name": "AGE", - "type": "int" - } - ] + "mode": "insert", + "column": ["ID","AGE"] } } } @@ -184,31 +217,19 @@ "writer": { "name": "dmwriter", "parameter": { - "username": "SYSDBA", - "password": "SYSDBA", + "username": "username", + "password": "password", "connection": [ { "jdbcUrl": "jdbc:dm://localhost:5236", - "table": [ - "PERSON.STUDENT" - ] + "table": ["table"] } ], - "session": [], "preSql": [], "postSql": [], - "writeMode": "update", + "mode": "update", "updateKey": {"key": ["ID"]}, - "column": [ - { - "name": "ID", - "type": "int" - }, - { - "name": "AGE", - "type": "int" - } - ] + "column": ["ID","AGE"] } } } diff --git a/docs/offline/writer/eswriter.md b/docs/offline/writer/eswriter.md index 35029a6c80..269ba1536b 100644 --- a/docs/offline/writer/eswriter.md +++ b/docs/offline/writer/eswriter.md @@ -12,36 +12,42 @@ - **address** - 描述:Elasticsearch地址,单个节点地址采用host:port形式,多个节点的地址用逗号连接 - 必选:是 + - 字段类型:String - 默认值:无 +
- **username** - 描述:Elasticsearch认证用户名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:Elasticsearch认证密码 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **index** - 描述:Elasticsearch 索引值 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **type** - 描述:Elasticsearch 索引类型 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** @@ -54,6 +60,7 @@ ``` - 必选:是 + - 字段类型:List - 默认值:无 @@ -80,22 +87,25 @@ - 注意: - 如果不指定idColumns属性,则会随机产生文档id - 如果指定的字段值存在重复或者指定了常数,按照es的逻辑,同样值的doc只会保留一份 + - 字段类型:List - 默认值:无 - +
- **bulkAction** - 描述:批量写入的记录条数 - 必选:是 + - 字段类型:int - 默认值:100 - +
- **timeout** - 描述:连接超时时间,如果bulkAction指定的数值过大,写入数据可能会超时,这时可以配置超时时间 - 必选:否 + - 字段类型:int - 默认值:无 - +
@@ -128,7 +138,7 @@ "writer": { "name": "eswriter", "parameter": { - "address": "172.16.8.193:9200", + "address": "localhost:9200", "username": "elastic", "password": "abc123", "index": "tudou", diff --git a/docs/offline/writer/ftpwriter.md b/docs/offline/writer/ftpwriter.md index 9b6995ef01..dca718c91b 100644 --- a/docs/offline/writer/ftpwriter.md +++ b/docs/offline/writer/ftpwriter.md @@ -1,10 +1,9 @@ # FTP Writer - ## 一、插件名称 -名称:**ftpwriter**
+名称:**ftpwriter** + - ## 二、数据源版本 | 协议 | 是否支持 | | --- | --- | @@ -13,220 +12,222 @@ - ## 三、数据源配置 -FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z)
linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf)
sftp服务搭建
windows:[地址](http://www.freesshd.com/)
linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7)
+FTP服务搭建 +windows:[地址](https://help.aliyun.com/document_detail/92046.html?spm=a2c4g.11186623.6.1185.6371dcd5DOfc5z) +linux:[地址](https://help.aliyun.com/document_detail/92048.html?spm=a2c4g.11186623.6.1184.7a9a2dbcRLDNlf) +sftp服务搭建 +windows:[地址](http://www.freesshd.com/) +linux:[地址](https://yq.aliyun.com/articles/435356?spm=a2c4e.11163080.searchblog.102.576f2ec1BVgWY7) + - ## 四、参数说明 - **protocol** - 描述:ftp服务器协议,目前支持传输协议有`ftp`、`sftp` - 必选:是 + - 字段类型:string - 默认值:无 - +
- **host** - 描述:ftp服务器地址 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **port** - 描述:ftp服务器端口 - 必选:否 + - 字段类型:int - 默认值:若传输协议是sftp协议,默认值是22;若传输协议是标准ftp协议,默认值是21 - +
- **connectPattern** - 描述:协议为ftp时的连接模式,可选`pasv`,`port`,参数含义可参考:[模式说明](https://blog.csdn.net/qq_16038125/article/details/72851142) - 必选:否 + - 字段类型:string - 默认值:`PASV` - +
- **username** - 描述:ftp服务器访问用户名 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **password** - 描述:ftp服务器访问密码 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **path** - 描述:远程FTP文件系统的路径信息,注意这里可以支持填写多个路径 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **fieldDelimiter** - 描述:读取的字段分隔符 - 必选:是 + - 字段类型:string - 默认值:`,` - +
- **encoding** - 描述:读取文件的编码配置 - 必选:否 + - 字段类型:string - 默认值:`UTF-8` - +
- **privateKeyPath** - 描述:私钥文件路径 - 必选:否 + - 字段类型:string - 默认值:无 -
+
- **writeMode** - 描述:ftpwriter写入前数据清理处理模式: - append:追加 - overwrite:覆盖 - - 注意:overwrite模式时会删除dtp当前目录下的所有文件 + - 注意:overwrite模式时会删除ftp当前目录下的所有文件 - 必选:否 + - 字段类型:string - 默认值:append - +
- **isFirstLineHeader** - 描述:首行是否为标题行,如果是则不读取第一行 - 必选:否 + - 字段类型:boolean - 默认值:false - +
- **timeout** - 描述:连接超时时间,单位毫秒 - 必选:否 + - 字段类型:long - 默认值:5000 - +
- **maxFileSize** - - 描述:写入hdfs单个文件最大大小,单位字节 + - 描述:写入ftp单个文件最大大小,单位字节 - 必须:否 + - 字段类型:long - 默认值:1073741824‬(1G) - +
- **column** - 描述:需要读取的字段 - 格式:指定具体信息: -```json +``` "column": [{ "name": "col1", "type": "datetime" }] ``` - - 属性说明: - - name:字段名称 - - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 - - 必选:是 - - 默认值:无 +- 属性说明: + - name:字段名称 + - type:字段类型,ftp读取的为文本文件,本质上都是字符串类型,这里可以指定要转成的类型 +- 必选:是 +- 字段类型:数组 +- 默认值:无 - ## 五、使用示例 - -#### 1、append模式写入 +#### 1、sftp append模式写入 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - }, - { - "name": "col3", - "type": "int" - }, - { - "name": "col4", - "type": "int" - } - ], - "sliceRecordCount": [ - "100" - ] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "path": "/data/ftp/flinkx", - "protocol": "sftp", - "port": 22, - "writeMode": "append", - "host": "localhost", - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - }, - { - "name": "col3", - "type": "int" - }, - { - "name": "col4", - "type": "int" - } - ], - "password": "pass", - "fieldDelimiter": ",", - "encoding": "utf-8", - "username": "user" - }, - "name": "ftpwriter" - } - } - ], - "setting": { - "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "errorLimit": { - "record": 100 - }, - "speed": { - "bytes": 0, - "channel": 1 - } + "job": { + "content": [ + { + "reader": { + "parameter": { + "column": [ + { + "name": "name", + "type": "string" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount": [ + "100" + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "path": "/data/ftp", + "protocol": "sftp", + "port": 22, + "writeMode": "append", + "host": "host", + "column": [ + { + "name": "name", + "type": "string" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "name", + "type": "string" + } + ], + "username": "name", + "password": "passwd", + "fieldDelimiter": ",", + "encoding": "utf-8" + }, + "name": "ftpwriter" } + } + ], + "setting": { + "restore": { + "isRestore": false + }, + "speed": { + "bytes": 0, + "channel": 1 + } } + } } ``` - #### 2、指定文件大小 ```json { @@ -265,7 +266,7 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ "protocol": "sftp", "port": 22, "writeMode": "append", - "host": "localhost", + "host": "host", "column": [ { "name": "col1", @@ -296,16 +297,9 @@ FTP服务搭建
windows:[地址](https://help.aliyun.com/document_detail/ ], "setting": { "restore": { - "maxRowNumForCheckpoint": 0, - "isRestore": false, - "restoreColumnName": "", - "restoreColumnIndex": 0 - }, - "errorLimit": { - "record": 100 + "isRestore": false }, "speed": { - "bytes": 0, "channel": 1 } } diff --git a/docs/offline/writer/gbasewriter.md b/docs/offline/writer/gbasewriter.md index e1ed14c344..d154e98b4d 100644 --- a/docs/offline/writer/gbasewriter.md +++ b/docs/offline/writer/gbasewriter.md @@ -10,76 +10,120 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/database", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ + +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者` merge into` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **updateKey** - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 - 注意: - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 ** @@ -111,22 +155,12 @@ "name": "gbasewriter", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/dtstack", - "table": [ - "tableTest" - ] + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/database", + "table": ["tableTest"] }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "INT" - }, - { - "name": "age", - "type": "INT" - }], + "column": ["id","age"], "writeMode": "insert", "batchSize": 1024, "preSql": [], @@ -173,22 +207,12 @@ "name": "gbasewriter", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/dtstack", - "table": [ - "tableTest" - ] + "jdbcUrl": "jdbc:gbase://0.0.0.1:5258/database", + "table": ["tableTest"] }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "INT" - }, - { - "name": "age", - "type": "INT" - }], + "column": ["id","age"], "writeMode": "update", "batchSize": 1024, "updateKey": {"key": ["id"]}, diff --git a/docs/offline/writer/greenplumwriter.md b/docs/offline/writer/greenplumwriter.md index 57a069ce88..2731ea73ba 100644 --- a/docs/offline/writer/greenplumwriter.md +++ b/docs/offline/writer/greenplumwriter.md @@ -9,62 +9,101 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:仅支持`insert`操作,可以搭配insertSqlMode使用 - 必选:是 + - 字段类型:String - 默认值:无, - +
- **insertSqlMode** - 描述:控制写入数据到目标表采用  `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 @@ -72,13 +111,15 @@ - 为了避免`insert`过慢带来的问题,此参数被固定为`copy` - 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 - 必选:否 - - 默认值:无 - + - 字段类型:String + - 默认值:copy +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 ** @@ -107,20 +148,16 @@ "name": "greenplumwriter", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=exampledb", - "table": ["tbl_pay_log_copy"] + "jdbcUrl": "jdbc:pivotal:greenplum://localhost:5432;DatabaseName=database", + "table": ["table"] }], - "username": "gpadmin", - "password": "gpadmin", - "column": [ - { - "name": "id", - "type": "int" - }], + "username": "username", + "password": "password", + "column": ["id"], "writeMode": "insert", "insertSqlMode": "copy", "batchSize": 100, - "preSql": ["TRUNCATE tbl_pay_log_copy"], + "preSql": ["TRUNCATE table"], "postSql": [] } } diff --git a/docs/offline/writer/hbasewriter.md b/docs/offline/writer/hbasewriter.md index 3429e34bea..a0c3db9ee1 100644 --- a/docs/offline/writer/hbasewriter.md +++ b/docs/offline/writer/hbasewriter.md @@ -5,128 +5,119 @@ 名称:**hbasewriter**
## 二、支持的数据源版本 -**HBase 1.3及以上**
+**HBase 1.2及以上**
## 三、参数说明 - **tablename** - 描述:hbase表名 - 必选:是 + - 字段类型:String - 默认值:无 - - - -- **hbaseConfig** - - 描述:hbase的连接配置,以json的形式组织 (见hbase-site.xml),key可以为以下七种: - -Kerberos;
hbase.security.authentication;
hbase.security.authorization;
hbase.master.kerberos.principal;
hbase.master.keytab.file;
hbase.regionserver.keytab.file;
hbase.regionserver.kerberos.principal - +
+ + - kerberos配置 + 在hbaseConfig中加入以下三条中的任一条即表明开启Kerberos配置: + ``` + "hbase.security.authentication" :"Kerberos", + "hbase.security.authorization" : "Kerberos", + "hbase.security.auth.enable" : true + ``` + 在开启kerberos后,需要根据自己的集群指定以下两个principal的value值 + ``` + "hbase.regionserver.kerberos.principal":"hbase/_HOST@DTSTACK.COM", + "hbase.master.kerberos.principal":"hbase/_HOST@DTSTACK.COM" + ``` + 还需要指定Kerberos相关文件的位置 + ``` + "principalFile": "path of keytab", + "java.security.krb5.conf": "path of krb5.conf" + ``` - 必选:是 + - 字段类型:Map - 默认值:无 - +
- **nullMode** - 描述:读取的null值时,如何处理。支持两种方式: - skip:表示不向hbase写这列; - empty:写入HConstants.EMPTY_BYTE_ARRAY,即new byte [0] - 必选:否 + - 字段类型:String - 默认值:skip - - + +
- **encoding** - 描述:字符编码 - 必选:无 + - 字段类型:String - 默认值:UTF-8 - -
+ +
- **walFlag** - - 描述:在HBae client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能 + - 描述:在HBase client向集群中的RegionServer提交数据时(Put/Delete操作),首先会先写WAL(Write Ahead Log)日志(即HLog,一个RegionServer上的所有Region共享一个HLog),只有当WAL日志写成功后,再接着写MemStore,然后客户端被通知提交数据成功;如果写WAL日志失败,客户端则被通知提交失败。关闭(false)放弃写WAL日志,从而提高数据写入的性能 - 必选:否 + - 字段类型:Boolean - 默认值:false - - + +
- **writeBufferSize** - 描述:设置HBae client的写buffer大小,单位字节。配合autoflush使用。autoflush,开启(true)表示Hbase client在写的时候有一条put就执行一次更新;关闭(false),表示Hbase client在写的时候只有当put填满客户端写缓存时,才实际向HBase服务端发起写请求 - 必选:否 + - 字段类型:long - 默认值:8388608(8M) - - - -- **scanCacheSize** - - 描述:一次RPC请求批量读取的Results数量 - - 必选:无 - - 默认值:256 - - - -- **scanBatchSize** - - 描述:每一个result中的列的数量 - - 必选:无 - - 默认值:100 - - + +
- **column** - - 描述:要读取的hbase字段,normal 模式与multiVersionFixedColumn 模式下必填项。 - - name:指定读取的hbase列,除了rowkey外,必须为 列族:列名 的格式; - - type:指定源数据的类型,format指定日期类型的格式,value指定当前类型为常量,不从hbase读取数据,而是根据value值自动生成对应的列。 + - 描述:要写入的hbase字段。 + - name:指定写入的hbase列,必须为 列族:列名 的格式; + - type:指定源数据的类型,format指定日期类型的格式 - 必选:是 + - 字段类型:List - 默认值:无 - - + +
- **rowkeyColumn** - - 描述:用于构造rowkey的描述信息,支持两种格式,每列形式如下 - - 字符串格式 -
字符串格式为:$(cf:col),可以多个字段组合:$(cf:col1)_$(cf:col2), -
可以使用md5函数:md5($(cf:col)) - - 数组格式 - - 普通列 -``` -{ - "index": 0, // 该列在column属性中的序号,从0开始 - "type": "string" 列的类型,默认为string -} -``` - - - 常数列 -``` -{ - "value": "ffff", // 常数值 - "type": "string" // 常数列的类型,默认为string -} -``` - - - 必选:否 -
如果不指定idColumns属性,则会随机产生文档id + - 描述:用于构造rowkey的描述信息,每列形式如下 + 字符串格式为:$(cf:col),可以多个字段组合:$(cf:col1)_$(cf:col2), + 可以使用md5函数:md5($(cf:col)) + - 必选: 是 + - 字段类型:String - 默认值:无 - - + +
- **versionColumn** - - 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。配置格式如下: -``` -"versionColumn":{ -"index":1 -} -``` - - -
或者 -``` -"versionColumn":{ -"value":"123456789" -} -``` + - 描述:指定写入hbase的时间戳。支持:当前时间、指定时间列,指定时间,三者选一。若不配置表示用当前时间。index:指定对应reader端column的索引,从0开始,需保证能转换为long,若是Date类型,会尝试用yyyy-MM-dd HH:mm:ss和yyyy-MM-dd HH:mm:ss SSS去解析;若不指定index;value:指定时间的值,类型为字符串。注意,在hbase中查询默认会显示时间戳最大的数据,因此简单查询可能会出现看不到更新的情况,需要加过滤条件筛选。 + 配置格式如下: + ``` + "versionColumn":{ + "index":1 + } + ``` + 或者 + ``` + "versionColumn":{ + "value":"123456789" + } + ``` + - 必选: 否 + - 字段类型:Map + - 默认值:当前时间
## 三、配置示例 -```json + +未开启Kerberos的情况 +``` { "job" : { "content" : [ { @@ -204,6 +195,88 @@ Kerberos;
hbase.security.authentication;
hbase.security.authorizat } } ``` - -
- +开启了Kerberos的情况 +``` +{ + "job" : { + "content" : [ { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "name": "hbasewriter", + "parameter": { + "hbaseConfig": { + "hbase.zookeeper.property.clientPort": "2181", + "hbase.rootdir": "hdfs://ns1/hbase", + "hbase.cluster.distributed": "true", + "hbase.zookeeper.quorum": "node01,node02,node03", + "zookeeper.znode.parent": "/hbase", + "hbase.security.auth.enable": true, + "hbase.regionserver.kerberos.principal":"hbase/host@DTSTACK.COM", + "hbase.master.kerberos.principal":"hbase/host@DTSTACK.COM", + "principalFile": "path of keytab", + "useLocalFile": "true", + "java.security.krb5.conf": "path of krb5.conf" + }, + "table": "tb1", + "rowkeyColumn": "col1#col2", + "column": [ + { + "name": "cf1:id", + "type": "int" + }, + { + "name": "cf1:user_id", + "type": "int" + }, + { + "name": "cf1:name", + "type": "string" + } + ] + } + } + } ], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": false, + "isStream" : false, + "restoreColumnName": "", + "restoreColumnIndex": 0 + }, + "log" : { + "isLogger": false, + "level" : "debug", + "path" : "", + "pattern":"" + } + } + } +} +``` diff --git a/docs/offline/writer/hdfswriter.md b/docs/offline/writer/hdfswriter.md index 97390a0a3d..6a8350785f 100644 --- a/docs/offline/writer/hdfswriter.md +++ b/docs/offline/writer/hdfswriter.md @@ -1,10 +1,9 @@ # HDFS Writer - ## 一、插件名称 -名称:**hdfswriter**
+名称:**hdfswriter** + - ## 二、数据源版本 | 协议 | 是否支持 | | --- | --- | @@ -13,100 +12,104 @@ - ## 三、数据源配置 -单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html)
集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html)
+单机模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/SingleCluster.html) +集群模式:[地址](http://hadoop.apache.org/docs/r2.7.6/hadoop-project-dist/hadoop-common/ClusterSetup.html) + - ## 四、参数说明 - **defaultFS** - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **hadoopConfig** - 描述:集群HA模式时需要填写的namespace配置及其它配置 - 必选:否 + - 字段类型:map - 默认值:无 +
+- **fileType** + - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` + - text:textfile文件格式 + - orc:orcfile文件格式 + - parquet:parquet文件格式 + - 必选:是 + - 字段类型:string + - 默认值:无 + + +
- **path** - 描述:数据文件的路径 - 必选:是 + - 字段类型:string - 默认值:无 +
+ +- **fileName** + - 描述:写入的目录名称 + - 注意:不为空,写入的路径为 path+fileName + - 必须:否 + - 字段类型:string + - 默认值:无 +
- **filterRegex** - 描述:文件过滤正则表达式 - 必选:否 + - 字段类型:string - 默认值:无 - - -- **fileType** - - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` - - text:textfile文件格式 - - orc:orcfile文件格式 - - parquet:parquet文件格式 - - 必选:否 - - 默认值:text - - +
- **fieldDelimiter** - 描述:`fileType`为`text`时字段的分隔符 - 必选:否 + - 字段类型:string - 默认值:`\001` - +
- **encoding** - 描述:`fileType`为`text`时可配置编码格式 - 必选:否 + - 字段类型:string - 默认值:UTF-8 - +
- **maxFileSize** - 描述:写入hdfs单个文件最大大小,单位字节 - 必须:否 + - 字段类型:long - 默认值:1073741824‬(1G) - +
- **compress** - - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 + - 描述:hdfs文件压缩类型 - text:支持`GZIP`、`BZIP2`格式 - - orc:支持`SNAPPY`、`GZIP`、`BZIP`、`LZ4`格式 - - parquet:支持`SNAPPY`、`GZIP`、`LZO` + - orc:支持`SNAPPY`、`ZLIB`、`LZO`格式 + - parquet:支持`SNAPPY`、`GZIP`、`LZO`格式 - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** - 必选:否 - - 默认值:无 - - - -- **compress** - - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 - - text:支持`GZIP`、`BZIP2`格式 - - orc:支持`SNAPPY`、`GZIP`、`BZIP`、`LZ4`格式 - - parquet:支持`SNAPPY`、`GZIP`、`LZO` - - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** - - 必选:否 - - 默认值:无 - - - -- **fileName** - - 描述:写入的目录名称 - - 必须:否 - - 默认值:无 - + - 字段类型:string + - 默认值: + - text 默认 不进行压缩 + - orc 默认为ZLIB格式 + - parquet 默认为SNAPPY格式 +
- **writeMode** - 描述:hdfswriter写入前数据清理处理模式: @@ -114,9 +117,10 @@ - overwrite:覆盖 - 注意:overwrite模式时会删除hdfs当前目录下的所有文件 - 必选:否 + - 字段类型:string - 默认值:append - +
- **column** - 描述:需要读取的字段。 @@ -128,261 +132,232 @@ }] ``` - - 属性说明: - - name:字段名称 - - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 - - 必选:是 - - 默认值:无 +- 属性说明: + - name:字段名称 + - type:字段类型,可以和源字段类型不一样,程序会做一次类型转换 +- 必选:是 +- 默认值:无 + +
+ +- **fullColumnName** + - 描述:写入的字段名称 + - 必须:否 + - 字段类型:list + - 默认值:column的name集合 +
+- **fullColumnType** + - 描述:写入的字段类型 + - 必须:否 + - 字段类型:list + - 默认值:column的type集合 + +
+ +- **rowGroupSIze** + - 描述:parquet类型文件参数,指定row group的大小,单位字节 + - 必须:否 + - 字段类型:int + - 默认值:134217728(128M) + +
+ +- **enableDictionary** + - 描述:parquet类型文件参数,是否启动字典编码 + - 必须:否 + - 字段类型:boolean + - 默认值:true - ## 五、使用示例 - #### 1、写入text文件 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - }, - { - "name": "col3", - "type": "int" - }, - { - "name": "col4", - "type": "int" - } - ], - "sliceRecordCount": [ - "100" - ] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "path": "hdfs://ns1/flinkx/text", - "defaultFS": "hdfs://ns1", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - }, - { - "name": "col3", - "index": 2, - "type": "int" - }, - { - "name": "col4", - "index": 3, - "type": "int" - } - ], - "fieldDelimiter": ",", - "fileType": "text", - "writeMode": "append" - }, - "name": "hdfswriter" - } - } - ], - "setting": { - "speed": { - "bytes": 0, - "channel": 1 - } + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/dev.db/test_text", + "fileName": "pt=20201214", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "\u0001", + "encoding": "utf-8", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "host1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "host2:9000", + "dfs.nameservices" : "ns1" + }, + "fileType": "text" + }, + "name": "hdfswriter" } + } + ], + "setting": { + "restore": { + "isRestore": false + } } + } } ``` - #### 2、写入orc文件 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - }, - { - "name": "col3", - "type": "int" - }, - { - "name": "col4", - "type": "int" - } - ], - "sliceRecordCount": [ - "100" - ] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "path": "hdfs://ns1/flinkx/text", - "defaultFS": "hdfs://ns1", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - }, - { - "name": "col3", - "index": 2, - "type": "int" - }, - { - "name": "col4", - "index": 3, - "type": "int" - } - ], - "fileType": "orc", - "writeMode": "append" - }, - "name": "hdfswriter" - } - } - ], - "setting": { - "speed": { - "bytes": 0, - "channel": 1 - } + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/dev.db/test_orc", + "fileName": "pt=20201214", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "\u0001", + "encoding": "utf-8", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "host1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "host2:9000", + "dfs.nameservices" : "ns1" + }, + "fileType": "orc" + }, + "name": "hdfswriter" } + } + ], + "setting": { + "restore": { + "isRestore": false + } } + } } ``` - #### 3、写入parquet文件 ```json { - "job": { - "content": [ - { - "reader": { - "parameter": { - "column": [ - { - "name": "col1", - "type": "string" - }, - { - "name": "col2", - "type": "string" - }, - { - "name": "col3", - "type": "int" - }, - { - "name": "col4", - "type": "int" - } - ], - "sliceRecordCount": [ - "100" - ] - }, - "name": "streamreader" - }, - "writer": { - "parameter": { - "path": "hdfs://ns1/flinkx/text", - "defaultFS": "hdfs://ns1", - "hadoopConfig": { - "dfs.ha.namenodes.ns1": "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2": "flinkx02:9000", - "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1": "flinkx01:9000", - "dfs.nameservices": "ns1" - }, - "column": [ - { - "name": "col1", - "index": 0, - "type": "string" - }, - { - "name": "col2", - "index": 1, - "type": "string" - }, - { - "name": "col3", - "index": 2, - "type": "int" - }, - { - "name": "col4", - "index": 3, - "type": "int" - } - ], - "fileType": "parquet", - "writeMode": "append" - }, - "name": "hdfswriter" - } - } - ], - "setting": { - "speed": { - "bytes": 0, - "channel": 1 - } + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "hdfs://ns1/user/hive/warehouse/dev.db/test_parquet", + "fileName": "pt=20201214", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "\u0001", + "encoding": "utf-8", + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.ha.namenodes.ns1" : "nn1,nn2", + "dfs.namenode.rpc-address.ns1.nn2" : "host1:9000", + "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn1" : "host2:9000", + "dfs.nameservices" : "ns1" + }, + "fileType": "parquet" + }, + "name": "hdfswriter" } + } + ], + "setting": { + "restore": { + "isRestore": false + } } + } } ``` diff --git a/docs/offline/writer/hivewriter.md b/docs/offline/writer/hivewriter.md index 49c9842da8..58b476d8ca 100644 --- a/docs/offline/writer/hivewriter.md +++ b/docs/offline/writer/hivewriter.md @@ -1,34 +1,37 @@ # Hive Writer - ## 一、插件名称 -名称:**hivewriter**
- +名称:**hivewriter** + ## 二、支持的数据源版本 -**Hive 2.X**
- -## 三、参数说明
+**Hive 2.X** + +## 三、参数说明 + - **jdbcUrl** - 描述:连接Hive JDBC的字符串 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **username** - 描述:Hive认证用户名 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **password** - 描述:Hive认证密码 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **fileType** - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` @@ -36,17 +39,19 @@ - orc:orcfile文件格式 - parquet:parquet文件格式 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **fieldDelimiter** - 描述:hivewriter中`fileType`为`text`时字段的分隔符, - 注意:用户需要保证与创建的Hive表的字段分隔符一致,否则无法在Hive表中查到数据 - - 必选:是 + - 必选:否 + - 字段类型:string - 默认值:`\u0001` - +
- **writeMode** - 描述:hivewriter写入前数据清理处理模式: @@ -54,9 +59,10 @@ - overwrite:覆盖 - 注意:overwrite模式时会删除Hive当前分区下的所有文件 - 必选:否 + - 字段类型:string - 默认值:append - +
- **compress** - 描述:hdfs文件压缩类型,默认不填写意味着没有压缩 @@ -65,26 +71,37 @@ - parquet:支持`SNAPPY`、`GZIP`、`LZO` - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** - 必选:否 - - 默认值:无 + - 字段类型:string + - 默认值: + - text 默认 不进行压缩 + - orc 默认为ZLIB格式 + - parquet 默认为SNAPPY格式 +
- **charsetName** - 描述:写入text文件的编码配置 - 必选:否 + - 字段类型:string - 默认值:UTF-8 - +
- **maxFileSize** - 描述:写入hdfs单个文件最大大小,单位字节 - 必须:否 + - 字段类型:string - 默认值:1073741824‬(1G) - +
- **tablesColumn** - - 描述:写入hive表的表结构信息,**若表不存在则会自动建表**。示例: + - 描述:写入hive表的表结构信息,**若表不存在则会自动建表**。 + - 必选:是 + - 字段类型:json + - 默认值:无 + - 示例: ```json { "kudu":[ @@ -104,45 +121,114 @@ } ``` - - 必选:是 - - 默认值:无 +
+- **distributeTable** + - 描述:如果数据来源于各个CDC数据,则将不同的表进行聚合,多张表的数据写入同一个hive表 + - 必选:否 + - 字段类型:json + - 默认值:无 + - 示例: +```json + "distributeTable" : "{\"fenzu1\":[\"table1\"],\"fenzu2\":[\"table2\",\"table3\"]}" +``` +table1的数据将写入hive表fenzu1里,table2和table3的数据将写入fenzu2里,如果配置distributeTable,则tablesColumn需要配置为如下格式: +```json +{ + "fenzu1":[ + { + "key":"id", + "type":"int" + }, + { + "key":"user_id", + "type":"int" + }, + { + "key":"name", + "type":"string" + } + ], + "fenzu2":[ + { + "key":"id", + "type":"int" + }, + { + "key":"user_id", + "type":"int" + }, + { + "key":"name", + "type":"string" + } + ] +} +``` +
- **partition** - 描述:分区字段名称 - - 必选:是 + - 必选:否 + - 字段类型:string - 默认值:`pt` - +
- **partitionType** - 描述:分区类型,包括 DAY、HOUR、MINUTE三种。**若分区不存在则会自动创建,自动创建的分区时间以当前任务运行的服务器时间为准** - - DAY:天分区,分区示例:pt=202000101 + - DAY:天分区,分区示例:pt=20200101 - HOUR:小时分区,分区示例:pt=2020010110 - MINUTE:分钟分区,分区示例:pt=202001011027 - - 必选:是 + - 必选:否 + - 字段类型:string - 默认值:无 - +
- **defaultFS** - - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 + - 描述:Hadoop hdfs文件系统namenode节点地址。取core-site.xml文件里fs.defaultFS配置值 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **hadoopConfig** - - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 描述:集群HA模式时需要填写的namespace配置及其它hive配置 + - 必选:否 + - 字段类型:map + - 默认值:无 + +
+ +- **rowGroupSIze** + - 描述:parquet格式文件的row group的大小,单位字节 + - 必选:否 + - 字段类型:int + - 默认值:134217728(128M) + +
+ +- **analyticalRules** + - 描述: 建表的动态规则获取表名,按照${XXXX}的占位符,从待写入数据(map结构)里根据key XXX 获取值进行替换,创建对应的表,并将数据写入对应的表 + - 示例:stream_${schema}_${table} + - 必选:否 + - 字段类型:string + - 默认值:无 + +
+ +- **schema** + - 描述: 自动建表时,analyticalRules里如果指定schema占位符,schema将此schema参数值进行替换 - 必选:否 + - 字段类型:string - 默认值:无 - ## 四、配置示例 - #### 1、写入text ```json { @@ -186,13 +272,56 @@ "partitionType" : "DAY", "defaultFS" : "hdfs://ns1", "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + "javax.jdo.option.ConnectionDriverName": "com.mysql.jdbc.Driver", + "dfs.replication": "2", + "dfs.ha.fencing.ssh.private-key-files": "~/.ssh/id_rsa", + "dfs.nameservices": "ns1", + "fs.hdfs.impl.disable.cache": "true", + "dfs.safemode.threshold.pct": "0.5", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.journalnode.rpc-address": "0.0.0.0:8485", + "dfs.journalnode.http-address": "0.0.0.0:8480", + "dfs.namenode.rpc-address.ns1.nn2": "kudu2new:9000", + "dfs.namenode.rpc-address.ns1.nn1": "kudu1new:9000", + "hive.metastore.warehouse.dir": "/user/hive/warehouse", + "hive.server2.webui.host": "172.16.10.34", + "hive.metastore.schema.verification": "false", + "hive.server2.support.dynamic.service.discovery": "true", + "javax.jdo.option.ConnectionPassword": "abc123", + "hive.metastore.uris": "thrift://kudu1new:9083", + "hive.exec.dynamic.partition.mode": "nonstrict", + "hadoop.proxyuser.admin.hosts": "*", + "hive.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "ha.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "hive.server2.thrift.min.worker.threads": "200", + "hive.server2.webui.port": "10002", + "fs.defaultFS": "hdfs://ns1", + "hadoop.proxyuser.admin.groups": "*", + "dfs.ha.fencing.methods": "sshfence", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "typeName": "yarn2-hdfs2-hadoop2", + "hadoop.proxyuser.root.groups": "*", + "javax.jdo.option.ConnectionURL": "jdbc:mysql://kudu2new:3306/ide?useSSL=false", + "dfs.qjournal.write-txns.timeout.ms": "60000", + "fs.trash.interval": "30", + "hadoop.proxyuser.root.hosts": "*", + "dfs.namenode.shared.edits.dir": "qjournal://kudu1new:8485;kudu2new:8485;kudu3new:8485/namenode-ha-data", + "javax.jdo.option.ConnectionUserName": "dtstack", + "hive.server2.thrift.port": "10000", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem", + "ha.zookeeper.session-timeout.ms": "5000", + "hadoop.tmp.dir": "/data/hadoop_root", + "dfs.journalnode.edits.dir": "/data/dtstack/hadoop/journal", + "hive.server2.zookeeper.namespace": "hiveserver2", + "hive.server2.enable.doAs": "/false", + "dfs.namenode.http-address.ns1.nn2": "kudu2new:50070", + "dfs.namenode.http-address.ns1.nn1": "kudu1new:50070", + "hive.exec.scratchdir": "/user/hive/warehouse", + "hive.server2.webui.max.threads": "100", + "datanucleus.schema.autoCreateAll": "true", + "hive.exec.dynamic.partition": "true", + "hive.server2.thrift.bind.host": "kudu1", + "dfs.ha.automatic-failover.enabled": "true" } } } @@ -217,7 +346,6 @@ } } ``` - #### 2、写入orc ```json { @@ -261,13 +389,56 @@ "partitionType" : "DAY", "defaultFS" : "hdfs://ns1", "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + "javax.jdo.option.ConnectionDriverName": "com.mysql.jdbc.Driver", + "dfs.replication": "2", + "dfs.ha.fencing.ssh.private-key-files": "~/.ssh/id_rsa", + "dfs.nameservices": "ns1", + "fs.hdfs.impl.disable.cache": "true", + "dfs.safemode.threshold.pct": "0.5", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.journalnode.rpc-address": "0.0.0.0:8485", + "dfs.journalnode.http-address": "0.0.0.0:8480", + "dfs.namenode.rpc-address.ns1.nn2": "kudu2new:9000", + "dfs.namenode.rpc-address.ns1.nn1": "kudu1new:9000", + "hive.metastore.warehouse.dir": "/user/hive/warehouse", + "hive.server2.webui.host": "172.16.10.34", + "hive.metastore.schema.verification": "false", + "hive.server2.support.dynamic.service.discovery": "true", + "javax.jdo.option.ConnectionPassword": "abc123", + "hive.metastore.uris": "thrift://kudu1new:9083", + "hive.exec.dynamic.partition.mode": "nonstrict", + "hadoop.proxyuser.admin.hosts": "*", + "hive.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "ha.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "hive.server2.thrift.min.worker.threads": "200", + "hive.server2.webui.port": "10002", + "fs.defaultFS": "hdfs://ns1", + "hadoop.proxyuser.admin.groups": "*", + "dfs.ha.fencing.methods": "sshfence", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "typeName": "yarn2-hdfs2-hadoop2", + "hadoop.proxyuser.root.groups": "*", + "javax.jdo.option.ConnectionURL": "jdbc:mysql://kudu2new:3306/ide?useSSL=false", + "dfs.qjournal.write-txns.timeout.ms": "60000", + "fs.trash.interval": "30", + "hadoop.proxyuser.root.hosts": "*", + "dfs.namenode.shared.edits.dir": "qjournal://kudu1new:8485;kudu2new:8485;kudu3new:8485/namenode-ha-data", + "javax.jdo.option.ConnectionUserName": "dtstack", + "hive.server2.thrift.port": "10000", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem", + "ha.zookeeper.session-timeout.ms": "5000", + "hadoop.tmp.dir": "/data/hadoop_root", + "dfs.journalnode.edits.dir": "/data/dtstack/hadoop/journal", + "hive.server2.zookeeper.namespace": "hiveserver2", + "hive.server2.enable.doAs": "/false", + "dfs.namenode.http-address.ns1.nn2": "kudu2new:50070", + "dfs.namenode.http-address.ns1.nn1": "kudu1new:50070", + "hive.exec.scratchdir": "/user/hive/warehouse", + "hive.server2.webui.max.threads": "100", + "datanucleus.schema.autoCreateAll": "true", + "hive.exec.dynamic.partition": "true", + "hive.server2.thrift.bind.host": "kudu1", + "dfs.ha.automatic-failover.enabled": "true" } } } @@ -292,7 +463,6 @@ } } ``` - #### 3、写入parquet ```json { @@ -336,13 +506,56 @@ "partitionType" : "DAY", "defaultFS" : "hdfs://ns1", "hadoopConfig" : { - "dfs.ha.namenodes.ns1" : "nn1,nn2", - "dfs.namenode.rpc-address.ns1.nn2" : "kudu2:9000", - "dfs.client.failover.proxy.provider.ns1" : "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", - "dfs.namenode.rpc-address.ns1.nn1" : "kudu1:9000", - "dfs.nameservices" : "ns1", - "fs.hdfs.impl.disable.cache" : "true", - "fs.hdfs.impl" : "org.apache.hadoop.hdfs.DistributedFileSystem" + "javax.jdo.option.ConnectionDriverName": "com.mysql.jdbc.Driver", + "dfs.replication": "2", + "dfs.ha.fencing.ssh.private-key-files": "~/.ssh/id_rsa", + "dfs.nameservices": "ns1", + "fs.hdfs.impl.disable.cache": "true", + "dfs.safemode.threshold.pct": "0.5", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.journalnode.rpc-address": "0.0.0.0:8485", + "dfs.journalnode.http-address": "0.0.0.0:8480", + "dfs.namenode.rpc-address.ns1.nn2": "kudu2new:9000", + "dfs.namenode.rpc-address.ns1.nn1": "kudu1new:9000", + "hive.metastore.warehouse.dir": "/user/hive/warehouse", + "hive.server2.webui.host": "172.16.10.34", + "hive.metastore.schema.verification": "false", + "hive.server2.support.dynamic.service.discovery": "true", + "javax.jdo.option.ConnectionPassword": "abc123", + "hive.metastore.uris": "thrift://kudu1new:9083", + "hive.exec.dynamic.partition.mode": "nonstrict", + "hadoop.proxyuser.admin.hosts": "*", + "hive.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "ha.zookeeper.quorum": "kudu1new:2181,kudu2new:2181,kudu3new:2181", + "hive.server2.thrift.min.worker.threads": "200", + "hive.server2.webui.port": "10002", + "fs.defaultFS": "hdfs://ns1", + "hadoop.proxyuser.admin.groups": "*", + "dfs.ha.fencing.methods": "sshfence", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "typeName": "yarn2-hdfs2-hadoop2", + "hadoop.proxyuser.root.groups": "*", + "javax.jdo.option.ConnectionURL": "jdbc:mysql://kudu2new:3306/ide?useSSL=false", + "dfs.qjournal.write-txns.timeout.ms": "60000", + "fs.trash.interval": "30", + "hadoop.proxyuser.root.hosts": "*", + "dfs.namenode.shared.edits.dir": "qjournal://kudu1new:8485;kudu2new:8485;kudu3new:8485/namenode-ha-data", + "javax.jdo.option.ConnectionUserName": "dtstack", + "hive.server2.thrift.port": "10000", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem", + "ha.zookeeper.session-timeout.ms": "5000", + "hadoop.tmp.dir": "/data/hadoop_root", + "dfs.journalnode.edits.dir": "/data/dtstack/hadoop/journal", + "hive.server2.zookeeper.namespace": "hiveserver2", + "hive.server2.enable.doAs": "/false", + "dfs.namenode.http-address.ns1.nn2": "kudu2new:50070", + "dfs.namenode.http-address.ns1.nn1": "kudu1new:50070", + "hive.exec.scratchdir": "/user/hive/warehouse", + "hive.server2.webui.max.threads": "100", + "datanucleus.schema.autoCreateAll": "true", + "hive.exec.dynamic.partition": "true", + "hive.server2.thrift.bind.host": "kudu1", + "dfs.ha.automatic-failover.enabled": "true" } } } diff --git a/docs/offline/writer/kingbasewriter.md b/docs/offline/writer/kingbasewriter.md index 5844b52b90..62eac805a8 100644 --- a/docs/offline/writer/kingbasewriter.md +++ b/docs/offline/writer/kingbasewriter.md @@ -6,15 +6,48 @@ **KingBase 8.2及8.3**
## 三、参数说明 -- jdbcUrl + +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:kingbase8://localhost:54321/database", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ +- **jdbcUrl** - 描述:针对KingBase数据库的jdbc连接字符串 - 必选:是 - 字段类型:String - 默认值:无
+ + - **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
-- username +- **username** - 描述:数据源的用户名 - 必选:是 - 字段类型:String @@ -22,23 +55,15 @@
-- password +- **password** - 描述:数据源指定用户名的密码 - 必选:是 - 字段类型:String - 默认值:无
- -- schema - - 描述:写入数据库所在schema - - 必选:是 - - 字段类型:String - - 默认值:无 - -
- -- column + +- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - 默认值:否 @@ -46,32 +71,32 @@ - 默认值:无
+ + - **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
-- preSql +- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 - - 字段类型:List + - 字段类型:String - 默认值:无
-- postSql +- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 - - 字段类型:List - - 默认值:无 - -
- -- table - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - 字段类型:String - 默认值:无
-- writeMode +- **writeMode** - 描述:仅支持insert、update操作,可以搭配insertSqlMode使用 - 必选:是 - 字段类型:String @@ -79,16 +104,16 @@
-- insertSqlMode +- **insertSqlMode** - 描述:控制写入数据到目标表采用 COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'语句,提高数据的插入效率 - 注意: 目前该参数值固定传入 copy,否则抛出提示为not support insertSqlMode的RuntimeException。当指定此参数时,writeMode的值必须为 insert,否则设置无效 - 必选:否 - 字段类型:String - - 默认值:无 + - 默认值:copy
-- batchSize +- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 - 字段类型:int @@ -96,9 +121,18 @@
+- **updateKey** + - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 + - 注意: + - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; + - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} + - 默认值:无 + ## 四、配置示例 1、insert -``` +```json { "job": { "content": [{ @@ -134,19 +168,7 @@ }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "varchar" - }], + "column": ["id","user_id","name"], "writeMode": "insert", "batchSize": 1024, "preSql": [], @@ -167,7 +189,7 @@ } ``` 2、 insert with copy mode -``` +```json { "job": { "content": [{ @@ -203,19 +225,7 @@ }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "varchar" - }], + "column": ["id","user_id","name"], "writeMode": "insert", "batchSize": 1024, "preSql": [], @@ -237,7 +247,7 @@ } ``` 3、 update -``` +```json { "job": { "content": [{ @@ -265,30 +275,15 @@ "name": "kingbasewriter", "parameter": { "connection": [{ - "jdbcUrl": "jdbc:kingbase8://localhost:54321/ide", - "table": [ - "tableTest" - ], + "jdbcUrl": "jdbc:kingbase8://localhost:54321/database", + "table": ["tableTest"], "schema":"test" }], "username": "username", "password": "password", - "column": [ - { - "name": "id", - "type": "BIGINT" - }, - { - "name": "user_id", - "type": "BIGINT" - }, - { - "name": "name", - "type": "varchar" - }], + "column": ["id","user_id","name"], "writeMode": "update", "updateKey": {"key": ["id"]}, - "column": ["id","user_id","name"], "batchSize": 1024, "preSql": [], "postSql": [] diff --git a/docs/offline/writer/kuduwriter.md b/docs/offline/writer/kuduwriter.md index 8bba48abf3..74597501a3 100644 --- a/docs/offline/writer/kuduwriter.md +++ b/docs/offline/writer/kuduwriter.md @@ -1,38 +1,49 @@ # Kudu Writer - ## 一、插件名称 -名称:**kuduwriter**
- -## 二、支持的数据源版本 -**kudu 1.10及以上**
+名称:**kuduwriter** - -## 三、参数说明 +## 二、支持的数据源版本 +**kudu 1.10及以上** -- **column** - - 描述:需要生成的字段 - - 属性说明: - - name:字段名称; - - type:字段类型; - - 必选:是 - - 默认值:无 +## 三、参数说明 - **masterAddresses** - 描述: master节点地址:端口,多个以,隔开 - 必选:是 + - 参数类型:string - 默认值:无 - +
- **table** - 描述: kudu表名 - 必选:是 + - 参数类型:string - 默认值:无 +
+ +- **column** + - 描述:需要生成的字段 + - 格式 +```json +"column": [{ + "name": "col", + "type": "string" +}] +``` + +- 属性说明: + - name:字段名称 + - type:字段类型 +- 必选:是 +- 参数类型:数组 +- 默认值:无 +
- **writeMode** - 描述: kudu数据写入模式: @@ -40,153 +51,139 @@ - 2、update - 3、upsert - 必选:是 + - 参数类型:string - 默认值:无 - +
- **flushMode** - 描述: kudu session刷新模式: - - 1、auto_flush_sync - - 2、auto_flush_background - - 3、manual_flush + - 1、auto_flush_sync 同步刷新 + - 2、auto_flush_background 后台自动刷新 + - 3、manual_flush 手动刷新 - 必选:否 + - 参数类型:string - 默认值:auto_flush_sync - +
- **batchInterval** - 描述: 单次批量写入数据条数 - 必选:否 + - 参数类型:int - 默认值:1 - +
- **authentication** - - 描述: 认证方式,如:Kerberos + - 描述:认证方式,kudu开启kerberos时需要配置authentication为Kerberos - 必选:否 + - 参数类型:string - 默认值:无 - - -- **principal** - - 描述: 用户名。 - - 必选:否 - - 默认值:无 - - - -- **keytabFile** - - 描述: keytab文件路径 - - 必选:否 - - 默认值:无 - - +
- **workerCount** - - 描述: worker线程数 + - 描述:worker线程数 - 必选:否 - - 默认值:默认为cpu*2 - + - 字段类型:int + - 默认值:默认为cpu核心数*2 +
- **bossCount** - - 描述: boss线程数 + - 描述:boss线程数 - 必选:否 + - 字段类型:int - 默认值:1 - +
- **operationTimeout** - - 描述: 普通操作超时时间 + - 描述:普通操作超时时间,单位毫秒 - 必选:否 + - 字段类型:long - 默认值:30000 - +
- **adminOperationTimeout** - - 描述: 管理员操作(建表,删表)超时时间 + - 描述: 管理员操作(建表,删表)超时时间,单位毫秒 - 必选:否 - - 默认值:30000 + - 字段类型:long + - 默认值:15000 +
- -- **queryTimeout** - - 描述: 连接scan token的超时时间 - - 必选:否 - - 默认值:与operationTimeout一致 - - - -- **batchSizeBytes** - - 描述: kudu scan一次性最大读取字节数 +- **hadoopConfig** + - 描述: kudu开启kerberos,需要配置kerberos相关参数 - 必选:否 - - 默认值:1048576 + - 字段类型:map + - 默认值:无 - ## 四、配置示例 ```json { "job" : { "content" : [ { - "reader" : { - "parameter" : { - "column" : [ { - "name" : "id", - "type" : "id" - }, { - "name" : "data", - "type" : "string" - } ], - "sliceRecordCount" : [ "100"] - }, - "name" : "streamreader" + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "string" + }, { + "name": "name", + "type": "string" + }, { + "name": "age", + "type": "int" + }, { + "name": "sex", + "type": "int" + } + ], + "sliceRecordCount" : [100] + } }, "writer" : { "parameter": { "column": [ { "name": "id", - "type": "long" + "type": "string" + }, { + "name": "name", + "type": "string" + }, { + "name": "age", + "type": "int" + }, { + "name": "sex", + "type": "int" } ], - "masterAddresses": "kudu1:7051,kudu2:7051,kudu3:7051", - "table": "kudu", + "masterAddresses": "host:7051", + "table": "student", "writeMode": "insert", "flushMode": "manual_flush", "batchInterval": 10000, - "authentication": "", - "principal": "", - "keytabFile": "", "workerCount": 2, - "bossCount": 1, - "operationTimeout": 30000, - "adminOperationTimeout": 30000, - "queryTimeout": 30000, - "batchSizeBytes": 1048576 - } + "bossCount": 1 + }, + "name": "kuduwriter" } } ], - "setting" : { - "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 - }, - "speed" : { - "bytes" : 0, - "channel" : 1 + "setting": { + "speed": { + "channel": 1 }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" + "restore": { + "isRestore": false, + "isStream" : false } } } diff --git a/docs/offline/writer/mongodbwriter.md b/docs/offline/writer/mongodbwriter.md index fc49b2cd74..ffc892853c 100644 --- a/docs/offline/writer/mongodbwriter.md +++ b/docs/offline/writer/mongodbwriter.md @@ -12,75 +12,95 @@ - **url** - 描述:MongoDB数据库连接的URL字符串,详细请参考[MongoDB官方文档](https://docs.mongodb.com/manual/reference/connection-string/) - 必选:否 + - 字段类型:String - 默认值:无 +
- **hostPorts** - 描述:MongoDB的地址和端口,格式为 IP1:port,可填写多个地址,以英文逗号分隔 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **database** - 描述:数据库名称 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **collectionName** - 描述:集合名称 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:MongoDB 的文档列名,配置为数组形式表示 MongoDB 的多个列 - name:Column 的名字 - type:Column 的类型 - - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中 + - splitter:特殊分隔符,当且仅当要处理的字符串要用分隔符分隔为字符数组 Array 时,才使用这个参数。通过这个参数指定的分隔符,将字符串分隔存储到 MongoDB 的数组中 + 当指定了这个参数,写入mongodb的数组类型只能为string + 示例 + ``` + "column": [{ + "name": "col", + "type": "Array", + "splitter":"," + }] + ``` - 必选:是 + - 字段类型:List - 默认值:无 - +
- **replaceKey** - 描述:replaceKey 指定了每行记录的业务主键,用来做覆盖时使用(不支持 replaceKey为多个键,一般是指Monogo中的主键) - 必选:否 + - 字段类型:String - 默认值:无 - +
- **writeMode** - 描述:写入模式,当 batchSize > 1 时不支持 replace 和 update 模式 - 必选:是 - 所有选项:insert/replace/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与MongoDB的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1 - +
diff --git a/docs/offline/writer/mysqlwriter.md b/docs/offline/writer/mysqlwriter.md index c7342458cb..39d1bd2ff0 100644 --- a/docs/offline/writer/mysqlwriter.md +++ b/docs/offline/writer/mysqlwriter.md @@ -9,70 +9,110 @@ ## 三、参数说明
+- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - 必选:是 - 所有选项:insert/replace/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 @@ -80,6 +120,8 @@ - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 @@ -116,16 +158,16 @@ "writer": { "name": "mysqlwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", + "table": ["table"] } ], - "preSql": ["truncate table kudu;"], - "postSql": ["update kudu set user_id = 1;"], + "preSql": ["truncate table table;"], + "postSql": ["update table set user_id = 1;"], "writeMode": "insert", "column": ["id","user_id","name"], "batchSize": 1024 @@ -187,12 +229,12 @@ "writer": { "name": "mysqlwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", + "table": ["table"] } ], "preSql": [], @@ -259,12 +301,12 @@ "writer": { "name": "mysqlwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:mysql://0.0.0.1:3306/database?useSSL=false", + "table": ["table"] } ], "preSql": [], diff --git a/docs/offline/writer/odpswriter.md b/docs/offline/writer/odpswriter.md index 64e14e4c4e..76f5c80ca4 100644 --- a/docs/offline/writer/odpswriter.md +++ b/docs/offline/writer/odpswriter.md @@ -1,63 +1,104 @@ # ODPS Writer - ## 一、插件名称 -名称:**odpswriter**
- -## 二、参数说明 - -- **odpsConfig** - - 描述:ODPS系统配置参数,包含以下参数 - - accessId:ODPS系统登录ID - - accessKey:ODPS系统登录Key - - project:读取数据表所在的 ODPS 项目名称(大小写不敏感) - - packageAuthorizedProject:ODPS认证项目,不填默认为project值 - - accountType:ODPS账户类型,默认为aliyun - - odpsServer:ODPS服务URL,默认为[http://service.odps.aliyun.com/api](http://service.odps.aliyun.com/api) - - 必选:是 - - 默认值:无 +名称:**odpswriter** +## 二、参数说明 - **table** - 描述:读取数据表的表名称(大小写不敏感) - 必选:是 + - 字段类型:string - 默认值:无 - -
+
- **partition** - 描述:需要写入数据表的分区信息,必须指定到最后一级分区。把数据写入一个三级分区表,必须配置到最后一级分区,例如pt=20150101/type=1/biz=2。 - - 必选:**如果是分区表,该选项必填,如果非分区表,该选项不可填写。** - - 默认值:空 - + - 注意:**如果是分区表,该选项必填,如果非分区表,该选项不可填写** + - 必选:否 + - 字段类型:string + - 默认值:无 -
+
- **column** - - 描述:需要导入的字段列表,当导入全部字段时,可以配置为"column": ["*"], 当需要插入部分odps列填写部分列,例如"column": ["id", "name"]。ODPSWriter支持列筛选、列换序,例如表有a,b,c三个字段,用户只同步c,b两个字段。可以配置成["c","b"], 在导入过程中,字段a自动补空,设置为null。 - - 必选:否 - - 默认值:无 + - 描述:需要读取的字段 + - 格式: +```json +"column": [{ + "name": "col", + "type": "datetime" +}] +``` +- 属性说明: + - name:字段名称 必填 + - type:字段类型,可以和数据库里的字段类型不一样,程序会做一次类型转换 必填 +- 必选:是 +- 字段类型:数组 +- 默认值:无 -
+
- **writeMode** - 描述:写入模式,支持append和overwrite - 必填:否 + - 字段类型:string - 默认值:append - +
- **bufferSize** - - 描述:写入缓存大小,单位兆,odps写入数据时会先缓存,达到一定值后才会写入数据,如果写入数据时出现内存溢出,可以降低此参数的值。 + - 描述:写入缓存大小,单位M,odps写入数据时会先缓存,达到一定值后才会写入数据,如果写入数据时出现内存溢出,可以降低此参数的值。 - 必填:否 - - 默认值:64 + - 字段类型:long + - 默认值:64M + +
+ +- **odpsConfig** + - 描述:ODPS的配置信息 + - 必选:是 + - 字段类型 map + - 默认值:无 + - 可选配置: + - **odpsServer** + - 描述:odps服务地址 + - 必选:否 + - 字段类型 string + - 默认值:[http://service.odps.aliyun.com/api](http://service.odps.aliyun.com/api) + - **accessId** + - 描述:ODPS系统登录ID + - 必选:是 + - 字段类型 string + - 默认值:无 + - **accessKey** + - 描述:ODPS系统登录Key + - 必选:是 + - 字段类型 string + - 默认值:无 + - **project** + - 描述:读取数据表所在的 ODPS 项目名称(大小写不敏感) + - 必选:是 + - 字段类型 string + - 默认值:无 + - **packageAuthorizedProject** + - 描述:ODPS认证项目 + - 注意:当 **packageAuthorizedProject **不为空时,当前project取packageAuthorizedProject对应值 而不是 project 对应的值 + - 必选:否 + - 字段类型 string + - 默认值:无 + - **accountType** + - 描述:odps账户类型 + - 注意:目前只支持 aliyun 类型 + - 必选:否 + - 字段类型 string + - 默认值:aliyun - ## 三、配置示例 ```json { @@ -94,23 +135,10 @@ } ], "setting" : { "restore" : { - "maxRowNumForCheckpoint" : 0, - "isRestore" : false, - "restoreColumnName" : "", - "restoreColumnIndex" : 0 - }, - "errorLimit" : { - "record" : 100 + "isRestore" : false }, "speed" : { - "bytes" : 0, "channel" : 1 - }, - "log" : { - "isLogger": false, - "level" : "debug", - "path" : "", - "pattern":"" } } } diff --git a/docs/offline/writer/oraclewriter.md b/docs/offline/writer/oraclewriter.md index 3161b3dd3d..1c21510781 100644 --- a/docs/offline/writer/oraclewriter.md +++ b/docs/offline/writer/oraclewriter.md @@ -9,70 +9,110 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:oracle:thin:0.0.0.1:1521:oracle", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 @@ -80,6 +120,8 @@ - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 ** @@ -115,16 +157,16 @@ "writer": { "name": "oraclewriter", "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:oracle:thin:@kudu5:1521:helowin", - "table": ["TUDOU.KUDU"] + "jdbcUrl": "jdbc:oracle:thin:0.0.0.1:1521:oracle", + "table": ["TABLE"] } ], - "preSql": ["delete from TUDOU.KUDU"], - "postSql": ["update TUDOU.KUDU set USER_ID = 1"], + "preSql": ["delete from TABLE"], + "postSql": ["update TABLE set USER_ID = 1"], "writeMode": "insert", "column": ["ID","USER_ID","NAME"], "batchSize": 1024 @@ -186,12 +228,12 @@ "writer": { "name": "oraclewriter", "parameter": { - "username": "tudou", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:oracle:thin:@kudu5:1521:helowin", - "table": ["TUDOU.KUDU"] + "jdbcUrl": "jdbc:oracle:thin:0.0.0.1:1521:oracle", + "table": ["TABLE"] } ], "preSql": [], diff --git a/docs/offline/writer/phoenixwriter.md b/docs/offline/writer/phoenixwriter.md index 88100c2ac3..e65be49e7d 100644 --- a/docs/offline/writer/phoenixwriter.md +++ b/docs/offline/writer/phoenixwriter.md @@ -9,80 +9,121 @@ phoenix4.12.0-HBase-1.3及以上
## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:phoenix:node01,node02,node03:2181", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - 默认值:否 + - 字段类型:List - 默认值:无 +
+ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `merge into` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 - 注意: - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 - - ## 四、配置示例 @@ -126,18 +167,15 @@ phoenix4.12.0-HBase-1.3及以上
"column": [ { "name": "id", - "type": "BIGINT", - "key": "id" + "type": "BIGINT" }, { "name": "user_id", - "type": "BIGINT", - "key": "user_id" + "type": "BIGINT" }, { "name": "name", - "type": "varchar", - "key": "name" + "type": "varchar" }], "writeMode": "insert", "batchSize": 1024, @@ -206,25 +244,21 @@ phoenix4.12.0-HBase-1.3及以上
"column": [ { "name": "id", - "type": "BIGINT", - "key": "id" + "type": "BIGINT" }, { "name": "user_id", - "type": "BIGINT", - "key": "user_id" + "type": "BIGINT" }, { "name": "name", - "type": "varchar", - "key": "name" + "type": "varchar" }], "writeMode": "update", "updateKey": {"key": ["id"]}, "batchSize": 1024, "preSql": [], - "postSql": [], - "updateKey": {} + "postSql": [] } } }], diff --git a/docs/offline/writer/polardbwriter.md b/docs/offline/writer/polardbwriter.md index 4028d820fc..26e4b0ca01 100644 --- a/docs/offline/writer/polardbwriter.md +++ b/docs/offline/writer/polardbwriter.md @@ -9,70 +9,110 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:polardb://0.0.0.1:3306/database", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String - 默认值:无 +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - 必选:是 - 所有选项:insert/replace/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 @@ -80,10 +120,10 @@ - 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 - - ## 四、配置示例 @@ -116,16 +156,16 @@ "writer": { "name": "polarwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:polardb://0.0.0.1:3306/database", + "table": ["table"] } ], - "preSql": ["truncate table kudu;"], - "postSql": ["update kudu set user_id = 1;"], + "preSql": ["truncate table table;"], + "postSql": ["update table set user_id = 1;"], "writeMode": "insert", "column": ["id","user_id","name"], "batchSize": 1024 @@ -187,12 +227,12 @@ "writer": { "name": "polardbwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:polardb://0.0.0.1:3306/database", + "table": ["table"] } ], "preSql": [], @@ -259,12 +299,12 @@ "writer": { "name": "polardbwriter", "parameter": { - "username": "dtstack", - "password": "abc123", + "username": "username", + "password": "password", "connection": [ { - "jdbcUrl": "jdbc:mysql://kudu3:3306/tudou?useSSL=false", - "table": ["kudu"] + "jdbcUrl": "jdbc:polardb://0.0.0.1:3306/database", + "table": ["table"] } ], "preSql": [], diff --git a/docs/offline/writer/postgresqlwriter.md b/docs/offline/writer/postgresqlwriter.md index 4df7a7c9c5..69a008b25c 100644 --- a/docs/offline/writer/postgresqlwriter.md +++ b/docs/offline/writer/postgresqlwriter.md @@ -9,62 +9,101 @@ ## 三、参数说明
+- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:postgresql://0.0.0.1:5432/postgres", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String - 默认值:无 +
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:仅支持`insert`操作,可以搭配insertSqlMode使用 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **insertSqlMode** - 描述:控制写入数据到目标表采用  `COPY table_name [ ( column_name [, ...] ) ] FROM STDIN DELIMITER 'delimiter_character'`语句,提高数据的插入效率 @@ -73,14 +112,17 @@ - 目前该参数值固定传入 `copy`,否则抛出提示为`not support insertSqlMode`的`RuntimeException` - 当指定此参数时,writeMode的值必须为 `insert`,否则设置无效 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 + ** diff --git a/docs/offline/writer/rediswriter.md b/docs/offline/writer/rediswriter.md index 416981a19a..79b2a65ece 100644 --- a/docs/offline/writer/rediswriter.md +++ b/docs/offline/writer/rediswriter.md @@ -13,35 +13,40 @@ - 描述:Redis的IP地址和端口 - 必选:是 - 默认值:localhost:6379 - + - 字段类型:String +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 - 默认值:无 - + - 字段类型:String +
- **database** - 描述:要写入Redis数据库 - 必选:否 - 默认值:0 - + - 字段类型:Integer +
- **keyFieldDelimiter** - 描述:写入 Redis 的 key 分隔符。比如: key=key1\u0001id,如果 key 有多个需要拼接时,该值为必填项,如果 key 只有一个则可以忽略该配置项。 - 必选:否 - 默认值:\u0001 - + - 字段类型:String +
- **dateFormat** - 描述:写入 Redis 时,Date 的时间格式:”yyyy-MM-dd HH:mm:ss” - 必选:否 - 默认值:将日期以long类型写入 - + - 字段类型:String +
- **expireTime** @@ -49,7 +54,8 @@ - 注意:如果过期时间的秒数大于 60_60_24*30(即 30 天),则服务端认为是 Unix 时间,该时间指定了到未来某个时刻数据失效。否则为相对当前时间的秒数,该时间指定了从现在开始多长时间后数据失效。 - 必选:否 - 默认值:0(0 表示永久有效) - + - 字段类型:Long +
- **timeout** @@ -57,7 +63,8 @@ - 单位:毫秒 - 必选:否 - 默认值:30000 - + - 字段类型:Long +
- **type和mode** @@ -71,17 +78,18 @@ | set | 字符串集合 | sadd | 向 set 集合中存储这个数据,如果已经存在则覆盖 | | | zset | 有序字符串集合 | zadd | 向 zset 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 zset 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 score 和 value,并且 score 必须在 value 前面,rediswriter 方能解析出哪一个 column 是 score,哪一个 column 是 value。 | | hash | 哈希 | hset | 向 hash 有序集合中存储这个数据,如果已经存在则覆盖 | 当 value 类型是 hash 时,数据源的每一行记录需要遵循相应的规范,即每一行记录除 key 以外,只能有一对 attribute 和 value,并且 attribute 必须在 value 前面,Rediswriter 方能解析出哪一个 column 是 attribute,哪一个 column 是 value。 | - - 必选:是 - 默认值:无 - + - 字段类型:String +
- **valueFieldDelimiter** - 描述:该配置项是考虑了当源数据每行超过两列的情况(如果您的源数据只有两列即 key 和 value 时,那么可以忽略该配置项,不用填写),value 类型是 string 时,value 之间的分隔符,比如 value1\u0001value2\u0001value3。 - 必选:否 - 默认值:\u0001 - + - 字段类型:String +
- **keyIndexes** @@ -89,10 +97,10 @@ - 注意:配置 keyIndexes 后,Redis Writer 会将其余的列作为 value,如果您只想同步源表的某几列作为 key,某几列作为 value,不需要同步所有字段,那么在 Reader 插件端就指定好 column 作好列筛选即可。例如:Redis中的数据为 "test,redis,First,Second",keyIndexes = [0,1] ,因此得到的key为 "test\\u0001redis", value为 "First\\u0001Second" - 必选:是 - 默认值:无 + - 字段类型:List +
-
- ## 四、 使用示例 ```json diff --git a/docs/offline/writer/saphanawriter.md b/docs/offline/writer/saphanawriter.md index 487c69e0b7..86daa8c031 100644 --- a/docs/offline/writer/saphanawriter.md +++ b/docs/offline/writer/saphanawriter.md @@ -9,70 +9,110 @@ SAP HANA 2.0及以上
## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:sap://0.0.0.1:39017", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:针对关系型数据库的jdbc连接字符串 - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String - 默认值:无 +
+- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +
+ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 +
- **preSql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 - - -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者 `replace into` 或者 `ON DUPLICATE KEY UPDATE` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 - +
- **updateKey** - 描述:当写入模式为update和replace时,需要指定此参数的值为唯一索引字段 @@ -80,10 +120,10 @@ SAP HANA 2.0及以上
- 如果此参数为空,并且写入模式为update和replace时,应用会自动获取数据库中的唯一索引; - 如果数据表没有唯一索引,但是写入模式配置为update和replace,应用会以insert的方式写入数据; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 - - ## 四、配置示例 ```json @@ -109,9 +149,9 @@ SAP HANA 2.0及以上
"parameter": { "connection": [ { - "jdbcUrl": "jdbc:sap://kudu3:39017", + "jdbcUrl": "jdbc:sap://0.0.0.1:39017", "table": [ - "SYS.P_ROLES_" + "TABLE" ] } ], diff --git a/docs/offline/writer/sqlserverwriter.md b/docs/offline/writer/sqlserverwriter.md index 4c3317ab49..95181476be 100644 --- a/docs/offline/writer/sqlserverwriter.md +++ b/docs/offline/writer/sqlserverwriter.md @@ -9,38 +9,82 @@ ## 三、参数说明 +- **connection** + - 描述:数据库连接参数,包含jdbcUrl、schema、table等参数 + - 必选:是 + - 字段类型:List + - 示例:指定jdbcUrl、schema、table + ```json + "connection": [{ + "jdbcUrl": "jdbc:jtds:sqlserver://0.0.0.1:1433;DatabaseName=DTstack", + "table": ["table"], + "schema":"public" + }] + ``` + - 默认值:无 + +
+ - **jdbcUrl** - 描述:使用开源的jtds驱动连接 而非Microsoft的官方驱动
jdbcUrl参考文档:[jtds驱动官方文档](http://jtds.sourceforge.net/faq.html) - 必选:是 + - 字段类型:String - 默认值:无 +
+- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 + - 必选:是 + - 字段类型:List + - 默认值:无 + +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **column** - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] - 必选:是 - - 默认值:否 + - 字段类型:List - 默认值:无 +<>br/ +- **fullcolumn** + - 描述:目的表中的所有字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age","hobby"],如果不配置,将在系统表中获取 + - 必选:否 + - 字段类型:List + - 默认值:无 + +
- **presql** - 描述:写入数据到目的表前,会先执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无
@@ -48,37 +92,43 @@ - **postSql** - 描述:写入数据到目的表后,会执行这里的一组标准语句 - 必选:否 + - 字段类型:String - 默认值:无 +
+- **withNoLock** + - 描述:是否在sql语句后面添加 with(nolock) + - 必选:否 + - 字段类型:Boolean + - 默认值:false -- **table** - - 描述:目的表的表名称。目前只支持配置单个表,后续会支持多表 - - 必选:是 - - 默认值:无 - - +
- **writeMode** - 描述:控制写入数据到目标表采用 `insert into` 或者` merge into` 语句 - 必选:是 - 所有选项:insert/update + - 字段类型:String - 默认值:insert - +
- **updateKey** - 描述:当写入模式为update时,需要指定此参数的值为唯一索引字段 - 注意: - 采用`merge into`语法,对目标表进行匹配查询,匹配成功时更新,不成功时插入; - 必选:否 + - 字段类型:Map + - 示例:"updateKey": {"key": ["id"]} - 默认值:无 - +
- **batchSize** - 描述:一次性批量提交的记录数大小,该值可以极大减少FlinkX与数据库的网络交互次数,并提升整体吞吐量。但是该值设置过大可能会造成FlinkX运行进程OOM情况 - 必选:否 + - 字段类型:int - 默认值:1024 diff --git a/docs/quickstart.md b/docs/quickstart.md index 7fda36c0d9..a790207976 100644 --- a/docs/quickstart.md +++ b/docs/quickstart.md @@ -1,3 +1,19 @@ +目录: + + +- [下载代码](#下载代码) +- [编译插件](#编译插件) + - [1.编译找不到DB2、达梦、gbase、ojdbc8、kingbase、vertica等驱动包](#1编译找不到db2达梦gbaseojdbc8kingbasevertica等驱动包) + - [2.编译报错找不到其他包](#2编译报错找不到其他包) +- [运行任务](#运行任务) + - [Local模式运行任务](#local模式运行任务) + - [Standalone模式运行](#standalone模式运行) + - [以Yarn Session模式运行任务](#以yarn-session模式运行任务) + - [以Yarn Perjob模式运行任务](#以yarn-perjob模式运行任务) +- [参数说明](#参数说明) + + + ## 下载代码 1.使用git工具把项目clone到本地 @@ -15,24 +31,72 @@ unzip 1.10_release.zip cd 1.10_release ``` -3.直接下载源码和编译好的插件包(推荐) -``` -wget https://github.com/DTStack/flinkx/releases/download/1.10.4/flinkx.7z -7za x flinkx.7z -cd flinkx -``` - ## 编译插件 ```bash mvn clean package -DskipTests ``` -## 常见问题 +对于不需要的插件,可以修改$FLINKX_HOME目录下的pom文件,可以将不需要的模块和`flinkx-test`模块注释掉,在编译时将不会编译该插件,这样可以缩短编译时间. + +注:**部分模块有依赖关系,请注意**。若遇到这种情况,请根据maven报错提示,将对应依赖的模块取消注释。 + +```xml + + flinkx-core + + flinkx-launcher + flinkx-test + flinkx-stream + + + flinkx-rdb + flinkx-mysql + flinkx-polardb + flinkx-oracle + flinkx-sqlserver + flinkx-postgresql + flinkx-db2 + flinkx-dm + flinkx-gbase + flinkx-clickhouse + flinkx-saphana + flinkx-teradata + flinkx-greenplum + flinkx-kingbase + + flinkx-hdfs + flinkx-hive + flinkx-es + flinkx-ftp + flinkx-odps + flinkx-hbase + flinkx-phoenix5 + flinkx-carbondata + flinkx-kudu + flinkx-cassandra + + flinkx-redis + flinkx-mongodb + + + flinkx-binlog + flinkx-kb + flinkx-kafka09 + flinkx-kafka10 + flinkx-kafka11 + flinkx-kafka + flinkx-emqx + flinkx-pulsar + flinkx-pgwal + flinkx-restapi + flinkx-oraclelogminer + +``` -### 1.编译找不到DB2、达梦、gbase、ojdbc8等驱动包 +### 1.编译找不到DB2、达梦、gbase、ojdbc8、kingbase、vertica等驱动包 -解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用插件提供的脚本安装: +解决办法:在$FLINKX_HOME/jars目录下有这些驱动包,可以手动安装,也可以使用$FLINKX_HOME/bin目录下的脚本安装: ```bash ## windows平台 @@ -42,6 +106,30 @@ mvn clean package -DskipTests ./install_jars.sh ``` +### 2.编译报错找不到其他包 + +解决办法:在$FLINKX_HOME/jars目录下有maven的setting文件,内容如下,**修改仓库路径**后替换本地maven的setting文件,重新安装[步骤一](#1.编译找不到DB2、达梦、gbase、ojdbc8、kingbase、vertica等驱动包)中的驱动包,然后再编译插件 +```xml + + + + + /home/apache-maven-3.6.1/repository + + + + alimaven + aliyun maven + http://maven.aliyun.com/nexus/content/groups/public/ + central + + + + +``` + ## 运行任务 首先准备要运行的任务json,这里以stream插件为例: @@ -50,35 +138,27 @@ mvn clean package -DskipTests { "job" : { "content" : [ { - "reader": { - "name": "streamreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ] - } + "reader" : { + "parameter" : { + "column" : [ { + "name": "id", + "type" : "id" + }, { + "name": "string", + "type" : "string" + } ], + "sliceRecordCount" : [ "10"] + }, + "name" : "streamreader" }, "writer" : { "parameter" : { - "print": false + "print" : true }, "name" : "streamwriter" } } ], "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : false - }, - "errorLimit" : { - }, "speed" : { "channel" : 1 } @@ -87,20 +167,26 @@ mvn clean package -DskipTests } ``` +
+ ### Local模式运行任务 命令模板: ```bash bin/flinkx \ - -mode local \ - -job docs/example/stream_stream.json \ - -pluginRoot syncplugins + -mode local \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ + -flinkconf flinkconf ``` -可以在flink的配置文件里配置端口: - +修改flink配置文件,指定web UI端口 ```bash +vi flinkconf/flink-conf.yaml +``` + +```yml ## web服务端口,不指定的话会随机生成一个 rest.bind-port: 8888 ``` @@ -109,9 +195,10 @@ rest.bind-port: 8888 ```bash bin/flinkx \ - -mode local \ - -job docs/example/stream_stream.json \ - -pluginRoot syncplugins + -mode local \ + -job docs/example/stream_stream.json \ + -pluginRoot syncplugins \ + -flinkconf flinkconf ``` 任务运行后可以通过8888端口访问flink界面查看任务运行情况: @@ -120,6 +207,8 @@ bin/flinkx \ +
+ ### Standalone模式运行 命令模板: @@ -161,6 +250,8 @@ $FLINK_HOME/bin/start-cluster.sh +
+ ### 以Yarn Session模式运行任务 命令示例: @@ -175,10 +266,26 @@ bin/flinkx \ -confProp "{\"flink.checkpoint.interval\":60000}" ``` -首先确保yarn集群是可用的,然后手动启动一个yarn session: +[下载](https://mvnrepository.com/artifact/org.apache.flink/flink-shaded-hadoop-2-uber)对应Hadoop版本的flink shade包,放入$FLINK_HOME/lib目录下(从flink1.11开始官方不再提供打包好的flink shade包,需要自行[下载](https://github.com/apache/flink-shaded)打包) + +[下载](https://mvnrepository.com/artifact/org.apache.flink/flink-metrics-prometheus)对应版本的flink prometheus包,放入$FLINK_HOME/lib目录下 +修改flink配置文件,指定flink类加载方式 ```bash -$FLINK_HOME/bin/yarn-session.sh -n 1 -s 2 -jm 1024 -tm 1024 +vi ../conf/flink-conf.yaml +``` + +```yml +## flink类加载方式,指定为父类优先 +classloader.resolve-order: parent-first +``` + +确保yarn集群是可用的,然后手动启动一个yarn session: + +注:-ship: 启动flink session时上传FlinkX插件包,这样只需要在提交FlinkX任务的节点部署FlinkX插件包,其他服务器节点不需要部署,同时更换FlinkX插件包后需要重启yarn session,需要配合修改flink的类加载方式。 + +```bash +nohup $FLINK_HOME/bin/yarn-session.sh -qu a -ship $FLINKX_HOME/syncplugins/ & ```

@@ -196,7 +303,8 @@ bin/flinkx \ -mode yarn \ -job docs/example/stream_stream.json \ -flinkconf $FLINK_HOME/conf \ - -yarnconf $HADOOP_HOME/etc/hadoop + -yarnconf $HADOOP_HOME/etc/hadoop \ + -queue a ``` 然后在flink界面查看任务运行情况: @@ -205,6 +313,8 @@ bin/flinkx \
+
+ ### 以Yarn Perjob模式运行任务 命令示例: @@ -218,8 +328,7 @@ bin/flinkx \ -yarnconf $HADOOP_HOME/etc/hadoop \ -flinkLibJar $FLINK_HOME/lib \ -confProp "{\"flink.checkpoint.interval\":60000}" \ - -queue default \ - -pluginLoadMode classpath + -queue default ``` 首先确保yarn集群是可用的,启动一个Yarn Application运行任务: @@ -228,10 +337,10 @@ bin/flinkx \ bin/flinkx \ -mode yarnPer \ -job docs/example/stream_stream.json \ - -pluginRoot syncplugins \ + -pluginRoot $FLINK_HOME/conf \ -yarnconf $HADOOP_HOME/etc/hadoop \ -flinkLibJar $FLINK_HOME/lib \ - -pluginLoadMode classpath + -queue a ``` 然后在集群上查看任务运行情况 @@ -250,18 +359,18 @@ bin/flinkx \ | ------------------ | ------------------------------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---- | ----------------------- | | **mode** | 执行模式,也就是flink集群的工作模式 | 1.**local**: 本地模式
2.**standalone**: 独立部署模式的flink集群
3.**yarn**: yarn模式的flink集群,需要提前在yarn上启动一个flink session,使用默认名称"Flink session cluster"
4.**yarnPer**: yarn模式的flink集群,单独为当前任务启动一个flink session,使用默认名称"Flink per-job cluster" | 否 | local | | **job** | 数据同步任务描述文件的存放路径;该描述文件中使用json字符串存放任务信息 | 无 | 是 | 无 | -| **jobid** | 任务名称 | 无 | 否 | Flink Job | +| **jobid** | 指定flink任务名称 | 无 | 否 | Flink Job | | **pluginRoot** | 插件根目录地址,也就是打包后产生的pluginRoot目录。 | 无 | 否 | $FLINKX_HOME/syncplugins | -| **flinkconf** | flink配置文件所在的目录(单机模式下不需要) | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | +| **flinkconf** | flink配置文件所在的目录 | $FLINK_HOME/conf | 否 | $FLINK_HOME/conf | | **flinkLibJar** | flink lib所在的目录(单机模式下不需要),如/opt/dtstack/flink-1.10.1/lib | $FLINK_HOME/lib | 否 | $FLINK_HOME/lib | | **yarnconf** | Hadoop配置文件(包括hdfs和yarn)所在的目录 | $HADOOP_HOME/etc/hadoop | 否 | $HADOOP_HOME/etc/hadoop | | **queue** | yarn队列,如default | 无 | 否 | default | | **pluginLoadMode** | yarn session模式插件加载方式 | 1.**classpath**:提交任务时不上传插件包,需要在yarn-node节点pluginRoot目录下部署插件包,但任务启动速度较快
2.**shipfile**:提交任务时上传pluginRoot目录下部署插件包的插件包,yarn-node节点不需要部署插件包,任务启动速度取决于插件包的大小及网络环境 | 否 | shipfile | -| **confProp** | checkpoint配置 | **flink.checkpoint.interval**:快照生产频率
**flink.checkpoint.stateBackend**:快照存储路径 | 否 | 无 | -| **s** | checkpoint快照路径 | | 否 | 无 | +| **confProp** | flink额外配置,如checkpoint、内存 | **flink.checkpoint.interval**:快照生产频率(毫秒)
**flink.checkpoint.timeout**:快照超时时间(毫秒)
**jobmanager.memory.mb**:perJob模式下jobmanager内存设置
**taskmanager.memory.mb**:perJob模式下taskmanager内存设置
**taskmanager.slots**:perJob模式下jobmanager slots个数设置 | 否 | 无 | +| **s** | checkpoint快照路径,设置后从该快照恢复任务 | | 否 | 无 | | **p** | 自定义入参,用于替换脚本中的占位符,如脚本中存在占位符${pt1},${pt2},则该参数可配置为pt1=20200101,pt2=20200102| | 否 | 无 | -| **appId** | yarn模式下,提交到指定的的flink session的application Id | | 否 | 无 | -| **krb5conf** | 提交到开启kerberos的Hadoop集群的krb5文件路径 | | 否 | 无 | -| **keytab** | 提交到开启kerberos的Hadoop集群的keytab文件路径 | | 否 | 无 | -| **principal** | kerberos认证的principal | | 否 | 无 | +| **appId** | yarn session模式下,提交到指定的的flink session的application Id | | 否 | 无 | +| **krb5conf** | 提交到开启kerberos的Hadoop集群的krb5文件路径 | | 否 | 无 | +| **keytab** | 提交到开启kerberos的Hadoop集群的keytab文件路径 | | 否 | 无 | +| **principal** | kerberos认证的principal | | 否 | 无 | diff --git "a/docs/realTime/other/LogMiner\345\216\237\347\220\206.md" "b/docs/realTime/other/LogMiner\345\216\237\347\220\206.md" new file mode 100644 index 0000000000..818851d50c --- /dev/null +++ "b/docs/realTime/other/LogMiner\345\216\237\347\220\206.md" @@ -0,0 +1,392 @@ +# FlinkX Oracle LogMiner实时采集基本原理 + +本文主要对Logminer基本原理以及如何使用和Flinkx与Logminer的集成进行介绍 +通过本文你可以了解到: + +- Logminer是什么 +- Logminer的使用 +- Flinkx如何和Logminer的集成 + +# Logminer是什么? +LogMiner 是Oracle公司从产品8i以后提供的一个实际非常有用的分析工具,使用该工具可以轻松获得Oracle 重做日志文件(归档日志文件)中的具体内容,LogMiner分析工具实际上是由一组PL/SQL包和一些动态视图组成,它作为Oracle数据库的一部分来发布,是oracle公司提供的一个完全免费的工具。 + + +具体的说: +对用户数据或数据库字典所做的所有更改都记录在Oracle重做日志文件RedoLog中,Logminer就是一个解析RedoLog的工具,通过Logminer解析RedoLog可以得到对应的SQL数据。 + +Oracle 中的RedoLog写入流程: +Oracle重做日志采用**循环写入**的方式,每一个Oracle实例至少拥有**2组日志组**。Oracle重做日志一般由Oracle自动切换,重做日志文件在当LGWR进程停止写入并开始写入下一个日志组时发生切换,或在用户收到发出ALTER SYSTEM SWITCH LOGFILE时发生切换。如果Oracle数据库开启了归档功能,则在日志组发生切换的时候,上一个日志组的日志文件会被归档到归档目录里 + + +从上面可知 Oracle里的RedoLog文件分为两种: + +- 当前写的日志组的文件,可通过 v$log 和 v$logfile 得到 +- 归档的redoLog文件,可通过 v$archived_log 得到 + +v$log 文档 +[https://docs.oracle.com/cd/B19306_01/server.102/b14237/dynviews_1150.htm#REFRN30127](https://docs.oracle.com/cd/B19306_01/server.102/b14237/dynviews_1150.htm#REFRN30127) + + +v$logfile 文档 +[https://docs.oracle.com/cd/B28359_01/server.111/b28320/dynviews_2031.htm#REFRN30129](https://docs.oracle.com/cd/B28359_01/server.111/b28320/dynviews_2031.htm#REFRN30129) + + +v$archived_log 文档 +[https://docs.oracle.com/cd/E18283_01/server.112/e17110/dynviews_1016.htm](https://docs.oracle.com/cd/E18283_01/server.112/e17110/dynviews_1016.htm) + + +**通过循环查找到最新符合要求的RedoLog并让Logminer加载分析,分析的数据在视图 v$logmnr_contents 里,通过读取 v$logmnr_contents 就可以得到 Oracle的实时数据** + +# Logminer的使用 + + +## Logminer的配置与开启 +[Oracle配置LogMiner](LogMiner配置.md) + +## Logminer的使用 + +1. 指定LogMiner字典。 + +1. 指定重做日志文件列表以进行分析。 + 使用 `DBMS_LOGMNR.ADD_LOGFILE` 过程,或指示LogMiner在启动LogMiner时自动创建要分析的日志文件列表(在步骤3中)。 + +1. 启动LogMiner。 + 使用 `DBMS_LOGMNR.START_LOGMNR` 程序。 + +1. 请求感兴趣的重做数据。 + 查询`V$LOGMNR_CONTENTS`视图。(您必须具有`SELECT ANY TRANSACTION`查询此视图的权限) + +1. 结束LogMiner会话。 + 使用 `DBMS_LOGMNR.END_LOGMNR` 程序 + + + + +### Logminer字典 +#### LogMiner字典作用 +Oracle数据字典记录当前所有表的信息,字段的信息等等。LogMiner使用字典将内部对象标识符和数据类型转换为对象名称和外部数据格式。如果没有字典,LogMiner将返回内部对象ID,并将数据显示为二进制数 +```sql +INSERT INTO HR.JOBS(JOB_ID, JOB_TITLE, MIN_SALARY, MAX_SALARY) VALUES('IT_WT','Technical Writer', 4000, 11000); + +``` +没有字典,LogMiner将显示: +```sql +insert into "UNKNOWN"."OBJ# 45522"("COL 1","COL 2","COL 3","COL 4") values +(HEXTORAW('45465f4748'),HEXTORAW('546563686e6963616c20577269746572'), +HEXTORAW('c229'),HEXTORAW('c3020b')); +``` + + +#### Logminer字典选项 +LogMiner字典的选项支持三种: + +- [Using the Online Catalog](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014720) + Oracle recommends that you use this option when you will have access to the source database from which the redo log files were created and when no changes to the column definitions in the tables of interest are anticipated. This is the most efficient and easy-to-use option. + +- [Extracting a LogMiner Dictionary to the Redo Log Files](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014735) + Oracle recommends that you use this option when you do not expect to have access to the source database from which the redo log files were created, or if you anticipate that changes will be made to the column definitions in the tables of interest. + +- [Extracting the LogMiner Dictionary to a Flat File](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014763) + This option is maintained for backward compatibility with previous releases. This option does not guarantee transactional consistency. Oracle recommends that you use either the online catalog or extract the dictionary from redo log files instead. + + + +翻译: + +- [使用在线目录](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014720) + 当您可以访问从其创建重做日志文件的源数据库并且预计不会对目标表中的列定义进行任何更改时,Oracle建议您使用此选项。这是最有效和易于使用的选项。 + +- [将LogMiner字典提取到重做日志文件](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014735) + 如果您不希望访问创建重做日志文件的源数据库,或者希望对感兴趣的表中的列定义进行更改,则Oracle建议您使用此选项。 + +- [将LogMiner字典提取到平面文件](https://docs.oracle.com/cd/B19306_01/server.102/b14215/logminer.htm#i1014763) + 维护此选项是为了与以前的版本向后兼容。此选项不能保证事务的一致性。Oracle建议您使用联机目录或从重做日志文件中提取字典。 + + + + + + +### 指定Logminer重做日志文件 +要启动新的重做日志文件列表,需要使用  DBMS_LOGMNR.NEW 以表明这是新列表的开始 +```sql +EXECUTE DBMS_LOGMNR.ADD_LOGFILE( + LOGFILENAME => '/oracle/logs/log1.f', + OPTIONS => DBMS_LOGMNR.NEW); +``` +可以使用下列语句额外再添加日志文件 +```sql +EXECUTE DBMS_LOGMNR.ADD_LOGFILE( + LOGFILENAME => '/oracle/logs/log2.f', + OPTIONS => DBMS_LOGMNR.ADDFILE); +``` + + +### 启动LogMiner +使用 `DBMS_LOGMNR.START_LOGMN` 启动Logminer。可以指定参数: + +- 指定LogMiner如何过滤返回的数据(例如,通过开始和结束时间或SCN值) + +- 指定用于格式化LogMiner返回的数据的选项 + +- 指定要使用的LogMiner词典 + + + + +主要的参数有: +```text + OPTIONS参数说明: + * DBMS_LOGMNR.SKIP_CORRUPTION - 跳过出错的redlog + * DBMS_LOGMNR.NO_SQL_DELIMITER - 不使用 ';'分割redo sql + * DBMS_LOGMNR.NO_ROWID_IN_STMT - 默认情况下,用于UPDATE和DELETE操作的SQL_REDO和SQL_UNDO语句在where子句中包含“ ROWID =”。 + * 但是,这对于想要重新执行SQL语句的应用程序是不方便的。设置此选项后,“ ROWID”不会放置在重构语句的末尾 + * DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG - 使用在线字典 + * DBMS_LOGMNR.CONTINUOUS_MINE - 需要在生成重做日志的同一实例中使用日志 + * DBMS_LOGMNR.COMMITTED_DATA_ONLY - 指定此选项时,LogMiner将属于同一事务的所有DML操作分组在一起。事务按提交顺序返回。 + * DBMS_LOGMNR.STRING_LITERALS_IN_STMT - 默认情况下,格式化格式化的SQL语句时,SQL_REDO和SQL_UNDO语句会使用数据库会话的NLS设置 + * 例如NLS_DATE_FORMAT,NLS_NUMERIC_CHARACTERS等)。使用此选项,将使用ANSI / ISO字符串文字格式对重构的SQL语句进行格式化。 +``` + + +示例 + +```sql +EXECUTE DBMS_LOGMNR.START_LOGMNR( + STARTTIME => '01-Jan-2003 08:30:00', + ENDTIME => '01-Jan-2003 08:45:00', + OPTIONS => DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG + + DBMS_LOGMNR.CONTINUOUS_MINE); +``` + + +### 在V$ LOGMNR_CONTENTS中查询感兴趣的重做数据 +Logminer会解析redoLog里的日志加载到 v$LOGMNR_CONTENTS 视图里,我们只需要使用 sql查询 即可获取对应数据 +v$LOGMNR_CONTENTS视图相关字段 +[https://docs.oracle.com/cd/B19306_01/server.102/b14237/dynviews_1154.htm](https://docs.oracle.com/cd/B19306_01/server.102/b14237/dynviews_1154.htm) + + +主要字段有: + +| 列 | 数据类型 | 描述 | +| --- | --- | --- | +| SCN | NUMBER | oracle为每个已提交的事务分配唯一的scn | +| OPERATION | VARCHAR2(32) | INSERT UPDATE DELETE DDL COMMIT ROLLBACK.....| +| SEG_OWNER | VARCHAR2(32) | schema | +| TABLE_NAME | VARCHAR2(32) | 表名 | +| TIMESTAMP | DATE | 数据库变动时间戳 | +| SQL_REDO | VARCHAR2(4000) | 重建的SQL语句,该语句等效于进行更改的原始SQL语句 | + + + +示例 + +```sql +SELECT + scn, + timestamp, + operation, + seg_owner, + table_name, + sql_redo, + row_id, + csf +FROM + v$logmnr_contents +WHERE + scn > ? +``` + + +查询出来的数据示例: + +
+ +
+ +# Flinkx如何使用Logminer + + +使用Logminer在于关键2步骤: + +- 找到需要分析的Redolog日志,加载到Logminer +- 开启Logminer,在 v$LOGMNR_CONTENTS 查询感兴趣数据 +### 1. 查找RedoLog文件 +从上面介绍中 我们可以知道 Redolog来源于日志组和归档日志里,所以flinkx 根据SCN号查询日志组以及归档日志获取到对应的文件 +```sql +SELECT + MIN(name) name, + first_change# +FROM + ( + SELECT + MIN(member) AS name, + first_change#, + 281474976710655 AS next_change# + FROM + v$log l + INNER JOIN v$logfile f ON l.group# = f.group# + WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE' + GROUP BY + first_change# + UNION + SELECT + name, + first_change#, + next_change# + FROM + v$archived_log + WHERE + name IS NOT NULL + ) +WHERE + first_change# >= ? + OR ? < next_change# +GROUP BY + first_change# +ORDER BY + first_change# +``` +查询出来的数据示例: +
+ +
+注意: +如果Logminer的处理速度比Oracle产生数据速度快,那么理论上Flinkx只需要加载日志组文件不需要加载归档日志文件,而Logminer加载文件会比较消耗资源,所以会先进行RedoLog文件的查找,如果本次查找的文件和上次的没有区别,说明Logminer不需要加载新的日志文件,只需要重新再从视图里查询数据即可 + + +### 2. 加载文件到Logminer +通过一个存储过程 查询到日志文件之后 加载到Logminer里 并开启Logminer +```sql +DECLARE + st BOOLEAN := true; + start_scn NUMBER := ?; +BEGIN + FOR l_log_rec IN ( + SELECT + MIN(name) name, + first_change# + FROM + ( + SELECT + MIN(member) AS name, + first_change#, + 281474976710655 AS next_change# + FROM + v$log l + INNER JOIN v$logfile f ON l.group# = f.group# + WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE' + GROUP BY + first_change# + UNION + SELECT + name, + first_change#, + next_change# + FROM + v$archived_log + WHERE + name IS NOT NULL + ) + WHERE + first_change# >= start_scn + OR start_scn < next_change# + GROUP BY + first_change# + ORDER BY + first_change# + ) LOOP IF st THEN + SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name, SYS.DBMS_LOGMNR.new); + st := false; + ELSE + SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name); + END IF; + END LOOP; + + SYS.DBMS_LOGMNR.start_logmnr( options => SYS.DBMS_LOGMNR.skip_corruption + SYS.DBMS_LOGMNR.no_sql_delimiter + SYS.DBMS_LOGMNR.no_rowid_in_stmt + + SYS.DBMS_LOGMNR.dict_from_online_catalog + SYS.DBMS_LOGMNR.string_literals_in_stmt ); +END; +``` +### 3. 查询数据 +```sql +SELECT + scn, + timestamp, + operation, + seg_owner, + table_name, + sql_redo, + row_id, + csf +FROM + v$logmnr_contents +WHERE + scn > ? +``` + + +Flinkx就是在一个循环里 执行上述sql语句查询数据。 查询日志文件,加载到logminer,开启logminer,读取数据,更新当前最新SCN号,当数据读取完毕,代表本次加载的日志文件加载完了,通过SCN号寻找后续日志文件,重复上述操作 + +
+ +
+ + +从 v$logmnr_contents获取到数据之后,Flinkx 使用 net.sf.jsqlparser.parser.CCJSqlParserUtil 来解析 sql_redo 值 +获取到的sql_redo语句格式示例: +```json +insert into "TUDOU"."CDC"("ID","USER_ID","NAME","date1") values ('19','1','b',TO_DATE('2021-01-29 11:25:50', 'YYYY-MM-DD HH24:MI:SS')) +``` +使用net.sf.jsqlparser.parser.CCJSqlParserUtil 解析之后,flinkx根据paving参数对数据进行操作, +当pavingData为true时,数据为 +```json +{ + "scn": 1977762, + "type": "INSERT", + "schema": "TUDOU", + "table": "CDC", + "ts": 6762187276702322688, + "opTime": "2021-01-29 11:52:02.0", + "after_ID": "19", + "after_USER_ID": "1", + "after_NAME": "b", + "after_date1": "2021-01-29 11:25:50" +} +``` +当paving为false时,数据为 +```json +{ + "message": { + "scn": 1977679, + "type": "INSERT", + "schema": "TUDOU", + "table": "CDC", + "ts": 6762186352151891968, + "opTime": "2021-01-29 11:52:02.0", + "before": {}, + "after": { + "ID": "19", + "USER_ID": "1", + "NAME": "b", + "date1": "2021-01-29 11:25:50" + } + } +} +``` + + +### Oracle10 和Oracle11的部分区别 + +1. v$LOGMNR_CONTENTS 里Oracle10 比 Oracle11 少了 commit_scn字段 +1. 日志组字段里没有next_change#字段 +1. 如果Sql里含有ToDate函数,Logminer10的sql_redo加载的是ToDate函数日期格式默认是DD-MON-RR格式,而Logminer11则是Todate函数执行后的值,所以Logminer10会在获取连接的时候,执行下列SQL,设置日期格式,FLinkx再对其进行正则匹配,替换得到最终的值。 +```sql + //修改当前会话的date日期格式 + public final static String SQL_ALTER_DATE_FORMAT ="ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD HH24:MI:SS'"; + + //修改当前会话的timestamp日期格式 + public final static String NLS_TIMESTAMP_FORMAT ="ALTER SESSION SET NLS_TIMESTAMP_FORMAT = 'YYYY-MM-DD HH24:MI:SS.FF6'"; + +``` + + diff --git "a/docs/realTime/other/LogMiner\351\205\215\347\275\256.md" "b/docs/realTime/other/LogMiner\351\205\215\347\275\256.md" new file mode 100644 index 0000000000..519c7fe9ee --- /dev/null +++ "b/docs/realTime/other/LogMiner\351\205\215\347\275\256.md" @@ -0,0 +1,434 @@ +# Oracle配置LogMiner + +目录: + + +- [Oracle配置LogMiner](#oracle配置logminer) + - [一、Oracle 10g(单机版)](#一oracle-10g单机版) + - [1、查询Oracle版本信息,这里配置的是`Oracle 10g`](#1查询oracle版本信息这里配置的是oracle-10g) + - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档) + - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意) + - [a、配置归档日志保存的路径](#a配置归档日志保存的路径) + - [b、关闭数据库](#b关闭数据库) + - [c、开启日志归档](#c开启日志归档) + - [d、开启扩充日志](#d开启扩充日志) + - [e、开启数据库](#e开启数据库) + - [4、配置日志组](#4配置日志组) + - [a、查询默认日志组信息](#a查询默认日志组信息) + - [b、查询日志组储存路径](#b查询日志组储存路径) + - [c、新增日志组与删除原有日志组](#c新增日志组与删除原有日志组) + - [d、查询创建的日志组](#d查询创建的日志组) + - [5、检查是否安装LogMiner工具](#5检查是否安装logminer工具) + - [6、创建LogMiner角色并赋权](#6创建logminer角色并赋权) + - [7、创建LogMiner用户并赋权](#7创建logminer用户并赋权) + - [8、验证用户权限](#8验证用户权限) + - [二、Oracle 11g(单机版)](#二oracle-11g单机版) + - [1、查询Oracle版本信息,这里配置的是`Oracle 11g`](#1查询oracle版本信息这里配置的是oracle-11g) + - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档-1) + - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意-1) + - [a、配置归档日志保存的路径](#a配置归档日志保存的路径-1) + - [b、关闭数据库](#b关闭数据库-1) + - [c、开启日志归档](#c开启日志归档-1) + - [d、开启扩充日志](#d开启扩充日志-1) + - [e、开启数据库](#e开启数据库-1) + - [4、检查是否安装LogMiner工具](#4检查是否安装logminer工具) + - [5、创建LogMiner角色并赋权](#5创建logminer角色并赋权) + - [6、创建LogMiner用户并赋权](#6创建logminer用户并赋权) + - [7、验证用户权限](#7验证用户权限) + - [三、Oracle 12c(单机版非CBD)](#三oracle-12c单机版非cbd) + - [1、查询Oracle版本信息,这里配置的是`Oracle 12c`](#1查询oracle版本信息这里配置的是oracle-12c) + - [2、通过命令行方式登录Oracle,查看是否开启日志归档](#2通过命令行方式登录oracle查看是否开启日志归档-2) + - [3、开启日志归档,开启日志归档需要重启数据库,请注意](#3开启日志归档开启日志归档需要重启数据库请注意-2) + - [a、配置归档日志保存的路径](#a配置归档日志保存的路径-2) + - [b、关闭数据库](#b关闭数据库-2) + - [c、开启日志归档](#c开启日志归档-2) + - [d、开启扩充日志](#d开启扩充日志-2) + - [e、开启数据库](#e开启数据库-2) + - [4、创建LogMiner角色并赋权](#4创建logminer角色并赋权) + - [5、创建LogMiner用户并赋权](#5创建logminer用户并赋权) + - [6、验证用户权限](#6验证用户权限) + + + +注意: + +1、某个Oracle数据源能同时运行的任务数量取决于该Oracle的内存大小 + +2、若数据量太大导致日志组频繁切换需要增加日志组数量,增大单个日志组存储大小 + +## 一、Oracle 10g(单机版) +### 1、查询Oracle版本信息,这里配置的是`Oracle 10g` +```sql +--查看oracle版本 +select * from v$version; +``` +
+ +
+本章Oracle的版本如上图所示。 + + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 +```sql +--查询数据库归档模式 +archive log list; +``` +
+ +
+图中显示`No Archive Mode`表示未开启日志归档。 + + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 +#### a、配置归档日志保存的路径 +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 +```shell +# 创建归档日志保存目录 +mkdir -p /data/oracle/archivelog + +# 进入Oracle目录 +cd $ORACLE_HOME + +# 查看Oracle权限组,本章权限组如下图所示 +ls -l + +# 对归档日志保存目录赋予相应权限 +chown -R 下图中的用户名:下图中的组名 /data/oracle/ +``` +
+ +
+ +```sql +--配置归档日志保存的路径 +alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` +#### b、关闭数据库 +```sql +shutdown immediate; +startup mount; +``` +#### c、开启日志归档 +```sql +--开启日志归档 +alter database archivelog; +``` +#### d、开启扩充日志 +```sql +--开启扩充日志 +alter database add supplemental log data (all) columns; +``` +#### e、开启数据库 +```sql +alter database open; +``` +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 +
+ +
+ +### 4、配置日志组 +#### a、查询默认日志组信息 +```sql +SELECT * FROM v$log; +``` +
+ +
+ +如上图所示,日志组的默认数量为2组,大小为4194304/1024/1024 = 4MB,这意味着日志大小每达到4MB就会进行日志组的切换,切换太过频繁会导致查询出错,因此需要增加日志组数量及大小。 +#### b、查询日志组储存路径 +```sql +SELECT * FROM v$logfile; +``` +
+ +
+ +如上图所示,默认路径为`/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/`。 +#### c、新增日志组与删除原有日志组 +请与DBA联系,决定是否可以删除原有日志组。 +```sql +--增加两组日志组 +alter database add logfile group 3 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo3.log') size 200m; +alter database add logfile group 4 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo4.log') size 200m; +``` +```sql +--删除原有两组日志组,并继续新增两组日志组 +alter system checkpoint; +alter system switch logfile; +alter database drop logfile group 1; +alter database drop logfile group 2; +alter database add logfile group 1 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo1.log') size 200m; +alter database add logfile group 2 ('/usr/lib/oracle/xe/app/oracle/flash_recovery_area/XE/onlinelog/redo2.log') size 200m; +``` +#### d、查询创建的日志组 +```sql +SELECT * FROM v$log; +SELECT * FROM v$logfile; +``` +
+ +
+ +
+ +
+ +### 5、检查是否安装LogMiner工具 +Oracle10g默认已安装LogMiner工具包,通过以下命令查询: +```sql +desc DBMS_LOGMNR; +desc DBMS_LOGMNR_D; +``` +若无信息打印,则执行下列SQL初始化LogMiner工具包: +```sql +@$ORACLE_HOME/rdbms/admin/dbmslm.sql; +@$ORACLE_HOME/rdbms/admin/dbmslmd.sql; +``` + + +### 6、创建LogMiner角色并赋权 +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 +```sql +create role roma_logminer_privs; +grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +``` + + +### 7、创建LogMiner用户并赋权 +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 +```sql +create user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter user roma_logminer quota unlimited on users; +``` + + +### 8、验证用户权限 +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: +```sql + SELECT * FROM USER_ROLE_PRIVS; +``` +
+ +
+ +```sql +SELECT * FROM SESSION_PRIVS; +``` +
+ +
+ +至此,Oracle 10g数据库LogMiner实时采集配置完毕。 + + +## 二、Oracle 11g(单机版) +### 1、查询Oracle版本信息,这里配置的是`Oracle 11g` +```sql +--查看oracle版本 +select * from v$version; +``` +
+ +
+本章Oracle的版本如上图所示。 + + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 +```sql +--查询数据库归档模式 +archive log list; +``` +
+ +
+图中显示`No Archive Mode`表示未开启日志归档。 + + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 +#### a、配置归档日志保存的路径 +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 +```sql + alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` +#### b、关闭数据库 +```sql +shutdown immediate; +startup mount; +``` +#### c、开启日志归档 +```sql +--开启日志归档 +alter database archivelog; +``` +#### d、开启扩充日志 +```sql +--开启扩充日志 +alter database add supplemental log data (all) columns; +``` +#### e、开启数据库 +```sql +alter database open; +``` +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 +
+ +
+ +### 4、检查是否安装LogMiner工具 +Oracle11g默认已安装LogMiner工具包,通过以下命令查询: +```sql +desc DBMS_LOGMNR; +desc DBMS_LOGMNR_D; +``` +若无信息打印,则执行下列SQL初始化LogMiner工具包: +```sql +@$ORACLE_HOME/rdbms/admin/dbmslm.sql; +@$ORACLE_HOME/rdbms/admin/dbmslmd.sql; +``` + + +### 5、创建LogMiner角色并赋权 +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 +```sql +create role roma_logminer_privs; +grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +``` + + +### 6、创建LogMiner用户并赋权 +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 +```sql +create user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter user roma_logminer quota unlimited on users; +``` + + +### 7、验证用户权限 +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: +```sql + SELECT * FROM USER_ROLE_PRIVS; +``` +
+ +
+ +```sql +SELECT * FROM SESSION_PRIVS; +``` +
+ +
+ +至此,Oracle 11g数据库LogMiner实时采集配置完毕。 + + +## 三、Oracle 12c(单机版非CBD) +### 1、查询Oracle版本信息,这里配置的是`Oracle 12c` +```sql +--查看oracle版本 +select BANNER from v$version; +``` +
+ +
+本章Oracle的版本如上图所示。 + + +### 2、通过命令行方式登录Oracle,查看是否开启日志归档 +```sql +--查询数据库归档模式 +archive log list; +``` +
+ +
+图中显示`No Archive Mode`表示未开启日志归档。 + + +### 3、开启日志归档,开启日志归档需要重启数据库,请注意 +#### a、配置归档日志保存的路径 +根据自身环境配置归档日志保存路径,需要提前创建相应目录及赋予相应访问权限 +```sql + alter system set log_archive_dest_1='location=/data/oracle/archivelog' scope=spfile; +``` +#### b、关闭数据库 +```sql +shutdown immediate; +startup mount; +``` +#### c、开启日志归档 +```sql +--开启日志归档 +alter database archivelog; +``` +#### d、开启扩充日志 +```sql +--开启扩充日志 +alter database add supplemental log data (all) columns; +``` +#### e、开启数据库 +```sql +alter database open; +``` +再次查询数据库归档模式,`Archive Mode`表示已开启归档模式,`Archive destination`表示归档日志储存路径。 +
+ +
+ +### 4、创建LogMiner角色并赋权 +其中`roma_logminer_privs`为角色名称,可根据自身需求修改。 +```sql +create role roma_logminer_privs; +grant create session,execute_catalog_role,select any transaction,flashback any table,select any table,lock any table,logmining,select any dictionary to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_COL$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_OBJ$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_USER$ to roma_logminer_privs; +grant select on SYSTEM.LOGMNR_UID$ to roma_logminer_privs; +grant select_catalog_role to roma_logminer_privs; +grant LOGMINING to roma_logminer_privs; +``` + + +### 5、创建LogMiner用户并赋权 +其中`roma_logminer`为用户名,`password`为密码,请根据自身需求修改。 +```sql +create user roma_logminer identified by password default tablespace users; +grant roma_logminer_privs to roma_logminer; +grant execute_catalog_role to roma_logminer; +alter user roma_logminer quota unlimited on users; +``` + + +### 6、验证用户权限 +以创建的LogMiner用户登录Oracle数据库,执行以下SQL查询权限,结果如图所示: +```sql + SELECT * FROM USER_ROLE_PRIVS; +``` +
+ +
+ +```sql +SELECT * FROM SESSION_PRIVS; +``` +
+ +
+ +至此,Oracle 12c数据库LogMiner实时采集配置完毕。 diff --git "a/docs/realTime/other/PgWal\345\216\237\347\220\206\345\217\212\351\205\215\347\275\256.md" "b/docs/realTime/other/PgWal\345\216\237\347\220\206\345\217\212\351\205\215\347\275\256.md" new file mode 100644 index 0000000000..13eb4ea428 --- /dev/null +++ "b/docs/realTime/other/PgWal\345\216\237\347\220\206\345\217\212\351\205\215\347\275\256.md" @@ -0,0 +1,237 @@ +# FlinkX PostgreSQL WAL实时采集基本原理 + + + +- [FlinkX PostgreSQL WAL实时采集基本原理](#flinkx-postgresql-wal实时采集基本原理) + - [版本限制](#版本限制) + - [主要涉及模块说明](#主要涉及模块说明) + - [逻辑复制](#逻辑复制) + - [创建发布](#创建发布) + - [WAL日志](#wal日志) + - [WAL何时被写入](#wal何时被写入) + - [WAL主要配置](#wal主要配置) + - [复制槽](#复制槽) + - [局限性](#局限性) + - [FlinkX PostgreSQL WAL实时采集配置](#flinkx-postgresql-wal实时采集配置) + - [postgresql.conf设置](#postgresqlconf设置) + - [部分核心代码分析](#部分核心代码分析) + - [执行发布SQL](#执行发布sql) + - [创建一个逻辑复制流](#创建一个逻辑复制流) + - [业务处理](#业务处理) + + + +
+ +PostgreSQL 实时采集是基于 PostgreSQL的逻辑复制以及逻辑解码功能来完成的。逻辑复制同步数据的原理是,在wal日志产生的数据库上,由逻辑解析模块对wal日志进行初步的解析,它的解析结果为ReorderBufferChange(可以简单理解为HeapTupleData),再由pgoutput plugin对中间结果进行过滤和消息化拼接后,然后将其发送到订阅端,订阅端通过逻辑解码功能进行解析。 + +## 版本限制 +逻辑复制是pgsql10.0版本之后才支持的,因此此方案只支持10.0之后版本 + + +## 主要涉及模块说明 +| Logical Decoding | PostgreSQL 的逻辑日志来源于解析物理 WAL 日志。
解析 WAL 成为逻辑数据的过程叫 Logical Decoding。 | +| :--- | :--- | +| Replication Slots | 保存逻辑或物理流复制的基础信息。类似 Mysql 的位点信息。
一个 逻辑 slot 创建后,它的相关信息可以通过 pg_replication_slots 系统视图获取。
如果它在 active 状态,则可以通过系统视图 pg_stat_replication 看到一些 slot 的实时的状态信息。 | +| Output Plugins | PostgreSQL 的逻辑流复制协议开放一组可编程接口,用于自定义输数据到客户端的逻辑数据的格式。
这部分实现使用插件的方式被内核集成和使用,称作 Output Plugins。 | +| Exported Snapshots | 当一个逻辑流复制 slot 被创建时,系统会产生一个快照。客户端可以通过它订阅到数据库任意时间点的数据变化。 | + + + +对于修改一条数据之后 ,pgsql订阅端decode解析后的数据格式为 +```json +{"id":"schema1.test1", + "schema":"schema1", +"table":"test1", + "columnList":[ + {"name":"id","type":"int4","index":0}, + {"name":"name","type":"varchar","index":1} + ], + "oldData":["2","23"], + "newData":["2","name1"], + "type":"UPDATE", + "currentLsn":23940928, + "ts":1596358573614 +} +``` +主要包含schema table以及类型`INSERT`, `UPDATE`和`DELETE`以及WAL日志id等相关信息
+
+ + +## 逻辑复制 +逻辑复制使用_发布_和_订阅_模型, 其中一个或多个_订阅者_订阅_发布者_ 节点上的一个或多个_发布_。 订阅者从他们订阅的发布中提取数据,逻辑复制是根据复制标识(通常是主键)复制数据对象及其更改的一种方法,因此在上面订阅端收到消息数据实例中可以发现 具备数据库以及表信息外 还具备修改前数据,修改后数据信息以及执行的type和对应的WAL日志ID + +发布可以选择将它们所产生的改变限制在`INSERT`, `UPDATE`和`DELETE`的任意组合上, 类似于触发器被特定事件类型触发。默认情况下,复制所有操作类型。
已发布的table必须配置一个“副本标识”以便能够复制 `UPDATE`和`DELETE`操作, 这样可以在订阅者端识别适当的行来更新或删除。默认情况下,这是主键, 如果有的话。另外唯一的索引(有一些额外的要求)也可以被设置为副本标识。 如果表没有任何合适的键,那么它可以设置为复制标识“full”, 这意味着整个行成为键。但是,这是非常低效的, 并且只能在没有其他可能的解决方案时用作后备
+ + +## 创建发布 +为哪些表设置创建一个发布 +```sql +CREATE PUBLICATION name + [ FOR TABLE [ ONLY ] table_name [ * ] [, ...] + | FOR ALL TABLES ] + [ WITH ( publication_parameter [= value] [, ... ] ) ] +``` + + + +## WAL日志 +WAL 是 Write Ahead Log的缩写,中文称之为预写式日志。WAL log也被简称为xlog,每一次change操作都是先写日志再写数据,保证了事务持久性和数据完整性同时又尽量地避免了频繁IO对性能的影响。WAL的中心概念是**数据文件(存储着表和索引)的修改必须在这些动作被日志记录之后才被写入**
WAL日志保存在pg_xlog下,每个xlog文件默认是16MB,为了满足恢复需求,在xlog目录下会产生多个WAL日志,不需要的WAL日志将会被覆盖
WAL具备归档功能,通过归档的WAL文件可以恢复数据库到WAL日志覆盖时间内的任意一个时间点的状态并且有了WAL日志之后,逻辑复制就可以在WAL日志生成之后,对其进行一系列操作之后传递给订阅客户端,使得订阅客户端能实时获取到源服务器上的修改数据
+ + +### WAL何时被写入 +WAL也有个内存缓冲区WAL Buffer,WAL都是先写入缓存中,对于事务操作,缓存的WAL日志是在事务提交的时候写入磁盘的,对于非事务型的由一个异步线程追加进日志文件或者在checkPoint(数据脏页缓存写入磁盘需要先刷新WAL缓存)的时候写入。
+ + +### WAL主要配置 +``` +wal_level 可以选择为minimal, replica, or logical 使用逻辑复制需要设置为logical + +fsync boolean类型 表示是否使用fsync()系统调用把WAL文件刷新到物理磁盘,确保数据库在操作系统或硬件奔溃的情况下可恢复到最终状态 默认是on + +synchronous_commit boolean类型 声明提交一个事务是否需要等待其把WAL日志写入磁盘后再返回,默认值是’on’ + +on:默认值,为on且没有开启同步备库的时候,会当wal日志真正刷新到磁盘永久存储后才会返回客户端事务已提交成功, + 当为on且开启了同步备库的时候(设置了synchronous_standby_names),必须要等事务日志刷新到本地磁盘,并且还要等远程备库也提交到磁盘才能返回客户端已经提交. + +remote_apply:提交将等待, 直到来自当前同步备用数据库的回复表明它们已收到事务的提交记录并应用它, 以便它对备用数据库上的查询可见。 + +remote_write:提交将等待,直到来自当前同步的后备服务器的一个回复指示该服务器已经收到了该事务的提交记录并且已经把该记录写出到后备服务器的操作系统。 + +local:当事务提交时,仅写入本地磁盘即可返回客户端事务提交成功,而不管是否有同步备库。 + +off:写到缓存中就会向客户端返回提交成功,但也不是一直不刷到磁盘,延迟写入磁盘,延迟的时间为最大3倍的wal_writer_delay参数的(默认200ms)的时间,所有如果即使关闭synchronous_commit,也只会造成最多600ms的事务丢失 可能会造成一些最近已提交的事务丢失,但数据库状态是一致的,就像这些事务已经被干净地中止。但对高并发的小事务系统来说,性能来说提升较大。 + + +wal_sync_method enum类型 用来指定向磁盘强制更新WAL日志数据的方法open_datasync fdatasync fsync_writethrough fsync open_sync + + + +Wal_writer_delay 指定wal writer process 把WAL日志写入磁盘的周期 在每个周期中会先把缓存中的WAL日志刷到磁盘 + +``` + + + +## 复制槽 +每个订阅都将通过一个复制槽接收更改,记录某个订阅者的WAL接收情况。
在源数据库写入修改频繁导致WAL日志的写入速度很快,导致大量WAL日志生成,或者订阅者接受日志很慢,在消费远远小于生产的时候,会导致源数据库上的WAL日志还没有传递到备库就被回卷覆盖掉了,如果被覆盖掉的WAL日志文件又没有归档备份,那么订阅者就再也无法消费到此数据。
复制槽则保存了此订阅的接收信息,使得未被接收的WAL日日志不会被回收 + +注意
数据库会记录slot的wal复制位点,并在wal文件夹中保留所有未发送的wal文件,如果客户创建了slot但是后期不再使用就有可能导致数据库的wal日志爆仓,需要及时删除不用的slot
+
可通过以下SQL获取相关信息 +```sql +select * from pg_replication_slots; +``` +字段含义 +```text +Name Type References Description +slot_name name 复制槽的唯一的集群范围标识符 +plugin name 正在使用的包含逻辑槽输出插件的共享对象的基本名称,对于物理插槽则为null。 +slot_type text 插槽类型 - 物理或逻辑 +datoid oid 该插槽所关联的数据库的OID,或为空。 只有逻辑插槽才具有关联的数据库。 +database text 该插槽所关联的数据库的名称,或为空。 只有逻辑插槽才具有关联的数据库。 +active boolean 如果此插槽当前正在使用,则为真 +active_pid integer 如果当前正在使用插槽,则使用此插槽的会话的进程ID。 NULL如果不活动。 +xmin xid 此插槽需要数据库保留的最早事务。 VACUUM无法删除任何后来的事务删除的元组。 +catalog_xmin xid 影响该插槽需要数据库保留的系统目录的最早的事务。 VACUUM不能删除任何后来的事务删除的目录元组。 +restart_lsn pg_lsn 最老的WAL的地址(LSN)仍然可能是该插槽的使用者所需要的,因此在检查点期间不会被自动移除 +``` + + + +## 局限性 + +- 不复制数据库模式和DDL命令。初始模式可以使用`pg_dump --schema-only` 手动复制。后续的模式更改需要手动保持同步。(但是请注意, 两端的架构不需要完全相同。)当实时数据库中的模式定义更改时,逻辑复制是健壮的: 当模式在发布者上发生更改并且复制的数据开始到达订阅者但不符合表模式, 复制将错误,直到模式更新。在很多情况下, 间歇性错误可以通过首先将附加模式更改应用于订阅者来避免。
+- 不复制序列数据。序列支持的序列或标识列中的数据当然会作为表的一部分被复制, 但序列本身仍然会显示订阅者的起始值。如果订阅者被用作只读数据库, 那么这通常不成问题。但是,如果打算对订阅者数据库进行某种切换或故障切换, 则需要将序列更新为最新值,方法是从发布者复制当前数据 (可能使用`pg_dump`)或者从表中确定足够高的值。
+- 不复制`TRUNCATE`命令。当然,可以通过使用`DELETE` 来解决。为了避免意外的`TRUNCATE`调用,可以撤销表的 `TRUNCATE`权限。
+- 不复制大对象 没有什么解决办法,除非在普通表中存储数据。 +- 复制只能从基表到基表。也就是说,发布和订阅端的表必须是普通表,而不是视图, 物化视图,分区根表或外部表。对于分区,您可以一对一地复制分区层次结构, 但目前不能复制到不同的分区设置。尝试复制基表以外的表将导致错误 + + +
+ + +## FlinkX PostgreSQL WAL实时采集配置 + +### postgresql.conf设置 +``` +wal_level = logical +``` + + +用于复制链接的角色必须具有`REPLICATION`属性(或者是超级用户) 需要在pg_hba.conf做出如下配置 +``` +host replication all 10.0.3.0/24 md5 +``` + + +## 部分核心代码分析 + + + +### 执行发布SQL +逻辑复制流是发布/订阅模型,因此生成流之前 先进行发布 +```java +public static final String PUBLICATION_NAME = "dtstack_flinkx"; +public static final String CREATE_PUBLICATION = "CREATE PUBLICATION %s FOR ALL TABLES;"; +public static final String QUERY_PUBLICATION = "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s';"; + +先执行查找sql 判断是否存在 dtstack_flinkx 的 PUBLICATION +如果不存在 执行创建sql语句 +conn.createStatement() + .execute(String.format(CREATE_PUBLICATION, PUBLICATION_NAME)); +``` + + + +### 创建一个逻辑复制流 +```java + ChainedLogicalStreamBuilder builder = conn.getReplicationAPI() + .replicationStream() //定义一个逻辑复制流 + .logical() //级别是logical + .withSlotName(format.getSlotName())//复制槽名称 + //协议版本。当前仅支持版本1 + .withSlotOption("proto_version", "1")//槽版本号 + //逗号分隔的要订阅的发布名称列表(接收更改)。 单个发布名称被视为标准对象名称,并可根据需要引用 + .withSlotOption("publication_names", PgWalUtil.PUBLICATION_NAME)//关联的发布名称 + .withStatusInterval(format.getStatusInterval(), TimeUnit.MILLISECONDS); + long lsn = format.getStartLsn(); + if(lsn != 0){ + builder.withStartPosition(LogSequenceNumber.valueOf(lsn)); + } + stream = builder.start(); +``` + +### 业务处理 +逻辑复制流接收到订阅的消息后 进行编码 获取到相应信息处理 +```java + public void run() { + LOG.info("PgWalListener start running....."); + try { + init(); + while (format.isRunning()) { + //接收到流对象 + ByteBuffer buffer = stream.readPending(); + if (buffer == null) { + continue; + } + //解码为table对象 具体信息为库 表 字段信息 WAL id等 + //然后就可以对其进行处理了 + Table table = decoder.decode(buffer); + if(StringUtils.isBlank(table.getId())){ + continue; + } + String type = table.getType().name().toLowerCase(); + if(!cat.contains(type)){ + continue; + } + if(!tableSet.contains(table.getId())){ + continue; + } + LOG.trace("table = {}",gson.toJson(table)); + ............... + } + } + } +``` + +
\ No newline at end of file diff --git a/docs/realTime/reader/LogMiner.md b/docs/realTime/reader/LogMiner.md new file mode 100644 index 0000000000..8e159692e3 --- /dev/null +++ b/docs/realTime/reader/LogMiner.md @@ -0,0 +1,236 @@ +# Oracle LogMiner Reader + + + +- [Oracle LogMiner Reader](#oracle-logminer-reader) + - [一、插件名称](#一插件名称) + - [二、支持的数据源版本](#二支持的数据源版本) + - [三、数据库配置](#三数据库配置) + - [四、基本原理](#四基本原理) + - [五、参数说明](#五参数说明) + - [六、配置示例](#六配置示例) + + + +
+ +## 一、插件名称 +名称:**oraclelogminerreader** + +
+ +## 二、支持的数据源版本 +**支持Oracle 10,Oracle 11以及Oracle12单机版,不支持RAC模式,暂不支持Oracle18、Oracle19** + +
+ +## 三、数据库配置 +[Oracle配置LogMiner](../other/LogMiner配置.md) + +
+ +## 四、基本原理 +[FlinkX Oracle LogMiner实时采集基本原理](../other/LogMiner原理.md) + +
+ +## 五、参数说明 + +- **jdbcUrl** + - 描述:Oracle数据库的JDBC URL链接 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **username** + - 描述: 用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述: 密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述: 需要监听的表,格式为:schema.table,多个以,分割,schema不能配置为\*,但table可以配置\*监听指定库下所有的表,如:schema1.table1,schema1.table2,schema2.\* + - 必选:否,不配置则监听除`SYS`库以外的所有库的所有表变更信息 + - 字段类型:String + - 默认值:无 + +
+ +- **cat** + - 描述:需要监听的操作数据操作类型,有UPDATE,INSERT,DELETE三种可选,大小写不敏感,多个以,分割 + - 必选:否 + - 字段类型:String + - 默认值:UPDATE,INSERT,DELETE + +
+ +- **readPosition** + - 描述:Oracle实时采集的采集起点 + - 可选值: + - all: 从Oracle数据库中最早的归档日志组开始采集(不建议使用) + - current:从任务运行时开始采集 + - time: 从指定时间点开始采集 + - scn: 从指定SCN号处开始采集 + - 必选:否 + - 字段类型:String + - 默认值:current + +
+ +- **startTime** + - 描述: 指定采集起点的毫秒级时间戳 + - 必选:当`readPosition`为`time`时,该参数必填 + - 字段类型:Long(毫秒级时间戳) + - 默认值:无 + +
+ +- **startSCN** + - 描述: 指定采集起点的SCN号 + - 必选:当`readPosition`为`scn`时,该参数必填 + - 字段类型:String + - 默认值:无 + +
+ +- **fetchSize** + - 描述: 批量从v$logmnr_contents视图中拉取的数据条数,对于大数据量的数据变更,调大该值可一定程度上增加任务的读取速度 + - 必选:否 + - 字段类型:Integer + - 默认值:1000 + +
+ +- **queryTimeout** + - 描述: LogMiner执行查询SQL的超时参数,单位秒 + - 必选:否 + - 字段类型:Long + - 默认值:300 + +
+ +- **supportAutoAddLog** + - 描述:启动LogMiner是否自动添加日志组 + - 必选:否 + - 字段类型:Boolean + - 默认值:false + +
+ +- **pavingData** + - 描述:是否将解析出的json数据拍平 + - 必选:否 + - 字段类型:String + - 默认值:false(一般配置成true比较好) + - 示例:假设解析的表为CDC,数据库schema为TUDOU,对CDC中的NAME字段做update操作,NAME原来的值为a,更新后为b,则pavingData为true时数据格式为: + + ```json + { + "scn": 1807399, + "type": "UPDATE", + "schema": "TUDOU", + "table": "CDC", + "ts": 6760525407742726144, + "opTime": "2021-01-28 11:52:02.0", + "after_NAME": "b", + "after_ID": "1", + "after_USER_ID": "1", + "before_ID": "1", + "before_USER_ID": "1", + "before_NAME": "a" + } + ``` + - pavingData为false时: + ```json + { + "message": { + "scn": 1807399, + "type": "UPDATE", + "schema": "TUDOU", + "table": "CDC", + "ts": 6760525407742726144, + "opTime": "2021-01-28 11:52:02.0", + "before": { + "ID": "1", + "USER_ID": "1", + "NAME": "a" + }, + "after": { + "NAME": "b", + "ID": "1", + "USER_ID": "1" + } + } + } + ``` + 其中: + + 1、scn:Oracle数据库变更记录对应的scn号 + 2、type:变更类型,INSERT,UPDATE、DELETE + 3、opTime:Oracle数据库中数据的变更时间 + 4、ts:自增ID,不重复,可用于排序,解码后为FlinkX的事件时间,解码规则如下: + + ```java + long id = Long.parseLong("6760525407742726144"); + long res = id >> 22; + DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + System.out.println(sdf.format(res)); //2021-01-28 19:54:21 + ``` + +
+ +## 六、配置示例 +```json +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "jdbcUrl": "jdbc:oracle:thin:@127.0.0.1:1521:xe", + "username": "kminer", + "password": "kminerpass", + "table": "SCHEMA1.*", + "cat": "UPDATE,INSERT,DELETE", + "startSCN": "482165", + "readPosition": "current", + "startTime": 1576540477000, + "pavingData": true, + "queryTimeout": 300 + }, + "name": "oraclelogminerreader" + }, + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore" : true, + "isStream" : true + }, + "speed": { + "channel": 1 + } + } + } +} +``` + + diff --git a/docs/realTime/reader/binlogreader.md b/docs/realTime/reader/binlogreader.md index 5a7269f3cc..fdfe5a680e 100644 --- a/docs/realTime/reader/binlogreader.md +++ b/docs/realTime/reader/binlogreader.md @@ -1,16 +1,37 @@ # MySQL Binlog Reader - + + +- [MySQL Binlog Reader](#mysql-binlog-reader) + - [一、插件名称](#一插件名称) + - [二、支持的数据源版本](#二支持的数据源版本) + - [三、数据库配置](#三数据库配置) + - [1.修改配置文件](#1修改配置文件) + - [2.添加权限](#2添加权限) + - [四、参数说明](#四参数说明) + - [五、配置示例](#五配置示例) + - [1、单表监听](#1单表监听) + - [2、多表监听](#2多表监听) + - [3、正则监听](#3正则监听) + - [4、指定起始位置](#4指定起始位置) + + + +
+ ## 一、插件名称 -名称:**binlogreader**
+名称:**binlogreader** + +
- ## 二、支持的数据源版本 -**MySQL 5.X**
+**MySQL5.1.5及以上** + +
- ## 三、数据库配置 -**1.修改配置文件** +### 1.修改配置文件 +binlog_format需要修改为 ROW 格式,在/etc/my.cnf文件里[mysqld]下添加下列配置 ```sql server_id=109 log_bin = /var/lib/mysql/mysql-bin @@ -18,110 +39,153 @@ binlog_format = ROW expire_logs_days = 30 ``` -
**2.添加权限** + +### 2.添加权限 +mysql binlog权限需要三个权限 SELECT, REPLICATION SLAVE, REPLICATION CLIENT ```sql GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal'; ``` -
- -## 四、参数说明
+ + + +- 缺乏SELECT权限时,报错为 +``` +com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: +Access denied for user 'canal'@'%' to database 'binlog' +``` + +- 缺乏REPLICATION SLAVE权限时,报错为 +``` +java.io.IOException: +Error When doing Register slave:ErrorPacket [errorNumber=1045, fieldCount=-1, message=Access denied for user 'canal'@'%' +``` + +- 缺乏REPLICATION CLIENT权限时,报错为 +``` + com.mysql.jdbc.exceptions.jdbc4.MySQLSyntaxErrorException: + Access denied; you need (at least one of) the SUPER, REPLICATION CLIENT privilege(s) for this operation +``` + + +binlog为什么需要这些权限: + +- Select权限代表允许从表中查看数据 +- Replication client权限代表允许执行show master status,show slave status,show binary logs命令 +- Replication slave权限代表允许slave主机通过此用户连接master以便建立主从 复制关系 + +
+ +## 四、参数说明 + - **jdbcUrl** - 描述:MySQL数据库的jdbc连接字符串,参考文档:[Mysql官方文档](http://dev.mysql.com/doc/connector-j/en/connector-j-reference-configuration-properties.html) - 必选:是 + - 字段类型:string - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 + - 字段类型:string - 默认值:无 - +
- **host** - 描述:启动MySQL slave的机器ip - 必选:是 + - 字段类型:string - 默认值:无 - +
- **port** - 描述:启动MySQL slave的端口 - 必选:否 + - 字段类型:int - 默认值:3306 - +
- **table** - 描述:需要解析的数据表。 - - 注意:指定此参数后filter参数将无效 + - 注意:指定此参数后filter参数将无效,table和filter都为空,监听jdbcUrl里的schema下所有表 - 必选:否 + - 字段类型:list - 默认值:无 - +
- **filter** - 描述:过滤表名的Perl正则表达式 + - 注意:table和filter都为空,监听jdbcUrl里的schema下所有表 + - 必选:否 + - 字段类型:string + - 默认值:无 - 例子: - 所有表:`_.*_` - canal schema下所有表: `canal\..*` - canal下的以canal打头的表:`canal\.canal.*` - canal schema下的一张表:`canal\.test1` - - 必选:否 - - 默认值:无 - +
- **cat** - 描述:需要解析的数据更新类型,包括insert、update、delete三种 - - 注意:以英文逗号分割的格式填写。 + - 注意:以英文逗号分割的格式填写。如果为空,解析所有数据更新类型 - 必选:否 + - 字段类型:string - 默认值:无 - +
- **start** - 描述:要读取的binlog文件的开始位置 + - 注意:为空,则从当前position处消费,timestamp的优先级高于 journalName+position - 参数: - - journalName:采集起点按文件开始时的文件名称; - - timestamp:采集起点按时间开始时的时间戳; + - timestamp:时间戳,采集起点从指定的时间戳处消费; + - journalName:文件名,采集起点从指定文件的起始处消费; + - position:文件的指定位置,采集起点从指定文件的指定位置处消费 + - 字段类型:map - 默认值:无 - +
- **pavingData** - 描述:是否将解析出的json数据拍平 - - 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时数据格式为: + - 必选:否 + - 字段类型:boolean + - 默认值:true + - 示例:假设解析的表为tb1,数据库为test,对tb1中的id字段做update操作,id原来的值为1,更新后为2,则pavingData为true时,数据格式为: ```json { "type":"update", "schema":"test", "table":"tb1", - "ts":1231232, - "ingestion":123213, + "ts":6760525407742726144, "before_id":1, "after_id":2 } ``` - -
pavingData为false时: +pavingData为false时: ```json { "message":{ "type":"update", "schema":"test", "table":"tb1", - "ts":1231232, - "ingestion":123213, + "ts":6760525407742726144, "before":{ "id":1 }, @@ -131,210 +195,248 @@ GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTI } } ``` -其中”ts“是数据变更时间,ingestion是插件解析这条数据的纳秒时间 +- type:变更类型,INSERT,UPDATE、DELETE +- ts:自增ID,不重复,可用于排序,解码后为FlinkX的事件时间,解码规则如下: +```java + long id = Long.parseLong("6760525407742726144"); + long res = id >> 22; + DateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + System.out.println(sdf.format(res)); //2021-01-28 19:54:21 +``` + +
+ +- **slaveId** + - 描述:从服务器的ID + - 注意:同一个MYSQL复制组内不能重复 - 必选:否 - - 默认值:false + - 字段类型:long + - 默认值:new Object().hashCode() + +
+- **connectionCharset** + - 描述:编码信息 + - 必选:否 + - 字段类型:string + - 默认值:UTF-8 + +
+- **detectingEnable** + - 描述:是否开启心跳 + - 必选:否 + - 字段类型:boolean + - 默认值:true + +
+ +- **detectingSQL** + - 描述:心跳SQL + - 必选:否 + - 字段类型:string + - 默认值:SELECT CURRENT_DATE + +
+ +- **enableTsdb** + - 描述:是否开启时序表结构能力 + - 必选:否 + - 字段类型:boolean + - 默认值:true + +
- **bufferSize** - 描述:并发缓存大小 - 注意:必须为2的幂 - 必选:否 - 默认值:1024 - + +
+ +- **parallel** + - 描述:是否开启并行解析binlog日志 + - 必选:否 + - 字段类型:boolean + - 默认值:true + +
+ +- **parallelThreadSize** + - 描述:并行解析binlog日志线程数 + - 注意:只有 paraller 设置为true才生效 + - 必选:否 + - 字段类型:int + - 默认值:2 + +
+ +- **isGTIDMode** + - 描述:是否开启gtid模式 + - 必选:否 + - 字段类型:boolean + - 默认值:false + +
+ ## 五、配置示例 - -#### 1、单表监听 +### 1、单表监听 ```json { - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "tudou", - "password" : "abc123", - "cat" : "insert,delete,update", - "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", - "host" : "kudu3", - "start" : { + "job": { + "content": [ + { + "reader": { + "parameter": { + "table": ["table"], + "password": "passwd", + "database": "database", + "port": 3306, + "cat": "DELETE,INSERT,UPDATE", + "host": "host", + "jdbcUrl": "jdbc:mysql://host:port/schema", + "pavingData": true, + "username": "name" }, - "table" : [ "binlog" ], - "pavingData" : true, - "username" : "dtstack" - }, - "name" : "binlogreader" - }, - "writer" : { - "parameter" : { - "print" : true + "name": "binlogreader" }, - "name" : "streamwriter" + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : 0, - "channel" : 1 + ], + "setting": { + "restore": { + "isStream": true }, - "log" : { - "isLogger": false, - "level" : "trace", - "path" : "", - "pattern":"" + "speed": { + "channel": 1 } } } } ``` - -#### 2、多表监听 +### 2、多表监听 ```json { - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "tudou", - "password" : "abc123", - "cat" : "insert,delete,update", - "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", - "host" : "kudu3", - "start" : { + "job": { + "content": [ + { + "reader": { + "parameter": { + "table": ["table1","table2"], + "password": "passwd", + "database": "database", + "port": 3306, + "cat": "DELETE,INSERT,UPDATE", + "host": "host", + "jdbcUrl": "jdbc:mysql://host:port/schema", + "pavingData": true, + "username": "name" }, - "table" : ["kudu1", "kudu2"], - "filter" : "", - "pavingData" : true, - "username" : "dtstack" - }, - "name" : "binlogreader" - }, - "writer" : { - "parameter" : { - "print" : true + "name": "binlogreader" }, - "name" : "streamwriter" + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : 0, - "channel" : 1 + ], + "setting": { + "restore": { + "isStream": true }, - "log" : { - "isLogger": false, - "level" : "trace", - "path" : "", - "pattern":"" + "speed": { + "channel": 1 } } } } ``` - -#### 3、正则监听 +### 3、正则监听 ```json { - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "tudou", - "password" : "abc123", - "cat" : "insert,delete,update", - "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", - "host" : "kudu3", - "start" : { + "job": { + "content": [ + { + "reader": { + "parameter": { + "filter": "schema\\..*", + "password": "passwd", + "database": "database", + "port": 3306, + "cat": "DELETE,INSERT,UPDATE", + "host": "host", + "jdbcUrl": "jdbc:mysql://host:port/schema", + "pavingData": true, + "username": "name" }, - "filter" : "tudou\\.kudu.*", - "pavingData" : true, - "username" : "dtstack" + "name": "binlogreader" }, - "name" : "binlogreader" - }, - "writer" : { - "parameter" : { - "print" : true - }, - "name" : "streamwriter" + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true + ], + "setting": { + "restore": { + "isStream": true }, - "errorLimit" : { }, - "speed" : { - "bytes" : 0, - "channel" : 1 - }, - "log" : { - "isLogger": false, - "level" : "trace", - "path" : "", - "pattern":"" + "speed": { + "channel": 1 } } } } ``` - -#### 4、指定起始位置 +### 4、指定起始位置 ```json { - "job" : { - "content" : [ { - "reader" : { - "parameter" : { - "schema" : "tudou", - "password" : "abc123", - "cat" : "insert,delete,update", - "jdbcUrl" : "jdbc:mysql://kudu3:3306/tudou", - "host" : "kudu3", - "start" : { - "journalName": "mysql-bin.000002", - "timestamp" : 1589353414000 + "job": { + "content": [ + { + "reader": { + "parameter": { + "filter": "schema\\..*", + "password": "passwd", + "database": "database", + "port": 3306, + "start" : { + "journalName": "binlog.000031", + "position": 4 + }, + "cat": "DELETE,INSERT,UPDATE", + "host": "host", + "jdbcUrl": "jdbc:mysql://host:port/schema", + "pavingData": true, + "username": "name" }, - "table" : ["kudu"], - "pavingData" : true, - "username" : "dtstack" - }, - "name" : "binlogreader" - }, - "writer" : { - "parameter" : { - "print" : true + "name": "binlogreader" }, - "name" : "streamwriter" + "writer": { + "parameter": { + "print": true + }, + "name": "streamwriter" + } } - } ], - "setting" : { - "restore" : { - "isRestore" : false, - "isStream" : true - }, - "errorLimit" : { }, - "speed" : { - "bytes" : 0, - "channel" : 1 + ], + "setting": { + "restore": { + "isStream": true }, - "log" : { - "isLogger": false, - "level" : "trace", - "path" : "", - "pattern":"" + "speed": { + "channel": 1 } } } @@ -342,6 +444,3 @@ GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'canal'@'%' IDENTI ``` - -## 六、问题排查 -采集mysql binlog 发现采集不到数据
1、查看binlog是否开启
       `show variables like '%log_bin%' ; ` 
2、binlog_format 是否设置为ROW
        注意 binlog_format 必须设置为 ROW, 因为在 STATEMENT 或 MIXED 模式下, Binlog 只会记录和传输 SQL 语句(以减少日志大小),而不包含具体数据,我们也就无法保存了。
3、从节点通过一个专门的账号连接主节点,这个账号需要拥有全局的 REPLICATION 权限。我们可以使用 GRANT 命令创建这样的账号:
     GRANT SELECT, REPLICATION SLAVE, REPLICATION CLIENT
    ON *.* TO 'canal'@'%' IDENTIFIED BY 'canal';
  参考:[https://blog.csdn.net/zjerryj/article/details/77152226](https://blog.csdn.net/zjerryj/article/details/77152226) diff --git a/docs/realTime/reader/emqxreader.md b/docs/realTime/reader/emqxreader.md index a4b6ae60b2..aec21267ba 100644 --- a/docs/realTime/reader/emqxreader.md +++ b/docs/realTime/reader/emqxreader.md @@ -1,41 +1,61 @@ # Emqx Reader + + +- [Emqx Reader](#emqx-reader) + - [一、插件名称](#一插件名称) + - [二、支持的数据源版本](#二支持的数据源版本) + - [三、参数说明
](#三参数说明br-) + - [四、配置示例](#四配置示例) + + + +
## 一、插件名称 名称:**emqxreader**
+
+ ## 二、支持的数据源版本 **Emqx 4.0及以上**
+ +
+ ## 三、参数说明
- **broker** - 描述:连接URL信息 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **topic** - 描述:订阅主题 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **username** - 描述:认证用户名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:认证密码 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **isCleanSession** @@ -43,8 +63,9 @@ - false:MQTT服务器保存于客户端会话的的主题与确认位置; - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - 必选:否 + - 字段类型:boolean - 默认值:true - +
- **qos** @@ -53,8 +74,9 @@ - 1:AT_LEAST_ONCE,至少一次; - 2:EXACTLY_ONCE,精准一次; - 必选:否 + - 字段类型:int - 默认值:2 - +
- **codec** @@ -66,9 +88,10 @@ - 当其中不包含message字段时,增加一个key为message,value为原始消息字符串的键值对,如:`{"key": "key", "value": "value", "message": "{\"key\": \"key\", \"value\": \"value\"}"}` - 若改字符串不为json格式,则按照plain类型进行处理 - 必选:否 + -字段类型:String - 默认值:plain - +
## 四、配置示例 @@ -78,13 +101,17 @@ "content": [{ "reader": { "parameter" : { - "broker" : "tcp://0.0.0.1:1883", + "broker" : "tcp://localhost:1883", "topic" : "test", - "username" : "username", - "password" : "password", + "username" : "admin", + "password" : "public", "isCleanSession": true, "qos": 2, - "codec": "plain" + "codec": "plain", + "column" : [ { + "name": "message", + "type" : "string" + }] }, "name" : "emqxreader" }, diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index bd94bc72ad..f754c2846d 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -1,5 +1,21 @@ # Kafka Reader + + +- [Kafka Reader](#kafka-reader) + - [一、插件名称](#一插件名称) + - [二、参数说明](#二参数说明) + - [三、配置示例](#三配置示例) + - [1、kafka09](#1kafka09) + - [2、kafka10](#2kafka10) + - [3、kafka11](#3kafka11) + - [4、kafka](#4kafka) + - [5、kafka->Hive](#5kafka-hive) + + + +
+ ## 一、插件名称 kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: @@ -10,7 +26,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.11 | kafka11reader | | kafka 1.0及以后 | kafkareader | - +
## 二、参数说明 @@ -78,7 +94,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 ```json [ { - "message":"{\"key\": \"key\", \"value\": \"value\"}" + "message":"{\"key\": \"key\", \"message\": \"value\"}" } ] ``` @@ -137,9 +153,10 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } ``` +
## 三、配置示例 -#### 1、kafka09 +### 1、kafka09 ```json { "job" : { @@ -176,7 +193,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 2、kafka10 +### 2、kafka10 ```json { "job": { @@ -215,7 +232,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 3、kafka11 +### 3、kafka11 ```json { "job" : { @@ -252,7 +269,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 4、kafka +### 4、kafka ```json { "job" : { @@ -291,7 +308,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 5、kafka->Hive +### 5、kafka->Hive ```json { "job": { diff --git a/docs/realTime/reader/mongodboplogreader.md b/docs/realTime/reader/mongodboplogreader.md index ece557f274..8b7ad0a590 100644 --- a/docs/realTime/reader/mongodboplogreader.md +++ b/docs/realTime/reader/mongodboplogreader.md @@ -1,13 +1,30 @@ # MongoDB Oplog Reader + + +- [MongoDB Oplog Reader](#mongodb-oplog-reader) + - [一、插件名称](#一插件名称) + - [二、数据源版本](#二数据源版本) + - [三、数据源配置](#三数据源配置) + - [四、参数说明](#四参数说明) + - [五、使用示例](#五使用示例) + - [采集test库下的所有集合](#采集test库下的所有集合) + + + +
## 一、插件名称 名称:**mongodboplogreader**
+
+ ## 二、数据源版本 **MongoDB 4.0及以上**
+
+ ## 三、数据源配置 [MongoDB 4.0副本集搭建](https://dtstack.yuque.com/rd-center/udi643/gufhya)
@@ -20,63 +37,63 @@ - 必选:是 - 默认值:无 - +
- **username** - 描述: 用户名 - 必选:是 - 默认值:无 - +
- **password** - 描述: 密码 - 必选:是 - 默认值:无 - +
- **authenticationMechanism** - 描述: 认证机制,可选:GSSAPI、PLAIN、MONGODB-X509、MONGODB-CR、SCRAM-SHA-1、SCRAM-SHA-256 - 必选:否 - 默认值:无 - +
- **clusterMode** - 描述: 集群模式,可选:REPLICA_SET、MASTER_SLAVE - 必选:是 - 默认值:无 - +
- **monitorDatabases** - 描述: 要监听的库 - 必选:否 - 默认值:无 - +
- **monitorCollections** - 描述:要监听的集合 - 必选:否 - 默认值:无 - +
- **operateType** - 描述:要监听的操作类型,可选:insert、update、delete - 必选:否 - 默认值:无 - +
- **excludeDocId** - 描述:是否排除_id字段 - 必选:否 - 默认值:false - +
- **pavingData** - 描述:是否将解析出的json数据拍平 diff --git a/docs/realTime/reader/pgwalreader.md b/docs/realTime/reader/pgwalreader.md index 9390b88ec2..28940e095a 100644 --- a/docs/realTime/reader/pgwalreader.md +++ b/docs/realTime/reader/pgwalreader.md @@ -1,47 +1,73 @@ # PostgreSQL WAL Reader + + + + +- [PostgreSQL WAL Reader](#postgresql-wal-reader) + - [一、插件名称](#一插件名称) + - [二、数据源版本](#二数据源版本) + - [三、数据库原理及配置](#三数据库原理及配置) + - [四、使用说明](#四使用说明) + - [五、参数说明
](#五参数说明br-) + - [五、配置示例](#五配置示例) + + + +
## 一、插件名称 名称:**pgwalreader**
+
+ ## 二、数据源版本 **PostgreSQL数据库版本至少为10.0及以上**
+
+ +## 三、数据库原理及配置 +[FlinkX PostgreSQL WAL实时采集基本原理及配置](../other/PgWal原理及配置.md) + +
+ -## 三、使用说明 +## 四、使用说明 1、预写日志级别(wal_level)必须为logical
2、该插件基于PostgreSQL逻辑复制及逻辑解码功能实现的,因此PostgreSQL账户至少拥有replication权限,若允许创建slot,则至少拥有超级管理员权限
3、详细原理请参见[PostgreSQL官方文档](http://postgres.cn/docs/10/index.html)
+
+ -## 四、参数说明
+## 五、参数说明
- **jdbcUrl** - 描述:PostgreSQL数据库的jdbc连接字符串,参考文档:[PostgreSQL官方文档](https://jdbc.postgresql.org/documentation/head/connect.html) - 必选:是 - 默认值:无 - +
- **username** - 描述:数据源的用户名 - 必选:是 - 默认值:无 - +
- **password** - 描述:数据源指定用户名的密码 - 必选:是 - 默认值:无 - +
- **tableList** - 描述:需要解析的数据表,格式为schema.table - 必选:否 - 默认值:无 - +
- **cat** - 描述:需要解析的数据更新类型,包括insert、update、delete三种 @@ -49,21 +75,21 @@ - 必选:是 - 默认值:无 - +
- **statusInterval** - 描述:复制期间,数据库和使用者定期交换ping消息。如果数据库或客户端在配置的超时时间内未收到ping消息,则复制被视为已停止,并且将引发异常,并且数据库将释放资源。在PostgreSQL中,ping超时由属性wal_sender_timeout配置(默认= 60秒)。可以将pgjdc中的复制流配置为在需要时或按时间间隔发送反馈(ping)。建议比配置的wal_sender_timeout更频繁地向数据库发送反馈(ping)。在生产环境中,我使用等于wal_sender_timeout / 3的值。它避免了网络潜在的问题,并且可以在不因超时而断开连接的情况下传输更改 - 必选:否 - 默认值:2000 - +
- **lsn** - 描述:要读取PostgreSQL WAL日志序列号的开始位置 - 必选:否 - 默认值:0 - +
- **slotName** - 描述:复制槽名称,根据该值去寻找或创建复制槽 @@ -71,21 +97,21 @@ - 必选:否 - 默认值:无 - +
- **allowCreateSlot** - 描述:是否允许创建复制槽 - 必选:否 - 默认值:true - +
- **temporary** - 描述:复制槽是否为临时性的,true:是;false:否 - 必选:否 - 默认值:true - +
- **pavingData** - 描述:是否将解析出的json数据拍平 @@ -124,7 +150,7 @@ pavingData为false时: - 必选:否 - 默认值:false - +
## 五、配置示例 @@ -182,221 +208,4 @@ pavingData为false时: } } } -``` - -## PostgreSQL实时采集原理 -PostgreSQL 实时采集是基于 PostgreSQL的逻辑复制以及逻辑解码功能来完成的。逻辑复制同步数据的原理是,在wal日志产生的数据库上,由逻辑解析模块对wal日志进行初步的解析,它的解析结果为ReorderBufferChange(可以简单理解为HeapTupleData),再由pgoutput plugin对中间结果进行过滤和消息化拼接后,然后将其发送到订阅端,订阅端通过逻辑解码功能进行解析。 - -### 版本限制 -逻辑复制是pgsql10.0版本之后才支持的,因此此方案只支持10.0之后版本 - - -### 主要涉及模块说明 -| Logical Decoding | PostgreSQL 的逻辑日志来源于解析物理 WAL 日志。
解析 WAL 成为逻辑数据的过程叫 Logical Decoding。 | -| :--- | :--- | -| Replication Slots | 保存逻辑或物理流复制的基础信息。类似 Mysql 的位点信息。
一个 逻辑 slot 创建后,它的相关信息可以通过 pg_replication_slots 系统视图获取。
如果它在 active 状态,则可以通过系统视图 pg_stat_replication 看到一些 slot 的实时的状态信息。 | -| Output Plugins | PostgreSQL 的逻辑流复制协议开放一组可编程接口,用于自定义输数据到客户端的逻辑数据的格式。
这部分实现使用插件的方式被内核集成和使用,称作 Output Plugins。 | -| Exported Snapshots | 当一个逻辑流复制 slot 被创建时,系统会产生一个快照。客户端可以通过它订阅到数据库任意时间点的数据变化。 | - - - -对于修改一条数据之后 ,pgsql订阅端decode解析后的数据格式为 -```json -{"id":"schema1.test1", - "schema":"schema1", -"table":"test1", - "columnList":[ - {"name":"id","type":"int4","index":0}, - {"name":"name","type":"varchar","index":1} - ], - "oldData":["2","23"], - "newData":["2","name1"], - "type":"UPDATE", - "currentLsn":23940928, - "ts":1596358573614 -} -``` -主要包含schema table以及类型`INSERT`, `UPDATE`和`DELETE`以及WAL日志id等相关信息
-
- - -### 逻辑复制 -逻辑复制使用_发布_和_订阅_模型, 其中一个或多个_订阅者_订阅_发布者_ 节点上的一个或多个_发布_。 订阅者从他们订阅的发布中提取数据,逻辑复制是根据复制标识(通常是主键)复制数据对象及其更改的一种方法,因此在上面订阅端收到消息数据实例中可以发现 具备数据库以及表信息外 还具备修改前数据,修改后数据信息以及执行的type和对应的WAL日志ID - -发布可以选择将它们所产生的改变限制在`INSERT`, `UPDATE`和`DELETE`的任意组合上, 类似于触发器被特定事件类型触发。默认情况下,复制所有操作类型。
已发布的table必须配置一个“副本标识”以便能够复制 `UPDATE`和`DELETE`操作, 这样可以在订阅者端识别适当的行来更新或删除。默认情况下,这是主键, 如果有的话。另外唯一的索引(有一些额外的要求)也可以被设置为副本标识。 如果表没有任何合适的键,那么它可以设置为复制标识“full”, 这意味着整个行成为键。但是,这是非常低效的, 并且只能在没有其他可能的解决方案时用作后备
- - -### 创建发布 -为哪些表设置创建一个发布 -```sql -CREATE PUBLICATION name - [ FOR TABLE [ ONLY ] table_name [ * ] [, ...] - | FOR ALL TABLES ] - [ WITH ( publication_parameter [= value] [, ... ] ) ] -``` - - - -### WAL日志 -WAL 是 Write Ahead Log的缩写,中文称之为预写式日志。WAL log也被简称为xlog,每一次change操作都是先写日志再写数据,保证了事务持久性和数据完整性同时又尽量地避免了频繁IO对性能的影响。WAL的中心概念是**数据文件(存储着表和索引)的修改必须在这些动作被日志记录之后才被写入**
WAL日志保存在pg_xlog下,每个xlog文件默认是16MB,为了满足恢复需求,在xlog目录下会产生多个WAL日志,不需要的WAL日志将会被覆盖
WAL具备归档功能,通过归档的WAL文件可以恢复数据库到WAL日志覆盖时间内的任意一个时间点的状态并且有了WAL日志之后,逻辑复制就可以在WAL日志生成之后,对其进行一系列操作之后传递给订阅客户端,使得订阅客户端能实时获取到源服务器上的修改数据
- - -#### WAL何时被写入 -WAL也有个内存缓冲区WAL Buffer,WAL都是先写入缓存中,对于事务操作,缓存的WAL日志是在事务提交的时候写入磁盘的,对于非事务型的由一个异步线程追加进日志文件或者在checkPoint(数据脏页缓存写入磁盘需要先刷新WAL缓存)的时候写入。
- - -#### WAL主要配置 -``` -wal_level 可以选择为minimal, replica, or logical 使用逻辑复制需要设置为logical - -fsync boolean类型 表示是否使用fsync()系统调用把WAL文件刷新到物理磁盘,确保数据库在操作系统或硬件奔溃的情况下可恢复到最终状态 默认是on - -synchronous_commit boolean类型 声明提交一个事务是否需要等待其把WAL日志写入磁盘后再返回,默认值是’on’ - -on:默认值,为on且没有开启同步备库的时候,会当wal日志真正刷新到磁盘永久存储后才会返回客户端事务已提交成功, - 当为on且开启了同步备库的时候(设置了synchronous_standby_names),必须要等事务日志刷新到本地磁盘,并且还要等远程备库也提交到磁盘才能返回客户端已经提交. - -remote_apply:提交将等待, 直到来自当前同步备用数据库的回复表明它们已收到事务的提交记录并应用它, 以便它对备用数据库上的查询可见。 - -remote_write:提交将等待,直到来自当前同步的后备服务器的一个回复指示该服务器已经收到了该事务的提交记录并且已经把该记录写出到后备服务器的操作系统。 - -local:当事务提交时,仅写入本地磁盘即可返回客户端事务提交成功,而不管是否有同步备库。 - -off:写到缓存中就会向客户端返回提交成功,但也不是一直不刷到磁盘,延迟写入磁盘,延迟的时间为最大3倍的wal_writer_delay参数的(默认200ms)的时间,所有如果即使关闭synchronous_commit,也只会造成最多600ms的事务丢失 可能会造成一些最近已提交的事务丢失,但数据库状态是一致的,就像这些事务已经被干净地中止。但对高并发的小事务系统来说,性能来说提升较大。 - - -wal_sync_method enum类型 用来指定向磁盘强制更新WAL日志数据的方法open_datasync fdatasync fsync_writethrough fsync open_sync - - - -Wal_writer_delay 指定wal writer process 把WAL日志写入磁盘的周期 在每个周期中会先把缓存中的WAL日志刷到磁盘 - -``` - - - -### 复制槽 -每个订阅都将通过一个复制槽接收更改,记录某个订阅者的WAL接收情况。
在源数据库写入修改频繁导致WAL日志的写入速度很快,导致大量WAL日志生成,或者订阅者接受日志很慢,在消费远远小于生产的时候,会导致源数据库上的WAL日志还没有传递到备库就被回卷覆盖掉了,如果被覆盖掉的WAL日志文件又没有归档备份,那么订阅者就再也无法消费到此数据。
复制槽则保存了此订阅的接收信息,使得未被接收的WAL日日志不会被回收 - -注意
数据库会记录slot的wal复制位点,并在wal文件夹中保留所有未发送的wal文件,如果客户创建了slot但是后期不再使用就有可能导致数据库的wal日志爆仓,需要及时删除不用的slot
-
可通过以下SQL获取相关信息 -```sql -select * from pg_replication_slots; -``` -字段含义 -```text -Name Type References Description -slot_name name 复制槽的唯一的集群范围标识符 -plugin name 正在使用的包含逻辑槽输出插件的共享对象的基本名称,对于物理插槽则为null。 -slot_type text 插槽类型 - 物理或逻辑 -datoid oid 该插槽所关联的数据库的OID,或为空。 只有逻辑插槽才具有关联的数据库。 -database text 该插槽所关联的数据库的名称,或为空。 只有逻辑插槽才具有关联的数据库。 -active boolean 如果此插槽当前正在使用,则为真 -active_pid integer 如果当前正在使用插槽,则使用此插槽的会话的进程ID。 NULL如果不活动。 -xmin xid 此插槽需要数据库保留的最早事务。 VACUUM无法删除任何后来的事务删除的元组。 -catalog_xmin xid 影响该插槽需要数据库保留的系统目录的最早的事务。 VACUUM不能删除任何后来的事务删除的目录元组。 -restart_lsn pg_lsn 最老的WAL的地址(LSN)仍然可能是该插槽的使用者所需要的,因此在检查点期间不会被自动移除 -``` - - - -### 局限性 - -- 不复制数据库模式和DDL命令。初始模式可以使用`pg_dump --schema-only` 手动复制。后续的模式更改需要手动保持同步。(但是请注意, 两端的架构不需要完全相同。)当实时数据库中的模式定义更改时,逻辑复制是健壮的: 当模式在发布者上发生更改并且复制的数据开始到达订阅者但不符合表模式, 复制将错误,直到模式更新。在很多情况下, 间歇性错误可以通过首先将附加模式更改应用于订阅者来避免。
-- 不复制序列数据。序列支持的序列或标识列中的数据当然会作为表的一部分被复制, 但序列本身仍然会显示订阅者的起始值。如果订阅者被用作只读数据库, 那么这通常不成问题。但是,如果打算对订阅者数据库进行某种切换或故障切换, 则需要将序列更新为最新值,方法是从发布者复制当前数据 (可能使用`pg_dump`)或者从表中确定足够高的值。
-- 不复制`TRUNCATE`命令。当然,可以通过使用`DELETE` 来解决。为了避免意外的`TRUNCATE`调用,可以撤销表的 `TRUNCATE`权限。
-- 不复制大对象 没有什么解决办法,除非在普通表中存储数据。 -- 复制只能从基表到基表。也就是说,发布和订阅端的表必须是普通表,而不是视图, 物化视图,分区根表或外部表。对于分区,您可以一对一地复制分区层次结构, 但目前不能复制到不同的分区设置。尝试复制基表以外的表将导致错误 - - - - -### PostgreSQL实时采集配置 - -#### postgresql.conf设置 -``` -wal_level = logical -``` - - -用于复制链接的角色必须具有`REPLICATION`属性(或者是超级用户) 需要在pg_hba.conf做出如下配置 -``` -host replication all 10.0.3.0/24 md5 -``` - - -### 部分核心代码分析 - - - -#### 执行发布SQL -逻辑复制流是发布/订阅模型,因此生成流之前 先进行发布 -```java -public static final String PUBLICATION_NAME = "dtstack_flinkx"; -public static final String CREATE_PUBLICATION = "CREATE PUBLICATION %s FOR ALL TABLES;"; -public static final String QUERY_PUBLICATION = "SELECT COUNT(1) FROM pg_publication WHERE pubname = '%s';"; - -先执行查找sql 判断是否存在 dtstack_flinkx 的 PUBLICATION -如果不存在 执行创建sql语句 -conn.createStatement() - .execute(String.format(CREATE_PUBLICATION, PUBLICATION_NAME)); -``` - - - -#### 创建一个逻辑复制流 -```java - ChainedLogicalStreamBuilder builder = conn.getReplicationAPI() - .replicationStream() //定义一个逻辑复制流 - .logical() //级别是logical - .withSlotName(format.getSlotName())//复制槽名称 - //协议版本。当前仅支持版本1 - .withSlotOption("proto_version", "1")//槽版本号 - //逗号分隔的要订阅的发布名称列表(接收更改)。 单个发布名称被视为标准对象名称,并可根据需要引用 - .withSlotOption("publication_names", PgWalUtil.PUBLICATION_NAME)//关联的发布名称 - .withStatusInterval(format.getStatusInterval(), TimeUnit.MILLISECONDS); - long lsn = format.getStartLsn(); - if(lsn != 0){ - builder.withStartPosition(LogSequenceNumber.valueOf(lsn)); - } - stream = builder.start(); -``` - -#### 业务处理 -逻辑复制流接收到订阅的消息后 进行编码 获取到相应信息处理 -```java - public void run() { - LOG.info("PgWalListener start running....."); - try { - init(); - while (format.isRunning()) { - //接收到流对象 - ByteBuffer buffer = stream.readPending(); - if (buffer == null) { - continue; - } - //解码为table对象 具体信息为库 表 字段信息 WAL id等 - //然后就可以对其进行处理了 - Table table = decoder.decode(buffer); - if(StringUtils.isBlank(table.getId())){ - continue; - } - String type = table.getType().name().toLowerCase(); - if(!cat.contains(type)){ - continue; - } - if(!tableSet.contains(table.getId())){ - continue; - } - LOG.trace("table = {}",gson.toJson(table)); - ............... - } - } - } -``` - -
- - - - +``` \ No newline at end of file diff --git a/docs/realTime/reader/restapireader.md b/docs/realTime/reader/restapireader.md index f354c823db..f13bcd1da3 100644 --- a/docs/realTime/reader/restapireader.md +++ b/docs/realTime/reader/restapireader.md @@ -11,21 +11,24 @@ - 描述:连接的url - 必选:是 - 默认值:无 - + - 字段类型:String +
- **method** - 描述:request的类型,`post`、`get` - 必选:是 - 默认值:无 + - 字段类型:String +
- -- header +- **header** - 描述:需要添加的报头信息 - 必选:否 - 默认值:无 - + - 字段类型:Map +
diff --git a/docs/realTime/writer/emqxwriter.md b/docs/realTime/writer/emqxwriter.md index b5774574bd..8275c78fa7 100644 --- a/docs/realTime/writer/emqxwriter.md +++ b/docs/realTime/writer/emqxwriter.md @@ -1,4 +1,15 @@ # Emqx Writer + + +- [Emqx Writer](#emqx-writer) + - [一、插件名称](#一插件名称) + - [二、支持的数据源版本](#二支持的数据源版本) + - [三、参数说明
](#三参数说明br-) + - [四、配置示例](#四配置示例) + + + +
## 一、插件名称 @@ -12,29 +23,33 @@ - **broker** - 描述:连接URL信息。 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **topic** - 描述:订阅主题 - 必选:是 + - 字段类型:String - 默认值:无 - +
- **username** - 描述:认证用户名 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **password** - 描述:认证密码 - 必选:否 + - 字段类型:String - 默认值:无 - +
- **isCleanSession** @@ -42,8 +57,9 @@ - false:MQTT服务器保存于客户端会话的的主题与确认位置 - true:MQTT服务器不保存于客户端会话的的主题与确认位置 - 必选:否 + - 字段类型:boolean - 默认值:true - +
- **qos** @@ -52,8 +68,9 @@ - 1:AT_LEAST_ONCE,至少一次; - 2:EXACTLY_ONCE,精准一次; - 必选:否 + - 字段类型:int - 默认值:2 - +
@@ -63,37 +80,35 @@ "job": { "content": [{ "reader": { - "name": "streamreader", - "parameter": { - "column": [ - { - "name": "id", - "type": "id" - }, - { - "name": "user_id", - "type": "int" - }, - { - "name": "name", - "type": "string" - } - ], - "sliceRecordCount" : [ "100"] - } - }, - "writer": { - "writer" : { - "parameter" : { - "broker" : "tcp://0.0.0.1:1883", - "topic" : "test", - "username" : "username", - "password" : "password", - "isCleanSession": true, - "qos": 2 - }, - "name" : "emqxwriter" + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "user_id", + "type": "int" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : [ "100"] } + }, + "writer": { + "parameter" : { + "broker" : "tcp://localhost:1883", + "topic" : "test", + "username" : "admin", + "password" : "public", + "isCleanSession": true, + "qos": 2 + }, + "name" : "emqxwriter" } } ], @@ -103,7 +118,7 @@ "bytes": 0 }, "errorLimit": { - "record": 100 + "record": 1 }, "restore": { "maxRowNumForCheckpoint": 0, diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md index 453e011411..2fe4e04546 100644 --- a/docs/realTime/writer/kafkawriter.md +++ b/docs/realTime/writer/kafkawriter.md @@ -1,5 +1,20 @@ # Kafka Writer + + +- [一、插件名称](#一插件名称) +- [二、参数说明](#二参数说明) +- [三、配置示例](#三配置示例) + - [1、kafka09](#1kafka09) + - [2、kafka10](#2kafka10) + - [3、kafka11](#3kafka11) + - [4、kafka](#4kafka) + - [5、MySQL->kafka](#5mysql-kafka) + + + +
+ ## 一、插件名称 kafka插件存在四个版本,根据kafka版本的不同,插件名称也略有不同。具体对应关系如下表所示: @@ -10,7 +25,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 | kafka 0.11 | kafka11writer | | kafka 1.0及以后 | kafkawriter | - +
## 二、参数说明 @@ -66,9 +81,10 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 - 默认值:无 +
## 三、配置示例 -#### 1、kafka09 +### 1、kafka09 ```json { "job": { @@ -115,7 +131,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 2、kafka10 +### 2、kafka10 ```json { "job": { @@ -163,7 +179,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 3、kafka11 +### 3、kafka11 ```json { "job": { @@ -212,7 +228,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 4、kafka +### 4、kafka ```json { "job": { @@ -260,7 +276,7 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略 } } ``` -#### 5、MySQL->kafka +### 5、MySQL->kafka ```json { "job" : { diff --git a/docs/realTime/writer/restapiwriter.md b/docs/realTime/writer/restapiwriter.md index 6a515b9a1e..9064126e98 100644 --- a/docs/realTime/writer/restapiwriter.md +++ b/docs/realTime/writer/restapiwriter.md @@ -10,42 +10,48 @@ - 描述:连接的url - 必选:是 - 默认值:无 - + - 字段类型:String +
- **method** - 描述:request的类型,`post`、`get` - 必选:是 - 默认值:无 - + - 字段类型:String +
- **header** - 描述:需要添加的报头信息 - 必选:否 - 默认值:无 - + - 字段类型:Map +
- **body** - 描述:发送的数据中包括params - 必选:否 - 默认值:无 - + - 字段类型:Map +
- **params** - 描述:发送的数据中包括params - 必选:否 - 默认值:无 - + - 字段类型:Map +
- **column** - 描述:如果column不为空,那么将数据和字段名一一对应。如果column为空,则返回每个数据的第一个字段。 - 必选:否 - 默认值:无 - + - 字段类型:List +
diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java index 8cfa41b3c6..0cdaf15be0 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java @@ -95,8 +95,10 @@ public static Properties geneConsumerProp(Map consumerSettings, switch (mode){ case EARLIEST: props.put("auto.offset.reset", "earliest"); + break; case LATEST: props.put("auto.offset.reset", "latest"); + break; } return props; } diff --git a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java index c3cd2e307c..5f09e75d31 100644 --- a/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java +++ b/flinkx-kb/flinkx-kb-reader/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseInputFormat.java @@ -33,7 +33,6 @@ import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; -import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -77,7 +76,7 @@ public class KafkaBaseInputFormat extends BaseRichInputFormat { protected InputSplit[] createInputSplitsInternal(int minNumSplits) { InputSplit[] splits = new InputSplit[minNumSplits]; for (int i = 0; i < minNumSplits; i++) { - splits[i] = new GenericInputSplit(i, minNumSplits); + splits[i] = new KafkaInputSplit(i, null); } return splits; } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 2a464515f5..5908122fd9 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -76,8 +76,11 @@ public static void main(String[] args) throws Exception { temp.put(argList.get(i), argList.get(i + 1)); } // 对json中的值进行修改 - HashMap parameter = JsonModifyUtil.CommandTransform(temp.get("-p")); - temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); + String p = temp.get("-p"); + if(StringUtils.isNotBlank(p)){ + HashMap parameter = JsonModifyUtil.CommandTransform(p); + temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); + } // 清空list,填充修改后的参数值 argList.clear(); diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml new file mode 100644 index 0000000000..c80b54f930 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml @@ -0,0 +1,106 @@ + + + + flinkx-oraclelogminer + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oraclelogminer-reader + + + + com.github.noraui + ojdbc8 + 12.2.0.1 + + + + com.github.jsqlparser + jsqlparser + 3.2 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + io.netty + shade.oracleReader.io.netty + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/entity/QueueData.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/entity/QueueData.java new file mode 100644 index 0000000000..bf0e56a10a --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/entity/QueueData.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oraclelogminer.entity; + +import java.util.Map; + +/** + * Date: 2020/06/01 + * Company: www.dtstack.com + * + * @author tudou + */ +public class QueueData { + private long scn; + private Map data; + + public QueueData(long lsn, Map data) { + this.scn = lsn; + this.data = data; + } + + public long getScn() { + return scn; + } + + public Map getData() { + return data; + } + + @Override + public String toString() { + return "QueueData{" + + "scn=" + scn + + ", data=" + data + + '}'; + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java new file mode 100644 index 0000000000..a2423c5d3c --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogFile.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oraclelogminer.format; + +import java.util.Objects; + +/** + * @author jiangbo + * @date 2020/3/31 + */ +public class LogFile { + + private String fileName; + + private Long firstChange; + + private Long nextChange; + + public String getFileName() { + return fileName; + } + + public void setFileName(String fileName) { + this.fileName = fileName; + } + + public Long getFirstChange() { + return firstChange; + } + + public void setFirstChange(Long firstChange) { + this.firstChange = firstChange; + } + + public Long getNextChange() { + return nextChange; + } + + public void setNextChange(Long nextChange) { + this.nextChange = nextChange; + } + + @Override + public String toString() { + return "LogFile{" + + "fileName='" + fileName + '\'' + + ", firstChange=" + firstChange + + ", nextChange=" + nextChange + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o){ + return true; + } + + if (o == null || getClass() != o.getClass()){ + return false; + } + + LogFile logFile = (LogFile) o; + return Objects.equals(fileName, logFile.fileName) && + Objects.equals(firstChange, logFile.firstChange) && + Objects.equals(nextChange, logFile.nextChange); + } + + @Override + public int hashCode() { + return Objects.hash(fileName, firstChange, nextChange); + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java new file mode 100644 index 0000000000..f1ffa5317c --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConfig.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.List; + +/** + * @author jiangbo + * @date 2019/12/14 + */ +public class LogMinerConfig implements Serializable { + + private String driverName = "oracle.jdbc.driver.OracleDriver"; + + private String jdbcUrl; + + private String username; + + private String password; + + /** + * LogMiner从v$logmnr_contents视图中批量拉取条数,值越大,消费存量数据越快 + */ + private int fetchSize = 1000; + + private String listenerTables; + + private String cat = "UPDATE,INSERT,DELETE"; + + /** + * 读取位置: all, current, time, scn + */ + private String readPosition = "current"; + + /** + * 毫秒级时间戳 + */ + private long startTime = 0; + + @JsonProperty("startSCN") + private String startScn = ""; + + private boolean pavingData = false; + + private List table; + + /** + * LogMiner执行查询SQL的超时参数,单位秒 + */ + private Long queryTimeout = 300L; + + /** + * Oracle 12c第二个版本之后LogMiner不支持自动添加日志 + */ + private boolean supportAutoAddLog; + + public boolean getSupportAutoAddLog() { + return supportAutoAddLog; + } + + public void setSupportAutoAddLog(boolean supportAutoAddLog) { + this.supportAutoAddLog = supportAutoAddLog; + } + + public Long getQueryTimeout() { + return queryTimeout; + } + + public void setQueryTimeout(Long queryTimeout) { + this.queryTimeout = queryTimeout; + } + + public List getTable() { + return table; + } + + public void setTable(List table) { + this.table = table; + } + + public String getReadPosition() { + return readPosition; + } + + public void setReadPosition(String readPosition) { + this.readPosition = readPosition; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public String getStartScn() { + return startScn; + } + + public void setStartScn(String startScn) { + this.startScn = startScn; + } + + public boolean getPavingData() { + return pavingData; + } + + public void setPavingData(boolean pavingData) { + this.pavingData = pavingData; + } + + public String getCat() { + return cat; + } + + public void setCat(String cat) { + this.cat = cat; + } + + public String getListenerTables() { + return listenerTables; + } + + public void setListenerTables(String listenerTables) { + this.listenerTables = listenerTables; + } + + public int getFetchSize() { + return fetchSize; + } + + public void setFetchSize(int fetchSize) { + this.fetchSize = fetchSize; + } + + public String getDriverName() { + return driverName; + } + + public void setDriverName(String driverName) { + this.driverName = driverName; + } + + public String getJdbcUrl() { + return jdbcUrl; + } + + public void setJdbcUrl(String jdbcUrl) { + this.jdbcUrl = jdbcUrl; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java new file mode 100644 index 0000000000..5a3f7dd249 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java @@ -0,0 +1,661 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +import com.dtstack.flinkx.oraclelogminer.entity.QueueData; +import com.dtstack.flinkx.oraclelogminer.util.SqlUtil; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RetryUtil; +import org.apache.commons.codec.DecoderException; +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang.time.DateFormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.UnsupportedEncodingException; +import java.sql.CallableStatement; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +/** + * @author jiangbo + * @date 2020/3/27 + */ +public class LogMinerConnection { + + public static final String KEY_PRIVILEGE = "PRIVILEGE"; + public static final String KEY_GRANTED_ROLE = "GRANTED_ROLE"; + public static final String DBA_ROLE = "DBA"; + public static final String EXECUTE_CATALOG_ROLE = "EXECUTE_CATALOG_ROLE"; + public static final int ORACLE_11_VERSION = 11; + public static final List PRIVILEGES_NEEDED = Arrays.asList( + "CREATE SESSION", + "LOGMINING", + "SELECT ANY TRANSACTION", + "SELECT ANY DICTIONARY"); + public static final List ORACLE_11_PRIVILEGES_NEEDED = Arrays.asList( + "CREATE SESSION", + "SELECT ANY TRANSACTION", + "SELECT ANY DICTIONARY"); + public static final int RETRY_TIMES = 3; + public static final int SLEEP_TIME = 2000; + public final static String KEY_SEG_OWNER = "SEG_OWNER"; + public final static String KEY_TABLE_NAME = "TABLE_NAME"; + public final static String KEY_OPERATION = "OPERATION"; + public final static String KEY_TIMESTAMP = "TIMESTAMP"; + public final static String KEY_SQL_REDO = "SQL_REDO"; + public final static String KEY_CSF = "CSF"; + public final static String KEY_SCN = "SCN"; + public final static String KEY_CURRENT_SCN = "CURRENT_SCN"; + public final static String KEY_FIRST_CHANGE = "FIRST_CHANGE#"; + private static final long QUERY_LOG_INTERVAL = 10000; + public static Logger LOG = LoggerFactory.getLogger(LogMinerConnection.class); + public int oracleVersion; + //oracle10数据库字符编码是否设置为GBK + public boolean isGBK = false; + boolean isOracle10; + private LogMinerConfig logMinerConfig; + private Connection connection; + private CallableStatement logMinerStartStmt; + private PreparedStatement logMinerSelectStmt; + private ResultSet logMinerData; + private QueueData result; + private List addedLogFiles = new ArrayList<>(); + private long lastQueryTime; + private boolean logMinerStarted = false; + + /** + * 上一次查询的scn + */ + private Long preScn = null; + + public LogMinerConnection(LogMinerConfig logMinerConfig) { + this.logMinerConfig = logMinerConfig; + } + + public void connect() { + PreparedStatement preparedStatement = null; + try { + ClassUtil.forName(logMinerConfig.getDriverName(), getClass().getClassLoader()); + + connection = RetryUtil.executeWithRetry(() -> DriverManager.getConnection(logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), logMinerConfig.getPassword()), RETRY_TIMES, SLEEP_TIME,false); + + oracleVersion = connection.getMetaData().getDatabaseMajorVersion(); + isOracle10 = oracleVersion == 10; + + if(isOracle10){ + //oracle10开启logMiner之前 需要设置会话级别的日期格式 否则sql语句会含有todate函数 而不是todate函数计算后的值 + preparedStatement = connection.prepareStatement(SqlUtil.SQL_ALTER_DATE_FORMAT); + preparedStatement.execute(); + preparedStatement = connection.prepareStatement(SqlUtil.NLS_TIMESTAMP_FORMAT); + preparedStatement.execute(); + } + + LOG.info("get connection successfully, url:{}, username:{}, Oracle version:{}", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), oracleVersion); + } catch (Exception e){ + String message = String.format("get connection failed,url:[%s], username:[%s], e:%s", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), ExceptionUtil.getErrorMessage(e)); + LOG.error(message); + closeResources(null, preparedStatement, connection); + throw new RuntimeException(message, e); + } + } + + /** + * 关闭LogMiner资源 + */ + public void disConnect() { + //清除日志文件组,下次LogMiner启动时重新加载日志文件 + addedLogFiles.clear(); + + if (null != logMinerStartStmt && logMinerStarted) { + try { + logMinerStartStmt.execute(SqlUtil.SQL_STOP_LOG_MINER); + }catch (SQLException e){ + LOG.warn("close logMiner failed, e = {}", ExceptionUtil.getErrorMessage(e)); + } + logMinerStarted = false; + } + + closeStmt(logMinerStartStmt); + closeResources(logMinerData, logMinerSelectStmt, connection); + } + + /** + * 启动LogMiner + * @param startScn + */ + public void startOrUpdateLogMiner(Long startScn) { + String startSql = null; + try { + // 防止没有数据更新的时候频繁查询数据库,限定查询的最小时间间隔 QUERY_LOG_INTERVAL + if (lastQueryTime > 0) { + long time = System.currentTimeMillis() - lastQueryTime; + if (time < QUERY_LOG_INTERVAL) { + try { + Thread.sleep(QUERY_LOG_INTERVAL-time); + } catch (InterruptedException e) { + LOG.warn("", e); + } + } + } + lastQueryTime = System.currentTimeMillis(); + + if (logMinerConfig.getSupportAutoAddLog()) { + startSql = isOracle10 ? SqlUtil.SQL_START_LOG_MINER_AUTO_ADD_LOG_10 : SqlUtil.SQL_START_LOG_MINER_AUTO_ADD_LOG; + } else { + List newLogFiles = queryLogFiles(preScn); + if (addedLogFiles.equals(newLogFiles)) { + return; + } else { + LOG.info("Log group changed, new log group = {}", GsonUtil.GSON.toJson(newLogFiles)); + addedLogFiles = newLogFiles; + startSql = isOracle10 ? SqlUtil.SQL_START_LOG_MINER_10 : SqlUtil.SQL_START_LOG_MINER; + } + } + + closeStmt(logMinerStartStmt); + + logMinerStartStmt = connection.prepareCall(startSql); + configStatement(logMinerStartStmt); + + logMinerStartStmt.setLong(1, preScn); + logMinerStartStmt.execute(); + + logMinerStarted = true; + LOG.info("start logMiner successfully, preScn:{}, startScn:{}", preScn, startScn); + if(startScn > preScn){ + preScn = startScn; + } + } catch (SQLException e){ + String message = String.format("start logMiner failed, offset:[%s], sql:[%s], e: %s", startScn, startSql, ExceptionUtil.getErrorMessage(e)); + LOG.error(message); + throw new RuntimeException(message, e); + } + } + + /** + * 从LogMiner视图查询数据 + * @param startScn + * @param logMinerSelectSql + */ + public void queryData(Long startScn, String logMinerSelectSql) { + try { + logMinerSelectStmt = connection.prepareStatement(logMinerSelectSql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); + configStatement(logMinerSelectStmt); + + logMinerSelectStmt.setFetchSize(logMinerConfig.getFetchSize()); + logMinerSelectStmt.setLong(1, startScn); + logMinerData = logMinerSelectStmt.executeQuery(); + + LOG.debug("query Log miner data, offset:{}", startScn); + } catch (SQLException e) { + String message = String.format("query logMiner data failed, sql:[%s], e: %s", logMinerSelectSql, ExceptionUtil.getErrorMessage(e)); + LOG.error(message); + throw new RuntimeException(message, e); + } + } + + public Long getStartScn(Long startScn) { + // 恢复位置不为0,则获取上一次读取的日志文件的起始位置开始读取 + if(null != startScn && startScn != 0L){ + startScn = getLogFileStartPositionByScn(startScn); + return startScn; + } + + // 恢复位置为0,则根据配置项进行处理 + if(ReadPosition.ALL.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ + // 获取最开始的scn + startScn = getMinScn(); + } else if(ReadPosition.CURRENT.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ + startScn = getCurrentScn(); + } else if(ReadPosition.TIME.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ + // 根据指定的时间获取对应时间段的日志文件的起始位置 + if (logMinerConfig.getStartTime() == 0) { + throw new IllegalArgumentException("[startTime] must not be null or empty when readMode is [time]"); + } + + startScn = getLogFileStartPositionByTime(logMinerConfig.getStartTime()); + } else if(ReadPosition.SCN.name().equalsIgnoreCase(logMinerConfig.getReadPosition())){ + // 根据指定的scn获取对应日志文件的起始位置 + if(StringUtils.isEmpty(logMinerConfig.getStartScn())){ + throw new IllegalArgumentException("[startSCN] must not be null or empty when readMode is [scn]"); + } + + startScn = Long.parseLong(logMinerConfig.getStartScn()); + } else { + throw new IllegalArgumentException("unsupported readMode : " + logMinerConfig.getReadPosition()); + } + + return startScn; + } + + /** + * oracle会把把重做日志分文件存储,每个文件都有 "FIRST_CHANGE" 和 "NEXT_CHANGE" 标识范围, + * 这里需要根据给定scn找到对应的日志文件,并获取这个文件的 "FIRST_CHANGE",然后从位置 "FIRST_CHANGE" 开始读取, + * 在[FIRST_CHANGE,scn] 范围内的数据需要跳过。 + * + * 视图说明: + * v$archived_log 视图存储已经归档的日志文件 + * v$log 视图存储未归档的日志文件 + */ + private Long getLogFileStartPositionByScn(Long scn) { + Long logFileFirstChange = null; + PreparedStatement lastLogFileStmt = null; + ResultSet lastLogFileResultSet = null; + + try { + lastLogFileStmt = connection.prepareCall(isOracle10 ? SqlUtil.SQL_GET_LOG_FILE_START_POSITION_BY_SCN_10 : SqlUtil.SQL_GET_LOG_FILE_START_POSITION_BY_SCN); + configStatement(lastLogFileStmt); + + lastLogFileStmt.setLong(1, scn); + lastLogFileStmt.setLong(2, scn); + lastLogFileResultSet = lastLogFileStmt.executeQuery(); + while(lastLogFileResultSet.next()){ + logFileFirstChange = lastLogFileResultSet.getLong(KEY_FIRST_CHANGE); + } + + return logFileFirstChange; + } catch (SQLException e) { + LOG.error("根据scn:[{}]获取指定归档日志起始位置出错", scn, e); + throw new RuntimeException(e); + } finally { + closeResources(lastLogFileResultSet, lastLogFileStmt, null); + } + } + + private Long getMinScn(){ + Long minScn = null; + PreparedStatement minScnStmt = null; + ResultSet minScnResultSet = null; + + try { + minScnStmt = connection.prepareCall(SqlUtil.SQL_GET_LOG_FILE_START_POSITION); + configStatement(minScnStmt); + + minScnResultSet = minScnStmt.executeQuery(); + while(minScnResultSet.next()){ + minScn = minScnResultSet.getLong(KEY_FIRST_CHANGE); + } + + return minScn; + } catch (SQLException e) { + LOG.error("获取最早归档日志起始位置出错", e); + throw new RuntimeException(e); + } finally { + closeResources(minScnResultSet, minScnStmt, null); + } + } + + private Long getCurrentScn() { + Long currentScn = null; + CallableStatement currentScnStmt = null; + ResultSet currentScnResultSet = null; + + try { + currentScnStmt = connection.prepareCall(SqlUtil.SQL_GET_CURRENT_SCN); + configStatement(currentScnStmt); + + currentScnResultSet = currentScnStmt.executeQuery(); + while(currentScnResultSet.next()){ + currentScn = currentScnResultSet.getLong(KEY_CURRENT_SCN); + } + + return currentScn; + } catch (SQLException e) { + LOG.error("获取当前的SCN出错:", e); + throw new RuntimeException(e); + } finally { + closeResources(currentScnResultSet, currentScnStmt, null); + } + } + + private Long getLogFileStartPositionByTime(Long time) { + Long logFileFirstChange = null; + + PreparedStatement lastLogFileStmt = null; + ResultSet lastLogFileResultSet = null; + + try { + String timeStr = DateFormatUtils.format(time, "yyyy-MM-dd HH:mm:ss"); + + lastLogFileStmt = connection.prepareCall(isOracle10 ? SqlUtil.SQL_GET_LOG_FILE_START_POSITION_BY_TIME_10 : SqlUtil.SQL_GET_LOG_FILE_START_POSITION_BY_TIME); + configStatement(lastLogFileStmt); + + lastLogFileStmt.setString(1, timeStr); + lastLogFileStmt.setString(2, timeStr); + + if(!isOracle10){ + //oracle10只有两个参数 + lastLogFileStmt.setString(3, timeStr); + } + lastLogFileResultSet = lastLogFileStmt.executeQuery(); + while(lastLogFileResultSet.next()){ + logFileFirstChange = lastLogFileResultSet.getLong(KEY_FIRST_CHANGE); + } + + return logFileFirstChange; + } catch (SQLException e) { + LOG.error("根据时间:[{}]获取指定归档日志起始位置出错", time, e); + throw new RuntimeException(e); + } finally { + closeResources(lastLogFileResultSet, lastLogFileStmt, null); + } + } + + /** + * 关闭数据库连接资源 + * @param rs + * @param stmt + * @param conn + */ + private void closeResources(ResultSet rs, Statement stmt, Connection conn) { + if (null != rs) { + try { + rs.close(); + } catch (SQLException e) { + LOG.warn("Close resultSet error: {}", ExceptionUtil.getErrorMessage(e)); + } + } + + closeStmt(stmt); + + if (null != conn) { + try { + conn.close(); + } catch (SQLException e) { + LOG.warn("Close connection error:{}", ExceptionUtil.getErrorMessage(e)); + } + } + } + + /** + * 根据scn号查询在线及归档日志组 + * @param scn + * @return + * @throws SQLException + */ + private List queryLogFiles(Long scn) throws SQLException{ + List logFiles = new ArrayList<>(); + PreparedStatement statement = null; + ResultSet rs = null; + try { + statement = connection.prepareStatement(isOracle10 ? SqlUtil.SQL_QUERY_LOG_FILE_10 : SqlUtil.SQL_QUERY_LOG_FILE); + statement.setLong(1, scn); + statement.setLong(2, scn); + rs = statement.executeQuery(); + while (rs.next()) { + LogFile logFile = new LogFile(); + logFile.setFileName(rs.getString("name")); + logFile.setFirstChange(rs.getLong("first_change#")); + logFile.setNextChange(rs.getLong("next_change#")); + + logFiles.add(logFile); + } + } finally { + if (null != rs) { + rs.close(); + } + + if (null != statement) { + statement.close(); + } + } + + lastQueryTime = System.currentTimeMillis(); + return logFiles; + } + + public boolean hasNext() throws SQLException, UnsupportedEncodingException, DecoderException { + if (null == logMinerData || logMinerData.isClosed()) { + return false; + } + + String sqlLog; + while (logMinerData.next()) { + String sql = logMinerData.getString(KEY_SQL_REDO); + if(StringUtils.isBlank(sql)){ + continue; + } + StringBuilder sqlRedo = new StringBuilder(sql); + if(SqlUtil.isCreateTemporaryTableSql(sqlRedo.toString())){ + continue; + } + long scn = logMinerData.getLong(KEY_SCN); + String operation = logMinerData.getString(KEY_OPERATION); + + // 用CSF来判断一条sql是在当前这一行结束,sql超过4000 字节,会处理成多行 + boolean isSqlNotEnd = logMinerData.getBoolean(KEY_CSF); + //是否存在多条SQL + boolean hasMultiSql = isSqlNotEnd; + + while(isSqlNotEnd){ + logMinerData.next(); + sqlRedo.append(logMinerData.getString(KEY_SQL_REDO)); + isSqlNotEnd = logMinerData.getBoolean(KEY_CSF); + } + + //oracle10中文编码且字符串大于4000,LogMiner可能出现中文乱码导致SQL解析异常 + if(hasMultiSql && isOracle10 && isGBK){ + String redo = sqlRedo.toString(); + + String hexStr = new String(Hex.encodeHex(redo.getBytes("GBK"))); + + if(hexStr.contains("3f2c") || hexStr.contains("3f20616e64")){ + LOG.info("current scn is: {},\noriginal redo sql is: {},\nhex redo string is: {}", scn, redo, hexStr); + if("INSERT".equalsIgnoreCase(operation)){ + //insert into values('','','',) value后可能存在中文乱码 + //?, -> ', + hexStr = hexStr.replace("3f2c", "272c"); + }else{ + //update set "" = '' and "" = '' where "" = '' and "" = '' where后可能存在中文乱码 + //delete from where "" = '' and "" = '' where后可能存在中文乱码 + //?空格and -> '空格and + hexStr = hexStr.replace("3f20616e64", "2720616e64"); + } + sqlLog = new String(Hex.decodeHex(hexStr.toCharArray()), "GBK"); + LOG.info("final redo sql is: {}", sqlLog); + }else{ + sqlLog = new String(Hex.decodeHex(hexStr.toCharArray()), "GBK"); + } + + }else{ + sqlLog = sqlRedo.toString(); + } + + String schema = logMinerData.getString(KEY_SEG_OWNER); + String tableName = logMinerData.getString(KEY_TABLE_NAME); + Timestamp timestamp = logMinerData.getTimestamp(KEY_TIMESTAMP); + + Map data = new HashMap<>(); + data.put("schema", schema); + data.put("tableName", tableName); + data.put("operation", operation); + data.put("sqlLog", sqlLog); + data.put("opTime", timestamp); + + result = new QueueData(scn, data); + return true; + } + + return false; + } + + //判断连接是否正常 + public boolean isValid() { + try { + return connection != null && connection.isValid(2000); + } catch (SQLException e) { + return false; + } + } + + public void checkPrivileges() { + try (Statement statement = connection.createStatement()) { + + queryDataBaseEncoding(); + + List roles = getUserRoles(statement); + if (roles.contains(DBA_ROLE)) { + return; + } + + if (!roles.contains(EXECUTE_CATALOG_ROLE)) { + throw new IllegalArgumentException("非DBA角色的用户必须是[EXECUTE_CATALOG_ROLE]角色,请执行sql赋权:GRANT EXECUTE_CATALOG_ROLE TO USERNAME"); + } + + if (containsNeededPrivileges(statement)) { + return; + } + + String message; + if(ORACLE_11_VERSION <= oracleVersion){ + message = "权限不足,请执行sql赋权:GRANT CREATE SESSION, EXECUTE_CATALOG_ROLE, SELECT ANY TRANSACTION, FLASHBACK ANY TABLE, SELECT ANY TABLE, LOCK ANY TABLE, SELECT ANY DICTIONARY TO USER_ROLE;"; + }else{ + message = "权限不足,请执行sql赋权:GRANT LOGMINING, CREATE SESSION, SELECT ANY TRANSACTION ,SELECT ANY DICTIONARY TO USER_ROLE;"; + } + + throw new IllegalArgumentException(message); + } catch (SQLException e) { + throw new RuntimeException("检查权限出错", e); + } + } + + private boolean containsNeededPrivileges(Statement statement) { + try (ResultSet rs = statement.executeQuery(SqlUtil.SQL_QUERY_PRIVILEGES)) { + List privileges = new ArrayList<>(); + while (rs.next()) { + String privilege = rs.getString(KEY_PRIVILEGE); + if (StringUtils.isNotEmpty(privilege)) { + privileges.add(privilege.toUpperCase()); + } + } + + int privilegeCount = 0; + List privilegeList; + if (oracleVersion <= ORACLE_11_VERSION) { + privilegeList = ORACLE_11_PRIVILEGES_NEEDED; + } else { + privilegeList = PRIVILEGES_NEEDED; + } + for (String privilege : privilegeList) { + if (privileges.contains(privilege)) { + privilegeCount++; + } + } + + return privilegeCount == privilegeList.size(); + } catch (SQLException e) { + throw new RuntimeException("检查用户权限出错", e); + } + } + + private List getUserRoles(Statement statement) { + try (ResultSet rs = statement.executeQuery(SqlUtil.SQL_QUERY_ROLES)) { + List roles = new ArrayList<>(); + while (rs.next()) { + String role = rs.getString(KEY_GRANTED_ROLE); + if (StringUtils.isNotEmpty(role)) { + roles.add(role.toUpperCase()); + } + } + + return roles; + } catch (SQLException e) { + throw new RuntimeException("检查用户角色出错", e); + } + } + + /** + * 查询Oracle10数据库的字符编码 + */ + private void queryDataBaseEncoding(){ + if(isOracle10){ + try (Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(SqlUtil.SQL_QUERY_ENCODING)) { + rs.next(); + String encoding = rs.getString(1); + LOG.info("current oracle encoding is {}", encoding); + isGBK = encoding.contains("GBK"); + } catch (SQLException e) { + throw new RuntimeException("检查用户角色出错", e); + } + } + } + + private void configStatement(java.sql.Statement statement) throws SQLException { + if (logMinerConfig.getQueryTimeout() != null) { + statement.setQueryTimeout(logMinerConfig.getQueryTimeout().intValue()); + } + } + + public QueueData next() { + return result; + } + + + /** + * 关闭logMinerSelectStmt + */ + public void closeStmt(){ + try { + if(logMinerSelectStmt != null && !logMinerSelectStmt.isClosed()){ + logMinerSelectStmt.close(); + } + }catch (SQLException e){ + LOG.warn("Close logMinerSelectStmt error", e); + } + logMinerSelectStmt = null; + } + + /** + * 关闭Statement + */ + private void closeStmt(Statement statement){ + try { + if(statement != null && !statement.isClosed()){ + statement.close(); + } + }catch (SQLException e){ + LOG.warn("Close statement error", e); + } + } + + public void setPreScn(Long preScn) { + this.preScn = preScn; + } + + public enum ReadPosition{ + ALL, CURRENT, TIME, SCN + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java new file mode 100644 index 0000000000..bc82ad5f6a --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerListener.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +import com.dtstack.flinkx.oraclelogminer.entity.QueueData; +import com.dtstack.flinkx.oraclelogminer.util.OraUtil; +import com.dtstack.flinkx.oraclelogminer.util.SqlUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import net.sf.jsqlparser.JSQLParserException; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * @author jiangbo + * @date 2020/3/27 + */ +public class LogMinerListener implements Runnable { + + public static Logger LOG = LoggerFactory.getLogger(LogMinerListener.class); + + private BlockingQueue queue; + + private ExecutorService executor; + + private LogMinerConfig logMinerConfig; + + private LogMinerConnection logMinerConnection; + + private PositionManager positionManager; + + private LogParser logParser; + + private boolean running = false; + + private String logMinerSelectSql; + + private transient LogMinerListener listener; + + /** + * 连续接收到错误数据的次数 + */ + private int failedTimes; + + public LogMinerListener(LogMinerConfig logMinerConfig, PositionManager positionManager) { + this.positionManager = positionManager; + this.logMinerConfig = logMinerConfig; + this.listener = this; + } + + public void init() { + queue = new SynchronousQueue<>(false); + + ThreadFactory namedThreadFactory = new ThreadFactoryBuilder().setNameFormat("LogMiner-pool-%d").build(); + executor = new ThreadPoolExecutor(1, + 1, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(1024), + namedThreadFactory, + new ThreadPoolExecutor.AbortPolicy()); + + logMinerConnection = new LogMinerConnection(logMinerConfig); + logParser = new LogParser(logMinerConfig); + } + + public void start() { + logMinerConnection.connect(); + logMinerConnection.checkPrivileges(); + + Long startScn = logMinerConnection.getStartScn(positionManager.getPosition()); + logMinerConnection.setPreScn(startScn); + positionManager.updatePosition(startScn); + + logMinerSelectSql = SqlUtil.buildSelectSql(logMinerConfig.getCat(), logMinerConfig.getListenerTables()); + executor.execute(this); + running = true; + } + + @Override + public void run() { + Thread.currentThread().setUncaughtExceptionHandler((t, e) -> { + LOG.warn("LogMinerListener run failed, Throwable = {}", ExceptionUtil.getErrorMessage(e)); + executor.execute(listener); + LOG.info("Re-execute LogMinerListener successfully"); + }); + + while (running) { + QueueData log = null; + try { + if (logMinerConnection.hasNext()) { + log = logMinerConnection.next(); + queue.put(logParser.parse(log, logMinerConnection.isOracle10)); + } else { + logMinerConnection.closeStmt(); + logMinerConnection.startOrUpdateLogMiner(positionManager.getPosition()); + logMinerConnection.queryData(positionManager.getPosition(), logMinerSelectSql); + LOG.debug("Update log and continue read:{}", positionManager.getPosition()); + } + } catch (Exception e) { + StringBuilder sb = new StringBuilder(512); + sb.append("LogMinerListener thread exception: current scn =") + .append(positionManager.getPosition()); + if (e instanceof JSQLParserException) { + sb.append(",\nlog = ").append(log); + } + sb.append(",\ne = ").append(ExceptionUtil.getErrorMessage(e)); + String msg = sb.toString(); + LOG.warn(msg); + try { + queue.put(new QueueData(0L, Collections.singletonMap("e", msg))); + Thread.sleep(2000L); + } catch (InterruptedException ex) { + LOG.warn("error to put exception message into queue, e = {}", ExceptionUtil.getErrorMessage(ex)); + } + try { + logMinerConnection.disConnect(); + } catch (Exception e1) { + LOG.warn("LogMiner Thread disConnect exception, e = {}", ExceptionUtil.getErrorMessage(e1)); + } + + logMinerConnection.connect(); + } + } + } + + public void stop() { + if (null != executor && !executor.isShutdown()) { + executor.shutdown(); + running = false; + } + + if (null != queue) { + queue.clear(); + } + + if (null != logMinerConnection) { + logMinerConnection.disConnect(); + } + } + + public Map getData() { + try { + QueueData data = queue.take(); + if (data.getScn() != 0L) { + positionManager.updatePosition(data.getScn()); + failedTimes = 0; + return data.getData(); + } + if (++failedTimes >= 3) { + String errorMsg = (String)data.getData().get("e"); + StringBuilder sb = new StringBuilder(errorMsg.length() + 128); + sb.append("Error data is received 3 times continuously, "); + Pair pair = OraUtil.parseErrorMsg(errorMsg); + if(pair != null){ + sb.append("\nthe Cause maybe : ") + .append(pair.getLeft()) + .append(", \nand the Solution maybe : ") + .append(pair.getRight()) + .append(", "); + } + sb.append("\nerror msg is : ").append(errorMsg); + throw new RuntimeException(sb.toString()); + } + } catch (InterruptedException e) { + LOG.warn("Get data from queue error:", e); + } + + return null; + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogParser.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogParser.java new file mode 100644 index 0000000000..afae8bc552 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogParser.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +import com.dtstack.flinkx.oraclelogminer.entity.QueueData; +import com.dtstack.flinkx.util.SnowflakeIdWorker; +import net.sf.jsqlparser.JSQLParserException; +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.ExpressionVisitorAdapter; +import net.sf.jsqlparser.expression.operators.relational.EqualsTo; +import net.sf.jsqlparser.expression.operators.relational.ExpressionList; +import net.sf.jsqlparser.parser.CCJSqlParserUtil; +import net.sf.jsqlparser.schema.Column; +import net.sf.jsqlparser.statement.Statement; +import net.sf.jsqlparser.statement.delete.Delete; +import net.sf.jsqlparser.statement.insert.Insert; +import net.sf.jsqlparser.statement.update.Update; +import org.apache.commons.collections.MapUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * @author jiangbo + * @date 2020/3/30 + */ +public class LogParser { + + public static Logger LOG = LoggerFactory.getLogger(LogParser.class); + + //TO_DATE函数值匹配 + public static Pattern toDatePattern = Pattern.compile("(?i)(?(TO_DATE\\('(?(.*?))',\\s+'YYYY-MM-DD HH24:MI:SS'\\)))"); + //TO_TIMESTAMP函数值匹配 + public static Pattern timeStampPattern = Pattern.compile("(?i)(?(TO_TIMESTAMP\\('(?(.*?))'\\)))"); + + + public static SnowflakeIdWorker idWorker = new SnowflakeIdWorker(1, 1); + + private LogMinerConfig config; + + public LogParser(LogMinerConfig config) { + this.config = config; + } + + private static String cleanString(String str) { + if("NULL".equalsIgnoreCase(str)){ + return ""; + } + + if (str.startsWith("TIMESTAMP")) { + str = str.replace("TIMESTAMP ", ""); + } + + if (str.startsWith("'") && str.endsWith("'") && str.length() != 1) { + str = str.substring(1, str.length() - 1); + } + + if (str.startsWith("\"") && str.endsWith("\"") && str.length() != 1) { + str = str.substring(1, str.length() - 1); + } + + return str.replace("IS NULL","= NULL").trim(); + } + + private static void parseInsertStmt(Insert insert, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap){ + for (Column column : insert.getColumns()){ + afterDataMap.put(cleanString(column.getColumnName()), null); + } + + ExpressionList eList = (ExpressionList) insert.getItemsList(); + List valueList = eList.getExpressions(); + int i =0; + for (String key : afterDataMap.keySet()){ + String value = cleanString(valueList.get(i).toString()); + afterDataMap.put(key, value); + beforeDataMap.put(key, null); + i++; + } + } + + private static void parseUpdateStmt(Update update, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap, String sqlRedo){ + Iterator iterator = update.getExpressions().iterator(); + for (Column c : update.getColumns()){ + afterDataMap.put(cleanString(c.getColumnName()), cleanString(iterator.next().toString())); + } + + if(update.getWhere() != null){ + update.getWhere().accept(new ExpressionVisitorAdapter() { + @Override + public void visit(final EqualsTo expr){ + String col = cleanString(expr.getLeftExpression().toString()); + if(afterDataMap.containsKey(col)){ + String value = cleanString(expr.getRightExpression().toString()); + beforeDataMap.put(col, value); + } else { + String value = cleanString(expr.getRightExpression().toString()); + beforeDataMap.put(col, value); + afterDataMap.put(col, value); + } + } + }); + }else{ + LOG.error("where is null when LogParser parse sqlRedo, sqlRedo = {}, update = {}", sqlRedo, update.toString()); + } + } + + private static void parseDeleteStmt(Delete delete, LinkedHashMap beforeDataMap, LinkedHashMap afterDataMap){ + delete.getWhere().accept(new ExpressionVisitorAdapter(){ + @Override + public void visit(final EqualsTo expr){ + String col = cleanString(expr.getLeftExpression().toString()); + String value = cleanString(expr.getRightExpression().toString()); + beforeDataMap.put(col, value); + afterDataMap.put(col, null); + } + }); + } + + public QueueData parse(QueueData pair, boolean isOracle10) throws JSQLParserException { + Map logData = pair.getData(); + String schema = MapUtils.getString(logData, "schema"); + String tableName = MapUtils.getString(logData, "tableName"); + String operation = MapUtils.getString(logData, "operation"); + String sqlLog = MapUtils.getString(logData, "sqlLog"); + String sqlRedo = sqlLog.replace("IS NULL", "= NULL"); + //只有oracle10需要进行toDate toTimestamp转换 + LOG.debug("before parse toDate/toTimestamp sqlRedo = {}",sqlRedo); + if (isOracle10) { + sqlRedo = parseToTimeStamp(parseToDate(sqlRedo)); + } + Timestamp timestamp = (Timestamp)MapUtils.getObject(logData, "opTime"); + + Map message = new LinkedHashMap<>(); + message.put("scn", pair.getScn()); + message.put("type", operation); + message.put("schema", schema); + message.put("table", tableName); + message.put("ts", idWorker.nextId()); + message.put("opTime", timestamp); + + + Statement stmt; + try { + stmt = CCJSqlParserUtil.parse(sqlRedo); + }catch (JSQLParserException e){ + LOG.info("sqlRedo = {}", sqlRedo); + stmt = CCJSqlParserUtil.parse(sqlRedo.replace("\\'","\\ '")); + } + LinkedHashMap afterDataMap = new LinkedHashMap<>(); + LinkedHashMap beforeDataMap = new LinkedHashMap<>(); + + if (stmt instanceof Insert){ + parseInsertStmt((Insert) stmt, beforeDataMap, afterDataMap); + }else if (stmt instanceof Update){ + parseUpdateStmt((Update) stmt, beforeDataMap, afterDataMap, sqlRedo); + }else if (stmt instanceof Delete){ + parseDeleteStmt((Delete) stmt, beforeDataMap, afterDataMap); + } + + if (config.getPavingData()) { + afterDataMap.forEach((key, val) -> message.put("after_" + key, val)); + + beforeDataMap.forEach((key, val) -> message.put("before_" + key, val)); + + return new QueueData(pair.getScn(), message); + } else { + message.put("before", beforeDataMap); + message.put("after", afterDataMap); + Map event = Collections.singletonMap("message", message); + + return new QueueData(pair.getScn(), event); + } + } + + /** + * 解析to_date函数 + * + * @param redoLog + * @return + */ + private String parseToDate(String redoLog) { + Matcher matcher = toDatePattern.matcher(redoLog); + HashMap replaceData = new HashMap<>(8); + while (matcher.find()) { + String key = matcher.group("toDate"); + String value = "'" + matcher.group("datetime") + "'"; + replaceData.put(key, value); + } + return replace(redoLog, replaceData); + } + + + /** + * 解析to_timestamp函数 + * + * @param redoLog + * @return + */ + private String parseToTimeStamp(String redoLog) { + Matcher matcher = timeStampPattern.matcher(redoLog); + HashMap replaceData = new HashMap<>(8); + while (matcher.find()) { + String key = matcher.group("toTimeStamp"); + String value = "'" + matcher.group("datetime") + "'"; + replaceData.put(key, value); + } + return replace(redoLog, replaceData); + } + + + private String replace(String redoLog, HashMap replaceData) { + if (MapUtils.isNotEmpty(replaceData)) { + for (Map.Entry entry : replaceData.entrySet()) { + //to_timeStamp/to_date()函数有括号 需要转义 + String k = entry.getKey().replaceAll("\\(", "\\\\(").replaceAll("\\)", "\\\\)"); + String v = entry.getValue(); + redoLog = redoLog.replaceAll(k, v); + } + } + return redoLog; + } + +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java new file mode 100644 index 0000000000..c653c18221 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/OracleLogMinerInputFormat.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.dtstack.flinkx.restore.FormatState; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.util.Map; + +/** + * @author jiangbo + * @date 2019/12/14 + * + * 名词说明: + * SCN 即系统改变号(System Change Number) + */ +public class OracleLogMinerInputFormat extends BaseRichInputFormat { + + public LogMinerConfig logMinerConfig; + + private transient LogMinerListener logMinerListener; + + private transient PositionManager positionManager; + + @Override + protected InputSplit[] createInputSplitsInternal(int i) { + return new InputSplit[]{new GenericInputSplit(1,1)}; + } + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + positionManager = new PositionManager(); + initPosition(); + + logMinerListener = new LogMinerListener(logMinerConfig, positionManager); + } + + private void initPosition() { + if (null != formatState && formatState.getState() != null) { + positionManager.updatePosition((Long)formatState.getState()); + } + } + + @Override + protected void openInternal(InputSplit inputSplit) { + logMinerListener.init(); + logMinerListener.start(); + } + + @Override + protected Row nextRecordInternal(Row row) { + Map data = logMinerListener.getData(); + if(null != data) { + return Row.of(data); + } else { + return null; + } + } + + @Override + public FormatState getFormatState() { + super.getFormatState(); + + if (formatState != null) { + formatState.setState(positionManager.getPosition()); + } + + return formatState; + } + + @Override + public boolean reachedEnd() { + return false; + } + + @Override + protected void closeInternal() throws IOException { + if (null != logMinerListener) { + try { + logMinerListener.stop(); + } catch (Exception e) { + throw new IOException("close listener error", e); + } + } + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/PositionManager.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/PositionManager.java new file mode 100644 index 0000000000..b5a1ecd899 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/PositionManager.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.format; + +/** + * @author jiangbo + * @date 2020/3/27 + */ +public class PositionManager { + + private Long scn; + + public void updatePosition(Long scn) { + this.scn = scn; + } + + public Long getPosition() { + return this.scn; + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java new file mode 100644 index 0000000000..559a28d6c5 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OracleLogMinerInputFormatBuilder.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oraclelogminer.reader; + +import com.dtstack.flinkx.config.SpeedConfig; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import com.dtstack.flinkx.oraclelogminer.format.LogMinerConfig; +import com.dtstack.flinkx.oraclelogminer.format.LogMinerConnection; +import com.dtstack.flinkx.oraclelogminer.format.OracleLogMinerInputFormat; +import com.dtstack.flinkx.oraclelogminer.util.SqlUtil; +import com.dtstack.flinkx.util.ClassUtil; +import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.RetryUtil; +import com.dtstack.flinkx.util.TelnetUtil; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; + +/** + * @author jiangbo + * @date 2019/12/16 + */ +public class OracleLogMinerInputFormatBuilder extends BaseRichInputFormatBuilder { + + private OracleLogMinerInputFormat format; + + public OracleLogMinerInputFormatBuilder() { + super.format = format = new OracleLogMinerInputFormat(); + } + + public void setLogMinerConfig(LogMinerConfig logMinerConfig){ + format.logMinerConfig = logMinerConfig; + } + + @Override + protected void checkFormat() { + LogMinerConfig config = format.logMinerConfig; + StringBuilder sb = new StringBuilder(256); + if (StringUtils.isBlank(config.getJdbcUrl())) { + sb.append("No jdbc URL supplied;\n"); + }else{ + //检测数据源连通性 + TelnetUtil.telnet(config.getJdbcUrl()); + } + if (StringUtils.isBlank(config.getUsername())) { + sb.append("No database username supplied;\n"); + } + if (StringUtils.isBlank(config.getPassword())) { + sb.append("No database password supplied;\n"); + } + + if(sb.length() > 0){ + //JDBC URL、username、password其中之一未配置,直接先抛出异常 + throw new IllegalArgumentException(sb.toString()); + } + + if(config.getFetchSize() < 1){ + sb.append("fetchSize must bigger than 0;\n"); + } + List list = Arrays.asList(LogMinerConnection.ReadPosition.ALL.name(), + LogMinerConnection.ReadPosition.CURRENT.name(), + LogMinerConnection.ReadPosition.TIME.name(), + LogMinerConnection.ReadPosition.SCN.name()); + if(StringUtils.isBlank(config.getReadPosition()) + || !list.contains(config.getReadPosition().toUpperCase(Locale.ENGLISH))){ + sb.append("readPosition must be one of [all, current, time, scn], current readPosition is [") + .append(config.getReadPosition()) + .append("];\n"); + } + if (LogMinerConnection.ReadPosition.TIME.name().equalsIgnoreCase(config.getReadPosition()) + && config.getStartTime() == 0){ + sb.append("[startTime] must be supplied when readPosition is [time];\n"); + } + + //校验logMiner cat + if (StringUtils.isNotEmpty(config.getCat())) { + HashSet set = Sets.newHashSet("INSERT", "UPDATE", "DELETE"); + List cats = Lists.newArrayList(config.getCat().toUpperCase().split(",")); + cats.removeIf(s -> set.contains(s.toUpperCase(Locale.ENGLISH))); + if (CollectionUtils.isNotEmpty(cats)) { + sb.append("logMiner cat not support-> ") + .append(GsonUtil.GSON.toJson(cats)) + .append(",just support->") + .append(GsonUtil.GSON.toJson(set)) + .append(";\n"); + } + } + + + SpeedConfig speed = format.getDataTransferConfig().getJob().getSetting().getSpeed(); + + if(speed.getReaderChannel() > 1){ + sb.append("logMiner can not support readerChannel bigger than 1, current readerChannel is [") + .append(speed.getReaderChannel()) + .append("];\n"); + }else if(speed.getChannel() > 1){ + sb.append("logMiner can not support channel bigger than 1, current channel is [") + .append(speed.getChannel()) + .append("];\n"); + } + + ClassUtil.forName(config.getDriverName(), getClass().getClassLoader()); + try( + Connection connection = RetryUtil.executeWithRetry( + () -> DriverManager.getConnection( + config.getJdbcUrl(), + config.getUsername(), + config.getPassword()), + LogMinerConnection.RETRY_TIMES, + LogMinerConnection.SLEEP_TIME, + false); + Statement statement = connection.createStatement(); + ) { + int oracleVersion = connection.getMetaData().getDatabaseMajorVersion(); + LOG.info("current Oracle version is:{}", oracleVersion); + + //1、校验Oracle账号用户角色组 + ResultSet rs = statement.executeQuery(SqlUtil.SQL_QUERY_ROLES); + List roles = new ArrayList<>(); + while (rs.next()) { + String role = rs.getString("GRANTED_ROLE"); + if (StringUtils.isNotEmpty(role)) { + roles.add(role.toUpperCase()); + } + } + //非DBA角色且非EXECUTE_CATALOG_ROLE角色 + if (!roles.contains("DBA") && !roles.contains("EXECUTE_CATALOG_ROLE")) { + sb.append("Non-DBA role users must be [EXECUTE_CATALOG_ROLE] role, ") + .append("current roles are: [") + .append(GsonUtil.GSON.toJson(roles)) + .append("], please execute sql for empowerment:GRANT EXECUTE_CATALOG_ROLE TO ") + .append(config.getUsername()) + .append(";\n"); + } + + //2、校验Oracle账号权限 + rs = statement.executeQuery(SqlUtil.SQL_QUERY_PRIVILEGES); + List privileges = new ArrayList<>(); + while (rs.next()) { + String privilege = rs.getString("PRIVILEGE"); + if (StringUtils.isNotEmpty(privilege)) { + privileges.add(privilege.toUpperCase()); + } + } + + int privilegeCount = 0; + List privilegeList; + //Oracle 11 + if (oracleVersion <= 11) { + privilegeList = SqlUtil.ORACLE_11_PRIVILEGES_NEEDED; + } else { + privilegeList = SqlUtil.PRIVILEGES_NEEDED; + } + for (String privilege : privilegeList) { + if (privileges.contains(privilege)) { + privilegeCount++; + } + } + + if(privilegeCount != privilegeList.size()){ + if (oracleVersion <= 11) { + sb.append("Insufficient permissions, ") + .append("current permissions are :") + .append(GsonUtil.GSON.toJson(privileges)) + .append(",please execute sql for empowerment:grant create session, execute_catalog_role, select any transaction, flashback any table, select any table, lock any table, select any dictionary to ") + .append(config.getUsername()) + .append(";\n"); + }else{ + sb.append("Insufficient permissions, ") + .append("current permissions are :") + .append(GsonUtil.GSON.toJson(privilegeList)) + .append(",please execute sql for empowerment:grant create session, execute_catalog_role, select any transaction, flashback any table, select any table, lock any table, logmining, select any dictionary to ") + .append(config.getUsername()) + .append(";\n"); + } + } + + //3、检查Oracle数据库是否开启日志归档 + rs = statement.executeQuery(SqlUtil.SQL_QUERY_LOG_MODE); + rs.next(); + String logMode = rs.getString(1); + if(!"ARCHIVELOG".equalsIgnoreCase(logMode)){ + sb.append("oracle logMode is ") + .append(logMode) + .append(", please enable log archiving;\n"); + } + + //4、检查Oracle数据库是否开启ALL追加日志 + rs = statement.executeQuery(SqlUtil.SQL_QUERY_SUPPLEMENTAL_LOG_DATA_ALL); + rs.next(); + if(!"YES".equalsIgnoreCase(rs.getString(1))){ + sb.append("supplemental_log_data_all is not enabled, please execute sql to enable this config: alter database add supplemental log data (all) columns;\n"); + } + + rs.close(); + + if(sb.length() > 0){ + throw new IllegalArgumentException(sb.toString()); + } + }catch (SQLException e){ + + StringBuilder detailsInfo = new StringBuilder(sb.length() + 128); + + if(sb.length() > 0){ + detailsInfo.append(" logMiner config not right,details is ").append(sb.toString()); + } + + detailsInfo.append(" \n error to check logMiner config, e = " ).append(ExceptionUtil.getErrorMessage(e)); + + throw new RuntimeException(detailsInfo.toString(), e); + } + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java new file mode 100644 index 0000000000..f13e174473 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/reader/OraclelogminerReader.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.oraclelogminer.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.oraclelogminer.format.LogMinerConfig; +import com.dtstack.flinkx.reader.BaseDataReader; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; + +/** + * @author jiangbo + * @date 2019/12/14 + */ +public class OraclelogminerReader extends BaseDataReader { + + private LogMinerConfig logMinerConfig; + + public OraclelogminerReader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + + try { + logMinerConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), LogMinerConfig.class); + } catch (Exception e) { + throw new RuntimeException("parse logMiner config error:", e); + } + + buildTableListenerRegex(); + } + + private void buildTableListenerRegex(){ + if (CollectionUtils.isEmpty(logMinerConfig.getTable())) { + return; + } + + String tableListener = StringUtils.join(logMinerConfig.getTable(), ","); + logMinerConfig.setListenerTables(tableListener); + } + + @Override + public DataStream readData() { + OracleLogMinerInputFormatBuilder builder = new OracleLogMinerInputFormatBuilder(); + builder.setDataTransferConfig(dataTransferConfig); + builder.setLogMinerConfig(logMinerConfig); + builder.setRestoreConfig(restoreConfig); + + builder.setMonitorUrls(monitorUrls); + builder.setBytes(bytes); + + return createInput(builder.finish()); + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/OraUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/OraUtil.java new file mode 100644 index 0000000000..00fe3db1d3 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/OraUtil.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oraclelogminer.util; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Date: 2020/12/28 + * Company: www.dtstack.com + * + * @author tudou + */ +public class OraUtil { + + /** + * 解析LogMiner异常信息,尝试分析原因及给出解决方法 + * @param errorMsg + * @return Pair L: Cause, R: Solution + */ + public static Pair parseErrorMsg(String errorMsg){ + Pair pair = null; + if(StringUtils.isNotBlank(errorMsg)){ + if(errorMsg.contains("ORA-01284")){ + pair = Pair.of("The file or directory may not exist or may be inaccessible or pathname exceeds 256 characters", + "Please ensure that the file and the directory exist and are accessible"); + }else if(errorMsg.contains("ORA-01013")){ + pair = Pair.of("sql is timeout when query data from logMiner", + "the default value of logMiner parameter: [queryTimeout] is 300S, it may be useful to adjust [queryTimeout], for example: [\"queryTimeout\": 360]"); + }else if(errorMsg.contains("ORA-00604")){ + pair = Pair.of("too many Oracle archive logs queried", + "increase the scn number of the query to reduce the number of archive logs queried OR set [readPosition] to [current]"); + }else if(errorMsg.contains("Connection reset by peer")){ + pair = Pair.of("This simply means that something in the backend ( DBMS ) decided to stop working due to unavailability of resources etc", + "This is not a problem with the FlinkX program, so there is nothing to do but restarting the logMiner task. For more detail, look at this: https://stackoverflow.com/questions/6110395/sqlrecoverableexception-i-o-exception-connection-reset"); + }else if(errorMsg.contains("ORA-00310")){ + pair = Pair.of("The archived log was out of sequence, probably because it was corrupt or the wrong redo log file name was specified during recovery", + "it may be useful to increase the number and size of redo log groups, and the restart the logMiner task"); + }else if(errorMsg.contains("ORA-01289")){ + pair = Pair.of("This simply means that the logfile specified has already been added to the list of logfiles and it may be we find same logfile ", + " you can restart the logMiner task"); + }else if(errorMsg.contains("ORA-00308") || errorMsg.contains("ORA-27037")){ + //todo https://blog.csdn.net/czmmiao/article/details/84173603 + pair = Pair.of("This simply means that if your Oracle version is 11.1, Oracle will scan or dump the redo records within 12 hours. If it is found that the required redo records have been deleted in the scan or dump, ora-00308 and ora-27037 errors will be reported. ", + "you can restart the logMiner task or The recommended solution for Oracle is to patch 8825048 and upgrade to 11.1.0.7.3 (patch set update) or 11.2.0.1 (base release)"); + } + } + return pair; + } +} diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java new file mode 100644 index 0000000000..42d781a7a4 --- /dev/null +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java @@ -0,0 +1,401 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oraclelogminer.util; + +import com.dtstack.flinkx.constants.ConstantValue; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * @author jiangbo + * @date 2019/12/14 + */ +public class SqlUtil { + + /** + * OPTIONS参数说明: + * DBMS_LOGMNR.SKIP_CORRUPTION - 跳过出错的redlog + * DBMS_LOGMNR.NO_SQL_DELIMITER - 不使用 ';'分割redo sql + * DBMS_LOGMNR.NO_ROWID_IN_STMT - 默认情况下,用于UPDATE和DELETE操作的SQL_REDO和SQL_UNDO语句在where子句中包含“ ROWID =”。 + * 但是,这对于想要重新执行SQL语句的应用程序是不方便的。设置此选项后,“ ROWID”不会放置在重构语句的末尾 + * DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG - 使用在线字典 + * DBMS_LOGMNR.CONTINUOUS_MINE - 需要在生成重做日志的同一实例中使用日志 + * DBMS_LOGMNR.COMMITTED_DATA_ONLY - 指定此选项时,LogMiner将属于同一事务的所有DML操作分组在一起。事务按提交顺序返回。 + * DBMS_LOGMNR.STRING_LITERALS_IN_STMT - 默认情况下,格式化格式化的SQL语句时,SQL_REDO和SQL_UNDO语句会使用数据库会话的NLS设置 + * 例如NLS_DATE_FORMAT,NLS_NUMERIC_CHARACTERS等)。使用此选项,将使用ANSI / ISO字符串文字格式对重构的SQL语句进行格式化。 + */ + public final static String SQL_START_LOG_MINER_AUTO_ADD_LOG = "" + + "BEGIN SYS.DBMS_LOGMNR.START_LOGMNR(" + + " STARTSCN => ?," + + " OPTIONS => SYS.DBMS_LOGMNR.SKIP_CORRUPTION " + + " + SYS.DBMS_LOGMNR.NO_SQL_DELIMITER " + + " + SYS.DBMS_LOGMNR.NO_ROWID_IN_STMT " + + " + SYS.DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG " + + " + SYS.DBMS_LOGMNR.CONTINUOUS_MINE " + + " + SYS.DBMS_LOGMNR.COMMITTED_DATA_ONLY " + + " + SYS.DBMS_LOGMNR.STRING_LITERALS_IN_STMT" + + ");" + + "END;"; + + public final static String SQL_START_LOG_MINER_AUTO_ADD_LOG_10 = "" + + "BEGIN SYS.DBMS_LOGMNR.START_LOGMNR(" + + " STARTSCN => ?," + + " OPTIONS => SYS.DBMS_LOGMNR.SKIP_CORRUPTION " + + " + SYS.DBMS_LOGMNR.NO_SQL_DELIMITER " + + " + SYS.DBMS_LOGMNR.NO_ROWID_IN_STMT " + + " + SYS.DBMS_LOGMNR.DICT_FROM_ONLINE_CATALOG " + + " + SYS.DBMS_LOGMNR.CONTINUOUS_MINE " + + " + SYS.DBMS_LOGMNR.COMMITTED_DATA_ONLY " + + ");" + + "END;"; + + /** + * 启动logminer + * 视图说明: + * v$log:存储未归档的日志 + * v$archived_log:存储已归档的日志文件 + * v$logfile: + */ + public final static String SQL_START_LOG_MINER = "" + + "DECLARE\n" + + " st BOOLEAN := true;\n" + + " start_scn NUMBER := ?;\n" + + "BEGIN\n" + + " FOR l_log_rec IN (\n" + + " SELECT\n" + + " MIN(name) name,\n" + + " first_change#\n" + + " FROM\n" + + " (\n" + + " SELECT\n" + + " MIN(member) AS name,\n" + + " first_change#,\n" + + " 281474976710655 AS next_change#\n" + + " FROM\n" + + " v$log l\n" + + " INNER JOIN v$logfile f ON l.group# = f.group#\n" + + " WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE'\n" + + " GROUP BY\n" + + " first_change#\n" + + " UNION\n" + + " SELECT\n" + + " name,\n" + + " first_change#,\n" + + " next_change#\n" + + " FROM\n" + + " v$archived_log\n" + + " WHERE\n" + + " name IS NOT NULL\n" + + " )\n" + + " WHERE\n" + + " first_change# >= start_scn\n" + + " OR start_scn < next_change#\n" + + " GROUP BY\n" + + " first_change#\n" + + " ORDER BY\n" + + " first_change#\n" + + " ) LOOP IF st THEN\n" + + " SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name, SYS.DBMS_LOGMNR.new);\n" + + " st := false;\n" + + " ELSE\n" + + " SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name);\n" + + " END IF;\n" + + " END LOOP;\n" + + "\n" + + " SYS.DBMS_LOGMNR.start_logmnr(" + + " options => " + + " SYS.DBMS_LOGMNR.skip_corruption " + + " + SYS.DBMS_LOGMNR.no_sql_delimiter " + + " + SYS.DBMS_LOGMNR.no_rowid_in_stmt\n" + + " + SYS.DBMS_LOGMNR.dict_from_online_catalog " + + " + SYS.DBMS_LOGMNR.string_literals_in_stmt" + + " );\n" + + "END;"; + + public final static String SQL_START_LOG_MINER_10 = "" + + "DECLARE\n" + + " st BOOLEAN := true;\n" + + " start_scn NUMBER := ?;\n" + + "BEGIN\n" + + " FOR l_log_rec IN (\n" + + " SELECT\n" + + " MIN(name) name,\n" + + " first_change#\n" + + " FROM\n" + + " (\n" + + " SELECT\n" + + " MIN(member) AS name,\n" + + " first_change#,\n" + + " 281474976710655 AS next_change#\n" + + " FROM\n" + + " v$log l\n" + + " INNER JOIN v$logfile f ON l.group# = f.group#\n" + + " WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE'\n" + + " GROUP BY\n" + + " first_change#\n" + + " UNION\n" + + " SELECT\n" + + " name,\n" + + " first_change#,\n" + + " next_change#\n" + + " FROM\n" + + " v$archived_log\n" + + " WHERE\n" + + " name IS NOT NULL\n" + + " )\n" + + " WHERE\n" + + " first_change# >= start_scn\n" + + " OR start_scn < next_change#\n" + + " GROUP BY\n" + + " first_change#\n" + + " ORDER BY\n" + + " first_change#\n" + + " ) LOOP IF st THEN\n" + + " SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name, SYS.DBMS_LOGMNR.new);\n" + + " st := false;\n" + + " ELSE\n" + + " SYS.DBMS_LOGMNR.add_logfile(l_log_rec.name);\n" + + " END IF;\n" + + " END LOOP;\n" + + "\n" + + " SYS.DBMS_LOGMNR.start_logmnr(" + + " options => " + + " SYS.DBMS_LOGMNR.skip_corruption " + + " + SYS.DBMS_LOGMNR.no_sql_delimiter " + + " + SYS.DBMS_LOGMNR.no_rowid_in_stmt\n" + + " + SYS.DBMS_LOGMNR.dict_from_online_catalog " + + " );\n" + + "END;"; + + public final static String SQL_QUERY_LOG_FILE = + "SELECT\n" + + " MIN(name) name,\n" + + " first_change#,\n" + + " MIN(next_change#) next_change#\n" + + "FROM\n" + + " (\n" + + " SELECT\n" + + " MIN(member) AS name,\n" + + " first_change#,\n" + + " MIN(next_change#) next_change#\n" + + " FROM\n" + + " v$log l\n" + + " INNER JOIN v$logfile f ON l.group# = f.group#\n" + + " WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE'\n" + + " GROUP BY\n" + + " first_change#\n" + + " UNION\n" + + " SELECT\n" + + " name,\n" + + " first_change#,\n" + + " next_change#\n" + + " FROM\n" + + " v$archived_log\n" + + " WHERE\n" + + " name IS NOT NULL\n" + + " )\n" + + "WHERE\n" + + " first_change# >= ?\n" + + " OR ? < next_change#\n" + + "GROUP BY\n" + + " first_change#\n" + + "ORDER BY\n" + + " first_change#"; + + public final static String SQL_QUERY_LOG_FILE_10 = + "SELECT\n" + + " MIN(name) name,\n" + + " first_change#,\n" + + " MIN(next_change#) next_change#\n" + + "FROM\n" + + " (\n" + + " SELECT\n" + + " MIN(member) AS name,\n" + + " first_change#,\n" + + " 281474976710655 AS next_change#\n" + + " FROM\n" + + " v$log l\n" + + " INNER JOIN v$logfile f ON l.group# = f.group#\n" + + " WHERE l.STATUS = 'CURRENT' OR l.STATUS = 'ACTIVE'\n" + + " GROUP BY\n" + + " first_change#\n" + + " UNION\n" + + " SELECT\n" + + " name,\n" + + " first_change#,\n" + + " next_change#\n" + + " FROM\n" + + " v$archived_log\n" + + " WHERE\n" + + " name IS NOT NULL\n" + + " )\n" + + "WHERE\n" + + " first_change# >= ?\n" + + " OR ? < next_change#\n" + + "GROUP BY\n" + + " first_change#\n" + + "ORDER BY\n" + + " first_change#"; + + public final static String SQL_SELECT_DATA = "" + + "SELECT\n" + + " scn,\n" + + //oracle 10 没有该字段 +// " commit_scn,\n" + + " timestamp,\n" + + " operation,\n" + + " seg_owner,\n" + + " table_name,\n" + + " sql_redo,\n" + + " row_id,\n" + + " csf\n" + + "FROM\n" + + " v$logmnr_contents\n" + + "WHERE\n" + + " scn > ?"; + + public final static String SQL_STOP_LOG_MINER = "BEGIN SYS.DBMS_LOGMNR.END_LOGMNR; end;"; + + public final static String SQL_GET_CURRENT_SCN = "select min(CURRENT_SCN) CURRENT_SCN from gv$database"; + + public final static String SQL_GET_LOG_FILE_START_POSITION = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log union select FIRST_CHANGE# from v$archived_log where standby_dest='NO' and name is not null)"; + + public final static String SQL_GET_LOG_FILE_START_POSITION_BY_SCN = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where ? between FIRST_CHANGE# and NEXT_CHANGE# union select FIRST_CHANGE# from v$archived_log where ? between FIRST_CHANGE# and NEXT_CHANGE# and standby_dest='NO' and name is not null)"; + + public final static String SQL_GET_LOG_FILE_START_POSITION_BY_SCN_10 = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where ? > FIRST_CHANGE# union select FIRST_CHANGE# from v$archived_log where ? between FIRST_CHANGE# and NEXT_CHANGE# and standby_dest='NO' and name is not null)"; + + public final static String SQL_GET_LOG_FILE_START_POSITION_BY_TIME = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') between FIRST_TIME and NVL(NEXT_TIME, TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS')) union select FIRST_CHANGE# from v$archived_log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') between FIRST_TIME and NEXT_TIME and standby_dest='NO' and name is not null)"; + + public final static String SQL_GET_LOG_FILE_START_POSITION_BY_TIME_10 = "select min(FIRST_CHANGE#) FIRST_CHANGE# from (select FIRST_CHANGE# from v$log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') > FIRST_TIME union select FIRST_CHANGE# from v$archived_log where TO_DATE(?, 'YYYY-MM-DD HH24:MI:SS') between FIRST_TIME and NEXT_TIME and standby_dest='NO' and name is not null)"; + + //修改当前会话的date日期格式 + public final static String SQL_ALTER_DATE_FORMAT ="ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD HH24:MI:SS'"; + + //修改当前会话的timestamp日期格式 + public final static String NLS_TIMESTAMP_FORMAT ="ALTER SESSION SET NLS_TIMESTAMP_FORMAT = 'YYYY-MM-DD HH24:MI:SS.FF6'"; + + public final static String SQL_QUERY_ROLES = "SELECT * FROM USER_ROLE_PRIVS"; + + public final static String SQL_QUERY_PRIVILEGES = "SELECT * FROM SESSION_PRIVS"; + + public final static String SQL_QUERY_ENCODING = "SELECT USERENV('LANGUAGE') FROM DUAL"; + + public final static String SQL_QUERY_LOG_MODE = "SELECT LOG_MODE FROM V$DATABASE"; + + public final static String SQL_QUERY_SUPPLEMENTAL_LOG_DATA_ALL = "SELECT SUPPLEMENTAL_LOG_DATA_ALL FROM V$DATABASE"; + public static final List PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "LOGMINING", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + public static final List ORACLE_11_PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + private final static List SUPPORTED_OPERATIONS = Arrays.asList("UPDATE", "INSERT", "DELETE"); + public static List EXCLUDE_SCHEMAS = Collections.singletonList("SYS"); + + /** + * 构建查询v$logmnr_contents视图SQL + * @param listenerOptions 需要采集操作类型字符串 delete,insert,update + * @param listenerTables 需要采集的schema+表名 SCHEMA1.TABLE1,SCHEMA2.TABLE2 + * @return + */ + public static String buildSelectSql(String listenerOptions, String listenerTables){ + StringBuilder sqlBuilder = new StringBuilder(SQL_SELECT_DATA); + + if (StringUtils.isNotEmpty(listenerOptions)) { + sqlBuilder.append(" and ").append(buildOperationFilter(listenerOptions)); + } + + if (StringUtils.isNotEmpty(listenerTables)) { + sqlBuilder.append(" and ").append(buildSchemaTableFilter(listenerTables)); + } else { + sqlBuilder.append(" and ").append(buildExcludeSchemaFilter()); + } + + return sqlBuilder.toString(); +} + + /** + * 构建需要采集操作类型字符串的过滤条件 + * @param listenerOptions 需要采集操作类型字符串 delete,insert,update + * @return + */ + private static String buildOperationFilter(String listenerOptions){ + List standardOperations = new ArrayList<>(); + + String[] operations = listenerOptions.split(ConstantValue.COMMA_SYMBOL); + for (String operation : operations) { + if (!SUPPORTED_OPERATIONS.contains(operation.toUpperCase())) { + throw new RuntimeException("Unsupported operation type:" + operation); + } + + standardOperations.add(String.format("'%s'", operation.toUpperCase())); + } + + return String.format("OPERATION in (%s) ", StringUtils.join(standardOperations, ConstantValue.COMMA_SYMBOL)); + } + + /** + * 过滤系统表 + * @return + */ + private static String buildExcludeSchemaFilter(){ + List filters = new ArrayList<>(); + for (String excludeSchema : EXCLUDE_SCHEMAS) { + filters.add(String.format("SEG_OWNER != '%s'", excludeSchema)); + } + + return String.format("(%s)", StringUtils.join(filters, " and ")); + } + + /** + * 构建需要采集的schema+表名的过滤条件 + * @param listenerTables 需要采集的schema+表名 SCHEMA1.TABLE1,SCHEMA2.TABLE2 + * @return + */ + private static String buildSchemaTableFilter(String listenerTables){ + List filters = new ArrayList<>(); + + String[] tableWithSchemas = listenerTables.split(ConstantValue.COMMA_SYMBOL); + for (String tableWithSchema : tableWithSchemas){ + List tables = Arrays.asList(tableWithSchema.split("\\.")); + if (ConstantValue.STAR_SYMBOL.equals(tables.get(0))) { + throw new IllegalArgumentException("Must specify the schema to be collected:" + tableWithSchema); + } + + StringBuilder tableFilterBuilder = new StringBuilder(256); + tableFilterBuilder.append(String.format("SEG_OWNER='%s'", tables.get(0))); + + if(!ConstantValue.STAR_SYMBOL.equals(tables.get(1))){ + tableFilterBuilder.append(" and ").append(String.format("TABLE_NAME='%s'", tables.get(1))); + } + + filters.add(String.format("(%s)", tableFilterBuilder.toString())); + } + + return String.format("(%s)", StringUtils.join(filters, " or ")); + } + + /** + * 是否为临时表,临时表没有redo sql,sql_redo内容为No SQL_UNDO for temporary tables + * @param sql redo sql + * @return + */ + public static boolean isCreateTemporaryTableSql(String sql) { + return sql.contains("temporary tables"); + } +} \ No newline at end of file diff --git a/flinkx-oraclelogminer/pom.xml b/flinkx-oraclelogminer/pom.xml new file mode 100644 index 0000000000..ef17277ffb --- /dev/null +++ b/flinkx-oraclelogminer/pom.xml @@ -0,0 +1,36 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oraclelogminer + pom + + flinkx-oraclelogminer-reader + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + + + \ No newline at end of file diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index a544ea0ca3..43a3d1c744 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -394,6 +394,12 @@ 1.6 + + com.dtstack.flinkx + flinkx-oraclelogminer-reader + 1.6 + + org.apache.hadoop hadoop-hdfs diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index c6d83f82db..4e2a2eb175 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -66,6 +66,7 @@ import com.dtstack.flinkx.odps.writer.OdpsWriter; import com.dtstack.flinkx.oracle.reader.OracleReader; import com.dtstack.flinkx.oracle.writer.OracleWriter; +import com.dtstack.flinkx.oraclelogminer.reader.OraclelogminerReader; import com.dtstack.flinkx.phoenix5.reader.Phoenix5Reader; import com.dtstack.flinkx.phoenix5.writer.Phoenix5Writer; import com.dtstack.flinkx.polardb.reader.PolardbReader; @@ -223,6 +224,7 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstants.GREENPLUM_READER : reader = new GreenplumReader(config, env); break; case PluginNameConstants.PHOENIX5_READER : reader = new Phoenix5Reader(config, env); break; case PluginNameConstants.KINGBASE_READER : reader = new KingbaseReader(config, env); break; + case PluginNameConstants.ORACLE_LOG_MINER_READER : reader = new OraclelogminerReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } diff --git a/jars/readme.md b/jars/readme.md index e6a930809e..312bd13725 100644 --- a/jars/readme.md +++ b/jars/readme.md @@ -1,6 +1,6 @@ -# 打包找不到db2和oracle相关驱动包临时解决办法 +# 打包找不到驱动包解决办法 -下载这连个驱动包,上传到本地仓库: +下载对应驱动包,上传到本地仓库: db2:[下载](db2jcc-3.72.44.jar) @@ -10,16 +10,28 @@ gbase:[下载](gbase-8.3.81.53.jar) 达梦:[下载](Dm7JdbcDriver18.jar) +人大金仓:[下载](kingbase8-8.2.0.jar) + +vertica:[下载](vertica-jdbc-9.1.1-0.jar) + 然后上传到本地仓库: ``` -mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=db2jcc-3.72.44.jar +## db2 driver +mvn install:install-file -DgroupId=com.ibm.db2 -DartifactId=db2jcc -Dversion=3.72.44 -Dpackaging=jar -Dfile=../jars/db2jcc-3.72.44.jar -mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=ojdbc8-12.2.0.1.jar +## oracle driver +mvn install:install-file -DgroupId=com.github.noraui -DartifactId=ojdbc8 -Dversion=12.2.0.1 -Dpackaging=jar -Dfile=../jars/ojdbc8-12.2.0.1.jar -mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=gbase-8.3.81.53.jar +## gbase driver +mvn install:install-file -DgroupId=com.esen.jdbc -DartifactId=gbase -Dversion=8.3.81.53 -Dpackaging=jar -Dfile=../jars/gbase-8.3.81.53.jar -mvn install:install-file -DgroupId=com.dm -DartifactId=Dm7JdbcDriver18 -Dversion=7.6.0.197 -Dpackaging=jar -Dfile=Dm7JdbcDriver18.jar -``` +## dm driver +mvn install:install-file -DgroupId=dm.jdbc.driver -DartifactId=dm7 -Dversion=18.0.0 -Dpackaging=jar -Dfile=../jars/Dm7JdbcDriver18.jar -说明:这几个驱动包在我们自己搭建的仓库里有,并且这几个版本的驱动包在已经在生产环境中使用,所以不能很快修改版本,需要做相关测试,我们会在后期的版本中修改这两个驱动包的版本,可以先暂时下载安装驱动来解决。 +## kingbase driver +mvn install:install-file -DgroupId=com.kingbase8 -DartifactId=kingbase8 -Dversion=8.2.0 -Dpackaging=jar -Dfile=../jars/kingbase8-8.2.0.jar + +## vertica driver +mvn install:install-file -DgroupId=fakepath -DartifactId=vertica-jdbc -Dversion=9.1.1-0 -Dpackaging=jar -Dfile=../jars/vertica-jdbc-9.1.1-0.jar +``` diff --git a/pom.xml b/pom.xml index a7d057c196..a0d5e8442a 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ flinkx-pulsar flinkx-pgwal flinkx-restapi + flinkx-oraclelogminer From b4814e67d986ced8b327e498ebfbbe42e7e6f1ec Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 5 Feb 2021 18:19:44 +0800 Subject: [PATCH 099/136] =?UTF-8?q?=E5=A2=9E=E5=8A=A0setting=E6=96=87?= =?UTF-8?q?=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- jars/settings.xml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 jars/settings.xml diff --git a/jars/settings.xml b/jars/settings.xml new file mode 100644 index 0000000000..85c4fa82f5 --- /dev/null +++ b/jars/settings.xml @@ -0,0 +1,18 @@ + + + + + /home/apache-maven-3.6.1/repository + + + + alimaven + aliyun maven + http://maven.aliyun.com/nexus/content/groups/public/ + central + + + + \ No newline at end of file From 8f221221776220ee63bac8ca5877717452093ae6 Mon Sep 17 00:00:00 2001 From: "jiguo.djg" Date: Sun, 21 Feb 2021 11:52:48 +0800 Subject: [PATCH 100/136] readme error --- docs/restore.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/restore.md b/docs/restore.md index fc2c607197..82df956b5f 100644 --- a/docs/restore.md +++ b/docs/restore.md @@ -111,7 +111,7 @@ checkpoint触发后,两个reader先生成Snapshot记录读取状态,通道0 > > Writer_1:id=无法确定 -任务状态会记录到配置的HDFS目录/flinkx/checkpoint/abc123下。因为每个Writer会接收两个Reader的数据,以及各个通道的数据读写速率可能不一样,所以导致writer接收到的数据顺序是不确定的,但是这不影响数据的准确性,因为读取数据时只需要Reader记录的状态就可以构造查询sql,我们只要确保这些数据真的写到HDF就行了。在Writer生成Snapshot之前,会做一系列操作保证接收到的数据全部写入HDFS: +任务状态会记录到配置的HDFS目录/flinkx/checkpoint/abc123下。因为每个Writer会接收两个Reader的数据,以及各个通道的数据读写速率可能不一样,所以导致writer接收到的数据顺序是不确定的,但是这不影响数据的准确性,因为读取数据时只需要Reader记录的状态就可以构造查询sql,我们只要确保这些数据真的写到HDFS就行了。在Writer生成Snapshot之前,会做一系列操作保证接收到的数据全部写入HDFS: - close写入HDFS文件的数据流,这时候会在/data_test/.data目录下生成两个两个文件: From b133a3444f7823ef7627597e8b59674e76813dae Mon Sep 17 00:00:00 2001 From: "jiguo.djg" Date: Sun, 21 Feb 2021 13:50:59 +0800 Subject: [PATCH 101/136] =?UTF-8?q?readme=E4=B8=AD=E5=85=B3=E4=BA=8Estream?= =?UTF-8?q?writer=E7=9A=84=E6=96=87=E6=A1=A3=E9=93=BE=E6=8E=A5=E9=94=99?= =?UTF-8?q?=E8=AF=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 2 +- README_CH.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index cd8aa4cea1..cc266f8808 100644 --- a/README.md +++ b/README.md @@ -62,7 +62,7 @@ The following databases are currently supported: | | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | | | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | | | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | -| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | diff --git a/README_CH.md b/README_CH.md index c181ce45c0..5b36a2f513 100644 --- a/README_CH.md +++ b/README_CH.md @@ -73,7 +73,7 @@ FlinkX目前支持下面这些数据库: | | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | | | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | | | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | -| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | From 1d0dc423ed41069c272fc9056e629d920ab6bfb4 Mon Sep 17 00:00:00 2001 From: Ting Sun <11710108@mail.sustech.edu.cn> Date: Mon, 22 Feb 2021 20:42:21 +0800 Subject: [PATCH 102/136] =?UTF-8?q?Update=20readme.md=EF=BC=9ACorrct=20the?= =?UTF-8?q?=20syntax=20format?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Just a trivial patch for syntax. --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 313ecf207e..169573579b 100644 --- a/README.md +++ b/README.md @@ -7,9 +7,9 @@ English | [中文](README_CH.md) # Communication -- We are recruiting **Big data platform development engineers**.If you want more information about the position, please add WeChat ID [**ysqwhiletrue**] or email your resume to [sishu@dtstack.com](mailto:sishu@dtstack.com). +- We are recruiting **Big data platform development engineers**. If you want more information about the position, please add WeChat ID [**ysqwhiletrue**] or email your resume to [sishu@dtstack.com](mailto:sishu@dtstack.com). -- We use [DingTalk](https://www.dingtalk.com/) to communicate,You can search the group number [**30537511**] or scan the QR code below to join the communication group +- We use [DingTalk](https://www.dingtalk.com/) to communicate, you can search the group number [**30537511**] or scan the QR code below to join the communication group
From 25866642dbb89ba9367589f4d3e63c82518b1810 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 2 Mar 2021 14:11:00 +0800 Subject: [PATCH 103/136] add sqlserverCDC Modules and docs --- README.md | 3 +- README_CH.md | 3 +- docs/offline/reader/sqlserverreader.md | 1 - docs/realTime/reader/kafkareader.md | 15 +- docs/realTime/reader/restapireader.md | 296 +++++++++++++++--- .../com/dtstack/flinkx/binlog/BinlogUtil.java | 4 +- .../format/BinlogInputFormatBuilder.java | 19 +- .../flinkx/cassandra/CassandraUtil.java | 3 +- .../reader/CassandraInputFormat.java | 4 +- .../main/java/com/dtstack/flinkx/Main.java | 1 - .../outputformat/BaseRichOutputFormat.java | 2 +- .../source/DtInputFormatSourceFunction.java | 21 +- .../com/dtstack/flinkx/util/DateUtil.java | 14 +- .../com/dtstack/flinkx/util/GsonUtil.java | 9 +- .../flinkx/emqx/format/EmqxOutputFormat.java | 5 +- .../com/dtstack/flinkx/ftp/FtpHandler.java | 2 + .../flinkx/ftp/reader/FtpInputFormat.java | 4 +- .../ftp/reader/FtpSeqBufferedReader.java | 22 +- .../flinkx/hbase/reader/HbaseInputFormat.java | 19 +- .../hdfs/reader/BaseHdfsInputFormat.java | 3 + .../hdfs/reader/HdfsInputFormatBuilder.java | 15 +- .../hdfs/writer/BaseHdfsOutputFormat.java | 1 - .../hdfs/writer/HdfsOrcOutputFormat.java | 2 +- .../hdfs/writer/HdfsOutputFormatBuilder.java | 16 +- .../hdfs/writer/HdfsParquetOutputFormat.java | 2 +- .../hdfs/writer/HdfsTextOutputFormat.java | 2 +- .../hive/writer/HiveOutputFormatBuilder.java | 2 - .../flinkx/hive/writer/HiveWriter.java | 57 ++-- .../kafka/format/KafkaOutputFormat.java | 13 +- .../flinkx/kafka/writer/KafkaWriter.java | 2 + .../kafka09/format/Kafka09OutputFormat.java | 11 +- .../flinkx/kafkabase/KafkaConfigKeys.java | 7 +- .../flinkx/kafkabase/util/KafkaUtil.java | 2 - .../format/KafkaBaseOutputFormat.java | 51 +++ .../kafkabase/writer/KafkaBaseWriter.java | 8 + .../kingbase/format/KingbaseOutputFormat.java | 4 +- .../kingbase/writer/KingbaseWriter.java | 1 + .../flinkx/launcher/ClusterClientFactory.java | 54 ++-- .../com/dtstack/flinkx/launcher/Launcher.java | 7 +- .../mongodb/reader/MongodbInputFormat.java | 4 +- .../mongodb/reader/MongodbInputSplit.java | 9 + .../format/LogMinerConnection.java | 63 ++-- .../flinkx/oraclelogminer/util/SqlUtil.java | 8 +- .../flinkx/pgwal/format/PgWalInputFormat.java | 2 +- .../pgwal/format/PgWalInputFormatBuilder.java | 2 +- .../flinkx/phoenix/reader/PhoenixReader.java | 1 - .../flinkx/phoenix/writer/PhoenixWriter.java | 1 - .../phoenix5/format/Phoenix5InputFormat.java | 5 +- .../phoenix5/writer/Phoenix5Writer.java | 1 - .../postgresql/reader/PostgresqlReader.java | 7 + .../postgresql/writer/PostgresqlWriter.java | 7 + .../com/dtstack/flinkx/rdb/util/DbUtil.java | 12 +- .../JdbcInputFormat.java | 4 +- .../rdb/outputformat/JdbcOutputFormat.java | 5 +- .../flinkx/restapi/common/ConstantParam.java | 112 ------- .../restapi/common/ConstantVarible.java | 22 -- .../restapi/common/CurrentTimeVarible.java | 43 --- .../flinkx/restapi/common/DymaticParam.java | 260 --------------- .../flinkx/restapi/common/HttpUtil.java | 83 ++++- .../restapi/common/InnerVaribleFactory.java | 53 ---- .../restapi/common/IntervalTimeVarible.java | 43 --- .../flinkx/restapi/common/MapUtils.java | 37 --- .../restapi/common/ParamDefinition.java | 56 ---- .../common/ParamDefinitionNextAble.java | 31 -- .../flinkx/restapi/common/ParamFactory.java | 105 ------- .../flinkx/restapi/common/ParamType.java | 9 +- .../flinkx/restapi/common/Paramitem.java | 31 -- .../restapi/common/ReplaceParamItem.java | 48 --- .../flinkx/restapi/common/RestContext.java | 188 ----------- .../flinkx/restapi/common/UuidVarible.java | 21 -- .../common/exception/ReadRecordException.java | 10 - .../exception/ResponseBreakException.java | 22 -- .../exception/ResponseRetryException.java | 22 -- .../common/handler/BreakDataHandler.java | 26 -- .../restapi/common/handler/DataHandler.java | 23 -- .../common/handler/DataHandlerFactory.java | 29 -- .../common/handler/RetryDataHandler.java | 24 -- .../flinkx/restapi/common/httprequestApi.java | 76 ----- .../restapi/inputformat/HttpClient.java | 194 ------------ .../inputformat/RestapiInputFormat.java | 118 ------- .../RestapiInputFormatBuilder.java | 66 ---- .../flinkx/restapi/reader/HttpRestConfig.java | 163 +++++++--- .../flinkx/restapi/reader/RestapiReader.java | 65 ++-- flinkx-test/pom.xml | 12 +- .../com/dtstack/flinkx/test/LocalTest.java | 5 + .../flinkx/test/PluginNameConstants.java | 3 + pom.xml | 3 +- 87 files changed, 894 insertions(+), 1947 deletions(-) delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java delete mode 100644 flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java delete mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java delete mode 100644 flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java diff --git a/README.md b/README.md index 313ecf207e..6007a5bcb3 100644 --- a/README.md +++ b/README.md @@ -70,7 +70,7 @@ The following databases are currently supported: | | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | | | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | | | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | -| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | @@ -80,6 +80,7 @@ The following databases are currently supported: | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | | | Oracle LogMiner| [doc](docs/realTime/reader/LogMiner.md) | | +| | Sqlserver CDC| [doc](docs/realTime/reader/sqlservercdc.md) | | # Fundamental In the underlying implementation, FlinkX relies on Flink, and the data synchronization task will be translated into StreamGraph and executed on Flink. The basic principle is as follows: diff --git a/README_CH.md b/README_CH.md index eb2b4ce4e3..d5cf5b1813 100644 --- a/README_CH.md +++ b/README_CH.md @@ -80,7 +80,7 @@ FlinkX目前支持下面这些数据库: | | FTP | [doc](docs/offline/reader/ftpreader.md) | [doc](docs/offline/writer/ftpwriter.md) | | | HDFS | [doc](docs/offline/reader/hdfsreader.md) | [doc](docs/offline/writer/hdfswriter.md) | | | Carbondata | [doc](docs/offline/reader/carbondatareader.md) | [doc](docs/offline/writer/carbondatawriter.md) | -| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/carbondatawriter.md) | +| | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | @@ -90,6 +90,7 @@ FlinkX目前支持下面这些数据库: | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | | | Oracle LogMiner | [doc](docs/realTime/reader/LogMiner.md) | | +| | Sqlserver CDC | [doc](docs/realTime/reader/sqlservercdc.md) | | # 基本原理 在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,基本原理如下图: diff --git a/docs/offline/reader/sqlserverreader.md b/docs/offline/reader/sqlserverreader.md index f10f7aabb9..9b465721da 100644 --- a/docs/offline/reader/sqlserverreader.md +++ b/docs/offline/reader/sqlserverreader.md @@ -398,7 +398,6 @@ } } } -} ``` #### 4、增量同步指定startLocation diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index f754c2846d..045ef5fb08 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -155,21 +155,24 @@ kafka插件存在四个版本,根据kafka版本的不同,插件名称也略
+ + ## 三、配置示例 ### 1、kafka09 ```json { - "job" : { - "content" : [ { + "job": { + "content": [{ "reader" : { "parameter" : { "topic" : "kafka09", - "groupId" : "default", - "codec" : "text", + "codec": "plain", "encoding": "UTF-8", - "blankIgnore": false, "consumerSettings" : { - "zookeeper.connect" : "localhost:2181/kafka09" + "zookeeper.connect" : "0.0.0.1:2182/kafka09", + "group.id" : "default", + "auto.commit.interval.ms" : "1000", + "auto.offset.reset" : "smallest" } }, "name" : "kafka09reader" diff --git a/docs/realTime/reader/restapireader.md b/docs/realTime/reader/restapireader.md index f13bcd1da3..8635053f26 100644 --- a/docs/realTime/reader/restapireader.md +++ b/docs/realTime/reader/restapireader.md @@ -1,37 +1,231 @@ # Restapi Reader + + +- [Restapi Reader](#restapi-reader) + - [一、插件名称](#一插件名称) + - [二、参数说明](#二参数说明) + - [三、配置示例](#三配置示例) + + - ## 一、插件名称 -名称:restapireader
+名称:restapireader + - ## 二、参数说明 -- **url** - - 描述:连接的url - - 必选:是 - - 默认值:无 - - 字段类型:String -
-- **method** - - 描述:request的类型,`post`、`get` - - 必选:是 - - 默认值:无 - - 字段类型:String -
+- url + - 描述:http请求地址 + - 必选:是 + - 字段类型:string + +
+ +- requestMode + - 描述:http请求方式 + - 必选:是 + - 字段类型:string + - 可选值:post get + +
+ +- header + - 描述: 请求的header + - 注意: 当请求方式为post时,Content-Type需为application/json,目前只支持post请求的json格式,不支持表单提交 + - 必选: 否,如果requestMode配置未post,header会自动添加 'application/json' head头 + - 字段类型:数组 +```json + "header": [ + { + "name": "token", + "value": "${uuid}" + } + ], +``` + +- 参数解析 + - name 请求的key 必选 + - value key的值 必选 + +
+ +- body + - 描述:对应post请求的body参数 + - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算), + - 内置变量 + - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 + - ${intervalTime}间隔时间,代表参数 intervalTime 的值 + - ${uuid} 随机字符串 32位的随机字符串 + - param/body/response变量 + - ${param.key} 对应get请求param参数里key对应的值 + - ${body.key}对应post请求的body参数里key对应的值 + - ${response.key} 对应返回值里的key对应的值 + - 必选:否 + - 字段类型:数组 +```json +"body": [ + { + "name": "stime", + "value": "${currentTime}", + "nextValue": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + }, + { + "name": "etime", + "value": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + } + ], +``` + +- 参数解析 + - name 请求的key 必选 + - value key的值 必选 + - nextValue 除第一次请求之外,key对应的值 非必选 + - format 格式化模板 非必选,如果要求请求格式是日期格式,必须填写 + +
+ +- param + - 描述:对应get请求参数 + - 注意:参数支持动态参数替换,内置变量以及动态变量的加减(只支持动态变量的一次加减运算) + - 内置变量 + - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 + - ${intervalTime}间隔时间,代表参数 intervalTime 的值 + - ${uuid} 随机字符串 32位的随机字符串 + - param/body/response变量 + - ${param.key} 对应get请求param参数里key对应的值 + - ${body.key}对应post请求的body参数里key对应的值 + - ${response.key} 对应返回值里的key对应的值 + - 必选:否 + - 字段类型:数组 +```json +"param": [ + { + "name": "stime", + "value": "${currentTime}", + "nextValue": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + }, + { + "name": "etime", + "value": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + } + ], +``` + +- 参数解析 + - name 请求的key 必选 + - value key的值 必选 + - nextValue 除第一次请求之外,key对应的值 非必选 + - format 格式化模板 非必选,如果要求请求格式是日期格式,必须填写 + +
+ +- decode + - 描述 解码器 返回数据是作为json格式还是text格式处理 + - 必选:否 + - 字段类型:string +```json +"deocode":"json" +``` + +- 默认值:text +- 可选值:text json + - text 不做任何处理,返回值直接丢出去 + - json 可以进行定制化输出,指定输出的key,则对返回值解析,获取对应的key以及值 组装新的json数据丢出去 + +
+ +- fields + - 描述:在decode为json时,可以对返回值指定key输出 + - 注意:decode为text模式时,不支持此参数, key以 . 作为层级,多个key用逗号隔开 + - 必选: 否 + - 字段类型:string + - 示例 +fields值为 +``` +"fields": "msg.key1,msg.key2.key3", +``` +返回值为: +```json +{ + "msg": { + "key1": "value1", + "key2": { + "key3": "value2", + "key4": "value3", + }, + "key5": 2 + } +} +``` +根据fields解析后的值为: +```json +{ + "msg": { + "key1": "value1", + "key2": { + "key3": "value2" + } + } +} +``` +
+ +- strategy + - 描述 定义的key的实际值与value指定值相等时进行对应的逻辑处理 + - 必选 否 + - 字段类型:数组 + - 描述:针对返回类型为json的数据,用户会指定key以及对应的value和处理方式。如果返回数据的对应的key的值正好和用户配置的value相等,则执行对应逻辑。同时用户指定的key可以来自返回值也可以来自param参数值 + +```json + "strategy": [ + { + "key": "${param.pageNumber}", + "value": "${response.totalPageNum}", + "handle": "stop" + }] + +``` + +- 参数解析 + - key 选择对应参数的key,支持的格式为 + - 变量 + - ${param.key} + - ${body.key} + - ${response.key} + - 内置变量 + - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 + - ${intervalTime}间隔时间,代表参数 intervalTime 的值 + - ${uuid} 随机字符串 32位的随机字符串 + - value 匹配的值,支持的格式为 + - 常量 + - 变量: + - ${param.key} + - ${body.key} + - ${response.key} + - 内置变量 + - ${currentTime}当前时间,获取当前时间,格式为yyyy-MM-dd HH:mm:ss类型 + - ${intervalTime}间隔时间,代表参数 intervalTime 的值 + - ${uuid} 随机字符串 32位的随机字符串 + - handle 对应处理逻辑 + - stop 停止任务 + - retry 重试,如果重试三次都失败 任务结束 + +
+ +- intervalTime + - 描述: 每次请求间隔时间,单位毫秒 + - 必选:是 + - 字段类型:long -- **header** - - 描述:需要添加的报头信息 - - 必选:否 - - 默认值:无 - - 字段类型:Map -
- ## 三、配置示例 ```json { @@ -40,11 +234,49 @@ { "reader": { "parameter": { - "url": "http://kudu3/server/index.php?g=Web&c=Mock&o=mock&projectID=58&uri=/api/tiezhu/test/get", - "body": "", - "method": "get", - "params": "" - } + "protocol": "http", + "url": "http://wwww.a.com", + "requestMode": "post", + "decode": "json", + "intervalTime": 3000, + "fields": "msg.key1,msg.key2.key3", + "header": [ + { + "name": "token", + "value": "0aeb8fd6-02f9-4c84-836a-301ede439976" + }, + { + "name": "Content-Type", + "value": "application/json" + } + ], + "body": [ + { + "name": "stime", + "value": "${currentTime}", + "nextValue": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + }, + { + "name": "etime", + "value": "${body.stime}+${intervalTime}", + "format": "yyyy-mm-dd hh:mm:ss" + } + ], + "strategy": [ + { + "key": "${response.status}", + "value": "3000", + "handle": "stop" + }, + { + "key": "${response.currentPage}", + "value": "${response.totalPage}", + "handle": "stop" + } + ] + }, + "name": "restapireader" }, "writer": { "parameter": { @@ -56,19 +288,11 @@ ], "setting": { "restore": { - "isRestore": false, + "isRestore": true, "isStream": true }, - "errorLimit": {}, "speed": { - "bytes": 0, "channel": 1 - }, - "log": { - "isLogger": false, - "level": "trace", - "path": "", - "pattern": "" } } } diff --git a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java index c587b22fa4..2061062679 100644 --- a/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java +++ b/flinkx-binlog/flinkx-binlog-core/src/main/java/com/dtstack/flinkx/binlog/BinlogUtil.java @@ -45,9 +45,9 @@ public class BinlogUtil { public static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; //是否开启binlog - private static final String CHECK_BINLOG_ENABLE = "show variables where variable_name = 'log_bin';;"; + private static final String CHECK_BINLOG_ENABLE = "show variables where variable_name = 'log_bin';"; //查看binlog format - private static final String CHECK_BINLOG_FORMAT = "show variables where variable_name = 'binlog_format';;"; + private static final String CHECK_BINLOG_FORMAT = "show variables where variable_name = 'binlog_format';"; //校验用户是否有权限 private static final String CHECK_USER_PRIVILEGE = "show master status ;"; diff --git a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java index 2e59ef17f9..ecbb310a70 100644 --- a/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java +++ b/flinkx-binlog/flinkx-binlog-reader/src/main/java/com/dtstack/flinkx/binlog/format/BinlogInputFormatBuilder.java @@ -26,7 +26,6 @@ import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.RetryUtil; import com.dtstack.flinkx.util.TelnetUtil; -import com.dtstack.flinkx.util.ValueUtil; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; @@ -35,9 +34,8 @@ import java.sql.Connection; import java.sql.DriverManager; -import java.util.Arrays; +import java.sql.SQLException; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Locale; @@ -157,15 +155,24 @@ protected void checkFormat() { String database = BinlogUtil.getDataBaseByUrl(binlogConfig.jdbcUrl); List failedTable = BinlogUtil.checkTablesPrivilege(conn, database, binlogConfig.filter, binlogConfig.table); if (CollectionUtils.isNotEmpty(failedTable)) { - sb.append("user has not select privilege on " + GsonUtil.GSON.toJson(failedTable)); + sb.append("user has not select privilege on ").append(GsonUtil.GSON.toJson(failedTable)); } if (sb.length() > 0) { throw new IllegalArgumentException(sb.toString()); } - } catch (Exception e) { - throw new RuntimeException("error to check binlog config, e = " + ExceptionUtil.getErrorMessage(e), e); + } catch (SQLException e) { + + StringBuilder detailsInfo = new StringBuilder(sb.length() + 128); + + if(sb.length() > 0){ + detailsInfo.append(" binlog config not right,details is ").append(sb.toString()); + } + + detailsInfo.append(" \n error to check binlog config, e = " ).append(ExceptionUtil.getErrorMessage(e)); + + throw new RuntimeException(detailsInfo.toString(), e); } } } diff --git a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java index bc0e631faa..ab972b492e 100644 --- a/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java +++ b/flinkx-cassandra/flinkx-cassandra-core/src/main/java/com/dtstack/flinkx/cassandra/CassandraUtil.java @@ -20,6 +20,7 @@ import com.datastax.driver.core.*; import com.datastax.driver.core.LocalDate; +import com.dtstack.flinkx.util.ExceptionUtil; import com.google.common.base.Preconditions; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang.StringUtils; @@ -210,7 +211,7 @@ private static Optional objectToBytes(T obj){ sOut.flush(); bytes= out.toByteArray(); } catch (IOException e) { - e.printStackTrace(); + LOG.warn("object convent byte[] failed, error info {}", ExceptionUtil.getErrorMessage(e), e); } return Optional.ofNullable(bytes); } diff --git a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java index 52f1311861..393477398d 100644 --- a/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java +++ b/flinkx-cassandra/flinkx-cassandra-reader/src/main/java/com/dtstack/flinkx/cassandra/reader/CassandraInputFormat.java @@ -144,7 +144,7 @@ private InputSplit[] splitJob(int minNumSplits, ArrayList s .divide(BigDecimal.valueOf(minNumSplits),2, BigDecimal.ROUND_HALF_EVEN); for ( int i = 0; i < minNumSplits; i++ ) { BigInteger l = minToken.add(step.multiply(BigDecimal.valueOf(i))).toBigInteger(); - BigInteger r = minToken.add(step.multiply(BigDecimal.valueOf(i+1))).toBigInteger(); + BigInteger r = minToken.add(step.multiply(BigDecimal.valueOf(i+1L))).toBigInteger(); if( i == minNumSplits - 1 ) { r = maxToken.toBigInteger(); } @@ -157,7 +157,7 @@ private InputSplit[] splitJob(int minNumSplits, ArrayList s .divide(BigDecimal.valueOf(minNumSplits),2, BigDecimal.ROUND_HALF_EVEN); for ( int i = 0; i < minNumSplits; i++ ) { long l = minToken.add(step.multiply(BigDecimal.valueOf(i))).longValue(); - long r = minToken.add(step.multiply(BigDecimal.valueOf(i+1))).longValue(); + long r = minToken.add(step.multiply(BigDecimal.valueOf(i+1L))).longValue(); if( i == minNumSplits - 1 ) { r = maxToken.longValue(); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java index bb67c3ddec..d629c94ae9 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/Main.java @@ -115,7 +115,6 @@ public static void main(String[] args) throws Exception { PluginUtil.registerPluginUrlToCachedFile(config, env); env.setParallelism(speedConfig.getChannel()); - env.setRestartStrategy(RestartStrategies.noRestart()); BaseDataReader dataReader = DataReaderFactory.getDataReader(config, env); DataStream dataStream = dataReader.readData(); if(speedConfig.getReaderChannel() > 0){ diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java index 239ec59e86..02c2a9de40 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/outputformat/BaseRichOutputFormat.java @@ -34,7 +34,7 @@ import com.dtstack.flinkx.writer.DirtyDataManager; import com.dtstack.flinkx.writer.ErrorLimiter; import com.google.gson.reflect.TypeToken; -import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang.StringUtils; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; import org.apache.flink.configuration.Configuration; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java b/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java index 537e4e1cd9..d24e01b787 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/streaming/api/functions/source/DtInputFormatSourceFunction.java @@ -148,8 +148,6 @@ public void run(SourceContext ctx) throws Exception { isRunning = splitIterator.hasNext(); } } - } catch (Exception exception){ - tryException = exception; } finally { isRunning = false; try { @@ -161,14 +159,11 @@ public void run(SourceContext ctx) throws Exception { if(null != tryException){ LOG.error(ExceptionUtil.getErrorMessage(finallyException)); tryException.addSuppressed(finallyException); - throw tryException; }else { - throw finallyException; + tryException = finallyException; } } - if(null != tryException) { - throw tryException; - } + throwException(tryException); } } @@ -276,4 +271,16 @@ public void initializeState(FunctionInitializationContext context) throws Except LOG.info("End initialize input format state"); } + + /** + * 抛出异常 + * @param e 需要抛出的异常 + * @throws Exception 异常 + */ + public void throwException(Exception e) throws Exception { + if(null != e) { + LOG.error("DtInputFormatSourceFunction error, info: {}",ExceptionUtil.getErrorMessage(e), e); + throw e; + } + } } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index 919a515ecd..660afb3f2f 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -42,6 +42,8 @@ public class DateUtil { private static final String STANDARD_DATETIME_FORMAT = "standardDatetimeFormatter"; + private static final String STANDARD_DATETIME_FORMAT_FOR_MILLISECOND= "standardDatetimeFormatterForMillisecond"; + private static final String UN_STANDARD_DATETIME_FORMAT = "unStandardDatetimeFormatter"; private static final String DATE_FORMAT = "dateFormatter"; @@ -67,7 +69,6 @@ public class DateUtil { TimeZone timeZone = TimeZone.getTimeZone(TIME_ZONE); Map formatterMap = new HashMap<>(); - SimpleDateFormat standardDatetimeFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); standardDatetimeFormatter.setTimeZone(timeZone); formatterMap.put(STANDARD_DATETIME_FORMAT,standardDatetimeFormatter); @@ -88,7 +89,11 @@ public class DateUtil { yearFormatter.setTimeZone(timeZone); formatterMap.put(YEAR_FORMAT,yearFormatter); - return formatterMap; + SimpleDateFormat standardDatetimeFormatterOfMillisecond = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + standardDatetimeFormatterOfMillisecond.setTimeZone(timeZone); + formatterMap.put(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND,standardDatetimeFormatterOfMillisecond); + + return formatterMap; }); private DateUtil() {} @@ -236,6 +241,11 @@ public static SimpleDateFormat getDateTimeFormatter(){ return datetimeFormatter.get().get(STANDARD_DATETIME_FORMAT); } + //获取毫秒级别的日期解析 + public static SimpleDateFormat getDateTimeFormatterForMillisencond(){ + return datetimeFormatter.get().get(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND); + } + public static SimpleDateFormat getDateFormatter(){ return datetimeFormatter.get().get(DATE_FORMAT); } diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java index 47a41a150e..69c2b6912e 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/GsonUtil.java @@ -54,10 +54,15 @@ private static Gson getGson() { .disableHtmlEscaping() .setPrettyPrinting() .create(); + + return setTypeAdapter(GSON); + } + + public static Gson setTypeAdapter(Gson gson) { try { Field factories = Gson.class.getDeclaredField("factories"); factories.setAccessible(true); - Object o = factories.get(GSON); + Object o = factories.get(gson); Class[] declaredClasses = Collections.class.getDeclaredClasses(); for (Class c : declaredClasses) { if ("java.util.Collections$UnmodifiableList".equals(c.getName())) { @@ -141,6 +146,6 @@ public void write(JsonWriter out, Object value) throws IOException { } catch (Exception e) { LOG.error(ExceptionUtil.getErrorMessage(e)); } - return GSON; + return gson; } } diff --git a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java index fc15e5c688..9cac743adb 100644 --- a/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java +++ b/flinkx-emqx/flinkx-emqx-writer/src/main/java/com/dtstack/flinkx/emqx/format/EmqxOutputFormat.java @@ -21,9 +21,9 @@ import com.dtstack.flinkx.exception.WriteRecordException; import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; +import com.dtstack.flinkx.util.MapUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.types.Row; -import com.fasterxml.jackson.databind.ObjectMapper; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttConnectOptions; import org.eclipse.paho.client.mqttv3.MqttException; @@ -54,7 +54,6 @@ public class EmqxOutputFormat extends BaseRichOutputFormat { private transient MqttClient client; protected static JsonDecoder jsonDecoder = new JsonDecoder(); - protected static ObjectMapper objectMapper = new ObjectMapper(); @Override @@ -96,7 +95,7 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { }else{ map = Collections.singletonMap("message", row.toString()); } - MqttMessage message = new MqttMessage(objectMapper.writeValueAsString(map).getBytes()); + MqttMessage message = new MqttMessage(MapUtil.writeValueAsString(map).getBytes()); message.setQos(qos); client.publish(topic, message); } catch (Throwable e) { diff --git a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java index aa8a3d083a..d51fd3adb4 100644 --- a/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java +++ b/flinkx-ftp/flinkx-ftp-core/src/main/java/com/dtstack/flinkx/ftp/FtpHandler.java @@ -69,6 +69,8 @@ public void loginFtpServer(FtpConfig ftpConfig) { // 不需要写死ftp server的OS TYPE,FTPClient getSystemType()方法会自动识别 ftpClient.setConnectTimeout(ftpConfig.getTimeout()); ftpClient.setDataTimeout(ftpConfig.getTimeout()); + //设置控制连接超时 + ftpClient.setSoTimeout(ftpConfig.getTimeout()); if (EFtpMode.PASV.name().equals(ftpConfig.getConnectPattern())) { ftpClient.enterRemotePassiveMode(); ftpClient.enterLocalPassiveMode(); diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java index f90c860ef1..92042c9a25 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpInputFormat.java @@ -94,10 +94,10 @@ public void openInternal(InputSplit split) throws IOException { List paths = inputSplit.getPaths(); if (ftpConfig.getIsFirstLineHeader()){ - br = new FtpSeqBufferedReader(ftpHandler,paths.iterator()); + br = new FtpSeqBufferedReader(ftpHandler,paths.iterator(),ftpConfig); br.setFromLine(1); } else { - br = new FtpSeqBufferedReader(ftpHandler,paths.iterator()); + br = new FtpSeqBufferedReader(ftpHandler,paths.iterator(),ftpConfig); br.setFromLine(0); } br.setFileEncoding(ftpConfig.getEncoding()); diff --git a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java index 49badaec33..277fe09225 100644 --- a/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java +++ b/flinkx-ftp/flinkx-ftp-reader/src/main/java/com/dtstack/flinkx/ftp/reader/FtpSeqBufferedReader.java @@ -18,6 +18,8 @@ package com.dtstack.flinkx.ftp.reader; +import com.dtstack.flinkx.ftp.FtpConfig; +import com.dtstack.flinkx.ftp.FtpHandlerFactory; import com.dtstack.flinkx.ftp.IFtpHandler; import com.dtstack.flinkx.ftp.FtpHandler; import org.slf4j.Logger; @@ -49,9 +51,13 @@ public class FtpSeqBufferedReader { private String fileEncoding; - public FtpSeqBufferedReader(IFtpHandler ftpHandler, Iterator iter) { + //ftp配置信息 + private FtpConfig ftpConfig; + + public FtpSeqBufferedReader(IFtpHandler ftpHandler, Iterator iter, FtpConfig ftpConfig) { this.ftpHandler = ftpHandler; this.iter = iter; + this.ftpConfig = ftpConfig; } public String readLine() throws IOException{ @@ -97,7 +103,19 @@ public void close() throws IOException { br = null; if (ftpHandler instanceof FtpHandler){ - ((FtpHandler) ftpHandler).getFtpClient().completePendingCommand(); + try { + ((FtpHandler) ftpHandler).getFtpClient().completePendingCommand(); + } catch (Exception e) { + //如果出现了超时异常,就直接获取一个新的ftpHandler + LOG.warn("FTPClient completePendingCommand has error ->",e); + try{ + ftpHandler.logoutFtpServer(); + }catch (Exception exception){ + LOG.warn("FTPClient logout has error ->",exception); + } + ftpHandler = FtpHandlerFactory.createFtpHandler(ftpConfig.getProtocol()); + ftpHandler.loginFtpServer(ftpConfig); + } } } } diff --git a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java index 5e9c274864..38f349b58e 100644 --- a/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java +++ b/flinkx-hbase/flinkx-hbase-reader/src/main/java/com/dtstack/flinkx/hbase/reader/HbaseInputFormat.java @@ -44,6 +44,7 @@ import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.List; +import java.util.Locale; import java.util.Map; import com.google.common.collect.Maps; import org.apache.hadoop.security.UserGroupInformation; @@ -331,28 +332,28 @@ public Object convertBytesToAssignType(String columnType, byte[] byteArray,Strin if(ArrayUtils.isEmpty(byteArray)) { return null; } - - switch (columnType.toUpperCase()) { + String bytesToString = new String(byteArray, encoding); + switch (columnType.toUpperCase(Locale.ENGLISH)) { case "BOOLEAN": - column = Bytes.toBoolean(byteArray); + column = Boolean.valueOf(bytesToString); break; case "SHORT": - column = String.valueOf(Bytes.toShort(byteArray)); + column = Short.valueOf(bytesToString); break; case "INT": - column = Bytes.toInt(byteArray); + column = Integer.valueOf(bytesToString); break; case "LONG": - column = Bytes.toLong(byteArray); + column = Long.valueOf(bytesToString); break; case "FLOAT": - column = Bytes.toFloat(byteArray); + column = Float.valueOf(bytesToString); break; case "DOUBLE": - column = Bytes.toDouble(byteArray); + column = Double.valueOf(bytesToString); break; case "STRING": - column = new String(byteArray, encoding); + column = bytesToString; break; case "BINARY_STRING": column = Bytes.toStringBinary(byteArray); diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java index e5e6a8ee37..c1289beab8 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/BaseHdfsInputFormat.java @@ -45,6 +45,9 @@ public abstract class BaseHdfsInputFormat extends BaseRichInputFormat { protected Map hadoopConfig; + //hadoop是否是高可用 + protected boolean isHa; + protected List metaColumns; protected String inputPath; diff --git a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java index c13b6c3898..35329692a4 100644 --- a/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-reader/src/main/java/com/dtstack/flinkx/hdfs/reader/HdfsInputFormatBuilder.java @@ -20,7 +20,6 @@ import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; import com.dtstack.flinkx.reader.MetaColumn; - import java.util.List; import java.util.Map; @@ -54,7 +53,11 @@ public void setHadoopConfig(Map hadoopConfig) { format.hadoopConfig = hadoopConfig; } - public void setFilterRegex(String filterRegex){ + public void setIsHa(boolean isHa) { + format.isHa = isHa; + } + + public void setFilterRegex(String filterRegex) { format.filterRegex = filterRegex; } @@ -79,8 +82,12 @@ public void setDefaultFs(String defaultFs) { @Override protected void checkFormat() { - if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ - throw new UnsupportedOperationException("This plugin not support restore from failed state"); + + StringBuilder errorMessage = new StringBuilder(256); + + if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()) { + errorMessage.append("This plugin not support restore from failed state\n"); } + } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java index 33f9da70aa..f7315392d0 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/BaseHdfsOutputFormat.java @@ -24,7 +24,6 @@ import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.SysUtil; import com.google.gson.Gson; -import com.google.gson.JsonParser; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java index 22925d99eb..be11aacc12 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOrcOutputFormat.java @@ -240,7 +240,7 @@ private void getData(List recordList, int index, Row row) throws WriteRe case VARCHAR: case CHAR: if (column instanceof Timestamp){ - SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); recordList.add(fm.format(column)); }else if (column instanceof Map || column instanceof List){ recordList.add(gson.toJson(column)); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java index 0d8b40b951..e1e4375c8f 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsOutputFormatBuilder.java @@ -20,6 +20,8 @@ import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; +import org.apache.commons.lang3.StringUtils; + import java.util.List; import java.util.Map; @@ -89,14 +91,18 @@ public void setEnableDictionary(boolean enableDictionary) { @Override protected void checkFormat() { - super.checkFormat(); - if (format.defaultFs == null || format.defaultFs.length() == 0) { - throw new IllegalArgumentException("No defaultFS supplied."); + + StringBuilder errorMessage = new StringBuilder(256); + + if (format.getPath() == null || format.getPath().length() == 0) { + errorMessage.append("No path supplied. \n"); } - if (!format.defaultFs.startsWith(ConstantValue.PROTOCOL_HDFS)) { - throw new IllegalArgumentException("defaultFS should start with hdfs://"); + if (StringUtils.isBlank(format.defaultFs)) { + errorMessage.append("No defaultFS supplied. \n"); + }else if (!format.defaultFs.startsWith(ConstantValue.PROTOCOL_HDFS)) { + errorMessage.append("defaultFS should start with hdfs:// \n"); } } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index 84cfa835a1..f635c61348 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -238,7 +238,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ case "varchar" : case "string" : if (valObj instanceof Timestamp){ - val=DateUtil.getDateTimeFormatter().format(valObj); + val=DateUtil.getDateTimeFormatterForMillisencond().format(valObj); group.add(colName,val); }else if (valObj instanceof Map || valObj instanceof List){ group.add(colName,gson.toJson(valObj)); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index 542f692704..3e26af9faf 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -209,7 +209,7 @@ private void appendDataToString(StringBuilder sb, Object column, ColumnType colu case VARCHAR: case CHAR: if (column instanceof Timestamp){ - SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); sb.append(fm.format(column)); }else if (column instanceof Map || column instanceof List){ sb.append(gson.toJson(column)); diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java index ef38a23131..cee32554c7 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormatBuilder.java @@ -85,8 +85,6 @@ public void setTableInfos(Map tableInfos) { public void setDistributeTableMapping(Map distributeTableMapping) { this.format.distributeTableMapping = distributeTableMapping; } - - public void setHadoopConfig(Map hadoopConfig) { format.hadoopConfig = hadoopConfig; diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java index 81526b546a..80539c3ea4 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveWriter.java @@ -20,6 +20,8 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.WriterConfig; import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.hdfs.HdfsConfigKeys; +import com.dtstack.flinkx.hive.HiveConfigKeys; import com.dtstack.flinkx.hive.TableInfo; import com.dtstack.flinkx.hive.TimePartitionFormat; import com.dtstack.flinkx.hive.util.HiveUtil; @@ -34,10 +36,13 @@ import org.apache.flink.types.Row; import parquet.hadoop.ParquetWriter; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; -import static com.dtstack.flinkx.hdfs.HdfsConfigKeys.KEY_ROW_GROUP_SIZE; -import static com.dtstack.flinkx.hive.HiveConfigKeys.*; import static com.dtstack.flinkx.util.GsonUtil.GSON; /** @@ -89,34 +94,34 @@ public HiveWriter(DataTransferConfig config) { super(config); readerName = config.getJob().getContent().get(0).getReader().getName(); WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); - hadoopConfig = (Map) writerConfig.getParameter().getVal(KEY_HADOOP_CONFIG); - defaultFs = writerConfig.getParameter().getStringVal(KEY_DEFAULT_FS); - if (StringUtils.isBlank(defaultFs) && hadoopConfig.containsKey(KEY_FS_DEFAULT_FS)){ - defaultFs = MapUtils.getString(hadoopConfig, KEY_FS_DEFAULT_FS); + hadoopConfig = (Map) writerConfig.getParameter().getVal(HiveConfigKeys.KEY_HADOOP_CONFIG); + defaultFs = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_DEFAULT_FS); + if (StringUtils.isBlank(defaultFs) && hadoopConfig.containsKey(HiveConfigKeys.KEY_FS_DEFAULT_FS)){ + defaultFs = MapUtils.getString(hadoopConfig, HiveConfigKeys.KEY_FS_DEFAULT_FS); } - fileType = writerConfig.getParameter().getStringVal(KEY_FILE_TYPE); - partitionType = writerConfig.getParameter().getStringVal(KEY_PARTITION_TYPE, TimePartitionFormat.PartitionEnum.DAY.name()); - partition = writerConfig.getParameter().getStringVal(KEY_PARTITION, "pt"); - delimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER, "\u0001"); - charSet = writerConfig.getParameter().getStringVal(KEY_CHARSET_NAME); - maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); - compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); - bufferSize = writerConfig.getParameter().getLongVal(KEY_BUFFER_SIZE, 128 * ConstantValue.STORE_SIZE_M); - rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); - - mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE, WriteMode.APPEND.name()); - jdbcUrl = writerConfig.getParameter().getStringVal(KEY_JDBC_URL); - username = writerConfig.getParameter().getStringVal(KEY_USERNAME); - password = writerConfig.getParameter().getStringVal(KEY_PASSWORD); - schema = writerConfig.getParameter().getStringVal(KEY_SCHEMA); - - String distributeTable = writerConfig.getParameter().getStringVal(KEY_DISTRIBUTE_TABLE); + fileType = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_FILE_TYPE); + partitionType = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_PARTITION_TYPE, TimePartitionFormat.PartitionEnum.DAY.name()); + partition = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_PARTITION, "pt"); + delimiter = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_FIELD_DELIMITER, "\u0001"); + charSet = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_CHARSET_NAME); + maxFileSize = writerConfig.getParameter().getLongVal(HiveConfigKeys.KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); + compress = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_COMPRESS); + bufferSize = writerConfig.getParameter().getLongVal(HiveConfigKeys.KEY_BUFFER_SIZE, 128 * ConstantValue.STORE_SIZE_M); + rowGroupSize = writerConfig.getParameter().getIntVal(HdfsConfigKeys.KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); + + mode = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_WRITE_MODE, WriteMode.APPEND.name()); + jdbcUrl = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_JDBC_URL); + username = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_USERNAME); + password = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_PASSWORD); + schema = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_SCHEMA); + + String distributeTable = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_DISTRIBUTE_TABLE); formatHiveDistributeInfo(distributeTable); - String tablesColumn = writerConfig.getParameter().getStringVal(KEY_TABLE_COLUMN); + String tablesColumn = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_TABLE_COLUMN); formatHiveTableInfo(tablesColumn); - String analyticalRules = writerConfig.getParameter().getStringVal(KEY_ANALYTICAL_RULES); + String analyticalRules = writerConfig.getParameter().getStringVal(HiveConfigKeys.KEY_ANALYTICAL_RULES); if (StringUtils.isBlank(analyticalRules)) { tableBasePath = tableInfos.entrySet().iterator().next().getValue().getTableName(); } else { diff --git a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java index fdd075b0b4..85a741b188 100644 --- a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java +++ b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/format/KafkaOutputFormat.java @@ -22,6 +22,7 @@ import com.dtstack.flinkx.kafkabase.format.KafkaBaseOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.MapUtil; +import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -42,6 +43,8 @@ public class KafkaOutputFormat extends KafkaBaseOutputFormat { private transient KafkaProducer producer; + + @Override public void configure(Configuration parameters) { super.configure(parameters); @@ -50,6 +53,7 @@ public void configure(Configuration parameters) { props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 60000); props.put(ProducerConfig.RETRIES_CONFIG, 1000000); props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1); + props.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, "com.dtstack.flinkx.kafka.format.PartitionAssigner"); if (producerSettings != null) { props.putAll(producerSettings); } @@ -60,7 +64,12 @@ public void configure(Configuration parameters) { protected void emit(Map event) throws IOException { heartBeatController.acquire(); String tp = Formatter.format(event, topic, timezone); - producer.send(new ProducerRecord<>(tp, event.toString(), MapUtil.writeValueAsString(event)), (metadata, exception) -> { + String keyMessage = generateKey(event); + //key值没有命中如果强制保证有序的话,数据将打入一个分区,反之将随机打到任意分区 + if (StringUtils.isEmpty(keyMessage) && (!dataCompelOrder)) { + keyMessage = event.toString(); + } + producer.send(new ProducerRecord<>(tp, keyMessage, MapUtil.writeValueAsString(event)), (metadata, exception) -> { if(Objects.nonNull(exception)){ String errorMessage = String.format("send data failed,data 【%s】 ,error info %s",event,ExceptionUtil.getErrorMessage(exception)); LOG.warn(errorMessage); @@ -71,6 +80,8 @@ protected void emit(Map event) throws IOException { }); } + + @Override public void closeInternal() { LOG.warn("kafka output closeInternal."); diff --git a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java index f382cd087c..e88c1a2f8e 100644 --- a/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java +++ b/flinkx-kafka/flinkx-kafka-writer/src/main/java/com/dtstack/flinkx/kafka/writer/KafkaWriter.java @@ -53,6 +53,8 @@ public DataStreamSink writeData(DataStream dataSet) { format.setDirtyHadoopConfig(dirtyHadoopConfig); format.setSrcFieldNames(srcCols); format.setHeartBeatController(new HeartBeatController()); + format.setDataCompelOrder(dataCompelOrder); + format.setPartitionAssignColumns(partitionAssignColumns); return createOutput(dataSet, format); } diff --git a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java index 7d792f2619..add067e1c7 100644 --- a/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java +++ b/flinkx-kafka09/flinkx-kafka09-writer/src/main/java/com/dtstack/flinkx/kafka09/format/Kafka09OutputFormat.java @@ -21,6 +21,7 @@ import com.dtstack.flinkx.kafkabase.util.Formatter; import com.dtstack.flinkx.kafkabase.writer.HeartBeatController; import com.dtstack.flinkx.util.MapUtil; +import com.dtstack.flinkx.util.TelnetUtil; import org.apache.flink.configuration.Configuration; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -61,7 +62,15 @@ public void configure(Configuration parameters) { props.put("metadata.broker.list", brokerList); producer = new KafkaProducer<>(props); - super.configure(parameters); + LOG.info("brokerList {}", brokerList); + String broker = brokerList.split(",")[0]; + String[] split = broker.split(":"); + + try { + TelnetUtil.telnet(split[0], Integer.parseInt(split[1])); + }catch (Exception e){ + throw new RuntimeException("telnet error, brokerList = " + brokerList); + } } @Override diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java index ccc5bf84db..d24df8c172 100755 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/KafkaConfigKeys.java @@ -18,6 +18,9 @@ package com.dtstack.flinkx.kafkabase; +import java.util.Arrays; +import java.util.List; + /** * Date: 2019/11/21 * Company: www.dtstack.com @@ -34,11 +37,13 @@ public class KafkaConfigKeys { public static final String KEY_CONSUMER_SETTINGS = "consumerSettings"; public static final String KEY_PRODUCER_SETTINGS = "producerSettings"; public static final String KEY_TABLE_FIELDS = "tableFields"; + public static final String KEY_ASSIGNER_FIELDS = "partitionAssignColumns"; + public static final String DATA_COMPEL_ORDER = "dataCompelOrder"; public static final String KEY_MODE = "mode"; //partition:0,offset:42;partition:1,offset:300 public static final String KEY_OFFSET = "offset"; public static final String KEY_TIMESTAMP = "timestamp"; - + public static List KEY_ASSIGNER_DEFAULT_RULE = Arrays.asList("database", "schema", "table"); /** * kafka 09 */ diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java index 0cdaf15be0..8cfa41b3c6 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/util/KafkaUtil.java @@ -95,10 +95,8 @@ public static Properties geneConsumerProp(Map consumerSettings, switch (mode){ case EARLIEST: props.put("auto.offset.reset", "earliest"); - break; case LATEST: props.put("auto.offset.reset", "latest"); - break; } return props; } diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java index 9d71909132..7d81c639d1 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/format/KafkaBaseOutputFormat.java @@ -26,17 +26,26 @@ import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.TelnetUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; + +import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeSet; import java.util.Properties; +import java.util.stream.Collectors; + +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_ASSIGNER_DEFAULT_RULE; /** * Date: 2019/11/21 @@ -55,6 +64,11 @@ public class KafkaBaseOutputFormat extends BaseRichOutputFormat { protected String topic; protected Map producerSettings; protected List tableFields; + //用户指定kafka分区字段 + protected List partitionAssignColumns; + //是否保证强制有序 + protected boolean dataCompelOrder; + protected Set keySet; protected static JsonDecoder jsonDecoder = new JsonDecoder(); //和kafkaBroker连通性控制器 protected HeartBeatController heartBeatController; @@ -73,6 +87,11 @@ public void configure(Configuration parameters) { throw new RuntimeException("telnet error, brokerList = " + brokerList); } } + keySet = new TreeSet<>(); + keySet.addAll(KEY_ASSIGNER_DEFAULT_RULE); + if (CollectionUtils.isNotEmpty(partitionAssignColumns)) { + keySet.addAll(partitionAssignColumns); + } } @Override @@ -117,6 +136,22 @@ protected void writeSingleRecordInternal(Row row) throws WriteRecordException { } } + /** + * 根据默认的字段和指定的字段生成key + * @param event + * @return + */ + public String generateKey(Map event) { + List values = new ArrayList<>(keySet.size()); + keySet.forEach(rule -> { + values.add(event.getOrDefault(rule, "").toString()); + }); + List collect = values.stream() + .filter(value -> StringUtils.isNotEmpty(value)) + .collect(Collectors.toList()); + return StringUtils.join(collect.toArray(), "-"); + } + protected void emit(Map event) throws IOException { throw new RuntimeException("KafkaBaseOutputFormat.emit() should be override by subclass!"); } @@ -158,4 +193,20 @@ public void setTableFields(List tableFields) { public void setHeartBeatController(HeartBeatController heartBeatController) { this.heartBeatController = heartBeatController; } + + public List getPartitionAssignColumns() { + return partitionAssignColumns; + } + + public void setPartitionAssignColumns(List partitionAssignColumns) { + this.partitionAssignColumns = partitionAssignColumns; + } + + public boolean isDataCompelOrder() { + return dataCompelOrder; + } + + public void setDataCompelOrder(boolean dataCompelOrder) { + this.dataCompelOrder = dataCompelOrder; + } } diff --git a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java index b320303c0b..27e69eee4f 100644 --- a/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java +++ b/flinkx-kb/flinkx-kb-writer/src/main/java/com/dtstack/flinkx/kafkabase/writer/KafkaBaseWriter.java @@ -28,6 +28,8 @@ import java.util.List; import java.util.Map; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.DATA_COMPEL_ORDER; +import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_ASSIGNER_FIELDS; import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_PRODUCER_SETTINGS; import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TABLE_FIELDS; import static com.dtstack.flinkx.kafkabase.KafkaConfigKeys.KEY_TIMEZONE; @@ -44,6 +46,10 @@ public class KafkaBaseWriter extends BaseDataWriter { protected String topic; protected Map producerSettings; protected List tableFields; + //是否保证强制有序 + protected boolean dataCompelOrder; + //用户指定kafka分区字段 + protected List partitionAssignColumns; @SuppressWarnings("unchecked") public KafkaBaseWriter(DataTransferConfig config) { @@ -53,6 +59,8 @@ public KafkaBaseWriter(DataTransferConfig config) { topic = writerConfig.getParameter().getStringVal(KEY_TOPIC); producerSettings = (Map) writerConfig.getParameter().getVal(KEY_PRODUCER_SETTINGS); tableFields = (List)writerConfig.getParameter().getVal(KEY_TABLE_FIELDS); + partitionAssignColumns = (List) writerConfig.getParameter().getVal(KEY_ASSIGNER_FIELDS); + dataCompelOrder = writerConfig.getParameter().getBooleanVal(DATA_COMPEL_ORDER, false); } @Override diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java index 8696d1db82..67e923f504 100644 --- a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java +++ b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/format/KingbaseOutputFormat.java @@ -70,7 +70,7 @@ protected void openInternal(int taskNumber, int numTasks){ if (restoreConfig.isRestore()){ dbConn.setAutoCommit(false); } - // 查询主键时用table格式 + // 查询主键时用jdbc的DatabaseMetaData,使用原始table作为表名 if(CollectionUtils.isEmpty(fullColumn)) { fullColumn = probeFullColumns(table, dbConn); } @@ -80,7 +80,7 @@ protected void openInternal(int taskNumber, int numTasks){ updateKey = probePrimaryKeys(table, dbConn); } } - // 其他情况,使用schema.table作为表名 + // 后续为执行sql查询主键和insert操作,使用schema.table作为表名 table = schema + ConstantValue.POINT_SYMBOL + table; if(fullColumnType == null) { fullColumnType = analyzeTable(); diff --git a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java index d11b9e1fcc..ec7ddd5541 100644 --- a/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java +++ b/flinkx-kingbase/flinkx-kingbase-writer/src/main/java/com/dtstack/flinkx/kingbase/writer/KingbaseWriter.java @@ -41,6 +41,7 @@ public class KingbaseWriter extends JdbcDataWriter { + /** {@link KingbaseOutputFormat} openInternal方法中需要使用schema进行表名拼接 */ public String schema; public KingbaseWriter(DataTransferConfig config) { diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java index d97ee6ce73..99741af53f 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/ClusterClientFactory.java @@ -25,7 +25,6 @@ import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; @@ -65,9 +64,10 @@ public static ClusterClient createClusterClient(Options launcherOptions) throws public static ClusterClient createStandaloneClient(Options launcherOptions) throws Exception { Configuration flinkConf = launcherOptions.loadFlinkConfiguration(); - StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(flinkConf); - ClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()).getClusterClient(); - return clusterClient; + try(StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(flinkConf)){ + ClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()).getClusterClient(); + return clusterClient; + } } public static ClusterClient createYarnClient(Options launcherOptions) { @@ -82,31 +82,33 @@ public static ClusterClient createYarnClient(Options launcherOptions) { kerberosInfo.verify(); YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); - YarnClient yarnClient = YarnClient.createYarnClient(); - yarnClient.init(yarnConf); - yarnClient.start(); - ApplicationId applicationId; - - if (StringUtils.isEmpty(launcherOptions.getAppId())) { - applicationId = getAppIdFromYarn(yarnClient, launcherOptions); - if(applicationId == null || StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + try (YarnClient yarnClient = YarnClient.createYarnClient()) { + yarnClient.init(yarnConf); + yarnClient.start(); + ApplicationId applicationId; + + if (StringUtils.isEmpty(launcherOptions.getAppId())) { + applicationId = getAppIdFromYarn(yarnClient, launcherOptions); + if (applicationId == null || StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); + } + } else { + applicationId = ConverterUtils.toApplicationId(launcherOptions.getAppId()); } - } else { - applicationId = ConverterUtils.toApplicationId(launcherOptions.getAppId()); - } - HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(flinkConfig); - if(highAvailabilityMode.equals(HighAvailabilityMode.ZOOKEEPER) && applicationId!=null){ - flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationId.toString()); + HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(flinkConfig); + if (highAvailabilityMode.equals(HighAvailabilityMode.ZOOKEEPER) && applicationId != null) { + flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationId.toString()); + } + try(YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( + flinkConfig, + yarnConf, + yarnClient, + YarnClientYarnClusterInformationRetriever.create(yarnClient), + true)) { + return yarnClusterDescriptor.retrieve(applicationId).getClusterClient(); + } } - YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( - flinkConfig, - yarnConf, - yarnClient, - YarnClientYarnClusterInformationRetriever.create(yarnClient), - true); - return yarnClusterDescriptor.retrieve(applicationId).getClusterClient(); } catch(Exception e) { throw new RuntimeException(e); } diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 5908122fd9..2a464515f5 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -76,11 +76,8 @@ public static void main(String[] args) throws Exception { temp.put(argList.get(i), argList.get(i + 1)); } // 对json中的值进行修改 - String p = temp.get("-p"); - if(StringUtils.isNotBlank(p)){ - HashMap parameter = JsonModifyUtil.CommandTransform(p); - temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); - } + HashMap parameter = JsonModifyUtil.CommandTransform(temp.get("-p")); + temp.put("-job", JsonModifyUtil.JsonValueReplace(temp.get("-job"), parameter)); // 清空list,填充修改后的参数值 argList.clear(); diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java index d4e0343395..6e2c98b6be 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputFormat.java @@ -74,6 +74,7 @@ public void openInputFormat() throws IOException { @Override protected void openInternal(InputSplit inputSplit) throws IOException { + LOG.info("inputSplit = {}", inputSplit); MongodbInputSplit split = (MongodbInputSplit) inputSplit; FindIterable findIterable; @@ -145,7 +146,8 @@ public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOExcepti MongoDatabase db = client.getDatabase(mongodbConfig.getDatabase()); MongoCollection collection = db.getCollection(mongodbConfig.getCollectionName()); - long docNum = filter == null ? collection.countDocuments() : collection.countDocuments(filter); + //不使用 collection.countDocuments() 获取总数是因为这个方法在大数据量时超时,导致出现超时异常结束任务 + long docNum = collection.estimatedDocumentCount(); if(docNum <= minNumSplits){ splits.add(new MongodbInputSplit(0,(int)docNum)); return splits.toArray(new MongodbInputSplit[splits.size()]); diff --git a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputSplit.java b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputSplit.java index ebc942d0af..b24442c574 100644 --- a/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputSplit.java +++ b/flinkx-mongodb/flinkx-mongodb-reader/src/main/java/com/dtstack/flinkx/mongodb/reader/MongodbInputSplit.java @@ -57,4 +57,13 @@ public void setLimit(int limit) { public int getSplitNumber() { return 0; } + + + @Override + public String toString() { + return "MongodbInputSplit{" + + "skip=" + skip + + ", limit=" + limit + + '}'; + } } diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java index 5a3f7dd249..45775e6293 100644 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/format/LogMinerConnection.java @@ -54,22 +54,37 @@ */ public class LogMinerConnection { + public static Logger LOG = LoggerFactory.getLogger(LogMinerConnection.class); + public static final String KEY_PRIVILEGE = "PRIVILEGE"; public static final String KEY_GRANTED_ROLE = "GRANTED_ROLE"; + public static final String DBA_ROLE = "DBA"; public static final String EXECUTE_CATALOG_ROLE = "EXECUTE_CATALOG_ROLE"; + public static final int ORACLE_11_VERSION = 11; + public int oracleVersion; + //oracle10数据库字符编码是否设置为GBK + public boolean isGBK = false; + boolean isOracle10; + + public static final long MAX_SCN = 281474976710655L; + public static final List PRIVILEGES_NEEDED = Arrays.asList( "CREATE SESSION", "LOGMINING", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + public static final List ORACLE_11_PRIVILEGES_NEEDED = Arrays.asList( "CREATE SESSION", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + public static final int RETRY_TIMES = 3; + public static final int SLEEP_TIME = 2000; + public final static String KEY_SEG_OWNER = "SEG_OWNER"; public final static String KEY_TABLE_NAME = "TABLE_NAME"; public final static String KEY_OPERATION = "OPERATION"; @@ -79,20 +94,25 @@ public class LogMinerConnection { public final static String KEY_SCN = "SCN"; public final static String KEY_CURRENT_SCN = "CURRENT_SCN"; public final static String KEY_FIRST_CHANGE = "FIRST_CHANGE#"; - private static final long QUERY_LOG_INTERVAL = 10000; - public static Logger LOG = LoggerFactory.getLogger(LogMinerConnection.class); - public int oracleVersion; - //oracle10数据库字符编码是否设置为GBK - public boolean isGBK = false; - boolean isOracle10; + private LogMinerConfig logMinerConfig; + private Connection connection; + private CallableStatement logMinerStartStmt; + private PreparedStatement logMinerSelectStmt; + private ResultSet logMinerData; + private QueueData result; + private List addedLogFiles = new ArrayList<>(); + private long lastQueryTime; + + private static final long QUERY_LOG_INTERVAL = 10000; + private boolean logMinerStarted = false; /** @@ -105,7 +125,6 @@ public LogMinerConnection(LogMinerConfig logMinerConfig) { } public void connect() { - PreparedStatement preparedStatement = null; try { ClassUtil.forName(logMinerConfig.getDriverName(), getClass().getClassLoader()); @@ -116,17 +135,20 @@ public void connect() { if(isOracle10){ //oracle10开启logMiner之前 需要设置会话级别的日期格式 否则sql语句会含有todate函数 而不是todate函数计算后的值 - preparedStatement = connection.prepareStatement(SqlUtil.SQL_ALTER_DATE_FORMAT); - preparedStatement.execute(); - preparedStatement = connection.prepareStatement(SqlUtil.NLS_TIMESTAMP_FORMAT); - preparedStatement.execute(); + try (PreparedStatement preparedStatement = connection.prepareStatement(SqlUtil.SQL_ALTER_DATE_FORMAT)) { + preparedStatement.execute(); + } + try (PreparedStatement preparedStatement = connection.prepareStatement(SqlUtil.NLS_TIMESTAMP_FORMAT)) { + preparedStatement.execute(); + } } LOG.info("get connection successfully, url:{}, username:{}, Oracle version:{}", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), oracleVersion); } catch (Exception e){ String message = String.format("get connection failed,url:[%s], username:[%s], e:%s", logMinerConfig.getJdbcUrl(), logMinerConfig.getUsername(), ExceptionUtil.getErrorMessage(e)); LOG.error(message); - closeResources(null, preparedStatement, connection); + //出现异常 需要关闭connection,保证connection 和 session日期配置 生命周期一致 + closeResources(null, null, connection); throw new RuntimeException(message, e); } } @@ -423,13 +445,7 @@ private List queryLogFiles(Long scn) throws SQLException{ logFiles.add(logFile); } } finally { - if (null != rs) { - rs.close(); - } - - if (null != statement) { - statement.close(); - } + closeResources(rs, statement, null); } lastQueryTime = System.currentTimeMillis(); @@ -551,6 +567,7 @@ public void checkPrivileges() { } } + private boolean containsNeededPrivileges(Statement statement) { try (ResultSet rs = statement.executeQuery(SqlUtil.SQL_QUERY_PRIVILEGES)) { List privileges = new ArrayList<>(); @@ -651,11 +668,11 @@ private void closeStmt(Statement statement){ } } - public void setPreScn(Long preScn) { - this.preScn = preScn; - } - public enum ReadPosition{ ALL, CURRENT, TIME, SCN } + + public void setPreScn(Long preScn) { + this.preScn = preScn; + } } diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java index 42d781a7a4..3bab8c86c4 100644 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/src/main/java/com/dtstack/flinkx/oraclelogminer/util/SqlUtil.java @@ -302,11 +302,15 @@ public class SqlUtil { public final static String SQL_QUERY_LOG_MODE = "SELECT LOG_MODE FROM V$DATABASE"; public final static String SQL_QUERY_SUPPLEMENTAL_LOG_DATA_ALL = "SELECT SUPPLEMENTAL_LOG_DATA_ALL FROM V$DATABASE"; - public static final List PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "LOGMINING", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); - public static final List ORACLE_11_PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + private final static List SUPPORTED_OPERATIONS = Arrays.asList("UPDATE", "INSERT", "DELETE"); + public static List EXCLUDE_SCHEMAS = Collections.singletonList("SYS"); + public static final List PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "LOGMINING", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + + public static final List ORACLE_11_PRIVILEGES_NEEDED = Arrays.asList("CREATE SESSION", "SELECT ANY TRANSACTION", "SELECT ANY DICTIONARY"); + /** * 构建查询v$logmnr_contents视图SQL * @param listenerOptions 需要采集操作类型字符串 delete,insert,update diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java index 0c66ead84b..aed7f96f48 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormat.java @@ -199,4 +199,4 @@ public long getStartLsn() { public boolean isRunning() { return running; } -} +} \ No newline at end of file diff --git a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java index c32f225e55..2beae76f04 100644 --- a/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java +++ b/flinkx-pgwal/flinkx-pgwal-reader/src/main/java/com/dtstack/flinkx/pgwal/format/PgWalInputFormatBuilder.java @@ -110,4 +110,4 @@ protected void checkFormat() { throw new IllegalArgumentException("slotName can not be null if allowCreateSlot is false"); } } -} +} \ No newline at end of file diff --git a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/reader/PhoenixReader.java b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/reader/PhoenixReader.java index 5a082c4563..6d0ea51d86 100644 --- a/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/reader/PhoenixReader.java +++ b/flinkx-phoenix/flinkx-phoenix-reader/src/main/java/com/dtstack/flinkx/phoenix/reader/PhoenixReader.java @@ -39,7 +39,6 @@ public class PhoenixReader extends JdbcDataReader { public PhoenixReader(DataTransferConfig config, StreamExecutionEnvironment env) { super(config, env); setDatabaseInterface(new PhoenixMeta()); - dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); } @Override diff --git a/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/writer/PhoenixWriter.java b/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/writer/PhoenixWriter.java index f8ca7cad7c..3f3be00526 100644 --- a/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/writer/PhoenixWriter.java +++ b/flinkx-phoenix/flinkx-phoenix-writer/src/main/java/com/dtstack/flinkx/phoenix/writer/PhoenixWriter.java @@ -38,7 +38,6 @@ public class PhoenixWriter extends JdbcDataWriter { public PhoenixWriter(DataTransferConfig config) { super(config); setDatabaseInterface(new PhoenixMeta()); - dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); } @Override diff --git a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java index e9516404ab..d960251ab3 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java +++ b/flinkx-phoenix5/flinkx-phoenix5-reader/src/main/java/com/dtstack/flinkx/phoenix5/format/Phoenix5InputFormat.java @@ -177,8 +177,9 @@ public void openInternal(InputSplit inputSplit) throws IOException { hConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, (String) rootNode); } hConfiguration.setBoolean(HConstants.CLUSTER_DISTRIBUTED, true); - org.apache.hadoop.hbase.client.Connection hConn = ConnectionFactory.createConnection(hConfiguration); - hTable = hConn.getTable(TableName.valueOf(table)); + try (org.apache.hadoop.hbase.client.Connection hConn = ConnectionFactory.createConnection(hConfiguration)) { + hTable = hConn.getTable(TableName.valueOf(table)); + } resultIterator = hTable.getScanner(scan).iterator(); } catch (Exception e) { String message = String.format("openInputFormat() failed, dbUrl = %s, properties = %s, e = %s", dbUrl, GsonUtil.GSON.toJson(properties), ExceptionUtil.getErrorMessage(e)); diff --git a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java index 25d76cb568..674308a930 100644 --- a/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java +++ b/flinkx-phoenix5/flinkx-phoenix5-writer/src/main/java/com/dtstack/flinkx/phoenix5/writer/Phoenix5Writer.java @@ -38,7 +38,6 @@ public class Phoenix5Writer extends JdbcDataWriter { public Phoenix5Writer(DataTransferConfig config) { super(config); setDatabaseInterface(new Phoenix5DatabaseMeta()); - dbUrl = DbUtil.formatJdbcUrl(dbUrl, Collections.singletonMap("zeroDateTimeBehavior", "convertToNull")); } @Override diff --git a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlReader.java b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlReader.java index 02cc02a40a..e751ecbb50 100644 --- a/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlReader.java +++ b/flinkx-postgresql/flinkx-postgresql-reader/src/main/java/com/dtstack/flinkx/postgresql/reader/PostgresqlReader.java @@ -19,6 +19,7 @@ package com.dtstack.flinkx.postgresql.reader; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.inputformat.BaseRichInputFormat; import com.dtstack.flinkx.postgresql.PostgresqlDatabaseMeta; import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; @@ -27,6 +28,7 @@ import com.dtstack.flinkx.rdb.datareader.QuerySqlBuilder; import com.dtstack.flinkx.rdb.inputformat.JdbcInputFormatBuilder; import com.dtstack.flinkx.rdb.util.DbUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; @@ -44,6 +46,11 @@ public PostgresqlReader(DataTransferConfig config, StreamExecutionEnvironment en setDatabaseInterface(new PostgresqlDatabaseMeta()); setTypeConverterInterface(new PostgresqlTypeConverter()); dbUrl = DbUtil.formatJdbcUrl(dbUrl, null); + + String schema = config.getJob().getContent().get(0).getReader().getParameter().getConnection().get(0).getSchema(); + if (StringUtils.isNotEmpty(schema)){ + table = schema + ConstantValue.POINT_SYMBOL + table; + } } @Override diff --git a/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/writer/PostgresqlWriter.java b/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/writer/PostgresqlWriter.java index ec7958f393..243e1c1d71 100644 --- a/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/writer/PostgresqlWriter.java +++ b/flinkx-postgresql/flinkx-postgresql-writer/src/main/java/com/dtstack/flinkx/postgresql/writer/PostgresqlWriter.java @@ -19,12 +19,14 @@ package com.dtstack.flinkx.postgresql.writer; import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.constants.ConstantValue; import com.dtstack.flinkx.postgresql.PostgresqlDatabaseMeta; import com.dtstack.flinkx.postgresql.PostgresqlTypeConverter; import com.dtstack.flinkx.postgresql.format.PostgresqlOutputFormat; import com.dtstack.flinkx.rdb.datawriter.JdbcDataWriter; import com.dtstack.flinkx.rdb.outputformat.JdbcOutputFormatBuilder; import com.dtstack.flinkx.streaming.api.functions.sink.DtOutputFormatSinkFunction; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.types.Row; @@ -37,8 +39,13 @@ */ public class PostgresqlWriter extends JdbcDataWriter { + public PostgresqlWriter(DataTransferConfig config) { super(config); + String schema = config.getJob().getContent().get(0).getWriter().getParameter().getConnection().get(0).getSchema(); + if (StringUtils.isNotEmpty(schema)){ + table = schema + ConstantValue.POINT_SYMBOL + table; + } setDatabaseInterface(new PostgresqlDatabaseMeta()); setTypeConverterInterface(new PostgresqlTypeConverter()); } diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java index 1c5ea0d024..c9a1e86489 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/util/DbUtil.java @@ -404,6 +404,12 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap) String[] splits = DB_PATTERN.split(dbUrl); Map paramMap = new HashMap<>(16); + if(!CollectionUtil.isNullOrEmpty(extParamMap)){ + paramMap.putAll(extParamMap); + } + paramMap.put("useCursorFetch", "true"); + paramMap.put("rewriteBatchedStatements", "true"); + if(splits.length > 1) { String[] pairs = splits[1].split("&"); for(String pair : pairs) { @@ -412,12 +418,6 @@ public static String formatJdbcUrl(String dbUrl, Map extParamMap) } } - if(!CollectionUtil.isNullOrEmpty(extParamMap)){ - paramMap.putAll(extParamMap); - } - paramMap.put("useCursorFetch", "true"); - paramMap.put("rewriteBatchedStatements", "true"); - StringBuffer sb = new StringBuffer(dbUrl.length() + 128); sb.append(splits[0]).append("?"); int index = 0; diff --git a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java index d4b91d450d..535f0ebbca 100644 --- a/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java +++ b/flinkx-rdb/flinkx-rdb-reader/src/main/java/com.dtstack.flinkx.rdb.inputformat/JdbcInputFormat.java @@ -33,9 +33,9 @@ import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.GsonUtil; import com.dtstack.flinkx.util.RetryUtil; +import com.dtstack.flinkx.util.MapUtil; import com.dtstack.flinkx.util.StringUtil; import com.dtstack.flinkx.util.UrlUtil; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import org.apache.commons.lang3.StringUtils; import org.apache.flink.core.io.InputSplit; @@ -724,7 +724,7 @@ private void uploadMetricData() throws IOException { if (endLocationAccumulator != null) { metrics.put(Metrics.END_LOCATION, endLocationAccumulator.getLocalValue()); } - out.writeUTF(new ObjectMapper().writeValueAsString(metrics)); + out.writeUTF(MapUtil.writeValueAsString(metrics)); } catch (Exception e) { LOG.error("hadoop conf:{}", hadoopConfig); throw new IOException("Upload metric to HDFS error", e); diff --git a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java index 8d94f93bc6..70e73ebd5f 100644 --- a/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java +++ b/flinkx-rdb/flinkx-rdb-writer/src/main/java/com/dtstack/flinkx/rdb/outputformat/JdbcOutputFormat.java @@ -46,6 +46,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; @@ -123,7 +124,7 @@ public class JdbcOutputFormat extends BaseRichOutputFormat { "AND t.table_name = '%s'"; protected final static String CONN_CLOSE_ERROR_MSG = "No operations allowed"; - protected static List STRING_TYPES = Arrays.asList("CHAR", "VARCHAR","TINYBLOB","TINYTEXT","BLOB","TEXT", "MEDIUMBLOB", "MEDIUMTEXT", "LONGBLOB", "LONGTEXT"); + protected static List STRING_TYPES = Arrays.asList("CHAR", "VARCHAR", "VARCHAR2", "NVARCHAR2", "NVARCHAR", "TINYBLOB","TINYTEXT","BLOB","TEXT", "MEDIUMBLOB", "MEDIUMTEXT", "LONGBLOB", "LONGTEXT"); protected PreparedStatement prepareTemplates() throws SQLException { if(CollectionUtils.isEmpty(fullColumn)) { @@ -358,7 +359,7 @@ protected Object getField(Row row, int index) { //field为空字符串,且写入目标类型不为字符串类型的字段,则将object设置为null if(field instanceof String && StringUtils.isBlank((String) field) - &&!STRING_TYPES.contains(type)){ + &&!STRING_TYPES.contains(type.toUpperCase(Locale.ENGLISH))){ return null; } diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java deleted file mode 100644 index 787b6bf43b..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantParam.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import org.apache.commons.lang3.StringUtils; - -import java.sql.Date; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.format.DateTimeFormatter; -import java.util.Objects; - -/** - * ConstantParan - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class ConstantParam implements ParamDefinition { - - private final String name; - - private final ParamType paramType; - - private final Class valueClass; - - protected Object value; - - private final String description; - - private String formatDescription; - - private final DateTimeFormatter format; - - public ConstantParam(String name, ParamType paramType, Class valueClass, Object value, String description, String format) { - this.name = name; - this.paramType = paramType; - this.valueClass = valueClass; - this.description = description; - this.value = value; - this.formatDescription = format; - if (StringUtils.isNotBlank(format)) { - this.format = DateTimeFormatter.ofPattern(format); - } else { - this.format = null; - } - - } - - @Override - public String getName() { - return name; - } - - @Override - public ParamType getType() { - return paramType; - } - - @Override - public Object getValue() { - return value; - } - - @Override - public String getValueType() { - return "valueClass"; - } - - - @Override - public String getDescription() { - return description; - } - - @Override - public String getFormat() { - return formatDescription; - } - - @Override - public Object format(Object data) { - if (Objects.isNull(format) || Objects.isNull(data)) { - return data; - } - if (getValueType().equals(Date.class)) { - Date value1 = (Date) data; - LocalDateTime ldt = value1.toInstant() - .atZone(ZoneId.systemDefault()) - .toLocalDateTime(); - return format.format(ldt); - } - return null; - } - - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java deleted file mode 100644 index 6cafee83d4..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ConstantVarible.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.dtstack.flinkx.restapi.common; - -public class ConstantVarible implements Paramitem { - private final T object; - private final String name; - - public ConstantVarible(T object, String name) { - this.object = object; - this.name = name; - } - - @Override - public T getValue(RestContext restContext) { - return object; - } - - @Override - public String getName() { - return name; - } - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java deleted file mode 100644 index 36267d991d..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/CurrentTimeVarible.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.Date; - -/** - * CurrentTimeVarible - * - * @author by dujie@dtstack.com - * @Date 2020/9/28 - */ -public class CurrentTimeVarible implements Paramitem { - - public CurrentTimeVarible() { - - } - - @Override - public Date getValue(RestContext restContext) { - return new Date(); - } - - @Override - public String getName() { - return "currenttime"; - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java deleted file mode 100644 index 49deeadc21..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/DymaticParam.java +++ /dev/null @@ -1,260 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import com.github.pfmiles.dropincc.*; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.math.NumberUtils; - -import java.math.BigDecimal; -import java.sql.Timestamp; -import java.util.*; -import java.util.concurrent.atomic.AtomicReference; - -/** - * DymaticParam - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class DymaticParam extends ConstantParam implements ParamDefinitionNextAble { - - private RestContext restContext; - - private final List dymaticNowString; - private final List dymaticNextString; - - - private final String initValueExpression; - private final String nextValueExpression; - - private Exe exe = getExe(); - - - public DymaticParam(String name, ParamType paramType, Class valueClass, String nowValue, String nextValue, String description, String format, RestContext context) { - super(name, paramType, valueClass, null, description, format); - // nowvalue一定是存在的 - initValueExpression = nowValue; - - dymaticNowString = ParamFactory.getVarible(nowValue); - - if (StringUtils.isBlank(nextValue)) { - dymaticNextString = dymaticNowString; - nextValueExpression = initValueExpression; - } else { - dymaticNextString = ParamFactory.getVarible(nextValue); - nextValueExpression = nextValue; - } - this.restContext = context; - } - - @Override - public Object getValue() { - return getDymaticValue(dymaticNowString, initValueExpression); - } - - /** - * 字符串解析为后缀表达式 - * `如何判断是动态变量 - * 如何判断是符合后缀表达式的 - * 后缀表达式计算 - * - * @return - */ - @Override - public Object getNextValue() { - return getDymaticValue(dymaticNextString, nextValueExpression); - } - - //判断是否是运算符 - public boolean isOperator(String oper) { - return oper.equals("+") || oper.equals("-") || oper.equals("*") || oper.equals("/"); - } - - //计算 如何变量1 2 只要有一个是字符串 那么就直接拼接 或者为null 为null 2+2+ - public int calculation(int num1, int num2, String oper) { - switch (oper) { - case "+": - return num2 + num1; - case "-": - return num2 - num1; - case "*": - return num2 * num1; - case "/": - return num2 / num1; - default: - return 0; - } - } - - public int operationLv(char operation) {//给运算符设置优先级 - switch (operation) { - case '+': - case '-': - return 1; - case '*': - case '/': - return 2; - case '(': - case ')': - return 3; - default: - return 0; - } - } - - - @Override - public void init() { - - } - - public Exe getExe() { - Lang c = new Lang("Calculator"); - Grule expr = c.newGrule(); - c.defineGrule(expr, CC.EOF).action(new Action() { - public BigDecimal act(Object matched) { - return new BigDecimal(((Object[]) matched)[0].toString()); - } - }); - TokenDef a = c.newToken("\\+"); - Grule addend = c.newGrule(); - expr.define(addend, CC.ks(a.or("\\-"), addend)).action(new Action() { - public BigDecimal act(Object matched) { - Object[] ms = (Object[]) matched; - BigDecimal a0 = (BigDecimal) ms[0]; - Object[] aPairs = (Object[]) ms[1]; - for (Object p : aPairs) { - String op = (String) ((Object[]) p)[0]; - BigDecimal a = (BigDecimal) ((Object[]) p)[1]; - if ("+".equals(op)) { - a0 = a.add(a0); - } else { - a0 = a0.subtract(a); - } - } - return a0; - } - }); - TokenDef m = c.newToken("\\*"); - Grule factor = c.newGrule(); - addend.define(factor, CC.ks(m.or("/"), factor)).action(new Action() { - public BigDecimal act(Object matched) { - Object[] ms = (Object[]) matched; - BigDecimal f0 = (BigDecimal) ms[0]; - Object[] fPairs = (Object[]) ms[1]; - for (Object p : fPairs) { - String op = (String) ((Object[]) p)[0]; - BigDecimal f = (BigDecimal) ((Object[]) p)[1]; - if ("*".equals(op)) { - f0 = f0.multiply(f); - } else { - f0 = f0.divide(f, BigDecimal.ROUND_HALF_UP, 3); - } - } - return f0; - } - }); - factor.define("\\(^[\\-]", expr, "\\)").action(new Action() { - public BigDecimal act(Object matched) { - return (BigDecimal) ((Object[]) matched)[1]; - } - }).alt("\\(\\-\\d+(\\.\\d+)?\\)|\\d+(\\.\\d+)?").action(new Action() { - public BigDecimal act(Object matched) { - if( matched.toString().startsWith("(") && matched.toString().endsWith(")")){ - return new BigDecimal(matched.toString().substring(1,matched.toString().length()-1)); - }else{ - return new BigDecimal(matched.toString()); - } - - } - }); - Exe exe = c.compile(); - return exe; - } - - private Object getDymaticValue(List dymaticString, String expression) { - if (CollectionUtils.isEmpty(dymaticString)) { - return expression; - } - Map tempReplaceValue = new HashMap<>(16); - AtomicReference tempExpression = new AtomicReference<>(expression); - Object tempValue; - dymaticString.forEach(k -> { - Object value = k.getValue(restContext); - tempReplaceValue.put(escapeExprSpecialWord("${" + k.getName() + "}"), value); - }); - tempReplaceValue.forEach((k, v) -> { - if(Objects.isNull(v)){ - tempExpression.set(tempExpression.get().replaceFirst(k, 0+"")); - }else if (NumberUtils.isNumber(v.toString())) { - tempExpression.set(tempExpression.get().replaceFirst(k, v.toString())); - } else if (v instanceof Date) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((Date) v).getTime() + "")); - } - if (v instanceof java.sql.Date) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((java.sql.Date) v).getTime() + "")); - } - if (v instanceof Timestamp) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((Timestamp) v).getTime() + "")); - } - }); - - - try { - tempValue = exe.eval(tempExpression.get()); - } catch (Exception e) { - tempReplaceValue.forEach((k, v) -> { - if(Objects.isNull(v)){ - tempExpression.set(tempExpression.get().replaceFirst(k, "")); - }else if (NumberUtils.isNumber(v.toString())) { - tempExpression.set(tempExpression.get().replaceFirst(k, v.toString())); - } else if (v instanceof Date) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((Date) v).toString() + "")); - } - if (v instanceof java.sql.Date) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((java.sql.Date) v).toString() + "")); - } - if (v instanceof Timestamp) { - tempExpression.set(tempExpression.get().replaceFirst(k, ((Timestamp) v).toString() + "")); - } - }); - tempValue = initValueExpression; - } - restContext.updateValue(getType().name().toLowerCase(Locale.ENGLISH) + "." + getName(), tempValue); - return tempValue; - } - /** - * 转义正则特殊字符 ($()*+.[]?\^{},|) - * - * @param keyword - * @return - */ - public static String escapeExprSpecialWord(String keyword) { - if (StringUtils.isNotBlank(keyword)) { - String[] fbsArr = { "\\", "$", "(", ")", "*", "+", ".", "[", "]", "?", "^", "{", "}", "|" }; - for (String key : fbsArr) { - if (keyword.contains(key)) { - keyword = keyword.replace(key, "\\" + key); - } - } - } - return keyword; - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java index 8b7b9b1b23..a13fb6f195 100644 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/HttpUtil.java @@ -17,20 +17,29 @@ */ package com.dtstack.flinkx.restapi.common; +import com.dtstack.flinkx.util.ExceptionUtil; import com.google.gson.Gson; +import org.apache.commons.collections.MapUtils; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.conn.ssl.NoopHostnameVerifier; import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.ssl.SSLContextBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.SSLContext; import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Map; /** @@ -47,6 +56,29 @@ public class HttpUtil { public static Gson gson = new Gson(); public static CloseableHttpClient getHttpClient() { + + return getBaseBuilder().build(); + } + + + public static CloseableHttpClient getHttpsClient() { + + // 设置Http连接池 + SSLContext sslContext; + try { + sslContext = new SSLContextBuilder() + .loadTrustMaterial(null, (certificate, authType) -> true).build(); + } catch (Exception e) { + LOG.warn(ExceptionUtil.getErrorMessage(e)); + throw new RuntimeException(e); + } + return getBaseBuilder() + .setSSLContext(sslContext) + .setSSLHostnameVerifier(new NoopHostnameVerifier()) + .build(); + } + + public static HttpClientBuilder getBaseBuilder() { // 设置自定义的重试策略 MyServiceUnavailableRetryStrategy strategy = new MyServiceUnavailableRetryStrategy .Builder() @@ -73,9 +105,7 @@ public static CloseableHttpClient getHttpClient() { .setServiceUnavailableRetryStrategy(strategy) .setRetryHandler(retryHandler) .setDefaultRequestConfig(requestConfig) - .setConnectionManager(pcm) - .build(); -// return HttpClientBuilder.create().build(); + .setConnectionManager(pcm); } public static HttpRequestBase getRequest(String method, @@ -83,7 +113,7 @@ public static HttpRequestBase getRequest(String method, Map header, String url) { LOG.debug("current request url: {} current method:{} \n", url, method); - HttpRequestBase request = null; + HttpRequestBase request ; if (HttpMethod.GET.name().equalsIgnoreCase(method)) { request = new HttpGet(url); @@ -101,6 +131,51 @@ public static HttpRequestBase getRequest(String method, return request; } + + public static HttpRequestBase getRequest(String method, + Map requestBody, + Map requestParam, + Map header, + String url) { + + HttpRequestBase request ; + if (MapUtils.isNotEmpty(requestParam)) { + ArrayList params = new ArrayList<>(); + requestParam.forEach((k, v) -> { + try { + //参数进行编码 + params.add(URLEncoder.encode(k, StandardCharsets.UTF_8.name()) + "=" + URLEncoder.encode(v, StandardCharsets.UTF_8.name())); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("URLEncoder.encode k [" + k + "] or v ["+ v +"] failed ", e); + } + }); + if (url.contains("?")) { + url += "&" + String.join("&", params); + } else { + url += "?" + String.join("&", params); + } + } + + LOG.debug("current request url: {} current method:{} \n", url, method); + if (HttpMethod.GET.name().equalsIgnoreCase(method)) { + request = new HttpGet(url); + } else if (HttpMethod.POST.name().equalsIgnoreCase(method)) { + HttpPost post = new HttpPost(url); + HashMap tmp = new HashMap<>(); + requestBody.forEach(tmp::put); + post.setEntity(getEntityData(tmp)); + request = post; + } else { + throw new UnsupportedOperationException("Unsupported method:" + method); + } + + for (Map.Entry entry : header.entrySet()) { + request.addHeader(entry.getKey(), entry.getValue()); + } + return request; + } + + public static void closeClient(CloseableHttpClient httpClient) { try { httpClient.close(); diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java deleted file mode 100644 index 62102263ea..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/InnerVaribleFactory.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.HashMap; -import java.util.Map; - -/** - * InnerVaribleFactory - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class InnerVaribleFactory { - - private static Map InnerVaribles; - - static { - InnerVaribles = new HashMap<>(18); - InnerVaribles.put("uuid", new UuidVarible()); - InnerVaribles.put("currenttime", new CurrentTimeVarible()); - - } - - public static Paramitem createInnerVarible(String name) { - return InnerVaribles.get(name); - } - - public static boolean isInnerVariable(String name) { - return InnerVaribles.containsKey(name); - } - - - public static void addVarible(String name,Paramitem paramitem) { - InnerVaribles.put(name,paramitem); - } - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java deleted file mode 100644 index 8adc73ab6f..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/IntervalTimeVarible.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -/** - * CurrentTimeVarible - * - * @author by dujie@dtstack.com - * @Date 2020/9/28 - */ -public class IntervalTimeVarible implements Paramitem { - - private final Long intervalTime; - - public IntervalTimeVarible(Long intervalTime) { - this.intervalTime = intervalTime; - } - - @Override - public Long getValue(RestContext restContext) { - return intervalTime; - } - - @Override - public String getName() { - return "intervalTime"; - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java deleted file mode 100644 index c1bdfc5b13..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/MapUtils.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.dtstack.flinkx.restapi.common; - -import com.dtstack.flinkx.util.GsonUtil; - -import java.util.Map; -import java.util.Objects; - -public class MapUtils { - - - public static Object getData(Map data, String[] names) { - Map tempHashMap = data; - for (int i = 0; i < names.length; i++) { - if (tempHashMap.containsKey(names[i]) && i != names.length - 1) { - if (Objects.isNull(tempHashMap.get(names[i]))) { - return null; - } - if (tempHashMap.get(names[i]) instanceof Map) { - tempHashMap = (Map) tempHashMap.get(names[i]); - } else if (tempHashMap.get(names[i]) instanceof String) { - try { - tempHashMap = GsonUtil.GSON.fromJson((String) tempHashMap.get(names[i]), GsonUtil.gsonMapTypeToken); - } catch (Exception e) { - return null; - } - } else { - return null; - } - } else if (i == names.length - 1) { - return tempHashMap.get(names[i]); - } else { - return null; - } - } - return null; - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java deleted file mode 100644 index 0439ca70db..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinition.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -/** - * ParamDefinition - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public interface ParamDefinition { - - /** - * key - * - * @return - */ - String getName(); - - ParamType getType(); - - Object getValue(); - - String getValueType(); - - /** - * 用户定义的json - * - * @return - */ - String getDescription(); - - String getFormat(); - - Object format(Object data); - - - default void init(){ - - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java deleted file mode 100644 index b51b828176..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamDefinitionNextAble.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.List; - -/** - * DymaticParam - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public interface ParamDefinitionNextAble extends ParamDefinition { - - Object getNextValue(); -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java deleted file mode 100644 index d7f8f902a5..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamFactory.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; - -import java.util.*; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * ParamFactory - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class ParamFactory { - - public static Pattern valueExpression = - Pattern.compile("(?(\\$\\{(.*?)\\}))"); - - public static List createDefinition(ParamType paramType, Map> variableDescriptions, RestContext context) { - - //2 如果是动态变量 还需要解析为后缀表达式 - //先用一个正则 将动态变量拿出来 参数拿出来 - if (variableDescriptions == null || variableDescriptions.isEmpty()) { - return Collections.emptyList(); - } - List data = new ArrayList<>(variableDescriptions.size()); - - Iterator>> iterator = variableDescriptions.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry> entry = iterator.next(); - String name = entry.getKey(); - Map entryValue = entry.getValue(); - String value = entryValue.get("value"); - String next = entryValue.get("next"); - String format = entryValue.get("format"); - String type = entryValue.get("type"); - - - //1 含有next 2 value里含有动态函数 isDymatic(); - //valueItems - //nextItems 如果不是空 或者 直接返回动态的 - if (isDymatic(value) || StringUtils.isNotBlank(next)) { - DymaticParam dymaticParam = new DymaticParam(name, paramType, null, value, next, variableDescriptions.toString(), format, context); - data.add(dymaticParam); - } else { - data.add(new ConstantParam(name, paramType, null, value, variableDescriptions.toString(), format)); - } - } - return data; - } - - - public static List getVarible(String text) { - List valueItems = new ArrayList<>(16); - Matcher matcher = valueExpression.matcher(text); - while (matcher.find()) { - String varible = matcher.group("varible"); - valueItems.add(parsr(varible)); - } - if (CollectionUtils.isEmpty(valueItems)) { - return Collections.emptyList(); - } - return valueItems; - } - - public static boolean isDymatic(String text) { - return valueExpression.matcher(text).find(); - } - - - public static Paramitem parsr(String value) { - - if (value.startsWith("${") && value.endsWith("}")) { - String substring = value.substring(2, value.length() - 1); - if (substring.startsWith("body.") || substring.startsWith("param.") || substring.startsWith("header.") || substring.startsWith("response.")) { - return new ReplaceParamItem(substring); - } else if (InnerVaribleFactory.isInnerVariable(substring)) { - return InnerVaribleFactory.createInnerVarible(substring); - } else { - return new ConstantVarible(value, value); - } - } else { - return new ConstantVarible(value, value); - } - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java index 8f0d25c26b..afb10fd7ce 100644 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java +++ b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ParamType.java @@ -21,12 +21,17 @@ * ParamType * * @author by dujie@dtstack.com - * @Date 2020/9/26 */ public enum ParamType { + /** request请求的header标识 **/ HEADER, + /** request请求的body标识 **/ BODY, + /** request请求的param标识 **/ PARAM, - RESPONSE; + /** request请求的返回值标识 **/ + RESPONSE, + /** 内部变量标识 **/ + INNER } diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java deleted file mode 100644 index 7c2b4b7a81..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/Paramitem.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -/** - * Paramitem - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public interface Paramitem { - T getValue(RestContext restContext); - - String getName(); - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java deleted file mode 100644 index 1d5062a758..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/ReplaceParamItem.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.Collections; -import java.util.Locale; -import java.util.Optional; - -/** - * ReplaceParamItem - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class ReplaceParamItem implements Paramitem { - private final String name; - - public ReplaceParamItem(String name) { - this.name=name; - } - - @Override - public Object getValue(RestContext restContext) { - return restContext.getValue(name); - } - - @Override - public String getName() { - return name; - } - - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java deleted file mode 100644 index b1b0373c4e..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/RestContext.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.*; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * RestContext - * - * @author by dujie@dtstack.com - * @Date 2020/9/26 - */ -public class RestContext { - - private String requestType; - - private String url; - - private String format; - - private boolean first; - - private ThreadLocal> setThreadLocal = new ThreadLocal<>(); - - private Map prevRequestValue = new HashMap<>(16); - - private Map requestValue; - - private Map> paramDefinitions; - - private Object object = new Object(); - - public RestContext(String requestType,String url,String format) { - this.requestType =requestType; - this.url =url; - this.requestValue = new HashMap<>(32); - this.paramDefinitions = new HashMap<>(32); - this.first = true; - this.format=format; - } - - public void updateValue(){ - prevRequestValue = requestValue; - requestValue=new HashMap<>(requestValue.size()); - } - - public Map getPreValue() { - return requestValue; - } - - - public httprequestApi.Httprequest build() { - // 根据当前value计算出httprequest - httprequestApi.Httprequest httprequest = new httprequestApi.Httprequest(); - - Map body = new HashMap<>(16); - Map header = new HashMap<>(16); - Map param = new HashMap<>(16); - if (first) { - paramDefinitions.get(ParamType.BODY).forEach((k, v) -> { - body.put(k, v.getValue()); - }); - - paramDefinitions.get(ParamType.HEADER).forEach((k, v) -> { - header.put(k, v.getValue().toString()); - }); - - paramDefinitions.get(ParamType.PARAM).forEach((k, v) -> { - param.put(k, v.getValue()); - }); - first = false; - } else { - paramDefinitions.get(ParamType.BODY).forEach((k, v) -> { - if (v instanceof ParamDefinitionNextAble) { - body.put(k, ((ParamDefinitionNextAble) v).getNextValue()); - } else { - body.put(k, v.getValue()); - } - }); - - paramDefinitions.get(ParamType.HEADER).forEach((k, v) -> { - if (v instanceof ParamDefinitionNextAble) { - header.put(k, ((ParamDefinitionNextAble) v).getNextValue().toString()); - } else { - header.put(k, v.getValue().toString()); - } - }); - - paramDefinitions.get(ParamType.PARAM).forEach((k, v) -> { - if (v instanceof ParamDefinitionNextAble) { - param.put(k, ((ParamDefinitionNextAble) v).getNextValue()); - } else { - param.put(k, v.getValue()); - } - }); - } - httprequest.buildBody(body).buildHeader(header).buildParam(param); - return httprequest; - } - - public httprequestApi.Httprequest buildNext() { - // 计算下一次的请求 但是不会更新preValue值 - return null; - } - - public void parseAndInt(Map> job, ParamType paramType) { - paramDefinitions.put(paramType, ParamFactory.createDefinition(paramType, job, this).stream().collect(Collectors.toMap(ParamDefinition::getName, Function.identity()))); - } - - - - protected void updateValue(String key, Object value) { - requestValue.put(key, value); - } - - private Object getValueQuick(String key) { - return prevRequestValue.get(key); - } - - protected Object getValue(String key) { - if (setThreadLocal.get() == null) { - setThreadLocal.set(new HashSet<>(paramDefinitions.size() * 8)); - } - if (setThreadLocal.get().contains(key)) { - throw new RuntimeException("循环依赖"); - } - setThreadLocal.get().add(key); - Object data = getValueQuick(key); - if (Objects.isNull(data) && first) { - String[] s = key.split("\\."); - ParamType paramType = ParamType.valueOf(s[0].toUpperCase(Locale.ENGLISH)); - if (Objects.isNull(paramDefinitions.get(paramType))) { - data = null; - } else { - ParamDefinition definition = paramDefinitions.get(paramType).get(s[1]); - //没查到对应的动态变量 - if (Objects.isNull(definition)) { - prevRequestValue.put(key, object); - } else { - Object value = definition.getValue(); - data=value; - prevRequestValue.put(key,value ); - } - } - } - setThreadLocal.get().remove(key); - - if (data == object) { - data = null; - } - return data; - } - //循环依赖使用有向图判断 - - public Map> getParamDefinitions() { - return paramDefinitions; - } - - public String getRequestType() { - return requestType; - } - - public String getUrl() { - return url; - } - - public String getFormat() { - return format; - } - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java deleted file mode 100644 index 3fd1be6283..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/UuidVarible.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.dtstack.flinkx.restapi.common; - -import java.util.UUID; - -public class UuidVarible implements Paramitem { - - - public UuidVarible() { - } - - @Override - public String getValue(RestContext restContext) { - return UUID.randomUUID().toString(); - } - - @Override - public String getName() { - return "currentTime"; - } - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java deleted file mode 100644 index cbb9c9cc4e..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ReadRecordException.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.dtstack.flinkx.restapi.common.exception; - -public class ReadRecordException extends RuntimeException { - public ReadRecordException() { - } - - public ReadRecordException(String message) { - super(message); - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java deleted file mode 100644 index bd780a05b9..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseBreakException.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.dtstack.flinkx.restapi.common.exception; - -public class ResponseBreakException extends RuntimeException{ - public ResponseBreakException() { - } - - public ResponseBreakException(String message) { - super(message); - } - - public ResponseBreakException(String message, Throwable cause) { - super(message, cause); - } - - public ResponseBreakException(Throwable cause) { - super(cause); - } - - public ResponseBreakException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java deleted file mode 100644 index 19ca560944..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/exception/ResponseRetryException.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.dtstack.flinkx.restapi.common.exception; - -public class ResponseRetryException extends RuntimeException { - public ResponseRetryException() { - } - - public ResponseRetryException(String message) { - super(message); - } - - public ResponseRetryException(String message, Throwable cause) { - super(message, cause); - } - - public ResponseRetryException(Throwable cause) { - super(cause); - } - - public ResponseRetryException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java deleted file mode 100644 index 1b61b82756..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/BreakDataHandler.java +++ /dev/null @@ -1,26 +0,0 @@ -package com.dtstack.flinkx.restapi.common.handler; - -import com.dtstack.flinkx.restapi.common.MapUtils; -import com.dtstack.flinkx.restapi.common.exception.ResponseBreakException; -import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; - -import java.util.Map; -import java.util.Set; - -public class BreakDataHandler extends DataHandler { - - public BreakDataHandler(String key, Set value) { - super(key, value); - } - - @Override - public void execute(Map responseData) { - String[] strings = new String[0]; - strings[0] = key; - Object data = MapUtils.getData(responseData, strings); - if (value.contains(data.toString())) { - throw new ResponseBreakException("key:"+key+" contains"+data.toString()+" ,need break"); - } - - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java deleted file mode 100644 index fe0a4cf643..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandler.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.dtstack.flinkx.restapi.common.handler; - -import java.util.Map; -import java.util.Set; - -public abstract class DataHandler { - - protected String key; - - protected Set value; - - public DataHandler(String key, Set value) { - this.key = key; - this.value = value; - } - - public boolean isPipei(Map responseData) { - return responseData.containsKey(key) && value.contains(responseData.get(key).toString()); - } - - public abstract void execute(Map responseData); - -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java deleted file mode 100644 index 94b94b9ca1..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/DataHandlerFactory.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.dtstack.flinkx.restapi.common.handler; - -import org.apache.commons.lang.StringUtils; - -import java.util.HashMap; -import java.util.Map; - -public class DataHandlerFactory { - - - private static HashMap handlerHashMap = new HashMap<>(16); - - static { - - } - - public static DataHandler getDataHandler(Map dataHandlerParam) { - if (StringUtils.isNotBlank(dataHandlerParam.get("key").toString())) { - return handlerHashMap.get(dataHandlerParam.get("key").toString()); - }else{ - throw new IllegalArgumentException("dataHandler key not allow blank"); - } - } - - public static void destroy() { - //gc - handlerHashMap = null; - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java deleted file mode 100644 index 2ae704a97c..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/handler/RetryDataHandler.java +++ /dev/null @@ -1,24 +0,0 @@ -package com.dtstack.flinkx.restapi.common.handler; - -import com.dtstack.flinkx.restapi.common.MapUtils; -import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; - -import java.util.Map; -import java.util.Set; - -public class RetryDataHandler extends DataHandler { - - public RetryDataHandler(String key, Set value) { - super(key, value); - } - - @Override - public void execute(Map responseData) { - String[] strings = new String[0]; - strings[0] = key; - Object data = MapUtils.getData(responseData, strings); - if (value.contains(data.toString())) { - throw new ResponseRetryException("key:"+key+" contains"+data.toString()+" ,need retry"); - } - } -} diff --git a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java b/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java deleted file mode 100644 index 69e203c660..0000000000 --- a/flinkx-restapi/flinkx-restapi-core/src/main/java/com/dtstack/flinkx/restapi/common/httprequestApi.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.common; - -import java.util.Map; - -/** - * httprequestApi - * - * @author by dujie@dtstack.com - * @Date 2020/9/25 - */ -public abstract class httprequestApi { - - abstract R execute(); - - - abstract void getHttpRequest(); - - public static class Httprequest { - private Map body; - private Map param; - private Map header; - - public Httprequest buildBody(Map body) { - this.body = body; - return this; - } - - public Httprequest buildParam(Map param) { - this.param = param; - return this; - } - - public Httprequest buildHeader(Map header) { - this.header = header; - return this; - } - - @Override - public String toString() { - return "Httprequest{" + - "body:" + body + - ", param:" + param + - ", header:" + header + - '}'; - } - - public Map getBody() { - return body; - } - - public Map getParam() { - return param; - } - - public Map getHeader() { - return header; - } - } -} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java index a3cc581dd5..e69de29bb2 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/HttpClient.java @@ -1,194 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.inputformat; - -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.restapi.common.HttpUtil; -import com.dtstack.flinkx.restapi.common.MapUtils; -import com.dtstack.flinkx.restapi.common.RestContext; -import com.dtstack.flinkx.restapi.common.exception.ReadRecordException; -import com.dtstack.flinkx.restapi.common.exception.ResponseRetryException; -import com.dtstack.flinkx.restapi.common.handler.DataHandler; -import com.dtstack.flinkx.restapi.common.httprequestApi; -import com.dtstack.flinkx.util.ExceptionUtil; -import com.dtstack.flinkx.util.GsonUtil; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.types.Row; -import org.apache.http.HttpEntity; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.util.EntityUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * httpClient - * - * @author by dujie@dtstack.com - * @Date 2020/9/25 - */ -public class HttpClient { - private static final Logger LOG = LoggerFactory.getLogger(HttpClient.class); - - private ScheduledExecutorService scheduledExecutorService; - protected transient CloseableHttpClient httpClient; - private final long intervalTime; - private BlockingQueue queue; - private RestContext restContext; - private static final String THREAD_NAME = "restApiReader-thread"; - private List metaColumns; - private List handlers; - - - public HttpClient(RestContext restContext, Long intervalTime) { - this.restContext = restContext; - this.intervalTime = intervalTime; - queue = new SynchronousQueue<>(false); - this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1, new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - return new Thread(r, THREAD_NAME); - } - }); - this.httpClient = HttpUtil.getHttpClient(); - } - - public void start() { - - scheduledExecutorService.scheduleAtFixedRate( - this::execute, - 0, - intervalTime, - TimeUnit.MILLISECONDS - ); - } - - public Row takeEvent() { - Row row = null; - try { - row = queue.take(); - } catch (InterruptedException e) { - LOG.error("takeEvent interrupted error:{}", ExceptionUtil.getErrorMessage(e)); - } - return row; - } - - public void execute() { - httprequestApi.Httprequest build = restContext.build(); - doExecute(build, 2); - restContext.updateValue(); - } - - public void doExecute(httprequestApi.Httprequest build, int retryTime) { - - HttpUriRequest request = HttpUtil.getRequest(restContext.getRequestType(), build.getBody(), build.getHeader(), restContext.getUrl()); - try { - CloseableHttpResponse httpResponse = httpClient.execute(request); - HttpEntity entity = httpResponse.getEntity(); - if (entity != null) { - String entityData = EntityUtils.toString(entity); - if (restContext.getFormat().equals("json")) { - Map map = HttpUtil.gson.fromJson(entityData, Map.class); - - if(CollectionUtils.isNotEmpty(handlers)){ - for (DataHandler handler : handlers) { - if (handler.isPipei(map)) { - handler.execute(map); - } - } - } - - if (CollectionUtils.isEmpty(metaColumns) || (metaColumns.size() == 1 && metaColumns.get(0).getName().equals(ConstantValue.STAR_SYMBOL))) { - queue.put(Row.of(map)); - } else { - HashMap stringObjectHashMap = new HashMap<>(); - for (MetaColumn metaColumn : metaColumns) { - String[] names = metaColumn.getName().split("\\."); - Map keyToMap = initData(stringObjectHashMap, names); - if (Objects.nonNull(keyToMap)) { - Object data = MapUtils.getData(map, names); - keyToMap.put(names[names.length - 1], data); - } - } - queue.put(Row.of(stringObjectHashMap)); - } - } else { - queue.put(Row.of(entityData)); - } - } else { - throw new RuntimeException("entity is null"); - } - } catch (ResponseRetryException e) { - //todo 重试 - if (--retryTime > 0) { - doExecute(build, retryTime); - } - } catch (Exception e) { - //todo 脏数据处理 - throw new ReadRecordException("get entity error"); - } - - } - - public void close() { - HttpUtil.closeClient(httpClient); - scheduledExecutorService.shutdown(); - } - - public Map initData(HashMap data, String[] names) { - Map tempHashMap = data; - for (int i = 0; i < names.length; i++) { - if (i != names.length - 1) { - HashMap objectObjectHashMap = new HashMap(4); - Object value = tempHashMap.putIfAbsent(names[i], objectObjectHashMap); - if(Objects.isNull(value)){ - tempHashMap = objectObjectHashMap; - }else if (value instanceof String) { - try { - Map o = GsonUtil.GSON.fromJson((String) value, GsonUtil.gsonMapTypeToken); - tempHashMap.put(names[i], o); - tempHashMap = o; - } catch (Exception e) { - return null; - } - } else if (value instanceof Map) { - tempHashMap = (Map) value; - } else { - return null; - } - } else { - tempHashMap.put(names[i], null); - } - } - return tempHashMap; - } - - public void setMetaColumns(List metaColumns) { - this.metaColumns = metaColumns; - } - - public void setHandlers(List handlers) { - this.handlers = handlers; - } -} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java deleted file mode 100644 index d093e35af8..0000000000 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormat.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.inputformat; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.restapi.common.ParamType; -import com.dtstack.flinkx.restapi.common.RestContext; -import com.dtstack.flinkx.restapi.common.handler.DataHandler; -import com.dtstack.flinkx.restapi.common.handler.DataHandlerFactory; -import com.dtstack.flinkx.restapi.reader.HttpRestConfig; -import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.types.Row; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -/** - * @author : tiezhu - * @date : 2020/3/12 - */ -public class RestapiInputFormat extends BaseRichInputFormat { - - protected HttpClient myHttpClient; - - protected RestContext restContext; - - protected Long intervalTime; - - protected HttpRestConfig httpRestConfig; - - protected List metaColumns ; - - protected List handlers; - - @Override - public void openInputFormat() throws IOException { - super.openInputFormat(); - } - - - - @Override - @SuppressWarnings("unchecked") - protected void openInternal(InputSplit inputSplit) throws IOException { - restContext = new RestContext(httpRestConfig.getType(),httpRestConfig.getUrl(),httpRestConfig.getFormat()); - myHttpClient = new HttpClient(restContext, intervalTime); - myHttpClient.setMetaColumns(metaColumns); - myHttpClient.setHandlers(handlers); - restContext.parseAndInt(httpRestConfig.getBody(), ParamType.BODY); - restContext.parseAndInt(httpRestConfig.getHeader(), ParamType.HEADER); - restContext.parseAndInt(httpRestConfig.getParam(), ParamType.PARAM); - - myHttpClient.start(); - } - - @Override - protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { - InputSplit[] inputSplits = new InputSplit[minNumSplits]; - for (int i = 0; i < minNumSplits; i++) { - inputSplits[i] = new GenericInputSplit(i, minNumSplits); - } - return inputSplits; - } - - @Override - protected Row nextRecordInternal(Row row) throws IOException { - return myHttpClient.takeEvent(); - - } - - @Override - protected void closeInternal() throws IOException { - myHttpClient.close(); - } - - @Override - public boolean reachedEnd() throws IOException { - return false; - } - - public void setHttpRestConfig(HttpRestConfig httpRestConfig) { - this.httpRestConfig = httpRestConfig; - } - - public void setIntervalTime(Long intervalTime) { - this.intervalTime = intervalTime; - } - - public void setMetaColumns(List metaColumns) { - this.metaColumns = metaColumns; - } - - public List getHandlers() { - return handlers; - } - - public void setHandlers(List handlers) { - this.handlers = handlers; - } -} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java deleted file mode 100644 index 48c6198715..0000000000 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/inputformat/RestapiInputFormatBuilder.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.restapi.inputformat; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; -import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.restapi.common.handler.DataHandler; -import com.dtstack.flinkx.restapi.reader.HttpRestConfig; -import org.apache.commons.lang3.StringUtils; - -import java.util.List; -import java.util.Map; - -/** - * @author : tiezhu - * @date : 2020/3/12 - */ -public class RestapiInputFormatBuilder extends BaseRichInputFormatBuilder { - protected RestapiInputFormat format; - - - public RestapiInputFormatBuilder() { - super.format = format = new RestapiInputFormat(); - } - - public void setHttpRestConfig(HttpRestConfig httpRestConfig) { - this.format.setHttpRestConfig(httpRestConfig); - } - - public void setIntervalTime(long intervalTime) { - this.format.setIntervalTime(intervalTime); - } - - public void setMetaColumns(List metaColumns) { - this.format.setMetaColumns(metaColumns); - } - - public void setHandlers(List handlers) { - this.format.setHandlers(handlers); - } - - @Override - protected void checkFormat() { - if(StringUtils.isBlank(format.httpRestConfig.getUrl())){ - throw new IllegalArgumentException("缺少url"); - } - if (StringUtils.isBlank(format.httpRestConfig.getType())) { - throw new IllegalArgumentException("缺少method"); - } - } -} diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java index a7f647e77e..dea949c36f 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/HttpRestConfig.java @@ -17,9 +17,13 @@ */ package com.dtstack.flinkx.restapi.reader; +import com.dtstack.flinkx.restapi.common.ConstantValue; +import com.dtstack.flinkx.restapi.common.MetaParam; +import com.dtstack.flinkx.restapi.client.Strategy; + import java.io.Serializable; +import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * HttpRestConfig @@ -28,55 +32,99 @@ * @Date 2020/9/28 */ public class HttpRestConfig implements Serializable { + + private static final long serialVersionUID = 1L; + + /** + * http协议 https/http + **/ + private String protocol = "https"; + + /** + * http请求地址 + **/ private String url; - private String type; + + /** + * http请求方式 post/get + **/ + private String requestMode; + + + /** + * 对返回值的处理 text/json + **/ + private String decode = "text"; + + /** + * decode为json时,指定解析的key + */ + private String fields; + + + /** + * 请求的间隔时间 单位毫秒 + **/ private Long intervalTime; - private List columns; - private String format; - private Map> header; - private Map> body; - private Map> param; - protected List handlers; - public String getType() { - return type; - } + /** + * 请求的header头 + **/ + private List header = new ArrayList<>(2); + + /** + * 请求的param + **/ + private List param = new ArrayList<>(2); - public void setType(String type) { - this.type = type; + + /** + * 请求的body + **/ + private List body = new ArrayList<>(2); + + /** + * 返回结果的处理策略 + **/ + protected List strategy = new ArrayList<>(2); + + + public boolean isJsonDecode() { + return getDecode().equalsIgnoreCase(ConstantValue.DEFAULT_DECODE); } - public List getColumns() { - return columns; + + public String getProtocol() { + return protocol; } - public void setColumns(List columns) { - this.columns = columns; + public void setProtocol(String protocol) { + this.protocol = protocol; } - public Map> getHeader() { - return header; + public String getUrl() { + return url; } - public void setHeader(Map> header) { - this.header = header; + public void setUrl(String url) { + this.url = url; } - public Map> getBody() { - return body; + public String getRequestMode() { + return requestMode; } - public void setBody(Map> body) { - this.body = body; + public void setRequestMode(String requestMode) { + this.requestMode = requestMode; } - public Map> getParam() { - return param; + public String getDecode() { + return decode; } - public void setParam(Map> param) { - this.param = param; + public void setDecode(String decode) { + this.decode = decode; } public Long getIntervalTime() { @@ -87,27 +135,60 @@ public void setIntervalTime(Long intervalTime) { this.intervalTime = intervalTime; } - public String getFormat() { - return format; + + public List getStrategy() { + return strategy; } - public void setFormat(String format) { - this.format = format; + public void setStrategy(List strategy) { + this.strategy = strategy; } - public String getUrl() { - return url; + public String getFields() { + return fields; } - public void setUrl(String url) { - this.url = url; + public void setFields(String fields) { + this.fields = fields; + } + + public List getHeader() { + return header; + } + + public void setHeader(List header) { + this.header = header; + } + + public List getParam() { + return param; + } + + public void setParam(List param) { + this.param = param; + } + + public List getBody() { + return body; } - public List getHandlers() { - return handlers; + public void setBody(List body) { + this.body = body; } - public void setHandlers(List handlers) { - this.handlers = handlers; + @Override + public String toString() { + return "HttpRestConfig{" + + "protocol='" + protocol + '\'' + + ", url='" + url + '\'' + + ", requestMode='" + requestMode + '\'' + + ", decode='" + decode + '\'' + + ", fields='" + fields + '\'' + + ", intervalTime=" + intervalTime + + ", header=" + header + + ", param=" + param + + ", body=" + body + + ", strategy=" + strategy + + '}'; } } diff --git a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java index be5b719ef9..591a474208 100644 --- a/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java +++ b/flinkx-restapi/flinkx-restapi-reader/src/main/java/com/dtstack/flinkx/restapi/reader/RestapiReader.java @@ -20,19 +20,17 @@ import com.dtstack.flinkx.config.DataTransferConfig; import com.dtstack.flinkx.config.ReaderConfig; import com.dtstack.flinkx.reader.BaseDataReader; -import com.dtstack.flinkx.reader.MetaColumn; -import com.dtstack.flinkx.restapi.common.InnerVaribleFactory; -import com.dtstack.flinkx.restapi.common.IntervalTimeVarible; -import com.dtstack.flinkx.restapi.common.handler.DataHandlerFactory; -import com.dtstack.flinkx.restapi.inputformat.RestapiInputFormatBuilder; +import com.dtstack.flinkx.restapi.common.ConstantValue; +import com.dtstack.flinkx.restapi.common.HttpMethod; +import com.dtstack.flinkx.restapi.common.MetaParam; +import com.dtstack.flinkx.restapi.common.ParamType; +import com.dtstack.flinkx.restapi.format.RestapiInputFormatBuilder; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.Row; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import java.util.Collections; /** * @author : tiezhu @@ -40,13 +38,11 @@ */ public class RestapiReader extends BaseDataReader { + /** + * http的请求参数 + **/ private HttpRestConfig httpRestConfig; - private List metaColumns; - - protected Long intervalTime; - - protected List handlers; @SuppressWarnings("unchecked") public RestapiReader(DataTransferConfig config, StreamExecutionEnvironment env) { @@ -56,37 +52,44 @@ public RestapiReader(DataTransferConfig config, StreamExecutionEnvironment env) try { this.httpRestConfig = objectMapper.readValue(objectMapper.writeValueAsString(readerConfig.getParameter().getAll()), HttpRestConfig.class); } catch (Exception e) { - throw new RuntimeException("解析httpRest Config配置出错:", e); + throw new RuntimeException("analyze httpRest Config failed:", e); } - metaColumns = MetaColumn.getMetaColumns(readerConfig.getParameter().getColumn()); - List handlers = httpRestConfig.getHandlers(); - if (CollectionUtils.isNotEmpty(handlers)) { - handlers = new ArrayList(handlers.size() * 2); - for (Object handlerParam : handlers) { - try { - handlers.add(DataHandlerFactory.getDataHandler((Map) handlerParam)); - } catch (Exception e) { - throw new IllegalArgumentException("dataHandler param error" + httpRestConfig.getHandlers()); - } + + MetaParam.setMetaColumnsType(httpRestConfig.getBody(), ParamType.BODY); + MetaParam.setMetaColumnsType(httpRestConfig.getParam(), ParamType.PARAM); + MetaParam.setMetaColumnsType(httpRestConfig.getHeader(), ParamType.HEADER); + + MetaParam.initTimeFormat(httpRestConfig.getBody()); + MetaParam.initTimeFormat(httpRestConfig.getParam()); + MetaParam.initTimeFormat(httpRestConfig.getHeader()); + + //post请求 如果contentTy没有设置,则默认设置为 application/json + if(HttpMethod.POST.name().equalsIgnoreCase(httpRestConfig.getRequestMode()) && httpRestConfig.getHeader().stream().noneMatch(i->ConstantValue.CONTENT_TYPE_NAME.equals(i.getKey()))){ + if(CollectionUtils.isEmpty(httpRestConfig.getHeader())){ + httpRestConfig.setHeader( Collections.singletonList(new MetaParam(ConstantValue.CONTENT_TYPE_NAME, ConstantValue.CONTENT_TYPE_DEFAULT_VALUE, ParamType.HEADER))); + }else{ + httpRestConfig.getHeader().add(new MetaParam(ConstantValue.CONTENT_TYPE_NAME, ConstantValue.CONTENT_TYPE_DEFAULT_VALUE, ParamType.HEADER)); } - DataHandlerFactory.destroy(); } - InnerVaribleFactory.addVarible("intervalTime", new IntervalTimeVarible(httpRestConfig.getIntervalTime())); - intervalTime = httpRestConfig.getIntervalTime(); - } @Override public DataStream readData() { RestapiInputFormatBuilder builder = new RestapiInputFormatBuilder(); builder.setDataTransferConfig(dataTransferConfig); - builder.setIntervalTime(intervalTime); - builder.setMetaColumns(metaColumns); - builder.setHandlers(handlers); + + builder.setMetaHeaders(httpRestConfig.getHeader()); + builder.setMetaParams(httpRestConfig.getParam()); + builder.setMetaBodys(httpRestConfig.getBody()); builder.setHttpRestConfig(httpRestConfig); + builder.setStream(restoreConfig.isStream()); + builder.setDataTransferConfig(dataTransferConfig); + builder.setRestoreConfig(restoreConfig); + builder.setMonitorUrls(monitorUrls); + builder.setBytes(bytes); return createInput(builder.finish()); } } diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index 43a3d1c744..1f3dfbd0ac 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -400,6 +400,12 @@ 1.6 + + com.dtstack.flinkx + flinkx-sqlservercdc-reader + 1.6 + + org.apache.hadoop hadoop-hdfs @@ -430,12 +436,16 @@ flinkx-emqx-writer 1.6 + + com.dtstack.flinkx + flinkx-restapi-reader + 1.6 + com.dtstack.flinkx flinkx-restapi-writer 1.6 - com.dtstack.flinkx flinkx-dm-reader diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 4e2a2eb175..3847df71fc 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -75,8 +75,10 @@ import com.dtstack.flinkx.postgresql.writer.PostgresqlWriter; import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.redis.writer.RedisWriter; +import com.dtstack.flinkx.restapi.reader.RestapiReader; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; +import com.dtstack.flinkx.sqlservercdc.reader.SqlservercdcReader; import com.dtstack.flinkx.stream.reader.StreamReader; import com.dtstack.flinkx.stream.writer.StreamWriter; import com.dtstack.flinkx.util.ResultPrintUtil; @@ -225,6 +227,8 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstants.PHOENIX5_READER : reader = new Phoenix5Reader(config, env); break; case PluginNameConstants.KINGBASE_READER : reader = new KingbaseReader(config, env); break; case PluginNameConstants.ORACLE_LOG_MINER_READER : reader = new OraclelogminerReader(config, env); break; + case PluginNameConstants.RESTAPI_READER: reader = new RestapiReader(config, env); break; + case PluginNameConstants.SQLSERVER_CDC_READER: reader = new SqlservercdcReader(config, env); break; default:throw new IllegalArgumentException("Can not find reader by name:" + readerName); } @@ -263,6 +267,7 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ case PluginNameConstants.GREENPLUM_WRITER : writer = new GreenplumWriter(config); break; case PluginNameConstants.PHOENIX5_WRITER : writer = new Phoenix5Writer(config); break; case PluginNameConstants.KINGBASE_WRITER : writer = new KingbaseWriter(config); break; + case PluginNameConstants.RESTAPI_WRITER: writer = new RedisWriter(config); break; default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index 472cf2623e..c3a3dcd36a 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -40,6 +40,7 @@ public class PluginNameConstants { public static final String GBASE_READER = "gbasereader"; public static final String KUDU_READER = "kudureader"; public static final String BINLOG_READER = "binlogreader"; + public static final String SQLSERVER_CDC_READER = "sqlservercdcreader"; public static final String KAFKA09_READER = "kafka09reader"; public static final String KAFKA10_READER = "kafka10reader"; public static final String KAFKA11_READER = "kafka11reader"; @@ -53,6 +54,8 @@ public class PluginNameConstants { public static final String GREENPLUM_READER = "greenplumreader"; public static final String PHOENIX5_READER = "phoenix5reader"; public static final String KINGBASE_READER = "kingbasereader"; + public static final String RESTAPI_READER = "restapireader"; + public static final String STREAM_WRITER = "streamwriter"; public static final String CARBONDATA_WRITER = "carbondatawriter"; diff --git a/pom.xml b/pom.xml index a0d5e8442a..e31f4db715 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ flinkx-pulsar flinkx-pgwal flinkx-restapi + flinkx-sqlservercdc flinkx-oraclelogminer @@ -128,7 +129,7 @@ junit junit - 4.13.1 + 4.12 test From c2dfd60103da04550097168e348907dc6254fdb8 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 2 Mar 2021 17:28:51 +0800 Subject: [PATCH 104/136] add sqlserverCDC Modules and docs --- .../MetadataoracleInputFormat.java | 202 ---------------- .../flinkx/metadata/util/ConnUtil.java | 91 -------- .../inputformat/BaseMetadataInputFormat.java | 218 ------------------ 3 files changed, 511 deletions(-) delete mode 100644 flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java delete mode 100644 flinkx-metadata/flinkx-metadata-core/src/main/java/com/dtstack/flinkx/metadata/util/ConnUtil.java delete mode 100644 flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java diff --git a/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java b/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java deleted file mode 100644 index ae3469c0f0..0000000000 --- a/flinkx-metadata-oracle/flinkx-metadata-oracle-reader/src/main/java/com/dtstack/flinkx/metadataoracle/inputformat/MetadataoracleInputFormat.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.dtstack.flinkx.metadataoracle.inputformat; - -import com.dtstack.flinkx.constants.ConstantValue; -import com.dtstack.flinkx.metadata.inputformat.BaseMetadataInputFormat; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import static com.dtstack.flinkx.metadata.MetaDataCons.MAX_TABLE_SIZE; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_COLUMN; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_COLUMN_COMMENT; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_COLUMN_INDEX; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_COLUMN_NAME; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_COLUMN_TYPE; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_CREATE_TIME; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_INDEX_COLUMN_NAME; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_ROWS; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_TABLE_PROPERTIES; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_TABLE_TYPE; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.KEY_TOTAL_SIZE; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_COLUMN; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_COLUMN_TOTAL; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_INDEX; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_INDEX_TOTAL; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_TABLE_PROPERTIES; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_QUERY_TABLE_PROPERTIES_TOTAL; -import static com.dtstack.flinkx.metadataoracle.constants.OracleMetaDataCons.SQL_SHOW_TABLES; - -/** - * @author : kunni@dtstack.com - * @date : 2020/6/9 - * @description :Oracle元数据可在系统表中查询 - */ - -public class MetadataoracleInputFormat extends BaseMetadataInputFormat { - - private static final long serialVersionUID = 1L; - - private Map> tablePropertiesMap; - - private Map>> columnListMap; - - private Map>> indexListMap; - - private String allTable; - - private String sql; - - @Override - protected List showTables() throws SQLException { - List tableNameList = new LinkedList<>(); - String sql = String.format(SQL_SHOW_TABLES, quote(currentDb.get())); - try (ResultSet rs = statement.get().executeQuery(sql)) { - while (rs.next()) { - tableNameList.add(rs.getString(1)); - } - } - return tableNameList; - } - - @Override - protected void switchDatabase(String databaseName) { - currentDb.set(databaseName); - } - - @Override - protected String quote(String name) { - return String.format("'%s'",name); - } - - @Override - protected Map queryMetaData(String tableName) { - Map result = new HashMap<>(16); - Map tableProperties = tablePropertiesMap.get(tableName); - List> columnList = columnListMap.get(tableName); - List> indexList = indexListMap.get(tableName); - result.put(KEY_TABLE_PROPERTIES, tableProperties); - result.put(KEY_COLUMN, columnList); - result.put(KEY_COLUMN_INDEX, indexList); - return result; - } - - Map > queryTableProperties() throws SQLException { - Map> tablePropertiesMap = new HashMap<>(16); - if(StringUtils.isBlank(allTable)){ - sql = String.format(SQL_QUERY_TABLE_PROPERTIES_TOTAL, quote(currentDb.get())); - }else { - sql = String.format(SQL_QUERY_TABLE_PROPERTIES, quote(currentDb.get()), allTable); - } - try (ResultSet rs = statement.get().executeQuery(sql)) { - while (rs.next()) { - Map map = new HashMap<>(16); - map.put(KEY_TOTAL_SIZE, rs.getString(1)); - map.put(KEY_COLUMN_COMMENT, rs.getString(2)); - map.put(KEY_TABLE_TYPE, rs.getString(3)); - map.put(KEY_CREATE_TIME, rs.getString(4)); - map.put(KEY_ROWS, rs.getString(5)); - tablePropertiesMap.put(rs.getString(6), map); - } - } - return tablePropertiesMap; - } - - Map>> queryIndexList() throws SQLException { - Map>> indexListMap = new HashMap<>(16); - if(StringUtils.isBlank(allTable)){ - sql = String.format(SQL_QUERY_INDEX_TOTAL, quote(currentDb.get())); - }else { - sql = String.format(SQL_QUERY_INDEX, quote(currentDb.get()), allTable); - } - try (ResultSet rs = statement.get().executeQuery(sql)) { - while (rs.next()) { - Map column = new HashMap<>(16); - column.put(KEY_COLUMN_NAME, rs.getString(1)); - column.put(KEY_INDEX_COLUMN_NAME, rs.getString(2)); - column.put(KEY_COLUMN_TYPE, rs.getString(3)); - String tableName = rs.getString(4); - if(indexListMap.containsKey(tableName)){ - indexListMap.get(tableName).add(column); - }else { - List> indexList = new LinkedList<>(); - indexList.add(column); - indexListMap.put(tableName, indexList); - } - } - } - return indexListMap; - } - - Map>> queryColumnList() throws SQLException { - Map>> columnListMap = new HashMap<>(16); - if(StringUtils.isBlank(allTable)){ - sql = String.format(SQL_QUERY_COLUMN_TOTAL, quote(currentDb.get())); - }else { - sql = String.format(SQL_QUERY_COLUMN, quote(currentDb.get()), allTable); - } - try (ResultSet rs = statement.get().executeQuery(sql)) { - while (rs.next()) { - Map column = new HashMap<>(16); - column.put(KEY_COLUMN_NAME, rs.getString(1)); - column.put(KEY_COLUMN_TYPE, rs.getString(2)); - column.put(KEY_COLUMN_COMMENT, rs.getString(3)); - String tableName = rs.getString(4); - if(columnListMap.containsKey(tableName)){ - column.put(KEY_COLUMN_INDEX, CollectionUtils.size(columnListMap.get(tableName))+1); - columnListMap.get(tableName).add(column); - }else { - List>columnList = new LinkedList<>(); - column.put(KEY_COLUMN_INDEX, 1); - columnList.add(column); - columnListMap.put(tableName, columnList); - } - } - } - return columnListMap; - } - - @Override - protected void init() throws SQLException { - allTable = null; - if(tableList==null || tableList.size()==0){ - return; - } - StringBuilder stringBuilder = new StringBuilder(2 * tableList.size()); - if(tableList.size() <= MAX_TABLE_SIZE){ - for(int index=0;index - * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.metadata.util; - -import com.dtstack.flinkx.util.ClassUtil; -import com.dtstack.flinkx.util.SysUtil; -import com.dtstack.flinkx.util.TelnetUtil; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; - -/** - * @author : tiezhu - * @date : 2020/3/8 - */ -public class ConnUtil { - - /** - * 数据库连接的最大重试次数 - */ - private static int MAX_RETRY_TIMES = 3; - - /** - * 获取jdbc连接(超时10S) - * @param url url - * @param username 账号 - * @param password 密码 - * @return - * @throws SQLException - */ - private static Connection getConnectionInternal(String url, String username, String password) throws SQLException { - Connection dbConn; - synchronized (ClassUtil.LOCK_STR){ - DriverManager.setLoginTimeout(10); - // telnet - TelnetUtil.telnet(url); - - if (username == null) { - dbConn = DriverManager.getConnection(url); - } else { - dbConn = DriverManager.getConnection(url, username, password); - } - } - - return dbConn; - } - - /** - * 获取jdbc连接(重试3次) - * @param url url - * @param username 账号 - * @param password 密码 - * @return - * @throws SQLException - */ - public static Connection getConnection(String url, String username, String password) throws SQLException { - boolean failed = true; - Connection dbConn = null; - for (int i = 0; i < MAX_RETRY_TIMES && failed; ++i) { - try { - dbConn = getConnectionInternal(url, username, password); - failed = false; - } catch (Exception e) { - if (i == MAX_RETRY_TIMES - 1) { - throw e; - } else { - SysUtil.sleep(3000); - } - } - } - - return dbConn; - } -} diff --git a/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java b/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java deleted file mode 100644 index ec30d06544..0000000000 --- a/flinkx-metadata/flinkx-metadata-reader/src/main/java/com/dtstack/flinkx/metadata/inputformat/BaseMetadataInputFormat.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.flinkx.metadata.inputformat; - -import com.dtstack.flinkx.inputformat.BaseRichInputFormat; -import com.dtstack.flinkx.metadata.MetaDataCons; -import com.dtstack.flinkx.metadata.util.ConnUtil; -import com.dtstack.flinkx.util.ExceptionUtil; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.types.Row; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -/** - * @author : tiezhu - * @date : 2020/3/20 - */ -public abstract class BaseMetadataInputFormat extends BaseRichInputFormat { - - private static final long serialVersionUID = 1L; - - protected String dbUrl; - - protected String username; - - protected String password; - - protected String driverName; - - protected boolean queryTable; - - protected List> dbTableList; - - protected List tableList; - - protected static transient ThreadLocal connection = new ThreadLocal<>(); - - protected static transient ThreadLocal statement = new ThreadLocal<>(); - - protected static transient ThreadLocal currentDb = new ThreadLocal<>(); - - protected static transient ThreadLocal> tableIterator = new ThreadLocal<>(); - - @Override - protected void openInternal(InputSplit inputSplit) throws IOException { - try { - connection.set(getConnection()); - statement.set(connection.get().createStatement()); - currentDb.set(((MetadataInputSplit) inputSplit).getDbName()); - switchDatabase(currentDb.get()); - tableList = ((MetadataInputSplit) inputSplit).getTableList(); - if (CollectionUtils.isEmpty(tableList)) { - tableList = showTables(); - queryTable = true; - } - LOG.info("current database = {}, tableSize = {}, tableList = {}",currentDb.get(), tableList.size(), tableList); - tableIterator.set(tableList.iterator()); - init(); - } catch (ClassNotFoundException e) { - LOG.error("could not find suitable driver, e={}", ExceptionUtil.getErrorMessage(e)); - throw new IOException(e); - } catch (SQLException e){ - LOG.error("获取table列表异常, dbUrl = {}, username = {}, inputSplit = {}, e = {}", dbUrl, username, inputSplit, ExceptionUtil.getErrorMessage(e)); - tableList = new LinkedList<>(); - } - LOG.info("curentDb = {}, tableList = {}", currentDb.get(), tableList); - tableIterator.set(tableList.iterator()); - } - - /** - * 按照database进行划分,可能与channel数不同 - * @param splitNumber 最小分片数 - * @return 分片 - */ - @Override - @SuppressWarnings("unchecked") - protected InputSplit[] createInputSplitsInternal(int splitNumber) { - InputSplit[] inputSplits = new MetadataInputSplit[dbTableList.size()]; - for (int index = 0; index < dbTableList.size(); index++) { - Map dbTables = dbTableList.get(index); - String dbName = MapUtils.getString(dbTables, MetaDataCons.KEY_DB_NAME); - if(StringUtils.isNotEmpty(dbName)){ - List tables = (List)dbTables.get(MetaDataCons.KEY_TABLE_LIST); - inputSplits[index] = new MetadataInputSplit(splitNumber, dbName, tables); - } - } - return inputSplits; - } - - @Override - protected Row nextRecordInternal(Row row) throws IOException{ - Map metaData = new HashMap<>(16); - metaData.put(MetaDataCons.KEY_OPERA_TYPE, MetaDataCons.DEFAULT_OPERA_TYPE); - - String tableName = (String) tableIterator.get().next(); - metaData.put(MetaDataCons.KEY_SCHEMA, currentDb.get()); - metaData.put(MetaDataCons.KEY_TABLE, tableName); - - try { - metaData.putAll(queryMetaData(tableName)); - metaData.put(MetaDataCons.KEY_QUERY_SUCCESS, true); - } catch (Exception e) { - metaData.put(MetaDataCons.KEY_QUERY_SUCCESS, false); - metaData.put(MetaDataCons.KEY_ERROR_MSG, ExceptionUtil.getErrorMessage(e)); - LOG.error(ExceptionUtil.getErrorMessage(e)); - } - LOG.info("query metadata: {}", metaData); - return Row.of(metaData); - } - - @Override - public boolean reachedEnd() { - return !tableIterator.get().hasNext(); - } - - @Override - protected void closeInternal() throws IOException { - tableIterator.remove(); - Statement st = statement.get(); - if (null != st) { - try { - st.close(); - statement.remove(); - } catch (SQLException e) { - LOG.error("close statement failed, e = {}", ExceptionUtil.getErrorMessage(e)); - throw new IOException("close statement failed", e); - } - } - - currentDb.remove(); - Connection conn = connection.get(); - if (null != conn) { - try { - conn.close(); - connection.remove(); - } catch (SQLException e) { - LOG.error("close database connection failed, e = {}", ExceptionUtil.getErrorMessage(e)); - throw new IOException("close database connection failed", e); - } - } - } - - @Override - public void closeInputFormat() throws IOException { - super.closeInputFormat(); - } - - /** - * 创建数据库连接 - */ - public Connection getConnection() throws SQLException, ClassNotFoundException { - Class.forName(driverName); - return ConnUtil.getConnection(dbUrl, username, password); - } - - /** - * 查询当前数据库下所有的表 - * - * @return 表名列表 - * @throws SQLException 异常 - */ - protected abstract List showTables() throws SQLException; - - /** - * 切换当前database - * - * @param databaseName 数据库名 - * @throws SQLException 异常 - */ - protected abstract void switchDatabase(String databaseName) throws SQLException; - - /** - * 根据表名查询元数据信息 - * @param tableName 表名 - * @return 元数据信息 - * @throws SQLException 异常 - */ - protected abstract Map queryMetaData(String tableName) throws SQLException; - - /** - * 将数据库名,表名,列名字符串转为对应的引用,如:testTable -> `testTable` - * @param name 入参 - * @return 返回数据库名,表名,列名的引用 - */ - protected abstract String quote(String name); - - /** - * 提供子类对新增成员变量初始化的接口 - */ - protected void init() throws SQLException {} - -} From 32d0050d80670619f89bcf29007e0a51caedeb32 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 2 Mar 2021 17:48:11 +0800 Subject: [PATCH 105/136] [fix]replace job path --- .../src/main/java/com/dtstack/flinkx/test/LocalTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 20b074564b..3847df71fc 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -127,7 +127,7 @@ public static void main(String[] args) throws Exception{ // conf.setString("metrics.reporter.promgateway.randomJobNameSuffix","true"); // conf.setString("metrics.reporter.promgateway.deleteOnShutdown","true"); - String jobPath = "/Users/yanghuai/Library/Application Support/JetBrains/IntelliJIdea2020.3/scratches/实时/sqlservercdc_stream.json"; + String jobPath = "your json file's absolute path"; JobExecutionResult result = LocalTest.runJob(new File(jobPath), confProperties, null); ResultPrintUtil.printResult(result); System.exit(0); From 84e75a5e34258a54e5e7e9eb21d64806ad8c604a Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 2 Mar 2021 18:03:34 +0800 Subject: [PATCH 106/136] =?UTF-8?q?[fix]update=20sqlserverCDC=E5=8E=9F?= =?UTF-8?q?=E7=90=86.md=20docs?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/images/SqlserverCDC/Sqlserver15.png | Bin 0 -> 153405 bytes .../SqlserverCDC\345\216\237\347\220\206.md" | 22 ++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 docs/images/SqlserverCDC/Sqlserver15.png diff --git a/docs/images/SqlserverCDC/Sqlserver15.png b/docs/images/SqlserverCDC/Sqlserver15.png new file mode 100644 index 0000000000000000000000000000000000000000..3345aaadda022fc136c6f24c7a5c9ab783b60cce GIT binary patch literal 153405 zcmeFZcT^P3*DfkJ%77>gh#;9E3lc=ZA*(}95|o@Ii6jBZNd#obQG$d)5D_J3kR&2F zL=hwh$wHu}sNm;?Mgxbri*h5pPL#xrv0I~uNdizyer+o_Kitt~?f>fAY2%lp4bJ}C-M`=CyM|AMQyniV`k&+BV5vF(zYS3X zb~!qu!OZNx4IH#|U)KHNf64-Cy#76$gGDBwwdH?)GA(kG`X8bF?FFN+D&oFuI8pp7 z*8d?@Nv7Yo7<~!ZR(B>ND9eub>y8&TdzM{@Mxr1g`Kpq;P6V+(p*c#qC3o`CDbS8G zy_E;GB!01Z`!!v=8m9s8{iW7RB8&IS`X7bdu;@s>VbT=ZfD*dudJ*~yJ)}-RIhr2| z#rbFsj9Fm4=8r8ezdE}kp%)0mttbqH`Ifo-%N=9^jx8xx?9XuVV%d=Gbi19mbEctW9yQ;JL?u09X_NxNKpcAWGC?Qr8L-<$yw zekw~^q-S)12N{(MK82?RpGaN!Y!d+@kz=~T<-;9lKF{8L?xVi3GXK$F>w`nT zap11q!t=J8FR{;I?^1#G&Cg1byyM5SwZ4R!itK;~-HwcQQLbP<(Y_31eO<;9iGyfN zYWz}YOXTS+8YzOlnlzX*6NZM?ze}>!xg@$;uah|IEW5kqX?i7x_2k_zh;JkgQ`R(g z)%F(WWV!ePt*jDHbWzj`Xl7RD{R?DTNW9d6pvX3*PBJc!yC+)RgnsU07$iG0HpG9m zFzV#_f7gg_7o&3D@cPozl?e(0XGup;Ds1zyZ{73BnN8|guhoXe1g*3AEl zUGZH93#BhtDk=iUd`T7;_^~_AX{DP-_fO?!mwxTrNm~Yo(Bl!#*}?T58k@TQ$3xGC z!u4B*+)FZYK6!iUsJV6g(d}?O2sf>3e|-d?>`+;fX<$X*{jr!5xajWAa8y@hM8but zr0$nnC0k@D4ds0rjSjaI(OK(RB3~MqkQvgnl5iS4($T$o&!e{V)uB|#6|?g2#Jz#r zpS`W(Z-g^=ZFVVtx-;M#aM$6!`H(&MNX=c)HkZH~hVEz}Q61%ve$*EDUkJZ6(|cHB z?%(BLxu_{IJ6KKOlKbn)IUU{O>}_2{eAUj6!FSbLy!$q(ht7(HGRkgNF^46V3#Cq< z7F_xiMp(B{dWYWEO!SCw-mm$+UI9Hm3uf+Ff>iVOVxXB~BDaKlnPED{epF8)?D?CQ zDj(hqGo|XMvPtUYLGpN)OpomUwy3BYxm&RBdRsTPbgLjqysCLaqq$#xfBax!xo#}{ zYRuiDF>@s!Rz_cM_O!^wKUaH>Q`x}9@84aOoNUi`g6;=PHn6s+r6xPfmv2eWdvhft zD|9|76y5t|ZIiO2)N?cY6a99#9Ww2=h)uLhxlQZPp)D3=Yg2@s+^k_4%BI{uOchWL zFseFI1t5`{n>2ej zmVh2-jl>G45i*h@0^stoB=$Yaj*2y1M49~^0`V1C`1s?Ock67%)~*ZAWo_jz2{|Em zqO(Mu92F(5@|yiF&uyl9l=W$>BtmSx$8xD`cHaxT#>ukpb!E#CpKMMDENbs0{=+3M za8di3Cnd!cA0a!^u>2;F{u*|-&7vj3=U_5CA=C#K8~&${$zDb-_{GhW|W+sp6Tl3msBck62r zKGlOk`qX9n{AiESYU(%rDcyf8vr-;kDP%x}5A; zM^C-Vow{8HofxvJ+tmX_#k)Fx>?w#XHfBx5+SlfG(`Om1myMMv9X#<;qB(K~mgm#9 z#)!)P)Mez%(Poy!E02|$a@$4b?dYn2`SUc^_m4ItjvX<_Z36+623kB7zg{?Wpk`wQ zI&NnqyU%A&>#w+LqSn7Lw*sT6Y2J~n{ZQZOXVI`j>npQ3?RT~^3|`Nc*u1kky|xFk zME3_ec}T=}jA{(>>W=nwj$d2lWj|QuoV4IB>Ey?EA25r*K8yA7-r}FW`Ii30Dyq{> z88v}n7Ph$^m;#?6pm$9p(r;W~3ZzvN9Ce+1vQdAQgnMkSx4fl8WJRxR%zM8j!Lymp zC8v3@`O5ACP5lR>;K#VbKVx;Yo~xhD#v1~r)BG-7>o=BY9>}8KUvX^~`H+I!BQepG zRlZ`g54ixVypRtBBUEJc4Z0Ft=@#~sDS9KWygGBSw9mk-Q|wR9{_yvr>skeA^jN1} z?X8)Zq=rA$vtmx0!|&AsIS8`%n?!_`dK9ozhPy4AA4ckq4_EHl`uoKu4S4+-DIe7$ zi+ZDE`z(87^$ST~(l}(%$@PN{hec<177UNw-IlBjbsy^5l)5tm2`y0CP8_cWO7zgo z^~zYgl&jPk2Sk2zom}DdUJIFD;WAhY9&FaJ%6(v$W>>w|G$=FJ54pFJ#JJDG7~~1ZPoJ|2j8)(a!bW z?$Mt*m7l_SJAc5Ms%%RUDYxA$24EXI;yjegR|gmDSgGA#E)De@t#&KDRMtQc6y%@jsPYqA!+h>^yW)zn-6+PpIAd zng5UmxNgpY=k$R4qWMZ)8I{@b*+OD}z&M}D^r9>7S+3ZtkwD!KTBZX|_eR}qP5lCn~)U*Gp^JgpH$BQDC z*K_>B0>97!JRU=2U9wNx{P0uVdJ9+I+9T`+MylzB1ONh?-H79z2={xTRB4Lk#6-{L|p3a_bY{a?}TcyMUQtoS2jZuEXlVK+8DA% zqAqTK!e;A5_3+hu_4lQdT6ycLP0lj~aonq4^E=l-VcfX>Gs&*D@&G{8^#j?}V(ei~ zaM7(^tqh`D9ZYFo2>dXD$mFN@sMdu7RM4mxj@j4u+L+qp5?%64{Fx??r`RR55Tk3V-#Dy-V7mEQF`({lPS^Pqy=*75q)JQ%7lFzjV&HNExCBBitw^L2DmM zI@?U=$s~M%WoIYw`t`rFY;*b??~fVU ziB?`^w-<=tD@}`~)@MR~Y3Eir=a@>fkD*F8tdvty_L!d#uo?k|7JFJqTi ze3XMh&0JPh{KV1`wG5&rVP_Z{p)OqWAfs;cD=JmttIPabF@&6F2mglB&~g~cf?f`@ z^HbBXW?7}zXH}aCrCqWl1G)2cz{jZ}W8w3KvFXUpoEH#^gkQIXA);+sb}=-A_T-`< zKvcekbZ~Pc5X$c}@URJMdg^%imDjsaTGM;gkD-S{Y0qoLV@%hM!{^VZv7a5m4GNv3}s)6>%hrkL} z?G}SluAjk=~==-Dvdc6L!0EsrtwnHv6SrC2< zgU`X*gUG_1?Yg6dx|bLnW_c&#z6@2l#KF@rQs1enX)_VI9^>EF70eN@$Nk@%3~;LI z%@DrhW}3A_MajT~ke5FzZ6+|pQ+YJ`Ur!G~#~R-mg4@Rx9PM`cun=@-$g&BGN%&tebJ&vfoIZ^R`tKk>T=yU_6bMmoI>7BXvAyIjDFaQGp| z{_{}Zc_33gaNBx~-DWMIc8zjBl#9lR)QEdbLBuwPIe67?$&A-S+wHf4p|EKs-8Sj) zZ+0g)a*L$75E@goY23j$yTVPx1xuOmUw7srA2%xV<9H6D;=5#Y5{@XAB^ z6CB>)tsZ{q1yf3o%vNDwX;NfBcSaWkcvVDgAgP*y>VRp!m!gmw!(uO9HbK5r$>uU> zndKht&`8aTdt*zLMThFVzMgE{Qa~qB;axG)&F)0{2F{;{Hxp?8Lp-RA#h(IU9hZB< zkYT^SxkW~NJ!;SIyaF0wy(5k4)JEpue0IJt^lqUB{1e#cD7o( z23_*G$1V1iJx_v+cxV4!0`q0dz;BPVZ%fECMr+%K&IyN?IKKEkC0of=d`Sg>1 znSvjN293TuX`z(vPg6PcZG(d*8YMIPcrAozQF^83rkYcI}xj`hdxG>3@?y}xQhS{F+3M)_@IQGE3j z>jRGa5oTE>8OauN31ch=?OF(~9_HvPV-7s1rG`k_+WzUv>Jm9+{cQw)53U%q$c%AR zGX6B=JtVUzb(1{k-Z!Rx&J3OxQ~u=2-wK4wM9yXQyyuK6LbtB^eM6RMZ3nM1O~0OM zCQhwX431cfm%Y)+-K04au;8~5yC(FCfta*iE;MihbJJ6@f_b?=fdO`HQVs>y=aIqV zkVC%l6$+Yz{(IU@FYk{%&ItP;%Tg4QHbsy^t|D64!H=a48tZ2k#?bXg8`Y0p>-1h< zIoJ1a)VY5iUn;4i-&L4GTj}DO*$wG_1c>`PkHCw5S+cKk&2WU-7Pyjtlpu^tc%yWEOai`wVs@k7J{G9~SyIL;*#(?D8`F!ymsbI{$f-h$$r13O_zk-;k27(Q|R}*#8H@4muPc0>1Y(|-SDT@8Mp&|=L>T4>z-_rj4zT7?K~%Gw%jM|{3<62 zo(pD)bN2C?FwZv+7hg^e+hw1vl=l~mdMMbkWA9ss<<`7N)6^uovCh~+>hg44$Ku}O z^@pZ;j_GO1voWj%EM4?v)sn$h{tMA{x*cC)!`2teo3+^#>o6L~)SU*hM=<2lug@eJ z++wrT<7Qe1PaCq-wCp8&yl#+BK9E&rv)_*f5#338xZWS~wiA;&h+oXsR4`4wdz~b0 z`=Z#M7Oz1`)8Et|wuIp}ikT%unY$U0O%H!l>j)DQB;^~{n0XnCVVSm;TkIxSa-+Gn zl&x}%mX=*(viYVZ~3MYMHWGvi(Q;lU(6D>QSTib_e8Tzqs!d#NPO>l<>_lNmknN~0V<~SEVdDa z?1amaAt~!f7p@w{l{tueubG_dBRf=;&<#h)Xgjnd3bZ_8K6)zlmTtCKCH1iP%07ly ztx39LOrXnv%u#Vn@7THoSB#ByY>%EeyU%UcP9m!PFl==)5Ggpwe6SX@ZQbU9jOb@X z;%+JG=bdX5ZvBXeKd67~KCSbT6O-`)60eQz&@SUO*9!7e3 zBT+)dlE}us;YNo^+(j{=a-en_m8zW;4jDn;#vK_-GD4H&{OPxM^zel~45OnR99tEH zF;AZg$W6SAa<_I9yuqNV@(C;vIkC9%qJ!9^G0{M2{kZ49$=VMyJM zGODvz>TOdzdQ8g!w^Ja+!me7(Z4zpeI@>7N+&`PivwHWNs?gFLdv!Pa7OA=(CAXh7 zs+pmMlzp{ca8jD_muBim>!nOO1^Nw3I9hyhT(QC2Q_d^hnEPj=9a&qD`^H(=BWk94 zaUUV{%FY!g@zt|uCL*jtH$Y6N9fs~E2%6)GstpapJR2=)ajV6_Keh388Z~^{BJovu z8&Ssuqm`g4cy@VI)I?uw2eqlVY-`8(moonzKy+f)RtM!R@H8wYUol%NFhcM0*?mFTiqRx{cI8p+t& zt*C{tl?UrCzBX}{NS5p`y*zh#-O$UK?as218G$sZxKhn_1DcVk1 zWU{Q=%g8yTjRSc*s$!a)5fi~D-Nq7d9VOKBmaikpo_Vga%RG_W>L!1az;-q%nH z1`K_((5Q4vYvgV7$=4ZUo)<0`CS~TfX%CL8H$_i3RhuQ$vF|X>P#OLSc{@RXe;KdM zg#|>ji0}c|u1k}j%M%4GdWC!L$ud9JGEbTA-}$_CcZxbH)N6E6-lLI=>jD+YSan;G z4Kk>wRf{xtSDdR)T*27wBUkH~wp{)CgL%@LuUB|y#fj6JsGtTzPIXPnW-qe(+O$B0B`gM{V8Sl?DI-%lWQ-=+u#^@xCL3@F= z)+ZQbS9s^$$lLMniSEMdVjP&0Y3*eg;PQi+qT4~iMrW}bp)66%jm(6CyZ*BK>94Cx z~%E zlr)pY-iyhzP;O)!m;r4Aqeg+xaIHLTarGBF=GwQBQQ~r;1gcaIJ*)id>}OuW`E6GD zYrD^HyWLT_b%XHuvqZoY%k#1e&@SCYya3a)U-St>g`j-rP7;myHbQXtm;GdG`*k7B zO1#ita$a%z=A2wk6$31L_a>$pha1;%`16jbj|qh``YnREwkvT{w}tkG6&+&2ug^z@ zi|@@n39;xec$H-9kXo3|(`g<(h*9*E=`U3!Pt~uU|L}3%F$iurSd12&oSvZ?3o2VJ zqMjFccraxqlHII?LdN1ga15?OI6g9Nyp#IvxCYN4`GIx*$hhAR1G^C;)(hJH$s}h z&xjvNJ|ku5$LNfo4T4jaX;^vv&l0yGFYsVoKOUz&u^A8uG zi2D_58>NA(am{(7J0b>D=qHC^_rEHS-XK!RlgMK~kmN?5kNI(4B>9WDW2(?u@^RYm zvr7nCH_rRI1j`3YzbsM<{R#TmbL2LghxsKirJhyHjxu!H<6##)Ws-q)>KMBiipiOI z9lbG38lkAhv?Ncu_otOB==L~!18Ic>*@mur%1jAEJJVF6zvWm~s<_WRu!_ORx zUp0og#FeQvg)ufnqc!|YY>_5@4lvqjZW~s_^{EQ449oPyewDWum;&Jn!R-%O*6&iO zbrfO?3pE6>G)8W0UJj0m9b{hx<)t)=C}dWDolkzuqPw`UsLREoeIpqDmCfb{!6D;| zO8y9{S?Ls3pB`r6m?}H31znMebG_)DIcBf2$yPG$K{lP<)LEM}jAlnM!czKs39a^^ zJnHBz$@=i3#i(O=8TW&)^mqLWqc+?q&>9|ItD&ZQVI)%+_jK=(2LG}|-o~3RUhXBb zBkSR-JJN@nGMh0NnoeXx>?|pB!780^V<`>=ljys*SRZO}y#D*LwOYJY47z61@48?G z*&C&FV%0#eMnQ>7kw3y&)6wQ~Jzv@G5{ZQJP9*Fd!2M(cteV&)UpJX4>+XONd%aFUGjl4#)?)T zU1%v*bIJAtWJ66uE61nVVj9DE_6Hi6gvJn5u`;wyLzvu>CGmHnLP>=R+VjS|f+l|M zHe<2m0rE-cGBt^%m}Eh=HR=V^ z6KSjra(=wqk-w^%OEXL%H1Y*d07AYK;;KODCrYLIl&LE?dE;Qa5{UhjM;Z&SiFPz5Y-m-F zn}pr^;A)ZhjBE=rl^Z#ARd!HKTBMA2BLnSy<`=6tOWbcYA*X(op)%1j2l_kWi9x8F z$bX}J?C~mftfb zabx}~Tk}Yn8rE^`R$0}))lva=BR9;ML1e_{QIhD=fXWyrbEo$pOR%ilp5?;Z{Nyie zt!CQi!=9_S?I{oS(ocl4zS?Co-(#~InoSUQ6_1(D#C$RdN4?vYxhWbXY;CjejGYru zc$*y)yMc2IpH3;HJGT77e#_OlB22tna4Cp~jcK#^W^$cU6V3hfVeA5*RsWyzB~$*w5=x;m32SAF$mqA*(jKpgyNzXy(g1~mzCZtcvd?i0mQ@ z*jLYEmLu6C0OMd?i=ZhMsq$O+wn;EBDqOy=KrKo>?6 zF2|Lz-N4n}EfSiM;$`7802}9%QfeYl93&0M5-5jQy4$I+$%7?=t)uw2F$m{bXzzD zPU&8afO@9PZJ0}P+c!*b@IAL)#Z&k>+oukb0#p73(3G$Mf*T^{=Ae<^#g;n+bT>b2 zpEB-z3xT?Xr=(nGU5$ka4w%L`d}9jiF>H1G5_7;wqk9KR8N%XZ8C7K*j`T;Ke!2KD zl111O%1?leWCW%SIM(C-rC+xHy&)z9VQ9obBesU4-Hh`87umfwsZt5`RQJnmfR z2fBj;iz=D>$F{(CSW!=KQqm&HO;UoB;Eonv@`C@kj5>^#py2h1c<{kj5t2p0u+&Sw zfU5IzrV{5STxQJ$LP1IVdxW9~t6x%HV7@&xa#pT7WqVD^11-)$3bDcJGItAkc4e2eR8BE6 z-w)Psb5r}nlc#`5g-`?Z_k0{!j7C|bzL3PA{{Q`niUzFh{)z$l|8_UBz8>hurNthR zr~fqY_XQJX06R27x+hxnAI)e1Y51nN6Y%`|#gi9|tRX7fq&e6BXy&W}7*&D#O8x(! zErtT>h=cX8^(n>e-;Np?L73@0Pgn78$4|!KTL>}M!jf|T8P4C45p#o4)hpMW1#Fc6 z{E1HxbhFUD_|<>A%joM1FoU)~+vV54Up$r4{}n!%&i^a?6P^D5t>&M9UOyQjF?w|x z+H&akZaG>xKFTj(%@l2(QNNd#64S=&IC-Gh?%DtGWK}6bp2zQ7*l=m!u2|qli0?g? zVUbQKxyyndU^}$#?31kjW0(WZXOWmO6>G*LW8CuLpH+7gfW<{(7YGxckxPO}}`fdP8p6&=a~yL)8z!q`9Lb4qV=3kA&;^%OazRlZv*f@CJxpmE6L< zerzg2xn#Bnm`mk>%l`l-pEMX3|BGTkvgOl)4ZTfkOy zOOAGNi{Oy;q+kHCd0T?2^O)Bkho-9*u@JAr@$>pw$$e(YS^pEq2Lh+i1b}&s?{hdn z|EclP!>xm^-8jO?w$yEx=D$)B9C*uJ+;(Hyt<%FG5zws%0F~*3>wsgYSnF`5 z*x%K>ljw+};8OSDoFX{Wczljh@4ms_kN7LF>D^hZ#8$^|-&3Sa>x4WoPkbjpMNDwT zA4!L3<-CBJq=(v_iU|~@3}PQ~VjmUp(4xjeHnYr3L#)L8rEj0*PQqU_#=0Mjd;b#*3(xp(e@)|5GeIFgUQZX5RjRkP6 zIxo=#g~I6qEdMEjI1kk=q`e#Z3`Uv36!~6b9ddvcs}}O;$T4BbA*;V`o23LVD-!x* z4wnxg7Uoe7U^Gn6Wkf8)aXm?77E2nZ7X7;g=!=)2aTRb(6vcfpjgY%1uGvQvj%kXu zxAu2fpM6|GFTjpCKq-%G_3Gx+I`ppa zzU*j;6$Tuq@ZTS8;)QEobFp|hnQxwK)Wi0CK@`MbF2?{oSx)xnl7p_3VbCOG4aeIO z0F)@yxy%&5f$&1ajLb6Qt4BEJriIe2%h0>M71OhpD7)RY%=@q~kO(lmZ(gke%L$Cg zO9l+uMg83ufR!4gb5cwLM6wBJQsfkpo_XFuQCWNbM_w*NS#p-GU<>7S9Ss%7aKYaa zDE0vr!@{2*EAJL*T1eNOh%$`ORYLv7X3xjt0oLHt`xA zL{g7FgP+sqKlQXBtSFG){kCg(}4Tzx!XWldw+x1>$k!K z4gFdOtDkg`*RRj6)*>!dE?($&=vqZDF?7Jj|@rM+e zdWFl5$zdB@xg}~S5FUsWhcTLFbX3l!4smu~dbD*y0|q4~R$Ye2dwuJmv(?KoitlXm zXNYhHDOV3qRoWX0YXh}^$NdsjWGKcOYY6LwNP6qQWKqUpk>|(uNB{K7vNmHR z=k1Ey1|es^g>m)-a(_t#h2JZGa7BHw`dxPLy)jm&7wIxJTEvGE;EC zIFPOds%Yt;F*eDHcMLG)?F_Fy0dL8CNR#)7e`5jCk;B|oW%1d%>I8+a+9x`q*%97v z%SHh=P;;g|NwZa5Xm=*b&fwhKdMoc(kX}ckKvvmtH}*)Qb#%AG#d&blbECa9)U4?3 z&V3U1O$j0#eG1jc={RelafVp_kOSlVK)6{IiLQ`>NprKw$fnviU8OLF(K`hpJFpTy zvM7RTx2=~nwkMGv0taUCT^k54ds%-}9&QiCm-gy>lol=g;=EGx=St)31;d>N>T@ac zyVP>WM7XH?MW?~CtdT3wrSVFwlR_i-$wY=vA;NBtc!&G3=5X7HE#wpnI~5T-19~pE z$5N@L%=5CPf?{BJUx6U?r&&S1&*4IAmkfjc_L$G{la)+fFUtYI-fayrJeY6~)iRrO zYjx~ES-?LThynS{x6OgFBmg>_pt!z8fAcKdWf%Igy`(IAq^Gr-6@Cp z?5y64OmgU`Ntc=h{Vg|KjIwYx;H~g-!W$sEZuS1i3Qtywv(9c;gttZjYx?@tPFHD& z9{6z=&=AW#0cnV51^5pi>O~g|zO<}0GZ;b)x`rVmpS2Lf%W>7qKn2ZyKG0n9bi7f! z^(ZfL@vC+h`{@d!1XA%%;^iXhsyKLhGf6vz0rk2z|BfR?ON z`}sUgEFh47BA3&%NO79aQhMxov^%TW0jZglNG&tY;b9{O7A_}$^QL%X#HW?oPX~Z( zrd*oZK@lw`CML{&0y|<*@ysOoTMqz=cBFFzOan?p*dRV6zFS8WoGq#eBc2*HY!V^% zM8MyeX8Zu63&ejo0-S74O+zoIky&zvx9x&MIgAaYsc^N?-l|deg;$!@6BK3{VS*o= ziag8cH|Kl+&I$8Zokp5oz>Ew~cE1I6P4yb&0Ot}dsLAO4Y|&}-CR5C4+cV<$}iv>tC-yVL#Q z>^V_HRL>AtQy(KkQ+m&Dc^^zNXz~Z8i5_ltf3$>gw3-6C{;;xUgy_2L2z#*Hh_~Fo za|R5j0n2xZCJ0{G~XeNIe!cHGSI# zCUBSBW0ApS*nUe}ZgjL|Bc25r4Uy93cl%lDb|yls^yeF~n?FMU!c$sEVNVg>5JGsf zcbn<};nnbKuCbhDZOCmW5!?K}XFcgsw?EQR)Y~KTbUl;>d;EClDZT2qILs8p-F#~g zmS&zaX%++Uraoqud>v(%*u@PR?#*rKF%2vTy!AOT*56x!t*sFi*<%B%h8Pdl-9%%j zcZEl@-ahapl~H3J9c_8~A5+#&H7CJxrO&Uh{QZxpukfk>zDG!=Q#hV?lf) zu`|ccVmd4zN>A|)4A_*9%qQs~q7ffT9Y~IwF{UIc;d!U{+6m$7!R6m+0ADY_NFUWu zMfofF8{9sGIN_9fPw$AKO?l+{y|#<7*R3tF@iTnA5KsBUx+yx(8-*eRzMbRJ&G|LD0MiJS$mP>ds6e0C+?-1WpWuQZybM3;5Z*AJ;DnvMy@b2Uz$xgU zsBKUrkUY{8k~==X`6TiGY|CpnSbSeR57}w8Uyo(~L3M|!t9chqt}Z=<`QnVN>anyb z+4^BI^zzU^8Q8H@I2L*{T$0=7WieMMyGk%?m_dkHvh3 zSHVT+OZLB{s0?B1dvN@-#baLK>&YQ0z4->inzGvyIdi9V$eR}c_|~<>dPCqVe`dUg z7Dcez5$18R#Gh$;^cPX!U=7Ng z!CiwE%;o6r6>6vvloR@70r9zkdoj&}sX+YVP_^A8PWQzE5svkFq1IoA``@f43ts{( z6@<3Jt5YINCF*XIG-FzI0+<5DmR%^He5G_&t=aPloR2aw-ZgSp$ENgWi+wX){6xne zX2)8`2D;P!JcnHY9=INH&^f3BFWH~=sLH&8%4r@vWfbh!>AIc#`08kMX3QN%N3`L{ z#3~2`9%!2ubMM+7}06!aT^? zD;?YROgF#{Ig-xSRM< z(o%K@Ei*?-9n)jcljiUUxk0wJl8WpCq%FiMXMFA3Z}s)Op8^4zHdm=?PKi6Xe3`63 zx51v_9rO8xTpGiVbg7Uh;gX&s3kI6;9AhA^FI@yqM6TPBK=PS()V~qf5vW`I{R%WI zYeWb`ir-d3lOgVUL&n7{3b?zE*&SY0Nf*VPaU=>Jsx&$t^;o$MhOIC=@U0^Dv12&1 zxTE#I8IuX11|b54Cw04|_y}qo%B@%1;^()iy)jP#kAJuT3;BlPpzI)if2GbxmL^>% z2H{j3!776iV1mjA-GGa!8DE?SDtM+_xbkzdkVh!V$L6i}gGv8nObTuT<)2XGhRV(E z6P6%y(b+ouZeK16#XZY&hTMHdD2q+bAv_qH+9{&5IO zivVOE+n_m`VBZC~k_+DXAOv?>AUKYnnS62XLJLBMRu#hK*iXw<9|g*nBJfh~x}2CY zu=xJ-Fo611*dBjyK$?8dt)m#b@)74->B0xH`&qGtC9MzEl8Zu1eWcT~{;Jd{Q250f zR2Kvi5pZI54bsO5?wVu*N8~e{Mf>Xq2K$43+h3{TnuI2Cw}` zbO{s(K=26kml-ba-}<9bA^*_=eKS-DW!{-O@^LwQFh4Xxv_ZlwL(u%}Kg;YIUcg(* z7to9RWRkJmoLjBsaDkVe`_XHw(Ts8B%7Oky8aI^#6Q2>EeCoRuWJ*tQbd$~(Q|Rqj zz4eJg7miDD8Lb?9S&a6%s+;~bRNKB1t&#dyx&I?k1+m>=a{h81j9NOraq9*hiq*^RJVLFvasHhV+BAVqw3Jy^!cE>=E*gS zA5X&z0B!#j#{$eH!}IS2PQr%-J+=5A{`7Jd;kdsFcHR#J{CqP5s!%RKEA)(l1T)c) zEM3<(U|Y-7B@_%r&K)1on|LhNo}f2?R6pqK2(68!#}L{QD=@x@juB;+)3{;Tl`b`K z>%TS&Kz;g;3#VII*L;pS3QAnka@65f-58bW@=dYDj-3K{0WD)tR(Mn@PO#}0Y`Lkj#G@1Am>7U zowv5v*3X5?;{x>k3{>NKvPRCzGp4Vmhwyx8h?@SWFbLcwiC6*#&w_g& zyu6G4*)WCVdJl}1Pkl)e8Y{8NbIMuLS1i;)k&3CTx#0QL$WA}>^GkN^YX2;}^FUc0 zNX-ZMp;fNpsw|EW-wJ8prlwh@Hl;<3W+zR0 zU(JBL9q6lZzSs2hqp0O*cjhh(xUQT*m{Px~In!er8jsp}cr3G3U#F>%8u9wBh*DSt5mcp}oTg$oDq5ap#0Pe% zi$G`+4<<6BK;=?-rQ@u#pucJh?$4lAe#aW4z)PC8UuSt8%C497rx%OvDA~EILlKLJ zPaYx~C&E7bSKBwQ0_kCu5|)AV?h%p(h>rV{e|XMP$nbp&z(xHdrfebvruQ71@%4MLM-@ef)Q76qF-{^AR6{UQl=LzCqMBNrV zbV0E5(Pvt+N``MzVGev9$Kw*NIL~!0a{hx97V61Mfi@@>W zv~z3KouAd~wrmxLL^%!HwXcoxc$C@jr(^5pF$u=nMQW2?W_??qQs zy+ya;8rUhPylvVK@^JvPLPdbrRl)o-4Gby@hNM2(3;~w*XP(;2{Y=rq<{c}k<_?9_{*YQNtg82OHrXY&a9EAs`5ed zM%UmaeXM=gB%afP48_qeeu@}l?5Pa%AQ=R!n*bT0NiiUa8b<_GnCA&ijT}yUo`~=o zer`e=38VUS$l+fQY6<}WS}pqo6qqL5$Iq2CWnij&I^Lj+MPaKQA90E-T|BxdG<@HC zJqFt|$f=SVpV{clWo=o{KLkAI7Pr|HO{iM`3Huhx2tLhgnLp`8 zX!mGOE*eT~(~0a*`)-0~YIDHpNB0Kr^CR=S=f*_fD&mizPS=!yH8`pULYBaA$#flm zs)_eXg}4d>aB@Sh`Wo}nGT2zUM%;OOGISj(T2J=g`<)w972#zK70q4;HR_&Ts*2Y# zS7^&ovqr6d$p3&0rvo@uAnq)QXhAiU>WDIJA=MD&{U*K(P9@N5CYKmOWf3REj{kvj z$m<8Q5_JcldwCRj*!Y8dOT#O!~kCAMF5jcA;2bHC^=d zd$zqeaL!kGBJk|NKG=aYh}cyb5^DaPuWlSG0?sAFFRnTPgH1q$G3`_i3kUgw>%JJ4 zmVO3ard?=ffv5xu!h$i-26G!$qK+sH5_>>qxShw)&^#*6$)W8fJe{YOkq4y6>NN0P zkdAx!VkjCF{NY`)F!C+n3w>T+w2=68Fza(bT#qQ!p_3=vIvi8gGa}KOm;$!1hSo8*b%)z8 zzBN+%bKl(oI|a|$c`K))*9#-Oa>4^PDD|Ipk-qu5qdP8T2=4$};bn2N!<2P9U!Pzz z-~E=Dbs!d)5uFfHRjfcp3ynPMUT?DyFCKiOOheH`0%MJ8^CJ*vm8L1V$!dQ2#4o!*lSkgoxhftViX5&|Cd zy|fxP3m;K`mL>#(n@)v@E%K;KYaI_jJR4W@(IUmdlLF9DJ6!~|MHMLSFT2e z)gsn&9ryD*e(dCm=d4JI87Rz$b%HY#hOW_GB`@oT3a_t-ldt`VuPq6#fATi=8Vf>c zGmv*j@@?#NUP;h*rekn~gSHex^|o5a*DI=@Ga|&R;<{a0g`7y+&^!I4lA(C~Ezu7a z{})~F0Z#QFzK?&Bj0)#aA^RLVLS%2pUYU^@2_YflWF_m^dxb+rM#(CYRrbi1S%hO| z@0tJWt@>Zz-*tVz*X8rMu8jA2kJtNoJ%F&oY$Pq>E(1M!70`BJmFbh}2)BUo&2~Wx<3OEX3HF#$1p^*;y zX(298A}TXSd_EQ#8!5io#)L;+uSC5kHn9=;Y5Pj4YBj{B=sjhkCzCo}f0R)fO}>cy z5*RPJB;yhu{qXC2noBnAGFd;DxqkmdnZ&R=A*gvt*?qfsJ7FI%>PKjjP(IDM!1q&l2<9|uaY zS~qTfY10`?bh)V=@?$!&JcW2osCcrAw*wzisHK4akmf4#MMyLg6C&i}toVx?#HTP( z6bIaNL~1EN+xZ!0MCO=rq@=Mt1@5+=;*G#!gtYO^!}&DGV-&p*eWXmuZOkX-@S@p3 z(Gi=KCISb3cR-s_KB8%f-Xf|Uc{X*)%)oRzi&9VH!+`H8;l@{wGOdiZ`(u0>-A>=r zk+jPDngOwHTwii;s>Zb!`2W=O#8xtQrHPcMJ?&R^l@h$w5?*uC-~~!V zP;LUMcZ3ZOQo(#*k_Nr(2}tM9#hm( zrVsr`rVdmlv{f3CFf{hPQiAIlpHNG&V;h@81E)abv`Qq}A|1+&y^Awc@cKEbmRN}x z@h~KFM9VX;0bZGIqnSpBqkg+z_1C2^#FYUyw~*dxigCzZVvM2hD^vrPDg4}csE3Hy zmopOdx?jL&5k*NfC018roM6@Hl^DRz0=GEE9VjjL2ceq@RPQeCfgSx#XW)H&xh2FV zd_|l=nXW}zb{*^c`@mIdCZEt`*s?<*_6D<}Y`f@L#??y(GOiO|)k7dO(dC0({yrEL z935^meIPqQABy=YFIM9b+(y+9*u*AgKrqHCnG~bxWKQUZ`8ZY1NF;ARD;Aj6IVjOpA0Jn zWLQboijyG2BK6xPz@BWtVyV~;lV@|d_~qM5ARm%9S$?06N&ghRZ&JZvo)HNW>SxtNgwt~d0QDj%7*3$HExq+W3m(zOp+Dn zZC`49jcGfZ;Ur(-cpsl=7a|Ual;|7c^I4`Z!@r6MV%M5lpzU!JFv3|k9UTI%WYN4* z*p5bJC}S*Jo?{#+cFkMMOtC^;G`5Rkb&V^&zvq1m9V8oNrfGN4`fviwJ*+n_MLmGq z3RrX&YAG(4>0YfbJt|A8eLFUNL1H^1-#UedX3H-c=fb_O5~<$ONl;1Vy&(7PH?RgX z^S*Dd%pDaUA#hhIGCqFi4BW{a1~i7JchEK{WAgD4!oudhH37_yO!8Ev@8S0wT)a?2 zm^3EIo51|(9(Lhs7Jgr2=VOeibGQ_Sa*U$SLK(-Am!{p*7rA-XgCVmWSqB>X$Obj_ zOT^;?=*SU)-d+|Wtlk^Dx`*Z=+?_^^2y_5R>uCMr>(oM$#ERFZD?cZ_8Eri5?3P^K z6$O*+E9Jy2+~C<^8mj~XF1_lSuY?4cCr#Y*9!@Y-c}q6S20>NLsn7ko{QzEiV{VD& zkYqF$IR#!8<=9q*E0$sZGwmzID+v`|7Xv8?;1WU9s&Sf9C)#asYmKtsqAI}5gIfZfCT*zm|K>aJyelm;r)q^;Qajg zw+FFuVgy3e<27l_2FS!JZ=z9Z>z9%6BCnTO>3F99HJ1CsEihpiWfYN{5*N3835*!2 z#UqP}CEnk>Hp0BLrLSYpT-YY^F?U52=nI;XR+>l= z*R>RKU`$^nd78f5)Ve#d^lZ7bb;#TFSWYD4=Yqz5<5sfQK&*F*DsysAUN0+8y`OF3 z!NafZmT`{H%1{ZFCFHkRwJ$j$DhHeI+LK$UWzouL&@7efCRwwEU=h)VL$tfR7Rr2r z2U>n^ImW*2I#@{spd(UL>3H@hsQdy!rB+hz5a4gRcvXJ$&rdeUV@o@?qHFr%9FegG z9c`O;P{e(2F%l?Z2M-Q4Y-+f0Hl3;gJco#_1G#`vdtHn{1~AJ8IWJX+oNQ*%bo8^y z#$q0%3b&T&CCOh2{) zqNjg}pv+TYOI+^FtTYYp928B)TBUYV7)*@=GIOO6uQ67Mjhhm8me-AuZetZhgW}cC z041WIYFPISRM@U9N*P>6Eu=0ZulKL{j=3Oz{GUtTnWE<$l&HY3E2FQ)Y8pqwj4^_V zRdXU$gDlTNdd(ND1E@vdsE)0wiwNrjQ)a|lqvz|(4Dj|<=ayil4kM#QO0$AV1?I+p z*|M5DdESvxEJY*cA$DV}zK6)^>rCMe_A!-e`AIW9-m5OI_FUs#^L+QJQKr;llLO22 zWU9`PdDu5c(H451%d!UK2aTG}rq2GEYmcKG<2NFdvZ}=|DQ_pu=8wn;L+YsFdC83s z&B4+D^%gbKOA{4g88ruq$H0LkUvHYy@kMhII4z8AS9F7~Q_QMiDWctf})p%P*;m@i~lXFqeB8 zFGiNSPjqR^i;icqeOxCbjVazkFxvVzED)oM{6J-7<7ypu!fGPHxSVD+}u>qvETTiW~vew@MFIvPyOe& zP_^!jaD+o?l!R$eBaPmH8sV(?mQvF#RZ9*rpf$A%h&e1FW;nE}VaA%t3%sk{9s5Sg z!)-x2L*`nh8QH7ofsjTcc}^G~x1;s1-0$R12=|-q-tO|^Bg3?Bs9bE{KotBTCzBrm zu%iHb?g+rE3Mj^`=N-#YdC@2VW<*G@Y$hY4hm!{RQHX^TDg0UnOq9!2esH~vGScv( zsz5AD(ct_5y_YGTAJDVxLPhzoBVWHFjXI9Xud&l@Jp$Ej%?A@rlf#rceGOVrlWtE`ZCBaaMwV|I%Tqh?yuSZ0o8W8w1| zS(i{awB+C@h>+f_(8K52L0l1G9ZTL|x~v?YE?VL>Ts@&DwHM>GUJ$o?&^y@kJ%Aym zwZ=F&w-j|J>i)BRd$W)5T#A*kdHf*kx+V_o1_HmS~bNuJGfACTX(2g-$ zu9FRr8);QWp$*W9Loa(v5KhBoHcL_{?x^_w*PxE(3WjrOV@M03k9rRiH|)R8z)=su zx{jrtB8iVS-E`_~M=rvEOAfl)ZaIfLYZ+kk3&4!Ao5^Nhjj$mmw;Uui^*JMo@ny%? zHeM1q@ujGE zW!tzua?XI>695lxiR0Qlv;#aAJHR+fWqzf|HjK=xBOnQ%iM1H4h{(@sz8mFTmV1S7 z-WBrTsi5LSZ588Fl1|sT>^m&Zm$uW&qw>7oR>?ZQ#o7O%8EPSeSUdhKDF9xT;E&OG z?Y7D(DLlxLLtV0RK;OV>7mrgm-MQhBub4iftRb?4_-Mg{rb)>0M(K~69li=Ls#YJr zJNCh9yV=Pu??hTFzjSis8xHyfz8#@F$bJgPR`rEX9WqN{s>^CS$ZcvmV@U)-Z=8efrl8_#~Q<~ z7L6z$JvOqXfXCKUn;xp2(2~tFxGIfw_&9gWP@;D&^H%DwAME+&Q_74aT1z2fEbZUB z0rQV>@%b_FY2h8=;M2y4R}RBNnx{WjwDZKQ?Mo9X9O#yxCb=`{NLiJB8>3pR`#;a; zBFS#6suJSjz5Abi@Xw2kKLL(B{S{X&mMed^n2tSA;cyJp8z*yhp_Q^xG+b`hI1DYq zi7+s@q-_;RmCB?KI(D)<9;wrHjF;xAFU<*Sjl@^3l*T+;j?-S745rKIe|O}y^Z{Mj zm8OcepH1H4P&J7hUErBl=yxA;pTF4t7(stVvPsuNmD8`Uit17Iw;Xrx@im5xukhxH^zY|vXk06t#9#$;Qm`;$9uCb%r(XfkU8kdQz9M; zMDxv~ngDA^hX_eCypcDKyxs*Q5%xt|ZbI~J-AJ%zwDFTK7wE*O&kK%f;>uS1JJNpL zwDWGS!rAJG-RtnxmO^(%NtQRGun{>vVwF?Z#ReVs27H*o+GCD?`$t^!I{5VGKP|us z5MPgymFGl(wHjt(+RA$*s|Qxk z9OzobCi+UF3{bQPW=jeqX34;Ysy*P2x--BQOzW&9G3S|>WbCXAx#3>g6`tc~2DH;- z5x}I0Rm>Q*P#?7~A}x`Qzg%v%7HDD*zkSKu*Bq<-Hs+(PQM=ujXeG6kT37w+^@I|r zX_VLvA^XMuhZ*;a%>VC#(C-;;<|DXM0C_{u32mk20RWD4_p!)%SOnCY5pgrfvSVKX z$zc)VUB|85RvNG|~k73=o>b*4${xO@D@CbEZONpFet&V5#G&GullMB^sh1tHZW~(SI2P zgkiMUu-re%=>Oj`;O7S%u8h6L#&fYD%x5?~BB~8JzL=L`DnA3E$dC)H^Uk-sWPAY; zQ2H~}bxWTUh*C**s5>A_Vu-Hz&<)u)Z$Y4{d6df~%A0SkMuZnK{U4c_ZVq%0N0SmB z1OE3X@+Z*hF zDp&8h7|pQtudG^XWAHCcq$$G;lU2 zPJU%CZsWW;Kz-K$(zYsDQ%X2iu&%Py6MMU8st$>{w zun--L)2I@HIweaCxPZJD)X9?I7B4p{4_Faj1|N%CFQ@NIDo4>OuipdU(whaOXd8cA z#(oo92PAu(*nTbaO{fT@C8jNnvAT(0%F=Fgz~!+CsHdAj`dN`R8#@5cxB;PIKOnXD zP#L6~Qed?m?$%pQIR&&&|BXKyi`PJfhkKf$+{)x0+>azCk z3(A^INu)A0T&3e6pcQ}%KhUG3vR9n=J$IL%U6?ggC4~h7QI3&!MXc3Z2+;p!Ee%l$ z-@Hu=T{fC2hc*4g=S6xrT4`O>KDNEOu z%@^y2(Umo`+&fe>Y-vv#qI-gyum^FVmeMSH@`oinOa(P%{k-NH3-1asbL@~r@Aqi7 z7eOAO&$Di3J^GXvo>Vg%T?z^Li&c#b8$N;8>MreIHRL3tJ;Yu4y==~_!XH&V!qmD7 z-3pY>hi>nV#15lGnO4cP6fGg zI0;8JMY{e~cGQAy{?V}VET}&vSbC2{NC;%C65%5Zs3H~_4 zPlvih2Hx#wq0dm%!$6+JiGXb96Cv))cBmi@&$I%}Es%kj{$Z99qF$(T8>qSxLHG6` zNoo65>_)cdgoR`8roI9HT~t3mrGHDq#@Q42ybUPNG}Em`8)n`jqNiaE7*=P{$03CC zXEnd408CjR8|yd!q42RlKdty2}?&Xz8@c(m|k z{1ygH9hWA9bLbt>v)pe%)0oB8*fiby2+#FytihT$I8ZfDYr;QB1@6?Mxk^Wm2 za11E*0`}`rz`pzFH?YmFg+kdZNY~lD9AjO??NU}~CifM%v!U?;&R6qGCGyelWabb@ zsrF^cqrRsngjYjm&c^Tf*#ZTLNCfM^D<$(!@U|}Z0P-1226%j%M$32ZaBWE()nnF>@D|MrZO%=3|?~=KHe4Vxs+Ri$9uMv3wO*q*F;JJ*#H0Z$xsvYx}(B2al z@|21^IT9bCJyz-@7l|3Q2;&xYM1V}L#CBY5*t><6eub)$TTM9W*6x8Db4UeTxZQT3 zqSI#_J6n>IW|tA7Ry*?MD&fA=6iokY?*+{d1j!h=Gc}FhA{d<^V)C%x349#I!ZmiW z2{rWs3q!&XK7Wq5pRL4=%Qr%Tq%KO z)zc3vKhsfyy%_|wvidnyM;0(^2F3&D#Uuelv7+uj0>gu`k5Qh^<)m*1;otj-Kl8qg9mu+>pjyd0V!b6Nuj+Tp;9=(n zG!W$vX}(?e!ZngfeHV24t5WPu0wK*?#6gtX?+bHu>NchY;;(KuH1b>5tOsslG6$*E zw6hs**8n!EjD}9?U6rx>`^)vXv~fUIENL%Goh)i*^FMa@BcPIwf}-z2(?TK`Nq{3k zd&be<4y^z9t1zQT)66|(5_~R+{Md^;&c0;iG|D2qmiZq zl+i}@?J2)1s!f;vsZak0e)k+7=AQ#B8c4(r(@r73c*jAoSD!e8oih<^yeWC~ZaX|0>Z(Dnx?~vXo1$l(Axm_joxe)y-eKrANO+6#_irTZCR(dm0|^nNl>4+{c(txi#JZby zlsk}acnHuIMhB%mTc2nfAwl!IVaDMp(2->T?d*raW2V*q8%e*||GXNaK)A0lKk8~v zo=kv0=1Ap?#HMZlZx=&O4@go10%YFV2naBMK=lXIm_UFwDlwNy0cjZ1sqb$0d4;HH zuNeWF%Xs(=SwzuH*BD=_!S#M}(pcWXOMYb3LU4KHbg zUj;ut|2ps*;~;8lc#Wjz_x#Xgde82`YMo<7!tO)2xpeo3K1Y7L8)9M*iAs1$yUul_ z)^ks#+_r1YVx;80?O?vvmuPWXWiru)%%_0QdL1S8`=fzH(Ioe&@}^N&G0jgY9nCNJ zYM=4dGGE|xG2ieQkg%fQJx9?-tQ7FZe*NbcTR;@{=ITJ-`}39T_dF1HS;^QOkJTLS z8^WB)u;^%I^yPEnWbh?z1S^TQzZANAANr}Q+Gp9aNJnh&W9VPtiw*_8;s3eX2s+jR z7x+N2&h&L*=;NU?@F6l#8tSRtR8j~3GJgjSm3G5mW$^5WpYgB1d>;5fZqih<8~VmW zjnA^qbK&-99|Rq}Hy-?0hraF>I8nb2cuDlz@7#L<(2<~nMG_k*b@R!JLD%4S$O_JS za<3;_=+8y@MKYQ*jlu_ias#7H6tG25L3_klz4nRQy-7zJp(_?^YKv$JBUYCwPEgnb^XI_1B`@Bute{a-$w6=#14`v~@ISzbxdAtz6rrBz*_ z&CVx+-o$}<6E|`}&ehY`_9BrV`c>#6Oz^J7eyiQXRzVjTAe?56MO!2dfq5?i0m#W{ zl$tZ6AkZ2$2XyUk%w&gK6`77LF+}})JgMOEd|0cu2G?-M2@iFB9(Vx2LNU!gCx^Kp zDDV^kon?Vy6utZb-X*T+2~ehFCh)ix2}^2D{#|UGs$iv-RZdWVC6X_CPW~$BFB=tD ztJ!+}&ZL?ZANfL?Z59aje8+$&Kg{aglL4^=J?n*o)W5HR9`rzVx;-bLYgzAHeFIsP z6Dfx%%CjpcSjW+kDk4wH`Tjla=g$LkdWp z`^oWEPYmZvL#Lc#$pnz3W4yW;wfBO#hg!({$S(u`4W$0kX-nalEVu5FRg0t(tDutW zekp50NpQ96Gs(4SLV|?Z@O_iH`mXPFSqtJ0XD90?$_C!kd7<7$A-_V3-o`5Wu@4-V zjXew;7vGAC%@qSpz&Oy5z%%}vI#HU4z_IfSR-Upk3Qp%tz6!|dKXu$V72mow|2G6= zPC(IRoNmhleB&jo*Iz7fOe@~2h;W~k#9IsYT(3K(%MGNiRryJuad>?J4&7Bxv^UfY z@4P@-bBf0wRB~L`_;aYG6mZuiCMBYNKrivS`McmP&X)&C? zTM10gQ8|`$iQ#ZcNji|S{P)-t!TYe|F)aQN#4pCo1=2;JBT5(o`>990V?l1Bia#Nl z22wF{dlb+Aeq+#8ueM*$nlNx%uxyoq^vMWOEV|hkOcv$5^rqV#+7-@E5nUSj{0P#3 z*R@xHK{|n%I3dmrcD}{a@PBs<3s&*a){m_b^~`DfZD}9isvr2A`JSLvM|@^h@NOvb zk<(%5G-tDLPh&5w5o#nrvpdpVlLtt^-UB5`lj9b^fjMcv2vl-&ru|*)exlHhp(xZT z2Vz}v%;yj`Z1M=z9H>~sRKhZ-sc+v!@WqA6Z;~$*vT={+zZ03%W2f}L=hd_h`e8Id zf})9OSB_)VI~O)@^CwWi!Hv0E)rWmeEP(I>`cwWc@V`aC$zT>2zhJ|}CowLLZ$y;W z8LxDH-4s21V>O^ZldmXcWjZUO<11$7j7_RQ&kl>oX+asxR zw+~CVrB*%`{<>G;V8ZIQncAc3nlsDSev{{JidUC1r`i6n@Os{+$VO4c{`B$Eo8o?u zPNjNelocJ_G5fpF)G{T&6R2Qw*t5c8&CLX=qk9V+ZdT#$H zgi0h=LsBnjy{$Q-BURNZ^{9P|-)`90+pgobeuSSO?)Ki$ezFP8Vf(?XrF5^LP|$H4 z5}oQrOWuSYO~W;#EFX29q!{{cG7*68VSW3CEzqpo^S0pEznd{YQ4+npUDd52Bpf74*lhH5nMOLA-RL>87hUJ5|IpFlT4u|UkeQrNQ7%Rqk^McXCG&7s zg__WN_<9uTm39P2Wo}Uu{XvEBI!{Jby@=IKv;Fo~QP-tGgNbJ#59??u_#52@xk0G< zRO?Frkyt>|Z}Yqa*>7@6e*zK}2!4%g14D#fb*;1ASPL2AzCjwbgtg~u)OJ% zc;igtQOu)T=Bix@c!y&$EA8bfsE+GSRZWYc=-#suLdgHC@#OPCrb<>{HHwF6Hg+u^(Ebj^!uod{Z zw^6jBOshIk`-4uZm!Z|FsvFu+)km*2xzl0sjkwTglFnF%K8GN>?)&R1yRn~|)-G-i zhS(mu`h$;pcdyM?B@cMN;zoba5@oaO^2T`=7K?1C8+F~TpR9ehqcYPHk*YKN_}_L+ z2lAe>Mn^wFu(9-#`yJnB5Tox#^O`xRYXZ$0WUE&a$FBzEs!hDEawAUyIxD$1 zCxxlTL#SEo#EMnFQX1qjBF>&B;EMhJ3A~hiJtIuKH>&Bx-0Y1$R*?M-MhkbqJ4Y8fnGUwQ_d)-WO+ACPVhMi($(vAJPpkhY z{LgPMGD}|P@#W4PREDAa4Ph_Jc%sxT^}UM(GWgLgZ=BeRzP-~=In=q=`MRi6K%v(2 zB|m$VoGa(uj^y}YnbCLLt#5tCI`URCveNemGYBvug}-%iAWta8GUB1by_};Bpa9MR zwIB!8@1x1nNJ%ONE665;k9E|NaY|_vVAM3$Mx>;XAXR%R3I>VE<TY%(=DuWA*2|)LQ7-_cOFAP%$ZAhM8 zcF}7jkyLz$z91i{rE+8=ZVhe`MU^h>WI>b(>IELAejH(I$GqhO+nX6`RiM8-+tL1k zq94LKzHb#YLz0wA`W)}S2a)DApJfY5AcCbiej{>8@@Sbt(9CN?eqVQTvl{RVS6=vn zJQ)hPL^5Ar^Fs;9c2pEJR>!8;$!3{65dwqy!K6cKw?X*mICzu;psg+$l0^$wJ)~gm ze$@?F!}8t zK*Gw01!>Q%!z6RDt*`vqb($HHwgBECIVX1hz1hVRS- z!(aRf^T5Iq-&yF(UWh95J5ShBHK?r`*uztQT&BFY-#Cnn5|91#bdmh>(e6qrq?&}) z12MI|wyrZEX*3#xNG`>^cCxvg!zrhPFlU*&cF*P&Y8(RFh2crY3;4Dw#;7!?Pr z^*Wpe($Wwc<~sA{8rTNWI+I*bKs1o17Y#tj?|R@*96eJL{HHPh8q|gik4Px#)G0j% z5>MJjtD8{k?RIl=2%}4}CfI#oE4(7J@jCrL!b>*`)`PP#$j}!<;(p_#GEU#laWh*d zB(MD0JVkFn%k-JG_)V}(S@GwBqI1qCN+P0R1{k@`G$L4259B&yAQ;Lkhk;Yq0O^Qa zrc=a!^1@7^Fdq;^G{b3Pea^X)9L~u=>nsr$|O?(z_q=#_ZmIf03 z8Q6b_p?U$TcU`NBn1VIHpgA%(4Y)t7Ws#%-+9%!OW)*Dc`=i`4xhKx?@&ma^rq^X@ z;B4n2>H4bW$Rq^t`Meut_b$dnm^tY&Y-{=eTtgi|QKnxh;31#oxGXev;aZVAbEfqC zX(3I!t3Sl(JbwDYV{#={VDE5M44B6e!GhaI0AOvV-3ey-grak`StV~Wz7a+88J z2%asd>B+N7hLRH^&Mc3E4PZfm@)Z0fV~hxb1sEg2WI$-_8Hy@TEhQGS&SVNtCQ_fv zS@@o>m5sO_IB~`1Kv~E z)gX~AD;K(~6v+2IJYp^O_Y|2g##vB5>o?RiFI(cf-KRK}#l=4BuCINm-SCyw8EV4o z*A1a6{F8K1T>vs+ymO7yfra_o7ch?W>I>Nz3DkK6{*?cJIBDsPRUEl07aaZ;T%}+K zHKhNf1b2C&AcyV+;VaC@{ViJ5<=He0n_IexZEnK!OO{-W@Y0ZhWkl0a01kdXJh034 z<7;0;64#gSz+*2fGQuiScp4-7(^ljC%svP-NYU0g!F}^`x-2gu))3?zt6#Drt!-4q z#3+sqHEII3?dG&k?rT`6#GE!!PkvJNHF1wTw;W4#)W^m4%=(l_! zkl{K%cc~F4{ES58iBCCSwDxN*ix}Jtvtb@<_jIpeW7i6X;Xbov#c4XhsCB?w z(G=@1?h}$@Th7qL_&Wslh|Tn zxzq_kt@hHiCoqtN1ztM@NF)RZ@_r!k^o}7;j@aJ8ea% zfHrjBY2Ef(R_@Z>0(@JlZw9+>HivSq#TLDKK=Y@leg?2ELo)YvOOVU62f*237aW+F zS>ip0ofdPGBH5YNj^T?Iw#>2@rW!~y%VvKlPdGd{;E3TGbB47uf5JS!FT&-3mLe-{ zh;0WDkZ==eU5Nb@89eCcV3{F`L-NBYsU+t6egFS*BIBI{DAzO)ZZI^LL&CaN$H$R^ zcsJmgV@z~+tn6*#Z@3$Ve^lqSm)cU2RQ#7|cL@ zv?|QR0p9-T{EcIBSC$S{kr>FNt!_PvVJDDL52w?o7r=<;1Aw=%qZX=DR(TbUy1H zE4*34qY7B;I%;rd>Dvv#VHw7NHkk)b>nEth*3k1YQeX~J#De@4 zCmdez9x0ju^0tT+#lIn$LP5Aje~Vr)PwNW&Rs0cSaqcIKc2?fkFsmHe5tlNT8>(_H zr5Cc3%zQ*pTp25wJYyRg>LyjFK3AEe-5! z%0WyNNsOOIpIF=oynP%vlVqn!Ug?v=l~%J8t}t)!3nFQbdx53XkE!GfPzYvD!5|5g?W;e15vWfm)1N0PD^@lgS@MT+cGLNvXXvA2jaP!KLd zQT7F!%{2vy4k|0%VnuXfyhu@8X(6Vu zk@hb)5`}W3)kF_zaA!UcDH-2U1lbXDordUQ&Qp;8$tMwA3t(2fc^$?$xB?1!c&Y4r z#yiZsO7JXmK!3+pv)F(_Y)8YQ$g*9qzH_$M9r6D9YlZYDs24NNvQ+ss~ z&hUjYO_oy)e}OwQys)OYz{yT01u&BAO|)XI~Ok?kPvQ= zNaC{Yt-f4E$$Q^4E#67Jc8r0&!J&%Sw>Q2fon*GAEv|N8VGPBV zrJvR1($)Z#8y#JXZUxmb4BsPvSO!E^W?6hbzh1@{!6jb2NYl$a`^wjv167h8 z`|mHT8I#AL_U{LOwz9o(0DJ&=5#boAmaZ~AU>#_oyk>IiF~S{VGU1ELpM8togBxW9 z$E!+jJZ6_Hu$)OtOR;c(~P&V=c=cZc=iMCDBA};_YXNk$Dd&43cVj+X2(pQ1dH`n(Sez29%# zB(vE2O3sp)XZp&yPG^IxwD&Aah3F@1u($gbaSWv1^Kl~N@F9beSG998_ugo%eR$+^ zuqJ0!q|D81B-bwZCwA;egA|2+HMSLCbiJsN4++s zu8UkIrg0Ny0Yl$X@tEc?z3>>6l9STsX8A?3H87UTgC|~2RwGNvGLMu=#jGZ&ja+xh zz^9{MCn8wW=ks&)Ery9358|#zoU6xZWtE_~Oa1tcp=uO1!BsVTqFFoBzNx*8`Ly(z zq0StdGoAx@?3*p?0Hjmcz82Htot?HPWmwp$Z8X<0BC`LM&<8&B^@8tLGqk5GKi=e&x4=Q;l?=;DPef{BzO_CiKrr& zIMF41>;xT3sif-iEYVd|Ea`;RcTim+$(4;JE}*zN5_Qo{$r>HPBQo*1A8I<4#EvOs z8hJVeMfTH(yw^3D*N&Vp5dm6P9?rf*W<}P)>_l~w!dT}p=1qy|cR~bqSH^`B>$DYc z(X*AJ>weEs%-TI=_ee+Xw1~roxTIJfwY?Tz*6-AKJ6Nig;O~=iJ_{y=LW^KNArMop z-6Hcj09tM?(?7JL1@54{8OP?xOg()=mPCZ5<~F&Qq)WrQvRQx0=BRYn#U#59k!{jk zVJ+69vf?3QBc+O@eF|tjNfo#NO)L6G2Img4lhyuO2 zT>g_TzhuU!k*Ocm6dJ;>Fi*5jK^fm_EruD~c!wZ_^V%E!?r7K6L@}`q=RDKg1k5dF zk9pK7zRLlYaPJ*;w~wKUD%Sm&Dg&Q%M%%RFy3p{TQ<-@D9-hdiU|9lLqBFuG<%X*l zTbyPjY^u|tbpEB2Gh*9Dks5B&(&iaXOK!bC7$Sjp_oO4TC!>Tp$)Dv5iYkN(aXF-J zQv*pjBNNRjsz!ZM5}#1z1H2aRx8Bas?xSMY&4}kSO)j%H*a{0udD3LM(kWP$Zup2v zgkSdEAG$H066_^A&UxY*6#2q_$6wODfd5Z-q;8h{i(vE{t{x#Fve7~b03n)Uhi8A6 zB&Dy)tRgE}q9T?zMZNOHd*%i8+KiO+r*3*iLhI@d2{_!hk3HC2;z5aXAuv|ix{I>0 zRdaA{Ul9x7`x$?ZE`AXmKN7p&AMs#VPq1JrjrYFn zGO6e8s5P5L-Mo@WM$b>9xMf<^4g%>hbRvlz)L^wg#FEhgb_1`80RITPk=m!YwwO|C z7$Q2ZD?pfvY+BdC^;O_C6{v}EP^)E5l&5ar!gwo~&VCe~fUJ)SlnJ~A2#8Ok*g|M~ z6c7eyL{R?)izODg=x6*~S6<0^jTyl*_Hh+ZKKC9K(7B!!dIdp5tZe|8%-3q}gM;+QK-Dt4v9BCY(PD zvkSo?UXF2i?`%77e#E`;0i@#z<(dg1aIjVvdKQwK7`JMv?2PG>&7t#vo0*OpD2OxfVVNy z_~6|vXnnqZvi)nA2NGIrEK+1;5cPv5G*(2`A+1W$MnE~isU_3sPoa^4v$b>)BI0|y zc6HS=vBXL>yo8&224gQT$;_G54V~zUtldW79sIWFR^5p&pk)$z{Qcv5P}@@?wgA^j z#sr)kQeOvynLrwKbRj(;>6d&5EJQ{dT7eESdKL8Z!cvSJKIn~Xn6v^`R>3qzZEPNm z$isAV(t|}(^9x&k_DEQvMN(6YO&}JaJx9#QssLV)TN&k#^8p*tk&qPa1rqQFrh^+A zma9>MfTxSH%Ezd5L&lUaKa5oRy<43-AoSDMjot~i%gR9AKnuA5O^O<1(1z&)uCN>c zy7axY5%L3^Q7Sze7sH-?4Dcz=7K7aIzy}Gd74wgvggQbUY>K68#n#q%ByxTW_jrCU9GCZMC9Z-_; z`h`N_r!8VvKs|GUSXT?=@SAAWWG9|Hjr8lXkv3V&24kU&U)?R`xt!JU$;I%~+#I^( z5pMP+jmlMS_PIq~cj^G-->L}~&*RVqX3FKuZ%m91zP16n-t74M0+|_#-#h04)1sN* zpL{kOdQsxV7SYMl{uFH&t{!lrn5%LRYTz%(;{a-W$#4HFe@|0-UYa`$?g=4sm}UTR zBc1Ai=15uWtp#voAf%Mtcvrm|y8B*^))e3#rE$OFG$m7nITY~`273S{d=HcxS6=`K zNh+eE2C3YnwRYb^=-Kwdd+@*%L!(@W4B1E+B*j;N^vD*_w$0V4Q<<`$0=e|ebpI8!Wmqx(K8UXf-aTS0BZhHJK zocNsR(}mvcUuLJG4nWfx==-5S^4XsXmQ|W2+h{8r3cLA z7I^?#r2>hqYLM+m7_V5liYkw$J6XCki)tyiSjLdM21Cz&`VqbcxY8{;rzW^^bqVBn zu|d~f=Y6E`AckO5)3mxGDElqhyG2b|Z3h8@(YqWUqTlxa*foZL+{{wI#0u`7>@2Hg zt`g&9mC)J&xQ|XV7Cz+cg?V?Zb}ln!L~UYtHRPEM$2w$E+_Y!67)1BElzcu1QuuNi z{Q%~85c|Z?r!INY2Vg7#K&@mE3%XCBt&PiHX%w5*X7P9_!8RjtK(BYrXUkbcMhQ?R zRiDDIR4YgcDCT}0mM!aB`!s*XNq%DjAWpjHO3F)V6AJ0~N@{#DB!VN1;5{_u%d1Jl zvuAN$de+a;Od$C3eAYU^-PM5*z|iaQc|W(~jn+I>o>~DDKvOINd%K8*NihH6`Y@%L zm(xoDTkS#1Mr)a;?1=X2F#7;VCfgh&vR^Y5MDv2dIOK_nQvWbe{r3fMBSLbE0O$1Z z<3pDWh!hjtpE6O~G^uur(0oy{*aNXrVr`oz?VPj+YtegZ2fJ*t5s zAlL;bBzD5pGVc?`GgMq1mHEl&(-KK%LL0>&XF%lC|x+;UwIvLBE z#;!R;BHDtVzfc-Ov93hXxFKv40GmQ^6P#zXsEv@*bdvxOUM#qPMvU;n;|bnG2n)}s zG26UOF_Q{9`YqGqfr_VlDeIMv@Z#?n$hK+#NC2)s>&B-2i3hhK)zXZlXCmtzU=^+( ze~WmNh#|3p6QL|Z_A7#w@wSIa|H)HRGr0dDD0201W~*`8ZL7o1NW7w9{x{ zKiYI_47}|U@F^Tk{AQk8FrAyh^6!q@aAiacgxy0V2ErSJQ>3Pf2#2@OA(Mv49w>V& zM}7aZeM*nb_;W6nVEeRxd$c$THgWRa^Htic?fv?7%E=69Z$q0|x1HEwT4$z)5!a%F zrI(@!wvlK$tL34>^QR-fBs#w9dt1J#S!QoctVk=&OPfu76qht#r^gU<_3-@pi5PL9 zuMYJzc(v$R_@_J}Q2_hXm&f^emUs~=|7jukWU&|=SUPkAdLoqMJSx9Zg6A?1IJ)MU zX*zVHe!kJeR>l=|IG^ia6BpJEau_vTiC2+c5~WPu%ogqU@EoN|l>(JznM#)mpOleCX*j7R42DEy2%D~|;&E>fnK7y=vG0YF&(!07Cx{i!iH zp^LE)`aW>m?-`QOPg?^tdbSM$ZKQ?a!>YQ8xzX<)w|R&q0t~qM5_t@pla>?a9B^{3 z+pl~cTM1e^dnlsndzViSo4$o-5*l(yG@PSpFJ?%&Lq8|!bZu}}Lgbq+tW z`;_OzOe3vxw(haY^U1UT#MjM9#>BBqo3Zz{wos}Hsbua?4#pM#xfN=eb&!^2_+-;Y zJyRQ>`JvKINw@!}4#-Lfqe2fXyDuair;CR$=biFUxm>&Op6d~7_uywk;B$hKc0+tDJ9r=Lv{FqtFuv|e zz?E|y4mTp-jV`Kw#UbF9<#yK$-*7fvodLXh9snj-U023#{{W0|v($_C9WfFA&X@uc zoHPlw%7G+ONnV%mS|}>Zm%8kI$?<_$Dk_DI%H$4G^K3VgO#w=#d^mNNXBs7VnlKVdeZ2NPzK~vT>EtM069Wq80z14rZ*(C_bt^L6lHH^=R zlAhKg`;qe@OSBBqUdWEzlhAYGcA*h|Vvb@lTFGw+I#ds}ij=LF`xa9E&c*0|70;<< zMi9pblBgP0j#z{Vt~Rg!eQH3W8v;HG`9r;@8G1n;`U9n~y|fF(U&%&3R2zk!{wMAY zeWN6L+wlwzfd1NCA8fDPoP6VufKu8Yih3n9ON)Qhw#m|BP>M#_w%Pz4R+JwcrrXP1 zT=4I4ft@;_6X(4*@}Waz{s61Jg65I?gTLY(5;+hG?my0Dvw*%)t_XC)yC16({yqs^ zNZOOIRyTj9QRZN|qwsie z>o@38xU%y+xj|{l7i}$b*Sgx*mJq(8c5btO^V@aOm^tpk`t6W+~ML6-hUP{hCdkQ>8JqD1jerfzE zk|uo>b_`xd_|Z{v;``1uBUjoKd6T=QH`A`(FafRAoprISD54Xt(%Y zc)$N>y4QCHUlJnLE@Iszk3ZdCiz7C1pZiVd#k@fpeG0alo6*tsj8vI&U+qI8>Sw=C zg^EY5Lx69QU;?s|)E52gjBIIw&tCpnUM@~x<7S*=dboqf(nqAZ2o@-y)4C&{#F6G& zV`t2@hSa)4q%FM@Xwu3MSK}t(3Mehf-zr3s-&Q4%OCM=PI!%16>7Eie8q8DdkTUof zc-|R9zpx3Ai%N0#w5RWt0k6m@t|6V$vF;uHdpHt-pX%9QNc{%@3Y3Tqk=rGWTqr^8jmmq&;3m>iDl>py-SKxIGkQt`ZlyP)#`jNc`bs^Tk>Tdr0!~v zl)K6H!JBDIC=ftMs{_;}-g@QH#H7iKE?%^OmZ5tBN;Kx&7~st5C$k8kDx|lej|n`A zudDD>zc50>%_H^ul3AczDv`CX8a2(j%Gjg*9-5jMzajCg?(JSQg2J?i!AF_{J?j%@`foz#l@Zl0-E;X`Z02nErBE_wzDdSgj&ng zqff$+UgEeTswN#IF zaNT%JuS3j1$VjdtGAeAD6cuPyyeW7KhaOzIj;VZODr$1smCf8d=>gW|2-d+807{B0 zV-mM)PwWQx0B3Y*p^$^y!CEy{r$2}B?0r$snRceeFeQK)p3Y7G4V2k(AbPv8_&x`R z$X7x%>So4U`?OqVkmw_lN=yi$Wui~4T$MFkEz+{jDN`gVK&{=LMH3Gb)D`iVc%$mG zCVC{7vU%HQ#8sr=;ihGwnrlUzaI3;ra>(fjDXOyedGj|?Qu?KubyBgV)WG(%hwSAd zJ&)~Qle4pKAN;JH@#T+jv_Ev1`Lkqr?m#E+&5}dFGx+)#W1k=s_X^@8CWT!z6ew3i zL~*VbR5^tEFE$HMALtyGJ56Y1;R;VI@RMX$>4#-_7pnnU@?i5lv2LdsQ~_UT--D-? z-(l8V_D^d3NWrOmG^o0#UE(oNGB8o2--4}tiN^3tt*=a@76Mg7b{9JCz!WXhQbe;t(JmRTDD8=>xpLl#P%&qz{i0x5r6rcoqforu zFX*$hg%Vn-Tx$@QJV=V--@^ zg;VNoF{tfVh5;x0MYynly76;Fw>xA2nQI%FgpOLW?bErL9Jp!W1%h4Sof&T(CTGD> z_mMke=T|!cpOs(#tOlIgzn_^EK8Vm;G7~-^ZuY{^LdhkP7wjPFKOkbWBQorYh6B$i zT~klAS|Fc^Q~=<_N%G5Pr9-Bw-rsVjvf5?5BBRa$OM^_TACE$aBNZa`tpPWHh&|Ax zQUZgSG##e_p>XXB$wQ85=X`(f3}8}FiOX_{a!LejjV~l?0H4I*<$90Pac%*eY6J{A_xDx{3oqgz77IyeZVLbDkoE=xu`pB-%%apc^0ra zgx|_ahae;DJ0CQV4&AWWz23c+Um45joZA$Y^pCn4GqA~n@H+s%E?2_xcret8u?0Xx z#AO1YUOk(!o_k3GsZmrR#w|oVT?jm~d2=gE!s{?1LpIdZ$x4!3W4h&3*UNZ8ahsxB zk+*1~*NST=9pF!ZQ4u!;Ft9{*;2Y-xrm(#RM}@@l!0|EW0-)1T9`uq>P#x!F@`l3~ z?wRbhQKtb;6)oxF=LgM7&Sy znsf<7-?wv%Iii!VQnb+4A$RDO4b(j^4#o-rN+M(o%pRYUc#en)*xgS?q;gYgS#VqRy>^7o!og%4Zc}W7EnWb>|M1(34E-{Lp0OlaQ0uvnH zE4Ny-E#5QorD#G=St+0ozNfR6`V*(2Hbma`i!+c&THW?Phi7?vVJl+>f1Gu`iA^BC z(nhDQy@TP&Vmtjger=6Cav;DYHY^{+OY3?)7ro~BaY_7x$WsD zM>KvqyX_I$MJxd+qz~`o)34)X4!#6r3g=FpOy%4w9>1IcrisTi)%ue70n73$;evBO zbhl2(s+yUoe;=JrWn=$OP4txviu>)2^Sl5{>6O0nC?^oQt?-E?qf!|Po78iTPYpGh0pOaU(a44sV!K;uHJ;M&2YW+_aH_c;b@PxaY;A$2jP zLBd*|tHj9S2W+|JF`_;%pYQx+R51sF&p9iVTTWodM)6I5Kp2#mT(Ij&sflI<;-14| z#KB`sbNA0sXESXR95h-U$T_jLW23+AMVEj)0zGZ`C9f$E*up+6QK_Db1E&9+${t5+jC;~E8zW=!fCZ$aGo3z#O^2HUdJJoA7l`BFXtu^0=@>W z2%1r1vBdJKlSoAO1KZLB$eCqaDP)yyo3{)UJiD0-V0cG5@ML%h)lhAOdYjNZjy8kp z{DjL(;4Pz&GHu*;$6PKNW=+H&sj~oc$jz^M#JURLQ8Vh}Wefg15bP9}ui;L1KG70M zBRd!n%LGx!@f5Vb((oXKG3bPJcmNZGc0sL2K{R6*rw6!NT)TY=Vil%qyxVuHTn<1* z7fxhvXWE9wyeyB1+kJ=2ZFUKKJi@_R4S$~syrC`)lpgb>|D%L0vIVi&)#psyou~-! zpQVbcj6sebgk+#w0R6yM1;rd_5+EDWIr{ly;W5v9h-RPIsvV6>7(~fTha)2wzXx!Y zSF5Bwq8@&+H=ihZbi@$FbTjccMV){YsQ{`YZ;}hJU_?*9%g-`mpa!qB@&s=wLeZRd zpvx<6K=RD4iAZ_T1<(uGF&07mCe0FtuNN0IHN8_pL>)2TXX4y3FeiF;LxiE)94NaNgMNHW!X)v;6X9Lmk;hPu4sM}7dam_hsX zk>W>DAIEmVO|gmTgAMy9EQ3HT3gn>{Skljv!25rveg!T6_sYiMIhC)l8v6$7@!+p> zADS5x4d`-1A@Au*85JNg90t1BaI38L$SV|*-tI_}TpQGzcn@U*Fkr*p8pt(XVz(`W zsSc5mPj&(c7$+6t@f_P+ed*8{F<2F^a5t>x&P7GK=+*+%j4AOmG+A`_ejGGA+hcrMm!JZ z1dqMq+jv0?V;bVSh)>3s3BivjF2OpNOLqGwgZ!EUt@B$gVo+QHBRp_6yJ|zC^J#*9 z#Ix$qg>}{v!z>Z)NT#ZHY=eBcK^!WhD}wI<00rq&i0KGukX4dCr$JxmezZ=1@FJO4 za*lKVl&N6?$fq=6m~=0YN_=?QZA9A->D*uqh!vKVR8iFC=esl<*7D+WDn8-t<`$uE z@FGLw)qIf_HrC~K%K~3$jG)m~I~SmV_eI+@@hXu#L&1$^gwFl$yD4~mb71ZbzSMTh zFW-iRS7Jj*82xpBMHpFz>4}kA~ zw1Y((Wb;CsnS2%WRARsYu9_fegu9Xp4_>Zuyd#Q>ekS<5&%rnD*eJODZZ}FLh{Y_# zG7noBPyTR+bx`31Dhva7BKXA0U?zy^eKOogMOK(XdyoG>dd4{~USbSgnDTQ#Z0cMf z$>~99Cq`-C0j)iy&=%f-uYI%%_byIQb{i&*WtR>thC@_iFfzKq+M zPJJu-R*EHoOZB~KAwdjTxPt})QpP>+|NjmYG>KVr)#eXD3_dU%`lYlO?`e7x6_AN| zVN*8GbS?`6`ltoy0%pt)(^hxoJ-7S$*+1H1!p0d()(Q|WU@^Vd)G`$rDnF_Eetorv zq<@7pHe7p%wItd}^im9XXqz$XY(h}Jdo59=0s@t6{z@6rkBmn@AsK`61S1~2NclaT zhiv)}`BkhnT;fl5lnI6D4w2PPRc|=bmZBBUt6JI8$Y8EZU#s+%+epE3Y|B*%k`k+S z;*p#-mcQj}`U{*z5*Q*g^LFK}vQ;0H8voIp`PYM!)5)IbJ&;oS3&Ck~0{Br|fI8j0 zitnFKrBko)Q8Ref{64!SM{5%prDWD|uMMw5TZL24+bq!58)ZVDt8lq1XL4a$_?)3t z`B3pG1A}WuN)R?fSV@D?RPhb3(}nEJNbhmo5VgPxt&DRN9t*|p;?g42=yD&c*yOH_ zi+aH7Q@NRVQt@Uz2#08%B4zv=Is50h48!|}X#==28UnX@?* z^bQryQjqzaSw!Hme5h^YbM+n$hL!m2-%u-1{SZTd#FDx@B?vE^gkR+(6`=fRNS~{u zdg2vkn3W82XCl;Sq#}s$lQcsd_A=0=X(CC$`DP(UO+_eV)B`AYgj#v(mQ~CJ4>_MM zku#0M1HhcIig~eEwj?!4n(jJZ(CiAM(wIold#fxc1op0&9>wVH1Yx`Eh^A!~&O*xh zCn-q-g`VRRAtp28hoa7YMHQ}L{_fZL&xpS}QaL@%iSP_EGq8gS*7{ZO#6Xo0Wi%WbKu z;+t7ly71kk<$G6sq=N855EEc4XF=FKlVw-{JYG2DCdIe6qX29gp{MYi`kL5-x z1@0ZOI2e)&#npofH17lG3QMSd_@%51+LWPdpK-Zw44xNCU9b*86yL=6Lo`0HF2pKd zBBV%y`HzV2v;c-r8>I#i8=2jzHh*F~*>!J$X^FZwvK6dE%^HnAMc#kxD?V}%MJwgN zp`fxJr(s4QgW!Q!>}|M%TxF0j%?!xn)ZK2$XKO|kEp*CXpI3tk9{4|-_i+j&>${;iI3?cRNHRAJNaHD| ztIhl{-!8TlQ%ID-tRGk$3P6M-Xzba?Ipdad&BYsDiME7K7eK*@Kd-|8+B_QRcI$gU zILd^$<)JPaniIHNITD9W|5?+#XCrhP@Qxt*&-9(ecNB!yRpurf{Ki1zFPk6gzf9k_ ztbNL86Nb$=0GV8jOWb?fZJu`y@*fqXBVK#_USEe&=H6nQ2G=rv5`Vuiz4~-aJkLEp zn{wTyrwL$5-&Q{Rqn5IQmb4GnR11K^-e4=`o2oWTfcUx;Ku5x{B%{Op-ROLA;kD7& z__AB7{xtLZDsFBSJr8^Ctdh82-bVErsFGptkziZYlIO{5>QBgQxpb|}+Ju(Gp{b

YWj6Y? z+$(V9eRXtyl+l$%4?CxvkFF*U?ZZpYm$y(&?J^~(slv$~u9kab*3-eZua?i3mTVML z&-(X{V>2eTnJBbJlb#mG8J9|;POFT4GX@lwY~E~m+jYk!ZtWBV@~-8pZdvKrs@v$Z zb1G^sy-9ppuj*sXRWerN&1(5%s?xlpyZ3Lm;=fely5TRVds75Gf))W2i-N#iNnfIv zdeV|zyPnGO0oz@Z0O|K0-6+MALh&PRv4o{uglyr|z4tb61Zc9(HG{LJLQwTI-7Y0F zC<8&op(ai@GcB&Maivj{3mpZMC8IYV?Bcb##0p9l9&Dj5;~k32?q$Vy)uN1SZ>)14 zseEWJa5~YfKh-{)s+}CZx0PVlA=3yE*?nu58Xl^q|LMg@|9}9~G{YtTRAAe(nnQhk z5f*FS9$`&KTGT_cXsN>X=wAQcGwvDCem0s96QVqRoqknpDUY_`_jWw3q=)LH(q+;tVwp5wm7pdWk(luuqO&ME| zn{u8RUllznmnDJA zuM2koEsI6!Z4yl9@GGcM7tEYV3x={SUL$alvYdcW%^qxmO*HPTq zl81M-%PftQP28ET37s0H&53zGN~??ne|<^Cay`$x)G3tBcx*lI9+|mSS-R*6`--63 zq!~TU-Qod})l1D+k?zYvY|MQZMcf4_Mhflr3W(G-bT6T2`rkMIKTXujSv6{SJ&*wk z!e9S>b{dZ2~qUMf>1b|A15DU8o*PAaQKYGWh9;dZH#??gB!NcI$6o zpk+nP65VbA;l~_H9|sSIqijj)NG4PCfORh6`^uI@2Pwq;kf zVR({_^T>QF;Xn1!6A&5>ypP1)h>8gO&(Si(Wn=NX-=u4Z&~ENW$~5oQ0sam!_MixZ zMA>Q&2&p_6B}Gn%8D?}wK-Kz?0m0_;7F6%nAobe4i8y$(Ex9bYo3|m(oprZ(IHze+ ztE{T(CY?fnlIvZ)viA-5PC*5k56$f%Xah=3WAAsBz&J*$eabQ|>e??hKTv6ZJ_K=} zeOXz;r(59=_31e`iGitq+YuY<@G;5}nqfYdq)}ERS)0IJ1Bb-f3(r3>%L8ujmYzb% zVScv`mliigjU`w2n^yu^^kbB4Jr^aM+e)YpnlvErK>swip-6Or=XuVg2}AcJc+#-Q z$WXsL=Qpf&M-6mr9yWl09)OL%JHCtN*$wj|yczGj?*EYW=3WuSU_DeLNZB2i(KR1q zGHh8wdes6TofJHE2g%asOsR2}zO%2wMQ`-D&~gv!BT*w=cv5{!g6*VAhaoKYL6vFV z>%Fbrqs|Mj>S=<_LZBb7Lf?J!kHl#BL_wklBq31ly{jfeN z)%e~ft|&SA*Wb{Q0og|lQ>;8nlODxh&B^qLQ&d8cU85d>sbI%}(V&tQt8>QqD8yuu zBJG+qMibaEYXGBBhe&1|Qr-aj9ERw-9+ME7SlO#{iHT25nQIMdPbVGhI6#y_&>)s8 z#@u1D&wse;H-Ki3oXb)Vr6US}Jj`@f)hA$tSIi~ofp?Hn1M%;woNpaa{FE-sP+gMl zL4xAklp<^2ugf_h)OUevgDaA6vl(EY`@nA&-VQkh2*AXuzib0~2#vG6>I&8+!3HG% z^aQY4EEpUGERr5!8ZLh^;sWJo(ARJN+rWI2?^Ho)J=%SNoEYLDLt_Ta`WN*s2mCJi z5$wJY1jEw7yom><-3tIzxKKx3gb|>1Dr#+lM~Pm5emcfLSBaFIbo2)34sR4hI{`I4 z2Shnp13E2SIQw82)nRpO<|=cK9}RFxAhi`fmH+DOCm_^*0HtOn)&tv#1K$?+^b8CC zu0RcF1+Lk9yk*jSgzmYDEl|w(r&_xRy zOm_3B)=YpxIHis@_Y?I7B^y;yfD~&Dk5}zN}EvKIPel)6drR-3*=5< zxB3pizFNVg-SlCQFW$IS3Ri&Aw$1Wh9e}e04+zBeFB)eqxRw17r7G)p8s zYWG3W%DF5GdW&XUsTu$fkZXc72o^$G3LJeq$cT?2MgAIXrhSDtB2~-d+|~~Hky7wjtxjpz9$H7*QU5GDe6TxfTBIwkK$G*Ez0!m zB#K+l|lB%b{HvH*RYyTsx0f6z(g15623jVWzAvX9sisiXfX+0dJMIJv?n37ZE z3%Drjr{KD}ozJ+#pz%h9uJM%71z7|~H^2*zhTu{2IBGC_TL+s2VZ1&40oUIJ{EY-X zR82=+@?;T-UiIi^2nMM~5W88xbAnKNA3(u^!Se9Cg$bCD^to2!P&+l(2hm?)&dQ0k z(4N>ANP0(0>2#i4EQs?0RtgO<&x^2~gjhw6k$#99&DH}`EasWxOCiEq3Q{BV?Q3rU zR{-|uB}_}rTeGa76lqX47f8TFh@5>gg{}JY8eNnd;+HV$vhH;Y5Oc%Nt*kkLstW1` zRVw2*PgG+{jqpm!-%oj$n-RyP7RGx7=<}dh_1TTWfW%P8+7<+ZVGim+3jm3xFy97k z-3Y^gqI`kNhNc0%?3uN|A9?maNQd#eCj83PJQeY9a;?!l=@^mA+QG5YE_b=DiO~ zweA>zaO+b^;5^+LbxB*YGvF0P$x$qC>Ht_Jj*(J9443)_k6z|#bdQn*@AQ^Kwe~8Q z&{4t+nY)^uKbQ;?e)5jGt$y;IK*AAk$PF?=U*#139*aG>y+W$NZ-58P(V*>}!YI2WFQJjBHv~}lx*UQJNAHa^W6}}p%x&MnN1!hZU2a3a#+sLu- z*Grh?Nolj)fK91;w&FN}vKQ#PwPuERPrl(~-7=U&W+w|heIUO2t~bW=wdk!3ISg7= zvWaF&yIqM*l88-BbG14sipU zZUab9OyGv3pmS^r3Gw6fV;t&2!wRKwG^0jZeE5Zw3Z+_i>CD`iIbXg5*Q^mw;9F1O zzGP(RBHGGVmt;&`4GDa%R!MJDB&t0T$9C5p@ZS&=Tl04z4P<;}PKGyaS@DxV##6d^ z9K)Xyc6IoWzs3RsACgl`;d}=l9jSYgtApU*61equ1FpiA5rmJFZsBVcvZS`l88)l1KI+g`SSMgjN3u=F z&^H7>Yh_*ooNdAd3o?dVArR zkWX#lAN!F7j%ccU?aRif?&c@c>s&F3z7o^Z4#-DWX>z^Lndz1U-6F; zM7}2BwY!3G071067=#q+lk>P)qL2EQWw9L-%ghw3bwWr9Vi-z#z5m^$63`}HXcWy1PIO{w9Rmwx#&(T|>8V+#jo@45W(O6renqns-e{aEr;9ya`Iv##@$RAJ99KxN{kOqE_m=Jroj0(hu=a8jIFNwVcWLTi)l~C zjZ*GUzgZ_MKV6Z~6T(FOb@V)doUW#~vpXEnZ^N^(9AB$Y!@H`~Q4bF50FSHf>W65} z4lw;0`vwmB^h-I@3XFSLx#ppaiAbjp8j1`t#>yfDywx%SYZp?X4C@|JuRIe)oLJai zz5xD6#0aiR`i8{sbL>Zo6dyF8vCfJVPfumVaN6i!3Dcb`S-^dr{FM-dl`o^)3K7h* zF^&}?QSGw7B4F?akkBbgY}0C&WoUTl$6S3F?*Ih0k2=KbmoId3%j-38Y5<8x(J+X? z9O1pdEG2-C!G07`&_Ib$B;%%2f0~SIiF9i(Ehx1+riT?eSgvQzhl_^Fo|-REpU!6} z;FxaOgJuKE6Z`!F#y5~gHX_u~bU?yV($1|d=Z7V%1W*l_d(2_G75{lskRwRuc4mh? z`fa*wET`A1k`E{ElBtNmt|Blz9!3}4YJ6Mb-Nl#`VOcExk?2cn;!-LSKauP1TevEL z-TtR4`Na3@Iav-o+8Tx!#40SIZq&}byjri4-RDD@WnFeeug*p$Y28A zbXNOZ47qY4Z8rk+E^bnZ3Y6%DTzO!bpqESScm%+c_OsM+B4zn}M=NV~*`&+lB!j9Z zVy||!_=s&f)Fq7_t0F^*l>AYJ{Ikmads=FCk2=$nrN%bR?vnL-9&VELlG2AX)7Q7+ ziM5@>k`a&V^><9k$}SzH(+F7E!ap)T2(=Oe&OTUk^o}(?=hD(8-{a-a zXX#C8B^s|Q{0Y!u@UG+mk(KnFRvJ=xx8R*JR}I)q(;a2MQ-&MAW?yiV+K$~5A{%m0 zeC2URXZx_^KcE6+F6G%|OZ^Tv+@Ki0Xy59)hPv6`Q#(MZ1U0Bs@#<6 zvnj`pmps}Jr>+W}KKf~<&_=wg+!EF_R4zH{Pvvb<=v!mUzJ;o^y#W+suD$+;o{I5| zMVoGH(>6U4I`OeeC6b}CBXNPdJN9ok1r?j$u*NXMyI#xNv^Iq*4beEa;$xwE0xL;2 zgSjZLu&>1IocfP@*X6cuE8{=8{09a4A4}gzb!q%%@gP8edBHMm^fNxKwP`rdk+U+nK_NiH$s7o2Gn71;HkzXA&L-2~<&wNwajqipq_gE1)YHch*2;O(< zSyn}b_|@o{m=D*qXWJqKvzQ-Ip5wVJO0xD`6mkBV&ED28$p9p#e|VAqvE3{pK$D1I zA00WtGi|mG#Xznb-mOfK(#FBb)kTycxg3End7kh6FK>9S?LMO!o9kcv%y�%6HZ2HwN;*%^^eA7X{4EnMDo{+Pq z#`7L?LI3N7(M4ZJ4zl~(MLBRsYc?6^oMLDxygRu_6xgJ|EEHTg>!+OeHq0{86r-$J z0IFFBlPn0?j9!ZtjPxk5x-OEvS*ATh9q-vr#Op-j8>$=I;#DPacPA8< zd8^$wDR0uQ=~_{^FPSuzD2mtjaN(?=PDxk85LKViuEf?j#q2Rz9H~7~WITP(5-}92 z^nbxhn9R|*n6FDh4j;gj*Y`q>j7%3#rqnNYdzN`NKc@H@@^-Kf#zfJR-13;|290Yr z{Sv1>+!?mvaTM=Y^#k&dvf{XhH8D|5Mu+R~zPZ-ZoG;NCU+%)C;=4#rnl#EPdp20( zG=N%XXHoF**Xa2+w`7m6Wr#z6lgULw&mM0NwNzE%oe7pvVT-sZMw;i-iB0F`tzzYy z(uRkuGyif3|H}<@9>aa-Tk9Rxv|=l{+Y)UXXf(Q2eLX>%_=h#?R)wA?F`8Ou%ag?> zJ}Y#gLTpLELqpw{RiEm8_nS;#n-iU1RXz2y^jRV8*+T1q8{z|kTe?g$Mm>rq!4fLU z#rhuiv$eSV)*m+4aShye^$Bs-l5f1@-RKJyOKL< z!7o$WJ87?zKFUtJT=f4gQN{CD@YjI|bbBGk2>(R9ZUujP$#%j&TJ!|i!WN824_x5* z)T{cAD7c~IOPbbg3hJY^e13`zyfGW`X~KgWy@8fS$NaAY55u|Lt3bP?ks{(cqC!V`H@bUPUCX-Zpf9+FIOxr zvq}nIdHZsL1t91v;;Lo{I<^hxnqEUr6vpfcrIS7SBAjC@WA*qopp^DC&2`%BbYAw2 zAGlyGkAFYjB{cBHZ(@=4WGci?FPLU&RDGd9{YY16y=7Bz%;Uq)ly#?;_3(I&l%2E- zz)!JQ|F<-NoEv^ji{b-wsTU67kd1tklRe$xGlmy9!ei<1gr`roOd}_$fZcwt8{d&efEe4+dvE?mN zGX?Fb-E0WEDTI`pfS9EJh$Uyl=)reG*SlX{lC6AZ8qg1E-|Q23!lIx{AmX-lx(i@b*JdTHJ!^}5+nBXs1G&vO$o_``m3nw*u5}oIvh)M40Pq_IEnG`C&WeRvLA&vaYS7fe zWJUptW0iDZR$v7wCP6@rnlJ2=a(}kN3iHi7H_;PfZtHudXSPiu^x|FyiSyM+4MD!u8UTS)KoFYlXS&_m28zee}9C!3`h}m zrZDmUk#+>sMl^?-r2zd-qCH zLB-5M+Dgt}&vPsQ?rhTT{QL%S>_u_GFYb3@7RUR0lF~p>wR5?+HuQ&NcJNgz*YAn_ z{e8g@FXC(MSRC}${8PYpr;E@V{s|L6djnSdM106^MO(HL_$>SL9*IYPzB<_)e6@4L zS3HQ-nB5E&qemBetAOwK`w#SheCmebdi2}sE91pSXjqHcEl9OPuR%5wO(|fdseS)Z zymsuWZt8#8-fL2UZyl{4%lzGL=#{1=2`5avig`l`2moiX<$gb^-_cAr z00ap}`*+O0FT_tMeIxM;d<;Pk+3r~F0MJu06<1M_{)xpJ;ooOM4jxj;566Kq=$G;V z$xEpG?RV7)d%*BLj4o6OBOj69J<|TS-vNC7!L9t4p?Dma1*Vt_IHmj=NM&uhc#5ahS_m3gG-M~5KB5K! zm68bg{Xv?D!hbrxrs!ngN!}S5ORU%bT9X4^4JG(U_YWtH zap-+qDqexHYSs1f^*|cJI;6r~UAMI%#w`HSsR@m1F$`1wSz-Sx&f)QwRZ`CsuLl+q zfTt{8QsSUy!>n&ik@N2Z{eMJ81i@iEkQs^EbOJ&XlK;@k?`z9evAFzo;aAO$ zg;u@CcL{6g{*+$62)c$Iqv2zv%wC)uQ$oTYRCul6Z(E4xe+Nxu14%&Z?Ry6BT@wab zCjho{+3j-`4CQ_%`}3%H{O>}Yg~g{w`Oq!R`eC4!O0c^DbFLYAw;2o|P~)-2c{{|E z&b+j=2lUod0tILdu74{vlqCR;3{790 zfgM9}H*>%fFir*T-I?T}m(usP*Jpoq@DqV=^>-ix4c~VBSv{=&+otxH=Rp+wF8gs- zb>VLX9Me~6@Y*O=AA7PfeMMrRwr|ssGt#EKK|v11;&nv#f${Kn^uTrU7-hqL))fY= zOn!%+X7WLj^X|{l8-UryWenKUp?%@U9g)Bg`F4xGaQt@xI*Ng=e`4Ezxe`3WW)UsM zfxEGY0p2OmjF>o}Me6B^SL{V^zw^MnBKKA4M$EvqR}+E*Tu<_7s4bTkfZRzHEM_k| zQy0)DLDanTOo{#Df4?3B>7%9vqb4xeWTkTz@V8XOqV7V%57?Fnpi<-+9o}URuzvNr zn8o5g#{JD|ybXXB3baZYeT|)=brprgO21}zfh=ACwo&r@xBm&)y9baed-`W`hb{VT zhUjo|9BS$juY>i3;2UJx1TidZ*VCju z29SDMA`n(p1FEIGd5xgT-{q(eoi3g$;`T&ESI|Ng3JBxHgWid4i0}2rLGyfQ;b|y2 z?`u4Lhqt~&=z55-KYTM~-Gedvdd9RrVP+HZ@!dxG0^>1GF~`!sYZDEuP0>1K>RcQk z5~c1=V2gDIT)G0N(dPE7??mYeYu`rt9U!sbc{*9mw0+lY(O?SbEPfvby{zEFB8;nm z?}zRjDb0n+$iOP6T9xk#+NP1s&SE%y^+GISLjLy%JN?4XeMJD*e!>M@e}DkZzCh)V z>tM2UA@F&au(p4R1(Xwz+j|aWBjgnl3B-q~i>`#}cMVZ%qD0BxF^;KF_*AD7D3k5+ zO7rlI`X4R-s)MFLT_{r9Iav3u(N%4rOQJ0AGkdg!WQEL30d|G^BOQi zPC5jn(24!U$55n&&U_E(^vv{{ivjl)kF7+yE|YHaw6684NKD3gw2#Sh6OPL*w2ONO zXN*fU((REHV_I@fEq_1!7%cwpHwvf#C%ak@P2$CZevDE=WwoBHw(I&#fHp(A8z%ax z59<35$b|uTln;^xFY*34^gmL9d?KCR+a7GBuO=Ao*ke~kd*`cffFyMsp(#(Z})ou&Fdt0-w_hEJ;}a7GY%>*Z|5yt6Z2Z(|7%lLX6`S_1OEfU}6lpYLpusg~ z3#aMs79wvPYRYs}lGe_O!fiZ%uw{AVx0!+Umf_3jMOOu_H0?bMv2>!DjXFdk+k$UHNGy3L<^_(7hRu7uL?1dC8mFB7c%wBS}%W?!ap@8#=Y+~ynWD{ z9_v}s2u0gM&&TWgr6s{uw_OjIXnLHedx~#f<$QBmKxC}-P*t&TKh8va<-q})0@r%n z4M&c0&q+q17`vp*stkpX*raImUun>pVlUu}-t?vzHvZBKx6^9_lyXC^n$ZpDGq~6Y zF9y?3N}I%t(9*}|8Hjp@=NXM~dzq1o*4aN+i4Pa`dZWJ7Of-fGhgsEQmu_43uUdfUYW33IK^-br-@FXox?pT)YY3q^e6K*L*b6qp-l$N zLh_9xCW|A~UF;eMY+UQi66+k3!+2v%v0GnN-VZ`A&J+VP;6>=7v}RQVj0c_m)r(Hd z;Ahj<=}w^LJ1~lMt$%s}yM_7%IDvw0l(%W4Gaw)p;{0ED?$P-N+=5K`*8r`dyX0~= z%5O1wf9CSAb`1n+`)D;jv$M)2FnHAq;vx<~heuDBZ%jqH@QbKiaf?F`>ogWRqG+Vo zXyobES`#6P#swJtWlzvzpkWs-z_JDAIx9o^)Ak1*ZL5Bu$HD*2NJX-{y0!lQdPl)B z3V|G$Dj1l`TcVoem~}?-Sn!V$_;W9iuzf14ZwMcaPIkP`10Su3h{eI;z2hIEz=~!_ zmA^0kP1p&8rb2^oNCh~FjjVxNj1uHE;7lf|lm#l}VLD}X98zCtd8LOGM zLl`hq46{brPjKEZGk;Kd!--dIh3y?k3}1ucDPlcETirTT@>J2MfLhuRQ|c6nkp=B% zH|!mWZ_n?ypq+lOYHS~$EtCI&|vCxT-Ucz3DB0^OIiVHImvD--S%OauU zg?E(1q1AYy%a?BHUw~bkP7J1IK(vsRJ$Vvi0nX~Lveuv_+M90~BTwAr(FPSTiT5G* zy~sP$fJ0%{{=2LTj#5A$gQoTcFnVb~oeKf*whBC!Q<%RCJnl_KG{Wtw zaRP2U&WAbEz5}VFkR8;F=4XP^$#1M@R?MO!ExHj&PT+0#W9V;v>QOO-G(L;Xen62o z#McAbqbAqTu6L6N|B&TA6PFc|_#V3I`t;N#P4=<3E_iaV<3fND+yB(jG1xk)##(|iuf2h=k0 z@5P(RisY_ue1D?-OaSHCfmL9)s1UkBwle|CC9#Wo&s40s!bgNtkXI579;V1CTIfpt z2@ht5H~#8I$BT69%p2>wW~Gfk8*m2n9Q^$cr9&BxFV^*2?zS9%;YRtBH~}YdI|9b_ zAXC7%cEK^b#w!fW*@nT3j?V;)K5C|HTqpifU7Z4GH1SDeBdEeHKi7c8S_nwaLUF&1 zOC4|Ce|bJD$S|vS2JFsKm8U_~n)37o>0DnMJC2r&l#q-SBwgE=`n4xz4Pcrci-1!w zd;6?SU>u9uopvKwQzoJeNJJtsl;V)&2inV!s?GhOJQ5F-mqufdiV*E&{#No7}8u7pBT$*mQ~zs)R4z8F(cCwgl|+x{VaZ;#V{q z%0gpSUR31KL~*QvtD2VS%usL%>wkRLA%|;`WMCFr4aSpL#&Z53P(>G>e|k?8pPu#u z#$1Pp$-~Vhv@7s$j3O{O)Phg=+f(-L-_<|Ji2IMoo-ST$`Nr#0P@VaGK9csDzIxjC z_2@whP8*&=HWdL+%v&w)wU8jRKtF`d2(M59ABMnj4*E~8PNJ*Nk>tC-&isCHOMo3v z?`r`^cxwn_^3@~+>^35(2<~EDy5n*KD+U2z+_B;>x`_Vya4v*q`E>p$+F1$hzzJ{d z3^WOc0EcT&_3=*hmj(nawa@;bYCtrzG`ai>gl1hkbs4oYKGFeH)i9T z17RJ}h;LWeTVKEqOi7(BQU|BRLhO9W%wKXIf6(gY5k@ zj(h;iXc*K-X~-0Xo)LqJmqGJnOclO_syjyAKhw2#l;+{>ufV}0`kj%?Wg&$T->K}q zf;MjGy_ph9oseP@I@a^}!sJ8tiuXw&1Q-V|Q3mC>hvV6oktWZ(V2T;MKUTo`UX72w zdnfO>aa6LjqTolwi%^T3o>?^bzN%q?*Hv{I77o29SA91cXXa^`4R|?YWU4WT$lKm= zm96Gjc-l}}=$OphQfKl0%LPc8A0yZPocaNTEI}`PxXX>U-639|xGL2~3r{xJZKz5G z+&r6T9BiVXYXdW&08H;9w55nCxUq-OBF;yiab<%LragL_12OH%EX11ik=sGXvvC(j|D#tnIe7Dm-b%gBeS zMAovHvbgoZ7*)Z9pI@nCuR@y7djLgf6ZipIZ z?jX}$3uKoa5x8Tst?`CJrH_8F_$Wr3^nbc`W|rNuW2eoX`M@oADcw51nks-hZIyd+ zK-VO|rN^s~w7j;Hd^m4{G*l9aPpX?pnlY7@E+@;{!=zdw2R7S><TvmbZnA)Q@b~ zgN%pkfPe9Nu*B+X<0^c<0Tpv`^eA2Rj7Y!cYX%*s%Ar%DOCO@Wuj-v9JVn z4!VwNJ*9Z`D#~$npJ-nmx_18SgY+->bzE?OriqiC zdaV||(XqovbCoRi0mtW;z6~?q8)rOt+Ky6w;kNspW(;jBT>9^Zz=}km<-I|BNqQ{f z$b^3pe!PED(Jt6~_Mlazd&EdwC*AoR z)JVP4@^-W7hx0l4P2b>VSld(32%0La1aWK}iUEW+%Xhec>ST$4*~AmquF5 zZODT}0z2_kyTw)kBPAh=w1edMD4DS^b7~>>4AR^XlcAUFYa=%ugJvDAY&gU^%xE?w z;iec$6sWf`4EPc#CB7QH131OTEV;+E*Vle6-)ZS8SMsMEdcL}pz!A%+HjUm{(bVm3 ziZd%OH9MHkHCif+`!cFd`IcCp6TW7qr>+#QPSJv&|0HY+-^RxVBSJcj?&DGVj2RaF z*H_rdO5fg&sWRApg9{ZpVa$Id#m~VUy|L;#>p#LSARxVB9C!g!wkXINlgy5i5p`WP zy6SFLT3{h+d1E!2TKvU>cb&7;be|r`HbsF<*nvq*Rf1s(U|~07{gbwVk;aw`sYT%W zs2H4;J&LS6Ni$obsT$rK6-w1gMr^t1($J8^E3AQS z?@sWBtyAXN?F`OXb9*0)^S(eEwn}NfKFHiI7tZOfw?<#U|0ChK){sy@Cv0V;U@WXt z(gGffA%q`rOKzF=X=jIE?B-A0SMK6CL!25cV^dcN4x%JaS$cM4j2Cw+{QG?C?kv76 z;P7c{&^(ZG5^tl;a7~Fe?9LX$?CKEcr;o|EUo^PWW2)`G(L76Dr5r8Uy~?tFSOY5U ztq`!a&kj;JFJWjzT{M+NUs6IXI2%4G+@MD%-y!0^i2cMrO=>384XZcXQI&orb0Cdm zK#{+YlO)}F_X%g%=>8K$@@Yvkyv{4bQt()|LQlOr!;};G3T`{pvT9EmBgL^P(Q@U) zs$sYXRf%}>UC%RY*DamNOIvj$+|^loURBV^jmvtt5m^ROqH305=*U{t?autt#qjtF zVvS7Qn#h|0g`-;p`Kk2%b;0jocfeZNIA{cFUO?-s-{^Ff`gh zpo)Tz6Zk37FY4+TI_Ja>0ZYmBCE7Ms{%f|US~90DR0~QBmf))a2% z@9$gdoOQnQ=eO4RZ&=Jc&mH^T_l|2{yF=}OsX`WVVqSnaiRaOu{eXP~PLBXWNCMS8 z4Y!Cya<=W=E5<;L!9n`%>EehEf?sdwD{a1RrzV_IY9VM%jRV~j%i{`F0vF(R0s($p ziK-z5%Z&c^gr>LnrX`Fv+th2MIwL6KkZTT_tV4QUYwhszLIAbT?LD#68@tkkAB*i6 z&FsPIM9J;hp&pFfhOcV*H52%P9^aFj(cf&;64;Ci#52vio1^$%?cBM-zk2iFalfj~ z(oGcotBD}3Q`s3Kxmko$ese7Di&o*~df!vF_=73k6oD-bB!H_^cd~A>M9-;d@!`6u ze+Chf>4kDPcYeEJ_~|}zPsPJ4r;mda0AuL5wdR;0Q@IFjc7HH2yU>-Lp?u9kO1y@N*_&2-8}^Ct zO}tWl(pUhP7`yw7rLslrH;t^+w!gJ1V+Jxm)({N7mTm8wql}J8XM0TM&$&-wleB?e+1_xx8qD5J! zmH&p=J=e*XS!6>+winpk_8I5;an4Z=;%b3QJu!mr@q6Whn)>nbJGCYkB!RtUco+2Y zi1-*3J5sDY8;fc%pOhg>I#ta6H7@qNHMPbvV!>6ueB$2eQiitw&N=HCQB!^OF;y(5 z)y}#~lV7@!jm-8+jeg{3z56AMw;4eki5uJw9Xi@euPea^sVJ9CY6DJf^56x9UqnL# ztpwwY`r@734+I_6^U!NQn8xPJdkx@pn$vlRa7??9!H}85=bP3OEFL3D2*5Q z9pom)rBpdnS;SrS!|;hbl-g;8OaqDS6BDFbsVaxsVpM+Xb8W4}UKtL-M8m^Tv{C9fZ$^j_Fk2VI>dAcFaetk5Tmw+7h~?;?{<>Nnmz!BZ>M5hgIe&HQ{< zy0wk$bo|UbVBSTY3Fb=l#j#!v`He?4DE>U9N+NR{|CKgp+N5*!t=+jY-^nLg%sQw`<+scyKRFe#WC2&+|u3)-Yp}p(O3+C22>$$W(s{DTe9HH>&ot+4Kgf zH+^9HB*_$XP9Qp~8G9QP(J;RRez_vS>7pek4G!X~F~uyW5nywI9dT+T@ul~&ihe$> z2~g)fky{@y&GSni{du(}|Ks4=iz3>11j1{;GGAho-@(x)V(l4;dm<7_ z2j@xv^P3Mhv3LX9EZ5#1!f+{0`l%h{o-$2%Pc55y!q+lguJc=}u2#?`b9t~qWhM$- zw7*-(mue)2uQ$n9fY#?|-lpeb=&GFNEZFW1t6_AqH)on=mt~CK$c3YIM&d>2z5kth zlGJ1)@0Pv8xS^tuCh3(lxo3h0PdzXpdhZF2Ck{B2*rZQ8ST2n@vHz?b8ThMx_X?93 z9-cmD3|3J*V+6Sdh+*Q&x5wu>!>yi8Nu}0X#Vi&qg8;u(eq%Gcx(Wz~5v_GqQ|+jL z-^e$c_EhUY_MzwhKOae@9iH7BuQ8I{`~@}+k$1eKo8^+N&oeONeJ3~|`PUDdjf>a3o73rK-h22BAtA;*7aNOvn z1AO);Be*H49h2uw*c>9K;P@wnur59x-~jm)^&yg%`y!;V6mX3?``b-htwMHa|g2O4PLdHPQ_aub@H?kprWO96gS1ttLmK# zNV@ccXd5%VVOA~Ar3BIa+inJ9Jh=C!4WU*8r~-h9!E`9cc%@4TdVDx$@FKq7Z~~!h zeZa!-STW7uflnIMjuseDPs@UQ)g&Y|djInj4azOAA$m-w5&){r0>(D(RwEZ~^n=G# z^z3+kfBQg4)D6v7r4MjQdVMk4W6TyF;3yNR!pj1EfI?{}P>Xdm)qypPo!X_htz8qx7KQ1{WO8!1EhpL(c=Y2jX z@^r{^udTsA{Z%GUBa^z%k`6(x2xzL!v5lkU80cX$#~y)hG~-$7?oliW{U9ut_aY67 zmL{94$td19!q$08XK%*28+z@5D=zkfSzJ=;0WrJuk#oRMT;Q7H5Dk%Mf9>xs8RG%t z^y31vwOu-7nY|0GVg=BMazmfyU=bj{A`J2-&buIfIfeTbfuf)L=0DfxP0PgVsS$G3 z)N824QCv>zMF6FFWdAon=41%h^ld(fwJRDKPTXz70v zT{TaH`d$E9+_=T@RU!an>`^!1l>5RcK~v&mOGr#zq^Wk((Db2Bq}7R?lj0^6oa_|N zost7UswxmEuZs}IGqrq3XML{$SooSi9O6;my9=S6w7Q@B&qTe&*5CgN?3o8LY;ndJ z6oDJnHFV@9D~VWw@(m0WELzBGUG)Qr97J7Xw>vyEiq>2tAk=>J1=qp6jqhXv)1tRP zb3e|K-c+azpwEzSc=hC$BK7*s_t$OmY z^QLZ4b`;uA zr#}gM`;(kN`K!Bq;23+1^R&#YHSvkkTS*FK=7exCSMIbv!es^^TzTh3|2C_Q3Bhu*t>Y1XNmn-(CNcMT&oehJbjG+&P|OZ z&MLw35IKKs6(eqBAnM4ocv|uw{{WvsZ*EGlSu=xqijHpGxQ(3 zy#pq$)iWXSf6CpOqj*5RgJ2c7=fYtpIpj1K$KU_=by z1(P?t^S}tSimkz@>V_AV>!3uRb%>sabgBSZE(w$i!~y@hg;i>HnJ(6ahDR+Dzwn<2 zS9Q$S%X=nK?J$CG^;Fia+yGay`xEf8M7hDr43J`Km2iH0tyvp9v|$BI;u9F1)aY={ z<>FljOI%NFyWhwNhFMD`RP#ZjmDBzjG&*P&9e7&&qEH?_Lk&FVs?D`VXv~uPl>dZ8 z6vq}Mubs2~+VUaEZ-7pk*~7)0H1N|byn1(U=$Rd1U_g10<3mbYb(vUw+#g)Yoh%DKHbf2R(ECJ{S;m zG3J+SNPuZ40LUuP24FN6kX_mSJ!>9(k5P;c+Hrhnfw45|z-O?)kZlB3zUe;?K3cYZ z_PH6n)dPBKUSohd^j0cp_4i2gO`z?eIlksmR5kuB!Uy-KhT z?a;)qq`+Gto6&am?Y(O7R!QisT7?d#y3lqpwz`unazMut zvQvGb75=z!^KykVp%sp8^tVU^V|h!IJqPqHI3LjVQ-kl_!YRxAcW@xZbVZ0{ffNiA z82A52ltizG$7J{5C;ABBMQF5obT5-h0!h$|az#$jXf5>lT6(Dp+I(UUmA+!0z6-YD=KzD@y~|Jb9K>M7p|PmVqU zG^a1t-q+yH27L!1nk%W(j6aY5etWbF-0$(-0{Kg^Na&vdk7H&G6H;si<4h9xL-Y0? zT$YN{5IUo9JODlEZUZzj6@d6wELOneEp&MWfN+x*WBCa{l;}d7D^LkUnvSz3i(@;O zqaWl6*)D9cytRiUt?Iv$My_oSsCJCf^~H3o?-crDp!$zL-yZ#yrqK!H1n`dw2;(3C zbzF7!af2_dOGoA30p@Wt*hE>+8$G30z(DJ<=1Qt1Qkky=2|;Em1KzP(5GR!mgA0XdDNVES}^9u5DEb%1TGfxlD z*H2|Y3svc<6D>o)ny*a@PgkT4b@(^+3Ls<9z4{+HB&fX9d(i#)0Pk2Jj=r^ve8@O| zzgbDFkAO3X-K=tO-TTYsqGdt?$=7RcCkm!ai)&;4BG%bzNC49K{FLFGs{q4EX2j`C zO|ba4`P4hX-VlGz|NWErZ1;Eun=jHR=c%gL44y@<4KZ z0QEbucIQ-8&$H@BHvVnY4d_0_LJ7Zv6n&n5_Wy{Y78^B!N~1Rkl3qKeno5qUNC{_0 zt8pm03z<4T;4EYfPGHWV&9eKwJ{jOX&kE^iIUH~Iy0LXYuli*fajGFQ{u2O`#hG>Y z{XE|*UGC6Z@Kf`|=5q_k;>p}0S9`9Q7Kt^f309F_0tcRKQk>{_H60dPwxDx`Z}jqQ zC6H6oAIK`pO?6b@+@40RQ zFs;Vjy(G_SFthI7t8tG1#R9y&yt{pYeE?v5G;d53#tZnLdK79iNH!p@aOw4(I(Fp< zGi>o2Ez}OSfHMKcp${zf&7mBbicsjFS>68UG9*E1D)_w|;=su7!xFi|j@o@=?15EjXw)P1-cu+VHg2kAD6KEo1E$TX&*1K zT)=~H!|3o#5*}p~1nt$#YB70bjDJ$6P|Pro6&|PU*_#8V#>Jnrq;s|Lr`m@H) ze>TUEc`eYOTzT^@cydhGfq&p`G_j_6&MG+Y*oUAU;=|WRaRmTFGz+R&ie4l`jxj{J z9gGxbM>W0pY7IXE1lb0FF46#UOU0Jn14I$fiZAA{EYD->_qy5FXdpodr^bU(6j;NA?0H zdD(pi(EKUSdxT+@n5Q76@5d*m*gHn$I?lM{5LnJ0?*g)SjF1IZkY(U8uliqw=AIOe zdg}OEFzqF%HscMU+Y#M5l&&`Uyy}!KB#3Ei{e6U3{b6kYlEcNCk)eKKv2fhw>#r zqQM&gdhW6l8N_pivDPYyNZ8a#PeL#{DT$#u3co;jv!?a}$2dL=0DY%Iafy0UpW2E4 zFEJFwfqTSh;w?F3u9z<=+yMZv*Q@Gz?6Omul$lyqh1yG{Hm~DP0gBW4@_GTh)ffPI zh|G!nC*Eg})vSyq+cOWOVZJ{2zs6ny8%}EVu8o~0>M6=e;sh}D>qG5{961W1HVMue zM6vPZpi5kEk}Y$qCSJh%PH*S}b*nb*qn?A=Lf$;M@}S84%Hn?EtSgY{^Z!H)=p(3r zo>9qtRi|_{<1fTn)nw&tVTO1QG~@@ihhwNtKJyY|Uj381h?sr~9*h;?fe<29mSC z$lu{s3|>NhwZsE-oIMg{Db`2hj#Zj^{T-rS06>0mz896YpgheTgeK zHAw1dyY3|->K|KuZU0pOrU*1mYWv*1|7pec#8B5z^c?N#NQE|ElF13^f?&Q{_MMQP z6Dqegq6Kd=2Q8=l@(|?M3UrbbxwRK2qC+k%qw4?oYWb%Btb8+JPq>xh zNaFxe8wn1jK&n7$2k4X|b_pZPfRLDpELNI}De3irDEsL}i zaNH}v8TTu_ry`V9i0CK2JQd`Sk2EOPHj(E98JfVi1tDE#4v|+xC^ogR&w%c;4X}|_ zK|JFtpjK!}j7skN16<}S%P5L`L}?idt0%+j?x}HfGY7&V!!@bB}jYrvPXuz&kShp z=qiA==Mw+t8DrO)_yQ=dB_=VB3}6zf_4imq*HP+DiBg*USSFFVXwud_0E=t`w0>3K z@_oJ@Ro_2HWvtU<1Y$P|1`k28ivgrCn*xY9))(vSnOsnoPCf^|lB(b0mxUNIO>H<= zb+wZf-o#a?`%~(@MDNO>&mJb(WyS|(RMIP=vkkZSUfkvQfi`7u%rt%HSTx1QBRhFFhMJdYVXU1R1R0-|Ip32>KjmO+nS{Amgy&cpUP%RBk<@smUTMN*K z(2wE=UBHqc@8Xxh$%OY~8R7QJAa7QbyV=uMJ%aj~xdD};4_!5m+U9moO9mY0_!JKr zX9IfEp^BM2fW&4xDlAWQZ#s^RbaGg3JRG^VT-H&e*s|izq@iQUYtVwB)k{7NjyHRL z8vg`wk@3NTdBU;ovl|+bJXX}`!sC74O*h#eaNh4)gr-^O_wB{H6`0z&wHnBoX?eH& zskR8*Q7cjp8js(`G>Kt2e%-t+!13;8Wi_tVc)G=F3)#_gD0I*l>NIfy)N$*hglMx@ z8a|fm0WMmgwWEZH0J%_n=EwlNh-zLLBKdOz^iBF;qrKE^aJ6>5oZEJZ4~=j2uYt0< z@zrqvAQpm}eUCT5U&_ds=Sq16P&|9v;V9olaoI@(D6D7fO@FHN#YqvS7og9@_89;P zONv~z0_LR=lrDciQ{zX?TPIgu#lAQZ3kyqOdH1T_kI%gCz#>T8$}HwJ2vNS<|IJpI z#*=ku{1-@O7KGAwZpmpp9sUkM%;h;u!-GwKyo#5*X>`Cb9zjA8xf5y0>T=xsWe=t{ zH`1 z7@b)N;Y&=Su?}X9Yt`gYK5RGPu^Vw73;p; zptT?7e%P0Mr~us$*c|1cq)qT?x>5ION+`f|GqvA%PEHNJ`K1j|M@u8<+~#+ZjxhOpaZx6#n(i^eOB<{k0_%#G0nI`)VoZE2nS6ST+QoS1=!^{ zMWbDx^7tXZz0Oe-r%|N-6a%MwakbAOj0Bf#$cwW^av}cC!YJOq@H_x8* z%DWVG)b&x&VEZX3Jpyohd@-&@mM`ptWT@P)Tn||mA$nGX`5`p99^=wjV7i_!{Ql8J~<+@H~DAX6M@mp~uxh|!$R94(? z#)te#AKXG&`}ZEc{AsII5SD+hdlzH$=wf!r^_GGsM2vshI0|)f=pgi|Q$k(j`Q@g& zLqR4e0r8(8GwkqalBDL`$ z`^AebhwRUKe#61py@32tghpaX*y=2GB%(+i{;bb&-^bElPyZZVv8fD#Cv zQ<)Edg}~f3oBl%E&pzO-Q{Dj^S#tx?=L5RYoNuc%`J%sz$zk4 zdfSWu<_XE`#vvk7ew@@BBLnAv?le!43lKMtN6wcTPZE61IBHXcKT-3F^LY7$5XM3< zuYdo^UCKNMjvoQBAH#1SZkzG8nXRe~Lq?h5=*Nn$`*wN0Q?g9*7i#5DoytM1I?oYH z#Q{{lq|bQ-Tv{8L*k$v|?Wd2FZjS~_@Q@xy$~gx=gLmIP0vQ}nLjAT}n6BKR%kuT@a_`?D9>O)xsY~IJ{^%tB z_U85eC_;uj*B1~Jv($1(?=XV7cDmVC<65%Fs$M0N_?I>CgUQXSOTWK;cqy7#7=x2R zwG{P0&5@6S>H{Ea$3W;|h0p*1wKfF(B3(y_k~!%qpSTQ-ZU1j>Djz(F{t7sm)QQ^X z=0Wnn{@Ca0vve-{%mN-a5cR&B8(rvjC+2tTA&&AAGOjbp57ShoV5}9|O99G+B=ce$ zQO;z{oRNH0$U4hlLQ&ud>QOtN)YdiXTG$1;aL4LXxG2K0Nl!%V{N89Taf}maQMTYx zD2Qa=3F37oNM?$uG$xsL4HcoZX;J*jwT&508Wf+DFm3$X^ys(8XZcf3qm?u>Ce+D{ zdRnejoc0CqYA4I%au!Br$df=0>3NA>(9F1>717dt&^IXh;Ta$x(T4~OxC`ctIiG|` z=gN^R4fc>I>5Fguaq7T=X`EV-=-xufvNhwOlorm2J7({sU_fI5`15?eXOt>99yiBF z_YvV5=_5hLm=iJeN{F}GMNCis<;FTEYqEGK(Rc;Ru+o_BZ{uLdyZfLi)l30C{KUG$ z*{aW}jpM)IhDQ(+PS6l1Nxy()Wt}-K$!VWu<*Eb4HomIqnu`sU>ty@+Igt{)!q%t1 ztIkZXp-x~aUHuELw8pDH9`7Ui!tx$xql?kO!(OAd^z#RsA7i%|;m=-O;ooczLXLKF zvtUZF43S-y+Ewj-n9-Xz=9fj=X-f(C*vF!guoE8JRW%2ldY?A|Q?!eLejvtoAGynx znPR2j=6P1ewqj#ym%p+$P!6j^wZ@WpxcUnKg5Rz~eZ5SU>&f?egY_RJMzDAckR;bX zMyug$5o<)NuWDjI^m46jgwaEv6Zg$#1ZBWWP?AO5!}b}(BR%?zfLHt@0XzQ>g0<*j zLu6*QRVS6{Hz@6Epa5)I>d=P%NP<&MJC~OiQKxbixg{fK$$qs;e%JQXUfZNJ*qFa{UOUDR34%U8H=4~X#xz>xJR6pXZbpRvX`Y$H z3Ll72W2O}8cgA{+l`?sQTR&$eMbj|os$R^;zd?adaKCS`I0mui=ZPF9W8GV$7q#d{ zYZb)%q%llcN?rZQOdk znxIt3ZytCoR7uRSvj3*8%4Z0$&=pqC4BEi=FAs))gSn0l5DYf+WQL&CZwANf1>oE|~0Zj%1fy z6GnSCDEF2MjR)Ma8Dh7Kq@87Auvfnh<g6btFZL`o*CIA17aV5Uode zQa+nX_4f(-Tf7;|E=by9Tg4=$5y^>II9 z@dOKZ`nk79QvL&M;l9xj?cAEuqwv1u?(_2%jQ@#}dG|-aItrJSd4HYX1#ZRAX6}h+ zouu+h9fM^E3~e)g95@m-!Kxe`H2QN)!e@#m?+y~k?Y6upd*=jjLNAP}&VoHucI z=Rei|0GMb0=2ZPDO^&Q-*#X=|SR;PMn`1GYv{jam&BO&V38MFC*2iy}qwqP2HY)kb zkqkOCZW@1}5|~v|O|7wd7k9U!{067~>D zjg8-|POh%J=EL_7VLtWkk(_Tz_2q7aJO;p4Apy zrqAL5KJ%et-=bsGUdyd1y?vm))>`{{1}^k-YS-BvhS_8>`MZK>ic!hX zhCd@poh$u9o+>)7*ZHvWsetkdr z(KnPnAsg6Qws^0CX5Z$!w=h3%qGVwTvs8pJqmMgMIgQYpQ>Bw_)l&r9%{G>U^PWsxv+*ppZ1NPRfRBRn7QOc>1PiP6klqwSNBTS+ z?Qgv>p~*Z?|60MmAFsR_gf(Z9woFC&)*bF@g+c)}|Nlm$f3p>@wtnlo8ti0MdSlcSN$T*>sdWRyuzph);3o}ZZJRk4@@P`Ff76(nYm zB{9#xF(Bjy)Z=vTLzx<@vqiE^51$)M6H@FMEvD>g!stBlLRsRA0W2&+SH4^5b2Tw=X>uQ zxltdI!rH3YZ9Ld!v;2Db4ShLODNEhGM><{sj?aU5aT2#&(N+Ph}_Jm z+P{|&{PW|&T1ifm2f<&`KEPBfUJ>b_(1ct023Dvpag0)xI9{Y$EXf)$<{WjWnB{H% zD&;u1tfCN%iSH$1+9zVi$NSule7R0lAv8~X{i$WQ?*_$I={-zWl8@X1qYi@JpRdqT z&N0Cv**|d-$4+~!(6h1t4WE8UeDq`$WbFOd8j0e>F{*J0n9AD_@_^5C^+N`aSro1S({$vfP?` z2{Q}=koLt&8rOmY5o}CXwL{0qq=xz)_dC)x-DbZ*7_-Mo_QU+GqGkTO3Y%V@Cl2}^ z*#QaVg=-amr%)^umdN7~i3e)0_5;e1XsDz< zm_1_@OEnU2xO#J1z^;xF^GlmYB8d8>%w5`mA-S7Ze&t*nJK`&1LFf$K4CVNdn)Ej7 zE(b2#XP!ejw4W`3mi~PRPZ{P5?{Aemhi3u?dHYlEL|s%l)YJnw)7rrVa(yhU}J;eh?iq6=jQ5kUQ3T4Z1Dmq5x~j zY+&~ns26HguWIz7N(@=vq5#2tVblBUcPW+kxSPLTnU?B#wb4EtR+);v2C8;kc((P` zE1#xR2{mtK@*vG~eMTky)oNw8+u<46keXOy1ws~abD?vyLr<6h-`(Q>PU5+M;Fs)t)3O0u<2Mdh9lNu2t1d+?(TnLKU?-y8&$+zZ1~^b zDjU@dBA_XIN@h;~`pY}?oW&O#ZIcei-D7lM#vNd?ae|dL>D*ZX%UV33`qDO^hSM8M9{kgv&__SH5xobq; zYmw4}iuZJWZry8-=a*{8e4Z9fsu3lbk)ok+8*ev5=@gmQqtYR@7(-G?6oc_ha{Q?R1jx?U0qu;(HG63cgt#Y^c&e@ZMj+X+$?lNcB;bE zKz1-A!xZsBD{1cVGj@Tc3dyiXOPw)CVWddnKq8eJvQuvoS(dALvIfi*X2w3USFFI3 zk%m!Z6!^$uww0m!*G=)M`_IKE@uayF!_Hhx_Qy=UFBiE^vMxI865=~~pES=!0nG+&5| zcX59BEU3bw`=)uzDIv`)L3`vWrI3Pxr^w)(#TJ$6%e@T2$eL&RxvuBK5>qZ^XM3yW06Fq!?iD^`g7U%!9P+Aoav;ms69QcFly}1$g@Cjwb`;Yhx%Z;197{c71tMb^YTP+#E zXul?Z37VTMY0#6llV?i{iMB!>N)??p1Q##fj z`nh*uO~NpcMh-&vc?4l&qrPNuTuqi@aZa+%i0zc#gBgE;Dv9wk?JyVd)3ii6ne2qq zvd~(dk(LngcTdy>#23GmdLctBS;x}drA&-jC%8AwT$}yL+Y0tPqgCf36w^|@>6YPU5Z zVf(yAbpAsjr*k97q=ZI)))AjY?)>=OVUJy&`9Og>3AC}r>-h6A*k)_i)_ukSvy;ZeV$i9am_NM%)ZbA`QF>Gl{US@atQSy8z zFJOnc%XKsStkkPzs-&Rq-uW{>W7hoK<2d5p^!k|e1T-Q*#&Pd}40p*ip@k&GMS8pY zzN5ek-7;o34Ba~m{jvnKc9*N0=2VinD@MvD`$M($f!{3KgGDZE zMuJXJw#36Z zsLtpjRWOUgptHtsSA#XkUxJ^>#a}7I(#zI4RWry_V9sJiy=K}aQKgP@4Yrjj2~X?J z4n69)vi^5ZG}TkENPXTFF6&Ix0I;xHDvv}+*A^4-9ZSBA9D=fk#1(#H0V}i8fzn1O z40Z;xA3I_cS5HFL-^@+rU2r(@=&gEJdF;-Irxr2=x(on7%6y8dsJ!86Uk^yxDN>l1 zk?pLlwc}@OReZWK_Xseye)>j8?c4l}OSp2o@G&#qF2|prAO0300~PbVDDStu(TcTT zdN3LMM{3bNGQ^P4%i`7y$1T)AILo~jcC<E7}8IQzcNK8#C!6KHH0Xfm7df(agB)97yZVNbac3Hjz ze*~uSG+D|0xIWtDJU5l2-G||-3K=#g+3hF>t}M7Ydbo0U!K!P)Kc0favoW8?F3hL*2?u5G(ocX{vR4@ z85XfhpvaiumZGJ;R6~6#xs_r1>07(%y<-9&6#=#f1z*ru6`JA45>SNl2h06mR>p)E zOKbS8c(Ep-L&$#DT$ikR3dNKTP+cVYQlZf6k8DofU>$uC$fy*=kgs~j^g#)TlQe$} zvPydQ_8T-0gAnY!XM?hS5> z+@~C;(}6k;2*~*Bu|E;7^Ud;nx0g>)UWN$1>=KqW>c3;4x#38{bGux&A2Pfn`NSpL z6`xy4x{!??%YnNU*_BW?JBSepVv*`nhFYn;zk^98>%H=17Y*ee23-#i(7_i|phEeV zW8ksbOn+gwlYEdoNG@v?7gE`)O!dgKB&f)<_lGjweq;J;`fgRn+*@-9)spZp4dsW2 z^D7YbUS#e5*E>*axNW*%bxwA-6rEmE8=CZMO|sj=Ybm4E6qslMVRh~lLShZOCr>0+ ze`_rRKDSJl=H!Rtsg>QbAobYae!yGgOI8!#@nFg@epA#oyCeRXh{BWe5FlWt&L6Po z*vwUZ9U~c(TnDvRlP=1D_KnpfEaYM$eg`CH^RKE}vM1?#zqOtP?uGcf(W`bS(Le~U zrF3@B@BkNgOi0zMzoRrX#aiJhu-1LU0u}8{rF3v`&$H3#+H$Loy@Tat6vCn}hhJTQ z3eDVV*nx`F2EBLuF4+x$Az8i(Y$d6~P9f=c8m$*hzkUL^WgIhmg|X z%$QE92OV6U@<0YetR?(A#2pa(ZFdDA$#y}md$F@--vB2JTvSV0*|9)vYbKX(rn+ofD?~8^lE3LywaqT>KTFeLS+S*+ z9V4yq>e~3PPQrGh029R%OuY zAol8PyTPi^XV(r~R**>H{D!kRhTAJ-(@_i4Imh&0D9^uDC6{4S%!pW1b+xS1&O7CB zSPOaegm@`F>RbGgm}ovmuJ;m=ge?C zXxJsk)UL9tBEMg~$?7sQysYD?Rm4|-G+BBQL-qg}17Cnxva&MC9aG;xY`yzydAoH; zYWCoAb@L_v%pkT;n~wmoQzG57XCy*_SL}(F zfK)zt2yPd*s$~ILzb`!cMb5lsJ}so%DOS5{j7R*NSAA9?|nL9+*oYkMAc6)>gg4#6Tr!oI{9ur{}8;z)-=hLjOV;fA1O)I|Ee>4jeN) zIE=636^`u?0pu=;tov+|I~K3Wt5fLa)zn}zFG0An^-h9EMOkD`B_QSyiI#Xf+EqCY zZdux0X5@Re>+rO4DufeS5GERs zLPR1=AJ!^{d=V={-Pc6bVMZ)H1Eoy&IXsluQ8wz#xfeQ0n+5dAd|qkeb$tdHZmYQq+_FfQDmc1VDZ|ytn;Z zcA@jt_kTZM{a8*%HKN?IY3PvP=0F7wG7n%5ek}de!02^_>Ke@i0A8Vx$N^- znfNdyc4Myr5-3+@Lo)HOya3@Ca>UuBZv4WkOF2O$214jLuq<;;f-0B3GEkZ{+If=< zsz`2qkj;UWdRL3vkx?VcKaTPlxSuKs9aD&LG;oq>W(8zBqeC_-Fx(>EEF3U{28#@} zx5NYC`l6wkZuOxoz5A5Y*QBniAlRaN-`$9inD;?*5XWw^trJJAG9sl^~dUQ^5uOA!D{>D0k$BI*mz{?8H>dIn2TN=~k zvvHWMc2_E32(E_((v0tk3Ymec5LD%Og0>F7BHa4RlyCewTeOUlOzfSHtjWEyxoeb4 zFGd*H&x9ZQ(I#efm&nSVkuSonQh(p`>*#o$j1=IV)IU3B{^~CP`q_5D2-HsfN?4gJ zo5nrl&z7lBA~2nU6<}yQdq`o`z>}bn=C`>hm=%Dk7F!HQG`3-M6Tfok6$_GLK|Eag zhU^xvPznmo$S)m$2a`Z*XCxo-n*3P`0w3)o5|#6nvYaN6N@pCRH)VL#o2oxtK^m#< zS;`#x3^ZjmlDKOeIt1~5c4%7G8l(-(B9yNrgG`(JLS!#GoF^#7N!r>obDLIiu`Fzx z@h8AGY}DUr8NN0VD0(~cI`FBa0Q%nyPI)A~lKtzN+6|&^W&gkFBZ>`vOCv8P<=w{P zw=!;P3IxAlnT}sjnEUt>eN>He8uY!g0}h<7 zPMn1a-(F3Z5pYZ&Gf?u;fjMx&32Lm51IZKj_8$jf5=JAIdFe=>D;(0FR8BQ)(kcbNo=A7)y_ zSf}K82HFQUz8Nn@6>1WD+6;cK!z+^Gpy(l9nCY1Bxhe}H9AzMGulXX zN@u|@x+fmaS;%urswC4H{Xn%I!j&tqIb1mb26s{X6+$vJAcmAP^HZS)?7Ws=YYh<$ zxXeUc*i4$$ZwAOaV;^V!&U^YD0a1^&;NvDT(mypSsYR+(ZVA)5(#a&%sRF-1-yFoZ7 z`!u+UC>kvS%1@|E?S1SqWE{Pr)|N;Zx;Pjq_*SqWkdi=lbDhI$Q?Yk3qQJQs zBCdE{PN(oa^ph(51fu8dwPF;7xsiu03owo!$;NK(a*%ipg30 zw331cx2ENVfO>%sRN&fovZ*DwEZ$ut+5`#NT^zzkp7na06z33ZWxvkg%HCx17{p7k(_D>-dGzsa7)y_302~kHZuDN%>vjW6QyV; zu88k_aUiYjCiMHT8#XH`>Yv0Q&aGVh<{L_K8rgqln{bNc{8;DhXlJ(lhTK8Jw%m8J z+kd`Jd?noj*5N&QpB^UG_cw$t7mohMI?cHItM5b}H*n**&Z_2^*sQE(YYpKml)wZI z3Tm%jZ|V}~m0dXWJ8DtM`A6t0yFF({zL$?uboX55Ndtq%c$j_^a@}C{p&SLV4dE(L ze>zf)gsqD5mpTiQL0$8%_XLRlFS_13E~+hT|CW#t1%_^zVJJb6?yjLbMUWCHm6Q$v z0m-2olrCwcF=$je6;wh(6hT7zw>IiI=Xu`u`R{x@!|Xk4?R~Gc?)$pGm+Yh6Q|wc= zafa>!J;vbrg*S;-?`uEdK{HA8Kh8$qU9oqRaW2zv$+G_<9e`>SsG);jSRibYw}M)b$3>@1}CB`Px8^|*7Pcq%X=O`9lle3*5T^)c0F z5B(Qk!*I3rYN>8@?fp<_piVVU${lC{DfpBDwp|cl*h`Z zKd?@!NFsvkBPAq7jn0S{b!>^1wGV*WT|z7SqV#O%$AF{dWF-9>DmZ}d{%`wj-I>M-wXNBD)?D6sI zDYM3Yg{MBnlTQzVK3)jmXM!c6D<)9`Hfs|e6Ez#dn|M3CK6r8f+gICzfoKm>6lQ5Sl~$Pnr-!o(T@? zm|E}#ZV>Z0C4*AtVeveHim!2T$O9d%Q4rHebEJ#tR2tK7PLqC9coaCRlRzLnqp7{he4{YeUSEA~`PPN}V6?lQ zrfq@zH{BMS%P5sW)g0N6Al~oz``ze-ECt9vqwiD@#)j`-+8Hq-8p{pl_8r*{KaiMZ!N!B*`eqm=?aYy{!8LF{DO6my?T7Au$0XO=g5xC?i%sFypltt94 z=!%06-@#}v)0IukssKxeN+N6R3CR~;4^1utO}J^tP}8OeTao6w^ml|9L1gS-B+QwI z-rzd%u$Dv&;`oTilVT`PZdP`OsNR^k zJ351X#Z@IKQ9cWi58k_O$mbSU9n4p4ER&-Q!vlEZnty{Cn1vi}mBRkf8HQsS}`O+5hP z062c~9G2JOv~7~F#y;&n!+y3o9D8ka+z?F!gIV>lY{uF2-fq)AjQ!g2Y}>a|>~W`k zJQV{)Z6n?LToD;w%FzeWm5KM_z|jemx9@Aoy%Ow*sW;nY_KE{YeIWT;VBd@jasa zh)|fhK?Fa;jR>B@A6$HIbKKe@v6tWp(>Kj<5GLM8_|$QZy$=`d?x$siT=S7o;bhP9 zYt0EX=ZiU%Bm2){E6a z2wt+LmJP|W@g0k)u*q19f2J8dfm!&IjPCyZD=s!6yx-@VW?~sqF59W?Rx5oY=IEMSMK6)dw4EitsK4o~-6Q6P zW}i6$P7(sM<&A^P9CRp3S)|6wD)V()vhBxgB)Z-D`p?pZ-haeEcfUvFRn;qzY!4RQ zfdGp0E*Ty%S$fiXXX%dzG8FnGyAos=U4S?OS2cdcuv6aYRS2O(F?I_5{cCZ^>8Ds$ z8e+?Mke0&wEr%B#r;V|inrNT!{^e!-t@%NG<{eOFl>G6(21-Ik<|SjsWY~``<3NXj zv4#cD@VLEoqDpH;U%~a&kQd5b@>bko5t5bv~d~)H1f3dGFul?cNaJe^v_l11#_lgRWSlbl9 zvuazc4ZvEEyg4DdAA(n~{>eAw$`Xu;XCqo89+z-i?S-Q^Na8NL3C+u;;KpDN$nT(D z`}RI7QCGIh=c?>f(4?xSCdv%u&{#90bROT7rC@jw?p{O4;EPuiV^()N%f1{D>2(qA zO8jn`&5M0TB3!`J!nQ6?nbBY1IhNLU$S$QAJ3ny-Mr`ek+&!_*7eFgG0h0jy#CP>Z zgnifcuh1S4KW<#<`oEs8Tn=8XCkMx3@QOpObM4V8VUN7R<=Oro1Q1m z4ULC?wE#Vro}xs(ky{V>G>n4}Uql}&B0<=myO+S&4*~3U{&yt15G*Yi`)j3})Cvc7 zJi?)1QrO`8<`Z*c=3dZ6TE^?_fL~7klj2)jD|%P5ZC{6v1`p8*ow5b>c{O}qXls2a zjdOq4MoklnR=#yTBzno!iqbELzLo1CrQTPoqcF#=Dgq`KQpeDs0JbLFW=4~h-uA1Q zr5q_~belJ`9UaO$R$h+?OcZX$t&eZQrs{*&N}wXgybaqIUT4aK~TS`tLc0%2Sz)GIR&;Rjemu_XXEIIJ@4hxA z?`B8WjmHc{Bcnvq>I`264N_+#loLFv;%~RtrG8|Yc)iJ=gTm~bTLsom6l;&GuE@qt zm1;9OI(lEk!_|vW332KnEh6b>GiHMWjlkaaFTk;@uBfq}{ML)_ zfF2O@m%T1{gEqt!T${G5VOJg+-y0C;f`7&iIIGSk{yKoq{0oY6zy2HNng(H?h|~g2 z5A|Ys_1vfDbEI$LV1|lNt&#m&a?X>45oPnAkX<$=DCN*G`#L;!&#I`3<3|@QHF#UC zu1~79Qh#8tz8j8#L#tJCciF?L!Z+rD17C!xePOaIzkMr?<15);@Vqw7t}f>?BFqnV zEy|1jxZY&Jsa62vf7Qgk}dH;;6mTNS^Z~S$G zr`fVUI$o}i?`!5rxVcd0=l14foLaMo)C^9w%uge))#B25Xi82`e$?rTnzP$IBx=y& zYa1jvs&)jZnY+hT^x%{gbJGrOt5Gl+U0Z|(a0Bd!pqoD05$(iQ0YC2n+(QQ0lXrEC?t+TaAw#J}mC?s9rcl4swG4Xw^f2}cjB@5#c7BYf$Q-V1N zJ-MO%i0#1X;9YDevAbmOmr@}tSU@Wks+vWTmd9QG){nO>EVKTAH1LHr|Gdh->WU_P*4nom+m9vvKr!GIzXU*ul5cD8Uk$#C^ zkwTL0a@kU|4NZ!WQ5tCFaAOFGgu2Lj#ot=ddr0s!3Y#luB|SR|ay0LHOE9wK^C_E! zJ+KL9)}w1&$b%L0a^F6HjhG6?Zqm++GCb_m-+Wa0)=5`9Vx~PMCWI-e z%gZ_{_$=~@&@@x0Fki~G0T+z}=2#Jg*4v4*GABn?=@0hcRb>931znk3yj3mJz(*3! zUs#4J;Ffh%7q7538B#C9e)h7{n<*c#3+t{NF2)Mgrr=;ku(?ECYea#=vtQ0+mpe<5 zHoUyC){!%W9CM9s)n-L!a-cd}Dg@gTZ^>TJ9QA`a&OTcmop9xq(iC%8zs%Q|w9@F& zJ0aI~X?Vvzf;`M80k4EC9N(4{&2zOv2{Rg7%b#c0H-9sCbl11wt1;`Ia-Clx$K(T3 zkM3=G$vBTyy6oCqw{@c&@1R+uY-;w#DKObl6s|8mTiz zI)d!34wS_IC%dbZr-l7%Y>|om3z5#|f%V%sA%cj&fIgw4{Aj+dHZA6>@46;MrnNoV zR?y!z()U7zb#A;`OkXw?ZyJq_5t?P+>?^K)3{N=AZRf{GDKD89f#YLRXzS|YUDP+e z#6`gwnuOFQ!6Z|uA&cU|eDaE>?-5fu7mYUBUr$>Grj6Hn34u`67sBde6_>*+&vUAc z80=$yPK(fr2OJ2Bn6>f@U!gME*W-^Yrz0{vGs0^M4UAh&g?-2^H7hvY|W2WczyFse6rNvg;&j$BLs?+Ys? z_Wdxm#(6v80gfyEa?h7k%CGfvsdBw9-r%*rdNq7w-=icPdAm;Gk+Q>`^_G5W0;Ol?E)Q5)1yT$8`y_kW}TcV2D5W?Ydn$-!_tT**Zt+`I5_;II|UNdfM|6`x(4VTbV~=ujz+D!l?-w24@8~ld%uy_%sntx z_D-0v%kW!ruiC`vzotre7C4AA8!6Q1Ie@OhnJ7=;u-MkIQ!h7j(K_oL-()WqaJeUz zZx&drfqWkan4w3!cS|mk_CNYm3xXpl<{_U_lNiP)0wZ5G?*o3^H^e)N6+GGaCj_?CLda#N)NKD>gs{KWCE2?mc|#IM-MNhIuO?W+7_Q0cRJW7JIe{__L^+ftsE zEAtvG*Xa&VAR@r4((Y|Wa-=}G=c)&OnP%XGvZWmLnir3EuhOmS%-RSvSfgUkP&d_slc#MlIZZj1hKX+z$ZW2N8hJ!}H%*Hw#*4ijvdco0a^E3&qIaWT zS*S5sndBY|?q=8hdGD2F=3Q&^{ zJ1gZ?;U%GvyUo$L%7^)97865x+oi=VHD`UjVg~IKN5;>S*QN>W=Qy8D8BTcaR+-|8 zZiilMX-dsnhbxOM1as|=9C7%M6ggxYPksboK@sF66TpL!<&gd|Uo7y(=ETW`}73Uwqc;Fm%O z*Hr>ti>|IVUzps>Bl0axlAVh;Vhpr;^f!|~#!}rFXW-?T=^^zVmc?E6OYXmYLPG!5 z8yJ_X;M&+R*!Xw^3k^UNl)|nTl7jym}K0W^}|& zo5Z-eB?pj8`*P-hGjP?_3A=>`!#`FTu(SfrFrwF2ipw!vSE7`HR#L7pT=LsoYt{CS z&4kcFqQ|D@K>e~DZ~6qbN{+#*n85`vQ6b@-@5d))Er6`1eOIi!kNF;V8kg4?KWj7W z-BLqKm1yko4200U4`oO621w6e>$endGjanc1{9Y{p}O&KAYOjoA|z zm?zK%zUv}!?Kf9lfcjzHW{V38iGQ-c|3sSR2DO5PC>P2cW!^-{!8-n0YnP5T_^yo1 znbzu899k!B=7hQ72m3Yr#8iMnW);`^Fvs41h~wkdQ$QnXUY%|zEQ++bajQX>d6*gV zP%hYJRjo)>L=zx@)Jq^{`m8ToDK2kR?e?mQU0h6KbTqO-%P&f7%D zlVJ4)e}6y`s5O}8Ho=3NEA1NGF3%;tLpb%4zz2{V3UPtoA`jNrq0_;ld8=LA^ayhE zK7i*chSaUOeOFt(6@fR~b<8vQ3q$XK$6ZFC*MY?|J*7M+Y^H=`nUc*5-L^L*n=1Ix zTuyE3NCH~o%&`$XnKp%vRd##-5&c0XRj*Abxv|r}>0gCshFB}7DF}cwKMYmBhg3mw zt|O|-m#(GZ6TWERW4mdvax2r|6!$%MD}PUxhG3@PpSa z#Sa6orybt9(MlAIW>&tfB_=s_=5gA4bl^N!2V$WZ*NvzB3bicPO(W04ycH~1yKo*w zQOmUtBm~#SY6rP9fw~l7aN|>yYFDA*bP={i1j2c1u*RH$mEGctwlZ5)0-JhSOBq#Z z$5M0g(3UwL7P<0`c|iMif5Y2r!-Hh2U2l)C$(=6s?Z&cgeJ_Z~0)JudLioY;I=DlO zcful{LrH7`C{NBa;JY5!@N6?irWlbobUioBr0(`I3-z^WZE$m*R9~lCCvN3dhqJWG5nkL7w2h`D{OY=RWz@+1098< zh4IyU56rU1*P?VuCA+6#W3_77+WQkCd%}}!mA(z7B9o=-Ux$3#;z6#IN5MWQ>Nq2s*-M&ZVN}N0Obozw&?o24H;}(w0|IihPf_KnlL|J7NSz#1^mp7w zJFJf_KZZjMDHJ!5^3+W(Q2pdf2!#l`;n2K~oyR{GaA}Q+G5o*mn7sG<`IdAuYhp$D zpiHr2o2>i|5a0-79&b*$^-f+ickEVqjV)PsXr64=>b#C~!kQ2kQyXbxiDdaumY%u7cdW{3x+1MMBTRIY{8@-W7=LdqMzV@3}noBtIf=lXKq&9 zdp~QXw?Pg1k4o7iTLgX8uvPk$?*GGBN z!?ln{!oOK3*ZtVl3^hh?!R3U|(Ju~5{2mF3ue&(>q1|%}U;sE8>slDK`{;y^0UF#f z5r#j`Zf6W*-!JqRP$@S16iAcHW5aSLeMOTxcn^_hmuR;M;+2N@_a_fpv(l!;55Cs!F13X=dl!O^Vik7o!7)6MR+ z?~a18$dL}*a#IzX=P8+Zpp@5u_oL!%rZCSvp3g`RL+q0cJLz&kl1@j*bj^B*6rCbF zL!$_FM2?t7%LKw-q#7ST9ko$QyzczVuKz3J*{@!Sm!n8E-da`b2L^V5yM^_hFUYx? zLfLAr_n3Rbm7}T9S}Rf3PuweLUAQYd!S*}pk<9MTr$c1YN#7v+3nNW|D-hOdEcV2x z0imuIIHG&Z0wmBjXM&eT0+01@tX+^@^h(Uxi5HXTWry|gQg2bt-2C;{*j-6&|01=c zH&=uJ%aMSV7fdZq7hp@96Ax1+vl=}%P_!^>dqA>@%shjXon~An{-yBp`hkMptw3Vp zoDVgjks)ODlnL$ELnG+dyv`V}aeSVMZb{ml#-j?p2pc{_b|>`=jRSOMI^CZw^Q8r!)e}We#&R>Z}|19TM_IIFQ$Z> z1Td9s{UuS>#x_HdIpMJmjf?Al$HjzfL`urvn?valPc4vWii1Qo{0gI#1Rz>^H}q4r z;3Z5ITwDL%Q~!hM6cCpZZokVi`6;)qU}(wax8UEUW51Tv2C)czl$Zi?tXj;kNLdpk zmKqItE9L26FPyoZXjMu1?r!(%%_+UFKURiluk5T?fQ~4d_oLN?d6)vb%;L7+%(Y=2AkO{;{Gq_9==GTp9bU3njT}PdXqZ!wgm=rR(0t4A!kDnZU^4 z9Kd`F8v#Ln`XTiI{i zx!YO(c)DaOt+yl9eynJjPcQ_xg65jz2hjb0}_5r zb9J#^^;a5!llW|12qgAJ!>cy8ALdfUZ;bD8hv3YPR!lqB9tf9a1nD zC)n;5nY`!?PTdrK%n;mQ6n41{%9dCGbG{!g|7eM&3e2!wOoU;-p~286*~R>nvdylx zPkzAZqr@9k*;GdKTL6P;(Seyg{336cG2N_y$q7AauVb&JGL`ZfaqSMq(Q(K7f zN!eOm@rXi6tW^Bh60+7kRF5GkKo%*Q8!`G8iZm}PR7=DQ&%THM9gLR{ z(ae(hHdxMsAV4}26Cqy$dlFr6Jpn0LC>CWcSXQNuT_jQ?%vqX9u3?Da;86{{1Qx`jb;8nUyO~3h@NN?PS#1;`Oxh4)0b^V z+#66iiU60b^Z9y6S#*1xUXgmgFT7YYJEJ|`oyqEjq(VY@hIEOTqxIj(bEg8M8+HW{ zmd_Wk7Z(sgD>1;{P9nY^_?ok%bUfDAp%~*cTH>5%zOnng)K6!jq?~O|=aJA-ur1FL zAX8;Xq9NKQ=G*+imk@0Ae1O?$L3Dl9)BATV3W5>CLd;xhIeppSNs#@wa$6!mZl|Pa zW^k?_^h%$O)S7ooM%6!w!m&z^VD*KAA93~pw=EtrN^Mlc)@vM`PoZ{D{QH6cPCUeu z=g4}?$q%-e@_-kY9V(NIH~PHpofs!+4YYd(%)!bp=;#xQ(r9mlJ#SUIyA$&|1xIuX z{E-*j2msbL!j`}mt;0rxDJ=9HK?*(4wqW0HGX3lKMywa_GT`TCS>o&yUt0idCeF=l zGpk6JQKQboi5;-}wIzvyk}Y7!ak?NI6nloB4V3iU5R4Ov-LZmhiD3A zQ?IeRzFZ1662IpXggG2A?zYkLZ!ZC2J{c7FetbMZ6APg0QDhhqEYIY_3E%p%t-MLs zW4>d6$)T$fV9@&%Ge%>zQdzY>KYNh7mFwK?crM-xj$h;N-p5MXr$~=Kbq*B=-!^;( z*4#bmFYS$Y5Hp+-+ya<4!(@<#1s+w#w?1Vrk;6eVh}YqOQAa|1;#JcAkc!CYt10T` zR@dwcE5WPE`VGX-hW2;}aZ^b>L-H+8t0gtZ^dj3iEghPL-~~Xr<#Iy(K5tlKUy@-W zQSM9O2EKc+cu~MY;7_L$_jH;U=NPm0-&UB`pI7Ia+q%Hm77y;-Nh*04OuCC9k2s6b z5YBc%6?LZB)IU$p9ceH)|KqT`2sq!mT!^Ar&RpNoZOvLnbbBI@+G|oI+v)Cpg^8{? zAyXJ;10drj=nUT;&u{nc4WKhh&qmxu4;UMP&iQ59n%>%fpPw_xDnnXiznPr0UrOFq z7Cg77Q9-CQJgP0w!k>vh2N9O9J^zrv0>2BHG+m>|PzYb=WIRZt9MgHU16t{`Qb40q zJ!lCi2AP5K>3`>T_z4($3ZZJ~#e(qPVaykrNzq;UXi2MiaZ%tYBw*7Q?+(a}Ec2hK zY$=q;mU*L*i4x1CZZC#_4^N89*VK-k!VF5jS$&aEp`m|<#Tas1qr|B9Gz9x7ww6g} zY%wAXm{3GMUQS=Zz_fTX5P23@zJ62)ATm;tXU9JUU*RLTd<|}XH;9v1Lsa_N7|V%! zZ!fj6xh?gf*FZ~GzYp@GH>y?n-(dl)ZN8^EGP4S`L9*5OO1I*fF4#wP#nj_j3g+tR z#fhz#Y>0;H=Kw7=M^3?2?`}!i>kqzv5CE*ejb());f}4C9bk4e(WNM5t6cd4|8KZe z%F920Ttqa^kEhnyI2f2CdefvZ{Zs$_xzuwVH#@YQN-fOsm^4`f*(Ow+^r_e5s`;SH z5NBi!Z0rX^VMNIAUq3R?V2vvT2=%5%u{)g4$VL9(|G}?thg@aCu9W}XkH5r=sD~*7 zONRXf9bxyg%#X*ciLW68CTLsWO9hVv@0mvK!5cf0L4)S0hAtAd0?@}NmRoEraF`*EMa;Ob+W@oO*{!ndNpVS!{`r4Mm*RZDVr z+#P5_ktubm#a0uknsdEE$tz$F_9sxTN`N^fpw;`kjd=(oE-mlbRgYy*@aCjsE#AH= zIxhLGr+z8Y*O7TZ^xtLivgY|83jh5>LS-3+_`6r>RqVoniTvL%a0{CPvcOfues*(0 z9r-@2Vi%(B69s!U5iW1I`2iN|81MoIh3s<<`k#C1gMQ&2)87#2ejU^0pmfxWMf8r{ z;m#WU2e$`r|GED=kT1+_n3D@g5K~||cz`+JBEx7)AV#74p%MYTYag7T((cbUgXov% z3z;-HZTx)@#HcVL^7j<4ez`y&f!vG)7yI{N)PiDI=$i}6nJ7ZyA9pA5+<7r)**N%` z0XVfdgh;9>3g$BPcO|0=4ujiqck2Ju0)X_*_0Yp+a2(3Tc;E)U)CT+M1J~DAgw;W8 z5a9dgDnYPSXowh|P^V#%_6NV5tE36Ot8js+pe?@UjLSKj*IMa~E3mfs=O++=$kBX( z4l5As`+iv8B8Zj|MiApK1M@jQ2S;Hk?C*7dS8N>c%sXAEI3)(`52B>GN8sNLz^s)= z{F?vg`&O@ji_Z^J2LW$Ij0atnXhy6z;>D zW?sYncg4K}*&Hbiuj87K_6J;;(Exw^0|qW8jTy=RZ_qXeceHUaE0`Tv)8dWKK@LEy z2?03&en1JjW$em;cyP)NktbeL3p z*fmQ36Q!rTs}J1yzE(b4vW|XcEDW;uE)C8zDDNa4(-vf}e?5J(mzH9Bv}0J9`QX#Y-gdn3fe03&5j0_UQsSg!OHvBqT7kNKGVBbX-Ad+kjrR0+$3x}h z8pA5gq`#)h#*^1zxWjtlumQxIwFJZ${C6ZDq*}(Wd zxox4WVn^hU7soru#G~3p0K5}2kW&c>w!m6(Dv~l$+abr@ z(4#bGA!ohUrvObk6X^_jDVQj7MTwQ$FIh?JbNd!5@7IPHdl=wm>TBkVj>3+pexiE$ z(83dOhslOq)az;ha#C__j#Q0xGH!W*t;)fof=8E2bY11aI4F{xzj~LY&f^O3uK0c4 zAUvj zX#ntcn*m+s$<|BABd|lMQ3?&(4VON)H1%Fi=6_Yc3rqwHKAxTKuL0{I)%_1{S{dRG z?sgcNBU2y+CY1OP1ACt_aF$L5tlLi8g^*gP=zOiFiv&Mic;1VdRBd#el4E_ay64pB ztD*#x4QPK+n*{^n*rnzXmTonh%x!~mJNf>gQw){l`LH-T;TCydNmCA4{X~*nR&`k zyoBN-C^28$dGPk$ZGjPjCr!Yaa}1mwOyz%p`|xDddmT}WKG6x8jI_H6vQ^>a)`Do8N&=F@R;e|Q66jIM6BJt9=ou(mdZ%r@5 zjlmFoSqCkUWAy{oD(B{l5%3&|`-RfebwJ}i(Z|q5b&rX!?-XR8hV~2aeK%^94?$`u z6Y5L*Fhy=;#_Qbl`H|@6W6ozNK-4k8d`)g>4A;n!5C!ThMf-PIq79M-tF~``+-C6l z6r6rR4)gh}L%Gu+)b3G1xlm;nON5;@vT>oAxZBq=)7n~JPr_3(i+)ff*KhD z9D{P}wB3@N`+vd`J7oX$>QVQf6B96`iy-v_?;ZC-p2#bh3eJI7D=0&?snZzqm5M#K z4~Z%)=5h@CRM9YLivREz7?Wa1VC4)t(*(EWBxk)qvvZwhiYp1k2j95z`fK4H(}Q}B zd%z`he@@uB`X$bcHNIsc{bOs3^+I$M(v<2md=AP0QFi3 zmmm;Igy;?H-DZ@Y|0$qLzsb@MwfSfVAlvD9F6?x(OOmW@!CZ zpM)0ybO5s?V3peg-r|g2`IZjX@z$~2>=uo+bOD_lWTY+Td;ebZa*9(gu-nRjQi1qi zpyp+!l+aXj9}b@K?s?!(C3IFDB(;Q(Xe;n$1r1p>6>%;Uk4%OGf3I7&fGZ+6v;9av zPo4uCBYaGkhdc;+hIp=NDA?#n3wz-|1pelG3a zSJ`T&>#MDRwLA*Rf7{)?zW#Xkc`h{T9Y|6`eTXxL5d^M4VRCP>bk9*ZDlNiz#Lh_^ zY_O$ASA4$dP$DQ(zb$ASFR@9de+TWwZ5w1yRYg5X-D<{DZHr&of-FQIGDGOag(_r| zHX*e%Xqu*2&0a@0LmmY|{>aLBZ@e{!^2!v*AI1Gky4`I#-zBv2m%Y-seG_DA+KOV% zCtz3bGaXguo+3c0=7@%f=ye)C6cpEog@12kuCJc&ulxQU$6Bz1QLzfhHL)%Y8}0or zogCji_67Wq>t-P51O!5qHh4?2KOlg9XsRP*<_nTP?mpnseAG9ojbJ`$BAJS_QDbw+cm^_lB2LFK z#U&!R2ySaFZ_L!y5#q5A#sb5v)YQ?1mtmo#ZvD)7Ti9wT4Dy%k84=M&RJ2Tu^;&s^ z%d5%s#BzPt4>qC-FC&^E)VE|bftoExcr>37Xl8Qh!8`e5Yc@=MzV65VyY9058wJO(6VR-8fk`7pElXt{+X}(A%!nCG>-LNhwLH>cu*nO%VSq z2$y%fvuP)NWI~a81#)99i;`XH<9_m z#G-SFj}#U$kd=JjL?xHvq96`@R2fww~>nUH@9c>ws8Tv79e+tytE;0bgBPE{#>Wg2^khgW>%lZ!p= z<(+kUUW&zw94d5dGYzdSE8M*94P0xs%N64n3(WTPtwNFE^eqac)BfEaV_xo_{K& z*4QoB7_f^iw2{cJP?b*Z0hYn-9#AclAjQj`$M+uA)oqJyBMFg^NLnT4T_C2D@QSBM zsjXFxx6KoCQg+r3f1m+`?P1E?-)llkbSbJ(+5ncuhT~pNzwE|CYi*HSL^!Bm(xW2I zH54NonqJuKh1t2^zIkmJXBU1Sl#lfqp&$6pS5j0mSx;|xVNioDn7Ro*66` zVk#x}B;>2gEJYA`JCaw;2iWkSHbf?uL4pZjT!~b z@!l-hVWiSCADafQmKR7Le@iS~Lu{W07w z#>ntT1~GS=Ll@l+CiO!U$@Km{U=9>RMeSa$i_5oYQuJ!=h<$&o%O)7lt1|ErTZEw~ zl9kmhnpf45UMgLTpuvtcGOLbG3KAxZ<4}`@XJj!9n#=c*`pVR#7nX%W$MwaRXTI&H z*j&?U_(EO;aLf26(9|Z*81VX3Ee31Q#^li@YFs0Ue6A-aZ|tNJ+h1{UC^2HZ!u4Dy zII-g42=xav;WxW8dia)wJkxz5@_f`5ngwC?4jVJiBc(Q&A6+Lc2?YFR+Ye2k&oc3O zvL}A6ynlcCsrBzHbja_u`R#w#W?z_!_f(J!c91|-RuJ+>tjzqCk|2qLc=e@`{%$_a zl!^s==4dKjgd|w|y-kpVT=;!%D1w5B!rBemu0h4d;R+MIsU!K=h2*yFh!xkd4ie}CgWl4;0mgiZ4`%N>a=5jkSCk@!wX`_ncGTOct_ zSJ0DkUFxd`$8Vl$Wir~4%|F@nkl*Rvp??R682tnsV%A#*1P*!!BZ}36{rWI47BEWn zw50~kL@|e)gcgbf*%AhWeYsGQZ)oYz8wTte0h%ju3#XH`)?Fqu( zA%vxU8>MvOr3HUFUU+Rs`DhqdWzjz_Uau{sK1?#+bp0VYRqZ~R$;GvU(bSHlecq>x zelIW8YREhasH>;YeM~1h%R0dq=e+E`{QntRdb;R*sibP8;<(vXfa!nqPZZ>9t5^6aGDN!fwJqojOMm<+zc! zAqA*je9iLQ>xCWeqk$=@s>RzL4xA?jIqpnHmG}HP8Pnujq0;$$Tmie!Co!{0KpGUbhH8Rx|i1tFx;`O1V3 zyK*G<^j$O9eKAyiiV;Oz%Hcu7-A8dYd6&o$v!o?{3z4L3lMw}N5eVUpZ?6qgtdz3H z%jXk~c(Q<+xH-nhe|ZAAAzy*xu9mZ0LcDhHbEO#gi}CpG8w-^u8p3VdR^GkMk&+19 zs4HM3mQE=k`9ZS~X>+i))St+&3BV*~F>STj6hyXy_Ys`o^uZX@2)Bhuw%eP`i?1XZ zl$`R9q5Wz&X1}NAG7BT3pFxfA1N^=zCxZ%}SdRVS3t2ftf`50TL)GT*Bvan+WF6_h zXJZ-c;O~3MD1;iQU_LTa;IX*(qb3&?M}_h5rAofnOJjCANJJ z=fPU%#Kt=XGQermMgx5Md#(wFdC@PVlv9S+uhZ?+vhm_*R)h`r1Qo|8+TyRKN`4rl zFjsP#P?SO@vpkRfIrrdW_K^mIwPNNH=`Y?-Xk~_e``uJtOS6gC1+FFOKRfC>=ZVVI ze|OXv8i#9lEh25SLS+h`!d9)aON0|tZJuVA0flWQXs|@95|)j?`Brtr`?`gmb~t4C zjFulxw0to7Ih-%C8{C#=D0-M&)!0+Cqx-1!w~XzM5z2#MU6& zwbr_>#yt@KCE@OR2@o~K`RjLBt^e^+mRSKwPm^imjDL?n>NFHZBtiJD(dJP%Q=LNjeomV5^FWfTIstkSMgMXS3|UR{T`QI248=q&AK^X zmjnM7c=4f$%5sM(EB%DuMDpGK{#XameZ=zWUg3&0E%gW&@4g^4U zWXf|j0b2I!DycS4xs&l3``In_yOv71T^eF-Sn^#hWllS)4_VV^`+#|6{;ymBdksDdF`{z+Rm_5UDie^wyddk^YlImofT%0?+*WcwEQa=6lS(kd2hQ{U&WzZKHvQ z(4^my#|&3~@fr2MnRJj@c7 z2dH9dksG+pX-@$JAz1RBqcdzvt24TtgRYKLd0hgcD?`a7&3qtY6BxJct8kE$G$Z847MtTy=2g}C<2O;eo%RgR27EW4kx6~Ao@6B zDZx}InLHPs;p_B|Fe3;Wl%e3iFGD!=Zx)X`kUo0WXA5cw7^j_u1VejI%NpR4Xb zl@=+?2@5MA=?JTT&x@%fgZIh4i?bQ*dZ+MMK*k0+-v^A21x|kEp1oP{nJQ>YAN~BF zavAseQ+e<2Q%R^KV?)eiy`w6}u#67X-p(NQMv(W~KBK>kZoU1_o$f%lO7!Bgeq-d4 z41CiwM&<$Yh0>*61~sl41y=M%@=^w$t?Csjm{wvR4H7EGpKt9}P0ugp^UuYUWeVbd zfei2)+;4YGIL-S(n(Ue{^Kzs>`o<63SN{~<*6VWOKxTcjYh z(btTwuV5La!oyy08z`4Ho*fVb8i5Q^pV-|u>m970LV+JGri9&=$zothj%>F8t*uo6 zt$MBM@B8@$d4-2bUV>r}D(X!n_BLe77-yP(VQpQf@1jy(7~bm4xTRQ~-O#i4(~x#F zc7IpD4}nJYRiLWEkbFR9c*fIXhjwpMmYS?A|B(Lsi5eCF->M-$ZF&*%AFm^GR)3f_ zeaBVaESsu-60{S1pT?`dgpt$WgHg5gCY|36jmIkNbJxSwvAyLTKW3#8Q-Zb4gM1sfE)^x95KH>KF{ff9zx zxn-^B{K*BXOidtEwxt0k4IOMlsl0c3`K!ArZY-@tLqb3;QZ~&uTL77r*aP}vbH#HM zxd))s>#?6|2fdIq^NkWeWyo+2&ZVmSQ=l*RO3cn4>NR0^{o6FCM|75Hoph?(os+x$gmzY^I=6V%Ej6x9s|`@owMgb z^ZrVXUC>#S#Jz%p&OUzpbOri=k@vo`tFOhY7x{-zCuCMkrtydeItz<_3}5 zA%#5^CUx6O;TALSBwjB}vXT2Oov@OXcDR;N;QPhB=nFs>TESfv0M&FM1HVaQ>lgv2 z>1y|GawDz_RJ-+u*1?z`e?Qy*Q$6!~RN#7Pu!8HQFH;?R4H@1U$n%>xiI&N7a7^W*$`FgcPLI@r^*5ke}GfqwI}T5uo!GSq2b0Og z9uZ(j`bre5mOD$fa(OQYZa{__QAs?2;Co(Q9A1g5whp}kpG!HBnRkkq!|!&kW64vwz&kf)_Ddrf7*SyiE> zt?zkAtb|sSWmEMEsAhU?q zH6++zaz%#)y&;X%N;T#9c9yj9xaRO-B_#1WwKxJg$0>6x65H%QmA*-^M$Y)CXO03z z4w+xKfzEgC0y#Yy`*nikaF*xeciatp1DXx7^v< zKoG~hy}yF$)P(evR`TC50R=?_?DL3__yt%}2t_il3(ky$KGWBov3YQF4hVH#5fi&4 znH*YlwVyJ=IIlIoRA3c#4}Zb&4CKs`T0Z>=uCvh~MrLB%)o+iq$&X0BCJh#D_ZJwr z44CZmZq1#j<&pDF|*+7F(gV#Lz!RNzx!>%`L97WtHG)gh+HLenW>| z{iH;g5CjJ>W0!HjbP>?9KrysH#ZBnL{nzfGvG(@0!?4c7!N5cf0fYAE{f75!H3ZiO zz~^^pF?UZtcC|r9pSRT_&+;95cH^}zBA|8?a7QB{1$BSUViRCn(?~PIx|@y?!oOuX zp$te#f$;Do`&W8bWvxj{C7f1Y{?!72HR}2-9T+hR42+YZNP}eM3U@BHo9kyKW!Wv1 z)TdDK2Z%w}&a5$e&QCdcZ=r9e{-iXKi4uBjJhQFtLXk$4H?a(NM;{VLz^%`JSY&xmC-i6Z&t{GDQ5{Gu_Bx_cA7p#dAkis3Za(;u0 z^9@9!gPb|Kyi0@r?L)`KB$*5`s&Iwi39B{x}oxD@y^^W&Re z`fOp_u?bat+diuKq1#vAB^a5}>o(-~H(K@Ylc^Ey7RuKQ9FmkvPIL#{Tq>1*JW}e; zm748PXP#~|UbLef&X=rAuS!~;vt*zoxrRt#=+-fJt%==HI2nj)Qjkb$C$Q(V5W(Q4l*^&4(1xwGyP-8bVvH*JZ=pfx4{29`qs) ztU*;ysJy8OmVkHObTSHuP_`xGx}{u<*M+0wgP%V;bXmioW^@PUmkK(GMh(Ss%SLRi zA4rp`jJL}yV!rl|2%~l&B6cJupMm6uM6HrpV`t|1UKi~XV}|#U$cyRhS}i_2l1s{x z166ixu1ji^HA@{pGrzK>GY?6`jtQlAZ#tvQ4{HA(dvE;~<=VFWF98WbQa~Dp5=m)v z=Gc1OcfTP-z5IR2(D(fuW@2dtT_e@8{jN@B97*&o8cR zbFBq4SDfc@9>>0acBvI!L$51^3MjDM?OUm7DBi=k%wML&RSUbPt&NGkETPP(zL!vH z)6te*tFSQoyGqj;hpj+u_enc4f3ktHMSrhu=HRw4eXdACml%LpAD0aLwQ(AL4#ulZ z72$17KojL!0g*8~z)raDeVSRTpD%l54Gf)I^En>>%2%-@LqD&$l)ufqrK9BtL|A0p zw^*PJf$ww?@c6q3fSceWYT%O9YKl?$9@oSta@Xd!Apw-i1vY1u>Yd zQK>7cYPrHb_47*Yx2wmOmC-#+R%2IvoX63A>hGGAw~?VH`Ag zGO`NqID~-tV%fOCHDP%m%}0#w1CE7DIwu1)wpcZ~;`yAv;G#DF!&cJV(6l)a6^_;v zXFS>)o^Ev9ip*}yy;N&+;8XwNvSx4x9BDS_9L;w!RvtKMkq6nheu}c2I`L@AW3@CV zIJQ>GrMR*bnpvEGEN9s?CzytF&8Ng)EXiNFY|bQ2%I)y8Ry=`;hiIMt3>AZ#IPrEh z>Y!XYJvmUX_EkUOed%MP;~Js8%cXUze4VqIKaufn&D96SAS|Av16^EGTIs)ZGtjK= zmA0XKsof1Pa5}sR2^~!C+6R?W3X*lX6+E26vIhq3-UGvwZD{=QJh1}`2l(v{8=^YX z+xFt*-J@4&b-)sp{|L;ezpLJzvJW|KnvWB#0P+XttqH2W9e4IBcmvQ#UD^f*BQb(c z5U~0d_vryp(hWplq+%ISanR$t4E0K{>_x_lv|S1~3$0@9>7E~7cp_4VA0p)o6>8L!n&Yq$wcYg^ar)a2SDk}`vHTu z(h^%p?JinG1&ADjp)}U(ON1sZ!46n-T?6(BwSRqD56*f8CIGH4ef+dDZckjkPCKd) z!(nGXTPE-B4@S>uFx(6B%cui-bUli$P$dO%X!>ne6T_b@Uk8mX`jw7pr6-UElf!W) zgSwASokO(d!9ppCmj|x(y%@Snf@D<;4iquJmO9ztFq`0Z0A^o7qPC#kT?VR>>&>js zDl>yQhrfz`22z);->*)OQo}q62JW}&ImE#ZVC~TQkRo9Z$|2qcmDD1kOV@s?u88&{ zP<0c?8bx0z{|tA9eg$L{V3NJSVe~63jMHY)upbX;Mu*fl8#HqQ zpQR`!Fe#3No-J$!*I885(@cs~=8$Lq2!8~r?w3IX9#+>@N@SX-yVow5I!xB9aXz%s zg-NJw-)8F@4SpNRalS$?2aj}Bauj9JiRlK$)<|T-d9cg)Kn8`!i7n7Uc&tngxbNFY z_N{?TTwLxZ`drvelm4+OP_#8ig=RjzOgPM&X3SSK`;DZVD;X;?B*+!luA2tO3}BGl zemLbdSR zglAT3uVxquv6Vso;faHI1rfVT^>k`8$vPiy0+r*l&kSQ#|K#)&4~}qIWxC*P9HnfhASC=NkO=F9TELXCC+Uyr|b&W(;%{qyMXuObT%6zKYk3nu5Z z*QAr{_BYq7=~?1iRY<}(B(=6?I78{y4b&FRtvxanE#P^q+0VdY>dLenhB4U#BKzS@ zzC8ET!?*$D;tlB)rhzFsL->^+|CGuG%RMVnoOXp-jf75LfL2U`Q=!ca)IT|0F?r}z z;e4$7CqM;t+3+&gJw5p5Prx!WiOwPv1SPGrD`f7XCk$}}$vnMDWYW?PE{ockV$P9}a7Dj+ zMRbsmm2j1sRqGfeeWV>$@7@Ck<|lCx=GjY-cm`4=%}5M#X-y}-k5LMlPL8u(4WOGD z$6GnwrVEsmetqsfJrAw83bQytST`jI0k>XaMA+VIU0|?8CH(eRUNh={76sB%=QP@p z2gP3?IapniCM{FD%j$w^l&W*XB5?e@3Hl-z@dQaCN1(g$?H{HGNRpJsuc&xa`5A7l zm-$9Iw(#fE`X8FG3bq@L9Ad*~c7EMbyQuUL!wm>P(Grc$RALBf_(9xzx=*Yz5i%*e zTHHNcGGk^`wb{}6@MGqebOy3qKf;Vu=ey6&V_Yw+3k>rQ6RVl5ZOE0Z)d#KJ-QNoJ znC_;cU;46TAa^_vp%be1O_;ApGwY|fGFM1fzZyEFyMYOMAEA)L$LE{>`#$O)NMT)J zHL=_QJUP7*WG4VqkxRxf^o3K}CPj`9wb{UynJ=zPA9hB;NA;X#u;xf_rsNRM0y15Q z&X*L{;@P;3^i>mcl0eDgx$*{0***=lVA3?Bf+ACIjlKOXw(7!EAH%0h&#TJ8IgA;x zKhXS;f0aCk>$Wxzou^jI%J2zpQTOUP%vPg7eehj(FCOI<=GRuPZ2Y6664aG{iZajVE)ZZfWJ z!q>g&7gYwtjdsPvM-zHA*oAgE(EN$V5qh~f8{uaI*sS`=ic9mgW!R<9&awf!KC0}l zN&;K%$el5R1dF)9_Uzng6V08G>cHh_9@R@5ZI_F5A97MJIT08HyC(;Kqv_?2WjrTm zAZBK}jWlOlUKiG|8{%lrr$+cF&AAc6)R}nBmQXcDNj7X>BzW^x4da%~Rq5p=T4K=` zs&IKE>1X3(^7~wKH#!W$eZMVtAhk?F0FApk@mBNtM?u3Y9%97syJeI1Gu7F=d6?ym zi?~ZYx>5Yb2bj`&p$?b(BD!YFioQe7`t3A-0?Rw@M#$&@6*nea3Ku6?|2)>0Ye1j^ zo<(#V8vZ=l^UGNF2)_t{hlAo(#BZ*btGD{>W>6r^fnUT!J%5;5H;{=jGx>)mu{#O* zY>H-T%Zq8nSp8Mq#}~F{zR7(@$rr;FsdXRxBdG9*CS-XzqLmO99t7?OK?e(Tc#l7w zB5a&CzWClU38|%OUktO5`=SjcjgGzh%BL5_MJ@ELmyR$jTuc0Gnaz%z9?8s$`%mWN z5p9;VRk7FB&>^V{&(2LpC}djeB*luWMF+AyDOt!vtD`>(o@eTHL9|Xptu#ocV!bDI z<5=q_i$qX5q~c|?_e*@hF=m@abTz!Ggi*dGnlf`Bb(mHg$vPWZwbYR>6ut1k*E zw=Im^PQAg|t%hKllw*ObR*6ZGR7t0boR~GG_m<$2YjV+nHoLEzn;?$C1ool!xad3> z!CoATYIjC+^-o7cQ(RN?F?*NHGW#-KVXe5Y0^&_@9QByone9r`?Bd4ClPC#{gtT}0 zUEeHwMsO_9bVhr@$msVP>7jK`G@&FZY174|j>WXeu7^O_;m z!^JWfqhuv^KsbvL4qGH;%KYrLl(fBc^OD1BFaSs|Gz(seX@MNuLp#VG3C$A(AK9I( z#dPT|6g)E>R%zit>bO1js-GaxkA32PJ95<-C((l%e?`0{j<+P%|H9-D zvdOQo85;=FzfE>;Q57PGgMNDNOGJB24^qitj5UfnpXd}7y!e@@i%g=(GfLnhbxYFO1&6r9c_ zbCP5zG2~3l=Y%`aN`~>h#p|~RHkFbi;Ot<*&{Pu_)D{r^>|`S7r6KRb`f$P+p+ZHL z5at!`FqX>$gyo&XJi3Qc&g79PuNqvFwWZc#U%k18JWO0ZLq-e`}nqGYjIxo^cx`yfyQ0e7x&cOSyP+o&8ME zIVe|Yt$}TUMpcZ9-V;q8=CCiclhZWFja3jiJD80VC;Q%ey}4Zb-bCfqxJ{ukl3hy` z9!|TZuRaDFH&2+-f=JJ@i)VJLq)$BJ`-ExaNQUp?m2Y>&1+=#b(W$xIVw6_a<8lNw z(bab4*}WKrRb{GmDI9w;ElH&k59cS`vJ?lVQHIiX)$Os8!1XK2nec_JZ~wH(7H4vt zzo-ef#(8eOQzy{1?k2wvyK3oD%bPy-*hcTJP5SWIO&^tWo5W+T{rPbCr-Vv)^Tbya zc>&OjXwh=EP25T@q6{6otLm4zdBdADDYFItYWOy}hib2y>}HGi@X0O8)zRgurfx4o z0_iUZO-i(u{Y9A=1+P`Ie)C-pamp`)sGGrlNXBEJC{Z>ooUE$h_2EEKn#dkcBH@uuaJ;s9$oIeQeKcs@B+k$NWxnBpc>%R z=wd7l&1`ceD}7*0FDn;vQ?A7j(OHm^rxZ11Ho>wXlmhX0%Z#M{~X?MpiE2!lj7*9NU;cJ-osyRkC7=FALNg4Lbosu6;$JI4vOMxK1#tA|(wUi_g8}4-5HRt)NY_3wP``!*` zCL`T}aivo1W0vRbJT7arTux?s#K8pp;sv zmft30Y!(c_WCgFJ4a+n3?g6iEt#EQ?@MMMvzTDndn}Ml6l!Kr2rdp5z7w;4mbxpm0 z+~`1Gwyyc1+UWV^*AY(-dY8^J_J=vYXOcTAvY?&{${zy2#2cy_VvE8VzTL zMHR(2Rw&2(-gCwKRmWN>P=0kDzC7nNNY;c+X7}UTfAjTWf1Yj#S0?p$$WL*ibmof1 zO@!oCH|seMpT5TQRi%NO&M>)=(l5JUT(Km5+W++YZ~wC~K#y9=)486rUhRt=B9~Rd z&1HtA4Cx$1w1PenOJ!fKptxVyE=Da2FUZ$97p9~rPSmW-?zUjzI8z=7PY~*Jj^&PS zR0@yXETKsoKBJIAv=hCXu8}-8dT%+)19Tlz2kZu0Il1oS?FO{T-U5a&rG|2j1kROw z(pgh#b2ICaVsQ-l+%*xgudugxqp@kNK+wvb%0((Hk90UJ44E%HW2COnpEY{9UC_u? z=(QGB)?KCQUoN^!Xf?>I$~u%$-`4ytl7pd4^~#RpA?}ByO@Uv$9i?G-;MW{pHHV=f z_cR|f*SZTR59~MF)#cWh{lK|j`I`x~_f`X%l=pl6@H3iE;F-76Dm$Lx1aD{jqZ>kh z{oM+oflyTbX@i`Q%`q8F>yyJj$bf?N!|5UqSpQ&B3kp^}H8i%4(()nS3u?<Ls z*YQ=7PCI}YJ&4jo8HRdn2XA=`wv#1; z%7&ZttwBm;qH?y`LtWPsHOYE<+@kV#>g|M> zwrMX)?7l#sq?DHk?cpSC52AvxG9{grSIvf9*) zVJbkpQLN{S29a-z&!Hq}2Souvu0i5?AsnaBQDs*-SAW=oFMLUX2PXVjTU zwi~FD;RP^Vg`p&b{JC9?u}bL+j)#EUgl0X!T5!DJXIpoUaB|eU(W2Sev5EEso!5*;1nMRa>Ov1$n_&`3jPMCVV9Tx{1m{tYM z-N$yQG|LOOmgoD*KF*R4gLrF+YNHjq2)vz|9Z9#ip*t?pXePP?IIVnJkXaw@qp^Ig zR!(vWWY#;BYHYn_ZD_+Q_0s#0!RdbWK(|;sUF!RujXE`la)b;wqDdfVh5sRTykq-) zd}+eAR?=ZMV@FY{m1TmkO(WaN^>xMy&4mf;iMFxP>MNik@LgGIvjuF=;- zTJMv`OKGH*uH6>gU#iQhze*neL!O1aRu!?cANSC~>dpF;q#w8_KbBs81{a0m0yqv5 zJ`SW>%hlTb=MCpufY3YXa!WO^;XUh#e(FUNZrM_0ZvNpLiu@BY;nHyzA(fLTbNzuV zu%mhB(pe*tIdJ5(B+K*ayY0itrfOV1?Vr;)kIUqCyjQ0ibKJi=LxudaB=BU*8dk}9 zUZCRQNm-=su&{k-Hr3-+)b5Kbil7`sbwWUz&s)JGpyMb3nJD*Dm{wJuHw{tv_J4rNFA4flhLW>T~!Y(2O$i?6fG;t{+g z^%=n=Wq$u^0lpN3ttLil;T1G@1yv53-emYUihisuI2}MRhU^d7y9{&Mb=)Zu!I2iU zPusO|Hsdx?D$6*;<1QM@JLix*RY~*tIVK{SBzD>&WB$wX$6QKDKs}g73F3O2fLp#=EmYw zxR|el$0g#uwM8^NQl^Q@Oa1(RwS!@Xk_97mbiq5!=|M6JcUhEYd=qLne_i?e@nj z7LZo7RldDA#l5}U?-HW^I}~v1D8?f1?*mz$C48qZ$-gGC?uO6TDO~$$Lv#_yP2aKE z@4W&0i}-1e$_7%8fGr;oPiVU+t0`l{eINjn+pmqUZXjJp2aGN6CK4(8%PK|FT_?jp z=S)P7fUZ54n~>bX12zmZV_95KI<XYBc6@;PjzAVuYY}^1*U(C55Kx2PLkR<%cWl5b=w z>>)>+dZcpdJ3xqFkjAG!NU}=`bKj4QXu6xDXJ}mNdEd7qw(bl)+B8y;M68hw z**e;T@0_HU3qp)nV{zB5#`1B@gdSgtR|RgGPJBws-?~P1rvH-~daHj@UL1TA_tOIk z$N#uL4XQQq;D7ysjJ)94cx3A4V3!fbPc=UvU;o-jzK@!3j;$+KD z2T~8HttfF^Z`fR1_{Xq^1sTZwejU4;@6!d*|xHNsv$`|D^?v2eXHLU)%I z_?a)-+W1|k2eYWmd`-Ebn~sVg=`u_2=H_(~kagcGj1j!bS&?E#i3{_II8Velrq9m9 zV#u@GcTm$PAZe=fs{Rg!2k@Gbjl?R zk6MbhS4KiV03q8Eo{4-Ta2qJ;<5m4RT~_PU*>}0-j{|@{04+lc8~iGrf25Nw>*=tj zW`Vh}tyN=~&pbrdNUi2G&M`-YJzfvqdzmXkhrmyP74F92nDPmk-_3{9lpx5qPa4me zGO-A=ze1G3Y1_MPs3ygDU}yp1r|5fD4Im!R9|l{@|G6H-f4Ls}7Lt6TjpvFn0W^sF zutO4P?(YLKMk*bXu{l>VwNh8 zbK(5$yjjnW`=nw&ceQ+f%f5_;w(Q4X%eMOVRUXXAHJ^RI?NF+HPwJj|=Z~z!f6}ZI zTZI;W1L6~gKhFz)=RPBUy7``^4hLAUty3g|L&KHsfqFESLln~dYV(o2|MJwS`ZAjO zDgmOs|65gQ(%^2a7J#vJSQ*rlbYH99^^?bf)zNNCyo4lms8(r1vZOF5vzIlrG*mNv z@76tlNI)olcggd}jE8+KJs-0Hmb2`klyVLt{C4o$e56TBSn!XB0)x+GOtgFqSqr&? zCuD(O=4Iv9&uVA~z6r{6BI>K~%`zYjphAa>Xl{PwvJ{dC z0@nFESC;2QaGSl$BP@tbPHR6-S3sP+He)uN>9qZ^ks@=UxLqew+~B@5(~}t9=#>o} z+F5`d)8Gqcou=z3UPXQcbdI~9DulN`gWVEw!qZDBPp!?jodVAs#w#XvWPx)?cm=dT(9i$$0oVzq2F-U7;@Zz!1GhEjDU`}pWZ2o}|VeQ`FqmOc=OZ(rg zaIaN|7sHCs@0OniI6$i1OH@#+yZ_%aD4+Crh5pbDN$h=-@>(hwQH>MvFdNSB@Q72j z!!|`?5Ftz)K)plJ=;{fn;q2fT`-fu*tN|CS*yBm4+3*ua{dg+UNs&aUhqB?mx7A%T1n z{F6bsx%RKrnFF*ARtdF^1%Md2@b>f0r@zQs;W|ys{2go0UkihBJ9i{*8V)L${mF|! zb^73-hvTx6q zy!Qqn$*9-{*i!#XWu`b_*+qDU#`0Z5;V2KdJYEpOlyX1fg<^Pi)N@wo=q{8-#|JFa zX{XP%$4r%a=*A9&?P+I+UmrK^tIjAUn!g?F6g zNhXi{_h7};mHuVgk#TKH&JM!IZS|EF2!DnjePeX7xnp;pqY~VKc};Ol_2nHFTb)`P zUs`g*W~QAqFrAHgSnyMiNOJ0^v*7B(J3q@%5${$T1DsDERYLHniuV`0JdSlbAD5)H z3Y|5;VmK5uipyc)lgmBwQPcbs+eWH}jtGshu;*`u=}#oO{X_)U9DI}DivZcN>}+Ut zw@((%SPOPHELP;xP|Hfk>ZrG|sg+GqiQ(l^WZchLKf;T*zuQgXs@U!P4cTwYe+;h<@RZCW* zkycKUu&8?(bhSwiPU;v;n$W(n-G1t5kXcAG$~|^XKy=p{F)6!}MLBDJ2o#KF(eCE2 zR+i>GU13Fw@sG;Aw8kO1{k4_N^*5Wo_0UU84z2{czh@m(An;tNMIeS8Ok**~n&{P_ z^Vbh`V8LAH?`{0(#yvtwq&63maZmHwlp7JVdeMQN`HAMr$qGb=HpB-3-^HDiyDT0< zc(7VSl4NL0O8h$M>lVQCfzq>6`X8%&plqEFN+r_wFQLIrZZ%DC=Qg6oHHxf=wpWWa z6(3w5f7|l)K%*#w^!iCO{v)2<+>XQ>Kn^E!@l-IWOMhGqO;Dq&Mfv6JIDH<*^UqW7 zegeew8KC9$9GHBAqbHdx(hlOu!0iJ7u|KgNvDDX!^Wni=w=MFfek``u5C1bgZ8#Kl z%|ZX2V;h^tjmpA>R|fO zI2gzj;R`2(J3`3mUcjCn0?$72@zH+H-2sXBF$O5r-~4zX+Aa6Gmf1Uy5PClsD9$1J z?$#rvBQBobp%zDsk>^isyvr6CBR00Msf?{MJaST2bd(E$)WJ%E?h2K(-#S;u7g zF({H@9MVE>0?9OlwtffD)*fJU(1FmtFnw!3e_%;DuNd(A==_pImv{AI+6fXsB5uyH ziv{eiZUYjR+{q(`uooN=mP{?_6_vnP@vfKtpsG2LjQ@ffdhsGGXUpSOoxG>j;dU7=d?xhe9x^*W}?}AhOb3 z=qd}N^zQ8=R5G4f8E&rMMCnL6*38$yQ_VY>Zj z{EsU%K`9>m2ktZ}PfZ+QGnaXu0#Se1AC8{z5QN72<3j_QBzZzem;@m~0dy|bDT|`% zB`91^G0H{i&?8HVF|P(EUHa51K%P+I27e_Tp6UF@@BqD+;3lA;KTb*DB0zeK)+L_( z^e;Dh1O~2@HIY;81o%iNP#MZgxOVT~&v}~$ zy!gTdXppJIWBs^n2U-hAp|zmM+3MeKq5+vNU71E#Ee@S#B5`4z^Q1=XG zI;=UDn>_e?N$8q!-?p0id)c8fOlXw}V!He#=jrjhbSnC!|L^A%1|4d&x>AcyVyshb zN)tQyduHgG(KHF2`jk#T3WU0(_z&um4185v@sXbp7!mp?VGLkxpMh~&mxIZfnbwVD z@b`((H6y)|34}C%zRBtHNen%o|ND5HUbp{!JpT9b_;>Ya5F`4%e)3y1{t{@fofND? zHZ7Mo*~oO!)aqaMAR_1W^_L(2^F}W{fLH6IM@A?cp@en$WPOa^dgI4>f21|L4gsaHN79_Xa4+bVWa@=A6 ze}Cg2j_#v>eD~ zY03X|!Jig_5B`S>p1K$C`2>Z5c?q(&N~w?`;lR=5Jq)(R*(P9Oa?ODT2g;Jw|H5Rz zlHwx+7NN(9o&T=U(cBLv%`D0R=e`UWHC)iL;Mm^u?&^=SSUP?XFT`W|9+-{p-k#>R zig^7W%erkWw8r;q^9}P!m&+A^JRvId>u~IpxjQp6E)|--R7pp&7NZAH;Zl z!qkaMNU8OSZz)L{t4m-sE{x`Nmqd8P5k}gn(5jI3Zqw>KP~qR6QzfU43!|=Cmhz#! z%G3G%{nl~t9!KG7$E$$7)wySnJ|o{10MQR9NUkmZeE0J6n|wmb27WJUMe_gJ;WeP` z_d+AX5J;E1c(WD^dB1d0cL|bt=HH59s(_`ZXo(C$aXM9%7U4%(w61?@BWSO*g_;R{ zQ9yw2Ksd(d-J~ZL(igs4ynqmo#-B9JuO(N zd8Dq$>6lq8zpYJq3+dt{|UG zhc;nZks5qPu+$wsQyiCdzF#v|PeuYmJ>tCws4N=8KH;^yIx4VD4O+il(uq+CW+uR# zI*1KL2{m9{XK(|>9Gpf~ziP?NfL3a>K3d7H(ynzYp{t4dl7c;W@ZDBp zBbDv|;nw|Ea+5sv58<1I3~eHX3CAJ?L9s^aHOONianP8q9Jp z%eKvHq*b;*%eA3dv@j4y4iDv{m<7LG@qqh90j%N6*|LJgyQAuii5;naynDd2bkvx@ zA%_|9zMW8+*mz}yskEBCPGH#V%JANwozch^{CGiWX}(l?mK5pSo=DR$`)@f=HyJK9%dh=SLT0#*2DudO?>;&49_=UhmHw6xkB(3Hj=hyJnZ=Fz!vC; zp<3z&M4$Q5u$JWlILF;njTv|>t(*Qd{rHu!K4-hj{J@QUa7eMV51V?X1@~DRnP#a5 ze&q_M>-C*$ZKy&kd29d77&5ryRVF{3OzuCWLe(Yk#?iA}q{GIY*J62t<-abuH>$}| z4z-sH03&ge_7^nLf~c8pz*M9QR)Je(z?XOodE~TAzYks9US#?rSHiv-L&JT_kkm+G z&5`wSecC(~2jX3l;Ddg@d0v8d_E+oHiJd7p$IuQl4AU060&%AS0J%Q6`Yl0X3}9(K zMH$-5{Kc8Sh3X6nf8naTuo(g)5~FgsXe+kMe zO&!BKfxW97BABm%!;$Beudoi_WB(;i987#f;j=2ypMZw(EzH##0y<8Pd#VO0f-c-+{KW96Ba_|y847kq0gf*9u-P}kADy|XlblU6^)D<0*) zQf;X~8rJ*r*Bgo9i8v|c8Vf@ez5Gji&J;%!znGSKT{-zf0_&C~^b0+E0~w%xU}IC^ z@E}-wVL^B-zZA`ZKIhb;%hQp+YvgSKaMOh!KlfY!)ve zEL=M^i~GO`)CtIFCR~dg4$6mB@s_QP-*fNwyDovLt>qzHP2*wAvD{3oVWbE6&x^*v z{`X`$CP&NMT}ekWI`$pZBya%;EgwKwigM9UiFkMymAepm4siqVH%g|+1}qmteb3xHARvXM+}T{B&CAId7It!(&m77y zMSSo$0q&>xcGaOEmsyQfF8T!t&&(Ldd)#; z0ILzR1Y6aUX-w|v^%aYbeG9hHjP2TeU`4YCH?RG3QaBTwZtareJkQoH#~ieI?>pY$ z4@n1r0>CLQ$x8PwIFwzOYY}u6_XaT){b#O0KDwtthF*7!{sb7@K6l2hvNtXnbn!b| z>d^v^>;ArfLC`tIP^{SyqlON$d|t{MEX)C-DQA7g^-JZ1-ON5xvV4q!Zt6`jSP=)# za;LG2fV>PEPqtO}_!B!8e&Rp9< zM#%Ng*zx1Zb$@6954)!x%7Ihk12fRiO`yf$D|q4O^V8p>1bz}L^>o0!33e>1ak)1Y z$|K3N0m_rs-Eeo(GWpa3+1!>H@FQwq561On`do$AGjKbcuxzvD_r9k9AY#JPH5ViX zfi*YnJFEAbD{cS{PC1`_;$OE{Dpfr8>5EeH7{ zfY1+{Gxq5z z6yZA68DO*$VuMngmWm=O4)UL(`dUsQQ(}InFEapbftNW~(iOlv^q~I9Z7e@rzvkgX z=thRpXbDhG(NNdczw&1WbnisV%b)+U*_w#K(R=F4?ER+zuoG36O0Ukd-)GaAn`dP1 zRpkyLG1y{4%@Ne9JV3EP*YU-B>TiO;4M=Wuchkf2Wvo+AidoxhsL|=yX{At&9nX-S z>p#6`mlHyR^Ly0BAa^Q6Xt7)!o;gRAuMfV|%@JtbZY-9Bsvb!NMn(e=r~FWysX`tP|11z;yU>2lo`3ie4E_s?S4frVut^sy&PbT_o)6EU z6Ql^wrr+YJ$CrFfY0OWdABTDEti+R=XuPb+dtaPz^s(;aJDX_tLm)=r76@w9GiLPn z4ua4~dgNzNY;;(+Vw~6Mg=z)sO=i67_$-*RM>Uw70~MGQYlf5tE-qWQOqckF0>59n zfW22bV=RaBm}u*f9Zug_&WhL21Ay9AJO!565m*dEhPHHaz-7~ytxrIQm!HTl3Ba>d z2KN8lG3a|R+Z;ZfR6~CrF++ouk~g|sT;s|<{mf-%UX`c>6@xad_y&4y3lV-MpDN+y zGSnDw?b;?v!gM>@>&7OqABMK7`Ge3_jd?nDRS{|tjxjeQC_;dG0F zXp2KV=nMKl?B8X{#LJhO!4x;h0t+vfm9gyAcW$faFNO0>rMe=jNtqvlfCIV7w;ide z#d>P#k`M9X%e=|rpXj_7&g7KX*cr50R$BfH;7;N0+H!*BaL@nQ1$V&3YGMBy;$1gL zo~LCV97Ft3iLZ9a!M7D=9EJ_RR^fb zm%&yRN8YmzKBo^zHz|XP)_I`RP#Ak}@7K{~Vswfadv1)yCwQo(ZY?M-@&bz#5dio}g$zlxxoZg#MqXdb^Fn^_)6HFjBmDRvT1@mYMs>vU^wN#zpiYQ%qt~*rn+eYE-Ts?Ez?GZv$W}!Y%0;1;bItqXteX_wO)VPS8ptsqpFe9%7t7r*h-FuShkxV%aS6-7 zRGq?Znb7Xi7iDl1KMM~2^(+GO_E%y2Nn z0at{saPxHsMZSI3_8tX|-J7(xS*)9$BM$*yUlNK}SO!lW>C0sfA}d{Mh-A>^&>_ofbLLcCMeZ!XT{3S>Cm)5DJ;c=0i4gESXhVH z%es!AK_bGXJhzIRF$ek=t`1+&Uv-!D>JpD`VsDY zq!#!MAx{dD1_f%M5Aw~)l=K1*#rizBF_{pxLss4_N|)s7$wsu|L}tnFUjvE;V7>CE z7Hcnn1wwodM|4zGH8yF!TbP5srQFk6cq>R9f(l9q)9Bw zdkd+~pbbW9PmoqK5zjBk8N@FrajdB5E%V#ytOx;>n;WReC|+V0Uw?O?#tMBueD#v% zOO6b3^`~oB2jo;ZhISN=Ejw#T_gL)-L@!@gX74 zVZkdYUo%)&HID$p;=+FWIUG0;CPa8T!Trs_8a(|>7Lc6!O^h?k=ldc@8n-jLaMqD2 z0Z2)QT^W^)k-kG-8Nkzund}EkGwYDarU4qB;rz@`ab{RRKGj)~-x=7@$;v5x8w$NP z`xvPR&Nbq0_&qCE2?@2sp`LBLaXMY9kS6)5^HM3g*+vlfbiaV+YCQO z?7+)k&pVHW@tmPhp7T-%Oonh5A zjPys9KWM%%HEN=Na1atg$Mig?yD0Hx{LVWyuN$Nf%|UJ!0M);I>WsJMQKQ!01fe?FZrPOoJ+2)%fJ>$=R{-cN6h65@Z$liPo1m;*OWE@39#kT zx8vkHEMsI?05a#tlthb*Ak7^_+VhBTnQ_6M9!>``w*;Z@ISWVi1Ah<6~ zemkurEq|g7G<$o!%rn=F;cO*U2#Z?1wt*A7{YQ z#U8pm;!cO9WP2fbdzMV+TnIgIcgXZ~sr&O6q(6?CTKF=?ML-bz%! zV3-#T?e@DCD?w?qTNwOD2(r)ipzDlOxQ@N|y#Iu;=0Z)rG{EYCMd&nC5H0%4G#?(^2Dh82})K)6~V2M;`Nzsqtt=|k?^(+xW+hD%( z_GN-dr4n=d1<74tPt=rOb!-*9x{;m4p{S6^Dkp9bRGtt|chibsNI{2$d&GjsZ*CPN z$KC+RvEbvTM}Q(jPByawh*u;p2^NDhHEoX(g1c#b9b_SssqBKOe7S{vkQ6CnZ_@a6 zId_s|4w41xLePIQHV6a-Rxe6tj*E;7S271cHmuQGDO1>LNNt?&}(#WyZbYV{6RS~Ne~{viX6i<@8o51F5o6->~p@~R&XS1PO!1JJx|PE@>= zwXmLIg{b!YAGxvM3cxC2huanhVGueQ<$5zT(vHaj4Y`@X?JNeN3}krAQ9-!i;Z zf2I`}9vniJMv7I_vRa7MevfN{lg=7)mD`fx&n1Rtx7tdX(9Bebr?wdAVwt}gnF*?` zG-H6+{#ORL*5J)}`$&?spwx-u zxA%ihpE2I&KOQ<%XM;D~{Cbn<05Z$iBhmT)kE0RuYohFUPPDnGF#0T$8oPLv|IXED z5t@Fuu+`uM${~ZMR>Vy>r;oU~I@@auZFtqmUL-4Pv|1p(B9n?t;w4 zq^;BD%Ts7vF7;y98nE0ti))=gw?>Cj*Eu0vj9%Nuph4`qQ!Q!~cDL2W*uK#YxlwI{ zOWei#bnM1^EU{bF9oZd}Yqnu6Za5?40cH*ScoqV%B5a1d$Vw99>dhE$psL6Bm3T*3AxXl|`W2ZI&Hu7fMwhYSWdKW{a|! z_QGIPtu`EIar6uEd#P!!Y&4Ne#r3z7aRKAjhJ5c;Tb<&hb?XNo)@KKnFRRwYA3Ee5 zRKMUyG-rL#X8V~@Y3Z?~aNojt2R{+zZk+$4I;#?I^BnMT5`q^Nm)q@7?|v|hPh@o< z*v1Ae*qY7WSjaC7NiWm}R}O50tc!8Y(OuBj7;yt%!`!_W8g+2$=So<{mfajJyS^8v z9(BFWGR+Fdql&`ke7qB0`@+&3Kwk1R;lG+4M}v6n7j z=r2%eyR00=DTmMCQ+!_@fKS0LtbEc{$Qln*Kzv3H9hS;Gf#%^#s`|M&lhEXgj(I*eUJQuck9wQOlDjYL$onjx~4HD<;ZF+=tMKR@0-+?u)Oyw3A{KA*?&IBq$vcgVKrZyGEX zJy#0>(#gs8Au;gV9658)S^?I+gYH~_3bw!5<}~MTqPPXe=)H(i6*QhCOoPKBrfygx zED#FCrX}mf+&?*|&c>!?jeK6SvMsSo`GG(SyO1PR^_(d8QrY?pD$MRkIQvN0qHhYi zZ`L6x=hL>xI4MtsIq2_4V6V0P$`Pbmp0)bDjNa>cAZ_ zaB|hrEn-P1yHnpaTT^+j{bl{gBtF^i=Umy~5lZppf||t7)Ij>4D4@f83v9M@;B<4S z&;}0!>SUQ0a1h-#R)}(X!a1|TFO9u?Nu4ydfug!sDDS%0u2(vhnIjB>Yvo3Yf&q7M z#O^XWgWOxNLtv5O_8dp}#kn6<$Q=-oeG?s=%!!imx&& z^r*oQ?rPqadP}2_Tz3WRKIHBKzlJl$cxx4ik43%T0Wc&-nZrcYD5(F$A<7{3bDQem z*EvtX(gR-Cs$FmZ2b?@uTJrxaEvDnbMGU2GE)#2ix?f`=d@RZ8}x+Ex5k-Us( z0yiRY2v1pzGY46K zxCgF<14hiL=#V38OO_C66&A@CI4AvEH|i;8?6F=W!sJ5EYpcKTV2j$v0D0i}DTMiTu1D4v`G-D32OoL`;x;W+GM=Q?)^?xpp zN0z*72cQ&x&Xn0rV_sb<(KUh$JehT%c0^ymZ$ZJ6E-c~qz(nOm;vQX()xy;iL!E*n za$jI9-4cu1UNxX2df)ly;eEp)V{~W;cm-YWjujv>e?1mnlE*GQo~V2Vl@aWq5|@U! zwqX2wfEiNAp*hU3feL(lZ{-vF<96*M8C445T*jTSW*gmMaE5bHW|hGt)NQi6`7Zbn zMwkasFNA_x-UZz%Ya3s%+6Gtqm;?LAAWxeS3y(tX^n?>oZHln)VClSV`Q3~Tg;$W8 zFIT3_ROW)BdlfUN$M^eW^DKyAEW9vz$wEE|CeC{8Hak9=&(Ek!770@Xn>)sZP5)QA z!?lc_Z~<>2TP^12rok@B$qW;biJ%prTCBC?m*Muc@khz)&nBQo2pvlMiM@xXV@5X^ zwDaq+SsNUZi;+05RSrW7-doZx%`CsPk2-SQv=;qYsGgD4&pU4)D8?`5qPxYY?k(9W z{8GCQbq7ZP?{?Dg=y2pHh)X@W2H~K~O*ckzVwlz~=`R2A)j|sg>^xoqeZDWPG{KyT z({ZVqrRQ(u)UHOy@52_tn*Eb}FiuM}rhA&pINY^`C0oCw1sA5(PCdWVi6z@$4yToR zFu9tAjwfiCP!5FJIHdJH|_2npo~yQ2!}Q@~K&Rr~f*f;Kf$ zT%Oxhu7-Z}#Z{14V0BKR0234-Sq}FH?ICjxF*EvXl4;0k>T?uIy_@s~up=U?!e{KY zYxUtgwcHk}ccyNb+BNd$hr{mNt| zaM*PA7(3b*G5ea+^d9}(j{RlYa~}3m1-0`r_4m>B90jF>2gr|t4#b!I-y*-d99`|w z^DBQ;ybVb?2M&G+`hyWIfS>dMOe;QaC01eV7-&%4rdG(!7@wRIt_On(FL$_oJ~8Ua zRFqi|%yTqJP_tf%VGzCnhCOi#1RNunDF~qC&%(e#-)SzEHbmvx6}i>4&zlPkZ34YK)6ZXY`&opV0jM%_pduY#v) zT?D~?Hfr3f;qooIot4fp-#~OK+f=)-_$}c-vgHlbo?rt>@_6*vy?iKKpkqn)c~0=i z5mOS1uEt}*;YjCsY*}qCF;WpE=gu!XRTw4D={`asxQdcut)JL^?;$ zyj0;+NpL-Dh2lsV0X4O&7}f=tg-?u;upAR-Cjt#0zF#23(?b4%rLi#z##;o_P*MOy zTlfuM|8pTH2TJT1gz}cEYo9`#8%X|EW%zyAumk5Bvw4Kq+j51yQt`y^)0%v&S#6}+SIfG2viPS5gVHwfji!tjJi7ipXRVC472@26KaJ$RNp@a>2$WGnEV81zz zXoUpl6?`cw%GaQ}k5#!9v0D#@yKYW)4(_Q!fzl=AI*z_ATzz14At+Bn*BC7@7vHS7 zcRed6HJ`Dddmca4yZQs_kU|LC)dw>{P{%A@t{$cKOGof&B2ljcvV;-}Ty@$w zBcnm@m?`}jP|Lbt4rYYm|I7%@&#@ZE97;YfmQPvIW)uoWnGfyf-v*3g_BZk!j0djz0ny}jBPuf);k%f!=1dV zW@HYB<}LAVl;yPIGE`(?H@h`8;)0QI@E4?9<@{m!PN6yR+y~oGD)X-jS_K-}RjeXj zf6G9Xv7c6sdi|;HIxafICF}~TI( zOq8ZW+H3Z$ov|;6>cH;s;4SNuWhehXESdsKX17ty3R%q^0)}fB-c(p{TIV(hm=cD9 z3>snuMH=D`XPlrTG^e~cx-GOUQ%9Q))}V3>zYj%)b!i+UMtFtdgVc(KHbWhGVXu^@ zAB(Koa63@QJ`eBdSX+SbjM;7wmwS><1KT|`eu0d%qgS|t`?WZkL2VQIctWb)K%XYl zgH;vw5vqM}aCbJXX~rHKdeC4b8NXM*DwzAtOU8Ypyx!sl5dP4lqQi5bnmuEXSN-EiXpfw_YC6Wjev z17qgREopTHaW4F(+I7ZNQ|)Aw5p1)0HcDuXQ%G+$$Igva&(@RfYyUY?Z8Z6*;I>3t zaN1X@#1u&vyH^%NMuNV3jzT(`FQJ5yAJH;njw29)pC{kC7346ceK{KREhf}b>!FQDFO_V~VlM2x z<4Mz#wHnn#V8IcSgvYyRdp&q0L%xJsueZ7JBD84NrJrvDaB`B2E463- ztJ6*RG1-YnOo56whZ;3pDJPr>do+}R(?OG70&6DPmW03_(&ljPV z@PF0kfX$yb1|D6;`nX{pU`LnP@b@Uh$IX6POdJm8T}X5@rya{ZkXxqk9y&w9J z{#7bFWnvR_LFgEU#FF6Ai-i{L?3_PJ^$N_7JU~^9ax^}xzb(W>*V-dansvi{(*Fke z;tlBgncYo(WJiXDvC#;aH9S<@GV=HHth0pUaN3%BxOJHCf;9Q8$gEC*>?sy~{GsLW z@^&RkgBx({KAI}-NLPDDVpf>vFx7)S7B-!2IEoLh?QOz+hwkyq_saKiYIAKIM=O4; z+O8K;M5x~M3AR2`#|it8ediFlf;vRcx2%EfywS@*a;x$nB$IGXfn|0)ngbBA8(+j6 zL-8yV%zx?}*EyOYRw_EOMHL}MdfLON{3#iRuQklMbB^+w9wbI*`4DEzm^49dIULf3p2LBqwNT#p|4Lhab+0Xet1( zXNEMlW>}Bd?@=-x+avWmZi3`Q_xFVF9G8(SNRC^15lfQ-89NAtwaI%WMm9-;r~gpV zjG~yQ7P|tAkFDPlezBYN=#6uCzSMNDGY3&BF~J8o$6ZcC$zK({XaDm7i+#&nn3M!K z!(^CR)JnJJeT>qDk>S^do z4l0s7sW3Xp7kn1>8Vm(Yw_rY6V1P;bh!t;t@1BMC zGXZ&Slt0(XN^spv4g%~hYHDfHH~b8Wc=eO|ChEhmp%Kpp<4{cc@m*Gnw2{eysd7Gw z{VT>{QK9#saBCHCig{0oaS%K92BWm~LS>al9W%toLK#YW{8LM*&1gft*MIKocT>G19%s8t&NHMuLt=KQxH*X03Xgw5vMqk7Bi-__<3P`xtNcw1z7|LG zlti+P$)M&?!}78*BKEZ#ZnQ={5iQsxrz?mpjQC47g4cSxiC*|lred;TaP1(;*8G^L zb2!7u5NO{@PWkn*CkwjSr+Y6+ATly+kZZd9JK98TaUMvE%U(s zDa>|aX@0nc`k;40q75<3q#RW)oSrixMICTYv&qM6d zYp&5+k(_okdlK=^6NM)nLM{-;VLssIw*HV?*evIWT6vi@!(|DZ1x9jS+~+rw-D` z+1GZ=y~vV^qq_rw*IduMTdLQN5Tgj=Sg&n8)##-oHD?iuR95-bF@o!tt?R)&^b1L* zh1_p<``RtA7Shc}sKg0XEl)LWcuG$RYv+WlJ5+V;=>`~!dyZ&}qfzu+Q9R2aaa<7j z$UOs9-4{(yY?UM&*R+dWz2Zx$i(TDCpS^+ikvT%(DsPnVvzQloPR7Lm!#r zMzNRMMV}8B{auSRKDF+<{OLAMuKT3ksg2|DQC3OrH8kr>^W^V4XhQ=5BI~EU>pQPY z1d_*v(JfwdyeKhHhyJljbC&UGv3a#0E5JLyM7`xLJUkJ39cmHJo{7c60p_%TrKNW|bi`S*BNU;V$^&<1b-cVa)Pk8o1 z%>#NiT{5w+C@)viMj3yZ)NYm2psl8Ik%45Id8a%**@5c>H@&!~xdf)|fvoo=I#&Rm zco^3}MKQPhaKdA&Q?it?QeUCtme{o<_+l89c5Q561ry&r_YWh6ne zQ?phjZ0p_T?9jfIGCEHwL6lXnoz96R1%&2w#=;GzyshOj2>1X~4}s2WlTPMATr*pK z@8<`s5VD4jKVJ9+NN4{{KZVAV3q?L5dF9L6!d!H!g!))bB1TeVRA4!%XxMd(0CN*; z$y{$fzRH@}7sprWw5je%4t&R4knLAC)w|F+Rjnw~)T%KinfbL@z%F!12$Mti3oWaB zC@!ybhxO@T1wHRX1V0L%TBdaI)sEBs7oKQ1tczrNviqDk(J#CSnn_2WedfGqQs1le zQi`Qs40nm2|10o&H@HChb2kw{4*&a1wBRemDzPSI8M3_cly#=x00*%Lv}9Au7<>@v z1Sh=0lrdt;%P@$mQy_Il&%(X^xka;GAlJ)d-(LEB>y-V*kz5eub$PSK2``SO_^H+u zB-|NL+8{modZA|3$aZn+&h&bbT3JCY8OY?~1~fJ*!nZkj~*|EMk}-gQf|6>XIQJ{syutL;6}F zu#j~zrFl4Ztv+Y%H!p4)Qe9?^Chom z*|=Z5ud4Z$R=u|WxbMx&R<{soozt5pFJHM;1L?JfLdwwvKN{g)hrYV6c0l-N4tC0Xm@;8ba6l~_;H&B`-!1vf1=^_9QPyy-BXds-K? zYjeeI*i$-Jt>2E!)i3vJB}hu#;tp1vU!6EJW%5$Jg_QjL<*_L%JI`SqJQ;c8 zrl#Mfx&jH7;LcH*QsfjAzgP@{ubdZ?zuVM`XSbUs%_pb-gb8N232eI4Y<)`(r96lq zv37nZmu8%Ab9DlS}GI^$j= zkXn>gd5Z|{k-D~Eg7u!zcq?8e zGHjkW_aqi6p7hiQDr-0ndRm45+0#Ofg@}2XX`Dp^rcg)nJ&lY@JQNd=w%km$2(dwv z8@QK#Zh8LvR?6Ei6b9_r$ogxsmtLt6r{on#b{6J9v1SX8-+6woLOpdxr6)4GBbmt; zUl4FpTER5nRec&pj}iYrJ%Nw{|V~5SQ%MJ!g=dm_u^EbP*(1f8Drb4dBN2V%~qT3 z*w0_dc=arrVT+JzarCeFH&cF<&p!B$;RD*uy(NHdt};?IAKW$~-4(yFfW@god+^Eu z_|V^1?l4xyJc5>i<>W6Erq; z6zVHkE*aVD9hEOV3*F&xeIxiZ*(oqBb$DC%#tXbiQ0a`*r$jtH^Zm(*|SG%DZ&>*PlY?2>n zVUsjuNm-RdRVL4PHE$8Y5JD}rswc0T#1m!pl=xbV2JLY3TsDS(#DE1E9~{FSu5p`% z2rFc1#;C7C0ZG~#o*a#_J4+KyO$KTStesH8%RJ*(DhgJ4vsO0UOU*7Kz^5^iK+Qu4wLo0^_XMyAI88WFrusR z*D+#%cdu>PKGVkt?KVb(3zzJ~LNAm1UCE!;pS#!orMzj%ZTKLc@{vlVt>< z@l(3dV@2)TM}Qw@QsM{88d788s6DRV>T%1rPskx-fQaOGX{HoD) zpkPDj&;U2_rKa>CkYst0kYsgdrGv5cqW11&H6J7*tvWRf8Ql1R$a$`bMYCj){}Rt4 z2d8w7+CubdMEH7^PQpjrpjTm++m*4tcc4I|W8T4NX)?&l#1s9K47m1w@w1JlGmT!eum;EYlV8*LQ>#Dt^kkan!u3wT~i&o)Ql;)yB< zxfpJ_N8*?^l0L=$PT?UQEp7qL4k1HENIe;8rlXU~>vqUEilAL0WU22Lwg7tbb|Uv# zb?Hs567TdFZUQ3)dl%`r->lFnoj5~gepaNL+SOdxImgCWYPI0i$fMhO=U3EHB_C>? z`mjSet5CA{qdGT-=C+PNnJm@zI`E^xce;T97t(%%$7|}~8T>79@X=iQu-jr>3G<1~`s<*|MA&1ioV7P(R`6i8U38y82S=xy% z+y+QUm0q}-;UzFDPE{E*!>(!{XSxT3naAC669$X=A5%1EyL~=&W`W1H1V|SXH61x?a4oW+yAx(8dwZARtgp;gV0j z$JPh5lh16GRz9660I%Iku@}@aRM9n01r-iga9BOv1;izI$Df=uZcfUxFak9Vtvai0 z#6o&Kq=)Z-nW536W}6}o?E-p9O+G^2P~>4m?z0@x{AScP04ZzgDUY=PZ8-c2P|VMP z0njmu!S^Nh$~H?l=WqUv{x;p%nd8BLKz4b>2!Ch9)AQMbIP-9os@T0KRBZQK$cSy49GVo+U#o0PHhPRZXNT zTUr99@n|I_Fw`DlccWAst7COFI(x@I>8-7R*@)#=FS!pahtJ>~%FGlJc@4uV01lS# z+~D6Q2{jJR%vL{r1!dOqYo&tMm^LRb3PUBfMX)InV6sFux?v^3) zFnbH`qFN{oQlhV=jv8HSxB=$#+{pN;Tp8aMW+NDK2`HCKPx)U8X+e2AtTPZJ`E(SQ z&E3R117Odyo_B;=o$`5LuXB1pLten8oj2UQ5ykd^Zu`XWtxYhsGUQjosxFl4qEPx( z54iBg-ZMeKu&FB;@?c1N9k$QPAlrHisBRMo0D^W7+1rHp3@A4;3*W$QNNqD;aWSy- z)LSz!$)w)}7$XD}+8l|bl$68M8>m!O?2R;4PVceaCOjE$en58*Sj-ZSQa<}3@E-wt z=IOP&@SOrh8Weymn@0A<0c+K1>i=Q40}J%|r%Iie7O2U3OnXv@n8ME*#V3BBXy2abiB{!VTjiIjA0=tsPHFQbZ#mY}yw|Go1RfI<( zgsx#s(t0M1L!dtJkQ+zA`#r!^V7WYS&P~=x9LeU?8?NDPc^;_aCtZYoEgLc3c+}J# zy8fDGxKTC1gPT3JtI9OYX=3ih2aJiiuV~L3BV~Dn&Ae724@+Ip;UqClcow9M9{jKu zhzhL0Dh5mK15{(QY(OOqx3%fz5$Dd(OSDVRUNMVMi;R%zj?yl;nJI@w3-W;20T$6k z$SFMsj1!3=MO9sobxepa_(r+<&w+iDnX0v4uTq)m3gCu=sT}0CekAq>Xuev8qy?W>9NW(Cz1FnTm-e|oqvk$-gxD4o#cPVqD477#M;x5e+cY-YQ6Tp^I#AV70bq+dC{hzxH(>_fo~Ug5 z=*1-!x(mM?Mi?1fcqH}Wr}6>hG=%je!uew`L*C(^Fl{Haz~*S*7?4W}Y(dteb`j@J z4_Fe40LJ?GC6iuRfnwo|uq-Y6s;Wzc&vL?U8nKiKhmVd!EJ>16R!j%Uf_fLpFb$MP zKM;61O7$SS#j6Fo4?^9IF^@OY-Q>!4E$h=5b>R&CdHiYVlQmKFm5Qla{2j|il}Jv% z7(TsB(bu!n(T8GqL%8L|rty9wB9_s1{^K=KMw4t#U}Si;=2WFjMd~9$vtYH< zZuYygxS5-zh~+b{S46D>gd{zSsDQpK2#7%qZy2h|G))N!P+~59ex~HbaV4vkk*C|o zElI@m3e1hB!!s$bK&JXZ7#s2{n<1aoR}ltTR%f-W@xw)tWZnLTLlH-o-A3OT(nY21 z0Ds7-G&cWn7A2i5;tS2JH^mjMx1{WgLR!X~G>pc2W4lmXXbUk-;qhKE7CG@|Zsqmx zIHL{`hVa&}o$7dRwQ{uc*(2BtYZL)&2z^er3W=S7@p}#{LuY#9z;*SHi)lbi8C+wi zzU5y7*BBN>y$Dm9hsX7$h6D|H%GNCF)o`qt^)!U*D%D(QHjZ{Gy!`5GJzH{;?p-0{ zDuy|?eZJtu^P&fWTAT>`LydV?RkES!K%RSI(Q1oRn_5{}7t z;;KQ?Lyk+nAVb46m{il(q4X*Y7s)?_lFsaYPFI^*CsQiBefL2c7%ogOFMahlt#?UhlWOtvv zifFgT7bs!EwR(?4F`F$Ju=k6*RU2vq3B5rbE^5o$&B#qHvq!HA3X7_u((}{ugn=}F zyLmz_7i|aE;}>S0W6uyh_G=_Q)+6PR?_t&9PZMf{tp`ul){UND)(#P#Mr+}f9GFWh zdSQCAO*e0ggaAL=Ls>Lj(&oeYi!phnjrJ8Ix`}{<+p1?-v*70>OP%mR9XI9X<#X^N zdBT?VzrX3j1)!n0CGVr3d~N_xXtm`TG`ai)e_E5kx%EDIw1+jrGne~gKt<*tVk}b~ zRZcG3pCy{C>K^H7vNj~#AlPB@k}&*6oB-`=-tOfLGllz;v@@v593dz@eUpG8=HJ*) z?nO=N!Mqh2?x`|uqo4$eWUai9oqC2jx1c4cXAo4>9b4YsD`x5lQzYEQ1~cP93h1qT zTtT9m7oh-aW9*=lAdvRNZ&`}%cFRo)7Iwkm7moq=?SV}3}ep)N3heXD3hJT5dw3a+e~%|%LT zOHnf7$c=t1-zqRQa3g6?ST}6De&p(+iN#~=t*tOII7Nj@l+<Vzq&3t=`dA(lGgj#h1Iz=38L@S!hdN2 zQW&lhD^gO_ajVhwbJ@xLsU?KcV!5oCW%x=hTWS&W+Dh0rRO05xFlj;Fxu(5Q5#x|` z3qJ)JO_wEy#7BHCxpyc^x#Hdawfpp^A40a6?z0Da`dvs*uUS}K4#F;ue&jzi09}2q zB>S={{mQVMZVs#JsCZ}`f*7+q>d#+Xw$5ex_Sa8h>_wx8>!Rsjby`&y;58368z&#*#!`FVmZulAgh2~1G5&3CBumfEA z+Xm;1pD-19YO_C;KQ7PWjzm5}R@bqXGk%scFs4M}1LFNGCOf6`iP$%^j4DNKF5hgb zGFyRYl4$Rji%liC&65?-{Bxb1D~eOKDRTF6o!?l>hQ!4y)Y{c}iyg0D>*zn?JI1_$ zOTS6%K(t5=){*(nB^|m7K9*YIm%pw=$kb%e;}uyFi{#b1d6fgKR+_IDfGxYH8r&rA zEcU$r0lo~A5I7|c(}&jOUx5F7Myf%BL5}L3H-Kjr{u%nI{aZWI09o<>19dpy#9#S# za2KY7E_a7;`UjPVC^Sn~c-mDAE-A-9dJUYzmV7}CorJpK;@}^O^mqCu4gRC}5j5*; z1%CzruyU25{D9f{gGZ|YS@Y>IGJk>2&|CmV*qXbc15L{(2s81@y}u z*MJAI)tSZ(B*;S#b0--ujvf z#o<7BVN+lu*RAS8w$W5BHjN(vZ}f>=da|kg46rt?g3#pR^{d~67AxN$Fp3Ky4<+Cg z?4JP#<-zZ?F!)ir$hk{jIPiOJ(BCHds0%%9X9Kw&C5b!!(KI^XiYd^-R(xZD;IQh-3U1rSX@<-xhjNkc|oEa^7+wV7}a zz^x+m5_!(#8_332=ACd8g`O(d3An-Etu1GA%8w;CxPC8A%UC*HGokK5u<;|eV|!D? z^hbq~qbEZU|9rp8te{xZ;HV)7-kZt3OQJq1g8^P@0n6Eylx|11uA3j*cmE=U|68Py zS@gA;4~fb3;2})b;dUa-+bvG?>&4uIT&}mDfV-1c5j{EQnKe`Fs7vHA67c(^LpGQq z4LQ-@-ImXQHxhi*9u@G8UXJ$pp!mdS{pcGLJ9cgOHNP`p#A=^dJ@Jq?ADHXk8iF65vv zum_6`%PD^g=*t719|;JKU9B;#XXV*aljuk*o}i`4+H)o^HfYa#_+=l9)|EVG`lX#Y z9zCjV(Qq|95tt+tP5~{XCkV8(2wtc#ER|sgJi&u6fA}HTq{FnGKe%!bx84PK$#THN z^f%$g$s%h|yYm8+1GA7#4oQZ3=>UQdO#%a@OHV9~)9jN=)VcvdT!wnnKSQ&uYd$dK zz{z)G%dC%u2a>Cl^bg(N-@VH4lj+PS%o{-9QUuq|OsRn)_rE*sX=)z`D6xpy7<(Fv z{QW1uPm&`)h$BEu&+I9L*Q^5JOL>64agLC(v!(IX`*ogc)58{G6hQ+CTGbaoYtLw* z42kAO*C%M-&!NH>>?ltj;xPXR|5au_xUw=d=n*!<;(%A41boc=jBA;aYDt(j2#En1 z41A4vaRZu#1I!_p0cmiTX^CkOyv;ybeJ9St8&K3toj_$4Y&O4p zZ*%`;YIg&$4?KqE_o!)LCw~OF+94bfsn0_5ib&!;+i5tiyL+d8{o`B|^qPY63)*kr zG^#JQn!yVmSrr&_GKj3lJH^>ghjds-T&xdZl_q;mDJuaCG3Iw<{~01p2*ij2UMy#k zV!$=tdJu-!4rzTBRi*KrGJ{hfZoHJY142Eh9byfReHpIL)2SIoN=dGl&R*9^vB1T> z@xtZ4v&Vlob;i$BL8NAMAmW2j*Zr(w3jv_xE};ew$K@3(bPYni3=OF72m3`& zD0m6KCZWqt0Zag+MceVXRe9qMaqUcaQ5Lh*qb$-q$`_Y+uMI>j)6eqs`4i8)0bhMY zWY}5jN?AAkEZ8V(L?otbKJ}$c<-`_$ooSW1{&%tFL6-=X+%8E)tT(gfAOyL)Pk)_c zzL;w3D7gqKj$IqFI8sphvWYqt{n#f zy1D)h#ToJvq|qQ9yuk7Be}P(8KwxaQ{~fL^!2o_+T$qe0pd2~;dziE|1f&Lz9q&Z8 zS;#{5-=Nl_5M7iV;|@6CGDrVC#sALu zPWf*h?|+Lt=L$Y@V7^dlqYwBWY;7E9CCD(eSN4Mk`+q@=yFs|}mj4}jZpjXAS)x=9 zX#f*m`R@_mwh$)#|1A#a9t0uk{@>#Gul)D_-z|>0Sff42WL?Z47P7yhxqs%8z&ZUI zCj6X7TWqxW#p)Su|*FvE( zTZz8*w=sTa^cwKRmwmtdwUIG{z`VH`lK8~?;^o9gT3l@J0HT5f!FVAG=0K9j%aTeH z2$gs$X=W3En~Y6mD)hG6H218wt@7eRDlh!!SrQEOH=%HwJR`V�t<2^sk<-;PACF zgEfnl>EewZ8CfhNHse-QyjZjK>%1%!ik%ZKja_!KcuVe0|M_WKI%CX@yrSV>wm#pY z{F(G}{Zf9AlwHZa76@@y9M=Hk8huc`2bDjo06@{Ye^Sp{;P2fH+&W-qZjQeD^SyW9 z=F&Fu{ojfdI=jC>wOmjFuh%mpxW0=Xo|8MZJ{#t1Yi^{s_NZ@%Z6%@!wj#fvM|SNcmU3Ff^51F1gE#RWbf%Xp(CW~pdWm&k!an39@Vi|yI&mPxSHp} z*opeb>B-W0{>oFH?1jN|8OPZ&ZEIE~J}>^3%w4~)B&$NuwY`w6Z#-s-K#s)6EPVK^8Jf=R&YXw^r{1MviSHj!R#b_oE;fb}Di z-BDxhb_jKzx%R6ASi&#=49~m)F+WoBU@!7qlDGvv3&L|am9#0UT(a|-iozuF@J~_gX+_JC|27wu?kPy@aYQEMkRCem@fg;+Z zTXX;3*7o;(JOXf=RsmPAbr-iujX2IAR}^iw0qYSHFJWyCX&0$flkPCFBGEyLVd)4d1u>J3j^97uYs^P(U9nZ6D}j zc0p^SIzt%SpZmy|ANlAhaTMI9)6R2>suENv-< zRhG*5Qa&by(nCr)_xI34fq;<&r&kPw^jHw&HviPyQ3a@9G}oica*X;9cpvBt`nSO& zI9`dm9(8P4pZiKm8t{r&flpHaY*SK8u=`-78ovWk{vEH?jM87E4&PacuyH^q*8I`d z_tX%yXT#F%=1l+7i8%P;-vGp(d?@Lz2KK67D=`*hOuYdL;+urG;2Gf0FuzBE{zM<> zD`hBwVvwd+6~ylJ0YxAe8Y4zK@4q{J?nA8cgf5FZr+(GuM>@f%!P7~aAlJ92O!P&& zr)R+er{~n!Lx2w$`bg$@r>7CRvH+35q%^JVgNttC6n+Z|t^xWLAmn*`Ez>h!$d8Y{ zaY6`ackVPpY;(-DxyUd|P^5eM6(s^z7>Dts_W1_H7rh2IkM1JpmH-bp8}f8FuvjdB z;wS@v2YHKyuaBo~>Wi!viyg81V=0t`Gwo{<>>b1yoZxBJU(lq63y1ovC*kevj=4UyJy$P?|u z&Tzo9QsT7*$e7Kx3z$9hi!o=A#z^9k( zkPuIr%152vq<$DM;W|F`0|Md6*j*)#fY+aYV|H3VPGanx8}NZMz!xh^AWF!yC+io* z6!bzy78iBI$_iAxF1url*U=E0J27+&=xAWnVwkIqK)T5f$GTVdZvzHJC%#BVi+ZVCK zzFC1>)vmvTn`3ga@hnEL^BMVby#+N>|8QT@o~WJd9!?~g$# zF$-<&)%knYW_jW5ucp30qb zmbH`FsAH2jOtot znc}t#^~0Cr80mTFzKo}mt8CLk1Oof5DoR#>dOB7C$r*>BkKvE*(o$N!`TQ=xFEHaW z2b&TB1I9aZ5Q1)oC<8~Z$n-ex;!Mi1W-~8`#4RBH8ZbJJOsmVJVeSHwOCRA2Uo*pY z7CbcLW917<4G7D$+6c`~^dQ*QB&ZFy>Ofn~fF~+B z@*%S_8I$QlvTC;2Jn6sC|5zST$*W8Py`PtOpgX`T)8chy(@-m8MyYHw-T9Qu?&)h= z88bYE+^Y*kG9T>tTX4SrN-p>pzv@PPRjQnkEW!r%8`k{L&Z0e-9vx9Qj*eWoO_B(g zv!2(fwI7R9>pV@g;qe>gKs}&E7(~#U3nZt>C=eQ)f{zoU$-r;;tEd3a=s*GZ!!|@2 z2ny3z*1HLMK1X#7vjH;VthY9u&x#3K+6$811$O&+lN@DlLxd?aBQCo+%QpVX$lVTW zr+F>>1MX<1i5hnX`QQua)ab25Hi7`{&ajB&Hz>x9V6IOipwm@pPk!}bZ!2>VC$6hP z)D?(`$6rtyh7xoydnsI;yZGfZhe8WZdY0w z%{#vDxHd|wI%{`FSD2Q5@OO`ZyDv}98sE)UDnD;!q1o*)#hlWutgaPZ03!T*SqXr8 z1JPCY8P{m(iyAs(+Kwi{kaQLz$k`|k^FlTrfpnMNf%E}9tKomlqc`6~&ws zIj(e`yMmZ6c?xc+Gw59Y1TATBusah&%Xh4;rpP8 zOWCg|)QfY!x-i^%*!lTA2TB|RwR1kO5yYsY`F=I-sjYMBT(?&sk~*MB06R9iNpdh= z5!TOrW=x|UrtN-@=Oy7!3zNen=g_st5gitvB6W)K8P~_|X%21{g~%A#r`2@z7g?V> ze9n^`vtdz*<fg;jrZpVFjn%ztkBA>k3mSPr zEGicEY@VHQ5srcT@u|=-c-#_sO!49*qycS@_vC3j(CcZe8|b z$7_I0UG1&ao91e&AC=qN`;4~A>0BmjYCFQ3*F6qo zr~h2im|P12!;eUUcG`K^MJm$akG&w0xN;MqI?dMfRX~hbrXen<-xVzpETgQ<>32k1 zG^c@EF})sRERC7IXqY`xytCK5RcR<5(xWaqV2k~<_*|omRofu= zc*Pu`k0-D~z+0rws4vW*jO-Q=@6&#@l&Ed; z=lQ4D^xBWG)2!yy3R^nvf_Z7JCTYp0%U>$)amw+LPrQp$P~GD3DO31Og)ArGr!s`&XV&yq; zt(?tLzJSHJm`#u_BA^HRLIU@x?>i+isiLWs{9oFg3xF7eTuzY{Ep^G&=o`spLApNw zD9p(T%!f?1s#88&ARROyd(RNresxEt&&U2jwmxsKK%NYle9+pCNtn*N6HuC$R(t>H zZtlsffXpVF1~Yvv#~u>(jai5nAVn;(NkYw%0+US@OU=gef;R2~bp;(3#86@adQHgu z&VThbc6pDZ`HaGbjPUe}H_bi{?Xmq= zUQ8m{xGK99NtWrA=O%p+o3wSU^RMzP$qVcuIwN7HMU$Au$CZ5qfh0|_Q8XWr9C(;` z&%RR)-h@a2DgHe{tNsBk%9`g;$|@S~@`yz=E|Wa}SU&n!|8kucY{*XFe0c!I@?4Rf zb=6^5#1iYRo(d}pl{C^wCBQsP1ETF!xP&&~?8(LY_p1l20Hp4AqPc1iPDXn~HEe>0VNSbK$!dmqV@sQ#oQ!!`6K+XmZX9%jNo<=dAy3|o=*Bx$c{7;R%F-)z-PR*5|7vOC`lPeTZO?`s{JpczlPBy>V%+Pkf`n10+gtd3oM|nB}xJXa{E;f1EOlF$^ zKfUHbm!U0Lw>1hwrCTR3-YAMBo<2dCJb_(+z$aRJIl_tk+fQ9zLbfynA?m?aioF*e zmU#&lScSMgzMJZW&UKj1xx?hfDZgJYH5DWJ7J$t1Pz}`GXgncNX^*HBz-zWJ*6(FJ zXOtczq94^(laf6qB$A10;9qeuyQM2kcFx?3gheGPE6T{kD! zw3F_+@XD?dxRoibaj$TIX7?><;wUIcfV+*|+S?%N9EXJQ#cdbsam8`7nOsE~XI&c~ zEv-44a!idVs>Zqig-j}|`bkGm@iuN+FDDzNdwy#;D4~{S$7D+9f=Sq{>?GZpjJ_`1 zcvW9gq#K#QsyMcqQCvn%ox{dJIzGK~$on2?H#W5>uF`^3(*~V%+0syP(314(gQQ!CrX@N=dx;*VjGF$^r`!)V(P~;Y za@-v<-^f$mDPAecs&szKY>Lji6)2Ds7^wcTXwPXj<`SALmWk@Ra`~`9lA2g#yW8(H zKG?igM+8uQsPFWc-Y*16L)a@eZlGwd0<|fSS>wgmL`c|Mzk`TCb!nrCtIlOUeTFGj zt0sf8b?X8{c#B3YK(aHP+1K&Ndc8DWMiOUVwz)=SYTL-ozHq7fh(Y26_q>vEe^xI4 z;h6t5|AUE!SF-a2;zwV%Q!17KdbV|OKW=3g8sAQmuKt`i+LRModbp7Z<rhGBWxjWUp!kOo#LiYQ+gNcwSS{by6c=WbI=NS>)^`LKImwEbgetr-+4cpF7Lfn1S z?7(_LeJDa@_tmLG9*E%ilEbkgm76q8vQD6Vh${ea=4}ItkjlPJ3&-hst}88_(dXbh zT!&wXe>}U=wIibPJ+c!(zZ7~lDk69Gi{Z)-zi}>VXGGJn)145eA&qbm@CdSdy8&5voxs5FK5sv}aoKva z(z`&M@oNxC5#B`G^!R@YM~f0Cvew2AIGXLH@2nm!bCU+ISkzJUxp;9G+q3qa-AnykYS zAf)$a2|C$?*S^}TXeA=&q+B9<8AfRuLVtNgxEH}F6|{6N0*I>Yhk337LRde|e=M`V z^;Z)4k;~rppj;pXDB%~g(p2-EI{7x(f6&E7w)Klo6?YtTvl!^Ff2X-`HkI~@@_1^? zRb;lce$p62HBtqe31=n;s3m7!0Ht^5t|kR~o$0tPpOz^#GdueeFezoA^9!7$AYlIb zJsLza)wcwEE097vyH*K)6Sv*(dt-9}*noF@PwQ5?U;lBsL2JC`Gaw#vkoJpbjf?IF z;dLXfJXr}kRx@SA_w5P&c_7&F<4s}WS0FaALc%wPMc#Z3jAC4^O1xA+uP$47th@ZP z@|xFepWPfXYwSoZ9e$_F7nkb*sNXWK*8R~3;aV1)A=4MmYqj{ikh@Iy8gX=s z00p_*51Oh6Dq>;0n*crySLi-%G5^g`HOZhSIC-wpQL2>#8#5<}RC ze^Bam<)0e0;JGauKYw5zsqg@-FNa<2%Xq$`l(7Diws}gW{GK#DiQ$DDnwdzWL;dX4 z50=3%f#Ur?RMV1+6sp7^u~k8<;@yzCSHip*siAvT!5-958lj z;nD|IeIDti@e22wvwMxNd5PY&KYU-e%=rgLTh7|Q*&p;!VQI6RC%*_=Ve)&^`3*1= z?+>3Xh5{%2tt&laUCvK7Pc(EvEQ|n5j#3;UIT-@f66Wab9j^+ejQ6dDX{-F28sb|8 znfqK1*fh$4&MVxj8hSEx7Tu$gK879ifrs~4?f04SdF}K%b;Ld^x0dN+C?Z>bL8XFT ztnUNw?AAPw>Tm(j7`X`Z`Uk~rM(k91zbR=N*XWi4F0IZi*AoW$<_ z{@J^`{oAkC5-d?PC)7>g1*kCYsY-6$nC`%>vId(y_mK z(Q9!zVd_2=u=@}th00uQo+ZF;R?dOa6-IJl+8I|*!Q(X21-ZJ9<0o!{#PN?XNY|(p zcS|IX;tXSN?H0|=s-tl|F`$4IFqlD*6R~BvK%Fp%iP2_4gFW$(7npdub3`ue@?sRM z?bqB_J{_L-f}S#iAAR9i(|q=(0XcIAa0)ozJK{ll!~>QSW!BFwy}*p3zj)=S5pR!O zNY_>OSLh|e?A<{1Xu?n2?~JGNhto7&tMRoZz{~tz-=%*Yp}c*sv%&+tWgzT>U_SNp zO{EchTFwe9@PX<3lhw)gy$0zhAHW-V3HDcOurL92ZKc7YPZh;G=BMq&m%$&Iyuxg| z^wB8>sV+ol{CJLv+Shul5xPdG7P~tIlX>9%cV|6v1@KPJk8DT=z*$As_z&yTEp<{r$F=oQ92Yq#w%lkN5p=Vl-?!v%TR zlf$ss2Kah!?p{u7&qamZE}!+wBLCb7M{gD=ByF(MOOuc=tsar(pVO1=zDn?`1o%;nE6t>Qh&ikNWD(ozqjsWDv>EI$U->p!&hhhT zK>i$635a$?D9&vl^4i>0-?3r6Wse)cVtCxOuRL{+S-)`ZP^ps&*s!P4?6Vq^AHZI4 z=S`*SrG|=$ShlSJ%2}Prxf;zcn2O$5XLom3^S2x9N@NHod#p+{zI87KYnA1RvfhH4 znhm8`?=U<0)^yw)yFag!+dT2wbN1@>9N1v5aVE|(x~s0zVa9tNUnev({bCGK9w48* z{-{&^EOublH6(k}_i(V6Bt`9dFE%GyR1t~P%LW`5g$geG zI-WOHdqj%H_S6vq5kDQJ9#1{Wn89JYoC_B-%a_{$A6sVeYOE zpU8DYu{6PXCKrZhyP2=E%XqH@mKejZn5I4u3EkT2=&%ew-`E^WMj#% zTqHIubT)ZQ9Wg~Q#T(2R!7rTOnNv-r^Ya0hVK}ZWwgS*woZJ-_{CJC86T*RXk0U1l zU=~|Plv#g%SO%J3rySsJh9#dO#c=bE;*=*MF^1Zzz;Ts4#vh-(lrqpyd9DU-nUF+R zS|D>pNFAMnXKTN`_Ob9DZNQd!#S$@!t~<*G1a*(`3ZA(Mx;d4w36IUGgx`H8@QNxj zaY85{cua}e5$iP+G~M^`T#ZvWNNsq7wm1mf$53wY`@5g!Sb&0hwa8OC5s+136Kbtu zW6RW1;2Vfx(r>K6*ATPoCRNI30w#rTJhzo*MC?~qMaNYFK~izqR}nq0 zobuPz#ReoIdc;NWa=OScug%q)IDKTFuL<&*${{ap!@?$%h(pvyKH+D>gY24>H7=dE zzXdK~-1t;z0~?xXRayy{Y?=2N6F}L|2nPh#&9V0`2Ej#Ilb3EwN?mzc0Ew*bVj7m3 zVJbp<#hG)`hk6pez(oc|!uLqIJFGcr(O4i&O7AR2j!&!I zW`3{|{X1|!k^8(mm>{>^(i407^L6zultiwRDN~V^i_5fbT5Y)f1v3^!KiN)|gi!o^ zqta0hS@bMmYpVs`)=-6?j>bnxqw$&Aai8{b0JP}=-HY>pz3ZN)Y=@HL3AfZDQk@>z zv!?IHH^l$3KHvJ}!vsup@@{r5)#_IdVOmT*r6Lw5fxk#zbBnJueoqq*z@iO50*+4k|nNV z6$iTYjxDH7M9EhLGEGt@+~W+hy3GxKbA+KAX9C%V0<~P*9toKj-m)w|36eFx@(Ghu zQ|{vHB!4>nfPr-6sprB73EM}}t)Y9~G<_nB)r+<=9=79KW~Jd_Qfiw9uDeTP=Oz)z=5iO?P-IH~b0<@?*{(BjY@ zJ3XG_P3j#*E`$nmRU9w z6=@SFhgg6OB*?=)>{&PnmA>B?E38a>2GvK1J8>-WcS^(3*aQ-LI7uVXo(UPPRw6T| z%JEmJ$;CW8F&*yabV*9wm>zKa_4_%UrtqO9{*1G8nCooA z0ghrVvgHh5f$Su4-qdsgSMr}&yKRedlp9APSa=)rq6BL>byT(8oHc6+8(gJ3+b@0P z1x8Z&T;AVJjP~buu<+8h-M+*0Cn1_3ErvT7zJGAQ6La%^TlxjV^+GMf>vHt@<5cEh z@{}j*yGF~s?$gZ`^38-VxHv*s>jST(4~)?XaJrGZT9VdV%)bxhM0wqxxHgQN3IsG% zCYU2llM(4QlUiBDvIiQzSpp6F=c`5J(zkY%c5E%(@s+P27!~U56~}FP_uO%J=$r6u zFIiFZVK*C{wHVbR>o0X4h{nd|;Y6ppQTUe@%pJ6b_wM~T2%J18Xdm403YiSYa$Z@QnWaOT&(nw8R9+Yi7NpbgX!-n9b<@U8=Cn=OWY5;s2r z6N1?+a>H&MgwP+K8A*6)fk5(96Vp?!f@PMUL$WU~TM3ch@q1+}i?yS6Vjb8^&-cWQm-yr3nlml*^)?-p6^xtSmt_dt(FQ%YW^lqm;F0)xGUY7-jT zIIysKg0qz+a&@~%f7!V15*|}sqcGdW;*;!Z4P#aqz$BA6?=3g!`0$uJ($7uZjY$yZ zw*aS7Ya{B!S=Jx9xA1Kb^vecnN05PR-lSjUGYv!*=omqp8Y$K7$23oXvBsPyME4+IQ~CbIGk-;CyrC>4y007c>n+&v~Sn zN<++cBqs4YSwsqRyH;iCsq6iDXIXkm$6&)Z8KJFF(}$DH7Cv95948fFBR?59Gu_ZW zSKc>15#MsDXaR-xXi0IR>n-Vh__j+(ms;*#{^hhf(I`GE^vV2q{iNcQz)yOiH*RXQgg`Lo$~`&>IX5gC+&%=(nGN3bS2wN!9R%qzJ{ zjOau#6y!@iR^H;xdL+C;#avfL4g|ZUPl&N*RN78=b@Bs z>%Gohv!rw7z4$RQ@53jnnY%^VCvz~|PEkAe&)voFkde9t8nCA^W>lbLVVp8JlE@KO z=bV{Sk&lzMJlq9YGauGGAADi#NE9c2VAJ-HxRkYg;8?~fVsT?+PX}SQU6driD>7tN z&tDt4$&rIoGT;Lxe8l9ty`nl^lZx|npLy8TpJ`P0nW;&THG&Jouw?3isOb)fy&u?R zG$(L*zvsDhY*KOT`pfp*CoDlA3t#?xrWpuU$ZO{gM=p!2bt+--RbjuW&fD)Ib~^o;6cp!XSD*tKQ$oZwK-~^UGe

?toH5QQinW>_%x36e#^=3 zV)l3VAK^sVCmW@IJvIjL*pR0!m8CnbL&m|>A^+)lQvc=9)~k6?TTd~g2;`$(&3*-M z3~u(ERS!669<{j-aZ!KUdXSrVrB++S$qhiV_-xZUoCo%*jSvvW7~-(ni#zjlT!N5a zjUZZ?w{w<5EQh0A>E}HoC3xYtDR#f2Z5JGcnZi+l4 zR$;>~SQnuqQgd;4>EQM@oAs{Yg}0zcW-nLlgBb~jDjoxJjNNYhSyM<~?BoneM9oQ@3r}JNcxhYi^olRb(<0k~Z0N*NWe|RN#_P+6B~sZr|X1 z^~``(i#v3%wg!j6HBObC+D2^WK!nPS2sU6?Zt}#w_ZLpXtq%&7DHWC38;qe*yxt%1 zV-PbfK~1F;0i$kw(1GbZX^C-`>_H2>dmEmYEUKZu$?Z6jUf$)ET z+M$|UJ67x?cIP9YXc%{XOuzK_kZcBLQc(6B&+~|Tq1P3hY}d?q+g;r%Zl$aG0%A}0 zt^@fn!6Ib1q~!)7GP!Q%g)+sk@o;Kh`*f(faH~(s9al;nb65WiV)Pco+FRhwrWL!? zbZ=c^<8VfjL;;iFRTzsTwFffO)hl-^koA?fgu)QmwnJv6PT7Z=pvi}gJurUyx!)Zs z=*+q#Ym2gl{&azgRJkXFrCMWVG$ zawy+53o9}cmg9$k!3SXzi2f_7=H13{;mr?mN$KUZ{NXocYnSI*F)^#K>6&qS5U0o z0Sp%~)wIhe{(OBk=n*q9%TR7k*P8s1O;Z#xa`8MxLMs6sTyaOZ|$DFS=UC8yC>=(D57CX?4Q-7stcC2$}FE<5roF*MtrW*xlUoT zpvytz4@K+2JQD{5BKcsL@?tUYnnY7$Ko>s4@?_VJ(T$NYPkrky=HnOJZlecN%Dx0v zzSU3!KWP;ziT*t%1mC{*lyJpYv~JWRFiKLG|H?PdFoE7cc|X{D$tSI`FZi&*IT{g8 z688q=X3~0#l)ZB6mXm6Ihw`IbO*udN*kE6$O#BR3@pMgbD&OsIlJmkA25J^=_s&-mGcV|L>WVw<=AMipJGLm6&uRC?vL2FWJ-k0ILph7rv##=QBeos=(bt?0kj9IvOnBnry1C#fDsOuTI)_2qb$VkcmKt|rc%{v&% z&tcCid7fxPV>_QKq2dhVk3}_?u-;p-3piaRY@jE#g;jWrgWw?%gI>;&3eNBW*GuIk z(7Ybrh);FSQ@5kNyM|-kkTBoGvf2@yz{Afo|^ad8%l`YFeVvnjmO>}8mTdwS`vmEb|sPu!{zpbPGq>` z+d$~8l0gE`=!6@mCUOeC8>Y2NjB!+o&s7g_Vde=JWsD8I!X@ArW)e{v zr?D}Wc3C@CPUs-v@|*H3>m_c(qC*^ODta}8dh8%mDs5->eXw@z$&I@1=z4PD=zxa1 zi|k13;_xX4kd6b@ow=k?WT?FKb$G0in^#-!zD=TKDpPaTuQ?qwFF0fByQ)4>c`KdW zZkSie4KgX^HC~9brt9(?l&unz=W6`a3;A*g>|G^M4lt)=I$z~LJ&BO$ z?65JB568Y=FO>78?$Zs4P&rQy!BKbGyfxK(6p{jl895_W@?+|_TGr)5;=GW=t{*6l zXMJyS)ixXyv!V3r;wPf+M6!XO;#}mw8EYg}CCJp~5+DV_;qWPJJJL1km@Y$cpj7mC zqbu90cPa8x&?$%LT=~+dH&>>RlpQ1fwTHo2%@gaBOx>-?&y+XGGF;78IX%6~4w!5G z5K83xhZ^~Uu;Kk(%ubFHGL#2=55b^($KIv7&@4-i3?A3P4%-;H8~UGSelbbb+ZBu))7!ftIP#ir|aiDb4r zC;ggA<-!`n@o%tYD<%D*bdxTETn{5cB}h#Ek#waI228_nmpicR!;Vqi+GNeENig?1 zHwAOSQgafkr4jp7;K~&rB-VST%ld-+wN2B>bqsztTF==mr!mM$Q){8V0P$8hQ7v!h zFNkrP24aMH#n-BX%wx1s!OQ;$u~W!eItEV_Ocq%!i0>3(B$VGRJT5p}WW{cyD9flP zT`xa2d;MZ>8l}|?xvr)KdD{4bUq3OhvC9T_E$;W#>*rTq-1)%3IUyfshcP6se>3D8 zVnKPgZHI?lo&$p@7T4*nyAAV;bbj}(mbG+jh!g{7eCwZGc?Q1MRsB)QXINaF$oMMq zVk9aunSWxlHGx4i#g5oayj$!b0_QU@m1|5Dc>Ye|2g}94e7hUj<4Ry>3_d zrI`%|zMj;aM%MQFwBGLbj4LjRkCoSOI;l*66ckk?=dsW*KC@>hf5HB1 zaruFP$KlLF39biQ_1uU61e+D@}^dS!^S+bLjXJ zw%C^ls%FHpET4h;qAEGKwMc`9cT#+IslZ_(!2WR@8{WXg5y#_VEN2YyxGef1Xm~Ak zI@_}dItI7xvNht%eE7JcOl}~t8vKc_ky(I(j_+Q|~7m(@o6Vaev1sxQ^#3%Wk zT!rpq)aPMPk>1<-$!4qYQlo&3^=N#(=H&=Qv|%c4JEPBKD@l6f?Pl6^Vq*!~1WpyK za^rNEe#;{3@L_(}B()|_z|F{lV)8b)i3vMixm{e_z%vC!xE09&I(- z4|%b%zXP+|eYSjvBWp@%`=sXjBE9TpV#ifzQqf#Mb+Ow9T#`DfGT1Pb*vrvdbco8% zJfCj4-3yr$om6rVj&CO-Gqq30;{P%OxoE6GM!a(0Sxq}>@ENu^1E08MjAzWUc7Y_> zrGK)Il-;DvieIXhyOpS983B00^38)fbfHM}Rk7jS?0J?f`4e8M z06@Z{>Xvd8-@3yL@P&@&>;E0n6SN^c-lP1XPZRi~zP3GO+Ja&ZrG3pip=@|$lK{Yi zOaRc9w3C3RA{~2J=1SY#+dTy=^~@FY^brk-g%%M#qxyy4R1)|jvo8%_(}C05yAgii z>y7F!*rU?`18^w}Mv8sMLwgANe=Pqp(O7}IA&zZ~{$0wB(U!7fs0%eH8f6q!g&v~e zJTI7NUq|@FkI`RC%bDh>~=VEGcD?sZx7m*NeK&)-n;0c|L5xmtP&T*mb6^}R`Q~x$>zWw-<-v*Gp zt4O14b-s$=gXS8aSv&i@qG;~u%_heucCbK&t9kWQpn9zTtJ&xOHBV`+sk~{N(6WOa zv+(8t^5_K8m4N!&Y`KY-z%ri6%U7$;4VOdzWw&)bQ*CFTyir6e5JUiyZM(is0J@(6 zYkpf9R9~Q`($N!!)3`#N(yolL?{XRZZAd_F9WQS-z=-lySW9yVr|B9Pbc^KI?xe6j z-*B0mY@U2q;PQ^$1wA1hPe;p6?Zm?QAg%ySX~h#r1^7Mb=bW1w9SnI*qk_mTPKZ6w+-mCcLF7p6!`!QZb985Ye$7E%;sd*{lOtdTgQZ6E9!3%aIju@<(9aB>Nc+`RW;DoOq(_^itj$DDj zqvzmHF!I^wZ|QAC8I0e*Fqi;WV!t#uNnL9w$?kpdvm?Vqp5Svi8UKdM^508ChcZX zu(#bdAcpctW=UUo8*jm-8l}KQnmwJS3xZ4TDK~{z0>vt6j_N<$(7%KKpC59D&2qH5cr>+ zgE}y~y1Z{bmINN}XoCO8$F<|Y*rXmK^5y=0d$bVge>V|(2ADV8qvP96M_v1$zx}`d k_WyB1S`+_&9sL%P_Unly2KsXI81PR=69KDKw|n}30KLa)p8x;= literal 0 HcmV?d00001 diff --git "a/docs/realTime/other/SqlserverCDC\345\216\237\347\220\206.md" "b/docs/realTime/other/SqlserverCDC\345\216\237\347\220\206.md" index acdf78973a..acf1e21283 100644 --- "a/docs/realTime/other/SqlserverCDC\345\216\237\347\220\206.md" +++ "b/docs/realTime/other/SqlserverCDC\345\216\237\347\220\206.md" @@ -11,6 +11,8 @@ - [4、采集原理](#4采集原理) - [1、insert/delete](#1insertdelete) - [2、update](#2update) + - [3、流程图](#3流程图) + - [4、数据格式](#4数据格式) # 一、基础 @@ -252,4 +254,24 @@ UPDATE [dbo].[kudu] SET [user_id] = '3', [name] = 'c' WHERE [id] = 2;
+#### 3、流程图 + + +

+
对于FlinkX SqlServer CDC实时采集插件,其基本原理便是以轮询的方式,循环调用fn_cdc_get_all_changes_函数,获取上次结束时的lsn与当前数据库最大lsn值之间的数据。对于insert/delete类型的数据获取并解析一行,对于update类型获取并解析两行。解析完成后把数据传递到下游并记录当前解析到的数据的lsn,为下次轮询做准备。 + +#### 4、数据格式 +```json +{ + "type":"update", + "schema":"dbo", + "table":"tb1", + "lsn":"00000032:00002038:0005", + "ts": 6760525407742726144, + "before_id":1, + "after_id":2 +} +``` From 866dbd5dbed82ef83db46e764472d8355b53ce6c Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Thu, 25 Feb 2021 16:44:38 +0800 Subject: [PATCH 107/136] =?UTF-8?q?[fix-35459][hivewriter]hive=E6=94=AF?= =?UTF-8?q?=E6=8C=81=E6=8E=A5=E6=94=B6=E5=88=B0=E6=95=B0=E6=8D=AE=E5=85=A8?= =?UTF-8?q?=E9=83=A8=E5=86=99=E5=85=A5message=E5=AD=97=E6=AE=B5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/hive/writer/HiveOutputFormat.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java index 1a0ea6f90b..1d81725788 100644 --- a/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java +++ b/flinkx-hive/flinkx-hive-writer/src/main/java/com/dtstack/flinkx/hive/writer/HiveOutputFormat.java @@ -30,6 +30,7 @@ import com.dtstack.flinkx.restore.FormatState; import com.dtstack.flinkx.util.ExceptionUtil; import com.dtstack.flinkx.util.GsonUtil; +import com.google.gson.Gson; import com.google.gson.JsonSyntaxException; import org.apache.commons.collections.MapUtils; import org.apache.commons.math3.util.Pair; @@ -109,6 +110,8 @@ public class HiveOutputFormat extends BaseRichOutputFormat { private Map formatStateMap = new HashMap<>(); + private transient Gson gson; + @Override public void configure(org.apache.flink.configuration.Configuration parameters) { this.parameters = parameters; @@ -116,6 +119,7 @@ public void configure(org.apache.flink.configuration.Configuration parameters) { partitionFormat = TimePartitionFormat.getInstance(partitionType); tableCache = new HashMap<>(16); outputFormats = new HashMap<>(16); + gson = GsonUtil.setTypeAdapter(new Gson()); } @Override @@ -248,6 +252,10 @@ public void closeInternal() throws IOException { } private Row setChannelInformation(Map event, Object channel, List columns) { + //如果写入字段只有一个且名称为message 同时 kafka原始数据里没有message字段 默认将kafka原始数据全部写入message里 + if(columns.size() == 1 && "message".equals(columns.get(0)) && !event.containsKey("message")){ + return Row.of(gson.toJson(event),channel); + } Row rowData = new Row(columns.size() + 1); //防止kafka column和 hive column大小写不一致,获取不到值 ,全部转为小写进行获取 HashMap newEvent = new HashMap<>(event.size() * 2); From 095b57d0264adc719a790e7fc2e5209da1aaa099 Mon Sep 17 00:00:00 2001 From: zaiyue yu Date: Mon, 15 Mar 2021 18:00:50 +0800 Subject: [PATCH 108/136] =?UTF-8?q?[docs][flinkx=20docs]=E4=BF=AE=E6=94=B9?= =?UTF-8?q?=E8=B4=A1=E7=8C=AE=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/contribution.md | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/contribution.md b/docs/contribution.md index af06261f8a..063c3cafb7 100644 --- a/docs/contribution.md +++ b/docs/contribution.md @@ -10,6 +10,24 @@ 1. 如何合理且正确地使用框架; 1. 配置文件的规范; + +## PR规范 +1. 建立issue,描述相关问题信息 +1. 基于对应的release分支拉取开发分支 +1. commit 信息:[type-issueid] [module] msg + 1. type 类别 + 1. feat:表示是一个新功能(feature) + 1. hotfix:hotfix,修补bug + 1. docs:改动、增加文档 + 1. opt:修改代码风格及opt imports这些,不改动原有执行的代码 + 1. test:增加测试 +
+eg: + [hotfix-31280][core] 修复bigdecimal转decimal运行失败问题 + [feat-31372][rdb] RDB结果表Upsert模式支持选择更新策略 +1. 多次提交使用rebase 合并成一个。 +1. pr 名称:[flinkx-issueid][module名称] 标题 +
## 开发环境 @@ -32,8 +50,7 @@ ## 任务执行模式 -- 单机模式:对应Flink集群的单机模式 -- standalone模式:对应Flink集群的分布式模式 +- 单机模式:对应Flink集群的单机模式- standalone模式:对应Flink集群的分布式模式 - yarn模式:对应Flink集群的yarn模式 - yarnPer模式: 对应Flink集群的Per-job模式 @@ -510,3 +527,4 @@ mvn clean package -DskipTests -Prelease -DscriptType=sh ``` 打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,syncplugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! + From fd06305c51b3659945cb8122be2e97ec1e2055b2 Mon Sep 17 00:00:00 2001 From: zaiyue yu Date: Mon, 15 Mar 2021 19:14:43 +0800 Subject: [PATCH 109/136] =?UTF-8?q?[docs][flinkx=20docs]=E4=BF=AE=E6=94=B9?= =?UTF-8?q?=E8=B4=A1=E7=8C=AE=E6=96=87=E6=A1=A3=E6=A0=BC=E5=BC=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/contribution.md | 78 +++++++++++++++++++++++--------------------- 1 file changed, 41 insertions(+), 37 deletions(-) diff --git a/docs/contribution.md b/docs/contribution.md index 063c3cafb7..43ac77347a 100644 --- a/docs/contribution.md +++ b/docs/contribution.md @@ -15,19 +15,19 @@ 1. 建立issue,描述相关问题信息 1. 基于对应的release分支拉取开发分支 1. commit 信息:[type-issueid] [module] msg - 1. type 类别 - 1. feat:表示是一个新功能(feature) - 1. hotfix:hotfix,修补bug - 1. docs:改动、增加文档 - 1. opt:修改代码风格及opt imports这些,不改动原有执行的代码 - 1. test:增加测试 -
-eg: - [hotfix-31280][core] 修复bigdecimal转decimal运行失败问题 - [feat-31372][rdb] RDB结果表Upsert模式支持选择更新策略 + 1. type 类别 + 1. feat:表示是一个新功能(feature) + 1. hotfix:hotfix,修补bug + 1. docs:改动、增加文档 + 1. opt:修改代码风格及opt imports这些,不改动原有执行的代码 + 1. test:增加测试 1. 多次提交使用rebase 合并成一个。 1. pr 名称:[flinkx-issueid][module名称] 标题 +eg: +- [hotfix-31280][core] 修复bigdecimal转decimal运行失败问题 +- [feat-31372][rdb] RDB结果表Upsert模式支持选择更新策略 + ## 开发环境 @@ -50,7 +50,8 @@ eg: ## 任务执行模式 -- 单机模式:对应Flink集群的单机模式- standalone模式:对应Flink集群的分布式模式 +- 单机模式:对应Flink集群的单机模式 +- standalone模式:对应Flink集群的分布式模式 - yarn模式:对应Flink集群的yarn模式 - yarnPer模式: 对应Flink集群的Per-job模式 @@ -228,43 +229,43 @@ public class SomeInputFormat extends BaseRichInputFormat { - 调用位置:configure方法会在JobManager里构建执行计划的时候和在TaskManager里初始化并发实例后各调用一次; - 作用:用于配置task的实例; - 注意事项:不要在这个方法里写耗时的逻辑,比如获取连接,运行sql等,否则可能会导致akka超 - + #### createInputSplits - 调用位置:在构建执行计划时调用; - 作用:调用子类的逻辑生成数据分片; - 注意事项:分片的数量和并发数没有严格对应关系,不要在这个方法里做耗时的操作,否则会导致akka超时异常; - + #### getInputSplitAssigner - 调用位置:创建分片后调用; - 作用:获取分片分配器,同步插件里使用的是DefaultInputSplitAssigner,按顺序返回分配给各个并发实例; - 注意事项:无; - + #### openInternal - 调用位置:开始读取分片时调用; - 作用:用于打开需要读取的数据源,并做一些初始化; - 注意事项:这个方法必须是可以重复调用的,因为同一个并发实例可能会处理多个分片; - + #### reachEnd和nextRecordInternal - 调用位置:任务运行时,读取每条数据时调用; - 作用:返回结束标识和下一条记录; - 注意事项:无 - + #### closeInternal - 调用位置:读取完一个分片后调用,至少调用一次; - 作用:关闭资源; - 注意事项:可重复调用,关闭资源做非null检查,因为程序遇到异常情况可能直接跳转到closeInternal; - + #### openInputFormat - 调用位置:创建分片之后调用; - 作用:对整个InpurFormat资源做初始化; - 注意事项:无; - + #### closeInputFormat - 调用位置:当所有切片都执行完之后调用; @@ -329,13 +330,13 @@ openInternal -> writeSingleRecordInternal / writeMultipleRecordsInternal - 调用位置:开始写入使用 - 作用:用于打开需要读取的数据源,并做一些初始化; - 注意事项:无; - + #### writerSingleRecordInternal - 调用位置:openInernal之后调用,开始写入数据 - 作用:向数据源写入一条数据 - 注意事项:无; - + #### writerMultipleRecordsInternal - 调用位置:openInternal之后调用,开始写入多条数据 @@ -481,25 +482,25 @@ public class Row implements Serializable{ ``` ${Flinkx_HOME} |-- bin -| -- flinkx.sh +| -- flinkx.sh | |-- flinkx-somePlugin - |-- flinkx-somePlugin-core - |-- common 一些插件共用的类 - |-- exception 异常处理类 - |-- pom.xml 插件公用依赖 - |-- flinkx-somePlugin-reader - |-- InputFormat - |-- SomePluginInputFormat - |-- SomePluginInputFormatBuiler - |-- reader - |-- SomePluginReader - |-- flinkx-somePlugin-writer - |-- OutputFormat - |-- SomePluginOutputFormat - |-- SomePluginOutputFormatBuiler - |-- reader - |-- SomePluginWriter +|-- flinkx-somePlugin-core +|-- common 一些插件共用的类 +|-- exception 异常处理类 +|-- pom.xml 插件公用依赖 +|-- flinkx-somePlugin-reader +|-- InputFormat +|-- SomePluginInputFormat +|-- SomePluginInputFormatBuiler +|-- reader +|-- SomePluginReader +|-- flinkx-somePlugin-writer +|-- OutputFormat +|-- SomePluginOutputFormat +|-- SomePluginOutputFormatBuiler +|-- reader +|-- SomePluginWriter ``` ``` @@ -528,3 +529,6 @@ mvn clean package -DskipTests -Prelease -DscriptType=sh 打包结束后,项目根目录下会产生bin目录和plugins目录,其中bin目录包含FlinkX的启动脚本,syncplugins目录下存放编译好的数据同步插件包,之后就可以提交开发平台测试啦! + + + From 4ad8063b88c7d2bd46613770ac5b6c3e19ec5090 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 16 Mar 2021 11:00:34 +0800 Subject: [PATCH 110/136] =?UTF-8?q?[fix-35555][jdbcwriter]update=E6=97=B6?= =?UTF-8?q?=EF=BC=8Ccolumn=E9=9C=80=E8=A6=81=20=E5=8A=A0=E4=B8=8A=E5=8F=8C?= =?UTF-8?q?=E5=BC=95=E5=8F=B7=20=E5=90=A6=E5=88=99=E5=9C=A8oracle=E7=9A=84?= =?UTF-8?q?=E5=AD=97=E6=AE=B5=E6=98=AF=E5=B0=8F=E5=86=99=E7=9A=84=E6=97=B6?= =?UTF-8?q?=E5=80=99=EF=BC=8C=E6=9C=AA=E5=8A=A0=E5=BC=95=E5=8F=B7=EF=BC=8C?= =?UTF-8?q?=E9=BB=98=E8=AE=A4=E4=B8=BA=E5=A4=A7=E5=86=99=E5=AF=BC=E8=87=B4?= =?UTF-8?q?=E9=94=99=E8=AF=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java index 758be23ace..cc5f9bffc1 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java @@ -169,7 +169,7 @@ protected String getUpdateSql(List column, String leftTable, String righ String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for(String col : column) { - list.add(prefixLeft + col + "=" + prefixRight + col); + list.add(prefixLeft + quoteColumn(col) + "=" + prefixRight + quoteColumn(col)); } return StringUtils.join(list, ","); } From 70d93cf49bda652d1386f2c632a92e7326b8e979 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=A8=8B=E4=B8=83?= Date: Sun, 21 Mar 2021 21:42:30 +0800 Subject: [PATCH 111/136] =?UTF-8?q?[hotfix-356][flinkx-kafka]=20=E4=BF=AE?= =?UTF-8?q?=E5=A4=8DkafkaState=E7=B1=BB=20equals=E6=96=B9=E6=B3=95?= =?UTF-8?q?=E9=94=99=E8=AF=AF=E5=AE=9E=E7=8E=B0=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/kafkabase/entity/kafkaState.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java index 7ee7669f54..d49c8ed895 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java @@ -78,11 +78,11 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; kafkaState that = (kafkaState) o; - return partition == that.partition && - offset == that.offset && - timestamp == that.timestamp && + return offset.equals(that.offset) && + timestamp.equals(that.timestamp) && + partition.equals(that.partition) && topic.equals(that.topic); - } + }q @Override public kafkaState clone() { From c2e426b75edd7384cc028fd6e68b3ff1747a4f3d Mon Sep 17 00:00:00 2001 From: kanata163 <35188210+kanata163@users.noreply.github.com> Date: Mon, 22 Mar 2021 09:55:33 +0800 Subject: [PATCH 112/136] Update kafkaState.java MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 多写了一个q --- .../java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java index d49c8ed895..ca8e7babb9 100644 --- a/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java +++ b/flinkx-kb/flinkx-kb-core/src/main/java/com/dtstack/flinkx/kafkabase/entity/kafkaState.java @@ -82,7 +82,7 @@ public boolean equals(Object o) { timestamp.equals(that.timestamp) && partition.equals(that.partition) && topic.equals(that.topic); - }q + } @Override public kafkaState clone() { From 19de9102186b807a2ac8d96ab61f912d2a78ca47 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Mon, 22 Mar 2021 14:28:39 +0800 Subject: [PATCH 113/136] =?UTF-8?q?[docs-359][sqlserverCDC=E9=85=8D?= =?UTF-8?q?=E7=BD=AE.md]=E4=BF=AE=E6=94=B9sqlserverCDC=E9=85=8D=E7=BD=AE?= =?UTF-8?q?=E6=96=87=E6=A1=A3=EF=BC=8C=E5=A2=9E=E5=8A=A0agent=E5=BC=80?= =?UTF-8?q?=E5=90=AF=E6=95=99=E7=A8=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/images/SqlserverCDC/Sqlserver16.png | Bin 0 -> 180308 bytes docs/images/SqlserverCDC/Sqlserver17.png | Bin 0 -> 158396 bytes docs/images/SqlserverCDC/Sqlserver18.png | Bin 0 -> 281909 bytes docs/images/SqlserverCDC/Sqlserver19.png | Bin 0 -> 200830 bytes .../SqlserverCDC\351\205\215\347\275\256.md" | 38 +++++++++++++++++- 5 files changed, 36 insertions(+), 2 deletions(-) create mode 100644 docs/images/SqlserverCDC/Sqlserver16.png create mode 100644 docs/images/SqlserverCDC/Sqlserver17.png create mode 100644 docs/images/SqlserverCDC/Sqlserver18.png create mode 100644 docs/images/SqlserverCDC/Sqlserver19.png diff --git a/docs/images/SqlserverCDC/Sqlserver16.png b/docs/images/SqlserverCDC/Sqlserver16.png new file mode 100644 index 0000000000000000000000000000000000000000..bdd1ab0603bc8a8f78942ba40d5f6ac9b565c8fb GIT binary patch literal 180308 zcmaI7V|1PW+BF*6wr$&KY_Finify;C)!4ReHMZHdp{&J5o zf0yRGt{JJSEQ5?dfB*&thAb!hMI8(bvJ?yqq6ZG<@0SKA$O;+kII=Q*(H z*r2tKS?x@^RJ<+>v@cOy(6)+pYEfiLP=q!n3gSbxYHC~rN-;DZYLxEuLDV!f2<5W6 zl@z|7#)um#c_OZ#b+6u*9=HmLtS`PMAD7nzGZ*}>+ul3hEq=~8UX%^hrwIp<#p7e+ z;+o!{E`Pc(8V>sYn$YfcVg7|xC(}(1{RECyUN$iSldMx}R#h$w1Xa54Iz#XQdW($X z?7zwpf0S*k6Sn}mej2vkZyY?CeTNiTQwvsU(;5JAEhU0I+1S41D1dBF(VxW3rI;Be z3Dpd@112WSZnf3oqHYLuF-H+09OGtPLSzLe8w<&o1+r-8?q0TECCk-{wH9x4IZL%S zfzK2L2QT>)+Gu@l!YvN7um(1J6Te?j&ilZPeh+#E_TBDvy;1G*#V(}PJ=+|dg&B{9 z9s+p0uvndpb;FJwZ+#cF%H~nbP)aW;cDub}A#{HC&30S?UrC5&Q}EZirS(~GMId&< z&dk8mv(brW?bdgkmI(NH`TcgUFv8ES^-?Q>1+F~3uLSl?I@Fk0m-;#P1`?11k!Ljy zp0W{fsj^_7M7o{qMzzjBKK9X3FKobZ0s9Tjvn!z|o=4Zc&LuHqbC+E(r>54-=%c~r zg)2xxC*nR(r0SXM1`Se17UE*l(hiCW>dN8m+bUe&Em}mFO~H#%O@Ou-V?OpAf)#W# zgkIofCq%bcSRT}xBwp0Wm2Y7(EUoIUMONy+|J+c`BqCCmdX9$aNnkGNHzJ zsXN8lM8^$LNrBana})kkE97FcWPKa?Cwg~B7@VSTP!Eg-|N9H+e_`-xj$p;M9D8Eu zD<{Lt(BqOwLd;7zEA)zG;>|vEr? zOiG$Wt0HZv%;I9gK9!?E6+eOcDr9)9z1UJ}Jjot%uYYjdVmudKXN?yWnIOs3Q>9TD7U*9QqX$Fwtx`U?5v- z;Bj1f^HdlrRv&Z-r|2utM~-Icqw))!7TGLO%pOBl-)WO+VGQhtb)QRM47jSKvlw&V z6qrbjj*)ono_L>T7|J0`;1q{Gs+U`_Z^E+2(ln&N#**sHCvM-;d~AgQ*Z&4Z_+Ov^ zJ9Uhvp^{h!tgTcqzD-eCbUz^}pZf=e{KWVb7Dw-SXV7F4~+0RWOT)*eA9u?fztRqjzWkU208rJ`F* zUq>PY->(W7e{gGTrkP$V04}mn5rW22z{TZaegU!ShKL`Ptu)>aKNugO?=19H5|~P6 zx&F2mP6$ySGflT`k&EL&m7?eNLlVq9g8j(ee4(+YIs<%}Ok@LzxfNrz+lfWm<1{Sq z^H8Esnc~l{gBy*Rk!t$5{W^aif+Nq%*hXYRDCM7C$YFWDFosWqm|_z{#cF8>Fa1S< zRK!|g2H)8tu1@6N?Cw+z*j=a~qHQ(z9vRHmwU2U5zGStR=fY^X&ma4@Keo9Sg7X6;Yz1 z`0u|ke@N_xoJ9wZ&KWM6t=70;H?=~)R3aUKYzJXb`>R(koqLYyf0y*^JLkFE39%ciSs$B&5pBg3KHnH;rVZ^k5-Krd|R zcPlark@sf4>NaG;@GMp$#9}1)=--J8LgM#XY$s9CPWNL8E=2+J{RUvfz&tBOPl2F? ztBg)**75y2(f2*;Hs0aYD&<2%-UU!9)<|PZE2~E~%wg9L zOoRnUB$2B`eXa7!?5KbLem??RA@W%@x)FELk6-Nr}l!9bWdg^^+jVI~f#yF_>*#g3d%e_5CYR{MOdGj75 zH~sIu$h1P`XtSeB9zVeuNk4^0+=OwKZAHQ1_PD9rpYD3RKn*((VR7HXzDDV@;6a*c z_ESpKL2>HT4IfUz9CWHt?ky7{43u18Y}sq@_d}FTKT>-sS+Q;1UViy3{h!|PuUOyh zstthAKrY?6;Z~eG#0j*FK0zzYB$CzHMBI&3mla3mzh2A3n(G&4tLfuj2tGd z>5DfuO;CnRqIoAM%XZ;ez}KR(x;d0f=lu>d(PlT$upNFTybMXij>=EOy6!h22k%{R%~!EY7FsghNrt6rrNe^naUy^o8)81zAo^&^Yo$?( z5+mH)oc@_Vd3)uOs91P$!#C)Y<5EnbsJGhyyx(&eQ@XkSUEwC0Jd!86Q-#_cG+ikCg z#W*MnrjsK%CEslndk%>C(+h3-)Qcm7!a zPyXzt0@qC#sM1)WH9*NQV?l(sn7lrBsM)VY8VQOc^EEo8$=XXMt7DVsDlq9n&mvRV z9jamKp+Dfhq8R!FO&w4)93F)0$VKtoFaK*nI;IG6UQyTv0JvU@C{dT~*Y2k`Bm?(rbFo2(w{lBMpPTD0uf5PRr%}o{ zpprSW0Ldo)a{~`uM2nv@MDd=0v@l0rh_NvueyP$d zb=`_Dis|NAIbtnU3A1LN#s@n`yNNwpfl7JC5_uP1t`l47tG5Q#=`UP7Z+q}De-_}! zk+TA0pf&9Wutm>93q^(Jp-Mds`iK^*o5PJGT8=eZkUS0jz-vi)h-BH0xVlxyz1yc; zmVxX0brPTD|CKVm+JQe|6pJ8u=Utar!e-}Zjef1TE!*_;{+@hI4-3`&*b*r{Ta9*; z`D3^;+T1M^I}!Em#F-&B!KcV_SbbmRmU7{uYkBl8FW7og)zEncj-h@J+IrOiA`L|Z zc5>Sr@jyCGOim78S@_m&Xpw}4$eAC59+W-x4e37zlziAvo5L_%;e9O^9- zd|aDKCKE6d*+5nSb7T0Je2^_q%X?g*Cmj2db>zhNMJ|C1KTgZ#&)p=z{m?;swI#1+9iR$#{bdJk;=f!rnxo1GP6?rWYFG}>zC)&Hp z2RY8qpHRW!u}K3myjyO#7PGo&0>KPM3!VpbJFO_Ar0!s;b@VW={v0|<1>KWTI+cAU z^et!dbeBPe9c+U*Ob#5OJzr!0wyh>j1zdWrVgwZD)I^jM1^{RYS~KJOdi+R$;X9O~ zE_VBQ6Na-Q<3U%0>52!D_fkKg&7-6}t8L4h72%N|3Z$jmeYNRkz-}QFFSonoAY=MI z`q`(zDiYj9;;kv+-Q_7nf_TJa@?E8Y43utsrV@68QA$n4iv)s>W_sQm;bD#eN5O+p zwA2!39*vjpE|$U3Xk2ufvZ;PQuH&nkg%{JJR&7GmK@|d*6ucib^1G#C*!CgDb^jB= z-}{=>UYPZv9&-DgC=}%Hi#eA01cJuhOWiZ2{^S}n10RHL1drd0kV|`m>ctZ#i0FzR zsmI63bure3XIpWfa0`dZf zUmXkT?Fs2=mTL2{e^nb$E1ND%nan-o0`mrb(Zzr$L{p1g*l?%P-o_^EH=K9`RfEoxvV2LrU2;&zg{GgT7W8b8y}Y`ak+ z0J`aM2ASKoBXfO~#f44|K-Qej&#MaPR0MOMt3_99r{-SP>#k4tMR7sX<{l0kf$!_h zl4oxaSXj*xJASkj854ast4>l*hA++PU=`9+mccXhd$Zw}x>xXfxl!BZXF4u}+dRzvA=X~2u&ZXU3 z>Y?4=YQqf$B^cPPH7Z5v{2<{J>F&>8zj<09eWtaB11%jchF#Q{7;!ACEQE#2g{E&lkC-rM+x7vCyp<+0-iS1=lFU!q((C(X-2q<6vnGTD zxVcpbwCFO=-nD(e=!`4zBOV0K-zZ9dOb^=+Wk`8Y`)51$R2WXJ81vH-Z}fAc(0`3!y;)Pl`VD z0{D3dnZ3}3N)-J^Dxmjg8x*I-Y?=p!x@frA37QE!mq@a0rGc1eRw|oeES(rVv&>hv zqa`4#GD)nj+`wt;>9L3qxQG$MF$M#Cd2YXuqLxqum_mC>p}cZHpUEdCbnq#lCrIgc zZem_ASZm3E4ZegeUWkgj)dee)GMUA!x{B6XOAuIyM(3y>P=QplHV$~|RSD71eO7KN zm` zfO-WZ9lnwLNnV-0Oj-6`$ZJU_Ul=+a>5qGHU3Gw{=p=Ov`2R-B=-+5PzEpXs&}l7#Y;V|@yQRIx$HmWl*Nff*VGGGKFe21cOR_Zt z5+D*1UGY;xsErz~(22Ds6Mu+qkk3oUW(O46M>pAI>Vm7Or3Z@Vqp>;Ht zj(_CB-5RF2+BE^teaw`!tkbS&2%JZqk`5}1$dkB1@g#%;P#zlXj)#?

LdOV~19} z%vnAWV6*ZHPSm7nyiiP?`J3~Pz)}qc-eq=Z3U99pMJBnrDf)*!f$EpDz$xtH&!u)G2I0U7 zZ$>c?`)X;}(}d(I%J*-T^%gxcr0HbSaWby`^iJrVoHv>@=z$5& zp#^y-^+5>h>hhGB0_vs~#dfK8*O58EM+<09eVQ2Rg4Eo9I5EEz{I4lsr{Lc_==)mA zn-m-Mm=B5K;JPpa5|SC7N48aGU`+oqd@?kGEI*bjYv?DRp_g=q3_(YA)P)cHbIP0% zUp*uujsevcmw!@_ItIgeCzzx^eu_dF2Gksvq8T-9>bmQJTjY_CQQsOO(3>SYU=)7i z_{(L6oAk1GfvZwkYWbuOkf270HoM&#Lv~RtKFGEt%R%gF8**hh3#)pmsumuf&7j6_ z;U9JJe*));g|U7mz?#YgD;M{k0RGfapkfcvd!wV0wd;^E+*fq| zvBKxfZ)m9u47O@&^AR^UFq+yl=BA{JQBmL4+Ews;UqV$!WV`5~Qi^U`j_;58zCYgL z_9s(4K6AqXIA}RkV;KS&*f=Cu-GozakApxzhSHJ-gmvq~k#eLKX@?RYFvQ|RLzxoY zpx{G_i({b80Ozv&Db@mw9x&r2?y$oh_N}=5O$qOAzxATl#&q@Yr0nnE7Fjq`jbzFuW$LkpovyT(#hHp^U+Iu7pXFu4)}9+Tnck`#r3} zCbEy0DQm>-=WN7BZvo0oVEf5`73aX8 zlK+(ESjGlxoKKQ;7tg>&wER-xPb36aQ>&D;XxK6#vYl;s+|6`E0hd)YtlvZNS(t;A zaY7XF19c#7bA2aZxVD%31>@Z%aFHKLX!HB6y%i~=m>&~vlLan(PcZ~)X+D^B?A7rE zGTQhkl`UUj$lpP`Mod4Txq{&n!;7k!FFQ%|&K$s;wF0OFBsgJL-Eg?2VFwO=#|d$P z*~ajOqc|s3Ypf#v7>FUEl8xEM*53|WHkNEm1D=kFs51Ht4e2`3ASAmr@! z?4snOh1-@nv$~f8R2Mz>vZo5C~Ko3^(2$kYf9a>Om8zHJQ49sG%(B?%O@SV0O&do2g^> z47DQ)B%Si|rP==*t?5Ef^pIoU@Yv1xeoudegIU5LSn$y9Q)0MX;HD0d^Cwu=4y#X1 zLHHTyMEpqRMPSTcRBpm@tM?2HU=YC=qjyuT`GthlJ0wS`b$rB`?<%R?NbhS$R{tlM^tV9^5x z)?&e7XxVc!fLHqsSvS4j0H=-Oh5EYe*?(XcKGCxPgw{vycRN42HXYK7jVOP#B`W2vJcjX|S_sFHu#iLvj}Cz1+9D%AaVr zK;m?Wk}A*c;CbFRyun(2T**3;5cXcYus!>Pt0>wkpJuVhE3;R@5@Wrc-$ zie`o@)Cv$%)(K+bZad+GZF3PEeb`au<~(KwscIU(r7J-?ou^`|!e+mRWPUc^``vPBWhIQfiDg3G6Zlr5GZEmx5u34SICSisCTGTi z%qfzIfVwti*-SOdfuD|HLq9j{q_H@hZfvT`^}VM(-mT+UkAq6%T(7mjc#kAAI7EV; zp?Cy$oeHn%D^lEOp>!hVY1Ne}L}*B#07j`@`m}SO#V_?*yd2pZ$Xj{;kpM_&8ZT_=$}s3$v>@qRVfdk|u{<+R-) z8a5d7k^`rglS+E`xyBr>m%C@kd2O0}n zez^6p(rcqHAoN=`qc34nX{;2n2U)1i)WOGnm*(1 zCU52fC*SfU636X~CaHPY*KguWJ4J?%(H6h4V>9<3~P;kvFz6%Z2in6wYa!7uAD z1#yYfT~15r#A;eih}K=;w8Z`}ey+0zRvM~wGY3Vzz6QW_d3du1c6u#L3feu3y>$A! zKWnIQ!cVYRha>TXll{S{BILq~a=N{pAJX2;wBfh2qut)OXzlpjcMBTre=hW$G>=I0 zjXnd@pQ`lHCar(yavn{*x>S%c|IaNzmpsq@cjEOz?hVioLUA6`Y%(bQN*ID2(<3Wa z4CnJ{@X`6S&(G^cMFsflE)5OuzTImmG+o4&TpcCe!xwM`J+V-44CBU?M7Qyih`0so?l;8EC_+aog%IDx3q zQ0zv@02zidsSEi=)lEjuKj#)r&9(9{{PXt@ux+q-rK>z%Wju?mZtktMcxWkq(`kQ6 z7rghl|73}QKgIsvK%A@xrI~-cfl$Z2?Sj<(u1gs$_uAUtkn)(`3BS3W1c8Qaowv)? z0G*9^xTwU)#fr|bWlG=CaSge&i8L(82;Mjx*Top3$#jF~adYFW7t0XSc?2qb5rl9! zZ=e*9l>EO2L`j7Z(T4iHcT#QxP{X=8I9e^^J^)=7)Ep<$cjs%ERoXR)6OyEF2fj#r z*NQ2zHf2taIlV)XWp^@?nYENo8}^R}(;an%-DZp^qJ@;&HS2WTQLXmO!^WdT!Jy%) zy)vfk^|LQjr->Zl3rbyc_pqY{>>AO@`q3ge%7fgvMyPxwt&ST(gDAoT!^3&<=)^b@ z>N6I4mDdc>T#)U$ZX$(@ETq1W0Z<}1|V@Ud5!?J3@}RFUd<;yveo;O`n*`P*LK{PcvaZM5V9k2MsG_`Xn1GR z??^8oBTC7w+7Vo`WP}M`5C%g@mSs4O+o{wk-*E6cyrcDgds4P*N+N5`WVZFuPf&+boqt3DoK|gjQ@mxN^wQX^N zH_YXRR+;*)n%iGa zFN6Il=aq=0*7H8~Ur&46SU(w;>i%Ph%Ex}%(GUx+fLZ+fS#`EUpxco*ZIrF=L&vmA zVO**di+cMER{|p>8l8tMh?yn&4fQC`o#Ye)CiZNHRNo`jbSQt`iBJj30%Ote8_oes zFNqwY`ZL!F8epwx2wdI0-D?$XDz_I6ekIL7eUP{KI5tno^gI;T^nwu)D_sv~LP0di zM2?@I9Sf&bT6r6MY}`8B{Y)?3)z^ejddZLrGrwy+RNeGq+=X?!*(|vT7-FL<| zk#bB^-6J+|IYKA*JAx@|hk=$EGV$;k%~nWZDB>=46W>9|N5l$E@z3cQX6$3HAym~Z zKv=te9PBf92{%9`aUsSWOBf?#QxCiTacRK zuL(7`A3goyKUyqvIik!uU9C=qHp+>ZJLKvp61bA4G{{=2EQZaXhOGTmf>==rroe3$ zX04!WAkxH!g{YjdTJKAL@$FcEOOLv~fsm&4U|84lpurPom$R7dI`lic=a#_|P%z4q zb&z>hD5eG@b@J*8iKvUHx93Ruys}DtIiKIC3gNNa3Hmp)KLowqfaN%-eQ!K+HCAAr zW|CppOh!k`fyN@y^0_Nz93lILBbC`r`I9@n_33p%^uHo-H5%(*a?qWA50QS5b#%pC z!>c-*mP0&l@!ku)0q0#f;9*24hLE3RL&3&*L}f-sk|8=eEH=%LxMx1XdNFroRiO?u ze?r!;2ysOgTpB^Vo4}NXC?-SdHsK~PM8*Z7Kw^t_vy9qKs^<*;4O zx9*2G4Lr|r5Jb}X=|KHL-i`=<-E;;xZNkur@JnHBr6E~~n<>5k@fy0;HO7*lptH@R z>s&i0vR}G)5iW8UQj^68b}cGiYvpf+94wedZe?Zz%D|-WdBJA<+}n}%`e_HN2g+i; zImm*!eSNuG4IWAjAyAGR+Ob7($01*o_znF&;_$pryoQwWb>f=pUk+T}PC8D!oK6P; zJ$FjkekL+d8G7W@66}jw)?yADEy%FvD+GUTOz0v&1#ET9Cf?BLA0t_S9y4f|2VwZu zlo1()?GY_g7~{iySGDG2$8yg#QS1mo{Q6-8Nuy5q%dGs3-NLUr-nupTtgj$B+#2g zcqciFgvoQ*p$ckpS$}7+Mx#!=$Or~;S=NN{!vCf$z2|`ykh>m5-u7d3jTdPo-gdVC z>I`;|cI9h~xz@%ZTs5&>)^hv%uQRnI3I@)dR)XtHY&_>lRivLEqwQCx8tgYOO>9>C zK&A?V&~NanD6EV!_fa5ZaCrXTCNEfxLbS6Zqbn4!Jy`OZi*o5@Unmd9083xiEC z+7!ppf6vg?r0Pi!DK&DRmip$u{QAjT=P%6m(`GEL;e1-E-EB$>&}qgDgCs_kDiV!R*Fvfw*v6r;>g(;Yrg?NV*+bf(nfqR zj#J0h<<29VuX8=SFY`QGrb>b4Ydq^(bsiJb1p0X-i4-I^Hhf}X==wxWUN?Reh( za73$qqD#f{^T&UN=tuLSM{>?*5q#Rc;2&)L@jP_!A%1evrX0vPCh8;f==|tYE}C^N zqtqSlNGYgxm&z>F0%3Ae4w!$hj=Ipp;jHsY6KVkra{2ArZIil$-O3*? zQeL4u#^ih)LAB_e8E>LAMS>qLrkv_@!;t>jA%b~bnC9hD%mgJBior4lcf%>z<*gM3 z$cq}x=D$+kAd^TsF_u3(xbiVr-$2eWxn3+D0{ZNi2mI~t&-CRIlmx1Qd<+d%!$wzc zK`T(1C0jAN;Tsn(&>lmR2@pAFU!Q>2i|y|xWopQz*2VGS!oO?j1~j5`#S@`(0b0@4 z9ZYD`6VphNyxvLV^t@P6X+NW~vmesgP0^7g1G^BQfP@)v@|0}vz0bD4Waj@fRcJMgvzI%rr92+x>g$xU`+ZA z;{0|F@kLcRq~RCuU)sO96Ib98T^tlCA)%ntX6&l zhX?^rqkUVRDnr>Foc%VBe`@F=lLmuL*ICWGKfV;F4DxL6DVA)f zjm}nFkay?KFxr02kh4FzFj0k`&~|wXJKc~`PE_d%x}Tq43s)jVu|_q`!#R91e;R3_ z(B^1%7F6NXjYbN?hG~6guGYK3Sw9?MbM2QHCpA|x`1P~okJP3m>teA5Bp>O`bc zq})v*R_w6borvWGS zJrT|``8pb`61!Vy*l|yT5v~4Ah^_mgT|rS_#$FSYL~2#Z*(AR+mb|!qaVVqc*qc@t z39yIe2zLPI(Hy26%h05Uv4zTD;r$>k8zoKdOqWaNDD<)^kG6mAnp1sA<~Mno99fDD ze3>=gSmS#XVOqR?XJoa>TpeCS^XSSzP zS!for4mrog`eL3GumP8@b-daj7!|dR!3Mfs)ksZ1 zOC3&?G*F6`X>OnWxoLv>Ui4q2)lumm(%KTOLJW|VoR3o7O;ZUsYJ3dQ5GX^5N@E#0 zd4HkQPfLg&Em6=ZwEED8(0wzU*%WVuO2{J~Ao%_rLmA$*+O6taA_fDJK*mDylFlC` zI2drC1$+fkg;@fTf?kmmO9(m#{`bO0D(NqC9HOg0XbYnU_AM-IDzt90A`#Z?zIac6N30igzYpI=7TbUSagd7g;>_bW1|n3yQ~*N%_&` zn~Ws-95*w6{KVd@9~dllC4^>VjdA_f<4(nf7@6`IDW>#DTKN;^-UwsRPSE(+Pp(kf z5~*_N_57maWn4gjJDc!KxZd+7W~n5z67)V)bD#JxZe!;{^z?*HZ+ov0IoQ_G08>L1 z7NR)lI3hel`EQXfA7NfIC>#!Yza#JTHmK#ay9R*{5v>$n3Mrcj6sg?D2U*QiaP8?L5Ut(}78TKJ z7g6`|9uD+JG%(Bi_1uB^UhVDBC(S)3a3?C+eX&&79iz#dJ2?bE!KrivL+BOj4Add3 zR1Pm(A8eKf2WytXl-(1@Y?;$Tdrd_)V?fl5x(R9iPDmSlV1gctY>_am#@1`G32m{} z8AOM>*_8z_4Vp?mP)0hFK#YMC3Ajr%`wU{lQ^GNb;?(mw$`xzvO{fc`U*s3LkeJiy zx_iwGXq2FW3_DieS9wz$nTtUxx{_jpPJ>Q^MI?pj; zUe~w*UGJEIMi8~xwAyFxs}G|xzkWM|kN#HVlPf{9EU)D% z>KyTU#*bxP&Y(>F!N6d-#fOxpY~vLixF6xLr;cSCu2Ww}=3z1WFDnY(ES^ZQpS|)KKt^s6$q3DXZyX^Z0;Ypzc9N;(@I8GDxG?;R!oe&!Ktw z^r6vk%;X6eXn4EhLgLOv5>=c@a^sdywU(AtUy^Uh60bPtuOpNrndT)sbZal=(rZ+q z{U4a2JtT;t1c=g5@NXrBo$r_44B+J+w#51?hQbB7Y@p>siK##1>4sn!k$6k7E5`IZ zXgSddzQU|5$XQGhmmua1vtzgJi%z5dcIzx4v<U)+*U za$iw6P^unoD%Ax9VvF+GRF`w$C*54u)@8+MEDMzpC1)b1?8+no7JQte#>Pw>wP23< zGv;U%EROkbL%c?Xi3(G;IyP)oPgiL1DZh-~cqx2D&i;QEk<7hZyq|C8DGtmV}o`+5X^bb?c)@i z&Bz|!Uzr#!r)wuSrTH99lolWj=duV7AG~|4Aw}En>mU(OZ+%J^rKkX;XAvQwmdXpiv~2)e zhi>TX=XD@<(tFgKO%TK(fa6_aD=ZT`0z6G(& zw@#s_Y!0+fRNd(wM|fvX%wtd(v(OnhPef21VwZ+u{BEzSG?;YKlMn;U%Sc%8b2M>h z435=ZGu$N`NRg?1XiG=!)R&`FkPp7~TkwDCB_Ow7!Fgv;T2%w{dP|*U}~xRQok_Fjk!>up&GU~ zMZSqyq_tX)NSTFI4>|<+(CP)JkGMTREhKXwT$Cdop~EGQQ?#>5Xdhpxan5KE$I0mQ zNw+0;mKZ?NL~~ucO59CY9bQZ6$dw|wa?sPS)bS~bn9A){vN2`u-}JHUDao^uQ*YBc zTF})Wr6+QpqR$jaghB4JZ#ioGmKa@>z+m#DdR;Y!h#U;`f|YOIU?4+h;~@5(Fx*C!arlIo)M zq^Y|Wa5Ir?caGfYf~{|J6DFG%pP`YUB=&%e_b^N&e6c(0j`_EKSEH#!J}yfE0@ z$Wf8Yc&esmNVy{qU7l%BW^qsP2av$|ZA{=f@aKJh#UFNIzlvC%d+N-B0_HR^k7PP^ zT@9i@+|g)Ok;tGogYmQOzr+`gjEzd21yX{ls!_hYD!32PRmR(ZBq-dL9ag~6Cjo3c zZOub!Q`uc^Tn#?%%_J|LNgKK&v7EkOcmliVRce%ZSiYMaZ89JPUn2TQj@u~Kbfa3@ zHc~L{#-IYg$RI0Hyq=~^-Vt`T-whYF77(x>8B9|@MIk``9V&zmqCn;N78p85!klTF zudm35i>oCOo`RNo!rhUuf5G0J>@=w*XqfadFJzTIF&&hT9;;u5(qA!We=l_R#~EXv zTX&Bww!dRl9B!WJ0rTT0h7+`*w|u)8EhX5jo7|~5vF?;C%I?BFXhIW@=g34*7hcff zf>o5%-lYdZWrq6E72-Ot8}27ou20j0TcQsZZ4hL#`1(hh#24)=PR!idkP#3!DtY%Ls4;<3#*1vJ7URw z1y1zkoT+xg^OPahxZdh3sK8?eU7=hGHLpOK@EtyZmvivQ9PPk4-E>1Pjj3js)_QWy z1*OSSY~}_sHMn+NNpa5pSMqHyLOG5 zncYC$ndy*}X+7|RkjzOw*@5p!GpeJCy!)D)p?{b?JGD>VGNDHGDfjrSPm#u1y)G%C zT#LLlBZi!9yxsBEfvr^J;?sYc=@BW-wXqa2@>vgd&M4DBpsj&yB6!6J-nKn1cR z%@vywxm*Nrs2D@ zO(H=BBIRG^{`0cpU%pNs=6`%1dbo4EpT^K|5Y$HSr3?m|vGyywLzn{OhWgZ362Iu_ zD@H=#c@c^+STefGzIid|&o-sk;U=Q&1vyO^F$*}e_!xm0UEanjv;qqE)%Zz7sa`{4 zDgANF1VYG3(oj277En2%WG)UG?2#)XMpHgAnevnSLI?dQfD>XG*BH9TvgKo1!yZCm z@82-`8RHcqLQo}%zTB`4Msri*EBth`F5ea>)Wex~qrL&L+*_IeYe^0-a6Aq$($L}KT$ zp}qq!0m49G-tGJQE??bJpF0uLV_g(EamcfoeK-bspfo7hMF+$27S82PyDDW94-H(( z=_XUKGlb!#Xei32xPtluMiIt)DA5!~HE&Y9A%h)i_Dr9k4;p;ri2c6ZCwX~mv)gE% z*6785==@(2{cjhPE&ac}ek+{;J!`0la-l<|WRwK{sA-yDA(vfVe==mQx?nGd8mTYI zS7XK>7ICix@_jGyV)tuqq#MRrw@&{=3M* z8d9CB#WwNo7_;qVjauMpTmX&W5Z+QGw#ee$Q0{@7bgWiy-hyxmkaZU=S$ z8jA!?j^w2=>$CJt+={QHt1?_o^BK|i3`)qO&}@Go4&9iHG24am*stgJSm0tBTo3kG z8t=vQl8L4U1DL{$M6cky90QOw9J-!i=Gj?Ca0H-5+Lc4C@5kH9@Vt3o z(X>AA{@fs{q=O;2Noa@H@nc-2KDe8lH)Wrxh>NyF@n2s;r%ocwCe+np)n`Q7QH`V$ z+}b>MW!`_eKdqoYT>nYK*}7cEyQIAE1y)Fn#2AQ$fVHf^(9w%GYVDp^nisdrSP&_; z>D@6rpw$ji8i^25&~$HRJgEWrU~w@ z=1HDp-=`+yF_5BTz__e2DetjgPAOn=e`B(GE$w!>gq~)^4lcJbcu|vAxM;jH7?nb5 zqA$fSK=h4BglU^%WH)9d*ZF^L0e&$PIzNern;x7l|8;Ksy_K$GyGjPPQY64A6?DE4 zrpvsy45V1G8=5OPGI`0yPn5}cfHc$HgzKr^TTU}fF5&D=X5~!Q$tWmrNVq@oLQvFz ziM?FKPm3d_1-|X=Q}dngUQ!zBL~EKtDxYq^My~4x3Jo^T*XVmz8|X4wCi8-`kO<>6 zSu&J+6c%frt}GEQ>I@gcm3k25ru9vGzColJ2$NJRWutsme8Tt2pz0)9Al8ZP)+U-O zQNS(w(-Us8;s6_{h3w_VU>yx%Zr;Locx9@(p$a~!TxgkQh{wCn;mp2c)QD%l;X~@a zY|VE~kiBp#TdmhiqKyW;ZZzk+KGcX;H1FdPr_JF)b9Z-d%r)bPblB<)^ggqjUQ1)V?;IpXFkOxVTaI8gBCQRY?K9_twAkV1P7#Xq+JOngKp-prk5*K$QD)Y7u< zi?u7mBkxOEKJ4OuNx+{)|DH}Ss8@X907?$WxMd{+rq)L6y@@}$IomxBp8<~_f_Od` zDysT9(s_QCCYIh1lCWQ)W}U7k5ND$Zd3~Ooz`4lTQ4*$h!-c;YTMChP1eNHw7EGdt zlV{Zi{h^sFdlXyJwj8E5H$1K{puGD1CUr$X=p=uMsjBvs)?GWnx^Gx=W-z-@^x!0| zY;VAsmrTj8LG$S<2h(wMAfsuQ0Mzci0P=dje;!L8vLR#K>&7u)gm0&hA@T#azzqeS zRx-4tW(?ad8j2Y^p!`3Cy<>Q#-O??bq|>o&+qR7rvtuV6tJASbX6H1ZN_}R3HQJyAf6N-H=aO*0{O|^F_Jn?K9S?b3 z71~Ox&9?7@ICgp7KV5!8!f_uFp!6B+L>}xN7-o7r^xy1Li#i70=v>7KCabaN;`)yr zJ=NHO`|6le+D}kLoE1q1C|6jVtzoN6* z_YTCr1A`b{-lu!?2klVrLIWcE2u*A$8> z8X0zS!nTcIhV6_twsuJIC6bSx?9^iI5J-?XYQ^Q^^JYNERJccM-=@WLDom&Wt%&Vz za3Y4~(4)5f#<;(x)lpmXFgc#jVMf%msy8@7PH^lS zssE{S_cx)2Mz=G>Bf}k3AQs!be}>L4DIXP*HBbfD-zGr|yir!5e25rl>v1}nu6?Y2 zGTS%L!zG}$lJHQM{borVsr2#zA%?p>DfDw#73Y`H@3W0@mT9It-VZZNbp@Kp;{%xl zjL>A#g6HDT&mmIgMx9wd9?>F`?l~$rpY_$I(hub_9KpC))i9;7I zQ|a)BnoMgJsKC^rFu}DwpUmijuB;UL1NS8+pP&4XR@{z;zjM>`u%lC01Rt}}!vnT<1@1Fi&%{oZvQJ9q@{VilX()VcVs;WN1 zaJ*0vtgr%XVR2FC;g9_5!TzU)ds*2W2AdBOWKz+rr*?{+%<{V9cyV3g_%yv8=Wqfg zT)>^xaHbhc@G8F~NkH_*cf9mI7hHynPO1$L=^URt)!duRQ^)LABgZ*TrgE#RooU@2 z*e{yy3zIcjG*{Y#4XG^bx43^Y#6NCulkfU~Ux+W590DYQtCizmwW{7^(KR2q@a_@PE-QcV#`;ju7`5_7)hg@o(09nQhB9AVPk$i z2WLg_Z`$9MH(f41mL}4cC1N+|Y+q|FT32|E3d#F?pDpXH+V*3(s3|o%PvZ{*caaFW zT`pYwa_K^ZCa^EHl0RMgYgWOG`18a5Q@8%>_N>b3R%+LIBY&{R12+U~M5}%?X4>2< z3yW$kQdJl(cSH89f(K&dP>%U7R`|0WJ_}X}s@#e}U};Ht-;P8&7k<#P!vTAJeKpPg z&5am1Fb~N5B$b6-*Y;pmM}r`0#LoVH%(=%gK6OG&@92bFQPFN`r7H~mn5K_57GORg z_jCR~U>}y^s|%vozL!W1CKi6+BYzC~)s$C8t^DBCqN6Q}k{J6F{Hxo+suJ8S_QJ{o zFgm(hR92LPR|v|wPeip?vd@a}vx)tlaX+oY(jw!xG(m)?wWrzwwFTO!tm`CGIJ*EE z@Jy4*({~BAGm)f7?dj6XV~+E)?O;@=E~o?P7PznU!k-6iVAv)=-P&R-tyW$o#UEa5kTyUlnX}ka_s@5gFQFtsVrJMIn+t_bz3iIUrtH>&9sR>Vo0t`(s>=)OV}Z-CY%nypppkLUaOtdT^KTUt+I4!wX+_6@FP&9FzH# zt8E88wBBF57?{Z4USS2aWy;?Wt03m4#qk1CPN*Q?*BgE@!|`N&v%l4lyi_|d!V+_s z*?TBowY?JDjPq4EU#aU4ytl~LmD-DE@>jPMQ^igU?S;cx?ThXvj!9b7Sv!7*;{js5 zwl`WQaeX`vYFTNzi(9XE0FUImQ)jto04~J(z2-=}vR@~tQwDp{{)l*YG>fnDE5X(S z+@mEk7(jfuZ!nrjd*lwlH!`=h1K_7{09U^)m{U^-2(~&ERY*%H?8MHbb6jQlDX0(m zsxziJtzg#hapn7gWs>*D+dDWQQdxA-4X{oklx{TH%ma{Sdd`6BeZ=|=85 zJ;`56jMS4W)7~?YVCf*lFuvmxvS>JMQ3(;f3_vt0jIY{7O& zswB_?tT-81E@Qc3bhn0tG~y-{1w6ud4?J46oqUE3LV!1T?BSMMLKg`dB}}eh&tt^_ zc1sJ%RN?_ieNY2Vb^1b}H%}CZMm-oQZ0wE~74Z9sQ>k7RlBfBPL5q(!BK*lRW)og( ze#v5!>~a1`D;WHC31-*?`1u>?cc9w3&5RS*_Xo>Uq3Wku$VV9)hPVQroO z7DA=!Z1W6B2K~sw)2@434gx*OG5u?To)`-bB&m^()Iig2@?YQK+^GY)CPIZr@Ur|# z?Nn(erhI+6y1Ma%Ip)k}NMkxy6l0 zxC?qOCNh>WtTUyt5l@vO1=_02nyxQB5-3!71H+qM8$$LvgT(t6{RNuXe|L#PFLx*ssM-FWrno;~~RjgL`%k z&{IyE@7LEgDz+3=OHoh&PKI7P7aVlHSTQNr3aF)**dfa2ddE1AuovJ!L*mmFJvsFY zp&{5Y0hUc> zh0SZmSb}Sl6HwG|F6Yc{Wx|P_z8j4!x04bay-0k{JkS{k-`vGH*3$nBgkRECwZDy* zda104vDxzgq3A60vO~^*);&ymOVqsma7&RJ(M|x8<)$SVvV&ws#GV%1^P?lw_h%5_ z(levYp0806#MQ?Si69-G!ZpIi+J}GwghGRTm&EbD(8TB%p0pn|>fI5>TWz?ig~M4C zSQumR!3U1sMMOcZYqE70){{620y?28l)W+M47JUHJ;rdBqA` zVk_&&H1A23*9bY#;~>D0pF7%&P>^#={B9TQDC8w&+LADRnNwJK6Ifh%B%VmtCCO0- z!nqi(F~R^?5M5{xc^q0y*oAO;sgjZIi9F^(GLX?NS3*Mw9m7U-%%1;YREhg7{54`h zqI~YzfTI>7K;}qM+O!#rg9ZQCe(doJ}$9nBQK<+e{~U zGMn6xp#+^>K=U8@?NtAnAxdQ5EG4s2CkhY*U>~tJ7LM?9Qsb#>#{_1~=6W77Z4pkk z$Lu)DtkZ6m+So!$&R`rND8%Uqqwm_^+exrJ6W{u`N&S=EAeNBKae$TJ#z(px;l8EX zuAL=Ig_nL!QBui=E)|%RAWdM#9;b9q&qbR0@n?bI2QNEJwK0Z`QSXecBLH}o0;axL z5AD|-4)F94?LPA&kvA5d@9DlNO#l1N#0~3oEMcdkpjCnrHDXt`Cl9-F24{s*n1TtqxK$MAfnn+VT{|OhH|&p+V5+5Dm5j!bOkU_6(hNkq6hZXY2N0~oN%X@>m5r;T$ETl~ zZj)OCerW0c+oQJCx3mNUr5A0Kn1$2X2=s4NRs!8VEUJAMa%UU&fSOlmS;{$Ek^7df>h{lohyWw&>1^*qXnr$vX?j( z=fVf9oBT%vI@VQ@k#up63_I3Tc~%M|n2VZHLu95Dx2&Q{C~4oFaGGW?3Xu^NhF)JQ zRWd`AwW1)6uzIInof!FN(?u(SDw0~uR4jCyg<=fLMTJ6-#4vDEg10OZ@bFf}P7wlN zAw-Qp7RFxWV!(-t(fDoyLR1d2^$wpmHz4=rNkSv+Uud7xdReEbr%AG=kYwR;gHP4NbS{wUnga z($Z))V;<5lQdqs0H0KGALc;Q&D>wzv*sa}-yq>@gS#;;!J^7fFm*@Go&W+hB3wiSr zsO8Qohn8;pt;SO8FBMTVx^h@sV9_7r^;X z+v@t4shxrVivw^OwM*>|hex=D0nLfC2rANmVwo2OIJ<0QGtCh2LEOmaLv2tNPaqaL zPz_h_1t+~@cP;J%AV;2eQ$11gtGC&v*JuR@G+MAD)!|=BlHs8UN;<3zh#?ClFBe8% zru;vT`GPeu3Go4rB;A&aeH6TW#uA9>DaRIC4aC)=XREc*674av|(L;Do>8r zmvnM3|7(SRd*omLK>ZPd!{8+NcqKvz+kMcow4!$)A&HuEbe|>qw1>HIJ$%uo9aFsn zG-}h1RrY0`;LY>1y6d*~dZw7%V7a~Vn+_a>srN;G@~>A|=a-dKp9_SdBMp%Q;{E(1 zMA}h|#fpau%m2$_tubOLZ^n-rV^jJV#1CHMvzUMl=;{0J`dcr$X4Q1S>R3SbowDkX z=y6=Qu7+xpf3FN5xo^Q`wPI4-105SX-xri#&9SnBv}cGU8LPq%(8r=TVqrL)pIlBj z{SRvV^Hnc!fN-T;1dJ%jk-_n5PhxE1$XBp4mZhefUeJ1t=Oi1zY zf00~W5t}t%NR4%#5D}~b?{WD=g7J86FFPN`i2DeME+n~<6oh|>49AyzA(F-7@BdzD zm#eQgaY1Bq#k@QXGdabqjl_&R@*716X=p{k0R$p5yb22&~xMi(hA@1<@>NvM&A7~8;CIdWED^THtyg!dZ<;f0RE z;Q-TQ`?i3<%E=B~4up4$<_>NZ^m=`7Yeg6`-F7n`f0XQx7y5}KNjKf)5PPigE91Y- zsVak@N(VtHwYAQ3bS=-_>kCSnZ8``C+nx`6ew237SBwf&?$o~_+&y=W0j=Er6CmjgLtH_{CMPt zzCGi}kC;X0Dt@Ox^HId2F!8YjKB5F_$0W!E=h0oPsyS0XqIItgPuN@^{cn?h+9Tcy zKIRu-e3v6DSO`#CaF1nBm84{jAS)sO8U6{3s+@y_RKa-<9qb3vQw78?iy%T6n+5X3 zCm-rflzZhWfDqhjO~h0@l0SyfTVGH!HitMX_klcA|4%dfN37~11p2#2pNK*10x2Mc z3YjM;mJ~h6$YK%;;d~e3%dPTdpy~~~*N&wrgd`HX&NsD&=p)=3j47#503j^Uh(=Jv zBkP-^{Q{y62XgNJjZ+Kg|JdUJ0~}@=1<(Tf?bJ|873B(gR92OoLo#o<225|@7?7u% zOmKKolB-yeZ5^X4&#%cPqj@WG^JFu-QR z-)dgQ!h$|U^Y+HXdW!v^gR}$>IfcgZOnd)*p~m{H{IXsWop?kq{n0|&Vln6@xuL;i zYGuYR@j%um6e^F2O}kcd?8&5x(3&?2QsvIrt9H{uvqm_{CO>eH{4NetpdqtVlevHd z1C<L#Yrt!h}arv^Ic2L7?M6F&6L}~gy1-Jz8@nhLOTxbz?)sFs# zquicC=$h?Vd{|~+u9o81v>`P`RZ=n5a~qvVN>XZC4Ep0)b}cBacm6A){*|r%EzndP z_SKb>1~nvd_1_T(Q;NstnP#irs!*-X6uAHd(OpTUn`R#yOtpM{wq8?9UCny+w)Iflyfw9B|LOs^_R7H1D zWBr6qkNz(EyhQx-d^5b0^K!I>tt8ilm#%Mp%V1SkoItJfVz5zqvdURyQr8OHZf6y7 zmP+Wt@85EeQ_8Q<=t&g(v*KW+70keVFp;D|o0+->ZJO4H`TbX}CJoOtW)o&JhTQ!C=xGdLPcOzZdF=3})Ozyj4^ z87spgqNz*>;6kJ8WYLilao(u%xX|&V%_!B(60@uwPU#09Z%!B9o|lK3hW<@A-khUR_BsLBfXr1kRUWEL;V zw3`*}S{|(haUTU9zx-pZVv|SuMt5H}0n5=gug$uRf&uFN;Kk zkNUp33{#6wyUlYuCm0Q);?+(80BtjpubG5tIm2__BHKvCd&D^k$j5cx+(NsjMaX$j zL&ICATPv)^Jo-Pi04Kca)MCTNu;`>wiDnCtO2+e4Wr?;c<-8l#=bo+Q_cjr2YLb+U z%=Dwe=c-PN@77Tc{|RuXP+#wuf4VQeCoZJ`M}Vd8+1>)MIDcOEq1`AJ9)DE+vA(br z+tm5c$~$4LM!%S-R+)TKEe2ahH#C23@!$Y~J=3z%7S@bHs!mlg<~3Sha=@dB=zzcU zqU_ZWq|z8TG@t%B`DAvg(BE6}r(CB@BfF(x?qPm{xunQA(WF((KbUPBIX4!PESy$*~+BPgvEUpfm4^{G;I? zV!Vx0qYHWP z8tokyt^ei-a6Z2R{-Q%am#&Q)*`g>LA8u1_{!Dnpbx!I!;LWP{Cv|>hDVox5U1@or zBRmBL;Wfu%`+5~KFRTUU0hACIY^l&Sb~B%+{&NkyHc(|QZ=P!g#WUdHOiVBmhEYpNDrC!mKoNx>flxC! zl@jU#1TiHr?iWPPSznjYn(1LAO;YpMArW5c>&qWJx zRvWTiFK^YQ=hV0}tg1Z-tsqfsep-W!L!}-$crQluLP#Qj6zGCBy|50QeVf%;7;XPo zuJad7FQfm$V2#4$im1LiV`DH}tM8JSvomWE!EA*qPX<*Qic-k4@PD|OfI}#C6Di8* z{mwI0;IS?epVSp!GH0BmUHPCi?ctH+M+Ipx?|C9oNpI1p!?JO?@zvO(pgmgVZsL#z z6pKkzw(0DNJIGDnx9B`hD~~KpFS1d6Gag zJ;>h&J(-b}^ZNM=y2V?4__4Os3n*80Hb0C;%jX9@?Ejr!R9Pe1_BnbG6bwA&AvU)d zwAdSC$;h;;H`jolpJVzX%3muJ);pPE%qt2$3Hlop6XlM?M*rgO`)F2c9ASX9X>C}z zs9p84hF!2ac8b^zGS|llCS^!my5gX+!Qm!}REuXdLLwXG+ zrb?kv8l)I443!g&r~ktu|3M`}1H+>Z4%8eN&3r(!ufrRC2nHc3tN=42WJJwJu4C|| z(@aNS!r4*6@l{~Z{Ncfkwv$}TRgIw49;keN`9U7Ke1IURE|gZ}Dg{Wmqb|-XdEG`o z#a=N{^=jyg(R`r2&P}!UfS1UGRE2cDMl=idnns8t=6`w?QNU4=#_QRxwC1lgsi3_dC8r3)+!3u#(ar zrGylh*G5|rx}y5qc{4m9K*rRGzqPL-WXR>u38N^{Mn-7X;(BS&8)p@~+ZU^Z6z_jD ztd_i+%j&1+ib&RzF80xf0g?7yMMEUsWo2D~u2@q5+i zF8oR=CwR`PwvtR(_pH%;O0v);W!Op7JnS&Xj9eah>1$yvNE<@f!Jv-WQBftQLX(pQ zsC{aiiv`G{0L*{m8J&|;QmxAp@48BhDrngfy2u!3oea;x7?(*ctMn@qU&PwJNPg!N zir>x^LP~tPiWQg>A#|_^o*1nOhEAQhFAhbNtbjiWb$4T&UxY_~=nQ`d@{-xY7Sbpv zpe9)^nHqV91!gU&7NIoh-*c-9g{FV3Ik2lwEX!`;juUwm`VjU|#gy^VGE z*mC7|KW61tL^JeMU5(6|`_CYgnrAA>_p7a!N1_7D%wMxw?dA6tWNKgO(Ee*ma4`}a zDH+lgIT#jp(k}Mq65jfH`^;TQvI&aP1&NLpe9LJ?YfgT?Xk;yDfoCg}6ATM#oWxfM zrQDtF)P;I~edz6tD=2csR}mTI-YuJr`I<<%$%k!llXtvB1=z0?1HcOBV^_wDqw6Uu z!o!|NV0@G+IG-dOZiQG>#>y7IU| zN)EMWq3frKrWJ8BlY`XZl^liENZ>p@JF2I4WBdC2!4q)|QT3!Ymt5N8h( zWs5XQLVqebpy6txei@&Tg*aS3LAS;2Mh~^aK~{b#bU;p2!Z>a+w+*3=%dg0#W)fUw z>0*r^yIQCzu8LApRb-;SwBZIj-&d6Hi{xQ(xloh}05Kp%Ii4s#-d8KDJTvv3W!6)~ za#Kf!L){L}~0NxPSDoC%kLOK7oBi3=>XLU5Ll|T<0GLkm4xD;$gf@IX5slw?;nlDH(=$rhk z=$k_-_!>+l5M2CgGk^L$Q_tB8hjCvZ%Y0>OD%i%uLfp~G>C~*MTQdOhWH-o}vE(Ad z?IJ{rj{NC|`(j-}EfpzYL3JjWyhwgpZ}hjXaws@?3AfEz2sG8AI1>YU$g#(x3>)3o zNc-H zk!C}`dj0CrcLUTijmLDICGilK;KYFGLZ#g9I13FIavBu`Xau7%vekS^r}NW)8h*u& zu#I1Z|Hk&NDn<0#2@4vgS-hWXnx*;aqK@019lGBNZVx*HTx@8%-dv!gkH@k60PJ3r zQODfUZr%T{8g;#iqjYJwL9q;r%mLr6)ECu|9sXgym>e5F(pn#EEc1iseg}s}Yt45; zC>&K=)UghAdSl;EQqH@1l>2J#Z271dg+zngmuKo8u4;=Dr(yO1<7M88@#b{1Z&yw0 z8_HrVE)ygrf)`#mpR+-q%~mC6RD!LV2QwX^2fkbGR{X2qjuAr zrd!y$uV@{gz11-*-T`HnqLa`C7j# z7Ry9qRbgJZ?=afMiYy|nxt>9!v2T-EzBE@7&)YD&o8Nwlxv|53mhjS_B9G!}=2?6i zm6Jq#CWqoK(l9|;1y7CYPmG~g2~q_N!}A6Mp`!SLbkwgm+V#7r5nK!#UgCB{SX8z_ zQ8rY2K_Sy|V$YE}0Nntt$Bd-4|oA@ zyUQHszPDHNxK8~Oo4CQmQR&`%9*uo>#d1Ija1r=p-HDmYDBoVJ{FP5?m(FT81>~a0 zJGQLCHoKiPt$me0VPj4jfj!;2MWemzmx1_~f6@couk`PZXW_96U-WxK(Zir=?^_NM zsIQ*K*?I>&6lWIrW!%1r`fddXWaAdm^dI?bcK4AN z(wrTwL?zSYOZ{aeO2lUNYr760=i_Ng$_9pVW7~q5>7PigJx{P;cz)r(9W7mM!B6Uh zAu(OW(x90h7>xaS)f)dm5;-y7WW;i>W49l7mQ{xG`^q>=QX5w;5QsX7L6uwggRpXbYu04E zNN#61J8HM@E0mE)@lHqMHbGIw*VSw1?4e~jUBJb_x zFsc`JGR;S!@I1;#{31x9FlyjN4-(Jg91n*3TAX8RkC)-9uf6Is<}(O!chF!W_P4PN zC8SwMv(|I$U}y{Ithmt3Niu6gtXg1qo`n3lk$(p1W&lI#e8ZU-Wf;DZ-mRc0nT+pf zqjtFk6a&As>4?hwyN0r!bXW7fUHk=SNY^1Scj|svZ{YGE{n()8T6%!oElR_<<#Vb^ z;&D@%jTIJd=5fY3boJtV@4SfwP=&yy8r2e$Xp97^CAY9li&A+~I*nd4k4}_ff=hZz zYaNo8UsC_r!2)aYszZp6*~(38Hy0&vfh>JC|wey2oMwZ{6HQP6{%=&K4>ur$9eMY>SO@m#~? zI4M%HyV@??%G#amaZ7hudpt563SyZuvoP|W_-(hQquVRJs0CT)n0xyNo#82!duRSQ z3M3;a$)**^@b?E)Pm+n@nbT(PufcTay%B+fZ#W;jHd}vA>A!KgO}+HZV$|$TI3HIJ z`p@aG7;!{CAIJ(=aLZE=o^89bWueT##x$cgZ$x-~NJ`9jHzLJdv|#CO?97^Jt@v6s ztV18_IC5J*&l&-m&dmGr;Rwz^b8?nQE@_YZy{u}+Gi0!1*UTm6$RDLdgOj@AnI7Oo?Sk8)1SXtjd?nt$(s*ak>mJl zdtOQ3#hb+f52n9h1JXZOMo_VMONN$}ao5Zz)UkfW6o^S;BnMhB|4Cxj^ObF5+ns#s zLCX|&tN1pU;R!O)xrI~>5x0Tlv}xjo!}XvSBG#I7?P1wIkLLWmE;@uvZ6HQGQ!`sc zvlrWUmsTs+zvKs}dK&wXq6Yf5@B9?GnRR)Pnp;gBR(XEa`Ls^@dpl8+T)(GZjfmz! zFX{Xfdlt_VNa5=!QEs=hapk+VwrDlzB=@@r&r6pT#dns3?2-ITf)hIZ%5$gw>>wPn zlFd_Q6N6eAHc^L`Z|ypGt>-Pwp}f36>niV19d)~tjJx$<^{%}*^6I|R;O_A7d|asO z0INRzp)6&V%U-C!vsbU=o2hr-$_w&icyp;8^S*fX+TM`rk}akx1*py5pNDTq=o=ou zD^uZrj`?@Hm>+~Svmt*}B9rU5g7Ou$MZ(dDU~ddo(qTj`VjbrCTBA&bBU|%e#bJ{Z zMG&pL>%?z`@kux8y4YE|kX(y8=cAFB4W1*@_ueg%=X}Ri_obr|LqlxYBBw^!`I@_Y z4$8x01C>Y=CM(O~kC@YhE!YOKt{!Sa)6;V!&GD2*X!bZmLDH2Vm*#z~1}7xF7L0S# zbdinC;PIfGH|=jt)3e-&8#uVp4sIn0eRCxl6B7ZlCzbcM@I3_qkpl_eq6XP+y)aMU z$1xRH^O@p~3Mri>AK{@7%`B2qtdgWAF^deNMm~e~D^*{xh=EL>@=#!+A!ITV0y8$T zzYIAkjeg7p!`?zH>q3^4@|exqU<|-FXS*DR1W9EjdPgKzjQW`6Y3Pc`q(pty)q{m` z9<>vyqg#T?!j}2H>o(vC6Cxy^wl*{ih82mMbQH~cK|i4(cDp*m#8dD!VX`|*3P37~ z5Ryq~J7X0%Rjc&?7dG`Vca!f~_U*I@%wXhnX_^FdVJ>@wZ2znA+sQXE*UHG5k=ecE z+|)QY&{epL$C2GE{qW}fm&D2;AMAmdWXNl<`SA;M-A=6x|8OcbE*}?dBLju zk2;uLrxn;rKoeA!j~2!8=DWaop?z`|;5AN0+MPrPeT#iNW0hQQZ7D`iEcV>#4N)h3 zHFWhd&3{wdnG)fU@o+LqFIO1~`+gm$7+5pwvOrT`(xMSV`SIfz%`#1DyQmtxw^Bri zWY%D59|5JsZYnD~*>w}8nIa-miq!7`^>Vi~a|I$bUlD_0YgWSQ>1Q6_EoNKmlNG^c zIJO){7{82%Ac{0qn^eKPdu_X7MG=+84RQRR9sd$U6+G;l}9?Pu>9dy3tll@i0Y@95RR82VLhyOW^S z5HXdgnZ`y~rKhp!-GjZQvh8H1XH>*X0fYS*p|8z!lydpF>J|UyhLSG{V~MqNFOy~C z{o+*0Mc?aa($?)~eZbwc3AUC0M=^?Gst0c6xGosOk{l;&e4^o(kZ9ZIrxL0fFx+;4znt#KZryMN`Rktu6Xk}$CiYn`f1{nTH(x-*uKU(?5bJF z&{Q^tcg@7p8rB1$V~Q}m#t3j6`{*23dAbHu*M5v=bB2ty`iP+vxxsj8F0Yk#k8k`6 zx~Taq=ZF4Nwqs16HUi((D~2u1N=$?B+ zk~vI~G$ftkh@>cT5oiZ@7BDfK-{h`qcRj;BWZxOVGCc^eMlc-*x=?2)d+NSF<^+Sd z`r$wo4V}y=3W2PLI@oaI&7!SE(S#n!HtndAD}Y@4G$dd1%}NZgEOLI-KY*${j>_&A zxay_=SNPM#GG;xN62jdCB&Rm&2TL8EeQ@1?%QDEnS3qnRAt~w9Ql(=80#GrCOl??n zwb>Pm`8<5$=+CJgh|sD3Dl%FgrVf=aPUo;^jN< zbgJ&FL{;EfuUIKW=z7@5(H|6dSb8f_w4gS+r4_s<&A$5-t%n$+;XYCQ zVG1RS82Zi;_!%d9g7|R(9HVg-&w2mG!pA^1UTx}r8Wa%=Id z`5r<%kKSv#m+v1NmhA~QvhClUtq=;u==@7oZf&;%GI{-7e`}i@l^q+B>6NO z$JaH13Zlz|*ZWvDc3#%q|+=xaFwx1!TAfqFR zzSIKpXI=B;IH9YBd{G;HO*1p9vO=Jqk{c|^5GY#1>qxA^^G@*#5x8qT5 z*2TW9AI{uPpVz+`--?7qSdr0}!=v!etH4*VQu>1L+ljznH@?F5M;nD{PBPfQOa7{b z7H|#-4}HY??^m`B=X2V4%5o{uq6xO`?$*gc&mg?O3WTQxYyLs%+Qgj;ZnE7L>B{cX zNxqOwFPq&ILfHa&vEUZFvFE_Ek?+B8Wd zK(UADjE+;l$xPS!E>KdobQbYY%ksP0uqvJfk45ThJ9Oos35Bi{&!e^v&$-uvHsCW5 zRaH-HV(F|XqcxX?*1OakCpEp(-lAq=$ScuchA_XUtJx@X*)6+^7jeS|(d2|o)5J4= zT7x1B0|44hI?M1k!3JCrzkA?Q6~w>~XuNP=Gi?!Mo1Yc^QME5I4M^!)K0tY@uN3NX z(bseOAG&&((rmls+veV2_`1N8(Z_qUHY2-`_=8M<76F|oax~7Mxcjol8Z&4uyyRE% z;oMiN{AuCC4tN6SOhbeqyElo>y3^K$=tZOsj-tk(y1Olsy_%9=8nN<^a|M8l|JNa( zes-M@m_Oly`cTjVWi`{gV!LdFyz*;hq`*rDIt)#UJlw=&s0Aso z`|bXkKafc_t4eJknj_U_x#FkB^_D-~VI7qC+dJ%vQSBw|%?GmTR3>*j6>TUp7pG1s zC{+y1=jB$>r0af-rLrv|&SHJcr+;02 z=5wQ|{@Ua3F4RHXZ$_a4(&SU~)Yi$$RTrGQplRIB{CYp22QHos0UXCTSttaqHhgf) z`>9R$PrBd6+XZZ}j!ukxrCv;S(tLXD?k3y_Wq%V(&8n!84t9kZV^UEQmB1xLF+?z! zb@M)IhY7>s`uQg5rP_GHJD@durQ(-e`^a>?v5#n9<&$TSgNY1Ejx?tOm#4V{U7;3v zMBjK&5_7yJ?FLKCj;8U&=`^yR6oipf9H(`&cJuzF`xi4n62Li9aGNhDs?PQ&%8b5d zqv&xM#+5+IGfg{nhF62)x&o?J?uJGFLXJNH7rpGPtED%$y{KkcJ?NE%()1uX6m} z(Xjq-bao=Bd{y;Wu_;)526>?jX#5;ulQ4BtxE@_{I;}KV=?gJ9N|65kLpTLKynB)- zb)OCf`$=hy-7fSb#3y-zjL9bW{c%%wp>k#kWl>8wq^w$#S#18wD2U0m(pMr$2TZ|W zD^oS?epPr%bvT~zsC-E5B zc6Jw?CyRF=E2`-+Cu7q72yl?3=+E zS8N~SN9{(57qelhrxbNUaS{z(OvuDC?u5rIz3?u#c;Bp7E*Z|!{HngP3kRI-fuD;z z#V&HayCb4;uPOtvdBMZGU-OE05`QlGX_!Z48s}gXCoL`4JqE2(TIsm_~Uqhn#0yWVUkr~F1`1<5LB&l(5^^LyZ!xv z!(b9UYL`=`ZQsQd)m|Ikd{FV1Wf@4ZEXrJkp+)0KgR3>J8tHmNuKRrMI{QDw9O{dI z9X)Iy&eB18=Jm1A`p181V`#(jr8VaezE_nM%j8_~y^lFK+a1u*=uDGc)Yf6O`BwT^ zO4!=d4$-@Q<6kuO;8AzKzziaD1;v zaP=1*VkG_r4$3-VN@f2!kYn%UZmZu~0m5FaaDNQEah=`~O(uSR%~{=y*wnVo09EHa z&)cmQ+~h-YJUrV>i_&)5gide&VYi9H_$B~z6eqOFM|p=mg%1x%p_0N~L{r$eRBr=K zwR!k)CwZU1lX9z*DH`Z=U<~_olGN$1oQa}{ZRx%K@B&9gQXq(TnCVJ$k^J#BK@P^Q z_dX!!iVSZj%X`(P71D!!NnQeN(QBqPuJ#^+2+*<{^#q^_;Ft9f8N!6k`awU|b;$li zvyomaKjYTcx%184I*i_+=UELxhkwah+%ndHAO{%Z>d*L9&)Tw>lEhqjI2dtA;nEcg z*z*Gd80lH(C$Ou|W<+(rDVwm3De*}5ckE+w=uW(Z#<5RBwBQ*8XjxE zUTJ+rGnKsnu6j^T>vrspb&XQkK`^hRFL%;Do%fQkU3=mMVKZ98N!727zlD`7QW1Ri zzpkit(*2-fI>ux4LvMd)>uB3gT+$QZx2zKMI(yE=CsLy7ejLHzJ`=E%XgpSi@?cv6 zE%r>6;#11J$G`JBYSqUdHs;Fu95{-T+MX6Kp!TXZ@)xtljmd$PCK>@EGms)fQTJWN|Kpuq2sh9{j_X}opdEId zgcp_H@5B&!Era~&EpEy{kaS`jen8!)oP%*V9c$5-j2viq)b!163em8tYTL^HPH2mR zM!`qfKC9TMC`AJ`;q~W7>CM1<55=CyjwFcwLr9I}SptUb8l_;p(NG$dS2+#$6kjnC zW^vGhD7i`rG*Nv8Q;|wSE|&SC&w|#E(*%A}Q5=C7)KTj;fhf&FD;XZ;S0-1v;Rj29Se$sFnAaP?HQA1p!1>sCe@(0hm+o|s;n^h1k zk-S%hUAv_>0h4<{n6M4evz~M8c|~ZMHrq1Nn+)aRAZRRo_ZUs zwqO!fkYmt0KNx@I*#AS*J4I*KbzQp^+qP}nwrxA9*h$51yu_D-rP{I-~m4#p19V*0ci-AcS3lYNx-@lpuiyalj=hVNUfO@xQ;x*{;IOu+J3 zpc~gXHMccj4UoEEN^xcxSTm>>OG{_exu`mUs25kWf>MDg+;#~!!Rd(6jb<6s#7fd1 zV#tJ&s!vF0xbAB&SNec#To0DB%=VDj#DaMZC+BKD@%UAn5cI`Vo>iXAoC8aXx^q=@ zxq0D#K&m$DXQi}OgKMOE;Z9yO2d9q8fR&}es0e2=$0kwPvEEb1pvu5|Nh;pGHZ-=S z$Ab*WCi{i;uACXE$?KponS2YSAIr~OQ7w=uA#^fiOARP#%}Y|1MKo2rsR^rHK|~f} zSQp0nr826W5E5hHOw)Z;=;`#smYB=!`eZQU7llF1QDRhcCB~A_&6V$1Rh^hh-s-l^ zmUp#Qv=B%senSf}H;qTj;e!P;=~)_HAa8eyTGOk#m8w_SM|9yc9P0rJY1=94E~HKS zUEBmnw&thT<<$gh6+Bg{=HAJhk^jY)NS;^iZ}9Vx8!I~mx?kjUG)&8yr!L(QYIIaL zw;seE>kO2aq;e>qSvjY^Hu|%s>r9+6={0Q2zH8L}9Wsh)sY-28O4a_O_b1-Wm1v?^ zHtP3ZEq}gHUl?Be=2!jnDw#C+!scjefQet$teG`WBJ3Etu2_mJI8^_-E~+P6wP z^F!+|y6?{SltGIK%5n_4x#e8s5?mTX2@y@rX02}4-2TS?bBvIA7OPY4bpv3?H8aKN zKeDLsYu3vhDV-IV<<5Cn*UOME>*sENhfGm&@uf`3e)(^6`G361GSdH$9alA^bzRmI z>)GF>?P~Vj8sUg>C><*cz#;-{W^MSdi!-78tGq|wn6>6 z6T7(b$uOt|4HdAcsS$w_Dbd&mB?K(9aij`=O}0F20DXJ6jG;ds$Np4>}G3Jq9stsv}zio{|d_Y!i-~ztN3Fy z!BSrmVp$^IraZL0vkkFU)6{m?0DZUm%RHij&4qz4Zn;AfEQ&gekTyUU-lvMpkEIS{ zV}d_z!2=+pDAHC0jzUH@DxYgIC7Z1DIU&xjP?OhBI#p^<5%*0Fd{ zzqe}|c=tZ{5TOj8tbj_&28q)bbfTDde4R}l;<`wbN$wju`#Ijo~0i1KnC=2RnUz;+Ol>i^<8=teL!&Ur33aOimfz&oNlb!XD-x@3tp9NeMQ1LL{3ZzUu;bNOuYNmN8xWLf zlHX>jE{Ycc{60`qo4@INQ4p+Alwb3>q)Hc|P^L06PaUsZfKX?Zkl`fPUUKcOi5yJs z_0}-0vL&Bf^i}_3@+KAmyFqN8;6DO8mh(QaoHVIs0Y#)YYu~J%H4;vxBPL>OZd2VhjHE^i^1F~1&{pseP++% z>WkJ9%Rdof1tsEx#^zEfb=C73I=w5qQ`LP*;Rf*#@Fd7gT*D>|apxvW`>CHa66k)G zHq~!>v$>uDGBd@U=Exf2hs8MZ#R{=i)IXw(bU*__sUaj@{!04T4MjNCu*0T=hO6sf zNRgE5D(#eanB2J~Cng!so&5e&JO0mVydWMFbEFAPmfWivS*Zpst*ZzYqMad*oHj?) zl-wO|QbR0pBqdRYr7qRJfxawY&wCmAdtyJ}RXtT-jZ}XW!yRQu ze9QF5ZSO8{`Ocy&U8w*+cccf+OK>@piMItGoEhQUpt@ay) ztu*ZhrVC<{rR}0lVxwTFe~(%SWFCcOb;XezXVcJ8dq}ASr7^l`Nah?F&NqRP=m`6F2@hf@g3hJJ9E!ML~bltP9J?N|@}8i4?c%i0p|V@Sm&5;ciP1W{Duw6fuE5 zcsBumTtGFWVShSEhF?8b`Ps6c3^74h?q{aAlYt>ok#p~nEE-m0J2i4AD3|#B);C`Z zqP`_@t4I@&q@c9PXNobOCtC!jrXF2{lDfZsM*Dee9qD z<6xkuO>b(gQS+cV)IZio-lP=!;S%jqMtt8R72;?9jp6!nyZYJ4%*Q(7{=LJ)s1c+E z_;{1j-dw{(b+vbeHbN?Rw}E~@zuY2PxPx-=>;h~g>Disgqu|~UN`VOZmR}9a9d9jF z&_Jn>8^4}o<@u#f1uA!oOcc{`U8?RKPYKp?5D{Ak_p^+izxab2GAeel@_~QJtR5lx ztd^kIl&kjG-;$0ZJ~?`{@U9SA5CKPS=;m3T)3GSYtfBf)eZmd+&r*j(D|P|ssnAmX zFYkoK9pSTLpltiQ9+{!AC2O!$7S3E-(KCmh4*ZEu;l9C0#Y4NGUG;~FM|<4=i0*2z z{}_BI>as6fQ&}vd5sYT0&oHfjN?*o{VRCU%KEtVw5g5({>ia>G5DahzZ!!aYI9|AP zJ=NVM^E4yLb@DGM38ZOxRT70O<=(jZPcNld>L)C9d4pgbPf(-J{B6d7*>j$2yGUa#e(anPvVj_?^oRiZQc>xruy5}ePKABLcpttGWPg4+@MM<=!8 z1OA&&@3*GAL5GRt(&AP%@g+ih>)2vy6#n2Kh$gl~2EOb<mA<^?}So%jDV zP^@rF&<9nJwYxNm7XoDKrb5H0&qOt+7)0*&Gb{kHXaeLUL#niW2txNb#Ld59b;-)! zvmOE95D6qf2YckGAwyV7*lSGg2P9Q($-|Hq$%@1oYAYU3sq$?%IHcwZRB&!BLlq~y{19Me zLklt4>e?5?%OY1+2;xJBkm{dnj*yaOITj8aA>+=K!8Efq^BFKCK1;-sG(`ytI2Md@ z#LpJ)5Xd&vcyvtt`eTd*_2k>xh@+@#ixR9)4?HNrDS@ne=xiEVI{EXt6@Vf91gzb# zZ_JP;p5@jw=~&@W#{5-phPkg$&(w3oVK?1{p5XPF!xx z*X>3FdR6nv%*~v|@=&{zaS@qGvM}v|>jKx^!qV5PrboIUr=m`n*)di z#sVlkJpxgqRwhw%nf?1gZ(VoAer97Iqf&EL(dL3TDm{NwA?*CucFM2XCq9uYDZ~2Y z+pMwiT&6lzZ>R#>JuNbll>Uq|PP9b}GReJjTKqQ{yo9 zs=EmGbk%G$#ojK5yZ`qyiW5hN0-bm9AFkWoRd{khQ!M55<#d^ZJ))VXvRh|UKssPq~1aad!sZUst)BE`QF60aq4Q?vUkGW~t zTYipN)o+IYoQXfd$RDdD*WY*w4iAzQ(HnXiD7BWm7tuoUk?Tneg1l5uAb9*z^&1`^ zh~~D96Mm}3&%&Xe2bT$%8Z%f&SGx$)y9cX5fm4BVGrQrV9TycyMNM#n8bmM(C?wZB z6OKxAMv})RrK_kYK`@3$gX-8A9Z4hb+=7t%{TNHh($?zZSnP^d9bt_Y(OuqZyyj`a zz;4@D# zwkGuIwCtjaHWrbz(qP(Y3X&v;9M(VXZv28EyIDRfFr!T*8xz1q6Ii;|GG_{_i732L zR&+@?*fW5v%ftwQGgUd)>X5@1b>g|<{yz?)^@Gf%iIfg+-8$higQfoQW&87&eM39}wt3Vile}b^WA)KKoK1j2e)TJ$6VBNcGRHPkj(Q#9Xz;BR~zd)lHgpw1X zP=7`$FL3YY*toiFl`2xtZI_`^f~%h4kg_s5H1Wyx*zU96M~2wj*pCin#xaMs#DhoI z&z!R8w{eFTX6#W%w7XekOy%8WUknf4HqB$GPkSKhk!B zm!^_)k{kyvmX_viqRmwj!=WNi7zIPo&(mTK`h`Mc&tEhoGpNiM@6IxPJqxIKlEZl5+?G*0QKZsVbqdVx^WSdv=S&mSZYcLbW(S|^dQmA z5b@(iZW-h^VkHJPE9N_62C#>T_6x)+Z;M_8;hYixjD8Qik~gkotGt=YP->_x>R|`yEI+CRj#c#!3;jtj zscWWfv(4H1!Q06<5oUM+4aPWBom&BMlXCfy+i}%Bal&i7q;WXt0msBdXWSBV%7x$t zRPyDNy@odBVA~SL5@sYRHisg#K>0mX;&f~fa?=OpD3u#FOVq>^&@RumfS&5V$KcT~ zzI`3kmrTD?nN3j9$Zy-k7tBvnv#W|klW~ya(AnpHxT*7b%Y*QodT#T<`(TFWZjKRS zVUk~H^B%Euc&JfgcsWVw8bOjQ7GHdnrtl9(g50{?kkvFIh5qnm)x;LB4i;5Lwl$3M zCm9h8S*TY!XvZSfx=2U1CI=YK#c-SOasYIftw7#H)1zQg`fr5U@oh*z%(`)edBpISs!Zp;8;aqtyPZ5je^KF|j%1x2v3=>y z;iWGo8jF9c-$u%pgzJl}Bv-`;^S?M$RKtE{39Yg#NQ)nq!iH;XNqbo1D5n{tt9q$3 zd%-k7joLa*n~`{~jiq!%ivcDN9rs=$ zi#ASbuoVkOJx*~h^_V6e|Mr3zOldC`r`Qo(uMi;O+7nG+OI&nLUTB(8VQU4->pYTY zcg(FuWF5K|@3}MjWV8)?Y7gl>J2=yM5mYl1i%gHZM^UUeH+JAno=>jfCo_d7HY44p zLxgMX<_l*^!8+xwYq@IlM7LreX3?KU9x+k5%p{rgK@Ul>!m~{Iok>o8hQ9^KwELF` zjk_~-tZVXvG`qQ|txnOhD9GQES=p5*$-T>CE1i8#Yd>E+1r0}?ikn9ibT|V+ot&I= zs<_*NNASL#X)oy872PhHk1&gxj86^2jA+4-JR`+f#zm2%wWLJKLFPps%K*Lle> z{mPqgp=p*rlMYI-2!z&1``+wO)^T!blAp|jNwT7;1nXq|?1ogZ=&ANDj$4l97Yf-a zCxU1-m;@y@uoHKtBUo8uw^30D2aknAX$So(lU*mWQWYi{2Z4~H=9*unHAXe7V)Eav%9G3>#sj=rK|>x+M5SDb6c#M4boe zA=5`r;-3=e7cu@Vzrq+H@Jct!ts9{QwJf%t;L*5QE(3Ca?mMenK4WN@+En-H{#8yhr6b?S!|9=T-kbBfOidg_{D(li= zX)0@$7>=)ioKfIE~ z>J`W`G+?XX>f}v%@Ds&eCuYA74&iAYL*KK{;MNXM*x7NS7HYWuvRZfWr>7|=YQeBq z^RFOgyZ;Xo*a-o*KrgsI}eES?(PT6<*k&siP;&hko{oWPX4d* zZ7&^->R%rPB4*DU*zN%9SZ+&jL;*|i{M=h)+LM9ma4qMl;a&)~g@)skZLgWQhiN#3$anGA%mxn`B$rw&%2S60y{)2C#jGqdD zk+(Y}BChWbM8UlbY_B0tmV+A>Y`<0M&5zX((E`VuS15P^7h*|59#AqwKI{lWf7EO~ z>Cws__urF7p~Xr<&}>IN;ay$<6;|PH6$p^zt)x_3pi^GA5HRu{w-yrM!-`M=(V}e!f2d+b z?3fTFqWJpJ*nx)?LaBUnq)RWG5%bN*K7r(ks=jfsn#3EKBHbD_|{su^P6 z)G(aL^ef^3HYJKiahuR zYiSigI_KBZV&~nxK^EYc3APL1=afT$8YS65Q!_4`O7c0Dj~>C=6g zw{QCiNZa(=KX3wZ7`x7}<$c{jD7%4HvHX5QvApl)*c<`hxZo2pJpYCACWm`R8_d1o z={U!3rW(@7Lq@)MJ_{taK_0CTRb}jRm|N%D=>NM+`p;XW#?DkWCf|c0ZWCPFfW=@f z8Db|rGkZ|n{kI9z8I^aayHkm`-JOgo3oPuGZP&5O*>Prh0s*r5qHt!T%O6&eiMZHs zor72|hI3Tqu^%thcO#%%jVBgkeGE9imtYbf4jP1mVveU+6)WN3c5^6sqw;8T&`G^; zXoUf1V@HcXE^k!{4l-|3=Z2*O5y2Bf%jLX!=`sz9b~0o69TW@}Kpz;h4ZvJypQiDH z7T8aRi9Cs`rXj2GVhrJiP_(3+qv0+^{!*AjSf9?$eizjxw?RIEi!s+0jEK7Uw>SLh zNz+;7{i^0G#19LMz~)28wIlOx^SAvft-%SJS88Q4os-@4zSwanvnMx=jWjC`4KLeMIDKUW?{OYY`@@=G6_8`FUXJd=*K^f0Qqw=4JPl7 ztJD+Xv*M|-q45#PRhs!BpcfxgAQdMKfv|e>1H*z@@78Xmul;m3|qN!BdVv=ER*cg zmi%h$J|%vz4qbsL&a@YxREGNruv5hHFu(c;s>tojy=|L0X@f{V*&P!SC;;7<(H%M5 zj`UqA2XBw=!p4YWwwW24a=fj06x;4N^5}(!Gw5j?{vJweEwl ziXYaC*~#EY>XH&?Ig|%W^JP;ZAxH`J`e@dw+i>jM#R7vOHL~<6VwFU|i>_QOD?U*< zv=gUs>{qeMjfa@bPRh{lhG_Db?XNFK=+~Quo1?2@`X?-S&pIZ>fc>>meEXm`8xzpo zwl5GX@Mb(^C#Y@ zyK1{8<>&m8X(COy%H(BO#I=~iYuam*OBkp=ob4Y( z(>oqS?Eh$a`})O4wL@nH42N<;`L+S!G@qlGp!qkEyz;AYhA?U+y@%}ey}NcZVV_9| zu3LXZ#r{+cid#%wWr7Etj9Wt(58u&fKnTdzxpD#^M&zyFF{Xu!4;6< z{kmpHYA{D`IKMcCcCj@PejXvk(g)STd1tG2uj`)i`}Ei&@bo$V%yL!Xe6o1F=;Z}q zC#kc_3*di!J`Yj1{qg$n22ZIN0O}TGfisD_J(AJ#P7hhaDZH~J$^BInWAXQ&e_$6O zUb!lOc8%#DR%H)f(JW-n?xQ!{5J7vhO<3+qs}gLbAdP&0d3K^HZsBpR2P|XPyN`X# za7N-8+mD>Ur{1}gD-OoyiQZ7{Fv>YpgTQXm^=#bRv{|S4I8_=y-hWU=zz54Q%f*UR zv5Wfs=i8yd*}}krxDv)=eU~)sefYMe4nH2jueGn_r_$Hw(T~Nli4M8o5gb3mn3{M% zFqH}yS=Zg!xpe8P*;%U0LV2kyQp?APX4^x9oHD(}O~~bQ9}#0aA|B$Zc#Y>WgqF0U zrphHZOyb5Ce^|9bGT*Q*Plij%x=pZwq5H@VFazYq%rQlbQ|#g_^)N!;OH( z*eR>uKFf2+ZNHx3Uk7qk@ZMIog7PYFdAY}Bl@I%5w6j!x+dkbBJTs%HGLccMA2}O% zySN}!xisSCHL`ry;?4l00Po;ij-)>)O!)W@z*HPR|+7ijDrsa zB5oIwHd2+b_sk+TCVJ|Udu9qJC)3qb*6eP?&ISRO{xg((4iI~_vKSI!a0rSj1~B{H z>SkrYs7swkNC#cap6~S{nCtOUZ!JRQBON>!>RWCQ#aGY;THD+eJ;)?Lq`*=s50SqU z#pWb{ws!30Ck)D(%ol}-spDtEDxT#>ypN%% z9{UV@+6J4csnY*KlCW35#jMr39*(I9q>t3D{iqU3#f7F&EfM=crYfGjKwp8?ttG-s z%|Y&lEbc=LbBa(hB|kA`&idFyQ!ey1d*5T1=gK-hJ{-lCC6n(b$mcsF=v&+TM^m4>)r+^QV4k|E z323^^3u)}&IKLAI1K#lCJFh-dqD*>(qyq8gcH(CJy*x`cIuyXkRr?}7dzno8FC_Z< zSoJ-g1c4b$MB$W{xYQt6`Tm?&1s-%05FU1B;>)vP$5tv@=PkKIgq>H>=T5!T9jugC zJ@Ehk>W)aRlO z_}EztkA=*0VM^Eac*$FTzGFx@L;(T?~r?&Kg z4ve4v8P8^~F-%zuJ)i3?ekVpA{dR1}zvs4sklsh%e%`Q@8@q#e9oA=g&E|MYT3MAJ zn@uph+Ggaq2p=|L@101uU! z4y}c0NXjk$=h9zp+l9cr)B^lq%W82pDHL7u7%rxAaot$8nViTY`SpvK*}xv@tlRaJ zwyV7_5nMXwYqi$B*HK3?JgTw1dJ19#?f7dWFXZ)Cxf2>!(t-fDVzA!4J2ZTs*Abmh)9EES*wNofnuq{{7BG=tlfD_wn}oT=X?r z1FUgBj)erNLcU$Ja@F*D;%YWdB|>GSoG}MXu(UoUG9KIl_mKc6i-<$dj9HOK0JjHY z!Sy<5EyZ?FtWBUDJr~L3Z3+xO`v>B8Lt8r;oCHqBZv33=4ipaFXMf>0R{D|5NO5ye z-T8A!#%PEq13!q|P=wBBF6)k2 zB~0HySx;Cu6+!&CP$#&I;Os5wjF6`i|1u|#&4e9;XR?DB2Ct(^zWVEDk5Lh;00^WhZ^%pP{lM}Mq^Fz* zBkr4}TC=Z|-mvkEyeN$@#3{ok^qCHTSJ%=fe}MC3mb5V?W0w!nCeJko`{5dhU{sJP zoMOkjTUYk6mu-PL@pYy(Wq^TOg$8Y(PmgA4Fk_78(veZq;t&jcl-FSgGdMu zxK+ccdF&s;S99`CTiMe*zmqP)-V0r?$7YL zA{-_cf!AU^kxsqOJGURTvCkq&QTUqt9#oW9Eo_JzAA_q_-+yxN^F?fwZx4~D!J}P? zq0InH=LZ$$`w3-?Zgzb(?qg6k=brRQ@QCb07$UmtpPRRwz@Xbv!ssh|6r%M@y398} zD#y=31mMdoBS&HV%^DTGZu6z4Q-7PY(;(tk9h`25uY<>Ij#C?~6CoF(RN&i>oTSuv z+~ZbrESdPdV&-n&bu&u&Mjpv}Rcmh>!R_anyjx94<5n-ak+=;^C-3vldarHO-%dCI zpMx;Ke;;i2*{B|PPx_etB!R};;zbI%V$Ie(D2=ME=jc zh2^baQTC9&SGN-8yx1Ugx*5lVPeO#xTd&lPU1zPp85Ag;V@;0@@fxgN?ia63za6%d zajd<78!X~?mJ%7 z-P}F^PX%|C-{`V3a(ZsNAZW~oHE+}At6_{AkF0`EL4p0jFLHe{Aw2R~dHg&vmklde z`vsmpY%RE4hijGa zcsJh7?POy_MyUuyb?n_|L@SHMFTTs6)%9Gs%cAJh{3z%ftxb2C|x8*@4< z7<>zL8N*km(3rs*3#XRrLaDL+!C6ep3f#KP*wy07A8ofcc?Qu0WF0#WS*+wo;p8?+jd`;ebG<(xGCAKEA2QMb>79&&0^j0XFjP#4a}iJoHxdKep(L!!1pnDr z{t=~5^qKcV80+!p!nh|5c~UL=!Og=2voD#QJ^?nOR*#p*F;XpC5ut2mha&{w5zS8D zsn8pWZUfKHG7UxD^nuCD_x<8J{`z%8%QgF*LvY5PGbk^wt1W3W9?cTN0nb`)G>416 z>}M?Q5A7&dMnUvRw2QNGE0D>v{^sXTF2a0&ykzYt!lqSk%ZdCVnrH=Bq;0e&mUtg? zB#89@d4?gJp|$Dkgixz|Z^VMF^Zhy8ZY*8LEW&?EKp$k@8`PSf>p>kXFC%8k-E#C*sw9F6n*1V>^7!r@NH zxREBPXC1GEkf-mD#UKp{T76O>a{DvoIuF-dk5Sc6*zww> znWb`GbDA9#YrOhqBqYSXP~|)BNV_x@0YN&8KgVkorRL48u{3vaa^E^rey%Jg{QjFd z%7OGrxM=y_Z+k9G=f_)aIA!Yxo5Cajn`hm}@vI+>qtN=j6+j$tXf$kb>}+~g4@6UJ z^Fqai%`*ni*|ZI#QJejlDI$^C02M%BVv9umLG@pSCS?YIlgp2GF*A6}g2|EX?t{iq z%JbAP1(i`foG%qtb{piFL@QhIh;|;ky}V~)mTSt&{&}tV3P!dN zdJGq3n&NbOJ&i-v|4ZLz-itI#Vx8}k$%ACx+a; zqQy+~bBEaeO1KuaYWQACQXXIM3kJEwG)_M6EE#RCS*3Mu7uhxQ@}E%zO* zgr3AFwLG+RFT-?|_Nl8L$rDZnoX*6=Fb|o5aut3(> zgZTdQ&NOpoo%)q`FZoc!YJ5}2{IWWv+fIv z#eKb7lhH_gRlpSOikXQ?XkM)a4~psCg16^pTxlKq7Rw08hA8A`gzt9((zbt9>A_%c zZ-00^Pz;dozCD@J`+1~AVG|wO6l|-*40VQZF&WdeR_cJ=69| zy7}>ro;n|!h!LfsY377zfN?W3i4{IN~niTPZl>?l>X8Oq3RLZP|43O6%U%+xbhIb`qBRCDL| zzI!}sVo|oXu$G$GcUDu%{W=9!w{vSWut)WQg~+pao_W&^b~E*7(IH(^F*bF)H##Q3 z2OO7=#|7nkC+(6)UXy=E_kKD|^ZP)?Mc+JXr*Xfb)R?m?A8bXW5sh7~U%Mm$MXhWe z21V`ab|6*<_(1g}OtTfXrX(0#jJQSgMsUNh$?Q-iahBB*!3KTrw@Q)ry-X}=fUCe* z_do|xD9rr>xY7N29?!j?tY@;jn^J3$AQPin#Kp7 zT(qoGD_Qw&HDS_?SkkdJ7@nqGKCIk$C`-YyZ?)XuR`1Xj+m#gD>s;>sn5IYS;l&gK z_mR}j$V0l&{^0=|_Yo8nphIXY&Vo~CjkW2Kz7OWu9}mtq9WVx`IM`Ur#BQ0p_cT)3 z+c(Ix8SUZrdz##BnU@mj0CR5?D*#QYxi>Z|3liV0UtIt3M1#^mZ=u!k6ZDRVxvr+Q zhNK%eWE=Qk)#QfV2YTcBOwiR;yvXRgJkGbhXGgW!Ez4QgTUoB#U$((1ii@S^#fS+W z-Y_33!yLiI@9(^@#yNo$Gsg)u)3&C5B7>lOKdaRLmEJby!`Sj%?TC1kJo<2ZdlyoV z!|Ad_?N?IDix(1o=aExkN{9zakf%m-uPW&Gqxb!v>p@1QM5g%nm9+q^oo?il$zsnv zS8n@SgPT9&<*uBVzj!d;Iu*c+_`)U;xk1bQchxvy=6_J7!i3;XwkIoqikHV59Pm3p zjb>8bL~H~mfc5$#cf2(KS0j2uy=R#Q-MNfny{ zMUwi*DoVS?TlbHT2_nAoUP~kd9b5?H{to13+2BRI9yHEwv`u$jT}Hy52q|us$`X8U}rO1s*4zndkT7mnjv*dtLw?-8>&m*6ad@YdYWQ z@^fy3w@xl=d&9Rp)(I1OZ;ITPgoOSd3&1mdG4gitaBTm6y7kN-DL%3|%#8Uk(3r5N z=If6uPtfrdBn6Pl_&}F%Bh|AVD);!JWF?4AJ-!9RTGcOld`0JbB&;)&j^iX}nTCH+ zgDZ$P6fjxYF*f)z+^qjmSbg;W3c7mIFou2ryY^)&+j8#=1Mnaw_7DuyyyUrGy?E^w zLNNe_*3um?4HUuI=PbR!kp#VnHUqE0fD>3xQ~?KTSjz{(z0(8y0q42Ew!KXIzs!2=6f341 zY}k}N3%%D=-|suO<8)lpbvH7Y?D~5@27T+?mi2R}Pg96?z!<*cJGIN~PGR2zzZ&RJ zw`9=DZ^y>etsrMItkOnU~`Ti=5Y z`eqMKPEOMZ&v!J-nFDUCo=TEya!Pz4hy31w#KG`*LahA!DF4P<2LFmZO6KhYyr1Q4 z_gsc~bOk%2fSb|9%Xk7{rgHH@_u&=aG95K4&GJo0mnogfa<|R3$5CYvT$iQRnHQ1I zhX06@NmR@Uj?(XoFSFS`K90)%+y_TW-wQYURr7c6QmCC%J2=uz0J4L0HvMmVT9w~N z>purxMoo_ELxR+&^yvFbEtXrKmx&raX1#+RN$mT44o9a{b9@8g5w@>}N9 zW=>OAuXBri&t~?^MdPaFB!hi0jiGM#72B-hD3;(G4LbL@W+fIH(mGbb=M>e==-aha z1cqi^s`$UIp+GnNPQzi7l!T#>Mj|Q5Kg{K)I{2(M zfm1%0ICxy|$2e-vg96`)kC^do7E}&1ty5ZVxh7m{+EMhnw-Z&T-0!|rrk{7iCsJ^q z+>f&IzI%c{W0|E2$|h~hCuK_H47gv4Qkcz#8>LNnt^)2X@p-e%SY&FpiZU_`-5CR% z&y`qSqVMNX?&4LK3_XvTZjW`;6o@^TOB1I3bA@}BLhok?5=L_{%_vygES4R(PcFdf zs-G$N*`+T0WMV=UH)Q{g&4tiA1pL)A)ASxpX`=02Y<-i*bT z+baU>%Jk#j%6EcF6fBoabY!v^kHSio_8;yOB$VpcIy{X6{?P!phu;7*r*M}LAXZVc zhg$m&a+JlDfgHf>S;|1Tfz-wjS3ei@vd_C;*QEZM+i?K@O4a}BF>+8dv~*^?vx%+2 z0=-rloSNhiESP+Y;e(E-n!`ze4dK)0g&}Z(kf{Gyunz9&b17f@UmHGFzAsA8XT1T- z^PY_`Yfx!m0IbpLi6QXzdG<$x!-<~-<UkBe;C)W2&OR!kvN&xXrp}F+GPr>n{$cld6QIsEqzC}&3w-h8%~mG&i$4V39Jur--}0z>MP5dX;Q@1=ks(yvmA!eHE=ey z>p*N9$Sm0p>f_hBr6&B1-Vii6hbS(mMh-*0uDOvvHGYS+z?HxNtPzJ7${gWiwt}zZ zW6~*)eE^%%(Bt#_@JN;JbtkbrLpQ=NyPEF#P=Nol;6Wkp5i^5@^Qgj#8C(wgi9)8& zq{FIRLPFu+a)j|ub`S6Cx7#O&mO)It*g;MwZ#K)BY%8&u3gR-Mk5Glaq05bK8#?;@ zmtcwMO$3ey! zJ=1meXeY#$OQ}Jlj9r7T8^rU8&f#>ueYJu&P#;#Xql&NH_n4of1fJTknJ$9RYUc}M zFxm@VfT)Q2;Wh^6>#c-(YP-=sVWOwcBdxl}oDmZ_#b4}>a^D`PdVULkL>IvCfFVc& zZ450VZ49kLLZA0SBx}QKCHLA-D;ah79vdc;e_Q{1J#^f2qX7?JcW!|>{9x<%_-l~a zakABP6Z8LL@2!L4YP$8&KnTGJ?hqhoaDr>F5Ew{scXycqLVys0I|Lg%fiOsLcXyZI z5+qo#0Kx6fkoWt(mw8W}KW_bgr%v6vRa0bFdS-R6XRY=0TD`mX)>)!*TcFgVP&hs? zKL1VR`7xKZvW3^-Eg7Mct`p9N{FCDKF0p_?)!tf6x5kMN*o__Da{jd}8nH~D zaDNdUdwCcSyS*I`yLp5myen+go9FD=Ys=08H~nDAR{|&(ltaHCUo3qoi;#Q)eI2sf zrzV($U^$xI(gdWV2ROiTl?i-N;x8K=@4^15|wEWOP_^a}OIs9v);+K|XKhy<>q2+FqleVfzu}2TdYq06l-XL8cw8o^onzbCT z@`Ua=yksSJ{rT?ncoUDqZCZB-RePg(g`29ST?M=AGXr^8a2LgU+E@HYk-~?>YfZH% zFH+wyMx=rsVTm4x@*9s%?l&I}P3O03m9NCty$knM6d?BSwh#mG?>{0JJ0k{VX;7!` zw79Vg;+2uD(U=ucsIRmGU$yb5I8+f~Z+q&&Ltq zaxp9sBQbK|Ij3f-eb;j5gLq?lCCjq3du0ah#4pkg%;v;Qg0VLlu|G9c5qz(E!==Bo zW;rpO3?+Nyma~tA+c+u?L!fqEq+vZkR+|O<=F6gqUj}=^iEtEe z#Md8Z)Fex*?ZWt_Dfa_NnE+p6O}<2~_q;R(NBGpJk&G0-<)+V>q6?*RUm$;_WgeTL z2egyG8~o8z{uqBR1tGx%hg)3C!V`&KDEp=iC#~Cz+t5v{kGhv@x*~G5*hp39fsCXl zM1PR7{gBT4X@KHuZzj-#H6Ewz@y?=ya4&&vi;*86SHJw&Q0|NFDHoW|fF7ENrT2VY zN?t0=iKVl`n9qjwm#V7oUGZz~0dw8I>8UN4`#zzt zmy)x`T(V>{XL_Wh#^##!N+*LD(Wuj?>b@Nq7_c)`7&qz!>N9>}xk4#Y5l!X}@2lkC ziMm;#nqI6HQXvcIBCmKbSWn&`xM_ouq8E|)CibP4z;3xX*?NA5^hL;uY4oe)`@OCu zS1dluVI$1?>~#-Msn!LIB2=&j&Ggk>%PJd0Y_%{pT^TVRB2Qh#mAS^V)(#rmdD=(k z8Y-{Nm5PQIAGTw>kqvEoYimCFF{{7$e3SSPoA=XVStA#2*Mt)nq5egq&okswTfRMm zi0KNk26@Ik^hV8MZnMF$52<2Bz65<9(N+3lJE`OvwEh|+0Zq%W?O(vB>z^pD2*ZCK zDSnQ3i!nJkmwO4)xr{D&iMUZ|pCm#^Bu}fdTcY~Jv+*~^ZyK#-DG{sR+KR)%Z|yRu zBn%@lUBPC4#qVvWyAJlJ+wH4Tc;Y$5Lb?C~4rj8!xorOT2#VTmTzx6RLHFZh!n|qp zzH}d_w9O6L*Xj}<8hiDx_U$eul3y?lm+BsHipUxSuW~Xl029nvS#b)w#=yl5{VRQ! z18LacRRRGO8+u>t-E%51ml+!L4LDm#12(3{F`K5&4X#e$K(L8u0TtrYb(eLXis`zZ zQfxX<;Iv`g&fG_>DgzmNk5Hylj1mR@&3al6hiUW;W_sH+Gql!S@|BM-pN#mcHWKuZ zcYQY>(HNvwk#+u^lco2PAbD8tmn+weW&DLNt#8Z*6KLv)HTEG3&36qpcMYICXB$Y| z{RzvAo_J+*Zl<~xsEVjzgVo9QYqr;NC_;pV#Z3IJZmX$_;H$4jex>LQ?hs1mPYvIU zUwTeSr_kD1q4waWi)c~x>HMrR6&IyFKui7=JYR&Dg={KbqQ$i)JY@RWSH*O?53BUR zCRwtysjqi~wVXE4^~*sY3mt|<&2h^vfb^RUxmMczH*OTiJdz(%s?OF&nZU-^`6w!aJ-g9kn*$z&!xB^>4NOFKmqz9Z z`XT%wDiLwLWq9~6c#pKNWq+$QoB80^ZPjk{V4i-HJo6#Mvs?RhvaSSh;x$y%l`H`` z_*`&*!Ao)8a^8-){Iw-f6#CMJVOH)%>T>nQ3zdZ5>-_QzeI&)Y(F;8Mk+y6o!h*Y5 z^}z~)Y2?L7Mh;NY$w43fFa}=nY<35p zEe6vA;w^5YF3&tKE4(bw%zzKWTeP;1p%QJzXBt*E_@5W-8$sRXF^fs;FBkgTSiq5X+^Jwdk(ku?GL48 zI(1W&{dA^{@+}LSgVRdmLu0$3>uHxs-|n2-`fT)x;wYirvKeIjORv{NM|qKKVjt^uq={+SnJ86uUtNCx z;SWu_kR}9-*oWKi9&0QF;zK-B+ZgVtJQ*<+5KUkKfAqgN3OK>7pJcM}aJTz_Ijd>4 z7MAs75P0q2jm*^$o2|p_FKDSy&NL%NYbNcDYI;)l{qRhe{o1juV#+ny&T-$LXsTiG z@{_PoW!u5cfxiMLcEtB=4l|GG)Fc^ulTKg$)rQ?y)%uS8?EZWL<1=`PKi**W7J!?! z+4+C8H#kiN<2jxNzxYJ=JUX6lDV|v+hz=0sT*??>WER(N1SlEuEVRv3$sABthX7); zL;6YjmhQRNz4N@BDxqf;h))hyEj<=C_CPlz1uJ?8L=6NIB_Ds~dhf|($3Faq((5d2 zwSR!Xz37)+a&Z&pF5+jYt}Z~Yg)%a1+&Y;*dP2Jqn+Sq~CDkPPQ=d6QM zjYCdmG6P&zbqFFS`TWf2Wm7^6;HS9tgXAbNpZWnTNgo}Iw0da8*sxtt#*`Qp;b(90 z$W+~`#1c;#e3*`{&zPAq(o^>KgRpUX52<)pCj~?DOFps15iy{QFs@9CNn>LwQR*W% zY2xv1c8J28WW*=Plh#J$ke4#1KtSVeG-G84=n+m_X}1;<*IvFHB~Dhg2`hflAdf5A zp{%tJ4cHEKM2450H=27w*)$Q%O|c^p&AR5qHu6;BV{$-pM#7*>Mm zp-Cx0j$xf2clJ2~U;j>rYYG5iFl&NsO3xtO^BpM`*oy8D}<~+3aAmg*gy zB4T}c=)v*!V}om%DW|!9f4isgPuaa$lyyw~1p0Q#uA5heUhcgYp)J$W#^K|Z5M69| zvh8ZDbI_e25D!ir5@6{tKyq(*SM(d}&9kNiF?anM5@g!Lu-X|hU;PSdeg2h}muH#@ z-+L>cZUvphQRUxAiz7*iBk>~pDA58|A|-UYgdw_JO_Om$ph4kxV&3!a0AKDanL{v} z?FJ(7V??Deq`zTibku(>-WBL4@}bQ@%?&A2#b%pv?VN-zRHD_}ktuqPfDT{q8b6<> z27hyZ%El#_6W4~8eogv2o z=6(n|kWKV58l<2QcEeKm_ASLXnC)1k(mxv8P2JO5kaV~B7r#xUyVEXasfe&gC+kMB zR!ZznxOQ_!fGv6|>%>xodDgsbV2u%e$Y)IaCl#>WUt;|zRS1AO@Q*qvx(&2b-nqP1c+1bE!qJrti* zy5o_?)WtB1dtAcn{$tpdg_LM=VmuZz$feySCvegiy^kAv!e2g%^)pLZg7|b}Xour~ z%G0|3eYUaP*U4=IvUTp@xsu-81L`Tg%fTeu{5KTSdNnhkB471o95$hHXHO2|cUYkX z+jE+NTUA&`&m~55PGs%sj|W@qf|VXG(AoR#lh&Y)FNb^2R+PHp1$?J9cVO`y`Nj3X ztM~z)`@$Br@4Mm^`b-o0!s+oE@^>qRpTK--VbKlM5dKfdkzy0Z)os@ zgdIzT*mN0^P@Bq98`!+j95d5BR=+xm=70PyWnRv~7Tu64Brv6~_O4QXKt8sc6>Tjf zwUf8YBBVjsxo|u?Po?U_G0PI4yCy60s) z^sT$=G{9F&sT#U7YnTn~_+#(8m^XmxL6+GxlS_rg)!-*{TIIGq_!Y{hQzmdBb4ChA)(a%59~U=<^)iz? zEy?|J1vWsjITZbetNXiB>VE3KZ$9|0i6#go@8%aV0R}j^8Eh4ddPz4^NGBH=_fs~0 zP80Klma0>TuJ=&r)t0hNSWgZr2F<0U7PwQYvuRy?p;V0J`9=hp>q-@F5{AC7W$DmW z%BZ(u_9L0}Gc0=y#ZMKix_wD%_=IeLX~^OaZU1o_DRItL^S~Dj36nnA=0U=vvXrK> zz=}rbt1&ECJ_ZW@GTbAun76AnepODlvbYKqDoQXYBONj8Dl5&4vF&oInGVCGB7Je+u7)O)U}AEdaizswmYukTT?t$ivmV!m^|YSr1n7BO3v z-ZYzI!0r5-;%qv;7agAx?|Pbu=j6S}&2qwivtd!(Kna4;K!DcD=?9uAN5h_^)A}FF zAK2?nzcOVwi+v;SJD`!_$}_X6pL$jq*EJQ&+9>Q83M3&(D{7gjDUv>A(H>&psVT+L z74R75z108^Nr2&ywmRx#ufSu9}asH5I}VqzE+crf_m!A6s{2<6QagP zE<6l_sg`d2%M}<{MX6?{t?~ZTi37k;Lug|D}jEdrL0ZL%p`q#WJPDo|BIL28MrSY-(Wri(XoX!RZf zLp>cRHKhe=PMtPu@*JO%R_22XSBc6o`?%vdjY)@irksVttCPK4M=OL=;;QJNFN+o} zNu?a1wB;kGlciSUq(~FAOPN=1`-(}#XrIn9xqA`P@bELh=;_KDx3R8KQ4`b=ZkGgJCX~cD4jnZl^ryc5` z-EMaP2n?n*s7fR=>xJ96LpPBcNoWgCJVL_q+->hD@;VY!OXDN-0<~;TMy(q^@L1WY z@)tuM{~i&P%qvHB;Iw3J8xZVrAkasYz=nU;UY-q^U$vLyVc!mmKEz-!D9My-h^Cv| zcPQ&}+-bvBDJRotZY)|cq`USY2+gseongD(WmLQmr9peN)^EZjClni(j<^XMQ!>~b zwa9By2YfTa^u3D5%~>ZaV!w_vcbb+egV_jIpVl2Z-}k=BypK<30Gi7vYg>DBrIyVl zc>kIr(y^xGd-{?q0p*@5IzGbv zYzb7(EF+qcPUst&tdmSeC))aQim?Ki7g$N-k@)ak=-x`VNv7n`fN9^B#iDO(4%*n8 zcT0l)8TnfI-y_UXFT3y-ukleRJB&Og?%@qG2o}Cmw=H6Sr>j=%UT1P)l2~u6m>#O;gtl_o?Rf#y4_s-At zyKbBVKNLK&To#(lNd5_sUqmjbkmj zQ(<7hI*he%y#jky(?@URCs0V&{-I4*JF;J>iJcP;Y5F)lkvpz1|5U)Vl&8MtEHbJ} zUDs6Rdz0sxbLgYojbIL?Ueu4LNIyiwFv|EO>V4-sujW1!s1=&zMD}-6Kl8@CHcCc) zTUm0DNJ!EmfL5p5l2(6AnmL^QXTx!3$a}v{O(Mv#$*$P;gkDID#g9vciW;O#wMx1| zg9NXC?FHah7#S%saCT!kRnb2)%XU9_k*4@6ZSWy28-BECGW8Yp#pf!?Q;{ySi7>0x zlFIuxhDbdxUe%~*+FZ26a17Z~%tD{v(P$i9A#dvd0``Zla5^yvu?gO^6!9={35qlp zRfvERFa!NE8cf}eC3Jk})_A{9ejg%h+EDf9NgdFv3r?9TEM5y?Dd=gb$s9AkAjMnn zI(Bc5CHgYM8d*10Vfg$wGCjXN34NoiJ_$%hF;HBUt{23sW!kY-C=$NWtwBESvc0b8 zrN*L0ZUNM{O4zTP;n`ZYD6GN0Iow)KZ*yA9sav7!E6_d3dGH}qUQr+N9euyH^I+b` ztZLxirvTq3sr_SVguznH1fcdI@dbV%ZDw?wO{Z}Deh>=Rw=N+cLZynKVv8TmMPXWw zQ*B-ZL$gg^S2r(!5;Xcpa@%QC)^8mT-*7Wj`qFodaosn+s=xNN&u>ZJG5uO;Iho6A zOgT%HIq2yq^||@&cCE9HCtI<2JkfPx6e>oIRlyY^Hzh8i3GU0}tM*pwGpUOZUcznT z;p=N#bi}dKGRoa{-ixvU=7^o;nQH;OrfW7*y_9p8rKlH#E642~2_ihgbP)=_qF?5D)UVx{<&QM2Ct zLqI4K{pF!^|Nj0!hJYp-Ba%KVmS(@=ZZm0G-!CByVcOoKv)odFFU=xID6 z-12DY*WvQn19rN%T+dCV+({&f27R$&J<>Q44Tm*fI@*Uj5cjRq1@^WD`^Cjc1}`3r z=`ji&IqC_ww%U+%wMJ`&(^_NcK2mdi>{G%0zYCu%P%^rXMU$ zTU0C+$g#U_)`=Q}EL8NIQB}&Mf|4luvft~};1Q%{zd|;C8k=GI;ALw58S+>rqwtfI zti85Z1pVVd^>Gdq*_*;bzG&W|uid|l4f3QBqem`h2PCV#^80+R411Ssn?Bwq&@q{! zIYEe$`{2+N;mAK%A;%ZEU!q3#d#jsw3XcKcx;C23j$%#YXyi<7e|U8tmwlorkTc{c|EytF zMuDuYQFlMZ^85b85i4cQ$`>(mgPv(Kg*l|U(u>cK#-CSI9XRM~!8MQaQSuqn5GnBl z2@ycE`|B^OZ`%402~9A|3rjq)v}y3XF+2BKO}YW$H%?)1gehz+UHYwTvG*QqUCWR3 z>$BKHTvL)ieGhfTnz*-2%@k4rp7qZd8*08eVcVUXC-iu({uIzwy>9rzb*~hir;aSp zb+#F)ZsNHDP$&^-HbfOqBqW)jD(rPsb^L(Dcwc?J2J!Prad$NM{i< zz+EURG}U526ExeNdh`lbY)cuc;zV&!th7Eu!@jQOo_zA+2?VX5hiE}@EV|TRQ6QD| zro@dp_I@cw!-&&#N}rHuL)frPGk1Ywljc65g7vVpj~38#S^1W z-9zRN>jbHYCr|nlUAo#R_Ch2r2W7`OGct{^tlq>JSUCjERjX|7WXG(ua-Aj-5ZcxW zrHP&ebtN+|B&kWbMJ%U|a{KXw=#y4SJq)(Xt;{ArsxP4_>h4HA-YDNESqxutO}oFo zYN~^Q-h@UOm#|0kR3967+RCQpvp&}!T~#{C3TO~cdx2l47|Xg`5_f!XFs)3GmH)jc zdasLi#l5&?w^-djHlYUMOIDNk>GL9n$>y@=^D?RlvK$T~V`=bcVLx;$M>_%se>ZnH zmR&RH_Ot<*A20pE_>4|w@{~Pg-tlz7Gy?!N4J7J(TE%EHuK|BnQ>iS5Np&-rIN0MP78dYS!j* zF}w_QQ?55KZdgj1Q4$ssi`{{q`hVyIq~LWX*zL`0m{CJr?#dXT+h zN^RYj;idig`RfZ{#>V+a+13&z&BUD!6XNh^P9zM&Xhegti3WV4=>iPSJKSF?8&8D~ zEqD%F&Rr@!peMjWl#|L>PJ~zvIPL8t%hwrWzvjQWvi8qyFn8owS`9OhtZRKI<|ouw z)Wpqqv1kg2Cu7hhs}p^91D+e*Og?UR9$I)u}=yX zIUu!|V=EZ66e>}rHL}3*ZxG4+(9Bck3_3UX%X9mMV&<| z`UjT5rZ+%^YVoohIO&OSHkZP z1-=UvNCR;1m6!>F*G|^b4R->%?Lv-P#_hxLrPb0f}#YT2Qa9nw;u?ko4UUC{31 z7;NeEA!{-=XYtoAY$9As#=(k}>0+rTLAl0{EByst`WBHD**tp2_Mw{zML)FE!p6^P zkPSD|jV#Nbt753ukBQ_BEz~3e=}W4b1|U-fTX1TpmiQIrqCj=sPHXRW;A)`;cytu{ z4mhNK{a|4vd*;cg*==V|?wl8WVRThxY0a~*)vBYN#PCEB>GY|Z7IY7VHEHcS=~GL( zQ(oDdj&S3pHi`|gcFzt^kI<^pGqOxFoN@dl>~m9EjhQ&DSiEZ8qi`wUSQS}{S0#XW zI={BWAcQo;PXrxrx#32FggC-n51!!kfo6?UN7N_gB41oA2*21$w2owo-&%$~%c#Hr z{Psy)!BNBk%hQQX?KZu2uw=`K3as(zGh zq$>gmm%5X#h}PJh)jl%CXQrX^WDWM+^#SJc*DOmeJVGW#*mVlJztsGhhGx>EkjMOT z<@LR-t5Xy8E^GoEn-0VHzY09y?BWij!ke+L){mww@J*R`1o6q7b}shXr6Owbzf_|x zD!uXp4_s8NN;x|vr-r`tB!W=JVT!rL;nD%-xqdg@*3uM+nKED*(=~F$xVfww4a%{a z!}a%P3hP4#K{aJWhSzfsWI+t`eJBy_=LcXms}1A?>F1vsN5~M`u>Rr~&R+V4*a4*Odi zjSAgJhiKF*oBcphvc23+4O_iP&gVm{e4kv%XkmpPlU}Py z)L`2yw{M_-y@fK?f};{|(i-a!oI79YaMRPcM5oP0qimM4ljLL>iEs z#DOXDP9guN1JTYuja%xoREB^&(_Q7tY9xr}$JF&!+19+wy}W1_x$aOIT&UTIMzMpc zBPfKcIX_``q11e!;H%fok!+6-AqOP&9qY8mMk_0V+Ib?Enj~ahx&@+H2%ZmsuAFAx z6?|pFXqRAtgAM9W24uft%{f_9i|AnZ6@XSN@Hx!z!GH+vO+tw@P)HBraPt1ND-)MV z)qKnHGqkZO{}T%6JIQty8d*m-^+7J#!*LtZZlyyLBDpNJG559G|KqIxHW4un*@j%r zHE}&9L2EhhIgvjhFvj_;=>R;K$S3qeSg&{l8f%u@#pI9*cR<3k%BIi45W>zcSQXtC zvZ|Ek!&&?lzd3kN`yZb4M05e|xOd}2yRm+|e>if+5!xxobDc7JX!`adTJD54u_r@l zM_z*~Fb(uPl=vThVtSv}_`V*=B$e+ZF2p-9X!H9kFqe-5&#PS2v0noAMGSzN`HY+_ zMC5Z1B{l-vY1di1{o^W!Kq;=Bkx|eDMqyo5P$2+S#EQ+VpyRpgrwj2R=mU*^lMe&` zePKA2em^x4w|3MFLth>oA!sD$L>h?LC7b6!#v*qG3@V+#fe7tZ(^h_e_u|Xwf`1i5 zjHHF2L%?t%Qzo~Xgaw**sPDG;iBH=7w^$njpVr_1>!P_>?BNVI9q5Zsiw!i8XaFWe zYuZZMb$0U^<)lWdCtD%l-TK49DXZ+%=%0NV4C9f5AVP{6Owbx`jSAO9mnPsmtX!sru`5_q?I<#@^{Fg0qMqImj^z>*R^ z#Eu@&W)Y&sYWy#b{I}ahVPa%ro|5yWteX!=w}S8baxapp5NQdR%-8vKr}( zSB@<#Wvj>ig8Sodzx&H^n!Bf(4WE;WMWPvLZ$^+FxF2vICUojTwseM4hdi^#YT(Wvk z$^dp!l*HL)(PO>BHfO(It8Z7Y2(cI^$*vF2>&VEzAWTg`eOhz;X{F_UbU`0`-4&M8 z=Jx%GDvUyhU+bSI$Rc$N<}d9Xy?n*GoC9jruWGNi%6wXmc~1{t@32u;HP)Kl)ZRRt zon0Ini3q5TzuG*j5+x(Hb?_C#9^YMd+WM+9Bqnl${GTrr+?-7&B1}}=?xWVmDS@t; z`ZJ;NF2w4&3v@n<+pX$}1Xq%CEF#Aw8M=&hg1X2PG)a)fy;oZ?cp4ZFxccex52 zL%tQ-rd73VaB^elL3XKioW1_tai$q|?ase7arkPe2{zKQZHG7M-dng?bM!{FxhN{V z!^%B0S%jRJt)T2w*XG%UO4jr9Ls|3E4Y{@9g;YuFdmD!B&pHYlj)?kPdOfpU;tTZ+ zz2qBKmXD9d2lkfV9ZhU~P5wSE#AdH?byU=F+IPL|rR-2}0qyT-T3n{B62j*rslv>rQhJ>G3QV{QIhs{ zYblQ}UZ-fGT4a|`cT4kC!;#}SSzm#M!yM%aS)ogQ_QKohpW&+$U-|4KFEA$sCsZ6n zbdNfnx5mk;x~>9jD7JE|{!a{J@RJxabT)vUaC)uHcBxT`kWKf)`97flYl5?3JE(4_ zQj#m~IoEigsslo*$bH%4_~D6ZhfR}Ol0?8dXDrWQhASOq1@e zb}VzYGzVW#N*3@@@_l=|;#89IxzI#FQV+Xg;mzFZt?;f6s4qvXVz`LFT47%!>Ld9n z7|;Ff2-b=T{hBU|#s5tqZrQqSFw(ZCX^qYJR(rq|Y4r$_WlT70TgC&OV}w9QpvP(i7o|o%J(-Eu?%yzV8+igxSTX|8(|Bu zcH6JEv^cu8GYkcotH2BgQ(hNpT9`E{pz7Ab90KMgFziC1egtzJ7)p~+vll^qLG%_n zsJB2J#03TitQ(lLKy3$MHkcHs8^BQ1gL+Z}RNBQYv@j`9pMzlz>lP-B%DQ2{8YTtS zJrrdZShp}~x6s3+z`BRpq!8MzK2(}l&Qa%zF-!`q`~OY}+L1sP*4uklT$;T>RwcR; zjhf;$x>CHekz5Cialr2NGG0{W63yQ^)>Gfq5?V37w?5*Al5 z1!0lL2ZIV0S1_$%kw*s87#deF3tYe=FC3~qunDF|au5e=Y)JPSkdg3^n0SL5I;Eq< z+LEa;buc}!EkV*@R7CGMVxP0%CmLsmG4WEl$n?#*UF4+nWneDS^H5N%gy)-^f@n;*1>dGi#13Z=BR(ywKTBe1U69z7|VV~SLPy=66 zCU~J+;h^mZzvp*B2{jbHf5lcVon7hAdUQ>XH#D!!a@))4o42RSN(@^N`;++`8u~8%(ODvyfQf_dPNHiA zT0y3-I`P2qH+#YmdD7Yzy1~H<86cR=dK)RB`UY`L+-{>aBfu}Ns@jMrzHuQ$q8smL zz!H-V2cW^4wr?sf!KHJzUcc#@2wv-v7lxVWDrh6sucf4o1T6;1@xB1p2OlJ~AoAF! zi?~Igj-4G68UOWcFy&xXi z+Gu^Y40&8Y06APAdYHf{yRu##F}psP9{2#AaJn^!$v;;bKoY0gu8K2eX?VLS_iv^S z(?%rI0~3Zy4n2snThGa*?Jy9uS?`Yih!zq`$Rcgwf$d52*kx5sW5H7E zA2QToGGsXLbHb9G2fTx&5E~jAWV4jF^geFlf9!dtoQnZ{*x7GT)^I|Pb~(Q2%Yxbi zW;MVw?t(vinqVufw;vg5pKvz7!xsRPgoU&;O1$hIQ(Rp}Rhxv~V^v%ckvRrCy*Jb0 z386!;zXs*fKr(+7s>J;J$A2y+9v6@JZ;}=uFe{CLl9Yz6%b!6pkND1M+-qqS29`0h zE~B&x%k2bX@p5?mx#{rdJTdeyu?rG!Ek`&`13#I=_Mp5`Z;lmy#?geH$UMCqccSQ4 zZ?fXC+GvNRx_(wDiHlnbDZDWVVZ%2GWqfUO!%5RySQpoAxiKoF`H5b+hld7|sM-|H zU7`D#_E{(Bxk=zEN5Oze?Uxd-Z$CH-R<$oq4A&~YZ#vu!E$E<_ps|2CZ^$+#9p1=d z`?H-)i;&0l1NuoE2L%&QP2`TBKSoTZK|obO9a9)7bWwzBG6{O`m+QAl%%#Dj_7~uP zfoG`lBO*KIys%f#ADMVbst; zLqoV*cU<+ZLOu%7%`!v~&II#ZRc&8~evNbQbkIx>wP>DL?-h*Xl5&C9bxgTp4FZ{LDbl7>SLdb2#>G7A@y z*-kT#0oVIL#4QHq7QY3J;F?CDk6Ti01%b4_Xw$+4+4L<2^V?zi3@mq>IBKE2$$=x- zLRFz#cJ4sOD|CxNa_Y#IAH1Iv-5P1cEZRC8j>*hl7y?knpy$B!wxseNrfog*g%&ma zCNpDkzl0%KO{|fdtpRKt3|4j#qkIQd2Vi^GlW;}NI~_|c?m-#0oZgt(=-jQGg@Eqv zkRKyDL^XpSLBfIxpKABog@DzIKsCCR*weO^rO~s?$GtLF)bPK^gt-nh;JV4j@!?<_ zh`vR1&GJOh0R<+iBq={Uqzr>GdMyImbOQXeQ6W)YJSHReV6mm)|)$siVYr z->6jg9-V>b!CXsrV@o=%kjv`4VJ!1qTT2-tLD%e{e(MLJ{BF@D=2YWRxr>FN3f(Ofxigi6@^DRI&>JOCR-=XFnZvVM-t8n% z^GrT#lr+p_L3xdP6;}*fpFvq9e(zb#rsOLPv_w_*k0MN;Fm&Y1KHXVspw@ICac8mK z);H0b`WD>eLb9`IOtZpsZ_D=|w!pnLVP-r2KRGJyPefT?2*S+&e z;u`2Vn>dbnT|0yIKbt52og>^bBGfl=RaCj*IZESp6vfo-hnZA4ublQp6%9lUk1e6^m^yhto!41sPJcQ_3{C|pLwqPZtF?ph z6El-&?h;qAz#mY^cl6ici8}-(jKoO0NT>)(N(D@=)6mR@8Se@%E;!Cn!>o+tN|=qb zdALx=n9=|71y?Bl=RMti z!N-}|Pt7`M33ode9Sk(d(Vn~{w@EumF~6x86#hoI(}TW6cBM6<)W8*`6*aBgcA!4GSY zynnc_-R*Q{qvKx*=er>R6?`KCK3m*7(z-iEh$G2-fNr_`P7O`f z1#uyHf^gmQs{NUs^X#_j;hH*8ZpX|CePcZYjl)`>UYK zg_d%KRlfgF!URgl1;NnL!X5)pH5JZtlZAXZsR1N71>qkjt_y;!#{K$MOTD_y%i_Z4 z2&*)Be`5cma*&1w76B)P^jZ!CE<|i2{t*0&x=8_ef~P?=kpWL^4Sp@GSF;JxbA=Xk zy;9zAQ5*IS(L@`poG%Af4+pfC_e%WZ3lvay{@E+!ny$T{N+t}|?N>w`53MDM2zj35*LPrkE%|+W(WS2NUTu~)X2 z$c-9c8m2&;f#4h5iEJKVs-sAZ+-b_t&w&)vkAdOTtTf_07&33Ys%ZVP;F~}WaNHx^ zp@8jWypN9^wfxU2_xP`MwQtAs4A@J~3)pJ{EdUfMp_@N;OxI$*uiM~0M!539UJz0{VTBm|T6dfi z^C4|*L@vT*tRRF}M~fQJK_fUK zC%7-b3RzKszOnX*CZ;EBC)R7?x+K+8;w~iOh}=QY8xpJ#Zc9ODM);A+!Za_(Ul%D# zysNn(G7gxh_4mb``)03@XfjKJUhP2y_YyZ8bcwGF)zMXOzT!hxV!~QMDG$=4?~b>h z5(*P&HC3c6j+PG2!D5iuYB^E?Kb@qfK{5Gnxooe@0;oBTVZ$%zjJUD@K z-l*nhxUi9=c5Zg6z#$yfe1>Em#!+GIs4W@RVGP#n13B-E1{blrQjD3O1mzhCmPF*X zi3pqg1Zx8r92mIMFGUUDoH?o)^ym{uDq<*;|0W2amIfCFhJ}B6{|Zc=gu{qK{!hex z%-=xwmfMI`T^B7%hKtw$s0pDi zLah?r1~=U>sv(w1fE5e%1otbxagJ9g;syCFLtiG$PdwrRz+BA!##vGGAu8!hzKM38 zH`YJ+G4%QSR?vqIs7D(Eo~Om!?H2m($bK{zy&{t#())P>hrN^N$R_tz$ksos#*Md?gMZJ;0|ie$ANrbAC%Wv*q=77 z*&dB^OF{hy-*_UHX@f&Fvp6WyO`)vBavFAbyoSdxn4a!!Cl-)1-3{F4lqZhR6)V%c z_;GgJVEnC^&Tp>9^b0*hCM8JAN?#+a(3T7VE>B-tAKp7hme@G#Qrp^#%qU!daE&K* zw=ALMnzN&Qqta>nwu>d6|^!hHaQ3%3(tnj03Q1s-ik~Uds-cM7sdBTkm%cyJX7RL3xRJNWZ*FI z%T@CoBQ<6rY-`ts#MmS#dU};?fq9`XIK~_cb^%cAE|I8oeKQtx7fL+)adMUfF9YIx z&wmqN6Pkf!o8%pk!#M*m?zVT`)Y*EryBPUs{h7_<0cu!quKGVqV~ZK+DqrIvM#SqJ zl|N(UL{zLMwt8k|S@M6T(&J{LJT#GQ{}>ct@*Z^@1Mk;`&P^Ws{GZQT{@-y6ELqJGJaM0gGZMGot&uPiL&%ZuMw)moE|{S8 z&peeCkz44JB>n_n!v)aR5>#kneNPPfe**-5-=3IJj459M7vXa+T@0u@Bffe-|zLMKk>wq!N#NU(u5?SQ;mkPIb%$ zpaOLyvDyf$Ooj-0H*$qE6x&^t?cMj*yl%1$_P?cuH-a;IDEtQ~{O2hozH+~h*fj#V zVrTT=y)GExelIih=m*eM1BrF;v36Zx)I(KuT|9w%0h9@fXN zGTx%n{QD7y^ZRFIpF$$YIH>zfK2Mom#hN$7*Z#np{f+U_;t#ig*%0a20n&~Up`*iK z4gw!$8kL2zB*H>%aHT-t$WjU;AaHm)jRo+K5s+;|e<=J1jJWYs$)yc{F6VPzJrNpMbEW`atJVKUL?ODo03(zL;I)W163Or zOv(1#()_oGaI;}mRXVz>{#(E zeX0L|A)v0KDW6DDlnSGwH5#9fM39({oOdE9b=gy~`~*IE(s{VW(oW1YE*TsZH36|- ziwGgK&@yeGJz!GmRx}44Qu*o_ZdNJO;0~emN)?C{VdS9ej!Y4EYx~Zq(^WFlVMQ zPDK)7+Q)?FQdX24hk}&jA@Xsof0wM%XPnobC1?7?DQgt5*!>C4$LbSnC*~ou^!60F z(J#oal(l4~Q$iTy)*N4A?)~laor{p`{+FzkHEql><~j=h_p_XX zkeFVMbF-GdWN2V3l$8P$OxOaVXxNL0BXg*kVHOZXs+hMCR*I&Ho2qXwH@XrBDXKm3 zePgjU;krpU2GH}}Yg3%?fno4R&0LiET-izESs{M|OS%nGfBDzsI2}D@sF+u9*aTjo zMP=)0&c_mOtv%$Xe>W%?F3hHeEtaTAgO^3w`h!YV^d>MylhOhbW+Ow%yarV2RjB(W z2gFbE{TK@?YuwhV!ic5*Zoef`Ia}*?>yfJSB73x^a-`DtQgDUW5K2${-+=fYDTtI_ zr9yU~051m0Rg?OsZ|*C8?o+F9QElYS1MT_EI^^O?UPH%e+{-HIO$>7^krRf;B0R5; zh4nd8%8vahbZiH!a8@k%{Wh1rD7{i%<7KGEvJjGi8pd!A$((Pp_2E?NWYHmQ{iO_; z3#?!4VDxDIjYLllJf7S^D(X4zwIAg%OI+VTqFQNj>z}PZ5U_l=p!RRoA;Ao<7XeIN zbSp2Y zeLv)m)-G*d4&bM=s0Q2@dow(H1{KVpWc;F!0s9m&r4#fXf)O+Z25AgjWMg!7%mtyP zwHdyJ*6yfgSD|A&6w-c91j9xjc|9%*WjQ#N)Ba|6A|cGgg;qLj{L+(#%F`=96^^i3v@AqCn0!kr@0_nrhSU*PBwVT4XUEs+AD*-Ys4oYF!i(T&@wtB(fE`s&hQzDaAac7FdQ zq?QYF@#BoIwBVvxFwcV?#UamesaU)$XF!~6rssNUT1QXrQboU%g!4q2=D>`vR*750 z*h*a9AaU>PVW3jc#;z}NqLdx6VUS71I5(qQY(EVNrJ%mZ?4icfv4W16{HSwD7wl>KBDLSfGr5Rx0dIBVxO0yUFHipQ zMm#~!j^5YgrN@%sE(3X$Ky(15*Gf{_@<8s=JxbZ556ZLyq%U|nVeRy71h>-DO0W(&K`#-s3(ZD zo=lYtRQ4!kr%=e+{H^y%B@3O9B0ktGT=&!7-%fsNw*Q2vV{hAk&o0m9Lq3qY&rqTD62DG<#T(HuQ zakSq)RvSkapDH)$s>X9ST⁢oB0cCO4{I$*5sjJ^qJ8FQM?=?^fK>|&^t$B;Jl*4jPfJ9Ow)o*1dx@;=Imvh* zY%16{V;ovMA+5DbvLQ2|+hW-CC33F|w}udiv41b^3H%J&t5M2ju(UeuPoJ+&Q3AWX z$CV#S*@M&6qj4C5R49*sZvgzL)Fne#$4n;GWlK`%?P5{+?t{N;$mRB(J_f^8fi@;Z zSbEwF)aezAP;24;;|LL+o?RlGlT|Ou(tbBoFMU^3=;B+wiWsX^n0*EMCnSw67Ske) z@0;L1v(+5Zw*ox3x;_xw@3koQHcy0SyNagoaanxK<|cR$Y%p_h_nk0TT68W~quGm{ z2g@*<&D{{U9E3v48i57--XyFev1>YXDowfln@+rtD`NY(gx1DGsGbuUyAiygZ@AeR zR^BXe@Gws~k+c^0-FOc$MoVvrsTuHgEUOxc1y+T@TMGOQ&dbWChUrWxECh;Qr$lfS z3!;E%rQfTNUVej1#Dy1n+4n}$90~0%kd}GvqJaeRNU`|jjah80AAh@@miNG^yyAao zj3MereXFw^*If%m6K^LO!)&5!F{726k4>RK9FGxg{S~8g54bN^ zTs)*qfU>8p!lT)Cp49ML1p88^ZBsW^?{wd%Vg?!^T~yhz1Tv` z_@PGqw;t>L!#AIPB*ZesAW@u*99esZh(OA4@aqh**G-EW@Nvnitf*xo-4-!ChOkAW)$My z7=Bm4ybn%%SlU#w!;kcWnr<(_n|v&sRWhIpcnaYU1b1avrbitBAVo>vW3($UWje3u z76$@>sCPI?04{-RU61+`Db!s-(fSl6f{}=u5gJA=I3N&T2?QeP@%Ec=M=b>9t_-_O z+t`By3__iBnFKdf>w{$ZzJRnTBrPrZAgp%4GDxYMM?-7_UPR1+=_Wi-XHYjjPpKd1 zk247d399SmXME5e(K7ZR@UydxIiR_mqUd|v4PmLOl=XQVW=n{QwLV(o3!I>krvq~` znf;_;*0vq1m5up>?LY7#;W@!C`UDqN${LfJL$Pl}o8`$|;|14zu~>J#$U5K*-7Buy z@<#eK5ms|KLz7b(voeTJTVDBV^>+pHaVtCXiHZZ~B{tke2b+GDRK0h^=o~BG+Xyo+ zY*saDkBG_oLsQLk)orBhZ-3qNMaz{s7O z=#8T>QU*U)5xO~TSZ!i7AzvA|G>1P*WDP1u&(KsP5Q3 zX9^jvm=d#@eMwwa`fJDj+t3GAj^^F0WLX^p;YmTLk^#$MB0fPy`|r5Tdsc%z;_xgc zPHmHoMDDk-TPi8j6t6mRWko4vM|f!*dkBgo#+Q*>VP)JkHU6{vTF90zV=fE*qo{`0 ziW-#ZHL)&GmEC1IKVp7yLXsnb4-GH;DC#8~2pW*9ZWNDMwik(7wyzj);}Zk;9i?+! z#O5^!oI8d6dAcUQ3&Q?wev9aQP4qW!oWQG=$$N#s9+P#eMxh&nz?O~pT9LQy0%1h- ziY6nsmE7pp5_}cf*np#q-zqgwc{7^g5u%-e^fZh4d5 zDxDV#KE&JTG3stokALs^$?PFov;drvD%o|zJF;QfHpC5}vu7N4(5O~*zaaE5z-bW9 zSFGOF?JA!6%syc9Rh-tPv{r5SfErAYYNqwqWLq z1Y+wYsATV_@)sxC!kn1Lt$)D#_hhhMf)&EW0loE6+cT=3*v2V%L^;*PEaG_TmdlpuN6<2p9)hZdi0pj?88w@h5Z2N|tpzK|6Em z8$B?zmW5375~}H9i4IN&;npdYDD7L(7O!d6B{wTDat`I0Zh^bC*y_#Cf&Fi1nYZSp zslF4|3xOJuqZo&K5<67bRiJaFb*d~p6?jFd-*rwWc-3)&Sea(cskt-*lO`UBH$?F> z`bsktvlAhS+7?7!7B_(3;R~W%NkfRFH;32H5M2^JR=8VIz z{egvJ_QT6GN1+(jCW?W!w-Lub^96H6GP>>^l0 z{AHNwdV>qBgHaf`PF*a4CCzZ;!e&VnPyTPs@UdJ2FHN<}{7=3X+1_9glNxX~7y&=H zF*jc;F&X3jWJ>v}Qx4hH38~$HC~LhaVxzCz{k>b=42~0x9{ivIGLA8vk^#^wLR`N` zbv0ns##^@8Bq2R#A;h1a^JD_^SsOy`65%f$ecPZRqunii&g#@Y-y8} zNBm-6{o5Z8wh@AlclGJ2v?`wehTvW#HW&IH2#dYX?K-1K7nQmv2m z(>kNz{jqf37Cy~qI5~L_=VDzE`A=P@oaXY7v2Si7s!m1;i*`r7I;@9X+X;)IPE@U= zcR9^3TB&m)c9|<>uWEm+7nhAC2)GuBM?fr;+71DU+d6i3-aa-QedgvO%D&jJ)p4Iy-w-~!nsAo+o2{AD~>Pii=Ma(w%&Om z$yRjH&i%o8C(T^BV1|8xQc6r!B)UDnW+;V~pOlFR(WU$LS_omvW`0sUP8+fhO zGVsttcm za9gVVo4~%2{Ji0wRoEx3Kj0P%^Om==zd;cb;>IlAr6r6i`z58Fh^ zg40Lp@jivn(dCDI0rC%j4}cVvGMN6ZY@?gOe6`ayS|_;R~d;C*1LW zjNYx}dPhMQ9~SiP1~j$M)>pLEV56(j_u&~u@4}#m0Z-@ken^)cB>PJ=WbL#RCyX`4 z^s)5*T|we27oXtSylo`5I$eQ!Doz1sB$pTnn;jIkzH&7KxP5v7R4{0RtW`b*TE8TO z6d`~lpJw7G_!$P1j9JPr+7LbEmiocS5R}T7e*xCxUZjopWi6|a@~?C~^*97(0GAX} zddx2&fyFQ6sSHA=1}dS01CDTX9E@9ugFACY?_YjLa=!f}aB{QmwfbGlYwbIA)j+od zhs;GTQQlh3p38BmfyT{>bk=q6Tfv*}7lMuVWY<*fRQ{}v^1X#kH0PVkf$um^Z_r$4 z>s@ugmfCC)bfbN1*XOB=%_k?=)y@y%hPA#D941$o9C}ymA5Zp;@2}?Cn{EC1I1FmY zbt;!*%7(;yv+X2$=7w_t=q1e)O3}r^wCz!0yrp%U(Jj2@3L9&JhopVKv%<(y1)dK&=C4Z zic#fEa{$<%nKq+yLBA2vY1*Ou4E~L)Jd;;+*}|iUeh>cQ>@%O+!Rj5ukLV?owy=V; zYU9p2?jM`U>~c{*Ur33 zFun=gsS1W4M2L~bh`d<)Su`nlF@x#s`0=Fz7ND$1X)c7~cbLYP*R&q{*7JIySo?DI zyLK`O2eNG^aGD3j@J*D~PH_6WiDlPhhfT)4C#amVy@fi3ZqQ-t+aInpY{3GBBUKbB zubE0D%hzm^B_I^S+Gd^U@u3bh)0_)ugbEZ5&@=eqOx#EJD5ic+ZWGnjo z05&%76P02P_`;<5A+ZU`@Xe<0!Po5;4+hPSlk&|K`$xK$=c-?zMmji=?7+rkXN7Yc z{xY^Sx(%>fMUp1bbopnkHRF-$ohTjWbuEKYON2!x=e=E>Rm4%ml}18?RaQ7vxTea5 zSYyN;R>^PFoW8rDh0~e3AzRkZNLXQjPH|=V*i0a)6M>Y8E@8yz|%(w z`5j9d_hc9|S!OfBA=X}UW2>rNTja9s%)#%kaO^wGvUai8O)kavcU4x=4KNzSgeW`) z#@5vGmiPj-4kLv-pyX1Y>`3AcY4&rt>?uQbT(Qf-^oe*&yh2=7`b{K%U*HkcGFMZ<;Qjg@b+u1Vo{4am}OCz+kXhh!v2sU3CoOgdR~w{)&Il|6<1nA-0dwtI{+(D4r~%l&{FF!vZ2eL}FB z{wrp=Y-ETvn^9g?I9jH<`PZO8C@8oqGd}(hzD&af>TDPplW@ zON^0EOnj&OyUv_;%8DilQ5xBF5K^JrcKOzw7!ErST4o|$_2T+itxRBzSvHS@UdAjLB(_;txH1uwYL`!O8P2RbGgI=|NJm+X_CgELEeGnYZl@q^ZMwNH?K-C zzLC<9$`!40VM%;LL#1wgaXcHCh;0L+khRJKl@h0<^8ydF2;FR$;7ge^-rBGhwpC>? zPz=G>z!R4FI!`VN)}K0hdp=P{qJrMjfm0L~-h<03N{5;m7;JC?%Z5BxeiwK4VX`iF zd)SG09}QiudzD?T-ZFcw6)`VEJ<2>`^HGlI*($rfVbnq};#5@_h%_myO3z+IrKe<$ ztgWiBr=6?rJW1urp|LLxurz{R_c0d4mO& z?V~L^5x6}fQ@bM~^T!gDoprg7($085Y_DXB6) zo)^a37miBF2AuOJ5KLvvWVCU79is-1@ia1@h zRKD<0mMw;>0tPDCWG2h0f}u(FJ?9$_rFGc@FxXlJHAs(Lg}ug?w&xs}fE_1>X#IyO zw2m3xzzgB+qRWgh0^o3BiCVs$6Ov5M*GgyY65ln&_hn6!+z6Llt{rA;m|bH$vA)}A zyTajPb7H}Vl~F`mlmUfDm%AbrS_cYmV*a?_4<7%^I+5;zeB{@@WN<3E{c8R|KwJEJ zI{L1n`Bv`MG~fc4_wkQymZ30pgO_1^QCcef=^wlA{x|&UkH~AwA0jt8vX4KUBV;)_ zNN)d7%e$GCAFK623A^9o3eiD;!tuYDq*(0v-}l%3uzXrifC3_bT_ppRV%9ZmN7>y)RPz z&Y+mM(F_!>%xnTO(>S0ES13BnoJGmj#_sr*kx4C~Ild1H5C$K|}R z{pM2uQsbex<)-~^bl&7A?UN$*MYF668KCqR6fX>{BNpN4lR`Sp%mtU3?2n8}28JIk z`M?|U6&*-(SwYkUMQd@eHb(Cw*?(CN^;^kJQM@Ys@llteryhVAVAceB1zK?I z^+yY6#AQ$T^=AIJ&oNsk7UmC5=qxHncT_8f{9=eep64HFAiF_iNPuo&;g7BE%h3!Nr zabq8dG8+S8{c=CxpJ{&#SDw;Yi6HU=Ha*%tq7TvJCQq*v?I#;~ZeP3+HT~&?{ zGu;-0X|T-seiIB-U3+>*!#T|e>%oa3;Du!Tpk)N>60~zAQp5eMcFp#8@S^81xezCM z*t9q&+RKVI|0$JBW#hGOFI-$cJSYnMxx-#&Ja9DK9$`22A=GZ_V{Div$NPq6($1jF zsm^%v>5g!_oJW0dToQN8kE0$!YRHDNK$So715*m2j$`5K6mk)68XsB$j*EN|p&1L4StXst+Z+wjM* z%lhWpMC#jNa$vjEGR~ayC(;Ak%4DMzs57ZBRR>B1_He{{_l@A)iZqb(NlaYdmlA9L1M*1{KLL(hBNyORz(w|A5N+Ng zi|WEn=c><6K;gyM^4fl#a*8gvS-q4RvY$COPsHhPOL8T4`SMWzgzQkio%Brm2ns42 za`_YOYFN`kb|wI{c)YFR3OMjwXg-u>n?2^u@jJup!~EI=CILIEmyQ95)M^ryPU$RV zK<~{Jdo#5h`{>jeWs~1sKAc(;r;0X^8i+51Xt=6)yMZBtM(tJOFGh4ylfQaHe@QKq*arOs*^pma%w4udz znZwdXPCUPksBC@>265ahBZ|Pf#?7YBoEn4dkCDyxZpdawu!hj4e2f(mj2ncdhT-`O zE+fldeJ~N0rCi*ax!tZf@6Al`U*YT%AgbT zFzO!X^-&PzU$)h~0s_D#fl-w`NLVUe>2aFdEJGE335!DCg_A7JD0<}X3DN4P69JX@ z;bp{OD@Fshaonk2rQ~M&nafIk=IHr8HSXdC?Mfx$S~65;!L5*MoYv@$HwP(0)#B-m zK;h<+Nmu)Mr|DDg*O|KrJCF}q>FlfRR1#&v*3ZamLV28}KOhQn^VyhP!IP~g$MY4< zc4wr9HM^|)f|=phz$m}+=H7S(j-rA~ei`f^i(3VAgsa8ILw3sPWXTmetv7Niq} zdlJr1kYVd9D=xr8)O=;Ea!axx5QN$o$cv$HUP_}Lw)tON008mXwOg|#l*xS?-U=R8 zE3%WoacPdy0r3FP)7x*cV`(A2PmP3CW+#v|Zo}#V-JsovWn0`#YLK*cxJW}2G+-li z)$F+{I(C9FtXoi0CV}Od71{nfo>yy`i<{QgyK=>K)Ca2Aj}2B58Tt!5?wAE%d!qsl zUddZ1GwX}^Tf?{mnWyxEmV43$PH->=h*EkqbONskpp`G<$r4j)oN(^2RoZT$Xm;!L zUS#<%xh?%QLIG4JYiO=;iu!Al%**IrJFIZkhQd%<%~#-worxH=uEzb>?hB7kT_q1_ zJ0j}VW~v^V3m@s{pzZ$U)vWzROZIT|KF7!r@DsCKydw{XK9gQP!&3wmxmAcIhjBqG>sw!IT>;1;nra$ZCHMr z{fTq`NCFkfAkaFZF`DT%XbcXA5}r=;2QLakq2vc$Mz!Eh%x*Ce7w^e8g2y|QJ8efg zJ8f6QJ3cF&X02Dmf(_=l!IQc0vm~xQQOhc{R(&81GF)@=M*mP*R@i-xeDg3+Nx=DN z?n=GUf!NT8<;O;J=fS{Rwe~1znRwTtT6@SH)pEE_Q$XeG{h?%bIyH0Y1{_W@)y=r5 zp8CA;!yk^DBbmr_c*`*UUujz#Jzo8H+0gbR9hb6GRZ(H8wdOR$T zR$C@TTo&gNYD_o%nk)B>*b;STLAj=@Ex))r_bq;!+=zedyftF;cVUJY+=3@TNF`Zi zE#_&$tyvL0=MnFDm+4{tZsz$X;&1VQ-4HGk+7d4X?ou%HAs3D?^@o&z zjH#*isghcUbweONlkzG$tC7M)G6doH^?k&%RpWEJ4>c;b8T<(Cagv>sXr73@Mv^6{ zBqs=zaa|`$3?07IZ-0pf5BY7cUL)$Poj;_lohO@gyIck2+s!govp-SBi@LHSv`Qe* zA##0k`AYPb=f!v|y-DKBoUXHg2e_-M077ihJK>zA6HHI)n%>-S06Gs+S0=J&uIu`@ z&V$}UJ5STuPnk6*4cH(DjheRTQ9dq6dAIomaGKRtlg=$gDGhGa#Vlj!C9y-c3US}A zzN9&@pNU9QQRa&T!06p;a~r}3W0q)tnxtty^t0ui zz_oJ5zjwm%bi#>Sq_6%K?Ckx<`V7T=kVHpvQW88(r*x3aFi5E$tZlABY4pf#21#Qc zq|+Sf7N+KoskJ#*rfRQ{qzZanq&|8fUU!3o zDh9BJ-iH$aGD#U#rg36QWbXs+Ll{^!m9J$q zAUz%`AUSzx0)Uk@ao8#)efS;pvp_^}k`!&O_NF1j=OlMpVI3IrNg^Zs^QW;t7+rYD zfPoDRn#^&K_ZKoXCT5M&+i6*0lRyU9)S^mbJT}6+@)?s-Pso?Iu6T8M_<|1$_@NK~ zuz%DTp5ch{d^%C6Ibcc)x71^?SHE0In&FDee(_;v9}}Wq%)a=M*Xw5NPmFZ!d2|?% zt-PWgex{usLS0kO?|~6(xd{$i7d3-)z$VXXUBN^|!m0aF|B(4tB3|rVYZ`ILeLEvE z@{^S*m1inTNkP*ZOw1${F&RpTPtz%M9Q3GOJ_|@eckC7Glze#8**_TZF^c}`NxGp9 z+bFnkQ>UwD18;`-0gv*^5NiPO%=z~5hx*;ydD{(Ry28yNC4C1&z{8J{0g*Xx5=UMn&cp6wV}NPyR$lvGyPkFSsS86 z)_0SoZ0KcZKub*Uo7b|F`l7izdM8Awi^*Avm5=ifGRy4YGMDJQ5x8Wuf1?5npo6&_ zpFWV63^>Z>M9#U<4y~dcmsc3-?}b#6L%Z2xm$u)YLu*M(SXPRr2Quy#isd&!x-fdM zcE^&ueP$+}g^nJQ2``%iVv0O?bQYoC+Uutsz*-eHg+ zyHHPE?gYPb4^o*L)z-i*P%=Uc3r&ejGTZs+>Z2&eCzY-}WsA#3`boBCdt6fc9qT{T zXs$Z6pCo3sQ2_LW3C?WY9}^G*VV^sg9Pajk7Bo;FV0xY_svx#+~(t z0yw2rlN@bVsT8|GaAjyzS3ig^9l{!l;)Ucffz!P^=P$tl%U-fKl!fl=&BSxXH4 zMa1ZBT?Co?&*{+qZ|U!~F#_!Bpt%LB7h1S7va?0QzF`QzXJZ>@2BUQ&5yep-Pa;~! ze3qR?Tds+%4bIWsmu~FpmpxvP9kvh*9DO5*pw3DdiXd8nF%(8?ec0(*y$~FVgA!2{ zE)WOB8L#^eSx(1lv4(CY#Z1N$f4hT-*cB>j^!BRL`N5hXlwUP7s)J2-G?)%=!0+Ut z*h4j`r`oA+{}|>fFnrTBj#>P3_iLAA^%IIpX(zzIrEt>PeQS+d-*;h`=e45|wsxhO zVG{UYT#1RXrzLN%zN{t+y(YF-VXtU}zF=OGU#II4f@mx1y7WP*15u+w8;?0#<8vo! zhga%9yeXS!FH`?jqEM!&FQf5lV030PqBj0Ed~Dow{xiq4sDr`YUy=8!4|HGJ z{4ghGMi>Cl)^T}pxT6LkD>+!Ij^+l-YNi780)gL1?4*DuwdImpSJgzRDstOvpK>bv zmpxLceUDn>x5fX}1Ch`t;+X*e$|>>sn-jyj%C@C~idv4twjp`?h)^&ecVv(jSn#H= zLa_GUm!icA9+snusj{1d*0CPG={3obTyL~8z4J!jH)%s0s8lmuzGB?HdO~L&>V)QJ zhzYuvAl%tOs;?w5JH$%nbLIdkI@5VAwI~gn4JEItWI;ac006dW0EiC&0cdFG#hpeY zmdpf@fHy%UY*kX5kG;DdvNuvGIum(quAD<(tdNTs74g@5F)seHn@}W$6*@c^bw__$D0C`^ zOW*LL`#DCzuvp~C5}f@K{KU6#=YI0u!5j^Wqp0;zf#~(@UI0>;Rg9i!B*R19T#SfmOn$LY_jEYjO(M0t$4XMmgSTp{leOq#vGD&!lmd9=5k-v zY*Im5gU1DB?58l6{Ogi_CVTnp})Q*1h4_LsC!1c%`H62%Q?WWR1$)@DvIw?HLJxalzi>Jf}VtR7R&? zCwb5eyr_3YZk?NF!+cF}tnStX*A!QOqB(!zjI-#_n7FbU-H7SkB9$w1@@C6)3Q`BD znc64?aF_D9`lbOxuS56G9$p41O)`Zzgg+X2G_ES4pZa{aqMsB`X;>G_51Huw2RT4Q z(7-;^A?A0ibTe!UvG1|uq!tVqwW=cLH5xc`$1OpOm*=epQb%cR(!cxTS-TQdWTEfV zNj)%KDP0|-%)qIv{pq{E{`~sBJz@9aB!xr9hD?!kvEf(8u)~yO)e*-^RV> z7oUs=%F}lH=6Xs)Pi8iUH0WZXuRkOFRN*Gl8c2;!pW@Sru>iKmN6hH@mmY##3)yH~ za(+iEMwN>*QI*|MG$qDk4hUP@N8C$~rW(H8Gb}5O9jbNT<&O1N7pqyyY@S^I>@DU) zz>u%S5hBe|724J!B>wU9w&#$}GhzUfZ=|$wd3e*898pMx2t=ahKk$vIZGrK}EJGv@Yz~9IvA(j8JMVSThDxpq0A_=(w;CZ+#87RyB}~sVTyj zO#zgdXDhZRdNt(XqnnH8Qa05qIp(%i4!_z|-%#)oh>OqaZ&aLcwU$W5WlLm+qWmaY1qQCANr;rmA7BvHC;@@jO2*zF259!Pbt{;O9NE5 zzaW|Z6&Klyuu{IWW*8+Arsjj?XP&of&gXp~&!zQB!QK0=Uh-vGg_q7&XOU&X?~^^2 zB!QteRLy>8oz=(S_GZUB4X|M8SI($)2Y>67*RRF6E%_V{DITR@s+p;hR~a}kB@0sW z;=lNp0YU*I)H7X@sPI(9Q}@02Nj)JkNKecyIZ4}`>tcRQ8vt=~XFIBR+-V-2xIMf> zp}GC_JyR(QHqYIUkk-3$u699RmZ2=4(8#`_tgJ6HIz6Q*tl|=VYVEDKMa8(%9TBur zTst(UncNP(ocAJqDy1Vd8cpYBQ(b+#F@9{Xe&l!An)E8?Vqo?aj`nyFpel(rl*qg4tXyLFQdX=lohv4xHVuB*R70ovzMfh}{$d{=p`StM$_ z9fj)SAv+QnF|k}%FE24vFH$m9HvxX(w8yIX{XFfx=NY%?$pU$aS6E6zeK$UkyEtx9 z%98uQ8M};ey_KzK9erTJ{{*&01Ic24M;hF7!Q*CVerJd-$1Aqzskg=eG6KJZM%Bt~ z;r_4$Ck_+tImHu6aK2U*r@K{W&yck!B;g0)Yp_(1h(M4^`XB%u9gWabkh>@rC?)Ny z090AGEfOrF6_vn%q7^z9`bBCNNmi@eT}Ux~XtW=iCEhj1OihVD>G~Sf1pKa`#B_tq z#o>Apw5Fd$g5Pm_73e|vS}7`qFrTx#8rr4#P3DgRme$jEzCMpE$XXe6t8bZzP8zhR z8MQh5L+I>#lrkp<=*>k$r_sMe3bkhiu(C>NtUrF`GDPR^kd#h4@~FPV*o6O#NYQ(` z(ni;LGW~D-{K?JfxX;WhV#WP3zrhbF!oTZ*yi8_*Osd;_L>U`u~sz%?2h#1 z{3Rv-_q*vX8q8f(oXL%CyvQG4L~V7$k^j&Y!ucV*q;DWyk|7(k#-ABOeDqOiOANn; zB9S*p!r`7=RVc)ICoZp)84WZ)K^ffchUQ@O@#T1FX4${x-+xBNn8rV6489P7;FcVi z#_D!-<#JuYiRy>(2a~Th#7)FeIEvB>39btC=rvF|IMLgld1t~vGdRo+2DTWRIsuGK z*2u~n7U8TASU&LiO)3=Fx;APqEdNyLt@oET@}GB!e>6Aa9V{chA=N&p>PJ-lHB04q z*`ykY-)YRY5`pHts5PHa7?e2!TTLEN9Y3t%V~}+z>W~y0_F7-c0%J=T38f;Kqc0DG z&O%Es7E5NZ35hKZz8zO=*21O^pj!=6{r9&0v%aE)f7pm>c$kMIGAjIny@H#=>osvA ziJ%uKr@G%<=&2sh&b;rzIyLs|F}jzhd(G=#y_|nsiZvfr@c}e`N5Pa`h=nC?%3>2Z z6gecj`~Exb@mZWI6+e-nDW_UDp#P}^fVuX+FD@h40EL*I`dR!pA*?~;V#xP@IFING zqeC*$$Rz`eTla5Y1{TA)lXMMNeF)##$)D!vO*mBLCzl{}~X| zMF7)QXVa)KE}dHks4_~tOqZNM3TDFLh}L>V#-k6HjLfo129VU#fhkI}cF3g4B*J(o z>8o1{tX)vX4#`OIi`Kf!s9<+RDLDT_UHZ==U=Pv%4?1G9fX*}~xUI<_@xN=oJB1JWve*;4qu=+{xSLbTVrMXCKKGEVncXXl>*Fy=&EFj zM1Y|{K#tPM0qnRy$SU{pRVNMK#r^>%y_5K#Uk3I({w$ic@t0T76U>}L8C#26trv*- z`l~a-PoOPoGr))ZhrlByw$nm$dci=UgCwS z&dkx4LqbuRSPL5wq`K|C^{>j{KPU4&{qu+*iy9IpeFbKTPMnz16b~17%)-J}Y=>ts z0o80zd99oo!7l*sZ4O*UTd(c>4rX4Z{BBP8=nPMZ(oY4E@!yFQG zzKA6-P`#kOfjKs{(hSDjaCQ$L`%N0w&V%Kr18nzn4X>Sf4b|Yr#}5&p*$0BT^Nk5|35JI{|sS(sod-2(jOB=juKop31eUj_Q`>s+k3&q zO9hT(2AsFuM+^Y9}-nM%(>YWUNhZjKO3jx|81P1UFAeYsk-erX0~oNSM$P6eQ~fV`Y>W~JfWqM zYQ`@_sh;1rAaRPprPvWS7v7Q-e61Cw=x~sb-+iD{Ftv)wVrQd#MKiY;vi)MbfCHJm zsBVKZwD+%W^gm0xo_&$>h0DR(nmw3r^bo825G%i%TsA}?IJE5Bp_p~2ifmAYdWX7+=%2kO`=X3yB7n%Qlgnpw_&0>a-1 z(?f3y1cGsUn+KHQ9!Aa^q7bl!z;X-Hcx@sW{ z>k~$oA)esdAd(SZbL-u~ywYN7m=8&p8=WQ9dApk)%{81MIWo}$w9VLJ{5io3&x20T zkGui=`4SdH^S+D!$DIE@6lFg9<*h@4{XN;7%M z5h~WV*Chu6>OARh-Ubk$MwnOop`_SVO$dWy+st1%cc0BCP5h|T8r2c7Ms3$|lG1D-&qE>4`49b9-4@PF-%Vq3;B!L z?kTsdi|vcS(%Z(0$8@k4B(j$J|Fcq>pg%M)Kbb82MMKcHzT`jaEq~;RD&l*@dNv^0G!73RiQR0 zMYV0wu-yd(^ShXZ712Y+a9=KX8o85v6z$K z@>QkODrKHKB$IBnLxPD?c?Amv`QSt!cd-ae;cf(Lg|s#~@vaPTOElm@;9AhTs2e*H z>XtU1el2B6Dta1P_BiyDY&ow)P^WYMbp9xKOx;V`&x7%Ui{xnqxee3X9>olO{<}h| z{h)UeadkJkbENx!BB<%HxJvbCj~7KAZHzT zR?M}v;LFZ;Voo~8wmZkZ7|z2ua_7Ei*bTkhn{Ru_UzyHj_Nxe@>%5B^(#%_wtD2Y7 zL6C`=EB$T{_%ANNF79z7v@h+KxDw4T-Ht`31>~X4+P+2feM3b6-xCSe|KFdXDVDvD z=`IG-Dpr*#M`G{0lIU&XKMJ`1*s}h>#x#p-U&Z6M8-*)6Li@R z#W${)URzyTz3ciOm;_IcD2Eb@K|;#)x*GJ(^atX2jk+9f4t~w+oCLn(3DM)Grw`P& zeeOmr;P!kF*eYJ&q+t9aeSKYhM-s|kW_By9Eo!r|aEw`1u*)iJAqwcd5@T@B%GMZRy zvh^y%g;#gQaTO~#wz&T(d-%!FL?7=u`VXNJa4uDvVL{UWS*3?aAP9LkKzk;4rt_){ z6bmH{qkcIi+&%o|OTvaatorFoxb@z+W9NVK+CWT){Xz~Jd$0we&vO#7Zah9xWvJXZ z_MtaMX|H{)4Am-R*wEk$)jDEG6NzdSbA&&8>-2Jb_F4--Tz_>3T!8QY;l`0%(DonK z4W0hd{ZSxf@KK;?`B#PyvLv$1VR~}FymQ7O6ms>n`;jaDd{#ndLdO@@vrtVx+v`J~ zejy)KrScNwP{e;9BH(`_WQ4-*tax)h-QhBFla1zgaP# z3u9zzV=QVToC{-ZTG{5yTVUehe)b>;0ZjhP{Ny@VFN?cJ0-S=MR%@9}r_d46{+kyY z4`Ks=!!3*pgkH!=uB+AQS{=sx_Qd zuB zOsaF?m`f1qHY#HpvL%PGQK!BYaw3ANgX^SK9nDNC`b;YCOe!8viaF0wdc0)5u1viq zv}vebYov2-u)biF5@(DzZI~o)oU5_)lt=mW@34Qq49OAuakE$cHY()qFksjjB4piB z`}-T{@t?f;9f5#mz|tXcPNU8w!~#nyT_x5ePJB7jK+8Bbz62WWs8;PrpfZh+6wE)i1?d9X7cQ zw`M)bc2JLk1<|6nj4!wHKC5Qr&*WX5fvnCD|4B*Y-JJhEIvY~ZG!)Das&)eo%PQd< z>Ouwm@GF4$>h01GKh2}VY6&yhw{N`B$3vJ2<7PR@7`rJ9Wje_uhX{Iq4g^ir;V?Qt zZO1dHuv=^(?Q(7pb`gwp?JH;L9_Of^mLb5a;>p?1C-&qJ*tWN_`IX0NNu? zGLfJQmL9S}lcf$QZ8PKwRg?-3@^Kk10b!$1VGSt;jR5|tXmJ5#m`Rm#+JI+;dFW~4 zQLpBG*N2Ic*~#KYi66^!WPvjZu!)jbyZXo>O8AN1M7!P+6~DnRm-~)Si~Ei*UgKF> zOu1?mxe3G69Y3Y&btLOmB{)-my(qDpM>+e|HmUwKu2YrF1_u7OF<;K|9`h}d(n^oq zpjZnId?g@NM)#o{TI@#@oOo%0K=~1PZjd``4@EI>EV-wSKd9MCzO$sqs8>G^T00Nf zG7BND5US9UC{~=bc9NkaX|ZC;nMvKyyrPsRE!Y2ZlKs3bj0_Q~oc+vX*_8x)Aj9{{%j4{U+ZjT(qIj4q{TOnFs~!2oF>D*C+uEcvVp^vvQ*HHs zcJo3n!ypWE4syLfQ_i3W4Pr}zWRBlDT^=+6#i}T4I^-EZ47N`Q?HtN}%sC=>tTt72 z{xgS6UE7RZ=j27>SxTfa++gtGvw;<3bNe$`N>8p=X^(+JXUkdWz6b^ zF#X=pA>LXe9*TVc6@UxrK;~mR=zPy*$SnE(ccng4_@Dz{ZJ(lP!HbccV-DE?D`+oAt!x*{?R>!EBR$Mb78qaP<4j?E#1isMh0d(XtP+ zU9d~0Rk23=%TPl)T?_5;Um|yejBiifO+PAud8$oS`xN$j^^18#WZxyr4Ry?i1 z5lYt*W!y;39aOvoW^^+b+B|OfgdsPx8tF9eJD!XuFfyswW!s8ryM8-MwoZw>V3IKz ztW;HTbwhS~?WOsR}X$eX3FDIV^)+`TAS_1Xy z*!MwK(Ha@1Ek8zv1Ph`Dk6K)my#8RMb+y51XF+LaLEd33tAZC*p}99%3UY=Wdp|q& z{&eic=fqP+5l@d`G386^_W;BWdKMIN;K`e^Xm#q8!O7-P8u|*GXQ@Lgl(R+J*AHk| z%LxyeI{b-J$rKNhf0fK#3&9wb!9=^&L(DfJYqleL)Q4YGLQZsc;Z>=`y;!NWyd0Sx zmmjfRv-C|eB|-p`!wE$_qKwfeplFt%x1cFUKf`lM7YNoUe^56~A;Khjn%ol+ZV~d) zX=zEzc3AThBbsowAg~oA<~x&J{@Zc*=I&fL9hN9;F!nc-=jD zg?l?QedX}RcvmHP`RJnK28SSN+ZC#HO9L&RxdYJt3E=eK1imX-lFZM zK(VStccq9|(>E0~Fj-acJ>Pz|*f$x~4YvkQBN83gMe{XMc^4RD19iLzl&Q2@+FWBj^3|*(LAGVF%*U#lVGZYzx?G(j^N_fIiFnCWmXMS#bO8V( zbgvby*4Rd#NQW-&x1&T|AzWpRHgDe4A#TJyvYoAR*4zI|T1`XK?xnE{o<*qg6mQKr-`>A>=53rGI_|81LyShCU-C{CT9xn6+4aI$5 zkF8Lls0T~z@{+ZBs6MKlFHg=M=FW)~wH25WU^0mv`Mi6}%oAebyz9x9#IEcg2U+dH zIK8*5tCU!S{~I^aSK>DeH68L1nP*p-CK3d4lzer|R(7_evgRD6J(vmToWfJ;!P869 zxyuQ~4aZX@W@ES@R_YKIIOso=gi#4KkB%(p=wj)f7y9{4ir3R4Z|!sL0ePauKaW+HD6lE{Jj81^dQfZNHi$+7n{%tCWCwA-@g* zqmh4%Sbz&U=k~nSOWU>&z=!_?(*LRh@ysE0pt{Y+dMyoWsp%XE%@mD%>DeU>GG=>g zEhSIsVTDs0kHr%Dh()>2BYp;((u>!O1I=~JR&|O8Qw@!VvZa&KwG#F=UU?ICQ_Php z6NWH~{Z?;r~Rg|52*en{WJtF3=!izP&{KBat4*$j)1U8Zm^*Q(zLjg+2&T^SO*Pzzb5P@?2x)*WUyvnV{4{m8$#Lw_e2HHw*>gp9fs+JkrPpbsT= zuQ&|{@#2~1f3VN*w2zH{*YU0O$nW*iL?f{rvfoj#U+81*qvHypJ27c>CSY2HxM7E? z>&eli>;D%|P$b4ySekmv!>Xfe$wC+WngkG%s@aHJ2bSK)S8$#hz~{jg=yFcaJ}VbW z@eCXcCffjMPR2>RvUgl%crKb6ksgq5EUB@R(C{9oO3S{sG%z8hg&$i zK+&JOwc~&nM3Lob9{wpxAQx~rwt>ATsd&*>Q#!x)$O16&ZA05jz3HXRoPuNjLcB@P zKZDXLC7<@`1chz`jkRo8j6f%ty9%jwfk;VwXtB2z3_tQnz+qHc&x^BpkJ;yxXP8#__8luv>XrAC(orV4w>8?#bvT4~J z#NB~qeAq+ZLf>A^fEtg)8toMJYb5V@9NHfL=F7!V;*HL<9@h}>H~oeZ<8jCE=JB`W>fxh9`JF|ZYg=wxOZx?>x~ta<#h{tW7^iZc&G8r~dgMd!h5d)Fz7kS!GdEF=A6!~;kSPG!THs`Xkd!EIBISt~Zy-w(l3? z2A3!18~c(xa&NCQaB)PBl(TY^8FpD1L$KRYDPV$e9HwoPXU|&^X34v{&cD8Sfs1q# zPoSGnIze%q6-+HjkN{x z587ZC#}F47DoayF4y6Iy4|p6U(7E+STEFMmn5GSrPzmy1E96klbiK@OP8 z^A$Iw#&e;_Uv=O-d`o9=s!LsefJqBa}S(FU}7$b{qkH^<0i9p~?;AMRpdKdTIU zws?ltbbtP5uymNi!bj3;oO7jppYUw$F$>N!LqxXIz04lp6FV4<{(u*M#HX6=KS@4kAC9_<(o&ff?#XA58l* z|IL;ill>hr{u)LOoFOEwLGqJqQ%NaHQkFnzYl`MV?l#ItOG#$n26*x? zeCb^Nl!muth!gwIvxWA!0v^s1aE*(*AiZJAV#%5>u|Rz?`L(1>xGC8BEG(vgHY2Jm zT>eahk`dyT4n=d}dMcT{M7a0e1)LdM@0y@M*Czp2>e&)LlH^dNfGwXO_yO#v z;08xi!RG!YAC8qwHAjD)^Z-7HwV-|N?t#?1WYxH&@p%H!R7hzkWgB(hh;Rn^RL15V zskHijhtScPXbN;gpoMqjBQYpOlXI_uPI9U_(Nf?3voPM~Bp19>>_>IOGuV87L(X+P zhmPSb%D+U}gLcldGuo+FO!P8CGvBB=%zbM3c|qE1{=0o8A{k4q?1(HrFEQ@ndDh21 z+0n={$SMZaBPGAt0!k3_D;yEfT;g2Oq7Ma36m5!xS4G|^G;OTY0K9l39 zj@1O1p%Oov*-epQVsKnP zY>(f+f~w%nVCi*y@73NDr~ssUru32b=+V8@Kh{0k^vV`$&xFcQfR2+P7Qk}YHmi%> zz9s`CL&eJ43e+#aK{EuA=2LkI&ibm$=J^*pN{lh=wyO)tE+Hu)6Yl3Q?oKfL#8EzV zwL`|vIZ8=Uv5`m*h(?jfKgeS?xH^J_%Rvp0RJ?9>aWdc zStQ;<8VU>YMH&WMPynT?eF~eD>R?tyVuNNYtoF0oxS;7)EvJ$P!AI)ndg1%Oh8lC{ z(H2f~<#j_r{)gh~C?s8?=g2+&5|a}YRf205Z#lk%(Jw@g$MmrKB4dlU>fGR4c!XDv zaj(!fDILbLL}_Y28Sv9z|3g2qo%N1-)2@_t2!nRjBN;=f>|)I{d@VANu;QtGAg*8?-FZG6`$y$MDS}CLe<}WCrgXj+X%<|_(c#e zZ?8Ihe=oJ^MDf{i`cmMl2v(Ol|9TjQzYYP?w}8^GO|VfS8%utVC@LYPAo)6uJp){z zz@SgzwPy5nc#;l&zyS=P%kv@}H^WDvRyU9rM*Vv}EogVKNM#MRxK@N-QrDc1l*xoC zA<)(r9?tcvjYW_3;CQYr5?gc5M=!V^NYCxov=mE$40benvKHtKV?uUl#;K$@HYPW;m9&A+% zUP_*2=*z=#AyBS1-^yQ0*PH!1W+Z@$luAN7jfPm;h>`Xan{8q`e1RfYh{H2pY6zs) zhB7l8gv~t8*8j26d-*Z+-U)DZp=<1k_y#v;`e6;=yh+2yOhMz0E8Uk{MA2C0%4Z&n*C?cZZqVJB=^Vkpy6?gELuZ&+1arkJT-&v- zQ~1ZM(qmdoq~62RML-)cN>)If*0km z)X#9+;;~7ysZ{!f9uuR(l1>LlZh4PqRig;e~|&0pZNfkrysfFx*Nuns+tJN<18(+I%ue1>s;MqF~^L zsceiOpH96l$<+~8AHT}8uRc?T?1$rSJ>-Wj=$TTcN-JOCO31RN5Go!YEICg4h=JBx=rRB5eE%nYLH~*i=x;5+ z{;c3LBl=X_#}et1+qiPYA^Pf#Y(&Vub<2O+jq6{`=2(NFf46&z1a)!af4}MtHO>yT zUE{zLVFPm3{C6!tgdBOHkKjmO6bQ9{^V{eKtAht@`%O%`BRik8*F#0AXedAU91}>T z;lE4~M*kz2UdNxiXzVc$VYXIPy`bWW5G04N7DDBbfU0au=9_Xm$B7f&O`yH^r(GP{ zc!0}yJK2}y)2C)9ChURpKZj&ZkH_~tvXw3E(gLtfO3#NsiXu53kf$f3L9f`+*tnm+ zF!^)T<=mwss5c$;_{DujExPyD!fZN9iQ5<|t&~C(hZV=^e69e4K#wgZg2g>V%>F6r zf`N2DyUBzxA&a(F!iKhOdeHa)_{tm++Qbd}!H0%vQ9f9j}`$1N;}poGljZ zUDpt9VP07nHjw|2+-p5KNOMtW|Ax`e(e!YbR-;3tYo@^clK#6MNq_7!cl1!SM%q-HAlbcWg67BSs>H z^*DM3B0{g!vtKYZC1iaCkaelz$U&-)S-w)p}mG6rnd+y&d+mqj0X6k!4ly5J|%TBzJcLiMgrmTKAK9J1xv zp_!D!XoHhCG1xbUY%66+#EX|CF~v()6Pwnc*X{$3H!?plpLeOH?C)vw7Qhv7Z_<2S z9uq0xq1N&;FpsnCJmKVpUG5`VKr=!&`Y-;Mob+xl1#$WE?4DMrK=toqJ{H+w{C>NP zZFeN&+i%ZDI3CL#z8tZ?@io{# zd)j#|e%hk;d;4@T%3LU!`DgRXLc{4kP1_6o8+CshG{E(R#rG<)BWRzDSD=Rta5p0V zvfgpgN=Ttd?aUXg{Wj#+jkCS|5MlJ}X6Mz%TzzUT7N`BzI1ilL9bh$ThF$gzzI}x; zyySj^FI z00)|i#|dEMt=eu6KB=YU&?KIoDe2cp;}1E2Vq~RE())b+5Ps=ts~pomW2N+8O#3dO zfQ0rtM#7EnM(0U0AD}T60sy5qqi;_FqL1EqnZk*EM!u&XU_^FL82-K{@UO?1krhFE z?2m?Bn2`tJEkB?qZC>?X^f$KSy*;HnY(?OVZ(90Y@?Wkj0A41U4pywXkNcf zz8mQKtjca&h4pbem=y;eQj!T{nc=hi0$%Y%2!A6BWX|y~w(^w%JkJj0jyi`$j8eJ| zVoF-@UQg+b@|+L1vH$OqA)wiY$=~-m8qU0S5^bdmHw_%@(dzRxJLv5(k^b_xIo-3HeV#Is~5I6MgHWK(_g1p+JGan}HAFA!ef|Bi@1>3gp#6s-ZmqX)x z_qIV-4;Mp5Pn8ex+Ak<~qyT742^|K1`V8TqOy6CKt)1U=D_Dpb0+%elXNi?vzj2u5 zlw>c2JMNfBGc$UB>%er}KYiOIX(S6=#_^v%?PdoLtam~N6?4I7hlRAAxB!w)D($k& zTr|V>?7mE1)cDJPv%>Q88eNZQNU&%-wNT7xUoH#i5OqA2-H0j=pZiT7^e2i}=|Srw zY)i|4>pWN!$OK7_LYYRBW3H)TSw^=4|En^kGb#C9`f+rv*t%LF+O1*YW5GDiZ9Xxd z+V|*`1U^I`%fBSQ@2AY(Rc4z^*As0x>zGQ>rI6C`LM%Ml;vKl08S~8 zDj|s4Ff>TE=K%*P1m8UFkm7CA_OeeH0|?Lxncf|!d=cviW%qSq@81$RS$3i~d_7w4 zy{?7!AV_iaV!H4=Cj7>XqkT4*<`Q-xwti9ga7p0_k0J6TK<@n==IuRbjG52+ta)s=6WWmdl88H_Uh)))A)5c$k7U(?UN(`^HVdb_TX8|1cZPrHeo;#|zd& zx?$Vxr?tS!HDAnFfG2|3ZS5Q0K&&7ZMyTU-a(j%u*kz`pGZp5(26pAs?0Pp*9EBwe z)=M0J8=28-zxx9WBLKXkyHfpXJmR5#5t0KLj_<7Pj`y5^(?OunWcP1Xa?=2wB9Yr( z7Z5Vb(3)C?I)v!@0RevfFmYFCmo>^YzuS7NI?X4|c^`$5>_gs#)*l2w1xIEZ@C3>j z^fVsly{4i6EKvzpMDz!cMfpofTKgG6E%{=l4bF8g_3m?Tt~UtHr^k=lPjFahE2R3+ z&*Fu1yAsCF-J4@QbF-`Z5u9>u5^=~W_6UYnMh-vBg3v7`LW-SS+*iE4dZM@udLhdOE7x~l#R#4 zFh2PS@iW$0aI_flAf0XErIJKZ*7`4jU9;fn68>~{8}{=5^l6w(NWWo`B&VvyQ+ zRc9S?@+Rg5!n3ezt?}(qg#3&B=O3Dm`m<*;$*|cswG8sZU1DjiZYS`1{XAXx_IorC zx*!{d^in7Tb7wPLo0DLC`%w-*dA$YGp?wo-&3QQJp|sMEKzqfTprUyfqwN`Wz3bUw zHQ;{T8$vg%@Lx8*Ib&v^&T}*_bO(!qX_VA8+`d||5K4342AO6DN9bU+tHOL;2awK` zeGWRxl(iBKIdp`p2%+gkWs}gOiLuTwCjK2JOo~&x@nIYaowX~IVc1vY8&3?-3_4>1 z+d|S6pb@fM3u9!3Tw4uCm%GBPSlL1}Y0;AFYEcJw?oB+M@u4jno?E7oS`&KypQWJzis}e5@D{o4 z_pG19eq!&=o`iZpG`+1KNjlp*^eP9fLjY@T=*6- z3Uph*aTsUH)E55xZ;gOjN)&fB5 zX9LnEnE;yr{cQ8JI~h(gqkavbyfst#^+#!_KxDWrA|w76fn{3*>%C!<|E&cX`-^WX}@2kM;1Ut zYn@nz(8rmm1aM7{RCpr~SB_zd7yYFmBw9|gFEGnH7lpm8r+I$50o`&4E>VJm+Xf6l z8P1+s#~vKe=Ozf#U;#M)e&;PlP=_VV%$!A@DA+<6nY?&2&;8eFu84@TPhl?nV$_gG zy?TbH2eNn&BRo&@C!Pc>b+ZgLgx}I6OvLw2?DHUmR(u_xEEl8Nk_P33~pxm=SHI=Bd2)!3_Hcb`{PKm4RVGOgyc} z9W-;Nxq5~apiRcCZovvDQy0F#B_P;JS!4#~83!hiUTU~zJfY#z8x!Vl&x%(I)nU-q zpd-p_AazZ!Z@{y$?Eh6)?tQVGcyWd_kDlMMo)A|%W)5x7A4#?$UQFL@%{fV%l^T@~ z&l7pa-bOv;b82s~<#M@o_oO^PSE! zOilVn{*=JIVWhl;CUfE`=y+K-kGmsJZ|3>?_CK{akjewU2Xo-4024Ug8?hmL!^BUy zmz@`WJVZQKmOBo(L>#Z(9Ip)sPoUz4sEEo>z}E{Iw%| z!eg8VD28QNROQUn929vaM0-0HiZWEHwZLVHzPB09X3CY+t)wAHl0#HFRA}X#Ne6pu zY#68*Xqv%)6wMN;<;qoZvR-eKHlDSlRI8VvtndV{6k-D-V4b75G3a5c3b~%Y;nU!C ztw@snL&JDp$8=#dI*FaR{u1RP@E46Dz~{_0A4y z81TMa1BuVVVLa3~vFWIJqh)P!7-=Y{zm7G>C5*)uZEiLSu|#+fpVGb*HK8F z)0EVB(rZ4;R{ptxv-7-gn}SyLH|NT^H8U9NMA7eJv7%G_p?IMw*d zv4dJ07AB6Dl&6Q3^g`d0&r)A#`{o;mbK!2k%9(2mM6miV%@Rb;_&>Zs5Ta;&*PFcy zoFJ#F--V8f+R3#LspP*4oc`awSeRPcktE$fsZ`3mzRB}lZZ!tE!8hZ=@mv-W2^0U@ z%Oq%siVxdrC{+%ZlXElp>DVL|Bkl!R7w>X1JDxvQ4uG05%Gjx1XEh8%gWTOD*@cI1 zod>YSbF!tjY!1m{-oqHuV!XGCv4GIm0dhYH0gn^41ILX6Y8{fXS>su0QPO`LG}51AMWGQ4 z-qv0R9CupdwkI|bolaWg6buUvg)qr{2xVhMqm50kwk|dYg6RpUMO|;4NCmG1PE4fM zFWyZ4rYA8AH%}i5t@lMjM86Lvle4!=5M%#8|GDZiz9$ZnJwV+1E2OY3JtpGwEj~nz zrDCgg431?d0xpfG-Fqu}7IPZNX&7d4y2&Kc8*0*=t9cajh&{1L}>0pRFX2w6&R_GF>b8yPN+Fq42SDt@yllKvsO~Dtksx}vqo4A!v zm#kvObrteh3<1cJtj_XPv!YF5tX8Zw-yJ@U^BZ0leJ-ra@{CZBMDK{^M4i_tqIGlNedq0n^DJm()RE zWZs-yW0CHjUtUi;9|9$n!+@e1C+U%ug# zTHVr=*6ast;~b! z!Z1wIj^!b^@R2s+ah(Y*Q!c=G;!gp3#Qexqc8oWJBuWHI7KetOtTrlq5>c5Y=lXpg zi;`?6sS2z!V69pVkJUgFGo>XMd573f?rMlgNv-LcCyTk{I#YV$-|TgQn2@!`u^Qt4 zIo5qt9VvZt73_y=J57Q)?rp~xz@uC{_8+(d{WmKaFZ?7C65wgdDfux+4N+gai|pSc z=6Q1O4VTN_61TM%;$H2zI>OQG_l^Xj}EV z{G7|YZVg5qR>~#GN(0GOgCsz&HOvlN5%x^|gblB%Eo?ox>QTZZOqZh~qb=(bv15&{ zJ#>Ob0CueVgz+}gQ;}2RGYZkSRCL!bWmzFIt$`Mm|9A?|%;W@;mmD;B-q|YFH_NhC zeW!!26fqB7w)ur;pz*~nxw|$UV*(e?9u=-DkzCdC5g{`I*i1KK%Xb_&fNGUh(&bYd zB~uxs*BXmqW95Er<#u&|C3hWH+1cI7%BYEB&ps_=CAX|=h>tLVkon6zys;15GN|3( zs4GQ=S2vlI0^i+;)QK0PRU2j#!ye@qP160PFSFqya*J|Y3B6tqR+Z;3l;YTt9iMI= z6DZ94z%d*;8oF&1a9mgY{X33_=K*QicNw<^E-xVSk<@f^GYIerUJZs6DGlc@0St+#(Pt&a`GVWNZyf z|FFqrF1NcMI)c|Z=KPu5%i^hXn%aF7yF!<`fF_#pH_+8r)8W@`Rv`;4XzWaa9EpGU zAWJEdNOaAppARMrz{pp7@>^%!X#kfp>%EdHym zC6b>fo8Bm&{WI9MMnjr9lrQr)30#ZK$`&^VPpim`aOC~fg&W5Ag!~GC@L~Og>T>qEm1HpovZdsyGP;6X-ahF%88mRY@&r`;-qGcv6f{HAU;lW zhUSNB6j#n0gUl(05Y>fUrnyLNDD_IKZI~US@iK`Ps_13Jr;VrMO2q9?eoxa63j?Bb z^@k3E&u;a4i2w@lMHnqO%jwwinX4H6vYV&iLU$*I~5t!+**v=zHsr*zP80M4tFCeIs>UuT*T1S|g8 zFN6I(9!!Bo{w~S*)YLAH0_~^ZU;NbjrE}&;U}|uoG6f~!iu>~!_82rCjJisLppnIj zmUy1dPFA*(Uh2mhu1DiRd}8H530}vlFedR?>C#C!yap~rT*5vfv$-h4{HP3gEt~pK zG|4GlFO(mHW^w6XQlU$~TDn=OWC%=dQkH!iaF^JaJ;7zAT&UtH#9t^7tR#QwWz%-M z{$BVG&e!IejJOWr)fTjaZo1fiuKW#d1mw5G$1xj~dQwf=MVn7dQ-Sgb5u&jRMrWT&9B*m z5v5Qe8BgJdn6TH^u*G6LTSu zn=^4{wNk~~;^tNRi0ApCgO`B^4B@|Cg&QaUna-RyRG8DNjnrP2cE=Ts2x}pq;C1@?vGbeOG~=x zuJXddWVpiU&@x=ZUz5!^!%nkVuzRP`GCF^sC%1r8;J@Z`ek8yP(O_t_hoK5VHT$Xd z)lqlwr-RDD5*hS>jU<#t$ll>ER{b(C;y4EsDe`{T{r0oceW$PTR5yp*@ z6e?6p@7c1T8m&?ntO{nRl#R%jUxUo_FEv$`_R>Hi)H z|5>4vO!3A{%}$@y=55rX)(u3!DBNvW&dCr zhu+$;l)lxG9^ZSbVAQRYd>4$hCE6wZ^{D*NwcQ0#0XdUnkAFaCGhLq?Fv9~)$d^w* z#jt4~CjM^#c0h^0maJM6ofcMlU1Ye54`8tkqyePbz)}ZTYJ++lnVXk6@v;UKnRLkn zylG812?5yQt$~@7V>MohvV#DM!w@xNy|!A(7c*nl$Nia{)rf_xHL+xeqjs*R222iU zq{#t;E9!b>jcL*rStX-}i9nD5i36bmDeUZA@s)vFHBSZz4j6o%arI`R$ zPX20*vV4T)Bf*ISKPJ@hHM*$sL(Pwt>~mEEp=jf}3T$w2Qy`10?-nokB8V`hpla%B z`v{nNEs+aOLHQ5@qAg<(@<=`uqDkq>pv!8ioXkqSYLSBPMRa@?E68)xN{)i0Qm zMbe}ON){z#rN1qJf)&@QWtmuWB1|a52^i(7)uax*jjF8RT94W2$x6Ct{U|r0omY!D zT2lzoj;nW~H4O@&3JkY2-7OSb%~fR}S=gy_@?Ar%m$xPt@;#U5X|zk|fbv5CS3yq} zJI<}p5*&f6yPWa1dTZWQ6`f8=l|IM-3_`Je;6fph-lR%2fgXYOzT^?raRl64Vtf z(2AojeU;BZxu~=X!hdQj%alhc!~Wcoi7@%XMLKDf06X5(?}Qq?vW&C5&(Vu{xoAQn z9Gm3buuh63|B-epAV%3GsH41)uC%ntF~w3Q3XIfWCbbFR+69rPR+s&lZDc;hXBK5UsZrwM5r<6mI0TghpbfOy zSQ%o~!izjxozJf{r?gPv#FO-7p@EBrT}pXZcqLi@_H)v-tXahJLHH+(QsA040R+%8 zD{Mm++W02A1l0|Z6j`g2n{4Nwj71z<_z{rztI~Ju*s(gMaSY%e0VzS| zr#|(mRdC95$`kA1A1A<^AZyJk^D?NHtj1CKE#DzEDJN}?>0GcSse}6upi<#;s6W5>xD>`BRzh13H)RsgNv86 z(&_e4E|;-;_g(}d^YRV_lxk*@pO(gk))1WBvn~eyvAr6AFm3I%Zj06e@wxOfsJwh8 z^}dAP*1Ukc6u>2drueI_fowL5XPaHfMjApWNVf0tri}8>@mqFd|H)tHIEuQGmd~S zsQPiEEd6F>WeHC|eSB5^8??mXcdk~%z6bVT$Nk%}{l5EgVBdb+ci(+@;DHBl-+kL9 zJo)5PLQBio>a%gNM{C@zcf&ESSjge{Gv8OVEak$)*f=s7&NS(xMD=^V*FqtO2lnp9 z{rBC2UH9LIeYqbJqh1g`=x7Mn05F9K&>9lWOIf?@`IO z>JyZwbM01^eB?LhIr@*X;Afw+`i01wekaz)PjEK7vdFm_Ny$H!kn{L4Szi=zA`_i{?-3?#G2R`>DeCVHV z#dTk}8L5hzoN@Jvf1J>91&)IdSJeqTW%|q2drqb}-nNw)1%Z<4#!PIW+tuJxnb}!G zvzVc@d%UO6DHGHX4<(03O{TWdg`H2~bgkv&_3WIg|;m>i9B_$tci7 zyTi#k>*GY9bU6X!ST1*f*?lC`CiwU+l^WowsQ4CGQ+Was#1Q19Xzw2sVZGmlG zP@`OcD=>+_|K~4arB*=&Io$BI&*8CCCy;I`-jYY0%E|PQlhl?PyamuRl^=Zl0V0cJ za!3HL78^txCm%+gl|nv)jtw}|=l46gp@<(PIm1>^I!m=S6|^8?;g znvVuGY8F3)o|Y>%Q$5P6R!lUctjzjTgLoU&MqV^ftE-8kmDOfPfkRx!RW~J8PGE!NzS&A*xZq<-4 z#|6-K1eDiMLk>0Mg+{GcR6wy)L>EiwB8_6Vh_XPIytJh%hLIJaNdr!`uta{53V{G8 z|7>*=WiB{$R%B_5J&0Bsl#)`LX^UtfjtYuMcepU2EGSr0v64m!d0DQYv~i)L(<`FY z<`P)}m3A6M$g9ln`*H;*Q0!N9kpY#hkbR(n|hE1#t*Qlu<+)$=c-hb1(SMb&zWf^eAJ zYKXCYv6&N7YmT}EAWHK?^Zh((&78agf-yc7b?t<@ihGu}UO*b3hT7pAI=upo;KgXQ zin{QaI!v)ZXsN}o6hOVZ(g1W?s1%q&bLX~bJK09QuLcUJ_qZ^o7DejSWuQ}%FL)c17E`vj!~u#EYqG^Tps zvtm}c7M$_})C1g37rbuI>IC!n@#EOJb2oT?cd1;*j-9*V@%mQ*Dd}op<#0IQcDscp z0T#at1Uzg$+qFJ0ZGB)t{oNFkb9roSO4-G7B8lzWx5MF>#@P6jEG#n%w-m4q>dyjk z7BJuzr9FH02$;*}s2378e&+-wOfcp+kqTyqsRu zZb@0qJ8MGk_jz#Ny?0~#{r6(m&g}wVG>9R6hI{Y5SHOEX92Q=jI&})sXjC=Z`gdVr z0TUAwtNdrOyjy-_$VV}hcBa=E}o6+S!pegF2ISXfNTLKLU2wUS5ekOE=?E0Aq>aQ%%p%7moQ1~^>)&1Y{A4eE-lzytsI z)vwEkUV0DvwJfP~4^u^b@vQxQ96(sB4AdNBWTxB}jUp#sobD)%W&`ymr?1LXxGq{; zPC7VYu7O%2c05+cNq&W^ssbpQsC7!R(@l^M2S{pE#Xg^T=*5?v*3zy}3(?^;OBrS| zX8W4dNK(JsYsiFN!9rEO&53_kX^_<5r%YXxX@@cgRofJlX{!K0>tiPPkIziaO;u@i z+ihyWGHCW9a7CWO$8Y$1#A_=u;s4@Sug7;!J%+4I1C{A2wIFdY zw=#*9?;Ym{96JTwD7e6ImMnvZbe%-zEcnik3jO79yKV@Z)HvY)S; z_uIWRKKt1#u#hMC<7?=M`6s2m-p=6G+dhtWU-vqkjh(;>a$sL$`DxE-{Ns%uLb?$~ z2`RkybJyb;p92MgFBIX;`0=ify$gSF-J9_CYu6zyLFWkic!eD*8XBiCL=2|4`Zcm5uK{=v85?bo~mmwo6myyx%Ui-mj;Ektqa zcdp0Vul+0h$<=@T|FicVTvespw&?%5@1C>I-nFaBDs#r1kR(VF5e0LoQp=oB5d;h< zNimC=BuPY+AUQ0;$~mw8S|8t>NbR%lyYJoBPCND5u4!8?)>?DUFMo6NG5Qz-QL(d- zl(PwA-f>vqz}3q0IFh?h0EYt}QK&q6Rf?4CtuVuk5e#8f)>15v{ySE0o`r{XuVute z7oEk*gt=I|^)FZ&_doFC=ASS(aSisI`;EPd0D%_F$T*pXomck4s61=|OrYrIuh_ih zCoGNr4nM~IgdY+YVNT+5MnOKtaJ?e+wf^e8Fjs&l5UGI^7cq^Q~QG{T@#+)5^-B}ACdf>)q ztlhc<3pOmovY1tP-tbE1p*w7W6JOD7dWqGE^D%GzEG$c!kL76#(K7ZKhA=^}856;N z?7OrRmG8=Aeyb8qh+rJf@G#=GufoDLKO!n-Davb~V^ZeQFb*D1$Ksepn6>6-EZXoh z*2XP>(>;g)4A_^Of%)-2V9|y-SiF87wjD`;DclD?hH$gwC{l9sFerM#AOhYAL@_3@`->w{m16~KI3r6683E4-puyV^PER0!>6)CGR=o}P4 z=?qL^-{}m@-S{1TT=y3&ikpw9gq0XD(Xum&Q%B?Q-Ik4bGuR~x47=9}Ptb*qub;3b zI}L`Q2^Vgkz|xo>usHfZ5EVBYQ86p9Xw3>7KJ|;Nx0LLi`1rL)z^OYzYdBY;5^&{V z$c!ON#&>CSy$CJ z{9gyC1Uw3a;TsQL$L13JMCr*Y$!M$h%gEqT<_l-`I@1rR4%}c$aU0P=UNB zPqb&dCOq>Tt!m$j{wIHFSV;r3va`waTBlQYeriWmMWvKSp}zP=3B110G{v8{R6?_A z7Zt)^zI2FHlESer=(o1EO6e5JA3l7Dii(P9VVmdF)YM4Z2}r3#q$QY^&3|kIl{rGQ z6wsnuoA`iJf3q$xU%nK8t1BPaSA16K!aYG?Lqo%KZ?K+?jg6R`#k4fuC6cm@Zp2Z@sZ(i1zN9NMSj5*6ck*;)hh)k$iIR7YefP+y&gNBKYNIx z!fPll$QLWQJ|C}My@tY~8v=SgUNXp37fL%cOFQe;P+yDD!9Gz?lU>5T=6Fh_C~!o8 z3Bx#OG1bRKWKeNb52O0|Q{@}W&!He1>uO~zSQ?9*oU-8A^H-{knw6RJReVwbv6gd= z(Ire=ej*_}fBFm_mm5~I4G-==6y>*4&?x4ezAjmxuV267e$_*{zoO^Pq@k-ea~|^= z0^08GPTaj)BE}y2#&iC*x3|lQ;xa_Jz)uDD49li2i0JRr>qyBX`gHc~!Pj zoDgJlUGfyRXLQtn*RS7BD|1wB|NQ@t)&u|Yz_fY5hUNh)n)*#Bs(hkWHno|5)T3&S~=VS+G7YeRNx+an0kADrf^TolL>xU;q)8fqXmgHFfG;%A(Zk}BUs>d zv<@g=R0@~&fC5$FuwA*1u^xPO(aI6Fi(exx4UVu`EGvWrLTwB)0Et(l!?8>U_MFbd z!?%^9$0zlc))b|sx4{*%!xi=k?Nz~g-X|y&H3DMUw^>9wF=p_=8@ck$p zupezi%re%j1c}wOc+i5ueq*>Lbsm8}edcJ%VLM2f42E1n{R}WTQJ6lUJ~-%;r)ly? z;Ui#m29?Ez!YGv-oLAf|Id`aFu)+>g$g2EJH9)6mEUdR32%CVA9Sj7+b`r!|1rjnZ4IUg=suDbKML6yOp;sw0Fz-g|C+8XixX`Mj<^q8%OQiMO zBMc@hf;Pee5 zbJt3IHJf3Ju%{RV%PoE62^(?fLIRFmPeH=LH88;zAY!6cShTF%B1Ko|C(h(Sr?Lwp9Uv%R=b@+%HrJ_f=o zmbZG_5;PAq!v&{YlaRR@t%D6PL`D&n9fh)%+i+t7uE;2MpFV~Gn}MWQ*f9nR+7P#Y z5xPQyFd%@*2x-zjgo0xjH?|?}z(z!6ug7QaIEH2LClCnq!|weAJNmJD??$u*2c<4k zFu;N?>^-*?RUaQ?08^O4Bn;sJDa(#g0hS{k?ktpD$*reG!hv1KqH*k2CZZ3nll8|P zglnz&YYmcjZ-9puRnHh=4`ia()eAe?u_b#Jp0riM2pdcx2acRM4ztrLE2?hj36`XO zj}K$bu)vHdjG){80ZWsXqOz_G7EEFSBS<}$hVth7VnHMgH5eR$)7ORgoh$LM@h)7L zfCW=Hly?fBZAJt`?KtpT9IBctFp4R$#ZqAr60}HPBme*)07*naRO{?0GA|^ew7Ccl zObI3aQsoI8$W0do!>7*s_$gyMdILW3L#K0w4A}%oSK#KSY79u-dvWUaK^(YpOf0Z=x*@=cA=?LR-M<>^wnt&W(J$l79i4Vc z!$=QSY*~u??2&bwg?Sdo;KiBXFg zju4$C)T*Vc1KmR8;O05wCBRfYEM%^T@=Z}0#L~fQ=x-{*za|t@V5F5rx0Ld?nb)a; z^#3(r%H4}T_z0r8sa|M~y**#2t!a9lST#+S7a*Vur+jBQ1R{?gKSpI`<+RVHevV$^ z|F?Gq@(A=wZr?;(bCX&(tVb}6n%eiMe(^?tDTQa2#pe?DVfxF_B1a+JXf#fPXT3fY z*a;d5Qq?ue(!=_(9i^qEuvln;hgf|GXyjQEib&h!2R_S1_#ivUFCeI>umIDqq9RDG zT;@JqK4l0QjL>1m>sQaF1C%l7<`a!d%=$W}jTv8Qh>GL`s2o9paA zeXJ$8mih^bPOZzU+LziJ@2wx4OI%l6A3QjIe3{KwslN_Z_4EIH|6}#QzdSIdrIfFx zfhh${Igh!~YCcS~(oi5HZO$D6%n`ssRw^F=exN@9Bpo;<)*M>zC>l7$hfP#G5{QJ< zGP{U5Bd`aD;Ds5+upP94Fu08i1atfsfY-06UyM#mA;T&nn-M13heBo{W?6#-k4{*F zV?wDTg^VzQ%aUwW378R~m5y{nMs>9dHIy`3bAaCwcv;}_kHSXZK?34PK!Bv1G({L- zju?eP##%`bj}XC^5}#>N@`F~*31nQz#J%@ba0$(fXHfVZh7Cp-BNQTSutm&p1t$@J zU82hpU^p2il@+F|pPQ5rQVz{>Pj`fU^a=GnrARoo6%^4;q0cyQE&&y-W#|o!331Zu zC1g+v0jHPVSnLTK9GDdPnmIfoy-SD}qHhv|d-xR1jX+8Dnic*4X>oSgykpYigb1WC z5;WSy%0V(@5KdiK(`6^Iv=~KzBp?)67$CSM-ERs$u%1&g`er}JAqcyh)@Y8b88(b! z98*$?*i8k*^DL712$Y?OgpG)3zL@qPMO7EZHHxb>Vich3@jJvaXAF;swa*eVOATyc zmcQQb6hB=)ws6gc2cx&=frpCmaC^=u?D6La7x(VT5y!rL%7g$UGN= zik95zKwa#2(ybj4k{@X~r5~c2r=)1IZwOX6*BlXo*db(OFGjzcR8D0LRS&7KP+Iq0 z*n-NYOUSvHg4}yoFcJ1)BJ4uL;7g?avH`}x2y9^!HXquGUQZuv;Q=INu14j%N=!sZ zclE*=W_@j_@2SDUEsHP|=z$Hbh{>9Ry1`Ev=6!NMQ4j3aoVX1^d4LEi-0p*SNFd}P% zJ0`8H>|+Dc&ZXj3a|w4InV$_KudydLO)TfW&=8`s({R7}4ZLBh6|ER`wP4fEc^L70 z!4$@^?fg#MZ+#;Bg0NG_)`%Dp!6#fivl96&Z}C;2-!MLnH)HSVBLeamn}BQmD1x3YIco3!2uvx?Q25nB4X@D!M8507?Kc7Wn*hY$ z#D#(Po<8)=RcUq|b`##buEvY%XX3N|<;xcl!f7pi{_M2~-@Lr0wjPsHCIL$Vm_!(j zfalZNS5{UwT}Kv0fmI2s%I{R_^1DDN8W_>qr?-g$e9gNzAmH(OxbDJuU*8~BH?fQX zc>3%ms$aZPbz>nyp`_}EG)OzHUAv0$Nm}O>^^q1H3emLq5qz?*^fu_-!q24h8Vm+$ z$DKQO1l+Pr9=gy^&?X8d4G6Prc?nA?B$Vdi;bAFHZxgTQ-!ERg5Fpgk(=$DW{H?96 zO)SwYgLTu#lAqPR6_r=$ya;1-bXb6%G*BcHDi}1784AGda^S)JDgmY+KeeOfLo0?y zM^IQ?Bmh~j53PEnn{xcvrWenv#j2;HKEv<`T0eaJ9RPd%`n8M&t&)AcJyO4(?oOe) zN+kfHt+svqgree`LX#~lEEYym$LDUL%;sOehQi`}vF6fJT2y!&hS6~Wsh>MOp!oV# z6kN^4?cyTPVoNLOm8;hT?DBnJaCmxN>TO{^(OOz@_cnrFGUo#5?d-snysH=-9umM* zTU)PUVMKsO4<1X`a$E@3bs+b-qYXoYy(*ApH~@>ujE1^;X&UD=U4!mdJr=_V^%DZn zG#~BpF#`yGc}PTJK0X+KKmv1y#aFht@CFJCil)z_FKK6ICvM%kf&2F>W!~PrRRX)i zBW+|LLCO<6o;-d5Ex@kyI~9vAw-tHUF2Q7)n%)c4+*ZEurDgZPelIS*DX;we>(Uoq zVbB4F(J=w(w{Dey7U`)elYmpsJX(f1Kd3mN!lAdfSJnpIKy)kiwQB_!92{0L86tqy z;zZT`$3i2Py+>;P=YM~+9{86BrfvP>XzQOuOaFwT@@o4y6?6npbUojkwI6&0(Q1&e z|Ac%;P{o)T6mGZdJ1#zO99-I|L`z88uH+4&WzU2$^kLneWmppTJ(kDL#>I+U(eGEi zEWpO(B^Yw{32?Y}^9p`DbqFEar9uX*%Upx`NeeMAaRDyhzbyKCTFR!vlh}~C9;-Gk zLe$ztD86%B`J8dnVZ|s)c?S`@cLVOfFT=*@h$YWe z`rw&h_4i}zj`divX%?d5{x2*``U|3eNkG3?l0M_`#VsglD#6O^O;{Yg5Nl(X;dadv z(BqAEYEs{lvNmH`^jyr}_&ru7FGIhJVzTxC)uMjBc=r_RlUL)brAsKnS8txc$qNSs ztnWFIi90WEATE0o7Hyu7C9#WfzvdAbyO^L#6p)meNTK!cyFOIcmtg+dA2EOZ0&Gr+ z!>Dxtehk1G`Gj@b7GwVAIYKqOey>=BKl+hnol3*q+H2UbBN~Z&5|Oz#86{=c;EXW! ziVtP4?qS>hRP>IvU~B4HjG2dp9%{f8HtgDnWwA@JbmIaP6kQf?I0~+tcCN|b zrp3s+cS9h$C>p|J*p#sj%j2T3blq|k-oAz)6$=_K=d8iJI2b`$L_rU1 z;Yq~pi^lt*7cd4Jk(jd>?+3qt-h^DlR44UXIb+7-gb27y=oo&5w7u(K2@hi5sVqFKsSw{vb7&l~ zSzB=L-DA10vdwE2f7y3@a79Ltd>{rrJ`<=ep~47$GvFV@g)67xxBkZqzRO0x(9X0#W|AO5pR0 zHI)5d+MI_DYcp1+MWM3czF3c6jXX#EuNwt4so*& zM1AgSMa=$f7=&LGdn}J*LtjW&q!&p$*P?B>9aO8tU(CVHwrcUy6q!0;h2LjDe0Ds_ z>h8fRyQTv4+>gV-2K;tD5qBHziv@XH)I!`(9B_n2k$HIs?zC5mg3k&wc3#ZFrTgav zq(9TX!dtsmW5<4-8vibt-y^)6MN?FGg-J^RwRr7`tUl!ChWen3pJy!&@}cE=?9}>@f$_KH){Wc>|@d54NZjx z#XB+y$Z)ZR=o?LYyA?T?cHmCKO##Vn@r~xB<#>O_xnoO^*YXB!o^h;9T8?KmCGcSo zP5p0Ve)NXCFh^(&x5A1pY}vI86E1_W7wG3pg(a13giLlXXAFIUm7l&&htolT{w_YI zn#Gi0K?hKL|3{#s3p+ZH;b#SAG|ocHn!(9hN$z!Y{PepFW6fnjV&z$wdn^X82R znO{Elpr?C4tdmxo3ol>2(WW(LmI}5+0hQo}Z6TGD)YF*@D}jIsa|V7z(r3Rkb?p|-wmx;pF^T6Rg3 zl>2BVMt;v1>t0n=HT^d&z;(sGD7#y)v!}v;-xc zUrEVbIiH}OL4%5}7Yh)i|KzoTd;zBHYu1S%o%4x(LhCX+w(CnLTAG^0lEwgmt&L5X zn52?OQMYH#t;dfZ;B!a2YBQ<1`T1f2Cg|q8;5ZSSQE?&GGww-z2Ju~E3-O)Xr9MZM zCEBkA&r?qKR6sB$i1f5`hU_YNg8ZB2sScvPwhoiy6JnX=_>7EBh+;&H=4=ex zsYPlg6oR=yv((vFAD$2^0K9Em4lS#_w8qxzFed+4Nv*#~VKM~kUaKRu( zM#fNA+a%>F-%%=B{lWlz`Rb|AyXoJ_L-k1z;}gTkFDORU0|IZgZl67?Ms@W|xsT0a zMeVy98GHWD-x?Ykk~*Y8K_Zbikt>3Y$k`}$RljOm}i zRJ!sH`QIM~nChE~-b)M)Ff*5_p11TL-~Zp_F&!H-OCHlvgZp=ROc~^+Ysi2PT?6PD zwV>$E1DU|o{P7`8+vTQ%=du3aDe-5T0I>GpX~Z78BEK=3V{aHpK71TbN?Tm$Tp&#O z;CBFC6a1lZ5pq`Ti^7wk#{%4FamqNk4HZrI5k@ESZtunA2PaWG)PR(OyD;gY-<=83 z+m@ql^ex6Ph*5N5=cz#K)?N~SQ1U3^^BG=7m^AxajW^R2zIpKuG_m5PloSf48!ON^(ap-VbKzlH~Sy^BD!CHa?NZhp=PipVc9s&pYaiRPu)*ntnAL)CM9vr@qin&`C z;)`cM0+@94)?-2XDhz~2V8JN1WUNAQUt(=a6uw#}V3(+v+BoxpalI-dQ1j~1OTL+%SJ`pLv&l45R_DUI0cgiGq3hhkI zgY{UQvKM)EoZN*uqGqEyt zDW?1exM7ABJ&4<}1eb4L)X3y!H1{`PN#bgZh76eUwC) z?LGtcU&z73mNHnv9oU+^3XOf8n3M-I0jv-Gj?c(Eo&>AM09V+AooCZAKXDEgY+8Wz zi5v0j*f zA@!25+|k{RWgC}ZY1%S;33f^FAVOZzScF2b`G$~sARaxAF@Zk>eFVgvU!EX)&w5zH zq;wjwIwuYl`e~Sd2b<7XlKmd=wu)KSTQAbW8-@;=Vvbi$tkqzD0b+p%ck?TvvsWh!JUd z`*8Q;Gs)#FO=r}qhzIXG>o6~RHa4Wq!=$H2{Br3Efp9yHoJ_z^@r&@+*p--@uo;Ih zoRxrEq`anG*oq%he!zn0*;pF04E4jGFe!=#MFH(|H6S);8G2nE7{e$sE@$CxYo%Bk z$@s{)l!7_Q-(zvi4~W{d2)~{?C9I49T5)HfuSbQ zHt-g4hZ9AyHi>@3WUWGL|3^{GH1$?v$I*=#50PsSMBn5XqBgF;nk`GPE@=rY-cdMV z!p`#k%h9f)BD=Mo9 z?0&K~`jDEv8e_I$6=+FTC6$g!F~WBpRH>(!`k#PP1x8b!3Dopw^-NI8Uv;1~^Lkw{ zCy>#v76)2f{OJ{rkW5c1pb^@;HTPbsc>H?^O^a zs)xVchu^N50Zc_%q^_NqnnYgyHSrT=-%zfh6_h@tdY|ci#B%lev(M;n%FjH$wL=5_ zToZh0zLk1C8KstBjR0Nm16p5i+_*8_N2FUbS}*IvgXPoGOiL625`nq?{kQipo<4p8 zuiK}reF&qjt_Bn1CI#r|LNtTQt58oER7uWI%@wI0VgZ(4o<9s#u9Eu9A;CkL=+_+wVt9kj;ex(ek z#8>6Ls%fCA_a$lEckkSWpX@M3>}Pxfs|l@eW$0+FRA7?y>T&UH zEy}-&;(|OB=U)|TBI&;bwRu;s3ze7gD5ei>K2T9Xr3&Yl^aab~Z}d~ud{+r_Tbo-X zkQLW5=a^WlWlWSG@`FbY@wK;KrkL5{K*^m#6kNY5fyiiC&b@REFRI^&afT@b6>tq; zcz940JP#i}*7lPi>gwO1x#gqWOJ6^<8VmT8xvpLGGt~DS z&Jkh;vI>-vNmnct;^oWNYA&*G-!-7StM8k_VY;LK{Ns<*1OM{C^i$VYboC9PqiY1E z<<^>xiNr}3D(dMHtbn} z_oLh>%$TtDVa@Ihs53Uej9$3VgY{V%XdM2GsmKUy7({t}J{E8N9zVzY8@vV+#vKuTD?quD-!ckaQDMq+CtFY`xZ+w=is}fzhG9vkN7F>C;YT! z7ILdjz!~nr?qkU)d0!2KfPRxVq8%hG^2pR_7VDm3&=oc%= zF!r8E!o8LULS&`DOxjvXZWit|-$C=>bFor6A{4@C*&wucA?w_4xYt+%JAFbWN~KS% zA0yt6h|QUg*<0tKX1G!4bR>Z8JduGXE#!3cbiVm4XKiuf%tZVg(pd(_`kz>yHXlDm{~bSW{tH%YTY|H9^CZq`@~L!` zwv{R?ECWo139J{R)lh?W=NBy8wF#esgXrN4EcIcKU9a9gK^UTBze;x*@H0= zfEOnGj5+ahu{d!#7A7pmtmxUe{_HZ|_g7)Z(P;64Q{-%bRPff`C&)S+1!EAH3^JxG z3D5@YJGT#2b(Juo7x6hO(bCt(7_%_qD{?L+<8|*leDzKsEB7ewG`|$THU>e;zOW1B z4R>JVf9N=wK~&@7^eWbCOe-=52AAq&>=r zy_fc*VYnV+{tm=uu0-=-r@$r(O;qgkSUzIs$rLz(v{qTM=h6<8HQW||NJikbOS%v< z5c-5)ev3v~<0EvzkMrdxu=nBtxfdz^D)o>94LuJqEB4=Uqw$e|R%^f^0V|kGnBIJM z8*bxoTmFhY*DeUfHpsmrL{Mu$QqBhS*oK6BN;+^5lel*0Fs_v!70^#x>dDH}IQR5| zP+l2BHcWr&;5ZU?CSlAyAvEd;(+ox0apZCmit5Yol^(Gns%%Jk*L{#x>I=!QD-mVs z!E8o4=X^^!y}2c2QO?<9?7y)SPmNEo{9u{r*5BzfexG+6}z9t+yn<##uV+dm)K%mEzPF`>?$hfo>W$m|M$0V+lT*BT< zM}?qH#tVgd7n-o=^aea=EkTRrBbH=uL}$PNYgk3}&CnDiHVg@)f|01{u;9f99Q^HP z+-zw;ADlQ|ejIx*XNr}RRB#4YI(K_N&Qx9!it+@8aH0G-&X!&P)pBn-?;-Wza*PJ3 z#$lK?lJ4CYclU}9@35_#Dkf|{n}M?CGE50yzzZj%aFbRZ{)pdxor9ZAA2Ec0^n1>U zjW}Mg1x<#hSbHc91ERK~S2>>#BS_t~9tPW(_`)-4GPxN#u=)1>duppLtT3Ge0RydD z44~)rE8l$z`#fd>Pd)IF4$gGpkpYl+z7C9dKgkcNZlM+R4akKg&6yuBs-FmLmevzS zKc!&GR3x7}deGI~Cw`sd6H_AK^4_$N66EPtK9;8 zbJY$7NNEKnh$fAamOLK3=S*PA>$nN=Ji5_P1t2L-A|XjNMP?JJyR=pkK+|_qud@zl zcwbU)@7=qnXqkb4Tth3O4&bC*twGGstoxYC1{etmFnjj&vDAeilfI+0AkyNg>ZscM z@#9AULHv&XngpZzxYBw?D>}_tu>b%d07*naR4XF`>(}s|fT!l|I|YWxGIKle@?|w9 zC(If^)>R)`|AYckA6oA^+CPI9xaQUt^z^E9Kb$jsv26sxv}npa(iLZ73IsYob>Kr& zvjR0tdln8$zt9>=`l-n{DGwE^#RLhIq*;nN2RJ_%oQR{ND_soZMzplHtMA0ELZvu+ z^0-=PpZr1jwlXe-7x&8Uh<|ZyT?6Xs-+!wI>&bH&EKs9abL}wITGjnZ1p1`j?W1URquTr;Cx& z^&$64nmwk~@wjPy)s-a-Fhvz{K+<+`-m`3-j?U+9Y1wUzPnc9@0@*{T2&urt#1v?~ zEiNp=%jy?$Pg*|j-K!D>(5FwI@#5uMRR?YVVI3LSnsb_U*MTm3fq&DA%QzhsTz( ze+JP9&I_%V){$<~dw=;we5yF{)x&@qF{TA!&=?0A`|7dzmjv{L2Lu4n>qmJKF`Y^~ zeBlmk+`kZWw#`R%??)IB!u$SL*!Igt3`Tk-D9Cv5Gkh45NSN*Y?Gp6m6RZeCY;Xtd z0+RXYQ#&M6(%2*b(Aw-Myz74f(w;lK1lg>v!*)aq$hHY1N3%$2#;2C!Q5(|T)Uo4-r0MjRT=Y^wq*Viuo zvb4l891XoB8B8fQYd+?uEkXTgs|dM_+M0cS2OhPN!Z{@YYUrDleI^kP8ehUfp-KzL z;q|Cc=@bbWh=&1Ja!zJoMf_?s_q<2ja5Yj7ufb4oSi-eTU>rvAzjeb9v17-jy(nv| z1VNNLY4#qb`9ziz=;n4npai|I#4 zKfTnW2^b6y(9=+QYYOc7VSF5UfsDf&VF-;$7!;Mo(TC3?uaS9VCB}UOrA(PY57J5O zKDS4FGC4msZeNTK1ML!=iAsyz7n9LC@d32uWnMUpyUou_kw~QF=K|dP0A7XxT z6gqt)n8X;eFK);EmS-4_sNV3yftJB{NIMh@XJ`^u-x&6s%Rp6gDZ}^(*dVo%N+zHS zCod$S?0pr4d>{OPq#aRs*vK`dsQd_95XKO)_HV|J+k`Pcfq{6Eip1h@(%;w!Bl<8e zCmv-juT@GRKfS+)k-T>;Mtxj&ocE(JguYVIN>+$=0e{tnA_e2-2S zfo4do+ScGWw(d#BsAEL@%flFhC;SNq&&Q*@?IHSLQ`AvOx1%1|5f5_!D}jOmu(;M~ z@6J4%jmo-v5`Q@Y3&z|Zv2@!kY`>L>wZ~(`$CL5I+$yv<5G22ZYlh>=z*8NF-Mt8p zo60c~8o-KO%TUowMT|#+qlk~I+k&?PC0Mm>4*DEJFod1Rx_kt;YAPkjmdG5zkc9px zoTvMO6N50LADeTMaJP;Mw|6+18!L0KH^?wq4Ch69F=zA`cIKv_w5brrV82+K*KS*d zs)mOmGI<3h*WvNAV>og-8#muoU{AqGkR-06Zdk|--Rn;5FcQV*C8}v?%8K9 zuEVXmCQ-BvVi22lufg*cDzK~=542!&_B_muUx2ysi!p1%Uok)FM=Z`>f{#81fck=@ zJ#$ez(v1o3fDsQ|A*-ZFO4+*!L+)M}FpiAeT`2qTKzfx-2ZxVdrt}r>z{%V-D0%+{ zqvBy~z<|9TE7Si64i=|j)3H?aMCcJtg{4oZ^{KlyVbX1cnSQ^TYl>$0lgF0&TS?Xw z^!QRaQ~9$>(Qy4nu{f(xK}Xsz&mrK{ffPY$W@e_A3g(*?m4T0T?%XNqWGJB0y211G zgWvI9I+)V!xK4)!*9)#nz!wUtw1zNE45_@YUcQr5G0&@CPXi5F9to%jK6D|RWsrTv z?+ILWK&L3M3TW!}_{Vp>UHbO~nH1vndRCTKh@z5J=Sv5sJfGz#_L2f;wKcEB>cwl> zuN_^TV!>oRDNK_xNk2~>djHeHr_+)7J*|SID)M~QUo1yS*kTRSB0;f^^s_9;&l5mI ziygmXoftG}W|U&ym*9!iV_HoK9`pcAdN~Ae6#QwyS3eVA zN-$7U`x;%{%F_Dy@ly#%G&x}qkoU6s)ilTvfpAu!}}JwpXx)^IUIZ5kieAV&Us2O%r*S-#Zw8!$@-DT8}x^yY=UG) zedc=Mmx;}q-kZLxtT&%{2E(XWR`~_l4+MfdpJP8|G-&zZ!{V#Tab_RW54pIw z5K~6x3RiRZ#*JIkzRXlyFxZ(6tl2kASHy#O{a^YRhe_2DWSMO=*k8L^m_H` zjafB@55cdEnyz{sSMupPiq1lYL$ zB&H%N-`(cyV`AYU2IZn*_OXro4k|y*fJ((8q2z46V{k5A6E+&#wr$(yjna0&;(O8C9xS`k$M!M4!n?Cx$^9DA^ zypc3lM)T4$OT=i~0^zLakVAKEN3*}8-LXF%fS7qPBKn4BCE=u;&$q>~D0&8tS&fHd zv66i~sn3k8^=N5wRHYN0*ZPLjx4>a4pdpQJeB0FTQ0BjFE^A!M@M{91QmOnHIj&X% z*K1u~nC_JQlZjVFH)yVwxBJJNmatsn@@7_}I0>!Dgu{V@R=t>EpLf)t;hFCD)m7QI zM1r?{YkaNOn?a7`-N5p+risT~%w#~M*&G-?&k zFZ;+a0WVT3^f@F&d9p*b_%Q7PWj@2b**k~K-^}T~rM;t7$h%o`Df&nZbf|?s4hQF* zE-_TlE1}4VJFq;5uaeG3B&14Q*Qasq=T@F}}W86i0vF!U&@w%ya#g^+Q zAUmuQ_PFH7BFO^A(5dM17l5Q+@GbgEjqoBQfYosb!#5yGHC*s2`bLRMKEQTopG6os zD)R4O@IaQ>^kI6DtYffZm$AB%Vvx$538$*vN7SaQ>YheW5-Z|4H-<+h*Wm zVLBE<^Z*8d$jEfcsUu4wO?Aj*h$t&1^uGFG)<#sth@{rG6m3?5aB|P+a6?mWyeA)W z5ZkoD=M`^^PN zB7$`gvDmk$7RV)($6GzU54=QmlL}}{9&%EPuuhkIA{gt4$*}4!Y%0FYa8!>vw0lS$ zY)v1LvRf@6yj!+l6eVpLQO)=)*7Rl90<#&^@?5lEy=v_i-*$;L8y0#C;r8~|b_!UK zj~Ea+s6F^>*r$1-qOx|R@9grwL?YE8lM6>Cmgbq5yv6LBji7`+FxOn(V8A0>vT%MV z@Or5I`;_VAL{?N=5;c>1T0)9aff@|yf*7MCBc6er*=-~Fsa~rU^PjiTI(sRyeeyA< z8;3e^A;Q53`CpF)XVbk0zf^!JhB+?9N6vKJXYVw0U@guINk$Da`f4#_Wr0XhdyG(g zG_sqquO)hj!dtl<9q9?!n1qm0Xl=1X;qHgt((BKyc?j5c5>7{j({A&=5R?(g`VYUi zVNiVH?jx2Oa1DiDsCu`P+u3TV(~Vpv?K{n2+;IXIUO?gJLLZ!DE~%cTpox+DvD@E{ zo1Sq>7ys0bdcveVJU9avbqptqR3qeAJQ~L?e7-*`lt%1%fN{NXgTOR3QRrsp4l_2i zQRxS+!XZ3J-{69Xzh#`M$1@KmA0ian%jOBg7Bw(Y*QY-(_LEcBd9rDilpx{pZQXpE zw?{u4@|Oxz4-oOUyroBO@?>`==$1#OI+Ei{WYOEm{k(DIT#y8K*^X+fknsxNtTvA0+JA(Q|8m)h(N66iy z3cr)Kc|Q#WY%-f9s4szh+w)Z48tOm+;DZtA!;=$fKfU(ED2W)osaMeqCUktbB4_I7 z$f7O7Asrvv1bksZlM={%-VIzr3Efo&46KrMnw}Ju9)>GKWwLU?b?*;zINsoi~2VZdFeAvJIi7*V6+Tb5w6tFUg z(l7Hy?e-vPB9`7e?ifK&j7+yHWG(zlk^;h|d0YFa2Cz%48w3N*I}*B>+^{3&C?zFb za$pgd>n!n+9(OGAqHnM#LP38O^Ei}5gULH!S=3Zx!w(v|Um6ts;c4_q1a*Tgz+&Wd z{gLn!bb*7SNwO2qFdZhaeMsqx~bRb_vR;mJ=&MOi2gU*Q3ovr!xrQ&q> z@4t$qZ%82!c)TB>eySycfDinkt}&w&4A+i5(i*QQF9h8bh000MMuOIHH99OYSQ_?s zVQ`pD&POs1(KWEUVXO*Q4$jUgMO2Na2T1`9w_R7oWQnC3Y&?sxX-!z(IMyMcq849o zaIh=WN&@b*X)27_Oy8@C%vq8eQe9I3UrF{?vV58MF5)YDqm+X=|A<)#EA&5#n7B zWelOCwEdC)z>!;p(y7{?!TQ^-!e|ut+EX_aJc zHD?C42gIM5om)p?imW~~-M=5RW2kx2YY4c-iUtFKwIMYv#5&&{E73*f#)!`;r8OD? zPyfwE*Yj5@Q0k2Lr9+Qur5oCR5#KlOU~ypo$77``RJgEl)AWfBCs&7jmf+9IJ99M> z(8W5B0mDEX2mJm_4vUlrK0_OCi8iyd)&KV5p@IAfsy_7P&ns^y{9eHqA0!Ywpl&@A zIari14f^oAh3NT8^J1Ex?zY)d#guJ{2yT>-JKP6z2AZ!#pY^kp)oI?aDO#FMtcDqa zoX6W0QoW?6Utun)p2-)y0OhApqY>11zFz0S1IA#0B~pGY^Ja+;NM$JC;^6)EA3x~$ z3)0Pedmlsczbe>`dP@a7I>U-hRR+Ub_7GHsO4&LxGvMQq>A^S$C#AI~v`Gp+^dfv$ zn9mXgjb0b-R7u9Zg45i3GFyiIUsa-l9pI_4FJblPn;dlV88JuoPW2{(-elJ@=vex% zC3vvLzC(%he#pN;X{Z1j6I!vgdXR8=V{rvOg5w-4_yFbx3_Qd(9G0dHuIe!8QhuYy{h*O(`gbQuJ%wKeH<o%1m0rwhrqXTpUq2>An?>i!xw(u? z=Yb{*!_;8nmzU&PlVCszDtwU?W%6W>fCI40dw90~Udv&NB2Dx{8$g@Arst*i{WYDR z$0*>)^#%!(_t$Zlv#C#I9>qokhf;N3pPtdbDK2!vAuHPn86Xm90uuof7Z+avpG38& zBY=w@TA9jh?T4yS$W@EVr2qtc)m@0TgghENnDF1A`j|g@P?<`DX)BNUpu9Gp`BV-> zD=*Y7W-g0IpaZ}MIsT*y$+$Q9K0QClmuk+PKmrtUt-4aS0bsUZGgX+vMkjv}lrWN$ zH$@B$1EOoeYG6VPS<@ZPM{I!KIKa^FaxC%;NAjW_t>rU+-LzmcmLE z&#)kV;(h97-=2K?75y*4J2#Z>g~dSl*&o|&oIsjEQZxJklQ>0x;!>hY5;n;Gyj`NL z;S&jGMMDEzreXeMcH#l=pQb`hFQ=3~gM_Y$A|`8Ikh`!SIm~PH&`;`{6V|K* zxlpJP{Q!t5mizpBAGIa3s;Vc_1RiZGd5XCX;cWGLxB! z(t1tTzTfaJ%EK1Pp3=ah7+!%IrA^|prj`pf7I_)DVg6%33J$VxlsacvQ`Dy@Ay?)h zhbaMnOFuyYmy8w#=UE;Y32|{cx3Z6OIHXh69OU2aB0cV0YFRqrlwJQH-Q?EuV9H0l27I=qZ3;CakVC{}z z9#0a(nJ?|Q96q# ztIZmXV9FZ#$?oGwz{a#4igOKkago3|vcZ&q=EH{_j0zZfr6I0h7O9Pq2Lxlug)eGJ zuENWveG&!=7R@i&(^($|qnuTm$I;PmNAxSAVErAsz#l5!03f3`;z6fNGj*)#4e&k@F2vn1u1iBp>C}lyHJ+?G4zH1qGx}Jl z@u}y6Ef;{hY$}AyTI~l9Os&DTKsU) zgxi(lANYoTKVWg*I1U+u|PNbfSAiD7+oE&NpZ*1od@UFN#V8z zhq8eq{{E}I{1@8<7p_klA*2qe&G!vCJbKiZxRJy9@1mM|tI8eudH0OKUWf6t;S;1~ zJs$2cG(u<_qoQs<`ot@?Fw;d6-gJ|*6Tkdx^uBA~RyG_u%^!=$r@7t96S1^qxht%H zJ;2gSgPTh-5yR%dog4IYI52+1`40(~y93EZUq%lWX722v*hn$H6|Pu=a)zb>bAoV^ zCL+hpHh9Rf{b+o?p{T|LGvOb{B??`Wa)dg1xPgXR#%%Jm#mFIc zB}XVQE(?s?5yF;Sp?gRS$9l@Wmgk9>$Z7L`Dztw=133#T_flU=ehU&5ea(ATc>gjD zk@aJ`D?=*^pf&uq@HDi2m%4q?%eK0=1@-H7`Y%mXCaVwp%f8Nm%=t|nLH!@3?; zVIbfT1gclu+5=g;Fg=B1ZdK)wlITEL?@>Rj1$81?)VatRNhH0-xbA{20X7H125;51OD|`SOB>_&54R_F zp4sk*S}l*RvL196BVBVoWW{h_Lq^gL=(E4<0iKnUuy1)46_)qnnSU?LmsT>2gld$( z&mNB`W6KZcA3KHvD;1eeiZW0a;0=cd63&{KyHbG^^n*OV<>G2x?Kp%Xtv)h+$PD6H zU~{uG7^!i`JsoBYwjDK{w6a}@%C)hVR{r6EsP?$m#jUR3aW02FFZ5ITM0=e2X7Ppf z(jEoF1dD@@E>MoUCA*mUtOr8cma9mti5V6Ok@c?uzt}a)44j(G`^ii~5 z={5Z&2rStUT3m$b1h$R5Eo4HFLY+iJ@tE9{K}kynn=Y^i@`3mfV&tboud)dL<8BJS z`i99G7K4sLygwTpC#N=dKxHPi0uU#sdN~y*5MtIgH1g&e!&}^~^Alu0rEoOASp44K zm>4U;-9pzyG09`+WT3ry0T9%+&B4ts3)CX_8h$!=l_y4B>vrlf(F|lW{`oJ(V3&fI zi_{hsu$#L8Kpivk$o)|}Ow%c@5z#5~dhVh&UB)64i%J!U%$GlO0FiVY4S!%3z!yc_ zAGV1AHQrojGc++3#`C%Yh;DNgG;&?ClJV5=tWbXy%afUIQSI2?lnJ>9%2XVA&or)X z3yhM%T@;3H_9XlP*Mms*`&M`epNxvzjv9&8CU}w(*%wP=g!EuUK9qb;A=RzWdaQ~4 z&VI2hO3p=uIC+aU2w*AYNBd?nuZ*2<)K{w+yn0W-LVn!p3p8O<&>CyZ09pmk486eRHD!1;GdJDP}+yle*9eqMi17Wh)H=wHA z%aTWMkgu8LsvN_hURbSr)zha6HV&0dG9uaUf`mu#W!Z`NM`_3ME~uirUjm$ zpgN@a2?jEgw7ytFhym?wlDi+k#3^DU8CnoXgg%XBsDEOh=+lrdDy^E{uz3*3;zQl6 zwd!3_7w-IjnIKZMe zr_A%JT9m!Y^vMRKjU$h*2)Fr|L2sKB7l?`d0@v`)kp`adWel37N`a*>NBgywjpo1T znYWeO_nVr5t4F7hp|>NnPlJTR{~{^*7245>L#=lq_hs2h*=scxO5{n-J4oDz9BWGY|AcA&F|bCTv* zg7i}UI^fh@;Gh4P@%ax*2^%f=?<>esjA^%*k2NRa-@_WSY^nAmzIidXm!K{ds`jvM z;L0jB^LS=&%^Zs&17j-CSkbg)$TjrEGQ~SISq3jjV58sh<;EK3xs&$ytA9>)>lzj)+P%LZkXCQpLm&SmjEvt9BEe5(Id`OJpcpGmIyw;PUw3JdCP zKYBUXMCR3MI+CR&P|>KBfA~=_DXz@@JHQmeL1^pGO6WpFjOEwoi}NY}J1B5q-hGV* zq{tewa?5$<)jo0kjoP1ct_K1(wq=@CDjYq_(hL4Ecm4d{S*GyA)W8S~f}^JKonwB> zdDL4*5ogPsa>CC}Q3d-f3jJ0@XTR`SO*)QkT=j1X7j?3%d4fA%`@wm)pfxuy?C(T% z6My`Pl>Q$y)zs3{l^v~eP+VwLK)34s#L>+?=d%8kG&{gD9tj2KNRTm?wF(y?;%S(% zEsx2NTK$~x)tXfd%+;>QP~xb^|C_6J=#y5?G$rGiXZH6>MKKWxwHqCcIh(=c&YIaL zrF;7U_D&?3>DOF)4tPx2Pu* zD^2ZU51DZv$g0nRtjr^|Z1Ik>-%dD|?`nv*?>|U@)-n!22(oy+)02VLph^L^oKYqB zGpRX$2Gw**{n7x4(gz{(`c4o?T<`wmytfze>qlK%^ zAI#2-@1a&a&4W@9B$3cRJf=8TX48`qI`lm-w-v4br)XUqUp;Vp-Xr8IfYme=FhEU+ zXryAgl?DH~&DI|RQC zZ-aqiQQi$c;gRl7wTZ+<(tiU@0HMowLb~#AAY|GtP3sru9q$TI8EuP?xN8BV!O-%? z-?Rtmwc!~{{n8DXV)g@!r}+WmBK(GT7W@Vle$N#1EjiI?2b5dN=R=(V7nkn#Oa&MO zt-0aBs;BN)Hc*&j$x_G_xoE;CLZYyz05K2EJ;AOfT&Ar+VvqI#+XWkJk}s=)fJhfk zyf-L-TyO@0#>VN8`Bjd32vbMJVbB0I$qIR7B68DzbHHyZpCEl|QCSE3EGV%c;;{t>bVo>tqU{@G1bAAOWC#kZ>iU__3 zhUzhbv-v>1l0AU}`;LuHAB4V2sn(ubFFaP!^W znBlrBAa80g!efo`<-zSEM%7YX%V#iDj=d|5BEbem_d*IoZ0zsK0Kil`DC!v&3Z(P; z&Xbpfo=Bg-0ZEM0vVz#q(e!fRBeo{gzlf|M%J6o(41RWJR}GC5o=+mDxpek^km~f{ zLGJe4iIng;SR5ebg8LM$(r2adQCnod3r@N$w4q*9+s?yx6A2;#4PXHpvIOI;>IGsu zb}kkVZWIIMQf+uy^ghWE_j9xh_TpcXXILc#JEHMrcn@bO>X}Mh6PlgCkK9c@7;`K` zF$vZW)X=<6rq+$L1tf}}4Y5;4au!nI~{&!ts#Vz*sEA--p2JwbI=< zZ)e33vcO{ba1ETOK`rOiI2NY=))e9P{GM2W6B_sjv!w_rcmbwY836K1bT5yaguQb< zKro{eSO5ZSjrp0Wb`q>cM0_ln;!&e>M}>MG$WyC!RG;wY*s`IkNjUfcJCJ4~=`+Y> z#Ui}7wzeZquV*_n{hDmzh8787=V2&PeoU!881IVQYxr%*Pj@-@%C; zBtWL{cD#BJM(Q7G&ygRN9 z&N$}V<~-=vsk+LiXU`9FoO`+f50hG7hE3^a{&8Io0(NaUsvP7bot(TKy31?k_3ich zFG9xdZSTL_6~K17Ys6ambNkdGNqm`Djll~l==|9@=AKza0)H@&;~T=vrvD1C+{qeuSi zKdWZlWtu!3g0#GEdJ*ta7X5Er_>+gv z@2Ibr#t%EhB)?_MZ}BYwpAA3t8X~i|g;q?sK5H%8vON|NgW@p!_hj zd4D>xWUz*{EjAI@ZZN4WYZT?xWcdc%3Y*6>dEG0=R@1S>k}k_Xsite{*QP}Y60A5= z78Q{JKwr~EvXza8Hue7V}Ki(i;lf^JnwHjK-@^`DK7EDFBUo(Aj3SmJ

zXpa@rZI?)Gg>`bRY)m$6l>tCUoyK$%0URx>3L5c+8=O9?mFPUvC)1=(NcQn@|#zVggOM#dF5^Fn>x0I2U5T!Pxkt{!1kdwYW zD@=Xywy2TU-WU2S(VcK|0HQvG(GKK=CZTqhnOPm-S9|!%_r?C}MHJ7p1YF-R?ExF_ z(LY4l?c;6ZdK(efFPI+w)L@A`rr6XI+4_-|A@*>CSFccKt$r#!X?_4q2Y7oXQP z-_bYowHNcxyYEG&;P%JPH2@3tk=d{7|9{M!{^Xe9WqOMO^#YtK;J5F^lEC{``|K;D z%g*L&!?o?(Q;w59y~|f4&1xErtFN>zIP9E`Uaqwl@OI-l%m1GGv$$;Q$#xgQo0r?e zlPbs(dIUk9P!S9K8^d=2Sl6pY1MLKY5ZeWJys=M!*wPOCVNX}@0&?;&25M??2M1=?>&>8fABhWk zn!zi6FHMyFtShjzs4>RoQ?9BvpX{ua+U2dYi2V9W*1K*Qo7m$d6q^ZY*z1tbIx zQX;m0m##Bxx7EL?TvE;*82vyB_3~MDWBBJd{VG1mJaw$I+)WEUbN+O?AB_A{XwDUj zi~2cpxuzWz8%(=RjnDF@q*I4_hy9d@C$EZ2*DG4URifh%ty?nGb*VdfbK%MJ)4sC$ zG{5c1w({hsJYeb@-){S7x@^e;!0<@KZ;JKn@|Gx0+o~_scjiPL;6cCHE`55(rFbKC5kU^GKP53&UQCjnCPa~#XR zQ(SScTF5O>0p};Lx%ndz?KoUrP(A0wbo+}J0nYXJvKb|bOCN@&C%CETgI{eIcsD_1 zgL~k-e+2~BJtc%cpSikE?O$dEc17Rwa4&-xoM*X_>tEZ~xEDMCr2m|V3p^Oa>R(6b zF|jhIE%WOwcSMWqB{p6VxU?3oku*&^<7XzG?7$t%lRQbFbTBr^GFv$OpfAmT z&atdFk}lQVh0&g&fpmGwR<*c4tFiWC82#rMT`z!b!M|th1;N%6)={zzgyS_bcNCph zE$AN`faIBF*z_2iXJrpZ9}ASJKhZ~}1;Iy@t5x6%vf`}r6qnyUL)dzOgD1HngmgRJ zB7nI5))oGH9yB2axfKl{nDn}Hag*1+6Xi~x#0$XMPel++(oR%K+&K_?^fgp6?TcFO z$HuwIH?_y32NjlRgr&dU%R%1gdOnFBWlb8yXg?sDw;nEvj5>8@ARVbk{oq&NzqjZ=_#%3KyxyU_`z(VlkaibZ;KCHtuYlfhUg9AXWDAugvBl^$af4l! z0iBM_N)WCgd_#*7&e{Y4H}Q_l@uNuqgn;DvZJiMzg?E$HYoVf9?AIBAwyyfQcbz3F zaTaQN{x-N#4K&Hq^E>AlTU>LPut@0 zxOD^^|FyP-V}b%Q`m%$VhXesp6qFttDeRH9@@in2ZmYjf`%jbZzxYzv-M%3Gy{2?KgIbH|Gs7S@@Vpn*FJ0XH^ z7$6C)Oaix@V1<6h8xOonci=l>7W=wjBzB}&K9GB3Zo7yI2U$+PiToC9Pg~OR)_+=0 zRbsExm!Mi!0%wR!1Qu->4T}^}AJ(&>0Q66B=!U%3TiLc3nzeDhn=?J=#5zP+F-RU; zd=%|yjf=LlAhwoSOlTHl%E;do&-ZaBzJ@ifMI@hMTAf-yXG4oUq#^yYBStj z-YCp`H~M_RjuMQG>a1~}xjx`OZAtCKLeG)h`OGt)G->|3H+>S&2u7?w_`32&$!=X` zVD?C#Y2|KUdS1^$?qk#Yn%D4Q16v+aiOg{_6Qqk!d7|x&J|t412yExP;l|;aB9g{x$9@bbSFyK{M!6Af916wG z^|v(2fzIB7%=M5bSOP8T50J(BN!$x=%*N>~04=ZS%%-}MuJDD^84KRg{i1Lpb9>M( zy%t9h^r!P7EQ3#3j5Uc{w;1oJpJoZ)oJX)`ayF3l0Bzt*VFki5c^(CuFvg1sHoL=C zAVSFz;rbYvw}^Po1}*} z?e)J%@TQ~iLQSxw3lTwG&uY)={{A3D*22 zVb0HF`jvW3%XBHJE6WH26ZK^S;fyr4S|6##P%a13zcHRjtMS@o#*Q0RXncA-1eZCnJHB=z*X1yDGPCyi5kZMrsv(FTVMJVDH}PY=T!~3ub)!@7K08l8 z>!AoSGI%ewEv}mEXi#$YcE%=FpFO(ntO*5i(nrQlqk&51ESx1|QMfI!y~-#Hn2U*o zik8TTGSY!s1a%Qjd0lvz>ITjc`DwO*JZ`%pJ9rW4Os zAXgiN3@EvhhLIS~c0nm7`crE4Am8m-Sm3GbImn5FLP>t4g+tgg01WBVi8Fbyn3mEv ziInecA;_T8P^jriWedfCEvvHsC(4wc5-pbuUnH50Q@Sx5f}+e&qQ;JdKUl$=`=LW@ zlTWf9m{4Wo9zTIq0h3jU1lvcVDK4n$tq{6O&~rNcS?hznbiNTekHI-8DQ%f&=+P|*fbF%F94#t;5w}0~&vfV%bx_%ma%QflHd}F5Ox;t`e&(Ot>!ca4laO>h^pWT<* zxxIaHd0<0_AVZ5Ix5~x;C~@?k&oiG*2jdf$3_x6Hi31CdmH%0|If%*l`K_44p%ia$ zubcUl4Y&b~zcNUXEN}^Ark4^Es9Yz$sN4Z+^gQ4JY`_pWK6Fr6tk4jOmb&OsE;=?8 zBrRi(OkOM~TnvDm4Yk{aZ6JfAGH;PEq6$<(s+cZ`rO1UKCv7DI#2&x`!GubWiqE;s zF!!@#qfauekvHikIp%Pn0ajn;cP(y{ljCKI9GE z8sAZu3MP~7HW=rA^`B1fGA3U2Uzi)YqiIg-!C*onQ7V*$5e8_mb7SjWr5)nAc7!({ z!?&0oz21Evaing{*jiFg7KHRK^PTK-EF8ZEOaphi8?U1m_`Zjm`uFV7VCPX@{Yv5( zN8#yfX7BIaJ${T^Qsx~(Ue1TOc2=Wvfj;$X`t(a`3%8Oi+qv<2oCEWp_+SeF#a0x}7^%@JB^%i7ydtCZO#S3%Us}h6^bntjJsa&Rb2A!~1 zh6S~So-xkqvmOPH?SHWLI(gN}8GcFAD6fwuULXh#d=@Go3>Y~U{YfzgRrZP3;!|Az zP+ap-)bx(e+*HiyQoJp!Qe6e)RU(iUTgRtA|9Le$y!%`obtCfPMbv+=F|+KObSFJv z-A%^uC=p`GJ@p}Nk^M*~z58{TO$Ew~4Ml{*osGteSkK36>S)=}nnnwku9?Va`r&AK z!1$R#Z}UMuBX}(F*TADCj`62O^qTm}t-Hqv&_c+hhr)33U$zp`obujYfob2&Hd ztTH?Nf&SDzO)a>zD?5CX{y3;QnEp60!UfwfD+yy6ErR*9L~1)UW&;@-j}D@U)V&*R zIVUkC59;|^I|8Pq4E)i$xBSCTB6}MDdBf2v()d*=CmzhA@C_2d?M%3MxiNS>swrc| z4N=840CD5Xnh{hAOeUdnBO*2hA`4vMtSUr09gREV;<1BCTGxepG5@H)Rmt!m2&oNQ zvuJxwuy|vMnlMO-NzE^E?LE`JWN!uIpFZwXf9<~`4j2QjC+WN_X9N?zNxs4FzkVpj zsGt2*UzlREYLvadrrO-HO3wW{UVFvbwI|`+>UV`?b(5tZI^9~f&T{+${Z*No$?tbU zme^p4 zEmezh&Hl2^Qi8#m|`=FeqZUoCK zgCkzz3Z-0tch}R6C^hBFNzm4OO z!(Ikry`QR&5z^lDzrbI{jya3yOU7^ni#dCUhMpfnnHZ8!2ScTD_$a;=KJ+Unah5&s zlCu^6VvPBgJ=#&;0Mr;46mxye*Z389$2Duye52YaM0J%WrH(ugo#5lTojK|AwJi)B zx5)k{wy&|7?<%koczFCqff7GNxe#Y27n_=cJZE78p1_P2WZ<-x_Mg8T-FqTg!G$@_ z@q)WV#w;MYQUw^fGYaYCScnoPbg~WW{!J1*cmpGra9Az`T$tw^AUWwu@>T@V0sk|m z`O6|^^nks2cY9Y*rwDR?xOs1svZk<`rU3}?OSGo2?LhBuvW?}%UTDoJFwly`NF=fr zqIo$1KHBDeW>a(mV*82;kqUaK^ew~?2c+{Venp!-W)9(!F4W1zpdaNn>0ur~4I9jM z(3?H_gmT62Iz#cF8ry$?7JL``k_4m8YQKBXYyC0Q-FMG=wJYyFUGAR>*N=izACyxE z^QVV>MutqTUGfD#%IPxIU69V^YN+$TN!t4T;qJR>#*du+cmYAq9BJR$;bN7! z1s@h=h-8HUDdjI18%B+%uc}^`SM(P%4~~(C9RGG9>$OHo-_LeT1^pZhl01A8?}ssv|%{wHL- ze|Uywec;+rbIZISEJww07EccNCMGZ;|rW!{kG9!J6L*mZ=OS22Rv>a9g8n zaU!B-DXBzcPgKg6DpKWXBY!r?lsyQ^o|qD8L^W*U3{qS`#*U$-UV+filia6phAaMY z-ez!vXg+F>?vg6r30qL}OL-8grUI8eRUBYZJsbnFCJURtjh82!2*)bKjgnpo5Oc>U{ z0#ojRcg7cxPXwDoT8kqZnnKbSOEkp?2lG?$4?AZ>9TGA9FKFb#5IC=Fvg(F8SL)77 z$Lfz*&leKMKSZO(dVKCTrNgvBx?$-KQ>^k7fhy)Wkw15$gK>=(mDbru|2G$)0qh2S znX0Hnx=7uVsQx`(=|Vi!1x9&;s|ejV{(f+j1&)}y5@i$66SQMzA=EG&$F@x>Cfb6W z=OSwKOf-BL-0%laITP^=Bg4+$vYq_c(4y*7WO}W^!6X$j-V{4@c{Uurg~z%{-2h(@ zBHI^BSw*lZWpMNuGJlCTuO)_PYv1S-32ISQEdMx2F&SRe7%w0+5P5C$f~OSM+gc!F z+)_%;P)hzFKAk^4Jv5&qX{;KP=AP+rgt)}xw}!}Q-3w9u+nQauWpw(1%nLJLY2Cp) z3oTmnf+C?e>Nu!mez_m4f{Z&KJ+r7DwlL2%;9i-wFbqBHChY?#CsAz=yBr;xg9Y-5 zxUmYmgC3$kj*admnZ@r9#+NQsFdFN*&-YdSYByObiGnyblBrNKuo8sW|1JU$wITlV z74tDga)d$Ra@vSWhJiwEWUub*4+M-DF1ETWZm~rN*$}X`MCTQI0LjN>bGf(Shi&MV zSrR~4s$YhAVslkvbvS0xNdF(p**=ef&jC!Yl~K4lD!WAP(uUO?d_3c5B|R z`@s03nMe>h&kQkwHCufnqK9t+?A@eJDC;1S7t@zH!UAqt49Rj%b$5}Zc>XEHjL<06 z1oqM3ftl!7(eLwc{xa1eD%A+N=1mnzFD_Jqmy1@fu8k^IVF;z%_!n5<4mwo>kXQaB zYIY%R8-khPDCqb~q)-e-P zIfrB`R@68~$n1uAK>(rJ1uXp-_;iwAiuEn;PO}ey^~aTXMiW;n|bg z;$kKU~V}oS3RDeZn#WROVCXpAu{tK}*hC{^mCW9s@uZG%5c{N!@6Y$IvbOK#*S2 z)q_$Nla$pf9on>kJy;Sw#a+$|=vnsiCAA1z(Z+Z{tW=QMq?|&>7(V%%i95bfL2RnYc z{(MeQ6^@LF)`e2#7zK@g=#7cuOB>u1Eb+ujJfL{y?K84QN$7RYh@6u>hYkbR_d^%C zfXdgPZ9*W|Qk6z!-mv?WGbFLOO1@aeC9q${n?mzgJT$u3FNP$qUx|GaJ5X^Jh!)-w zD3pw@f(jZCEsxFYSA6k%<+jI<^J(37$Nk<}$%+L`lxGP(iIn9TATk!YkN^WVW6!U3``5+!#OxyR`6x@ur=TL>0|7T_;&Fb zxn1dzxXi0x%=g&Dfw<}Y-u8%+^>zuZ1#sakMC1Ncp#RTBUl=9mkITZw;bm@_uy16F zVi6yEB!GkSkJg*g&L;G>R!9^>FM^*3J*W#jKU133?bwVju%E9BdYcmisN1&7?sPle zW~5c!gguZNJi>*YYeCCgz(tJLEI*g4R)FPIs^j5Z_iUnN`4YfiB#}QFl3ab0mf2YC zk*%kgFWhVC(Y^VJ-kWqPdZm)`CXw8qvT4b^nk3k6!uln2D6%&`-R?5eeF;U(Qw?lh z2Beg9_Ob@l?Q-SDsTlYy)X>8E0Isdfh5)$k5?7>VElhQSYQ7Rc%-)vA3~QjG#2nas z>}!JSm0&|4R8mgZKJu%okt=T*k;T9AD0{R81?XHnwI1`_o$_uRUoi!d=_XxwIq>36 z58`Q*jSC>9qzf58%$rANGo0ck7?MkF_X8yZ92J(_6itZyihsl3wyXySZv0`JKDty) z-1+o`fzv3V^ShFI1+iRoq6ogMA?0< z>q2>%Y=Zq&p$QQmblcDIEq8%#6c|)pR?4C`1S)v`*`L4bjko+ge7fQI zDsQ+V>Xb~0H6xJX;QWtbt<2v*(`JnGF(v~1(~}u0a)orK2{m6-$?t{k%0?!6$v{>- z>B7HJ!qM3d6=<(Pxl#K>Xo4=ZNp*eGH$Pw@hGtKgF)v$r8I9% zB}CzKW5NAvX==CTm4o4RK53N3E~w=Ys;#mY<|=C@u=9q4?+swhAz@gKOl#MMj@U&6 z?Dkc?JN-_V<4z2ovF~aBdkf+!5Vx=Tg|4RO#C-G+RIV{Yx=(5+q-lSU>!gAA=DkSd z@akMRb}YpDJPk-j_6Hbk(3|XTLGS;Etgj4fgAK9`?k>TD7I%l@THJ~jcZcHcUbHxc z;!bf7uEmPGyK8Z~`F8K_etYka{7Ld8Px8*pJLk-qnPDL0mbUo#cL;TN@i}4KW9vJ_ z>B^Q@pM;xOPi9BK7I_he=nHpCPD%>8p5B|>_ut8hzcStz>tS&JTJ!o z39yhz3u;3BaohcO*a)ELXR6)aiC8s`PGWN+?We~`w34MdNPKk~@P>>=s`PT#{2mdb zX%H7ZVnZ#@d9}fM(6@XEKYzA+b<$PHp(3}e#&J|tc_H01?`L)4@&_<;l(8uTf@h1R z#4g&D85Qf+_`Hm)f6KRvSIAB(lOz<0P-Nk&HL^qTnWnRXKh#mqre?pJeKkupNUNgc zs^%Qf$;i10)e3b#s?fqK5q}|v>M1+8gH@Crq=)QF#p!){uxt0lXC)-z^eik1L)11V zSi$zwXol%wcP10gM8};-?()KKu6$NYz4S-BuSZyg+AB`4XdoY20MZ zt?+7Lzycu*}AsC5VG<)a_t?0}5Z`1nT8Nc?cN zmdz)@#GH17l@a3FycRg#)mPh(_shZ-9~jx&bDPj97v^IcQ{i6>>abS%n}gb4psc>V z4)fenRD8P@i92;u&4>a z3OMq`E4zkrM4N7d7Kc2k{cy$k^wke_FExg#+n)JjQU8-LT=t9vA}U0{SP}lstEqw1 zKJ@u!m*e(3S!T(8h4k1A)QdbbyQeKVFPZZi4P49DR!hduS?j)R|IsP6^GUN-cyEvk z%Pbbd&x25Xw^b6uU%F5_B;GL4(vw=LMq`MCvC{YTo-(>9q{>s4VVD4asMHwG|F9Xm zaa&7bY!O?}X{*D0EaVHlpgXx5M-?PfCAc&iQJRI~L*1NeF!5!rLdBqGsA)67KJra? zbbX_51f;Ef1#x%*aX)}&Q!zDO2O8@EkAzMK{?Mz}e}eR`q@*!k z*irQEaCfMFg3(y3DuUdR{tm{f8u<)ZsVw2R)agwyby0M0ZscdPIMfz(8&>9eg8|^ za-H*2RDC&uq#h^p+F*Bu4RcjWE2RBUT|scnZx>~gr5$F&uNFX9 zWCr~)$aROnmD?1R+E&q_bmR+@clES%P$o1J5r-71(lo|S)L#xW(#-JlO=>b_MsuUj z2JCUJH*}oOpBH~u&xO!Z<9QnB{C=^eC%_$bu{|6TP$#ZK8O4V!Hn}XoxZ5M|^p$#x z{vY#&6saTV|0CZeLC^iYHBhYBKD9U*6dvcGK`I<<_&s90yZHig5hV!iT%0UQZ>>mg;`o-D#NQJy%olrPcUizzG#*9C zg^_MS+@4p9+!^%sgijvhCV6#`s{}Xpeqc?)Q$sUmZoj~jt^j=_2{hPIhy3ve7;g=b zBfk=TL2EAt3WRL%-SJT)tJb4bS*BDW97@yygAV++Z9VC$gCDKR(HyM4k-=l%d_ynX z$6|YCaP>K=rO(y&l!Ana~KUxOn36Z?WKY z9b`8sRD^L3HVhKswTh0jSM!XZpF=}U71Z;WzQ|r*RHm=?5nAhUb z1`r5Y=^m0GOtUl~&O_ppaEUY!nc=s`9VSeP&Xe?RS_0DL*@M`Kbg80H%l4@9_C29b z9?MDJJDvtx!sDvrf{{Gr-ZCe0K5xVj)tddr495EjK5=>Yq>EQKR`B5EE&QwFS#X${ zKN2k*z4Bls(frUvER8xGVAe!u<{n)th}h}{aLuMK#g=C@y&KECW~d03K zHw%=vqd+gXt6>Yldj8BUF4X{!t072Ze^|nMa{gZ)3`$G&sUMp%&}>EV$BRG2^X^6R z>2?TX*}al8u9Mzvg*V27(GuUuq%?}GN|WoT`iMl^xQ9i9+c?I!a_Q&k!J4-$GsSOw zMSuc;f69;8cW?a&c7TYYZbS*Sxop$7ko@1re;v#+;y1V6m7qO-kB`}Tg!Du~&mj8e z7<_Yl(}GW0fM9QT6t$Z>a*^CZ1Iwg#&4#h{7{Z=x>Q<={qKHxEP3>j&L0 z?~d`COWaZ_1TReF36Ruw^j8~BfZTGZJWvUG|Y~7A)`o3;~OzYnOJXu5u zTJEr;4i7x@yV|wxC3rx?N1YerM)I@Rn7!kvY3ORZJQn($iDj2) zrPiVfEA3?z=Ha31o%StBJ!175i*7;FXaKL7s=k!0ElJu*%P&O=%E0ziRLPo?%vT4^ z+)Jn>-k<3GxeeV1!dQPTbTTFj%SRLQN27nZ9dcK;ColEbZ1odO?ZbQ z-+f!az)zG)*_D-0;&U(LE=3km^o{7BMfjWLD;E;}SN^BFE0?P%U!BtYaaw7z>FDY= zcMLo5y0p!u4Kh%4q#7aD+ZZ%wE=qy-)PgHWM9)=qS{qeJSlcztsIk9nOUO0?wi+2!RTPh3W z<+{%qseeUJ*ofL*ORY&v=*`d@(WpFb*1ftlChvR`y1B5kCD{amF8hgT0qWUHXdE3H zS%bOM%p@Y{88qYx#({hLg{vvk(MVvMtX`_4qDvUWE+14J0Hq!ACBXAT0ph0*)keQ; zuO#&5psLD{x@cTxQRu8D(jb&6x9?kGS~|aDP#JMNI04mRucU|hovYJzqq;}`o_Tn* zor3#zr@-T5$j=XFm}(V^5qX@mJNYXYead_I!{cIybTXv_QCc^iPQX=aSUz1yN=Ro9 zlHW_}>L43+d_aGzxnrAIKa$IGTGPtuHC&jZ808Bd3aEhfSU;$_9#y6qpG1kWCm^&y zK(>+23`VI_oR1rx4D|AFKpqYr2qCA#4h ztY`gr9zro2#TP-n;`~}d!`y-d>22pHBHbmOueMg-4Dg-rKUZOmR<|NtXS)q^yHZ*E zp|!-X!ky>q_2@g(YNqjs+8%Xhf;pUk8>kQj8zgwo15U_`(6|WAAL*Om@9+ii->L5E zdS5PHaCGa-ei%0dG{1a!u^D$a(_0;`Mca9PFM#%VLu+|CkH}>w>q#;6N*|927GLv! z2leQCfpFWGYisL};$OA`y7jL~zqlPJtL8dNE-yn&dlkuHAbWG79k1*Ak&dfT5^$s{ z)4_u(Bp+K(Mw^W~#Tl95nUBtV%-M)Ra*ZaTwcHi)&*<4hP=0VxP~!5SPBZ~@v54yD zO@5?!TJDI;(0qL5PfVTy*%4ry;Jtz(I6V72vYAK= z4?&mlk3r-Hz^k_ZrrUheImn^aOEByw*moTlqxYNqNo;`A9bC=mc~`QmBe5x-Z$qr) z+@C{y&L4T)&sfB4(_rQl_4y`+hZQ_+^}&xUCVmBg+Muw17oF9zRf4ApxVnHSucbEMh0x$$=gss6`-xuU^(St#@ z4Edg}<22RYIH4Rh8CS~T+mBNKuPfL6P@*<-!ay{u^A5_h5 zSj-j>`9GR}hQjAgz5dD&+n;8<``I&HntSCqcu>+y%MaBGf6-ezbWNc>0y4wAPAbE& zu+9Wm?S@Sjrh>zp@bJ$*!V-V^XnUeHVbBP^g_p$V7jlg!12eS|vW}Px+KQ3Y$t#1@ ziqCp`5vzP-fiy)4znO^5=pDJh7|416`c$mKrH?Aw^v%Zv8s?11nN{Jt4bJ^)O%4&0wYnO1+ri{<(HH2Uv(ixcX>Z5lbG2HIEKgx@jKO1j=qx!7!e+U|h((fb{KbDEivBz%#+Lvfak*fDzxh@3I1 zTLbpz#fr;ypKoilX%xuiUraBUmk=}PC;ec zMm4M@%t|IYP`Mb1Of!Zj+MQYoXBF~O+o#VGiMRQqBJU7rgj1L;%Op5BQ?3akmvJLz z@DERqls=LdK8dMtY1)5A#Ga{1V4a?haQ zQ!1`ipfR$iG8{6Xt5SlG-rYs9CBW&fTk3n^Rv#lPO#H+tg3kQ-JCnY-oLHb8IDKtA z{0%38Ks@1ybW$5WgPwWsP=WPcVj@CE05=msHQYYgTKacMYBzVrGWH~ma6*}D^b&#_ zUWPIT`(<@4yjZIy(nEQbzZt=rc%hyTEMd4R{Hz-o1D334kBrWaLYCo0w%$9hDo)&( zz!)l4*0!RUq+}0kzeK*cahij*ZVU@|5gN5ro(X2`G*#5H*bA5829d4>^D6d5d=LDc z8CNA57&eYpRE2@e3FXXsNP-(~Y@tRh5B1%GSgg$+k#L>@e!{#MbN30Un4M-)0ibD( z78pqTTQq!NDDZ>+1-E&!AP+!)9CjL&4AZG7NnsfD9n@`_Bsa`hAj=(#ypXImNFb2A zW`Z4Po!(G1gD71L2U4F()`GR6)CT_hE)EJ1_@{<-v?az05pFLw`SB7FWlkgnE^P9 zd5Vbdgh;PfQaR@3faGKwpw><&gsthUPhjS$7$ z5514+^x;NzTBZE9k14uxdS9l^>l>Wz^wP^%CGYrla4%1Ppf&u=_G2DfoOoa0BHK-D zvhEj^&(q+wVLHPUzjKuZlvR5vU>sL$0DBp)J9q^r1uIU}2-`PrV#0?NZ`n)tu8 z0C9&TyA~qJA6REXZAOs-;mPS;#6Ir~G19Eba(=53#(i=qrkG3bQ@L>z|IFVVjCI5$ zHazN^klH4uYfXP)@+j?s$aSKRN-|o$I>wZaPX*C9jv(aK!x13PX@}=>@n=}@ea8?} zJBEaM^T6-EGBizC^QpgJUW*6dppTiNe}w^9_qg0b0-4}=u;qGwa;?bW#8=(a1;xa`7?pqS%HcMs)Vfvd*}Cf)}pH44W6Q+Z1vM< zZsy6>1W^dcH;AJca?fs>AX^FFiAubNgnmx@J+|39V+)3{>5a>f$;FPz%iVb5uxT=B zq6_n7V~B#2Ltbcj$4@2k(RmMX#xseYqsWtD9rL}8v5(0KH$z=rrC_z(dXTd%oa=+N zBpwg+D+`^Kl7{@QEcaII`fXaNbIdds4xzkY0J^-uPodA|MnaQN zD#-w_#)UDxZvz2Phk6OkeZ9dTsa|!|351lXnujE5L zE$E+)AVG1fGc=Q(4^-&$f)Lqy9jaI91y=h`;3wfOhSzJ$;5wW5lJk2FRDV`{%oXT^&<=4y&^Cnl7}R(U zh3WLYXMCe4^xT4BF65kMCx|ef9Q8n==Td-;{dZa3Z;<+Cv{^mQCos;DCUhn&x`YaW z>?pC(f^DNcD*c@4a7`@526p(bSp>KXb9#MQ_()*?h1qp|BSMB@DS!Y5HME52clGS3 z0{Do8?wBPTl<|&yWDE=V6!@hVgZ+$&S#6S=0*#@9w|(iBihpzyl0|$Cx~`q(*<&s8 zm&GS^YQ#YVg^#cen{m$^L--Q6u)TU!uaHDuZE|WcTe+Zq6OMpm48f}b@@u*Ow=y}D z@{b|8qbU~@F^0xw*;!fr%_|>>dL?}=wRPzA=(;CwxnYI(m>Ae>L8<*F2s+3a+WO%# z0X3u_Za(_UQn)z&^-@ZM46;9i1yoLqpt5mnqUz-X{|e$`%SH`n$BPc+Dxn3;xTJ;z zTJx|1)KfkKiWLgXZ#Xu^HYE($W4Y7;$C)*`Rg6d7-UT>yetijqI1$>>M}i!xG7MDOJ5SSeBgdc*T-AW;eN)E<&GCsR|J>=bnqPD8TRhj zWNkkY@m?1sYlWu|Dp?|Nc8@^AeD+bbqykXMu=vY{geHbyrjCC|)^2XmD@APrZ}jD6%Y?E)Tg zJDAP3BsgT4-0Gfi3I1sq!A!D&*4F2aMs3G1OU<%HHSj7AQKAd*;+R!h! zAeHUM)g*jz>IMQMn(x-Uwi2SL_y7%fgd2)tfOay_9cCl?2Qz#zeQY=_foPGaECiEr z9^o0Xytl9dua+Yhr2m&~t;eALa5o|vHGDd6zZg_YujAlB8X0eegf#&ck~k9^6QVp( z-0xHLlH#~^GKUxT00+m&J@#AA_?(2HHU7jBb}CuCBqP|l6#TArudho#lNAVhkZ)T5l?vV z3;t|L82-$9^bP+I-^+_7<#~(-+pC@)*%|hGcqLN0m>nyl-<5AHvZ;&(TZlo$6WMHp zPV!QSY#=TF5*v$6NY3Bm4G^cKXv#zs#48YV!_tPDk#$EA;|LXmTqg7l)g>}0P+>ic z=?1Y*(V=kZDKwX-iG7$Mc$i#8=fo25P2nSwFJ@wD{1|(bh6OJNbKfN4vuIFMGCnV2 zzp4|>wIxlx@wGAajW)DjgW|GeHpMXI(6lq8RVNTL@;6fh1sYL?SGgDONsNL85+;Xz zVWNWW5J3$ETOR-fB{lu+x*#fXX?$P81^cMuCQx)=99Iu9N=uD;qq4jP^V{!_s1V^8 z{|h-!Dx~X79xj-I9NU{X=#VRz`T`3S7w$LKmCQUG8H6F}V)6eGv*PfQM#~`acB)8l zFXe2bU`>}F;w>}GVdYu(4MqTHJ_JE^X*v4+We(Ks3=}IE8KM<-V>}NyKH!_&8+dDT z@IB}`P@S8MkXUn=Z9?S$&5XT}9-iOlm;4Mo$|69M!c5NzX2l)&t9M~Eh9llD66R!; z1aCvCGgOnO&b|Jz z8PfwQyNghV5Ii%hWuMmxpJ-pafnGYUWJb6)%z>RtYS9zhQ0T>tR^x1a+A5hVDD85h z+-8qI!K%)*JCaq-nGO-2M?YgQh#CqN2(zSl*&2#JoQ z;+;jHlS)5r+jE(qGT@2*tG45bR8XsBc$pNYr|-y=2m9l17RP>gJ>Wcq^_EH%){(-V3QLQ`9pUVW&}6IFGB8%~V55yPqEP6M9~;3AApH=9~M|URr^I zLH~%_z^40TUE3TG;Rx*NV;ag(wg-I+3hPkoL1j1pC=0B$B6pZ{X|tw;jyBroJhfR++E(P_ULvb`E%D}c%-J3yI$3)_LJi6>0%W=T{98k zr{*btq8Ll+C5zzD2@e;zJl)(y1(%vyzlL|A8GK^{F5bj-K@5#B zO)DNftZEI%S}{GV=l0iH@5gx}S)QfbHGQ}}cVdkd#@uByNsAr_*r9k62Ckk;sA0S8 zKx=A&I4f%M^d?RmkEKED{ zp=(rzV~ni`*3QU?oEP>?-3X`BJdo5Lg40pSdGOmM+wiW8uq4&RNMAQ++7C+8TD-VpgnQagKB6s22z4dY4v5jR^;sWs5d=q z*%_L?g5JA=nDLKcmWu)S6-?wY{Y5)#W8F3L1AXZ8;EL|57AQ#Zyc;52hA619R!8)@ zt6s2MV$PvGj>b1xI^b5}enNR{S$69?2~D)_Lc~c=Of7gn;XJm0kT;UF+*HGOP~1wm z@dmX&;5#vP0?vE0EUotW1X7p3@Tvc5Atb@7D0q6rUE$??8_9TOEPaK1)QgT2a1Hjn zR}Z%BkaLs3&)wW~5TnMQGa}q?>{p{G*Vqo-k@oT8i!a0mv8p{ouqX zjO0KwRRSUC`BDvCxj*#E1bM7Ot-lhl4+we^Uu+Vr_p-mB4aGrSkO}rM>%UNZ72ORS zhDwB?M36*}B*El&f_M23`5X)_wEA@V1YB{`vH;2HY470KbGI`Hl8Q#JTzDbgeE>2n zzwjv>8S92?PZm&-Qt?f$#`lp*4C@+(&IggB-$&F=%|_d)3*Ow$YP67ZqScmGCnrgo z<~LBYq>6{+52+EMA)iXPG0dP|NsM8{`(EK%07pGO@v}xHe*kGngXm-|QU_wdBdTU8 z&4GIUXo3rk?Prq?6(*U=w-fl7pPG5Zn+iym{getX1OP`l5PeZ;n8Ss!*;%FZ#QE#w zX=PAD4f64s+vHh!goOupgO5$#vKM0AnN05LT)%8b0oF08OzyJRkHco~AquM|kTLGF zqP^)5k{jgNDb!;LiJ=R={-(K=th|`UwK>4;#0M*n269_Vsyg>MDwhK>D|a5z`U1F- zwFp9bchcT3Nf(s}Q>$&?9YH}RX|ho_qAkV!dTmX#j46WX^|tI?g*$UTS&6lc_!&|W ziS5-;V3VJ6 zqSsFR1%_E2fVjl^oQi$5*kGJ=;rY3=%@wBF-JR)e4jH(q1Xz7RDysI0Q8TS!r`l(^ z#@mop^TI}Ic5X5 zCtM882H9X)2C71UQ(w=_)TJ(xzgo~zo1B~s3Gd6cvOAQhEhE!ff51zg_bAApL6bkh zEN@>cwQQPc%j&U)q((T*4UH$*}(_I*L*M^m40?nJUlE5Wb5wniB>l$%Q9qt|C0HyAR(rQ5BhC+R8~^ za@?{p9VRFn{ag0DjGebI3}ofYFBM|Jc3aG z!9Mj_9gfsurvNc*YS{$4x=&27Dy1DN&%w(oClzR6r0@J2%wOsvrSW6~iM2P>4cG3^ z_0D~|+TiBj{We~OPk)_+9dm8~_L#G=QdO~wr^V&Oxql)@haEAzrA0VqBe6k!=!Q)P z;21%pAXXw9x6Q&DMBB(eillhz?y@WPCxhEJNcaKJokm}F*>4es&PPlIczHu8(x4le zgPWCe2GJVvcpJ%34`p&Yxxc;dV7yBf>Z?}0fAOE#UuHEqTLueO+mOPsJ`riyc7+98 zId(?4tU0W54LX}o#u%6)4k@`B4QELIbb)qsM~(UMBZX9$1e_^37Y0uM=t~pHE2u!p zJ(3fns_{il#g9HbL@S?f6%Y4FuMiOrx{;GE)ONgR(U(-rO-p#lN-<63CvU?HOlCp3 z?mJp(&g+M}xqDN+Y^Z83ftuG!5MC9&N_k3l<3erMW4P1zs$UC6E>%#1nLkQx?u_~n zq`_Ythnngs>>Tb)U2FkeR?b@oiY#_kw>x{@w0AjIxpluotHv$t99_0kENFR};*2D2 z5StpFYO0p#`{8xCc)4{$hy9A?Xnxf1xn%psXngwcG2bf2i zG_kpO?D(^Rjl{&aeD32IqkJ`8=9qdXwb0bY5lWtGCTX! z-iMjnCVbYW#s`(k5A6EKFDd7&^VN2SN4IGLhIi_f#c7!L>O9=-s@neR*uA67Vs>^U zJ@X%$9~ADntOw*%WNnDldkqAr_Pz%K|DAMVC-!#AHak zn}d0U?LyM2%I|k@8WCPY+^x9WQEjRSI1XI0?ob7sY`cSdC}_hKj9qO*4te^W^>L(| zKlPyL<96)x36ouJA4XWR?-EIL`M?iNpzaJmVu8K+!$Y#gNIG!4v(ohWmh<|aEgL=C zN5eYwkxpCEaW!X;Q69aDsg;V@Xc!sr>PnM5i@Z0au!{?FTg#qV64`E}eG_r1KQ+GD zst7&1o!Y9HDre#7{du_h`W?q-g0E&Po6$)LI*8B+S2V&M+h>Kub5DXaF6S!0cnON) z?*wev;o{_GcPb~<;^O`wBeR2X&#JCv@1459DW82$Dn1GiPisw8Y^WV(7SZaWhB8YZ zkDUXTiC&ge%X15CNay?G9CarYf!qh7R%f+9twI1TY*3uJ>-2 zISHKT58dKpeq=iukn~g>fcd=V!|w&FJm2j=Z~h0W5MSfQ(!YCuR*5gK;6D`x_Sf1X zoeCb{YpP=xbEiq)K$1{=bsG^UXIumU2y`LDe*R7KFboG=?=T9sc}>gHeg8Z=dvCfx zi1C)U6qBa=vbbs%;Xu-2rsA}#0d$|Waauw+ZQqn}YlrC&-fZwjC007BZ5h6fc72CZnon|4E$>}W6jx=RN?d;VZ}yhQT3qb*FM&kh5Y;t3M%^ybU26*caO%p+u4&2>MguF z?FBAmMX1c34-S5a$Lyz${*6q_?6FHEcM@j*b)&@AYkm8+c>kp_X3h(RRR;|y5>RwD z@RV}K*zu-zFzO#Y_jq%(1Y~`@iQL-W+rtO(G(OZ;t%s1JIb3lRw5h_(JvoZ24oY#Y z>(pf>-JdHMq_5K`X&r3nciR3}g#O1_gWzCCq|=|EtrHj>$rF0J1C7c0m>N>C(tl9= zsB|1kmZeb!^*nOfm zgu__x6vdwbvsHauicq}Oe951SP$XqHHlN)OT+nU5k~Rf4@eSXWt9Si2ZGzOtcn>8Y&o zF+o!&wmV2gpk-yvl{(bs*G)urj7y(d&6*BbWAHS2*OaU=J(W9f8#~q$3Z9^r+BD_n zPlb-muxw#}ab6CqVio}VjKe4+dluA!R!+tjXLn&*;T-L64h~Xye8RX>_sjMV+Tx5{ zvb0+8ffd$D^ikz|^I0n&7sYRE(sWpx4ZYU?@#fzrOwH@l``CXBE7xbYhhwHGxpg*= z%zN&}pD5&RdXu@1Nt9XjFC%{NX8dy>|7U2fSw>3k zU~6n$t{b3;r>v@$#ghIkEjH* z+>hD{fm9_+da}f0YkXO=G@E_3#`01(0?zSEl<`I@Tbx0hkzqUG##{5oK+vH4!`jAU zvx?BY^5ryw!0Gvj$-)u=m;mSjUBp4&zdc{vM&>%$Yh zO#$2)AVs=z;;MYQJzWSDyjU0DZsHZF$+L&mn?Sgd3n6Px5-pjEX;$(uQlA01yA#yj z{(#D~CP(hhg0lm-bKh?nt#GHS2ihWELOVg1+Rf$eSEAr4x_^m*0+R3(q`b6t5^ny@ z1wHH9^$I{50mmQsJqc@00 zXqM2(6FpZ8$pKL7y!kqqRcgl*$zOy^5VbFo)Ixo)Q8$+OOmDBQxKig=n<=UK_OLf+ zmNIVW;dN_s5nvi)EBr9~)iwe#x3OS7P6*9igDbPPIw=484VfeA)OUSaus8W@+IrN~ zXeYSh6-3?{@GnP|g|3iJPyKq(0n{?6gZS%`vW|ix4{a&Ub@-I&wQ2V!B@hArFfLy?7F^TBxdD7Gk$EKSWm8h~c;*6v(~ceT_rdU~ht$80gckMRkr-ASdN`m~|Mp)k9|_y} zTwuq!ABR~9`Nrj!*h~ru#9{#)b54d%s(7!~2gqQp+xuckLF?kmu$NGt7j@{|B`iDH zF>59w*f7HMM^Wkj#N8D;$0sE6Fbmv(Z#cBR%>{&xcS%mn?I`^ zRJ(SamS}g(-U73Dy+!OJfBZ5y_+z$UWjMa<5njABdv?RGFlbl0zsQK0h| z0P8t_&`*uIorsj%2Zzs8Z~1LVtQy(9816s&QQA+gOw8*S-?!s-9!9f4TlvC4S^J3I z-weS1U@{3M5pPmw62XeTifky7Lp7Bp!k+a}iqj4=F}D);*>T1p?39U>0YVBTkHS^?Bpsf5lOG zAjIClS~!mq%fxerQ@I^D&t-NKs>5$=cp;aB)2dizON5XTUUG)+dNIt|ADYe4O#TImI+MK{mMvL%cIkRawFl zpEDP<`IUL~-1W*@V{xWcyH!Ot>uO$eA&yz?9(LUm&k}Ld`?rblg4Ec16MK)Z$Vf(v z1dQ4J_6zgQBP^_@$g9%J_(_UHU@D|M+?Yb4IVXO*YeAsq>>DV>(tdF%zh}!MYl|2` zSCJo*V~PTyS*$4+edS`P7WxR_Yuc{_+?$DN3kJTWDH{xk!k3=7Z#V9fSrzE~;0 z9!DGzYrWEFcEam4fi|d^$z7<}1TRI5;8VFbEvq^Y29D-|+3dkseZ#Y;>sAg!Js8AS z-ueh}++TWw)YC2}q)1V5L_a^X@$ep9%GrX@-L1m{g*>-$65sasQ0VQ1NPAP8auIXu zYnLLz_EL(mk{PIn}>PX5FWWCadDYaeqh#VAkI8)%Ippb(!B8o5$>M9(Cl#kD`>L?eOBD8MVSfv zYu2wQr7DWAsuL}>_gU>z!a(v!@5kcgmxTR7xDceoAui03wUnXc-cs#x3cgcguvQF(K_3hT>E{Pvg=a)9~`)-R%t!}->!DwG0w;-UC^5}6J+l7&X7FH=KT zy>Yrf*p;_3`Qyl{U>rN1vA)HTKYhgUteW|->hbC0`N{RxWDyp4`KZ6*!2ZGV{UjjJ z=C!%@sw2Mkodf9);+>&lelZm_Q(p4Vh9{2JQ?<>4M_DlL-_p^D%uxf9bJUnkndCpo z6IHDV{aFVuDpf@!6^8Iqd~0*ZJ#pT&498=hJLNd9NV&y#MCh{yLnoFFCVVhlw*t-$ zxA?OKqG|jweUQ;=Z_cD;cJkeKVgsDx_*{@oAm^ zZOVL_^qQzo`J-@J>)$l3Nw3`EEO z9qr|y%~=(g?_d?~=nO>GBU!Rh?4{vEmOjT+gu@DxDdi1h_QlN6I!zJua%W>K+_Jk8 zsXe^qsFybW*)A6jUba>84X_lNSPra*5&oO?>UrTQ6BD7>*vstWt9wiP8g0z7gsz{xW_vtEiYh&zd}LKT{Kk*X=x?5D5;9mXaQo{DkdCVChYnV zm#K0AVV1}#u-Dayi*`TH+Y(A-T%^OSmytHSrmQE*;}^eiCUm6>K>r!+hFa}@ShMqh z=lc+a^Q+Z&gG>3g*9p4`X;Vd>cl}VMj~@-ZA{blFDV7*)p;u3wMK`G)qA&S-ZS{PB zF&W2A>^H(`o{`WmFQ9r|LsI)u$q6{&=8xV}^jy)J4luf>7cz2h-g#hUZ|-dHh()As{D^X+?8vkX!=A@*irclZ@A;XA@aRs?rB)rU{26rO3&MleT!Xu z!4yXrNTX@11dsM=yD9$iV6N(+zWl?B>z{~+lW31iL-(Bicn{u95tfjm_h&l3bHs?}mrAw+^%v5S85vC;0VJUP?3$3JicSC}Z> z#z$$NYFiCL^zEhX%Ct;vvx&G5>=rLodCQj-7grp5&K>nbE=1b_jDQF|x+uNAkMYK^~Im6PlL7z=PL$W?U#wxIO=8JBO!K^Zy;Cw;3VaKa;LtzHZ3 zK2MB82<-pP*FIjsju@mL>+_fDTa%R9%S7;pst=O*H@t%*;e95cW(5L8s;v76pQ$G@ zX81%thH+|czL(qeJv(d$fBOALzDhV2AH``PsfW7?=m>HQ8YCM>nemFDFY+vFOU6*6 zzD{zF+w7|s0DJTV_W{4~xiT{ZQ=JJDWA`RA{lMLgjF!c4VW-?rSjQ?h(-}um|9V1$ zavX#L1)ke#VOFTfgunb60!K{LvAB3V#Avzd*}4{|50rR#*5!0>5Ui=1gBsy)+xd&g zp!8f@yB;*IsFX0gvik9xqN0VJb*<5mHPAhWu%{TFmth&=@dUW%jU|o7cGbLyqYL}e zC_M8-Qs~qAu>ya?gIxA;o90hyPb!uFHWpbE=w;b^w$3L26UW`1g5ajmZq(S=0AAjr(x<*OY=4ggz?b? zF^o!MSHPL8$0)A$eYK57{!*@DRFqUS_PjB*iO&k;;U=ujn+iRv@;|k&6Ky{-=BsKq4Qpi}qI^Go0dcU+!M zmSnamIxC5dOg(!Lx&#UEn{|T?=R-^;kMC|^dW82t;ie2;BcEkPFHI~c`T6g7{BfRJ z*wh*6PzBJQQUyzg|K0%A^qocyiIX81zhi;@n+JC+fhFEVSq2bMUEKfXM~b^S z%vxE6zydE8tnp(Zqd9&0mhJ#Do{&aW?nVjYC&p&B3p#sOa%J%x+Uw%ne(yvRIS=pi zRSZ5YPY!911$J(gznxWsiBCI*p6-pRU30^e-%Y!j?H_Yl*fPha!sN*W{*7SgX9w9xf_cL_qUiD}>yG zG;9M+tYmih?Vui^Uh34C1y1MQ+X~;q$mlrOtX*Z9yO^quJXOrg-P7mA+S$)>2Wkh; zOY8f?`X5)+b=f$ZXnBa?NA;EsJI%PuJIGb0P&7|Cr!&ncU8-n1mz|42(=4r%D=icU zIp4zxbK9#Oy`pFa;QIN}3e{)I%qA6+$#xqa2cCqVx7hHo6am zzXSj(cjYN)PSLJ=y6zRQbAl&to;?{q%c$x057x!Y)0=B~UtRy*-=Lq68kMEnWZD@2 zoYtkBzciICJM8&l(M#L<7kUAz-)%d?8T^83UUD>eOwUe@uC;e8x4M3_K^-rjn^Mhg zPvds}X~_q$(`dD%kEaY?_*V`lzRR%3p0O&=5m_g=-m~R3)!V_&OJbjT-xCzfz$(|Z zYf)xw9z*bgG(g&5*rK}L@Vi-Og`i*6UO!`b=KOb_Q1Wc>l#U%i=bqJjW99q@$7GKE z!H6r~O}C9{g2&H?c;D3LHwTa_ngIkw$i&Q3#i?Oq37`az{ISg9Qe8^T?B$OiCat@I zn9y*>^%Lcc%eW zBe`C;G3#>)zMGHlh+t99Y>k!-;qpVdBx|UwBvQMqcxdbDqn~jBG$!m5AFlwtT}t{)v)=L$iTIJ z6iT}i;L#l+$&+yWK_p@`N{KSu(VYk%H!rr8$EfQthi`%BFous{Tv|vGk-_2zuSc(Svd=%Ds*#8TRnBnoq6ee5$WJ$X{iP;NUn}Zxw&O z&`8Jraw%F9kmtg=q#`M5wdG!Vw&Fx)*#PxC@tpd>Pd9D(<3{~%OGuxsWxlqP&9rcp z-LV>xQe_HBPf8mS+_Rc>Kjn8p7Xrh9*Nh$|x33!z>r+}Yv}w=*QI$!~B8Pnq(I;tJ zeg#JjO$(%XJ$PA)ZSOfGwPZ0-9!FyF7d+&+lu z(Y-3k!`X?K*B?XX0+F@MP*${9M3{6Xc05|=ab2&wZ)XTv9zP-kLYAiCm(Ec#YbV^g zbuQfjwC0YkN12ah2S^Sk=wHm&7`G>rGt@cF2f3_a6vlLU9@RbU$g|{dg z>F2yTX}3-}B9fh67l|TywIbNH3D#AcYZ~|zB^GXmHD}ubiTfCeJ&!>5V z*9;ozSq=>-<3zxZLnG2S(G9ak`Zarw$mP@Xo5@?5ew`w~oa>e736X)~($q)^kN`)`}&{Z<5oCCI>4=-Qeh{MqfXW z=b3A%+(je`r;2A49~(AjJLhg48qvnhz47wa8_>qrH;=B^56+9ccQND`mtd)hJ%ii? z_ak>*Pt>UB`h29BF&zFBsZVE<`ZJU?Vu*sB06I~RkuI#)k=_mbz_+T)bPFpS2u~et zZBfj$h~1MZi#^{VO&&i<84lc^9A_NL4EsFyEL^6S&g*@~W1BUu6khm{rBcX6dwm%H zn?#vFH_V*?GB4e^&~IyGz#71+&khSymnWcFzoA-LAXg)ulSV84i#}%}=G`TOhIK=< zOQpf>s~p03^CKK5=DZ@SYfcPm`DWuo=rj{S-{j$>?rlVd?%6{?Iy4&FB9F$ujdc=5 z1_eebwc{v7(83bML61Wb>(~m45FTFNckhv++W0ejq&B1bs96*Uhw0+Zcg~MKc9ZzW zl7zCvm>5ZP2Q6Q1T84h{Rk0$4ydfQKV_WLM#D9Z(utFy@I+|f>($ki?I^d~Dofpeg zxhFqAGv~y{8=!4PrqLO*9K^U+bac51Ykzr@xcbezCY+wJG*j)1k(;Qp$8A>bRTZYW z8IGJXZ6pe=09A57O}4vf)*k1!v4Gv|OI-4C&2*`0IiYjFCM`G>JXb6qJ%| zNzK|On5L!pG`aHBDRLm@%-5gkz8xW1VbUGPN}+U1Dd6iWGwHO&rZCZosdp2{YJk5? zsyx&2&%zP9z2b%?l6NBe%zgm2bYf)Q>ChL!cvO#7RU7%ViTox0!}RV1%f5jgni(_d zBHkC?>^S0aBOYMCi%f~@zUVhE5oa0aAHO+drk;-0puYZ=F;HuY-;j5Y>W2aTn8VWR zX=(BgHED@i9%;~mO%h){&JSGh0+BeTKM_a%%8py>TVoX$25{N-K%O zlOM%z+j1cpHm*ocNj!!HR!wf({36e#>}`g8U6d#>MTtAV z#f=&UeU+Ei=IIuU*Z3;OZ21M>JwHY!C0y~%QHut(YF?0p=8Atb?hc)bgXP2Tp&vCA z#PvT>rzy}z2T7L8KuP-0Aqq&5$SkWqUnoysGZQLBIUM_WFGdr&9$Bf9z3e1nbs0$O z&mT-=Hg-n;2E+0E)Ws{|7=4ePSBUm>^kf6}LDVs>#cS_vFXt~ddF9WLy`tW1WNA&x-$c@#Y?61ut ztRWF%iApzoPeH8k`We)3>_L!ZtlZa6s>g^s)GQ&76%ieg3Jm@^nDA#okyK%fk>Tx; z!LpHZZP>4(gVa0(8If%m;$#?7r36!@SYo7DQl$ms@Q(JEhGFll>CV$8kDKTa1555ViYZc@ zc^yZ$y}7efEpO~>(y(XAKyweU-Y({+fcP!u^x<7U+27Om-kpn%aKXHbaH`Zm{F%S@$4DIW**D7lhv67YnOvc)@%%MlF8v|@ zck-0YuRM__PrN=9U}Q)VwLEkno$COJOkCI-P_`+W-t*Lb5pe8W=aBiZwTOyMDZRwj zEEYLuH@nO3=ex zW&M)oTmWhb`9@#MuA_A{u67+mA3N(C66a>X=Gbep-GcqX&R_3?r@fZRrb$ZWM${=o z?|k-lCOM}wcSt!JT!xLXG2)KOdG(|5O2sVu-uv1#5~3yim>9WsQ;FCS;x19Ud3wD% z{3uQd&B2#q4eI~K*y7%W^j3XYpcgc*x7Y5oq&$mjk&lJf?ukr9EST0S2leM$VcBjx z*qFE+&UeU^-D`d|hgw9MwV_%&oONJ4gx?nC_4FdJVB;wd8YHhYylD%KCRz?6lmL`a zkcMH{E41)D4WWug2*52vJ56ChvQg@X4l<^j#~+#~HrJBtgcD-dVi;o37Ekl(-pge( z;6!`@MI{e}jBB9enNOgJE2P(#;VM`YxV;1{>R}C&U3%XPe9gdYUHDxHIbR^N3eHz` zR8)uA*6LVoOWm1S4L)}nzV+l{cXgQ|AEs?r zhNKdOhKP4nMxD4xznPCKFBxn(0d(FH;HIm6Mw?Q4^yb&8gdT3Rx7!379B4{6yvazQ z8YP29tWO>OhJ-$yzbBv&N?J6R%4y5khFr_fpvC$#(v0!xq5RTC#C(`_4US{-$Ew_u zjLC&o#Z=|7*Esa{(>6~&zf2~|FWU=nMxGPoyf{%he*ArZ;=p;s0Ed`doM}_Ojb`|Y zZ1}ZE6^@QrC$~IKv!<#VafA#PNf;GT2vx2Wj-na1(wdYp9eOa`S~>$BO;=R3LXbuB zSiGDfbz+JbTap9~BFHVDXV39*KNtKwji=Tz+zi&EgE(##X=E!)Q`lbLsflGtsdVf& zA^Qf4Jsp`@)4}~Ju{Lt0L9L$+x}W5sFfoRdUs&~IE~2j3&6>p~WZ6QA#ei+^SzU=? z&#HvTc8l(O%Z_=fcN%oY1xcj4N!eu*quAqiM$LlVP;}?ISD?%!M-96$kr}M496V?! z|JrV5svVY?+WV+5vs8>w^F8@G^v|-Jg= zi`2^4sB*A&A39Ib2Sm$7#R(s03>1WTj*cV^O$mJ&_wf13L#;%Eo{Z|_n`e9J9cYg^ zM-q9M0APU|N>ksQ-#eFTuxxci6C_nx&r1vOlK?Js`#||;G zzt9O2?E;Z5RH9}=#$Vg_Avz;71mbfGaBqcdCO4>n<^Ha*CQm zcA4QcFSv!98B4RRrlXJ&eG)tCVMj8%F)a8_eH*vsv=EOSB=C_9+`H`2rhI-6RZg76 znh=3?kHzTE8_)N@!!>@~m@zMc`9!R6mejO`$2Cipd?PJHEOXyG3?;ezAV2-oB>$R= zUi@wR-!?UTtFxTZQ2GvHlIuba#j;_&d2AW-pMQ7=9 z_Q7;up-F0yG>#j_I*egA$yC>Uo^6n;z7wY}P?+~$7g@_%ee?&D)D-=jvn3agAf)CZN=f)Pq=8R%&bu!U-N2vVkZJyXwF= z1SrmZo-t58%q`G?#UB_i$eRB}R`rD6jdp5G+X_=?8Z5^$6}?>j=_(ETtFj8J&>~dF zfgC0jJ0uxp8ct{VlmXe*HxOOtAM9QWmvsMVZk|z0B;}83n-9Bd4>#4y`W^}&T5m~C z;9GHOrCdaX8ngf|!-zIT9Pl=4i2sOCy@o;$taZTWXLlX-FR7i&HjzL3o3mah#Q|#h z9COd(b2U&lU!y1pu^+-1Ok!`}5ysNS9l|a?+DCPcQIgfNMr_W`_SwA5E_3CmoT$kh z5?sz)DaIjm0EanK;^vZdnUE^etvkRFQi`H=3GUAY5gN=wqRDXmT=b5P4ly_In{e$s zj9$~HHnUuK+F}{+LPt1o;qMDZrq=$=%4G z6VckDNw_*Sg25NVsS1Tc_ta>juuXUfLwqN4m9j@way4K{Nj1X3s}Y_$z2tPr3IuNY zr8Q&KSSChyl^K2;S!2AfMccR3)Q`6uSwDT^pGA5JZ&3fxr`oY!&%+m_0)jU#Xj=w zEHYfdQg!!Gsgfglbzy%&<)6zLiBf%tg2ar8NxW8@RKMfpA!<^#KP3%iM$^~QmOAEr z@c@-^q$N&E9)a^D5Goy$7J>Zb!@%kU9oV6oOyV=PT*?UVr^$t;Q09U3Ankwt`NTHh>ng zsckEPs)`#;r?Q%&f}6m~OeeQ|P*p`y7{kih&y|{nAgQPW`7$PmeFnj6**=I%FT?(` z@e(V4OXmE|`l`d|_A#O^uW6GSTvc7{eYs$@#U4FlE?&WFl|nbdyz)T_xUQem_5>Q= zOu6-BaFV|5%Q*8iHa!Ze>PUE(c zPlba-6b$?>*AI-WFk?wy023~^!R_kk>{dLqw7sY_51M;LUHc3|N@!cT;$BIyGZP%O_p_3dn{Y&u!TmDsz7HM2o_qV zL@=>>a05+$Uyf}YvT1ex<(J4Gm3;fgXt8h*IT_D@dWu0BF*|zbg0dld+I8M z*3{(x?O2geM3H<*$}MlNzwrVdnUxE||J5grpuas4vk2 zUhgwxlm?a@4L$u;8&+$4h{z~y=X#AHVe>d+x#o%J)pv%RLI|~zd~7uTxFCTA85icI z8c0wgBd1Qmoh;Y!Nn#)INH_l1&Ay!^wiI}eTKt#RLpBv`p zbVRmeWpSg_o8s&0VK*KO-ty`Wp0)n$HSVH-cBJ|bBoiKir8Nv?kp(^OF$7Y6(s

Fdv?5TdIq$DVXlhOfG9P5|`jrcZBte@eHHOC7zM0(*n}p5-a=YkJ7B3z`HSf z-_PsbWy2Txd65dd%M=<8;%W6F-Vqri6AlWS;cFGa6VD-KQnW66UF~lmgRpeEAKZWRDPHytcMTAH93Tm3#(_5D1U3`;HjAKV zv7^R}`psQBFM5dHV0<`DXb9+^32(9;Y_bSm+TwlOa1`hrOyO|v!6~Zk_e^3(hK#c} z(00cz926aJcU*!$bM;{`m7>C71&BUjUc$F}%AKp3Wd@EN$5HoQ4}Em?F@{J8nvgCM zP!#2kup#XQYFulX%k4^+tHb^PE8vB0`n#}2tU2N*wYHGXh7CH;nyR*t{ZHYm=Ndri zcjEdhl|prK_=QlK&XH4&hEX!7k&o5R`sa&2t|`kj-}e`UM==) z?^7rBp@#tdi-v0e26R;ez0c&t#H74L5nYuTY)Pr3qyrPFHWF<3ri>R!^R2`Ta$%`7 z+w$k!kNZ9MdpXzQw#ej(e_C$Xpl#St6O)O+^p$=Jls#>{ebgAid39I`$~kSeQfxMP zwSVBmem7(RhSsQlMZ^KK@~U50i-Ue{V>WLb)5g130Sh5A)nL(0Zy*n!+R zj$_OETaGuc>(0A>uYtM)%J&&cBAf7N!K+y6ZhmwnUw9)YFbe&l+h@^0)mKo65aMQG z3NUZ$Yi1cvkDyNK+g%dogXk`X3P%zjM^+rO4pkmMe>;a>-yLChd$G4D>b=+4{QA?< z^keRZBiSFamEBmWe4j+Dk*yMF+L)_Pw`RGB4YDesmO^pqrV0~kUt@a~WRF8xnMWj8 zM%&mJz@G})qB+Mg5{Y(~of!6WP`m7zXb*evWsko?|I-IIwT#6@UvQ$0i|cKq z`7@(+$HhWhG;E1#4i!{c9{P|?V^qF*F$rE!#f@B!gl=Yi8SwnW9xQaz05iz3i-4$F zHZaA~B2``mM28DUG1rz_FBJMkXgI9~g}7j%Sl=_Nq)f2OIcPUJH+qL?#Y|{PU{!&F~;R_|x@wh$-E-G-KH z#6Sw-!BX}t0OqD~0*iQp_51jEJxo9zG&EGW ztIx-^W&PU}1>wxM%`0GW zn@=qp@)9;rn$|u+rgmk81Q-rs2(`poeLJ`u`x`F$%2~BHT!7Cs_+}PHZ2qieGlpm5 zmmy4y+86p!!*-Pae}p9`{GS~n`Y=X$Fe?hW%&p$FNn?NMp}ow|tt8CjHEMP+`n6sK z5A!X^buT|da+s@1L-a+w+L+j(Y35mrS z{$>>7f!aFHKeo&7Zi90?(?ZiU{HbPis>~D1nM}w(UF&B+hvd1UWQmUCuU{sd4cBqr zp?g>JlnViGF!|U?Ha^6t)Z)*6MS&c*0n8@g-M_-}(%ha{PZR}ikq}H2>d{IqUpNge zlZ=0(k;8e&qH9Z?V1WC1d$O1a*h0{t_SD%>eRHi=WvIudx>1`w!b98XE-Y@{(lXQn zmUzZ@3K!I9H-5c_oUj(NUNUR;E>mbKa%z0h@;|LhYt4ol@_lwUcakP@f1-V8K&Cr4bWVwE#XKsy8mOzy0 zd5U~W{BMRSZ?TPjT@^3B?Ue>X%A1c50H|rtIFK3U{t3hG5zp?Y*Pds8H(5v*2>BC0vvUQtswHnfUww|O8HXZf+xz6;2LLlI=o^udty<~U^c7~NX(y0$pYJW(c*0%l1 z+^C9P>yW@6{?{6{0reQ}kH8XP9Qv@=&;2wQmhC7H6h;f#i_EJj>(D`o<)r@vAa^)m2&Xer7~Ss$N9LyPl_lnP8OS?Xh+~+^4xBsD3s#&^@eW0f9$i)D(tln+(#RS;dXOkPV=QdS3>;D^ zXZ&tz=k0sg{GdQzBaiPTq&mg)<=Mcpw*Yu=oovl0sp+~zf1niZ;r(H`5rFjb@jl&f zF;g@&YXItQ3ENv3KsIKxrks8$UbTi>ZOr22>|~id$?Kqi4v_Q@N;}aVe7h|r--1vE zu+b8qc0yV0=1r1(nni-!h_U-uuwUV}n6T)(;cg91#+_xUXy*f!`imA^#;B z1ovejvW#OXSQRNsY>JWa>B!c14bP=(=KTYWqidXZI-my5F+TC4mkcW??P{oHKCw5%_fOnRL)cZ^B7-8~`jy45SCogfz*%hO)DRxm|Q# zV5DyvP7-~Mnt99R+0&4#yt#~zuIesTt;p;7$n_5f`ySq1f`At6>C4j<$A>KkF>V6g zZVm+3Ws%BaqG+KD27VN`Ccdr!9XGIDyhXE^jUg^E%Wy#$+p`Yo%4AfWXB@B)A0r^g zt759|-K$!ApCq=?KW)_EUv;enHNVE9jZcvg5n6f$w*Ivkw1gpbSwL|4#P6c`!^C$;37TWdDl)3pN`(Dg z-{UGg14u@Inx`=0^y-15%2BgKWXX+;b+l;UF8*2jzbbD03PYV)Q;P9M$EABK#DWsu?Dnb{!HR` zvY?eG7!WHG`6i2}(w{-}s)bDWaeNQyqr_PfO+x??3<4s+W?}%#XQ+|V8U7JpHX--; ziRqw^Hj6J`WxoiqbwTpm0Nuxi+Rpy zKZ+0&einXT8QpI{N@OtC6d%C4!%`@!^kI%>n4hF1Vl@~Zv$Use8_tYwmQ_)VJ1PlL z$UaYQ=N=StAktl`(XQU6nm0y@_~dgnPc#8&8<-NsrT33ms8bA}SePF&KdX(&7z16O z@?XqZI@UT=Z9OoYazjep70^E&_{@po=w>#&`AAUKCnOwA{|LNvo68^YbOISgwxhjT zcmmatmV~&TIcg@>-xAzSlO{)-CP$Odi$*PvNEiusbd_Ub&fxzz3+b)AzB^NNmGnadq^PjlR6@XOkXYs zm_PCUw$1N;-K4nA<31O&VL!O^@eQf4m+{b}*{`ebLPq+K8Ko67otxmsU2riQN@zEE zM+s|?oKRTlEdIzQTjnqm0|P^(97_=W;gKTzm8$^nwQEGXi9|~DI8RJ>kPR-P3?C)t zGv&2Ljw0cA%>qNK6v7vK;^hvva5N#N0l*E`oPC%dHd&unt|iNc^47oAv8qlNiFaT- z1DxG&M+i8(cgw|7!uG&d@Rv_BrU8eTe(SfS|5NbJN1Du1Xn1VSyIqOiHP?D|=WuP{ zJ9pX8B4Wn@e#|sF$0E9mz}BJg6%Y^lmE+9JuQriz@mn%m!LJ)9eTy|%zw+mtsy5Ta zi#>E#p20t4*N5|?%*k5_SyF>}2T&bP?}h%(Uzr@>M;{%5p)c&zaEj{a*tFET1X<=> zyWJINoZh|#YlAms8}!j11;1TRLKCt?wn*uL8jkJ%C#KVe2_n&OGJle*XGO|lL`mg@ zM%dV2ofPpM{IC*fdZmRjd5u@Kkl5m4O?qJd>hUkz6SOi9#L!--D;jp+w(c9M8XCJY+5A z6v0ZU7SiY7;eb=opE>AeC$R7oC0@6_8T^pQ`Dd-}{+CcdZNoF(Rnu4MvB*p^UG$0I zAm?@_OC}^9CWSBJ89forb*H?|t$C6y>r45wxYfJp>YkMg8G!HV<4a0nNk8ad>K-wh z0&C0q2JQh_Ie%KT#h#ZE?*VWbrG1?Ck#a8N9h`E1rR6C)R7w%)eV{4GcYnr-=n)IZ7hj=qoA8Dhx`tL z5@Cu7INXaE)=JA%!xFY~p>U5ik@MOX)4wWP?y53f_e}xguo?N{limbs*GR4ly>Xee z)bKO`@ti40n}B!+{W$hXM`A1QI-4zu&vv-{v77m4CuuvPR95N&I72Z~p(#F4L91zw z%`)y@pm!^zk;f|mQez<+=VlT5<{T!sEiu!MUcEAJtphL>{HH()OOg&3M|f^F3cZAJ+bIAn&v|g^A*qSsy!jk9&oy>pwa!*l|Zg zZWc-|(m-RcRCV7rMUC6%0%Z6RX{5P3+6B8Th1!IgSek8WWPH3H9UZo;SliIK}Nhe++WVAqy2gL)1TtN z3~>OfZ>FGn(Ruqbjxwu64OT`cAu}F)PAZbzrc=ZCu_9LiwHXn7HPx|=b+d!T%|76` zSj7gmm-<0qrjsT|Ri1zLKKoJdi1UHwh=p1(H;2iA8#q zyXrI=Hw)RiUft+3uBm83R!BTgETf!&uI&YqWw&q1dN$5d{(;(IO|8^T0B5Vl431Mq zQ!k`N@q6(kOgL08oWG`ixoV5F)7S!?Z^!L^o|DursqG5{}PT?BIb|H=a!pWoF~XhqIwjPi4mHHH-S9#xtBjRo%EY0n|| zXu_gD2du%=cnowbigKDsK02EgCmOMd*LQY~qZpoEirUDxg zP=jreXPFG-VIy+MmOe?aGC!p5c|^m4pLABMByncAcE=#*dsH3HiP|6&hQGW+#6G%9lGsaz4nzeEC&7piZZ1aeSAnH+(%LFS=%fD8 z%q1`pVh9Vg0P_Q#b@1U!jO(TZ(bHa2xg+%=1|*D5#4I8Sx@mPmcW(Fx;w^cG^oAOI zeFF>XVO@uvo^4aJBhJq=&fPxqG>&bcemehbm4L*85A4Q=+G335^FHPjGjP~EtaT;0 zb*pQgjb1#@Sxr7jLO?ZF(JmrrTRIMnPE7zGeN^MweHJzCqAv{j%>CSOD!JGv0aglJ z-^OU9o76fFyneIw0tlcMhroU}@Q_82bB2hUBbjO8o9T&F?W{p94S^h_ zXJm_ph1%eX@n`fy=xF+Fm7`OwZ*r4F8Tf03lY^ExejiWa>b-HkR?-uHO#Q7pVr?o;#O^s}B ziA!j;P0LOgr0maNp|*laKk`AHcii~f(}gS4=_6!JP>E8lbwIm_H_Y=E(g8f{zz~6r zuw%_)E;?OJCvdEewzV!J+LT`&CRF!q6+!6NpYea2b2oF{b!>DdClp=EtDE9LnFOAy z@E&m(zWvmupm%brQzHrW&@Ho)*d4s7z)W0s{gX;Rbk@^PeMpPl@`{-)AXrqkn#T`;&$p3sV=%C5L z6ksCOk2IEP#QxC}I$BmF8mfQ3MMZt%uo36=FPDMgaV(p({P9(j^QY;wtr_rqiN?W- z2+M~aQ4~K!+_BW(GJbOz!Ssl4$e*8fE`-!FlFx{4q8P32w}_GM3-Mn|J8XhWG@b+26wm2?Ul!jvhq`GP*N$zUgpc$mNKg9b{4lm-{^Xr0QrBEqAs5SZc_;zM0 z?b^LAtsgUHWl97Q$Y;WGTUZ|C)?vee&O>7H)m^!0dUNaQz8ggp73Ax`LE6aY7cdF7Zl=_6{I4rK zL{ksVlz^GEIWD&spKhIa&rD3PdYm7=alOGC^lA8T-eOqCd4U65EzP)bvTw>HVj`HAPb8yIimH}B{aWGjKzN$;bN0PWj-`!P~9?X-Cm{Fk2lpbCXIeZv^N>Rfru zQYrG;qE#`^#ktmV+o>vIGqpT9OIqFQ3Up~@W}d+QqILs%bX8|+H?JwfS9jWex?T8g z_l8!blyB2?U>}oNbGh)x^o^z2OnH2Le#P6fD!Vvcvl&^%e6OwP-Np3Vw?wT34tfw? zL&J0z#c#j3tkA`SaTvxzVlH7Oy~OiBI}0d1vIhAFJDdFGK+`XarS!?%4MOaoxi`**KAF;kuS7AGuU|`JUDn} zNwTif;Wp8m^`kc{-9GJ2ig~@z1@Qj=iz8g`@^3rKZjjbR$D}OV6nB}^D8!71XknltMeNQudM@C&zpm{arhxzxsJ?n zc$XtOrgnfNB4|h98cIK3nupZ`LJ|rGC#VUOa;Ea00`fcLe`cVt+ScGWF?A=W<*xs? zzW6~q8pO^7K2z4KFsLu@u{ZO`7M>|`ew#gAaUD}cK1S-SQd^?Ae)Iqm42=Ng>BNKD zX}It!a)?HuTyaE?UbMCr8x%_!SEFX6F9kuFZ#I@rj0PL^wcyIMfBP|-eQm%gKZG(P z4tG<_{6KTL=4a||rO!D1H!m+%C7{g6%xz4X%d1_?wi^O*XQkxg_B8t`$V_0q~5es|W!tfIH&cIl-%iW78Ebjt-pQZvz(792G&E&a#8P5*kZu zwOyYC8H7!11PxTv9aGU%UUv{SVvzskVu26a;$kgj8KAirmeaLa$-`E4*}|9CB>+qj zu1)pzP=eh0`s>#a?fcqsB zb-&HEesNM!FZ_0DzPsr(y!Lb#doUq9`>=vLJUsq|3A%U>a~+v*J9oDJm4$FkY0>*} z02CBm@Z}8tam($fa(8q6&a}A!`#f3O%;Y)&!&qpJg*fbJi4CC?He=y5L8N}@!2cLa zn+^QpyQjub-MBtohe2IB~?wEDw;zM(DQX^G|gf{e@+@Kinc8G2Ey=dLH1 z^~4Mq(PbsTjMxx2+H#bPn zSlR#^!D(2h-Eo5$q(i!;iNr;qO59FR|2g?6fXRn}t`1&R z3}KudW7J8o~~N=Hx?dl?0C@cuxcg5rvw)C4zgs z37nDR&X~L>iZklSrUB zm|E zb~ZVM%SD`M;ynR3=l7=@wTJH))1ARRuNr>k-hvr2eni~drj9*gIHS0}6#kTYbE|W8 zD{6Qf3t|_C0UB?!S7u$yI7bC`$pdC?1z?yd!;9IruWjb$sX|{n*J`KaIEbD&k63J) z&Sz1J1YpQbVJ%7V_vm}~}wQ}3o)0h?; za)u!9(Yj3@doWF+H-H@i$@GA$j4EdM)KUKM0b>g&q^P~C6F9i#?c zZFlA}?h&T$PW)h*eO+>VvvOP2X8w%w^jqh&-H(Pjj8|6>nGcLd1e6D=lq!F{e(78Q zNqf~GCDE;De?Yg}FHun4OF#MG$@7v{7@9!NQ@iuSY zP6PD^eir`kW~Zky^Vdxt&bcX%wZ=Ft@FMTe%^8$1nd-hXylQy_$ydiD9Dj7^$#I)> z9qsM^v7J(LKQMlK+xkTDT4ekYFs6UKCca?(wnNw%b|NV=m!cl0>1RZzMbl+jM}w!0 zwb{-pjinKclM|cXrOo?x+b1Ba>Bk}Db90Xi#;x#gK0`Fn_ztT+-EJ2*O3V=pc%E#< zd0t4{;l%XgV77+yJLkn#Xr1xS+9OE&!ylu#Kx%d@9GSR<>`0Ygrd9Sa z*Ciw*h{GhV0I=z`SvQWR79 zmt@0EhjvB0%azP|Y6NgIwCyWXa?AAVi9#qK$N1_lT&YBS=LbeSuA`JQCmU-uJ?k{s zXS~H^-p|_>eA@S<0{@OEgPPp2BkyjebRS1XkxLiY)m`0g=2Uf*35=y|_p^c*#O5{j zw!+V%4PTngi-+v=_GH%lhNRB2m`ku{{eVj0X_$;_*t%lE;(idX(^ah>R8?cFPX>8-QldQw6@=YK`SN(Hv(WZx-0Imhys^N!&V4G}y#9Y&)FsL2&O-xczm1ttgf!J= zrqH~((x~umhHnFGAV-?^W0V>xU+mtk z%3=`vMfsDztuD6j2^AGYR|T-Nqm_Y^kT(3wL=!A^As1bNx~lY^87s+HljbZ<;^DTS zP2(X}HZWJ7|5I1(Aa!l<4ivPic;oK)#$c5N&)5jwUY;TPp9p}J8cbbLhj$5ELLv0< zH!C!p>MWbrQ1-BqRR+J*qKj11G=AusAb%_Y>%J<1CZ*@jf^U}Sm->*4X4FH z0jWO615Ao&t>H z$JrNUy1yM*2n`T7$7cDE5rh`XfIG8F79=!DxeUgXWllIg0(1%cbR@_C*>aru@2~XA zi?wLbB6`1bpX5#i-<`u@*y31NZ+_EbO6B;Qb7kMlGk?%`71bUdXJ-R2|+Jqfv`Yo z^hKFHlicp^Df0>1sGA&%?7yxohO8OmO^F z%%hsh?l@i@#u2m{5m{s)E*&YNh8h$kkq82r4lyYPB)|wng|r|bj%3>dZ$1{DHYwiFYRKv;%NgapEp1SE#c4Rbo}aoYaXzxv+$tI#fl>esu0eT-5L@G&LU9K%sHXy7^kJw&dMSOA)Ami{3IsmxPl9};BnESYBeu!&ivw_YVA>- zJ*}!wN_dW{jbewYZ;tv@r7Bc+7*+}QDue+^%>LjMvO8+ruQ}#hw2C3TqGs4mLIBRV zJ)8ON)koQS!~J~q=cedF@2SQ>;3GYQK>-~WIjTDVhwz58nPbC7fa^HgIDM%kn{_?A z5OYtzNbVj(&U3o)nosWZ4s0C29@7&n3X)YX7QC<@eo*IAaDv*i&z0KaYq~>~N|hb3(2-oT ztEEy+B?|I`^_KzHU>F+P^wj}7Pe1vzTb+rDzkW%V^8`O78-1QOu&vq-()I}j0pgRf zWWesErxGYh*>zx`MIUbJqSHI8q|@6j*e7{N_n)Xac0^iTJO8a^cUag2SB@h;}3n<@=Fwy z*b1CIsyH%q<)K;fOjK21_=vw)mUefA3-FKuH-Frlkd=RnV@Y4URs5O4+2Vsid2z}< zfJPt>y}Y~}i&|H@V9%4yG5ed$hrPX6vpVCuJmd-gF6Oj*jEpK#pSrIlsI+wt8`d&F z-8r4Q3y-{9s94{@%~z6>th0>{Qu8~+VYX#@INa?wsLBvy=NTcx|12eDUf%!Mi&%T3 z(0c02N>YK$859~&S=bp*EyDARy-mSNHjIs*+<1gL(lvZ)cw{yhSLH; zP>i9$wmq>Hmrt`EJc)f@v3&E#>~Buft5x=6d%==ec%ngZ}$Em&8aRoV%eF)FWI{xmN`rI zrxCb&%t>%F%I)GZ=b0I|7e9UE6mT8f^Fm||Fc!=FQ*O6_TVV1};EsqFrO3$MzfRP- zX^uqp?Cy#}{?8vL6;1TBZCVYWCmrtB?tfO^XxVz+5Pya3&9x7elI&}r^s^bkeE{0I z_@(2o?tzngZQw_48W_4%`lVDf&&~U9A?|e`i4N0^>N5a@NYuCe!3BGLvuPNVSy$&t z_^!k3wbf`+l%Ccwl?}v2YroZy?W937P&q7fWnQa#tnl(Pn<6J(`^qIHX)CO*C7dc= ztB$2gi`@&2xl^>EwKmlDy#AoCa|UUceL;2UOOpbH8kQLQb=fNJ5QORD9D7|dtH}&7 z7rtjMf%!+|04h(>2Cym>1+z2Qh{&;#Y;PS$LvjxGWuHc)@y7T|sv3u_gM|E&EbxD9 z7g8@Y;xk%95d*FvkC36%yWud+T8*OHRNMHtC%jnrM^iKaa_cb*uO9KEZL*^emtkmN zaQ88Sil##iPfgnln8zvOsDd*EPwA8(*(V1#KSO`D|y+;mgwY6LI#0cU;6|s7b5*)Khfmzf#_A>;~caH7`c9iw-FRb#zX^tj?K9yORpGCBDFrWOq9W z2RFw$YFR`GU-w4E&oMd#2OiZSc9JlciF%`;?k?eDBbeEJ?pOCDQ80^5um9F}p1^Mt z5~_@BSCPV&vTZNk=--F*_{vGYR9_up#;3QdSKa;c=RFxxiuw&`mm#)iLLKb@#c{gs zmr0H0r^O$OPd;8x8_WL5Usmb|pn80Dd7f%l1?!*tF9A`TH9OsIi?NxrS7pBf$M0RR K&QF~J5`P3r@sRxh literal 0 HcmV?d00001 diff --git a/docs/images/SqlserverCDC/Sqlserver17.png b/docs/images/SqlserverCDC/Sqlserver17.png new file mode 100644 index 0000000000000000000000000000000000000000..558a3bef5a6d262f15771d4ae0d24236c7f7c1d7 GIT binary patch literal 158396 zcmV)}KzqN5P)fjhoy3U&)7%T- z8kg&qxBxabU`$m%O$~$q(TmDCl8|)te$TsR&)Me)umR`0`M*EX^UO2*>^(Dk&6>4m zYrXqjGZ6!m?HFWD#Xw6660E66v=5dv$eMz4C6?heT~`0r<=Iw}BMpP>Lomob7)cJ5 z^Oz~Y)}gL(j^}k!by;r+5^cO+s-#3C*VidCuO(P0Zz>XPs(d2lvpUY123Z{#WIIQ) zI*?>bk(6kstml`K?5d8g`;GGRyyyAK)>-cS#;{XQoqFoDfleFvSK0vIBl_L-mG7x@ zy}xV@d1syT{id&59Z0b7{huQ5*#zql473hIymb^}t=WjRk4KDcoTPr%vFLAO%0;X_ z2eGzH#M;K7zjZX??3rgIUi&8LY;9i1xo>iB*1eE>vFw@m<2f_M*|Tu26l>2ytV5+Z z=KTGY`pZD9tz)9T&HKgLvJh)!JE7OX{#pioUH`t&T-QHWexdDkEYEg3>iB$q zv!2;j9BoO)l4Efp<8iKjo$vV&8N{A4l|5K<(BC>1F;?|l$JqFckC$gX);<<-)=b1% zGZ1gjloV&nz(pnN7^5w=_wSKnt)|YE4QLz6M2syL{q0)M=gKFbzioUl#aPE9)-oP( zwuy+bPC%?(Czcr(mN-}I#A_XoL(lPA$NFdfHs$D8SGRq(Ti|1`&o0(!@kZw)*a>6zs;QTgs1-Yaq?`%*vF%v z{J)~+H2o~GuGWKY=j+BpmDA4~bLzyjAKgyJ{A}&AjCubMeFk-Sc3qsUtuxP=dgr{} zpYyuaY)}2)X8ficMv9UDU3HY=-~q>SWju?OnegsYi9J-@cbu~? zCh6x&wHe;F*-t$eu|H=2O?@f3v}`ILhX}v#C{#KoxzB$TKK~K;{D9sJHW$QS0NM-EL@mq4MkdG1s~e&BMnbv_EQWE_6IP-urxiEBOmb z!|(UndmAx%?)``pXOE6cr~{}Rn3y22PX{MFDZt-!FwN_^3eX~`B}tdjPEmNS9XD~z zf(|F|rJX7Dp}}-s!s}nidF`<2oX!-1H0_j`ZRvC0g4Kc zN`I!FRNH1D#nF1zHl2({>hmFKy3e9tuKo45*W_h$TRMhV6|PJJ;>1u_36Z z<^wGQbA3+@-1B$nScbQ8J~{~3r*8jzb1hp?4nsfvw2d0}94ePOCNNF;lzp1_BQU4H zJ_Y!h;34l7+79bzopiJrJ(tq%)jN;xKCM69XLJ6z5a7?8H;r*M-l-jBGyA6Q8+u(o zmn@^6Nj26HZH<*+YH29LXa zrF^Tu6fHiFoNwW=zf4kFV>x(DuKP<7Xs!USxB0!2TKry|ZY)J(W0|C8{+3jVrUoyX zePw9&mE&|{Ia(Si8}Ch-_`4DX@hb$ zDp|G6k{joEKhcMe_b!umsH2vKQc10p<1C5iRVqWP|Kiix+P<)JXX%?UHevnkJQuXt zPQ7&6Os5TW+Q9!8Y(T#|&Nk=s{dO)HV3v{8sC%Vdr7^g%g`ommgBU) z0%c{pkYFESFoBGJ_AR-Hv*+N&H`k-Ev<3y9y(skVMZTv>Qh}Ght0fh9RL<*61>S1p zd#fc^Wz=~dvuu%fKMFkiQ0U!<0!gZVQE4@bN?EqbXoqd7_SmMh#d}c}Lmqw4e`cQV zR`=Ka%a?oUJ}S98_N$QnF%tXMF@^S(eNp{3bRfE5`&v(Xp>(kiMWrmGu2J^|)W4g# z)?YOWjWPvp>ZKZz8+Ah4r0zS)rOv5w`MP7atrPEIChE&bp1mldz9^%6FY?{h$Pcp* zeXlchr28bgGIXfNK=tFR>#Dx#G9B}d_v?6F{nok;zn?KSVs~1<#T5v;G<`T9J3T=@*`lV>d7y&oTaUL?;kodY^e%5e&J>a=sD z90={OYKPc#@=OPVPDMwsPdi$4&{a}kqIP_ZTo@_Jv0)pggU$Py_qxzy<*+K*O%NvB z0W(2A?VxF}?d$Iw>JV$$OnoRvQU9j1D6AVYW`cu`yHBx&l_5w1iN{x z_54k9%3!v^K51}8v8puSr^iCcp={Bt7cP$ma@2KImd{%(nVR~!GZmWBdL3P9yGowOd=2IUNwRgIVVe1z*m&$W7d6*y13=D4ZxQg+RD zG#GU5@zQma4H$EF2M{p4&!PG;^{PHxbeK23#zxN8cWsO6qX*kJY*fo+%!!nna%s6c z%E)<6jT!qy9hzlXhq}@8Q+Qo%2U_p?y+Zm<3p1g zGxdMF0_qLBvyVkYOG6Rbd?jdabfVQ)j7DDpnwtvIe!2uL^?7Ki%|mNlKH8g0(AHRp zKr^ouqrF*`Z>=jpyWjZjFG6#D5!xC{(B4!eskI>=0e=w!O~q(w$VY44E(99#(AHRl zz-brS+se?^LIAN4r+o!z^)nTs)nA0x#$q|gYk}rsX}6`J7_Gh%^}D&)IA4fBBkxy? z7C*~5(e5unJNw1l??ii}3+;ZF%6YwsGP2*{DbS>@1^mTeYV(&Mpxa`5zGBs1-LBt> zK!>F6smon#Zpcm9FF3L8#oo8GUZ?IlZJ^TzI&I)zWdnTgnC~U^?$YlVd5<;nopX-y z;rop5EnkuO4rnY1{~gx8@&4ocj&+(!(CFKPf!1_`;ivvdlYfq^St#%vKmh?Tj|O%2 zpun>mh2A|1J+|Sw z)|GBo*=~upUkyxCmjbTRhRrc5G;}St8kF0#T>%u0aVVBNRGv_|&+oXnbT4!&DWiU> z6zJ!zL4WHQ!`U;PK^74pI;_R8Et2NK_jk4vB>LX(yd!&HL)I z3hX1`XI>AN#k}wNoL5epl3y{d6qsW=^WmVBK2O$%Fh9arfM+^w=W-;^C7Wj6=$wwZ z2K-bV-h)o321pGM#t0kLAdHrij+{yg06W)xweu=rk7T@PpiNjn6I7U{4ucDEdPNo) z3&X)Q06&v)XpW&dW`?~GFbR=cgA>6%NW0oTI*u#P^PK9xu6t(Qv0nJ^j^m>N8J&ms zk8+zb6Yx`@6~}~jqI4YIPi6BO?9>1O`zWEWXRxfo-9 zf&vDVU4;UEYTU#1sQwp_HbQMfgDWBD&OV=9oDLL#qU-8Dn!k1b)cEN07m};zTR&T7 zuIyQjamV|Hx3BF==h}wFjynK;WS(hRMRvPlD4q@QJG(uEJ{y!HM7H3(qRy%D4t-Yj z7#aYV0s5&nRG&=QD5rj=jd~$te0JGF^nBw!iHAtNL?<> zUi1f+Ah@Vx(&I#sj(wMTQkNCLPXl?>qwW)r_4)8TG^97|{=vGsjq}O1-jvRjtZKdy z&{*0u#y4a<*nee1#+U`?wDIU?A15h-09^~gtJ7r9mEdIUPWT#m&WY2th43HSfwuYr zpv4KamY}_@2<-tU5GX^N--YH=g$VfFKwCN5nmq8;mY~J&0$PjF+PD*^Yx4wjWBt<& zyWl_e1)5LoKx>l|ZLQ@%dlg!mOL5v?h!%eyT6{av+PDjCO$BIaEI>1VHy0w%>PBm$ z8?E&&1pF?vwHD#@>0LN|Iv=g43#EKpQxTedPBhiiA)@1>ogM{f64X?Qzl#MRG#x8d zhJc@0w+Mko1^lVDbbEBj3{X)6f<`@6_Z-gHxtz3fo@XB}CjGPS-`UQ;%IWXCZ>Q~b z+Q7f34G5UmR4gCK0`g%>y_4kK(!>vKMdt?m&iO7ebg4gZ>2gi{kjHnS`T)k`CjVXx zun_Q5|8OvziU0I)4YC5yegXRkl!StQ8r(2J5(Rc>Fpz+j0`fEnvKxi&N|bmB7%2d% zz+D*x^a|V>kWv5#+t(nI2C6!?r+}=E)zAAv8k?*yDghtjV~(K+?5?!*45 zKB;p?-?a|By9K->fTxq$wh1088=y?;oNkL_tN}*pw}4N3g8EguReBM?Q|sEh2eg9% z_Z|g=3ecxzFaSvdJS%kb9;%I?Y$5W?Scd*SV~k8)srtID(A>O7p_}%r`bj%hT>^$$D@~ zzv2Bb9Xy^lollmXC}5O;NDQ&;$UqWcGUPsIIGD;Y)XX|9V^9a;6lw?23@eoKhAgb3 zR7RFRC%QX4SE?e zYgX48V@q(s2m=&-OaSdfn2^E#D`%YGjnW(6o1tvZ3p-ZN<@xr_7+-^>WcFFvR4Cv? z@J2tIu?_;9#(2xKtpQ}_)QNf?*`6MM(Hj|6(hlV_^%5)>TqEv$6gKXQ=pgnKQHXLma_W!4G#?t zm+7nZs)5Jzs}tUqDUUiYatokN9jX*sMqM|`(#C^KYSYHm9HPo_OmsVBV8_b;x?HE4 zNrg%)=45y%`I+FK(%;4O@yt4E9BK1>7C7HMwvmC7PRE!rUX6=<|C496ar^8f)4T3WnlYA!{6gGb7@)fHo3>Bj)7aO`A0{Qe>!z;NGu zG}aZNscAR-{yoyCHoqHzh9Y75wK7auLYce>_&q`PUn@hGC5*QmZ45nba3VmKqR%NI z%_`K`gFr(u0u6=2f@qa%3@b8(pyw|K!857 zst8aN5R@oYY$g*F+atgR!3GoX>%e%@Ad-MiCU8QKgY9TwM7POY*ij*jGJ;sTeF2#? zpmUKxNf^M=p)D;FWe6*0$g9tpK+xI283o3KUnA&N;@*v77l9)K#1lY?dQ^!35P>fZ z6smp`3{y#sfeHArTt}G%m?Ajku8>sZu0#<*tFRPuUF+a1IrSl6t-BmW?sAb$g&6Nf zfro9dZ3QekIo7T+6iYeE$Z%N`P}O6t{c0!NC z#~}RP`Wkhpk{$=ezBF}9AW`~CK-CyC)=@yJ0cw{K=rce|jh%-RfFqWde zua@J&@=(Apv|nTqaxF>?_^I&HTybh*EV;_5FaRSu8rs38Go>Bt@bikzq#cVO8$>`8 z&6uLIs{j$jY%!f*InRam4fcUr8%zU#;ZEO~4$--n=*%v!P$@Xn3%dYL!0$Atw+^=vmM$%_&HNIdK@TAXgj*R zjEe^RL~jb-bU6p>wQm^I~(#r{){ACZp~b&lB9!0H5AyME?$b z?+HLCbF#8&_CLbcd<2b62W7pxKy3j6_1o~~(+lw2OG|OccL0GvHQMQXocaRCDn7(j z-|vFzYB$<^2hh^64|R+J>Vo!DRaHU{a;S?7kZ$5&ihTWJj zyf-e3j6j#j2x)ij&12C$G6I+J9z1`E+5YAY@1pH^A*K$y3EjJx_kMoWBCL7x*Z5AP zlB;WE1p3AN7>AEm$y$SLegb|f#F?R2|D2HCuXpG=I)a_QJ9XA+1D!U|X#@Xn+kk*{ z5}I3tU*0+d^KMVayxY=n&n-i6&#glcXy7s$WSgIv0=@#Z{D!X}2pG!p8BN8qR7M-G zH&H+QsAU|JP1sn3Hh-b83Y!|MFwmAR|D@`FSon_y|A8#<9+36q2>9vHUIF6_kdDU$ z83=v^&y(S0fPy+FD51(#!bvuou>DLRMYF>+V4>R$wtW`hEsVw00T9ydoPQn;y72zt z0Hb2e@m$9?&kkT2_r??mkQhMmZ65UKEodJj7HvQOZ$Npq+A5wIgdfVG2B;L#e<70zJ`IhDB=d z#{m57YBLKBHbh%;6);DTj2i*yWB@68?zzxhJFIkqc+8}nZ*{C3d4g~zsAM{|nwh2@ z`ZIH>ZvZ89ZuPo+yiZ>`eyQUzz?zG=L^3H&wj%W%?r5sEReRheQmykvr_GdI0i<-^ zSl3Jpe>U6JI@0ayT%BhF7_!@hrw|+z!hDN{g%xEg3LC3KYjoITe-fEnX3&16u_hU zX4|ZZP_wcspryoCOd$Bq4px} z3NzFM`_y#U~8v5C%$WH46tRrP- zc8;N%6V)>uZJUI?c9xrjc^{r97i44d*_Nb0 z@z6O{KCD>AAv&3xN2qrQb#$&2s!P!c^%Vl>hF<5f{vB2}2*|7ct83csS%({LsPWKt z&N9kQm>Y~f1L!*!;Ab-ZR9&s3Z;~s!)Up^dhk)ShFULv07kGST;`2Yj41XrW@AB_= zN6mq9)E;pnaIz3>$G77B7aqi4R{j--PS>HewF=G0x1rr%fQiF;qbq^1hzRtIh`{BS zMxy8Cm!i0$7!AI?K+7Jq9ovSYPoBjM|9Ke>AF06nS)*}fWTe!){OT^~)-w{9^oT^4 zo{@O{%_mW}Zxc>>|Bn8>F2Te1&%vb0<8a_e4G=hvrHg-sXI8C1V|x?!A32Cd{~?^* zyAJU`>V|F+5xDB||3J@6B5`R%1Y&-AB`&)(65V@tN0&>x3%J)m<~lSU`U3l$f5)^L zxu|Wb$0wh@i`aWA-vE%Xiryrs1SRsCv9gBUtx1qlIFjhRf z7|*SG46C2|Eq=dXl5E~lcWMuYXN*M6(F17qRe*pW*FOxdvlImVE+!oJpHb)kZHKoL zu;?7WP8;a7fq!Qk&};DBb6dIqd}ON6yEP5-Z%;=%*VoC`hLIQpX_F-o@W-*p5aN&Phf6iMi3@@jr3`_(&irm$bie&==hVbrz z(^HNTcbN+FG(s|k0mh9B$ZC=>QGyb?QRpNfrNNHfDDhOF*i|a*FwOKTAzRF|8{9a9 z8(k16A-gNcygLU2?7~13f-pL~lE70XoSssYxJwmyCYx-i%@7p86}pDrR{((_V4s8| z8*A@5UAy7jQGy$9=!;Lb6<~YWUgQaAr`N7iYXJ&tPlrYu%rk;fPH!b#ZVyUa9s~HR zMhQb)T}n^fFhxR0sT%{J)BvI6a!#+Or~ypvLE1)nuEZPMBH1kgPk+}xk^f{AVDqy9TEX7K-DU7FDY>)b; zysEC=w1YBMqQqSZr@I19VLnzM&%+IC_Ni;03K<`Q-qZ=psx=L{!OJ%beS&QG5pT;zj5S+A zo(9;)V1SLlmU5P3dES~OP7|F)aeC|%a4pDq001BWNkl}Mw82zXk?Vt|CpD)yUV z!+uZidD)HcyU7fL?LO!=7;7~syYWRhthMgfKwW*5pa=Ge2pwD11bECpW5F0^#& zvkaTe3bLKpU-m2B&N0kTY(@(~P4-u@qG;hzwikY0jf*}!m{ZEFg=RklZ9s-KmB-yFoJ&Op_nI*iJlgg8eQVhD=M#|iaO zz+|iHwiPJFc4Hlrh2g5eP~)SD-C*6>_-r$gHO6r^<{Zv_2B@R;5SnW+R)wAFdm47& z7%%f$Z`(9vOn(4J>)Y{*`47q*!e>kD#8`qM+CCY5Q)XiHwEJM9?z7BeI%J!SNiv?v&NZGp9!C?fx8@?=l8phhX?}#TO+-JknEB31CV0y^Gz*H+ zo+Iy{INKN%E^8Yv`r>;bPTo&Cyw?Z|CV;F{kX1=gE;Omk&A4a2ZAAE;Okd;0FFTS!0 zwatE<^c_cI{T>AB$Pz5ThEHF`=o|;;&KZgJ(-jERx$wX}lkoccFCnk25Z1x*ICj*7 z)`n8}4s1qyV?I2eJ&Wv77OZ;Zd13WMUJ`*WT~xZH$E8^R*;@SSo(UK?JQ~|STaN*~ zdm~VH0Ixl_2;c4ziOQ-9T>FC`;fbeTKy6bknwqQ8?%Rg_uD39CMlR}_8nE$`H-)M9 zJ+k%4ASA=DYZr9s@ojvzVXXju(?{GS?7~ZU@5l(Oe&G?k@brVYv`d5lja?!l(7S&h z)EwJ`CZ9KKJwI-+!A(%wd<>!bS0*wzI#Z_(I&GlS20Crv|1}#ROPtF{+;iJdwNaF> z81rsT$33^EsSQ2!8hmHVeZ_e9)j#0g+tRV|!xiQ_u-tIUS1jl5y?rR&``g1|ZApl) zzZlK_y%=aSLwfZ;b95N;-TPFSDFL6-a(F%YD9rm3y?Xr^5fR@*M8x;;r$?U^V4=`c z0gtl~>puEBF1_qZM0Dwah%Vg`+5J0s{>`=6&fR`XE0E`O!RkoIBac3=!Xgd!pVM6i zud@*AKl(d*eD_L3UeW^*k=@Xx`*-l%>ua&I^r(P-0zPpaJp(}M2zp%jZ-PEL5wP;2 z)V&MqKYkZIF1tqh7}>oSR=xfa@=EuEyRk~Rq>Dh8nKVO;xvx(E$jcEC(Gxv- zT#dC`UD#fB7=@+#jqqSHy*jcD&Gq9nkfuUc73irp%ut(VD8Q8f>s|q@)_<@TS6+TK z*1WYAJIW3sUjRwoLxXcwaJnn;3Cn%&YOH$WLjn94)~mp%J#dy0EG|c}r$PaE?z4ek zp}-u0yDAh*IhJLpb2*B<71Cy*Q!(pwSg!(ub$#Y0xTpI|cA(OyDklJ^rvh%T2kx?R z{OS*jkveiLO5AQZ-6dG{(yNH*(i0I8m!nJ1Yw^j}9VjX-#hO(wqiaML^?zcwo_OZX zPq4l02&T=LjXUqW6Gi3wk>}Zq8PjIst~-7%fZ$H=0ptmbQ-PL>9jIlzxLn&_u&)v- zZZiB#z>kjIMD(`ILfnu!*nhGGtpH8~ShVtaB#84f9=#pYFnZRnaH^>t0RW8v8Un!l zd+&!ODOG^2_>{@WckRZb%bykiKsh^O1yJgfIvt~CE)3Rh2Hyz>4Z}kQPy$Dn6(i58we(Z1AaWUkN&&rs}B);B;u4btc~bU*a(Gjqx`5q<4b1$18gRW+P*>ezQ$wicXGGZL#u=I%+@b-HjBi1rTbnxoOyYTWq zzQh2_Xi0Cb+lf~{*opzxQAn^77-Se_ICyoYOkaKdY<3rFg93^hY*>AsV7Y*@*2xmq zIUs#J_SOdEIc49LE`Jt-tRuumqf#a#XZp|OS>$uh>x}{6=ktD#1j{gC07ng;fsL+{ zSi0;<*pnU7S28?%+ovKbWh%0#nf2L*pY<2~w=4xhX4wD%x6aMr(&N@Kr>+#pqe9~O ztOSz*Ec4?*@04knbN_NQ8S?v#Ig3xt-`{&511+NvOF$&Z*cuOpZ|e|O36U4ZsR9N0 z9H>N)YAX69&&7bjb5MQcB*u(mnN0J!4_ix9v)as%Vqvq{rb^iC7S}$JkZ0=V1S-R(Y>AwF7s~`z>NEQuk}X4!V9P@PWP;*~!A)7TJTew%e6aZHXIks#d~(j|O#Ntww%Ldq zG8+d@oR;@)3xG#gJdcFr48&W<<0i*cjGX-|oNQ`G8~d%_%kv(9H8~xz$>R{~n1r3~ zayOI3LISr%->Y5{LGc zVBF+M_{Z80@%j3f@x5-5xUzesVi`t8;QFh&2&0eS9UY2@E|&;kxax&Jpxw6x`<<_2 z>hxTk^4DYICvRZls8|6YS3mw+Jo~~59P>9|MD{pr*svCXIyVBf`Dm->Mk0H0+P5Fi zJ@qFnyniO@YRl2*#vkJIPuJs={}gH)56W7BTtr>iC=!N6hTg^OtjmSj&2%wsnDr^c zg)@Y-!>99$X`{2wh3majXPr9hw1G|=2)6;X%X|><^OxY>TZiIa4FG=C{B8hy@4fnv z)Y9V zZxe3l^*#J$(UT}}AHlA|Y78AY5$iW^N0GM-FRosN?p?d#}?@K&P4Q-%_!6ED8-Ji2%7ikDu0Th@jn2>79{zxa6p+X@&S z>D`YaVPI9Fq;z+Xk;O1k0!fNdMIed{G_F5LAcf(oior$p7y&4~MxNe4f#;lVFG}(@ z;|IOIhd(WT9Qp18cz*R;xG`!FHWpN3CxI>xH}0Su6(}iHZ7P6{U|=;$$_P?b2y2m{ zuqwooYpU%>iK`mkt;LAGp)Wq(R)jArYXtCPdnQn*HV2`8LPL0Yz0~8z?!0vvVU5Pa zk3EB(WhdZr@576$-#|pB(rWA~+b3Wm`={7=RRYLzw_;C`2c<4D-u5A{yhh!_P4*|j zK8Cn@QBdN9EoC_Vyy!_aMlx0gyU$%M03YwK#*?AKdr{yf!;YY=2ZhBYu-Mb_=S5E# z^{YhRv^m~`A?W#J+{xOLMz}548FaC&Dipg2_NiTvId){OdR_TgxAp@>Ck@5sf=akt zF1)zv8T5|oi>>)iv4uSM0kPxlMMX#(JQPJc^03=ojMb}OM3-;>1RriGLvcYqh7C!< z3vYako!%Na3k#4wI0dWTT8nLE$3!m*a8!&&c@D%zHBhO6KmGgcxdwC1`o^XmLBQ{T zu=iv!a&cP7f|!K<>33lFsQ~`E{5d3AZAeZXf-U(TjLf-RSOPcSzXT`SfxG4r97~sV z0CT3_g@(o!0R`d{b1=v{7KJ5lJhpNr;uto}&|ceQ0Z-;GRP}#8|3RrwR^YhlccI?T z`cGh>B}0G~&M{;tXs1WmNn|gH(@MaIz*V*|G02jk1I4n+kusPx!r~(9h@h2nrpe49 zP;q1Gt=Q=GW6`pwkZeuFo%bz3eH)OGH4BN!St0}5(@ZAr9xSX!gZZTTMAnD`+=NkL zBYPzmF^;k5H*5+#M+3+ma|`SPGch1Zt^1|{KRuV~FgOAdY8|<8qNmmB6H?K95F!Qci;aA@zzXXltm4mf{i5y@yq$Y#~{ma#15W- z&91}3l1ik5O*Y{FBH$MUNgQNZl8rSJaj7%0_f#t$dEz%SZuQZgfl& zV8s^zesS-wFwj0+#wcg{F9dX0vg~mr*#^rTL)pmeij!wl0q<(gqi6e2t}}Vih!4jSvUq!jb)pF&JbSC14+)GhvN!j&Klo zCwq%5FD_$1878W*b~00~If%1oAx@SixD^ShH>3K{F=USAc>;b0+l|j0K^Xy|!+{_^ z&;2Z92~I<7%2aIk?7<@|pG2H}gv^O|Ecg=$+ALoBG!m^tLB^H>?hWwEV7*ayGNh57O($(0}^a0xZ}Rx zpe_K6&b}3i$r&gpbm15C9uPZ-Po0KcMWwj=-usb2COSXV7(iciGUZcqiU5Cle9n^7 z-qic(;y^&LEq;h`OhZiC&ro?XfW<4HN20}oq||h5DJa8;?Av4mrMU|i8 zOun3HKS!OvRY3j$N#iiUG8P4HHzjj98TRnQ>&_OS1kC&mPt_FACb+fF^mobj)zuoxVUBfYX*bh;>Wfd|q z24cl?OVH&zm!iw1UC_O2BzlnP*R?A)f4K=;x4emkcjqEdw-N^JgMg&gx-@cw6Mhab8UtV4r~lk*=O{^QJlz{&6<`-#Aq+f#v! z|9BU+0rA*UxEnjlkINc=40&{Wi;!2i6Ic94Z#=a41p$5tMi4A=xhnAL^RJ+1x680@ z>rQO-Rv$yy zSze7i?_m_UYXm4Pc6(9asgdL?+YOh;jZ&u@9#=W?Jo^RwqAUU`dc6{^OBNbI9~CxR zjUs}A0wfXeE61AEYtXxIG`=hl17#KP61R5>GkY?h<(1C2+aR zgps$?Q=`_lD`l9c3#CpEN}XP5lc4F2vco9w9E5kf3(+_9!ADz*u%)sF`Q`h9z@BVy zLB=21a|X*VbS*@$+lf7SYcb3cjo&}E3R}t=Q0T3JtE3d4eee-_UD*pCeOZF7Wk(d~ z7u@{A01pL3Eg?3-**WJhJ#H zF)z9A8-U)z8RX8~SW^)##_ zr(;Lyag@3X@zUyNVHq_8UzGc>)5FbhLYS8lHci&50$aI8Ak&Jc)?n`K_hQv6?|{rp z0VoYN>Z(`Y!R(*igJn;zR-wZNlTiW33Z%~~H2^;@N=^rj4yJtqM$CK=d3#$hWN0pu z5>r8z4js`xwpkdEHXDae`ml8Q3eb_1kSIEuDY=-t@Q*m&+K%k3pTU+m8pS2WSiEw% zIJeREskq60Gy12^!J*nlELpKq0SppCnTu#!Hg0*~5u9iPMrYoF#KbX3NFE^}c?2N| z{tU2;5@rb5A_NcnIS5?QnI$+wHWD38f?EWRvJmgc#DJ91h_?)tKF}!@7F5bO^mdFx zRQha`9t|KTYcgz>6bu}iB|zsr_dbN=B!Z4P5+>EhF-=%uY;&MxB$6$|FfeH(`dcO= z%0|`;%Vh|lGSD_!z@IqF2=pI58KuX`@Vf=JgfWOuR^S#FyC*9~99g@XPldS@!bpsE zOvQ~uZp9{VJ(e$B2}e>Q*e1b@Q8VYGrlB1F3686N8xf&TW<=+Dq) z>m>BIGgNIj5;>mMQHZurLBG_g*x{_ik`+&2Q1W200fu3dbw|L0?5M;)tzrHQ-*SbE!L;X^wtXCmojJ|1}2S=Fv{rknb=d` zj>lHK0E;C}n12K)ZnDk5t@pEjE5_zzAUPJ$`k?SSiZk?i~!X)4W5kG*5%>D zcRxbPAiDt4d?uJEAKACG+gRIB#8d91Q6R9(HShS`aZZc3j6i~Im;gwzNpp~tax-cU zpFkD?KYqZF&2rQR7JQaNnRx{56$6Z6s|5U}i;eGcm0{82k0aJPQZ1cen~Kb-^Kqiq zkL=uWh$S#7OkHDX1%pki0Y7=J*#}utf%A2a=sYobCG(n`D3srB_km@ zSLRTTMZXlz1GK?(V?1&|mNVJ6YPp9|GOrFy9aF@J_Au7h|mZ_e}u3udBSWqhxyIZr-x*mH5S@m|f$yaV>6OcXm^ShQjpBA$6;8ERS& zV)d(!6W6+`LEDGz!#zfdj?=gK1O8qpq6oyCzlA7}oXvg`g?_1Vt($lF<_Q%{{X&}jpmHqg-qiX^O7 z*4?``4GV5d7vPVfwrbtHj^#W2HduND`Pv(u!q6jd$HcJQuhX{%{kZnu`>#BN1-A{u z{9A|0aj-t$RmMCYEblKCW;?(7GL%Ap&1BAXakxvbgp+!zWU~hMj%~R9>Z|e4qNgNu zaaU;#O3Erw?%jbG*F23MMyKM_f+N^PK*I>tbvw&Yx^)|F>~$TUd-Gjv^z24b$_PC2 z*A)WzxxHSLJHN!stDnLRQK|T>;3)FFHAa|dl>m2TJGP_ub-nQ7|GbH}HhhV1UmJ(@ zn>+%leYEx?T;8i6)^Bpd?e?H3ZyT<=>MC@J=qh2DbAGW<0GeISQrH}W@$kcs;F@cH zgdhDh25|#zxaH1!Ab^m!3eT^81HEsG!81=jg+6_w@I_%M7^+-aum$P%B>Z;SOZcMF zCjb@K`6ICN$KU=JA|kpY;*!hJ<+2;Fc1saTydLD|@4$7}T!%{{yU9Ap3{SqUUn(}{ z?T2fd8-0IxlQ8_YR31cLncB&=&|4`$TnWK8vf+Yj9})npl0@)~fZuRy3?5vv2Aj*9 zu&eZ-Y^bq&*H#QmPQve&K94UdPATv)WZgO)7A%Y~Zv~3H#mFnl!}UM736K71IlkBP zJLnS81%Fxm1fE^ zt|Pl+=3T!=p8F8i|6?P5aP<%I$(C&>C@aH5zyBS&u}sAG(e2yUV%?`7qt|ucN2Ek( zd>jASF9{oVlKGj2haY|v*YxU*n|dc;N8T=cwDupk?Ar{d?It0$e_s47^7FRix@*3N z$cQe8i0FYIMJHivVL9^iibdxUa<6Zp%eQ-B{g=haFRQ{1mlyWIX?W=2zu@X?Zp4j! zlCUMugF@L=mq4Z$tP56;001BWNklU=$P_D{duu(mmL9>n57)t-5RaTm6A^hy zH$-;riP?AEhrF^H>>^O;E(gI&uX{Vzta%Vvs{rc&@<0v0KY9<*NtxJO zPy?^K5bHjEA5qC!*idu=1*M0KP;0#@i{90jq01_9O-2R!y@)v-K)=k2~jKg-<0W5yzHS9mpihakL;Ho-|H~z6v06eZ8b=!Rpqx4uS29CZ3 zuYOdB_ug5D-Fbr#B zh#?*I8`H=@ti{S@E0B_yfcWH*h;mHCP3beS$#ViLAA1I=gHnWvr*@WAA&&xV80(VJ z368eRMyz8Ns*bnfFHfw&(dKrM_tD4K;I0M#Eun7=F?{duYmsOhjL|dh!aiR+MrHgG zR@-#!Iog63-d-!^$Vhtoy^oM!Rps{j+L4ic7ZMW3;?bx72MniV7`U$$$eOqS`;P`B z{EVS>)in)BA2kiTT>J6Zv#;U6sb=gx(uCw;vynM%p~yu!IG$t?4IDfj)h7bdp42~B z5Bt!G7&v4isB?yoGBvccV|30O476~gkqOx0K7ggqy@rFer?Kad57zWa>itE>Ic2iw zZ$rsZEV$=4uqLt}_h4sLBi4OhDC|kz|E<4&h5;6i8P~%ag#q>vc;}-HcoceEaJq-La7)Zh_vrR@H z>s+zVL$!W9x?%-ltqg6=5g?PcHuqOcg@HOGeH<(Tu4f|3HW@eBZ$*6CTpT%k4EN5T zkG?~ui5;$3x*UU(1|fmKlw$&BJ+K6xqiq zbv%HQ-6!zQyB}eYH649YCt>D&OHf@?kF>#DmoP`3kCC$%i2Y_~O@k#N9dW{bBM8U! z_$J{m&%PseN?Z1|03&m5m9Wl%Lnfo@q-j^Qt*V2KunoBr7TX-u96XN99IgpBMp(0C z&JwIvY(GBB1QEFwB3Wq0POe-h)H+>4Df3-kEO~sTd>9}Dv#)hF;!~%i(CNiv%a&n) zeT=!`1i^oXuM2}vh0PL3m z*^_>O%KCQH@j*lzjvT0lWAto%K?d^Ef5YKpe(XO`2gk69$e3|IYEG*;hB{uc>^WE+ z8Q5K?>j!Xbe>EIwQ!#MxOzbDQ%Ff4 zD{I&ZbJ8{o{cV%6qsW817W@W%>@zWH`tMM>=Lj}z{319%y!+l-BwA8&`@-L0Q^{U6 z*IFjxC#iGs(t0=Ed-r`fEQ63e^EMRjt%W^p0un5=)4P#ZRR_n=X^2nEmatyEMxSEt zDyAyKqnW-cwE1G6aX#NsmRv-qOhP~XUS7Hy$w@=yBL|s|KjnM*{>SCLIx0QY_$bEr zxdCpEoCWX60Pg+8pOKz85=DiDSiEutBJ#>M<7n#vG&fdY?zjQi_Wnb7a?#Cr?)9f} ztob<5S}7r_{<;b{i?-lL-@6RgfA2DsRl0H9cL;UZ;J_+m6=y?U2mn*96GcDx*Oh7Z7={X0?LT8Y-fU*b1&9e8{71NicbRrvj%eu-z+ zEJN4skqX577S|KxPP&(1-KTHkv~Md;pWK8A!;*2VvI31K52Cs8FdF?e2(+KXf`xbD z`PIwdZ{CYmvg&-BaKQDZg!I<=>k#l&B2Zf>48%2${}#_Yzd|Znskoi^}qX9EFW zAp#ABkg(cY(s1uBX>!bSUpKF-`M*TKz58w-9t7*=-JA|d1pUs#rsRF%E5;@=`)*Fh zy|<)e^G7Qc`Ctwp+keD)306&I!y4`N< zaejf9Uw8sPh#rddg|)&aV|Xo-05N%AV1zXme_pl-Z*AC$p1lU)lg(UHutY*IFS{lV z8#Y$JwQU>v{@_Zy{Mz48$S~oO0@zX=`2Diyu-Ul}4*Ljv=h7?i>6Ux}&t6>h68iS; zi!X~^*j2t4lV{w4MUO1O#t+`W6+I*I@S?|M6NZ=9JTHv@UoU+bn<^RvNF(cP^9S$Y zhAXecx-W{ct^Ak*0vV!Au&>vZczDSZC@kBJ3eQf=o-qmk5tD|`3XY*{0rS*`Cme@8@Os>+Dp-%<*8ceI(igF*ya{=whv9Kj z&zmsJItag8x*A_pHVC_sOhJYP69}R{IWDqpoVOItvI686?Lx1s`=IYn6Oq4jE7rX9 z0wTIx0mtx(aJrrN@Z*nh+0`-l?DI-|_Wnlv;EJoUcH8IJSW<{9d-X;7kg;&PocMU% zTJ*f)r+D$@b@=@6n{fRVz3}PQ&+*Rs576tX8}Q}lB`EY95fG}#>qe2Y2v++L{AJM! z8LRvfH&U$W=+W&8e6qPf!m$gAT)1WKTomMO6NccL=ikP+F1;4(zubzQ9w(Ah((un|65edA zKPW869eL<|!;M(Gtr$DXs`1k5=Ot?8;iXTbpu8HNtbHF>UU3)d%HN3}{x}hzZe|Gj0bztPJef&t z2*UAEU>~0w0)7Pl2>LC5N`Za^`j`j+u6X7p0reCkRE2}f7zabc8A7fYekS0@@YPAU zDP=llJ+MpwECM&kY-i%ofJc&d54q!}bia2HB^pc6=n2ej!W+?}K@k>Zc z91Lek0T!=Z1~PMc+a}}3L+-@Pg-_z}{zJ%+`lBS2PQtzHlh9}AY;5q<;xEfz$K;8( z!hhl9_{D;G$e42%_B6C3 zBWpgUOk9Ygb*;!4H$m8L0~1Fg)-oCK_6aC(d$5GeCWd1=#$&su615DS&Y6XQ1k@bk z@y7ZRB#oGk*pxB&*}{bqT0LgWZLlQfps>h=M^`+7DBDED4xWy~CmZnY-#<`*nmq@z z@Bb@~wE>x#x5Ap7A>ciM35Kn5mtg^5#3`qpjz_=Y(_~z7GG`%q&?wBke=$zA14|xz zUdDd(jC*mQu^sc~{RY+~h8~UXLn z`~{^)n~|C}4oS%w0=SW_*T*&ov5sjdEFr+P9HVF7DfX0^F$c*>V=#K=J=jCQFKa$h z95YdK#E)Ox_g})uO0tcBv#=0LS3HHN!P6v^xTQIO+^q2eVBNA{8EW<)LRwln`lgP> z%j=2+P_iWqL9B(#228@6Yd7P~4?ht$XfWj6F--u_QB!}7n)(1T#!tc^%Mbx|qioX= zH)N(_F0XhT@wQQjb!6giA8(Q{Z|mTn!D^o^fL|8JRYKM$O1L@!KQih1ag!eFX!(di zw&_6GR3}FO!D#CY0sQjZUIALj5F}vLH+cr)?30o2^a|sd=k?|v+$4qTttr5d>|HW} z)y53bLvNOOr{=&(3{4w_TNeCL=IXqBudF#YX2zYUs&B{eoO>~D{5|lUsK;2DH>aWU zQ~-tfWx{ODoOu_j{q6YKyd{`0@h&u+tVMS2Sd5x8A5}gwh$kZ@F-6w#7~whD^aG z?+Gk_>@he991j_fs#5{H{r+c2R#^-PJxJmd)>MYDy_@vBhPdv6*7>3z1=HXO*6YiNeAB&!R9$TDyWm5|>gnu&l zX1uu0DS)5NG6-?Q#=>(ffUL~fNKP7!xS^9|p2*6a3+o^<#j}N(sh%_SO!N8EfS(5a z&gZ#3-!l67=KHnx;8~dYzzTVm)q8n7l9ET^rzvwW^MNOD@W4TgOtp*MbG-N-A#;`= zi1^`X>5?@_PfkNgQ6UzucpMQ+o_H8X8}^{Bu?&He+YqSx4FB`uLOlJ-89Bh{Qd=x&uw^htSfr6Q^spV%%su>T1e_ zE%*4JXXDoo-ooW)(R#>(xuXYR&%Vv@wR+KVbO-JlpMb(mtMTiFld$^bM{%;g0;d}* zP~qK(l(a;YRac;~y#=T0_ae}=1^d0Pqj#_G;`?2@%i_r0E{Q<*t`X>dNd&&#EfPJt zN8*!>ucN8SjduSQ>??T#Q>W#izOfdopZJZ0>|PZaDGWi`w4+-DBD+W8-tGMoaeuCepY}UH|^lsfA78g z2VwUyBzThs{q(XE`rTp3b-J-y!d}Q4;6Ew)9}s@YF7WI(!d~~vdUXUY+|CksO1${( zZ~uhIOTGig$PAQv3$W(J7x07V!NTxk_$Pr3uD|ASc~G`%8-}OYgyFZ@v)5qwt%TFN z7fzQ4yItGx(kjjHJBB=lO}ci=+HYQ$8|C?1F*;=sUU};kyt835diF}hCz~o!<|)D_ zWPn{W0H1E$jW?frMxqjAvyiSma0!>`=n{cxci)fA?)|V@hGX#~D`n^5LT?3}ySC$} z*I$bl-+T*S6qn(;pAN$3oAa^Tm50|>J%NbGo`~r3T|`AiVaoLBc)O6u7>|+NzK3Vt{205u`{49c;BTv*K}_Ep@p(}Nc9!qOZr7K1<%JcvwqF`PEjWeZ zt)+;%z7N)JFP4zrymE%KlEGJo{Nf^9clEXC5^*UayI+mxUjNwG4OfL}dfYDT$zP9Q z2H>~3+%Lcmfzi@kJ1{IY70K+xyHtM?%8j*sn#bHOwhf_Uzd+NAulXeA)Q7@7;*6UiAa4c>80~+{_6T zA=4K`Yf&nLIHSv1xRou;>pL=r(bTk1i@e-{N4gQ{OH4oin<#A5_K)E{o41j zCOr#VgF$qU?~Qr$o|+Ok0$Hu;O^|k7H<(Ui%PKk)#uUfrk{!L#;ouG?Dp-%9TQS9?*0d{ zwfF?Qp&~r;yN7V~SHF(e-ul2YAwB?Kkq`L=d5Di6h84@#NHCB*Z$9GVhhzEjjmQri zP!nz|aLfiOcJo)VFa)8RtEfLV})|m@%_|Q=d=tVPHY7Fr(N)yXbq6xIwcO28F z`~f{<`Us%T1Ak}FAl%FYR>B>aIBO{m9Xf^qy)4Z^n$0qh0)tfa9)k^mllbKW^D%1l zU8p>994X1e&^m4s226Mer_0YEHJLt|=7|qMYGUypHACp2eC7M=5g(I)+<+Hzmn{=b zBsyDfn>YcXlXbXv>J#YYNf!Etq>jSi2{T30=-!!+h|jj@Udh;h{2ZoF`yFDUY2G?m zOb^@loq*k^>TvEHFl)+h&@H;3fIgGvEQIe!9eVd3iXN`P_*udvOq{(K$E|}`=RPS4 zELnhGmuUKaqM7Yjbm=`w4!c78IpDsjzZ26_ng$ZQBZxo^KLfPH_vj%$@Cf*|={*jc zeTOl9%5UN6(OZB(`l=KEd14|=Uwky1WVGC|XcK)0y2jsvLnlt)zS*&h9NWY8T*j81{i>%-h-&*8?n$(S@_84eygh<=G4F}rN-8IBu! zkHMy385S;h6Y(+q!Tz-U0q7D#O)F~RsGr5QacSs3`2pnat3dDgVTkEI7`M${C_cY> z_e>JN@8=0)@cQa}eE8nSi0|$ev&jyzNg&|&&PQv-_Zy?P-t4*qojj9}lOMp8>9f%; zW-K~+CL-6%`)Lx~;LCRE-w-(9H`;usY2H2pH@k-6W(jVPh9{qXN1XZR<_E-w*{!o4 zL+IE!#P>=?j~@NS_ax1we;zv#?c>KFC*P0zX3RnBUJk&ohXQ`|?>2ePJOq!biD68S z{$l#r&NBu#xhJ6m1VzP&wSp4!fnirZ&D zitw?sa3}l{?%0VaEj@wcH2Sn-@C)^qNgj7joP?6{I?;$@z?1u?Jt|ta1g1K8Mn^RK zvi(6Uc-r_h<(@eS?LDKAo$tq#=`%!gP0ZU}8u^)8m}1T;zUxd9j?|$;&?zx3vfn*$-yHcdg8R!&y>7u~Y8)?G1dq!FnpBGh z^1>yE?;b6|ny8uFE-o35zwl=P(W&`dUtNJwLzB_A=UA-wmtooBr^H3UFJ?a>8m7H_ z>$>9)JMH4f!RHNO@#19y_-zWF!ZS;rLeK79(Ot~8Ef)x}W6|1mCkBmq2*tbiBd+fN z+}LX}CeB$V8tDCd#%t7h&v2|%Z6M<1<%%jHJf z-lz6hqt+LORdG`rQmjT^4Ve;nvH ze{a*LB7j>YD{QB0lk zDB@!h;q&=qKaOhK@jtQupcm(=sL7Xy`igb<^K*CM*;g0ic-0A><`-(QF^DZNp0FdY?V3sH5#hf)2z;MCq6bh+u<*s|$096r1a9ov2j+c&+7 z*qC;>>+aj|^h?j+boD{h)n?%Q>1{|G;K8x|CBTJ4IDftvRcCm>@uTi^J|25u7}jrj z6P0HRQFkUCrQQ$l-EUokuUzvLY~1t?>gl8I^j7?N>3w+i)rB~5?lh`u_M)!Bj|)e4 z;I5?hIIuSd`$!;Cc4RfbVkwgSCOy2{0PtyoMUdj0j6x@+|ule1n98p5qRXY4IeLwX!sdT zKm#N)Xh#=n_*s9@@DCd?I}^Z@d;`OmP~TOz?NO8h(V)+@{KLux?goJ7ISA_4qX?et=6 z@j+x2GS~+JJqFQWa0><=p;;z<%<-KDZJvo`2<9U{%y<>WaPvw{1eIHx>dk`Hr0CVR*CE#Ojfft_GM9iJL zOyq*TAwT{5NBH)&*W#9YW{Qu-^&f4(4d4AC)~07*TPTF?@rjr>Z@vJ2ufO&J{^RBZ zY|J@|+;FM*O$=q_pl=+5WxR)N{zAA3FwR{fpkrQ`+Kvn$qS289jGnwb8*P4kBUbJ3 zBBN*@)~@;xvE931Q(lP#A1N-_fjh_>ihHv*v? zn%lXuQ9&9*i2FabWH#nQyIfj`duUI7#* z|6-~De$?`#k3s|Z30Um#;YaX~>ozxZ5S`qE(TTwlB=Cg#3!)!CYID&?5f7N1dXE#6 zyZMXXgeQJ54xKn98dr?;Npm^z7v>p(U(9(-G`EHfp{5mm1$nXX=_Tk$zi_c5#pJ4E z?@8ESUX3T0J}r*q>2RJ}L3F;)1NMnJATf0shK-(y>QiShbZ8oWZkbA+E<253Lz8gp z%*PQpT8Cb}lMxf$3+-tN=Sf4??kRA^3_-5nhq+6as86z30*$xe;TPB7t&g{0@zUqu z&(1+?bS&mQ_h)P>I3x$j&M^bgh8lXSy?FnF^>BChh+}#Bw4yV4nlMs(C8MAe=g$ID z9+-n$<~$*QUt(g4`1Jc}JT*+7!qEez7}zTw1pLH@-{4!}={i`zCkd<}5o%4)5k1^< z4(KX@{{6Vu@}Ne*Z*%wzmM&g~*zPgn2d#DdX#8@<{f07*naR1GEiuWB2gBtUG3=-Ut-I}wME9>x7LW}=;YBDy5phQr5> z3oz2wH36OCZ^hwb$8i75doW<~9irhgbjU39jvkBSM-JounbSmLuv7e4WO@U*XVwE4 zbo>1R+zcK%6*1jLAUb9cy2TB|wqPkv)dN$fJ_3)cH@rS-dOnQ~u_MJvf4V<}xyzP{ z<|hFSMwTo+TaUkfuo>NB`(g0pJF$-%MuYD{&u+=^<`>|JWzXSfag)R(y!3D>2J}&r z@HTP7(K>N7Hu#U@{wZ@27c&rfUatTv^eaXO|CDWw$eMpV z&DmFjzI{^BBf1}Mj313R)_C#J`+rA#cNe<286lW}-$wjpB>}%Av~^R%?=EysxDAI6 zA4hV^2pB#dJbWA}sY&8vi&|NYP5rL){z^dvwof2QqN2tza0<;`lhDpJ4qfBN;%Hg9 ztd9=;$Dyda4vQBoMSKsJ`o)VIgP-=iU3?R&558fdX(zzWk|l_b?uKr$gV2`2Px{<0 zny*hST!#4WF&ZJ2>n;8cOq#g_=W5PjSZWdmO}+;uoX;VX;O^cZgC^aJU6u9dn=}(s z?wyaL2M(ccV!Qyj4A?@zf;_${@m6dIlw;{ani5B&Lp0a<2+VtS1>X7iQ!IGq6=ZmK zsYa8Vrj55^qG$jf#lYT)0@MMy=0xlhs*oPp9iopT#EMq(z)8PFMePOrZRIN5lrTmjepi;2VR&*fjEVNrg#`SfamTDD5I$aqp1qRb zat%al&jbPd@^gHcHf1JmPEZZcrxq*{z>k`x+#{(G-p)N1ZQOUEbKGsJVLNOTZjKv= z0plM;amilv>eEMpji{6AMZU_#SpAn7R?ymkqYa17}ZAajaS&yRpd_9IF{X)LW45l@I z=~6^(-0%VZ^w?A!-=Bf{+92vqZ^N^XPsUq+djSVe?8fWQ&BWXXQ*q%`9{R+$5mVdv z#6CD)UXHF^TjTWMblfq)h1Xa7340Ii01(238k#K?A^p?0@zWn%jXrTV!V~kK_y(iv ze!UgE*&9$+QH=9AjMX2%gy93b_B~07KZlehVy4iQE?`SYMSnzF9yyPp}x|G-%lTgPdBeXS!EC-2i%D7 zwrYjtFD^h;)lpQ}(9}18y3^b7*0THY{Hya(e(oeHtM;I-B8UsevM|2SO`-gS zG)iwnz>lDxYWN+LU;zZ22u^&odOebdj}||33{(-w*nywk@I$=#&PT{7EXAv@yp3zF z{w7|3`y-9)8w}!;Rcmm~wLioko_roz;UZ-C8J)E^=FVFzKuK|EUN z{qN!r3l>iaHxa)qb+p%5xbtUw{-1*By!;R5+!@f*>`tiHy+P)*Y$MnIo?`*`5 z;&O>@TOhzDBf9R!gmJgy?z`@XHypr+YuDi$-@Os9zP%P7t=foX&n!b>Fb7_54!R}` z#bfhdLVi{ze)Ro+!=nqILw2YH;k>Qr?dgFZwC{s;d1c7kS%}t_N$>Vu2ay#fe+dBX zMs8r21^%f%040H|mtI|gR$uuZUU}ye$>!<6n;Q5y{3T{p&dbwSAbm6nu=HCZbTp!GvzZm(UB4qo5h>1-U zAedlRAXp@tlI?E%DYoVoBR6~$fB5ZOT=y^6VeQULYzqX%r1!~r^AQYsk)4}^Z-2kF znDjD=GXsvi@$w3^zNx*KY^Mhb1!$%TF{3$?*969BGRuG%{(?djrtiSde()pFkla~R zih}%HB*ezx)_Z3oCqw}7BYf{)zkyfYeGh+Gw*%k$VQZ}4?iGNNrqVf#E*vaIVJHi0 zKKTgWzP=OIZ!JbR=*5~(R^r2qoiiW%>7Z-{v> zfj>TDUDZ;weD_%zei6VgsDK|2snlHH;etTl;PJDtCVem3Q~S=vgT*B5I#GvdQy)Q0 zbRzDW{fKC9-1ET03aWXAVet4VD6g%P2#j62r=q)SII{hPcw*@?bYcJp4-d*THB7@D zvwttXpze9#ccKMFZ41^fJ6n$@m%WVkt}z%oW)4nQoX33+Jc!nw(fGqN?@JJY)Z`R& z=`$LPBD`S9YlvY$j`$&1{^52E7Hi@@_2jJre4`D!054!XjDu7>d((Q0}*PtB+>=A#c z*5Q@F0i#vYx1Z4*5fiRnw_$_79E+Dc2X{;yI>x8s_F3~lEuwp;K7@`vMxaN+1ROqf z4u4$w0$L{w$BMQ2$Sf#9S9c$zg&lX>NM~{`^{s(5GgL@*n_PiYjj+f)U znZFXCs%y{NaG3YArefeNcVcf94~R3-(=`@HkDS2$GiGR{Q_pB*6ohc^uO7s}TkjXs z%t6WbV$7(Sc=-1(3Q*HNAsIUY2e4q_i->jgMvgCtxl5lHXZVa@oE@ZGJdI9qNdm@9 zp7WGw*d_E%Ll0LX{`mYG0{o^V--WmyDaiK~<4;Rp!p(7G(Is&bj+Is7{SQ|Pu*u*L zx6OVW;lt+<*J}czqmz(V5X6&DEfe37@QqBAn!vyaZ1$hP z-#*+ZfdK|joQ~4!dQ6=<2eA@Ols?;%@z+%w@vfNkCQGoEpL>3ZLE|1mSy=^AhBA`o zBn%w;8#0<0s;l zS&MP@957?rELqEKJ;O0@;v5{UsmGEZiEk4z|-pvxML^a(4jIUr_pqGutbj~SS=b@F3Oex zcnI`a+J7zr_zc=X&CgND@)u*?(q}CoFAbCCEW!x_<5PZx=!D-|E21W>Ea)*~rpf@tbdvvJz}zlJ-y4?6Z5gTmu=cwpK?aP>$;r`}@) z$h~vgLW~}Duee4?O-)0`#L-f2`jm&@j_!x{y~bjFpaQ>~x&Wg_-h;}Mr;s)@1*zli z#_T6vkL-7TMq6L}G}`o{PsMTxBIJ(opncD=*cvLw;)PGa9TSh6V#nf^Sxd41&@uGt z(_enX`QvjR=)OI47~1xvPrfr)yl|=f;L@e%NbD}F!(Tt#Bmf*E1>Z5_Nt7Nu0c!oV zO&E`@q0?Bx`R@^9X)=?i1pUUKjq5his6BY>4DOxrD-HY;HxAi>J$U!ywGt_P==iBP zT~RChUfZ6j*b+V|`!xfd{9^W>5IS-W35mlb^7P<|_u*h=J(82|K#v~D7(C%V>^oDB zq#*v*&HJ268kKno$Xoa~g2hQt%4Db^<`0`2~$yCa{==gKL`3*?cpAY%z`4!UG^-Z&Q|V4 zO<5uK2HS_w>QHuW5yAzSasKfAAtY{p36NCjHn& zMPc@Dr{Kbc5H6h0#f%xL=+@;sSoycd#r*f0s3?5*x~s5f&t{aBW#C*@5NFGMs63d3 z4nO;r0BO&?_9R|>;kS5r&KNv0Ya+I+c>~K|dkn9=^dMecIvv+t-3p_}^+8SD5fm10 z$DOwiKyCR}R3F`lj{k9unB>xom%&4>zB&qD`-%n?xmHYi-$q4E5$Y_hzxpaa_5@br z2j99%qVqCBFa7adeN_}_q8oKp6gsv0F)H_GVsc7X9N4=BRkg)<_PK}f+AEJ?)4Esi z+gT$}Uz3NL@*E@&bmQcaBGgqESrK{dLzoGip?PzFpLyO~Znx(=$X)(*bM0R4^MADe zmim`JS1r71;Z=(a{PW1buY!aTAdkQz z0X^d%*1G01F&}QEEyyUnJUq$)^&*pA^)*INL89>ne0b#H-y*6NgEU-&s8-kF_fI|} z(O?Na1cM>0S@m~(^I8T}_^SB&yZY*H;g!FxMS4*watce3I#p!9Q;2 z#^&5YiORbA?;qov-}n};iuxM967^Ml^()`Pi|?++j_^U`(lj*?!o$CP1Xo4162E$h zgOYLG587efHfpg21X%gzwckcm)He~;>N|LB!HdWZ?*e~)kM<clpBz z2E&*zZajW|Q#)+V2_q-G58nKIIrvDRh^wx{q`T%IhagfYgpWR1gX_L~BVK>&6ZriB zOqwuSgK}K;HN*}WhwSiCgaZM*@$&Ol(2K9)>tAVwsne$7`u6>?Cif(=(~EH9k6UBS zjvVYP+>0D)n-PGrw7$r1qt$03v#QxAgQtWM%H4rJp6-Zh^)*CY{jZ3M`WAlL){SjB zyCmwYHylD{ULL;p{hwpr6VHh6MFIT!k3?qZI9`4EU3}x3>qG-@yEh*{_;Cj;fAjAW z5jQg@3pd=*9xLAbyJ+5}=jP%^H+00?E7oD%N)6hvcKc>*_xW%=%Ps#q{M7s`2qK|Z zGTvIT5}&Tzh#&sjzhm8wEaVgwWBkN1R^XCr5Z^CNcu7#|k>CAJ{0XzO5`t=)y?|lD8tlpl3oWfoB?L)u8Rjpd#r)|1nTXtB3 zc3gjhmTz?}T7CUTShuYJG!M@5h2To)i$Be$W+^q`sNu)7N0Q$kk|@2mPZ)vUEqn#r zi^`A}WSa~S@-FbwpkDulUn~XRynkAyIl= z{l+0HT!tcl9+toI0;2m(!1kgV(GV4lz@UPSoMStY9)gjo)k$EF0N+#3y%8y6eEPli zHZ}criy1G+#AE_@h=!j5{9+m9RZWuGc}9xq&|}YjBqnNfTrWWs7QKw_62ySf6Gvh2 zq**vtQ!gf%3|4Wb9+*D$S8#XlE0JC4n`(RDp!gf2*(#m4?=3xpxV|GqQ-S8R$7|}P z4+eT*8`GyesPzfZct+y!=iU(~^sK)vdpF90q}fYk419j99(d`m8*t*_ehluNfbM;Uipk;N zwxUmF(JUGWH*iOrkrwlkmjz@5u9#gH=fEHv-xIB0T;SL6@Q6 zAzyrT1%omQH>k}g7Gj;-(5dTxD~Ij_F?g&rx6?Nk@=<%w|!Q?2GTHp$;)bT zOkZps9^QxIFQ06~J0EXBd-qWFvDot$xMlV-(e@fXh3W7d=h zKv0A~2DGPdPJg+`Ib$|Z7I&APfvf)nw2d7B`bj%}u3q{u^=I5a9X(>``*#RhCyd4K zUsx&iS$9W47>Cc)W601u(X|KtLQWTek?WiRUluHW3Dme{(1xvnGAvm90%Br%iYcuG z(djusMCZY1A2%B95^od0e*U885FL}Cl)H0^o_iZyCmoZR&E9!vP|MsB-eM#tbtFD8?l^C2lRw7$FHT=kLC(vVp<}jdw zYWR%?P1$#ZN+a!a9vD!@)0caZ`U|D@Gfjg9@N@UVd#g5MYr!G)sXBA91^A6em&CCs zI#!4Kr#y^qG{q*sccKpWPkU7S?eU%(fOkEx2*XC*hw}1j3`rd$K|5HEQOJ8l4@CQf zQCMGa5_dhY03*lTk5gqe7?L^`ZR3XFjn!V^5$Vj$4M=2UY8U_h`446O4;(rH&yd@( zJy3>4PrV?KwdtCHk&br+4vCuu@{xaYl%KA~khF2KUq1HSM^gXL!4rs0oQQ5cb=|Qn zWvAjiFf@fhI~Z|!9JcsRV&UQ!;3n|yVsvEknt&f8B##lTVXn)^pL$s|E8E8mN0(k2 z1cvg&{bAAKXVA^nN1`(ioG=F`YwCrDs`K>oIawkQGjjOdGakW?{1SAINt39}ch7nZ zI|_D-d@$e=WklDVMno4n9DfsYtzT+gex7&oU7(8xYRD&y2l*#!OhTtHxY0}KLf0Cu zWZ6pxPndy|XZ5?v^-dlzKg1_-i7+JC8#^&1~5Sa&jRX$I9|FLmDOS3!ag+h|BiFiW~=ez{ON4q!fu>8 zQjBxeM{uGn7iZ37;apWd&Y#XjU0F6zw+Cl0lw!}pT%4~Bq4H!V>MBFPg?%VLo`cFV z9$37{9oZ~VcI(Ols6V?G7tS9*-T6Z}fA%0Q)Rv;A zdLJrlcA@rc5iV2(Ez>1N-pxVXDW3%Hpb7KYs@*tSRf2QX`%zhaSOb4l6rrvnfD1Jt zoUIAqe6=4JYJxal&E!K}bq?xl=vRqgVhQRhgO-Vvuc5|$*t$+y>(cV z101ibKZhBxQ)7hnP1~1~H}2orR?Bt6xozP`3qM+9phX7$5gBN_KVP)}+F)Pgn-KXf z5FDo42rF;4E(1VWK|&(m9W@7bYJoei2@2L$?GeC_zK9|RR9Bh=?Z^r2x3rgbfk8Wh zfdWasV4(m~*@cHCs6<|9znJ;@0)8-xEd9xae1&-IH^0Z%ulX+4?DS&?%}ERCLv9y> zfq(>IU|h!E8397$oK6* zLBU=Gyu0Bq*bUB2Znzi$e;xt>AF_h`kx5#E2gGcdeD?YM$O`R~`Ogm@hyY#mg;*FU zK-iaufS+J6qvCR&4J=)bzB9`(1Li4h=@nhA$oo4|4_}uF#zW^|(g{xa%baQEY zZR1kKpHoNAU~$Ob-PK35o)}Dk=7t^PhoM8UWK+>Tj;45{(Z)l~sUhg<8Hi5qM0AOz98jw!Rm{Pu zQ{7`2Bt*?DIj+ue41hq*EBg>G=6G>qaAW)}7&PfY6z)2J#KeK<5R->~l4-9yA2 z&p7j^=GW&*&M{ohwSXd)DKkL`@`{>w6mHKv~cS?v!vsNV#$2(8fJnOsPqy0?H$B6G7#;5wdQ&d6LEa zze8M#xD=q~e&^T}bvY0>R?0Du4rTDJS--*V!&Qh&n1F6QMo5q%?gs>bb)8Z+Z22;v zpMv^yiNHi)iqU}yhNYoh+)&ZBq%8MvbFb_t`&(Q0aM5UE;0ms{?ykO)xA&x|Nv~&= z@PTteO){>BuC5{I=1vj~RRXV_UCEX?@^G|?9U%aHXIF}7uhG1n+S~jAq-#u)B;M<~ z8;-7Sx+Ul@5s2w0l=G?cJwUWx8Mvl3e>{mBC3|33cYpbTg7QUPGttj8f2iP|-OiID z?jkzJaNp`D-W@=?IN@HzJy5mS@=MAQ+j|5$MJFo_+^4u-yZfPA49&agRzklU+!y#A5gyY_ zICeNXCyYdvcQ@{t_K@6nj2nsc{9@cU^C9`+$T`mf?1^Td{z%4qlbq$gh-;BE^35|E zH^q;~O&*%)C(F2cxO$MX-BzRS7(pWL2?-JTv=>PyA z07*naR2n^0=BPcR!^aOrRDEq2^)&>=@^Rrz9?qTd<6LFX0=Wusp*kPutGzf|Z^k|TN6ZWZ2;$M3GOkmANt_Smq@!(ry_{6X9ChE zX+KvJYSJx`40f9KvloAv;sM|1_W#{&-n`Y#abGE zjM8hRj_za#0Arax<_8t@^9I5K+7Vcyf4Lk1ObDnc@ZlA3rdUieLw-MAfAuv)wfYxC zMO_QGXqX)W&3FkQ5`>|KlC(iTZv>$Nj8seCdxa?S2eACr*R}rD--5gUXyk;CBA)?4 z2o{P)+#WHd%nR;AUZ7aoBmhG|FE_X!1S46_8&)$;(oJ7!)D&c}hjptzz_+e%ja6F< z1u&zQT_C($fgFjTx=#SAoWNcgBYo`U_-JNp-*fErua_4(AUvawLji)qyCtASiElgJ zc=dVwpnX5A_MR4ijwYw7$!BZ#5wKw*PiV$V?K6Tw^w*|-6d8@RM1UrOrRA6 zZy&t$ok;!$c4?H?KnZ+7`dwsy`{6At6;o=x=QFmU+GPGB`u)>;j>lI_lTk*MB`CHJ z1&kU?x-3wV&q$y8b}WWBuone>24b_LBjQOrJH}jx?pU%C4fl8zEyLZVghF^Y|eIKxZ7d7eSL6y-Vd03?8 z6%Q`pMJy8R-E9E(4Y`;srY&P2;4e*zGx58;eJ-`F4558!KAGOk6_>g zZG-I*WFXK(v%t1A4fKq_t+N(HH0?5Tixn7@vW$kwgCP%)jJTSNwlVaXG+GY21QZxC zTzueJV43)BV5iU9-YT?{!#?=AV8vJlhK}l8t2e8 z5*=ekpd(EWg)f%*aBQ-aRZwD-rNN>>GY#%4L37#v4>+N-F#1)hR9euxY#HCNRlt);ZUj1 zLomy;4L+l65}cF3QbIS|Cr^y2rGhflprmFN&86uNYyPt3*qOUWeS#4bj$^r@=n^|r zB68B4od6x{DTt$VbNuw#wR}}JmVcaq&aNQ>BL8_+23D-xfsU?0=pvC^8FWFx0ORvZ zgFRSag#>=M^tGw?dBFqnieM1sTzD8ueyLACni~^nA}EohEZ8 z{mZV&hfKH=M=H)CIc0)qs&$M>mgu(J6ZnG#0l9V( zV1YhKxtDRTp?vf27X|Cc&(3c8S)_I!Wl{G7%6&WTb#Y0eAzO5$7M`Bt&?9;{I>%VS zGz7SB4E7BZIcYdgT;n6utd{`2_+R4~joc=oGXZ2-4|J!%9~uaJbFZ`3CfzK=jgfs! z<)7Ud)`;*`z%}uE#%tX0!bKm6pXM5Za2|jW!NRt2?5&@WU5h+Ky*tV|6 z2pyDfPb%8PsJsXr+}CBjbMHz*d-o6p0EO4|0jv9(j8%S6B7n!|93SNY6g=RZVxZV9@(82O!b*^llSoM+h| zM9v%T_XPjwKlaT}a`Em*8wI#zWcC%G~OlAqYq9yMO{j61YM&{^S_} z#abWgYNPc-c6t3nC}*7#9ZL*TK%qQe5&41HAu z83R^bYBcb$Lm-TYBRwFgHbepFSI?;}!QdKfm+5>pubC8_6M)V-U`0SL9=04w`Y~l1 z9GWbktV)A<*r1`I>9g{NbUJCauP+De`>cMNdvCS{L{%m zbNlP(d|vGnf3d2Ocd0h?`$4cz0KXcqZT@S%^)eTx4SUb#4_dX=d(c$?KOWd@z^_wm ziUpQAKn41kBe)gOLR0XE;0;Y=P2d4)($NGs910;642bD00g5cqs0UKU>8CXP2A#H6r8z3>}vb!a~ z1%aPHD2&@Crb;B=N1uEN8KF}G_$e*go`NxIcB$r{8by`uD#%7aO#3%|kq%z79fEU# zPzbyH>3HMS7jZ+oUij2|T!KoFM*`L{aD!#etDqKvzTH+L_*Y^DXW)1(a8NYWsI8{s zQgDlarju@+H*Hfj+XT3yhL~xmQNN_wIk(0~oOveWvNYlp1XL|K$^?_V59|@3k+LA; zGjn3yn=wk>sDJIpo>%3m)s988g1icfvb@s7K8-e{1zI|QOr6JufS=H2$uX}ZK{7&n zP*Av6q9)VqSKo2oZS6pz-C0MQ1Siyjzx7*x;KLKv4jVK*MkW7A5*%+*r}b>*N|J+DFuK5pd8t z*hxJelqI-@(SqY4oj`(}XbQ|jq#PtI!0(MUAu**}wCEYc#xOVlwZAStuY2>)& zAWZ*yqTNjGJOz9hNtXvPj)OVxsS(3q2NP#6!@+}P=$Alk6>70@+!pT~KKtY#N8WNz zbpU=Gr@XWL`sux`L85cR@soeU1i0b>k8@;bcD5IJ7U78kd8}^$F+Du<;BVep0FWG< z6>Jlb%i@W&8F^>RNkldnNtD2WfIRlxaK3eHW{fPSkx1pa1;{8}3f7Ug&hcF8Kuw?{ zBD>^`gRhZ##^)9R2Ma(XkJUuenhTj1TRv27EkGsGW{QGV0$dRcAdpG(U&_1!E}R$A z!@kWt7@*?vbDi4={Z|MoD8DJI3gX!Ft}?8Ar|c-d2-Hv$i@{B(1=y8ly|DzDw2S~| z1YzAN3h+e07d1&Iu*V4BZQ@k(i-1Wd&rk&v*@no30;ZgMkzM(rfizl);I8Z&H0@U& za-B1%&WhF9Vx~Cnspm!8lTnyEy0|uJ!pt>c#z~-!d?z@hKo_qWc!xRtiRupn1k-hH zr97`0AR#^tcg&e9z9D0Kk3}~Z&FBf3^KT%oTkaE@%-j$lHrFQCc^`rHE+bFIEYz%f zv!{{&X8oD6EW`1e=Q>A5#?jPV_5eev3?G3F5;csf?4!IC$Tce4fzJ<5O1 zBLTnG)XWqO$U&l|C_mE34+08iQf5VNr7tsY%76Y4(K?>;Ivo7=P?PayewX4$$~CnM zWuFi}bL<>*L$L2l2mBl|pxgv%r}5*o&3=6@>q0;OBMf`)FzehvH+NA zh`MklN1TF-zn`-~;5>mgKQ5ffN8K5Megx|ZQBQN)T3!dm)E4Io1!%iaPH-!T`m;p} z&{-OCs_jSZH`bv!a4~R>ZTK}mOU*uN780;yu7Kb9Q#6;QhF?&@K!SxeY=_UZj9inJ zv!qp1-5G*~UNnq2XldnHz?%&K>H&w~9RWT8`660)7axK?7l2hsC+IZuwGywL$vkf| zCk^%?tl8^J0&-UW&OV*jP4;W(z0CeEhsKsNmp7L!d}`rSiwyi9k^%0aWi)qBy!-ADh}lRudB07H$K+|U6F&{7bKKoqsl3;@Jjzy|se z6R=K!Dg{YY%Z-|D)QBROLG3PT9SKNAAjHz>lJa2!9Tun|T7}f$Dim!tHOHhz;X&ku zOF?ZuwrA)N&_{qa1Bw{XMB6mr6Wdo1XOC!&5sdRQ8ZR}(9PMg*+K1h?vrP@8A)uoJ zj4|3;k#PvHQ!L;jwd^={n$=QMjAe2~3sL#$tYZUy%5Mc(S?=QG1PjgBOk0LOkzBO# z2vS-=r2uAzXLio=9JfiPU$)CJ>iij?(+R$5J<_3VD0pe;anc%jX3cqoMgho7AGS7O z!}krJlO}3?D!ukNv=2)L2!=KS+6jlGdFvEA*wVUE4F)}^@&L_4k_}YoLBkw4 z39K;b0ZlZtMAIx)wCzMoOUxk&3NY!x-W-s%9_#Yp!~^yy(YE6ul^~x2F-Gf<2X{Rr zv%b)0=#u`e1FQ2|fGORlqz?&*F-*+Wc%V}a75(vohkbK6H@Sj~&gXj2)PuA+Fq(e# z07yVWOnhT`P)rgtLm5kBP{7b`(I{rUjvrD;kKvOX<}KjLM1fSkYoGFrT4@$v^l+~S zSp~@~9yowHITQ-7^qr2`9H`BhOm6jKflLCzh#xC6|C$@VDG=d&N7@XyXSb#08s94j zp!`sN+q5}p)kC)eFV?jK1Z&u~8CRqF0uT}CAwZ*mBI_Bxo3$sj(lZ&SH)YzgKG-blbf+ep4jSO1+H1cB2x7izXU0Z!cfHr#`DGS^q%|6*!HcWkIuKR#l zgSt+f@1<`;hiC??9CQ5UT_m?O4vjoS_II<#S$mkho~-q40db^P>oPa;$nja_EIQ1% zB4DPBnf^{KjYJoGsiom-`nPFy z(yiB~%omk6=Ba7d{J0|Ev_$aLeU?952nZ-z_^R2;Ht4p4n#)Y&59cnbvOI)}lR;FL z7ozf15o#)Sp`tvDit<8Klog?(d>5)t??Kg>U6NR)>U6QB%CbV7IvPObi9%GD7lZc| zC&QBSy0WZD(1xnlV>&)>|1I@wd1~QB3olw^ z;Gatd8t=gNS0&YbVrCjnUIoIf+Kytd`X zVKQq#zeY5(4b}3q`JiiLmuP$n=qUgrd7@f;)E-rt(tEB6Cw~n6&E=*nX*Yf6hV~-2 zkbob3rtx5|=AJwls7dHhH8qpx=HNweBt=X-X$nQn4<2&We2x(eY5JrfioU0*ovFh% z^c`~rOW1~*UW(?Pi+wttiDsPEVL1V4EIP$J)X|=^Z8^AEfTOmrKn%?_Wo#Y;DpD)U zrpZZ{-5#}>Xm*z@w5q>DJ|o>G*YPMAXOEL(G|9B*90RY-y&S?^YO<<;Bx%+5+lyH= z`!h+$!{;62*f-m;Ol1j}HY!ItP5+-KS2}IEu-kXiWBACvuJpN~&jLV<*)KJfnwn!V z4W{{_ky+Ds8?`cA={RJ13rg)a>3VQ8jo6R=T!nsFNb%01VLCDW7(y=L2{4UV0> zvSh-NY0~<6(xv45a^^B3Ujmdh=8t`^^roT{p$^=a66wK|=y% zh9_pN=)GAp7xPQjqO~6BuaD-%?PQ(N^w*jbp+`Xrn##*OTJoavsrv_?Gj)(P)sXk* zoz^w7kScvie{lpN4R4eV(qiUSfjyN?_DOy*wRfo*uDO@oXpzbf8QgcA$=Z*k4Z9s} zQ`fVRnWp4G=FFng%*Ex%#Fdm`ABIn)MR@AigPWp@?HjqHS#Se>m_4{bPFXO?=KH1g z5w>)t+p;ummAX0Dk)=uK%F4pl%uH;_$ih}hq?dg#C(T*dmX&FxY;5JTj4Vl!{xhvjI7! zRe=)gJ$>ELCmz8s3*^c1vkwDkDeVgK>{hUc08EHHV6NJZ2CQLUwq~E2)Y|>4797D% z1132^8w1)AurZ*G^S!xl$m_=7pA9TI%dxI~uRxXol^Sa%ntmGQ2JlmDLwIi|kT z2BQPD;Hde>zzS~We8;5E8s_x3WX=e_le_wvm7+p%4 znk{;oAEOy3eQ~jTbA31coX+nJ51Q}8)NM$o-Ph$jx7#rNG1orLyDP~r&3Bb;wrBca zZrXP~yVNzy8=gALnRYJDIR{sgY?>}^>mPsTq`A54%k8%jzZ7uFK}% zJIj7i*V3o>fV@1uI@d7+1liw%=J;a9)A)VE=l1^FxSZZ|F4#9Ux;wkMf1COy*K%Bk z_FNnJZJe7r|Bboh+oW>Zc-+m?Mr~Nph-L7l$X+)DQJb<0u~||Wyxx>ugbi7R*qBq) zAZ^Gl1k?KLu%@ga)@R!3(+t1dZ)Dv}wo?qb=6zP7w8?fiWEEj!cCp-VVjo%BFLP)4 z%~?v<#;gz;rb2AW3`dgvE-2;JXR>@DHfDq(X+uT`b_!!d29rKppBV;|d8TyO?@c@F z(}R&Zrfy5_obb2UtcX#4r2=A9x4%0 z)qGZgE+-g5;K2YLCO0Okd?#j>mIjl6VxnbdX;@joDEM4}iil6Tra>C^OMry{jR9IT zcLGofk{Iwt0Tm0NVZ8|G;)sN50e;lJbB;}cAk|J(4MnzXgD|R*Wr1EY76tvZjM8e; zXFxyYm)(!;e~-TSY`UA`f%dI@Hei~`FP?{sK_P;fPQcV2%f)5w{sqXi#%IQ60ICS^ zM{T%9ehk@{f`0n$QhgZNapspQukXx!Xl{VJ=IV*oVI%EPmNoU9ymwxk_E=8*?HR2; z2RLlXn7pxuW3I@a10;>ep&5f}^$9;Cntt{in07BGm$hX1VWj^>`tg&%JK;lO4xbY` zh^*iNNemv66FP+K;6cfG&G%w2=?2sZ{Djrgr?@jNu#qb=kuno z4b5hZPP&}e=Do?AduN`Tawa#=nd>v1XTxv%IpxKaxpZ!5cjj8x@c&}jw%%XVhtg;2 zHSE{fz8MGGp&T_W{|)PF*~T)`@LuV8cd~rFxGyMv^)>yeF?QtJKF4w5cn>`;!jtxKLTUdCg!FS4T!`a@Kb8YsW z2KhyE=lnLBv5+1H!PB*1_{DD_pGR%ZDiZK)b9NY8a|ot|1OPO^*TuO4c=|kqO$`7! z3$)8DglRWYUqCvxukZMtX-ifZTeAzXCF|lu(9ZyRo3aAfloeE3GsD=N!NmRv^aX_u z(!Mc6%ka=euxPHqfj&L_SZ(YKU_*L9+Ge{1|E?4OG{B)b44V7K zxx6=rujcXw0HztkmF1>i1D<{sU8ely=BB+bDmVRpQSG&~b7k_YP|b*)T6YdW%mVG4T7$-Co3w9&NQn8W0m&TT zi}hS-wX_5U)Kma1qUlFK$V!oC1UE%nQ6mT|kf;GlcuoE`H%0|d8H_@KGzK+cbX{AM z&K{#_!{#A@Ed?+2oziN`N$woCxt2N#+}dq9`Ko}G`t#HNInT-)LytwT)juQrdM#7x zNWUUWI`+#0{Tc&hh9;YTHoXnoWuFb-=^PsnmhY(9X?SCoRsI>USD0agV3+ZTzfEfIx-`;i-< z1}~!-Gnk8^vtce;z9H6Sl;wu+8rCsuTFc}Lps&EA(Fzq80TNM~t^!nhxIr66d(`C3 z^hcUVv(mi}d2~G>0crLjPueF9q=#TD`#Ru=+Pz;QdrMs_!n4pKed!no)Jk;hCh6D* zM(x%gL+I<%jKlQJ+~$)8zfoEU!s zO%vO;=T+y{wD;LLY33R?V{o?H#I^OqPNck{jd`REdp~Wm{|0reJ`CR*@hjJAQ|FI! z(}*Ukt;mlSY)53qjK}1ro;7EN1}SUmm@>`fW0d>>C>p=<~?O(e&MJlBc9~&Zl(8Y#54~^w4G8axhS?!yB z(BvZbj_*aTe9OoW`}^T6+tirNx@nv@RrX4snKmz~XXeJp4aaA#2hy%QvA;vP0e&a; zA)h~71@|LrGr=>0aTx(@&J1F+0nG^HWdtn%&0H%0X22Kop1Io2xjO&=AOJ~3K~!q{ z87+a30CQT70G!s_lo7_p^pMpb`wU6HTi7qJ(}R&jI#`c@qJWASer(F{+jY}J*tnA* zTmTy~NjuwOvd5tGDd=YdbOzkh!<*8m&(^2&nFI85Y6+(MBY+_fDvYv?j#X~{pEcP%_>;ZchWw8%hX88ASf$rbQZZFd5G^baGB_l*;0 z_b@RL;2#&>AfxXx>MlVkUKgXlN)k*Xq!CE-B}yy(;VIZ*04xPb2!JWj<)hi8RYuKt z38rWtcD`GJYS{O7yV|}fZviU;xER_P;deI(E-_~thNcMc#ULV9%9DU1Y=`A`!5b<@ zL3np0`3mX7twip<;a&P%c%fhq!6Isf324WD*(b+pB}0phS^6w$z(2!J@>j>J;F6(L za|7CG@COEfXpkcO)4JLxd7ykOaJ)CXFnp9TN8UTzB|jbQS!0v7t>@B41cYozxA4;n zR$<0i5ZEO&nRdGDcr)6z=*{*4m?+if4YbX7Y{)PAYE+U^2 zb=s6y+0x)MW)2jrjf~UC8S4eOMwA`{5{mYnfN=KMgto|-c4=f^&aG9qA+4q!+ts9F z)UlGsOuj;uUAY%Jw4ULug2d{l&;|?*$k~|BrcVQ$HUw~$@1_iMT|%&kR$8SmjV?l{weFgl!dAa$)#HfR}PZ0NC(Q+qY#| z$M4t|gpbO9<*T8|F{iRmI_{0es$&g1*PH3v`Hb}H_(_|}Dt~}bpxT~)Ek_#6oT_}< z_hw$rdn3mU$7pD9=FauZ=ad8EfEp7dhEAhJEnZ&ob@dp$=^_8jZ(V@A~c3wQsI<-)mIfDf5?e9ihi=#~zo?huL?Pf7XwlRz16{8ISIV0{9VV z+nf;)@Qr|(f>H{m={2?FRLhNE6O(zqF;fAz4Vgh`(!-U4Gi=Llhk%>_M(G9|3R-=# zZ(ci+XzdwVw4Dv<2Jo}MK8p^a#{vL1v26l}G7bfqY_QKh_!;oXfRG#0334?A`^+K9 z9!n4#8-Y~h1K*pn2Andv{oLjuH4Y76=%h{SILCbPwb3-R`D*xU=SI_!?U6P!R@OCR zQP5D$lubE1*M9B!F!L3{hMjz8gLiUG-dlVpNa%b|;8J8Ioq#RDSnIu*aO(k64xLU- z%uEGT$vZ=Pq+a8`Tkae1+sXTu>qc_=Pb4d5-)YJ1J->w)m%|I1^cIP8{&p@#f}@7t z2r!x(BXLp#!W$@(#Av&kis1_sN%Y);U=e(wVn^ZunTIGo(`#q0Kp_Dyrb76vT3gP>?aT=-NjliVC5D#5@|OAV(PdI zU2IR2((DZtf=S10k59){jDkRsB-S_eY`&TD#X`5zPI~NQ>YF--z6h8TR=z3!ick>J zeuQ_@r!{uY59dJUi69%xDNt+XhY^tTgJI=oQ2D}sIX3c`z$g1P^&01FN9f{Qg{`qM zCn##ozsU_hIcA^Lru1deD0DjLV;NJ<*)L_$$dlob=E_6UpOdG2?`*?)&$dL?*f-m@ z=8t9coyxq{H@q)IL0EZ2d6Z6&7n9K&t_><#Y>KX~8N9P_y;BfK#4VU5?&Z*tOT z=%oCZy5_mLH?*7F>;s0Mvfib>y{=hTWFazE<)0aglUD7=rrq@K^%n|#h7RV!6PX`- zjuq_JJ%($*S>C=@IgoYIXno{!uWa;8>owh$^2(NJmUWJc*W|H`gL{?*^i$@foF%s+ z!(pyjl})xQ^%~KleB}OR%O0;~tvlzwdQck1^bvR}&Hsqo&t63*Gf_@h^<*6Y|ijoG}?e(1kg%% znE4Xe;S9J>#)B7wq}N~g`i!!r6p(qQpWVyfKG;9(R#B0K7xNVJGKF~ zhCrF&k;x5!#(Oz@TH0?mSlY0@DR1|!JW`&LUIL(+Hf8uC*PBJ-k>DEn#yL>FE8wQr z0>oue%Z}QaVXV(8!TRj|SeLaAYcuy+X&=^Q9l+Ym16Z3?inUpXu{z@jKFK_a)mg`| zI_o$-$vlCznTN44a~HNa`Z4p!u~H5;rTeg9dmi5W%irJPs@+U{_+Kw z{98K&hfF1p}#RLSJIzpx2Za&eUmpwXXE#c@16B7_0CzYxog({y#GvHlRMjK z>Y4L>Q`fA+eoBHOSWB(%l zvAyR0UFsd{7+OB7oU^QI9gtV(5?f>4U{0mTPy(JmaHOtx+93S+XDD-Z5B$)>aZmvAbvLf`te4* z{lNx2w_+vUTbrroqnQNMR1wq+NJw%Ym~L9E*z!24@6 zaOhMWwq#oEuH1+}f3N{BEMF<_$OqDH_+gJjX_s+W{4?+P%mAe(ryoM~MQFyX0IZrj zI{?`XUUR+*uxV$$Bt})-xFb)ZEApD}HmCa{i98WNlV-Y>*|W@ZHUpdlu`Y8LUU)Ab zPrnz$Gw+A7>^-JJEP6MLxo-yX*sDG~_)H$AFUi53Pwd3x$F^bgZ#H7YuQniU&L#|> z@i(l^I%I>p?9Vc-7SJ-?kA0>1jXT0i1ul)eeDS%?gPF6IyoDeCTV#NHQ43Jl!h_E%15OP;)$UUb zzs~L?l%1}{=`$5LeWns;Dywk1qC!&nnKO}V^GwUJtfq>}N-%xd|9F_HMJ>+aEb1E^ zcfahO|Lfc5SX=(JaG-?)|GPQxKXn~n30+rG_KTjaOZClnmwNw2(Nc$c)Sy<zItCpN`ytqAkI#&gpraofGyT|*>4)-;K}j?zZ3=J^++*65 zPVGhoX5V^mHQxT)s{ha4dw^F_ZSVhqYxiO&p#Ck_pWmy3ph!t06usW7NR!@sFE$YM z+UN?1g>n^XB81R;ha@Bc(nAZqx6n!N$-MvPUHj~F=A4`qnxf9|3^TLq+H3EbWPh^O z-k7&|6#^Enz?=okYQ8@ZIF;82kMcjQVCWMt$=?j2QbP!t`b*w3d^8 zCZtWkrs%`?Y3Xh(j!4Ji$aE~&nudVQ$@nEG2~*Zy#-vqA7{BBSzM6j-U(C6P|IWII z!BfxUy(#DM-p`ltdY}1N7jsVJr|h)E)BQ-RKLW?q8#W^+AIJyfO5m5B4`k&U$jm9g z<i(|f`ZA6L!jtnm<&PibV18d1g z4sr!$rW@(VM5ah>PjX#1Y+K>>6;w!k@?;EfB&rk~8yr6z`%J@enP#LQ(>69aR%+yb*15uV4MG z(sp$2YAPy@B^LkwyZfH&8r{{zFs;jfwWu$;a4&aJxvg+Z1C|CX4cvSUc!Wj8W5Mzb zsNbj~R)y|H5CIKgTj)--Y}*%0Hb!An+yU8?LtIm1_Q`#~ieR+v-XCF6aR`r&SMVz4 z0J^{XIr@G$TmV~e%mHkQJ%XUPBLWGPg&#qR^XY(MDqLCjDeE)BFoLi{XHu~_{tSlw z9FCE54q@<5`_X>HcDz0~7B9A*iwTR4V(9ma#f5k2x)9KD=xlN}V)mZKn8|Z&yZGvQ zHT2ty!>3v9X$=2445R0&a@!4$!|Q`%@M5a~e7EojhJL?D+=G{`4VE9(iC3}^nQ#K5 zC(d$!Pg|RmYqSC&8Z3rH?p79wHejb*lO2#mgFaq0IMAR^UkUtFz61m;UX7R?`w+7| z0kJz0Wv6ovIfJ)P7>=raY>f7@%AEA>LYkO zh^dPsF?Gpy{Jc08KQE5Mq`+u=^UGEY`zaJ3e; zyc};2T8{tpn2Zgvr%b?-ZntcUa=#6Y+KpL(i$r&H>)jN-6XDVO5w(L0a2`W?mVraZ z&Y@MuKIl7Wn1H?yz8H^pKKWXJYM5AtCLlD%4E%Io7InQ|3qY3Z=nw1O(#0QT3|PR= zvhKT9>ptc)1HY<%Ed}~j^=*NdecnP+YBti-(vX&#DyJ*dOG{4!&qCqMV>@s5uQi-} z)#>PiOU6Z*p$R8poQ!hX?Tj`GWFkv2;tmEej1yhgctSu0Q!?DFA9VBdOH&>T*hZCBC>~T!dfe>ZskR25NWWwK|fGOT|JL0*pFs zyd|w)hZ;yWl4L?(+IovhN06W(T(%P^;#jENwK}|dS4OSgmEhg03Vgf1f@I^00GFc5 zD=M60(x-DZ`1F1W-d(EUtZ^D%y{gE`F~ifVXH_|Ux>iT+j^0Q$Qf&QDG`udVzCTZR z$zz{wEE#ye7jVG zS5Gx|Ii|1o;Ga^|_nPmEMVb;E`f&79O()i+-N z`p(Qoi|?{f|D(g0v^otRj0!|>%wf!1z7BB-`>||Y5EiTn$Dl9A3-HqucCoG#*WSI@ zb1Vr#@s}`ZDakJfpFs`*1sjduHaa?~-0Wu9kH8`}FN2^3}ZDpQam>swn z+js55wy1c7gl)ry;83hyAA}XF*JH_wHCV8C1?DbTieKj~!SJy^AS8m8oP=P?0#jU? zwe?{1e#~7N0si>pk7=I#Rh}p9F;W2}8$gonykY>C4d8+S>^N6|8Vv?uQ_K;&b*KV; zgmdg~0^|gI_v18V7C@{i)AE4STmz}u29nbZq^ITM=%F)c_I3w^Z`*^gDBI4_vf+rb zkfa{fI)$YHPpaPQv--1upMGDgYiY*Zt&_$Aeirb%QNT}JdcB*8g`X?%%dmkzSGZTw z^UeG%h!!fee)DhVEueOlUQMOkZ zNCV*%)2PWH^C<|IWL!esj^2`w)-W2EFej~G>a_O~z{CkE!&oQFNH#8`PJ8A{FA?f6 zPlk~yWnD1N;*4<;#|w_YuWJoE@M9W+1x`#Uzp9H;@*msJ5aLh9IV@Zl#ysgps#t)j z)Le^-X|)BAvWjCqo=i{q>vr&#HnQI04{F7sFM<++gLB4d)Nki!Zv)FvZOE3^IILti zJy*6yx@~I0uC?vHc51_~a}As{j!In@j70c#AV|9`^$-@YJl09==t(}?%`_K`Gw|v5 zGAitJN|97;z@cPy7>>N;p;=ilC2APVzqctCynb*Y9U1^ZERz&~-Y zV2{*Gh`@f(dZApbPu0n*w!b36ruCw-93z}QZX8Cf0nfv$_lr1EZ~zAi5>TuE3pj0@ zz$xPdyxysVeFX_PT5uR=j8h^P$G&U-t8t{pE#veU;8l${XDdxLo|FX{)0Fm5`IOZe z%hd<*ifSoSsPZ^2_{26Ve6RbD$cAH-DOncv!m;Emhb~j;Ot+0!eR51NpXxs?3$JQy zT#>$_ZfOZu)S>Fh70Wo^?l0qnaS(~dG1TecW$#zY#xmI+NBz2;Y`b~ftM3lS4Q1E= zbiWrRhYn|YfD1W=!AJgp!||!{9Y;*Mks)nieSC-1SkII1BER$O zH_BulpKK$?8NU&}9ctW=KMY)}-mH9<1}qI&8YoT!9)w7tkqHP1--(v(df>f}KgB;P zSH;uM`{1eP{qbZaUjcZ8c|7|J8j9}keugJ2d*g}9K6tunJ^VCh6}}ib0i(zLPu7tO zir#~d$4o_!A!88~dlQsFm&xzuB3D)C4AFB!EEWLto*pH9bJX@BX<7f zuNR!fH|ulo?(A%|{vivm4a-86?lJgg@g?;8d?tdT4&j%1%j8#vECV=kDF?j=4R-;4 z1djV;$KCA*E@E}Wd5oEV7US3D;Jsh7(RM->UK^f?sy$*bX5o4C`D_M)qYq=&yk+t$ z75`p;{8A=*4<2R*ej55|Kxc=1QM<5u(-w^V;YWNn>??dVZUV;t@FTwdW+Fz8`5r?@ zkH^rjz85Ga>-E_;sc;XR!eZ&xyA$D&+c7I(p;-3?g>J)|4Ix;$ZWETRT8D+pRv~cV zGRzKKg5h6(htSBK;#SM``w0Bl546A&OHNsDZw~^NZ4^5M{_4tKH~1?Te|_YyZkfgv z^c~g&8OCv$PGlNDdI6AX0Bg1$f^UP)(#PTsu3fU7%HNG|CQSiBU(nXw2#%l(hY=Nj z1p5Yl;8y<&xtt;jMi9n+WR5{nKJFylkR%$ z{czSeDFEc8aomNl762~-k+xZ9xBvPo`&R1DT^{I>##u0&ZnV>sux}-a4 z9ENwF$~b47lC(qxoPgGC&wQ#rmR+~Kmq0s#hwXd>LJ=4cz|`yDC)56-+Rlk5U6kv# z^%W3B5JW&n*q|n*gm4*1Gp^vWaSoS_^OBx*5pwd%{Ism2#T%`Bs4Id3>g$qmK_>Q0 zr{NSQy`1P@GR{e!G$UEcA;hGG+zA4uUX_q&oD}OIPSA@w2r>w}nD2POA$a$%EMS0e zf!1h_Nq4S5)yu|3yTuu&>g)#sI2C?HbkFjX<)j0)xzLEuLVH@YJd|0rmtWMw~5 zmu!Ekam8-gqks~DLmFy#p#{`Afh)Rv>YVu~5A{M|$2<%t=twn^rA+FO{Y2dpP$Uwx z^{y;qjnG+H3=u^TEoDf*(5i|+h8B119|b>h1XL(Fm0P9i(_|@^V3IIXgG|bB);J~O z*{4@k8EeN14hTeK-xFTYVry@~E`fm@gZ}NkM6YavdXLyXWxb`^z;VQX>9#hdO4}(D zEyEO!qh68(eq1uni;mcL#U1Q>zBAM($J<5Y43dqD(pT(R_F0NepDdei7f+S`9L^i( z1z7Q!P+Yl1D}A%PbH-UDiA9|QBIz!}po(q+Z#ld{0;=$BRDLMut&e1gKtp zDoc7D?|Nlkjsd2x)sKK|AI=-6iyDa@pn8Y#1altlnv@*Zy_av!{@=@Qk zUS`|)4pFAkKtIY;yNegj8fT=f8W>W?9B=%-aBNT~>U~izz0_aL_Z9FvRIpE3hW6pN z>loP2Dj&Zgmt?F{e(LyI5005SZN25&;@zhzjvGg8Kw9+=zt;?-{;6Ajqm?%EP^YaA z5{*;xZ8|C6Yah`m^;*aF-L2K@MbSZ>HYy)Sl;b=9B2E~`_K?sPHc&a#QM!)sP$@d zEZ(pKLD5{Vk8AtU;%>hJJFy25uxt}rb$bt+xVvlg5dns++jYnC4O=kr*QI!~-9W65 zIWE?H1b&==D1c*I%g?+?g>6$01^gVK#sU5IBQoI}mV_nZ!dWZ{J%dlD zAH|T_S?D%74UI-#gzunJc>3)X7&?pPPDF6@5loxCNH&_dl9_|Or_ZBZzYh=`!!`I^ zH{2kUqh%jY0>3a!`gtb4pF9OKX3fPfv*%&@F9G;z>Mxl1<1|e8aYm_tpIAZ05PaL9JoNvnCI*c1MUHVsehJDc$VX~+0oI4_K#$Ur!QwBIorUMOaoVF*_UeI@Q(cm30#EG8NXug(2@B5{f`i`Xf>j? zZO0ay1tQ^HA&W=L-PR7u^|-dKh2hs7$$Z=E&Mvv6i}e+;e=MKRC-rN&4K?!t^Sp8-Z*D>V+_;Bbf7g?*(`bs3hfZg3U?yMOwZLs06Qfl_iz31Y@o&TYWF0#(?Jp zuCNYHq4RUaVy{UpXOVnFJEAw)Gcq$F{Ov1VVXlhXYl_QY z?G7~rMiN{wPrWvNpvxUC>S*CZt2%{E2qqzhYES?GAOJ~3K~()jU$oMpYznVs3+VFg zQBAIOI(dnOBY~2D#|}Qq{j^^t_;js?I&BC@iw19mNCaB6kn!nP1z!CtOFn`E4JtTc zR~DGwpc`%7&OQRpIQgeW3Eo&2Ar04Ne7$8o(HSvYQHKUD{vEspRZ(A*g(vIfgx;@9 zO?Y>GN!&9DEX)&j)=Yh!H_qC_8Bd)OB>Hx*hT2_g2$W&C>W3uM?c@Xh&R+2E=!JUi z`~<|+>EsQ+?ypGOz52X}`kiY_51QrYx^TKa)=djUT_1JFI9i%={7JvI^Mh~KYVhe^ z6@Fb_5&ddl#WCgGqbg1rN8#1IiU3-_PRas_Y3O<>7Jz*!OBozLYK=espN6^}{7}0S z+fd5}T2t_9n|cC92!#j;IgZ&T_6>6pH2QR`E|y_3E;@J>3$od7gj!oo~c6kNfJzkR2w@VE$ z4|U3(r>yK7zpmBA-J4K5%}A1QrL5J?DqXkt7EsOc$A5&={_VVE91@s%_o^zEU`35T zj$FNkUt3YrHHu4dDDF@S6nD2`EiT0Z!Cf0%gG-CHK+vMa-QBggySryZ#LA z>l8q|K@Ds`_#A8_Q_!H&sGo3PPH6jglq{fU)^67>c<&89(UW#_{D@H0IebSFk~gfr z`s^AZX!bTJjwGa#{#Yoex@&!J}ml0k98|3=hjSm1ke7vc*y-Qf0kJT zJemPf5OMKrPtABoh*gHq=OM3F1u;>>CVr}llh{-z*Usunv7sRjOAhaj7om*u6Slb^ zRjsXA^0u{?k5W7znXAO7X}-tBiENy`nz6(u*FC^B$E*dNNud1Y4r(S9C_00~6z8tU2Nd$aGm0yv=q+tmn5Br%H+DT(*G4vPP zj{UX;h8BAH{793XPQVLorC0A+iSgv<`$&k{=Fh}rg2qAY=lm!9LIWQRy}Twk%<`rI zw-cm?LuD~{vxk#Vw8)6QsYE-j%<@)@q+v@+*U|bGtY!YzES3rc{~>3U;96uB{^S6o z59~X#CY=%fCL0ki#+`-22Lfjdm=rpQ=f3y++|lpM_pIoJ=&8vYPCot;SRT5=-LP8z zD|;N+DZBE^?A92t;cTEJfQ#Rbv6<#U=V`$igs?{^#*fczr%exD5k$H||lJ!6UJ+I6B@LQV?K7 zxGYooQ+H-!D}r&DMs(-hk9f_wVP8=pX6O+RnGdx+0PysMB9P1^dWw&hF`6^EQi->b4LC*I6QcaeCmA%EYI8J;vjFWWss0--NXD zAhV~CJm}KHcjUKhZ*A_OwrRo$2_p`?>mm=pre2p7jSlGY-5-)X(Pzp7XYVx6roExDVGWb;;=(@?^hxsU3u!~DZjrwWjQO5?viT6e+gp`Wd~B;MCKK%DD1qp+ZZ3Tvb9+ZNxq09f3GAr)+W~bsF6bjcLbg zGg{N#dcMKy2<28@-Us>o)Zu;Br8k_KtuB;>@r zlx-PlX)-lN+q1TO85tRdaIsyhhw!jBW)x(ENEDH{o?uhM>1G*Fy~Khwh`S-rSos(Gt+D=BXfcae4V>B zTCIN@1Fm&7IsEuPw`~D<@Jf0Q3hd`dCx_PQMu}ABG1L;q{d>-OD8!$UBM>W{|2Q}x zl3hOp$yJ&GiPm6iN55t7JHwarp)L16wM$o_+SaCQvkk>?A;PD=7C#m5q~=ZZj*FcL zQ%0r{Uh216>|??la$>#2!jDZ7%&8qM}I zyJlOE)}G|Z5<|}vF{)H%H2MUS9*+{@V)KW{2WG4LnmA!E&oJ&ORbdS|NYjL2gO#LE^a+F!M< zNBlBb9Stp-8N&Z*7U{UNwrYsaByZQUjf{$HnXhcW0~%mL=?c|O>3u{e`i zIi^&zpAU}ET!fY;!sZ|>&D1~};`z&((6bB0EzQapHLWDLz0ltBy-zyLnBBXRK8#$d(JEPe`QZo|Y z&*g>6MEw}wY);}fdK>K5S}`}FR4t^5146Doxd73Z5x2H7a@Yv z?!w=aYKq;9iLm3QXirh7DoJC0o{&d7fdQFR$ZkOi!R&3-?}8kdk)PW3T4WT*w~=4- z=p@HJSpCCii3zNw4rMO(w$rABZoGR^Nv5&d1fYM#eYw}IpZbc2U4!N^6l5^rtvc6s zGI|cPxi>5U#nE%7ZYS;jFN9fOdEye`p(sDNF~1ffG4<(8tJpqZcTM_KGFwWBKY8Pa zMQ75fMY6XY)8A}FznZLgCN^77l;eO?Ndv4hE55(z6ur;z?vmHL1Fd!_ta-Nn*7NOB*i|woP518Dm#*VS=1Jk*xml3B zA3^V)pANp*ec?BrNba+q^x?HEiy$K2iC%6nfo6wVKl~}t^)MFWKUw;|(Rg|TG}j{@ z#)(~wtoft2P|1KY98AyYBU_jt;x5P8p596oJiz)fQkzb%p>2jN6rm95;j|X9BrSN@a*{uqSZV?WpBn#Bhp0C*K4gq6So;@EEUrc z>eOTEs8$bq-_!&o@uAF+R0d=s!=4p7Rm-5V7{%tG>}{ku!Yoj#iXGo6*}%6Cv{49L zdexRmfP__(s{TJ!?tEp-lkx@vLVEA*w<`iup69?*~g%rM$fODgD zO$S7u?f7-{fc_>k+|C-K-`;m&X5%q#vL?m*@{pK#TafmK?$07+5&>&_=@vj@Oaj*g z?H3s~yay)tFnJ7;?hmK|f*f!mZZ%pXxkKz^4^`%<8(fHbLDT?TVM(It`(*co)^!0Knvj?=3&@zvI(QJSghD7uv~T=M9)X&8M;k=3QBQ zda2<^ft=DOYzn)!T0OnFn-u#=l^?whkMhz)9@A-J=e|?WpWwuxAEh_%w8#w(@U8M*&~M~Y zAi#w242W07R~_?dMW;73!fSk2!CTs=D2k&5`5&7@CKDtEp$1FKm{LH)^27#S%)v6sGh zfdtaQ9OQYZAxSvb7kt{*`!mHXbX(frgJbezL))79ANWMd_2V4=(?+Jac;PI*yk~mk zZ0e2@*nKCIya+?fsNJsy2aEF1R)rKQSQ%h+_&BFvW79d+d0qe?JGvsq?~U_*)F28R z9KIbnxxSe~T)bJNz<2pK($KTmS=miHCoz+ltl>C1nhTZFuTPuzmBM0!O|%U7mJNEHmiC}7G5ub3AnY3f2t2EC^57=Sozp<)CXbVd4sClUL1I@3G!vdG^FhY|>%^6!;)7AXAAaGsAg#HV z=<2SS@KEwVAoH6nDI{Gt7 zzapA$8810G~-X0&;U>PZ&}oeXDSD^l2E! z?AJT==Q*mx5`N9Oxk{M(t2+;M^5X`~o@Nc1rBstj%9mpeu8*SWj`cLSVA=TClol6C z$6z!kPMyYZ;}gvtZm?ba+=#gafa`0(82&B#8B>bi2Ni~y9T9pmey6+H+htXWjojJ} zH}ea|g*Tf@eIoucivvxY#k*frsSlM5^xpH1sAwqXVByrz7K5N|F_BB*EEjcF{*1TK zz%huAyNI)~^(bTCh;goD3mLEdlEVj4Ezo%v#aW>3UmhyAk;G~oOk_$X%3W%RN_|Lv zgHzTnh_Z0*T97w#g7{3JbKp3OoPigHSIt8?uA@I&Ws~ycF~(kS>^V1%WH`g2Z;sA=bn$j&Ak1s(>1qk4mbHx#B4^+C<}*Y=WGCJp>JYz>pjz?VIZ6FECOEgu!-n|-f;(O2&aXi~TCFYJ@sV9#I9aCBVGB-|UPVA8MTu^Zq#Q*7u2 z*^Ed~*UmDyRp2P(LO=mIZ4^IE-H!{I`VdB(L_!vTfo6uh8 z_?Wi4lVkK(p6e}go&9mm6V_!-CXKQ zVSc={t(ZtR>g)`q5eK^o^uGk3d!kzfcm2rM z)l`&es(1cABJ)OUtD7GA2Cr2}s$94JGkizlu{0NW>unV5LEl$kQ(o!WrrBV?UjkjU z;}@swihmsxsIG&>-`hQo1Vg*cC%BXCHP+ss!=x@T@J7E)(661V@OJN%z8>7oMf~8!*o}oJ5*`>i>!YclyPpWv_2ccw&PT*J|3ic~Q&YHt=D6)P zS44a6XHn|P`o@nCU6G6Jhu|UPSBEbkG`X^TPItnr69Me+VC>P>c>@Tbp1^vogIz~TEcvG3*MK%j6blb&m9yYrT! zVyE}1A9BFgt1|w&jq*I(O~141KQ}JF#G1)m0E$e#HhyjHb$#Ezea&*zmq=l)Q40F# z*toe|)Z9tX;aIWYfsoHfGfzn$702BN<>ef5m4J5FPv*rDIfp2Wyx975&ndy9D-Hbm*wk;_hF!-%y*zf=!?$(h z`sHb3eNJ97Vh&>)9r6mC0+0=r9wdDV>4Y#7FtB)$_2kQAAvHYkw9!jic@k%n>MS(69~+o&10A^hQ#(YYaQkVKW%%vI%$j^Z|85?* zh@~Gi`ndz!@FG6i4DIi9o*(#E$?zU&h}J6$-#1n5{s=(GM|-p`^XY4Z^M~KGdcBgS zj?!uCz#~M*4Ep!)oX3Y-2O|lrf`#snU8-(hv2WdPa63Ur=Sd4WsFH%mciO@kcJ1oX_@Z5s@F3=v6l>zW($rt0B=2{k z)BXJA-ce2d%-of&$GiT&Fm+x8AImm*soUtwW>~0At*TQV{4%QdjRsV+JJ#xRd7u3G z6{p7CSw!J2t5AEN>a=2Sk?$O$zYoj}lITUA746#%SyHODUlAb)FbuM{o31jp^a%6+ zQj6Erq1ml`Ca^XS2D<{q($a&u2ntLFD=!bY4tRLa<}jdudiiJpz;;Ti(^dwfVLwO7 z$4V0`-nkQCa3Z5s&a3;6uA8;^-05O~G6=Mw6vymX?uw&T-e9FseF6PlbZ6e{ad7!u zU_DdEcvNp64sfdPui<+Mc2;eVYElfwt6?B32-2qa8SpN3xWK=`w2%3l=u#`L)$8s$1`b&0m_@||% zg!FE*InGTfSJBCw!V-zKGXg|_#INvF5A-60v6MP#ff4dY^_V1b?t|HI{tWWoWW9Xy z|4Vur8kf6W1zIJnBHyh zrafk-XM<<1CZT#+!(@nK{~+pBlty)MD~|}4;e{?a3(T+x*yr5)JZtO$1Q_FyKGuGr z-^eO?$C!eInx0kc+00uIflRU*_0399YL-BA&%_YnZ5S6$7tyM&O``NMO|zhLwMrx> zg2Xx%MDx*k8EYtr;P!!Xy^3RtgYrYlrGZDc*3IrI$?2|;HK zuGUN{-&P+WNWkCt@JZoaegHMlw5gw>>}BM^oVtb;U{dsu&_X+Wc?Oi_r^qHuU5g7f z4wqE970AZUrX1eiqh<|upqrqjG4<}I-`f;Zog3h7O}dfHr4ISR8*v9VSjjJLRI zv&cLK_1+$rg5F?u)j0}0jbT15WbVZS*vf3MCUTWsqCXQbM=u*wJ)-H+m$ z_q)U`DF)cqrRL->hCszf8g}aVKY3ytbexMGyvE}nEYn_2P&=|K>PlIJw>;~x|3@)? zM0T-2R!4~_eupH1%sLPjfy8S+Njl!DMycu&A4A-Fl4f`CDCgjlee7ejQAK>5GcRvs za-g%mk+NU%sTsxLVQ`OPrOMsUn!@nr3CA0oDmr>nzPDMW=`X(h4EtrG>*7hf z_I$9u5GU!e=K0F-evF_j1O=6rt|4Wh+b=V!Ro)h9W$egVCQ?%St%632J zF8CZYrnj!Qouj#*h|$@daXrn#Ni;+8?GMQ zvIz5+9S=Oz_N~+kMegb#KSyVDAeqx4|B<76@S8;0j|vt{@}Mf1()h~FKpZ~4zcJWk zK%q3!)yg+{;aAhvXfINV-b7Ft6iMIz64IOCo31i8kn#nwARf!0F*%t2M+=wRg+HE% z4gQo)MfVE!>4jNt!&k2^(UVMZO8V5gkNO=C5li(wj+=~<>iyveJ!F3mpy<#VH#ZbX zi1;($;XlL+m|!Cb62{vM&Cg#_owstKIB%*sj}}KGjA#%{b%V`>J(E8ITosN zRaxB571jYLP3^{!>k!HmU;PD7APVC!Y#p=CRbfcQCPI3_(NE1$UL5u9LEU&vEbkHe;le^flE9)?~qe<_ko|EW*Q zi_E$xmzZ=_9m`Bw#;k=-VfFA!?WuHU&LlzpIMw7q?L67cpzz_BTlVzT5VtOb>ik1n z&#y0ilT=9>l~ns+9Su#QS!_S8bFR!U9fz)a&A{^_tyRZWaTl^7IZ|Cnd3f`*UrG1d z;+b{YqjU>r)}orP$-ns??8zp6wz*a4;o%)#vs$1!9PA> z-ITTt_Q`sKvk54`WT(p>7B4)U? zJY;F^GPKt6Z~0HwqUunSa{4y<(bn)-*n5`xzQFS#!)jT$hIgO(@0gf9pxF$%9U>%M zBDs||K5x5o8%$O2ERn!>?CoXus^+pzACFF$@2Zy~7utln+r_Ee-aEF{Q`*X|6|SO1 zD#ZPY>dAFCQB0WPsL;=K7+(=&ux+5cL0@rQ?Cilr0Co|VAughBPKuZh7y@8;#m)p+ zSUz7;#_aBGN+i1HnQMr)7Z%Magdq`rsH(<~8T1h=ZNzB}@}|-=+NIdBJy#o)&g`qT zh%CV#CinT2x|I;SySgOM$Bh9nRzn&2r(gchvogKZg3*nLR=EBbtXnu4Ju!aY7Bau$ zPac2dx?h#LYgZN-Ia@$?5q9ssBVyyqaTEFLIPvx}Ew|EbS;dEZfoRozmHn~t>tvG9 z^U*nFz?b8LrE-6^30J81bK76drRS%%9hM?PoqIQL!ZD{sC|o9ae3FDf`95nA7DFOX z1I)7KC!tw^sy^GmZ1~qEhS(15+hSw6{~OOq+my z&RJTS`u?bP;kCBQUB@iKGuG0-BP^@uW%Q=-I!yR!eaD47M8>8>95pKLX5d6Rx*2P- zeBbylHTRn*af;B4b8w)epM>1`Yfo#33pJ+gJXU7iVbu7t>&KV;vdO*Z_ap*W`Z987 za)N9O$^~s=`J67I=P7c>yG!NG975O?P;*kUUKq5zIkOzS^<=A#fe!6&x!~xJeqzR z;;ZEgSQt-gBd25L8^DI~_HJuA4XSI9Y?p1%&B5UXRe##9qEGfB?#V(Fen(zpv#d ziPNd$FXdR3ObZ24nb&-R>_o@6j-5-0EZ;@vsdYo4=;Xb>k|x63s+5LQbaXQS{8hd5 zol(m%P#^x!DaY95ovDb!RX3dqNau!;cM-TJ%M>v^wECcT=H%GOPJL&UOfoVPWX=NJ zW(gML!}!FSjvDwaMrZe|1;0(2y9>q&rfq*E-V|RiRwVSee!$@CX1VR;VBO0Ka(9m$ zLqz+wch>yoB&?U}BO05i8P4}tr`LmEwE~2&A%I38fUO1voNYkB?cI?;JmhV^s{vud zYw30Hc!UU^ERe+}mt8K@JGAJ^*iMjBnT55FnMoCJ8M!^hfOab}S+$ZW*9=)sT;_ry z`vM(Tt1Xag0k}j4Xb<0?vSmOH%+mPP&Y1WG_&EAFRitcr58Qn>_`(C%kz<^RZAHF{ zD1rRQx}igH;6kj!ePL$~-?9E6P`lnH9>kMjnf3b|3bjQ0Q#~4RYI|(@P%^r`gpKvS zB9GO>Ys6%pC9R}%uIArI>c~T3)mdBDSk?yqQ|cR3pom&8%5%!%%IW^@jH||V5h&7% zi;7oz?eOy!jt%qiHm&HHolVzK*41k6-W<65jeqN?epb_)nJS@(m)^WJNG0k z&JSw6SG7u0kYuEgz7KW{aSN}a@>prkYsc7E`|6&w?bp(sw@A+G@nyU0{`h%;SS1Mv zppiRGj6W+isaEedMEy$8Ml|=Mr5@;%3@35J3}2v3#1KnrFZj1~{?l7DMID1G*feL7 z4%XklT78GrgfK1KB=%9uNOp(~N2Xa65u`f%l{D$6Qt)W0;<9^JCk%u{pYL!*um6Z@iH#P_aRNK5`$=FgQ6t;=UTAZp;pHEJBM$fp#Le?E42aobPw0O;YDI7Q8Ut3G;q0(w_q z^R8NRT1j;rPru|3N@Rp)cDDF#NR5v*R&?XSPJeJU=EBs>_YrWk?4w&)FndmS2XfN) zSoxm}hJfBwkGc zq-%%Ge-pFL*5t)|9d=*Bt9KbA?Qea+A?d7$_f3+H*cX!bj-C@W;WMhF3uoahwf(z3!w%4;F!2=sb!(xcp5b-x1YAG%bb`eoXUmY}CSt?Qu|BxkH8?n!)3C?@Umj*vahwUV!p z8UmAQJJyP`G2qRE?*6EV+vE4E`3-JLt#Wm+QM5GzHQ%>wGqtqbs)yUndmFwS6fA`d%qrCO zy33w*S!Vd(-MnBsM6AUYiz~0#FI_GJ?vPPXOxIeyDvkDw_wTRw>1k--0s~!6JJxX_ zVqdZ;xp}aD=aGK;%=J|#bQ1xF_B7Gu`|uF&*N%OWPpUj|sis*<_n!yH_|r>o zt+NZRlx3X12hO-69;ToSX^}*f+DjX+BN8?%_#uTK7NMN^nMc5heH29KNce(CwTt*8 z-8?IOe;?uG1fiuVdy^{A=!y`LD&smX-TqZkD^9Q>e{h>+B@h^$01bSz`u6WIYPCzi zG7F3eYW|yiaE$vxO&0lSL_5H-IDrNWYOoF5j3-@71XZ>36!5LrD9u~VdmChqLL2*^ zQZiN#cPplDBs3)e4psl5EdMJWZblK6i=m8FHybePonJTZO?Kk+9TAM8c*K&6zDLO& zh#Dy5+U9aTX>h(W%#5jY;4{R;wdMg}ax%_YbQdkNQ#k4e_Um+_e-6IQ0EsjErKX+biPP za&qD^e%pJ`3e{EldAir35f3Qz1I1iDAWdiTWVwRr+t#BiFD+rWD8hjSHIrhF1?n@E z*@3;Q^3An!p0XB+DX90uB4qybt|?IX>(P16!#<$j#^$P$@@x%fwf%V3$i}=egBPGu zCIE!c`@X*E)~#4aPeetxxGaS0!S<|guk(F?i4quYU->grI2o#T4G(fx(f6WmB7RObllo1YGETEDS}Y# zO!&-lcpAFsIC9TzGn-Vg;_7l0@F?bkaWXvW)_7{0WEdm@K$}8{gLtGI4t-P|Xqnso zYY!(Rgd5pgi)rzBuwHCq5mk>gtdY4!pK?f4a*mb#>L$i!iNL7di9r){qb@< zGq9vqW?08R4TBMAz<}VwOcFdH zqIyYsM7FBoDZ9p=i`5pVHlzjHZhDrZY1^cRUx0h9vb&Se9=E21Q7F@gOgrKK zB<}wai2iHStmIV#V8Pt9W!cN*N?l(?qf%H2&THDgcSii%EZ*uh68>An&P+MPo{nd1 zEk7r>4XaK}*qsACMQ-16Fd<+Ie%#QMdEhHxZ}m)wh$l$=?e#F1r-^BQ;z8TJn>@3g}FNK+BxIl|`Svt#uI?xER zBosm(GNJLKGZuB*>6QW!XWXW-1y#t!0`Z;o0Pk8R@jjNGybR2`oHmE zP=ez4DiL*eYl5qWGlv`G{e`*x$1_98j`IJLE|g^j>LcHcrWLTl0QL6he)YDC2yFWG zxO}60Wt#(VSTM9Z>qL*teYijqvOC~s4ZEACw(f#g7#(Wd%Y>7#tfzdyp+|b9Lq6+Z z{(eW^SOD|y%Y$^58Qce+vB6v|AzGcTw(+qgfldf__MNfiT{ruf$k=*ByF|k>XS_;T zdf=$i`=ksLv?E)Snh~_NOs@+9vR>BnXfGe>0nfSR~UC*?c-X3^74&3Q1!P! zEWqw;;}rW#hAzN|jC)}0RKg~Erk&7#uLJ-0?KW(A)$SU8miz%SIZ(xU;912uO2Z(M zrFw$cF8RJ8nH`h?-;AK$f6v!@V24I@*1M{LjXY?Q&B#x04kzCHOJZLm#YhoxEFg#1 zq2qxfk9k%BQP1ZxProxy2eBjP!Q3&+ELuWunB7Q90ztfvOWyyF0*&^VOT@|0@Gl29 zKzZl@mH^t=Us_|89tlt)CEZ2b@zEp@g3G}n-!w5bq|)*if2fLTK6v6sHCnW5H0xH1 zMPkW>DnB|1t^vaN^P0FX_!C>z7$3bJg?bu^{?w47rGs$?nnd+!RsOh;}__ql$7R>^|8;ATM%>a+E=Ia5bd#%D9bAJg+@XL zzAv1`y}kv7XYG1&{^aUHuJhM^iw+iS0p$#31Q>`8yUV~86#tT4GpKS;UA`d8dXGk` z^|+SE_uObQP)G5<;qvgGJ=HihxR;vgWw4nn6n(!2^eguotn9V-1k2ingX@zEVw{tTKK|D)a1b$URRW zg1ks_kGi-fw?Aa&G+nhpceeM1v|_1GWK~0pU|Wmgx@yGNC#6|Mp3LTZ+Dmefmsjyr za)bwKog|=awI)%VIaJzlo{pK^EuC$On=;BiE5M}KD8(qT%+A!m_#q|Di({v=q!)7nGih@Z^(C*qY^UZ4Vc%rrI^M+Ti;?@{+}#L0bYD^lDzMzg;O`f6nLdbH zvlZSPXfWBl?`+N{>b7d}EDnx5vJ)%0O#&%^*FY4)*fD%h>_uvG`HEivCWc`P&=1>2 zb!lFOYy+NF8LrEoIK{FG>R3<%uSi?lcWvv z_Xw_up2VwE`ZbI~JJJ*^pU>~aIZ+vmqmR_G#+oRjvdt+nNpG|dDaLeCxqeA2Es%-s zLxu5Ov@(SYK)1bf4BNdwfJ{bJ`1Nv>9!Z1w)9=!Sn=x>-Jh3+?8NYwJ^+BznUI^E7 z_aj_)%8S3%1|4Fh>%;OX;a&ENQApv~mZ^#+v@RYH7sTRtQ?_aUcxS`T~t3wjvoN@j=Z{=zC@lP1>E(_;6?usUuX(;gp4#)eZ zm!1%y0IV&hxArOg$SAt8NPweb?GWz&_WRSt*HFvG1{lrC`Qo@Ex^60!=WkP$s$s%NqfQ=eW5I3P7Vy;QF(F zDcJve-<15@H$QPXn?zK4V`U$TQkXsg5a?aRqDJaCxgj|lL-ytrWZ^ZFnY6YhM!t*l zs_%_M$-9h_go+fUkL4;hRd8?;>Yg;bZ~bC7M*4V{c@u?%nnon0`Uyp^Wn>e}PblwH zuVd=b-!oq84JxX3OJsO{dHM-$SiU@mP+I&9fniwB7xIe^T@hEdY;1xAp21FD=}ifA zZiTNyhr9I#$6KRbe74XNpiGFd+#Xw_L}d=nZ?kB`NMCLn%bvOYdq(bA90=SYalxA5 zH@Psx+r=eyB99fQhkfLlW9AHblhtH?m4qd$?)zzLKF~H29|1CH(!~iuY;Ng*!v&KI zhvld~-G^b0Ptqm*72E7MI4kTAO;{cYK5wl!-gOPO1z<-%X|)Wkp~t@dmm*)=SE2d1 zKewpCYqB-@SQe%^OM}hMTrLl+o?8B-Ud-W>&EGN|=)U5Jik|hX?uvpQhLCsY2-wJ; zh9@~J90wu|p>YXj^&335ABI(WPDlwZv?KSX_JT%SVe7es_D^miV7E~SJkva{wzfHBB!>t6Wx3nm&S)I%LG)O0 z!{Hw|O}y-`GBG#keepTrf{kwTuXqhSEs!;>8ahjPzj?k-CVOdEzxN2~Kl|Z}4JSo< ztriD)_cV63Sdd-igN!9KwHV2|Dd$HG;4eKVd3`>RPE$DO=worMe&Qbw0nN6T9P+#Y z0l>jIqSgN5plU}@0d-CE0C&S?G=y?icUUCWJ^GI7;@7^uq!8ZZ4c!|UD0i&FOWZT^w;4h28|ofoBa;cA==oJ7rY{S@w;w1{Nj&ArHg zPuhF8%i$ct_n6}X0*J^d#1~Oz>-T55JWliLi7MNO)Lud4s&SokZHv-Qvyn_BWOEa9BIjwE#|< z;aQK^tyXUe%YIy>E3d$<;^MOUpAz<;H5h7fAkxI=OQ!B^k71t@xwf+-Uy`Q%$x?M7 zul1yMe`pM-9iJ>y#`nM(CQ;Gwm35x8OJ+Y|OqBxTkD!;U0M(5zl|Ef0B3K9MPE6y{ z%l;et2TFO8S8~FS)nHZ7@`cO4{6Nny{1*fJ^Ea{h@3Cv=!Q*mKKYqBnL0dTdc9PGX zz5%`Wk;hQ8*J1b9Tuy7ENMpYuJDcF0ouAq}z4((1pCbw^jLNG|a!bnUcXAUcS3PYm z2fmj3CH*Q>3A@hF2D`T8`EZP}1@B^SX}s6! zX8Y0cH;uSCsp!yJXc5NT660W(J{_n1ft6$zLoYH%uzX4jr{AWq$N~Bi**A`M7?RwW zx~u(GG${sT7%72JHY^McmPXl#xwQ74+q`W?#Oe%RpAvZizn&-)=LPLhDCO>jsv;yn zs6<8u&{=1@OiGMD4sVp5*Ykh00A5MB)YbSF=j^c?$vyQC*Pu<~w^aTcDoU9hhJim; zG!G^QF{X)2KUN!F4Gc<49eKfOwxrRL8)TF}xCGpkEpMFc@(jwoPvy6X@N%gI*M4v@ zF{hRtkTXP>vIa^Ir^K!whs^_SCBMV>+G9(F`Z)IdVPQ6XEM&!bv)Sgk&F5rpktwUK!kyu} zg^UDi)|i@L|EysQ2$?U(YChy94LR3%hU_FFKc2$r^hq{Xj;)H*7(Oap(|W#=$dVmEd&R^wxQAz$mu-`aqOu+iA1om= zO%x56-jiOmZaf%q&X7q(t8jk*Nwf*$8O-sT$YsiIF?4f!$n(efBY@J9w)D=!1T%C1 z8w1t4xjA&QHF*7BM<5C8Pzb#vW=*Wk3^rVz z*DH5waHDpEnCRHKFipiLgWVKj8`$La(!{lntgPsa>!mewM)kqb$U!fal4BRC0XHpJ z3rikHz4h4{IV5Ew?J$h|$T`Ep_8Pr~Kp_ur0e~4^fWDS{bufv9YHj8YHN@!KFu1Nx^r_8g#0i>)c8^k~~f_ zb+&>1rMoJZUxHY&@0w@7DxaCtE6yrPyCIwxHi(z^8s)>6R_ak<@syx&zO4M{d)8`ypoy0&k-TzpIE zFvx^$+XWFP+ZIvJv1<0+ye;GdwaUa(AFLCzv=3&KRuX1pyol~uac+-GD5SqRg(R1! zKTBh`4tfHOfo=amf-U|V)X7Eze0G0{@^%2TFNkr_(%OS2C&dLak=PSv7OOa%mO)I8 zFkGS^K)dr==!C;PI8|-W@HMkj_;wpagxkn{sf7i$-z~lh zCHAg?o6sr1I>O5cX-sSA9{U36BRfvtZ_NQ0<-}&b3<2u@{L-bS*L5UfU;LKss*Hn;6$Mjj?+vZ$1xZD0Pl^r7u>e17Y z+jckrmDx4Ci~H@IHhF=bfX@S_^0#fsEgL)S*k_D_tS*p|nx|E9B$G|-7y{U`i=>wskyN&Q1^-HN#JK-h+9yZr2T`Gdtk*+KY#Wo< zensVJD=LkJRdzOlcVc^|D40N5^D z(@(;_+}q2A{4p4;P}o07$)XMttu6*5@(T%s>+*_rccEKc3w{y`g>>9XrOS#Ey(kPt zU1IfX1lbE)HjE&UH)7J<7gA5?`pC}v?yDD+b znS>Y9dQN_1^B@y=>rAeq?t{MRI1OD`KZ$lHhU^aV+9FlHvt)0_NL4gs5K--p?YN&I zFy0O4KtPROiK}{`a=Lhim`JDZQSN_=yy|{yoti(f$4H~qK_D!^em2J0*+KNkrOK6U zi&?s@!u`zB-3Z+v7P@&5B+9pFj8Y@Zp?Qjea#$b1r!Zo`ftjZ}s;!Q zhRWa0n>VCl=qsB)cH5!{5lM=CMs(Se`>1lY&G~y0{>13}mZHxdYxgf+f2wnfpy>Dy zR3Zwbt!NWerf$0pfyD-HmASBGxhto-R5_ynTT`ouB}R*IIPwsvN5kJEv1fNvhDWzj zZlne^1h(T6b1jWiu8X3YIIP*-ZX0roHO`^b$$HOw3t!QSrI~v!AgwlP6#q0=ehZE1 zd&RuqYTs0ny{*zjgdzBh6^iDADJ&n%LVR}KqJ2rWUZZOWC*|oY%y|(*&8#WrJHk}i zM@Nh~Kv3tI!>WYli%Aq^4D5${{R0wZDP)J~>II(g!X`@A#pkw#euG?e5yr9_>Fh!& zBtRyG_%&ACY_U z>7<5=W)N_gxlqy@X#<$+TX3o{dpXV6wv}VZ?QO|g1h#_7YR7#hF}sTdseFQRw>A;k z(N-y7tvw&fN56q1(mZrjLXzb-<_X=j_NhQAvcxZ9`+h-CPbv`Cz@}JpSRV<{OL0dQUrerS%6NJwh|3-Bw z`OqV_zDg zO|6@W_IUf|xdfm39r+jdFYZni9wVBM$)+hkM=r#@_V^V<4qx)piH1}VbOW%jez44- zlwA-OeqS)VV?!?BVg;^`mEE6c)_sz(Ug3f_9~LUq<9~ie7BgL!uvs_hJJk#bR{ZFx z8}i-CCPintx*>^jvn2sbei)>?`09r$R;6}TMD`@k@JQ2nJiDOlUJb4Pk-Hr%;Ikp% zbao7x8ElfItT#kRf&OqIp>Au7fhW5iTyh*4r`I+Wi`f(ZD$LXwm2SYA)%`7dViC0- zO0b`5^BoYfbqeVywsaEcK(PE$i*v2rX^74;n4=`uleKm@U5Fm>m68QRsRDP84vIEn z^MV(yI=!*SM6#DlAC0K9l5DCgEmZ&B3sTAg#ya2AzG&*)P^^;Vw*$gM;=yHpvVPi5 z`Nov}^=LRZA^KZeu6-%^@EM2O@AANc-N1nOQ(q;&vQDL7z&8`3dQRee?`$lQZ~Zf1 zSZJ?2@acH5yt!9q{-C`BLjpDpXb?-Ih0WmnC%^l%L?nXGelbyq=HgHqlk)IgQ#jOM zl=Oz}^0$7|i%tuBn!@O2G~6I^x|qS``KfvN<4my(sp@F+06oY$U_ojqV8fL*R#KvK z)YK$ro6l$Ijh&(dbQOOsw5;cItC@04#I+9J?*;~+a6xLlqhgo05IIrNTbIQs0E>Mb z=P|okOq3UyiSTMi_LYGvG0PW*8PBGqSIwqzmFBeTp_^4q3}(-X z=*9%F*FU!$<`7vCSqs-fQibWb51~vlgMt_;=(br;>TTy#*)V@t*rV2-IW?6oF&5N@ zABd*_UX~|VH>QBvCtUL(-V^va-@#sM80nV< z6~|cG9bOU?Q`d6z$rh_q*jv9kvdU|4quxXR$fw%N{(TKTTsHV-HrvEv&^=GW(E#kF z6Jrp1K$Y_wuba)I3>hIyj>~xJ3tkeVXK$9AcMgYodpbcihRwqzdccO7C&nI4=(rkO zXSWC{ZapEa=h4kYFj?fYkAntJl`Ch9%U#ICwlsA$ywAICDbO?ahRXt&ha1rwBq~t@ zvah`q&h45Znl~gjrPrwhiSgLl2vgm^yCC!Xox7qn!AC<~c{!|Ji22pS_y~}nPKN%> zIlz$B{zByA!0qIFu#>#@L3M6xozKCgtzq$@m?vz9!?Wp5`Z-=w{N@%B4v(pl-%<`c zAbVgO9q)nivUn#vF!V=o3f0PQQaT1qhV!$aGBC8Z339W|pBkNKo==W8JkhX$JP`+x z12>FjtG0tVTgGRCt{8zMg(cXB&AWOFgfB|5s_p_*ryHTUB4DTdU&|-NuKg?fbaD>v zI5bA@+)n*psvm|FNvqUxZY~v02Er8ExXU3^I8VNlKJHEB5%xnKI+>C?9WSFpQsSd+ z?*u)bv(hVEN(#nDzi2fG8&1O(WWEtHXt|2WC1GN=@Hw@WqC!DJ!O9Qb{!DEGh&dJz zECtuq4oSaKnHW&by-C0q zoRkGnc~d5hWiqzO9))HFJ@QX+m`Bb+VlCcs_9n*w)?Ke)PbcHPMy77mirC!B4NzW>+prTp=N3&~xO0mSat&R6#pHJ+ZHk)$2TKu}qM z6V~a{9&oy=Nj>z4GB_b&WIi{Vg3FFs86-Q9yA9D+>&*?OjuW~-GM3J z(H>-GiEta2T6D7h;e0^$FQk+A?Jf)5nU@P?sl4-wy-F|z~0F{zof?h##w(R=_y zE^(JGo#E6JtZ)K%-G%Ax=_@sy>~$P3ccE??fGgAKmq4IB6hT17jt6fC%bp})0~)8F&5+&WH_nd9&@WxHu6_G6)W z-Ch!+gd{1c(uoWI_&cz2SR2zzLg&U-J|YjKehE;Bh(Snay^$w_X-34{Fh!erd3AjVGBR%j>!G^>7lDs^!I5iz`B8!fOv1J(v zVQLWs2Vq8gveN4`pfLlC1N5=2!k*M%*pXrwkKyo5#uedx`of{el8tvl6q**S!@xs* zupdGk25Hpp2Z{YYra)dYi=pX#4VUUj#~Ir`jJqE3@CbFG7RJ54nw!q^Zlg%2F%I-ImZi zQ`x*8=#?JMzf|TV2YYMc)WHbZFVzG~@106$K^*WcH@4~0({5>YvfLB@z_zlV)5+?c z?2OoyD5uCR!+H1Lu_HO6Mv6EPk-)$i|K+);NB}#@-^RPsHk}Af;(kqto*hU_)Wuc{ zIO3y3aEGD^?d+Wl_XLIq-`zNcJ!ah{tW?hVHZbkqzSKd*LiaD3L0wsy#8`XE@}o~j zK1>)7tse>=0z*XQ#%}}x*{j*#X5YQ4aI3kU$~7_usSHxyo)kcR1qJ(P$uz&OMO7We z8n($}TP@y~qokRPyFm(pf~!nIYlW!9g8Bemb1nM8j9iXxX={)vO`^1j<{D1oD2JA|LcICA-amN3S?P-^QBJ}!)K@OF7aIWqk!1Ho;}+L zo;cIXjMPe_{q)lQ1CtGZ2+}Al{3CQxy@7)`vz-u-IHEeY;&nut%;7+5g*p# zqGc~%>6t~~FU|1zgT}n*te(F{p^60f)-|$9xfNjJ%ep8of~C)g`M>v~pf`)T1;pa- zMptPJ-UPMGYZVRlbqAt8jM^GWHoW?l)$~>#etTNE&LfPN0ub1;cV9%{jfdOPf^n$- z#93Ei-yQi6nEQ`U`?~p?#Lj;c;C3_J=8bDg(Ca~yfpy*VsODEc`V8C6#QB}EM!qvN zr2sR9@D5FHoefL511FkHo788NH(-wie%ki@r<;qGgcIQ#^X`IuKTrF_4=VlelmuVP z1SQ7yfrLQH7vDBdTb9bq`Z}E0dh6hzIh8A|!Ztf~qDc6wMelMHUyYb#ed)wxFk)cAGE1K7-X+u3;Ezf0SSMm39@y&ExD} zL{2&5ZwYE>I;UWnb1{bS%o6CwaQB9#Lz<)$y}yf~5XuGU=UytAyUTk9i48>-QJ;QK zgDhsw91KLh?`w9HjuMvL|IV_zeecO@EHcNaF=~dnFtA8KJ{d}hp7WB6BV{kxUAwh5 zG)xyVF3@M+`@8b{Bo1WaC^;f4Jt`QW@N4XgR=v2@t_MZmo1dI+mT0&)rr57UvY~kB zHG={8Vu*h4m2$HFQ+LK|=XOX!O>xxijB?Ue)f{Xw8_@~_Av5XseAc>Y#Cs6ZO%y?wIPg$nW?KW68O zbTeiTxRBM11+?mpf)Eu2Rd2jTH`G{|s6Y=fj z(_gCXSJd#nA`1%o0)yhdC5Gc;H144)_d~{#0&x8)-AK8N?*Q;BRc~ZGga-kM&>a#d zi$lOX;&UPDaJSqFjzYW*0I?*VulGw=D3@(gMLpHVPU>&-OWuRjU?Nc3w z(eN!ci|;)2wQ0pGJ&~m*HXf&|+3HD`EV)o=Vve-~u^gsmyd4SAT01dz*39)eS!??gmI z-VfR*s{ydDxUUH)uhhEAXs-@t^H9%QU#RvoPcIJOtpj=E-BNtqB$KmiSA0+cNa=BL ztPi9Eh+%7%j%du@k>$&@q_bK-aV7eo$!ZG*m+?ItO377@p*9P?8x&U2JP1LSsgb{e zOKjOWw5j@H_n4uN9H5JGR)Jw-fg>QZ0-Sv_fdyAg`xncYA3!yrF7EB z&cM}=z_oLJhiq2*hk*i}Rud$w4$0&6syD?s4ZO1ooL_2cGU+4ccH}2-`%{2=oVOc? zL#^Pzuf=dTb}s_T3gVC}Vu-vUH4r=W8U|dtl$Nz963f9%sHdJ8D%J%v-cukscTmsUTmI}50Q8=`26Z?G&z3!)CTjR-+{3>A7 z2%RCv8&fo%29Nx?M$GYC_Vy@fe0n9QPC^MfI9tFSgGnCO{NcK`Cx>(E{T|s^e?(}C zq-*OP{PNvTpKJ^47Af)8RaSevt=czq)(K+yfj0)coSWIV zOGk^e<6u3#TVv%qdU`bF5wUgyQOH*)XcBSyB4HLvV2~o@ZahrU17kfyg?5$PR17$=K^f?j$W3V5b?!|SQ$z=R>I!1?xSncWpS-%+YP)c7PH zwDEH{gz)e|Lso2~+f7qcTwI1{^Nn{3r;}wwk3b~zBXcvC!F)++KG^i>u-DuCJkH_Y zC$-qURDNpe6)2@f$XED5Q#6#4D0HwM!_COhkkp z_u`D>-7cVTd6J&a`0mzNq*Ilg4H8oX&T}_kR{h<8FqA}~wWp{YL7UUd`FO9s+zbcTdtnNqxV1>e^FO0RPkiVAQql5MO$jfcVJCHY#(#A`9VZwYZmm3cGVIm)~PlqlS{KC`)B?wl}U~OEF znxv0!srGo|Kv#IGLsujB>fgRGl_%ms+9t^97M0Sepy$*p!)Mz6W;oI{CRNh3b#d%SG9 zS}EgxOSUp^MIFK~Y}smcT_ZxBnLkbvjSsHtpkLS3cK=r*E47$M@7C}5WO|YB{%pyZ z(YIp*W*b>XTz~>=7^!^BbuCDfdOKCpkcGnYC#^aE{r=e|_byI@VmDAqUVPz&$XpMt zG(RB671@1oxr)+r8x8JScx_ne%TQyhR3${#1Sv@VDs9MWYFMW4dQ~bgUh>HtKi@bR zU@(Ji?09JUtEJUY=3Cv62-B^Q&s#q;i#w7ave`)^N*)FMl97E#Rry8)wn6ZLA8G!S z?IA>~f+1*m1J4Wqo9JX3UNcc7X;|cve&CrOR_X*v>IRu5l^?M@N`{+02>*jr1eFle zvJ>8#WHRN;2KJPo!M8#jQVFG#u@p{gtOhY^$8uEzWwWrcQQ3e2Sz(Pi-%}&7_MW7j zugkEjKHADttHxSV!n*SE6ok)Wi$xe5%2)$Wik;0O4weYf+T>2B?+8vFnsRZYl2>^xv_*!(<$K^>7BpEvw{imOp0 zio31lPu|*z4UaL!FBSV>^4&n065uSxmguqD%j#g>V+`3C*&gqo}xd1ixcgtx0UqRNs&&l_AfcPTfeut zw@I^*Yp@yCnD)XVSHR2h`z`z8PC@;5b`Ju>`EWfWCR+;%b2rWpR|`W?STPm#yc)`Q z3Z`y^N`nW=E8Mak5efC4ykOvA<5MHAKUqb+PrrNR@RkxB`DS)m%i7d#0ML%zehY2q z>O*>7c`;Al_@BFSKKZ!XM#bo62j!Nbqzel+3KFmOzY!9lf@=Yii=<#KkI1vKx1UIw zMU)$9fN0?xvcd+Snp91*Fda*cMr5O#oWd|RFuDfPgsPCz{PDd%@;p?VgwjG2EL#Dj!Rk*xuC}O%ttsT<0;X8 zAimwASf7q6fyd}f3|x}>^!&UZ>d3r5a3W;8o!bn2*bw%-rY}5D)8yu@Zgv*!*nC^UZrmLeZqx#$)nvSB_w)hn&O8v*1~0V&L)daBE;elr@x&KAWxb*H zJ_Tk8>g5d`1}Qk?Xun%bQfIak?PyeQq4j#8d-dv3XR42L%Uy+QZvW%=lr+(0LY>)M z6|CIRX{3ur4a*Dw@N47n#!#rILlglX=hM07lcwt0!T!t^{iAN$bhI4sucAArFia~d zN9K}_Ij(xzdNW^#$-fhSb|_6>F7tzV%?{6#7_|H3Ztkb$43k+49s=DbHxFY!gw;l@ zcE9_E-L>lVCd|L%JPm_DlpV%et!NhP67&^3h<96V_qrKb@gjH|qwIcSC%|;fn6_+v zJwAs~&K_nmK1SiGq%G4)*+7-AoV}(iLK81JDwJx*+MCs}yN`n}rzup9B2OleUPwsh zsL_JErnb=OByzZx3HJLW4gjrh;Es;3h!Q`>OLmwH7XuwapdW% z5h+r*m>7g{@;59 zrKO$u5KC~nywY}a1Jk83GX(BOWAc|##;3;ZKaJ-+&Qj9z6A7S|nH{$D+PiR0sXk>wS|7q7PDB&ZQOlfQLMYc_d(Y^smi_3_(`2nrDoH$-P7-k<jaU9Wq(HNg^jKMn?ut z>C2#mgN?ntBFmeRwgOnaUX|^y>1$x^4Z}p?(D&!@*im+t0q-J%s=zIU>4taaHqSWj zpnjLmr7X+_bw8mu`)TFdA|PuY-#q(zaPS@m4h5ss!1tnI=QzM_4FUeS-@`T}fJwO5 z!mIvln?&V}b8G_tsXriZ^2O=Ik`N95Rc^xXTijJ7wsOz)h1*$GW)Zw>1uo zHYphkeDlRgD(8QvvHx`b8(TB#P_~?uyfZUuW?!WzKbztc%aK~M!s$V*pd8RW~RQgO~==JU2PUEj}E&%Dq1$cEoD>*Asv3 z-fs3uxxTK2@jAnOl=H=smp_Z50yqs!aeARR8<4}%Iv6U$@~AH$O!WVy7eI)nLkEiW zT9LR}f8dHWK9uK}a>HJz*vIdj@w~beg9ad&XLrRc>f;?OshmHM&YXLib9TZ4p~-ag z`NTy0KWn!Y8C}j;ufyC-Uu&e9l;OmoY5mkR)H zEmkJ%Blz#G9A=7VZ@B-pa-8zz4#LK7Q{CH;p!dIHTWx~?DowR_t=E%7nO6OAUU0G4 z=TEtgDOxzn*~%Vw_0O+H_SiD6)n3UF$_p#a2+jmcx#Qf?{3*!)176jIfFD`vH=y~? zch8A~(Lb&|>LDT_b(oP>h%&+DXNrCK`+xt`#5VCVcS z)34aZ6YWrrq@1j&+r)#I&H7*C{4)te{=YWP8JUm)M*p2;3sT_3*jNE7bee&cCpurv zTEFdUvvn5+b!pdW>RA6{+Q7T#sDW2nUDso@z(+aIq6Y35h5jHcb3G|oj9IbM95^Pri5(Xvy zc?!EcP!MSs;1!_%7CkU!LIh!>)IAxsSS(-7aO`ar=^-y5?hBVwpp?@(GhmwaN|@dG zL1{&4U^S(&dstKDGYbIjyuiAGzR#Hq!_KW8{+#`)9vU^;1PzjH#^eD03@#!S~BULodlPnv*95YWMl`w7?zK%f@6T9Fup8ce|k zd-1S1<7dMD<#cI8Y(ayOX!#3FbuK=$%cCSjXD1rG`Z)F$e(!IivwPy8O$oJK25ilv z5*+sL6(~nooh*nSvd#HW)avSvc=o*#R)_IFJq}cGA@|>{y!Q5v=oWW5Lh4W1}h;^QHY-2-hLufT+r`; zs-fay3oSz-t2r&JSP$YAa&<$Ax%5TcH%g=4KvdiY7{zofaxy1SGLe09qj7rC%GB_% z`Pyppak6g(#J|LEYsqMDMcC#f%?jc-rRdh&=Yj{?ks7QJ`n}69f(Deydnm>U-STEx zQ#hQNk6~;(_NO?kLZ^*Gc-$Y%NzM6l?QMe?SXMwSS=Liu$dCx-{GxaHWKhK+5k$R4 zDvcCAOU0aSsmmriGXJG9s~CSwlBUIC2q{)pw_{qCIa@}lE|D}+sIK;#0X7G_{Zt8OC!FIxDGP>-7X0UIC zpNv=g+=Z*sI}M3jNMG!oY>s?P`@4=M0|efOeH~Ikt*mg~!@ZRS0%O2wH~O!C{>O>b zCw3ve6o~|EEY+HphIvPTNN%B`#?Ld?lX&e=F=G|>6s#aCPurw833lmR`Z8xl`mq@E zBLD=%t}aS*WORdGgK)W8O`$sUIPtn!pH~i^9678*#xo)n!99(%&0xf#W5gdnyrVTo zlMK3}j$rT>;e%U#a=F|yIh_b19YnL$X@r*5^$K6If@;(m@c2l*izCno{PZE1yu%F> z4n*`2yR1#zEO%!jqxqy5=EpSY37+0?URDOoBWr37Tw^r^xQD;LB5 z*yDFpUthYEg39vY>ir3@4z%HYVXyfYcmYC@ldEHmtG1^+@&Qqo$@Yffy^(Tw+<%#)=? zz9>g%_GP-eRq&nNIWHwSHsmQHn3wI2q?tXmz&s{qUA|lEMq;gl4>f+Xuz_gU=YmsewI#l_a(sqTjrE34Fzog-$)0Hv9eVjZkCYNvAhgbQZ@_ zaoGEahoL{{p!Rjzv;pF_zSEl>XXB6RV+e9;e00Wr5J!k z?qLPI_c?mMDVjOCJucCEw6Q*XrvRJ|mfjSJ%*@#7m7_qf^=%VgXZR9DYm?P}VSMWG zaDe$|_-W)_0s7e4Qsgqz`%XY|jm#1{)%bRyR*hdtQ|fU~bFOURAup@+<39YECp1Hu+uSPIXF@Xmf5?a5% znIApw$s0xP;R(8j9j(@;xo3sWX^wrAt(Buu-n0@6Rv`cq=6<2yOVL8f@{ zc!ZwF0IAi=2Pk#WxE<7)R=qG_Vv=Nm!t+egO-_rTNRBX2*;Fpq_d@USbymLm`^mjD zcWe?^YW*dDA2AP75QOq(HY6L z!{z&^YDx>=U9+Q>>C41I!n!I>@Xf&ghYSDZFPkB7J`thWWUnP+g4ZBzcI1^@@B%Le zptnz+3R#4KHLNGOyw3t~zPkPyfj4oOoQPDuDFOMs;+qS_q+E4|AU^L7C<0jv@atzM zDz(y;h*gQ*MUx^c9Ijzo4R{aFmxKanuvZ`c01883Hz#4?yer9t4r*qtAe0nWT39Ao zqG(N4fZ_K`uplc9=x7NMvsHYdoqfiTQ;TCZp`Kj9DuHj{YEVp2eSQX7BO>=(tcfQq z^dumKP=P74v28C=iPhL>n<)P%68j$voJMv*y^sjAN3icD+bs2(XZHsM&lFcCI_@17 z3(Y%KTq_cNcg0%MS#oUHI&NuUh66;8D7tt~K^o-$RT=6W??sT>MRYIpt$d5MgJRJ+B;5#;TUsQsZU_9wy;p&=8ADomMW zOw!e1pK1K)RVtF$L~)-+`hi*!%$M=dL$v+3Y>M51ID6iUxCw0xxO=H`J};d|&RdQ_ zSzno`MeRUUt5WVDW0G5LIG><2(iN#Q!vQn>xNbg^&Kzh}nP~f7P8B~L`SkwKDyIqOs-^5En8)lUP7)>$69q`J_WM;IQ zb-n1SCnqHal{|669Wdm32nt;r^3N$F90@OY5{`3X>@#>L=I);|{zfBR<+4Ea!{6|w zQ?F<f8s?Xsf`7mF4^h)4|1$BIxv^HJGMUm^p{;*a8Or2669j>*;I*?YWmlqC3V0F<&d}4@P<6 z)~a4fKGwy45lsO{xtr%QNVm4ppVYs9cEMy@BSj0k-fmJdMwV{I3NerPS0+VT8L~m z&LrlmIf)c=@f1y#pUPt2kiL^~hG&0D+};eCa=yGi5BeOaHY1Jr&nbj=l;2JQ9eRTV4R9*K<$IHeq{a&-QDBI+AsgG5E&L~-IG z7Ex4j^e9R#4Ng2MJPP466xqUr@NU{sn9dCkM`fxhHn^^KV(1taJZdyVi#e-lj!5G{ zF6)mrN@PM8gffNzV3J>A#kK2+4}AN9s$2F)(Bi(_cFu1?Jl7@kSRWyMyfz$sj*37v zy6&~;J}$iFeE$L6o@a(!v!xNG#;0;0NBbl`XC;(v-7gb{sIzWv(+J+ zHECo)u@r>0_dpadY$i9%PU2TJx2GlgJseang5hDBL`1b9(QEmFOvp}YHm{yPMo?V; z7bj`Q4Q9Isl9QK!e+$K*?vtGij1R@(cM}sba&mCf$*gj|8SXO*sFwAyu;^Ea=e&;| zg$y_m#m{5?QpMB)Aj;h~Dc2u?*JFg#yoA9uKhtwwZs5NG+5&$LXB%&oU16tg#O$__ z6`Xzu#+hN)vllt|F^xxJw-_=Uzve?u>|uiqNvI61EaDMHDjzkQdAiai?C+EDo*^tZ zv|%?tH5*AXX*z{jur+4kS}u4+9Avp;>FKRgvnCtKF7i0tdZ0Zk;g@}aa`L(q%An(8 zy3XeL8R9F4hqyRW_3fOEg5XIN#@T}Ln?7z>n_ailEAKIUofR2;=xqRL+w)MZyfXBU zIgl(w8;PU}N%mb0>*&e9Txw1OD$tqjd(I^`iPz_S%wil6FunWATrY5|UrDDpsAy?q z@*lX>dw-HEiG5D{`=-BrmklFWW6);f_fag;Ynp)U#zxmd5J^Mqn@jkO#~Av1%)wrb zTh_6b3x(VlxS|Y6oX^5Jf={>+zP|8co%G^zJx<@wQzM9l!K&R%NDQ*AFM1eDzwEX= z^u_7nq0#x`7IEM3-rfS@()9ys&8em$cWf!!d~nH`O0qcY0r|feQA*diP|h*uP(FSd3)(BjRcs7Uo-1hg zlu`Ym2J}ntj#sW#7&d|2uH;`+^?Tfo!4uwaqqfN%b7VXq-CroTD>FMD_h$uCSPCb_ zAOx?w=unJsX?MlAi^elxH@15FU)>~F9fH&|%)-iHs*X@v3<;AeG(dmmVpwZ61{9P) zODU@(oE7Fa|Ik*xg-;_>o6zO?z$D;nBgVpEp){j*sGTs}Z&RiIyo}QD2@V>WP<+)N zT%*A&wLQipL-qsQmE*q4F>`sRemK7$ML)Wgib4uL8VBnXD%z8uZ?zp6?kmTbb`?y? zuss*SgDnIkA2{7t>e6Drk2n@;Cg-F##zQouzqSJTYf1jct*NN@WB?P=^>jipT6uB+ z0rke2KN&k&?ME#{c={T^`z zlyL>(#S4c>8FY*;M^CPfs~tMQbUi7FjTIOJDu9kWES7$A#UBPM#?-?TZe-_u}b8|i)(uIj-OW_ zF!l*tIOX^xnan6$$q0vz}p^?5O}1R>#X zjoi-1G-nkhegrW@r$aW%bE=V<}+M7oIw1p7K!r97(I zAQJyd>i~HcjK8+3>pdJh)CRBqo%=}nGLBySd{(}HvQq;!LF8j$E67v1-a8pIDVPy>{n|d29VvEn z#6Cs%0s`5|C~@B1Jhc45Bj+OsJJ^XA{UHv$xhYP3`rL-xk@7*&Mr5HBy(ux149Ot_ z3-xjeI7D#B=i7Xp~j1(_d z^1N`lq>?;vqQ!7dIcr7;eAuE=`ed+Pb=K1e{=*@QRsW-&Ywk1`0<@W;ykb_Zzn8fp z9VQmkjrB{?>Flp8PVI^Q;YNQZ!XIB`GYRE&C~(r9Dn7QyObB^d*Ve@Ad0l5W^BMN2 zs!51Go^ zl=?jUJ-uqZ?Ka@RWKcj5cx!I1{`?xA!DdzU{r@rcmSItLQQI)xAxL+FG$`F5AR!^$ z4N?Lk4MUf-C?PR~NJ%N3LkOZsN=PG0$IwH6*8tu(_wyak`@a7?=8$XFUVH7e&ULQy z+S6rgU2qvZMLFJc_A>@L*qN1>&k@va8&NT5&S_FROrK+*QVG0T&JTa-RWw7pOJ4%(c8m^D%s43fC67g)vTHbB-fqu;YUc~ zU`^DlvT;$rAjG!&W5cIAfpj#T3UTtf7FGKCVU{+u@)EI2mN0yp;E((w+7EYVuU!J0 zf;Vr69yJVuCiqnqheHb_w`l2{Lqa%IF70t?Zz^4~_|wtry?7ww?d&fUd*Fj0sXiN@ zoiO%!u}s4Ca&z+Q`Rsec=_%Eiu{uXL#ZSK6FW;a_5l($!RXdzZPHm@C7Y-<3r8@2$ zn)cqsYPF6UHd|p3Evw^^ zcz)J!qLbpP5{f%BIEn2&Uh2q@8Tz4H#`S zPHS~&E+rKC3uA=fFvjch`TBL2I&X;r3s|vyAsi0`l~IB;7wOrv7Q6+DzuZa@@NyY! zl*orW^#%rIUv;7A#>jy0%UR)#KI`M!zhmcl5`Yt&Uzd%u-A~rOwrNY+iTQv3msCV#fwnONMqQ=|rT*m_=idXL z0Mun12mN=Qi9bK~9ri?~ScV3CnoMi6Hg|!}|DS1X8svc9bwQ|p-72;0-?~sob_E7I zjWO-ZQR==GuZ}Lc*KU4|FZC5&{trvX8~m0LDs<=U0|F;PA42S5R{y;-2SLM@-UBSP za=EN;ygva3SyrStUQm;x!InsOA@exIQF*Zo1us**UNc8a;c2x?cFZ_HF<(CiBvS#4X$N|*~LCyB+!3gZ^SFYRQw zLvl*&3A}^_t;{7QQ&Cax`4Me};3%1*N{g+;A+xxJ8J9r_&%?Q|I zeVzvDar{VHlX*pvn|zcE+bOKb8GYoM%;Y_>zF7X|PUE>E(Uh@D80mvo$TtC;a!o_e z{fTnyx6Jr7SXeDfu<_b)*`Ej`@JAALBk1=Mp=utBz63Ka@!w)|gTC~B0e zDA57svTBQK?6|Dj{cqd25Hb=-wvNEarVjEP7=g|+~S{- zzoBzkhU-u>T@cu@oF9qhcJ2k7ne#0pj)})o5QK$f`a|CLbPv|(f7-NLar$_fYZ?5v zSc_o-x8CfX`wAdsmWux%Ty|Y{)U#*nGa!4>TFy=TAMzcD4*+Tdi4qEREjo&diu$?^ z+#em~J}%4^6?Z~4l`jQ3Eb^+90IpJ}t|6SNMPYQX;$`Seq35@z&yypevpnE@Se_kR zG&(}i>f369@S;|k=K1F+tkXL*6iVJpL#cL3j53SG?Tt{4YNruA!(tk7X0B;wOzI|& z_AtE*Or?B+Rihsx=D#q^9vN~y*zyPqQq1=F{fAYFVqrn?`Qoqe5vb$vQODgOTR_EF zx$sC$@`(XGA;F(ABFvpRtCb1NBCk^kC_Yt0oP`-y)nKxd1EuVu8|wp_@4PObl@S_C zLgscD_O}*8vChe@v(#k4tpOUQy7b+Y16h2Oj&+wnf(m(i<+xx&IBEEH+gvX%^V7U% zv10*oe{E*Py{yv8Z&lmu;3wR-iCW-xgKP*FJuC^5nx1Z8BjtC<$do6#H)0J##V8%=!^G$@Ug?2Z0cM72mG-H2 z8$oQtrToJ$>^S6Nn4Om*=w=blU1KY}5Z$2PEpwI-C?;Dq;vCaOyblMU!*%%D9qLsh zZ}Q!F@X_PJ&!u90%&dHU6&!hW3Rc~*Li6|R2kT|-W)7$OJgR6+hF#In8ZkfsdRd8i z0cbJj+%kiMe)9=D6tyGV>-TC(9}syK?;026^3LAM_z;w5G)Q%^^Mz4+wqv_E=OUsuh|PrAf0@2d zJXb**AwPm_!ud=l#0=A%E6hwf&4JUa;D6rlC9nG0R z-)AA0L!z*rh!`8i61)e3$hA0}*^+23&3oiXrlwCx2@Zk;u!~PyJ2rH{^Ub)ho%ssj zS83^k{Pn;4kSz((pk?9OssA=NuUOuhRFSJ92B5{Hy4Up2iMs<{{&z-L5<B5Q8Zxr;uN*NV$Plleb7Lce=e3Cen10`f(`k25tsUw>H-+>G(&(<0Svy4j2n&Oz*MxX~&a2K_?07Bo zNl8O!xdLQ5eXtQ!=oRpe6o(f&Eit%#I)*f+&(S@tCU|L^MeQVOoo4W~iL9cw_bhYc z+T|1>hPn~aAY_11W^3M214~AO0#0-5ry0t7X%jd4%<54T@t7@#`YQwi|?YY%uSJYH{*CqOknvmifd#1)JTY<9JAOw1Z#H1c%Jb{B^#5# z0a8vk&Qo!W z&|HPmy-v?4GA*w859k5bsOrk^$Pxtl;A0?r(DO z9J^t>B6dq{8g(*vt-(T3^bNlY>>i?iW&b)<`zHYU`U;%5_hj=f*)0aAJf^q*V=jzM!6z%sPJ6`g^rP(p>4}LC0Z=GeIGXC7| zTE$V#JR|JS7RH;KRbPl1&}Fc7jLPqHf^;13CAu=vFcbfQVqp{2w0i)g}ISRwo1RX`gE1)yyj{uZf&-*D7z@ z(SDctdECPGZT96WEk}T#HzB&9h@B2~PpY}}$UFn_$8S=_nOPyZx+dd;PV(?uz4KW` zB__I*PVy*2P}(dPs_n&(;ghi_q`Rt{+7z!@g5fp>3n>c=brq)66S1$&xQBC%j*041O4V-T%)q&)z%R- zELwaKI5g4EPXQ4}UlN%j4T$v-)XyysiZM-gLM~2OCta&%x+f&Q@9uq(kI9dv5}syP z`+D2aqycW&KHnlA2Z)5m$JQYMcZl?f2K_HKoUqTa@fR9@D3hp7O z7rcX+t66TCxs{g8k!s?ufCoD5Cey}HxiLpiOxf`1s-JsO^2OcZC$fpVi~LL*nZd|a z73Skhm@tROcPu6M6(>qaE1wzg_&2;Gl>{vse8?Qdciy(2mxVG0)-m*f4+zEA9>{jI z&ut!s5@WhyKAouKd;UVSN!nKCSFj0mYmy+@6a+NxUVZxYC?Lxr-21YfV~p&!pB6zM z)2sYvIhj89Fh1QEwk9RQ-WK)oFg|e%`l zT!3RfMXsCEr3LN+>S}zP!RrdsuUg~5jNY?V`UN6=EeEMuJ7vY`uNt=*;EFK` ze_uhlgvj;zoiUoHUQHy<8TM{w?p#~WjcVxaBEFUv zrC?B@@g#A3{4F5Sn5{=7gQd40XM!!fS+lW3&wAn4{ncJMx77-F(Z=^DgE>~EMaVv= zujCsf{xC%5i;L!azLlqD<8ry)J&jY$vn&*;Qeq8H$}Bwu!O=E#0!0g5h;4FVQ{v^o z0w+lm*H;xmlgm9XAGv#_^RpjH_lMV-zbbcdQ@Tb&9KQ!*?!6i6 z+A&=7DLdxi*cFmGHaj?)qq;8yzByY#Do8cP(0sv>I;U5)k2V+xO1tcNYItI0+?WQ3 zwv@kW!HQdda>#He09PIc1S#q)pDs#aVhodbRSdu8`eNeojSI~KwOJ2hau3tIDMe@e z#=^loTJ{;S4M&sFbZlKrAi}W4gaOlDu~z1-|dz!E4Kc zILxw_y)2R~vgz0F6A-EBf6DcI&ONZx<)VZZ?6&35(4LSW z8_pT+J`FFv1bS;Q2-$YQE;Ku=6H}Nn$d`EX2#n{Abjc4@`XLtNi+Q}3n^lMt75ZE! zPw-5k4Us3eN5^HJ6`8SSzLMF4M@GvJ#FV&0%zffV64-Ml5$HbN>*m99yke2mw)<9W zZ2H{aWIG^Biq*{KLEYS*NrsrvV1l0?+Q}*O_?|Q(mPQa#Fx?%LU^6;xWN;E-JOV1) z;0{tdAxz0z&!|+E{7$IoC#+8f@*Z(7X^>MpY$HiddaJOKU%qu1-w7n&$M}Hv^xFWM z6k@4P_~-`*dhiV5#5&?9op63(%CGaCMJ3upn zHeEh8i5DQP^#RYl^4e=HtZ>|is0ZSiE!S8FI!YEUyVv}llpK`ke3~RU+3aXqS`;rWOZ+tlM_Z?~-U(v4 zPgVI)OfkOh_ab%dm)fJe)coz#!AEBMRwss1ie`RWsoNjt^j3a5wt=!t$HG7S0X+`b zUxtG4%}x*a_xA!MN-IuR$4?D~JhGNQ!9>f!`KhH+U?)G?fKNj1xo1IuFrLq|T6`OA{W+%^vmc52#+-5YN9M>=?nz}-$Gu*VeNeALod~Yo=Jz1Ib z*!)Y#G9|KARIAzLlgaT{ zcFrZ766PZnN8&wuO;ktzf+9-|o9j(pDCIT z@1HdGH~9me5p{7794Jm*2#7aLS^l)PymtDwvHMhX+wtwxA>bU)z;p37zqz&syU_&q z$5)vFNM-BTC`v%KS$>qJ7b=Hd%dlQ5?Eku%tap8hfqskl9eG}*$H>%|+2pnCD!mXH zw7{}UcdO{@ERv2I!VDD~FF2BS_B7%8o!rJ=e>uv-VP9icu2oasribS%y?)|Yr*t&) z$oV5)K!{&@vmE}JH75j&<4+<`GyKto-eFA}h;Z1e#{I9`^7uyk9x6yA+jHqh0=@<( zKC?YGfm)~gdo2~?|M%4oEboXfHJE)poKD%Zo^l6Y_-6wltNPev(Z^YGx=t-@gPuq-Kti0*3yGpbiwAA z!6P+ee*6THFaEq4R7VP%Vu|{z1(3?nF+4j*J;0OpkUf3orG7|n5?nzBt@bq8MogN- zfu!Oa50(dm?O?x-O=gGhzt}nB?r9!4s8JtxSUU^HCS_n}SC%U1_0Ssk+1{HzJmW_7 zm8@I#5?Kl;uL#&%9*E?;kmS1eZI3E&V0hT-qTg}Nqg#n(|ayC&yn4qPdES>8bWj})?C|$ptfP<{Sep#r(o`{E88fAW! z`v-~$X*?P;!Dyo)fbfTjwZLS)Mm+03)y1myBJ?7n71{3B*Y+ca8IPH`(`p5f9&Ri8 zIePza_hE9?DbttJ)to};Cz;E_8jRN|{(5-+M9AousELaQL9uf}=1?YM5g4L#_Z$cC zp>D>@S!A#+lhQy80!pw?D1GogcLq?9n9+0x?Y8NTQPj=$5bjr#W_^`w^0 zX-*Dik4BEx9nS*}YFg*!JZ5{|`m(f$8+>ysmby2!$2fITGi$T9d^}k}yZrWg#Xv43 z6~3ICC^G}G-mAhgx@(vF`wM&8x8Xta3nJGoL@Kdd!`j?jQR0*iVzvR3*zMM|XfIRn z+Ot37!#$~&FDi3o1Q;1C`Clau8~Nt;_zZVRe|}q2mDXe)+yep?TyIQuf;2BGND$q; zrgUZnQhgM}t&Tels zY*LRM`@X~WF%7g?i_adIU(cRa|0io;p*);VO-w}C{%p>hSvMf1!Ynxp<&3v!d--Zw zNJFa`%TtTN0c-YidM4>k=kn3QDgHKOJld*~@fp%=#t|z5%cXp_m-a=iOUoCC`so1L z>&(8-5RMaucYf-929^4>3M)CUs}Wlt)aJwkxh{PVMP+0q&Kb=l1Z!qy zr#_DCQJfo=-y0l5@-12;?^ar2naV!pJ`ma8C0j8mVX7#+abT-71O*fx9u76e!os4c zV?Nuv`8B`!lp*CXp;}4JAaciNg?^Kp>iXa6#Zhj@JE$3`B6=qJK zV$$xo&oK{z!G7Chsm6k|#D_FOn)n}GM6}}<-d@PR4aK&yUZ9iopNnN@>=0hGnhzzY z|E7UsFd)-XVxt(%k9TKsX5U23l{2YX?q~#qX?6j-1R??mk(Ae^E@>GhV|}O!KicJQ zK#_hBw}zg)JTRQMo4kC0C7BhpNC^9!@X&Tc_~8J}A4X_$kow6V{UHs#@bwDvkY~Gr z%5t*2tZN<<8VPBjRM*VuJ zOn%W5(2U5_|2AHaw0Z5)!~6A(>(P+1M4&2aY6?%D5L#!w8s5FE8bed%e06&XS*CP3 zNK!4~V7qxeZSAlnBN%zoVS7GcabH|W0UN~uF{MUw9g*+67yr2+7xbI{eG->}=26A@ zWfTqEQNswwNYhZ%itLPNwI#$9$o7`fqMJ z;z1A@A?@TeL0)ND&P($9ml5NneLGJjCRPLG{H3p4y}|%DdYz%mANShzH~SwW4P2y8 zxIg2)7?*euMZj1skJOe@4OKBYz)-W5&pU%6>Bs1uu|bnbK#_zB^c#6xj$Spp`7HdKGNWtPn`Wrjr5xb*jIvev?{sbVyQ| z`}0gX)s99mH#`-lf+ z<$ZXpvnG_tVwv?EXKLcEdJnCr%gtJ7&V7%1(!cUkafnB38cF@=XpcZ+?N(HkFF9Qp zj9EFrvb?uedxp3k2Isv{1WU?k7}~m_d+YYS-QkoaMy1cidOz(47L1r{0B|TwBGRYg z8yV>AW`8F20Mkx=_n!U7G>+gPjI(#BVAZ~bhA)x>=NG}T`sdx&W#Qn}l@1nU_-Rm` z+*wN&f?=NhR^$I1;rJH!ymi@S)vslISQgr2cj29I%`GZB4AtlP442X8Ag=2K)!ez~gvRrkXmOY-^tm+rzX0ql5%)A24iRy)TfN@?d^ zaR?>c=e3-{*XQ$&G!o7kQqf0P(QV_QjCYuV!7nx+Qem|NA7_}8cF=6Nir0DcrGY9g!LY`v$tn8uB=xJe_;>UYSQR?4E#dg z4H64Nl3{}H6pFXbmoo>6_<+=qkIe~iG#b&!S+vwrsk70Mem4X9*p^eUklRarcA*&p z4i0{PfuWJ{l-Ug?E5W8+iFR0#$>{57AH1d?3W( zl+}C=ee_UZ{tH^BAHu0x(HIVA;HIWm-CW985=ww)S&XZ9mo*bhLEXOWkq;sF*S_}Q zmI-1?WqU$IQ{^!4VEp!xtL};J3-Fbw$M``Ne?bGVszIA8Y}w8l@hDT|OtP=QUV{sa zjE4#Hlo4=|lha}U5YAyC-i$F9a7dW9tcIu&fcS=!YmZZ7)bQa$RfQ+R-PrZIY98}o zvAX=oyqt`@9KQt9FJDBEq7lhqCUnf|SEW8UhQxf{Yv@ID;_VI1QU=moTUZ#1HxF7U z;?B;_Bty?xAbXX^`Q?K37zdkJBaT2gEkrXZq~y%Yjb&M$>U6`@KUEMhD{hZWicE~+=?l-jekyLN09jI&?wl2heI%u*vFy|NKQtsjqQ#c7kiPxR*&~utPDw`|VtRX@46{fV))1tBt zTsbsH6W2G1cg_M9#3#&qW|F%ZnLTkM@To(#=x1`+0us~C_$aDZ`r~n{N{1d}%0vy5 zww&N}q`y?&j4E9(M|LHdZ!)2!ufF7^s(3osy|cGNH+|EiKGlJr;;+3O|2f4LNQeM} zM+@S@JQFMf;jrGfZGYW#C#B;J06Y2vOZlK;*=r69sl;M;VQ`qg z;L$JkFF1?QEoA=k_+1475fn#V)Nw)!q}Mr6ni7i_>L_PS^0%#qpWt9mB8(!Spd9*e zGb+Rm8XgPK0>J3vf>Ah~AbNK?+%R?4cSJO8FExJ`azBpi?462>?uI_6p9M5|GkL=Sk`w^#B_HAUBUCisHwB+@h@ z;OE@aab=(nbqCr3O3OKgfozFNq2-*hj{`Mr(LM9m5Cp)Y7=XRV+9XOhfd0x}ygP7v zT}D}R!Tvn#?87Nb$WnKo=t_P-1$F0RTii26EVSq9L-m~(YIVbr!aPfnq+Mc!x+6GX zw$8SYw>8*sDpM~bNu}HCbwe}6?HQgNK-AwrwsB})LwF=m`I#gAwVQCYI#?O2?Od$4 z28Ke~zK~zW_-(&^7amB+p&TOvy4m5)dq75(sPW{qk#DBBPDIYAAl#z zO8rdXy8upbBH8&JHM{S@d|rs0@V6{>|*@R{sQxnTqX)fwfRfifiqM zS8*k5Nr~x^P^keD(06nY5TmRnzRb}3q^)JBAbypKd&~ET{5`p$-(tyQgbJ0g+upNm zCTboTCwsom6c0f>_hr<^l&=7tYdVz#QR5oUA8nY?Di-BQYjVqN&xUFm+NO90OtJ-j zvK5bcMvwLS4yxD8T|`K1U*Dy=n4abd!^2fP`BgOLYSOXy{#}rYtV?OACp=s!s`Ee$w0#aN(5Cb3H4%Ozg8n{Vvy!B3gbYURo-N4(o%-a9vn1&j zngCJXVrb7=9{!Q_$6G~e*a5f;yLx{y@;dPjngW?;QswPY?mra`C_7W|-qC9nWfW%K z+bP^^MWc+@is(Lc>Pm4}BbhYr#QXZqdvqGgxP;q=S!w4&Y#s#AubC(a)v)KUpD1LKd;+hDeKEfBL zdI5wI#I)(f1D*OhnNnpG*UNip1%vY+25mI=r1<}Gz=?m;Z#}gJ8Hs1orOUmT1G!RDoHhikl!AA ziPeY1?|RzeH#25CW=(pxs?LSu&Bo6f_R6k&3IQV^O9hIGE8MmeLI4MjEbB$GOM2_B z8>iNT8yZ!5kQIP(Fh}2p+f-bl zO3$E<6_+--0#}{@{0leK5c8B%I`r|kv;E^R8=k+eI0PbY+;q4Z|Ms^`>H^==M&TsT z*PY&3YR>&Q)Zv)Qb!&`d^dqjrV+tpE4vRj3?U77)xbNs?&xK^9#zT;$LrKcZ;D5#S zfQmieV~suuo-=kQ_GQC5Icy+A)$w$GEmM2kB(0pScI7BiE(l~hZ`GUr_BV^ZfhWTV zIG6#D%%exVbdvO%MrWnE3yU6soxK}A!Wm6OHS(nkMUP~3E}GcB`(P^8&4g}`^jMg6 zkZr_fMI9}lV+QpGCM{FHsrA+^a^N!NI^$-7IwEMZ(p-Q~HT`PIG#;nzosHBlUKCnr zn=s;MdvQGb+Y>zk@&;x5NB#dK=ugt-C?GUsENpB|W9_hxGo6T+gHN|`LYFRev>xp`pz*hpnM+9OfOGOMdUJV9y0{A-|ntFu2B>nd&`m5r;Q+_qjS_;3=#d z!UMGbv)RY!2nCW~U&M`$YiLEUN8%_5$p;e>hNR}3MpCOyORTEB>niVt=E@Cm{pM#= zc42z&q=3AZ+Yx_I#fPEOD5rUnt2al_CaBZ2m9Ly?PA*i5bsxylsY`7I&Tdpvw4;!j ztD`o!2AFP`^&0ibb@*PLmmE11BnGI;12PJu1N-2)C<0)xrSW4I!n1j<==Q{Mq3%vf z9&uc-89-K5L$<@lI$W4OE_Tz4?&{J8n@I#w6piuYwE42-nZ`MMcwisBEZsr^Gx@pB zubukF;jXLo)HN1`4qwfR&%v(S=PU%PunHW1$Z$h#ZK_b!l)TZ@Q^r6LkH zh-#4x6xVfA=bSl#>KqqrAbvX_CzYSu{KN)ZoD?shuEK5T_CT{w4v%|8|aVX=yjO#ltVF>M9o3gufS=Yoo*14x)P_Mzv7u-Cla>_c2J zgc=mH?aA1=X>mMY;w1PAy#7#3JB`n0808&vVPnM`NCnxEIz@EN>gwc&rsvW%B78o- zh+eD@p{I+?`JjRd46!@FW?meg8$5?>08m@_3)DE^z*1h7~r^gMABUirz0Gq3~#Ef zn^IGq2lPI=UDlensujCo#-DTakzO%fPk%eq-4z56oK zm7S6B{WP0?OhNdZXc~4(6YFLg5Hu2GfgB$fw;+@QGZ;C=5Oswg$!hg@ud`u@awmuK zM~Pc2qZLvw*<&k#B1Pc!>t~x}ghqOw+-m$cKU`U*;-4m;b^H8xE(awZ$~=tIBvCbd%kW>^|*n04tTTEuC#-1kI#NV@Ba}( zy`y|aK33+92WDywa`95g-JdZ0ksU9mc}PX|sEKwe|B{k&DdFLdy+wrAjZIr8NsIK? zwn~!%cwziA`|FD9lXkV^cdD;OYv+@Q+LlL!+l&IwNOU-_TXIC>kF+ljyK$=dHPZ-qO@<_-O z6U|6+%yj%j1nPX{D7rK_`{2I0H**V%H@=ciL;ei~+xh$;D z6eCz0OJQEXZm=^-D%_?LmsYOZ{`hX!FkYjTH|V-Q45L1Y%57xsNrI6EQO7^BrY0d7BRlM^t=@&t%iQ~tg|iO4UelUJHFgji&g*5g z;k>^BP2N;0RX8%{VlXCC<;G$sLt5P!xEcnx_+tcAQE57JWO%ndJTrb-h_qUI5t2;)$wKra znMe3gISGg+c+SlWyR9$H&3nB-3I_0S)uka0T%X(U&P4#gH2jRi&qT{p*meBf4lLDF zLc_oisfhif=ax*HWPn@kd#WDnc&mxgP)!p{hs+SY@;6nd3}(J)_%jD)2rmgl7At@T zpX*H2q1^m3TU`0dt0wRqqW3|k_3m|C?u@|7|7ro?pii0?3l3DdPMcFxYM`aklcx#4 ztjC;Q(;8qH^)>;lMjp3|y%l!bqJ24civiza4lNOM^gY+63i$E zkOzf%`){byBh^PQ%p4;fc)HJ%pUE5hQm|{!Pv*9D5$8%IB_EWD?U!~)-LL&6^{f%o zLzg2+xzBfpyt)1FPhRcw zolyat#^5`nDnSWck1Yq?O*PeYUh-74G3rzPS__9RGPc}QIm`x}7GhKhU|y4o+g0(b z1zjPs^uBB4xvaOna_P3i*khnm;BXXg+m`l^aE&8VIrF6b_=Z1;_*Vkd60o)NXLUICy?!wDuW9QxdWNVi_Fs&J* zAH`8gd9l=fu_(onkQwj5O)F>o5hxofEY^s(%f<@P6$)z9Cf4Qhxic3iKx&7_9bWKM zVC@?&FF0P|_`+Ve#+m)OR?ob22buqljIgZ~C|{6ryI^iS5R(fVGlM-;(_D*vTJh?D zrm_Hi_VAhaP2IR|#6)AH6H!3JKWm8`^&}&mmF`0~`@`8z>G1?utrEcm(a5X#=BHn) zCU^sh^4=$mvMdc6{q#tADSAXjN_#h(BXsTtwn+gKi6g7KTmK=#=&<0@EBbU|X=cx7 zfwa9bk##JvJa@g@ZSzxkD5q%0Aqh`dNbcH z*`mpCV?`TOGc6G5lAnHgO(tN7!6rkeu`ru57PP@%8sm1&Pq#Ayb1Uz)w=UPI=^6;t zpoVZl*C*ff*5zN5M8+@SP9IvV`H$1ZflgIa-2|$1vodOSAbjPHecW%yL-kuY(&Fa^ zzgT@E-@QLFZd=c?>RL;|iG9-*pu7d1EnUhd$#Z)FZ%N=}P>5utpu$`(vQbit6_=#4 zY3#dty(Bob#N5}znlURgwxxXp1M!d1{^e%o;cTGs!S}jbY~V`cUq#mk>UFn`Q|;GF z&cUPg$PUgV+U|m!?<+L?EgA8wKmW?F{xjJC39mg&*4TN%dOF_+F*!=0n2Uv{9YF!V$4HSc!f$^Ui zZY>3a+u;|7G?kGY!&Zb$@*pcA?R7g1$6GUlSpX`T-P*XGo$4Xs*SuH;Rg0Aj&xwMb)5cw0uo5 zG6N)QleH|#jWFT53sk(y;Reu+Mhuz_c*=0g6?i(=%5W zoiqfbI??OpG*Qo9BFSq)ck{+ALXnTc)TN(qP{4H+xyoJulSKAPNMEOA}DZzq7zzK_}=lma?9aR^RpOk>C@ZO8a25{;h+NosmSEsZiSR?Z=8Y9> zR4(b6>-TQQDzQrbEo5ML`vWTGcKO5gsEY;?AwW5}yer98EFNQZtXLJG@>Z~2F`$Kd zL5nf*wJoH#S&K|~WqIQ0KxmS$v@=$w4d(Sh<*YdXgxZXK9efCIz76N*Y+qLzfN$f6 zCRYN9ezl)N<;jyL_iHRdP(ZAQeLuS48mJ@lvq#P;v&;jtEBPuFC?GuN;rLgJioEa4 z%wA~84wjk)66vOy%SpTsn~Sys`{K;VTJtA7t8X5cew~tD<932@-?kOjRvs-WOQhQ1 z=dkGCX3%usH9e2+VL7&DrKC@jpv0s5Jl&l)F{le?EgW8RM%~^}YIe8}tLKxWMZ`&e zE;r1zLtqq$vCpZdYE6L(fF7Hu;i^Vv>|`GgLTIrS83Rtye% zd77=4BAJtds15h(7AZUR{aYOf8E7xnodsuA+IUoM@DWNpP);Gm9Fq5LpZGIOIKD`B zer@Tz?+)wYMV{L>4H>pt*9c%I2{&+uK0XPZpa~EgYmp?>97q^qS#tH5IZ*7ey75an zGmu3bM_K10{&ftUH(cD3j92GbHac6rNlPd23{RKahf2j3pyBq1U_BqMKnv|>@ol_Q z&i2!o`HZXbCOrr=5ZIF?%Q3e^2ru~HegC^LYbWtS!38u8^zkRiIm0b!(=*;SPw?XN z$3KIXZ&XH4gMkH-(CrZ2Mj0qU(sxinh#Hjs53fI#Mu*S2r2S|AlTOOmfOQb_84z9j zhAt6K4L#r7`FV>PVsljk25V9S+%5bH;GZokb7fm@7i$BnP|BM>iUMdNR85;bom+p@ zSPlPvevla#9_>U(iILo~MBt7seM3XB7*j{bPu1Qoy&NnH^Bj{^rh{F8aIg`~D4pz=Z5_^ILS zhj=RKtm$kJb#}i&@I+NxbHTP?eM@0dB`n4&c^uW_?NYE`NzKF}Vz};PuY# zp@IVUgvp%Q#g0$97YVNR^r;O1-^y^l{`!wqGH|PuD)sdBSKRF-q4zCSHj=-5%S63m z<>$h$wUy>qfaR9Lr)OdFOA^A@jAFupS}#JF$RGmtdj(a668f@q$Aivc`#KcA`?d2C z85d#CV_qAeKhS3Q)!I{apEV&EtoHf3d#P>!oFvG)g!>*GI9nfCD#BS2s72hg7eg|U$GH@Tk z36LkbjphqC*u5u#8DQx-0Oe9~-b>u$0YEhTFoZQAwS8o9&^O#2y(w#amw$lEj0r~? zETvhuvLDb$tq&oO+LW(<^0dtfY`YNFGI+U&%34iV`Tku;zSDiRKeP%JrJ(qlVxt;f zKC#97Y`HB}^aolAd$5~YKUfQ&Fpi}b+&;QFEhvC=H#sFB`x{(StP2--C|n^Go9_zO z`~on(`0=Nqaev8$HsS7Y$H4sBaMS)VFG9iNAHgQ;A%1l393a0v;n(^He22!stU!n3 zeOM&p&%}+^F}zXTF!WAEOJ`qpRa^*9gmEZ{ekSqL5v8*%Uc-}=4iT8R}RP4n_glB_&s(0)!J)f3{F@ z#9iDi2oLkU&n5|iMYX@WAp^f=OD8|1@!vzrmGHAV3Jy(7I;*Za)xafYFQHKHs$fdS4*TENFKWGT#ZYol)1-eqx+#b>YtQrlC23N`1} zkAI1*?g%J?_lJZt*|^TSe|@(ngb4`PD@p_z7I`uP^0P@{k;tEb(5K3=+FoqyWm?o} z;ll@#{U@XALv0B!R=GN>LKb2B$q7MdV%np@(OQKJ+yTjB)MUJlHpmCwp53Ysx%Kg) zPv0H<(`_EYwG}j9o6&EV+DTKmhT6WpFw)N#y8N@F0zg{(U^j6_!as)CQUJ6DXxsP5 z>WqNA*tz*J_Sw7eIr#VrvLa_DD6Oy*;s2fd6ysR zKH_#A0nXe&@%tX_gM7PiR>#2KgdjXQHac_E0TeznaD|V8GfvynA5BKEbUg8VRM2q8 zwnj)bvAbQ1_|lz&H)BE&j7wJp6h(yb^a7#=6UYe_u6Z% zz3=}L)&v=@kH94L&;Az86~JIa#iKVk_s2(24JHrEGv;=n%2RjDQx&Bz&At3h z!zRoPMU}(p@JsZ6K(WajG)X=5q?K_+|#&u$US;=OY zwHdOJWP8urJiv1U{_udpXA4TfV&pe^0DZF5E~m>!5?mc0XQSb(BLNwJxh9-EickBu)tq}34Yz>vXx>2Hw?Vop*Vdq2exNTZD zV=0aY8`_LtD|z>a)*J>`bMIuTX6-L1BT%xrDMWr3gi59K(XTBp38P9|RUjvaWUi(& zgash|&EIu_;ArANmqJdYGVTW73?}6dx@*(69M7Lx@aO7pzDAP2K{b9R0J`KsY`e=y zZjxX^G(1>d00_!+^`uxRZGb`*K?wxFwod@2+0R#3ad9Hs2v=(b{K^~tr*?o}8M=4o z1!41|Y*4JGsvT8q%N^T60z@I>h-l4pO{GP@uqryXZGzvBqz7&AzbWF%HVL3RpH!H@ zxmahKOF-N88}lCzOfAb|=)P|h6YqqnWsJ?b1_tT<-_Dx)1AFb27Nl2L01*hzav$S2 z#aJerEm@>M|HIwkef|_)^1Jex4AUqi<(*75<19TgQ>AOU5G^A2Q!TF|tM2@!++!r_ z6jy@LuU^s5$NDH81r_5a2d>qo#Uiqe0>^3TS2 za}#%a8U6GmB>XCIFflEmY!`{R&`1c=ihWXF3bv8Whf%MwTc;7z>?vS3QEJ*Rhk+U( zFPH8oU(O6l4fImAo9N!3G#Hz2#r{q@RD@dd1Y`7@Z3~IVsT8^j-h*8IQRrKaNmXcP zT1qvYwBG|kxNG-rV}1)u1C2u*$r@&~ANi;?I-E$R-FfyW-gvxZer>js13+(J%F?{u`ie=E~Z084)YU67!;X z^SEP*vF=esO|=c~(LjFkdo1UGKj=eZ=Sk0MwFZ^J#D6?QCFH6^6YOR);qv*t&e00b z+Zz|HK4a%DvrvDwqcD8cD9!4dVI;ekXnCHQpCO(2r4_3QM}KyQeejh^><99qit}zE zd#nIdtdrCy~tuhZ8H<^4jpYcbM zdT*ma9d~gjDyP+=4O(ql{PHuEdxKnETUa&TpCIR|rZsCv&FUT!XwT3iy%Csg+Q5O9 zx|q(bTth`HtN{D$)GCXv_Z4Sd`5tQlbMZ`j74s$J&P6%!1E%j6p3e>67w_i`Z_2H< zhL~9|!o$PMzdiQT`&H{R3S@vck4@2fBLu~>)zin3ad#-e?7H2`yU!Ma=c^2TPSD{- z?8cwlPZw(yi*54zlJiq7SKEx!?0I$I=d5@N3)>?Z@_PF{E%=<0r^P|ugOxd%_ZJiUE9D$CMv(uyh3F|OxmX8) zyvo^5t_i%&x&t?W71A22cfElNv8h7xMxOXe#rZwe{Js6s>ZLDxRfnaH&$cq>N}()X zd<~fy$GlIAtYNd|VTJk=Vebc0C^#L3h_d6~#(b6!m~OUU`8(iA)%=Y)|9MjVJ0+-m z?Y*6n?J^7t0fTFJ6u@6%nl%R4Mj;|m{{6;7IAbjJfK=QBo@1o4%v0ITX~Ax_ZMm{q zAF>RZR`J^mo((Lju%@6qd9LlV{Q|AsZ`|^N6abmF9N8Hr|Hjd3e2w_jKNZh@zdwfJj zVAy*<&_BD-_)a0z%V;BpVcV`^Pe^R9TbrBsFa;Vl>Rh$X_4vrrm$xj~#s6sdI7wE$ zQY3pkmaXo1f_m(YgTgPnWXGN70!o^A2Y1$&HB#VkqFU^$y?1~pr&S;WPP5vh_`;5h z2%463Dn}IuyC*$B)rq-+w6*TDUngQA!$1-j_nVU+PYfls2%|HJ(Re8OmwY4Z|4==! z^#Pa5B6>&P-|AFQm?p9MMF(sRtLSpYpWW3CETtLEIG4u4C|h~RxD9U=Lmhy91AjQ- zp2LAt0v9s$qf#9YT-D+?R!0B^VIe(i||K+yc!B`f**Q}muchU0#7Cn6&XYRV0 z+>-LvFd3Eg^_yFtiSk1@7|WyoZ9ndx%qI}`S;sqU^(Npd-Sr9P*|#MK#kIAE4qFWj z-i&uylLX)>U`KoUx#*tkgGYFyCxrqP{=XldCbEv#(i11XC#HS7t^KH@!5>!Fbu}+E zUd!B01;>j7T0VSy_q{2=*b4JN?iuWXhAhmmvaa%-Jju~`!1O$cgMude*9-9I!?Jbo zNI8Lp4lmGsK%Zy3>GLARK2YK%s;E<#y!mA=lm6)^10&g-!<3tS2gk9uSai=d>WiQw z1)Di$UPc*-3S-0*mZmstpr8v@)^(-S6o<@zDqgW2XwlnLI!5DMb!55w{FDC68WMR+ zY4HrPb+aC{;`&X$Qy|ECwjYU=J2s3 zA#(T-Mg2LB%ztkyT$1zJ4G`6eqT2H?iM2NZ_T6u1yD0_;;FqC+Lig4t6J9JV9>Vw@ zM}Y5%nuJwzfv?X$j#tjx^2eQFJa>%Gs5HZ-Bzltb%ZKDKs>xi4(r$Bex&1hekC03I ziva%U$i_%+zaKYa zL#OXmJpEE>N&;0yo2=cM4qMgT470*iLI>uQe`;y^SZ+9YPawTsWd*1fCKi_8!~#12 zC&g%a9S+Tx`#=-!Ujfm9S$g8Dus))uNX%3U+0$7+!=aF&V_?(K z7^B-W+{*U1VS*9ZiU`kn#J_N|EJ&Q@JnZ0Dh~x< zz%rW#Qz-1`h}2_cc`4Ce6R6LDfr_+J!^e9Rw;+lqr-}jwq83|^pWJ{bW&lL_gtDLP z1U$%i_~>{0PY|S#yk|R(z_gnJUU7efruJqz%?`D7m*!#P51Fq=zp`l>9 zcoT|u3Q$*7TXY?R4%|LPn9T(giBTZSHw&3@So;dtx*5%Zr>0w!5E`pYXE4$-!sy;| zVH0d@e-Lm1bwy|wF-2dwb*Mei+tM5jn{B|5JRQwnMvbdKfJx~f= zJ&w**Gv61&&%k$;wVE)53sMjc``&^eH)|cl8rQ%bl!(7xR&909yD!G}J zZ8b3+XSiN32}jjgFVX@rWv@GwEa2O4t0$nmH}#`sXJYYT)uN-8MkC~a0$M}o$D*6W zPOs9eC#{!BPx64;(NUG06-lVuLT~6F2y*W0v>ENNS?yum_)|2Al?KF}CnCOE(;jw* zK3CK{E|{;aZDzP37nnZH@|d%uH3&k}V&BNPQ{T>wtzT>G!^5TnDGNBCn)3NX|M5}O z!SL2_?Xj*x{7l;#s{K<(-d{1mz4dePjhEVU@nqa6K#3N<_>ZcO9>1cv_>*O(IH0)W zsm_$*Jd~9$wZ*oGxB5l1oX?ymx?!;E7HJ7X-3&QU z%}*u)OFmUnw8J(8AaCCs?5(03gub|xOpK&wzh-Kfs+8C~-wX^F+Hoh`9Ik}x!6i+pf~oawffvxt4$mDPl%s8{X6wW^ly_y)jL^Y zEmj3GOY%zTi3&S|FPhsft}>#Q8JBZEJ;1tXFG^#o$ICd9em=xtEiCr=sq}SEf&liR zZagRSuYa|?5Lg$Z$HwOWphc#Cd_x|EmxyY#D^VVHbVfKp-qZS$5F1eSEe_*b6ArO3 zL;*01x+qHrM92TBJxfnDMPj^kI)1|Q#T7%fFCJObbyZNRZ|wZkef#j0A_2&eUiyJ$k>AEih* z<^R7%>L49~(Sn3vr3ZR#r?n)RMr`7;@SOf2+dDmn87La;~!h*63dGVxv0dTkMh*5?*#A@!~i38Zw*yRM3$lB zRs>vRl#X4%A!BpZEu*9lc#ui*dHPT9Ibhc;Zs1JCe`~8q`tbUI_Lep}?W|t2t7b;v zu(r7EToJx<@t1Ie>7z+_G5-*597urs88Z@N{?#4E6MPP6o3cL-8}84q8H|l8*!($k zcX-1COPF0hFBh&2QsGR^>aB8kyU?9jVMCH9dd7SE@&Bh1 z@3i+Q-dxSWh`v<5BlTV2&rX>YqA$Hg$jf~yzbXg}^Bjk4CZ6)VOgY7O)BUq6xZw4> z7Rl!B#ROl%-eBW!=ASnZ@lzy+yopPxLsJ2vHrVd#B4oyfBFX=KfD!QRQSvh`cn zrJen8=ebZ`^`dJgm1s43hjZ@g{9RS196O7_#UF4=lJyvbsiL8iyVWEkCSEdg&*465 zkX%p$p)1&;!kNBX%~v1)?!9Dy_lzR^TTyz;MEVl11p*a4FW;+WZuN-%CT$G<;Xsqy zeY&7dkj*LyT%$4`O^}gF&F3wu932#I>qEl zGUonQtV?pn2Zl4v1qO|smFzByvo;M>BfO|fiiNd5#pLZ*UgKO@PSxk+tHO`?{TQZc z9{p;TAEo6tLgVfCd}PF=o4g-P+*2FMuCa#BtBDDw} z6Gg-denexd9P^7^KE819WmTsp0YSn0`%5gl32*q zAxGUQ-NLK~vNAJy?LdlNH9=4(uHgWY2!1Zp#87L!nq&>_S1nv7+H_?}b}703R9egH z)195R3h@B4_Y5KQ(3y+rbbZ9qQ_mo8uk*C>y62ltTNYFr!N5htVOL`Ht4od_tlz9rZWi zFOj}xGKKp4nA4_zC20z8$#s%TN5^1_)s;*BT)w`0A$HpE`N)@!;K5~|KtJDZFY~FF zZr?uS7`%%;e*Q6%5%l`I^8xFuq|M!-UswUeMaxSRSD)KKyUbA zY)1AWRrK-yP#t4OJAv!Sh>QA=t|FX<=sSsbTOPZJ<4=1ykn?P!NzBWBAR7yrAV3SlawBw&1~mB{y9RvTh#oU88e9kliu(Zq21Exwnvr#jWwFk#81GILI@G-v zi|rg})Z=G=@$RFqa?CHUY7_YK7lZXEQE>4mm&LeEdS!q z9~1ex+cKTYD1X=-EVj2~(Kw;q@qp;&zY#&*ZR94J_P}lnqhZf@v01C%RgE@BC&^nE zf~}%gCU5IzaXVk)tUxeS@76_2yjS8P?#E2Hr!mL)6gp*)s-cbp_0Kgg#iqE7Z?OvZ z8-cJ|1+yFUNBFZk66Al+GYHcQUf@175WGUc;w)NNIPG6Ns#JkkjllJ|e$PvKl(~T! zaV}CI*l)$$rUSm9vZUE|n@-~((|~8omxm9da!Xuec(mnRF+xx_Nt2T& zIo4nit>r1E+$84%Mk!Q!!#B2dT=|YfM8LW&)0e*9DSfWQSUa}yuw&;Hp$H)ewO$bw zq*87Vb&9f^1uDm1CLwm6d4Rr}BECjM;LC5@YTuP=3)~q{^sM@Y`cuZDMMZef@=;{_ zQxk=hKn6kNUhQf3w?NMX*7UqFtVB97R)Izf#GO0O=kGdTt$pEJrMe%A=Fo3;(Ex$b z>5tA`g7nm!BS%}kr+bsdHu&LD7y7*ySfg7H6JCuqQL?u*1a%1=FdX1CMn}}@Np3_B z%3V2eD579DWd$ke8^`d%DG()<=ZfYSs_Aj#&`bpww1#{N+bWMc%j!)D3Fz$W+EaYJ z6Bf(ieyu3b98#6hwGY`~#&FV&N_U+`1L^509&iFh?xi-$Jwu7aK3Gae)QD0gyz=a^ z*!;uzb@PZ;kx=c9kPM;>mjNnLH8U2zyEhqA@b#PJ$y|-P|7Lco$in21kK03=IMAsJ z>G;`PF<#G72V$0vX%5qnX7@pwl{Q=*gV|?G)G49)z@6sMW70NTnn*oKclwSZlF>dl zN)gXQcn7wraCPGSFGO~P`XfVbYf~!>p?JIVs5>A7MXre|+o)re>-R=mVH|}=nIIpo z49BmuJ-c2CdZidLs+)bx(Zl{iD#!`6h>e{dQ8I;S(4|f!{s;J#22Y?>-T~S!>&`#y zZe_>{SoMRoC$rUE^}{V;;a%(7iO@JOFz}8OI;HBh^Tr*t2L6uGs7lii)Usu(@_{HB8x8qNL$GmYHS{w3FJ2>^IHGEyExymuX!epI@gYE+} z%IlJGefG*UsoIkXV@h{qFBzp+mGn`OG3<35sFm*`o&OknXC#Q`*K?6mcFl6eT(q({ zVjoJS6UyKzN${NMCnHGEIlHX*Z*`9En#@z`f7|>iC&KnkR#kK5KV0%JWMN`Rn0Qex z==I5u_}oP6*6`D8dTs;8J{p}EdZE>YrphpWytAhb)56rLMN~n1L;=O?$)bB&0|c8h zJJCeLwMO$F;MbfYVwlJCT;HM&^(BiZnur$uaj93iK39|LWpkifYwGajQ9i<`5k4Zs zA|^sT^f}&7`KX9a`iqu}SYNU_CN(%%t`)FDzo% zBOn52lq$WR`d&&=m~+ak*UQ)<7Lkug*l1qi`Zjwt2P(agyP#oBD88>WNLy_cyIOcO z+()Ffz@$Q@aTWh-Y~^qBycYn}$TPS9+a36p^Gm_c3Gz#IRmKks<5vu{LoN*CZ!@Me z{0ZJBxM(LNT_DBp30o_$8dt+t&>o(~SI3ZuhNIz3z)gEtKb%?T?td4rENq zB(T{Hs%qoC$66}S9UxRZ+pG0#bwm1_BdP-p!DdH}mqzY0Qg#KvU+XZt&BWg(@5D@_^r#q8K%@bTNUiR=K6uGbhrO}6k-58+*4n> z_xE`L=hwiFqVlr0;=O&YISXKfY9})4sQ*4E?2ikhU`O!(($E&kz`r$5pN~KM=SYqd zX9ac!9B?&MVa2$M_MnsnW0>Q1Vf&kH=htwCPcB0*!VlR_?e+A88jo-mEEf2zmCh{H zPlXql_P1Qz)YerD8_i-ge)+VqFBj=tf&XF~Hb6w)b&!PqLw-6-r4gfe(~*|ZaXArX zNXFAYbt0<_E@&DZP=&+rehX2oe#K;1Z1eE zt<@{f~DN_bWozh|Ntxr}YWc4m$kaFmCx&S!q|gg<(;{P2j9&Y3<0NVqyuy1VKIrbsZr759fSCF@aETJ^l|qJjbs_v^O`3EK8#D_im`v>s!Ug$I9bs$fUzgA- zhK%j)boIS>&wBQVgHVeuv-pqCW8l6RmPvIh=d}L4u~WZ%ye&V{5qGXlB0?;dtdm9-FmIyf}hsbhFABPKW`1Z@)<)A zyNGLVcN$*U}Gk`3Nhyr1p!)+H)FkTgQKZANV3s?WeLLJA&Z1Cp$Z%G=J2~~n^E{fDVG8H-Bm@OV8Wy--BU90A#I!63H^l{&(4Q}8^fbQ|%*K~OIce+Y z>VLcSW@`g!yFqH*4|-P`S6vQ_j{L=*`u)NAeS>-F24IaeEi5d43O;^}yz_0NNvZe` zK7J+f3Y5lxv%T3OJfN2lv3C7ozT0f;mokF(^-1LfLWJQTa7$N5=r5xSTYiv=^&|_n z5P?;fz%8FsgD$jmLOKLNE+yEQ*sbMnFpE#e(LGBa!wK4|v}fy|`;U}v+N8~Py4vnM zB7+i5`X)|J`^6R)$cgxn^YK{w_Qq+wO*Sy+)28>;R-emhx4wek3NYd9*C3Gj9w+{hJr$0))AGJ+{w3#ZWpA00>P&N~-Yy z08#xwUNCj?4WW@WQ8ZBCnyIJ5tC-xS^-C3IN_|EA~tSA?z~JYs3nvJh&IpsqX-IYDN%|^W*(1 z!<>agUYVJ*z(>!lCn()PsJ7}HW4@-Es`3}0g7_whWSP0HCLfKSR_JA7w!{&OE5 z#Harap70WTZD^ZQBVRy+f0HOF`Vxin;wt2G#HU4L2T|%>A@X*RiFmJg!Zla^?70{j zgw<`62zq)pgBFeE_~MU&)Jv%O(bm~BjBqNhIRa}k$d>V=W#Dv!k~E3j1bjOIAi~5G zEbQKf|67orLxDhz36)M`^cPiGKAnV>W+Sr%2#DdREmjp%GzN=+g;>b6^#9WdSitO7 zb~T|^=$s9tJs0bm5TVYlif%K_Lor+lv>8G5MrMel8Ar4jWPj{8`;~9TKf{)VaUAcx zHIqBlQIw|P>HK@r(W(%vy(2;723%1z`I4M&=1T=PBQC;ZUA2*8ntc{HmHBwq%~KX$ zX3eTb0$X-h5?e(tD*6x+)~G&gS4X$$aAMg>)g!#c!svEk2v-4*3luiIs+f zP&YU(yi$WFqqdi7E+a%h5v@1oTcV5vGie23ItE0Q55dYuFUn(8RvQ)|()aGGbc5MFHS&j!XSx%j?Kgd#3>@Xx1xrSnTE>~^o{MIK<8B(yD2 zDiSTq9<{HxYT;MgBad2L;>i5zS65JbhfUkMVI?PaAVt{D8lwgcmkq2-3^6+iE zK}qjjE26n7KyAHX)fE18A0AYwHyQb*CM30r=<2Bwc}qx)yqGWun4JJ|P@J^dKoBbP z@y3Vvnm$4B!~0kQ=hqUuM4;qDjTAAzsWRiLtsg3);52;-QZjJr*V3ld0okD@A^&l; zpIfy1(no98$`u$A+}&h5*08vCG=>^$F4tVsyUFP_cx_JP8f_T4b)AqTE)^Y@O;EOQ zE{<;rq<#wl?#EvY`v+6Y_Yb!ANoP+*U6}>{SgWfyKIXN#AnZJ`elL-h-!aT19%9oP zXQ29)Pahu=l&|xoRs1@X-B_7Xh*+q7iT=s>gJR0fN-uWi>#W9o_Gd%pi^tBFlU|Us zCG%&E4Ud6z>*3QY!hNm!^Ms?5S_dl$6mOzylktq848;R?JD{(YbH^=*q;xA6 zYsOe58EO(}!ELaXvs%J_l0RxfWJf%{*AiMH$4S`5dHh-)Lzim*k%I%5an&?YB7F&` zg^=NiPtfUV1H+`MSw+Tb=zm>Lh}Em+7tR*VCM&7fzLMKrh+K=ye5#!+&RLO{>^cV* zvsq+gL8VFNI2yrk`HXCn{n>Cu{goPH0v1Ln5rf4d%x$3*^YieJb%HBS)Fpa!Znu`7 z1prG`5nFGY6WwBM!W<)50FsoFl1C6d<@e5DF46!>ckl^ctIjbVZ9&Y=un0|kP#tCX zF$kdgf`{-}Vl0wsM^0aNl0*NdY$3(-{G}x{E?!fpH1#8h4au8`r40oKTH{NNIJ0`LO7M3=I{AuUiWEV3g)gv`nx497WEcVS!HPabd$)_%b8xkLB2y6QAOj2o&s zQmY6pxsZ0&Gn3Qcd5)9sdd+Iwu`3BJ!R9Y@6veSg4Zn;lG8oI!I(!lF*9))@@!aM{ zyS77vlALy&pNW+0Z0~4r)(a6UkcIYHgXXedgsEIDJT>Y65&nG0v#`jZJ~t)X)iz@Y)5~vuYDF6kTY?YtMkRXM*25jHEB^?J$Ew3l?Y$ z&ykpo8695Lkye+Shw^i<)gOQ7L|!^37NiT@oo0Vf8607gVRvU*I(u50Mtk`a0(`O> z3EncR)5-F~?19|xFU+oi|H)P_itRLB??WS4UP~kha^kcJCyP+4 zrXCA>VynmX%8MsXiWQ)oP>DxbdbCKqV>2z!MG zJSqQhgA!f6{k%dZ?zVk6gCL(S50%B3l6CBYbL%aRJ#kXgu{{1CRVNwYxh4(qQHB zzZjcm~0uRV7A>g|`p5_bGV|I!m-N&v_ zPFRl0IYXa%*B^PW7D!?*Z6tO7ah+610{LixE4I~Lo;)`J%STUkeOYhm#oLc;8o`4M zqk1RpEm%Ow_Q`Lieoq!~)St?_MtGv!|6V&Ru`BGq$uq3PyHgDL{{I8dWnmtaO5ls_ zOXS;LK|C2fo6cr&#-`V=?j8s+0ar+U1Ye1Ouc@$Czuzq}au~}&-&6OGPe7PWSsqc0 zIip2X=JlF`bP$`=>v0F^;36dbx|RW@KrkIYO9#Vs1*w3QTt)E$6Fw_~i8v1&hyL}| zF6Z09=cKCOr5vhxCph}xC@HI+pB6=(G+GYA+G9k*9jcWoGN$BT^>m=`hB7T~5b zArYLQAF=@LjUY8@VEkxT%6J?V?f{+;feB9E!HeAm**$}_7=nD(|z zX7458nzJBNc;y-)Xy=ua``SiD({QF9l5!wgBe=wK$CBT3&qG1?0Np`Vy*Ir#(4bYO z#kklIGO!;Sbyjku_BT<<$pmJ8V=cm{>^`xmECJvKJ!NIQDS10PyPx3GhP1RMj2HH{ z6Zqt%C@Qqf#Ue{0Kr$D;Z6J?NT@9C>xI*W7C>)5=BV*Qtg;bR4A6xMruiX79Z$S$3 z!>X^M{a!rQ)&UkTn&(6t*2XUX)4-HgpD(2u45~&$#ocND4IXXzijd1nqhn(pDkF$I z6ON!$Z!$1qiBn_}g`XN1NZZ=cl8_BNJ_#O)WA0Y_Uni9^Bd`ibevWm>{F8^j^OZ$x z{P2PBYgQJWfB@No$DhX>cJkjk%!I5$(X=qj_8B#zI`zB7F!G)dy9`Tc95#gTtWpqW z+ky<9b6`9`h&>%;iL{iLxbE$!;Ta(r zW`4T#D#zJO3FXGDSWk-|>ti=yLDB3?=uNKKdNk`4MhS_-Y9vF50)j;u5gQGpI=J`p zu=;KxzA`H8>6_y_82)*B3>aVS=jX>Y=NRJE(9p1qAf~dee%u)ioJLKC!Z>-i>InBv z;N#O*3G67XNedwhfiVUUd;Ejwewoqv?{X6&M0)2@t^QBEv~|3hb>Bq(h({dx6l{u$ z<0tiD@8d-%TO$Ld;2u?0U74oKG!<7G?w1Y1XZGq4Hy-~&?^kP^n;dQ@PFUoEI1B-5 z^CT3#7>y1|Q@28-Ak4_G*ucyCPkrS_q+9;e3Y01Y5Pm*}*N+j!uHSM{80m9+VFwE@3{z`Xb;&61RWXSBJ-s1pF~n4+$0{tH9M%Eu278mZ z^&Jg%o&jq950&D#s5LDOL`=fbO7pFFKcMDN$8oB{&z{Vp zd8C<23P%S1t+7ENHru}R_KezP`!z8mxmP`fDsV6|>U1+01*r8e5?>vG6HM*cvlt~5 z%l>9CFtGYIXf~9VotEv(dzWgW+Z%ZNYG5{xod%)Gr2BaOEsXCE)8Bnwb97#ygH~oQ z2B=E5r=5<&`98d}haM?A=>p7JXHSqJgT6RxUwa$=v`VZOIcf&jU}XWk zY)l={(HMD1;>c;Xrm;jpMM}%C4Gb%O(&SB8{l6_`z-YPCr1bjy;zOCk00KMWguRJkQKlZz zAjH8`qm=~^N3M8d2|?V4RlE>5MMfDM4;)!T@25YA|F(16$c3GY=PTnj_m7tkBrw>9 zP+tnK*S?$4Z976kp;nL(wy?De@DH*Q3xcG5buo1`#1EO>1z(;hwgmyQ#Al_>1?e>o z%0J_{v))6vG<|D+g~=4~#cK9%P$e(b0IHAH@^TCAWLpG|9fX^#hzlxeUCEJi%OYNa z@1w)K;_-yTDD!EXU+d>Igu1#;yE5;%7V|mu(j_=`*j?*koJ#;V9KgzBT_{Ki~A#; z!>y_BjUeXqrvjW_8_WY?vvm|1*IoOCn4x9^zET&BEj=LlWhP%bMJN7kQh~IPq_xGF z!Wi2Aj9(8+7Ic2xbxF^Whn+w}lZIOVoIv$3B5}jk?S~$v#?=fZOA0rQI|6 zlHxb|<+VNi)!;YU9qZKqMb@LW-$R&GHov=}M{3WR_*>o+mb+o$SG$PmJ3jlZQ@Zix*71t2~PuB;CU0l`5w=C_9(85(Wy)Zg*Yq zJ@gH=?Eb$YD)weqk55ootmTKFG)=M2Ml$*wP~?)t1X9RialY}+IB z7ox!+WZM5rSvA;J9_-eU%k8*urLKB_)4U_NqWjn>gP zY}6?1&31@*x_ef^N?`g)L`%5p#Q5#$r$)`=#_AYmXUE3(2Vi@eRo!^qlB27g+2^x) zC!&QNldSc;w-}K_e|pqfWQ(%`+3?n@(5^kzPz}$glmA)IU|5>4C5#q zpzl%Z$4A{HR)qs`$!vvNXuMIlP>E9Lq*yTM-` z_&iWLp+T_DSOybCNtv17TRaPiOpYwq9&|?>B8Zv)Ig@F?G?d~tvt$q9 za(avaRX3a-)#*~HaPO0^Z{C<;Ilaxoc43SB0^9AMY~eQw3|gh9r=wwDNWzed{GK8(!)L>WNateY$w_uANO|hzTs$Q*< zv{pt&e|YuuY-SkPOS+Q^>|9qt%X3Ak`OG?KAePc(doinhSbaa$-qwXV+vF9*G5$yyZeU{#R_z<|XwDiupBbs+1h$GC$_2EUgO#yiPa9|QUW#Uh!5xQ`G4cM%gMG92;O z=c>o}oblLK6l3}`{#BgLzpw&>tjBPikiBETH3V;h6@WS?BpprXM6`bCu5YUOA9dff z8K{nAgf$J)#*FomOjf>2D>(8ztwPAD6rP%>q?eRzc>1cBv5$KZl5%M_y667&GZs;b zLT!zi-$}2RNQn!wHx8oA@oo^HDcUACd`&FOaFtRiY^~Qx^>5CKN=4wPNrZb}dMEW%4vc&xb)79Bjcj<=mYuXJESU+_$?a&6kJ?{xn@74JHPpMMZ zExT`b3>#~&SGxZcvy4gJjP((4eT4>id!oWvh=)g2?OfGcA7AfJF{eHm%=txn(rhvg z5ZrA14x%B)hDqlu?!IRpom(a`53M6GXy&mqpqgT{f-4yE+zt<*qEK!U^ZNlNUOwf zY;TJB0@t656nqxpElFiSUMfGeugq?&G4gJwm`mX`tKFBOy(%0&gX=J#L67u!`^D3SOggun^s>iw$#~_V zaqCOZVq=V&(~TJV?4f}yYZ6i}ZuqbCZ9Y#%Wp`qUyuWRKuj!q-ZHSrzWD%lx*HU0@8^9?tmRt(o)iK{H zG%%ljC0CidyeKe&Qo&DUbm7vTwGH2;W?fxiO%l0&oxg>WlmNl}GUJJR((&zePN-wk zo7kXQO}=4<1v{mj`7fbTC?5oEnQG1lc+C99$LW2!iRL$n7o$MT+VHyY?-rEwkC?fQ zedAr(sFcJ;5E=49m!v=HA<#jITYWa!)Wyv{`OTJHdzJxh!}z7=P7%0Ha<^FDRdOE)s>eEBIP}BlkU9S5^b(foA2Q>*h*_C zCJcXCbKvZ_^GlTTPe)?e>b%C5uJ{Y_uL*%h`!CmNYX~|SQ`)=KSGzQu+Z@T(fCPeN zLDTR*@wW4`(u<Be8xeD9d;8I^%fJ<}>{77=IQ)8NV6wG<(2;M9h|Hj9} zRQ)BS!eIIfa&ZDqlL-XjBuQCgXteoVv838bSTQ@0;c_+j;C~mAp|`)#hvGfM(;^lN zQW2R9UuVj#F2wGGx=wjSkTMohw<%N@)(O;Wd~p_cs3f4<^C*`x)qeIhQ@O`M$N>#! zYdWI2RkrE1vacyBZd>IXhwzLM%xw6wKUs1!%bi7&Uu%Z_*oxQ2+L`2?f?H{ewUnyf-ogNcg z(m3smqJCB*8eZ*b4;$Zw<9XvUIO2w$%HV({%88@bk56Fy&1fjdi43wFA2}cIclN5o zz)?Cd^1AI2?a%#eV3z1+2xw{vNCi#kxJp5wH28J&&VAQiPL2f764S|BcsZ%!YX`t_ zNGJo8#t5pexNyMRkJGQCWF6>3qJGOl06VTBfkD@*{G#Qj&jM zK*-O%=YblUL7%B!Wi|K>2FNxWJiAGYdK1_!``J>&0@Zqt#_=ko)T-Y?9Uf5GSOj`? zdJomq31V_)ToX71zv0~6hNxV(6OW$A>%;Zn^#QXBy~i{*c&`;4gA)T5-Jj=6-#q{(K`6xf$5;J z;5hmekao}a=eR5gjrIYkFfWz(^|JEOOzA~I!VC4Oiyc%C+DSTu#`e&NFc;_H5l!8O z7Dcn~athCzBM7ck9a0nOYxnnNseON%1iTQ{!g$V&Uvd z$6qH=ocozVAFIApI7?u}Y(mE}Cxv9inp~c_#Bf)v@qamjNHhEELoFH)*=c{2+j*+Q-kUL)Ri+e9VtFVx!2YuigSt<+s6y?PHUDdu{br~YF% zm{@r0aPo%r_izjMX~}}1;h-jMgSpDr3vdZ>j~!yZkFQDj)UpeLQKy^p4Bx(QRWSJeVzzu%@du`tnN9c*jxF1 zO()FKHNtD}MevS5aqgSX6~!t6_dv!Q?lmWOC%2sJyo@5C(1UTXwsUyA!cpufylU$u ztN#mj&Xd#`)Noy=A2#8DD(xq!^%fIMaZ7^2_X9!bFp|0GUZK9Y-xuN=F%UOn;qsef z@zS=B;pkTh*W(|k`&&(luIIaa3p96r)B)|7XukEqc^MJG%7Mj7V)ybNDlwn2eX4r`PAWonT|l7gBQRS146pin%zR#Vxe_3D zq@PGCaWZ76+Jr5Z4-9y*f-~Ewq)oL;K3s<~hXQoZgqM7g@>SPRWvEbm<&OoGyK7yQ z0QTkZ1543AZS}tp!8jy1Jo^~thwonLFIg_uhjGptdooBel@Qj9f(jRloK+Ld!0;w7 zE=dSMA-x7FT)MTR1y8RTA8XRnRcr+CZM%Zk7-s@bkm5z9C|?O=5qLa5)jh{pXowff zfDYS5++=NqDFCaJXZn-*uFm8pZp>+7{2prB)qDp{@rW3Z4~*3WCFNK#W7m53J8xe7 zz&hBt5?Uev@RW}yU_tVMf_oV6)d)izVfyCA?vL`JDB3aA&Sm5qR_Ff|i~vUBA@AR- zSaW)Th-Gm&X)t-mUEWH6Fxu+UotgTsb{ZlDh=vzZjk=3g2pyv$d)K({?2Fnf1~XPIKLpr zrRU+nIHmMmdM;x!Iw^Goj*o#Zsv-)<7clF&;?@~f)R%H`&BB6brDYTxR{BavGB&a$ z%$nu=$)o~%=zvQVupbg$I=ilqESaS-{PhA{iw`mbOcG>`kO?>nb;l&suT3Zg9U1_8 z$=|Ess9^G@tS({VKb7d+H!P4}@qlxeLIg-cktH0fgu|FxQs%&0I=B!{A4z#l69X<3 zWEJteDfj6?pf7Tt{~^ROF<6<9hN)&Vok^Fgu6DPtvbyAr6w`UIa@`k+RY$~~?NMwC zl?QAcG_4TyqE-xjpwAnrDAXJd#cKZ=%}+&zdF8I4i)+ zsmtqOCH`}x(JT{?O|dgEp^dON0}RdgtnV&Js6~Jvi#kqzfKwUN5U@SJOGjr1FUi*j zf^^(?M`Bo{AML+OZ7*L4O?mm2t=dBI-$3c%w|Xzq=r^APo)c zL#qMmi-iZwP6UN3gVl+*bck{UjBHzoZ_WB=Yw@AEKHIyk*9zAL3by1MBE0}%YgvT6(QfX6dM z7%Mg|x!UPZQ-5>e8J7!5CkYD^-DRb@0X#BJb(YH8lRGC2dF6oXSdeYN_ zM6SKbMjyToEH4Cc90X9%Y^60vj(DmHLCLAy%OAC>hx&hSRLvQIZN}qGgT;R~V;)Qw zLqJw#7$?n}PG7=*^N z$y>SOjRfog*Uf+om2L^{L|LR(&b9j^G6Q}HC!Qy*@-cs7Y+j0B1>lOhkVHHY$>X&_ z^0D|aTHbc&1P5%SrQ%6^M$&KKM*xFIse%>+M3GhZV900MF7QU96`d^uN_)NF)h~BX zvE5~y=8)s6Q1Iyz0ZxB({y{d_s|nmr%$fc*&{Ly)u^upX4Ci&i2pIG?aK`oFvA#4w zs$qb8!6S(PLCYAhRx@;3s+wIS02E2{ovzOyR03oDo}SaJzI~hAvw84AjQn5;vR8k5 z7supbyUS>-GsEVIN?cHTnVx=IC42Ak*WHIuzrr!ST!@H0HbiZFG$T-uEGNp6t38_( z>M1J|C;&>NGr6#gy|`o2M8B_nGn}NWf>Yp6DU$R>itN)TNc-DtVEJ1+J%5mx`$LLU zE?r)gOwm901j|jJ8P)(=Q6h~^G%crG!y&XFu++yK&ZJ`R0jbbeK*@H`ZF=kjX(1@J zVY+81!Fn2~8RA@?-#JDXC@`Jr&jAhBVS#n!wJn;s>V@ndpEY1V#<1Cf@}sZM=ty)m z?g7Ff+8RQ;TK+V@-^Q^SgTpK=b$M3%$#Ln=D&(mb3f1`}sp@ZM^%SahY`(6kE@y>^&Lh#M`%df*c=nqX7jnUeHjLvJX_^R#)x1RTfPlcLQRzNqeb{CvZ4V2D&*ZU4TG38Rv7C{Y# z)0p7*aM_bAB06>YdB~A6f}!ZX$tejq_-ri zzWxCO{XsxAMs?uA`$(=HODM&{#{u`2`*+?#KcSx4`T0VFh|ktm{cc?qm2Mskbs8=> zpLI6J0Mg*~$SJt8`>(AO3PWk)+zwQpj(gscH;{p1!SWJH!X0l10P+b{?UC0IWaw~w z*RV%kN{a$|GGzM_`xjd4tN^z$r-N;Gz?1Nr<7jgg;u-E1{FnY#7rfvb682s2weYGCDIeFhf~jRAXGq5rcIgn+eto1F~jqXfYNp#gz6t79$b6B^!_&p0;w zbSKEFo?8&%V#PAst~Zi~9IhHYVW+uCw&AV!h^M*A=}@ui2snrVG$Tznq`XM~0QDD~ z9QWb4kL&E=_@hwZU&W5PcfX62rkLKE&M`^^vq-DWaY}@+N~_(|Nn1W$!6 zG1L2(3+~`02^*KZzDZ}TxA8#rOG8}0r-$MgK|u&JOIvGWCb ziX5&b*xK#Q`Go=h2WFP^GYjt?cR&S$6AF8Eq`a}XxtNowa~2RO6#R+ zjQ}rc=v*d31BDRwo_{k|p!9}-G3-_I_^h-qH190VpVb|MXn{P-k@27gD)4JxA0`qAQ+kXtmce zhxf$V3B5~c4O~Q6e2ouYDpHzup}>@lyz^1e;@`?bKM0c*|JH9rDEqExK>N_fGJgO@K}vdNUzO<3wN}W zd1*c$_AQlYk}`8bySa3%q>fTKO23?~ch4BON53H{k%r9!pmRg6OLG<*6kX8B*$+hL zh|~6pk7_A_OAbpDYUpg>(et{DXoTdZeeU0%!Msw%~0^$;`+qshw{G#={an7})Oc-4xuR+;BGlGSy zXC_^Rt(&7$i4h2gDrG-6!`S>}NuToJ?M!zPYJIJN1YdzS_Rkaq%sSidyxpeL(mF9@ z;E@UoK70LY;92AXV&Fl$)PD2*X#WOM61CXP%EgT{#i=!Oh{zQ?%17KBsfkAi^3xTU z3i&boF>1UU=@xT~R(fW+tXc0bcuz^fV~_IAuAiUveOMOWE#TeQF`|)!$iZ3Y%N*p2 z7WoP#{}uVq`|8;xu3Tun`t}(rDB7gO5%d*q+{D8!#I}H4M6ihHDRAT62nH<5^T6BU zj~#NI0b4mv|A%&caX$XvFu+-b|0z7Gu$nFBymf;2$(g7)h$Yp9O78lN-H9@Xv{e!U zg1L9!YjFk`Db4$BAd`1pP71Q8yAk7~D~e5I7Tpfok>DwLFHEb~Y$PE=?D7w%3CY3v z{`%ogqBF9 zBB6_ReWlXqM5@!O3OSL(Kt!-EYb$BQ-@g)sjy|yME!n*C;rBty(5;jDMv1M#xBlw& zy3#xvDUP1-l${$FCu)y<2TCLt`pxG?XmUUclt@m72ka!i4>NSz#m^+%Q4p}PLfr%= z2&|ah5FE(5AZL-D2oH+PbZPB+Q=e4SdB3P4vF53bN@7Gx%%?`Q!ug<80cw)d+{!@qyIF${HeRH zXBJ*!`)AGJfpJDX?=|zTeALt5^?7Hc)*%%}yk2mJJtLL*rT+ZCr9Fhna|`y~)|klE z*x%8_K3Ll0_)ZLAf@=%wcP|eup|LiAG`%tEs_4%P_I z8ud(@pTWVI7uN);EPD}bW8I^T7UxYyg1Jy5FCJ=zRa7lV0*4Ryt7?RlfXgtdv@=p! zC^Wz9IbvB_m1*iJ9D)^D6r90`m>|%fT@Be1!alkvq$=DY0o9q%r}HsqY6jynHlve; zu7%mCl;N|tuX=y_^#fKFCFk2%Kwv09n%=d;VxZFrx|xb*kX$t1G+VNzFD`REsOA;A zYn@HNozw16htkx$2tFm2(IC5Bz?#-D<;a*tPd4Qv1_5D&lkEB4uY7I=LjDE2@_3@2 zSE0Cv7cS&Wkz&+vDUF{Dy5HgPWoQuI_VlDXs*^{6>n*yA)HNeaZpfEnn~-;tK4{h| zdJ5Piq(t15*i*k~QrzIZ9>;+FV0-xADANmZX|D|P0^-tIoj*GIuZB2B8V$sTOrLR= zHZ2;Cpacm!vy>wK4#}eE^C{VOxJ^&~2+7puFeF$HqhCTYy0anpbWxL)t#~(+b2*!f zNH6N6Q$gx0K48YvEe_N#d(wT^P7TRM1V0-dW<~?&9Nm@RH{tIThfa z2jNH75B`A3GG6yse9Cl!*HC zQ1g3x&Iogfod>>Z^~+JM1dFGX{8eEHbZnWjU5~y^h*DkJUidB84Wbjo_+`aR#aMcN zH5?utxpTEP)Q;-(wx_qQB!X^AQpRZ0QQxTzI=;^t6yNG%*F5{fNWFtH{DPUg60rv2kWLmFDtH8@MF zg$dN{iU|V;t1fC0y1Ed|PJGk#?S`NE>(9!@&UPxyYtpok^sWkOr<+LdUMNW6i}35A z*^qOsYlu-t6P7c=qI4s_TpRi4Z}cZu*5%7&uFQl9wnFqMn5^q#GcQ3>d%)vJRV`xdaL zZSN{mk$<)85laN$t*t2;{v<86F<$@6RWZ%BZ8RmEOyJw1gm$sgzg*AUUB00H>%aDF4g^r ztU3s?tT!|S{QSlHDWa{Yg7iWhUspnxJB;q5WMhYoo7 zM|m1-EwYa^{LuJOWH*fMbPk3f2Uk^{` z%_av^LVIR+QH#gfPBveP@OCtqVxeKikn)4=#ODk$HBtiAfRe{r3*I}iMbF=WsQ2m+ z>A63nI%B_p%YJOcCeBtN?DI#&JF}uGi=IRl+@tmQZyJ{hQLADrt+UyXb@wRH zrw8*l7T911g-Z4#0ff>brhSONuIo^DU(!lDZ*9iid*N4Mo4o_{>XSus>p~ax6b@_fvsTthda~ohgKX< zbl=E^6%X8uh59#Yp9)cEmzQIPmLrI9vOBn6U>URNQs~nJ`cD{H8Z$l|( z0Br4#KSV{+$H9e-#7&l6j1%)1O+Hme0>waEsOPW@d%Ug9r^N*$@U9JJ++Qtuj6rcC zbH1R)bT8w?XG{L7uC`^epaEmaK{oU8Vp-~08a$CXlB+yCGJdDHEBV*9pJqq|GI({S zCXKeMgE`OSaRaaBo!FaCyAZ-asxG+6m|wg6eg?wz&ztrHk#6?Jhz(t%Q?X2<@u4E< z0u+=i!PSrLe6AZ%wszBaMeg`U{?L{VA}7|~H9YQN8_^!oy3{Nc7-%?M8|)j`Xue(X z;-c~pxOgB@&r?Y9ZRdP%|jqKb8y@kJ_XRSSs9B!%Yj;o9MI{gQ z7gYdvgSVXraT7uBM@@@TNCfT>c{qPVLNsf=Xx}{`pBzYs-BQi?YgM#5X!{4-qlLvl zKf*vT5#}Zee)Ov%$1AS@8>QaEY@3W_)%SL#Ho(p=pB8iDEj3c~`k85JeWJV=Ouj@i zQ+mr1ZhBh{!QITb2>2ujOvo+zL|^Cv=7Mg!{g2hIsd11fMD}I9ns)sJAkVH0L}T5_ z*;J2PIW3q^4oCy_Z{qdN8;`twYa<3%eT|r_U5St6-Ot@mxIUM^ zo`XZ5XP4d1*gQAMPfw%aPYN4&nJh`fV&b;-K~7$)8~83kZ;87jI@|Z zql~m$!3QL_DRwKdDZVL1LTkelD_3=1!g;#eHHSshlY{ljVvZwrTdzB%Xq$+vsP}bf zpX$O_FPcii4~(z$s0{0>o$6S-z; zs-En9Lx|t)0bz+TsG>9>s9)K%ssBbH5d!fuHzc^b=#_r5-ABzs|HbmaU%~p@GTu7( zOxbCWSF3BmNOu_w&+&QHEPNECc>@K0eS?4%RiyIZfCcPTnGOB%*`M#flE8!E{H^{r zyzc&d;Qi&Or`2)6WYm8sNBQb5+*wh#-y9M=Bu{xlZ2ov^ z^Nn^;ebRSldt5T#l5aVcX=lZcp*1*ox!j=JsdvZTbLK;oR$laZ@io=t&7O}bsAw;5 zViLnilaFP)3&_=b1JOGOI*Gr@KQFAELLXD^S$9v}H91(_J3S;h9b41@s> z9AY~ayPji@aP%X`Wb4d_SG)uTz4XA8i(ZiNNi#aO8Hwss8T$C)34n8fE0%s(v!dca zxh!l*t%CBD)C8Ow?y&o;^cyzRVZ`D$8`9Rok#KBj`T2TqWOa< zGmt#132&2=R=>clpFqv0(Z-!;g7YJFlp@!CQq@39RfM8d=6LKomk-~DZ?Hj}Ha2c6 z>ma=HULiHz=CMoXVi4P;t~&x3Zs%q^M-*~wmk^sto9W19W7j59!H7__$dw1d_K&h; zceB!;b5Y#AL|@$wpQe_&tnwcwf$TTALnVCHu5hQ^sT80dyiT5+#rWzFRHDrU^Ng8u zrpcYP6N0+}CSLKOZ5~iuM}`=rw=DMmumCSkWA^L6I1T-?$wdigW?iyqmgw1#-JS+f zHRIL((m;ZA-GHs7NP=|DfUR_6>I^5$K&S0_gaLm1xCDB$h()*|NtEPYbO4Du+T;DW zbloT%c?&+hsI4Vr#yyV-m$Yk^9+`wE<{lgI%<-7aC#}K{)h0_*|pY484XZSPz3;KN2T1xU5@LVxD%J$kI{l^t1Zl?DT+t9Hqm z2Z71V)en{t@iFm=s(7AT!tLm0PBQm`hm`s{Jz4$R41IGOF3Y75RGF+Y=Y@sbtwU!o zIwxX%U!Wqz){oHDx<^98BC@wHy@;RiFTJR7ws_y;;&m~uH&4b;@zoU>JFBxOq#C z4xLyr*yscAoRM%-D_E?HLCP`2I}w!XX(JUQXt+7Au$m^!tW&WMV#07cytObztKZ7I zvc1^gwDn|aH7Hh8{rO6W14~6vYUO-IP=+h|vOA?vm_L0&guXbWSVy$7k=LC#fk@*J zS#IC@`e4Pz;{2mEX2f~A=9``HhFKI(U|D3wY1+txgi^Lj><>SoN8Q^~jO$cQtJ zG`sw9=}ZCXqyu<@3Cw6T+w<}1=6!~d(1F|YI^le2=z$l97W>l(Y39<#O5w#>hR((+ zKBybWmTE15KvRVYAHoDj$wns%js}wOk+GP&Cq$Oh{-pQ4yFXx z6XOh`gS_ZBkc0@noJu8(gwqffM2Yjsw2uVddXaA+AAkoV6mwo0WAcINO~2&NiZ0(r zuW@Wdm9xmFV%4)q5tHlR)7<}?*!0A*7m<91G|Q}Rph8NIKDf2kefnhGd7emrz7@7U zSLdnOQYvwlcBa8d7q({3Al5n^{$(kVAl?%uw>s8HYlfdD0)rk($0!i1F=UIRuP_~& zAfpDIx;mDMzt3MHGK?i!4kR~%eLj$8r7aB4 zepnjDvbIGr>K@ji+0rP!15}@3MZ^#T$*qedRP|vmL1V$Y&)P*CzJ!2bai{n@<%d@7 z5rgoiJJb+E^EwlI|2b7!EM}^l8S0-Jfkw$(1j()>vUkShJp`Ff;C{b0QM-PJG9nQC z1H?&#qedkQGbSWni1YI_iFXxNMe%3FvhUsSQPIoN(btUORx*pj^RG)U3jcD7UpBA+ z$)VXoVij*iN-HJv~_Ni}9IYXGDt}BImf8ufV1ep)Es;1GW6GR$a z`O0a#Fa@d!9Fhy_fenfDJUH3A6!wp{wAXOaH+^p@Wo`{kxI(tRP>mnFqZ+@chCqtB zrPMdm@#FHGD>5%Z4s;qbOk2fhl-L>+JqQhxKEDGmfYgtO4NIai9#BqOXI$!}v)nL= zeqs>nP717s=?bG?5iE9ZsB39e2wjWJ{qmcRfk@~9vQ|Jb4MDj*Y}TZzG~&YtQhHT> zN9?LmHYwaD=p7Y^$QY&2SJ3mn*|qLlzsKnPSSE2|zFP?zB5LI{<9E?Rpm=y+u;7vQ z#Q?04TJ$F8W5DSu8HQc)Ntqq>*6a(OecmC98eNEdhgI!rqOiiFu(;QGzX8$vgS5Ll zw#1&|6M_paWwsDZI#Nhw9$jH~-5`bvZUEM$#uwPQ4{msX#~i{mIVd-N=cbDTubz6` zP{zR9tEoJKtZT&HJiG20Af!Sj+oBz*Om(u+!S0ouslq6>T?c2WBh1;gbOHN!d*vZe zs9Er?=))@)HQ4jcTC>~brDHxd{7$wS4NCJackXA+0@l7!Ag6DNB|!bmX|u`fYK|0T z9X;(E$Lj(ucON?Kqq!BF&g)(bRpx@qLdI+u6-->o(sYNZ*e|U${etU#3DpoGVbZzN zHg309%_#u}EzbvE=M#T>ii3ag6uL-0j#8_wl1%Ro;nC6?{W2b6O@J|N{aGULs3UKj zFm885MWXCLo4Iq>+YX=#z;lC(HY|ifdU^={M6!|EsEajPh>F`>EPc(GvCIj-*JlR5 z^GM?O1l3x|jP>#?#dNn?jQ|4n=yk=B zt>Qm987hk-*m;}U&0jEX%R%SkyGNBr(EnxoG%IN#m+NJgRS0I)zLKYd&1bmu`P(~Z_J~0y|)mnWdq)=iN z*q>yGsjkD)-r6vut>szBPt!88={Bsr2}badXXjC8-Jc6%!H0756yr#+L8FSR=y&w< ztyEsyag=iHY+kwg#2$O1b?w{sO18j1pr!XMxig)UuRa>R%s7Y6r>kqhwX~^xp5*nn z;sget$&}49H1H*8<{9G*%R1whfOCrkolD|AKTNZvTYIAYt664MBT_^KzPFaqv6Oa& z4LjYo()cyOgF@g_EEdZ|nApk-F{wW`RyBwO$-=^O&@{jx;_>7E!f4n+5pzz?$m`@Y z8CA`%_Z=kkt%lVY&EOghO}yy+>QIfUHrkyCz9@=aN)p$J<9SEqJP9oD%cYsR!q zdU(O;S0>GsFjYTw$od;kioK@IpveJlAGnPd9ULm|joRkLM6AQuB1^ae+iIHQKLc;IES)Y zN_QN)+2muUB=0|%E;*8AA^6DN6`ViSe=KD{GyHu24y7tR^f-td{!pAxkGONf8}$}s z{HAQ&!xN(5;H_4D1Cr-~I6<-#k}T}*`N_;j?v-kcsc+#a>C`YX_X=}X<*#8{>(-#) zbd^muCM>t#-8eppp$rMfQZ|_wU!u5SdDX0S$QRefX%9vzoXFfe&)hte7Ay!@!x6UoLwW(5~#y7v9CyA!U)sctLtAHtX%jgG1XNRUI=2V4jS znp{w72ZOPzx9gy~faoTk6hWfpcAb+J^p*R@5I`LmKs~Dj^{X*E7!oSj?=17EvtcTYn((WxRUv?AcS*ZntzlOVkc^Y8wZYk5=IvYi z_cd4%nO~1P;##?GbE>XL+NQUwk85zc93!TA^em)dIOg*~Q>7l=;cqMwYmoE^j0J-; zAakGR7riZ+sUpJpTz=tq|2j&P=}#Ks7@WEbT@X|p?ZZB8&Vz3otT}%B3T6NsT~w2A z%S$z2y-BcAMrcOFuJf$ai6Gq<&%N)lxD;>e^3EG;=+3)ya}w;6(i{*XNg;6}G!1s- zj-27nrN-g5%0xq1@S)ut7F5%(LfFktG!!yD6-MlaTOC0zR$L-g@Mj7;MkrM{CQ)uQ z0)cJT>I)`qjtL0ckhNA) z8#X$uL4&f#1xKC<3PZWh{C%v^By8duzQg+pJP=k{3j6*h7A@6)wtoe)qlt_y1Ke&B zMUNxaA{3>mb)o9+pkn_mOx71T75lkZOzpmNcZ)afd2Pa{uQO#=g9m`I#WvZ>%77rF zdGqtH)P8qVG&|nmY#=GY%?Z(-+bHF;B4NM%&_{wKZzSTbV}(p#JnCvCDuxQAz^xs9 zUFZ#FbxXnh-4hD5ncy?aCsv`vI_$BJ-jGAhz+zjuOz1#s)B`w^a~YC!Uo=3v_lJ!2 zo3{sYppPdh2OCjU^H%XK>p~jGH*0weLzTx~5KUQ0CTc4@)@VWO7DmFV)pBuPeDyOV z8`k{Du~m6D_Af~ZzTV{_8Z}*5BK3pm@S5lIx@$9!GXQZj15JiPyszbQgZzo;yM^R=C$k8DZ*;TSB&!zutm1 zXLpW7=X{2)qii)^3@+eu-S|4pX83qX#q$Mjs)rfc#*xWZc*B~NW;d%7Sc6t__iVX< z)GBC>kM_Es(8me|9q?!i(%JE4^^3ZylNoHm5wRPe(-D5-P|?A@Xf({f(|4DTu~IqE zcyv&mx&@ll`asw*g{nx9;ea!SpV3Xm4^$pi85Rh{bbm++J%~gmKKDA%zl>-iMo>GS zh?(&{@oI)fcnQj=-v4V*mYl;s=X;65WBmkWC2~i?q~RNZ3qHyWB_xqEre2svwV(V&%&Y0oG zTYM#WaksoR47kW&w7fezycCoGBypS2N-D4Vh1A;jw;x~tf$J=7HZY>z^_gxH?s29q zL0hd(Zocw0%^>m7K~Fj-L49qvw>Bx$P|4kncq^*Tg2(T+uddD^MMHMeHFBoy4~?vG z(YvBY4w!|`5kMoLO}FFC++tFeu<(?Oboz!)%#QwH?As$_2y)Uh6RlCg?p^hw;||C6 zY-?dMQ>V?Y#}PM&ceK~j!Dq%#?@)#WD{Om7c^?sYreLO%r}*T{)^WAg#J%FNb9Xch zofI(u!?#c-)h@=LM|p%&9vB>X(z_zOZ9yIfL{ZAiLFQcbCPz_$a@ig(QmAmdY7_%5 z@7xDQ_|`PZtOA`P;^Q=9f?DMn@hbX%;t7WAq`zNr?z)IEHeC@BN@*h;E2!w2E4?B} zX(HR46zyr8a!yTsLhLQ@j;PvCn}($jXQs!oTDL$KtFo|@uLui09Tts@Y8y$d{Q>nr z{5Dn14Mkapwb%(-cPDH2P5N}I?*}6mI9v_msU`VdLZAJ`v(KYSQSo=?z7efwd9-Xw zdf^U}B7J+qfil$^b=Kc!-d0*u`nr!xZjUq;Fct<)f1lr-;yL+QkFBMFYmLd*vv?M$ zI6}azo5_L(UNZDf4EkX~^eHVy^BQEKWW%XwE)raiKGXVVD*aByr~Yd`Xox92uO2u^ zr#F6j`EhyJ)hF;k)j=3=~3yT^1!cWZ13X;4ncA*~;|C z0$q@TLsXbePSN)(Hh0n>H_qK!0Vlf?AJNQV;lS<>Edow#Jsy~ec=~Dq%kq&#=W4y$ z&LLTDbEpISMM1@Gk=TLEwkVZ@+r(D~IdGU$TcvHLL$Kg+kK*7LbSM-F3Wk4lvl-`b zemE)#7UbY~`9rx5{*G`6gNtc(Y#-+abK|YCocdn*^xB)C4U)6|x+K?n@ch?%FA>pcl zvJw`n6V|ldPdaX&$X`ljUP;)O1iI95-Q($q&I7(eowaZIrZ z_HH0O5&UruOk3#{Yn0-0%s64_v`zcD27gHvl(=;qYmAPoCCO4`Bh3@Jmn zQA&ow_^Bhi(*q^#xJ$qQBi9K$Bh7#vNtJpAa!UJ3q>=kW75A&%2ky<_OYYRTDlr>z zZzCh<%wdsORN4@Dql9!?EtSZ|PNJxEGtJCTEsTMw_u7dI3xZQF9}wScpx(Yc&{2?C z&1S6y-<_49V^hDO6(QW#5TT>vieVcT=nK4wKRorvrpf}=Abcbx>Ul@RKx-=2`FPLa z*l9;R{d-ntmn9xwgs~BU@T$q!R}RXph&`Js^{?pXl{VUJMp9{0oPLn^#kF#7$0r*L zejil;ZNd>&EO~fA^3#J-eSXER6XM5H;OB<*8EG9Q0#77pyd8W9>j$=RG~g3#w!ujy z9OWIdw>ifsy(~^9n%)Qig~gEFi3St39Y~ZB7AZ%4ju4#~Ls@qSHs7t@#CiNU>l%~c zbVq9>yM59GZ!&Qw)UXA6k2{8J6! zDkCt<8_UF;#5UtOgpLMWid1?0+NUOuumxB!Sxn*&XE zN(rU4lUD$qSTRb`vK;4eeHmXA8js%!4cxydH1z?iQ_PWAAd+&pbQagm)HW>oQN8Xy z4WZume8f94+G|wi()XD`fK|&d{(IvHZTM)e`>p!fai<}Q6{7`lMWKxZCpJpj3X&2M zNF36Ok~BAXTpr^$M#2f!+giGKDl2YXj-w^6vao1TnVTw`)5i*;RGI!MQ!Y!>F@6{M zi7YiCXQl`~RHdk|K6PjBqq;f9pbH*|r>eZN(J=NKDX~pdtj1$fetmB0<@DrmWQ$}Z zDftF5rw0A@{Bv&#L8dp6xFZ`C?K87aOBrFcx~plgEUg=@?hpsxb58RQlU}j&J;88O zU5=IcnbO|ZjEA)w3=Nlb;NWD@gRSWLBl&fSvuyp`bp2jyu8%cJNSu~EnMsPrl$$}n zXx>g6D8XSSvvepjlC4OdPipLvVqoKEhZlOtNLf$82}OuGnWp(g-d3`qnFR!!w_0B7 zj_K2)*^?wb1rG((VRnEeN_~>CrU`=M%G#k*tdY41!bFN2*C;FwM`Y%Cq$mLkH?PTWc%38}oSk%Ii`3F7J^ra@J;XeQD;UZzx~F3@fpJ4Kp5% zbm)j6(phdcWMpBllQVn21Eu4x76;BqMWAM%9^F`MF;?7;=(W!7tp+#8d`_5sgbx^k zK#mO+B3*2#o9{KLX?I_D1La0F5*KZ^ujfszaIvgk)RtnW<1IYB`Gs%BO@22GgyyGh z4Qn*L`6nIk!KYcwbMQ65$YL>@_1-yY6i{9>#Fh)7^y82BN4m-9S;{Y1A@cCcE4S8L ze(9h!k*N-&PucQJN-?aidz%!?c2>mly?WbOrCl?L*qR~G`W%NtwjQ-FxX&sbs8N3Z z#nnNBH-R}xroTkRIhqSGnmD*8R|y8-FNsRP(*T+F?X6qBOg&JeB+V2acc!=oc^huc zMzJ?}{%a_RAl()7MW@jjWcT<8+m1rSdw9Uou)|W{5=a@9dDvU{Reeqyzq2AgODdrZ zhkB-y;i0xxsF4$K)dXp`(`2N&uBt&jq9|`J6yn`93((g^D(@m6+h1u$ z8Af3Kf+;1ZpU>=xVFrsvYr9n}^c%pMI@w|9xRdHB0)ZKrVBTbHvUMcTjsKcLse9Th z3(H^KTb_I4!t~K{S|GxPro9v`HHI_S({<#eSo{4I&A90)_3W$OvS9M^VbqK!PhqQ zT&HUjg==sc44jVt`oKUD+3SROOCh__PF_S zhAeShH-m9sT-(i_5bv6o;b;-i2KZy%H9c@}da_BG`C0^(vT@KoK!liQN#x8$lqP;i z$eC%PY_;{!)0u5GV*y5URzE@4u4!roUYhunzCvCrO0SWrY{UuGclG6Fz9rmn`9H0y zz^QqvskW5-m~|S7>=e8RA7;zNfx*BtGQo(QNc>9Q zW1=XMIdX(LePasc%~d(|00X&E_c}>OL#-f8N~v&X$M|5`qxbF`+O#VgJoMe0wv{7A ztVUreL- zs1|2V;evLBWOj0P7>zZxBycKT)50MI5_RhqW7s(usrZy(O#~&mm4-L`SV$F}9u}*VwR`AVt3=wJjc#E}L2$pTkbW^Lue_YP;Yv5L1IyVo~O&3i7io07iv1lDXZKx%~7 zl(|gmXjqq_Il$^qfAq?H@BSp5)*tK1wNuiL0&1Sk2u?cg*B^soLjYGo+u^oPj zESuHy4Um1m%Me@rWeHhkpvBMhoT3JYn)Wnjp_p-z`#p2sV4^~OInikZ2cPC3PAG7( zw`_ZHCzAJDtNdu&9Gy%%Oh(0_fjqO>@b-fI;k7e=M*ECbZ(bk%ypN~Np3i6N@kNz<-wtmBi^a9mvxcY*i z^)q?UHz#Kc*;#G4dkZRhgA`;dI5W9K-tpeVDC!1Q9Xx`88Gs}Epd7IknfWf=&LesON&$H~Y=;C5zshZb|UYM z)H`@myrOS=Bs+-?hkR>GA%PB1dW<&QUW|Cj$g|=9OGdtFjXu*ee#J9>zr-j(9yLnf zf@an|tRcJmUVB2IHl<*d!**K&uOTMbXrvm-xaC@1D<94dkzjU2BqY{D ze{*%&f>EgxtXQR-Qd9@&_j=pa^cc08_sX{A1LV?0zBq{`0ce1;cW#y3E+{ zvVVLEUbFGMtls`NH3*~bd-cy%g{%e=MfogySH__0;=@9ov zM}vbcJ7~GH!$orRE5#0RZ*1Mp+$!bl-dTi-S>%dYe6$N|rG_0(q0BQuz22}~QRx2y zRA?x`R~twBqmONoe@Ac%fJX}G*e834_G#rvZ&|lOJ1nyOMRM+TI@-%7VSTq&kz21i zx^v}BtM&6wkd=sUB=b|ar)=KfBD#`#gb`57DX#{>Mf2K+~}`{PhTw^!HULofil z8&W?@w@TST?H~rl%2s;gatQE99((rVPzIb=h|GL@V^k@Lb?=gDQK=)!xL%H>r z*TjBy!3~!pOT3uqH;SDNCJ(v2i`Ex=R=5{BlA<;J<_LZ}#{h6o0et=o9epv1Z!jjm z$H(9H#h5Qs`n*C`yQuDa=+@Tfg7AKh?Uq=9s`xwY)A{NrTfHdReoC11eYmTE5ZQiOnDm2CIqUJB z%-MS?gEBVe+hACcrb22jA9A+iu&0Hpzmvi2*x765`9ID7d}jII&nyK4kbXc_30Tb) zMTzwB#yRSD|56~6!9SsTp;W5h zX!BnF?^(d;lF$czAehuU>bMo0<8_p`GtpW~91=i@A3IwCcFa|mj9qc{m2K~><3C>g zSA*H$PMoA608`9h2k_bIx48@HoOw1H7!RSgIfkq@n8{Z(VAi$k^eeVG*IgN%E4SVV zoq?}LhWUhYDtZoKjhe6v3w8|^y-=3fx3a)Fwyph^IDRC`ra#b^=vxveB2bSSi^t~i3uW$KCT!&@#Nzv-zEW^ zULK>zRuJkpEVUkT2|X(Co<;0w9!@q~NO|17JL96g8 zp1`E*FrK8}pXUv6HmbxdT=Nzn&ef1UWbFn<;)pBY!%BS6%P*M74%)=B=9^pjFL=U*<=!9rs%9$+xi!uI|Q+0J4o_I~fFklx?| zTvq@`Wz)gU`2RKbE#OSH@&Bb1>7>M44waJ1sR+qoN>U-6P>z)tX2_cJCVD$a@s@IC z^_G;wma{o6Ipr`Thhc0ahMA3FW1Ib-k?Q^ZuK)k~cfIvq*S+Vtzvu7wbARvqd7d%p zMQ}bip#LzZRv0^eS1@A8HG^~7hXz}=at)Y4wOIfpSh#JEA&+O7f&h)BK{Zq4|Da&3 zs{~L$HK#AUx+BFuxX`xFeNDmYUgP-Qd$f-w-V3Pqw4Fm9d3ttA`bQeXS9)~K< z1sO~-AIR6mIU?dvckVp;F}BAk?9AxSm&YZ3TJvWmSzPOVjY(U)G9-3x<#_@0qkG$U z3!Cma^zMzI>_fydFM9pk`%Ogw>DP`Xr)7zP)J7cDq)+~{;uk*)MK~JhW?&%4^JP~2 zWp3?hX2@yB-o3Y+|J}PVYc5RYpXm^sOZ9OjrkHr5EjtU0Y zYS>REkw`^e?ND2HJ!W(;q{d8?Q(Ihvm{W0L%S1cN|~0!t%PcF9(P|K<&{zDUQrbwGHB2V$#-SRqy?8 z$u0!)SO7iy+D{m;HuzXLWIHc(5H^oyeA}EhNPKJgD0bf;VKnL{tgHE=gT6lj#ej`u zt9@|cBh;N}Adf<)n5-(=jw;Mt9w`t{ol8UodDQ3FS zh}bb>S=`=~L870>BWd3FOQ!O7)Mf?L76p{;gQ5s$Xq2)0_>-95uxP0qUfK^7uP44e z$5vwbg3#~p9|L4heILBua)DdD%{~6!G&736#YM#@;)y9U zFG*`k58nx zzG=Q`me?=0xopkzst=f^_}9(kU2^H)Vmgl2wSFU7UWkv2%M5~&%Xe-o^6K$(yxIQV zQf(M6`*c&>597^1Ag_YnTS`i;>q=VyJeL?Y#!UR_Rg*VFg3U2K)eQUGDWHRG8d587 za;E0=yk>7=jMxL-e|;ziz$Xlw=V07V>$Kv<3aEn_qJRu#SBw-naGOuG>>Gg@pGFLR zg*xCo{|fQ+-I6@(pXw-~ixntyB8yQ#v&sJyewE03)u6fJclK7~*zzvko>k)UVeih+ zlII!=i-Lma$%!Ua47YE$?f-O%Vj)(vKbea78BWH!`?mt+LVd#BnYlfe@Zh40lEUs8 zslHjo+MunO8nyQ+yaT`!XAjE+JXzo4kX0p;nHp7;up!yUW3$R7Tn$|0_wfS-I~XEj zT{hn-m3MY8zO&u9_}xmdQO0xp;+EGxc%}v%0dLb`2ei)gtz%P{m{A8Di!A?eb>Q&N zpHLS-o~u`zFl8S<;_f=-Tp!`QW`v51!pufj;gH;;C7ve6AEX(T$2UlfXk7f_;6JEX zqqO-_1cjtw1e|=AJ_rIeFi^*Of3TEq@Vbj#cRbv}v)a>oFWxzDwMK;?q(g)=`;Iu( z)@Rzt#&lX8DLsNQ1>WN=7dZ4x_ejXxg#Y?#wF>&Jn+N_(@-o09K}6HnJ)3?G-VG38 zApVIo<8H1o$&Ea0unl;N3*{SW)#ugq*a}WnOzgnigRLJtj8S2{^~W>&|0$NaK$-#} zcAK~a=~ePI1b@HSrMqjjg8`)83d#8^Q#RSUmpQ{4YEn~~`tf=9nyPi>b6*0>=SjAA znvtt1A*!pbXWt}`_bVeB<8DUI<_AGfHGS@Y5eOxctUB9ryD($o2Adx9xnZWWW0wtP2r&aGq&28*BC^cgiX3d;`M-`C@Q!I>IVPtI++L>W zM+p|%!oCb=*W3?dZwl2uur6$|5L8<`9{g6`ad_Pj+sLB&zF^UD$B!N!_QgPaZ0!AW zu?MQg?j~h9%vn@1H1{|Ex&9hG1=OO_tvY4P*l|zzq{YC(T2Rlovh@h0ftYq)Iv3(Nj-OQ?Iu^(z!~Sooxo?PJtcgmcIqq8*nfHMYPJv7-tk;89u9mK z==eAS z=L=H_B4a_m^Dt++?^-@hut(JztMXN+2 zrFe=THUED1ATKbk~TD4p)=&vP};u!#XWG|b#T z0l2X;;B&P7#x_6ec6i#U&s%xx4(4KV28g?B{goS=4qMzpiG@7`-XKG&{(@Sgl%D=6 zUhkrv-M^=CnveE&=ruOp)KGb}m*c43DzwuJ$>B>*tw&TXvuT5;9vNBjz zNKAWcQDdxsnyCUA>I}ek0}rCh`wmr8f&$?VIWT_YYZ<_&&mDXjv zzY6+V*zSRsq3q?ht^loFB6M-&_c6A+SJ>5%qybiE6)opM{m=s06^X?MA|M(DPz zfd@pIX%1mM%69r$!H=B`ICkz9tjFD;miuf#F^WqrlK@ysC9faLSJ)Ukhy8K;H{Bh zitExFox>;;vum#wx0Cfp;B!%`wDyNftE?4%3VZ-=yzxGV=g$93ZT-=o3&oHRtN?uI=4%UvwuGAQ_DUH2t-`C*tsXu(8_#5Dg{YjLoe|WxdaVC53(ZYcbdvs^U&Gn7WHqrcto{ii)b#I#j zDR@^`HNA}-4rg|CRbL^wuN~IDCNt-j{eW~u0)sgjwEYS7ZLv+Gb3g2;Usf)`b_?tu z_KBP(VPAc?0!{o5zoz1nX$DOdOXO)@Soa@K2~GkO6*Ynp{RR1oC56}>`*p}O0ws3) zKVDUv?u~kRbH^djb9E;wq#7TEGHrQMnaw$mdz3X0WHQDydT1Tye)ZA{^B?YRS z|9Vwq@MXs_3rnqXsoFJpomxj7JC9P+-@7J~9~th;aNRR|;E~6b_G;++eQK5?FCKb~ zB_<@rtuF2>u&9dJ_#Z>g90yvTMENjRUyGHDDIs=uKgY1w=d-maNRqK@b;m;!`BmbT zzIL}ThXPO4aUXtS21vXQsZ}>+Ccj7uKMdZKGC1IeB^l5PUVilJeEseEv-24e)@aY; zj-9?(&poc0nGPQ^{l1Ldts2YrZhz=qrX9z=JXsepvwf!`DKpzIGxy0THR~0$UW)(q zkH7|+2v+$3ezWD365>~wqi$`DKzR+lYRyY$9|n2}E|#mvgv6)-Sp0W-pLIe?*{)a^ z@q13+Y@ik!R{p$fla&D~rXH0XUXM!s_&uCp_ng-(WlglW-b>qz^6*H!dFR6E7a z_1)8U)7`Ci(50oNO2<lo738X+8JfGd7E$6nc&2tY_LcZBr!=)zmw8-5g9?_fGv#o$LcJ^9fzV zsX?Q@spx#U7wr2h5gDh3fbZ}o(;Co~cLr-_XG|v`R?MhKEKkKeB&aksS+c2C5$!2C zl=*58)i3DMrJl$Cjz!rg5lL~$tB+?UGs`@%B|kDv)3er{Ps#E-Ttyi3`?oQ@g10dS zXM+J%CtEE@-a+T(<(YQ0E2c`*3%)APPiHS25s$DN)snbZijH=@pjmmS{oCNwGLop&A#%>^siS53|vrb-C1Q-#HvJjz46GOSg{#_P8FW%yJJ1oz)%IrZ&5mAdHOcA;L89Qg?> zJ=Hv%o;IKQZXbnKw9f_WTfLQ#`SPsop$L@L<9`!XIv(sZvqDv;@y$L#7)QPOkW%yc zHa8|}8b0{y9IPT7atc(J6EHdKQgUprZ!{UBu#nz zS`>81qzVH@BvNCv(8_-#Q+kgvO?ULG4_5d-ie;$PcWG_!24rO5JLhdePzCAvrtr*dG1VS|S+Om(z5LtgW`1F3fN)gKa0Akvv&l zrONX-`A@{|FsTCe+j~I;62I#Bj4F`*GH{(*2bP;x;rc1YSp)IzVhH|=Dd8e9vSLB4 zBEq{Xd;a*H?^1INN=y!^=vGsgT6cMnR*~1dNOJK+O`|pHjeA{awnM(SX};}i->%D{ z7(xvCtYeM38rf7@sw%6(>h@wM$&Vzzau(nly>M@kB4?*=-xF|}hq+IirnRY$Nu2DH z;WOW@{8Ovu)__0Vp{~v*m#)v!Wqe7L81}UZ8uSZ#njG92=QQ=#vGR3zr(i{1!!>vs zvMT}=y^jJpda-5IOPzvG%CkyV8Lt9R4b@iOW5aRewAtD;P)AAHA63a+)?JY7XGRH4 zTv-b3Z*>t*0P_8U!eo{xE}XMRjgiSpGXx`(HIs|Ip|NR3FiFqylO+ZxrQ(UV7W$Mp zCn>v&4z*wUqT1V#zu9i3zJ*`D`SjHoiQ;i0Z=IGdB(tV!PD&$E*DJp`@#< z97_gIrMKUI^2AMQhg$6e34fq2?65EPuBVdxZ&j7fUaRO#V7%{yFdM$@NYFwk_%rnH zGv-3%G6=8UG>(P0d-T<)h@?D5Pa9NHsg&{}R;n-2S)>C5BJ2z5T{iJ<^o-BE}%9+cg@# z?}QbZwkPQMNwOoo-G&J}T!qQ|7PC|9sai#8B=h>+RpT z^S+OODZ}xm2+pK8JvwY_*`x>LNC{{3O=Wci-Nj9U>j^{arz?*}vWTeZ2Ifu4@GNxOzKDd9G`8Pc? zfn*!=-CX@;N5{}e`Z~7U`)9XLxj@}-1YiI8&D)EYuK4tkRj8wY zGX4>LRmIe8JM8->lekd^ zVUO?de`S4R+0n1-dIb-?7K@sexBGyE60RR1%qFkZX(ePW-gI`CPKph>TLeSef+`kI zceQS;_8~|r8;lrxlCsV>709&xs&OJfNAyVw-u|EC_FMx?z*z3 z{Mewfy`PFolKonor;p+JELq>Ko_pJ^;h$Q#V#eDC0`_(MFn#@gBUlcc>&JBqr9O&z z)7R;n2pnTTC~^+@;Kn`SaVQ6=Z&|hs=7YWHxG0m;C9&A3&&q_P$RFogJz&|jMvi9R z?3hD4k-YHMwk>N%+I4B|+?@_+W>=)c<*zGhpGti>$`miR=Jrn7@ZaSl;C?29J9 zJb9IT=~}^t2d0pVR|~e~yQ$SbO=KNGjEG3JTzeqbr1;|+crOtMes<8r6R*NRSM_(S zf>x#6#nFHhYPmlWL3VOl)IxS&+5k0s!;QMTF6NAMPmLB^?ZKq){rI?_jKQz zZ%01r2}{_>u5}4y@rk+PQFx9w4)yQ9nTJB7QG1!8uOdYzx7rV94@5UPdln#E%gN2)N=ww!fqD_lV5eBb=`@)@FL1R z61S+yV5M0hH1E3s_E`|ZoN*6QnyyPoiZMbUpBm+Ys@Y-8<3QnCeR#5+N*!&dnmwko z_RY*!Sy^8sAQt|$LJ7*fi|FEkwx0X*c&{3hD3*zF{ylL{dexW@?mKUj@+x{CZGW|? zJGe1?Mlb);ai0@g-A{iYM~%E_KWJDalkD$xirg3EN%M1kMcK-8xbRgk_2SxHb&i3Cghw#J$_u8)`ouksB}xDW9OiR39u8PR`D^% zJ{VDHcXo7CVjs&Ve)o2xz=~Ay)KoBdA+9u)F1Uc9TwhGh3CmzndGkIAz`}df2{>9p z;NjBe#~7I~X*rDt zZ?-ByWoJ`6wI7ec5i)mXPOdefi z?pp91#6E9Tlp>GDyt3~XJc}k}d118X0(uKneN2~YJ%-C8PRsonZb!29OT5% zhYwzV=l4ovIFt@5rr|cDnfU#ioLI?>h>6enBF-1vFwwk@J)>n&x<4eZ@ky%*=6MWm&D5s#RXw#q4X3KA9 zUEu79$>&qXX0_|>cTmQ7dV;j6fS0td@CiM@sWBORyP?pZWp`km_+Jr}d(3osDc+Bb zpsAYn?}=g_=M7@Dn5VDCLb2x^w#QWLx_SawU3h*;f8uua#Qnn#nO=42jc2afbXJc| zAF8tIy>hbGh8}f8nFM`ykvqQFHK`hodCyJl7M_mT1hS$%XNKu&z;y@N5vRkAsuX z+ounts@G-0M%e!_#ziK?@e=l>iF)DIOBM!$x4BscS>**AEnfYT6y8x_vBT;dzNVwV z%fh^=s-wfp;vBxJqrgEpTG!E`YGEFGtr46l>o*+d-~m3Lxwe7urNATcEL8XQ9(dB` zv9p$zS`Y5dUhv+b6Jym&M4SapXOC<{+Aum&zOc?)+OmLdj%YQ#{&nq{LWGk~U&Ab} zlTV4(Nlr_PE+{C7+XH)SkxbQw_7-dwK>d>hv%L~cvI6-%>-~g%#nN1$aRT9c6O>mP zOmBf+Dx{3p{w=Tf_G7WYd{9+#|Bdx@)48|pZ3$P_o7~X(v)eBNG?IxAKK1PQId~T= zLfpquyPmhvJYKE-h3~bZQ)%&q@8C$=I?c57rnsB))5B^vd-7aMy7s_l<)k$n!2$9jv?^*Se4Qh~CM{JB5E|pQKM$Rsfuk|G^ zDaL#5U|2@4EbE1m`KQ3CTeG^V(-q}f>seeS^Z17XD5bl%iL!fN_Lmo~y4&vZW$CEd zr#SrbF70Q$uv5}MA1_mX=I650@$5kGTAXW>?PqTdlXD;C(>Jd_KaqasVBy<$9e1K4 z%sjobEPt~LJMzDimwm;=vBZuMMnc>s&S)AKCu~et}+;v0?Q;hI#bAFJ3HeFH(b*E$B&! zTon_!j8PipK@zTf6#JAu7k6iUL{%KPKSpv*UMew#ILSl5A)vFBi?X@x{yFV-1B$a4VCGce!*A$N0~+QJt6gQHWT-r->kG)ya|i5IH>$;XNr zd&dl>5S$8^YdEAyF_6wTbjWe~!G&0w3r4nHC-Bt$vY)#uskw~&Dp}HiAL&NzC#~>G zN)c{B`RSm%wr_ek=~acP*iDzGdc4uNck!#(NlA0yRdw(S#*tEb&5HL8n#^Jdylhr) zRH6GD%2;sb(JLVf{AV@Kp<_n)T&!b53=r<#3U9F~2DE_Z^CJZcgu<8FMA~SNffEaM zJBWYb7GLiAdzEj`5$1NRXQ49qsnc^i5~C+b4yYiMq6F<^dDnB79ePAw z9ZP!{)+#ll)B*bzZ>0T1TGVi#`_T=nduVIn1MdeDtrN$#DkzXBPc&iq5Mpz&GG8jd zwd&C{ymFq%H3&gUKr7rr7j~C+C7_+_K`+5Ax@L8~sQD9J+7-^E7EmX8N}~crP3HgT zS=fY2+sA_N%-P=2iCUn53QZ70KdmSPuFAHjLLFE?Ds8E5mC=RtQgj0XBoFcGZGn4@ zv^dn04s)z8ncrKrS|mj5Acy|u;QG}fYsI!65C3c~f#h$(?Y>-cqT6{4Px~4_TEstv z;OmG!l@bpN0AH7hNhihs7g13Q$%rfp$Cb5E+a|&nqyXVFg$$B z=UL8Cz4oWvq1^(`sGdeFf!7wQ_0EOWQ)V_zZAX-$sdU6pau&2MUx$!pAdjLu=(F%s zbphFo_d`(PY}i6OD($Wh8g0q#f{Y{twOg%LLutEBV$nfkh!j*_WSBt`j5bg0v#A0a zkXn;i0|bM$lR6(v){E=z3xLigv9OFzSb;S&>uDSy=-Roh7nvlJOPhs)?(r1~kC)~x z$Rf=Sb7swvPG-1k?53Ie0!Q}IT#d*EnMGu&*#;51vMm>?CFoGdzt_sCY5AAmQN{n#2}FFK&OJz>JXS zk93QkNe;uevF;i`UBH!kAg(>(2sJEfF|_E1vjh@GFshadu>^sf;fXW|35Cl^sL@qt z@o}P%G^tG?cx(=Gy5Q?^cJo3F+h8ofYb22cX|@S%nNT!dvZ6R(MWsVqmaJHECtOR4 z*Yz)fH(fEuDm8NRdKQF7JQ2;m@sAuOOwf+Y$ILGbwU-oLYb_~wb}r1QzKaDm!j2>* zLgr*+i%=s31fe6hpu#O})`5U8Oz5nL^?4R{nDW#SRRJ?(HTPik;@G*`WEgZfkw!v! z$NxPPQ}bh7DJHah4Ctm4KMuh)pWvKc!wNZYEHnxT!>xD6TmG ztrrX;go*D8yNzZTR@-x%b7#1TkamVP+n&1tY5i^69^-qT6+|{5Lk>2c6pFa^!X9Q^ z1SJf~S-dECMJ|sHO`MFMnvH@_%4$)I>NG)7Fi1;AS93YtS%+_YGE4Nn^sc-j7kqx& zn<2=h_jek!33^<|vd$Jn{(Eo|4UE&%jLTv}^r=Z&A+w&mauh_9!l`Q!v8vle;XO-+ z!Mw7ZaX-$dk$q~>F4wh&NLD1R*{8JLJ_1(iV=%OKQcs6XPmINZ>~}&eNELGoBos`Z z*zpJ(M%7rq3*-|y76i7ag8!BIl0V%G<|eabux@kX zDj$t=L4aQZLO+uzFB)JY7FF`sa|1YdjkYnW`6BPOs&gw0Zs3BQR;n-!pWB|Ciws9W zN^M=J{9PzsB4a^Xyz~H6Pyuz+rxq>5o8SaPBp+Ob)QPT=$@roWG=VZz=O##dLv2BP zKS8X}?<#yo&^r-_tiK}F5IFsg&e?@Cs(+fyuYv9eOu}%p=TDFw3?V4Yf$Q<>rybU-9dPSJod0ZpFW_Vb zSSt{9RH31BUXPgDjG;Cy7+qoA#bittw&0}O&jkUM=QpRyYR ziRq3kuqDjR8x38|hJ@7-7N3qOcu>>wQP{xTtmIhFhG@veSIPPsU*xkOYk5Fi6LLhPr<_)ZQXxJ^c${Fe+K0Y@G+0sr0 z9-P+(6Uc>*lmv1{DsZ|MCC5J}R0LOyH;uc%Zcvt3@}fPyZ$-droY^Kgu9Z4FBN0^j zDIgRXV-b~~F!mEq1LtMC`KKkIcFzJNF;b91H&54OYcued!{ z(S6j|iV%FlOvvnfX;?kjg5BKL+yhP?${ll(Cm=xhPMxFn{HKKj#VyLVl#?y3NVXF@ zX=+T@c>(b-(PhM82u87LNmH5EjFaO7Q5_59#xkFU{xV1sh>i6_AwfIML3+EAi-w-r zlP23sa!@(Vcz&;QeX{-#b{i3Fw+V>D0(@f0n5K&xh_;>E7gS~jqgGPK7L3AN^6L9= z)P_J-I)&5Xe1HOKf0_*TcCNzM*$LBnx5+@fK(yFQM{g;VR)r_;2s~HCXu?z?N5j(D zz+YXB{mzv{4JQ8-S%ZWnXyXdl$hsK`MBNEWE?L(|G-8sJTXemhQ`a2v<%(X}VqW0< zKGSIFO-Wi)YHqi^=E~}yp2H1A`h-P;Rc+a$uoh0UPYyH}7pMgQ?iMf;71d4$N z3AcZzXwFlrXY&;(5su8d7l)l!)w>c!Y%kDzkv}94DVt4!ETVHPiA8jnEAYt!cI4P1 zom!p>0kP$ir>ldy+6mSwUrBLVUs;SG8Xw zm#z$+Zq98U_H1dwb3vNLlO!0PpM=A9ao^Fdr5Q4CY>!wit}b{oB%J~t=Xq_{CAy)p z?TiIaYf&0SNP|Yj$|%QHI8IQXse?030}y3I}?oidY+A*+KeU z{*V~Bw+h(h+ngO2(i~dX0v99>eKbI@+uhI*v~~chBKH{$LFnXSD?_x+Ms`t3qH}0k zWEH;DqmiWMS)>Y>nBhpJ_9ng^VwKB7`n+mUjzg2p>DA1f-NE$m|jVoY)UYyZzmemJq&^gQMcZ8isORr8K>(0g(79a_&H75W>hMt>)Ocw zt@efjCl@m?y<4`#12H1tGx=L6pdXaM__^M&R9)u;uB^Ac^9NS2CRt?)>qbJ>!$2@N zU(S4@+$XwjWD#riy|fYvdzb2xAEw8f27s0BZ9|j#_7?2Kb1U>DF`;STG3qF*8Pbu@{x>MIJmdb0(J|^%v03)ab5@f{V zFsn_TAaJ42$qS0ykY|7+%r$AV={;I;8Uqz+3!0UMNPKc348esazh%>cMi4H&EftzN z_0+!R=aet!2O-hvsKSbo-o*T)Fle2d1qzIBD`_Dp;`@rhi6M6_>t8_73=Rz1^`k$G zh1Uh4ii4>0V9>lDE*g&*>1nr`DBlLG5x5m7Y#BPl8?Wt__J*HWMorKIiYGza-UZ)o z*d^x}>rX@F>_k-rQ>Y0pG<|xHn+4%{5%yaVhzjZ;1QEleN3ca*VAQMu3)uoL*DGo% zZowRdRR&EHF#nG@?<8&!07r+nfE*Snxf_u4-}*FKJy@n8{JaY7u5L6xf!NcX(VHXp zjNVu7BXA+jI~@8@a4;5$9=VPRKtq~)TV`o}K6g<3V06&Rv$fuX?Wk@RT&VrKrjY9r?g-m{)>$Ya8TDv} zYhVvff#nO{!13^Ni;FSTI?ZCIXFjqn{}(6;Y(9XTD?>O3aF%ln9}Sdr61sm518>pc zli8saK4sBtBhmYF&h15FojQ6WL1~R3m*Bc0mYkOQ5T58?*Mfu<(j6wLy)#tA(3~ge zWk^dgMF%`OPjd6-8w7MELIA5+QcH_yC(`m~4CEmsY>z)0N=?EILCUl#`39XWa*+*A z_CcCPE+Oxu11uS``a@mVNnIBJPjg1R^<9bqKAhYMPqA2ry3 zj5*%7LLXRTzXbX^Z=4=gFAas$#Cey6x5g6encP!Vj9^Sn%hIIxR&5f0^?ubVan!lf zC$9nmIsvS+59>rNb3VV;t%`v&;*zQ}!WJ2*4PmmlLH4k%GiuA{?Z1UN9tjXT=k%_A zSv;R5IMh#c2Jtvun{t$wg%M6TpeuQi$+f(uq_i|vtSrO`@xYUFh zIbt1haXD?23pFEHXat~DkY)Z)t^rQb%7AtZL-d!8U;?a7<;UU2H~unGxUIxNPDN77 zsikRr)d79ODy*!Q(Y??DH~{Xmxs!%^OSc5>XZ{;O(FdqVrpf*`!uo9&nfk3wzEaDx zSOT*^#!1O5%-;Kq1ZLr0Z#=!cXs`qJ?A|-0s+X8rnwN7O7y@sQU%sCrq{&}2G+bfx za}XeH3rpQle3^?_K$XcH)dRnj=KB{wvbuc3`ej^2+yOWn8rT2J*ZvoF0FGbsL{Av4 z5Lgw?252IsmJGToJnCKqYWLdT@=<|)g?@d=jMTD^;YIQX*bP)?gU|&k&d{NkusF?R zhw)gK7J*Nujki_&p_#SO`v3!^1Ptz9zBN1QjdS1sRCE)pYwJhZ@{xy00G=(BJCx{fLZa77K7{ptC&fCL%`@|RS+?3jB+ zfk>UVlY6pc2;L%a0Smy?`<{!GZJRE0*ha z269B$m7PmGeYOOq>;D0PxvvmdErJ7Fi@PIonIm5mFdciIY^c4$;pcgwXRt3AN-m8A zz5+ZWt6OzBUkJJOJudW&if!wchloIIl|`31{?@^S8i2EkKZGuect)s#nOZ%a6#}cm zO#u#9-&yL=t(~+6{M3n3PW|P2$CCkyJ>(D6WzWL^HVIKmab22iNe!D105X4)`F?zv zfE^$K@kO!5@A(k`$+_8=&z6kTmkmsZuRO8-cS2yE1!zh>T6U`c5s!c#KVyX&m^Cuo zw5)+bp$5iO_pfMpkMBW1111%lAN`Dt?SKZ-fGy)+d2j;Apv4*Q9hMn)>A$dlYDK>n?ms=dqd z&kE%iY))Cx-~aCJ0UjF>KPvfq22KEHe&kRLA4Xsa!Z`crh?kVMschY8#{1PERNT6PW@^)DSAYOE}XMX!v9~!uwUa7CV z9JFOX7Na*F+WA{WS%7~zJ|ovHN60mRoBH707k(94znJ00!w74)6U#1AvJgAN0Rpolr@nN6fMccq7Rx1}kwDUBL$>exWudj+mj(o@sd!_Rrii%8RluKfrWa15Pu+<4 EKmPIqbpQYW literal 0 HcmV?d00001 diff --git a/docs/images/SqlserverCDC/Sqlserver18.png b/docs/images/SqlserverCDC/Sqlserver18.png new file mode 100644 index 0000000000000000000000000000000000000000..5be43b703ab40b169b6ae5c652bf5ea0f78f6f65 GIT binary patch literal 281909 zcmY(KWmFtd*QIf2Jh(%GyIb%;a3{FCLkD*n4=%yo3D&p=XgpYOXxxIkI}Gp4eD9k1 zQT3zNEm>8y_T6Wn=SHe3%V40Apu)hwV93czsl&h^8p6Q9t)n2lA93k1^M!$-f{~Mw z_~r$B+$oc0Ad^X({#NX3w=mamBD&g;@zpbj`^|(Pho&+Lrh!Y&%2Ly^gi7KU3Kasj zxJ~RNYC)4rwL8NC#Rm$U%(05hndn8uHNm?E>sKQIIJ^wJKMuywx;+&r_tP==mGuJH z@M-EM)zrY5Ts>%rwXfB*|8R-JsWCSB@72=ye4_eB6RBh0eWN3SN+1@G&rB<>XiqHk zss`W7`6qv5cl%>}`h|n(-wzTzzL+XzlvzDAbg8TNS@EJh&J|gkQ9GM47US6Cy~N|P zz#&%@`e<3oZt=r&vgJj|&}NMv$HgC1c1*9nVy}*x_u=+LasRNCfeEVSS(!2v2-Z*6 zzk?8t_L2`n@a6kHDE1I5_Ygn_GlharRsdg?1qI;riLwks#1ezp6T|3p!iL*fh>u3f4<4%ppnUHb;hGtM)?zYfo8w$Te?@CJvseCUj@&cQ@;OtK3zM zjN>>Q|J0|yaWi2UB3G<5p8eZGTG?c+5_tCdT^m<=D$&~80bLFZ2KU%{6;`n4r5npw z@JIYHA~;&(_>4qf5^zspNU%FRfNhn-B<2y;V0I=h9JXzpw|#yb7Wy)?$B->7dAm=R zu|=$oOfO5GZDu!&sX_zLmU`%Xb)K!ejl?j7a5CW>RDH0}G+uLu>`-Yoq>2;2$G?HSNHfN=+UQGZH zNE23$NcyS6JM_*4dj55i^PcFG=xy0ugyHEb$0HD~E{Du-licgea=Ot!p{;#Ve{@d) zOce{#+&^pq?nU#pUy(4W$>&?<=&No5b=L>)+WKszJWz=;vWGSgG*+se`!^3+Om#_TJj(~3U`Uq-e|C4K$!x-ph0i;wGB^Kna9rz__eu-19`aqVe? z`gkY){1ll|w|B_vE+qb9Z7$CkSMgW zwfX`B9=9ldds^Thj^Pd;Is|it_~6!SF7vlE9w=B^N}e&xYIB78oL&`kQyMR2pyT8w z?4Ca?UR9T9C%Df$tprj;4PO^nJj}YrebjblEx*0#KeH>Fx&)A8Wp`#$9PGW}wuwCV5q+VdMDv^8lc>g}6f{uVqw z<^9ZE5ap^Hg$y7}S{}A@phxX%j+^;0(AbtIfE*IVwen7A|ndJ(wxi3+c5QeQ-~k zle0UxZqbt1q|ps4f7g_mPuLV)d5rpZ4&k{Qe_(L+*hjNrf7poL<)`xxG=P##3l8xG zO>LsnbGFj76R!OO%Df2nRB4Mlm{%`lqy+?nh1Kt4HNGDfKLSbnDwuzqCK40-0rHvh zzY>6f?Fj8YTi$5#Gi1#acv@j4vaXFfRUNIqlNGB)cmuelJ&(43yI`Jk`OsZj; z5gd)-jV2PsK7Z}<`VrFSKgM2dX5#~2UKA+Ne2?mlaoTme(t=WJln`(fF2KgX@-)?)fLb((inHy{TG~5UFC`*FtqT$SHk8 zjUX3@W6nJa1wL?o7K~U6;^g7E>-ylfbA`^OicY^7eQNT(FC=1B11NJD&r)Qsoa)sW`e=KJ>kJ#NS4~KUw9gdQ?Yhl|2uE zu|mzAjQf{yMi1SNWZbAvW{2gPf%Et*>F12;1GGnUVT1nToz#SFMkr~Qp;ud80_fY2 zvcV}LZG41rWpP&IVyu;eAN9j}Lg@%3opU#=ZNmXR;KGghxpWZZX)SEeR6N~ZYmqyx zbcGzX*mH1M5A7#mIrc>Zv5D~Wr9yOwS(f2{`9k9|!?iM0KZy@OL?>_;afL2A*Uod{ z+CTZi$#ap*xEaGZ)TyQ4s)HC$?&z}iT@H2D>D?*i%$>$V+Dq9CD!bwoTu7+85vw>sYHKPG zH#1X7$n3rmW)_0R(-4B2JE=()obYtIj*bjV#!ue|%Gousk0$v(d6%Y-^~?^S>H_o? z)1|yOYTxsrTcF!3&eTa`sEHre!!|uI0vC?&^E2o1jtWQ9KBx?#i}_p@9;{qsv@b-u zMiiye4VjvmK-8ga9Rc3|D+!%GyCS5WOtCD*s*uAB=CtD~=qmXwHu+jXXGnpoxH|f& zJuN7f?=TA+`{zk=*B9({a(OeCp15<&paU{Ke}q!{Sqm$Go90j0Y^e-gcM~6P+!u3h z0pY-~o=Cx}&wi~7(?PR6bs#W8e}I1j=HK&NP*ye({^Vj(?g3Us9H6({B@-7}U8Vr9 zPzce%CzwIh8&-vb3FaW2`Ntx@(Cz|Rx_*?j{+K|dy&MplVikXN) z--uf8PwL&Sqgm;OOEBjC!VRr#dsb|Hl)D;{>F%G4es_cgPQFLIy#Z*kYdqqDpdMHN-u0FfCf63d(x${1?#(BFzrrgT8 zdsF_;Db!v?$L4~Z7Ileg>{^sFXS1)^OC#Gk-4BexQ-&+fr~TG*cbi)lV)QbWpI8}B zlpua}X%}x0xzEbqYCEsQ7?WO5C+E^R*t@<9P=9X0A? z1}N|;f8C1@ATx>P`)|mw2Vm`=U*5rlyuLiY??vBA)9ckGl4;YPuRE$xzi?JvGyMl<<5iQ5mbaDARZJ2{L;bj;AGBzsWpp^@>he>+# z@=j+(q6R)=IE5_OF>pI)$6?TFO9-zukv01(V#MA=qOic%h4bXv@jnb3;y;5NFaw^o zMfMY+1T;c&{>S!{?q&g!M3sw@TuQ?p{!zbcjq=`&G2TNTE*_WfzJqh$Z;{}C2i*W> z9)F;bS;TIXN&wedIGgh=t-!`BN&vq&Pl)0N)p3L&BlcPhEbcP$p=vH*aLkYWPT z_IRN79EPsF#T=jov?0{w7jw`A4s{V+JIRyylYH;%T_dW=|}0I}dy#uFX$EKpA2W+Jr;*^a9GF zpZ+h~(|#)xg-hxK;t+$B@aDu15yQDFAwrGtWuAm{O(#&d`-|EpT+Bf1zb1HCOTL@e z;n?UAlZ#gg?y%+_L0I@%NDf2?hdIls(fzK6{6~I46X6W01gxR05S;#< zkPJk$qN@%qZ?&SYtmSN5&z%~)YTHv9s3PvXRNNGCo00gc12Vt$HV!$^tH+l}l$EDz z7Ers*??he=+2;B_&&z?DYkz#4);_%vsl2X-Ja1mA)IEKd#n<`gTk0!+QcneUcxf)@ zeKU%3nMphYjB=UzqYX(s;Ysh`e}TQF{K0Xr-!nKg{M+DY0UOxaIR*wp+Yj!#J?CCi zbPzy>my~1k-k1QM*+1eOFxQK5>tDAt1VBjjV_ijH!0U0dWCiJeMk&77c zZoD98IBuU{_2rLAgY7~rpd%;nkDO!Jt>NFHP$MlJutf(Ft{h(8Z@fv(x&S4ySr+G| z!c&|{O(krG-TQxq7eBjr30}@Oq5#+(WN<*=q%=JR04G5I!Q$B zZXMgJMm$lJzZcHq$838ThFr@%6uGvB0LH36123OFkw8k=`e7M6O7GM8kF`YNqCC#^ za(}H5FnELDbe12rRUVkLc|Tl<)*b)X*wp7c?!eW=_xpeNn17AUT!z6gGH59D9=zh( z^?}ZFYeN;Q)a$UhCS+HY9w|fCWAqZYgaSUdZO~&aL+bW$I*k0KY=r4sYlkJ-G&QKEagt7M&Vudf5{SHSs|aY#_i!fG$)N}|Yc zCCuSeGN>BC3efIyDG>y=S}t>}dJ*^5O;%x|6O+&g9_m=6#&>!5yg%J5(timpO46VjTr)Z| zccXq;IUPz)k13e=GgzoG+VkyJQgOW6?EZpmbY=Vw#fGo{wuLjp z0(~hz)Jywu>N?rng<4E<@o+|iyFSc{IYg}DO0Xm22eB%$E~*N6vyI;3l*4^S9^8!n z9_fUNIf542P#dBCiIPLKC8&?y_z%wCf3J0ZwJy50!n!bN0ZW~nV{9E8sK=&Oi?JaM zd$N?dREZ*x)l#-ZLMnlOTU5(BU-p(iLska^yvz{2u_P$qhAg&$RK1pqsLy`E&vj;H zr-UkM3^?EX!yPX0p4#x*KQGlH}BB+J&>iGpJ zcK!Gpr#z-8I=Hbx6mIPRWqpk{n7~b^ZGAF?t4b^JmynhSoV1@0CDs?azCnJY#OaO% zJVg}WlszhL-s9ItSg4E1>?S0kkqLD6?^m7hntZ_u9!0Kr^z$C;Ar9)STXa_(s~%Bb z+yw`?VJVa3S0eGbRUuzQO$}Nuk&CRh>KcZrRFVLTnh{swCGV>$NERwcfz=fs7Mxf6 znwIa_L7^wB@cZOm=xnB`u)Z;hU)&X&+Zy{&N<>}GWt!_8T#MT4C`p58{5Dw|@6huP z1Na?YMeh3m=y59BR+8*|!Njn-Bzm7Y?a}%Om_hJrrZB{yR2a83jspRrR1+9kpA5P* zRh1eC2=3|^e}y#J3$uy#S^p}C)28CNbgW)uX)q@{Rv!aCRSO|v*LCU*H#z5QcDaSx@5*Sr5E!aE>OV zysm16;xCFUO@1v>Xjnnk;B`Gbt4LA0j@w8ewk0FpWZ6*<%%%=ll|(~9Ibm3ikqF-* zs?GKup4j4e6yIE@E*!w~L7;#=DjS5vj}D21Ok@g0O?nXYRHFy$+r}ufe@d3^-NhJ2 z7@Y0Rti@}qTHTB{n<3{DUvMuclv_>+#P%S8#qC1|%Pmp!A-WT9#tj|?ME3_@`#DUMrYPk!#- zDmq^N_Tc8r4Z1?nZaUuf@GpWZv`?;r6m80xwoV%jP2jtx-%^`=?rgnsZeB zl6AB=$BCQr<&xWUTRxNW8U4)Qg}7AUL9#qu-8Y|-_}Jhh4+Bi9f9dbh1nr27#ZV@nx{L8%E(;Cvuw`D!?;_iLuy`T(zs-! zPZB#nqpBpwXq1@#5?^S9Ddc8$dPrCwCakl#+c*u;Vp;C|Lm~3vq#mkFpmCK2M82)v zj6=gsVfk6trnzEIU0acAIZU-YX>m=5w?7g@)MLnn&X6Z=g@=4N+p7WDBfgDSNf_G! zw!FfEwby^{^x-xrFj)M2chfz`6=?iN%a)9sX}A;4;R$SAQkxq@3JRlekoUTsB5&2U zSL;!y(I6UGMqqJ$=zN1g+WJJ6;;oXyqT92!i44R#jLv`*_L^SPzYZ2bX?CflLB~I= zOKkdreY=l8*ezwWQ6E%6aeUAtxaeM3Kr(C+&=;S0K-+kOi5Bg$NYVISj{+43e^E%` zRJne=!~PLk+g=x*n)^X__cUIr)~;cA+BM4)jlvjx+%dlPaDC8M(Bh_Xq0M?=66B?jxN=x6Q|#r*s1} zfY3alvWbT+`OOJ+t;`)_I?Xv#43(XdG#8GWpi#5}sA`(c;edQ#EDD2r^eE}>ahC%6 zEQGXC4FgV6!Z;U4`V)k3nALtRHzyhM*Qxg)x}cyO=n|p~6%zdu1fl02x}L8zr)Wc=TTP$PyFlZ2 z8c)HZmtn{B*R>z?-z@Ptcx5=jHJ}~Y?2`~$RdZ|Af^<+7*;zGsVN~oNP_Pr+zxvAs zgDsjdmB-B7d|>^He)jt8;Jn7YM&dABXT~<=pa4mI))AC>&DGbeK|2UYfX5GHIq5A0 zZN4ad8l$ucY9al!lM-c)po-Z?3u7fMzMyZqV0)Vr9ALWsRTK@zjggGC76%6>yP~a~ zPJMR8Dua0Lj>(!%x1uw%E0rm|c5?j;CPHrc(Bc$-OzLUCit|D88|9&blXV=2Xh-R# z1UC4eJ>5Rd88)A@DoC^5fLP!_wz+cN6~nnK^gx~?(Lo8$uo?wjYYg~5$X1{L-7?>XP0kSqa1C#(?I%Dc)`oMtzQ7&=ERbn2@-L2 z3X&m6D4|^E>1PG~pd?}_gr|nxIm)9t310O^k#uU;;;T`IM!dc|R_&%~K%T7cr+uHx zYtQ<}u=f}EJ6HILEPgNbd&6zlN?=jx#h0gP_i?D+dWp}}PdVuhh8%vk63e6@Tns|E zW%Lr*S^l(uaDT?`PBvv8HRPLby~?MGJSHg~haZFonZlXH-mI~Knz(!N3+c@!?If}b z>&B0UJ8zaOl-)y6q?U#Xx#uEjB2mE&9H_ zt<_Rm*6hn5EgR{8t?LUbtZh(p;-za_55@i1k(;lQ9AGVx!0JBY-FuS1>2Z+7_|wM; zmw9FuEH|HuPYzyrlfIfTM|gr?S4;>RC_E-tM1C++zTy;|_%qgAe}#iEMG`5?R{$W< z%z`$g80{`_E2yoPq}6ckUF7f`BQPE6SpSVO-lex?6gj|+`xSjYAEXgnmWF)PDct3? zzOXE;01wtu(q`T#QObjKYeMLF4J8>9{5P7>H+Yu^QQJp$;R+1Ls@p8hQMqOx!zqiZ z?O|@Pmd`6yoJWz}%zecS3W_<@hdpSrf>^Q-pHAen=1d%+lEfTUzMi$U#8Pkf!7R(w z?dvh%9y!?0ZKlYJGP6IADE!dN0tvUdvuqA1Bpl7@o#y#AP4|AlMX-hdc9&7xT@IW8 zZm}QSl!CKr{?KVf(92rS{3@?W3-aCl0fC||${um#5lr4J<TEpEh8kBei~kQPjrh z(h^_5vH#Zrmr35mDxABX451xH{|jj@D0o6OPmOK72I*rMRg;#d!;7tHppscc<(3to z5hOZa2U)RnMw zUb6P<98Em^@92yNe1mRxWK9gm_bwJ8wp@DMl%sBx@v;DnJ_w?21O;n>^1mUOL!4i# z#rb$)=t&bdrl7|-NOYxBE2|Ni5hty>%Q2SPD=_4Wq(|2H4B{rVXsND#9{8u?-^3R( zk;hvDBGBjc3~InV)rU+xo^GZ^!&zTX$>bHR8%;=Bi*5$kCa@4`+4qJ{t43u^7)*RvOrA`M~&zr5#GO#x`X+SW! z*5n*2Qu>%$r9aQF=0o(thCn#3c$007%-K5KiHDUO<^+(G1g@Qa%Vm zocHk@e5&ES(KJkp{27^o6kQB4oC_nknuKS{mV17$RZz+94M5R+7oA&TrdHXWVSG2b z`+10cA+0qk1$2tl}Na!l@i+Z^u#DcMg-S4{6XN)>mO{SONuwZ5Ni)0Fmva#g^u#U;Vat2;8sZn2<&WHNAy`+(+@++=rW z=HYEv_cgDxe~Np+gEl06Y<9q%FqKaju6XO*BZLvFb_IN;;CJOHohduDy!y^ojnbpl}51)e)NVl*qpA5KxD zaASX%u-C2BIPpZjT5mV#=kS6(MFkK0L0{Dk>&b*oNZK6rGMQr;M2nZxl!AFu^M&$P zbJYnLx}Oc<(h^B%ax|UfQnG8U7>*Mu8FI{cPVgkDQf%TRTyDSVmTvO5*L8a>QQV~Q zk+z!T=Ax~#lLxC_wLcnBd(=Obl)TkHXue`2n74h0Dwi@$Bl(6|mqcuj_=b zO?h9(ju-gasMCEMcA-nCu|)H@ihvH2|B7vrG6P*A+Rd=ptw&|gr_GL9<{urm6P^WL zGao!9V=>XO_n3~DT^J~t*C8s=ahUxQalS&mh__6SPz;a&VmkG`Lfhsw9eml?uOYT|~1&J2tW+8z8s zQwO|2*J@2qgCN!3ichsgqW7JV{wv<+e-a2)LoB!qhxI;gryetK*_N*!-n33aXG5!x z8P$inHS&*o0C@XLQHc5oz60zCZsYivQ&7Q}l5wi~1K~qEK=14*j)gNPDgaYuN2NnZlxt@f!!YQfkOSjZE9^f#-WjX7o-uX}iVCjB^RHT);o->o)dlj{6EQjKsVIE(= z!>S4>!cv?af6J|(PVfk4rDvk!jvo^Bp+Ei@Co#9*>(Vl3D9Z2&V zMd{(6R7vjid?v+2V@ts@zZ@NK^Fy;PF4~)!CBD9pne&3QmqN*Y$B(o0lG1pse9Wv` z1EQ}1@m^&c``yS345S)CT^E*%43&Wn>JZ&vk(H{7hBMC7rfUUVM+CaSkqS^o0~We{W1op?=_3^PzFSO2bfgAalz+0r}KT*aT7qE&O5D&`NV=NWZl4i2`B zkT2CwySjg{(xuvn|B`5*^o$^%KYluHVnHENZw?yMWnC5rW_gr`J6XT|KnPw*3j%BT z{{HJ%+sI(~|Bi0^4Q5l=bpk#XC>^M_{tI!cyd{Ig2et$Er_qnZyN{r+8RSYh4M6Yg(BK}|?p#<{{tGF@szWl0zvzInc zimpuZJ=K1N#MD#UjOO7)g-r(ep^`Sxv6fUEjL<>X>ChOb`)<22Kfn04H_#s%gL0I;|z0bNK2 z?p2T9>~*h&N)!_t6${6z1i#*v92zw^#K{9dozDdhE;kF>V*S1brGYzV-&!BJfw-Il zYbD8{K50f7v_1uOZ2~h0w zY%2luhWza;Tp#-*RYFWhAJs_T6&QTVcRSo5d=|@y;rK|q_?oqrZlj^$n-Q}~ zm<%g`S)czHaEaCO)l9r$>)v!M-qxOMX|t4S52c^Go|9m)Y=y_n$5aVPoJ^6=9sSRZ zyC+;$E~|jw@;-TP&?;$!P-BF6A&l{Fsw~5{ay24m$OpyIXyZ3qJG0GH~m+?oc2@J z?YK$C(Le2H;*?e*8Tum_l4gmQN?zW3{vq$|ecfclteAvQ-+ zrxDTe3f->V>gvhH_H6J*2n!K~?tElV<<6CeoNC!gLwtdUw>@UOJ%A%42zCE4Cj?xN zJ_%z4Xia5DYveM#G5I#+11t+PCK!du4EYkyL7ZsO5tj_3&35J)jw0`JJ_=h*f^gJ$ zpG{pG8^P*Gl$9;1ac4gDRl$Cy6SwN zRB`fMoIR_8hLb~-ImT^KP$awOv6S$R(!)$7frm$SPThtrv8LMj%gKwDE~aw6hK_b3 zxj-;r#hIdi#ZLB;6=sFja?-MHP^%-s@Y}*HEy#YnPs^ag73(OswFiz(f7(_zm4ynW zCr*~+1-GHl^YD^!(%sGpmcOC`cy;RtcY%DIMTn>kgK|$c+Jn2?lQjPBoBV+YcpBrU+8a!~hF^_9V^ez7X5X=>E;zoOPH;Na_svh)?_+tmFY&LPl7V^WV4(8$h(fsuYB%#~`+7Q#=1 zCLlL2#8*iNKnNXCSVdQ11@xt$e>t>ONAqKL{c)~L(Q?*AYJ!x}sD~s0`v;*xP(r*m zRh$_fR?R6lWl8J`%ZF}?(UB?HFLmID{tjtlbVv$JnhPuUhtE#n@sAf{bWjvX}y82wL!beZP7SsPi$tX`eKG?H}bc3Ms!glJS0|2Q7uO! zH>{YbW~LiW$F77>3>9p9HZDyOu5zAkE5f#*gAxqb?R0`Gh2?t3`fl=corGaJSVL;x zPs0-3TS(>>n-pUHw47J+>t;C`$Z7dU>^1B7wFb-LLt`CHZ!Ndh?OonAwb|urEX@sE zhIba~2CzLHp9Fixx7?P^#R`^5W~S;XN<6GH{v@3-B{D;~d49Gz7&X`qGg);{I5Ubw zDhK{1bF*x#y7_ze!J$~0g|>uZqNcFPTGdS(@cYp64MAD~ax|A0B?W>rk=JyA!jAg2F&Fc3`X^OAztfzEHI#k*Mynmmj$NZn92H1 zrlyJ}xNo8`6IZJH>va`91orziiI5~tE(ryf)5?@Ped}a0G+UHa0BM4AeZp3z_WqFj z2eG&tMJF$LA-;RcNPHZ)T}L(Gh6b!5GyF!2BaFeY$^iX;lhBQ)%b*q|#zqh;3Gi2Kc|Mf#Po&=5C#*EY_eZ)|tMK96B617l>_ABj}V<~HA3ourt7 zaNW<1(vZo2uiQM1x0^rp?^G9{v9mqg87FD+ZX?7}E=kCPE8il|1;d7idJ8>Nc%z^0moCajJGA=2UsK`OQlVm-fo_sV`mshfHuNnu^B3<@{cw}rNhJD?;mb);aYO!eg7|FQMU3E`*|m* zT5A}N%jia0FSA7buiG3v4|!II5Msst6`@%$6I@lje|_gMfTKT%eEBCyl|AIfWika4 z#Ev&`87?WVt<8XD%n=?3HhK@qx2%+K2k8l?q$cNDSQWN}o}M^5eg_{u8B)zm6I(5h zjv`O93E*T)LPY91#ZvhASu>7TEZMfokA#n zYI$Yz_N>sp1j`m>y;>!-du5OEwv*Am7-W^mN@Yna1yT;sl($^v?Wl(ReXJ?X)iN zns>gnr*f$fF~s(YF8z>9t-y~|68Rx_iQfle%Lc8rC&l4N7@ndBZ8$s|o`B2*y|Y^6_9CC+0Ai0#S+PVG?-f{hQ{a-K8~b z4vB+>MlfQ(5vTJ9yBY8%2nv5VXKhc55XF+Tdp{`IK9M{a z9IMXesA8|!VK7hm>$dti8HQeBVK?^!tdK3WJ<0k!=LuiKR(~e*51As3Uf>BCQD0x2 z(Tc6jptJK@;vCz!VMrkMzC5-o(2QcZ7jTP8Uw=VI%b_EKY|S2;p*SxHp8+=KR)Djq z=?HFBn=yc^nYAOo5fK;rxG!IATu*aX|8;VDVCFdPTlcX-Gi#ypps@ z-cIh3bB<4U!J%yak?%+swyR3PRF9#iuF1*|N4iBz6j^7 z@=CP4NKKROq`3Ug%pr9E!LUgCODp6PTsKK5Em||U8o8?)FS`dX zyo^t6eI;V>uf+4E9&bAm-Nrdh0#s1k^9|ukBu^^e%Z7`oFGwL7e-uKX+;b~EscHMi z`xCKn?y}TvLjDosZG6-Z8k|=5w2r3rzMBpus50dzV}|_dY7CJO^Hwc)tOe(VZE1IS zoD6Df3uhR9bELNZ%Zl2VlkQa1wsXWd1SI?f>LF^FM2orXT=BCIetH(2(~4t&be)-N zLAfs+SQV*~)acuSFeE%pK}O>|VRLad=m4M4PT$uAE4=mqZA;6LetwlvK+>X`5P@tb zzdq1_l{eM0YZLGHD|Utbskx5`y0sf6#p#dNTEx30WXnBYV=pvGbMDreNmfz=H8gkI zzA#eW9Gxo*3s=?wIy2qtaB*T&i6j7H0Xk@>KMIB@F$Tn$r^Rcs+F1fEBJe0l@up>A zM7x)-je{7-Oi-`1`V%UVOYT#oxSVqtCBjOTvLPF;05b9nMI5UI^@_z^EI2PY(e({O(mc z4UalQOfTq}zA}@SHYnpF@U83f(Y}uUQ9-22BkDf4Mf&Cbp`L$M;zp4aEzAR4o^nBY zhhjlE_~D9S`>IUU2IDRzag|7eJR8!W@Uhm!1Iv;$xqLl39=sogqyD;z9FdA<%ndhN z7ayBvkPB4~;%UEkbltquD~-5EQ^bq7;_YqLs_Fh1C-K0%%)@n@^O5Ec-il-bce!wvOnq$@n1xg<=_S2tCn3fh_E zu>c|J)wg|Tu5iv%GWXwcp~>$d%F6es`%ElO0ot`K}Q4OVz~{Ma8qKmF~AM4SSaD9S7snN|t->v(ug@yd$x zvu5_ZFEsc=o@*-9i4_c9+XwqGza08F0%)v!O#+(EjID}X#AZMgxrcaC8=ZO2 zaAz<7&S-x>^$dBPG4)McoFa>*`Qi!GybM0in=f$@*c znd(t)*F0c4+1!W{7t&XVkT`byHs%PDp9%1ze2JRY@_T(Byg79ChJNOMUdN#ZXibWJ zT*G(Dzrjs}rCLs@(jEZ45Qzo6nU98lr$ZUJy9)e)L;be>(%mPt!g6#t@s9nA;X`t~ zo4E`v&8zR`!u7-g^3Zjf5>uN*8&B9%2QC9GuEYFV_y=S4bn`}1D<(FIZAdK0u7sew z7=5SFKPWi)Ym3it8|Q8&+-7rXp5%j_(aT}*< zZDP-_WNv4stXZ?6fIYn=U-@Ex#<5aBPIDV|j^OeA_1So8avE@&OQt~n=Sd2l^Dh?A zX|lxaq=J&c8+JVRlR_wE6X82Vfr}S6k-$t*r24+V=G=g3}HW(85u3$q1-eSnVWI2cy=i{)e}kDmZlf-5$s6e+O{ z6?S?YoE?k~l`}FsZiTnRtqh?aX_TbXv^2ROS$h*#wtEqov8}UYY0>T=G%)mXW z771d<6;yXn_BF7z7CzaW9QVZ@>QyBPl1`x24iA$CC#-lylYUiVuPQ9?H%Kp9k*#bD z{Ymu`rkZY_Y0SO4+SY+%(3eNmt-Ad~@@F*%e~d>)7;!9@n|wV8GQ1ZpWH?|?LW6lX zZ64UrOn!AFp~wy{YU?CLh<-^VzyHG7h`N6m&g}u}snebPenlJjjc>61E+3xXd0|t_ zoqSiD57==VbR80#?D~TsRDwN=`>Q$-p~X_N_nq>8XR9#tR=|?(lFWSY`h}FTNU37k z3FDws&zPPMBG{^W;^^>WZc>e%V~@?jqa+(sxQ1ls`tZ8iy`d?wZiI|8vWAZ zn5)YhBSrH~L7OWKpE2IiDRyQ%(#7zJkCeyVj+g{QCYVY(-x;Vuqiv%g&R6q?zyE7i z`ZU~ZfQ61Ll2ioIf0Hn=jz&o0vs2yAvokg)ort&BPe_WWCad6t5>U6ges)&M;-;skO7Zm1BzpF)F zI4rLlR6gMTUgMMISL*I1ISGXAufb+^Ajv~h<!G(HaujbR%$LWeOqO?K*DSe0sp`*sX>y3NC%f;D;$nXm_aVP> z5Bpm*#jI(fNAfMt`DO;i1zgiYmn3t3UmDPz@@{Qo;=J;%_?klLp;KT2dmP`J`8VYCuBcfyV+CmCsW?=PdX)S&WNRa&p7A^8H^=#;TxR) zaqb{qOY4=KAqCgYda)ulCC(#y!^)$hTye*FFQ`>ue~s_W&oPW<DR4IHRhEFXw~9BpVPR zj4XDxjszJ`)cVNzhpV^9sbTLGusx(H)5ifnPD&%`;HmjGOhRHZg>qvK%lE4~sQX3; z@W|vHq_p^r$@l)f!Y)faM_<8yk9xbj*plGLgaJZL_XaSM{y9~oYkleJh(6TSUoIF? z5nQA9xX#V=ewXsqZCPpImmb{#g1a!zm@sl=9_DozVMy+~L^aQ7)P({63bL=90-t>Peh{5(=i7P7v8u3`9(IXvN7ENg{+nnpGpwLC z52M1dpezCK57aTIiq(@U_-ULx9j`42?l_r`6RD9U9!ZxyRfgMym=shsHN+xVR47#G zSkQxH37^t@jfMoRnAs7$65bY`ztyDN!F91Y zAs>3q#D&MzxIb1Vr*K0j8hy96{kI$(qO?NgxS^#yeIuoW@5DhJv0`K?E3AYaL4Bb5 zsB&8n)@X5RnGgK?nk{z|4{8K`8unXLBVCI6%H&QI9HM*>?fl(^(NYn@nyd)Jq`1SF zTzUEu#&f-!m$G;xSJp)~G-;vF*7Ur6AnwqgumrKn`&)i^1@7o~0aj2Udl!bWAF_f6 zmYz?>EsJht^RR=9HfjM?Tj7Bp6?KaL00*pQ$ zrVWE=A>HYBe6K0dn6CDo3@7y@m1+kq)uJQ(v(i5Pak`7Wv?~qjsAr^*@Wk!cItkq= zdJAiIfy?Dy4R3y5DLNT^G0`{vgpT{ENI zPXg5$VGG@Yd~K#*?It2Xu56ZX!C?rxmfn8KAw>ppDwCJ8!JwEnY6r{sm|^@I(JqB$ z*&zf{#H!?-Gt>mU+J@w2y<3&YaGJEEXywDYOj-7zaOpfdP^D)kvOF%uG37&dFUGHZ zEEfK&x3!eKt<^jH=$oOyCr`MfF4vth6RK^aSHfb7xng8Tc8HCRqg1KC;&a+9ddXKN z5Q}!HL@9GdeZ9oOTppk@6$c$bNH;<;ns3fgWe(YY*6vC8lus=lx>7uu4O85$cBk{u zgk*5WC;G)vf%3EktFX@X{XZme6rqNWlbO+X#&6T-p`9P1I}8_7(SH%N9dzlry|DZr z79in3elo>jq5I7p|Fxz2nf?Cm?V@{Y{cY$C{_T|g4K3gzOQ`-MJ+I*&e;=)qHt(M) z6MML;=F#9TBVxQ))sWs1U*<0D|3^UYZ~Zr_5e!2tVg}J_Be;mN?zTpTTF$eUPtFIU z&-?MPnU)fI$<(OI(XE2*wh`dqvbqrTC}Vq0RsQ6MXUVqq37rpUp!@+>F;@AuOigk! z z6m5&UdvFgDXbYjZ2M@&|#XZ>1`^gyJe~^r*j&_#=?-)OSb-kPBM9hx$<4LOrD<@BWe>WSEHx(%Q zu~=NMunbj(QJqQO<_tw8%Y37p_;n@V^a+H8_RA+!CUTum1>mbAXB42CzCpL$)-f-{P zeqYZy=8q5+lj7ydJlE6+Z@d-NioSYNX0buB&}zlwcYiS7M-Q)RtQBK%jw1K{Ns0E( zu%rm<8;j@L8_h0yuvC7U(klcKJ|i}PD7LApBPX}w)jRb zO#a93wfPtOOR82$G|#zSGK6x_=@03@VP9-qiAah(;gwGMpD9P}miX*=RYvQK;%s;r z%YgEHe;~gP|H7C|xq@V|TRaLXSK{7wU5w|88y`dS8jF^_US(%E-W2)kN%3k&Qw5qd z21Yh;$+3sosBql2OZ#h()7OIMxpV#u=&cj0EV8fVZT+V&_@ghDEuNhbJSf|^H#-uG zt*;?IdBdC!gwHSYsO-!Mio*&s8t`_1$)w&E&boCCoI4oFv|Auz&BU~x5kQ1J=w*J@ zEYe)uz6RS=sydf|G(7;*T_oPAS23I6SQdm7Xq(Qp?X=MqB|J69MZMWh)6ltd*xkGu zn{p@jgjmO;a<9!0P&Ypla7g7<%^H|;`YM09MxG(1{B8k6q*M z)uoPS@ns=KdsDSG#WGX2l^@^Ce-zo!v!rG^j=@VSl9xHZ!sGcGhq8wP+8|>EH%l_ zNiKa5b{kIr1jH*T0oH6f#@_(aTa1=Y9=W9H08!EgH zzmmIF`}yP|{Ml(7`r7I7Vm8S(Q_fZWw(3Cv^A{9pZ%zi?2MOO$Z{KPw+bf+rc+D7= zuJX{=WfcaxojVPs&y-}M>-2+jFNF+uWZ+*Z)n9WxSwoBYttINFw>F15G)Wqm>6`O~ z#izRoppBN-u*$QK_b~3@VTNSjJ}d*-ZaUL?UfoUnlfL;P@nw&hg0(yF@-6GqEd9JV zhu1Z%TERltBDD)6v!?DiPRAL?<=!{+26%FYoQwIAnnqhMY%z5hH$a?@hT$JVuweG=8q5hmDp%f4CXhv}Kb&IH#A(B4}m2l+$rd2vrSi*FV|5JLp- z!g~Od*B@8{iNK-D;SyEUXNkOUw&pmnfEb+!!BdmRQf`T?bs4j}I#{3Fef9KjXFdA2 ze{7Cf?*0Hwf#HdBxpsc^lX{-ITLAbA@!g64-NfY^MMO=%~Z zW5O`JhNd?)tZWePI7bRd)(Qws4+D^X3CVB3m3aW-9tSZFnLgbspMK0XhTTe^4?Q_1 z5vD-A#Gu^3*K*^1Jic-|8oKd}*r@&}X%3yLWiq#381?&QKU$uf5XJhb`8;%>w_Ttw+W7$I9%8Q9j1%36|G z_m)iASKWVQv05wHJAzb%=)Lre+%C(TYYqJ~@iFP_iaD^xoNZo!R%gCYD1O8ZqGByr zv8{J+Aa(tDwdZi*-haCOF%QCZ#%z%%5SN*DfsSm9bNThI7XZ6O4M3S|uFitz&T#nD z3;H9W`fFcnebJPzlER%lE{N|7)KGQHF19fgQe;0G{Jc^rZNEU)(|5k~b zx1@nCvF7Xcxp;dEu+S+;r=_V!gTR7&E%}x-los4&T8d)rnLVVh#|FJEhWh|*fA#B) zj@h0e``kGp6oAr7+~Zvb{`p~1A+J#x`+ZLgTdhcEdi;6dSk|_Lw0Oo)j!TTaV5d6i zmDvAy#SSJb)TEX{d*8KCAzpC+C{cmoeRh>`Ptwg?aQ2~u-zpAdo~7YFnKb`rOQ%+2 z;fwV3$88n<5U*ktai)y{o0Qs*BS&G`;F@ds$GS%|*IiZES1LM@rIM})SK+L_Mz+t4 zE&L{}(e<{(VSJ{aTW2uy0F4_bMWh>E*;w(3qizMY9XjaZ;eYQ~tnOpU--%p9qHeT4 z2h_lNKEwfmb>D+T=3xFKr*}t>H+3n8oJ*djmQHMj$mYx{rMY6BXzTC1WxNt2ua;6- zi(<73zQZm$s1{r_5XASuHjy&!wd%h*=9W4bDcR~LHwPr)>R>mCVXfwND@0}v=h_e{ zw4~;^bBrraJ3INUuPM#&?YskbsOYdwh(lv=347g>2*%bjBVTq7%)~pZL@Ab6-Yr$2 zDtO``L==^nTV_xl`-O^yA(IwQJD`X>UWJ7csbN@19|-pyNX$i{N|AVIEzlX3Ux zJhfFd`Dx;>dRt8ReXD3QP0&A5GM{!Lbv&-CSE$8|5USU>pIUGB$77T!^kvqvOuF>V zN@POwh+-n15%Re)-j}i(p4N)_qW{Npcf7OqKaC&5Jok`l0b2&;#O(GCT;{@%e2ruG zC%J}|rxVB1ly?@Nua5n%hHd?};bLUZ7l7O_dvl)Ovz`MItWJNNt4kqwa-Mz>hk3#WO!oofV?jl)nW8ald&R*qE6fyb}4gjKG(xQm&-q zz41q#j5-{*4#f$mFPtlh;r!D#(C1nPTmGb{Uu3Be`K$o2`NLpUqyAFo&GXr95+VY~ zi%@wV-2j)FOQviulQ2qLwOz~{JdSgB-mrqDgo_+Gdk9`IEhc_$q8Nu@Vdxn>cx2}_ z9+k48Q&phY{1AJU%=kQp9}Ztrn9Oo-jHW)Z9~ykhT~LI9NklN?&qr#F>>ZoAa#HvZ zVusm2)qrP1&G?=pGG1oicek02O2Hagt|)?=NJ+<6q4g&|9NJTRlGJ<7$mHA$1^8dQoNCJW<-!|*NRigEpF{KNF9IJ*Mm2XjHNGKcbAGE$_O(ioeo z#@6a1{PY#?+XCIw7o$aGb_FvMW(S<`sZg$7ORcpGe}r7G$!tRGtbf{Y4n_OK*gWkd z?A_Q`|MHsDTHYjCD{9+~cdeY6=MxvMnreSkz#Ixy(*pPw!;ioj_Y3(V3~lu5v)+XA z_77|QqF|(^Li$lXR>KG^qz>Z`rKQ8fmVI}1LD1-~m)}Y~>{6pLPLHMLfu-drD`B86c1aoqS#WpCK{{CR_u?J+xN&!TkkyBb#rh`>zI#T%Y+OuR46tw8*p=APa8vl zKPyNnN3?#@Zx+1e7h=j^qR7M?us4DKUY24WcmMdw4LkXa(NAU^iE6w{X~dyA>jJtq zpG5x-dWa^@rpsq5hFcOMn@9(?|6#R%dcS6eEggJVZB2Fz3BQ6l3Hq4=gg;U%C^rDa zq*ZeftPpF0*d7pJ#w@lLmBJLZgFf~+4#k%Q2N!RvoRc|gH{jTw{K=Ij&UH~STo~@e)RcJ95-(`|CziTE{|<<`d`=94={LfXL4VhiBW!gRubO)$#=-v||F6SR?~5-{m~_=wf=CiuU0xcoBU+&76s#*L zxH~UHV1?FXxPar2B)uEn{rzvR($mzS4eLedBXu^>o9Z)7UBexV(%`?{Y3IAiMle2i z5Y`uUnDyoD9rDAYY*p^AIApc(F8v9Zz}E0-+ebBb_a~xF_cX1SY_hE{+}e|ugX8ko;VNApwArc&qBc$ zEGP4nPr@gl*8J57xh(Kb!P~rYYse<#TnML?T%!*bOz`MK+4I|K4;qTXAnDSPFV5rr z2?s4j6C5)A5ej|Eh#E%)-hBu|V81QBY$n~V=_0aMj4>iJSXY-#Lh2~C<@(V~Dm z{S2t9(0h}JdDWPBwt$4fOA>%enjXBlc`Us)kaFbyc<2MMe-gsoqxEy zHw?o84@oLV*KPsz?*76-4teK4SJ(egGA=vl@~LvDB5oBC_Z4tvF7 z4$lJwwq?tLO|{YaGQ7RjSQagh)z{$Ow5#FA$k*AWu##MGzQd&wPhI*}A77IjzB-nQ zle$?S5Pi|_?HxLPv}m&bJP^0GDBvrJ;@r9yra58regWoFx`bE6-ca9qb_ov84$>uC zg0yki)hGmbb1`%zvD9vzjd0aJ)InYy$+v1hWZXS89M#G^LgfuzPRs3*-4{b1miMo+ z(_nWG0&Vcpf@*4)7Us~WaieUkn7iIy`v74X^UE$@0MX=(pKu|=p-O{4*a&Ce;pxg6^0k# zs>v#V)Jl@-Q{{e&*Oezx=yopP8^gh~96i>4nzq2-veZwkpXLw@&J z^MRxMqU;Q|;_NcYGHGrTGMXJ`qmLiBBAzb!Rb_cq4Qj_Ed1;?ttL=p!~+_8Oq1}1RuqOOEvWK?^G_HqtkktzM1|0!1JbfdV2|*XMQ6K(F+$bIAx=PO*F4_v&B}W zTbQx1>)xA?TLV4^sK_$*oo5j>p+3Bi@K3!GVcNL=R5x~|f}@YJB1{JH858J(F!|mT znS4@d_(=Se)S6QNdsJ))OJaEqoCow#ss9(FZqgVH>}+2H*@5f`^Z>VvZzA%##fA&+ zB5tTUr?eAswukQ&Z<9k-_sxLdvAcqgvgS12tC}p_3mQuYye7tF`Ass=)dZ-6SwWpz zJ(}LN>>q6KQLwW$+X=CtHCtzjnhtQ;o(2H79`5b7(3iCh-Fy>>&Ct^#`HUOB(0m@4 z8(Z9z`!VyD*APqbI^&`ddR3y_|Ac9%ZsfA|A6!(RC*sIJQN9ixi5N!(2p>IRYPa6# z{kBPol)J&8fuQt-(Bp+u1z5*Ca8I-+gwLPnrhr|xn8l#gU<=5Vx>bG49m2R5@S(vO zPJ24yo6?exSUN)pX!v)d9!!=7T|iaib_@Pmr?!>&auU391wBU@kB|A?M;J-f?TcwH zf=}(boE`!2ZO@U8qfuK;qt>CJ5mkibOaQvW%C)20Zj=Bp1-s# zs#gp>zq|ZdnKx>JoIZ>PZjY^YRvR|BV7=@d7a3oc_8rO;ao9fe-x=x1y0T|8I_-{O)Z~mFgh?n##QAE7SB))5>M^9BmMICp*O3-D5EV~$by9F)j}d5 z8X`k(Uf2G>oq0GPs*#JM)FCdgBdt8GOpz!fjL@C!$wI`6-^;DAvivFHeFgdkPTF8S z-(Ei`RWbPy-IiXzstan(EPJj>ipRsQ$D2IY-WdQ5(U$UNT?0_ii6->z!9UfZcEOOn z=`KWrcH00tGx26*)2RY9I@~5dijRH7&Z5W>ykh!-D5kLtY;?%0Mm|d(3heoO3Edj~ z>Nq_#tc!q8Mr|8=xLn3H7H{pMVp`0v6v&iL71NLS5B$SHiWQP9jVr5Dw_1GV{`1+_ z+4K4Cgjxaof^(oTtL~8Mc8sH-q234P;d#@W;^7PLhuyn5sfXM;aA?5d<600jXh((s z!=i>2Wzo?o4W%7q-vz`;OUznd4lsQ6ujxhD{13sx(EEhRs9JimtpW_c_!>W7%h$Lr zuf{kEHljGkPbKn@TP4*E@;X^{xY-|Pdjed}HpzJ&OR5SUzT7_E@f(Kn+`#p?XKWMg zQ(_GnUEl=6rt*&&74i}2Qrp$)PY;0>+HgLBq0ZEhhrfmwcmw-M^=O+h`CpGtib>=* zHMEpWDEYtWhw50c^xrqv;|mfemM}lf#pWrR7tzc*HQpIpN;@K34NW_bXz!J)#Ndxe ztw?oXbx@W--?>j3b55PZRaWca&0-UJBmkOy1qT3qZ@_^(`3nN00{x(5x$UO1y`YsA zs%!JM;*DN3_F0J7cz{ydwxLz@#qEI(aqq`5w(Y%)@gTQRP zL$q&tgUlnNV17$b1<0a2#D1nvzDWa|CDuAsR#!8QU8ghSw;%9%!#DjbR3CHRfpD1k zII-vOy~|w(B8aFKSRQKFkbjsr!SG7Q*BkeW^LjPqrL5bWkYVEln9<)a`z+LS4LvOE z61P{F=aeb71#GLc;boGlx$^XY&C1%94H5vN{D6*w$y=cf521_{$pas=L|0beMh{%f zo}*l%p8@h%h;N4|gXZLrTCpr~)C=-jr5!@;qd(o2(}|pO2aZy2`?EsaobmQ)tCFCJ zV>aE)pb+Qdg%sF?9k^#~{ctFMcV{kSw)=0V0Ee0koq-{IuDCk~wTJGvUZY8?Lm0y>MTS_Lf~eTzgtIrZFTB850lES zker83FzI20nEWfpmZ}vz#|cBz=EJmeX$kIQ!_$eOi!b3%Q7EaUyur}K_J1mOF1;Q( z`J!Z#t(w0j8!2|wRC8>)D5ofoMp6oE6$1Dg3sg)UDpUoMI>Pquo8#xyvv z&W)XB#x&vMs^YCn5%T?_u+djUs+PyA#y9u*%(o7e=ij25P*Cavp4?zDB?udKL#-yE zXC?i$4sk;*$!{Y0Mheil3Pcu8un?t*Q;yo<F65U)ICAA)V4$f!8^AyF4a)hv(4b z5T7hqbn|^@n(;=U%LaoC9`q=;6txo{IGxvWcVda(#u60Tb>VAhdBUmxfSuJfqykaG zXGdDhPTq?Q2Uy)9Ncf8LuiUzg@1{_*4Fj^@MYMpr5OAjD2;4+==O6{9IdO zEW>mm#QqngmNu^(JDhPXsn-B;ou@O=&Gzo1>sp;2s)yQy%`Um!AI#dPMtg-!I)v-+ z+`W8b12J)-t6z4l^eIp|FcYR}u5SzEU9|0BTYeDe9x+Ow*EYeEu%gz73wFY>J zzV`>vpeMxszc(DL1K%SYd8kh<3Et&@N7SN7Qlp(@M`^3NYR%xk4qFHRuNR>3^vkpq zOYxIT`!KtKu5%N}c;*D`Fj|}5JeM7~Yu#cUs6EHOd1SFZ&>imt z=)awH*P8aed_8yIOXDk*)Exe{98{P(K!Jdpts>nVHl}`gYpIqu5Wb?!5yJEjW-4+I zStP*9+)}bMJ9{9ml^n}W-1BTjTC4>>i1kxC`fB(-Fx3H%@uFew|PpOGU{#f z3;(wvfhlB;dK=~PdtQp1GwQM1>~`eQii*Tl=n;|9trU78 zcR7?v^d7bN?_YP#G+Q$m1Ed zoq+4z0)Rq=)*G`a#cYX+n35-uD(_}m>lX>uu2dtX9ijU5=qdCyR+ss|&r1pF ze=xD9i?5?6#4sl-C1bQlRlJ?&LJcVDO=&a}Yu4YZ9=BW(2ej7;9{rzJHb}&dRVB=z z2PsWVwY4&S1tSa6!P$|38{c~-fi-`~pJ+N*x*hJ%^VOmLL0m;vAdlH__?sQ&7diyn zb``gEjgN38>EV#n=6lYW`-{{34b_|6_%}QMb!^K#i8+&h4FfR@6xHV(;qsvY~_37Tw&Q)#Qz_OuR zeAgX~GHGV4BmB9!V|L5~o}YSQ(c14$j+;o{xj*vS$rJ|}@mz3Qw#Wa@+o~1TjPP-P zYQIeh{()t2pt3^7D_D&Q0Ojywa7Dfu5c}}Ol(+>8Y-$a4tS%;Y7%Y)~04RV0y{3jgU37W>fi>lif%G0NCF-jpEJ0+ZZd}CO4PwX`+<=s7} zoB=ZP@@V3*&RR3EbZmnOX5)ma9t`X{XL0xcT*E`*mmBZMSYtqHC*vqmw&P0KY#0zh z=R!y1ybD&~t}D#jyQthb2mzyb5R!O9y%OQfTEC26CS!sDy<1`DR2U|CF=$bi07e!8 zQutIYFpqTg{es}`;#QE<>d9X1T|+!@u%OzilGOWI9jybo*|m&hgGTuO}SM^g!I2P3XqNxBsizvo$nrAC7PPYadlGbEqI)HDpZX!^n{2ok$BdUP~U|(rao=J zuxquM8*ip#%;$zRmvD|{MwvJ;tn1Mayqps=vbKEW{Nk71=cUT9o3C#~%P13qsYTH**Eb4V8bQmBC13=BMMINeT8 zr@wM`%pFF~LmxdGhxQs^*)Zo}v*aVAJXOdF0*~-^>9{{!9I{aNb06E)q#H(B{TXZ8 z+eFXnc@q+tlu0c2yJmP;C~$>rxx=%ZXz@aU;%Oy-!{}U}^^@bR<#Fv#jF0xuFv@Aq zLL00xts-_uGQ2YTm;T3Z-b5!U7I_g^Q@N_C`KZx-TmQ`($nz{$iJf_K?Sev(jO^=( z1Gy%`Ny^C1oNiP1<)2g{CU1a8ci~};C`of?ui)UeL0?7o){Mvi4KzplKd1rSX7gm_qYsT%-o=Gv*TE;R=y@H~1x;SifAu`<WGhUo|iR(H9f43JwMFEqS%QO>V=F!wPMA+@nST*tfGyr;DmZ z^!f7*oLWY!w|&%@@-3-FavZgXHTSyrHc+`lS7>@6AHYs+oIKdu49QYA5;dIvCbRzY zW_O1irHxf>FvBsTaTfd{t$b`tVRX8eloTaHCgvvWn*h#7VV$8G{#LSKLC~ANkoL_? zzN0riN1LYKX6rjnigroN8Kuf)pSyPJOrvTO1Ch1ck%@uYR7-Q&(_M$Mq2WzEbwpS&ACa1)Df2Jxx@XYCN6%o(k+y8NxXGjqW)xM8m z6sEstl6)gC$+$!MB4W4}%FJt2r?BxI+tG0`;upB3q8b?)5CyextUxA2Mafo3{In{Q zsPbqJ;qH zQYQ%eU*`=KC-wzt=ca6D$p0jz?1Q$x;85E(JX_1xmaLV*G@TAjN?$dP4LTy*^lxJu zH{*W^bCISa3X;}ct5yu54IdADX28@kc(q4iUZ*XwE_^1Zv&)vyO)55cCcJyvcO_*3 z4!t=(bV@U@uamI00DML|xa`Vkw`tXYZ#Q_LDx;T?ZJj*0hjqVe6IRM_|qc zi24;=J7Hkr@ds_UH4BFwj2(glb{Yto@xrhl0#cuWf@w|{$&K(4PXpZ^i4MQ%l^HNPX%XH2uOQ#+m(-Ym4PdTk?v2MPSDvR^c5h4?oJzH=L7UD?wR7@AuhGOc zdTX4hBIgZ4dfTrOT*?{TvMp2>F%0U-3KCr+(iB1Ig31YZb@rza`(K|}r9HBe(Gt_0 zX-^Ni%QjA@!c#{}rwK^|i@q&r4+2#N9&H)&Rs|RaHQLpSXl0n`3g7*$BQ*D^gqGs+2 zh}xUtn{ku2a*2aiT|jXgP%z`1!QpS~qt{(W3vs^cSL-!Xvd>nxyM5(XFLrXKfJ z6UceVtj4CSHFM4wY6AkP+|0U3nB#|hku-y4baiOTm>8YdD~H{nESAy$;*xm zw?Kw>xf~90TSJZ1EQO^V3<=S)jEe)D)HG=6E8HOiU4Z_o+K8Z>g6m&FT@oR0p zS={~gJwPC9U*?V8k&=Ec?PzrO8^F%5$tdB_Z|4ClS*TI(Ipow@|D*~vAUnv>>0JHe z;&(ijEQX=@wO`-yk~Ur4jteSRBewDKzp!1{>Gs8K2jI1u)h@d6DK$~5vHY4G5T<5O zi=+2m>Jz#hocNh>Bh8CkQ;I8fDR!Z-Ma&$$BeXu2N= ziGr6ttSv;{P4RObyS)oHhMRp@6Z-7SNCYKfmP6PjoIZ$_Rew9G#8L`Rf7z~mX4=z6 zA^SnM7L?I=b0D1bVOa$3UY7v=uPgoRvM?igdaON*YB(wB= zJj0uy&LpW!2M#@43(Q?jHaW?5-P2vO>)f|I83cqFIhA1Y4_hJ9VhgI#86=%MxnS4@Kwct6~ZqqK=U;kl;^9d1nxm9BLWynr9&7qyE<96(CcDdhW%n zD)x(_XU=j>Ge^b*V)fQ0ANIwj*LS9?2CJZu^X~|CPxgG}4hg<`oOW-D>uHR#RUG|? zfLZ)$thztEynbv75t!|!dlUEj1fQM548oHc=Ihh!SRfZ!cS2!gO3Z$?#@H`jvAog< zzkq)gtJH2f%|hTP&!;i?^48bDb+EVhLtd@B@JwWxI+A1j(dzvSj?;6OLAaXO^sDBf zb^efqp4yZ*P@No!mmAjzV5V)_f@FYaHsB!n{l zq~4)>r06d&Bvm$NDfURPy5BO-2bv4Kb8P}AH zT|9<2*4|Ej*NJ7$t+g3?d;CHd{4FUbpGOzZ-!KO&LkA4>|Gx?Qwks|^!W|u>23F%S3$74tJnJ#=N`<9jKohz zO8jNi+vZu?qV7Vv11-1a8QQ`V56ndI)PkByI)3ntqg)EFXP`diVu|^8o+b0xIrOXM z8S0{ht!%}Nbtoavlr(z2J1YAca9%YdJO8^7We~**Uw@+8X zo5c`2aY+8p;fa~o#Q{0Ga^Yd$mQ0UI4T_v)%6G3$X$K1}(%&5b@*<~y1`+;GRrGoEF(Sg(`b9H70KcTe`C*m5~rloy>!;`CHqxWD_bC;psXM(C#Sx?mgY%+ya^ z$~Lh=)N5qsI#Z@Or6KL;Z_5v3G1d2~4e6qWU&Xw0*jDOw2}z+mr2uYK1KpOlwJDsX zw$2KX*+Pp9t5qY)3SqN%Fysw*e5h9J~0KByP!k$_jKENTc)pN!ra$?D&vuqcvF)dyq$ zqgWLoPt{hUoV^pduA)ZhHdLuzsPZAfqBs*AzYSRu+_dT?|iyyIo>Uks}Iw! zH{2#z2ms%Yp!_^j3v@@Tm&aRFbQ>CE-_Dw-Jp{({RU%ZUJbx)&wtq>l$=pG08nK!udh_rSrBdYpK z{An2bPR)a3*`WX4eh7^W{<(A?S2>bcPiLaDPY#nZFT(ZB1YXbro`G2F z(@A~cC+RMoH;#ccye)5@m>+D6vv8qaxNn}P01zc z%1sopRb+((ypYX_OH+{mB&8L1yNUo3uRTf<3qZ_^4Aj?)izXOY4XXQ}a1GwYne{Hu zDdC7|Oe~s*v1w&{oX6l<5>LGS);gBJ7QNyK^57L*D1x9zjCK^wLkuDz2b5|bma0IJ|VMLh}*>-*j zV}VPwgGE@&h{oCze2a0bw(zx#Iu*;(=_DmQNT9}?kih9_qni}opdCgn!{&K}JC-Ix zSnd)A#9P)#K{LXE!F*~wK9Kaw9-2_^Mp3W&BA*50AHUzbZ|V^-kY8O0Q4<@K*OYxQ z4SJDivFdK_ll>#)9K--&Ex=RM>NRYEpAg_!5Z`~~&o@OK?1?Q&ES;TM_FCfh_>`Do z*V>0lZ#Oivh^|^}&k9>7fc*#N&TTq43M>ImgYD}bSgM5fj54Wc$enJ`WTRth^eapm z{$P@VcaxRd&r!r!PS~eDMyeiAOpZ0YDQ?*i`yJ1v51QO|g? zSW#Wxf~E}%U`4W-beNPVCW~I(&Bh2O8kyOuxyTHf9%>vKmSMR2a3e-nnhGdwuG+{U zqS#L)uVSultVR{2Hen@0f{Zr$WH{<8oM_P`{c`O)5_(|orI)e79i?{Tj z_DWS10@k&lMJA|J@FB=Y5pIZeu4K?7*GBN zqtOQ~GK)OddN+%)cta#ujtSJEi%|k*+pos9SZLj@EYca37$ZczXg=-ROlH<#ry_DO zLAxT!*j4&!BKj#nV8`zpdMs_R@uU~bZWxicFOn-%g=->ItYJZ3L; zP^`R6O(9$h?byyoS#7wHl0NuKM~1E^*Usi=t=(@A{i2oR`uNaaQ3zn-o zuyUe0TpAMn%{`WuX{TUpNr*$>3-B38Qct+{8p2l$?pZTA8_Zb>xQjRGj6WHFci}cc zb4cmF?G|}7O!E(fec6m<7rk_{U&im}qc6s624f4X3>q!RW|M6*ph@tKZRWIlc@%`bo2~rLP)o4uqI9#h78S z02pPkmRk{>q_hP8lsei89jkeb(wf%~d!x5Gpf8SjDB60IG3@86EnaYW7;}&2a`*Mc zx?jst*bgOJI>6jf~I$nUB`o}&sd3~6TA4^I4?+XENXgF@I1kQ@9F-j{benyukx#a zJ52*d8cLsm-FPcdOz2(knUjJL-TD5XZmddE+C9G4EAiH_PCVQKpOXAwB9dmu8K$GPBV{BR#%1P&yUKD5)O6QMpJLR< z%K5zm$ujhqKS!8!_!PWUHp|^QaZ~Am*QbeHSNnC+NA`?z{ZODt79M(F_~c4YEX$F@ zYcyd9M9se-o^=@G6f1)(xJQS0xHyIno0xUFQBI_W#M&HekbczSHIM2~1idQCZC;)( zAhap%(sO$D&H%~Cvaya{Ytv2Lc%tJ=u^bGWn}338;N&56U}ptA-q5*I_@|o1DOQB? z@6LGg(3^WNXZG(zO{e$VX#rRnS>DwTRe@M0xswV3H|EyAoMU?xyYQI!#@Thkz$JPG z)VhTON@r@O`ld-npm#_d97AR*-MK4{8V;_K@K?CPbOm4VdUKNdU%kmIXJcB3@!D@S zSQ+ARDGCx$4)PvSrOs64z7GhRtt0>(Yti@;pi^-6CjECg`p(;`>gcmE@Xi3#pLiWh z!cG;|;QN+g1q6Kz+{JyEtDpj)Idz7x0tRbuFeawcHudVa&`mhb?=v%Q^qcO`#EJ&v zAVfg~mZj*t7o_12r#{W-$&I>$Nu7Jn*d0H)1HZFvxt{;$oA-Mvk4zn#jr|_tcJpr!V{i9ltX?OaBsWt zEyl~1AzR;6c-!7|r7rOfvBq?^e=>zCm#0C6iqS$&YtHgxGF;!C;b7U#=9Qw|+ZQGp zOZPrx+fz|+1{Fn$+}afa55d3$NjuD)PV9Gv!G%LCEW6INd)FC)`!+6hZWTzZg_of* z9LQcR!>&*u;<3_nSvd_lVCWY<%-v2Eo0KvpH5C{CZT;^&T9(lBQuFtM2;ZZKcW4a? zx^7p|+Mbuy_)Bg-L7|04bp&XbNIPfzQy-Zae9*fpUa+e`Oar;J9^2<{-#O!1Zm^bq zxN{P70|FA3hiImuwcWo%i9!v(E zngVZbwXr%68vLHHYL6ob6o0MxM|p;ovqJ5w-mKb03x?}?_@Y51bUU}gy|!->DvZgv zL{`(Etz(f?4$O#Hgz_}ExMC_ITCuC^Ve3yzlJJ9dTe_d~Ur0|()msFQR8<-&sC~|m zCR%;Fep7CZY9)1qAfwWl{OPCzX{xFutV4s@7XiKnH)>*+2zPiNRNa6Y7C{_3@w7cv zCc#j{QU#g(iei!zt6W)mZAOkVMur>fJEtmZqdU$s>(I}wN{sa<)8)tABHAi*{QRl^ z*9+kP4wr$9s=Y>Kr8Y=&-1B3f2>D;#X}KA(!?*e?_L}rwFUMly74iuT;dLRhn4GFA zYn|UI-^l8fHYCWEt5|QH`c6XY2xt?a821@S$J@Of=HFgPkMB>LM&46W#(cfU^*OA4 zF7LNl%{7={;1f47)=mXl^a#k1u%ajU-+kspCJDit#R~Ho^T2)rbKj>==naASWA^v< z?bE+2J!W^=p%6B%U56)dk+wlqU9Z*BPbx3qi#}s>gK> z3N$rjV3rhIYgNH-uDddkJvH!Uj@yD;)Afbesdb{1v?mX(1LNsXTU<59)Dzl47bZp1S@@Px!^@c!lJk6yA!GzW@_ryWoni)DoN z=c)TG%e89^_wO8X-1H`6R+(Waky;Z0aYV%(Nx-h$4Oj;YO3F=^Gsl+NOXNmSvzq zCH2$#)HTKLa)1glfiA{;Zf%S8Yw)mlv7$@Sr>q*;mbTib!misEfKEVPWvwf&2m~NB zEp=31sPvhDgjKoq12V3OxX_t{XlD)v*t0PpITNuqmNP+FomT>1Nqn~5)Nh|ycC{lgXz?qU&&2J3``N1u1&3%kD5#TYwHXZ|PX#zg_H+5vgi1kmF zHkFk!=fPxL=$wuye!<>81_SJ)<#dmgFVdGzz$u+i@s+Zx0{NmXDw(E$d$soIF*uR2 zn$Nob=@@7mFW|GHPtAL7tnowZ{P22|G4BV-yjSa0AU&!dN}ddPH!MPx90bn6z_NT6 zSf0;i1B5OxjrFl(na=CvT$Oi*tm%RC-TE$(%^@SNe_k3Jd}{u4O;GOwJ%3F)auAi0 z<=?20wN3llHpZW2m`R;b=J)vV+!}La!QAW!50JPO|?vMSa2xy=&4Ka?X7~mX_1jiV}yUhOq*7a5O zmwoI=U%>m8{~_dmgj$2O3`X3w)Q34MEAi~QO86Suv9k@RZ3ouwJc9d{l;WoMz4*;P z+?chf5L2JrhzXCZLgwE-#U)Ry#dWXcV#43wz*TR4fj@s;8PS z?pup@pZ*gboc{oJH6O&$qZMf0w+8Kv`ItQNeDv!Zfyl@RoEi~<@1GWlQ_eUUCFO-^ zsINkMb0u2$u0zrCXK>EXzl%M4H{-@j((r@GNc3e}KkS1(r$nOfX_4r2S|tAQ#uGSH z{W-q$zKfV2_r*Q8T#Ct4a!|Xc7HB_!xewijXBRz=BWOm|t{T+U)nNaYB}n+clhHRK z0zdrzPjPzRNC9YL&;233cUmO+o!$?T{Z1CZG$#6N9ND!7)g|v^+Vm_OY&?wR%il%T zxFob4_Tst6Zo`6wk78%tevHb_!pap((7LAxze$h7_RZ^XxM?TmJv|2to_h$-J#icU za>Hb_9In8j1KThJ@{#8#^#%>2lf0DzlO;;BGMX*r6GW78~?Us{Pg?{*0aBve+O1Ro3C zRx8I!lz7Ue{z7jB3JhyWf>QWT~uv3@otUc9V1>liRf_EhZ>(&+F+@JTy(shN{P|m(-D@xS|$!l0m z2KSFvq;8J~B_6Lq_cjpdEEYah36LwGq2WtZ0e;Bo!l!!v)3 z=imGo8_H`@SiW7JNk2sf6ga!&qs92)56;5#uYZUQz8V2v19)Trzy!|~fLV!x(uz(3 zpP8Vw)MJ!4K~M`&9tyPWB2D@0-6HZWH4B3>O&R!S8=?AzZl1^sCOL$Y@xxxuzk?F$wYZ3?$gor0sZnrYOx(1je*QlA-KTnWmzrz&8RG zN=Z}-x5cW+IT29=Lq|-9cXu1crC*Muq)a4GiM1%gf!8MJDu9sGF;&JP-kE`TM;f9X zX^3+vS_j(G1q_w8Lny0O1-2uI5UAW*`@?Y`JL?u~uWQHeue}?CQpO=(KsTeW0;)|w zlyefsUVJmQ*SF*M*WZIdDQS{VKwtwbRiG)~6*-7b$xs%jejrW(sst%a3o)iyL01L5 z#@J|SsN_U}f_m%&`(f=fuQjM@O&4H=02={Wo04k+YYJTB7^NZJmWBk8U5+&^dKC!G zcb<%eeZ0s$X_6zZ1Zu@{HcMnZ)|QEQ@ku+u*jVVAPAh#U;#z^EoNr2|c*-@#nT`Pt zf{x=6uPsXrUycHd8vu)pUyzJyD^~^d2LZ5B&e+%xHzjuOXKD zl$&g!v483eq+N6i_SCiG>KkrD9M>{G{nqtKjjxtBYpmqZ00U|HPM*6cC*u*HlIGv6 zI1%F>st&Gi?0>*~<~X@1*ArB5<{-*70b?${8P)aexca)kB8g)s8#^W#rhFW;ubGIp zr5o1M1ddJX`cO3JG_ySWpMfY_rtq_;<#j#QSmv{1iaKFo%aH%2oXZ5;XL zW%(&{f}nmjqR0n(76x$rxA8rvK<2m<&R-?x++?ApCf^HuM+uI z>qis)fNO8v3vdiN33B>unw%!s=-!I^=RJK4n*+xBAn&>fNN{F=z^&$kzh6P?f%)Fj z?=17Z#<39)-980T_F0H`&A`^3hcPZ^nqe(&=9RV87!TDy@{?=3isjS+*B2)@W4QUJ z#v$Ix_d$ldxA|^}xaJ)<{ydgln(GVEQuhUp>|c+Z(RMs|_n*YzG`#uC(cHWn^>tOK+f{_DQ3J8)rDt*QNIkx6*pH(P)o44Ei?)NC zux#l|7@zIJuP+@X?gH8lmEe!pPQh#MEJUs^56+z6OapbIWQ+u>Tg^3((P&S{aT*iweG zfBL_8{HbSfu<0Ngo3^37ZXI^G-^BEpnK;;Z5G$6xCRVPrem&i=evRx$5cpIq|LkM5 z9Vo)|k-rox+rE7x#Pas}=l+IgpSlgF_ElD{w3@vz?gG^A+l-@)WyZ%&zFB~)R>H%v zd3-j^i!d+3G7y%5Z@LVqk6WsbUkT`XxKKb>ZSC4dKkOE_0PIfyxY|&JcV4+0*ZpQB zKL2suo{5Nv@8O;~Pb0UC08a(H?m{g6@I9RV-5($#;#5RL_C;jB)A7t3 zOR&Cd7xK$0kyl&-r)vZrcyPXex3rLDDDhUH%u|F<7k_}${_FdQ=+h4ok$uqT)HCqx z>r1i0w-0&V9VqZ_6HtsmocONnF*wyZD64=sg1fX@tVC&PDN0M-c;&kZ_&s#0yrYtbK1NHda z3fyS`PsdxuO4jYI#FCFb#+m>9Lp=A!M*_MQl&N- zJ{WYR4^@u>)d@0+rRy<_p;(<7zE+h5s;0m9u>k8{x9jDJ2*Ka5`9ko4i>z*9P7*WV8+ag@w+Rp z6j`Ob&YC$3zrFHz$n{lWQyFDg$(goB?j6{wWzITwyf&=J>Hfi2gLnZ{wM7%H&!*u5 z$0djxdI_oy9K{g;M*%!I?->E$2rgZiIum0qx&;Rs+tCJ~89;ptaKm-Cz?M7&u_-xn zx^APp5)VAKKwKD538gaC-!Td2rCx+Fv;K&EjqPXw&P2{YNikYs{x;>8i-N7(s2B>OK0@nPK26I|4FCxE zX#iJDBj_44awdFx+K`d{8`uVAVgQe_OZ!w-t;(mqHqxCrVr_cikW28n`+(3#J~se3 z*b3yNPk}3G7zQNMYEK;_)pUf4MF|$b*!*ykPGijVTuw0%5Vubh(3K$7)z{sP#N@FG zaIjeFF)#Z!=Hi3^1vw!inu^2$WS%6nHp4xge&~jEE@$DGP_z*C2PQit)>FDp6h=(430qcsk;QZ8S zcwup%9OM>M3h>(BF#+eNPQ|ZonIm(7ozk{DnS=`!V$@{uVa6zHnPVLDKaMy@nzA<3psU1ITFh&0k!sRuKC`B=|DixvYaX@fz_o9RtWk0H zv50e~BPw+|GA{lj_8xA>AFlti+F&w3S1E4+sx`=FjTIj$%fw-(AYtfqY(3b9Isk`S zfz0vKkTPf_Xelmv^*Up*3_e~N(fudrdeyaRJ`+qO|Ez&72*zG~i)^A;L04HPgdSr) z(;bL24e>6n(Z;5PA1E`yMk^o08d~PK0!O)t$IU!~X^Qs>ykuXk<@9wh53FFIwn&b4 zPR5-1FW`&9EjZUT4U0Z0l6Q+<**6tz7Ww8FkDGBdb{%cUVZL|Tft#-V6P(FI@y17M z@#=@`Fvvb0gY0Q|eaS|=^3hrhu#FY4KEc@d(PJf6%7*J0wGJD=sNHZs!g(YzLf#Q@ z<$B1Vfkm_5x=y`Lg}M*sgXD2cMwER9VjWYETUv<+=5ZdG?@uk~9m}e9l>DQd$D~fc z!%sYex8DCm-oGlp>JRG(+@w$^B9uFdVBl5VN7<$#%6>Tp4!#uIcJ9acoC%1xry0OE zLGM6UBC0+8t|q@|F+2^i&M8>ysg`#i|BLXy#-ZkRjLo_Ti7BHn&^{g!GZt;YtQXef z@|V}(mN!;m!RJNT-rSBYjqTWYq#ZM!Sc7p7et{A9Z9wWBtC0GqZ}bb_Tlc!oBONFmF`}Mu~fYDPlHHhKb2Pee=+# zYhx95Y+j1BD;MIXYtztPzZQEoe~8Owj6-d;56#Uxarl4-?adYV;MKW!>ctn~*;u;TTevty6em^n-KRdIJxFm=m zID675=+n2axFdM}xx3L`w+1^(UdOZ<1YHl~^G{#Hq%kpQJ6M9}#nSZ=?5aP2(b+jz zM$nZX;UaKMU$+fg(NtHB1&{v~x7<7n2M?6t{9pVGE0!(6!NxCf_{dHiY4D=0sYHC} ziCY1EoU@aoo59L)_};Xz3iBe&i?9rYW#C&U0~*vk#&-fiQi87S@)5^kv2?xmvf&CS z?WQi4efQPBg6;`ce)5Ib5DVJ+B52T6gCC6L2&T9f*lvQZ zo>G+NevWf~bO!$R;NvLp?#8C#Y78AS5lh#uN0HBqMbAHnQ~I8aSN`#iSn|;t^Mg-5 zMZf<#2X{aGEb_`Ku%V5s?@yFgne>}l;H?rXOJ#9NYft(lWl+8} z#qB}R2d8MQ>vfmlrA5!-{HSQG%_~8^X9x1Uwa6>mflc160us|Qv)EIPB5yeg<%s%L zv2@HAU#IL7pEIbBG>NsOuL{NPZSa0kgbUBP0H3TY#G1+)Krl)+=TA=p$#YgmHz z*cm(d9GhfDu;ADZH9@a}>vclk_8Z(Y3`B>udp{S%3_EeUCTz&A( zTW$EtC3s59)he;nRv&{S4+v7KqyCE5N@mz@p^w(W{}{Qx8k7{|V%T6Oo_YOatSj3qa-soO<^DkfsJ)5lN0#Gbjxzy!HEuhUf+nDf z0^j;OrXqIuZ&CSW8y=kh91gO3F1#Q3-l{W#fyR0a_`{%Q~^(EMH%nN600ow?4$Cje2CKeidIYW331?HZ6M;;6M<9$B+BF zrr`Wxm*I2IK|J*EV{qD25a$?y-`@OJ!4c*?v;fJ;V=yp<7OxsK4^-l{BC9OS$Qvrr z^oeKCO7{d}-DUL4;8B0WpD*i9OjZ^~F=COdER1O3Ou*#A!L*Re#vofd5>t4zm9%q1 zjx^Do!c_T)*}9T#_}jc^gs-&rjiW0GaXUauQG&)w$tHc|FRe`VxD$BUSiA)fJo>Z( zUg=v@#)G~Ya|Gz6b#(%*VFk3(mb2_zhGC&iYc&OcM7yTq{K2yXSoT$H$B>a3IB)O_ zEL!Zw!*id+;FKgdl2VYAJXH7^<(z_F-TDwbds^TcnS?KcWu|65Az1y*5{U+4b z?8N9{V=%}z2C>fZxa`(@;M;Qqu3^(LFqsx=6A|m;{K+w_$~bRm`Nog!DXT<9Kjk|^ zj_gM{rXnh31`>zNLiLwNasT|MkzgB(MEh85%q>Dj4rL(CsFR?7w&Av93S#XuFu*Yx zd2S#6K7T&qoV2>9)qExrT%7mBEpklX!D4C5xz0IH&`|k!Rck;t2#V3FR?T~E{A7W0 zOL=BJ1C!H`n4E?g`)J|rB*-%v8z^s&Jtx-uoTG`h5lFU=P?qIlMM=d;ep4VmHyFJwF1=HHu*|79OQ#qZ#bWM zdVymTZ7kAR!DDpl*X0G570SJ$sBPRT2I?wX48T(dClqFdps z*n!j$8Te)D3@o(FXJ@h_05=+o`ZzcESb-L zxb81VwvR&mut`{5x)ax2e>;*>hKmI}*JovI&pFED?xdMMRLLj0;vipXv2XCj4_GVk zouF%H>wyAj&AA_ZFXJy85quvR{38EtQ!&sr8M!58cyQk1h;^okeDXcSAQrm9W0q5{ z!8q<*hq*qbU34>c9Z?&o|9tm7IaQ(NIp1sQWQX{7H+%zg?o;*^(9HKFeIIkY+4oE( zEB48VPPrTdQ-6i3oqI7pn`?TSfa(fp=lrIen42Zq%@6q_8x^*hh;vTE7p0qV-(ycA z!Nm>0H2m(Sdj(LR`{+|hvX4TW%T$5l68 zgwpNBh)+Hb&2<%G)wDYIu7S~^&fje$aL))QZtpDgG@t=C?>8GK#)`LU)y*PSk2d=yN z3edWA{G_o0^3J|`B&Lq~84A}vgS7GSnEUht==0rv=yPfxoZKe@r=ApvKBt_F&)2QQ z+SRY))<0yS?eJE#9jr!MT@{)ewxgkOCz@Km#DRu=I8eU}Pd;@IUVi3w+oJrrU z1Y}P+1^3*0Gfs(&5I?Wt)0O^W`$Z!1lvAkU_q0>M^>>nC zUVxUa>Q9*7iB^Byw52OQ&gAj)%g`zSGGqiPYEoRdJB-7zX50d@I^C#!w z?!PY-u#6y?9HlQU$4k$=i0}0K9+s`!h>hjd$n)(&p|=(#r99qUhJsSM9;gw3&h06M zPr$*=$o1A>Q~6E-)mT@t92YM~iF*r*+ywBdP*7Hd!ZLzX_&liCra0q21@Jw$~8P}8SrH8#r4^zIak z(?Ytv@cP7sg`0G^X>nR*9QWS_cc~8*MQd^C%t^T8;f2`ft-)sZ8Uu8F4C}mmvA*n} z$h@y~6JA*O1kQ;`#qz=(V&z>>;)Y{bCSHE^9aNTVz>Cj4h2&Aw#X{TXF2G9*pGMNC znOIZS0Dj;~gXz{-0mG*C@7yOB;*u-=fahQSrvVzPXOuyvU3%p;nET`+0bG^OV_jBR zZRh%S2>3!}Nh?47ol`LK;=f>1bt_UwWFaYeFb3Gi3cz`RQ(3^)9%#TrbLaW}913ve z%E7O0z6bkTf%Nn%;7lHef}%pqnLih?_Oa;ioQm^Oe~sATm!S5`20S$HF${2y5rtj= zoz!exddq$C4UDwRE0CNx9!be##0O(EfgQ&b0o?}D8jeR79n%Hyi*Yi~P$b$%8Nk|9 zac@8%mhHwnMhl&^R-$_Z_Om~Mh}6mWW$Gn(X-Nqlp8Ggl$%EibPJzufRFrG#Ok%9! zoP_gJrwd3c0D^t2fG`7XW5s%jAa#FQY`N0JpWXob7$n(8Bf&lvF(W36dx7lqE8rL~ z9tla=%JNZ^P3mOyX{vz~rNnE6RQNUY*H}?{0FORA7pX~!G7j_+n>OnP)EsU{cE(KD z6G!+h<!1zbnH4UnXNv zl2YztTvW2h3NMrFBQd}+3jJN_h)&JHfRqd*BxNWt)Rm0{*JuoMjzw(pBmvbnm6T)7 zy!l8>86novwCv+C{W#ZX#HA9DqxBbQOc%vE(Kbwg!6?Tx#H3EcCU*rMp7$7%?Zd)LUhV>WKX#UWt+ER$f)5M;7pUTA=pN%yC_$d@@<@wg#rAo z3;{UTBn(KIj--^y$SjGVzkY36H=OxEm{KUFOBi1z<(IUIq7?hNa2cJXBLo@^>_z}rt`*8cnEo=k!9nmm z#xYI-Z4O$Xa`Pii<}m>&`l5|W%|KM@L?on4LP8QZl4wa!Hxp?XXj9|Nc^{pcjVPCz z&je`6^O(U|7~tZ3AB|X+cT7Q^yApFA<$UIRpj#FK(BwJ&c8)i!I;SE=HqXY$+!$mV zjcDg|#15Vz7MXMA&O@SDL5~$cn0}-wE2KBYF-kyt0_y@&a&twm7tYa$qbmTzO-X;} zH2iY##dvXX5gvKy38W+s#rX#4`q11bkeZSNXR-sy$&{xl7%*few(hRO=m{6&Jm(A~ z44s8NHG6QwwKt-F>U5dUg+)FjrwqpVuIc#It@BX8`J8N*G1c!$#(YN*XcdcB1B{JI zF-{&>Kv#24>i^G-^;q{T)<7@8P5K(>duEU=6G`?AUe80C-r>CxX}&Xf0}8I5S&xe{;Bzgqmi_B$<7 zj^xvy>nW%95f=b`=mMZ`1Wy0nci=A1M;pqpt>PUd{rqJ7^7KBEzu(D`l9yJoXZ4B1 zxj#G=!>w22VAFoo)$PExZNJ5cfs^QKmf_?GJonOLXgabD?e!&SJG>e-#cyHy^ocmo z(12yj-$2gzc(ffZ#lpvL7oV=X>h@!FP8L?I_y}}^&~`8nZFThBT7`3dehzND=?2_z z&Be$ZlZfVq8ZmSE`axXQL(o;Xr9ZmW+-L<|%|D3r#i&77S_qbU%2B#u z9e(C4gE*=~}$F=m{zRS;2nfd1}>>ckgDgtn{t_0vG(xSy=SO z>v&_?YMg#nES9bEh#P>#ix=a2Ka9fi)joJUUKHhR!2kUCCy0pXhseHX;Nq)p#wPDJ zY%D2*-8BSv|MhArCJmsi#7vi43{7Ks4G5rK0aPogf*2rkwNAbPWzKcGQeQ@8L$FSgq*AWqU5+eGXii@wBjePHZEPn5Eoc+VI z@#)$XSX-2bpZ(7-aq~5QMnuF(h=@1^vwnX&^1XYp`2CeQ`-eZn@^u@LUsi^@XWxmw zlIQz4>GZR(?6V~}>ukD3=!=Mm({N5yB0k$#0=sJ{?!NnP`0arTeDhlq&2a`gZIVv?~Y--q1XBAor>pDMQtk*A~2X=h={ z>O$oCwj$T&thdxL_@Y zJK_-8_cTQG{UQ3CehyZw+Jv&wjdEK8gzWdMy3q zJ@ik`z{-MJlz9q``%Em)-;aDxZI_^{6?`S|N^o`ID{mQq{}wEI?QH>L`AmG`>W|eK za>7j#jypeM7U!ZBn*v=4GE%XM9`$Qf9B#+oAA12wDZ?=^d5ln0eALv| zVC)E&DDuiml;tNPYRttb{jv>LU3))fO#Uqn?{7qU)KZ3zYg9K2bvVH!rSy<`WkGXT6glmvX6x={1UMaN-bmh_J zi;*zo5^UXf6c0SU5c`g{%W+Lw^ZowDJA`H)+kEG}kCA9kMcTzzVOw20#-?2Zr}H9g z+tY%DZ!Zxy5Og)6@(o7XtgEo?a68hn{(u3AIe75Ne@eYPT3&aw9hp;a!uEY_;(~%> zR9$ljBgW3a#*!*LwBQv~f7ytNT@6SXc?rf(zZrXuC_d8N!<@MbkT`fIs`j>tRZ}~F zs@g-a51)jALnjK)Ni*B3KMIV?x>ziU6KoT)zGMd;dG=N8IM^rvi*p14d-@ojh^W-5 zIB)nYeCFPZ8?N~i9ErmaJ$MFIlzu66a(tVb+c75V5=7Z&V*HFhp=?VHR;*klLpM69sgg`|LtgA8ZvF&5$b|FXmW}$@mT8Qzj#E$W&DASM+jR-h6Krl3W?s;I6{MPrrg42b)m2 z`!J^d?pE2P$jG<~wv-9jP*jaM3tmOl!8X54KR9<01`VEx>iwp?Zrynl7hmyb9Qd*c z=~>ei;5_tVY%|KyElPbWkdb*266ptc*bG!0Xv2%Ie=4%Xe!Thqr!sHjQfM7K2^XYt zUeChIOA7G*yC1@pm}K~wHNZLn<2bfl2dtng$CRKeHzzK|fFYM++x}ME|M)@~Q?5On z&sW_vTiiQvZoK>cVkFuJBmI)Au=Q{|#-?8jyJH5n?rX+#Z+-0V``hn)j3mcUjKBD5 zV?JMn0ZCc7|H;>7O(Sn*K2N(@<}+o3^Rc?74#P)J$0ql7Jo5CbsQK~;w(M$@`J6uE z7NLt{#=H;Ddm2eYF7or2XF3B%i#HWZ1-%BKE?_7M>(e=Vc0ZOec2}KHu?L; zJ8O~b$V6Uo1?E1z5LI8c$TN=9L!3u+Ut!P2eNX;VTw5@mJQ<&HIR++YVW4B2Y#_Y- z$)|Yz{iO!zs@4|)mkl56_bqtaFAp*gv#tbrI$+2Yp@+0kRw(ydcGSZ$^mj;dT!I}t z_aQxp!2DQoCqcO;A8A$2_n}y~({k394*J7ZR?L+1i{#Wwp4)@J&zq|@a2(Un-}x)V zx@I81$b*OG%|pc99~EHzfi{%4w!_`hj+fmBaryIWFy)CaF#hjrG5oGo|DV0@0I#Y_ z*9K;E6vsPW9c64&Y)A{~IW;{bq<30KK$=QTAYcc}NC)YN^scBND!sQPq@7;UfRF+S z5K+GDs0G9Ee=RWs_=h^IYc3pd|Z|#%yuJ?OisNXvd#nhvaPd*IU#P1O| z`xxF_pM~fJyD;vXOuTcv2-7nNbS<|rUG3oNy};M|fv)u9PV`{Gbd^9?6IyRr(Nb+d z_7BVOY}YQxwVp)ftrFa9$j7j#zDVEwF_zDL4XeNW9F;dKf#yQileh7!Y97?co& z>Kip^CgUs8$g9?4-SX*JwSFlo>aU}!u?7wGCHQb`5dP9xm~%bY^--8EoJaMI%V??A zB75ITJlILFTsuGT2>#*Vh)Y*YXuM%+W4kt!NeP*1{gQ!k$FaW*Na(1qj{53|fsPpX z9~=XMjjF@0Egyjxsk2lnpg-K;;?j_GRuW06>2WMCy2 zr$ua#m8aKX%bL{!NPPH7{E%kB*}MxVFr{P5`sH}0w=ec;t}(-uo!p5bYcS(n_9+C& zoiJnJQXIFGz}+*LB?@%r3osf@DAJ$A#0 z4HuzM2I8RyAH$(j+3ZTcY3*k8c%>&!YxT%3IFCU?-@x2ii?MC(Y6eO_T(AN;c^7bC z&v$s@u}86L$9|m7yT+;eltO=E_bxpD*yGrLT935+YdC8zVV0;qD+5nI`Dc7GZw0e# z3k(@Z8WM+RdiyXatxL0_`*Yo~Cykh{<;-*y>HcI?W`;#?fk8vYOQl>F%*Zthz?9?# zin0&DU)~$;-Jxd0okU@$hHzA*$jR!x zfpmib?n)obp0f}plNOoTnRxu^o=6l1jM zO<<*2)QQnR@sovD{`L~~XXue$Py!7xZOs-CQ&?{*M$({R==oB2WaMPv`-A)O(0@IJ zS@V{2o3Gxq9i6)T2_G+90fSYGtemrW_U{y@W*E%|Wann#*}u8s(D4Fjj3xI4T?Lz3 z7?@H2Tm1QY41awRvs(#p6%1DbT*Wa0l(z%(66i`yS7DSzh9ZKPL9DVDRYQmz1Kvmx@_Qk~SNGwFG7zHC@M|1smb*?!}<%{AG(+1CUrl zfAb!LAyb#4tfY+3CsqtIhj{QDkC%NDaoAjo*`Ixhfia^{cjX!)B8gd+fZ&ApQB_rk z@Q5h%@fis7)fT*w@*dpeYB;F^kf}e1&lY?RrOX?ddLycvfZ*^XPQM>9Wfm;uE%5Xn z40qRH?E3LAW-a*wy;a2EiD5$}VsUQUdm8zdZotPU7H)1K%%&oc(=X{=6xFsOB5W{J zuAU5%&>*|FUm^;tS}^akbquV$tQv^T`!%?9@e=xZ`@qpJl0nzV5CUEM!_i%^L5ac1 zY1yg>^ilSQi-%zL5%?+?O)(Il>qrIz7cKY$NDVsDz7uB1R_9WtoDk91S~6hYU=`wd=JAi3&l`u-9?E zz7-)Mli@8VFs&LHdIK|A!v?3~dUXRPq)bNt#Iam2{XUvB_ej7by32^d$O@IoK1RxB7v(Ia|Sbi1Zkx>k~vQgN-3A+4z&_)7c37iv5SGs?u z&V<=ZSMbEhrz^fiV`CGdqQWqG+Iu`n;^I3BF6u;Fth|Y_li%XW5I3bi4BBigS~L%H zSFOb{V>!C3hG1mMd|bRxid{Q*^YymvH(=MdM<93e#v4;U=66Qbtyc7pOk{Q_v1Ef1 zC*VRIF|%KT+&vf}NfU9tt`%V+Z$jk}iA=p2)h*3@4zUGg(%|QR~afpo@kII^6 z1V_iiP3FTB3_avSc@p8$m0Ill?t5nYF)L3V4+nJ&(#;fKRE7jN$bt}*IG#b?h|o8o zbPESDud7=K*c%21Sr~?;&PGKO#rsI8Jc6LrX&I#Mp&W$ZL2sezYAqroqTuK=fcyK` z-yMUSJP4#CI(q7SG~8@OD7C-e5aeHPL2kAca(7RJCX7RAeJdtTcn@x_0SFnKf|8r9 zpxB(bWF0&_)O;M7n@Q>NqOB#+RRDG(U0;CrBpK0DHX2Tzuc55E9)%TG;2JO-qb4ul zIL*n`qo+qaLI=IYOu*2naP&_a!}T2!k_<1|KvZ0-!Wo?wy%bRh9hi*kH4T_BIR*U_ z-{j+hf|C&+n~I9+Tj(Dh16R2hT;-%6O)TgE$T3oYh!0WKcI zppV9v2efeSNBubjucuC9&F`?#F_61OK%+BY?y?oETkYtRfJ;|Je|Ay?vTVs~Z_nlS zp#H3s1tZH~71!tfEVQFZrh-&q+|Y^+z9)uuzdvBQdRmv2EWOn9CZW z^h<=3djveZLUHk0BLkB)w^|V$9t)Yf5BkUw;q3JqGc&^?$>?oJTmO!XisVcSW+by| zp7frjHnPd2*!nq@jpYVden7P0&+o(1b#Gr2;QF$11p4?4N7(B7Op_F^BQ|~*9w`4Io8Jk;C)Y^*N>n)h`LndBdeHepR97EJ+#}Pd9 z2>hoXhWf3;0&tyl5VG;Rk+d`egBI>%6Rk<#=iz&*02fFsKfv$us zJIefVv%Z*_pRIL9G*{+g)u-d}mp^vFf@L3}qP7T4^%k^TO+#z_aeTdDDpqb;ijwM5 zyf-->Pd(5XUAuIGm%In`X{Yhu4?n`J(m!>35M6%X2@n3EGbT~W2iIrKS%VrM8JMWGnNq97sn~)UbjG~JtQP*U_t%`KK9_@_W#r_Rs^JSq|XQ-*?6z zAL@()$G*gkTUNBzonj-cfkPr$hjq>JcR@O^PjnWVu$_M|*s;Wr-LU!-G*_O%Fu#}i zx}AUbAcMUt*DXNizGZl-b7v6CmkhrC?kGp$mFuX#c@a$wMrOK_#6Sn=`U_ZC|ARr3 z_8>^d=Z-e*h=Gn6_{A|$r$cLl9=o=F!V7(FrENd)-V-VR?Euj_4U&oBT1Q}Q?#}{T zSzo+PBP0bz`C{kh5AogBkNG-BfB4J}yxs+T{VC9uoY0AdsWTS9WH7;EG{LMjWBT-& z==8fkK@}JZizy51)_s9zdwJndZUr*UWejQ(3)W;bq97+7{nT=-`Fa2dlV+T%Xt$6xHR~$X5#g;W+plA1&a5Sd?*?FZX(xqX<;7AM^_cl)CRWq}c46)2- zGx66ycSGj~AIGX4`;cY12%Rwx+t;i@pYAW?gr<;9#)=H5uzlSUJl)$DhjOc+O|zoM z^RHrGx&f#1%UP>dV=jc&Vnw!Ai)WsC8eKa50i7Rw3TwXFZ__*#psU#ix&|m5@czOL zIGNwT%wht!3v$mQ(8rU@aysv-4g5aOZAV(R0=y;tSTgFeKx@rJrY-}|JlzLdH|>Gd zqCvJM1Al(r32V0M{rV`g(sitjxL?MVbiuB!RBLm1=j(j97_?51`EsvBR-fuotdr=cKRb8{=<{lmyv~&W((Za{+K^^KCH~N z{v3bp;e}&4mB=!du-t+*I|E)aSExK1v;Y7g07*naRD8U6DNY&+AeRMT?x#yZQV%(K z#WuiIq~se71vvKoLC75CIIguKt*{h1dBid%ec1Cb7z&|J&%_JQJd2GxzD7-aH8Z{HwjtT6M_PS_yb1z?DE)$$+d)dV;lyiQ!rVjg9Flfv&9iW^*v6K~Q(kA?WTo2tB0mNgNNI}$(ZhEz4jZmuO!9mTh<^ihS1J(^7KN7; zL(p%?yQscaji`t)B&E*6l~%Md6Nzy~4Ab4;9)ikUjU0mpbC)h*aI~*S40`(|F~~M{ z;!Jocf^n&=0%4IcoUX`T7vy=Nh(zCj0m!@3f(a8ofy_M=j--D`4&Q!58Em=L2uvRT z9$Y>8aaw-Tw9jCwXn~J+1l&CO;{|mh5~nRj#l=ztd8^^*7hz+%4r9=jQmZLFw1;c} zoO}|QsY~Wq4NbtfNgoLPJ!byUfOHUE@*ahwrfPh~psOo`cvR1?`V7P|Lpg&v#O!ir zqbf;DkqnVY-;^|G$)S57dU%Y2vw9dxu2f>&)XC_nB*tPADlV5IC3zxx$_QK-fy@Npl6%1h-KH+dp@kRG9O2(ohx7?&~$K}m1oTwN=ILng!1V=yjX zsbiMi*p&B$E4zC%+?0{X(djXN>2lT&?4ufhbfW;iKtaC+bC)mW6k;b2N;Qv0FLg9# zFaL_$k&K3_nt|w$k??d5hF+`3oMoTmB}F2VCND)12F#z;LvsH81bC+nbOdEcnG5^Vv3PbRd-_Pc@AXeYnE1vH>SM> z%jE{R`n`eCla_Elr~inlMvOK3xAXg52uXY$1$C|PiFp&-cb~yqZ%v0zXMtz%5S-Co z#H8`>z{A}alwRCJ5sA6WzvgRX=bGT6^g&?KI0jv#LI@NK5)A43R)qB*15dY5Wa$l< zvur7+y7!Sqpu5i?95z;A?t&fgR7B$Pm0C131LG!7hl4Bw_%NKS=XmVnDw>T#nL z7&qkuILboN*CQ6rib!M|47UCp&HC6GW-HgHi$@TsKlh??s-w6+)BBD3^Ys?$&!ge( z77m?OkJ;3pRn(uCpsc){+fhg%#Gt$1ARM<)f8GQykABED=rMb#cwabs(7TN^Y;7!H zbqx9?PDaj!I(VyNxIa6|$oP$bP?CFyf`fk?tXEt3cO`m2GOpFMVr=r;Ji+3j4#nk5 zSCKsFef0Ji3P)ukPv%ZYnacgSuQCx?xdtRpPO+K!5rdT$!L+5mCgcUEJk;-KpU`?` zrYrZ!pX$H&-Y>}+(C@*^o@0?TWjRX9N)h0z=JyN515Nf&s(yyig4s(Jv&;tdlUIEP z@g(T%`CmdM4@1S(8g5VVe(ukcb$vaET}ts4hHv*A#g6X}GU&?MwFHdH1?cMNHH?|e zH(G$P$0AnT@x;%0$YRvlzcF9r1Gx!uzd#P)#`q*@V4NO!y9OtkWRi-3=JF z_ApY(MC(j3OA8Rw^-gg0Uf=Hzy2gP1Fe2?72RB~^U0Z6h(Q@r1N)0>l!qX4IWIT&r z4u3~+i3Ydo3ed^}x+B=WatxNPosTOwDp6B@8rznP!(&}KGZ5LmXLsz{dw?}ogF-wp zZBi_DZ2lNEH_oA^u>d!3W}xBv3B&~UL1TppE!T~hI3gGu*3LnFbqT6V3NSFt8%4!w zXlTku{Z&2U`Z=QNd=?yEc?>5{Y`~>Ur{LJ@Pe?od6_m0b7&Rsl%T_PP^@d_JH>RVx z_B5gbmAHJq5UsZ^qUnYOH)@GNq(w`0EM*Wi=n zXtdVl;p)|M@Ck`Pab+bM={%MdAZdXPd4Zn-6gu{IfL9$c&=CXw`(uE>Rg&|$+j4*4 zRe-J}Q$xpzWzy&XG}jOiYu|Ri&-7b(khjykb8TAMwA{_yPq>X7Tgvpo>!`U=E_tfL~WWhq5(pd1s zQ{6Fh_7})9m$6g3&T59ysK=(&oAA&d{)ByJ(s9OYg@(AUEeM zp8E6OFmv`QWSUBmZ7qh@Y=Xg{!=^Q)3ELGX(+*Vw4u@)9~rLlkmov zH}LjHpTQ+)7}70QaNx%uA#-uUvD_kNub$JN#mK=C7&>k;PUT%ernQ1KWeIpCAljhM zg+i@j@O8tNKVa+HFVOpy7jaBefK2NJ6d6xr>)IuFx{n_YX{w=1%R|rSdSLGv9nKUG z=qik=gwa<%^hO~uK+*y^78~$qE@WmSnS5QyIe-8g)Adw-9kV8lIuq8d-;Ni1y5NMy zfXqBH{3?acaDk1U3`P?(Uo*{@pf{WYeb3F$LYh7kPe1F3Et~eiV9LSSoHRWBtP?hF zIRKqWgUp=HAkm`(lZ3xH}Apm zorm!EC;o!N=_heE-wKVj3_5KQ0^~BD&|kA-7fu?_!BrW6nX?zNZfmYC7fNET&K7Y& z##@Je*o{XYei*~XO=V#D;CDyx?4O^*zVtMlFqz<{^2OXa^B8p9u%19yHI8OqX5%xx z!3s-GHUhlWShM3BoHXP?rtF8=GnX=RHOFFSh7wR|&|6_m&qnv>U&Nj?Jx=9cvbPH{ zU8zkna`E!tUc%usS;(@UV{p@8G9k~L!NyvDc}b4Fr}JR2>XDs$7SI1(j>E_DVP@m4 zO#Jn4G8{g74jO~RbQLCK66i`mssyIq3A~cL00FK9yb{}0+OM-p@gV?L0m2GRS7H^D zE*t|=47v*QDAs>dL@^NATNQ)8enS}`oV#!{l&WBqa~kn-)_kPFIHf$kqKd_9(>}yC z)^r^WrAHWadIRPyS;U|#F+01fhM|woP+Y8Tz?{X4*#yIh88bpNb>y@UakT~Ti=2eW zxXGxyT7{^{XmnR6ATZ$_)LyGXOn5McPn`*4MGL%rLm`)Wp_igR19z_O(U5xtBUh(m z1`MTl6X^JY=WF<2)qZUK;RKc}TLpb)Ce-dqP8X%YC8hdPCZHn?&MR9G82%>fKN6db zfB*-jFmLLs9*lHtIht<)6O*T6__WzDUT%Refv)aBc*!dXiPILLqNGfKuD;UHhSzItJZ+<8g$*{)w}hshC4d#btDTN^`fF z-B6~doa8fN(K8@{<0m3C2`X38L?r!RVvI$jmq$4I`b5E8(E{JdWJJcNpwOBRKVLuA z9@~eml7pz%!;U*{UU%4&6vMz3Hm5QSw18%aVopFA3k*y#iiwN_ZflLlb4{hq!j)9 zNyE7x8)%8x?~m@j18~@Q8B^Y#0j7dojSh2`lE!l|_d^0*Pgtrj zZ~kTmX_w^9>TsMj7%*?yQr0Xb z0I=V%NrGA1|8>5fMyJP|#fv!pdPtzF=^BHszHYuS>NJ?WgyL-&f`&{NOxLhbjG*_< zQN{o&^2`;gRaEX4-cLKU$=p%C@;f=$rBkIbW{yQmd1#& zQ>LI_(i37c=#eXaSSff`-DJt=g>CL^%eDS44bkXrDf72N2ze$a3Ubl26hrq?c^WB?dBPo zg1O6fLz82G+@0ic2yBeRcY6-to9_>y7cnnMBWRV8Wj4uqpTj(;y*)xmB3aP*|P7&mb`DoTn3XfEr|dd;s*eV-?DWS*lzplf;Q z6@*7dqA$H4D6_;inLra*w6NR&Pb`ornJ~#Crhae5P&iPggu#sYr0J>(6$Y4IiAb8d z5M?Em@bwBnr*&!h7`x#RK0SI4-<34u`{HKoKG%$0Ma}p+zX=COEYdm_ZY>Q$5`g-`Iq2>b6YQ^=U-SF!BC^F9M zMqYsiN002pryotg)sif<-ZZ17`V>~q8ioyDE=Tc|JZxM!1#{nyLQ_=^yp^xu_YZc4 z%GVc{tE=GT+8x&~rDMbpcWm7JDK3_$0WjcJeFj=;&B!>k4KH;29sE=;LFw@iJn}$i z{Q2R|(4IYns@ekFLK*h`uo_W;E?6=9E%>mUlU8)hIhsM&YYlmb4SboIts7U* zM?*se>It~5r{_G4txLya<=T0;dh;skZ;%G88O@h7k>K+Z9whTEGTC~d6IQL6i_?cT zwwZ4cv$jX?S5SWWB5u_aSWN?AX>ctKiaUUn|NX#9M;mvvaYqbv#K7HRp#5)x^xGh< zZNCx3e3h1VU@C#FoHb#a%%k6yiK7C9MtSZ1u4}t~ogPPjT#8Af`(fvnPiz@3#IhC3 zuC4#%eRK?Ot2dxkS{e*!Y$%4a;{KqkFm@`!kNb~snjNLW6R1TRuP;2^4QqFNkIcL> ztXs1U4|aY88@BzxY*M|^fW3S7;SY~IgHPtILRMZrG7LJ_RweCAjWHiOYY`g}=`DGn zbpHoBKa36AzGp2{GR0~;f6g-KELwbj;0HYN_>0)IZ7-~bTzoX+eRO%`CG0zvr%=%3m7?iJdUKLA=7AtOy!SHXD?-@r_p3ZfiV-u_I?AWJ`Qm9@W=8U z$B~v-39Uv0PlX4DzmW=!{oWX;Ceg*5l+>amjAH%X`OJOx; zajLbGIs_liT?Jis2A+NDagaPgZvJ^#a!58Wv025`O)RvHn4g6?he1r&4;QRPdR`UurgK=gZVNg;)D3ID+KViU1!)>B z{`;wyF?;q3&LRO$9Esb z^N&A`gJ;t4&EX?huyQqo%i%cao1VAl;Y;nL3cQgjolb^-pc`0v4gsYf3wfm!EmgCL?hLXLb45yZA?8`XMgCUkB^Bc+x(1{agz{AZCuTS}aLA9|d)7fl>%sGOGPQ=y57R+C= z3Qn$(aFfL#+mO!&IqXu+F3#jO9)l67AEBxh7@P6|pHHlpkc5e-YHY@w#jDU;HV_d5 z-$HfuElfy$i*-0ZS-ulZH*X<2A`*Rl<2a3X-e+srO`eQ>Htsr&z_9Uf@rXu_jzGO- z%ornv(2MHV;Ft6sbQiC|kY9k*2`NyxdBQ~%iO_^pW;YT;=gWO(a7uR(y*%O&F>ng5 z*S2DO$~y=jJPvQo-iV&6!E8pQ*XZ%tXN%F-e*p5TS}=d%I=IS$(A_(M8J86oFEZ%r z>>JMY78d*(A{r6Z4leU5^;A%*Pw@I95(OG#aT^0Ts*unGWBCr+-gO?u&IzMhM@fN6^u`r ziazooaP~|>34NcOD!!|Sq4e?~`&}8i-7d1#eE7 zh9J_4tZhYT*jNmVO~(5luMmblUJ*ELD8|ACYoYY;MYhg}S&LVKoX(w;(Za2M$#Mpp zd#mCwbm~F@iui`ZUG9yUt2Q$D9vM0sYPT@t>a3W#&c{nrG=5qTB4Mxf6hSQPN1JEO2I5QH7fv9%#gR?9W;X_iUOhn+|cUf*BJR+8vkHHBa;CfXZ zqQk>PZ2wBIRRU>$J?Ki>lLWfPa{B(frOP>mm_Xx~JQMKo=R27xUQ$*Dxz8}Tc_uMa zc|OIpERdOR+jbpCK+G7t~d5Hr$H{83UDDIC6AWoY5JD=A$wk zpM3r`t~9kGID9w;#J`RYXRc&pwBBA3IAtuytVLU(@(jn-D^+ZQ*GCo%C$B+Rx^5p_ zRbdR$kQ@a~DD_Z9BXD?%0A0hzLFE>KT%8#iI=x7DRz~8J&%edhCLlC;IMg2fdH>8M zYtdaPG<{iKAUqn*ev-VvQMj@(R2)X7&O>p@6?hV3m6)!iX+80MeoyxF9Eih)DvY1_ zPYj41jT31mHt?eBe!BcST)bEbU*AB43>k-#x>me7aV`|9C|tT)k53nW!4m{q_i03b zmV0C5v`=uksTG0Y(`@~@($;?y;G`OkvWhB9NO?=7$*V`8w4xegQ&SL_IEI<7p&?V? z`lQI31FZdnW%QGBjj3rnwe>Ifu zKFHCVF>CQE1}q&F5y;e=FmLG!HZSX?ipS7Y(VxBjBKiG1^K;Rk!@0d9pwZ{!(J^X6RkfJC-O#Tcm4ggwSlgRc)X+^0VhvT%l0M?3nD7;5Op-e(q=`{wi`^tlD@;UZrCxBXd*GX&6 zIFOTEi!jnUYbe0nrS$tDCS`_BTZl@Y)FXzyA37zi*oVO@4r1chI;_m8!+OmPtj(#z zs;qh}K3#(eo2>ACTLN8wM97SznDUhc;h%2Dn_r&6+ea+;@N5}A%({f%Mqa=Uy3$Ng zUwJLFG@%re z-tfm04|m2N9_Wl8cYgt=7ypc|okfa$>U$H=)NDa(vlf$+BjD=vIDYu%6J{=u_URM< z)dlCzpTyOxnP_S-qp?bd8)cd3^TMO7)w+D$Tzs+WJ-nA1kM~m(aN^)5Y~Jt*)_?I1 z)-0ZgM}OBD@q_$uqooqoyflm%5rmsnCsBX-C>;Lw5SQbDE}hWjcb)J6$sGKy6Q}r9m$ZU};T*>0szl}fHkwhfEgdWZ!=hW7@azGW!=(T%S8qT z$;fKoo;~Mg(*t;*(*yYZ@BWC@JN6?jj|`&{wr7NEACC5ok zKI@t8+q(yk{_$~i?(`sjZ#(~sul6B5uMFAdeCW(3%$PBQQ@Saw+b=j0k37{22Tqc) zmI?d!>_*pz$zICLf#nfX+A=i7kR_Q2ci zfSLeesm$h_)9_Qe2*ao@kD*iN$MBCH9ypn0MXvQcG*%PRb2WJKFWoVF<}z3f8YB!I z2w7kpGAviHZp{vKedtjfJber&HM!{aTrX_ix`z$G&gN$0>1W!1&%v|LJ7DX^130u> z7-=0&KZTQ8J-R*TfGu10GYgdTSJmGAv2E*a9Q^(;{_?~#IFy!&-24IzO&n}%udeXy z7t6Y?8p}BmOPxC5Z#^AxIMWC>RRCtrT?&n%kmUi)xtZ|tpyzxLojUy&0{acX6HmT^ z{bzDHmZndCpRfCmUamNun`bl9dfH~X^)R}0{Rk1VYoYNdjW$H-_c zY3vl_TFRg|=A%%bj?L>TF+O~b$OS-NS z!<7J6X&Ze`8a1B5SUN|<$vI~5lKw17RnRl#SOn9R2CzamkkYSX(OoqNAFcR~O_3zC zA~v>h?+*t`-BiaRByq;@PR9TMAOJ~3K~xH^+-&7^a!Ma1oxJc11fYYS=C%(n^Cxa$h<}O;xY#ege&eRuS&Qcm6hNHJ~0JBCZt@q1)$8qk`HK={U zIgQcLdk6}uTDXk#oEOTjL+KaII(A_R(@DU3jyJs8{ zr!8zVSgL3RR)2FC6&KDS#9Iv)?+{ojS`ZpG5^kmuoO?%6rU|awZd@%h&Pd)VC0nD2#UkL&ig? z7>M$UI*d(umqAxYMI6#Kg%~sWUG#4=-Wd&7w{U#9C%Y|X&T8@)bfi@2Pe0$zDaM7R zH{jDR9$AJ0%v$^f`YOXY-g^2B!3lE}q*yzxHKM%w7Q#Y@!<`I$mTu?ko-1#J%4WPH z8OgBmPGc*A!iU099>WGZ)c&&;tr7r%l7McK4G`|wJ=KHog69ZqI$*(~g)3N&!r3#9 zncNg-WHNKH{5sTr5pZN+Cmzn;1KD7T;*^eAuxJJAo)UPOG<7jE=VTsgc1=&3wiKFd z1H3&HEc;*wP1`e4f9YpCNO5<7MzF^OyH9}ly+}$yK8UkhGmYmldx=yZ@eFqLR>vdV zScIZW*V#y`w|g8rf?u}vNs74z)Td}NK{W_LL#LwlMl0rj_Brbj2M?Kss@hwam^_X3 zBgs|#ljYx_hU7AA^+ZfqXN5PqtE2Jp=ihPrl4QYYjR}`)nh_Q{8ZwVqq#H^(dw|l6 zseZ|{i8NsyWg&1>L}J^XthT;K6AF&xny!pO*pP{+Y!vSh(&h{a9}Ra|1hVu645Ias zhcI(8Xjrn)Dh(YE4>A@rp2wk#Y`z!8Pkj>*5<&XUVGJP8S-FFsBmYth5=Op#n~@e7 z=aGS*Bxyq0!=d3r;VKJ-qi+IFvW%ZJgEcdIs|Vnywi06}d<0kbAk12^gWr=BN2EC& z9+3!V_aFohPPT!r^Po^hqx@I8FUQ_PvQPtdbt{Jro4k* zvVm~&OhR!*1Kv#e0Da_x;p{bxzrQi%Z3L0wU2Q7@LnlEYAB6IXTD+O^7Hb*zRmS72 z_8eYMc@M!uh4yVw=oq-TM&ipoxo!P9BiG9DOr}h;R(!?Vi%M?5KWG554F#CP{h7?d z;?UE3C{CC}f2RH<`g0R&F1xvt?7$9w=aNyG%6|Y{ykpypn5aF-q$xOp^q!;Otc*pb zp$M~xIq_Psf6zjgF}7IF||U=@MhvT%I$<0)+a{y6%QJcJ?!{gbA0#=)D(pTNZ<7+>wp zX^RIspE5K^&vw+*nf!j|z6)T(&hJoosRl~#;ZP`tqPV06!I1-ncDgj-XamjdAa<-^ z56gwjh5)q(qK|SQP8-YH?n`Y}SXu{_e=J<&A#jw1qf_Xt@6i9#oe2GO7otD?4)HU0 zBkt4P2%oVNq0_%Zz}vf_p8Nxp$v;9laW`UToy40PG7&xh8%)@liT6(ywSlhpk`lPv z_It|<#G@zCcu7E9LGEq-xYc+GB?X6p)*@72PG{q%rUnyQ>+^voVtHzTrb5(~S<%{f z1r3dbxOP38*|AqjPoS>e1hf?6)+G&Ymg`YpSBh3tp`|qs#TWLWk+k6&3}~v(0$Pf2 z{gM?;H!tDJwH(ycWZ@RMB-iAkr7Dxnxf+|xaIPc=O^p`RUQK8ADVootsxlY#*D2*O z7ico$=B;xmt(8UUp^S~RTB`HVQf)+Iy)ejXt~a8o z&Vc3{bS)aJo<(bY4w`GMXsJ7gmO83K8^eg!brNf+19bi052|zkXB}XaFl*a+JUYNn$ofDe`{>V@XkqZ{tmP82^2(8GvcO<6z+|w%VmqH?3Fr^a zGkN96$}0nz1Q|_Qn9MphV9K^!K#rxHQ_2lyE6gS%OnM9S`a(`gKWi#sP)cViV1}#4 zbb&QGHKqavc}ZqqK*Dege%}0*mB61DI z3__Ecm%*F|jing;9M&=zj22i-2Ix#yQ2KekAq(p_Y(Td@f!L?L&YGSkgAEw7USNPs zVz;&j-tPQNpql`{b+8(BFq#O2zQEjd-3$iSE(2`-Ao}?b=Gnsi6Lu)t>9e>Y*#&8ij-9?yn#M;b9E@{tN3!pQV zLaQeoTVjIJ{m5|YJm}d>lqyd;qZZQT)pA@=8MCYv$T44p-jolMUdwv0SyXq{GS+e> z{a~sKGQzSN_57R$BbA9_tPEQ7C1hJJLSrpt=?cBU#1atnY-W=M76a8w9s`i1Eu3kg z>lQ&{%(DU8=ehkQ&{YCHbw(5NjOp0FcNcnk1mRfDMVP4!dg*%San^E~mmE_ObVf6b zIxXwY61aKRT*|R*G@4<}O^2VC5?_3M0O@&`pwG>Mzqcp8*l`G_@~-2onSl28pevbq z*@0I{W?=2+uNhoz54h5ClEy1BUfW9zNY6q{^!tIXuP7-t zmGPidM@~CbMzHbGKUZ#G&A!OcK~TE)1L^J&IMh=``isKMgdCDxWC5Jg+fxybo~l8t zZP-WEA2V01M|Cq06*>SaH-EUelhz`cp~Q)_e|ZSI#kZrcEE+u%~9tF|A&i+3!Q`&BX;Q4L`ISW5MFl6k{f?$7#)lBuY(;U3RGI59(s#n{_}%){dNIC7dNty2OF$n>bMG5|eP zf+0u_@s4g$AchyEhO+63f()mKK^lyn>TvW@MX)=z6Pd0MxT%bQE19Cm`~;|~ip49c z7zVEU$|wc`;j9c|$%)?ba8PQ(SPM$w<4ZtF zBzjOi%0uDgLGe!43$&&0#xld1)4COuVjKt;nLixlL3q`BFl%;Bob)c-Jbcm9XHeUB zg_{@pdIU4Db$6ij-ayN*KiiMVj@B zRdGC-LMiwZv(9oqxVVS0+y=`^D8%!Ursu2b2xh!G$OgkjF$fuk^O(70CHgA+u@0rH zhd+a)Wc1dHP5tP-6N^5wFb3Kv%f(q9%##x&SJKNf5IF8ufr0XdsLLIebe=aAbXVYg#Re$tUQ9q4CU)ex7x95M}-{0s2*2>`uU$mERf@95DFt{wqG-PxSZ`IUO*a<^u_C`Z=>*1 zEtFmfaCMJ^lRN2V2l6DScnMe?YTtZD8l53;u> z{f!`xMJMmnHSn3b4!+aYBVg)A^qaZ`{!_QYFLf)tQ?|lu(l)3keg*Y}FVS!6P9$$O zB5L-x82ZHlkcrj@r_bTTEYfl0suhvo)i z<`9#{$m~wiiEFMkp{2o!mYW1t>d;i5i|+@Sk3Sa?5Ov#Ey8(W9wGk7gbi z3U&@NWa~|6y+QH;X0$d^ne}L)^Q#SLt~H{m)(9#$m%Yx!Wg~N|Cb}*Sc6pFXY*$(c z7&W4$-UtFz#X`?w;qz!89V1M!B#lWLe7@I3@5f*H|E2>dnQ-F<*eT?fiV0Oayr8v3|`O5R;HhyFB{E32>GfuNII7 zDuYWlAeGFbbOc0F%C@lxIVLhol7M5Nn)3M`bRT+d0&AHmX*$pQ^(F%IY~?W1{Vp)TC(bQ^!DvL0{xmkOTZN~32Vjr3 zmW{#)sHAI(g~}#0TNzYqCo#|-jHAy~&jd))Lf6n52}C5YRe-k)=9vq)oMco+rdb4V zisvAPr_D%N8s$a2P0Gv|&wqu0a;z8881j1YK)`#=p*)N8lfsE0(eNE?(!*NP&OcD@u zJe6OHhYKJX4tn;4AxRiDdaUjE3mBA;h!LaT;P0ffnni%SO;(`2hN}d&-r43q`#ZO# zXa}m}oc5ZoZSn#H0K_o}AO>QjIVv>YC{3Hd97&6fz!Dmiz2X^%;Zx@_bF3A>+3Y+X zOp~D#>&4o%^BCNrff50s3es#NW^FWH^&EiLQs*+0lbCSXIr;3CO@=8v;3RVyc{qbG zedG}g22dHuD2gV3r1A+ZJ$g@((F!e;ej1OLRS5_lGFha3hmi)VF!Yevy+Tti9FB5& z)`+%LQDI0$19%#I+6Jb!ayTd=(38wW5P-`IUc-?%?K4!Alp~mov^@IpGdjqk znI+1pl&oiG8-)Lhynq0W`5jBZ27$k5WtS8 z7peW}epE)fKh>cqFUOv(zD3zY>{3jSq{X}yTX9BHBw|!Spt?QI=z8LOV#kT!K6*xq zNhhwiFl^m>7Mp)a6Ck)O6kGRZVDpb>7#tvwg5(NBO#XYIiv+m*lC^kVijCX*gNS+R zJM>(nIZ1l9Hj^=qIZ0zt^f&H3XCSP7FpW1A%a&{^(sLG*Y>r^P~dRHvz!J zh>XR&Wt)+%IVavn{9IH%n#du2Wx@6o?+9D_+G36ODX2eZV9OqRf8Mq?16%34oIrHZ zp9M2hWG4uKP-13^ZT9}&4p`)R{%zg?)NjS_Z~M5H+KWJPy03_r+xwA-f!o`YfXx{6 z;x=iol}q&_Ob10fi})4s+%6uZ{>c+sQQX)2C_^w}+DEWf+y<9Xyua zWn5qFjLGcvN0UY(jwp8N-ctXd-)MTba{Vjf&?$L)4km5OM)LMtByZCqWxD~%+f7K` zZbk|%+sv4>)q>+; z_~O=$i_C0c4LJe=8nkG=p|dSoG}q^%^`_ni#M;v1X%KkRh}Ihh+jbpVZfMbZQ-{_@ zp`Xerj09BPw4n8dNdUMEbV?wk4cMi7(RsW`15N@{%>t~orCt(X+ddtV89oFI(?Xyt zgTyv~w~bY5V3QFx1-S#{Ab|O|WMbK`cY8`^d!|r_w15P3cdUQQ*z34fM?7`J!2h8b zXdqxzd)xA#Xn$d(<;#DZOMl$cn*QLv4Rj?Y9s^kfrWHVADYT_KOFEL|ntqPA5qLzJ zn&fIvMp^`f5dfBHC8ppd$QjJUkkrF$GT6>1reQhGTFQ{k zWg#$>fHnJhw}VpxV3mMYu_hf(0T_~A?C_zHEDbPY{syF5>yTq96Cg8Rm#%GJ=-KJ| zbiek??f3lGvKWkJ6dKQB!`fBo*25DAw3W;_y{+7IZQJ$fdIVnHkNGP0pR)m;WO8;p zQ}U~ zZok%<1!I>$ayCe_*z1J0Ngy?y^HbOVS^L|bS+o&>u56^GluSo#z|v0)5N~7M$f7_j zrB~F0@e27yDD6x@Vl>iV%RVr*r(L#xkC4IEE2<%QRW*p2aKx%4=8+3AD`~)PGv1H} zX}@euU{5?Y?zeFIBWc*l+!QtgCow4Q-Ht_M9{@^&YC8i}3^>JLlm~z`(3I-RCI|4M z*9as|U4oL*tLWzw$V@P15ix^@o}cR2KKQ14OM0WUok1O&L5l-1wRnl;I-ujIF6;|! zZ@<|0N%s)P3gC^>&3`THirc{Uyiz?#_mI}@<3if@bJoB1yt`fdUe_W}K?0WT=Ml(2 zz^VX$rS1Hz?b?X8b3HPMLja3d7@#E(ToH=ivPfPS6enXk8z4Y>s(pB|$Bk5fzjQ5? z@84ZZbtJ8&=abfVd;a#v-tYcgPya@LrT*6*bh+2Ps60QnOroAiVuCax2Q!e9X)MI- zC7;7VN$oEHT?T}ul;?k0+xzU@%Oo7xrE7@3YVWW1*c0#A+xxB5hwtpa^xeMi&h}~l z{Ov!}ad&>d*ZuF@e(&$%`TzC0V!PCCZ80qYqEdTupRvDF?J4E<{v`FQyRD_`Q~h#Z zyYu=q0l@D@+av-VN5**k&JuCEX(A~S4$4s0JQc?g5KaBt-k<*!aJ^ef zJk<$-*XU`uiI`S@*euwoxI;(4WD;bgVx?}uYu1yYvKLQdU(CP4nFU!hxgm-ZQs|z z_uX{}pYt`6)*Zow-8p#oWFbDxEWwPd-wt%`t%yf=SNc(if|F+y9A#?M)f9@W|7vb@|(Yu5I_Z0|?}@-m&do z$NoFpIhOui!1XsO=g)})dR8qlpzpY6`)%A$7;q)Fu8;<}^Z0-JyA&79KqjVgA@|EH zO9>A&iIFG>&px*x-JcR?=`q4Ad*r&<3Z`mG-wxW zEy2hn-9`aq+^Y=t+iou(rH&I@hSF#qWx+Nk-2KkIeIJ!e(hem$v=u+F~mQm5<6v z$J-aW?u}Z~Y%S_oI_jV!20CJ(BL+HR;6EAzw;ImjS1⁢bue8eV6~AzioB(m*J!c z;6Z8IA6B+hRgs!6(v=x_Z3i$B;HsrmdIJ5Ju`0mVHqezBtODeuV|9GI0H_4ZPfGU| z4BFe6o;Ki<^jm4Mvj!;zUNHHDZl#nS|MNjtX5m_FtX;a6h0^FnSp-042hnZ=o9_VD zejZRp*P`o-@-f5l7cymUyN?8}aariv{~6Hrwle&xdiaglocFl51YrN#_5Getij8|6 zBgG7@=@@A(eU|o9eE-z7?sbo!+%Cn1ne>RMe+X%Z9BYY9|CAVC1d!go2sWR6oAg~; zbGje1i3q$RW{((bQF=E48oe2KCAO}IP|D1h4bq~>=)TUzj>Ep?`#lHO1@jq2YNlrjo|I6_< zmfQbh?>*qGs=Slad+l{j z-SfTctgZH-IU$SKVI6j5Kn!7weJ$9)mB(nS&U5->fysl{gV(dL!StH}c^%+`6$LG#wVc+m*vO0ET1d+u`VYYYqPSkE;|?NvU0FKI~QxSazYZ1uVb04Y^3>o zl2S8$U`okg?#Eg`+s{IpkBQ~%wAP=EG%K05*q$cSuI_8Ou(>ZAYkgT>@^s+YfM)}q z4R|*2H)R9nZ>RnZjhtVtzk#W~JXvJGd@-UYfcb@+-Xap8RAY5IfUAK(jN@uSSIgTq zmvK}jmMXno31KzgCdb9ol&|PBR;bmk`}JxCQ>>dSaYt=n7Yacgul|RHaRzuQ^klo* zHsyD2%f9qyATIk30bL5U!uI1T8!Th`S1?WRs%_?3VG4A=-g$@|9(1MNE)}%8>pAD3 zhHY53<38@JquW|WVJm-dFkZ<9%JRG>t#kRgY1?>=Gkk^xU5j_%*4RH-;MKe;ZC)nk8`Zyw(BQnBpqL>+*&X4+}^C{8>8}_>^xs|6ueaP}&aotuVF}_|p&LPwnrx47K0{8^K+-9|{;M zPCw$(T1mfa70QgW#S1aE68x$8G1igD@DO375|@}vqr7B6O<$eKoVeJqVuA6;r(eJ2 z6Qz>H_?6yNwyOE=jk~0;qQa^6wM|!T!0Dg23B$xH_jB6%3_M^>(miYa?P3e2cKH=U&D>AP(TTCQ1Y-Ex zoP!+ZLQ76~Z54mQ9S)yDmxdQyqo0AsTGuWWWvgp9o?pb;7oXEE&Hp*J@9_O>#A40a zKuNJT9{jx?gP&^#yzwsCW%WcEW_BrDvHKF?PIxNeHpPaK8>dl&mgW@U_My=-xxSO! zlaBs8g}d6vZpdyc>FCY(?hB_ZGl}?ph}}Y6{&TG`qGd6E?r8!_4qAa zYZpQDv+y8uB{S#Kpu*O7V{ayKbg_Bm1Ns59kUA^b+4V**%%=MugR=7=w?TK8qsGo9 zBkn@Lv}|JW8_tanikZ`rbd;^m8o8!u`7z1A$jq7fUb>~u8{5VbXY*a{fz&+H8 zc{l4K?A_tVcZDl#t?2v_TeqLIwbktx5kRn_E7 zDL$~@Yv3{9)T}hSo|5@;oax9Zkkv9;DT}xNkYBG zv>1|>9Wa$8aJHT1=UaIqn?zjXoecb{P-x}b7GB$DbTUcNUk1{m{(>i7vt5c30-{Dh z43%2h*iH7N3~0$eb3xYPQ2_(8;9}Igr1$>}eiQhSsyr1UIbb-dmKWg64FMXo=wybN zc!vb+qc1bQGyXHQH8@^EIS-SZxrPTU@&XYFfin>XuaJ74ZX zcI6V~iQp+oI3Fk)TnUhvHx^5R6t=zCvyy!Cx9&j*_DUeD>LY~7gAkiSpj9Q4ET0S3 z840|W5Y#7XpOx1406$R$QWq-CI>e?x3J>iikOMnucawSLYDkKEvBDCDzx#lgIm0^|VRwvp#GdLT4UlC1aT zXT#Ys)a-rc0*qb*R7T}0s&c|s{5V>u;Ly!oPfnh2Xbco+9Pgj{NX6lb%a!ehWeCkW z@;z#(~lQ->UU5Qgd{l34L`XhOqI&x}!98 zH#PQ8#pFN0$FxI_pSw4@|3v#zyhHyrUgrro?vP`8z7>RNT>6?<+FtgC)w0&w`WjZd zFt}FmuL~)C!OmO~%~Fa^vkeg~@~XMox19LrmI>AP_Bs%qA<)`nOF7cNbt01Ksum`+ z!?wF>LCSh6v@3LC1NGFk6>6K;wQb#A08uu28^kOmrhCyYxbu-l zM2*dytPFK1^0|<$#i}fu%<+In#t~N%4qF1`+uM zU2?%?{|Snfb?Z_=DJXe)&+)F0Nrnd}Qc_$j`G{HEsfhdEx+>H>E9LvA1G%D%TDc}$ zt2IQ``p99UoU#Vs?Xhna9l;J~2{r0p>(*}FVFHZUd6T8069d0yr0@Cmi80QyN)?av z3pEOso`qsz4SL?*ad0%X##RhbGqFklMvrtM zalO7-Y=LrJLP-~kIs5#51aDY@yi?@vNOiS;gt?>6ZL^YEGe5)GA54tuw}(w1c*_Fh z-$bx;Ke%9ca=c;y6zEQsFLMGM&Z%ZQ)tH9%!9_~mYEC`_meW{y z)8*gtA#!ehMYf^+4olftjmNs1H`=&5m6%Ju;>~^USC#%ckBF!#%q5;c1ylw9+Oo-! zT6ibfCZ0iJxW11chFqo_=(2r39-dFPvWVC9HqThb7Dc0|y&2)M%c;B`ZR*=8KUCWh zL4U|>Fg)xTG|yD^p3*&**~xYtO^MPglUqu%ZH&7vFzvT2Xs^#M*)1ROr9_vcgR0jS^*DpHG78BWn`zea~?yTVL@|!>tuG@Fx8ci1i zCu*l_zLg6_wqHwoTbv+GK_=VP4@)^EV7Ky{BDZVX?WN9Coh9DWuRCcR&gzdIbuWcO zWui56g@MWDZwdKH}3k4(uYz)DE(uGa*PeRi^_Uvj0N zGkEfW*zZ}N6a;)#c-(L&vEbTWzS%tK^u08K#w5A#p16%niG4-26yb=-;lcY{-(7}S zNhU!vd&!ewQFb0%Z_*JewovD@Jaeh`+3uDkt-+T-ep4XlyRP?Va}MYow#?wiy7%SN z%jai|tKh4fk1nhwb3vcc^+2fOE!5bVIH_Lao-CUU_&ccGz`xSKf*X%z|J6+z0(FPc zDgfT5zoD9V`3DPO)lQ{0V6(hKi5clMO}9ocpo{|ghxbEWF>L0+75-(kW;En`;yn_l z$S;;pDqp6V%qU^GwQ8$hP6{l=L}xxu7k|}wbGV)1y6k^(W^291lKq2xgawzxzELO8 z)GzhJQNVB?zljItF8}dYXV$~*QVaH5o9c_|!@TfLLXhdsDN+kf?I3Pq5 zVX#e{^uDu7Zkm2aQoSZR)j2)RVDKku(!ZZgq$vG9zv^#-aGcB*fBp;&G|t*hHa#KF z0d_>DUfvLMUrK6s@}*yxA^F!w1sb;s@2fr-r;ZJ>8=ct9G*=i*XI|SjkMYUI8T6SB zu)x?^zKTx~F>7sCaHiH2cAqQr4%0N(XBzaJr+M8=rYm>-l*2V2V&3)~O^AoR4~qNh ziaSer4d2H$UHSR*jbQ4z9Sa+BJXJDp(6N2o3uDkh0&K3C_ql2e@B6(Z9TycNc|!I@ znqwL-g{?KIg}zd##7eR+|I6UeV9{=ro!o03r#F?_Yr>iqgevG@U{BI4(KCmF_R@N% zzl1pHON(>l1l?=VA6CQZ^D18vaR#W{o+ea7T24tQtFY@U-zIm<~ zXsI4C%AORU5s=C@PlFxwH={)iWU;>8{$S{u<`yHLe93!Ic)lf>SjoL+jB^ndd%Q0E z>-Nj|lsQ3Cg5Z(D?pVh})AnE6Y?ET_ZwAH;ORMIGFJ)evDSB>yxpe##`55wGJt_oE z5TmF>fyQfGFF*K&v-!HTDcssO&2uPt!cObua(Lacrs1vZJ^B2wbK=$VYP-t>ZZ#0r z*Ep%Xy{!RzO`RXS^zUfDT5sEV)=}S{x7^fcB^xY(WUtA;`+0SAG&kgLyNZ6c&xtVL zzii0?di*U#;B~T-z*|&G&Z<8(w}byhC`fwb_1^G1M=b@LKc4#BflOlV&)i9O->BI& z7g{MED`RQZ4^JnIS}pIFIZvdaLl?81P(hLxSq#@G^itQx{d0ZYcDXT82W-r(QgA}7 zg!2*8H)x=U*iGhyf@6-l01W%CDzsqdD?2rIT?rBV2O2<2C~{%yl3i}^$ihvjDM&)Y z3ow$BDwtF59O<+=o&4zGu=eIO=aDC;Y|ygJQ`Lu}G4szi^IGtiD^X3><-EOn7}I#Ii&|5ne_H8w=pPcqjRaU5&UKLkq|75`ESD@ zn)J*3?*9&5*Z6%Cm1Fi_3U@S0Gi8rEE4|jXJ!h&*qF0);I;Za&=Oz-99<>{a1zMB1 zblE{=Y$>npes_)e4KWA9TA`4u(?6F@jmK79GycVhipUxY$ZxR>=N4XgVz3krk#*7T zfR_n#&ZD>ME2Fj~CY~+4Er;(uhoAb>pA)%6`H{%OmN2_&h*fH6fn#B65{k@>9A|&VMn>{`r4AD(vh9SnO;$NppFHL+0F{r}*<$BOnKhe^=BwuA*U2EftWiqjpnGFyesjFH+aj|2 zDz$1}!1-C+oN>OKJjS|1GQ{iFgYAGKJ>(yGB@i zGdZWP9ivEEBPnb*$!K|bjD)9^ptp#~_>4IHY$rJCZYG7|iYzI-IIJLl1`0eMJY@8^ zn<(*UdzC03T{a1Gne_d{-K%!@rrKnUv;4bu*jMQon2^6Jak&dNpCi#<0erDiFoFrg z*gy}DR!TbyGWac(J)$d4OKH2uePV!Y7L~BD7q8L`YR^{?*KgLQQ2-8P#qdwJ?J3M^{ZtZp!*FtKWU<}Ri>xV$jKj}; zeEst{x=(vjjR(k^?CkTp)w;bJ^4kMsW%dTS<4IR)k~Fb){k5Ux(5aGihgJokDoD91V{^V9SsH)7hWPGzby*c zQnr~t(@a|E&n#R5+NO&-v4;NXj3Q}^D~_B^@}TSMVG3i^L7fR86G(-vin)fno?8jo zU$r+#%U2z+Be+Ye*gDBz9h$#?8er_OD~j(&2MsQ2M)wajr|>z4P~vqVfdRRIROl@Q z{<=oXBi4m`hP2zgX8~bWq8o7`b??#>Q)GJEPnSAFNv0p}6cSSU{7D~oHZDoa0B=IY z)RK$K+HpyUIckX+BJR3Q{TJRi9A`MXV{(FCe#m7vO5h1gcV@NAOmC51p6zSYa=ctx zkzAiAd-%-)+PI%guqay~q*Xf_oRYHDLY_0&RS z_B~v$)n2JQ5aAwHzEE$0`fksRG7Ue(?%F4QjweV4c>k4Pzp0@I&GE(AkBun0`pFaE zK(xD^`$C|AuG!%bLUkIIz_aZ#a=hHxG+jjh$RBgb2Yk+dN}i5Zjb3Wtf3@#b^KfSp3(C?Blfk~ujG092Zpc0@C0_S*tj3+o6JAdbP9pqM_~(* zwwvGlSCuYva>zvwna|tq0cQ?O0UrugSYmo@LqlHH51rxlQ<3?Wq z$4>H#;b$YnD0{Su7|&TuK9pv$rtD1#zc7Sm9HDd=R`~G23bCDx9C>3m6F* z!XoykE45900aALy@BL!gF5c5qH_r{~Htk7mGsNCt2NchVBgnIgyWc9hN5`o8KeBk}KXOSIpt7Z{0bTN)(}PgwlERjCwQTnmqTYCP?i_nK6% zoPL|V9Fy~E;5TkvGrAtJ&mg`v@^%!~raAam^lRUj{05Z?_u7WLnzVuZ_A2w2Zv%f` zwr%L*N#lnOAoEX@Xno~hevWv4jXxrMv+ZaXXu%sTB6Y>XUCMo;gmeuuj||hV4WdW! z4HOIu$l!1y%awK%Y}A_}$1LCstF{gMZ{EdAH~w*Ng|gMCse}H3MZ!heU6ZU%Le7ch zP?LbK#)7sjQV}9~*+8}%NlLww-L~ahk4_Oi*o)N(%qKo~=Q*rPZKFyDwV~hH2Kv15 zoOb(uH9y0TdNHgyCTuwOx?~|gpw|*gCuJQL|9!-(`9H|>-|g4ULalu7QZHUMqnboj ziClYYWa##BZ+H^vVE-Vr^2%i!cyr~%Gs!cT9MjoxOPb|(4f=SG+5OCa`RBa#vBhX; zNnSI)Rx78_%eKfg*=T9gRA@Dydiv?!P|A=@~O=c_*CvFD0F#% zjtIuslmv=mG5<6@@e!Ma2@wVV=`glgasq@mOLO=R(fTAh5}mS%1a%l>4w#rIY*dQc z_e#sgz!Gz7V~?P}Q!&+PfvOlpm2v||+lP~WXDLd1*4<)(ke&)iY%GX`npGrnh}7bI z{15lu<(!p}$PK~LvnRQTzAuuWmpkvDr|pz_2Og)3FD4tme~#*9MQg08E6RCez2RZ( zLNatFOB|8#qBK>l254?h1)+~AJGji{S2)e75Z3UZtQyD z#iExOi6V=CHkOx|!eh@^G_LOtwqWgw-GIdbCm|(Q?7u?f zV%w<*(Q0T)dX!x|)0rMy8+MxM-Bsuf$#ovGg2pdJcS3umUN=sUzMK0!Ga(kp$w1^C z>3@JV^Uo2E5^w}%AJvQlDOC)+wo{9NCtT^>BgnBUi6G#=*R3-A)D6M-i5V$O#%C(5 znT@acBU`K|$W1TL5l=19kVk;p@hDu3ZpkgGr@9OAoVs9wF}}oIY`hw4#9G+P_g#4d zJhbjZz}G;pdzy)Fkq#dnluUCiO!U?hP59pZl}H~~UL;EENk7NitEUn=9U_Dsa{*>$5fNEytkBGLX;ouP6JL@b`A?dEkIzO46ltNx-I z1Eb%AQZ7cf2T)oraLyy@$#)is!ph8_2KkU+sTKngifR(tf%uf)$Jo!WHa}=C5ib4) z1!VG(EV4eAX?nwt^};N~iL}MbyJ$^+44LD}&F?7|!cF90DcSE$z)^FiS zmdvHBtJyC4o`id05WbO{Hu6KRYd+r>&2RpP2|s~hJAg1iWt-m-=21^T7ga!I7}BG| zX9wK#%=|5qZl+_1n3_IX>*T@QYnVFtm6+Gylf>+${8Y^ZxZe#M5Z8xc zHabz@dhby_g1L8_+_@>5{u$!8j2C9GI*r|FNfDO)CA^=V3?f_P%JzR=fX(D6lQW-J zagU-k9n|s*k6O1on};7*Ub`QB&NeJ$u5HDf6gXE&Ug*5qc=3QwI^AJ>_yS$}fZ=EV z@+!3eJ$QMQDIsNF&1f&l%NSy$rXwo5B7c;EiQ>kWAcb757z(UxX9V^(e$wx$+jrIj z@9)Vxm=KQmT!l9Jp@z$E`9(W*Yf5JJaR*^h71y~rpbnLYMl84{OS z{CEDxFRomFp0`uDoZpS3`PU-m1Rme`eaR~>8&*`DKq;FyT1=TZ+lhVWVX?(P!s&~* zO8txxibWlUcpBZ)Wd!>;8syPxE%a0R1TZs;)5}_Lk9oEoc@IZ9kvvwZnjQ3SBQ!o5 zuWlx$)H?7NjcI-Aay>(X85(yjwP>*GGI(SPh>CZ#?B`=qqmAa%bxu=Cp3)vWzE`5{ zG~oW+6^@`pijY)fFF6VlEW6eoK#UzV3t=q+wt?J0h&}AAmu`$NLq* zWQT#QM=cJ@0sd_)o4K4@?a(&p|HW|ECAa>=-_kGwQN~uV-@ofai-u|dwNDfMC*g1t zOK1t98)^g1Lr4)4GSuyNQcL+-r8CP7fC+b{41)^+v zPek{e7$FBJM)Ga#%L#nGwr0}*Wh*!^JklO_C1FW(Er2IRK|mrc^F!03D<6LB(N;FP z#3|lTh^eio%+h`n31LFCzG@I8SY%)CZo~Ca_veUFwoiyT6Mxx6!#46W{DKW|lcj8s z?)t`$PU8j`mh_EIM+g@CFdc>)UW4}@Z+H@D_ZTvUZcwTlwLkP045MS)IDDuayKVIJ&wj!voZlB9fC~Q4OAk;Nv&Eg|I{FRT zjZx<;chx`LW`=Ls8Qu-j|9(;*1vyLWIlZ7mGn~%bhK*d{3EIxUtD$@T_q_eyh>(H9 z5a;2SND>Sg!ow_gCQK0u`c= zAFhg?fGtgx?WJjCWIT-PmQFvJNBH~UAH|8w10a-L{ltAfM$fAgO87TcL{o7$QzK*Q&@NSwGA#LjM;l!; zU5N`~L9`uuf8*fBimjSy1+1|*3j+#R=iI5zUqOPLEFjE*V!J+JVf}7aBpnRh4`LcU zLZ1DrZ4dJP5RGySs-6$Z2Do@jsi3@OsN6gtX! zD!apr{wb*Q40osaq+iFtuJ>?8As-^G=5dx8^y{z_?e(88DXOnlbg}-uX3pWR!9;vW z8aAb9MBs1aJ$Q8d=8Eiu*5yONwRR^Mkc#ri33r{naFysTfy{dxh0adI``TN`U#EOA zP<@+G6x%P}{o!g{k6^r{L*1W(?JBojatPc$)EQ#MvMoXrt(otnU?rA#`MP`CA!2u~EiEceT zWQ5b?1WzutT}eo{FoVThEwBI!-#yt&Efc?utKoRTIf?H{>aOfjnSx&}@&aTv2o=Ex ztpUS3PS0eAsWNafdiO7hm0WrK2@iSmuG=%e;Q&U*?^XYfOyb8|unko9S~wiZAxJBr zI&66$K+2F7YjOdM+`F{LBPA(3d!WdZTU3<}t>{tL^eXt;Thv7jYkXw;VuR7)fZ)!4 z&wmn&5P8#2@^y05;L2?zJyPwiVB(x&;B#)7SG&|O_(_9f<~{9vcNbNk=o}XS=^{}X z>5Yhzbh{5`p1uvsqgp`kU%b5r**1jyhn4qc=;UuB`RF(zuC=cp?duMlBkB~Ddiy#& z{(B_XBMgIF_v3ziGCbh<^Ma0-`YB@8d%qI}yNZhX`2z?mL6A;quaW@<=px}TX~Vc{L@t~@IQOtOqP*I{}|HhPvO?j=;;eZaU44)g>7}EjgGrP;?kl?MxM| zUG;dv026m`{pcVD??^R2;&zuoM{OADTbRqwZk}6|{L*1Z`LB}SZv5Sby3a02wftLm zF?Qe_gW3Axy~!K154e%u2MT>xHg?l9a2~6}jOb@`SgBC4O?bDLHP%h=IGd>w<~$Az zsmii-T6SEqM{DQ7nA#TDm$08$!3zz1ez!%I0m(*)j=;K-h(4J4*2C>E8YeS(H@BJyop|O|j3PXmuGtDm=es5+ZBokW>?}fEBjbVM z5|Kg+6cyPWsyi95vbS#2QdY2p0RPmq_3N*~mth6?0VYj{;e6Hh!!q}ttW^{0AV1dC z%FT^O8|QD=HuP|8IM1|v{jDv5+a$tAUTQk=l5blGcafp1ep|MaXvI;%!}k1T=+W%w znv~`BpUaMsvfVl{!==v?=TyK0O2TKp(^h5n&q;@At|WH;RR%92S0;XqGSn30$D3k? z$xmbJYtei9V+Ab0b2RpY4BqdV^VhI==g8HD|4*jVP|2_+7T>s&CFw_x$e-3mVa1yY zN~C{SAcDcR;DK+_qMz-6{y^9~j)0IJw6|^5z7tguOw`lh+7ac^aGHJYM+Sj~EUb!4 zNufSm?MHOy)0?=?EBx2+O~PgU$D@>s%|>Dz`=hCSoW`->?Osr7gsS5RT{9)z z?hQS!rLQ=*mxpdx$ivS4(&{B6M1?{^fU%`TLYq38o3_INPke3^s~NFjVcs2K-`0qH zuKHOh1m21LTkZ9*-?V6I)cNw!Y{prIxR$BD%Nb+db})<+tdl2kr`&MJEqNgx57*&0i)XYi< z$B4IBpBb9rbUwiw&FYlXf40b&ndnFhpXkW?K$F;|l3Nw(wC<~tM2I$T{e>r;ssI2E zkr^6=6qH=X{>fs_MKNdB3@vdc8p`{LG|OtQ`j(YOz=1`k5cf8nF1#E3ikQ(A;GDWA zc*oZE`R6w&^3nHYMvB;{+_pfM-GCI84nZ3D#xnW{txGcz;$v5va1 z^|?_dr{H}lLwB(+Iug2yiN0m1dUx2J@CB~o+3>=+&b3u!68lPE%4(x=Hi}|+v1Uvx zOX(UIWt429<^vLY)0jrX_A;h?0Som&>Q>_NE;)wFcG>c>#$HP{l`e>!|6mSVj~}~- zKRQM!DmFegLCIg*Y*{Z4F#m4W0VAW8%KigKgi?TOp7$!#zUI2-WLc7B+^PqXV74`# zSiCLgJ1>s$PltG>lvqc^XFv36goOL|62GI%@|kcHTDO@ikPuW-$<~Mjs6g4KE|%CI zKX}Y8RPiTgs7lnht)Tbp6+-TQ1CrvrcMhR|idhUJHWSt9!vwCGuko<_T{440woWA` zhe+@aF#RY9AzLXL3Sha?jXYdc(VtrN&6U%yHb+M~$N(vud%eEK%ZI!i&qk=9Mg}v`yT%m*lY%7W+#g0ZtThHZoJ^1u{6pQs{(&r-A+IYv$IfYPi z4;3V~E$j%r3rC=gRR_0(s+s@vrwh(tY=4IWU3*3qttMxS}@ z{b+aiUWRWRl;Li7X>eGqLuP+zk^a32Jy7Fd+F#|k-F*Cp5bpqoJa|%Styun3Vl*dG zVF~h!jChoksB`+F*Q~RAe-X)LqO-i`Cr(%Cu2E_WIbrD6rXqrR>fFqCfl}wbE8sX3 zz1zVUnOQp92*b|@+Xf@;sWzT)e)jO!Vv0JkG1W@vBwp} z@Z#kQ$sX+&87W-wq-(L-=>n48^(pdc{M7^TuPB)7LC1}3_%WKPOBA&eFIRddrgx_NP)8RBx#!Fj%7lTdd z{pL<5z4;yTDtU~kn->Pq$^A`=US>VY$P=MvRy%$fs1S(&pDSXjc8%t##xWz9ZSQ{A z5GH;&lejc{*gwVd81jzLxR@sT7c3uq)N?Ct} zwVJ9Gg&5ypJX{KD>YO2$k!2rvLSa7BT}wVQjX7!J5g}^V_oMv&;sb5vt|b{Fm6WN^ zDGI3UN756`v=P!N9?yu`uO;3rvvTIxFH221>;Ej5B2T$zUnRA!rLK-?W7 zy}vU)(EV6+&tbBPhegSWUGgHK!-j_2usy0(RNnrR#elmM^h$L6uY# zHMWa0A;p*zsFfLS`GoX>Cg)i4TnlrUyAS8?aI4-KPclb>VR&wPVPYGV3XYqp{R?XM zzrCSs)kxHPk}~#vlmWKCPUa}Qky-&%Br${BmNThF7^A>qAG)k8i(n!oHUa4o;Kfy} zF=^6v5MMXiSDZ5*4I~{e(jUU1#55Uu6m;M!sZPTlJY4lz6j_26JT_cwT#_mLDY3Mf z`NIm-RjvlQfOq&Hx^P?c?@UNg6otR~pDoC3WQFZ8XzxnhZ5D`-mOQd>8(bUJ)%`30 za`-myVsX7yL~eATkh)spYQ570w>p;GC=X({h4YNKg@OH*L4lnos&MZgRehdr$g;L0 znX2@|9-2b)H(E+Aonp0cV2s(8*JwG4Ira_;)M4;b-K`8E4s31=qJJ+%fU*N?!pH-3GeVaJ&-J zYaA2~9&q!V#CbCrSnYS*!F^)F^hQj7kuSf(C;9ormmhw4oVi>0N#RtqHuhkj8{5&} z60=20L33>CL4P#@t$HM0V#Q<3Up|SETEPHS!BIZ8*|6H{@s|c2{h0ypG%gjzHuq|g zQ&~WD-FeH_UwNrjZ%@Uo_LPW`^NxNc1VAg2du!itil_ZHoV<^`SOSj)AR{qkLXf6~ zHiko5;UdV`TxXOCDFH^Nm3h$H9kZQLL=aPzy#9}s>o3&rJPA^qmi+fh%eDKc|hgZ zk_)<5GjI48Oe;ngdBZ35^PbP(iy3GlCLahMVhOh|knE@O4-SrszY$)qX;=rJ`UU3Z zcW`iOJ*QU-SIzjtRH5-bj?gxK=ha@`XYgXeWhm_9=padD1HtS?ALcY@ViQc< zku>aP4tf6KOABfD$c3k;X?>dH;Snb1-eo7;8?HhU0-x*fu$3P@R&ZdJ;uz2t`P9Ol zl>w@ntyXUo(43~+_g4O5-%VKzo+^_r{!088@37^=ADvO&7ch*eS>MqWSb6ha?|X1% zY9zUhcYZ00!GvB^&y!PuTxR(2C>F(KMemgBP|BsXpPXOr7lmy*u@hvNKU=APdl)X zvR^S6=2GhsCH@wGdc~5j@sWBp94EC&{`5244o7Gxff|TOZ+A&DQB6htc-^ahf52&= zY);z4))6_pknR}@wwbAo|2u-PFxWL68547@yLeE3CEF!|FaQcs1|Xoi0m}J8Bd3P> z@{7&P9D6vu1X@JPzbZ*ol`2Ua{r7qZ>%jmmWQ$!5ST65%KO4qhNK&B?XH?Tje(|zi za`*+=cP)wyhd<7P804NJmC(!@8G>i$!%I%D4+>k%o5g@d+~0V!xYO@4@HTOW9iq|m z_yiv|cnx%l^_wVzO;>beD3QU~I$59$$Ac*hJ@#L3C$Z&n5Rl(t@DkSWU-#c&kYcbj zk2Q0_8a9ZLA@03{gLl5kKNhlU11i<$*+s{=#lZb1 z0LFFYeI)QbaL6dw+&Dt+rHk^aTkC?)Xs zUIaqt}(*v0$hcOJXY3A(hqhs?vazpjhM|k8`GfwZ9lea}v z{DK@q8j;zO&qPtshDM_4_Z~dqm{wLEi02ZcEHx&k=2IW4NvQqW5kUInzXMLAdcb5W z(Qu*WO{WckNt>>Hc+&AJT(SuhxRD6-!=z_F--xp?kk+0@Ehu`NdI`7Suv$_Hhzfrh z+ne$u8Bj0?O4JgByB%9u>ONK*QG!Fsab^6?#mbR*s5CTX{z~n&uviHz=&uvsV)8j= z;oyl$b=!@~FaCVDM z7r0ygo<&|erqucAt1sf*?wuH7!R~2aTkzZIlVnV;cXKyJTZQ*$Q##Mpg5HRwzcq}2 z!z!i~n&DShZ^iZKXJ*v?1XPx2nQeBruyB4gZj64hd{6JoPif#QjEPV08+DG~(bc3# zZ}im1D6Xgx^6OxO$q1F=Ol9+2o6}P(i3%Kfvr57iQ(eqEbU+_A|a0EIR5Mw~W?--n|h>5fDf2yVshf1hjGfn@U$odZTMva`A31nQvPbvY+=pgN>zUV-CpC zy2gB0e>@=o@wAJ6V4Te~7dBIqM8_fIW|i zsmO|o5D|vLB3dpE%$e+zi@8lK*-~5#Fcr+|bDdU|3VU3^cOQ`flH+to21v6OC|;kK zu}wm?5u(DI%~GW1j3 zMdx|Z52NJOzJEgOv5Agz!d*-cS&Gd1o9#{2Dci+;-NS*p{uD4mnn5vjG@SEn#G5mM2G%9*mMt(%0vdG zgKJEQWI4&Hrb{!V%OF&^sZ6%*N_OxEY4csIHx(c8)%)UI7GV?&S`3b{vinH1C2f8E z;#s4e4tC7O1_GD<7iZ2R}b;Po>STqK1k%c#Ll4=9j%vGAEM5|B5`dr zT-$Bt$U3$fvz^ah?Ok!@kC>$%;oD7cw(M~_lcrOR2c>vgnt2vU&$sG`*suMsOE?Ra zO%M4k8*ybQSTWWcm)eGp=6mjm3`$%2jY)9gz}89MZgAyP=T{i-@l>5F8X}m?I7jO# zNTyETQv<%UB&C^rVB!%FmBQhP=ymI-sDG_Eo_kwzuIuyI9&D1 zB`)5mw$PXQ?e||%X7t+HBf`sJ($3zDA4is#)u$}4XA_n&{$MpNK|m8;Q)-yd*m0Qt zm&l5H4$WXmjKk{3z3-+rECXp*Z#Dlk!0YlDHdYjgb56Sy&v%Us@~zU*s+ia@W3Ep9 z(a;9gyJ=a^CJj!d!ZfI)Bu-ERDvQ{GoSc5NNgoB4#uohx4=?0f@rl+Y%T}N;Dv6s1 ziY?Z(ApZ6xlQj!_#s5f?d5=%(H`EE=2BrsbczhrFN}11Ozm=Jk^9@h ztYkuet@0^Ia0F@ue(JE+CHzZ7Z9Wb_{0JL&qOEYtykDx-atP!8k!ogCTj|);>fGl4 ztsZB%2bUxRuA1O2s$^&A4AVUiy|)?&im{yvO2#CUytR5ylVHT9sZJ{Z-DI8xc%W)G z66_88qAOrLqDLzFFeiJ5YPx<$rMlGOF`Dy9JAc@^6%j?ysa>&2ks6|85&d=M;ThaZ)! z1NKG7PE;ipJyX1yCUgZ~eHbJ<+wsVPZ0E3>2&NE3*jAEu%W*XB|iYPr<>m+oEYuBmFwql4hHosPojfEa0 zKa?H&nw$*roK}_hrp1W}jcmQ7y!^x)KV)p)b>i$|=GS-cT{=`1S-0%6BXAG0E?>%-paHCwl%RH37Dzv&Zn8t);r z^nnQ9sm-{haM(5nm=JxY=sH9CAR6}H!ppEBd0|S!r?)ZJ9$Nm#0)A}MAh@C3Gc=g= z&NJTau0Qjb6C{r@Ka1Hi52&VvFGob`LB=rXFpmrDBRqi>$@tjUqBYK>^y*WnCwIhQw zVwCg$$a>45IHGo4bdVsyAp~czK#<_>?ixI}LvYu@g9H!m?l1!ccPBUmg1f`uZUdKZ z@4M^lQ|DJ#b=7oFcdxZxd1NsP1<@&Gh9`{)`muqS-}v%nVf%Kn%l;T9#^MUtt;$*7 zCFXyZ^$&|u`e4P#K<)|>^j|3FK7fLQC2VVdA5k#GW{JIR7q-xP_QCSY9p!DD!CMW1 z{8rn3?r$@rHQNc*^}6an_1a!lIX{ZO@PbaBZT6jZus%Hq%K|SkCz6s(?|v08Jdf*V zRWJNFPAFt+?8EcgZNln>s_+D?!^DL<50!=uX?gK-6Pxd9j?=p zK39nTXyS_~bX{Uz zhT^k;6F3zz8`#%DMZH_85)ca3kB@-3yV^-BlORneCJmL5_^=R9*=+C&u|6CLKyeT7 zxGRRpc8k!fw28{lJBgKb={byk?-Sn3vyUzTIaER$6)RNUvJR4oYCRTGMj=Dgk9;E6 z;>E{^Mbnd9$?sy{1h9Q|{0@zpM7*>)xl44eK5bt;js``auGq;>QeAs5oY85`yPeUb zhq!%YA3r1f*1B{kDjUL$f5q$FdLUWJWxhZ{+xva8b{FWqI8L z{A4G8^3<`5Un{N05IDSXS6t$#{oZ0s#`gK-Djr~$vd`CO z52kx+ow~R{aqssoVCD;Gr0ay65S*}$BQKDt1`^c^2W)I)mBDbzTm_HDZCzqBujITj z5d%I>@tC!^j9oX*IT6>7&X|tY6LpaHTf%GV!z&->@>>>QTpF5=q}O~{+3S$& zu;vR0s04Nxo_V)tyOS-D*926W-45>l7qKn~DusZ7BA!j(zJIhEPGa1)gw;gk5*Rsk zb2TRZ>^ti?RkkW&y_90|4&8}s>V}f${$&0)zM-@GlRFW~RNFeqz1yQ{l{`y7LY#1j z(JJl_Bi16(Z5=$*jTFM?%lYa1fiG-c8^in4uwuA--Nd%@727`6^N#8JNj-<;x;yt{ zGeHZ`C!OS0oTkB*I4dKPNG@G`co$S#qCTzzdmkLEUi2X2uaREW!b@G$DfqOe?#ekA zTjLCg1DUvW+$R498+6*extF>w1gaL2n7X;9C*$9jIMTUULd|u>&)?A7Uo+J}XeONf^G-&ke+ma^b`YVWSf!2j zGH0z~)_U2VTdR%6=DZ~CEx17)9Oh*wHX2o1_V>WbEHgR#gPQaF%$H49Y0$E5^UnQ zye4MNbRS**+A_C3Rd4Yr$jWp>2j#0KTal!f>a5o{U8sfhI4+w({MC%d$v7kzqP>S*2 z7cay8NG>I#{07A+%j4rrS4o!L<{Zz6c?Qb%xxyp{5sT@UtQK`3lS=g!aF_L?S=_3( zQu^NmsOvrW?nKfdi8(C63ca|m-d^3#q~c}%>-WLa|S zc2Ihb7;-Y@a?PMDxN+Vlk91_nyK5hHO3{3)|)M{^K+YVc-Vhq-*$zL z9cXMvxzx8$j5m9|*(C>)58YxBZkn@;(52NeP1X|Hrx97&THcniaP&PJx!c)A`bwkC zBC2tYW@yEb>ux#1Bsz#zXWKDf=7vsVCjYytnE#4^A#Jx1zDaWXF35U{M6=_ogo}F0 zu5HO5<2fS3?oh&oI2@96OwnF6^N(9rYIizQ4MG`>Y6nYj<6E}k+QMl#C?%G5k{J+y zoYvS_eKb{-nf})cCMJnG2q)pUz_pzTno#011IHM^);Hq-nxT3dL6J4tNNo)V@^ln%a)!y!4^tn#gzE1?blO1MCdLzJd zI%|V|Y?$gPtNPYrjaMmTmp%|N_tw*)*+~V1V+UFZsZ(y~l=VVSo?;8lp@ttE)*`cl zUGBN@DL)N`Yo;ISq@;xZh|CPOQQMMcQXK0W!%vTZnb9%&z3XZ2I3U&tqtg&s8VKbYF}C^-kWcIRY-o4|JSy1;d^7-#^bXu_|#0yy%7pNnP$4(jCZpg1MzBs z_-Wh%Iidp{+WJtN?;yLWT#&O?E>Te_6&as}Me->3`73!p0b+_hwMolmyemhb8%SX3 z#^>Tqs`&qNpWg%<+8CTu3l-P=WGxqYG&dgqSmDs|jbnAAjOpEWgLO#KP)w?^;Yk{p z?pjV(_SxXqI<%IoyjwqRS{)tAZ!na1FqKFhd^de#Vb7urAvWW`_xr?D*i;ad$h04t zGhq+n$_6yf0%XJmfkHMINt!5s1gB-NG+b8%KileI>XPQr9!D84MP>ABDlriNS5zAoM`-?M($%$Z3`f( z8()?S!kLygxO?nNQ~Tka>*m_Q(0sD(wsr&Bre=8u6(eaC?`%2D=zU0LcT}Lc4t;!gaw74ZbE;~=5?wEu% z@yzD0JjTsRyn6m7?};b0sQ z-q>Cx40`|KKW#i(yUVj(<$s@@>)6%46eXbDGE3gkzK6y6CI`))wLtJ8Gti&tGSioI zMavix47DV8gFU@iLSxqcr>FDZBFO)GweAH$Na`RP<3|Qpvd%wa-K8%|gvo768%N0A z*QO1|75(O1CMd#xR>L>Fk*nenpJ)_CYxIQ#4Prd2F5=ypi=+;gB#qyQJW5*v-C^MO zT12CX!`&an7mP0%-C(z3iU$Y<5?TcP3fBa|8(frhjA?|@fsvGOiaLQzH0JwMe3?1z z6SdZD6J~P~ejb##u2;b~4deaCq*Yh;l8=u{e{{NV(xF$i{wLB^lb;dIT8JUWKdNYX zYHddm6&do}_6i(_dR5owY*!y^z^!v25{;$pcvajjm?X9>oUPR<3r@HEpQEthCFYLK zU+jdvL59;vC3NKzgo9Ltgylp7Z%l$DRrvD5AF|_2;<$Zs=g4rQ)bpcUG=rWLSS@Sf z5RL``9=m@ap|jo1o4vz?Mp|Ym0R^?{rEseSnk=48*3g$Xs4i*uCF5E z`js9lW}OQ#+fPmNEN3F?Ft^&o*uH6q;DNI$Rbhg7%C0E^i zv?uc|_1GlNTr@5<$KD*YBl`?r72@XC%SjxX;WEG?hu~OA3}Q|I-p~o?|M^_szvsc# zkPO8ZZEmdL^EO+3+G8gEp=*Q2I-VL$PegyuPp$O#!#clTw$0nYCk@t+mX#hm32V8h zX+tdJK6V*!YLLJyqpJ#~K|waCK8IJ@a>x1N)7B{S`yM$#6icgD(^iZzI9)OThX!2k z^}z+-x8yOTaSFMCQFl^QQ2<=k*QT%zTi35n-3?}h& z4ecOFwzA~(iOH73*dU98P0nK&deEa8zUm`;A?*7J9j%cJpXD(vDo#{hHQW;JP9ZAayWCZ)+e%!%!4t6 z`aGG5yg8T|wjIv%O7!aI3Vc4;>*RH&3M6qPkA3~;g#FT$D@<9V)wPCG z^m4zR)BpONmr%r-7x#7c-2K_Luk)93HYN(TkSqCpYS`*U9EMRJmIHZM!1w!%dNHTR z!Z66TFApf1#NqKjzk(NeJcRPPYUyF+;vLk*oUL+_ueUwm`Ms9lP_aCj8I?}Lzb@5; z7yY(EHy857ypQ}uH+0#TY1G%n=|?Q6Y^}GhLbEWm{T^}E{;*0z9NH7z z*SqSh_KT@$?FZ_9Y_DdD&z4NW*opzHK6rhh7K^_}RW41YZQM@Igs5_|Z?ujoGQJ!v^MeWz`* zo7(4Fm2}cei%WdB#II=r7Gq3^adMZsQK7@|>T|q1`Z!?Yidqa@lXuh0qyPIV+kw|;gz}rlbW&rVb@!Ex8 z2@U&=H1t5XsnpMx+JpzU7L$0e0<^g|SQ6d_Eu^*-I+o7CW;@5k>*@d%o;;KZwCz*o zb!QL68$u|nc)tXg;JSYd3YD}gkv9>ppfaifW8Ax`yiHP40cqW} z!E}x#pZlz-WP9t5qYA)F`KD-=U@=T6M7Nug%2ked(&j z@uu&4A$d_kOx)HuoDWAsE#qbrp#8?DtU1UR&pSQ6#j{JkrX1dsuGHFq)cx_q)&Cs> zo4Mb>+Wlk^(`gR-3(fh)?v@TfEW!rQaUvi#D5nNrK&!CI9P=x#wCQB*!@_SE$?+OL zTHAmZ`ncu7+>5_;NNh=2ojX1AFBq)k&i2Kpwb36C zB7Bb`5f;-U0E|xNbX%}SM=en!6ttrW^J6{wS)AV|y*W~h#daNn#pbpk+s<(;(avF~ z!D<7fYJ)bD&#==1L-Nn!xNH^((aB2e10pDa3v#houE%;XbtKn`zqv<>gSOD5N3KKp zqfPk+3paXxjM%851!31!a6rJBA1iUgr?!&>_SEmT4ni0&#++%^nfX^*0lMuz^Woqy zbdnzma%;~9z^nc7zV&@Ube{myx3nS9vUZ-kJrJ)V?CJZypvr;IOqUR?aiUXVPR@k( zYrw*}5?m=_n$;xyzV{0$V)x7WUYG0voF~xkDq#JZfE0qc!P%iSNqjc_dp{_^;Xb%q zH4Kh|$wHT|ewRD{0Lzh_2|H~cdSyZw%2A4NpUZ=RyGY0$2znk>@kQMzUQrB%ygtrY z6{|GD;&{3&9`mEE4p)~I%)yRnK~6Q381J$zvX)6xe^mJdZVRc-bPpf@q z9McuX8T}X#nC!YfIO6ycoa}1Y_fr>E<7WwE%3efMw|se*I0qiQU-^R6J_4_wAY>qe z;>E>gATr4mjMpl!udSV|InFG z8u>I)G?b3#%rSp8;X2{5vcLLdC$s$5!`*0Usuk zrrJHd)b=&nH5Qp*E zTO;CI4A@t5kQf!9c!)trSvPJI$?Io)az2um;&xkDV&Vmf0qP$awVZjsZ6ZVa} zOwg4*K2KfG^;4Mau6Nd<@A{Kr4EG{kVjJ_FCV4gFGK>)_SmNH$IZ{5<*HE~h^#(BS z`55lH=geH4n*E=q|9@OS{*wR$rZ!Y3tVu_SfAlSGtJZF+Cpl(>UScHHSpFmY0Q@|d z#zpVt?2p+BYW(o8fX%0i{RBbvX9XzA$#3251HN*v%$4xqzOab&zC*}`T@1QGxIzTn zkocSNjW*tBgPb2SXUtikp58jkR70mxH>AJF@SD*-oW?fL!8%ZTjr1o;X8_(K`aL$! z^M>WMzb}nL^9Aj-w`~#WJjiqH?gY!ZFNR43o@eDA?;7O$!J)ar+l{gQis~ch zo%4!~(K-2;bY^*7&@OV1>VswM%cHy9LBxVd8bacKu6}Ll%O6YP$h|%Na_z_JX#Lpo z_Lmy3>0n2~t_mt3@0Y>cg~wm8%^~ zKJWP#-w~(_UkmP%(BH8nTuCbc+kr3cp$7_Jk$)m*kd&M;=YOf7SB;h%|B^e+C;qvo z-09rLQg5T_IQQ%e|Ibe;S^Of(Af0{P+$$1(;icbeA};~Aod~>O=m>6+HQ(`w*6=Uu zw;lg%+kbNin8kD?&I#1*qLRm1MA5jIPT-=b;(nq{;FL=EsfdzI5^V}7G?yErNy7tB zB?9EI`8=rIPLZZ!g!S$^{kvt*jDMNSSH@M0rp~}#SNB9-U#|L_dmS*JnTzO8_RHu( zVJAkVm;S_luwBq6f00no9u%*&vnV{s?axcB52+0V(ih=4eU{W5fIQA~{DOa%nkI5W z;2bc0L)F%m5um*TW4IBlA?Gt{g#eMyn!;t{4xig7j<4>1kOdqpdx0)SpY<5>hJ2r- zN?A-h-{)2Sok`Y`ech)3{1W{qBQt3}sWCzwf1OXKO}#Y?itutl_D%TJ^f{;i-B6e< zyu+}sL6W=(OTBIS37gGFp!235>Q|EjFxY2BAzrgoZH?nKn(>PEg)=BWEkvAS5Y>iv zL=D~T>kc+uo8|de%q8-?!!H@V7VIc;@n*H{gS(HBF-nFj5fr-VJyOrEY`_>KXOT71 z3tcl|keI_dli&+u7oPQ+Rlv@Y^GAso9*ykf8zSMb3@xs_@1p+# zE8sK{Dv&fV{IWT*)qGCLK4g32Nm*%cjvV5D=5NFOQEDh-@j%Pz12&tUQ;LvZk1=BT z7bdDLgW}czC<`sTk>J&9ENEauZq2%@*}dd5|G3sFbl|fbQjd5b9|%?NO3JgHD;3o5 zN{f>0rhj036u*8i*&r(J^gCLe=mDT+uxrFJH6OGawQsVyWG|~E62W&E&Pun9{dML{ z6Vv5s_PS5jAizuYi4R(?c+F!4gNqIDAzgflPeT}&Xp|EPO+u$+k9*RCWKqY+u}@+B z%WqK4tdFLCyHYUUp-lWQXF^wY%(|jb@oC6`8RiZ0$B_2_;R5vCAkR@Qseijp$Z$@2 zD&X9=AU(~c9_Qk^VnrL!%_Wi%cg1#99u-8y%`vmiaf#X76n!cbOd-9x!c^DEH5Hn@ z>nIU9d+=L)LYt4vI&*rcdyaa)A8ODQRNowjd`o$PvlNaTkGmA-1}ypfwtEkVXr#0e z;RGxE)S?#nRs1RYb}B2n&nxK2l2npG&RXOzugjVGy0|3YKm5CAlc}|B7{PbC8IADU zsI=|w&Lr5CFwndS(`@3ATJ{s_iJA>?uj-Yi-k(o|FZVp&j_5{gTQ2*D-A?K4vB?@s zukz=o{6=|@TKZIGy;QGs=$^RQk3B^YYBrh1f!=lYplvbtNbR3FdiT%Q{f%F)P`=77 zV3Ogtnsh|)+YoUMIZ>|C@8$9t^VsNDu-vE|20ckE=(lS1g8!0Y-LAi(SO|BSv%d*d zY@(#P+#v5^W5c|FUvIFqI@oATbHdw+@F?@7;~?a87_LqE&bDm*oYmkU7d)TkW^%X7 zRruu6vt;j;cF*&~FR_mEIl%0U^tCP!jH?k77bhbw4v&kAn-Cx0FJ2kH>mr3-6DAC$ zTbq-SDSfY6hjl-Az;#gn?p@aro?7T*T@_KT18Vx{tr^%@IV+pFuMw?;b6 zb-fI|9vvQ&LFM`XGoQ+cRo^Dq2DBsy<@y6a=s{G4{L#<*z&T62=kRXIu!}@_8P$!S zrC?Y$n)V6b%{?ph_YFl>HS&hT6CyaJ!-?BI{I?Ar)3|T`;sbLCm?#(3tvIB<;QYfy zi!k0m4!Baw>ID_70O%n{%cz*lKkvmPeKF>0%p~YdN_c;gp}SD#jl)Nhy3sEt1U3?a z3+%S~biRauD1gJ}LSt!}BURA(G4j!e<^U;lCaDwx)S$hh*BHce!&nhkGa9nTy~dE` zVUd&;tc!FyoTYDWT@kXK~E<@l{pU1 zuuB`J@5^P_(*!Xr`|;-1lEDt5xESO4EfMXKuVK;Q^fw1hyaYNks$$oU*+P zgk+H(2@ewIf!`ogT{=%4{l%E!M@Hi$Q|`^oN%Nl)P7|kuZgu<4ZOL-9ZdK-)4HbN~ z-h9K{z8$xC8IZ~1x-~G>pYaMEz^HV1lF6+k%u+w)ZwzQ!o|#vYeH>I;`}M zH;L#5AK$#GUbo>q?deQAC5O zgXY9OA9o>1f72Gijo}Py$Lc5C>coehz8d5~nNAoF$5mNq+(d3(E#nR510I_s*6!NL zTUmBt^b(6a;alHoU%a{^p1!V7Be-f#WNw+p{b@8^El@^y`_w~QpA;UNG}>3%n{r_t{sV?%wG82rC-7{(`UiT@5)|2@4#|8>tRA>ag^ z`rU)P4{i+i;=8rG7zv6A<~RRF@;JTxVb|wvM7P3ZHs=Ew3%k7DvnvN&v%B1tZuNKa z@pj(5+adQw&=pu4m}|efKhCqI$?=i?GwJ_8`ofu4i6uqgNqx6{r|K~rQ9{D!q=!Yh zAsCql{g73in|FHjQJ?pb+=Y8$i0%mm`{hY1h)>hSv}RrROIJGyzOWQ=KW4ZgSV47 zX2+uS(_x@!;kKbtszl(p3@)8=befngm9fAnLz{TK_<^1Y@BNSGbjc(Hp7dgL9%)Kf? z9wS!(k^Yqs=!%nS;zeF8x8>d>xvJ{>=IVO3T52nLmE@Z2O!QTRof-N^@O?7XXdg7Z zv|On`Et+vTyfpcNoq2hf^M-MGxVBs|wzR60DfOZ6$80H6qS*>Fnc=>Nd-##Su+B`t z>R~Gb#2X%>wvxF5>3lCY_`RQm1UvhrTW6p%YU~tgI1iL|(x@Zq_SNn#AG(Cm3AD#=|L5C+uYOpB=UDdyl6VySsIH<$wN}jcWvI(@fR#&#N*}+ z$}2&#h5oTWy%pP{NLYwa#@8>T28WY&F`QRjz!c+Se}*`B#2)(}VN{ZjQdjJ)UT2A* zpOjB|bOJJegFbg)BnXDpU?o@*6hu2nrnggMzCB;mAB@$Ar46zA$UO(C=zOwT>F;!Q zsEd5x@Fs_>rRojAF>|}-dW0&i9Pz8T+X*mA5(TQk)JHndGL9M5vJJu_M_rR?r@f$Z zQG#7XqC3%OtAki{!U#W(87+xD@*>9(&)*Mq)q4)n7z0BYazJwhvWT4vdLqLb&x!)*MYTU!Rc?u(!qh$;$QvIV0ENSi0ZeX$|I7+= ztu~UZ;X8RAEpERTOVmVc-&_EnQ^d64t$^NJ_=pG8nSiuk8&8FWX}NS4J-1?Szq{cUggggqzt3}K zB57ec*v8KbSS6RwZ(*}R@bO z3&r6TnM}7Qy#JYV-W}(L2=YA(pktp`$$cs@D(?;@_%sBCUqpK90q5};mD zvL%^D%X*EzhTl2l2eiIir<%P;=2fEUUAy93=SWWZl8UXP=G~!B`F=d$8@H8Fu|7G~ zXUcqHLOcm<|8X!}N=8loNK0#(Of_OG75sukdmzt7Ux_{AwwqSe5=&!%U^;e-UA85y z(TFfji-r(~*PzWNrygtm1x~X%!W<9`h~e&sRH=cfpb?wp+wX^Vy>9Es<{~ILSys)$b`{M4Ox!S0g4Gsi##3ZcXaK zZZ-JMh7s+l3s6N@M7xWS)vm?+7Z`3!!RYF4^{=QDO{`w>8)jAM$1)hSHlwUkzWCe2 zQPZM8qO#0ngg^rEieTN58GdRiL?Rse}jBju9fwzlhQ z;_s1!Do1hBj(WED6zmhTKM_~Q{hwI*DshLGkox0crZvf;YK&XgWD)bqpBK8)ilj<*I!n)BRLqbt%u9Zd_s z9}>CVrB%`QNS=JTor>Y$u2n9)F+cmmje9Pi#~ew*wwN8k(Nuf7?KW<=KP-&fO&=%? z*BraJ(Ore*tGYrUSiLL4r0c0&w2qOkber+@omo((}edcKL7Sx zh%oVFu>X!s-w_r`j(yGmr_ZOHwDkk9T1f*>gAYvrRI(R2n|}jm&cKW-P1k zH^15#TtCeY1o~$6jCAp+S+cC>UZr-zA-0Kg3SZvcoc1Q4l?2f98l3GMRxr_RTwF{? zdL0j^d)c|Y?-cPtLht}tPS%n4n#WY3639}0Q9G@EesC}> zhzzekY<5>W&2~SLh<5P@uY8P8K>xLG7f)0+Z_7z^Gkc^LPL5cH+=ELaw{Kix59=TB zrtpKVcG#?CnfKSs%@OnE&Arb z3xJ!XJ8%a@8{W~K^9hSJU17I3hB$fp!K>`cI5nJw%(Nm;G?MauYwVe>e-|63^A5i z9)G9c>>cnvrSutoOO%bT!H{Ah=-sFmD=V?BU7t(}zyUF&3}v{%7#nh2dtudoJpKK< z70VP2>`N9FqS&r>h*;JsCY73YoX&494Z|O<{?jVnM%Y%ZA+iN38A|R8|2~RZutwswlvIJXy#WXo_a|(}}^l^!7)qm+# zpI-<&)XQ)#RvC+%6W{LetssbOrHv1_ugZtxn*&^1_|sT_iQrsdixo=aPq1qb>RqF_ zC0OgH`=RZemxqE3`S9ECm(d@a5t46ZexHf-UpMgO)jJrOkkQ^dAIRfihKK$&B5Mf_ zc~>FmMTQA0!3+0FxE1B zyOoVd!qi%T1;oGv$p3gJa$>+x3JX}0xy@Je3d*RG<<{5^?)A?)L8a0)3imSma;q~y zXbyA;8}xFAprN)}-loM&cm&&t26ukke+-Su_vsZz+29`b`rW0tgpB0ADA9n=(a2c~ zif;@$YR_KobEP?Fn^O%ppnQriYk05X_Zw|`kGVfH!-?N~Sr(aO6c)kbL7_DtEW(X- z5dK)wFu4qsD5%A3lkkAh5}K1J+}iBzDbv`JB!K?KsSAlAsTE#h-f|a15Z44(quM(u zN*t?)grjOQHS=$0J(oHiN<8n&q!#S!N;#Q>3R&zb%r+imjj$cFj--O;ep=t|;OEru zyd!b^-fyT7=*}bN8`Z^tz19uyW)bL8h_&6wd)_Ov>UXlA=|I(Ey!j*vpFiRowtY;d z5NOXsYv-O%gV_43iSceYj|DCrz6{tj9VNG%UBv=#Fieuxt-Q$Mo%E}T%~v)toZS~N zGmo?5%qX^_65d;=y%#Q6^?-&VP=K+c`=w`BAP};1GETl?APYG+GqMFlW+3&a>G#gu zP+q!ynz~s-6(({6`Jlht9;S(1{tocR^w8$XQr;&U*mb#?R~sxpS$)ANlNTq4jHnW& zf^c-?C>Xtcf89&uRPJ!e8&+uST_L51C%0>~`2x0tKefUYpi($!<2Lc2U~}V=fe4F= zb`SKC-+&rQ-&oW1)`u zo6m~+u#LMv%jcXEhc86ARWO38v>Q*rL67PS&cV`f4tS@U*vh&mKLya3{FqSGHjtxh z^CWN&=dWH1ROd{VWnB|rLN-7TYd4a-Ici?b5@O!bfsf#tr?fDlOLb(trA}|gRkD{| z-tTiz0HBu0A2%S?6pzIXrzelC@>u1Dz-4{;w6Sq9PX-kWF2US6;o61qBe685JhE~9 zCXH;Z6wJja_K%+YE{PrxyAmm>&c%F# zxvO(Gb*Gd%AEgY8#4<1hk{Xf-<`9$8f9^(H)r>Ayc?rL`q!v{OHg^uwBK?kOEJJki zM7)+*{=~m1Y{ztA_EqcMmQqvvqZmQR&t73{5i7>8cU}(`3i=)hd&dN~A;%KGQmnDQ zu~{`Cx9K6svafgNGy)pK-Z0rtOnToF^xNq*rP-z$?Y(w`8#|z)VCuKE?Z3j&^P1ar zJP~&-Rm9G%v`8QKPw%KEG{R>=+>Kb!IQOflJRJ!kTF2_X-A;-%mK(EgmVCdiwTYgK z-D4*0ECZRB>?FQ=a4FxtGhx0n3kD+%KYrV(e-~fZlquHS7b7mRv&dMEjH{?clUC_% zV7^Ujr7scu!henLuYZ1RVrhx`@wDX=V9CIN=KE6mPl}0!U<4(&{BRmc>(!q>Q43dQ_J_Eysmg2y6~(ij|o-wVw}^A ztYl$`TFBwByvpIB6EmflXA<$daYRsM-b4%XM)ICQ;OkH|Tzlejh&4zc1bmeW1{k*m zsLZp#9WcCVJI$N?c}hB(DTQ6ed8>Ks-z~`h^!S4yMMzZeISA)%BEjnqaL=)z zFH4!86t>ID z3mKj`=|BGI7?}1P;o8c6W-i94RM7j8R8QGa1GhJde?L7L5YqLM`dL^~VG0fVfcOjM z-&VA#WWZcW2`wi#cO54uC;CtIhm%+ST6=qRhn7F#ef9Vhf}BZI zWFuhF4_U`K|q$ zLM1|5GEXsKK2TJB(!M8hT-A>V5{6okEH>QCvl_Crhehgf_DltveStfs(JSeSym%yL zTGag^XoT&&C#|?-X@>1{8m4C#_wdc?$;;4I`DVjQ9ncm9^Xdu}J+YL`)Q_MN*4Y~U zWN1+E`43oZ$sf%DF<8gAeZ(WRTc(xn4??;>%V*`ANY|9DvCZ#pRDU-_qhp1K=-#Ju zzYtR?uTw1@3|BTMAJP{ndG!C_9f<6uT$-J*$BlTMn?n80vF3&WY(%Q*_!DPCgii^Z z5&VWDbN8!f@B@4Q&1hB|r;NJ5){T~M9@C=E;K|{m%4n6O;Kh1|fH&s#--JDJzjo)H zmNEU_>OW4W7M}>hYjmm5ME3*iD!W&nibYwWJrxZs?6t?$Qgo< za{u3T<_$|UV!@EiyZhi{5Fi<6hUIh8HZ{{iwwis`H&5BE<(Y{J)W^^qmCKC_vf=t zJ(S`7!OXvLd)cX}zB)9s2W%qbt>|$tZ%68Y#;51Kk2GO|2?ws4J=@k&;^Cmtc)aZgSy;A34nj8@68X@R~b9Rc9I_1K2?6$ zcE_ODNFEHJ=X^OHua>3ho)<-aH|%lq`5Y;qZ+arD+Ls9q`(R`PldLFlHK_UcyZ$Az zGs7$U#i4tg1|fZ@@wUO zWy$qe<8a3eM?D3ry}}W&w+{GH8!b=4ICCRC%b7j)t%>^Y8mzzm*u^Lp-5HRmRCaWa zIqD^2tMf0zs$p1P;OxpMo~9x){p+x_)0rhnCnU4RdM9WONK~iD%&}d*Thb#WxnUc% zm%n29>UW(_l85Z&^NqFG`Jids3RxiUsOFhFgE6T z-^>>%!i^-#l45!@Z&Gz#*p1ge7A;PbUQx59&%)gH#L7_!*v1&J1Gd1HX3~-kyR&K# z;Rm5453R@aCC-zk(w5}0|B#*X*Z`Nxu{@1i#;$-Jdc|3W9h=9j;qgb>NsIAmt@g&s zEXZiI(n-YpUpUM0v31{CZB;?+82D+S*=wZQKRTgutyCqRY(@v~rpadCg)A5z4fSRb z-@25f>g^O~=GpcpjdR~_8MId7;9U=(k8~Lw%Lv^NpkJ;;_*g#fVS<}@OL64FY`wPi zS-Or3og>*d|E+2@jwd;w!t)@Awbv9A1M4FR;G{Y!-XAVG)EUodU05J={3&eYT#LRI z+`0MbuIdXPpS%7ymg#nu+*!6#jO?Nx#=cKT zTJ66`BBD<^&0Y^MV3AZ7_>D!VY14IcYP)jiRGq(6=KX*7Q~!sF(7=Vg>69I^ z>Wt34OjqL$3I*T74!*C4D?=UXn@$27+)0?eoE28vpfSZ{PH9;M7b|1gD)BE>t_9x= zzSm*U?*7_^bEvo0j5ei-+WoCp(4g=l(Mq;xs1jhH4W4K(OhMJq46fR|P_OOO_#ZAn z+lAZe^8_!FfnL3iuSR;g;)j5*(@$tFS$Fy0@(j)M{q_=D zf2M`S=*$kWcGC((H4N-#j&q`l@SfmQ53X|rb|IRs*3u}f7)tq2?K_Q&XG{D1hw0%X z@*I*2gufXmoK)SRV)$=QPG`YIu{70_<1)SYC9UiAqdtIHpLdYFk`B#b!`$COmcfZV zSF(iBnf=Ug$0q>B;k{ea=I431$RNrE&;Rb=C}@UKOdBrvZ%7la6B%MF`$h$#!7AmH zyB9>q0SIrf%FLsWUct`>XyqsRLk2K7L%LUoeun!cVvz*~27!gc-+@9zwXp`-I>KT^ z_~O_+&Z98-X{?zpI>MT{P_2`1gAAu4B;v(y^XUVHevmUQYKBs6QS8FbJ|HhoN8VZj zeMLL#N>fIy98N{EdV-vhv=3Z8fd%8)vVXw6FSo+y3h8cjSiOBoHgZgG=GynQUfO8i zQQWw+Se+O|O}6$vJZJ4}-d}>He}HN(#5CnvzqhyUr}X+Kyg3~5<6z6(1_{E3)!l#9 z0xpZkc9<^R8YsVqpiM8@%fdvb$TW~c(z-^&5S8#umAEflD)<6RCW_YzjPzbCHQWe zB5x51ado^0mxqwOrr`{R^!l6o?G=8x*W=A_L`me{G`-Ex8{RXt_lEk3WC@GevWsVX z7ay&Rx)sOSu)A9yy|8F4w9MY>riuG^QV4J@GYzF_S_N~!ekuv!l0%j)Q;G-@kXXBA zdvx+LAd3qI-ffbeWT6#h1 zuU;r(>k(N6prj=>EM}Wv{<*WiS<;HENvvfG^Y**9(6owjV@K0g;ePs;%Nz&6u9x>z zRMKoB{EtS~YZv=nUA_C8hZXhI3_9Mf{^@qvaEyyiJK^!X#K4e3?k*|nvJxeCu|e=|NN{Lv|WGttMC zc_8n>W8CX~+Ie`A+IzsXLamBW25L}FB?}{z%#>4)dqneS zZ46vj^Jb*RtvuWEI_tt!$k*KD1Zpkj0JQ0oc@sU8vE(V^)HjLv1S#mdH3$NzPMzKZ zW_Ky#zCQ4Yc!PgV!aki>_+fjZqqZ^#eJuydqfKfqBHz7bk`B7c&IRH%Xx;xTw7ps} zMFMv9XYWI%XPokEixwjn?9gIw-lcDaU|RdY%&`itJ#N2RU@E)v1oSH9F{&Z69%P*K zEZFfx_+HDJ{eNtIRa6{MvMuhe!99fF!JXjl4vo798g~f6-2%Z~8+UgP?(XjHuV?O? z`)2Nazuh0bR-Myzs&?(Y3u2jSTq{wv=fz>a(6!IH#r-4Bjyw(OdG$5Prw7}Ya|zuJ zerP6fR9?T&)Q#eYXnl|1?2T;dCujKhD9x(?8hesB-@=t(a` z@FOc$Js2hfR{1lLv*paASGecp^u9M-(ZSgjrSwWDOfSbg@)Wk5Y^7KY2r-T~*Y%2g zcowN(KZXDw9c0y0`6D0i!~5UJru z1fBwcp$g>9CzmI!L}%5-tMOgLWq)pGj{Wo6%@(211vyO?b+1fx6+N$V_?a15R(Y?u zcBKX4i|)FyZ*nt$o~L#4b-}ra#8WspfaZDAh#@0MmBd*ghnjDltvrNn?NrzfcB5UT6(?5%$u10)4thbjOAaP?`|68+rl|I6rrA%= ze=OS@P~;nHs_b`a;Ne}#KWa@Pt62!ra{sp;LT3pzs36NJ+R zpwaTX(ysQwo;c-~C9>f32|?NM?#$n9rTW z7kg%f|7=+XyHQwz%PIk>FOhm0YLZ-Fjnp(J%L1*>Uqxb2b>U|@7Clj!0oFv1AaXA@ZCx5}z^KUm z%;bP~?QNLQ)aOgcKpN8rkg@eVE*zZ|DR$lMbjYKeZZ+pZu!tKOyChNU1uEtDnIZn7 zD~WDI*>up%G`+}uX~vQ2A+|5#{b#~b@Yr6$xA8GwFyO^rGQBw&{rtI!&fh9kll8l9 z7ip0gvT)WDc9V2NT$`n)UFWX_S7b-e`vD0-l(^-Ix9lLHVFO!+F7`Fp`JL85FBTqI zM^kJ~rUF{C)jsz%+(Jxu`bK!GC6{6}m};jNb-4o9n#@A9;a^Cn4RDv{_UJ0LWzFj+ zEA*`Ty3wXSu!RDQpzF8{ZUG z7jhc5R;&-?oxu0uas~$m}V;I@=M=h|snDV3(1cP=`pJPn6 zw`mB|?&<^C=1s+Gp1xM@%=xTIX{Ut5VWO90+kE&M$`T7 zpo2_c+nyq*%O%ohhdrrwy1;tfNX**2pnb>nzJ$7j_8|%V^u#DAML+e0o9H2>dC3L{ zW+>q8tKHBRI~?4TOY6pPs|q-uO^LSbyTcb+=nu?-EiK2a1|%yWsXsYvMb);yVpQ@> z1y~W_v(nrk;hkFcko|#vos`RzX$4_4Zb^@gwfV!gS~tBPC$xu7VKyxXa!nwk*WY4u zj>&cUTWLSJn>E6E=|KPPXeN<7^Z8vm+>0^}J0>F(gr!OLO~F0Ruq`YRy~X|S(cm(x z09(vTp&5QHKACfqwSCnA@kHKG43CYXk%T8CS?D)sXK01xmnw2V$WDsFj;XKq5Ff{G z#dE5#<}uak%-!ExC??u~OuWP-KsWRInb4HskFKrzZWFgI8#_rAQykF3Lf5ei+ocS7 z{-``eISI{P!S8Qo{GP<}hc4{hCpFcQC+T%OT5YD7UW3c&`gLAGmTq@lCzc@#M&d=S z@Mok##OF-z>3Fw`yqooWQ%6UlAw@)_lE~hFHKKL-eNYBUHxJeVve0D|djmjY;l_t{-N!d3 z&fQa+L&uNbYcwc(Sme&n_&V4%rO0O4zEgUp+TYjhYO-5&1MnYt=>DyO=b3{ z`a`VInreXaNSMGGWSdJ3zSTaV+DnkTi6l+thDy|(rYEbw32~-DR>bPBe-0yUnJk_W zxA7IVy^vJvJj(HTVQh^hNQ~t(q@J?>xHM6};HZ2<6ZI-b+&y~U|BJLgu(mNp+vKv4 z)vYE^5i|ibm>A(6eHiOmbCL77oF4cJPWnhy>B4}*R&pdXWOIksVP;7E<&`MyBh9K% ze!vuDw&asvQfqkGEtFe9A#)uTWGwpi#rME0Fjk7La&f%)zc3L|8}h)%)5FW?@O5%8 z?s$ty8ZQV7UeDN-{oZPGF0|Q|s~-UTcdeO#H}#6gO$jQHOKYt2>d$y5s^gL5JdiQR zyeYGXVRqRqP~Eff>yF#bQQFgT=hpKTI~?Fo+PtBM0+zXhbQiS=b-boz50-5J-=T=Q z8;$Zb&?>`Vm%fViZ87pD;aNoZAA*=69(b|_Jp$au^a7pj$c*j&gFS1T6pB&Zt5%kB1y zYfMd|>lF%((BfwOzqxq`^eu??$5VfWP*SqQx9LiVksF)BNbuicWVO9bPd85P9hr3~ z=9zP3Mn#LaR>GsD-~#4#ULPfex{48$AC`(On=&fgf1Jy;)tsW%y#!WOKb)ly0lk$B z4^m9@T4H|f0CCgHUQ;`?`@#m3EMa$zba3J&i|f$q4y0tB1ErRz`8C3FgV5IVpwIM& zOD-DG)h=qd5Vn`8*AKW&u=Uj!(U-kU$v#_+#95Z_PFxPx#P zRtmoqvK)(=Bbt`EvHVk=*6UMZUPsB@9oV73D7a;nI_h$6=B5s`e`YM+%& zogw&_Qs_3+0%p;dmzd&x^6TvxZ}M`PLz@dvK2iuEIf0|FbeLmNharTIhif!+TWztXO+ zZj4eal3YPEN#F4BqsHgSS;%4Y%C}AOqG?=U_P~ zG%EBU#;VN}KV6x~<7SDQUuAnJ|Q3sabi;b%b^3$#_ z!d-jjPB0ufYvP&kf;(_RJk|fX{YWiulEYpX%C`@Wvr{AoRaCuEs33k1z zZP+r@CyhQxL5Bdr)cKk#E>L{iS+K7xXVeT!bwAtDku7j<1kTv z3gTQb3J%d2?@C2}!VJK~#MHW7h8mOhf~1l_G<4Z*B<;RRI4~NUHGYAkT0O^GY&+93 z85@WgT^9*yO`(GVCE;x@d^|g|>zo)pxKKVEhCE7$!PIbdOmvlw&IRX53iSkjEM7pF;N^PZ8_DNu|5B+0*;5-IHn04f zjsnF)4}%0B@(C?4-yN;*lgNt<^C`kr^S~!QOtZqEYFk_EQ2gXmS%xz+lMwE{(N)Cd zhdSESmY)xfV706J;8t zvxFxxg-l0PQAU6$p@TmzUAOf^9sp+QQ!HCb)(!W!)#@2YawZEavvXX z+(;KGFDZx-^m#wuNi@tJnzrD2FiJfDH}S7Fr-}1W06Bu*>Any7_$jSHzl%#ukCUXJ zNlIzBo0mYfjJ6WsnVgiq!D#kIkb7>xr0#E(UoJ+DEU2(cca{xVu3|(|<>f*(b6_Dw z-|@0=gJ9Y@J{#l)+Pgdj%kOje+^8^CAS*8H5%03t&F8Q1;H|^KbYD>NW0=JLNKXLe zSKQM-sp1Fai1CRMH+p{@S*?$gS6a46h-Bf35%4_{4$L_G0l|t%lJ83%nfHBLgHcwW zbbuW19#~);5q>bk&#i5ZHMBO4_2TeQAX;G&*Zl=6n^Km267=S`WkL9M14nvM)cIm{ zlZr=T{hB@S#=j4`&-tH7=ViYox$)a@nbsdKTpPv2+HxNKU-ivqT0XfrHcq2e=eO#= zBM4)iM+4ej1b0_!OeW;%K~1afAG>IJUxp>rhgH;)D*9RnVBJt#Iy8+Tr9F+H$5QAo z^RhnR{N9+S+>Z(3zr0dY3%%`qD%Z=n>M!?IszhsJyVy{^9geVUyN0iu2Mv~#T~N^6 zJcRl^Y_PcYcN)ufKuXi+UfyU6b#KVOWj#ut#^mtDhUvef#Pb8}w5n`z!9$a)08V8f z2CC3cFP?mE#Jw4dfn2#hI)V8w$I{F^$_liBUQZD8_sMxP6QZ=)svX?iA^gXEy&dJU zt8v$==HU_ScVtk3B1PHGUU`R-^C>%{?>t%P3qL65X9G1K5Gu8=WJ^X@W?8CTu7 zvZfU_vFa7s=gH+8(a8?*x7-7IM(><2?FC8>cm8IAuo#?sc1{Bar~0!VUu-oV0bHAw zoxeKY4HR>ZlX00+Vg&TUK9bN{YT}xRztGe7^bIFW9wGMZp9jrYe|MU_PzHG6l937d ze}qbll@*}_*6lGqY?jm2Y}YIUSJq%`UKn4OvfeQEqb}Gggo|2<>_dV}(0B#x3-P#} zZm9>N!UET5+H7eI%DEFG_A3Lk(e>f6+VHc>7OcK!)e_S(XZ276euRqnqva6BR)6No zmzq>RWnHGHaL}^C7V*+O|6UN44Sj3*fp#jpY(m-q*%mXb@Y5D0I9Kk}u7EMlrGL?s ztC;M&pYYoaaqaN_V;}FBDcjFRCNdZa8TRpY3^~@O#}stI^MB0hu+5GRPSo?kJgMR*nKo z8VxyDi3_rIn4DVC()!sucS+o9L}JGs_F_Z=pDeD8PymY46r)`NEZMSFHvPnXuRqU< z?)g7#r;D~*2!k2b|Em6Vam;*WB$zzVszo=+zdg~K&HI){JDw_b-1?yt!e2l5Dfqnj zT&v&Zfaj+#U;g?LS0tKe9`M5@;Q+mtsqT_MpCX;>hpN76+K(l<@vNVqeXpIOER1ocnWQc zANzQasb0nJSW1o+?qu@Z#_FNOeZeK=h0-CaZ{d&9jIGDon9_814k@9Nr`kds~H$$hmby4&V`_B13Q9ag%F}@CW zOF@n{jpS+!<~QssdL3=427=zu=P3jwJ+;g>lSqXNev&%E3yC2s?dadQw_*IISh)@Y zTB7ifA-#G%ZVLHy(DrmQW_}y&3Dw zDanJ@s7&eB#P@1%5w(dwaq24J-i?6J7u=PFHTw|js)Nx#^0gMtDnp^YQ8cR2hiq9l z7n)gDITvWw?0HD9$tp;L*P>UA7OS*R~27`S#v^n>^6$Z(p%|rJ!6>_6aNF_p0ngoWZ zt+ac3X2rH#YUwY>@b>Va1ErnOX2;a+%;Sp`1DPhw40-137A?wgCb{WklXsilBbjti z`@?Hl=k*%O>uuKHrbAXV0kzP}!NKFhmcGGqQx6(P+W{gS;p76hK}cr~gfvZ+G_TUU z@<8wC7si!eC+Dtb#d6w$di-sEG?#>e_-15VW={57E~XuXrT^45|99~THV*LNz^L{< z`OL7E9!2v6oYt!mMe=(!gac_)B%2+{ zVsb0?0TOEQSN+J7e!s4j7R~eJnIwpX2kOJLqD$ag!6ne=I92N2oo3QyFAD~E2VJ#g zel^Hlb0TkAnGvvuJZNY>!WQaxMG)z8MDS`ozp;hS_Rh7P{EEi+*Fj9uKo`N0Myd(^ z@TAYClCgj6wpMDIi~PIhM0~M>aBdFb)w2ZkVOU-zCSnVmqe%A5zqkN>HYL%as`|Md znKa)G&$~Bt zeEw#ajZ4E@NekOrRw`XP(M->0nu(_GdkZ#np~BVpcV-%mDzrPwgeby%A8 z0maMXe3M&8-#xT3Nk7heLkVw)Vna+SYjNSmc?)4JAMU%iu6p*Xdyi}Laz=l3zK$AX zUzrm@WvFxeaKnG>e7vG9v^-CqpPR%c`m0>$(~x&rICm{926lc#bc%@kKred>96EW4 zoFGC20BGFO2&Slzps$y_w|5y_NzT=+mtSmr)-Tj09 z#;Wq`30k9Dl3mZv$o-a?XhD*R5;d9q#pQpqzg^>gGoh+KJpP0$ykgaH0raT=YHF=1bjyf=#G zMCpk*K2lUFhr@!jn^;G3A9HMUF{xB=-R=TqKn?6WgYeK*lBso#ii%0 zVQcdW8{CG7Jf68AES~&6NqSVP_eABi{noox)4N~OYuvcC;rBr-5&6uS85P+RM)h(z zjvhko+8Swx2nbLstT8lLc&KJUwKw$AHB`t zXAX#cZ2qXH*ktG~kR>yGf0maT&-W2@|3GW>OMG%`?X;R0ZuzEmk4@UCnvpZ2I!`7% z@~rr)o~$=Xxx-TSEO?x-*Rgw#^&tK%v#)PKt4= zA^_O)W9%^AQZmekVK)1uFZmDqhoc(#Cc>T*y3yXzo|zhZ_l+$pGd{EQ<&dI~(FBwK zY4q?9CAum!=6va$W#<(dFQ^mIrXNWQ#3;E^UtrnV08?QWp{!ikpU!b%Xp)K=@qlYI z+(Fsa!A!PNsYYoxZk+ygWFTtWtV#bCZhd-UaDNm(PFX8zjwX3!ltciKcR9-N@QT!S~N1ph0HN=ij{sB%oi zWUeELM5HH_m7i8fXh=s4kF2IiAD84#{{o%CJbHb~+s~AcqP?b#M?sa3Do;aFw+vsA zF$2~PEyXV%TUD2T@>o;$U!n$C576t*r=$QKq*NVOF#4~(M4zrM5z?F8!PQ&@m^Cy2 z4GYLn;L`&sOj2ls#E^D_Q|_`u1%z7~9<3K3y*c{$FT4S+%?(r%ntEW0a zysFvrv5EoeEVOfO|F+A-EvK~_)9>TJFPg}O!}7)@Z)OcSr3@Z?KlLT?gL-o1x%|G$ z+9BG~RBF;3ma3q7E9+PG&}%F4#r_3cwuI_u(f(Td zaQYbbOxHM7v@%%q{c>Ma&MdkVg97#=5ceIj4_=7~A-ia#odylXp6U6$(Vkm&?mh;I z^m}sq;fx2JO8FCooD?E8o9MFuK@$wV1ll6P|z8q|1QS7r7`}N-utO}mKKYf1FFYgY0 ze=*5Uzro$RNd|4=>mW=cD|#Y&I{1BU?`l6f(YNWmo7Z)}8Rv3XvHilL5}5i;Z9dMU zM%9d9k8Y;7zxQMfq-aJqtDvn(fXKVKjaQsR?E30N;YuFD!)s^x0{@tsF|q4e29??!%!vLdH&2q+p$=o7r;d zv_8t2r(~ZDZ;N>O!Fhq?M8*P6b4#194J`NvQ$Zosmfb3g0)jw~RmbU~cP2je7NG4a z13%zQ0cs%~B^obT)3|4M_>zaAn29fBL*T9RG6Rgzv^W}4!k)j! zdwrbvz-u17+5Evfbyj8UJizyOK7`5bdd$F(Y5xzE&aM`dzD^5L1_gE?2GG0{-49se^H_~PbvaJQCE_?>&!6abn&{XtVn@G`hoVn5tF}F za##gp+~j2AE(feFD}kFyfM9ns3rKyB4`R8LQkJbVFaxy_i+;wS>rbkcR);on?;ch; zbLoq1_6c(?I7Q`Z`##?~XT#IoU6hqVGW+x5=q(02_nkcZf7fiRjSx_$mto$Vk|gcs z>bN2y=5vXDQngcn*RzMOdoIUl_lMBSu;&T_iSOb5<~BnL0tqz06;gBixZ_Ji;jC6t zeVq7-T2-I|NYPjA`Sv?v!x@RzEgTaqA)x@M+-{Cs|A6AbPuu5^2?>eZnT1|>7ZL$i z!Z?$Gt}}a)r8$0n&gV+UZ;d#ONJy7EgB?+-Md-IdAW0|L;Ks5Uw2+`aNT^Wh^NWNh zW~B1hj{*ZI-7sCv>k~9MKm?O5^yW*cy^70vDcpPKq(4%NZ*@u>Kj7k)wp#{@uan8o zY5eQqvrF^#G4KBvI9cOa+ zy4vhLH2+#KYBUcrKVJ`x!*u8TCgBL;ZJC^!kNkMwct-(?lKjVKqDHp8U!PNplvmEd zSVoM1ik0^>Lwkfm|9#VM-T4FSU+rZC6B0DrLr47FtFGJtKoy^lBTwsA{mN#wTi4d< zE+;^N3>w5)RFnQeykZnIF%Os3i}Yu(TO{lKfww&Sn_$z(7D_PM z;&8;Qon+By7@7k_#<{(F2A+F{2GHHpkb)O71z*1i)mM_pn-kfq!nL{8gE-He(HA?z zpVI(?go#-d7XubT6>rhbyI3=(7LvI@p_^V-qw5&$+hExys!I(#{a~s{7g~y!zl8q_ z`NX`hz9ug4&G@^fG&74ZtMDxpObZ@hk#zaSt(AG!^Pi4y>x4?*7bbM`to{Di%<-S! zZgx zbuX*LbC%OlVVaoYaP_3Sp$a`e`E6GE;Ra>>e_wbig zFZl2?hsteA&MFgnXau#CwQ2xAknkAFas$a#aEbbH`KG-vgJA(Blv0&Q)GG0lxUp%% zBI-KF)D-PwV-;)((o!8C2Lf*P-v~bEARz8#9Png%O$Z#UvSsn1Tzjm0OpK7x8j1J0 zm%sj?)DV+{%vO_{X7e&M8>%K7sQY1}bj~Pwu&^$XNl)YqCPvd!np1L`BkLdDk|sK- z{+T8;HMJm?ICywQTH`BrK-aUX4js=OOE<$SL&aOc)vEDM)nYTW*AuaxixNjJ+zF7( z`Z;5m#;*Q83ViXfLN?0t!J(ryNX5-vJi;+{^VBmb4E>kNF9*$ z73K>G@|MokjJGQ2$&Zkkvou$}a+Px2|FBZ#-5INdS55pEs7ZKX%s1^cu&{~Tija_&1mn>4m_TYmZB3A+Ce(S;*?* z1;7F+&1dW$=FeR)$KepN3oVor4`xtu{)jA);E#SoU>icok>fyO9_bK7$tZ(5E0q%I z?c!-TM|iD3-jN?kl$PW1F~cuxtbFHR-Y=%`9Vcean4OMv0?jbllz+FvmpY>Zt(b8n ziJf8`dao}LMSjIk!T=%CIt!iI2svk|D8ow0jqj<;(>oym`d(mWS!8!8b4Z}rQIE$A ztHqvT>`0V$Bmq*K-A2UU3v?_(%iXFD?zV%XuT$mJ3PJ?z#2X)vd0GoYf`H(}`Xz?# zq8p=I=a;HqRr4QY6c&v0a8>iIkc|2#`ZUiRv*Be81k)}CwlIG)fbzW3>zN>+0N3@I zX^GXy?D2l(Id9lPmrcN6x3luO1{)OTBVkt7tDJRE7##rJNnxSX@AN4WaTXFX8RPXNyJYwt*Zr%6Rs;TK@a1OCg#zoRK;3n%l@)^JMw&Yzp?kuqKR?jC(Y+q} zv(u+h5@pc$K>3O#lUGNbOz20143AQUX?FIFx?=Td2Xk}XP@GK$ z@*-sr6fR>bpdX$a8%AO)*-}N%)>;Zup@9=zFi+(6{YA@#wR}aRbcVdi&kIf9RY?Bh zK`!!eWu5Ga@b&SFLm5o^(BSq(hscdZmT=x+)-NEG7K;UBn&2%&TR-fiqWfi|1T*=< zcbVCM*IR+Zd7o@4W)`3&8S$!Q9H_3fC63tNb1D_1n3nq0t*iC%$J0a8^T3H=z0S|y z^}hg+5qC%wmW=by#eQp9aeJ4U>Yf3z0@ECcyVTNzdQfJz#jG6l)xZjpIGq_bgpPq} zj>nhqiB#b=Ov57`x6^UHKYx^#Z8>X?;wz5?{*?S-dg2;&i2e6SZbl;fOayf@_Y0eK zXsEVn{I#&3+*4$})Fu6VA*4?MxbSI#KJOH!1ju(2mV%^Pe1-g=+NOOU^E19AH}ull ziGg77Y6BaL^Jy(G)(Od8(u4jx8#&iAj-zx-OzQA^D||ueWc9&*#E%O@_Od9o%(p z8L^FnGgyilmMRAn5|^VKRfG1 z%e|qFXPcEL=2VQ~K-9zvoGWWQ>q_Ysrt}vM?-Bh(sm^S%Wq&M9oVI4HDi1relcfHB$Y|fys_mkP;u~%scH|sFs$%s2r zF*N+SR7xmP{(P`i_WiuUdCY`-g2JlJEym@gpG$_DgOeSHK8PA zW&U_~fwhIyThz=O5$}s`n_cJsPXGV-(=O8(deklWIf6z_+6OkP*G;ap97+da#uNwD zC;s?8!)CrXve;X_@}K*al8|~&^1+v<#Nf-~y_d*gfy< zgc}c^Y8szdk?)r|<8l`!#*vQx@j%meGUJJkibI}!^zu8~Dy`ngsCuz?O``HULh{q) z57CC&N0dAObDPvcSLjsNPHFr&PWId!?WjCe8i!Km^kV7Tq3`TOn?r$Z!H)owvgkaI zG1hM9%;VKnf7u}oZOrMU{*z&E#eUXVmt;`=3qIaP6ymHHIMDJy>x1RsFQgM<`R|!` zADPwlH^4`xf4j7XQ1$X%-UC_psD^(^@L0AGk_M~RnLSYOIYps4|eb4a>8{RR(YMwRXMf4`ps41ff z`&C-AKh)Liig4^=Y(l%`*SqROgPX+H~IJKr->gS-T6r7TT*^0 zore6DbwgB)w3uOP+szkq`=t=Pi`#A=#kziIdFFJf1>q?|_cqhvX=_0enn(K)$H#oT zOI#*Vt92-xB1JqpSgEEK*E_N9_>ZSl0>rXzeV4ApEq|P0h4!)huD( zFH#ZxFBk2>b+>A5 zw>`DLdH0AI^hs%)+Tz&@fRH{RQKDm+V(F7>?zMK#-VbMf7SBDXq5 zs5THbZ$y)?_2&coy4|DY_pdz(pu(Y9weV*(F?MS+eRWkmYo#v>r3$%8rs;6Ol*%q{ z;)D?LbHd_tILcDyNF^mlv2TAY)eBKr_Ep$%YfAmCRl7=oprLmujra!pFZ+7$Qre9x zXA^Ha&4NGB$1#Enn+npx6`suKRS>^S0$W0;*DIIw=>TDnl3i(85o{&eEgCQIS$H=k#X_^27!!C%o*^F>I)NkmX05rH=AB*JAo@fgTXD8 zJjpiyk+Mcp{P(U%^Bh6Q1t{6Ze^=@TzbJXMg-op{TKjoi?`*km1wlT_?i>UI?SPO{ z3zwIdUwHt~!So~xI7iQC`4p4LegU+!P@c~WU&63AU+<7{`p`x0=QE`?E=jTS)_Zb7 z(H2`b&1|2~&E!a5`3X1XFfe4;AA7an61|kpT{r)Bu0utP!$q7~w_P6-Et1?|e|dI! ziYH@rvxgxRm?vgpw)3E&5@JLn>v-UVyFCeG&HH71{<;LNT%pVFKtcT)6H;W3d#Isn z&M?Se!=^=M35dzp?UY0Y$DW=3va{s~sRgVLTp1fjR>_v*QkX_P1c0ls%Bo|jev=xP zX$Ex}3?Lq{MR>fKlchIQwnS@A8@nS0$QlR$9lf_ z$4!ugfMAjK^7lSM;vXQf-q?#03IBW>-YTiLeRM%{P$g^@{9oyXF;+GB&QcNijx721LgG$DA#XV>#uPXzeZ0k35Slk=$U_g*|t}69w`tKNBA}-OqBB(sQ6Q-gQ zbfH)B>jstcd8LA9sRb3eS5FqCY0jPwKhFC%ek@ky_A2Lo+Qqg*lJd+eUf=XZyoe6l z03HMO-tvhwqv>q(N5xLgrjvPk$k=5R-y|cg3-lj$KJ?r7XSkT9I7Kad=0t@0UeMRX z)G|@v-BSXT=ysLpyg31Lf$u@Tve(Kg^ZHKudc@hP9rupE9znvPlSD4AwO-Eu0c?E% zLQ?9|OY82a?=yDWoEd=s*kU%#oxc|pD?6$v0Rl3Z3Qh6VK5sFHe7CZuCwrmGSV8+| z%fhbsBAU)ys^LluJX=G|eT;t1s1?f%11(WRhB>n+w=u$hU^&AP8gwws>$5Zat;Jv> zY&ggC>kwX|sk5zWHMsS&5u{7$2>Hd)+NMAx@iAnQaf4L(^oR=K)&+;N8)zg0e1LVa zuyoX{M4CSOu=&h(!S2l*Xz@Nq#05$R0^)Pyrh0GpJ1A~`Qf>i3cO3ZeJ*ZeMjay^x zTPJ1H@N}B3(w3j@9KqF?B{sX^1uqCCvOE^_l&0D(nKCG~K!J7#$n$;}>mYEuxty(c zBOnqsxY^Kk(`Ayzqht<88q#8Osa31!=p}#9y-?a};c~VwP=c08e@RHj_Wznve?+}Z z7>2kKq`y|u${2ghadb$BAg>dt3L(77%mFEkzcB1o1u|7h<_6>YpU?R?lv zKu6#%0DKr#RA$IezEWpU0&s^)BYmNtH&ZHzPy^^j9ryP0d`=pMm>9%zbVEjWf>oIZ zxMmJax+de4fCQxSA|u_R?pwnLvFcjXVszqv38&Qqa;+clCXIk+*LeI+OwYd8(L>!f z`ED#?%y#zXsI5rtlCSV+jrgP-_-sW!!T2lff&Aj#ql{@ekmBc|i5Q%E9^{n> zw!(owKe1xE?E(%02HAgPucGjybx0g^n~Vp;Ho|AZPY}2>y8r!0&z&=m#Xqlyyzn2@ zPzqPT4J9B?l|zXq2?&NCs?)#?HJZJR6CD_|V2RtOmX_I0$W^j(X6QGVVXaqYZ7j%T zD5n-e6(D@xaW5Q6*hf;UW?%okBWLk#!MS0Fyy1SNt78$NukWHe`H|W-OtCNd@ncb=9zf&lnTcf!Vv z2Xx@G8?7my^Hk{^(a`vG&R6~77LB+d^|6AX!;Ma^2HPhBqymUov!Ui-%Y|!PQ(U~< zp_5;u@^lbDsbGb+^>^Gz(m7-!N=5rmJq&*S{W@fL9}_r7Wd1=RXd*G00+CWWx&n)! zHaMcj^S-f?nakevqszmm*fq4sELQyXkeJ{ZRZ5u4YEiGL(^SdWlQFS?^cal_9mDR` z=I!=Y7ermI*U)D?HGR+-Vml$NdiZKtN{R|^v{-gVNwFxEWA>!`q6=<2Lr z3O%?09497P+(`ekJZ9VgJD)QEqkmxXpRaLDmq#2(ol;j+yakD(tUD1zgzfx8H^JNz zI3?Ml?@N}g+PQZWIukA|IEnZ)rc0 zXKm{#{zp~0WA^>c)9B@XT!}+kN|TPZQ>2BrjkIZ%+FJcE9IjF&6pI7XsT*_E>p>_k z{n($O^^M2SEY3lF-UUu?gXZLI*AIMWHos$eo*+`ZOEHouUhdwKliO5{c1OSTPf z0kYE7kz|(P*6NbdEkAc6F}q(0&;En~HFdQvy;!QXWKV!y(^0Zl+;5%GZ*20JNgxASTXjq(smhvQ2U&?WkrlsOTMTlTEUHIxgEbsJh@T zlz<)RziqI|E=0UZUqAobYv*{5nC^EZbAa60Zzl4?L>F=-p@}`Kvs0I zoKvfpq3I#O>uQ!wvDdmRGQC=5_v@Z(y_Oie)$jhVoIJJjwK-G0=()}h=U@blFz6SG zL~44d5t&O@x<|yOW6I770UN8^J8^Xh?w+p8VF4vcclE}7uGfL))yfuC)A{pB-)AeF z5^K~lc$@Yz!-@q6Jl#*{R@WFR&Q4SNdtB8%CJgKq2uwV>QVY#Dc*0XShxW8>I(Rsm zpY?68zU`0e`eSjBAa?d|9$D~_vvs1xtDJ7+%W_&{Qpo+I!8$@RPX?vjxDj?;su#|iYB%U$epa7GH>9+v~r{TrY>Sy zYP%bRtCHkYPZ&(8MYpUzG>^e!q8Z*GbJW5bP;k=!bj6dQ4lk=IgRTVd3`hf|9wCeg zn%mW>dO~Vbg=y8YGiV?w?eijq14&qbRf!?a^MmcvZUel21Dj_QCya}|Qo?_d0)hgjqbO%S|S6B@f zSrYfj2B7^);Ui+3G`k5U0_Q(sU8`BM;+;?v^m4^geSH@u5GrI zWB+d!dL=bLGAP{1-g@ zF8~w*n_B=X`Ah5mhEy4Z@{y~U1vSOjH*U%qC(P8epyIln+;PhZVN)GW$}{ayg;B0P z8}VoVF4UUCM;sy`BCCy9i@jWSR2|p@8(nazdc<4@U2ux<7M;!)P!5A~)Z(WJW4yCd z%^`@X`=BJKp_-dkPp3S-Y>Td*wK^CJN}QPsou2)ID;(Me5^0e^3EMKl|7F+1hT8@Kt z=g1H6vKgfpaz$`FfKyq%@14&FQJ?sMEv&}6$6qZvhdvBa`Pu?R{_akRx}39;C@z!cuPo=hsqsoSH4xFjE_@Xz0|j$m~R z*IzGkj(LIOkXdPvb4dXy%N+rzE8E#EH`zl=mMh10wo8xnz&UvFxE$+nq3I-FL9#U6 zFUhNgf#7TT+}?N6frD5DY{$v^+DZHG2^aFS;}up_EHNNe4ztYTP~brqm~lyvJEM9v zT+VaIYxGUSEI;R=P5O}Y2X*dr!j$u0Js^n?WZBO|7bCM8N~c=Fsfs!eJlr{ECmW2) zB6`4|8eqG-kbd1Y#1Wz7GzX;aLo7RLa>=agHJmo>12U;aI=CsPx-gHLo^!#eKEcZ! zKdH=H>lBi>-Dk91Iydv?v!PsWyZXx8!3F_)?&N*5#xQ2U4IdBICF@;UE;-_vZ=Saj zQPBiD)p};rTL%Nj*$c{;x+|m?tBeGGp%H`^wH)0pD!??ks|iWt{W=a=ZQAM^oOQq$ zhujGv%WxaXZ^2(NCzvYq#O&u_H>4j#+}HJ`Mqcfx#+QF_46MnW&oz@7>u?2v}MNnEnuP-1OW!)ZIey)z*Wene`0ht}XKN0Q^p z7wAZof-$ra@^yiGEsE)cmb>O(XdTV*t8t>7Hea2uk&=W<4zxQuWN2dPWwRKZUmy+62=6 zpMb`otR<4z(&G+)`CTYQ=v1f+X!=|7jF$xBsT}USqoyF8ggQZ#Tpbr^0My|UF>yks zRv7{RTQZ2O6w#)U^!EY_ezvB;@f-9){UhZgz`RcF;!fzS@8~ojGVaB8nE6M(7_sX- zeBIHvmCEgli?>elLv$p+JkC+qx)?xd4#>qw!#@m$UxahTkTrp9=`%5?104<2PL$p1 z8eMWM2_CD8bC@l+TzeMSqI;2*22q$`kSR#-yRL2|UvGZl)L<;N8V8eGlupPx>-G7Z zp5C9f!_k$pPnjItcwhvIg;3y{e9{^lVVcNCmsc_$gng?+;zTi~_k8+rjIB!VwqXMH!EBsvxK#}OchJctz4x3 z5_f#-1oO|9|2&0qr9R6N`~q$$&LXYc7*!TYZkM98E%p*eMwhI#MEd~RqS?fP$*#}$ zci$!(Xhra*9U}F=QleZE(ds1r9_&Uo7wmzOv9N0F5LJfjo$)F+-Y7eeZUwzw=xMG#G|cG!!4#Om%R( zMkk3x{E-sNC~kWM=i-KM+XH6zVEK@z{rWT}Qw%16q5L?ndkIX}krm(UNYxz*^Z2V< zjv>N`>XmsSfOr<}`LTY2KJ4sknIaibvWp6^dAt&5GYh*htI_pD|eB`VN zYQ`BpwoV!z|Mk^WtGAOi!Jd?l(gma6`{i>h=QS`qx`N~}{v_}ZjKwPerrD@z-V$%f zvZ2U3d*Qx9&rN@jsOYUeIa94oiYf(_tnDaT)0Gp~_kMOHS0cOam9ELN^SB_} zpbq4@CckxNlX$V0XER!N3QUM|bf>l%sGy+bALSnnh<`P$jbD854{2%e=|T8w z(Y{v0+6WqU&0r16s$HxPCxN)%P9C@}2UEzWTD0htA3XVogQn2}d}Kb@PMSVre`J;x z)f*c(6q?b|c@Zlm$krz$mUDQWC_X5E1IyuDrNGhRr8JAuV3^KBmp{QFukI!RFe=y- z1XBvZlAL;BHN?cOerb}^W-~gYV3_m1fTkbJsWUi+OPc&DnQcwDr6Z-Cg`(jw(!e;F zpyN=fpOzj;-znvsX|ofVhLxsB?5R;{0n2i|Y}1&$kV2zW?|ZrRMbSLqAkhE`f3u0c}sBxH2Iypi@n0>AdI39W^zA%u2Nclxr*a0 z$nP>5G_U;_dC3)&4NukkKC0u|Hk49O;b57WSA(zKu>Rwn=eKVo7EXaCii_ri3J==e zA=3r)QIKD_Q@DyRsV*a)E_PMH(kMWRyn@gObo&wvyaF=6_P$%+yiA^?J2sAU!Hm<6JDAac%gQv;`w~(r`*mh z36o5pY?)$A<9@+@j^BZxD|XSTKuoJr!5=t{RLed#2jD02_K@4GX@5Hh z{@Dj~yO%N{DFKb~W^^Sqw3FL(YLoILgMkx`^3G#Z3hs>5 z7S)P+S<#TGtE|#dgZMq}Glf%tSOdCAxN?V{cS=8zwG3WLt0u|ul>AiIbl0oEj|bS_ z4yIXhH*(J@F{O8A230+FULwYNy8|;kf7%GzIK#jnAPov@gsLD0zGrIf?(#cSG10aTPdbN=6<7Not68 zuHi$P?9&$G&r@v7RN~}_4_3*Hp<92_cW%T>Qx-6v_ybFA@6e@@&$)cV`5913uZLYz z=Ia76K8HA3rf;U$$nn{p>nMzOP3Y6x2IyWQG#z*XAC%N{cnHXz+SC(*krw;0Mq^Lo zPio1{2Xpi|y3VU#(b^>^jDe+Iw4p_i1%-w#sg@s(QZ59~X)IYZXcF zFuq7x+uRFVmy*d`6BtQ+{Ni{+!9#_oV$|~)28+SyUFx{~JP012)g%T~_p$GB7lp&o z{^PeH(9qBJm!-x|pAx4DP%ary5@GF0DHQ*tB&AB)Y0x+n$bVt_nnR?^?|=ISK< zb=M~E8N_5dh>}47)on*Zw+-)<+UAG!2517LzP_7Tk5XLj-FgEAQ zJCH+}`-s}c5eEL-HM+JVtlcX5|q0I2ENYh(-Cd!b(ry*>iYEHYR)sBMk7sXW@ z2v?&Kmmwfa1C*grQq5qZ(4~o#4w{z(r<+f&a8M4k=&!xYZzdDnN4`hKFQm4k?Yh0Z zppN_G>HZ<7Q%s`2^GlMYWmu0VpCVjQ zKu|SKjzHyo-~qxCX(zPM7P`3+i%W@(V#AboDI8kRY7%gMd~BAO+}~BpSyQ?eONW?V zM=DW3=(_`Gy=HQEML6S9Ss(oF)q7w>xxn@BEoAfxxH~Gb_CDpN)%VFdr)(F2_+{;2 zfrC6-j{OoaxJ6xPf>pgk4YHjjL(77;v+TbLT<MfwU4PJi$7FnHIQf z+j!oR>SiR-eMMY?S17kzRNq) zC{K+uRoUX+XN?{Em*8KF$(Es%dQW%fpP$J6$XGFUeRb;H_8BRx9kN;LMqyB#SMU;j zP6dmUJ+>Qq-(dYfl7Vf$WU}#ujQ(jE6~0Yc0Vm6ylWguf-_DjpPcIRkiEXEc4f=Oi zTuZzylq9se9(T_eC{5T-?mT@Y?=}6@m`$n!S1*xpDzyGGCAhuqekzhp;CUUsGTH{i zSst+3Moxba#3lwYw3}`4T(^T8_Pjy=bx(YNPM%GhL6ior)F2n7;7OM|#{5|XFh|sM zV^(v{Ou0OVTqk;Q&=)YSLgC>7Rl=oCg;rZB-KGIKA%H)d2qrf$apD|Ob0%T zjK>+XFBk1#EUCJlN+X z0WbfMzJ)?IpOAB>7U8};-r-}_56c?i>mM6UwI6mbk+mWHz9m7WFj9H&7f@InH_|p` z97tP3P+F{7XAK;NJ4%7*Zr4g7Crb_Qxonl=`6^Wv-?bu;&05cY|G5$TFUenS!|x(6 z3<4d#Ger>wxtjC`Rgm}oQ#KO!hGTBVN+0db+RCU^Xy!<#q^`kis2r9xnD!w|W`L&z zr%@VHX(12o8J7Q8H8dFu8>cRJx{x*OKZLKEgph{kWvCJjxZzu$()g2mG1vZlpGt`T z)Eok3>>-ledt+h^3V?M;K23T*jr_DHk_b;lHq?Z%ZX6CM{R0yTWv$N4!b7fK7Bc?H%Q)s*@yIOoPVo$uJXbXuJ_PKW#o1B{MsT4P~c zqw7S9U8b}%X(f!MSnOWl84p`q+aX)S&CShu;bLs%TyDg(0n&>d=4hcH{JC$R-CuZ- z;XT5Wy1VI4h{048M=sCcW~!~dwqdr}?Cs7%wZco0MZGVkp}0;+>SaZ%OcR$IR4Nn! z(3BR@62X7V&6t<()PkgL?cFEY!AG%ODjpLH4 zt@sh+>T$M?yv6C(^(r(C9judhRri9$Ir8b_?9v0F2L#ltX2fvXphPq|lm^ym zns3$e5*q|{sk$4@UL9F5x17l!+(Sov?D|5*12u7^9#TF;nZs<0buj`2-r%s+>6d!1 zUvx{&7o;^`Z)hoLTLz2j3@v`{vzhPNO9Y$23eE~K!1rErmH}+f69w}aFST{ zH7fMq%B&vrTTk_V#IA+0*)k7;rK4l~O#*&|-9%qc-}N?e1otzd=CNbI(yJG-iTEX}t`6)cXkDq;r*c?nXtS4Y)=Ed)(@dyAq z*w?W>@q2|0fDpp0E#9-zLv?O6V3~MnQPor=fJcUD!Y*}O!rD~%Jh`;`VL!)-R{ai6 zAH49wwSo5^l=Z*bb9IG2$x7H@(e!B2&+wT|o+z>t`m~fL40TR`M#`OIwa+pKPdBKA zi@^zEaS~tfEnhgIG3zMbstLOli5)(B5>+BV8Vtt0_0**B-^60kvTCh^;c7nLVATK> z^ccJ@XphlzyLUx#)zrKSW+}|`&naJ|>F@;U3Kb%+K$?ZYbNRzj{>kV(-zq%P9tnf0 z6ADCij3g&)YZ4cHuc|${4~>P;hnZ!ay9f^*W!v5@v{0ESr26j)I~(*OFPeIboZQ0T zI?yDoAam<+E_co0HtoswRi;Tz2jXeefVaeZ6W4LG6E&72oQ{9hTa6Pr-wO~e)I$*M zUe(k!DLIMhZU}FZ%J^`>ChqnHP!CbGW5w_8jM1P-$mO~aooYNo1RWd-h@@O z5|D;2(LD_hyQK^u5|Z3P@a3Q@X#t|2?ub3D4E%p__Hi`@nfqORR$b>OJhPwrpI*%- z2Ef$cmz0@g@H$KwuDoYvT1ICB3+_ka1HN#W4^Y-9xdXfEG#Fkd+JDM}8Mnx&gZOBy zn*eAx0M^gEDSd}atQjC)1Jwxanx*IR1UZ*WvBF(^iP=!>3{6_uCY z6OASHB;gaVF%v5j!CXp!IioXOfCN3hl~#I8b2-%K_6aG+>u(AY(9HV>Sqym(!`3DG z8ksmSs(qfLyoE3<=5_?`t#{y@*E0z02kSt&Hqm@LF|6wAd()D?9CC-#7NKbbg$*!eMj0FH^XYEEcZan zIU=bvRdi#i-?!^RS!)M(?=$_vH?xnMdrozIE0K_&tJX4vB><8<*#82HAbX;0Fg0)A z=w(@~{uCKyV2##diz;)P;_24x${P`GBuM$urN1zc3?i7;!(zKh`KybjgnWB`_s=4) zPe!JX)APzWIkSoJe^`Kgc7pKUtb_`d%O$7Dk&=si)XOYp3mwW*^3kEL1G9Rze)RHA z)XE34Z*)TJ)?1WaJnq7}bR8dxOooOH87Z|Yb0~+zks+SK(HGB!*zUfP>LVB#@a@wd z(m&+9*ytaTgtJhF9C^FWm~ruJwsv)MlQQ-r%QTuRS$E2JF1NPV&Ml^&75F#hhYw6m zFqAOTs`6_!1O>rmwVZ!aRueEj!=_Jl>`Mg&Z~FJ}o`dzJ_amECA@cFd zP#^5qK7qfFDThjA8|mNjF`3aK?2i_$iNBL1|54z%vm)yI2MNHMASB9lVni3}>?g`Z zO$B`yxc50Es!?P9rlt_NjO?!$ZM+6>Lnn)(dS{q>Jm5h1L;ehvQ0o4i%YE^dLz zaT`{XhmfMlNbTyk9ohC}JExa%YWHtjxdC3)qVmJmlY?ebfyf!8O zkl*ltF+P@P1!lC2f;jmMm z+l@%Ios*;Su5+ou25-S_EmTeJnLyyovu3Rw4>k%dYW7&NXJ=qFB!uKEQV0L0;z%nR zR(t1VtULpA_|}5Q$pz~QAn1Kf>O#S0eo|5%Mj{>Y&jIZfZ#^w)x{kuJD8Rfne3c=C z$+02#)eBQ88>Ch~IR>u(lu`K?o!+yIo4&tgTDXTeuX|_h2me1p_FpEDfC_q?QIR5v z*}|ofS`#@gr$aeLZ`tcLR0QD~e{v*2#jDz!D`2r|t zr^9xI61EQ80xZ0X2IYDB4&hmcCkOrXy-ESAguMv8Clno!nt34)po0tXc+FpiZ|rAV zX|U1H1|}Al)F&OoCKLrsi2E!%pz+7O+pmtYlQPraeP=0j>s7;Q@4HaNYh`Wkl z!W8;D7hO){-RouINOH=_C(xdR51HI=-P{J7ZMP|F@)gRKPznB887A*|SlSAE-6IfNJB$Qi~{}GgZaI91?=?8O?HI3tvsUU^q1_}!D zbBtFKm-Qo<85=nq42vNQEB2BQI^`S@V&cR-%XrKd&ILOD%ah+h>^?sg0cT?PIO%Tv zS=4lpki3Ovg#ET)Ljxrru2pE|D3-0~*AiaUi_IT#rUGAaq9#WtO6W+W;&5af?o&L$t_cGrhqKbEi`zY((r#-&iowUe?bbnq%uk)Km-Hv!(Q(K9 zlCHHpM%IZo_f?)gR1c|ZBM7gd;^LQXo^|GZ5>W^<`X1iY$2v?yr+6Vx=1WbdJwa3C zNz?MaN)t6vImo8LB15Shtpo5jdCF74n1-UJYFg!Fc}R5s()SPHC4#2Kh#B7_*U|s$ z{$5q3+J$e^M9oCNREL1*-1{*G;uSJq@@xMJ(Zb~)4aWZzUm&AU<_6j6@38BpA#OEG z-(!jMH?m)err}~T$9!v1y67ZIqH!Iu1m@S%s5L4@^14XL-#K;5>Gd$*Q%OZyYuFQ?bHX{gwg1BE+k!g!6LrOJlB&KOcPoO5*vh{< z+$B3$a#CUttNR+NkzvMr+ZL5*83UY$rl%qJ>M;_u)GC0U#ayV*@U6>jcr%NgsnzAz zyZPFI)9+Fe7WijfPlP2UjPx3fN%y~1R{+Q1^34HBjc&=WI_>6ZeW6J zzE|X@9@zWi_(iLiTm2os@y$6Ab#ATpES}iKWz22}>hcKQct@*l_qR|dd;}6lL;7!GFPCG#&x@x_qJkuhDEvY`pa5@(lFf=KJvsyhvAYt51Eu|i=d-Gs%#Oo6 zaZ*X?G2S>KkYy-8W-w^vuv$p2!%!S7tYu{(#K!FM& zAMzN27qn9&#yygyIy&3h71UCa_eC;WN{J@?Mq2v<) z^DUIP#}53S_6TbyR~#gs0@}Y|6ZayFCMA=%`3-xYd9OT`3M+>$z$O`gKE<%)po04I_!+EmX%=_vK)te>T=OzC<+*2`Ubz(N^GE=H6 zl{%M@#r3}#f2_q^&{ecCZz*`2x=c#-OqC~G0&mI*ueUNGH+z=6J`$gGH54X+o)XB0 zz25@JgMaOYeRJ<$|Ao_0wxP_muagc#+uI zdx{YBJXTi_<4kK+ziG8bDjvlZMYOyD61xH&{{<)5N%l1^pGj+1QAaraXn=jswHI{q zo1bXhHYg^;G?*wKoj6i&?G^Esl6>on=jFuAjef=fW+M;hP*OEzBhUAw`f?g$g&v3= zGF6U8*C%;q&JTxokcSj*vXP^-lt^``c_GF9^=*)$Wl) zw2#l%)3m`Js){%k~}eE0ENeI9?8T{mX` zF3rQiGzcrf#Vn?MwX>dQht{x#&SbR;M^pf|srLz)i{OV`g4BT7+n*3g?C1xY3JAW+ zdz@Bp=v;4DbH6W^T`*GKot%;$m(S-KI-!YyRtF}#&X6@99_YAtuxd^!hxC4#J?zHP zxnMkm6pZ1mA-g$FKLldgVMM?%Z}t4^u{2m>5j$fyAc!9sv6nW?tv@NSKcT z+&YnZX$6%j@)RE!MCls|Obw#s3VHc{{DEoa$SiX#ibx4x@d>63F#x{{nY<9dsSx@r z@FsNo?W+%PYG?lvOSR^ygP^LAs=}wt>Ti~0aW@tHuR|_Z5_c|XVPzoroj^K}*>G;P zp8h!|>xR|T$%9z*?OfhjBqgEkF-~&)XLpkB!Qk3Xpfd70{Wm6;KALY%Kw#Q;AiW03 zp<$%R_xYNd8emFpZaidyHIv5#KK##fY%bq$(?M}Ob!XREiNLDJ} zOh0yM|LNEtugettn=HYkbDIt7@aU7;&q%$AaIKI1ySkgnB=pIUbrgh>rbk5F@o;6x zGCqT)sX%)jpt+7W0(Hmx6vKBJ<@!7R^E)I5iqOXX93=jP*6}lV?HgY_*}ab7x|Vuy zu7v-h3moMP;tpJFDvJg5jKW!;M9I6uH^T!4Ar86xog_X3toA|k*K|ogBc5%2ntsJ> zr^;d92~1$KDdXO=aefFsvk7c3{Cy_U91EnN-p_bjYOvz4{gFxJhOMaot_?m9fgC|` zsm}`RaN&2cv=Iq!%2x;92A{9htM0Pr``^mxS4@a{^<(&FF#`MTI7a=)S$pZ9X9>~g zo$t(4iNaNo9Q3obBU(mC?yXOZoBcZpG*JSVvQrY>(m{jZZibWsQ5z?n(@#s0bWaHgDyGt{y~mQ0OBg7Jl`W9T zUews}q@udvH+8;+nX{a!5Z66&J9to)OU~P9q6lg@-jGN({Vp)Mahi4y41qg=W5Oyp1nej)O^Ui8s}B>%zLT!Jnn>&dTjpnhm5#1r;PwClyp#G zZ+Ckd(j5HXGtGajEODxT&!ZLDZGg}^+aQ22&*S1Nvt5Z1j}-XTD|1oRd$lWH zK+Zj+Dfb@}Ul8Q6434>%dYQw~S?y$*n^39Tr@#2lAYE=&}x2xxf36?Tw8yZTfW@-ohck<*K|78 zp`hdF$Wmtro#Lic2A`Qsuqx{#8YxOf8_9JrqTEW)%Mw`=aegAvDM1^}sIm|+yRJIX zjtopJvVIKXl!>?Ce`DS>C2OcVHF;TkL$uX1Vir29mmo1HyRp%e%U{n$l>1u@{R2X$ zPu)NQgq_NYad~Qo)evda%Z5ntXC;@0@x*Is zW0ys7=F0Szbb=4P$Avana%4I(&@I+jU4|6cHI}L)W_*21LxzM6-I7%H(dO7Zu^I5 z36?lJ9UC~E`fQ;G)&)UhLYrBL`YQJJF9ln^gzlLs|5MGq2Li^^FzlS5{tn)ZS!U=pF;kJ}!X~s`*9z zig+(Vo-zO)Rd!y@sGrq&+A?i@{8sQc4NAb=~e1H1}%myZOX5XiiQ)Z`BAAn z5tYa~T*cN3z{g^($~sI?idiO+)sP4tJW(1jAZyXy6fMgVSE?~8B{uE2;LaMvcmHzf zNAI&=2Eni+pE8K+POczRbT?N4d(R2~B~e8x_caO$5EhRFetJ6VBs&khq6il7Cvi#v zVmge?;^HNLN7gO1*Ux3Paw$S_@sAteGI_LeX^c3)MY5!-olOTpB&m4pv?5FI6r61A zMvs8){i%cRKE4;<6MQ#5qvP%Nm(@zVC4u$LAcAZ5ADC;qpfF26KE)F63p|Kz`SY)B ziL%K^`X13PPz~2Pgy8G3AO%Ym3pz_Mw9@M}zhIZIS_~wb)I}GQyF0fqbWuPh6LmHC zQ+sN6ye{Z^4gzE2tM=+djSrW)SJ&8`d@a$wSm->dhzx;5J`(sJ}JdGUNyZz(4Z>D9dCfA z)RAnFz7u&_ovB@VhSsR)Ag1|v>oE%#i8#?lM}AROg(;Hh`(msK_?=q`UXE$OV-84E zIiC}8Xa?OEC8N06JPZ@G+@RrpN)BabthVDuv-t?d1R-IuG`IzGG2fqP2t$@UQa~() z)zD8}bFl5_>DdMQwnbgilD}KoEY^T6Q}pCcp9$L5f|<@j{s`~~?&2XKr*Ji1oS%_X zGSMKh@ddzemBsfbj_*Mgq{0u^Mqv=fWx@(kh~{4|&3QMk-hlBLP8{Hc^!0kZUFk}KJt+Bp8zmb%u@ zbq*KWDfMVk=c{bN(Gr(|Tt!dR3QvT~hDSwDhFSJQm#WSz7sa7r8RJlk3? zWDm$+`Wg}%mLZ?Ig8xf}bik_5;Y8~@bdB>SdWp-aYyOo^%ez_>P^ao+X(fs8>Sieo zHg){#jmmUn378!6b8$6Ua@9hHf*nOQM3u9-G0%!z-1k|-^Egl(XQbPMi6cMVON=hB z;oPFFCcnSoMUQHeTltjH;P)7tDSDzH9C&v_0U4>()RE3l?^uP*j~Ydyb`Mxe9P6$3 z4Vye!G#EAjesTWLuZP1By~5RAN1`zOGQ)Lo_C#I~vi|!1<-wgf>kaK_u^5e&8A57_ zdl_r316Q9uZgj$PaRmO!fIk0@w!wd!0*GIaoVXnMN&o2vP;4@(3r^v$#;kj_(sIu_ z-Fx`8Fh5}>I~z zL39$xq1N;MbJfUSLx2&^Q3sr2ZSqxdCJk9;M02jDSQ9L|N*14Y@B#SMazhisJ=^Os zQkzSqF{+hpeT#M*!H~3gg#1Wce(VczZ{OKL@KVPKjVnGWveu9M;%l*L$h^uHsGACT0~Ywz{r^tb;W}%%=se z^P{Wmw%qIIkdSB5nj4n`5ofsN<`9&uqrootCaj*5fVX?kf-X2S)T8M3C4GQ@HJ4|u zqD(5Y_zaIp2|>&%PU3{ez8N@cYdZC-txIckRW(j=;iKuYOPPIXPBVcO$6|=jzMD1>|vQQjsL##k|=-cm5CR%m9zcL8k!`PEyt*&~* zFXyy)g73*gmRL^ZH`lTMd{=T%VD@FeSqvy~02tg=BCzzbO^vDtLCex)kRPJ(44ilWNcHEtrEU!55RL($M{OGQe~qOUE-L@Z9_z>`hbqp`-1+Z>?fwpRw`iu=WF)i ztK?$&q9Nhi1h*CGYrqT`=A1l`%=fg5^yzD+Gb{KpP(FX}(#Z!Zi_7EaHedYEZZ4fU0#m~L>s1;R^%7vceU zV#$lE>HrMucp!nM3`11O!-i(8j7Qawdlevex!XNg)}xASI`OD)`)F8`{MTu`l01tqy`*J;j>1SPVe;dQATV>OZuIzz-@PnxQC? zJ{8i4`@6b|-K^nyzO66suh0o}I5a0AKGn1Y`3t;xc`k14i#|1wmy}jfg38l0E1Xm{AL@;_GEozj+zZ7-%4MW-e2t41GR1Yn5niU%YhuXNfUmKs@ zD(~017@OR=v)+w+HNezBG4&zW|KAT|5@h+}kG4e$%$M|**oo#V38>0y^0Lu-kJ}4A zB7;*cd{T=jdoMUs-VE4ZZgc1ksw6dVXRQr*wXQd)WS~joO2%{@#i$i@aFcYZi+jo) zn)AxA>Z=24%5a|N3!-aVdx}hS(hT>o#p};_9J8Bx1U=SoZaVarxzpCdvvhyITuT@d z{J_9lAB8V~l3FYaiAnBD8H3~fPVaW-S=!j$d!Y1@;!#VqF~41DTCCAzZtGkKJ z&kxdy+YgrT`p~+ec;^59_!$Wk4`p zv+i(2&x$U{Df+e8`WVZjqY`GBHDBJiW<1z~-ZHypJ*%yQo}pvUBg}q!Y~a@rUG?B} z+W;#)tyXf|8q34hjOkz^$PREBGb1$3#;X89zUYxZw@f9gxq)5~_4}3#1~+m&y}Yy8 zHQKW$?W2?9c|O%QDoCKH|*caIiK^3~mMh{ALGu)?qT zVePN4bOTc@2(MGE1OZ^(0AK|Bp?P&meexZBzjE@-rhyGqmAYNariyyS^7lgtJwe=K zQFm4ZnSv>l}pJ5RIW9qmP1-@8a9KFQXLrmT4Zq6T1^%?7$m-kY7evsPEq}k zvDaXpdy(Ee#;x`;)#4b80)8y44Az#=L&4SydNdVMj*kE}6L_raRPM$z6%?g8T2vSr zg@CGOK#C|)AeD>l0i{M5Z~DHe7IcfBR^BcJU(&_q{e|FK_8rgz1U!32Jx!csZ<{6= ztbr#~M*=qgKHUFz)T(;W4$g*HKRZfs&7DbkS!<`9xn`Rsl@-5Btel}5-}PzTuTqJ) z2mH=U>r2}LuU)$9O^KDTu2l=8YU5d#fQCUVm-n`ZcKU)J`SE`SlaCX`f6J<)Grvo) zf#Qam>lHA8Ghrw@);v*mT$;U64rPiLU^%G5biXN5W{_6*>bFWAu!&&oug1#~kE*(7 zp;FU^0U0iBr&`vf5LBH-l=&4fN>nHk;u`Qty~FI|=7!0%IH3Jqqd|fkEFk* z#2$VTlY2Uw>@SSk&H~NWZAWPjz%-X;0{4%m6qb+mtf8T}7rUz}x zrW#|o{V)gHT*(HxYhJuJ=MA9$KGsvmoTujsjbe7sol$=}5E@`6RI9RF)CbXVa+c)Q zb0;u)7rYR6VwbPAqHj&EG0E`17wEr&(;bY=4FjNwVd|?+OAt|yJKF18f<=#e>JL+> z?3gb&KE=?B#5Y&d^UopDWqn30++dAR^ntbs2%VuWU>3;HT;(kQFiG~4#ys5DLB-7m zVklxsVy=R{2WMz@I0E+cbo8M);OX_QvD7^`ymvR#*ViIztX%KY zK0H^w`zox-Lu%Sa-rM#1W}*6w`X}Hz-=0OmocF2L(Nf zs$HG;?9Tjv()To;6x_d@fb(^}&9|@)kC+Z|Zk1g^yeRxra1-%U7{VMx2CpQhE z3|sgJeYl<;)*P}0edGoF*v1^A@&%)?F-s!kPzc-e7z1q%7+-%vVm-?gItmp3hD!7a zDWK||o75M3o~LI{=>mLgnd@xw@vn#BUtip#Gfrk+EMwNm2h$0FMh!aOv%lRu&! zuJ^`%Cci@s=#*8~vP))nfTFNJSg2)6XBNddP^3U(WOq^(7QNs35|fri4q(wj!6Fcv zfvcFNm3IkC*JNt$IN|(Z)Q+k7hbF0rO(vUHkK$7a@r$}mo|6miySWHYyLaB~4R z$(wG)^$HO5?jZrH=T#KzB^nKqHR?T1k(*Ja46n(9{v?j2ryx02lBkf0OtUBxLv~99 zMn%0H|0ZwR|5m8{eWV7d7?Y(Xl-dMY!7+^fxAB+#I`IZ9NpWur*%N|;sV2UaFzkaY z;^^6r+r1!XSTGyWdZRb7^HM8ZrS6j^0{Ft2?lnaOgR6JT>d zbZPfPMt4K*Z$cgplHWO8o}vl01pok2CfOzd)Lkw<#5-vJEF+!|aP4sq1aCu|@{-_v z4vwHC7Vo>|ln?x!I*%@m+2GU`Wn%FyHU}bkewdoQ8Ep?{!3QO^9K+(01H(+&aZ1;Q z)UV45?BAeBR_{sBYXXr22*eU1e&w>?4Bp!Ee|p@Hb4Dwov@S4eJJWCU^|Q7Tv4Lr| zq5q*c^7g!=Dc#ZqCnX!00Cc2slw6z$K+m1FFt`~C5Sh;efoTV)jFY-$d|T}>c5H1Z zD%#n(ZoN=L&e3X2dWjNmo`tSHQTR4vUw^0$J^NG$u6J0yYrneft0WDVL{V(@rwma$X#Jy+=+H+ zG3NpBWO~s4KBqA|xd9j7tkeoCGSMH9GfhD|^M;#~+!mML9+;BtBHTB`1@nTRDRA4b zH~5~Hx_r==no+jp^)QEp!gydC)-Tfhhe-uWLMDRf}Oe+cmSX#KB3sjc>-d~dH{H@G7 ze5*l|v}K-vApL`LnhpjJI|C27hXU^Gje~K;6pxq~`2aF9a#BVHLNaz>?c0^*{7DE! zz@51-dc1UjIku|*zW5|5?8mF#lR4akrU_tHY?Ud2E+1&nkRdCuZBqbHI-EUXk=I z1-%d^JITc(@^s5_6TSPvy^j^l8W}H%D@Epz~S_{E55IW7WUsp&yw%gb#yxM!71K@ID_erQ8W?uzz_wB1Vn#TNB1~-RSm4NQw7!&3qU#MSh=JK=Kixx&ay&-wh>0p3>Z$=K33o_m7q zx@C0l@R2+CPF`(g8gztNivr0YUB*cD$Or!GL3cWPl?ltoy@C38Z*11*L@;;#SonaO z39iEGkLymOM(aoha3%!JGM3L@GWlZ)syove)L;svI$$c)U)oO}oqkLnm)au@$_B+@ zr*3`)kv34f(gu4CddH+2N~SEEUuINea&kBl;(wcvL}6I2>HPzIlS!U*gRQ}P&t6X+ z6>=rugt%tiJr{P?5s~O^w&z#}~8qhuQ^=v;vDV3u`j< z^KqIORn@#sjN=WL{B%iJMCVyIEK`rE?XX+vv~vhN;c&OChVLV=OB{&hi(RR=VpUNBF#CPgQ-fYyrVg2jatL))xvP~!X0zf z6GW#Md_4(Z5Aw%#Kf1W0p!Z8{e6&CEqqiLqb30$qg!o*C6wW=Ije1e})}P?eJDE17 znF{Q)kUW*F>Bf)~w$9AVV$WO8Ws|WahhI?v_ILbOu_hOAqhHc3J2dT~;j0`_O^-j- zB;UZ9NIhc6s~DTVy2x}r8(YKchh0}dF5YW;UbdnC%53#8XpHWTQ#=!43J|ugg^qOm z?rdnx<(S=W$XMABH}Nx5kK{@oxAL!#sN0%X76>gZ(2ETYLmzOWU34M|5TzDd3CI9= zeQ7fR=iO~Y0GFlN`Di?@*D##V;Vx6&qH{U#r6m_lF4ZhGbEnd_Kx2sTx$O^8#Ozn$ zSbO6Vub?@}sg{o!3HN^4N)$q|6RjpjA2-IBK+?(UpMR_{uK)(U)k#I0;vw%>c?weg zua9e|;fbPz&eeq=o!H>~2glJuI-dKYpI1n4&cTuj~E0#5;e& z$S}-DbdrdP>_sUZlPMlL?s@38k+6iFUxp{qRh^qxJH<_evpn|01~hDXQmn%YA)){| zGXIZfh|#ka!_GAAmG+H<*Yn3-#d_xi$zNp{QDib4Imb`+?*3R+6Px2$3HFy0XaAKdilED@8)+`onbu3m96djrT9XkCYUx|x!} zF1$Tvw*q^rq`-y-Dvqb8_{_$0{^1fGzp({)lj@^W4ihBs?EtAuKD1vua_9CYs31iC zMW5};y3`#8&glF2OT^Xd5o34&9)~dk>Pa|;80CRuFQgqnqsJw_!Q4|J!WNaUrZH5=C~TP{5_6fw zz9yI)Ebe~$ykd?Rv`!j#inLy6ED&|Sw3#U+gzq3m(sv#5MEAN{9#XnkMQ!p<0SpOH zwaed~iR|L7LX3k}!*`J#w6~2&ruiwF?_R*32nfP7fyl-?Q5kTPda*L32 z2cL6EF}&az>o(e{K|87gpQo=6sWqbY*E$gj%@C1{h8^KBQ4$%uTKr(h>T9SMePKbXPY|7Z?+FmwS@F-mN_+g19Znrt~C z$i^&070AHSB5SK4OFVs-J0etXu6NhhKZQE{2)sBbBN@J+nC~1EWOmV!;_F}@CY&o% zf=V(f8d0i*c^J3?k`^K`fj|4xdf1k-!if#Y9_k>FL-9wJi8pQO*;Ae`*Lt^4h#G}0 z@1y{JG`}=M;o3YQ;kzmshAxWCpEd20(SiG*TMYTzPV}_&?>1o5&s~?Sjw-@PB8(~~ zF0Xv>sS-&TV>TOFr2)jOj|#40PU4{M=h6`&AChl*2cah#9B*OW>En+Yon zbZW7ZKL0~AaocD4=HP6xlSK4hM>X#)m1#^?dHco{L7~dNjg}@sh%shwRn%B!IRO*@ zfl%nas2DqlFo#Bm7G%ryYcc^30fmCnz zCfp2T`^yDK8yOb*HpsQzy6nhq2Es`nI z@nr96kw-uy#JGrQLgBheX~g7#WhSFDF+9t~VtLwNg?0KrcV=$l*dHF$6w51M6rdOe6;2SLJ0o^) z_+%7q`7Xm}`@6S_96^zitMf!3O|c`dUvL&qZYh__sk+-BoQ`)vg3RhpQk*h<%Y?SG zycos;E)g0F1wVs|N{hdb&x6fBuHyMz=96|`ojnsg9uk8$6iq2}dZ`0Y;4LuNq-tR` zlh;RFkIOU?^g8$icHw4g!k%4f%t1bOk+F@EhzrGDz?Ss?dVz&v0^#if>}eQFh%B^~ zX&8@3`*%_UR97=Tp+C6u&=kXM>%?#i6fZi){{EIA2kjnAcvYWNYw<=KkkGTxdTMbO zE_go3S{gauk0tvURBkeW)fZ1cm8=Mx9g6|hkhG4x>u zOy5w#HA(}r+pwf?6UJn32ML{2Nqm;l05p?OX>|2M8!-04MRl-2lBRvM0Kf#Xlqm%F z02f4!(s>c-d8?lq?mQ{sAn#E8y>;N~TJ-5`n&nN=d@Af!HJ`{kACYf_#$f;##f9^& z?TK_S0WP7)dZSF?`LjtWF-Qv6rTamkiP15f(O;9c{cJt>*^)BB6TH?f8`_Q=vnNwq zf%~of@?CO2s@&}O?GMiHZzR4GFvMUMKhxsJ%-B5al|G12xEA+Rla<(1s$lloq{`~Q zAIrHH=8R9NVRptU86GmfUyX1@mT`r^JP6c}*i1n$D_`B1Pla-%DO|dih}}PoM^qw@ zofm(kNhmt%nFj!0DcM_45_l<%+6FXxa=np9GgW*0;AC(k-sTB?mFV0St#-J?pdRuYWgY&X0v*C5U)g$R}<%r}^=^2KN=cy0h&df0hX&Jy8GpR)|n;?(pum3$9V|Qw~x9H97 zd}ym`Tt9>a!k_)aEl`yt4(vZAqDYn7Xq2dU;lHGLjx|l2njfi#KZAF4oP%E(w_0IJ3 z0R0Y)GQ+u(Uu$%yAB9rd$d6yxUBuCXEy}ZLiUVULgkc z<1`;hmzr`}0A+X|o#Wm8OZdpmh}lq;^_l%ml+sq-L_=#s*`@WlqVACY3xXBL?Di^e zsOgtu=W7!nmxHn?i|6F2butYX_%x|_0j1wR_@TTk4fVDAr%{!`*Z+wT87C)!%?vLVC1+#jR(5C z#W2MEVm?=|&R}B5)~ABoF;^6VWhwoq zl2Q<9f)u8V)|JSO*-B6=h%xnp23Y=XdPB)D^sRPC0hIKompbY^3NgCw41=~mmH+y> z!p`yWbw}oCju4i(oJ1W*onxLtTmfI*x&&w4xM|Z(g%f3Y?2(mO#>IbAbTtUXAcT}r z=E%amO`m1umg~gcpIZj0lw=6QP0x#+hq7BocXX=7CFW?%x&KRrxmvJMQ|R6F zR#V7$9a^UD4;QdQty1@c!#iHR@D@OGDcF+JG7{sCEXOk6QoY^h?D>2?dw!b8v{A2S zN0M>aV-QxBw$Yd?CJTNW@b#VQJRBT{qq0yOTm&BVcfUU|PR;=La8RS{ygNFML&%&+ zh_%xbS>$(5cq=!vLSSlD!=`V=jbw(O$q#oJbmQDD*pU}&`1c9%atTFDKoDDgSiETP zpJw9Kc<*oK-@SZd)UY!AIjC`7t||IF`KmDu`c>od|L5Q0P|XP!^Txoa7udAzW8QHg z4Ou;X4yAK29ZpZnqkVshvXdhez7~!C`71BVcS)*fVb~cPYNPLf&Au8s<#H6N41ue| zg2jY1mo2nExXA$EW$9-3;Bl&O(Fqs}X38T(VCA4I7he`^^m5ReLTHQkw`TzzeTPje zhfQRJ0*eITSVF!l9tR%zW*b~ak13=>$(aO5mP(uMm+u#EG*?K<>l0&-s0iS=}5?CO7;6IvPt5K=mj4K@`yB_EQJPtOxk6ED@5o zmg~c9RqZw6#{CZ}_-BTFuV+{I_oR5rv*4<(QwyLGFY@y({TDbuel^6UtgvRqDKbNe zS>MSuAs9@iK3acq^8UbR8z+~|Ae2!D_H5r)`HkxlkCcGKB==3RYyz7@E(^N~FXIQk z6*HmdwM3!m+>#E@f+^DtEgum#Hhq z5EK5M@*h)2USw{-$qzWM5(#rFlgr>f7n-J4rS@a$g1=jQBR`p(Y7~B2I5Xw+`nz-| zmvpT}{V1spe0=Ypn!<{)Bqi#g_*t8oKWgKV^B^FK?E3E z*&Axbl=Lk*L|jE8X7WKF6J(rTSE!;orWum@6RyGLfhm3BF01?;P2?>WNbLPDJ z&z$8p&#RN$*eD&+6A?FX{5uvz3@OrfC{&<15IM>w&v+NV1ACca?xcnu`p{t`)#4;1 zG6Fq8H9gNojyls+hoWrkU9qo~5bBGH>@-%gM+tk& z+Ga>Q0Shu~%15!qv}0-e1&cQ;ooHnjzr!7tf@_}pu|HE?i7bH3;`x6@)rv7tK@*_2 zfTNFbD`w%-c1T*ll6ZACc_$*-tFpoTxv^40tH)=tv1NA>u=o^Zw||kKb8ySA(KIpB zrP)R#Ggxq+CN(l#c+tTWhEes4 zC)y2rUE?nSPCwbPR5o!w)^fSAcrh~2Se(9gR1yi8aC%t(l+&oyaJS=N?-URz5d z__MC|3!~N101JtzoBb;QguDNxnq0Sj#3h@b#(O)Xf(&@TCO8! zOR2B%G2_&K&+6lJs2sO)%3V+aS2iD7G3!XjtU`n@phej&nOES@nW!KfJY#>3myJa53y)j3#+HyP>m# z=unW4`TuAE(j0|TnyO?Slw?%u7MyCVUXp*I)$6Y=gAuTAX#$(oXL}ACv_yjM3Yinp)9~IjSPOo$UVLS2Uh>K++T^vHx|fbdVgtORxXW+e~R$m#-Ow;gfBo3PeMZ| zCb4TJ$^sK?VG|m2oQ15Yv=*5^TC27G8Dn2{Mf}UHPAp_yJWn=SeI3M_;^iy@WebiM^On0+8r186UDF9ynDQasBiM6YC%#i&?8_`RKW>3x{2kPVNy^OO7X5Wf z-!&GjEg?LF=k}<2$p|wx4817$%PqDKX-WuUyugP(6PIPOdVo>x?|t+CtpP| z4TyDKY&i%CQwQ^Wa0rp)+KG3lgg{>!Y~PdC zhsB2lEH&Hs|KJhL$8&;dM9;-xJfciKHPQRr)+zVlp5~X|BTpd2gup9iUZN)1ts^FG zJ6B`4-22kXX`{#(O*rv3kHNY@&j?~_A#z{2AvaH20@sbmuT;J1*4{!LA;`cl_!;I3 zGnDl7RO0&u$jm0vfm3>K=;fjmm!yvpdAMX4ZMQmPc!#hQFje!%FVLE?bGsn9^(GQ6yvBkgHhjYRMGuw{2 zOBmh^q+kg?^IH+hpZ*-o)ZK)zTDC6W7#&lmivev;3O+S(s%ilKd}Qch&4&iwtO2r0 zMeM6z{%ozZ0Q#pq>Ej-im|-YkuI$tm|4!BMXRA>vo&xB}l7{70{(U~1oDqjy1PF$^ zRZGX7*>(0yJdqSL_rC>-SlX>(!I+vxThe)i6c19O;8qMmMhT9=$6M09OlHVO6Tdqy z%^e+0&v1QGS6&>um2T-v4d-_ylVr%6s$fU@Q=>Qve{UL^JMJbv858h*enrT81cR7A zS*1uDhxLc9Cu86;;lviyNDzOcu(yaH5^%5{@LH2X-zW4Pj0|}G&ExzYnau3(aj$lh zWy1}rgr?A=Up0dHK?Qyf9jQzX2O{Lmj%Jfx>Q<{?1A`{&|IsTx-V!9OXLk}k?FS}y zo&UD5Goryuq*O4PVLdY9MN%MsqS8; z!QeTff*cP1f&|0Z&xZ{rG@_xxRZ$pACQfiL zBV_&~nw;j1C1~pXiXaMgvjskXyIFb6(AJsaq*AM z)H?(M2_=+9-l?)uB)s=+(N;!%AFmG6@R%}fLi|oUZ?b#R^4Un!G{$G=KM^u|P#{ef zyk_(ehkQd6XBxWn7zcu18(L`mVKQ&g0gIoL!-3a_NNN4G8Qr^IYz}xmZfv8ZI6uvs9wcz|(13f|?>%qtFKv&ezK2elp% z+YP8slQ%Jbk2V}4<^Tpo9}m!*qj#dHBnp&WE(Lg+B0$czl+r`6s0M_2kP+0Kg(?xL z&PkQd&C+P{sRJGv+})2iNXuZ^dB{5FLXl-?p0^?sBFdEUon{P@y}C5HV_dM zHQEO8Vm`)Py+0^N<%_CB*#xMJA$A>w%!XwW_|KuLeeW!rB78;QFxQq5(&+sSJ2It! zjTdVWNA^7H!oJ+`WX(v@7WDlGUnF)m?Kj((pb?Q{t?KZp)aGjyhu1lbYSy-_SJUi) zb~#I2=Jv`D&*xHL#TNTca{k@`Jyn2&qRL|@3|-*+_7|2^_pkj(?^DDJugv>vrPHL$ z@s#U+#-vbkm+oN+jQ2#lUz_+{Vhhh<@j>Tn=CFi%-0QMH#~mLY^~>FVJK+#QYUHAM zRE%>qKN|g#oF0Hbj^$Iji>o;V?kM-!dyGtyzLsT> zM3%BD&9v&;72ycNYRZK{MIFLcoV?2{=CGxHiSM}j z)`x1>7y*GSzpRV&VJ4lHMQUad3$D9TbeVLlu3FYKVCihx6k_cj%vaIno-XM*Qe1qP z=qJnoMq4gLjIGicwBNJAF==UOC4kmMtsr&6BNO5=P?uZ&RZ7g!nQ0^1qY%$aB5APW zk%mdwK#}<3GTONbxrpK@HY+-Py8Ynd?| zA(CJoVRnx_+zcoGom+t}r$BE-CGsYA_y$Hex5c4uhu{wIM>bXDq1=?&XWrtexEqL@ zurK_k7z#8Fu$Hv^GY)e3d_U|LeqsN2JuQ}X_KpTq5I%~*5WL043S zGo;b{U%Xg`p#aRjiVUd*H2TpgK4QLc%t05aD?79iY?5qz)9(b%-}{bif>B=WBVUTm zUOIM|(18iuwK~u0ip8X@!$8;<)EtqZG)eLt3*>Coza@k*aJR3qSJA$*VjcTkSzf0j zYC*F5y9pbvm5b{;RzcF*gddW#R|7r|ZG{Os|3@=x3=e_MiLtL_#B<$Pn=dsT8#H@i z&k$V=XY+7bEp$LW13M8fzSO^N-)4U1KX&N2QNtT5(!>J9fbDzdxwPd>2VDQwZnm&; zBx5;Mz_0z?UA?eH29<&RN6UBC%2QYG->e9?@ODVidyJbbV3T65(DIz1$DJrl&EpHB zS4vgxd@>5#vWhzI1~f=F&=-so0t!a96v@{eeb;wCc(8bk+|r0^-8pb6#V~0U_D#Zu#?K=V0Xf&o z_4d#7cUc^ti}we0$f&Ut7$RPfcoU7+t1*aNS4ZJnbY;vJ`xYOLN3j8}x^dtZcwB>jSxzXqlKawS=U}7tPmie|}j4bKeskPWkZyjE3nTBBj9qy_& zS@rYn{=y=BZ7YB(OtD=1PKuP}i-yR~H^OP1RTTP0677Jwun7Ft#XBN9^wGdT-y=S8mT0t;@XT*H0PF{XdQk z8C@y9*;8a*$z@|&e|Lhh61!*Y{&|>B)Es|22<73L1S($UFaGjkLR}Y~Mq;oFYdunZ z|Vp4(w7>hrhuzBkpx{^#J_ zH8*-LV(0$YFMljL9){dA`Mlvj&f&r$|Bwx{!k8 z{%#c)@d)U<)Mr2oy7=Tx+gU?G(bf^R^{9>I<3J4y076>Yl%AYe)7oG>FpA5ipyC$? zqN6A%;cab~T75k7asNGyG-3QG=-l>Vhu?Pp3ONdgej74O^{z{l+*z1-(#r=fxZ@PZ z$6JyTp5fhU7=+rHaQ3x%s4rN`kNy8u0Xq%6Ls5b8YdTWn_? z>G{ZoDr`VS$HxlrTMxqE-#K*%;|hPv&rdFZWBM*8w|kk}sA<`4YxV ziOW=XR+Am_Cj2Z9!wz80!wT+jW%?6qZIOqAG)4yLY-%$ShhN(7Ul;!A{Y zM@@78DgEBpPU3e>-}2>w+CNa^jmfi+Hj#Mfsf|Z}K44^4Px#^eZi+nHKOlS`7HQtPRm|qQ@PPYPVRKJpaT3^cx zm#ziEB0$w^_4SxV2CY5w$1zeE>n$7 zvFS=EC9sOdik-?fh&G~r5QD12m^eJud)jyEx+U1%^~#~CI2*OH6`}adp6tGB8uen! zMw%z*Y;?(|H6uSB}ED<^C{Fz#P(Qz{?k*lk+}&-P)jS zMi1yRW*sS=;G0>PvPc#dj|LvZhv9@$X|T6ZMT2eJ>i-xdN&YfytIy_HNY%mczh8D*%OxbWp zBIzj>`O24q@EI|aT=b1ghbT$#;quCn29GYwY8b(jnNRZz_haiYrce?9&0C42t!w&K z?FDh>RAW0j5VPj_p|Rz>$!xQhDKD}g(wiD>wz1ysEE_tDnrDizc9yTo(Kj*Ab~dYG zOjETX#H&Z4;Sn?+4syBj9}WlEYw_el7GIxi;eY=2N7yK$(@o9f!&(=XZpm2;()aRwbDBj@J`knFWUux}#X!FT3!3*q4q| zYe@tex398kJJM}*WLjlEtv9`E>S$6C}kgiP~Li7_qeiy_mE5=diDNo zCvr0QvH=pSdLR?h=B+^&nbK8Wu%W9r>z8zl$lmM%6IWRrjl zK27_hu+5?ifr#8*oqYb*jm9*G+jeu$R+sqR>M2t1V7VSYz=*}VY7W+13)SE_1>IwI zi0{aflw&H);+@B`EobsrI>u{5^@gZQSU2F>!CvHBb-ZuUTyCgFK-%@pq6h%{5A`j z_$FfMyvptf?0h5H&W@2KtgtzyE4EZDSTf8t)b|yfdH*6?N}r2;KwNMwuD{in z89TkNnp~4W1TC(h;z=R%FX8R#{8y~t>{01J^vA>L0Ix>7XDssG4DA@ncrefm?WX_c zU!92S2|?0;Y?|pV1mSuKrr-24BuUlfB+=%21V{hR!KVn2()F)S?PIc|8C@yx85_WxJ^a#9p zvT#LJK3`v{-ki_yr-rHEeEbg-nL*UK&}Z5icZ>~mKTVoat23xLkK6Od9V8h|&sW9e zm~DHZg<2$th;@biQv|nqgq(CpiKGhiu{16>(yK?UK9IET?e_7}vjx%Wd_XDZ$Rzs^ ze|HN-^Y3S&kBOpF-`y2_zD4ompJKMWtJ|6|UD&g?&SmDs!O+p=5^yqj@B5rd*^#l8 z{&3otz7yzkn;0xFZ6T~Xh`MAJqD~=d{#q7Nhtfx{%*C6FtY|qZK*A{2=~XdF^ULx zW6qxGzz71#CPHZ_#RA6ia57AWgTUf*@Xbxoh!x?E9^`9h*!^)YgQ;m8cpL@AQgJQc zMApUN%D*qD;1vKFCfbO4%d|pCo|hlct@o_6^LigxoXg4HCDVPz2AaMD4Zm`c8jAFU z*4Qp!%v7yut>?Z7w>WGGUAqP`^W6vhL zWVyYS#Y~URv3Ngz+(h^JD@LD!PFk#X5RR~*Ta4e&e{ft{un=OM4h3)&Qp+{M+GZ3j z4$k3AVNBelC$tfXqe-E#eX~1qf+r;F8)-4{xl-lbALCUI9peW7BVp@r%Pp@&ZE_8g5Fc`QUt{p3lyU94dGc?#r=b4yBuAKNI4U5Z-+&r8?jhAVmEmY-) z5l55F`3|dmx+4lT{!b>W-uqRzqmmCBjTZPxR9F_{kyR}hp!OuFTVM6vYxc4JHNj^x zRCi2G0`@2~oSvymOxltP^iw@T`1w$bV>oRuPB`3_o@cK+FN0934A`Oun@q-fkNy{A z{?~zSa)6dxc)`?c!2H-VMTA2zFIBR7*OO}P0^{clYYslYUJ6T;kh)+8=dsbBL(t8QtAjvXctYDX?UgsIQSy>3Jo0A>$=_ zmO)9eDU=<_+lo7D)!lQnG+XM_t z|JNqaGdydF=rFWTqN!H+O$<vi)op4br%hcYq5t()pHdZO!fI_rw>5!L{u?-oayi^D z9SRXzS$)_^THSkI(_QL>ZQ2L$b!|P4YKyFJM`bbq;8m|C#)jM8#-8y>PxP2cs;K(L z9c-&dDa$oFqyOPfE3$p3l(9Y04(pNFP$5w%XJ(SEeQ^aM-V^Uy#D!a)+swAEMXBTy zj6*fPXwgvh=Y!1#;)GQ;7Jg}#e6ZS^)5;WbVaQ>iK>|kjoRl>!h*Z&ExW6oD;W17; z%v=%xuL2R)sd#bUgB&{+?|2-Sn#g{l;%xMm=JxU^uncMBCK#T(omE8o+8Qi}Vm%&L zb7ifkE6m`ZOG?^(xG zh_`F42IfTqh0Sf1CKXkVET$`6Hneziy>jJNjC?ghSHthJ%4S(li z97M!D5gF*>1jg?U{w#x~OHneI3u3DJ)R4bne(1f}_+FsPPy|C|zbkRDK5DB%j;`h! zdcN2hs3bJO*1NRlVL0U(yrr$F3V>H~C_yqJr)ncB$9b;||3Q^BK0!P$3Q!TU$vQMD zE<(q4ler5iC!f+o%(bLy^M-`V*uY!&T-e&&3Uwx32os`V+tTDFJULB@eTzhq`ox+r2H_`pLNl zE4fms^jQ}TMfmCH&+)@p~@UD2{y<#jg-_mg&Ax=(P*q z@TfRgXhg*@!Dep5QP9RazqG>HZkhc=yG?KXeLw_F<%=|GvRw@PV72VD$84-#njT3E z4ud_VwF_t4WdOW`;OVEzG^u${+id3{k9mSoShwbI5n5%*1-mjX6M7|wzWTU(?RlgAuXb7y8ojExVPnk83tkE z44j$5VT~xOyX2PloV+;fQEzSd5!R7(52DdX+cmdTLbh))icVts4|lD`yV3zikS-CA zgZV7e9SdWyPZ+3m5r7Va{;?Dfo`OkSVdi95d_r#(KnkFXbWp=Ai|FgfnvH_+xy_7c zg+MLBEUmzv07oB%LZTSL>%jpxzps#`9jCp73}pFK8L`2myfOR4tQCW+n{!55e0I3Y zZT(7GFLaW7<1GQwE<);0WNuv%A(f_(54TAyFp3>Rk@bVAmhNq1KC)sw#;xw!B^ien z9})pY-Rlq-0izSE#Yij-5Yxm$Tu=FOhQzv&hRCf;!SInM4Cd?|$Ehyqf1lcKl`5-; zF2I)L8Dj_K?}xd^#E6dlC25WmWX{o0rE84I&y)S!86E5sv+Ux_4{SOTHYKCF(4@2BKmhrez$?iQ`)0Ex71s)N=Z)Y?IG7!#4x}+?<--2=NF&D{Rfrq4%ocl*~i7x+02DYLLsy z>t=DIwb8T!HgPe;-|Up3no4+@ZK%w2E10RLh{`tz#qA+=RcL9B(#$pZJXFv8Z~XF~ zQO!CVv1n5Nb{>KE%QvK{evq zqSw*^D8f8Owcbo-`Wr5uY0(Z;aX;3G#ytxUXs|(u=c8K7ld&KfdomWeyQJ0?$1+gk zR>hs+oe3E-e6L~#?P9?~uq%|&+Tngz5;T8W;hSa`j0GgWuiy95&lmn{mWJeEz=PXB zLt%-@7c-PdN0km-`*qK{3l$m}?ybbRvHzT*d|#L?Ka z^!(x;RR{q@^tx!gkyYjKVn#m)Y|ePwnSKJ=+q;mx2M5MQUzYpUqK;>DYr zQ^i$=8s%$CDf<5|e=^**%WHd&6bX|&QCCXfjGMim1GiDtCLZ2M=Pw;{QL|oEgcVtG zB*!#X>Pnd*q?IJVSib}Ou3#O+htBBMY%67d;>qMcmWhqv`celDNpzDz48G*QPj^M1 zl{(K}IsZT_pWfwt;uV@RKZ)x5g$k$OeRr1*qJNlBsaa-f@hKOiiAM277o-ugoaPrr zu{j$dN`EIfQ2UG0>HP5u^H zspwYhCqdMo(JD1`Ea+cv7WQ_PQLVpGX#m1(^|PkQWE-PuSLQr&`I zvi}dG6$FsiqP?{IYl$9#^^&g#E>$Wws-_Knm*{c(xL2ta@t<$p zlV}8QcR8V&qY}H66P)Ae&#=wE$V>VI`#L?+yTV=J|r%G3HP zng#c6&_;0n`&%j*m?J+7FK6H_P}EB4Kfdf&wy)_VVekMXEtPlBp!2fDy%oc@J8=_;KM*mitCLeur(S=?}VjReElY>0gl`> zwZ4UepAGjexiS=^U@d0R&kSY7wS%tUQe#CC+xkc%h{d#4XpiPnp7AGO5ZKBYEA773-Q3D9}Tbn!aoNKqi z`ww026JHnI);lO08(E0ep*^84C<=7xX*rfx12?Hu|UsBx1YNW`C44Xm_+ zET1!OA~JHfz~TzAIfL8$ytitrul$6|BuiSa_D?Qa^Sr&e<}}#&%rxBFnx}=zXRl8h z%DURsBmv%mE&oJ)ZVb_JqhZ=j%NX;YeNQor4{6|iPgCU|bRkB02Lx>T(d~H57eVng z#0vgb7ITM7GU(e(_AyqKz=&7x(el46^*K1{$|GM{l;Tad#EV zWcOrSz005nZ9pys6~n!WbGRB%m>koQtWxg=2cuM(N+v!VIDivLS%;T9?fgjv(m@n1 za1!PSpI`%rpyraHqZyzRUhQCzWQj&eM)tkcjX->uKUKMRo({C)xY6sT)~It~a&|p1 zu1mvg2>Nir^WSqNd*Y>gHoh80J##dkjh?VS??X4f>diJ97& z&SxpME%YwwCi(~Sys?=r2WkVxA+Q)%S|F-QoBfRL&o75Yw@SFju3Y-l*cH|~f6BCf zpTy+v7#HVF{p_@k}|4L+b**k$+I1+OjD35T6tV0#8xbc|v<( zV~0G=AzY>%B(X%E&&ory^KJNvfw1te|7=7D$mH6-yJYEeV43^rccAQwLtbrrkdePP zcy@Vh*x{<+v%zWqPTBD z@W$S!GRd_Br;o=RfUyb+7~DTFnHK~meTa>SNm>ogQS7RXaf&9lk=p zU(%IO9|_d^vAA;ZPY-BN(v{;80eoqy4~jGseD6Ee+>$-B))83Mo$bJ8wU*iP_V9Ss zWtVLCV)kh!V7Y6%%kS$8>z$`bY2IGJ!ws15_yI;7u_A#?!n~BefRre&?=kM(Sc*uK$ z(-aWSlNTN()e?(|OX}b02lF4>W@@Kxj|^1V*x4&HTy*HXkh;oY`P-KW3FWXv$1-8y zIOLRN=BjV~H!B3$QXpX{1+rF5oIk4B;DB!*b8-HCYOsy`clPbpxLz z2-kl}$~zC?I6v?l+AGm(u~$&!jXi7rTshoz>IrB0A~pI^YqUhIm*fNY#)nVv-m5`O z${gdkgyJgTBIG9JWIW;$_M&1%u(c82XdqA3v$5hRux+N5d9>y;9(xMKhiR00+bAWE z9(sXZc77DC2upBnT@sO8o3=W@8v+|b)*WzD;7kjIuofxKTf^Nulw+Dg9K8mB50NaE(A6@d0DSVxhBRkBNaaEkB-;9a7UNPvBsh(ab6+X zipuExgyhur6+39hpi?qdDsOGJ^ID*^lm`r&ppBu-%_`kz?&X~SBV)9jCvuwALyd0^ z5gEY=;hOw6{XA2O7y+eQ)o4nYONonX7+N4LD8sH@m~N!Iy9Md)nxVT( zx|Hrly1SG{dT5kx5EOx-lmzb)lruQT%-9+7K$D2PJmtoa2E%2=YKfn0>?Uwsbo`R^{-w2FSej=McnWFz` zoKQg~U`D(E{Vw2G6Q?I?kuNjYM%~M=PqyxBRU2m<|Mx8&>MKCVh8$@WXd}#7e`Tkk z#4bqs=9!>rLW)pg!;pAdMCW?!9BcZBB?#DhqFP4bs@yB^+=}cQzh7cHGF{P{fgn!M zwzs8v+-`yCkl-xee9bH-bbKNs-2}$)JsZ$FHMYGDMipG!z1_Wr#0&?Bqc}q9LNm;jURYxHgiDfZEHj@<68M| zK8ll9YW*nX-rz+g!%m-bH3$7tgEIQQk)u?N#sJvzjor~#L*Zb??9e-`1#a>b!~#2Z z5BlPUW?#u@fuPE(GE)d6FeyimFt0|j&nGENZ^Y&L{L+i-4TayErRz=Y1x~Y=rTggP z-FsCavAMp5o!kHO&RQbKrGyQA?kx^g)bUXy zS{!Jg^ABMV#fONbagiUX?<14QNE$`!kqE27F!Tpr-OU=Xfm-(!vjwPA|8ea!f=3r*I)tkrN~Qba z6WffPt{Tib33mu8PJh4uUL(mXi>_-=c^HbIqd)$U(CIiaqDS`tyIXhL`RZG4#lBt- zf*%jTXT`qgSjDoce%AkeMQ@()c;pKuk`C{$t?t7S z(S8CAk(Onf*u7Y?>z2a=HUk@l{TKk%)w>Zo6WkzH`~)G_A?V~XUgNGwtu-HB zs`1tYrU_!-0XgT_s))CC-Zb>)MxARn79AxlW^7q-<9Y>_5jKutoW_~wja18x94;(K zGVf?p(b{BiZ4GVH%jedyx-9h>>-EyDvI{aj4u;VWmU>i-P7C#WSxy1Uj9IXsIo~Sl zV1nMl%caseUSFX-Up|I+O%^|UyPmJVUf zGxy@(hZ^}3?y#x-6VrcUlUzdBK)6pXrH`&5Cu>c$f~-0G3x$!_%L7AbE9CKEt|f+_ z1xdrLx;DB52xT_AnqKew)cv52_m07_{+)06O^PyV$IHn>X)r{(DkNPH^+r?C44kKNq^j@?aA7)+ApHA)tl-DQ4d-5Q=Z^H%&*~RTM5x-b8ET`7YaFG(&&6&iM%fZjM6P%3A}zH@q8707jx5ZFRHEpxO#*bg6vtZ4$%?OP2vmnX7L+Md#*g%#D4!h z`zBGxG3b8u{m^-WEN7F&52kaAIP!#bAy3ZtarZuLB_W}~_k=6|bgKquv_=W1+k z8(La>p`*f~>Z(4+|EXkUA@O|x=9Wq=ZkkzGW4UUb1RnPKW7>`1DsxbaCy7s?WSr3N zpoqTu2V#y4%#9)~_`U5DJJPmrrdgFq2ZO-lRrii1e}D1+egylcnJueum@aWNT$&ihrf_VbsF@e&hIw zqft}@^($ZE$z;t?vm{rk0;X#J?^1U{~`*wN$ zyfRI_R@A&x3U|LLjEV+wqUG85i1WYMpgs;@gY=Iq$x2%X9T{Mtmn!`!Sn8Z4Q(0%? za*56Ce1=v`I>BCR2@WhGEn>)K21k_s>E7p9x<#{0zWvQlUs3@qYaZ{eIMYy|f77}M+{UD}FAjN`f!{gCRgWj<`{YU4F}QGvB{D)yy&Uiq0bPZt4k z>565Nqu{IAOQ;CC78i1u(SXpjfYL^hN~mA<$q^c zzEy;8uiUl_4q{#ysbDy<;io^xz|AU56k2}hXp*(AUg zwOyaT+lEhM>u>ree#u%6JU__xlY4WB)`|AwhUKJ6mkt zOznhkH35t{i?k)(#4}9u!g@CHHX&qJwj|bp#xECGY^=O>c_WfP$hkO*B#*rxtv!Ac zdb*oOr~^%m#$R~c*v=1qno5HJ`|%ur9*g#f)aLJORlAH>`G_=4jM=I<1dDyAvxKPR zkE`0DlIs0HQ1{?L4_AHdC>D_FHFI`Rn_f%gt8F%BhbH7-q-c6Qq(cU=1w&xWlHrXNSL5SDj(50nCqJlN1ZI4My(K0Yyl$41!?DD? zW9yy#OP;T4xa9$8vQ+x@7^Ao6By`PQRIj0V70yK#$SY~+Sbs-aMWU^HJN6{bt*iN7 z{(qoY$A#ygl-1={&+F5r$ImH;5c747P}Rpv&#q4$MP1j6kg>pjf)+(l7+pmT{0tM3 za9${$(svSutp^3laSl0o8Ih}^zF4!o5i~W9;n8x+x}+)DjcP-AJCc$Wc6t{JR`U;c z>}xj%1a&IkG|M@ZI(;6YsiyT+RexBsva)!{lculc*3K8gh6>gc{2wZLcDSO;oUoU^ zmEfv8T7mCT!3-AiGe>D((5d2X)TZc8X4Bxv0}Id67l>Pj0O-to^yot??c$CLqpQMP z4&$#hn)P?p4H|>rM)s#S=6JtqPr#=`#DLjxZL)+ER_%hcZj;6~HbrBQ4+UCok_sRZ zS`gt5`+yS>U9N4NQOgTkYyk%^!?d)l_#ag|4#X60dpG&w-{b$E7C`2GykwNDwFvPo za!JWRB5^uI$#+Zl8=0=LZP4i@25Bv4v54(V>9@B8Y7XhOR<8^;XK&XxY*aC93efFz zhADPf}JrKW6K*%kXc-FTn52}n7C$)>2;4_hM%O%5}q)ZAMjVQ5(IoDNOT zw>g?w$~({N+l$>hYYm}VU@+m4qoj?H{+Tfje}`h$#4WNE1vUT5-K*-^t;dTIRM#tamisO%%&uS3=1x)GOyr;%jI*Qk z99C*p<`!~~HvEP%sb7Qvj^8gAn$<<5=8#k8cT?E)Dv|ip3GYUWf<*~>eeT&=0@^ZZ zB5FtgPB-?DCC>w#m@kzV6NbZRJNCe5qv}4yPmd=`Rs5I^mEY>?z5O0{Sa3UrNQ%DQ zQP^>s*D4zrBVZN3hX5CFHT*~UI~fgbS`hJ%9b^7{_E&E=c{2?Pn#5cv&RR08GL#|X z6SBsP;3!2r=S8Hl?zPx}YU*`}SycW6@4*UQseBE@yai?1_h{fs0@W&ui3#d$@rV3< zY_e>{eE1YGr-Ge6wfuPES+nfBx@TgaajY&=oQw);PZCZ!V;?-1*IbDe9XB@#Z#A`D zWAvQJU#Aty!7863-MKlk^K`TMZO;@KOe?&)Q9&YDEnP$}?Rn&?i&1(|o~h3QcDx0;*8dC5_mtV6xmvmm#dC8oG{g;-$r4P z*VTfAQyT((Bb=pK^MN?3@_@_&5IQPTL(3%EL4wwh;EUVfq|N1!Q~yuY3TbjSFT!u zjoS6V1sUHaU$E%M`)~coDaRsFmF;MMDX}&wW$deQ%$UsQ=6DC(D zKV96>J!$c5eWcD(Cq(eX<`;d4A+7gx(g!1 zR}$cQT~OjDUVgU3Pczbu#V%OfNb0JCDrn4NMask&cNrqFrcQ38PM zaX?vePNk-ZQz|5nTw-P!sp@REYi<9UEBuDj=NN0h%ClV;P-Xw6dRBQpAzrWF^R4Uz zb7*hznFZG=CnofVN?ba}s%3V~$BWQB;@AZfBr38JEym07~ zIFgD)1VYd&&!w1~KV>PJ`UaRSYV!UdI{h0inYQr=MH2!}{yv*1y;k;_)`){JTC`Ly zfg7JJDc-aAY1U8dm@aX0>f$dH-@!CHn3Xy^pEmr*kfQm42U70t)O5CBcKfY{Y^&6p zSV&pcQ*O%cKgi(cCNxr?k0h2RU`V>6rKk2wL8cumIoV`((Gr>oLRpmwEexroUKSO>u)iYJD zKI~~Vdy5@Jhw2kBQQ}Q8e%KO1I;0_qB78E?mkoqa)zDHw^lMDLzg}ql-S%f?)%+9A z?&wvN$ZM|8la~GU<|uxvXABSPn+^SQs;7;Ua1&|dB*~x4%VjK-6Np=z(e0EY(3xez z6x!MFGxv}OFhJ>a@%S$3NCndVhFpyk^xvAxm36|kag~Xco7vStbE1p{l%-_ptV8S~ zmhIz>6E!Om-kb>V>ky?fz=)iuO_JK3MH^eQ0vy+R-Z>3Zb9yoFe!e_y5 z>WdWneHS;gD}Q;=wa-)8Qe^*KngT=#H9DzmI%|?N@!x#h)#YP4*wWjDSbxNZj)xub zsWN>tke>QU={x=J*Gy0q4o$oEEz{TU>CMKTVK})bYs)`UQxl6XcyH|Iujv{%F17BF z%gLG~U*Q07Jpwg(1$c2xaYYz9A56z}eSy%k>m?uqa8RBA2dx?B1-yCw27iHiQ0E<6 z8l=#pQ#$}2?>jNU(KD>)N1JC#I=!^hn}NMlYzIXS=9EzM26f8yQ;9YFZZhA;lhBe; z?S$9XbjgYbEc@|SSNe$s;$+Ld$H*19nhbU(VOPAIxsU5N4(_SxvsqrXwBFgj#Vbu`m1XW=@7%6rlqm;t{xC)mSSuN3S{}k^P~E@*)r*s^ z_@YE!7O!K}%AQ~|&ZuHbA)j|e?~<-JPQdX+5RG?Scdr4YdYO9^=`lXxg%h8tSMke>MXwsA}brYTX2~?Fwbh8v>co0&o12Gxfn;~Fei`Rgmzae zFG?FLbCe}l)F$(T^!}xvR1-MovG+V2M!y#exzX9ZAU`J_kP$I!Q#LRNfER#zBWcZu+y%X zcV0~HN?TbNR2*!GreguHnvZzJx(LQvp|910b#E&%0N9lM+(+4jxSYL=q6krC@g+Bh zq!mIW7^sShw{{cw;Qu?3hG*VT>~B0XWk>j4VL8&FTPIu)zhkf6cp#6E{Q*EpGe9R& z)G7V3EwL}FWsUubv=d&ImiN9uO%94l#pd9-8^HsPu}S=2POdkP)9f;l4XVe%XemJy z{H&5PiDo29#^hcE{B07}N!HFAnydW;Z@o>UDKBCsr$p~FMEWgsw}iT!V^Uq}w$g*i zMB^9hT2}MtcJ{OeLWA0?kD?6FEnfyn5i)xe5ojAXZz>eaYeN6A2@bT7Ag|V4-4;px2p>r1q9J4A- z{AC|;wl428%r_>e6GAe>yj7ke4sfk(cu%Z}&E!FLt=M17=wv;=>Af#md_3TCU}S8r?gc5#-m_8t}H z`aUr9f6kROknKN#h0o@;Jo8JghLU^YydSVf{l0O2{M~XbTqpO_J(_H(cIrB9X>lif zUPTsb6+|h>=VklRkBAY>k5(7CW2bwq)rJ&Ak^PChoL}p7z*A>gz*j#t{!zMPe909X zZ<=66=h-;w%ifzOMx0=mI2bR|r}RaLz@5h?o1-3%%cA0?>-@DdmW@W|HE;QFRCO8z z6cx#8*5(Db1R$F~fmxhpnC9M`rSq$Pc#Nh3F!pgTiM9sGqB{Mt`y-JT-hx0HezrQ` z$rt|hKO?yPu;q_SSouKRop0q%C0CErrv6EV zUz)KoE4&-_2i(_RDN@I5e6-J7t+ei1aeFDnL}o)(Y_UFG;$-z^&kUU4F5 za|7a$iv9i|E>;!xN8NQ4X#`q09z>oFj0!@B!!PBcJlAeaEiZ^*HF5f6rOFCo*NhM1 z$_83t<;n)G?>iTVYgk0805#4g*x!452IAzyEzb#_*sJ5 zwCRQ5LBY6fP>^b0C-ZxRmvDZLfs-D8pkQ&c42Ss_cSG{v+|Ogn)t|Fy$~^$6ELHR! z`&G^=7u35Eis{g-%Wc+05XfkzAW$FX5XBi6&sxE%X9UGGsb$f(tkz6ET_4*2t2RmLpC##io9Rew0k`weP|1JAOgSb4XuF@|jrU zu(LDTyptJ6*(sb4bo<9d-N3-BhEXPU?vIX^?m5IM!Q3&e7JQ^1r;TJN)2!F(G?*n{ zT}P;vSi^?MsPF|N0!=G>{#Hz$Zrmt6mY1<&+EhEnZ5{#&?0Gl;d-rjgsFBmGp*9C^ z{Qz72FN!8*sewP&^L@4>kSZ8I_yddk-kQ?=#hv@=CG3r)>x;F2ACvGWT1!I@PHM~- zg(DA?#xp>dZ$(re&zPlvX(O6V)d==MW3*R5*PkgN1T}Zin=pMT z4P)MWD=@(R(=v?`mkAXq{N2lOGkC!nEF+#sZS<Qk3XjDtvu%V3uHZSuST(~rUibu_xDB6wP=u92nAnv+<8ds&8`OB zb=>Cu_jw3IH|2ttd(%P?=Z*iu)?zrC4}>P{MgDZ3Lf6@f(&24qlS!IpsvGg=+qp6q z>)_8!-5ZEmino~W>lKTqShnM5Z0@N$QjStS+*B=KO0HAU7p7m1bX@oldJiZAZm1o> zi)e0{n~&U0p0gxB2<`;vagb=~*jKWSv(7qSR3v#!XUk?mKytFcJnvRayVg98M#r7k z?_7Su)tbGNs~Q?Fi#a)Ih%T#}L6Lc*jpXuSE*YhOfOrO_R*$GkEdtr~vck{xfRr*R zUa4&ry=AUkSuftHO@)}>#Ch7LI)ag3ReIu*C#XMz$lRW3; zKKu1eki!{3D`u_<1aYtb4e_YY6mCrTbF-v}8L5NAbw1E)Ke#Q%!On<4<3rH!JQ>FgQIw)S2a<@xkQHUe0h0`eJqm6gyOHQ-4KF>A2@Bv^-y|-o2z<&XDZk>yOOp`^rt9Efc?A~iUt1zPD)cw<&d2s=dLS_pTCe%Oy4mYDsuRacsc(V_>*yxqkB4J2 za{!qi{i7^;yFBH{1=0RV&RczJ1s6#K=tL8Jk1QUfH?OTbRX>~D|4O- z^KA|C5%t>BnU7R`c;?D$;f&g4@l}Q0`4K`XpDCB>M}82t!chph%9pJm)UIH)tR(L2 zw_8qTyX968GbKrPRls?!g%UZpA8D8Bf;xL|V>X4|k0_lE37eK)B>ad+vF)Z3AshF5 zy4EQB2qD)mKu+($Pr}~u`x9>Q>vaj7jF&7iQtO<>a4mO=qKzs$`=NU7HgmF9k2-y@ zQD;B^4IpVFrTlw(3j^N*6s8tC6Jz1+hI*|_3^Q+5kz-z=C3URR z-VakEz056_AHDiO0$OBl6?2XETJ71FUXNjyFwyj;LzU{>C&uCW>jbt-6KS*8UVO_lEp^oWHfF$$04v(+gxp6= z|LqGMdP7ImE+&GaP#&^ylk3vaBYBlWc+<$(Ilq_iP@B3t1Y0k3J}xXeR-3ZcR|cf2wNJ2W>xIl zK2dr20)-AxEFSoV{YNf006RQfHCZ%YULC2r_irHZ`%|3LMxEag|L+c6aKIyibQC)6 z<_5XT6yprd3iac{S~2QWIg~7&*d9tfPjAB=RF?xtzy89&P1Q$vPGrBan*++flyX-e ziJ&$p2`pa=r|$(UKXeGZNG0Y&%g=w2UfOVc8)xk9@HK`tz98oAVF#CEp=_iZ^Y!hg zD$s|dtzV(C-+OmciSIZ|WuL;l&x=qk_xvYb)|*v0`~6sZS-)B|EsVPK0iU^796eCZ z7d|zW)DvH5-Jd2g=@9Y`@dH#qkO{pq0~3X@!P$ZYT51Y8%m%xdwqd2DP{zS5Lc*D6 zV)a$nZywfUT~f*|^>Ce1`J@>W-x8W;7}xvXCe)}b+FTJBXIn(v%i z^1J5O&|2EyGt8g(zJE}TpcST_u={bq#3NujJQVNMCezy9&pcnIz^eF0kzY~r90Oc4 z?_hLY)O3D3tt!-?23hT+`y7RC>I__%27%M?zCvQy63w9Z#&LE708{?A+N~4c4P;=@ z>hzJgWQi=UrW{MTA-YfH%bopvzfcCa9={$8)o1%zpR3H^sH7GDk++#i<9^K4F8f+e1vogCbDcZs9E zYid6VWzn_p>BdJl_PbW(WuZ6yToW3{$NjES55Br4leAX%BL~+T*E^j3FzMH-lp#-Z z2~}_0OVw^Iec)^XLNt=y?A+ZBuGH+F9LAo2aG=j;Gm{vbk(Mjq%e z1HO+!giMdm&8sypM z-nDUI9^p`ZxF})dgH$)ce75Lc$4YqMag?F8cBC!*bfS7|as7A%>hYt{${U`I@5|35 z6P&Ka6>YjOa2mu`Iqo_`&K{-(AX7c^MD;6&xnQqS}M8q4CrluOhqx6p&s{j-%P0Xe}Gdl z@TCv>{%_#P?YG3|;he0Jdus~{;JT>z^mKGc2*2D0A^yti)W}(n!J#wJL|20Qe0S4w zld_rd#t?QlOrXI!m{ra}BzkO8UW*>M@YKcto!4pu$9Y{cn*EX;?SDI(0Y@$#RGANF zj}84re{!NzrnOu?+E7DfGV|NZ5_)smd4VtqI8`>=niv}WkIi$J!tRf5$(}~;^=%k! zJARXB8i!CnpZ&VW%hAUwrmCOhJqkQ=XDr|Dy3;Iou2%0P#!@z4{F(vLH&oyN)^w<{ zD7Z$xXGtCtQ0TH6y>h)Bv4N*s4JOv(Y=|c4vdZH~??t^v;Dt!5WdPTmZ^swroj2G0jd~Ld${jye{z^_Z~`Y zI1JcZ7UR6^V0)W-Lwc%~sc1l1#CQ^9^e?p7k0kz|l~o8O;Q|zxY{>9CL3(?pGY}PM zZ4+l1`B#SlNG31R7|F(ZhzJT!Qf}f{v_=OVgIR-kPZ+vbq`5(PEnAUCr#U&S=C>CW zdOD4gvAVOM9rzJp9v_5vHHZW0Y$oQI#NG3ruN>Jy;@9t-Ieb==((E@_xfOM#}8$a`l z`4=pl%~Yj~{p==|M3cEsPy8-xjYRr0@U@(>&3a``B?vfDC;VUx1!Y;x16f2Mk&P8? zr?4LFi*x!3R8tc`)>6BF9ODZFfQu|RmBrq4^3NfyCqzVV-yVl|ebM`-is_+M61e?u zLRnVl3;+ML0O&w+IRs5~y0ACOHcG01Y!&EX_PKynijU@X5!(RyTl8Cfq6c1%Cf<5N ziVV!tz9Q1dn9zHSCmkc@TAgn5V-d4e5~zm;_jI88qB?BJEFc^+FccHajoan!g#*oc zSA}NpS(eUhIh8Ci;rn#~8b41*F`cNdO7@IzULfB4OYD>DkRX$pbbTB3$f&G5^;)I! ze0#fHV!;Wo&(T_Pdh_dtKT1*x9&qe&Gp| z0RB_0ah*N~5=egAJ>KgBCMZ2K2tr|Sy9Ky001n1|D^8JO3{<@11HfK5kFC?u0MP2R5D=yBM;&PcJHD!_d6*vw z2>35A3t#VdTpu6Z^(1)~#rzA=<@RUc^4!=jlv-fMEqo(yvzcQ}u$_r8dID1$OzeOw zP@c&23=6g5`scJ=l)sHwTU(6DG0a~er0kWNH2}s7?W|JMw)YS(7k+Xv;#mw#%>A8? zyher-0n@|}i0P0eS>wA41V%H!@do%^w2Lgf0WVTc$w9qZ)yr=q627bGZzDC>CN55TvJ3e zRApblPGzARF4$5ZO+k;Az(3oJRZ-vb{aG`<41u=kG&+^)_2sbIhB5jO0@7K6Zw~DX z#YQ{>ob3R6%8pm#GajMeWX#5B$&^RDq&*g28uq>P_Z_1ZSgSF~gW}+~C=x2y7-oir zT)kOVbvmmY@gX=3iA-=H{^xODq<_)(2qT~3nFWMe8*A7Ev@PcvK?p#YHUKE9RxHgq z^ax}ZP_zH_gBeWtx&s!P^ZD(gz^D7}{~6FPkg)plIJ&+VCYAW`eX~yO@>+f6{s8tn zR1cUtgKV~%ZaGH<$w}s{9w7ZngPebSDn*5XTQZm{pdFW93qKR+Z5wR%Ga;v-YT*LI zSUwQ)<4X&%idtq<+t|`xhfDmrh~iisV?yvAE93$pf+)fvhE|(eAHSTKX(>w^D>gzQ zE}qDto3=`;mVPIujkOw$wJnSm1TJS+7zfWO;bz7fr8nuzX#dVcLjR!f&ZbS8xVi{i zKMtaHz2rT>EJ>3uE?oJ)0M|WW!O{zZ^UlFz9aD#n)Ed_?$>jfeT<7t5keJD4)@^NH zi|8|xHt{wgaNs8z7HY!x-C%I40D^RFfroVV%_4_)p+Ri=qr|kr>{QmzPk7YSukccE ziXtcg3zicNuH{H)MFYd`LD1a$>qyfkYOW+P+AQP}x_MNlG=j8G_eSN%q)8e3W7bDv zgfOR`yRTQvSefR;EnibA7#o$FqT1}0Y-gYe`n`~C8xgH(|E4{9fX#%~mV;UEJ zD}J2m;;RabV)_i1HuhW7`7ev=1GOoQOj@>) zkU*^ydDV>?Kn&62^v*ycoFxwg&Dd)z?TcMd^gcSw=d9@QsY{w4?O1d^UQkNqCCn~wU-Ct`?~RoOnaJs`zSlQ6YysijJn&6s5g3W6tA*xlG~ z`Dy4QNB}?f`qgG+hu?*7=pcheU&NTArq+3<`EeS=;2F*_u13@a<7^;}iiVEE34JeK za45hXk$p=_Ztuvz-Q}|7$JrE~q3|ywI03l)hNtF7K_G*j-D2yJj#v5o!ls<#@uEl@ z&JVw+hI4kCr2@UUYEeDzN8U^6wCYm00(vXJz@nE{`oSYpo=u^<)26mOiHL5d)QHzi zrB>%!U(^6fkXr7ou*}Z;$+_?QJuEZOREXaQ5R7~xLDm=s+H(QVPpzWgJb_<&-jR3V z_tpJBiu{f>tGd3ytv;^DYg3P=1t$Ab|0wbRYE}~o>pJCu=&n|rXAh*=5jv;yIdrb! zwEu0<@MB*n`Sb4e)W(th*w#<(pwn*;S)`8@G*6VDiKc>@D7iz(!lWjz$5u5YV`3D0 zfYo{;OKJ9+mw0<;Bn8st_^tpMDx55`nos}Kv^|^CDrd*$k?qmm zS&1{PCWsaj0Lx!Fc0>;N3S*BW?AS9GMJ{BvZ2P0HkO|f|*n=}@ttFn>GmfpTg{xpS zG;jy$$0yHYjGKt+GrA~I|AauZma0-F6-#rGTy`H%Pp9n$px(IVY(XIAX>0)TT@CAq zo-ahI1Q6ejN@d+n+v|X9PmSQJ@4+G0=SQPj5^w*_=KpdV`1HG*9xAW5#V-t#zbg&S z+$xI;(!65HraQX{c-YEg!F#V2oAqSQsbT8>I*z;BA?Sg< zLZuV&Frb^J2Col5@6J~T7wr(t{>r2#%KD~=xy=(55{NH_>$^_MzaM!Y?XIG0{@vUB zWt|WA4@bL&N7Z9EBT4`6?gj=rY5a@6q_12f{J%99OXoU%vx`3nV$gr2mwL_4%kYuz z_a7lYm*GgmoJ_>$Id-J-tk=NFH%!w@$f_nnOP(4-oKhX-R#YFfW&r#~n=O>3LqOex z={Um7W(!#cJo)6ky(tD`6+FD5qbp0 z`75*EneZVw$_8o?PX5+yp57Xs`wb-@bUw`2nD<$S_`-kFf-I_Kas(RypxDgfi}i); zbl{?ghaU8R7LOQN=XKmkioGD*ODx5rd&h@|^}3F~95o6Ab3z)DDtZLXDU^2a)C!a= z#>W0#gyO#cEJ75p2r=ssm9poAikf+Q>5#V^P=UKI^a`CQEW6hJ0bBr7e6kWD4gCIS z@H!X)WB}&o!Arp^`jQ685x1S#Uqa~L1izA|&(Ox!EU}*W1@^b7o|;zWo)&_E=@(iO z57oLmRSYh7pO&VcE(5v1ZMCd`ojarTY}o0w!8$V=W{~&Q z{HN2?%dx16eI{-W2w=wl+c~1Y6lmSBCJ3xjbL*$4#ZEJV?!Vxae=qfaNlJQ)kgy83 z9WD0xjcP9q%J(-vY5DoFW8m~5#`{f*@?yVIBCdf7wBbIHT>{L)aVkWHOb<}nKOSjG z8lVG~oHRc{q!=HrzjO>9F)vQwr{JWrdwp6q_HS9cS;PC9OUhU|;pNVz+W6m@cR14O zauhqA2KAO}9|JP#jRV&!YF}R0ycCdLM6Tn1wNL%379`_TX`NorXueues|V7O)@sw8 z>u7t$5Ah5UJz7#)*XW`?j;Tqt`)_`w?|2>V>EYkavUI@XF%<@oFyXV8FDd<%*ho^k z5!+NpMi`7|_8c1K)kdO}+y3m&=^|&IUs5r-4&27#;010wi%V|FAXO5-_P2|qH!HHh zEtl^^sW1QpXYkazP)ya~vLUvS{>h@lL>i+9w+<@OvAGQomX>lD%HW~YK%*A z!uyu0fC->7v~d%!+Dw?p0MFi>mc$3lQltV){dig<+Ao>R{Rm(a%d7(@C&Zs9S_`Al z2s1n7bR(E^786=2cBuoRLe~Vpd8rSk{7)2_cDYOiZ<&`nYx9I?a(aSi!PWHd|B9aN z|B9Xs|HAs&+*AJxAlSjd#W>&~E*!zc^x$b1g-mlw~rmSNyS`l*@aBBOLM)B`F=N%P^*`_Hk3- zK+q4n3lZhG06^}ef1VQ$a*9sdL1ey6j2QMYktPZ9*_5D+rBDyc^^qqx19Dn(Q2)a6 zl2x<75c9&oOT8f*l4;Ag5ww3%V%C`^MpM_<0yfp~#MNZP9aQ=Xo%uv*D%IG^EEaju#=Nu3GaorG;OA~ zrHBbkzyRGGo!_K(H~)?T=_K>Rf&Ft7AB#3! z!iz8343z139Y4`YZx8%PovCs+&*6J>#Z=;&1-`0*-?>(L5z&GH<72+A9jqph>4c7e z)aLI!xrIO3`A3@qp!t7f`%|;LFm^LiX!)$Tq1L|i3;~!PrGoU=2gcj)(dDnkZNEr9 zX}p&b#Mw;=PJVCQk`}ss4D@&*{$2{Cu$y<;DxZ6kFk{wq%bi#7(jnJY!F$&loku^K z5KBr*atMQia{%%-4jpe z0BsGPtL^LuI50r%sI9(uRBGB_`q>=f>pHUReI1G7yQNE=*{Hc_;4yF7AD5$ z+wspEY)t)9WS|$t(_;)^isA6BQ~3-w4*B6sF)1=9zpL3vlMOL5#|RokLivj#$&5^q z{11P&YJXA~-I*VMa*=4vk9SDrb|2~a#E3fb&{-$qhoa7BSicA-cW*zkg3*;iWYKt) z`>8`T)WmR4v*|62Iy<%IKcxR}w@hs8WT+fyg_8gGf7~+0swQ+<_d}UiIGKvQv0D;l zsRW|I+K^P!MfZxi1E*;^w8bUvxvpC~cg%GCfXN zb02=i#1Q1seBxFUc08gE%rcfoA|jtkdQYHM1%bo>bAVGIaf6&pZ?o575W@)`0jbO1 zTXT;NXKro|KE1l4ekkbpCZ_p(9VXN9{FQQI_20`Bri2iEu!T~-V?g(+i?WG~DHhB& z8286TzEFMSv`Y8A3NYZ!gE?MBMRVN^@BY)2(r+sofzGyq703q|vM>|?kwA?=OM`Ki zq66WNqE5&De4Aa^I$Fe1B-kPemV6M|1V@VUGa{afo|v<;Q0db*dYz+Di398a7Zs43 zUhcpr9ifMRGOckGE`+Z17V$UNExDS*Xl9OutlyV_72l~8B6LTdmE^{gNKI>flSVE^ zQX#VLP210izAI01%D=^jN2mse`ust0E_xna?E+gBpa!e+!VE}Yv2$m+vo`&AW^9O8 z8#<%e*O`}@93n=Ozfrn*cl1nIYowjd{8jTG?64oyjsRakc@L;shnd z!EiYQN;kKfgJK0AyOX59BIQB&9ck)HGe9XUbW8j1nlu;O3GXr-}#vSgH&^}IMDRi30`cde%Dzu5} zU^$4G837HH{(p;0E&BaJPaO1f$6p&-kH;kw^l}Smc9RAY;2f2IcLPxd@Jd;>MZ!*M>oEUg~U3kP{k*W#3vFttL-@% zO*o)e?ChRKK3}u^zz;FZW0E{7$A9dwU3|4FYb>YG2~I`kJ78w`*e#6zN%Ms<6Lj80 zjR_E?>FpOjevqkGjxY^OG`<&NFxj)H3LH^-3=n4bc28l4bnNkkI!|UbI$iU zZ~p%mTm!t};*Ndqz1G@$aam`w01`I;t65}RPdume)f)B(@@b_^n3WUc`7DRW#j}FC z>`W!6XRvzgYa;>w!0&}xKj#R47!;q~UXQCnAo9`N4f2`Kl$+i!>6O6D+q}N+S*lwA z!?mhO{tnmrD^i2VMeOmaeE`&Ip$=Ds|BF-hU?t-xpZ)r&riG%v{-DWlHTiw?n-4|{ zR$m*6bv!<;{C;MThM{anX=`t5IXh6D8sPspo-p0x;|=nQAH%V>>Gp!fFnhAM?%T%C zE1QoSHk!5*JY972o2U|%JfR;gYc|C7v|oPF5TbuZ*1j-zW>1Gh++M+=hvJJfwb$U- z`V+c;dcPth_4Bn6PBu88!QL&lxkYeG&bSGU_SS9Y!2PMpA%tIcSHd>-SPp~*)W ztI40QzI-CEG8Lqcjgx$g!sg0*jx6v8ApuD52hMzl=a4x>-oHhqe{iGx*vB>0Vs>N^ zulyGW5}g;PEzp*0sYIgbGG=v>2Q$Jgs^2SN+)X~FynFQyN6`DD$5Y-Waw!(VqyC{2 zX8+=&geSvl(U0f*vu)X#V7QNXG2$GD{dcbq#c8Nw4rLzR{rqYC8m=~X00>yNI|qJ5 z4r{RyL2bEqOI&O}%ES#8-jjK`oTr)^3?;Li9bWrA0*q;)dOY$lR7#o_^U&0sBG}P* zKAypvq*hpw^LFPuIv({8z&D}x{uqFB#ee%!EPcl{9NPx7)SuNXip6tG%rvn{t)F(6gx=96_0{O|zQS=+sQ9z1u}%y3dk z`+THa%Xxs& zRS0P3geOIYKoMiba{i2UFi=JV?$YUq+t8S&yf%I6qkhEs9c-?~iB-4+7wST`Qm1-x z)FYE3o8V$XPY#yYr=apU;wzMQOMWb9Euw$wT54z8Qr8}aT2vkL{y!J{Hw1Q~`!*u~ z!S(y)U18W^PUIootcnTJIky+PIH_59L2Neq@bo{ejK9W!GT&7i=Z~E+F~cz9BpBE% zDiwbA8*3Mg!W06v=`TJGDgN$P+h6=bzh0=>-Hg3#-c>`vuhu}-be3`D{&~xt&m%w+ zj3vr5Qv?%nHD>kU!H!~$6Eq`0ED+T06)K(Pz&~U`drWKwm*wth8eCVNQtthkvL3z> zDOVx5X}=JBSSSPcD!G_m(sYDuH)`|hv$Qgi-S$>!@@QG4WLfM*YQ70p2Fuo{88%Qd z+YDIUoke+wxy+X+&GsRxVh?RK!A6-13I*l*b%bU|$n{XB*oSbH8I*wmV-N(B^-AtR zQy%=HD!TI9oSz5dKnOo(?yoSDjNrF0!Dzxzsy#U~RHJmr>sL>VB{(?~JZAYA`f6Fr z6xODAj85i%Nn8~!MyBRvp~aX)SJhF=-R~f+=TyIFufvrxGGh|GIZjsP0?@blrn-26 zSgJQv$11F%B>JA(YbKHvtZBNg8D;+c9$=?*YY&gsbVue5!iXX9f> z%9Vmu2;MNhR5mD}Xl;b|K4_$S$SSTrPq1_zTR~;%lQ4_vWaq}5GnWr#N7sKkGJWh* za@y0=6FvI1!A0Qh*y<9OkEQoqUK6nh!>Gb6UTcBkLg`_w(e0z(`1cw@IdEZF=AX15 z?8`O=S8sVEcpBHnj)IBUU*YJlNS^JSlSX$)b!Bx2a2>Xt#*~+T<~9%{Desy4q2!5& zI?ROkFmdENm6`yfLR>DW6Fiism;Ed?{c4tFb-zoR1CP@-A*QEV+@W*anNO2N80RK+ zwlAS&Y$@01X2R4ICZ-5sx;?PaK%$*%&U}Ii5z-?~YK}^DJ>`gmHJEx&>N;{@_Z-Ib zl$1O}(2i?erwW@!7`qU@jAB$wP^&j3p~ml~6^g!WuP{+{gUMjS5ge-qrgzDWr`-Z2 ztGM-_w%-{EGf{ajG|m>Pln%YA&lypqK^?1*a=5R4`7>PZeWybD9>+%#VU*NlYko zPe7N#PV|OSu<9_lXl#k+GdSuwHhckw%uD%=@@)CKQN$o=yf!uI- zKAOI!1x0_o?8i@PGmt{zF;gN@`)0=lHP0P+5z2CJ{giE*C#<}5*aJqm$)>u|zv`>> zwj^F~6mR%xFFVy$%59{q8Aw`AZp&yaH5Yd6aCfk4C zgMVKKJ=BqgUXeM#+_Fr6sKjHKaCQ8W;1(5w zaGn6R8a9iknMHIo9 z@L4pE5)po%c1#OqX$-CjHDCyHgHAcjMbu^h|7D?Ud;dyRc(#4^?&?bLc<`(A+{K&1 zyy_&3o6E95q$JWA8IV}}f7ez8b>!Pe2JY3zR@Vkx1}UiyWA?u%%(b~a{>3V*;?zf+HlIts!Z=S7xcLm9*r@u$b#A)@%AEO{lesEH_QO-- zaLwLm>!Z>%aX;(F--KqBI+_TT1;{+?ZwpD9&m4mBgK02 zRuZ*+qfJ^Thv@0DJ&&h*^(|WeNsxXNC!mM_RysYHMM0GxPxpS;>2irfDwkvz(_Ir&4aR2SrmH&csp{b8mlN1dsj0fB>-WsRCu@IGbiSLrI0eS=JW| zxZh3Ol@T|%-;sUE{e}8ADu$E|p`L^^6^?T)_PC@MuM~{?m~3N+D%VnDf-SSR+- zS_UMa?CFmOi->;vDdd`TjyQPj(r#um@6$1G^41sHoI5K5U;hH&kXJT_Y?N7tJ_pdUXz$@6!+uHAAvV+3BCN}dI1VVtdvV`PJJ<>KvgBM zd~r#4uj?y?vQYq}Kr|&pZJf4r^4oQMh^3~|2U!T`Q!O?lqKIydFn3m;5`1G^`VK*V zS7`cmh-s^LG~5*_x=vykexU>Wzd=ujH~z%a0@`JgA30XDY0D>PYgp~;%vkho!L zK>-Hl^iKj#YTPQX;s)0QT0I3N>1sKMtZx^dKIh-ZU|CT~=Bs1HWZ|%0v)jiw9N2y5 z!~GoI^+^cB-%5v?vQ&?WyMo-=rtbJIOxqt4BqzO8&&Y5%L!IJtfN>K;&KOSKM0!P2 zj+~w`8*W24JN0-Ri1%s3##5NL9V00G<)fssCNI2*nK?W}YO=H*-FkWX_$^ihQb|L*>!Gjjv&Pv48y{E;3`@4S z{2OT&j))Hjaldu~1_p~3x(sRmlcM}PG@GN2sJpXooGZj#jjY)q82?bHXg=UFzFLf^ zWrTOtq!m%iTGQHp&yEq{0z6NnwfafY){=09f*04rFFMesP$|GD^CxP~of(y`kQXA}>AR(KPyUQ5a5KHwNsu0+JfK4*KJ zuVCHdxW7o1T<@G^HIrC7*L0)Q7kW+d^D4&u*JTHxmLLI3lH4d%4HHD9{iE*$Q%W;ip5V8HZ*sFrB`T z)t?d=J1~%1H<*$kpuH-r>H5)75|2rBNilfd9A-JE9to|U{;3g)Def+9h_S}Rw7U!K z;Qlhy1n+szWW7Loy})fooB6qDk`X-VfOgh0%(B?!CO_$by20Fp2Y7T&XEBI?W1#@7 zge!4MYaQr+*LlEYk#FU)Y~G6-pE%{)BNluRdKc}gY~Y3?tzR!6Lcdmp(I!$3_wC;0 zRthu*&Aij-zL_YBUA#xtVZ~%k5S?5&CE7f^#;RkzVdeVNc^oi0-#bwsvO*cI_s1zC zEtfnzySQ*|r0Pq}`Q(eawg=YzC1LuYPQ!QE>5C|-tAKyf-p+*BPp6M%d((E!Iv15` zJG9u6yYl*2u<*k@Urt+PBKq9-TVW)DDln9SuM!4iL8wBTLSeAORw$ZI2pJD5Si`6i zQ`(p0bBPNW)k0%U-@j8d+4ZZKTF5A-shi8}-Kl3@@akkp*-J>aZu zJN#$j!iT{v*L2B>Puq-o+S>qGQkxhP$G({E(d8)mt{*nuX&keku$`8d7F(ZCX1Vv1 zwQ|zOZ{C4*pl2=_BMX(~Y}fjpSz zU~VY=pmv~@*n8_4tMO!vPdx37V-mY)FV&5jqL>i06pSf0qYHSZIUmD1H57CaPF95t`b}$1E%YqF!+1!+b*sQL2@T`I&+cv{rH>rQK%T<8U%HfZ6K&Yg7!! z9<2WX0DyzhHLHoLe$@HB=W_~}S*UgBXlHM|LSOsSruD@P(T`ATxyt1Z-1%mj=dn`Z z*Sc66eGNaJFG}I(1g%W-U|_dJUpfS#%CRrL*%>%Y(p|5Lk7|^}ERVyGK9Sb|wN0Od z#q7R9DcCq}(PZ}7l?%>9bIm*74b(N@nty~)!IHcw;a*l`d*G3hBDD5(?%*qV+; z$!WQc`m*|n$ioU;)rj?o+E-9MtIV*@<75}Xxc5=9K{_fhq5)1pw)H~jEKRCj%oo`% zC=}Acf*#Bcf!%)7q2wA}#5VHvP@hHjNV|G$Ot4O;tiA2kf@-6GRDf=T7~CTUA#oWn*~s5sc&ZbU zQ$@7;6T9qsoc)G1R*xS9Pbj|MIth+Zm8_ZG1hBEQTlYdG<~m%6#|}amY1WY{qIM0@ zo&6K_%W_nLEDedOMmL;#`iJjkDR8C=Ku*|ohf3{j^!m!`mgUZcJF}o;^IjR+28ga4GEGuAow_(^8 zDy%p+BX58dt%`zctUp54#BrO((MV%*=6w!TznfYu(V^s=CwQHzC-Wpd15#EhzOLhI z2t_lIVKvYhO704)M%O#g(k~mz)EACutwWS4nK+fj?He1Xqv|RTPf%#APM+uCj7-#5 ztqLs_QoNS?oA(xGf~9nVdX*!x#TH0jj~-NwB$eV+|8G;T@M;(+b6D5G=4d4n1PxeI zuze}sA$={ux#yi`7Qo=1bWL}o8A8v2$Lc95f_vgAARh2bobX^dzF&8xZ$5;P#b&&- zztl(~XlV7EXUs<=Bv;gR;aw7)m^tb1K;2W6vRCgyW$`5h$g~Rth>TCv{P**U$a$5F zem${eykZ?Y3S@8)m5JQg3$pItq(YrTOY|<$*FQGc20EtS^m9mEDml*ZG+>r{uYHdL z>_dmkwAN2

oZDd}IJj>*R!GQXi{yN1WC@+1ZK@;!vU%kFK0y`NhC;_*1i1Re*Tl zvAWChHG4i3D}FgX!wSz+R)m6Zlpt?6Lf4yFj*k)V_h@)k9eTWVq>811E=d4C>ENqJ zv&_)lh>w<>t_$p7MO{~q*R+pv@#23Ghh@q0p@Pc{Fv?)G?^RF?uo@pwX2Mc5ATE(* z+zu_YH$Dc#6X$B3;3gTa2~WV3>pAF3D5 z%EKEasodM{sAmo(@yo!C`23=~PA*4LjV&Pe^8f~p_miM5qh51MyidP2Xl2Ya9S|rp zPZkm93)z5sv%zO8DF^`OO$c)k(l5m54iZ6Bcn8wZEY?n4TWkIMIiB!wcE$JOdHmj& zc$xi2K1D}#Rq+QW)|s(sL@?tjXgpfX zJRILVO1#5O$mCNl;kN4T37{4T=8q<#i??ECB^C^PgsV0J^cG;ctk%-J`c^p&4W7Z2 z?F2Jg+94K}f)17f0kir}k*jzL#7p3=yeYl8Sqf)*I!$w~N2~Q-ZKFV&R@3sUP@}*h zO-U$Saey{ej$&#a@)bJGmYtDL7_-kUXo-Zg~Cp)j(lSZ84Uo>^gR7;%?Hr zcCs{D0qXg6v=ACNx;YlhHY|K-SqQ~(J(7e%fFi}G6$?ftSj*H*nm~Gb(zDrbn5~ms z^O>|VX}IeYP32cOL&&kmY=?i7NoTbGXJ@_%xzBD>k#(jxHMDEzwSx@K;$@O8tAEzJ zvi2V05`Ju0sTcA>oU872n{c;#-~P6Ek{H~NW!B1SzwB)&2$w=J31h}-`c{59cYM5j zxsgDyc%1IO8okI(8QgG+_b6T_62gQR5y?$0<|?}+(p4RLMi9(51RCL49c*sqbSEyG2T#Tokctykt6}|KbdO);L7S@5Z|3PBbyIH=+cN@ZKEWeWzaWjx)o0Y$Hkg}ONPFZ&o;N2 zJcbQy_<*7|zE@koqr!mPR6M0=QoFpyM8NS6dA?j0=Rh(3$UPQ|=iMewY7~lH--7A1 zM++=1D=Xpp)c(^?bIewk-k$yh&%3K(OgvB&3XHASj;}kuMvJY)u6q|6^SOy`A2SjE zRW$q~9ysZ8%|g(J;27vGsg&n<3cOnB@5D3~Nk8EGjCYP9C5gK}@7(SE1v8NiJV~oM zavLP~x4P{Bgj0{C{XZ~6(=|oS_p2+L{nb{gf?dv&^gW$=;_0QDo09(8_~aoTOx7d&Aoc!2L%|2-~=r+-h7PHuiu9qTAz@Pb<&%K1) zIKqy7zG(NO!ARJt=cKr)*Sb5?wXm4Md~@4=Jx1<|`4yYpRl_u=iuKv=J(QOBj4(8} zl1CL!e~5ejN*yv>u0Qa8B-zAB=NEz#(~4O)8CPPJkd~c^5z|Ur_XS^_a;&Q+wVRrh zG=hHdjph1X6gCY*#5yxGwPY}QG(KH&YHATRtsr^P2)N69t9xM7#N*eM_LE$KSM%{v zcvS0saZiTgah1KhpSEK7oQo%N=ZZQ|0NONsVp>=;9afER=Kg?u{)cQu z{BHoi^P|8P{|PlVIa(bC~7J#zECd@seRf#FxD>#+EpD!*Tne+L zIdF|4m>VBN3E(1BKWZFDd0SRFY&oaW!Rcu*p;K%>7Z^X=L#O z@NDd(aSZW3tS5%SJwN8}K;0wsON?uMH;mumGbRfhr6?-Dpd_T9OH$@CuQqhcDWzZRw>PBv^QSg&+DY%}E4|@2E-Ou_QhtXKqj2jVpGrIo)jifG z7=k;3=)UPuBxF|hHGA)92}|Yc&}4Gs_E&5S77f+S)g;{6IBxJZ>O@B*>|~R);P*Gs z4&j>xuvVdjb^}g=H6Wh90IFRGynk7~EA6){q2x{DWv3gw@Y?9^_R6AhY9@F*lo#pj zMEBa|FXXpN^_u&1=|3PU4{uIOi6%MHFAk$U*HfoYIC*-$X^b4-#AcnmonvCNcrLH< zikA2H)+c^BM)v-aCp=FT)U5G;YtlueFAg&Eu%PFxBcHvxG8w{gKY zOmxuE4Uf*NlJ32+!91B(%LBMl&m=`T1{10;t@*(u7C>przA6}&f9v)*P<9qg&6naC zox8;2Wa_XVlT-?nzpK&B8kr<{83um|(uJ5UA~u{F*j|5Z4N4TqX?inJ)X{L zT{lqkxLV_NS!u>2tfGyg@;>RFOMCu8GqN2@OPEC``l9=%9hbI_9(S0tla6#IpV8ku zZ+)$Z4Af%pkl!D}n4}n9kInubExmM9Njpx%e@G48j4GPNb_h5IuO6ydYhKSgJ%?gu z8DkD>M+{FYYHNKG@M%{7@}9*EA+!6=ldD_`g!JFXxbi~C{(INi&ZJ2`8?nb{!chfP zus8USN42C0(5bGQyWRkN)H#olkJ&d@AY_d^xAg+ zo}-TChLz*)NYZ~E(rCc=w$Wz$f`L+qyU~riL6I$X^HRQt)=1@G_#qNJBoyd zi~eLm7{}%A;sTqw$|4{-n>Fv$-`>fD2qX!^6A=;pyT_iq@q%V}$M)lcHTJitsG4|x zzIcfL@CcX^+xu48@qz|5e@I@@wO%Zl4g<*5)tORU{BV=`KL*;$-a^VkuNYDy)K==Cm@f?aWi*~rP+}%fRzLgJoSw9b}RJW^{R zbla+PagWvI&i)+gI=XU|q`^3gmD5&Q2e5jySG3$>4;+*5vy@L<(*rqFUN9S(|;WSH$s>qcHfy)^Z_}t zb>DG!kBa3TLSe4mhE?=nAxX1wzbmd+wT6Wof7o$nuEv)xnO+K{@kXAtoO7f`ioN#f zOze6uzx5pP@`&#hKRzB!_&RH}%}E``6Tc4$FK2%VmZ8AvZ2Lc9;CcR=K(M#2*_zfQaKO6syXm1Dg3q9W74h_MD?9H zL=m+rOW((b#|QqgO~i(%>VdzsKeN+!SFlRezPWb!ULeZg4CYbZ>9Z*I`8tYVpA$^XS2f-sr>Vtbcw6IEN5 zBSjqQYW{y_y#K`(8ow3gm>Jz!tVzpk756CJyB=?422$JXvy1ytd2ntN%9E(;V~{N> zlMBjj5}5@Ws2=?lvtsqzlNJWu&V@xKt~e}9C((d+WMOPI%A zy-JH^ZM9!tOP-%=`=!L4FG!uPLe#hU1C3bJmA|2qI)PQHjQP4tIDb*>QO+$A0N9R% z0hi{z-1ZmF)|%h?BB#h%83mC$k8RevY&ubYaHQ59Z{*H$K4<&kVn%L; zfCh#<^qanqv%e|=j5ZXT<8sH~+DIj5>5+K*Gp&nCtx8~A^{rOoUslxU) zOY}x?pVnuUJGvc`BPQn~d0XYSD3|qZr~|68-TUE(ukp9#T#`W?O3? z3y|Tx=c*B}KpEFGeTOk@hpo;%04?{q`?J&0R2}3lJyq|gl*wh+wgQl^uA$%$3QqlH z>Waj*o@_eFqw>lZ9I1S8C95-kwu3vm3!VU4x>DcXGm>%86SOtQ+phxI=UCD=j7y!rpDM6AZ0ymky$+qmJ`Qm)S}Ypy)i%R!OYZx z!RS4i)e)Nj$cZzw^EJcfXWvP-V2I>5*H=UFm>Oy8nFr!|`C(BbS-o)RlkX$_cg;fD zyB<9%_Y_8kqdtb4!7fIN$gx~;%SIH@VA5t=wNUe`{v z`kO8ulk=n+J&O3>q;zJ$()k=tK&`s8jNW`Q_QoDTo!^R{+zS8Pw5CImih(*$naUo% z`N{J@+{=TJlG_9a2BUqY9L}vyKm$WJ-wE6+B>)}*JT{Fe-ASfN+VU!#mfbb|7S5CDGBN{N;L!*dmnr;K&>ZaN%C;>|5&?G#l7DQhJ9NlECjKTj@4j6{R z^t;}Z$)bSCnn}?3cCzSYQOJ$f(SSuGrXfK~tMhyEiSXrMyBm$f!K^E#+z zS)Yw-?@Tdf3dg34oKsf(0eCyi?*(y2o1Php9ttkXyaTHxmb^kHDC-7_Nxe@C_LcTY zGN~op(nQiWFc5Wp^E8XYG^G2t7QmTl<%dvbwD0f|*VkvSai#JiA+s$7MSCI-lIuH$ z;go;ft0TmN|LLawR}KVk22n2TLuC}NnndMi8`EeWLZPQ?P3HRft3fbzL_hK@`W zJeKRz8wtoGW|yf5`4G;3@BUcmwSs~I1e=iX@UGu<9XGLFcle+1JrI`}a1HA^cT)gW z>9`61VAL@xHhJ?pn!?{|8#7 zj;g-YLFCySyDCRg71wl6M$#Bc`iq5wRn!aj)1_wj$&dt~5#xD#-a2@^bFEnU{X^~J z$z%@I;6w@6wRC_QPH`jKkw0{>&dl6boUCYsM-DX}bW&w_XU;oYbwNJ`Cg=O(ne+haB zcZBq~@L2LabqARe96No$d?ysfzTB~EjQy1 zX98-ccY0;x&S4YkV%0h#7%agadir-_Q%>)4h0VQB#b)-7r<+98vfP7hKGN&wDg>yG zdR7epKWltHP+O^{H?~Nn^&8o7|Q2Fz*Z(+}fW3SYD6_fj=Z_TuK1s6Q*>oSHnrkJp$+8lf zPpAYqE2x6tarNo&lrRmF8z*{`^L}L-R`?CK+{*&8$SNaCt)0f$9ChRAUf`T4Js*pKCx<_x-YR_kR8QR9<-8g|a~J??((EbR?w zU8FE9(8KywIxn8k0JM4`13;_SfD{2<{UcHV8=d=p?>hjo8sBfh?+Yh7WtKUzon(RE zT>K974)Uh+mifOjL}=L74+X|g%QnZ)q!)1(OFl>^6jq&j@z{&xBI!t`gIM?xmL$VoPq{QT&v-MvkH(4axHIA zdGfRNum9oK8zyw!=Nk~Ia}oiy2C7sc)N@LCx^+Rl9Ua22lm?&`jOXqxZ#Hsg)*|7c z&p+$mj%TL>Z29zbM*dDj%vfQztLM(KSWj8e6%q*34s8An_ zLZ*Y=mdgHk0iEW5{>gLz&)s`a=5?GhA{JYN8`Lk&nXX>LRDzH9exw#rNr9(ZiLFuD zb@xt(J9%^vy5ZP#Tk-a4i>%!fxq)tG(dh`;E7i!W@!S|t?=fBdi3iG`Yqi4mD(3ja zB15kB!)Z#qL4ngSy6*Fjq4;2W2S4eAcS)KXS2qQQqB|HU7nw0G)5}i&Fudc&0<)9h zWErizAH_tnb zqq4&>!%YfeL3f&aOIuKtbcs)7;4SJjDWbbLAaJ%#mKQA2HKVe&uo<9n0ZPh?&KYz< zdY?);n|j}ld-Dv~BNm{1WNv-9rewgno0H}I#kp$Es!uybG4atA@|>N|91lC1S3*r& zwk;eIRZE(yn+wMO$$lG#bLn>p@>M*dCn*}<%HB?Beqr}3_^ledVtSRK$4qj-i;V1` zQgXZv+!{$vJml>&zNOiFU+Lp#t9gViloS-*vA=#_2$wzG2y!`bgy&-xd9{+m4+$IL)~*a$!5=!vgj^;9SJC^A0MxrgH`Ij=>Kh9N!mfupPw>r=N?Lr%SFGwjqO&95pmw2(2ou?DmJZwO6`o0PRarWe^Ajc#7Q~Ow!PR@Uy zUHa_I;#hw6tr&$508xP|F?~YzYX@n^Dv}QygXd};4i-Tz#=2#_`c3Hd$>WgCn4W@T zpy4KjMq<$yLHp%Uf}3`-KIbJE1jpT#7M1p&4Vk;2e{+XeY}}wsz$iRiN{aNvsX;d) ze&o_Um!h-|z4#?2cZK;NGz3zgK(4I}BEyXJdo%T|KABDMtaeqc-s60$!sV<>pV*$~ z^}k#_irD2*Snc~f{3V2@+E&wp#6dG8I=e~SG$HA$F(|t3W!E(3ymsOW-$mB}C7xT6 z!&mgUZ59FqqaEc)RSo)bTnut`qb7r@rQ{|iJH8SK zT5KoMvngaN)VD)zGs5=_}v}ST@eOt0C0ZGI|(W8Xiyj zZvP2gvz{_c+u7yBRrHJNLde$-tXelXIZQ$^WMqz&$M7P|Gca@F(Y9Nq_eyS1Xi zU@8NXoH@3UMpw$QeCY@&Zc3N=3yFiGs2)6ts~<#f6UpL?WL4(`*;HI9=p21Teb^yJ zB^kxWPgRmkBb+Kg(0y_m*e&5R>!o)mpm<3;Di>q={ah@l(;#%q+rznr@O312H+I8P zviouXhwS2&1h+zCFX^2|@U00lstvEdrsCYAT;H(Whv5*v4!efu!ISzW$|6X;2BTX+ zp$BGr2wgegJ@JYidCG8OZKr7GUztr+vu^Wo&J2ZA9eY!|hkuQ!C6$pI3Vw z=Cf|*V*inAEK%bc{$>ZS?MBwwo`0XtEP|n0B-N36zaQb}E5T0~Be&KFP7Z?;$=o|g zPu-`8O{*BIlsaeenxT8JBh(aTt44f$Ew}$X0F5Yw4QDhSS=18y8H^3ST(Q*>nBgd@B8r7gOZFbWCc0o@w)IsPW|C ztRFN3qWBl{P(~&xGdW!{r_D;WcF*iC+@qB?zl-*1Ai#qk*hwZtrio{0)?g~wI9jg# zL6v;#H#RJMT7BF4PZM*jG;#Q9rz~-lzPY$rJ(Go=vM*k2tMxofaElwesr47$ItC_Z zv={Hx>$-IdVmdDZaYzPjw018PJ)c6(NeuOEv2=8Q9j~soDq+=ACJcQmN%|HYEKJ4j z#y9p+E@iuLC%h-#>v=9&y%(=nI$=>z*5Gk*;ZlM(hNzz2tkRE1sgLXs)7zTjKGKvQ z>=v=j2-bGFOcthKaqj!mK{bG+T;O4&3ucSqCJa2vE3e%vy=S-{QXr^xHc}2E(p>*6 znMW(b?2PU(u7JrPc!sp00+!qoA7hkpq71VQBPQ5D4G$9;`{-xc>87Z-Rs!;(Xaox9 zNfW++?b6$cEez?aTREJy8?DB-Vxua?m_F@+L8w@wG3|eQDXm7c!=XP*%D=dVjbNGK zU35m6vQ7d_(*h{i9;e0N6Tk=F9tDs(SbL!il~=&KLi}XHg=*(}-a=+<`l}BELEEoN zdiTEl7gHGNp@R!UfxhX2txwB_5C?BxdU6Vo1vJn5Z$*_cgTZL!7z#5)iYq+`N4$`U zdVGgZjo64l;R0zhp~cbJY5Zg3FKLwb+=XpMGCHV)j90iD14)BW|g}_5pAS{81GOa^+RHfR4Z5~W%el@JwI*4O~#g?F08RzIZ3=45z5^WDd=QiD$`Igv?L>HH`6Bv+TI zKMc?2!B4!>F1hD%#62yMR&6t|Lc_vOI~Qotn)~$&&k}q@JHKQKTD!jsYD+T+``bm0 zA^GBOX~8B9G&%ey%8N=-lZa;)U3;3@^gh0w>(g8 ze-c`+;_S+kVeD-I&TFZ&`CClF=ul7>2GQ<`OJV5R)Tz zciluDEKil@h*8*|$1ZU1V@Q&=s5xbXm9Z3ms8i#^xw&pb|07Wa;KkftRBrW#edBj| zfi3Sl_o}d4&hx0R*V&#Y>zutUQH^Yuuh|k^a>D8PNQt)+us4p~a?lEVbt>CevOmGT zl$)WCGT_MM%UMqS$T`5VYWVFl`_U2;o?AWoMzTp~I4ei8HL}~|N2atrk3u6e0+?XE z7(-F`Ho1mE{p80LbGcus)$;pcYDrIA-g5J{Kf5O4fL)U93o@EI{Vgh>M$lZ}{J}@! z`I_HkWr;i7mq(sMX%;24;9s%= z*T3LV9{VQhzuA@l7NbJwDQfb^Tf)>?IEc3EPWg8upNbdi#-@>#uDKYXs)X=UICv!h zhr;;#JY!0;09F5E(A8ymeaG%vJxL3xe-#kUeeSMKs60YbItXasVt796 zL<2ohiF`!0{5V&5aBoufJYrVk$$USY~52zLJk}iVhf)iBdVwgy6zp=7xkHn)gns z*I7^IZ|usmbWR-S{PAyJOV@q$ZMt1?NKJQ(QJm0+_Em&fK=zdUx{cG4yi_ILdW~<) z1ql-nPNV62_2`?%uP%k~{lvx9HaG}ZDjscvuRLCJy3JjkpHB8{KihaYtKR9@6itEP z{#!n0{*mVU$0PB#5e8kS>(7I#5dTf@4)uMvaAE;J#gXpyx=-v4k-g>AyB(Glq{m~G zP=0!5G6$=vJN+yFS?%w#8LW4fI4xt8q9DoVuHS8WdY8y?zd5;ne=~g~rZ_w8xpOs4 z#<%uR<|d0!%Ymy9hY^s6o4>UANVey2+1szXVKttO<0mvB;%npgtSW#qeWjz52BF3Z zoH6G|#x6n1T3?%$+%?BQNmWC#^8N%@4;H)XeigD43O3|GABA*sUrB;E?GceS!J5;D zhf6ezJXCY%BCh+s)si`n;8laPDmB@P=s`MHFLp)SmbY}pa`~~64urBMf5Hn$=zQqh zYJrAAfz-nDoT3Xiv#B`L5>}!vyeY4BP0XXkg3h1W z`f$g?_ALGMapPL5uiZ50dY@wv z{&h(wf1J1HdS5Yrg!A|#)w-!Y^G~n%v|Rnqs&AOzd#m$v`mmR}$;BFR2x9nrOxGi! zZYkjDHH|lrpJiPAJRwrE57yzbnXVK|3-rPLHZ~q<@D;bdK||}b$5KWJKOik6wd0TPo+t2daGd}a}sk9p+HGpKq8lY4)#2CeI;ulrH1gu2`| zn4BVhlAxpwGjvyocpEQ4+q}vO`4BSK z#y|9LhbDn~wAJcwphMG(^4pi0e|2cqlBG7?y3cj)I|fTuNtH%Pb1n32e_kL=Z|o)3 zCzv?bD^mn7bg4yuk}P^6`ga&az{!gx=C#iSJ8GFYftg%ptGMbDexA2tNW6KP7l_%* zOL6}{Tpz^MCr+p3Smatkp}{*u)OUx(!M@U{?C55Li9K;P7DdLx;gf-=olQrcwwuNR zR-3J{5>wb_IBibv>6`)x9+t}Fe&W_?K{MCKt2>i>Dqki(x+7X=y!{5N{$7Nr4ZWnb6gZVD7^6hvGrC#akg92aDV{8-Q5zL5G=U6ySux)hu{(jn&2=v z3=(v3C%8j!cX#Kzv#a)7&$Iu(iUa0s=JM6ut9$twUx5^aJa3l~vlOA*b}34|%h1(b zrvv_4bi{Y^{M)n2tD70y)&N9Pu!$#l$NC8=TIs~4yTp2&;;ins6Gx5oNX}Amvoe5} z9G~HIidI1%g8;b11G5IDTMsEsdabtNvbHXehRWtgf7+q96JlO4Gfkgr2c$U%rU&90OkW#s_En37EgeO1zJ4G!mm1> zoE1%%%LVRSo~5!Y-U#bF@4nh+G-g7k7Las)?AvG|)vo;K0XymjT!+F5$4+$H#VRsu z3DL?l$@^L0nwWu*1bp5AMr!S_@m8sik;<12B&n}W%>`>$AB}#bCGPpJh4kk%8kzkzI9}F^ycTK zC`8sA(2D9~UB^b*ngGO@@wo71_2y@4t}U;>{Mko9nDDoshD6M66a5&44WSB-XMsmi z_YqzPYp(i#deUAHBVJb-DR|fl*}yQ#JZ}17C5u$7hhjWC?~_eVXac}V=Y!)jhDT-X zg$POx+?UC!fOA7RTJcq_X zyeQNLo}&d~&ccX>qx60_)Icy)`?KA(cPqKpli=S~W7JhF9KlB#jaWgNze*Nq8h$IR zoQJ}tHhj1e*O-8vFU1)sMaDDVC^dJIF~94!^>;oxL!hl!d0OvXMIwK! z;BBWmf+ALDHY>>JlU3vo&KpEj)0oLoDUfgCi&sZs?xpCkO;-C#U-uSOlH1_;84i_L z7Bd9_RUPA_wg$?)HPBYcez4fi-u~<6OLis)DnPa0&#ozJ2bU=WZV5BRvu*whL>8P} z2Cg6L>lA^k-iqU~zsSNr3@x;RXK9T}D)W~#Iu5Tn#q?Y8t_^ZVWq(u}jH6!ly(+D2 zq}>m(x=T`9WDi+fb(^t8AprzLK{>yI4(i@>6ja{--Ej`6KM$lMDeFkL}LOL_ya zXuiQO6uteir9lLcJ_G>0$}+IeBv|7J21otwo^fL~0{wbNTA~#soiKB7shw;Ni@nuE zKhabwf~lQ5<7A~?{N3J3nsc55jb6Xbq?=K@T6pHF39y@x^~ljF4brXe21ATp)sY&d zZ`tL;3hBqya7w9j2aQ8?Jv!D^j9qOh)73SImDG6EMiA89MH6wnh-uC~59Y|QYw+9s zB#M$1iB*eIH_knO8gBfr7Qn0RG+P;{`(4zLauyp|QyBfsY-gZj@F&Ug`{l#b0m*h5 z1>M7Cq~#U=uRN>qkPh3&ZVgkpOJ24Ehw1ORhp`W>J%?=sIDll}-jM@+7{mkgZ@{Dx zazgd_=cDk4PK++~J-JD=D*rtF@knF}4>>jLpZoV2RW9VqV01n^lixNP4O#Hn+r2KX ztdz9ZC$Tv=5TBlz%Lliis$q&y`39Hpq8Z;v`%^IiZ7`^RkTdMzAX)ZrD=hA8!oauu zGn#;s3w-?ilh>7x1oB?YQFwUYi#SrgyLzgk0j-fZ2H+-PAf-`=h}Fi)h9&ehZ}Q(t zazL4?4rL9hg@BmTzEsju??-2e z_seXT;_J&>br_PDFY(_92FI}j=O>OG5dyOe`&BiSsGEzUxfdo43$Wo1$gRD`5`*9IzSegzNI1zmOxw;y+p!JGBKmLYuO(><*ebh zA1>X7`Q}XQfF9E>PxQ+3!@U1!Q>I3OM|a4J0)1Z(%Z@E+*9EhbSC<<9e{kpFjkGQ2m zpjRbyoIew$qEg9nD0y#v0))Tl9lx%H`4#PLV@+#p-yi^(V=tf+^wn_dvHJx zwESO1TuPkjY?i==>J%*^$^CK8nJ@r^r>=B8a5`b0w%j{eOro;m$q$Tts$bQst?wo_ z$2MPxR;H4T%yB#%^uLhkid4HE@+qIp*Wd>o!Tiv?FC>J_bm`jn3O-!-{N7?L4rr{+ z)$T^)`rN1eXWlk-qBTKOhMU zyF+rbP5cUbN1bm_`Z&^e1K-2TShJw?Vke8aWV+!x+u2V%#prY^pyP>7E2wrFQ%nJc z$~7Joml-UN-rs>`J~z*`bqz{9pnEbdQYPEWI%l8mZzt0y@~_;=@|N4okp0&!2Bndc zk6i`o?cb;?%I*qW31A>?x=npYaOG(C^2DY$Fe)Oij$>2Gf2QWGq_0mUQo@mQC_F6i zBh>phj=)-}=o~{e-gvGpt44nT;2#&>d>z6~y1#&Qg;qkeS=?J+QD)2_2s4xLCY24U zh6A9kvEuczQkjHm92XnTk#H(RtCKWTD+t0Q$6LoF7nLFqckDUr==&S6-FN} z%JT7i-9*m$LQDTg2s^%IOJ0nhA3d4aGd06?f!MOJ8-#NCEa-SWt#JC0C}=qob~PZY zrn#)FtW=Pn*E7Ixx1xhpT7jP0zS|z#k}!gUhC*knqu!mt$#JMZ;m2vJX3ugbDJ@}& z3{+lzU%q6caC|Ee4h2$v%AT^h<6y?9%6DXOSuqsRf!$n0-y)`=YsqoNo8lG{ znPkLWA+u`xgd`*=^sKC`CM;vI{>UaK8#4D_u_-w{=y7{vJ2&PltJqH%{K#H8aQl%Z zmoiw;1s%~%ECQk$e`J$e9Kc6pPQq8K3Uob{7{2|=7&$jS@+~Rf*SUMfU3t+&m?;|8 zoe|gBuEHOIu^v)2Z_{m3f^tYRXEIgKADje7sI=9^rq7%2I)(>rQvXhuv%D`k)iAD{ z*wT3ehMZ{s1E%venM2>N<<~e$ebGxeF5waGuiCEd-Q@*CId@RE+`R{r`X8$8RT()Q z8^%XC|^hV<@9|Ly(k#Qc`=wXGdvW5o+auyv6ry+5X>^A4X__lD}B^$Pms zeB;uoupm#$Id4F^H&|pjcC3vwymI3uT-o;4`09o|VeUFzKS`m3c0((}G6EaM@HDSg z&q%hAEXSX-&t4HD0nAlr^{tsKpk54!TWFD>1VYPfU0gn?sH#Ta($Uey)4YFQk3EaM z&w~PSW#Jq2@JhBnqm!mp;_S9GL@``J^!n5u?n6&FIHB!CoE9@}kLAQ@{Jw4Pu|-PW z$)Z01l>Iqq#j%YzLXZDfX!-r#FO2yb|AqmY^jd3zg9S#ds*mr{Ph~1-FF43Pe9P+M zm|KZ~>h6!a1Ny#Y&vqCT*MEDA8C@iOe!84#a0;FI;-i67&WTiBci5@3t5`MLgqw{_ zKo?%Nwh|9sQpywYDy%fJ&T83 zOxSrg4gH;E=Lq1QS?mV*)H0CJc@F$)p){kG(%2R@^`?)Crged~|GU0^As*{@q5a-00frj~mkm0$QBUj`#~jRBt%! z+CKonlc}8QMcg6gD4*)!1t3UFuit!`Ei6+AU51{>8{f2ssB~_OB}3!G$ig1-V^koOAug z0o-QQ$e>cdGyOj$P5vF_E0^O3oigM2j3Ve62={t?h96)z2RF3P&>5v*&)Tp|byp)R zNgVMbL>rJ0w2)@9?*36o`*Wx*_q4eBvHr9lpU;kQr(!kwTvyQ)L|3CuoT{Z(2D0K{ zX;jB5U=W9d$!LDBsEJQPY2P-z8LnJcPh4{ae;unvzP~9rGan1#az||WAkpn=MVpjK z*7@TKwML^g4n{VsA8nI2e|0q-P6hev;SaSUjDmyTa}!m@s%Dk|KHU;r%!{|C8Jwpl zCg~ro7#>Ka@^%2ZI%&8AiN(P0MVV3iBkZ?^KwOVyvH?b2YLM+yOeL%7#HWO*89WD@ z?OEx4h>+AAMa~(n8y&(;a z^X@F8v_IiJ=nHMxQE`;V9)I?CUV(>Ck2B(WA2|f@yf-dQUKqV{4!CKjZ%q3o9?)3L zpXE~?!@E#`E%I3r*dk+C-pvkZKs z+s<4i&+Bmhb^Wb!e-9h;}lRHb!@ z&#hNc*Avx7$wDYzt#i2kQdqI(z6*zYfXFz$?P(M+?S)J%pkhVQG>=P1DGJ&l-@fzU zn{C6(a1eaB^)>|IHcr}f($p?E9S<2-TuN#k8if2oYG-k8s*L{)uOKU=U%X0LzOC;sDiBo>^2dCZbILI*D8*?bj z3qM*{Am|5*qo~~FBgP__DeT${%60Y_DMAC-07<0FHn>$B+K@r3IkqXJBxe00c~azT z2*Q0F;0zQQ%wN_aS@)`W zD=07z{zp4dL^fERjX5wl7Sy9H_xa*tSS=aINzR1Z8mQdDm8l;NNhz7#66up0}l;fXTc~r``^^Mw$9kzft;# zhDG!0ka&|8zVrIOlC;)tB|AQq^|nU%-w88%=g|cC9VkEDO`G1;4XMuG`El8KSMy>T z?8S|=LX7Vd?W=09Mbl>2G{gj*xKj9*-Xbls_mAI@usU8l__gwrPjA)M*A1E9zn+FUYIaN~IBUD#CyC>#R zr7~7Xr{DylKBPb8PG^8czTn#2ouqd*9_|Q670+p)afW53$Fx6fzaQ3mAj$~H$>{-^ z-Ke7}rG2?0Sj~BjK*-B%F&R+BY#wM=%E*H8Z?pmgi>k}p-u%|4)sgoti?bB^6y@8(B#vU<{2x-Z9AmJuVs+Q>&Nsoy|-KLok*-H3DYspV8H8o-!R5 zn^^MIjWhmft6L5|0<*}hihMD5sNam0A35);m%FuAPK?Wsu0CtH`w?Ruyb`z)Eyvwi z+)}oAQ!!fIzk%alf^!XAj$*-5+&r`|!FR`ynrm!q+{gg`{{4GHYsZylUq?e>4#5;& zXv_ds(}%BCb!v@|ozS}(j>=Uw5LE7u_d%m2R#dPh${#7KyIz~5`k=Xp6}*aNwUA># zP?oK_RRVw394x{T#V0V`em%PZTa)FpemV|)UeY;MtyU(GLkq(?CAq$7Y%0#{-HTrA zmJ(fy#XC{2kG6%g>*fw4z8?`u7^y9rf}}0`tMMaH6moKKgg{F%vAyv-% zkMmV`f_LsGhj*vbc<6%lL_2&*)x^FsvYPs7Mmk%eW^1*jIP6F ztY4^_g+K!@Ns384Q1(&?>M@T|ZxlyPpmDYK@a;@kuLn0{9av(s(0=WPz#0V_i`Eh^ zQcCkz253N&zUJh~+~6~XF2}}Hynh&S%4Ox#ZpqK4Y{A`wbWQ0J`mbe}^adkiqNgIE z;L@{V0@}$Nqkah)ZXh@)`O>^2^@mV05c{H2;KeZQZ9dfNx?kDiJD6C5<1_xtPHiP) zw1(H=Uze^^FnZO~bUk-~UpWD8Z6LmBme*Ubx}d2@B{*&Vob_q^Uis;^8ony0mp>xl zOkucXmqlyg0P5+^%k}9_O!n!_N*3ASnYQq`3NE{HCA{-6&S*L=2FU=D_1MhNQ(P(7 zTQ#=YQ{)d>%be5Qm>F%@+@GDxmCqj<-<#zr4s1nfc%2A)nET`Oc+Ke?Fyzj;bQaDd zl9R>FDW3PekviA3r&y)O`eX7N1SQNW2W82*n2hr2wyQ=p2k6;wLil|y4>%>Pd5Jmfj)_-U!v;7oao&>I~)Qi;f zcbQ^jww+iReKv}dBg7MNs?73v>VoSxoWMnV23^x_`=Nn}$p+gN?m_P*gP{g1iws*C|G+yT~e z&AtcW97O_=E5 zorvh{-bP95JwWFr1zc6a;SQ)k_p3>MPl{p~j{^d6c@iIx5tS!%X_9G|b!+;T+G=;{ zyd{7Cnmplt^zrFaoo}NAB*N%`>p1eDZngra4XaGfAxO{fbu6ayL<0~*z( z>;k-Z+GYFfyWcNa=t5Q_IFb+{F(w6I##uZRp<#Z+I+FL$5l^-(ESJ(U5cnB1$UKjkg6?JMJuE zSu3o8o5Pt5m!pYDR+qi&drIub>K7EPaS#S>gIyR+g^saw*)0B2DN|bp1pPHmJ2GIX zVKbeiHf{D)8)DExmg4GLbC3+~q#ZxIkXI#fD)}>@Ir;H)5u;dAnZ1rVxNg?GN{h*A z=16$H9NXD6v{yqCWNh`MWNIfpA0%kt4ttbNSXFxK{@rcZkfW zS}%Mdb@i)g(bU@b(ZJj!tN$d|-h3I(H-iSJTLnky??9)>@cMFoq&P6#T)Tp1B(?h{RBEA;3RJ;e$2S?wZ z%TF3iWGHcbSkfVka!Ud4-Yj=`vIY;*0I7v8-dVDa76f#ssa?t^-F3M<5K?oA>|BPR%u_Sh#b))|uDmVQ0eg^|bGv(Od~C z)^;A4A{kI|9k2FPEZ1sP29dmzQKN&C?$8ipbV-kL$+I#c*p}IcMRsN>U|=dOB%rB^ zz-7u2?@BH>)DFOLa(%DV`**3ct|7V^rZ^>Su-3-jFZs!J|MnFLN42+XBlW3hxpcY> zi__I0%bgp3{=GT=V#a^^ue=>*2j~=Tw55>y*i(@se@~N%XOybqa}urTJr71}Kll{R zHmqpEnf}Q~A|vBXxHP?22}Y|M@(taFt_;p^8_o5{XAX}J_`l3JBP(B}+@Ef4oFSYF69PU_nW$IT?)r<5m`kt_VZu}=o zeQ%*4JmGMr0BK#br7=-!Ei%+b9Qa$Hi&EVhtpbC$>TNx3iLZ-R+boS?xwp$ETb((I*w( z=EE9{2kp6xg`;=Up3hA0E~?}1o{k05u$z$+F~7lowS|*7N!hr*{W*2L@j9Ndj5W)+ zf1u*3yvG-;jTIJ^eW%OlZd|S86SN>90~IX}!pjM4!m@@b$8|ftd3$rb5<~AVSXtI0_p`_=4ZyB}QDk7EGY=0scn*Bg2f4kW++PycvBMma!rR?7fQ!S#^jOv^2 zf!32ydtXFQ9HE4=Dp80Nf1v{vsri6sPA~EB2DVfIc>Ku+wMTs2mYVzZh`Kf_m*exd zru7-!mY!2ur+sOdHo?G3oGWHl%bDb6(h;{Gp4;mizqz0@Z*AgVc08fFTJmGjM{4-- zm^e3Ln(V2+V!9IE-0zq9?cb|i0XpSW;ECAOz#pkQ`zVNow2`d)q!pfuXeHF~vmUGo6Wg=)qvmi+}q0w^Uz~zD9&JR&Kaso~tjtJ_yWB2R6jK&U#x{rjwr`Qs9Nl7USg8O9v(i{vS!j7Ns1`4cyo&!NZZ8&!04 zP-Tgzh|)57wY4?x&&dbrJ0FqvW$foZ9EXU~NE1G3dwpInwxwVC2}8;RKlR=ni301t zS^&RD2)4Rt0_&_!l~PiymKcy>PJ3~UxO6MTX0}b-`{G{~n*m-`=(>_dnrHYLXxh#Q1qMd{(4w5FCP zPS#R05Ma>UXdtQS<7~{J-&1dlY|J#yRhZ{UEyiZ1@4)1Y@%)Wi7K;M#2dRYivwkMo zxD@x}u0WRn=zmt{fMujV2t^crSSU9>;D! zw%D@xE!=a*5Pu{_3Y6}d?!m%ur6S4q1caj}b}^_)%FTl^bmZoC(iFT7zMl#|5O>Hk zEo^2<`mD9hj;*)N0`AP48kxecl=+aod zS$#ZwktA=A#PgdEHwX4r2b^)pIfkO+GJPpHZt*8C6>?uPnVhbiIQ?V@GyX_AZ@$j- zPisd5;9Smt&UbI-qAzu?k7O@Sf41iXd2l=&q4f3%ys85Hi9csXA%ByGsQ5r7mhRWdGz=!K z`u^Q}n#`X0W9xSkbtyDDv9^Zrl*Y1!qyuoV_EK{pjnxFwnwa1ky@<(VF%VMp<_9C` zCoRS}qEH&7J|pKRuQ#JXF%=!UbQ7R=;y$Q^%YGcpby`XQKjwGDcXeMyR$j*vT6aTJ zHBYND6$a6K#j*AlhdS2@qTID+FA3zYcUbfZ^GPO54ZHAlIU8jA7DcOQdkm?XEe*d% zN;Ntej-}&ddT+YPh?hay$Cyo)JRj8*AD07DXIOQorN<%IvVDRGy;?-87*wZGG zaK%ia`V(=R6!=U?OVAW|QNoUt_%rWG??H09FdYh?%=^6eaXmBc6#W|sK@iMiw=zh;ff!q`ZqBE!s7}nn%i1C^1|FaMW zNmE|a7+d^^j%NXf*5_Rym+Tzoss@lrNhuAy9jNy&=l_+M2YmR7Srw07aU?QPnSb+k zXkeMzkgeB5H(zX2lc(t2qSF2XOstMRkQ9Bdg)fN1y)QlPlDOt#7y3Oh=xWNg`m=+Z zP|Nr7X(+2;{!CRfD<$0&KWSKC?#?YzC6z##eNR6&tH_xNp9Pk zuyf%~MB|yR9G&0!ocU6IQM|>?F|G8B-#iC6krI)7rlZ5$PD%k<^1)*XU#J$j))vaZxcf*_7IR=< zx|GW0FuPPT)ETRdH;kJHyRYTPs)_KE$kn6UN6w=Ktaz|M<7I1Og)%Q_D;}^e z>JV_u)eY|-FE$2vj)P!YHO%IFucFltd-I2NLRV#E0fx8+CrdgJ61Gg~1ImOAv~a&H z1EUu>k?e3E{#>g!7WGaITs*)e)KW)Zu8oZef3wW)zjMqts-733Oc8S=`QBXj8pEsW z9#nY-1S^V}if*L1Q0dDimTu3a7uXidv1O}HTL=G)Yil*cOQ=w_9K*u^)mlxCwZG6$ zA^}EmjY)iRnO7C)H6+da(R``E>41`qd*}jLtFcv5nQ~qw@g&@bU2-tn-O9aW(~nQd zgi`lhI~!%%o3BSv*rBD=WI#|&HNmrnjk$sjBld`Z?ZEb$QM0m5h`!o9f&)wY;HDjbklv{xaasqQt(<@syIga>rYtE!#a_2C;;c5O{|cVofb3$}V;RW>R1?eYxiKG@WeG z2KGpY&QR2g==*57vP(<9aL2Yis;#o2-Tv6XCZg(d*?1Q>!524oeK8gT=t)tKQLH@F zlQ)fx#mJA~&hCfqP4}G6W0Zt=FDXVWF;>VSr(;iQ_XEs;#6qOK*WpOu50~F6QK5Xp z-w3}E$j*x3z+{L_7IM!g)xz2)q8g9Pb{7=l(JF#dJf6!zW#3V=f5L|Eh=UDiFUnNj z9$nxFnsyXrM_5UI&lj-dX|X|d;Xrd)4iJUF;Z{!K;0T06H;9Jn&l%tDDRD%FfpAXg zV1r=pN0H@71PMtX?GeZgWFE^f@>977JQ<~kgG4f+)b!hF*L1(VUIi;RyA{L8HE+91 ze4uO8CG={XD(I{2aKR9n60$+spGZ77GboiQu_Eqx9`=QywOWutN9+~0U@3H?eL{o- zn*O9{g+F%n-|?6?eywh@8TM2rlKqq8NtVfx0hLL85Tb*H?v3ym=^0oikoc2Kp@I@0 zM6=IB1x&Qf3hFJN%BMk&?+IX)s<1`X9$R2jUs}Y5^;eS^Ji>XA9IpL(jr!8^+Lew0 z<9UJH_)YPv>FK+9&w#M<5r1bFcSFYXbNa9=Z~FLvwc)I2HP}i|Z_I#aS`*8-jxY1x zcV27cK-8tC6=_pEPDQ4}(?Iq`smcSiU|iPcNLz4|hFJ9iBX*sQ@X{B23d<7yNKH)8 zHT1p7K96|0PDNcTacUBa#`H{ZuLlkxN8nphMO5Iuj(78cS#l%H@a()QVLtIlEWOC5 zNwnPhEd>=L-WCK1d-~#i;E-iK>lBn9k_D&C6@OwIHR#Oe~U5_UBK!td`$3z1x zv_&hPsZvYD!_I3s)AzrO$58GV&TB4Tbx0-%65`ygQ zf;v}56o$0*mIcQ{ZkyVHYQarYuz0H;rTUjV6<^2+wke@<8nuBnxS&X>n4b0CcW*Aj zrGR#}CMw2fMtC1q3Uac<{;lBfhoQb+_uJCHnb0=9Bpx3zW zuQQg={zY^2B4|mR4hxy5C{M4R$SL<74Zg)GC8}c}$@iFm*0aWMO_br0IF{>gSX4CW zOYv`{i#NzQ?@ky4K(TU*KQb7VHA?>4lkN)BTxWE;38z}2e*ICOLDSKI@NKcL6X zm7Pe45zQ?HX(IvizU#x!Ra@Rqo+5aDPZqx>3(COjb;&D7MEw zCKODzOh|UA`r+=8zT=laJVf#q%5nLoYw7ryqZE_V$;kZf1Q_des%BdW6>oihjxs&b*-DsK? zK+;(3fUu@igTXn#-{Fb-q$^z@sg>B?Fq5$eAy9^ICpS`|prRc#+7^h-lP`o5$lG zZtH~Ux81H)L1n4{?1I)87E8oxHV}`QjWPHn1D$*m^1%1)uv{Y{yl!-xYpt=BJf2}T zZrK($pHm#7bJ(e}tFg3_$B=RFmI~iI_w-fJz@i(WSy!wU7yIt~!Buh7sp~e4%NDti zSsN!F++P*=%x)K1<;Eda3N5cJi%X$a;KV_x9=(XTPhUx}F%zNS4wt8dVoXRcAF`o_ zIK7tU#x%czUY0i&X`IJ=-?CT_g%D5M1{D?{PA4GJf-lK&nE`RmwR~#6N z6Aw6G$If`bJqZzAUz$Mk)?E?aIpB(iJb7n4v@v*$_`+R=)%RW=J3v7bD9N^gDHEoy zPxVPf69Oxs46_gx8 z_FCocF||;*L^j$7ISQG8E_RU=TN&B*#kpm+x|%}MRv*bW^YEkX`R#ml038@*E1^z{ zrI&=smty1)jd4&t2w=q{eJz+C$gH%Ps_*HdZ3%bANTcawHKINcJ}GM+=?W4cNK1eY zOTWUH2v<0jzzjlg!L8+GQa(_>q-N-TstssTl`7+uz(K3jX$1`Gl~PRB-R(XL8xtlK zO8_?DB{5m-eSkERvLYx(hvTOAR+^P@5($5!4uo%}tb^s%$dLlRP|UuSBcJ@MB78u} znZ|xYP2FAaF8-SRauTdc=cGO^Yqj`K&+$#9w=kb*nsvbb#Y8ER1Q9f??HB zXYd31u?{Hr@iMTx&ptf*k-PA-t+2#JUM3dIM1y60S z6*m#Git%Ya`Bbzy>y&a#mcR@t!PyMYp=n!#SoU`}35qTiTX)j@;=4ivD45J z*rgQ81#Sh8GKhb2&e#@|AW(yyHe1FtzX@Z)7~_(#xs77EBW@_O%b@kAplLn|4n7~K z`I!9?3n{7TaJ;%bbcyUeHsfqOaf?tg3Z1^@~Y0uk1GOs(`AFUNU&zK`=|={ZTmk~I3?NM}IcTgH%ji#!_{I53Bh?_ z%^=|*F$|bMqaEz%+BY;dUh+e}vm?j!wCjrTR^BLjot1U$c+d|=iKOSD61JcP6;`1NKg*M9y3$26-*V1{!KB-aTYy)! zmi1n-Gxt#$#OwS@!eufwn<+90UTlO9X}{z72rWHFEG8K%I{h7oGPE!aw-L)5yrI?$b;I#iE2+RPUN0!bZlbnF*|(`>J3*58O?XiU za+|p_GAmE`kV4HUZA9n5T@mER!r?qNsUA>$mW3de1sse6?IB9s(rkzoNc>#UikDbl z>)EoWSaIlNL?1skaNN@EVx9O{k?mZL-{kJrtd3XTW&E>LT)UMghn`fAnv(-2ClW*V zyZmIGg&1TsThMUJz93@Gso+X#4m#$3$J8wI4yV6CvA^E1t)uUBwfmvB^?b2~>uhrx zugb`dYnb31+XC4KdP=|-+VoK{*qF>sJVcin_UkA9OBn96DYR<$DMzpM{eCBVdj^m9 z72WX94;?yXV6Au2p8!Ji()qB$GhaI~y!s-Q%=nk~nNakslMj;k&MXkMv!*zwhmaon zLbL7S$LWQ=hiD)0Ew26cAowbSkwUp>EHgX}_I$p|8YK*wfSl6F|DM_H_ytZamF!u- ziwtSgBHGv5uBm`@IXWHO_ST&VSI@cE{qwO?F8V$GxiLJko*;Mp)b+<%JAOZfiSmNq zI;C8aknb7!RzUqse`_hjD;fbOUh-5DC#;uvJ-_V%n@YeJ2?lVs@-%P^)`U#=vwhS17EYue*4HrG zc|7+=tm#;X=%jHaY!TeBzGG|OxQ^x?h1bU8>NlBxAJqQlY!{x=Xg?Iu@si`vygZ?aJ)01n@QzHx<451UtSsER#jSzQqr^d^r?uTQTq;#E1~O z2vbW-<9-L;~COl zZYTk&XBb!!?7*R={s<=#gCD%!9iGZ9^`9YD`a=i*&fZdhi%j)#TtH+@_Hb%x)Q@<< za?dMwcUnL%lQ{K%?jS{Iby|HGjEdlC=*}x>;dL2cMSaNVYdE~kT^4-EUC*bvZ2BJ5 zS_NY8j{r?lLzTed_Nzeq>X=0x6vxMaIJ1I~`Bgo$9wjHy^FU%DI!Om#bgKlkG#`|I z`^q|M)@W|VWS!%CIAnZ_jYQC*xo8R-+t*1^V>;Eb0@?QVt0(MKE5A{?UIs$|6@;ILf@KUIv= z4Dbo9IZb^&Yx7fJ)R%qbTw{>Sfgt>4v^M@STF!KqVj?QI+LFH7A8vPS*TJg1O%trK zmf*x&C4cL=Ev=nLGeZ|3%yOLb!hjq8a%E{&Z|vXV1mLE|9jx}Tprex*`>=V9JX4(D_qW(+a=^OyB=9Ty61tS&= zmHXnZ`T}gSbJ07=7a;%@`d`97XY!ramWk$0D}}M_;DjqqB``=B2t}*`&jz)=k0z0B z6}6tO32tJO*VOg)QN4XVOj-dz(d9mxs2fO9qN-s5S*x%F#XRkPlJ zyLYG;&H8~mXsvstGQYm}i;hOVRt2t>U22y}D>SaGJyo8B*zB}K_z6qr$!Iv(tg~Q; zQkO-jei1Dk+?l(D*d#-lvIUm)>?i_t6_^1kiTErRwWV_4u9$gyv_Egf@{TBZtt0zwqN-o07d8Zxa zfrC-0I~J1#sIeLKMGZ%KT^vAo^VQx^K*LMVpqLDc(tW54gh$Pjz*{Z$sT?AbP*!rc zwp!Ww-lcQqORoi1Lg_|Ep1Qq;7If1FBPSBi&0a+968qD2pRV$p3X+D6XaYDzr%LO~ zt$|5JI0H*FMzf+h<UMj)!3q_;^XITlC*tD8?+5Cg_l)hA*+7~qt`>(FbEKgTpA<3Qd8?IiUMpCh&~ydBwOcpE;*S@sY3!tb-sGh zK*=`gwWqVA!U){cR(-^+L)y}xX?1E`9yp&*uox4XQ{XpJbTv60)%SXKWQi=+){#3D zJiYE)(ss=)*lVl|z&hdwIHhc))0RtS3&n3=@|tt#nrK~!yPpt3&cxdRYv zAhmCXfb}x!6U2lD;(m+}UMlQ9(WQf^HbQsP#~M91x$0>4m0fo3Ebv-y=q;u2U#6k> zGVNdQdf-FSU-^Qy8S}vdrD1V4T=6VJDy1h(YWMJzql=60&oVr6_U%!e*RG7I51aOuw!O=rIiN~6f_H{PxE`quPM&Bv%b!_ZZWqS@ zIRRCi1#V{vUPMMsxaX0?Bj-0N8eXC%gVcz)88@nhWGVwHD)phn7NGIq;xH(kEtsAf z&=(=DyUhLX56s{IX4W#coX3&8Q~M1$SrA$fmy;j3D``jJ)+W#GWD?dHDdi_=F?x_; zZBG80#O4%VOfy5FehE(O_FE_9W-c*PSrbZ)52T1mgYv9G`>=zlV2LQGC-p>?Auci!rhZk{}V@+42SNHZbpsCy!F@w`$qJ5<-TBe&^n z$Cu2guUyMZaVKZn-gCq0!XRVzlT*L`i1S7S4CEjfs6ZgEOyXsa2UNU|?)%byq_2g~ix9oh)|=Njp+(Gv z8NJyL7&c3S=*6=v$T<%fw8RxA8H}oDF_hC&|0-pFb!eQ9-ML%J2>sfD5$BfDxy9nS z^1)CLJX=gT-kZ(G+mXwT${O>)v6)e-fgA43MiDXa&=5kmTq7fSjiR!(BsU(iC|BSi zq}Z3Ki363{Fcuc4sxux+4*-+efpsCYsH23ycbx3um$aN%bA$6I5$d8_wx$ zGYg{BSG`m(j+ZI_TIL|>|17ghkOn3@)`ubO9ByYp7ptJu7Pt>Z^ZJU*_X2gL$O70{ zU|8{`1)|+by`=~c{H{%;pC+j#>M2SW{Z9+zM~(C+;6uut=2-MjQ~ z8cD8X{!>YT>F-qSJ1b1{CfweYB{Pa-(AsvNV!-29dYWk=^E}!wI7DxZacghxD{?Pq z$}C((=UEQm#AR&ucbzPq7ttaqTIb^w2Yg_Tx_&omW9dsu?O`V+EBNA}wY49|dl4J3LCE+yDWAY6XfC6R#wf=iQ&dTq+@Pw=DwE!G}fUGzz?Lllczo^802WR z3DuLd{%e7+OB3T&f4gve-RMV9$2rxlH2OPOivX*O<%4W!XoyNc;3dX1ACe^mj@>8U zzjK$F_cee4xM5?THwkH)+s+$(nw&`5n|^qLq0A;eav zj$T8vJDTZR$=ia!baZF3ZaZEl5J7>&E@)$ky&0J({Y}5nx+7jLzeotA zvfu`k*hN2t)jw~Ha0Y+La>qRL;=(LxCTdEBVPJ)M7$(MFt|g==JPR|@3{qJBV7HAg zHXgmAFYl~>cAMdPGK&)gTafj1=Uv5aI-U-}swuj)MAYfGoySpPyiw({9OKkcDV*~{ zHOCked@_tzeN+b>vKuh}9l~}|ciHV~_S+|0YNZwL<0Ut+pDT74$mu*mbU$I7Gq9)O zLk*KFS(UheRQ@G%!+`>o^G`l7pc}1|q!gbkU*>?ouB*zY=fsB>)A!HxMuGP~1jH}# zWq?SukDQei3lL1idNdPyj|^a7a zra=t&Q4cP#!4Ck$N`b7D9+4KMUYZ^C*+Be)Z79xxpBr7q2@s;9s15rP-D+?+^)spym^V|9?&^QekYv+1YovH@>rYYlgRL{>= z&px_8s$g=;Z3eikc5bcciHDzxXG%@6;)&jp-Z$P!o2|7`POW&;Y1a^fxbA#=wuCby zu&1W7m53F8O(hjru2)640GC=IB_Mj?NjWU1mz7d#dtQ^h@-*h1O>)jy{I)doWI}K< znr1~*Dep#qP3YF~z%@$gNPE&g;q3XBKA;ma6o^zbulNdyv~X(Fx@m0ITAngaj=ayi z!q@4x>%<$SR(}7{lfq+iydvnCBDPX1+7V@enXIK$pFJafNLv*@O!YDBi_8R?i{qO? zRsLRk5Kx@1B_lnJy@rC;5m10AFX%A0ugUDHN3S@~@@J7jg%f9aePwTethWAgFI@Fp zNkDrRVyE;j+E zxRbGv!(k!0U3*8jN6q{quAJgi0mxF!CirDJEG-K{+_g#A5VO}CR^-JZ}4FrRZc2+E{GeYB99WTf;lS$Uh#_6&DJdAyIiAXjuQa&=sECv z{`%*c>4Y360Wugmt|U8IDRuNJ6`=x4&A3X3yx^WPtDyNNY`%f7_m|i`DXcgT=F01w zeCPlOo_%~A(n{YJSAXk-|I$wDjk|{IwPpeEEv4nM(4oyp_$_ZI$(HS$2sFx5It}xJ z>MSDg^T>=OIkXrbAm(iODK=G$#xtV|%G5Td8O1x#F4j-AZP2Xg*$^yde}oPu(pEJ> zYig1i0kniCC4kw};SM2++&c9@tYcKKzgOsLB_+|Dn%|>JqB}0vT-cRoN;1DA+R%b3 zmjVqM37Y>-?R{ItrDIq8FgWdH$V5Au1S2~VFntiLTo9_E9_k2GETvR-X6GC%9F9oH zDG(19(1l*?(DFB~Xy0x4@l#<^Ekd=76d93%A^D>C-rSIgyg$P>G4Q9S;)DyGLDx;bWY96jy?r-LD)4onFA#-W5ejHcz9!0>pRovQ(*C;z2l>w`}1r>P{u z=-x6I%>n2240b(KdFe2)T+`9+Y{4w7wP}}i_J!)6*vc8R(d5y4&-)2OE()``x^SdSW?=`FhDXkVFJ_{uCCs<1Bbc zX-`R0(6XaBX8nF(1a^gzBig9c*V&TE@u!Rl6-t))3K}@|ia^I%R@p(7QNg=XvA|Nf z7tUS|mSE7``^mDtDb|mwSTH&k3@qg8*V5Klsxb7~sw?l}1`>o?H$9Ro`}7bzJ0VH_ zWQx6df*RBxXlO>bQ{)xZS`>z8&f)-^EaX=9E@&<&GsF_$Lj9!@1H*t72R_F^lcYge zaP+$f=i>p_d%`{JZSLf*6buC0N0vYO#G?rX4(DrO-OZLktvibNl$-9bvlfH#=i5+% zMR?w&H3EvFGS6ZmW5T5%ys8E)@p7dZnjP|b4QoKL+wDGtq|rf71)8^B4DIS^M@2A> zYs}#DK+eVKe`l9HkrE;1QZMEVO+O==Mnow`*ae8-!|Bs}x=;|>dp3*Z107cy$-&Eh zD!BUzClQ`#O#%cR1Fcqd6|-;(Df^64!Rg4XTT3p2kx*sf{>e=>m2K85Cu-@POc9Fu zytYA|H^PEHp~IBJmn!-wIz<}n}{{~vU(`h@-u6imd#ENu$X+>HCG>ERm% z=A!t%+kU?AS<%s-0I{(ccssK9cP-Nii1Nz5i(%o1Fyv#U>SmXxS0?oPAXXVcwEm7z z{KlybtA5gn%F*Bh>CxRXs3Y8c1C_4vNQd0#(%tY~-thLz=;Un<-f(QC=pa`E%-Kr@ z35fahehfV-Lm;Nt) zUg`q_fId^klyEYdziJbmzS_2n4g6SbO|dSH?$2v;OSmZjG2daG>R#PU?7{J}|68mh z^Dz;)@DxarJmr*i<0v8pQ?3DxSIS{_xD%ZzTxO#{W(7}eYv8kS5)=Ru7ExS0J`hG%I zpbq>s#8b*#U3Tk7q?AQEiADUFDD4Cq-tHSKoT<+U%JCIq#6wa8?iWqELd{e1FXOv= zub97a>6DtK!e~kKf&dGs2$La+#aaSDRpp=RrWZLt2a4#RxCfZP>A>$U^@92j%qCj& zAD9iw2^7ApAn75<3n`!c&pt!;;vW#wa2xZ01_5rtj{F((lF7i>XFFXdY-{O@{ylOAu>hkzx6!##QZXWxNOxwbINZX;MiuEA$G98R0> zUmtGM3PAmo5HKX>M-7`tM{JdSz`okpig^uU4Dlqp?7lsFvYkPXpYsQ7J+}12aZzg= z({QN1<_$1|&O_Ymb2yw0rKd>Bmr?^&C6ltWe0z_6i#AJXN=}yhg{lFBTbu_SYQ{3m ze4z#l10e3JvM zXVxwlnU9-)x}(on=YQYC08{G|n*M=>zhW&i0+V|$nCHy{Khyfkm-Evj`U|aj=>6yO z8?%*jC6?v_wnIJJo)e$CjoTyZ#uKhIj*c65yjM?9+m$D#mM*Ph9#^<+0aPtZoR)8{ zR!J~Al0qCDbg?exq%5DWiq-NDz^WlBEOEO2Qrc~5HrMC+jNjMYN4J0}Yj&?&+ zK>2d~b_^(cn{X9t6LHv3`=kIm(E)sZIg;4?urvMvt|xoEkMV7ni!to|)GiRjuRZ+7 zzYA>tVM^e85PtvXdj>8zKfsvLkvBmSsf5@_@-m*RXo-w#*7uz3LQ)h369b%SV<_e4>s#n&2AvQ5I2@a-Iv{?yRO;af7dwGh-cgv*9X57BPHkzst z*qO~Mfwjx_&BxAMZJ|PF?7h$18PvHA9D3!P-N|Iu#@mKeY3E-bk9XC6^;{#D)-qgw zAG=M@%>vaaq=xz@pPl}?M#ubpJoQ;2|D+4h?m_qz8N9(Z#siOR0qDK!aX((6{v8hi z-;Zy=CWp`44Biv-4ZNVXT6rH46&9A%?3&DJ$i^}?OZVVEH9A;Z8roNGCP@2u;M2J2 zpPON%2vSf9*I~u^zNU&ZA3nfdpW|W1FeU)UUuEZjr0x>!z(4<`^7^TpSDAHAG5-fe zUd&F*boRtME5GpwPxRZ)3nE!m-PEp|6I}&~VYT2JBD%XsQRw>4(M5=Bs zI)$L%4tDo|b>ZMKJ-#8V1T&ab$X!BEKsuq*XAZT^`oMf!yQbd#ES#w~%kwGaY8PYb z?ZZpZB|_S}?p}8Pc~H7;wyEdnIo&rfFrcme_CB_;{7^Q0r4+Y_D>J8bckJc!|5_2g za5HqkQ(MkGHNt*TP5@9*?7(5NC5pK3M(wK4l2y9&%2AcZmoG_t*F3ZpK-AyMu z_dBCag}NzM{0Vxo;-ZU*PCgIwkx&z6Ocm#%V|}9t=Aghe&1VT5n*a3m9cIbts7Y>g zeqQcstll^fq@4W)Z`{8CZ}aiVV{i7kmsPJBSFzd@jj}*j^mg}YUxOCgN$Hn)={p)n zLVW3f!hJu(Wl})3*iLfAwiu}WDLhjUTa+nNGBfLFbzeMuRsU_{F(H?k2gc9@b8A)M zb%gA}EkbTdQ4GGA&%dzRO*JIYg7p5k;#_&cPP;V#E($o+4<=_kKh zK(=~g6{f1~F_prF-R~75YQI&@VMB&J;1$q!u?03kbMWPGG+ed<72@d?PueIi@K=GZ z@pmRmZ0z1~nCD=Hf=A|A3*uezsGh0D58xlHB7t9xUAVQ5{ow#tZZ~Kyj%S)x6;`wv zJ5q@C(En22FPB3!uz(oE$^r3tDz+fl5#VWQA|24=F-fvmOgMtB<&7H6m>5Na3g7=o z+9mF_7GRMAFEfKLmbkFEmPW5Je%efg^1Fs&K}pTBk{`&GbLuYP3Nl7Ys$gM6&-p6P zvwbzP$3!CSRP*sWNuYxO+YzoxM&@MW+PaL>j9R$*Vgf+6ltxz%0lG(_F!*QFdwEsgL2yp*@8hyNkXfuKOCVc8~jY1u2LQNC*(_^Am> z=AX<&%G{jMt(4b1b!}FZcRQezlfRXZKXm zF?s$3fAJGXS1@>mVbdz=HD8Lpx3##YvDZS7J`D;xT%l$TptOGA;(IfFPoTqGP*%vb zp>#Zv;Wh#BLuFV-xE+`O4QKK@83rp9!z`RjHa)#-!M-j>uaHJ!OcEqDoFY;wYG$x; z{*x&0N)_jZ5bP_JOcmQxGLA_vJQkCaYw zCzys5i^LE@5A3)JfL5NNjq|Mapc45#k6chbv!q24ah_x&Fr`!u%BBb-{M1*NlK3c~ z^!^=MWoctbW>WvGYKVr&uM&4eYWm&0Pfure}&hws~DeUYBZ3n3Q&|HJ~Z!uUg@{{gE+rC6pN(owdVqbdzduiEcQQNQiCtW) zVO6ccAll~cBa{k)6KPiRA#-YdL$xgWx-$BrDF{SHMESKYaUg#?zTkGyclUj8I%W`97+qFO>Wt-gDPBOz}!n z@qi9Q7e-GpsUc7xg^9_6t>XUWKpIU}MdXmdD@8Y-#_CL2E~BC4>Pc_hV|Rs@``7Wi z)6aNbyUnX-=~r_Orw~Ss9bYop!K%!Kn7qzu$zf?sKB++Q@%nyc_W#W6A44J6-Pr|^ z_fyOVPD^UUETV24{vlMyK%Ts>>V zTa;Sf6J>}?s3+w!z!_N5g~_|WW>+alD4ehihi#{~I4v*A$u9zm?lb&0E$9bs1 z8jps;tB1=ap>@){6nW#m+gd zpzE*D*tXzOlKs3*H2&Y}qwp&C!c^>KqAge^(lYem3?#l=cA*1>)JGc4{odo}tQZo6 zz&$l&#Hb!kNmsy;G_b&TYx%@E`oY6mFJy7EexNL2>o86E*4(W1ctHGbRAmAc_$U(E zWAq7`Is!brg_MB-@#WPOam)eQ&}A86orT97pD{vG$q z`)&sl(PRF1on(ntJ~-mF{YDBM;#)Ew-YKBb=Zja&fY*mX8k3b_QPG;vL@m}CKxuIj z*eM#9h1x_BQp2RxlBd&;3AgCR^ht^el(8@#&^XlNrs62~x0=A2g;$oujj82FO>cVpt>rQGSG4$3$qk;o1+9t()Eiq&SCvbwDHXc zsD2f`dR4w!YQAhKx@3VZJqO*r-w^^oBJC0NA*t*5GtPa<`-JJoO#`BaPtKL2|7{_u zE`44K`=k=+AblRfW^I?R&;E6mM91TVD9%G8q(tG+AKsJ29G=VNBr(SWyVYZ!tp^iO>0CRdf?hnp%5ja#eh8NbN_>Gwe2ZOzZVw4T*9Z-%{S zb;_mtf0-l71q0ubkmCgJMLnDZS%z7~zUJvL$~mj(@ht!TYHC$f;delGd{8VS2Jsra z$|Ik6F$;m8|A$X8Q2{^RtX*A@gog@o6k{(~lI4$r3|>42hWc^AO5rbtBI7jb%%G8X zch*tv8hVJ%tYJcW!Z3;7RYcLHM`M$KI;(80BQEVAEq_{x{md-6u*K+%TNZjgDCrrwblK z<2`)&;2=7GZPD@%HLZ6?U!LlqC6$NEXmYtl^DV4bHHWikNHwr_?w^a48i(^(*>|d) zNcFe7_xOTG7~62C3mGuSw{Fz3wa42Bx#JO99O+@j*5@FE7Q$QW64}{O)jFFQX}_}V ztD=oj_tC4rz6-g!`>0OeUq8R%w;W)j@^oo$2I%wbPx{TssdKVLaFGSy046 z0(n##p}X)DYvy^P)1`#cGy~gC{m2gb%?~01ldu!Uql2&=*QaS^f}5I{hzU%vKMhhkqUhT5-F4ElyY|i#~C_@|3 z59cG9T_3 zUfHZzq8~`9=F76pboWieVk{(d-#@#v{4B*QwAqO($XZlZ zp%L|^dHuvPjYK(@*Vq|fIj{5NGswR$mP82yk$DhNV=k>=s*+b~ z#s0YbrX87OEvX(3fBWQ!Ke8BM$?(W|ySJ0a@xJ#5mSx;x0gPv&t=jf%!3=bwuJ7;# zHFl~}B}FQr&hziM)ft717XYB$X`v#C`CvXJ$9RDk+b0RPk*S-GlhKQR7E##`fSJ`| z@0pm?e40(OBrIUQr?TIiBQgIyLHuLBjqt=z8XBEF5)BTxE~C7S7gADNKd%@I1VW?yWYaFT*0Oe4DH?S0)51!x}Y!m zQ&Tqzq@9mZA4uB3ULdm;4~WP<4~yjEm^l$H-aIGBt}o`Wu^(87MyEETC=aw*OEa!x zQbTV$D7yP$%jCiE|Cb98jhQ}qiC8|S6IG?=9i5B}qXi3Hv2Irv+%t=tFukk)tGCA0 zdf%b=_GIAN-gf=0XSJ_J020fTRR!c{qmXaO)a-oVS6?>YRh85Y43WS4@?p`@ z#WfozeD)=%+ASa*hA%?OB%0>{ncni5u!$lX;pqalp0SOfXlG1mILLM1Z%zN}q7LQ< zwEJJEh*f^1OVR5%{7>Y!J6L%vsNH^`r;AR4yN?~^h*;hjd;!MH);wZou?1x>wjZ(p zdmpfB#~psCe?vPglz#%xy|u)FJIw)>p4DYo7yYus>?BV0ERVhz6GZZ~*hv z9M9vqlH6uI3NXqlG+b+as#%ZHo;+&T>x-*Q9GH!e_(UY{*o0Th=ZYXb^~IpSyhK8e z2Z#BKY)^OJ6%(m7%@odPPrl~Wr-OtW1rl^r>m<<+`X#w+Z?vRhzJ>}6l$#+-*p zpjCZpfO_Uh0FEw!e;y&hd|eYZo?3-}j+?^n&H5>W7n9R2KAPs^`OEy)F9(M1RZk3Z z)A|K`t5NllKu5l-<14#QMelXnl%9X_r~AOp>vvZt$+ClUG`Nv=BS z#qiYZD^BE}IwTYb8N)u9{}#`h$pJM9+r?7oiUciNn*iX+NGyZTj4HV14mq5X{euq4 zFFhuk$Q5Fh0AhVK7Zz|oK!M+Iw=;mT%5r0zdUZoMZNJzo#;@nc4^!e-*y*orI~9Es z=Q5IarZD}L9vKv|r@#11QbLafXS(l(M!!CsJS1gAjSzs;}j~2#-6rPmkWF@tPeR@#sSR?Gy6};jp`}$3?!9>J(_VYgR z^KCO5$2S+mhtx3V6++SeLqwG)WMb>%qx*qsmpFhWDQ2Btwo5B;2eX)4-{XfFg778X zjUoAqmH#}V!H&A#djHxBG*rQk5Zz%j44)q7>Cprd^YqlkNhLw!>E>0O&-KU0j%AcA zdhIZ|3?;?*Cwh0MCY$-k7xw;La5`&&r-?%TXURwnq zcT8C^Gp4Ai&W~(lKJY7T=UQhyQmK&p+j%3~v_2bp#x;E)H^mY<{rp@qy`qtIL7iOj zh=R`$e$9)3EDnz!jcwP!gHnY^7$g7`z1GvvAqdVS>h!&=^Up-^w`m*=Cj4Bl7joOl zEgNjK3aT|oZD(&t{Xm!&fIKyG_e(#nvL6lV=zMJ@NqmBUO~TxWizcY{?4iI6DGQ`Z zez4v#)kW$!D~t-=xqdhB60`?T5&H+QO$g4!`9H)~WS~s3&t@^jl2i@c}7now1oOu8mN-kL>tp2PZ{cb<$@%`-%PA;9X z;Y0Y&H-4|rrL7O&_-^TCzO!QehC$wHcyiCl?~Skk0BOQI-2Pt=m~LI6_d|Tv3nt%8 z6xts4r%QO*sF9wVVbQJ%3WG)QQwvv%|O?W+BePA`SiQ1IZ^W zuKJsS=f3j|<1R`1&LmJjv<{|@SmQ%!WZrqAS3rc%E4NI>Em0sKNZG>8(A(Y$ugW{6e*uo0OPyw>wf*-vZ#ZQqjiSX70F-HDCUp`0RB} z@GzI~8*mzNl_(8Q1pU8sX}0T@ibW>Ma`Z=ZpAWlf&z)@iVk^b$iEUb&k#6^}vZ>xf z*M$8v*ZAB=D?Q)rv-AMeN(n%fV?4E2|2JF$#4I4$WlfeK)dR$ybY%KlN%zvwOt)?7 z3YjhFYp3bq1ZBZSBOQ9(!F}U~JLLoC4~gv>*Mc;r9V=+cAwTkGePJ zyv`%@uQWD{;L@(Lhl}nB#rs_iuj+rUhApc?B& z={4FcP`aUl8V~kT3?G86h(UlV;of0@yN1L8>aq>1@iKhmlgp#yw`TAv!cGKX@y#?N zEZCjqix9oKt_{!O11&ld5CRHSG2O4Ov&vT3k-0B$-vLK{kF~ds_sn0YFNLs+Vyh}0 z>2G0beD-W~+%Cqe`PpGiz&jRXBG|ZVl>{`*KgLt*m|Oo~@%w(r zZ7b^Ua=jvz2qeTo$ zb=$z42ZJGD0`HXwsR}Un2SpvXhw+WCbwacu7<+YmC@l^vRB(Z46IXV2XYT0_3ACTg zZ_$ZCC?|`)L@(DCo|#Q}r8y+o$s2>W6^+;F4uPnJS+DX^(joHG3Ojs(nJCY?`C(Kn zq!Th&!CL-97<}kn8Iz;a6r&n|Gsryxg8fTUijmtcMIxRc4J{XMi zS?BnewROdo1^JxfjMO(Mo8_5Z3(tf%R(Q{X#@j`0MZ025)l(hB=O@zl{FZ3LFRHMK zL?SUyw*8Ho2UM5h>yz)!Jyh{WSEDQ$S|LLmV!8U+t=bww`4SByKM@5H0{nEA&?JRrO2ah8oii1}EFB0Xs7eM%{ z*X%HuN+V;DbdLWtn+w?7oAK69nyUVo_$ETsgf@3^Qs14lxIa8qemQJW#8uXA^S0PN zFuvS6g)kAUIV*)0eC?yQtx|C=em*tAXt=VtLvolHNVtPgV5XS291!Ah-M#T_yKzga z23futkfYvtL@oBSaz&IuJ!*`5nKbbDcZE-2Ku`hIjk*x4f3l9M*M#F@egvCeJf7ge zqzZKE!psON9)QoW0+Ypr<{i;^r4L}J257CL3-%L}KYnC;$=%C#5Jd@hgbgn3tSUaI z#VjZjo<&5HlvZphRDxD^>M4Q|u>j8!BUymSh$j2w#Ha#7yE3-`^A6g*)Z9YiN6bPg zl$vKL0v9IbyXHw8_C0)_V8H+v#V}G$9aj)^_#VzccHzgHb}`V9p|F12lZ}a@#pxLg z7lRg9dSocq{N}qUC_o#yNh=UC!>5r6m}-!I+-ADLNHYTf;Fvd$epaCHsCJZn%>s{4 zh`NZoY+%}zVi_nYpVSyn+0Sgu{Ac@o{YeyPMcy5GvGo+0>jJ}23^eQ!n#6^st>dq! z^o$pkvftHy3zJSjejdpQg*Ac5!%*&COTr^X26uq=r$}kz2IA%d&Y95{H zt{M)K{beK?Jk>m>Wx--wf%+}g<_2w?@vP!B?lA!pTj<+d4%23Px%VYses9wJHC>|c zd{s*K#0oG&9hWQrHfmV^=)(c0iK3J0HA30L{@e1FtNH#>@h1{iy!_c})H@5NdX2fZ zwQ0?PxXRIU+7R0Vm_GsPQtzciVo1cuOnId6l}w9!vYB#;f09S%FB8l29-vi!6LOK6 zaGy9T-5md(OwqzDJTy}Uk(5SAvM^qzE=+_NPcrfFyaW@@V#kA=lS;%uMF6){{+HyG z^6c{j;v(Wmd@jXKNYs;qtJX+-cJf3yNPAfG$Sh2qTEcRNQc|H=otmiP;TWJUnt?uy z(m)n7^p+P+XdK#Bn5X2=T@8rDmxhzOs#>;^ds+@mzvm3$nfsb=rp4$ei)Q7SNUP3l znP2gu$=W4_MvcQV3o*;mCuWpfoIL)4Ha>w8@O9G!q}^pfDW>n3fD)r0OtO3D&+q-b?le1Z^CX4DmV2jq_USoJjn@){CQFqVI9B9GI zq_JTBP^mN-QkKrvdN;{HV(G1}q)BXLnQLtFre&Zt zBE4Gc#LtRW&N`(kwbb0rtkyU;CjN3w^+lRxnw2;0Mf2nZD+%(aDqR{U3*5HOSDEtk`uUOeudP-Q6-CaadFi!|k=7}9 z7}~a3>X}z@|2!zZa4dhQ_`-pWFWl(aI<-v;?e^eornctj8&V|m3?Pgl{<2l{%Fn~s z_oVgW%7`ziW=&zN?B6|g>(SMyc!h7wTh6-1 z=y?A2YN5Tx`qu4auV!mMm9J(iZ`Si)0x^3ad_+YA(c;Ajs5IMrJaBg>0a^CxSGXoP zB}bB7P?9nj#UwlOVsTQ=qZ+c(Xckx{4Kz=wuIjA}PAO|X8TT(hSsrt~^me>9ZMQAP z+{Bd7mL8pHzbzKomV2gLe%o&c3f)z9sPSINc3Pl1SxZ`XAr@2a*Yc|nfPrERoQqVt%lacWGV+sn z~Jnu-)3b`YdIirSEqjvv07k?MdNFpkzC{(TY<`#>wabpa<2Hc=^ft; z)%`x}^Okey*Bz)7o`3#}C!zoD23Q9zqKy28xMh|a4zG8~`96U$(`y4$L1&v=qD1dL z#+lFOQ@Z3$o67X3kwSw}^(=oN(w|5pf-i`WU|oGbfr96Jip}KlH7Jjh^L|7*2K-ZQ zA9~e`t~**)$3?0}x`lsnprT3cD7ku0!Ilh%&Yu2cE=%i^k~Y3BJGQ1ao4zczb^h%A zCaYSrl%%*gdgZN|^pl%sm>q`+lEfWzC|@J^6UYf-iY4 zzg@@QXGxPBNL|*ewJOc9v{@ph>#lFNdo;3Rm`VMC>EbI{U4RZjTyefTO^uMsD%0%J zz_6tzj&1odhOJy_f@mdi%vb!;58Sq`T%6I-Gh;-5cL(iqfft=gRWiL=1s{PUCuaYu z_I?%g$!jUQ8R*C`LP$>yfK7nyPXdmtpm|DTYD2b2VqqA=y<1r@p26|o0o;pDA@h+R z(OE83vTUf-RKAfw&$;jCH4U7n6cOP#pd8<^*V%Ez3cYFAbFJGcO8V6XAqcPHLe@$N zt}(PJl!$yX-&La1VF4zzE{CN5LaBBdX)d<2c0Zu;>6RCdH^NNX#}ID{O9WRfGVI4J zP4JStDE_V>av?lfo@?P&d<%b9OkoHO?K@+3I#%e`M9cn#K(huZvNI@*59d z(rI2pjOoPHye|81w^3*drPz@?%=)?d(t}SoN-9TB)ec)ca*TG19i(o86bdOoP-^DE zeF`T5;XO)(9ei!VJ$%%gWq@KJc~MXT`W(?GmUT-&_<`_QnTc zR69T7UT|uJt)dq;wkmixhnGtwdi0V+U%RbC=WZ z@4>|eJcpw3PRgr25nvoj_p#f1mamvJIey+NT%4wIo&*O#h`&L^_f50$x zM?$~*7!$_1VESk0U);r09nn(ik3O#F|A8Ta5C6I;pR-pW{N#+?3PX`&JmA5R z$_gH!=gBgbNCq0I(eLMefAM&fyE42lRWySwvKUD%q4!OfSB{y{!4V?dCVX@H<@`X<^#--oT?60CeXx|-|9m(XX5BiQ$`m#n4YRKZCi|P< zy-X^p{%9VIz>VVxvl-}U(Oc#EqY5b<{7Q;paL#+Rn$x(`1o#!^q~Noory2226k`-j z953SxYe_<9DXTG5;6SA(J)dfe2n^&I`U7}0J-a#meXKhn5C2RrNfFP}Q~dw*Mz-w_ zyQvPPdAIjjsJjJiz$zjLIx+hXYe6RMz&`N_7h8e9gPJxV#Dg(;TRK0+4UQjW%>8&} zPM0-@(DQf~UWPtAr{F*oKup^3+a#moa z*e7~xa6S+RI0npeK>DxAg=X@rItlf-w}h|07MFYLxR}F7xUzWJ#g4L~A@byCc)qz% zx%?L?Uy~2zZ;OuYy^|TeOBtr27Zydw)G<<}tP^!R0}CHS(ad3c7e8kR9A6lpMLe-< z(fFsTpsGp1$}yt8EkyvH2%mu+CkySQoA7rQ^~p~J2#gdY(-JpI&GD`U=(M1t2*(Uw zFHHYKXUHVPz(f;RH-52dPc|XZgbn^j!e0Tl0xPUwndM&;Qy<<(Key2G5iMcAynW50 zCoS`gmHH_3UCYU8@dx}0;c+PJGpq`Y#K|hbNQV|u3Mmw_;+S8LJz;P=r&#EGYr5^l zG*MR2Dla!guGm2OcZ|t`J7HL{qMvM2rT$8)VU;C5z+AS&$Ony7K4hb(HNWLz$I%7% z`#P(9)wN*huSWDo#c&6bczQ#!i7YF_;M!~W7fWUkrDl`mqshIy(xu25;R~Kai@uWc zxlXyO!)P>z7MEdj@LTtm_8t!F-FW6hXE=PH{kAcH4C2|P)8)Ue5dTr>0DlP31KChL zUFx>XU+u(KiFv7C>7U;bJJAZYC&9WU1KJ$MR7jt(ecX*wWB!;Qkpqg2ttiO!r@jSe zD?ya*uYA#t6+&dkf>~oOVWU;)73myXlohecj(}Z=2WfsB9}+1(+a&!B@jm7OF8E^6 ztuJ!{E7#p&KzY=NePNFu&=J%zoL|GN2}@3bnDayS;ufyB+$xp!3Br!PTDQxL=!-Bl zeD(#9>d~XPrkgZVa**V5NvSFez~R$UVu_m|RZJ@$E&Vy9My$)aUy9{KOKKamT+odB zffmZ*I9T>aX3PzP{LK($vlNG(sSg z!zFBBB0a1}3b9H%nBV3Q?6x=9)dZe`GqNw^k(-J!Ur@J+m)-HS%5jDV|89ppp8M)s z82}@>;W(B<3dB`Hv)O0`Ftl-z2BKa|LGU*7hkwa*2bT+hkXw`(vB$=buC<>&v-QJ9 zG}P1OBz#z^HKd~%Ik$_6UPx;D0M_^O9RvVJ_GfUT1`?Sjg=dza^i+v{qPAwjW=yC4 z+PrHjCve+;GdO5L7+0yLVp)M$uQyM5FVTF+H8k0qD=+dhyjZOaWz>41f2LA&vWiaY z$msX-5|9#h7y&*>WPfhS?aQLs47nmIh7{Td_UTZUk697X|IWQArGTnEES{^+y9Up^ zuMkHfeVR2gj0j0^q6Mz&M?o=NT9CkL$OKBXugd_fP-AkH7dn(4PiVS)%f>`^^a5WV z{nYCNAZ?f)w+J*lu}=6{&{|1sY<&;k=CK+B6;ZUVp@__Ng53l*V+YVdayH_?7TIGE(T z+r4oYhm7YAIKDPB^pZJgMLC3Q{QPWtoSJZMrJDO3Fm5vU$3mXw$)59Xnw_#Xxk_p4 z9~p<6qXzYOsOCtH!+WK|%1LJM{=Gu#m0bh7U7-b*C+vt>On%0H4$(2YzT|v=BSOeM zYg8Unp~VbP2C>n|Wk&uZ=&X7C$>;8|#=kR*=cB~}c+D;4~;sadlG<$G9EA2N<9Zakv8NZ_e z>)0c>FwYu?BA?JMThEVV$GzVj&dnboW$+h}2R;uN93?vqF1y28rV6xXesfG}Qf7Uh zimW`ju8w)H$}TfX%QmLu*ipX$l-OcEuT^3Wty88t&2qUj=-%PEGwCB-s(De3&kBwh z!MhkiY}>h;>?}3NqTN%wnst6D@>kjpiqG1^c%*n-@hqErU#f4|>%;x#(I<}n6$YH8Y`$#yn>-=l`e%o=R z+i_-?9oXeKjWE}^ui(1A09+K*2R(rElYh8izxB!Ne4T4EDX?sb$Zr4^z;;QCRXeSr zH!ea9EU;jy$c0)TSun8Nv6jV>Vq2SY!X>na8MU*vw;Fc)LJBNL=fSc*`Q`uF%-8B4 zl@QX8L1gh|VTbb0ul>j4Q}s;dmpcC-65PK6|G8+jW{Y#n0v9KD4{vdnN2?H9FnwK(r= zXERok+~9_V+{o(0RNq|CLHc1iBxK@`k6Z}wiOJNGfqk6bqa<&M53e&HKB|m&rI?oO3uU=J9CV8Hr53RHAByS23SbQr)P6meW}56Cudkwsk?TK zI^Pl-U>4!)-vNYi8G4AlY=L8k-B)QyKiRcDQ~|Hw+GynVkEQ)#p(sY1E8OZrb$%YU1MuzKvD!Q;C* z7uwAMfb0wF5QP>c1E?asRImWdpa4wV`Agml=w}z`l0LaYC%6N}1BO!zl3;NyhM6kL zf6K~o{~491(C@1rTytHE@XXIxMb6KgNtM*(0ky2&Zavn6E*=|Kc&-~n<3@AW)DyqZ z8)r!X%Nv3Zf9Obcd{z>?GlbnYm#&J9GdM}l?MsR`HI_`<@+$Hh&0BKgQ+?;3|bU1ff1(-Z3& zhha+(AI`DWWHXTF_`le|s4>9$xvDf)qQL=#T&XnG52@t(8X zn@t9`Vb|~YWJ4I(8cG9mp>lBfsKlsQD)5_6gDU?YUvC*#W!HTR(fXxi(UVXROl|i*Wjk5i^>hmHOUBhkKW->Vx|_;q5}#SP1B( z;c+R^u+3bnSIyq5TFxB4I@Gc=6Lk9KZj6tx-=CbB!z#;DXHLH|=@Xrn1s9pXth|7P z$4J$f;S2DVEf56>Gq7GIp)(<$<^g}r{`|tBeXZb?)cJ4lEC_%Mn+`7jExND{pg)AM zC^H-sK{^nywXMfErT#m}o~ZEU>i10T#!WqZ|H<*lct`Z0_ghTdx(@`yN$=O3#w<+U z#V3rTl)cwn|AzkqunpVFbX7Lcd~+xROa}^TrV?z(Ij;qgk|^y~bhytfg_}k!nQSoq z(F`Tk2OBSy%B7Ve`dq(oikh$(*=Iy&&evRb*5_>`F^yomC1VJ~;wdGk7Z`seeW{2}{(g0Z%~M3gJ;qa9<)^^I zd_~}>FxUUo@&A5kXj$;lYyx^hrEM+xhe$7hnhcp&OA5Jl2&2#L)MXDaLI zpjTn`++$m;&v2~-Q(SY+wh9q>F3yqkiYHUBYn=u4nh+|6NC6n&-BA`0^Zx25YCy~b zSJ^uf=pP*?;rX@{%Jl7;*wV7MON@>muTsx{+1jM;)YzZ#*2lKCA5g|dPRqMxxn#5` z{+kE;A5Z33^ac)?plw=x#wnA~B=&}HA)4Xnz@z?jcD#;IKC0@I z+^{t1Se;qr;P8@%7mUd-LuBEdf>EU%s^UmG zujhl>b~}wIfBo@tJI}MoW46(S!;J$f%K$mHozNhW(|ItW=zz}u_P4I{=4S+XG-Sx& zItao!5`|lWGbZz#mhuc$=t9jh`?5o7gD4{I{S~csgb=>da%ie2r?`SG- zUsH^zhBGca-~kQh;L7G42Jzi#9^7wb)1sP>qhA6Cpk-Ynk}0|CECras58elScsM-z zFP2;aegM*0_J9cb;#-hsozB5yU@HDj-!-2xpqDLZTND6z&p+Y&(*C1<_z%(#1=Pcr zS;00?F9>w-n8=RWv%a7@6om;)`Y+zR-+@}Od&QDV7?&uKc)YT{h?5McOU>wjG@f>r zN%yq9e1eXV{nf_|D{pAg29;NFUQ4p$m%|G=sYl)ExDcZ!cVeJg^3}D^OK`we*0Sc3>uu=J{eQ+!b^tde$#awGy7* z3MvHPlNGgx#NzuCBz_FN|`XdFeg*f__e%>4r5C8WbgVs+@09o8KL zPo~&8U(TWRy%qJ1E(S-WS4?uHN`5>+o~!l)7~>;|AhE9ZW9Dn4-n&0^7;$cB zlkou_m;*aSTz8ykK_0!r~@l{yCT^uGHW@vOHjf?;?`5tDN$?N5&+ zqcAv~dx)j3rz20XMtun*tI?kcF~@kV23{X5*JBDJ*E`-2!!8q3YUJTEz#>$Fx4rEa z+YB#a#N_y=iim=Hd4xexJnw~s@5W0+z!r8(OH^GP2EdU9vDp)Zn>0!D=HY*mY5c<~ zR{`_!51MLKuwry}*GadfjtggYO1FKevV~f-8&vqZgw{I|C6=&$r0lNp&=J-=EOq?a z$x-^?sTi4p*Asp15gtoEc|GOfBPXKDIAfYJnxg0HTLJt*#vulwx?HQAAxal_0prVY zNSjv^v=-XnzWX0*yfksdFARz9OilBD_E&*7kz5c+N~cX9bPiiSoxj$7LA#q=ccMh> zPlb|iyw<#!^ttDwob*{yvY3b?sm~?I{icza(s+nb3)`c@CL&ChH&bhjR05bXZYZ`D z^FOO4l+}>QC@KZ(?+d+*icoZ2hC#!|wOI*LBqMlxf@Q@D>)pl*mC(6$Gs7TRzA)4< zZu?K<3>>hZZvfs@h2*RKrU2W`GI-T1Fs*#E9LjbQqL`;a$(qC~~pCLJZExN54DN@L#)L3Dp} zGMGIFxT!;=k~t)LB21?$@#=NPn`ptX`0wb43)&|nX`$nX$Lq5x&IAwZl+HhbD2!2A z9}3!>cuAZJGvF;z4yTVgh(r-r<4H-XZ5Ie%5u9-4v!uuT`e|vh*Dln5B*Ha(e`*dm zK@;JY3@2&45I)f@Wx0tJxK(w!)@;I7mR)>4*rZ_LDP1saWvRyt?z`J9wIF@3cM#p4 z?v}?o^rnEo+w|&uz!m;*8TdMzVQNZ7P;};_xPygyb7CvX(x@0Aw7%*`@j~2Dw*VhxY2bfp_0nL7PWxhXw93Lalyyh!5-*<`c66FVoPBG@gxo7sH@>1wd>Mk2YP-i zI&0b;_`kHKR1YMTD?A|H6Y^Yd0YId%QV_z+a;OiNEP%wIjdaV7c~&7@aO@f?Gl5B$ z2K_wdtkmvLw5;E)R0|QuvEYxXD!l9xy1$Ha=rC`$PROSI4Civ`bh5e=*Yh>@SvEvS$n3unrwCH7sXyzs` zCsC$#kY(LfYWFB27=Nkgi?!?6YTxRG1=8Jl9xpm6?f- z$E1I_!Uhzz2`@wtm)nWG7;5T*O}QuRsn(D_?>6wyv%l3OuU~vsX#9Vk=9lSr(WF!+ zrwKg13r!+Hx}AP|@XtY?AYjlZ-~_F>8KL9nmUfc4V7Q~;Vem+Qo%nNmU5t#Tj{`6x z;vgpSsqwT(s4GmNX!h0Mnp|g`v16`cR@-coABTbeEi#i9KL+*=Q%}Akr*PRCc28Fc z*}5F!RdE%NPL1F`qnX8_u*B8d z@?X44s&r(wrb&tMl{M_c%zFFvQxcgfgMnb6Po;uZoSw|ykx;=NoC5c*fTCLFFk{-6 z!-WIp!60$;KogSDX1iRc!@69+b4GI9W`;v z`cL#W=$HqHM)`+ez$^es)2Hcn`@49ZEg+fPH$+eQTBYIbt}b<`ZM82mOjzP;Zcea7 zBg5QC9@B<14%2anXf&S@#K&{>)PM_^dBBO@lb}kAF5+8`K6rjW6liuA?c4rCa5sXv zJ6`Z!FUdhTDaENC45cb)z?iEg{&lvLo&QRSVc93(1E-x3R*apx5-w9`a{C=15GFv8 zzrX)TTKa2e{_Y?vd-hP21E>7=9o9EpPZRlseA|}Ysf~hS>LsyC^%xaYtzMZoNMWAn)l3d*B?w%SMGPZUEwi{0; zDCWTCD)}-QnLJ%wvzhd~Cg@~U@!?0G$iKyHIU6)&yNQX-#hMnT()%32x6C|_!mV!^ zrB`*HIxJ>!C1Yl{5XqriAvM$Y*#R^%1`G$4yCrvVTuH$Io+l0jB49$3b4JEs^(zon z5jqEhvN>G&7mtGdJ79R+E4j%S0D`;=+pYB_cc-SlaNbG zd1}AdWo>5c-*s;Xpz>9gDS@3aA)&(_!AMfzMtv$XLuI}Z5S+&9K@KzMeZ!bSu9rd+ zTIO#A*TQEI#1&1UXB)Pk6Gl>Py4@;8bmg21l6h+0q$*WyLyDg(Txz|b zq+KqD+hqG{)h`Gk9K-KZx2$2Lq|HDIGs{235uaoys(F0`rp=BeBq#x=8B|T)Dorq9 zk`L%;Tcb`orNp)$HzbtNKAW0zoLzVvbj1s9(x}tNx1TI>XlU(fxdQzoZIc}B4lL6M||Ik|)pIR24`%Yi%-HKki5EH-omTgY+ z^OFi{1MIS9ZRY^M%%6Ed(Tw;K6dOZg>i9EjqH(q@a&PnTgK^zO>+u%7b|3eioLY(NyM9`nOd;le8KzOUUf2tQfmTqB2fU9#%y zp%@@~Txh@LWVD=Mt(6Hvf1`VCa+m@elhA!>1{g1}m1z+fDwArwDbTr34A?JvR%4lW z$F}J*(E>!8ifj{;?xwfvdyWe*XRRzgD6(wrRp^!r;U{V2))RQsmtOkKHaU|5RII0? zO-NeDFJM)?YkLUW1NfDt%amTIPDf%_vB^2LRd%T}S9x~PBme`>(1$OxA8$^P)U^M% z_rGu2UN7vuyz$TONrKrqSOHC}eQjK?3;H$I7f2?mO}F-E&yG{8jZc4S@$9qt3Kp>7 z`HkZKTycmk=|1kw2N?L0A;(<+1K;UC4E$Qccp0?V>%(9GX$wBdlboryT5~e$Sj}m? zDBTw;4Z7$>6<@S582-rb^t|M?T@~>0E`w)xo*ox{N88G8xtH#m&ct8FKnWK)N4Kb6 zWYS}ANmr~@@M1NOb?|rKXAD$}mKbU>MX|n=WUY304E9=dV|&izzBysp;R&UD#>iTX zzAxQ~rhOYBY}jIN1T0`AVl%ZGl$QQWlUT>%E3B#7*ZsbL)t)M0Dta6Mv;1t;LbpEh z%Q?VecVyV>%bR-^HHe{P0i$^0|AUv6dnI=Hp-PXoUhx2QU`a-K%(%?v;oBmp*Z()^ zgbFxn62yM;X_#Ith@MOfW%aG@@U7kUJM?BX8$>V%w7hP3-j;i~io$EWqh;3fi$}F* zc3H#62cJYo>ou;6;QM=j+=o*XO|Qe6`kbxPQ`0fLm;Ge+U`y9fLbZSFN8Cc;a zAC!iiGMrDhd6U5i(+<1Kjje0fGf;;hZX)GZJ3Z>-Na?f}#754gJW?pM{pZmy+-$YWATn0Y5y!WlMBP9U`98+^MNv>!A zdX1@oo-7CEHw$IOuEcSeIH-b(in=rtu`t?S@_VB39#kKt!+#7!;FMCUm2kzBl4$!( zpU8n!>W}D3i+q?mrvH$cnqG05efXGUE6Ss9tAe@)d2kgCv(4=R?aufDSs1j1t904t#E3g>g zed49)kfm6!n#uHm^Gel_=rV@O@__i?RJ;FdDID4N&rp%8{B|pKufNyUCNk)YKdHwT zsfPH4P~Cqzw2;1vNhG{q93&$OcsHlPm1Sk;uiWOqAcz-mpA`w-;dRrk)KtyOsBDSX z!1nueQYW|iV%R1?uSdJ|?S;w|mqhjLKV+u)&BMt2y%Hv-Er%8!MRP&YzgSKGWD$En zz#Q}tc(eR6plzNW@1~BOa>HI%(^HmIdPd-AFxS^V8#AoX@2>xgkaAa@1Z?wJOWz!7ag|$o=;uwF;ampf3|`DOZ4++ z4JR_N`_w;hw`{C%Dx}WjBT7q0(a_TOu;Z)JMKG&V((*WYG&K2<7^qLma8J^ci&-g^ zVqUw+N8}K(%Bz0X)Gx|v)pgOFQr8j>vDlsFCHzy;H3~5(2T&_bT|&H&v$mVr59L_V zn%z7=Tq#hEpc(af>J=>bNK}cw zW7GLK$Ir%4jR5neF8j5U$f8YpTve$}N7zOu~ZF7tr;4^fHd>&t3EH;p&%p!BCI4W5Zz!u$9* zg&?K-y-ET+Ud0;%v$Za79&olwKfS)%GMNtz$XOZ9@d_As8_M8?Wx3vuFyEhEYyFkD z>cktjP9?nn!z?MAu>|JW9?n9HD8mJ~eoh|ys3(UFVNR+zS!^@cVmqKd+C`W(hwVQvxthr)An04xEr#$=qUAW`XU7AJ>VTT=okg+FtKR z5&QSWfx#`8%)t{HGD@!T-22VH3^jq6vUypHOeZ9=qS)^7i|fT^eR0r^4bNf)WE(vq+9?w9O7{cNN4I|EVaB$n5MrCB@j8cPT&tj5n=G43$ z90#4Y5qZ(jj5Y$*2`BT2>bHs#v%;jNnrksEP+M$Ld5vHCo}lT>n(aS>If#rCh~=2O zr?P6RH8nLH$B?x-^!8d3nO?l4&8q%{i!mrQD2qA*C^sm#@~MLU6?BLiZkVL!s^v~r zNXl4N7as}ws1VJr)L1IIB}zsGa}TT^DdK8>0_1r0app&Fio7jQAHv;GjNL5AkSsM{ zhZm~lYL3tXEBdsNYB!m_O{p(!rsALhOt)|$#=6-@mnUbTM%n->LS^D<>}w>ih{nmV z`vOA0O;~Q+rW*FPbD8CJyfPk{L3CQl^fbEPmg9dcI-+3r=+(^H7&ZU^cfd;ar_N^5 zrSey;?vC@32UzQgA`v#3wXvwNoX5vBHhVD0IJHukLrrO=Dy2#+b$Dqt53NxwDrlR5 z__nQFew@3=HqT%D#7r(b{w4QO#jO~pi?P2~e*DEW6=x*>L9@j<##tzGdc*slD+x_Q z4n$`%m0dr;lr=tDY|0(V5+ZZcZx{ky)@Nx@_;phGW;{U)N(UfjfQVKb-H{J{A18yT-kEmhr?T0uD?u%%bF1Hf$(Jb$`LsKIMB~=f@J<9p zymelP${cpK&RXEEodekht{CO+WPx$GuSp>yu5K7_r+aR_U#e+UkNDlNxDCKpyrbBi zdyJsfrw6O$UBE2sT^#z7)oS>MSF3zhH}nez+CpJ7tVX516bT&mcXBN^XGXp$@j=ba zZh_s@=k0}`7prh^?>9{`y5`pTxpno(d$$lx=V#t{F}K3!tSydFG@B*3-u1GrZIQDaDa z=EV9n#2#H#adoY6ZvIw`H~;Y@5x9QCjot2MKm z93S!Hsj9l~xY&EM5+BC_lY@4o`Uj|I}BCCzCT^zb$2oehlZ))4e zuLG>UWI9nqfmr&D<&X0hZ@g-rp~@Gl5q<>XgI8kCYE9e@XNk7p9pL<^wLoX@>kdz8 zac(Jhxs0HuE&SmfJ7Tb^5NGr~`M&yZ1XcRvc)5(WE@fKb%A2n`fk%(0Fi&hg62Mb8 zuI<@6u?mry=VZTbXL@P5?iq>#46FyJmpb7|_G|JtD*i+2`pimm5)xDR5@MnSiOU-I ztD;nUmzLI;;gU?Dzc1?9B|t#Oh&BT5DTe?Oo|ef;_|Y9-r0o*vW}3XbTzK@{5l&`@ zQi&DqDZ}~1?~%s0`8o3J9m`vm4QgMuL**~KD}w3%7d}1|iXdHv`~B9%RD7Ct000g&O=b5@s z!H;YhM`mjjAdN_N!8ZuY|+<4d>{Eo(=CVpvo0nK0V*PH9O+U#JiMrc ztSlA^14w$VAp~py;oT4jowP?{EK@FF7EL}w4lellnkfVBdCb$8m;l}Vn(@Tm4}I&6 z;_7G-#@Ym1ChJMbiX_)URO{P>QWT0DlZTPs)NIfMZj76IC#24QTVX6wp$PU-GA4{B zJVGpqv4Ck{z79oHqLq@pKjBHOOk{WdafAZ`U;ERao5))EqB!T%>(TSIYvL~K63_L- zx>Ex7O={noJ=s^@?ypm-48l?k{Lsi0(t_)(tqE>agvy8X!b$EM)-8pe-?S|U5Rfcf z3GeuTDUMIxe#vBo4j{RLc7eg<7cwGee=jIH%8laKtE}g_1c=A!_C~o~Zsh^g_B1xr zl-HG|3hU}2hNo|}(qf4!9YP*I!PC(|Y=C39Exs+ofc%Oms_L7t>)V&*cBJ^Rn%^v= zrf44tXJzMi;GSDkW7}&$ODK5>@HC&ZTHiy*=1Se)3{jEiGF5_)c8#0Txq{OU$;z6 zeEY?UY=o5%*N|0Ie6&}(w`b_de!+6A_E&kRMhl!#Z>?;`FV`Z-`b04V)R;2`eCn|v zdmo1q-Z=gG<)qstQDcs}cv}|gcXj&lw@B=1}_eLv!Uql*PRnrJr=w4r8kaZ zOX@*x1<>Qa8_bfl9rIT;fs(_8)aZ8)N=RO*-pY1pacPF8l zH+DI*Zl-_^zV=1dU@O`$Lv!$kj?fEK*{9DCxFP|AJ|R8NwE(C_Y}d@naYrMv>FJhd z^U1J}YrN%kmnn1Crg7CqbLL8}YZmE^#mdY8+YUvpt&mwJgUHHoyx09@=Z(wpTLNkD zSJI%+&_k#B7*^a2@!tzaot0z>Mye=pwSSH^xzhG986sbr%dwk@2g}#4o9Pfb+@Bjd zQ9Lr89V%$qUE~&m2_bW%{k`L$#VXQeBN+D=dhvWaK@J>kOlj`f;1#^=^s2oLQ7O}n zR_6jPD9^><@0wQF{RP|#rU9g-Y^HMcLv}nXn8w}b)~%hS@7HR4kv-RV8)Kr^6rSO( zuqTWK`3c3Pfu}t;9JssVF|&*AshPsmdb5Q0$h)Wp*yI@GsJ0HnES?L^MA5^#ft_|= z#Z2LIz3KA(WGa`&CYcd4W8Af%JR;1N*|EIo$m(z>cN$!A#L^aS_!D(B?;2`g8tdYS zr+Q6Vv*dNb^m{FyV*$XM_!VEI+I{*SWJMUHQh2YY_rI zN=B*19@`W%8Ii-7e(|<-8#CQPc^j%D^XySr!ZOw>bsm^(XL7kYpHSl^iG*m5?VtX?xurt7<` zs5#DcZHLaL)6|i{Wn7N*#*-OiJCJ>X!M2dYALm%_&Ed9Xh6d5oUu7#E<2%z`;v1Hw z5AF?TR8p}YCcyWm@XNj7!zmh9?6P~`ZT-9Zdc`+L8GA)EXI&zmo(xl;H_Dp0(r%o% zf8u?dlqsy1gg{dZ@ELnvO* zCm2iAcLSi*4ko_r0hDq)o<9f0~ddza2c^+CXGPLJeFBHd(hX6CwOB(|C|2 zl0v|Hr2dB1K+{uY?#5hHf2~p6tS`2)5yw_29dB}1OnTmZgRc_6&v%zDKU(;X6!LZ3 z11)6u_19z~u1;}^WE~&qs_rnZJg^8oe|XO(%j?Ut-&PMrDL4Ij`bN$wEY!j!u&sFr ztHLtf36)V)oz|d^x4Oplzgg#ooCcbzf5WrR>9H;>Inm!b$$nBY;~H{7j6&D z;GeAsW1U;1tH$qU(j{lK7^tO6=v9G}W*1E;rw9hkCWKI*TjrUL_z$>x8LMX~QNhV)}=%@zA}nQG@d$sGF4l;dX8z`Tr)x5uPIrcAy~9VIK@8B zUO~$YIl&}MBtgBli(V~$Amop8%WmC&2YT3-m@oO0YQ1m9c0r~~;F#BkC0?%!Sg{{$Zh>s{wT zERFH@=EpSF4=|4H4|E0x&U|12=XXD>+^LV6n=l>PmjhE~h(_0WIodCEU>(52fkajI zRB(Gu-8nbvSgD+tIWkd6xjOVERU^5Nbt7+cjY{eSzI)&7{YsV8LN)W}2{c$pWt ziw99Ll6MmN5~p(tCzpIMw7s=bS09qG31vjqoOevlk)krRkE41M_1P&oTY7Qz=~Hv% zfEKv^X;aBXx%A<4-J>nzXSJ%~Eb60Z;Jh1rIV!)P|E-V?utZi>0E&Z7MUS815@Ly5 zLJSJj+;uS1uLcEkDrx#Tzua&WrQS^_+%Mr*w*XUFZLL(j$j_#y^sY+dsLI`VOQ(5n z2tqvJOvi?PubNQ({?@BPc1Aczx>hx@YW@TCp1=Db)LT7stjhEY%96cUtEoGc-)d_z z`{`*@E&&1oKHp6?Zguy!7D4H5$vM(9l#w5{q>3RQOV2GM?@eIjve^N|gE6oB{_i{q zh21y_oIeIO>u5(_?z-TFSL%6k93_NlvJ`{r(O+=g{kk}`T5J|{Dy65RT;BS8dYX2n z2xnYZ*ZlokR`X62l?AvHOyU7G4M_Y)?C*cRMlW+xbih9qRbH#Lp;Edou*iC0*d-Pvd=hcE0o$pnLNQA>eIQ7?p5_8*k<+ZY>P{Ut*w>FC!-VkJRP+ zw4*2aWidr)#n%%Edc~W7E5}U0^T`e23o`>00FtJf?orX)G?O3LOgGWMo(Z})`zI%; zksJ6sEMM_w+N@wMKZ5TJzo;41Kej>O0c0h^N*}OueMAi+VhtjE5f|?pHE`%XR{K%Wd7=Z335;o5aivZ+yRoI z1vGjfekB5fVz=b%fuzk^HG^e@t6%%6*hDFnmxtGwvDYque-!Ad%0V7QEXt#mI5pkO zgXe7B71;g%8C>%Nngt*Q!wMCsUMPMjy*9CZHGc@6<6O=||7F#fLI9fM9B*ikyubjtAYa0R9;x1)5{DDhpjv>QKif*p3kZ0YQj%eOt{Z?Fk+Flk$Men!9tIY@=A zVhz%HQJa}&`51Pdn4tBFLp)GJfwIaFA;?rvGR1tdxdWG$y?LDKdq&vN} ztdVX76YI>L?Ip~$wJFj8i&wxZXVqVIww~ZiT+G=@>r8~lIA@uvF4E2tU$b%c`-T2F z&4O~$DD%=MB?_S*I}W4zxaX#6LP2pa7IhjYHFa@3u3n>s-+q_j!NzF9>NgJu1wSCG z*t73=B_=YCG3#?e;4$S5h$)NLAio74oMVjjWQ(T#Agz2gx_`lXbHPuN4Re^~fNJUwEywvr<2!DGNtY!M34l&Jxh&o+F1!a;xyJj&Y2w)}ewvkGNYk^6Z8b5>? zjO9YpPzF?YJLbR49*P70yK)*|?oppL8PNWt5HOTm>vVRaGXbWLd=_ro!~J3YeK zu{BsZv5;bt}U zCyY8@^s0BzLY+e7yiL>e*Ru|Aax&sfnH7@`S?47qUrK%Kyi%#uZup)mAL-w|T9+@w z9K1e{Q8heIu68k5Ar;PK#{1xJvv^#v-)AH!9n3#W_bc>*(>yU7d?}`xNy9FqCCDAd zoXYh-4AUBN)RbnD+b>i*-1=h9mCQh1SIz{%8BywUUw|32F>K-d8liaw1bTK~3%J+5 zu0uw=%JDReaszD`tuwnFaw!!r;j)#T3}=62-MvI=uVuR`pcY|vCok0uNMb!~nLZqT1>#a;m1TpJdr$aTH^+kP~ zW4UfXP_iWv8l%K2mb;cvo+FB@||12}}gV&+?^_rOb@;>#kgunI8u8+Q)5wly6) zQ@R&Tw!UN}jHu5#8=Oa+-uP>5Jm~I4Ya_GOg4FKo{VRoiMJMgpE>i)x*d&-(ccBdN z=2fN#gs&O)O2d601zWMUH=^Twc1<%Bo?dN4U?8zQLiOgT;4o2RRoB^voLu zESPD&WwC?5*)7TwYiDfW} zU7(}PDLwp}(I{!fDOVt2y&5pu_1!sICy!rQ#u_|@>g*Rt$wt11J+TO3WL6@(feRVP z(hEC;yLex9)3ilUG@~BBB8{3gmnB~~H>$PB`5ssKJ4fmRSvlH5zzwv%kGM!~lQ}hG z+sY{{*mIvgzpR%=R7I_%R7P`Uv8EQcR*K31FwEtIO{sLh!l%P1hX$_SIzRFKwuim} zxQ#>sA~N;skl(=#j97`CmoN>mGuFbLesGSqA?ObHE&9sMQS+*woXCO}g7$Oz-jud_ zFb)4BJp#l0LO@=2zasRm=Dad(Ebm{frRlbxLu=U5#nEjGVG< zK($GOE~?SBKU-6du}m_?66#ifJ>^#FJ-%%6hD8WJ=-baiqn>;=FWa8kIBaYxDiUF} zEF>)j+O?_~g3-zqBDymeR8vjUb8zwrFHXxMBbHArF)lW7+%R>CF2jzIk(q50cdTi^ zxW<9BKxk&k2axcg97}c^m25bbbhXYOt!g9YYrF9hW{E9LSAMQf1SkWHa2k}VDk2WVYh$^t3*65p4<6noau zh{#?*jmVzt|8}r3Z2L9lcROicci7(&I!xd-o@;iS>b>fRXjf;YAwUKQ(h?9T9j}Em z9&!=eGFC{t5#9MEt5byJ$8)I(_1(JrEA+5rGE(S&58rntheSg` zWX{1Fj`N9U@5`fpj_)eUgA?*q?bro%Z|M7C$f80?h*;f+$dhE=%wGQC{F1?qdyCGZ z5-`*?VGqtG->1q2dv+wGU3%r(Q@OaF^*&TsG{Zzsgx{j>7|)&cihvn7QO&+%!iUW@ z=50|*30Z6GeBrRnw(Kt&6O-G?7R~F6Ik`N-!QpLL@6}c)!#x@z=3hKh$QT$9`!NCk z^AFA4WFa1KrcU-KSAyt)tn+7F{r+7i4|cI1P;uH`5h~^H-{Q#&CS;dK%T_a0{9#M5 zn^Lj=8WQ`Le36pt%hnPvFkBsUM)i(4$s(@R`W|Cf+O7^7z|DN)ak>dD<1j}$!#4Oh z{nh`*-4x$0^YX(J?xhqwfEnrPo9F}~Q?7Jv?tVLg(yC=ZP%Vtb-!heG2|=C^!T}X? zB2Q*Y|Jns8{i}b(=9lfQCXQF=PGH!|OjtKpM!K^n);c3}yZ6qbq)~ft5DLV-ngXZ< zGjU_cF#Mu>9t}+uWppYezzBov+BGJfxr#Nuz5EAH9w7`7ey)o^E(sNU;N9)6>K7vQ z#p1dBB;xVpvWf5JRZ4>5I%2fTSCu5+ER5l7E?p%$RC+6QPmiW3NCKZ%Lr`lLJe3gk zd3eO+vR0ffMbUJ375~(36KZp=rFx#D*w@BB;FA8V`VCPG^X@EMcakfKEvzx?VKg|?>p`+UBWc>s z6KEwwWE)+V4J{O3(S?|Es$k53_cRqQStO1F>+D9BL@qj*3{lRtm` zvFih!iYnUoRdPpg|AL_a&k05pKwom%f>FZJ=$($KnQa$>=52+(JLwx8<2P@^|ENF4 zO9uAIqPOts7(F_J+wp3Vk`p>ZfS6Zgn$RK^@VAa9?+FP=XfMdt73kq$y z_b0#C$eb6l9D7R2f6}?lzg5k*eJA0J;IeT!I@bGwK$oro2(&Dm8e|@=Lg$)+)*ztD z4ehB_GkGE^HnvK})#3lZ6$Gf1it_g}6)!slV#~%Vne>;^ge-9m!jP+X>;m8UAtxE` zbh+3Oo~iax*p9r;SUx56U*d$4T=#y6iZ985tJHG26}0 zn&MLc9E@Izkhw}3!RFl0_yO=j59c0M$lfJQ04RC`>MWDXPC<@y^UHSUS_JMyDOGS} ze<_u|Kn*V-!|*bq!%!9n;Vooky`ipv`{7ZI%8w1}o(sM2nh|mRH}!~d_o7t!T{dHi zf_-e*F{qn%W5ML^^RU%zLm4ACW0Z%wy0uYQ+LkOEhqsJyJm+^?;drgf4&6O5GcilH z*s>qXugWLh%2kObGIquJ?m~sf5(c}MY35jpvFy5O=FX8-2AfEW+KhWs71EeCysn_F zmKAHV87uD(`?8dIXGryVcPfw`K}FF>(+P`ZEtgj(e;?Uu0+lYRV!SFY;ee^$!nEu2 z%<0myqtS<*Jf9t;{6hf%wRB)Ga1(c(O!@O@VfEz`V*~k!!uTaOuizt9+iWeNu`%y+ zM&(_gyC}htcQMEC_IIZTDHPs%>zMcO7Mr{>_)S?*d8Zf?W31%jI6+%Gc~~)Xr@EHm|KhNU2#emW=?>uK zKp{h)E%6_)t<3u(c%?hgEgmbVO_Lr^(>RwhuHcl|{DLNh438VQhgcUX0F<6N zU3JPNXu>93KVF!nisF}bG6`Lr09|deuyV$0%u*NTgIdfe6THj8ucoR-pk}4ri`;?j zAXSEr^%09aQ2_3D_c8|Yy_MC-5-YIMBkFNObfF1vfB!mfB{Pb`yC&rTr&b-iGTmN+gNvaf4a^{Sf8o4V6w2IBnnNf|ZSKsA6b>%a9JblaP)|9&f_@nG^dF zh7%Prk&~Q_V~VQ)N}h2wv8aSL`_CyuAsRWRF5Vm=ovz=0o0PV@4NQ8jFmAmFco0Wg z=hKJ$&#cKz)xYqhqHq(|eLf~rb<>D3{sM?jld%jE6}2P@cK8dqt#;uQ3F?!1f^^D6 zWhNoL+o8P{5xH7o0%ax<+8)AZtx4j^MK&vS3hPre-^F#y)&p~Adq2(Mz$X=dtyj1$ zh>P_7GI&fGqxcCTBUJX%y8GXNX^8oLL0o+L#YL=$J}Rl~$F6Jc zidec*#L5&K5e7Bh8UdF{ddLva{>9*P1Q5a(nzY7gbVvkNY|db*Pd=4rnQ4iPQ0h8= z*#>bF#y?+(AX(rNzh7XwBpeGcHL=H2 z@vW(!De@Wp%!S;>dkR@1L=IUT@^h;6HIC7PjG^oB;=`9q=s8pc0_Q;C;M7Ycw z;FLaW%qp&0)oX0@s-9tK>44_ zbg9YipsWbiniFm3;OTz#BEHDrMyPO38HmritZKCf@a4;Etk`=OpwkQB%QkeUQX)x%ARK-fh9cge z^$*n!tKLQn9UgU+?{u0vjYgxyDq})8gJUHZ)C|k8XUV2!?k1-J0A?%cyzB1{!3P+3 z9dNzP5)qR=NLJm@RUJ$^-$06(fL%j-OK??4+WjLb4;q4Wm7i~{XhV3FPuzL5nFka% zW2wbQv9pOTa&-k)&52*5>S!T!S1JL&Bx{7W=;So`m=b98LAz?|*(lIDB~EBKo}IaB z3+Tff@q~J$v#A=AxRp$Du)tyKE}w#EAevA~ea1#Ct+(;rw5jbW#B#&<0`M3f*)K8Vx{I}>@*n&jW|T3ax3|8V$W&# zpN}|NjWlTu@pjx__6nrb-@he+AjDA?;ycmtRP#^`aAM=|QM(G~9#SYXGg`pW^g3W( z+Ht<8Q}&6+!N%!@Bf2cW5lKOP>F;Zv8Kt_e6_ZE$$uLW$!PCxLrk40U9m}qN7_1b zJENf=8@{8!n7FF?SS+R-nOp$0*OZghv{&{730ImRyFs9AH?L=gm8Ze}Pfr_8_j@Ku zDUAq0zgCVJh-LL?j^1||$O@VIY60)fe@P)2B9Q?3L%<&2r3>0yj}XHW`nc5_AyB>@G2Qn)|+sdTMM!XN~@%4PTiHBhC{k?e40&$>xN zV!@qbj6jsyzGYs}{@&glu?RnWbad8w)=<46`f^9e{c+4V1pT>I?3<#5lJH}4q) zr07lw^za%<8Wl(oWkx$L&ks?LXdBx&cUQgD1;VFUwmSU~kNyQ!h6AJi091KI#Y8eo z&;a#)U9fk%ac&mtnT^x&riaVZOd1s9`{@8hF7k(paR4$RdbZ|@0aS(vQ~XoGbsdX# z6V@rw#;RroY2)JmBI_%F>gsm3k>c*|rMMNB;?fp(JwTxaio3hJYbowf4yCxe6nA%b zcl|f7+H406`mNmlaYNx~(5=}Kxz^xL4AE~!d+UYUcu>dJe_dkzeGViQci zv&yW5Z=#x~TFCShDG*DR3u3qcAj#d4_XQn&gY*8)m!XLh^48g>irG&9tk=LyO~@Ux z)0%6OWn;Rnn$$gDSoA=e?Y8(Fr?85Kbqpj*GrM2*&sO<`L`q63sWeCnfts`^eB({t z`5|B^_RN$xoKyE;62Ag4)CyabAXI;$?n$+yIeh_kt;rWH(pq_KG|QX>yFc~J8esxy zm+!Gdo@fP!zPtOTtG8be9>`oAB>F{%V#e4x;&F8iLX$%(xUb2@Ilt!l5E+2^>$U%T z|8%09dV{WyG7+dGO`@fkv8d#~^TiS>l;FCiQP>Y4XlFjR}$Z?koZ>|{ThW>WlaeCvCe>Cs7e+|C8Uw!>hqznLcI!1Eo9P=J2*Q7s$fi;`q_qoh5^=ffX zy>!tUQL$jJ4tzbn3f=9^#-u!9>5_72AXlHX(40gOmHGVs4x?R;#Jzk0!BuuI>h9pS zu#@!D`Mib^zqi5*1IOaw}8e0c>0o9PGZ{y*-) zWwdZwEZFHC4uHCpgpej{RUJjZX-Q>%lB{?A$?GXdJQijB5#W2lttQ5;Rv-EGUqI^w z@UQgF!fu)qZ!D{|ySro9q7Aq2ZVa<|-h@#!@5umQ+a8}pyV67n3a_zeTa{NWs%+U4 zR=82!!jJg8ysluNT>g&r9T3PbSXV*nS?EH0INR-XJikg~CY}blQ0F!Z$RwYDqrup2 zC6aTAp}%2vq@P6V6YEzUHr6M5jmx)tSq#P?UE31b9?^C2Ik1P6Rs!UP*R(=IC-=!W z%8p!h<~LwxkfIC*X12?nf%Yh_H~fJ?11~p9?dw-ic?^%wt4lpO;!1fX1s~fJ{4G>* z0dF;{$KnsPJX!ehwZi_GgVCAh!2CV?TniVWQ__d&R`>+?Q^~cvkQ(|NM?;H)9_x=% zE3v4pUlw7K9N+{JpIg&fNQZvl&kVc6?z51!+JDBhMMD;wI+`vw)FhK39POb|^^0P; zC@A}hy<`9beJ{20IIGNNd!~U*$|+Ykv!ccyhc((@#hvWM@7*rCBp!pO@ynE{ z8mxy>Cf3H#MER5Z3mM_zw|-Tc;JSuH{k+M|l$#M}@_H6CG5Wqiu!C-^R@e`9lm zTt2P4Fj`fSq9s!v;`J>KuLfeVLbFvqG^zzKOHUV%tXKcx+{$EnV!pt5H1_CYlPua* zWw1-r^;sX)(bT#I{H6I`C2>Mp?mN~FrmoU!z|`2V5f>P@c^VMrQW@%hBn@v|gtj>g zBfYWpbPLCy$ntdoRi(yr6vC}bp>}oOsA@FY3NEGblDc7CuQnC2i+?uFOn(#%$u(2Y z++d76m<=?(dnUTo)gLWZogr+2(ya+szrPZX;nfrGOXrJ=avZ+$-Jrvm-$Sm*JDIaP z{Wxq~ejoe>H2HoSZ+t_9*&YC4Zg;K;;ks!XM7z8z<5%e1=5v4yL?z zi(6Cxx2Mn{>6Wn09j3zqPsLO8$~(i%^53maM}U5QQ6p*`+m4B58Bl7ulGnRQVzb;x zO%|{SjLw!;V-mZ#KrpEHVWV2^ZeSNgw*#B5Ic)nt2k*_Jcw+=zd-@oslQB}tC6(&< zQapR2x;CMr2|md8qL!go6VOzQ6oF`H9UGkU@S5nuvelc0+#gD4WLd0r8iT_qK@qz_ zY%39~)~-I>_d;Q5M1oy#qdDcT9bpo!7QoGz1F2y)VsQGS`BviorQ1%Jn?`@{*7_Dx zA{*j{zu%|zH<@NFkO~Db5RNp|Kpx|jw>yC1LR;LosJ;Q!SZG1DB50QrF&(); zbw3z<&KMA`akQYACrM^CR zBd89$pUVTXs1}>$${CYN&kI$$#=YkSktq}StM!BMAtoRT%aK`iIzXH?R5h||KeB(B z*_g$Ma{8U9uWIxBf7mXW{|GhEEkZeJ?L_uhkR}|I9`xP zHvoBjy5syl!v)bZB}L&P@ye0;z&`*hqutL*SJ}tD3X-y0CGCqU|K*o{Lsn|nZ`E|! zO1loK)Vkc1uHuNaKnrl{~)zfUq zdii-T9fesH{DL>+IsM-Tg2NgL?+s}8iERt8f;jbwQmuhDsEgl2MhSGjN6%B10XKw7+?;=>>qC4jv{%#x#_{7h#x zu|avg7|CPtGz|nOWQmtsfcvr zJD)F6Dz`{7GqP;zqIU(ZTV0(Ii+-!lKMmsu;eF0OsJEC`iNu&I^lg;r{pfpJYSymW zKl!NI*89SAoTyF`HC5ZhSN*E(z_$SXKb}~!Fz6);X4b`F%f}pW^%4+tK1K=mqJa{F z6%0&rTkr{{0r#w5bBL6rw%g2K!dBkIK{A24SE~1s4es;zQEFG`@;U9*Oeg5Xztc<9 znwHItttmd17d=5L5J~iEl~xH2C1P8Yql2H`mQqjVJ8SVp&Iy$&9x%KR?fzn3`y#yh zINj<3I<_K#3_C!L209Mma3)~=@HtcU$(|8clK4UFUK=(S85Pw~cz0Q4E3p;>#OltL zK3}r7d?&BeE2H}8V1Txo^d&z?CdcPYEuid5(AFY)i?lqf?!=nc|*(u_m8JjMfW3t^^76|$w3MTNZ^x|M3@ z{F`jJa`t*93Eb4xU-`aq=~|p@VY!}&@fXss*SDP|$P$TJ@85oApbj;uy1n|KVEz_c zBEmKSn?S1=vFhgLQuV8@)to6NyLIcg9}|7P`zDym^JSZ`ej?g9F8V@k9{$Bd8hZP* zwTrjaRgP^QFf}=@S$&mPf66F_3=ruNZ8>aL{5y5iS`&ko1?0BgOp|T>9Oq%ZYzX?4 z01~b23q{t9A(vL`RV!gxQ%yDXrUX$tw8)&P0{7l)-I8DCGb5~&AhD31bAL1lCKhfv`?#Wc7?cqYDfVEy>v+wme*B|oJ@j?n*1=$H2{tNvi zk@zT#T`u*wt+(}1MW=uufWfW5Dk{X^8X31GCzR$Nm_GwQ{7a%ww0bn-*RP`Qb4p~h zy>E@E<_oLYjfh0(v|oEY^Geby@9PyW?C@_LEjE-J50ff9xhaW?wjZ0~xy$U){A6_I zpWuOxlHJ!D+k-tq6Xc$g>KVs8oSrjt(NCGu-B&g%urRArljp}{n!d}{VHm%=c~|F1 zJmj_P(Z7&Pv<&TzcJi+E8sg<)qRI^jU%p(={HFQ0nmBV6qwsyb;7C2uukZjASa<|o zOixwK;<;w}o~{4ma;?|`eBw*nx>$G8e}$Tm)dzrR<@a?n->ZMq1x}r)Pl;qKf%28} zAL#5$^k@p$8?^|sntxvyj$rw!i9e zduzh?P62Q3>^dt!_gf=hE`wCP?l-e8sXzXDZ6k2iH8qx5`SJ1tU&?}tf4?DkiB=X6 z=W6j5_O2@zze2q$MDmIQPNvz0^v}5+Eb+kUn1IBZ1?03RdjIEBqtB^7AQ(^PhO;*< zY5>9qVFb7i!B`edo~7lBsV#U^f9v`ZJvVCFeb&&ihTw5~L&EfQC)CT016zgQ06VFy z2JD*(RI;sQ{U4S8ON~%o$YQ1~PuTOKX!e#6%zAS3Pw>ghUhns`C&O?)bkBwS$Qu3b zwh`K`aeHg#0dpmFFmV`q|HC=TQD9G`RAM?198z66Nt7WDh zT5kNu`N5OUwB6Oh9!W#mPm&W+_Cs|N8bLLR1?v*!BYVNv>`#Z)P6d=}kFf9YLa50Q z3Zn0?uS?u+vk{FlnP%Pq!*S#3+g3g1*)pYwGA9B?sU-C$DzCF3@C9v&*QP8O@4ugT zfnOmPd{bu;uix6*GFYg!(P(mY=*kq7( zAr?Z?<{D_*LOJjBOMP=UG%daJc%kq<%33b#?)`RX7gD^2h4aQ$LyTd}CzkOzt$=Ss zyBM)+F2vsII4Snx^m$BPzXQVkNIw+?3{KNnD95roP7oz?tV3s&S1A>LAf{|^60Mbk zOIu1Z0TuX(lm&@Mh4gh~TG?|_-d)2fc-Q6~ixqGS8`p|jNI4*@3zJPvOh-XRd|N~| zq)1B*|G_SleOU3RUp<}@V_g;ez9@kKSARF(irj^h$(>^Rx5M8iF|!^a+P>*I8X7{(~L3w%$sfo8tr;4Ij+Se!{@Fa6M&?H7r{UT1Qa)u;9eHXh4?R;oj{ zoRL+W3_=z4QoT#|f?*L01f7DnvEYOc9Kd zKCcs(wRhS8bcMF}LxB0mfI(YyCB?V4Lr2~DwX=e!AGP9hDG^lqzNSu4&~^5$QU3;i z!@wEEUA#j^NAp@+Eybo~n?2D8YlyK>^?zuHp5695CP<%;>^>A9+Fh?*{bnR;yJ!Ci z-pm~b6YcT!5dDqM$%0^IEM}{jR7)_|8pAh$SxccTVL2i~Hidupvm|SrJ8)u{b1!rx*57c97cx~OqfH+Nc5EOab3gGx2U~J`P&JQk{--Yh)kXUlCkGz$ z|6xDiy|kEx*D{+1!{h;VS|K{~Q?~SIP0r{omK4>UgWUj-RPOS451@yFT;R0O8`U_B zma})AOd0-{Z0qGlr|*C0y~Q-ci)btEiW8s6<)aqaNv9L+PPRd|NOye}GPB{$N<6<>=YJH&?0w zi3pF*kYB-|t>1&vn$aagUb3WgS1+|c%u;g%1b$hUa5N$s882JFDMJjf@_Ala`&O!V^B4U^nUFM-o$jWLfhIGH=PA1ABYb z73mpnwn@DFl&mos)R;TZ2IoIr-d*yBsb3PFw7gtGKGGk^4xQ2f4da%d{|xOYRliaDHWb6R~p`=8D2buBZ~eS&qa; z?TQJElMAc8vG{^n(Rt9i&U7T)Ox?n0sfOc~ODf%&lUdAI)%?|pma|ZDUI|nsSR^GM ze5EDC6<59&6mN={58!2stsB*lmyP&LJkFn#;qhgHg5Vi7 z(1WJ+h+k)?fYFulcYtzNpHf(x7^In^hg zY8E2z9gMWjg}m)6=_qAP*@e(vPv~gp9Ui5tH*2GCWd0K3mP!hRSn@TsFiN8~$f+kl zQvEq4Izm@U@aRsOz0-1uqS?KLpN`IweJ#G$iGfx48~x;>0KS9kg=@{mahqzST=(X`9*Eb%FA|Vu-Tyvjfb$t z*%Wfk9lO~YZ@NwNGIX(%W=5Xtd}#YvL>JDYSe>?LEB9weg1m-0^sMU95+6h}IpFq_ zy!^kUw)g}T;PG5fZDPLofUVT8(LdIkjw1oa|3Wj`?_vD6bFqbS@{WWSZFh2vUpcd9 z#f`gWtveQ_+ZE1u|43Nn+S+;-+3f`0*CxiPelN$-5LfS5QR2A@63|`qYf3GwYmcC9 zw@Mtlrbv7?L^kbaoD6=|1aQJ9PQr+Uq1}(qWm!okr8;%L+AVp;szf;e?&TkKw>qo< zJVl|P_}c8JJ@ssHUSIBhUvkt+B5J$gEJD;5IDst->#ZOD{AL*cgILPCz_~QQuH8&| z-n86e;NS0cH1q2D!HFy8xbRe7KacFiaIx|>(pL4GINX=ccx?Cw_tM1(n2fLC(&ig# z)El9u?>^Vx%&o^X>8;L69n;6fi1jpveN|otn8wL(ub8msf2`@>JCTW1Fq8%4oxHlD zw(42db`tdoBc&-#7I>fr1S&k^Et1W=EED-=GL-LUq?q_h=o+CZk`!jES@rWt_LJ8A z-Y#$Ob7!ALIqik3^}cNBiTgF=S{nz|I`)&bA;W>-XXH?1Cdr~w^y>xtv^!fyy@_JE zf}n5h6DH~%39LFU(5R$>$ni!7>%R!_-x;PWRJ~RlPUj8XUQ)>0pH-H$XNUt|{7j3C z=+8I2O|sAdOZV%4>X|i-%QVfU{vE zGXQ=?AE+yJUqAaRtXql2thL_VYrCU9UN^l}8HsI)AC_S9z-W7(n1VWvZluvG`Vl{(VH^8J(HwezmsSetDoFHA4< z#%CWm#f{TYAy`D}C;a%PXB2aEk@xS3EkSO@8=x$|< z5=7>@LxHjRF$NrOa_=4k?#Go6l9mgc{{?M1u2y?K-5OT~4`#Q^B#sKSH)kB6v!W7O z#ldem3d>k?GZ_D2d}=fS!V$0M3s4Csy%KmM(ZceYLBckY+HZSYf_V=$3mwa^J<><#qk5(M-zyI)vF_E7AL7iFel^-*4#HCj& z8MqO0cFSC&5z+FgM~o-hmN<_K25rfK!IrTuTAKLgUbZH)?g=rMn&eWk!rvKzN*EEyaGmKo4o3dbKrc1t&hGT~Je9pgg7?jfULj99m zC0o~N6$+r?3q^|KD0M!hMW$d!bkH?S?m`*l8?J>%b*7Bo<{3YP7IMD{4O$vlwjHek zxv@2Qt5?e5v5bvzn^NY(X70kY2P`yU@^z49j_-G>z#!!GMdpcf5%Gi!nyJztea=86?S_`~RsRQCoI}v5wa=5s{AS?Ytcl@z z>Xlw(mtMUu;b(5W7EC1qKm9#-PjDXc)gk@Dug_XwV-HY#-FPHJp%{?(Ze}* zizOP;`J05hSjMz|S<^4rQVJxpykOKdHVC2Yf&uHD92#TYnY6bg8%Kn}2W9sZhT3Sv zAS|EB@pO?ERPNhmB{BhW?vDxVwZ~TKBE}}KJZ^AA(Frg3@SkR&QY7+d%Uw@9551() zsfoDCpgr!c#MhSI?O;rz#SZ#xQ#BZBFThfuP#55JFMGD>eIsI}`lh=A^Z8jTSo_%) z?d|J|=y)~;HT}DdU4RCS<9Zz`S`_>#uTBAw_ry%&8d8%59^#;}tBzGQ>-E9XpIlE3 zD?aNA?Up)Ds)nEYjr|8KW&v7;fVE_2tZS2_xI~k8{6@ymSM_2Zz#@QY9SF|h1gE-q4qZ~whdctht@>ewW>|mURoiOK?ulCYvvT#)2?Mp-wS;9^Nn!quczWJ*nx~-uSbkNmKDyRq-LqQBL)Flk*6Z)+VVO`w$v7!G0yE4q17uQaQ9NBUME+R zW}SReK^C4~5n50Gqh1da_an416K3dcSXAC^vcQ4r!nuaq2A-RTUXqi+z)2hY_--cm z&Pa}YJbGgBXM4)ynw(m&i`@>OpqSTv4Brfn+w?q?#v!BT!}+xZ!^M6Vbbd{U=NT=y zKIcisq#v6MfqYk=HIeAUS#Fn!hJ=UF9LUP)dr1~$@BjO-EBV8?&58b7a+M-!_b+j8 zpifwe$z|n-J4^hQ=otj#qrH2`+|ktv=C3*Bv3{AzO^Z;u+v1%h){k7*sVi#e1IuUly#PaUg5*sDoDyYKj~kH*Q;-+{&X`bh9LKAf7l|(A+Dm{I#CI zALJdO)Ijv;b`{t!!osH5Q+ni>)oIao$i3*x3c})OzLUh4#ZHwDD3+yctSfCqiO%++Z~i*Tx!=!(>^yo6h7u)s|!in?1p?NdT<;O1}@{)pIeap2%tZ z_U$$Njvbk1ukS2!$8YZn@X9&-n1AqUy*uMDPq)*&LG32^+l84xe*wbNrr zJAs%jOQ)qOt>s_Bf`(fpTB{akf^yJ6nc|wYIe{0V9-EI{`ti0a*Wceitk{dq!)`IG z0e2wG<@8^e`X9Xd@|hG!>6M~+Q72!niVq}T5WYd(k92|+428sCLO2C;Zj>Jf4)0pA7C)&8 z02Oeejk-5pP}Cpl$wf*Y0qX2$73!~#I(s@;RY`ZX?u|$-cqup)YybVAe|_S;<##|< z1T4Tbb;t9*J{I3GB*`HllDza%UpV7Y@O`yBISz!S7VUeW0tq#CT zhP!F}%S*eAYpq1vUTvOQv^idX4SHNl0MhKVq`hxtSu)>zsPt%e(oI4STDU;)dFK7R+lPkaa}8Z~{`gO#`v)3Ws_O1K<1{8JCOs?loR zZ8)m`BPHI)KJ`v$KxK!s{<`YlLimzn@x%n&UfNGhGEoQr18t zY-7^)w`ovzE%|TM-q7{$y*eeilKWcJFuw*-OS0cE*>dx=tNog@{vDeu9J?blhWH~? z_O(`li&Ir68ll6nh+i3k9@ZZp!=(2b@M!4;My!?p`w=YBPo$@p)?l<=>(DJu^FeLL zfjRO}Wq)GPkCx!>R)uS2AC8pi{U9Q)RCH&gbb1_ZvNQ7Xt+RXSt$CZdvlvS0ER_C; zur-ipvLw%Hc2tn*`FVD*| zIB(&zsL${AG1aZ0eCfxL&YpF*$bS|3FVSVbdNE~(QAyZH>!l3Q|D;QOncPv9_$-W(miFwoW^5T``&0|5 z25GIXe|Gja+1H^T>c?&sbN9QRdXo0fJ23yqExxEsYhL|}_1cfa43XpK>KW{ry5XH6 ze&Mzz`^f_BJEHqKce_Z&P~tb%c8*v^F80h~EY5B=Oz9dekz5_kf;fq4f=!6B9Gi46 z4o^7<_w7e^2}OWh+_S44tG#O)ty%802w8rR-~s(k>I2L@+7m9h8H?p-oqtGVK$4&0 z#0_BEVZ=TO5H^`GaOHnPPugn@#Ksd`N{&2bqhT|e)eP4%?cB+`#c#@mFigY_<*>1V zb#1Ozsl0q1W#>RilXDtEZPN9U(*$}1I~xr?ibrTLz0Dd?@o%SOKnz>Zz-XvDV4br^ zyr7vSuE1h{%4ph$h%4rsXK%>#C5%wS;+)|u?+k^X>eN8r#SJFpH0{eUn9#^Bhk7H~ zm548rm{(zs-XOXAUdphbgm}um*?$w?8is5{b{pboQl%#1)N&!nmwECjB3wq zw+8IW^vy(4AUV&)Zs9kB)|>d(AMz zX6F-!3zOsAzk=pD?;)|?X}hP0{>9>JfhI|QN&em8={8u+Qhe$gb?xP}6Txr=9%H+M zM*}2n`%dLHo9mf4wv>tGC#1JC};*n_(Bh*$plD8=zw zoBC2uN11zY(9=6KvmhDd9fM8GUA``MHb-@73Lsr?#ocGo zd%rou#VtmAo+~MQ?%T9_KV)wmbKM4H?#DZpLI_slCG(2@Di3n+>Id|sZ@&8T$CNWi zJaF}{{fBp8sK1xnXBaK0dQ5NPSiy0K!dLpYgE24+Q-rLE9!LHjZbRG-kL}jj&jf6E z+uyw`a-MCzLW8-Y_h6J&?X@!mSbFyo`iJu8eZelhC^%I71%w@A1*y+lkM5URjJl`1 zxJ>t$WJAx$Vp7eK&-deoy(p99+-2}tQ0e>M@LFJ+0~MlM4%0G9K;QKc4)39n5 zBb?ma)83VIbaW1YmUs%;(fM$K*H6}Whkv#_?D(} zS+s8rrtX^<8NIQmfd5VUrO4C`wr0>y2E}-#1w%&3E;nU74i~dCJjNCMs`vJi&sV)t?6xz^S;{DfZ zO;`6x^ac}KxZZ8R`lWL_S^7N@Sv4BoB}^(}V@t#&3ZiT~)&KB`tTBYn*Zny$pTu-~ zB}-QSyT+rhTbhWNr8Nge)TKA<*j`pTBl6masr8~}sp}+$;62$pl#cF3pXu(#faS+s zw$Ix^F(mX7j=@4daPHemXPBq)Pw0hJ;=O6-i)>cnGwDTCNYGl{VJcnc36o5spV`b8 z0o6KkhP+-a;3f0?9-+E;!B$pJYG{&@yLGR0BKa;z^B5w6Dw>_LG*wOfzXAh(0)l3q>>^S&)7FCbIn177(!I9ahAiglUGpp~t zeX9_xP|(E0O$ykkz|7RVV82<)aOl@%CYn>-KX#@T@!&U(3Ed8#>4dZU&b8^z8yfY0 z)MWGyUEx53^ujZC`^q-#;7r-hY&-DNxhB;(8+7|28yieGx5{+98>318Wa$Iy;WzK>2pV@S9)`zTE7<;YaEFlOhzNje_sn(J(yDd za_;M&sE_>ChVC+V)@rM^{%cl*`(I9;$o(SNp>}){sUFjOa?lX=p^Q~B|6N31c2^;@ zGa09Z6_pMvit22RV%4px$`23RO+BHIHTfg4wlyJgr>U}&$;zkXCnYoiYqtXbOa=wt zxjljJt`iurGP0AJR**-|U;Xq`p<@JXj2=`pM1Q9Af*Rm!mOP^%KW{VZ1QUT9FogxX z^IL*P;|elq5p^byuag@L(x5?)G~+fP7@fb?RVHF3wmr!^R~f(2p{oOXSt2%s0BQ{E zc=2+&2I7t$nYLRHv#EYup&@jhIhOZp)%ohU@Ci%lq{?zLqLIn(6@<{B@gR9IH6!LT zThZQrruIITsr57oy|qPZy_Dy%O@wpzSp2uHZ~1hsVbsd{C3P)7JrGX^8doFgdF6st~e&#PocJ_qT;p*%E*GvQU+N~G3A4`U5@O+D3frj5*>s0h7 z@Pk|`$mVyy;oFmNeI}<5!^^5a0YTRSZ9_yLGotzLQ9n0L zqGcFgwB;H%tWA5!L$*y?upF5(iF169NOi+LO>6s?3veZ3K_UOm%z5!KDTvLr-QErs z)G1sz+pF6`-12bgvE>Jh6|6_}Hem)1*uOlkz^_5;Mp;*&VCc}^LPLJq4#)bhHFHis zqBO$!&ZClm15$daVvD7aL6ir1Aeol|wstH2lc6KqSdUC6aK=BzKTME&&%~DSUWEux zfB$ZZ4m7vtVy{kh8WMPLTIxwYISO(%exi)qlCo+HaVltuRP5+67(JWh+4e12Z#Ba> zJ4f1cESYOiNy$Meb`?>%ETZ8*o50OO^Z6CeKT`FDpJ8DLTV3c6pTpWre;N!J;yZUt z%$?#z@H{C?G;o%888iy2jS#Et+=>b8sFRvvicL=v@gBSoF#X43CZieAm#Y@N?hQg* zGRFb(QcgX+G+u#3%tD01De_#eKHqy(1bsdS6K$hd?feg@`1Se+Q6~} zz8N@B%6dwyVz53Z@nXv~yF2+b!-C%G(*|+H*@ILm^>c6)zWSC~QttccPL%h}>rIV+ zZ#^(ngDQ*`Vm&HBY~YuEp*AVItzp)Y*eTl z+!zj-`{2ju6~WxPyN3TGxJ=rBhhcFtN;UNXwjMKh^A!3?9lxycf*Y{AlB!3q{VT>5 z>sBS=y*S}v`j4dAvV}hNOSQD;JKY&BUg&D1jpBsqHeia8J<>7rZ=<&!2wl3OrW!(; zw%65GZ!;CrZb{MI7fV#;<3p3yBH5{X5-!hv3;NF4Z!^-?Lpf3njm!AF@SPj!GX>S* z*IZc_qJYtvX*dgc@FD%mKkH|KTIOcvz8OkjdZHlisx`Jm*Cs7vspQFlw$=DkBbfxd|DY3czr zTa0YR-NR{Ir0cNHSiw_ktB~?53FYgf_uYCF{3?+P=uQo1?lXZk-s(zy$?R4_tg3wR zMND32RAj($&5AYOM{n$H|JS7c<+2^jajpofXUMY>5D+*R*IUk2{;qULXAqhGY+T>` z@Svv{!;OTa{U+1CLjl|K5*bmDzAc#O zk+c5$aJ`_=1#68oo^oQ!-&*WgTS0XV29zUgK>#b@90dug)83pTP5M@$M-!f5I^>cU zG?PSa7sK|z8=icJ8@z})bm@97>Exjf`OorsmjzXdQa|*am#9l(l9sMibSqH+nT#Jg zObBVUDS+s-tw(RYD~n)frcOKVqnNkiN>L$llgF8bH>aZZmFEPS{a`5b7ieIuI}*HHb*TXPguOXzyU*ErPg`%(>m zK`Q$cZ#+k3M#H$}i#Xut4MO81(2}tRk}oJ%Sz={|!DIgZ@5MhF7YO zmPgi;hlWOuai~w*(7n3Xq*ILY9b?y|J-!4^hD-56D}FIGYY|zGY`X!v_@@)9@7>N2 zkv%rvHr})jpKfF{tmo!ji5wiLmKl56(@R-62+9fW1Wpz1m z-75&ix^tmpOvAm7=)zt1FtDv2FwxN5;jIQT{Op(;F=t&=$*mbiR3?^6`xbg>E*p4p z`+!-NL?_dhH7-8F1^YaMp}5adA+U=l7pmA)#r;0_BuvckTzrFn96 zvU*reJhxH42BMmP<&J=YnhpY~nl6N8={AI*spTFo78og#@4i&1A70hJr8xRf%0(84 zez~05N7k_>Z+;4%E`(@Jy_(mZ*kr*zPYT_)sGhVV{;yOnHbIp_);k`#v++4S@ec7H z6WZ@hAZ=$XlYsd20+{i1LWe-ik_AKRYY#)}nbJvN)~f+{InvwG)eVevqzVc8hQCVK z>M}|pL2xbbM~FDVA6hwRSi&nY#s^w%wGsPI9dtRjO6Pp>x+B_tmhYCQfmOd0&Nd=y zw`fI1;kx^)_k77*4^a2>wm`N=p@?5T85-`FGW|$@cIJE_-22e;bM9dO&CbVYHFAde z4FsJ{FWUadLj_2$S_Y$GN7_zLNL#T0=c9lm1sx^A<;xFq@eSo@$g-y~G6ihUa!U(n zjLWDEpUq0>-qtj0Rm*)-S|=$QAGlP(D-MjG=DE*wZ2d555nMfg)8`fVKr_680)=M3z|=^I)? zU?}`+Dcl_%kpXQfkyA;{#r67Fy8#njj=-ilSsAI2aH0qKHD~& znwD)8R5(lB#|~m(W%PJ^x8w{5ZaIIpp0uc0FP=Yc~bvH73Pvd8T+f z);Z+{6!W5gy4GOr_03Cqox5Ob$Y>gT+f?*q`RLU!J$5AY4D9oUpzHE) z*Z3a?xfPLTq{?&bhAGiF!;92~;s|pvY8i%YBwnZxhhEaL5M3KdI$BjB{Gi<+oR}XH z&?O(Fp$0@e$GuSIZ3l@Y=Ecz>X>R*Hdz>>nh8U|M`^760ND^-J`AIG8ALThP2KptD1mqDUPWpbOY`VQS6_0&h zB%8P3)3h=HU2?~K1}7eL4JNA%XUM&iEP+|Lup3JRJDs+xDx&K&MsOT;Mv|4ZN43>p zSL9^yEC5jvn&sYJ)@*0<9!{mSY;7;|ooXARen-33oml|ftLGZG!-E$Mp1vEV1^*$I zL28+Fnfo=H#CGYx2qWx68?jZuiMEvFulOKJ(XWwHWKB`Y-hQX?13^} zk<>`$)aG#KCgcp$>jcT@+Ts@*pXS=_bM}HMi zlh$`Hq;>ARZn>VO=flFkH)3&!@UoG27ae)h^Qx`?S((jHvXG$ts_Uafz)&e-I_H0{ z!SgrLtH~%`|F0SP7p1F3$t|a_;q-$VXBdBc$X}x6z|kFLBGozN4bYiUR&hfTf@I~JTOar^2-48f;st< z`XO;z?^xPOfA3;zft}65_3XO$^SYVK(?d;blVIHB%x}l#%hjppBYtlFb%zR?0YF%_ zA3Lb7BmE&&pbJBlugtX3L#oRkX_r8BOxE>SGMZr;TjgqIf#w<^flZ0stL{~v-60hQ zj4jR~l2uK2AyoV!e}gtlN-sM%9!L%uOsz@gXJ=W1iB1WL zNagl>gmrr3qB(=#!VUDyysegb>^3>$;}UF^9bf@W_+qUKlJoTb&!)T>7xSi@OB+Ln z&`YB0<*{3bD<(IOgQH8cy*sCUtDYZziIW+}Fy+@-TZ&Ko6Thyb6E~NC%{1Jb^>5Y~ z>asT|gBI_W@3(noUFR3KmwmvccVFXHObGI2oC%cQrgz1M8?*AR^U4f4BM7s3e=`!# z8ODT2fbF9WPlXRP3p^(=!4xr>+~#?5gEJgD8&;STov9O4j7;}-q((2|>e5%$E@OVB zgA`MB<;8PwFVyZ!y}vJZ9Ib{Ow%rs|?E|3%1^McL9{^;MeOH=;44=j(Pj{*`-X4=2 zV8N5WUyQFsaacd(I*bxi1Cs?eh{3LWum200Il#r4AstEJ4}HsRCBlIyPKhC&GrG98 z1-%J?cG~gMuL&iTty*lR=b>U~(Id#!Xr_wW7Fi=R$__UzmWZKAlcCG*bO^hPhs7Q= zBIk*Q<%Miyy3Ptbn+ex#!pBw{=k2WebI0*HCFUcAB5vn3Z9SsrG*2DXh}}?A`C9nx z9yx1Jm&!S&hmZ8PL2iSkUIRn-q_Y=-%V%0Sq=SPQ&sbF`2FCMRS;SOWzZti zR=P@D;*aOJmqb+qnCotl`&PGpYRq&Q({7qLz1~OQk^;&lXOcA(P@| zXyE;KhmQ+Zs1|?)+X8zI(xdU~ojN*L4n~xf^Nb{vkq>h^9%!79a$A1S+6#+4_6mq$ zd%~&caG1LhNlrY)Hj=sF&yOVX4xu9J$W@frcc1G>W__RtC$VMeMyZ1{3|a_c6C-02 z)-#7In6{%v2SrI8??&A}KS3d8|Gw()j^x}K&4GLKS>}q*RMN{0V!@WY5t7^fIQ&!G zvan6sS7cKQlGX&VLWx{PJR!k7h42i4jsq9!Iak9LJ$-4~vIOtxyLIxtz|rtM&O!0a z{Y}A4vxwQng~Pz*PeZ?40n~W!3(r|*yoFz!GH(m`*;U?f&VxIS)OjX?C4p;)T{l$6nK8084&u)$Yr&vpb36j>eK|Uv5Rg*-758I zNJ2)FE+1^I#Dg^g0`{|2T%%iq%QC$CHC`mTN(Md(FMgfZsfGzBT_2<|UqH<)rs2J3 z_#u)xH{U^pq~N)eals{A)2Ft6;(pIsnfxo_6OSjM+q~&mMhU!Iy=R5Uvgy$qm+>WL zvAgXWN${d+uS9X0yuirVO}%>+W33#(%Lml{l^Jtc4G2D<-)D$qTO8|NM+~LEXzb@q zu8RI0+$WhcTH*$*iD8jlR{n4-!Xc8zteP&t==i|Ka;$MvVO0kx*7GW ztFTD#ITP8Nwv^;grg(ZkQ8F?UE2(GBx0lK_GEa`BbYn)5@yhR-eA2N8;M?5&u-Qw6 zE47zA8`TDa3RY@^5-zvsFmxVVhW&EkVy7L9O8nYlNTwLO|Ae7825{q|B{wG<~ z6#QMgy=)OPql(b!wxv#Ip+!$-c;&cD8DeEpCJ7t!8C!>J3zavj+CGkVehzkl`bT%3 zIp5klh|1JLDHD^HMsHc5IY6*dDKu2-*51~5xK~>DQOq|XDA-12<~XXgyZdOfW@U1J zf0o1A$kR1u*G4v?JIyU7etpLw)#F{X{6TAp5bE!GThHYSDadIS6ydY80J>?8vl&wk zmAPkff)a{?y8+o^PBFWjphdY*@+;4LjL)rS1zr$ieR{frjR}*DwJ#qp;4woexzA^l z_%7l}1fqI@1egfBYnBM4SCN2!ZwOP2$HjkP|2-PDZcM+f?0%Jn2^1q^G+Xl@Bp^Cf@>*`H)I+8O-$srGfZnQ+yg z!xD`_uhS2yFJpl}8A4t#h_|`7b9^$++GW*h)qm*y=$S1IuPvkeav>j3k(w^s!^29_ov z-@Rw&Iw*_=RG_r)CKGC27+%ltTY}1`wDVLca0q5%m`hB{S~@j}H(!2KC=|H+%HTaW zH-Vx`YBdh2(Qv5ABWV2T6;uo*%U!?)leRu@_go1y+8Gag{S*D7yjTCZ$sNObCVMXT zOHb_^U1P*}IM@OLVuJJIzVH&)s8qvFuP%Hg>u2?=dU##Vd-Frpe6HJ)l;Gm`Vf~-q zRGTgx-8iY)YpJvpq+$Qy*{6D4hjew%6GgAbdR$}}X?2kbx5uHto75aZYIABzDu3XV zpJNhM=6=&^^2vO5+&CX+h)tQ4hSR@miR)UVBd5xqCewVY^Qe4RZ)$=xs>0wQN(}mR zjA7YqO{X(a{E_GOF<*Vx`fJip++#wNT4cW6QFA&8hUmr=WcpmWCY&qw>7#OZUD-c- z+~p;y=HKxhFrkafYkvJ>M@cfhAK52Z!@aRhWr{$Z1hy@BGM_f4z2&noXiaE3`4S-H zxoqCyy1tB`Hiz%!i{6Sp3SE4t#UP))L5$_1=n=BPjd+8<|$Px@hQ*)F;6 zF3ZJsCDsKqWwX%%_a|pKY-VqDrF_3S8@qLkdzQ7)!zNeVhm@#Z z4n>XT_w)-cS2zieZf2g4-W)UT|6|`dLKQ*VZWo&jIWaEV!ik@w_*MD@j?SMk1eFpeKML9S2&?`J8_dTmsmn}&)QElg6V>O zf_pkbBLA@EAlZK|7^}#C$*MbOZSvtjqN*07P-hFPs{EztvR(V1zp9;N=ht?*gZaCe zc{obdV!5WJ-u${ufi(G8;x@S_I3FLI%yL|D&A@VO)h;US;9*W;o&54zNK_dsR9D5{ z2Td+$IZkEq%Rs#)wYD0Y>Ts0V@}o@hEjZDXFQR=1eLXIx_tLc=ug_s|{vCVc<}V^0 zT1}^>@Z>k(2Y^aC&rN0vx5=Z}pJTtC-+mh5wk6fRP}H7PTypShB66^7`n*GUCaA*U zj-LF+-Gkkk{-D%N*}#xp4)=$4V%V*T@|RcCEA~3Svr_gvSNLOY!<3X_RW_B#UP?XR zv#9^uJ~Am)e*T`D1|0B~m?sowifAo0^Hd6b3k*|yT2lUcUQ@ov(`?Dav+;6je{=1# zcgKN1CR*i_w>Gw#H-);}V`Cm(qTqrJEj`vj;WfHF-46Fc-z?{72$)tODV6=b+t)f7 z${iOeIebRw463Ss^86Z!)%ea1wE~X4S6K75LexVcxwj00%OX`H1Lm&r)IOJ#2loOb z>0(t0DeIE7yT|&`FHgbWwhRB=YW(YDaSSAugwmxRh6r(sp0Eo(M#{16J7 z9DqA7WA)`yd-MT~+toTqE<+=u^n{YPf;q*NZ9AOklklzdt^#fXZ?)4`jnQ?GYHTBF_zK%nM7i4E#FmsRJz%1$ zPkZNltf}~~nIA>%EwANX6@U$?jQRS6DyP_n{+GsX&cg?cGIc{1%ns0XRC38?16l`M zJ^fSiM~Ezs!%JFrWsLV#HLIKzM0_Z}4V>+A=5fDIo+x?y6RIs#XQus`Rhb&t`@v2D z500 zOH0sgv9U@wuMhxlL8L3Y$G$RDc<`D_9sx6E_ns#ho3P$Aa$ChIxQ=Zf%6QY{^W40# z5KN|@{T)du=p#=_^=&KCmXwy-dcOMwAX-U(9Q*o`kX@*>&zs*qh&1S><}+xJpLhH9 z4^k)9$?jUGw?BAP^fZcP$Wd|jD^l@~ep|Wu!G{?VA%Z}+s8yEN2)U|beEw-x zFqstfC(5t=xz+0G4=(9Jx|$wBq0W+_DnzfeuF6Hh$~PI?WSL(Glh&!qPa$FPbQS{E zPDZT*Q)^G?7qwQHhsXBgn*NAU3-$;ohs zDC?%AgT(voMcteFz}w%l0HuxK4Wn6TW`u;}Di0TEL;!yXO05xtEs5A8Vy0{GAZX5E zUrMriIm4d9r|lvGUR7+j+4Hdvo?G^a6T=#z{rT0Fl|k;aKSQ;n$uPrH)zE~3?REs{ z)VWx<0{;_p)zbhPF=0GAk}M8!*m5>PDkGk3bnE%u%^5O+6cs4ln$QaMx4i!HC9!AX zPeuD2{)ujvcfj)2X=h3<+y$saaJ&i<{UoN3)N^jK0j)T8CpPp}is-3G7;bnp6-qc_ zC%4rJj=B%*5aVfZ^^>K2L+6H4a~waX!TJ|rKzcj4LI(=IZQ|UdH>$6m7@D=sv9@6( z6kL|}2NcV!MSSuKalU%hn5kd0c8%u4{{0t;GPmbxB-nmdJG4?zLx_UTXJJ83s?HfR z4Dgtpm_Z6CX&}6NQ?+C`{iaVn&NrCdZPv(rpt_cYGxqe39fH^uc&75oAP5@(h*5xx z-dC;i*6p`Wk#)Hnq$CPgWw3|)a7Fl8s&TwpHaom}E|#IloY}9;&}uS;*1ji&4Js3~ z9uPqeS3hJxZ*eK+CaTW0awer?%6`6~s%|&ZC-!djSk=6dK?Q7Sbo4~>ylP^V^HyYT z$!~P}IWGpVn~!Rt(=?cuqx4d)>t-B(#Xleul><-9Y=r zZxIU#CRJOxNECc(W!(?EQr_EmK&C0ZT#P!{QXykQWG-bm5pE_vSBT%WLJz$Y$Ft`y z$Y2UE;)5v)T-JGqDWf&*8~F9@;n-R8!uIa;kOgg*gtX&O*+Zvzh$2i?Zsxe^8CZ}_ zM>?HTY>~xFDhc9uiu^7!dC;Hs=d43f@1$n4yKc_5KODC|S*#DKezr}h#(XU;@YaU3 zRzdDMbr;CBtII++AJugvV@y3H0FbtofG=`Xz4!6HgJ>_?2IT}52ZRMO!fAEZysv^E z)aC!;#zw#*7RBa-RnRs&Z#2ejvqn0n>B#5>WQMJ-R`S<%A_9LPBFl-6wySDZgg+}H zs<=DKhSQ{?^;FYJD*N@sS%+NE=Jq%d z4Evrw#!%7*_^5%f3jZ2=emmJm>F)WWmUrJm^tEhRX#Qq)@&bvDI=FtED-}RUwnN)j zRowJBs3Z3@EqkD77=W+6X4L+T)vcnj{P7JnCTs6{S+>6fsb}1T!b_zQbr%F5BsbQt9Kv;%{sI&oqhOodBZd&AuNyEKjdO z-A4Sz>l53rFD+EjQt<$PsZ$g(8EfusIPuYo_5WVi+zt(AWu>%Pw&yy$H2$>SU zF_QGATMn@BbC)w1O?yIRCq}$UawyQu7@vRfoI|efQVTJ)^pm?Y`rqt~A4evy10G>? zQ_C~i{T<$FjLevfP|j=x9)Gr>9(KcDp-mTu|Vh(@RQxi z#Js$`W0gh`*LAEhU4g9rc|xoW4j2oN-fNUiFcLroj!P8+F+W8rAx3ta*y=BI%S)Od z&Va41NDpLU2x$Ymk1M=@Cm&24PU_gLf=&&cyUx5?8BFx@ilcj-w6RLmsaHNY_*?V;m2v99PglK0+%&C0_^j)G1x=)NRK3aY+K(G z()(=cf{`jtW$xMlNG}>xfV>AO0%79|hBNDAKRB{WJH*HJ30za0hUdJ4!HaG{TkJH2dbGQn*QAE z!l|eel5Qlpu)P0;$a=IEFZj9g_>prOgn6vdlM0mu(HZ{DR>0-27-RzjZ zXht<(>Ra>TiV!aO;nyoCzmR>XwXDk0Y#f#G;Qsld_QwvZ>dRD|MCHrmAU7>6kF`IT zN3WKLvOXvO(ivhWiavh!!E3}s%S(%}EaiZdE0Y2&KcnRiTfS=g{7zCj&ozD@^|hb( zv&T)nZC@HwSOu*hr{9f=2v4ADp2^DyqXpGGMrvfM3Kd8A_ZJGhh3fK#@8e@tnY)=b zCjz}I@1*KFIiZJ}vP(OlUU+$kO9t{DGckb?#(G^-0a9OAPD`eIp$MjmdqO5xrIV!A_?f zpC0Sp@n}|q=MQJqS)Wu#pM+HVqouN!5l?7USc26QFOzRe$pWtdkd!x37c;8aVz4b? z7B$2sLe8_*>IV(to=Vn5R%&{J+NC0)A`&dd<1mv=L7h)|bLvR42hav&KJNT_1LNIr zYZio|o%qVTxSy+0x%c}gnwL$y-5cZ8GQru9o&D=C6Vd)(OvG`l$d8(kvcTHK5I7f3 z9`4^!jZN5mWLRHS>1`uecQzoDdcYqqPH@)I4@?)du*gR$Q|_tb-HKI;7Q#H74}Do8 zlb0&+hn>l#lh#%z)!XBT-DY&t))0aq;93WpLKeSJd&yyap_%!i+ZJ3Cinv%f5e{{)Eg^d}>FD z=4@AUOhL6x_dBOmTdaE{cn8d}9it`*g}Z>8^g@rcg-?c&B~WTl-5P$ z)<5s|l0Hm~#JXaBmpHZC->?Q}%@CIXbrXID0x)p8RBJK{gvjV11*ORfetppTu%HCt zwkm5PY*ZaG>Fa4ni)%$L#QkM+n)Yot_23=}b9s*lz>`Z?mxTc9FJ?De4%=m23E#v) zO~8t?iNl4+)oA`YUB*bIU9^+{y%Xdf_k;-TspD`PLf0ZX({OrdFTQpjuX^ihZ_V!k zy!p0C>5%-uQFcv}n(Wi2kNQ(&5Fk~x$kWGWtYap|f?IG!@SGRn*~KAxR(svA*a(~7 zEOCOdM@9RxQ+fkKDaLc!7H_IWr)ZTs4yKt(sRWdZ-m7ksi86x=ZTrfN$VNSMY2rq1 zfHtdFAWBqC5SUivsb$NW=XA7rz0H*%4<)aD*UwV#BKABlnl3_Rg)ZA+Uys=A6x;I3 zj5n_6aDY1=t%0YbDb>sRvsLn){+Y6Gff&av)XY?}{kRw8%>sQ7RJP*mXZ>GPaX;u> z)POHU3dd}>Bu(rsg&5Q2SJW(O_aE{p8?H|(RpZ@+l@((=z5;NJp@B0If~XUmuC}rw z3NF5VS!w7KWM(Lln^29P$cHJquJh*%VPH6;{=PVs&c$y^Te7x829}50X<@s`6y8F~ z`bclVVzEaLPQSofja8s~MBl^4faQaC`z+nv@fq%Z*VLcE?l$LWnnDtHk4 zi7g=6%SNOwumXYf8pr4=5ISAfe+WRcML;t8L)CuH_p{oXk>W_9g6Dzj-OPj`K7FVb zqnV10&ITr_l=uuGwP8iq6>aPn;CCQYN5r~M9CmAxFk_JF$l@^x=N)0fZ|p0lNfQK2 zW>@~!3>P5WhLprgZrY2ox(cTSjDSAyYo=MK(#msK<86!c2{Wq+Bmqf-9WtV6EO)kO z-OI%R5e$1;P3O7`V@|_dp_5f{nViGj`}_Kk)*8@65)8Q=2x0}URV)^5<4fy@K0ORH z9pYzSj=KlwY)XLs-lwqQ2XZWbC5{UMYD&A2kaE{m#~S{40O2bvI`PE1?tob6NMe)) zFuj_JSx7D5ZZ?XY2@k!PVt~~y2TqNgUG_q+y|DE4SIn6CN)w~{pmnO9GwuB%D)~K- z^yHErZjQ$!qR~83K2bnE%+zcS3B}xzQ|%8*8N~I<7q#5+ohyR9RD6^^*HKheNKPy$ zJ}>VAx5`~Tjef8B&^X=X*6`|jXZ0JfH%MX-FOvzO1cNK2=~L!adSQavOXw? z#R+yPZTo^PPS~ysLAp#qil2f^TjQVCO`Lywk~%qZw9$bS8gbo1!jOsRHbHbmNt#1r zjx}VWv!G2|wZcI3ur>Ml-cmc4B6cXgQOYrPx};ml8O9d*<`vALAY)=|NBi&h50jmB zIc@!OGU$ItbwU@AfF)IrBQf2wevOTRCvzIbSn)PpX zrzbBh#1Dny-oQbIF~eg=hwqZL05Z6h)$P-04XMP?lPcSEDZkUbB0k0S^o4*K0DmRq zvtU4PvTo7lXw4}Uv?7F#HfVjTdn>pMe1lNw=Qy0ak*$z}0!%N3o;YK{)~u(EOYz2_ zt`UK1jE>G&$VN`Nkf34uDo8|bZ$h7R%sLy-dxJ{xe02xw$w~wsM*aG{p&CEkcO*U2 z`Y?Nc46sH94HoYi?v|r1_5XoT`X<43TAz5`HbN&AkbYj(w-tE1Jyux27MlIK!o25^ z-{~6Or1QvPe{BWYc-0EF1h_9a|!C5ngy6?Veqdnlnb^qmBw<(H|3yzEERA z$Mng%;`k%UY!unuqN8icEyf@*EocT+f1YnNT%5;rm7ybK-lcT{uezD-`40s+_q9IH z&=tn68w~dXoknK}?j&?>d|Kp(-21Y?x#1@~+y}n<^RdZil3HA@%1g!i7KGQ>^PAe- z{#rH`vi{jNfIVmKF3poJjq=}SC@@E>jyfT`m&%yH%jo+Qy%pJo;MA917os}!)`3G} zbROn4I@gH-v4=X!2-LK_g_H)#YdqG*U(z0cXL|9|JW4{vJ3+AMIOofRLZtmFKLqVA z{V7=ja+1>%Q(FB^p3TgtP#KIs5hpf-l<*AeBC4J5__(mvC>vRTESsF|(w^RJz@fGF z-20kd``$GRcT$tk(~0Ud3S90cqa0nKaP|5u*>)38!?DYF_4l)DWcyHL;xz0I({j}?u zY!{|1)r(j-JKm&lj~0S6<%-}JR_;s;IguSbR!?QeHcG1iS8g+q_drJjE zBM5=2%CmAYK-YwbJwV}^9pzdjS4RzSpl{P`&si8P+kXh`-mdq>pPfYYKioIf9VzHJ zh3;(MM;@%fljY=2D*bv4lyDQKGkJ}j$uvO3m zT|^LOPVXA)GgDgWVTpKB?EXSi+7MfC0OGKLFZ>jEfb@Dx` z)~fyeZjnKjR>5TrSk6a#7 zYpaQ4gVEm-pUeQz?Zz{&gitd9l`~%q@+4xu8oznEESh!cN$Wvnjo(OoX^jAqZdq^x z|K>+m-1W2)zb%V*w!(urs}m6?c_V_Ow)?*YN5vrPMH0eIr2o5qxurw|07fnN zc~6*Ss)MrNP@qx+6SC~s+Hv!)qvF7*MYXFf`ut zwZ)cu@D)2|q7-ceSuaM^4pl<#I(8$4xTTkpxksuh)uGFwkOnbQ%HrQQR8?J=Fb2!& zqdIdflS|{jgGiZW6~DY4fZtZ}9z8-Xw|A4UdCv84!KQrD7`7x(ZY9$HJ7Ps`q~BV} zkOad4e`|REPU3g+M5SG;7e?Zc``Mx|IaE`>;N#;8MB=8pk1PUu;*!nGAf2}(jTduc z+8nxVbyxvt^3TpFVpk|}t-1_F{34MG5`p)AgUUQxiS!eas~#=jlu(6A!;#OCOMYj? zOtQ1c=^3Olm~goND7E(}QX(sP5BVxRsnJ{dO`)>?$ zOKe0;9)?dM%O;Vq$@S{T!l@3t=mj19guvGFvk1Wk(h?oCVl5c3q0@)^{B?m7bz1O> zHt~%~l#P?oS!fXM+jAvVi)HoHNnAT>G6$;E`P^fSdoSLAP`S^jPw?MP6d?>r(QL+I z+OWx)z4n&Vr1YyJuAMq(To2REio!j@Qd%|D<7`juE&R0f6nFUn3s@5B`Zu{_k)EG>*a3m6j+=%% zM|Ykq-l_&A6xhtLAj_up@>d0OOT|0!U!%+Y;v~AjIT<#)-www?se!^VHLja3uGqUA z7q6dLSmAQbbB6Ais0@Lwk-%co5XG+R*+`6T7RYt=^=XColI95Vopw5SWYL_cW=61m z?n>MB61p>RQ9kT;2ZW6n7F&iq9u( z@Dmbdtvgu0O>}%*cT{m!0Xiz`N8t_(6|<&IZ%lL-6pAJ5Z)|*#Ks+b5krIp|td&Q< zZrr%Ot3?9kwWfpY(gbe=2;psWk)+>*Yx1zr-_C#0AP~Lo#Bq2WKcNMGXwBZ~jB2yl zUz%Mh5)oM+iZ%ZR8hOT!xs4i{@GOHY9@Re6G>B7{fxAoN!?95B9q7ZOu241 z6V{7R7C^*^IcYbPr(w^x!A60n>-Vg42t&jnQ&9Q|!C46LZ4uWO*|wH!CY=gb@x!LW zrN+s2OT%%Mgfx+J_Ph(?t^P5fMfK+?ghcAWHp&+X=+yX@i{h@{w#Wq`sM?&$^m`i$ zG5WlygGLX$#y8AuVRi!q4S`fM9=@a)`>@%p@3Oz(@@0e2`Ys}qv!DnRaJ~gL+CDp2 zYA*}-g`R}_Rz0&@H7ig@H3ni4#6L2<86c}p=d5YuGT7oFm}BqNW7ZJnvq{90labL| z7h=6;3SqKI*YSld&9wZt;u9$HLpghMWc@x^hm^?d8u6be#-B|0C3(8v z%b$H5_q~n;#`advGHe?g?S@9>N3|+a>6TASdr?cv*>5#q#wC@h33zh@$^c(5Ao z3}!?NZFcTb?3H!Z$Sh3m)|Jk?>K``OEuVqP!j%v68@Udb2riC0U=mTRQf^~n7yJF{ z#q@$aA?;n4Z_{Qc)-@t_doT_*c@MDCFUp+c(|jTyspa z-n_UY(4Vf8?xXy4_M|*R^M8dyt^rq&R9=}Xi{8#t3E@3G;~|x5@aftRBbf8>cX@3FVdfTg`d3}mY7$7o)u$-=2*2E`;~_DN2ktl3 zjLc(?&xglTwlRNZr2Db=W7;+ab`z5U>sSC<-rm>oJcIoTn}Ah!D{{<_GG_EZiI1jb00&G00~ zeCFP_cKB7m3Bpf1~;A{IE;6cMxPMA*W8s@ks=%enXD0bsErlEY_=HjqzUzJ zw4e(5_q$rwGy5-H`~PzENgDSKlAOew%GhUP)0MR&`g79UJ#tftvs1>~iAlLt(oqs9 z>yUJpI?l+XGUvM#uS_Wl$$3X9`EvVPz}cZrmzX6$2wE%2AR_{PlquqM^ez za-nUv_T7shHOU|dIaw8nz5JYpXB-T>d~{02Bnn^%)iEij&PgQ5*R|Zy`WpLsfcWLD zwq#Jk>zg2QvMuJ_89 z?!J`!_%L;TKJU04;PQHUx#8gxo^kQR?r5PRhk(~q&Rr!%-L5T@-nBR!{A*+JrFjQ_ zmo3=!YZcdfkIlrhTn~N~w&ABGqGHiBT9tty?`AdjydBR=BsuQT z(xck%0*1j{%KmR3w=G@XOHl}}5?1fJ?&cB7{W};f%c4vul@Ck$=~YmfYJd4%%hUOC zLCQN3lDvgKqt9H_2gd@Jz7uY#DiL4z0QTN-u=Qx{r12k*A}(GB^-Wx7o~z%iC+}!U znGcb2>{7q25lnKmpx;j_jgA$Fy2GcL+wTn_yG>5bdX7BRNYxR4N&}-mI)_kS%l-Mw zkDK*#e6|bE=l63eR10~YeQk`%nOv(i8a!^)??JRJxglBPyy;S6pi89QFd?|_YPPH8 zg@Vf&9=@?*&2Ytk+Lu{&x&8Gk@{IL&xqo~EiPc$qKF8-iQa%L1y}e2S9XBi|T5b_u z`I@Y3vwM&E?ZF>r(o5Qq>}N_`kNH+aU+$93QO#vnK~=5{zlqE6*5)I}5mV;*4xqjU z0wqU+xYaM5#2i#WOzE5njWzXl_pLd!*Q7r&i@a53jn*G|l(h`4Gnt9X5} z#5ILCabmv2opTl`KLPCxTG|(^!oeMsHhHQc0T<^HWLCgAZ+foAQ#yIV$nf40`cv!v zctm&$)mIQtm?n2X=bxUK0g`iRfpWMe)BHYA0XW? zEaD>L^&2jw65?2nxrAB}C-_GieNQj(kN=! z0dLe^?s!aBtlZ}~Iy;lHpD?7b^Vx-oFo~bSIUfE~{``3IcSmpUIfTA)%qjOV zKbPMeID@cw|IdiglL7q6Y^z7Cb6eh5ecthWPEyc#EXf}vE2Pvlcj@wbD!$i~kE#^r z9GpAn%?_{saNgNG?yO7ZgYy*DIsO@cax`C4bEs)i*&-XHwsG`l?&s>IC17{`H%hOS zJGb{&p&!yu&PV>V=c9uR;ssm5Cx__kl}RptdTwZ@$2!=`KIHI}Zi|J7mpY-E>Cmuy zDzEA$w|b&YndT0y&EED!Al@o^^W9PMJ1VWcD6)U9Y}X42-b8G>Rv07q@;LuOS#FFD z)qfSAquT2n|Ch|J>hTrulYweFJ#-=IsjaE&Z!A=#RCzMz++R1;t|xWr-n{V4hx=C< zXK>hnat8v*$L2t0CgoVRYz=o`^xRD^@b!mWB2ST{x;AF+AosAnt8KT^l)T`s&jpGV6n>E2^Z zK`nh`Zc5Md>b--Xc+BRXC;Qa|HJ+e$DwQ{C^^WR;M`>j&@T^a{vC3oxS)=sKXT)nt z(tkST!8)#@ZNl|dQm&3?kTc)pXZ2U>m`FHm*Haql%wZ+F_Jq0?2iR$Cm z&1Ui?KxTDqGvMRNS@+p&=3c#5S+Vu+wzVm~+`79m?v!g%TQ-Zh*(Icw=PahT`-8i0 z-S^q_F-7xj5{>g~J09O|^4pQAIoD_Qrs^EJ zRmX^4o4mqTp#OyH5yl$B2h50J{v$5fCXitf&hM!r-N29Kc9TSYsMVAOYjjt-ABKkl zTZdXd(U;LIrp+izzQx_8K;r9G965%ds!J=3X_h33U~~*;zuL=MmQm_li&#`jiqzFX z97l%xuX%ISF>tXUUpA^Fz+3ZV=~NG&)4m+uDn?@?N`G(zjeja{j55<9qdmFBjXd)l~t5R zY+MA0niFXpuXnqhhYP+ndL*S^d*ufk#qN5@V7wsvK!XZ%et*X|EpAf6hX)2T==vSe1DrvDbv!~PuG=ZdS4qzB=i)ydEgLt!_Msu%I z;xaG2O{w+A*4@VF2A6vJu(nITDf^y>UJ|)eCiygm#S!HK-KJ@}boNb@W@TQB(&EB^017<0Yt_PJZ2`^;HMyFh5voZP-Paf4W zy}6H*Ve#$vW>Hzzg)d%_w|rXZ!#2zS3)E4o57uR0(xAN`xt#hj{2j$j=LT+4)>FIY z)7R#+LTOlAnbcqqqBVWtV6$rSw#Nz9p4M-_I8<_S2T=hfrklF{a&@^tQ@X!ZJ7KaZ zwXkc`rmx0O4Pqgh^(eCk@xsCbcnjD@F+w^>W*nc>$f1i-|y#S9JXM}ulps7FF z3hrdZzZR%0j%{PwWx2r;a=C`rFfnA8+f(+b-ti@!6;xzozXI2CBER>m5rG%&J@rQ$ zkyjV5Y>Zusv%JxG`t*h`4%#T~cw^`6^hG6uN9WX$d#8e6PGdzCS4!k=r3};`5E&Jv zuIO>q#6@G}#I|;=i)F1avmg$4D+8a)5NBm>1BuQA49rIKZYE>4Idr+DI8G_aS31b{ z@h6bUJGHxSR+L7|S&it0#GCyD;U9W_eL2>{c>ChApjzpND24ps`}01f*jiejDZlGa z>0tDyloVR?W=9lco|hk4P)HW=rqt7s2g_3DZH20J`MCA@Wzq}14AndjjGu64qtNL7 z@osvRRQ0N+yZ>mD8Kgd$Io7mU^(5XQ8R5i4BI?V@*1CZ7eM$uNr4JFkCy~oqu@fDZ z4O2m7Q%C8tfezEh1mLS?*=>5~0D00@YwLJ+U-=2}Kzf8ECmwoM@VfJh{tv%tM~=O- zCsMAht54-##eGx^uzLTo+bqTVXIL7;x#r`%!j3JDK4D=Qk5Wl}9?U;#VF!aN9BY#~ zI0fMU6*WxD{zi?uY_FpQ*w790BrEf9yhS9*s8ed}JNA9}lhCho>4=-9MZcM^ugo1y zBc4{}OkF})m_kCZ{$83@PXQ}tZ>u{j_y5J*+~W*YGtr>YQz8hO{~$6@K%q*jX!#$Y zn>tD2B@_m7-)F-p=+;SPo?Q+T{YcJbNWImkcZ*Lu%O{Gos3j;HajyKbhzkxh>MY6k zyP+{AVLnt4@ZpLMNl$^gQoub;%ODsf7a)H%OZniK$635fL*dujObMMd$y`qR$#^}O zB+Ts8I3|ojB%PLv$7lMxM2Sl&6jE;_rt_YwU znTiZnuE!r078Cg9M6U-%nI_Jeot^NM+@oZwbRj9j->1Vo(PTwM%#Rpd)_kaN6?Gx9 z%&D76i(g7ys=z1Gr%EKQ%l0ugzCW$dN!D52EC%gNgDkTMr=V4#Y(9%=``cOLB3ypu zRTW4DZZuFnx63cDeo>?TJfBEk%3g03QG!4ZEV=-yo=JxSQM_jJUb8=B==T zApYWry;%aB(tHq+VM5Z<(*G4H26sAPS-?sd@IOVg_1}%uSg#=Re{CerzsYGQs9xm= zcauWg_M>pQofEwZHX%e}%tftj;7*un66=8X_SDN-#3o}{!flG`DaKck7p}WA!1Llw zikPFvcE#?-ArsuR$vs<$H_@=duOYR)WEDdJnvU-+FW01i45=w5Qe2|MHtf8l8u;_H zo6DGGALL})dCt9H_1sR9u6ZUTv}~9U($rbs_EYI5Se*oDzw6)6P+3))-B(5kBpx0q zChs2M70#O#RiXXSW>&0=ntEEMA#OfAJE;BsfQt#j_X+#fjF*UT^z@(QtMfZ*N=LuP zd|QYFDW}7!89L20pRp<5+f8PH=6_%ijP`K{Z&OB`@SFl`s8{e!8&RccII0NkYSZ!Sv8_VAVZD2%c%y^`XR=YdwPbPUNQOtBMC>uJY4@ zRzB@z1?PO?jcd5$!RGms$b>q@HacYnSCyku_x?7j#ra*Lj_6Ad_wTQ~pu6#zfJS?x zT`s=)Rp41BJgWa<+AN^_$Hmn#AU#kuCbqOrcFg5`gGXA0n|+74Xx^}n8ipEr_KL~1 zGw+J2JhR+7)0%P{#=8@`bONLf%0qW!BPA}>h}g0-soiK*&Kd|YW^oWd7AaM&Hah_=#q2^UfEYMG<*i*6l1?sg|^(6z-{k7G_Z-WT{dLt4)xt(tBwhGs?A z5PSMtmD#z&V1Y{w<-Y6uF zt7`6P)wo_=30$3_XwP+uBE42}Sxl=4B4V#aqSd5ZBGMHGv)J9db0Sse`a(W%_p9dH z(TLHeOwNnT&NLTj&u8S9>&1BL!2#!B{h&)^;$@~H91kPTLU3J+WuhHq+Mw);^m zvAh0vxB^bX$4{1!$6MQIZV45X7_`w_XR7h-%c&C)GzYuOnL`w}3P-;)jMt@cgMz+2 z;=t$aAsa+)t%%9uav5|(IA;7n4jv9qlT!rmPGBR!5;Q8Ve#MoV2(mGBa3@kk*2@1LRu7= z%_M$p&a)`~5QH91$+d6Q?Jtz+9?|SrUS8%a_iU$I%&3G(M{v8D1XioO6`30MiSiz9K3h}eV&aWnep?EbmKUwQJDu$d(ffee3&)_Bm6`_i)Z2 z>D}V5bMKzC{WNo*-DF^IGF+o-Z?(1J*l4!yYSmcm0zVn>|QylY1(Wm-+i%nuG56zmiN2(xIWJk@D9+3 z7h63ngJD+QUbHH-252r6ZL5ZPuE3QiFL-|Tr|nOs$oN1rj`{!*XPTjKF0M3YycOsb$2UC=9T3p<$vGg$Dfb2}j z*LvCq9;MHQIVFjO@8#z9zcGC^d2If>oT7aBD^-A$+u=)9`s6ZHJ#(<;qwhnB3=fNr zv&Q%ej+d6qfBVLz#vb6@V~Uq~#WBFVV@ zTJ|FYZgE{kb~-CTa&gjJDo`(uaqlPZaN2ud8yB z2Fhn}rQx?15cs)4OD2;h?GfxCk9$(q{i3y4AGPBq7Jp`6j)m+Eva8&#Ihz;!f9+gp zR8vC31S5$F4VF~1cg{_BO+pv#bA)a5w$`< z*~8i*yUM0uMXCo8DT_3qXxZ%K-A7(Lr$76r=On+#<<87E-^~5)y!?2&K@~mjwI|e| z`;&DrG4o+fJ#h>WOOFj;DdXGA6<))UwdJdeKLn%x6}EeR$-xV4u+Gp zw%dXGrta6z3GVW_+tu|V)Wdn?AS-*QR8r*BTs|DVZ$XUVP}ax7y=zpRja(V4_XTc! z{;s>ZtfHBvsmUtH{ZN{jtee}Ez9>eE@OU2D=KSo+eyLob2)@t$^ zZqC_PH*6oL?<6g_Q~4t9gL)&%qJ1$Y?pSF*j6A5T(%F_)L3?$_5n~YvA)SJk+YQen zU5XWibXu4i+fq4U+tWfi_oSuWA(v)Ch?!XaBvQttxFW%;?$*~hDQr$k^`=j%T;|Jm z@VulQlxwCe$9W=?@r zh&oq@P?(Y7jeNRcnE2#eie$Q-J?^`ev#@DBDw881Hn?j}aw?^sCjis}H+D3l*^`O; z9((jmDCa_NKAl~2QEeUapk;)Kb=F`tOJrHrtpErICgiXQ2oXt8wpWh5R0uo}avXg0 zsCpWaC65=<*|lT5Z(s@2hO*q?xXeJ*ZU@Av_dB%YNIDFtkDT$_jg}J3A)fh>6BG$} z0t@$Dw%U6l@(_^=5I%+&UP38BD7*ra44?VjpMZdeD<>WacgT{fL+?B~yT#JdfNa0X z%87Suc`|bv?+VyoN#4H?3@R@-y^NO_d1}CX4KloJu7T_v5uEcDJaC`1O6S!6Z>{ zY-6=Cd9+JzLlB3m0HK^3U8U36ht{`RlVZaG@wyq+SVn0D-E0HQdS56BnJCJG0th-_ zlCCVL7n_t5yDNXYj*NKGpf}6ZuX#|{Yo#OZ`_MN%dn4gOtObt=oM(fZWw5bRAlsu0 zb}u4j8?N(c8aGZ3<|fenO?jL|M81yfW{UU)Ja)OKTtEn7 zGl2z;{uBmv`P7BLD`{x=_7c&Ql-~mj?}x+|2oT}N0c81R$7@f-9JUUU{`F@UhHM9K z7R*BG8bzsSgkV9nSUMa{xt4%p4a6p~Q9?}EEz~G}NbzjP#3Y^K#j1t{hM~ci`$??W zaeb(u<4RX*Ai~{rCN^9Wy}2MVk+|jR5(qpa0e>=Qw*iG`1ZWyLbo5680Tu{&hgDiI z>i@f@@kOZ%W6Bu4h1^gYZHBwO%{oWKNE*z)MS{>~ieY-}Ql)yofNT6VZcAzDi~z;ffufN`#%P`q+TyS%D3f-JQc;1q9UQzHP4XQD!9 z*V@$Pk?q6)my84HQ{|3?{I%~{2+R(tJd;cGv+*z?4(wOI@oJ_jy%REb5Qh1wAT~cA zK8aiw0##q({xeX_1nKPNy3bj@#E0QjJ}yw(ic;5tc~8e|?$^|)Q@0;%N=tM62p16- zpuuajdcT*U#5DkxhEFC6QOAMZjjwXJRFYLt{;G@j&ycma4YhEX=VK-74<%(kSn}%< zuR=?8eBfbfMjUxUq}eXF2XcSfK8qXy8zAjQ;l(I2ZP?M72sUN6TTBQSmGEfr@MuEz z2CAH|0c6b-&%eo1obBvdN}Wl?K!~_d41|iO6VwTcrxU5ZDIS#-%BWDr@ta)nyNz_5 e_+KdF%TG#o_|s#IOFpn@@Uh>>v=Oa$JM#}uOk>gj literal 0 HcmV?d00001 diff --git a/docs/images/SqlserverCDC/Sqlserver19.png b/docs/images/SqlserverCDC/Sqlserver19.png new file mode 100644 index 0000000000000000000000000000000000000000..6872934102a1fcdbddb0a847a71fdaeafdce5345 GIT binary patch literal 200830 zcmagFbyQn#w=LXK3baTI6ez`tySo>+0L5L3gyKaC1PBx>?i#d3i@RHK2~s?`y9b9r zKKgsl8RvfIp7Y*6GDh}JMt1go=2~m6x#ko4QAG~><-3)oGR>SK6&!~iGs9*rpMER40I0yty`p{q~2GVdjmB>=C!$LV_0oz^FGr!p8Sr# zQ5c!@o2LZK;|c}TtuS2N=ev~W#*!$zt;uNyTW7vY+*kRyuL-W`PnNbJLqiaar7IVe zI4@w^8992v2|aC2Om2jF^?=htFYEX3M6kq|-pg*gI@&Gr8Rfl%ro2dVq^_xA!xJJ- z{QTEM^;0?vN($O|oS&!}P8&{BH0hVUE5FsIxvz?Rmo+?k84q=q43_gc=^0O~Ie>)* zpyPoymS4qc=?>;lQYHvtD&LuP6-^a?5g7Ra{(=$HQDaRFVyNNI%crZ%PS-Ie|BX-; z?~2y}gcqmAs(+Lxj%mDmJ!9-DsBk0mM*n2ZwiUu{MfKk2R-C{CbFJ~P2vfHSBtA;!&r)L)Wl-%#?ri)RNlgU%^mfXPt@X)q~WxWdYJZv+n;K?6JWeO z*-FpPMVJm-pGsJ%Jr*(15Yc{4O5-OSqrMgye@itLMmFc`&#Q7na}`hMJCEckD%C$H zl8p3msYGkW=QBU0{ehbB4euk0;)fRu|Lx$U@BK$j&2Gn8!?Gf;Fej4wxYXD#@vOtq z6oPq><09pEhan4j!(N42t*hO87|r^Ad)7f;?G>8K`m(KqukBcvyBO(x-_C(~mE=*6 zQr6%ixDk`ZORzr6f>eVb*z%Tl6cwj<1NpwA&E?p4&cz!$M!&d0lZ-4{JGb3uRwHQzINdnfFD zS~8pc>y`AdBuq*DtCesHpS9ILueD1II}YoYMrM?f$Eu{{Oei5G1SAHR^cKzoc{=fF zB&us2tm!VNQ-~U$tO0k1PZUIaZ3&V4eCq?F+jUT6=U}9hEY|eHPty_GFE|RIp#1z~ z9@=DM=(*nV%|TUp{~LJ!1gft# zB^A{#mNk}Z%Vb|%*-lh&ktKL9&&@Y$Q)`W&`Q}^AaO7PH*r5~oFz|^ceD{cEZ zn?Y>vHgTeKH$sJ*px))8%X>GdF+<3PN?ocTf_h{7w%ODE5>+IiW+AfXa0WtaZOc$| z3EyfcxUo9MvAY1qnv-qxP#T4}lT-CEQL=r-J~SIpp0FoBk!fv zpqE~=4JS4V=_KZtl`jGqUr=A08b{QdM7_G6kSU1>znU_y^y#VRN!YJ?etS6;MS{3_ zsb1CE7uq+PQl6anH@QNLvYnOC*PN;TnA%sw2kbtq&Pzh+KG zOO_fI+klCj+|CL2mJ4Gse&ualwKEF}kI{r1+%4`9I!q{`BH3kBq|3cjY-aKF35ac<;rLDCZ>Bsj^iFVi^@pD$3a9lF9yK<% zBQ@xy^~_#7ssM_Uz{tiq_b9!nI0VoA>5P#Vzo0c$Clfv6Cv@dhLsejY$U2j}a7EDB zs6$}a*b}*xtj3$m=1N8GYgf2D`7S9S37xHdk8QeqQEsT)x<&XL8~>BtiIQq+zcG%S zl(FxHeOdb@PTX1eki}uYFNsB`t*QKW3S}noJY%1fz6>NCn4ps`# z6ND5j`3BuvD!eg3_b6JXemrZxa1S$DKyI|_X3&_GzcfsEo!k;}*R-wiH{=ECjn_6AZ|`DU}M zfJQtDQBs>re{UiEsqXy{a;>`Mo5QkKeIDQv^B@?BqRByj`)uTO?8kfV5J-D&lCE2;r_`4-Bom%GyxXuHDZY-F7QEuJ35Qg_5KmA4fY#J7$i5?|=jCiF7d zWDS(~_Q1b)rosli#?FXfhI{JP?~zCbAhn3ee-zvgW zZYz6CsS=nhPQs(PZAWSbL;WY2IBO2v6%lG33;9M0@xu;!-3EE@q13n13S36jGM)a&`$qhx)nCXAeeK z4?z(@Q-{TRops84HT&^dm$8SR7TrS0xH=M3tauyc59{iE(tJi+?F;{$#i>9NC8H9R z>a-MpZuTlXC!F<=&-<-NJ3mMXg{N$a9>%dPSbPNL#aV5}+b0r7cW>wTc0*YfMzGi# z&t>z64D};B$CUQ8{!O|3f%=ews)?0vR#6nYYjzp&g>dLL@HzLgc}K}BU4iXqTL6mA zgF;76uR&9wibnthAB{tFj{qp$)>(ji4|#W!4~lKLHM%Pj+9Z~y3NYCNAg_93tfn%c zZQBmw$)6XS2&%1yON-Wgoi(GhoI;|a>rB@WjzjmZ&3e!U^sD7WwGZTuNT z@YtnVJwn8AXS|ZRP_S`JPjPA@*knF+0b2gxy)iTG!m=sSU1li#`A_@4hvFLNs1rF3 zD9O?Z9Xx6(l6$-?zb@<7_TwoYG0SgMI*bzb*lto(zNVyHyK;79CF}?2AI?B8_ILfS z!N15E{MIEt^qA5Iz6CVh*85tDkAHJ&v{a`t*};U>?AbcBpE)KZDE+e3t#xJ{J0m2X zjWaTa&(`$^=H+Vz7iiY%aTka$sw8H8HK{)%+fA*};kB=79|>m|#?DNQ$1jvXRWt)1 z3fb1NSyA;AEdmc+d_^DbW6Pn5Ow9K;mh97{YJS{oozs54#%%fkRC{!- zq$2FDMGVUSc;Dy`mN{~tmvhcC%*=yb^%V~)gr(2&J-%Di9eL(z4?^cWYo}g@Xw zgOV3D_wOiqNqv$Rv-GQtUV*%(@y|z_2iWTYcD%mI4~PdBWLr_oHFX-&yq=Mb=T^lv8bi25keC7IDw{atf^@vO91Da5bC6!Lk;T6H_ zyN$fNqGaSfZ|I?AnR8jwkyu=H>sOJ`!ofnhzZB9hKJr^whe{o}mT$uEa_S#u75K7P zm-|B3x{4r?2ljiaC(M@P2@Gew(xSlu;#o@}Qa6@;6$i_eHRoR2POD#u###ns=m0e^ z$LWOR+RbPC_#U^Q5?ICl6`$n3I2{UCMuI{r3$Hci1y0sAnTAZN!v95E{}008Cyw!m z(89MvtE_c~stQ0@qi?=5d^x=n)!J(R*54vTud1)OvBgvN0_(4>M}!sj8Q zTFEA#ce)g{qbudsJmfacv2Kl_+F6pY%7-p1#mizdWhnqsuULB=t_{|?1>`c7 zC)-1|+|eo>&wNr%yk8XFK(;PyR23+FTQ~6blxT{-eMjt`_gZWvul^)0;E&!fZ~5*- z5s~XRIx@3FY}IXCX8|3}Lvd6S;WzT8o3QddZeBl1j+=E~YNhQloRCh6IlmFM3p|~1 z&Ks}Y%mAeFL@Ke_j0ESfz zPBdokS_+W*_NpD*EqUCDx+E_%4QK03AJg0`>{aWRR_Ae~I&r7Az zTm;^&Vc2zJyWgiBID?)Vi~kFHS%0DTAg28)@pt9xkDMuC9bcL))Zb_AIXd*uxEN%v ztJaQs>#6=;40S-h2til@2qjWPZ^qkhh&4U+{tb@OpZrH&|FmZ|7#RJFA)+f0`K%1# z=(O!CA=9hL6i3aV`c`81$em$fApMOP#Qe zjj!g8PHg=AV&O@LQo~AzFVNLL&t02MJL*QxGvz3!O%G*?&Wju}lrwDJGM5AA=PqPR zwmPBh^Iy&P(%CBqxf|$zuhNWPsQ71lOyN3yyeMf!8`DsMmT<5mzjl<`Klp2K88~q+bpIL$y z{%df%dC7ydXaVvDpdOB+elWaVL^t!NQOgRym3ei=XQXXIHpyPTkQRvNq)lj$?CaO% zM0YXLinHJx`DN@;#wy!J#?f2rHrDpOUlP5P-z zmFIA@A$Yq>md8@CdS_|R4LFk3AGP5+(td6{KHln~bknYD{UfHU+R%>&?^a#SqHwl`cmz1i%bK7%Ui_#yOzLx((ZEi3gsm;c| z#)bR~#JfkjuED9x%bDVfF)*`U^rauZsoVbkZWf_-$#oOU7y$(&ML8nnkzRl-ie=Ae zB4ID-Ic@TCp<42P%MpB-|H()tHdu|2*B9g?*}6>d_B<+gN&?7)Q-QR4tU%nFk)uXM zo;auy+;2D?KPf|Lm$euL>Y`YKtZd1>4hTiqQb-xUT%G!44xnLi{3PaX?8Yf>zu<=v z&!R&tIwv}`$GXpmqF?s{_%NjEXSn8boBquc+X)px930xOmG$j*J7vezR-0wJ%)l|X zL6Sxa?oE?%Ar4lkCM@+ftDtGBZ-ES)5vje$9f-)O0{jdo9y7m*@A8xW^-|cTQAEXqaGAsDYhnWJ&A5mp?zE`VqfdjK0~`9rWo>)M zcKWPPN_~$KXMai0D&Tz^Q%j^Ayj^I?EmiCUDQXIX2@Z;it)V?co@m0n_Y|q}iiVweShK=&^QA-k zy7o&bW+XU##(wqZpFS-N?eOYMrbmIo8UK;!4=lFtW#9>D|AV5pZ1t^2&7z;BEodAy z`gsOy17$CHI>j3Wr+?<%_?Vzjp+dWrq8wA*TW;m@9mbKHTgnjia&Rp1khiA2gl`cl9TZqxYcR_DuZ^D%< zss-QLCer4JH~ z-7#^^BfEBnH@w68HFQ(bXiC`zP=iP4z(P7`RQ1=i*+jOnmPok%gp}jqUpbBBA>Y%^ zhi?LY+Ct|;GV5XVXiDo_;cPVGj_!GHSz4WEC5foe_6ZU1nd{3q+)N&sRClQ&kAqhU z*IfN^XVcet+h?b(JZ!2?8xR%T5x5y!bIpr0tG>q-b6;?k!+Z&T2%AZaZ5h0pDBZLN z=vD-hLMh09q(88WjowKuS=yXye#a)ZXdn}5h{Q}=Qm_FgzAfqB3*rfYDmG4VP1| zY5ySUK}rMtCd%7ZN9G-aw7^sy0_b7=+xN8_@tMtOW%r)d<&!>MTlJG3-q0|NvTq3s z88t^_L+Q24ADsX>h->|ywf|m%5%`Y;rrCf&az;mR19bw9bYzG@!UD`0W-s_aB)756 zO|SIC#S{kl$po+U3@(Qe?EBdiRJWyfH&=(-DDWcZLseoAoRewHpDnjbx@Y>;^}j-d zi^Dqxok3hX;wEhGL;=y*B z6ghS4lj^yhj;?w~YK(Rbm#^{0U&b+1ylh0g)T4v9Dx&L@&ix*CUr_K9PBU}l94@z! zSa@fEwN*QavHK_TqEeXoY#yCl9j|Wut}seWEhVUWL8`jA+;XwER=}voaH{47rqxg# z7{%Dj3YECAQJ3J<)%I)isa&_ITmLt%s>{C96L9_MsGPrUtzNg)O)j0odUn2hT;?Wk zXE8}CpV`qtXW`e=L z;kDxMUJ|hN^oGdRNV+wu75ui@q(-|`QdIM=6r5i37jolnr9%0rKDdtye!tpd3fIzY z%O9LOCIDo3Q9k%imPiI>@ZopF@d)yJW@5GS+Yih0o_Fc3Kn!o1`#W5nVfKl{ z9;R1@Y`8Faw%KL~Id@{SLt8kP6)}N_M~yuPH;`Sxe`9Ogrrb{URTQ%y)xtC|a~JvA zW%@zhVbkKUo{!4>ePl#SX&{bAR0cs05bJ(gKiUKYiQ7`MqaAZH)d7~=p%ggIxg+NO zr2cP0_CFoLHSd3rIHz^9gZ>#t=tC^LMlgXq;^&u9`VB{~;Dd{2@!Rd_BI}MYAFOvnUkmQ(D9sgji z^h}hpHn=ul$*k={-Iwv7V5{lA=_o9^t23o**KysuO&`76@v(1^Y9h_BEcL77%{W=B zxd${p(>alO6DR~wAZNe<>fmp)q{?F?N@5Q9qkVPuC74ZCo;JH>BaqB+;hd1GT3Sp8 zu=RLf?w81})z8wVIr+KUVQ}m^yIkrCkQS_dN$(ia!vtru(U-U0w%7-BNffW( zY&8aSa~I!K>Q*~f4iN1a5OQ(6|MsM--;bIj=Gz3D#S|I^kq+)5#lx%ZbRVJJ#^?DX z6w@r`C-cuL3~5p8y#w19$NgKlyn=A?f0h6PJoEn{bd3LKRR`K%6<;9C{_3w?;T^fE zq8_o5B<%!qc4lj^DJ+~i@P{(<-O5?T071ZjFHFA z#-(vLHElp4nf=)9dg;3Rji=aYi7JQo#aF$kgQnsrZ$ix^-kIVsSTb0=_#m#Xcuju_ zP_DLt;a^iEqGee;en8!*UQOnU!>L*ZThg3eZFi`yz0odjzjZPS=b3IX)-sq>@@O6 zZu5V%*8lC`q#5_0hN}oIIXsr4T?N>QhsW1bae7Bt@_>-h9yuN!FBH9Ds={}Xt1}<; zLec_v0;F3n-cy@+#nhWPD4*pQOnCr5Y85Petv{xjX^D1Nj;!|`B>Q~l0Ga|{&bjXw zk1$7zREbyw9WS*c0Ib%-*c;$Sa`N!&Wa1hV<+7sI+q1yIRPQbAZ<{>s<;4dT>{BC* zg>k?;nycT@d4XAB2{o+n0@AkfVj1hcb=wJKiJaoxu6idMZJkHaVozL^l+m-OH%&uBOuhJ zjh7G<&T{nGO?3B0rZ->+9BMdm&7Mo}Ce?c0_C9NC-Q8pp;iA%qs~i2@%IJe^X2_Vu z#jTk!xb{`lw-0AY1J+i?@-{w65bxZ+K9^HsU%o>@lxHbicg;BfUJDzwZ-H)YjW=H- z-d$GSU&eGvw7YkKpB5~(+zd`^VW&Qkdqx^1&TacWFmPmjf{|k4c=*B4+0wE7t3$8EMjh=cY+D$x{KiVmm$7YQ$v)N(u!K#wQdxO zAV_-BY{c*;7mKst$EMI9$>b|-4l~Xs(eseX2QO#VLe|V3NI3@rii_AzoL-n%2a@l8 zWxq==+%RxhgVy7ndt*LKo!LB`!Xg*WBpFCrK?(XGnr+7)_RGGfSLE&|k1tNIMei;^ zpqcsBE3<`}@`=5hyM_WQOmdGp+S5$%ZwE7AOXsh0S@<95_!4W)!Nrq1X0HBv7Muh- z`i9Z_$0 z>s$ribe6Y|E%~&gFGn6QSn~D0 zzks}6-bQ4FZKcbnRj=^?y{u)=X1df^S^iGNyWNbrbk04$6Vm<7XA#2dUWV&5AhhrH zcA07%$1eX=CQ`56r@E4w(uu-~|AyJtZ2fej(b{SAE6p|$1Tou;-*DMUm;2@oC=#4#TZ4&5{)6)101dIYdu?-rr zrSc)snfY@kZ}>u46QCUivAw*?W!bj-Dl4pJ7h!p*nPApnc&;06_ohx;w6m!RpN^sP zbK2Q?h|rDwUX~YH^91~}^Qc+d&_L$@hn*(VwzM_Vs{8A08;OXW35e>ffmeRtMsOE+Re>#`p@m* zHf&{>rf+UEylbvy$hQE#;mBWf`RDx5toa~uxfWt3BM$}R5fM&DbztE3E^VkNtW zW_Ee^oZ?s*b)=)TK6P8wEmX20d6oZ->c!xPw0*3Z!nHl7vKVfo7fe2aBWpwiQ-u{D z`#vn1|1j;YK<@i`&9&iOOX2;+Se3yf1MP`8{a=hJV$rF;eMN;fugi#b))x)`rX{(M zy4elLvX_?c6G}B?DCpn_*DhV|zDL?CeA0gEYwPwbIIj9L$+(}hl~6pLujB_& z^yu$yGLO7@fCbp&-R&zZCgEpkVJZ9ZqO+Kt%c>Vkq_L-_0Y1mX3Col+^c}M#9b}hY z^KBJa5!@PzJPlneD!C-gciAV1)T;%h4Oq$f;rY7AD}`(thJFN2XA*I0fl*Z20|`j* zqTZw7QyTGQhOs=jT`a!I4e==YmJn#4zPy1m&egi&RCU-lih+T_q1Le{_&~AECz6(( zP6kL9i}3x5P1ofQEo(}KO@&IUrVq?Zp%bg=OjJKpmgW@p*ONja$?OVK7TycF+q@cm zwuItU-OYDL9mnI_!XYSj8GYXSG^r%@H+~k@$`4Dm&xsmlo1EUO=qw9wUrfKF8}*D; zH7AOP^mjs4LvS^-UygCzEM#Ocnpo$@#>X>Sf5T}FWj7$ zE?jJ6EFjONO2U>U{5H+5tsFVsH3Vq2+Z>4%*vXwc=K8xK>|8ljqL|^5_{PG89-^h=ezCO|;?3gIuvo)aUz^_~YsSgJa~s z_yMBrZv$Zy-r-Dso%XUr6C>aJL*H+t9oA#+x<9^CRkqbZp=Wr^Y8gMs3V_Uh>wv!{ zGU6>DSEBB%-FRRVRX`)-kVtoM*LS?ERL~pT@m1>nO}tNdx|@VQqg~hckCVrkcmi|p zLq!n3r7SiR|1?RV4-RsDlVn+7F(3)fSrV*qSRM!|0Rpa5f+Z>V%{UCYEl!=0$q;@Q zp-hCgzTs@dWD5#DF^jn>*9=cvnAA-V9rEniEr-`pZYn0VK3hvr;ewltxwhl%C#sOy zi!d6Nc+X}>qrXkIZet>BPQNUyLIT7~6}mB<$; zK|(S?swHL~&{{b=pA6I}7k>V1-rC#knSu1y>!Y0tmP&te4e^B6+G#(ZOdOmOeq6&3 zc>c;WX?RJ~j9aXr&)}+Rra1_eIZ~QAO|v*qGV6!kb!R<kiDrkQ*6=k<6l{GGIBxVa6T`z#jVV#c~x00S3 zKGCIP;?uFgs4=XWr$}{i)-@T@VG*_-$QJGP=`>EJ1{;*X%be`8Z|`3DMUN%g^#Bd{ zv(10d!}Jc%xv|081rCps7pln!XLIPhy)Xa-ym%3s4jz~)37?Ou=Li>OZ;zQGnI?|a zKk;q;qSB=#psDMCDno^q#%pE9DrKZw1`w)62p`U&YlW2@092R^^>}ROQl3v(rO;5m zH=~~2%lz>|+yKk1YwGIualixwLBmrQfZe8`W~m_Owl2g) zyiI7w%im$<@B(&j+Y2sgGDv@O1MJ&`LJR0d4 z7oaU>tJjh^{HmSoS?yo?+WoO$J}sIv@F#sf*sr_tipmMz<-R zJv^S9nitn6n}YUNwi_P0B5y{gv|3y4=P9QZihO)t`0aI0n5e6qG5Ki({T)jsW@?Kk z{vP|s`13lTu~H<@8}T6A!h7;{$>Sc|*bDsnewkGAV5@Yi*xA^a(D9X5R=|L@-0`VU zndbtWm%yU7lYY*fhEos{Ufz<>et%fLaQ7ShIFk>tOuPD4kPstxl#|)z*QK`1k|$t$ zx!snc%r;%7t0EQLW#|Yy;nB8(IVxq8IB)ev@IaM>P>|ifb%LG1XUz(X4BYT4A``Dn zJ(v9HDnJ{8(4Rhex|CdYP>|Hh`4+MrAIpwAXqel3obQyw4UVmS_hmifRornx9>)G{ z|2ZXvy9_zM@tHe(T;`>gq%%UB&Eb}xi+e%$2=99fY@&{Pg~!+z#8A)I9_yce4@lwj zN4jCsLj9T3?gK7t*Fqrzk)MvScw)z@UUfUU|)+9km*A%1Oy8$V86I^hndbI*Jbp<$IL56rMAGzDGzcES~us9o(c#SV@s)JyjVL%tB?XITSI(;S2G?>|UONbV_Nr1m7Or0v zcphK99re#YcNi@mt*=`7E3ggo#((g|0oGmExlEH2hhbNj_8kGASIr zjsA-M$HSxN>&zU5#t6>e@?&oyq4}FwfEeA6)thsWt-w=k_T14QtNF$nWgRz{pnJ@n zyJ>lpcZSkg64V&Ke4mcq&GFtSH@h_@#9t~mm5T~q)gO-@w;SIrm@FM9Bb%(t+hBL0 zhoc5{;t%_4kJ9C}(^K$L)%_l;*D8;$fYz+zjLXy3ZX}vmCU&0B>ejY?;<8D5BioJl zh0f9aMawSxarJ18ud${0Y`Sfd`=oGk+qen#=+&%?Sg(W2L+q{`^6P|g08?h2uX=k# zFwvvSowiCy{w)#aCSOt?c zqZN|jK5-6AC*;$r>gseHmWdm-o(H|1p?$EO?#=kP*{@9%-gWb(O|c6Pmp0Su;_;VZ z9fN&L(3L7B5`6As68zSZGV+V$@z(#@-uoYz>x+I&PnuX;MAoG3YL+uJk`?u0Q5$)< zXS>`sxUF(N=9geuu+s8a_%LXBYxt-Agi%8M(q`;6J!2z@*FIawpdY$_ilOyW{HxKwmzoI^CGGzhWBZIzw9xW3cATx&lsoMcY}mk!F#8}H7`0E?T9hut-n z%DQ}w_G)M6){E+n6EC&5IXlV2f$2Bb;jiLmMwRUjxwj0eFSyoSaPQ@YIGz$#I7h=- z4m^-_8Kf6C$dUekTn40nIY%(}*c;j}Onvw6ll z>BnCh=i~~FIiU&kTYXKp8Ha(5@53im*mWE|$!KR8?|n%}YEL1>1f|D&r`@CX_F53u z7aFMO8>mxj(xY}zeU}exhgm<^)?V>Ft7PthvE6StY8>qERF46vFT=3gls!`ReZ&#q zEoOePG2qJj9HZph7Y4I;w+k@Co(3Vg)iuZZe6qGmeZdFsbSiZZn;|A+-*sh8EbSS` z`>_dxuhnCSyQ+9zOw#^knDxg~F0>yr{=xhAd1^Bbh)Sqy??6K6Dk_=)hAYpCs#Ut2 zqC{w;I(P-a8Exdj^wN4Mm$N{y+Xp@BiQf3Q@if=TIi_Cir(?D3;Cs$i_2Q_+Z{1Id z!=7!m7k(+p`v0c#Q?n~{L*aBCtBm9)GiWN338?B*PDT%@_7md97Ufa*5+~P3hBWmO%EtsZrWR-(lZ_<-0R2@vG!|dxoo7 zvzguYt@2UdCAun%N>%}`;3NHA(XFdty3nq z1v|)g-hCYloU@}yQu*RRCxPoGXx)5&O#^6RZ?Bo4q{@hjwRXi&<+haS)OdldN=ri( zaJfnxC~bjHO8JoVeqG`gcC>9~?($KZ?~t1NGex4xM@>3=>YfJ_|C}HVA$60GkQV8W zgqZ;Oz-WmSjA3=`G#p&wIwYsc#Hf8v9&1_-hXXU1TuUXbAJ0D0g3SuA*U*X;-)!(t`D#f^(j&JgPv9HJau*-} zjY5+y@Xtt#qg}pB+(sO^XMm3j$mG>r?v29CsrVJ>Z1ueP^5CcKl|xCm(KfrbIs4`< z);onPwPi}&v^iOe2}Uz^qdv#Y%1YrKE-}ALwHw)VS~Ct@q-*$s+-Sw{`CFns^H0JU z*h)EI=}O$d=Q0FuDPuyuNx$D9`c|7^8s33enR24)o1oy?z3Gpdu7pFfK+l zN-uU63Y9k9nrC{hFQx`N_953$`DlTos;uw{GLhZqN`KhpqL;Gu6S;szci?(<#6|QI zmsqxeS9qd}u)OCflxn~~uaMlOWxfh#rAMPnI6w#_sjTsj>9?DcC4AeV+joL0nH~RV z>V|FA!pybo)s5>il6r8UYx{3rE%qJdm|s5?k~>%nJ+;+?9B=l#5C(Y<0h=Nh6-S*& zVA`KfFH06hIBi51yWA!y=|U-s4SSQM#L~LU0^01$Eh*DrH*@JC{?yJpujBhP;$IOX zy;NNANRx)Gb|2v(Y=@NUoWKZ`FQ`L`=XwB))C+3jcg{>;-f(abZuiaIM^vN39O?8V z$uGKu600&bku3BATv=?Zr>?kn)H1qB()CWlJAy#w=*U^w?aDihD0x~2rf{59Vpj~J z{2-v=rLTENSceUwj$%IaA3Q}xCT|H{{B#*zL@+wP+&sq+Am|hz@yYs=2RQ5 zZu&k>!C2~GORGf}`FAhO|Gl5*zewhzU03x#D+waUc4>P#<$^SfnZ@}fR9<0}lEoP% zNXC*5YUJA{Z<(Uvs*AReNP#+;h=o3bp;Mk)sjbOGbiKr@(|zI%J7PZK8J+bLUbiaKTON-!an!^8 zRdB^>+a9C^(>P3!TtT_9K`)|zf@%1d0kBD<4{$C*7N$L*{ehAqMk zB#VyLQyr}&Ij~>wB!6WxOfSAwy%7^kFKfSf0}`~m)DNj(Ui9a|y!2%C89IA+s+IIy zkWe<|Ql=;vXY9_yD>2*>U9?3A;c^>c2x#xmf~Bu+ZmwbugN|dxPc*WF>w$E*RB9vH zTsNPsiNnXckyu?T#hR}IwvYH9!mkgz@(5f>>Z3<+=XZJSg>9?sk-=JL;&uumCf%Ey zrq|nb$mPw{&lWyg_Bk&-+&&3b8e^`G(Xs^LsLcjS=T_x+nng}zD^2O z%vMgiI&W6~cJ`YAn%!+#X&O{%8}!nh+iVxg_Qlz@xAhfGV5!3jJ7*T*v0TX6_>_Qc3p&XA>k)~h?7x%Ms7zK#dlgxI!^$}gI zq0ZCh`gBm|*7J9@5gpZhsh@T6teJaIaIH2CVyrr@kE?dxI(_TQ->k0cR$5j--yv-c zK5bt3ZW$fh{CFdFNuddrWWv(mc~v7+sZ+Hl@y>r~Jxb(>=Dh&pp7npMmy zpep6o_4BL^vM#05LX{O!8?f32PdGl0sAQ5|QJ1TP(DFf{8aZb6$O5YhmeEV!KP3YA z#6!ovZoJ4o){#OwH0B(8SIQ22t{2PevVnf=stXdy{YEX#;RbDOn=7EF#Kff zia>Ia8q)9y4?T$sAbb0;f zeauX7x33_-0t{ zD&w>0na@234g{w^ZUG|Q=H~bqFH~s-n9^4IMTCOLEOI8ydI|Y6`Y31^ zB(vQUQ}XVOGg@b)vpv5W6EF7mR`U#W4cR+XZ+T8N%0ZAdc|Iq+%+qj=S4Ca)}F}%TAFdh<84u z9v{YwmcW|=Pw{H1*q&{0vEfxUb=20O(lej-`$x`QfIvnr^vMX$aU);3tAuy-u zJl@s|WQ^bnznEVK!U_3Bdk^08*wo_&C@*eo6{GZ#xejTX8onwdug=Y7dCCX{_@wPo(s0F+ zPjKIG&Olvql1`@&BD)1yGs*6Fah33IoCB_rf|a4j)X!<3aj^&|_~`mD z2Bp3EY~$0!;`yq_Y)?<)UXqhEhqI{(r_h^K)XRphvg7ZwxSUy=d7IS-7}JbY ze8@`^sc$r@0b2W7H?+YR`8Hb#)kO(K)Qmc3B}Ms7(KPNxhtsk11`=!TE}&lng69r1 zFB~f|RgLi^W=w{NK4qO~4M{)vY?jhEikjYbGmZY-G;#cN+RxDvq|ovf;h2ea6joo; z6vW2j&RAZLxmC+|SUH%{{^vpHVDcZY#s9LE`H}kk=6vkeAGZWN1|Tv9!egEZnl;QM zzcz7C245AJP`XKpq)i`f7SD!XtA_jH-E}#i42mA_U9}!IAA!-poo6|Oa9vFnXbT}X ze=`U;=_wQ`EZrPJI;Y&9Gte6mOcYs+(o=Eor>W_KtnY7X$BW@N?Cuxh^x@2nxN>T+YJJ1(M+e9+HD>45El+vrcU9nP^B`PYQC`*{l~pf2qfeE6^xkov-Q_Cl zcthjSSq@)GDr>>iP5VH8`|+;hgx4mc*q6E8-#Z z*D=0s=E`a!E@miu)mt%hT1Rlx1k2$~_mJy%MT^Qus7K8yKC*lAtW;pPL3DIjj)rdb zEMcC9hl{pIbWjg5G6*@TB_k_9$EVO;X-`XN)EkZ@4h+o0d*}1(J2K8{HHoTPI`5ht zfQan~>}a3`O;H7eJ}Dmx!3?0D@NhCNvbHs-oC%q4`c6hg!xb6RZ}gZhI<5#@DKZ{s z)pty@UF+2K6IN?(Ny4m|DRHCtIec8EE2abiH%MKCf9t>cyQe(Eo{EnN9mY7dxL1NF zK=n89UYl`XMi->7QnbsNbv~s7r1$nJa=z6ak?h6x%|eX7tbU&}vkANorg)u|Zpdm3 z!|i)g_ZV>|v$~s%3$DYw8^EnJ6e|0t=9t9BITraJeP>Idj#LI>nUevBRcCtCQ_;?QXBepao-|?@FLCb$y zbh(K@ns?kT1ikrzZ^NmxK_^ws{R;_pc9%&Of7)hQOAuc~GSfv{a9!D3=PrI5sDsrF zKlOGtN7sGYp+2k_G_1`Z>VtZe%Ke-E_3w86f4PZ=*vC(zqe8PrXvJvLaw7~cX=$pz z&ODvAhsg}(Hl81fevAGdVXD31DwZaa@uDl3yJVU9ahKvcyTc`%ly16caUrP+H8dLzbaSF6;2L z0=c8DzV4im4rx0A4V5nzbnK(boK&p1!WU^e7D+N#k@7#yH^{X;SY+9UtCHDBYf~KK zMFFs_{5#kp9Xur@4{aC*oyeKt^4P+uu36#M;NIo(X%E%sbqP=F>on8bt=fN-QaQnL z>_CO8;824_y79@XH_@RXyq~p2jQWo9r7YO&si2?nX)iZVj$RT$!&~gws&^<_qv&PJ zw+cC#Um*F$V1F2?65YmBs6A|n5|#RPo}H&Qh1Uui8WO6B)dH7;adHORPGltkYA zZ8)@PJu&XY0?1d3b+;-#EHmR()-`rsXMBLM@mKRT?+UNUWxi{T$~5Qy<6=iD+Sdb@ zh6#z?{0wx&0qqFdFh>X37c8fH;6ngO{~uv*8C2KSL;(_lOK`W~?(PJ4LI@W8;_d`@ zcY-?!1W0gqch}(V9xm?8A@6%LQ!|q{Rp$p)p~&9*bgy2$x_6(im>r2eys~JzVS98_ z{v>bQTtV7|nYmYIR|q&$+zoD>)G&vV>3oH@PNQx#_t>P8tL<_rFQ)9tuj9W_i!L6>D(aAc%Ynv5gq*%7;6VfCXt zw%^R)B~7zzFsPXe{4;IngRw>4Rt)~<%-TwSpvPWM^ zJP$|F{-tu<1MAyZ2Icv`ihwwnVu}cjR#QGSwX+v_@h410U8MQM=B>p53Y9v@Q(#jv zDsT9ZldSN-;p>#Od; zb`*eO=5(TG$D_C#$v#HP$UQX&42|H%$#jJAdjlIS0vo^UwT)paqxD_GxM^?ZObvw1 zz;!qhAw@PNyV{gUWy*b#h-h@y{5vwE$c8l~fv)3la64&dmhCJv#*F5em+->pb1^No zb<5@1=5+jpukbn?PlVDg0~A6kuQ|^nY_oj)P}$ZD|EJTfsxgfBZD>;_W3WY$!;f0^ zg%f!BLQ?Jf@|pLE24+KDKU01Qpv$4useOpY`to3-f#IpqvA%SZ&v?NyhHC_ahZ$}_hQ{`- z=YzCmu;4U|t$7e0id5y-e3XCsbpJo6kNi<9Zmj<=h3g9$Fi2%8`jJ7WHQW~axgAH( zrw<3QZKrC#Dix2_@8b19!WOoFpV+}uOR?4kC$sY*X!Oy!U&XIAKyt!#3nbdLVUCli z5Ylh7qO(J`QYk-OJc@Sj=B@bVUI2iHx|C4Gl3dxSw*dQvAtL7nYv#j-&0Tc&w>-R+ z!?LkY-zKn(@LEu`!)Mi4+#_3%GDgRb>u(Znm|M3z{J~nZFZ{Uwdt00pzBD%rZMOkkfh!0_i3t-zNa6 ze&;*Rak&Tffq^vE)n$uYfZLt5rH|q3*dArRa5hOz%B~dS*Xw>`0bUa#qOQ-UPjty> zF<~i+WZv-VXt6TY>;~lN;_L(h>{-h8OuA{=J;5I$k_}xZ-8}unsAfDS-z4pLz~g0$ zE}nw4P_b3y)A!`v<;Pf_g!wHf1^Y_#cW+b$1a06AR;f?jw27IKpZ#^oy|=&>i-71I z=Cl7k42fz(R1^FB7ccC{WX54-GlR_8xbed~K5?UQR(FzYQ#biM&V8kd=SLD9$qbsf z9SYH9e+HD_kMG}=p;aJxdc{h@oIihi{{IW|`@(V90||YO^|-^*rF1t(5kVIuH&j5s zngw68P1xkRVF%-o_$M2N&+sG{9)7=u)a{+C4mH~x7{4Bi-g@^?EmWg)&?yugt1{uPv^pooIG48&df22ie&%B0Yw|wjHoavbU`wi3{*bD3WPaYG9J44SU8V{wye#W zv*&q!l+(hW7g0)PO+$kVGZIF|SZIhNIt1{Iwa)71KR9~)x7(Urp!anu$UjW&`zS%@>dMcjtb`R~UDM6#jaRcUlv31v~K zY9*;N+sPxURCxvq;xA)kC&JK$&U{b&ER4}$hGnIm0~xw0O~|QeF;JKXdt+{O1%$j1 zb#GU~baWNm7%|qrgq9k`zmGZhvETIL)dEJ8G$g)3%E*kW&&y4zSdD7nUv|~Mv+wRs z59qX#q)dLxRRIyhFx;umRd#k((^V7_m%t_Qo?3UNyC@xQKT_dtW#`xF-bSwU+@ac6 z-s}`AP|c;Q%hMaHLGVVZxgLF)-ZQrsMyv2aN4;?$%iE_={4tl1VACC9(Y8SC?N?SJ zov$ADW=q*>DZ`P5hvK5dNeH_ViXo5)xDhzKDXaU7*TEFc%XaRlT%ed#5u}M zQ-K-_0|}Ajr8}^}J68&CvQq0Le)$~pQ=eqJ!3CgS(M_yJ+A`dXr4z6i_3AEhQzK8n zn{-p`g7i0RimxnOy^Z0Y9%raH(8lyXwYm%Kcv#x0LqqmD2ZPG#+Cg!WGQOa4N#0{Z z4)_Fe5mt>Bw15HYFH2|jaT(P)k*4M746<2a&=EU7;Gb^d$4*E2tJY3~&(u|IklQJ< zI>c4BAc&)gL!;7bH%omk{g1_8HR}Gt3z<4Y#lwLfu}94fej~XdCEQ+&De1820x3d; z&5+{PbT9l>T;=Xq8%-CN5ArwD^W5xLQkW*<0TRy=fq++BM)_7fBw96+7*izra6h*4 zZ64Lo-UYdb;Z$i;!W=sNDk(P^hypROq1bjw95<3Pjv<=q9%;15x?Lk6T81SVFitei z|JEN{4A`P9+qk;lm}Zt<(6J||wGjy3a*JRw?zGdmvX;0{Qb3)lhL>0Pl=I^Up7-Jd z;X}D0iPbCaA029L5M>f>zuEkiWm}iN$-bAJ7Y*+qwAu1`;C*s*)2ffQ>dh%)YCnJ< zTL)iRpFS>@0owI)vWrxQB0Pq<-)vI2-*H*jj=CvKXosf(*)MHr=AQBJ^V2&)LDuc; z7TF&Cjw}3`eYGDyw-H%_prn2O`utSz1+j#8AjJh&xaN;2eh@YtV@-a`ITW5CPajP< z(0#@WPczp>a@MuqVyU(nrh6?y&jcl4V?-q3|1cF+Fc(ecjH z=6b_VlOKE&>FjE;FgvGOY%us)5h=})kl8b(1WL-*$Fyg2ihDgyQ|okVNl@0e+9{df zwTZF*+AyYP@_Gks)3NJC<}X*XUYjN1LzcU-QSp!ds%}09>~*-WakMH1cY0zq4_S@w z@CAgdYY?P(jtS^bVpe~2t8LUArq-#iw@tP7ZRprwJ=vZX8{P;l2N*Z?oNMrX+rcJt z&r%JGt;MmN%Q);ISmm={9VW+3xS^kMclMiSS83~1neuK&H#aqghZ*DhDG!aD7-Pfcj_;#=}jiM-jg5BwS(M{-VNi=u2?! z(!hF?`B4Wcsn}x}9mO(k;7xb8>1L2Ip2qI@II|G4Jv<(+fF;UFb_+qpKnS?I58H}v z=JfXu%Tu4J8H4^vB3uq?tjEn5!}p!Uo+-y25%eXf(T^B0n{kw>gYmQEQ?;P+;#`aP z*o<5?6EY@Rq5zy-1G$RG=eM9urMGnR(Rd<;aCP{4 z5~UFJ42yF&#vAP|1Iy|bCG;e+kl> zsanJCLNI0k9!`YCQPd$)kNz*;-fvOi5Iy=)7LB|b=!LPskC&22RE{*d7249WpM0%f z8IVON8Eup#Q+0m$@J-gNl&qQ4n2qarp}?k6k?Ka-fJ_2u2kP#U*gvLCC&J?%MciVL zb;)sdxu33_l&+M!`kNlh-L0SW-6;X(&g8Z@S-fo57dI7fnYSOPT(p_Ri6V)Ky<1;G zXiT45|0HDfrrQYHO7j}NQOAdxq34siw1jybC_U&Mlfa39Q{x%72AhDdiR4ZkQGNaL zNPquXrzKVY6inXGb;76x-t!YD;fod4oAwkVI)MWi z!G_l;*=x0je4R(9-wGpB%YhM(8iF_X>v%-%V= z$33@=ood_z<0RWqRw*O<$bIgJo~G`!i?LZXJkd`YE~rxKh+j1~oPLUVTk6GK?BN`Y z9uO6Q)aOG+f&`4NtLs7!XGbYp^m$^NvVTIhcYC?d(`(%b&1>|0 zH*jYc$~(uyRsEsQO9mrao2hleJpqm`24@z@XGWC7`4oX`{yH{D38#VZynD%ZAKjAC zZI!-I6K7EruVs`ufN!)GS?tvCVTNV8b~T9;nXE%&O{>(q$RmzI70cxdkH?|O4CYb?m)V&mq5|Gb$46gW6myh7@OPnW zvI5CrpIP8ddCQ=#8EZj0aX3n&g8ub^o`t6=GT4;e3wz^li3cSamm?Szkx5KFF!|BE z1o^BHVa9j@U&%T-!(pg5vG^1apoHOCsNsWpsy=NPQ2s>k4Ji06ru_k-x73-Ib*_v3 z4XFKyqr2Di$2h=Kl)2Dc7#EIv{j-<%aI#AUzKq=VM3FS(Z|o5D_&k{g&0ShemJQO+ zKUWIwar2A8?`F9PBNDXW2QyUP1)5+lwM_??G3pAiJmUOX&AH+5Oj_?i9CAdB&j{#{ zy95h!GDEY18}`z*0}C5WC@B^-zOC!vJvVu%X7}DCAGvJCnQNW47mMNY+|$5N5UFh@ zpC+TUltgrZ(1EMgieg*Xi76rT_sfIvi{pu0NEwN5z@0P9tQc%c=@FbuIBl>)O0K78 zD2f;9aSOXH#`}Bpx$=Gyu%;}X^hd=Y4#=Db0Db71c&^qPJcNpJ>XZz;c&^T!8ievA zKfmA(Z_W_0N8nnCtcHLGt}X49o?mG+*$|aJA0pp8-2V!VJS-B|%grVT+)6CcpFsLN z3Yvm9$~)@5#nB6^S&S(RN=kEpz^O40QCPgX!OkK|1BTXJhPOXlx1GA7I<{{QODzuZ z8GtLx5Yb!3i(mwXX)w!S{yd{wF7_lizl!=h?3pYB-U%mCZ`mas=XuL%A{ z?I`B_UNOg=wRRMe^=K|4yS)JB^#Nhu31z|4kE^dvH8{gge{_1v+I|YdQ@L9TNdDad zt6LN}$Z&tAzk3if(P`A_e=QuOq09Tb^3qKal0xLK&sX?=(Lt@B-c~_P8P z#Te0(VIX`A9pIt#AABb-N=$}P9I9BEs|R8N?HJKBt9r%Djq_JP#f7OY@7y4XQ)9oS zn+Wa~NBo9y_SU+4WK`rR!%BQ?K*d^|axOt%K2U$sW;G9#jXF#XC+CtgwuHV^*X>7= ziBGJTSLg5Q9gSfmX6U-=YptFqvibumx)DyTb zA*6IJP##G$Ay-r-{1{XfSi#Y$K3D+S?=F@+FO=j}$-23F2((1s1P`x7tQ%5KvZvTr6!xyv zkG@AzLpEbrC)4lG|4b@>GGT@@5Mg*BWhxWoJU)8yIls&aoe6j3!%5InDjb^V_;Q zQG`BlGih_+5@Ij-JC;Sej zgBRk{8?;;NKQK+0U#_<2`Dk6vFEE%C-jW!m;7ywHPGeAAqwjT|=Pl!%wllJJttXa= z!%7uO&NXGp>C@5txOV$zc*oPSTAYn+Yvy*M;|2ZgVYfk#^g0%zE<>9x9qYxlQ7)fe zqU7p#-Fge4OLe_>j@X7)Tg#!1TLKTc)|MCb)kHQoa#F>Q2U6LZOmlCFC7RR}PRE{N z=um9$jtJBomHrAO0LE|v$s=x$ez0_x#=H2{lu|OYBSD*v(cJkt&j*$y?<*lm!ADFN z?m{1W2n4n~wP0AvaZu)kD-L!%I$jxuPN!E8 ziFu|g>jf=mxZwtv5Ow*^hfD}R(cDoQf!Z@ZE(sfQ6Q?vtZTF0zv8uPCt0Skm;i3!+ z_`X9gWhcc@4#VDFFS2zYC5vZ}73a3QLPN@GfekP{>Rr>6`lvC6i?&Vq>DS!-$be`y z9AuC=2>x6A<#^;|TjX=ACT*w;!+X(&6`74&M!Sc_)6>WjT(ZRr>sBy{$ncSXIJ^%h zET2azy_e!e!xuwCFY(b|L9T4Gm0^rzbdgzFzUCgwH#}Vd9qcHy6KQsEmRDNifX~+y z-u>hA&L#}=^)EhOzmuLz=@$;i;lP?gWUhJn4v7jxzx_ljtwVuMF@GV@mM5iDWHvZd z;IY)0BIAgqzrik6Zqub*g2Y z7y8Rs3xTbGirV3|)<$q(#IzohS*f;JwD#jPq8SrG^%ntGM8k$U@L#_xHX}k+~lC**h;HvR((2K?*egGD0mD`!B!n zhQtg|MluQIFjDQWTQ$s=@8V9s0u2;cR&1R+YuxpGVtCxrYHmAojC6O$bt%3>FU_Sb zKYoNRHI%9_{DtFMXlpCN-pW6_=w|W&1ldgH3EL~>>@_)@k6}0TbPfHxUu6VD_qFn5 znZ-YfW}Q9!QWCHo;q{w?0z*&{H{@k7&7hOUknW&O#K5&C?9q!r7=>1eQ|#ct;B}Vx z-l6waMD{Zjdg-<1$&kd!JUvCa_v+rUMMphFdFeMssy4^X zq~x2ephM-$lM$w!#!MU46Zy<9|Lp}}1AAh#^UGORR8@^LuS0#?-J-Kkb=8iKO{##W zzPS2gHHetDs`aV%WPTWj>cgO{9&0C`5Xr=bGvh>f3Pn*GqPf&rLW@jL=m*_qgoLOG z@a@myNSUa2Vawr(M65D^thMJNeUCn=0_AH_ieY7;n`@+q3{G<0k^OQ3*-aTP2>s50 zJ<@J+%1wgmy+M zA*CpGx*yVE8^KZdN@My@KBTHbY^Cg&0jhX-h+J&IE99(%-?fv;frO0GLQ1k$&+3Ra z#rbOE0aK)Z<^LsMTeidW9rRFUeWED~cTQ@BTsmdzgJNLQvqbDk`vA=&7ZjPR@Yji z1Y|DjwB6_>GZ&T!z6tB4>r&@%>9Fno9zb-}aW|7LBRK81Uvu4_{b196_^Mg5F|q_z zZdWm6z^JocMg=oqobW*q9Qs#*2}mD*%Kob{731ByxStWQqh6oJhvGj#xuZOApM-f~ zUW*F`W_iJYhC{)wIktL{u1Cs9UPow=9#`f_`-`xrP(IX5tA-(*mSG0AEg>N$Hht|k zACV%3PZp!UJ|AHsr8r_fnh>da@Fv-Xz6R(``@9sZnIav`u%g8C{103n>JfFNh4Cp0 z!D%YvsEc+(a)m2uIg~^_u9BmM3GX&dw%U*VD>R0Ac*c>Qn*twO<4lj}Er)1STx_RV zCoRvH%!cOVNc{RFvui4nU63XHyrk3ObQ8Tr6FC>+1vcl*imcpteXJJnQ{)wBV2Tce zQ+yr%WGfIf0u+7_!0Ux>Un7O*06|9SBDn}O;UbG6shz7HDf`o-y$h2n5gks?hEu`7 z_>>~gQ?N~9H+!Xupk_zGuGWiEcos12dgXXAJbyv5OsY^i^vw;4)r4qvqv)aiL5o%qjv#U|;yh(~fHtsN#Q&4LB4dozS<#zww4NXdUjI zcLb$Xou*7Rdb}>-L!pj_$BZtIjgbF(P;dS2r2l8MlER&J+1veqxhXQZd;;x6gVpGgW!KKHuK zd!>4sUx%^s!5U_SXDg4+-7nBGl7O1SLzJJ4ZjFzB?lsSoIN#F^prqmC`s8{_?7n&! zjGkT7KYD#G*QZze=zGh>gL_F|rL>@O3|oEuuqQJ}`&xgqD&0btCGwBJ2GzF_75pNw znf&B^k6xGn{D>LQp~HlkONwxgSot88SW0d31p+?y?+?d^e?yS}D&$$<*CUnS%H0mm z!R!H)OwM27gacHsx#$%jK@QAzYUcqJ(re_B{6YuI-yZc!Xi7qY?Is#W<2U_Z&2e!yiQXK zZ1uVa-BwEI!~v6iUzI-oCcEkt78>X{<~ERbD0Xfm*1n2YX6hfy+Ov17D3Cw zjYRZ=k`@7|o|;m(ep(%zie6@@VF=1w`xA7lg^D&OXq+9BwUOk$E;VhqI8h=22TYaK zR6nw^I#~lKoi}1xMj-{KP>n7};SUT&7J7l|As^UTx4_SL8%ZGuxaOe)Nbz}Ug&cx4 z4y?(-(Jd#-L**l8{C3RVwGX#XE{LznpuIe`h-R$f8Z61Xyy$O`*yUuOBM zNJ!%xsKSl7!Char;+?NNBbX>$V^A^P93!hLO^VL_0EWGYYkNbp4MUfQL{c#)g$Xu} z_$3jvz~n09Z8PY{SDVJy{&3gN{cHHepS8 zyM&xxmB=5q&He%Vnl!vWSYXF{^qEA4dSJuK)_zqOmK$730;w%a5T{h;B|q<&g!F@9F5Bu68N!o8zead)G6a@Cas;_t-9k}xUzYGlv(ilkgTyh69P3VTYY zcLWvVXiuYc8;#_Ch|AYwH-&MI;lT3z5tf7R-1KPZ@Q#QAqZoTL>!0M)ANW26r;bk> zbw)4)Go=)wU{hf~Z$6wqL8xFXD=w3#5=7v%1~_)g-_50E29)pMJaQ;>{o_JdeXP@CKcxAlk{@yJ&#HA#j#p{*<)p&iXLhpT9`Yg&%JFKKiu9 zfjLgh&ctrIP-Z*%9rEp|gp>HGwG*qXf%;HhxCLSk+x8Z-v%#>RKh zMv@jm>%etJqp;vH^__1r>FTHZF-(|?oD5n!f%{*u_RyM;d;FG9Q~pln{($5k6SMEd zA>pTZg>J_24}3?N4Dya!paT$^)~=BO^#^tjqHyKaa+qM!`EAi^t?wJ0WQc_?=&6A3 zL}|s{ghAGpJ)>brqV#n{5>hrPLsm>qZlh=`n$IGRsQPm2Juq{XKD{F1bL#G`AsA=A z^7gE`Grfo;f|8P&%tFz`vwAS5ahlu0oK_4;#Saj^Sf@K=Kd%Kf zDjvkL=imoT(Bh)JY~WfoxTG$rciDo&B^=3=27_l)gQBhZ)>=JaA#@42N*>=Rd~)ud z@MH!uz@_O5#xsX1RlPSM!Swtty}i~15b#L*ic-%)b)b&>?9QKI9~G0uXmiD$;_w_n zTjw!``oqF<+n$5e>8>#B>GnE@h-mY|YGCau6Y5Wyj;xa-*!79fToAVq!#rOcAa z^G_;;(v-o?*0e%+-AVj#FPgRK`|acNij6^&<|1X{`{VZ;=PrwO!;ap|q6yM(%#}-@ z_HWJ{*%K)$^2EEah|0PaG8=QOFV4kIZ!71&3mAJ=23@_O`q9(R(~MD~^@z=gG@+~E z7|6+wbFSuz_9c{NtT_nZr1^ne-g>1sdXu>M-VdXNOjv4KtjEmf+#;jg9P0wFQ6~RC zD8)3>zcIU}sd5hgg?ill17r{KuHuk>K+HzW>o7z&YkrB|F`)9TpgCe-09M>wEwZq| zMIjn4yE{kg%_(^^_~4KyHmgKj<*Nkmlf-X_!>`z!+^5n@`o)WtM4y)yF=jBZosP%x z`aU>LP*Y)3Mq4px@L(AzDD2JK#v&+d6sA?sv1*u#A;IV%xeR>gCZ+B|RLX)~X8!HV z;pkThn39EgakXVnSCU&pD zGwP0axop_aEH5x#v38O+e%83~H&JEzuS#Ve6;`RlR3Elf`Cyq>s~EkaRsVWjSVHalB9FE_bY zq`aTZkh0SoBVc`gWL8a)raU+!(Atp}Wm;Hw*joQN#BXpP>}7oRO39nFlerCj-=qrN z*_countt&C8gE9MXRRl{W;My3+;Va{kjkeXVV_sm6=4akt@-sNBEqyQ`jukGmx;?f7 z%|F6T<>g7X59MaWw?Qm%XbZ2(;Ep3TCwAMj(FLqD(^trlr}P$3R1R-baDTTiKPGqX zOrl=DM=Nc7^1z&{p$l9=vPm}mkD5abP#pTx&>(qSLRTyPBAs;ngHavYu9A>2fNde6 zFU5;&QvK765|$w&;M%~-ZSEaw%HlI8sG|{|ZP}ARYx}KzU2UaYaR+aBe|lR6=#}yr zgFW4^Tg^3XGs`2E?+~neM&!Mf*jp?UmX&pIQc^!%3qei?QUJ)Msb_CjWw-)*luUk? z5MD7O5w5)N50MefEt)t1+f8T_PqIcD3_w%C8<>lO=%8mUe{JM?*>3%PGXF`45?n!Y z!wT1)XO_?J*uELw->rPLX~RA0QkDI9d2zZ6UOBrpUg>C5XEbvWziNf0Pi|39D|3wX zE*?yuzjIzCFV;kSsCqV(oy(u1XFTBt;`YfGjJ>7x-;KNM8nLOBuS*tJe2cA^YoBiz zZJrOY>=H0(SrabavVRW|aC+xR+`X1H3yj`o)}Od&u57s(>p>vUj@H(mYNf9&S=CGX zesms=TMV^+Tqe=jlYiI0@ELcux)o}Fh9RX!RC{@y?P$4LGD=w>Kxe#^pDhtrUQGO6 zQJVemQt=_pUlFJ7T_Z11kDdOOqY@ojYG9bvN;7ZkavUU@i?z-prOmUJM?wPTkMY<> zA3iTnHnr_XB!JR> ziOe?+VOtSAmOO7@(E^7y(P!XdUf$$)HR}`AM9$smTp2_>{Z{Ee>e&|Qf6^a3vZw`J zEvuN9K#26GlZhn&?wSHs9$--V!~*1gy?~%m?oJAWp&8Y0(=GUg;QY%jz@V}dPo;5| zLItEewHgvk^ST@4zym)?RHMfIGuF0GT*77aWh;%vITA)9GyVZ0>12a@t}TvZaq`yyN1!Mk_|D)YUhy z!=$-t>mvLia6+!xV*%HV`K}P0fPV1Lm}N~Rw3JH47M zERUU3S}Bv7xZRM8H5lk_Eq-mbxC42#auJV!aobs5Furw76vw#EhsdGIER8gC2Qo@! zuMmDzUpkLz>#|I~W&EBeZv9D_uTl4ozgW9fG!r&PssGcjj0hTeaPtEdA>%0>wots- zk&$4sx^1EoeBr?>MZ)WiFK^^dk0N?#YAsxCQ!(=8^4HFhnMi*7o(TZ#=5I<0Pc9I7 za&d=I%`+x~`bGgW!N+>F$tT9Dk(=i7rWXGtF2OKhQ~JTYn2b9a%z1qTc!xfo)q6`i zl9(5!ODekSv;w2TZI_7>Qj6s?c5ftpFp#;uq;juBE8@#^lU-rXA=5B$(fQ$|CO=H< zgv(9TEdIK=&#cQG;4J}njlTA+Lw$5#jm}4B8lOg@Qju$JdN8a{bnOqK9Dp}7VEgf~ zP9CxmZuuMyOtUoUa`xdP$uDOKtho{F{G=Tiwdm4?NRV+t2(;2-nc>ZR2*Q7`KHs^q zzkf$*jlpcduU__H9lk5AR(xt_7u2a&-#46rn1i_C zaNosIP2O3R@h4m}tS@h%SjVOx`u1&fD-e~wtDXP0!+S4+P-^Z7U&%Oc*AZYKmM!TB z(Em<(T%iMKZTm~cel*q;)vhR_QVB^v3HX9a6r!FQYbHtWCuK9cwfWtUgYyOP9d0DX zr*UjvH!ezuVcu#95;5Q1?oH1t04Rq<6)-p&dUfH6&(fWoFL#aE zEVSfqPoBAI^DWiP$1qo$^eQsOK1qa$aZIEMYRo#9rR|@@IZ4%jrQVhX8 z8ue)l-M75bAI$$KzmydKuZ7&9icMveIKoqwec#U7QhCdhUv>-qS&$EDb_qXX6s@u+ zCt?9K!Py++-mZ1akkO)zz%fefaU~BK1zfM=j54p~ok>G|4=N8`IbaWb(iDA`pI~gp z8Fqoa7_)Slv#OjQ%t%(?K6|`Ro9pHSIE85z@Y0Ya0U5Vv(;<9Z3i6GK!C_j=IPC-1 zR=rchSg*6N0ZNn&4>O5ymOZ?w8xM>2&C)n^O*x+9> zjz3744!f2cy|~#b&+Ym)yI8!OHCZ*VsceQmYKfr+;`!Gma})LPlfigc;g(*7bu?at ziWQu&?oA6~FOz*cojhbeIC9Ss$sJWhMke=zk?@^OgW=3195#Q;vS0Ig{5VNyZ@LRL zsqd?!IANpYMezK|M>3X(Uob@@Z<<=hsSz;HQV3zQ?uExROJ+W-&zZBNDgbOh1LlwL z15-D2T14SrF3ON1EEjZd39|4@+I@ik;ig83RrDipz>={OSzRY0@&OX-Nf}3dK#zZk ziN5sMmEEDrLeD|<@;U;;CIpX&84$$ztw3K`!rTkRRWuwqZkK~J>a2b6Y&BGE1?%%7 zr=XR>pw)F2Brj*DF@wv?^Ucs_Fx`D{^p~$%3TB(sekg;K&^VHzCtBP^ttR?bp>nB} zR9gou_S{Qvw|4|dTiwi0*vWXHafOB&#@_;!cBn~NE_hP-R*>`qf zRL)U0EeH;d6A2(|w^Rs7Q+%_-ftqr0$G=CoQj~#v)EFv6Q_iP<3Ev@(L`B9t@1#68 z(5>G{T`{H4mJUQ79^^j}21&617`Bc@)wj`BB;Esw`6D)OZ>8A+_#60d4Q#1E4m%1-Ci%*L5@_8~E*<^O9Pc+6v ziYxro`QcWMIVOCp2rpn5oVctSBj072)N`ig7UCS0?QZorZI@RiZ*!-nEN;Dje)*7I zn^DX-Agr1Cb548F_~55bji6$!)ZZ(kc6@LjrEai(GW6*Cfmj5q<3rmU#l?S=Sc)_E zzg}s)`zDfGI*N2m<`?{8;K7{PgbK$Pe*oasiiF`fr9l2iB$GOvM1-bpat(dB{`c8q z;!q|v;i&Q`q?-5GRe;3lH}#^axii3=4~1>fXf&&4_H580W6;**F~*#I`re>Do?&CA zh_ZJ1y$f;VIB*A`*X9M2*3R-qnt!DhFO zP0^+;WE!w*y9J=N%4GqqD!^T_9W@})9S1o^4OHGHdEqyI;;$Ok7#SPe*e-W{hxOpb z$P!2aw9R@dk3alr#&@TLIfvEc*wSl+lAok3%nrPt+SYFEk5v6M-2r@RWsR##D+O7} zhHFWadToH&fA$2T8r@70QQ($(ZFC@jx(rpfXcN;=T5Uiao9yo#3D`*CCe%;5yPt)G zFR!2+SI?EY4Yyd!)z%%@#_v~uwsH5ejXS)mkUIsDu#QfTT+0{@r(;sx4)OQI5jM*n zomQXK$Y2nWFw6-TKZZ&JvgLJJ&Sg|Q0KS*fNG1bg9da1G`op0(tR}-egt9|w=)g>; z)blA;^E1~-e0CVaKE>{dX{2rXV!Ef!3`Mjr14Gaw)&bPQ$THVG`VQBJtXKKztQSfA z`iu3@pX`YVg>Bn!PSaVpd(&C=^V8oqy9jcGNKf7T_eRZzq3F=vR5>$W-j>F0C@#?J zyso^5P8?*jj)XCvvZ_bO4+L;UATuq6LNCnl4ee;W+{^9OU$z0mcanjocj3?iWXz=D zZ)qTzUyxJ>$9sRBJU}pbpOgZRs?uR&n9{HD^rL(S*M#BIKux~+E+@$vFh|LL8hE%$?i@dNVWE1iB3Lmy1K)1q`lJb+VFYE zw!@M!P)!?Yq~skvQy{;!Fje4Z32<=9sRn{_<*i4v=RL9=kxWQFKV?xd!A{)YBM+d? zaOgv-VtJ{4j0TmTfyzhes^bGchv(xKziRu;!vUD{&zI85L9s@qJvu6@M)FC^xQu{* zm}z;anNt&`qE~eJ9MQY1)&$gI7M71jIk>~wHZ`(vGCCYHJU<+!Dc(Q7lmeK$_5HHy z_-k4T{_O?Wnv33q_31qgDhI=2Jw3eFwqFs`y5NC4rh)pH(jU5lijtezbVy3p6xg>`hSp+G%j@}#X_-BR z=@k8IGS<_WCD|PUW>A!X#7HVQfYbN5Z~22RBxCL$+MjPsPkqPMIMB!2g;whnG;+&O+N3 zzrUOZ3B6DiG8ng54kbH9KmbOE9AvNK%1jAEUyUOwDvFJTeIIBL4CGvacpJ=Os3w7N z34%TPHGG;@kx+`{F9i)iZnpW|46!85UrmD17v1De@)qWqKGoomg z7)L+>sdo-(uyeu_)A#5XfV~Fu|5}Fu$V1L3MqwBPL-`xv9uBBnkqic-jJPtWPqbS< z{-E5nyjhDMI*bFU0_M80j>F+CY^mRZ-fQP45^`h;zbcV-IE;BpGD#E+TG9gYZ@?KS zX;>*@E=1_FE+)JkK1DLAxeL;lV&zdo*$p zt4XJLirmuob&@HR*?GiYNGE6aVTE zFNffV3Ww?PYbym*4k&GegW(;GF0;UQxT?U7H*89{z^wshc`6Cv8skcMXIfWoe9J)gDJZOZL8n~Ps(kV2J#nYMK*L>^*nt7*qpmJ3 z4FBPj>}%njTiD5^>h{y!d_lBgqo0WVh zvB=DsCv!4wdN`y?_@o(;qLaYi!qc-?g60(F&+ z2&MO^lJ9V5A!ggn^$qUV*YW6H2dE;;Vn+F*6;1V` zLDeVOm~CwwUmz|x#{iZ3%n>)ERSLVG|G)tYGlByASzwj&mJQxd4#p?De9e+OXNU!V7@@K6X% z77p^P8g4*nJUN5lRuk+##$Ulcd*g?0!0 z2b}-9k?}(KX>kZ_jgUW#OjzECgHNhE$I_KVzH~FI1OJl1H-P#dHsr_m;={CA5{an8 zy9ONZk*KoT#odjk3f!>x0Z8;XnSNGonapBbp^9_6281 z*zaW=UlYS}6j+f#vQHZmi#Wb>thvafuQUQ=$Ezkrn8GdGGE|3v(Q?+rm7%<{T2mf36%Ta@;VbzEc zqMNP;{7|y{tD|c%0)53->RzZaw`Geyj%VGvu^jrFAo-D6*?-LF<6UGA}HI~IkugZ#OGUY{kc;Me~nWqzNi--1|fhsPzF!zH`ve(M8X^Hs~0S>q_0 zO_aIl3`s3>%%;YMh2SNxVgx)T7bugLPo2$xa{v0a0gE25hx@qu4pdmFiWLAXCaS&U zD_o}&>m2a|&=~-7iXldOYGH!$D6nUwi%$AKkM0vx0xp9nPJnY*Xi#9p(E@oAsN6!? z$N;_Y`QQKf?pbID?!OLbtr!E2^7N-7evkeJ9!j@3D~qD5F_P9=Uu@<&oyM1_b~sZb-yeiCmiZ0`2&I~*J;ziV5c~Y zJ9NAuK+ZRyDscZz3`GK{^+P*Kf$U;EU#>?V4M^2)26iE2fbZJ@KKD|aVrfGfMsC;Q zt9iPZ%tJtdv@NG+=tlb2qyLg16b6dW_^CTjLbes<;|`y0t*^4n8&O8YX#SsXB-|H0 z;BqfO5sel28V|sY#Km(LX+t}T)##60nifJ+fz^BePZ1Me*bb(o#wd#<34kXLHh7q` z=gG)IjPs>MOR0m4h|f1XDm0j@cVs4fCt5LxV+b70Hf5^Z-!bbCoPCA=lRjFlef|YB zB}wQ@hEfCMu8jZxs$AZ{e^fxqM{Kg*L|tQ# z1`Me|vt_3AkT+GZxM9!y=JVZnD{PR++}BQUG$_ZG{I4K?DJJFx%_@2WS8O z>Hmiq&hr`yk_3O|CA@!>*QjpL;ufwICx$dT@|yu(GOgPW@-kP z5On{2Q2Dn>L*ScVgB}R+EO9|D9s!#@ui`|+C;^(#f*ufdI`ldRPHX1Lx4VZPvh!Ud zgc9+Jf;TJy4UA>sZ}sSi|2m!n2ER18cF4n*dq*0XI!hdp|NHF&_*ZBF&W97QKqgu$ z2zn9sih~qGlbMT_-)Mbxg#kpXqstKs`oJCwm0Rqhz}RUC1GOe(t|nxc^qOGI|AVo& z4yx-}+JyrIhY;K$KyY{6xC9CA?(Xgy+#N!22=4Cg?(XikH!k1)o%ftm^;O;b)~)$x zRZZ3aG0h6n$6As5+H`tVb?awS@4tyoDxqNek`=9(^WRA5 z2JWzX^5+?o>-qQPhz=<@EIJt4?%5XBYKJ3V4DfmD@IarH|KAR(Io#(c*bw#&B_#1F zedHaVZsvdgqwbaO^6@w_d_ja$Tme9!bAySoDJeafh=8n|g!swdqLYK!xD?m+dRfm9 zT}}W13issQ5f?W-j;$mycU4A2yw!Rr9=F=0KvF<)$@~a-?t(viGbblt06JZ;1lR4} z@O`}*vRo>$C8pDR^B}1!Wd_SH*!3-Xd(a!xAh3H-lW#yz7bsnLEJOzKqagPm3LzhH zs&;wM#pGAqH`=6mErM_FsT6j=BKf~a@4tmP!11#s4J7L+w6EIFDSfaiwGChflh6L= z;B6Otj$(=r$m-zp(eLCqgBTwa!3uD`y%Wa}JZ$Xvc4TtlWCq(yaIu$h6PSai#?@LV*@ zd6v*!@(&#cylTe38**B{@F&vUKpZ|msqzaHdrnLKzupi3|K%Al88ZB@hKBeIwsrfZ z(@-f`a?D%QjXkwl!ZrVPy`Dk=5;XK%1*U!^`&65fJjyVt5fX~%k1YTyk(qG-K`N;{ zV@v%pwxNb3_g--1};#q~lT zfmfoCc#IL=UWJV8PLOtVpyA>@49x4|s#+knpV$6n_-RNfCf6q;GCnaIWh&kOQw~S3 z=q&3p?r%C*Crz+ZLYx^#yOPyq1VC?<7X7~vTEK^99s-gbGXNJ*_}3}Lxmn`&DJ~<085&JgLv`9+- z)@p8sK97+nbf~nVOb!ElQyIhiBN-E{gk;!71`Sm97%>Pvd{O(B-;Fc^@MJ=M__YOL z=E_moR!SCwt-d3Fa?;!}q(f!zq09|1F2L7P^9y+a=lIk2T${ZIp`wQx*gY$(yQ|UN zM%1%p^~<^xyyqm%`mGdQQzD|Lr>2I&qf_V~j;CYd^)kg{<{5|sz?@oeVwry0u;;aG zVfcSP_5bi7KLkIo{MmFKksPKi$dCzhRkD}Y1^Ej3=Dr8}KY-~Iys{|RfD_Aeawlm0 zHI(PbtWGg2hUDr+K*xcc#fQAHrWdqliXZXG-9}clz6|va8bw zyCA)y8tSW^9@t|y7nSB^D+Z=9VmDd_pnFK%oQEAT*$t#hiJC6c_0sDInieSg2^=&m z&Frk6**`5T?SLb`e`EA~K@GDZH0VJq7q0Ekr)5%bht53GXGs9CLZCy&s=O=m1$|-L2O{xSht~Oemwud2K7X1lhN$hAnOpWjvG?Gz2257*fSx6c|ZfGktAqN z+aWQT@t5d9PhD-`TCA!pJZzPZ zgIPc4Tk>N^wQ^U0hYfWuI3II+pn2`4s_|$i+oq~tVh#&$P^BU~uFx1m*PH=`t*M`? za!>Xm%2_Mfcyqq5%k42}DppnA2inkhgz zzcxEEtHC9#UXLoxQAs=KIh*#Z@au0{;+v^Ae76RR=rHN3?tN_WrH|Z9=g%yz8pI22 zS?(3V%{!xI@urQ%C{OQa3C_oZ$4A7%kxb|^KYfFOX}h>eFrzC4zu8u4A4NJCg<8cc z=mQQiezQ&~Rti213p68tSZ~2+%jy{)f0}BiX1`>%7q*-YYljNlf~K??c>LJhu{72M zdZ2W&v@yFL_dWhhgEyGa)Ze*{m76uTzevRR5>J0;E^ax>@W*MhQ2lXJO~}Mj{D(D< zN?YJ~@qR!2gabj4rbw~G`&Ar zT7OvgrE3|guqfwXKvF4uXvn>;Bgfiv}o6b>2RSX4D{7YaW&?g1bj@X*Y2JnNFQAY{3>I3uhYzSJaRLwti$}> z_b;OlY+?TkGbkXXlnNWZ@68xAAF6i2$+QQ~ur6C@)LJL~``X4M2sjPz`7X1M<=jP& zW(uxrOEREd~DtM$%xcY_W7RxW4R(w}&ws4erWorc71+!e4x) zlgVQGk}nZGc+8CM0CZPP#;zTsFWu@+#lLR-6zl$iAd_l@k|Wu99?jR1qm|38xV;P_ zGhB7;Pqm|09RQ+{?%yul4A&@3EstRr?Y0c5h+nHfR#2gk<&iSqog5`mm3!;tM|XEP z%U33LLt^e{wd=~06;dk7y@NSFNbjL$^<_t8d{4Kl+JV>iYGi%dIYgDiL516%x7~eP zl`|2Wa@%Typ16LUe_#*sL!{vSyu>XR?%x*++v|7zB=z1U>JqU3*d-;g1aQxq+5;r1 zeW0CBs>lxz<%os<;I?%&o7MxW`2tAkp`ixXV+*weg6k4I-{1S;aI?2q zxNnbOT?Q|r-9C^G9ar7E$=Ssy8iO>`(oZkbYYAN~*IZ81pi%p@ zvGAUP6h6b@phK>ZW`4JCH+@%eRVSX_ei^g2E2#!u=Pr=X3M-B#lhTOEL`gg3PRH49 z`4@tNo{@%$oH>>CM@>qTQ58|p5JiqUBZH|ojAW%7$t2tmqOx7y!4=Y05uaEH8mm6M;JgKvd+YwZ}z)ob$V zJLeZ9$c*BC+D+i(4rxb$^|o*;H?b=6DO5Xgr2iOndrLEADKQouXTN?(v^Z_y1~kDB zU6UsWu`=gl;&2QU{<#~&UmhfiL$`G6UO$1@kC zCan`y{$1=e*Ikp&-)5fVDfg8h0r>ww4*COncj-w+| z7Z)w#H@J-2lo5|&=X`MXh*r*$tB?~x{~Mr6Vm5T^n;h*6vFNFkWJmogXpO|9la*1v z`h%*{(kI=~jv>$7Pfq;9GmlblvnezFBZ{>{>?y4!cnTE>Y;m)vK4(2iPt z=T&B(P_NCL21?iHKQ%#!A@MpxA>I>-4GElFxjpgiWmoyr6glI_^!Oc`?Z_dX|4A7Ms2L{sdtDdXRNg1>?rU^n3Z*r0~htzDxQB>%>{^EQww z*g+$2);zT+cHQ(%Z-^;|Er_m$k4_(AFLV5c?Eni3E3Mqga<64kd#% ziOL8uv*BkfdQD{mN`ZH98F`QKZvm~~u=+o7@wowIX>5j&Ia8J0m5Quh@i&TUpVpo~c8p5W)!gk55S`cJaU? z`56lKBJAI+h`MK}?VESDbROqh*P0+c3&jt#rD|GxM!o&fAXN|LT!GHSN??`S0YsqE zThOh*Fx=~|nqpzp`#{xeG8WCH^z*_8^(jge=)EATzMzS&V~kAVttMYV)6(-+>Z^kX z>^_EZX{eOqkX>pL)jhli5LTUqX2rP-*g0Q9Q{+W6T{GH8u0iQZC0ddn`{VRS)@Fo( z*}>i&vI5a=KF)818uNPnWkdMWDSlVDaD&EXN5m&$lbjoGU!mH;cbtn;nSh9 z*oA${4J(ibHm&(M?z`0ejEnrJPoQZYX+h^yj%IrwgMfga5^kwpc44sD8I|p-;dj5< zC&jEW+CUgJKk2^wq5R-Oyc0HwHr~5c`G|jR;VEgcCUE6w8g$4PB?M?AI|;IjKSO^yE>rf5``M(q zI-&d7fW7Z=jSj_I64FFjnc-=!1*K@)U@FSsjDe~Lg zP5U8-jl#4ep-t1JmC}WX;&nn!=?e`rTKlR?A`=S5SW%s(!YicQh7Q-Dfi78XOQ52# z9htqdp?x?*3TPra+pM_r`_EIYebS&}{)d0^E2R+;#5(lSp}>|6;55o-w{WcL(eJ(! zN~V<5s*RE)r|%maYhU-LJ|k**$z_@J=;>5nR@zk;#h`bDy372i7lMzIj$S9DR~j6WsXjV46iTs zH@^n;N#TwCY3F24OV*!yQ8c_=<GFDmw>SgY-%q1mo(r zMqiRngu63B6N1{9%018;$Gr*pQzbxiQKSyZ3UOWi$a)Hr)6MoLFp-!NEIQdCQDwV191XgZ>(yvd zNkTv6z0|x9+^@FA2Y0(D=hcXrSgwmSu`YACzw2JRFwAkH%#rHYlok(Yj!K zLN3wLxp*=0JX5;$F8!;DKXsB)zM33B^Z?a7w;A&lc#ghchGfOH&L0u3T(`)PJ|wCf zFVz$JauqS-pc?yJV+aRQ)+oq5Cxb*Vqzf}~9Dh71yO_<3{46v^FtmehS4ARp2c(p% z?6DqNPb-1?+L51}y7g@Pl1BiLd#oX>0wR{eLuPEQ$}VGR;{mOD(MQOwQ9Tdd`MY8Z?>mFCA|6F1<;gl=~D@Di&v!lCmbnRS1+p-xe_rW@~sqj$0(=jwozOn)Qs_zDw#+ z>%SsnH`VWCLuCSw=A5sOh2r|2JaahXY%py(pEr#cMZ5n!w)sT2k8pxIIPG{&ohT!( zseXyg50tV;%Ilg+l=X8||6&(Z4#FB9bqlNe;%9-CpxK!65YHUWm08u_~JXvzdhyfc=NH)JMXVs9^2pi&p< zQwFD9M14!ZoXS^VmgF|dn5pTJ}?{vG~a4I^)V+Z9PBYRJFYeI08H zlDLas?nQkBrK!c9cr8GPp64@ekVE`X%r0?PPu$`J^EZw!O_bkvgVZ+VPulq(*Ii|B z9C>XmP(#l&zvro~F;o-rMJ)}!L%*S)?RGYcHy26L*~UuuyNnPz>|T%5B}JMz^#1Nd ziDs~YlXeGiQ~u0gKunaL(-m3DTFRHF%V;@P{WVY0cXRHfWFkcR(ofVcs+Tt_)~~-? z&&_k&;9q>;W{83}#O6Psh;0zhoW`UhH*`%J<+V%lON*}hml!Dx1<_N>Kn`rySwdpX zWqt!(8G1~0SS0gF3wHf0jRC;n(?s6cH>ufRG8+)g6IG}F44(|a16P7peK)%f&*lEDtIptVS86=OC(}t0Ug2cm3 z9_5IA9oFrDo>x@4)#!vgxQ^gtCL_Fhz2=FSu8Y3)?luuwBybw%t|5j;?lUj|{95XJ zJYkexP6)npSinr}Q_02EQRQLs?nZX^e5UK6pndb|ANoZmv!)o!l54)$Pb_^1j(T8E zGmVpiu(3Sgdr$kxUUxrRZ@mN!g9_pw$O669Ne$p52W#1{`i5-6+MkXW8)5miZsahD zutIK>^x!qW(}r{bsV>jlps~u=s0Q^TQXQ9LXu0#fL&QY&XMQuX@dK(0ulfp^XR3cK ziK&aSQOJ0eP#gwrKJ7TQ*sH7=kEd}%8=x9g|I&*x%gR7kM@r{9^(*uyT}t^;P_aqn z_%fOIM7kt-lay-v_#)y4vmYGEvqBl0>yZNxHH@b)Yd}G3i4*Qy>{EY!+`ySVlu|aH z;|gBTgzy`|jcU0wZU0!zcyxYeS;xr!^-i6sa8eJ|sYtK}q<*Gg8?d>Fwl}(SFZZSY ziB+vf3}V{!2FBv7*K!YTdLj-!d_dX$h#F{&UJJA@_8ddQ086NDKijt~(B~=1In$BA z?cLz2{IO(C^N;3YjJ*|zpwQiwl6ftL{XgrWgUrYVHS?ijbJ7i|86I9YwyW0jHR^00 z;eu%gq`xF|qU`QJzPT_PILiZmiG8!V5I6vAeB4V)^`4@rl9kJ*SsOi!*B^-;mGX^k zKeRF1Zwr>I=i;ZwACWGDc|#P;AcLCf4(+JIGJ!eZXMMmEz%U==>Wn%lmx=y8Tjhpl ziM4PE{%qvUeEAzp%Jnl`WbxKnrQ`B?DB=Me|H-)La zXsUWAXSj7!vqf>(3|Kt+fXVymPbp0Iw8+=MdTzY@@q7$)2?0pk0UA$!9wLuMG$lc^~@hTPP0kaP_1+?%R>DUOUds6&%ZBXrBe{GyYhkF!id1rMNB_0^@dPr zD*@E?L4lGJj~Q6hyrUA7mblSdox%6mHwCbK$Yxg@6xHB2SBp#UFj9A45c2Njt8GEH zCoPcPyx*_nt+-o@>bc)v_3FUeTwefFncGd;feYZW%HKnh-x08-P_E+M0qhJ!KImvS z;EHPz$*o=ONco{(`UcgGcE>!N>~#$3Y>%DE+k?dfO-Le8e1hh&Zhl0|32vFv-^`p{ z9ZT_d8yrXLUcF4G9UzvK0mqzGt|B$A0CxTzNjy5BD{G0wA{kVV5&jx{p0UatA+*1zvBeP*&5JhG_agUK z25^zZL1ur)lRV`wioZ|Cs8(uMnaFfT>hsnR|1>=9G0c50XUyuG?*-$PUEUM5W#Sq} zB3=-AJgit9&Bt>cHoC-{Sfks4sPD?^6x1G4BiSjr1xku1T%;w{UTWPOVsG#*!`qA) zTx9EKt$S}q8{hLpl)b_eBhl+cshcks0By zag~awW!oZ*Uq#3Q;CUDf`pJnjpxLIYF1FMDxQ7gGD;;N?o*+7Uo2T+YP{w%86bzAQ z^X{i{gp?C4xr;zqat@-jAqQm)>DzlF*+#|=ta7G=XHmU4@G2(a?SLqWhLGtK?49aT z(rAhNR%FKtyE*{i3ip)r$@OoBO1^>2>g3jx_rWn!e!5sn~bx-6LHk03#PfV@iadZRW^NElx{GJ*WZ;ZWl6l;&`u3HsY z{L|ow8Q$QY>E_FAEiW5BDUNPrddU)A_armmesFC+qNo05TXl{+JAI=x62RlMEQp?BNNy|>)9oT=h`0qM#U2a| z@|VChWQ$=7{f>q#?ynG+-j#q36OILuFilEoYl2$6`PJ4Jl$xKEFauhPEb?(gVL%c^ z1Ou}~=Ets&W3f!Pt@sM{Hhy7uhMBD%?EeEfr@u}|xB8weu)(lK_b{-fb3Xmg#6NhH zYwPRd$)Nr#LT?MF+}k#<5!N`R%df$`ms33zU~@YSUhWN&W$snb+?(rzzyQv;hoLh|7Don+G57>nb$AK-&|( z!-xa|f}X~<8;ETU=4fHPKVL74Vx4^;X5XF#8dGWI2wr@7EdUQ3i3^B3ah1G6$Ba&|r(P>1JIVf)m*377vl} zKRkF>bUx^E6ThKDQI2OTVb{5X1JTD)moIs&*IcATIzIEOp?VfLhfM5Xct7e{<}-xv zwpdYq^4u0LWL8wXKY%r6z%ypT#$)uUcFdx)uWJPD#7!9uV2<28 zdTt465ODdB5{}UC4%FdHEKu-|INy;IngrGZTT!af9sA^Mn>>*ofL#cPOMVP%+zBWy z_C0C3V#BQV&*~6pS_R+6eDGwu&?mHz4Um${#{Thpn4s~7H`E8dsLc%XNXg% z3stJp&AS0V`ix!lZy%<5|NKGlt%t=}nZqbp8hTBd%YC^J=1_8!B5}a$g?5+P9>UCO zmNPBw(E>?C`&2HfAy!x)TIFW47+7*^Uqf<1j%1^Y{m{QOP)!luTp>yMO7cXr-1z~) zzl18&x~JI=Y^r%YuUMdZ-|<@iO%MvQn+m+Ub~|3HW#!L)<9IcB;;OX(*hbieCR%m& zk9NIbqI@69+b@fCkShy$JY5)Ze;esC8Z9n)Zf?Y*nO_0^;qutet~S3g%cE4LXR(B* z8O=1$ScmpwNkDmUOB0O)Qx3#eH(T*u)v|E5}@trko< zif1C}!T4i}D_p$7lZwyH0>(xJ4Y&crO6&%gig%yRox^)UAv>eG(UX}5AYk~=|} z|CEJTHU^&U@~^H@Lq^4fw(Tdn7o>}koJ>OU zq-nk40Aqo;Ohj4;aJD82!c~&kt02GH4%fJ22gp_QK0A@^DZ?;WEl8hu>>J$#z9(F2Kz^PA9I7MEiNOkQxE&};-UPWaIHPfD0K1OIFksu0tGnEqF}XL ze{nv*aZLK5ayFboK2(ww*Aphb`;18wU^j2;c0r2l;v2ji>Gi6e_eBLjRqyw_)a+z( z#(M$bp6f3HWA(fUlN*U|jvAw@(@)ODsGjxdL;Bf!nzC(pBfaO^B(So4cRK}n6`#`z zX_Mapg%=0Qbxzqp$(56r0>YAafZFo|cuAjd-*^k>p&tKO^7dO^ggx;LEU>${!Mv>X zXAQ@@?h8FUssr&IVu6(0lwWdO63wkw+KS{IfC}gMi@p zIW&TlakcRzqUxRqlLtN9`zvv|{2M|;pPZFMk^>$SwzQj@zbYK&yMg*GzLY3{aqK4!7a28}r$U@r7;Uls z;(yB`kl^VZgbzzBbg%-;NE4q8G7g$@LePDYJoh%?u(ex&%GwxGXuR5LE^|S8Zs>wc z<|bbTjUqOenTRa=Eyo1*!8%wOQd)bTAU{Ud*?aC;uH&WNqBn+@cNfI_Aw*3r69Ij@ zj~aI)IKcY!iln5Ab@h?1s9N5;jQ?$4?|Y`OR6~9pc6=_-)|KGl{wlm4q!*@;$Gmy; zD3fUQ$t-3qemRejMW<-{wkHu z$R_$mL(&6Rv;GCD@#?UdfjCk84RzjZT;rtYGy(fLglw6PFF#tuT-hS3L^2oBNL-0O z`YECy^&$pw8o>Xh)m7rgIU zn8gptEN0eg!VPYV>YKUa-5l}#C8jGDFVd)f?}XdWsJl#lv}n z;N{kXf~^GLhl3)<95SIsY%n-Yjhj2<~z^1nSg>+r7#&vQK)i}T=7=td_7cWUla zHCvzC>EJNgGi@RQuY4aPo#JQnm{Y1W7@bnv$+XO3pX&SGhUJ8AGA`uT~Ugn>} zp~F$U+XVX$w*$qtQ}F^$PGpRj=5WpfF?|yq&>{Xk0`5LTm>_Y$WqKhe)=6I(G`By< zJ72JWGOKO5e|17JqWEx9dZ*m`z#7!o^LqpN)xw!>Qb2YgK>YJ27MML)RpLnkfk5gh z8DRtX4lvG0uExftj5X*@DzQ&`uP=o7TM4|qe+cUSzDLOVUCIZz*#ngZ*L}rOA?A#AXX+m= zu*4qvEf0%b1bRK)NogZO(v+bW(g=pKS#|E6%nSv7UQHIqZfT|{A(!QrIv}gQxVF$H zTEb|VB;!hIAVxOY{b~`enq_(%{*4A=%{8khul~XQDo*)v9o*i-MdW;~0o9IC$A8GI zgCr7lt??qs&vL0jeSd3iXNJ(8A{()ap zK|95$>s~{5VWN#;+7pNB_Je2Cl}Z4?&e6J^F|g~L=`w$2@mU{5SNAd_+(*yvFJF=d zgK$T6-xh_RPOX0Ym6%M4PuYaD*zNw+JgM7%QBm+B%OTFA&~RDyk%y(l^G3s}?PUoirCMtzV+6gmacpN6Rzg0n z8U{CsaA{2uvmw>|>=%s8zeH00wzl!8R2EY_OqFP3`DS8r&5fcz8mHR~IIvV+^}bF` z?$t95>f%GsIP8e(kyOhkjnw*a*n2eEojIarxCVq)f4L@8f7#xk+Ngm*`OYc4^3Hg@ zdMv?gG7sF2K*}kwEv9s=_S}igOSQ$jk)Jk1b8!JnEJ5fIC4e^4atymO;$}`y<$0rO zL1do1`J%E5#(1DxyKMI>x1l-rD^*W-K#6*dq03i+QFn`@L%+z-#AhH81!x*oqh(%N z?0DfXCQH4I)vHNMbJJFXdh?~z4C8vid?;zb7XovffJfSGo_6dC#=jCUo^*MkS@!dh z#i121v0QO&N#LRKk&BUeQ44Xr=@M`w;RaT|pA|nPT`iQ?sNw$>t zH&bT!mXAyg=k4Ftt1;O=TaCJ_46)v#t$QEo6%q}H#9nKSH!5hdoiEugaT@u;;w(luT;B?A;CtO0oar^V7 z)gp46X(znjhy}riBe`vqU1$`0hCFa)H}2g>-5N=8&**wn>u&o_S%2}OX~$2K6E@mv zKW<|`V*AM%;vGj1mlDI!x_(o81u{R2d{o`3Ko(o{84;|xeGHOn8wa< zcY}fyPwxsqDxN%s)kX0(nzJJ!CJ=`x;yThrBT+1ao}u{oHh{xs52p&v7f;P6HYKq* zG~ouo5S1P({7ZXB6)>gueNdlPwXzg#B(z5E9l1=65o=CifY`5vycEidvyK=M{VD*| zm*-%SImmTU-*d?*HrAMGq_%LAkgbJCH1GjD3#(*i|c$A>1`S)F1IJ;ZgE216WV^6 zz@IMClE6qRK~B8;Vwz(#$7S47tFX-|R_3ZxwukmdY#4C2=jz91Y`m4J)N8MP?8v3;%{|>YXx$M}Z8~7Q&srCNyD??#CfvNiZmFsbkjN-+cLY22 zFWX$hEf7(iI1WFaGp2Fbq$F|D+puS4@QI9Gh>)igO(eUrxj)9$MNK&yKC9r8pNfM* zU>V;P2Ese^d`d@L0Q1OYjN;cCVrFqyZex#h0-;v(=z6c-E zexyBx0FBl64?*-p;=Z>fAgI_%g z?`=qN=PDY%pqwfI?+tOj2ltY)Tut?kiV-X=4AHxNsGM0XC&rmV560j{YD~$fzRmQe zGQ0l1U6z)sVUYvGp2qHdoyjH$6&)ZecwbLHrUo^k-f?aH$(`WrTG1{1m^;j{Hwqr_ zV@!Dps^mw~_?~h)GA$;`BQeMbO!67aM1egdT&K&cpM}Tsx?0yY`9uD!8C}OhV3}nJ za6};7`#tRaTnvI`vmSomK{Lv7Qh@B-@PUb(Q{BD{cr)SmaZwcUZqmw@>!RM z?5X>vj~ArFG^p>}trz37W!uv~qNLAQ0{3h@F;xy$RzBESSwI4U!h_YoAq`7@s^azp z;tCIQL>^u2_Fyx6zcqEFXge^eN4Izw*nPKywvMTWz2*04_t?@YN=ci1uv&`dQ620F zv$v$Ehl(JO5sO-bZSW!Sp=dGMeyw&buqw+Hdp_iIGxF%V$j1!B%8!tk&GI-#kTxrT ziAbEwyxbztRqA++PAfnIzK1#2jC+_}j^A!AvObO)hy8cbKe1o1yR!dGY?<3vKercF>DBsN zZD^9P7538riC;%BgLdM)ET3|ma8E9hvxO%vg0{rY0;G6i@U4vo}`*56}B`dpAssHXBDH7g& zQI41p15wHttmWXW6Ge8%uPH+f?*>9yiKAxq{Z`qYorfG1DU|b#NgJd){R3BUVanm5 zTTbkX{%hW5IAgJ@6y%$Y%BlGKOMD#OI}PjQC&E?GkKm4wTXjQcjoj`6?&eRb-@GN*Sm`&X08@WnmX{Y4Vp@iF^~jQyDXBnKOE<~ zUyn3M-bl8ecP?d!!Z;5cFbnP@25zHRzaE%SKEsxuFbr zyfbMNYh1JTWTPn;ttx1&b(b1_pXFQM`0%v%5^l@Zo_>jjg;EN_ZlM7uq{v-|+Fr=3 zpC)^v9E#pZdeHP{Ra}+1vYq#Q+k+p>xm0;hUU0Y!ajT&8$_Hau>w1}^SNFzr+9 zrpLNwwtkz-J@YWkq48c*-SvQnL4K*zpK$ghJaI2|+mH32E^o3e%&H0%61mFF=E3mN z`5|=~lF1trQcJ%!f$Nz?&wq7IeS@L8%@e}tW+NO*YZs8;Z|=tv@D!P<-!H#FR4m}P z+<|5w|ofLvT}-w@Nnl_ zOl+zz>pj1AI5We1>CnSRc{1R25PfzF?GQZMifnKd@=$ntX{6T_0LV0uiRqlMnogyi zZlez=75*Ruhb~pm#O0DqDBH<}F5sXoy-u;zEX9-gwKrNT=x))F{XFl+BG=2nb_VRucOqpW7OX--|!0VLu~7<8zyFOF@Au_SH@t ziKBat!|C*q1{GP5nJcsTOMK?~0_EBY3+NTZa?wxti{%=M z!NTO;l?2)|t+9Mj>3v7(m|zJ>BZigg8av-b%I&-~LK>51knD~YvZ*vCx{`SLFlp_N z1HQbcSb4f!5ad;RvS8cgM=yPe1jz?3Jn|RM(oZxtT+atQ>#OaUo!%B-yerWk?;A>R z;+6@?d1Z5)mnhcPXke^#^=M#LCQ+~>nU`l`Q;qswp~2pLN-0-jY@@lT=rMa~ZNIs2hHYbN(pb& zPH-9_^7ZH;XkvCY-#1L!F1jD1I`Za1h$_&Gy$2s#Bqno2>bQ(s<#M{O#HC)&79Pjt zVuik}^FQj+>2^q#h34F>JpVNOCHz)1`_Krr$HO3Xkfah%=c<9P2O+pk@iZ=E>)VF3O}L$h9x%Vq>&mkVgbl7Nmz zQ#{stI`oFK*~BeCLk`287h91Jm!7YNv+NgqH1P#_=q}Pnz~GhLz`$ z?y8pH=IS_HKw0HHoB?0T1glfxey2?Qr7&+SNycoics$7vG zR`GIIbea*Uf84l~pL~})g!Yw)%^GAb38}JwZ|TzCC9l4S6Sf%I;2$fOBe+ijjkcEX zXO1Gq463fBsGU$u!W>utHgyBF+1II9-Bu6XV>bS@!X7D9hq2-Q;L!~o(#^;YIX4H6 zto~(QhNifsXrqAq&p0B49L2HDafiy6VCuu6!z`*ph%=f%X8D}2kY-R{h_8`3GA^3J zj-Q@x*8gq%cLh>aZ3VX>lIcWagpj%^M|2I2c+fvT%-9KF)iR&1rsL&(;pAcusy`=S zmL2o7^flN46+e_0GdCIYkPKb>l=46D}B%Vl#ByqpQC){p0jT;kq}@vHmbcvT~u; z^|Wqhfk7;1x~`*YavK&K>A7eot0Cl=l$}GOlTvu!>tZTUfogy3|Me{&@o+Ds>pRLE z3{tO;B;gMIwySiH-UjVWe6|pyi@8}0Ltn_5;h&Dz=_lz_C@5XiljbNE7 zz9Jd?7aY@$R}R}YR(BE=>7%1YaZIi(TM=tRvwf~dnZZBHEjIb?i<-^0L5w%Av+@PIQrUE*j4!$fHh~~l(`BY4E_Lyni?wmzkLU8 zq?IZXbXqqa-w3(&SDRAl2IQaIGebA&Gj_dhnn4tUGT8xFa%(f+D_QTaxV{n54o3UJ zp{CXkp7=B4&_9NWaL1Ub&(xbxsuy~Xi__|f`p~Jrc;tBN!baEf^qwjXVzfxi%`jI| zTgN1F{c*whxtmaaEV6?hxFAySvj6+#$HT2X}jYX3jY?@7=j`-$(bi7R{!=wQE<^ zs;X60|KkdNwa$V$K*0R{fXwtPUzNiQp6|tZkYT(((QS(LQq+L0D~xK6$M0MK{Fp`8 zsIz0wLwTBC#MiOd?@JLT_o|TQg@ByHP|@U(6gqlYTowGr!>B94cHD4A!zbRXpey1sA7nK7hc9$c8^% z;J}1Ujb(0lwc|J%2?G`pemI5WQP&|iV(?n1%!xZVK`2_n@Xl!%uLuh(TDrC4N+epp z5CY?@Rg;0rM1h_4QEuqcBz3W#s>xPy(^L(YUQgWGi8Pi~6!ier&?*^4(J|ul|0oxLeJy+C_55 z8_ejn@0iSORs+Z>*-rRu_%7<%p1?}d#D9FxE^R==<9y8G%5nx(1QO8L5O|5|jb4qO z;#0TvwxgBoTnb6*=L;Lm9dGzbR;&BFp@aTfTxG!%%-bXzBP8Rjja&_)UP9)A8`* zrrr|Fp|6|Y2|G`4Civ4t=~rgE4Ehb-xn1uZ+h=ah6RPh{RLmRp`Zk~Nny*Aa12-He z>a5^abN;d^0VeIlFLQCtQ&I29o%#o%JZ(Fm9yz=}W>(-sc@e zxSX0UzOf-8MXRRcF>&jFfmR6C+wW?1jG0lwrIz|xG;?@CMQ3+~&E4BaQh7$U?8TuM z#~r-?W%sA@DJoBK%2$x#EoBjM@$t{AAXUs7lwoqgO&X1ldqW`1J$QZwLC zr^hkHW;qFDFQj0w(Yn|F9uCdmvM5s6nP)D3BYx0~qxO>v=Tri}#zinFQIHE!%U|2zTxq zmLEv+;?PeQPC#dIU#SuD?OKEC-Kt;ITHW%Aq^z(b>D)V$E+Sw16-?v{3CqT4U7idf zXH--{Xw8k1El3BQFUDOspmsChyNrfxnJqF}?Z@hf%O@NsRw;{U!_r?(5$WgKx;##A z%Jp-rJ=f?bjh$E6QIcj%X+^c-VRBSYV!TEIwJaCA)%MZ(oh|P%Bk3!Gr=u`k3-HQJ z^@;kC?RPhBHqa0(`NqwzU@YHz#p~D{%Aw%fX@3mMq!7Ea@Ugx*!F_OE;m7`Z#O^a= z8n~n#_QgL~|L%u7)PYN`KEVa^Sr_|&m(ci^-a^`o6D~9&IaRp2(-&APf;;FT6Bca+ z64&BBF0oFHaP>_rTfGypGu+IddBhwW4azIu+KxCSZ@mI_%kg!4)1>m+@jMznkfzS` zMrW0QeIsAHuWhM0FJ21uT==~9Bb$m<)sJ6I)EOVyb}F)eToj(C@$)ORxfyue45jbc zK2~*p%H_}QmfP&+t_G)`^U<3xX!aQ+l}iG%W~xC|F4ypC?uLSg(bLTQX^fr?SKAK= zQjz015Ec5iSRiUVb+$Cz!kHWCS;oP{88v-DxC3gb(JQnF&Xz4|yeX1(=ljF7x|R=a zIk)bDWV?3Kn{q-B+-&ufF(tDdM%p|g*Sniw+CVHGLl0%NU&cY$CM}<*61MIT@%7Rg z{YQ0i{L{e2&N@HQbBVA%EJSk++c?ToGGQ81TLjC7Q0kAVG<|ioD3t-V==tl(ZTVQ5 zuTF^oh6DmR%lB7GnBE<)hBzm71TobXY<*57Na}U&Tg^Ber`Zw}2r1g1g;Xg-$3btG zESos)+0%>{mPXiyoZLm%lwUVKI)$gN%{T{`okrD;l8Q%bA@^Ev#NZ8&)nGD#iP14V zavxgOZ3*@wQv#2pIoGzEtnwm+tcYGZN$^YP9r(OnW{8F*35AXr;M>(SgP`3zy8QAh zH;PlH&-D`tzHaj9uf2N#4)~tu5(@t6mEepklq}kgQ!gwzr++k!az0Q`ylB>6f;+Ir zt3*z;4#H3v3;bkrZ17G`y}^Gb7WE+yKiLgFNNLi)4Aql=^klyn)h^oPLe&M6z7qPp zoJap&Qoo}($@b+tIp7l^=yHOk8spvZ0$HDxzR_qGT8a~^*`mbJT1KA-xvVQ3PpkC) zlGH7pzBOOP_5K$!_DC?t;|)?4>BGC;{8%Pm0d4do{WSir?b64Ox9w;f40F{eiCq6U z;)NXYb_;&yIDE)g`DgV=71`?czg3Ss3H_kZXM7*vc*G|c0p4GrQ9$Wp6{;Gs0M*vS zH!`eF4@h2OQ}ijRs{UXCpmsqKc3xfOC0HsI8lj6mu%ck0>FJ^0r~`aprxBQ0a|@Z@ z`y3d@R0C1FJ01>cCO$?GcKFG5@GK2FfX_!C<({N@=}~y4^{bG(0cB88P&7vTJcx09 z(LM8svRlOx3XDcm_VdloL>f*sM!Y=luqY$?0}3g1pMD}HpN+80H=hRofc07A$r(cO zgNnZmtFh54RDDx@qrj-#RwxbIfba#Q zVM}1Wf&F|RZw^pwJ37&*YOb-u3%ew+^pTKhIk-clGjFzutAowTEHmI}$?4}HY4DDI zW#dPD=LYM@G_1{uR#oIH$=`kv{*-J%Qn4M7)GS$(fV8y6E=Vy+tQXl9?an4rA~Y$S zMl_Wb-p>P~ddTsN`hH?d-l21N2L`fr@i%eqd|GSeB~TJl^4 zv5g12h7az2gcRgxOmY)GI=|vW^5p`nr?q2x>m#n>nrPK%v_WK$rD5DK*GqRj#CRxh ze-wLcdY$GE#oM~HUImxmI`p^D3G@IW%-QLc#w>zY7!NNJ^;}-leu@Vo`^SgZv1!B5 zeM;1yQz4SLpPu!2&&YqNf~#-xiZ4PzrkM;IKI=v7pv2YYdlHFTuEyX})C(o)M=*=k zI>}k!>#~JZx`e=U<3V}N;`&QlxPWd0PxmQKVWQX*PgOX{dxWy?v!_(k8RoY_!iHm> zGF;Dg!ey>?HEQTaF2wMV@36`F=4(l?6=hdjCYa^cSU_JH6>T&XE|s2e8&ly~;U707S_ zBw?r!yfVRwK6QRZMot{*QJBt5Vs|^-mSDA%({7m72SP~3FRW!YYqR4s7uKv#_!E_& z(l4o^e48q`6XH%sH9@jcH+K=_Fxn!;vCf?_57Bg77WX1K&Oa&ySwpYiJX9%o|RF%o-p4#llbkMUp zzX(W?w5{DMoHiIyKb4=<=t>y_7YO~{lbRGxG7f8aO6#qmAECBjG}3nZ9ii={qfT|e zaAKDF?e8QvAkV|Md`c2pz~vQ8F}%AB(8G$7o6S7$AOj&l8{6Od_UR*X&T_H~=1pdNqBm3PY=l8q=UQH}}OwR4b@)_R!^%r{|o{ zp_^9d8*fce%Q{Z%k;oB0+o9Oi6KcszcQb(z>pq)ser~3-uJ9q5f%ZOx>kxs zf{Vz~O%*P#r27cB8FZ~kBAazEyD=VVyJ*95#Z#fSFu}LE zuRh~jeJuWPAM2G^42YH^)3?NIJmrl&t`2!)w}?6(VCL$~g87K7p-^pk#X3aR$f6L{ zHu?f)?}Y4l@`$()ybY;E-x2sOL`RlPvCk7C$=SJ3T1iPOZiU{FY0Q<7$`En`7e5Hy z<d(5;x|4vu549lf^IYE^wp z6wFY@a)RoW2f3(h=!zs}n{Z_*5KK48p+9$uD5BmkOcSxBb-T$y^0RChAqDIB8$6u- zX!d|BF-YnLm4igMsPR;XM=zj4w~yb4FehZ`V&-)CZhP!yE?h=go#T^+Vk?SB_CuWP zK~gd|LZMe^SYJasJj^9!-xGA{9DV8cfRia-<4l*&*ZR68L`U-#_%$D3&%Ucy;yb@< zQk{;AEt>uTw7b+iQ`z3g{KXIIFpI3qrBm-re7Q?Gn9h$#p^OG@kz4X^(CQ!2_PHbP zl|0d6fI4&txbrpWGV!4A6gR%R&F$4>+Y9Q?(DxjPKYP~hi3L*DS7tq=ZVmM1R3`9O zh!zbYGdv|&O$L?_lEi0H$*nH0s-M*aNs%RX)FNdpUVl!^MPE)$T6H-6@Z%^6L6|K> zBr!u>Z_bHIqI9ZR;%RZ3O3f(^Eo12!2Nge3G-Fsx&}CY+2=A4Hmu&~S{7LCDuMT?{ z;PyoDJW;8tzdY5(#GlTC-&og)bo1OV=cL19^);&Z;$F&tKepPG&dQW9(wf5pU1)+s z?3tjEoVEKa9Yaqr!+0NRIk(`$Y<)1&@xoIvJO87` zNS1KLbN=@z(<=v4b9TT9Y=Vn3IGn^^idCOcb9}fVkMj11tBp z)!l1}|J-8W&m6U%3DR zhLaEQp39#rN9j$yH;HLzkuZ{el2TSe@jH`4QrQKavK1Dyu0h=Ln`vNWlQ^fe6t53o zbL+imX^{vjJ238XyxQ6wxPJeL?JRu5r`E5xHpWnGv%t&C=9w6A)ED6;-S^(?g%Q^h z($j5O!&BzH;)<{OVTc|?E&1ydTp5$mZd{-LHai^06z9Qw1_T-+&*mx&u?>7u?3=GY zVry@QVg8QaSJ~cf!>ik`Sv&_j%E47Jy{B}QZmG1@B7PWJZYC`g_CfaQD0gt}Uc-Bs zu`cKVpXKY!n!=as!f2rji*##T-JfqPwMO_MHpGWZYQ8~j_{4(tGNAU+7W+`vVd0F6 zN@Q(Fp=AGgBr@s48a5zn)^r|)%a#1BnP!4CpS0%qSCY@Qa7#ojRo2t6Z$#~TlgrUF zZsxUZ54Y9tI{+{Ez2f_(9LMw#1w2}J_${LjGPmtw$}G8Ocd|Vi{n)8&Jd5VAG&zY4 zEb>6*+Frows~Ye$EVEJA|Avt)El961EG~N`1E*%5~t8lHX1Ih=seEUal4^uV>@wVL7NeH5yTEYY9 zO-otgqxe@JEi1TB7AwM3l7?Y3JHKXz6-W&K+z^cYBXJRm2~OcP9`%`QbUgMD6;<3N zC}{3wlBcfqLiv#C05)VPp{e(ykLcT%sf&-5Tp9zHCL32}3U9R$o7WrU^3Rg6!Fvwf zd*@v$8gSm!sFKNeuZ8L`+ST~{c+*NHjC|^qVi6M_BJ>h_G?UXzbdg*i&hYknR1}ej z8eP|0EKM-{GMhgC^zO5_{c73Rvm6>F+3S6zR&76AXtH#~Zt(b|)O}}db$y;-nmjuY zPAEJ?#*c?Faj~Lm2y#?b)a|-kkNAaBvkkrfLOFy)Z}sVzEAFA+@uoujM^(X z>K=|CbvhyJ*jk0YjZtGgQFi_cYAdOUpN{A*CG}O%IHWt^)qk$^%fy$j@GC?XsX#~ys$sb0jsMd5^+~QL!%ZO-( zVkxO&2nW+Ib-Z9?X$m1l{5|{vze|~^2yw=q=z60@@h2~bepe+u3gHqfy(g#M_H6?^ zouL}$fDQ-Fer21Kq$+Dv9Un$r6uuZ%CIu@h-hX<-(5_s>9D_7n#JHQgRkq5Q?m84V z0J~M?6kmy=g@ZOU3^VxMxnQFmD*U?Q&KR_=qsK1v49haPEpqSv@he_is)r=&sBThT z4kdeHZw%&!lpS6re@RB*KzOD~Jq^${)Ymse$A_9(7#ccANebq!DOm|YI~J8+miURB zT-4V?Bj!_3XBonad{WLS1L{;9+eBQA+%3SZn~Hb~FMM+GH(ge)X5~_>YQ^Q0aRJgw+ zA>y6+!XVCt&Rf}xs=DJ9Nw~z~$UChRR{QDV2d9yw-;XQads04f<_N=IL~DV&VqVLe z31R8ABhti%I@>yY3=32!d3+(#waQ9I>lwcE+O?LK<$~ePVcO()BT|RK_h;aKK* z#B^<-%QKR$51OG~2-$N-t2nN`=Q2Ad$XRhK?m)5GG5z|Y7_;QXb8c5Q)^5~t;O6j| zC*|5iA3Bh5l65<9SN%@$x3qq|Ka*;dUa0ji9vWGx!oid;am5L3 zF(e1n{2=y(QtaBH=x-U@ftaOdA2oMN-LUv!iQE=I{JjD~H1#L%6S6*Ub~6{*BDsEJ z3RMcV$!j86E9v;f(jWGLq4*qKpGLz$npLZgB-T3@1AvNf%x|D$M0&rTV-ndB`(>^ac3;8 zr-^Mc?vX2BL!^9g7iFp_ab*(}7$l`;lyg1oYg{fKW*+6coEk3HOUZ>&2h6%qqhW^X z+%D~&Cyak#-9)N2zTfbP&1_SQJlv6MsuGsWLVT;MoWT3}l{Vg63!NI%*pfGUCfb*B zhZw%K0q`8V>6R^v;mbZLx1@)>yGNkR6b@C;ej%s|-GfRn*he>|Z`Ama6f5&~+jCL=@A zh`f@IWxx-{b$~Mcf3cO`M>4O!@qs+?|6_9Ce!{K!2(A@AJVcomnk_sNP~6%y8_Vl zpij#~WjmWmr3f6&GVnvPcLt@Ow^u0wjBT0J5AE2padGA_SUjzCdR3L>K7-ojkXu`# z!pB>O;5=HX1ZmGq6!O8!SJv6} z&RV6b&rQ+c9SLg5H%iKA+h$b^RA0qo@7~7Kbg8ArFk=27emf1{L3X5Q-fVVNCkGAm z5_am=P7nT>NEdVSBC6WLBsmxnIX_dY5E0lnb%(4bqs#4&)H4`;9#o}7r#EBV%dO8` z&Rw!s+WD?19EdA4ts>x~%YNF|@OXYhhF;F39s{eQ7pmWxDQx!52x`yk9 zKj%&bivif;(S^1F7d^?MOGS^R?U>8CS&4+U%gAbG`9z(0XTIGV@4>tpz<*F3c{b4|*vy!DRdZSL&R-C5 zRr89wyPM9%XtC@~j$2!5WnP(nr5?a_(xjVM9lcRn`t#!AxPM|0F#Ou+4E#$}jX##< zn&Qgy`bn_I(N*pVp!myvMw`^qM7jgbGq+HY`mXf1gY>n{bBV%4*7aSV{WVX)%{(g{c?y_Dm|ojgR8s3h zjLKKwm;#>PR}|c8@S=&x$qJcemlM}*qL}q%WHvy```t+kv}D~vybMv4=&ysuhtL)I zSVp=cW#LBUUdTECzp6dJtk-QN_NEg#8^O>o@`pz-v}&DSY0qdwHyvEt+t!E`8=gJhMcFEUCxCz>Lxf0>W!&aA;5 zc2eN(WW)-bn&s&HERrN+{V86mzzn&G-rbB(!EU>Jj_J&^6{*mM(&W+ zrn?rDHF^(3{mP@DX>)-9s}Dq&{T_eRXG1y$y}`JvN2V!YN;XAtdK=b0So4ppfq!HU zxCZ*Tz#S!ojm;@?z}Yv^u#W2x?f^&fd3{5Z7#nZs*MZU;B^O->1z=o0-*Zh2-qc%$ z!eMBYZ;0tF-vB+lA~r2`cY4^$z4PJsFcG~aYdlwNyo6``cdGd`n9&AS9DM1;(>pnZ zEbjwhUHMA&CT`68i;di-LyiUG{!tsm2>0mu7`={obDB2o{heE2zvcXDKiFV6*|=~l z2Cd=AyW;%*R$wqI^QtfT(f=<2Dt#??#b z#_8PdCYE*W3FdqJr5%a;ZbvS4mpztn=jQ*WN=p&bF^|q)Ymxl~~R<46`AZ zQ~yZXG;NuJ2Xn)bp|_zucSVvvXhE;=qNbB;IPk`QH$2Z}(vl9l4MXcsY_wz(uk!9s`fKqwb(s_X zm16SGv7Y!$Ap#ul%(&uGGC$A#y-zU{vuU~4?ogfIjz+vNo!q7gpoLUe3V5f&lur2G zzj#`_a0-7({7_VTC`Ky!cDQmoyCDuvmsO1=GD#ZcV9 z$i6#q%Ww2h*eVK(on{S>q<`0kt)KX|A>D8B!6)VxjCCC2^V$MWzkRLF>W%OEP=4Pg zw_mOK!ci&uI(0EwH*j`}^-13Y$uoD(>4ew~;P_;Vd95g*&!fOS%u#6hFz`Z1f+ z&BR)2lD6ddakLjTq!%)`mX)fjSFVoTKQph-?9m|tDFa?B`TCDiyS;%- z;N0Imyt;%x6U(JRz!A;i)_b|E^?om_Y^SsA8IELLBJGZN;v8mkU#BjcRv{+9sey8+z|_UhC5n9tyGrore&RZ zr6RAuZMi+x9YYxSRcrF|*ISzcZA}?BAsVbxJ#glPgRG&970&@ZTqD>4Z}KsP``X6S zUGn+R4tJxG*Rh37I{_oDP~6qr9xov|b3&TIhRv<}j}8~;`i!OZZ@83$0UnbRQb2!d zQpVM3^axQME-sNOH9kE?BeqEJOL)ttTB`rI2>frq?Q#5(4pq6-J+(}6@swyC$MY`> z5&5m}Vg$Hp<*GRXY{Hv=il2R2-BtP@YUmIP+L2v`lSnn)|ggb`T~Rg?{pO= z=ilaU8T4C>nO0hXmC|CBMhv5n=$PV^URiK~z%Z8{O)3*Fj9LmqJ}#6dce@&*!vJr|wYNt$G4Y=ulP@4%QX z@sg1n91jr>K^*1Z3;wGl^goThuc9zsCs1Jrz++ae6}vZ$yo_)J6)b%I&m0XHwbK70 zPbUXdJ<#9rgyk$960N)W1TBa1gKGcE&|kjWV|+PKov*(|sId%uJG@d${S&fdEo`Rf{_BqZuUS1Y zr05|f6KocTdfEQ>1`J8wdqst6VVd7*PtbMMNgJ>S_)pS+xO_~l+ro|4Jc!~}pJ3uv zAFv_g2ieYMM69(ZJc_nvoY&~~OP{!smfJh}L;zOqq%a6wQFu~VBB%NX(|dd+#fc(n zhTfm?x0v`}QXLY+t97%NFbp?WdW6tIp#MzdO8L2^*gs{m5hMe^TjU&XhJFQ1~gK zm(G>-56zgL?M~&p*dH*{R{!1swOtX6c)7k??)b>t5x`0zpDaXH>gu%1ljV7K&)nIr z#jx$vfCvL6*T|xsNvF49H zKv$ClOLgfFm=;1Gb^D$P&xxy#iS4!Tw8bN5C4@=%Ke@2T}t4SC?^4RNu$S+Hx;v_TnssLvX#q^BYqcFj;BEQYQtwy z6vBJX1|q7UVTA4pK1Nn9NoJq?(=u>;ai!XQiigOab`?wUIQM@*K>2s1#~A5YWlEUP%;I@=zG0xT#XYdmwpXZ*O0&o;gG3T2j_QYzefm+iVic zBKLRo&~<71CY{wh@qJDh3OQamr;DJQzZ^hhH6PXb%7@a@2}xB5(bu{o7X)o47U?v6 z!1Y-(UgO6ALn@1W=4ddJLv-1glig%+-byFH`rF0luN1h^hT`s)^L2z6C0Zngpojt^ zNI;t2YdI*15E@F|ex7cd(6oYKMLlhQCd|xa$9s&FY+BaE#y$ja81V+ZBMc11%Ibab z*Ba8fe*Qgf_usI!KKfFS%FPJ~8>GqtkAL96p0u>WZMM-+`?D?d8HoK!MjIH6KdAV# zXwR6gI?5xSlvL!M`r`pVP5GOTw>+gzs>P5yS58T0wkM}=tMhLE1{U6wPr_`C|Fz?x z0*S4W%N}x8Zrd5BxbZ8(U3PDw_1K6pdd`ykd}Faxi-zB5)e|J!pbyC`XSxm~zw7T5 z?b?d&-a-Wvo%QV9Qdp4(-ea5*xIOZI3IkbUNawf?dSqa9#NX{m=SN|Jq3k{wl%z2O zn4epFW3?)8WFsDmIR&RI!cYoKt(qJg$^9)gI$?W=+^~5r+NrLTyA1<$0mLR1Z%{^~ z>5c@zAse~X_TI5LLscE)2T#aOo{p+A{|#jbLX%>I55EPPIA0shMB#E#63wFjh0FYr z&v%h|{*6QHrm}|3{Tl;2>AWjm`#$`7I8gK9mfERx`*p*$7Vzy8@pM>~<3j%2Y)RbH zCUzrpLQ7uc_n*INC6&A`I0sMbqeA0JNuIS_uwubRBWeN80&3wR!>iyCyj`7HDP*SS zokqr!9mBgv2(iwTz-LKaXYd=>@axx7bqt{|hR6_T?nL3U?`^q@b=8L<+cO1U@ry`F z496s&B)yc6zL{1Ci`XyM_vJvCtBkYO#3UYEL&lukZ_`tHs zS?d^7N#6jE{zP$uz*2qvAqt1DBc-FV9CX>b7G||#{qt^ce0GKL+r?&hrEY(e=6s7I zjdHnGU;>TGCZzFe-%x`2+2qhQ=wkLUnd=CLHt{N-Jo%+tP-F4=u zi7{s+!{VX1N2OCN3ISarENU^v2XZ|XfR-$d(HQ7Q38+{YktS}y82?6Rw9x&0BZ#5O z;o68+y&l8;-M(^sm5+eSKy{?#JB5C(uqSf+uZdj-d?fx7Dc`#lHLvN5wzr|=V zEQ$*=b$$>_iJ72+sJ>hjT4uL{gIBjalfk zeJ_A0Iz3+iYFfiaCsro-g#?W(Uf&bXu<8M!I{i<05(xaye|Y`ihl&TkXYdOZhCIFw zg1)~+-1h;2(~vQ?i@r%?Q0JNXJ6Ox@zFtkpqs>g|x>a~F^}d@oKyvdeSbx4UL8osR z*~avirt~P!-3CdMM2viGtU+F7NU7wIdoAPo$gP6hKNLMN25dB{fuNENuhmGTC5w>d zRZld`VbRZ%<^(FN4E;dSZ@~XL#ryPhaa2vY<}-&CutuquD2!Ojg(M%UExG&smn&J` z_YN{8$9#>Q;neg-wMMnlS_apK)n_l4r!UeTPZ9y)QdHuGRN@4`*HmhXpR*4QH3i7a z83ob|sx-ANdy;k0AQgzgXX`QFTy)MrPsaP+=VOof8=uZ}mW#v~UCzW&-b31aQ1VAV zK?!JxGUir_0%=L5%Y^@{6vqU0ZMl zrpSw0wVUO&yX()Nn~nU!KjwdaX~%{~i3P8AhLrAHcFa7g-1xM-jbRfj7j(W}_B1ALW{60gZH}l^=?)WWxVTmZpyT?D;}wzn zQ!`2L?#H*scO|Av^38}2#joefD^YkB+a;)I1~UU-o7`YHA*KmLon*V~ns@=!Cb)5H z#F2X-ok;T&x+Z!iG`vh&2!LhcXg!7l1OxlG)8b^`zw!Ma76}Bl06`8;-=M1H2JZ52 z$j1&ss9+j@J^@E|{)G#ma)oLz+FV(WJ5hNdR$ z4M6s*H)4{87mo0U(j&}l2^gvbNj_JnRhb2;8OOxZhB0R2w3a>@I)5eI-Lk;?RN=Sr zZ$J%psOZt%5z1Xpk3(Yd7m9s_i#YxN2(aZoth{aa#6=dFSAZx(b#Rz3{)XR8}Yh(Jw z#BHEur%1e+ZAII>-LakXwX^)Nq{rjGZi;U)#l?Ie-(BVK=x_M9Eio#)Zev)rS0GbM zQM_G|QoHw29yTD12K`f{P9IHpwQRKh3cJASQWJ7q;V5{#a^G-Jr{pU|DfZ-wdSmZj zxd4N%dlCsu}g6~f>bD< z@fh_1+(y`$t~+eYPiX=WT@T($+Sg(07cMa-50LoeT0=qZz!k{gs_d9PKKNY3uh@|% z`3a(fq=6C4pcR3a1v3AG7GYRqRg&cIU=24!4L2s%STGqkNDywi%s94(IS#DH#C4wd zZ{Tr%tLA^7ux7sbgW8d6d=?Ua$<_om48#t6;p}`YZ(6@tDyvh!^#TaoFSrG}+}!V( z&!}GPN(S_N!%h;GD{^Ie&cC0hC7EA@cJc0cC!Klop1!S&a`W@zzRpxJX70A?pr%>X zwzHCU0*_&}qN5U@aVms558Y=j8O$IqBq*Hut*8EDB$jHO|^%84IwC#p+z zI9AmznY`S3*(RQGf(LYD5T!E{s>tn~M6c!LleSBZRl_$)#t##D6Dkko;d0;1#&lYr zClAnaQOHkwOzeb+(^(+v&X9mY$^93gk`Z)un^;4Ua086w;Ot+F#*P0gulVkgPQ-Yp zi!05!hV|~!W%U@|ot1Vz!^>7YV4_)bhI~_B1~lKF1(YT(Yt2%bxBss2kOt5>GzE^GV1R#kEGvIXnEK& z8Fik0GHO5LE*N#elI7z;lNER+DUeQ!lxopVQifCLjU6RhHlM zs3T$wrTZoFmWpd%&>np&Rx4ylaFudp1eDFl{A8L@9#UA5WQBN7VY#!szx!K5ufTjatg=!m8qnb|0st-E)3+Xf(wKdQ)37tO*04K>RCZf1 zUkmK1ig(;7S-V|J~mQgy}RX~1L`*0EVvqNQo7R+76o5sO!2X}dg8!N zwC=;P2{oBj^jx1*KA^+WAKhDau3zQ3#eH=o@MgT76E8_) z6~6&R*PekP}+bAqnpaV zUK%bQoVtB`0r8D?XHk!z%-}^)i2`-eqPos>(e$Sa?>S8rL6cw3T;y*_2BKF*hbqQ`A99y)#41pX2(Y+Z?MS8) z(X=|3->UCk1a6!M>)!cK`1jhtb~F0vww(V(b6-D3%V@ISdhhT;Z3O zC*UZK3RXP5VTyISVk|^S3!NovbaGUN`bkRbk+^j%_$c` z=Ww8NQhRtXO7>;4;Dg&Mnw5?{0}i z{_qRT!#oH0|1{^<#VH_~o}UEmr~kN|pFho_gob|NoEhaz^>`};E?FH6swIAlmFh5C6S*AV-5O{3!IyUumf;9;) zvtFz+HvYvyh?PR#FMQa7*MjS}v?Eqk^hSNoy6Xo#OtITz0v!Z!3_sPVUHSe^>i}h0kw3D zTX3E8725%#bov!1#n-Z}OjckV#n+O{%%WY8xmW#TV_()4457kh)oeUyPB$vc%8Iu1 zv6T%c&8^d0^HFd|*6qVo@JKK-C~-h12*alWc4HC>0UKw--w@KEb#G6n^V?g^+l`qd z?cU{kDH}QD&0=G%u17m*GQ+CWwS)Wr?rKbwLGBI+Xl_2g?ZJoWj3YS1y+38c1_t65 zD__PYpeV;FT#|{2OExcG9Jg(2u8eu`$9-nU`0S8iaWwEyR z#rIoKK+>lX72te`L_iA$B8M#_BU@spuhw| zRJw48~i~SmBFs!1D1@i{sbqJ!;ta=gki1PXo+*@G-d#UX1&yM5w7%k4wQGm=J z(B1)k;e#W7u4m_-uZ1eSs|G1NBSN%14$ki1(b9jnSpS~02K>WxMO|Phlln_qC$3=L zllarAwZVvpyhlO`H(Ln|3>?^P{ZWJtkA!SfRRr-H+GpvG=pNIbU(ocr|tyY#-gFuErEE)GNvqo_&4k%d{AVQ9)_zEWrqJ@FtQut56P zwgSPu`=77RNRS8sm4E&`?#SOk=ezfatio_4?X< zIPQPfc%`WD5?WW=TUlALJ<576jmYn{QTZE&qBNL4I{gvX=n0DG`T;MdGZ9k_;Jnjq zYt+MHRYAAnL(shK*|}Ixp`Je9fH!xM-k!S){uKQ8azFl0OU0Ko;~&pS>|y-P++Wbf zYplL~gE}NDKXp>+ggQ)-fFsnstT(ENTqM5v9cf~jwT6u9A!f(RaZ7_8(qrb-r(?|! z61MP(Kz4rvMxp5+=7dsrHF*qlK2YT_fo@!^m9Xb+# z`LBr7f1vmq#5~f^e|S_a&tT@iJObZc$_aF!1vsad6C`IM0-K96>wr17A*nn{>_3>k zEt#oa(zeY#`usax$bL{~j49Qfk1pT)A7=~)5DiNxj-dTzUlh>tO)p$Wjha=3$$<_< z6M3OAL|63ljYB(xcib?lq5cyC{LK=d`5@Zm8(gVzcvbRWv7#R-h^&fmG*t0h2tZ&N zF;uiix5M=jjqIHA(A1z#@U(MDL-EC7nzlj`&t8lST%4PoA-7jfi&7`?3G+#3{v5=L zr(;g$dGi%ba9U{A5}*rcLD1<d+obP;8j+* z-a2liesrkNoTb~UGo6O(R;Ax-LiJ4Ke}BpMz2A1M=RRFSww;FmD^tTEOB`f94TCzN z``T#yLWXzQGofHIK`F?8QVs_@t0qMhb;^Nd&Nqm+< zhb^wXf<8IZ7Na+PvbCZFb5TS@ z0cnjMkbXN}*bRH;cXpk4+Q+vX2Xvm2Yd6vAyH(^vNmek~;rlyCg7$%B`ox+Qj1Dwv z$=|z#VdA#PpeI=3!8uB?Bb42#)CeJJim=JVT`1wuX1(lK*Wv^0tuFC^&)*aCuJhkq zqZ%Qm=)Ez-&6PZ&VcT)Q_8jv0Dv{v%zikx;cn~r`0^#QUb`5`Sm=g~Ob#TD4A3wUn z&=Y7I>*Afs@dkg#8nZ#7Er#DI$0IR(z8f;Ki57Scw*5-~#hZRQ=B22azfT9#dJk!e47xIY~4Q@<(z`ZM@+azO6*Es)lL*|=}2(czbr5AecB#c=(LRLjj_d| z1C})x?yY7-*Dhwj@nH*2OUYc1YVqeqysvkX8&=BJ130~Ea&n(rYPsl}Vp)-6jON_k zLf@|EH``-&ZBi{i;<#x!<8<#RMc}EdvFmaDFM92B>BoO4+S2&}4VXW#F<|oT8`7zE z!M0_IYPug4nU%kgvU8wHU|^2CRKCAxR_qwUIuN_qUb&}ry`aAzwETWV(aG}|%QVwE zz#%{9T59|2L%QKLALX!4Nk?jBtt?x^Cn+VBnItZ$W;a=@+@YzR(iifCY9&pH84Y<) zG~-r>@#5LvAH+O)1{Jo7xdxi-ik+`L4R{@kZ%U%#i;b1E4@{mb`PKAE>6vp$0zaYhA>m<6DiUI(s2easnoBG>>t6}wo=>a?Gg|Be zES|33Y$r=)N%hwiX8rUR65D#oj^6aujov8U)As%Sj=R&9V4R{P2eVU>H4_1y0qnbD zog=$iygx~drjqHz{ldEL9FnB`QLBas?fRA6%4F^BT;r*q`C?Y6wi6fpqm!jSkh-VJ zt5L^f?S+wsE&3LlIEDuOO)nk-Y}3{PcnXGPkmv1 zFHQ#cMvjcJ{$-7GwZ~yx(?;FtM#Ie-*vE--Gw6n3vVE)B<@`SB2sF%**6uVac6UGS z?Kc(1-2af~DnqgUXol9(a1_$5!z3BU+iFC(f(afO@^7kjc5Y+zF2->$H58@9GA&0b zQ1$EJSn*TQ)}D7T(aU#nT*SV*yFNu{S8`wvMc(U+c|7fognltNu=~pK(fxn59()re z{t>ZFY-Zu2`Ez^bv_trihyte$_gD~Ei*VxT?BtZCH_1 z8tIa5*mQ@av`9-gN_T^ZbT=rS(jeX4-Q6YKY?^nW;{BX+pYwj-?{)vPUF^N*Tw`2g z%n{d|%l&vCkK|IdpTvom38`EDHphXL--v&pV!%i-yMa^1OaCx1D6!7IJCeppT{Sk+ z-`Rd(QB~}W#73r&#(3~sa7ym&Kl=V9#gl?@w?mab~$&M?$#QL zVm%6k-Gd+T#uv7Q5UfeFrfsD%dJVKY#^O1ODG1$TeH<3!^^`r(hzf;qS z&|-#^L;m({Mn`Ck0L+W8DxUYpj@)`*MP(w=1VXYcJdknzbN8(xkEXrdKBo&+Tf# zkJAQ&G=Qw}+@VM{hgzO@n(R2QZ-{^s@t}}lxLk<)nEK$fZ|*_Dw>Ng%8oYUonGI1u%yAW4{oQq3bEJlzO0wKO zr)YxAZmjfuXdjgmtyAo=f|-1Ox9Xy5;RJEcoOO;>W^j$ZW`I9W@7``Fr-hSJY?+*m z!-VqSJ0L5#m5i^+%rE>F{;M}^c#*Q>?r2AN;|(?S!yOHpmj#ad`1bJSZf6ixsB*71 z%pA(g8kzzl%wk$D15n2U*$7;NLO78r*h?)*C8uqR; z0KZoo>?jQ7-q_5jEmfNJj2k`+#UooEzTfaJ`!7j8GgjkmPW_BX_-Diwuvv)>tEOBlZb)!V-XG z7;uKbRJXbn>$^HEO_g_XKFqV??B zD+;j6+FF92UgPz%Dj}%vL05-BRlO5Uk7CL3vY(v-1{~@}PGpv;7GQ~>*VR;OsZe<8 z0v~dv$zA12d;zt*P>W5G==j{)MyBbgda<>;(_X+m1&>TAHdCpVZNdUVY;_84Cbdgh zvs1g}HF2w=?nx)%dmA1;c9;=1a9o9>cHhgq#FA^to253iut<{gQ^q)xZiTG&lP`04 zcob)$S8UdyW`S&-n>zgzvpZO_X@gsF?0P)V2n2q0%!BQ+3^k}8cRTf6A8dFu-SzlB zy@Rg?)0-f2a*7eLZ`Q`0hm9v2S@{Xq8+&wIVli!Radp@j#*&!F zEO+2qw9IXHY5#zMuLAct@MCkD$q4TcFLn<3cFu;k<sW4?fa{1I!Zz{s?NKH|yfKB-~+3 z?43~(Gyiz$DV2@9nfgc+F<&%tU1~8h07r7)?3C6P#P9($gyO;&_WaDwCkF&@B>)ylykGw@3OJQWt2d)h`FY@ zHYbH^DtabE35Q0+am(hy?s~O&e~;B~rjdUv_uaLK?Kff`+r>mjw|(^;M?Ow2IctFB znLiT9JkWq;!oL5xt(bZSOg3Z3h2&G&7V%kh!3Aa^?%KJ){yfxfPwwLa=}2_Y+Ys#1 ztklI%vvgE%X4;Si4f=vhxoz7muIE1BH&?r#!5G_&I8EXEW&6941p<#$yWa2EV+R+n z*ocQvs*Mz>rmoh7ECSqPV&SJ(h=c>Cv*<8FI+D;X+DrVFzT*QvWLir5HN(+FO8f1y zgS%hwO_ZM=0cVESq8iyX!WH`YJF3Fb^JYK0F^^@{%oq`lrN`7>osgyV@xxf2WWR$b z-K_tBA4!>SP?*z61G55CY}sM#y3@8=BL9M@XjB$tx`yWHe69|U%RDpVE0Dr{=ZjTU)2J2Js^YE;EpLJ`)z5S6JjYuOjDml0!_NLhaKVN zP*g{Rao?qh+hpZ@c||tq2Tzg~lo64=5uU&>oR*Ok@o`O(Vp7>GQz|J_{VaW^yLMZ0 zT<(LlMl?eXj@rAN?2fP>SLWf(Upz~b${fzA46JilbC|GuIT^ZRYd#X>CT*)l=UwJl zpA~jkZS=-%N4Rb}<|Z<01xM2>?!2LB$>Aggryq~cB%XxVpF5LP7`EkOIF z$85*SBwK!{XdY>3_4LKtnlqZV8!g>6>DZMdT#rDNp|W&aRooH?LxU%wr!TSd8CEhw z^oXT7$&V$2YO1;S_hOAPpY|C;A zCFtIkQ7~-uC1~GTp4tTqg+6its`or-U-yK>k=dkf*{TM#QG)iAAvXJcJJH|~Rj7C8k;IHLwq4yU+ z_c2-A;kv#gniD?VANG~PjFCTcl}|WZS!j9GLn^Wb>u}YkvewC#>Twrr@}q1 z6p-zcr=>5A94_Jb=EGiD;1wBLA@`@8QqA_C!$JkHX2~99Lsy8L)t#$xa|f$mR%g;*VTopD##bM02l;pL zZp>Dl4>YQ}wA}3#@5@EQV*mr*nm3z_cMA8oAFY;Se{8&2xZ3~7-%Mu0gV(nVwE&9=_m_|)&_V|sbvI&ME-ccAPX3{W<3E!;Yp2#KDs>owG_-}jf z@l8SauVAjgO3Oz2PdOk=`gh6r*f21vc^mb!uyeR%H+=+3u(zdF<``K%)9sv`y(G zIg`sXE;=p^G_`wK`pbBKRnq5xj%R-`IJT%sh@?YrQ}Y>=C>7A*C zE)#^$iHOU;82-AAG=-X2Fi8E!@H$o_sLvYzXo>F&Mq>zp6MxHq2n{mQepyDL2lzIa z&P#PUdCfNk&#;;>^laDvi2YQUM?GDWNXf_1@f)3&zc2=9Z}pX4w19f+KGH%Hg4tl( zz$Ia;!g9t(_rgyq-eg|4T(ZPo1-bqlF>yz!{=%Fu$js4bCQJnfV=j(KOY^e=Gfs7F^8}7-N<}nK)T>kf- z`}{jTZ!=nOdh1u8)Kr*+wj8=c_Uqbm?qpBZ6js1z&E!^3Wc)xKUg!bcBsB^FwAWSI zP~LMjl_S+uG6u8vXY}ruk<4#32`buXK7m#UXLPNSIqwz!dA7r?L;U4U zG}30W4SE0qHao1_eRkK5Bja=4V@>beH&1};P&YNMS|a0&-@d@x5634KZ^)}bN5<6s z_od@+LX*OZ`cNj!XWcg`^#7CY{O}hL(l3s|!Ipx1Ezu>%B%OQk3L===p@-$2@A3 zw;PZ`D;;Cysvjrr0B0-M#Y~RJmRcAoaBK8K3G)9nt0+` zNn~$kNWdCdPL0C-wnllRYqdf(v!Y$1(|{3c?A7*opWryp=Mmz1NN-glulPe8oCS|- zvmiV^)Y@~5lqQX!XQEUAa|T0p{I;=^60E$RVzrV&JISS*8@%@6*Qio*CGZYA}zQ5P7Um$gn1A3Dc6WR@%w?o`-7_$H22ygUcOP zYVwdzYII_h--kE!^)R)0X{`)@qi^@qyhR?*-L+pwrOJamDh}bd-Tz$6mFWFl(pMFw zBsB9Sg@mD^Vn9Tu{E-RJSD{72mdYRmxJZe+-QZM>PC}s<7_%_sU-@`=E|MKDjM7Yseoj`xm>(EbM#_y6&QkKmq*316oK|mO z`o(lznQ4f>WHd2~!d$)?4QZX5)GS}49iGdORU?YhLed_quUt))3r%Tx_iR)Ih6bkZ zSsb^`%fkElCokigur&(vVwyXG3H(-gO)U4R#B^3EzU3c5-!8sMaGlE;`b^-j;}bej z&^bt7gPfVkb+s_|OsIobr$5`gTVjH;6!BTfEA#DHn4*S;pGo`I-*LX~{OJ3+!Xr=Td=GY((+}I9wGpXd( zo&&f^iiIC4B zQMPu^br@H!xw5x7FBKYL z=|22lQi1bFhr+_KIBaPNu&yWy_Kqp@xbxa*aMZVu^KfC@|c>>Z@KaYf|skT^9+BWqi81C z8>mhq!PuMoVpBE8HLx$8*AXsVots%Xb%s=Zm&bJdVT!%detkI9ubigri&({1#9Y|D z6O=c|hji`W4%J4DGH!cwlwWG8am}N?uw|=mEvlUFbS>mEQ1Tq>pzbpteuFP-uQCC# zHhvArp9iCQzy&pnRy{iG>xux&m)Wbk z=VIcucpn^iG~L`jq$S=!?>8sJN77>NUOExtF=6_{+piW-GsWF#`yhv6tlktVzcPMv zZC!P4CgdB!@71eu;^p!aCxccOf08q0kafb<+B&3zk)pzPneM4>W*om6ll9tE2V3XL zdi;si9Vccw$hcS3FIHP%(ZoujFpAJCy&sWa6y{L_9wb_t>8 zXt-nC*yg5ts(AfHpoQF>5y#!y=;6{7m9J~)*Yd9*y_PjVF%}{3m7kcZtArLgaUGpgg@A=dn1z2NB<;{ZUZ@npuZ9 z1ZySBO- zUF1o^x*e}%Z4J;hkcaMljtw8CM~u)9BlJRmIY92ax7^b6)=W&H$y=iRJYTOMbk(n9DNelVrdrf_VLv02Kh{qIC&QMtm{ky zXKlHqdj<*+!(JD=tKL7Ion&9O^?!jUA6MXJt(q^}kX|F;c8(%ens-Vo=k@7{e(yUV z6g`nFVPQSl*JMr-gqY|#mYbt`qTnLaogR>osngpTIK1&irVFWy@Wi^^^59koWWAOS zw=6lKIZ0-~y3R_Hoy$;KveV1qTPPS!_&&`!Z%b=iLDfX2QFQ#BsFb3-vs5&pm+H>- zc;6H!%;3NvP+V2RgH?KEn#(|Ma@rE)ICQ!DqQr32y|Ys8MCX5}-W3G?_O>QsHmr}{ z*6oqP;Y}ZV#;`;u4Q(Upi@GcVCm?f4rkDhzkp{<5iW<=qlcM7$g$fh<>K@0gINXJ& zqf-_A^D-Oci}E7gkw*$iJnTw-ANQS%MjHDxfLNQ&0RK;uevl7(1pQN%Zz?v~P?_$v zJyfa*k+Bj^Me^AzTL+);=G=Gwpb2+VgT^i{8toYL6d<7lx!u)HG>28(L|wi!5l-ow z#e2`pdy<7s;xwjEA5gZ|nx$xw-VcXw1(?2hbU3OlS1FBVd?{&lB>aLwf`Z<(w86Sv ztl`itk>c8mEW(p8a_JbNAG%bU-_C)fSMFwkC=BMH@df+&y_Zn5$q&!zzdWkPhVEbb zLK(*|nn-z-bT{0ewr3NH=tC{w#yb$8xWGb*Kur{+F}aQApngus;mHF!8fS$gYp!SX zy3x*s3}$cUudY6Z9Jj_VE|-~wlz-GW(BJY@7l3rE`4~TDzezHS6P}=s{47~TZ7<9 zg2h}ua=DPOzVdyxPJA!L9p&`J;QE5L ztMd5_U@X)4lMaM@N;C3({L?3zN$l)L z^D8R#8_(dY&ay0|I8uU25?8*OX&jEvVmox4JwWO>tJ_pJlGUQt*e^c%g!HY-z6wCR<_mQIhS@rPG> z9%m2(IbqqILzPW2T%`{!cl}0MZ1R{agLI{f`$lX_I>amJY01n=^3AelYg!BH##p$& zzY;>!h$*%daWN6SSjYr{?1XMnouQ&{<@L*w=S^HIW#^9XsCT|!4|RWa!n=zMA}2ZG zk4H=M#BvnQ;s)r)})=hxp(WB4);e0~zA#Tok$()U~VYoA+%5qo*3s)jeW*na`l zV|}&r-E#)8GCTtzy;>LxQwryrwtk#txcf$LtycYkp>JS z4iBvx;Nz#*0SQnH6jPijuLig|OzjIuc_;+JFly#ldByRKMrmlM?N+I;PPHQ@G#ZFe~SOe%DEu0)7^?i<{8 z*h~Vl{mEf^Vakt)gC6FmZ61sCjm@+@teJ$-_^{P+tz^bUWM?f?2_oBS8~ zVZPRj?60DvI5_6|EROXvfCOSj|&5c+B@c)B;RU7A`fae0^)pvn)i2=_#fY7ko}s+3_^?G zqZN4?{$jfk85A4d6X1P!VeMXQP;ztPaV}YO;sD=NPL1@LR<+0=GLd`&A2<@ zaYNWQ8lhsjgIXZ)k@%0X*jy`~4a3`G7!C!mBQlRXCl=h{ zGiaX{z@##IKUOUEe;fOir_e96weQN(hJKV))1<^Frm4w*Z(iX7fq;?#mSeF7cZJ^A zAqp0gF%|(H7xX>$IV?1y%gY0|s;55t?IA_>%c&@#zX+Nv`tS?4+W0K8a4XoRyrTc_ zKzN01iW2H?dXCic^vx13@vnJ+LYIBiz|BK9L#I#FMPdJx9eoIElB$A10YOZ_USdn^ z=Aa66gW@1i$^VDP!uhTJX#(26q>l?6KpfXNuWVpZyC5F02au##7;FKi>5JE+UY1)# zRj0*vCeE-vD4z=@Rk6Ox>2B6O!!~=}qm8YjOZI=4G|MkFOP2Dg{AF_=vI6Yvb5?;3 zWYRDw;LiZ(HLLa+@|NUiz#a>c`<5usNLdxY&cDRAfkf?s{v$(pLR;w&e|Vu%VPNlYz;aYGJdI$SpY~!D zJO)%B7V@A^L9%(sk0+y7{B0xS*M`0C1#JssK)x2|G?uoHH}F5$&aZq8RE`<#}d5w{{m}cwrcw zH<><&TS8^3r&OO2pmw5sIWHH1MkAwyC}n~zt-~_(_QxQSW9?ChV32=aO8vz#KA-G4 z^Gpj6qX)ok?(f)+=US1oS>Rcfv9OdkMD;N*tRyzPm*05tXkY&dqV<`7EmS|8?ydL# zxa6S+Aeta(g`2$cY^4CO9|$mwUo=@lrPi7<=D%SONmYW-z!ax_(i+h&ICS>O4q)-1 z^}ONYzR@FI1*PloSnk+WHLOxq)d*qsc^tQs`yrEG$>I6+Y7`I~6z_QG)>~`RsgR3m?&_D%Jnvg=N3Xh=dyxY5`Fk0h z0Z0BRiD_hu1FVUzM6^ySdyPY z-uOEO-hDx%|LOSt%X`m>-}I`9>e6cJKPvAcc!1BQnpGC*Py%EK5Nk5LUe+N%tG+G2 zw7lQWKD=EKJ3Jb7T=S@I&0y{upy$b@iNHtS>%6(3U?BZ?u z`+Dnw^o2awg!jI@FZ$n$oxaqs03e9{1LqRrN1UgG3oz|$v8i1+b`@nD9hz-t@_^$TS)gHFHd(2$- z-fEf-P9j?@@r~H>!_z$p-S=HwZ*j3`^rgtI4{?=!>Ftco5=-dQ`zddzDb6>!Bdfq= zndvijFlt@^`L+l$gVn0Xz`h^*-3`Cr<&7UyuZ|LgAv*GjdFR2v?Nh!a5DKFO)o00= z6eCUUe2qJ~w0c}J?#bntQI8-8in=HAHLQ{-Cm#NbcD&emqrU z#Om_lk-%I2{K>;8a{W-HY0nKIK_ujT68J|$d{Wz12pcy?MSkAYI!bX9Dt0H4$M2r3 zPh0@oI)W_?&~=%j%UIG>xNfaZ&F*<`&U`7B82EXxCo8!T<_9@3FKa8CgCId!iwKVE zMUBlfRH13dfpk^NLCGt7aHDUl&-CW3I>a3vj?w(QdzM{n_{>efW-n>o37GG1ap#=Q zw30IZ#`dKRG;#!4KnV-LJ)!)N(--==-RuGyJwbs+o4te-T^*%e+}1n38k6^BK)k zRmty!+a1YfbD)$O-OM4~M$_nJ6v3y()73ti&E<2x#*&Q$4{c1DzH5{T*B_W7*Y-<6 z@}ZqE(%*LNT^xuGeEFMl3uX(PvI!WWN$zbD+_#UWYE;W-gX`pHb=Q?#+>m$YkSNc@rj|YxFs}6^@rz%zM%A)if!ZzB2Y1+N&c>9Xo{hE+~=# zk!@Fxx5IAstnU`fVUezCDWZ2Nc&c{I^O~S3AFB7&E;QoJi^8fZX6oLX34*K5XRU|Z zJ1>wKj+RGkWQ3aZR8|#ZzWuZwkS+e$n;llbf7sx$G@$0Jy7-#9t1_JXmQPCByZXUk z!w?{;*$NBDx{X>BIR72tcxL^MEl2k$&qW_I>Y=xQQQ$}for(X-z3j1=;!ga!7#il1 zl_2CJRP+o$z;=@K>m@6vk`K4pSa(AtQ{e~AuN|;i9N^&lVH)#b59rlGD6$(uZuhBQ zVy6b#o6%AqhpRKjMY{)8T}l8?!<)U>L-P34!f8RI`H@(Yz{_I`elFYr@2Z>|-EGEB zaIqqLrRE&}@YtplZI737p&F6h?h0ig{vwSmj7SOMVC2I^L7B<5(5d+>%I!>r&>*52 z{B_Umv~agVe>qsx4W6{*5bz+ zNjHtWM7%0=sV=7b?2e$6haM+H)_1urdNf?+!X^Vudm$_Df^?@!!Zs(Ty=CfYm?G>p zr_Eak_t<mJyi}%_cH5SIN!pceL^I2z3eT$_2P zG)_5XII8J@%C}cS0^V8~C$w3JMJU2I{&M|dM(}(<{7LM@*VuYu?In-95T^;pa--ep z6Dls(N(Af7#)S0zhET3^ZWZt}SCp{iQ2B{$iJivY!DO6PAxdD6m!N}FF*vUXWIv{oKU;%&DJs&zq@rl2UyKE8KV+B>{k|i4bJBNS zwKsPD(`mrK&shT5(S>~9*a7ePM!yiPg11WX%cBtqZiu+Hg@B#!a~06fxDtYq2)=y~ zdl?wj6dTt3e6!wD<(00;U;rj&Ymg7h)Ezdo?QTHm{WVWt4~I6FT|Bj93(kBcrs3V5 zh0<8@#vLaMiK5EQWUVDQ9v=k>%53^MdIkJM9)o7xwH7E(*!c%TX)bCq&z;sy>dA)~ zctg=1lyQd@2BSovyDfikjYht3BKxIrCiGj4y4NuRRNQUjeW20ZE#|8viPnfZ*2jHTa*OZ+|9<^=+kI21qI=kr}=kB)^~95sh*+GU#qf5iSAuq*}q(>t3#xu9`GH6 zPu=01N1;TO>k*!#6)O5>(JpC0+t!Dfm40uAh}|~^I$c=Q+kjBp{()&I#$fYNsnkfe zo$eVPlggZTD)>#U1JLZz^4f2mb7H`25!s*u&gRIS`&w&}-KD*U0=w5PhP&7yOJ4(N zu>Mi8IL%!ZWdrc1b_WG))PS@`vNSvF`$@HgBW1Br^#ZS;XlrJ(?~b0;n$$>arz$}4L<8p?RLaq)qQ<^{bc=txk3&y>2MyY_ z#ng|XssJ;PpiIEOohF0RC;FWSS8LF_7TaQrBH{YGUD5*y7bWs?uLhz&j-jz;-Ra0q z-0`mI%a84pV}YsFLgORvYU;5IevlDLdinbzGcY2b5DNE|!r0tdVHWb9Y#zE*5Qdx4 zSj}#A>P^ghi1iT)(Z2O}v?4>sjLyb0bRwdzaQ(8lndmbfD)qXq?@QOaXsxim= zv0BQNc593beE*&B1zszMO=jfr0Bt2Pz1W+cw5XtH zbB$msW}3X>pZkqVMMk#5byFdA1_AIGaJmYM4RM!XkKVFe2IlvOX45ZzCWGk!BB-O) z%Wo9Y9Tr>(K>@NcE7jK?UU1+${AGp{tI{s*9mFN?`&){n%SQfZM>q(1H9{r^D6cd> zt6iVB;j`1k=aPR`rp0PvsI~pvc~GKX%$moxQoDHhG@ym7qRvo)7Uch2Imqq$C^c-{!fg_Rt0&aH_Y>r;eH=p!ghUO(qbcrofubI0Ep zccFZX=wS>eIoJxtez8!Y4>}vw1_S3LJiGU)wlU&@DUB*}?B=`47(I&PHVR>Nxcq9} zd+wJ8e|vq}l*m5r$`72bg`)M4;-)A*6oGU3C@_WoJ7^Jp*Rs6W5pe&8-7Hi9P^AMR z6T<~29%`&)OjG-Nvy6sQu<$OH96s>)!LNlnmYa1#buL|Ybbh*<-hV_Qz&XRJaIL_k zM`=DqR`=mg4^}1dQf#ILr-Pl0QXQ@%t>mu{QXG8ci!b{X{VBEhXpWT+MhDx<_XgTn z_qbQLOKXTdW;a3St|6L6o2eEH;#Jka!{G+F(PHhmaKFVe_+ne7ThG2f<++7d|K@;E zXtoHYoP0*IyI75jszkeCwd|n5wK8*_y%GTYs)3P1Ycu|3MJ^hHugRt0+}OUp`s5vp z`t-WkR4oZV5w$^C9ke1-p049j@_pN$#(;1csj1k4AbQ^$Sg$K^C)&M(>|Q$UYBVVC z`Oz|Sx|P{AVpYSf>d#0Au5$)W<8CWi#4Bck`upTVi$&iB4G;grtY?naiUx*vpq zmlwbd^GFB54e1=>#D2zlwF$lSv9)=5a~ksp_KLHgj{PG*Sl<$u*$~&@$;Bbp6G*hB znB=%NDaA&_@7XuhU4T>YKRv-E`M}h1&NuU+;7rZNtmV>4rq$!RNImIr)8mr;BCzp} zU+%qC5*aDs{!${l<`PqQu7Z|o&)XOF?hS(7+A!8-oCr@m z?Z-Bv(g%h3#A1Si(!uzaXOeFHAn88iw!I%&*szk`_NTSRKT z=f*)uEI1_cyE+(HZM@&}Xx1{;yH8F}7TTnvrrLTfadS}e;U2|y>$A#1+ZXL}nbss; zXDWr;FG#R#sav{GcRP!>gcs9;cQ4$}Pwo-YMP&syHAebeVJ~Nn_P;T#FvQPltzvqB z7v&b6v$cxOvb!~5VV@F#)M5SL+0+f+br3;*`GkxNAt>;0`PIXJ9C;v!z5LpjOv(Z1-w}k6!pP9>2;B2cc9}OMJ zkbQQleS2W;frd|(&E|EWNY)TTDV~&F0n~ts<5ff)cd#o;;3a;Z<|KvPq3 z{^9QMvG0P~1om57#IA8%;MYA)Fg2f{1@*Ixg z&@-{u;M4iZ`viC2UGE1Ww+4Rd(rz{3Ghq1-asQURrx@8QrnoNcP#k9K`oH=(Y(+)J z-t}Fll0}0MiUz(cBn+MYE{xRGX)bYQ;%TUw*}ji`)qLW|6Wf-(J;7()I$lxDpYL4I z~Aui?qq6rzo z>lN_rSFgIs{yngclz=`cyxGUfv3KYZhVXCz{1E&srbvjeogbH3>fwL!=>bo z;F2oQZ}IXZVOZLOWNGS1y`0J-!lK~%Mxh79Y1vLto; zY*q6b-@&I%^Hag1vNSmJ{fEx8OXje4fls^b>A1=m-mJU!YOAfNw%Xs1yH13QrUu3M3Iy z%zrfaC{`cV-}9JwO7DgCI1k8&c|ekj*HA(xL3oDKc{$Zs>G?(Zn;ILp86}%{tOrgG zZ`hNpIzn$S+@oR^IW)WUdu5(rEYGcrs+-jLU*Sr}@mnzV>%>1@H|g&T7k6 z4lT1oJ{3MY-|1<*Y|gC{E(_76&w?*($KUVHdB7{z&r{Onbc|%?S>`mLG`xmP9Te+{ zyLps0Fl19C3gC3QFS_R+#qrOWsWK_GwYXsOP+~vt`_Nu=MLXleils~No=UU_&WS|P za(b?4U!fz@^0sP2a>pAzx1&X~i)--%;90wO--xymhA``nCI17dU=rY_Z@apz04P3) zS6Ckunk~|;Z648V{)(3lA&+gp1O0%G#p(UfuD#yQIj?1wYYQ2%g(H_`|7=HmGXcg3 z;J1Rp5-|U0=@lfj5FnQ3w-F|PXZ@7uk04uPo6`NXo@vlLc-mG6yRNRs&+y;_z~aRp z7zsdjocO)Efr_%RgOijTcfP$5ihIM7_3N2iW4B1uL%ncwas+bCS0jXrCS*R*1= zW4R?d`l?sh#{l{TRPtF|^xZ$}<{%{irKDm1aPxnzJ<2r+)zcHQNDMdw4VaHCl|m-= z-WDn{zCpcAEwg;Ll z*pF$|hd?FwwU+Fn&fyDlK;C%1Rws-74~Q={17LCi5pvS}wS>pNV(mi`h}_`(eE-V` z_f3R5DE@qAsqfq1(JOTyBXbS2wE*#$6y_aptnz>f>}} z+R=-eOzJ%AkV)XbrsKObMhwUsur@6AUB7zn)2p4Dic&Qq`FOX2OVbZHC}pP@A^Cgl zF(7FgB?W~ool$oFGJy{rd1a5zUYi409Zfj~a)ZQSdANR0%Km7Mvri`8TYr<5BXXlD zi3M|F?efhawP<853O0pIOE}Am^EFOO(!tlxN<=tydtL#si}THENVsF^Ys7!Y)`mm_ zei>4>0bb)j!1`NlPnP7`iaZO^fiurQs&5`u`JYy!U4;yM+YdO&!yKt@AH#9p?9g#$ zcxkIGIO|*0{+4ihCAy{q+FX*-++o?OvmTPUW9{&g8NNcom0QDW4GPTv0Ls$q2W27S zb3Q8KQ7WI1vF;$DQR@iS0q6jp?Ou_X*8amf$WHBLxydOQo*JX0-zS*ah z*i^aXBam*~@4*}vS9 zb{yKz{cYE@U0?$K@N9jte(E;p6tzFHia|qTTf!u7dfPU3n2U2yUI-WKZz;Ca7nctUr zd~lTUDxY7uG3k8%$&JuTsO_LAg$T3@+s13>f5+t)sc?`_@7UK|k|pm8r5aCf>zP!H zHq*Ox9;Bb;9ku(9&N?4LAid9#vtOD(50(YQZr>qE4IqJNO$GYIv9>Hodxp^=HU>K% zEebuqdDh2yaEZwa>gF{@^H;&9v8i&b*kVi+xE#MXyR*avABsscnj_+#ORk^@&x8<< z%sBAf%w*qA<0S`pR;CxX5qgp>AAb7AxeVI}C*~v-y11GCjIXvx`s4jV@IGhmj0{Il z0v%jVfQ8^!UH+!Q(8W$8y(HjCPyrMHROM$HLMKc^qLW*l$=O3*Q7g8rV@iQyu zBgo-#(C<&mj!M3e%xJcoG;Ypw)4R)l36&!x++#(YVRgQD%AZL0OX-j4Il#O#-NVJz zYPBhlj?CEvD6|eqo5QEj33@jZl-@=Uf)$q*f zvxD04B);A(tS`OVEy62;x&r^hOWU#S$!%iX#=HC4qk5Ms7wb-GLJzx<@o~ENX6m}Q z+ql`fcYw`KJ6JN|Jvv3%hge(|znTRIf5accFh=bB)KoNc>KD^$1V5ggotYV1R+f@Z z2#}HH2xny54Kc+HVm6sQVHXroxUrAJa{`f$H-)MD%89eiD9v7%66K(F#M?G-?bf-^ zk1kB=DxjX7ECh$-HDE>LHmg~)f~yaVa9Sh5u^c>(>GpnFbnxPeAdX&b+nAO(1w`Es zmWr)QX*PBs(q!F%HcdCg`}hzykI_y}7H~NzW9)+(T@V%ZfU#x}RWyiPx#`W3XhMXd z)Pl>cEVubKL;rA&yWW!3Vb$p>i|7?z!-v(0O_of=j;N1Vk-gdy)IQzfYVjDg%SS2Y zlejc(1<=rD^6=%3t5xYPCfZVkpBdv<3Lv1B$@m>!wi6T}>3&lPmMg@$84>Tin(*e5 zJrbWh#aEsizYrr!Y-z#`4lmp?F;a+lUs_+%cYw2pgY!&FZQLQW!aaxgy^!09bn=nH{r5*_v+DVj z$|q;H*R2NUnE2`3ZO!X8tvvWKURt!Ldepw(H8~I%M`9&EQ~&DgvXQ8lWno$mKEylk zXB|xL5F!k1P#)YJ{50`FYmo)%qns1@%MkAzS7ywUho=@10MB zNf~EOlL^ga2XX9Y+wT1}iA*^{WII@Hbd@#~HWdB^$)#}JloE==`W-b3s7sH8Gz$SP zZgJ>Q@8ZKj4X=ncQf6S48(d@mFhcbRM8{friN_C~msbcwLB zg$oGbz02}&x_7*`tQLyp8B5B*iqU6*d)V6*L>_xZ+jZcK~SDNxN*kfL27&oN7P4? zgGz41CpCbFWp)XgVa4|K#vrSp+~+S3``&FzitZ*WA$O8lgw~!MGGCi>srW-1yDucN zn3SO7XO)+qpR69r);0K)S12|qsxOn2A;2XjsLVaw*4P@INj5Uk5u;>IpOjM?EzNx^ z!$qNw3dQeARrRhu*XlTC%N^L}X+G zW-Qo}Wnko#^eVbZnR{{S4QL0XxS0SdG!Zsm#*m(Aih@47fMjb?1my z(anO>1*l^@Z`E9Rc`KDko{d5;Z_0ENXhHp$373Tx*huKe!+26sHcl64Pi=EPN zHadDg-L&2=ZdDNwO02Ew)gP$ey5bjA!|qtI9-I>onA;zj!}jyKt;9q$+M-A;@_&UD z9mKLH#&Yu{W|}%oP`OL!Uu^3YQuX(Dn8#q+!bp%+`7d#t3a2U#5db=3PwVnGAxNzQ!@tq zn0c#2C#^X&$h)oCc<+B+lajg@y(D1$rH3U`A1*@lO;QT2{5q&cLc18V)6(P zrx5o=A+SzMH@Kt7z?T@#^M|0ciX&osGxD946I=&}ycH!@%MMLZOPETJ>gN`bml|U~ z`+3^2gU71I?6yyYuEABS&eY&=(sa^D&FWr}=@M#?lN`Pbv{0Pln%D4qKe>?S(i^Wh z-^7Zxw1;Nj$1|D6t3!dfjukh)KaC5I8;>o>Yb^f0hEDdKGX6U}q2l#h4EW{Qx3GcM zZXnL**ZM8RSZ}90qsH_tqnk~8KTUrvH26X7-5uVmdZ%Dm&LpL&;(w|QYxCR zz>|m$stzbK5l5x@H2n2NBps%x3u4fKV9<}))+L3Vjwpra>GfEmYD7V7s9%5RWPrU3 zos`I)VHB-pyH#jQnB*{JlSEq0NB9?zWs=2!cY0 zu_CA)+vhbg;785pAd`z-xK+FM#VJ6@ zTl;_P{R&6Y8xK{F-DEs3{r(W4XcFquVVt8?pTXVe13%4H+1Ew7IsUIO&XpWAaa1Kk zn>r((t&@C1oi{yyG%7Z&(S8YcL_w-qNM(w!f_`n}&t=}fZD4L4-MmEU<+;YUB7DF& zK$=CWq4(V26V2!8%%d`tBEbfk_JS2%>y2gsn>RMHHUfJ|@O_V5fFl8$r7oZLa42T0 z5(AqU+Vng;Q5Y+FIaP!$w{6OIw|3Dj0?AbL`7E2&pm)30&}9H?of9<893A$vPmey` z^#4%y7En=j-5;-JmpxbPB={10o;-(j^R?(nBkafOO~3jf8Y4Dcv1@ z*Y|mR-uSQet?yfRv53pe+2`z-bNBx3-(hke^GxPtAuXr2aT)cj*T0s?qm)l1-!u?` ze66IpY(7@-77%kZt=AN19nzWJ-(Duw#=Zu@Q-J|mTCJ{pPTzCfA{_BU=v==Fbt*q};e%odJmY?y$ ziR2&t=ejV)shWTIbP4?mXr+qE@eOCWAF)~*HO%$S<@^B1gMA;6S5EE?_D-eW_$M>y z7yLWdVH&X7L_kh~Dqn-MzT1tv&uMhNDBE01OrldQzWqPxu#DT~r&EEM*;Zqfg^>!E zYYv_cdxIi_N|%^?>3lJIS-a3P<2+D<7PxVC=Vd6=Xuiwd{t85m0mtZ^FW$UhUrd!IQ0xjNY1 z9du8!?;J_w8ygq~D*fe65nj;rqZ7?aodoAcba)hFe+zb@AHm`aAHqCgB5Ot2k4J9o z1JQoG91bqIcge%?Mza1fu@OE^EmFAkpViRKk>Pxbn*a#^8DFJhiV_%FC{}-PPG(`ulH<{k!`iTH9?WiO6EfJ}Lq>EPTkrK;*)zg7x@gQO(0qOcB0ih=m zx+j{|6OSKpw(;mP1-L=#WH_eo!*(i&a)*QSo^%WBqJEqtE{xB{dHe}WfWk@(@;OzYa0Pezzr_(@|Aj19d>&=p8ag21Gs&TBkt!x2sb&etbLKsJ9OuzAz0PS*=@eW_};|2*hN^oWQ_) zi5i6=jmX@Vu`oh!^^F)tX@ammSeq69Djem4VXCYWJQ@@J_;}!wI?}JE>(Z6bbm+YZ z4>1-7rMoy2Eaqcgz=R_r_8|o3!!yEJp5>5#ifT~BaA>PhpVF#@BycLi3_6(Xr<85L zTXBO-32gFnS|!J)~~T&DhW# zH|ES}bph*#mtq%rB)6%R1Vh%iwxz5d)}oPN50C62A)#l8d0c<4Jj-6sna%wuwJ^R} z$;T(iWtEM~>~alIp+rin-jqrAc2Nsb$4&yV!e?4eW;XTh9_fZF{gonixH13IhV*B( zlgr9uAMz3=GTn|HHrXFTbct(K$X7$F)87Cv*2TjSqDil~esl3XL7d9pC8=g^74)lE z*(ltO=_GD!<}+q=rvJL+-{Pnz_a^R%-Ha>%DkYd4o#Euw@A}SMHZxldB>4JkSsY9L z$zwnWfW!oP7DG=!TQBgGtdGFay~G)S;^NZU&X*DDaeD87=v@(V2-79ITMuV<)B{siYvB-An`4GMV3gBpS>XdOt &FEcBJCv@6cM{u2d5Q`P+ zu-5S%3`Z(axUp#wz_7Ahj;m~yCTL;GW4X~SyrF3Z9EC;$N3sTv%HJ&WJ)LxtpY&IB zc<1BD7_iR~!pN3m-<|iH@S{I>`*h?ok0G!T|1@M>afld(p5>z{NZ-#TmmJ3SfN)3* zW;|9Y19Mnn+s20_Y0l(7)RyPBN}%LkP&0o^(rzXd*4tT1NSO4{$x9F?{!_nBZuctN zJs5vhq=?Aljt`Q)XrG^lk@Qc?tuSUV;cDEx_Fl79j{l*!pD!rc6}axlYc|@X)nru= zA^$FtQ?DzflALs#hq{Hj?Q^k*EEBcHii^78+Tf&0x$24)dVDBb5}ySRS&2k{lSQ6G zUB5wH4pF$JCv6Mxf}Al7ubq-WILPRA= z6u8jJ81e)8oMKQZV$uSupqPAT@7joxu;9({E1bVnF*z!lL&E0#r=&#N{Re{$2M!l2 zI@kaoo=;;(5FML3Py~{GmqWh(HJt!*aEhsByg2=QF zej2)A)uM>>E-+8mUPEv4*a=P$dw6OKdSR0(?qH?O!B)d~-CLo6N`8FMy6L-l2kI8Z zwH-v6Xz*<6II;{gLUwlrv^-gRS?v>-2vx;E=0jnI-5$h7 z#XO!$atm#1Mm@YVa^_nI4^QxirVSMAxAKN~1JE=cu0ou+ByC1ka(^d70plAeGYij% z%*|eo>s3C4u_3A-Q^EhZ~1+l-7et?`c*|Z`6%>F7fM`a zcrFaJ0wITdE)LzI9wVjTdiYDSn~vm#q_tgKHgaQxAaXJu~^xW7A*|mF9)W3hg58vILp=e(TCST6)7<)dS59zbM&`X zjA$v+ZInn{0qOayI8KC1iLJNP`EJm8dhd{vmU>G$9-bShpEClh^n+(Hwr8 z=eRQT@)nRwlwwzJoQ{jcawV^4IEw#uMO=_$e!66_+Zws{$b-ZpUwG~vwaUifOg^VT zHh6qpSkU9GDBAwP%& zDI&%Uwv&;nCU!u<*Ez|sC5e8=CmDI-srvFw*Jjb~?Vjh(AT+Y%!0pB8&o|P~ZUwXx z;*kP26fMnTeqj|aZ08R|-nhM}=g+yGbq1%2;pbLcJ@rNN)i3OQOJS>dV&!HxNjqGo zSps`F>)ajP$0um25g5(aymK=d{dioe$Me~^aO!YO1|0*j)?q(up%(~-nPtn@IC8;b z+AWB%i!c1ibY_hERu|MiCS&mGu4*3-N07+*v%dc*jBGXzlhnJ6aE|*@!%Hk+>j|cz z>ZQo`;Ib{KyOXxF$q}UN&wh;qOEE~4ASQOfG1`BVpff}hOlRp zc1zLPzL*^0`;JT8LDJJc$Y@NP#1xESWqEV;k-TDi_6M41EZhZKLn^Z3QjOoOkZg}3n-%%dDYN??9gTI8~_b4G^3;o0P8d#Izu_xeEC?Yg~& zAO3p4JAK=k5JlmDK$zs7j0sDTy>JHsOdzXR+G9O8(B0KkxmP}b6Z^7OM*rn}r%_?^ z%cUzGpKiQP>enlhV_qK8--{I{lBD)qJN)Up2D)EJ^@v8o-F}(6-)Gnd7gcVM~1wxdM@n&yA>gInWVl#AKI|tS9;ft4^7ly@y4dF_ouW5DHYuat6EI8S#H`)D>#_bH`Se9|?Q;y(VTaK>I z%~T%IGXj-KadKilsFYiz`Ei%XWk^~2`z{x9YHbOfrh_|=h$u-^zNDk#_oIoS0x${vxLcY5C~>w_$Ay0E)9%ytm$42UgcMG(&qg(so(*p)teMn%S>wrz^n1X{*7xpt z0FRWGD8w^zzfeB9a`Wo%Cowdu$loN%Rj4wN{`YqI_isT8I00Yhv+Yc!5t4%J#7eD- zKet%u!M0Szm_QPXEEXt(dW*5jko>7FGX8#GRzTs1CP9&j03vyY;a6B0J15^X{$qH0 zeU=4pv-Ya_0CbTQIJh|F>+IL}an`J5E-P=0ju++Kt#Q`}Q^K*A1lo43#oMPK5VyM;8-g#M#twP`PD<24xb)ZyvErCn$tk7eAG`zTeY{C?%k% z1zFk*xA7v1Qee2(e$SLq^&6;kYNdtWp5a;Ui$>n&E2+)}6-*D2CmoPvnF%eI7e%&ME06u&)rfu*5%>9FG1>1I#@LL_&TPk92a(S_Y3vhd^ zBdDSg9ul_TU+&NdVYwq6)^AAvs0M?Oq?NGMFb2L& z|NWQYTa@e-e5lbfEp_W^2N{JGk4O-&D$V=HJlKf=EBY@Bme)@4@fluUSw3F(N6Cv9=c-IvqU>}UuW?L`C*gGe<^HG^ z<8bjvhV5Yx#90+hCA2<`!nRTY=zPetZ{I*0GBS=?eL{4uDDGkUhOk@s+(D$jZg+~u z{mR8l_iIFP4P7{t#fGbU8jo=f)ZYEs5Nvt3GR7|336cpoM$H4+tvLE7+z}Y3Xx)oT zsOXE~*9q15A_^ovdmAY_L5UiI)wblZ9HuHX*DDT+ zKL{^+bXhBX(ZSpQ3W06z$UN95LY=A25W60)jILSV5vol(E@zY6*(+L5k=Af8?>M`; z23@IJB@f3=z|FlwPWw^@QFFkiYOHnKMx>dMNTHH+Rd@0wW4DAQmHmW%sO}U`qhefg!66(mI)@7*cbEulah<# zk61G_6Fw}1SjoZ{FYf`)ZCXRqlWH@Pz5WKXHQD`M%o!YExVduPG#r;9;;<-!aHg>J zd@Rqp*X{ohbi~#%eL|Cl+&y(pD%&jv@%uWt=$iGoOASVuUwr@h{1HDI&*=yi!F6?# zoEp(3|SQi0pfUqPZnfRs&uA`ne`~m)aPyUS!;1gmVqo+TJ`wjEJh(oeD zA!b!gY;-cYYro@Ik&cg4OD7@GwOV+@8JaR!_}x>5`Sj%sa3#~K?`?1i{Rpb1qIKm{ z%}JMa9J6Jrw=~^9ThNEN@+iMX=Z2r&yUD!*C4ruJVm8H4%g;o7QAaPV^htcjKw~ZZ zPT-zB=Ea^1rakNaO0kLkmlCu-6B_!ri5x`WpLYvZw{GsnlmmkCk8N+RQ%#k`@9Shm z1N3u_lUWBUdMC*_k2YGc*?wZ0#hz+eEa3WCq{lbwl_gn}owu$ITEshSQgIfMR{!d` z>@m>1aN437hH8B!?K~cTygkH)NJpc?q8oEZowG+R16Rfkq{mo1g|X!i53H(~C-I8} zR0PNTlwO->FxM)tDZ_Z?^g@J@FrlA6#@1{=ivq4s{i(u1N0~O+0AoB@zzuc_OMJKJ zxC>`KB+vMjL$Guqb8!xAyCs2xM6RXXT&;1tyQsUT10<$3Ky6t!jf#t&xXheSRS-4F z&j{tGqfDQVJ_iqN1|{>|JB5XrxpU-_#;5lpjL<8OnVxy_D}0N58ew|Kg==nf{dSP7 zU5cxud-^Aw+-f0w7xzB6_l%zROgJKnh*>58CdJU#h0yeq%3Xw!2HG*>%|= zJl3Wp*b?zH;p=I*UGVNt>1=i@iVKHmj=T*|0%yY&u zmq+9H&XtfyB+YsWn(x*b#@q?!wty|^O2_ncHbj&i>O#u!%aWjmraGi7@AFu^D!W{A zVBs;JKIx~+alR4_Td%T4jO>;1dZsjQPfoDWaEx*|X@~8z;N3Qov>iHwO}4=r;*c^> zdpkR5ed7WmVky-RBoR8s`1NTwNHrzN7rj~@L5ONr`J!v;3pzhAk+Px1F~Owt=EOA z*2XHTHfNBE53}o&4RFVUbpXZ5Og9Kc_ zNqW5cToE-mb!X`cgniHfJcsVujScPBFa|YfWg>;`QPt2iVkvA*qy*gmf&A6SvCJr= z2vyOCvDBLjPi>C+cL^39cGY=_6WZUAE)RXnzi6%y91jl(Fo%dL*UJ|9O%_OmSAH3t zPlzKw)gV*}4NQCbbw`tTdOyjs498N1J&5@dwWjWYe%xBO4G7!`ik#&hVaw&76)}(^foPInx8-qUM&Zyc|QPR#6 zu$@PrV)dPNa<87CSI_Z*=bWWi%3M^=ZgpDBYj>`M>p@LT&=6cQ+T&8qDlnGAZWG~9 z5I&`ioABZJVi5PS{YgJmp4~7dKG@n?C;{i}j_e}DqRhz$(GG(Q=jpc35DcBY>!g13 z9bO&9%v$Zh?gRS+8OKo=g)WFeUI9O*?H0$zu&l0os`E1|JUPKL8|J-DG z@J-BNW7)^fV44%-=*9ctyO^ctT{A*F^p;6y8LW5JWz$X?J{+C7A)68!K-FU?**a|# z((XcT_y)C1u!ykqwnzE~S&b+6@@dn75~&$}&7tSpeqS#vEoo7Po#bg(k@ql>=YYYr z^(07whg%}CM)`eNMOIQaJ03PkQxI88OxK#3``{bRss6ZdyL%w0Mw43FWl!py>Q=-Yi_@zH)`2YH*o<_tN=JGMg`A% z+gKkH`+s7zUkvpFdVURE2vHkw(_dLuQ{|)&y6{~y=7lBX1wQ;pez+MOdbH6&_ z&+c2mEN!GPcEujfs1JyiG7hU;a{msJ6^o~cuz(P`FB&$GmC!@y4j(pMAbC|X?jC2P z9x`{!?VCBUv?ulh&d}B9#t~4fb%wN+iZUrJf<3jxSgJ*h%eQY^wazrZiFoejuvczO zVLMjE8B+F|7oio0j8(E7tz4b>o;3U#VLS77x4F%rIBO?WWJT*_kK^rLVNb@mk4kzT zuR8NCxO?^Iw~OcfE=KiK?X5%40!4TtpGiiA&jp>6=2Ya!Cfh}X8W^#RtPZv%0kLkV zd1wr!56WUFx@c^OO#a{#S3w+a@VY;u+DLGi+LfBnp{G3WfHZM^As~j&DKNpB$vvz; zto{h73vT->udQoFF=8V5I4sj#IgnllyJ60!rX-nt{_Z06>Y$Kwn4ey$1tU?vL_-{qT)it z+-UM=;ROs7U>DF}XdxJ$PKUIxu8tik_AF1BJEmti6CMz6IFU9$#`DWMY2j5eKG1NO zJLu`M1|NP9m$TE7&r5lN*{b#;`0=gaH_;k1Lc!;)qY7=92N!oh+%43Q9O2Q?jr0J^D`$3p>RerG>NCvuIe=z zjKJ=5VyzcxXft47>>=h(1?;I&wDL;3>VB4#by}ILn2`=Zhk;mbVO^w#UI}UnI~k?} zV+DJ{AnYBrkda`e(QAz{iq8E+Scl@80$GM--@y_otj%`79QpPaNdj_!TFo&fHdOKE zg~3ioR@$4yIkt{ReKIUo_la6tK2;}m+w|K?^a8AaMj(bPEmM_2v;-OPlb833D&D*` zkRdva7cNW)!vXdmjNdq@_%FNaX?*o#1Jk1D#zrv8L^~ev^+6xASCX7F7Lo!eEre#+ z!PD15R_Y#Nm-n1P%V@hA+PQY5wtJ*FrG~f9a%V0>BE-UYCQEg^)~_HA-EV+BaulrI z4#N7i4janwcq9fs`lX}yAQ-yjLfPtM*FoZoiP3Th)GTq(S-Y%p2cxLtT~H70u{BL= zEP1K=Ct>eMoNDSrI^hV)+C)ASuE9g(qs$(lG!28qNu;j4+PZm{cn^3`!VYB$W!bCq z3(;q9Ht6Hke^VQaR5_zg{4v;f3xn`I`y2>~{kWx_jP%ImWFP3KXxkDtBrlO-KYiu6 z-qE{Vin?x);C4Db0H(iPtbYJ30oH52Tg`h#l}(Vj$_rx;%Y7%U*B1-O5hv2h2oh_F zFs#EljVY2&GFI*%+1}r72V`fX9h+<=6krUs-#Zqp04D2Z&C3q2;S9?hfmL?@(c~T& zF=NRwZ*ut`EWq{kAvKlLhYfcFA2IMGKf;LLS1v6#+@egy7jw8I|3GYt)TdNaP(DQ} zj+2<%+cE9>c(Edmt%*XmJ&Gz#WONy*!e-VA&oLyJ=bS>{eWP!W369O6Ti@wzv$yi? zs-Q{$z~koKs;u!jT>H2p6Bo zdhqdrY2lO^BSkSmUh%12#0Ll58Oju-q-uNB+Q@O$jy1eHt=uQrO+x!y`Aq5yLc_JKhd%m3Gii<^M%E&|a zKNsYY*^?2J$!Qu{b2WlUA^59hsVR)F?Y<^0dg$9v^~mg)f|M-FW=M>2T;h0R;1|*| zTz}f7=ul}D0mL)La!i~wwDVt(X0Cs*XLLE+Ss78xqs{xi z_k`|q?VnEWaDoGMs?_C6ND$&Hd@6`Dk}ONxF7jn36sk} z+}>%!6rb1&bEYB^jNaIfos>ejELj<-dprhYTUw#qEL~oGYbOO3$)leFAj}FM$e)_& za3)EsEV{;IN-dTL4h(ng(NpZ81CKmnO4oInl{J>;TPGwJ;;mSb^xv0AbNm8Gi6#`$ z89!}-2exKrK2}J4+&N#A}lO(96i2_-zp1mg)H&10nwPI*-OrJ-QwR-qcWTdtP z!5Fl}6BGv8QnJMKneHdtJM&rT>DdZqF#r-7oP{3ju^*uc0jXa`z1Psg3 z3}enXtk!oG^KNT!7(&z>H;WB!;3YVSWiai^`QC_&xE{@X<@~@qn>Y%()nvmIdoK(lUCzzNl;@f5WcrfXSQW0D(PY$7dmwBu0=p@IrNt`4JOU$l?JrsY=UyX80a+fPB(s8Ah($t8JnCrqZ6cU#j6yM ziTl7G^Eg*^pj$7r@0i1#YszA&?c$>JpmA*`O^4HZ`|-`ziuPW<_qBnzl-+~HjKZtS zRG1ZnWshx=%+>Vy7pfI=A)XN)2MivT9|S;lWLcIjARzq|!S5{=d!D|wi^13KThH3jdOAqyz3etgJV(5NMTBe4p*At36NJbot|c+Bs0qtVVB5 zN$NhVXQ+L^J$UhGpHs4ZKXh6mlAOx({pFtM{>D+hxw)&U?T&E9b1T$pw2!l8O`e&tUs*tIpFMnulbxBXV$sUY)qZC?v0v9K z#Sp6WUq#J7B~F|}a`j&_k6^&58A=rG{;&4vu~cqf*6-$BEyX6>15jkpwNlSgVhY<% zFCo?k|769p|CJS&;lL+FkqP@<-;775c&wEM_W@GbyhD*iim`D)N#`eeIo&d6dll zo37-H_&RxtDmKQUs%&H0De+htB3|g*_{wS78o`k;M=)9+kOMmwC9^gjHo8KsR9*NrIK;05c(9PA9DgUo3(9)jwPNmWcDV6sj(9$LP`8R&D6KcS!l@h>2|`> zTl5?yr|=6a@4KU_OTL+Ki0C1mi;>p3h6x9pzNvbP3X#6)X1?&)^7-)-9>Ut|hu}$S zeOve5CX!Bj-S|UlHKPMut4~)86-4B_mk`l#$Z4F{uwtInvD$UWbJE1bs{^G^`0=@+ z&+s%KYjCL{;6gxAcf-IERgPa10 z6J*jL$JJYGY}pC2gpy5k6|5NuvxTX9eRR*0?ci?J++T|Z-eCh}V;<=>wN%+F(dB80 zw3t7_kK#Ht4TuxtGQzm++Vcl8;mKbgN=}^PAPYMo=Uns)KzGxw`n8*52 z%c1N}lk&1_+C|_!nw;bZI%~y3KL<-@K{^68}+Sytqn~&04sd640;FCg_^Qb*jA}w$$S?Hhvpn&eoW&Tuy8HbY~5B zs(xaJs*=x**4V_bXU%oW()B3Ca)vFUl|DBruq8=o;`k;;SlK3I*~9wGE+TT_L7W7x zy{+(&!;>|&(%|>2b~@x#*8)H_gW+&{+e>00+rDajQQtENCYeI!k))Bv+27c=_UJLy zaz#bUW%l*YhNa@-u2&nTK1-@gbD^V$*5u}MBWt!oKHI^}lIa6Zs#0TL5-x26A4|gd zS_ntfUVF`}g^`6BE?>^}Z-Sc&$jeNlP++tc#sTS0YX4rK*={*~rO68=c}G#)I?cqZ?@Qaj zL?&Cu=DTFGiv5xP4o!X9Yrv@ANcsN+-H4CmUn|Vd*$96pBmi*xN^|rt&KE!juhsoV zJ=E+>IdrNCJ<9*=Aiwv!Rrg+08W9TP?;eoz1SI9#tA~A1&Vt6zHM2#WA9qix1rs~l zTMvqx$NXmpUW?_X=Y!3Z(QV`wbP_anP7*BBns>Uh6GPv5HXYQZem|{h-?dg~Zln>K z-@Ta`|B6!DNJ6`t)>ux3yQs?VW{5%3@9O3HmI~Sj^|~R|CyI8g=ViZ?Pj+P^CbhD0 zjJb*Jf!(OoG%ecTXj6_DW3tc5=YCbZ%W{OpV<{)p7QN>w#E~}?1V@3nQhWR(ii|>` zTnQO=@q*%{(EbdKOFpp#K92W&iAd&yf}PZv7>u~*5_3=1 zwqiM;tJj6LF9ZjNy%HYYE!X`Q(r4?W0O>aJ??a7SIse8hR4ojoqBML5J={;}l@2F{92*Pc9hBwzo^q;OqrAAhJ)Yk+CJ z_`P2aTPpiZ&#^KyZlQ`L5zL#zFAQI2{VPHm_M`~9S`nxNh9PW4z^}-um2vo)bdZW; z@o~I(utN7dA5q2CqVF3TN@8$Md~~WViY}AKR%U#4$~uDtP6~cRUlj$Jct(^CWhf$k zx=G^HFpyK{&XZ#K^Y2@Fauswkh8pu@OwcARX1lNPrfDz7t}}yzavlytU>`5@>OUb| zNzER?8nZn&5pm1GU$Pxj1=zo672uc>CQ<3|q~c~vM9*ar1ya{)znMtgzQ)qGlL^du_GivxA2 z+^>^eA<#qW>y@MKTk)?A%XL2lRJZBFSC*a}Z%l^95A%X&&&yOWvc7b5R+2^z+p*flvR&fGxm6;kU1tQ}cYD(@*0i%?R2C>SjM;=+{ zRORJsl@Zpe!wMFflw;oXIkp@f&&FRxxj0^`hD$tPVeR_h;rn)H%;`cxHkshD@we;1 zMYz1rNCNWJh+j;Fg^1GyUqWhbb^jW7T-Y~{?9B8>zK@ow5(l*B#llTpzFwzI!UA>I zD<7gu0$bWUu9|U=0%+dWd90CLdScNM{!TK)sb0;aR1-_%1?vfI_Qgk6@#cZ5rQbP% z7i96bk&E_$5SCGDD?Sqs6DK%s&>2z0cdDOKH@xR%FMY;eb0|kIp(J{o-t>jO&*M8m|3hOOe#To>JY!y7xb* zJstp!|7Z7v9hEU7=zhmMfDc%k8B`jBIvLSmQ*xe!hS>Q~(2orW>Pzm=;`vckPR zyN-YIB5En#BsR|iWF-gkQl8CrU_WhiOj!5aPu=ba~s0Nz4hGjz0^^@%2> z$upG9TPASLO4|M4^`+PkK_LPB`b34)3^0+gi^>n5))VX1e<8ScbwU-oq^aP7R$+cA zj9pp|EI(xIypQF|KdwLZ6^x9L#g9FHwfCw;F>SPTJDg*=YpN_Qx9;=hq`AWGsT<>` zhzw#{Y;T^T=~(^_hpt0qZ}nRqk!r;FDr8S6JdVBQbXz}aLN0NUJkN7eB#dg;eMMi_ zX{weqB$CqCqwS_Me2M|3$2n7`+J~>c;_&KH9VGQrE|S|q?J}O z_yJK^!Wrgj_IX`SDRcH}{7PX*MAu!Spt^LDsf6>ECJX2u4c~HqO>G(LWeZPSf7%z3bW<$^uWK|l zHz4>THC!P5C#QR3%0St@+BKUrTD&zZIfb#>*jcqH|2!$Y`$1*^eWp-s4xFYD-6?Zi{+#iQUj1l8~qr@~N)=l<{dfAT#&2<%xOLSnkpS*3(-?*9$4P z^V{t6wvD}J7mFKeu72#K7@iH@5#AA^2mgC9vZALn-oDy9`ji{IT6!=P9F?k!u9b{z z7_DU$u{}r~$>)t2m9o{-{`Ltu=fAO3Yp!!=J)Cl?p~HyIsDt~>259E{f8V2k;S0a} ziF6XG#ZTj_w|Cai)9PnKwDeNdU8^bbzi7B{OK|98Tj5~;YLBaUuR#iQbGam+l?Xc< zU_m`Bd4?aIA(D~ek(_Tm4fQXM+~&T1&&xFQj25U%{bps4{S5kUc*UARo9+6>^#{gR zS`+}$jY@3R(pFcuJ6nUhvblLlc4Y1eDr-jwi|u|Mj-7n%kWB?)gwp$JNvGU@68Mr;FyRy1j*9 zlkus$Y=^pY3Ve^KMM)DrHe>wd&-(IRw#b~HxCmZzToxb=r^F}O5%6;;By2Ivp?q#n zj9ZjrD^4=#;BK?$s#WG{l5*)+r@7nL3v^G_=*Qlrdw>m|@HOwVblP>tBh+}9Jsz;g z`rcpeNn3V3bNor=2}(UM7p=U*IXOFRUi?)8IK<%o`s4AbANAHY0O-aF0a@WFvAb_^@?RO6agFD0R`pMMv`*mAw7^SG8>iJS>i0u^^h z8q(iMrELOa?{j+}t69CUrGy-%f`82vM1SjDqsOt>``^Bne*yJ8aH>w{^f8Fqwz?DH$TM$G8TRL`%1c_0^@d02PN3cIY^ z-8v79UUQJ#9;sitTRhyHTJTKs$veXkz58f>bxiB}OV}K*b;Duq_S2pxK`JuMbIgYT z`jFGV|$0W@Nx~U%_@6w?7%6|NM^gjr{AK@_b{q z!~A{{m$Ss)zpAr)WQLAL|M0?)Q~&NF>QVYagwpYwA_4yc0ARN^^D9yvIB~vuEp{?e zh5xI)a!*<8hNSyPOmPvvy}dU-=Q;qAp5gG{{AY0rpTwX3;ic*Q&12M?;Y-m> z@bxz#9sxxJpP%5y{F92!OO&*N-{^<#q`pq-uvqkaYEU>NeDNsKG z5{7!S>Q2=U5~l1|rfcnb4^cmSBkznITL=0ee5zSM3RDkz&(UKX7NXzp1P`S=Af{IA7wlSVlYGtp!j3!fo@ zFRcT$*Rue6w%UJK@eY44@gY6ky?}eawMoybKUB^2Xlkh|NhcC<)hF3>D@SoRg&aUs zFlA@r2$~v58ZW4aNP$`c$P&{#8@G8sY~X!7QpWgPT0j&x@)>6xAJszS5787L$@kFT z%ioE)cK;uo!#@mu^v{8%K!%$Wj_EWh`}Dk|`I(wH}^sPc6xop0t!#S$GG7$uHdcJm*Re*7j|`vD2(MgOC_ zf5dKE0Tu{2A*&~vwqEO>uy8^DTT&VIcq>f5Pij!$;4))?(`>nc1+Ifb2Typp+Z{9D z3UMA*RnboiJREoiF_pvW<{sK+g-f)GCZUyMar8q>-5;}Xsr1ZYpMCsD|Ne8)%cVGE zMy`>OWTyR3g^GxOb3tAFn(-t)sOuL~6MEOMvDq?yJUHgW>dKPda_%yc?(Vq%JR@T> zdGr1X6#gK`Tv0ktOrk#h4az6102NknIu^m;dO#Vv^cXp&TgUsPG7D1-b5iaK+l0?d zE4hMwFN&C+O_~`dZnYl_G~lJd7uQ~&j1brRGM$j{6CW0;s}f_{`q$SVm#TO;&2zH+Ow2HDod zJ?e21o274Apq9K!-zpg^gAwR?a{}-jw7tM6x`8}@H`%;zGuphrsU_tlcmBp^v=L}q zGs&@nSd@(qbBX;+K0wrugP4F$X0Seq{w4B1(yu2!DLm5q;n55Ai8kwn9#iY%Vf=No zKpL2>j+>@65vt{rdlcYlK7B$8TeT(ymVo9(fxNY%#q%J4qj&qM4R6jBGjPt%=RBQv zzYwN-kEDuT2bccrE?JnJ-8+9c`Qc}-RT)J<-19$+RsaUxIl;N5fb)OWafU^m%{}3Y zk^+aqwx+W8?(Y{=E8n9i)*TVw4SFJZ-=gfX1%$ZXg%#uEg)VyK4|=kN{$^UE$u|icVuLjh=%2x!d;muOfPVZxig+FU6Vx!LQg zuV#PQSKBNzTMcgw%qvs1GfU9f*W8tq~|`arDe=bh`Z zMOiNXn{Qdyi@=xb>(N`Ws*Am;cZdD?)w>5FTf_>iIK(2l@=6pG6pyoK)k*%CFnazzvQNX=a86TdRo3O&oYQDLlTXt}zx$l}Df-GW`rqR{ z`9}N%z356yIiWYjf(4X6E$$-Rt1^R?2I?w-F6}2oAu@|a8Ltnmp z;W)?Qy;q*za1&UUIDam0JOT~hoB50l@=?$ecU9b2xDIQ%I2iC~^wvHNmE4WY2TmZp zGM8QXW1bK|+pB+k`;UNj#B4fu>_+&2X%^Q=iB!@4<-QP4z+oSq4z*uBDzd<%r zI+Qlh4Dg8?`cHd*);JzecQL63p$Hc_&e#X`D`(yW-Safimt>o;*KQjFt@+^b@1dU#3W!#f9ls_W!l2ZWA z6mgGtn%|+wd=rJAU=4!dveMNf30}p#vHF$#1o@w4h>P9qos(zr~MNK zzWsLe7bD<5bNUlu)hYd?`-&!}dZE40T;_=CKbk^>0V8P#xPefrpZ?1V28r7$I>ZHK z8)2pCk4U|~&wSW9W-9NtC>19JTX)F3D21{FT+U5Dq9_)W)UP7?I|PTL`cKJMPy5W` z{F%pJ0)3ps`p5irw=-#G@&CGy-~=qBQZDxP$;S#{g%TLxU7pRf$lpqg6DjAoXd6Gp zTpxf+R5-(l5UC32MLYO7)xm!+D%24LX_MVIwM_HJ`~J+PKo=!2R{|<9H%H4&mf6sRQ_gtusHgd4Qc$5TOafHi~@xvdjkHR;~(eo3j{tG{{(c?xPj5X-S?Xepl6R^ zW=_AhO(48+7yx4sZJ#Ndi!*Z|F)|_#-vq9_3}b3&KHs@5s~t2>eX@>E)dz{mC)TS8 z{9~MHSg56u{q-3WxPMRiALXqF{o6d9;uOyw{&lB70+>~`>##KsY8W99RHf0eyT%@} z7Aur;?S|JDin!<#Zr=wlrgOu)L?@_iP8WrE2E$5h=B67|(Zf46Uq{v06i?6Se}pkI zGGf28z{CE3SbOhys{1#7ydou}+ukZd*+=${Qg#s{Dy%UXeSd!6$M>J#U&%P{*Xw#+<9R)=>w3LUA1K8SJ}o<~_<;^qBBudG zF$p*?{)r)0<#;N|7rGwztrE;QVOx0OUpR9jMc-AH)+n$qyT8n$Nr&IFKooW454BM0 ze^QB$)R%Cbw;RFHLredbWE;lU-t(Kgl7Ue^a5nH_P3FnWu3DD|$Qh5ouT?Cq1~Xdz zyPmp6#jGK*ViZpqfujOSPVrmG!|Kg9bNZkEM;(OE^+(oT8j=I7`L@W5WxFwjKLjGwiCl+s4izSGY`(X~+ukI_ZoUezF3sUhl)vg?rJiY({9g}xmB;?i=DWl^Eb+0D0=mR>;mWI5;9quWJc1i;#!V3w zB(TYWc^_gina$}#WvcB|9(`OHyjdnJevxt{y{Jd~*jd4h9{Xgh$0lR?v(IN0K zPX+sAPawCrll%c=_RK!1>a|oty-r9nX|6+V|&hP^h@dJm}-o;`hpV zL3`>d1i;@6&by`$VjpuVa60ch6&960J=oKzm^-K1%fknbk#DwH<4T4A_^O!AC-!1g zM7<)qowB|<5xn^}Yn4wHiZz~A5+H#~|M_6^xH^Zq{01b(H<}-k$|6oZNjgxhl^yD$ z3m4UKfNvx*<{3gnb2}qjiSVi2riOrFnn7;GR+k&1F#U_1c(YtoJtAJJ1)o}5qZ3@@zv;Q zoE}XPxzb!Kby1y=Yzfb=qz!ihN{*hv0`_6PzTx#0LR)mF*yW6O#dS=K$Tlj_$1Vyh z2{!#d>KSQMnR%)=$q=xF$IY&vqOQbP{8|Ew-Kr%$+|=4#HmMfzv^Tz(FbH~lPqQw7hW>%M}fHT>OrZL(&+ z(I^GuK%#8YILcbS#~?jiL*RD|$-(G8a_OTTR`jQOJRdLhaU6JV4B)V5cyt--vFyNV zAA7zwW5AVXenqJ9iLfl&L<&4Q?0f86zhYZpp-NYD>{|M@-G}l^ESt1ple}3IxiXhn z-L~HVM?(_PW{!Q5I-U1)>BFYSiI+~@9#-_P>dZ)E@{S&Vk$_M^hRyuL=ua1Aj!9dB zFT9$UxvKEfHF4x-8W*Izld#`z1en(?0lb$?BjJ2`%Mlk^UPINb!@fy{}3;bTc+Krc?O8#9@)_GA-KN+zzfL{7yGE6)M7~9C~yh*_e7A9Jftv7A@rYBRgbNBC=cic~^ z+B~a5Wzlxuk@Fa!4y5a+(qBUI{=fo^F3ar7ZLAq$VMH*=?_<{r_H=k5@JosYaPX)u z8hIyN??4uybl`1HN||{ksu;DqYz$d$zdsQRJa&|SGel%Pgh*t!JH6DDBW2a!pIrX) z5_Qfe$?NLaM|ooqeWZLK5~~=e9#5V1kK@kTj*!ki{rNVC{g!Agd*+Mp`GDa)GAS^w zAwiCl>LlljLz0PEu#)~uOiZLIA z()u4T8hPKK63`C^7(SP#LOuuCxDC^u z>+^dxHi9@l!I{m`O9QiB*0RIZW_(icGolpGRNiS-?$W2YGcUCa+$& zHh1P!HEywj^POXjnV0@aIrP7;Cy9MK#2&LrB(c$XUxcg8ji}g>Ykv(VOJV0`yg^JW zNpb5Rha!Y=D87Qe%m3q02rvE)X97OMh4|PNo&|e7OVxQN*$<-@SABd*+-*?~lIS+|JQV1AyK(^o&i_QLzhaed&`OXFLS13Z#0Y)CuqW0z5J74uS>+WN$ zN@Za3XL6^83@+?rPIvd@7vwsghAmef0D`O^OOUZUGdo&%L~9H;OoqnEc-RE z%5%CW(tmhK-p3r+{MbMQ;~Qmy9IvUuFkgk$D^6c(f|g{P+_!OhyF9oPqo=%oW3O)e zu?~EdGR_MS0aXz)tpg-jmB z9>3T{ih)7TBxu#iu)f|5xAco>CDyL_!MXVcRf)&&KT$wb$DuY&^y$$^8O2I(p7Tp3)Uk z22mKdMsnZj)bM^|uRHjPnt*9x#4s&)QdCM$ox2GpXS7s4Pwrj!8nfU#c-wvfiaTuT zTHj%=keFP-!p|Pkon^se?xBm+meSkZ_1LHoBLfuD{$jN5f9xXT%pZ*NSgJGg`Y(M= z{sRhP1i@dXf?SI`mlN zi`82@pbT~#F+Nh9;8_Ii22h%6j<*lYLrU366kNqUe8talo)+8?dxN=`Q|g$pk2X~se`C0`IJ{|9#q;pTDh`?KyEfgFOr(h zMXMxMJVm{V_QLL0$^Zh6AR^pQ7}q=qzzBSZ7TbruM3+#flFJAf+-S5HlA#M;i6ia< zbKd(rX+t}R`tfpAX(zbDz!&xgWU%lHZ{epIk9h$20L}!f(NpFDyK;l;-pSJ@+cvew zP7n7xIJw_{4FmZMOIqL!>?#!fLCkS@QOJdx6%D?+X2+#VvNH)DZy}yVj|v^AhGgB4EvcF+G$7e+kvEqvxptNBcaQ73w3E3QP~A zrF;Pv&@p_E88qjpa8z%N;=N5pmUZt1;VCaY)062NoUmnz zLXAvD^4|n=ojNYC(pz_`PboF5gpw=qahtf}oWDntHcYyKw?CTHJBHUEedMNQDXNrY zEV+ETB1W*Hc^^p2E2({;ZGd*A2uK~EQh>sst1syLqSv7EU9oQ~`TD}PptVA~-N7P< zYeSAG-TcMZV{k+;>fvBHVJ0s;Vn~z0@LrF>;MS95`d{bg?R)(dVVUX=VBqZ-9SVR9=QYY4Y{9Y}*&=!k zP;`K`;5wOPf?Ts~YhQ4vZ#UhM-;HsKC8UtUW6y6@`)y0}fk63Rb?jp0ePep{m zqhbB=%`a1YS_ocao(q1z0n=lU%;zkt@DlxpN3@m45ThHVr1gqNMXs!`MntR=4X;zP zY`s4_tAuqwX&@mJxN%+Mm6Q^OQUugVOFzJi$w4-r_X`#8$&9=pgH8)yT~ctC5a#l! zV5lD-aKjs;T>tihx{3XNAxu+V@U0wtb;^G`zcaYXKCdDwUz2DRy0ZFJ?2bFtLkE#i z8FPf9*;1t7V!f4N?s)hBDEDbUT-g2eI9SahrPOg`ZU`|O#NEr<-OimPW?!*#miy$l z71kL8Rc2vow7WP!0X!~=NkL^)O66XfKUt6Q?f9GSjMeRdorhLmohwA4Q4v63+dhk)6g498fsOX-N++FKJ~+w=6p?4kwC>&;+uP zkEFwipCw+Q08>Clt3dUZlIWJPic*J*;e+vQicW#VpNdXaVx2Ki8aoW>mUY7OP3qU- z)tv)CPUq?d9mP2lOwmJ@ItS!8AcJZ#Z_CIag!j5jNgj46lx%Kyl%Rqr7sI)AbD$^7 z3d=--OVjmb0PBR+x38#4IbwwxLk_gC_;q-~e_URM7Dkai;tGsd45d&t%7QGV$uBAJ zG~%=ayss4-e=+N<_j!k{Oaj=3 zx{N7xPzJm}Z1Is-Y=rWYjOCi5#~``;-+e_iYSjXWNldtXD!6~3QlVMj;iXIJh%NMh zm?16`L<+C<40T2nAt1uCQk6=4dRVEi;uyao3_JVlM^|x56TKS_>(Yvv5D6~0B8LgbAhmn1e; zUPxq=b4lo2;M)6Krhk-vFt;F?ny!eK&F)3rSz|CpX@zJQ^H@r3qjV}0UVixohYqX{ ztSDK750f=N#w&^Ik^Jux2cx|2P;(7S?CRN$D9%rKm|^s*-D~9@!Kp$J0JLOjxNq-p z@jU_1_eaJBviVMBbp1&l>-f61V{V+a`wurU!KO=1R0G}M}+Rv~H>UjK3 z{gWFZGqU+8=7Sy2j)KV$;-!QaNt40@ajZT~yG?h1k@D?w$44Z1?>%)D-wXElS@<_Xn`erm#1 zoF)j^Lm;dNVkh%^Usz?x=p-D5&8i)T3Axzp4=fA^7+5cF+m2l@DVTuL4#H#M|3Bd_ zqiQ)!9uYA9lk+B~w;Ytpc-Z2!-dA{#ATn7XCwHB3aRdDsMZZiJT~9N6lfP+<{|Ay# zLk|*j_-^e7_wHyKfLJIOd@j=(h|Q$!q>#<81L2{hMsao7VS51pT<>Z^kFZwU__xY~ zU+(n4i;pqMney1xwVJ3LWG`-(~>7ghL`QJ_K8)s=b|_) zW+NVk&R)Z7lu47kpaeWvObu{gy_(>MQy2V`HYRXCCnSdZ)7jxeVjdVY$!~b4L#Jd5 zZHqKNu3U*o6pAX)YiidAYe3cK)mTk>w8`Pl-s@YM_~FfX^i$VM!!T3>XGyfcRt3v{ zWS6V~*?{ZVxIT)hQdx1^`rvtp;q2RgEKv=fViDDqBN z*|i(Fd+U)?JmH{Xw&B?|C(E;YS7I+Hf5|Iz;og-_tep*LrDObh}Xe zy=LdakWpgX-QN#>)o}(w^~Cl`jGH;}tH8)52*-GfLpgUDI+K}aSFdnYO91KW|JPgW zMu1_d#!^##8?&+Lhv(WrO$-cd-y!xJ5cazwE|XTVR_myl+J!aoF__)1&2D1Bs0g%V zr=50Zx~#zRViQPh9B##C^-USY+;>LB{;syD0qx_v(c*L^gA3!5Z~1_pp^C}+OQ}3i zdVrFjN<(7s4dLg7?r3fJo!^dh?GM~;c4fjGW2YlVuv%u+_QY?-ajptFb|}m(!jqOz zO(C1J5u9g+LdI>|c(pf7& z4PyvLNc%L+2gPCl7^M@8?j#=pV9FUr=)^I`H{2G!wMgCRG2*0OGcn+}X*JwO2IN~;U}whndd`;M*RPuuk+HmPvrLd^}_ zU$h3SQ(CX*+l(pZldbs=sTKSEOYA(g)E6h&J$EOZTfqG$U-0n!noaVi-IX`mbyF7H zPA=(1S0OwFkz?~A4|)t9w$iAx#yk*x^LbOJRpDgxS#H+0Wc&`;NBa-usW9oJb`d zTsUBbO_o|&{@v=G;vupe`AUc)tnT%yeM{Wh?Jvah`<>Vz`4b7K-@$wCbi9@~Vr4$` zeT32dEpgWxxBbwP+=XL=8?92~@z^R7aK99Ex850E*A@9wDWwLkv8}a$?p*0z09n9) z(zL4l?xoK37Uo%%h_f&aCm4zC`%jiFo(7OR3_`j=r|Y?(e7zJHX4q6|OX~ zHeWx#V2_yp%&X6Upr!lf(l%x~Be3Q@C{%HGt1i2ob>Igk8T4``2SsX&VeNN09KU$K?9@$=7H&hR0{0F#hSJfdv$zY+_&~i z8aA!(eovt3yf0(aw-MvO!9-uYL>LrclCFlQA`g>c9pH0ucQ@ZAqx!Ry{x(ehD>7Ed zB^-APbXglsI%ZWpFGX5C=}BhaCtQW?B{b6Fq(yT z7j@RV9iv(65H0x`m*|E~Wq>mV;}kch4fHZ99|pZ)zfo(Mu*lY&^w9)Z&L zqE(l!@?6%}!YU(C_&M#yvT6Of`nG zK=>K?4;pXdSHW#$@Wv?Mzj{d)f(!(nf?4G`$@eMePZNy_OlC9mW<(PAmJi4)04q4w z&*Loa6THjw9Ojcs9+Fz!N42UUl=x`&T$E}KR64Xa%xYnnAWnR*F$v>pRCTEVYZO1W zEK*(!l~^6^m)Dej*r;8^7ap_*Uu8TtV&ibyo+y)_=n3gPA(xF7PqaUxU6SHIZ0W5I3SRPCyptcH_!5$ zpLSKSseqs}=c<=MjyoR*cD)d$E$Z*^rC8q)NyKncZFqfFyjV!_b1nDo)sD0o*>$t% zHviQ4>a_?fQkrOZTJ5Uw;SOWFlEnD@kFl=j^uv_#ZKseQM9eJ{9+w{e_4dhHU0P*5 zv3>UE#F2el`6{7oWC%?o_xsU$9?>-iNJi*fU;b1|vrnV>&LrZo(YSmlYp84g;$Qp> zbbWK06X_kb=rFF@t@MM@$0x}hdFTMH!{*Qdmls+eUrg;exhk$!3aKgz5kyTx`#(rF z?FLoP7EgYjYMlN$rTGsmsBVil&hLbB8(Iq;aESC3(dG+X z?H09muj!Fo+XlvE7CPd~$1n7a6%eN^) zE={O)n{^w(!Ro;da-E^6=)vNWRmLZWIXQbRj*gqV`+I(zH`W)_TXZ6=PM8paH%Znv z-ta_pTDB%_#&JbV;3lX0(lKe(z5cb+TWmOhWZFEb%}+j1TR88=bNd>T@12`Dx(>?SzU>j4txjInq`MfQ>Ka>SZmOi!kNdbvw*RlvDp*9q}H) zIMy-Q5k}lw5x~X#Mcv8BQgl~oQ{e7*5EVaR(4;F(j3|;B+?-Zan9l$6OB2M3W;M@n`DmNFd%!*^Anthd2e-tsQ9=2hT7zq@hi8#s9Mk%o zw1R|%$Fa>%q~s&n#70;!uNPG-Xg6YkN|_X?0AWWS?%m#=0mvq$A4-SfjvqJHZ`2_&kTnDHNb z199p~s}x9Y4>#oi0|Jn*`0Hyvk?S%7=BF*^Y?#vCoYTDwVq6M8khmK|v~rZfWi?UP z)I(7$Aulw)XE@PG4t1xHihJSyICfb6ZMb$QUQqP&b623vq_qK+H;)L{59s zE~CMSOwKJflvFpn_~dQpVy0BvapQun8jc0=Zv^8Pr=@9#NwG0QT?wH5FU@VbIEh?A z4~+9xrh)<>F`{}RSFMD-IjZt=gyuVptdCb5iu~0kH=pVjNp4KLm6@ZTcj@U7Y<)e+ z5C7{QTm4~bP=v73Su2ze-Nk)N7&U+?d^znS5B9!~Z-jwdYx_)a%4u$RD;#L%jYR%s zLhlh))oqBDLb&1e7G|&3mM_RN^Iq2`1|b=OqH>Niy9`6F&X(UQoFxFo#98Ya$yIHY zD`Af?hYbp6t>!j@_nNW4r~l9_{$j)bz60_5=d&NB^J!-L*+0h6m)pkmAs!8@HMpn{ zlr zXpo3sOGEx*| zU?;p((9L}!P~(M zA%YY6UokDS_z$~$Nlbet2cO-)-gjKu{b7F%vARo_3@EZNjnEkTAa0)@1@A^>TSXua?o&`aT^bpkpEqU9)Qn&NnhpNlYsHH~zib}55;kS8SE!LyK$ z8Gny@!vzv;=N3Ngtwu_RZt>dIh%Q% zMRi0xn9$3EA>qu(?C_guGAM$+Xu1P}az_{C9#;ma%zv}%>@GL zxAE`ak;1S2ljRpO0X{vf{xvaG47QENKp9w5*G)aA%ie(iQwb;m&hX?Gi%%+~&$M>; zZxpU^K?0`?*vpfV7QeT*sX6Fu9tzP6jVSaC9TSVsKT!WDTh1`C_quz3+)GWUcjxvTg;!E{G4pJGgs4ZmG2O6P>V)ma1!QK`wtv7=G|cOpYq`z0 zzIDlT``QzGU6(O!*bpRssv{Z+k1Y5|rZYemY`*kPWvPW&;`6*ruKC|iODYl`0bI2+ z0>0mKM=4dZZn~?U%rl+%OYD4{8lbL4R#hGyth19L)OMuqOw}5tcJJj*^@8vBAZbs2 zgqApJ22HxTMx(pr&m8(0HhpVTkQI9Tk(LyU)eu7=!xah=sl;*bkzhB1o`*?hT|w&>V1lkI;7dbS@PSV#`eWkQ zmc?o)%b;z>}um2)_yNgzqFhO{-gJI;F zDDYA0k1)Kc#f`brlFI8ok<9@sjRF4@<22d-QxhNI0mcG>GA@XQCDscpi8?k`)_8&_ zXF=8RVrkbaFV~1VUx=E6_1=?wgPZ&;B7@_7Y0WacoO%al;ybhcx2khp21!Lmiv_y);da;3sMMy3YO|n7R&z<mDC z&T*i4>c8*)eB;$=gi|-i2+ukdT=(b3SKa9MPvIqh1!LTRNBKGnxpm5;WLyTI zdGIBJ*?C-LOnXwMalx0Vu}I`vDU&c8P(+WE?h3?Rk7fVb&WQRbul00or$ao>3j5-y z@rnnO6&lA$h!;g+4 z4vVh@EVWWj5Tqw}MXx_lNeg7IDrFVUfCVs#xUgjn*L#?c*6N#bRoyo|>ORRN>RBHs zi=OOXKj{cvLW*BdPg)s-Ya(`RB(`c1!5dswK@5FzC#&hD+py2t*F{S$+maT`yoNtH zze@r5#1+YL>rGRYV-x?B220p&TAHma3$%av*Qn432V*vR*|y_;bt{`+A?4h(&wxz} zOi>ln;=)EAeeXKw7SK+*@}*7I`V^o&1xV|jKnH&uU%uv=$fR!1p;|YxGq&Nf6E@SK zxQ>HL>Dhd@5*hrANALBz5~TKu<9tl%){~dJUB?gEKRnZ&Jbqra|Fk?ElXd=kWGvgb z(EP5hX;ZF+RY@6F)sdTr`{s>(qs{Fi2IOt6i+`fem6=o`ADe7@ z7`*xZdVl~wgW_UNYSN$wvE&R<@t$kUr2 zgsy!#W<6M)+ShdBFHx{z6#49RY}3<^a-KCen@m+nI!tnCVTwc6u=)^XUF)4iE)QN= ztY9ZAzjnRKagKk;E$I?zEgFtEzfz#Nz+Su0xgJzed!!TiRTO5(Yfs$nW_(WTs~aBF zJYuLLMv^eK&0y`hpT~&vwxS^xE+@akoS-MWaP2_C(Yw>yE89BZ+aSe=R(UnpdV>Ct zXs<*1W83vx#=ESrLoayo$=5sR4$%|v%F`9S;w0(lyG9I0n|DJ6v)+IQ__tN<3S{N- z1rt(SX}~n?oMp&89YN{D^q4DV6cXWPg@Uhcn(!dnZmkA9t$HWp2D4YE%+bmwUw2rH zavlYZC78Z!eiO*0P!osVF3H}IAFpU4YW9E+P%KzhZ>-ng6j^^ZaXdSiXbY!9d_xnX zFvwEdVN>5+<&#`KY(Ej*wvHx9*qp(HzvEo7os zdfV6YH~bQgjwLJZj}d<^8SXG7~t5bwEu)P2&cTWNSe|Q|gA$YXq1u+heS*0AT1cxpk=?s~VSRu9; zT(~fK@!n*Fh*y)ghg%cP2qGa@#(c-qY={Z*vg3Fwlh?3Fy_{8%pKcjd`C`|NFsO{3 z4zcl-BceFRwCFw{{sZ1OOhy~&e=1`5a=o`K-kn_XJ5kjz+3F+v5++vNyT|$({LGc) z@*h4Arr$5oJp8_z*h+4_)Ia(4q3=7n>dGfW8>BXDpN|<}if4*R6mD9I``vIOf1uT) zKcN?DfW8v$$!*S3GZzS(%6qJ0g&L(*rrUf|=BE|9!lGWlS`^SS&q9PrD>K#zf1ow+ zPOKrs(V$NS)RM$B)?9y#zg+uMW3uZ!GZmVBc*wg@uF;c!Vq{$9zZU5ddcCk4@PpTW zHr%*x-OD{~bRxKqsb0<4zqt4wGq44kmXzRpQOD-y_QAQx9Ep zZKP_(I&L5iHgO!jleL8@`?!4+!6D4PY*`5|-$Rp=*4VywBfUs6qinxqw{f;`1&7Qp zpjsAAt5xugPm5YyH|5r;f|5BBjFC3I7B_Z%rtw`J_IzLet-T$OW6Alr1WkpnM!)x&q$||;(NY2+ON0}Q(*Vi z&J9ZIbI!_p@wUppj~oE`AnKT2o`_o)pyVk&gEJ(g-Z3Wst-aWY>!l`^suPP2pLNIVo0EwV^~EJ*RgHF>m{CTgx-p+;aZ{YTxC5GV(}_n?nSC(-}KZ9cd7dDjNY?w-z9GM z{8Y^kHG|VWFObt>I_kbZ)j(DJIC_cx$FZx5;W*h?u*{XXRvwx~DCP>@U{}%z%#DEU z&M$k8B4g*Eu3d#r=M^djLl4MK#NM~xpeybPPU@Y$9W!J*pr0=6Bm_o@2H?t=F>qlY zR(;FU`ynz60es|l-!c=yJ%`C)-q$x2GE{v3oDMEB@Oq*>BX-Twe5swZNvxi|%DU4} zd&gg=ld-SY%L_uCQ*fc^`#Ejx;DB?~mesoFJ7(dNC7Ms63JHGCZtmaUJRFRkFa{bP zj3F7dOF?;}D+*J1NI?iZ!faWJ$kd>JZ%THz{H2>3+HdQGX)Y4P2#VaJP39({$MhA5M1;f6(`Agq^D&m?v?NuAz;y0@TQY9x;@qVc)~J|kLUbIa+81uG&uwLLy4}`-+w>ZZJR650 zIgAjA4_OY+q~u9_Ms?p7_vz)+nhvKZ{^eNCUR1>N3}AZdy+^NtBF-i1&h>){%1`dR z?f8c1A|Pba)OfKuB=d8a&B&FAdP9~83&Dg`hRWs}oHn(j=WlD~Zz7u!QBw^0lOvTC zR^txUoOx>kU2sVX#AzSuLAE9?O?BpLu*TT zsSy=F6k2JvtSvJX+dBX3uu5d&4%%F}ac&p|%xmjO$im+u;R^q-FtRqO>#6tH@2)-c z&0RqEGx!zigv;JiU8zbE)tlLs@~WU!<`p7h?c0)v%nBIKm#^`aTS zrNbhhHmgTTcOs3?ag}ZSdC^>1qF1%;_&th(8Sd3Y3#UD$_0r?k+d3;3((n4Ua+t)9 znTAMv(jOnbRLH=MTUgM@l`TkRZ2&XD5 zu@TGBEuX4@uMP*_FLoN-!+VOS`1M0iZDdo(9DcEMo0G_PVpzw%!$O3atMN4#tFOX0 zL+u+}r^WukZ`ZB2c@-Vnut4{3uxXBja;`e&(>yaHN5CZaj9UBVbtzcUVf~CZ>KJbt zWO-CP#IxLab>voYd|{4GN5iFB&-XiQtBg2AgU8b=&-VC{{=z-ciwynCG5S3X$0i8+ zVn+BpXfmT6-QDd!LC4pw8_>5b;4lfi<&LY%e{)=*6h405Qz!wpJCiWA$w*gqltv!l zjOQ0%sY(lp39G+#c#RHwxygU-paG+4&nC=6R5-x32}*@rT_t2=&Nn#Y%&JDL=VQI; z!RwP&j14}X!0i}J7sMJ0-G#d0z#>cByKgSMw*O<1>I!(MrmDv|ds75eCEV7jYF!#6 zx`AIS2dEGzSzi5&IDLFJ{exR9j^x>)q~>5Wah!{1{eg-#v^*FpUYyAq)1*K;n#QC( zMPB6qL1G~yo3%u6PhTPop{P&5jXmatxBLz~>KWSRHYhg0N*K@304}!%sv}s;Y z;^ZA058w_Bu0d4ycFk+_xn<*oS&~XCIx(#+T%nCMZM^@Wnp^7}_zIRPqSbTu4);q1 zBu?0^+|ki)sD1sV5lX$jV(ywW+me3T_HT?Vb_dtrAvrYUZ?draM!?KFDdLj8Br$=D zOvtLES2q2_cKqIz==F>rR=7G>hYhj!ue2IiCq;_TQ`QHz8}v)=Mz$MHB39k*_e9iJ zz6Xp^PFZ(Jl%A^(!#D-Jo&Ldu_EajlW5Q#doM1TowC#;KUy~=I;s13jNkppw2n;H? z#rl%go-;#}l&ZTo2GTUZ$lL@ZO(Jf|HX}Eb4wLy%ynA(QWp^UKFKU7qbu$_v8E=lN z@DkA9OB`bnBFZz`KJt+&#Z1#Z)){+2{5zk0CFT0pMbGq0-AT%vR+4L?<9t0Q@=Ar5 zeV^%m$RwQqN)wT%I9@b!Rc2E4Z}dRa;kvLPyP)G(l!W7>XuxfzRWRg^7h0MhI;h17 z8gdB6FLr)qO$wJRNvo~xL(MMFlogt>ec0;QyYf19HQPZWVwKxgFO6=}uE?XnU3=jy zwdLypKTsR@Y8pAcROQTBFfDZNGUze~|7yG@ESvMyr%)mw77Kc2_>BN`Gt&_m|Rd!Y43;c1=?+|0R?*li&zsf&OguwtsH%K<>hDIiuo+1L)yeq@d+z=Sk?`iv`Da4_sVUIA=n%X!P0V z;lGAV0=zPwU~eh%{sf#zAVxpT!C(hSM7w+w@h}r8Jm(Pb+_*E}D-qVJhBu#@xh%$@ z(SGZTqZawQa2Wcab~nuzu}O2T6y#Pp6g?tV*z4)eoFh<49^S3$&mC-D(0`=XxxSP| z_E&2g7!(O*ea+ugS(+}KXR7b=L`8D? z01cwhv1fVHQ3bJCO1o2BD&V0Vnm;xj*#fuv2CxS}Q%Sv9$T%_APRb$oC5!I`p=SIn zeWSE0h~uxX^p@xb_u1jg<2p&b@5Thp+UWO9mJ;ITB1OvYx5}*i8djLK}wb7dZ=@!w>ZpjS~&D7W$)On#5k4E zX?j%KJPiZ0X3?5%#yLJ6)!RRQe44&}m4;k2nx$Yhgzj6EJ}mEkr(Xu4yhC1jLBMM3 zRP?OGpxfM*XY9aGLV6ugVuef8D-I{Gi^YFAmE*g<|8>;t%#CT6_p=+7f5gJGu=f1R~Be-AVa^bfd zkuvy*Pj=|(TO(G4N}KP?Yub$Rq6^$JTaAmXK}Sytt2dH}Fx$pJ)dNSyb*{DdERu*m;(=jk z9#n(nTMQMytH-e+vDKFreTK!MccKCoy~O&v9heIU#cj-bVKz%E&~nVz%a%51PR`0h zB~COte`iStz8s7S5Pt)1${`s|)35n7w(eSoptk(9GVdl{?AV{ES|%=>lMBqV#!-D? zYU{Iv$C#T3>O!1DKX+WoQ5v|}W|0>*<<>yk-S;x^EA$5?BDW*DB>qa2C}WceJ8Lj` zQQ1&4w4YnAch&i%m2_-;lO;#zDwpd5&zSuaHh69rEAi-+L{h|+qQt3c#|h~Ef_XJ- zk%7vKv(mSQYK~UzIzR@QZtWphx_kUlF3vLMAvd4zSm%$@vD&yw53(Akj^h4vY3Tjr zpLc2=6Lh0aJ>V$jHjc4{B)Ci+z=Z4%@D3wqBL%@#4JC51jqmW7=?OY_P3$`SnCZEf z3QZ>sT^>pqC{ia3JP39@p&k4?qyw!ZdZPGTD3f$VtP72^<$K+1!S<_YvG8isV3YRJ zWb}~JZb;mLx*`=%Mcvj6dZa9cW)MGg6hHWdtP$zXWgW@#U@Jsz`_eflcS(zu;>=?E z>1;xlg!pe6e$}&D>~`&K4@|0=4iNL^k@y+6G#JZmGzFcjXGOw~ljwJ2n#D*ewyoug zC@N^GDjC)8;fQ1!CVYE2qPKDpCD@uDXJrH}nM3I(6>Ju(#Ej{$327un&}?eYY8$F0 zh2`3C9Gx7D9gdAUH8*DryIzF%!wYEkIg3w5*|Qbal9*O(7Lv%JL-RQT@LOZG+~~cQ z63s@M{$%ABLeS=5XLRi|VuHM?HIXM3Xt#(4_iG$p+2h$3`7pRnPnudT!p6jE6CZt4 zLFsTLo5$a5wb1R1{4rW8az#ZsBO5j~IFglAyGM|~W_dqPzL+Pvx7S-D*X!U3{ogF| z!%3yrhC@gM+VZ{&HllUReOU5_ByH7o%3-=Y$v84{$RPdNtcY1K95d^6hr1S)cCR_Q zs*9ClV@vE?5*&k6O%n8QSrqLXX?o^Ot|LjrWaT zRyMb^l+CX`n0N8u?9~wvPhZQM95+WT(TnE|=|PR$VRdnn6~m>o;;WKWT{`1~y?KKA zUYs_uFshaH!RGWKi`8Y;QYR!$u1>LrMdH)kQVY0Q*C^8M*+jqFT-SGZdr^Wi)0_;# zGW!-TN9T0AIO7Gp188$-?LpgMQO}QMev6Xl;IDPTUu)*WKa^UON>U_k2tpgW(_jXQ z?Qa)@HTC-HU%30-OhieWn&EUr=$_yFbB~7l1#iaVMY;5euxC^2%FgL5K6gJ==;g8e zEuC0mL=wYYwRi445z$Ddco(kFEY~Q*iW_Bt??Z-qJQBq==*~Ffj9xS?R?G`ISw-zZG|u zeSLhik9_pOEv4(`;cfbfd&sozngxE&8bq|nLLMbDld(FvY067_;b{6Us38S)S2_9! zx_?Igm@6NNc8w>5)7eBu-c$M8t|64v+wy}>nW%8GU{&)@td3`zq}Q}?t_xFI(p?pE zM|XAZv=)b{DyxG|rBFm^$V1>w#%Uo%*Wf9tx97@xWeYZs0aoVN74Z^@1RpEP!46l@ zLc8LxuwGTJaBRRAwJz5XM?NicE>pvQW38W;XT7`|^?ada?q=(-jfH%0K%4(b{1?w@ zEe*N+L?^KmDi&giO7w!B_4P@p^>M7SuFkk=!)gm>&*f5Wm&veta3KL-e$KcwxQyy> zH*WQmGc5&O6FG2Fsl#j2V5A+~Z(&iESf7}rjmL+-@`=`KW#a1>a0AOuMV!|s0?i{u z%JHrL!_+%QM;dKi+p*1xJ5I;8*&W-qJGLvfla6h6>{M*4la6g$U(Pq)=ZrJ<|6QZ% zuBtuPn%ADIm9hIP`@b&+^(ZluSW#w3g9L7C0xtTv=tO&C*}(s?hmK>ZxbU~1@8R(B z@!{n!LjNbh1_6yRb*iPJ#SP3^+?scrt^~`Qz2DO1?|b*`KT8z)e>3Ve?X2;#$KM1O z|9dHcEoLuid|@>?nSK&YV#-INDHKr{xJyIz#88!|<*kYK;z&MLAqvr`OBq{|x_5~}`Gg~~hxDQ&(YT8x&3u7N ziCVdA?wl4n`>x&c>Efm3Lwa+I3I38JvbE|kdqv^KgYR#YV%Fh>G!`Su>gd(Q!^OX` ze@jpjB+4CYRn@fjS4 zs?yI^4Xd&{j-Cchl}!KDo~_>fjYBlrSV`wBuJ`x-^}ONpfu>OKmp|3&%i3T6^UjB3 zj}kq|-}c39{k)0SDGe+{ixJUHLisKSb?&+*)~+|tPou2Q#aFpW?e=Z^I9e>F zlCJldsDXc9?K}EhoomlWmTq==>uQx?3+vBxsGGbGJG2|@V^9WA%75i)5ajr3!x}T_ z>%Bi5JpA!JL?Mbkh!SjbSFo7UjH6v#uSQl7sub`RTW#vt*wxbvdn=5#tP`p(&DAcY zUaq>|%YUCWb;DkBQZ0{Fuhz3koVKpkd1tFO=ukgD?mlY+Bn*W%Iga02E8Op2@?I{M zq4=R)7Zr2Z$l`m6Rxa~zJ2t$p{#jHIJRTLNVj&72>T17SX_E@OIdRrKo>D(AL&c^U zSn?b9-0n%@tz)VJh85rg1j|izTJ(KQbhf2;YacndY^HMrHf(HsNjvmh`BF|Ul3CrV zqS+5Hcq%Sx+`jG0Ky6BQZ$5R;QN3X4!K&_MtG0TD^7_(`MObvLRCq zmuKsz)brPGG>GS#n(%V2+oN=uFI`XN@t$OHwoo0$W$#GouRBE2;FqYus#~io z-{&~5J8js*y;AC|ZokxYPyX_{-v4qjRjBQC*kDaKPV;-z;}PymP=uEJS2qK>@4>r{tbb@x@0MibRssRrNI ziMx!MO%bRa;$JcES?3|X$l5CzsyFJ?Yuo?ulRfzA5cAOm zV_yh$_@S4NKb%HZs&J<&t5k_qE(sQXHG-2zgZ(;ut>}~Ma{lRFOK`NDmzRdzGp@LG z`hs(g7<&H4Cj7cJ`OyQx`G4UP{|VWPrulj*|H+$5HLaV%#Cv}}k1OX%Qesb<4V0BU zTyNI*9(wcaNN_qv^)IQZNCTaLw~UgCZJ68}$1N!w7FtvcKQvdre0IA>w6ofsOcvdr z=js;5bZ{>GxHRm>T1>THz^(03{nf;ddspDX>!M(x!#7{Ti|mP7lHIKMLTne648zLs zvTEmF`abN1uxj>xLNsIds8>>)W>ChZoza>6+!f$je%!nEGUr~SMDN^}6xsbO)Kkzd zW`ykZ9(-Tk^sBb}Y4Wz|srI8m(DhhL+v{(e7t`N(U;iJFzZ<*V<`-9qk@umn?<)!| zazBR0G8Yq$=YJf;+xu@RXZs%Kp7gv;8ow|1=vN?qJUB}uHI@s_SAIV4(z#3-dnWh!de`yROkN7r_Pk7KEF}G`5K3Cue4A#n zyiWU6jq3KP;q3V=kf*IyJN>ns*z@Q8y?*NLB=<<6$1C;YwD|D!gk<9=Larn^hPGVi zqDT8{)5nw1bXw!j&?f751cuj}&*_cdPh4u%-k%DdLYG~K&X)p7kKBZ2@gm{^WVGQ8 zit$Q#v3X?^(2)(q=!$r2RLA_Hh#^zL$NV#%@);;rB~xDa8OnL@8ON4iz(n5{u+>Z1w$$UN=Zu8XXjelG1jK_5%g>!!FS#@*DE9eW(eDx!^};U_MRq@ zOPOdWtAfedk`dZY?gyJFmPA-L9>`-1z@kfb+?bsR`j^ zDu@g?r@aEVDG`~OWJa6>7I@}0+YeS{tiL*m|3WIY@ak5plU!+;$}Vokto>|`&l9$Y z?gNE+oc_%nKN+bONzw%}k;W&b9J@m!9x@OWrVzGi0$0SNp~q|3hjd7QhUa$d8&2Qp z7%~r#I6?6`o{N(@2lU(8o*K2r|F8i)iGNr%Z5O!k@xU|NaNF8=k_^-4V*eqy&CAFr zge(hvmZ|E0sI>~195wxp_qb*?cvKmO-e_XyBxRxwJA5K4CBePUFD{QgY`CJm^(9=q z5Bz|&{;=qAD%hNhn0OPmtHk&g+7dv1q3h@3Vo5x??oHg}Rjd;)-a+5wgDs27<`zSF|pA zYvx3PakQ=$KyWaiBPQl~+!xXHIX!9s4Be@tTnK0dZ!$8nor8gG2HcVE#ByH`!xvV?Od25hc^%Od(4miFG;KftVsg@R)3lXU)}D4#NM%# zk~N`(+#woJ9sEdcmZ6TawX(Z^sGMySJW6k#*ES16dG=>u29`y`A8ks%OoJX_-?6sF z6lTDpnF-oi6=!S@K*BJ`1^0CS{SJCJ^m`@rJ<%RJE`Uy}b|GB(2TZ@Ast|aflT0mx zO4e}6gTbuU&*7i9N)HB0BKMkYzLDj#hLXVUD5E}SrTU}`Cl?eE3ve{qC29_YCE!WR zcnaY$RTJ_A7iIT%Gc~{&&Xr9&T{;3#uIIJvQVm-5zsoNg;_DfjsH%*YA-ug{DH_$0 z1b~hpos>@o@ud8`GJfEpH?`vTZkV@m(y}64Z{G zN_IU29hxmU0N4F=L7pBam)^_GUstH$V%$HhGjn#a?;Q=XmH;6s1Ft1wWWqW0 zO3}x5vivD=7PTIOi((h76#o5%Y4a83#7AFK0(XP&iMKr~rVfryf4u|WvG{Ns4TXCE zPVo7IE}b|>qBn>u?OR6&Jo(uPNBG7zMCaRssCe2wh!#q>0N?0SGvcYw=R@)j zLdezRzyfS01Kk9XzkI6IS#-=_EL8{U`~-N`N!h7tfJ>Be-=EidXg8O`;DIhkKd^IP`~4{ zdgRQ}eBKShK2gOdCkNe(W#pxI($KX9R=ovTInppF^V?HUWrq!NCKaX;v}vU&>Qk-{ zCMz$%YGWqO8}7P9F(#+!N?6^a@(?BfwMoUKp=i;Iu&bklJ&JG+$&S|^wBvV^sc7*J ztd8Lr%n3Rq6Zn2}2IW=4$q#!{DGzVfO}DGyi*G_XHLz`Gb26Fw(@>f+-c?uobj9$z zI2=1smHxUz_{PVDit}1sj&4E7m5a;sTPx40v5|d=jGO`eD%R0U(J2pMc|e!|53GsI zRa^cPKtP0CI2#aI;l!w;tvv*m0#(6Z_u$4BFL@hrkZ#)-b-nF}rOOnLYz%=ijCNCt zD?b&O{{=wlG5v-)m^)Sl!g8um|G6#d$L*@6X$$@L*7=<|or)|AoX@EnaoPIW2)nI$ z_8CfQXkh3Hk_Cdyb=83wH;4j0a$)Al+IXfSjnVT%_?-ZCY#c#JUyY;xtw_IJN!<%l zV$oIPdYn$SJnU{{qAhaCjgpA=UPj+jqS&l%Wz;nRXO;TU+vx^cvzo0KzW9&I=d!Z2 zbXUq}{XSnYrIyfp(MlPzHJwNPW1tcup=*QRFmD)oSP8a>(u1Vhx zFE9K3-?s(dF8n`&H{YjypW-$@Lmn|(DZsnI{)xXP&ed*R9U8z6l5p2FlD7DkacIt% zp6xU7D&Z^OAHr3q4(mR@N#xecox_PMLY9Naig_OW?)#qE~*(^iF!>bOx} zVyMPod@ZuDIpK_y$GKMgLc=9i#(S`{HQ)H|NEq5 zT)dJJ{7H=MDEtW<4~3iqo>s;K6tO3*l1f~;`R{?aIZ^$~@&57raYOWV00!sTlON`n zmci>wsE>yTZi-Lbqi2UfKe!U53z#W)tVOaP9LsF$0akQYx99Qe(7$kq)N zy=QL;;-hnw37p`PiaFVjr2FXJruWFt#Smk~J@%0RQ-E)oqw>zKjYS~w!?`6+uY7454u}k102LM| z04ukzWy{-z8l}z)+6Vm9tYfuBbmFVj(kQp$nt)VWxqB9L2?zHO$3r3Ug;z35<5{OG zmZpF{{^w{kIQ5H*tcEoxkU9u-bH04Fw0yb^%vQm>cyc2Ulo5c6W6oPmkHYTY%9leb zm{$vb6!M17Hclqxt(YG+5q`wNuLkE1h9^M16s~O86gprxuu?}RFQ&)HEe+Y3zm-BW z$QaS!N=b|&U<~b0@84={SW8W-F*dp$CYWLd=uxVM^5+c&!}C4V0v#8I%GvijiBRT* z2s7m`hZBX7lf=(5lr~+_z$vpzmo(8rR; zyjJv<-LtNOk)$V^qjKSuMXwMb`#=6;?|+$4%)hEC6QNHxh^cMAeGvod05BRb@4+z)e-Q?baXw8I?J zIy13S)Su?>lro7;%?~%I7zzktmH2jyG0g)^MrO+itZk%4WC*f>LK-y@8W$0;tjn7b`u%PW-S$_{@Y=j7nZ z?_`Z1N|-9WcZ-@d9-OSmYYdr}CnO>UrN|ry0K{x$82Fs;wPEvT2YYwi9}8Zb*_g)ILFUVx6+@Jx zXe-Dzsf#5>8)45%uw>16``n~Um`kPk#Wkmr#C(u?tGKBO)SUd@*xVxfp)cOcU#vPg zHWfTArO|_`QG)~8F3Iu8`&A|E%@^p>=wa00Qus&G(W0m$0T=#|$z(ym7X^N$T3}>y z(o_!OkxvVC9a$N(pmo#2rM@*w!Aih)WV$Wd&6lpg*rhc-H99cgPn^1_?|W%C$w2`Z zvF_-US((CbN7Yi3mC)c;C`|^)e>KsWeh8oZu@c_i1d#T;c2F)#_;(M;Plsu+zNw7F zsBTtKKLa8TgqQ?>pRj5}W^*R0RAA6cBNq3{hA-Af5vJH&99?hX!of6<3XofbUMDnF z2>q|>SB3Wbi9mw}up6a3d%E8JHhkc%_4S=A>xA%|RR`}2YH_#TePB)xt{e%h+*pa@ z?ZQrpAxiqDX{JI@fxjkPPDYNU7_KZXi9wGQGqOUFywvnUS`CJVg#;_-FXw>tUPBxJ zGG6hG6k(DlI#N)uRLm+p6R|gfB1WgD0?*AH64A9JwW#Aht8y^b_zU-d`)G8%P16Vm z^3Tjupc|9UT|WNxZTgZo(zXHDOb;b=EC`&+sneR(<%V{>w#?v<9<(IlT{&uqfIYn+ z7Vw!wuNQH1U&Q2Y$2b>N1~seyxB}emn@;;BJ&l>Eb}s68kM!=X$(??(OWfwpKMrj5 z#P4YSk72J@oBmIjAE*8w;h!(>zEA5U7gAl_xJ_PjmbB3lSE_~iQ5s%E;rC0|r184VkJ4HchJ%aF6y>l zSH*}qGVXxyw4kN1zJKa!nG<4h;~mZ4KW=Aj+LwHonUcz>AalV9<|ry|vd$m`#Ea*n z(`irbCv*w@<~g-+-~Crc1l40v;=cv?e@gQ$xBnc!=gCTsl?ZRm|0(mlgF#36+ z?Zbu^b=V|+AYlliLfD?*bNbEqdm=;Tm%>zRlzgd?8eC@geX(@R8?bKKm-OxE1p?fw zfwk#E&>BS8(ZKZzq{m^yDP>9%RT~c4a2F7 z=XC5FD8V=(fpd{<0Dl3v4k|xzWsHXZeuFK&>w;8VLu#x2p4L2UVvxM@aJOO41YqVf z9x?^8?)>8^h*SOYYXY&2B%zpx8PTig*I;oUTROoFnt8P+mV&Jp%DvdWX3wV)z4ojB z^;SFemF^KAk{xoC+r{Bqd>BjG%kLZb&=5qRF@Pj%1CQCRnSyG#l!>~%J&C#<5~rT0kPw3eXq|vjD|N%0TJ~`R`OEUv6)zg8FVD7SZXs7_1Xwx5UWB zvf|2e^nOQ6{^%kiYC8+rNuARs!i2T}Nn9|0AQPs7Ap}lz1ZUN17lgAIF(_*hb$dK{ znwkC|09}411ibQ5aJ+p&hXeOBoG%ca1%~gq%l&)lawaA#VpI!%6jdC$iE+3Jez)fp zcGkoz#I5=#{OuBdfWnkNV&M@e^YtTgcB4A6F6oqF^FB@NM2SZ$08h!`n)&00$17#>%!$-BJ^$f-oXXsc*nqs zQ6bCN^+ZRyhmA1Qp1xv{(Df3dz_$lmTO;H;a7YVSv0+H5R7l4(tpxAc@``&6ox0~H zD48dsQ5DW5VvE5-mSls^^uQYc`>%2Q)6i)%yX!zwW^TJA=5(36o%f_)--bf5%v${&+7(ae$a2MT%mq z;=iIdu57A9hDI?G9?3^;UCKmkX`N!$bh%dR zMxkKmM98HX4%I`VQ&T12>n>A5Pd~bV}%0s0`>302;6=9@I&NE)``DL zovj8to*>DPW@6W^u)G*s&Wrp{4YuLJv9S6Sc54WPcLFPBR`a#ABuwyE@uXJrA=2GK z=e%mqLMn|pg`$aH9s7w^O-@a(-{fae)!STL3=RMDrzIN=Gxp?nRE(tguddfi!7B7v zq)dFaM(=XmTu$@TDv{2r@Mofa*v`=axWI4NjfolO*Ts0h5!fW^N=13c>*35sCZgvX{Ch zTTx6qWQ>fAT>VUs{1}S@S*G+nVt$N_78Z~-kb@qkUrj%NJ&dk@3kiVLFaJ| zZVbVrg=bk&YkXMs<3-Y4oV}0S1<99(=qB$2J|8ba=2>D+ zX#B9Rc+WDrBjxil@Mexo^Y!(8xeV955+3ha!VfHUwZzhrUtnZxO4LT@s+9&EF`7?m zcHK<;%Q2{_ zDvEz|oYtl-xHI?3kjwom?YS}P<7IuMEisnPbD3vnc2?wHYw9uVRDcYRhLX10xK}gR zvKS^N zFe>_(SZ+)xM(!^@r2&SGlw8a~prd`lS%1)0Y zvO&QxkT+$gMjZ38TZ`{yPJq*1`oBD%0g}*hglxS0Z{D^CPaH7;m zvinP>$Gp5O5_tFsPE(U@pJq;!RVrNV2w4NXu(~I@7dYX!feP;EMt$e2HL>hWfVjY zdX?I`c^{7*{ZT`%dSytcFtao){vH(d=DZ*;RU9Ug=B7~&CnxKNwe%h#3pRO@5gqgVZ@9L^P2zK;HCRN%42KlKe{9P zUMa2u;Z6Pj=?(xodarXwoV@2&o6Dsmf6t*!DDX&$GP}iRT1mbb$77*fmd-2jFhGUf zcznCK)m%=hGSpoTdxidbIIq9?B?un zmVDLl%GL0;yequoheS_V|EiwC*7z++KZT?rhW#D9v@vccQa`11z5O(QC*TDG-$!7C zKzjJ>-hUv}gkYBP#c|#plj+lu`FSu@=rCAEJh2{94}R zv6CG(dt_q<7o|x1JmU_x+ZV3pomzj3d*3B>x6_*0ANOxS!08(#$5UdWrvT(wO#lLFMb<6nR z8WabUE^7xLqSi`9)j~2V1@{l98qF?ID{jyCQ>`i*wgp44=?}@ZK>dP1NLuaB!g1K& zu$bAZ6h?(q6mtj%M~aZI;=3WMS+#uAHoU^Cf4&nOP#x$aM{s;vxz%huhgW^(Wv1a( zoXZR`gWZs<1O%!q>|V{LC1~PASl6OYUwrae`35pzk*@_}{n3N9y(Z^IBu|s117muCg~G;!2A1h{p8<47UkU zt%N{!6%f=CYKAJzi5j5`oc&WXobFfo#bg@BSXpOup5V?Sx3}PV8V2gt4_q;B*AGa~ zWYPTdT+7^6TUf<~J57ibnsAigO&s$w?&0^TzVI@_M(aVEUWSvu*%=u;$L$i1-igx; zz1$AW2DPu?+3p^`=Ho&-M6nX`fAD{_KcMs1ZCky$PO`1M?AwQS(VXwTQfnALtbX=( zHu?7k-N$_!SB}&dA$Dkt?jz5r`y3#6;w4AQ{HTXs?;BkCW+>jp5(7RJqNBg18iz5U zxhrx?~QFjM+1LC>dDf3+4+vpwFc@} zVyAFb27zIbz)}2^l@A%0D7+NWy5?Xf=)i+XYFEl)H=iFq58Fjec4}}&FV0g1!`Jen zMPX}#h5!!DT5#To#UwY5Um>7M4u_&;rbn{41t7!*N_#8nI}uff%7;1>VQSTpt6x3e zWsK3Vj4at5=G-;z@V?&391qd5@cmS4b^^jGm_w-4RGW+AFK-^S~{XW=oEtq}m zHmPo#2Rf#ywPY8D;GU^o=+$gGf~onwNumzqjLn6FgcqaL5qyi|vt_U!s1tA47Wmr@Xm~9a^-Mycv3e#q7%Lc~4kkl~b0a)Oomq?BEl_ zt&Or5R+?&FfwZF{CJobf(=e{cRNPw3=ikTtsFs^YBi}HL$D98j8U7)fWAznV3yW7D$M*1N2|fhC*TE z2}xq=zgkDhWkgN;_)VmwR9S(d_?XBbmsJPdsh;hye*QB0?nra7Wz%=*&Zb96T8Eyhvdh2#*&)R)j)N(8v>(Du?@1Ab_vK zj{{&=W@{;>YK!rSQXcbD+)9qE-x%sf_4K=5esFn7kbtWGUnUX!P*5WafS71M#gMsg zO|$v`Sb&P$ynMDX<<*JqY~hJwJ(iC@^B{sDMJ+U9h>e(E6=H(@u?j z(3BIkAicXGh}!B0;mvm9dYEh`YMPy(ZljlD=R)M-rfxuu8``qh*B=VL$+zee<*Zyd zsERThQDbcw8)kUj=k1c?_j2~wyt?J6+8TDl(T(}=Dxv|khO~L%(2~WiQ6MipFIJd@ zQH18ly@byiZ{RwmiKYuT3iSF{@4>ns45hU7p}n=M?>q=yZUERt21)?39IlHN(XI zqf!L=4EI3S6CX()WbDRckkcrnR|6ntkAvPvBto>vKems{NWh=m#wS$;cb;k`Sp3Zo zLM%4xm)BRJugu=NpTNZ|;QHe<;&K|i=5}6SsP>*{ zaklNKsI|Ue{=UxWg7SQUWHuPyC_GuP3d+|h}-3NSTU2hi*Ep1FkoM=)dsQ7Cz#Xu z^AS4nU4&nB6hvqMd$ERaJJw$|X8&`4=K zAa*zyiza_NJCX_iK5hzUu8L&xBmHX1{gSl#4Wcv;@kmAhFxYw|GE!5l+4w=am(ego zJ>d`~ulk@nR3(SHlRt@S{FSJ@mEii-s&S3mjY7=rCy#WvIzkXnGca1QLP9}`Ag)5t z3FAL?ZRCxrd+4V&a#G8m{2OiN%~d#NAr{C$@axPk6r5=;JlWY8fZa8!d3Esh(ZI~T z`?Gk@8(bZ&_H3;UFekg=?zFvU&TJKWN-`?;gYhb@C4}HU)Jk>zLJ|wY;Q2zQ@asM` z;S!}yX1~8&y+{$nFYCaC9Szsj87?0~(lDEWC~J1NBGS;c5=X7RVo0!2NWAfs%<$ee z5ZOjRHqU{`Wsfl?;lqU_j1`}mXcRb7Y!l4~Bd;xipvV}Sak^f39(*mEfZ z;bs9ug|M#iDZ{U{CG2R+laaog_dO)Z2qqvdb-$LBe9dso6a-)pOr7{|xQ!lVmS*z5 zsIpG8I)u|6B#BBukW4b^*@{d!?R0t~Ribh$@4FS*Ze{oW2cg1B6ta~2jVD$_RV^k* zHlBy+q^`KSo-s}2T!J(Sh*UO@WHbbSLhBZ6gSY*+UL(a1Nk;SR6|`oe>KbsNe>_lm zIHifrWc1`nlikF3PqbH~#o65HeyM9IW_k_?0vqsp9PmK&9LekM$91X!)lieJPahO79=(#~pIH5O6YVf&mOx%U-n5b_d@to^=TvTys z9rGU3%H85Ca|8>P&OG`bcX+Q5po)idQBXl$zuc7BVPE`w=Jc|}%Gb90^=#>cQoNa+ zy)b`6E=7N0oi-mINg^Ro!wjHp@U`L9V;^$+(igiIWBfs;>?U!8Da3NWi@E-t+VPCz znMY_P;*O7dbK}H(;Z)dI6t1#|92@vc zrPjv(wc}LdLYR0J@rGx4Db=JE_!ZIAZ@*WV7(XibRs0XOHHJmh1D^dr`tnh@0&QO^59evFbq1 zLy2d?$o0p`$*4)k3;|7b)xnAdF>?9*Q9SlB-d~k(^#`fyfYnv7T$`~= z$u{T^KaU9OV`c6BH>Qxd=UD*do7c;q2S*r+>ptvdPYIIYgETp`8hL6Oj z8=j(+qHkk!d^8SIWv*4|Mi-oAd+7N|^KQ*+#^!j)N)M}M@9>rzLpIfg6iC#ak(m*H zsNnw7k*2hL5F2~hdq1O@98=fbAhM=;(u2UGc*<$y@z+L-V-Sx`lN8$ z@$p8BaodOLgr_(!xS;|*{u)xTk@6`#Ap2+mX&`4>*i|PEDgrBh*iy<)hs!u=4h=6m z{HC2#2#$TdNhZ*%O++uzH$|xfT3qwnXIlSn+BIkDQ-!+P*{k`F0B`<7u^6hc; z9V&<4h8d0OM2ebFOtz&WJX&lCJI18yOIu0eh~$McCWm`9x<{kJAKogBrA5br%V2VwSVAw?c-@@wl zBIO4bR^pc~P2kRcc`Ixi$Kk>%(_YUO!xRBJTP+`rwzr|^1keu=0bd|S? z{eDo>S*Hy}>(1NyNmUelS@?8t|kUG7F&%*&F+oSsf>!7`1D{bm8BvTe9Cd z@s4=`p&K~?6P=tlHbHk)WyNjzz3L)pxK?7fkt!1MCx@3&EMSE{x8x#Dm+bVUNci$O zju;eI?U+|qSfaCb@p-+zRFFgwS zxnhWm3BV}?m;!7`4{UV;=pICckWxc&1-($PyQz+Ry+(zu%7Xz;}#;mDY9^6PXxD#JzxOROW&L9(Nfxs(*x(Ba8>N_%8 zjlHH|*JpA{B2vr(iCrSwd3DXI`kb+XU8SQrJIZ?pVnA>_I!!~T8N96ZJH3*j zw=mF@6ojVSmm{OdG-nf1X@G%@xCH5W*`u(!8_tkmG2K>ALA? zx6N({P#p5^LSXIlj&u&a6^;(4xrL=S^NL@$!_1Rs_6p7k540<~`z^wpu99qczR{N6 zbRojuxn1kwvVO_zzJAH_YZ&3*-S=2m5Oew>31VBwPXPS~l}K{G1F*N}_z@Al`varW}CKU>$Bp(Y&u-0BIa z2Mfdv%pulAp?7*avF%inwfej4CmAF|Xwz`HNbfTHDiI^md`nLhrFi1!rDkV$!N4NN zr7hOI@(f&F_yZf8AUH56@k*3M_ReU;%ihuyNLUVcGCDDu8EpI2M@UFchf9k(X!zs7 z%sChKRHVv%u@zmE<-4qt8@7!xsC61Prk0!nGOw4r+NEDwX^|pU9*unztqxdgA2d>}du{pKxY( zwzn9Xrff3KyIVPUaWVxntUm6|AEvL7WJ44LSENi#)0bxlzvj3CZ%5(JMqEEc@i$3L z-&h6m^lPQ!{)V1)`e3DC6|`4CU9%+k_Z|+`buh-Cqb4Th0bNwUiXU z`XH|4p{0=E0j#|}<@Uwi#LR5?IBz(09WG@?-65#WKK?qL=(n7hiMLWNjwNy?EI;qc->B|c$&6K$loT$Opdiqb*@>W(^S(o${Ql_9%`Z8@ft->U z{H9we(aMP9>J{X5h7|g9xDFjDh8J*U#jH##3NMWQP2!u5-60q^cD^H;6@!?j0|oqL zYTK2_DDwjrY8|3R`+67w6%(XU6w09SM=74iTc(fg2wrc{FNEj!dDyA?N)5Ba8Hy_j zUmJEbxJ7~A=FDKgQNM&rtZGmmx`>h55GhtVjagCXyl}~riiCdsGq;&(>fb1f$DE;d zHl9{nN{c^^@IYvw;h8Bm67V}d%r9);U+Wl1OQuH$DY{Lj(b8J`K9TD^_%Movr7qD> z!LHl7st{B0et4Y7Gi&Bm%{`$0vdW@FqMkORW_J=Zfs1@%ii-c|Q0x^d*0IID|Nl&)&+dCa;0B1$c#hqh6vkeL3zFvnIfu;+sTsZJ zNaa`a#JJ@`OB(e@Wc`}mwg-jR?ItDnKke;^45<6ZEk6oDCLJYOJ)kfKHeYbJixaem zKufW?S~|K1=GC}%;F4D;BNjRcGYz5nz9R^kqBNcb$SfF+f0!Ww@8KQVs*%EeehEp> z!(I$5cm4kR4dp8C&gejnq_WVO3#`kKKHe;~R(0G!L3ww}-VzmWG&N$_78-M@ zvhd7fOH9nV2>hz`9y; zOYiaCi}2OGhf_D&QBBY|fA(=jM~$jykKx;_-1M z+4$te%oyo-^~74mCwYuNY=|QeULM{X#4xUSd5kJ;Btbh5d8ecaW1X-`h+1hW5mt-0 zCT}Nd%+kfRy*Oi$S1AxkoPYR1G#(waVG|4B$_EhOLq{J!Oay$Fi;L3Fug25cm_44T6|b zE!skxlbPCyaDCjZ;34Sh3l^M>eTgvU|JZO2uj(9#btL9?$lrs?S-%0p_fb2-5jeV9 z+*+J5d!l$Igb9ht?-d))2`iz3vFnSmXX%@nEpu?>T_mF36cIH`CZQl0c%GZrZ2HB+ z8v@kphI(P9YOjA7abs^!`S6_On+b?hxadXe2WDGdbL`$Lh&rq*9>Na6tl)ilhJ$z$ zh}2^f#~xne2D1lCCO@M4q4~}VAn8!LVa_KJ9ayc=0zgPpU`Rt#I-9lPy6;?d2;{cL zU^v_NK=mS?av8Lss6h50L{e#rD$T6v-Jc(PdnIW~59kNg??kD@f5%?NU9wz=xQ#A; z4df@0Mk;3V32ZCX%;|}Y!Gs{oZ1|g#<6{sA2+ z=yt4|FDAVk=AT=qC;NmXcUKGy*32)5!8hpU$Ek!v8f;FvAz)S47%(e~!) zlCWNtkoC|V86DelgNH!Wx35L#$!tqV@eNVI$(cXOi;@YLKZg-fKgLa*HDe{E<3fe% zLWeT{4F%+Ph(#g0l(Yhi#Sjd2H7;~@;D$+v2lS(upP#>MIlR@HsV3FKqxPG=zXMy- z6GGZUVN}&q>IrmRCD7BXtgS^kpHCm!c!a{c?JIxy}w4bw@jx8#@&kBkHka|F%JK_sg!-aIK{vO_vMH5i$U zFqLV86&I`IGwqi(wGb#}IEY)3eg&gs3nd-dwxX$8Dx1*|=RyKqN( zaC&+fKNH|D^{tGc2snRMd8T1!x2WLB|Ih2bfC&>2__EmX?5GbyhKFBGhXJFXb7Er7 zM}#%u_UR3YN{ACP9%*5H{}vOL`Yv&{A#VUIRx#xwB~=V1#7X)g@78F!_d5$|hQ~!Y z4}!CYB_?3ME6fleo^QxUfEOwt9_y5Y;*hSLpKrHa{@G}ee8sx3ITr-G6FxkP=7?Tcxq{f7F#)L*>u8?r|v zW2S_Q}pn6u^Qn$}tLzQp6U^ zvU~-f5elx#jd`0({($1RmtJLeK96g0+|bw4=ra{WXIm2cbtCyu`<*+aljXZ-x51F5 z4tBfE?A(f&%xkurI7l8q1;-Al7vWj#?wKg|pJv5o0rwQ|J~aeedbBMYqZK1WHc zC+K1gxfs9m9X#PZAAIKa9q`<+?Y;E2&F%htNbzLHu@8m+vkfEmr#q1Nts2LL9fe%e zuHeFR6h$Kn?}-O2T9@Ovi;(4gz%Z&?3st+J?)PK+ih?Upakkyp3h~=#?C155hQA4! zdrD8`a*Oi=GMghj96|4QCm^atO6Jujs15BGGPZMHQdRxI;@^WmZFx84uQ%A)c6~;X z(}Mf8l5>_;ARWuIFpm<%s;557zRy_9%Wb${Y+z|-J{*XWX024CG#5H}WmXX!i+G{% zOhJDK_9UO9gRiE>?35k{c1Eh0SaC6|wWIiLMFQwHJVkvp@FTcFQ-`5xYL=vh! zuqyQWz3MRXP9G=HWdhyNmsIx&jFYY-e36Sjz;_Z9okp|u@AW$WFO22RiLb!9CqHB1Pl2S+*L|t7x(yA&QrZgz8d+q z0ek$+q=wcDQpqnV_CrRy^TsCf%=2(q0S1ev{aU%_V($eop_tUQTT1`kjcIBoO~uh` z7b$Zhax0*_c!#eEPeTLH{jEa~Wf-hpV~u4}>4W)Kwdtt(M#vUa@v+u+F+0Yd1Sb!6 z5a)dtL=L(gUmt77I5VO2Ui|%nv<=r6B*8}nlS&UDWE?G^R(#)ZDxFTjqPQo=Zu134 zFkb!Jkgh&%f77Jl`BY=o!_FE>B|$z=>G44RQEF!?wvISnz3NC`sKo>yqVJXbAi=mH z_3nZPrD!661#u&IioYud>cP!B=>n_4y94X1?0r3zFB_tW+&#(n&U-;^FRApdrbvW;F;GbM&TdVkGptlPt;qZv*w_>ONXOnjp)rrDBgE3wH}=Y~&ri4-Ohb;7 z9yhf9)7^RKHi_iHL>uM@*%$@gADME&Bf=5eHN;IG`A`Q`&c(%T;roRqA+iQ}`9M)d zW+w5-j-7Eof^MpC`{Lr#ZGLuSoH3FkYwQKDDykIK5+u982U{7dFd`DOjv*2KM*Hpu(RpJ z^nx4{m(g}KY4JPVW#ZdfX(@B?%1#ib$X{`SxF}{NuGTvzO|-h}3Lk4Ohs(;`wmPI! zmgrA3MXU>OA(B55K#D~kT;@^u_{?00mYu$YI+UccBNd|OZ8$dcL%x)rWx zl@>t(VtdSf%$a#76I`kMQF3Wi{)E*>ll2>EV0nEB!WS0nj23o9V(vXxYKO7N=*Yq5 zA$p?&9dQk5tXPaWR+a0ViRG~~Ym><6`movGaY#7(OgGp z{P#8%;rFQNbdy=4(PGYJ`3>_FQD6ZZ7q+sarB01*LB%0v99)lumSB^QjKUK5q?ca@ zrl+3$+5NbjCpmTwz91YBleC&U6kV3H65XFG08uvLq4r=`ftbl+Nu#^Ih>^U&O?mbK;V9~s0bS*i zNg?NY6M?R$bzzrL6r5GmIPk zBg8cz!2*e>wPIe;kf`%=+2mcugebLO#F*)rbQc*^2E7d* zD_f;=u0@%t{ev||5U@X|DS?62*}r+nkPF!#UR?h@y9*uyL_?76kf1hFicV1f6ynLVi6>Jy)@`2aE`hGM{YO$_zgmHv2-QgHI!Jr~=_DT=fjX*DNl zdi-H?KgfKyZ6|`Wj*pDSD%jW}pUH}oy>D)fr9r@(~ zfkdfjxKa7^k9+Oqg9mZ}2L^6on0f4zmJ#XWIXg8OeUV81c}7+>2SEl?3IeftarO#j zvw|{YQW&Bo=xjTja4VA{0v!A2Y&5CV^uuTv11|y!PqU6CaG7F1zuH7y@;kAT=JOO6 zCH{=k%-B?vA)TcnE#N6(J;KP85~kSuq=byu%-$9ADdV<=3wA946JH>ES;4R3kXn{V z9_>i)ADwW4B!jIi;?p7*hzU@wmp zw`p0-e-w*E_>ULe7BFGdOI0f%m#|6D?c}^DvN1||j1Wp;#&7a7{sHg@{2-o_Zl{o= zH=S$}-0TIC=c*riEy0+JagS6}%@r8pNJw4jjElRA5L}*9?lG)`*UZts^$vq3xcN5{ zIqC{D&1B8bhJ9yEWpLcN?BDNKc=_cD)Kai$U$D!3`2>-E|f(z#R(R4|tXuM(>WaA&JE3FOQROGd`DedHdc~<*PdU z@P(S@oY-M?S37TkYs4Hs9%fiL(jw(fnjamyovNo7*7ms)7wP2IE=4Gwf2 zspgjrlL+yaanjMy#SdY&TC1#asT)3h2p z#-IW}37_JyCrP&WV{dCezs~&DO(xK3@m<=Acj3cjO;0Czig>>7}90td#06 zaPF;cp30rX{uAv_cqgbHEB#N7)l~5pIJC?mjl(dV41cSB*a*a8?#5^sBEp!~T&edf z+SgLqeXV}oK1#LwymJ~dv1-A+kWmRuJzk4JFkl9%Nh?aI;5Y{4OU&Xtky4fG2UiHg zIX7lbj0kBr2po&zer_Fjn}~_5VudgR^~)!T-$zb^_P*#ka0<2H6R!3JI*BGjoEX78to z5M_p`hk2?vL7xZL<&1-2nq9DjBrX<5C#Ob1z+TgY7y}N8?a@7BS&|KaJYl~x9UY6n z*E4;*%>0ufyoJ4frO>>%`P2RzGK_PzN7_bgH_P9@p)01HBfwRMpmZ2~_#&}~KHDK5oZB-fVkQ|U>BJ4A24 z+I)wD6oFG#_%;8jxCacOYHc6iLWz?9TcoX2fMjY<+%cugbSnOyYLL!yWXfak3~>!TEFT zn8EdX)K=Ls9e!fRb@iwpdCa~j6?&2AodZ6f-h@1wSTjp~qP?FPyuXowgMz2Ze8)bd z!b0osp3tnc3pVEYesQv3`pfvgFZX+Fox-sEk(K*DqO-sK@E0TU6ztrvUrw4J?L#wL zg-~@8Ro|S>sO%&IG!u2dQPKpYMqO)uFo27YR+qJZLz$%(`+j(Sj?bq!%&=0jSJ>k8 z%MfLf=VkF->KuQMjlP7i4MH7bt7ITacffHG53=7?>o<5YGj6!-hZ!YoBgqm*4TATw z1Y!QuQs@pZywQQh9O%_rz#^7+ab^e-%Vr%3wEpqR zdIJ@D(Te{uzD>1UKa{{1#t?#m+G7E%V%EGz;L3Uv|JdTT_Vw9k9rgYyqvY7THYI@UqBQb9o8qsAq3iT- zF=K@x|4a?<-kRa!dvLc0bKnVrBuKqz$)9Yb!kp?bRu~O|3BqkOIxDO$Yr*u;-p67# zw&1#N!4{B^gd(ywZTS;_a87Sg`RE+`e_!5ZEBbf8p#f@^CH`mR2Q)+?;Rmy;m1!pe z8;!tu#ITpY9wQKtV}t6nMhd*n2d7a$g&R_A%7zE;G^0Tkit@bYF$oELlo!;{H9v6i zR*9KW<)=epOkXV``QCzt&L^sZ7)?%4ny-V5iXohPzg7(vm;35pRBi8rHU^cjczEv_)^duWJ)6 zF|d8rIGs~Zi0j**iIqpu{5rLUm}MXsud!Wg*8hApmvzQx;4ZgvHD}wr5rYzTJY#>> zeDg&WS#% zC4y3tyQ`ZpAh+_Ivu|0|QyYN?&^~&&R0MU8f9+J--zV$_lO42j4#g()g2{?ylaw;` zt7HxV^WCJfJ>ZmKJ#u%l|J)ruUzAV4lV@|LJ7&D5RNN+J$e*fl4wq>v?B|&oDr{Nw zWTAu9X-18#Z7TaC&;-g?<3S|cYRcf?MvOMLvIV-pZL1i%Mq$qvdbx30)@RGOzV$-9X7|$9pX!~2WjG1d zj1XVi=Ni_4^{}kLONT}+CA{V0dACsm+FJ$n?OgMm z;U2>4%GA@-!7fqu{ZrY_AGxe-gLcS+ZJW@!V?btFuvLxL3fu7gv2^JBInq1Ax!=^n zJgY^8?dt2*@zUJwy7TyT#_r?$B$sDTnGKddI@=2YzIuK2UrlzfuSxZiibMM)$HBU4 zRoTmgCyAg~l*0JqOewP?#kuAc5OATPwwEAD0WcfQHa-g#lBBWiFj^Bv--@E3Ukm@a{jy)ChFZ_+pm{{)D7xPjoCv%2)*0VsLon@};2r-!4_WjZ_2>NM zZ?Vh7T_sSeaj~&cCrr+Z*?GkhLCn1DJ2UA}jsAk~nk#yE)MU_}P_JV>lj-i|YoNWe z+qq?S(o5VnvEEUf;%xfZcRF!)m;c+>)ghUJtc9#iSRvo5pv5~w>i5=KrKwlH>LFn# zzrE~G>sk0y@(8Q!7kbmV4e=Z2%l*iveS@fimiBklu-!u7NFwd{8SJ)}cMh9Qpj5;F z9AmCId4eys=b;sa;?gSVt}W+ZepaBnH)u}!1joPyBAX7hKaE1h752wXkyDzq5{eyu zm0dd?NMv51A$E#!`q{gl5hjS}1lMhpHZ*dylqsa~AKa{nDj0Dp*}$%}}bbyS$b)!7VSB-IQ6S`rg@6^*+{O-=I(+=FV*nBq%h$#xwCf zD(fo8`pPWpz{_J9v3aeea_V8O$j~SJc>P7yYU9c==X@{??r>{NOW-Gv3)lPiA{`N+ zhVN(vG0Cju1x>u6V)1Lk0q??(4n@5*SPsOm*|2L*ol}Q*)b+_kzaTr0;_vhBkArQ# zwYY7~(mY}#XUo~(hv4OW?+tpfkT*hO}5) z=KzecCit>$uu^6CXQwvw()2-4MwsC;orp^&FAvyAxDUG)AGYF-ba z)p?dSa-i@s7E;1EIA!*V-<{*XSjqMmE8B(`T^aG3e&bFiwi3&qS%~&a5Y67zlzJq# zw1h!ETZt7E)}p2vVY-`e)CH0DEhNlUIEQsA=(@nFT7;LGUYoHwATq9|&lIT8%G{(G z-F-q9=A_kR_3kdbqY+3VpJe^;#|&+;5{}g|0@n@R4Zicp=gaQdvcQN`+)Kl^HB#lY z^b{#p4Yex3=fCb3ES`pkXrXaZqo0=Y?nHUbYBxhrJEYkbl-8?6XHRuhjl4o3OHD^-FYl@XJ}(>EJoKiR>Q>D~HHU}ZA_0yTX(Cy=(Xq^S@v zskm`|a;;c68QRJ91>$ldzmv%S)b~y3+sN~3IVgSQnW}4gQ(W27W!W_08IbbDu}CY+ zCqgJIFOMGwGg3f2D8U*h-;(}+Hu>}Kzg8iYqD>~xXeY*)!(ix#E2fwtnW*O26`2|P zT^wwDW-{?mpSS7_Wz6b;uyY39JL5qJjJE_Ph5zuqHssvU4e|1X$7j|F4<1o9z=~8+ zuZtS&F)kD60$-;EB(ufnjRh1Eczs^^X>uB1s|MbhP6B{)zhT$2gvxJtAfd4!PAxhV zA53{+M&q&|s&myx9P&Uu+@JY0H<+*pI`JL1dBL}rV_^@hgzOsy6duh{*)oAk8^==# zVGrw2C`Y_Hy;*%4*_?YZ0xjRAPFH?oUg->86-`Wyn2VeWF4g*y=5@~Kk6(UKZ?fV( z`i_y8`v8@$dFUzrz$$L>7YZOKk4lL+3t9}GBws>&X#W>LKvu6ZVQoxsiXMMXWqMc8w_nRMlfFSr~O%1 zP8EXLu!sh~rb_Yo8w7SiY6Al5xDEwsdXCGteW!xw=LQd6R8Ue~HIH5oP79#ad(Zun zoNhlOUi^W|!MuBC$lXN4XDv`43wRX??^KzM=#(X6h$ z3j848)oM=va##9FCqwd)sFTf~>L_gbVX;9TXZJypc!RFyy8 zOi5Lf>?A6nT5r!7hM3%o9gW{K{D`C}BA$W!;SUCyb*iTDZpuAJ*CFaz=f!aF(T%wD zSr;dA%=z<2<=ZPtZ>M-wi^FN?2BeXS(42y?D3e^77=m=U*)4d>rJB=Ghq5b6ze$1- zX5XJRb%1gf`U# zF(vkpPn`kSJI0u7h(K9U-SrGUEClK?T=RYH%p&m05HHW3`tkgj|y2QOd~Te1XPTfc?^>aqu+ZiIkTSEe6fM15p1+JxDyG z^SmIR<*&S{O>Z}q60V%m$Xrwtr*c#JEhV}fjy9=>*^xIuyEj7xF4Eq#N>PoNzSzpB zdE^-2lE@#J)MLPAY&%l;-TD*fiX74n>;zY>LyyA^524AUf_5LtK(Wv}G^mC6S}Zg& zPl}&eja*$5Vjv51(UyXG_%W*V?@I>*`?O|W9){zWA0ahsL`Nryu=y1%L%R%>kH>gJ ze9Z{#9`LwR9-(yiEeZ(mxGAMB&uyp7gL!B!AFDSHM#on^dC-T=^J6CEQAid=+-BsR z21}xke&*mzjR#jp4bE?G@#;lhXQwQ4(QrE?zTF(3vWA&m++NX7LT_457m>>24GSU&jbkbSg>QOY^ zGlfJCT#DfPlS5dhNa>f|0*-b8<=&ncrD!2U_zJcdx?3?CD&Y|`QplvlBvKr20-B-w zhf?R_kp2=8TU&C%s)dgC>zxmu)hH))ou!*$-Ti@VtCR=QEUZZ+@*D5l5BfR}bcafI zOt&zly67rq+=b75d{%|{3G0SPJKXf(GCWE}B3{Cr9fWD3B$)QI%TJGhPL=hHDx<%c zX7VW3PDIm}`~i_ThbHiZ_=uK*Q7Kv^i(~Wo?AI~zHVw4|CBHPHL^+WI`LSx(qV#RQ z_GQ8fv;@3tO~uC=9EK8&E1vH@s)|y*s#~INiXX4d~t;+e+}!g+LA@0W$xE} z!HuBR&V$0|hDEMTX=AFxv907}9t7L^hPoU&F?8@3ma);Bkz>{YF+!2f9nqHgJCr`w zji9EnB|r@_uj+>Z_xUdQa65543`4vg4P~m)p&iA!_dV*!5q5^~ksu{Vn?iD$kS4Gc zvygx&JURK#F}6w0X|s=`Vg&npAK{U^LfT<#r zSbGqv$oN}mqer9AIwk{geA`M)jt>Ok=V;TS8_6Yl=q?CtWrH zUCdukY7%R5%z13SUR~lgM2T`Z5afT02kh6kE@yUg-0kOm5D4_A0AXPx9G~U0HD~=F z8}tdDRS*6hz~w>L(|unNx*jGyP%37pYs)ajX_6lsQ&K^5&d-67`qygL5KWX+a~WZp zr$Z`FUDllHuz|v}#l&}6k~8c$g8C!GtZV8NpC-j0I6xD7kHUvoz4Xq(8uG}Ug(_$X zkE{BU8;~YsD08Cw55-tvWRW2``6zncC9F7wbEW*#G5x9^Rv!;fHgm?@Lvr8Q?L&WW zqez9y6sQ9bQ3Ho@2%2qRTp~gC6tZXmQ|^wya_n4$fNi0R6cj+ddoBR(YPTpC#Zf5& zWVbN8>WjGsN+6_B?6!Sgq!A0lbua`4j~&$C>b zoDFzP4g?@jH{$*g)v3Ft(2-U8jIr!^;{3^#tZTSOm3i$Awu9x2D&k(K;~A-gMI^vB z0^RWRHE@XD+H!-=SAL@lOykid_jHEDirn=Jl^c}WwN$NlB$CC)a!c6-{YRsI0v(+0 z*X1_S4O?ANdLwEIb2fYmXI`w&R!9ns6(gWpqp&tOe%0iNp5*jS=eHZFKIpvRSqKR( z-w=nQNVfs}pt-xuD0DSDqI{48!`Zenyho6SzdLH`;&JKU?RO+}HeT2ElVIEV<~CeU zJb;&>c$b>Z_IH}xogNIB5IB5J0~b)gVVZFRfLqzIf|2-I1d2_bEM;X+88Uy1Qo1u!Wo zFzK+NvD=EV5Sz=rnZz&?c1ncS$Fm9O{#eK13QA9@=Z)3z{yT@&n7IWta7sJI`d zm+FR7nz&?an)X=;q|O2a%sU`iD^LF+=?1SFM+rtx<#53_N7uXO6_gMXWiw$uLArZs zq)n^PDdyebE8*Z=YE-+vGGa#gRb4F?Ikx)3w0@NDViN?BL(}8yJ5cug?h(>q-%h5>srDx$b4GhV zjA)rM5XkLLy62q8Z)d;n7u3xRz=MnnVqx7P5nNO)e4Z7z_3yP7*8slW6Sp$04rN zAqh?f6F2_YYWIx)6cf$g7Usuh9I+VeszhW;S>3WFSfv*~3p=UpnL4R&*PDhni$sc> zIfhX^kx@=sX1S9d_E)UL%aA?S#Vh9G4#j3!mbmuUb(G9@DA*+L{1>~$$UrJ(fo~}R zKOBqpNU#;faqGh(^-zolK%`lNm^?>>~PUEi>{P=s$OS_;cH@7kNB$J$Gf0-Jz-TAk4Ym&TfeI$Fp~j37nR!kQl$vx&Z4FXs z`mUfl7lm>Mo!GoAIu7={O&)Y5; zk>3gbn6jOSBP1Tozj7yOwoM!S7oI{mXv#gPZtyNZHAk643Wu9-7M%%8w!bR~0l^Z> zs4BNO7+Mz^Yh{bP0}UEWQ2jpCR!VdiA~;=MdQID(5_sLxVyCCa`X0FwSH{J$KwJiy zTlrbvK9Tx+7fV5!m@2*&=^_UrVo5QZGHaCJ)+{^bsIGFrCZQNo<{e&5l8ibjN)0Ls z`|n=_)q~myWo!MSC9(zZ8Zm)Gv{hr$U264+Ra9<%fv9j?RSuRB#3oYkmr06=i^B?v zivC`L(7Gx^wQ5~?%3U)0uDu49AF|rv`M<)nF)Dy-bDMs^!QVoTEx{w{t4a8o1OSY< zaOr2~!NSENqo4&9o=IvD-lI@e0ir;@+8GZG8qv)>m0RUuEkPXb z2q!`dtAK=sy;r%#RoO2X?^3j{K2vGp`%?Ej5*dZi*Lv(`>8HB+8ep^u` zCDh1$li{ZrV}*&aJ82Q^1cATB8Vbi8I(`a$P#UlBFXHCtQR2VhE$aY@nML08&GWKz zP}@`PRYsW=Mf3b!T+B#{`g3Pxu-yxG5*9-p7L<8)7RV`tFNue||14T+k6a zg&QY~iOmPmV)Q`R1${?*CI`~7Dgk1`N#)|j+?58rb6>)!_I_3&_Ge=&xkj*}9dp2e zPh@4MT;F8T>j0RWF(J8Pw%5Qhe^E2cH2aZ>>VVC!L5dh~=d z^K-U4=~G6<5cM64gr9PfdYc2Rr0+F6?AoOjnG{)nu(lM1{q0O_2&IlE8U2Kr zKj7`xI3Rc3WdH}ZD`G!)E)!dwRspRr32&~zqcci_PM^hPun`2`h_Y44P=QgaPz)AT zawQTi(_s~OQh)ZTs3phe^*!w?Fg)(u(9f^cw7KB!H zO?b<@P1yg;l6?EU)0<*<`O%Aetk_+z%%sTIyYo|I@|sS%UChFDwS^fHgh;Ix>}zap zXdOXj<3)>#j}{%oCBgN~vhduX{Z@_COH1%@rE$Ebd?ZV7{i33Lv^6~I30Q>(31Mev z9}ab9WcixS(w23&%mQR|Ed_`}k#RLR(&a5QYGz%^ag18B8K~l=+a8Ce&Sk}782RXd z{r}d375~7M)&Bwcn9=#F&I{q`GhvAO4`|suE322zEpr0@ZmSz@W zI#)AH^o4XF!-eQ2+YsVF%PtqW-r zPLXk|fg(aQAWnmi216i@E^@o>+yWfPwO2 zAi(#Cq3v#q&du9^q9##N76V1<OY$YEp7E|<5}k1v||%xuTO%yKBHD-lh?KX43cyt<#V|cy??kqoIYoQhk%)8w{0fEyMTL0H3Rs1jbv1G z8dB2<<>t&j;q??zU!Ri3ak zQG^dknh;1wKAu%2_A0A~{@6Bow;T(uc%h$Pj4`O#Mt)K#uP5ynun-=ZU-aohSkbp) zwb9>Pw235*Os)tC=BiSwz4~!Q3oP$ZVbZUXYL{PZ_a#I3=wf=4{;QSvw}gW)4F-a0 zX&eq6%A1#pqKhV(1w)T|2z?;hR0ww8Z_==NL97B-s%iOs`}iaDzIUsY z<9I{LkQM>@K-hQ$y1?R|D4V!Iyk<>>jrNxYcJI8Kg0T4%N=3c#1&e_wFM|v<@ zK=&}FVa?7C&C1rwxpW&rN;r}ODyJ%FunK9PNHrYf%^%p;12}q#*qIW74s6EMXEr?$ zH-2yE#MAfN8Ul?u)@Vsq8b5ODUcB6!WvbaM+6TcD>A6v2mR>%Cdq>Xh5_XWH9f;C7FWKM`cz zJrswlD>=QZ+712!fU~r*w)Q2~6AFv3xNBvrkJJ1s*|;z~@Oylde1BWL_s6K?UrYt0 zlpLD|3l|i=!>yP$G%rpDj3QBK3Iqtx77ZzNgYiwN9g;kZ1c~C+C;ceJ$_SBo7;3Tf zjOO(nG`VSuTG<)aqS;*4Hvce0ZVCC2AVT3w48|JX@2Ue)EY*KWIy_Jjlv7EFZI%WCrqF zzvbM_i5{<@5{+?Bn%M>oG`@emTkjsKgLJ1a#lA$4K3VfDTDi1wn*i3IE*;x=UUijIi0w)bXrYNGk4uobrCW_7!OThx1PN7SgN z%j(9D-F{-R?4}?G#F#|gF8X< z6ANn?$tDQcdgqZ-P*)SEUP98pnRcv^w&s`0EJTzPE|t%*NuGrMhABM1K-PPomFrZlE%pUI! zlQZ};YU&7jo>_7`y5a+Te`E>ysueR>$l6;~^7+r=YynWfnJ`3FFMP1jC$vc94Mn>D z%c8ln!ImP79Swjsvs(z6jrczl^e1?aatMdv!JN4Y2@j+hCaF!UaG-0*?KVLW(gun4 zSZGbBYqT{RHq@s}Om4Dnk?y;Q6E5Pc{ea&x#>pT=BL^4Yj24(gsmIWtw^tTt(?Gqp z{V{#1`-7i8B=_}Xc~~@O>2)bXZDlUVaftU44j7bKa#5yruz$8tC#kpc%lXS2Y9X2M zFNmhPA&CagtV3H6l^1fvS8rJN1Q%55w_lZ6KB=R#CuXfa z@$;hy(wFnroI`Im-R~VlSOYQ$G{GExqtSLsXh^h?Tv%}u-aAh@$yBS#{hFHihf_4G zn;+|{;mEz)@kG2sOTZwQDQy6NIxKRv=l8S5gM{3Qi1iOgr)nh5L`udv|D{K&zMo%0 z&)LrWZzI0vwja5=xjza7(WTTn7VPN{SOfuyF-`2KW#ynUP=6qwMq28|Am!t*dk8X7 z%Iq=sez9plXCa6&{!!UtTZa({hXAvvYhkmsZlu`0t4FK?AzN5)#rfbKzHa*@mDo? z7(a}d9L4^Y%d@HemH-|ZV%TCCu5=+Q|0)4QkYIv%-pK07>-bw{Y|9}8QxTm-a5&F(*1FZRwHgtq*<(P47vA%lz3J`=}Tx^{2Vi+70Y zcg@e5>upb9hy;0_J;K&pMqX(A)>hSIAJay4v>~+VJhib}`5qGWjY`{bh8(p2Qc{Jt ztKWwMtzK`>c}c>$&y#Fh*}uv5{bXwRY~qNvLtnc4MjJ5kCp z1kUY$tZv8c=Ru4(K`9(Z>A$~wAJDHI^xd`@c7w)Btf3ZOV?1!kf#l=QTEf+m5OE4X zTh&~bRT)H_7Th>!Pf#^Z?LLZU^$aRgBU8Z_-oOSz77~F zbxP&EfEu($QaORAD|t=J-8yFg+L4c?pCx2WtHyoj_Duf={REgG+{Jjr(1fD@h>|~9 zH@JtzYP8Aq&-neD!RX>2=7E^(Ns-gcJ65I*`UZaI&yx#Sb)-U3?B`Lvi`6?CZ97Yy ziwCLIP8qK3kD173f}d8L2zYRgr{v6Xw7bA(5Pbqs+9}sqkyy8vCm{+FLL?nf);zS^ zOn`bK#Hkhy0+GF0UfA7N$>5ZFTu2Tz;;PA5)w|up>gbLLj6w)Kl;Y_IkBuEbsyad! z8}NYr8OX$7U~xpV7g~mKU$&{Wzwe?dTu>?qtVu?BEbv~6)+0Sg)fD`M4@M4oF=`nTO2K@S2ZJDC7@mml+PgnIYa%G zu_SqikRzeq;qOKm)0_TO9bJ++_!tzE(T9Yde?t<+#TYi7LZzq|VrPL?0I;C0Vv%XG zBt*7glt|1?Oflr551VBIm_*F4U^X(Ky^#A5vB~Mh+8I)Rh@Knmj({oz_WF$K+WKuo zH*SwHL{?KQh46<BMf?j6CkjAgwVyCf-SpvZ`ox&7gtxv2LSdkRe{Z&oQ>$JadXWJ*Q}vAS{pPmFZG zV)pD2sLjSVMH5Squ!!%v4i4@NR#WWNGwKK&zg^m0RUsp3T6VWZoBzHWvTGwr4Z{?N zg*_?W7%hn&?^Q>BB(bicW?YKsz<3|KP{?Au5Rk(L^E>axWt1oG_A5dI7MxTJxGqD? zmC?1@g?oGg?3c_g3>HZjcQNVmBDVHBs1ohW#KrN9M}I`yq-OiDORRC>4uZ zJUTT^a{?^9wJHf(d^WapPa7)4udA!e6i=O;X87|)egR#v=SE^LJz-}PwD=f+06^D_4);Xgo+&qx9WEa7LBs;ztB`Dmhf{i+|iV>I*nX-q?1?;|p4@$2O?taT-E z&gmRRVAcbou7@KRmq^Fty;K& z?oM$(>E7RW&VTh^@Z{ktd1cKt<``p+1@Kf2qMnHl3;6a4U!MCCe``1YX|DgsT4*fa zcQ~GWm~u}6=j_yL?iI1ITzq0cMqAYfxjWkG_pzpsAAnnVyAXe6rpKaRWIz(Uq!CCx zw(AhsPnHBp3V4LN;Qu`%zfL$bjebt@l*y`IAnt-9Y+}A)`1P1klut=$`xpGjt4`6A zK`Qr^iN)1Fo;1~rDc~io5-5h2C@QcN(3xITEEL)w1LwZXCk;{Eg6g}-x4iP}-RuWe zDW?-P)1eTqIM~uc(UL(S}EIo^mXi3TD55~O_vCQEXofi+W2z-geNQ5gbslB`d19;PBb z58{z?WwpJOhOE$Rx-VOiA0pAEF$!Z;W%euaYO3BQ`$@2~l4WSb?rmv|Pj8#$a^bop zV&P?(TD=9qG%Ko#o&x9G?@A+;zS8|NNLCNXbn-d?d*@ICuPUAv&yEIalfsGg*rkhP`W36 zpUPK@=UoX$Dq5Ac2>~s?Kw6z7NEr0%Aui&kNf?;y$W-i^+4TauChL|}@>py%tZ-C0 zP9#&7dA^@GvG+2}9&TtH-0p=mYZorSnqI}XnT0K~{Ji$6OTd+p?&>wLGD-k1+e*?v zL)`L^vbuT8N1AKLekCM}hCEbVa|3?l61Wc2C9y`gcW<_;wXk7-4L zcW~r)4HiboqA9N?nUxow!X*AmX3Qx57cz>WF#wr*u>N-bj|+2*2W=7sFxD_fH#FUw z4f{>XZ~FcW;1jRX^A*>`)Dla4^wal&Y5|hGQo;3xigOltza0B}vwoIFQuTtT>(+)r zM4(BQpl?28{+u)FvpUThyw>|x+#{UKOo*l7kV$ie2?I+7grfRsB_8|0i$Wx>L4 zWp=wMT|f6RTuh$0T2UA?MT2NT39?cL`N3C|KUMj*_8T?aE`7Ze^}~G1AbYD;9f-!B*m(BEYs_QJNTo=1hlqi8_iTE1 zLzLdwjg)BhL!RjEu}s{>>wz5TytMN$sT09#UcB4cySxitEb0IC0>D#htY3kmUje&w z0;tMsBn=+fwid<`cFQe468g+-zDJ}yUJK3MZTI{1XS;bkE_Vw7Q4tAHWYdl740u@a zBZ68o^-B0qJQne-(Wsn2I@i@o=&ayAI1F}Pd@ws1B22;K^Bq?@Tiiz8dKJWxM%vOw z|KUke$e}?aj{3)Fi?%@&KGXjDsr4U==e~l2LHOSg%K@U!MYPs zcTbc0hD|2OavKJlRlO64FSTz|N&rq=P`Ue=)FT0)HF9wJp*fXqLL>6hfC378zB{m` zg4AQ2`bX^3@|{03Z$c)QLKx4_J&rdQz`D3*VJ(@678yHI9I?h2QP6Cd!8qB`qX>oT z6g&&9o^0tL<8W^1Ar|7ii0{+TgpF)zH^96v4}LWeK4kg}%rcmA<2U$f3LiS#OtRB{7|mf~VT(-qpmpBIr=yPt2Tl8J%jQM~gRQ6FWb0A4Q} za@VCTrBlead1&&q{P5GuxfBAYULL5Lb(Z@SJ>ao%WQ78yMLH6hbEM=w&uu4xr1;$x z2n~mB{qU=4`r6sz>j#xAD#9z>t%VrEApCKPvXx`9q3O%%4%M*XdLls3cgwbODGy#t z2iou-6G|U66cjaI2gSTbS1tJ)@0xB=Dan$HM z40E`w^-cV9`F-AMN%-%)%iUODZXW~#H3mf4Mu2R|XzI9zdEJ{$oe4(`Kxi5F zP<+MM*;bm;KUs}Kp3ntp$U$q2u__h`dHybM|Mw+m*2A9mo2Kswaj}fY@ z;h5VGt7o1~M85Vo+F!q2(%%HTtbDD1FAUaa6vvI~y~Snj)vKpa!|B2$&olaQ-4Xi| z%CO6AZ@lbvx;bz3KExo5;;wz6ZGZ5v;OYwcyAHS6Pls2KCngSeX>fN@6f(bQX+d&f zuR3q1Jr2x7ifP5=?`}4BCOSNJdg;JiH0pVK_GrPPLFsWUQP-%Ebm{yjMaMYIkn91( zFUQD0TC+Udy6a#*w_YlN(|;XAzQuxlTx+!KB;QM-)m?-6InEn*s1sCQ*ify$xeC>D z3QP+)!UR;wQBw4;gF!BM^rQ{cp$n{k@)y@h_G6i-HoW4Iku(k};ZFj1VqW6)?TirK zWRe|C*Vn*x!Np_5KKSV!sqfu^r~Y5DpjOp=OYBW#=&ebR`uNi&6R=1rLM=Kv{VOGG zJ^V|H7M+dXohMdc&X^}>p$V)~SJbMCDcK{nal)ZPM?3j9jY>0I#zyl>cJ7VZ-+Gd2 zTt@Gx%Q@1T-DY~Lwcsik_8#gdO+ReyrPz&(k&7Rd9lN|Bjx$l^agl&2-UGdg;SrnJ z+btq6=*hQGBYh(S8Vr+@GNel^g`2%TB#tQje;VfwE%>0b;V7LIT<{k~<+l&(Ft${$ zW)A>61B$+~qUOK8&vDAf%adKw1RZ_RfnKk=ox$^6Es-2W^)iNI;r1L=h|?o^vEC6} zp7;wEjn7|=fagVTsG@ZPvtlp^nR^pEk(D-kzT4)>NT&JYE!DthLoK$5A=8b-aB~l- zV#Yf*n4qsW@}sl{+-*Ym+ywtf4nmY1x#dV&?u{NNKI(rXzeZDS1)dMcU_HYX*r1Z; z8$A4U{=CDB^4W=g(Y)WsS7ZgpDBSktRwPJpmL8s}3w{IB<_e}D8W-4E`&E~;zVu=f zT-`#eofbBgy0`XFg&%I98C*x8>2x6tUAu@vOz-YaPV-T2Eqp;%>)*Ayp)>bIG==wt zhF?*A?_a2mC-tGSS-j^Tq}Uwper?`btr3hXas*6aLmf}e%y=Xiar1mqfO!;Xrhl_l zbTjKQzKS2krIpsm*RQ{l!rpS;HZ8%V$Hv?_xBxTVvRiwtV&)yvy!|19Rj7uen2f;Ys*2Zx><89uj0!8jCg1En(mTo;6M@VUW2c5{leLu z-iZ_?SYoCSgeWNSNcsp>bSD3(>pAAgCO?w>UGdEA2Z8m&GSX_^YDhSo`mEr#axhmS zNIL7m1*~oFW1#c3($am8kT<$VcOh_JJ1pk!677PkIZ7_Es-w^nD*B$P!W*p+^OU8~*W>aZeLLb7XyPFE>ub=re4xR-g za0oO3n&7fE3dwgHEfD; z$og!tLHWVHu}nvrS*ylqk{yxA8Spy#DRn@uT}@hRI7<3Plq9a<=*y zvtdTPu~MXs#H5(x!X94pkS!t-IyKgi??#b6B}CetRPC*pKo3ifXzxG3Mz+?|Ava9| z0O}(dH1C-L`U}0gk&Srxt4@7G0z4;(-tk03m$Ib(gnKYvQd*ZxQR1LS4q89&soPA4 z__4?M?V&q8zn`zu!;^Gm?EC1r-%&_%T9=Ndo2G{L#VPiAZ zE1u6Ucb*<|WluZvdj8(6XV;qBm&gdS1v(WwZ4VFi5y1#2z&zLJRKW0x2H@Q=ynT zF<$`QGg$DT`RXUN_U!+edH<*5t}*38tTQ+rp-*N^vX175+nJ*` zrO+1?BL34}lf@nMrP@saq;ZX}MppGFDG>~y=-7f4dH2#zH0buySObe6Ht%1mdHV+X zi7pzlou~QyrO-5E-)~Sp-Q`7oA8HYbQ@=}N-3vD!(9xAz5jQ!ABR(5ud4P}CicMod z2@A*&f#2w81)H#S2~D&Y2(k+*1(=U4$(^bCSgi-` z^VB{0%f*mCt$EUzX6SSXy0oyVvcNdcNxzlx=g`x-P+hV2n9EG!k*4=;J=TTS}r^x;5803!DYeDjA!SN zQ@2Z)Vk!5cu5%X@4%G~2E<^F0|bUVWrgA|Q0UkcPJ( zd<6J~wr`ue5!Nwm_FRAADQ8wmzh4%)Ja8wKB6Ndn%*owedgG{9DW1WiJ@?#CUj^G~ zgU9B>!Dg@7!-RNP$QM`UcyD-|U;Q$2def})h>5%sA&vK|gxmz+%)u_-9QaC%orLN1 zyUS*20A-iM%b|Sww|0>6XJyy~F7Ci8vLO9{2VMg!Ci0ftqagaCxS;f=3!zW#H?&^_ zE_ikvecUu;5j>ikcx9_GJhUza=bfnJJxjND8WQ)9)NwJGvc+%{yjMt46}gM{W(nmi zINcGA{-$nUPLLg&EJ+N*$X6gud^vO@k~Oo=+*_;t`9;bR&C!3oDs2s;S4;{@Onwbk<+#(~JV zuVJ*SqCsR2$|kvm|6KUIvSL$fMm8ywVQ`-yh))fG0nlop{^5v-38J6?BD@A<#OeJr z+}E!PoNeqoHB?NVmAN*R5G`>mu$2Ch#%}lS}Ns zfz$V_hFnGaAwCozcL#npQ}S+(S+~3YA-qwDwK;a;#T!TVjPT8=C7y(klwe|Q&e2UA z^=em9L_FZGwZ|3x_(Ca!vIa&VWhZQ>q_JpSPimI)$<&Th`+2~_Q-uAD|M;=FAEIr< z^s~?4x7H-&0nWkXz(k&y75tL_8}jj=g4g`7;58GB5W~Y7LXfPEfJ_E3kyZ)ttr7wR z3263y-lXnA*WCoX_*@qH&+wm*I4e7*L;J^6x}yZ7=egiP02V5C zLr@w#+2_w38}^zTdv%a|=rC`fPuCG;iIcRIPdB)M09N`5n3Ea6+KB^MjJ53QoD8Ma z{B1URf(*h-A6JyZNpHm`)U1S?RbquBu(KO5kdczW)?Reo%Tf-iTrwph`1TxcWu!>F zu4n?(cFFN-t7OtEFnO3nO|ePFc`P@Z>RiwEZ}|JPNlYabNxE-8A^Z4W@zZIOtS!mB zaqzT8Fv47p&KN{S=S!jEWhH%jzU7xw(TiRP)!50t@`G^P{~-)GQwa&@)u!Vr!E@>` zkSW_S`CUqR`{$B~X<`Ak+(4&U-Nv|v?IUdUCY6Z+Uh$AYK!IN}U|lM9e1*3lL|YW? z?;7zBx*Fm;H7iIx6rYD|1D!nhj=T+S!g|cyq%d~&D8r{0z@iJGcM8I{k*}KM0pLBh zVUO^X*V#4PeaKT?F+JVqthp4T$o^!n$u6+lHP>f7eXI`Z#y2z2)!G{U)t90;e!Sr+{ zxIPY`psy0fk)H_@7ftHH(tOpoRuiizs}!e0mjk}&0`R87?sDcX zobRYFy*{Mv8J8Zv#X-U}k$7P5IDB>iuSoN@L8Z&UJUth}>3s7yHJs=bXab8zo3@?` zP&IT^4G^OlHK{+-5?qRH#pP+khu@`gMDAd6o?p6AXEUFs{*GS5iyS;?2o&-Kkc4|U zd1;bkV7wCzAIeUHqdZan#nG;m#sAuM;q7i99~5=~_xq^4YitAi{Ivr>TPi~Hhrh$s zDY(-B@9}DDtL|)O7wddrH;Tvnz*+41I{{wxE9&{AvDArF+>!yzrWTN|zbgelx$*n< z@3j8D7*X2Aa1`|>Xom|%%GdAs;2Q^#FQCvXiMge-BU$nFCmOBE@Nn3BTRopIUE7q~ zZb8hJ@+0f#*;-M5yb(k`w+U10zF~gH+`;@V7ITYzyM8fenM#=zn$o@!|DgQFWoLAF zF?5?f4bmIl10R#Xw+xH2mtIOxnzt&z!WWXDwzOibZY2Ox@uPoxO->?Cj$ODjKD0qh zQ2j8@JH$iJ7**RuovO1>UWv6g_^ilL}yH+=Z@ zY@?6e5jrPwopw_|?Ww4ydTfh<@lH^TY2%0o~r3R__O;V(w5czUfV2i#kk&jpKYg2?gVn_9{+4{ zGA0|+sVxG9&p4^c4a@x}Tj{@!Lc$*CHVwAv}bI)YW;MBq@KHzIe}H95cdN9$&-$$Y_PojwoIWoBkzRA#{MD+X4g3p`9j$lR>l+=9T$zC!ja^&@;1 zL~L7c&2PXLGV$Ur#8uYzySJi&Eb z0~^6waBdKMvTR{3yOoM5!=yiby1sip*!&s4#vRVrp)}Zj?LSSsSxX^f$Fr!GvWrCa z_u6(raTx6#kMUlAsA4`p&&~ltBHyvf_`l5Yu(~04FW3#J;NpH);uCl^FdzEfEC$WI zQmVLslE%M@>PGAZ(ezSa{1>swbEf^?pBLO)2uu!zIdgr<9sK;pl^CvJg#Tpq*qem} z+Yb-+#dux%d$2r{G?PgJd8=fT9VEI9x~QMv+D5m(7e3);BF|j>1iTxR!(z7LXCqv= z;p*b`Wu(KNMPIQVav|I`Iq9%}d@QY#Dqe|6H)p2yvjCJjUY zjR#vvYvMPTV9=I+((X%wGzforvoXhqP#rNU-Kq%BReDN24ta2VZ$#l~&s9bl4^AJy z8Dm}DZXfkY`$rOP1&{-ZG%U)z%GJj9B=3z%5LxeLJ?<}rS;YM)XNLkw5GK|*RZeS! zP&BvI#a?^jYgERWxcuRM=r(RRX&L8?7KMI*bs58csYyhQH-DK|!o(MH0rZ(|)_Z)n>^hfx)tI7f z{;_KPCC%T|-GumB_%NZr7qjJ;DwWG^qsS)^0(>;8L10SLA!h$`k;UvQmT^-TsDX7y zho#nm{rcYz^&XJ=WiG^s+Wd&5iHklWLn^h&r2An?Ay=GwjR?a4NTE;RkGQvB%%-Nm z6c2;1FRz|?xAnznGQxvH&E^eUfDx`kXEGk}PQRxAEdDi!XWJIAN)VuQv*bTi{XLwU zq`})UwW(yxJ6MR?{px`H@5UOD1ovhSQgu%3xKpbNs`Cph#TeEnMCL-Eg0=|kBJEH#^uX_q}`wRSk zBSWqKz@i;-@{$rN7dGl-xpHwp1v)@e+MJ(Ls=CNQtws4;ZhGm}Dw*hdGiDqk=qP^9 zQ`ye{UK<gjm3_v^jb_pEOeWFUc2osfW}iCR^r}RvO9Bl*YlLn1Km~w zOhC8Gwnv<$aH~xQ399JDX#FpZh(9|T#aYDWc<>1aA=X(taAU<4-bZxn^WKlU-N)2G z5}OUi+j2@{-A~xV6%pB8QYt8CZDd4p5RjxP3T+74pL|!SiUSItJXla_m^M;a`opVb z-9YOiWEa;v5Y;26Pwg`&T+XImv_0wn9OfUu0)+~He#6Iw4%atAuSfMAhm2Q3SL^cN zZkv$wtn^sFIN*>(kN_<_T*T2#m`)?oaWl`T&-Pc3d~pGhI+OEu}!Lm5^TrYm53;7G00!zZ*u}Z!KeqP_a6XBS#FuQFd`(bb;|2T#GB3WLG zSrE!5lUb>|x&y*GzwG3LTLlGw7%BMnV7=sCQ$c?WliB{1qVn-`6KvMX{hQf&t^uK= z`%;qLDNOkE{0MxeZpMcVlI$!Kmo(A3X_DMO?dv_d-G)^+n?^SvxfxxyYH7zoiR^IKq$I~t)c zVSp|poKwY-t53uFcASJBCfmdVKq`{6?)ApCBsDVd|bCKG)gULi6q z%fxG1NWGjkQzEr^&eUm~Pn+w>se*=PMa6ia(wl!`w9BK$l#{0)@6f-9zCAWHc!&`S zNrZ;MaY_FS?oZo<-RdFn5*&Sdm49YkG2ttU$E%AF(DQesN7uh%z?YY!Z)hF{bFQbN z{Y-kP9D3g)Z8Hl8+>RbLgdFB#8TDHk6;l}(PFF|`nV*dw|N7W7tFCyv)vDABCZNpA zG$1C+;6o6>2<38L+#rCMUac9Ma_9;o5@`Iz*Y>o@YBC@Iod6}G@yu7L?AT>eyr;>k zT=*iZMf+PQ>9Y$dmtbtc4}aqC*z_!wG(jbPy2;?8Ml0fHql5~%VZ>aw`9GY6%LPe_ zL7wFtx}ucbI6^YUBI+kaiIK_YRUc3$G-;~41?InT<{z1Ixns$U$R=EcT-&2rSQX@R zfQc@}pi-Rj4Cp!WjFStT_q4_QpOc0|HP`9YtEA0wi-4vx5}kDv0>)pN99s&0E=4MS zMo3w)j2~v4k~NhltzL1^ee;LT7Mm)HH+_UdG{I%bCMUkLnFBH~_$xM2DVHU+RlHa%yHw&g-b@uU2-K|)%&*JCL3a6aEdQ@j| zUmLzM8<5tc+5foLu^7xk;lIp~>uKyPK>M|l&}M~4tYIOz?0ReSR4^lc3wPsBhu@45 zb^#yC7u$7=kh2>5j6!p(Vr6sWjkuCCQ$_6WJ~$r_^&H z(E>xDVe2fPUDMHl%c)g=uAxny5x#0L}EaZ(I_0gzJ z-{i7oLdp@Ol^AU@8RFlH{NTJipYtc^%|eU^=U9YJ6FYty#alb^6i2&(6CVkqi7S)O zvACUoDe&b>==X0uXNA%&hG5F1MibYj?yAUcJDv^fb761vROYj0qYZ_z=W`B+aw#RQ zExqfx%g7%ZS(oRPj$b#hrQWA*eqUK;JDO@(hopYE&f+zle!b0))nMnJiuJJC6C|yt zeKb`8<6@gjpJR&4(&W5XDPd-Hnd4$xO5g0WZJ*ufh@idFPj$5;nNy~ls}lY&?_Ww8 z*2sv23*A>|XRX|TFtM|1t2bqwr3S}to!-Ey`5zLhF$D*2Uo6Xin_zn^E# ze3m|%2u@1Va@zJ_paJu>RoW&*8x_Y=#Aywf;+K3Q!nGRayFe@yf7n~Q%;Yrj&}bl( z{2^9ak9fYWvzw|t^C)GGU~-w55xPD4JS{@5$$&MOcVtoE;%v> z^!)ZV{~WgD;d2IB=?c&&D#7OP%TG$v%-(IuV*yk3wT)uuzz_d}849{Y?9e-}J9{?XPE5RcBpU7L*K; z4XgZ&EO8PKi_ztz$VaG~RYy%33KhtdQzNtd+?t=k7H1D5i$*Px?N<3Mk zAk1ugS1M$vZyJ}2Gk-pWrj!0;RVbZHEi5)u?U^a`qKODEt5myisdUPb0TkT~(}j`l z<{&bpeqFD?R`8(X!je#^>0ce%UXM_z5c+i9H$6ka_3#dNbxFGSYil=QDoojxzHN%?7lIRFEU_}^t5jt7CCSroLJ-fPO_*l$6naG8mXsO)c$ z2=hB1^Y7>OhwlaV#xKY{_gA0KB>eAe)kXg5^gIi`e;@HKvOqtP?C2v|wZ<6GO=cL9 zygwGR!A^X;g4>PNPXJ{>t$686Qe+0D{lVS*iv|JvfkGK^!;cg(Y|`GZ*+~@)1GQs< z2Z+ug34Sq3yz@umz`R_boW-O0dpTyz7LGEh4gAAdx7Qz*=y^d35=5O-8YADmjz-VX z&!c3j4H*)QGx?+6o#!Lr3H-P4z*nH~L>Ln%Va2F}Y%zXf#;C7hVQ@ytGut>eJy_5H)#9m; z17dL{-L`Qk-Ol7K99%B*aM+Wa_A|wZD_$yTd}ZUz%~sFg%fr4o@3;BB|w9@V5XS;;F@yZ@+rC{-LMm{S{G_Id{Rrdr`CjveTzgKN>zT? ziNnq~q{E5O#FgHloztbW$qcgb5NL##bFi|v^rLHBv;T-R|m#B%$*S>vwIUl+a2=$hB%Vwu-?8cJ5+9r*vk>rX8I(JSb?5mL{4ZF@z z(T~+I^*@4O4S+YrYP4t^vp{D24s}1ORZ%>QC96oB4>SfbVA2r$;*t7)7gjnGVk)c@ zBQlF85LI<8bW~==x_u~X63^{5IzeBkwTMV;K>`mDCoUXBP4<0unx~xAzVrM-nS)Tc z9VjOS8i81LqLFx0jUKBe>(}KkYlpfwHo@f%-`jp%y3@bJ=`{{SQH_b>249*t3X8DQ zj;;#F87u%m)-ULH$`N=vm24Xwz2oNUH-XYz(9Ua9SP{^AH}DUKw)61N;i zo5PO;ALy4xD^Kwfqe&wx=M5SW#b`H>C z0l8fU8v*2QJOktyscx5Sm#nH|tlXqH`&B$3z znGG=v;E$z$OQgJP`)n6dgp!kES5}NtVU{&rM>EFG=cEDSp+g99frl~hN8=G~@HzU- zjhbEz-~r?q;K@&e6(i)oY#2m9rWps>x{m1{qFIty8mHg04ydZ%t7D)OJFrs8C;^>w z`jr@IShG?_tr@4t91izUS|ZGsGFjM%T@2Y@%fw0#H9k}&EZ*p?+@ zqcZwe6`#}djQ3)V$5C`Jpt|vGiS72``)|}mWm-4_B9z_l+V!6f=M)(oeJ{PzZqy{| zepj%z5bru^X8hUHX#e}@oWoY6#AwNkd$43c&23+Q$0Vbr2l?KO{JR4odxeP>s(0kD z)$#GX!v$Tg zJtUD_H~Wgkkw3pzJq{_0l%x}lv6Yg)x4@e9lG} zjZtAMh4GFcwLCckuV9&~{!?Q*QHBVl%#Lf(jWuLi#8vs*mdx?Hckd6)MO)$w>KM3G zP=28RSi`1Uhr3O4%^S*ryq~my3SHG&sy!S%ZG2b z)0M~`&$tig+#Q6MCHL4jao(0wW&!`U15rUhp#^yK(YYpFhLMcbHo9M)%p7TB>`Pf4wrWC4 zD13>p{Z()7+qRc3`KppU;vDI$*Jw^+p@=_({<)~N#+2s1aAn^XwIu-`ZyWnN>GfsF zH%v8zOEIZP9L^0azeT3G5mpSGPZ@s0a!U#b`7S<=C~uEvyQeh?Ft$7ma|7DM)5?f> zk@8&&)ucM|JzHu%3>6Y{eFZL3xBs@?y?Xd1tmR!6*47#*gACKiDzm`!RI|>#wt*#> zF`wTz5UvY3AECrDVnm(J)cn8j3BOY#h19CKJI?N^msIH8!^!+)LCaT;%Remn|T~Q&% z^{q!`i+S~8rW<2O^_{}+C5~c20!vODn(77iz8L=7pY(Ro_wvqnby_T&FoD@o`Z8dbQom!yJ_xdz$$p*7)0dF&eoBXj zITY~xz`ZZQe+of>wU(BZZzpPuV=U(6>08p;3K5xW4|DkymrXO_3;R1MgVA z1lxwKgq2oo`@J?hL2>zGp@TSw)QBy^f45O0K-d`n=#@rg(T3X?^UdjsJn9<$3r%ay001 zersy<@BcH9F39W^9rQKq#Fnsx=t?Smyl*zL`xTfHDHKWzL<_{{tc^v~85-o_RsDq* zDleVydTe$Apri}He!lET6^{U`H??mgkic0)|via>$ z#(~;#RcR#^r`lE7Vr9j_?trg#MyrZl*`k_2&OvrpE(SR8P9C!1-Dl8=G%ImMU6T6Q zRbuws8x0^S`QoQSoFE{*M+K5Q4aj!GF$fv^g^>0YQ9iaT!fJ&i5oRY&q}}?erxCgR zgW+)9(EzFr1()V5PzQNZr**@?LUdAe$#GKhB8Qiflg06M;rxyqk`=swA8F%yiBrS* zuYVOhP*yR(qF&Z(^R6+Y7X?WQ(D?>QyBVRB^xNg`+|=RHa>pGHFfJMs7!xQyg9cEi z2~UOX(Lzez?fyYfbhmHyHgUR#IBE53IP|L-J$`QUIWbKJx_v)+8!{0(vE;rc%KZrP zBzEvQ{O&KUtw6!Gr2l^FtZ10c0vvX0@tM8jA#7R@9I*22cX>5+MC;*iOmj)wqm`Zu zEDfg55RO8jVZ3_>d#Cswzujiz@0A6(rEavGv83S4kFAI_<#X{5i>Sn1jl{%k%n?=& z^a^pgEiP8_{W?k8q6mm0x@KQ8M_l=RdD7P}88ziJjek(clnHCJNE+Xz*6wX81geZe z_|-BxQQQqlTEIFvKoTSQU1l<6X2TF!F3u(L7UChTxrz~k!Y#9I(|1pi2GHkQd@|)$ z>@?q*E#Z`Q<-bwprcspAD|3!L%@@Dioa*{-bVeZjA1)Nr;r0@OJ{cyyV==8DzB)2A z=SoT5`5(Ojpi>U$#E~zw$H(07KkwuLPcJVhu0sD!>8*!Y>A#mU9oH5m$}HLg}aeX0xz_K;m{?&8f8*kVb{gQf#$z zN&iXTUKdaVj6HbD8li4A}1R5n9HAFqg&y?iuxtF$OI9ae!w}+Fr~8aM2vqh->I8EroJ;e2Ogl zg5ZmAD%(SnfV(xVLE17cFM||yq%^d5ScH4^WTIe~-$N71$X0hcw~%K!{(p`J7HMMquNuYDqlo z-pjgr?3{DRb8G#>mTUiEI%*suHS&M9zkgVF`Iy?pN1WQ_Kd;CL&s*bEO*lyUIma>j zcCm}9P^Gal*%@sQ! zpTxaOZTUZhSuYY@{`0ZcOKgS3la>R;+s~^A_gYL1Fq8L|;Igkg9MI{wawWT27H+oQ zGHPEGz@{&l!xtsAS0Q_19iK%`?B z97MW>^c);;PBE;>^abX?$eIW=*A8S0ev>uompN`ad z`@s=S1TgqxS|WO~K>Z8Qh?oX$0In;vh3Gz3ru*5*($q_^l&k2xeE1RtB3T$@+0?yR zLl;4l`|(?lV^Z?C;{gx?bX&W%={nCPLz|kfm#8AIQ#aQ!^H;^bw$P67Y*9XBvG$Zp z_z?4?_nOXVtTGTN@-?%}hWOe_$HZTo@!G1w&w6*VGSye}>BFy#>!_Q&1>bKWkJlC- z0;Qq_o&2}w=kpBN2loQA*}8H)QRNdgTB;$wAj*7WcqV%u?GcpPe4pmixtPDx@PFZ- zz63xQ(0Qfona!*D?-lX@jvw>95|07`3#G$mxaVH=dR^A~i_~b3f&t?w3$G<8t z8!0T5%lL-*L5GX>ZclRHL?TkkiZ9a6m8IoFM6`+Q?~jyXcjB$b5{OOhnCy8^tdps; zYl&5eAzY($JNUfmJ`O8@GG@HzG^J4p6$jLpK9Tk)%Ge+05|#Uf=YLpWc+G*ZA#p%e zWJBlp0>8`l;w6+`BF#Q#lqJUX)6=(!h>+uaOywa@?JLhAfWc*&o!@@^tvDF8N15Xg z6EY4|l1`n}$S+!b%DnSf5<~4x)4b^mC!sClq5jVj{(%xNqc={ah4^XN6JSva^|!2b z$Gm^8*3CxhA4DKm?)_~?_@Fciq)o%P-nOG6I52y!nkIo>6dndKP%UWDe+EaZ7t|PE zgQGk0jX%n!(gZnu@Bcf$F{jHOjF)=$8)JYu6E810uz!fpk%iSTI9(zkP?}kF6QTjY zVVb_=$6-{@dM7&Tw{Z#45Rsu&8K)3OQ@S%UqB@Zh`_%#5RQfa7Q0LXdiTh@>s7|cMpBIE`1{q}|qnmZovZpNFQ z_?=1oHqvkY_17N9jqbls>pZ0@xcq^jUBYj1+s=t-ge^yy$K?78Um~27{w& z0BEY+Wh0PTDS)UvZ+a6o>OZt^Zdo5}2H(08VISek#(GFo;)PTEp(Y&V8dU`y-rjci zAho~zUk)hm9|ts*KGln|gN)ekv$s`qQgq*9r3<@)~IK|Fzv+6yVIa*{s%nST;(!G7R+42S?~AGisHp*5`mU=l0K%Gb}Mel zUL8(bPwel88)?PIBK!5Wg|b=c#XFJKha2g{$7)eqQjWK^YRk-&iDypUS_*IQjh+2| zNP`%-<@q4kbtx$=J*k6I#qlB-X#$DTZ8BS5EY*jE4lN-yc8DDOHe|Dvv)jSwU-1Xu7TXsc1Y15f}zS1?-=bR?QgbM-n!NhmRiu(Y%pxfpsg-?Hy8I~ zmi)1w&p(R3^k$kk$eR9Z1bszlM@4ODkuG~vY{f1I?@jtR56Osp>6D6`sEQb%=2s3k zO0#I!?U?)v%GDp=y&UPpOdOOc-hWkLhZB(?BV}1xlhJRVk>`qDF*6dhC+%)b(-Br% zILbXH!v<#^4ubkWh8GX?gSmQi9_ttW4`FW|Rn-@D4J+cMuC&0VyUR<6NGK)UotKtw zkP?v&=>`Ew2`Nd*OXp7kr5h!rq@>~9h(6Cd-tUiZz+fPbanCt>uQk_PYt6M!MPTNS zIj(*Q)duP1D(}jRnq1~;_2pEmOV9|*qf!R0|2$tcz6JprW6hvSwf{T6i^B3>CB*W& z5@Kn8*CP;PMg=886{6P3!lNDE3*`?j7`uEAk4@Z`QuiX2kojc7dP0(fWZgZNyLyea z%9aBXyOEIH2CfgCCre(jc$+!B=$bemagpb=AGIr+NfWlyC{dG7LL8|}@V#9?M~2N* zsSM5AnK78y+xg(#e3_Bo*7>%r^+wmWK9}LZ2RyM9r>@-W3E!XOhB7Q&1?uTUnzphZ za4R3=3kj8v4Visep~da46>Q3;r~c*|l@@k0feq0%3o=sVLuFC$eFU(%WLoYTL84dBRZl@^b%u`Q4Eyb zD*+DFB~6r9%YOD^l{FMsr&p!V%pJ;&|7ii{?qbaey+nb~T{Qku@!6JBqAMSKBNqix zQRy%_Qi?&=%YQfg56T}=eoKV+U%nQk1Xeo8FZu*Cwapcs-5OTP@&&w5;0cgDq$+Ud z&cE0E#wFb7H?|l@|`bjj|#yL%@jBp5? zoWx}an4H;S-ZkiI*%aIyw2>+$a`GHXG32G;j9vz#BTypT%*w+ zDQ#HE7dy#JH&0ksh08mQdb%w0|243)>*V4rbszSJhR7rAaj7qU{vJS>qQ7ycW_kG# zla=+s`yVrytguMJ(n`*Z@cDboBRGp(t$XxdQqq$-g_K&i4`_LHqq9oSA%;qd�&a zA52ZDTlQr6Y*HrNC&p^KmnrD8v`p1yzTtUr_iT4H>y-U+aWeUKWczAK$3@mL3TC(v zy8F?B-`R1%i_6w#^te$W`-dWzy&cTVbqItluHXLT=Xb3Pw-BfSxmvsHt*4Bb;TyQ5 zwjHY?Zd`Hk@m(jntkb)eeg}_^y&wWS6Eh@BGgtt+h;VN+1h5Rtg&EPmF8YFHQf#1f zbxRPf3=?nghn1L^D(Rs5y$;R0-a#;#9)foGgpywaO!9}_@ehZw#m#(Gnt-ht58R+G zw--M{M3dnuY6H-M#IQ(WbK2>}Ulq`9_ymdcYM7pWOVjbM;PPoyV@_?L0vj`HAPa3h(4!mb~DbepBB>#F19XQ!XQ+ zz1MFiMc&0X0z#;KO*(fXTpKG7y3+GbUQYrdSBo&vts%5`bsm#AkqXy73{`1Q!L9Mp zj(^&oMCkg}b;t_8Q+kcoiCtGHt|u(1N&35P{7Sb){2em2Qtfn2CUA1zEi2j>+}A$1 zCMhjhpH4{ddll8r)?D`XW>)cF#GpXb6VDh>bXGn)&B0j0O#BFKUampz#>8$(T(sid z`InsT+lCnpls)3PVzT*kXpmHS}t-EVEy zj5)c1vr$SVfGI}_1`tr_CF`5kKpY5Nj)0knAwQ3UP?72{LT3#Lbm|KMR)#vkFQ($j$32x9g^}C{( zmt^F8&C%O2gTzwfalcM!ntqMAOsRCgQpOs){B2GX;1>9*)oXKl%_<>WlbJU#k?S0l zDSiZZbF*js^56ll+tG8l_G4hgd>OKy1eQa0k4NS&w4l&_?2)+yVO!KE!(9W%r}I8| z&2(RtvWM1tmZQ&MKfNImwQz>&VUdLC$g4jJJNbnbjJ3O5X-vCveTZs#I0=WEaxnKd z5nYH4)bg6SBQ;cBsmbh4v|PFAyYsf0lt@rWsvp;L; zT9$R9@*RjE`9P5fDFG`@G3WMKB;%6oB5u9RIDUP!^6b`5-wjy>9?b3vjEv*p@!)t5 zYQZyv$^b&)gKv(6nK@B^b4y5#mLTlU(U zlnRt3#P}p(h@dDh?M=gksh^JDIr^bvQ6@XohC^ke{lcsbC3xO@O%qvm9q(+J+$~Qk z+3M`@VYXGa#?^`xOkUo$FL~Toy>j{`wxwBsWfWIDmxMKsHLJamn_ULo=q(y*Toj>q zxbDqrVG0G-rFfI6dAPqrc*}dc^Q)YL?mmcpC)f9wva~;&T1MW&i}qcl>{D6TV6+yT zEC`xpw%)D_!fBK6)60>5^fK9fgvKX5awRiLx&7ypZPKi{%Pf;@&XlP?wF0xp?t?ShV{Dp4-Vv z1*z6Iv2)`W`d#Ub;AF+$I#=J`ji!9!zxA4#;3U823OnHPHZ^Zl`#^HVcYo2R;%D!{ z-;bhwA$A57VtcIh>aWkXPtZ|jL6PhC!mg4ZiUXA>=c(AB^hM5g?f9c#6CED!-Zx2a z9>wppMxrJ8plnH}f$met~UhT_SOqC?_rKj$T>LuuwG{LJJI|G?BUNy+qjT zn*gTyjxpabX7q#I&eM?ahO#AE!YGLPfDQ%AunTF28>-7mF_{L zd4=cM$Z{0V-%3Tj-F39Jdw(J!k5D84(Uv4JlpKik2BDfX_L>q98xY?=DMILI21CR+ zfI0TNy*#-+wiqzENU(q4U)#3w&5ZjY{u~^+ZS}0k550WltFh(F_{b)&%|5r~CsB7= z=T1o@J^b*y&sxLh$yufJ?Ga)Y70)Nx;`nSq#v>DtWJNS>#Bm`GVB)Y}u=MQ0p~B*u z!|qv14LD(xzfE;fOaemoY`BZOl$%7tQ##j)C5c3Hp|cN4^F}4hU$gpa>TcT|oVJSZ zblh_n=!0GN+m3Is{@02YFpS<2*I4O#a>2~Xf)qIAhR+4{h+Y!ce|~|p{eaZ*D7?|B zWrmKv4B>&(aA+I=Y``RxJwY=~GAGN<1V#AO}WZy)9VS;KHp?TGm4zTXA+XTiA6EgBN9 zwRMkPbw9z89ms=9;YhsQ_5#WUfeeXO$34t<5rJIzje*BMem(PDg!NG}20e)DP7shX z0mZYYVa?dPJ@PrWfpN&CJ<>|HRt#^KpN_wp@z48`vpVA%cl>wDLJ&d2}h|!Juz@pW@HD>wMnpXQBznMS#SUy+VZ-UOA zKkKVr<>783te$v4WJ7M^xf#AHcPNp73<|>r_M)o8|-THDu0p$1>Thf{-S_plWs%?(^A$F7A z>(8Q<_-IP>i=SK06-ah2;4fXWQ6IbLS|-8R9WY_+&G;pX-n-s$MRV?0u+nTx@<(M) zJKyq}%IQABZ0t6+$2RFHQr+w;7BgoKY<@H}0cIKXHQ#8znY9%QR-zTe{*; zo+9+o%{N9%Dj7PkExGBVlvaB`!0^vnoTmcZtVB9PM`C5EX-}jRDbH`=vFIDdP0z6S z+a6kRN#r%wu)C;kMqtdR_xjGnSp+%3lEd^%uMoR#AXac6VXR%vKi@t|^6vdhX%@!t zVm{-gb*=iQ*AWo98zrYqHElV1r9U7+mFj*?LP2rS2HJs=j_OQOvxtZ5$D>z`cs1cs z(*dz16vs2BM&f#E5)gt_GPKuk?;M|+W<66=%8Efs?=Fen_H&TV>;0nSmV=Gw zhB79zz87n2R79T#*Bd;2a(wI+MMF8=UnrY>cnXv|L0TCv_Ru~rm$x3=(`PS{iXDnr z8j%&cY9Hur*7rvVp-Lt{tpENTcM{3ik8pR8fGopfLF@B_7S2)dKXP$S4a5=W^-42x zjNH?xL^a^-j1jarB&l9uKU9Z-gqukXc~-F72f1;X+lG4XKAR1MnXAQ z_-$ofjI|2x@V&$RZGKwGFc~$Qs@Rb6Epm$m`}mpIdrGLWsr92J_=YkSis~SxtpXeV zi1vMRlE>!AxuY$U*1QqJP48A&bdz$+q~%0ctj%Q_0T5VYZIN?4n!}ux@3E%U`WGoM zXbPQgrK@gDJg<6}XG%V&jQ)78tZVA4U@6`nZkRZ``w$bPWIhQ_VvZKi@QkS+5qs7L zpRm1(Z|nI}f*gzl;LWq#<9)+4+RhoWffl6BfS|1`5&52UX|oqGJXtFxRrVjgbQ}`G z4Ihd4EPAQoTCHE=_2%f_@_^roCur}4W|)qu4dtAP%Dr9f_)|ztc_sCv4?*vFcyJ(* z7^+U2Vx3zchDk;uF}{G#>iZ_AI5-Z;KBlW*DX45{=+CA}TsFa`VqemBrlV7Tl$+H_ z+zn_oNz{UON%YdX#b;UsgjF7+|$5{>X}q5+4KnJB4%)mwN*!yTGvFHX1JWevNY>IA#UOf_6Q?eZBglr1Hoc zUlvJeLjyJvs$x+w8IR04;}_)Dne18Pe^@7>B_pOSJ?4=yctBfjqi^4$dB3sYoL?jB!D*;ZgBL*qTI4lD zaHBEz%-!V?Z)5#kTz*RFZb?fvO!9jb0^qHAyL8q-D{(rAU=LWtnh*UC`*&BlDB9`k zSJ0Xd1y~?g8R5S@Qq>v)N#dveT z9gKCr9`BP%emQT+kZrpBuQ3h!+x)x-sro_Q?3R<~>2o}tuBAQN_GH3qbgY`qB&Wi* zoC1}+-w9%+awUX1)yzB-KS%9L{icIngBro35Q)@Yy8`_BZn0ihY&y;c^zzIiTyq$X z#klE4k@IFBJF)Y`WhtbyU}BL16KS@K39~iOy)XN?+n9B4DUp{)pWZWdy7EFDR%@29 zoD+lmRC(1bTOAb-)eN_@vJiN0ph9f;&b^S$vC^AdMzUEDR!Tx9&*>EP@IOI`-<9=8 zGZTp74hv8mQBW+E@o;gk@Jygk0B3ZGO;|smESf1~#bGH_=XZ|i1I43g_A(Ev7TrHI zNQf4Asn92#8I7A5Q=gAo#uT98rN_<7G$1j@9Gsniw0C`I|M@1yxdgXK4BH3}c)=oN zB?M<@IPSXVgX|-Y2$Ln)^Du|o`96_jE9zI!o44wq{$#F5zG_NY!Eacyxdz#*)jIr zxm4-na=n`}x)$18%j;%co!PhdJ_jmb`%pON>r4_h4?!+ax_Q za(4{-nD57p7cJR4%>X`AKl3=sl%e^>VKXal(5t{fOc1E>%xR;E4|{FRVEp=Pls$Tq zz7v`H?-Ii{dkWOq_|y9(S01oR%D}ZZQ>DHJi%GMt6HFoxCi7mr++){s36ZNlxWArU zD`H@NZ{Nhl`Yu#%Cu`_PLWKXOS26hN)#cS23rCW<#D4+5iHMlQ5c*a~PRBDn}2SUK` zAnWwRVRzC&((-ODyD=cMBF$6v<^Kr!HWa|ZHq=9ejUdJ5nvF=MB7few^Q-Ovs51jp z6(G4G^@*8~dZBxu{^Z}aEvabE)KYgTF!vmjc9M#4DH)gtp%JbF|J~hADFQ`L^YpFS z2S7}X@m<|5S6a?PAr-lqiJcHS2-^A)Lk_5SF$~U}*>+FQc9xY|&iU9q7ub%6kocQ} z`H_fX)I4^U1|D`l$-weZYN9^sTAo&oUK4K@P+qt%1;P@;B)jaRxEJE7sqf{#PmO*P zm=}(!v+1sQ@GY#y>f1z$O3+A|Q+-)*1xrU*XSgwM;Hwo}1QmLiym9#oFG4Mxpez8zdvR-9^k2C3EzFyGxMD3~QH!S?6O388K!y_*@1=ZMO6)1Z@bQD0@`g_C# zbiYO`6v*24I)-18bHLgy;;|mjm*CsUP}*SJUM&%nFmi8F3#cEI`Wh%E&AyH?fOr5? ze;E^))4(b2bS1JzsUlUOVMU3Tt~Ow-4;m5rXCZ-S9oNpUj0$G8^2T-({@IEN^{=P! zl#cP+$quKfNa+<5nC*i|(iW*dxbl{UJ9W*i+3n(v@7mVRv(+|@7^sw!01M3G8HFr4 zP_Pj$22ow&5|?Z5?txcoE~{!VIR=gWqO$U>tHJP zg~iY?XbLCbZHC=r5rdq)985n(rsckHc7e2f+!lClgx?=-$0sc zQj```z{P{wLc$>P3Ju+r{BN)t?Q|88o_y1`EJmh5amG@LB^k>U7}-GVM<@t#7$L_J zXl-RU+mpVsX*(2mda3d8r`b-#SoEYF2$l~03RR7&ZsME`a$8T-&eCL&RVz~{N4=X^ zq@T6Y9?Xk2R`{rbe>SLQU1Mp+My;fRu^H2br#Mi*mOt@7;|>con28!z<=Iyq^3N>d zrsxy9_8epR-w?TdQTG@_1VsBJWg%zcpU}u%l96a(i0UnzG<2yWM>Xq5P`T~&lgCt4 zf5p}R*L=>O`sMzsHdnI>q}6p{*Oqaz*`jJk=*f_Q*-eKUhF_m~%s2UaZZ5OWoZFHj z{4F1I-BhL92GMzs-z5=4Ev9~@0dnO78G@S>p-yQ(3f(2TU;5!)}|N8%Js6fbxg96Mm6cBM;3&1MunPKuGor$T)KBBN6B{T_)ekqw-JI4*-uy4gLTTzyl=*Kx*yOhMLGi9 z=-;GDuM`t)n9A2^$8RT9X-mE7<*f~us5%gg&5xk}n8ItV##B(k^Ak79b=F7f`wV?f zmzlEXlZTp?#}%B-*mvJ93#6!pQ513-9~JFCZz2`5+85wY$;kO^GqO%zlX1^B(_Dl*!jH#p2ael4RI-$p+Qxj_PZVgSqb#8x6W zf5S>bn_1QP;TX^!&5nmeT^bu$aO*I}$x0GU7AyO7g%`G7QFLxkL`D1lO0qL6}LUCn8k;*m=PYAVn7Y3D&gp~nPX}Mcb<+4&IAv8}lhEecEhhsH0WV1CxHn>$vRB^s;a57tOPnU@FgJbga~`g=z7f^tN=|}4pey zP02kX^V<<>YBa$rjeW1HUh|n+Qpy+&(H?9)r;ssUgvWaL>h>b0!7yuV(~nr4)R88j zZRjZGjb{E_cUsLDlJCjOB5$w95%>VVS$ZiB4pS&<9$b_7abYI^%v}{fr5~Rmh&t{y ztw~HjZ!P~DSshh%|CM#0DjZc-%pD$@(XXzjNITlp274~7EOI#$p}q%hTPKOo-)&|) z=1oyIUa@iunB+>E!UE?Fqiv(0!9`VGx5G~>sZ`)H876CN#Oa+{$AGKn97Ko zBLDl-?1dAaYx1UTqW-%AR6Oj7&;p&^ z^wqAprnOqahITFY-rok_HO$h+@nNz4?WbXdbK(NIb(2<;_MJU_b-(dlgjTC2w!j71;=-+>()2uNTt(*ey=?~`OaU@se9XY*i|l~KfMxQUcrX_ znBR8Ve+vz0nR#=on;=9kFp_u0=kTFm%}2Gi*}Uhy#MoQ$(36!>ZEJ{DoQ z)AgIQlL?6N-574$R)Wo#HX`&9`?o<@*Ht2{&EUNsf36On;5Flpm<`D2$M$g$$IlG& zCgy5!#^h>tW(83g%(=es>ap^%RD);EzkrI)dEv8pGzAonG1Ar*W6|=YTvzy2!5I2t zcscR2Ij)XHc{XedjZlFEtFCwQWZ339lH`L`i;lGhdSz{k^01I9)+aQ{A9Bv=5?={r zC~SS81@eP#%gGqW0;of?2dw)(4zhmJcvCp`4FwaI$#hZs`|L>L$JHPyL4^^0@w#vQ zSfT3^Ay#dEN+m}=?zCh|rSN1V1xwIGtZKG`)#cO#=kHu!4v-kp)8(U2tA3PIONf~^exwmqI9s^abtf;~ zr*(Pu52=txBmtoj|G=x0Uw`A1<@LWU9qxJjEqyTJ)@vT5$zP_`+hf88fiz|@AU_B5 z>d4`e_`l?Dp^-+C*u`;+Qem_AB4#57{=0>_d7F^v+hS!;Yj~;HW6sCtIr{W=mN52P z_(%Yw+}fV8e;M#Cf_fIfBxecv=fW`91;DSdP$H5irrBr==J%H%&yvIBgC@}D& zbv>ZFm(;3cucd#Z<$`kipDYCQh<{?ePMWvIw1#WfDMEOViyIO_g#^L*I34xprrex` zMG4i?U*VkI=-u~$a9JPgfs!2~mbG|K_6t<76&p)x>6>%&VQ##0%%*SaSH~N!1}I-4>&Y*Cn@QGovc7AD6Huw| z?hOeqnA#7%xgK&%C%Xfx0DR&n~Vy1btnD;QW&~#3sB?s&&n`5%O%9Pr4_eXAO)4Q7V!} z4&b6Z8JETbqmuj2pd?ZvA!7 z>mTUIFGyWS1-j~bj+@e91Q=dhQXB}WzgA7g#F*i?S_^7T`X~Ac1m>mTHK|R8d9Xgt zF{|Lw@RdG9+a3H#*IT;X^QB2YCu>>D1IgJP|@w7GWm1a0DG9%O7^~gZdXr9{Z zELh^;a8|y6Zp!jWMDOLr#htYsswRtOWW{@jlXhPQt{k>pG|ltBnr4@HfQBynEso`X zo&EQdxI?H;%CLnR zSt;j@TnImxDz<&Ji2?})n^v>`&Ib|J+H(C1!OXBnuc=CfgF#8#v8>qO<$DI$rZy{i zCbqy&-QiKu(!>V3FuATbNu3~bb!s*{vDSesKe)Mg<29peJPP*y$LQp)8QqV8XE(`? zzuvP2&3=SM?7hfNE-*v;d{QpdXiBgjRq6*Vmer$~C{%bBGoOrTjs=XJn*Jf6@3Y#g zbpONJO0QX4sYhmcI)(%RIK7Aa)hU?9fOr+K6IeMNJIzgod)S14mQT3!(Ood~s`get zt5`u{n|h@AX_MaLK&-^uDO;4O+oylzUIMdrYZTP5QZ0gZdvz4GEgeV%ufhDxYp2xS>3!0LkN;|j+H2PjvgVOe9*rR%4KCq-`&jRk7gj2! zjFlXAFh;ld0yCW45CmqYIoPelh(V7Un$MG@;B=|RDL_z(VV%znr9xtQIUkcLBf#Wp zEIkm5OTo42c~bBj?JfG@69>V9b-K*gS7G*uD-QsSAxQdK{`*4fB<-)o3SMs&HiP}2 zA_@Y<#lP)C91@#C?)F7!nyi?j24h@i5wWUrgS1_7RX}QyL|#C@v@DGMQX6iY$zR`O zr_UHSAn{*NGUm;+#kC5p>AJPj)a*9n#ONpo){)VSI}v8$O7A0y1ObK;ol>l9 zm;x!a$RDARPYd?;;Vci{0 zB`~I%0BVTM_Ka*KTOFxLi*we6JA-T>A`g{JcfAD~ctk87@S@mR#}@sY8$iD{?8w)a ze|IU0pnh`oi!eYV+&@inY^Jmvo&bQHU)xJBly%Ra>z;>Bv1Jy9m~+p98nD#0>&PdC z&zS6*ash-o*XW!D>vq$E-L95fmW&ZrRD5`%24w-)#5^CTjl``uS%&5bmLZ)hKQ z1sJQ$?@b?mTG}6^1^Z=z$4QlT%|^Jzub&l{dgs5J3~xW_)wzle&*q%_MAKW9#b0i1 zKWvc7xFj$v6IWMPU8j%SYsddE_IfZ2TnDtik!erhDW!QTT=d*DuXLKTi!gPbb}(g@ zkM6sw(5h;D#nox|shDVHt{h+brPsGKKWsA~;l9*obgqc5N-z6?%ckoSK8qqpCksHv zH~NNudj?=t+8sKO zMw#j+vcE`Uf)Z=l^K){~wV?UBc=@M|s<7AKshZXcwQ&E6byD)>XOQGjND!Bfa6o2s z#Oe=q$&SSkR+tPNCj5}A7926hW-TE6A=^)Dy*zI)xBxu8(Q&t&^6vj%v#SlRjZnrk z21BT$#+iHJ4aQx&=7ZL`=SIhuL3Nz?=Z{W0H$c zMH7I-He>GgD1uX~g2dyIUf$l8%CydcR9|za0oK+VE@16~5TY0AkysvfjZ|V!uCAo8 zexw}>-Z%l+r;zIkoiP|`ph%ecs{*h{ZId^vuvdV%{F$E!r$xI!*VXJ{&Y_af6J{=e zBKV_UFe-lT;^5~R)bujk_Md$EgO|~R5PuLu zsK9XjAI%*Yc?Hmd@+sAaMIHeGaG48_{m}6odho7(kWqOgM1dioi>V-c!Z3i=rY2>i zJb^j9ln*rgz$~kM-2h8XAwOtB4IA3lHQ;7Rua`|DJkY+(ibl>|KG)e;5s6f~A_2a! z@U6Y^KMa%u(#3yjiu7w>PCp9r{!=7DpnSX@+?cc+wEUa<>4`5`WQiUf;AGn@u9CUzO*zgtn^;u96uPS zjy12)x&DBwcWPxaeq7e4MBz zW`b#!M|+uq0g~-EamXn8e6hPVxkgfb++piru5szk9+=y#@SM(v@`J_eyxlh-zu*O) zsF9Lp?hgpmUo+uGm(7#Gzkq$67Xc0I{^Z{ndx~lw1j?TLOOC3TQRGSlpZf{$tD+%9k3ox%>&@j-{$B<(jA}0h z$^A(L=b7NvpMwba3{{-y8johCUtMjz#d6fw^6dG+JLrnZQ2=8!N2IvxPBGrrO&N*f zwO^VD?3AIo$7|=$$im_j$Dd%lWGrnW^Kz`R@&{V2Q_3%HnwWw`yzI)&CidwnaDZHO zqt`^z@43dt5s_*fE2=o#XH$Nwk3Gt(tF)4;$4sfN-ac~K*`8b7$#^{S=A#63I%ZG8dFk)_D z4Bo%pk8qKU>!JB`{S*h3phz}7xHdaqr27y~9o}oU$Ll)XT|>J&KFez>Bn@iuJB-}} z2D;EKDRNGQ?t$gvP;|!W-ui}fgT&;aMKL4h;2$O9@1N_Wr!~EI3sf=1O0jG8(a@w<7wYN72)@t_jP?(yDouTx2Eggy~Ge^6;(e=KUx?9F9l zGr{`Bm5a^&c~EtzznB^Km$o1;Fzeu8_$Mi7=s6a|mp2a?tMWKIk{k<;*F4^?%hrdjA9=Lg|dq^tT+=mIT z=pMpTdJ(7W=G=4;Oa8s&Z>fp-3;{AB<8esA>X>jS?KOg5&aPu*7Z#Gu&RPx^-5m@YrVR~)7BK4S#eZPPNrByC*Me;HhC#>LdlARI z3P*~pRL#5gy*N!iv4583&(+jXnXl)wv*Mqa^(#=w#PyLgh);Gyv}F}cG27y>nyKU+Lf>PyiP5}aeO^`;|rsC4mn_=0)VyQ3Gb zO+MMA7jF;Ob~E_8Dode^cpdDw<-13R-Xq{+_LBG!@5DE#ln7*itRJ1xgv7j}X()~A z#;Urfs*PH&q&Xt7BbD1u> zn+Vl>zS{WZf&^CjI_o#Q!L1)*t{U5P51-nUm=?eaR_eVDu%w~LVe_cG=T-BwwO$;I`cD_gcGpEF}<4MpJiC^&hlkO_q) z>A)PrxB;Fbf$d`R!1cvxY@cc_R0>wQMoDdm`^N}wVI`vb zCRJ9ma4c;yZ_dW4zDB^=?^ja>l+lXz!^Kb(r+LNQ=hH%xsrWIs*;;PAAUyub_LWqZ zXHXOCUyMni0w1*!!WMmF(cDL*{Bmpj>K-RGBFn%cCkKhYFr%%JeavFIOB-NC08as)Pfggd_syoF1o=im!SJhK z`}@kE{@PyQ{^l~DH4Oc_NchBz=xrg|e~%^#9_@!wPZKyV6$AX59FL^u`2HfWJj^Zi z9X-Pl+UKw>#+=a9kEuBT5914Z?W7SFU?Ybj3F>G8RQY#NgFP8G$;euz|BIYmv|vvv zd-t1e{PZJ(qa--k#nYAaX(FR)cgJ)jiEcB-#JRYmxA@wYJyLpn;Q5p<*MA*Jgp&P;n6EJnK~}Vu(@PD)e;-4g z65M;D)@qX}e($que}>r-J_#wrp>=7+qMRAJx+%eKLnWRq(8S%Oq?v8cR zKhQ~j&Sc<)OPAYrQuoYhu{{Wu=o28ob0Q!0nfUKR$V+C&v`&YqPm%_|hqK+U z0LsPHG?D!ANLApGw#)wu`Uu%;3zq2MlFrV##?Aaiocu4uq=B=G>WO}pQ2sfZZ%aC` z!GxyoXrI@H#@SeGeaQIM*zi-Kw=DCyWBcjTQfNMAEKOES(m?4$x+Rmu1Q$GMj8PO~Eqvi?kOa1{HU6_l1q;Qx}hTch1=jJ%5+f zAElEyZG6fy=_*LB@ac%Cx9hlCQ2T~fx&4EP{OGM3Ub~t8tX1d2VE3TD7t4f^p0UWh1<@j{K>_HU^$AU!Z>1oqs4s?YPEwj1RRo4uK|}mA($(pJs+;^ zD5Q`Vs;{Ud5r){(MctY;bbam{%HC)^5}n!LOfo(cYW4jIw(XsNUx|~Dg5&vks`#6T z_e2LjrWY}qA4YDG)>~ETJx6%$2MMJ9K2_tcSq1>mQ~(+%&x;06KWq(j;4Q4U_`ulI zaD_#3Iz-Iv^Zj;HiWtDrEAfr2$VC4Yeo;gUa_JCPNMJ7fa+e zf)>}?>VDQPjr+9%^2V>g=(korNOh{SQ|eKB`_GrZj~(^zh1H1fMh1Ub!z~PUsQ)@` zfE(#kWr$wNrv;sx5WuN~5ZtvX-a%8dl2D^t`+~yC;r!8oB!tfN_#o-3%a7s>LhVAO zsb$9~=5lg1_T`VP-KiEXljV01p)r9#F3O)hLyza$mpO=kt=Z5da;lnDtjz5%uhaAd zaVs%1DdZ9OD_7^y1$>(ldtY<+qd9Gk&IjQ@U2j}Ieini(9{9wMX|C444}Kdz<(ieP z!|BERRUGgG>=gMHx6WTp+zd z)|vPop{nTZ^XqV;ve}-lMfD?2&qb~yDPEJ)ra+nR{8`0nTeKrM&L-#GhR{@#`18@6 zVxEn!IWe>pdKezFMMtL8d?*-|!UoU0Ia`$SRbx~&G=?<{kK@~gBZ8mM#xkebEC$^y z(Et8?h1>fu3UN^}YcD|)i)0R{{BZMsgm?VKdeS5z5ey+3X56vJv?lTQ(UFUphoTbH zcg%kAWA9vG!q<)7({7m7-!g(YzA<-vw&gLLe)|;OMxjKZ4A*usgxS)(Q*B;ajtm9{_kBj~J`s#%nuE&i5uk6L(<)~lL`q(4m#MUav4+1eg} zMRDL)FuS!uuIVK$LuEx&5Z0t0@x$YW$AF(7830<;cGe4HBrt)y^8`e;(cSuX{bA_O zNWmI(!AR~;ppE+56}jqUGXDEb{Z0hVLS6X@b&F>6Jw=Yt(KIv%N{3F4l9s^kC;{73 zZt^+f>U9i;0G*Go834TrxW`Bk3DwN}OtfR22|{x^Q-u)wUvBVkL#|2m{m7`tH~gBN z8zfT|>hLg$kp!gRfk-tO-o$x_z>H|H2H)q!SYNJpUOhbT=Z|(Aea|R0ejbTLa`|1@ z)O&AsjdBQ54=64Xc~vrZ(sSIm2>K=`x|E`H{wNqAMaCuf zpt}LH5Y>W`PK(h7&+O%LXrYkk=v0rq)y8Z=NYZ(^Sy!uR7TnqmH-*x2)wcU z&&W{n6BtJ*gC$PlLXGqkUsN1)JTzH9%z9!Vj>ew_N*S@gsjatef1O}d6$iY6{Gl^t z)Zes(H3NVvro*i!U;JKBM*!GDtoZIBFFM9!4lX&!N7?|H7uJPDQjJ7Xbt^%`b{+pY zd|QaGn+qy=d8G8q%Z6&|0A2COb!o`-S*7i_WTzY#lEUf)S-kbHW>+cr1N4F*jP0J# z`L|y$PzJ@(U*`Qk3-tda9@!p?O!2@NUN*NJ8O6#7 z74~m(8Nvara%36g zAwQp|$_95(iN3#evDiP(H zkFFyap0R@bim&y2ybGtY*3r6Irrooh**LC@37XsivORkt~@H~bPcORmdOl)ijt;@ zxKLtBW|oGcS>~>}uNdex^>T4EC9GV?Jt-R(N?SDxF(a32luhBDOQxB7xD95LsdaR$ zOuZk;bkDsU4u``(@V&q1ectbR-k0A;i&!-iIZe{XM$BHm%;_fZ)nFB0WP^s*&79}f zj+lJl!=8>Vo6RL2{~6?tB{eu+VHYAOiFS#8QG6R#md;x-IUR?gyZWc5w zm4nQEUn7*i-^%)MQf7;{T&Z(iY zwI{4;9Dj?p&Db+QXf~>2$ixad3{RlD$FHanSdo^o2Zb?>!Q^;{JPceS5b40HWlk5P( z{TL!=DGC4I*TIJZu!Q8x&>0(53ZXuIdO_1PnCC*Za5BcP*piY$y6kc1{-N z%T#skG4}N%#3WQB*hpd#k`H=#yE$znbfPa@U;wW;OyT`@(C6#KQ$g?h^{wWtljTZ< zk!m@`;>-yf4d-k9NSz8X0+bD@jxwPRHzq@k;DXY{RM`n$9Vg2xS5Sp#EBAEx#f3{R z(w)igUmURTs#fb-t9xy5_mxlq)G3yF^+cH`Y?A&~YIuc**Ue66gTxXE1gVtq}P@d0M^#idnh}z52+UC>un#bFp zvnJf>Ox8Fhr39;O&7cye`p7z#z^8CFSy^$vBLw z5d0_+kecl}+e`gnT&3Jz;$HJ|x7ZT|daY^H{gLypUnjhr(M+{bK{;9nW-Cm$qE?}- z;_#+{y3?9nE&9$V?Ca` zwl~0Zy_hxq&mVT4cpqXV2e9E~o!{&r-b*KM)iYX&o>4+P2h#mYxa$8YSPxXvM5|LPlB;60S?oSZAJRyHv?p2F77^FRZ`i|!65OuX zaR#DMosn^VIMi&-bZT%l`N$iFh5HkEy)$6rj(u3GKfEX%OK+U%Kk(o-RgKD@k1k%E z216Qpz6Toe1nuMRCU6E@0d-`jT-E!blQt6mAryRGm<$^E-*rw_mK722)1D`|#eoA; zi(*t5F9p0R_1{gR)-U` zr0|3!nO=TP(4}8`!A`ir+kd;8Zyck3Vad$3?g`SN^6b8X`l;22UjsD{mDt#D<5lmq zNR8g^%zg8`L|l}cVo zHv!rB>-So~zSGaMd<5V*HOJD6b{BSBBcwayr0O(?T;q%aVT{=})#ls<)nAs1M6}rB zPYA~=pAK8fVykkZ5^Ca=aBK46dl587XLL*?l;AB}y+2?}+AUiY%3$B0J_M{I#)#Vo zn6@{pV^tz4lOSP4P<0GQ(Z>+_WI(spHVf%#FIDUt=4d(IwXqR_3^3Q@<-vSDHwd7G{KYw-bZI}!tYMcF@^aR0OZBnYxV+=ZQR8~-*^@{l&saNhoA{EpVQ*^)Q` zX2<}u+d(^-LyM4YG9{lokF7+v4$p^#ez-sz`KsdS>$YlBQL#T`q<$+eLXI-T||Sa9}bwzBl> z + + +如显示上图状态,需要启动对应的agent. + +**Windows 环境操作开启 CDC agent** +点击 下图位置代理开启 +

+ +**重新启动数据库** +
+ +
+ +**再次查询agent 状态,确认状态变更为running** +
+ +
+ +至此,表`test`启动CDC(变更数据捕获)功能配置完成。 + +**docker 环境操作开启 CDC agent** + +**开启mssql-server的代理服务**_ +```shell +docker exec -it sqlserver bash +/opt/mssql/bin/mssql-conf set sqlagent.enabled true +docker stop sqlserver +docker start sqlserver +``` 参考阅读:[https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-2017](https://docs.microsoft.com/zh-cn/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-2017) From c599d90b23b22483df435217b1817439d5103b63 Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Mon, 22 Mar 2021 15:49:54 +0800 Subject: [PATCH 114/136] =?UTF-8?q?[docs-359][sqlserverCDC=E9=85=8D?= =?UTF-8?q?=E7=BD=AE.md]=E4=BF=AE=E6=94=B9sqlserverCDC=E9=85=8D=E7=BD=AE?= =?UTF-8?q?=E6=96=87=E6=A1=A3=EF=BC=8C=E5=A2=9E=E5=8A=A0agent=E5=BC=80?= =?UTF-8?q?=E5=90=AF=E6=95=99=E7=A8=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../other/SqlserverCDC\351\205\215\347\275\256.md" | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git "a/docs/realTime/other/SqlserverCDC\351\205\215\347\275\256.md" "b/docs/realTime/other/SqlserverCDC\351\205\215\347\275\256.md" index 0b8d97b5d5..cee0058106 100644 --- "a/docs/realTime/other/SqlserverCDC\351\205\215\347\275\256.md" +++ "b/docs/realTime/other/SqlserverCDC\351\205\215\347\275\256.md" @@ -2,13 +2,13 @@ - [SqlServer配置CDC](#sqlserver配置cdc) - - [1、查询SqlServer数据库版本](#1查询sqlserver数据库版本) - - [2、查询当前用户权限,必须为 sysadmin 固定服务器角色的成员才允许对数据库启用CDC(变更数据捕获)功能](#2查询当前用户权限必须为 sysadmin 固定服务器角色的成员才允许对数据库启用cdc变更数据捕获功能) - - [3、查询数据库是否已经启用CDC(变更数据捕获)功能](#3查询数据库是否已经启用cdc变更数据捕获功能) - - [4、对数据库数据库启用CDC(变更数据捕获)功能](#4对数据库数据库启用cdc变更数据捕获功能) - - [5、查询表是否已经启用CDC(变更数据捕获)功能](#5查询表是否已经启用cdc变更数据捕获功能) - - [6、对表启用CDC(变更数据捕获)功能](#6对表启用cdc变更数据捕获功能) - - [7 数据库配置](#7-数据库配置) + - [1、查询SqlServer数据库版本](#1查询sqlserver数据库版本) + - [2、查询当前用户权限,必须为 sysadmin 固定服务器角色的成员才允许对数据库启用CDC(变更数据捕获)功能](#2查询当前用户权限必须为 sysadmin 固定服务器角色的成员才允许对数据库启用cdc变更数据捕获功能) + - [3、查询数据库是否已经启用CDC(变更数据捕获)功能](#3查询数据库是否已经启用cdc变更数据捕获功能) + - [4、对数据库数据库启用CDC(变更数据捕获)功能](#4对数据库数据库启用cdc变更数据捕获功能) + - [5、查询表是否已经启用CDC(变更数据捕获)功能](#5查询表是否已经启用cdc变更数据捕获功能) + - [6、对表启用CDC(变更数据捕获)功能](#6对表启用cdc变更数据捕获功能) + - [7、确认CDC agent 是否正常启动](#7确认cdc-agent-是否正常启动) 注:SqlServer自2008版本开始支持CDC(变更数据捕获)功能,本文基于SqlServer 2017编写。 From 420362897533083ea1733febbab9981bb0178da4 Mon Sep 17 00:00:00 2001 From: tudou Date: Tue, 23 Mar 2021 17:51:09 +0800 Subject: [PATCH 115/136] =?UTF-8?q?[hotfix-366][Oracle=20LogMiner]fix=20#3?= =?UTF-8?q?66=20=E4=BF=AE=E5=A4=8DFlinkX=201.10=20Oracle=20LogMiner?= =?UTF-8?q?=E6=8F=92=E4=BB=B6=E6=89=93=E5=8C=85=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml index c80b54f930..a9cec6b8f8 100644 --- a/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml +++ b/flinkx-oraclelogminer/flinkx-oraclelogminer-reader/pom.xml @@ -95,7 +95,7 @@ + tofile="${basedir}/../../syncplugins/oraclelogminerreader/${project.name}-${package.name}.jar" /> From d5f56f7e01717589c376abecd36b5395b0b4041a Mon Sep 17 00:00:00 2001 From: tudou Date: Wed, 24 Mar 2021 09:53:07 +0800 Subject: [PATCH 116/136] =?UTF-8?q?[docs][kafka]=E4=BF=AE=E5=A4=8Dkafka?= =?UTF-8?q?=E6=96=87=E6=A1=A3=E8=B7=B3=E8=BD=AC=E9=93=BE=E6=8E=A5=E5=A4=B1?= =?UTF-8?q?=E6=95=88=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/realTime/reader/kafkareader.md | 8 ++++---- docs/realTime/writer/kafkawriter.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/realTime/reader/kafkareader.md b/docs/realTime/reader/kafkareader.md index 0fca3b17b6..faf28c2a48 100644 --- a/docs/realTime/reader/kafkareader.md +++ b/docs/realTime/reader/kafkareader.md @@ -6,10 +6,10 @@ - [一、插件名称](#一插件名称) - [二、参数说明](#二参数说明) - [三、配置示例](#三配置示例) - - [1、kafka10](#2kafka10) - - [2、kafka11](#3kafka11) - - [3、kafka](#4kafka) - - [4、kafka->Hive](#5kafka-hive) + - [1、kafka10](#1kafka10) + - [2、kafka11](#2kafka11) + - [3、kafka](#3kafka) + - [4、kafka->Hive](#4kafka-hive) diff --git a/docs/realTime/writer/kafkawriter.md b/docs/realTime/writer/kafkawriter.md index d8eefddbdd..597dfc1607 100644 --- a/docs/realTime/writer/kafkawriter.md +++ b/docs/realTime/writer/kafkawriter.md @@ -5,10 +5,10 @@ - [一、插件名称](#一插件名称) - [二、参数说明](#二参数说明) - [三、配置示例](#三配置示例) - - [1、kafka10](#2kafka10) - - [2、kafka11](#3kafka11) - - [3、kafka](#4kafka) - - [4、MySQL->kafka](#5mysql-kafka) + - [1、kafka10](#1kafka10) + - [2、kafka11](#2kafka11) + - [3、kafka](#3kafka) + - [4、MySQL->kafka](#4mysql-kafka) From 65a6bd3c3eb9d48725185c2389599e4eabb0be07 Mon Sep 17 00:00:00 2001 From: tudou Date: Thu, 25 Mar 2021 16:51:55 +0800 Subject: [PATCH 117/136] =?UTF-8?q?[fix-35871][core][test]=E8=A7=A3?= =?UTF-8?q?=E5=86=B3launcher=E6=8F=90=E4=BA=A4=E7=BC=BA=E5=B0=91-p?= =?UTF-8?q?=E5=8F=82=E6=95=B0=E6=8A=A5=E9=94=99=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/options/Options.java | 12 ++++++++++++ .../main/java/com/dtstack/flinkx/test/LocalTest.java | 6 +----- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java index 0296baa0dc..68b59e1624 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/options/Options.java @@ -73,6 +73,9 @@ public class Options { @OptionRequired(description = "env properties") private String confProp = "{}"; + @OptionRequired(description = "json modify") + private String p = ""; + @OptionRequired(description = "savepoint path") private String s; @@ -246,6 +249,14 @@ public void setRemotePluginPath(String remotePluginPath) { this.remotePluginPath = remotePluginPath; } + public String getP() { + return p; + } + + public void setP(String p) { + this.p = p; + } + public String getKrb5conf() { return krb5conf; } @@ -285,6 +296,7 @@ public String toString() { ", queue='" + queue + '\'' + ", flinkLibJar='" + flinkLibJar + '\'' + ", confProp='" + confProp + '\'' + + ", p='" + p + '\'' + ", s='" + s + '\'' + ", pluginLoadMode='" + pluginLoadMode + '\'' + ", appId='" + appId + '\'' + diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index 03f24323b7..3d6c82aa4c 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -58,11 +58,9 @@ import com.dtstack.flinkx.kudu.reader.KuduReader; import com.dtstack.flinkx.kudu.writer.KuduWriter; import com.dtstack.flinkx.metadatahbase.reader.MetadatahbaseReader; -//import com.dtstack.flinkx.metadataes6.reader.Metadataes6Reader; import com.dtstack.flinkx.metadatahive2.reader.Metadatahive2Reader; import com.dtstack.flinkx.metadatamysql.reader.MetadatamysqlReader; import com.dtstack.flinkx.metadataoracle.reader.MetadataoracleReader; -import com.dtstack.flinkx.metadataphoenix5.reader.MetadataphoenixReader; import com.dtstack.flinkx.metadatasqlserver.reader.MetadatasqlserverReader; import com.dtstack.flinkx.metadatatidb.reader.MetadatatidbReader; import com.dtstack.flinkx.metadatavertica.reader.MetadataverticaReader; @@ -85,7 +83,6 @@ import com.dtstack.flinkx.reader.BaseDataReader; import com.dtstack.flinkx.redis.writer.RedisWriter; import com.dtstack.flinkx.restapi.reader.RestapiReader; -import com.dtstack.flinkx.restapi.writer.RestapiWriter; import com.dtstack.flinkx.socket.reader.SocketReader; import com.dtstack.flinkx.sqlserver.reader.SqlserverReader; import com.dtstack.flinkx.sqlserver.writer.SqlserverWriter; @@ -115,6 +112,7 @@ import java.nio.charset.StandardCharsets; import java.util.Properties; import java.util.concurrent.TimeUnit; + /** * @author jiangbo */ @@ -242,9 +240,7 @@ private static BaseDataReader buildDataReader(DataTransferConfig config, StreamE case PluginNameConstrant.METADATATIDB_READER : reader = new MetadatatidbReader(config, env); break; case PluginNameConstrant.METADATAORACLE_READER : reader = new MetadataoracleReader(config, env); break; case PluginNameConstrant.METADATASQLSERVER_READER : reader = new MetadatasqlserverReader(config, env); break; - case PluginNameConstrant.METADATAPHOENIX_READER : reader = new MetadataphoenixReader(config, env); break; case PluginNameConstrant.METADATAHBASE_READER : reader = new MetadatahbaseReader(config, env); break; -// case PluginNameConstrant.METADATAES6_READER : reader = new Metadataes6Reader(config, env); break; case PluginNameConstrant.METADATAVERTICA_READER : reader = new MetadataverticaReader(config, env); break; case PluginNameConstrant.GREENPLUM_READER : reader = new GreenplumReader(config, env); break; case PluginNameConstrant.PHOENIX5_READER : reader = new Phoenix5Reader(config, env); break; From 7a35094ed769481acf2b4858ed1107edd6c29153 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 26 Mar 2021 11:01:43 +0800 Subject: [PATCH 118/136] =?UTF-8?q?[fix-35555][rdb]update=E6=97=B6?= =?UTF-8?q?=EF=BC=8Ccolumn=E9=9C=80=E8=A6=81=20=E5=8A=A0=E4=B8=8A=E5=8F=8C?= =?UTF-8?q?=E5=BC=95=E5=8F=B7=20=E5=90=A6=E5=88=99=E5=9C=A8oracle=E7=9A=84?= =?UTF-8?q?=E5=AD=97=E6=AE=B5=E6=98=AF=E5=B0=8F=E5=86=99=E7=9A=84=E6=97=B6?= =?UTF-8?q?=E5=80=99=EF=BC=8C=E6=9C=AA=E5=8A=A0=E5=BC=95=E5=8F=B7=EF=BC=8C?= =?UTF-8?q?=E9=BB=98=E8=AE=A4=E4=B8=BA=E5=A4=A7=E5=86=99=E5=AF=BC=E8=87=B4?= =?UTF-8?q?=E9=94=99=E8=AF=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java index 758be23ace..cc5f9bffc1 100644 --- a/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java +++ b/flinkx-rdb/flinkx-rdb-core/src/main/java/com/dtstack/flinkx/rdb/BaseDatabaseMeta.java @@ -169,7 +169,7 @@ protected String getUpdateSql(List column, String leftTable, String righ String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for(String col : column) { - list.add(prefixLeft + col + "=" + prefixRight + col); + list.add(prefixLeft + quoteColumn(col) + "=" + prefixRight + quoteColumn(col)); } return StringUtils.join(list, ","); } From fdee7cb2488b669946af94e1e14424780b3a0498 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 26 Mar 2021 11:02:34 +0800 Subject: [PATCH 119/136] =?UTF-8?q?[opt][test]test=E6=A8=A1=E5=9D=97pom?= =?UTF-8?q?=E6=96=87=E4=BB=B6=E5=88=A0=E9=99=A4=E9=87=8D=E5=A4=8D=E5=BC=95?= =?UTF-8?q?=E7=94=A8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-test/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index 1f3dfbd0ac..01dcc9814d 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -441,11 +441,6 @@ flinkx-restapi-reader 1.6 - - com.dtstack.flinkx - flinkx-restapi-writer - 1.6 - com.dtstack.flinkx flinkx-dm-reader From 7e7cc1443c0082d61ebdeb9f367550ca52e52c18 Mon Sep 17 00:00:00 2001 From: tudou Date: Fri, 26 Mar 2021 11:12:10 +0800 Subject: [PATCH 120/136] =?UTF-8?q?[docs][docs]1=E3=80=81fix=20#368=20?= =?UTF-8?q?=E9=92=89=E9=92=89=E7=BE=A4=E4=BA=8C=E7=BB=B4=E7=A0=81=E5=A4=B1?= =?UTF-8?q?=E6=95=88=E9=97=AE=E9=A2=98=EF=BC=9B2=E3=80=81readme=E9=BB=98?= =?UTF-8?q?=E8=AE=A4=E7=94=B1=E8=8B=B1=E6=96=87=E6=94=B9=E4=B8=BA=E4=B8=AD?= =?UTF-8?q?=E6=96=87?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 84 ++++++++++++++++++++--------------- README_CH.md => README_EN.md | 84 +++++++++++++++-------------------- docs/images/ding.jpg | Bin 73843 -> 290427 bytes 3 files changed, 84 insertions(+), 84 deletions(-) rename README_CH.md => README_EN.md (59%) diff --git a/README.md b/README.md index 9f42ed6a24..0de4d29d13 100644 --- a/README.md +++ b/README.md @@ -3,47 +3,57 @@ FlinkX [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) -English | [中文](README_CH.md) - -# Communication - -- We are recruiting **Big data platform development engineers**. If you want more information about the position, please add WeChat ID [**ysqwhiletrue**] or email your resume to [sishu@dtstack.com](mailto:sishu@dtstack.com). - -- We use [DingTalk](https://www.dingtalk.com/) to communicate, you can search the group number [**30537511**] or scan the QR code below to join the communication group +[English](README_EN.md) | 中文 + +# 技术交流 + +- 招聘**Flink研发工程师**,如果有兴趣可以联系思枢(微信号:ysqwhiletrue)
+Flink开发工程师JD要求:
+1.负责袋鼠云基于Flink的衍生框架数据同步flinkx和实时计算flinkstreamsql框架的开发;
+2.调研和把握当前最新大数据实时计算技术,将其中的合适技术引入到平台中,改善产品,提升竞争力;
+职位要求:
+1、本科及以上学历,3年及以上的Flink开发经验,精通Java,熟悉Scala、Python优先考虑;
+2、熟悉Flink原理,有基于Flink做过二次源码的开发,在github上贡献者Flink源码者优先;
+3、有机器学习、数据挖掘相关经验者优先;
+4、对新技术有快速学习和上手能力,对代码有一定的洁癖;
+加分项:
+1.在GitHub或其他平台上有过开源项目
+可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) + +- 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群
- +
-# Introduction - -* **FlinkX is a distributed offline and real-time data synchronization framework based on flink widely used in 袋鼠云, which realizes efficient data migration between multiple heterogeneous data sources.** +# 介绍 +* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线和实时的数据同步框架,实现了多种异构数据源之间高效的数据迁移。** -Different data sources are abstracted into different Reader plugins, and different data targets are abstracted into different Writer plugins. In theory, the FlinkX framework can support data synchronization of any data source type. As a set of ecosystems, every time a set of new data sources is connected, the newly added data sources can realize intercommunication with existing data sources. +不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。
-FlinkX is a data synchronization tool based on Flink. FlinkX can collect static data, such as MySQL, HDFS, etc, as well as real-time changing data, such as MySQL binlog, Kafka, etc. FlinkX currently includes the following features: +FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集静态的数据,比如MySQL,HDFS等,也可以采集实时变化的数据,比如MySQL binlog,Kafka等。FlinkX目前包含下面这些特性: -- Most plugins support concurrent reading and writing of data, which can greatly improve the speed of reading and writing; +- 大部分插件支持并发读写数据,可以大幅度提高读写速度; -- Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [Failure Recovery](docs/restore.md) +- 部分插件支持失败恢复的功能,可以从失败的位置恢复任务,节约运行时间;[失败恢复](docs/restore.md) -- The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/offline/reader/mysqlreader.md) +- 关系数据库的Reader插件支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/offline/reader/mysqlreader.md) -- Some databases support opening Kerberos security authentication; [Kerberos](docs/kerberos.md) +- 部分数据库支持开启Kerberos安全认证;[Kerberos](docs/kerberos.md) -- Limit the reading speed of Reader plugins and reduce the impact on business databases; +- 可以限制reader的读取速度,降低对业务数据库的影响; -- Save the dirty data when writing data; +- 可以记录writer插件写数据时产生的脏数据; -- Limit the maximum number of dirty data; +- 可以限制脏数据的最大数量; -- Multiple running modes: Local,Standalone,Yarn Session,Yarn Per; +- 支持多种运行模式; -The following databases are currently supported: +FlinkX目前支持下面这些数据库: | | Database Type | Reader | Writer | |:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| @@ -75,42 +85,42 @@ The following databases are currently supported: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | -| | RestApi || [doc](docs/realTime/writer/restapiwriter.md) | +| | RestApi |[doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle LogMiner| [doc](docs/realTime/reader/LogMiner.md) | | -| | Sqlserver CDC| [doc](docs/realTime/reader/sqlservercdc.md) | | +| | Oracle LogMiner | [doc](docs/realTime/reader/LogMiner.md) | | +| | Sqlserver CDC | [doc](docs/realTime/reader/sqlservercdc.md) | | -# Fundamental -In the underlying implementation, FlinkX relies on Flink, and the data synchronization task will be translated into StreamGraph and executed on Flink. The basic principle is as follows: +# 基本原理 +在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,基本原理如下图:
-# Quick Start +# 快速开始 -Please click [Quick Start](docs/quickstart.md) +请点击[快速开始](docs/quickstart.md) -# General Configuration +# 通用配置 -Please click [General Configuration](docs/generalconfig.md) +请点击[插件通用配置](docs/generalconfig.md) -# Statistics Metric +# 统计指标 -Please click [Statistics Metric](docs/statistics.md) +请点击[统计指标](docs/statistics.md) # Kerberos -Please click [Kerberos](docs/kerberos.md) +请点击[Kerberos](docs/kerberos.md) # Questions -Please click [Questions](docs/questions.md) +请点击[Questions](docs/questions.md) -# How to contribute FlinkX +# 如何贡献FlinkX -Please click [Contribution](docs/contribution.md) +请点击[如何贡献FlinkX](docs/contribution.md) # License diff --git a/README_CH.md b/README_EN.md similarity index 59% rename from README_CH.md rename to README_EN.md index 2a4adfe7c7..5e5bfdeb74 100644 --- a/README_CH.md +++ b/README_EN.md @@ -3,57 +3,47 @@ FlinkX [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) -[English](README.md) | 中文 - -# 技术交流 - -- 招聘**Flink研发工程师**,如果有兴趣可以联系思枢(微信号:ysqwhiletrue)
-Flink开发工程师JD要求:
-1.负责袋鼠云基于Flink的衍生框架数据同步flinkx和实时计算flinkstreamsql框架的开发;
-2.调研和把握当前最新大数据实时计算技术,将其中的合适技术引入到平台中,改善产品,提升竞争力;
-职位要求:
-1、本科及以上学历,3年及以上的Flink开发经验,精通Java,熟悉Scala、Python优先考虑;
-2、熟悉Flink原理,有基于Flink做过二次源码的开发,在github上贡献者Flink源码者优先;
-3、有机器学习、数据挖掘相关经验者优先;
-4、对新技术有快速学习和上手能力,对代码有一定的洁癖;
-加分项:
-1.在GitHub或其他平台上有过开源项目
-可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至[sishu@dtstack.com](mailto:sishu@dtstack.com) - -- 我们使用[钉钉](https://www.dingtalk.com/)沟通交流,可以搜索群号[**30537511**]或者扫描下面的二维码进入钉钉群 +English | [中文](README.md) + +# Communication + +- We are recruiting **Big data platform development engineers**. If you want more information about the position, please add WeChat ID [**ysqwhiletrue**] or email your resume to [sishu@dtstack.com](mailto:sishu@dtstack.com). + +- We use [DingTalk](https://www.dingtalk.com/) to communicate, you can search the group number [**30537511**] or scan the QR code below to join the communication group
- +
-# 介绍 -* **FlinkX是在是袋鼠云内部广泛使用的基于flink的分布式离线和实时的数据同步框架,实现了多种异构数据源之间高效的数据迁移。** +# Introduction + +* **FlinkX is a distributed offline and real-time data synchronization framework based on flink widely used in 袋鼠云, which realizes efficient data migration between multiple heterogeneous data sources.** -不同的数据源头被抽象成不同的Reader插件,不同的数据目标被抽象成不同的Writer插件。理论上,FlinkX框架可以支持任意数据源类型的数据同步工作。作为一套生态系统,每接入一套新数据源该新加入的数据源即可实现和现有的数据源互通。 +Different data sources are abstracted into different Reader plugins, and different data targets are abstracted into different Writer plugins. In theory, the FlinkX framework can support data synchronization of any data source type. As a set of ecosystems, every time a set of new data sources is connected, the newly added data sources can realize intercommunication with existing data sources.
-FlinkX是一个基于Flink的批流统一的数据同步工具,既可以采集静态的数据,比如MySQL,HDFS等,也可以采集实时变化的数据,比如MySQL binlog,Kafka等。FlinkX目前包含下面这些特性: +FlinkX is a data synchronization tool based on Flink. FlinkX can collect static data, such as MySQL, HDFS, etc, as well as real-time changing data, such as MySQL binlog, Kafka, etc. FlinkX currently includes the following features: -- 大部分插件支持并发读写数据,可以大幅度提高读写速度; +- Most plugins support concurrent reading and writing of data, which can greatly improve the speed of reading and writing; -- 部分插件支持失败恢复的功能,可以从失败的位置恢复任务,节约运行时间;[失败恢复](docs/restore.md) +- Some plug-ins support the function of failure recovery, which can restore tasks from the failed location and save running time; [Failure Recovery](docs/restore.md) -- 关系数据库的Reader插件支持间隔轮询功能,可以持续不断的采集变化的数据;[间隔轮询](docs/offline/reader/mysqlreader.md) +- The Reader plugin for relational databases supports interval polling. It can continuously collect changing data; [Interval Polling](docs/offline/reader/mysqlreader.md) -- 部分数据库支持开启Kerberos安全认证;[Kerberos](docs/kerberos.md) +- Some databases support opening Kerberos security authentication; [Kerberos](docs/kerberos.md) -- 可以限制reader的读取速度,降低对业务数据库的影响; +- Limit the reading speed of Reader plugins and reduce the impact on business databases; -- 可以记录writer插件写数据时产生的脏数据; +- Save the dirty data when writing data; -- 可以限制脏数据的最大数量; +- Limit the maximum number of dirty data; -- 支持多种运行模式; +- Multiple running modes: Local,Standalone,Yarn Session,Yarn Per; -FlinkX目前支持下面这些数据库: +The following databases are currently supported: | | Database Type | Reader | Writer | |:----------------------:|:--------------:|:-------------------------------:|:-------------------------------:| @@ -85,42 +75,42 @@ FlinkX目前支持下面这些数据库: | | Hive | | [doc](docs/offline/writer/hivewriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | -| | RestApi |[doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | +| | RestApi || [doc](docs/realTime/writer/restapiwriter.md) | | | MySQL Binlog | [doc](docs/realTime/reader/binlogreader.md) | | | | MongoDB Oplog | [doc](docs/realTime/reader/mongodboplogreader.md)| | | | PostgreSQL WAL | [doc](docs/realTime/reader/pgwalreader.md) | | -| | Oracle LogMiner | [doc](docs/realTime/reader/LogMiner.md) | | -| | Sqlserver CDC | [doc](docs/realTime/reader/sqlservercdc.md) | | +| | Oracle LogMiner| [doc](docs/realTime/reader/LogMiner.md) | | +| | Sqlserver CDC| [doc](docs/realTime/reader/sqlservercdc.md) | | -# 基本原理 -在底层实现上,FlinkX依赖Flink,数据同步任务会被翻译成StreamGraph在Flink上执行,基本原理如下图: +# Fundamental +In the underlying implementation, FlinkX relies on Flink, and the data synchronization task will be translated into StreamGraph and executed on Flink. The basic principle is as follows:
-# 快速开始 +# Quick Start -请点击[快速开始](docs/quickstart.md) +Please click [Quick Start](docs/quickstart.md) -# 通用配置 +# General Configuration -请点击[插件通用配置](docs/generalconfig.md) +Please click [General Configuration](docs/generalconfig.md) -# 统计指标 +# Statistics Metric -请点击[统计指标](docs/statistics.md) +Please click [Statistics Metric](docs/statistics.md) # Kerberos -请点击[Kerberos](docs/kerberos.md) +Please click [Kerberos](docs/kerberos.md) # Questions -请点击[Questions](docs/questions.md) +Please click [Questions](docs/questions.md) -# 如何贡献FlinkX +# How to contribute FlinkX -请点击[如何贡献FlinkX](docs/contribution.md) +Please click [Contribution](docs/contribution.md) # License diff --git a/docs/images/ding.jpg b/docs/images/ding.jpg index 605928bae9b06b8f6ea7da5e4417749cd63da6d7..d069044f6eedce937eb3d0ebcc56caa3452bf09c 100644 GIT binary patch literal 290427 zcmeFZgOe@6vp+btZQHhS$F^_RJaEwrv|b-?#6@zWx0N8?h&%BdV*a ztMk;!&irKNr%$A!yaYT94h#?w5WJM6m@*I$s16ViSUwcUPft+gAOsK)u%wl!sG^jp zD3PMGgSnNh84wWF9}`1E6e&8Y5hEi*!;wi^Y8Yn^<%ozFWy656o(ZDfp3(e4qKtH1 z-5o5X9Z=OipdzK-78|%2qPKd65xP=4y)|o#=W{-~mX+_D#rMJVGbJDmX=^8zgfGwm zt5I-qayYmcYN2LlFDc-NO<>b#G+=yi(`n$kdEyIrB6pM!Kd?#)vJvc&ZeUYbG7B=0 zNkd;?({nQ7$Wqd}f~Vg=@zE9W@gik^{uPA60v{(LNGsDy3pKj<3!5pKd;g03P0Ez4 zjLvtT_~?oj0~Z5_jU{A+p!CTNg_jy*{ElS&DuabZ*Uiktq#zbSE@x!8vNhC0S}*uLP$bme3kXOENJ>mt)f4z)6Eg5n%}wxzv+H?vbyeT8t7-3PnLZ zyCF!f=lvfyMm8D){T~_rdlA1J5NeW*Wo{z7fd4Jw&!~6^HR6AS{a<%`a)bFB;b?Fi z*!cN>Eb}8Y-~XEUpRf7<3;o}}_W#FsdhneGICtqu!HngL;9D(W^u^~$x1s9W)~5S; z{r$qB*Vd9}ed)^XZMs>Y!OBLmqvc$1{q66m%71D4jciFz0|B!9EKOVW?QLXYA_+iN zULGCeYCy@v1a)w5cXy(~&>}Ix-`BS@KcA_a;qU>^H)V8Sz|+f1YeEdX{yI)U76bi= zqr-e&mQj0?vLxVJ@A-_2nMGS&oo4suM$*X0$jVB}mzogYUn4dC`MLD(SA0vY-f3eC zmBROx^xeT1eL7vb9^yoE>)yYAdNGeL`jb)?H3Hx*dw;5gy#-14^NM&Xqd}O zOJ%+R--d2(hxTAIg|nlPeT5tQPAvlp7w4!ugpY*y__(-)jI^+zzNV&Pv@!LiAzuoW z=-k_xKc5Fq=G~I~{QTBdF4{v$h&e_XM?d0l{&D`Qduck!YKG?lSY|9jZ+=3HAuxqS z3l9$kKfg`;w%7|4Z+mH(;0Ihh9ObsCsJqpt@#|9bjzz)GOy!+bksuJ`!&T+w?8gZW zKT^U(QraS@{G?8OwWbd3u_JC3WktCD{(hz6QENKg{Q)y6DQK>f3TyQ-smI~{Guaidh$Fn+40l&he`KdVj&HCEvs_@=AVn5005xS`FJJ^ zi!-7mDk^HSl|br(EP`LPa-)~>mF8&20hi|r|g*vf%duQju z`ttJDGA7=v$zj2KH7hAA_ot3<9?{eylOnYgbRWt4g8*fP3uYX}SIy)`=IMe0BPe$H zu+p@|8ol{>S-`YJk^hy9tA?E~T}t5!L`$^ZW7rf2;daNevT0y#=>*%z@JvwAKdYn=`OS zB?GDkx~kT)%>9u0DKV3jy6RIf1K|o>)4K(`cTsnh`Hr9n-mT%I2L$DH&;B@ASh8RU z8KPwQ_Ubrh^-IFVqz&5*ERN=WC8T_1GR=5I%r#IO?0oXV1w)C6U9ga9j7WUDS$B6) zr(9~H*4FOw-l#JI#gnM0Nw2nftU8$jQa`qO$3uAsapM=)CJ4@;vA{s8WPwn1WfHl- zk8Tp1N2w}P%WRaL?JaL-1U%7m&i-Qg9L>EB&XvKNUtKewKc?l(+L79tpi>I_&l7&0 z-__8hG+7;CNV*Ww<$*rA^sDu`oaA^rYgzL}43FiPN;WmN#A=(mEh+0v#7T5i7Zgh_ z(6{O92AUFoDP;J?Vs(by(W*C!SfEm)vG<4@kdoOZX8j^&_PBW!cQnxZ-|3@NRykkE zK1n{krAy-5q*tlv`eH+rfO&36iH!1LsSagx+Ghd$@P|&=?qp%i#9E0F<%6D_&P%*p2Ne#*#+V9MA_bjKP%kc-rnbD z)`$HheZvP{*;7}enM)-pnWv>)Lg?MYT?X=p}^fhMD2?ujWu;D>Ms2~undYSN$0~LL`Np= ze|2<;!LUM}VDy(k(?uA)c-2Z97y!DFvP}}FEk~J)Kn5I~0cmH6=2^;yc||>hm9dH^ zi8DMRNu60&ej}9FKHlCmG&Ch`#vT^nB$40bJX(8=ZY^leadnIx{P6}ZQ*stICcy^o zDh0^FV|)f3&gw_de7@3Wg6qzCfw30u;vp&IK94BkOdTj?UD@c~y5`y04)Q)PKDEg9 z8?AK(1;pn&S?VuSIwq7Iwl>Wlf~+n4f?gmWNlDWwiKRMEx?lPqO${#LuV=21xAviR zPAmu@vUTFGXM_Rdzb0%OoLMNQ5`RO5&XH(tR3(>w*r+#(_{e4{g&ot0lN0Z(ddD`z zj%8T9F%|a+$Oj2(idoLtnk(cFWH6i)_D{Jd8){Ty{mBpmuVH``>Fre1N!W>v# zm*!U~L0^!M;At;(u_|_Uc4|OFLlR_jL4m+_tN)yv?{DmZTc{^^I0TptG#4(ff7pzqgZv^uIx@1q9JLXcGK8I(9q}C2 z8V`N8J3xN%eZA}}@Lo3h#`^9AK?b~z8&n$r^g--lwS%NVAr-g}lTG5dd7=`z^f0Wx z8uC6b@`Dk0E^7Pb=B2iJdxdMnJT$vM@)D0dw!A2jw^vqbEoVsvm7tM{ei45qqtzjr zfgp*%pFS=q1?q*z-+jFkww{$&ELCc$=;+Xv16SX9JE`B^W2uST#lRbxp(vH>3mS4`sh_hVQw81)aN zIGnmSaz9-3-!B@8`+CC(yk9bHs#n|HXc<6!weow_mysn1UJEh19Y+(%6}o5G!v6!V zNq#&>#LQmn2@kUt7;r-zqRPZewx98R>SW%>0p(JS4#4;@N)(abb}$P4`$6G*KyG7= z2NS)X*RT%8_e>>^M2FLEt<&_juxh5#X+OoG`GFaE@n`U-TFRnQ zR0k#eb@%zzd3DFta?8fsjvwMm=tWbF&nDn=9l)r|W#AkzKvf~++8WralsIIV?^}-9 z6MJex-X)%Pp|v;s0QK$z)D1{-J*$~Y92{z2y`-(o#*!E5M2=p#if9wc_9eQpwS&{o zcxtp(RgZ7MiptiG9?E5g3hP5I06P36SA0Pk8s4gI|rJ;{lrY=LPTBqY_pYXf& zZJoR{z%ZX>mF+l~@WYVoapEC&;ERQ)k@K7(zKGG{EgP|orErr?M{0Coo{!bS8}WED ztHr4Q?OM-C+#rCkn#qqH>>}55ggAV_x`;(FJstW!k$YUcs;Y{jKh40ujX7 z87_OphQ*?63l#ru(M5f_!7(97F*2;*|NT1Pv^dRvvwct#ndC>&ZQkrGw|%a=9C)w! z9+ADVN)HL&X2J>lUv^tM534M)Z*}DhQd4t?_uDCcdXlr%;O7UZ!M_5jZz?51w74V_ z$Z4ZxfSpgd(dK@+mgUfX z>fZ2VxBgdYdNfkXIBr;sz51qwmZ!1p(i7+--{5au#d zU2M=_lFi}LLpVc^e)@lBPw8PQ|5mcqlt*6kwJG56=Rp+G=f|E7A&${>H0r|Nisq+> zibN7N+dfBq2~-8QjqgQ^5x_@UWAHq%_rbnh$QZoM1$-`>#aH^9<^ur#>P}tqzCTti z^6)vYA8lM*STEr+*f}{Nc8d3D#~w-Lu%)g>W!HSzcsXBHDPSQBk|NOglq{<+EHJn( zBVj_#ssL(TK?Mb}{`ZTGX?Z*JR1At_?<{d{4p(48$D+2g*`vc}<61_Lsfr zn(jC2h=k&MP{NlUuj8&1dZBC)Bo%V@M6 zSOgLc;pzVUUSFZlHNfGrC|+3{k?*F5-(%aZjX{3=@G3swUJw4Zy_iXh1T%}2c8!*V#f2FzJ zYgVOdYHgEUk(?-lc4CFxW6}6IIWEW3;~B5lyTmrYIeNM;E4erY#VRUr4)!0(YNK^7 zq&S|H#=MvA5e=Br6Dw`NUJSUqMezdd&WA9MK{k{u#ASlcu$&d1)NnVO*Z!Dzy4~CU zZ%QudqBkqsACucV%#I11^K3vY9Vdm+-0<9xk4*|)4mdzMtzMON9aoh_8R#Ucw|l=u zd{GLZ#Vkjwh@x1heZOp$zxsC3<{749k!LXs$QTk*No%SvhBos*K|I>@cYa&zvuS-Dokx!*aZGb_twGr?bR_oavu z9DGE>=zkln!T&-8lUdeaqwm-YUZGLbebWb9R{;$~x)+C^gSQ{@dW&8^pbMX4AiaMz*e%o%Oc8zh94YeHeUV zn1rzy&i^5&IM2s&rc$me+yH8sm}yZM4k7^k2)q;g*Kq8t0bhK8xnv~e<2Zc5_9tz~ z@Se09FS(m&Wd24LafHQL2443I&2~1A&T$EWw?E%sZ_C)=yR%9I_8omQio3>6(x?(1b(Fcndz6z`;=w>OC8*DxO~ zIz)K5mnj(*A)$?}tq5d-@iY<#EJIQC+t+iqfx4D{(nViGnl}~+^@PzGOfmnp*Yx}x zl*i(*TxWb(%RoLmh#T;&hMpU__rg`P&xm|7MrDG}zdNaBYe6foQCTAQtE{buCy&me ze|5t3bnk;BpB-22;};;jlLoarv+OnqTsK0M>oY4ao3zzP$8T@Tfd80M!a|8k7l~yB z|GT6gc$L{07>K&RtH|YgQ*HS>;}V9p!eVjb-^9QQS8gR0qo|#&I%zJT9Ol--y!)84 zm2+G$eZ}kmr2E=;In>AhcJw3a=8>H^;Wl)aXB=s2iP)g|SA`ZbD1ZowXl|gN4jODa zP2ij&4!YUiZXWs9r~UMOSFYdG2COOQ8%;jf{rbrCD%h!>JD8rT#xfB#7u5wz9@12X%YQ**$^Y1qzj89t?wL)7y~4O) z^3X~!@0;6>{juLa&J~Wq)x+ZlKH5I^BQolA`#Ksr-XGCBc+Y1|@tlX+0u5AiAMOMK z<>OFbARHF#5)bhkBAhuVqpUO<^+Tirk$hb`1iaH4yh>m3J>ga#jEy;SD>}%<#+n_P zfy_B{Jz{WO3?#|(T+E1;S?N?4Chm2213!iY@Ob+G?sKm3d@kx#@!!p4#&sFJLysUJkqcU>uwY^Dv-d0Y zSPHs&6nE65Q#OLNNTRi=WRUQNKIhUH4o{2&Ggiw=&!m8uypVQtJT}q0+|DLWD!A(u zX8`r}(#QX9dLe7jAQCpx&c?RO7v+W+hgKj4CRmt!3Z#ED zA#bDGPv`x~z0>K!X?W`?YrWjY2DlFV{^;@PQP2Cd_<6%+`p2GMzD^6G=7R~y>d|q` zEs76T?i-0^aj%43wX9|N3Qw=DL4lIwWuo{2Y3!oX>KO5w7$H`&G;F+p#T$)#)~Lve zr?&*$-sc_5d>`bI@|mNj83FXupX<{JH~*i0Ae&yFnM$v+1$zcfmaDyPL?^eL&8**6M`HH4=nm5%Z4V zHd`bJYU(KI{%qjJ9E4!YX^GXiAS9 zPb7l;j1d>1t^GLOi-WlHEXfVnrfj&4)e((vdFrcVAaJ+=qXmv&cjq4f7;Dz#vz<5 z5b3R#Ax;!_j~|4{81NbZ`U$GtK|8wtwmj3o8Sg8Z4*KK&xu?AlNd8F1pC9^h&30@( zGW2zJRGcsA*mGMBMhWulf^ou~{cSG}xp~|E+}z#@`1ZT@f9@EmMne$r;Wk4A@qQ!9 z$hCdDKZOtAvnnawF1^ZojBwQ0&|gUg&}dxKv?J#C8S7yx+!DXPH z1r5KOtL!Gij}07q5eN>3!XOECn|}O#HRtmkGc{x*7UW@%^N_~4$v8ZiI`f{>mXg7T z_a3N%!0X@ZAFWJnzFk*N_N~F2+)B*VgWQ8l?V9}qf4#vq3s+!Hs8 ze1^w!zuayZ)HHvCpejL$x$0y-mF-n^um7}{Ms*p{k0^N6x}Q!=+5ls?w+8!1K^#~B zqCv%Y4+{kzo83{C^Rpq}Fdu*j7Q|IBG|&sL%7+oa&0pp70IZM#j68Q$_z*aiy0ml^ z)V--J5NZHi?pwMNG_pFw=1FkrL5qnOVp3R@=Dv-OEoe>}lgXR+utLAAU2 z7Sew?20V!v1%!pojNeD|VpiV@#&~e?TuZoXi&MTw8obLET|bvNI}6gLk&nB{O|OViSbxTY@I zGL-wSD^WlqC`o=Vk^8m}cH8Z7@!N)~Qmdh)?qB2xIAj`wUc249d*XMrL8q86NWh<| z&0s3|)7PKsz<#n$JmpV9^7w75yis z_iyEgr3&s2`VyB(Q!ay_l>4#Hb`D(6u_#;K@0Y{)E-@O`xMx0Z@6>wFuo(0Q8KGnl z!@q*~-X94450IZX+?E*~I*oSc$ki5!eG3Eo%QXnr)9uX6Fe>_JIBKJtBtppKc1`5?MgRmrxtI*A&1h>#IMVGg~!L=rXtMN!NI}W-u;Fb(#gr|)HLkeH~(*= zsjqdSSz|p%b~EQa2rIIkrz@CweerJ3O#&aNcSZ)A7s_Tm8X1(xZV-5HtC>N3;qa3C zf!AZCU~>zDV&c6>^$TA#?Edmr_m<^R$bRs&E|X>8g2^l~FFoz+S{@J9?b8HuV=tE3e{uF*nqsPZj>R63=?g{s=S@w z=FW(tla-T~QgLeB+8>G5t%ZY?1u-Uv$yWCzv!Q!_J}UXACSh{f!pgJEvp=`Yz4poW zQ!U0GLAm-)h_|PuQXz&!VwZ~j(98x~^iDEddSM~+uCEW{t(T%E{f6PjHupkLO;J}> zLr;AisjaLhsXrVU(9Bn^h6P7v{Q~E?E^LOl^35gd0rx4X87N?NKRkNmvU5be_86z9 zm{Kb$eaUbZt#VhB)fg%nj2iS&YieJId)$lnZLO}`Mg2)Ph<-@k#hx!Ovupad*iUV= zP13tG+irhlF#7tBH#f^Lw2|Q!+@fi4PTDgn24;bR8iMIsi2bn*n)LXUONPRrgqX{- zfS#9T))}PIDNKtlXC6hFDD=Pu#`ma!)+w?D1xK#2MgHxXtN#b2AywBW%Td_supK5* z{+?DX6Mp>s;_6@Zzxl4}+sGbjeEiLCpGS6sk2wvT%wyZtH4YXY`I4+#hqZdQufoCO z&bRF6fF01HB$K5w0+$Kq;3A`<)Itzvz7Tyczy2ZkX~A=aujg{Y+PYfg%zqA6Kk?r< z;IpdZx(gD8Ww{Y^;%G8!z1a@d|0g8XqDVSDy*5d;$z-({KblPJO{Q~9{jHsj=VmeG z{>K|=^&=E^b!)-2vRWT6z(!^~k}}uxwWXUs3C_}F@$cRb;ICxtUC>!?uE)M*=(-VS z)UCIrZXC-Q^8u%X{ue~tTF}*Za7xL1XPY4Sd8$zJwf&9$=T{gDoiK_S`k`vRZ5zaR zlnZI6WI(fmAy@m>iv@2-2X}|X()~Ygy`%)t;$WK|&mP{RRA;x|?1C4JZyixi zr)k^e^o)(xys(q3BH#V>b}sN#7a#D_kIBlnPu?tKBhBUb8fTaES`f|Xy}rIRgfscn z+*Vc9rXP+`KD{e}AF8A#%FeH}+gl)f;kV+EvaG@|uR0EYNQBIX5T+<`K2RT6BN-4A<>#(=(LXQNZZ z!fGTpQgif-?vPumbwR6@Dl*sNQX&o1UY}Q8UcL_40cLTqgdrY;Tfygbx0Hu_1$vNU zG&||TGWRuwEn8b$U7cAl4RLpMi^N`8>*klGE2WT=HWCJ&(^{eDDK2Rr>Q=oy!PNVQ zFrXbMk?}9;j7HC#HqY5dQccYpstGsMy30*xy|b;#!E?Ut)%VjWl4qV8L;HS&RJ(kN zy@`Dy315%5e2murGi$r^(e8K@CSDTa>*xYa@bl2%>u|I*8-)XyiXkqBxOhJ`IGJrJ z(Ftd`2p;1u!V9F+PyQZnB3S*f*z5^N21UV2NzbY((%!;D!_7?Xdk4rpF{{F(7SLY* zruTz^n1}J-dl7wma5@el35ez_y7ig{*FEd1F%UOA=VNPpE^WTw&%g20LK-UWCw5$9 zEfF$Q1S;HBKR$8O}hQ3HNJgM zEgH`=zIL#~q3)4mo-w+a!+M2-+iGfBN=lmR1bTX_bHBoPwhHRQ3H=|wKj%AsGSdbe z^EJ&y`%ch47J-ncv5!p>gf0{3+cyxe^S%6{7h#{aKE!<1z8k5i?q5YiCra`>r?fIM zT}y&&3mY>l>&riw@LB6oD&tRfm2hr@Bv@5h5jnPAtSK z?ZLACGg0MZTkSGDBP)C-Vc>1Nc&6os8g3!6I{|)-CH}M&?&v{K_5O7F91ShC)rFO8 zzuUHHKZZZ;ItAr4V6jIB!CStPR%o`HGIG8SExoISu1#P@<-X02>zVVA$uSqZy zz9&*s4{M2t$RadU7!8H!dvbx8DZoW1{~YbL4G8yG`-s!@GYZ zmtY}~0Fhz8Aia>pS6<+w9PsG-b^&A0Qo)BMi|NecesI$I=63z^0h zBxGEB5u{%Sl?@2I2S`b&`G2dPkn-|DP?IRI9DL`qzRqgCV1PgvZ!m3*zFqzT1=I}` zoiF+C-sb_1&IrSZ#;5Qj%pD|VVL?Q{AEv(^zPpb3W9IciLC`{?g#NyORkAZv6SRcq zC{HQo7oegG$|ta}WaVwpPVE?GhiIgwiINFH48_%Gt}|dOf~)O*-#ADyzcf`9 zZQdWsq|uB_CIOC`8}oaHf6t7H0>sa)H3G@#_X(eD7(6s zJb2sNmC*4vX4h3r!e>WEk%EQS(Qf%t6U1;LunRA-xn_-ka}CA6F)@~wm$WuS6{*G{ zifc2?S!^kn6*Zq#b#?VLt|5@Me~1(EQ$vNUAipf3aD_7x&2Cc>Ql?L;J$luT>Y~P? z(q*L`XO5@+?@;$SExu@01(~>**_HGdX#Ik8`l8xH#FmES`0C$w!wIw(9aJoF*_{89 zt@lk&lsh>7Bza!;;}KnGY9M7~u~PmJio9(%yItYcv9f?b^0PuPSK)SQXsF<3SwZ0{ zPUS8OC36zTI|K*c9(e_P@L3@F9`qBoT~rt}oghlt(MER`I#dh|`}9-P0Q!F5@fR!| zXcRI_prUgT~TaL6d*+S5b`Oz9AX1T2iK=y0^O1gs1!t9WrnU%laX z5nB{Yl=L{-ChE+vPTHE|Umt z^R^hR7~tV>-3NXTeO@4(G)jz;Q%ojFh~ns+C^(`FehQHe$)chLYz(0h3O2ypt2!nP z)$gUOsHv_x;V=3+>-CdG!l;!5P#~+eWy$#|(mcUu&^4uo{;5bn3#Oi|a_JWG*xM^8 z=&i=rtAwqhcS*6c+pH&-N!ec96uSUg2^-#V79QGe|ms-?d3$!H_@ z1WJT`v(V0BwU9T)t}>B--#8X{4?^@$fEQ{!nG&1-A|}u@04o*!Fo#0=I2c&inF%Z^ z`u-#(Y?_9>nCz4uC8nr?!%`9qkx_6!jEIj$dg_B^sEx6@B#kE;Cd0LMVeMZ({VCu~ zd33v+77n&sX$Qq`Mlr{du~T|oSgq)~$WrG2B7o zQ6q5Fc)2-O<8&}?{py0kG5ma6({@CCN`wxJ5%%FI3Nn$^;T!0R>oIEYAKCQlx1tBaK0e4L}$-;s~}C66z{pBtkhpe`#{~%Sg~j+H$o;a6NQPqY&T&B{5}} z6aDNM5eTM$gNwTeR#dOqPn)U;G@#RBuLkH23c{f(tw&iYFRy$XU+#M1{|T2(B>Tn{ zuA^ zMO5w3%j=dNR5OEW@iQ3$Z=gjXmW7>n@xJ^31l}XWz4FtS0!%yk$ZWG(i7#%CqKJ03 zHYzRcLmL)7o=TL`KsaXtV!@sCV$WhWUwh z##MG|=shM5=7{mWChpQWwg~6+_1TG~ZEim!7{X8at{}G|YS?-8NH5*jn|xi(9!ppQOC<{?K5h%;@JySs*Nm+18g4T#=;g~r2NbG2c!t7lOK5x5PqLXp&@6cD< zkls@2C#Wm9I9?V&Sw*SD!MiXdUP2P(rOL=+F&&Po?BnrI$wxUI!=S@ICs@6kbXXv;g6_qVhMVBFrX~iHbiHILp9SS{X2{TQ5E*LCrU7^N&N*SaY4gx z`>U&7GFMtREPcdvwz%j;SOQvD0lt4L|5m7omta%*C=%K;Q5wPPdxhh)`7IA20Ra-c zdAkT$BnEA{oD}ZtYmx|c0>Y1bG7VeEP>dIj)dD3@7zzF5rGn4gkTts0lk_bczCYXtzU#YqlB%8lr~*do&krjjKnJS8DTZa_I0Xv?OY__*~198 zT5$yccwKFru9{yL%&l2RaAvzDS8fjn8BOgFI~I?h5#&*po_VO2U;c`}TvAS%PF_LznR4QopvlRmpv$IW#PBW`^lZzc{{&~feWM^@qVX#x&batbTK*$d&3VWdH|r@#H^RNJ zEyhLG4jwoX+;TJWr^jvlEERrfxj&S(*f>x|yA4%PLE8Cv7A31cru4iX-eL|6S$ zHag4aGV8Xp!7U>CBkqjveL`$4(t8{nTo(!E(b}6?-3P*37F0SB3XluT+0HyWkF;3V zX+(6d)a&4=u3(Z36p;>f=nBZs(+hu|x3=icyW>68XPE||&+ zw*R{AeoE1j%aa5P*AZu6pTeSz*o-A9D43hd&kXkA6a>sqdv2;WsQ-(s8_Y}CwM^Y(hD{h=Sn zKR=;#_H@Ab@Oln-#l18p;v^##^6Dz-zIrrNZ?t4-V`jA5XuLJ}el!61ubLGGEepPt zttc8BeRVE{R_cJXxfU{q_N}PNr2K|jE2baH$IrY$ zklqtthrW$EgP|6tT3L1lk1{SUmyYYpTGd}-bPEH|QEGI1f68>s-7-}$FpzO!UExo} zqmQ6O&a)&bxDnE5$xqjxO+X?@9L=ZlWN#|B%-G zP~3}|Lmg927MGvxCP^r8iDoz%eH2)mgwbL68=QWp>_~ZZ2qro>dEbU3rci1{d$3ka2qB42Kk^?_*%s5eLXA+61w2VM z!|XsPP(c1}rrWu|?dW&wZ5XaPe`kApd0~bVLmSr9nKj4@thb-#ibmH}g`N8-oivG9 z3UXnjFLXW=7GLa%`!@m>fxd35V>hM#GI(v;8S^h11iE=;w4Q%2 zuN4OG<@>Tkfc7_@`K$yfwUj7qaJq<}f`#!zbv1Pvd_~+|&<^(gNj+o0>cDXiC?Xm= zZdPEw-qq?5p6kg8;)&zsTGKnd&XlGl8#Z+<2wFkJpIaqh=(;udUCBD9A!ojDF|Ke& zPAqa!DjdBeIlVxvB2a3qiwc}G^kD#;8*rwwz-#pS4>#{(I>hlry2~cwT>Yr$Xvln@ zoTP6>8NNl2gR6BL7}8Qi&u+8X&hEwNhs*bt^BFmd_8cX}Bn6Z&M?D+jD@3@Yp0-A@ zLz!SKkvN3H3ddd3#NMt#x=s`g_5vZls4@T@AO72!L%oTgbM#@0EHDNd!dQ+y2w>4 z#2Q+Zu~*lp>?@RMYdO8)?DKr-QGu>7UJ144r%wc-8EF6;2hr;@`hoojH6(x9q zi8bSa#Y#s8zkR2>!hX!y%5Z!ym>xcjBZd`RJ;ELp%V4_Pb~jd0{eG$d2v}T<5#Grc zmcQz8JZf9IT2u=a5_+#Yp&1(z2fi6c0XwH0YVmMi0cYa!gN2doK?<~$Gvhj2uJ*w` zA{j@9vR4Y+tNcU`Q!;JcO<8dOeEtv=sdl2>O^2Lj;rYY@gI&TMIxabIF@9kHb65$( zA{!25%Ld<{_^UluP=sJB^*Nh5O+666fF3g*5xLsrd^-9$Y{G2nhH2;G#p|vC0WIuRZOnl&G1i-4$J8u#mVlAKE@dZPS070ju4|TA+BbupgswGshe2V4M^e6w$s1?kY$*qeRJ=zMiHv1Vthk~^ z0XQ3?;}8!}UKFLZRtS7GN$bi#1*_!*f`(S_wSEYL>TNvshmp|Sts=Q{h6T` z7lO?$#^@Nx#*vxs^{33jVMtN%NAW*H7IZl5hK9U_A^(An|2~1OUrV1mJ$;NR3e;y_ zgkzbDN;EtShLT_*FUnBX4sX_Jp?6u<^Q$1(Vq;THrYw-Qd?S$ z3G=3zlJ$;kd`P?82Vy^LhJ3|ibE>ArLwSSr<7YR1eikisJhK6TT&~%^`Jsir{?K@v zydTfn?lGxylA#kU(NNk8~fB)s4{2^mzW$*1OB<3(PbRc3HPwKJDPbeK#h&56(VHOhNSA29|OQ*MxD6mS$ zLXNaBZ>_9krW)jP`TWhNPvc3lo_fm3J&)V!bAtm(n)Q8}7kFV7mbx1t|MQZqUw{E& zzmD!GtgQkg!`3~H3#PMErb8g}QQ`xw`R@u-`q(9LEa3n!aec*F%UzP`RZGynR4mX< z;!2@hszHVcNDn}sUYy!7Z*TA5c~f-Ds7GtIF#>J`8(P2^@paLB^+&Vai(Rzpm4Q%h z!1`NaB7c}f#@rJc?p%u{%HY{zxN-ps+W&;#!)TX6q|iP-IX!j0z-B-!a}*#gsCoR_E`b z_Ec^8dp(Q7v6b+`s{I%=BGYcvF=?ORf54*vh1;)+(vRjhY{cH^b!6E3z2uK*wS- z?2XOvqTC_>CY7qkP7E{Zrt%%&M~*vzrSGBct;LAk?^KyL%JY0>)5 zwkj!*w20eb48ga0Npj2g2R&Cc@8)NFDnL2t(_n!Er#4gsKeGZofq{%o`mh&ZYY~b8h+r>Ww z>0K*%f}mMgOa{Ko#`y!Q!o`JTAG2xuMHzqqJlf6}Y(%XkohD@TFV@hBQ#ya0aF%Dm z1;6Lbie|e@S{nPxiVPJ+?x%B|FGMuW_8Tg5kqAfI!oc}-9aurKh?$x7U}-NLoM&0u zJFQ@-B4Mh`U*+W?j>2-nweIeZWG$h+YRoJm+l_4O?oY0eBK-{gyol9MEJHAVWsfFU zr(MsuV?0lc!iB?=v#Cp3?@sai5yWl-taj7<{nqV*`$-FD#P@rnA@LeTT2~|(x?e&q za>ScL_)xLsP$lw}t1xwMrhW+R^mr3Uk%h$Nr{V)Xr>D+itpT2jG4f32 zP|j2gER*2E<|WS|>J=3ZJEtb<=)vrri!@ zr;(b+F`D0bUxO*%65;0<1KW(DYHt zn2NpA=t@_;y0_PfNUdNDL;#5rqi&s2%otNvCLn!4Od$F%PV?Nsx2V87B)@QY}bdc7KKB-8_U3S2#5ups}bPL6CvhpQh()u zYUg45`gti;GA8DVWoFeqm^S*e|(&}=WnM6&HHb(ls(m5D zdPkHCnBX|YdRooArM!7x&_@O88q*vd=Khe0yZ%ia+3EP9BhzZHg+&NEN1jP}ayf&m zCj%*yX*f;I2odhaOqJt1QLYP*IBJ|Q4Z#}RN?B0$*LG=mll{+s0U(_fOcg9GE&ces ztCns5TnkEVsq%+?Y*FA1>!uFqkd@#amML2oYg(oB(Ty&vyJiDH$8G+KVTe~*99Rn2 z;qXfzp6-jl{w4nQuQA&8;XIE!D~S~Kv@CNC3g*aA)c)XRTv}>KWINnBjS+p-dX>0I z+gk3SiSYNnq>cVN>=oye5N+1FB6_IS=kDh&_lD1vVL|)GYs+g}Jtjt~&fKlBHEWPJ zxgQ>=guv5mb`{s*f{MY@T-s}-d+BuH(s5m3)Lv=ettF(h*ikR7`U-}oF)ngaOje6M zvfm+nci8t=-nUezxiCi^Cdz}oB#&o>+ZJE_V*S`SBj6*3b;@&)R^RF1kHjncvG^p? zA61H?2okE=rK8lt@ic??BEn-?Lh^Q@M|VnUSS9Hv+#=Qu-?OB=_LIHnBDv^gtQly5 z$$_xaFjM~6!@>U$;0@lUzZ?7Ar#?vkF8~HX`M%U6&siGc7GkO-Wj-4lm9qD)o%6c< z1w^zO?MaL=`4c|lehHmkSpnBla(W6NgO)t`+Sk5z)m2w9#;jet_Kk0R!}FeZ<~!c; z4snDL^=6xaRX~BT^+^D`JGbCLuJ9M&VmKh%i0~V?uP2l!pcXu~Be^9V?1Dq&2BXLT zmN6XSj3Qg;^cyUv=5%mFeM8N}_}V05!K_PPfQC%zGhkd}>t#kX0k+SJl=?_GBt zam+C|zt2{mLJaj>$eAqi-qAK7AsGvWcl5RB$d1wRG1m?XEiwb^FHF86+wS8>K}1NP6Z_tme# z2A&86ECFVrvn%!y!G&$Sf)m17ZxfbhhAn|514D~zZSOb8fES4%nZ{~Jm~Tid&s}oB zo%bb(Qz2vlF*y^6k;Hx))-#2QV$NJwcTW#9JI2+mW;;^Ck=DZ37Uu@dCadK;9YGU4 zyelOoj)$%@{jl%P%xfys)zN7y22OrICanxBXBgU<7-uF=pFGVJ2s~0SWv%CAhTJk> z1iE%+GMjobH~i|3e&*nj@xHFk=B4!~{n1fo%Bwf^+;sbm6=fA;BaDT=nH)UuRuFie}1;Vm5c*v}F^&9`&pgm0@&Z zb0Vo@#gdq`FbKRwB0JxLIU#3#;zKiOhO1=tVL^=GgkW-}&u1gdnRvDj_h2Jro~6I& zGkIE)>r*E(uxa&dOL@A|Q+0`zIV{?Uysx>$M4>`Jg;@LIYLL3_dQS=ep&?CKgwg&FVQV~$z3 zc5Tk~;#C?R2pSt2gf_q(Tb4jb%!)bF#~{MiGv`Y5-h+c zLd<|vtj#K~FjKnv+G}-?p(IL-k>-8xeXpT0GCCH{;7bB|p<_fjM(~Kivr&5NBMHY}-X+Up}|qa!ar?4GaJ)>Wt3J_RII%FBaA71^#A6#|S zwze?4BS{1hEMB(kHGlCJFMZj|c#Y+(f#wFSt-%Ygfv)KV8PsKAABJ!->8U*Lv&dDr zlNjWO(3>C~@#3`@Nx5;;Mo)+zbc>rvLw?bY9`l+0$$}Z&*&(DS;73?iXGxH~nVvY_ z^CsKS)5JhLX_<9<_Hq)JTpODhg(`_i35aY$fjcZLk!x(`V1k*PWyh#s6719?3SiTg&6E%`fM{{? z+9cH6`SU#&DMi>Fe)!>ZTQtSd5O>^h2Q^+gQCh=j=E9SvOFQGX$ z7QDg3BSgGCDRREWM@b8s_jA*D#nJ&1U)@@nI>yI4cXTkDk^73KHv;0|JgTW>qhTo_ zl&zvboOvxRPD9R_zZo;@pxHB=s-+PQLR!(7y4x$F;sp)uy(`3zL{r0zGVnt|QAQW| zMQwfVa;MHn`iy*FrH<}{gG9GRwEeT6{o?bV{~T1OgUl%4dy@%W1|7cR9dEz#$}1TQ zK_xg69JGuYYwdTye)?GZvZ0g_ZD2WcStltENYzvGbsa3vd!^hFyHkyHx_%T@-*}!o4mQ9-o z;mg1I%@2L>Lk!qZ4vV{xTx*K$D$3wFQZRz?)Kd`Qsi&U$-uJyPXVGsu_uT7${9_PG zye6Im*SM|!{vZG0Wzy52uFuKSi9F)(!_udPp^jUKj0R1HMre-c*woJBqN_tN`XKD@c zd<6vv5a46j=%AFP&=f8jv(5d`kdGeS_OKNcum#wm>dF->jA&u6Y`2Nw&DqCOzxnlV z6bcJER)_h|ywbUgcwptqmAWPpB+dzE5GO4DFn2&ea>G0sxFvtwlOC*^?#Z!GHdr|n zmaLiKs%rF~_$ph8u`U5-skL_bs^~y~gveB`KwDXHmrD~SnIXL$t4e^*-LPr4dLiV{ zSv?%>LqlwTN|!SrmJ={I5j$i6;?&j1Huy-FcElN?92H6dVTw+qhb>k(V5repnG z1NH=kn5D@X(zDyU>DS*s1x)uXrzhYFN-P}Cm^N+RVISO)2kzTckf|FT=yq=BA055& z_#>AzlxE7tGKVhCyz~Xf-+JoVK9`MiWD*>R2GgE zPmMr_c9LMGr(wc0CA|^~JZli4P!G67e7ze^H1Y>taRSu8{L6cZ@~?dP%jaKk!5iN2 z+OF=thKAakZ@zgC{mUy7t%Ffd5U){$-F(Z2vy(_+V@+l|3>Q%2g;lvYxr@_8~TW+~!$!#@&z_G_^yo9kWxK+-azBHWtmMF5krafv5h=j4Y>1}GE4onil6W8uCM@_F) zK!kg9Fp=Q-5aFhc8=IS(DGxR$^gwX?jV)ceWP0_U$@i3ykQksngj{AaY;4zGf8F=4 zxB|dR7N;|#2%ZTWDpI;VJ$+YPafR*2-}le|%$3ac#ExhwiDeFn8V(JmVncRe&2{j# zxc~(yIeLDDf8iXkY2L154|gMuQ!7H8SWWGm7d)Tshq4SutL5Lc($I)GEL^z24MbuV zguBGE+IQMM$qZrm&`+4ZV{p*=Wdrux-}?4~g$s=rFpI}iKq07L?L5;J?BO2U;Z)}N zJQIP8RbPe)+^pfz?gfH8_qorFVH>+O%GXm*KE)@{fIdJH5XPd+WG+2m>fOt~dHI!B zT?MAVQ-bq+hRCC^QYUOo9oL)3`JQ|>ZrteLa=(@*k#@Y{PP7DETIaR;75&x9(9=?& zMxGvWpo<rLxfGG?o&=751timA*b9@|Xmw4tCf!Yci ze9+(AK>k+7pfJkBxDlfx3ODa0CS-5BjF_DS&iYB@spftIaLWdV2!R-C>|q3#@pEHh zBUCgvt`u*D{Os+;WLs)I969Fyf=Ds4!Ned+sxU!iBMe*`{4hak<+?Oe@FE+ewM~Av zyeX`XG6wR@tC@9ScNkq_9EkEjwqQgzGqcy`+FD0mg_v5O)-of6(G=&sGt;L}o?Z$N zj@Ak5NC6PoW!7%mv=xAgARBdkC< zxO8e{dsh!l+OeT+yu2n;b>#6!WPrk|BI-;S>v%CQv=t`yverUux^pxy=)pXzQy@U1;t~mC%<6iaZS2s2`dD2i{bNAi%FI~EH#@jm|5S03V zcw&S~rui5N=rGF+ZPjtrl~>CD)?07=%x6C%H3`U7bnMZ`yydNL-Mo48ZMWWP)7Lr8 zP3;Z?>1cPRh|Q=iqP*?bzjhQd`EKc7Wl5Q>y%sNC2u;SRKXGp_G!QmleS&ySwskOj zrvHiB9X(`VA4rz-mA1{B#WMOyD88Kw>uSRW7Pg)yM$1cUAVEi!Stv0!G|p8AHE@}U zF7F!}B;O|orYtH6*;>(%$5DbPY7~p66x?)mci(l_UH^Fg`SIcO&!-U)I>%Z#IX4_A z#5Mo`k?*|i`PwzT+Ru>t7!VfKDwx8q!#4J-8*W&%a3SQOlGZDLGL9Rz)%nBrMj0GD z|NMWn=;TwM`jlg1+S@ylz#BD7x9e>{S1a)O1LaJ_}JzzbDEmgt$j3m z5cY0FV8MzACKwk-hpkuwT{^aJt4|i>?v~hByP{+RFszd>l!7HFi8H%;(uFJwS%kaTas!HPzAZ z?0f#-|KoI4@%alD;42Xzrg;xP^l%8q6D()Gl7tM-n>Rm3YY#YQW7a(cST;1= zv3>h@zV&TI8-FOlU=L-Qrs*{g82l{!w^67BIWJy|7yy2d@E2 z8^_22Z4ks?u*#X$abYK}g2cnC#$gAU0O67=;Irh$e1Hj%{{c|*mW6H#wJeES&2u|6 z2_#T|R>-W4(0gJW(G%aqb<>_C=+oZ6Q=p;5==^UNUBqClL?aXy2HgN?VX_g!C52D_ z2+1~X*uYk8p8wOI{`4o;|Afy1lVi{6>9Y6IFI&F+`7d|@7eC?$2Yq9sX@lU)Y?|zi z)^={&j@j_1#+p(da_FJVkSQX}l4vjA7%~aEb1L!llH!02xGX&OtstB>$3+yx$6%&Z zjP3TG%suyw-F4UM9bID$4GSj62M;^!;FFKfG*xA6s@_n<_R-gF=~rd-3CCtCYmWN) zjceNb$1@eRTf0Vo^wXOvYEN1^C$qRZV@t5oV&+0qXKF4iHuaB)*;wv0Xt=XZo`>Id zUE!(3d({xKtE{Kx-`ud{X{pMd3l)^dgdzTyTa&SnFN&}O2RChMJN1;EjxgJD*_ri- zBMx!KalYK54@l?<)5YeFEjRq?SKt5M_iw-LH)NZ$FTF+X+llx2XP#-c$|y*>IinUL zG)XmKC4gWNmcj?`y;tG}du10Dk;SIM7H_Jb1*jV~Y+}f4XlUdBu4Ifz=CAqcL(cjs z8}*`5yiT6bra51+1dq)2md!}$T5(ql6(jV0Yb)&=)~t??2Pc%Bc%7P2RprP&Iv~v8 z?PgqAre5?CONNb6a^~-0GN%u9h;^g=s9{;nO#d_@LSX}l=2NZ22GMc!G332V+0Cr% zGwzB0vL#@28C%d}1=9Ze@Bj6$f8ELwo>L+7LJ6y?YwQHRX3d&}>jr4kJMX;Xjc%Yz}#oJlK-~LC} zt30Oq+0TBi&IBpeN_y|N(m&5@WjJ{#nM2yy5!G2}>^ zKH4i3;8fZ<`AJ5@t}=FFF3+-$!y`M+h#MN3%&groao7Ewcir3OsLSf=In_1w2kpP~ z&_go|n#0b7g%jPGLLAaxQ#>Zz&=QlC2V}Yj_M0d!-PGPG(5qvx``YX8I_|Lj%8#fK zj95|-kH-^-VDT|H7y#(5s^s{0_uuxIILG_Xsz=WnL)!yYSw6gO{RW%Q`6I{eNoR`B$0QbPJ|D-_RC{Nq zqtF;;7%XXV5mc{ygSF*{-%vU5x+Rhtnv;=Jj zCDbLGsKKd(V`F`N%bYptM&C#YFG7KIr_d`%Q7JAKqkLW}B~mH0HC-M{58kd2ZXfK0 zyIqDxpy}ScvJ!l^Lc*~g)m3#4BQ7kYBN+>aMn^1M-825)EtAMOyFL89eWkV8R}|;u zJ>3BUo^+C3@Vyqv37ub$H~$1%aYfr6~uSO9y_K_RHXJS6Nz zM|+1#>#y(^xT&bbc6uyn7V}B~KS7o}q}Ny?&P}u0UKPinYDNASb6B78#ivc`>wk$ zIRAVn@_{HIl_?OY0hu8PK&JkRQQOA~n|5b;ipiTa{G1Jkno;D-A!OA> z#$hvD`aQOkn%2t4ZnG!~kPByXd3{&Vi$uO^g{<^ce@}N|K|M3MMJ@4tP>jq(E3W4| zZofUSij}o-Yjm{?fG`8Z!;~ZgTO%}SLsODxhal#{0)6QD>NRViKzKb_4_?k|ZY?&W zpv50+)~>zpzWe|82R|VFKK#*-&YwR|L}@Gr+aBAZy301q>%oOC8H-b(na%(?o`)-x-=nW5MHm)nW>(=(oTf1tS>Ko=dLUz^5UwMKU zz=F(3W#Ld&;aGq7WM!tYytbiyEHl`XX|Bn<@aRlKS>09F-Pb+PU(>MkrduAGDxKG| zq^6`Y(=;W%p|E1AU*Ux%VJ>H)5XyCYGRV5iDCooh4~jBlg_-e!Oq7I`lT$UJB~K}7 zo>7@S;}9Ws$#;>QgltJk@R<^YK1mRh!ZrCa~)bpl#o4Af~pOI^3kYEqShaZ89<+1V|Jr z=SWj?Q>P6VN=t1-heXgG=b%5kZF7Qyj8H852ecu7Z;_Y+&e%4P(XU}9opcGUGqe6i z5f$R_!11who+f+fpax9KcEXcO1G>CL_XO7ul1+>c^nn_!6j)c+4l4=Ogl&U0g`K938csU?1a9A)<$9eRPDBB_j+~Q~4K?iFbogKV#h;&i@=5>k zAOCU172mbXojp+WC*u}=cD9P*-tmsVj`9-c7TOG@Bp8QE%ZGo440@_V=aI${aUR1ITp)JKe~3UDscj`=ZAi=)%^JQh7B9SE6H*V(=*4lwa#hz!4H10U_l6HWIQ7x1n;ud z>87PimcZB?tU6avf$mwme%;4D`qBUV;w3ETWWpS0S`H5G=;;w(fxOJEXvNWKTyJUH z`u-1o@Q+VBt*wBY$)vAhvfcK^?I3j$tr_BLAuL0N%hL2RiWI}msL}k8Cb=LP9EJJg zS5q}N6UkP2S>CR%Z$x{QvHr$!GSR4s2*4lC92yPluK~MI#4d+JT;3b-mbr0(Lx)RQ z2(cgGkbPYZWAGp0fMR}fB8ydVMx0F|7qrgX+_u$>k#mF5&VaX$M1-Z#LyOMJs;V9C zY(IQ*2^+wX{@?p^2EsGjqG>?RZ1F0PdK zBcTdRX>@0OwbdKC;UfD|i?;pBo?b?k?m+#-(ZI_WT>(j0H0V_dR%z!Zh zL56KD)T!?tTJ_+6UV7=l2OoUs!G{>CQ6tDi!(9nu2aktC6-5y?#WRNzI4F(d!hNRK z6Qe+GpguD%&f+O2!e>U={cMR860quR7p{-j0&`5%hK0n~MW`Q7T~A+OWoELZ?EbZx zTW-v(dSL6wP*qLoc*$hXAqUmGgC}}3-JO}LIhnG8 z(ZaET(z2pc_Ma%7IOzJ@x7@jQaKQm5-hOZUrna%OU$F3{M^USDGQ%9rCDzG@bt^>$ z<1`mgSj^NCP%1uEk;2SKQD)e_#+HpG&q&AsKW1L}=#VJSnaQ8IPo8mza1W#+>nUpq zND#x@T$a!r8W}QS4dXr>K{r-hSmKn51R zR)pMi(@od^{#RCDLd5isH*FG6X(eN6Shu?q1dzh)sR^~)_D<^hrs^?VMFR_r6GZ_?6Dh;qbN%}ai*=|gf6;P zNS4J1-}uHizx$o<=%;+PE8$tl`?}Y??r;D0Z!Hyvijr0og3`zX_Pzi8@8^=cm8HRnvLFvcuhILlpAln(ewuX@KeY!SY;gU zQiRz&1b$_Ri=io)Z`Fej#;j(3UU=b!>({M|TWV{gOG6mQ^%P5HLjMV#bfX4veD}NF zh16QE&X^6x#!HB>aPcD4(-z&7n5^&#CmuFY;0=o@9zVS<-=S`5)B07Vu(;AOysN}V>$8iha;Bs zW*A&1g!K)a*T)@qT!dC?C~=&qlTp=qW^@q_>P@3ow1+rF<#zY=H#XGv^bV4-+%*k# zH3qNV6b7vhLH-wp?Fn@OT4{*SaDqTp<${H^2A$c+h3~p>;rxZry!{_!7@%Rogv7W@ z6Ff|>r-K54hwT>r@F4Ldo;6ld*Z8sNv2^=0B(gP5fV~_0y2OVQP~hEW|qmY2k&+ofl5mNv~$&)k&_Bw~^g4ZYv8<(|_D*IaYWnnxZ%2O17Gb+dHYGKVt7 z^H)efPv{0#H&zz)zv&YN)=H0*Kl05@F|+%*&_d%pvFlXppBePoc40%)!y>S}HAJs_4}W(7DP)1wKMYJW^P9+3}Dw=-~l>%(4 zsNknFLjYVfQ(Bl?WuO@iv59f6Vgpw=IvZgreQ(7}aiC`m7|``Edhv@Xwr_mn8`oWT z-PX1?(CUN}PQVzDKR_90M;H%@RWIZqyRfpP5LCs({Y;w-W)&6N+B?~SLXYHf{l}s$ zG#0X;jR>U^N9_^q0FvU+K{3J1PqyZ0sj?y@*wKrM|L+(7ch{YFhEo=q=SvcwsZ)2u zCOCnS`Th-S(^_KuW~ z^Ag1D>=C9&24<#KJV|85cd;~#h!n=7-i)hcl6A*ne<+Q$*B=ui^oAg&f4tsm^2@vK zvfKHuZ~ZmAb=1*E<84bXgH_MAI3eh^(l%6$OO`k{&{8(AJsbgI_zn*fXvrin7&Y;O ziB?Zn7X)l}=&MCFsKHXMjeasOVxq%%W-r)2ykZN(igMc2;0^4!#|vaMugIn{U6uUS zTW*ySenv$z)tH&8O%5Wl;UHh~sDR?Z!lJ&R(kk1; z*nVMK=4ZF{+`IP9vBCa=iO$w}WhWoM|5>LrG*t$mUy!Na)HhL5(+U(m+MT)lyAKza z&OQ63b^A4Es;wX{Dj&}bS7Zw6N=r^SY;I{)rn7ydd&pr<+c$Ld+&0?LQg-<92ba`T zHcnO+jcTjG=Z!&SY`e?atycLaaQ1#PEDqw4vMZ`Q`s3&M&sU>=Ml zjeFZ2x08TJ9(e>0oj>*%Kd)N#;HN+JskM(jiV}zbV>LB33h}nDhk|UQ!^LKmG}Ypd zX!ejUS-)=m0}tF!VNhngEu-V%oYpV`l0g1IkjV^dC9)9x2oBmZ^!d7E$&#AtAntd& zfJ7kOLBhe7z@EpJMp=fVqMgnc;?h85lxGkc8)r`px`gz6IFR(~4_X3wd80&0hCFsGdXqjq^$G6J6 zs6ed3&dc#$LZ)UG>WRa9RZCZ+19JU;_U;2fkE+}s|7`EQq-@G=dI3U*1SFs|K@{=A zy$UEzub`laD1xYB>;OSbp+|D1OqqHOqq_gc7r z$M`Zk@4Pc-&Ybf+-{kZpQTKYH@EJmdEhO>{+v8I2b2G z=)ke#DmuG5MJ8}|qp5Ve>9gZbJI$Rp*CM&!yXYcCA`SAw#M--_iwL^-l1q+1?s$IK z?LF6s$74$sn@dKSZu*mF*1Rwy^khJX34jQB3(6EV3);Gl5$b*Uj6sseUa@-h`RASY z^>fcPFak0Y#n_k117L(Z6Lg0A!Dy2U`ZVgsj2%P&nTE!p(O6VkN@n&XcssSW6LX=D zRl)OXLa|s(VX=q;JoKlA`H7xA;=>;%XS9uI4WK_H1cdp{czPS%8DbOPkmPlBbu7UA z!h+>1YuLJL)~(OW30-ZhU^GQ{8ZBArS>A{_Nz_o!k^@N;?*(MnC0ZhQ=C)gJ6STAc ze)Guz;|GouLlBP&&oW}q|Lfr2tp~&o4n=LrQBD##7@`GJEG9DSze%#wy85H;p_HsB zH)hVpZ3A`%{p-R@nxB9nf+bKVfDGZ!qn3C{^a%D(Sy$n zKeM{KXRx-ecg^u1oj84Tc3H$ZCX7C$f|X|^?NS*fA3W{Rr#5#tY*G-jzNzo`kF99$ zn7;q}!lMu4DTgC|AL2MOIFf-wY@v}(o+Jm_FoYi@{t8AnY)JUUYBPNGe{6ftOcI(h z_ENq11pr~PGJ}wkjS=2fsDo8lSVRjx{OF_h9rX5`zza;>(qZw>X&3f(OG`7+CRXf( z6Hd7EjypCrHBA^dj?N-SDkn@JkI;uCby&QPKi1+fQ=(MYd38Y4n<~i_fR}cbC>lm^j9JUkIpdkp-s?)pYeU~dtYts#<;<6 z{uGC6+Vq$>agHNYpf06TL7^eMZ{EnOR8g)}pytu#h7Bs2 zs?)JTLc;;@WmuWF0J@AAN)ec*R&|=^0tfZEvplsU0E_TPxWr>plkU#NR$35hTfTL} zE~9gdBEvH7YUlWgk=EJ@bM}o(#AM+?yvmP%{Nu7rH1dE0-tV-AhDHW}omHz>fk}3} zQ-eRz&w&gBBRJ!{R++*)=|#mwjg5^dQHte!j{N9Hod4F%XpQBZik^Dlk%u3-^pcCW z)Yl1+ee21Ns9MaL(%Kk2Il?Y_^pQvOp>Ti=wY8(l%Vit`>v|7Lpvgy-mKB#2W7drU zsrcHwy^FhA0DxXh0tGrV@fV&pmCLw4E39u8j6I_(QJ7HAC(cNSa=nq1ENUVAI^=eQ z&AA>*)_ojCj6l8g``_PZ?p(x}0kHFKyPtdBxl^W0F(6r(!LYF@%2-(Rz=IELBk3U? zP!o2knpo+YfQV>g2-kk12y=>PNZBWB9!Fw?XXhPmaR!?^g3f;8UBt6P_ZocuwEl-C z1^JfGUmZ}+hT08^tw4U^)jJ_kg=+G|%GS149${I`-n&__M!ah62Kn8lmAa~(y*&k~ zDF&XfMG&fPW3Ap9Us0h0b&=8NQqDB6?F5-7*drsIOcQ3QKuvLRsgVzYgXSVQy&!M- zKR^3fCqrw_*IN4quzhdE{`R)maqL@z95c#D=;-Rc>Z&U>vZSa$W0=zhf8Cab^3oD@ z8@$%Ofj(W4aOQ*sk_!q$DIcOtRHA`bn4iIh|MrC!4xfMi0rU4aJ~09Qkv*?ZknJ_) zU&sH(JrK(lsU#^Oc93k(A0A90S_Tqp(+xzHEoNu)2n2R;bzSuIqeDxUHZ~9S4HdRe zm|QS#hw=N&9XC3;Ev3JCxN9)V8sD&aAg3a$zBBs8ubZD+y(zt7&S28;Q?L9oH-F>% z_n5Kow8_QkZ-ocb2{PxZ}sYNrC2{>qUb_Y5Ko(j__IIa&Y%K?@7DS^U~- zzdHA8BsfV;Uk3RVJPV@ZL%=oJ>bRc}bNDAD5FP0RmTqi$W5|#|2qZtDw1WRd3|VR_ zL0MW-N^Ly(7$iZB+)_L2u*1nGpG;K< z*&&zlA3#8;gg7)fx}w6v-nZa>Jk5r+Yu$)nTgA1LPC4bULqABDl$Dh$e4UY<&EciW zhO)LLWgMe_TBy`dtdYvgNm3K;cps8FPz&j4RaKR~ed`8{zzck}@mGX$jKZIL_BlV) zw=8=tbe9BPq1lF}R8`HJH*d$;JNQWqCh!WJT>ccrXy0?iaL|$110G-eFFr#6Vn;^+ zxB@8C(!oI|P%0drp&R)btrCG;fzJ{S-1r6k#YOz-j$Fpxet%Lw-<*HinwOR3jRtEF z=Q$kp4D<|jhp-YO#LM+7ips-T`3>T7qui`)na-l*C@okeh!b8sHThCfJ6cEPCRs8D zp^+K;h;_9t(o{jEsO5x;CnMDw9AZ)!{0AL$Afy;mvM6+8iwyRl5h8@qdpQmZQI!r*dkQ)ghnPzoFI}xol9Knh^Ko%$G!T_t`7AG zQDP52^bnLFvQy8 zKp~5Z?dfJDz(3xF$UJRuV2^o8hE&JiP#|MwhN}~&jShs1@D~lG1 zvwY%@Pq0ymEQ3K&Lpdg^RxD?_jvYPPne2hBZLMItnC8KaE?(f*WR{ zO7@*U|G%TmG-lziAJM?PDy2*gDHa)t`bDxNheLf)UI$?uWo+$;UVN?j`9;ibD6x-g8Z(Qyn@lmuhvArdoo&4-Q3wb+|#gXAafwSwB)Jh zRwNCi7i5i}H=czsDr)OZ?dcC)jzy4W6ZAQg&AYh zQZhn<3%b8OJ?ayr5&&R;gNAf)76XSgT`ZRWrd{+8zxaCw5$dVC4lN0Z#*}GDp&)d)eJr(3Y=LzNl$m_wkssY2 z9P|`^VzkOSWOh?{L~W3y$;s#zSp_dV^NdI)3SL~}%{SdxSv3iBId7kN6CBLXhdIdQ z-i{6+PLl*P`N!EUe%zC;Shno>XP?nU%a$%Z@uZVPlBs^@e7~AHea6R4jUze zc_L*YX(;K~
V?%uA$Ax=Z5<%U(lGRw+wYz~I~MA-<-G*?&Gi1PIQ{^F+bpEP%@ zydj*JuZn*jf9qNFB+sdYAdl?Dg#?14CBq7 zUR_&V?Qd@(Cp*-rGXNatpoZ;S0`4@NsB`v9c$S1uf27;8BD&zu+S=N+YuAmH<&y~9 zq$4d}5|5URo9Y0Ucoi1V5H8oeSdT&vhL-mN26;@F8!m~9tQxw=p^S7HPU*%rcdbDs zV*ngB4NaLmS$yP(*VkHo&sqzEJi+yb zt`0u>p8CyoR_0KB#(VLCWJCc#x)^C52qF@RLjw5FJOL_UCn+Z(qd zKCFY@FC!-&(xgY+z1z?`bdG{mpVFpu&f>*O)Wq?A+uPfq9Eeb7di;#Er2MgC4f{>C zVP1oVQfs-qNe`XRz7lodA`Oj=s7*C=NXVz3d(K-s=HLH)%B0DHOvs;(Z-u$9!W~n?+vV zt+(EK^UXImw=|1Dg?hag6?%w~!jeEIJK2;ZalKa((v>GxR*92Q`9akJbBTd&=t)q6 zWnX!D;iZ>eidh%ia18_e4n@A=S6EJws$|bjy9ik3P3BWmHjdS>F2Q z*Lr)KN-8Qlx<X>*U3h6^|FqRs0I>L;K6!=2u_!iGW%{v)7Ola^FNNDTUW@=Kv9~_1@AXO=H z;GmY9GAl~~zYXiwS(L#=NfT5MZ6u|RK%(^#58Q!518Hg$ zk~RapnwlEpmXG*J))I5B;5u>a7kEh@{O6}X1D8}ynL?{IHZ;gZm6Jige(judEMrfC zbNF&w`_^R+(0gQ3C0DMx{E$ z?7N5*LgjJoAXI$nE}s-FP2h(YfA3--sm+yJw#6SJln=(QSB=48&ic-;svWw_fszsp_c@iPCGeC^6cp88Xymq046U#98 z>{C^)Q?9XW)&PG)0&Jqwf}LWxBwo;;?)S!?>vPDVrUa_Sfw1v6;2`h~Nyf3K_F+dl zd*h}?>=ZAp)4V$yq6hkS7e`nuVF@#1DJ|8gbHoW~aS5b?5!g92cT+>-WtaW<`zsxq7Mj;-cbBb(<c)a`SYKB=2`Ti!Z8wcue|cgZP$GB&LfXJiluV|GK<}{ zl)B0Ng)_eJi(mc9qf+jP=&=-|`lr`iv){h&GXmKL<~Xy&J2H=q1iy^|4Tz>~hB4MH z9y3){Ro0J=nJr9P3WkOwVOm`w;AyM_ciPtD-8?2rHqY zDa=kUtE?=PYWnYA|N5qzZd$r{$>_0Txf(XCT~|7)j1NRM%=n5>hSNFj;N|Igwp&+E zDDm^bLk@Z6_~Wnn*-sI|OfsFX6Hxw!!7pyU*%OHo_fUEu3UeY#1R#c;e{KJu9)Rae zQ@T=odarVW33f2RK{`Mkn6+zCUwXdAVqiIiql-)Xi-%i2d)T2v5qFK;;gW%T2oOuzi#QWXP&Aq8ZIA|K0duPDo7}< z^rVj7j-;&o!g;$WbW^$U-WO2N1w~_?c=V;7_KY#bW6CGTSk?6znxeB+5kY~6Sk+T#tESqB@NVXWjbeJFZ|pRAg*S80uDqB zTuVnA@mgQMb!^2rnu+>^!005w+N}FOg^pomGPAIQ%xGGWu0$E`w9`%=c0}uxC(@>E zw5Cl90V5c~cH13ySo?uduC?NLxP}F+AwK}An{K}Of&1?VLBs%!9XC!%YHdatlJb)# zPqwO_ni^AQ%plF-KWQC;$hnVycjujM;n5h=RJ37V-_#^7o9;w=Z1ebNgQmt|YIb5% ze#FLadxODwZT1We(w9q@FB8>HgG#o@$;{S2YLSc^Gj?m!R=VoDmt3;zF1v2Sj`2?X z^Cq32k%@Qt?nU3d>#n;ttXr>4?xmMp^8OEeK&WN7e!~5yu7vf|cyBD` zkbK7a@}ox{X~kjgW0%FuSv{IE{>YI>Uh$JFGJub`HQU-T(vI%cV{s=NX0dCS1KZ2O z8m6WS;p#vC>tBZ*&p-csM__Jo2p)BFT^)tp+|)c}`qY48Q&Xjd9&yBn5q>NI3+l)Y z;V>;OEm^y|rl7d+#TTAu`GlG>%3hY1u323za{{E(IaFyN;SxZkrn<(1l#dy`dCL}> zTk?2scc@GWDg{mdMh~zVbRb{kiWMur_ucQRKD})5V&6`lGR0TQr2Aw|xs-#30lE&vcn8X5MHk9_3AM|@bDF`(0?P7`Qhp!$_ptYQ@DqzB}{ITELE=|g!=@VBfwri&M9 zle&d(+j-7jA3EZQ-FDlJ4XjcDki^Gbjq2vd2({=_B-TJBt6Sbd`t|E zNu!@!et9V3Jvg{&!v>{HSFNsArQbOBn#?9e5sY-AIF2ESN4*E}QjBCaIg>zZw~cB$ z?yh{2g9{cs@WUVeux8EL%Bo7STE?m%9zt6InkiEzf90#^Tz=V)1wR0M1_lI7z$#3` z*!J76o%6NlpMOs5I3AQbz#z7E@^6UwO137A;lW)F!LuIymKO+YPT22Z7(249VgCHr z_BK7RJr$4;7;p2tKYe8+M8kzqUlt}#_2~i2P)A+up$1Jlqjvq4==Te{mM>qPk)iZN z)HTpBeP+d%k2tijF-p$MfEOF0!R9E%nOmah(u+3LZD0eGrR1d-=cn$yYw59vPZgf{ zyNUh3xut%>lzo%in;v_7X+urZDM#--cUEHMlY+D?0a?-J{wQb0}RW%FL$Fo=FU}YPi0pDf^tw~6c)v6BKP+b z)^SUM$NP;q^|Y z{)X!%&`@k33*e&hoaX0#?Q1vx;ul~ISb*jM2bU~cI(z1Xco=wVa4if0&HVYV|9*eP4&vB541~oy2)M`o@Q3xm#k}?t#~%Bk!w=IS z1zh48k*qFv!i>>=k140#YlN1Bk@?d;{Fjjze5`N+de{u<9|%!;iY=g*3}4ze;0a(0 z>25xRE&SJuXl`mOEiWs`&1c)Y7B0=Lk(x1{_7()#S=+;~TwPORKr$g}YilvVCXTrV zWFU}u*i|nVRqD5DAz2o3;lhO<_`nAM@!<57SDrp>1{v5oA8!POyp7itRitemCzw)3 zGTc~6aL~|M>x(FOGM1njC9`oR%a8aN7j&(|U z=tX6u%zB2RM;6}}EY-A7q`I}U9Vi*`rY5QWg|76xuA4S(ntBEVom$z>Cs<00iVG@3 zUi^mI+J?=W$scT-r#517gR;UkFOgiFSaT>Ph3HVI$by{uihyDO>s4mevBw?DK4dyG z*xV~UBOC3J1x9|!T#HRd)tg@Gcf2Ze8ZjzEi8NL(9p)t$+`gJFSV>EG!r@5bin{(G%^o;1*%(Y(2O z{KSk895!!{-Np`f4<~0P^=<9xEy+wy4{-3YmC-Y=47Chrb*Bw=wX7LimimGDd+xVa zMsBn?J*q!)w?o)p3!iSSuWuWX z!`pia5mHF;>=p0gl?2$K2A)Vk08YVRQiK{4p$~~rRMo2$+>guU%hhO2Xfgq&(jB+y z4nIIR#E7VFKUq-_?reRQ;Lj-hHRppy-*EMD6%-^v64D*jrCxe`_sJs5>sc^is=wscC@l(^Z3Y3Kfg&W54FJ|hMINj9IRfo8t3r9f(L&5qaU69<+EeZ={P;Ky^H)mj~g4H z21ubFoD8fe9u%mBD0uCm8IRyNkLUmMQh%cppeRZ!WC~{~hLBhvHj3`K=WZN{I@J$7 z^!r7xyrNLwF~=MYldW9261~kWZm>cF#v!#mWy)0QP0bR%GmS(PDNM-K7!opAItf?? zSa@D>2mu|6Tjj5BZZ9s#FH>WszrUerYwOk)AhN8u5F+HOGt10AuSPgsOpyW#aDbV1 z?>+Zo#dNG9mMQrKR#98BY}vmZeY8k5wPVmoEgkK}Lc7I54)psKf=zJ{e9z8vc7Eci zCk@Wmdy;2pV2}r&!}t!GB$ly}o#Um5UD26veN0Abz;VYN_lZw@ zf{1Vh?Tmvc;+v9Dd+)iot8KbxvsZ&<42U+-Lg6T!_--v7JC2g%KOtXme&?V6^%cvP zD*#zm6v|@5sqm-*if4S`3)5%J2-0@@8}UL+8kfPlk@JiRv>SCcYyeH`_=#TU&nsp6Y|VV4c(~c*+@!LsZn+Lk1uoa zv%dUgE(o@ITmsYE*LCg&NZY3%Vw}3YyZr0;|F8!{d6*Cw9slX)J^HM*fEdBdU=TRXEdil@w( zo|B*4krs8NM`BU7mPN&TPS4FA&Rx*l&?8B)rLnqYOKPMr)0`b=MR}M)DQZz>TGC*; zB?+QFi_5>Sov2E`3pM8c;oOiOvDr7l1I>eZFzUGOI zgI3hWlU9PMMP0SDb%;RZ5bW(6C@n4wW%JT;*ot=)L56;6qD-1NL1gOMbsKO=aCzqV z5PfB#V@t?5ho{n&WoaOp=vP!$+)k9?jW^t2sDrR*Z0kVEbVko0gCBY15#eq)JQK|@ z@N=)gti;2bb#V{V-;$_kbc2MhpvdES>NSPGN7~HpIG;D@a|$LA*uX9t>D+KEj^~4x zY+4d~lVKV?u)Uv*c-%}+kbof;8N4G*h!HWnqEOml89MV5}IM|$FV`v0- za}Y$^xvvG{q@BAgpN2<^k!#!qGPj_*4<2^dz}!K->|~zv^@SUl zcgpC@-FvPRV0SMIbFXxw&Whth9LL17ClfL-+Prb&&b#gY$}0;=vy)Cd@yKYepWIO`dZ6@yExZhq09v|L`w988#gqER#^g%2(oI z1qQg6eK8NGoN|i(^h7Y3Fh`v*dD;}t;_kMNIlJw0)W<#s4yrIDaPTiK`{qBlHa8u7 z(7_DOv(7rJytGu=OI7ZDiZ+o=z@Kx~Pk!QIJ$1z>8AyzxkOAf`83sTqbs>%B%@_RF z{yXo1#LRvNr}M)O2Ud{vDY-Z;vzTJ$3trJ2Em~B&{IxCV*(v>9b>-!`bN8G#Z_ivr zdRo+$k)KEKWR*l6NzvmkL@Vl}?yc<^L*2&o)l_J7|_hwM{WDz`IgDoazUx}ZC1 zE{e(z*f%Q5o3UgW|4)Y$<|PdseGfiaIChuG1?kzL+#oZuK070OQbwkdG|ByC&pxwd zRZC}bTC{m<p1%6kbB zn!d4eA#njhB*d<}?t1$jcWmoIbDK=`5x+2z>4=cH0vS!VV8MbX{`g11%cKKbm7b{a z>jZk#PlOEavFmR9HOrPR-Lz$k1&$R|3kI&x?95D6Gx%hs<88ZFykl_u=GNcbaNYH5 z*R3VgxP;V=JM_>Go_@v|ab|!er2e{fGXYrIt9nfhuLte&_4ChX!7ID{)vtc_qep%e z*FqtMa&UsQf)yT^l3*4m*i&_C_S}7spZ@G;*7}t8J9FkNUOWX~L|)d`ZlFJ4T%P3c z2tk2`3ev!`zk-$>SrNyNJHjV5=cw2tEBmPdSuuKaqng9`o}odgZe2}{v{1R;Ke_VC zZ6x0aSBWAx?}{t0Jm%m2&AQA8t@?)giQ~qtUb6iZlp;4aJQsIi zg8s~uF=K*odO8RmwKO#smz5xH1jF(92Ao$^TqfeGvxC9Y+0`SVS^jxP3&)?obe($6%{&&H_0p2C1O2BR82U<>hNFJ`?FuoENDI8|aB=|2-iHULq zS{qD`H`?$QH{y=aKkQb&as`sd3FH_n$UCyHI_b}EyeZ^H3=gvmxs^-H$~fpHe3-#x zUI5>OU&tuxG}l;Y-cd{t#tAE4I+23*lfrK~oNKPTPDrsU%hhFe>c;yYSa9<%ZYF0q z*<(NZ>eba?mUD!iuuj#o33IcqPSC)QF2CH+@|8$OU%idkh_Rp`2bP1o#3j~mz4bQf zz{==&D=so^>eSa>eT^GLH+Hpk%-(6o^Ugbeuf6tCfJi6=*3da4ZYYgow{jEcJ zrFA(`ZgTW$b@brF3+tP^sOsFomdt_8d+as!(DxNiwf1p;OLuEqa_KnpqO)mDRMb9a z^yHG!QEz5c-fw zmT=tCx}`g1crbIfef9(&d6(I%`EX&%;80qk{_NX~^Z&KJ`iEZVJ%b2oaSXq{A}dLa z_)pGI;En(t(PtD#rhA1iy!gUBci;Ws0}rlPxiaK|CMp+_zD#%e4@Jh11qaXm>Q{H! zWfvvNZo2L|tANQr5<94t1Um5UyYIG$Nc^T=Te29svj5V;mw$KHU61|YF$x>EVJQXI zR88aFd+!yfB*6cOBMt`#<%Cii2%C{!wAPs&KHvN=JY{}CKC9}Lg)g7`wR8XN=zsg} zcP>)mWbxwHM2Y#jrly(#C@C!={jIM>3rU&=BlN?fSHtR6@{#AiZ~kZg^D{~q3W1t6 zbEaQWrJkJ}Mo;)Q8vG>iYq#~=UEkAL)~Grt6A zh9W7#t={HZBYhQoQVB@7?J?%y1@)9d(fA4$W`R2DSSif%Fv}u#x+U^Ap zJaoY~zez(&gjZ5oUP(bw!Li33|Fv_^E65KhVcP~J?6lJkCeHYJw$2|Q?^2ZK=+{z*qOu^a;mL33?(5G>x_eGNib z)7@b^`z{-IvNs>ooK;nog7Cy$2x<7_CqFrJ)+`sl{`%{0xb8aE9pEbq#Lqdl#V*G& zCn5qh2b?lVeZ`tGed1T}>Cb$|ns5C0;yRdZJMO&mYp*UETQTm-Uphc?OWRfwPp~vn)PFgqO2hV<{)}8NmOGX3q9QHB07FZ znaMmHj*CV3v&T-H`WHw3$^U%MAj05#3XZ4XU^08{a6qWx&ja-rT}+aq9)wXtDw060 ziS*hB@n7t@*Iur6-0{ah^3X$Dni^e&{YrI( z0uv=Ar<{E9^*7wW9|{g~(uvM~$oBi+D_BG4**N7InwlhVtX#R0ZOwca)i-b6TzVas zw;gsp9zbC*z3HC^O0s~P6`B6)>Z`>SUGvkQJ@(k+aMj$od(*_e_Ut@RuGx{u9^f2Y zzh-T5afycg_P4j+c*9Ni+uT-@dRZ=Pgnrn&TRHk_Pv}u#4Og`&NXW?t#9>!rWP=G)*?QXTbstFTT zR##gbthKeJFFoY!TypWnFt$v?($c_zz1uC;WWja_ywP9)G}YA*A%#s72UL*o) ze*6Rk%)7n*|Kj;E$rn>`_uqG)5_ATvnCATaJTHsa*y}#;yz}h9jtYx3);CBThVnrV zaa}5A#*HzyZ@&5GRaI4rlaCrbYU>gvaiDF=z9%7Q6Si||30bq>e0!Nzof zS}=T!sQd1_8?$o2`}e>2l1q_B@k@-8PiX)v=uDkFSsS>FNAqZJ+&G*!Z2DAIRcU~U z)6w1`eZfcw#AE_UK_u~njWG&DWm*GFE`rCB-2&DuTD0i7XP*_`Fk+W_O#m4oU~Xoo z2B*Q+<`%`V8yg!(Jl{j<7IqXQ1jxl}>S!U?>WJrW?;w$ZOM`uZm@t=RsBpsf_ToX+ zzczY;WQzYs9a?QX4Bb`$R~Ru<4M5Qw8Zaec(Y^I9h~d889y63A@mr)_JW2&?LS4=m z|M$or{xGSs3N}T_v;7>$8RI(*VCDf?sL%C`D7`i7)`DNLd&F-;dlUbNwA2iqXF|NTGy-K7}4%riU2=UcBFAJGZA!r<||1jsG7{AQP3b_RmE7i7_SQ;p4A z&5`obG9Z4zz4uDcvA*bGhaF}%iOmSFQqTOA=J56!_5Y*e#@m~o)+kGP)1QCZC3d~S z9(80gQW}S&HEW_57xH7sQ%X+FF3m5_nz=*P&U=p@UzFM}T(G}4Gc_j~%4uPiM9~vZ zwrs9m8>OnHEWXV zo1z}cA(^>RX6e|DDL-1#B#4+#3yLL|Mtv%o$PB;7(m>sq5;tc6JT6 zw)Z^p$f_NuRA%gumQ(BpG;UtXaE`ckQMW-;8gwIVBUG07E0yXld+#8^#H1yT zgT0u7Z{Qs1&PmISg;Ct?#wL&#q;^lwwbx!N%8Kd91{Fa+X7p$hL21@t<|dp3q&F!x zwqmU0O~{EM{_5gar%s)=`|i8%vFD!mF1TMPp?EtocZWH1YB$tUIp=)!t2f?oqX@rq z&pGFbC!QepZKADRibsZ?t|0D*hRJlg4XTK>4v{@lh3Z&J$1gb+*MIi;+;h+0fB%9_ z8?D2bJZkiqRV!8=^^qeF`QQh?d(lN!*qbq9CKW}Cbhfrxt;AI(R81me5sFqf30VbQ zT|d6;@|RwG@mm*Opn^kbaTz{|c7!b`X-Nqv9KVV~px&_a5*0Y#<*?KL%a$&?`***i zp{piLTvff=1$45CWolIb7P4C$0SDg~<)?fXZ;*(E2`Q~4}^Zj^)#Hg2oVK!{sv}xJOmBo4aqCjJ9YEWh5=2~8FOj-Ge zDrkF+f9K73`!mTJ3pU7fh8-Y6n)H#!{;+Y&W(gZNc{h}xiNvql+}ANXhzLKqB8M)3-q`M&om5V$7e8Z6XmRlG)c}w9fRulSS>nU zXU{o0S(Bhr`!S%mV;cHA=d0(a0w{AXF0m;(&^jMx@v~1qjXdUCxbcP?aQq%l86)*s zCrzDNQ@y$?JtOv|Xp+*5;Gmfsnvp&N6bKKSY;~}K%$#iPeCCB0F8J2Bzp!1SC=j=DKBeiY5ew?XTYP%4OkF9PlkH}&T^zD{_8JW{Gxq^o!;&qVF@CY z?PnAgNt2T5^q~)ZNM-DH=~n6KMS1y5-a4M{)Kn>V>J+NNqY|yP!kHrRsENH#;o5(` zY5(@#*PoyG`d_cNX~ggTFCHAkdMBg*fid|#3WNpNCnCrE_i0~1Ksb_kRSwoiSxOab zPKsXJ(!Y4)>iW*6r0k@k^4#?D^m&KPBo{<0XGNnk(iFNKN=ixd=k34iSh1)^VTDj3 zGqrcr%yAPA+1a>Q9c?LyMx`bVHp%H4Qf(^SX47zSZ_<9P(S$XtqD8+S+On=Cr!*(C zU`ROsL7&`rJkBB-Zj2hUlT$}qqGZ@Q%8`KOdLgz^G;Vh0q`X~gZ(W(xGBCby?2^Zp z4E7gfPb%*%j`EYEyrihSH#sdUoauo*T|71ndJK((DdW!tlZQzy4=R$!_gb7T&N<$D zue|^F3?kH_BDpz75dLxJn$J{bSpO}*z!6SBQ~^ft`RAS!o>Shi%jm^}K`G8w`t8P82y z+eVd@Y-wn^<=3~s4Gk=#!6AkIiwlaLeeU@~4>|O#v%bU|NOqIkAhjh@fL$6_O=dmg z=@1f%W%z{z86o#^Y>jpX21)(xgx1kSWgtC^9Y#O>$HMkVFTM2Am(Mz@y{pUB63Y;e zae8xei|UC`i1oo-7Hah3=phhc8z6}Lfr%e)vG>P8LGf}JzkliXMa#)1;n_zzh5~Fw zMb)cTU2x$A-~7fmy(1x7K)fg`KZ)IHK%V{W<*52<+uhx3!GjMd(=#M!KeeYYHxJg? z+TK)ERpq^(_{mRst`V#~6Fumlz*r=xRx{nCH=07=SomV3ZTsJhaJDPvAOc-BB^m93 zv$^YccNG>C=%q0g70vC|qs#8@X#dp7Cu+d@4eQapjV-O&fjb;jx5L8Q`|L9>&a1U@ z$`zMi*51`A@XJ$5hPe0c-+49Ep)rpj!CPb3fE!0#&0}q@+l21@{As5)w*+eez9}sz z63w}6$+B;M^INywcB`lW9FG85?h|8lYjf)-k3H_ES6!`&7(=MFx#gtKeA?=KUI!o| z?E+u!I*_-QZr~wz1CSbeea&3p&#RXTU@}UQC@m@G%GFe>@#Z&jG`~(`zqOm-OPk}Y zHJ=hP-g?fPk1&M%8OS6>b`p!!A*ACI{z6#Cgp&^RMq@G_Aj5CLTi0KI!x^WZO8bl( zTVWt^;5LB8IXMOZL>s5JNR;;MW5$ebsBgfXC=_ZH!8?BYTQ=dB&N@pCZ8wbviW|9X zm_%vm+lCHj0@z1DDX|ncPfMIoQe3K?g5Mmp1K!2W_N@-YRzlVaSEljN3n-L&>@6m& zFO8Q$6+-&M+Z>AJq@6fun83W@rkg(Up(C^;5#N~%UYl+P3Hp5DE3aVhLM7DBkg*OF zvb5!>?KgkEwO)9owLyb@rAyr|AX7=cw%{}mn`h*lxAdY675mq!1asYW%a$*PXsxK& z)Y8)1-j4e(<&SP{9UkZ(J$B5eKYaL^U;KRH*L~T!%GC8TTZV=Q)&F93+k6=FAr&tC zH$1(-lEnGp!=I>Q^(7wp{X=u-&JB5fTdc~|C@DKHb2zZbL&LJ}9)J9g%a^bC*vCG0 z;e{9YiSjqG#m=H8z+?U5I6k$ltJB~?0ib5%yR5FMj-$!rS>|BepC+S}KB|F-+Vhi; zw1X;%+nmv4X^cB*ifsZ`&W#|;`uYYjxPHPwDk>`E|1q6S&-jji=T8s*&bQ%cu)Via z;Wd+`vp=e9jI#2g^xSA;eUy?NRw4;8!kK}n>*$yZ8I9 zzKPkRGNM6HATvq{uuCQBfv7PrT0cDDnfs%qk4Bv>$)i$=6{em%vvAJ-a}DCP%G?O& zh{|%(qsIOyZ!ma`yPBeEbYWh=g((xF`G-wgc}LW?B`VI?dqaKw6}LPv|6}_eP?cMe z8_s`uO_Wv{iMWDvW3cPi*#lAjV3gY2H*-w-ki7ra`rJ`P(LkR?7ly5qoEE0-duV$v zA;NgBCgedvAPf!l^!IUo{WsSz=h|zFi8iT@1P7reu$nltPdBx06^+(hSXfNZo99HA zNcj(cpFf;iZwND1+B!zs|X{C zdDNKEM4@^s_b*s5Yt}5a72*q;|IP05=MvX%nBF|cHDJiYZty5=A z+p=Zzv7h+(&b#coadVw!FzZ*Zu4Zl9Z)|QRB`Ye%x*aQjYfGCdaN;0U8oTSxyDq!p z3W+$%K63zZ^FeId2FW2aFcUF3A>J`FFBIlbItTa&9iE8Y+}36yudp!S?_l497hRnf zUT}fJ94gL;a?}4*wKnK9Pc1~9@^G@A6z)i8Kn92kx&xxMAvpru5tK-{gRe+r{;%1y zXYaG`J`NL+Y5f+3zUW5!#5{&zcq86p`W45-DZj)yDl*U(Te zKocVc9PhEmo@gYGz=fBEok~usZ)mU#q8^?!c@p@dwY9bDfBB1>CwRctR#%gx3|Z>5 z0}ncI(xj>t%a=p3nHj-b4ut@@ss(}gIId%lJN5@Z`eCRD7%FWJcC*eiGvI!DRIPPz z(6FJmAU@2apD1+l?F%my{HMIp;fEh?ip6~u=Z8|-&}~dbkJyLkYRU4(0~|xeI|qBc zy1H8J^bya0w+FFzo*Iv3QpDDXdu-fP8xNSct+cx?0_+SYKs+YrV*3Gjf*5Z3@BhB? zik}#V&`faJ=H&xJI*@yp;s>O3l%!*c|0Ita`FmI-tchSuG%Oy#F@SR~OS5|g$|!_+kuY`Mje>r=zKKzwSzwad%t24F+b+9){*2Sf zhgTOa#OPZW*lZvZJjVEnikh`+!6G&A#Syl)CDv_{k9_jU@dVL6_?{V-FegH$k=q+f zX1X1;v`1qE`llZF&wlnZS$%{O=ZPT!Wpd;2R*T}*nEt^5cTuJCgp*E4JcEaJ`E8$e zw%ZMaRgfS4?>FkjL7Rz0F%dnM>|^Loo;=yS4p=R*Ahq!cfO4$AFku2vc-?i^$8gZl zizOReh)pnh{aKt|Xfjq+R=IdwrZI+tagv-7lAwzFJ)=+)=Xg4}*$(l@#<8P$78f8( z!NK-Kt)GH|Kzstq1Yn$V6L+9Lb%n3oI*u5z|4+W5*s?N<5Mo>VqJ@h$KeupIN3T)> zg>?;WgUM-OEidKLr9y=y<1Qh3J=xPXw&!;>wG1}*r}Y;m4-K|9MoSwTHZB(0Ka^9D zU%zQXo^=Sfw#=xUys5f*-PVDm{D~#0<%MabE%h6oyleB5$?X%Sj%sdLnUU6;Iy8_v zn4H?5oi>n{+E)+_XLV#Z^yGEdFVE}QG%-8f5;fgjEj6oN>1=4O=o;!x$!$){XhKEctX1&B zYA5fl?Oh|4fB55>Hs<-FMT@M1`S7EUi0>w1#PEsH35?)yB2geCL(U_GR#s9%>&qYx zQhR8yx1*na>gns}hT|Hx7yyX|+X?tHZ{ZknEyjqaopze+%WJN_nhVBtgbaDTvwW)4fc2TbcspS0I;>_m4%6QJO?&zSUr8_j8!!?dD&UBXUvpgpop;= zA|k+r(WIy`lA5B7V)>Y{>W$5uIa4t+G7G_m4$vVGK`4#;#)j3K6~n^Rtum(g`m-Y zvC}FhD&iXw^NGucBwrAO(=b+q^XATL+PW2a`A21hlaneEZHA4!phb&cyZ+j1_uqei zJW8xbU5f4v;RKd23GBhhn@;luI0zJaWk34C5Bj^hl8TBUZdk-u9tsX-BoO7S0uM6s zh5q(OH*KoLOn4G21 zQ=d9P4%Vx$zFJzM?o6O!T3bUR$k}Iq`N=1r^7P`{eCkhOwswv^_SoyLyOzcUcMwQ% z_67|twb{2AZuGqC*dK9|r0*nOb@%mRcSL}ht>BbUl>gudV*V)~ZozqJosORqjh^rzKQXT%3l&cd2?Dsh zq*O>lL;aSjNs~`J>7?U7`N@hg%5fW10p7jo)1UE?_$Gh$nRtUqFkEbkBqVeumkzV% z8{hcG<(FS>`98KQdrvJ0P3`PT85$U1y*%{r!zwZ=lXQdV@bYqkI>u^XtHs0(e77+* zyey@@V9uCnc2rXskfD8MO?rl$6#H@D5XXXRnyu1S!a+xe)sPCy34(y}^&NLYyzd0f zj_E+>a5wpu{cQiAev5k|+0+GNw{}GXNNYe#=Zeh~vFfTE#J5e@hU zQ58d0M`|6LGmw^)rcKqzy1 zd$00`4O`Z9Mj1r|$!)`NNy7f5^p3jp&LRZ}Gdh}UyVeXeH?(#r4p1?eSw4_4F}J{) z%Sr5&yuRdKD0OX5e{Eh*UQv{t-nOxIAZd6oscSGXPy-?yY|9;JX;`!Q{rk+NoD>#zm$R3^kg)g4@#V?#* zyKcj{iQ`wUt}ZVvChR_O>~V50So{LUS+Rlg$dFGR;<3~XS-NB?ed`g(Wf+&NqQb~8 z9(E_7MsOjMCQJ-U&JW*MRE=q?AO;cEA|@bH(tnVap2wq5eqN`~)HfoKC{k<8I3e_NW~MqmLL`N( zVHA8ttjlV2!ji>He)xkQ{O|`q)a-5RE6_~dT^uhpV&}SoFgr{MhU1u{kG7DS3L3;w zs6mimaG4n@`2n(ij+iX|$00(rnK~@@FSx&Y^=e=_z?j4$U}~~x+tJ4y<0+IM_l^*% z^_%MeT&kOP!Jfu>8c(tlyF$PW^EYb7DiWex1a!^Fwx`%-+RwJUpVtjtCo8-lQ6Ch zBTu#d85<83k=l$Rco_!nn6q9)bB`~V&-@0|{o_YFdW*8CS{Tlxv zSTiH6g~QpZS^Mn2|0$pO%rVCt<2oi?Ft!tQptLp_*Z;<5M#l`R$$0O5_X*><@1A=@ zwsl&Xm{-F%k=2`?mzv72LZmY$Ed*s@64g?n_U_(}tas@xjm7vI_YePtv4r2qXg>1D zkLb^@eB~?nbu~EUQF(kvaw;lh9HI{Gg_TKp$$dd) z3=Q=*4|NY@MH1lnZu*D2>WWf3EeWUmRC>QffU?t*Qu;(t2$R%S2Q+tI4erdv4HGgl-$uZl#*3!)z=|`dJ;aQhd`#n_|nEE4+%~! z>>V6#ZEeWT+L)TLB{Q|73qP1YrmZ`(yFYCpCC3^=y~8;v$!RG`X-WMZgDF`BIa$fW zT>}>D>}_xB>*cfW>xq(DwxnfrWGD3{XJ_=4kLyVuJ)D)&(UUfilrcD*HIx+UK&20} zd*%HNWTy8gW#{UL&S8sXW^@jA<)vldtp7AO{>k6v8_j#qAVRE_HY48S*bCmPQ%?DG zeo^6l_ua?ly|>rZ{^Q`9rPf@AAw6MGuW zN1yQwx1&uy-_=%Ky?XYn>B@24eDjT$Uw-Ae=Y72}FK1OvHENLmmuF4xo@Uo>tgD{q@(2oyWQ2 zb%*qP6(|N((>?ew-W(gW zN{5h}*15Op8kP}>MU*=yfZCzqJ8!=O{y6BsgZQ9g?;g#>v4wOAf@GvE4jrQifBmap zJ@DXzxDXL+NI~h2MTKFZ4wUdoC!M4SxnBb@BVFU~aV#;4is+jguDd?g4PiiAP%a~L zOMU&6Nt1ZYm0Nr9g%=<;-76AH?3YefNl_hgO@w}!1TqHZ?7FKE%=Ged0~j1sB>a8* z?5lr5G?p?ciK2H@I~vcBc90?D=lpJ#y@6dqFx*gBfkRf7TSGbLoby!$m9QPiBWLn? z4TMOZ$oR*tV#Vyo5dffu{xl7=U1wTmZF`7t&Fa;DB}@%GO%UoCV25gXv3t%}|HCaF zd+|kJ96S0J8)|C%6Zo^tF+)Wc#RdEvVI4jAzyng#UU=?#)0{h%B*DBJ*uZPu<}E63 zf8w~~e)yvwIgJ6y1oEa#D35?iGBVXJIP1(aUDFouZWbV-;&+ItSi62b>rwibbln3F z{6HIKB0U{vX=D;F9f<`nLsXwoQg0~H;QwI{#6LIn;wj)Z5RwTiHN;m5LbQwAGqZ#d zGwqDGxR8z4=LG9d=H_IN9y4azv}u~;e4^N>(XqZ|02?p{l~Y+|p4XL~wr1^`J1xt8 z`|TEaN;p+o)iMMNcw>y!wpI(uQ`DU^XUxFlee9@@xkAF3U|a>_`JHciFSfelpXhCJ z$UZ{_;#Buu)b)Aio*Tj<%gPK(E|K_re(fN%dwX~ZjBN)!If?BJLX%fqafQ;{7Ecl& zL0*Y(J>=knzIEY+vu0RNJdC24AP42Iyz)xr5j+u^AY_T>$&HjBSQWB*T4StoH;ukfBYv;Jnmx~ z-o>kyTMeRO>dYuLySX{4SykWN(Um;3WS5!XeQr`_K}T=i^6Gj4E`p8whf#3ZJjR1b zvP!IdqQ-qdglR&`2ghc}H;sC`de-OXH%yw6lV4og*;~1}K{bZ_&c2-C^w!}}dAcA% zMWiOBWvXsewcGCFlLk9ln^PJZ$FArI%P40kMVQn#x;$@OWnN*~#FoxE%U7f|wMEvz zg$Rd|~mwd-!XfAUkGRF8xA4RZ#0glQkv01dXXjpTAN zLWYJ=WYs=x>a+(IELga3;ffV2sHn5Q@)bP63(r1FX>k^SV*E#vk@zlG4&$cHQmsr=QNW0!!)% zb&v6~x(I4qzI=u19x8WuP1LPkqCo`|&YCsL5ff9!KmZYLnW&R&M`!|qy?>xPJyCQp z@#UX=R8F3v>WW80;V=n+HPjS1sHuN;-e3EN!3mXWtTB8Br1!>@i8O8AR9g#q8{1-0oOZzl-%=qZ5Q;1|>>$T*VhoF2(WRAD zRUbP1Lrh0uEl)rBr1=0tz?6E@qsjIMx`adccff%M;tSum@4gL*Wx->>>iR5I$&6u| z1U2`+v%Q%uUzNbXk7Zb*JUg#y zV)23djhR*kK9!Ho#h5s)sJkyoWx{^dqdKC-*7k<3x^dI8_dPh9O5V~Dwcpj$xTQIx zcgxIOXHA{4XI^GXZCAAE+K$a47E>}y$4_eQZVxvIiyMW_C$BLssHZJICo9R~KbgZR z(`FSMc<}5AqB7YGR9&wP|T zah4ZQar&blJL;qpPZA*qc8JZVKKVYVKBz^#17JzQvwF>% ziV5SuK@y)4Pkam7&dJW$VfKvk&;LelUbYpS(5ebn^E2|pDn4?-`RAiHd57cntXj2- zaZj3C0K?UO@Pi*b|NQex;Le{vA0i1%Pr~U;--F}s`=cNIXyL+_v1>e$6lz{UKGF!a zU~xP=mRF#wrSUd9qooCUb3~a&i{6%&mW~}Y%Gxk4s6)bM0(KGK_J?rwuN5n{K*E0UCqZ=+#Y-2f}H_^l5+y zDuouuImw2=YWvzFSUrQy03ZM6aL0_j002M$NklI0%~uvCHVmG4?D3@J6eGN{va+_ec3j2SOE0~Y z4nuFe{_X;s8!jlWz<7XuG|8VZ62)&vh6TSO$Z3DXO+cP}XB@B*5fez*5(py>>>KeX z@9JPiB6Vz=rx?&Q&EptB*aZWMO)gdkXedd#NoL0W68DMMu(ckFTelG0t5EM5Y#i>yVw4R!ax7HC+B z6nq!H`uKl*eD~dUH7j@9Z8vGKah5Slm27dr|E*5YT1{dRF?~&Hf1;V$$&Wtzs2n5o zXJnCU%a$Z+xM2N^uBnqIuVHjnSM!*YElduzR~Nqg^5@Su!vJ7#vX5iZ(bZzp+ASQ3 zp}c|g*;79|d-m+lopu`ksc@aLQKJlF!N+RhPnkY_haGnK`ZvBIu9xtUjXr+dc-Lv~ zXvs_PWcL5ME%6>gU7qKje_^jZ_izz`4Tl}}L6L&zop+wa`ZCMPjCQQvq{)-{vVZmK zU(cVvpV%TaI`hc+q>=Rq1efRH7zztE)^6Y)k>Dqh{11;jN`@SI@F93Ux8R*nbxUZ`mwids9g+Y@7Ab)^KfTUY5Ew9^l+|O z-X3IHl%15GNm2x*m_OLxUE9^OWN@IXG@6qV75C*1_V&Hf)89VWU)MQY8D)mEEFr@C z+tD%5H{6+>TGZ9rH3(293+W3)B&IMLPEQ`p8ffgvEaawc8y;LQ{y#5wLP1n8n5J+| z)ZZ3$Hbq0diV7;pFUsHgKq(#lZTbB*4K*1P)61*!C-shwax zDpZvk(BvG5kQME%fd~YIJ%eT?vhO;Q23vDwN=aT8H;gmbWWj)vcz&i|));qMtl zxUE0{-weFP{v*%@Ero*8k31sdWA{Dx{LE)QGi&xNL4dKuM;+;7oF9>$laq_CpaQF^ zs<^;ZSyP?Zzo(yis;0VzTZaMzSx7+4pYS!Tm;k{*KEKJ#k|VG_cj{?akZ*nC8_zuX zWKn4;cLVjzq;n2chcz^rY2|BwcFi^a^;0z__Sj>O0}eOaGho>TfPj_=2?FA%U^(=AO7%1#*7}bc6AMwPItgwA%H#baLEFXmMmQ%PR_}3 zdL5^^1uA%E4?XO#>C>kZ@p38JySps<&gzA9v7uZlbosc7imeTeAjKd5_{YzGp71{d z?AA&i4RDY&zxn2yA9&z_fXOn#y0?P%yZU-zte8kZym3C6^jz@xOHvpmkWj#d-WN@e z`%o>`1rwHK;6O<51Qi1KhA=N{$`Q4R1Z#>mMLc5&VoOU+lom@HH+l@gEme>x_r?XS z@37;Jhkx)el(65D#cX#B(1_i-Wa-kgzx-u7zMvq~B3ZiRwaTi=YgeyUW*PgHotF0L zPoLsA@DDTLE=Rn`-{GK+rFWlr{Bf%lD>{vIVa6&p=+?W-G2VTT|NhM_&p-Pdl&t#= zMgDi3x1yM+I}9IJ-Q$lx4kg#sZ5%&g;#RzPl*H|O>gPTOX2n&emHeXo_?ppX9#&^U za{e207%^X!JvxgO;YLQ3JZuI3j15C*)vDD7096-<#&6nGH*VZmZNtqwLTd^$4FDhG zl31M#`)Fk1Arr?&_)x$lUS2tbiY6z^`&Lhmm11^}aQ3@D6c0tlu*o$6uVX@;+&Jve4+ih$%QL4@+PNzfv|` ze7k;ZNSjkc4NGa``t{|b#{h_Ht5+it2D-Yl##Rtb8#dI6htZgd#Og=ZoMb4_J}gg4 z0VGXCoTn)S!S14HN=u7%uLiTR-xG*1QKcCz*Pos=n4gplkq!2A4Isk%dIz|Hd!qEB zqF22~x%#r}zX-9=G%eoj>EU@rErCo83& z|0XrPFcgmI;aLuXa4ITR5ux_YhDumCG5+Vjp=`9_lp6IFXUPjmYmbK7ng@E@lKPv{ zqKcueB#p^Z%1EumdhZ0zgmO;1e}iX+aj`0!qu;!|qhC_CSrw5yt%&Pu;|RnyKZ*03|oxbkkq-E+NHiMej4?*s|PbyljLl zkvH5C2#oZs%xkW@mYz<@{wNDb>O(t#8~~F=o{9$DhD$v1Uz;LWB4iX|%W*>O=fiUynfeO&iuXg^~(HeX4BG3opD72q5ms zcbKENs2GMg?X#x>lq_QoBkGe3Mc5JaI@wu1-OJAx_V8*FT>(pu6%HnZmb7Nf z3a?-A{dMvEg$9$W#Hgja!XkO0Qt=Sf0>T2LRFMw!Xr3BPNESQaMG_AcSVB9tO&c~$ zpEawoX=Q&%jZNow20(@Nh%Xcz$)OJsLM;V(uaoo2XT1e~M3Ub%V z!R{XQVvMea&6@>dN!#QyCS1@Gx|f5`I0TVPii?e;=GIn-2Ukt^=w|~=B@ysfr_(y9 zZsDUbkpua$l)xPW!+I4lUdJg?OZ@R377HGSW@1)uxaL~*H62vZt*s3ZPw(t#|M|~v z5;&-Oi{nZi#d-cMjD(+!#A&+y*+`g;<( z(^p+R^$B(dHhbQm|65y6Uk^xq)KN!4nNUldjl@U!*dHHXx^(Fy4?TR(efOGL-0naa zW5CLO>Y{V`X&`1Ag~_+5l+qC&{_y+vKft)6w#kSFkaJ7))3$hJ*{eyyRjac^5)#0?Eq;B!mQB5+Hd5xFiI4z=dFp_f@uR zOV+-RW;FZs^uBlX`<*&cXOOjcY+-9im0CU3Ri{p!^PTN~z7~I#2SWV9y6PfExnrd;+z6A@cBj_0yahH4$5SA#8AP< zDzJ=(fTT*9^5^&e{M+CDHbxZ+#qty*5090}!8BNw!AKv(H3D-QQV~WdEd`!tIP-c? z4y>gfDPIj_<;M~V-I7lkhEjeM=q<$`3*pF@~{g&_PscD8h z;TGA(s5pB{Jsk@tS>+;b4g5@!@k|xf;n;m@8a< zpHWO4325VWo<$LJiIgwE4#Z?zTQr@CRXs#?KWusti#1IQi8N|_7!AXN#8N;?Af((F zvko+)wQ^B6%tAfLr4tomeSWl$MEecV;H{=zWm?)=(uR)ZxvQ{ZzR|1RA zk5WrsfBK;(Z@+pV6(0#jd)~NB?B9M|&t~3m?Y0(=FHuY>j~7M156)=3bJ3zb#s0l^9&KbbmA&aYDVC_Kf4#^2cj$3 zD#+0ojVx5id|_g-cEZD8h00bc%+X9m_B73ePy#(M1uzvd^3_Wb7PPmweGe~o-~Ha+ z7j_}{;c+liBAqIxZRJJaYFh+VJUDZRbg|uI9{lcizlSW0Y5LK}9wSd!TPQBD5;D3Norp$Vn*}avd%OCG73#rCvNjMW!bw(d?1W z*d~GaaDLNmZvw>dQ({5CRBvLP2BUBVHV+j3#P9z;dSDpkU{C^nqSdH4MWqD2eSL^x zv1XuAV6|dK%^n>YDEH+HerX2?m=za_K^52ubOshDoFe_x*Z<$Kkr5OJLq0!}M%oAk z*5gAbuHL+v-Nn&kxXV4p(n3fWWXaezQ9v9_{`dF49~t`J|NY-nVMKB)`7qnM+S?~$ zv7X-E+gSNRVZ1itD_AjdP-tT@MmG@?Kf<1X8d>kw^amVb1mCFMWyO1J9mLg~AMD6^)+S)oWX&)XLDd%$1bELE!mpoBSYd`mw)$ne;3%ww)stOdb8U2u|JUxPJ({4 zUmytD(wAL!*%MDZ0c|_GyBLXZOwlE!1Sk@q$IJ<%iEl7pS5G&Hki8|%2c+X7Gaezb zFxaQ#scjw?srVPZ@HZGbGw1D1JPmYeFL!nt10!M`&@guS#G%|XMR!Ots)iO%wX;IF(iA*QGWA`Ap_L$|c z%N@aUJ$^rce~6PAtqGJGWoQ$kX%;F|m=?35;o`}tmJ+daM}6y#XOC^WYCTI=PeiO+ z=$#&)jF>#@FICOs@K`3}ZdjqLQ!I9NXclGwMz2vV@^MwmNUWoNGa@6aFz=OxWzT-# z!c)`XY?@ut=kp#BN*Hps13@rb)xY_|7hszW9zRa#VPA=c?1q&9S?t`D?2OoG4Rt*att9X4!7l;!f z0xNWY?1Dj-?G-CmBAX^iDf9l{|HD53r{D>>Y-StKj?L)M(D3(v_`|>c>%SSBn4(D0 za1eKzCs|r}7ifl^C-XZC0EtoCOQbSvVS|1z z)*G_B%b3M=NXNp#Ag{LZZM7ZygPU*0YN3uWqlgPp%PSRe7=|xyxz3pqzbTR~U$zWZ z0a?S^ged~kjtPtD5`p+b4?Qd)EF%wyhAJLK;bG*$Wl*2QcjUlK3WJa}U*-UHIF$fE zgE2{@DU|vFG^7L_z=J`HCX2a0^QpZCt1tDFy0GxASh)h_cid|uqoE>L2eZ1P%?;n> zuRi~I5FrZ>dGZJgC*nL#Fd$_o@cGYu?ngiQ2_Z4KLUnQ?&q7FcB&sAfUn&o#p+SYqx&jgCFed?5xFwgEEXb zHR-U$LmA?%1!fq)M?@qKF60DpKYZ3z1FQc0PydXz;}Ph?1{Dq;xq-r^G&!FKjKWZZ zKA2#y+puBNRh#I>v<|8ufAYtF!riSxburu1F02&r^)cg9AxzkizZUd(h3_;7Hw?p$vCg)JJs*rW8lQ!4xdw{?UbewhH<~a)*~0ktyf}OJvF-j~s}hPz?KsB;xWL6)c726&E=YdbsMZ zY6T5%HF8}Lp{Q9c;|SefE@a(sy;DP8Gv^Bwy?WVYmGq)qrE{4Xm$%!n@UT_%Xu4ZS zqhCV8Gtcs#a1d#Awon?j@`t^K-=kG?MPOt@qF_|Ly$?S#xc;(DEk4oU5?8NoI(j$} z4G{@Nj}^@q4kgDYQl=}Y=es`NkEq>gJ?6B3b{1?tK_tR(4d%;R@PLaT){`p z{5s}8B0PtT9whht&gYSbKJ(O5zEGGL8Z->r)3(1oX=}+i2gU#_(8NNj(txm|R7Tka zbkg3|x_r3|o3VQJYMx^HWBO(~W@cwD14AG@{l>ri%j1tfE}eKyPNI|75UE2d3p=N? zy#pZF)7!%e%94*F7cK^fEk-_4*h>6Q*b(*tpYQ8m|N1C1NLvdSk>v+y#f%Rh3q3{2 z5A*q2npyjC1uPE|fUC#{3joj$B1LAFcrqP9w*ueGD_64Pq@3i>PXvNM9@YpzBtPMi z00G$n(`6Azed%xij;xr2kaSaD<}&7VUWe=u%0MDeFAJ4~q{L=zYUDwcEwE|p)~yQ{ z{tK%Sb0BLNLTi?GmJ`+_u$?VM2WUx_Au|AnjTI}@NJ>JhRowgH#|fq>7V`S7yjz6$OG&G^2Sw3Cp@sp$?eZP^RM6e zx3~WK?dTCQQv(Olgd_a}4?g(EpZ@gVi4&b|ZBT}`kh)x6FH%uy?U+tsC9q}lmXCbo zBh(b8AZ?-|&2SEUC5KD{%{yV!50R}kp+h2;4|1)SO>$_1s6W@(8C}A`UYH87QZ|1yc1wIIx-4X zr=}+9JngbC)71BNnSHpJw@3`ZRFMs({6iKx4!YsvB`=4zP|tYiBH^m%Por*EhHbM zfpFEPtH=;qp)>-K>0M_f+N*OB*N*@-#5J-=a>A60HqcPHS!MJ5>%aaMXv^O0;YS}C z9XVO{`FlFLHgDPT&Ue03QAS~Xo_gEuZyg>QLR|+I03^!0+&DDFKn=AD$jIncY(J(u z`1knP!Da=-50{q`kn)q*)~r}z7XzK5Q#7H+a^(^~;FEFSdaqb^72Ki>kIsMvqha?~ zsTScXxC~2&1t*i~70oCSgie?0^gfsH5afX%k?4Ig6}WLCm^Cz9zfBvu-=(wRm+-BE z$%iYW>r-LxBre6Oa2I{kM|MAf&jsu@dU~7d0tr#XY$a}1E771oN}z$9mC0ANP-780 zM)a6O7FVc-U~vQ&%3yM+qidBxy*S(yPGck&33ZPqJ=1BNO?1bEHa(hn_DK51^*-d_ zmoLL*c{voVCz|KsxY#$Ur;9$~#$uMH5vl~n;;C7V8d6tu6AgA#dAOd`sG?QMxXR$Y z?h0CIl1}sKn0E!919AKN```R)zlQHnOe^5NfF!W+0u3*)c~u105`2 zi&rnfkZ{YEElZXz!O01-O&DX!px6_Uvj(dID2iDa>D%vp?Dr7EVJY^*AN=6?=bjrK z8$~xU>Z(AX5?vz&c2U zV6nmS!;7DUfUKaUQa%r7lQn=16wEE}vZYJFQX?ZLxywEsILJlt8Ol#cSK$pX=$KP@ z906ZrQ^Ul>ILHGXsjlvBn3u%mjD*9hRuAmnwF?*uN^7V`Y^VVRz=GgN$_ajgZN^-Q z7#s-5ixeD0;RcF2%Z#1kpL1G_o*Z@8nn~atX;cc;ll0PxAS&Z z^cmX-nl`0yL>ODLc=76i)s$1IPwm>h>;L)EmsrBlupAy8MfH&K5a>_h-CRyCoQ;hb z=zR1eA7$cZ`KQXtbct~d4yIH#uam0(5)X+~?AOrZ07Rs*>?Pn20b$Stpml7xP&t15 z7$AmFU~q|`48AZ)iQ*b;Zpco_2_8);p)>p#@H*PAY=oYC{BbCT<6kafPIUO_QC4x# zPh&#^WTF4(2xZ3p#ODJ^c)dO#J90)aqHI2!Q-}dO+xmCB;~kGa@z%@L)lnQk{yET1SOMY~ED!owe|cbhsz$XqEwB8%V0E@9hOEL7=7~oWoNr`jxf@$If{rw2X-*(3x7>#{p>l3#ng+(=|^(bBAQ<$%KVA%mA-`wDJPwfYLaGbVw^p#9L5AP(a|T7v0{$|Guv$jK#t$i7+B%(x{ji_z85{ZUJwr2Opur5eA@ynu;;0WuE!a=JIK z=;fQ8ZD-XjDRtT zl)b=`A)C5h0#CDNg2^DtOq&bz+Ws7>Js2Pfh&^nqo=UO9q~IJOM14-H1ipW*Emu1viy& zTeg}kC7(yfyhntPTg{ti5FHG`tn~J`zy15)`yL`8mL*iC*fA@UG3GlQbF{R!Alt>Q z9pDTe8!Lg0#-!h7TQOY5VzM#Tsk|B1jC`pXhmCo)3NA!Lh+Q>S!1w;_XP7OYICh+Q zoVoYtvEyJA79PCO@-PAjUV&6jjeqPm08&KAVBz`cPwzz|M>3_PAcjd;N0h`;gs>G< z$4Uiz1D+5}yJgFk_x1JRK?V@;t#5tv*pZ{?FC+aS8G3{P9LWZQm^blH331s6x3;$s z+@!vtZtdE22)EhVs+#=8U;gDAZ@HzINP2DcL6(R_Hp5$x1&@tM^>7SV6*~uvh0Zmg z6VG#)1G83oUQkF4= zM`^G6`a0GdtXvS63%~oil_h~dCu zhYuf?(nF6&RblCh6@YjgeBXTIO>eyQjg$vmfMNp+)qCIfJ~lT0`t5Ij@IxO`VPk&d z9q&MDjNc`;DBxYt96JH(4p~^Mkz^ruq{(1VDavEPfV~mTfh!O|^e6t{4+z+Rf;?}* zOdtG&fB5rI00cldQx(Nd>J^xJ88MQ5&>QF#3h{$KSU8og>UnZ|g|~n$j3B^n&D|aR zR6c0j{?R@6U||3bqhDdbgv1taotVMm3kKy7NUf|xhK7ffF)Tbk3d6+0w*nJ%*K-w* zXW2~B{}opWj0NVVePEK&!_oJ^gXxVo-S{6r`47yT34F|^1wKn1U;gr!(ZB(Q zqM$;(ux$cQlOg9gSec2m13e%>Pl>%sNnE;cp>N!Y4xW#qJ3ji+L=llddx?GZL zaETwb{@_5Ra$X(hADQ!`DrcoRt`_K`Xw0w}AQGbe z;07fKaM{$@zi=Ur9xlJ)3g$t$_e}k@iJJ^~e;Se){RXyZ1csv!6Zr>;fBI*~k00;r>s`BcjoOCtASX;f z9mTTmt8*wWIwlZ@rxr3L4hv zX1_RGr5Py?h(0u=M90*noAZF}A8wrsiz zFv@P8hf(h7?dk?C;W`lrNCNVt7~zl`P>6`|```QiC;s3Q|NOOoV&#TS!ircdS5~cA zLj+Z9B&lo~J{Dm&yi#7n?v^*c6o7Kp{-#P^`@Ucr09K73K=D|XJRWn|(xo_5r}5}c}r4IBUQt6#;E9mhcE^I`e}lbcR*_owdWrPLQf(YUNqbjbB<*Wok{ zQnRRITX5Ni%UH-Im$bef%VmZHwgng|!`D$r4OZ?gx1#yfugVYcDL?&6{Uj>|2l>gW zr!JoF8>E0Pk*(Ir3TLx3dB4;#WuO6cr1}t7b)&1R`>AK1K3`U^_#Sb0P!UGoZqedJ zbPvo*6x$CwQyhK&`**%`=)gf5eZv)3yy?wve$RW}qk^-k*DBT^a1e|ui;Zf$t!KhK zKtxQ^&_B;pz?y87T@YA|9Tb_+I<-g8aJb{`cieEp4Zv35DH8}lp>M$gf~v!Sq<=Eo z06=M3+L3ec)@MHR$7p@hTQJpvh5$jN{A{~$XM+M0i1sf(`&rVfS8UjD863M$|Ir^I z=iR+?Cpt4YA!p~n{tB$dz=2Ip5QhfutfS~D(Sz+-N5-F4To z)4+*0+()_)w&(1`(3!!AjS{~3Ei%Jayw+P)stoX@iIJ+F^v1HCCC4oTU5p;I8bI$ZezCmmd+O8mHA=(74vCRn1n@wK?o>>N63UiN+$89 zYN1Jo;aXuS%Mi{7KM;A{40R8KH!Nl-xm_&cKXx8_D{JwR}lmSF*q*%N*pxUtEGtN9(NfgiF1u1v7%qLt zON{Fxd`emOWQy9c;jG{A_qSnStfercwiW2@a?Pi&`HcwWx^-@9BS68;9;@SXJDw$G z#@aP&@sIz$-~3H>x9IYz=UAp#jX4LWx$PZ469Y?Ezz!ruR3eps(6;#?ne41gkWvuf z#TvAh3H*bto3~uMdCQyMcpDs)Z+rtU5(S*mgI)ITKfpq-)K+kp%ABI~#-tEdRM)Lt z{iVN6Qv?73w#Fb5h63;n^9I*eBw!{&IC$G_w-F(LEh^Vl zk!s%!ktq9G9yxU6$fExKuhJGEvAVOf zivlydVNa*fKq|!`k+MQmly11;deADBVqmC=8CDQv0qIH>5Qm&M)pW_A;RNm8T8U1b zz;B0rXxe;rE&%}f%|UZ;5dk^Dzf{tYKR|nt0Cax)uiw6J&t4$Q)WpQ?cii#EfAS|R zqBItFNyEOJDV3G7P70bUoZR(4KFl)w=YR1R1YZU%UAA^DQ3qdt(@nHH-ih%mLEBSs zxF>Mw0Ku{(_(Q?-_SqsRsLsHAAREH2Jdg8JeRx)$)UHs&7u8zTNPV-CvhG4>m&u z$#tN^bz84pyLt^Gr+2^WU1)=S@B<%&iwxr5uy);=fz>D!e)F5({HxD?w!X3PJ$K%@ zX5|2W#rymEcotC&imsFmq#N}6iRcYChFI0y0v!@f2Zgrg7Rh%cfHWu{EqKsjM{65u zULX7D$I#0}xdO#gIy0G|vSXRZaWrL;IXOASxKQgQT^Ba3s>SSYMIZEGX-23m`7v-} zNI}oQ>WJ$sshI0hZJ6Omx|>_5Bxc0P7S~u6U|^#;Pv~B`g`gJbY0%kgyB}_G$i((j zYO6y6W)C%ifTBnMF}TRcB^l$9554z0|N3uFJ^eH`UbozG3!!>QrZfwQ#sxzpHZ@JD z-hA7eKJoki7lRONhm?X1Ax;S>3`{{D>0Ph@8}OU1zk#}(A5q1 zFN{r1og6;7cFmfP|Id#D2cZDN7TaU`1FvH?3-$J8P^u}HTkM#{5+dcLgCv4fl7Nf1 z!%_UTx1bnuTZpw$4~b|K?O`%8w3ymV%Eq)o-aC( zCY)N5@a_*S=|X7SkPJUnHlg`uEryH4!O5HtCgj>P41B1>UJ*pd*k|^@sEgv!b(Kmn z4pzZ%@z_kTUFujvF__=7G**?mFihw(tOi{4+nSMReZVf|elkhn&Zm}(Hh)J#fjyDLd8v>4QSA+xw74>_=f^v=O&A}fv@6GOu@Y3zqXZtQHc zgp$&_9a6HQ=XGaJb4(689fuo*Hx#J{(K`=#ucC& z9{$9%DH3jd)O0slg?gOM81<)cw%RMYO1idDRRvs(did4 zx!A-BO~Sf^HL_L^^>%~nsRMz+V5_fkWk*+!h)xLc#EkES(cQXa5_jRk5XUq^Z`kv~ zQ{uWc@sT}7w72SSOl1VVIDw3KqAXW-2LnrR0>LaMgYP8!BX7F()}BsGCfaVk`DQAGxKlB@YUyX6 zAYkR^lK%eMMGD*yiTvMx{3lO5@dWHBz5x)cmaka0eEDDe#a}9P!W5=vH}#Z4OHfPg zeO0BPogyDTo$a;0_J#|eoT|_1glCZS^kw;%z>y^88+UFS@a?vC-H{_lzWFcT05rF? zwUR(hvbx(HIt^1G5$mNV=to2Ax8D1{HLC_b{Nev`$J_2eRt!_5Vmp^X9;)8LupHNvPS!PO*RTkpl!9yqyJ@?EGDH8<3R;yWM1+Vq5vZZSH*#W#o>LoZwUgPIT-GCVS`<;J3b%RBVX}&6=r*{I16x@~1+HH-*96-$4KXhUU#4tHJ zhDkU=Sgb4o5iF4eOh%~#&pp5_0G}KjYI;%ICKQNaswo%x)48^31?4+DgjknJ21fKV|RUY`}U`P{No=#@W2BH zCMN&mKYrTY(h98gBV5f$P1N_5H%R-wr-IOrm!(t*M z%!p_|a-{w1pZ%V{81R*w*=1se41Dyv;(jAmEe<8~oeK40 zGgDgKo>AZ*a--rXM}AFOwc`Z;WEGTvtjLj8ID;-J~aP zyXJNMz6f*Nj}D1{e?B>5(da45$W4s;x*9}wG?#Q?t2r8NCy9GZi2J@bh=;$SZyzY*biXa`T9yD;8}F#rb$HxmpqP)==Rp6A(D^A-s5u8-W++1be&1N)7l znr0owc;_kuE>B?zY}vATU|vSp?v^*a{td5xJ?}+V zhM&w$N|ecD$04PEXIsEtl-V!w0{eIs4+GOwCR60^C|^=MfE}hP>cv2~ngoZ9J`Ft^ z(qSZvHO%Ke_xW|})-PHlVKJzV84W%PqWZZ`zG~KnV~QA*-T&gni-?+vq8=_M6v9KX z5(9QPj>sY*^6WjpOCb|=sU-tWQAi1@(7T?RVu6G+rwV|!*2$A2=SQ4a%)ldrXcY2p z3`g(2hxV0NL5gst43{QGX$kQc$!afsg(u3>@4&#ymX;4g2evvC0DwL?cmjTNZGpT5 zN8$x5u4KfF2*jQwMl%o9I@C@8XSTD;Kq|%f&0wTV8>xl!f_Yvr0gXsw(MU{OS6{sa zy=U~!AR~QPf;u~lxNJx1_n2UhgxHkvrlGlILp^%VyWaJ#l`B`GDuZb=zERa1=>u@` zUk3jIrW!d=LCWy@*WZkGBZ7Ep%_w6-TdNL&Y?=Su4|HmkzBS+K=`CBffV0_;f_66})UxUqhc= zqjS9iOChMI?t*|p6`X?=&XxY6T(4pA*$iebDl87YffV%KTC$7s(pM)F@P`7j^eB1} zs0j6mUwm~t3Fv{pc<~keQrsr!hI1nHB9y`jIjW7pi?dc+M6LtLU@lTr_?kD~c!Nqw zK5Ut6`k$R!`Qi-ExT3l@`!pn*Xuv@eEac8e!?<9YQvLRfTNnH(ZNqcY=!#KKQ8}2Y z`uci#kS!MPWi!SJTCkA%`0+t-Ik%kSRi=Qg9V6LfQ=>2B)1@*7B8FeM{TSG~EfavI zTOQq#p;@G?pBN!FjLM+@*CdAWBIS1%MFiz^v&DK&PT{IEj}v~2$`D9TM6e&ZY=e02 zK+GSmFIxdE7tnyTr}_T2!W7j~It=I@RPu#bDS0yOv+{*_rht6{u0;?O%(LTH77-%( zW4p}UgbA(!5G>(pQM{y@2=%PS#zs~Vm;pgt$tWv_a`{|KCRl_4px6UUm#8$>o|
@K(<(n12Ujx7QvC262DoSvGFAPdi2tD$(1JVmyYCzFvTN4y0nkN-cm94wP?FW_Xn za&EM&_~`=H3YdREYI?rp8k(lsw=lG6`WpS&X}s(a zE&CN6Uf0N-YO;Ie3jIfA$cUhC@wV9;n>kTKiYdw(>_7@nw_*RFJToW=&fP?UI?aHL zskj3N=S16!npmz8#DTl%rYj%cdF;uZ$C9aW5hNHw5Y;^l_N0PDm5v6E%FcF zjX$w}JK_FBxYcsCxXSG!Zf1v1bk*y5++2d^Y$hKLHmVj**@6-Rf(79nN(g4!h)|mP zmN4m%-!@F7`&UtFM1@h~A`C@TxNXd;!@h&dNVcfG_jig!UW9OmYr1?PZ!qM`PUHht zW4>qw2#B0NTHjt?xvbygnPjlRmI)hT=|a(5Uztv2!=7kiI&JFG%v6FxNlH%n@G4$g zB%EI^H{RLc+q!M-m75lqtciGPrZv-6^RpOOu!Q}s@HpV{jk@t z$#8tgWJ>XvWYzl3CYVL^JVgT5ur}b+kF|#>m`PJH1J3ivi*GTlGU4)>eG3fSB`Ja$nz!sv|CV$!)(kdMw>x*@{@B7`!^*4vQaeWTE0M%!W1G#@$RR1Av{ap?J3Z&T@^q(8QGgjAhl{ zGX%Mq&s#uw)_(K{&hor{iS_~<(=qr4oYA#v=#cvq55jW7O0H1Fx$>Fob>2eP0vAf+ z6UV9`O(^6WF*3zQ;WUR*)sbYgOEuSby_)Hv7?h{)DfNFUVxwvTCR?-uN-4EYb#Y6D zmI%;}4M44Hk-xIIX-e=Y z8A2KM8ceftCu1MAQK^2uly4p(2P&fytKq{QBTcjwzn>w-Zjc%=7U;Tkt#j?1U&_0j zG_QaHvWfY>eyQe(F31b__PsL`xndQWF|uCl36a-ICsYz(NK~c)-pV;Fh?}C*`$MDY zpkcHKSC^~Y6gEY)dsQL-e9_D+UZ8|rpveko68%H83QvJA+hj}U8=Qw%Hn?+AzFd`8 zqyd{eIhB&X7zZm0BrY=3b#zMj$O%2GzOkXJrvrDonJfaRkY7)_O9`KAw6D9puNzBg z3`0aLnb#wt&LdVXZ+PgzePRE?nbEwzIgE=LyGE3e&dytT3mi~bGcybOTDA;yUwt(` zDKV^KGdADApsd+x!!G;$SM7_53GD=G#bkYI4}0lVQOQ=Tg`q}xXIqs@`IkjPS z)^x90-RD%f*{@(024N7^jC@#Z)Y`^H4hq9JT!Ay(2hQnTuRJll85U!;$SEf{c9dp& zblSPY6yYxm1oUK)X0^80ExE41Ai;`s_R6Oq3tgfHALmp;9)~RQU9<4?%!~|C1OG@Z zhwx5bHrGwjoFbh8#KD)W$5aP?B>JYb%=u9WCBbrl?Lv;lpV&$^{zJD92nZ=*6xCMl zIaU9HiECX8B4BWXH*6kJHbClGyEtW=iFL{C(9Tsm*uM-?_5&1NrBVHm&TvN8Qly5$ z=gMkM*O8gR9%F|9Re2ZOLAH*#8E4-}KAbB7D?>v~ail@nRCNpnxesvny zon}GLzJ1?f^8-w~7b8Vg?DOV#=1mj?ZOJ$>Jf+FMniwH9AK3Fa z8|MUq^8pvqlh1tK`TZ(UK+e5}bPJ;`Y8Bb7UGZ|`)bo?+se~SgFfUf}pgK3wT)Bow zE*f!zh<@ygSr~};3VM^<-CMTmtJ3|KD3)_gND=f5e%_GF8gLjfdn$X}Xde&(j~8KX z!VTEZ_<@oi;X*Tk{*Pdo5OzEYGOJ=+EJ3PE-YvPTqPtq)TrCoOGj0(mbJPdD{hbX* z_e_U;&3XZQf6L13ZLUwPToIu^l?++X{l`YT*Yu%bvv!rf@41|nPPmD$j$HatSF;Fsb1$@tw{;ao{bAb!o z;-zxV#qs>v4yOCFU4E6WvE1+`Sc2+($y8FQQk}uGtf44IDl9RAXwEJl>j$(2pm2~< z9jH2;v4+ilLCfqp2DMW3ytFQ35S zt$Gk{5PTFsyz(4=*4*g-@Px<;te}vzN_K{tqbe%Q&X>W8k4fU{gZ@!YmvuuN;jHYyo+v0k)SNSS|tFcsjx9>1c9H}OF(~>)ywGl>Op!tCyXO_&$FSh zx_icVP0W49_4E8B$dI~AvBB1mT8%Xd%uAUqxvNwrF8KEIyv4co3MpU%EihR=Q%Jcy z&0Q^`xygsGG$Si}7|0B%E&c@AkE2o`HT`hZA}m;RctEWfcouVObYR=jAga!Ci|qqf z0SA*nOn24mv5FoLA>r|=B_ib6I)&i8B2W(}oWzYVD(Ij1Oo<4Ka0m?|4p&MBksXAL zDq&j8$vKZ@x}~wR1evS{yN(Ghndn#@X$)(ZuU_!j?#Eq$j&w3#*HDNh_Vum`u3lMB zsi!hxWGJ@lz|pQXeW(*I?iT9?njikr@n~ctj7d$0)d*iufnDjHK+33rfL^P!OM2}p zSBkOtcuO<@4@H@`zCu#J%8&obB0?I2nUMt#2XD-hARcx!O4Rr&llc6vtF0eQcE~{0 zG&edrHk+q1I|rpz!x~Ses19r%)&rc?y$p^N3k-`iOf|KHKr6)vQLp17i;9AS{HZCl z%`VKVa1M7664h*MXTK4Ufn!J|h=g>tTa+BKPB%YN$D&K-yoy;x*!^y z=@KyfK!+g0vSF2MT!_!YqFeX?dM>W)y<>Ex%N91;QHLF~W7|o`wrx8d+w9o3-LY-k zwrxAPYoD|C?fu;!_x~ODj8S9#Sb0-xR@F1BX3e+idEhzQ)tSn3M=AUu#zU_ZWi`p& zD8OXEr13?0SMYdl?{Fx+?1WAigs2Z*Z~&=|@x9t>@fCC9`W;>O*%~XIFSsn#=}X>r zCtX*4h2oI3vo0T+ttIig+0zHE*i%f&R*FE#rJ?W4LPQtrv=%nd+-oDsqyiBn~KQHZN;KBb9}= zq<)v+NqOH`2Ib6i9`CrHVa&>qPrRxVPt5(2P|}Sc-w{tSmpHFJ!Fhj^JW`LPJu-a8 zT|6=AL-^)lo7WbfPAi_u2z4uelRo}c`IwyNckm>}ppd^`{TNDTM3y0JT-n2b`Y-hu zRyMx|lkn4(+R7`z_Wf|-{IE=0qYCSo-UqaD`=hsd44*aaXUYS0BGPw^^>AqjLf^0XjhNHgE3*(T3sLAz7`hkf=6YV9@nXRge5+>Dwk=~v2#O zcVR^btOqXs7SG~Y3Nt5Xtob{x8 zeJ`poYFGc7D?=jUU92ua3o~6Cc_3~wW_g6{WAj0A*XE2#K7ZOIy9s+(Ff6G5Hhrs; z0t`YiAQ09^x~3=pMz=aNBH#{(8meDjEi^!AcxHX0r6qa;zBtJDFg6(8`;^@gVKQg-4*hhq|L$IeMXOk!aPsp3~AH&9ubTj2l<2k`GDh6rXoNjx~^ z-*mLp0K#1D7nN{7#8aZb?m37svTZZ;Zs55+lT0Rh21BFemu{PAvpVbvLK$xuN?I&Y zy(~I0*9p)FQxs1?Nb(gLk$F|vACy1HJJd2_FL9!%m(TyUxIVpx{r?_yfc_qEHXHerQg%^^ z9uP1Am120h7~NrZ(Fue6a?g{4F%_*eN&is%Z^LQIyYFA|gKAN@e(L3H^B+#6RSHJI zBsxcQecUEESj%J`;lQ(Lr41b;@o$xH8S)d2XpeCtqFjm!_l3T=2w`O*UjK5ZkPF$r z%W=UN*g!7CSBs$m8_-X1>1NS_uX+!>%|%8<3NT$b!xC%ue%R|MZri(H1`WXBFQUhU zbdV1aR0@|4+eMleuktRGhmdo} z=hq3{W)9-V7m*KyydrO)(dF^8ZZSZDFN{6tFSP%`fG#X@72Q=8$e?ee;U3$!K;0a( zy%Av)aL?qW6qx^C8G2X&; z14`Q4=4#}27&3XJk%SsBI&J!GWRX&eC!Sh42X7Btz2+6`=vIhv{1(mS-X5*q{WU*g zzsZDb%OQ;VkF?OnNd(n_$#8Pkv{hEO2EFy4Q0KJ20v9kn$=pB`*Z8}?b$1A6`eFL| zrP~3|!cTJkAhwib1yAi*5W^OHQg8IR_4?BfbXjEC@G*7U`wIBsi3c>NG@OKZwA8wJ zz9LHuogRgi7-_p9WqI z3%^imiM6P$HzRVwTq+1)!wCzlnCisc+Z(JeyTP+69?Irn_XHh$A%4_FMnF{U-oExt z7!bsm6Fk`okRjhU^bbU0Z4gTh!tMznHj4TmA#tx#H;h9Hu2Ewfy%w}$5*;$Qz;`qy zbjlBl#0U9794b^gjhy^5_x**2u?%3S+pBLT6R_~|K5T_yWfF=fX{!6~6er<*6CPKP%`ih6YkQ~hxr8IY3lD2$NXM&Ulcmt}HZJ zQDCz|F`Pdn%@Cj>@wF`yzkR-n_(e>*OQ-e@ju;e^TQ4a@Y1M=FU_EstY*fi%QEiK9 zxM{_Y$%I#hG%)DgUt{5HuMPU%`u}l_r%EXxo0#(q8F9zFgzmx{ub!x*0HkgGVdnG3 zP@&nWTp%3^rW6)&jKz3LtdK*Y4tRP_<0{VG29mXxY>}||I(c_{lfJnT43!^o)R7Sd zBHHjc&hD!%G0DzM7-NTm6(vFJ>)JZj%==k1>FMZ^L5zbS%lr}=GYs{-$4!Z8A%jr; zUBKmY3s^_jPvC7AGlE|E^24Z_gAyVpiPl=~7WymuA#*G+`3j8Jcl0XEo1{J8yu2w~ zfmvA9zPTrxwR=mrF&rUH~cK_-Yr$Gl!4_5hhL4I#^PL>X{rw& zOuY`VZI6dRW{!f`q0mUyJ|3Pv zxby7#essdSV|tbRl2{OEOIqwDW_tY1Anr%AzY5!QxhB zofO1!zB^9jrohqn_myidfwLxyG$`j!1zL4I>LaTM#k{~lt*DOdQ31o#l({EgzgDOY zbBat7Xws`h7Uq7PpNZ*{FEGwpej*x(#e2qH>SIy?y@fESa|r1OyExVw&157+Wg)6+ zPrrcOlX{MRS?SzgP4T*m%*{oZnisPk-1c#-muJR{Ffu+D_$E7coo4(jbt_#Sn~5}H zRM^*KV2+0gLUW22Cmf#MZ6A-YyqFf%rIowb^4|f|_JW8Or?MU`v5uI)!f;}7ejS(P zPUzvs;Kq-M(9(t}nLKcd9=fR)RDoh61Q9yoXk3tD1@^q>c&pfl{;3od14N8xVq``q_6KpH-F20zNg z;e8*QWo>B_uTRi!V2+Oa`q&gFk%Bu3m98d5)7ZOkKkU=ZluCOz$sgL2cU?iW|4yGX zzX+A7wL)#R-BHS2Lu=k(&!G`zaz9GuP@=o=@gJ|VA-h5(`{FiqFyllgn{<5NAcD!~i>|yC5CD?4>h=cVCg~zA zD!=P1sv&#kdM4VTI&n}WRN-LfY^GA>&`;v5(-g_1O9s4*AN-&k5{O0x?w84KyPE** ze8$e?VAoTEDj_5=DI?UD+ez(kz8|*u0G3pTq^rsZI?X4}-Kln%1SSfNaE7c!C**MTEtd{ z_T0H^`&yt)=I2wKsgyj7shOOaSvX~k(B?@EmRg-#iw*u9yylgtJ(iW zJT6-2A&0d_>cmOmWW9MhPZ97G1CTI_SVgs5okO*iGS?pp!wA_2dF+Om>#j(LSIeYZ zMbWJizAx3~lXu}{|FMXAtuD{){YAW8)xu-AViY#&Y6e|Tn*5*8wJWEKGxrO>!CQHg z1G*z3t{Ng);F>xafmYp+3(?bg?_2VBG3NlATk&(s^>X9A55b+G>Y9Q@Om3Mo!(-A- z9;emHdklsZQDR+mV_B9^M8SH*A4~8X`v}&B#Y~=4V`Ee&>P?PHw=2YQ&K=)x5Lsvy z(0=rE@b;gVgx1j5;g8Sngl!SqiXt2Aie`o>ldv(!gUi~x|7uqB*!;+|e2)QBvUV5I zn$D2Eg>Vby<#jhXZ8Ax?gEN%!-IrNlicu^vEhNubH^@Wrc^gEOM`p` zWDUZ+PrOQ49(cb(RB$`TAQIzlVB(S4Y=f1C-F)Ff6)K}p4>GD#-<%)B=mXG50!0$y z*$6G=(IaL{9bJX$Tj>!^?-xXVAe41ME+*U#Y(Z9B^A49-Z3XV#}(Sn`1O4nD-+um3GAZh7>+Qik>k2wW_ zWe3lxb;L48PhssRi5`Uo$|2L~yV1sgHWsbgA90m}JsV$UObq*^mi&DtsP-C+ummx? zL$C5WSu>G>qC(C&`nI(yc3kM-De9e+*-@QfFEGmrJ^F2}Yc!p0oNqy-aw}!}m{ssv zbaP80aFE@z5Wrz4_Uuw#+-j@o?RKTwOI2RKul03>nbiJlB+bQeOsKmUJP$8{xsUOX zbnMzh?w2pt91y7Ir0C*5sW`nEXk*9X>T#Xdw5L5IF0G_{EMb-TXGhy*E&~K+KcY@d z3I*o-9s9n9VsaRI~jx}hQqWs(yZKcm92;lk?7E>z>|15OG$ z76yYYf#*|x&b$aggl{$-q!)O_93$X9LPt|P<-y3l&*b4_vt$MfibEK0KZYE+`4T$j zbS{n*_5q zOkjT)@YX^c<4(X~Uk4WW;YCG&4v*)o|N!XW+mG4KX3Xl`~*mw!z0lgM! zg4{Y+Ze0EPp(9`w+Eqh`*qB*lkkVz~l z>|Ol}Km;|_a*^rlI)MQ|*X%l=lf%KVP8E`=DYb2Q=jzl-by<$|fh!K3jbWdnRkNUN zGi}MiXf@61;wDs1L9zu>m>{>sbXLNv-2xNdAcaZ~kA&#~$RS~Vx3iFmf)X|bjvH3DmAVukytbhdTRfdvC30>x96s+#?N zDEf&616455HsB*{W($zZHSh?!g5Y#B_8|*(?Fh6%g)->sqXShInLjPXuWFGpc@r=z z%#4Y$kPv6aD#0EjNH9YA4A1hbg9I%hVga{}0e6dQUX*pUrWsfw`+BiDk@gk4h`1g9Z0?n{o*#UdOpa^*=-z{NxqKOr~$eI{;UHuAj# zrlzAXiZ%F(nfSBcoeduTK0qHqDYRtNh=|TZrT`cyordjSo8+b;WHO0r8W*^HA`leH z!eroIPD7gt*^ru`**i4oP)Xy-r-@f`^XPXrruX>3c}yZpW!JEZRn}{Ht4Uf9`52eBJ;&_qu1Bh=$g(` zx6&HwX(0U8M{=VX>(Y;QCz&!b#g7Ov4w=*P zq>SsyzP|VLpbN(>5PhT5uwfv?9z*Y;3;jU=iYbo&d>o|@ztV{<=n~_e61P5@>!S~j zXpR|H8AmF^{eH(tDxKbSM)^J9^aATodSFXm{);cN2VZpDIm!1nBA#rbEkxWU)3>Vr z!W%*DBw@50d8OWUhnAAv1iCFWbvnFUJsEZ-Q6dp$88(dx*vQCz$IG3E>^nM;FLgrI zGmkoI!p!u1uKP7sJ`h4_RgmFpWP#PAj#%&J!#VoTVwFgf&}qT-FH;A=U;( z7w4{+OX*0TOM|49;DjgWmYmwm}wpwi7WEG!0*&Q>G_B^9rN z_>9J6C>URf^B(E3+jWpFq!&*K9gUk=)AN9|!uj&#yVd506p(X0CJS?vbcF1<_?V_A zpVqL0BU=;fmxf3?U?aqEPDhM^G|A0%Gj=h3+{W=saxBzRz40Z{QVLp#fif`SS8$W0 z@1@!Lkm_3GQBP29N_OEc35NwySzq@BiDhqKrX^--YcGnzo+QxLU2v$1V!q0E2Yd-# zZGjtzYifb#^F?{`;rqQ+l8YNPrn1lOmydKwek59EH`Qp9nEiUb*^$SA=HYTRI zsj01Q+*td6784%<-!KuNB?CNej2W~a*x)mouo~yE`ToDf?Zt>h8Lp5>v3@v+C)S}4 z$v{{KTmQTlZ7I{CMG&(Z^`TvwyNe6j>75N6on>TRcG{H5RCDk&5YbSucgAi~P`5|7 z>iWK+#gt8!2_4WHjaF<(f-`9};+98wL7rvmOnr`879eFc9pCsZ?pi*13{_*bl2Z*H zQp)2sthL8@aq0?ioN{eiw&~dsM8#Sg@m|us#9xKPJ!=RIbSv7f_e8LYIzn|vw1~1t zD`oHyJBW&zPn)PE=?;+Ii?tEO`2gG#2(OHiJO%W>nh1Vs+heNog5z2o~- zjsQoNHx6;nUq|Plb!Kj*3I2rzd9p74UOPYIEJaNoFi#@M9ZhmXJ1O#&ZrZv_32J`o z3iDY+7UgTkqqrrBo!{wJQYv@!T9ktP-8GXztKIy?TuHF1v*6)lG4jP*-|GAcXQH4B zsS2&kZ{$Rq!j;fYyo6K)YS1N(pp;Kk1^e#n8r99pqacd?kDO3othH~E*MO?bE66_1 zqJ(>V1Kgb~E-xS(o=7(BuzPkkNUSGF3rkYSd-u@_vXciI8?X&h8oxkk5li7#_p#zQ z1v`rb{3bT?1$Nq(&KS)%gFN`v6A&xxB>&9J0~;j5%CmC(t3c4O3=6!g!nt{Ei@z+Y znxkXOPa6172hf>7b>YRu8el?aRL_y2n;x3(lBP%^H6>Xkg?fTQ#ZWQhh=eMMp>fhU z<5?!GwU{e-*KTT2OY@MEAmt1cx;vA~QkOaGl|)qhwjMAia+Lc~Up3@z_p=yx2?_Zg z)4vnZcAT@-xlZ0}>E+4ZZ;%vc^tqzVijY0>hKs=oY6A*s2e*UjhOL&>(c;+0o#cHe zA*r%`yTm9rnhHfiyLJOYSP*vZ5J9St$pD}O63aJ)`?jsiwc7~MNu}nC2$uELJ>4ci zr#4Z8BOs%n^EH(vxJs5xJ#qhb8V6WFotFEN)0iRwHsIH6HOB1uz#~qhX}Tdpz4hk@ z$m?V0>-yzR=*{F{GytdbmzkY; zalBDN4(AJYTm9eiA;Dj1sfY1D3r=$UZWDOmF2(O8_?1p zzvr9&uEfN`(yY?$zPhrZL+9<3ExBFEND|2b;W6~cvB#>9KWw_xbowW`@HvAV{eawX zf^~E;2J&>5n>PYNC8#W`xX18qrex)NjXnu1rO=If_19cvAzEN16;=A z^U${TwOl4+iOX&?4#WBJz_&_ba$jf~8Q){!iJt^(WG~+|IJ8kmk1CStO}S~Cg9nZB z$9+P%NXP4z&f&>d8jZSXc>Hk6tYsP}F7iMT1cjuL1wH5cE~&T2tPnfJVZz;(#|}=16Nhj(oDliDO#m5=k6XRv=o3iaL2k}a`%=Xc-Jl$wm8+{E z`$;aAA3xmQo@}o)3*@^aM!QbtB@(GBO2abUe<# z%E(L}9hsV#kn%dscXerQx>vlJsGNk`7mJ&Y0Huod*~)x7lPF@0h&YSMPe{UctcHrN zr|vk*4maVveETQE1IVq@MgV8GB_+xh#)ep8sk!^>a8`0}(SvnMCT%akcpI#zDnSq0 z=L|Z@Jb~o!PrNUI1cg=)lqb=i+&$a%Q0y@RMIAptpGiX$rOOnoVE(f95-@Ecays*v zLt$N*TECm}%L9{BZ6~X%I_?_}M1E0U8)D;1 zzrf9S3;86%cZtI67EQ{R0BN(~Lo}>>_1muq^2&zUecwO_=2@M(%1V?nlG9ne&Xzhn zUw!-x3=Eod@aeBa>%VV}c|9o3ZViaG$nbyWC^{IV(|~ez2H|=c0*>(;8$*~UmsVB| zild?y35DAi#I7F#bSx%I|cJjEL zcH;S$%I_xqc=-m49}@I{2LL1i)cV#*)E*+uRqQe9HLzH1c+?r7{{$m8+)#icoX_&< z9i(jjoe<#7AY-wG=np^ydx|vD4M+3DJC!>{4jeJ&(QQsD`aFhzCwv}Q*KFS33* z=sEHw3|6K}SvbFwD?D_? zpRlA!@_xG}jUDcr&J%gOSckT*tgc3A3M?X6WA@uaMw3Md$HO_vCu#roW%L0}ly ztxTWZnehPZU}MGc_dBqz0%4#pi{whJI+`9&{fMO<)84c{5%XP{^4itft`idnaj-Ud z4xCfoLsF)C;bab>w#!!~J1-W#aj9WUD6Q+2(<6o|P65@HKy%b>dVMN=zIC2ty6KTD z)ok~Lj7*?KI#Lxc-uo*p&n~KWSlzZ@sysCIT!lb7Pem=Y^qgvAe6-FTy@`)7TEpu} zorI?1se=13b78x0`MySBvGvH{ZJo@QE|#ewg^lBJG!Se6a1Z?x1P=XlPbn_{)Ah&nJgw~%$ z&t{q|yNu?GPpxfEIowwoCgyh4a(MdY@LxGr6{jc(j?~7}9L$A-g*SuJeN@h6XA#=X z%b2J*l?XJUg33g{dQ40==EgE>8s|3a9u6>*2JQ5zNbi14t+yM+Agw4K0>6hcm#b7* zpYiG-;Ii7jjVvY7qm}D-LSc+!3aZ9E{Hp6>uOsrFegGNPYn)zK2y{XIy8ZsR8MT*` z2=s*aFuXlEwvgMfUG>AL|HcB)Ej8c$f zi1gI)TMozFJ6z(Sk8U`xFJTA>4**3ZQXMnKL7f42nM1ej&S+pmNp@Mhr)2$J)b!NU z9u_?!#DkV?JEai{*r@<}d-;EdiDE$R81FLER5)3dm|7L0PJ!)`HES(Q$465ID&LSc zG8%sJ&OH-EaZ@1<*R1|d3qJI=+hjyq4zygPHJwduBS@|wkt`2&wQ1s(Y-$UKmp}`X zQVFSii<|P)f0nm2yt6T|KRGFVu~uJEmT8KUpEFC&AF|X9BH_NWA#XdT)Ywoh~!N<$%5oQj9pPAi;&&&;kunv!} zV;TB6naS0~mrbiutckoh?{5Jg-g3nD(;H~g?t00B?93-u>p*B1-SFpT?gjk z2DoJqt9?)1ad@|pG)-7to1+>8i@lTP>>Hl*^*1u1IaO_Q@SbY1E;~$*eM6Rg?|)Ak zz=C-}BC~=R0resy?yk-m!5|i72K)}Ar15P$lIIhKxu(@1iY1r(y^w((HaJr}CV4!3 z-djl9WGSJN2cg2O(;^QkR`A^9$^HaVhZrqSA2W-~o*vGs(`|}{p8W>i7AS6U=uRq~?-@iyTL^i))znJH-17t?2^r2mY}L3sV`IOPPp| z!%e9vIxoU`DpTCL)7tk4k^htBQ*4d;+%+vz0VMNafRO1*exY9=9 zumy>vj*a;~zooH08>8bOb_(3b2zF(wyYpwULA{b^FWPD6101dAeHKF;Wy*5FSu;MB zUkYwp@-wbC%hhg{hxO0xy^%f^r$z49?jQ~C-Sftc_>|?W;`6b(8qRHxLfZy%#_80Q zayyV=&gm^!sLuwRe#h^<(sk)8h}26ns^SBvVcW?G=E*u|W2O?LwY7{H8Cs2-UK%o! zG$ZAC87bnDzWS}F^Y_rUOwGeLOk>;iHT4xI*50pTOau~$fAJ6nGYO(+tpE50|4PhF zFWFF62T%LKM)N~E%zU>oWHjX3)_7R=(&OA*XLV*qoS_OKlil&9Fzo@!K#!dz5DyIj z9_X(kn(5<+-iohyVS1m07J*SuAqG>+L7al>2h5NaknaHx9c6@s7?nHAtCc{vRv>ebsoKTzu`!L`3zM+BBR$u<} zB0(~7JRajkDQ$FqLdEciId6F&WLgnMsqT&_>y!b;mXw13CE(adm?8&4t?HopFq6Y> zZ|Je}(~~Orh(e<+&}tLli+583aO-ZTRvyCG7x%zd*$HlBzZuxXWlV zQRC12D?Fh1vh!0rE&td0G{sl`XqRUjpYv#!YfXzbJx3V)QSv+DLh<-huE#03jxh%I zF=uY4Ggsq{WLFfP@E9UJ!yi;+&0+t6P%y7Ns7G;7L^R!d)`xGLm(xU8tmpRAzEhAr zCD7#ln*7XeeF|G4_@?9U+Pn@Y<7nyTmFnr{kJ_Y654(okhN5cf+r1dc&*AE`xBiuY zX-qc?oEGcFUAW7N-};){7REs5DjCK3w{&rpLsP0X5}}w_g_8-OPU!@7Qs+m~0U$_r z#|H0%-fto;QF3y3buHCSqhs8|8C>oU#YLI=Ac!pqn|Fm-J&f#=Q`1J_PrFMC9Qn8l zYmh1Dl2h@&xfZ&+ZOzb=ZZYRoIXG-SZe%h58Kp z;XO#se|m1Tt6%rM))o+5jP32WyFdOVcGsr&XeE3$J~75XkHPi?pc@NA@Ga9B_X6Db zT{pE)M=*3iN+U2-&7oEb_Kl`+xOj6##y3Os!%|SX3ZaG%({*k-MW1 z0dsedwbp2wHmb^HRF)-=8K+sT$2DKIXeudK>9XG5ERS6oIyK~S z^Vm1%nU%TAjAL%t-1a`MuPL%|WnoN?nV6kuFg|}pW1KdO@srd}DH|PAYVz7OtM?Ds zuiiz*zZHTPXaL84;2C(Rz8I7Fypzd$-rA|lV1e767Qx*=i}HRmQ10S(yCds`N?92T z01z3Xp(|viKsLjJY~K!Min!3T!}Beyt{MpQVQLQTA_e9cU49Y2Y zgq%$}saXePXEW~7RF__^zu)Ci$h~^;Mon#KC&7#oXP>eL^0?4&ZvA;Md#H_3s#&te z(L9%&;btYX_}sYe{E3OvQqP{2GQu=xs80EvaaRGF`kNA)F)-_Wqh>XOS{(MI;bC%L zP3%wgu#p16EZ+7^J1U2>3D;TL^y=fNTr<*!>q5 zOK!lDj9&M*C0KXs=~ni;P|K_MDnj6uEZ_VmsNG^!N*G5I=2MOn#)@CYmIbsZ8#^oR zr?6SJjPxy@10!D=J34MZKq4$RdKPcPl!CcQ@FssK${4Q83X&8xSxVm2rWB|(F63zV z4}4RZ8zimTwebJV)p#7-NQI01aJJlQ)t~%Qc8G@7;*5rxk$if7-kWJM>xdznJd7py zxlBN7%E7MvjlkZW9#p%@CI&yml(r=t4yO$RWoXbikuCru4xeCbm7y=~By-aJ`hp-` zfA^j`9<}vzd}3k=cy(o^q@-loWKw=8S|mK%oe)dy2DE(c12twf{Ptc5yqOj8Iq2ymOn z^GoY|PDuYOZ!4KxBk)&K28W?Ga+Br~ix;S#&Kn_udEl<5SuZZeK+V!?PcETgp{Xir z1a}qjrnG~f6!hQfl^miW`SFVW=|D{QnvB;2J9@W-cqzj3A~zP%JTlVo;G_^S#K)hP zSi1~)u>>I(z2UGcee2`Bz-D1-SzP82mu1zI6IDC{pcpGM$IK(78Dgc1aS`B7%@jDH zPoJJ68rEI80gz;OX6v}Sxp|!w;T*&k4~vzhX=!CdM7%ENMb*(4r)Z@C5=?@rYCzpB{Yo)CQBfQeNnD!{Ek+konYScV}!b#mGlXgw>Iel57fE{?Ki zOGGK#{sw$5%B=q@*kned4>bAKiGCnbbe^{Yu*ruW4#u^n-Q{e7KUdXgZHdrUN?2tF zYj!%BBPS`&?z(8&g;gXm=IqN-Ln;qV;iI9`4tMp}-Y*AZWZZQ!ER3vi%I=q#Ysv;b zJ}enwl~l#YJRZpZyW{Qk(To zB|#k-8JXiYjG%7g7&>+X#dVPH-)2+J{de44H4iFluJY!(nxc!K`F@)_$=FUb`UCD^ z@VfLnT(1l<$9EN~#l6H6J7^e`J5Kv$v)>I5%$pMgM{MdtXGLSUgD+)-@MaPkhqTN* z)fM>7bT@ie0=@LZ2F8iQ{TE!n5gd?D_ zYa|Jbr?xRSpM=>nViKN2tn5O4Afe{bP%me82tuk|(9qDX#)w5_!je=W`*w(dM3vG? zei>lDFW);-(cfN9a)(90s|}~ezFv01X%Mo{D^xHaamxG3dkk`H_!{;dJ@wOi#c0~I z;XEDqnY7eq=*>%1v7KoA$Gvz#8{)rM?xA>xV^O``?ihdj9v)`yjnyjAhPuj_rKoH~ zaaz+nVBu)4vZbET3d5Yx}iOQ`Y`P6EseVb@pX92uv8#>j1{?pRZfEi;C zuQgCoT9Qr;bXg<^KT1|E^UXkH3qkptVSve8EXIbcqjFU;O1W_EWTy8=(=x>4`Rbqt z0*r5F%oJ!2i&?Z>*yyO~>S);u0hKUzYxWX)H9D=Od2#21&Adhf-s%p#44uJOEW!^vcFidE8Y48gkI)mbo~ zEX1{yBB(jMp#Ht?o<@8T#^SE4sY|YFzI(dv4!3JBs6Csh-9^qxfDdFk*pn}Kd3}xX zWK9kIj*Z#NJ*c*afXaMk20`Ml_(;>3fsIQAgq= z|L}}F&S<{I4L8bLx@3qktjI7laElsI5qC@iNH~}V_vig2&R4bZ9DyLolw~@OI^he3uiIx6i0FndVxY4*sNKvgVyc!G~Nm+Gx; z@HiaEuC^ePIzqcE`>a>2)dhT2I4o8G9m*r)G&&CiU#?(P>OBG_35{WB85cmk)0DfA zD38$lE)Tcb7it%xIVrk%FR42o>y+?6n^rvFeSY2X4SAa5_KMkNKv?pXkpg}uG~gxK zJ?uVG&S&$DWHl~5FI!d!g1f=*6(Em2@s;Rq;9|A$vZD2}Ff364X1&*mm23CFUbnb# zGyz*Gc_7SBjS$Ky27@cisPrqTsV34z|`jeq(2${Yds++yx&IxIF+M~CZ5Q}c-FB)2Cin#4S zRpHL}b8DXWyIBWb`%pMkXO5}Hzc-AUpP;54=e^04udN@m9vMYdXDV6bzLkx?bia+l z>yx|zl2tYbV;Ft2%e5vK-dnzA)>OcwQr?Fgn|vqq&VJu6V6izxsU!)i<=o1Udu8fY zmY$XXL_$Zj#-JeL&?1E(?j$XghygM%0jyvH?jLW>uNSRwl!S4j+HCd*#h0FNVuO(+ zeWSkxD^iowyV?eU)|WjDGT&~JI&TU9>Oy?cl42bIG1JSoTgPP5YFL7gbUQ~s2$d3a z>(TE_8!xOG8RyA3)l@q-l$|$}-tUXv zWl{gMItbKMe)<>D7&Y7OxBVR+ubD{gpknGgBKiV%$6yT%VmB>$`YOG20T?}LcYtj3 z5n=&t0uO#B;>c z)3ftAt+Sbi{WWCZ^~~h+16t<^m=e0wqzfuZQ8)KZPKQ^X342^9hfhK>M9-=ror^ zTRU?Nn8m6d+nJMrQ}XNQ+vI0DR}zMJaAE+;=|PnOZs3?>(rM9uTHXaq3QvM$s?iWX zrcBw^C)Ga)82V08{_w8HmGhKt8fxuCn3+1gGVt8rojqZY_Ubim#YNt0Y zQjqmhJLWEyXP+eTb^=I9;PchT>!pe1A^&H<;O6Uwg2PHKo?Jw@xS=wwBea3`QbsS} z7RbwBba&drrO6aNN0M$hO!%TQ)ZI&J=~0z&3uH!{4A(s}dH!zlTJ`V-w)2|Rfj})E zDv9-OwNQjrgCkT*$j#1h+$1i?0OA^`*Y%E&WCF{&L#^Ppu*2U$C%o?*5}7+D{#Sl& zh2+L<35*vRwUYoTj<<19S;g@ETdt>(F>$odrR`*MNghC?i2o~=&(~NAM!^MwDz}gl zV>jK}Suk0+Z5Ezk2C!Y1+f8v)Y2-O`$NC%R_R;_^I86f-`$TgK)Mt4E0{o^W7JX9g($7 z2LZtq??Hy(8lGm4npF<#buLvi#+3#3j^K#Dsavm{^&Q&fW+(0_S>%xTFX%Wwmnw&y zh%Ja2#L312)Hn*RMf=-ZaH|g2$;|Z+HSGUpiGqC)f=w4kzE2N0wUyP@KKCWH6Wk%- z9UB`LCM70i#EuxJi2B0lzok2N68K!Hk%h3D+Y71+!s7=&^)lkt?rje-#XtlMQuvge z56&UIF`|Js3kiC>-C1y6^`}tF_cjS=_UthC53meK7{kZ$Ux@us;LCYQKxZDq6(A#A zKwz5m_k+1tJTPlfWhp8eq;?8UsfkTt&SW5=Z-AH>=o=OKZ!~R!7)`K-zpwoe3QvhP zQYL%yOdKSE}A`ZrUjPK`V~MO_vBxdw5D34Pj!W!RkCn^)^ghHe zW84l#DG5x3y1HE6J+!ttb}d4+Kdl3-ml)ynu2C0U-c%Wy`oQ(mzOPCeaw5Ozpw)Z* zHy?4v3v9cRRNjS%YAtVD+^!)~bB2<`!#l;Jnk4X|bPU~~CG5c?+S_C6*uk+#&`$+w zY%q0-wSYe=ro)lymUodWXi0L}!gLjv8U!xsX?(~C7f@rZ)yB*`-!wKczE43({Y^7$ zzed)L;L;-m!|4!tg#=o`h|#Xh1k0H*>E&ULKa2gpA<5D(Xb->=WktPf5+ziPjF0>o z*YO^5cOjQ?QF4O6!}z}vZj?aOR~ZYjN*#Y~lm1)qXn@Fjn)3H#{`rCzbVi&vl&+k( z)Fb4-c^AMZuO_g(zpdDC*56Y%0KxeQKFE6N63Wkr{|$=(pT13?`4GFYu{{6BPm=f4 zC6?U~|Gjhr0%%)EQG@|vV(-6WX+Rn}S{YFNPinP&i9c5C0ALp`A^!0%Q_BI>&#Kpq zB>l0Xj~Gt~Jt_X}U#6-9sxPcJOrZT^MX&^*MKt8kNPkSN15__*K1`wW#|ro)K#Mfx zZ%F@``V&xn^8agxNyTHi{KEzCm+}Aq?L6L#oqpdYul;`^Nf0jr)D8uO0jaQBH-YqD z;~|B|!O{u8TFd!&YVSWl%a7)DBF>W0YltWQ^C-OU5~v-bf3@0h-k(v=kK?6DlQJ2I zr~LPW$)HT-L?mRc{~F~v#G&#sQ!G6*2^EZgea#BWR7(U){QP$&?JwvuA<0(Srb?*% z`@dQLg0pNa?w=nKWdwK&Q&PnZ$sdf^LIB{LL&Wn3NYTcB!I?&DU+Q0Qh6lh|K+Nl3 za322)&YCSFX@45lhYx^rQ0&`3D+IvvQvZT;vqJ*QA1i_d0C4_)nh$`p{V|30A1mO4 z0C2_{LqGfEr(vH}*+BG&jlkw2!+NCK)KFDPyMms4T=<^N>E|AcOT z`F{$a`u}yg&1rp zT)$lvMO{_ZQEljNe-$x>p=0|)8k>WiexME&<(oc~ooeJmz+&dl%+pzQQPSM}>`3z= z6NaXYgcNtc%E5t_jQ$yG$3_+^P@PhPIr|;DqW+|Xe!#K&C1Dnppu|)cA^(7_8CC+4 z=NmIG8E}bn4T}NfF?p**K18LwVP%_LV3CJ`fv1KBr8dXPYK5Jzv^yvu9carkF~PpQ z<6VQI_UW?D*RU1-IagXn_WF{798^F|T1dIzqo|^ETU$`ziajv*22%=GVEenG;zmtL zNr^<_CxEjUR8>P`)LBT#$;gP9|0fEjBYKd2Z@0G*jflJ0ic4okdiqXN%gTIi*W}d| zL<>=tPH}NDJ)EOseLx6gTL~3b*p94vBr2N9Ce+F4DP&F|NxZ%MWYEk^N=GRP3CZtm zlx3=OM@!3ZEM6}+O$BTt6;I=ri318BzPAm7o*PR`tjx>+Tjtl&R|Z&PYI1)PVfEe3 z%*JD_`@HToG!@7V?i;R27(g7f_F@j-$fVhI^O5d znP0PIdl?%We+37hX2Hg;z*$&aMER<8bA5GnwYRr-vEDkF&erODnHk*Bxkq-^qS6R7&-DxM9rNN6UxfGKBD^K)JHI>Pmlc2IKWYeD6Wl zz}+_~;ri@P>KxkMB5P1>!@@Lo%b=j$yfiQ;j-U<3eL0H`QEVt6Ah2v9WJ22bm~OVD z^xKGmRHh>ww5+t0(t-ccxB7_d75f=u2A( zAqJ242fvmlwlJ~<5CRvDAUhXVgbxWJmdW~INq162&=XeZIHG#=%Yc-)K{vM9IJitL z-KYa#y!2;N^gn6at_UdFDk{93oC@pfslz=&AuUME<^5H1Oaz#q3WmdIGXd{ws2`p8 zOAXy;i_TF}bkS97+v?`l){wD@HjGV-<$R?gBAPWL7a(@^m6Vu`Lyn*;Z)?SDd7*^s zC>PoM5o6Jmam9CeIi$W978d?2FP3xYAa(qdo*Ywb-dJ*7$t)=7-#1zuPt_$v9>`gG z1twl!UY65R@IwuM<_}A)Zlfa#OREMb?YS+mY4iJPlP1M?F4F; zQBza;2VL6}aL$7g3N z-m~L6&nJrudXnRltoQh^K`ImJ0g%>&!Us{@*TaFr!e2y`*7YZU{AfH|u6-N@JiB){ z_SPd`BLp8LkPhm6c|v~o&<*$AbF}2*Ab)Pf_3&nEWq6Yud8iGEl}m2P3BBYik)-3j zxXy$@S6O1rt7q!D8hR)o=upgKWEoUikz!#lA>l4^I1Ub*Y3%mfL7EfTh)3GIG=Py> zOi8JtzJC8^e^m8*a3npH2KG;=9vwZmq5q4sw~UIT>w-oTBtUT2;1b;3NpN>}cMER8 zT|;o!1f78}xJz(%cXxMg=YH#3&-?e@zpSBWO`p?!cGa$`eaYrn#nA`B`C@J^9EySM%cN(fcGS_)vD)f0l&z@s*nRi~Ow`oW z6cMywgxAN@;osd4(~XDY20m5Qf-j}kU0MX~8B>32RU5b=0iXs~qCi_iQDtju`Oc)S=%dyvYg7PONJ!?d@xu1@(gD2}L=m=*q{xR3QSP ze$32!l40S7{s+Z^K4Qol{2vvvhEqIJ^|FW@CgX+Y0yRRw1ks3yYBlm_> ze0=>3-AoTCgwm!QX~8b1&#-aUmw|eTLl+g-xXvuX<`3Ce;l8UA{d)&+8-CoW;1NMJt z4E~t!SQ{wxPUf2bw+<13;Q|2Z^7pSO{{co&H-KbMK~S3R|A%w?2NI2-0eFZkwf%qV zd*IXmA1+31xNdKM+?sf>|4sH{A$;a$S;pM_C3S#FD=bsWzdYI};n`kOGvIzf`31`5Q{tAKf2K#nB*W!@g z1nSt(Lp|Pf3aYg9@-qgzJ1#n_tQ}7Xu<^-Qb$53&R#wIdpKF7gR- z4s^LCkPy1+qQdwCM<#qo+33q?9CL-Cto&A(8(Fy1mVc?vX2AO9JA-+X2{ETstOu|p zezg20+ScS@#v({+YQ1fiI8oI>WT>LK`euO$Sc;L0B~Lf5X3)OQ1$MQ$+Yp;;Vd>^p z%I(PB3#@fJ@oYK)Z|e3EJ!OZB|a`u#Ro4B_7m|M;u8 zx31HUEZDpDvenZ$Z{DfvSyQb+lA|nWDONAfbhuRd3(gm6{)XX^QEDw~#*ep##>S3u zzoWKHO@C__d3z78#6u=nssmc`#TjPxkNrA7ad7eVEoth?M%Opo2{_mJAl?wpxUC5U z*x1@Sn~B@D{TB00by~YXGlhe#65nMyci-dWYvJBZnnj(NWb0@8iimhM{RVS(^hd1O zrtOdAfI2$%0a@-|&NcxJ0~`L*y~E8&TYo|o{aKp8$|o&lRzblnB(>Vw(_8ugaI{;v zo|>BGRE`Mrtpu<@%p<38;$dECi z_>t^|a$o0=HY!rL&IH4cZ! zx*&>$L0t2@u5#w&MN^xXW1kC}VGT7I4XvEnfAtFg!9ZPoWdXm1+Zuc)E>110G+zF@ zGJW50rIOMllRhd&lhYGtm4IrUGtd@S#k@w_vYv@7i>)a{Y)tK#%#i|Fh0DxO8=0t4YoJwanVDVE%yKJ|J zrX((=9r9lpm?$`;kv98aB(sM6gnM0he0GQBxliu6r!Btp=ByeVdr6;6Mz>kUTQ|jw zU?3S65uX3pgn%06R~Kclq_lb-|7hZSmYv(WxMXBboY8ix@p|&Iq8lBibOFvdeFKB8 zQ*-fzX}=BJeNG`QZEXV`CqJJ3Z5RB?Dh_KC?MPk_W=#SGQhochtUy?`!k|^@EU7u z;0JS?)aLJd`c=||!wj3S5-D^Ch=`Z-98h5{G! zSXb#746sAqHam*<&xdx7=kH;>8n-yBE6W`t!q!JE=vSZjOu7V$3O)m)u@Eb$l59PlT(e=c zIXFIieYgww{y^lc@;W(Gn&Y{-Cnez~ub=g?U1+uLPo?YG>dY5(Xq=-`s^I)P^n=J> zs1g)HaKcn8E6uA9@VmXlO^@{SBT8<%amyoyynHK)6EFtsdF73U6+;G9Sx+SHS-v=mG)4+!#qlvT^4?y}FWql@IxWp{ZM=C4 zb8gPBoH{Y9@Tia<9gc08=@9F2m*!T8ksw7U7UiJoD#V>l(zR8kU-eMbt^Oy7dq z=T(xrUn>ptB{E?eUR|i)mZn^7Y(D7kuRfQQ=w+f$(t}?gk8ZdtM{OiaJ>XoP$m9nN zP{e%XQ^)I~^w^)jG2`&Z(Nrw={iWH@Gkd0D7q{wt=%(4aehJhjWG-6a6z`|a#Eo?O zX}UGG&l-1R<8`dbE;RwQj0evYi;P@|h^W7!f6dEwli#``UygJgS-6(J+>xbGU5dvG zpA~W59rW=V&HHON7fBY-fCT+yGr|5q@GssverSj|0?{jUF)<_zUhRzch+3N`n3y1WM+TAZ18 zEq(6JnVBEHF9i{PGOtY-N|Y>-n3?`V!S`cR!=;PMkL*Zn!A}6-s_fwdGs3>fg^%cwxnt3#;ad(jf27zrAFKT}f7`WExjM{#cm1m} zJ)>Wbn5bwd&4_!O78&KUkB=8Md&}9yh?CrE!Bj%M{YWsDao2r%eq?1RckN=6P7eP( zJU(rb6VVcSR^8?{qI%#>M2U&FysW*XOH#}ffuo$qE7m&vj+$MM6|BL_#FU;@3LWtR zx(k{k=w``{oo8V?sdLC7qqMOHov1{iA*vqhtHHMtkR_=etqKy2Mc8$n3s@WFI-VdH z`X<~WNlS?^0pZ$9{ci!!yZ{;WetRVqy=|JbNE@1Eu;ti$WH0p6Z&excSKr|&oUY%0 z;d3_D%6uBNSeYiG&Y3eI(Ce@o1lR8m9B$911xk9d4@EnQvy1*q(}{O#mNjDZ*BCK>SQw0o`X`!5 z5R6W!%w9-()806~-03_$WfH50_0guEF+{VKEnwhO(j7}*A=P3_s9at^ceM7RHF37{ zO-kJ@k+@w2A&bex@?P`dP&okd&Pq0}UDI^I6ih^VX42X;8#gy(`T_9Y3kF?xQ1!bb z3-OBGw2>N8{BlK1LD_vUTKS0*Mn*;gX2tVgI&+7*?MMW7(eDWSRblrdN+c2z652c? zjFt5e561G0#_6yCK0bb}pd}43WCo}Exki?)d>?(NHyXONRj@)eb_`=yf2+*o2s$+Z zeYV76$6zq1H)Y}e=Z#%=l1?(e=)z@J1=Z?w&uJX~$5ENeUrt>DwTztOT2rF}pk+bC z17*LyzCLY+gyMMGHDL0$gBfW4amHBm=lZBgI7DrA5%M|Tr!_%JmwVz|XTYH8o7Y^` zX@}BYI+uwS3%TDmHOz7~16Am=#rwDL}6k+>>_|-wjerc( zEJuz6Y5$23c(t%N0l5aZ&W((WD#M6>2*5szxECWSIY89sBKAqL?OX2NCbjgU zBE3*kHUae(EVY~nR&~0LK`wq(In;esaj`qc14mr<36ylezDP3tVn;@9DKoco#9j(B z(si-iX@j74K^@sm5!CQ*Lntk1BnfQT=LeLkoL9B|v%`gAHH=%*5+K*{4Hf!DthJu3 zxd7Xr7IU;Q-;X5|+eYXBUl@FCZEY`(YOJ}1*Nv@1Ny`h5On}V>GJS9}0?NzF9%zlGw{hO- z!v`i~Wyr_f+pKG$TOqr@k8NBq82+U#T#*W@48Ta zv*wD7Te+QYsHm&EalU&vu=f>>`7uSS7x=ilreL!C@wrK~@@nyJm!v_kaGnq7Vs(?evXsF>MPOSch^{Ga zyiayNTlm6nM#2V|8GdJnn$E(FRj7~D*BVq+WC4)oNAC0X^fcB1t(?KY-(-F_45)a) zTP;UN=D+W@cY7h4E@`2-{x3&{Z++ZtfnGl&e$8NEa%P<$gqGI+M7U7u_M>DbFt z``bb8>ukovx{ptr+}GZ@a7LnikA9bJFlIYHI{oMeD^RG;BRB|IFo_nQY>tdIAl3{{Flt1 z-JX%L{T#8~X-v=&{%XCM0{6L(xE2jf#vx3$9JL6hcDz*R=C|;3w{Qyw%Q_81VzA%S z9fgBur3~etE|JyeUCDUCJ0c2L#814p(F2DFOw0YomwqdC&i*b>N*?pv=>-4mfk(i+ zXNxQ{U`F>R07H|Aq8Df@8=#|4%##RuU$f^v#4IIvWD2qL_`g+ow3ZhZQh3ju*~_fs zNE|JkU#8d+%K(vO_g%PcXtx4(I}u zMVKFS?40JCtO%{Qt!}I<2{SX`I~SnISF59ADf$CcRi~bpiBNmI&Q=RC6AXwmhL6To zB3h%s-#KSy3I%9zc4=rzy}f-ZHEm5t$MLf^GZXdySpJb47!<7tPR6QaZrIlmmr(Vk zgf8#y>sc4$mwB96Cc4@d$2Q*G&~K@oYP16aGLa?e-gPn!PhDhsuyaLmpy_}p<2CSU zE%d)y%oO@~m=-k{J*8)6ej>^JmgIa=7m^I;-Is`wZ%cTjCNIzU&=u-?kv#<4uEr9T zSmUIV5X1ZQgI&}Z;d4F}Q<;!avq>6(-<`dE<+S}dd2}RS40LJKqY^%z5D$kyOlUCT z0J=EQhx}g&Bd$Qy4Iqc-DY4C2luPCK8LneFn4{-pR}e+m7;@gtgpdEp+z!8ZD>kD4 zO5Q^_SO*t36^i+Hb#-P|j~q|<1;!I8wFt+W3Kt=l_s`|^DHR2ASPCvZCh0OgD9lKT zTbG#M8le%{aDynx!5{vzTWBil+aBq)M-6mV?vm&*wfK#Wa&XNqnNibW)R8~#@B%Jh z$1tE?v)HF{V)Qn56Y-uEv?--tMw+rczgWBXV$eZdJnRSAqIp~>r-{`pRQx%;E|$3;>xj1#E6G?s2NVi=MD~L#oAPtA0ae{ z*&&u}9Dc@+4Ke@K`=+|{*_y_B11Pq9S$z3hC*ZzVMJREjGC{|fZ6u}mys#m&tY z_0)*b3O%1-;A6p5Q}HlN@4Zw}M^kdo1Ae>Ed<86qb!SlO5@Tm$$F#Iq7zP$B@I6gOCcThW$$^#R$03o?WSXyV(+zHj3q`md}&JY zFwjN+-h9k*Q*!{#zZ)wV0B8d@fZCU*8Bd&yO#Ub@{J@C9r_6 zU>pC+D3AZ;sNoHaa8crL%UEG3fLoBerJGUI(Q!ag!W>g#f^mfyEL)>J{qEC8u5q(1 zDlN{6>Oh=~+Rt>u9?}D%P~WJz9XHv_W1YTR$6lH&zZ~kJWmP>VSg6>FCD_XMvq4%J z&4{*b*dxIsy4XRyA4{mKk!}v#eR(4^*ZlF7tAC*mqZ?Dd-~8}K4MWjDCeLCzRkK*6 zNry`D2jmpK0P#A?vr0RV4ODX9e^2KdQh_>T4893WbT1mGgElq^1SQB1g!W>reR2R^ z9`m1A^NmAy=Btp)b1%Md<#HLM++t7=aR!Dtsh%pXop)x2yRc7B<&?-E+uzR=}j;k=|TwN_=@6m1Qq|!z6MNQ zSw3kznWrCVhj-jlh$y%GQ$LjWl_dzer_Hs^Ja-Sm>c?^I95Ya;FD?med88ROI+6iB z3dRsZ8Y=@We1smKI$bc7EgJpF2s4@4@dOD?NYV?a2p{!B0c2wHnqo@F(2<0jV^Gyi zD)wiTTvb8TjF+ds_%&C?z{f2#$j75WITqDfL?Aa+!^_S$i2BFvq|VN|R*g1a*}3A7 zrEGW}_V7E*{ap76iLixZ1^=dub#(Y^l3gvW0I8hZF0nhexyO~GwmmoEj{ybm_LFvD z!=zEOv$N9_Msn2lEw{i6`LxYRbAj{_p){j9I#E9(WkeP6?6ouqhJKB92q_UL<`dty zh#)o;^qbmW5#!9$a584@vnyE~^U0)-OBvse`1@9jr60SaMSXM$^VxPc4k?CP93**2 zI}Z*Zaa%0BQfYGeTO-IvoM0!zT8l&R_-?&%ajt8&uDJK@k z2MfJ^oh*2lN1CPf{XSev)6>JQHGo~TPUdBUH+@?E5Ap)-(r5ED#-#$)uKy4sDH(tU z5dzg%{%hnZL4f!F5M)2u=|9jY6#+m9F;eW9|HP)rjKE^mF!E99{f8ql$N_~YniTB+ zh(^r;@ifRTS^?pIkZK6fDWrx`>9oTS3=5OcKrxku^o4n+lvxNagq@$vuaHMO;PTX~ zG1KqV?AJ5aVW;rn_}Cg7V$Nnb*w_8cD#WEBBp@)JV3@5p6=bh>W}z<+Jw~i2FclhO z`s83)J2=yu$&7;ZmBCPemv>=qZm4cMd1gfe`U0n5YaU1B)*~ZoR7Y~}&}N_lcR&2c zd|*9ZKBLhG))chp+9+dkFs?dDgtM=GrI5yegQKG*a?95w<3P|P528v};D~Q5G$N)% z_*v!iFxxu5`x?Sh%mgY~h0s0mCJE>w&&y(BYGY&LH*Gzs(|q_JFhSM|BE-tj*nZhl z9ZUqDRiQkPNkFJ=oalzrJ!`jyr_qd9^@2E*!&MD)BKZZ5|AD(lB6mR(pIljBNbIUnu7>-ObJy&AO) zbh6#c6B9yn;~fg9fB+;gvL|t<75RMm0YJKKcdMQhVmOhO zD*yt(P_9O#Y9JswVZLd>u1-LZeMm_39Y<3l}KV5e^%yTU-oOq&9|H0vy_>48h1jY&|?__X7s zWY_2+ESuV&tXsjqcWaHbrDx|S#Z~uF9%dFjfm{#nt8g_l z6g(*si$#E#e}4Qp2T+F72)|y$!%-*83yr&{k&G)ed%NrN1OzJPDXvW z23+6N8)s&>xIA(+x30JhF}6Q7zx~x+y`1mrg7OYt;h%5qK?R+Ek5%>Yc>**@ z-RP$8NW^k-Ep3~PWM_l}cId4TU1VHM_**A$r;ca1r`w%2L-m`R5-S-_$5uy}Tdyb2 zURKWp1CF(fzrC`~rsj8w`jby56rl>;llDj<;6D0uxqjcMEG*mtlEHRI`3+@x=9?lC z^Xk2EdjQH~>6m=G$NyE&Yp*A3 zuX6z}2)df826*8tW{>_KQCXS=ANLE}Z*<$;h8V-Yhed4Lr4enbY59GI5dDU3u?Eqs z4l%dO5xo9mhx5aUo19*s+g_d!f9#ReN+bASTrxSoz1#`E-fsn5O9o6YEe)OW+bM57 z(F%M{fxe?9djr061+c&N0ep5_8=+_PRvSfMmr=F%^75q0Kl1n;_dy_BG7!u#E9q=e{%n%DE!GHQEc@BtFI<# zY~Z)_5@JVVW&zXCs#e-)S`xLTh`MBM2A@@zPnSYEYSxd2x84jGu%cyQV)l z+!MMNCE0^3k7m-U1PY?ZNlCR9a!~x_f);|#%2tv!KVUu%aebyhz9*IQm$1{uA&|MC ziK;{$4?z~9rVp`@y+rXa{+ z77V@$Y?~OtMJ4I$7}ky$8EMsNg?0jTAz z&(HbQ*NJao!6l{jhowc%>+9XxUE5v0Ap*`1CzyH@-gB;A{-4DumB&)6K=O*N?uxt3 zg)j@lVOs==i?tAI$)q)lr&mHJ*>XKDY5h_*AZbi14PIUNtB5^f%JVWg&bN* zkW5a;^Kk`YGBJHMKOQE!pgj5nOHlZykc)T{iSzT0o$sQ`#;N0x?p1GD2jWd>si`Xl ztz1rY=NQRW;^MFb(bTuozo#Pl`}({e=b!|)cmxE*#z=8EW5})e*)hYs`aW$)Z5g|a zw>Trud6(|qm)L`qfD%UYbNq1an>=bAzXYkKR_RQUef6=={B|b*0|yWhL~2b|8g1IR z@UsKFz&Gz10omk;)x_v1iZ>+?B^X+U;*))D8W|vR3wZ9DoA_*PY@UEj1xOPC88H~q z$yS#h<0lvnCzPF~TB*%J#AcnQKnUg$)s=ECCNwXN$)KfQORJ3x(*m9b4N2eK-QNKv zy4mt=B<(jpOB8TpsrybO4Cw@q(>{37vrpjTdH$iuUi6IcGW-fx_fV?^lNcOg>S((h zw*9A-^ZK6vZQl1Ib`bnmw1ij|s5k|ETo};&gst6HEX$P{y8}#`bZhLv)e8Fiteu@K zCQThg>L9!87(`4NmZlxe%aSUVcGgpWWB(aEe9r+HeMTHCZ`l#9FcKINMu)t;I>_3> z&$#LSsNJ{$$_FVrq7GR7aLdwjySd8;$kX^=*cfKG1MImKP1<38T*##z)x}6gXCwfw@lVhO@(%wWT|V-71v$FaWTuMJo} zhF@WR_S3G?cqL8G&5|P{B>Y7<%X$ulH2~fbh?1w|3GjjAI8=$_*b(q{Iqz1Q=C{X# zaHjXu*zYGaV8deGhvn!kYGYCI898^xIwD?M#}SV3!lU#xg@-EMm2=(#0^$$4D?qrQ zlATlm=%5>kH*HqCaZ2H zgm0Er-7=w>Yu}-|%|-=-@{<$Gevq-N=`HV6g{8?WZiEs_h z{M9oCdiuK^((e9TVvEJG6ky0hH2oadTbqmf- zsBDZtTe%MIJ)^NN9TPwd7sI{j$2lqsZ~@f&KiidvTMFj2zw%guk@xINpia0|CO@pQ z7MePPqtMlk#2~rIm6cg8f}$rUZf*_+?4SNd^s}&jO=-FH$R|N&`L=dM!~|V>&L$973y2(D#1DzMTU7~=tP(QLN1NK#iSw) zUo?WxSidTvrRGO!tcD$eYhe*%p!5Yrg-WL1Wu58=CMNth8B3Qz87vh-A%^H_!|GR+>5l}2nJv!0>Wsr&gS521S-9A3z}sLE zS_rI4lQ&=wy-)ngHmntgv;BQvDGOa0#-bwRiJT>?h{*M4BW!h>_IEDV&z+A6YJAvn z@GuzjmrOYwT05ViE)ZCZ)&w#InG5i{m%O9%GGU3911ya*4iaydwn^jeQGWzyeIjs= z+&~|bVo()BcNs5OZ!Yw6YiU{d0d$+rNG5_`#xylK-XeJCXflTyXU(bNd%k7=+{)jg z&?g^ZNF1AzNnC{4$t#{p#uw|HMo7QJJhn8WM**JHly~LOPV1K73<^F(i%0XzkK)7g z&;k@YUNi`;beZO5JcsG%vG9H$^1~h(c)(yGtgIL}D{)i5aaY$`t z9Zs-Jly&I2CcNX;3hPqFHk@RtOTTBRuz-;m;$NA#0tRzvUfBmx3Q`Ukehb+NWq%Zf zfdblCA50Se$HR9_3g8z0U!b4r|A7>EO2GaWAs+W1lmP|*(hVW>*Z8$68tNx#Syx?s_Hy>-<$kZF zrDZh705LOi)=XL6^6XqfpI)mO@!<=}Gb$ix zwAc{~Fzyl=bCJt>M^9E8RLBUgshw_)-y(b;EbP0VAO6cavZuF?k+ZH&GM>oV!C{~} zi%iZ@maiVw?VKi;208b>hsNNpX>KDB*;9%T+tiKZaNQP4&)(j0_UbBy&d`{<0oTxX zL-=j)jkBYCZ~IoVijufKg~u)P>8QK?T6Rp}N={MnyviAdpOdp!-ebwW9t`1K%A#yW zG@<52vHM*EVi}|6t0*1W+H78XIjzzcgfyN@5cJN%@idq>rKZFa3wDi4166=3rt(py zLyR&Lxst>`u=xu#nKk|PX!xd>@@ogIKoLG%GZSCAKtFs7wBn9uoFA_bNPOR(LQjiH zrr1V?ry=$Ab#;E%h3$`t@xGL!1)XC;Ps`PBkJb6GK0ZE*tAen~#tk z8}=ga+Um>8huC@#d+Z!K{jmdIchVT%>)I`!$knopWeh6{-(UqC*I+g$Q&-Jy} z08y1pgff)GLVV7A*1B_buLOMK@p0ma0fz92p9@!1rY3jw`dTgMhRX|O!Ol`LT=$ER zFXzeRA+$J>-0EY(83W4N{P?=#g(kfAUvj0>fA^*pMm6ab40d)2BKi{x;VFSNxnf=V z{u7S1`oM})Xm2bx00@3d8sr-DxV`Pblm31z+J)mzH11^bgb>pOAzWzoKpn_)!^gwx zAap^ylFbXS!?x3t*mPBYh#Ee$FxUD7B{<|94G)TkPieuneAFM+IA_*HS25_N{+@AA zW~KhKn$6D|4-ab*ioRj#%t5{mL<=8ad_%*6M%1VZ^7DrD65TjCIOGGM5|CsdU!Ca( zz0?HF>)X)lseeVe^vCc-xFJf{wVIT`qW%o`WAnRsieCrV+XYv(3^5wfV$#*8DkNZY z4sI~j=%a@)+>_Qem(<3@VGO+98SwL~?=Tk04wmNU$DuAP7pg0MrDykkP#t4(`sNQ_ z!WQI-Oz2em!ehZ}6mC-er7|kqe?G_Y%NGh{)nU9$qm3^ z?6<|)m0l}8e`ng_^HS!>(%RdMfA^_cPFj6so2Pi)}n;+dARhj&RfRpzo#lMHoCf=WQ-(R;D(UC`NggM zs-`Ymw-zrbO(~ddIiYyb)OOG%9arB)B!X(7Ed5X#!bFgovYEy_A=F1xFLQjxFp$v~ zLxH)O8I^(Gl)VzMMB5mvtV+Bv^V8GQtEe=le@71NJ!+Wuoi`AZH zOcJ{v#>!YbIXvNEA9`%54wqx{$_X64WB}MjGuc$=j2IW~oNh7N%CO*GIq65+GrF3yDz%tlzb9e#wGPq4AE@#@tYE!+m& zU7C9M572~UDUfHGoO;}^Q`^@ZT*B;6um&EXT}CaUd8K%9H^|5(@NnT_BWAtd;dU$Z zZd52qSAT6-@q6nbK7}Gy_ks>Njt1EPDfQ||S~AWbEsVYMvWbaIKGn3;j;*wt!1T|j zy~+3j56{cT6{;b&D=&eagUT?28Nd;wjce$QF=jf8FR!6TQiYGm_+GI@<^N^@IN~8X zmvCMZQrd;fAuG`BS*+)0;#v0>iOYQtiCGAff-dZkQ_)E{s#xP%CI; zu^I8{*4onBe{$Zv(i=U+buH>!bkYL4j-BWIpFNOWA-=kG0qXhOZ)w{Pe|0?);NePk zoh->$V zMAk}HvxIvP)}_2<7N?KZ-y)1a{ z)KOi3Vv$g%^t?>T3GK0j=S@%K8}8C{c`Ll}Cl8AtXqBZh^R;2ZeJnA9h@8^y8jUv(dh#&A zz<05+JjL*vVdZPr{XW-AIliBR;Z5*J8Wgaq@WA4zjw`ndX4A0qGippfpyd|of!cDz zde6*mjHc6A94Kj-SyASwGSn0{%S!tkCae%MA9g(==5HHv+%r+G$%Mg^g$j3beJrO%teNDfA`s7h$h$jcmUiBadr;#BWFy_lgRNL!wd>k0kP z>r=%hS){6&Zj@=60k58V?Lu-9K8hF}dokMSDZUywpO>lP}kE!LVE`f$g>8?iwMtjKW7txWR?Xr3ifr ziy_*~*8C@=_5BZ(F4XKihD#1Tw?4ML4Ay$K1x|lF?2_o4$zz%^UCj3u4A(z;_I-#A z{undd`<|}{TE+|MdNu|ehmiKg@9aEJ9;y=KmgWk~9Qo*a7(s9r-7=&V9wN2(Vsl^l zTBQ(e>8j-4CZ?y6FjK6ZmzDtk*f)_m@}n<8MNyGcuhK9nnj z`kGp;UxU3S6(Z(JT9BKw3?I5t8aTMMayQy{eCl`lXd;9pZlr->lOly_%Ayh=2Kg>)jBi;Ig?I&y~4L|wcVn6)y{$<1MGA| zq(GCLssl2)NGsUeH+b3j>M<)ZP(*|B#vlI$CGq|r{fiUY89-KcXD8XN9l3OO^2f4o z?iTJYEn>&shY4384yuh)O{7``nW#~OO$#LfXFLR+*F^e<(wpsOn8{aEZx{EG>P#BlEu^Mj zpD$~rBSAPe)^GI?a9dtCV7d{c^o~PUR$^3+-Oiq5#g>gVH}IPPS7TF3$ZV6|pi#@Y zA|Qr;6`Nc^+Hl_i*={xb$ZA|@+}OxZUr3LX=61};;^O4gW3A^_5Fzu|x?UKDs2GQZj>xSa z5VUZ6-JP{kkzSI4yUy1UWvGzMQu}^kb9E(~VY~F^&R&0W_>WQf!6gGIe1dCGREObA z^aKaZeMBQHZ1rZF9L8GsfeWi>N=*unWX-61Di&6h6n|t~z$jH*$i(B|@Q~w8YODiM zIqI}Jzpb$K6KhXIw=F50bR;&~CfrEpybQOOmwD(SMqlTdR;jx)Z&|P=%A8&JHa$x% zdRzpm^3Ye4GS;qkr-A`&U^qwsxWrYKyWxkfOjyIO+~uO^Xop56A9{+?A3vsgcWw1+UUD5S$F~=U_c*=o;-G87^QE%)%vfm%fD;byvsY-T?TslRuUo zYA6+sb#a43NQhNHVAzL4_Hy$1zBRo1@ixwmi44`RCQKP7F}6Za9rdWiTlj#v#0L?z zUy>umD>6+x2#(t*PE1$FB|sBTFv?i8O}P%#qT zep5!!2oG6=c3lrt03n*DO1g(ZBLiI?jxMW4LjEp)6s{huo!HayKpT;Sag`juy?U$C z?7L+NGBXl*=)Ej|P~1L-&EV1q^SyPf{{GA$RX_}GNVQGdhzn$->9op{)uM zHsheZA4h|Q>cL%TPcO`nwOfv#@j2(~x2j8)!O&x4&`)b;<3dfV-)NzK5?fe*;8jTS zJo>P3aBwXW1X_+}nd8#7QyHX!y4*(f))MO3VVK+eWkI9Df1JrU2&CHEKNOo#?96%X z(u^s^By+f38$Z0WSDKR;LugVG>5=7=uTfM`0Q0YIPn+SvNlo@IKrK=RC)qo=&@+VX zl+PD+y;ttHjlmTA`E%cKe`oeXumjRaT|V?X+Y z;}O?=9}b7<`+>JV@*>iC>aoz&RH9_w7dCu!zbWWFM!%?#vCvjSvzMFgRQE`L%0UdK zPd>>drvS%S+BA;6#_)}`oW9Ui4B_2_L5d(FLHU+0$f8D}J3)9)XNu4Fe~&0=fqgk;IGUt$PY1^W#41cUsYRbcFt<%^ZfZw47hhMP zCJh~9N%li%?`aiNo`P3-iQ}~$la)rFX_|Uc4IU8@5k5Yz@BJ*xTW^!8_L=a~TC43` zZ(y{RCmfT(x3J;<-lZ+c&elG718CVgXHzn?P^udW)3{C8?$or9VZVuhTf65T%Kh5X zdJj!SRQ~y4MY-9TnX7YF>og;*KvCnL9PGkpQP=Zg-wh~+|EUuLU+Wj0n|bWq+6I|? zX}j#1YJ^|6gkJ$0lN4g7N}oWUvPBLn35B-PP==bz!Zg+rip+j5K=M2-t&L9oln7eD z2yNDuZs0XK@!DRoUdM^1h;fGNBU$jp4m{1@aYZfATPC}DITA~&(@ zxfRRYb`I65xi9u-v1>7kFff&JGvcc1Sl(mg+V07301~eRsQsOVWG@GmzdKaPhGc&jLTW#x+ zotuX(x^1U!9unJ&0b}_LIbOT?gPZ#kg|88b5r|t^1w!99fw72pKyPsDd3fyl_VyF+ zrQ`D-%@H25uTKx%!uQbI70h*B4Cl#CKTAE8BhWpZJ6os8c7S->dthcn>+4N5w>5Eg zSS8Pp=UPqfjYOl zUTRDn&xac&5-3cZ4ZIs)_zFr^B0_>p5}zSm3i4JIMRDhb*VE`qXuQ6S8HpZS(8`!y z?<^ICADLq%8QqAA&i|&Y{Q->Va%C#N^}7JR2ewn`;QuyLInQwC%Nd7>_0xY z_#emeGpp)0&=(* z*7K8qYKhK?7E!+T`NjrMbBVOPwGcX&fo_zWo-!y%A5Eir4Hb=V-hlb38D5CIeEWUS>+1vm=G=2Ron`Kr4EcURMsNrzqMpis7TJ>v#J2*=7gz!4L zmJ@$Z9o9~I1QqrK5w3XWMO5M=FxDSr;Rfu&I}~O z<)iNEJmg*qbPaHJ<>Y9h(bW;I@LgS=wrbqcHZHdhw@=E^fY(iAKG9BMQNJYy0=jSN z;XvbFlj8NM2s9>$L0i&)tA1Bhp4~bP?JYo}rK~FJO=zANrTncDAVVoj#CgB zjy*GAfa4-0J3lx`PuxiC+$q*|Yz@^ww(fTb2~4~kx5#90KPC;HQk{b6QV=?Rty^<) zM{Fr13%-j-i22-=Xg|(_XmGo`P*=`BZn=@ueq)=-S3@S5w`aSaHv7mTz$QZH+ zJ??Q!X+__@GK!};@zzY70C^ZYL%~`}N`ZEDUtxBb;0B-+`F!RVxU7x`f=vG((%ven zu4vm9#NAys?(Q1g-Q9w_ySux42=49#2<`-T2=0O41PD?qx7De4&u_gy*o!vVO!iu{ zk1@J*O^G%KE4X~BDq7dx$|Mvp=&4)IR5i~>e$8$ygx;r{e)ljgGU@g|Q!6|%C911# zbkRSaL=J62UvN)RG2(9I&z6$4jbp{oWrWq!nvQMt|ND5UZ{&H7_BN_zY{EoBT*+}@ zHnU(Om76$&vdtj&hhoN(EdE6Mmku|fD(w@}t8twPtiXco?W;pK1%{225LAR|p-xv& z3SAjpM*lt5FJKSn=6@?(yjxlD^-*8M;_LvbUo2Ly;rsOU>Yo%@FK4hu-oeUCL;Lsl zohK%)uFWcU*e|UI02u`pqxe{}qQVOIAu4xqdwby5+-UL-DmoN_Hrr~VZ$RggqxWuO zgoBB3+tte}$-W^9D$E-CZ>V4QSIDBRySB@-EOLc!aLu~y2oBcvtuv`K{AE6DHLeHxkJmx>&%^nvS56$?*nvLHX-Tx$^(_WlRDHrs#PgPfec7~`? zPxlY`WEXv8<|w+H>4R7VoZq!w<%(@bCIK z;U^w{pi)P^7!`TtA^9ZfqM)GAWOwZuGuXNUOH8q4cX2!MOTxlp27581L4+`ldOBhg zEsj*n|qju)NRRut)&#(8`q4uYkF&0Z-dWwYffkp@Eeuw+z)$2W*n!U z6nmCqQe)m7$Jjd_YNpNOQQueYrY+0~iGKLEu=TVc%tf$~Y|Mj*efsg;X9u?+Q2$g{ z`)a_U3eARhg|FR?l`ixzlY~Kl6MAX`*;2Xn4#mgc zC6NoK09Ij#JW6UCu)SY}1>4`_kXd*O^y1V@YXUE@6Y~bl(dxNL2XLc*4GuOcl29ZH z3~&93xEmJ*dgFD8nuaHRz~hpWbDt3c`9X#aDa#!B7gz&HVBl)RRku?rRLM%ZDi;1 z6)?;E0ayW?qhhLuVt$!TiciQ3M_oHy z7bF0U6YIc|HJ9nMsjgKHk4HrG_L!#pKI-%eV~C+JSU43diqdMuUD&9=A=wO&etdXm ze#{gI_$(X^`ejvEQ1u<<(aCpt++3WWJ+F-kJ+=t{ZAr)K4L1>2Q?X2^+eezMQ0k3m zOFH?ULxr(-4NMPz9c2Mwrf2L}Vp4pA`H;uxo?adlbMvx6k5)mC*F&gO z7%U-JBSpac`RV4Mq33q;<9C2c(;DUJOhK+^S|LwhV;=xF1HQg2Rh;gKj+=_|1?@qQ zW0ZByb1;*<%td7?7no}>kHNWzv%w;5wHCbe%R>--AqydHos@%lN*m~Vxe9u~4Z7L! z24@XB8kLul8hi2eqx(#7`*GR#n)Ug0!Nl1aj^i-q<7=-McQBwLz`kR_Tzus5(>iec#%-D4@MNed z3`Xx>LMRwdC*DB=M_!P&pNq2-FDNo2MJIHRyq~Of@cZaEgr}K9zc)HVHa_DouGHh@ zq%*HS?p_`bB`>OfvQB=@RMqqQR4YAH{jNW&J*(u)piw@~`T(3LSo)^#0vdWB0jbaX z{n=tm%T44<9i`yqtcV0p5Z%ra?y*ylwIajF4zk zt=1Hqa3d%(D);&cthAKcQ+quEu%+AncgUV(R54x714Vi%eb|?jGXyk2hcj< zEVgLf-gw@JZ>gcnUHH&uO#KMbifQWV4-d%yD3nLNlF>ilZQ_Whr8p{1y;ZQP3aur@ z_Z!Qt#l9&-Lj^tjFrkZ&s5D$$K19h8yT&ob)tTG_?9T=;u|?xw=7#2*c)- zLcFU19X<$8!1w+X$S$2~&)vxVtXGW`<#n}vl$$dQ@M}$lQkG$8iibt;>Z)&R&~K#@ zd&+;XN;vE;=>>g^0O@>~Uq}TwFo#5VxrJu*Sq72N;_GH4Y(*CvGy)67btT=syyUNj zO*ufJaN3Qido#k+$$OM5@Z(g&DLu}UgAaRAB#(0ku&_}C1m{hxTpg_drJsbvjMF63 zt8S1*xV62SY=Ri#?|d_xp=R0Ez;%0(-!u4LA+Egja^(nGfANVnOONrso3?2ztt!fN zA)J<^nGGnopy|`MP6rOX%xQ|aAirTje85n~fT3AtA>9%~`J=rQz--azF zwB|5$zQCc^N@GP;55V9M1phIiwi=}$;*Bjsvza=0!O|e*G_f%oEvCg%>L*_4IDXnoF2V2|>7R1cK4zoOK@0)V1_Q6bqsm@| z4v4~yUKhQ191mHy*cl4!%--wrQpkh@<0%@_}P<+cMS-zdE{)~U{1=B$4$OQ3;*Ocv@PdB|mh4im5`iB*6WtyNj!s$RU zLg0YM1xxmmqai_PQ6Rxc>&F7$82{&5$i8I1ZTQHhSElR#{M3Ig43HXg4H`SQmi@04 zjL~V0plYRbdIGHII{q)0X5__y#{T1Ejxnb6BzZuQ zw-}PgKPhLIx)Tcyk|pqbJPcDPMqC68EI&cEQg`2DKSM}26^UB~VL*5|iJ@rm>^D3O z0^;_(!AB`}tDXNohT}Jv+`e;!DX@soAcZGd;kCpHiE2nt{!Of(vo^VGDF`Hemk?)} z|A8KrV3)6#H%ar81!b5BJZdBy0g`ECWqCDclExIzsA@ua+^KM%V&+8QJ}eJQ-r0nm zA#j)C=4vS;2~}xiLNN%v3x&nS-~e-;*IQoh{MUIWqSj&}q|zW4;-8mF_RiXJ7tu-> zd|X~eD9~uc0aw6wc!dgO(PzBuv_vcEP99ni-Ug`xDVizaf=eSq3H;Z`^4kt)12D;( zedB#t%$ZM20AsOz$6o;rWPqKO+OvGn09iS(y_8d zpsU3R^Gi%)kFiLt{`V29TIpTbW7KAFCsWlQZ_~82w9yt5xqe^MPZ1|$8xZ^4vuE+TZNd!#0m1H};I4jW&|le5lEGs4MCna{8r z&9Z-dOieCPVwqEEYSPaH-dIxdMP;cr6mHsKQu!MTuz3KMsjTSqInkXfcfMQ{YRRl4 zz#T~piy>i0-_D_vA$yPBRr}nal^WNax|O4IxNKVjHBr;ba7`BK*mOZki#8csphBfH zg1xq}n>!p&MF2LZTHWzm3mg67BTs(+F8h_!8Dqx6H7OAA3x- zp>9=ZZK>tlswpX@7VJ{$TIlz=m1;&=rBdTBil3iGLUP)Q$pP|YJWgv0Iyyb_gs*H? zj_*JyLPQ=G_=^4|HK8GCNS7s%Jk2*0^cp#N70fqin*P)7)*d93@Ep;fUW4Ua7z8s| zveDNowMHr}jp$;K>HQvRVzzGFmm8eStXJr)0o32C7tn`sQUNnpRM6U$+@@_~=b26J zkXl_whnI&-8~5-+!EGGmYvlQbJ)%ajS8>1DsFn#c?$%J0{c>lF*bv8TK?9ahz0kKZFo+0t$Yf6M(Iqw(jb15;1JiK#v}f!MVqi1nuxIRzNnjN zO`>_ce!T2e%7mk#zh@?PL0X?&ZJ0H^PC>dJ8bU@^22EzBOiFKGIuJ{@WXa^&sq%^S z?~SxeIkF=sy?d_P$&4m{BwKq+(`Ssz$zfEV-Q5j5OWkA>V#8{+)I?=U3?d>z@x`>S zHak!+lz0L$7v2!7XNwJ0#s-mRU+n{3biU}CEuQ*V`_b1|b~#T<>xLsKgtc*TWp4on zx2jfInK<)A3$$5ZPjkoJDZ8&BWB8S_>;l!v^^DipO67u$u^DoZ7tI%zJ`ahDt6^nI zrqgUlz)zmHd62BSgg`rXH6v*3iR8+_9g72AQ+%#%F;0GdzA89ntJVnMFMtt6R3z{G z-gm`AT-X>)+Pr!9oZnoSGx}W2^V2uhMV`9#dfMbl%_5wQNL(} zpW(|XeB0}1D%~++uovg!SiXDD3jS$)h`dabVry}|)NA+*GC(A2s_ikb)V1LE5NbL* zyAr%Z@(un)(GM*L2G`L1*9S1zN|xArT&Zy#3lR;e584}qel*d}Tn1Nx=f09diUIvB zx%&QhNnHS1GgwR3NlCe;1>Er~U{oi@eo0GqZx%Clc{pz1VFA`sSBD%E&cC+&GrPPcUrI$AVmAWNjoi3M zEkij3y+5Cz8hR`P$_ZnS)z?555lo8%zzFiE*UnGBLpY1UK&-*Bw%7kWJrrIq5Z;ng0V5d~dcjxV}?Id#s%RSk)`cGT6q zg`o;^@^Uhw=6(;@BMtjfZ0t=8)ve>dVpi$6-bw2U5?RY2QTWmcx%hxY?y#EQ1z9v+ zGO;3+ZD?~mj@8WsBu`%W-ENa_%GQbOA$kd+Fk))Ymd-~rnq7WG31Sf($A6vP zHfzZ~Cm{&$3bs#9k(D@!b!}CpCT43E?h11&)Cj>VL0_erK`H02AI=F_Je)in0Ha2&Za=lU zWsi+F=Y(AENfQs8VL%uNW+1@mPQx05$!}Jp*7nXCFZD^D`Gjz)3@GxvM^ktxz2C-5 zi(Wv@yvBZHwUe8PJ zI>MXpcMwuUBm0R&NKBv{nxT4P+NjmirZ8SEhy0@wc+hqrb~_o3{$?UVv;DsK?R7np z!W8t2W0zayQg>|VHdg}TmTh69>#&$iz(E2JDv8i=ky@Hr`DnQwr&t@1dOT1IpIX_o z&0#^zYtT6d6sktBL%QyP#yopHng_*eF040U@F$_{C)|tzIq`5<`4$lZ=F$<2lf5krUm{&a!09MEU}@4$k#vT#8KCw17cEnXf*|wDmrkZq{K>X zt~L%7R4!=zq0%^^;nr_%|G88uEuS- zi1a37bk40(d$6`K`$R-&nL9ur9Tg-n-fZrAt7hp&DY?yt3+9ucm+0h6sC3&ex+H2_Tct`2BOvteyutf- z-auXxxyldilcjmVVLfv#zDc&cPBts+{-5IEUs-@f0Td97qed;9`v3a^90qs|I!sJC z)71Z|D!}7J*vNqb!t{bmzU9Bl4N~+*0+VFxjpqNaF8Tkp^c%4KZ9pR`9|O}J;g6xZ z`Xna*EYuN&QO}nV(;6k05i7=szYn3*sQZLT&WOpL6cvTTQ%aL)UgRiRTUpGUDso;YpHi+=?*Nq3ZlMdvQ zoT=~Wgfams@R*czg_hR!PogF7`+C7UX5xltSU?h-?LHOm#LU7#LP$6@ts76r`w0uy z35s+eqeMC$v#)Mb>^a!Xr!<7%cySR*M2sK3YD+>`@Dn>bJ+qA@U~*&WYhJTpWofDY zQ-mB0oa;)98x4lUSJ&Y4L=lRO;zo8@8K!DoK4gzyoAf1lPWhpRzsw(!)tdf19a-;az6K_WKDRYet`nWa(uNy&dY-q-|PF z_(zlvbtKgHGzp(M@qlT6PY!i!{Ngw&D)BUUc}85~H>!EZ!95tV)OoC=S>I2d%phSX zXe_whpn^V`;Tw+6%Ksaf~{RW8~ahP3=B@pQ+HFpeiy z0ks8Xygu{cnEzkVrB`K94|;-2S^Np-X44}N0b_K}JG|nhSi_3B5L4Ba zkwFBM6q$P+5JQ60CZ87bOqq4oCo|_wozS7Ce7r*_Rv3WQ$g^Vj>^O3B6W7w*ob%V0t?mdy7UJ=e zg{Jh3-K*lkf=^!WcAx)l?Cuu+Qoj(GEAmHUVJvd$eOLjs)|lTGIY{O^5Qb56xQ??R zKF3s6*rfO0OP>TDF??Px=*|2L*tewvb->V+abC>GJHXYLJYdv&R72w1zANwpjy!TH zX{(5eX&YNt4-N4laps8`J$kxeIUkkzP z01rBJk3eEwcT$9X!NT5_d<1ZAx}Q)E#0X(lTR50FJ6n6cJNY>J{7z+1bzwxzhv-bYFvunr1ZZN!DKtdWOm-BX)3dUE{9tIQe}8@Zcp{lA403#zq$JQ3 zZ6bnljUnlM#(Jqk_Yt!r*VH_H0q7sfLAPmNPb+r<<6M9AMcj%Kp`aTnQvdz?()aOp zH9I}rg5vXtKlc~N<4y$Ym@!P5U_mdE&a;lbfL0$rU0ZXTefxQ~s$|8K{C)TE9*C6f zXP?Cq`abuXoG}jKs7)MB#ZW?)_4Z9ncwhhema`1!1~q3Jt{Xg9uY{sCYOigQFJicB z#f90wbxtjodD8Ne7z83klwzc0@g*BYZJU{rcQbX6fVbUbQ;S%`@JL2aO$ZO&vU=|y z9+9=;i;tg=BSY|kuCfm(nlx%_#z2X(7}?s`7z|jHntkf@g)b~AAqpd(Ba@ITL7x`T z=W1v6o|n+oN32!g@T2>c z*y1&fE>)kUm8lBb5j%?x-Q?{b5?0L~`z5aKB0KoJpwm63W&9MDv5d9jIzbX=>SDk6 zZyrKm%}?MUAvNu-g!%-@n3Io366KBgaqT)f_$q*aX*ivW146U5r?BTi7S}Maq>}z! z#k5;KO6{qncu7vf4(&p7wccp@M+J7EV(xV}D@ft_0T4|F^!DsV5J@}T=M*%b5db*A zmUfGU`RgYo>M4dvUQbb>XV0L1EBbG7J*1H)%DMEa4O*%5mSe^!3-B z(LHS6Xay|k6TH?1>_YXPewN?F`Iy?wibp92QDe`T!tl}x=yf(X*D9Bo!Wil4?ZF+e z>-!}pp57RTIF_x%(JM3c=!PD*wi&eEj84|r#8dM+yPH!wvo&kphG0Wqflc@(VbiBV ze2<7w`^$zc_$ARa1Yv}!RM$4r!Q&D{g~S(?Ta8z=2F>}F<6iu!7@eHh>HrVd%3BiX z-I6^x94$#B{6MYpaCpr%%EbKG&ow@P{+*9eE0tZ=(6r9lZ!W;Pt;5U_%ZhJ`H-2mG z*ughShJTbnOD(XSK6yFs(Uq@`Ne|rygHV+7Ykl2^Z@HEESNW^R_rg52We`ns9>F}J zIkx`kWCnY0I~Nl@9ak~A8H;-z0R!A7!xthVUzhczEdf5RHcf6ZG7Ktg>^N~&JiUlF zh5K%9TTG0BgaFSADKoPY+h&O(qrA_EdY^ijj)qj-b7)Pn{o7+__tL(-e2W-1ZLOB` ztRF#F78PB~A4I2H8$}CnXC8Cl%Z__6G^*O>1qY7bR(=`G!%BQy+6e0Rq99#uWyKMl zCmBGT6=U4J4#x&-3O{7oz}chg5g2A<*67_TuaqA*28M(gzkBQ2fLs=OJ2c8CCs){~ zlj?FIC*ZlfN@u0HIaf}o;Rnew2HcsXBbMsFFZN8m)gb0`k|FN3$Jmu>!)~i%H*)Ea zCyEjn)9d;YzEj0%#~XR`XgICX&ywnU5+@Sn0LcouEA#83ZYoRWs9u805pNOkX$a}L z*6)QVlk24EtgaKxseFGew?u32q!k;#K6!>PaF>`0d+gezzFU-=TeC7}hj5Q6(3+_b z3UgPep&Ac^I*-h~RkC-Out)`jZW&KbLWNU$ByY()%3vwrS>{KV?m0WDX)08RRT&TK z2p&^-tX%S>BL65aiCWAtu~@0+9e`$R()c`JCU%pO7tB44nTL|8{p7YU=^2H+d00-m z$H_ARnZ_Szz{}jf(Bu>U3#cnf19O2ql&-qD z{|)Q=xy6B(H*3Wi`TpOr!oT0B1I0}Q-29IGhe8YhE%FGy*fD#*QPyYDW<{@-yVxK; zK0aDrOx@whVIOBZ`+2mi!knXvgA02WgEe)IHUYKBy#fD$xhJrehWXtxZMXV}6~twl-9hV1biz(LP@usu)-x4M_tz zZ>-88M)-?}c#3iciw8US#j<#6{uWp3j7B^?mD)(G4r^1D)RvXef$BNA+Ux2P^oL!8 z{W9!&`HvWu(sdUC5)kPOh$$lz44`P0O`CMIwP{B7)AF@sp7X8~8icsUza)iQNSE7m zVZni82?GWZJVxcc5dzCOhozqb3u%8UdS4QgkXXguSfjKKSdveS_?WC1&EdOC62Ar` zy4yNAs@|cKL>7r{e@L6FB7ee zy(0-U&3S!YI7r#KQ6tRB&ITK5fAoYSeS!+1H6Yo;F$LUVy89ZEc5Q%m)ifHKR4dr( ztF|qsG3ys4_BE5jCSSrl3gubI_%K%bo3P;c2rIv@s7 z)v7ZJod9H>*;`vzx=_!|&o=?xzRT_2PdIK#19*9=HMP09w&iZ2@XIU95BK-%^qZR* z<)<$Ar?mP!L)$3em;TQFTVXE)TsRz< zMbotu7|XSixe-<#dra=pmz~w0AFpd>|3>+6Dr9Yi7<^_0ds^&5MWH{;x??5a5YfLm z?+t$3JkNgR_n2(+Pv?W!Z%UOy=!Ldq=nR^UVKz?*KrS@NG&GkbyB` zWff*idJb@90AF204cNGOkfq~r4ITX{KvZx7tZbm!ThOQ~1^86oi&Nmh-MYVh8yy>~ zuCbfLN*OUU{F(G#@!|g0q3>Xx|8oZ=7*-O%Yc>MdFXk=g}?3cA{2&>oMUS3v} z`x1$VUZqt)`)gnL+>Q}HKMmG2^kB%8jj%HpVkYEcQ=msN3}Qn?eDb_l`{!>&B;>z_ z4LkjbmEMl{aAIw3Z6FNx@%K;Qucvt-z^$~y#@_q-c2V(<=>Y`M#T67yx%9zBIQG3g zl6*XkN#IcL^HGd{Do3xzuw%s`<_u|y$u8z&M5KJG@WU1@ZEtT^RERIJ{kV=2_6PDT z0v}txHj!2#MA_NiH5pr|N%U>Gj1m3IzwkaBH$e)g{Z)CEv+f=%tMqaN%c=J*9(x2} z-W2UsRTTKXoK*lU@U4!ZzIuiSh=c^{AwH*|znfRxK(qU*?<*BQL{5aeEqB0W7KzGz zL*F}qdl!W*5`Nf6LrI-UlsXqpO&>uxo}ZnWO!gO|RmGxVdz$Vq5P5B7PPK5O!s!t1 zJz7A<@J{3g@^c#!B~atw1is&W&4V~?+7LI%SoX1xtemn%4F18DmY^c$G{J-M2vC)y zE$&sT+z$#xkkkrhUtb|GU?t~1rK91zH_fZ+jcp{JAw%Pa39mvi40-bjaVx3HAX|p6 zKQX$Ta4Qw12a_w6sf>qW;4f`0h%NC_(^#`=iiSS{c_hzmt40yklQQAY@j|)zJ=T*! zW#ujYK-IVE)OCb&&#@Fxv5MhPCzYp8%4^JLEMNKEXylCCmo5!<>l3>GNM;X9jqxD# zfov&q$S=FJ?2<;g;Pv;Sa@?h~np&TUt3q`tDZYx(uE~9pIY!x}VC=UY7tuZyUc=kA zh5!f0z(hOzx++fl!(7!ayFZSWzVSCQ^yA_u#vc-)caEjzS5_&#<51?Sf4Bd9w<_={ z@2H4cb}-al9f=!=_k!BkoiD-LAcI8W>)%&Q`uCrV(j7jnT@XSg;e%)w2z-57&0$fv zG_`_Jo!+_Ydp!b(@eqijG&ONG_56o&jh_R-V0CUBocstgWf05}yNiqcZG7-sLhYW* zCC(jei^s5PrU=52K{|$~Y=RbxToCK1N@ul?rX1*KK8gu&6rLV6)%5t^YI77g!x@HZ z+cTIFFCk}2D3^|jYW&p$kpwPoZbY3+;3ondx7PL81@{>O{QW~M1W(M;9V>)%3$zA* zst&TEv8&2D77Mr)eI4Nq-k2I}UFUnxE*su*EUEsz*`fVtt;5rzoB+CE&F8=-tI*Te zr)Q{HvmMh5#v%4!THT7?88m1__B0rr;AjIJh70Q4P`INxSvWQA^;9G5zq&L5EFQ*F zQB6IGgHY-z{FTQB+zAkMb}=lnx>BCE~r@TX8p%?a9mW783}h*tn17 zitz`2JW?@76R~uQ%dwc^Wj7f}TBD9oGl#d(KBMyj-a*3m}6=CvtH&Gvu ztD$U=4H0_8y8PB-SEDA@mIe$WTRj{D9TV?H@Q7tjBm^*p7dVel{$S^KVpI*fI$yRqEy_FNfPptB zGq^8}d@tG(<6_Y_5@oZv+U9a2jsc#E8jE6Nd?FKj+?lkXHNnS6GirUAjypXVDM_|K zNQ!EKb7*-*?ey0~L{6GXK4)X$yl=LKxeC;*2YeK(ZT@{RHtyt7$=ZoqxB979uO1#A zaG>~xwUhfHz3ub#S7vI=7$CX`1_ja39d}(Uszk92fHjj>Q6xHn3@*6zUFW;t#2L6r zjRd}}cAkMol*D6cXsKdFF*~ofkXlIGg?tEu)6n|lIz3Za-8nl7oS?E~ca%XjyjAuA z3(I!895^UyS&q1NIzmyA%&W>UP^t0gkkG92k$0!fdP4Q^4AGP;rBJ9ho5>OpmnN+~ zWAl8+@Zl&|hAp&))zp=#w|8e-T|x$xX(C2?VT_c*IVnCb_Ap;Y&Y4Xl%Vk|vVzZ^> z?>XBXZM?CUd*%?U1+)^bq25=T=}%?o zW}wNf!(wnMnzP*(MAFITmFtXHeK{?Hja`HBl?&z=%J$E0jlelmtjmD&ZkyLAguL`< z#6{j$LAS?TyY2H0bF0#ZLg#Be82sRQeXl>Cn9?f4l%@#D9eaY*0>=Pilw|!{ibWG%nBM z7Q!O8j_WUSP004W7G5BhP1ZoZVCu_G6Y0FIhsWS=FI6?vEnhjYO-tfhX|ea3q`?Yv za&g;M&2azNpJZo|YmT!xCs=hKtL?OnG{z>2TW&oH{}slq`aih;2x`Syr`p_L;3yhi z0;U0BCZSV)ZDc`>vx2IZ_M}nXux@(29LLr|URF!`0V#^m{O{Vm%p<4B+ZMKEQ>#&-zcj0hTL=AJfeWK>W#qD!BzMxau z*h>)&?}-}uYA!m&zKP`eTLfwIaL#?YN4&##5}MaJvXPmtiLmdQHgU(Xg$)>1!l4g6 zyEX_6#~SGr*&EcEK?F6DMCtnQq$QZaWW_kC=4KR4@o~Y;wxFbArLkXn85HV7kE>B{ z-nyPnp!zSWo5qPLi1HmVFa*lxbljTLPiph0ro3L%3~sE|?>f7))Db?@PjYdLiOlnRcG!*)rZ&3P9Yim_ETUsfikF6-toCO%AVZ z18s^cFA-+6tv2oObd(sQCvQ2Zohh_f<;vH3!#&mx+w`_n-8#1?Vh9U+_jk5k*(R%t zy+QAjH%gu`b|Kbp_4F;>IrMpt+3u zz;AXnZMU#dQ*BBU(+$$1i3gz=Xu63Lo)K@R4{i=-Y%SM4>S*e77nNbv@rr9)Qu5-j zw`wRD3O4j1Uk#8Vye){gfJ|Ri=XuMX0k(giTT;Bq1>e9xl(Tce&!izCgc&J_Yw@r} zD}N7?NW{nz_X`m?7ES(05{m3#(3XqkoVN{^+;GXX|1Wx42pKS4 z>Mv}C{;wGot=d0&S}Bp$MEn1)l+p11G2f8n^ppRq!X^L5eEYv&dPUsBllCTfd2#XY znexYbL*N5L{^MRkOfV>AyxQ#kV%G1t|Ku;GvVhBOKe!m>-ea1z#?wbSt%J9&ot?vo z?(XU7sYA!k`w-ya`0@8DEBZJ`a$lOb!~98^n)wlhU}McB(%WM%=~Et3aOl{Oiy-M9zFy zbSnsH>*01KUi9KGyxNl<&w#((*Z1FEf(T-!Js-zD_@&=fCgQ zXEFoA{MVbLAWk9f#6LDiH#3Kx)3Q!>cF*V9eJ=q2IgS0MN^3`@?_89p1Z8k~X=OR! z{&SGIw|97MYsqAMd<}ejr`6Yh_O`Z+Cp09U*Rla8Bte5G0OFV5DxUH7P%A)__5t~5 zCXG(|Gwb0lA$gxWA;v}k((;?vFiQQni?r3PUTl_1=|J4a>#T>z!$b!0u7_9?w!m*z z?17j`-73sYybQX9!Kt>fmaUUmN%r9xGRHQq^w!XhVugGcka5C(4 zXiAGbYkte6Do(?LV`GfJ=b)c<^9EN~lM>*8_`Ti--5H4)4X%7wy8(;-UmAGQp@>nF z3~;wa%rcDd8Nat%{`3#jJmJ^JgM?fk8Uh0bHU2KSD?~KNQ=8rBM!)WVynS4EVp+Tz zOUgu{#-?FQW+^< z{F?sDu-j`&^4QBw9xdU*+TYv>(Yx4&?x&g?P&N3dNqYU&ieV76&?nd% z@CS%#h#Zx2dbhbU9@W(BXl(~@O2uO$gKRiw&sFbLi6MJ#B@q%0v*Nc@(S1X&9Ywx5 zAmh9gu~hsm5mEh__>5GDUuZyqqVeuhv_hIMWXtzZoCJqEZN7>Bw^&&o2lFoM#{2tU zlFIt3#UXVs=?%#M@eep{m1P;9usd4VIk)IH)znd; z-65q(d<#ApJ_svxU0AiY&bg<;x; z4==wOKIf{0#6M|Bl5@;#$fyBaezxW00zl${N78pzPN?{RAt|2Ua&Y$N5$&lkdTP|K}m^e5A81 zmVquMS^+Q9xq+qrnJOSJ2onNovPq)2@g*pRm&{`hS_%@hMRo=jmDO!rkK5^3TKdVaFCZ- zH+K*!2o53BACDXoG6U zAuM7z;-1F^j-qK4+%O+YN29?5R`hC->WNZXqji#*Otax<8D{FG2%oZkwtU<3bNb0i z>WcD|5g_FxV^6N#LAgM{1K_fB;+zfBE6)6wZE=}~AtiJGuMS({73P`EWD9~?5$7UX z@YmGLp8KABE~gwAqd*gGEREg;!bRyi8i!L=ln{o&RPU`s8P>_oiD~hQSDHwV47Z>m zpb`>J<$k8~JTec{Z`C;@8r+DXEI&SIUYP9SBX5^f{ke8u^hu5DOhe7XSQm+>rR6U1 zRQtRX0AKh=I9tdDi%s}x%(QKA@8fo(V!AeT*J{ukd2~297+({JVbH5b?fpZbkrdOo z6zJDqa?ea`eRJ!xBBLRoOajLum|4f-0N2UT`!bY7>ixk_kaT-s3glwc{V zBF<|QR580^8*f*}5NryLZ+aN%S%`PzT@kJM5l`@#qn4I)2l!l!KXtWP65>0xG_0Td zqjz_qf~K+7C~`inAxR>f83)^FpnppU8PW-MmEw=1qU5XKLg2EkVW}>6je3RCXznG4@F0rZ%P042%)fOkb2os6P`w}x|+Ob zVUF5*!A8(K;;j`RQJ_d-@mIuIro`7KAA<9pXT9y&^YKboqS*f!#xpzol6UfC0I&Lu zFJ@t5MQ#}hX{8O0os;!~r`$KE9gSAuoV~&Abj-YNLMr8NK#bv(;EdaZsxW?&b<}Uv zCj|@a1k;cNn=ky)y^Xcg;ACX{XE;@-%{eZmkQ(7-zO2kKS-s#e57p3ZWmtX~RP6aV?a$baM1siV8_Y13g_!!!gn_*c3^m z{!lUUHwX&0!#zx#AyAVATFxnBtH1F@G)Bqv9_^TgwV#{?Tf(RLSG-ctMF~@IJ0JI% zMnYQ3I6sL0nPYw;I|CWY5`PyPl9i!CLQGL7ha}VO`xCfi+7BOhHFwc_Kb2%jYB7oy zXSHmU2ubvK28J?)IZEGg%vCXaQ)&plnD^;1Po-FeS!bG)+88(^RaJ2Wc`Cs$M>GT} zT(MX&H6C4^x5ml*+*r149rs&NBcMA2`m1QTT&5++&p(PrMiYaC%?U5_T|)tY~xJm>pfqG z#Bey+=HQDF1TaU<5PNi=56BZDYlY|Om-1VirblO*@Ub8pGdUrr7<+t{R75F#bFAWZ^s{BG*0v4Bqt?0NvEt z4d{tmoU)JSs4VBi?+Tq#$$)W{G*m9*wI)o!j|3mfAE%aKm)TV-(+-oK1Yt=$q~S8+ z95>}f*3_WqQj$Ac_!<6G{OgQnC9A5VZkWJ^>;q1!U{9?`dwIt6AuG+1Li%$~4vVms z&U2cnF<(u21#GM^7K;;odN`Udp<6-H-N4JM!(J-yNegF-orHGV?5qLBk+4EjVdE6G z_BaA8+Oo9r7uPMNVUSs?CCDVwSQpx=*sj2z zY{NaPCjkmu1BCTyOKS$%q-0{S6D#46$&cYOZiYsTri++;xIqz`LTnA8@xp^TII9}N zGoL}#au?Wk+LSq@-i=`@=|G<7_&*0dyb<@tQL)f&=eg?=CmY{1Ce+N+*RIw{SoiWt zgbUZewRNjf1M|ta5qvDZNe$k_w(_?P#@qW+trJ4|@n8ajKTFeQ<_I4RR19X`$|&@R zGj%QbU(Lr4km_i+b_mO`^Kob2*{6AvDG02iE?&O$6Z^+O;L}|lLY^o& zTxyt&upWLdmV{WBUb?wCeemVLf)kW^kjJIZsW-KcY@?;ZX>KrQHy2Yox+Ez0LtX_ol|3Fv_S&?bQ&+;pctW8GMSS6!@A^!^r%c#Y)9t3S1e!Do_espAFJa z+Gd+|bR9_!zfq!m@pRU51y9=Sdet6> zoH!b4nxtF&8edk57*ci}kL03Qg#9(Rj0U*2yl^bG%}RDzNfq`it8)2C3&l`L|4D}W zs_x{hyKFfZ1H9X9_&+)0Rm=|$f~!$z&YbcLZ}mRh9O&W8{^gtwWzJ!2KjoYdBJP`I zM7&c8PkpF2(AL>`O6q0#M4FUN^gl}LL(Tbfo#qxF#~*x_SyW-Wj3>vKA-Y{Je~LDD zOLNDQGv1Jmnt^UY)6=Q-STJE;rI>6D7^gWrZermj`6M!(W5etgVNVxcR!fX539Qpz zQEtI&!tSeI`BoeWvMg%Af2C|hF{$c7@8pN5+>+%?n;Fo?B`&&v3@ppc_4EKKrarDUXPSy-3LfT>%`;s)vvx%3Z2 zWrg5rq`<|DpWMc}UbzflnhusWoLNw*AtgeL#{`HLibP0iB@Yf@@QDt${Tjm+Hh$7^ zh)btpa%EbTHTUWA^_nz{liab8K_`(RMlfa57VrGU3N5Rgv!kd6NOGX57C@|wZ zUM*wkE7hFK*1iK%a5e`pAKiE8C*raSs z1*HRA4aSUPsie5o;rvwnQSAh+&J-apy%F1=L2nI>+n8U_r>fhCRXl#X|EEY?JBJoL zXg+6nbQDA(nR7dp4hHu+5iVhM2ETou-r#=Un3AHZkijXc+~Q`OPCq9 zb866D@Er4{2xV4kgL{8A^El`HfIxhp^i6VG zwe(nR#FSnj%lN4bb-v!(BpaMm%D6tY*vN~@cmvxLm<~}P*%Q;$67zn*M7N4825uH9 z7k$$ly2o&S1d(jZXoNAHgwrx-)A}+9HIzwWgcl~OMR$TQ{K1L^^`0Yrz$`6oMORYJ zkK*{wIv)Vps?KZu&Wo#T zknbUO`rE;nMlgTGo`MWwYh5VoK6_J!Wh;i{iUWV++-pJEH;;_KS-SK*O$qZ6@3T6c zgJ773IUF+qYmKRnw0sSOh6nb|nGQH-s{Jnpl*u->IY0If*G6Fug@aV5eW`zMZfQi| zf~rNPY{bX;m}@LV?eZ4wxx{$YFh>%glUCz3!?@Ve_d8^nIMVI6c@NZ1VD3ogpr|ws zkTK0k{v$#;jj(vPIt?>L!m&I-i1ECzh!JKZw6c249?Td`2%At6s(=XReN}=E)z0%v zoLuHUcRFI7Z)mWOs@H}}s3hEyN{t268!J)fn3_yYhEsnx`I$*vk$wv9hQBBAgV471 zq;`7Iy}=lxp>iVIWd@IkQThT@oG-@FOi&d}Xu1E;;++r((i&Hcq-~;poliU9G6ii7 zD9~avT6v=iZMEV>Xn)S^{}wKc>qx)diW)C@QYaNe(ZU~|m?9dUtGF-wJDmdx*U_E4 zwg75!Wz6oDe`20Nrfcv^sY_{KRR4zRo{a%}wd!3J_pY>K;WKy}Ba3>r`@w#fIMUzchL%1-fA$pr z(A8KzndhcW#SwO);!-8zpS>(7Ps(KA@Q|I8`v8rjB~Q3nF#nP}jE7n_hAK&wnqj)pCwp z?FDZd95pF^YpqbFhfXCNo~C2Oy1_y$BuWpLyjmsUB!hAot#j3 z8IDb+vd|;g#W${UZYl!|Mnbj5ye!}Hf0ul>C=QaGhA0=l+O)Mh17k?N{MPE>4d=R= zTqU~$j_=wj8#CabgMooD;o1m=fF9DC<;ysLSK9e3<@dWE5=m_s)0eDXA;YA^Wr1J! z`gFepI0cn{#;3%*PxLhL(=|-b6~B{)P(fcR_L<9Ea%Hdt?$Ln4>eMUdo}H)s$Q-v= zsHE)R>kOE2Degd1vQ*86blSN<7*+Q`L+q|cdaPt=xK4fq4D7krk0px zPEu0Tu~#VV7Kx(gNM(~VmRisq+=9gvG{S2D1hQt0=o~s>Wq9yIXHyc2p8Gu)`ID~-RWdW4TI(nE9^k*rg<7RGbEXsY zj4`S7fEn#~(=U>sj|AmQQkt;Tae-^*L80ntjyA6+-)v^xgDG2zxd$wCKzAv6wS46! z`g(d6Om=6+P;q;KHX%oIE9)ngMI`T%Ba{i?3_{(a&0;fJNYUgzpT}1^ST%REu5e&i z#32+dluau?1vWr!N4zbTiU(j_wiNx7Vvc17T0CD7M&$$ODyfG8YM4^~8)?=wln4_I zb7m8gfSH7vdoMtB>Z}haoFYm0JE*rhPeQ5ntz@ie#1ax!%q+zDf-98f5V3?nwc{3c zlG^>_A@HVut_-O)RucvGqo=!J_!ZAazvD}V8yT7-QsH~rb*xXBvwaIjk__@9*^4&Mv{?;B-c;^P z+g?Q!&3KZyaNs$_iF6yBamu;;6OB-^Xn6zau=*jmh-65qVAV9s!l1qO);2W=7;+Tf z44aXgD*mF`;V@g;LyN&y!Jd$KE%P4jS^6X7B#I_5saF*` zIT$+u0e2!YjXIl8tjvqnE{V)xJTHG=R4@*E_C>2^%?(7Q)!p2Fgd&Ec5TQ>D&2ju@ zIp+rvvX4ZdC8mjT@Bg6u=KJM~Wo&uc9x&w!f^QRhjR$l-2Z6!$cW+i!7lC@;UkCuyh?jB>??8t+2`qF_G~=fu&%X-?b}2Na(I!b`gc$nO zP2^LxAxcvtomgAV|IjO=C28Ay&vwWlFfub~J%GDiEl9PIT=J>oUz9t%ef?M0Nal zJCP9)Ke1y)w8GI+yAp9aU zqi*k8;M-QpW-uNzjFAtOS>_Bw?zg|T9=R=;Q(?F|)vo@nWJI2mMSM<6)Ah2leNu7Z zx3;(AXNFQszWup=f9v_P<#jN=M2Jm@k6(nvN$4c-klw90`N^cRy1KY@#qjv5%}{>X z6K5F)Y4<<6q}vf3ypg)!*jXe6;DiPLs5%fVybTb5DiUo1R$DMw{&nN{-78Mi*=Vpa ztGa<_cFzZz0&f9Vc+3VW-UTYn^zMkj2}+aY@ChHsnTkDU_c_}5bq^rw+2*^b};(Rco74Dd=+iSLdZ$B6$9p!f$Yjzr83!W@}m&N(Z?|0Tkyd+?@w z85}6i5p^WXEg_7|=yyd+E|@QL%O&Xy6@==awyE#<0k0Y=$a4# z0lSQzzmEgx3u0@9v$?qOInQ?z0h?J~s{YTrA|j7C0g4{%FX)1Orr53f;<%|1_+DEc zUdwe*u8Hs{emes@e1aOje~G*nMjmPw#XX2Ah3*PbrkpX@f@C!)YhyQF$EVO>QK}T% z{l;PQc{A)eMk?6f86@I&WzFW`ACv`dLJ5pi!aw8)g~|It$T?CK_Ewt zP#N6U|LE}PD=XL3fiu&)VN(4SIeDfkq&(_efM7Ok9`JsKADS2P^xzGY4Z}0Zt{OMU zE|k5?M1d*BjG>Iob<&Ajp!eCgJ4oU1TW+Mqr=5}WdITwTHS)$P#JCT9qmDjMw^|>s z%{;ia%(7Xub-IX{Nl2eD9#4mJxnNj>S0xnCP<7OVJCNHufpUQ~`=|z$2>j=79^=)N z?LV?;1H>WxcSzTnejajmQ{F*F@(4HL%kSI?wKF6pC;Krg$@i08iz-R3QA?wN+9OrK zY$dTE0K*$yd(%qkZg-*!nVdm=S39JCP3*jfQfaeA*n(SfTHyKJ8i^Iu22o}9{dI=` zD_-w+(4Y zP&j=1H2R$a8s&V6CLvs{XxC)?D?ROCxnyEd^UlHJV;hrgps_z`H^G@;2|MeF44%9x zO=-bHv8D>*e!2>35h|J;;iIlPCAVIuF`>rS#au=7-n5Qhf?DC@gB;PR?_EC=7LcX0vD>;f)}3RTs`$@U6fv~qtFkaa_RXY9~K zBfZ2W#W2|uooq9o=}ej6*=SUcGC*)Ebr63;HX+N%aDAe7_Fn^yeryQt5g}=(p2r|0 z++$!!nr8wU(CSLv*jYvwMvl8GsF|pfbG3B)HU{BDTWEiE;moBQ4YCVD=OpxX<-yQm z+UAs+3g`GJ_YsK{W5`?x3mE6jG&OU+aZ?xR%t?;ot40I)iK#J05f- z^Enlvt1igDW7(`TnWdvs46?l3em$9TE#=z+Yq+Ui6|a?!;1S^J0|>835dqLfQx`$V=tHiLWid=9V<{JRV!jS zP?m;}e4G4M(KQ~a=_WvL*}TSlz7Z9ZVra;g3icgi21{CIDEdbQGza+JGxt2*X+Wd_ z6@eILWPNOqby1Y;29%)WAM&|UdQaO(DugLzb${of*)c1S=~!o4UR6y^5>?~5I&BTLP^iT2n5y%G+i1I)a zsH(JHT3X{xi>LcTEk%D7BGDj&;mHbF5@AaB;xi-p@8E1d@YKUZ`g!$4ciGa;PQKv= zTp0F9vF8MRE)Ngn-yUyY-}cSz{Fn5)^@~ji^M_ zq@GeAv9>VqBNE#?*!1^yQ62m}gf+`HJ;cIFd&Z64JW7}m55mhz9N&W-u1Fa{$yT8j z{-cm;OgLva7Z#dX89jnk6_bPo{a&za@?{8SNDjr_tx_rXjTMRNWOz%ujgCC;N98?b z6FwtrW|eo91al#{tYL`I!62peVCt?VHvl9%zUXmg59B(#+Uz=5o4#&qTYBnrRcLbs zGj*KZgpmI*YqE2+qf}-a%#V}6!K}8h+ARsl*3n#Zw9?~qKOrUS)*YFd>2!jfh(N7o ziZ2GEBRKG_9d9jSU`z~~>M%(?aVV%(wuT=3aeJ#fwx_Q}6dP6dDLy?NFZG=PDHECd zpW6j&(f)9`ZTWO{dT;?<->K#`NzUsYcTA>4G6Zu^qHbxq!BqDX<1VYh#K9z^no%Y~ z`W)AqrD)_p(dfQ`TTlX`1C|C;wH%!%N8g&hX>Cjovg$|>VdYpG+*znBt+u=cc7A4F zox7(Fe-}R=35C93-7}<|#h)nGgl3%#3#|wZ%>}e{YHA3&z77O23MklN@^3l(;+?lA z_2N7;I3f*-==yLPVxD+ocIs`@2r~4|9m2!{LR=#j5|_A{wl-A53?o4E2Mjq4LHlzGNcxZmbbqdom zZ!KxrQW&rJrgnBc6;(~+v$s?CT+~1ycoTR8BNDDygiQx|t;Mt4K+OOv>ZlppMKR+O z7XEHS#Z+B9sj{3*r0LqI)Td9M?DezT&aH*TFI_v}jwBh93KKIgCzVYNQ!tGkpS0qU~l5k3CU@Hduv7Yq*aCL@VQn}O$7k9pYeI>lB z05W$09mtnIS1t@?JGM+rN@`*Q42v{62dj6#fdM zqoV^Qr5>AE0+*GsA07vW0c(d+*KYAu{cJOR1IO$k)}p9a17NP>r4MIc9kEs z>>R)i(}73^T5~l)C!Z3m;*(3AH70%@HYWk437E$<$H4l;w0oNR_^ z5uMEUqor+w_j4sx!COH}LkLE(iM_)8>Tn3Uda7O8%quCSEq-uadpV6}_3SjBLzKCY z<`F-s88j`z$xQo(>-($EN|g5@i!NtD)? zkn(VG$z8(fm~02ah$j>-t$Ohl81d*MYA;Y^(4egOif3a69HYbWEr>Y_+7h%MKt#{IE)itDYrwBiaI99Wo) zl|&oBE?oITJ){^ggQouTAY~oadoOaV@9olQs8KmHky1n)=DB93nHin;<*V|Kj~ORnA3*Lf@j_^X#jy$! zr1AQAL&*{PB2BcdrnL%J2mfS3caY5n5%#kYxBKkmYM12wCOjO4L_U1s-SeMI&9S1S z#6>)xLRY7=uJ1l zRitl-Ikg23_>xLoXrcD!rMi5Th!_G~QL!GlTun{1vdKcSKRi+}urRt*#$KDP4m>{z z7j4p}h>tx=b_qVD-J}9}{bXamF+W~!$LHFXR|dD_w&NTIfYZY@uiV|K{Xb+ypDF8o z-oH!(EF!FvB?xIczS+}Gr9KDg`<6;d$T`xvd3k{jbd{(=|ZB<&H*+0w!p#5B*`xx^y=KWS&YQ(l|rNh4Gqs*ajlxXRSA8qvaI_}mz zs#E;2$?3M%fncAPg7Op>bOAQj&a{|kHOS6?%GVUw-J(X)kLFah!wV{Vt6-DEFdZx$ zP{JB4e)sXzl9LYRm-SCcHSAjKw9+cFVJ$*D^BT*+XrdB_^l6hPDzSSe6!cc9jn56F zandCC(?D3SXJg)-BH`BX2kL$H2w(O59cKoMQZ9DHDubB&GUP8mf7?rVMetztPXB8N zKb-35J3Z%s@Us~0O1bw4wwet-oiUIx!c?Jwp>5LfJRG=b2PzfAHG@)8FmjaW#>7V$ zAn{0u!tK_rc5_&;tl-iUhH3?(G3*YL88Rie?$i)z3-QhH*oCORr(hqMc{~c|%rL{= z5s2z6+sQfZ!FsY!483y+jKFzP$Ej5Ex0$}M8aE&!t##v0t?}w8gY2^=$NW||HT7nDnc7Nzb`E2XbAx9KA)#9lAtc=`#7B|ir5^c0 z@1JzD8DGz+&c!=D#D{&jKNc+rb-atuq7^r$edsPvJWN2dl!=h&PP}UdU>g%BOE~?& z>7CvkzoAN7J&&$%t_*wa{c2TJ;|dX@Kg2N5^w~*J#m1Z{FM7$|3Wg7_bM z97r-|PoGBNs~Y_vh5hfVac3qd*_NG$OJBdTFbQl0977I~k@72D4)W;uzmoQR;<&~7 z6rZtfFMo`N*X&I;T5rTz)s6gfr7l-t@38Rf24*W>vn6t;e}W)fF(_1H9U^ftXEnQHBh?47?(F2#%=!XbDb_Gj&ett<2;oyK2Vg0syoaPWkBFdOanTER$ zfuy;ZGa`lMK6Zh$TKr1%WeTWXzY}`~Whas!3@O#|I&$$1)1a8E2W1VVyi&dpZdXrY zXycK<6GScccM0*}iQl{3qo-hlWOT+=$b9o8d!}i3T!WyPi;cRCikY2h$Rr@IN`X3#CTK{)kfJ<OUQ$xeihi1{L1(e$j4zP;;z61%1v2fe^oZvnP0|+r8X{&!aN$%>;0@KGu^5;P z9B#yP0Jl;vI?iO&@IZ>TX`5Dl$qgTeqBPy~( z*eplgXMP|>T?l(!8Bk@bU$iuRQ8S8b3@qzBlEJc~mqKN&PY8r2WM_y^oDd9om3f>O zEvX%M0UvAK+R7tNp(IL+>iHWttGs+Z?y~ZV{x?Y!ZknSKi~FB7OhsOIxU!4Nros>-~j zW#Cjan3gO5)F*vlFXYyNt!Zdj92=q7UnBst)CXKA@|4{&-)ZG4crQZlFt$vFJCClf zt}d?6SF&L&1irlw)*0%|ZQd@DjjQ7nRj$;v9^}E%{3ObNvA4HF{i;KPx9*@-;QJk$ zodd%>&bll;ELnzJpcp+dh`U>(^~K~rz*LTj#>Yo)L*cy$GL+!Ju;r}>!_EZvRTO=& zjS6tiTS!-|%JDR|XleNZi!8a)GMB29=~3x6VZ3?0DL~!Ho%$&g5;C~U(?d+K$mn9* zp{)v1++P@2GOM>uB384u&Nnx?gG-Aogg~yT+%{4`a9Nr2LnD8}bWvYgCERar9z>zd zZ3gYIlQ%$6=70xLb+)znn(ZYq!YN6uxyJ&f2N&iQ&Md!V?!psbBMJ5mf<7iBSm0|!>CzKGOoB=TU)EM0t9%oHL35PoY2Z+ zB}Gr1ZomK9R{A3No2EpZhHiWrAf^gG%t_Sz82+A@rBr9$hzqoMFSfc$z)ePk^Em@U1Oo%3WS=B1H)pMd%Hi5V|b?jIF)>J|*bPMB9sJSqx6t|MeXGfU_B)kJlm% z)gj8*-~04X+u!XB)8U67T@~4P2)o9}e5J)Arg)>xX@iFJY&vWODRja^gzPdFA^e>`GtHVYkWa5>)qywEqBn z@uyr5+G{QYMszF`%NoZb!6VwIVztz2s|Iu4guHuHdC ze3|uk88QC&ZEU{Y&pt>i{lpO6^$RvKHs!q^mUyQu#s$VfD)d%k2q5T=F;)uC0*U1| zn2{)}C%HDel+l*~m@3f9?fjp!2q0*W9$6S5tY`CnwSMtZcT)&$A_+anj-KuVC}L?^ zX@zH69q6ZJ%qo8l27h<4Lu^+^oVfC|-kjk-WGv~So_zZo+{cmly*tU`O>o4-Ss5w` zd7u0&ny!!E;L`nVXYS_9_Mdmzmhas`Qf&&+l&V=^pG7{(Klu2C*;siGRC%fX7+TN? z3`4JRQf;YU>Nk}32BkUP)mm7zye$~2bE2|?Gdn-iwU?@qJImg}MMBq*kMDk`9 zBt}D%d2ctm*N;`2i6AZH%Zps8gYSk#TcrU@$t^#aY8F^Z?7+^y3kKYe5V71M< zF$RzwMSLHarVZxpJfmWg5!=_-1x!3(veHW7w6E`g#$lp>ripL1kRM&GeEd^vt(Wa% z9qEqaiK{x1mks{4n)3trKF1ekdIDW)a90pn26Os8lvH2f5-b?zGS zzzNkC%^KIyu~9x+AtA1rLdi$sVRYr#vAE45#7|xQq7kuQZ6|zMJG*0WtkzWemq)C0 z^^~6{CM&xYFPj^_f`QlURD`BhRvJ2BOA!BU8;k47janVWlE|v`>lskuUM{}aXhi)n zBpu;rQLKgSSA)-&7gV@+vBp8oUm40osZ7_%#Dx(b9~;+r5vKRqSWzO_lEDel{L%a` zjP0x;|KL!<#6PsGBdLNZ7~5`wmf@BN8?l=ekpz|&I9R*&BOq@g)BBfOFE3qD(zfociZVYy+#aK9lx?zuy(h6Qy zSU^{jRXA;n%iNMl;OF?F^V0;59RrdyN-6>_Qw-zek4J!kG}R?^6CNu37py9&m6N9l zo6IdKUt{cTjlSo$EwI(OzlZP8%d==<=4Z*uc+s{fb^x7CyQ8{G7bl)pYTEg1hU{*y ztXR{_v`$sR4SUlCqlNqg8;+K3p0<+x@*5Gs3}4bF+nJh15b|g5-6X>e^0CSxVRK?$ zDHSzFB0b}F<2bpWOp2 zp3@tR6R2nD+4f73HwFpCoPi1_hBSMGbFLVHJ3SK=Fw0l084PCaem{DAaDWMsV@TYl zej+ew5;IP@r<&DM3c#5P!v^0VvIX(Uj=4=Nr4xlo$6L=KBfyje3K3mv0?Eks0?-_N zC1M}wWNPepX=%0ylfOg~q9oxcg~7n`fpUD`DN@1s_9Uc|BAz_Uwk%N&!{}F8v|z({ z?KM>cjmC#uP`$`okYG4GYC5pjh8OT zteE*$)_jsGV{xhg>k<-*8UL=yWGKLlov&K{fV=58<;sSwPA96Wi6zbJoibF|qYw$j zf_tk4TU1k7IbZB%3VR~XPB}S+w~q@)&yYPQbs)&bRrYm!NK8{XwWYOf(1~4e2_G^X z#xcL_$ZAp=Gg^z@ECl@^G<%w6^%@&m;WttARt7&e{h6zk)itIuuQyJr4-SC&gcBXh z)+)E)z3c|waN>)qMBy8Tc+b{d^(~hnQ5kzo|pffG)t^YIlk%elq zB6G9|+9L1LHQWN5JO$4&jI9m z-Z-o~qPoBCpxk8z7vVX1c{t%d^Cavx9C_S}3-}GYXb0$qq6zKPdLh%@P^IX%Y+{Qyco*Pjkmfy(8sW)K$6NT%@AUUH_vPDU$x@M&JCJ=KVjTJ@9{?7ckKl zss9#D- z#rv+hf+qWqDEV$r!oxGTMmhk~*hP)EwJ9i>JDKGRq5W=t8j@c)14ECH75uZe$Jh_- zt!y_J!RUQ$*})asEiHIMQ5#f)+sp`fRN_|+G0NUQ6*%A4y-R7x>1Q4)#9w56LUZ#* zf07FaLdJ|mwTtV`@F;KIMA8#u3kimQnbSSd?{zMaqM#`5N7US#BiLw>5XPwNSomyX zGki&Rt9ebnKNJSmfE!TykCkTlNB&z2d_LNJS~aV8f)V+H%O_%!gNYB?k7 zPT5Q2W(n7zOq6|#3iH8{n77~%Q4XS@9iK^^VluhoW@P3XePCb*b&TeDG+Vmn#n}?K z&fF?>0hh*A>;a!yXS4vVD6H-*NPKLRVU(^7<1i-aArvVgHeCr`(Swx(PNE*l(8htB zlx4<+WQCegQspTLh23F9lZ(t$ah?Lg)`|=PO$ep+Q+Eo6MwlL@h3Gncu7^njRLMv` z=H`e7^qH`WhE8chV5x$VA*;)W;?iMyUbx?lJ{p@-K#=nge@Rr90n&a7@#5_F$_VS| zIFYx#W}*9EhWm_fN2iFZRFrZ}Amzbv2 zohm5Z0llVrtB$?STeGAUoS7;`8?3!9ibsOk2q&*F+dz%Pb4B-b{U~;*_(^mK#1}xD zr#q?&t%XckADLIM02mNfRY`H5#TPr?-+N!eG;7m`TTqh!uzOUCGRivv`fiiM+^j56 z6!D}5J%sfR)jQUM39B=`B85ftx0excSE)?(`^@`52FX9JpW&mC7tpSqgB}p|KvYnn z9HzvS0b~Ps%O~^rg$Sf{rW*%Z+FtF6PH*TXVcAi<8t22CW`r(8CN&_uqzv-`?8}5Z z85!vmO9dt*!*@mAr&N?hsnV9v5vHJ^T&NbGMF{W)+TKVJ&;YtVN>C_x!bHtWMACFG zLL_GtO7%&B9;F1BoJ^nr;xB>LE_ZApT9SwC$45}07gyEwS?=XszVY+UWDap8EBAhc z=s}2Akyk@%5DX%&*J6?!gey8iG=#^8Z@p|gVw^T1k+l94*2;n)LCDsSZx&J155LD| zfB;~C@h8_%A(Q)#k@K*z?{%ur&zyR8HeFs~obM672&`FLO|QV5-6DaJg$B@I=K(uW zVDnlo`1Kim5cj}u)yj-<)Azgqi*28{{yUyiN8Lmmf(Wdu+`!%_pD9QAAIfOoh$5z z4u^dDdgfN&j!&6|>^qv?`*7?GjSurC-oepY0k`+_*Dpk&l#-ca+vRgS87e8c3Mac4 zK!b{~CZ&_qCN(9A@6#@z-h{V>%x23Wif!}o@v*hAfSM5H2WwIZ4zyiQ z{v=GNN~lzNm{%u7i!rbtFXAs=JCVF#Sr~D$K|M&$Dx`U2!S`&TYHj z0T_kNpQ}x;Egx%ZYnl~y%W-+?X3mE7)-UbYC5jQ6nu*g6+@(8k1v0B5C{V1v6vGp& z{KAFJl@#RQyPnryuWKu-QM9GSOX1>Fq%tsE$^0ihEGZ|Uw_V#Cff0u8hxcn?I0+sx z9@eB%$pYtBd$PuodLd@zx1)Tg@6SgyP*53S1qGsF0ns_4aH-Noghr}5TwLuZ;*^&A z+bPelEXJ}894+00?XYIxTS;djWSzu#L{6XWrxvh3c)_keeus(_ragyBYVCdMdmmkr ziMJ{wgN>VEEbJvl{^UXOwH!t&Vlzf_mZcu`q5vec3<{Xj^#17?yl0(`D8FxnH#LQ~}XdXf*Alw0}8$KY1+@F;Kv#S^jO(Wyor8Si&8RLLP_%NLHkD)e2Dzx!qJN z-v45r@T;n(ypUP(ZSVI&H!<=V30jJH9rSuG;d{Nd=x1vepso)wZYR2-VWHbYMIimun0v`&y4 z`;j&fhXB1h5NmCSJIn)hpO(PWqL|zlmIJJ1pubc7BoRaYpbiNc8|s8IiCobL4`duP zg@M?(swiV!esSf4nXz9xFaTcAUu4CqV(?}n9;fz zMwG7tw-h`pPoz$5*A65w4E@Xy`Mqo9Zi9v&h$`Z8icZ6V zR-!?#*=#;EI|G2XRY%TjII-iSV;~N|{?tR-D%+9u9!Q)8cNz?>5|R)Skcd=i#pJ|% zG8<;neyRQo`+Vxa&jf@ZVG+)#l{b4&enD_!nJRMx`%euJ&n0x6ImZf+(pP&D5d~2f zvC*Z-77q&KI-2+ge8kFxJtwho@C2GH6^V~xN=vb}&UQ3f!yC*cZ%H}U9J)i0wT@Z$ zAe^AO+7cpj#JXBq#(Qa^FfNnII;}j>ixp{OW9^*cF6r~kv?P3#7&{cC-T0n2#1w~# z4l^R>TCPbClf?Rb=xMZhb>ZEKJD$m9CwD{K2?KGmP3ogv(5=n;Fsk!Sp)a22 zf;O4=_ih>?dzDzAuRMK-DGtJCy1%U~ajn7v1C;mkb!!AbXNMd$F=xreexx)~np#=| zos7au=Pt7uu9K@*#;F>$F&s zP~mFFlM?4(1_$wxVm|v1^P2g;KxbvJfiNHh2EQQtgfz18eTwcS#At5sc zU`&H|%dy}qmzX@-eJjuE&es3*oXybOcTf8Nc0Bre-TS@1YH<>0ss4H@`+Oq#*zf+p z)dJF(Q7I{ySMfFDy>UM|nS6Jx>`!qFoM+TeRo7HW@AVl!0!?AbS_1QA$_j2w!bS6t zuqioT)TJ?J^J94Uc`h3&4P1G8yLsVvY1$n$Q|vPrIj9==PaL|5u}Y%Mp4QE#R0@+D zM&*l;gL(wN7I(#JjisiNVAXxxi{NkqDq~tqw&Y#Tnla@WtJbirz?`#*K=bJt(4)|_`1cbRYhiIVsn;ARW9FFD7LdG(?p{ROzkQ)l1rP^J{xf!6`7#H zfbJ$&tHDB8csL*!K0U=tL9ON#RIpg# zwZxvB(>G)amx~Z~`-)ymk@};Qx!<7km}*#ldwChb8Hz1O=+_t;?m=;2 zZ6{`;F=3~#WR^5Yw|qUy*Pf)x+Af2*_ffe110xa$XSG_jDVv>D*5%y_OJeBh;adpw z0U3mY!+XQp>Ifq5l7P$IK)yKnnW5K3=aNVkmU*cmz?{w(GX5EnDjl%|m^F&k;!oT{ z;Rbm#tSpl4t`4F4`M3`Kj%V?w$(G=DsawW)sqXV)-U43Fq~w-*3RqDAu^b5Nvm%+X zLk`Y{0kk{EcYng(NP?cE%5 z$L)lN5-m;W@ByT*NvSE2u%P~ZcZ>ygznrMtAYOKoXsx1nB3NjIE_lW`{1-t%=>Pp7 zD1gVqAu1T>ca;bH*QfunJU}MoS6{pzve5th#Q!hHo0rvp_h_q-HN0$l+yyPQ%1nJs z4tagzVd-#BuiiY!st7w-F>bea@_r)Qsp6 z7liLA!20WkU1|Lt9WQEuUTbJ}SqzeobUCQgE^@$kA20WkpFF;;K1iml`K~Wof6!5FV~`*MK>Q`|hW# zhP`9h6W?7h7lyS7JAC3HvF<5#r|?fKk*ci}zmu*@78*mPt-Rg$u(OQ6QhtItf zRSjYSua%NPkeoH@V0~J*uWxN>+0(T#*pxMhqSZBM`hw^9`SU)=dYw5SQs0LTG{?%# zf5w>JW^Y>CO~=SgelfRAMMZ~$IID4PoRX3i>-ps;4ThW$5q@{GG5UQCB`b9234zdS=9-{gvmOf~x0>N8jue^5k&BX|RR6~}?= z$DO&kq+S98@N{wiVKx*_nb4ii>|HH&^}lyhG5{RgoCzCCf$9)TyJkpLCaxPIVx##> zJD0Gq6woGR829+!9sFJ~rg4D^)!COpwy+vu>gl32e8$m=^F>3kamU&0_|jS59UT;-fF>Ed-xOAiyy7^bC9IH`qwf0|2o&&ME6jFg3f`u0j2{z*KvXr8O{(y{k0qy($m$j{ z_L0PO>thAw``t~5yzh(TW#vf9XlZHfg%agzoHrD(-r;ep!2?5KI_l_GsaotHurbMw zQFOYSx9e{oK3(lyU9tE z_xEd(=Rq787}&;8%IK6#1}M;b`p1j$H6W2Ly-Sjd2WJ z7(UJz1;5SYqtF~j!(*NQxj8b$B6ugOuFk>VQ(PSUj?Q}}iz&hC2R&8D<)WmjuFoOD zGgi_0eB{;96cz)Vqa7xl1hVH?C%^r984}C7y@P5CS3k4v&Q9pfm&i~4ckh3%t?@ZH zI4G_z3`!4(XQ$7G0HrPvC9h0h@MJLK6SYgy(#4ewOYCi8VS$-L&mK50#uxtS46SNis$pV4Y(`rqAoTKo#Jm&7;#S33dj6Z2O;r*_7BD z9>IK0yyV7YCSmsxNE3>BQSpgrXkgu5o#W->lIjC1f3iB#)iQz+a_YVv1^&R~r-_Cm z$ZreeB~WTO7Jh4%e*uH1Wo)J~>7_Uv$M)0Hm~;OLiu<{c%!GoHBVRkd2cfQX|LpASbQyz2 z+3)3NzW?EPVhrJdwT6F#6*_uEWG?(Kw;w`(m)nh$IH|2t;Hu&85UFFTc(!R>*;f}n z6Yrs!P7^_maWL?EytH&QONtgWCb%QH+RKrjjBs|ailC6ESlb^Yh%y3)X)pVAu=U3g z2}%3WqbAWWFqB=__r9F{8*^-seG6fM>3HP+yMeFtx`)Wpv`lfnW3~=Y7Z-ND)*K#6 z-~9~U3?|74*2o;`)svtxpj}|Ah_}aJ3*-m<5c(kGCB!m<(JsPMbTGRp!gX4x7GLv6)C??~G zrsV4M6kO#0aP^i!Z8Z|y#jG?n$cL&%>__U^$-LKQn%nK`Ho8DNHc{O@Lwsms(#(NBn+<=Zl2}M1bfPWJ=LP30)@3 zH=&9N!VUy?6c2822GWku9b^WnM9O1Al_079391cR3j~4N;`NuP8E4c{+(wiC&sDIr zy5~gFf6Y7FQK6HBZ@CFqPdR~J3?WTwFT7#HE2eOKWe&-AZ*1fqooTL5uq(%(g|(Lw zORsTYVx}=?mzcz;L+puo`u_EE5nMavrEH#YI!%)SCeJj%JsR2xq==6jOhjgAX!9!L zFutPHJS2lHVaJN0t8Y+f`2kZH+L7`YpOeTV%m#;fXH@P4&G-9Ll=rie(Ke5fgVV`^ zl9%am(xZ!$FIC{aq@o)|P$8BHZRZa4hHTro=7HS;|QMu%3B6=w=9TXr+ z%OPy$WJ*Rsi3s!5B1l->O$M?T`*#~ZQ->{*kGCD5dacX1{3x6 zqaEIH*hgwaGdP(O32nT!dd1bvRX`PZOTAF=uvYXCy*baQyu!8O*5r6T3b6(kPZE-0mlltIfCvQSDQGClMYK}CPVnJG^$CFaFuDDZsx{~ z0U$+);^T}cUk>?dug{MU{O?+_5=RLC$a3bPEQ)|L%9k$~B4eQ*%sUtvdXy^{=ivU3 zYcnvD0}WRF9L<@=rL@(z%5ONQE;N3XRRC%;)>L5SpUcl3tF84^Ex2e=RTjFdy4Cl< zL~{L8hN`U1@z}}S4`bzR!UD-&V5D_K0j?h<5=|)B_b*TP7GHc;qp5~9EEKd59L6cB z^k-g%N3W>F$iH%ZrHYs^BPAs* z^_Q$8ap_siOm_5wR4aH)AZm``=|5AKjl2Y-`f%}N6UTfj;>Qg&D~wCb&Q=_)m(0?x zmT=&p8KxPWh*$_W6Q`|(h_@lg-KhR${2KnYpsPZYLVM{kov>4SCnJAXFA+I+*T64w zr6yCX>F2QZf1m#iiMw-Pa_q4k5p=aRj+tO5%mFY}4xr_FT{rE3Uu_$f4~Y6g^(bAL zxfU41p{Ov;ZrS&JN~uy9&LWY5FU6YPuuLfE8GXp^$DUYQM#*>KMNwhls9%wIE74e^ zTweU~rh`kV4i}D^kQ0ol^2AE+guwBqZHCb@hvF|j}yW1-@6rOW_97#}Zl zVp&E-t3Q;gGJa=;iV7+#;Rht;3@oapn&=2vid~bRp&j(t->cb~2>&3s0s}%HybhVe zoziCH_O&e@rGje^S7btXzeYHA+Mn_c-z}AkPm$F#lqDYyOCfzQ-!_{IueG^FD+}Os zC#SjuYEvX?pdrI>bc`E%98)q|=H`+gE4VT*Qo0AHi!yODT69bJ9i2d< z7e3A+`C{l<8mz7`z97&{rq%JelY_5L3@wjuCXhvC{#q=_1HrW>49dl=32rt>^~o?t zh|hnR^fv*H6yDgt1Wj$$hZqYk*QJz}7CbaAb_DJyb^ZwkpWvodI#!jAP!H}rp>D3P zJqMJ;a;u+Q@Pvrv%bcF(=DfFCYF!ePO6G|x;4UFy6VB)g!!o~AscID+v_Dvc9ozhr zhotCo65=B~+^CAfaQxx-(mx^{a(G8Fsn?LgFQBi>WIok{GwX@reo_p2&@j{maMLxg z>UYOUd*|?BmY1^@oR_2^AZx{Ka9ES22TDV#>y*}9SMI5+gPf?;(eagL>#q;mDyPJN zb`=>xUid6$FabL7AEh|M!0z)^3tZOwv~|H+DVy?4B7$i8!qFeVe@N*ep-8G$reI;G z%eU(LLhEmS3dQuz%5GaaFhwkplZoj~5P;>O>u53_rmWdx&R&G`=;>*(IN>fQRqcd- zd-f(WA&v$pI3`7mq=}%CpJ0}2@*ZWAViNIvN?b8UR2mRG6;4IdE*m(`$;Bh1byAU9X;z1XyAzi*k5QJaeyAj|5#Vht53KDK;_{J6PcZC}T861HT!0ORkg~Z{@(y+|J;=!(@P; z8fC=X;?ZC&(|96|rKlH*K;U+*#^(bMFFK5rJ5SkNHet*E-=9yq^!jPad1$xLD@bMq z6&`;Xgpdgr>95FTb)RpKolf(5XcuE>G&>AK$7B{i+I_LQp_9mPsCtpOfb+@}>{b%wR-$ zd0Y?7=L}6!I3z_lMF8gAj7Hk`By3E$Gr>gJKZEM%TF=O_=C$tvfndq5m}Qh83Mz?I z->YYGI#<0EwJ(usHm|%nSX5EURK22~fDSKqU{>?wBsldNd`rT2a0f#kp}Dc4f&k&g z(FN#pFLvxGb|y?#t%WRIGHu}#DpOWg?dGMu7Rralpj;v>CSe?jh?8S~faQJMW)~+` zx$B@*sodHpIXM?(p-bwy4vgP1uG`*;pL6~L&4gA#=rtiUL#otm9Any${|vKjE$Hh` z+kE{Ng>e5JpJbsRk3j4~Kao1@e+Jm;ANbShwEh_9hwA&E;fMnerQV^H{$ugKU{MF# zzt=lVj#uHSC2nk_-{qZPjd0h;<+4fU?qwL2U{2^2Wg0$p(o zkCkU<&~~T471_bMP-H{3)J2-@II5$uv2g&J>wnA}yIr+9`=L<;#>+P#;49(4+~ zb?8y;Wo!QhBRjYz9{T@H&?b8u-ct|}JuYSUuPiSQv$a3Q9BXlnlIZF#FZ1l+%oHRE z!b4681@PV;swA#2F7Ozp*XpTn;Z@I`#kwBiSQ-z$TXZ9D^Std*Qx>BPJC1fe2aS?k# z!vy*)(1JOMhY~DzV}^CX`T}U- z1el{uEjSh}o+_h9WtMJ_83t|>|N23j86HRSc#xzjn6}yBK|cl9UVAhGx)|>=K!e>s^Tg&VG)(Ryk?s%-%H%aM%U*)N}>TWJ}dpr zPK1LeM1+32)vdJ;dULarpfAL~eE#%)Co0Opt4(u7#p4+`pyp1+9j(xXnl#!aJDVJb z_w{siv-Nebt1_7NbpL+cekDEX|TnbW=~(-EchL+IOy2Bor{5DA27`rNIa> zuC4|8DLVwoghzCx0r#NIVJj;y7nbTOYP03bhJWWFjQ~3G!^~n`4jNLr3jXw!Zl9Ck_Y| zRo~9u9+jtz?8~ZYNMf4~NsE=$r?>Poq9^sQDNw&^a>|>E>f6BITd!KSp8^=X1=fB| zlSsAEk{i%Mv;OH{=?G-52L~lpMSlE!^$)grly_M!@?!PMF$%j6cEt0kk5tKSuzpY4 zQPDcz@6jN@GT||sPUA5H6+!!h<1Ejw%;z2>S(+=D?O-`w6Kts|=@JT_fgIziFdURqi$Si{ zX%&LUM9h(kApsAmyydN}rR{A(i+~&=b(7`fOEuCsgccBCXhCxCAFdu?90!4$#8f$M z?{tr^gG4pFn;l;v+X#u+8%k{*%OZ}0CjW!PFNq~fd_H+Q159ph)JSOr`#LcnHXK|u z>zIIWg%`6yhR;Koz~ zU0c*+M;oHSE9R?n9+^D?m*ng*lEh+?wNVr7(|`iAIg$ol8ua#vf9_BVEg#KX0-*2# zjWszAargU@oO!M#v)_52iqcUd7Qj1AnBNCp+_#bv-#b>YKS7`Q&f3E8sBD1u*f>XP zLnHeW!_aIIS|Rj#_+gMD%C*l*UV~dw7(-f3jpFge=qh_ERt}1sD_13_siDUl#{yKq zK~CI4G^B-v)us22jJW%IkAuli**j%Gwd-@W&nh$YGUR2?iGeCNau*(dn7@K0^k2D_ zhpw0GuM$45TvpKK;1!#7+RPk6hEe z3?8vGqs!s@Vg*`fHXbQISIbi52flitKXUSJ02PtquD)3{ete?z28W+S9i8)~kuetQ z^jl_5rA1!1{aR$2XoUTG|V(};OG*E&a_TAdof2# z8W*#~!*47`15Fc}uJpoDBqktnrp@cj{A~Qh7!}`2sHNdjeqk=Ycr*QGEp$z$%p8$n zSHUEK={NWZkKXP5o{|8@ss4;7g^sN}=HKL;EIO5od32-r9v&)kwVF2%gklz+Cx|t( zU7Y>9HkpcNF0!HKb`6R4g)4UI~e4@1p!X&k~H7C>ZmhnHc#}0zx`U{$9PX z>s>RM`pHz(8Cy!io$37iYvM*&fJ7@+w2=7MJI6YhE?xn?L3}vI@Z4j!Pp5@bwMt^c zRk2C8xGK5EG0NgAYwOvW8D{$Mz0iIXoeceA9VkfkyWnpyY*@4 z#`9!21$83YFvE(yb8z7zlBEVNVZmn1==<6!7}Eu9e_>@C4d(n)dRA;Xxf%UL!}mvj zegIK@Wr?Iof1(D>3&_)KQd%Rh8!hDF#Kzj2?Z+Olt)NJk^!1Tdwv8jT+TBFAYI+}W z3hEQw{H?o=TtjGt$U+0RGLQ$AqBg#doY&WZcs%5KDaTE^E2H7P7g32SszFl>XL+9% zAMEbxUlzW0=IHooj5UzDs&GuEu_Yl=gu6K;3UedxaWu228Wt;zqEbdHR)eKv>R1%md5NBtI6cqwPG~oe8DUh@r;dNrnE85@mPr1}QyK_oIMO zg)77DU6X5WIM@^_8!Gs9i;pzJhOR=m${XZ*X02pY@6?!B2uUUmkOa*g9W%}iQ0Z_N z$3+=2#016>WX_1FJP+{E%g&ThTi4PEpaCMJD0gLEF?zlU&*}EK2b9LXO3F&rRf0-c z7`8J;Cf8vhF!AZ>Q|-Kp2wl(!nHD`Q9$YnW4vtneBM_K+nrCjrgM#8YiA4LsIDkaf zim1gKJYpF&_M9C51_{E$&FQLE*u*=9$La+dkiVGS%~*qA+Aj$!djqGf&>LkO%p5<| z?e_dNjL|09uc{hp<^QfMLH-UnBlCw@nj1exHXfNR7y?Hl8*@a>+hqE<>A0{sFpm_y4ZD!e`;3w)ZOiB zY$ES|pYA`!1Cx$)_LUS`%TQEROLExzX5iuB%#=Xepom5F26Hco7cWNK z2Bkj4MpQ&uA;*!ClQ-1VAUfviW+qTuVaB!P<>ym|ToflEe2h%R<{2oe{GA)uv>dxC zrOE2Zn@Ru4TH{W?Lw%fqSt7Kf*&fmeNvu^uhj7~D_{ofpT0~$jdP%~Y0~r=5Y2Wwp ziwH7HFj@|8VIpb3#o<3>GAd1=GZ+LwE4JN_!s#qyDyil`psS*MjQ(;N=-xY*P6PFj zk|aaGWMy}kj_LrPPz6O#!~q$@d}1mht7Ltr)ao7FtWZYv2S0#C^d<)vF+HTW-g9B2 zS#~YHRgWnPnOjr9aPWj%3v@>d;O13^T*Emld`~-%fkQ;Z4*g;EL04Cmq;+XBZlAN z`-mzQ%16^n4tY6lCS!l#58iZL5kj&sp@B&mQRPRuI@;ygi%RB3^F(8#15lVUXBx*+A<-5en`Nw_M%uL&Im6 zoIEX5EjG7|jIO(RM)vHY3q~C-uI2j?))6cCMH{omDf~T7UIE4Pj!Og zYR;NY5pHx8u~A*0dqRd>2;~dQJ^5DG|CCs>G3Jy^u((o{CR@W26{f-sd9*7V@9OYV zo28nJ;7yRvhzqi#eA2<&gq{} zz|W+(BOyFc)4SR1j4J`Qp(c1N5;w|1#$&9v z&#V;?`GRDo&XvqjaRw-(Ozu>Pb}+MgMD6&rM21Yw$Z-s3B~w`JfyW?>^O*V`dyN(` zpX>WtbM8Y6o2sRySh_h7g)cLA6#@aQzkRHca59%S|0eLMxZB!JhRcfg!3rDD*5{nn z-v5yjJ&L?98F(&EMied_{ln)Zi-5kg_tuGG-l zQeV7A(~L2gr*?I)dB+N+H_*x=Vn&)5Mrm_bCDKH8fuzU~`D~9K^CKJFF5^VSuL;=$ zN-P*k`8?nt07e#zomA_2#0zX0rZyQcV*rzphfQqPARG9}w|n!oRXfhsaKF+k zy~>=uCyO#o7A?#IaI&>Y`5`K)0tQWe>((=N0N=pZ_(s;BkeJk+F(qBXVmAfhAn74_ z@H|x#QqtH%@`S)62(OR>Ry<~@e3hZPwnco2zc)Ye!ZA}@EEpqknP~*ZFsV|KBrXBM zG|(yzthukojWnFgbSTjJKI>1Z#jUnaLO@)1;?Irqwl zkz2u#kFowslphT9xX}-b^#%BFAXR zgiH^m7{#rE=9s!N%(bR_6LbcP(9xT3xNuUK@;~#*N19nlTbbafhBL*087^QG*)7)d z%8SAE2gsP(4zP5i3b-;5yVFw0W{|qTh2^5CjQS>?%k23P3oBRtp@HG*mOwj>t!kPn zWyET9#>!Rq71Q`tcPO(pqQU+Tn|@K#vO7l3pMYffg%1 zHYe?|5&zX?yJ#e`WXaJYU>6v=g}78PQFD4Z>Lt%;s8hV~X!rA$J@T4+uCmC`4(K8XJVM9V#NVdl)vRYye5e0YU?3co8`otr@C zw^bb4z8YbD8ZRwPfw}meG|Qzj@oE2fX&cf$KT|RR!)>twVybHaj6+#0SY9PdDcY)K zGZo%evP9dqQS{!ywjZR1Z_3kxKaqXhU}q;LZde6;I1Sh18@#;ux9^x#W%)f(5MhOYf2w{Y^MCZh%fm?G$B@+Uew>A}eFHZbdIe-S~<(jJg7HFvKZdfi}5v5%T;#>a^ zBi}?|dp5I@e~DVaxVTdP!5FDKG(jchnN2}QEem#k!{RP>1E1A%mQ78{zg z5T%T)_Om+Lq^zH;LR8C!GnpT$m%x8Gi2m*zj%&WJ3}`u|52}mclZk`hPQ|a<3pFfN z2;-6~r9$9D(=^sVJC^{;wAd3Qm|g(!P>g0L0ETGGOM*s|FcwLzR+OKrLaAJI|81~nN%YEZhlN-wpA1jf{zC+lh&tY2CfAds*WX_`r-Nu(e& zjs^_ofW0XjXkY9|c8?mLW=}`v!l=YA&`t@vkRobm!X=rCc3s`-LrxvYBOoJ`DhsCx zd#H!KKT|5gb!`d7pgsj@=Ev0jF5DlImM~BELGA`%hizF~op_@gSHpV&>n4i^8*U|y z>hSlC{#xv2I)ZUX-^0+;j|+(f2sb0V=#SqrX$I4fF)S> zmVs|Pdp=tp(>O%JsuFJjOQ$q$*m?)Pgx;B)1@E{-S*9fd4C9nxTsP{P&o~8%D;hI1 zRx$>vW2P;DDlNj62&7nNA9$G=jwX$iqmXW}Nkff4P#4R!9qwLQXFBK+IZhaA5aL&M zluM(dkhc?PE=U4VDCFCL2pA0b)1dJgJru(RN&*jSZYe$2H3MRiwT?_HE66o%+3|5p zTimVWUfeDnjZLt`>SSg+iIp#b4B1*X`ic20sg2}Dg?-@{Um;Edpu^4->hFBj-sgC- zTZ0~htJ>OnnAN88i=i;Bwi||Lcw8)cRk{y9xx}((i#t1A&gaND3mglN=dc^!v=n#H zbl3RLaJsOCPtrpCNgrXu)0aMPB}bxL@sLA}uyd{C$U7$1ClSm9G^FQaIRP!CW8PwI zG|XgXCl>B$Q0?`O8uj}sMKtwlEg6Xku??KB%qfvZTverME669UV? zs_a+fmw86A#OK4Cxwa`q?JzdeF*ZS77EJD8uO00D)={6=NLLEC09F+Wh=whlXQX_t zb_%v z*M_F{|;uNb>=xNUv@2%Id>TEtIws&Nn_!#3j=rVcw{{61b_#S6T&8_ak^)<#7vNX>!JQJ&hGsY0TK*Vp`^sE^nMg~4PD(3d1vro~F|7Sj z?T$alD(4jI;4EK;`#$Rh^m05kg8w}&{mh%{+w#zTzSz*$&N@(Y7>`IA3XUrLfL3+9 z%V2ylMlQ~RFhKi(i&g)v@ws=XhSeSGhX)Q=*F}(G@U%AB0c3YrgSqEkz;8%JcXKu8 zhM_N(n7FBfUJ0kx``w&x(LE$)1Z(DHRnBEqw@JVZt0kH+LXMQc`Z9r-L-nj4izmcj z`<+EMc?v>Z*K^=|fMIlMOynq)?e0D$c zQ@MmlR*B);3x+74R^e}fZpW@oOG;XH50qo%5R6vIhyuIsW`SVS9Xttc(HRS@-@DrE z+lEVYYS2Cs>8-6Tgpx2jCIEg6=CEYi``8WUrl;Jf15-4ThScsgbg2{t+?8-jPP$JP zbKf+2|Nkt2n5?548V}QYL014VJT8XCG-LD@_c%TV0yUURr_55?);e^;DNjR-jAH?*(?f;ak~fMcL`7#Mm?=n^ z4iQV-&i|*O5ghW8dQe}Voo6nuk#aIK1ubq+aBMlKDFp=58+Viur$qhL3F>l%eTZ=o zr2QcHH^K@(@H8#mJ`@lC`r>kM$kW^`H0gK>pL##&x5=Oi^5H_A%CDw-s%j{oZMQP? zg;<-oM(^is;MixPM`C&)@y1EoIq_ucTe5rq0DHb(_m~mhREwDhq39`uBgby68jlL$vJLvRY5R{V zEEIGLn>iszSP0GSwaU10`Fs8M^}El&`N5#f#yM1Ey|7PIJrU9B1u_}tVWIqnJCum9 z`E~(~;C;M3CKk#mx{vfPx<8SG!ISFXl{MN_xZ_fpa_QKa zmqZc*!&HALoAUS6^bM+rr2)<5PmgA4qD^tV$zTPARs2m&#fG8ChuNpuN>K36bq8o)-ot#Ai^)6KeDN(=hK9&lAtMWmhIZH^SrRcwP7@qH9=a{F`O zDGT-2YWx`-}yaQpqQ77N$p8@WJ;GaB@%ax=3-`=djK$+r}5XA{i41O(_y@wVH>)6$Ze%Elnas@Esy%~ zXZFvq5iMyoi}nXiuZNAgD3h`*BAoqYsTf@OXd=16j`>=kJNNr1pc6>8!Ybj8%6`$| zc0a}w8Jp~F73y-^>CVMhdPe@QEaczc*@)1M@E6|a#m=GsC`lmNlp#>MB1hsnQ0YH{ z5m{bSHJ|D>|W#I7t9zo5Bwa?bmXP870s676tJU$-#NBg`$|$kniG5zHs} zsvKczeo4NWy}!7KX@ruJ?0`(4D|-By3d~lB85HnO9>xc@P#KLSG#T@ z#*(_3?ic6J2PjH;^FPa>K3Wrp3Kstp*Bx?K&6}kqZhe6xRb5gT`MJ^5&`&P6>KFpZ z3bqA3a%|XHZl+JDq$zE3l*UxN)3SAN+qleCK z9~VTmO-y2!IoZ`KDcx@9=z98>9ftT^Ub-!u%^$Z$`vienodm;b=9!Bo!?QEm+S*6X zNWzZu-jVGV%lj1w8SMJE-W}>O`7>FbWk1SpWmcBbpn@7jcB`{ENA_8pP~>&F*{DZf zEUg+Gln9)^RoVuTNaFJGhd1H91%ICBpb0x@2i#7nNK=|3A5diiR9|AIHd0+s46`k_ zu$&LZnPA*A4`rBq^`)t(GnPs>D7tr6eXou;$tR=FRKVY4xX$a@XJepkI|i-}uOlOP zY$tmm{or?o00}sia_R!40c*wW!OQuR+YC$?m-s> zluL^@S67EwS{e;{B4i`M?y_*@IW~$CEObPMA$KO1t*yQ|I(`|+KSQumNl6Kq4Xnr4 zFlVsudPvO=IZ{zD32$|1bWu;o(K?+;Jj?S36i-VMIoQz`LO?*Q5`4I&vUR+ z65n{jWrv~J+EgANb+XaQNyTTOV99Z+;WHq2X4einu&vWEG^ARhCrn{!#y2+9TwYv2 zi(3Yl5ozAt#`DTAoO1DI4C6GExE9UHsH*;a%tN6uV7b8Bmv;KnVU}i=mw1lzjz#Sx z>~>iO>T7T%h}Hnm^n4Mid~t28X6Axg-t$`;Svlex73Wn8YvwN)$&JTue?%^8-mZ~x z$0XE-Y1oaKMTyf&{JVD}ovW=t__&Imw98D*YecbGbU_{KT^hIxWh!XOf{R+-WC#(= zn#V$*miTBo(O;B8ty$9s1;!ub!Bp5cn3wOGFM~yIQ{~)_io}hBIuFEbq+hO&%8}Pq ze5*|j4KZ`&msR7@Sb(tTVqD@$H&@?Qw9j*x)jD2MiJf7d#t2dNp=?si5G$*=FkijC z^B`@A6zxNBso>(q0&m5vzQqom^bP#t0SjF`uH2YvD_WzX`?qkSY-J@SIeU1;K_~WL z8d8q2lL6a#a{mZY#u#yT{Kci6j1QBmnJ0O0Xv4oR75UGJ6YVmnh0~-X;n;k(Q>ocKMQoyN{WRJ{nS{@8A(%`zsoY|&rtrlU z{@*bb?KX#BRehe*=&7p4Y8H`rlox5ny-I0r#hBgj+)a~MVL#I6YdF!EJ<5w z_>O<^q@=OY#>zLMlPC*W3V0x?Ulx}TQ&VI{e`+L<3c;pzUPYEe2czVC-o|y*WXu?3 zeSc_AmmgwHr%;x%x)c?KXoOHQvK5z@0zghf`LgFr)6_|IR-N{M2S{bG5R6z!MFm7Q zO~5hAzA>lvwINI&YI*y+qpz#i3Vae|krfyJnm& zdEUL*y|!kvhuR?NSiF~s(;EYWs3~@a4|T+;6u7Y}V|J<8EN2Ir*18V^|tz^udmb4X(!kBA^6E!(8?CreS&hFG@*q=PUOi;YQoh0^y3SY8R2g~Tc1lhUXPKE zUmcS*myxH|S$jEOo6LxmEulLJr^&cmy;1?&7)4NqfDGI+AGo-Hht^wgq8ST!V1 zNWo|ub}vl|YGebD9Zp2y3*5sqDS6@)us?(8aCgsZ}{fAS4us=}0ALfih3FhO&z zDQ-cme_7Juh*FpzY(QLmOxT?*+`UZ)-1iX^lQZ>}uVk4Nr37;i_+UiDv}@9Dy=F%E ze7Hx`^O8zXnv2ncZ=)kpE-RaV z8)%=BL(OvldvW&@WHN-i4>p42|OT0eJ6^LY_G?hmrqK1dd(5pSmn40ceXPZQI$gofjp6X8Y z4?nbhdO~lE+;aRF{g|=6w#6+f9QRb)BJD(p8Eh7H8yg#xZGs4}3J7PMGgF~RQ-d{# ze{E@CV`E87Oi{ZQGW+59G1hTKEtzQ^deagKjio@Dwst#u-x(v=$OH#GSo+s$A%(y` z-g3b#I>1Rgs;#@l!mGc6ye#PAIMOjIH5 zHdav@)SH$-2WMSg%+PySRrU4jm5!;g#3U{5Tgb~e#je{p88^+SC#n?ar_?^5fI$2b zt$N%IqBM(18KOV**=WISDFYkwv+gg)GyZDmDPl8zHeKiwFxOIWFk%=cWh+K2HB#bN zBK8DnPgU>NGK!!ZXcc4YS`Ieacfae$Tc{{wY^)L*nDqJOsVF3OlGsDTAuv~oyOs+q zdOLD#`g)~LZ0VTjlVsq)9*H%A!!N?Ky|IFTU{wRhEX8#Y(|RHsX8k%iC^ZwoS{CwW zd;e+mbMUCvq0c; z@-v~C!C2p3A@G0aDc%k#xZ%NR?m%$^CFo&zvFyEik@)NQvkb34s?6L&f^ZSZOL6n} z-XHzRfhRQ4m}GDTmT|>VW3EIA1G-e)bm@tqA$JTRcZ_88g( zkEJ<1x(2hc+iKfMi zASPL~`c%f(k2RC}e^!)?nLmbmN4ym?3ORgZT|G+9TJZ9fT;aIX{N(jal<)}z?*6y^ zfA+rCeqUJN484P#+V93i_XYg6AaEa)I+{pbJv);(TRu7Q{QDFQ!#(;;E+U`bwIvs{v09boCWVUP_Wr|#4ZZtFTff&Xr!s!DCIfvHv@`dm`R~81WdTpJfD!{sIr0Y zXW(_|y}Sl~8V^`I{vAaB^9L#CshHl#$Qag~hGy7tG&fgIzXTwWU}~5nuY^ph(=y-Z zg4Hff&)B}(>oe_)`YXfivbb+>28HV7Un02M3qrcN74k1jSyVlkY*C@$mk}t465%(q z=M%@$u(s}HIDG>a(tvs=SY?%nuW+tZ(q%C=H&f_1wjCxDnUZGQPd>#kk*K>#XQ-|% z$5SiH;g>jLBee`q>Q$nqTLXwYxrgW~DLM6P8&Hv^G}<6in?UqhulHCqGR5< zYzdRJclu88Zfpe?SF&$7GYAPht_xOFRAX-^Vc{d$DiOU0NE($33vx(uEO7|Yjo0f@ zk;tw=w6-gXx?ZabH^`e)5qp^uxvj1$d?yN&-fnFN{2;ST@5v`m=1cov|1Kc{%8$#Y zPqqvI9F+QRH?24DVQ9etsB2MFkA>J7H#s&Al;ZQI0#llin< zHT=_?tnnpjR`-o{;tC{{*;W%EI4;IDsbAUAc!#E^(y$@?41~;2g8vIQWOM%m8!`gc zJ7@pr)K5wH@4$kV8A$s7fDO6-z=o;5uiO9YH%USJ2R1}1i6{L(V1vp(u;Fy?r_=ra zf(?=Xzy@hLit+z>+XJmiDA;iOn?1WNl}UiM!ge-D8St9kauYNJuMeB-;8O=O{_Gsq z@`jJ^2Ik3z)|eO<>*OyaKPsK2X~oK=JxBRx?&SFef_K96YD?G!buH|4{g{4^_%MXL zl-JJr6XalanHP(6JvbDHz|;FK>|lsv zpsP#wd(d5#y)!V;o`bc~U2*7jqs>9l@3Z<7v#zIi;7vwF!E45>>weA|Uw6Vnri6Y? zD%&e#wubvt?v!Az4q2YcQ8;!H@1x0>0~}$Rtrro^G2D^It;nV%u73BI(N>;Lk9!#G zn6Y7kU{`)bG9o~GUOGM*r75@JjT`npumiKNYYY%HavcQvF>^iQiZ{Pw1Tx(>V}oWt zVr2{6OZ1C%d8Q+=pLjsQ2v_&vTP7KJ@@2Wvm(!zY(Xq$`Ko{A3R#!wxsa*uHM{bEw z^ONfDL*vlFq>PloLidGl2Qu<4JHUi(^(macKOA;R+FdFBQUhKvRtZ01PviC=&8g8Z z)}ZmkIZT_ou31%yRTbMgER)Z zG$I7hXgS3d6{%~5X+!jNX(Ho2HGEP^|LE(xQbY@*Pf~fg4R|X#Y;g;55o^tdEy4b9 z@)a*1D%S3Y6`t6y7g#`bWnE&2Ee=!&koll{>3$opnty11`kUjG`OZvwwCCM+34J%0 zfwKsfJP(z-1iyy*OEuggJ;MrU;`z|X#d+`)-}zyo#q#(VD!jb;1hsun5!UwOWgB8F zQ4spMANM?b%q3Ta$~l`#S&&eW42~yC%4HM9HK{~hgM5%zcm&k6#h%fM0z9L;@kPIM zZS2S4qR9`I{_>qxL01fW+lz3+4lrKgNpJumXCM8LWq0{;eO-okW_D>^ArU{UC>SRx z>ht^e@8xd$Gt&L{-}Cr8$9cHI;XS@wi#o7@{*S;ZMNmpI!Q--RrbcUhMT8k!QBC;H z6;Xzr^t7ba-rnH`A9JC~mVtp`v2c#}8Q>oGm-qw?jRoDHvj=T{Y(Q1BxQX*rRlx>Q z#97h*E%?4sLtO}Icz$~c$MWm9&DPRglgAop-OEfZd$Bq_akQWXjkwb8sx`G+W#vtq zarC<}J$=m&{1O91nB1UlISkvmL`D!ugO4|Ory-XpMv}K29W$wk>afKSEWKT|;0X~( zpe(U(@XBG?uy1BZZyI{9LuP~JW{JDn7fpn0F75WPvK2Sq#IMmly^D4xCaix21`~(T zB!R8jKU>U5PEAc!Rk?9-s*;#fiDDqD7_R5EQ*5-2kkb&qTWS9#ejjp5Ro9k*E+z4J zKY3F;LM_$tk!Iw=W!7)xnFX)i)Vp#W<&{*wX$Nrf3fPs_2` ziZi%G1|n?Fovtz^8IBo;=K#{_a?`x<9_3>vVx8ei@F^+~@zJ=?%E|)2Wf;#TL?J)a ze69mr9EbMw_Ok5Fi3M~JGNjX0R8W#eMTH4vW@;k2$QLY!MBU4LKMp$#4HrLw2NG+R z?JDzAF(fLdJxKc{F*ER_o!@x`2BMQ$iAoERmRGq;^KymD@pHeRXjv_)-6t%61}Qzb zh5bVr6w)ywW3ntjP@_B-ise&mEE>CMrc{4QQLF9T!82TE6l|WoBB)e^5}&@sh8&BQ>>c|3Tm%7wc)RNF z`cklQWsBH9|jfxm!5~bE0VTq|_+7k}o74L7LB;Zls#Xly$t?+b_(r zhH^#NI5-v!p(gVt$o9@g41Tn*rA6*yha^G#lbkLUwi3*F6)QK`+In_64-0%mO-eUQ zJ2v9hu1pD7l-*Z&d9Y0%t`y^4S)ZSon{rH4OYP33d>l%Qm+oZ$|A>35t~%N-Y7=*N zcL?qtoQ(u`myHu3I0Sch_u%dxEVw%ahu{tg8eF>4{q-2{(|@2(Yj8qARo!!~Ij>pk z6c~ex(Tfv-ir9Abx>#pKMsWN!_C7bawtG{=?2jopSJ2IC1Wik$FqRG9*Zud}7}H8N zMV`%3C<$Rl1SIXH{rR7Ajx<|&K-HeXQ97|=+WYOmYWEKX)0aJVlWl$au zQ$Y2@ak3j62oDB5Rcy)52Tb?x6IMJ5=U?<2;6}YqfA2w!2k62|ib}h!If%BlRlkCt z;i9?5bn(qJ;$vcB;^It|o*!rY{T|<5fwNQ9SAIn+iUOJS_4NiYCZX6Tt$g}qlVNBV zFFbQq`R~8ni3JgxP{|dZIGMSqMjo(x`!u|5Y#Q6!EdYd%<@F;FsXJqYij;=oE&=5S ze&6t7s*>f5L@8zDCOy6f`UkV;hbSO_8Kgy~8;M*`9OYRV1&(4{BUqaLHP(s_`ujC> zmbIE90cwCiIt9?8JG?kx6T9-IkY4-xw|M^-vr=vX9uqSBmBKLa#?LZkGgrul60rgv zb<&7xtET1Uw?hY6shS0?sBt-Ic#PF71g2j#J$_7SvH{e05GZLvV%WJmjtc~+A<$z@ zot@cfdR&wkmo@DjFq7lT!-rqrm)}igk$RCzgS0d?7sZgSeSl4q@A0hou8KAL;bBPE z?=KE!A(b=NoR>i*?$Z1)#eB$utVq8)+J>dPy=DC0J89H1bQ5><;A;caqr_>;EeAZGHR@1Ok1OKD?g3i`r0GwLtZ0nshqgn zTh)C}kN>uWM<-mkbcEy6;j~bF zs{7?^>mlr^ufGLx@HuO31X~aI_7&W3crq%$vdgJrx6~+#c2?fTW&5+JP71LEjwkU- z+$9VjURM?rH0Z1%4`kE>rQgokS^5!Lnp2TEdVbd!Nl0|~BI*1%sz#7q$V{OqB?6$* z!GzKG=kRy-nZCG|xa@D)d^7B1@Co$nipQKPAQwQZ=fJn$8ks z$R|c^F_aG*WUZSI`)g~&{+JKUiJB%?9Wa57? zN)f=hA1FUO4AN|2-Sls($SP0<#rR&Qhrm*K$nvoM(B70LfE78yNLME{G&F2<&uP^7 ztlygRI7%5TR8+4{%#~~RBP%x(AU;`Yg?oNs|meiV<^K$)*2$cTVE0UQ0>x6{fncAc< zL1#LlD=pRe#xgg|9uw6_jnNrhWngEQa~(WD$@UFuJt2h0!1fTPwYjzaOal$>I(bz7 zDL<0QpPBE%V`NHas2oe(e^n-8o{ELIX~Eegvl00kg5v`no<%1OB_X*7To_PM8gLEd zdJ@vN2AxvnJK4psOsqW!Hlf2*h0j|RD_@+KaXv5kem6}Yr zIQ`_};gLegj3+1H5&lM%S|!q`DK_~&%uP?tXR9kBfdIT*Q7Dx}>IgFJCh@06jL5lZ z<-eNm>y?a`q3CCo9tvU<5UE;Nq{J~qeKXRKCUsF{)c@v+W4)=k^ z_OpDPL#SR)4dB#thijfpsr9I{wq73~yt!Tlk3MdUz}~`Z zj9nIhN`<|~$DMw8?3;Tq1vywqf6&jf7^SA!c}aMJIE$oyu7MSaNp6$h7OH@B%^bz?UA{+76M5zck4B{+Z;qVp%|f~@f_FQ zQ&gBl$X=l`k1`soDNwc`=TlcLp*b0{;XA;kAHS+akfbx26C4~oOPNjzhAj!6w2-nJ z!RF^KXRT08$2(C*urqz4bn;-I zVdbaiIMDYBCyZz?DDB~yAs7a1`RL>0DKS{(S@xIGftjN_=+0O0?kyDzwGxxLa&XM* z$ajH>Tp@f>TwVD=-MDy#5pihl1?49C$4;uVhlBjAIJ?TMCbt%N1LamtpS7*Zs;-?! z4dueg4-avw!bgR*MQR-nRUJ{Bv3^qLKwsd>Dp_6zS18Br^^a10q9|C+2hEJ-7f znaDemrv{mcgp?OfW8O-aOuH-V;!LnX`S_?E1*P9=wwAp0AKBZh4$l5J?D#T|*-i9c zfvzw84+s%++YSD&5dR|a@A@a12b=%JM0g3_SQSy3=n}Vs1l!AEnQkph z%i5;LH?Hu~9A7(6vxr!!4sEu8QQ4p4U9|ax^@p~T*ooVA{1pAI7zf=K<>Hs>>X}Sq z#n{*lNckGauM;iG8+apz<0YD@wI2_l3HJ$mNsF7K4;|m}LY<%a?lpEid1*dM1x6x9 zhys+|Y>X}~F?bhWpkl+o2saDO`G%b*khX0xQy!*7pnc(s{>ToU*dJX3!^A_!87gh7 zXl-xXmUdwRJf5%ZQwqv1_XIKcsavC0}e^)A%i=$a;z01TU+r1@mE)fBn{gqkX~r$ z=q~G2DCEzmKVtX^`ECCM<%lF0{B{dHQVfqlkpfe-wI%noSA6!cwl;Oc6>NJ#_M?E; zk@$PjPeoRU*?Jq%9?V7kUM_mIG=-b>KvHO|0>e+g>n^NqQE1s6dR259typC<+_t^z zCCyhGGmGs zmnvKQfB|w{+m=`Ae2S~B`Yv}49!}z}+RER@#P*8#y&8IjFQEk0$?!d6zT3_`96vJ> zY`-&Dr=rdN`8#_y?wk9TZ+T6xJ#XnFFH%_Oml1mSEqaNXJq8h>ZTDhHdy?UP;D{Q z-(V-hy?j&by%*TDcv7qbC=Wxo#Z+-QIfT(7O>FXbn_koX>)u|7#^7cSKO`CW>TD>G zRP7TA`P(-A=E_K$fur5ZsCT}>KmboUc+myE-xd`l#Q&8HmT&E&}Lo=75!qnI=9hYxD+}De~Ek0uUsJQ zsArGIZU?FEwp-=7EyB~pZYNb-wBNiI6;T@+lFB``V}Ve$BZ^U;@1xneSGGh0%0paq zcO$Z<+@sO2Zf5b9SfiO{1zDe-o(RxT)*Bj}c1$)_1~qrHDf2RpQXJrtLce2HU)uJ# zPmGA%aa5~=`#!g zR1+{cMJnumyAw`6yp~uXt3cOf3TkO-p{)5=U719!a0Uc{uhu|Z=+!mg&i=mTqqlW` zYUKnx^)Jrsm`Jz1;}d|?3l9q`Eh`iJ*a@Lfe%kq`TG7bMg}p2)6x#EC=A=Ruj#*^dql1=SKyK(H{n$JjxpGN z^g#jppSK+u(I?CoxTe5Ku(mYDru-118+$hs2=9X4)!7+Kk{T(PM@!qhA2K=>7gr&6 zI`88Btmo|vsPVv~E0Y6E|SR2(|&m&*Y2q?5v7ohAG ztDDSiZf;)q9DM})d&LHYgcn)1)a=SE;)m9D*&2xXTPvMV>8Ri>ya21b8|0DpJA`hE zSL1V>jGU}QA$*Qf+GLj-&?>W6IceTj6551TiC?#oX)@%TAeL(Z4{JdeJ7 z|K8e2&@tD_$IVMfH2;;eyT3^>9{XeR&zi5le9^DP*F-+@%?t>2I+Hp-{PEuWCL*Dz zuaJ*|q_?oJK(fht(^}g47WeVOIhB4#O*jm$@$5&BQyyV3T{5Z&sAK?Ma56{YKv)LZUC2jg7w_8Szq#9dwHPU~enoN9F%q^Cz? zTU4t@z<$j51i*{}6{vJ=bqv6&%wt)4WX}j=AuZ^FIyDK6PdN&WA9KZ8*sY$~mq+gU zj96O^R7XB&jp8xgMB!m3qy9SA&&kT=WioD+Fhwxyx8n6hj(qT3)YNW2)>YTHm0R@w zx-vAA^@3yw?&rH!u@(fwq@E(mm;WRfvtHVbZLW3ve8M9#xzHfcfFEm!n&Esxp)Km} z?%sXbkGv4hqPqge@vGrSprV9MKb5B1s-I5aUb1XGAvB&TC;{E7-ry13P{e<%EP>lo zn}$-jMrW8Q8Q$KmyB|!l)(bIKkGh((ys{!o3IOt z5#$}7mKTf&3|84CMkYyXE1_NeNzBtT`Jj4|km>jP>r*=h@8#GjQns(5b9_te+}stg zCS2Osfn35{U(fI;PrYhk=DQXBHgPAOMmL@NJ2hs7BJhVX@6H$cGZsY>O=FynMj$H% zCn0y_lTi&fJE5r9@Vk`WwX;>|-Fnw;qN?+iNu}&B{9n*ueOYQK<1NPgMs>eAcOWhY zS}CTEk!tXujJ635XFWBw^Qbb--t{=AE&ZVQK)ov5w?JO0Yd!Un@FHh^st*_-Mr?(C zJlfWv21JyG34JtBwDeR_W#6Qvgmq9CEWIT8;Mw4UhQI$i;E9${3mnqrgp<#o*Gi5= z`r$7`{v>h9dYbpHT;&d^jumOyVM64F7&EUGr7Z-nd#aj$EK(DK^ zN6!#3RAn8u;QM_#$A)!Z@30d={t7$HE?pzVQO_>29_;v_jtP<@V}o*op{ zTq6@1%{P70-S||c8G~D?7apBFvot4FRR;Z=1Nx_@;x}!Ae`>Ty$?qcLK3FOs7%9_9 zfuaEIlFGHZJ+sEcIYEUqOTQ;{Ba4y1dYhuo92pk9P%bzTQQV!Ehp*{~kW|uVj+=5^ zIUi9^CO?wX7QA4fQDwOFvWJwy*f0lAA)Bj44|18q4w9cMHKojMzAiTh=CYn=Hnt7- zlZ6{gDij~rA2MaWd{@Ro>z}wayp!Q{l5@p?T|v;`~MZ`_op2#Mv%O zt{mNtaD!qU#&8VQNr{a`mD>rT5%8a^+O?mY36E8JCZvfd3+A3L^Lr(;Tr3`^0;))RV6?EKO{P0uJYGbBi za3sVrPDF;gt9ni|)RYv&_*!@iQAXqpC`ML(A2Cv?@sB!NTW2UvA|Np*=KQ8^7i7Z2 zeg{=5#3T~RB;HEJNa;5@$0AdiwSqs2BqSb%dAVg1-+{RsdC18#!~;DB4WvsQuJihR z!eCn_RY!1Z%*97JQ};oUIfr{za^X@Sy+Y0sgyLe%l#?Ao+yoBE{KyZiT>+~q>qaScg6Ap-pP- zToA1aM^XI;_g{45Kohj<8aaT=drVi>Bu0nwryeIHaBOvjw&EJtw_1wskdQ;~wz0J> z@fYy^dS^!-WN32FvxzM22OSEU#}Bnnw>1C?t>UBvcLas~8bHS8WCsSuEPvK48m|k&$UnOjIjJ;Zy0T{r2HKD~ssQ-CiKE z{^MccXHDxtdHqkAsb~iF3YCzxEliC4k_=nzKGrC4_!aETn0|zm+WyHV@>NMUJ_#P9 zf}X7BU!nMBPlvU-OZFb{JZ)L{D!Z+1>_UFF-8Zz<#J-gjm9|7f+8%nsbUcV>6yx1( z7e~SuGi~?lZ?b?CmrnM6BV#w58q#QgPQQ_feqBg;4{`>g2oU4!v7yhZB z-wXdSFgW+K@Pvg=jX?JKP7{U2L`1h{V~!pz2d@H4=;qFNabpng|7SzM9P^b)#4X5T zn--lsdn1`SorViG%S~`rtK#6);Sa<}!)DiAR>bAV$5A@%4PVGA2p4ZJuO}~WC&rb91%*1u>JtlX%*1}f7YKDlB}ITm2$lw>xxLzw znijNn?gy#bpz(J_;#KN?mOoDy@7wS(gb9SE-zzJT8jzKXM5=5njfL+Ruf2SHNIt+H z_xul4U&n?S3Ac*)^sWz{{N?x7+wa3eOP^x3K%MLOtTAYGA$CsAsFzeE1Ucz?s0d6= z&Y3W|Z3`by-#;#S-ay4bA)ZNWJgBy&YT(0@BHypu-21L0<@a0V&W;X@GkOVIAMy-x zSe3`EmqWg-uUoz&$|QBQm8DF5F#MdK7qu?IxfD_d1t9;7@&Ge1$_}cNIPw;!uxu?I zN5lKu#rxZr@4xUS&9NoG3yqD}c`y58Y<^eKfb$ZjQmy`9vUE9DL6% zy5IiR<$1ljX=AGXh4nkSI=r9cn(^7i&hvPCJc7u-Wy3I{3#nCeM*n&R4St=>a^CxC3V3FfMvs%qbM-|bWmL1urO zln#Y099#KNw2sK`om}_Wk*+XWmsaBmzTrG(-yiWU`h$Hxhjeh^PtQzqqLI7qUM>WxEcB>MAT&lcr1TjP;czk=H(fK%MnOYDgzLtHrSVGLVS5c zF6)651fJY(CF{>Ktyl3t$}1`EN&d9UT}E!2yrd+o{JO|qFOHd6%7c=W)QwXc33kHP zmXV6*@bIt<+GMW!VBv2o1s4>>$??$z$@Zh^cyU}4W)5I<(U37A>i zDe04kHPXGixogZ^WTlyY@uW_9S*3ZPsOm|5eH#QS zCPTj1U7?tSYcVpK+Sgq$DGtqGBY!FX?O7!rXemvq1iCp0eo~G$c{2&iC$Wmd{#8Lq zY5b1~I1>cnsACoe58QZ+*qY&ifg<*GAjSnDf=^Ox-8vQMN5QGC7=qg+mTza@q=mw_ z_4_xeqhPVrgNiV{gHNM=@*tKhW`H-5)X$WY?~RSsT`0w8<3g3);Fw@rr>jY!1eQ;! z2M4ZSV>TNMU&Nw84)Yi$+FCC|y(czV=*neceYDzMCT{|=fhZr&BA`Lg*F@99^=wy! zdcCxGTJ=BRT}Igk+gLudPfjps(``LkwDoCJGY|+cb;2ozy_hfR#9)^vfD#)qBCL6x zc(2(%ipKj%Ntv{$wbH*{xs1mqaA)9oHA0N65vEa!D~U<+vYCh?NT*?}U`;X3&|snr z9&%J0FskfHynR61Gp%~6D$ymqTl?<7mMx-@n)HbSxq=Aq8aowD_~}7U)w4Z@dKg<@ zKsS0cL9}$7LMW!L133wS?6438)v`u??}-ZhB5wkv+3L0-RMM(9JQL zV=*XltyXM!6{YX(UF|L*~<)Z|m^x!y( zX_7Iw5~PUX`K)CZImN{b%?!D!jz<9}jnHBjqS#y=137s`gd~ng>9g7!BE)$Z(s1Qu z6j}buLyBnB$4=o|b$zh~GfmU%oQim6;|NS2fJuZqJu@DKgrcHg5BHP`hjbz{*J(EZ z6&HZ;h|PjrMYXq@0CQ&QI`}fXFv$^Gv!j2`)HoGf18>csqrHlNjo9?wr$Vs_|D%Z) ziu6_%UNGI!CpIepuD{Gb=yLuHFXp^)sj|Id zj9CsGW~gUKNK9<;I!q)`E9So3SH`S40d*FaOXeK*RdO;j@fH?CouIm^6Pd>M?ZxvN z9qnZ}xC;K}*|}{uEd1tD<)y-jij-1?CRkeWSucf^9jUJ5!aeM`G5q^I=>& znpgsRbHyfP*LW-(hZg=0)1EQq>>>^4H)_{L9}+VL0n4Eib}1xoNvE+!L94PD$sh#9 z)V7L1H+Zyd1;;nK(x#jZFL7~&sa44_Ihg3|2=paBm+IE|G-_BdyH1t`I1q{+Zkv>j zQ6S1>%zO%_TtU5rwyC3&$5}-VnIAzRCtK^dpsj!mTRwKBmzfZkH$oigh}|v~v8S24 zWvrmyqH;sZ^GFGv0u|Xas8A(YmKf1Vky9#F4TDNnZHyGblHHP$ZaE zClk`W)jqSRxU+=SBVs<}c&|q7lMWXpJ0qzW0g-w6i`wBQ;d{qL>{;q({lP94mkbQz zg7H@Riq6uUDRgFdRRizp19m;t|Ca?oY^JdYO%53ZGtsR8BAE@LKK8RG46vEnBlC*YmPA|f z^V6Eh%bWP7P*u9Z?X84x#t;3)%#^Y}&ntqeea6{kWECP$!}&m0nlcMZJb_iv{U=bv z(nPhjpq@`is7lrWm+BJ@jdI#aVq~uqn>f`R`^c#C9HO8a;Xr!&>d;|G@IE~q1IAeY z3stGv>!+Hf2FGeP8Yr!xLAfaTHlWVOjOnciYb~T_+>7I&j*t^sAD1URS{9z#I(2{94sNv7N%Loy@FjT&^!KCb$H(toq3)SR&LuQ0|m2 zw$BFo^nSBp>{k{2VUID~+@S%f$qauZ&-w|W36l=XIe{=CjnMFkh3b9%80|6{zaxh!S3CRnxNH&6>%k`aILpG2`^ z#Xriue2t8+_kSt$h5sn`lLVm=0u+>yH^TEhe?OPF zN0WD&7++@Qmygn-1ZdGC12YH&4G#pm9x;iiHB%H7C4aOo44=E}9-l!QlgG=Mn040{ zKdP&1qs|3a+4hE_4~ApCd+w#X+Ww!F>w4_Jf0plW`mi@%!N|6k>=y?7U7zpWo7*Rp;D$Kit1EH^uCgusYpHP${ED-@#YL=vN4d!Ili%w^DnpMx!!^T%8aEDf+-u$lbpMa<>mX>I{$By<^oIlN^#s+%uN0Z zdB6>L0vHw93UsRUfa#zaMnIwDm}pvee*uI$0sc}`(Qq<^Zw$7VX4yL0+QF%>x*=6c ziFM6`u#OjbI)1(1ngJcuyKf_N^kY7Z-u6c;(g=POl$Ms+!(xJM1`VRZZ}>mT8xxTc z5)zQ0Xv>e)?w=Wb{p-E?Y9Epwp`mmo^{chD6#A>QwIl>LKswfHZrubZ5rCAK!jwap zlDh)@$sjUYJG1mvKC4CmD9e8~y-Or@o7&$4!z0H)ES=f32`&{QqX*Pt9 z?rM;zz*6cJ?7jaEva!c`1G!mjv2qBSC6Yn#w7}eR$64K0k3(LL2x(iU@7UGQ_g{C@ z#P%LsKw`dkY#uC!VP(9PdtUu?ufOTcC72P})KN)M#IDbXONdq|9@vVCikj1m()zr4 zPu70X{fu|K|I*cB^a=D!jR$?x^T^Qo@QdtW`S-WKS0nGQTkoWXtng6SBr(Tm+bFgI zvgpis?XZqC%yOEHEd;NiKP5DU$H(Q>1DPnLO9@l{$ku5eQb_+3Z3?6QK=v#Y87`zEIa*e=0$Bw%|7w97wB}WVE=nzamG75VP z^`yW&=p4qM!OM?gubxs=gn}Qyd_P|CaMQ%Kml$Q58s3`Bvn6r*67+V2k9Xx!{sJl5 zAe4J_57b}MmJM$9zL)L(ELN-a!|V#=_b#;UtA^2I=h(Wg<`5!IBtXcEyZrrJ5TndeQ$9RPMOJGAboz^W_T< zC6nQ+#7)7%RUHJ+p(TEc$&`8`G2Efb3=I|wi=wRN7d)Hb(>OsC*+2jLd$o%+53!dVdQZ!9Bq&+C+jy#n_&{&Uxx(p(IlZC`E}B zoK|c5wMMsNaF(m8!M3{Aa1BzHPFw_-LLsNQ`VJjH%Vt9%BIZr7VCWk-CO{>A7!|#& z;BOE0jA6?|?(g1vmEHqSf|DI2BKDb)9`f*cfc-;^=YvWWMAT3yY!h>+OSE7DrmAg{ zXB^sqeuRi5UVd?t7JOlvuA^UZfzU)Z?&k~yYVDp`pKA9&35p0Y)Y`;yRpJD#zzQ4? z*vZ`Kb46})UznO|d+u%uw zQvoSj(pDQfu4#5&xOZrv;hwFJb^r>QiCKWto9=pKd_7imET80K_rzSqLJ)EeYUt97 zV<)?gh19|PDI?jK`5zl4~CGjcA47rh*cF$lP_2Mkd1LUAS#8yYv zpx15X^~c)sUq%%;TtqQw)9msCy68#reF;bhq#Fn%8#b<^r-x~;49;-VL!Nf-m4|1Z{ZoGA%`a*tAc zr@|v|s99#jEIq+>nX79TkW?ee&bF$LoEX29p@PS3jJyWT7T{jniF{9}l;f+b*(I#5 zlH#M*Gjl=s1Z}z#?(>wC3ZEg3Gz%}iJ4MrXpzArQEVPP|wkZjjR(bZeh$?6`mLcQeV`8#-z)x9rg(-&!7mx+j#5zf@v0Q#oANu%s>06~6pPgqn$F*#r z!h9TsyzDdBi4JA<1|rydH^i30|e47R2^BuL?U0ce3wb(f=8i2 ztFkz1^Ipo@HZD4YK~=fQmndg8R=)_fvgvHOhUp>PkF!UP^)iGqtTtl&|e?|`HXUCM=ldw{z_#s99W z=9n6KtD~#`X-kpjQpjlv$MFDNc?-Cm4!^ZJqb+P>^zpaGhY^{h4iD2#K3k|%uwdch z!p%C=s}HLsIqPsGBDWOAUST9eH79iN>3s9|Df@R$&X6Gi{3{I{KV~F9(qOz+t4S@I zJMRJpmU|;lJbi54ZD+f|4R{%+Uh%bFXzs_f8#gGCh_=_qlG1s%GJ_{M~ z%dAeDOLa6hrY+G~M~Wten*ESc7r4op0kw~gx2Eba1n zmk|a!=NrCRXSH}QC9KGm3+LA!5mHkIUCUZ5#EhR47`sv*!`w3#p*TZf5?Oq%3HqL) zJ3RlA?|zs$)3G<2>z7P;Oz4`KeVc&n)2kqMPWTXJ0=r(q!ac$@i+WIVj`3S1#h|K+ z36A4q?gU@vYfB6QjuN(~a7=bN1jboE4~fQ-g*Y;-ek|JKyshoN88!xLUyw;#0+wt) zb6(Zw(rSTv4k%mrd)|lyQJ1fK_%<_A0aBra^^WGo%t|no;H6}Efnn*p|ML-w=&G}o zQ7sFYE);8c{)ujL#gb7PP#9E5Ug8&+%bMR7D@?^iwDpUCS+a{Hk#r3& z9)Rp&Ex{SSRS2UxPeBY1LS*ap)NYN69*WP5C5gvGQ?uQ9sc&%a73yMBfL1|yVo4V4 zg&CieLJDL6Wx9_x=wIu_=VNM&kw$finSsEs=euI)MsH`YA) z$$95^+xlzRxMZ7a|&VF`hVs#s_kIki;<c{`xLEnVEU=B5x#GRx0!oV3xEgjtV)x1RsjK1J7FQ$u2$G z)r5zYw34Zy#!qP8AC;5}S(rKL>Y3(VJJCVea^9eJGCEmu}8LH2V94&vXo0_P2o*N>Fz#ABWegPQMBOIS09AHT!X_uBC} zgh558n#RwZr=@?={CW$C&Hv-}Y-fT__`OumMfw09Gs{MiYe00#bXgIEzEiyY3w5+s zNNs{ij4v@`$%-qgnB;bVzKoW+swxhWMln^>%q@C3_6qfMxu7xav>IJnF{FO?MLr(q zXl_LWcsmKQ>9E0>?tzo!9C>Ittt3_}70V)LA~Tt54O9}Dngo^&f5a7e9ztE7dNE(P z{utTz`5gm3AJ*Uef;gdLCkPR3F%UdyKLqzgLwz8UmlFbfX;t~6qjTcFtvX;V?IYY@ zAHfesWJG+asj(xGFxYO7QTgem3k-?*?;Bcc@qA*uEM5aY znR9*~(l<8?!M1d3*oN^pAHPn!0cZB08PYrO-nu51a*m#B2x7eb_}0EdF2VJ zQKYb}QqH{y>oPW+O*`Au&=8oD!QD)_G(b$WoK1^?hgNjKE%}g{s5fSpHTgR_n!Iot z*#bQUdsc!_2)CuB7Xe1Y-i;16S}6=dnJb$uJX}VlnF}w@QFVtF?`X{QXWvRxIj$Wd0mHBO$y-Zt2!4du%m)%oTx~X> zDrIV?w{dPRQ9#KnOG~DD;89T#b+~m4Jdlw)K4WM!{T$X+$M!=8X6_&#f$GapP7P4GL{yUPqnI zn^1(?#aUY0C@j1V_8J45ZT403QjkN6wQYFpB<>_geXkaJ!C7F9!jmB?xA^RY%YVG9 zpA+$-IkSjmANLEInW{;OjXt`ma$CzH)D%we!+vOgeX=ddoExXFh5$(x%SR3mIzcCM z)$F(dUV1sN@1q>yaiL~T4WrD#t0Bw)O=>Zi1?2m&s4smTHvZ@;9Q>k8A4L@$9g5TdIY@nt z0-pp%EDG$!IXR3e#JD!n@Kobf`Z^nRO?oGy{ zBCQVW>wnJ={XjYdrJa)p&`TJ|N4pw2aEPTF7OhA4glJ%GK%^R<6QO4)>sOk#Aw@Pe zO&}I*pXp1qenCCo4MW5KMeA;`Ez(Lm6LQ1F(yg^t@6~}c~9xE!yLITb`zm4kJ3k)D1( zNwY@ii$pV0#i$NmjDgBmjJ#6;#Z`wFNiOwtV1qtR{TqMb5vFjAIn9OnH8J?JhFtDYN{$T=tN=L($!EfN;w%}=U_dW7zKq(Gfv5`aXPB-em}r75<{ZBk;7<8Z3q~tZ{UzxInup^VaP{) z6jHMj6oV`uPr^jzjTj6Ilcq;|NH+9Z!4>~0u}a3_3%n=>nSib?oOK71_-_R3n1A!D z9?i^(Lc%{UkCbB3rkW9P)Pl8_s0RJ8`4Kvu@r3xGj%PKCm9_tk0xU%W6m_}#eg@E3 zpn(Z?q{QXmXuU-=chP%UK4fD$EjEAu-td8~R*~S5n}R8x0=t;qaL>!pT4%qz%2@cv zk6*&Vlfy$5pLw-7DN*f583kTGgtwG%S^wBvn@92Vf1I##=r{>;eF}RwQzcGE48Z-U zza(5ysX~+XE35paSA?L;tI-`X3Tm>V^tjs!pY%yTHu_KS%G5Dj7l?$LK1uLiRac6( zarAf;IE}i~Ub!@VnB?hB30MzvFz4`9pi5pj3<@@f9t^I%1aaclvj%L^%!@DEpq1ex zV$QldLY$c~lOhiv^u35aB+O#$!+)xu9NvT`AZYOdyetV4D~hr|^35Sh#8wg}6Ya-v z=$sLck-q8;e=t1BTKm5b3u42USVdzQ&V5O>MAaJ9fz#E$@ppq^f9jK^uR`ciTF+(Q4zJxNd3F4X9*Z8%Qc`=BiNj)u#X-UYXK2tV+g=++8=zmI_P6d%z)| z)mCwuuZULnM~*Jj+|m}(llOrkfc%m^PsFXGs`-E@RJGLIz=P5$%X8&!?V^rt6N5!WM6B!d(e-$?Nc3qgky?f-72C`bhq3JJ^{MV% zF!19XQ!{Es2(S_kEIm0n5l^k8?YG#yz^zqR1$!XC;7%R?wl#hn_kVvmBClT}=D}Qr zR5Fbrs5`6cc|&#ZfOh)Lof4&&BKunq&Y(2NQ{NUHQHxh5` z`}Zti0tFd2v!!c8rne|w^zEStnCt?-=nnVL4vUe3w)gOd;S!9{xmB;glNkwjS91l* z*<^>8Z&R^~4nBLt7am*BMpxt7yWFE+Xc%sz`S$+}pna~sICy6t6O3YEHLmK#F(uw0 zppyw9l=V3{l6Hk7t5>+lc;~!8P{nhyMI55!rhT3m^f}D5k`4;yF>k^ zq(#&Ba~EQ|C&%=NnV0l!m+fsAU?!ufrDH&p!_g&f;;06j&|N8EP844{6RcbTqjgP# zSC#&`sT#%nMcZCnFnnD^G|qY#$zRsa_w^*4-mO?~XgzW zY+Fl9b7Fg344<>qtE-plDjI6s>yG?CmPvr6`|TgD=IizGbGP^ZlrMWA(E|&s__fUsq>9ws%63S#ulsuMQ!iTAC^jcrWAcW(AY^JAfQaeaO_V>o<0XI_F6MA%lKlLaDTkbYHQQs5gp|2Y;?%~?IB2F$U$=meMuAEh8yw&e*WpGg zGikUBd5M~S&7vM$G}`X&yKO?1GhZ=09ELaLfOS2XD#~oA9$Y^Hnko=03=g+peG?X( zhWt4=zs)@ZjPY>m9OV6|rlO&|C@299;$?|lOwra}MS^y~`<;Jo&7hN&Ud7%?gLA}q z!cF$E*1510+cH?D{K_W+@;iHCKlL2s4)qP%XXU;@iMg(*g-T;R?=i zWvOZOd5R9u4(t^UmlIEgNnQdb2#REA)Q0^K9#-(LD z=Gb;3Rl|{yV_a6#39_!(yMm68eJ5rbFEG_RPC?(X8j;B8NRSI$X<>}DVzY0s%@sn# zOmE6pEEM!(1I1WC(W^Od`^NIco{9bqN$RmdVR99mxC+6L)fVpTP9j+g$NA#Mc8rU{EY-z8c5eBU`n?c`7 zUzQ32**Ihjw&BlV(op$9mqKD02IBl99K3T1(Uil&d8RB4OvzdxCX?9_U=6a~b*x~GlR zPPS=YkL1yT}U+sG~@9g9NPzPL-mjWk8FfJVFocXmMYCNIS8=q1|Memjydy;_=@)lpx_ z7bko|wd^cWmC?S__lUYwfYFNd8Ij#JB_}m5re|iN7^WDC>I-ZUaOAm@M5`8{ki@Ef zym0sBI(M^2!e^zPVfkaA6Xcuo+#FhloFI=5U3C@HKbPP&1Dt*M#)9g{mP5D7< zI5RqS-#Ya5o(I0fH$hU>y=2F8bEh%{X~&UJoPA|$e=uEk%pZf`##jBLF=Vvpmt*R) zY$}!|l;a)r4?sWSHYpuCl&lB#GRp)>jwUsG*(l? zz#!igJ~(e?jMYs&IyeBKsRt1uJPYK|70c8tcpxV!Ko(?gZy#mIO6THd9$bLHGN@C( zwfe0=XjivPhXEUo42iAf>}uBK9On^=l?B-N+orwZ;^WDEGMSN)ksG(;qZT)n{LG$5 zYO@>K<^7cRC1wP$sm1WH6p$5s)=EDp!m-WMoFF+duz01P!hC4x(KFE*@yY7mi>$IH z2|X&Op@l42oEC74%%L?$ookd})M_@y-;=9#>CqD~p&v!Gu>IYfuI zO?AOYz$!{bzX~xYurmsPTLZC+iEQ!ds6W~mD@vgW z(rD}};iDmckdx@6`VQam{&n>Wb>K*`^63j!+bvAhD{IYTuLZ#$CphQuW2!qlDVQrJ z`RIMTJUQY2a?jKoj)A?vKCHjT!0_&^CL33U{h20Ms#fVh%9=6K;tz2Oo(by(Pf#_2 zXr`Z&s~Qi;z+PvoHuSSOH73YVGL+jUHi}JGhoiloO@v*zud7gl3mTsMLfnQ~?YOrz zCqF+wFU#x~BrL@)Z(Ir^h#gRDJzZZ%0nPHqFc5lO)3>)_o)X~Qc!pA`76*uik55m{ zcLNcH`1z@)yEY=$nEu>k{Ky@$BBr(e%VImvJ(HF4(?VojXeBh7nwFYVOV5|z_2mH& zm(R|U`BN%(l*lApY_1>ahmASP0g1HsjM74c*-G)1w*Uv{3r&ixt(esA4DPRs)m1uw z8tm#y&=qkrcA?%-N)!Tt?V(i@2? z%Taq21Nt7{j+zla$rN#1h_ZZ#jm1;&H4LkViM5CS0I|PIYV~a4o6{I<=x2WjYj1pP zT`w4z@Eg7<%jjJXhrJCDCv?F=1wcc=ELk0l-Bwhxd>TwaVu8_m0jBJ)zgxHSkA7WS z=Va#(NBaPI9J^YC11!8AU29Z$&PGJ^%%@$E`gpNO>-8kR7spb>>=(4qCWogJr{m+7 zmbD?eGlR*n^$1&p(&Xcm6eBf2v!!umQ>zT3aUKz2*x8|Mc8;cYVc~**tThGb5OZb8 z{O6!PMeOZS=)-MdWV|$}s?&OUQlDTG)mD_3Pv?J(3HedijI&(|;jyA2{B>QUcWiCF z=g9jr=xZ#E#nzVVh_9HJLxIfIZudZnC3Dc9?RBTl-_WEFy*N6CK?4b`AwhU7&GLu< z66v|YLPjr2T4wp=_2k^{m)Pts;9{~jAOh7l*QI$X(lB2Q>Dxg@HSY1S^M`dm?}P+A zrhW4Z^FR5*`@4n*Rukg{<3({t?Z9ZZ^9B|IX9q6Pk-aOqAA*@hs|U!27rBm7@$m2{ zl&bx0)(*1~qA1Dy^(_d#=V|GiIZMf$cf~-q$R~54?6_|w^?!RDxFV_(mjjvu92ZPG z90F=J@u{SOHeOy{D`2DA-^O1y|1?fTin-$SEBx&W_kMUhdf!2R+X1xM?(R)>%ykvh zxhEI$`lNP%ssZw%>to5+Lx&n0(lqV>+2Zpi>=_oY3C)lM`p3KNxAk{|8`*28>wC;g zRo2NFQhqndC5qSaHt`r8=f&ib>6pp`PBXK`ld38J-Si8H+e0ZJP0^7a*T-IX0jjm0 zGjJzs=Q)QJgwDzprU2em5%>GQKnWkX2KHwQaz^mr43W8*`Zfzny$iuf}N9dGXN|C z?*Gb<#|j%g@fnZRNO1zv{rT_ZucJ9i8EW<@Pi}VdoEKoc3E3;2bh*{tSyz`PxwU4f z4@S_7lUmTwOgFYHh(0$QF5=)37+)s77QikSQ(6zkYh;bX+mx1(t|+&T?Q~?-Uu2_vEs4!HLk&6&(p%GoUT0yU`Hwvtq-8Cy(2+CrUR=M9S+W=)s{?`ls zWr{Uy5pMGpMX;OUiP~e3n{Gg;_Z4C>$w3{g&hB-2iIuDJ-TgfhaVg|a4SSj4F-zL! zUrT&nE|_Y{4Bf3!X<$3|>*|90ARssTyRvd~acfO_bJXEr*w}Zm2-(`d-!%n1x2j(= z|C$gGWXvBg$}^2?5cf9tlP}IlBAw=^S~90)f9F?$=oPYoeH3*AmP7)w-st7GP&2>k}?*A!sqkMjNum}MI}rCa;o zaMI97@<;&fs*JIvmcG2DV>UDK&=&8;&-qEyBE7-i*-2w>wd9QShVb)?S+tMPUV&%Fvh6ma1NBdR}v30y2@My zC@E$R)f2ZCpg_PPSe58=%hbN(U~SKO*}=qGRW{jwFN3=g==UL3fc1u+GgIbL1l)UM z`Cyj!N*e`Zti@-zPnBC=3b%54p_k!fJ~_v+LltFgt7z(Asz)7#0^ zcQ;mgfDj#a$il{vn!E2F)U5=aL1@+0F)A|;K068wNU$t?*QIpa590roCFbPGi2z5HW}<6)v%&k z@=qv_)~il25ydlAGc?C{gL9Q9RobmCHOujmo4_sMUYJ7t{JXiItzUnEk&4Sih?%F~ zb49`a?LQ0bDiW1N@a~sZ7bRh!iZ!u1H1%m|H}r`#`X!%4_XYUBKHNe z6dFnDIG)5S~>`87yL`)wAsVG+NI4q+Jk)6!*^z5Sc7%|CP_X}3po+~h%e-oOyeN^-GgIhdg_+s&Gkfyf`qM&*rU4mkcqXmrA4;w{%ZckNCM~t zpwi2`_vIFu2ky)_BJIUPX1j8qq zzq&A06N4vSs(>Iz3dH#D*3unG%C>xv214SyhYrSPlxkR7_}fTY3dKTwS+92)WI3be zQ54aH8b=TIADP!`50K9jfo!L&4FEGUj|vYt7?Yx5hWG)Y8H2hQqV^!A(kD+ot4G5N zL4b;=ugVK)BZ^B>1wEOIZm&W?PTW&IhaN;=NoT>h ztpl>jMheLC4DAafavfpTH2M8$E(y0COH~P!&+q&6i)iat&d%YS zJh_w1ptsm#6X*C2IU#6U5sL|{jFp5mDmyY?TW#6{c{lFrvwW?xB3Z+SxslzQ5jA($ zl#+3|O!%OU`HfgFs?iE4uBz0aiFU^@^HD9u*_L!%1d6%)pMui{V(kfrO!R=3UK{d$ znCbJ?Bopxn;y#QGI=>rfqjKQ9Dxc^{OFY3wEu_5X}l!pq#O+>1aR(xMcS?~6X z=a-T{IGFNffntHohiuKdoGEl5cb{z8PNhXyxaumfzOHVvb!UEbD!3yMONz`b9ICjt zl=%FIWNlrW0w_<;6wO(A$PDjtRSa>Z8qF#EYnG zWm611UCWIv=ko3`1?v6oK^?Vg^7&o>6-HbLdJzK2E4ZFT_5CcFMZAfVm<5^&WLU2s zDqd_e)ZEq97Fb4w()GQO19My?y0ad!8i)kTggmc=Fl3;fiZ>3p$4X_zg}VOR)~alp za9%O<9xuQOLdMsxuPG8C7p<4x{RD2A*GOorO(gnt0v#KpqYSuPe@Qv zCERrlg1A%D(>}0kx)1g%dn*c!ecPv}9h-p3;E(UM{2Z!c77Qbyo@>V4%&k$e(5NG> zlGbrXR4oFydG^Q%i3kZFhg2>9);Amk91J{b~N`7O3dDU#11qho5 zb09}1#$)TK3O+*uyKGk=G4*2*q<|nVtu2ZeQ*{h1-VPq;*aK&v`GAC(DFQQCh{C?6 z&r~wf+P(ti0ugFz$8%X@qu7#=UKMp^C3bq3LFDY(Ga&dL8&;iuF(EXpY~Q2UxJx3H zQk80$j?tvCC(GdcVzb*uxqw}*X!P^TL)1s#r&*QuEV-=w>|(-Lopc5PK|yvFJUrZ= z>+5qa%P*Lo(Q`qIk9Q=qA&P_CtBb1$z5Qo!7lUiPrg{^sTIMF(AFA|=iH?k&>&@;m zu3?O))cRh8C?lH}R#$h5Z{cb<6QH+u8HZLCV%omBh=7R?EPeaPf zACLMC|J2vlrtm`;3}x=y-spc11v7|RM761&XK#0@rhJimRA5shBsxVDSzTXGS!&r) z*}Ls83r3c!STpAvJ2hsjFtq7!Q!#s2g$=|p(RXU8OT?5@4t_7Uu_1Reus@rfO`MDm z3hJ8%3crLnv*+%+c?E8PXkI_Gi3VK^y!X`TK(#+%{j&rzd~P$V=`3#;;pF9Xu@SpL zDj1ZK5ilP2tJS<#(xwy4c9+tg+(ZBzczD<-J)!9?&u4(LA2Z>l#^m*rAKYnEWAM18 zoV|xb?38pMX6^5jlPaKr(%!y?l&;Lk_~Y*=CHRoSbreD|2?ZcGt{4S;6ihed!97MQ z73cZ%cA^sCbGk40LDI2peB{EWu_yvKDZ#NZ)=c`%W;C83_%OWR0pwB=?`x@>g*a|z zrE2@?Kk88N?>IZy;%j`GPCbv-YquE&!rlUcDGtVcn_-sihuQY|KBrYCFMAnSSigYZ z`Q+ll+5+@d>{AHBw#VzAn-s5w<(EDU@NhH}ouJNN9uwV`!D#8#_4QTc`*FaasYhnP>Hsr`pb6HBJU^66J^L|{Box=feog zZbaLu+D_@u4_3M*k4?k|Vvkv3VqDinkY21e!P_1ui%5RAS^k%Z(z@ZMU}8e*Fu;b2 zg5qK{@9zw7C!U}LpeO3VfCtt<<>Mc!r2&7{@Puail9Ccu-?a(<-z!c>EDpoG#1q0a z_STq2K3M6)8^tVCC~KxVA7Q4OS#=%-c}Wze-@`Oa^W#iD*)=y-6VePUVLxTi{kSe~ zW!gE^7lUB@X^s&{ljPT4|(%~@C6S~s|QWzh1Yh# z@QGD^voQ*4LbCtfK0QYSw!M>@e`C$rb-*(Lb5?eK+LoRI3)!_%>n2)c2{>JQSqmqu z#=cjPsfto;K5Di=3Begb>fR-8BOyH0Q(#)FboDd8DSH2_a3$fT5q1!Yb_M;lbgkig zkhpTe$C10;;FD<-oLvwF#g^~Lb7&WZ-wvsi8O;#f;n3Ft#@22A`{L~C zq#p`i;G`j^w_Ry^h7Z!&W7-rx#zAQc+H$`Hmet*DYD&0j{aM|mUq;sG+UD3I9W+eQ z+h37)OD~Vu1~h^V&!`_|8~`mM^FW+JQyQq38%gHN+Ib~v$WS{weDTd`e(udj-(`EI zU0rnEl@oAy89`g8{=3K``l@!*sq}4oW$s;=_WXzgGEhMD7(cdY(va`7Y zOuxWB4G2roAzK*ekgkvnM>WR7jmz;0>hsSte|{mEKXZzl5T#nWVBHXWZzMazyf6r(6o4s=DyFbvtiwrKfAJjij&J zTF!pT^C>N>vI`z7g~_^^>C_Huqh^ZhW&9grRRtnHYyX6&0If;eIqN#s!0V_Lojs=ZEM~;|Fr*6bJ)IbrwF^N9(RtpUCV| zmh?hdbZF}3a9k0!UUo$U%yG5^5*F-P+^ws`FgHH<7D!j9MbH9^0G?{t8Ag_BEgm#H zfuS*%>w)G8!N_s0MW4yb}k>h0oaOAD-oF*60-;7lC?O?FcpxB5#(Ti#jEk zypWpZk79~tg$UlcMN*o%7A06|WWwfrk&2Vlcp?e8XN?xLvEVST;H4IrYdId`yeMt( zx<%9mGTSSLKIs~XiyDPPiFZm|QPnNnhI?M6I6)@Z+cN6b{b?wHBm^S&poI1aVV+}c z^lerz5ZpD#$=H${9o0-8i>cl0{JUrT?TcEW?8yfV+^G7b817+#VLJ4fk{U4X$`0m; zRiW6QBDj0Jh%ye>%Z3ry-)BF;7!J4w*|Q)`OCepB)j^G4PuMK`!rzE*QtAGf+7We0 z(=*Tx{yd#LXM#Xhtu&rhpj!0h@uI-Pn>8-{$bK!wqi2te!PJv@azd~LLt`NOqb;Ty z8qKZfi+oVb`Q@IhCq-TTLl6QNJ}k6tn!)Zihk%EwaTDr{VREO1#VH$`4;AajS>_W5 zrnn%-D}J%opB?)0aIXFS4XIsDvsUr(I9nG~!V}TV%J!mVotvp;R&Jyxh;V zBL{BZ9mCUDwOo_ozQQi5hJ}kIhn0Z7l6Kl6B&{xiO@cZP*}Z#dr|>&lb?zY41`vWY zvo{{*y?mlc5p)onD_L1ypl`GesNOGR=JCWo1%((genx$zgXgrFVPb|x-4#g8C7uI4 zC3esoPKC|o+>G8Aw%=KC{+L;J&(W*-Hu1j!t7rcJt96{XJ=Fi5)|UNyfXa$Gzb*ae zFHHYWzVA7_HyD+2I7k_O>=ne=-&yKli)kYsx$yy%GzM zQ{ooBV&2W@NVx}P^TX8*mv&a)K2=hl?2UIs_;jdZY{Jm7k zrxJDI*F7?G<d0aye2|$7Q74)Ym1BKgx#rAA9P@$;I~&qy z=m{>-+!2Kwmc%Th8kk|6yf(Y^Q7|KV6TgU=$HMsEc;7j!=oVx^w3xYX^5oQ|A=WuC z9z-7qgZtT}^D*KGl+b?dM>aM+RT|*^XrC|2WFwEzTV7tqD8bEkNi2;?CJvR8b-fU0 zJVeL)o}N(iQhDM!g*=3x-AbjU@kv#6?#TYmx$ug(bQb%3v3csCZvIPF6P@2q zW2jfRUPgE)@Hj0^3rj5|iuT&Q-vTr9;TlG~TmZ_TcA++RoeTmkjMm%R3N{oTitT9o za9@d5whIPllL~8$xzre^K9utLPsEYfSfDsebd5=JKL3n2Vta4V;1a`)o2y75|5q4S z67Jv$nUP_bs0tikr-LeNHF>HvL)p}rjjmFvlstr~sw#f-?kg;wN-$W>#{O)TVOge_ zI{0fON<*@fda>ayMirwvfAC6FRauGsH~B7tVyN_JG(}>5NFqgP89sQw&6HvSd9O@- z229Z~vN)mRy;jiP=;Mr=M|(8uugDH^>e@b-Y*Ni!Ovn&{YBR_``Z?GuU7%speva8= zx6{)Lc}#q^(maaP>Y6mWk%$U5{a3Lb@wmKuBa@1LPnJnmE-pG`6Kk1=s5_t}$3^lUgDH9Bt+Jqt27+sn^oHYb*6@M|n1JIZjuIA>&H8nM8e>3k{D)I64-H-gS z+M))Qfp{o7ysU|c*fvCxwFxR>L+wZmM4MLS36`!YO@lHt&-xxHtWbo3&Zw!Q>I|VJ zZf?H??w+A!a1JpK5&0$;NW6-~uN$*X^Az4bzv4TO7zq728-e2N|1`&w8E(y(3K zSd^rqzH+^gEz=P7j2RDGD}{`}pW01Mnhb50rHR_vLJ}W#uC9f}Bc~|G;>fzuS^2&( zNrs7Np$job)&?Sl?yl)1O4(eSrdTNA%%Tc~9%KtDwaOG1D5II_r$|Bx_NGYI;75q? zu+)kbASNFa8KfrOLyize&L&0J5Jz^cQ}Z2tiTjz~b{O{1WZ$e&(B?N`B)Bt&;^CRRfl{AU*(?=3~a3_Q~FPBq;dQ zP|pZ4DWz;aNAT0^?MW?ggC(em#UF5hcoLEO>x7VqHX3}q8crxmH&Qr0Ac}ntfR0jP z)6k)o_48&?E4w-4>+Q?hk+4h_6BNf3eV^rM zM=y@Y)lOdm5c5!dv$Pa#Jyf7p6~by`6@N`DjCt)59GfJu0vu`pb}73m#^(2B@m~l5 zC_dCP09uI3QpUbRlw4srF+=TbR7MyIqwR2@i9F~}?Vk?JAHXQ4hmVsHY;MWJ!sy0P z&iIr1#AiHBnDZkurVK#0g%E|1lN8fjY&%W+)^!W+6dk64`npUxR1Q7stzu#lJ~eSn zX5Xggi179@@#CexNJ1n+fq>5|ik*;zmICt7 z>gj^)XY$V8f3Ea=KAt7d%q1@m^*kG=l`y;wmQ6r{{e;Is=IiMhB^&$C+l}|*LCT&J zWQaf^j6e}OPqh6ojP8H+f&EL}wFOdHe>-)_CYXHF|1pRV606ArVv(qEE z|NRfc)6*jyEhTQA&=;|Mqfo|M7q>PR^>3vAG^&~kQVve=KeyP^KY~P|FIjgh2-pdl z5xJX{kO&Fc=>Ln`BmbwRwfnUVfqnGtEX2hU8P7JYBeUQW8aGP`6@yL;GYc?Fk_Nj zDHBl2^O%PZB*}buG!c2b7Rfs@#?K+Zd=0r28=_yH;UI`RIbc96k&^1Sj{jEoHlgBw z^YG)%9wDU@2~jS=)fg1$E8HZZ^XX+y)#WKYuEDm;@ZJjUcg`+az#2O`sjJQBV?e#lAxGsA_?c?+FOj3MGRI(Tq{wtCh zc4%%T8gb~URvTLyCL`*X?q*{@+6Q48+KROKsmgWG)hDOj??s|5QT5JAt*0HQ= zGDC#Cd{aWR7k7x!QTP%c;{c=T1rGcg+AF5(>TQ=+=M@X9$mmk9=}|}!8`&T1QHvB< z;}8O3+x+=cURVH)FkEVloYs=v_xXmZHv0<{H5MZ7HoeC0S6M1ciTY-cD0%KU2nbx- zUX+D&eGL4SjYntQOvqNT!+0O!l@t^LevHq-inqeO9Y*9lW0SSyIAhT}<+@#FOJSRS7!W+AxcBMd$rEC>>Y)=;M> z;*+#VBtOGatZ3$LZVnb7e9uVBDi~ks&lY#*JdG&0QviX*qV*DN17h9WrzfYyxw=AN z9pWb$I138A;IRseqTNg*WGPBNsW|_FGVLPJb7wdrC@o0`x(47{}*(9+& zKOCAHUK)*eW};VS?c*Hz`QyQmr3dq))UR+l5pS8;*K;@zB)8%p%Qthpql8XVO_-Zo zfe6_O?b{PH`s~9<v>o< zP*t~j-hrm1CT~{kbL<$d7K&P=iJqN_s0;U2g)B2w z!%h^|i#-_3%R&bu8QBuPp%Ba{AE?EBjKd-|{Jy(dDA$3^7rj=ufN>()axSh!x}uHT zlT^uKAkxS1WA{f7ct5O0$zvu<3tp5Rgum_CX55bw)oDUBw0O0Q;rj_b$S<_d?|cNI zHjh=dXn2Ge`6f|9&6RZ+(gcP>>o`I>ZUT`pBxoip4M_drXyY{%@sny7_svpL@B=}Q zNn%HPTm=VKqWMC}GuHH4CDREt5?v%=i}sG@ez)TKwz<6Ihq+vmztQKujSZmhVPWoC zJC>_#Y+;)oTXEdqdAO^V3B-vxin#R9k8T3DsO~CI2!-ERdjGe7y5UuI=vV>f$<*(U z@rP)+rS(-a{q;28rpb7>T!q^ z2$w~b#8UH)oVS1YRm(l?|EpG*V#PI_0N%5ux%#*jjLuF3NM9z5lp~5M_#2`G|MtEl4&wS?$BXqyabs|K5f?=+{fs-aRFW#dg^bse%P;1uW= zNREEdkO=7@x@Hs4Cvy|pzJj1-cCR);?z7&A)6|lDIi|FkQae5cs2fRF9LF%6k(WTE z1wnS=%he0msN2$XFsy*OslM{6E~<^XqADyk{k={p6E6)1rlYplWD@z9O+bKfUIGn&7E3d zJHJ8>1*Tj%`N)ksg5LvGv=Hl^S*TlY(X3efjK?2B3Q`e&VsI~p#{eyL+8Um8j0 zpN-Go{pfd8Dvcx559u~jXjPOH6+xfV5fTQu!CwZEX{zBd(2|Yhl#feo8!_-gV-9hB zV^bzrI#W|}%P?r(WR&8fv-%`)n9;!Ir|LU3#(z#L@_p1fY`$VCNbGBWWu7A$AL&ad z8i{LZZpM7@GFKlmp; z0uxY5U{DkUysTOPZTOgGx+q4856Zi=^3Z(-QwaLEDG?s=b_Y2P(0LkoW(<)v_ERE! zTwFq2-vi#?wzfV*L4->effY;3&m*XIb{i~4Si0}jm5r<|gC6M+@!pB~-9q^Yh_kyp z8pqqP2qR31q8bOQ!=u*9Is>rWvUB3%P%GJkN4U6o2YO@0zqE=+6TN5B1q9mt9D6*4 zkuEvNRkUXY*gjY=<0&PZyB|&2X?;u54I{cGsQYqkFT%rpR+AHna^ca?W8)GmXo@lg z1%BcY683IP^4XWcQsMd<&t4p+r}y=dpA%Z4{0$w)O#>IgnJDgp)#x%=m0! z<0}G3Xl|~qTGtam#bgugXogK&mr1ZlV4GaXELinK?XPCP}=_vw+M@CfHM_CNf>l_{F&{Bh&wSXSp?e`kSs7-{q%)`-- zG>y%ca>N7#lx_1{U!TcXtK+rFX&@J2kEas#U+O&R{|S((#69EUV*@z>a8dr-o4+-h zyRWTRTWiJ06IFbb{kQJ>Gn4n>Z*QmH@`bn z|9D;hvgK{#>-&7ZK@3J-p-h^ebLsBxwB=MOx_X$c9~w#}PAv#fqw_DUTg}+fae+KM z2auWtUx6g-M%>2lvXy8!8&HqI-ES?eKVEs?CIZ?|$9439F?A#;3pW34(nk2U`|_)^ zwRJ~JIc}VN09Bg4p*~FZ`}F0@G;mcu5U6Y`v8ET4J82^ye9zI}uh1(h%Ii^gV7g3D zupsl?I8$x{z8-QKjqSfFuw>zrR>+~8U0tzVm$)$L-F9Mn<6;l}La_3>1WK7TfJtMN zg)AE(5@q?m(KO@#v{2v=$PdYX?eD|5z;pKM6aH=outdU-V=Bh}kC0YdwqzHTw>i>R4CV&F4b-rD3kCPE+}h*$ z?AS`e?^D_X&PStwq^T$uXl@m5ZD#Q|ZNW`LYiC~jKtGDLw+$jzuZ_G6cz*`s09osM z2AyVpmmnQ(Ee@iHSO54$a`~_z^186KWxNPk#0Olf+tm&Z-$cOMZNdL=X;V~$gNo3$ z&F7;aO}~~>MIF;8Nv&#ce%w5o&i{oqp9~lxmmq4dY4iE+s&!uBqpij{!}R1ZkojAS zr>F1#{&;_N_P@$5mr!l*_^I#O{~Kr_N6{8Q1n*mO(qpD6_hfm&qkq`ft4^=4 zpfTvfOH0GqyMY%7go6lX(E}NWA<8K5>=au1zf_2L>g6tSaOR-(2+eK>T;Y6!Fb&fT z`aNM3M1$}I7z@~fA3E?6&NMctNh(O7VK(te!igT}CSKbUWhOa-5MKc- z7cbjv!huFif{T1uL^+^;nXP-@>5Gq7zUZGpMn3b3$@=2^^mHr;ho3vvAp$e90HMMC ziIkxWslHq=Hki-9_K&m@(|uRrB{z37862VX#G~|k-bZ(9*S-&;lSAl2x|#x#W$Q!9 zq_@dy%y5W~!={2SFJ>%;H=#J>pD(ZeE{u={T>5TLXi{)$FPCK)AD(i!|30nx2IxJH z01cvE{_yljeL<$GK}gFqdKZU+Uw@ma<>{l-_Nug_d%=O79IChWJLA4*$Ip*n-w*`U05 zOF#0VtcubTA`rxmAjgt6mW87tHY%v6|DY#;#ZvD%!;M!Y!}WCear6R*^@_6aAXRm>!}^W(Y1mar=cwG+^MY0N9`t$45)#lw4@r(_v?P^>XUk|QjD;0kFW&-wkr8M>l-VJ3?A3^L#<7X6`B-Z3$6D4ufRPl* zi*#2&Gr8NmPZ9IVpj?&B2yC_x9Sx9J0%QX))v`vhTk;O9dFT}(+O5rNYxe@wHMF!7 z=VCU@t0Vu$qa0S%BD*6$JNugziC~Lm*B@Y0WQNNR(WhxSn2B(9n0Sjl1fuoT%0^Vy z3=72^Ax@G%bFg1yS$$eBi+MZ5?T{pyDFA=p6N z3+`CDrrzLcF$rsS3YopwwhmpHi8)Wm6A2lzbONj-Kz*OU%^9WlJJhia z(|PEW&^@A=Ux#09A?73NB8-V%UqN&WRsruUC~ea6ALedgz@SgrXU6Bnq*R z1}u7$-^lbVS=%z}Sv^k3!!RkfB)ceBV&8rwRWe)3S-$ydo{t@=lzgbSEtTQu;{!(< zw>zD_tc@c4qVlj-_f^bgCv6w^82;Jc-`KJKfc681z5i@x5O(oxX1XO!-WXy&TXoV| zq8lJ%QESBSh8ns*-%_%+?(Ay&R7d!Q<^d;eIvkzDMM=T%_xE&AG9Mpb=J_d<$e03= zJjBup0Y|t{rf*GE6{5GLaVkN9YBMd7aOac8?v{a#jgt|sI=5P703JZq6Eysb!|Y_KrVtlQH>ZPJPl|IUJ*Ru>C}hQ00$)R5tdf_$uH2vsEuYztIa038iq9b zmtbQi#kE{`Vvjgyr#Qg*^voWXW&X*(u3=2&_bu`_f%&dkWc%?Ebg}B1`5NL%9vvL9 z8RW(w{u#CgkHcd3$Sn`8-G$qtK%=_D+kLdLkQ&xi4KZH`sD{gs*8S|Eso`N7*YF_N zb}|w~%$u7ap;0c;7{Jdui!8ZD4k@leGYHE*b?lLdtHsWGWS1*Oy>8SDuZDW-1<1E zS899B^Y=cu?~<#+2B^#aWUgrFtnph6*>Nbv!9*sD6cDSj=rRDx^l8wXp;eUtl!c9$;&sE)la~jDnu2sYgwBsYAERemP(kb8$B!HuFfn z-L5mZeSpJl@{;6mW^K@f*Yg_EZlj?6CJ#zx*sO2KUMsKiE7P@Al(S8^MakazDpN*~ zX2Hzx$s9P~ek?|wL@vS+h1}}(MAp$=KiPw~wr>EQw3(rNx|9rOYl_A>rydxA_V2aC zI&Bb{X$V>yF9=H@v*{j{eMAYNsn6Ek$78iRhglNFu+={oCk}4hLf@nL`^bun6Og}H zN@wGZ8YJoWIua+|#qrnVKBB2cM%UKV@CK(=jMvnR^?)}@(wVy3t2{P7r}48tx18DS zl3+Skgvxq0Pi7^apgIk-MpR1#7xt=V)jLGOuuld5iEBx>Getu}?0Gf4>14@_-dmxP zKp@@+7aFiwmXw}#;pj*`mVWtOv0Z^7N&`V%03YH#8Jz7f;#y}9L%OR!Aa($A@DmpP z00S;EX^-;0Qh`^&dKNtyo1T>vli>J{c|AaPH!m;mDg-L5h7sFIwUJqnHF$DdT76f* zg?SC2*))UXxXwd!)9h0)Z#3UpUGymr)Dk~s^luQ8p!o76nI7>yx-yFYE>s}fr_aKc zj)4#k5<_Cw(Pm)oOb^cdl;kwu{!U-&vQlS0gPgUJh^!SwW#%HGJ3WLBB+1}?WjXa| z2zo@zh$~O{WP9g^ipR7t@SB2BF}xO857TT4a*p=dkIk-hzBae)HS0HqVp;z#SzR1( zV2H7MEPRT-;>F@`y{wdlhWAd)nr@XFn{rl{lpIy5n%lU+gZmiCCW&T$S1Lt3BLwn#v@YjFU>kz=_Oo6s<82@dNI7p z;x)tQGr?=~A*~UKNM|EZW2#^gNI|9Rx;rUTygvO4{@d*Q;{NISMF{dcZsCUrw~4;! zA$#bN88=x4lXsd1f0%LY@@9|7 zx;~leaEy^>!zi`y4io(oHD3CxI07~_wi&1W;?>*WmXG(V7XK=c0)SeYj*BxAR+5Zp zM;WB(X7D!5+vDnG8ER|lmQEE!JdQPOaq3RGEgPMCalaQM1Ma*K7%xhS4Mxn%(Be2D z-kCgq9$xt|O`e33A`r^T=H7HO^2n(=(Qy;hMyW_Won&mNPr>yFPz?(FPpg>mt)`_& z%|G;XAO@WVB7LVEZo%8RHu8uc>8lC@eki5E3iD~IQtD~P2(!khL4A!3wr?} zBW|+qJ3A1Q!x1OWjtm0FO3oem1|=51PLD?vBPaag*}mf*V}<>rA7d#emT2^a5yy{5 zEd#eEV4u~niZdp%Y}l?4M$e1_W6c~0tM|gxjxJ2(<4FyqGkKfOP*sBt2_YZx&=7BA zeC${;dd>gxgQyJf%`HR)k$`XN(#o+oT@ni<6>4l9yZ>(aZa*#O)FrqG@FuH4&|#GQ z8v*(Q!5h@+dE8BMRD?(?k!oBlPqht2eW73uZ=R#Q_qP-gv4r5Rs9X+UEU21EV^u(B zkh@G407=)IDtOt&I3rOSvyj;M|Dmztgvd}(u55e~&^Lv7tR#ZNYP-Pc697f@+;^i#h=>BLV-ko*)E1PEG|b%A~8v3nr??Z4ptX0)SZY ze)cONqhri_No6uwhV<**+wg(@WUIBg7qx7InQ zku31+vX+hvy~OARc}CWY>LDQvj6kC`-A7^VkCA$8vge8K5EjD+EKQy}y!VQNOn3Vl z(?x_}oQTDOw6x2{C+_@~`U-2aBrvJL&;(R3*yQ`JCT21nPf|?q^r)|)msl%9{ry7p z;Lq}j8d2dC=zHRsnVRen3O9Xba@~NVJX?+s0#@^m7C!ISC#pBiy=p|X;aBNGSgdCj zFat%}!omVFJuZSyr3HWR%N3@!me$@#SuPNDieZ(8Tx^sO7Y_w*^a83<=fLwd?sX=^ z6!%mZAnSxt^Jo^~lR&hFhK)SIYu13>A7LERFlBz>nhh19A6!!ASX^1zEugSX=@kn{ zfp9LC$DIxL`w+!I+S?rbA@m%OAOntRAQ^pPap5QtQ;cQf0MK%Eeg z;0#J11Ve2D1EphSF-^U&TdZ$H@n0>mCZ^n*#@)vjJ#q4%+vJjDRTclHr_+2#26OBX zl1Ak?G-kylC9x~o^z?o6y+5aLBWzVwW{FDBaWHdbCz+4dbx%vfeA>ir8^dd7f4V|u z#ucGsU{sxjj9@2!y&8yuM~+ye`LeAUVK%@=d2)@LNH;KY+*E6#r3Q&U$vXMPZh!lgTXTk!}>dnEI;(;>00o`M-P-H1twp_ zgq-y!vSNw~KIkj>%E;uF1WCptwC8q9SdCOJf!_3Ply5{$sp${^lcx~rXGS_2VzzvP zNVt^5=XW5sfs)to6_b>xP?e(|jYUBm{Moo(AXlCq5(f4GnMq1dP`7lja0a>vU4Dc! zR9pH7Zk05>_KScz{JfsS1!lC)EPAJAP{{eUd5D^X#OnsJfsT$%dHHH$uhD7(4|ILK zq6=Hw13AN7^hZ`(p2?;4b;feSTkN>_it#0XpXey@5nci`d0$bSVE)ipw$hqBBO|%7 z5GCr1m#MWQkd`o85gFeg-cyqpR0J1KI;bBk29Kxzk9x%C0qb;;ZjQ+Oj%!%ytqs7dsjnri6p4 zbyc!S8eQ65vIb<&*V}6NH`!2Fqka%fJJLAHy)3w|(6GL+H`~)>+ZfUhILEzAPAMYM zVIl1h!n!k0&M->qQZUg?advVhdGwlShwxN<)I9>IU4*R6%urDHdV`EP97e+AjO-SQ zmIwMC0~BDVpPG8WKB5K0)zrYG8f)PYmUfh9uT&VmfSToih0FB0we`y^L++CUEcEH` zpWQ?<88xqp0{A`E6y4~d@VhQ;nud~Uz8L{;roNeXUK?2t-?0oP^bt16Nq`7?ZbW$t z4_!%XLfjnRlJSV$U5s9sTnYQMA>eUot;j&AC0RF*Vrn6)vBP#rOoxVt{W(3&Om5B2 z^ZEB_8cHKm(@-|)GBF(N>P)vTwuwzB);@U*QK{)CT~^C zXX(#{oV2UOp>*i-?&WdKyIx0iCt%5aG+8-=%F{BV#oN)n;PF^c<5#vG*A=s_#<&D4 zjxHYvz!JEAo5dw$sTt(Zt0ql29)1u?-#j<9#p)e@fT{O4Rc8sC*kh4I6RuZjt)(9+ zPs}`cqFow z6m7LSQbTu4l_3Fa3+V75Bk1%&HKh2IG<|_g<@TGWEYy zC!U`Ah91VG)oM~KXie7>h})$i%yGbhB|`0-qKCFmEDRb0QYm^<@;IBBMMi)}_(&g0 z*e+tciH~z=WY774>?p$V{vX6HHCZ25q)hzVyP`7{E}Q0;QuEux_%_^I+E5px(v)yd!*exfC zmQkNx=&WvO>8KRHFnALECZScE9-v7ks-(YGt6=FpJUmDTq0mR^g3b#hHC` zap7|R>9YhjP+B251NN!D4a+P{6;vagW}`YKY;M+069-+(jx#1{3KpH1^iK9FVt6u= z>YbjE&Z#bwgd-s7o3PNy=?qV!WC}KjY*(B?UPedQWEy;kVT=a5oet_7llw6L=aJ8r z24K-c5|0UzJuuq`Yop7?D`O)%e0|nC_;5y+IajO94xHRwC~p+ImEd&_vYZzjb0I~% zRAXnic_l0r%vjrQpOd;3ZI`*9jNcXI%Mc%VG2!zfLr&(QVd*h*_cq=v%qc8Qg1$@K z*y89W3F@rc4SO++zTgsU73f#CeAE5WLKGw?Gi;EXG%=>0;Gl^`4+Eoa^gWL6z!jWv zBRreScXk4Lfw@Xel4=wb1xS-nf0vMcA4yaGXg(8Pa%X{_=2=^Oxl4KY2mGu4w+92VY{wB{CMh9|l} z`xcA>hA7Y3#aV0d96gem2@|3hA7c^6<{6}R5?0VOX;UQ~N&SSlhA|Wg+{9tMr+SM7 znXHF}z&1?`H`k+~eoBcWy;|qR!xGKd>1zeNwo8k!sZ>jyK&sqh{s_^vs4z!pLCXKt z^(5ewm^p8A*52a4%Y=mnhd%dmV#Zjc z=@ZN|84pAJh@Di?>6iU7F)m895jA9PiHNuqc7~^>l*v8El9jh+e0yqj2 z(&HgKxl}wdaoHGZ21WyYOU0j{{wVp#SA~o`7w*ukfSAkXf(7$~C0|%>$8IY-FFw5V zo9<^}ikb4siVC*u?4{3>X=0DD@0jSl)wep093G(@(%S1=E89*lkA2yfWG<7;VBe<+ zqw*j=eZ8Q;m#|`rk?e?zk{p)D5}$d75SrK~2Vt;yPjr(Zu61iN&=#g3!rY|jVq@>a za(+fQh9l9s>Xxoo_*s|ER`*1;B`8ovdBm4op!bl;peF9(p9O87;~*-G#f|Ml=48Wx8$UZ)m4xmFCmXn+nx@t78HYcgOFMc$~5%aR?3!O#~E( z89^W&ao1!=FB`aMc}|xM4I1owWOJF*$g10D`N*o%2T-05Rnc=2p4os-Y)Ei&?GA=b+09=0yX>Q4unrOE`adBuLY~;F2oLd)HqtUOK=4IHg*6@cLuYk0 z`H71*81w}?dNd2T#R1A-6DI*zEwXg=v7k5$o@aT$6>0!CxH{HG$=fGg>IoILDmcV; zmj-`^*Unx8bQGg{sZC7y?S+AWAJTrdLFOPfr>KBUne@MK9>W2n%wbvsb7x_ea~>AE z3rs#|BI+qvqX92>#J9s5C=zJobTSM2PN^jz$We?(D<>x$eiqA^RLV{ZIW?prYp52I zkBq0p)%v&t8T}yVw5_U!LpFVF{vc%?IAyj-vmQ5`3>1Rg!ra0$Cd_zMTNOt`!x!xG zAs~Y@ospDHc7rp{Isrde6!kdrNi3mqT89KW#J!J_gkvmZ$wUFAZ8C}TFaKPL_2VR@ zf38<W{TP7-_dW=6=IBT+nP0*Ws`boyRHm)A1e>GJbkD$TLo5Oc3$JlEZ z>F}m(Fv`*uP5d|;ptZ`F(KzF`Rkb9=#74szX!D<8AvK@rejAjL zZEpB-w*L2_`vTX#%Ohr(M2d<$jbC$amE@u#WAEz9N{2bW@Z1YJXpTy)^6_Q&*-gQp z+Ii!^Er>ilwq3rNG#>R*n<^mayFC^zU8|Ay;RuBUyz&&3VR6vv9RK?iHq%L^_)=cL zj535e7+*`D12ws4vEbCB#ZY>QkoOPDj5_Kt+7{)?|6}bAs(^)f9Tq8wi23C;_$fT# z(bGU9yWFSiiiOn(K`>!;#@PGKW+a;YXUW%-~D8%?+|9VI#kq)D%j9U^*sQ{%Hl2&BuAev*7$3MIkmF^vmEC z2@Z#y0(+8>M?XolbgB3R_!2SQqXjXms$G%24dh}BY6f9+vTD2mth7iZG*<@mI@r6lFAJrdk2V;7`zOcuf$vAGjEmfqKt-+a@bJ|NVQ=)! zkCD!pYeeecOxNWOn6pjRoU#nf^);_%mG01%?NQ) zk2lmcm5yruti!)nX#fHrOU;hrOms|pGW0ft2EmkyoZwU$eP}PcWWbo$2V2fYSy)!+ z@Emh|mDQfo?!Q(tH+XI(*>5W}qs3AYInXE0%}Oelvq*bXu!?L?ME+%~1?4@RpdJO2 z7o+e#&|5;EG@w()#xrYjf_MyTk6RZFRBf24sMI)}krfV2b|S`L5+FYr4V19(Pcn0Ph7pkyt`H6qJ1K1Na^WmVix+~Rm(k2r5eo!4pz6`J`z5BD_eumWs7AjB<7U}(JNUMX zV9+m8lg6>;5hESV3wndFiVY0>%2h>3*5o|x$Vd;m!W6zN4i1QOEX&|?zg?`oFm(83 z0Z?B$QXb5NaG&&FU!G@n27lydXKQ8u`n@LQAAplYjWZM<9}n@qk*R3w5O&axce>cv zkT8X+MhL`6s!h|1zJ!eCVJjD}R}`fk99cINk@#?h4vowV7AcemZM{)S0K$&K3_O8e z)B7lbn9o->ExtnLiG9A`Dun{OtGm0R3yCt$Mg5{P>*eQkLx=!l%ikoMslDqnBCHr3 zk@z(v|EjvPJh8A>fY;-Uv!kt*?M-IC-|vcCC%!;Nm(e~J^tGqX0iv59L!z)_D4Rd= zyaXB*!mu!&PognlS-!;u1joBn0Wpkp3CX9Q0`x*|th%6=;R@zz`;;czD%J}&2q#U2 z+ggob`O7pK?n~5?pNcCxJHG+D_NVE}icRIo8V}h?tbnT(`e0W1^a^!}wsH_cRO01G z0Ya}ok&zM;-xRw-q?S7P`bOfQ`Hm%DL}aA*-@gZgjCDZQNf@~|LeC*?Cge8q9YaxD zos0oUJZUFZCZTE4`QzPR#{g;b6c$Cu^+Ht(9)zhO>DSPU6H#Z1IyN!Ewoi^&q^Q5s ztVmJOtplg;qN$vZO(rZKP)*R-epKPF$gx{--x$K7^kH=EjZ|53VKsQ;w){B zLUy}nxB2NA;x6Yd9ZiMPFJ&yq&I)41=QHzUcf5|{MV|&y4;&uRB*3YuoXQ@&A+!r$ zR(`f#W_gm{2b=%2Wh=WW7i(BD=Ga`Q13;}NKiHjpeZLC7J!-sqWLpJo0Zj}<3P1j_ zAy~`6M$#lb0O7(c`{#geh4N-V&v|OBMlNHC*@@A7MPRSbS!1$*>US5~k}R6ZLS&%$ z4*%8-nqu^h3?=@|+yARRPcJir(~AapWuOSY*_!?dHB1qnT;;eu#5GGivr8B7x-b#9x*uycF&i0LnW0)m?#8-Bh*eH5uRFT za;f7%a!C{S6~s+`BA3r+a0d5?(t}=pXf1qWdL0W@^cIN13G1vsz1eI%C;sUr*ttPA z;+E74Z#{d7S*iy1eEKm&Kd^+2A5}HHeMgq&`HpeQyZ4B*mwANOtajkBnhHBsUJH#bcPvx9%#yhXuAJ0kx(5qz{-fFCC$^fOY z|MaXgf~(~~KkjGW+h5Cr3zH2KsguZ%+_rmId=C80?Z8{nh+nc}bQGcvz21bZrfzmG z&hO*<4g2jn*=*l>qF&KxNFUKAUd`OV3smxi(oxev<)5Sx-hSbdmgoQ=7*H%P>(GVj zg%`GpNSKB#<=5d0RXS0JGU4}4M``eO{fWT;mal}Mzx>mKyE9q(^Y4%U6tIc_j%e{Y zyL6EYs%5yR`q0#@i~&Ler{a7`tp)qIr~i3 zB0=s;Q2n2t$tx7nSg|)`tSM^Xlt^t)XTl zT{vXDoZN1GW8>{?MYk@2b&NHN6Lcr&#NkM9i6`3LJlU`O>{*Zw-eDSS) zxs!@w1T3d;W?c>4<@~=HI5Z8P`YUJ_v9gU7xk?6&y#9t54`Tj|))=}!#@*R_C9TG) z9M@oWfLwQN^PFIRw+Q9V%_S($M#~=l=+2N$PthrJHdUjDO-P78*9||RN=QCA-%Ns~ zUW3EOBkSVgQ(UXevX|hM&O>sG?>W>%A&gYbW1BG?l9cBnw&&O9Y^DO4+NYF!u5W?W zqtmqSF!?SEHJ4ADGkj8pP$dt=+!x(Iv%uCl7>|s+yDJ>UbQBK}f%ZjF?aaHYcbd5} z2+Qn!LIddZq7w~bGF_)8a&fYSLswoZG*%tt)g%$drH0XVvQq#2*i>Q zF*3069?*CA)DW%3^7i)C&+?fd=0gef~S*HfiBpcSy(ZoY)EAl^Vmg#cc-D{$P-l=tP+Aq zc$4QQHWI!m=R}-PaB>vfD7Pt6Xd>>h-qt4zwRgHgoVoeg!7*KCUQNsuFCV*_((lMX z-~jWfcq;jIND37+cQT#!gPGJQkB;Xo<MT%@L&uG8RJ<;hDtP+{%KDN$f{4plNCcbN6nMfN2QW`092aBt=II}~)7`wX=y z;<#{+b+FOf9Q?)dn3-}i)-qDw(5^s2_~>7aF=%6&@1+(uRzeigKyc3b7lzpjA%XCk zGkvWPh6SmL!QAUixcW z61y;5%Hc3U++|5ehx9C#UA*IJZI@#i5tp5YIb{Jkeh@bHYLP|W(D8h{%#6l^ySoc^ zi1FK%HX-A81j?`Sq6Xo08@ee%6f;ACXDOST{5!*`%6Yb5+B#8D6Qin`^13T}YQBBb zpYC)(FlmT2qK}w%jYq$O4M&S;Mmr7Jf=DuD=EQ-v($h1bDxX0c8S>>--_mXGJ^E5FK7YTiH26M=--yP-Upf-7QzOo7Bph)uf%@#KwUs&Jh8?Svwrg6aY#bW8@X5-eK z@?~Ug2DzYAmQ0u~U!x=KE%9x1_GbC2W*oAe7|pBE_<;&Q(<`Y96S zAMj~+&{xVCXCya{mS*a)xE}>Ss;h0Ak??EjWT{r^?KPX==L)z=d zhT(#frpKC_SE)=`#<2#UyTAMXabXEL_Cb6=yqP3ew%;aqCt1!aI}CiOsUij5Y~)Hh zm~b2rQYupI$SXu@mLkXWh(yu1iEm<#K0>xZUAW{nS9HCb5g5O*H7)J(KAP!C<;01yQNCkkMS# z*hwa!4&!*mL+o@q?-zfH|l8o{M2_YQr!HBV?6un-K!`uD-#9f5V6XbQUCFm`V z{8Ej%UOyPJ(PA+|A}Yc206rfeC+uzk&G$&L4I8^?-t&v*P&no=CD*QF=l=W5vI-&? z9w8xYD2O72-M^T^J2!GU)#R%p*#Y)h_H-ue1M&Ak!>X;ofN9eXb0?UVShc3_^_~u9 zA;TK0IQpX-7j&Q?=%eb&f~(tG%~m~KT|>iqB@z~+Uax}za4ng%s>FHD;LGeH*}CPo zIr@;rMfNf1WvXeOH6G|{^wXd!cc#YvMpV@*t4qdT=(4a3X6<7jR88QJ?{^MFNN}ws z*2ikq7AMOZ^mxGIEzaQYuWD}UvsuyGQRPIY(n3N)>f9fa0Gm}Q&si z177M!Hamo6?hpi#3{$*04q(~su7*FBC}fA47y?dji#ll-SlDFg3_B3QT=8||2ao{$ zH-fj9++aczuT|sakilYfF69K>6z4rQJQ4Z`NENZZaa2s!=~h*wGc#9iG6&2Xfb-$B zI_p9wjTiRs+!zydblR+Of?lg}Z=1pU0ATd~`TpZ)h|(g*7)Xu!e7?{+lrZft(Agm) z`g(}E(&9VZgEL3Lnx7}^env_9EPHF7Sd2ONZHCV+-t*IOVr#$qal8!J zQYMSBv^Ix-1ST#UjNxP(5H}wu6=nu+{B`#oBI84zIi{i5R!J$|?_-ekb&xVqXvjn1 ze2b}0SXdYcxbAg19O{P`nYrGm;sc2a^4eRVuU&NZ{i^@zJ@5MuVeJc;J)4sDoui1G z2^0BG+24x*Qy>e#-))QB>GyjA%~<4N%*Lm9XY*ZpK4?+&<9S~6_eMW=#_*sZ3PH&5ihfV?#?8&l!Kt|U6eA{l+sh%~i!Vm? zhl?7H#kxxx4zAM56*=TXy4JJ|M~*XvxENK}aR}LjH?93*a&QD7h+QA4kRtVR9SH^0 z;yl|u3ju!%0E_0asjZX!E@)PU89R)rB6Qy#qP1!I|5yNx{`LL+h00LBAka49tj5Q( z-rWPAy|k@qmk{5Nb9V#B$V>=IS$~fLkmLl@Sm9qI(U@W%qR*+K&v%i5^Ez{@INhNy zhj!+pb4$nn{0Q7ZnWZFnSYEm6YvDQ?cqcr=B2S@AI^r9c;7To^(vX(}K9`J~jz3wH z#xrPjOs8y^K8FK?q2GLZDCI|uLdcQtrYQKX86)$`P8{uB*h~}_DsR%7skpTU{oGF` zlJ%Xuxf@?v9zq(@=e#`Y~P4m&(emlrZSD z@TvOFzA<`V4>KnuQwrF9CKeW~0MRWYUgzcsSy)YT9`j({@4c@psVZAf^P3z|Cr#!r z+f+ogT^yAp!?y>*oG3W`uONjNO`aYrbPacxSKd4$tphM|qlcpJH+wBDl2t?Wh$NV( z1NZ{aFaSkCOSx?0iHT|vXJX)FlD8D{p)j7|3~p@na=8-xu}nPP{M~|UQeouK0KgbD z3Vw+bA0bQsi|I9}K&lU`>gIw2{nWEgOv!pJu(t<~>xMmcQxwpJ(A^wL1ysnxpWQ&2 z*u{G^Q5z6REmX01xT6;o){-qNZQsWg$FRt*NZNhZVXg)>Iim=?+OWUR6 z^Q%^o+iJn0GCClWf-hShm>UaRK#E>IESl3t^D#h7{xicx^m#NHY=`dai_0uRSh_im$Rg5{(!>lKCLDYN>8qKVA+T)rq?{~0Ia zz-{^AICuV(|KMNaZt4&q9yR9mJu=@D-TiA!Btnv)pN;btPP-5MYaFBx1M?X3`(NrG z{cD;7{v=XZU@J^?7Rfvh`qvm>j0EQCb9-6p9QoHY1MXy^C|;6Tr27q$EKXG=1T45G zJbDmQD(rtKw%#FPs4SH{)A=ONNlZL>(=QWGC@HT%>J4d_o%aSrcxk5Z7_5^L81!9S zwZ+Zg&J#R7UHGxN+TFM=HZE`})!_y^S(Yrq%vwugoVfm&j#MHP7@b*9ctvPiEag{X zdeph%7}KcLna&Mfr`uZ4(cbdkTuvi%bs8>}4N0snas9>)w1Xa#Mo;&W#;N_hT%zfW z36!M_IBEPSGgqd0arY2rK@(ca&Bw)IS_=*7r`Z1I_!Q#3_n3{1sCjRn4-k;F?vtnC z96eG7#$e5#1+k;q$lyI#MBWY{k3$qLDaXQVh1e3pzPk~-UP%nb>)#vbGaqLP#y?Xw zUa`jRfZEvcb}vYKjqflc9Vpz!erQ3Vrr?WcmC2%B`w0TH~6Thf4a`e>3U7P zeoGQ_Qs(B7r*Kv0K;Jow5@m2(9A(&(w&sYYkbm%nkFo{+WkdE|V z>7gz0YoL%BQH487m7^wkhaT@{yVA)bRR$ zo5O859;)D#{P>i7p*Ti!2=|_A(_eA?ioB|A`==J%&Kc580p};W12%N=o0a zt@=C5POh|9oZQWiJ?bp1tP%{E>+8LeIWp#j_sGmiT!-s-t`v2A7ul=^^KoBL2Es}z zzsgef1>H;N8%2cH5yZpFS}`3?NyeZG6p?I^@?uQbnduz{9K()pdO#(*D1dLK2LH81 z7c%#ui1%xv-mCV4ipbroHiTlm?s7?o6?&Ijwy@#Ua!nhzJV4LHBh($#LtiBixA}8A zH5p_1svc#4AOBuF{iLg5NbFkL#Yh?_DJ>(>s~%Ka>-9d`36DsVby-7OEwK{}wgOkm zWBd7W_0&LOPqk5u3hE+vfKoQyz7sfE-f_^HOG-DhBf{6z+ArLB-V5Mh(wTZ7q8aQ) zIKD&)kvWtx#aI8rZa}jM`3%)Zh505$Ua^zw>OB$Kr%XgnD|i)kUu$XGdO>y~VBD1u z8+Y}nqLywW>!oA3%i@kU&5%C;TLLU=?AbzMUj_t)n(dTO|nV$i~q4NyeQ}>v1^j=|J9@y!zxmXU4G86j{coD5=wKBzFymBHo zcqP>0TP-FoeUSM;OpN@_@!Z_p(z>GU%rZU;`%LhUwRJ*R|{vq z(l9D2XxJfJEmD@45(YIi={3xn3@kjr5W)#_knYV`8#G)?J_TON;Dc*>UtHWkIn_u> zaKJ+*x!80l;jEng-#b1l`$(gQOod9OXjo*td+ZgtK|j=FfaKV);4sM%pBAI@#j&BI z9R2pi)0n8MD$|upFz|f{;3ijzn)@^rjd0FMh0e^)%FJBYs5dLZafk;_RxoW1>WKVm z5eY?745bp&WLK5S#9zjo%maYw418e32q-X>n-`tfOPia;q-w;=c+fF=>-U%740?u! z=mxSvtI;I5Vi^VMLqLv@Ne*zj)NGC>eUz2*=9ktbB+YU?WXsj?%9NMM?f}tiFjd2W zz0IlMp+F%~Opk^YtzBqNSzbMQl(3kv9x}dex~SxVod^HU%KAm+SO&Fd7Gs84DYEg& zS|*ETz`GsN@_FOgWU>i!k>m7$D`o1ELs$k!o@8KWHQftjxJG^5z0=;^L@;TJRY}<6 z;DLf#jZH~fgcJ%#Zb3w!-RfuXY(hUq5YMAZBuk^Fb>O%p?#+;b91O;FUP7CWp|5qu z8^&X#?F-zR>fz%(rxU>SAQD|{XzNWJ{V>bm`sAq81s9uGe7->n!!M+zh44?rNuH;0X@HvrLN_#M1;MfB3gR<5#3}IXd1V zBO~+aj*obr=YJLXv*XU%_oRVSbT~2s_0X*Du4rXFm#4;_o{pGNk5FLtAa+}l_Ott( zWGzs$s`<)}ZY0Ts3BcptP9#(MxY8VJd`@S7ixm~V2dv*5f11#6ZK8(E7{wQA*kjpK zFeZjbSoTDV>pMID6i4pSTuYhzkQLE@w)fv7H7b=&SIVHar6CKv*|Uj6174M!eK5m^ zzYvU~pqrXMAIt()jgM91H|%{XX;V{yau>&;nF-H_S>AoO>DuTKJzc~41)=ZNbWDuU zQ{M5Coe$g4-?HsoJ7Nb$`jIBzsn&60`7wy0@fCz)!cE?a_0M)^U5K3!l$|h z3JK2WFdgopXNo5@<=#+sGT#?y9lwvaKYvI?kf;x^D(pn0YYa{gff9`S!-K3X2Oa9* z=L(5GsBkbHp>WjpGBV?=z3;#1Fqswbr6fdKwpLa~)@UKKD;KBvwM8sGXa1t~^Lj4N zl$GXOJ70xAo~n^&I^AGNF5mf8h9u@f&M1IJr39%A{{yJxTP^O%!=U~2EON|RX}3Sk>+qyOZ4!{H zQfTJ>Smv4W{#QD^#m)8Kj}O1TBieOp=8y!Q-E;%g1MTZ0WHxa;vpNZZC@9ev;}0$Y z5plwD+e@kP$ipSvkLgUkZFT=*Fkf>G$@SMt*4JMiW)TgRDjZg ze#U?bM`e&wT%4V;zmR0morst-%%~@xi@_z+&YYxZz6^XHn5ZZ~%+CI*T%kDKX-$rw zRF)`yq2w4w-$zMvf?aCJN#j5)GFW<9U@6P_sN3DrvX42-v5q9|W=T{0^t#QzER6#% zhfU5A3Xu!LgAl00w16@=HMQ^{B4By^Rr;AThU2~Lb?1NIVV#;xMQfDzZQ#d4BBjn1 zX}SBrHXJ88NE4K+nIHmC(!&k)dT!VBot3`(qL*O&#ayF&By^T_S?g+TP1VfQagwmr zgjy5fG2WOqAJ@QOGK*^=3ZMXokzlf}WvrEP>Jsnw1<3Kz^9=<~^ck~eH*%+^*mzeI z4qTHa18vnyKt zLY>zY=9JmE8f_CJa9{=?7`TLXSAJicSe=g{?Z5Kmj*YK~f0{DT)g8T~H#KLp=$@x^ zv_MEO*J5jIBBaQWx`Dm-VrD#OwVvO&;KDH)kbRnQBN8#0vYgLXBDH`ov#TuuQlT^| zEs~+AjNp4FrTgPiJ3wC<`f6)6YStj;^$X#oi%-PoE{9B!9&&8)91i+5QZFmK42fY3 zVJJU^u3JSHLlNe0%y=z=KhTatE-;j(bs9LP_kaeq5}5ni+tYbRylPNUroJII_@pd` zt)+zazT_1Wjj?IPvrv-=mI)~lub7jHb_tQ^LHDh$tZ)cf-_Les5=2cktRab&5c8WZeti3hQ2EF%4cwk=ET-pVIQGaP zTKjWcYiq4AsVPfKZ$K_OBeFya-r;cKL|4D|1jfKCgHFBT*@crG+RK!&ni1hk;{06O zJ9_ly%%-!2y78e#t+yRbA1DT5r_tgPjxCV5TLw)PP{DHY)nur^Dg3h8Y)6xpDKKZIsNP*YLrvQ&)lU^%tyv;V-HQL(P*T>4r#wKa#6d7La`fw=v zu!?jc!(GZDymk*hW_3JbNO!|M&7DnXsBVfkD#KffMnaYBuWBB1=)<=!w-97oTy9+_ zLt`$=vIrvqQb!MdY<+3^tEc1wYVR}3?oOFRel0e#;tlY!mTg1!o(}CD zv$#y|ejpdtW-(353o^%OT{z7})>3V$EeOj$5S{yd&%z*|;u==b!mLmgo;E>Gq{G5m zsp2hFS5p&ColR1PKN5QKXvDz5q|&`BT{<842$Dubv^qjB2r7^=sZt8yLxn7&ZC<*q ztu5iiwuT@UnSYKy@O1EQrYvoEEcBj$6e@z!lM!>0Ji;D(A=BB zW2bpRdpSFa-_l5pf6m9QPPmqU3xf2*&OEVUX(pOjzUx$_kjed3+iqrnL`wtX0@)!; zv(5)StdiCzU`JfVZ~hV`E8YNkxDXtrw67j65B12hg!BYsEO2M zyEa0`ZQMD)iTBdzeElIz7gyZ6qnV52QNiQdEVr7yfu$*B@ z+Q`$mN_~EuMO3e?5G~~{3@kl3i>>r!(413Uh-@3$K84U?qj*yfj>v>^)Iq{~5>{P)Nu0!+NAxFRS2(Mwz$-2l(8ee1q|tuU<6zsCPg zL&NGf@gKS5VCw{U|La!$f3^Or>^S^0jgrw*(!WHPp!0$DNkFD){quKWfh{w~ZJJuH zTULX^#WwFgQD3^-FTp-fZfv(QMF%9$cbt2D+GUbI(IeoQir*>6-Pz{C%7E(tnzfwR zsa#pvkhSy@2#Jfk@%~8DH_|k`B>n5?-1Qj6%jNPw6n5B^YUMUupA3m{ITh)_z%jIQ zt{BJXdTylk0|M)aGo}3FM1*0(TN5& zh=8_H_i0{;0gVR6M7#)T0=GmWV&da7mnDsSBy~bz;M!*!;WF_n3C7H#nMJ`Pd;d&7Mb;`;Cq zS2Ah+YCfN>0+O&$iG@d^A`3375zUx1?UekFGAzsROmI!NqOQ*a_ZmXN)lNmM%;tRq zD&S_Z7=(mM&5Oy>g|9Jg1N*A^`MU-Q^X++F#Am~fV}7M`4%&-T^<{3Aj9018DUtwS%o#eNRNs`-ccX3xT_JR}vnlyOv|Kb-Xw63v&8*4qf#{UWQO&ozMev^dU&)A^UA7g(xhgDXgn@Z9$l-D5jUR(pF?(^ zXt7L`iyZ;kI>~vTI~D>J<3|qNRdyG9`e;JRUe>gPcS*a_DKi!M?UR?0IN7yxfFz=e zO*k{HG)|;eL1fnwGVLuO4SWdM4%{RRPDN_tsmS@Q-4bbB;ap3r#uJ)!0NKfef(1L% zfKu6) z3az>Ncb3lIQS*LLAu=Ro@cmuPl+qt>4<9nSmTTj3L-dh;lYl_M8j&|6AD1y^CZa2I zF=lpTpd-%085va#`~bTUf0ZeA%3_HR#KVe=lpzbQJgQG=s$!-z~KNr+-pyfVjXCH{Hp6ATt3pN#`{ax>l}!4+Tw>mXJfOt1wzwEke7)1{uH)q35kp94H;w`3 zH*y5=Tm6hhA4W#RL^uS@Z=t?{fe?fAUWxYbj&~YWQi6KZ99T}z4m3+0qRTqRxFmcp&Jx; z2ald9$naz1tkYnxrqfC!D7=JS+>Dq(?BAed%OK3Rj*%85%)q{|@@(;2Ev@Ywhd9=> z05sXKWicE5AqveHf_UMF$K^5kho~|gbWZ_ouTcV{#yx5)!TlBv@n=<bWMFZxa=evRJ0^&WzAPq+069EZQ9tz! z=7S9^SVnGXeop{$&SxrJ)sqq8=uswQ?7t43tIup)ILuCvG0FlUSD5-mp69BbqTI5(iH6Z`$qS^8^ukX zM*174D#ObN;AF3N_D94Q1FJ#=B_Sh-!$chpkFd@bO92H5qU~dI~dV$7YT+jiYzTVwV7POs8fzUHx z>&MojQ!q90E8)Etvr`1F0U_3k&aq)6RKciBQd&R{+i%v7Q{PeBEaq5^-kFP;)56!2 zKogGva6JTGhpKf=lD+rZexJX$ouBwe8TR|!-2s6E$Ow(1OA!+SB9IedB;OE?pN39G z!v6V%zql=8e}kC z0B76eqL^+dn$k=*5#l`8i8-R6j691on?7OB>-cI*&$2}_V??ik!sa3IOA6$L!Uyhb z@abxM^knsxfU*PIRm-cU*6Pkj#RnxjXC=>d%H(sn6hy4TG`c1eb0a7-IVcX}(=7%b zKJmP)`CtD0tY^@E0Cmnv4M76)SW!oF^>bt4)BI;MMrNR=^1Pt`EkUqEA)~Uv&ue`6 zXIy%VNVI0vn|BCNka>GzHiMRtnw;1X{qF;Si|+lB?NQ@kn(-^}ceUuxn=36T%fI$^ zPVc3n@0j{(`1jop;LX8Sc9l~1Fv@4pv(Z!$8S>`5wh8I- z1AUGr;$_^6gIgn=dw&KPRNr@zB|d8J#q}wJ?P4#w6dJ}4Hc`FAeLO5 zkSdI0{g5T5)8;ybroq3B%(TxhkLOwyz8D3~-UPnVI-|kh6+q$}R43_;(2&L%Ti3Gz z?qYzTr!RwSf1d@~S8(7h>`Hs@*4{hgkk{RaoA0adwC z#$a+@eGi+OlB+T>%E`{Yz53DNxHB_vK4xRmNP5g7bka3*b3;VkJl&zTm+^uB{Av~Z zB_M!(j{X7`65e zLli>o?c8^|Q^nX!*BaI!CM`vJL|taqbC*?sH+qRU1LlUYIM#XZYDxIx>dFEV&pYV6 zyqz1bLcNlQ7hW;^0v!h(4~xHsPD_PJM<|sNr}%_A2|aOLoIeyi_`IC7i`=?+C_1|N z9y~RI{X4UInVw|S7CM>3K*w-nhXln@Gm6_BQZBN2)Lh9|i`+fw%16T=AU1aP=Dt3d zNP0B?kf+&sS~XlZzLc-Mk5&Lh9TSyO*3*ZJoquwZ6%YF+sJ14VmdVv4M}xm(Bzy_g zA*VE??2yQVrk5Zur#=2Fp>T&6lHrC4zkXcINFO5Uh0V-)cng*~3hft3h9p2;z>Fs# z!|@L%CU~e3kZ$Rc<%qEJ{U)GU*A_Efx#5H1KU78!dx3v>HybWc_+^~$0B9Uo<4i4A zr%FA=wzM0Lg=ECxn3 z2~Ieu4fwxv8S1kf*%F* zk`IFg^LJ1v1JY4>g~$0#x@XEf4UWzc_#hc+c(PKiy%&t2>8$vPLk85_Ti^++IrwB1 zE?LSqOs!D*_cDb95s`w!%1vHH1rJPiR8LkDah_$>slI@B)v4HqmTPIR^XCrU+YCnW zV_+sLW`+*5n$TF94k90R^~hskDQWimFOaE+r$wp~#1HP^`1NUK zhUgj`Du_269maFXMostU#DvC^NoYFN!ZKA|(JhvYg`tp>J?_KQ#fHvT?MyVqBY>Qj zP`K?7mR4<)sjG{=OnF5z0GpykVfq55g6>=pwMoX9*M2W<(ohZKPV_hpP7{2LCWo4l znTtGjMnPe2i3+~JkDFD3ui;YghwTKr%|*-9`3Hz>DohW}p$H5Ptf1BF6Dv3z8>9xb zJyH)g(fkI|de&DY0q||S8r0cgc8!gwiTiieJ?IK+rnG>hL$OI4JugNVqTznW-pb`9 z1-$3OoJt_2O1fkzMSggOEeX+*GR#W8R0m2we@BVNwB^NiQQHj`+H7D*vy^c!KSp(T z%nCr*a;0m-t5Ni2jhQl81T^J7Af#mEEzqz>KfxwQ}z`-=jRf~8| zFyL;uYe*QpK@zIaIU+ZCz$L8oTjJ?0dwiH*dHWy#@0;V?^Q|<|g(Q&iua_ZA^?E4-8vDWx<2-Ps(8&>ycR7)$o zJPv#KDOxvIHT3F$UuIUmU{57n_uEg&>4Ockm^8%PrPtjUvd zWFzhY$*JEI$zO&m88G?q@^~9f0Lq^SV~bDkuh8raZ`_lq4w^BlgdIo4Aqi0R5`~THQ?9PGzYx|;lzx-QJ5rU2RAKu<7sIE5Z*2LZ2-QC^Y z-F4&cP6+NAG(d27hu~r3?iL`pLxP6j>6O3FsXkv<^=%h-6jJQXTI-#2jPZ;a;37|$ zu`uud?~-I?7WjPQ1KI6;z<<0lI9~u&4e!fPf6yG{|M3x4n-6^6_KNbReE>Dc)6d== zvA&yN^;L1-o2`K@gH73VRak(>x(6>Ct5e6>+FB=m|6K&}^!|KyF%64UoKKH{{~=@I zcRUbxfa(08=W3SJ&mLcMM8@(WJ_!`@$ux@Srq;%{L~-XA%JhP4 z?BChUR-x*`{G8Tx-dNw4Z?g3%=HgrI{npj`&W<{pYsx_b?=S3z;dYlYggqiPBo%eK zdOW&Q|9mbL3Hi#~{7`b_VFc>QBowe}S27J(f;8V%@`4Y=2I#R)rUDsH{c%pn|s$_|NA?)i1>Fd^DF{LWWdSAGGE82&VY ziII8)&k8!{CHiFZSpU2U^qn5uqob{@&F$6b&M%8K0y=^Ra3--@2y!22e_eHA6T@Sf z=9`w5QUlw2Y3ZTUk8m_fJ9^1ZBB}~mRm_iFXBn8m@2{}OWY_6iY9wq)HNfa^7I;>^ zs+X%B&7W`^@2PWs8~@$zGc&n67VUFH4zavlk$Vu$t{Id*w#w@@%!v+Qgn>+3Fbl6V9b98in ze{flIt2U+V0evVJ@0P#}?I8ps_Qj(dlSsJu=;E5$TtEVq#E~ChnY|9F(?BxFGRM?A z-6Q_1$D0H9Qf`Uq4wJ62rB6VCu0fp+t4Ibl*Aa7LD7ip1PifROD~xv$e>FlhH6e=e zt9m@QYsDrY4gG$2n4P9cYzNenQBPl{#R_sza=OTXsF1P|Ctn4}WI^ADv-Z?fOhBjm z9nt(LRslzNadUfN7WRO~>a@9=0z~Y`zwPhs>l^IsoDt34O=--EzM6S(7Hc$^Qnjq8 z?n&D}ZU%Ts(eCF*r|~^SKXkgwhD6+A&cWx0+EQ?$(HOFOwDt|#1p$L4MCu?_u@$;u^!+1Zo*QgcLxAL6CHx02-+&!Va~Qf>Q_(C2j8n- zl)`m3T2`Y{_3Lt%Jv(m&^o2ob!v;7x4+@G;?Yz7R%F2Mf_Yt|VHi7#hR5LKpSM;Gd z<;RCPVlD*azrz)e!03Ll1`OtdDT`K3Xz|QY6(PHHzr)c+DqRA(5X}Wb`n$@HuJ?w% z-3g~``|S|t6b_gopqmi^r*MxWXbDJX$axw160uO~P^pTyn;VaT9UwM4J*JB|dHMlR zivLATC5oH3ySfY^m9PC4Z_C4_om|p&=q|oG#1735Ym2Vz#&?xihs{*|aduDvw^AhTu@F1k@T(-mYOz5X2d?Cre-SJdn2}+=O zN?q)CwYuobCAR=XnNS(2hpKwT(CHCigLm_&!AwXcNUU8dgMYurF6_MOf5D+qqsugC zXtX+p!Q$oRFBJ13GC?LAJcE6F1Y)%0e;>y}eD`GneMzHz1k57PTm4TFoohcl+&Aof zn)&f3j*VLsA_PM@`uy*EWe8QlHW)Af^{UkPk1^s>50V1et#LSb@WD-P&CF&*vP204fVi{oWqj z{ytoo{pk)1gUomZfc!sZvU#(v_jX=Hah8EV?=Uw{mIiW?Y8>Ga*LwCg9^f`3bTu_K zU7q6j2DE17%tBOP&Vp}x#X&WrFp z82Tjfh0C)<6vU-7Dkq=AXt-3gs)=DRh;WeMU}F;yhB+`OYtZ@1Cceiv2hMqip>FfBNsWe7qsJN;f)zSdVMj_;^vRdDFrwe zZqD5=$fWS^!rrG4htspOMeUq~ShiG8edrX{V?!}Ou z>T0ed$t=?FmZCV3ZMHqeNa@wzQ5bEEvjRwcM0jnEiJ@4cViT(iCurtZ4Y*Qq+TC`< z@R;UPSW**K#G4>$*0Ba9@gAfuQF-81=%OX8G{+$h@Jn+=WL?!^ouEfEIQyK++K`88 ztWD%x*Vcx|nw>O9?;!&5_!40;5g!NO2|x9nO-_mep%q{=X%1LuMqIo*8T11XJ~y>; z^1X$2d#1rhLV8iVSV$&B$UDz4@Hm_DeJqFxcDKa29g5-YIJAu9VhH@;*NVcCh25AU z{Vs)vPecmL_$DNH7b{_ zafj?_uVOCoHoqADpjD&3NxGIgf}|b0@|pBle?DScxzLQ92zie`8#te<1vJ2v@Cau1Y%WpT95`~F3d=1MN4bq112TfwlG;)D@3GsxKdGY^vo!2V9DDwxY78p z+N(Et#u9~c&J>{(`fIE1HzvBeKU$se^>pbr5U#$0gz2oZgr-qdMx|JH86%wE(ksA7 z`EYF5b#^k#Gp|k-{CROd0x2ax=?Bz-^TvSu;G^f(z9e#^Q=f zS*E82FFO?io2W0vQ{(2+FDol-;kTx&pxo;Bl@%aaI4}0}xB3U%!I&;tV$>zbIx^Tj z4SrR#1{ae4iM*COkas_}zBbt1CRGTDi+`*EZ2)$OqV|3$W=7A9F-IDTJFM0V_3d*@ zK0068VGX_JBLS>MhJ%a0#wHpsnJu%sAoF2`Pb-RwijGG#dZ-oZz=lV)iU#FTWPd9U z4k`f|*&fN60ssOkIiL)!-A*zc5g<=cr?FbuDjf&qs1i0iH?Xpot1ym2tvS^Yer*i? zJrg(qdyVUNymt>6GX>J1daUR2-JEHlCjEYULn8^^KJLe1H!+(qa-Nb3YOqwPQLr0h zVG?VR{D^3_H{d$w`kkvgb{dLJ?%hBF%>w2{tGMuCdsu18sxB-l+l5B5Xk6`hh|*`m zwVMt94Y*4(Fs33KreF1~ZyNz^WqZn$F3 za`fT$E~-Km@N`a$W33;oz-iA9;9#z-@bW~mf)r6`V8vQB!q5r7|5hi6jPhW7oH0_{ zrxwz;o8}A+E6f=1W-4Jt>V#*%hxqQrLMdd}Z3L=sDPK2GF&ZvI<~a%aneX?*kRc$p z;(Stk=96(fxyg~Fvre62O)c}u$rQvA=9G~`1@H!2qDADVLSP)CFDsh`rfWm~_;LJ` zLeo`=+xZmIaK1aft+t@9swzrk)j>j(2i>~J0D;1P^Z|-c^M?VI4c-9;&J<1GZ_HUO z;6c&lE5LZfO*S>P+@lw$NZ7D8X6-&uqsYBz+A8kk=$L~auom`|tqD13wLe_*IWSdV z`19VL1&Yx$HjAo%6`Z3>5-M661WYKGMrE==M^@&@h2sjIgsiN)o(iSZaBitoO@Tq; z45MIiRj}(7Hs48+O=uZcdDN8c2iU>bZ|=qD^GzXt#o$D`v2+O4R1%rV=P#i!Wflfd zmQl%V!>bv2Q%i>zen5FY{QADM23+U(rtKYM2jiq!Qf50t3?tommzKic1_R_jA{a?? zCMqsv@!J6xIt>>U)d6-+W-wDCIguWI1iqk%Ny9k6%`)U}2jNMi?ZaBL-Gr(#wY|Lbi0oJvLmnXo8B)!d}9uuwc3wmzC)2Z`Ze7*7E~ zwcX)iSG%FB-cn+FYP0KUyfB;JxrnJzYpdUiR6$mQfvR;rUE(Cx9P z2V#CL+!Ngo!5?I@wF_IxtoW#^g}SW;1u0=CHq_j-C(k1_EjF8!S-(_4j1=LSIg-(( zVd-V9HQ{&?q45l;KO`eFe7|jh3WT{RC8^s>O3utg+T?0zsE(7LA9@)tvW~ST4?9!J zNmU`&7mIa)7Dd=Xv~`VV#nuQXv&d_04)4QTVb#ALVAgJDN|q0$#5sDUiXmO03m*;I zV=1Fa<8nZv<1vzvb5TL6^gEL7j65aSPSdVK@NI9}SR`7EK)^($T8-uDeB+|Ho;40xX6Wnzzop%*==tt}g%jFoFzw6myA)aLrZug@nLsMxY`5HXa z_CggWjR(bE>%e?uTXhPSh9-!GxJ@B|K&8a;mKS-Ox@_>9A1qJB?!Q4r#{}JZDQ}d)Avq4|Of)EDEQ&q`AuG9I0~w=& z7AB3!__cACgIR-E8KE;p46+2C1^Q424xSv^^u`$QUCCGqhW zHU-#38&}I!TU-Rh_jGFb>7{_=oW>~Yk~sTn0~soBc_IF`5n0dgyG2YL)r$Gj>U6L& zY%FY5o|c?8*p{X4dVMl<7vjLq+&;~0%6HImRD$tTb8^czB0`6UuVLH6V)CQ`Rd5Rr z5}wzh>rS$Tqp(&~e z9aUy|vDOOF0efxEuq9sa=Ji*Id=R^prPm=`Kd0EpD2sf*HCr)NT51GjED8iXbm?az zUdOvYA&!zSNLXo)$)8-b#bN{)4N;z`#9`YoOW<{|?E{2*GQ8``)xN;|DI0`8l)b7X zu3qaaOb{VZUcVV{3I-x0xft1-nHYz`IVeg$GhR`NpEXz_T#Av0WGE@x4*>Al+B1 zP)YtQo)#PYx7qsHd!zo}MOXtWU>o#zxo>qV|Ia2Y^xtA@BjBL$znL);V!*m|y>V){ zqx65Q!GM*RFp%|XyBih%Z)B{K4^Tq(+85^fGyl7<^ZjR5JV^@qZ(_{d#U1dt^bNgj zEiK*6`iV{dR5FPk^NHB%9;5X#X_oW%bxnRz=&HfLD)$fUyqgsUdH}BJ3<=_`I9)O7 zQA|>HUeCXvlL^FR(YW z6jNR5Y(cdp11Lv3e_zB!y-zVzxhv?&)`GT1gI*u+9`gq9V{gf^ZxCH4&QI-MF73o$ zkiT6c(;Uv$k5-IInfi^X)_=Ppz7p~}M&a+-9}NWJP=o_9NJqfd0)x3g9Oz2B40&q| z@dGBq%}0y{%L8iHvEc~t%}Jq~y}e4qpHw(`iml(aDlDm6buYjC{*HEeophdO;dvBO z4A0k(vWz8@5|a`S$tG=;jERPnbBZ{PnH+`HdRNE3v=K@ItnjerIq^RxkzYmGAbrU2 z<6yM@lz)37Zir1!7-qDd`WXnp!dnT7-70xWJki~U2v0)(qRT%_{&5Bad6p10LJKtoE-wep$olYXrpLl# z1-WX|K~HYq21VBl|K=JS0rcSaS9Y=V?3}6c>ywmk{(xU|Iu-OLJI0vVz)3fim;IPC*!S~nWosfj&3gJk%HX<@5 zue`&(*C;UqNyfi3at*q(J8J-p_d07<~P$ z@1v>ZjW(}ua%C4yo* z`y`fR1j-}YN)+gYKXdF$nhi&|4N{jLa0ucDg3gGqdc>V@ahP$$9y?+@8CO}8gvpD> z1kB6$BPmzK-On0&QjVlll7otZ&TZEFkLuCWU{e#=wiLx>@Nt8m?Fyg0BH_~(Pn6Co zyu}`ynBjYqu9c*(6Q3(8GD37wq_V>gHFv$7NgE|QPX>i!O|;B`V(0Kgfb4|7LPl=V`@ zS*TAA6WOtdP+`q|yv0Gs+W>Fu4Xt7G9WRn0aL_dekF<3WbcQuAFsjileq^xFg32dG zruP)N{vkziq|c)evm&3zwdr|8_V<2CeFqq{4d65XRPIJ>n!NjE#sE+X1>ZA)Zrmr$ zU?RR@m4f3p)Ex@MVqLx30l+Qr(orsR2ubvCwNqJW1)@*VE)+Hkl z8yp|lb5N9*hktjlwLNxrmAxgvPb3W_Kfyk*nruXOJ#uuab=h)1f-bPQL7Zxf;nS!~ zXcMb}$x5je=}bmeZWrqwAc@Up&xj*p->!9f{J9%ZV*9mBW92{yA&CY^ex%&g(_&|- z7IvP;NG+UAeWwF%w_9}3U%!yDD-z1?S}6%*SbdIh;q?_Cn&9&6dbbV-OY7^f z`=l@Bp(IxUk7AOQ4W!Us-u(B~)j@uc-=ALv&w(t=j%gi~5V@guzn2T#SVlV#wB3fY z@}@IRC0EUy=OCOQgapKKLo_S`O{|QJjBeg#8j3=cV8*VsuhVNqLci`p*CNc4&66S{ z-$j+Qc9ln^c{ElQUBfY`{OKfe72tvsMIgH!)jnoh6-EO`s03>Ej6-IuKjcD@|c%fgi+{d=ApoGiq-Nl^-)AnQ2b5-u1$&pn&0UK6-^K zFPA#Q!b>T6wea0Fw#K8U;B6%-yRIWpKw5D0&XH47(@{Y73gq)8 zKc5ykkj4iGC7u2hd;?7}kQIjpg#(2=?*WAX81x9S9BCo@&0(RxpKgWQrz?)&)$FNb zM5#N3Gq#KFx54&6|*uHsYcs29c|L2w7AVJuLyvx+J^3uwNKt)rsVsm!g7U{0VcKfQg4UKRq#DGS} z(e^DHA_rhT5+RRuLkR8hXj{q|o=M0~uTr2`S;kod|LRk-Upr31S{Cj8#P&X8Qnd6| z5N!=Ct7T(FiU)#gw3Q%GlpwUJKCJfjeXzE4a&kJblT&&!)Hbe#@Mr1BQek6bVUPfg zg%j(7F0}O6#0Zc%y&(@uW~9;tKD%W>rJ+n|2NE||2~I@tu_-weISjz;+4rU)N2S3L zL(o{zga)`f+p7H>+CYJ`ws;@Fo2tO8#|Xf%F5Diy-v~g`=lrbfcc*|yGp(X_3sfh= zLCDTBhV`mst=DMN?cG1~4K^G|`QBtxW5+q@DUC-2!onzb$i|yI6=g|+D_d%?E-0v& zB!VKgP5uJ6Gm%l*hO$TR&E3!fDCav_b@e~d(K?%4n^lg=dY>2$Qk59MPkzeh+0$zNX|K08**&*h`Tu9ola(MWCdRA{|GR02wkWFL#mK(gHCvAWATr@zWnp z&KgZII0bu7y-0`3mX<{8$pLNMnbqu$3W6o8L`e^h7q3M0Ry=qO2s7f=FP!o^G@W7G z$#-7i$ha-3g4hv9PQ5h|UOqR{`(%h_~%JKKIK&(4+w z+3#|DFbV&F6luj`B#~!ey`ml&APsfoki6416>-ryo(fUu{N>**dYOvpQ33aow^*jb zFn_iol>w&diM!b&Z=>Q5Ff})f!~13dC+A-*e^){+_u9>j!aWoukZU92b@_bg6>=R>37maHt1pp;g}4$iKu%e12=LFZAs@$mu= zvSg|F7ixjICWGrB>&20Qp)`JGv13-e)4VVFviyV6iTrIDKU_9ZW_-B4S{=zvVkMK9 zUJ5^U!=fSK*gB3mQwNMRWhi^KR6z1U=2x4m`9kr7FCd9md;@@&#We1Ed3BjWA6PJr zr?}ecad|2Zyfo5a3v;eUY-|ML5%6(S-mE8<8w7+Q)?ri5arFVEn>vZuuilyP92oLw zO>9#FgGMIi=HCPh6=5BA8V;buT6<6%Y(go}d6q^YtA{hubK9&ZjrWhGy3Y*K}oJb z_vzYfnh`mKb_r&Hz9jk;-7!x_LiNO#R6dY`+E-pcLMfBKYj&b(W>%U0qZ2_gB#Dc% zW4^DuJ4RIPeQ?tRaxUb?Z3aI23JGJznW|9^%5*_c!%hx5Yhri z6=-^{5KdMT1hC-n*PD#*vW?P-aZ4|`qvwSalY9q^G8aIWRW+ujx5NPPJG_g0Ha(1Un#P9e-wM(Hz zA6^58kHf-FZElJAd62umI>Bf+;*_w1`hFrNG2?d0$AkbmE?$L>aHB1Mme#OIAy)R9 zvV=kD6Mi!$)*}aZgr>=lJkU#(#rPDyj}HrI9*aTc-8rtSCDf8ohC$r@o0Q<7q%s^{ zdSVH7ZV=OsRHiG=+XD5{ZJf$Y0T;4sBJMYM_A*`Ks;Kw4uJOWf$np?MQL zJURsRI(J=L=Zs=|FZ-R|=4GKqJ_2dsYBzhss9m1{KE1gVCp2T6N+as59Yk%Nl{~%o zfKi^+ur@EhzI{*UDPstk7 zKfxCoP?Je8xF1X)AalBkktib|c<<8`WfYbs!YkR#(49VK>6-=Jj988R;#)lXr{hN@ zvBU;reDR>rQED(iU}XN#l*^^{h3F&5UTlEGU8JtrNhxtBBZA8=0xCeA5?#&Q08hN( zS+egF&T@kT#^sR^n|)*(e$d3a9qw>f4gh%TcYlSYCJ(hY0hSQOzTtN?tF7b>LG!fu zj2uE-PzERU!lSUY-Y9={U>*;AIaU{)6BYUrO){)g?fqr(E-)>^dUWdrB9x1pCJ}ff z4I3Mc<&BDK^aGRHS=t*Lt^<)eFw}1GYuqOxV~cq2mk~sCUN*Keg#;aHVf&^Ru~+2G zrWFm?ikaSI35`d84o67B>cD0rnl`RZQZRgh^Q%wrimY;OLph2du#S#pfGNeS;w+M! zkti#R@h{xUf)!wxB)tGSZ)6g%Ist9vRzaPTaN^Le`G~S~=eslPgy-c}2jx_eftgWm zVPR=lxTTGU8;zLIVjOzIEySxO-$)|--<*2rMB^~YAHK{>xCHpwx)!lm#Yem@_|`&W z8hx?FZWfT$5ICOwf4VOkTAODx(K|em2+T3wicUU&eO~zvat$$$@a$*9`=`&V5qio^$F<48 zc)&cP9wS*DA}*D@jFbpO9V66vIx&i>FSdTpsv|O_{qsXc)-1aRcZm?eP|T8E??Q<) zCo%c{@4R-*lfNnZz%yKPYvO=#f4qLt+S+E&oFcV4;yNHB>qAyXy+YrgF17MgcH4Ht zx;N(ty#0t2;nf|G$wHBe)U%kG)s16435k&PJ~|guk+d(6?guz>?Wl2t;ap(#bp*}* z7V(xhqtFhh%f*BecjCh8z<{`K_lM^SkdxBcA0-hT<#ppWx!Lb_PF_1b_PcZR-hGR9b#W=O5eaYFdDU zFaElIAVo>`uB6ly9_nLo>!5gG>gK|T(sAho=I09l?5}91&Gj^Zk2f%CV!?us8p=hR znx3uXOj?th&w`OXnp%y>(Do1u5|otkYyXXM(oO&x-o^7*4f%R-xq%o)XfNkAZV{vQ z<|$!@%-=}0)%@of*5L3+hyXsJ-_b3X_O+$FVBg4kGax*p-Sbu%E96BY6v=-AG4Gr_ zA(%4~C2RIjDVX;qjaZ%23?kq+X2nnaw@EfI(AfEh9q&VcB!LBOqG=?O0qRt92b6x6 z-qztEPRt1bJqmP5^Qf!|>ZdIBHHtvxv=Yhn0Lny9YpKOMZ8^!hmDoKUk_-@|jA*e| zyY@!nFznyBtj~dl@+HC>hsm^z^)Ai;V+@aQpLnEE*H%fPt)v#Bk1svyAsl zK?AVf6cWsV66l8t>9|!0{3WxJh)R$QiVEOL#L4Ut0SV`1=V}>0oOk3dCumrPaB1mY zdK9ygcrzf#MUx$%33aiNr3tW+G#(tW!e)BVHKZ1%Zfre)rp*S%0d-FYDLdXPFYoRk zR7tVA`$?qfGB-~4;{>vVM1U5^(Q+;oOiRt27vt2WK{3Ef7|pdEX2>V#w}PI{eI{hD z@j0M6#pq>^ciYDv1f7+?banVw;kU94{Mgwasj z#F;0aN=s`Ndnzo*Qk5h3c`r_fy+BY^V|~LMTpl1}usf?eX>Ubk5qfq2bm1VGEUoU} z8|8opuz3boPd491xQLDt_F;FL?xZ{(R|a9?xp?tr;{4lw0_%n;?R(zXPqfa>PZVE9 zcHP2uRnDjJ=8JgSfxnn~L;|5hI;O>j9&S9A5Pn;B4IXpD1wOKd-c;drL8qP7 zRo8TQeX&SmqxEdyO=yniB)N~3;G%9LA42@aXv+lu1$P&b5j@I|6U*0Y;hUC<7TB6( zuKxx>ivlAE&nPYZ=v+FsiX2EATC8(v@$uuE|3TXV;$S(eCsiI-s=JOY2FILsLr~L# zVbvpG7?N;T22NIgl&`x95?qK9qI6f?px`YIn4%Yv%&eK8!SzzSE_PElu}uT7c061O zO(mAalg{8X3?|Se?xy1mOkB7$IqmhTt|nNu0||@vAnF;l?L#MV7 z8a)WjGE_7o5&HyXK*uC(dO_&vB8=2ir?O2VD8Dj@x*fKQx+WFi79ml`wI#q$Xkaf8 zOb!3nB~rPsH#jz*-~#pmLJJDp<$jk&!*xK&2ETb=Q9h8I9b|+(A55vSpAf{Ou{7o7! z2J6@!I{hEM+X)naWARwR1!Iz|5~GGCrhl3Cbbhk;w~wVo>Tgnft>FJbPM4Pb`UqmF z;YfvDL|1t_A5ye%*R?qs34?StbspS^)ww0S&sk50cq1@hQ0wdKNiuBO#+<_xUFCKj z?C@(F3Mu=5>(BZZNu=z3q9-l7? z1D+kMdIbELutpwj)^KYN*Vh?#hz#rwf$CR6TAYUyQ@)TaNQTp;9GSE_aPzWc?0 z?9IbTk_e#FI=n0{RYG%Yq+0$$E%5SdY5NIG+7FWuiJsl$+YOCLc_T~0oNK(Xv?p+S3}MKgA0k8p=hwoSS#<+ z$GwV`EK@u0&LATankoMBeGce%Ad98v0;Dbi*lL5KCy~Fech?xs&%n=~t0)9S?z*cfDAx?yZUSH3Al_w~E{SJ^6n{{PmtfOLOp;Ya5Gru#1eMQwhT z`0syVh`SsA=niXg;xBrE|8b(P15AP9eE#>i|HyR%?cjgL^`zj3`uqO_*?3LT1Q_O} zgx$jFO)#fdX0~2v&06K;$4a=}&q%gqyA}$F_qql@*X|*D+1PtGTx&P5(xx%rdK5~8 zdOnT{Itv_S#_bJSGYzJa?hGPzIj-yVFq03%LzjDmu%Ho?t-o#{HxBz@>-%2qd=uY~ zcXce5_5ATIvfPZ6TAaJTi#m6z1pgS{om5iR+Z!fd;PuGUYVp7<4td~)c0lz<#wShN z)xvp2#IQU5_XJJG&$kD!)J~%$*vqRcsaUgala7ZUkCIR)Pjp>ecxRV73350Nj||p) z(~iLK^a1N=E?tK{r1)DbN!J{!!~7AQ)v`@@yV7ijNBN_=;uJ$BY)P((V;3@6ot8pZ z8z1FH!X%b-*Ooc12G+Yfi55E0{YuW|9~t*`dyPq ziSi5c^Vf#r;Ljt=O=uZzSa}fDLf8|PfHmhPTRdHTkHq%{QK71Goo@D^`;Wi^8(scz zici;+*TsXp)6C3YHkeA+>%<}Lra$j;K>3%Flai8>k&Tk(%$!8}8YxY*6VE0)&*myd z9G5)RVE|wYTV6Z}o1(r@Zi`zD18i)8%55WcsWU2qg(-;v2 z&&houHUj&M6f3=I*iZ$C8#ZR;voNZO2<=jZ%NlY6oIP-CV(cdre%E=~GQ!se-IsfJ zh#^SlJXIR^Bf4J(?h%X)CyB2XMXiei&=~QNBXAM5D|gF%QHSfUAWgSDeZEP@!-CR!mC}>L5EWo40tox#386$J}=x_lQ3E4{JZVlKB zHKc5Ip9c7y+KoqkRp&nf&z4_@0w?Z7_)ANOOarIf!ot*A!|Kfm0yp3sr7@I-=P+5! zU%L{NoiJc8igs4mq4{Q`LXi64UB#u(K>z8;l8=GV-Hccnc3e^F;dciRF5Fan!{xYVlj;3bPZ8*9>u}lQ*n)2?@ zG)XLmX`j^xC%OH`V+9~0cdzE;gv>YqDR5`=TPmB7ls<)z8_S8PUD!Yi3y`TDojV2G zG39^1T61SyZHX)J(1K#9k%d*m(<4K-kQqTrb!A7qE0$-&dXxKD6eG_nM8pl_ndm$i zZ{_&yC_e34-hOaNcn}>W_#h%33u^1qtl5?@htSe&{xZZvWg6QN5BE;ZVGv~l1N+!% zOlEXp28wTGD<1yJ2G47;=>%5PL0EypY_T-_^xQ+cT2p`Fiy{m({Z_#A%e=Yqq4heRj2)L-n#|Nk`T=DC-(Sc+CQjVBk@?j(G> zQ4*|4{D{GwMG+SsF=^b3K9&ZK7%O61N5`aGJ?k29!VB{*gFk{sUHIhVRoMAO(nfN)$sFO)Ju1G6+%KnGl zblL}+&0*4WncczVLmURAbx8G$LDk&N@>~A+o@MK>7a8bdB@~3j| z{fJtTL-{mqwn`64b8g`lIIDY2ky>j6a4|4K~Z^9*8V& z&CsA4%N8*vk|R$L6j~;GYUhn7olO|c%}VkyHMK;!XAlwxVVP@25=$2vW}z9gLo^3_ zLDO><)}DDIvsVi$Yy?E*>a4TA5aRiduVYA1Zr6#7&j$JdCK2W49@0Ytjb~%`@TYy; z{P|TJU`n)%3G?yIvZ2FYC!M*ZW~chQLByqGX0@1DN`VHXBUnE+AMlb3M4;#$2;d`q z3n3+z!^QB5J2@2BO z9BAKNDYTD4C&D`c*I@XirBio#dEtn@h7P{}yBa?Lh;}!)<(Xp z18VMnNo+W2t`JKsX)n${z(G~mV5})ALCwymVMl6wJk-1Yq1{jH{qbNol-Nx~>OuK4 zJGWsPGqugV`aE=Y^v!@{+)v!Hyc(dOP2No4UCMiuO7w^ne>N#!DB$)_$lpKe`9i$n z;%m$6%ON+Gb}@u!-M4$DSAlCAQ|#($DiY$Y{k^l(9u6lA?lqp#|J~;;LCt;zgGpZ!cvT@K; z&S!VApQ#~%P|Cj|HRJ7DJQf6k5f>uDGSAyXMWcJSaP*T;MW2>pTD@R}H|(9mxyeb6 zzy=PG)TU!2FTgLlJDF>Z+*K_Zo<TS)_Crjvbvy+bl;Rkp&?4 z53lt<@X`s#+JPY?tkvtbZ%6M>-fwRjX4P=6c6TXuE&~%L;6}zl;PdZ`i_@G=`+^-G zmTHrMF|-%T@g|l-ncmW8=c^lg98i`am*z>0(#SGSS>SFvx&k0r3{sntN;#oI9YKHZj2eP+XKXwX|DK|#&g$@VxJ z0{ZI${%E*Rz|w<;B@B#Bew}s=-OR~II{cd4aWVN}Yz>N*AeGht zf4Rxo-vISS{5chaam4t*z<{S}`ogv(`Pt4^^Ax&N7eWW%zqOOPZWn>8c5=Ll7|4hxeJZ$p*te=P&)A~X{{LcgEVasinMT)|j zY|4Q>bz@wg!M~))(CLy)iLkCRKfs<}g3_S# z!52{AP7P9Q5=D=FY09cOt>}p0^-f(0xa#$xw^k+|L^egPxpU(xXQ+l%Kx1pr~gfm)A-~U|M>>@{~x}bGbaQ zE>9Q1$Y?;n!^Hw?s>6GIIBH@}liz;fnOG7b#YEH71eVdyu z3G!xTKUk7rp>?>wRHeO{eEwPIYg7v1A{=FIdOe@H^Lb3bWs3EcGDWeuI^HiBt#mC-{vBdZSMF zJ3jdM$U;V0W8Fj~$svVBY_X#BcUC8Xl%x91AV&PRMo0%Tjy7|=m zaVgSwPewSE2b5LXP0;SZfT*!*Lyk2;oGlE!WX9#zTf2KdB-yOSj-+9hm&hnEK=%X% zV8Z*7&+(EL-Z>Hyxe&>iS9JTetU}6?1>MQWOe0PRJ2%gP6av4bDTFy)$S<1-`WF=E zaIb<64l%^3VB+7ppaQY@Y1*VZB_ZbI7+f|#zSCGy#z&8>@Q%Xqpm)P+!$NT8;;PKi z>;?wqZBtfdoiTKy{Pj);ydAE$g9L+wq--)CNRf^h0t#Fn8x|@lNErBEttrE9i`$dX zy(s;o+$I%@^qT}+8zoece>pAOnrO{w_28^!g^GlpY~((c%ujfS-b`nsP8aGj2JAJ zM1x}ej}~=LNI`J`x60-iw4TsCb~ijcyu3dsRs789HaJKKHb&wf%a{sSg(Jd<)W(Y7 zUhe3?b5_t@(TxjRCZS{FNd(RZI<5qkoRLYr1KNy$pYgV8jS;zmG|EJo^JI>js>sm) zRA(GlArWT(1qQXk_IoC7Q2Rj*e5c#pDF=spYB>v50!>qk+HMT{K4a{Gj11 z`h7=ypeCXmvlXsR!<~I`AL6&RcDi@1nUbuqg55ADw`D<9oyLU=$;)-pJ^rA3%<|y2%|}XVY_DJ~wBMYU$#&2RPr`Ww3~I z4mB}HGE3*tB}}R9>;Js9VmU;T3IA(*dJ_0 zSVvL?ok5b7^cNAplo1Q_R@2p^;n7B~qA?C8Mfg7^iMG)zBL3hALGq z-Y~-$kZ zsgEiv(fbK(cdq8Sxzj;IkW_{Prz44}YV9A4^`bq33$j>PIFb!AMh|=hT|0=TrfFLM zT+Y|4PvGzLbYgTe8`+smX?xGQy+kc&!zk1KUkM{LYYcc8uxSVQYre$Ce-j$th%J19 z0SFmqQls<{@XbBHS2qW&>tx-_#*JiDLieTxw#7UTV_6LSne0fHq4)*vG1S}=QcX3N z3)JRUSKoo?ZJwZ)+?X?cQ-AXRMcF$>SJs5xqOol!JGO0iY<7~4ZQHhOCmkEzv27>a zv2Ew>KHoj(zUSxtwZ^EiMkQm_+O=vvb3Stx?^C2XdTfOnL`8+wOcJ03XZ`y6e0xJC zrWCki3%zDZQl=JE;@+%^A5l(U-}VAlrVVS@mu1awEG;ayePFx781_5N^IQ5v^uYLh ztL1y1x~>`|fg4U-%{QSf1(2g`UteG3kMX*H31zQs;`hF)AQDA-5 z(0re%>ZyH}fQ%y&WJmyI5y~qUi@=oI3m34#=M|&64oyE*75z-BkD5L$;5Pql^XU6` z9g`tUI}$2sD=Vw;FsQ^RE@1mS*JeQ&zW|Yfr>W87#fJO-Fn$9_1BM#3zcCBKMNYRc zPLxc*S-UV)J)~~C186oeFFz#Q1$kov2xRw8p0^VBzNr%+;C8UqY(_1BYH_IC!z)Q+ zaPMJs23qhM6enqkgq4*Q*sbh#)%E1W8u+_FtQ!z7J-+%qq_CE;8yMn1v4BO7XZa>Q zEGNhcjZ`0W;J=OL_N!&vM_v3i5Qv;tCCdDzfhOcLlbi4Jw?7%W=!pNvaK2j6h#)|7@l-k%EAP_;m|z9|(>r7mPeFaz7gX zxgrEWW>*sLQkwFd&>^<5uqh!u@d8U#|9yoPF(_FAJP1V9JMbV7XFUGQw)^}SDQYkl z_d5Nwy!GUJrGA?By`O}#KUSA#2=+S!i%pnhkYPhwTIbuJ%kH=Cx(7i|Ki4KjP1C%es!xl0oqkJP zKNSzg9QGZiY@OW}wgsIbM}&5O->fG)-^iG~uWrb-+{Yv-}<7w4>vOOR4hY%J^Dl~S+w#7fV$f3%u3b2dj?-e7bzARRrV;&}^_0g)#{d;&Tt+?P(=4$kkAW@jl6-NYKD z%8v_5!0_bjXbHQ6jqjb&N}) z@0zOZe{yzK^G&(o_vAQz`-_9YL86z)fFsaOVF3X|oAOLLH9GXOhM~!Q){ej=H(`&$ zaYHN!g{QcPQ#lY=GOMS+p4G~35c8>Xhl^#AX)9Vl(g{iJP)9@Oqdn1OHv)2IIz&IJw_lY3n&>t+}pqo%}Et*!m-VH$?30YBN`1fnEkpLt~560c4Ug4gQpjBx!Pcl^A z2rjuQZ^j2g0n;#9DV7@tjVCde;wfrpQNj={OJK>%_({L#xH;i5sS z^>fHh_|a?5nF{6eX5af2v!Od)skzYzD%YB$MF3sj_yy2qvv3y8*5HzJ~y0e%&Q_Z=}7%cne2C2Gw_<`K(^YGpHhQ~+rK3^myQdkG~gh^cp|UM&Cb;0hJkI85D)*OCg&|Gp1+ zyAn5W6n;d;m-at{4!)!XG1srvnhdG`o#FrAv|hx9e$_yxwLJ#ASXGyRmy3(Zoy0L% z?K8pj9Nee1RZNgo4hvv)QUPOzgtMYcLC3(p*U13;LY%eJs!Yio-`l`xU8 zv9aEsV&~=z505)7HK?poSk}Ifp`+E&7gKdTCE)gUb8>M43SI4-gwAMP`mC!vk{VLl z+7fW$ZWnq$%h7FpcIs1qg-h;~Nua*4I@`PDQXeBFRf!f3Bf_OR2M!6A3qiny~+u)GNh%3YsDFraS@(arSB3|qrV}LT|i)H z$+~nHV5iXmqSA@K5^p>azU{P@&6KY@Os$DxkH6ug0}LZxQeV>bxRg(?;!#a~PO+ls zSWY`!!+qkI7B^ygIr^0CWNWdMXyA$r`7x{8(HTEXq6s*R#2k2fc+h={AjJ1EGLF3Z z^XG?6?LI8d9mh+dyR-juY9cCvxMM75n*`96{!=%3n(maYuS^zl4qAdze~QNPeAB^m ziNym9;%DTpYEMRI-!)bn!Ydv!zys6z_o$45KV@hDYyZ0xE4Slb^JZAwg)P|V$L)() zS_AShzf6y_{}u0(EZJa({xUQyCoPUB^V1*>jo%-eHuUv%r<$){k+i`i>4$hH99x0*oMJxC7fq<(DpVV~%)TskQLnD4v$rfSw~etKBeU|}Nx4a|CyCQ4%9zX~*Fxdc6Om8*k36;I zU}-|+O)&+t<%Wvmcbx2RWrekMH5-;%jV?+aM7m0pJe#YEPV*&Acy2-R>5sy+V33at zG=vA$H8poT!ZSn2f-G@Xi0sVRNu3iCjF%pX_`#jzLBOZY&up~VPPxqjv_#XQFuyd- zkhhIBw@RUEz4eSm9LP57&;4WUX!~H8pSm5hcl)$1N#|!>Jd=WM^fKLd4|8CFX`RYFNC0 zen2Rb&BnvWEhGs#w!vX&go;VR;-_;mEUNeml;LP-q|Ncm$HZ$sJS`M9`iv_^1fe7{ z7RjO&Yg9n{3 zs!S_rGmVpo%LLfXSYCHysAvds62I}3EmT>D-5b=0AVFf6%t^3PVD~jycv3|uEBPl#1D}FgTJy0R@V$8o9UPnw)i+@_)R}YQE0=-{Vn8B4EhiB+PTJag{!2 z!v*h?^WPy1baXaQY$MqA@z@h7@b4Re(1}?Tipto)*7@#n!~O|$j2a$&mm!OXvci4K zbp_a8O`ISfatldQ$p^ASY`72yqocRY*K?T`!$M5ODf4!;4sX|&K|<`Lpgr8uRPWPYQ_oSdtBtv z+yD5M4Xv$dH%z7-#O5anSUMqR&738d&czh>JP*wX-H-UosUkzU8bc*W$5o6w-6hi_ zb=%R=L>ceIkZfxQmk$zyOM!7w-ZO>9?07IWQ~N*TvW? zpOs$`C||wPn;@XF{0d)fn@(Yz19Vk*SZd-N0q+;qlxs zLKJ+Fs52}-TOT4wx7EHUq2ch6i31jc|B4;k`+B&1KWSr(J}jWDf1)n}v$)Lh`%%Bk z&d+enhEAqDeQmojcG*w>?6l=U{bx>JO zky1lqj#jbR{k-zkajmepJS;Z}B7w$Xvj&qiu}|v(H73J8+Tuz`n2XJb#DhMR_*=Tt z@;iIax6ji%+l%V?!#c2{kJ0?ktLB_zNxGBWqrtJ{N*Qh?5p~^_S!&OOa2Ty32kHWh z(3Hiak>2iP2&Ijqi108;4QAC9wPgha?ZWGq&J`i4Din~?8!0g-jnY(bh{)?hgkQ2F z_4lKgO4FlE`nVl0lMI=p;n{YKc$Gk%+o9h&GK^G55#LFV*Vfqys%GD2ubHRpq~i#D zc9bA_M8GM`or5)}?KZe66r03h3;n}&@({PO!9J|+a9yTq(Wx<4!}+I3pF!A_fpZ}` zCPdd9T|G}(XJ{%3IT8yqA|@iD{Z6V8Xf7_Z>Se?dDI3#Yvv}DWde?`0oQWGZ?;^H` zd7j3J?3~HztgQqRB~8tlm!z~qGYr~N*^~^+e94E#hQ3#NYtPA*xgXD*H!18?Xgm+2 zz%c5+;7I37Y;+wt*A^l8Jc1hLlD0*=9OWqUxTb{z?j@=P`OMieXZWYeG-F<+OnPb0 zD;k>8xh#05$m0OKdYg@Sxsus*HRvpcbiA#a!j102C82pY!A*jNmLFhnMaF%r;sV1d6k^_d4@)Py0L zGud?VXY?#8s>@P{Ss4c~r2k`=9@u{vad5z4xU_E=FVq-5PmXu~3uaXqd}&;2djeK| zbkY~#qGR)m{YW;0@mT}ln+OqHtQ}zz5riruQC~IhUcGb(eG6AgNm8?d?TLCR4qpy1 zIF$Q%Cj z9J0R7`6~lQI#73_L}n4nZj4|iw|5{0m77q*H5k5L~BcU5=X~k>seNL zUEvW~$IAouc~q&sEL7$s=L&H~;d+J?*LM!ZB6i~cA>kqjF*MeN zu&c4Hu|C6(eRyz+qKmFJDIKaL%z}uUm$|r~Sy9~aZ!FD#cDAtb#s>gnXQ47vjU0tS zzt+@HzA0%j7DrAVfeF|;IrYoY!f5NrW-CTvM=|2+k*Hd(PY#| zPq*UlxF=!9f7&SfZ(TqW;xq|!)C`ISaE1#+L4K8_QaSMSD3k)708k4)z&|38_mkz$ ziUEO-uy^2Y#T=U*sRPsB4tA4{rOtKeMiAvU>*6~M8>HD8omik}1R>QMJySwyfwIWZ z`UX)c&qmMs22e07qrSjCg4FyR*m|3p*Jk97kpWzziw!Vh@w(un&(H4AP>GIH3NP3h zj2$<_|Lp}pf8(rRClQ&1{uZ=9Jsa9FFnkO=uk_mn(j|OSyEjRJ94G1Rm=(=|kz`*2sMl;G7z+(~ z8}(;o+As}t4GPVM#g+;U+mT!;3OzYASqfxk>vnCkXc5C8vbv<$5Y=K1?eY0>#3i=P z3MxTZ7^CA2Ic{?6zCu;SO+R63=OtL+2R@B^q9x+b{}0sy(P{z-5Urv^s1~b0V1P;p zqbmKoY%Bdkt4bQ#X5as3;D3z=LZB*1W{_my{a+9^91YYoB)oB*ec``|8ThSX?T6@F ztoZ*X6-MPw0q$q~DNALU8vprYBqG2iri3Yr{K?SX+*I{TcK6_t+Adg@JJvTUZbON} zw6-c0`mbw?gb9ic+1?&HT7vi9exEy{4eAXQjyM8-lA&CoRI~1{gXnzkiRpZynIVJC zp&>mx-EmVp_t^GPIuFRj&8KLtz8npcy{mIp`Dofv`0g4?MuzOHtj3NOAniT-R2Y47@*phkuzQt-S4c>x7O)E_)C`=yZDy7JJ_(7ry<@j=X8Z2>jL{c?P#L_YhD)C8TiWLg1I zDLXEc05&vw{hk82gecv=qrMfok`eyr+Y+Ww$p-4WhqmX$^F++}5u;JJ#J|a4D_s(M zMY?aHRFIWnjuDrXk1=i(s>>el?``aCYN3|%+a)n~B8Eapjrh1l*1vBgY!??7Z{pi9 z!41ds$;jcOb)&H`lUO9YNkPQ;IH1xUUL~L||4wDvCi)fMr!z7FOxodLo@K(?!5OX( z0bpAnnL&+7p~mHNgEk2ZlXaw}VF8p_HxFNr(!OK|(jD~m^hUCA0KFi6!>?%>6ukvl z_LX2J_nA6&xVRlj_^^z+4^v#h5SlB0JZ|@a0+kTd(8EJ?x5Y`gA;du-iSFST(Rj_m z#Z#`LwI#2KbCUm7SzFs@fc?mSJ()U<0}_U^@3rBM{WFr^pb?UeJCNnzzH*^FJ#fZ( zjbnvgEP_VXU!OQMh?>iwwxrwcFMLvVpwGY?lL&`zvyV+T#uU?Qx+4iN5@?U(r>*o1B85ud}S7-Sd zIK((OID~}g#@$<6n%dQD4+CdX6PTis=tC%^h&l*uFchLn!f_1-3fZ<+R-Z#mex3t} zROmGH)e7REhs7B*xkCPdesgPUu;uQ!tHeT~~QmcXuwtfs>xr z*4mbuSP~O__x+~Vhkl^x#ryBlBI$e-H>O#VD>7&yZ?Gd`6hxc+Z# z{%r^0#}(?S;M{9lNsC+uhGBJ;Sq64#ngjsdDS4VfcWgk z!%t0~WV3b(dMv4Y?@JyvfB!o!e`g?F^rOS)Wgk&8=Fl%=jJ>T{PC?<%zornu=k5Hj zmwdKC(W!FxRbdU3NfnlBU>23RKJef27|O{*7*I!VeoXuKu^PkR9PCv^Q2xBU`n;Ub z_roSkrge}UwOIN4cV;rY;jxs-mqv23$hB%(>T#Gw!<(8^cAJ^s$I{gGfQ-;i3ueJQ*1W ziHLw8Co3yNdRF&DH#qp)=l%e(-$Rw(R_4sW$$7k0Jd#nbiE z_f3L&GQ+H_Yz<9~`g+$HfjX)B-=v{H3uB?rn|S{nczt0(Xt1f8u-^`!{9Xxc=F>jU z1H_)&fo~1IBP0>)NpieR;%fRIr<#B7@q2*2Q0Dmfcr+w?lo~3X!4!#`B>69s<`qx0 z#uYQw`3B?hR|cOxt~?Z6DV_Tg!nlS%B6d9sqFMiJx^37R?Pu?ll$CioTf^UAGZld! z^0B?!*bCf_k|Ch&$FT@&$&&JzPUhug7({{&+ zOY@wo=D(j6+u9Q7NZjr>h#3w0`N!uF;Sq#MxBdFKI3E}v)#>xnxF953N(=^4(#d(5l;5vZF)qO3& z^j%L|BDlL_iP5aZ`D4M*q;`Y|!Hiy*P$2z!47#zop+S(tQUgbVbnY7ZHP-04^L-X6 zo`??$x1XNeS@}_vBm&Gch=jGOIVGz#cCZ(UWtFn@k0fBK3?4z*$TxBA9N0g%?sBYU zRvZa$oi>!`_i9--RgszK9jK;%b#dbQ!r_*1c6#a$?2xvmxAqdmkA`BjB$4FBu92$#}{Peq7Mfa5`hD$OtE%T7i!{x^3(Z*NS z)^MYs79I}L456SRi7Q!8FQ0FWq3gefZIQcgUpF(erTs&o>ytr!k?}`S>;%6&j6NmV zxZTdFo5)PS1<6{|RIp_y=q$3PHfMZ%P|Y{V_Ja|l=Ml7KF<5KH(YFeH5yyykcN(JMgc=Hy;5~}nFS{O!L(Kvf%UDnnUHe_bp*Ahl9!>Os~Tn1LJ?{`OI z_rFX~3`YqrQ5 zN33-%ZzK9|QEPgq;av^mMff^glfxtx(!5>M_lrGI7;R++0JC61oo-Gp27u88nUPXt zwm9jhp~9f^!8qnGMB&4O?$JW*#?X{ukg)*q+oQV}J|ENM#rJ43r zhG@!Nq4z0=I6u^$a>cC(Z~MB;tO0Sd#mNQR2Rx z^A3L=Z!;6aDAjP!&6+kQNPouU+Cv0dsX9)|DajXZY!{Z7dowlVa@Dk70PI7wiEw_e zkxURrQI}+H$z&W%eKyO8<1xp{**v$R1J0%d&le@kE{NP)X=FcE>%trzzC+rMjM{w@5RhF#vx6f>6&S=u(hRIpbLGnR=2oQ9WY zx~is5!Zz-&U4{9xv1Vj703`|*giYN%mc;znx|!4@#e$$9OyeyZAEI+?H@kgekzysj z6C)6K>LeAOKo7~YOr>)vjJ;q;p|<%sg??_(69&S98#-6tJc>uqq*x3sRZ5DLWP}+J z7~%@cRtv$x#l%&JjyM7xIs1kcGzi>yb>Jd(pXpqMRew1;T9 z7Z=H@s`M{`oD9&@&{dDQ^oC1yi+)3s9A?wtJ}rF9>84-CS+0;G-R!POow(T zXH&*t&xz32K(oj{H4?5r=>(2Avti;L(GGDyp>@{Dh?Rf}4DQn2WvLdq1`>2S8`!(R z>agiBAPvD2G(=ZNtD30|K_zi|ilZy}O%moZI{%X12OBU+LTAYE;Bp|f1t;38mkSB0 zu1EyIojgRIS_tf#(UASxS1TL2Ds&9fihzP4>NTphH=yq1$2A+X;hLSSrxj6?o|`Ii zPQ^!Ahrl*NDRROcCJEZ!Q7u8uDQqY#obzHfFSoWf^mS{Wk^#dkx``>Ns}Aa7#@Gdw zjCdzgJ|=O#Pu*V59&go1;gS!mWu!y%lQE6_8823BF*Hgm8`vQ@1UUtVJb41kEhIEE zi%b{A%`%`%yrV!vok(vR4DIw@kSK~#dyNpLXh9+{r|ENsQ%!h}ubVw#xBGSQn{e6s~Q;2ky&cPM303a5lhbZOtQ=xC11Z*e7C-iq> zH0iK83`gz;yq2<&H3Dp^v6CgdYzARa+YqW{=ouJUiXce+^pNU=V;my!)h$2DRhZfK zi+!YI+tZomn3EIC3>K>j-yoa7SRZn;bqW^EPJ23NR85+$PzbcRf9Qg^iS4!Khi|R>&tH(0`B)Sd z7x&A9_Zh3tT7aQ6GMjg3?da@L4cr%fgt8@c-(W*38bnayJ$b!pAstKf>z{(t#EgMp zIH}R1)dQ=7EGt1#5a&Bee_%z1^Bfn&=h;JcE%tIbDWz#NkvVtCsNA+E4A-HaD1a-T zG%?NHM4_2qL`N=aa%8p7=WgmS<2wXl%5Ukz8#6ac8#`8pyE&1Y_>5i9F(^Ft)^WJz zw-wJW**@9V8Y)>CxH`$&|3K_sfUq#`4xoa3s_4JWFU7xum-NokDCYk`-9Ug0@ed$F zq?Tx_{Rbcyl>A#Nn%{UG|D7NlTl2Yn-Lanx-uA6mT)Uh(wb{JOOSR4fvdxDxF`3geBK9aU|amtrnBPeMxMYzB=y6L z-|NEia_jx0#5^Kn_T(yH)0eaU;9*>|dww-BJ2O$IuaayXiVJdnu>%Z6R!AzI`0Myv zUB}mF9Wdns2+xYyl(V9=-bx$u{6F~*ssCPwLEkMmveC8~N4|yf%|%7EUli_hKeL`X zE0(hiz5~zv??6UtWMcB1{z-7&-uGh^TVB`wl}{eKLD-p)nE0&^+0Wtp`DUjOV9#HP zv20;wZPng*--`X$?RD|wAQh8Tq?J4PI!fc!y64e4IXT&UI^zwdp=kfUG7cIZ)EU3z zQ-ZX;BMgnF2v5H#ci)B>aq=$z1DL>+r-E|4zan^Sh?T9!bBu-<@0SlC4`>b2g!k zGfa1;ecRc{%FD`2(&&(nHy5C0H$1 z?QHh?K>zbp-=TcB`#~{Y6}5Z6v-8)>o~@1!-|j)-UmtkKd2N`pH!?)&19^MDQS2I5 zCfAgGX$}0@RgYSyGZ(61stgL_-KtJXMEoR@#dgH(2Vqc05pZ&-V-KWLQ~WclNoHmZ zQBX%4D;tx>xY%$QXT0qIynswz{yKQS)`tkn)z(;GZOP{Zi@-6FXzmy%doB=wYgvJN z)+sTZPKc0fhiCPrwKHHuA+>TX@0QARFL%KM## zLLz+xUqxzCy$K~S#1Tq7Z#%pwa*WYLRuLH;?SsGtSk%09Mw~N0JUnU`B1;RB=*3-- zqxh@`Yw2taqs*5Er%Mv0=WJt5DIJU)i6BA@yU7)8o9owtXPxAGiO^fzi~#!_9CS<# zHI_4Oe9s~wB37;1;N%SeJh$J1o(e$^;Ve6C(F+)g!JJ}BjQCeYR8wzfNT9HTG{ zglwg~nVcs;48w8iG97|y3Xu$X5g_30Zv_T-QtQCr;b#}LRzoP0V&$TKbXNl#{^re| zjWM(5z$Sd?9hn#z;01x*zEs(y^-uCdgLX8G*BystlEii$EGD|6O7+4t_wWOi?Z$Vr z>9t&WAwPgcPgXhBqHLPJmZo%Z6$40pgcMls;#0sALL2jK=>cY$z|_=~ja2U~D-pRsIr+$Mn(yznYqy7Ln&U&x8Z$yXDY z)!@fTek@agJGewzxrRwJhuIFNyJXAPObFw$^YMP6e?6igX^9dnb@7i(e%G;{@cF{ir+vtHN1IFZj)#m1E zyAh*iXS^lEb|crgKd_O)lhiHCNM~olNR)}OQ)tfw0 zk*|g-)5?OwltHviLYVv%^DY3>(A116>Pb?iWI#lWZLWeLnC}B4PC*sAN7YBB#VbmB zgx9YHDoRU-`_k3Ry_Y8eWEwd?&|S|jFU#8IUDC1AS)1(Cz^%=cS6>nd@$Pl;kUjHq zawI)Pp3G~;PKZl6E|N7P1j%7z(G6W^p|H)TCpMM;z3&0lj@X1+lwyM82}*vUVT}$k zQl-Yx%RjsGY~P^Inz~wsDv-T}OGTzpen_^Na0)_I0X^tieyQWI0>n`4n!ZtWAeX~v zc^PvuDYLpxpZDJ^V#~<5A@*P3{2qf{u%=EQsf$dCNL*~xU>p~fj@VTt-Lcw5`XPjhe zbdt|D{M^g6au~E-=>4;~I0(npgpq|_&2QKBCZl=kwFSPi2@dZjfRyt{wYNRyh5QUYZt+j)5ipH z4u3~Z6f+5ZJt~{-7Q+}W8X|WJD%5$X|0QRU`N752*6d!L0nXB&!E~DKKo#VPSX}3u+q1Rkr#)qbsMc zkYcT0QM1&|BJe$ANx6_60oPY~BV&Pz=-sji@d1&(k*Pi$u*#zGd|s4*klv~Pe2dZs z+{K~w1${5i*~%mnVBc|tTzC`_UyZ1zb6rlt$YqefKi2m;Ov3!g{0&GofedS-rx%~3 zwy=V+ZB^PsZxBNb9tw1*e)ZW#QIc$*Q4cb~4o%n>Ylk$4mCa>fOSzOHt^gSWM3@L_ zo|Y8vnX^WQPfGQW5p);{526H}owM}lHS+8N5StViw)V$rhD zx1&8j-E%{w{9OSxRgu4omjd;i+}0lx#0HtU=lzYOQ*m#-xgh9KjWrH5n;kRc7eg+< z4bq}Gs9y6Uw76w#Ow8jiAW>7ZqfKI6$tjf(G0bUdiec7^)S>|q(so_@iHl4hb%Cfb zn1I@FMz<_+bu|}x(YiqM*J&m#yxS#j*NCf$QLi>&Js6?{ve3E^a1i^=f%&dCSMm~} zf0QAa*z)TbY{x%`yIca+_vp8|Bw|ws3&(+8WBK!}NRok*@J1SL5Zll$fvta`surgY z=G~^0_T{&>wr&7ED5}J{&e%ihSrP?t|1XZBq>admMQ$b1fo6n$j`QLSeJ%Ed`YgQY z83jq!FMR=RQl!LK0NuHrpG+qtrzqO^@$K4C-{)Z!RfK@Ieo&5S4eEvf+q4atR4RR3BlWwOx!cO$THWta0>{ytTc&lh4z?%TVw{u2F$|)wF&Z%)Yf5=kmlG47%(p6 zkQ*=V`9_psC=J3Oj$+U<6i!JkdmEI{goezfMp3n)ed^0_7aN< zL!(w^32I09n9)53JMV~Sa8GE?RuqTYDCY{+N`q%{$bJpwVL6Qh1Aqk_kI-N$ah^?9 zQ_;EPK7(E~o&me5X28rHsLRuMc!nnf2a=1M!i5&8=pEaeK)W;0>D0)qgZz(amiQvO zRGN5#Ss!QW_~fpuLY9j|W_S}O4m>XklNOV}MD6$ekccA#2q`^6D35kYVVuGQetmkm zkP;>k4$>T76epFa!oKSKbDwMDGoumtuXs^l`2>Ua)ouo5o06Z*WfJmtAIqYQ82pQ+ zruw!O{qRn|9IJ@rP!`i*ocz<|VWK!`40VgWS7);c=UAA`2Yj*bTV$%wfQ_Kj29w(fR)b*9N! zrO5~0wEf{!7phpFF|~6!wX6HHn4qS}MmP<@FF^JXNO8z0TM=YXD~r^l1L%Qr>?1lm zLEjg}@vU(>JaTL*fM&5V7BYRr0@&Qb2Dp#(7>n~ceFC5wPpr6aY8o^LH#n7*Mgk&} z1I`#Wp=(|DP-#M-(uSklb1$$EY)96{h<|@L3 z0c$a#Kb<|V>hQYwc5EQ+W4PWFg!>cJLrLMk+hL1YM4z>5I{Ey zbQD{KH#8^5v+yunO;KoP)^(4fgptQ|2|XO|+6@ndm!a{w?PJY$FeJ;9jRnULS26dC zj@ZuBz-HIH8e&B4VA!I&gx}w#c77xVAF2PX7F7v$KAT>&*72*Ib)Lol8++bgXAG;#xIR{jBov1H6>Od6=BN8x5H=ze;EUL0&fD$Dz%AR?3 zagj*xC{}nZ#Q1}tKPSFU_mS11vf!A4N8j*iZi|ZeIJm({-HM=q^89W87@t+<6>`PA z(Fv9*CUr)|8>!AHTL3d~2!pcL4JtJi^p%IPnUM#U++N#WK_(8M2n7XIVvWwzrPGS6 zy_S?qtRTa{Fi*QG(zh9=mu(Yl(f~~~R;IIT7=o0gO&4{yP5M-gb6<`QtWhdxEjrer z;^5&WHH=40yL^XO@B;cWyP?!TMZ@UsZfz_CpKl|@mfWsvEf}-A%gV}mXD+NPbKuAu zk`GK&;09SE@pwaTphy(6$?E!plMPA|WQpP~Wf+qE?mwT!z24?)l$53fh@EdGTEFTg+kcL&HBBP~ z<1c1fVs=W#RSMH7Gorywj4FIVPt8#gv!VwFHR)Iph^tCKm+#&2a|!}-hBB6-g~>mD9Qs7@u39a2{ye1Zuo3 zb98cId9RoNMY%~Wp|t}?S!~1el`SKz*SXG zGM+z?1FOOD*{v+kBW>$>Jz%;)@1lu)p2-td0k#u^3& zYA6fxiV-bp$UpwC%WZkSkJ|deQUk-CfV!9PHffqAS|nq8;r|0E+I=ra5WrB9l}l7AT&PG&$jN1i$S>@G@i(eXP!&E zTp3b(ca@j-VttbEq>YH`+bE}8Zci0f1OnyEm`E{~&Y>;|*xS^v*EZ!$3>?`gF;Wg! zLh&SB?*j9=^5ayWJAOfyrG<^`o-{fY`Kg*Gq3iYCbrqM2`G;ug7nEk5PJk%*NuSAS z|4`wv&#)K#Z&LJs3VXOfuDtyHb+KyE$|GF}xYXou5jZ_UxnR(Z_x&Pm3LZ!@6#F1o z#Pmht{}L3`U@MS7Bvlt}wJ7?3y6b>9)WiQP8i4WtU-HCW*MEfWSrJ|f%75oL2Lo;( zzd#g&|E~@^Jrba1v9m&wA>_X_$*+I)AI9E=g8xpSiw$H{elz~O>12e^_+8?&DKY~- z%uW`C-_3%VoDS5&q2@(!`6? z(cAB$W$TL(|6t%8JzO#5V%-JL6V+#m*j~z4Hnag?l~-;Fbd`I!BCj|x%{6F7CnxEZ zf%*#4-Ner!Ma#)w@{JYOjt63!;?;6{r#7`C>DzO+%ywu8C;GN&Ddt_oWh%~y0&5a( z6TLmZ8_^7Z5GHpZ%pfqY>IBg-pE66K^RLfp?>IQD2uZ@Aw7iq4$;!eHv&ED?pUx-s zcW9H{P*JjCX^;;L7pWP<4rBs-T%-BI3VTuT0jvH4X3Yz1P+2SnB#*^0`3C!_8!if8 zM#eJLxk-p7EQ!ga{4*!SbUzjra8z@Le!w3>3<`Fw7COB?@qo4bCRJV)0uKiE>xme3 zh$&siyQ+P96cn*Uh1w$HrLqy3(P-7%TG=pje$1czG_dQX4ows&_jkpB$gnXeBZXI= z#d?u2V-Obg5bHXucZ*pQ zY-z`NC&8|=YYXdqEIT3Sd?Jy8W;eUC0!d2H5xl3bgA{%%Fb$YlU)wmSsg}K{L-et+aXL8F+f>Q zT&S!Fr@+A5htD^P4C>5evxt+9PQ}=G?|3PyzG|XCfg$;LcV5L>mhSL^r2;e2u3D4& z|6%W~g5wIBHcg9}S+ba!nVFf%VzihoW@fUOnJh+&nVFfvvMgrU)b-Jqil(I74e6r}*FN6Th8_P7q5_;k9J_(J9K^lcWic_v*s!6(15TAGIFXT98 zwu9U(G_mO7LnkX$=7kfHpL(KESW_n1+2t-R#TQyfFf%*BUWkxqeQ4ZDBofs<3E}ZF zBjat7mnJS89^|L*OI(1}`l(mEb89~3*7T!TWrR>2Z77I&xP_(fv(&<_Kk+9RQ3u_d zUVtHDI@97dA%v(YM5Ljf-W!my7(2)`B9!!PXTcH>5$gA7WXqTsq2B z%GlPYUqe-cW+1nQ=@p|GSMRbUQ^Cj(Yv{CcxN7H~^2}(JPpB;~ZQ*8YjAbacI!s^S z_WnND#2>LBF~ILfRcn%v2?9NZY$FdStSVGYpAkHzro?o4@~+Y3q==)rxy6(8O`(J= zEU2e7y4AYnTA)7E2%;F*ga@9W=4ApL#_x>!8JW659K>-_EWK(@g~p3`IRP5+nlp$wvnI@3 z2hZ>wYzv%3oXZ>>N5>q&aD81>vd;4x8?V=MiFatehKoqz zaBGrmAeSQ67DgcYvYX~>l@!6*s9-1!6Jz~@QCM}7dck&(o%p>*hD1GJ=RfQ6jbDlczuZ<-7S686HsSDJC}%cqx#;uZQ0 zQ!%lML6m|}_@6WUJ6@Dqz`q4XFV(4|yFu)h66a%KmDhK)K9I2ewD3u}7$S(1fbIL4 ze9=={UA=fQD8_X;k-KSJhsE8&5kqp&--L4TVMd`}ZfGJ$BMy~381ve@APMwchbG;f z{r(d3so`6_|G5b}xKAQH-rtt3e#hDFJPo0-7It>A{I)kYH{TfzLcLuYTME6xzc9H< z9J(AZftxH}ULr@=78aWLn6MfKQ!AKoy-^;9^TgODLkq{M#b zW;})v3AxqY=U47U_lGaB92gjwgO8WZ2+F`A_+1yE^&IvD+?gF)G9Mt#zCRon6BpC= zbJ)#o^%jV||9NdJa@6)(}3Kdt&s#dxJE3Pim^NyH*Dx`6L zfB*R}!v@Al)NJM$COa<&34&c{MO=cefqR3;69DaWwNKgJ(K%d7vbXv}vvJyxazf>o z6P%8^4#f;Jgt^5oknA55F^`y(m!}J9*nWawY<2Cv#?b`=B$hn<{JgX>!r&}fT%O}H z8zHh`Xmaj^aI=Jz4! zcQ2SdI#xkG-r8$ycZUb$(*f{~4?rlAQY`h8I0WdD{c}K}yukD9$1UK|txvQCb*?LX z=x^#H-;zqkdLNJy24uUtxyj2#=W;s_Nvf2Vlo0s6UOjX@JaBSyTm_lebyUK=6|-JL z4+ABUhsB5Yv4~WLUXr{trcxgVVn=MAVm*BZC>FA^5I=VnKW-I!1K2$iKfxS_sq)T5 zEvBScZ=`q3Ku`KzTh z%P{VY>?k4`gQtH<{B*1G;GI39X|0}GdZ2&(y-d5Ms850eDZFeulN9$Ylz*v0t^f>UoQx%n^FAOctf9 z*Hlm9)Y9L`-YI|)VrJoI7pITnK=ie7MV(qO;pX7v+|5NpVk>e!&N1QXQ#245MWQ`F zQN@WX|E%$2Br%IIBE_=pTPX8PFmQyx@p6Hof1!D!SxML7VQX*aDX*`)a!j^%i2+(C zG&MKZALz8BC&i?*qj6N5H@V?bhOlg(!w74)wg$%%!u0S+)R$us=;}Qwc}(m9@#xLb zD!5r%xTr(+_Y{;bU}5iE{?JSi!7#mW+M34lMV-~1`CU^j_Hr()Mm$mx z2WP2bkrXd6eI2uhBq?GfjAW<|h96w#@nL2kt41e8buft02n`Jlvo6TfRUab0N>1X) zla<`?3t?2~`K74Vh7Ol!>sgd;z58f$ZMCfHD zZaU3fGg0fh5mHiSghj*-<~askJY^ZcBDHL%zk9U(#2PX@B^qQXTl|enpiDVVfcgw! z-6tvrMa`cyoq*yzOrsxhKaCzNz>paff>^H)Ue}{r^A3n?KxjDv)2@l_Sm@HUKBslT zB4#NXYhZHH+!0C$?ZOr2Pyg#yCzAqu?s(l&)5O2lV4!LUTS7GVU!fC~B;f6*npA&H z|JMo)KsK17Lj`F60cR-1{uMtB>nyYUYekj-!hhHZKL6Gc<$VTj)H>ZX6{4l*I^-&o zMnFP)1lqPkL3yJ;*Bu$zyrhx1qrx!pJ>m9x43;A29w#;u)^3fJ%cisqE&~eQ#IO}6 zNmUPI1fxfbWEa)2hAeuD^1IV6h${H&)yx<|BLPzcBGR?Kf&KY62Uo&v)1Iec+USya z+GvWKpB98O$Y2^fJ)2W}No?lfK8JhsoRl|{zc=*WhcV31q0qdg&CLQH6 znTOz)mWsb*Nspv7Co=QV(f^1yF8@=WnproBEsCAm3LbXyjM~7UcOg$vhaPCb+8Lour~q7 zXt3PoTIMrrv@zd0cxg?FmAU~<{x zsq?3}VjFWqkkUOD<)jAcxOjVVnVfPt)DSly<+xUmY0dk#w))jtG?7yncTrH!q+kIK zlFml0k_vnCUnmuj2k3_OMypfmX06EImzEie-s1UB1aP*Hhq64bm+gc9CPwXMz0wuZ z6i8=BV4Zp2?+tb4m$&0wL^GILFsXyQS2x!B)cSpi`-mooo$z@>6A0^uXjMM7%L(|T zHx?kPgQ$QA4Ct+<`39nU3Scuaall7wC$n27;eX273JVQ2!hjb+wmd3V8tpWli!OvG z_D*VfdTK*(Y0tr78XU&646l+raUK+WuD`Go#drHK5-K3c$cT0e07(>#MrRcX{z{(} z$(*lbOCbmsus4YeiYvyn#B z%2Hr-u)k55L_!l;i0Q}Z>$pR|ge{y_l+-c{vg{@Z4fN9lI+z)-&Ef0lW>CXvbVETw zS*8n2w;~}Kh=hcO@RNZ-CQU?lRYAE7-y~_i1`AVW()L;j1H^bh&I~S8^!b$m>=XBAx+!K8sH#ya%`$Xe33Gbi)e)S^@87?t z`HoCjq^MUIS{muq`N~`g*(X+n4FHXz9v&WATHI%je4c{2`!I+^KgDWmB_n*TtX$c? zu|-AxYC|VIf8cFv^Zn3i4C->bqB0DpHb+-eS9ZShv?OyXt8GxTrmO2qCPK*-yxbzd{u(Iy)Xw^ zzSG`bIkB;q?2vvCMruc|x>xz)%X}&x%+etKwY{rNIy0EPeUHg*!PmC-6GLcXOpKx1 zIJo{C&cIYunzhapzOP#-O?hF1K^kI5Jt*5J@w28T8+gl%oE&Lu3cWd0{NRxkK$Awl z;rsHNZaWPrvl77DV>1^@W@ zN8ss{6(#EBbwkfWbJ--0U*gv$df+eR4NPT{T<^Tix#n~Xrg}g;8t4`u`wEnwNv`h8 zh5RH|DJd--Wr_kDj>Y#n$>+@UDN7)9QDDmf+IrEC55w1y(X2g>2tjlfWg;OV`8@b} zV{P{gaZWZuOGpV(MAA=AhC-e9{AG|nJ~R5BaM@UiC)UCp)^uEW0#ovPzcHAYYPZmM z1n{Co-K6AS0IL7C(>RXGExXf~!M7PgfD`iV9B*65nw_1KmDnhQlZQoQ-C07s`H$D3 z&xE29Ln6Zbr?az9NftC%;c(%UnG^REmh<3(Dc z_zcy*E-gKDf4t>KVO<4zjNY|zGIaR{ga@f7NyCaO2&Ld!@Xl-SVmvWjFM*YUi%r0L* zm2Q4~6v6c9N6ceB=DK$vo%acRCp?gZ0`W)dv0s8E$|7oYj9@gMdVC8enW6JnU3bMl z>s#--=DE4u3A!ti62n`ULw=hxE2i^zDK!yqi?+6LelNUvC@9VmUlNS0mS&w0* zLo%8Mqc0V~k}+ROI0X-AJ6tWaQjT-Bwt(Z?ClH?Y@VofCne}x^*KiYC{7RD&H1Wr2 z*sjc6HLBI5C2_vlJ6US7>VV$fzY6D&;zQ_x1B}hEp_@LeP4j+y+*^E~d*%eh^h4RN?(gL3$L;}ccuB=3gP&ZS^@F08*KI6XM zS`e1du3l`cLO?~LS5LK2n-i5c-y?QKL}IdYc?5T9OL2sTG77%7NnV$%&-O}0sTq9q zPzv6iiWI~g9*>QjP`>Xm+P+JVxS{1Zf4sjFzRzm#!)Pn_ux5WRHk8hWdO4qn^T-L} zg9t+RE!#NBxAr!MwFj^^#A?p0slQPd;P~GRay)essl>p!8_t|v?&0_>$ie%%uow8T z0LtbCYqf&0VGN!i-k~2Co zq242+kR4KM)0SH9f&?_ z?5uXRGZMZZzTJ0=$4)M zFc>^b9-q5#i5rX`t;6Rhrr$`+?z7QAS%x|BGL%C{mlO#s>zti`buFs&Wptut_9lFs z(J*Ahzs(42SRT9Rs^DN_7cr-i2eP$5HHw{24>_L-bour=aze6bbj;+aNE%A7f6DdQ zAT;1XmO*J?=~KzWB~@rIhQ?BRb8!+OX?WZMa~kS}<|d93JIWJ_miyjAYpm%8^5X&? zsh*7Z7C6pHPa)aa9r3S8Xg3GeH#Rk5H-!>=LoPz|8R=Z&H5i4{bFs5uifH6v!(y-s*$JNM*i(Nz)aY=?=HlJmv)glM)gx>WTzeziLI%byyo0@j@&Pt_j`t-{= zJ`peU?0lMtfI@U&G8WqzQc!X^f~Q#^+9!00S31wwM8KmWTX(QbWE##2vJ;PrSL%A! z-&o0ugnE9d(Q6agH`*~0gyNeEV~vqqDxG*@YLHPEmsN;FM9o(e=(#qcti?I)o94SY zpXA&gIMU?i-W3~~NM2fQlM7>)JgaP&|19YKJb;o#g*;QOsj2FAVNqWB%k&~+VPs5U zV|0{y+1tg_ zv#yDOEPX*L2^!zfw!ps3O{^9|l3j!2?mSRg zIo(jqYb2}OW4qSm*1|~>@>}1>u~teThfNm-caApT1k71AtFdAyu|pEf)*d~}ZIJVA zmeyZ4qgg3yq!T~)nL3lZ#B<%SxXw;gXZn1ET1au+3-!iJe+`4itXGYp2}Mj4O-6v{ zE5J>FKlYW~4#>kxktbg#S^n-eH+J_E53AH$s2B3j@yn2E=?dl9Iy~hmthQLGY`3AG zIynA-UynU)m7Dh37ZsQIM?AdA5rmk1B>;1|)rtLCBn~dY%HteaU)QV|Y@Jjj{D%XU zg_={6|1E?i17-ge<|fTbZWy68rWq5G+7EJ-DKcYjk)xSwl^{96gM(JG<0x7!FxnsR+y8OQIKGx_H#Qzm*jA)NVZuUlRsN^iEAJ#dbB<7K6dY8&l6OGu`vNr={OXR~Sn%*ysaoO5G<*4)Op35o;-lj;-|LzIzAPQJaG$oNeD znNjl-9;;Drm=Q(6$Iuzp;3$+XX|%%uRj+3P=$X3;^vFcacvkUm2LrbI-?Svvk^p|J zkUE?$Da^&QbarqGMLtMEA%C8t3&10)%zOiO8$$;9M~R3$zh9-IUx&OYL=7B#4bm98#dHObe3 z;FGO|gb^8LO>$~P0`9$j6rK_J=b_-QG>hx&2q@p)pN?^6ebk9cZo=Jw-gnQ?lYGdA$2s;Uo>7&USOUO2a(RKKb$8bVq(rPCoQ4#=SA9Gyaht z_CTEp{k@vppwLex6r-pJm}o3=7-a%gynWN&zrz$_{EaV+C#LzVBk0vgV~YOv?p z?>7Xa;k`lT6n3VC25xNg$?>%J(Qbo0)^~HG!v>mv-TBHvDC;4s+XE z50n71?Kq+iDe2?`&}f0t_bRHEUoYz$sksi_oou_{^K6`<{fg;VD=V3?na_3E0?=SlS z_RGiH5hrQ$=N|mQ+Pf55Qb6PL9Z)>U_5dx2IudhIR$JdlYP>x@Ubg*)@!gfcr8RHa zkj-WZEzBhRE_gwW`C?py6utNMfxO#@oDOd`!`DEI8)-%lpfR$N$-MBu&w6{GRY*qx zrbCmQPiV&XB(fyaf`0D_k_0$7%;v-HAJk3W;hCbjSw*>HB5lx^@s!88@3*Ce-zE1- zeDPlZ#<%z3m!A$--N0-L3=$M_R!I}RRlM*EZH~~g!@h<_q{3xl-ajp2XX zey-m39F@J8$b>y>lqB@IEd-i+t(;hXnLbx|*@60K0nUF&*UtlB^GY^9N&IlillB4= zFZ@3Kd`tt?x%h75X7$3LAQp-AF1tSgWzinJ{kNz%6iA*=tbEA9%twGW>=|f2M_^m8 zP0|9>+n2gZ89xKXTd}hBRw(#PiZWT>n;>_glznd6^|JZrOP7%RkA9s=9td8rR#p#|d+m}1vk20r6 z0*SQLH8X8PLJ6T{UxgL!e)SMD@$fv~{qXx;?)QzxehM4cbZV#zx}+l-_iOW?V}#Gw z{vJi&=olDAA+fjQJ36}X@c@GF>tc7^~(gKDJua%piKEDLDu@V zzu_Z0?%3T{ACTZjB8BWu4iCvWW@=4mW@ls34RwiuE~Wyn+KNZ?qUYDw*3J#9&oJ6X z)NSRX)v^GM%#^&nd8jH=RMnJc%2|cbCASCeG~lfVZgp2`f{Y9ep4-sKQjI6(B0Wp2 zXj~76!e}8EK|+zjWhM+vc~)U{-D1QNS&%tODn_c{Qu7vCU;+9zSUL|zc`&^Zysm( zHx|&dZe#?*$;@*jv=?E^@K$K__aC~Skc$??(0TYp$Mu2%^yD_3dE3hF;OHspcJN1R zFuLeEx~st^8G7O&tGIRZAkJ?udeFU4W*qdN^||n?8w$~wPk#OB^6T!s-&nxTLg1uBt_MZB zsU8c7!+^slBf_9N^;8Mpg*PDXrw|&;+8JG|vT8-2H_*`E4UgvTHPW&;?@covxd8Mb z(MqA@+{M3c32O}amKT@djm7@re zE+pNqsk&sJ5}^@!18VtK)#XxbZK}qDWpqLqm9g>U%W9gcjeFq1`OQW1%tQqx_ zQ&kSXktv26L?!y(gbS(7jbgsK2&?8WqlFO1 zN_rtNZGPx` zd5ks|rZJFB$VFmecz9>wIGq%Oe{56 zL?Viu3PayIewjTpd^L5m9S>_TOM8+z-1?WDmlFgdnRq9HfEz2Kb0%! zrJCZvDa0?Y+U3AJ0VVBZL`S*FT{)sx(DnkRpFRLwTpJq=xkeDoCruT+Izscb+oh6okm7vIfG5l06x7q)`{r}^j6RP9)koqQi>`W-LtBWa=(Or-M zF1WP&s=dY0K)Rl5BwJU&rncMVc7(_tVMLD~djoGcrMlNv1!1FM?9JizfSx0yZDR^6 zPD?y9JG;Z%Eck0Y^X(LQow4{3R#8KP>X!uFip98v~u&dM4vj=?~XIp<%|8N9r<1FoMO4nlngsAfH_@i*x8vph67IRaF%ct`uG444&uXz zIapg+5%avizwe{Jl((NolQ4kd5UIuJjd_h$H`YM)%GRs;(|-i`nm4^%VNfR97qd44fxSy^qg)@^NW}066_oLm*?J|P@J}yI}Vcrv`W6t0OnsF={z3SAnS>2 zcP7KJFh?TW`xs+?c+2m16p8CNR}ihN?9#K-jS#0lXC7r3M%|1`*CCK@tE*Ela02QU z0>mS=2#oXgSgMe>pkM9&9rT~&zqz>qsezaC)!dEeK&+W6X5-K|uxvLm(x`jTZ_^0x zR2-?PskK=~H48r?GXg%{&CQXhT5EA#vXqL|8an{o3dkVxaavj}k$!)IhW@{re3vso zd^W?&%s@gx=zRxaK@L+!Lo)%$X^`pfWVM)GP_ktrwl3fh$!Vfu?rR&NGIznQnV6XY zj3l`#`h>C}o@!2iXj^R-MF|T4E_}ZVfCBXAS#Eac?8`VwzW|czElVIg0s^6+@5HGe z(&tl42W@J6@j@`;0c|a{-EvY|#AI)wM53l05k-NgdEmY($aNHf;Iu!LPy~{!kx-<` zu3=9nlTDnQ&Q6pYu0{1pTj~U8)&j&59mb^39gI!GuQG`J;SHINfUqg!x6$l~A}z!O zfM~5c&hmt)nr@Nvt+}(4|EkHK?}^LihgmquvK+EL|8` zs%q5a<4-jfPMskslPq_~1mdtJbJND@XQqAR3WEczda!mywWz3ZvJAZAt=4la#A1d` zBP*b)n%M3YlM`lyiYhu@9gcY=4I_VLankAF`0}#&;-Y4)ou_P6nCWbg58;kqBa>d$ zcnY%f=1{OjY!6fbN?t(9H@IYano&vOsII>2fL46>Lb@rePnB-!ci7#2e?%Pb=Yhf)6Vtdu}3yv1^Dl#E8=yW2N7HYW4P3D;|hoT;nhP?QAc zH6CN_n?oWvRj~IC2H-&5dqp$@Vrj4mzN_QESq!!LvQgmb(E8Q58ri zqoyc<*!W^wES9#AEYl3}t4h#V#wPEa%*;6Wcm+_#oXp(Xx~l#4=%h3l7W*)QdC>!u zzu>4)8jiM&?7cpI4s#{9`-c_(g+;Vp1>t!ID!CapfaOR3D2VEFG<8YtY~bTc`g`V5 z%;M^b^CGTy6342kt}dRJ?vHPoxWzsD)!;4h0S_g}buCO2l4bCd3!GQDuNKG>5kYjx zSMZ~!>CJ9&6%Rv>6-vehe&R*z^9(51cY&NpNopN2jX(^HP2H@35~jYGcm;7$lSnI7 zNedzn^yQ;-zXqRK1_7;7xx2e_gq6I-EWmH9Xtj*D`F;elNOv&J0DceyLT*rVTNh;M z(k^%M)w-f`rJ9Q$f0Cd;itvp?l=14`2TE5?K?W1d2c@PmnW*R5}|{ z$L3duW|+Zu1QMg6!6#X09M<)>6r1GOHWJxLb@h}JI^~ei<(Izn2mbMc%bgJ@?JmPo zb0CZJO+kZJA{eF_im*t!Ey_GOCZ*mTIaO|=IL&pPp$BXqVjh0&{+;8zh-u@+qrmO? zkB=RqVPqCZo^3D2K^9#{8j>Lel3{8rKJQp~e}yBL{rNHU2fW~X`tRYIm3qCUm4)CJ z9!)iH00#@UL}8Ob90*4b$~M&|T4gW3dt`)#CrgVtXR|Swtx5&dUA?5DVb|-BD_zX>_Ch zgU|iHOXZaR4G**YJsRPtH)E>*V+Hx!iz>Ue>_2=avtYmzVyUp9ulUy$QUJ7f|1Xm_ z3pP_*)lvI*NwR=>y7U#q{_CG5qx=m-m(@3w{cGg{w09@L;`skA9Tn+ck>7t8^nXLu z|Ni;!g8ugx`hUc&|9#~D|GtM_el)lf_^V@R6L$aje)|Q6I(Vx$FftHgx&Ew1YZ>&u zzDj@DUK8Nfdkwau^BRNb(?(K((sYsw5fOK`JY1hWWb$5O_RErCM`yGR4g1_2O=UkR za({3iPeh9d5zj;+D@E60sW;P90I%|pm7uA>(LrGYYN4qlurV<)%Kpa!65=nChee!+ zZNyk%qpk+L`oEVOp(sT&(b3T;{oesVLWCGagqZujsI69%!Tx>o|6JZj0*s-hrIz=9 zEPwC!CD`9MxU;NYT^R%PpUZ>AfiY@p%Y^d12x{=3lsUFg4$*MEP}|M4jQ|9$H? z`l7aEV4xi2nF^mVlv@JPLu@K}wH1~mU(>u>X{EL&$buN0MTaM@c*nyfQq z=lL>@<{SOrC8r*sHRPQwGgLqJhbZ1( z&8sKNp5SqFz%Wt#V^x-s$9|r|RK^*ggJ{Qjz00hk-t%=umjBQF17iPf{iLS82@cty zY^{M`!t!P2Vc8}tt>`nFy^A%}$`&=h+S+C?i+p9ZytA9MxaT6N?&au(juF1HSTlM? zfJA`o1x3i%CZZz9eU12mSv@(AiDo4mwe&1z>65MV;bE&4nrbyuz*eY#!^W8c*{P%A zor>AGsNS)Mil2yn@okr%;Kw3t{=&uuN8v2hpI^W5jE#-&`4SY2aa|JEq=R#xR3&Mg zezO<@)qpr2^+YEX>Z!si!RYUx$ zjw(nNc{fj>gVkGx(fAUNw%PYnuWNS@?A7%5Wsd>~?C+sV9A3%upUc_tn*6B^n{j>Y zdZFce{grRV6-8&JKf!+lkwCH)ZugX~#=R$aV0M#_qTAd-ot>S_H^D$Y_}`X+g%w3l z)WwZ44}BHxSlYX3C8@5{>6ug8kCCYbj|ohMzN)wQ{&@q3(OaNh3oap;>^lGlmL*s*ahl!JnZvs+p==3i94nhnAb0d2~R1V2G?pz3z&*9W~# z9%T2BN&Gf;;s!n%=a*7bM@FP7Q1>v5Uy+HN5Z4W7!X0=X$2LCcEa^XO^M|nn8%5$d z4Ox^ahtzM*s_G-yTU$fYmd;c2X1@o7=1m9?x>8kE6?!I|x-+ZNJpaJ@>*X}t@qM9Rsaoau)>y3D zsMu})=rlWacGf)7==<(JYwVi#IpNFX%*I?r(>HdDM$BI`dSxiP@J=|eoWAsBnNiB< zeJDnuydwoCeGF0JtlAK|HaEEX^vYW$*p-z06!Dmwo^ymlpE=lI58=UnV()M2X$)n=O2oTe`Cb3&B5>pxv*>tm^J?qE{E3F2FSrwi&28gWvAB73 z6pD^qU|Y>lo7Rv@CaWRp(#5}lV)iZ=2D!;%BM=BeusbU?fUiHY zLh>KwL0MZe^WtXaCBoFfKz1ktZKJ!6LKB~>Q&Q9e;xrzcm?hy|^Q=YwoZP~mwPIm` zrB5HTV!N$G;fV1~PDV#4D=T{?to=AnFwkwA>ELK#7X>e8lz6I-ydD?;hqup=x8!XV zksZKZA|=U+epHLBksjD1bpewy(0>hHz8Vp`-z=?3VNM7rY37L5NGk)tyzvs)<7rK#jC+SGHGQ$36QRe3qqh@BhMgg=ex!a>}k!gGpF&A32>oZErb2X>qOf>ChB`u zOtE^P<>m{P9tS+OQQp7gG8Q2PA)Z};h~SIs9ww&e%g;tV{?_hHl4K`XVaI#K0$rPQAbB5B9$I3gH6Wdm&8|_!U~{3SFE9_( zkN>Z~&gx2hz(Zij6>Jq?Ew5W&QqkIX5_Mu9S-Z6OHDGA0RV}Kty0Q4D-px8G8YCqJ zS$jrPyNrB4m{=@*{&cozjON6VHql;7!3ErNBT;*=KfM4m@=S)l4p`Q7DR-TsBw}Ma z+V`3_3W0L24#kgN=8VZa7-~5lIzyO040FUXPLQM1DsQAv8bhs!gpV<=m$Y=6Hn)h& za_3yFRw7-xd^J=)MG^-i#CWl0b!j+nqR%36+l=3cWpKpoBsIEy^y%d%Ct-iRped|igpvv6Z~5DyQYrq$$iS#KnhAqD5%dxJG=PVo{M!DTv)_5zL7Ol2c?wi-&fxYBY{$P=d#C;N;bg^h!o&dK>f! z{CUQdhIf?+(G7X^m}b@B-p33#_~mJtnR8UsG?!(5XPk!;&7jN|tG~SZ^4XFZ*Gjjt zr~NUV#m6$4_66B}3T8^Y!+gHs873lf|)r6n<4b_|Mbv#?jK^Syyi zdv_hD?|u?_g@hy0z`<*ToYX2^-O|_K`AF(J63QVLXs?mJ;b1a-wke6+0GWMya-{GB zx-Q%1z~*jdhTlr+L7on;gw>%&JvA%Px6OWFOa26LdGB+hyU;eZaWZs3WSV5EL=t5yjg#R9DAHl12ZN zEwB(mD0=F?n|=|F>-k53MstPFt-dNu`LkUxp^=Yh(ELSh8ohZE4n8q^$We4%4m|79 zx~+yO5-n43Va_}?=+z%|J86-f^odmDUf9e4A1tXPA(c6f8Pf?V4? z@lM9z0900}kvkXhI(N-%93YI1Ag{ zxv)9Y_@7O+!&f7w>Py!?z7*yYoPP%XvGERD7yxDQS-nctIAiR;e-ExNRXIM<1?j$K zM9Eu{u-`rlKeUrlAn^T;AMpLA9r_F@S;;wN7~$d(wB@t}y{@HV!{ilXUgrv5$R^6n zd7bla-3k^`7FOJ6?usN~Q^lmE{kw(mpqo1ZdQB~49;0CS zN%O9~JsRxNUOh0{`V0J}lQvPD~R2W}G9)FyjmEXgd!XkFvp_N^PwTgvqn4wI@~W~tNT4&8 zWgD_23%qbZppZ30Iy(-Qch^oY6yZjz!v)Q0V)JK{{jWTxj5k_%f%|+5G>MphP zvJ+IX<**)Gd%e`lS;JA>cW<0OGkg-6!`No5gIe;A^h4M`Mw^4+RFw2p!!5Kch0nPy zmvrc3{g)I)ZbsaIw6p;B;4?BI;qUGYo}&lU?3&k&kGt# zQ2Y|Oh#iE!09hU+V!TwP&Di~V4SY9F&dy588_v`Ew3;>T=k?=uT~8F@pa~DcUA~pl zpy6ezxmZKpo_Dwom6g@_0^W~h%7yL4Qm3b1veUu^k9{{2fvi>y~F zbH_uHImH^DE;D#jwQh^Zri!X+TCvNrE(_~C9H(v9btG-=k>E!k+}pJied?54k5=`! zyK6#UkF(R#HXJ6y9*~zmlwcj?vQFl)aoKqcsEf#@lD@pTVj zveo*l-Tm(wd=MNX7d+;5^%rng0c4K<&ZEG{{`R(FTGY;^yU5q8yU_;Ztw<6|GDr;re1-!hZjXzS}wQuyr47xY z_Z?bn;qd6dp~2&-B?cj83i_Dv<56qT~QjBFmvA@~a8-HIRvTE9L)b zpb5k)|9=2ULAJi+!*8KapG?{n=&xA0B6{cUr09f+BOifi^%KLsZ;kGJBAx< zEI<6P92Z-DqJPmhslM8}x?L-gpJWO&V_JS@W=6k${b>KFhx_;M&y0&JKR-VuIXNLA ze)zCq&ptbyE=J<<@$rWoGM<{EsZ&z9Dy%@pM_uvHB!OLUem4Um3!(CBXU)NB4Y z;X;V*SXMau6$U(-!3Q5}_q4kC^KSI0#Kf?+)OfGRAXW-+j zoR!KI$M3%TF5@p$pr)naj9nRj)5sNGZSfd^2vWrOG`!Ig^k>efjw9*n2Ib2 zW~=1HMD!@zwr@{QPh(nNw6%zzaK`>-KH%F4jn=V1TxoOf8F!XJ=fCGGIQq4Aw!2oMMlCDX`;McZeKziF3C+HF?4yT zhGZKoS9OaifiXsJxIz6P`5|#GBFe|xeewz%6=*_ObvJF=M6=-{@jLIlbHs>|7K>Ak zT@qDztt?2pL^}COS!ikZ+;h(b7hJe#(V`n}xZ%MEAEcN#dL||%@gSumaq`SS%J$H_ zy0X&EOer@vueP?XxOf|tICSVRi7ow$__BXN{aagGnB#?D#|q4<5&BDMSs4Qe?SD&A z5wFP2$!^oxd0J{3j)4*iGQhJ}UU}u3HESMx@PYF$2tRI2nluSbbEts2SOQfctwMq7 z=68&MVuU?exl9kmUjSa^!^MkTr+2smYKEfq1$}IhO@4oNWU`=ZHn$6aR1 z9GDV2X}|r~RP6uR;-ZD$tt%-0%4mtT+7hkKGXUX~tcY8%`Z7r%V27QK9hA;-ZqmDX?1R2G4i`&fsJyI@x zhgcJ~X+#uO)VuDv2fbi%X{pEOIeqeEguqKLzK9f0O-VuLI&u8?_{1b62qjNVPJy{Q z><(OxYHMoWeB+IC&pj6*A^G}8AAkJ*`|qQBJLZ^Uphv_ zHAzK?uEPjlFCAe`n$Uvksw(PSDk-J-lyaI=M&zr>JRd2#x`t-BUaQ?%) z6h=b7qdia;@Cn+OBo;FNa1q!fGM*Zwufa~XmzHK`Wc=slm$B(feoWh-Hn(ltwsh&z z{r20hwyxG@vwypI@rvckNyc;aHUI!X07*naR0MV?YmE_r_5{>JhYltD@y8!4EG#_m zz;WlFe;yS~-db9knHl1&#zh3d#sWg;CskIo%$C+xC_7w`7D)A>=cAF6F`2i}OoP{v zPd;Is)F~B5yCU$3-f@fG#vg-fbUL(?(uI!tZN zjkWIP#+bx(**o!79+jf?IW~LTh^#GDO`k6Mp~e-cYA&s7EO#56W=pie7OB~SUXxX< z8!6hGxi+p~y&tZvKFQtSx4DzU^hD2msI@F?_@U z2OO}upb$gh62$JIBVia|& zR;@xvAW<+pbqQW{8)$?095!qiZllnST_2~Gr2-GWvTC$Bj7JPyB=&-{MBUgB0B_}1pI44ay`ki;)$?MmzpkQNuet#ZeIV2Je zb0o4835v1@25{}QJN#CWxNp7nR&8w!bnx=a|4KGitzLtWL-5lSGc)@vUABzZ(Hcnv z&w3z@ii&P(Y9xDDoXOiKpL~4FE&swK-M@c-St6=&;S@Wpm&%KXvW3GB{_^6jSYha4MLTRvmh;Yz`%iwrNCZNQkt5UN-d8ZDaH+m zdq?9208%5egHS=tEqNUZ+dVA4iuPMaKrzCetU)N4GpO0UE=SPC{s3)er^Y^|Y(yAv zCifjz(NHk!t?yb}On*GQf1=T5j$=%c%@u9dtp0$-&uMq8G0_!+fk?4A zjWA+|zZg~!B#c??<3$W9pP0F#PJj^-!k~{hVaCrmf_Q)~Ff7705czMBD{faBDfFI~e4kywPuK-8}O$B22)KgEz#l`gNmp^*+Xc!*T1Go!< zfBg7E&OZC>i!Qn-JG(DwQXpo6lQ7>5Vj9&`7bQaQlQ&2nMPS(h z(1QS74`N6-P>&mT;I-FY3mY3ge7IDrM}WA#G5~TwoDjJ|Ux;Rx=0UfGMxb(|Q=?L% z`k;zbE5#IK%jG7uHEUMmle&KWdZ;09$FMGEdSnAGPJ3f2$UfqvnU!-v7Mzxd({7$GxWlx?z;QZ!l= z?Y3`LR-mSa$xnHC8J>SWKj+dLKKb|)Xf&pFx+;*OK^2y6qmj}95mk?G1H4VLLwqm@ z*|J598`56_=0_u8+<+ifRElvUGgBWowiM;${KU9H;}I3t+A4bom{VAN)A-H{t$zKT zAfOmw4^^bDh=UPEv}q1ctIb~J3(VAN(TW0XhCgna>mgS62M!GRzfq{*-@ zrGD7>-4)oQ?*g|5!ZS2^w(3%%scmdZ#M968Uords`a7hyt(g{U}}Wwrymxy1FJQF>&Om zQ4c-zu+9ePLmE^-CWwNEQbU5rj~`Ds+BIoqgFWyS=Evvf3%+=J&Kw%#4LAH9CP4#$ zvLFKCIyE)bm`cmb%Tdf^W@bv|99t{%R8L*cK0BSggxDm7bDilR(NR&*2mq(QK&o1r zTTmWDoyjdb`{0D?6*gh7P{y0h(kBvyUt*%@jLaP1(@Y_fl9DgJ_~Oeiza0A}<&s+D zH{X0yt|x#4i20_bEnT|oq?1nU(Vk8;lhPS9(g3> zn=)Q@*=6JzIu6~WmD1~RX<(`K;>F)ECuD|p`st@r7G90ZHA5(6WHf*;;wr#ODX@~W zEOLU{j~*w$w^(3rV%*Sz)RY)ETw;a>oBZFn;TC7*ef>3R+;2$g&wu_iBSYA6gZ2i` zW;qt)2K1fA1VsFGgy)`MqT>995a`Ll?l%O}KFTW6%z?+B-)#!E1r0d1vn-@dY(EpU zc|9B~GBCjI9I*qxD_po}&9X&n;t$B{ck0-obzZHdwk+Dv+BeQ-5Q;uuRD|E(usOn+ zI50XTBib=K&rvio>6;a)v8B${^-(p|8u~}gXET^J{w$nt=kl3w?e<%YfdK1_g*93J z?IOV7#9M)o%~Hn4UCI(Ye|mrgzZ{Hw0GV&41jmz9whe)wVZ3W!_eG~r0c5Nq5+ zh>4F14XKYZM(T|qe{c?11)_*w5m7{V>&-V=@qiiYy6dhZWqzV|-=8;s!}|3TCr%ta zdK9mI{`u!=GjXx8{Ra%-1~kN8Xg(j=+qkiSbg&rWgNSDpG8Nqwk`EG)nVCVUCC5b# z<3I-ViV2cPWyl0h4tzy#hZYJQEjl6k7yPNJx|+^K8L8+1 z3yHie8)9c-19TP|hUO1?jb0JCPOGDUa1J;nMTXlkg@c7)f|Ot3n{*0zCsoFlMQc{C zra3_eAv*9FsvDLd^us2Bs;qjP4kII@cuI6~ond1`;4dRXsHSjTrLjS0^cvObAdd00u6<^dm^2LP&6unZp$ZaZ`!oKPMA0mMYSwGo4@cZy?|mul=%;f@XQ%ADl4mbHA~%T zc+ftIkdcwWEStz=k59-e^)7oBP|Tk{|JGY?Wez~IEGsKxn%(Ah?-(~Sc8nV+2-DNV zxN+HKm!5at`Bz?f1s=vb#tq?S8h+fMCGZG4KutV+BEt@|4>^zOq{!;0-xC6g5%yGV z3Q0^~=lmp}!Q5(0aU1i3!3eQZ*wK1qm*b{9h!mK)d_;ppxd?hyd!nXpZ9 z+C&3KX&H?e0h5zcs;jHFZYw52baV{c84yjn2Ne}7<{*v6cdhl4hFz5jp&cF!>Xe}37N;t9kI-_HC0t{2?;D!!LBD#h>XPjsHK?m zRfCN!&5o!@1WOx4L5P1pA{@CQ?M(x~_{L^uNMnthg^4is_bF*_$=b#FLdmwRSVH#0x47trG!MM=-~@0f$OJ#3WWrtv zS0*w!r-g72NrIga!jywR;=l!IKL;OtFxf8LvL!nwCwr&EEi8EV6l-Cp1|52w(0SCX?wy$(Cwaf zTuk^QXdS`c-Sk763dnk zK4HhVJ%B_-`Hdl<7-3IUm|$I?Z#HTz_=-gg@w7x~2~3=!V>X++EXv`vn7y_*>)K`7 zgyXaEh{=LAB!^{H?TQ|B!~%&kXbMtny*tQ zXm9!qfr{29Z_pnw)wTNR@DZ`eY+@g9J6aK=`rg$6W4+N-r*Ne#bd-JB`Rv;K{n{5@>p4v3rPoO128_X>XnYlMQoKU2Q^KeBqW&ufO`rh%uvY z_`e&QTbtnxWmT0Q%%8upu%J(7=Ba0#-r{O~X8JRw+e=c?)8Q@$9()jj8J!&d0;ION zXv?FIJ=)mXGIaPbid<4!a@;Y;;O&a42|fxVp{8`r*^ZSTsI?H}sFB#VhT6={&8eua z2AK$lv#Po(DKQBg$Oi2S<~MHt{aSSsJo1(Bn7CLP9+ics8rrh{`mz%AB~rCRmgF`Z zvb?)~hOPHqBec8Mn^?QB&c32vdghs@$%an;`bb@!>Qc_WN$Uz5R{LW$Ae|(VviqA_!g9+?V(nL1NijIkfLM>Uc zgzXVdJMC1O8Kod5dm6xKV=XaIaH@c4 zSvlEr-hGFdWv>4CV~_phxN*V7KQV5g<7YPtnhmpesr{wAXjSzM3c}2ILkOsw z-wgta5%zRd>iRf#i2hMNFd;WJxJ*HptFfT4XuCT*!>*;IWjA;NsD^w=iO$HrQ2~$L zVlrEuKBJW_qMar&08yPf9rS3PO19;9TKX`2c>KcX;AJdwm@OP%!8|eOx9B?`B0hG$ z7po$ak~F$@F%Cl4atr(H{JUNLQ2WK1k4PjGlcLSiB-2O)OZxw&lV zH*)02{r1}*$9RYowoyxb0#_x(g7X zvt*v4p$??7W22+f($i@(l$6QAwk<`tT{20KQxt?Td=#0Df%M~#KH9W-b82!j99qgV zJPxT*p2eksPcHPVC>>Flp*Dk_5Q3x!J#?HX1BuEa18N^L96zCUl5~p3Ko$4M1#bfx zm=o$fGJt)Ph1jUx$)TKX0jXKKDd25&^DB8(35w@3>&6qW_ zuAyEGSgiMIZl;dt4&Xd>>QvH!>9XpD78M^WX5HT|UBV7ijV&!>_uv1C$Dg1?bP$$~ zp#qPKiDi4vTmN+nM&t(`dWa)NqHLm@Gr^;w(l2NX`nX{jI&>&~jh}EP`8neTZ)Z%# zEz#R1JZ|OCEo}%?5DT+KqAaYC`*r3E;S0;fN% zxJ|2S(ONvFCe0dUw6jkfYl7K^7GEL~-~bCI{<_haXZiQX#8l7RPktbi*z%P@7MaAv zH4s)0p-H{X5AO_fA^DJD*l@9Z@CmP!%!w5Y$ZV1#eMtg%Sf&;x z72Pi?Y2c>pQd{T0+JHp*Sg1(%iOf({ZNj*Gy9 z2C)hN8ia>A+n6!jv%sQXem?Y?;U50Cabp3poXCU_89^oKV{CLxb#1jZ-WnYdK?T5W zA%CovKyG=x9`uGN6?hBv0`hVh>F$~A-VkyKnM3T&nmKdXvSq_Yj%1k&d)=*Exe`Uu z+O=z0nhL|a_L^&-eEKQ$g=O1ICLVd@!bOWB91d1nlNo3qi&W(l0u=}gCQ)nASQFzD zDFt+pIFKHhZOI6!EWUpJ`R9<){SP=`^yo2s6?Sb-YOz>Yf&)-=49+eiyl6`i<~mfZ zl1%f7s5V+@rYN4Y@4x?^{Bw#IG4YNdWm+KOTt+7F7rrF1-6=`Q zOvX?Sal>06C3EJ?S^C{~BgTxm`r2#h&VcjL#~)2PY7)O<6n^QY7g#66kvS-0VOO}W z0uha0^3gp9huV>i6N&?Q;u97NkrMhpJr&s)7^cD^FXQa_>)OSn!^(>Ztkh_J_UWfg zGhjb?d3h}MTD0gZ4y`%ml*yRdX}4yRn3Tfw(8vb-K@`Svydsc$Sx0RA#7;dZS(#sA z*=TBQfkQVmHA0=yeA5fyd3ae^OrotWS+?})r=D81YE^!}{OfPHfe>;eOFCrGV2-JL z@V*BcYHRo3fB!=dJrpB=RC2R5Ju$gJOboXA)Qgp8-+c3P#|`=j;|7{o76^%YgXn^ef~xfP>itWD^Acdp+4Jr@x@>$Ew6A!M%f~w8r`j5FJ0L$e%R!(MvGnB zj!O;FPB-o%379!XapYYThgCI-B`I8D!#Rc*RxVpHC4aw4E34L8HCwV-$ zGXkCwwyM4sM>rWXA|*phC_4r-GAyNM7Y}y$hSRmnea}twnmPyoa6be_;WMQ)S)b)rkc9by=%D68tq!fGw;_|_~`E}LRD1KOG zRJdV7c3vLEsjR7C`{1|Uej7#L!Y{up-oAaz{`*fl`e;NWzL;oG$QoucJ~8j&jes<@ z^yx!_s9rG4p}=UUsl#<_z<~ZzTLbk0Dy&*-)~;Q;bSaCOh{oITXk;m6R9tLbeH}Y6 zUv|l*oc91lhSRW=*bGoHoYmn1e3+JK!d@RaN8^QuJ`@4%4pwP001>BEprY$R9SMAk zN{8@Kzo=1h8AP!M{e=;dZ~6jo$rdPA$uM(}90g6!k|p0_e8jyHxsD=`n0y*dqTzAL z76?bg1DLOC*ROl{p@?oEz>#rX?creOl+Su?BBi5{2OZ9#G&0!ZD+Uu^P zqw{jvNzu7>l{Wx8sPa;pIx|4Z$SgwSNx#S#hre+v)?VmJ${jy+$3K$4()q|goJaPK z1q&8XB3xtHBk6$$?nhDg)?05*nsgM+iFac3U_u2WWN?7XqLgJ=lX(%AT%pPomTNW| zT(x!$lMozy#TSh~_z>oG zbYxmSo)T!dZ@B*Y(#neQ6DCZ%;tDxFpu6R9o`s_^iRGT=Dl-7ZWZRl<88-rkfE;hl zx?j+aaYNW-#6Topr#^q*2q;GQYp7oiEBTSR7PVb;OhOZG#_YFXj@?>X`06{K)|6!B z3>h#mU#j%*-$xEKd3+5~fi}C*Yh>IEa83#1oC$59W{wYNwoPr?vhTc=Rc@~}#$b4f=&TaEE00c^JsGd!&J13t}7v4Dty9Y_}&NfDyfByA6|yur6_sT3h%% z9UwrYAnbJU@f>xI44S*Yg4*!0-FUciVf-bW&1I0dH~#(aFVC2Pfr=2;4sYGO`A;XE z1eZcmA{r4JYgVqr|Bp4H1R?-feF&q{YaOG(K|I7c2fVFEcsi2~)C^oKu;88w<*oO5O|eUm6;it@QlXhrkJRxE3dwaBVth-$HyncMnxlf=g*(7 z&qJIrLe`hyc>49CMG#6BC%yOHyJ&Z?LZWuU^GrAh1_G>Wh9seB#$OG!5UX%UrmlA3t7N^MQ*Dpy~z+25rU-=1Bi{ z=Ra7I(0@BD)}KFaR993+*qz(Amy8%O9BnA`b9xq;QFJg=@4x#S#Rz+{iqRp7 zQ;3Df_KyZVPI`(>PjywgTy1`9Y$`^-O~s4rw|}0$VoCYtWl1rP?6jnTnTdn@B=%2@ zO-_vA{19A~Q3|3tsEmZ+fBC|&~dEjLwG0(yeOuq+lVt2X+#&apML=4u|1hea$uORVyT`lbub(t{L*$lX4-0 z;l4Yc_}{-+jak#Td$|atfE0vkfR|t)=Izq`@QDaj~GKn^5;lhO|_LeVS&T0&@N;o^v@JYdCe}wtbehLZ- zmVEmyt0nsP@6T+C2u#^%b=;sQgNj93qNptRgg*5Y3w5E<>>`bK>!vMRaQ7_Ou%5P8 zQnIaYb`FFiEj?q>QAhpZ4}TD zNk>lkSTaMy5DtaichSdr9sQcukt)=O?RDT`_*ydp&?GLp_~H*g_yCpj?Af!&A9`rN zyj%*1ONPt>SaCznX)`EpX&Yz_rPV$^FR#hP8vg*}AH5H#>B5X@+?Y9&DHG!`orQ4&4QLOxe~Fd){ACbOjPRGixx=^Y zB!fR1@~}c|UAbPfM6)qWgT;}anPM=GiSU<<9veILpJ&FH@Eo$(G`BU_qPZJ2zX#Kc zX0mHWdyC0YC)O)#m74ZpiT1%)UN4FGsAZerf>1#C5{$6>P9_{w}h#yaY#qeAvy zTm})b@ZI|#dXQgnpa1-e1rraOAW0#*IF4sP&Yf}QnOH?JdV*MDViLN~haY^1LZD4# zFK+DSu2vUY^RXOQbIRxV! zR7vVN;C$#jc=oI_&cIC&Ud4O)N{^rcRF+rZ0mdeImt1lQYED=?&t84?H5XiPK@X}) zXcE$7jGhwN{FlF+1%u&;IT+#CvHO7)%0l#cV1o4=sz+zUuK=ckA7@N#EQ;ZaF1lD& z3JO@DkoRKpqhF}Fcm$)`+)g~JQcbaBcyUJAp94_~oj1vp=O_0`O^u!gcl3;dGj zU^)B&3)E1i)1^=PI6M};7xjzhD9JF#BYWhD5r9le1_p6tN)XCwmtN6PWH>au8Ad20 zu=6?73FuvFO7e*(okZVfAHHLbJr;dlVPOFZysNLi8vLbe9gomv_#{4^SEj)H1gcF4 z`!2C-67xO6>EO^QG}+b&C+-I_CdS|KhmOZjk$t%0SqvT>&6qiK*f6Len6oqrm3c~H zBGvKO$F|Bgf#LP& zqyKQwK?f5D{Sb=G=AHY8wM(1%bd$>!WS6*@IHM6IsYn-T4fA6M1Y|QTkd=VqOjqF> z*97%!mxBDt1U+ncM}aVada-y#oc~4~X}p#eWckRRl#8On+jf5iNL!E=;mU<##_5c} z=aOw+xK zo;eHaGfZgw`0rvWEBBS3xWCo0uqY@+_31eM7_MCf~JCLl|Yb{4yJXyEVmcrYzf3uKJ$f{d@)Rsz&}VHNEzJV{6P1?UBNrbD4= z!j%Fdh;dU8JfnfLfXE>+xd#(bc76A0REem0AkA<%x%{8~qalmLM5`^6#R(M+E)A6{ zBO~*!yYJ>JRRdF@D)I54A>H)Pe}28_tMv4YSKgkB`ii#@dCr`-Ica0Ugb62~{Ac~? zu6=+ld8P^|7jzs&6b;*AGO@BmCWOzcyuno6wWNP4wwaScb@mJap)PXS6{u~sP zn+tnG;dj>AXQL+rU97st9(ydu0W-QmBB7F!p288~NoWVD6qqsUWRkLaMJ<$-m1SpV zLmZiYFi&Hi$vr)qj-1seYuexbcKR8o=jHWdz@gyKKH1Z6x%C!$aY9TCqrt3KUg0fb z^e|cGzWp{!l3}QK-*q>M(6jJ)&`mrV?VND~;!d4LMB0D;xWNwL1N-GOkznh1-S>xc zRUPyrx+n7lil(mLJp$d_*?#wUb#DRb^I{JKA)gF8R{9#n!V!p>Jua~pPdR>#XJZk~ zoSf(J70-TsUc7PtoUF{OB((dMmbTcwvGI1zV+b$@^`iP{6SNRJBJ>RUBuq49!;f>J$jVNBw_7>cP*5)BKq0{ZB$td}Od*`>93#+OsWw|(qjZ&jac>ek4S=O+7xvHwF@jTo8 zy|8p-YM62b{EYR|z=}qJ>3-qnO>e*bHthg?q2y9{nv3QEU+j|wNx~TpJpo!bJW3v~`kiQO~Vy&*Es zkXKn>9KUw!zy5_)>x{F{W_2oU4Z=ZA;?@_-XgI04d3;&74O z3r?qMs;lQ$S}cNj=+I#?F;OH=OqrnkM3y5U>9e~UOnDtFhaAEch%e-McU<%XCZ{jWb1h3mQ6Uk5Plz(q1jQSVpzB(?WC^egA2IxZ0}cR5 zvIGl+Bgz}Ff_%}aBn2n4s1-5c!a2ZJ`1ZB6(Ydd^<{Er7v-)OVcf$>1MvrcDx#=2M z;h%lx>7t?{^pxn;&?Jf7(K%Mm-39{CCisNgB{r1l2m{_ucjIj9=bn2G9b{8eBkSv7 zfi&vc`bHWFUbDCyFn`GF$FXd!8239$s!QP#A|BbWVZ)$71G(Y+9&(O{E64f)9T_7K z^X_AhJ@&^FPGAPh-h?nh;9a|R9gYzEDs^V?MHX7~RYEFm^yx}=b~Yfurzt+WwzG~K zB_b=y=*YyOBG~YfsUSU+ugrWX=6(0y_r&9mv*+j^PdwqutFB_dQo1?bw8h2658QV@ z(<(an$&)8DU!rHRTO=cZoHA`J*a&500?^fQgK_`xi4$L$HEV=82s>-)l&NHfsai~A z_HTvcEvk1@kYz}Kr&&c33(yXk$qehL)&dDJ% zQUrO!IJ-jw5dI}S5ZY0>Sj#DNNcHvk{rkhcn2w8yen!S|#~l}@TuF&tWdNK8fnO23 zE@!4^B9hr_gZ(d1n6Lr>nZ&Qia+Ws&I0S_QYVoi`K?oy6egG5NQ!z~IfU)Ty=i8!7 zL;Hm&5{h3vH)z^qwG)kZ5m%V!bfLBHfQAV`nxV7&p%l68w%g`^_+es73U2Z^7nW6& zL$+8@!cQlja3a}YCmNQ6U~Xd(2oe9f<5i6$(Wla#d|@roJs7sTUWQ zWM*bUeQ0t>TvX94BL3lrA3{ZgJi##P>*~a^Z8l=p7fUj$!8$BglZ0Z1Zs-F%Lu$ZI zJ~3r-tRiob5v0IlG9>9B!VgYBOjJ!!3g{rKV&O(arnTdeA4>M;+(=EHOhHTl#D*_M z2<{S{?QPq(Rc@3cSCi0EAS>`WcFe&^1CIrG;Rz>9@=M&m`>d*rdljyvX9&YeSvg;wwS>;J*wE|t~QAAj=6=1rT}5{NMYhf!Lf zRF+Fx42B)OS1KCCyUNR%JoL@#OX++JdV+t5b}e*=?B!{;Kt?2+?n4>8k(6>#xsy?>)}-f~)n*$r&ir#@-j~HfDog{LN-%PTo+$*$?RFPFnCFcj>IDKNH|RM ztRO&|f{6s22e}dFM(lQ>BV^4c!AQ8xn>UjI$S9}h(W;?|gp+eV<&Mz1>O$lpzoGQH ziw=^Y>}}9c?$N!;av}a_%$y0|U9n=t$dMyaV8dW!1hPu@=#?x+M&*Nx#WBYmLn0k9 z80?RzqoCsYVz5Z?r_Zj{8Hghcy5@CQXiowA2Vy zK&8MQAd$>bV1XzIB_ouLoCm{RsMZMIvSkbL$BZ2-O6S*TIZzP7QP(kI(Cl&GL;n;u zg}n=eF-w&ykDq$VDfpi>G}NDQ=8Y`A0VS9y;S`*&IJUBzBeR0@&p#h4IpNGhn8WPF zdu4y*LAmajKFbz^Uw-)|oR7sNEFe4mth1ha`f0+Y3J|3}E)!oZT*yr3;>$0`D;K3B zb3)J-*~3ZE>!HeK&6i3%M5TLS#j|x6okCOhuV=h3!$k82MFubYRuC}BlHtkPo%wI%-As; zzJxGl|5Y5EvEN;K=_Q|i_BkSZGpONRBoNPXKFBbwPw4#CtdvZx3hj|Fbh73SSV1xJB?3^{1keGnN3DY^kfT?^W zT8+ATlmRGWB^~4uGz}T)LU)DG-Lz#hzhcTSD=#NR5;DXnnl{YP=!f7jFi37#da@^Z z`#EvoqX;TQ*Yszf#niQZ+cr`|wqbunL(Fb}xv;$K?4d)4_SY9Ez+K2Z_=%%{=<>Baq8A>(+6qA+BbeOZUU_A20_?>3HhwqBg??4UG;5Mpso=<9!8H0a9op8?;Er%627%Dtg>)Jc7%A;igSc93DY= z3CJ`OCJ)#NS@*zGe3j3#trBm6Kmawul|TpwF*{;l?cTVDUr|QNm{cVK7`q-X#rYMV z-KkTjP*`RuybKZnkt_DhSk7ANRY1QgAKm71Ld1gWj zZzd@KBoFe2PXb?f;d!)!AAI;Bt4#9p@!c1$vqYbKzoOpQ0#o_gDT1& zndJ+{X4Z;0|{m*Lb7eo=Qq?ny|bcK{6D9nh%io9EHxdkNw0zj(GGBf)iaTy?R zKtY>`XlJCLpa>*6jK;@`(_?>aSJ-kPBq*1h{vr$6?VV+F8^OP&6Eib2Gc&U-GsKCR zV`e*MW@g9C%qTO*6f;B2%n*ajw95Z(-MugN3vAWujjE=kQTI%brsvn^JSPd6IvZ55 z(yb?(R?G2;T&J3*0z=S!b>+|Fwt#B-TX<~u;i?U}Ow1G-UpWR+Fi`mYWhZ=*nVSw~ zTgIar$bsIn@n?q3V+2hjp&o}2Lo9p0n94SEt;Js2YbPYcH<=Ul`RF0McY7ZNd39Ov zDZLQ&b<9v7dW0w#ayFpQ34hmyXIXM{kv>(4^|6dIPzNmE6G> z({`-9_DHWSa|BD?IJC5-wd-t0!kJU2BJ-5co{NN~+l+}1tY=9242KYhIAm$RCLhjx@R2z6~j|BwPsR_tL_T?a;#jOWp07QkBf z4mdta&e$Wa+e)~)^KFnE^~+l8ivoP&LQB38rB8H*ykD=Mm<86jiK8Ue7kGhjVTDJt zlYW{;+(3|}0hi^BSj0KMMc(9jNSN2lMXO|&srA0lPo}_s&S)_K*KVy@1pStsT0v9Qwt_7kxYT*ElXPbe7X5Vdb`Ku7V`+VQ-B`VBzW{K&Y7P`%U+OP6M&DR zi&M~Xizqx6lfjAO*PdVj{Nb!ot1UcXV%*DX>uLFjS2Q~oKoCw;Pe&WatizUoG{{wb~0tKNyj$=iC z=QxI*6=4q^leS+Yo^Lk-3~tyzIw;x`ufN9GH;L|vOZOgOQGi{l+v>Y?PgFkQSD%80 zvOQ=+Ed;DLdfxWh_h{Q%`!1-JVs-X0fR~ToTr3lGK54z_E3y7;} zShZYTzEZ7)pye&{{5YDT{w_5jHjFTt(=gcEsKmi&UFm=)=*EQMb+O7FfP;@7SeXQe z25kT!lsT6p=GQk`Y5dF5btUoWE@e8%f0RA`kWpZ7=>hyfBB{jHV4XY77NjYKaTo>hxWNnknsZ;JC&w8*QCG+!Y;x3rw`~tJ^Xk{ z)pbOW%E(Mf9uPu5Y(p9k>zkuOOBqBa^ULxxPU>Vw3 zO+~Hvo6N3hu+yhNkiE*1I5&FKWMp>R<_g zL)q*fFz=a(4`R`i^y@4pv!B2Ki~1%J|GvHf8s#={#V~z!mODS#1jmo*(|4)-V4{XIfK?1aUbS~;nTs{r`vXNTmfEBqO7d1Hws}()!Efxr?)mB z?tHavcC|k{DmX?BS>m!+=P~7&a>7n$LtG!xZ+?=O)a|(7b?iJMQrIS4%NX*cER*i8 z#ct>yqU9{=vgd__fP_4Wj`N)pv#*nviG$&4IsQMO3xJ?&$o^;Of(3Mmd@Vx?7&`q} z+K}=D&D{Y@NOv>N`gCP2jb(_@XO@jrZi`8@^EB+s2_>5It3Q2CiK{d_y zdfywzj~{e`VDik-%`Kd!h`q$$-HPe8%SG@mkud5I{Mo74yUis~*(rdsQo}{7s|2#2 zI3g{yJOZPF#6MpLSYYJEpp+w{%{gbA{kC$$U3==da!4%j#wY_Gb>8UyadvVfIOul` z`w4O`(77UWA4scGhevN=a_p(pDSf{C=(*F-*gtpgd*|opXYBbPfuL$l?aL7bk&Q8T zt|*GIV>cYhO1q}CHm3juza%9`@o<}%N$%0{6~;;$pO;&%%qdp8UU{j#=9~PY9#1p{ zoQe^4LQHH9hOz@+p=mxNFW!mq@Ik@PVkA&Ay-Amo?ZFqQB~#-W*xR!JxFz}i_=ZVp zTrm}QB@Bs!b#r#e_7Svs&SRtg&z$l3gN2s9z(z}NM@j;0P&5oZr~FtwybBwbnAJ=0 z+1bKM#M*r7P2dSmSB>`UoKL{V8v% z97LRq-Dp{pUy!Eff}XV)ctu~9&0#VqixGk%PfVUNrK!QSC&=!&Ehf-Y4~qJ}>u2Ks zsYyMCO569P@l_*TeQo_Kg%0B?j8NnWnt(+AHB>0wrA`Q`@5)_xUI(p%2ei=Be)5l_ zo}83L3$PE-HYKW+Xj(4BwV$$W8)aXq)cbEYt}ov9L^gY{=Q6l}@N2&qQ$}_Z^!Wv# zov0%$%lU)ncsq|qWw;%O1Zi<^U#@*UD|9n^@^;v!PHmquUWA}>%w9>c-n?_uAs+Jti%`MZ@heM&ui&Ff_GV^LW~t$ zDh*s_`^t%(6-WjMOQ4|eCexXx9?6|2ZQ}%GYd;BtYZ`NnXX~1Zr0Zq=-40BOBoK(S ziCVMQClg+=V( z4wu{R*@Rfar-l{6kC09@2w7%w1kg08`l2JMi@Fxz0Ks#D?V&_ZmBo!ieAk3xXYyi< zn*QdJKZzPay*A+RzGFTqY0GPGvL=gXPu2yjJm6H1H=}PDih}qwxUGWpdj{nXMn`v; z%zb^mU`@-*-%kHa-S!)OHd|Seaiyz1N%;-eHR5yZ#+o)fTG@2oNTZekkfH)Pag*Li z0J!oUOU{)=^^0l{ejE%}Ws)Lu()^*oIL1>3EUiVEuhHxH)h`u+XaHsVj??FHc-OA0 z%U>Sycz0(~8?4t322HBVxsZyH#a zBvxO6TuC{?STW0BWFFo7bQm#!S_VMEn=!v#a#NT@CKpk_fGSJIgPxcc(C5Mk%5;mJt2rhhE<_9qZ@XIjJ`WkA z%o}tOWDg~+hcG8_+-B~zA}p{vKR;3*W`+I}3Mt_VPELwUt|X~)p|dA*tDW{I`zYXO zARTbmQ+0FT_aF!+FufnG13xVCx*tx?@_5V;+?OU?RPkYx{e-hfpL8v)nMN-=4%lCfyE%$NrL1I(p7L;LFMg*mJO?P)TaaQW3nLLk` zC{~Ji9}0ZuX6dUh$$oDP?%80h;;47QzPr!UIM&30YzJrJkh{emWxD1|`3kVE`+imM z@-Zp)FrMz9dO zukqB4g6P%lY{ScRSX-PTbVaVORqAUL@twb1UNC=0TYVN9J<|p>eYAc&=QQ=NZTb!5 zZ5dSw11px<>|eLTMY;6Y8s63?c@4crsg;c=N`)oiY(yq78fi9r;{3fP^Vrne@0N50 z1qHFR@i*^Vwm4VnhY+=Vc9-_lt2f#@2c3vy;Gigl(qq99=OT+-p0+Q0DpT3Ljz$3X z1w&g0j;q4g#*PJos za(jEXd7K{&EnSut+Sx1U`}?x%(E}Lvk^+Z~xNWDC8kTxCJBlobIM=%~Zf2ssN|tmU zdsn0f5l#JJq@>bmOrXTRiTp-R>vb?qe9Zq{m`)DM7_DoXc`-WxF)_Umoep+4p)I-A`>ow8QrLM^Rlq`Gg3#abUJM6Z+*+`CU=SiEa#+yiKSS}}to zBei5Nf^OTQ)yD<9pUeVk-c5pR~lti){q<1_C1i9 zx%84S;%q!WJ@wm91XSBrhE^!(IMvhvm`athl+%2`>JTFb2wF8l8yZYPdHTx*@;2~8ibe)^12N}`;(4pW zpBG*0t}a@)hCRe)@EbHm?i=F&Rqt}MqH|Iv_w?BO^;<7JKdqkQO=);zKNgX8KfnFO zegC`Fo+x7L^I3`p2W`FftnwyVMa1>hNujl?ORka9V#3FG4^dP9&RAdpJRu+OHm<{1 z{$_VrjUVOyxBJ2GAB*#8FFu+IdeLsJb;D)Af_{gtyc@a@*%D4_F?~CP^(pk@FvnN8 z+u)S?H_5|QA}T!4@T{$XFwg1K3Iau`Il3s9Q)8xw5#6pqr8MNQZp@Kqa$te+GJ9QtyiO;|Avqp-g2@WHb?G0>p4|3wm3QXJYkx)A11uL4%X=U?Utr z!l1Xcwy&`fhm1yOkRu#{EG{^3i7&S$V9)_DQ7hzpz2u#t#FnFN4BQud6b4y)K_>J2_dCVB`?g8uRqt~#T0dh)lVMElH$tR^i6~>k)n?as^0&N?q_MJF>Lx)VrzDg9o5hzfo0o*P5iZp)&FT6RVcw0$Nv^G zj}tVXrEHzEAWD|7n_wqh+xsp`ZXrcEXWWfyhF-nj?mBHuJjliir`X>?b%DYVWT47v zaiAI%KmRj`p5e$Nn==Z**b*y+fT`k399m#I0CqqKhw0d5L(-Q!XLd55N-<_}P1oTK zBA6E1k&gI|VW(QOrE}Oi{mCPNR{TM3GM~F5?P>;tzc{Nc}vm1 z#HBclg}bH;ne6JgNc6=YRpjx!!xdtk?+>xH2(idfp2Ny-m69(!KP6lTO{X@`1o{jd z{BN^Vo*sXTj&cQvR`Vp#_;)N@--IC(GQ0Hb5r61}p&(7rO2?__2MN|EPk0Q|H!_|r zJIq)!R3SXaOb&kRmS$PW-OQ+rNwFOU3qx;bd|-9Z$6JE8pRwcH7QBD42m5>R6< zK|Mp#7Ef28ko;z(%?DHs_tS*XnU?>`C9GOD-;Jh@MarL77P#t1n=ZN$Hb6l8c>?2#`}!|%H}?L z-!2>qfR_?owpX`P?md{E2hGWgRR~>2fha*Ew9OPrZ^zSqEp8X{h1R;vW|UPp0x^I9 z1@`sY0YaqD{ZltpU_kg{X2&|17tJ%K+v&#KO^|@gYLKqpagYe`tCZb(nsVRn%h*G9 z@CddR>2_nr6#i^>*>yX`ImM@Mb0`N#QoXwh;w9xpqEuDdgDDEq%(l95tg+?GqYrtn z#%_vW(BfJaUF2fTb!@OOR8WbD^KK`z%}*9XIbdw3O>jE65+lRg`DIIcoU8z7nzXbz zDL9Z@lIuz4t=1x7!c<|Q?7za=w9?ES(?O@t@3pcmruMz(#VOw*+uA8nz(KKP$n~%N z^9qV5fCcCbKl*cHBP$KKT7kAkZ{%f;2}8f!E*$=v6f7m>2L?XVmuxTa3zR~23Nh%v zAE6A?va^@gm{LZyU7OhkeCWnyg9akhl>K@}Qxzy@?*_V;gV(LJ@~Q zuRx8Ql*FVmOB@r`=a?hFyyl+V-1n2Pdj#yd0ZnkK{B6#z+cD8hYqMn9$hw$6b*qZC zK81=TH~F~y+w-q>B7C=?{_z<9@e*p-V_B0N@*BJSy51nO0IZEMU@mT5IFliY(Chkm ze}UH|V`EZBTuU&*80@;*Xl&+CSdk25Hygt`5BZrSgMwwH@hNZsDCH|BC>+m^-g%v4O%mOrdJd4c}UrRc@|ZRLexngmUkj+wNg##SAWPHI_C#^dL7cph|OvUqYE6qjJkjzQPKIyK;N)}3Z(d2oW6ulp$*%44 zS?X=<2Iqq!yir)u#{yf=oAn>P$LHrW8uoha&eUmXqse7N-scUE2!*5Tu@Wd3hlkwm zm$NW9`->CFFbYDiub4UbXjmKr5}&I#ID@sDPp`fIzOm{_@>xaZjqMzQAzx@(YmB99 z`+n*2UB3%t8eN~-_TmnbZ140|x+)zL22x1h>sL?MFk~Z%3UbY*9_dIbM6VWp-HN!+ zD-p;V4_*LZUDf^~D_v}X5k@tDv)a*nGP}p8d(ju*M@(R(x7VJ9XeU@L{m}E!P5L|& zUfX%8qpv)0;Cz|pMA-{x)vs#7>#0jnnu+MIwD;H4bdZGV#LhKNz$+L8btDjMy&HX; z4|BIm8RwPbm!f9Ey24tQHRP#-wOo<65^^zv?v$~mjs{}HG^U(&opU^<)l&jpLgdbkc^lIZ8SA1-_E|cW z-|8Ki_}?)p+4J#OBYvzAQG%11DQ$P#p^G-g#eZ*C%DzGQo5xDH;Ws`cfdcddKHU{6 zPGT%LBC(ejunk^7q6&$X1z|Bj%F5vi?G0k8oNq4WoaE;OfhX?~o7J5`c*zXG-Za9_ zklrG{IaTRSd7Z~Os^FqkgzxtKe#P%0Vv3pC4^QBqFV4`sf7Wevn>E;${PPNyYtlqy z>d+xIMsVc^lw~_daYQ793(FQ!38eVDjfldtu?cGz2|aEHL+uoXMw>E@QSRnCJXGwz`9kV?dG*z1q?;E?5&B<8 zA7T7Dorwh|1CpAXiL_yUz&K-;IB2Z-955PsV$Z7(!%S1GUXwRF27Qn-8gPwlBe(ir zE0lib3aU?u@7bR@(@UJdb_wBI{w0KUS#YpSkSUPi$U`ssIVD$2JDs$BzvckD zeSgudX_)hh;o|D*6!h@Z7{XshoN+Lc>u2Mg5*#2ZsxXshNUn07gY-_|U_sqiLQI;D zH_$;Mg}(Ge%Am9DFW_P8uDOnV7I?^b*u}Be*yO&6qU~u}o0BXY?<~i)f~{Kd_Wsu} zU%<`Np#7Acg_{Nv3^oTzv7j1L|u!>b(_rPEzT;u>f%{mh0<$Y(Hs!ZqTR(`zuNAKj}Wupbip$r zlr7CRjyB57lRYFd)&oroNEgUkgGhm<`9eD(d-_yOaPWC1D2VLx>y`|PqsHf(oGYI@oj;vQizQJ|HLznrlF5%|h5NmJx9)OYtPTthDe&A0nz|$8 z2}1iIuj3p*@bZ?=l{iVSbHUEvt(0#g#Ly$%xjqInJy(`C)c#Z2$v73P;!WDwve zvbo>J_D9uIne-t78*!a*G8j~{?2SLAAA_xGUkB^7Pt#=W-33oQ!5v|QpGAaTwz5%0 zlTPu|JIgl9GG&=-h~yr?LFhHje+njjRXG=r_dVl3qT0~qd7`+GVJs;HDtDvdW>iHS zzN{|yR4%5H_zRV0Axh4suUT#t@Zcw;%gU%vpj5jTl|A^poo!}zn-}`>rtnzfal*%5 zzcdk2q+EbCR6pnK*{iR`Xa(X+biGC9jmR&Fa4j8kvVCqw!E1KJ@3BWG;F~@e$@{Sm zMueUQeVWV>4EMWiS#!wqT{zF3J9M7RY&N&-E0)g|{Sc;5*m7Og2Nl^Y*ABs9i}5=V zV-9w)oYOuF4X!p`wqdZ(hjw0etaSi$#U?WZ8O{W(_9Jm`me~VFkXPnfD{?HM?(8mV z1>km(c{c(M7=hxFU}p|*W1KYmo!DPlk~rrK*Vk#^%OB-!~rf!FX?V z*24Iv=4-*NyIL^ndGpMc{5^89YGsq%Q)(rHF6xApqv zSYGERs~Q2yF1A*r_1TQFvrDO|`L27)z&a^xJr@~0&*7^3ojnqHPuBN#zsc2H33hZ6 zIzFW#qv|e>vQz~#;xYs?j+S0c5~33b9Jq%UiM}DTy&1L7)2ryOUE1K^zvn$JEwrapVk5ogYIb@uQbVcrNT|yEJBcdcF#MrSkjC;0S5CkJa^<3R!-6-ucuC#hN{J zUWDbj^_lE$mjg2Ve3u6P1Ckgo6q0cQO%c6 z)*bllXVwQ)(^Ngs=*3Q{ydJmn#SmBs+`7bdlqxJk!8sUqWs?9BKR$QoSqg3MA8psk z1%&Ynzo4qziUyVyLAT3~LNWh}AC3f!n5+&Yac*OYL(f|otrgzC!vdLEQSVVuzeSx~ z3>_z6V`cGjYY?0<)WjAkQGT`G8#(Y$#q`USG9i&4QSBp0ZF;tCNV zY33|fxjMW&O_b#p)cO;7+rO9hNP59dWVtu#s#2Uw?YzuGY^a(HwAkj_er8c3$6C|t zf3EeDD4ds^x#knwr?ttc)$_eu#A47sgH(BDfcd+!C~1)2yFpC}Ji@iHZ;N3*mq+dXyKQOu zZumgyO7X>II)u8`{NS(lnZcT4?MJ!)XxqoFK8bHd81^<`XPKP1hS zWZ5H?GPS^IK;Y*sNzy<3_FP(o@+5(|F_J&e@Y>n@R4vn1-8@(8~FEgfmXs0mBVWS)aqB zt%#NMei|O{mCuQEmT$=iz)Nna8@ARo*xO*r@P(nf<@A6Sk_AE<=wVtNDjo6+V$SI- zDa{G8k{HakQ1^ba2Z?bjv+q#2$HolnMIit;D@tx@D&h%}#!^e(nwBR<)wqJB{w>Xn zpbj+o*KvIL%bon@(Xix&Z~X8K(yY+@a()8o33zHcl1dH*f@d6ap<+m?P;BieNQTIh zUP$~51FvKzf25BlwzuQCF;dy4^`P)?l4Y*t9!osTOHMW)8?C1Hx&0AD=^4$9^%Pw2 znWd-b$u|Rt%72P31jYbFsuP)6-WV5$Fpa5uU{*uE%Zb`7F(Cynrnw4#JCq4tr(>?F z)-ziVA`C$&jZ*#dGgYetU&{ppzXMvKkNYil+{0u-g)Z7OUBhcj{xAxn4LlnyWo#<5 z$Z}eux z58?WsyuOh>_7zG;&&`RI^nVNl-Hp=rW*8g$N%iO-YWnU|W}c2QvcT5C2;BYTWgM;i zP15@1Z~cB|WTK`R@7(C3vZo^`E)L~sd`ShbM#epwmV_QR9VR_9uo0Tjz#x@gIwX2S#$hwFEIEkS9Y2~7Pz)`@tGO_HgVq3_+cUX?B)Npqk1&2d+Md1PI-<=;~)Q7X0Jo z+cJ0fxUl2vJs-a!x~%%J(w#Ru_dKSNhX%Gfa_#0DO>g^VP~FZf>_MvaU8P34^<7P>N;7EjMs%T8vntJ* zg1THKHOF!bT&}2hK9KM9Fzc4(VJE6n<|Zh@0~;F}D7VPyAwr?i>HBp8K%O8UsF;G zatl(IhUM(OtFI25xA3}5IT*T~XG;TGS#uXRHISPApnAF#zhb73!QAmE<&wm(k#}JZ zY`hG-6R^8aKZyC(Y1COy*J*57KYdj&Wd4x6@M`ul6R>oOt)x%O2nq?6T!JqM7F^yi zIN@1zEKl|4CIa`5`Aw?Z0J&A?x*}DDd)<1nS4Sj#_0F2_yzW_J{6lqsd<;kkeuQ?1 z-2de38;WkvD05D%Y1q_sx1@4GPf(5Ls(W|L5NFL=jalU3K|q{eqeTx{fXY!4ND)cpGww5@n1YfRroaYvw-aW36q+vtxSBeC@|*2tr>~(2$6x z1H^DQ!%{+7Hly45$EL%U=UrTbPva`(&J^O}sV;M#^-0x%f4P302CG8&3X^jy1Sh)` ztUi)V8zINbgH5wrMRV`B@iHANoBw-Pi~((gWU%*)ES7oeV^=fV_hEA1?FvV+YDRB& zBP58@y%u8b!6gszrMp*vIZn6Tt8_xoYkx}8v8FBCTLQeX2);S*An>~-liim&J;NrA zxiVr?SXggbr6QYsX^9d`bOK_gc#gFslglo&4`_(MPf&QnX5$P;pwQ83U{72iHe&q* zG^f0Fei&bj2qqj9#D7&Nl&daG^^vjMYNR>MX>^?Y_1_%T*%yRXSO3JfCrZfKX>!Fc zww~iLKc7YgYBZR*!Kp}Q933^uWWOi9{+nwK!OaT*7xxYidrF@Zc|J$ca?7>;2yOTL zGP|DiHQ3c{8zlFX{;_VDr91lT7fUVy9FZebTNgM0?3EQd8?zwTO%(~3%SojffizLy zYWFj%tq>a;xN(8YLEwSD9JK1!S=Iwzw?}QF)FuZ-mp1@8i4&<$TF<8#9bSCb5_5;L z=&BhNn7A8ZzEd`ogl`ej6hqCeD9-q$?H7AkNJ%N0QZ<&Vu-%;?!Q7M!^N^PMtRepX z*sw}TL4%n!;j?CiB+^p~k|$T1xIvLgavNuy7#Z3h?9a;U9=Rj_OA-846hYZqL%GQ` zHc*_bAh-5@{W@|`NM}FpQJ@%^BOe)B%++?&7jyk0>0rmepoyQuey+^WDt<_Hvn;OA z4h(*(zV0%By@t3#jJH`RWLW{K;MHOF)|H)j(|A`8Vj?fs2XM^ zv%X9Y=y$%a3#hnV-+UScrKH_C`l*(7{~Z*4G&*ggV7h?R80k;w&x7%x&`v73GW2d^ zUa?&pGBh|MmVc^*5$12soaEon=~mJgLL*^rs81LPXv-Y%D1)@DB#{ZDW*Z9a=$>%O z$~omgR5lS0XjzxN#`(<&#=yD+$rxYD)}gsNY=t+Yj(XbZQA; ziPs@V+OAlMKYxB2)Y`Yq_KmH8017w~;r`V?2|bqo^2fXRKP?s8u9qP5iAAkCjOvGh z03WLVoK!KgD_t=%mylj)RsUcnpr1W2vwj87dWO}zm5S@OL0-GANN33kce0PjJ4oMu z^`iEX;9s}@>)V*9fivjJT|BV_;L-ixJ5(ivFuwRgXpkuIzklIBZ;|2P(G$5`s#f&h z9|Ui=mL$V7w)2oa{hu3YNWlLxv<6u+vHxSQXywS@5<~M2b_L}B`6;w?5Afm5_OK)1 zM)>bm1&ZbKKn0;Mg=rR}{Ev-ByX0swj=RlP3v~apU8~?HDXstC*#GB`?5lsYk-{WV TG1>P|;Fp}VvQ({vQPBSahjw7i literal 73843 zcmeFZcUV)~)-N1TEFdUd0a3byCM6&ss7Mb*APF5L(wj;*6dS!s54{ISLg+<6K1gdgB^4Dc90ewmbv^b$E4DH#O?`Bm!MwA55o)Lcx=^tbuB1@H26^YIExC`${A z%8T*xJ=A_AucD@@r70w>XQKDysj`Nq`Y$3x6ciNHl+>KGw4CZ7K9KtVb~^nGpuBWG z^_<-~B31w~CDA!bqSFQd6M;$Mb40)RJxdoZo+r6PC?nj}y#^p6K6mZ{>4l5uNY7n7 zM?&Zm@wxLP7bvMNQnLs`dueFtg+w)8Iz<-JJu-0!k6;ykoA#xyk4-C+okQlaoF)YR zzW!O!CJ5#jm7XyzQv97$%+&ea41qKQq0N7!vt|hbBR)rR{sQ412PNSSfS8bo2rd32 z5m6GeoD)3%h(v>mn$_fG_}kM_z?E}^N|fg)0S^GhZ~h|l|89=8Y!Z(mRppqS9LSLe zkEI`tz5axrI|Y2EN|j$VOHC-D*)KFlQBkJ>h3`+A%I$JtHX8g}#u)nahN()$V=^zO z@Ln&7BWD=ffPEcGb6VM^yz)tP#u8|2DhHE^jEMU+kRij-0i>A{AP zqiV_>m)Cod%v8Zy4F#0aPgYL>8=Ut!L`x(U)bb^St=9dqB$UP>vCdyzBZ5KYfFYQT z1C8R789cm_k+ni^K%OVxxFB%&o#4`O;wgZh`is}}@7&CsYQQ}<=9H1{?=SDuum8^6 z_@9c|g&dz)m`Tw%2U9f|LV_b)@%M%m2tO%8Se<&hE&r8 z9-r4l+oSQL2U_WZ-QEqABIZ_|RhSLK$s(BpZ>-gaoHC$nv?)hmzTbAfHOI<}dfkTs zTYeU&fLDUkcjhR%Pqv=bX%c$)l+`0`pqFQX<$q(53^p@Y&BuehP`Q1NboNh8HhUlV3&P1BVwjY9216Sein z8<$ySVW<^s&d2!9^XgHO4-Q*DTvqtEnqoR@TquF+Gh7oRv}5fdz65gk668heU{Fb` z;pCDQv1j7D;LM7qU8GHlDT7ufWkg2AhmiNcD_Uk16@$_+l_zx_e6bOWS6(eG>3I}& zbT!#t=Yz<(mvS70SfB>otv?^`QR)|1N%0qt!7hINIIM+nfGY9_u?V$n4zKb%dV=$& zgg)OE9UL66jBbbu*AYj(8#)#>D#zr_w4&{~J`d^sY#AWE5;fbzv}&oiU;>W#0hc2p zYokp?Klyq)YPi!cSuu74E^$~ekzcBsOVJ1*Pj$%Z%1*>6R8uXVy|v;xnu@VB8;8ZCmo0`Of-ogEuSJ$iAPGvtBdB5fj(J%S3+kFUrWx9~$D5&D2Sr4cTl)I7r6u zj+>W$rxk!y+xH}QWmr8fDz$lhe?L;Zs=29v%dd$JV^LompcZe8nzYD5NG z8?U~s=%+7z%WY>{$|*yW7})G|ov$LNg$HryG=Wmy?dxuXtQ0ORtp-srnJMtVSy?mk z`qA1YiSn(Tta?mJ0i&6gFZA6LH`1F*C1F2i%a{W+QXAt^Rpmc0%ymk$_02i(M?ot1 zKdI>Fa?P=PN3TW4Wj^}gAz}&*pxbTGFUezi5OB@s{)jc!0)@I=di}`u-Qi?YK6bR6 zKaw>-kXscyp%bR$e+ko4Fqxthgl zBS+cHaVYa_D^ryDHyI_jJc#2Ba;j56db!n5-i)giLKw0h{(4sP9Gs6<)4Vq`Qj>8m zi+MlPCSgYj zi-k#(KjH#r@v^KU{#4Yo7i2K15-?qjmvT1Hwjh^~OM!}R>5YVutn}fis#XaT4zzXW zfTTgpmKH}|xxEPQr@Y-hHk*PY# zq2F01s8oo}0NH7$BTcoz^yihm=8OjOS%y)?|BP5WaX9<3Gbc=`TOs$7s&Afb1>NkR zm6UXG$~RUwfpOD*oDWsD6sfYB*~SODBR(B9@VcC#b&?_&E4Vz=%Qm#Q#34Y(B$f%Q zr2(&3P$H@n)j2^AzHg-*l+%S#4eRE4gmCUdxmvLC`KD&?@>I3cCteB8Hf5hkYn=jM zGh&i`BcVqdC)4f>)l&C_Y<5d=})MSysPP4y8oiw{jSlEB-qf-nq{>)C5ZXFw_8z| z@b}?Hofr94R0m<5m4k2&o7MZk7zaeeU|r&ZG;>zt_{M`z?=%;%kX;D(3tdW+n)8rY zq<@ejCBw(0Rfu{(QTmuQjuXWgTpW1~E(;C0e8ALLtH!wFV!g_x)h|+sip30AjbzvG z2Fpvca&*he3-r3;$X%&N3ySbCZOjtu{`$Kh^6rL^S`DGuI&B--&B#~jttra=rvRc; zz}r-%djq6na~$!ZAq>Lt4baM+YEQF6g*AH_T8Uy|Hh~+@ipVzE2Kb@+5yv;n)5P9J z-r;{s-i!HQD`(9U9~LodtY;&_{6_D=X2B=(jk`Q_aI9>MX`>9;4f2i8fy4f5V!(qc zn#&!&Y8-Q1bMs?b{yT4J3n?oy`+YR*hgQW(6Lgi+)iN`GWUxcw4;CL+^Xu<1s6?Ft zhJ`(sUe8%2S=kyeKGISej_iJiCTE$p6){xCdevH`)C{elY^;}scJvj*r0(SD2);0B zK=S*7=oPG~85&D6I!7$G_xVE$zY?*e^GZe>GZ$4S`&>A`?I!e{j`z+>+FS%`IGkQr z7(IMl253)%p^Y?d!ty0npO;NnmHQ!flhNi9mBCF>svWA>}x$tCL8dY#OIrsTc-Uh(YL1n zHxswZ2hG2qyqMjpkkeHc8Eq{g0NQbEsaXgW8Z=t0j<2jYdSx1w+#VbS-vn7?vUE1b zzrT?m7#5x@CnpDi)~cHbw5lJ!7n?+$z~b9qTiw|SWWS8CokZ|(YAlbBjXA!k5aH8BjWX8{g8gyojxq2Rq| zbvd_^m6dN1i^mS?!Z58uy&d2tr{(r}9b5&=OwCR$jfkL}Mt(i8+!KAk$ig zx9+bAll$gX$&Igx(IeE$-bXP$KHO&}r8A37{-9ah=NPG`ARjoO4diIWX=!}-VKIBw zsQAR}PA{vZmM#Sq=GfnE>fw7nk^-}7AsS9go9lQRt-@=aI7Pf{X_*(3!s7rf@1a6y z9ylpYYxVxYGYgm8Tkp%Sz3NDYed}rtZi#S%%Q{Bv2YI%Y@7!WmU8(&P@!jwM94?ww z%A@}QvoU*)wG!&GP&t;u`i1FXABU*;yIB6UJA58vN!I8L`Rt~HLCLoRflq@ zF=ZBeRne~7Lew=`UmXO;d8EM1&8C~*GRtkImohiALn}0#U&|giJU2X^MFwv4sf7;8 zN%!;t9q%*CBbY2dLWlK&(9-qh4|B~vV4kcb3ccJ~ArqPR>RiSmCRWwb5$2@;BAS1J zd_Ps>oH*IyF89I?3raN!xSq2nbx@^<=bM~!e~A2q&kah-o5OxxhCi+`=S}?x7Xar4 zP|{u#J{o)BF(vDyXt_}DBqp$JfJWSUhzAfo{3rShT>4~^;- zcBQt_8whXZ0V$Kmaf+A_o!C4DyWYc_RswiwEF2syw1xp3X@x8%JAw+y`YtK*j56IVvKQVCO{#hfmPw4A7i zdECxFEM`uv+Eg=}gN;jil$q0DT8G+UM00d7@W~+C@VWi~3t7FEa(bHGC&R zyPG}ZyCR6f_4h^V#DytV;NKkG*|8g1tkt+3T}*Rl=T6ju9<84+xdt|rn9XRVBtx6t zD5%;uw70z}Dz6jVp&|_z5o49+L4uK_Mw#!hW=qA6-&3Ced3A?vw&&O?=Nu*wQw}Su z#{Tj}L%K~Pt!OvicwO({M(5=YF@thvv_xz5jUtg`LB-{r? zlyofOCk}nF&TuaFN_&twRvJ8CmU-*4YdGl|C@L^hiLO(~tS3*QI;4VfvgDA=XjD)B zi4~@WUoJT`Ff8F1zf<9UziqIsCqqx7O^CJPsi0$|Ag$Ac7xrKX5g&7yJ91z#a!4kr zmnHhjT$i^QlPTg(3vEfRsec+L5?p9_nTM*KGAD4Jjc!{Gk3k;N6mp`{%aBxhvAbY7 zTi$CoeYfH1mOWZRrOLRJl5kG*o0R6mJGS+3iK7`ruXd~Z`7&Gt`zHc!RV3h7Q5fQj zipEOBz{T#mz+vDVj2$GWKZ+h&>Fw0OGQzS z3>8Q2C^)Y-PM*4AS*ZYTcMc=jbaNME(A+)_)By#7f`^`kecuqXMa+v;AwJ40< zJ%{2=OXb7q=6QyEU!byY(}0^~Sa z9B6u`Zo>JM*`B?`++0nmT?j%y8H>+Ng4fn%GB12lR5jT7M+k4L;|iD6?^ia8cc&@KH6)Fuc+Yk} z;;x}E9fMRab8O5h!0e!=UN!1~p*n0ubG2G0fWm}Q%DC5cdXsS^iA`vbwHm}n3O`@M zRTqeNX9|x)(QD`agxLC6$8k_OxEL|rySCo2`a@f9JYIl9SENL>6G!F1%38Ju7H`La zu&-$jRn4E^gn^FaSuG8XJVZm)Ylf7!R(EfYWvSR~xnE${e|Bflu9}5ya%Wiuh&+Ol zCHl@z`j#AkG1TOh_vhO1hIYR3H4-Q1Te^A1w0Z~&>`&XkvTT3z$C&=WVF!ZCl*DNonlG5o7dR0~Bgd zs}82^eI@~{k8&&kiffv@swwWvErwDi(w`SVUEJNX>doyk<+cnH{Oos`a;>ML+Opab z{@tm04|V@M4}FM9MS%gP0u6!^)-VRRqgw7xH@s`~A^IrbKikc>W>jl!`FJmAUSjV4 zxhr7f`?Mt}1yT|S70#*waA=UhqX<*i%o>h1(UMDPuZ&a^&1_X|C zqVaEu$O)0P88AaCC!2H#F^2cdn~hf6#XvNtn`2-{%(oj&E$srxpW8%i>whpeocf<- z@(f}zw-h00!NoNH#oIp*VqM{c8;vY!m8R}1Sjc?VCJ&x&f)`E%B8iL#wqh}%ehQIQ zC%ny5_#^FA7UvtwIK!KgQgrw0uYZIt2fYof$*hubp;Kk!azb<1D(%%EfQY1RofNpu zA*8;|U^X|2+33RsLz7$ucKN4pP>8h=(&do6T{^DlxKvj#vjxw&U~X|+RCvzO=JHJx zr8ZqcwaxaE$vLX7N6As&6tDV`T9es3NVVVtC$U3=$@zX7hJ7~C!29C}b^0;KsYV!E@t z!c*xJUqa_4?OBjvVvY{b2Bv$WQPN|+OR<$EGJY4RO&&cnsfQa3mh#K0!>|7Z*-x+pT*MyxucWiCRMQVFjLPy*&T|6$1v#fP@9b|j zz+cO0$$foUY>PsdTna2$m^Z-<@eJr+U9Fja`vWVRoRHe0s;>S5Ssy?}CD<0YuarLP zuu^UBQUqo*`k>UBm}_+%XIsx1Lu!O~DRms+H#YfU7l>{jVF(t_daUd1&i2KRd>FZy z5#CY@Kgegf4}3s8`x>Hf2W0@1$6NH8$ zdJaY0v0$7)f{~{tvCCuVk$SO%9wFSr z7?L7}PT(p*6AHF=C{N-vU!jUV!2v2^YPDNGxJsqB56O~g2&@b7a+PjQ`p;Kj5kP_5 zoHaSW4>IAlv!CiZzsOYr=QptuF)Krl_*y?PG%!3)cw&A7bfDRCf-ufk*hBcVk33Kc zpf4{Sc8X_Ga(k{uFFN4cl)hx6{Wxj!uwzBTl0|p<`6Unf#vsZUb=IuYjcdl5wDdnv z=)Bj;_s5-AZ5a1K=YwpVBlVtT^U+^%d~L=#;E<1{LbW8@?9we^i%WE97$D;JZYhl~ z7V`J)A@HMA6YNcx0VloeiIiE`E72tRY_q$kA!Q zkwb8Mfe|4S2 z$=n@>1rDd<)@^4eR#weU0q;G6gDha*u(Bst-BTYOd!M&YSU0p9t$q8+q_(v^RWoNt zZ~R-qBkM8-0mD{Ax-$kFPVPxsYLu%_8^CsV(Sv0_{jo8MJggrl6_8P0Qde@<+Tkv{ z<<;l_(kmhgAWDI0%A2@R(2{<*SOPPpM(e>tKMFlf-dDriR6H}FHXH1_`anHrm5FHEs#q>{;rHd>0gd0Q?}&CGO#0ZH^VU1SSIEjKbaW^)@UZ z`(46htfpw;t@3-i&WD-Zg15*y;z*X(JIFZpoClT~r0$S~>;nj_QgA zsd~rDo&qiTk*9Ks7%MufZ4(9Gd$Hm3-ciP;r9|sQ9Ez1(WHX#1ExK#5i zi{vT+d80CIiod0P5Fko>>Eg(uh-)yB)s8#Y`VTai{N-&MtdsQW^ zddCfXVc3|O^ukV*se9lHaWO_)Hf_VHm6HSQ;pd{vbSyUtMeW!=C~zm%f|*d=)y(fKM+I| zaicuxFTUwV~-?F*AZ$&C;=p_p83ruM1 z%ZDc7)Op8$^nQot;OoZ1DW$Y2gjeUZ`93Tr5hkQWSK{$ILdr6%&~GxPmqK6_ANuma zCFz`{hoqwj{7{b5Pd%R7LP|ZU!n=X7$+M7}?OGXU>p{Y%f2I$-SFNJQq&YDdyFDWX z)sQ48;x)C7UQ1kf^Rq72b+A1L8vCHbG zfb%omv+HV5G(*vBd@y=P6{=`=1ogTQ(Pw&Kt=b)z0gpot7HF^B7&)FjJOmdC;W#IJ z?oRlw(O)ms3$Er&qC?m)TT>Z*WDL;8`-mx)f-uy@cb`?52-@qqE#_If^ADyF7MLuo zZLD%AK1i|;KbgU%@d&~dL-1HEtmN>@NC+5H(?41<_UMLA`IkuR$rf@$uOR`P61jh& zTJWN?asy8qSKoX7*@YeP)`8WXpX)iW3-Sq10>nhN>>(l4j#;@<32UmK&M?pqLS zMF6P`^g%khWR7AyROAww!;vqHW=+M)Rpfz-gmEn7Ioma)XRh0NPe4(8@%w-#Bdt9+ zVr4*kO+6q8T(Ct`B~yCms$&`mQ;=GvX4k$_JBYLv#zEuXTFy?_2hcukdWR{PVS8!l zRRUSokDM|p*5IRM>xXMo)?kI`n+b(@1)dpu=++S{RGdlU_gonJG2yspm9Ryv4~o(_VM0j zKdaSC9kED1W~#3gY#M7uC6MZ?7XEkRh{hg^}u5 z0_od7>B4WuGV!FAcK?T6ZNXtztLOa}sf~dP7;IiB?qjnDQ-kRjqIKAKrFq_=|Fqc$ z2f~hgg~6_RqZ~NTL!?6bLf3hNZ4ocK)mr>^g{tX2L3ny;-v`B)w*tD964qBm!_qSE z*gbn*fp#z|N9#}Ln_t-L#9u5T*1fr)^^~`9ztAvEJD&o3T+aCP4<;{u ziytc0+s%kaJEWr)vljs#791Q4C^%TP&Q?c|Y5~-mrb0@doZWM~I^ymSja>D2=JEX%DK26YT?7a&c+>aT)yVA4U9&s*iaQk}%1*i0 z1JJA9c*vp36yWIdxZ7ryWL~37Lgj}Z0KQu{o=?W^hgQV3ndSf&E^HEF9Lb4VmKDu? zW>T0-n9-!Oa%tX5{UsTq{lQYSqh5$wCYbQ#k)uW@EDr^aItEyx2jv4)lKmt(fb$AI zUi7@$d_(E!i_m4LSSmo;gPF90y~&*U6{8QUiHf$f@j&VUK+po+2cj!1?kmWm406|G z)3BEo!UJV9Awhg9QGS$_$-}DEk~G5r5L90hoU8gdE(g3|Ohn5N8%?QYlu&vK*ve`$ zI)B5G?>*n;urv@qVc*Z1YuK-cz$H3&q*B!iQ(T1WnsAV}=tX$e)Rz4aietGDr}~yn zz{~y|WeygH81TeYsu22h(Z45-nQW>NJVJ>X;@C3^N>df_2~X3?HtAAo<-NBV4Grz8 zw(oU8=~tU?{u0B#B*7WQwg&%&2BhzTq#o8eJpLGg<Re0z_i4Mq9wZzw~lpcRB*1G;A?gGPO@1{aDn)0f7 zRsu*Ph+AA(>YPwqbM7ZGRF|S;AkZKuQ?s|TQ@PARfSX~NLl(mLeIQPs?gZ4KRy@8=`yGA?dmqM65~@h??%7N z!P*tegXug;&g;9l$>QOEk;+-Y)!`jCr3UGLK>%V^W(+dn%RJyYq z6~*p!!?)=@1-zD7ie7ctz!mLN?55(@>0eE_lGhi(+5ZklP_c8x}NF(2X6r zC?8UfV#~FfqzR8UznMYIx7bi2!s>F%UC#WgVT{jRG^w5^0ny~_I5Lm@#FVohmpi|W zQ$i+2MaMfWH@QRoaf-#S4tpt|p9C82#gkqUZAz=LJ`aqFqLT@rMU&bu0&}owXfO7| zCs8eBFSni}lVXrDvcAFQ=D=$H$~EPHmzilJKPx>lbrt)zatd!ZcrRrsIAGG34@i;D zh!#-Kc=4do{iA+hTpmR}tj1wq!)e07s?pV&g(u8Zd`t>B$TpYO_@+_zJ*8eQINV;r zMnvRLT#`6MSS^5$hk{+#RYo_a*)h5uCt#iBy(+eSk6N?DpQ=A>T9o%frbuVCL8vS^ z+?iv-hq6M4GMmo>@68bX_^q;x%EDImWWKT8$=8n)s+zO2Q@pP*r7~AIC)~f@-p}R_P?l#D-xbRP6GlBGPOF>qY~H7Wl2E(qJ0~Bna)mRj~a}*6{six9O!1z8%kPpn6|7 z_(reo0)hsT^pq}kbTpZM5`n1TIr>qaHm87A)(HJsvjOM%u~{3O!f_Dw_=5Bv*I?=-C9J}gUQLpTY%I!4%F&+oqxni1`>B8=cAMXo{S~5MItQGjqKJe8C8AvS}7cBj>(RJz>0q&x{=yS<(jg_wniv ziqtJIWAoMW4|WqCJ?ztU!v*k(=9GWEyWklZ@Ua`K%AO=DqQCCK#Jh)-VTf0u*W^n5 z5|maPrr~su&V}6ovOgrg^G_1{&;BKrRM-6&eetnQWZ(x6mkxFn#U?STsn3}B^{RyB z!>bhU#V&*{bA|><-n1TKCuZ zWF1PiWo-0I)A@#22`A0+3bI*2;k8lqPD(CQ{Ui(@UmUiLB_afnl}pqW<>2J~ z7HzJcR2(+7oJz4ILhea*%4RmIJ{MKk~CENrcTEpwrdo2I6!-R#NTsG^}V%0FBgAh_t-xJS>(1 zD28`T95exaHev{@PXA&Vr4xeIIr82^&ooicpkKNM$VMg z0Le19VJj;Bohigda|fy)epSBGD}l<(EP%m8Qx%Q)V(;Tvjm2MJr**dXEG8XLx@=)g zPAgq{?CIN-%^ukoM1J!fn>Gz$cIVuh)9TwKLpV}D8uCnYR#x?rUn+>Ozt)j~%F+@O zzeqUFmp|&;Foa(UNrW}M+j$SV8b@0LUKQr(<9RB`R*@GNqN0qipnZ7oFT(xrg8zU7 zzQzdLzr(^P5Ol{-<5Qe;5C^O#Y7=#DBmv|GUb6OW^MkxKLDTX$4{G zX6Q5*-Y|SV6d15V~M^t?NSrj98>1)5B1BE-HTGpq4tMuD%x&H5#UZCZ~vFS!m z6SfT(uSdrgh*xSE`N$gU#{cfZIWVl$NkU}dJX+oS*|frRr_?AtEJ{xO31h0f_@kl( z?R~e}Oo#VoqWPQOUn^3*kf**QssUREoiGb5u`lN21ja`Jq=h42u$#_wc0LdzCWCG` zx0SQHQ!6pgU1+m2o}M`gW1s5lvoj+t7XQLsJhHeC9d=Z=_Z~v&`P;uar>k2+gQ0q) zmpk}O%IoS##n6QF5#Fc-yDbcRy^%ZG79=7rzZ;J?Q8GeCQgp5oZyn)3nb>_(0M&R4zzqZ-lpNu0ONMhmr3D z+sLCvYwH`2xklB-MkXH4w=i_td{}&&h6^VX}IQ)!_+e%;lZxw$2ZU6tAA#LU*yyft9 z>!hs4y-HdP5EBrYt7O=PdVT4zCwybV2&*)Ru>F3$Ginq^=$ ze}z;g)Xmn43G-nSY3`6!v~w&N53EK!OaNHpI9y1QQJX(w1zC$3gs({ESG&6dM&H$( zkx}Hu;i5`bgd&Q1Q#U>C6(g-0y|BpP^WLbf!oKbGjvp_buJKrzSm0=1a?NGUO$&<* zyTRdveGjx$s@7D&D=y614T1ON6o1OBJ{AAj9x>>a7qfOW!UHHrycW3FHtLq6#pXSI}j>Ig+$d0Q_U=Fyo zS?;1~-Jcnu-0|rBm+v!O#@V!^dFm$DW|OfcrZp7fBpzfJ&y@7L>^=P8mh$?VnA*I6 zi6K}&j*&3CeGlRmWRQ{)z8Y|qf*y$I+o}1sSfPyq0$)1Kr`Wi@)oMH#I&8CBeGXnS zoJ%%OS_xQpKn(tP`3zyKyuth9TxyFs^nkf~Iwe4uB~E)0vFGb`TN9NKp697wL_g2N z+SxKBjT<5RL1al&10x*B97(q4e@9}+%e#}F$9_~Y{)y8(;SUqr;VL2Zpu5VxW0=9Q z=`uqN4GREcR9fWRjefdfndX)m2LXhA+sNJ7EX;gHLImi}2u8rY>y`%bN^d!9zUj72FUPmgS)rYH`*(@w=ScJI+X@3sMHeMwk z;+wEp6+6n+EOXjutoodf%T@taPd9}PYiCan@I*###?A}Dfz>#d61kPTlmZa7f(PUS zAxoY-pNcyo#W+CPAg1UWQruqI_M$Yr2@tF3R_^`cz7M9KU=5D?fjhFxJtXl&n3CvW zIs~$Q{%J#Chh@jz^_rcv)pOP`U9130}vwecCQrecS9ydQG4_DH-OFq`U9NRNfE3p`x_SL!F)%Ypnj}*Iq z^r86?)1jS&33P+voZ?kHcz`-LrTp*w64#QvKcnTK4jmOTzGzQe^Is{uUm|*1c^%&E_h^jYAL?Fw6zV~2 z6%iWC<@$Qckn6r~<*JP&WO{4Uq|wVx*Jp3SVb_JK3kGx-)tYzN^OpfKE(Rx!^$;*u}^=Gn7#xoW8Z=HD~s5h7aj}c_v~z3IZ}!{6BcpwZ=vW? zF>+ha+tzjDmXur{XQSYq$qN=+t|}qh{H@tnQiairgMt3VKdzb7c{%rN7Q$8{)@ypi znx)6oVxjK!q3>8|Z488SBc}Fi%Y7-`|POfJ<0FqplN^@ z2_^KNz}9)`vgk4p5mO?A?dh~egbxX@rBJ;8?VSOPD~vMlEwPEja1lk7PVxQN0M8{? zV=JlIve19$m&+TfF``Zf<>4_t7Hy5{=4~EblO$#TVLR*@Zriqzlhys{e|P$U#G;!H z*144+rkXQWn!+ajP*BM2n-YKdf`88a_6S8?JWR!>fM>7sPlo^I2$B53e`164_=Pd6 ze$srmmpux9>i$jNFLLpw8CjPnzV6d2fzNV^v@yD`N|J|bqpf z>01bd9?#RlTYK+70x+fgs!zLG>vam?7EG)Bs$d6P`|ev134I$adX#&r_{AJ4evM{R z_aj+PZc%$@=5FIr=CddstrD#-=HImSND`dj$Pj8}IFLm8FXdi0NYQnYgJkm&&hXqgY#SJPShDI+T1q2pJ)Tl% zKE;=C<{7*LUB*Vam+tjNYAx;y#^<^_s(^bvbG)4Z>?02{TVZyr%8;iEPv?zR??qQ? zu9T!%Y`L42IqYvtlF*P1MUL?mBb&Xx|CYuR_MeY3TYBvnl(kalQ|B3u?nRG6VaGhf z@&uJI{^mNk&7C5$3-|BbFuOc5E8clsfmT~5#qbkmVWe;a6p-GXle>a-X_jBC5tF}(~ z3QDEg^UQ&4%l<_EBt{S;X*nH@i_cJ+>cSmZwq^Fa&d*9)e<%iJWM>2O#TqJdL*f)i z5;i?Al&!4SYPw$=AUbQX&swHPy_aM|V9N^h+uWl+&8`Vl{96;6ijyi@EJulZSZolh z{xP=+RHEB&0yx3p)D@MoC8Ld3aq&kAHIJ8nK$=+8?oNk?l|C@Yzbg zORYaE%~MMXFaC>N32XWR;faSLyZ=sd1d;w7RFVD}tuyfyo@GgQF?)Utk6r(a25+1e zf7SlCiAZNI<&g3C^=|vUGZ^p`_8Y7z8&nGYAxTSWt$+1liUg8>MKKP(vv_v<3{z?5 z^Ia+WVI@`gnp^uX@bowJ07zMf27q_%%+#2G-u>4{{Mht{yJTf|Ip)(!1M}EV9218Q z_nx222;ut^laI^vUq%v^%*hEV`v*v;@E|ZzxFyHMBYKcc8uR`ZsZ4#MZ)jB#W|QZQ zlOJ_onJMzkM18=Lz7Y7c{gn8pMIQQ|18rBUq=s!=9LJQ{OC$x;y}M%+)P%RG`EB%& ztXzeFV^=|U3iF_y{Jp3pq$7R7yk#xnnCPpAo8*lVzAa^WND3Z?T?rGyK~MoX@Uf*a zle+RqgwZ)yB8}NIUl&50!p27}&c`7@vS%2`#93;;CM)z10QfzBA(haCr?3fAiwkoJ z+#pn8=d7?_gGhUvk?NcpAKia3HQJVVPsQdRlEGI{O8Hy8`D^e>J@((%V5FiRZ!Udq z6JHJ#D15k7;ONH$bxGU^UHcXatWtArcG;A;f9BOt?#;A@*^=L?UTwxDCM49?Wj#}< z@9dJ*HXL!g^*6+4+K%T!rOnS}bKWjG+^i3rM;-g8I;D!&|4H)upJd!_GE=;?gJT08 zBrbqXj>o!!eX^(pPL_$Xf5+>;&87gz**c^1owRm}#OqYi%D*-i-IW_#0qw4LBXO7;9d z`#k03CNe0)t~jU`N5wO~S80e@5PuFewodSK)GoN%;~1sfsKHvZ95r8hQa#pv|0Q(P zu|~4|ZSU+^1kc%@!Ims|xIUNGSf=j3`@>PVm@>|DA1>z@ankri@D1SKd${c1Xy*)@ zUifPSKaTrrV4qXBtz>}`J|7{zxhQ$&`ri;t81TQ2eqt{DS+qu*-xM#Z{iF7;C;lbT zhUyZw(rSlJ0k*|~f2}N#JpD}@)&AQ-zXjcKH&M-zD~-MvmT7L(tRk+_={+Z>HWoBR z)H$xDHPY{OuAG7!y}DFOG!_~TaCD`??>c>bvsNEy?Jn{Bv&*;4hcvt3SuV3RxNK}x ztalg1cEi3keg`o?<_+NCJsbBizGtES^O-ZT0DQgKONHXER{h@H6lIrsS9Wyd>kR{lsN6@jvoU>=PCQcKFAgUJ!d1 z6;T(Jr6(|(Yr?NwcO-Y5k@dydXdYW*kgW8t8MQ=%vqLB5$$zgCNWjh?MK!35f13TI zJ>f+stH3wmpY`Z3xB~!O|GqGbN;jY9oST+j*{op&*{?#SJ?vCQg_0#*qnZDll(Rgv zxAT_&hf0!%3i~M_zg|yU6Zw4nOCIer@$t*qs{8!o_p-8!vyLv>OGy6;5RZG9CK^Ff zNn%ryFJ|JCyAn2Ueq6eCQS7Cvx`paK#khw#2Pf*qW}IcuOGn3G*w^n8250^t6*@0_ z`SM8YtJYUa{<$@)D!zH(LyapAXLG+)h{vCXaJ`VFhDPCWV6)(wesmr@agwIh4EW-vj1x%Z>*14 zq(tSW*Z+F$avwZIHj2~FQ`d|H0L}pE-%u`L&XBDtbCBhpb<_zjfU&52V{PXo z)>ss+_#$?px^F}F*CugQPTI#a-TnVC@V8NR?FjiWgwKqQso{E9N}bV1qXFv)2SijX z;NqWNCxoTy&zS$8nCEx5&SBCuR!7?)7MfRV4^#X^=X!FIlva5FBpXY|F{1%fwJ)|$ z+U;aFjX$OX=xeig^{u(y9oP3C@|6-k(>CJ|_#eETc|6o>|NjS35``#Bo9t`$tt27) zGK_r**+#@zDn%U0UfFlaZtQ!=PGzm^iHa;)lhEGzT{AwP;dFHFbANxo-`5}ap-`++S`@KK^`UvF4frU4_LQ>RN`Y6Bl3PM;m0 zxGMjMHgS~Xa#cPeR3i9er`Y`RSOIOW(!PJ*{BXO0`T%s(2YsEo(ZK;d81#7<;nI6Ea9w{Z<%OGE9?(3H9f6+qQN-XthNC}I@Wlmh2I-`WCcUDDtH^JI zpIA5wc89_@6e7QOq`5B~-U_eLgwTgr3_3cVTiKt~6xT5Ow#A#@@sn$$xuI<|e03o| z4S)D_G%7q_#kt1q`YN+ZmjOy7tImE=@*%w4fwR*YX-D3L&`CpO>>olR2YAdz`v#}J z?R%XB#fFGSRc}}iIK34CeH_V>md1_NiF-*n3~v9(y1IjW$?(_aJ<^V&&7$CZVtbB)yZi9e&zDII!bJIBjQh?q~uzff8c4!35HPYeTSq;l0b zc0qdQqejM634GVPBy3lyS9LaD1%<|_600D&6~bsGhR&lMjKfj- z3;cojw<3J}-E$gRbD<+BY?>a21e=umamD7smPCx5y63Z*QaH#&+Uvk0`qh;37$4I9 zsBKZxD)anO%@3tIyNey2(4^RL4_ec2T~)A82H7M`rDQy**lc*Y6LieV*OOimjuGU! z@LN0eHgw$}%5(}~`KH=?sJ?WTGN(eMj`}p#dm{MXZVvx;uW@udIZ8gStd$t;joD+@ z<9Y~_IlG>aikJl{slWLlK7NTeX&~nfGri@W!<0#FNevpj;-;7zQLzYanO|6ni_`zg z??mYN{SWs%8*`--$OgM$yat^aAK>F=eNZsbXQsu= zil>-&%{8E_DTppU8&6c6<`+bcQnZhP7l zU0WKdm&_WW2S#G=9eT_UiNXQQwOs))ZWu-AmC_9}NXJfaM;(mR9_TzGDd54Be4kdm z)7y^&Zpguo$Etm`!riT$l>VOHhmh5hr^#Ol(Q9Pqp>F zN{8B+J6cW_;Vc%-10ErRQ}u7PVNeii@G7>qw)13MQx(l5rNm zovOCr)XmQb7QLkobtAV3Wy8P|S7c5acCJp;qS_Z&m?H-~LSvg=hByIp9*MT>~F3_$sZ9Vg^MYno8d19~_$rO1ymBK664V z)0+M0;~eo5Pr^@(qiYrBF|S>82q9(~Z)k}!mO)Sj#K=ZVu{8Yc;HLi#uV8^8O_28H{ze8E_LLh1LNeyV4}nqA|{ zf@@BU4_tnzsJNv84Wk%74RB&yKNt+kSkjy_8dJX7!*Rx-KA^X@ zR(6aVa91VW_Z@ZWMXpyWb>zkhUUpiHwk=2|2n;s7(&C+?M`G!uMw4zDlIsas@>t|0 z6xLCD_jXCHJX)=feq}iAb`XJ}=DCcc0m{618gLj-1JJeQfh!OZiGKl~+=-iiB_mUw zh>EEN-sGHEYfm)h%Dri8WdIlXy}|IhI`vXSl~E5niOy|I2&JRi|r0`>E;WyLWt zJ&Jy}Z6B3S+G$~eNbX>%(^M1{3>Y5l3VxZG^+W1X?*$Ch5Qb{P)wu^XONN|}jM$el zQ)Qow@i9cPakOWmQJBibtedq-i!<+`8FbamhnF9sOCR7Fq@PMb z0Yk1kBelk&_PvJe)~QWzJ{M09xJ}fW{s6IPwixu7#)}biL~LYiXxBt*>zXmxCTn-a zZQGpUSOCKmAQ;9o!_UD}Gx%|t2zV-Z3FY#gCJ(zdH32f!N20Y7;YkEK=1ocElE-s> zi5;y;08A%+S3}jW))tkgL{+LtolhPEhLvu<;AK{5)5RLmiCZs%6FuDTT0HV5BmfE-R7Ww7@bEJ93dNpSj3-^bpS9eybTsTX-_GVu8 zNX!NyKq_lbi))GfRZJlp>?}S9A0B);ys;HGz7zWAP%&ZSG6Hei?I~(+CmcN5aStAd z|1e)3@H*Bem!o=!1UUrAcfw1&8oPs(JAjk zC}Mo|lG6?A4`_rXgr!)ySeygyZqG8UAlBn@erf}AMCH)C-$qqAh{dFTu~Z&6IN<%1 zisyr?Q$s>R`)T-j%#WLVt-2u4WrR@p0zEEs$_vM&dg)IQKBR(%749{ZX6SI$L?t1l z3;q6sS1W5NbdYAbo^A2)B%rbIRGet{?+BC5V|Z(CRs)w^571$L+IGVVgxoWxaEKlg$ zo1@nmQi7X)^d zbGtFyVfX@Eod(SKP0j1QSHc1V!}FRY)gF_Hc$6&GAy-lJ zaubv5ofoG*yndL=yOCdDd&bw?2rC~Jmf=#D?AK)L8s?v*&+{{+sE*%#c217re|x^| zo+9h#Hhlzeri(uee!Q{ZXI{q@$`caFnuA{FJI}0Y}`S;H0_|)~#>zczXpwl&GAi@jM&cEyd7cbWXgSixfq2 zi9K2c8qYOJ69bEhdrCSg7j1g`Z}2Lg;@fI5V z6)i*-k62;VnY?Z!m#aOQiUv!w-(E3b6^B+G5HK%qhDmP|e5U=U#2B1M{K@Z8#)ed@X=}c07{b~4kKxK(_rc5*Ogz+&rS<7MIG4Hey*(v36@ z72UlL%P0G@_$h=L$@o2-SSo+Mywd=|k?E5UGr9)(DFqnC4tqET&(}K1z;S*e=gE4@ zlig7Uk%DQ8H+Fei1_BRBjAwt`?_!TcK{T-%VbsrA`lif2@h!QgR-p7pstzAImw8-o z?qW*OsJKg#w{%>ck82VRxk%DvN(2Ts4Aow(C`GA+8tbbD(BBuqBeWXfXgYzl%t;U2 zNmS?3W=h*Fo$2G9hN3}yjI0NiZ?SVa=@pE4*{MFcrF&E{uGE3bY>!gW_#P8pbhT68 zoGjN*3GAZe>q9ReL#>@jf!M{J(AJFp!o#$Dj8yziF2OH<9=+hwrf=W#dFhqW3$a*V zoiYkRq#u>j>0Q@cBPo?`S5xz=4z4}O>hxplTR{$LTXd;+Hs2uk9KSA)i@Qe4#<`s0 zu~EK4Ig0csynl)d%`LPN8M*K1+1t~l%o2qTM#)NnRV)WCL6~6bxC5&#dx5%&r5^I6 z9gypE1yM0p7iyI2K1)TvG{8=1&PaCOXz=)$e$4)6=bwn007?YB*P*8URhmWnrQZv5CH2O+kQ2J}vO` z?l6ZK(u858{gwWCWF*}#LALYLheszp@|Zlw($=1lcIY#hCM85U3jERa$T!GxqPtb^ zb@z~q^joye^QaFm9~PD7eSbG%OMip6J74Ykk@dJsE$*K$)gpb(K0u=@0inE;zXtzF zhM+kzwRi!PhPW^r&QkaETINWhYs?yIJ@+j9@0-)hFpV2ZU69CkJ^aEWt)d;1Jxn&) zJmHC)&r<$OIcgIJ2#&ftgXW>y#(txO>!F!(xoRrys!eyOOnWXUIVTK>It>N~d2%Vk zk7Qbj^;SW5_NS_9s6Gm#G3~w};X>Vm8Ox=nUqjx7=?%G!`t+s6qOe=NIn&h(|rI&o@;znrJhK!&7!dm!(d z1(OpK)wy2ta|*}4Lwq#idcIqoblJ&KIiF87tV+)h;n-^RCD)`XY*B?KAIM&cq)OBZ zp)5J3)Kzy|_4x)zyyKYB|AFJ9+pC7rTIWMVyYHwNsMRNObZgErJsA9_-l_;dWo3w^ z^;=VG(|(rq|xj8=lNWoy~*I*cN! zP^eZ?lyk(KG(L=krmZb8u%ICkl#Y1R-<{6&w(6>hwTF|QeR(~lFqs%#NQl^~`{>wZ zB9gbkg?-7u3KZT=&d2X#<*>1sVVh!SDg9O=#8@q`C{MBoBll~DyTXry*`!qD`fAoA zFKMa{<%QMy4gt0U?veK*qf7f=Tm_LF!TXI4=2Wcm4heNC!Bwd=9Gf`^8e#56@%}fG ztVO+l)b_XptrgXN>)3f0LGwLMwK2XTh{|Sx5O+t|nx3`GF8p5e;kYzOlPZ&x3icgL>?Lpd@@pD%6a#peZk~Ga zU`bJGK2p6iYr+`*{{8CE37IL;f;t=|gs^f{EV#8qGY{0!NvoguryMl|V$0dNh0{Z? z%Ah<*TqDBZnJc9Y?^_)e;QUg)yA@9S(Edt<_AvU~^+O>*zCuKWju9EDo2*1zdbfAK zone(UNu-fkU{1q%TJg4NWL)qFYF}Jh|P&6erpYdV) zr-H+-PU_@dU!Ua>0$pvXI@Ll&z3-fI3UGn@h{JnRXUsi#)>zi=pM!>&kru2yreO|I$XJmM3Z+pR(r9)kp8eEl()~8dd1cq-(y`1OI$%z)J zyD@&+dH)4m^fBudbUd`FX)?N8XEI`mmAs@mQg$`|ADVUkfEK70vH<`Y%JWe2`5MkD zZgYxZ{-yz9lg|>?Qkfz)BdDYuj>Ne*28hYSGeEIQ90SzJmCYt%UnTBesQ3{7H6WEf z+qx+9Y5$H3|LvH`DtbY@5mbr7GfAR&Lc&E&o*p*HB)z*g*ERRBhZ7C%WH|j6Gp9O0 zO}og`Aj_xGY0hvjXV(*PlrfKSdi!!sXhf4=5L1Z6KyOV2S0*u>L_qJWG1&n?TTNPp za)xh;(iI$O8!qY(8NB%sLMU%|M&Ug(NKtF9R5U~>((T=scapct=;_os^SsN~!<~1| z16u5T1f+ldLvE=Rr*7^@TjHCjcara3OiNR-;wDIAnyNT-HB~%?sF3EnZsVP%ZTbA7 zRChtnPr1lA;R8QTUb)oPUB85Jm?wU^!}cH1j&A~jP~#^kv<2f z`fn9jcTUD;IIAM_8jP9gy*0UX`mc$VMjvE|d-3Ts6|3p)p#J(sHkj<+{sRsPk=?~b z2^h+;x{H>sf9Ryz;$UfO(0x&Gy@NlFhy&7j(EF0KN8iIf8`Cz*z&1W*J$XdEuzo=w zxF5=0A%&xzf!d-3kb*!^CF7$I+y4&;2O9ENe-_vs6UKD&ZA)UUBD45wa6FA8NAN7^ zaT!Zpo)`mp|9I8U6?$?gBrwqgt9j5i7e&@`;)2Yg6Jz~|Q>)j{28GaSd#Pefuh8`o zGA8_0f6qvKIS3hLh83e3eONci_FYI}wd<`MXC0DnO`_&|^<;r7&C$|IAj!$^z)NzL z@sgacK$3F|4*2i52PddT>YTBBr@gR9d4b*PZ1DH_G|K=lphNMx^_`GGsRtNRInN8R z9J~f4JJX;&5W8M6TqJK+M@GCwL_}JK&l-i3-k4(8=|%JAR6`jTLgw5gU+mI!I3h^? zY-9$lU{);I3TGQIJcLM0*ji|D*; z`bK_){wVZykKvZ{?)&@iSU2-;%^?8~cs^x&iuYC!xRISPxuaLC!@AQJW3bo6bF! ztcljiX_K$hmTLS?>N6TOVA7A#y|9}3HTgjsc5)2dIave+ z{EE<1fTdu>0F%S>#Esf;sMJv#tIxqPuBlRjpCKh?fpwmUJm2#$U4v<%Pm!Ptj&wXhQ5RVDeLLmO>;I~DbaD1?^z0>G6d9#hxnK#4B)6YdFS_=O}xb5CU{~AI? zE+mz12gN)pBWOOPDcabV2%e((Hu$r!QDi3+OV#j_+Pwjv$Y!^^3{AVtTH@1wIQ*C* zX#~@q(5ptIJFWRly5kI^V`BW6Zd$Y@p`PFRq3;oKcvH7ychTBYG%bzyd+R~xd&zDm zc~Lc!0A;-X9Rlbxhr{y!Ug}{D3ldLUHOO6fp&9?OtHX08rG!c6r$XBhkAzH;!P@JP zFg2C;|II%61s;#(ugxi%ePVxow^fs-E+o1D?pvQ7Iyy( z9yc(u;>irqQQ^o84B3c~#4lkW#|(W}qs=5FnnRsgGdrM*Q;hp)RxLO!)s(M0+NrWz%=q|A_#SD+KB568UbO1r$Dy z_u`V`{`fwBf83I7h;7U(Rkq0yZVr|5Y@*7gDW^BwvfFl@^=0<*uhuje@BQEVUJH;v zr!G_+#on_p^o5g`_#;zz$2>%nvCdzvoS0}3%LjEgs;^0t+rlK5ewam!X0Y4nvPMQ% z`N5Ni7y`udvgy9QUtnt_BB}J4^XM7ul{3>Y;}KR53Yx2WRkh~9NCi)?77tWzuz1zdAGyS2AQUFxAzLuO4dtILuW^m0(eml{7qqT5>qp8LD(n$9xET_meuE zic;U=Qxt^+K_Iq8`bVk^Pi{$QqBd4GAz{{Z8*Uaa6Py=)b0RMa>#q?je2iWWg?qBm z>AWI&^Kw71pTcRpA+o6pnE-H62yFl^2^`>h4FK0!_{+%X;p|$4?BjBNQA*5ST46Lu zA@0WRMtQx_G4a_x@5tkDd$wH>&0yXC_J<3Oj+l}~6kT{1o!fIjJIbJ>I9HnhX2@k6YoeNR*dIk^NfBEN}@LOb7au8ITgyS@n_W;0cRljeNnJU+) z?ddD~wfK2df!ru8DM!8IN#5}@J~JgRh6*}@`ms8x;N&COo_Q~GC|OG-qvlap1~~C` z*L>HXGqVm5$?d=m%vhH(8yVWcn;c~J|Ft&DkXoq3<0z9OaH zt4&RFJY19|x1d}K!F6_I#qiOPelM(Gb7Th!Hk>Gm9XR6a z;kxL6nZK4eVbAC7(>DU36{k-F!~WV2fL5SSiyiiHErI1wec~`2VL;FIRSFeGB}y{= zq0B4R)}5yfs*_x*yW~DUC(Bo?ef_>sa#{+7I2F_bK6OM2EB!z)0XZSz`6buK+IfvSun$c~Ds|wFDVw^4F@+-+)^6 zMqvd~Nv7(Fwb3-QroFf^=^z<$`o^N5MeSaxZ`GYSq3{=z*}%(J(cP@zlu^>S`@t8( z!`gAh>2B-=0>T}=;Yq?XwEv@cLwnW6q5@+9aLLorZ) zC||@B)Ebso9O5cQLUaNWH}5s1hOKO@-;%Y(SI{gVCSuN6<|&8imm`rq4B_x=6zHdA#ikX%9{ zhBD+DehGd-<^eRdD&O>ryTSVT{ST4xlqUwQ6D+~oWq0u7aG48T)z}~NOTpz4QJRF3pY_*W10CBbry=5jD^QiZ}^%-`?@Ix42*H$p^!aL!q z*A=nM^k5Iuh6t<%Y`J@%IFC00^R>7^L14Rz7g>4N;>Z;~BaaXhL$lzHfMV+#I`gf1 zNkZlaDL4Fnv@C^1)F{ zoL>?QE|}rgJ%4k5AsR52f!=XEWQntsEDW6j(lh(3*9kIYF611fcjwaDUJnKw$adDy3fy2 zK^O!U_%*s-k&^89)T8@(F>>JBttH7G_Q0ar0@;;+uh`q!A^&i{_F$!jJ0&1&~kteZA>z%OFNZd zR44(3;Ykvo;xbc5jkc5aft3Ge8=}v%gcI7nUveXe1ln&tBu_NBJAJ+-w1;Y~g*s=l z;ask4AU*ujEdC6AmrQRjCL-xp&3SmH4oH6kYrGDaoQwORvnA->p{M8Zf$;fK9uhHP+}F zD0LQ6SdPPDKDxt%oABN18LEw0hThcbepE;qudp6j%d_sCUbCL|53ulDCrNKIz7ULY zavad06J?ay?;jvZZF*T6H5GxNBh%%k?Hw#QhSU7n?zF^<{@$A-x%R?~hw9GeKDGIQ zjlbRRcE9FXi4-=`ZputoY)NTLw5*w5c6d;VAPA{koF{8`*gJ3*PXoW$pEPUD>v?+f zk^LFqBk?hH=I#UY=I^=s_zC^v`v$S2h3&(ePNUW!K}KeF=gV0f)r??6%XR)xA#B%i zW$~4(Gx4*IqL1nJNrvCc#K1;&W5fa=mht*U6@6G|MNb7CeaaC5zk7t8Z$UzZ_r2TG z0_UOUsDhO534wT>s(KydjY8!I2QFE=V0pK%%C^TVoiEyVljAGwY0Qw8JQC8hw8!!y zz6}iqnjom0U@qAfM7f+M#jGcJ2R&MVG0CY5P=^&PBuWSpcZl6f?V0I>iO6|*q5H%l zZIK3;WlMz{yDZ18P@Nw4ko(_5%;>R_56{JL6?K*R+-B?aGp%=vXATxLBo7)MCDNQy zl=H9|$WX><)7~to;lye0&5e#x!H)w(ZbI_6K5HMM;$ePV;iOm1uWQ!QPLM*8!O1T6 z>abS~l^+X(mM^1ad%olqL6(M_g3cw_sanlk@+%~XurP}^blb-P10O<@sp(JsY~raZ zUIZ81uWQ!Xl@6=m%nI{#P859%%_@3bb@gcp7m6)#O6B@U8um{Fa`6!@A%L^x)VwnUSE;R<0do|<8sz&&#*izLW+ zP~dulfRhT_Qc;B1`F1HQ*-zoblyt&-lKfD9cv3=zC|8i7(udU6+cQF}66on_spmR! z30b}9E0JGnE}1VKdQit}estGaZOm` zX8*Cpi^Ya%RB#Du`O)Z6SuFOPLzh$)j0l*LcEI^6vXStT@XGuZM|YEM#;Ul!=RDEq z8YvGF08T+khk}Cc4zzpC&&%^8Hr0)mt-sEItR8|S1;~>ihmmPqEO$duidaPuAB)~a zoyD8b66;69^ZD!n3dTb4YPy?!WmB97HX|e&wWfhf8)5YikZ5!KzEs+d*nNXrh_+0u zfJL=KO#$phmKQ`Z=AK`l?}9Nin9!27PsGVyx{2kUJH$ z3rM*ilpte8!as^zeWh|7dhJoJktt?v4xecq=k zqpz2Kg0RUa!u?y8ZnPWaVwuTre|*~M@hbDnqWVR;xz1yR2xK`nmJ-7>% zuLY5b*h=2+DG|`DPz@xKuWTH-i4mYHYEwP$J|tPJ4(DvVI>Hb?=5Rmyf5*_IohK%Jq zz1tH6n@1vz)99Uy}{EkGG0O7Z|9>D3m zTNSMTLL%ncmT~`FS%nOdFe9`6bT3{!_j+g;ty~BP3SW; zG%C)Wtruw-4@~vaHca)RslNip0pC7H6ZLZ(1p=ov@ma`+efB}70W>y**Hxx0SeN7# zJnUPlMop?U6AK=mN zuR0{QIi!cdryx`>vgJjWQ6(GZUU~jNPMFGR@!+K)&u_{%AoR47jxbL81xJD3UF7Lo zo{Mg8|E%W9AV_C6SZlZk11N9}4wroJsoJOYItKX$EF&J-J<_%rmASnGt@@x7< z7%IhDC|>w9-R%vORx?1G)BabbWhjiYM8tZ=o{TUaLBs&>i|_O1!R$NO#7>wR>Sta_0v+l|9Nib5nPcVrwaT zo&{x1HJq^~STz!B!tF>z0zX79Odrk<(IRlYV&wUO{OP{beEbP5)hU@ks-&z5082}V z3n`G>Kb3Lp6i{%;-x`uea<4@?L8`n~OM<{_zk3D_Lj!|E9EJv_oG0~P)#Wa*SPwuR zNXS&+${Ce_bLA}CcOLJ`ISX7lBZrk>yG@38ID_Ng%IR=R;x|9%3QQwoDwt_+c0KZI zDjDxivu_J9{W*IKq5lv}* z8+58=>=}@_{_voSd7_?$*C&&@Vp^dy zd6484ynSKbhJ7K|#VEHXX;7n{c$WAydLg4CTlqqKeArIEiIzn9zF?5`LH!EiaTCp( zlYu~Ti+wgVkz0vhvComP4ak=Oo!FcRW?DG!ELJh(K-UZZctd+TVE@Z(*k*e0(!gc= z!w3Tt$kcLD<2SRINIYYT{l02ur;^;=7_eccQ}^hb<=nI1g)mUvifyAm#?nTQIb8jW z=*-lViPw$6chU!%<3@`rIqF}aW!SMwWUVxmK1F2}h~8(>ai-d8=BQa=U44PnDVxXe z3Qg^QeB2HG{WdN&tO$P$m2iLvxNeNl6L zv_H6B#E#E(EZNhnE->>Xe4^L`<3bo3Or{d;D6eGcdGey;=4gI!smMX9YllE7v7BU; zOsOqSTmQ*1P@mvt{4ZrwJEe{6m1yWA?sIW>?K}U-)!^lTlg5B(7`Rfx>Tuiz8xq_P z85urU5yRXU=l)Ke3Uprc;ZC~Xnu9-J5oBI;0#N_7z*c#@h$tBPpy#*u_}jF++iFdR zyjMO5z>ks2&5ZmNp*;tCAoAni2s?u88R($Q(X9;yhK>^_fF>%hK(mn$A97=O8Uwk< zrZ$F75f?}+mR-l&5CSOKeEV6|>-#E?;fHJ@#Ms3u^3I|PGxlTEGA?!lvlV0D{+D|y z5bZ-SQp@P~-}Fy`y?S`us%b}@Actd|Nf_};0N}g;Nog1l(0-kHe+mQnIx6F>08bGe zfQ3)+A5;+3-VjcD!jD{7Z+yHS10#R7C52SgJ3vGiqs5&fM?Xq=3$`` z)Z4Q@s@eW1Dgl(x1AddZ5_%&yFT=1<3(X0SlNzt89o-5WAP04%Dh0x5!>_X7f1)z+curQ zPxW^kK6}&MmqZLWU8I9m?{=~OBKN3)HV)j1CqX?yeV()}QWZ>}n$&3JsIYik3bC>K z=c4|UR*o&v077bkT+rs_Mk%A9F70llKNru*P@GCuT%@g98nw3pT28G7M185h-MdH$ zCiA=#RUO|mx`y^U+uJ=ao++^zF0A>^xI~QQ`ySgjX>&2H6$f2@?pwuE7m&s;mk$SC zA7D(0MJl2UT=!YZ;e=7wP{3SvPKy=xF5I7-m!qiyYQ{axm~7p(?`?H{02NQdU^bG4 z?}`5kVNJr)4Op=?y^Dl@J0+3e_9m0t>w}D8sf{6nX;gfIqkta}+V}(cP>Jbfanm*j z*eUG*mFo2It``bH!i=o8%K=%Ow?Q>Ujsj)hz3pzhdU2|+GeP9v+2;Kh^$LbKa+Q4u zo){oqd%H~!_F@TZ#amEq7bf+YNwsnL#fXe#4?_|p0Xf1Dq?~JVlHEvz*<|!BUDbEl z&jfQLF3g*Dx9?H7PN&5z<6OYZsTqN|B2$Yai&e!`O5~h;h-UP)(W>g6KL&u|;i$I@ zL>;Wp20keU1!+gtB?ZojUQ+lb+X@3J7*1`~G5rWo&6W!szMH!edQES=y4WZ(RB^KXyuT$88_Q=j3Sxn1g-b_pVnU~>lqMs%>fS+c2c3jm^ zAwZCfm18-!@IZwTt^?Q+@Pz6G7vU_}`T%$}z$^{rU@1cv1?qu)5?qY+tmrn5rhku2 zH2AH%Adx%_V^voyv{LY*8V^fh3F)CdN`*MbO6*w}7O*o2+)a+H2%@nBd9f1GeYE4C zIP|IPTfA^ow?Knpxsf2zBwe;L~d{lQ`qL1tGY_1nt~g z=<_~4{lypZ4In5QAp0n{IV=5KGXOd)JcTpphovelZ06lgxgYMWoSZn@)O^}DctL+G zcz*4^1^oD==isB6wq%Fyt^p>k@Y!Wc^)qi4oG>J)#rn19o^}s|;1xu6P*?(r*E2&a zi0GDQ0QGX37%NBh9rP=pzSA|gOwrW()faG2&oAvIXhj2HDU&CR7C)=?sCSQ#>j5QdSv$6pNwxjjb7GPk(jN(s@&sWp7dagDSWJ*oVCOuXn5)u`J=zJ2jasqks) zFUInSy%~URu-Ht!X$}5X)chzfW~o&O?keIYf<|C_#6iONPqwrSsZ`f0mdEjp0SOa0 z?Q<@M<=VbJrrHhb&J2=l+26i>z#l{gj8XFG-N0OW?o62f5uDp#UC;sv242KfuwfwA zOHFZN&tBjbV4md1D4g`zB-hg`6&HL*Yz7X~FkYyL^by^}oJU>2Dd0gdcGkGC--sao zVJ8S*VBp|PbN=rdA#KPC9p~QY$hV!tSmoyFcJ(SgBJ`z4T{b)_L|o+izVj7kpG@*F z9OsX0^V#f#t$+hY4S!6DJwxV&WwYUe@_Ov2Q>VeESCQtXUpj+_c zpdM#6z2*TL@Vbom^B02)DdgK;DNZf#IEv?&0znHW6E6&AwN@2icd_xzAP!a z7I6=PPd1RcsAS%c4|x$pn7fSb_-}^%>wTYoOYRqRZ^^tp}!hM z605<145TIuF@S0;HB+BSL&c=`Tvxu@jWZy(QQQF}YaNwEY-VrsOK-3IA;xTW;MJPf zhVJMz6s`Ueu(O=t222T!(h{eK&;LChWx`slj%+ec_MeY7-5#nN;vNmGZ+;689r+27 z{%tgF_p1pmVbNFZHgOlhZ8s8G@*AH`w#~_kPkk1J8~u46&^&CAuDW@(_Z!Y9e~8%fkr6}l=XU+>82$v8Ny{n^#Ylwvrg zg#w;8+|tl)BUYK>v|}v;q{KEz;<**5%=JxO>z`BuL`Nz-O7%LI&FJ_T#mOHBhwJv(Y|bN(|K@AI!$MYXnrweU1KTf@MXp zj~J0Tag#|DN}KM=Bh7OqFN1%y`*#ei@ilsbzZd37r^5xnNHz@QMPRWC3x8K0a^z7}NVW*u$jgUKg;VUw zI`?~&uL$gBZgP?@1+hzgTf6^S?q;Q%v35Q(mG&n~_P?wuP3d@2tg@{NH|^epQ}DEdlS>};C{2t~^iMuLM7 zEG)RdQ!I2@pcTa_$ex}TBZDzp5SNx?PmK9NrU?svc;?{=MK}A|U1f6eJG*3PTyT+t zX#uro_Z7sR8r_gNcS(D^mW%pWMNfZl-FeG@9MFJ8#}7B*D23O|fDZJ5Wto_Jljx*oB-?d=@ITK-kF zg*!&I9DHaA4su+K*9sTo4WX+2@r+XlR*7aEfFtX?Wx7DO6x+!5RvvG;hNB!cwTPDp z#P`1_2A=EW7bQmoTO}2~<*MVkAzJH`vyGPJI0L-Q^ve1VN}lw6-2eGG`&+ZS!Ygr9B8%|W(i4m$VATwzYn0Kr9gQpdq<8; zro{#3ZJ7Ku>ZY!)Hh&V&brQDw8&dG~p|}LcdtuW%A|xTbu?L0=C+5i8Z#niqoqA~H zdHSYZiyvy(Yl^sMQ)QyLz4(f9`8WP+xGGk;AaIT4fb>Q&od;+cm&zaR#hc6iCnL50 zR%Zf`q#u`id;2a;*B1hG2Dd#b44uS5+GdIYpgy=ehTRYU)|DzI=p zi0TPz$PX5~*ys#Jtp$$UTjA%gHf&m*cwG41Y>vp_kH5>ZbORt!iqd0vqUFU^=~S9o ztTGr%URk!EkGr50{2(c4-ng6y4HBXQgcWdA=nR)hHr?T%FtmYmJIO*2N+2&SNfL_0;YeEfirn$C}-d)QbAL1@; zJ$>X%bY$!uvXtsu@_~6lPf{vs&wNzN>mSgTsMOKKl`6CAh0@i-pdoL=J}J#sjlUcG zNicHXAY*mS^Zitg1pAtM@cew7lG@ZhNhOp0C`g=f)i?2WU|995Iz}&9LxsPo@}B(2 zt0>b;W{a>f0`9jVRF3)@JQc5pceuem;r6N#ouW?J7Z(M5RNFfr*|=C*dKP>t7&G-T zaL8oDgQ4w2Q>XBv|J)8Sv*-DpI=Ckwj6BVG4;Vg0PX4cz(QCI6X=GVk*PK!X} zsLY1fI9lN~j*I#p;RKE2iBhN@<8HCRm^#MiD2eMhtz)7RzTy-hm$?WEe!oaGp0m(< zTLZQ19lZ)B4EK*)1a?KS3tZ}c`vbE7Df0{Pm5j1!_A|Gb2Z#;Xql{Ov*AQiA2>KKg zdK;&c_)<)#neCn#&D2Z+Pcp;^$%hd}Y_b9qs3UtHxKO3VvJ|cHj*?Cj3iOOafoNxX zDuO8oU3A`4WA29iMx3MEs$%b|%hAHKV)HTUqxBW+*CG2qUj^c|U*q!I6=Gtj+_Wax z-)%E6_)hdHR)OE~XuEQ&ftJPe$O50bvS;S}dYp0;lNL6xVt@4wP?3r{?yu+lO5@0H zC)STAYoPSm=D*p#7h=s5lAc?wYz+gv3nUGc>c%~ zvRhj2wYUG4Ta_RjdT0BM)0zQY(BHH}A1^C`YEl<~PYD&8ip`r@WszsmtWj74O0_(+ z<6QPdh2(ol;0DT46vogoQ!BMvca2*1J`=&l{x}zqyM%b^#6I#~(FhC-49Z3PAS>+r*2(_bzSG7TmLaP?fgudI;H5m0XBM( zg-MYRMs}C()~UfNmNbrG8|lAjxZsi!HYqGD@)-6EluW6P%cSkm?v92XcKqi`hFC^( z7qp}d;7q*ePcR0Y?iiRUh&B)A+wVljmgr_OBFeCu=-zqF$=t}-jF6KZ1w)?S81PZ1 zA_$g?Tg=xt*Zf~HT`8@%(>NhhU>iJ&olxbvO!YCNLA`mQ>on;S;W$IU?h0Jhud_;o zbX-2-A~NQh! zRRRUFMQ8p+p)z220qIxlzrh;51Ipd231B6<`uC;V;Iw;yB-X5Q zJ%sy*Tn4U-(<$|s`(;IBbLRBOd&kAC zXZvU|m=AATeTHOMmDF96>Off@Y@)69tkOVDB1hgQ%2BclI0h4to|LA?4ODf}T&?n` z|5?E&f&ld|9tt@^_q9@DD(tC@FqU}nltdvKWPyjvRyFm&qOIFdOB&xT|GgxafHACb zzGb44smLpSHbg7o8gndO)}@Iyvm_!&$RlDlQ>{1EN1H{Jh+nsR$tc=%@G8ekUU@pc zS55nvXs3Joye|7mxRlt|lEYvAR_$!>5YFCsquSZ>wGD+mp=jg2Ed?ZqfDal5tzBM^ zGnZrw2@$6@z9DYfXz=Fmqvm!MwuPmDFu}@vY19{nS^vdTqOAzAK zoJa2d3a>g~3f}{XzLLpEMStg_X?zkQvEiW`rO;@ii90$LM)UjZuQS^0xyW;X-tpah zH7K#Ch4>wVjDzu@9ivkY1LB`3!@)d4to(#KyyEUJgq?R|1`1(;v5Jwo3WL z_hc}0E^qiMp}y#eD-f&uuk=CsFHk?-9EC$p$Jh+A%k2H8CO!@#|6{o-4`Jz3qy8u~ z4ph*r+FGlw?Ss{LJzZoLzHnJAn#UxT+w0aO;>72XxP&=e>9u&LO$Fcc(OBdq`=AIZxuK}#MX}Cx{i*i?ra0; z%docato~z0H6}`>!n{^C28FJv)zmaUYeQablyd`X70z4PT5_Z-kcPJG#K(x0NNPi` z;dn}u*!8r7Q-qApoX=1P^Urgf=HRi;E&V8rS++XsGg!~86gVX68BKRdP7%gz6axcj z8@it^Q3K!B-%CE?7_5K)o@N+u^xt`c{1{0`v83tiQ(m1^Y^)yUN zaU#Aw(KkeEvLD@e@-b-d&RsuJcuIgvVbP#}0BoXZP6*wS_aIXhb4G67(e$H1@7-zT zdY7@t@l#<)pTeXn>0lLH7btb$S+Ha0Ij2+$Hv2tA> zGJ0j7PIUhOTA#uOq<|5VP)XNd z0Y_DOcau9`Ro&ZNQTqKEFJV_c3iucjRc&xY@$*-21>YSBDWE14 zPaR`~&LKF{^_A^Vs^VtFw?{o;c3dc&v0Vnv*zTDt&e#rI)_O8G6?=g1(pL8NJn2!0 zO2|2W{0ydIvHO&hc?sng=sfU-uL9WcX=`)a1N*pU?}=jfKg!>GR+qQwVqV>=uvrlT znv9LI*s1&1{uUemsNb5SQGkZ)_=`5i*8&h#$})?)Rr zY}%sUQ{RI5S<+fDS(&JlJ~F+Sl9{Y=6_qk(yB5vXwPRU2(pAcbWYUOdU6~>EKXmZ9<4c0a_ZIr zSeSSX&fw5uGTw&YXBB|M)L)7-K{S08EM`It$OM=GgH-l^QJ&C6W8~@s+QA51zUkz_ z=SkC2Q(zH@%I@v&t)kHddMj-@t`N+h2q3r;*)qNA{*8&H%cgxO)y+&JEE~>zF&imz zksI&$I8;E{zFFoI5yP7?aE5`ABD%=Qejf89@H(6jL13Q$lwq-ZvtE#48zI+^ zxR_8t7rA9}-AyP&+V|IFblo&$$j1B3J#a#46tE5xX4md1qnOF&sM&5Q(Wi?yTq&lWb zNX*Ra5E&Y2DdOpAbA;wFGr#Uy=DgSA7eYlUDEW!mZkfUNJTEr`5VEtCh3x5(gRuz? z(8cT50&0z$2js0aVs~X`;9LW98`-VdSsKK`1<$s)ciu4iMs5fj`n_TO4#~B;)w*w` zRM1lM^p;_k&+8tLdEz2yi5AFpQZ3?%ghe{~z`f6SH71=S9rU5NeYBB@h$5ei3#tZm z@OfaE4@KEY%V5n9tT*C5vE%E1tI*o2DswxPSlC#u+jE4E34=Oodhp~CD&DEDF6P+Db?Oz^J(nl|u4DE>N&ozXCG$Y6mZkbRectPnb;C{` zH`T@YzEW@!Bs?_UkU(w9*6dyYzMPj;9Nvf%r+ympK_K8Wrf_P{b;5fWcR1Lwd&IzS zUcp2ePZPVv4j+sws{23fy?0bo+xzYtQBgo}CkRNf&?Pjf0tzYu(vc3K2}lb?sY*~p zMd>KL_ny#u6_g@fO6Z_c1u3EeB1P^BvI2X5aqs=T=iEEaIOCT;9F8TDWUjg9n(KYu z=lO7dQnH8#MNAzxGHZ0BL`n|}z?aIoU(w@l?+8y*_6C-TKcP(iV$y*?e&bUlGXHzy zlQ=RT%H}8&aWu#BJ=d~v^oVQ*)*%4AH~zw{S@XfM#d%r4Bqnky{tUlzIFHX}!kWKQ z_C#iK-N|LbDQXxC9jB&wm5t7V0GQ9{S-%tcFnu20M11Uiti0{A6u)pf700n*k}vx6 za;Xy)SMx!)o|y^Az@eH9DasGUY5oZQ)5ihLe;%j#uM-jNZfJjZ36O%~`B$FZMSnk= zT_xo2AR8cd-Xt0y=a|b*plzCkosUUbBytwM*V@h@ucQue?J&7syWFa8mfVK7<|QiF zyHx`|G^DdUSR73>MCN30cd3IdL05e3l;v-C0ct3XbJnm$AH3+C9jrLA2s{t>%+V-OfroqI=z6Pvg>?1mkF*co zn~QI4YI*hKhHmZHHBBSrMadagpXLdCKRlp@|6-i^k>=+HYT&fNkH7*Y(rbuf?&I9l zds?&WUX>vYF0vOiG}S;Lj@dvxC~Y;RXqWW{ue;$tr*mFF!qsLbGO=_iFE(n5o2DNa z*lld0mS(DDG^jwn>Om4v^@{4cMZInNaRE|e&Fk!o(Db2y(xU%T0(f9-e#4r=-c4T0 zYa3Dp<=%!=;ca}zyWpW5@OycHqNE{2KoI$au)S;bP^Ec5M6_0Ttk;9tyPAf@Kokqc z4!%skwhbEjGu`g(CD6S@u+0^z+5N5ojQY=mh5hN%uX?Z%0+!21sPdJy!s+FZ46x=# zpado!8e++VjtynRwcKl1dU&+cgAkLu-mF~@YI_HdfG54bu_JyGcI1p2qeQT;cP`%@ zC`HKyg*Auw$(TJB*r(v{)d=|ya+`w%ei?9s@Xn~dO;u-38|BHlUn$|)TcLs`cLm*K zRS;cDL12!+KSIKT!rWWC6%I}b)LKFi!2&hG1CmnHz4KF!By8%$F*b_g9PSvvvJ9k3YU+DdTWHneL}cr^Us>We z9jN!$wrR&eoZ2A+aE!sOgx|39@P^K|%->~Us)3~~0}Kdvn^Nzh*}QD6FfToB%Hk=gGnehB&1S2TWsC$xw_rg7@YV z2`bI-sz*r&hMW}c7Y?f+W3~w{pFgL)v7>p0tnAk7_VLBwO-Xy~YJc}*t?>Gw#2Xpp z>PX$lS201)@^03(3-{|rhED2)&9kI}2@{9qK*6#szsPykM03nXlJ=>L=?ImepVR}E z5xaLoVZA9Ruqk*K4tBOouAmT2Vlj(jSWS98pDF7bc{Xo3uQCRfRnJj4gGy&t7=DX< z2n*ScSo#$JmnOg4`#|`*P4-G`p~DZERO2T!7qwR2WuV~83AUhSr?%@O%gIt_kZR}> zde{EyLqS=?>vr~#Er?{_yD*WZW%^^F2$X9Fa+^qeSVEs+^c=6kgV<`495+4fpRVeN7vIsop=qAb>7KLWGWWc7|9$&JfHMp zeR@-G{gO~leOviYq$N8HzX3c43g^!d<%RRUm=uNKt={Xb4u05gyT>pMYRrcH4Rr#j<;U}Ni5u5RbhPvav2 zS#I&|6qbn9AV6Bw10bFjBUIo>E$KrmHf7WYg4s-#6M;G^nnz36Fh^?_G$% zZ#iMHf5~h8djs5&*QSaZw~rHtJ~?xE)&50ipF#HQ`tj0(B!9`TvyC)<6Biij6%`&8 zhEpF_J$owh)Q*RdcNO0Xa~o;U6}t1G)=Eg;@+7t_3#scFTgL;|#_4v-$dAI0iR~Dm z-t1J7Kiu5uGxERB9i^w5R-YdteJ3qN|GQijO>YAdoi(Bve12CQn%V0LFKwQiz$RJT zr!?h1CPxKa$;CfyD!ku=`dVZ2TyiU}tgp|%UDo&BF6-;>lxp2d?F@KM<5a@%PR7gb zqC66IWj1A^Ncp3J3dXO}v>hMa@FUZ>a1_Uy@^$QEi3(GwD*@qQY;ybiS0Y_^G|twF z{g^$}!#nwvu?+hqfPVKzTB2hve5kOUVirswrcIOtT9RqP%G%v0Bov3->haxH9O1WK zUZS?t3SeQp=H5=WRQH`xW%G0q!MqqbuU)~tduoLzSDnim&1WmP#dQBt!mrr?1-7QJ zut%K7FK`mlQ5v|Ow3bTQ^fU&Qjayfq6Plj3)8`k;d^sm?C(J#6MeHfm@>9r3oaRR% zZsBWH8<1sX#=zhL`}+Nyi=jfgBw*)xk!-5}MqXI?7w^gqyKNi0xA5-fY%kvDXZQ$= z#Ec=dpTs|;iP`?%K1atJn!DcLG#_-Z<&zOql&r4b+ z7Ew^;FERl!hdU9JF*Kiw_=vY4Le0p{^}EsIGx)LXb2*`mi;ZdO>wXaRjzN&9)7o~k@ z^6?Eg^Wm-OYOE=^lvJS8u$`>F7Vxo#y?dFgat5S&vHuIB8ozV^ql5#F`!!kBZubuT zvgZ=cs%QXaAkN~AU0lOQu8ZTCu*fW{>#u>u;YI9}X#Qu7GK^i~$R|~nQyPz$NB zxZll2RrU;K{9wayt32Wq=BDA7xhwEuwC=iai&L_MPS2I((jELAv-XAX=c+|S+$w%B z8$!c{wl{Wl)-#SRcGHz!pp{Jy2+gI50qSW9giM)qwL{VVJ;t|n3iZn&#b9zN-EYmE z_)84U8^m)u!lFw*iiRnfjO>l+Mk0L^wc5vP2OU~hoJ;(^)UStr4O zzdNVF!3MiOVnMl+4zwm(-5S(Ctk}v?FMCY5bc24rU0#4eWJ02hTc>wSdnYv2mb@wR z9zPHU0EZ?tH;f*4vbh-GUp~=oH&Yr4u2qMRTaMN~v4r0l<;mRk5~@r{tk9=k5PZ$Z z03b=Fl`Q*bJmA6@6x=ED=HC@w4r*%yTrurf{zz%+4ee?GE9Te-=o422O+Z4B59qfU zBe%3rEpEsS(GfoQOooZ&bvt7~oDA?3@7OYXh5ds})sDoHfF}`AABwOdO_!*$upBFJ zrrDNH+yE2IZgbRb)f{5UE&yb%(R1SXW;xrDYvr5LRpypp;hUfezo*T$wuU?P_rfq{ zXYVvIx%X?=+40RVe@|&Idthz>Dj4yMthha{%!8dzm9z{9I*bALvK)UdN7Yo(KTs9L z;{7pV9Lg?e_>JkX*~yR_lsQxt3R@8Rryb4 zV=}d00&(E601rENOrj&~eI{meXK>;f&kpo*Y-&7v5sFiL`aY~#JAT)5P0=qDa63?z zqKz2A=~K8QKG|07hScDB1B(BK;HX93AK>in8E$~ial+WtCKM*gT3JMnG$6kX9ZzivYdO#*Z zA_|AbYZVC3MO#SfET2T907crjwOEn z`Y(U%ptck+l4JGT#?F!LfqXF7DH1&J2!AF|DA<7==75RrZ_LGerLQ`vciO@juS_05jO>+!jW#26b&(AUE&?6R& zNyv0>HMiX+SbZ*^1n5fd-43x}03H=r1^`NtviKZ^%4@fw4N=9&c<;$Oe01~R7mQc> z#B$2VX}l^Y-?$Tk>3ISl2a!E*JaKN+(Z$nwYB6F-%~ba z<_Y2sKLNkG`##vJV^{t?aE#**iMf%AfIH!5i529E(!7vsO-3WicCjD=Qkf*vl9>tchmTcnpKkT3!%;ZjOv5K=2DG)#TK} z*0UAcyEMU(CSLu|iAN*`DtW1o>pPML25B0%oql~1^Q_h@^b13IZ`)bV3-IL&aLAGlrojk6DONOokf3Bz4;kH7HbFsrbS5GIN2 zl%`9K+|~ai9C7BVMgLWCBC+~kKql?~3u$Pn_5IA#eJ=A5(y`)F0D<^HAO-LOJ{9YC zfhnE8ihjfum$vCvlgz6S0~J;1r?ySvJY)2af?~Wsh@7@m_%=V3f=FWt1B% zAORN0xjANVus@jLd(Z+96@bgEOPj?c{?&)o+^HUiu?oFScZc0ATad~=OUmBMW>4VR zP7}-P|I7e9cnN=6onPh^FF0+(%AzZ`>1pcQ^faOw(8Mdp4_-m>>i9MdS=~@=43KDN zai0JHf${PAkdyb^uCnZlz|{k`X$o&6WBUNIw>C}KqB9`y&v-uCIh|1`9@ z&{oOq?y8n zPe33_|8Je2qyk{%L2dkem;}6P;OBL*f9TB*o5$n__Tqegrhudl`25JXeSX(LWz#L= zz`rBR{hcjtU)3C76SDRydLasYF5n1c1q%TEIw}DwD*dv9{j7fFj*8>I6LMCQ_y}*5 z0jshE-iK#F$PiehCq74&vjZ&}-}R#?eho9TP^0Yb4dckQZvlFp>~kte2m1z-|M(Y| z(1Z4VpNp4;f&6(%1Dqkq$#Dj?!Vq_;X+BJ{ime5pox#li)Qmlx$iTpWyf%4}jlflr zKe?Odm2Z#n5GrN*CH7$C?jHUoSiQFR8};QZB@uYRNxuO@(?Wk^!MUk%7S9x8r;G2p zWsVEI>#)CJPl_|a;XF@s1m|&&5~MNv0nfz8{(cz$^Q`Pr+FmEC+%nGx2*XDNj=-wP zn4LmaattFuG_W5|7}%R5_}AnIM%!&Q;9E@aqsy!`+o&;Q-h7qHk^%BS+wP*73-Z|b zECdz3q@sV}*oaFgZQB9A68upGD3-D9^&k+mH7y4vYGjVJVNEDGtkxrZgr=q?lvKD2 zi5o!!(NjQE)oc=zHK8E6QM8-yz@ws!Jfeb*mS(OOLkxiIuLy z*|Zt%0&P!&V*-~q3%-Yk_^?VOsIZ{1WdctwvXUCT@`|?4Efe5uF5w4;x&t41a}_!>1T{1J$k5g%jTx7+D_dx<;?}RbaZU=~q<6IW z7LW<@E_m`wTxHPq<#dI=7ms@&dxGbU>`skx#oIuRwK)zyM=L-JOo$GQj@v6+ys|_k9Hj_@h2p6&}uhP#Gdbgzxa`{x7 ze$C~p?aJ&F65i`*)yTfJ@3#euWiyn0kTZA2uq?4}Tr7c%lgt5;2aT;vlj%Z z_@yjrhj;LB=xPI>=m+2>0c9b;`gZN>{<0Hf-^{23-FI-|^r!uAk9#K~GZX%{Q8(-eF(VyA6zM6s5dReTQ~v@fWHD{AYq=vKyD$OrU3Vx zm3cxrD zDrqjzmc}#u$!=IviQh87E^^+8jVfFkaLyR;>@boS$nk)&NB1$Wg;3b2alONv%@%cq z_g7cgKJ2b#blsf70&G5yTN@tv#6)>!RTqzc-jO5j!9WyY{DlB~@Jr1#6U5O7E`|l`PpzLvq0fM&+ z`53e0edx|CuSu8kV5fB#9VlIp5}6HBXh|Kj!%KoB_J2tlb#AL63jpQM)?RyBVB6z> z#fg$`3=+4_*?3XRZ>z>S0px}C;Lq&_t{;&Gg%hE{h(eP~PM?KK515;1{^%@h=lH1| zr4_z z+))-oPUFcV((|``Cwek6Kgy z{<%f&CMeQ@5so4f2ljPKf(WWwFxWHs$}Z6EuD0+GVI59|FrXCqgoci=XmLzJstkio z$Dm+*-~{^pZ0tmykgXQi|FOqze}g%j%nuzFgh{FUbY&I85l+4H6A;Fs1C#-kz5 z1WrR!G81dVpTNjh<@!hlvvjhRRDvLog!!j}36#BRW8wi3E~m~s$~S>*O?A2rGlq<{WV50l)iMQ=8=65I-zJ9vO>~pX2?qGPIziPE0d}tFOT?Prp}Ayd>c;#a z`L>3~la@m$&X$!^9N!OFKf!l3=} z285~_&7auLOYH7t!g-}Giqut!^>Cd%{Ku~A%?}QTC(d3=9l=vT4&L5RWR&9lTWRrs zmy_%iH#3j~t9W7<^X-MP0Nr*FY+6r~i#mCFSN}_ou8InU{u63)2a%MFA%M{fh?GXO zO*OanV*{v|o>ccG&yhamWw$7TnXnc@|$Kya(mj(^9Y z1$ch^)j%454|0IDX<0+|xrK^N5-r~m-j%V}m=v1|WpglDa}RS4%&7&yc-#;F754An zYGVIr2MdXDVd3b3(OP?) z0?4njm}9N^r?4Jkyx;ezf+idPCtSWChi~_`N#+;33u>1oXJ;tuFcm{HRGG7vpYZBW zzO>7ra6q}eN$`k=i|u6kHK4n+WM)O>Mge2=7i~XkoEoQ(U7_*<+qwxtm?$*>G0rhU zMjI!_(P_6Vja36;9OEQtaXol^C$7>q|3!99({MFZExs<&KU?cZlS}UK{Lx4ooh3?T zP`vzayP~mjcuY|7X$q5?+l}35c12-Rcy+tKt$-ePGtt< zbZlwtN7k&{m)s!lwVLNhM0H2Moi~!iolmk^K)+lkJ-Q420=#@#!8jyTerNnP63PNV zLWT9;Fabx5po1Yaa721wFbxs`+!{9qCF+lp0u~j^Ic9vLi5pPlmmTP?ruGCy|Yw;-e%fHDI(bU2h5E;$rH!9@>(gb&iLlzFvTxY=F@-`F5T?S4>? zlW*9oA&;Fad2IN9`!U=}*#EUxn%}z&F^3{Y6bA+kAYlM%qIK*StEar;XgI-Z(5FJT zBa}^|_9b?4aQ#&|(u^k|0kw)fjuA=Zyj2g5sT2w@*#M>zpjqIUN*xgVv)7)EjYnX5 zjVusHwoHrJ3AZG8WuyMd8Y#e*9|>EqFNq?iR6$dS3`e_b6KU`ou^VC{v#X%4UD==^ z5rD*F2N5S|CFMSlLlebuE;3JG^Rm1Wb@Q+bXa$nyNCCiDB)q@CSVA~3*8848X*U8^ z3oDM0MD(2>UQM3dZONB7B+4hMa|%Nt?068=ZJ8G)Tw$E_L<_(0OTFpO080q&^-sBd z-!5`g4k}q0G^>BG&f-J~;NHjc{OD!D7mIlsvw=TfcB?=r^^UM$*HXw7T{$*d&hQYO z&O!=KsIQaaty!!fo)!OzfF=k&3x$oWVXZ5~-$A<=C^J$1L4YTNF;S{C$!=q(!aH%; zsXB#|w?EA@T{%ErSWGFqogxUJ%cip4zenly4Jz&3AwwYljf3a$t;BF|pI_0Y zfU{=JtA=T}1 zoc37>HQd8Z>KKC0Szf$0`gPH;-qkVl^|DSjK;1eDQt_P6f!3)TTa(`Sj z!{uOFGciv9C_?`J@L0@)*@v2j@!+bkn>u+gDV^^#`o_e^u{;28$b>2?5BzqiOKWFa z`0HZW@6UtUhypBn#mvvd{)ex2^`K~}t=>Zc)eO>b0-mPj-~od=$<1p)vuyd*o<3_{ zLjS)VJV+cQ6ObgcC~I}B^_c92(C|Eajx0JSy$y17W8SME=v#p4)XYHRm*{Mlf-kV; zx$C<3g~jrt*jGKP(}Ua?>Eg5%LN+7}un&^wB%=bNpI%^BY_36JI<==hmnTt@<2{U?y=I#R9 z4`rv{n2m7)P8`QNcgT4(HiM=$?oh^+w^!=?YyH}Anp9$>KKSLCK9K6syj3xmqqb0q z@LlfTf|!2?EHL&VDC{7FFf_~LoMS{gl}f#u$5V_Y0qvW}+57Le@D$%CWJYy{$p zCz6vd4Ikx{$}2$+$6%7z-SC6+KY#h35%`}G*lz?5pd8%@7Wey?|4)u>xC}P1znic` zgrU*X5|^d33erXBgzWx^akAy9tH5C6ma&eDt2trOqjZ%&wZ}%CHAMA$>Rm>Nqn1ua z@}VR3`ePGCDV>mfIX-;Uvp>zyQ7#<$%BGCXO4oo;A|yM{a>^R{P}+i^{fZd*=lkv^ zMq3a&mf+RdZ=(K}HqOw-EzQ0C`c?SOw+;Wx*szDM1vu5>LhMA24Qq^i($tST+#z>P zBP1xts&Gt@&atD9RbV~XySQsag^=zUq4Y)Og&2 z(CWrdgCks9kg);6{I_b939=qAhW8Y4SAX^>8N41hb#;j&sSO!-jf)>`w;vZ)$|X2s z#U9*`P3H_k%ZV{4h^tF04`QBeii9bL20<*QX2~MLwjfL66KFnCJL6|TME2TtGaXfT zPhJx1x6(ed(O8EKhdLvdk*DGVQq?WGI$*Y^vmAstwe0f+`!R7$X=t`|o;}&E}3D)xod+{p0Ji0q2o@q=JF>Pzh z-HFT|eU<`wg}##h=2=~;Vr>YDbHRSpjE7adFJO>gM?w7wAGFBJT!z4KkxN@bSL*?) z5qa}yCMSn)f!849<~h~VvQJ}oguJQN_fumxz1YeLIF*IxkiZKfI~9Y^b5}! ze|<`p)DXFx+0O3)I=PY$`0j3wWIdepNVVC5R21cG%XPE~9E; z9ug8wS?0dwnRUP9xnXMRbKUl2wZtS83kF6|gFBLQ8+`<+g|n+ z*ETCoH^$|=II=72%=XxFB==_Ygi1URG!yHW^<1t2phsEePe{!k(`sf8Hp`#1HD#V5 zlg^7Gq}9r!Xtyt7DKO(I&S!B9l97GlLM6DWHJm=%HQV_B(LnS5F*n-`|Ja#Rjf9uhCHnCnq_F2LN7-Y5Aiy^l%A%f2WLlerN;Br5^u$b`qp z(kNVe=Icr++BcFwxgoMpS{fOCP?Fn8%*`QI$Yg zeQ4D4n3oI34;$$@4leX$diOs*tXr6C#+rOa>50nIi>y>axp7>xaik6+D`_zeYlkr* zey8bJy3tu${!)qM>dLU?3fZ+6rIGKS7|5dq6L>Bk&IS5QPTfSK-+L)@SN8Q=qNWm+daW#Zr8;Q-hOVL{GVJsyi)>v7A_{96=*C^)3UwzG>k6C8#Jf zJflUrAsL%;h8O)}golKHvkA4hcWmKY#qrxDqw*(Ep?Copc21Y8qJ~(qz-y-u_2&1D&@;q^6NsEbh%+^tz zsP7g;XZZ_OB4x#Gzvh{I)L+qaNsud>*Xg7ZPp!ITy%LCEMn6$a%lsB!OdhZBO6#~z zdA0<$da%kXX>TTZR|tG}S*IRR?V^Q+lTbeNo~78se`QiXcoG(*9w<4KZ* zJz9J`v|6F=FIdhHOa@EVd-P>CzR2KBGLBxG*bu%eFT!;Axfi{~bz#_N>!yeOx(c!f z4nDUCB@OamT62&3WUKi&S=ye|j`4Pd(4=JNlIvIXt6oZD{VIhs{q|O&Mm%qCNub{M zmN@zf7lz+iC0{`@PO;Jrg?pXq7_v*uSyI`8JUZ;yA4WPIHeETo z`krFECFJEt#VyE-ukEV7ar0cc*><(jW=qP5jKPuxaSd~h`$;J?a?~^SNfj`c@}MuP zQQzMFxSs|DYMyymH-2yRUh^LQh>%Fyo*4)@tsVGzXXluUwZsRdl#-kI`4F(_6ewlmSG(%sgR zaaQh`Y!06NpC+CS<&^o@a`|u)h$J>VB4bERae>;bdP{k9WB3ZIR~TFaW_lt94c4KT zWh%yga4*{ZB)?0>9k>NK!aaK7&h&ip(x=vW?oH3QaFg*S=( z+7agpE*ZdL4s>+#eSZ2h-4kQSP@$18fHYUVlkvh7+2b5BogMM4-Xq~f0L7pU61AE9 zDp*#PDTbx_ez{!vqyzu6Fp+u9Ww-G`!M+z;kk$g{GwDh)?L#eM`Sy2MpnlK23?;6Q zEK%t-eC$LKHU%$sbhfK9Spef$EUs~)T*AuH?wWPqaDAzG@k-#~Q?mq|Movs<=J{t? z{?q0QRftB%Xb)2(RZ6B!4W^@SeV1Roj;@y)>qIfkmWJv$Ppb&JaxMSg7!1#tX zQfzA_ub0uJ`y^0)*7kdyS&};NcIqAX*v4gz;dl1JJ&t|Dw^QbsDX^us@bjb6wf8mi za*c}U!o$=3v0fQZ&O-Mf8+}r0akVbLn$w59|9&>xa?G(A*v;h z#(P3ZuW4S8CHuKXl2SPnjl?}}4Hnj7T8a`>4)$3`%N59hQ$>ZS6pO&Fud1+)NyLWj z05#PN8qTgNZlQqUhcU__1M^_P`Ab5a_swS?eb!9UpjGEb8!WEN!4g{|G8E($)YI;~ zZRrfrhBw`l_47GpBe$@eg!*};xusnyzGwCiv|Ug^fwBh47)Rl;DJphdfDZQYL9?{g zhF@ZU2Rl3L6fZ)Ym_dZ@lA`BAl}L_y*G;*%N=-u-uhp|9c9xp6mdtOI@Iy`DMGWjJAL9LG*ySx-v71# z%hSQt`HQ|$M|0z~eo($-hqhlYyKRShhrXZbu4Ri<|KV%>lKE<@O^o>n(Ujq$eC!s) z_1YV!Z=*bl30JZQ`j>=mC1%QU+e7O)tV2(Js>1NNKVDTiFVjEOKsTzG1PX}rX?a&)4b%UaqpI4Q{6~wZMf%q} zQ^*iD&pbG}e?7I%f#!bepFU6C^tl2(1v;|37S59ivgkv=8DM@)rH0Y8q_LQwJ*V#FmcX%Vj zTOJMyKB=zwsdnksw}e2~C8Zz3F8S|Q{Ng_^Y(YYfQ(v04imM|3d6(e4wH`whv5f~Y zB8H7b^v1L2Qdcc)c?1%NBUSrd3JaU>r%1Ww+wpdNc9c4`@RTT1U838`o!8Rxbyg9f z0^0ipEu|@gBXggC>p4^r!jjUldJ7`B1$lg6^8L?`d|QxD)x)Yi2{R^dS&f5M$|YSx%k7*6Q2$VI(Kiv{{YH zdyBC~`rO*@4)s(>9TT#3LbaUldQ%XXlaCSiSTaoJ;6$ZHTcp9ZAU_MkSETY_HtI{w z9vvBtuI+rMvZf=UGkseSJ(H?rX$E*PZFKLL9RvD8PQO$FLb$X@M#NF^ZC!}fP|wNf z0K`&NhWC2*vN_WPveO~E?crF%79`*h`=nKtn#d*o7h#@$vHo86!#%Z!F-bJTkq=O? zZ)5=$nN?&j0>2j68kcd~zYj-=@Jsfh(UI8q>a*C$Tl_w!nS?q=Tnt~kd#*lL)5(^Y zuOhv6Q^J3#P_U>qG%R#Yr3+b7Or;vL`Ei+b3*t^%6=^N=RhbpuNf(g%KE~>B&w<#~ zw1LbYgVHDNW*#l_>JLm0E2b&Fn2=PyY{@e%b6GQk)5+GTrTK$~O!K=87b=~1u|)=& zTac%sL%&(@j|?n%4KzBLGV>Uz!Sl{a-8ai7J23^nK`m^fHJ7IvDs_8!6@SRQt90`{c!(#1al#(h~_~jy$a(iR349LZhuxqk83is*5JX9r!TRYANS*d zz8K}J0@AaWYXNDHY?XH>jc#~`wx3a+&>?rNn#{SVEJ)x zW=StYRYwz>e(Q0|y%Db#-$`DKNDLeqL9KntJ5zm)2P)=MJ)8cD)APjug?FQ`Wn_70 z6N)X%%gh3L%B#y-atp#9ja_Akt@in9C;wGx_3Qm=zlusYWF{;#idj`cuuMNauFG!i zC*j}|vvH+*D~nJOIyC2+1N)}e+)7{*>J+bbe}XI@45_I<`;w*ZG)u-R{9)6`$85;fZMGnGC7$o}LW50%&P7uW)Bd%9tPxe3SBB z;aln2(1PdTkNc&4y@Pyh_-D%;5(MTiL!qz<3BXK2%9E5INDf;H%TGD?B1z1MNp~nD z=e57%TK842h^NgObVKLAD10}R%V`gM(tteeA{u3CPeyl_`MaY5mj4xfo+pOX7l}p% zj`2JjasBGE1+pxiEj7j`@3Sbc9zc! zs(5;aQv`eqFW*FvdJ@oUJkip*7hvAR)%OJ{(8+?B8yTptg6eE?8kkHyAoCI+X4*i` zGMO^8Nm;2$TWboG2+k`i9C9P&Nx-yff8t9JL`jj@r=y2q;melT*ls0m9)dDUHtyu6 zdJjfLE6tY-ue7YHnjD+vUyfN3hC_7>DKP{oQa+579K!11()K6M*m1E6Kq*9B^VN)6 z8?Q!2hCM%}zi@2wzR=xGI@dHfs)v7MP&zXnLcm+>UGPxSjB3+#fvM(bl%)WRf|y0I zh#|{qtYspT)R>xr%!5Hchs++kQz=oIt=eVj*6VTq!j{N+NRx@e!pF6ldh>x(Iy zchAIMGq>kqlt2&mcXf{Qvd)>}o$Kl`X|JDf%LjPz3G<^HyvO-TLqX&&mcUX2H6mRE{NjAVpN+_JJSg0P8edWk= zY?NP?uAAvxaEThcgE~sWnA_d~jV_zT$Yy63%+R!`vFG+0b*P;pGuAx{`))efH~*1) zQ(sPpjP~^RJU`3Wk}_p3O|C3avxHQJe8-tJiD9^P>XTH)fETO@f-8~+DsYw*;4-x+ z!l+D|GTH{;vQLtetFz))U*oJ280nkef_xYh_Cqmk9mxl;ok-nT1=^;{i-cxf|? z+mX}-%-Zu?YOcLWI>d35Cg_=?s(ub#dMHL4osgzYtYcps$@L42BEi%As~yH~GqD$1 zrgZ%^1SlxlJ2OgeR|W~EPhoUD(=+T6*IJzs4!%LrGs436$Q}wCid(9?*oY@FZI|c% P#l7wSmLGYzHSqrcWYozo From febebc604bbbad2c581b4996acf8e82ef5af2f9d Mon Sep 17 00:00:00 2001 From: fengjiangtao Date: Mon, 29 Mar 2021 19:23:26 +0800 Subject: [PATCH 121/136] =?UTF-8?q?[opt-378]=20[flinkx-kafka11]=E4=BF=AE?= =?UTF-8?q?=E5=A4=8DKafka11Client=E9=94=99=E8=AF=AF=E7=9A=84log4j=E6=97=A5?= =?UTF-8?q?=E5=BF=97=E7=B1=BB=E5=90=8D?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java index 2b6ff0bf0d..007afbc7c5 100644 --- a/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java +++ b/flinkx-kafka11/flinkx-kafka11-reader/src/main/java/com/dtstack/flinkx/kafka11/client/Kafka11Client.java @@ -25,7 +25,7 @@ * @author tudou */ public class Kafka11Client implements IClient { - private static Logger LOG = LoggerFactory.getLogger(Kafka11Consumer.class); + private static Logger LOG = LoggerFactory.getLogger(Kafka11Client.class); private volatile boolean running = true; private long pollTimeout; private boolean blankIgnore; From 07bddd49b5ce2dde1987f8d2403b0b62e21a3bcf Mon Sep 17 00:00:00 2001 From: dujie <2774584057@qq.com> Date: Tue, 30 Mar 2021 11:07:07 +0800 Subject: [PATCH 122/136] =?UTF-8?q?[feat-34017][docs]=E4=BF=AE=E6=94=B9esw?= =?UTF-8?q?riter.md=E7=9A=84idColumns=E5=8F=82=E6=95=B0=E5=90=8D=E7=A7=B0?= =?UTF-8?q?=E4=B8=BAidColumn?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/offline/writer/eswriter.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/offline/writer/eswriter.md b/docs/offline/writer/eswriter.md index 35029a6c80..2e2456ea1e 100644 --- a/docs/offline/writer/eswriter.md +++ b/docs/offline/writer/eswriter.md @@ -58,7 +58,7 @@ -- **idColumns** +- **idColumn** - 描述:用于构造文档id的若干个列,每列形式如下 普通列 @@ -78,7 +78,7 @@ - 必选:否 - 注意: - - 如果不指定idColumns属性,则会随机产生文档id + - 如果不指定idColumn属性,则会随机产生文档id - 如果指定的字段值存在重复或者指定了常数,按照es的逻辑,同样值的doc只会保留一份 - 默认值:无 From a6e1672c1e1a451c1842e2a3077418675706a680 Mon Sep 17 00:00:00 2001 From: wangyulei Date: Thu, 24 Jun 2021 15:49:54 +0800 Subject: [PATCH 123/136] =?UTF-8?q?[feat-423][hbase2]=E6=94=AF=E6=8C=81hba?= =?UTF-8?q?se2.x=E7=89=88=E6=9C=AC=E7=9A=84=E8=AF=BB=E5=8F=96=E5=8A=9F?= =?UTF-8?q?=E8=83=BD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-hbase2/flinkx-hbase-core2/pom.xml | 45 +++ .../flinkx/hbase2/HbaseConfigConstants.java | 45 +++ .../flinkx/hbase2/HbaseConfigKeys.java | 72 ++++ .../dtstack/flinkx/hbase2/HbaseHelper.java | 230 +++++++++++ flinkx-hbase2/flinkx-hbase-reader2/pom.xml | 102 +++++ .../flinkx/hbase2/reader/Hbase2Reader.java | 118 ++++++ .../hbase2/reader/HbaseInputFormat.java | 365 ++++++++++++++++++ .../reader/HbaseInputFormatBuilder.java | 106 +++++ .../flinkx/hbase2/reader/HbaseInputSplit.java | 51 +++ flinkx-hbase2/pom.xml | 28 ++ pom.xml | 2 +- 11 files changed, 1163 insertions(+), 1 deletion(-) create mode 100644 flinkx-hbase2/flinkx-hbase-core2/pom.xml create mode 100644 flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigConstants.java create mode 100644 flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigKeys.java create mode 100644 flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseHelper.java create mode 100644 flinkx-hbase2/flinkx-hbase-reader2/pom.xml create mode 100644 flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/Hbase2Reader.java create mode 100644 flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormat.java create mode 100644 flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormatBuilder.java create mode 100644 flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputSplit.java create mode 100644 flinkx-hbase2/pom.xml diff --git a/flinkx-hbase2/flinkx-hbase-core2/pom.xml b/flinkx-hbase2/flinkx-hbase-core2/pom.xml new file mode 100644 index 0000000000..2da3a4e592 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-core2/pom.xml @@ -0,0 +1,45 @@ + + + + flinkx-hbase2 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hbase-core2 + + + + org.apache.hbase + hbase-client + 2.2.4 + + + log4j + log4j + + + + + + org.apache.hbase + hbase-common + 2.2.4 + + + log4j + log4j + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + compile + + + \ No newline at end of file diff --git a/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigConstants.java b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigConstants.java new file mode 100644 index 0000000000..fef2c0e013 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigConstants.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2; + +/** + * The class containing Hbase configuration constants + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class HbaseConfigConstants { + + public static final int DEFAULT_SCAN_CACHE_SIZE = 256; + + public static final int MAX_SCAN_CACHE_SIZE = 1000; + + public static final int MIN_SCAN_CACHE_SIZE = 1; + + public static final String DEFAULT_ENCODING = "UTF-8"; + + public static final String DEFAULT_DATA_FORMAT = "yyyy-MM-dd HH:mm:ss"; + + public static final String DEFAULT_NULL_MODE = "skip"; + + public static final long DEFAULT_WRITE_BUFFER_SIZE = 8 * 1024 * 1024L; + + public static final boolean DEFAULT_WAL_FLAG = false; + +} diff --git a/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigKeys.java b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigKeys.java new file mode 100644 index 0000000000..344ff7749a --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseConfigKeys.java @@ -0,0 +1,72 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.hbase2; + +/** + * This class defines configuration keys for HbaseReader and HbaseWriter + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class HbaseConfigKeys { + + public static final String KEY_SCAN_CACHE_SIZE = "scanCacheSize"; + + public static final String KEY_SCAN_BATCH_SIZE = "scanBatchSize"; + + public static final String KEY_TABLE = "table"; + + public static final String KEY_HBASE_CONFIG = "hbaseConfig"; + + public static final String KEY_START_ROW_KEY = "startRowkey"; + + public static final String KEY_END_ROW_KEY = "endRowkey"; + + public static final String KEY_IS_BINARY_ROW_KEY = "isBinaryRowkey"; + + public static final String KEY_ENCODING = "encoding"; + + public static final String KEY_RANGE = "range"; + + public static final String KEY_COLUMN_NAME = "name"; + + public static final String KEY_COLUMN_TYPE = "type"; + + public static final String KEY_ROW_KEY_COLUMN = "rowkeyColumn"; + + public static final String KEY_ROW_KEY_COLUMN_INDEX = "index"; + + public static final String KEY_ROW_KEY_COLUMN_TYPE = "type"; + + public static final String KEY_ROW_KEY_COLUMN_VALUE = "value"; + + public static final String KEY_NULL_MODE = "nullMode"; + + public static final String KEY_WAL_FLAG = "walFlag"; + + public static final String KEY_VERSION_COLUMN = "versionColumn"; + + public static final String KEY_WRITE_BUFFER_SIZE = "writeBufferSize"; + + public static final String KEY_VERSION_COLUMN_INDEX = "index"; + + public static final String KEY_VERSION_COLUMN_VALUE = "value"; + + +} diff --git a/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseHelper.java b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseHelper.java new file mode 100644 index 0000000000..180085b480 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-core2/src/main/java/com/dtstack/flinkx/hbase2/HbaseHelper.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2; + +import com.dtstack.flinkx.authenticate.KerberosUtil; +import com.dtstack.flinkx.util.FileSystemUtil; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.PrivilegedAction; +import java.util.Map; + +/** + * The utility class of HBase + * + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class HbaseHelper { + private static final Logger LOG = LoggerFactory.getLogger(HbaseHelper.class); + + private final static String AUTHENTICATION_TYPE = "Kerberos"; + private final static String KEY_HBASE_SECURITY_AUTHENTICATION = "hbase.security.authentication"; + private final static String KEY_HBASE_SECURITY_AUTHORIZATION = "hbase.security.authorization"; + private final static String KEY_HBASE_SECURITY_AUTH_ENABLE = "hbase.security.auth.enable"; + + public static Connection getHbaseConnection(Map hbaseConfigMap) { + Validate.isTrue(MapUtils.isNotEmpty(hbaseConfigMap), "hbaseConfig不能为空Map结构!"); + + if(openKerberos(hbaseConfigMap)){ + return getConnectionWithKerberos(hbaseConfigMap); + } + + try { + Configuration hConfiguration = getConfig(hbaseConfigMap); + return ConnectionFactory.createConnection(hConfiguration); + } catch (IOException e) { + LOG.error("Get connection fail with config:{}", hbaseConfigMap); + throw new RuntimeException(e); + } + } + + private static Connection getConnectionWithKerberos(Map hbaseConfigMap){ + try { + setKerberosConf(hbaseConfigMap); + UserGroupInformation ugi = getUgi(hbaseConfigMap); + return ugi.doAs(new PrivilegedAction() { + @Override + public Connection run() { + try { + Configuration hConfiguration = getConfig(hbaseConfigMap); + return ConnectionFactory.createConnection(hConfiguration); + } catch (IOException e) { + LOG.error("Get connection fail with config:{}", hbaseConfigMap); + throw new RuntimeException(e); + } + } + }); + } catch (Exception e){ + throw new RuntimeException("Login kerberos error", e); + } + } + + public static UserGroupInformation getUgi(Map hbaseConfigMap) throws IOException{ + String keytabFileName = KerberosUtil.getPrincipalFileName(hbaseConfigMap); + + keytabFileName = KerberosUtil.loadFile(hbaseConfigMap, keytabFileName); + String principal = KerberosUtil.getPrincipal(hbaseConfigMap, keytabFileName); + KerberosUtil.loadKrb5Conf(hbaseConfigMap); + KerberosUtil.refreshConfig(); + + Configuration conf = FileSystemUtil.getConfiguration(hbaseConfigMap, null); + + return KerberosUtil.loginAndReturnUgi(conf, principal, keytabFileName); + } + + public static Configuration getConfig(Map hbaseConfigMap){ + Configuration hConfiguration = HBaseConfiguration.create(); + if (MapUtils.isEmpty(hbaseConfigMap)) { + return hConfiguration; + } + + for (Map.Entry entry : hbaseConfigMap.entrySet()) { + if(entry.getValue() != null && !(entry.getValue() instanceof Map)){ + hConfiguration.set(entry.getKey(), entry.getValue().toString()); + } + } + + return hConfiguration; + } + + public static boolean openKerberos(Map hbaseConfigMap){ + if(AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHORIZATION)) + || AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION)) + || MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTH_ENABLE)){ + LOG.info("open kerberos for hbase."); + return true; + } + + return false; + } + + + /** + * 设置hbase 开启kerberos 连接必要的固定参数 + * @param hbaseConfigMap + */ + public static void setKerberosConf(Map hbaseConfigMap){ + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHORIZATION, AUTHENTICATION_TYPE); + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHENTICATION, AUTHENTICATION_TYPE); + hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTH_ENABLE, true); + } + + public static RegionLocator getRegionLocator(Connection hConnection, String userTable){ + TableName hTableName = TableName.valueOf(userTable); + Admin admin = null; + RegionLocator regionLocator = null; + try { + admin = hConnection.getAdmin(); + HbaseHelper.checkHbaseTable(admin,hTableName); + regionLocator = hConnection.getRegionLocator(hTableName); + } catch (Exception e) { + HbaseHelper.closeRegionLocator(regionLocator); + HbaseHelper.closeAdmin(admin); + HbaseHelper.closeConnection(hConnection); + throw new RuntimeException(e); + } + return regionLocator; + + } + + public static byte[] convertRowkey(String rowkey, boolean isBinaryRowkey) { + if(StringUtils.isBlank(rowkey)) { + return HConstants.EMPTY_BYTE_ARRAY; + } else { + return HbaseHelper.stringToBytes(rowkey, isBinaryRowkey); + } + } + + private static byte[] stringToBytes(String rowkey, boolean isBinaryRowkey) { + if (isBinaryRowkey) { + return Bytes.toBytesBinary(rowkey); + } else { + return Bytes.toBytes(rowkey); + } + } + + + public static void closeConnection(Connection hConnection){ + try { + if(null != hConnection) { + hConnection.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + public static void closeAdmin(Admin admin){ + try { + if(null != admin) { + admin.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + public static void closeRegionLocator(RegionLocator regionLocator){ + try { + if(null != regionLocator) { + regionLocator.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static void checkHbaseTable(Admin admin, TableName table) throws IOException { + if(!admin.tableExists(table)){ + throw new IllegalArgumentException("hbase table " + table + " does not exist."); + } + if(!admin.isTableAvailable(table)){ + throw new RuntimeException("hbase table " + table + " is not available."); + } + if(admin.isTableDisabled(table)){ + throw new RuntimeException("hbase table " + table + " is disabled"); + } + } + + public static void closeBufferedMutator(BufferedMutator bufferedMutator){ + try { + if(null != bufferedMutator){ + bufferedMutator.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/flinkx-hbase2/flinkx-hbase-reader2/pom.xml b/flinkx-hbase2/flinkx-hbase-reader2/pom.xml new file mode 100644 index 0000000000..d283668b9e --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-reader2/pom.xml @@ -0,0 +1,102 @@ + + + + flinkx-hbase2 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hbase-reader2 + + + com.google.guava + guava + 12.0.1 + + + com.dtstack.flinkx + flinkx-hbase-core2 + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + com.data-artisans:* + org.scala-lang:* + org.slf4j:slf4j-api + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.hbase.com.google.common + + + com.google.thirdparty + shade.hbase.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/Hbase2Reader.java b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/Hbase2Reader.java new file mode 100644 index 0000000000..afa87203a8 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/Hbase2Reader.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.reader; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.ReaderConfig; +import com.dtstack.flinkx.hbase2.HbaseConfigConstants; +import com.dtstack.flinkx.hbase2.HbaseConfigKeys; +import com.dtstack.flinkx.reader.BaseDataReader; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * The reader plugin of Hbase + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class Hbase2Reader extends BaseDataReader { + + private static Logger LOG = LoggerFactory.getLogger(Hbase2Reader.class); + + protected List columnName; + protected List columnType; + protected List columnValue; + protected List columnFormat; + protected String encoding; + protected Map hbaseConfig; + protected String startRowkey; + protected String endRowkey; + protected boolean isBinaryRowkey; + protected String tableName; + protected int scanCacheSize; + + public Hbase2Reader(DataTransferConfig config, StreamExecutionEnvironment env) { + super(config, env); + ReaderConfig readerConfig = config.getJob().getContent().get(0).getReader(); + tableName = readerConfig.getParameter().getStringVal(HbaseConfigKeys.KEY_TABLE); + hbaseConfig = (Map) readerConfig.getParameter().getVal(HbaseConfigKeys.KEY_HBASE_CONFIG); + + Map range = (Map) readerConfig.getParameter().getVal(HbaseConfigKeys.KEY_RANGE); + if(range != null) { + startRowkey = (String) range.get(HbaseConfigKeys.KEY_START_ROW_KEY); + endRowkey = (String) range.get(HbaseConfigKeys.KEY_END_ROW_KEY); + isBinaryRowkey = (Boolean) range.get(HbaseConfigKeys.KEY_IS_BINARY_ROW_KEY); + } + + encoding = readerConfig.getParameter().getStringVal(HbaseConfigKeys.KEY_ENCODING); + scanCacheSize = readerConfig.getParameter().getIntVal(HbaseConfigKeys.KEY_SCAN_CACHE_SIZE, HbaseConfigConstants.DEFAULT_SCAN_CACHE_SIZE); + + List columns = readerConfig.getParameter().getColumn(); + if(columns != null && columns.size() > 0) { + columnName = new ArrayList<>(); + columnType = new ArrayList<>(); + columnValue = new ArrayList<>(); + columnFormat = new ArrayList<>(); + for(int i = 0; i < columns.size(); ++i) { + Map sm = (Map) columns.get(i); + columnName.add((String) sm.get("name")); + columnType.add((String) sm.get("type")); + columnValue.add((String) sm.get("value")); + columnFormat.add((String) sm.get("format")); + } + + LOG.info("init column finished"); + } else{ + throw new IllegalArgumentException("column argument error"); + } + } + + @Override + public DataStream readData() { + HbaseInputFormatBuilder builder = new HbaseInputFormatBuilder(); + builder.setDataTransferConfig(dataTransferConfig); + builder.setColumnFormats(columnFormat); + builder.setColumnNames(columnName); + builder.setColumnTypes(columnType); + builder.setColumnValues(columnValue); + builder.setEncoding(encoding); + builder.setEndRowkey(endRowkey); + builder.setHbaseConfig(hbaseConfig); + builder.setStartRowkey(startRowkey); + builder.setIsBinaryRowkey(isBinaryRowkey); + builder.setTableName(tableName); + builder.setBytes(bytes); + builder.setMonitorUrls(monitorUrls); + builder.setScanCacheSize(scanCacheSize); + builder.setMonitorUrls(monitorUrls); + builder.setTestConfig(testConfig); + builder.setLogConfig(logConfig); + + return createInput(builder.finish()); + } + +} diff --git a/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormat.java b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormat.java new file mode 100644 index 0000000000..41efc7623f --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormat.java @@ -0,0 +1,365 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.reader; + +import com.dtstack.flinkx.hbase2.HbaseHelper; +import com.dtstack.flinkx.inputformat.BaseRichInputFormat; +import com.google.common.collect.Maps; +import org.apache.commons.lang.ArrayUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.apache.commons.lang3.time.DateUtils; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.types.Row; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + + +/** + * The InputFormat Implementation used for HbaseReader + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class HbaseInputFormat extends BaseRichInputFormat { + + public static final String KEY_ROW_KEY = "rowkey"; + + protected Map hbaseConfig; + protected String tableName; + protected String startRowkey; + protected String endRowkey; + protected List columnNames; + protected List columnValues; + protected List columnFormats; + protected List columnTypes; + protected boolean isBinaryRowkey; + protected String encoding; + /** + * 客户端每次 rpc fetch 的行数 + */ + protected int scanCacheSize; + private transient Connection connection; + private transient Scan scan; + private transient Table table; + private transient ResultScanner resultScanner; + private transient Result next; + private transient Map nameMaps; + + private boolean openKerberos = false; + + @Override + public void openInputFormat() throws IOException { + super.openInputFormat(); + + LOG.info("HbaseOutputFormat openInputFormat start"); + nameMaps = Maps.newConcurrentMap(); + + connection = HbaseHelper.getHbaseConnection(hbaseConfig); + + LOG.info("HbaseOutputFormat openInputFormat end"); + } + + @Override + public InputSplit[] createInputSplitsInternal(int minNumSplits) throws IOException { + try (Connection connection = HbaseHelper.getHbaseConnection(hbaseConfig)) { + if(HbaseHelper.openKerberos(hbaseConfig)) { + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + return ugi.doAs(new PrivilegedAction() { + @Override + public com.dtstack.flinkx.hbase2.reader.HbaseInputSplit[] run() { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } + }); + } else { + return split(connection, tableName, startRowkey, endRowkey, isBinaryRowkey); + } + } + } + + public HbaseInputSplit[] split(Connection hConn, String tableName, String startKey, String endKey, boolean isBinaryRowkey) { + byte[] startRowkeyByte = HbaseHelper.convertRowkey(startKey, isBinaryRowkey); + byte[] endRowkeyByte = HbaseHelper.convertRowkey(endKey, isBinaryRowkey); + + /* 如果用户配置了 startRowkey 和 endRowkey,需要确保:startRowkey <= endRowkey */ + if (startRowkeyByte.length != 0 && endRowkeyByte.length != 0 + && Bytes.compareTo(startRowkeyByte, endRowkeyByte) > 0) { + throw new IllegalArgumentException("startRowKey can't be bigger than endRowkey"); + } + + RegionLocator regionLocator = HbaseHelper.getRegionLocator(hConn, tableName); + List resultSplits; + try { + Pair regionRanges = regionLocator.getStartEndKeys(); + if (null == regionRanges) { + throw new RuntimeException("Failed to retrieve rowkey ragne"); + } + resultSplits = doSplit(startRowkeyByte, endRowkeyByte, regionRanges); + + LOG.info("HBaseReader split job into {} tasks.", resultSplits.size()); + return resultSplits.toArray(new HbaseInputSplit[resultSplits.size()]); + } catch (Exception e) { + throw new RuntimeException("Failed to split hbase table"); + }finally { + HbaseHelper.closeRegionLocator(regionLocator); + } + } + + private List doSplit(byte[] startRowkeyByte, + byte[] endRowkeyByte, Pair regionRanges) { + + List configurations = new ArrayList<>(); + + for (int i = 0; i < regionRanges.getFirst().length; i++) { + + byte[] regionStartKey = regionRanges.getFirst()[i]; + byte[] regionEndKey = regionRanges.getSecond()[i]; + + // 当前的region为最后一个region + // 如果最后一个region的start Key大于用户指定的userEndKey,则最后一个region,应该不包含在内 + // 注意如果用户指定userEndKey为"",则此判断应该不成立。userEndKey为""表示取得最大的region + boolean isSkip = Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0 + && (endRowkeyByte.length != 0 && (Bytes.compareTo( + regionStartKey, endRowkeyByte) > 0)); + if (isSkip) { + continue; + } + + // 如果当前的region不是最后一个region, + // 用户配置的userStartKey大于等于region的endkey,则这个region不应该含在内 + if ((Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) != 0) + && (Bytes.compareTo(startRowkeyByte, regionEndKey) >= 0)) { + continue; + } + + // 如果用户配置的userEndKey小于等于 region的startkey,则这个region不应该含在内 + // 注意如果用户指定的userEndKey为"",则次判断应该不成立。userEndKey为""表示取得最大的region + if (endRowkeyByte.length != 0 + && (Bytes.compareTo(endRowkeyByte, regionStartKey) <= 0)) { + continue; + } + + String thisStartKey = getStartKey(startRowkeyByte, regionStartKey); + String thisEndKey = getEndKey(endRowkeyByte, regionEndKey); + HbaseInputSplit hbaseInputSplit = new HbaseInputSplit(thisStartKey, thisEndKey); + configurations.add(hbaseInputSplit); + } + + return configurations; + } + + private String getEndKey(byte[] endRowkeyByte, byte[] regionEndKey) { + // 由于之前处理过,所以传入的userStartKey不可能为null + if (endRowkeyByte == null) { + throw new IllegalArgumentException("userEndKey should not be null!"); + } + + byte[] tempEndRowkeyByte; + + if (endRowkeyByte.length == 0) { + tempEndRowkeyByte = regionEndKey; + } else if (Bytes.compareTo(regionEndKey, HConstants.EMPTY_BYTE_ARRAY) == 0) { + // 为最后一个region + tempEndRowkeyByte = endRowkeyByte; + } else { + if (Bytes.compareTo(endRowkeyByte, regionEndKey) > 0) { + tempEndRowkeyByte = regionEndKey; + } else { + tempEndRowkeyByte = endRowkeyByte; + } + } + + return Bytes.toStringBinary(tempEndRowkeyByte); + } + + private String getStartKey(byte[] startRowkeyByte, byte[] regionStarKey) { + // 由于之前处理过,所以传入的userStartKey不可能为null + if (startRowkeyByte == null) { + throw new IllegalArgumentException( + "userStartKey should not be null!"); + } + + byte[] tempStartRowkeyByte; + + if (Bytes.compareTo(startRowkeyByte, regionStarKey) < 0) { + tempStartRowkeyByte = regionStarKey; + } else { + tempStartRowkeyByte = startRowkeyByte; + } + return Bytes.toStringBinary(tempStartRowkeyByte); + } + + @Override + public void openInternal(InputSplit inputSplit) throws IOException { + HbaseInputSplit hbaseInputSplit = (HbaseInputSplit) inputSplit; + byte[] startRow = Bytes.toBytesBinary(hbaseInputSplit.getStartkey()); + byte[] stopRow = Bytes.toBytesBinary(hbaseInputSplit.getEndKey()); + + if(null == connection || connection.isClosed()){ + connection = HbaseHelper.getHbaseConnection(hbaseConfig); + } + + openKerberos = HbaseHelper.openKerberos(hbaseConfig); + + table = connection.getTable(TableName.valueOf(tableName)); + scan = new Scan(); + scan.setStartRow(startRow); + scan.setStopRow(stopRow); + scan.setCaching(scanCacheSize); + resultScanner = table.getScanner(scan); + } + + @Override + public boolean reachedEnd() throws IOException { + next = resultScanner.next(); + return next == null; + } + + @Override + public Row nextRecordInternal(Row row) throws IOException { + row = new Row(columnTypes.size()); + + for (int i = 0; i < columnTypes.size(); ++i) { + String columnType = columnTypes.get(i); + String columnName = columnNames.get(i); + String columnFormat = columnFormats.get(i); + String columnValue = columnValues.get(i); + Object col = null; + byte[] bytes; + + try { + if (StringUtils.isNotEmpty(columnValue)) { + // 常量 + col = convertValueToAssignType(columnType, columnValue, columnFormat); + } else { + if (KEY_ROW_KEY.equals(columnName)) { + bytes = next.getRow(); + } else { + byte [][] arr = nameMaps.get(columnName); + if(arr == null){ + arr = new byte[2][]; + String[] arr1 = columnName.split(":"); + arr[0] = arr1[0].trim().getBytes(StandardCharsets.UTF_8); + arr[1] = arr1[1].trim().getBytes(StandardCharsets.UTF_8); + nameMaps.put(columnName,arr); + } + bytes = next.getValue(arr[0], arr[1]); + } + col = convertBytesToAssignType(columnType, bytes, columnFormat); + } + row.setField(i, col); + } catch(Exception e) { + throw new IOException("Couldn't read data:",e); + } + } + return row; + } + + @Override + public void closeInternal() throws IOException { + HbaseHelper.closeConnection(connection); + } + + public Object convertValueToAssignType(String columnType, String constantValue,String dateformat) throws Exception { + Object column = null; + if(org.apache.commons.lang3.StringUtils.isEmpty(constantValue)) { + return column; + } + + switch (columnType.toUpperCase()) { + case "BOOLEAN": + column = Boolean.valueOf(constantValue); + break; + case "SHORT": + case "INT": + case "LONG": + column = NumberUtils.createBigDecimal(constantValue).toBigInteger(); + break; + case "FLOAT": + case "DOUBLE": + column = new BigDecimal(constantValue); + break; + case "STRING": + column = constantValue; + break; + case "DATE": + column = DateUtils.parseDate(constantValue, new String[]{dateformat}); + break; + default: + throw new IllegalArgumentException("Unsupported columnType: " + columnType); + } + + return column; + } + + public Object convertBytesToAssignType(String columnType, byte[] byteArray,String dateformat) throws Exception { + Object column = null; + if(ArrayUtils.isEmpty(byteArray)) { + return null; + } + String bytesToString = new String(byteArray, encoding); + switch (columnType.toUpperCase(Locale.ENGLISH)) { + case "BOOLEAN": + column = Boolean.valueOf(bytesToString); + break; + case "SHORT": + column = Short.valueOf(bytesToString); + break; + case "INT": + column = Integer.valueOf(bytesToString); + break; + case "LONG": + column = Long.valueOf(bytesToString); + break; + case "FLOAT": + column = Float.valueOf(bytesToString); + break; + case "DOUBLE": + column = Double.valueOf(bytesToString); + break; + case "STRING": + column = bytesToString; + break; + case "BINARY_STRING": + column = Bytes.toStringBinary(byteArray); + break; + case "DATE": + String dateValue = Bytes.toStringBinary(byteArray); + column = DateUtils.parseDate(dateValue, new String[]{dateformat}); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + return column; + } + +} diff --git a/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormatBuilder.java b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormatBuilder.java new file mode 100644 index 0000000000..14a99e40ec --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputFormatBuilder.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.hbase2.reader; + +import com.dtstack.flinkx.hbase2.HbaseConfigConstants; +import com.dtstack.flinkx.inputformat.BaseRichInputFormatBuilder; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.util.Preconditions; + +import java.util.List; +import java.util.Map; + +/** + * The builder of HbaseInputFormat + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class HbaseInputFormatBuilder extends BaseRichInputFormatBuilder { + + private HbaseInputFormat format; + + public HbaseInputFormatBuilder() { + super.format = format = new HbaseInputFormat(); + } + + public void setHbaseConfig(Map hbaseConfig) { + format.hbaseConfig = hbaseConfig; + } + + public void setTableName(String tableName) { + format.tableName = tableName; + } + + public void setStartRowkey(String startRowkey) { + format.startRowkey = startRowkey; + } + + public void setEndRowkey(String endRowkey) { + format.endRowkey = endRowkey; + } + + public void setColumnNames(List columnNames) { + format.columnNames = columnNames; + } + + public void setColumnValues(List columnValues) { + format.columnValues = columnValues; + } + + public void setColumnTypes(List columnTypes) { + format.columnTypes = columnTypes; + } + + public void setIsBinaryRowkey(boolean isBinaryRowkey) { + format.isBinaryRowkey = isBinaryRowkey; + } + + public void setEncoding(String encoding) { + format.encoding = StringUtils.isEmpty(encoding) ? "utf-8" : encoding; + } + + public void setColumnFormats(List columnFormats) { + format.columnFormats = columnFormats; + } + + public void setScanCacheSize(int scanCacheSize) { + format.scanCacheSize = scanCacheSize; + } + + @Override + protected void checkFormat() { + Preconditions.checkNotNull(format.columnTypes); + Preconditions.checkNotNull(format.columnFormats); + Preconditions.checkNotNull(format.columnValues); + Preconditions.checkNotNull(format.columnNames); + + Preconditions.checkArgument(format.scanCacheSize <= HbaseConfigConstants.MAX_SCAN_CACHE_SIZE && format.scanCacheSize >= HbaseConfigConstants.MIN_SCAN_CACHE_SIZE, + "scanCacheSize should be between " + HbaseConfigConstants.MIN_SCAN_CACHE_SIZE + " and " + HbaseConfigConstants.MAX_SCAN_CACHE_SIZE); + + for(int i = 0; i < format.columnTypes.size(); ++i) { + Preconditions.checkArgument(StringUtils.isNotEmpty(format.columnTypes.get(i))); + Preconditions.checkArgument(StringUtils.isNotEmpty(format.columnNames.get(i)) + || StringUtils.isNotEmpty(format.columnTypes.get(i)) ); + } + + if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ + throw new UnsupportedOperationException("This plugin not support restore from failed state"); + } + } +} diff --git a/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputSplit.java b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputSplit.java new file mode 100644 index 0000000000..56343797cc --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-reader2/src/main/java/com/dtstack/flinkx/hbase2/reader/HbaseInputSplit.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.reader; + +import org.apache.flink.core.io.InputSplit; + +/** + * The Class describing each InputSplit of HBase + * + * Company: cmss + * @author wangyulei_yewu@cmss.chinamobile.com + */ +public class HbaseInputSplit implements InputSplit { + + private String startkey; + private String endKey; + + public HbaseInputSplit(String startKey, String endKey) { + this.startkey = startKey; + this.endKey = endKey; + } + + public String getStartkey() { + return startkey; + } + + public String getEndKey() { + return endKey; + } + + @Override + public int getSplitNumber() { + return 0; + } +} diff --git a/flinkx-hbase2/pom.xml b/flinkx-hbase2/pom.xml new file mode 100644 index 0000000000..001e91c782 --- /dev/null +++ b/flinkx-hbase2/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hbase2 + pom + + flinkx-hbase-core2 + flinkx-hbase-reader2 + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 7d85bc00d8..b8ac68013b 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ flinkx-ftp flinkx-odps flinkx-hbase - + flinkx-hbase2 flinkx-phoenix5 flinkx-carbondata flinkx-kudu From 0790141616ca010c4a5a6a73def0455ff3db9aeb Mon Sep 17 00:00:00 2001 From: wangyulei Date: Thu, 24 Jun 2021 16:40:44 +0800 Subject: [PATCH 124/136] =?UTF-8?q?[feat-424][hbase2]=E6=94=AF=E6=8C=81hba?= =?UTF-8?q?se2.x=E7=89=88=E6=9C=AC=E7=9A=84=E5=86=99=E5=85=A5=E5=8A=9F?= =?UTF-8?q?=E8=83=BD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-hbase2/flinkx-hbase-reader2/pom.xml | 6 +- flinkx-hbase2/flinkx-hbase-writer2/pom.xml | 105 ++++ .../flinkx/hbase2/writer/Hbase2Writer.java | 151 +++++ .../hbase2/writer/HbaseOutputFormat.java | 556 ++++++++++++++++++ .../writer/HbaseOutputFormatBuilder.java | 117 ++++ .../writer/function/ConstantFunction.java | 45 ++ .../writer/function/FunctionFactory.java | 52 ++ .../writer/function/FunctionParser.java | 180 ++++++ .../hbase2/writer/function/FunctionTree.java | 80 +++ .../hbase2/writer/function/IFunction.java | 36 ++ .../hbase2/writer/function/Md5Function.java | 34 ++ .../writer/function/StringFunction.java | 32 + flinkx-hbase2/pom.xml | 1 + 13 files changed, 1392 insertions(+), 3 deletions(-) create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/pom.xml create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/Hbase2Writer.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormat.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormatBuilder.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/ConstantFunction.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionFactory.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionParser.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionTree.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/IFunction.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/Md5Function.java create mode 100644 flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/StringFunction.java diff --git a/flinkx-hbase2/flinkx-hbase-reader2/pom.xml b/flinkx-hbase2/flinkx-hbase-reader2/pom.xml index d283668b9e..f6b44e9746 100644 --- a/flinkx-hbase2/flinkx-hbase-reader2/pom.xml +++ b/flinkx-hbase2/flinkx-hbase-reader2/pom.xml @@ -83,14 +83,14 @@ - + - + diff --git a/flinkx-hbase2/flinkx-hbase-writer2/pom.xml b/flinkx-hbase2/flinkx-hbase-writer2/pom.xml new file mode 100644 index 0000000000..99ab947403 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/pom.xml @@ -0,0 +1,105 @@ + + + + flinkx-hbase2 + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hbase-writer2 + + + com.google.guava + guava + 12.0.1 + + + com.dtstack.flinkx + flinkx-hbase-core2 + 1.6 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + + com.google.code.gson:* + com.data-artisans:* + org.scala-lang:* + org.slf4j:slf4j-api + log4j:log4j + ch.qos.logback:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.hbase.com.google.common + + + com.google.thirdparty + shade.hbase.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/Hbase2Writer.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/Hbase2Writer.java new file mode 100644 index 0000000000..4e2974a6c8 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/Hbase2Writer.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.hbase2.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.util.ValueUtil; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.hbase2.HbaseConfigConstants.DEFAULT_WAL_FLAG; +import static com.dtstack.flinkx.hbase2.HbaseConfigConstants.DEFAULT_WRITE_BUFFER_SIZE; +import static com.dtstack.flinkx.hbase2.HbaseConfigKeys.*; + +/** + * The Writer plugin of HBase + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class Hbase2Writer extends BaseDataWriter { + + private String tableName; + private Map hbaseConfig; + private String encoding; + private String nullMode; + private Boolean walFlag; + private long writeBufferSize; + + private List columnTypes; + private List columnNames; + private String rowkeyExpress; + + private Integer versionColumnIndex; + private String versionColumnValue; + + public Hbase2Writer(DataTransferConfig config) { + super(config); + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + + tableName = writerConfig.getParameter().getStringVal(KEY_TABLE); + hbaseConfig = (Map) writerConfig.getParameter().getVal(KEY_HBASE_CONFIG); + encoding = writerConfig.getParameter().getStringVal(KEY_ENCODING); + nullMode = writerConfig.getParameter().getStringVal(KEY_NULL_MODE); + walFlag = writerConfig.getParameter().getBooleanVal(KEY_WAL_FLAG, DEFAULT_WAL_FLAG); + writeBufferSize = writerConfig.getParameter().getLongVal(KEY_WRITE_BUFFER_SIZE, DEFAULT_WRITE_BUFFER_SIZE); + + List columns = writerConfig.getParameter().getColumn(); + if(CollectionUtils.isNotEmpty(columns)) { + columnTypes = new ArrayList<>(); + columnNames = new ArrayList<>(); + for(int i = 0; i < columns.size(); ++i) { + Map sm = (Map) columns.get(i); + columnNames.add((String) sm.get(KEY_COLUMN_NAME)); + columnTypes.add((String) sm.get(KEY_COLUMN_TYPE)); + } + } + + Object rowKeyInfo = writerConfig.getParameter().getStringVal(KEY_ROW_KEY_COLUMN); + rowkeyExpress = buildRowKeyExpress(rowKeyInfo); + + Map versionColumn = (Map) writerConfig.getParameter().getVal(KEY_VERSION_COLUMN); + if(versionColumn != null) { + versionColumnIndex = (Integer) versionColumn.get(KEY_VERSION_COLUMN_INDEX); + versionColumnValue = (String) versionColumn.get(KEY_VERSION_COLUMN_VALUE); + } + } + + /** + * Compatible with old formats + */ + private String buildRowKeyExpress(Object rowKeyInfo){ + if (rowKeyInfo == null){ + return null; + } + + if(rowKeyInfo instanceof String){ + return rowKeyInfo.toString(); + } + + if(!(rowKeyInfo instanceof List)){ + return null; + } + + StringBuilder expressBuilder = new StringBuilder(); + + for (Map item : ((List) rowKeyInfo)) { + Integer index = ValueUtil.getInt(item.get(KEY_ROW_KEY_COLUMN_INDEX)); + if (index != null && index != -1) { + expressBuilder.append(String.format("$(%s)", columnNames.get(index))); + continue; + } + + String value = (String) item.get(KEY_ROW_KEY_COLUMN_VALUE); + if (StringUtils.isNotEmpty(value)) { + expressBuilder.append(value); + } + } + + return expressBuilder.toString(); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + HbaseOutputFormatBuilder builder = new HbaseOutputFormatBuilder(); + builder.setHbaseConfig(hbaseConfig); + builder.setTableName(tableName); + builder.setEncoding(encoding); + builder.setNullMode(nullMode); + builder.setWalFlag(walFlag); + builder.setWriteBufferSize(writeBufferSize); + builder.setColumnNames(columnNames); + builder.setColumnTypes(columnTypes); + builder.setRowkeyExpress(rowkeyExpress); + builder.setVersionColumnIndex(versionColumnIndex); + builder.setVersionColumnValues(versionColumnValue); + builder.setMonitorUrls(monitorUrls); + builder.setErrorRatio(errorRatio); + builder.setErrors(errors); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + + return createOutput(dataSet, builder.finish()); + } +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormat.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormat.java new file mode 100644 index 0000000000..2852f79a21 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormat.java @@ -0,0 +1,556 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.hbase2.HbaseHelper; +import com.dtstack.flinkx.hbase2.writer.function.FunctionParser; +import com.dtstack.flinkx.hbase2.writer.function.FunctionTree; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.util.DateUtil; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.Validate; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.security.PrivilegedAction; +import java.sql.Timestamp; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * The Hbase Implementation of OutputFormat + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class HbaseOutputFormat extends BaseRichOutputFormat { + + protected Map hbaseConfig; + + protected String tableName; + + protected String encoding; + + protected String nullMode; + + protected boolean walFlag; + + protected long writeBufferSize; + + protected List columnTypes; + + protected List columnNames; + + protected String rowkeyExpress; + + protected Integer versionColumnIndex; + + protected String versionColumnValue; + + private transient Connection connection; + + private transient BufferedMutator bufferedMutator; + + private transient FunctionTree functionTree; + + protected List rowKeyColumns = Lists.newArrayList(); + protected List rowKeyColumnIndex = Lists.newArrayList(); + + private transient Map nameMaps; + + private transient Map nameByteMaps ; + + private transient ThreadLocal timeSecondFormatThreadLocal; + + private transient ThreadLocal timeMillisecondFormatThreadLocal; + + private boolean openKerberos = false; + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void openInternal(int taskNumber, int numTasks) throws IOException { + openKerberos = HbaseHelper.openKerberos(hbaseConfig); + if (openKerberos) { + sleepRandomTime(); + + UserGroupInformation ugi = HbaseHelper.getUgi(hbaseConfig); + ugi.doAs(new PrivilegedAction() { + @Override + public Object run() { + openConnection(); + return null; + } + }); + } else { + openConnection(); + } + } + + private void sleepRandomTime() { + try { + Thread.sleep(5000L + (long)(10000 * Math.random())); + } catch (Exception exception) { + LOG.warn("", exception); + } + } + + public void openConnection() { + LOG.info("HbaseOutputFormat configure start"); + nameMaps = Maps.newConcurrentMap(); + nameByteMaps = Maps.newConcurrentMap(); + timeSecondFormatThreadLocal = new ThreadLocal(); + timeMillisecondFormatThreadLocal = new ThreadLocal(); + Validate.isTrue(hbaseConfig != null && hbaseConfig.size() !=0, "hbaseConfig不能为空Map结构!"); + + try { + org.apache.hadoop.conf.Configuration hConfiguration = HbaseHelper.getConfig(hbaseConfig); + connection = ConnectionFactory.createConnection(hConfiguration); + + /** + * 写缓存 + */ + bufferedMutator = connection.getBufferedMutator( + new BufferedMutatorParams(TableName.valueOf(tableName)) + .pool(HTable.getDefaultExecutor(hConfiguration)) + .writeBufferSize(writeBufferSize)); + } catch (Exception e) { + HbaseHelper.closeBufferedMutator(bufferedMutator); + HbaseHelper.closeConnection(connection); + throw new IllegalArgumentException(e); + } + + functionTree = FunctionParser.parse(rowkeyExpress); + rowKeyColumns = FunctionParser.parseRowKeyCol(rowkeyExpress); + for (String rowKeyColumn : rowKeyColumns) { + int index = columnNames.indexOf(rowKeyColumn); + if(index == -1){ + throw new RuntimeException("Can not get row key column from columns:" + rowKeyColumn); + } + rowKeyColumnIndex.add(index); + } + + LOG.info("HbaseOutputFormat configure end"); + } + + @Override + public void writeSingleRecordInternal(Row record) throws WriteRecordException { + int i = 0; + try { + byte[] rowkey = getRowkey(record); + Put put; + if(versionColumnIndex == null) { + put = new Put(rowkey); + if(!walFlag) { + put.setDurability(Durability.SKIP_WAL); + } + } else { + long timestamp = getVersion(record); + put = new Put(rowkey,timestamp); + } + + for (; i < record.getArity(); ++i) { + if(rowKeyColumnIndex.contains(i)){ + continue; + } + + String type = columnTypes.get(i); + String name = columnNames.get(i); + String[] cfAndQualifier = nameMaps.get(name); + byte[][] cfAndQualifierBytes = nameByteMaps.get(name); + if(cfAndQualifier == null || cfAndQualifierBytes == null){ + cfAndQualifier = name.split(":"); + if(cfAndQualifier.length == 2 + && StringUtils.isNotBlank(cfAndQualifier[0]) + && StringUtils.isNotBlank(cfAndQualifier[1])){ + nameMaps.put(name,cfAndQualifier); + cfAndQualifierBytes = new byte[2][]; + cfAndQualifierBytes[0] = Bytes.toBytes(cfAndQualifier[0]); + cfAndQualifierBytes[1] = Bytes.toBytes(cfAndQualifier[1]); + nameByteMaps.put(name,cfAndQualifierBytes); + } else { + throw new IllegalArgumentException("Hbasewriter 中,column 的列配置格式应该是:列族:列名. 您配置的列错误:" + name); + } + } + + ColumnType columnType = ColumnType.getType(type); + byte[] columnBytes = getColumnByte(columnType, record.getField(i)); + //columnBytes 为null忽略这列 + if(null != columnBytes){ + put.addColumn( + cfAndQualifierBytes[0], + cfAndQualifierBytes[1], + columnBytes); + } + } + + bufferedMutator.mutate(put); + } catch(Exception ex) { + if(i < record.getArity()) { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, record), ex, i, record); + } + throw new WriteRecordException(ex.getMessage(), ex); + } + } + + private SimpleDateFormat getSimpleDateFormat(String sign){ + SimpleDateFormat format; + if(ConstantValue.TIME_SECOND_SUFFIX.equals(sign)){ + format = timeSecondFormatThreadLocal.get(); + if(format == null){ + format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + timeSecondFormatThreadLocal.set(format); + } + } else { + format = timeMillisecondFormatThreadLocal.get(); + if(format == null){ + format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss SSS"); + timeMillisecondFormatThreadLocal.set(format); + } + } + + return format; + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nHbaseOutputFormat [" + jobName + "] writeRecord error: when converting field[" + columnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("HbaseWriter"); + } + + private byte[] getRowkey(Row record) throws Exception{ + Map nameValueMap = new HashMap<>((rowKeyColumnIndex.size()<<2)/3); + for (Integer keyColumnIndex : rowKeyColumnIndex) { + nameValueMap.put(columnNames.get(keyColumnIndex), record.getField(keyColumnIndex)); + } + + String rowKeyStr = functionTree.evaluate(nameValueMap); + return rowKeyStr.getBytes(StandardCharsets.UTF_8); + } + + public long getVersion(Row record){ + Integer index = versionColumnIndex.intValue(); + long timestamp; + if(index == null){ + //指定时间作为版本 + timestamp = Long.valueOf(versionColumnValue); + if(timestamp < 0){ + throw new IllegalArgumentException("Illegal timestamp to construct versionClumn: " + timestamp); + } + }else{ + //指定列作为版本,long/doubleColumn直接record.aslong, 其它类型尝试用yyyy-MM-dd HH:mm:ss,yyyy-MM-dd HH:mm:ss SSS去format + if(index >= record.getArity() || index < 0){ + throw new IllegalArgumentException("version column index out of range: " + index); + } + if(record.getField(index) == null){ + throw new IllegalArgumentException("null verison column!"); + } + SimpleDateFormat dfSeconds = getSimpleDateFormat(ConstantValue.TIME_SECOND_SUFFIX); + SimpleDateFormat dfMs = getSimpleDateFormat(ConstantValue.TIME_MILLISECOND_SUFFIX); + Object column = record.getField(index); + if(column instanceof Long){ + Long longValue = (Long) column; + timestamp = longValue; + } else if (column instanceof Double){ + Double doubleValue = (Double) column; + timestamp = doubleValue.longValue(); + } else if (column instanceof String){ + Date date; + try{ + + date = dfMs.parse((String) column); + }catch (ParseException e){ + try { + date = dfSeconds.parse((String) column); + } catch (ParseException e1) { + LOG.info(String.format("您指定第[%s]列作为hbase写入版本,但在尝试用yyyy-MM-dd HH:mm:ss 和 yyyy-MM-dd HH:mm:ss SSS 去解析为Date时均出错,请检查并修改",index)); + throw new RuntimeException(e1); + } + } + timestamp = date.getTime(); + } else if (column instanceof Date) { + timestamp = ((Date) column).getTime(); + } else { + throw new RuntimeException("rowkey类型不兼容: " + column.getClass()); + } + } + return timestamp; + } + + public byte[] getValueByte(ColumnType columnType, String value){ + byte[] bytes; + if(value != null){ + switch (columnType) { + case INT: + bytes = Bytes.toBytes(Integer.parseInt(value)); + break; + case LONG: + bytes = Bytes.toBytes(Long.parseLong(value)); + break; + case DOUBLE: + bytes = Bytes.toBytes(Double.parseDouble(value)); + break; + case FLOAT: + bytes = Bytes.toBytes(Float.parseFloat(value)); + break; + case SHORT: + bytes = Bytes.toBytes(Short.parseShort(value)); + break; + case BOOLEAN: + bytes = Bytes.toBytes(Boolean.parseBoolean(value)); + break; + case STRING: + bytes = value.getBytes(Charset.forName(encoding)); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + }else{ + bytes = HConstants.EMPTY_BYTE_ARRAY; + } + return bytes; + } + + public byte[] getColumnByte(ColumnType columnType, Object column){ + byte[] bytes; + if(column != null){ + switch (columnType) { + case INT: + bytes = intToBytes(column); + break; + case LONG: + bytes = longToBytes(column); + break; + case DOUBLE: + bytes = doubleToBytes(column); + break; + case FLOAT: + bytes = floatToBytes(column); + break; + case SHORT: + bytes = shortToBytes(column); + break; + case BOOLEAN: + bytes = boolToBytes(column); + break; + case STRING: + String stringValue; + if (column instanceof Timestamp){ + SimpleDateFormat fm = DateUtil.getDateTimeFormatter(); + stringValue = fm.format(column); + }else { + stringValue = String.valueOf(column); + } + bytes = this.getValueByte(columnType, stringValue); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + } else { + switch (nullMode.toUpperCase()){ + case "SKIP": + bytes = null; + break; + case "EMPTY": + bytes = HConstants.EMPTY_BYTE_ARRAY; + break; + default: + throw new IllegalArgumentException("Unsupported null mode: " + nullMode); + } + } + return bytes; + } + + private byte[] intToBytes(Object column) { + Integer intValue = null; + if(column instanceof Integer) { + intValue = (Integer) column; + } else if(column instanceof Long) { + intValue = ((Long) column).intValue(); + } else if(column instanceof Double) { + intValue = ((Double) column).intValue(); + } else if(column instanceof Float) { + intValue = ((Float) column).intValue(); + } else if(column instanceof Short) { + intValue = ((Short) column).intValue(); + } else if(column instanceof Boolean) { + intValue = (Boolean) column ? 1 : 0; + } else if(column instanceof String) { + intValue = Integer.valueOf((String) column); + } else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to INT"); + } + + return Bytes.toBytes(intValue); + } + + private byte[] longToBytes(Object column) { + Long longValue = null; + if(column instanceof Integer) { + longValue = ((Integer)column).longValue(); + } else if(column instanceof Long) { + longValue = (Long) column; + } else if(column instanceof Double) { + longValue = ((Double) column).longValue(); + } else if(column instanceof Float) { + longValue = ((Float) column).longValue(); + } else if(column instanceof Short) { + longValue = ((Short) column).longValue(); + } else if(column instanceof Boolean) { + longValue = (Boolean) column ? 1L : 0L; + } else if(column instanceof String) { + longValue = Long.valueOf((String) column); + }else if (column instanceof Timestamp){ + longValue = ((Timestamp) column).getTime(); + }else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to LONG"); + } + + return Bytes.toBytes(longValue); + } + + private byte[] doubleToBytes(Object column) { + Double doubleValue = null; + if(column instanceof Integer) { + doubleValue = ((Integer)column).doubleValue(); + } else if(column instanceof Long) { + doubleValue = ((Long) column).doubleValue(); + } else if(column instanceof Double) { + doubleValue = (Double) column; + } else if(column instanceof Float) { + doubleValue = ((Float) column).doubleValue(); + } else if(column instanceof Short) { + doubleValue = ((Short) column).doubleValue(); + } else if(column instanceof Boolean) { + doubleValue = (Boolean) column ? 1.0 : 0.0; + } else if(column instanceof String) { + doubleValue = Double.valueOf((String) column); + } else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to DOUBLE"); + } + + return Bytes.toBytes(doubleValue); + } + + private byte[] floatToBytes(Object column) { + Float floatValue = null; + if(column instanceof Integer) { + floatValue = ((Integer)column).floatValue(); + } else if(column instanceof Long) { + floatValue = ((Long) column).floatValue(); + } else if(column instanceof Double) { + floatValue = ((Double) column).floatValue(); + } else if(column instanceof Float) { + floatValue = (Float) column; + } else if(column instanceof Short) { + floatValue = ((Short) column).floatValue(); + } else if(column instanceof Boolean) { + floatValue = (Boolean) column ? 1.0f : 0.0f; + } else if(column instanceof String) { + floatValue = Float.valueOf((String) column); + } else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to DOUBLE"); + } + + return Bytes.toBytes(floatValue); + } + + private byte[] shortToBytes(Object column) { + Short shortValue = null; + if(column instanceof Integer) { + shortValue = ((Integer)column).shortValue(); + } else if(column instanceof Long) { + shortValue = ((Long) column).shortValue(); + } else if(column instanceof Double) { + shortValue = ((Double) column).shortValue(); + } else if(column instanceof Float) { + shortValue = ((Float) column).shortValue(); + } else if(column instanceof Short) { + shortValue = (Short) column; + } else if(column instanceof Boolean) { + shortValue = (Boolean) column ? (short) 1 : (short) 0 ; + } else if(column instanceof String) { + shortValue = Short.valueOf((String) column); + } else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to SHORT"); + } + return Bytes.toBytes(shortValue); + } + + private byte[] boolToBytes(Object column) { + Boolean booleanValue = null; + if(column instanceof Integer) { + booleanValue = (Integer) column != 0; + } else if(column instanceof Long) { + booleanValue = (Long) column != 0L; + } else if(column instanceof Double) { + booleanValue = new Double(0.0).compareTo((Double) column) != 0; + } else if(column instanceof Float) { + booleanValue = new Float(0.0f).compareTo((Float) column) != 0; + } else if(column instanceof Short) { + booleanValue = (Short) column != 0; + } else if(column instanceof Boolean) { + booleanValue = (Boolean) column; + } else if(column instanceof String) { + booleanValue = Boolean.valueOf((String)column); + } else { + throw new RuntimeException("Can't convert from " + column.getClass() + " to SHORT"); + } + + return Bytes.toBytes(booleanValue); + } + + @Override + public void closeInternal() throws IOException { + if (null != timeSecondFormatThreadLocal) { + timeSecondFormatThreadLocal.remove(); + } + + if (null != timeMillisecondFormatThreadLocal) { + timeMillisecondFormatThreadLocal.remove(); + } + + HbaseHelper.closeBufferedMutator(bufferedMutator); + HbaseHelper.closeConnection(connection); + } + +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormatBuilder.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormatBuilder.java new file mode 100644 index 0000000000..0b6b52cc6e --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/HbaseOutputFormatBuilder.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer; + +import com.dtstack.flinkx.hbase2.HbaseConfigConstants; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; +import com.google.common.base.Preconditions; +import org.apache.commons.lang.StringUtils; + +import java.util.List; +import java.util.Map; + +/** + * The Builder class of HbaseOutputFormatBuilder + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class HbaseOutputFormatBuilder extends BaseRichOutputFormatBuilder { + + private HbaseOutputFormat format; + + public HbaseOutputFormatBuilder() { + super.format = format = new HbaseOutputFormat(); + } + + public void setTableName(String tableName) { + format.tableName = tableName; + } + + public void setHbaseConfig(Map hbaseConfig) { + format.hbaseConfig = hbaseConfig; + } + + public void setColumnTypes(List columnTypes) { + format.columnTypes = columnTypes; + } + + public void setColumnNames(List columnNames) { + format.columnNames = columnNames; + } + + public void setRowkeyExpress(String rowkeyExpress) { + format.rowkeyExpress = rowkeyExpress; + } + + public void setVersionColumnIndex(Integer versionColumnIndex) { + format.versionColumnIndex = versionColumnIndex; + } + + public void setVersionColumnValues(String versionColumnValue) { + format.versionColumnValue = versionColumnValue; + } + + public void setEncoding(String encoding) { + if(StringUtils.isEmpty(encoding)) { + format.encoding = HbaseConfigConstants.DEFAULT_ENCODING; + } else { + format.encoding = encoding; + } + } + + public void setWriteBufferSize(Long writeBufferSize) { + if(writeBufferSize == null || writeBufferSize.longValue() == 0L) { + format.writeBufferSize = HbaseConfigConstants.DEFAULT_WRITE_BUFFER_SIZE; + } else { + format.writeBufferSize = writeBufferSize; + } + } + + public void setNullMode(String nullMode) { + if(StringUtils.isEmpty(nullMode)) { + format.nullMode = HbaseConfigConstants.DEFAULT_NULL_MODE; + } else { + format.nullMode = nullMode; + } + } + + public void setWalFlag(Boolean walFlag) { + if(walFlag == null) { + format.walFlag = false; + } else { + format.walFlag = walFlag; + } + } + + @Override + protected void checkFormat() { + Preconditions.checkArgument(StringUtils.isNotEmpty(format.tableName)); + Preconditions.checkNotNull(format.hbaseConfig); + Preconditions.checkNotNull(format.columnNames); + Preconditions.checkNotNull(format.columnTypes); + Preconditions.checkNotNull(format.rowkeyExpress); + + if (format.getRestoreConfig() != null && format.getRestoreConfig().isRestore()){ + throw new UnsupportedOperationException("This plugin not support restore from failed state"); + } + + notSupportBatchWrite("HbaseWriter"); + } +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/ConstantFunction.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/ConstantFunction.java new file mode 100644 index 0000000000..983a38870f --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/ConstantFunction.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.hbase2.writer.function; + +/** + * @author jiangbo + * @date 2019/7/25 + */ +public class ConstantFunction implements IFunction { + + private Object value; + + public ConstantFunction() { + } + + public ConstantFunction(Object value) { + this.value = value; + } + + @Override + public String evaluate(Object val) { + return String.valueOf(value); + } + + public void setValue(Object value) { + this.value = value; + } +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionFactory.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionFactory.java new file mode 100644 index 0000000000..21390d7c97 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer.function; + +import org.apache.commons.lang.StringUtils; + +/** + * @company: www.dtstack.com + * @author: toutian + * @create: 2019/7/23 + */ +public class FunctionFactory { + + public static IFunction createFuntion(String functionName) { + if (StringUtils.isBlank(functionName)) { + throw new UnsupportedOperationException("function name can't be null!"); + } + + IFunction function = null; + switch (functionName.toUpperCase()) { + case "MD5": + function = new Md5Function(); + break; + case "STRING": + function = new StringFunction(); + break; + case "CONSTANT": + function = new ConstantFunction(); + break; + default: + throw new UnsupportedOperationException(String.format("function name[%s] don't exist!", functionName)); + } + return function; + } + +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionParser.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionParser.java new file mode 100644 index 0000000000..e28b6e54b4 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionParser.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.hbase2.writer.function; + + +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * @author jiangbo + * @date 2019/7/24 + */ +public class FunctionParser { + + private static final String COL_REGEX = "\\$\\([^\\(\\)]+?\\)"; + private static Pattern COL_PATTERN = Pattern.compile(COL_REGEX); + + private static String LEFT_KUO = "("; + private static String RIGHT_KUO = ")"; + private static String DELIM = "_"; + + public static List parseRowKeyCol(String express){ + List columnNames = new ArrayList<>(); + Matcher matcher = COL_PATTERN.matcher(express); + while (matcher.find()) { + String colExpre = matcher.group(); + String col = colExpre.substring(colExpre.indexOf(LEFT_KUO)+1, colExpre.indexOf(RIGHT_KUO)); + columnNames.add(col); + } + + return columnNames; + } + + public static FunctionTree parse(String express){ + if(StringUtils.isEmpty(express)){ + throw new RuntimeException("Row key column express can not be null"); + } + + if(StringUtils.isEmpty(express.trim())){ + throw new RuntimeException("Row key column express can not be empty"); + } + + express = replaceColToStringFunc(express); + + FunctionTree root = new FunctionTree(); + root.setFunction(new StringFunction()); + + if(express.startsWith(DELIM)){ + FunctionTree child = new FunctionTree(); + child.setFunction(new ConstantFunction("")); + root.addInputFunction(child); + express = express.substring(1); + } + + parseFunction(root, express); + + if(express.endsWith(DELIM)){ + FunctionTree child = new FunctionTree(); + child.setFunction(new ConstantFunction("")); + root.addInputFunction(child); + } + + return root; + } + + private static void parseFunction(FunctionTree root, String express){ + int leftBracketsIndex = express.indexOf("("); + if (leftBracketsIndex == -1){ + root.setColumnName(express); + } else { + int rightBracketsIndex = findRightBrackets(leftBracketsIndex, express); + if(rightBracketsIndex == -1){ + throw new IllegalArgumentException("Illegal express:" + express); + } + + String value = express.substring(0, leftBracketsIndex); + if(StringUtils.isEmpty(value)){ + throw new IllegalArgumentException("Parse function from express fail,function name can not be empty"); + } + + if(value.startsWith(DELIM)){ + value = value.substring(1); + } + + String[] splits = value.split(DELIM); + for (int i = 0; i < splits.length-1; i++) { + FunctionTree child = new FunctionTree(); + child.setFunction(new ConstantFunction(splits[i])); + root.addInputFunction(child); + } + + FunctionTree child = new FunctionTree(); + child.setFunction(FunctionFactory.createFuntion(splits[splits.length -1 ])); + root.addInputFunction(child); + + String subExpress = express.substring(leftBracketsIndex+1, rightBracketsIndex); + parseFunction(child, subExpress); + + String leftExpress = express.substring(rightBracketsIndex+1); + processLeftExpress(leftExpress, root); + } + } + + private static void processLeftExpress(String leftExpress, FunctionTree root){ + if(StringUtils.isEmpty(leftExpress)){ + return; + } + + if (leftExpress.contains(LEFT_KUO)) { + parseFunction(root, leftExpress); + } else { + if(leftExpress.startsWith(DELIM)){ + leftExpress = leftExpress.substring(1); + } + + if(StringUtils.isEmpty(leftExpress)){ + return; + } + + String[] splits = leftExpress.split(DELIM); + for (int i = 0; i < splits.length; i++) { + FunctionTree child = new FunctionTree(); + child.setFunction(new ConstantFunction(splits[i])); + root.addInputFunction(child); + } + } + } + + private static int findRightBrackets(int startIndex, String express){ + boolean hasMeddleBrackets = false; + for (int i = startIndex+1; i < express.length(); i++) { + char c = express.charAt(i); + if('(' == c){ + hasMeddleBrackets = true; + } + + if(')' == c){ + if(hasMeddleBrackets){ + hasMeddleBrackets = false; + } else { + return i; + } + } + } + + return -1; + } + + public static String replaceColToStringFunc(String express){ + Matcher matcher = COL_PATTERN.matcher(express); + while (matcher.find()) { + String columnExpress = matcher.group(); + String column = columnExpress.substring(2, columnExpress.length() - 1); + express = express.replace(columnExpress, String.format("string(%s)", column)); + } + + return express; + } +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionTree.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionTree.java new file mode 100644 index 0000000000..de4c1bc45e --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/FunctionTree.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flinkx.hbase2.writer.function; + +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * @author jiangbo + * @date 2019/7/24 + */ +public class FunctionTree { + + private String columnName; + + private IFunction function; + + private List inputFunctions = Lists.newArrayList(); + + public String evaluate(Map nameValueMap) throws Exception{ + if(StringUtils.isNotEmpty(columnName) && MapUtils.isNotEmpty(nameValueMap)){ + return function.evaluate(nameValueMap.get(columnName)); + } + + if(CollectionUtils.isNotEmpty(inputFunctions)){ + List subTaskVal = new ArrayList<>(); + for (FunctionTree inputFunction : inputFunctions) { + subTaskVal.add(inputFunction.evaluate(nameValueMap)); + } + + return function.evaluate(StringUtils.join(subTaskVal, "_")); + } else { + return function.evaluate(null); + } + } + + public void addInputFunction(FunctionTree inputFunction){ + inputFunctions.add(inputFunction); + } + + public String getColumnName() { + return columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public IFunction getFunction() { + return function; + } + + public void setFunction(IFunction function) { + this.function = function; + } + +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/IFunction.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/IFunction.java new file mode 100644 index 0000000000..5044086a76 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/IFunction.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer.function; + +/** + * @company: www.dtstack.com + * @author: toutian + * @create: 2019/7/23 + */ +public interface IFunction { + + /** + * 具体的计算方法 + * + * @param val 输入参数 + * @return 计算结果 + * @throws Exception 捕获的异常,异常类型不确定 + */ + String evaluate(Object val) throws Exception; +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/Md5Function.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/Md5Function.java new file mode 100644 index 0000000000..17c18a8ede --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/Md5Function.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer.function; + +import com.dtstack.flinkx.util.Md5Util; + +/** + * @company: www.dtstack.com + * @author: toutian + * @create: 2019/7/23 + */ +public class Md5Function implements IFunction { + + @Override + public String evaluate(Object str) throws Exception{ + return Md5Util.getMd5(str.toString()); + } +} diff --git a/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/StringFunction.java b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/StringFunction.java new file mode 100644 index 0000000000..24afe1b8e6 --- /dev/null +++ b/flinkx-hbase2/flinkx-hbase-writer2/src/main/java/com/dtstack/flinkx/hbase2/writer/function/StringFunction.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.hbase2.writer.function; + +/** + * @company: www.dtstack.com + * @author: toutian + * @create: 2019/7/23 + */ +public class StringFunction implements IFunction { + + @Override + public String evaluate(Object str) { + return str.toString(); + } +} diff --git a/flinkx-hbase2/pom.xml b/flinkx-hbase2/pom.xml index 001e91c782..d55bcc0443 100644 --- a/flinkx-hbase2/pom.xml +++ b/flinkx-hbase2/pom.xml @@ -14,6 +14,7 @@ flinkx-hbase-core2 flinkx-hbase-reader2 + flinkx-hbase-writer2 From e8bc40fa7a3130dedfe3f08e9f4b5e2c38bc371f Mon Sep 17 00:00:00 2001 From: chenyishuai Date: Thu, 25 Nov 2021 10:07:18 +0800 Subject: [PATCH 125/136] =?UTF-8?q?[feat-543][core]=20core=E6=A8=A1?= =?UTF-8?q?=E5=9D=97=E5=A2=9E=E5=8A=A0=E6=94=AF=E6=8C=81s3=E5=8D=8F?= =?UTF-8?q?=E8=AE=AE,=20=E4=B8=BA=E6=89=A9=E5=B1=95=E5=AF=B9=E8=B1=A1?= =?UTF-8?q?=E5=AD=98=E5=82=A8=E5=8A=9F=E8=83=BD=E5=81=9A=E9=93=BA=E5=9E=AB?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flinkx/constants/ConstantValue.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index b996539e7f..d21116dbff 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -47,6 +47,7 @@ public class ConstantValue { public static final String KEY_HTTP = "http"; + public static final String PROTOCOL_S3A = "s3a://"; public static final String PROTOCOL_HTTP = "http://"; public static final String PROTOCOL_HTTPS = "https://"; public static final String PROTOCOL_HDFS = "hdfs://"; From 67d0655235680763f31fd553ce3a5fa3f2b87f93 Mon Sep 17 00:00:00 2001 From: chenyishuai Date: Tue, 7 Dec 2021 10:06:42 +0800 Subject: [PATCH 126/136] =?UTF-8?q?[feat-553][oss]=20=E5=A2=9E=E5=8A=A0?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0oss-writer?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/constants/ConstantValue.java | 1 + flinkx-oss/flinkx-oss-core/pom.xml | 133 +++++++ .../com/dtstack/flinkx/oss/ECompressType.java | 98 +++++ .../com/dtstack/flinkx/oss/OssConfigKeys.java | 63 +++ .../java/com/dtstack/flinkx/oss/OssUtil.java | 202 ++++++++++ .../com/dtstack/flinkx/oss/util/StrUtil.java | 185 +++++++++ .../apache/hadoop/hive/shims/ShimLoader.java | 103 +++++ flinkx-oss/flinkx-oss-writer/pom.xml | 148 +++++++ .../oss/writer/BaseOssOutputFormat.java | 361 ++++++++++++++++++ .../oss/writer/OssOutputFormatBuilder.java | 101 +++++ .../oss/writer/OssTextOutputFormat.java | 257 +++++++++++++ .../dtstack/flinkx/oss/writer/OssWriter.java | 173 +++++++++ flinkx-oss/pom.xml | 28 ++ pom.xml | 2 + 14 files changed, 1855 insertions(+) create mode 100644 flinkx-oss/flinkx-oss-core/pom.xml create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java create mode 100644 flinkx-oss/flinkx-oss-writer/pom.xml create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java create mode 100644 flinkx-oss/pom.xml diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index b996539e7f..d21116dbff 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -47,6 +47,7 @@ public class ConstantValue { public static final String KEY_HTTP = "http"; + public static final String PROTOCOL_S3A = "s3a://"; public static final String PROTOCOL_HTTP = "http://"; public static final String PROTOCOL_HTTPS = "https://"; public static final String PROTOCOL_HDFS = "hdfs://"; diff --git a/flinkx-oss/flinkx-oss-core/pom.xml b/flinkx-oss/flinkx-oss-core/pom.xml new file mode 100644 index 0000000000..50f7fcc952 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/pom.xml @@ -0,0 +1,133 @@ + + + + flinkx-oss + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss-core + + + + com.amazonaws + aws-java-sdk-s3 + 1.11.689 + + + + com.amazonaws + aws-java-sdk-dynamodb + 1.11.689 + + + + commons-lang + commons-lang + 2.6 + provided + + + + org.apache.hadoop + hadoop-aws + 3.1.0 + + + com.amazonaws + aws-java-sdk-bundle + + + + + + org.apache.hive + hive-exec + ${hive.version} + + + calcite-core + org.apache.calcite + + + calcite-avatica + org.apache.calcite + + + derby + org.apache.derby + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + org.apache.hive + hive-serde + ${hive.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-yarn-api + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + parquet-hadoop + org.apache.parquet + 1.8.3 + + + org.xerial.snappy + snappy-java + + + + + + org.xerial.snappy + snappy-java + 1.1.4 + + + + \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java new file mode 100644 index 0000000000..761790580a --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +import org.apache.commons.lang.StringUtils; + +/** + * @author wangyulei + * @date 2021-06-29 + */ +public enum ECompressType { + + /** + * text file + */ + TEXT_GZIP("GZIP", "text", ".gz", 0.331F), + TEXT_BZIP2("BZIP2", "text", ".bz2", 0.259F), + TEXT_LZO("LZO", "text", ".lzo", 1.0F), + TEXT_NONE("NONE", "text", "", 0.637F), + + /** + * orc file + */ + ORC_SNAPPY("SNAPPY", "orc", ".snappy", 0.233F), + ORC_GZIP("GZIP", "orc", ".gz", 1.0F), + ORC_BZIP("BZIP", "orc", ".bz", 1.0F), + ORC_LZ4("LZ4", "orc", ".lz4", 1.0F), + ORC_NONE("NONE", "orc", "", 0.233F), + + /** + * parquet file + */ + PARQUET_SNAPPY("SNAPPY", "parquet", ".snappy", 0.274F), + PARQUET_GZIP("GZIP", "parquet", ".gz", 1.0F), + PARQUET_LZO("LZO", "parquet", ".lzo", 1.0F), + PARQUET_NONE("NONE", "parquet", "", 1.0F); + + private String type; + + private String fileType; + + private String suffix; + + private float deviation; + + ECompressType(String type, String fileType, String suffix, float deviation) { + this.type = type; + this.fileType = fileType; + this.suffix = suffix; + this.deviation = deviation; + } + + public static ECompressType getByTypeAndFileType(String type, String fileType){ + if (StringUtils.isEmpty(type)) { + type = "NONE"; + } + + for (ECompressType value : ECompressType.values()) { + if (value.getType().equalsIgnoreCase(type) && value.getFileType().equalsIgnoreCase(fileType)){ + return value; + } + } + + throw new IllegalArgumentException("No enum constant " + type); + } + + public String getType() { + return type; + } + + public String getFileType() { + return fileType; + } + + public String getSuffix() { + return suffix; + } + + public float getDeviation() { + return deviation; + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java new file mode 100644 index 0000000000..cb16edac70 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +/** + * @author wangyulei + * @date 2021-06-28 + */ +public class OssConfigKeys { + public static final String KEY_FIELD_DELIMITER = "fieldDelimiter"; + + public static final String KEY_ACCESS_KEY = "accessKey"; + + public static final String KEY_SECRET_KEY = "secretKey"; + + public static final String KEY_ENDPOINT = "endpoint"; + + public static final String KEY_PATH = "path"; + + public static final String KEY_FILTER = "filterRegex"; + + public static final String KEY_FILE_TYPE = "fileType"; + + public static final String KEY_WRITE_MODE = "writeMode"; + + public static final String KEY_FULL_COLUMN_NAME_LIST = "fullColumnName"; + + public static final String KEY_FULL_COLUMN_TYPE_LIST = "fullColumnType"; + + public static final String KEY_COLUMN_NAME = "name"; + + public static final String KEY_COLUMN_TYPE = "type"; + + public static final String KEY_COMPRESS = "compress"; + + public static final String KEY_FILE_NAME = "fileName"; + + public static final String KEY_ENCODING = "encoding"; + + public static final String KEY_ROW_GROUP_SIZE = "rowGroupSize"; + + public static final String KEY_MAX_FILE_SIZE = "maxFileSize"; + + public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + + public static final String KEY_ENABLE_DICTIONARY = "enableDictionary"; +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java new file mode 100644 index 0000000000..bbc3d1667a --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +import com.dtstack.flinkx.enums.ColumnType; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.io.*; +import org.apache.parquet.io.api.Binary; + + +/** + * Utilities for OssReader and OssWriter + * + * @author wangyulei + * @date 2021-06-29 + */ +public class OssUtil { + + public static final String NULL_VALUE = "\\N"; + + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; + + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; + + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; + + public static Object getWritableValue(Object writable) { + Class clz = writable.getClass(); + Object ret = null; + + if (clz == IntWritable.class) { + ret = ((IntWritable) writable).get(); + } else if (clz == Text.class) { + ret = ((Text) writable).toString(); + } else if (clz == LongWritable.class) { + ret = ((LongWritable) writable).get(); + } else if (clz == ByteWritable.class) { + ret = ((ByteWritable) writable).get(); + } else if (clz == DateWritable.class) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof DoubleWritable){ + ret = ((DoubleWritable) writable).get(); + } else if (writable instanceof TimestampWritable){ + ret = ((TimestampWritable) writable).getTimestamp(); + } else if (writable instanceof DateWritable){ + ret = ((DateWritable) writable).get(); + } else if (writable instanceof FloatWritable){ + ret = ((FloatWritable) writable).get(); + } else if (writable instanceof BooleanWritable){ + ret = ((BooleanWritable) writable).get(); + } else { + ret = writable.toString(); + } + return ret; + } + + public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) { + ObjectInspector objectInspector = null; + switch (columnType) { + case TINYINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Byte.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case SMALLINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Short.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case INT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BIGINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case FLOAT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Float.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DOUBLE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Double.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DECIMAL: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(HiveDecimalWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case TIMESTAMP: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Timestamp.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DATE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Date.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case STRING: + case VARCHAR: + case CHAR: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(String.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BOOLEAN: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Boolean.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BINARY: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(BytesWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + default: + throw new IllegalArgumentException("You should not be here"); + } + return objectInspector; + } + + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision){ + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static byte[] longToByteArray(long data){ + long nano = data * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return dst; + } + + private static byte[] getBytes(long i) { + byte[] bytes=new byte[8]; + bytes[0] = (byte)((i >> 56) & 0xFF); + bytes[1] = (byte)((i >> 48) & 0xFF); + bytes[2] = (byte)((i >> 40) & 0xFF); + bytes[3] = (byte)((i >> 32) & 0xFF); + bytes[4] = (byte)((i >> 24) & 0xFF); + bytes[5] = (byte)((i >> 16) & 0xFF); + bytes[6] = (byte)((i >> 8) & 0xFF); + bytes[7] = (byte)(i & 0xFF); + return bytes; + } + + /** + * @param bytes + */ + private static void flip(byte[] bytes) { + for (int i = 0, j = bytes.length-1; i < j; i++, j--) { + byte t = bytes[i]; + bytes[i] = bytes[j]; + bytes[j] = t; + } + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java new file mode 100644 index 0000000000..68638c2952 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.util; + +/** + * + * @author wangyulei + * @date 2021-06-29 + */ +public class StrUtil { + /** + *

Check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.startsWith(null, null)      = true
+     * StringUtils.startsWith(null, "abc")     = false
+     * StringUtils.startsWith("abcdef", null)  = false
+     * StringUtils.startsWith("abcdef", "abc") = true
+     * StringUtils.startsWith("ABCDEF", "abc") = false
+     * 
+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case sensitive, or + * both null + * @since 2.4 + */ + public static boolean startsWith(String str, String prefix) { + return startsWith(str, prefix, false); + } + + /** + *

Case insensitive check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.startsWithIgnoreCase(null, null)      = true
+     * StringUtils.startsWithIgnoreCase(null, "abc")     = false
+     * StringUtils.startsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.startsWithIgnoreCase("abcdef", "abc") = true
+     * StringUtils.startsWithIgnoreCase("ABCDEF", "abc") = true
+     * 
+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case insensitive, or + * both null + * @since 2.4 + */ + public static boolean startsWithIgnoreCase(String str, String prefix) { + return startsWith(str, prefix, true); + } + + /** + *

Check if a String starts with a specified prefix (optionally case insensitive).

+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + */ + private static boolean startsWith(String str, String prefix, boolean ignoreCase) { + if (str == null || prefix == null) { + return (str == null && prefix == null); + } + if (prefix.length() > str.length()) { + return false; + } + return str.regionMatches(ignoreCase, 0, prefix, 0, prefix.length()); + } + + /** + *

Check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.endsWith(null, null)      = true
+     * StringUtils.endsWith(null, "def")     = false
+     * StringUtils.endsWith("abcdef", null)  = false
+     * StringUtils.endsWith("abcdef", "def") = true
+     * StringUtils.endsWith("ABCDEF", "def") = false
+     * StringUtils.endsWith("ABCDEF", "cde") = false
+     * 
+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case sensitive, or + * both null + * @since 2.4 + */ + public static boolean endsWith(String str, String suffix) { + return endsWith(str, suffix, false); + } + + /** + *

Case insensitive check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.endsWithIgnoreCase(null, null)      = true
+     * StringUtils.endsWithIgnoreCase(null, "def")     = false
+     * StringUtils.endsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.endsWithIgnoreCase("abcdef", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "cde") = false
+     * 
+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case insensitive, or + * both null + * @since 2.4 + */ + public static boolean endsWithIgnoreCase(String str, String suffix) { + return endsWith(str, suffix, true); + } + + /** + *

Check if a String ends with a specified suffix (optionally case insensitive).

+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + */ + private static boolean endsWith(String str, String suffix, boolean ignoreCase) { + if (str == null || suffix == null) { + return (str == null && suffix == null); + } + if (suffix.length() > str.length()) { + return false; + } + int strOffset = str.length() - suffix.length(); + return str.regionMatches(ignoreCase, strOffset, suffix, 0, suffix.length()); + } + + public static Boolean parseBoolean(String str) { + if (null == str || "null".equalsIgnoreCase(str)) { + return Boolean.FALSE; + } + + if ("1".equals(str)) { + return Boolean.TRUE; + } else if ("0".equals(str)) { + return Boolean.FALSE; + } else { + return Boolean.parseBoolean(str); + } + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java b/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java new file mode 100644 index 0000000000..11c91dfb2f --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java @@ -0,0 +1,103 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by FernFlower decompiler) +// + +package org.apache.hadoop.hive.shims; + +import org.apache.hadoop.util.VersionInfo; +import org.apache.log4j.AppenderSkeleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public abstract class ShimLoader { + private static final Logger LOG = LoggerFactory.getLogger(ShimLoader.class); + public static final String HADOOP23VERSIONNAME = "0.23"; + private static volatile HadoopShims hadoopShims; + private static JettyShims jettyShims; + private static AppenderSkeleton eventCounter; + private static SchedulerShim schedulerShim; + private static final HashMap HADOOP_SHIM_CLASSES = new HashMap(); + private static final HashMap EVENT_COUNTER_SHIM_CLASSES; + private static final HashMap HADOOP_THRIFT_AUTH_BRIDGE_CLASSES; + private static final String SCHEDULER_SHIM_CLASSE = "org.apache.hadoop.hive.schshim.FairSchedulerShim"; + + public static HadoopShims getHadoopShims() { + if (hadoopShims == null) { + Class var0 = ShimLoader.class; + synchronized(ShimLoader.class) { + if (hadoopShims == null) { + try { + hadoopShims = (HadoopShims)loadShims(HADOOP_SHIM_CLASSES, HadoopShims.class); + } catch (Throwable var3) { + LOG.error("Error loading shims", var3); + throw new RuntimeException(var3); + } + } + } + } + + return hadoopShims; + } + + public static synchronized AppenderSkeleton getEventCounter() { + if (eventCounter == null) { + eventCounter = (AppenderSkeleton)loadShims(EVENT_COUNTER_SHIM_CLASSES, AppenderSkeleton.class); + } + + return eventCounter; + } + + public static synchronized SchedulerShim getSchedulerShims() { + if (schedulerShim == null) { + schedulerShim = (SchedulerShim)createShim("org.apache.hadoop.hive.schshim.FairSchedulerShim", SchedulerShim.class); + } + + return schedulerShim; + } + + private static T loadShims(Map classMap, Class xface) { + String vers = getMajorVersion(); + String className = (String)classMap.get(vers); + return createShim(className, xface); + } + + private static T createShim(String className, Class xface) { + try { + Class clazz = Class.forName(className); + return xface.cast(clazz.newInstance()); + } catch (Exception var3) { + throw new RuntimeException("Could not load shims in class " + className, var3); + } + } + + public static String getMajorVersion() { + String vers = VersionInfo.getVersion(); + String[] parts = vers.split("\\."); + if (parts.length < 2) { + throw new RuntimeException("Illegal Hadoop Version: " + vers + " (expected A.B.* format)"); + } else { + switch(Integer.parseInt(parts[0])) { + case 2: + case 3: + return "0.23"; + default: + throw new IllegalArgumentException("Unrecognized Hadoop major version number: " + vers); + } + } + } + + private ShimLoader() { + } + + static { + HADOOP_SHIM_CLASSES.put("0.23", "org.apache.hadoop.hive.shims.Hadoop23Shims"); + EVENT_COUNTER_SHIM_CLASSES = new HashMap(); + EVENT_COUNTER_SHIM_CLASSES.put("0.23", "org.apache.hadoop.log.metrics.EventCounter"); + HADOOP_THRIFT_AUTH_BRIDGE_CLASSES = new HashMap(); + HADOOP_THRIFT_AUTH_BRIDGE_CLASSES.put("0.23", "org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge23"); + } +} \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-writer/pom.xml b/flinkx-oss/flinkx-oss-writer/pom.xml new file mode 100644 index 0000000000..963060549d --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/pom.xml @@ -0,0 +1,148 @@ + + + + flinkx-oss + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss-writer + + + + + org.anarres.lzo + lzo-core + 1.0.2 + + + org.anarres.lzo + lzo-hadoop + 1.0.5 + + + hadoop-core + org.apache.hadoop + + + + + + com.dtstack.flinkx + flinkx-oss-core + 1.6 + + + httpcore + org.apache.httpcomponents + + + httpclient + org.apache.httpcomponents + + + + + + httpcore + org.apache.httpcomponents + 4.4.5 + + + + httpclient + org.apache.httpcomponents + 4.5.2 + + + com.dtstack.flinkx + flinkx-oss-core + 1.6 + compile + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + ch.qos.logback:* + com.google.code.gson:* + com.data-artisans:* + org.scala-lang:* + io.netty:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java new file mode 100644 index 0000000000..c6a12168b2 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; +import com.dtstack.flinkx.util.ColumnTypeUtil; +import com.dtstack.flinkx.util.SysUtil; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + + +/** + * The oss implementation of OutputFormat + * + * @author wangyulei + * @date 2021-06-30 + */ +public abstract class BaseOssOutputFormat extends BaseFileOutputFormat { + + private static final int FILE_NAME_PART_SIZE = 3; + + protected int rowGroupSize; + + protected FileSystem fs; + + protected String endpoint; + + protected String accessKey; + + protected String secretKey; + + protected List columnTypes; + + protected List columnNames; + + protected List fullColumnNames; + + protected List fullColumnTypes; + + protected String delimiter; + + protected int[] colIndices; + + protected Configuration conf; + + protected boolean enableDictionary; + + protected transient Map decimalColInfo; + + /** + * 如果key为string类型的值是map 或者 list 会使用gson转为json格式存入 + */ + protected transient Gson gson; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + gson = new Gson(); + + initColIndices(); + super.openInternal(taskNumber, numTasks); + } + + @Override + protected void checkOutputDir() { + try { + Path dir = new Path(outputFilePath); + + if (fs.exists(dir)) { + if (fs.getFileStatus(dir).isFile()) { + throw new RuntimeException("Can't write new files under common file: " + dir + "\n" + + "One can only write new files under directories"); + } + } else { + if (!makeDir) { + throw new RuntimeException("Output path not exists:" + outputFilePath); + } + } + } catch (IOException e){ + throw new RuntimeException("Check output path error", e); + } + } + + @Override + protected void createActionFinishedTag() { + try { + if (fs.createNewFile(new Path(actionFinishedTag))) { + LOG.info("Success to create action finished tag:{}", actionFinishedTag); + } else { + LOG.warn("Failed to create action finished tag:{}", actionFinishedTag); + } + } catch (Exception e){ + throw new RuntimeException("create action finished tag error:", e); + } + } + + @Override + protected void waitForActionFinishedBeforeWrite() { + try { + Path path = new Path(actionFinishedTag); + boolean readyWrite = fs.exists(path); + int n = 0; + while (!readyWrite) { + if (n > SECOND_WAIT) { + throw new RuntimeException("Wait action finished before write timeout"); + } + + SysUtil.sleep(1000); + readyWrite = fs.exists(path); + n++; + } + } catch (Exception e) { + LOG.warn("Call method waitForActionFinishedBeforeWrite error", e); + } + } + + @Override + protected void cleanDirtyData() { + int fileIndex = formatState.getFileIndex(); + String lastJobId = formatState.getJobId(); + LOG.info("start to cleanDirtyData, fileIndex = {}, lastJobId = {}",fileIndex, lastJobId); + if (StringUtils.isBlank(lastJobId)) { + return; + } + + PathFilter filter = new PathFilter() { + @Override + public boolean accept(Path path) { + String fileName = path.getName(); + if (!fileName.contains(lastJobId)) { + return false; + } + + String[] splits = fileName.split("\\."); + if (splits.length == FILE_NAME_PART_SIZE) { + return Integer.parseInt(splits[2]) > fileIndex; + } + + return false; + } + }; + + try { + FileStatus[] dirtyData = fs.listStatus(new Path(outputFilePath), filter); + if (dirtyData != null && dirtyData.length > 0) { + for (FileStatus dirtyDatum : dirtyData) { + fs.delete(dirtyDatum.getPath(), false); + LOG.info("Delete dirty data file:{}", dirtyDatum.getPath()); + } + } + } catch (Exception e) { + LOG.error("Clean dirty data error:", e); + throw new RuntimeException(e); + } + } + + @Override + protected void openSource() throws IOException{ + try { + conf = new Configuration(); + conf.set("fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem"); + conf.set("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem"); + conf.set("fs.s3a.connection.ssl.enabled", "false"); + conf.set("fs.s3a.path.style.access", "true"); + conf.set("fs.s3a.endpoint", endpoint); + conf.set("fs.s3a.access.key", accessKey); + conf.set("fs.s3a.secret.key", secretKey); + fs = new Path(path).getFileSystem(conf); + } catch (Exception e) { + LOG.error("Failed to get S3AFileSystem with exception : " + e.getMessage()); + throw new RuntimeException("Failed to get S3AFileSystem with exception", e); + } + } + + private void initColIndices() { + if (fullColumnNames == null || fullColumnNames.size() == 0) { + fullColumnNames = columnNames; + } + + if (fullColumnTypes == null || fullColumnTypes.size() == 0) { + fullColumnTypes = columnTypes; + } + + colIndices = new int[fullColumnNames.size()]; + for (int i = 0; i < fullColumnNames.size(); ++i) { + int j = 0; + for (; j < columnNames.size(); ++j) { + if (fullColumnNames.get(i).equalsIgnoreCase(columnNames.get(j))) { + colIndices[i] = j; + break; + } + } + if (j == columnNames.size()) { + colIndices[i] = -1; + } + } + } + + @Override + protected void moveTemporaryDataBlockFileToDirectory(){ + try { + if (currentBlockFileName != null && currentBlockFileName.startsWith(ConstantValue.POINT_SYMBOL)) { + Path src = new Path(tmpPath + SP + currentBlockFileName); + if (!fs.exists(src)) { + LOG.warn("block file {} not exists", currentBlockFileName); + return; + } + + String dataFileName = currentBlockFileName.replaceFirst("\\.",""); + Path dist = new Path(tmpPath + SP + dataFileName); + + if (fs.rename(src, dist)) { + LOG.info("Rename temporary data block file:{} to:{}", src, dist); + } else { + LOG.info("Failed to rename temporary data block file:{} to:{}", src, dist); + } + } + } catch (Exception e){ + LOG.error("Failed to rename file with exception : " + e.getMessage()); + throw new RuntimeException(e); + } + } + + @Override + protected void clearTemporaryDataFiles() throws IOException{ + Path finishedDir = null, tmpDir = null; + if (outputFilePath.endsWith("/")) { + finishedDir = new Path(outputFilePath, FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath, DATA_SUBDIR); + } else { + finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath + SP + DATA_SUBDIR); + } + + if (fs.delete(finishedDir, true)) { + LOG.info("Success to delete .finished dir:{}", finishedDir); + } else { + LOG.warn("Failed to delete .finished dir:{}", finishedDir); + } + + if (fs.delete(tmpDir, true)) { + LOG.info("Success to delete .data dir:{}", tmpDir); + } else { + LOG.warn("Failed to delete .data dir:{}", tmpDir); + } + } + + @Override + protected void closeSource() throws IOException { + if (fs != null) { + fs.close(); + } + } + + @Override + protected void createFinishedTag() throws IOException{ + if (fs != null) { + fs.createNewFile(new Path(finishedPath)); + LOG.info("Create finished tag dir:{}", finishedPath); + } + } + + @Override + protected void waitForAllTasksToFinish() throws IOException{ + Path finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + final int maxRetryTime = 100; + int i = 0; + for (; i < maxRetryTime; ++i) { + if (fs.listStatus(finishedDir).length == numTasks) { + break; + } + SysUtil.sleep(3000); + } + + if (i == maxRetryTime) { + String subTaskDataPath = outputFilePath + SP + DATA_SUBDIR; + fs.delete(new Path(subTaskDataPath), true); + LOG.info("waitForAllTasksToFinish: delete path:[{}]", subTaskDataPath); + + fs.delete(finishedDir, true); + LOG.info("waitForAllTasksToFinish: delete finished dir:[{}]", finishedDir); + + throw new RuntimeException("timeout when gathering finish tags for each subtasks"); + } + } + + @Override + protected void coverageData() throws IOException{ + LOG.info("Overwrite the original data"); + + Path dir = new Path(outputFilePath); + if (!fs.exists(dir)) { + return; + } + + fs.delete(dir, true); + fs.mkdirs(dir); + } + + @Override + protected void moveTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> path.getName().startsWith(String.valueOf(taskNumber)); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.info("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void moveAllTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> !path.getName().startsWith("."); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.warn("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("OssWriter"); + } +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java new file mode 100644 index 0000000000..e4af3bca12 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; + +import java.util.List; + +/** + * The builder class of HdfsOutputFormat + * + * @author wangyulei + * @date 2021-06-30 + */ +public class OssOutputFormatBuilder extends FileOutputFormatBuilder { + + private BaseOssOutputFormat format; + + public OssOutputFormatBuilder(String type) { + switch (type.toUpperCase()) { + case "TEXT": + format = new OssTextOutputFormat(); + break; + default: + throw new IllegalArgumentException("Unsupported Oss file type: " + type); + } + + super.setFormat(format); + } + + public void setColumnNames(List columnNames) { + format.columnNames = columnNames; + } + + public void setColumnTypes(List columnTypes) { + format.columnTypes = columnTypes; + } + + public void setEndpoint(String endpoint) { + format.endpoint = endpoint; + } + + public void setAccessKey(String accessKey) { + format.accessKey = accessKey; + } + + public void setSecretKey(String secretKey) { + format.secretKey = secretKey; + } + + public void setFullColumnNames(List fullColumnNames) { + format.fullColumnNames = fullColumnNames; + } + + public void setDelimiter(String delimiter) { + format.delimiter = delimiter; + } + + public void setRowGroupSize(int rowGroupSize){ + format.rowGroupSize = rowGroupSize; + } + + public void setFullColumnTypes(List fullColumnTypes) { + format.fullColumnTypes = fullColumnTypes; + } + + public void setEnableDictionary(boolean enableDictionary) { + format.enableDictionary = enableDictionary; + } + + @Override + protected void checkFormat() { + super.checkFormat(); + + if (super.format.getPath() == null || super.format.getPath().length() == 0) { + throw new IllegalArgumentException("No valid path supplied."); + } + + if (!super.format.getPath().startsWith(ConstantValue.PROTOCOL_S3A)) { + throw new IllegalArgumentException("Path should start with s3a://"); + } + } + +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java new file mode 100644 index 0000000000..784bebd813 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.oss.ECompressType; +import com.dtstack.flinkx.oss.OssUtil; +import com.dtstack.flinkx.oss.util.StrUtil; +import com.dtstack.flinkx.util.DateUtil; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; + +/** + * The builder class of OssOutputFormat writing text files + * + * @author wangyulei + * @date 2021-06-30 + */ +public class OssTextOutputFormat extends BaseOssOutputFormat { + + private static final int NEWLINE = 10; + private transient OutputStream stream; + + private static final int BUFFER_SIZE = 1000; + + @Override + public void flushDataInternal() throws IOException { + LOG.info("Close current text stream, write data size:[{}]", bytesWriteCounter.getLocalValue()); + + if (stream != null){ + stream.flush(); + stream.close(); + stream = null; + } + } + + @Override + public float getDeviation(){ + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getDeviation(); + } + + @Override + public String getExtension() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getSuffix(); + } + + @Override + protected void nextBlock(){ + super.nextBlock(); + + if (stream != null) { + return; + } + + try { + String currentBlockTmpPath = null; + if (tmpPath.endsWith("/")) { + currentBlockTmpPath = tmpPath + currentBlockFileName; + } else { + currentBlockTmpPath = tmpPath + SP + currentBlockFileName; + } + Path p = new Path(currentBlockTmpPath); + + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + if (ECompressType.TEXT_NONE.equals(compressType)) { + stream = fs.create(p); + } else { + p = new Path(currentBlockTmpPath); + if (compressType == ECompressType.TEXT_GZIP){ + stream = new GzipCompressorOutputStream(fs.create(p)); + } else if(compressType == ECompressType.TEXT_BZIP2){ + stream = new BZip2CompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_LZO) { + CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration()); + stream = factory.getCodecByClassName("com.hadoop.compression.lzo.LzopCodec").createOutputStream(fs.create(p)); + } + } + + LOG.info("subtask:[{}] create block file:{}", taskNumber, currentBlockTmpPath); + + blockIndex++; + } catch (Exception e){ + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + @Override + public void writeSingleRecordToFile(Row row) throws WriteRecordException { + if (stream == null) { + nextBlock(); + } + + StringBuilder sb = new StringBuilder(); + int i = 0; + try { + int cnt = fullColumnNames.size(); + for (; i < cnt; ++i) { + int j = colIndices[i]; + if (j == -1) { + continue; + } + + if (i != 0) { + sb.append(delimiter); + } + + appendDataToString(sb, row.getField(j), ColumnType.fromString(columnTypes.get(j))); + } + } catch (Exception e) { + if (i < row.getArity()) { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + throw new WriteRecordException(e.getMessage(), e); + } + + try { + byte[] bytes = sb.toString().getBytes(this.charsetName); + this.stream.write(bytes); + this.stream.write(NEWLINE); + rowsOfCurrentBlock++; + + if (restoreConfig.isRestore()) { + lastRow = row; + } + + if (rowsOfCurrentBlock % BUFFER_SIZE == 0) { + this.stream.flush(); + } + } catch (IOException e) { + LOG.error(e.getMessage(), e); + throw new WriteRecordException(String.format("数据写入oss异常,row:{%s}", row), e); + } + } + + private void appendDataToString(StringBuilder sb, Object column, ColumnType columnType) { + if (column == null) { + sb.append(OssUtil.NULL_VALUE); + return; + } + + String rowData = column.toString(); + if (rowData.length() == 0) { + sb.append(""); + } else { + switch (columnType) { + case TINYINT: + sb.append(Byte.valueOf(rowData)); + break; + case SMALLINT: + sb.append(Short.valueOf(rowData)); + break; + case INT: + sb.append(Integer.valueOf(rowData)); + break; + case BIGINT: + if (column instanceof Timestamp){ + column=((Timestamp) column).getTime(); + sb.append(column); + break; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ + sb.append(data); + } else { + sb.append(Long.valueOf(rowData)); + } + break; + case FLOAT: + sb.append(Float.valueOf(rowData)); + break; + case DOUBLE: + sb.append(Double.valueOf(rowData)); + break; + case DECIMAL: + sb.append(HiveDecimal.create(new BigDecimal(rowData))); + break; + case STRING: + case VARCHAR: + case CHAR: + if (column instanceof Timestamp) { + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); + sb.append(fm.format(column)); + } else if (column instanceof Map || column instanceof List) { + sb.append(gson.toJson(column)); + } else { + sb.append(rowData); + } + break; + case BOOLEAN: + sb.append(StrUtil.parseBoolean(rowData)); + break; + case DATE: + column = DateUtil.columnToDate(column,null); + sb.append(DateUtil.dateToString((Date) column)); + break; + case TIMESTAMP: + column = DateUtil.columnToTimestamp(column,null); + sb.append(DateUtil.timestampToString((Date)column)); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + } + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nOssTextOutputFormat [" + jobName + "] writeRecord error: when converting field[" + columnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + public void closeSource() throws IOException { + OutputStream s = this.stream; + if (s != null) { + s.flush(); + this.stream = null; + s.close(); + } + } + +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java new file mode 100644 index 0000000000..526c636455 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.oss.util.StrUtil; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; +import org.apache.parquet.hadoop.ParquetWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.oss.OssConfigKeys.*; + +/** + * The writer plugin of oss + * + * @author wangyulei + * @date 2021-06-29 + */ +public class OssWriter extends BaseDataWriter { + + protected final Logger LOG = LoggerFactory.getLogger(getClass()); + + protected String fileType; + + protected String path; + + protected String fieldDelimiter; + + protected String compress; + + protected String fileName; + + protected List columnName; + + protected List columnType; + + protected String endpoint; + + protected String accessKey; + + protected String secretKey; + + protected String charSet; + + protected List fullColumnName; + + protected List fullColumnType; + + protected int rowGroupSize; + + protected long maxFileSize; + + protected long flushInterval; + + protected boolean enableDictionary; + + public OssWriter(DataTransferConfig config) { + super(config); + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + endpoint = writerConfig.getParameter().getStringVal(KEY_ENDPOINT); + accessKey = writerConfig.getParameter().getStringVal(KEY_ACCESS_KEY); + secretKey = writerConfig.getParameter().getStringVal(KEY_SECRET_KEY); + List columns = writerConfig.getParameter().getColumn(); + fileType = writerConfig.getParameter().getStringVal(KEY_FILE_TYPE); + + path = writerConfig.getParameter().getStringVal(KEY_PATH); + fieldDelimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER); + charSet = writerConfig.getParameter().getStringVal(KEY_ENCODING); + rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); + maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); + flushInterval = writerConfig.getParameter().getLongVal(KEY_FLUSH_INTERVAL, 0); + enableDictionary = writerConfig.getParameter().getBooleanVal(KEY_ENABLE_DICTIONARY, true); + + if (fieldDelimiter == null || fieldDelimiter.length() == 0) { + fieldDelimiter = "\001"; + } else { + fieldDelimiter = com.dtstack.flinkx.util.StringUtil.convertRegularExpr(fieldDelimiter); + } + + compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); + fileName = writerConfig.getParameter().getStringVal(KEY_FILE_NAME, ""); + if (columns != null && columns.size() > 0) { + columnName = new ArrayList<>(); + columnType = new ArrayList<>(); + for (Object column : columns) { + Map sm = (Map) column; + columnName.add((String) sm.get(KEY_COLUMN_NAME)); + columnType.add((String) sm.get(KEY_COLUMN_TYPE)); + } + } + + fullColumnName = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_NAME_LIST); + fullColumnType = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_TYPE_LIST); + + mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + OssOutputFormatBuilder builder = new OssOutputFormatBuilder(fileType); + builder.setPath(dealWithPath(path)); + builder.setEndpoint(endpoint); + builder.setAccessKey(accessKey); + builder.setSecretKey(secretKey); + builder.setFileName(fileName); + builder.setWriteMode(mode); + builder.setColumnNames(columnName); + builder.setColumnTypes(columnType); + builder.setCompress(compress); + builder.setMonitorUrls(monitorUrls); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setFullColumnNames(fullColumnName); + builder.setFullColumnTypes(fullColumnType); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setCharSetName(charSet); + builder.setDelimiter(fieldDelimiter); + builder.setRowGroupSize(rowGroupSize); + builder.setRestoreConfig(restoreConfig); + builder.setMaxFileSize(maxFileSize); + builder.setFlushBlockInterval(flushInterval); + builder.setEnableDictionary(enableDictionary); + + return createOutput(dataSet, builder.finish()); + } + + private String dealWithPath(String path) { + String pathWithPrefix = path; + if (!StrUtil.startsWith(path, "s3a://")) { + if (StrUtil.startsWith(path, "//")) { + pathWithPrefix = "s3a:" + path; + } else if (StrUtil.startsWith(path, "/")) { + pathWithPrefix = "s3a:/" + path; + } else { + pathWithPrefix = "s3a://" + path; + } + } + + if (!StrUtil.endsWith(pathWithPrefix,"/")) { + pathWithPrefix = pathWithPrefix + "/"; + } + + LOG.debug("Path = " + pathWithPrefix); + return pathWithPrefix; + } +} diff --git a/flinkx-oss/pom.xml b/flinkx-oss/pom.xml new file mode 100644 index 0000000000..510558dc36 --- /dev/null +++ b/flinkx-oss/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss + pom + + flinkx-oss-core + flinkx-oss-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index b8ac68013b..703eb34d87 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,7 @@ flinkx-kingbase flinkx-hdfs + flinkx-oss flinkx-hive flinkx-es flinkx-ftp @@ -66,6 +67,7 @@ 1.11.3 2.12 2.7.3 + 2.3.1 4.5.3 ${basedir}/dev release_1.11.0 From b56fa0a8a3e8c3924d2a3c43b2f1fa635f2a7343 Mon Sep 17 00:00:00 2001 From: chenyishuai Date: Tue, 7 Dec 2021 10:37:18 +0800 Subject: [PATCH 127/136] =?UTF-8?q?[feat-556][test]=20=E5=A2=9E=E5=8A=A0?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0=E5=AF=B9=E4=BA=8Eoss-writer=E7=9A=84?= =?UTF-8?q?=E6=B5=8B=E8=AF=95=E6=A1=88=E4=BE=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flinkx/constants/ConstantValue.java | 1 + flinkx-oss/flinkx-oss-core/pom.xml | 133 +++++++ .../com/dtstack/flinkx/oss/ECompressType.java | 98 +++++ .../com/dtstack/flinkx/oss/OssConfigKeys.java | 63 +++ .../java/com/dtstack/flinkx/oss/OssUtil.java | 202 ++++++++++ .../com/dtstack/flinkx/oss/util/StrUtil.java | 185 +++++++++ .../apache/hadoop/hive/shims/ShimLoader.java | 103 +++++ flinkx-oss/flinkx-oss-writer/pom.xml | 148 +++++++ .../oss/writer/BaseOssOutputFormat.java | 361 ++++++++++++++++++ .../oss/writer/OssOutputFormatBuilder.java | 101 +++++ .../oss/writer/OssTextOutputFormat.java | 257 +++++++++++++ .../dtstack/flinkx/oss/writer/OssWriter.java | 173 +++++++++ flinkx-oss/pom.xml | 28 ++ .../com/dtstack/flinkx/test/LocalTest.java | 1 + .../flinkx/test/PluginNameConstants.java | 1 + pom.xml | 2 + 16 files changed, 1857 insertions(+) create mode 100644 flinkx-oss/flinkx-oss-core/pom.xml create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java create mode 100644 flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java create mode 100644 flinkx-oss/flinkx-oss-writer/pom.xml create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java create mode 100644 flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java create mode 100644 flinkx-oss/pom.xml diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index b996539e7f..d21116dbff 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -47,6 +47,7 @@ public class ConstantValue { public static final String KEY_HTTP = "http"; + public static final String PROTOCOL_S3A = "s3a://"; public static final String PROTOCOL_HTTP = "http://"; public static final String PROTOCOL_HTTPS = "https://"; public static final String PROTOCOL_HDFS = "hdfs://"; diff --git a/flinkx-oss/flinkx-oss-core/pom.xml b/flinkx-oss/flinkx-oss-core/pom.xml new file mode 100644 index 0000000000..50f7fcc952 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/pom.xml @@ -0,0 +1,133 @@ + + + + flinkx-oss + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss-core + + + + com.amazonaws + aws-java-sdk-s3 + 1.11.689 + + + + com.amazonaws + aws-java-sdk-dynamodb + 1.11.689 + + + + commons-lang + commons-lang + 2.6 + provided + + + + org.apache.hadoop + hadoop-aws + 3.1.0 + + + com.amazonaws + aws-java-sdk-bundle + + + + + + org.apache.hive + hive-exec + ${hive.version} + + + calcite-core + org.apache.calcite + + + calcite-avatica + org.apache.calcite + + + derby + org.apache.derby + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + org.apache.hive + hive-serde + ${hive.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-yarn-api + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + parquet-hadoop + org.apache.parquet + 1.8.3 + + + org.xerial.snappy + snappy-java + + + + + + org.xerial.snappy + snappy-java + 1.1.4 + + + + \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java new file mode 100644 index 0000000000..761790580a --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/ECompressType.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +import org.apache.commons.lang.StringUtils; + +/** + * @author wangyulei + * @date 2021-06-29 + */ +public enum ECompressType { + + /** + * text file + */ + TEXT_GZIP("GZIP", "text", ".gz", 0.331F), + TEXT_BZIP2("BZIP2", "text", ".bz2", 0.259F), + TEXT_LZO("LZO", "text", ".lzo", 1.0F), + TEXT_NONE("NONE", "text", "", 0.637F), + + /** + * orc file + */ + ORC_SNAPPY("SNAPPY", "orc", ".snappy", 0.233F), + ORC_GZIP("GZIP", "orc", ".gz", 1.0F), + ORC_BZIP("BZIP", "orc", ".bz", 1.0F), + ORC_LZ4("LZ4", "orc", ".lz4", 1.0F), + ORC_NONE("NONE", "orc", "", 0.233F), + + /** + * parquet file + */ + PARQUET_SNAPPY("SNAPPY", "parquet", ".snappy", 0.274F), + PARQUET_GZIP("GZIP", "parquet", ".gz", 1.0F), + PARQUET_LZO("LZO", "parquet", ".lzo", 1.0F), + PARQUET_NONE("NONE", "parquet", "", 1.0F); + + private String type; + + private String fileType; + + private String suffix; + + private float deviation; + + ECompressType(String type, String fileType, String suffix, float deviation) { + this.type = type; + this.fileType = fileType; + this.suffix = suffix; + this.deviation = deviation; + } + + public static ECompressType getByTypeAndFileType(String type, String fileType){ + if (StringUtils.isEmpty(type)) { + type = "NONE"; + } + + for (ECompressType value : ECompressType.values()) { + if (value.getType().equalsIgnoreCase(type) && value.getFileType().equalsIgnoreCase(fileType)){ + return value; + } + } + + throw new IllegalArgumentException("No enum constant " + type); + } + + public String getType() { + return type; + } + + public String getFileType() { + return fileType; + } + + public String getSuffix() { + return suffix; + } + + public float getDeviation() { + return deviation; + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java new file mode 100644 index 0000000000..cb16edac70 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssConfigKeys.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +/** + * @author wangyulei + * @date 2021-06-28 + */ +public class OssConfigKeys { + public static final String KEY_FIELD_DELIMITER = "fieldDelimiter"; + + public static final String KEY_ACCESS_KEY = "accessKey"; + + public static final String KEY_SECRET_KEY = "secretKey"; + + public static final String KEY_ENDPOINT = "endpoint"; + + public static final String KEY_PATH = "path"; + + public static final String KEY_FILTER = "filterRegex"; + + public static final String KEY_FILE_TYPE = "fileType"; + + public static final String KEY_WRITE_MODE = "writeMode"; + + public static final String KEY_FULL_COLUMN_NAME_LIST = "fullColumnName"; + + public static final String KEY_FULL_COLUMN_TYPE_LIST = "fullColumnType"; + + public static final String KEY_COLUMN_NAME = "name"; + + public static final String KEY_COLUMN_TYPE = "type"; + + public static final String KEY_COMPRESS = "compress"; + + public static final String KEY_FILE_NAME = "fileName"; + + public static final String KEY_ENCODING = "encoding"; + + public static final String KEY_ROW_GROUP_SIZE = "rowGroupSize"; + + public static final String KEY_MAX_FILE_SIZE = "maxFileSize"; + + public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + + public static final String KEY_ENABLE_DICTIONARY = "enableDictionary"; +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java new file mode 100644 index 0000000000..bbc3d1667a --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/OssUtil.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss; + +import com.dtstack.flinkx.enums.ColumnType; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.io.*; +import org.apache.parquet.io.api.Binary; + + +/** + * Utilities for OssReader and OssWriter + * + * @author wangyulei + * @date 2021-06-29 + */ +public class OssUtil { + + public static final String NULL_VALUE = "\\N"; + + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; + + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; + + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; + + public static Object getWritableValue(Object writable) { + Class clz = writable.getClass(); + Object ret = null; + + if (clz == IntWritable.class) { + ret = ((IntWritable) writable).get(); + } else if (clz == Text.class) { + ret = ((Text) writable).toString(); + } else if (clz == LongWritable.class) { + ret = ((LongWritable) writable).get(); + } else if (clz == ByteWritable.class) { + ret = ((ByteWritable) writable).get(); + } else if (clz == DateWritable.class) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof DoubleWritable){ + ret = ((DoubleWritable) writable).get(); + } else if (writable instanceof TimestampWritable){ + ret = ((TimestampWritable) writable).getTimestamp(); + } else if (writable instanceof DateWritable){ + ret = ((DateWritable) writable).get(); + } else if (writable instanceof FloatWritable){ + ret = ((FloatWritable) writable).get(); + } else if (writable instanceof BooleanWritable){ + ret = ((BooleanWritable) writable).get(); + } else { + ret = writable.toString(); + } + return ret; + } + + public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) { + ObjectInspector objectInspector = null; + switch (columnType) { + case TINYINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Byte.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case SMALLINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Short.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case INT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BIGINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case FLOAT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Float.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DOUBLE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Double.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DECIMAL: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(HiveDecimalWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case TIMESTAMP: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Timestamp.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DATE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Date.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case STRING: + case VARCHAR: + case CHAR: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(String.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BOOLEAN: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Boolean.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BINARY: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(BytesWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + default: + throw new IllegalArgumentException("You should not be here"); + } + return objectInspector; + } + + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision){ + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static byte[] longToByteArray(long data){ + long nano = data * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return dst; + } + + private static byte[] getBytes(long i) { + byte[] bytes=new byte[8]; + bytes[0] = (byte)((i >> 56) & 0xFF); + bytes[1] = (byte)((i >> 48) & 0xFF); + bytes[2] = (byte)((i >> 40) & 0xFF); + bytes[3] = (byte)((i >> 32) & 0xFF); + bytes[4] = (byte)((i >> 24) & 0xFF); + bytes[5] = (byte)((i >> 16) & 0xFF); + bytes[6] = (byte)((i >> 8) & 0xFF); + bytes[7] = (byte)(i & 0xFF); + return bytes; + } + + /** + * @param bytes + */ + private static void flip(byte[] bytes) { + for (int i = 0, j = bytes.length-1; i < j; i++, j--) { + byte t = bytes[i]; + bytes[i] = bytes[j]; + bytes[j] = t; + } + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java new file mode 100644 index 0000000000..68638c2952 --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/com/dtstack/flinkx/oss/util/StrUtil.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.util; + +/** + * + * @author wangyulei + * @date 2021-06-29 + */ +public class StrUtil { + /** + *

Check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.startsWith(null, null)      = true
+     * StringUtils.startsWith(null, "abc")     = false
+     * StringUtils.startsWith("abcdef", null)  = false
+     * StringUtils.startsWith("abcdef", "abc") = true
+     * StringUtils.startsWith("ABCDEF", "abc") = false
+     * 
+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case sensitive, or + * both null + * @since 2.4 + */ + public static boolean startsWith(String str, String prefix) { + return startsWith(str, prefix, false); + } + + /** + *

Case insensitive check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.startsWithIgnoreCase(null, null)      = true
+     * StringUtils.startsWithIgnoreCase(null, "abc")     = false
+     * StringUtils.startsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.startsWithIgnoreCase("abcdef", "abc") = true
+     * StringUtils.startsWithIgnoreCase("ABCDEF", "abc") = true
+     * 
+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case insensitive, or + * both null + * @since 2.4 + */ + public static boolean startsWithIgnoreCase(String str, String prefix) { + return startsWith(str, prefix, true); + } + + /** + *

Check if a String starts with a specified prefix (optionally case insensitive).

+ * + * @see java.lang.String#startsWith(String) + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + */ + private static boolean startsWith(String str, String prefix, boolean ignoreCase) { + if (str == null || prefix == null) { + return (str == null && prefix == null); + } + if (prefix.length() > str.length()) { + return false; + } + return str.regionMatches(ignoreCase, 0, prefix, 0, prefix.length()); + } + + /** + *

Check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.endsWith(null, null)      = true
+     * StringUtils.endsWith(null, "def")     = false
+     * StringUtils.endsWith("abcdef", null)  = false
+     * StringUtils.endsWith("abcdef", "def") = true
+     * StringUtils.endsWith("ABCDEF", "def") = false
+     * StringUtils.endsWith("ABCDEF", "cde") = false
+     * 
+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case sensitive, or + * both null + * @since 2.4 + */ + public static boolean endsWith(String str, String suffix) { + return endsWith(str, suffix, false); + } + + /** + *

Case insensitive check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.endsWithIgnoreCase(null, null)      = true
+     * StringUtils.endsWithIgnoreCase(null, "def")     = false
+     * StringUtils.endsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.endsWithIgnoreCase("abcdef", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "cde") = false
+     * 
+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case insensitive, or + * both null + * @since 2.4 + */ + public static boolean endsWithIgnoreCase(String str, String suffix) { + return endsWith(str, suffix, true); + } + + /** + *

Check if a String ends with a specified suffix (optionally case insensitive).

+ * + * @see java.lang.String#endsWith(String) + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + */ + private static boolean endsWith(String str, String suffix, boolean ignoreCase) { + if (str == null || suffix == null) { + return (str == null && suffix == null); + } + if (suffix.length() > str.length()) { + return false; + } + int strOffset = str.length() - suffix.length(); + return str.regionMatches(ignoreCase, strOffset, suffix, 0, suffix.length()); + } + + public static Boolean parseBoolean(String str) { + if (null == str || "null".equalsIgnoreCase(str)) { + return Boolean.FALSE; + } + + if ("1".equals(str)) { + return Boolean.TRUE; + } else if ("0".equals(str)) { + return Boolean.FALSE; + } else { + return Boolean.parseBoolean(str); + } + } +} diff --git a/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java b/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java new file mode 100644 index 0000000000..11c91dfb2f --- /dev/null +++ b/flinkx-oss/flinkx-oss-core/src/main/java/org/apache/hadoop/hive/shims/ShimLoader.java @@ -0,0 +1,103 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by FernFlower decompiler) +// + +package org.apache.hadoop.hive.shims; + +import org.apache.hadoop.util.VersionInfo; +import org.apache.log4j.AppenderSkeleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public abstract class ShimLoader { + private static final Logger LOG = LoggerFactory.getLogger(ShimLoader.class); + public static final String HADOOP23VERSIONNAME = "0.23"; + private static volatile HadoopShims hadoopShims; + private static JettyShims jettyShims; + private static AppenderSkeleton eventCounter; + private static SchedulerShim schedulerShim; + private static final HashMap HADOOP_SHIM_CLASSES = new HashMap(); + private static final HashMap EVENT_COUNTER_SHIM_CLASSES; + private static final HashMap HADOOP_THRIFT_AUTH_BRIDGE_CLASSES; + private static final String SCHEDULER_SHIM_CLASSE = "org.apache.hadoop.hive.schshim.FairSchedulerShim"; + + public static HadoopShims getHadoopShims() { + if (hadoopShims == null) { + Class var0 = ShimLoader.class; + synchronized(ShimLoader.class) { + if (hadoopShims == null) { + try { + hadoopShims = (HadoopShims)loadShims(HADOOP_SHIM_CLASSES, HadoopShims.class); + } catch (Throwable var3) { + LOG.error("Error loading shims", var3); + throw new RuntimeException(var3); + } + } + } + } + + return hadoopShims; + } + + public static synchronized AppenderSkeleton getEventCounter() { + if (eventCounter == null) { + eventCounter = (AppenderSkeleton)loadShims(EVENT_COUNTER_SHIM_CLASSES, AppenderSkeleton.class); + } + + return eventCounter; + } + + public static synchronized SchedulerShim getSchedulerShims() { + if (schedulerShim == null) { + schedulerShim = (SchedulerShim)createShim("org.apache.hadoop.hive.schshim.FairSchedulerShim", SchedulerShim.class); + } + + return schedulerShim; + } + + private static T loadShims(Map classMap, Class xface) { + String vers = getMajorVersion(); + String className = (String)classMap.get(vers); + return createShim(className, xface); + } + + private static T createShim(String className, Class xface) { + try { + Class clazz = Class.forName(className); + return xface.cast(clazz.newInstance()); + } catch (Exception var3) { + throw new RuntimeException("Could not load shims in class " + className, var3); + } + } + + public static String getMajorVersion() { + String vers = VersionInfo.getVersion(); + String[] parts = vers.split("\\."); + if (parts.length < 2) { + throw new RuntimeException("Illegal Hadoop Version: " + vers + " (expected A.B.* format)"); + } else { + switch(Integer.parseInt(parts[0])) { + case 2: + case 3: + return "0.23"; + default: + throw new IllegalArgumentException("Unrecognized Hadoop major version number: " + vers); + } + } + } + + private ShimLoader() { + } + + static { + HADOOP_SHIM_CLASSES.put("0.23", "org.apache.hadoop.hive.shims.Hadoop23Shims"); + EVENT_COUNTER_SHIM_CLASSES = new HashMap(); + EVENT_COUNTER_SHIM_CLASSES.put("0.23", "org.apache.hadoop.log.metrics.EventCounter"); + HADOOP_THRIFT_AUTH_BRIDGE_CLASSES = new HashMap(); + HADOOP_THRIFT_AUTH_BRIDGE_CLASSES.put("0.23", "org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge23"); + } +} \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-writer/pom.xml b/flinkx-oss/flinkx-oss-writer/pom.xml new file mode 100644 index 0000000000..963060549d --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/pom.xml @@ -0,0 +1,148 @@ + + + + flinkx-oss + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss-writer + + + + + org.anarres.lzo + lzo-core + 1.0.2 + + + org.anarres.lzo + lzo-hadoop + 1.0.5 + + + hadoop-core + org.apache.hadoop + + + + + + com.dtstack.flinkx + flinkx-oss-core + 1.6 + + + httpcore + org.apache.httpcomponents + + + httpclient + org.apache.httpcomponents + + + + + + httpcore + org.apache.httpcomponents + 4.4.5 + + + + httpclient + org.apache.httpcomponents + 4.5.2 + + + com.dtstack.flinkx + flinkx-oss-core + 1.6 + compile + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + ch.qos.logback:* + com.google.code.gson:* + com.data-artisans:* + org.scala-lang:* + io.netty:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java new file mode 100644 index 0000000000..c6a12168b2 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/BaseOssOutputFormat.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; +import com.dtstack.flinkx.util.ColumnTypeUtil; +import com.dtstack.flinkx.util.SysUtil; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + + +/** + * The oss implementation of OutputFormat + * + * @author wangyulei + * @date 2021-06-30 + */ +public abstract class BaseOssOutputFormat extends BaseFileOutputFormat { + + private static final int FILE_NAME_PART_SIZE = 3; + + protected int rowGroupSize; + + protected FileSystem fs; + + protected String endpoint; + + protected String accessKey; + + protected String secretKey; + + protected List columnTypes; + + protected List columnNames; + + protected List fullColumnNames; + + protected List fullColumnTypes; + + protected String delimiter; + + protected int[] colIndices; + + protected Configuration conf; + + protected boolean enableDictionary; + + protected transient Map decimalColInfo; + + /** + * 如果key为string类型的值是map 或者 list 会使用gson转为json格式存入 + */ + protected transient Gson gson; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + gson = new Gson(); + + initColIndices(); + super.openInternal(taskNumber, numTasks); + } + + @Override + protected void checkOutputDir() { + try { + Path dir = new Path(outputFilePath); + + if (fs.exists(dir)) { + if (fs.getFileStatus(dir).isFile()) { + throw new RuntimeException("Can't write new files under common file: " + dir + "\n" + + "One can only write new files under directories"); + } + } else { + if (!makeDir) { + throw new RuntimeException("Output path not exists:" + outputFilePath); + } + } + } catch (IOException e){ + throw new RuntimeException("Check output path error", e); + } + } + + @Override + protected void createActionFinishedTag() { + try { + if (fs.createNewFile(new Path(actionFinishedTag))) { + LOG.info("Success to create action finished tag:{}", actionFinishedTag); + } else { + LOG.warn("Failed to create action finished tag:{}", actionFinishedTag); + } + } catch (Exception e){ + throw new RuntimeException("create action finished tag error:", e); + } + } + + @Override + protected void waitForActionFinishedBeforeWrite() { + try { + Path path = new Path(actionFinishedTag); + boolean readyWrite = fs.exists(path); + int n = 0; + while (!readyWrite) { + if (n > SECOND_WAIT) { + throw new RuntimeException("Wait action finished before write timeout"); + } + + SysUtil.sleep(1000); + readyWrite = fs.exists(path); + n++; + } + } catch (Exception e) { + LOG.warn("Call method waitForActionFinishedBeforeWrite error", e); + } + } + + @Override + protected void cleanDirtyData() { + int fileIndex = formatState.getFileIndex(); + String lastJobId = formatState.getJobId(); + LOG.info("start to cleanDirtyData, fileIndex = {}, lastJobId = {}",fileIndex, lastJobId); + if (StringUtils.isBlank(lastJobId)) { + return; + } + + PathFilter filter = new PathFilter() { + @Override + public boolean accept(Path path) { + String fileName = path.getName(); + if (!fileName.contains(lastJobId)) { + return false; + } + + String[] splits = fileName.split("\\."); + if (splits.length == FILE_NAME_PART_SIZE) { + return Integer.parseInt(splits[2]) > fileIndex; + } + + return false; + } + }; + + try { + FileStatus[] dirtyData = fs.listStatus(new Path(outputFilePath), filter); + if (dirtyData != null && dirtyData.length > 0) { + for (FileStatus dirtyDatum : dirtyData) { + fs.delete(dirtyDatum.getPath(), false); + LOG.info("Delete dirty data file:{}", dirtyDatum.getPath()); + } + } + } catch (Exception e) { + LOG.error("Clean dirty data error:", e); + throw new RuntimeException(e); + } + } + + @Override + protected void openSource() throws IOException{ + try { + conf = new Configuration(); + conf.set("fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem"); + conf.set("fs.file.impl", "org.apache.hadoop.fs.LocalFileSystem"); + conf.set("fs.s3a.connection.ssl.enabled", "false"); + conf.set("fs.s3a.path.style.access", "true"); + conf.set("fs.s3a.endpoint", endpoint); + conf.set("fs.s3a.access.key", accessKey); + conf.set("fs.s3a.secret.key", secretKey); + fs = new Path(path).getFileSystem(conf); + } catch (Exception e) { + LOG.error("Failed to get S3AFileSystem with exception : " + e.getMessage()); + throw new RuntimeException("Failed to get S3AFileSystem with exception", e); + } + } + + private void initColIndices() { + if (fullColumnNames == null || fullColumnNames.size() == 0) { + fullColumnNames = columnNames; + } + + if (fullColumnTypes == null || fullColumnTypes.size() == 0) { + fullColumnTypes = columnTypes; + } + + colIndices = new int[fullColumnNames.size()]; + for (int i = 0; i < fullColumnNames.size(); ++i) { + int j = 0; + for (; j < columnNames.size(); ++j) { + if (fullColumnNames.get(i).equalsIgnoreCase(columnNames.get(j))) { + colIndices[i] = j; + break; + } + } + if (j == columnNames.size()) { + colIndices[i] = -1; + } + } + } + + @Override + protected void moveTemporaryDataBlockFileToDirectory(){ + try { + if (currentBlockFileName != null && currentBlockFileName.startsWith(ConstantValue.POINT_SYMBOL)) { + Path src = new Path(tmpPath + SP + currentBlockFileName); + if (!fs.exists(src)) { + LOG.warn("block file {} not exists", currentBlockFileName); + return; + } + + String dataFileName = currentBlockFileName.replaceFirst("\\.",""); + Path dist = new Path(tmpPath + SP + dataFileName); + + if (fs.rename(src, dist)) { + LOG.info("Rename temporary data block file:{} to:{}", src, dist); + } else { + LOG.info("Failed to rename temporary data block file:{} to:{}", src, dist); + } + } + } catch (Exception e){ + LOG.error("Failed to rename file with exception : " + e.getMessage()); + throw new RuntimeException(e); + } + } + + @Override + protected void clearTemporaryDataFiles() throws IOException{ + Path finishedDir = null, tmpDir = null; + if (outputFilePath.endsWith("/")) { + finishedDir = new Path(outputFilePath, FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath, DATA_SUBDIR); + } else { + finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath + SP + DATA_SUBDIR); + } + + if (fs.delete(finishedDir, true)) { + LOG.info("Success to delete .finished dir:{}", finishedDir); + } else { + LOG.warn("Failed to delete .finished dir:{}", finishedDir); + } + + if (fs.delete(tmpDir, true)) { + LOG.info("Success to delete .data dir:{}", tmpDir); + } else { + LOG.warn("Failed to delete .data dir:{}", tmpDir); + } + } + + @Override + protected void closeSource() throws IOException { + if (fs != null) { + fs.close(); + } + } + + @Override + protected void createFinishedTag() throws IOException{ + if (fs != null) { + fs.createNewFile(new Path(finishedPath)); + LOG.info("Create finished tag dir:{}", finishedPath); + } + } + + @Override + protected void waitForAllTasksToFinish() throws IOException{ + Path finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + final int maxRetryTime = 100; + int i = 0; + for (; i < maxRetryTime; ++i) { + if (fs.listStatus(finishedDir).length == numTasks) { + break; + } + SysUtil.sleep(3000); + } + + if (i == maxRetryTime) { + String subTaskDataPath = outputFilePath + SP + DATA_SUBDIR; + fs.delete(new Path(subTaskDataPath), true); + LOG.info("waitForAllTasksToFinish: delete path:[{}]", subTaskDataPath); + + fs.delete(finishedDir, true); + LOG.info("waitForAllTasksToFinish: delete finished dir:[{}]", finishedDir); + + throw new RuntimeException("timeout when gathering finish tags for each subtasks"); + } + } + + @Override + protected void coverageData() throws IOException{ + LOG.info("Overwrite the original data"); + + Path dir = new Path(outputFilePath); + if (!fs.exists(dir)) { + return; + } + + fs.delete(dir, true); + fs.mkdirs(dir); + } + + @Override + protected void moveTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> path.getName().startsWith(String.valueOf(taskNumber)); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.info("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void moveAllTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> !path.getName().startsWith("."); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.warn("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("OssWriter"); + } +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java new file mode 100644 index 0000000000..e4af3bca12 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssOutputFormatBuilder.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; + +import java.util.List; + +/** + * The builder class of HdfsOutputFormat + * + * @author wangyulei + * @date 2021-06-30 + */ +public class OssOutputFormatBuilder extends FileOutputFormatBuilder { + + private BaseOssOutputFormat format; + + public OssOutputFormatBuilder(String type) { + switch (type.toUpperCase()) { + case "TEXT": + format = new OssTextOutputFormat(); + break; + default: + throw new IllegalArgumentException("Unsupported Oss file type: " + type); + } + + super.setFormat(format); + } + + public void setColumnNames(List columnNames) { + format.columnNames = columnNames; + } + + public void setColumnTypes(List columnTypes) { + format.columnTypes = columnTypes; + } + + public void setEndpoint(String endpoint) { + format.endpoint = endpoint; + } + + public void setAccessKey(String accessKey) { + format.accessKey = accessKey; + } + + public void setSecretKey(String secretKey) { + format.secretKey = secretKey; + } + + public void setFullColumnNames(List fullColumnNames) { + format.fullColumnNames = fullColumnNames; + } + + public void setDelimiter(String delimiter) { + format.delimiter = delimiter; + } + + public void setRowGroupSize(int rowGroupSize){ + format.rowGroupSize = rowGroupSize; + } + + public void setFullColumnTypes(List fullColumnTypes) { + format.fullColumnTypes = fullColumnTypes; + } + + public void setEnableDictionary(boolean enableDictionary) { + format.enableDictionary = enableDictionary; + } + + @Override + protected void checkFormat() { + super.checkFormat(); + + if (super.format.getPath() == null || super.format.getPath().length() == 0) { + throw new IllegalArgumentException("No valid path supplied."); + } + + if (!super.format.getPath().startsWith(ConstantValue.PROTOCOL_S3A)) { + throw new IllegalArgumentException("Path should start with s3a://"); + } + } + +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java new file mode 100644 index 0000000000..784bebd813 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssTextOutputFormat.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.oss.ECompressType; +import com.dtstack.flinkx.oss.OssUtil; +import com.dtstack.flinkx.oss.util.StrUtil; +import com.dtstack.flinkx.util.DateUtil; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; + +/** + * The builder class of OssOutputFormat writing text files + * + * @author wangyulei + * @date 2021-06-30 + */ +public class OssTextOutputFormat extends BaseOssOutputFormat { + + private static final int NEWLINE = 10; + private transient OutputStream stream; + + private static final int BUFFER_SIZE = 1000; + + @Override + public void flushDataInternal() throws IOException { + LOG.info("Close current text stream, write data size:[{}]", bytesWriteCounter.getLocalValue()); + + if (stream != null){ + stream.flush(); + stream.close(); + stream = null; + } + } + + @Override + public float getDeviation(){ + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getDeviation(); + } + + @Override + public String getExtension() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getSuffix(); + } + + @Override + protected void nextBlock(){ + super.nextBlock(); + + if (stream != null) { + return; + } + + try { + String currentBlockTmpPath = null; + if (tmpPath.endsWith("/")) { + currentBlockTmpPath = tmpPath + currentBlockFileName; + } else { + currentBlockTmpPath = tmpPath + SP + currentBlockFileName; + } + Path p = new Path(currentBlockTmpPath); + + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + if (ECompressType.TEXT_NONE.equals(compressType)) { + stream = fs.create(p); + } else { + p = new Path(currentBlockTmpPath); + if (compressType == ECompressType.TEXT_GZIP){ + stream = new GzipCompressorOutputStream(fs.create(p)); + } else if(compressType == ECompressType.TEXT_BZIP2){ + stream = new BZip2CompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_LZO) { + CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration()); + stream = factory.getCodecByClassName("com.hadoop.compression.lzo.LzopCodec").createOutputStream(fs.create(p)); + } + } + + LOG.info("subtask:[{}] create block file:{}", taskNumber, currentBlockTmpPath); + + blockIndex++; + } catch (Exception e){ + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + @Override + public void writeSingleRecordToFile(Row row) throws WriteRecordException { + if (stream == null) { + nextBlock(); + } + + StringBuilder sb = new StringBuilder(); + int i = 0; + try { + int cnt = fullColumnNames.size(); + for (; i < cnt; ++i) { + int j = colIndices[i]; + if (j == -1) { + continue; + } + + if (i != 0) { + sb.append(delimiter); + } + + appendDataToString(sb, row.getField(j), ColumnType.fromString(columnTypes.get(j))); + } + } catch (Exception e) { + if (i < row.getArity()) { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + throw new WriteRecordException(e.getMessage(), e); + } + + try { + byte[] bytes = sb.toString().getBytes(this.charsetName); + this.stream.write(bytes); + this.stream.write(NEWLINE); + rowsOfCurrentBlock++; + + if (restoreConfig.isRestore()) { + lastRow = row; + } + + if (rowsOfCurrentBlock % BUFFER_SIZE == 0) { + this.stream.flush(); + } + } catch (IOException e) { + LOG.error(e.getMessage(), e); + throw new WriteRecordException(String.format("数据写入oss异常,row:{%s}", row), e); + } + } + + private void appendDataToString(StringBuilder sb, Object column, ColumnType columnType) { + if (column == null) { + sb.append(OssUtil.NULL_VALUE); + return; + } + + String rowData = column.toString(); + if (rowData.length() == 0) { + sb.append(""); + } else { + switch (columnType) { + case TINYINT: + sb.append(Byte.valueOf(rowData)); + break; + case SMALLINT: + sb.append(Short.valueOf(rowData)); + break; + case INT: + sb.append(Integer.valueOf(rowData)); + break; + case BIGINT: + if (column instanceof Timestamp){ + column=((Timestamp) column).getTime(); + sb.append(column); + break; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0){ + sb.append(data); + } else { + sb.append(Long.valueOf(rowData)); + } + break; + case FLOAT: + sb.append(Float.valueOf(rowData)); + break; + case DOUBLE: + sb.append(Double.valueOf(rowData)); + break; + case DECIMAL: + sb.append(HiveDecimal.create(new BigDecimal(rowData))); + break; + case STRING: + case VARCHAR: + case CHAR: + if (column instanceof Timestamp) { + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); + sb.append(fm.format(column)); + } else if (column instanceof Map || column instanceof List) { + sb.append(gson.toJson(column)); + } else { + sb.append(rowData); + } + break; + case BOOLEAN: + sb.append(StrUtil.parseBoolean(rowData)); + break; + case DATE: + column = DateUtil.columnToDate(column,null); + sb.append(DateUtil.dateToString((Date) column)); + break; + case TIMESTAMP: + column = DateUtil.columnToTimestamp(column,null); + sb.append(DateUtil.timestampToString((Date)column)); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + } + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nOssTextOutputFormat [" + jobName + "] writeRecord error: when converting field[" + columnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + public void closeSource() throws IOException { + OutputStream s = this.stream; + if (s != null) { + s.flush(); + this.stream = null; + s.close(); + } + } + +} diff --git a/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java new file mode 100644 index 0000000000..526c636455 --- /dev/null +++ b/flinkx-oss/flinkx-oss-writer/src/main/java/com/dtstack/flinkx/oss/writer/OssWriter.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.oss.writer; + +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.oss.util.StrUtil; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; +import org.apache.parquet.hadoop.ParquetWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.oss.OssConfigKeys.*; + +/** + * The writer plugin of oss + * + * @author wangyulei + * @date 2021-06-29 + */ +public class OssWriter extends BaseDataWriter { + + protected final Logger LOG = LoggerFactory.getLogger(getClass()); + + protected String fileType; + + protected String path; + + protected String fieldDelimiter; + + protected String compress; + + protected String fileName; + + protected List columnName; + + protected List columnType; + + protected String endpoint; + + protected String accessKey; + + protected String secretKey; + + protected String charSet; + + protected List fullColumnName; + + protected List fullColumnType; + + protected int rowGroupSize; + + protected long maxFileSize; + + protected long flushInterval; + + protected boolean enableDictionary; + + public OssWriter(DataTransferConfig config) { + super(config); + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + endpoint = writerConfig.getParameter().getStringVal(KEY_ENDPOINT); + accessKey = writerConfig.getParameter().getStringVal(KEY_ACCESS_KEY); + secretKey = writerConfig.getParameter().getStringVal(KEY_SECRET_KEY); + List columns = writerConfig.getParameter().getColumn(); + fileType = writerConfig.getParameter().getStringVal(KEY_FILE_TYPE); + + path = writerConfig.getParameter().getStringVal(KEY_PATH); + fieldDelimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER); + charSet = writerConfig.getParameter().getStringVal(KEY_ENCODING); + rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); + maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); + flushInterval = writerConfig.getParameter().getLongVal(KEY_FLUSH_INTERVAL, 0); + enableDictionary = writerConfig.getParameter().getBooleanVal(KEY_ENABLE_DICTIONARY, true); + + if (fieldDelimiter == null || fieldDelimiter.length() == 0) { + fieldDelimiter = "\001"; + } else { + fieldDelimiter = com.dtstack.flinkx.util.StringUtil.convertRegularExpr(fieldDelimiter); + } + + compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); + fileName = writerConfig.getParameter().getStringVal(KEY_FILE_NAME, ""); + if (columns != null && columns.size() > 0) { + columnName = new ArrayList<>(); + columnType = new ArrayList<>(); + for (Object column : columns) { + Map sm = (Map) column; + columnName.add((String) sm.get(KEY_COLUMN_NAME)); + columnType.add((String) sm.get(KEY_COLUMN_TYPE)); + } + } + + fullColumnName = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_NAME_LIST); + fullColumnType = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_TYPE_LIST); + + mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + OssOutputFormatBuilder builder = new OssOutputFormatBuilder(fileType); + builder.setPath(dealWithPath(path)); + builder.setEndpoint(endpoint); + builder.setAccessKey(accessKey); + builder.setSecretKey(secretKey); + builder.setFileName(fileName); + builder.setWriteMode(mode); + builder.setColumnNames(columnName); + builder.setColumnTypes(columnType); + builder.setCompress(compress); + builder.setMonitorUrls(monitorUrls); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setFullColumnNames(fullColumnName); + builder.setFullColumnTypes(fullColumnType); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setCharSetName(charSet); + builder.setDelimiter(fieldDelimiter); + builder.setRowGroupSize(rowGroupSize); + builder.setRestoreConfig(restoreConfig); + builder.setMaxFileSize(maxFileSize); + builder.setFlushBlockInterval(flushInterval); + builder.setEnableDictionary(enableDictionary); + + return createOutput(dataSet, builder.finish()); + } + + private String dealWithPath(String path) { + String pathWithPrefix = path; + if (!StrUtil.startsWith(path, "s3a://")) { + if (StrUtil.startsWith(path, "//")) { + pathWithPrefix = "s3a:" + path; + } else if (StrUtil.startsWith(path, "/")) { + pathWithPrefix = "s3a:/" + path; + } else { + pathWithPrefix = "s3a://" + path; + } + } + + if (!StrUtil.endsWith(pathWithPrefix,"/")) { + pathWithPrefix = pathWithPrefix + "/"; + } + + LOG.debug("Path = " + pathWithPrefix); + return pathWithPrefix; + } +} diff --git a/flinkx-oss/pom.xml b/flinkx-oss/pom.xml new file mode 100644 index 0000000000..510558dc36 --- /dev/null +++ b/flinkx-oss/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-oss + pom + + flinkx-oss-core + flinkx-oss-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index f7a9216bdd..f41bb7b38e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -264,6 +264,7 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ case PluginNameConstants.PHOENIX5_WRITER : writer = new Phoenix5Writer(config); break; case PluginNameConstants.KINGBASE_WRITER : writer = new KingbaseWriter(config); break; case PluginNameConstants.RESTAPI_WRITER: writer = new RedisWriter(config); break; + case PluginNameConstants.OSS_WRITER: writer = new OssWriter(config); break; default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index c3a3dcd36a..79606ca67e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -87,4 +87,5 @@ public class PluginNameConstants { public static final String GREENPLUM_WRITER = "greenplumwriter"; public static final String PHOENIX5_WRITER = "phoenix5writer"; public static final String KINGBASE_WRITER = "kingbasewriter"; + public static final String OSS_WRITER = "osswriter"; } diff --git a/pom.xml b/pom.xml index b8ac68013b..703eb34d87 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,7 @@ flinkx-kingbase flinkx-hdfs + flinkx-oss flinkx-hive flinkx-es flinkx-ftp @@ -66,6 +67,7 @@ 1.11.3 2.12 2.7.3 + 2.3.1 4.5.3 ${basedir}/dev release_1.11.0 From 75fe2982798f604efc441abcb1339b21394d5f57 Mon Sep 17 00:00:00 2001 From: lkjhchen2 <30791835+lkjhchen2@users.noreply.github.com> Date: Tue, 7 Dec 2021 17:40:37 +0800 Subject: [PATCH 128/136] =?UTF-8?q?[hotfix-539][core]=E4=BF=AE=E5=A4=8DDat?= =?UTF-8?q?eUtil=E5=B7=A5=E5=85=B7=E7=B1=BB,=20=E9=81=87=E5=88=B0LocalDate?= =?UTF-8?q?Time=E5=92=8CLocalDate=E7=B1=BB=E5=9E=8B=E8=BD=AC=E5=8C=96,=20?= =?UTF-8?q?=E4=BB=BB=E5=8A=A1=E8=BF=90=E8=A1=8C=E5=A4=B1=E8=B4=A5=E7=9A=84?= =?UTF-8?q?=E9=97=AE=E9=A2=98=20(#540)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flinkx/util/ClassUtil.java | 1 + .../com/dtstack/flinkx/util/DateUtil.java | 124 ++++++++++-------- 2 files changed, 73 insertions(+), 52 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java index b073524412..7fb6aae459 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/ClassUtil.java @@ -38,6 +38,7 @@ public class ClassUtil { public static void forName(String clazz, ClassLoader classLoader) { synchronized (LOCK_STR){ try { + LOG.info("className = " + clazz); Class.forName(clazz, true, classLoader); DriverManager.setLoginTimeout(10); } catch (Exception e) { diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java index 660afb3f2f..935a654128 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/DateUtil.java @@ -23,6 +23,10 @@ import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.Calendar; import java.util.Date; import java.util.HashMap; @@ -66,32 +70,32 @@ public class DateUtil { public final static int LENGTH_NANOSECOND = 19; public static ThreadLocal> datetimeFormatter = ThreadLocal.withInitial(() -> { - TimeZone timeZone = TimeZone.getTimeZone(TIME_ZONE); + TimeZone timeZone = TimeZone.getTimeZone(TIME_ZONE); - Map formatterMap = new HashMap<>(); - SimpleDateFormat standardDatetimeFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); - standardDatetimeFormatter.setTimeZone(timeZone); - formatterMap.put(STANDARD_DATETIME_FORMAT,standardDatetimeFormatter); + Map formatterMap = new HashMap<>(); + SimpleDateFormat standardDatetimeFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + standardDatetimeFormatter.setTimeZone(timeZone); + formatterMap.put(STANDARD_DATETIME_FORMAT,standardDatetimeFormatter); - SimpleDateFormat unStandardDatetimeFormatter = new SimpleDateFormat("yyyyMMddHHmmss"); - unStandardDatetimeFormatter.setTimeZone(timeZone); - formatterMap.put(UN_STANDARD_DATETIME_FORMAT,unStandardDatetimeFormatter); + SimpleDateFormat unStandardDatetimeFormatter = new SimpleDateFormat("yyyyMMddHHmmss"); + unStandardDatetimeFormatter.setTimeZone(timeZone); + formatterMap.put(UN_STANDARD_DATETIME_FORMAT,unStandardDatetimeFormatter); - SimpleDateFormat dateFormatter = new SimpleDateFormat("yyyy-MM-dd"); - dateFormatter.setTimeZone(timeZone); - formatterMap.put(DATE_FORMAT,dateFormatter); + SimpleDateFormat dateFormatter = new SimpleDateFormat("yyyy-MM-dd"); + dateFormatter.setTimeZone(timeZone); + formatterMap.put(DATE_FORMAT,dateFormatter); - SimpleDateFormat timeFormatter = new SimpleDateFormat("HH:mm:ss"); - timeFormatter.setTimeZone(timeZone); - formatterMap.put(TIME_FORMAT,timeFormatter); + SimpleDateFormat timeFormatter = new SimpleDateFormat("HH:mm:ss"); + timeFormatter.setTimeZone(timeZone); + formatterMap.put(TIME_FORMAT,timeFormatter); - SimpleDateFormat yearFormatter = new SimpleDateFormat("yyyy"); - yearFormatter.setTimeZone(timeZone); - formatterMap.put(YEAR_FORMAT,yearFormatter); + SimpleDateFormat yearFormatter = new SimpleDateFormat("yyyy"); + yearFormatter.setTimeZone(timeZone); + formatterMap.put(YEAR_FORMAT,yearFormatter); - SimpleDateFormat standardDatetimeFormatterOfMillisecond = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); - standardDatetimeFormatterOfMillisecond.setTimeZone(timeZone); - formatterMap.put(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND,standardDatetimeFormatterOfMillisecond); + SimpleDateFormat standardDatetimeFormatterOfMillisecond = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + standardDatetimeFormatterOfMillisecond.setTimeZone(timeZone); + formatterMap.put(STANDARD_DATETIME_FORMAT_FOR_MILLISECOND,standardDatetimeFormatterOfMillisecond); return formatterMap; }); @@ -99,9 +103,9 @@ public class DateUtil { private DateUtil() {} public static java.sql.Date columnToDate(Object column,SimpleDateFormat customTimeFormat) { - if(column == null) { + if (column == null) { return null; - } else if(column instanceof String) { + } else if (column instanceof String) { if (((String) column).length() == 0){ return null; } @@ -119,18 +123,26 @@ public static java.sql.Date columnToDate(Object column,SimpleDateFormat customTi return new java.sql.Date(getMillSecond(rawData.toString())); } else if (column instanceof java.sql.Date) { return (java.sql.Date) column; - } else if(column instanceof Timestamp) { + } else if (column instanceof Timestamp) { Timestamp ts = (Timestamp) column; return new java.sql.Date(ts.getTime()); - } else if(column instanceof Date) { + } else if (column instanceof Date) { Date d = (Date)column; return new java.sql.Date(d.getTime()); + } else if (column instanceof LocalDate) { + LocalDate localDate = (LocalDate) column; + return new java.sql.Date( + localDate.atStartOfDay(ZoneId.systemDefault()).toInstant().toEpochMilli()); + } else if (column instanceof LocalDateTime) { + LocalDateTime localDateTime = (LocalDateTime) column; + return new java.sql.Date( + localDateTime.toInstant(ZoneOffset.of("+8")).toEpochMilli()); } throw new IllegalArgumentException("Can't convert " + column.getClass().getName() + " to Date"); } - public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateFormat customTimeFormat) { + public static java.sql.Timestamp columnToTimestamp(Object column, SimpleDateFormat customTimeFormat) { if (column == null) { return null; } else if(column instanceof String) { @@ -153,25 +165,33 @@ public static java.sql.Timestamp columnToTimestamp(Object column,SimpleDateForma return new java.sql.Timestamp(((java.sql.Date) column).getTime()); } else if(column instanceof Timestamp) { return (Timestamp) column; - } else if(column instanceof Date) { + } else if (column instanceof Date) { Date d = (Date)column; return new java.sql.Timestamp(d.getTime()); + } else if (column instanceof LocalDateTime) { + LocalDateTime localDateTime = (LocalDateTime) column; + return new java.sql.Timestamp( + localDateTime.toInstant(ZoneOffset.of("+8")).toEpochMilli()); + } else if (column instanceof LocalDate) { + LocalDate localDate = (LocalDate) column; + return new java.sql.Timestamp( + localDate.atStartOfDay(ZoneId.systemDefault()).toInstant().toEpochMilli()); } throw new IllegalArgumentException("Can't convert " + column.getClass().getName() + " to Date"); } - public static long getMillSecond(String data){ + public static long getMillSecond(String data) { long time = Long.parseLong(data); - if(data.length() == LENGTH_SECOND){ + if (data.length() == LENGTH_SECOND) { time = Long.parseLong(data) * 1000; - } else if(data.length() == LENGTH_MILLISECOND){ + } else if (data.length() == LENGTH_MILLISECOND) { time = Long.parseLong(data); - } else if(data.length() == LENGTH_MICROSECOND){ + } else if (data.length() == LENGTH_MICROSECOND) { time = Long.parseLong(data) / 1000; - } else if(data.length() == LENGTH_NANOSECOND){ + } else if (data.length() == LENGTH_NANOSECOND) { time = Long.parseLong(data) / 1000000 ; - } else if(data.length() < LENGTH_SECOND){ + } else if (data.length() < LENGTH_SECOND) { try { long day = Long.parseLong(data); Date date = datetimeFormatter.get().get(DATE_FORMAT).parse(START_TIME); @@ -179,18 +199,18 @@ public static long getMillSecond(String data){ long addMill = date.getTime() + day * 24 * 3600 * 1000; cal.setTimeInMillis(addMill); time = cal.getTimeInMillis(); - } catch (Exception ignore){ + } catch (Exception ignore) { } } return time; } - public static Date stringToDate(String strDate,SimpleDateFormat customTimeFormat) { - if(strDate == null || strDate.trim().length() == 0) { + public static Date stringToDate(String strDate, SimpleDateFormat customTimeFormat) { + if (strDate == null || strDate.trim().length() == 0) { return null; } - if(customTimeFormat != null){ + if (customTimeFormat != null) { try { return customTimeFormat.parse(strDate); } catch (ParseException ignored) { @@ -270,51 +290,51 @@ public static SimpleDateFormat buildDateFormatter(String timeFormat){ * @return String DateFormat字符串如:yyyy-MM-dd HH:mm:ss */ public static String getDateFormat(String str) { - if(StringUtils.isBlank(str)){ + if (StringUtils.isBlank(str)) { return null; } boolean year = false; Pattern pattern = Pattern.compile("^[-\\+]?[\\d]*$"); - if(pattern.matcher(str.substring(0, 4)).matches()) { + if (pattern.matcher(str.substring(0, 4)).matches()) { year = true; } StringBuilder sb = new StringBuilder(); int index = 0; - if(!year) { - if(str.contains("月") || str.contains("-") || str.contains("/")) { - if(Character.isDigit(str.charAt(0))) { + if (!year) { + if (str.contains("月") || str.contains("-") || str.contains("/")) { + if (Character.isDigit(str.charAt(0))) { index = 1; } - }else { + } else { index = 3; } } for (int i = 0; i < str.length(); i++) { char chr = str.charAt(i); - if(Character.isDigit(chr)) { - if(index==0) { + if (Character.isDigit(chr)) { + if (index==0) { sb.append("y"); } - if(index==1) { + if (index==1) { sb.append("M"); } - if(index==2) { + if (index==2) { sb.append("d"); } - if(index==3) { + if (index==3) { sb.append("H"); } - if(index==4) { + if (index==4) { sb.append("m"); } - if(index==5) { + if (index==5) { sb.append("s"); } - if(index==6) { + if (index==6) { sb.append("S"); } - }else { - if(i>0) { + } else { + if (i > 0) { char lastChar = str.charAt(i-1); if(Character.isDigit(lastChar)) { index++; From 0519f7081b40e877a5d4706ab1b9c5715e158625 Mon Sep 17 00:00:00 2001 From: lkjhchen2 <30791835+lkjhchen2@users.noreply.github.com> Date: Tue, 7 Dec 2021 17:41:22 +0800 Subject: [PATCH 129/136] =?UTF-8?q?[hotfix-541][core]=20=E4=BF=AE=E5=A4=8D?= =?UTF-8?q?hdfs=E8=BE=93=E5=87=BA=E7=B1=BB=E9=81=87=E5=88=B0=E8=AF=BB?= =?UTF-8?q?=E5=8F=96boolean=E7=B1=BB=E5=9E=8B,=20=E5=92=8C=E9=A2=84?= =?UTF-8?q?=E6=9C=9F=E7=BB=93=E6=9E=9C=E4=B8=8D=E5=90=8C=E7=9A=84=E9=97=AE?= =?UTF-8?q?=E9=A2=98=20(#546)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flinkx/util/StringUtil.java | 15 +++++++++++++++ .../hdfs/writer/HdfsParquetOutputFormat.java | 3 ++- .../flinkx/hdfs/writer/HdfsTextOutputFormat.java | 3 ++- 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java index 473e2816a0..76ff4e99a8 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/util/StringUtil.java @@ -339,4 +339,19 @@ public static String splitIgnoreQuotaAndJoinByPoint(String table) { } return stringBuffer.toString(); } + + public static Boolean parseBoolean(String str) { + if (null == str || "null".equalsIgnoreCase(str)) { + return Boolean.FALSE; + } + + if ("1".equals(str)) { + return Boolean.TRUE; + } else if ("0".equals(str)) { + return Boolean.FALSE; + } else { + return Boolean.parseBoolean(str); + } + } + } diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java index f635c61348..02841a3ea1 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsParquetOutputFormat.java @@ -26,6 +26,7 @@ import com.dtstack.flinkx.util.DateUtil; import com.dtstack.flinkx.util.FileSystemUtil; import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.types.Row; import org.apache.hadoop.fs.FileSystem; @@ -246,7 +247,7 @@ private void addDataToGroup(Group group, Object valObj, int i) throws Exception{ group.add(colName,val); } break; - case "boolean" : group.add(colName,Boolean.parseBoolean(val));break; + case "boolean" : group.add(colName,StringUtil.parseBoolean(val));break; case "timestamp" : Timestamp ts = DateUtil.columnToTimestamp(valObj,null); byte[] dst = HdfsUtil.longToByteArray(ts.getTime()); diff --git a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java index e73ac5d850..daa9b32478 100644 --- a/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java +++ b/flinkx-hdfs/flinkx-hdfs-writer/src/main/java/com/dtstack/flinkx/hdfs/writer/HdfsTextOutputFormat.java @@ -23,6 +23,7 @@ import com.dtstack.flinkx.hdfs.ECompressType; import com.dtstack.flinkx.hdfs.HdfsUtil; import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.StringUtil; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.apache.flink.types.Row; @@ -215,7 +216,7 @@ private void appendDataToString(StringBuilder sb, Object column, ColumnType colu } break; case BOOLEAN: - sb.append(Boolean.valueOf(rowData)); + sb.append(StringUtil.parseBoolean(rowData)); break; case DATE: column = DateUtil.columnToDate(column,null); From c41cfb2f5f3965b57b4894a316ef52f3e1d28834 Mon Sep 17 00:00:00 2001 From: fengjiangtao Date: Tue, 7 Dec 2021 17:42:57 +0800 Subject: [PATCH 130/136] =?UTF-8?q?[feat-554][flinkx-hudi]=20=E5=A2=9E?= =?UTF-8?q?=E5=8A=A0flinkx-hudi-writer=E6=A8=A1=E5=9D=97=20(#558)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-hudi/flinkx-hudi-core/pom.xml | 14 + .../dtstack/flinkx/hudi/HudiConfigKeys.java | 73 +++++ flinkx-hudi/flinkx-hudi-writer/pom.xml | 288 +++++++++++++++++ .../HudiOutputFormat.java | 303 ++++++++++++++++++ .../HudiOutputformatBuilder.java | 105 ++++++ .../HudiWriter.java | 151 +++++++++ flinkx-hudi/pom.xml | 37 +++ pom.xml | 1 + 8 files changed, 972 insertions(+) create mode 100644 flinkx-hudi/flinkx-hudi-core/pom.xml create mode 100644 flinkx-hudi/flinkx-hudi-core/src/main/java/com/dtstack/flinkx/hudi/HudiConfigKeys.java create mode 100644 flinkx-hudi/flinkx-hudi-writer/pom.xml create mode 100644 flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputFormat.java create mode 100644 flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputformatBuilder.java create mode 100644 flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiWriter.java create mode 100644 flinkx-hudi/pom.xml diff --git a/flinkx-hudi/flinkx-hudi-core/pom.xml b/flinkx-hudi/flinkx-hudi-core/pom.xml new file mode 100644 index 0000000000..ce04a42e5c --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-core/pom.xml @@ -0,0 +1,14 @@ + + + + flinkx-hudi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hudi-core + + \ No newline at end of file diff --git a/flinkx-hudi/flinkx-hudi-core/src/main/java/com/dtstack/flinkx/hudi/HudiConfigKeys.java b/flinkx-hudi/flinkx-hudi-core/src/main/java/com/dtstack/flinkx/hudi/HudiConfigKeys.java new file mode 100644 index 0000000000..0b9277369c --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-core/src/main/java/com/dtstack/flinkx/hudi/HudiConfigKeys.java @@ -0,0 +1,73 @@ +package com.dtstack.flinkx.hudi;/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * @author fengjiangtao_yewu@cmss.chinamobile.com + * @date 2021-08-05 + */ +public class HudiConfigKeys { + public static final String KEY_PATH = "path"; + + public static final String KEY_HADOOP_CONFIG = "hadoopConfig"; + + public static final String KEY_DEFAULT_FS = "defaultFS"; + + public static final String KEY_WRITE_MODE = "writeMode"; + + public static final String KEY_COLUMN_NAME = "name"; + + public static final String KEY_COLUMN_TYPE = "type"; + + public static final String KEY_COMPRESS = "compress"; + + public static final String KEY_TABLE_NAME = "tableName"; + + public static final String KEY_TABLE_TYPE = "tableType"; + + public static final String KEY_TABLE_RECORD_KEY = "recordKey"; + + public static final String KEY_TABLE_TYPE_RECORD = "record"; + + public static final String KEY_SCHEMA_FIELDS = "fields"; + + public static final String KEY_PARTITION_FIELDS = "partitionFields"; + + public static final String KEY_HIVE_JDBC = "hiveJdbcUrl"; + + public static final String KEY_HIVE_METASTORE = "hiveMetastore"; + + public static final String KEY_HIVE_USER = "hiveUser"; + + public static final String KEY_HIVE_PASS = "hivePass"; + + public static final String KEY_BATCH_INTERVAL = "batchInterval"; + + public static final String KEY_HA_DEFAULT_FS = "dfs.nameservices"; + + public static final String KEY_HIVE_METASTORE_URIS = "hive.metastore.uris"; + + public static final String KEY_LAKEHOUSE_METADATAURL = "lakehouse.metadataUrl"; + + public static final String KEY_LAKEHOUSE_JOBUUID = "lakehouse.jobUUID"; + + public static final String KEY_LAKEHOUSE_USERID = "lakehouse.userId"; + + public static final String KEY_HADOOP_USER_NAME = "HADOOP_USER_NAME"; + + public static final String KEY_DBTABLE_DELIMITER = "."; +} diff --git a/flinkx-hudi/flinkx-hudi-writer/pom.xml b/flinkx-hudi/flinkx-hudi-writer/pom.xml new file mode 100644 index 0000000000..49ef214452 --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-writer/pom.xml @@ -0,0 +1,288 @@ + + + + flinkx-hudi + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hudi-writer + + + + + org.anarres.lzo + lzo-core + 1.0.2 + + + org.anarres.lzo + lzo-hadoop + 1.0.5 + + + hadoop-core + org.apache.hadoop + + + + + + com.dtstack.flinkx + flinkx-hudi-core + 1.6 + + + httpcore + org.apache.httpcomponents + + + httpclient + org.apache.httpcomponents + + + + + + com.dtstack.flinkx + flinkx-hudi-core + 1.6 + compile + + + org.apache.flink + flink-table-common + ${flink.version} + compile + + + org.apache.hudi + hudi-flink_2.12 + ${hudi.version} + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-client + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-io + + + org.eclipse.jetty + jetty-rewrite + + + org.eclipse.jetty + jetty-security + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-xml + + + org.apache.hive + hive-exec + + + org.apache.hudi + hudi-common + + + org.apache.hudi + hudi-client-common + + + org.apache.hudi + hudi-java-client + + + org.apache.hudi + hudi-hive-sync + + + + + org.apache.hudi + hudi-common + ${hudi.version} + shade + + + org.apache.hudi + hudi-java-client + ${hudi.version} + shade + + + org.slf4j + slf4j-api + + + + + org.apache.hudi + hudi-hive-sync + ${hudi.version} + shade + + + org.eclipse.jetty + jetty-server + ${jetty-server.version} + + + org.apache.avro + avro + ${avro.version} + + + org.apache.parquet + parquet-avro + ${parquet-avro.version} + + + org.apache.hive + hive-service + ${hive.version} + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hive + hive-jdbc + ${hive.version} + + + org.eclipse.jetty.orbit + javax.servlet + + + org.apache.parquet + parquet-hadoop-bundle + + + + + org.apache.calcite + calcite-core + 1.10.0 + + + com.alibaba + fastjson + 1.2.76 + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + ch.qos.logback:* + com.google.code.gson:* + com.data-artisans:* + org.scala-lang:* + io.netty:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputFormat.java b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputFormat.java new file mode 100644 index 0000000000..26c8aa0143 --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputFormat.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.hudi.writer; + +import com.dtstack.flinkx.hudi.HudiConfigKeys; +import com.dtstack.flinkx.outputformat.BaseRichOutputFormat; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.util.FileSystemUtil; +import com.dtstack.flinkx.util.GsonUtil; +import com.dtstack.flinkx.util.StringUtil; +import com.google.common.collect.Lists; +import org.apache.flink.types.Row; +import org.apache.flink.util.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.HiveSyncTool; +import org.apache.hudi.hive.ddl.HiveSyncMode; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.org.apache.avro.Schema; +import org.apache.hudi.org.apache.avro.generic.GenericData; +import org.apache.hudi.org.apache.avro.generic.GenericRecord; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.dtstack.flinkx.hudi.HudiConfigKeys.KEY_HADOOP_USER_NAME; + +/** + * Reference: org.apache.hudi.examples.java.HoodieJavaWriteClientExample + * + * @author fengjiangtao_yewu@cmss.chinamobile.com + * @date 2021-08-10 + */ + +public class HudiOutputFormat extends BaseRichOutputFormat { + protected String tableName; + protected String tableType; + protected String path; + protected String schema; + protected String defaultFS; + protected String recordKey; + protected String hiveJdbcUrl; + protected String hiveMetastore; + protected String hiveUser; + protected String hivePass; + protected Schema avroSchema; + protected String[] dbTableName; + protected Map hadoopConfig; + protected List metaColumns; + protected List partitionFields; + + protected transient HoodieJavaWriteClient client; + protected FileSystem fs; + protected Configuration hadoopConfiguration; + protected HiveConf hiveConf; + protected HiveSyncConfig hiveSyncConfig; + + @Override + protected void openInternal(int taskNumber, int numTasks) { + dbTableName = org.apache.commons.lang3.StringUtils.split(tableName, "."); + // Create the write client to write some records in + HoodieWriteConfig hudiWriteConfig = HoodieWriteConfig.newBuilder() + .withEngineType(EngineType.FLINK).withPath(path) + .withSchema(schema).withParallelism(numTasks, numTasks) + .withDeleteParallelism(numTasks).forTable(dbTableName[1]) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + + hadoopConfiguration = FileSystemUtil.getConfiguration(hadoopConfig, defaultFS); + client = new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(hadoopConfiguration), hudiWriteConfig); + avroSchema = new Schema.Parser().parse(schema); + + if (hadoopConfig.containsKey(KEY_HADOOP_USER_NAME)) { + // Config the HADOOP_USER_NAME for permission. + LOG.info("Default System HADOOP_USER_NAME:" + System.getProperty(KEY_HADOOP_USER_NAME)); + System.setProperty(KEY_HADOOP_USER_NAME, hadoopConfig.get(KEY_HADOOP_USER_NAME).toString()); + LOG.info("Change System HADOOP_USER_NAME:" + System.getProperty(KEY_HADOOP_USER_NAME)); + } + + LOG.info("Init hudi table schema:[{}]", schema); + initTable(hadoopConfiguration, dbTableName[1]); + initHiveConf(); + } + + @Override + protected void writeSingleRecordInternal(Row row) { + String newCommitTime = client.startCommit(); + HoodieRecord hoodieRecord = buildHudiRecords(row); + List> records = Lists.newArrayList(hoodieRecord).stream() + .filter(record -> record != null).collect(Collectors.toList()); + + if (records.size() > 0) { + client.upsert(records, newCommitTime); + syncHiveMeta(); + } + } + + /** + * Multiple write cost less when every writing needs to sync hive Metastore. + */ + @Override + protected void writeMultipleRecordsInternal() { + String newCommitTime = client.startCommit(); + + List> records = rows.stream().filter(record -> record != null) + .map(this::buildHudiRecords).collect(Collectors.toList()); + if (records.size() > 0) { + client.upsert(records, newCommitTime); + syncHiveMeta(); + } + } + + /** + * Build hudi record by row data. + * + * @param row + * @return + */ + private HoodieRecord buildHudiRecords(Row row) { + HoodieKey key = new HoodieKey(); + // Set first key as recordKey + Option genericRecord; + HoodieAvroPayload hoodieAvroPayload; + HoodieRecord hoodieRecord; + try { + key.setRecordKey(recordKey); + // TODO Next version will support partition table. + key.setPartitionPath(""); + + genericRecord = Option.of(buildGenericRecord(row)); + if (!genericRecord.isPresent()) { + return null; + } + hoodieAvroPayload = new HoodieAvroPayload(genericRecord); + hoodieRecord = new HoodieRecord<>(key, hoodieAvroPayload); + } catch (Exception e) { + LOG.error("Build hudi records err. Row:" + row.toString(), e); + throw new RuntimeException(e); + } + + return hoodieRecord; + } + + /** + * Init the table + * + * @param hadoopConf + * @param splitTableName + */ + private void initTable(Configuration hadoopConf, String splitTableName) { + // initialize the table, if not done already + Path path = new Path(this.path); + try { + fs = FSUtils.getFs(this.path, hadoopConf); + if (!fs.exists(path)) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder() + .setTableType(tableType) + .setTableName(splitTableName) + .setPayloadClassName(HoodieAvroPayload.class.getName()) + .initTable(hadoopConf, this.path); + + LOG.info("Hudi table init done.", metaClient.getMetaPath()); + } + } catch (Exception e) { + LOG.warn("Hudi table init err. " + tableName, e); + throw new RuntimeException("Create hudi table failed:" + splitTableName + ", " + e.getMessage()); + } + } + + /** + * Init hive configuration. + */ + private void initHiveConf() { + hiveConf = new HiveConf(); + hiveConf.set(HudiConfigKeys.KEY_HIVE_METASTORE_URIS, hiveMetastore); + hiveConf.set(HudiConfigKeys.KEY_HA_DEFAULT_FS, defaultFS); + + Iterator> iterator = hadoopConfiguration.iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + hiveConf.set(entry.getKey(), entry.getValue()); + } + + hiveSyncConfig = new HiveSyncConfig(); + hiveSyncConfig.jdbcUrl = this.hiveJdbcUrl; + hiveSyncConfig.autoCreateDatabase = true; + hiveSyncConfig.databaseName = this.dbTableName[0]; + hiveSyncConfig.tableName = this.dbTableName[1]; + hiveSyncConfig.basePath = this.path; + hiveSyncConfig.assumeDatePartitioning = true; + hiveSyncConfig.usePreApacheInputFormat = false; + hiveSyncConfig.createManagedTable = true; + if (org.apache.commons.lang3.StringUtils.isNotBlank(this.hiveUser)) { + hiveSyncConfig.hiveUser = this.hiveUser; + } + if (org.apache.commons.lang3.StringUtils.isNotBlank(this.hivePass)) { + hiveSyncConfig.hivePass = this.hivePass; + } + if (partitionFields.size() > 0) { + hiveSyncConfig.partitionFields = partitionFields; + } + + // Default use HIVEQL sync mode + hiveSyncConfig.syncMode = HiveSyncMode.HIVEQL.name(); + } + + /** + * Sync the hive meta after every upsert batch. + * + * @return + */ + private boolean syncHiveMeta() { + try { + HiveSyncTool hiveSyncTool = new HiveSyncTool(hiveSyncConfig, hiveConf, fs); + hiveSyncTool.syncHoodieTable(); + } catch (Exception e) { + LOG.error("Create hive meta failed " + tableName, e); + throw new RuntimeException("Sync hudi to hive metastore failed.", e); + } + + return true; + } + + /** + * Schema for example: + * { + * "type": "record", + * "name": "triprec", + * "fields": [ + * {"name": "ts","type": "long"}, + * {"name": "uuid","type": "string"}, + * {"name": "begin_lat","type": "double"} + * ] + * } + * + * @param row + * @return + */ + private GenericRecord buildGenericRecord(Row row) { + GenericRecord rec = new GenericData.Record(avroSchema); + + try { + int arity = row.getArity(); + if (metaColumns != null && metaColumns.size() > 0) { + for (int i = 0; i < arity; i++) { + String value = StringUtils.arrayAwareToString(row.getField(i)); + rec.put(metaColumns.get(i).getName(), StringUtil.string2col(value, metaColumns.get(i).getType(), null)); + } + } else { + Map map = GsonUtil.GSON.fromJson(row.getField(0).toString(), GsonUtil.gsonMapTypeToken); + map.keySet().stream().forEach(key -> rec.put(key, map.get(key))); + } + } catch (Exception e) { + LOG.warn("Build genericRecord err." + row.toString(), e); + return null; + } + + return rec; + } + + /** + * Close client. + */ + @Override + public void closeInternal() { + LOG.warn("Hudi output closeInternal."); + if (null != client) { + client.close(); + } + } +} diff --git a/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputformatBuilder.java b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputformatBuilder.java new file mode 100644 index 0000000000..36d9825a39 --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiOutputformatBuilder.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.hudi.writer; + +import com.dtstack.flinkx.outputformat.BaseRichOutputFormatBuilder; +import com.dtstack.flinkx.reader.MetaColumn; +import org.apache.avro.Schema; + +import java.util.List; +import java.util.Map; + +/** + * @author fengjiangtao_yewu@cmss.chinamobile.com + * @date 2021-08-10 + */ + +public class HudiOutputformatBuilder extends BaseRichOutputFormatBuilder { + private HudiOutputFormat format; + + public HudiOutputformatBuilder() { + super.format = format = new HudiOutputFormat(); + } + + public void setTableName(String tableName) { + format.tableName = tableName; + } + + public void setTableType(String tableType) { + format.tableType = tableType; + } + public void setRecordKey(String recordKey) { + format.recordKey = recordKey; + } + + public void setPath(String path) { + format.path = path; + } + + public void setHadoopConf(Map hadoopConf) { + format.hadoopConfig = hadoopConf; + } + + public void setDefaultFS(String defaultFS) { + format.defaultFS = defaultFS; + } + + public void setHiveJdbcUrl(String hiveJdbcUrl) { + format.hiveJdbcUrl = hiveJdbcUrl; + } + + public void setHiveMetastore(String hiveMetastore) { + format.hiveMetastore = hiveMetastore; + } + + public void setHiveUser(String hiveUser) { + format.hiveUser = hiveUser; + } + + public void setHivePass(String hivePass) { + format.hivePass = hivePass; + } + + public void setSchema(String schema) { + format.schema = schema; + } + + public void setColumns(List metaColumns) { + format.metaColumns = metaColumns; + } + + public void setPartitionFields(List partitionFields) { + format.partitionFields = partitionFields; + } + + /** + * Column type comes form org.apache.avro.Schema.Type + */ + @Override + protected void checkFormat() { + MetaColumn mColumn = null; + try { + for (MetaColumn metaColumn : format.metaColumns) { + mColumn = metaColumn; + Schema.Type.valueOf(metaColumn.getType().toUpperCase()); + } + } catch (Exception e) { + throw new UnsupportedOperationException("This plugin column's type not support: " + mColumn.getType()); + } + } +} diff --git a/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiWriter.java b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiWriter.java new file mode 100644 index 0000000000..25d7307dc4 --- /dev/null +++ b/flinkx-hudi/flinkx-hudi-writer/src/main/java/com.dtstack.flinkx.hudi.writer/HudiWriter.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.flinkx.hudi.writer; + +import com.alibaba.fastjson.JSONArray; +import com.alibaba.fastjson.JSONObject; +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.hudi.HudiConfigKeys; +import com.dtstack.flinkx.reader.MetaColumn; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; +import org.apache.hudi.common.model.HoodieTableType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.hudi.HudiConfigKeys.*; + +/** + * @author fengjiangtao_yewu@cmss.chinamobile.com + * @date 2021-08-10 + */ + +public class HudiWriter extends BaseDataWriter { + /** + * org.apache.hudi.common.model.WriteOperationType + * UPSERT | UPSERT_PREPPED + */ + protected String writeOperation; + + /** + * Table name to register to Hive metastore + */ + protected String tableName; + + /** + * Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ + */ + protected String tableType; + + /** + * Base path for the target hoodie table. + */ + protected String path; + + /** + * Async Compaction, enabled by default for MOR + */ + protected boolean compress; + + protected List metaColumns; + protected String defaultFS; + protected String recordKey; + protected String hiveJdbcUrl; + protected String hiveMetastore; + protected String hiveUser; + protected String hivePass; + protected int batchInterval; + protected List partitionFields; + protected Map hadoopConfig; + + public HudiWriter(DataTransferConfig config) { + super(config); + + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + writeOperation = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_WRITE_MODE); + tableName = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_TABLE_NAME); + tableType = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_TABLE_TYPE, HoodieTableType.COPY_ON_WRITE.name()); + recordKey = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_TABLE_RECORD_KEY, "id"); + path = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_PATH); + compress = writerConfig.getParameter().getBooleanVal(HudiConfigKeys.KEY_COMPRESS, Boolean.FALSE); + metaColumns = MetaColumn.getMetaColumns(writerConfig.getParameter().getColumn()); + defaultFS = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_DEFAULT_FS); + hiveJdbcUrl = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_HIVE_JDBC); + hiveMetastore = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_HIVE_METASTORE); + hiveUser = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_HIVE_USER, ""); + hivePass = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_HIVE_PASS, ""); + batchInterval = writerConfig.getParameter().getIntVal(HudiConfigKeys.KEY_BATCH_INTERVAL, 1); + String partitionField = writerConfig.getParameter().getStringVal(HudiConfigKeys.KEY_PARTITION_FIELDS); + partitionFields = StringUtils.isNotBlank(partitionField) ? Arrays.asList(StringUtils.split(partitionField, ",")) : new ArrayList<>(); + hadoopConfig = (Map) writerConfig.getParameter().getVal(HudiConfigKeys.KEY_HADOOP_CONFIG); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + HudiOutputformatBuilder builder = new HudiOutputformatBuilder(); + builder.setTableName(tableName); + builder.setTableType(tableType); + builder.setRecordKey(recordKey); + builder.setPath(path); + builder.setHadoopConf(hadoopConfig); + builder.setDefaultFS(defaultFS); + builder.setHiveJdbcUrl(hiveJdbcUrl); + builder.setHiveMetastore(hiveMetastore); + builder.setHiveUser(hiveUser); + builder.setHivePass(hivePass); + builder.setSchema(buildSchema(metaColumns, tableName)); + builder.setColumns(metaColumns); + builder.setPartitionFields(partitionFields); + builder.setBatchInterval(batchInterval); + + return createOutput(dataSet, builder.finish()); + } + + /** + * Transform MetaColumn to org.apache.avro.Schema. + * + * @param metaColumns + * @param tableName + * @return + */ + private String buildSchema(List metaColumns, String tableName) { + String[] dbTableName = StringUtils.split(tableName, "."); + JSONArray jsonArray = new JSONArray(); + metaColumns.forEach(metaColumn -> { + JSONObject jsonField = new JSONObject(); + jsonField.put(KEY_COLUMN_NAME, metaColumn.getName()); + jsonField.put(KEY_COLUMN_TYPE, metaColumn.getType()); + jsonArray.add(jsonField); + }); + + JSONObject schemaJson = new JSONObject(); + schemaJson.put(KEY_COLUMN_NAME, dbTableName[1]); + schemaJson.put(KEY_COLUMN_TYPE, KEY_TABLE_TYPE_RECORD); + schemaJson.put(KEY_SCHEMA_FIELDS, jsonArray); + + return schemaJson.toString(); + } + +} diff --git a/flinkx-hudi/pom.xml b/flinkx-hudi/pom.xml new file mode 100644 index 0000000000..c5b675bf5d --- /dev/null +++ b/flinkx-hudi/pom.xml @@ -0,0 +1,37 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-hudi-core + flinkx-hudi-writer + + + flinkx-hudi + pom + + + 0.9.0 + 2.3.1 + + 1.8.2 + 1.11.1 + 9.4.15.v20190215 + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index b8ac68013b..af22d481a1 100644 --- a/pom.xml +++ b/pom.xml @@ -59,6 +59,7 @@ flinkx-restapi flinkx-sqlservercdc flinkx-oraclelogminer + flinkx-hudi From 6287466075d78dbb2421807233abd3a5c58ff2c4 Mon Sep 17 00:00:00 2001 From: fengjiangtao Date: Tue, 7 Dec 2021 20:47:21 +0800 Subject: [PATCH 131/136] =?UTF-8?q?[docs-554][flinkx-hudi]=20=E5=A2=9E?= =?UTF-8?q?=E5=8A=A0kafka=E5=86=99=E5=85=A5hudi=E5=AE=9E=E4=BE=8B=E6=96=87?= =?UTF-8?q?=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/realTime/writer/hudiwriter.md | 181 +++++++++++++++++++++++++++++ 1 file changed, 181 insertions(+) create mode 100644 docs/realTime/writer/hudiwriter.md diff --git a/docs/realTime/writer/hudiwriter.md b/docs/realTime/writer/hudiwriter.md new file mode 100644 index 0000000000..364ce95985 --- /dev/null +++ b/docs/realTime/writer/hudiwriter.md @@ -0,0 +1,181 @@ +# Hudi Writer + + + +- [一、插件名称](#一插件名称) +- [二、参数说明](#二参数说明) +- [三、配置示例](#三配置示例) + - [1、kafka2hudi](#1kafka2hudi) + + + +
+ +## 一、插件名称 + +**名称:hudiwriter**
+ +
+ +## 二、参数说明 + +- **batchInterval** + - 描述:单次批量写入数据条数,建议配置大于1 + - 必选:否 + - 字段类型:int + - 默认值:1 + +
+ +- **tableName** + - 描述:库表名 + - 必选:是 + - 字段类型:String + - 默认值:无 + - 注意:英文点号分隔的 {Database}.{Table} + +
+ +- **path** + - 描述:表所在HDFS路径 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **defaultFS** + - 描述:Hadoop hdfs文件系统namenode节点地址。格式:hdfs://ip:端口;例如:hdfs://127.0.0.1:9 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **hadoopConfig** + - 描述:集群HA模式时需要填写的namespace配置及其它配置 + - 必选:否 + - 字段类型:Map + - 默认值:无 + +
+ +- **column** + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"]。 + - 必选:是 + - 默认值:无 + - 字段类型:List + +
+ +- **hiveJdbcUrl** + - 描述:Hive jdbc链接地址,例如jdbc:hive2://127.0.0.1:9093 + - 必选:是 + - 默认值:无 + - 字段类型:String + +
+ +- **hiveMetastore** + - 描述:Hive Metastore元数据地址,例如thrift://127.0.0.1:9083 + - 必选:是 + - 默认值:无 + - 字段类型:String + +
+ +- **hiveUser** + - 描述:Hive用户名 + - 必选:否 + - 默认值:无 + - 字段类型:String + +- **hivePass** + - 描述:Hive用户对应密码 + - 必选:否 + - 默认值:无 + - 字段类型:String + +## 三、配置示例 + +### 1、kafka2hudi + +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "kafkareader", + "parameter": { + "blankIgnore": true, + "codec": "JSON", + "consumerSettings": { + "bootstrap.servers": "100.100.100.1:6667,100.100.100.2:6667" + }, + "groupId": "flink", + "metaColumns": [ + { + "name": "id", + "type": "int" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "age", + "type": "int" + } + ], + "mode": "latest-offset", + "topic": "flinkx01" + } + }, + "writer": { + "name": "hudiwriter", + "parameter": { + "batchInterval": 10, + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "age", + "type": "int" + } + ], + "defaultFS": "hdfs://ns1", + "hadoopConfig": { + "dfs.nameservices": "ns1", + "dfs.ha.namenodes.ns1": "nn1,nn2", + "dfs.client.failover.proxy.provider.ns1": "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider", + "dfs.namenode.rpc-address.ns1.nn2": "100.100.100.1:8020", + "dfs.namenode.rpc-address.ns1.nn1": "100.100.100.2:8020", + "fs.file.impl": "org.apache.hadoop.fs.LocalFileSystem", + "fs.hdfs.impl": "org.apache.hadoop.hdfs.DistributedFileSystem", + "HADOOP_USER_NAME": "hive" + }, + "hiveJdbcUrl": "jdbc:hive2://localhost:9083", + "hiveMetastore": "thrift://localhost:9083", + "hiveUser": "hive", + "path": "hdfs://ns1/spark_1/lakehouse", + "tableName": "test.flinkx01" + } + } + } + ], + "setting": { + "speed": { + "bytes": 0, + "channel": 1 + } + } + } +} +``` From c7f9da93d5721467bd3aa6ec4150f43e5702b201 Mon Sep 17 00:00:00 2001 From: wuzhongjian Date: Mon, 20 Dec 2021 10:13:38 +0800 Subject: [PATCH 132/136] =?UTF-8?q?[feat-570][flinkx-alluxio]=20=E5=A2=9E?= =?UTF-8?q?=E5=8A=A0flinkx-alluxio-writer=E6=A8=A1=E5=9D=97?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- flinkx-alluxio/flinkx-alluxio-core/pom.xml | 112 ++++++ .../flinkx/alluxio/AlluxioConfigKeys.java | 43 +++ .../dtstack/flinkx/alluxio/AlluxioUtil.java | 180 ++++++++++ .../dtstack/flinkx/alluxio/ECompressType.java | 80 +++++ .../flinkx/alluxio/util/StringUtil.java | 152 ++++++++ flinkx-alluxio/flinkx-alluxio-writer/pom.xml | 128 +++++++ .../writer/AlluxioOrcOutputFormat.java | 275 +++++++++++++++ .../writer/AlluxioOutputFormatBuilder.java | 79 +++++ .../writer/AlluxioParquetOutputFormat.java | 318 +++++++++++++++++ .../writer/AlluxioTextOutputFormat.java | 231 ++++++++++++ .../flinkx/alluxio/writer/AlluxioWriter.java | 161 +++++++++ .../writer/BaseAlluxioOutputFormat.java | 331 ++++++++++++++++++ flinkx-alluxio/pom.xml | 28 ++ .../flinkx/constants/ConstantValue.java | 1 + .../com/dtstack/flinkx/enums/WriteType.java | 30 ++ flinkx-test/pom.xml | 5 + .../com/dtstack/flinkx/test/LocalTest.java | 2 + .../flinkx/test/PluginNameConstants.java | 1 + pom.xml | 1 + 19 files changed, 2158 insertions(+) create mode 100644 flinkx-alluxio/flinkx-alluxio-core/pom.xml create mode 100644 flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioConfigKeys.java create mode 100644 flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioUtil.java create mode 100644 flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/ECompressType.java create mode 100644 flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/util/StringUtil.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/pom.xml create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOrcOutputFormat.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOutputFormatBuilder.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioParquetOutputFormat.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioTextOutputFormat.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioWriter.java create mode 100644 flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/BaseAlluxioOutputFormat.java create mode 100644 flinkx-alluxio/pom.xml create mode 100644 flinkx-core/src/main/java/com/dtstack/flinkx/enums/WriteType.java diff --git a/flinkx-alluxio/flinkx-alluxio-core/pom.xml b/flinkx-alluxio/flinkx-alluxio-core/pom.xml new file mode 100644 index 0000000000..e1e85b1685 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-core/pom.xml @@ -0,0 +1,112 @@ + + + + flinkx-alluxio + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-alluxio-core + + + 3.1.2 + + + + + org.alluxio + alluxio-shaded-client + 2.6.2 + + + + org.apache.hive + hive-exec + ${hive.version} + + + calcite-core + org.apache.calcite + + + calcite-avatica + org.apache.calcite + + + derby + org.apache.derby + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + org.apache.hive + hive-serde + ${hive.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-yarn-api + + + org.xerial.snappy + snappy-java + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + + + + parquet-hadoop + org.apache.parquet + 1.8.3 + + + org.xerial.snappy + snappy-java + + + + + + org.xerial.snappy + snappy-java + 1.1.4 + + + + \ No newline at end of file diff --git a/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioConfigKeys.java b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioConfigKeys.java new file mode 100644 index 0000000000..88a01ccb8f --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioConfigKeys.java @@ -0,0 +1,43 @@ +package com.dtstack.flinkx.alluxio; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioConfigKeys { + + public static final String KEY_FIELD_DELIMITER = "fieldDelimiter"; + + public static final String KEY_PATH = "path"; + + public static final String KEY_FILTER = "filterRegex"; + + public static final String KEY_FILE_TYPE = "fileType"; + + public static final String KEY_WRITE_MODE = "writeMode"; + + public static final String KEY_FULL_COLUMN_NAME_LIST = "fullColumnName"; + + public static final String KEY_FULL_COLUMN_TYPE_LIST = "fullColumnType"; + + public static final String KEY_COLUMN_NAME = "name"; + + public static final String KEY_COLUMN_TYPE = "type"; + + public static final String KEY_COMPRESS = "compress"; + + public static final String KEY_FILE_NAME = "fileName"; + + public static final String KEY_ENCODING = "encoding"; + + public static final String KEY_ROW_GROUP_SIZE = "rowGroupSize"; + + public static final String KEY_MAX_FILE_SIZE = "maxFileSize"; + + public static final String KEY_FLUSH_INTERVAL = "flushInterval"; + + public static final String KEY_ENABLE_DICTIONARY = "enableDictionary"; + + public static final String KEY_WRITE_TYPE = "writeType"; + +} diff --git a/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioUtil.java b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioUtil.java new file mode 100644 index 0000000000..a4545b33f9 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/AlluxioUtil.java @@ -0,0 +1,180 @@ +package com.dtstack.flinkx.alluxio; + +import com.dtstack.flinkx.enums.ColumnType; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.io.*; +import org.apache.parquet.io.api.Binary; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioUtil { + public static final String NULL_VALUE = "\\N"; + + private static final long NANO_SECONDS_PER_DAY = 86400_000_000_000L; + + private static final long JULIAN_EPOCH_OFFSET_DAYS = 2440588; + + private static final double SCALE_TWO = 2.0; + private static final double SCALE_TEN = 10.0; + private static final int BIT_SIZE = 8; + + public static Object getWritableValue(Object writable) { + Class clz = writable.getClass(); + Object ret = null; + + if (clz == IntWritable.class) { + ret = ((IntWritable) writable).get(); + } else if (clz == Text.class) { + ret = ((Text) writable).toString(); + } else if (clz == LongWritable.class) { + ret = ((LongWritable) writable).get(); + } else if (clz == ByteWritable.class) { + ret = ((ByteWritable) writable).get(); + } else if (clz == DateWritable.class) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof DoubleWritable) { + ret = ((DoubleWritable) writable).get(); + } else if (writable instanceof TimestampWritable) { + ret = ((TimestampWritable) writable).getTimestamp(); + } else if (writable instanceof DateWritable) { + ret = ((DateWritable) writable).get(); + } else if (writable instanceof FloatWritable) { + ret = ((FloatWritable) writable).get(); + } else if (writable instanceof BooleanWritable) { + ret = ((BooleanWritable) writable).get(); + } else { + ret = writable.toString(); + } + return ret; + } + + public static ObjectInspector columnTypeToObjectInspetor(ColumnType columnType) { + ObjectInspector objectInspector = null; + switch (columnType) { + case TINYINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Byte.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case SMALLINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Short.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case INT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BIGINT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case FLOAT: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Float.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DOUBLE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Double.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DECIMAL: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(HiveDecimalWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case TIMESTAMP: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Timestamp.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case DATE: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(java.sql.Date.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case STRING: + case VARCHAR: + case CHAR: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(String.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BOOLEAN: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(Boolean.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + case BINARY: + objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(BytesWritable.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA); + break; + default: + throw new IllegalArgumentException("You should not be here"); + } + return objectInspector; + } + + + public static Binary decimalToBinary(final HiveDecimal hiveDecimal, int prec, int scale) { + byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray(); + + // Estimated number of bytes needed. + int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1]; + if (precToBytes == decimalBytes.length) { + // No padding needed. + return Binary.fromReusedByteArray(decimalBytes); + } + + byte[] tgt = new byte[precToBytes]; + if (hiveDecimal.signum() == -1) { + // For negative number, initializing bits to 1 + for (int i = 0; i < precToBytes; i++) { + tgt[i] |= 0xFF; + } + } + + // Padding leading zeroes/ones. + System.arraycopy(decimalBytes, 0, tgt, precToBytes - decimalBytes.length, decimalBytes.length); + return Binary.fromReusedByteArray(tgt); + } + + public static int computeMinBytesForPrecision(int precision) { + int numBytes = 1; + while (Math.pow(SCALE_TWO, BIT_SIZE * numBytes - 1.0) < Math.pow(SCALE_TEN, precision)) { + numBytes += 1; + } + return numBytes; + } + + public static byte[] longToByteArray(long data) { + long nano = data * 1000_000; + + int julianDays = (int) ((nano / NANO_SECONDS_PER_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + byte[] julianDaysBytes = getBytes(julianDays); + flip(julianDaysBytes); + + long lastDayNanos = nano % NANO_SECONDS_PER_DAY; + byte[] lastDayNanosBytes = getBytes(lastDayNanos); + flip(lastDayNanosBytes); + + byte[] dst = new byte[12]; + + System.arraycopy(lastDayNanosBytes, 0, dst, 0, 8); + System.arraycopy(julianDaysBytes, 0, dst, 8, 4); + + return dst; + } + + private static byte[] getBytes(long i) { + byte[] bytes = new byte[8]; + bytes[0] = (byte) ((i >> 56) & 0xFF); + bytes[1] = (byte) ((i >> 48) & 0xFF); + bytes[2] = (byte) ((i >> 40) & 0xFF); + bytes[3] = (byte) ((i >> 32) & 0xFF); + bytes[4] = (byte) ((i >> 24) & 0xFF); + bytes[5] = (byte) ((i >> 16) & 0xFF); + bytes[6] = (byte) ((i >> 8) & 0xFF); + bytes[7] = (byte) (i & 0xFF); + return bytes; + } + + /** + * @param bytes + */ + private static void flip(byte[] bytes) { + for (int i = 0, j = bytes.length - 1; i < j; i++, j--) { + byte t = bytes[i]; + bytes[i] = bytes[j]; + bytes[j] = t; + } + } +} diff --git a/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/ECompressType.java b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/ECompressType.java new file mode 100644 index 0000000000..d7ff48ad00 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/ECompressType.java @@ -0,0 +1,80 @@ +package com.dtstack.flinkx.alluxio; + +import org.apache.commons.lang.StringUtils; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public enum ECompressType { + + /** + * text file + */ + TEXT_GZIP("GZIP", "text", ".gz", 0.331F), + TEXT_BZIP2("BZIP2", "text", ".bz2", 0.259F), + TEXT_LZO("LZO", "text", ".lzo", 1.0F), + TEXT_NONE("NONE", "text", "", 0.637F), + + /** + * orc file + */ + ORC_SNAPPY("SNAPPY", "orc", ".snappy", 0.233F), + ORC_GZIP("GZIP", "orc", ".gz", 1.0F), + ORC_BZIP("BZIP", "orc", ".bz", 1.0F), + ORC_LZ4("LZ4", "orc", ".lz4", 1.0F), + ORC_NONE("NONE", "orc", "", 0.233F), + + /** + * parquet file + */ + PARQUET_SNAPPY("SNAPPY", "parquet", ".snappy", 0.274F), + PARQUET_GZIP("GZIP", "parquet", ".gz", 1.0F), + PARQUET_LZO("LZO", "parquet", ".lzo", 1.0F), + PARQUET_NONE("NONE", "parquet", "", 1.0F); + + private String type; + + private String fileType; + + private String suffix; + + private float deviation; + + ECompressType(String type, String fileType, String suffix, float deviation) { + this.type = type; + this.fileType = fileType; + this.suffix = suffix; + this.deviation = deviation; + } + + public static ECompressType getByTypeAndFileType(String type, String fileType) { + if (StringUtils.isEmpty(type)) { + type = "NONE"; + } + + for (ECompressType value : ECompressType.values()) { + if (value.getType().equalsIgnoreCase(type) && value.getFileType().equalsIgnoreCase(fileType)) { + return value; + } + } + + throw new IllegalArgumentException("No enum constant " + type); + } + + public String getType() { + return type; + } + + public String getFileType() { + return fileType; + } + + public String getSuffix() { + return suffix; + } + + public float getDeviation() { + return deviation; + } +} diff --git a/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/util/StringUtil.java b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/util/StringUtil.java new file mode 100644 index 0000000000..706ac49f6a --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-core/src/main/java/com/dtstack/flinkx/alluxio/util/StringUtil.java @@ -0,0 +1,152 @@ +package com.dtstack.flinkx.alluxio.util; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class StringUtil { + /** + *

Check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.startsWith(null, null)      = true
+     * StringUtils.startsWith(null, "abc")     = false
+     * StringUtils.startsWith("abcdef", null)  = false
+     * StringUtils.startsWith("abcdef", "abc") = true
+     * StringUtils.startsWith("ABCDEF", "abc") = false
+     * 
+ * + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case sensitive, or + * both null + * @see String#startsWith(String) + * @since 2.4 + */ + public static boolean startsWith(String str, String prefix) { + return startsWith(str, prefix, false); + } + + /** + *

Case insensitive check if a String starts with a specified prefix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.startsWithIgnoreCase(null, null)      = true
+     * StringUtils.startsWithIgnoreCase(null, "abc")     = false
+     * StringUtils.startsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.startsWithIgnoreCase("abcdef", "abc") = true
+     * StringUtils.startsWithIgnoreCase("ABCDEF", "abc") = true
+     * 
+ * + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @return true if the String starts with the prefix, case insensitive, or + * both null + * @see String#startsWith(String) + * @since 2.4 + */ + public static boolean startsWithIgnoreCase(String str, String prefix) { + return startsWith(str, prefix, true); + } + + /** + *

Check if a String starts with a specified prefix (optionally case insensitive).

+ * + * @param str the String to check, may be null + * @param prefix the prefix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + * @see String#startsWith(String) + */ + private static boolean startsWith(String str, String prefix, boolean ignoreCase) { + if (str == null || prefix == null) { + return (str == null && prefix == null); + } + if (prefix.length() > str.length()) { + return false; + } + return str.regionMatches(ignoreCase, 0, prefix, 0, prefix.length()); + } + + /** + *

Check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case sensitive.

+ * + *
+     * StringUtils.endsWith(null, null)      = true
+     * StringUtils.endsWith(null, "def")     = false
+     * StringUtils.endsWith("abcdef", null)  = false
+     * StringUtils.endsWith("abcdef", "def") = true
+     * StringUtils.endsWith("ABCDEF", "def") = false
+     * StringUtils.endsWith("ABCDEF", "cde") = false
+     * 
+ * + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case sensitive, or + * both null + * @see String#endsWith(String) + * @since 2.4 + */ + public static boolean endsWith(String str, String suffix) { + return endsWith(str, suffix, false); + } + + /** + *

Case insensitive check if a String ends with a specified suffix.

+ * + *

nulls are handled without exceptions. Two null + * references are considered to be equal. The comparison is case insensitive.

+ * + *
+     * StringUtils.endsWithIgnoreCase(null, null)      = true
+     * StringUtils.endsWithIgnoreCase(null, "def")     = false
+     * StringUtils.endsWithIgnoreCase("abcdef", null)  = false
+     * StringUtils.endsWithIgnoreCase("abcdef", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "def") = true
+     * StringUtils.endsWithIgnoreCase("ABCDEF", "cde") = false
+     * 
+ * + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @return true if the String ends with the suffix, case insensitive, or + * both null + * @see String#endsWith(String) + * @since 2.4 + */ + public static boolean endsWithIgnoreCase(String str, String suffix) { + return endsWith(str, suffix, true); + } + + /** + *

Check if a String ends with a specified suffix (optionally case insensitive).

+ * + * @param str the String to check, may be null + * @param suffix the suffix to find, may be null + * @param ignoreCase inidicates whether the compare should ignore case + * (case insensitive) or not. + * @return true if the String starts with the prefix or + * both null + * @see String#endsWith(String) + */ + private static boolean endsWith(String str, String suffix, boolean ignoreCase) { + if (str == null || suffix == null) { + return (str == null && suffix == null); + } + if (suffix.length() > str.length()) { + return false; + } + int strOffset = str.length() - suffix.length(); + return str.regionMatches(ignoreCase, strOffset, suffix, 0, suffix.length()); + } +} diff --git a/flinkx-alluxio/flinkx-alluxio-writer/pom.xml b/flinkx-alluxio/flinkx-alluxio-writer/pom.xml new file mode 100644 index 0000000000..8861f27cab --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/pom.xml @@ -0,0 +1,128 @@ + + + + flinkx-alluxio + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-alluxio-writer + + + + org.alluxio + alluxio-shaded-client + 2.6.2 + + + + com.dtstack.flinkx + flinkx-alluxio-core + 1.6 + + + httpcore + org.apache.httpcomponents + + + httpclient + org.apache.httpcomponents + + + + + + httpcore + org.apache.httpcomponents + 4.4.5 + + + + httpclient + org.apache.httpcomponents + 4.5.2 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + false + + + org.slf4j:slf4j-api + ch.qos.logback:* + com.google.code.gson:* + com.data-artisans:* + org.scala-lang:* + io.netty:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.google.common + shade.core.com.google.common + + + com.google.thirdparty + shade.core.com.google.thirdparty + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOrcOutputFormat.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOrcOutputFormat.java new file mode 100644 index 0000000000..a18da2f56c --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOrcOutputFormat.java @@ -0,0 +1,275 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.alluxio.AlluxioUtil; +import com.dtstack.flinkx.alluxio.ECompressType; +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.util.ColumnTypeUtil; +import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.StringUtil; +import org.apache.flink.types.Row; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.compress.*; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioOrcOutputFormat extends BaseAlluxioOutputFormat { + + private RecordWriter recordWriter; + private OrcSerde orcSerde; + private StructObjectInspector inspector; + private FileOutputFormat outputFormat; + private JobConf jobConf; + + private static final ColumnTypeUtil.DecimalInfo ORC_DEFAULT_DECIMAL_INFO = new ColumnTypeUtil.DecimalInfo(HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE); + + @Override + protected void openSource() throws IOException { + super.openSource(); + orcSerde = new OrcSerde(); + outputFormat = new OrcOutputFormat(); + jobConf = new JobConf(conf); + FileOutputFormat.setOutputCompressorClass(jobConf, getCompressType()); + + List fullColTypeList = new ArrayList<>(); + decimalColInfo = new HashMap<>((fullColumnTypes.size() << 2) / 3); + for (int i = 0; i < fullColumnTypes.size(); i++) { + String columnType = fullColumnTypes.get(i); + if (ColumnTypeUtil.isDecimalType(columnType)) { + ColumnTypeUtil.DecimalInfo decimalInfo = ColumnTypeUtil.getDecimalInfo(columnType, ORC_DEFAULT_DECIMAL_INFO); + decimalColInfo.put(fullColumnNames.get(i), decimalInfo); + } + ColumnType type = ColumnType.getType(columnType); + fullColTypeList.add(AlluxioUtil.columnTypeToObjectInspetor(type)); + } + + this.inspector = ObjectInspectorFactory + .getStandardStructObjectInspector(fullColumnNames, fullColTypeList); + } + + private Class getCompressType() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "orc"); + if (ECompressType.ORC_SNAPPY.equals(compressType)) { + return SnappyCodec.class; + } else if (ECompressType.ORC_BZIP.equals(compressType)) { + return BZip2Codec.class; + } else if (ECompressType.ORC_GZIP.equals(compressType)) { + return GzipCodec.class; + } else if (ECompressType.ORC_LZ4.equals(compressType)) { + return Lz4Codec.class; + } else { + return DefaultCodec.class; + } + } + + @Override + protected String getExtension() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "orc"); + return compressType.getSuffix(); + } + + @Override + public float getDeviation() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "orc"); + return compressType.getDeviation(); + } + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (recordWriter != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + SP + currentBlockFileName; + recordWriter = outputFormat.getRecordWriter(null, jobConf, currentBlockTmpPath, Reporter.NULL); + blockIndex++; + + LOG.info("nextBlock:Current block writer record:" + rowsOfCurrentBlock); + LOG.info("Current block file name:" + currentBlockTmpPath); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected void writeSingleRecordToFile(Row row) throws WriteRecordException { + if (recordWriter == null) { + nextBlock(); + } + + List recordList = new ArrayList<>(); + int i = 0; + try { + for (; i < fullColumnNames.size(); ++i) { + getData(recordList, i, row); + } + } catch (Exception e) { + if (e instanceof WriteRecordException) { + throw (WriteRecordException) e; + } else { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + } + + try { + this.recordWriter.write(NullWritable.get(), this.orcSerde.serialize(recordList, this.inspector)); + rowsOfCurrentBlock++; + + if (restoreConfig.isRestore()) { + lastRow = row; + } + } catch (IOException e) { + throw new WriteRecordException(String.format("数据写入alluxio异常,row:{%s}", row), e); + } + } + + @Override + protected void flushDataInternal() throws IOException { + LOG.info("Close current orc record writer, write data size:[{}]", bytesWriteCounter.getLocalValue()); + + if (recordWriter != null) { + recordWriter.close(Reporter.NULL); + recordWriter = null; + } + } + + private void getData(List recordList, int index, Row row) throws WriteRecordException { + int j = colIndices[index]; + if (j == -1) { + recordList.add(null); + return; + } + + Object column = row.getField(j); + if (column == null) { + recordList.add(null); + return; + } + + ColumnType columnType = ColumnType.fromString(columnTypes.get(j)); + String rowData = column.toString(); + if (rowData == null || (rowData.length() == 0 && !ColumnType.isStringType(columnType))) { + recordList.add(null); + return; + } + + switch (columnType) { + case TINYINT: + recordList.add(Byte.valueOf(rowData)); + break; + case SMALLINT: + recordList.add(Short.valueOf(rowData)); + break; + case INT: + recordList.add(Integer.valueOf(rowData)); + break; + case BIGINT: + recordList.add(getBigint(column, rowData)); + break; + case FLOAT: + recordList.add(Float.valueOf(rowData)); + break; + case DOUBLE: + recordList.add(Double.valueOf(rowData)); + break; + case DECIMAL: + recordList.add(getDecimalWritable(index, rowData)); + break; + case STRING: + case VARCHAR: + case CHAR: + if (column instanceof Timestamp) { + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); + recordList.add(fm.format(column)); + } else if (column instanceof Map || column instanceof List) { + recordList.add(gson.toJson(column)); + } else { + recordList.add(rowData); + } + break; + case BOOLEAN: + recordList.add(StringUtil.parseBoolean(rowData)); + break; + case DATE: + recordList.add(DateUtil.columnToDate(column, null)); + break; + case TIMESTAMP: + recordList.add(DateUtil.columnToTimestamp(column, null)); + break; + case BINARY: + recordList.add(new BytesWritable(rowData.getBytes(StandardCharsets.UTF_8))); + break; + default: + throw new IllegalArgumentException(); + } + } + + private Object getBigint(Object column, String rowData) { + if (column instanceof Timestamp) { + column = ((Timestamp) column).getTime(); + return column; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0) { + return data; + } else { + return Long.valueOf(rowData); + } + } + + private HiveDecimalWritable getDecimalWritable(int index, String rowData) throws WriteRecordException { + ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(fullColumnNames.get(index)); + HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(rowData)); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); + if (hiveDecimal == null) { + String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", + index, decimalInfo.getPrecision(), decimalInfo.getScale(), rowData); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + return new HiveDecimalWritable(hiveDecimal); + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nAlluxioOrcOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + protected void closeSource() throws IOException { + RecordWriter rw = this.recordWriter; + if (rw != null) { + LOG.info("close:Current block writer record:" + rowsOfCurrentBlock); + rw.close(Reporter.NULL); + this.recordWriter = null; + } + } +} diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOutputFormatBuilder.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOutputFormatBuilder.java new file mode 100644 index 0000000000..fb7ba478ba --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioOutputFormatBuilder.java @@ -0,0 +1,79 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.FileOutputFormatBuilder; + +import java.util.List; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioOutputFormatBuilder extends FileOutputFormatBuilder { + + private final BaseAlluxioOutputFormat format; + + public AlluxioOutputFormatBuilder(String type) { + switch (type.toUpperCase()) { + case "TEXT": + format = new AlluxioTextOutputFormat(); + break; + case "ORC": + format = new AlluxioOrcOutputFormat(); + break; + case "PARQUET": + format = new AlluxioParquetOutputFormat(); + break; + default: + throw new IllegalArgumentException("Unsupported Alluxio file type: " + type); + } + + super.setFormat(format); + } + + public void setColumnNames(List columnNames) { + format.columnNames = columnNames; + } + + public void setColumnTypes(List columnTypes) { + format.columnTypes = columnTypes; + } + + public void setFullColumnNames(List fullColumnNames) { + format.fullColumnNames = fullColumnNames; + } + + public void setDelimiter(String delimiter) { + format.delimiter = delimiter; + } + + public void setRowGroupSize(int rowGroupSize) { + format.rowGroupSize = rowGroupSize; + } + + public void setFullColumnTypes(List fullColumnTypes) { + format.fullColumnTypes = fullColumnTypes; + } + + public void setEnableDictionary(boolean enableDictionary) { + format.enableDictionary = enableDictionary; + } + + public void setWriteType(String writeType) { + format.writeType = writeType; + } + + @Override + protected void checkFormat() { + super.checkFormat(); + + if (super.format.getPath() == null || super.format.getPath().length() == 0) { + throw new IllegalArgumentException("No valid path supplied."); + } + + if (!super.format.getPath().startsWith(ConstantValue.PROTOCOL_ALLUXIO)) { + throw new IllegalArgumentException("Path should start with alluxio://"); + } + } + +} diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioParquetOutputFormat.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioParquetOutputFormat.java new file mode 100644 index 0000000000..276e16f419 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioParquetOutputFormat.java @@ -0,0 +1,318 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.alluxio.AlluxioUtil; +import com.dtstack.flinkx.alluxio.ECompressType; +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.util.ColumnTypeUtil; +import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.types.Row; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioParquetOutputFormat extends BaseAlluxioOutputFormat { + + private SimpleGroupFactory groupFactory; + + private ParquetWriter writer; + + private MessageType schema; + + private static final ColumnTypeUtil.DecimalInfo PARQUET_DEFAULT_DECIMAL_INFO = new ColumnTypeUtil.DecimalInfo(10, 0); + + + @Override + protected void openSource() throws IOException { + super.openSource(); + + schema = buildSchema(); + GroupWriteSupport.setSchema(schema, conf); + groupFactory = new SimpleGroupFactory(schema); + } + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (writer != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + SP + currentBlockFileName; + Path writePath = new Path(currentBlockTmpPath); + ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(writePath) + .withWriteMode(ParquetFileWriter.Mode.CREATE) + .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0) + .withCompressionCodec(getCompressType()) + .withConf(conf) + .withType(schema) + .withDictionaryEncoding(enableDictionary) + .withRowGroupSize(rowGroupSize); + + writer = builder.build(); + blockIndex++; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + private CompressionCodecName getCompressType() { + // Compatible with old code + if (StringUtils.isEmpty(compress)) { + compress = ECompressType.PARQUET_SNAPPY.getType(); + } + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "parquet"); + if (ECompressType.PARQUET_SNAPPY.equals(compressType)) { + return CompressionCodecName.SNAPPY; + } else if (ECompressType.PARQUET_GZIP.equals(compressType)) { + return CompressionCodecName.GZIP; + } else if (ECompressType.PARQUET_LZO.equals(compressType)) { + return CompressionCodecName.LZO; + } else { + return CompressionCodecName.UNCOMPRESSED; + } + } + + @Override + protected String getExtension() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "parquet"); + return compressType.getSuffix(); + } + + @Override + protected void flushDataInternal() throws IOException { + LOG.info("Close current parquet record writer, write data size:[{}]", bytesWriteCounter.getLocalValue()); + + if (writer != null) { + writer.close(); + writer = null; + } + } + + @Override + public float getDeviation() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "parquet"); + return compressType.getDeviation(); + } + + @Override + protected void writeSingleRecordToFile(Row row) throws WriteRecordException { + + if (writer == null) { + nextBlock(); + } + + Group group = groupFactory.newGroup(); + int i = 0; + try { + for (; i < fullColumnNames.size(); i++) { + int colIndex = colIndices[i]; + if (colIndex > -1) { + Object valObj = row.getField(colIndex); + if (valObj == null || (valObj.toString().length() == 0 && !ColumnType.isStringType(fullColumnTypes.get(i)))) { + continue; + } + + addDataToGroup(group, valObj, i); + } + } + } catch (Exception e) { + if (e instanceof WriteRecordException) { + throw (WriteRecordException) e; + } else { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + } + + try { + writer.write(group); + rowsOfCurrentBlock++; + + if (restoreConfig.isRestore()) { + lastRow = row; + } + } catch (IOException e) { + throw new WriteRecordException(String.format("数据写入alluxio异常,row:{%s}", row), e); + } + } + + private void addDataToGroup(Group group, Object valObj, int i) throws Exception { + String colName = fullColumnNames.get(i); + String colType = fullColumnTypes.get(i); + colType = ColumnType.fromString(colType).name().toLowerCase(); + + String val = valObj.toString(); + + switch (colType) { + case "tinyint": + case "smallint": + case "int": + if (valObj instanceof Timestamp) { + ((Timestamp) valObj).getTime(); + group.add(colName, (int) ((Timestamp) valObj).getTime()); + } else if (valObj instanceof Date) { + group.add(colName, (int) ((Date) valObj).getTime()); + } else { + group.add(colName, Integer.parseInt(val)); + } + break; + case "bigint": + if (valObj instanceof Timestamp) { + group.add(colName, ((Timestamp) valObj).getTime()); + } else if (valObj instanceof Date) { + group.add(colName, ((Date) valObj).getTime()); + } else { + group.add(colName, Long.parseLong(val)); + } + break; + case "float": + group.add(colName, Float.parseFloat(val)); + break; + case "double": + group.add(colName, Double.parseDouble(val)); + break; + case "binary": + group.add(colName, Binary.fromString(val)); + break; + case "char": + case "varchar": + case "string": + if (valObj instanceof Timestamp) { + val = DateUtil.getDateTimeFormatterForMillisencond().format(valObj); + group.add(colName, val); + } else if (valObj instanceof Map || valObj instanceof List) { + group.add(colName, gson.toJson(valObj)); + } else { + group.add(colName, val); + } + break; + case "boolean": + group.add(colName, StringUtil.parseBoolean(val)); + break; + case "timestamp": + Timestamp ts = DateUtil.columnToTimestamp(valObj, null); + byte[] dst = AlluxioUtil.longToByteArray(ts.getTime()); + group.add(colName, Binary.fromConstantByteArray(dst)); + break; + case "decimal": + ColumnTypeUtil.DecimalInfo decimalInfo = decimalColInfo.get(colName); + + HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(val)); + hiveDecimal = HiveDecimal.enforcePrecisionScale(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale()); + if (hiveDecimal == null) { + String msg = String.format("第[%s]个数据数据[%s]precision和scale和元数据不匹配:decimal(%s, %s)", i, decimalInfo.getPrecision(), decimalInfo.getScale(), valObj); + throw new WriteRecordException(msg, new IllegalArgumentException()); + } + + group.add(colName, AlluxioUtil.decimalToBinary(hiveDecimal, decimalInfo.getPrecision(), decimalInfo.getScale())); + break; + case "date": + Date date = DateUtil.columnToDate(valObj, null); + group.add(colName, DateWritable.dateToDays(new java.sql.Date(date.getTime()))); + break; + default: + group.add(colName, val); + break; + } + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nAlluxioParquetOutputFormat [" + jobName + "] writeRecord error: when converting field[" + fullColumnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + protected void closeSource() throws IOException { + if (writer != null) { + writer.close(); + } + } + + private MessageType buildSchema() { + decimalColInfo = new HashMap<>(16); + Types.MessageTypeBuilder typeBuilder = Types.buildMessage(); + for (int i = 0; i < fullColumnNames.size(); i++) { + String name = fullColumnNames.get(i); + String colType = fullColumnTypes.get(i).toLowerCase(); + switch (colType) { + case "tinyint": + case "smallint": + case "int": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT32).named(name); + break; + case "bigint": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT64).named(name); + break; + case "float": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.FLOAT).named(name); + break; + case "double": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.DOUBLE).named(name); + break; + case "binary": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); + break; + case "char": + case "varchar": + case "string": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8).named(name); + break; + case "boolean": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BOOLEAN).named(name); + break; + case "timestamp": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT96).named(name); + break; + case "date": + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.INT32).as(OriginalType.DATE).named(name); + break; + default: + if (ColumnTypeUtil.isDecimalType(colType)) { + ColumnTypeUtil.DecimalInfo decimalInfo = ColumnTypeUtil.getDecimalInfo(colType, PARQUET_DEFAULT_DECIMAL_INFO); + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + .as(OriginalType.DECIMAL) + .precision(decimalInfo.getPrecision()) + .scale(decimalInfo.getScale()) + .length(AlluxioUtil.computeMinBytesForPrecision(decimalInfo.getPrecision())) + .named(name); + + decimalColInfo.put(name, decimalInfo); + } else { + typeBuilder.optional(PrimitiveType.PrimitiveTypeName.BINARY).named(name); + } + break; + } + } + return typeBuilder.named("Pair"); + } +} diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioTextOutputFormat.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioTextOutputFormat.java new file mode 100644 index 0000000000..2ab5e7d97a --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioTextOutputFormat.java @@ -0,0 +1,231 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.alluxio.AlluxioUtil; +import com.dtstack.flinkx.alluxio.ECompressType; +import com.dtstack.flinkx.enums.ColumnType; +import com.dtstack.flinkx.exception.WriteRecordException; +import com.dtstack.flinkx.util.DateUtil; +import com.dtstack.flinkx.util.StringUtil; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioTextOutputFormat extends BaseAlluxioOutputFormat { + + private static final int NEWLINE = 10; + private transient OutputStream stream; + + private static final int BUFFER_SIZE = 1000; + + @Override + protected void flushDataInternal() throws IOException { + LOG.info("Close current text stream, write data size:[{}]", bytesWriteCounter.getLocalValue()); + + if (stream != null) { + stream.flush(); + stream.close(); + stream = null; + } + } + + @Override + public float getDeviation() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getDeviation(); + } + + @Override + protected String getExtension() { + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + return compressType.getSuffix(); + } + + @Override + protected void nextBlock() { + super.nextBlock(); + + if (stream != null) { + return; + } + + try { + String currentBlockTmpPath = tmpPath + SP + currentBlockFileName; + Path p = new Path(currentBlockTmpPath); + + ECompressType compressType = ECompressType.getByTypeAndFileType(compress, "text"); + if (ECompressType.TEXT_NONE.equals(compressType)) { + stream = fs.create(p); + } else { + p = new Path(currentBlockTmpPath); + if (compressType == ECompressType.TEXT_GZIP) { + stream = new GzipCompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_BZIP2) { + stream = new BZip2CompressorOutputStream(fs.create(p)); + } else if (compressType == ECompressType.TEXT_LZO) { + CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration()); + stream = factory.getCodecByClassName("com.hadoop.compression.lzo.LzopCodec").createOutputStream(fs.create(p)); + } + } + + LOG.info("subtask:[{}] create block file:{}", taskNumber, currentBlockTmpPath); + + blockIndex++; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected void writeSingleRecordToFile(Row row) throws WriteRecordException { + if (stream == null) { + nextBlock(); + } + + StringBuilder sb = new StringBuilder(); + int i = 0; + try { + int cnt = fullColumnNames.size(); + for (; i < cnt; ++i) { + int j = colIndices[i]; + if (j == -1) { + continue; + } + + if (i != 0) { + sb.append(delimiter); + } + + appendDataToString(sb, row.getField(j), ColumnType.fromString(columnTypes.get(j))); + } + } catch (Exception e) { + if (i < row.getArity()) { + throw new WriteRecordException(recordConvertDetailErrorMessage(i, row), e, i, row); + } + throw new WriteRecordException(e.getMessage(), e); + } + + try { + byte[] bytes = sb.toString().getBytes(this.charsetName); + this.stream.write(bytes); + this.stream.write(NEWLINE); + rowsOfCurrentBlock++; + + if (restoreConfig.isRestore()) { + lastRow = row; + } + + if (rowsOfCurrentBlock % BUFFER_SIZE == 0) { + this.stream.flush(); + } + } catch (IOException e) { + LOG.error(e.getMessage(), e); + throw new WriteRecordException(String.format("数据写入Alluxio异常,row:{%s}", row), e); + } + } + + private void appendDataToString(StringBuilder sb, Object column, ColumnType columnType) { + if (column == null) { + sb.append(AlluxioUtil.NULL_VALUE); + return; + } + + String rowData = column.toString(); + if (rowData.length() == 0) { + sb.append(""); + } else { + switch (columnType) { + case TINYINT: + sb.append(Byte.valueOf(rowData)); + break; + case SMALLINT: + sb.append(Short.valueOf(rowData)); + break; + case INT: + sb.append(Integer.valueOf(rowData)); + break; + case BIGINT: + if (column instanceof Timestamp) { + column = ((Timestamp) column).getTime(); + sb.append(column); + break; + } + + BigInteger data = new BigInteger(rowData); + if (data.compareTo(new BigInteger(String.valueOf(Long.MAX_VALUE))) > 0) { + sb.append(data); + } else { + sb.append(Long.valueOf(rowData)); + } + break; + case FLOAT: + sb.append(Float.valueOf(rowData)); + break; + case DOUBLE: + sb.append(Double.valueOf(rowData)); + break; + case DECIMAL: + sb.append(HiveDecimal.create(new BigDecimal(rowData))); + break; + case STRING: + case VARCHAR: + case CHAR: + if (column instanceof Timestamp) { + SimpleDateFormat fm = DateUtil.getDateTimeFormatterForMillisencond(); + sb.append(fm.format(column)); + } else if (column instanceof Map || column instanceof List) { + sb.append(gson.toJson(column)); + } else { + sb.append(rowData); + } + break; + case BOOLEAN: + sb.append(StringUtil.parseBoolean(rowData)); + break; + case DATE: + column = DateUtil.columnToDate(column, null); + sb.append(DateUtil.dateToString((Date) column)); + break; + case TIMESTAMP: + column = DateUtil.columnToTimestamp(column, null); + sb.append(DateUtil.timestampToString((Date) column)); + break; + default: + throw new IllegalArgumentException("Unsupported column type: " + columnType); + } + } + } + + @Override + protected String recordConvertDetailErrorMessage(int pos, Row row) { + return "\nAlluxioTextOutputFormat [" + jobName + "] writeRecord error: when converting field[" + columnNames.get(pos) + "] in Row(" + row + ")"; + } + + @Override + public void closeSource() throws IOException { + OutputStream s = this.stream; + if (s != null) { + s.flush(); + this.stream = null; + s.close(); + } + } + +} diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioWriter.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioWriter.java new file mode 100644 index 0000000000..1dc04d3ec4 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/AlluxioWriter.java @@ -0,0 +1,161 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.alluxio.util.StringUtil; +import com.dtstack.flinkx.config.DataTransferConfig; +import com.dtstack.flinkx.config.WriterConfig; +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.enums.WriteType; +import com.dtstack.flinkx.writer.BaseDataWriter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.types.Row; +import org.apache.parquet.hadoop.ParquetWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.*; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_COLUMN_NAME; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_COLUMN_TYPE; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_COMPRESS; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_ENABLE_DICTIONARY; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_ENCODING; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FIELD_DELIMITER; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FILE_NAME; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FILE_TYPE; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FLUSH_INTERVAL; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FULL_COLUMN_NAME_LIST; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_FULL_COLUMN_TYPE_LIST; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_MAX_FILE_SIZE; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_PATH; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_ROW_GROUP_SIZE; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_WRITE_MODE; +import static com.dtstack.flinkx.alluxio.AlluxioConfigKeys.KEY_WRITE_TYPE; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-06 + */ +public class AlluxioWriter extends BaseDataWriter { + + protected final Logger LOG = LoggerFactory.getLogger(getClass()); + + protected String fileType; + + protected String path; + + protected String fieldDelimiter; + + protected String compress; + + protected String fileName; + + protected List columnName; + + protected List columnType; + + protected String charSet; + + protected List fullColumnName; + + protected List fullColumnType; + + protected int rowGroupSize; + + protected long maxFileSize; + + protected long flushInterval; + + protected boolean enableDictionary; + + protected String writerType; + + public AlluxioWriter(DataTransferConfig config) { + super(config); + WriterConfig writerConfig = config.getJob().getContent().get(0).getWriter(); + List columns = writerConfig.getParameter().getColumn(); + fileType = writerConfig.getParameter().getStringVal(KEY_FILE_TYPE); + path = writerConfig.getParameter().getStringVal(KEY_PATH); + fieldDelimiter = writerConfig.getParameter().getStringVal(KEY_FIELD_DELIMITER); + charSet = writerConfig.getParameter().getStringVal(KEY_ENCODING); + rowGroupSize = writerConfig.getParameter().getIntVal(KEY_ROW_GROUP_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE); + maxFileSize = writerConfig.getParameter().getLongVal(KEY_MAX_FILE_SIZE, ConstantValue.STORE_SIZE_G); + flushInterval = writerConfig.getParameter().getLongVal(KEY_FLUSH_INTERVAL, 0); + enableDictionary = writerConfig.getParameter().getBooleanVal(KEY_ENABLE_DICTIONARY, true); + writerType = writerConfig.getParameter().getStringVal(KEY_WRITE_TYPE, WriteType.THROUGH.name()); + + if (fieldDelimiter == null || fieldDelimiter.length() == 0) { + fieldDelimiter = "\001"; + } else { + fieldDelimiter = com.dtstack.flinkx.util.StringUtil.convertRegularExpr(fieldDelimiter); + } + + compress = writerConfig.getParameter().getStringVal(KEY_COMPRESS); + fileName = writerConfig.getParameter().getStringVal(KEY_FILE_NAME, ""); + if (columns != null && columns.size() > 0) { + columnName = new ArrayList<>(); + columnType = new ArrayList<>(); + for (Object column : columns) { + Map sm = (Map) column; + columnName.add((String) sm.get(KEY_COLUMN_NAME)); + columnType.add((String) sm.get(KEY_COLUMN_TYPE)); + } + } + + fullColumnName = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_NAME_LIST); + fullColumnType = (List) writerConfig.getParameter().getVal(KEY_FULL_COLUMN_TYPE_LIST); + + mode = writerConfig.getParameter().getStringVal(KEY_WRITE_MODE); + } + + @Override + public DataStreamSink writeData(DataStream dataSet) { + AlluxioOutputFormatBuilder builder = new AlluxioOutputFormatBuilder(fileType); + builder.setPath(formatPath(path)); + builder.setFileName(fileName); + builder.setWriteMode(mode); + builder.setColumnNames(columnName); + builder.setColumnTypes(columnType); + builder.setCompress(compress); + builder.setMonitorUrls(monitorUrls); + builder.setErrors(errors); + builder.setErrorRatio(errorRatio); + builder.setFullColumnNames(fullColumnName); + builder.setFullColumnTypes(fullColumnType); + builder.setDirtyPath(dirtyPath); + builder.setDirtyHadoopConfig(dirtyHadoopConfig); + builder.setSrcCols(srcCols); + builder.setCharSetName(charSet); + builder.setDelimiter(fieldDelimiter); + builder.setRowGroupSize(rowGroupSize); + builder.setRestoreConfig(restoreConfig); + builder.setMaxFileSize(maxFileSize); + builder.setFlushBlockInterval(flushInterval); + builder.setEnableDictionary(enableDictionary); + builder.setWriteType(writerType); + return createOutput(dataSet, builder.finish()); + } + + private String formatPath(String path) { + String pathAfterFormat = path; + if (!StringUtil.startsWith(path, "alluxio://")) { + if (StringUtil.startsWith(path, "//")) { + pathAfterFormat = "alluxio:" + path; + } else if (StringUtil.startsWith(path, "/")) { + pathAfterFormat = "alluxio:/" + path; + } else { + pathAfterFormat = "alluxio://" + path; + } + } + + if (!StringUtil.endsWith(pathAfterFormat, "/")) { + pathAfterFormat = pathAfterFormat + "/"; + } + + LOG.debug("Path = " + pathAfterFormat); + return pathAfterFormat; + } +} \ No newline at end of file diff --git a/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/BaseAlluxioOutputFormat.java b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/BaseAlluxioOutputFormat.java new file mode 100644 index 0000000000..4b71429bf6 --- /dev/null +++ b/flinkx-alluxio/flinkx-alluxio-writer/src/main/java/com/dtstack/flinkx/alluxio/writer/BaseAlluxioOutputFormat.java @@ -0,0 +1,331 @@ +package com.dtstack.flinkx.alluxio.writer; + +import com.dtstack.flinkx.constants.ConstantValue; +import com.dtstack.flinkx.outputformat.BaseFileOutputFormat; +import com.dtstack.flinkx.util.ColumnTypeUtil; +import com.dtstack.flinkx.util.SysUtil; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-02 + */ +public abstract class BaseAlluxioOutputFormat extends BaseFileOutputFormat { + + private static final int FILE_NAME_PART_SIZE = 3; + + protected int rowGroupSize; + + protected FileSystem fs; + + protected List columnTypes; + + protected List columnNames; + + protected List fullColumnNames; + + protected List fullColumnTypes; + + protected String delimiter; + + protected int[] colIndices; + + protected Configuration conf; + + protected boolean enableDictionary; + + protected String writeType; + + protected transient Map decimalColInfo; + + /** + * 如果key为string类型的值是map 或者 list 会使用gson转为json格式存入 + */ + protected transient Gson gson; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + gson = new Gson(); + + initColIndices(); + super.openInternal(taskNumber, numTasks); + } + + @Override + protected void checkOutputDir() { + try { + Path dir = new Path(outputFilePath); + + if (fs.exists(dir)) { + if (fs.getFileStatus(dir).isFile()) { + throw new RuntimeException("Can't write new files under common file: " + dir + "\n" + + "One can only write new files under directories"); + } + } else { + if (!makeDir) { + throw new RuntimeException("Output path not exists:" + outputFilePath); + } + } + } catch (IOException e) { + throw new RuntimeException("Check output path error", e); + } + } + + @Override + protected void createActionFinishedTag() { + try { + if (fs.createNewFile(new Path(actionFinishedTag))) { + LOG.info("Succeed to create action finished tag:{}", actionFinishedTag); + } else { + LOG.warn("Failed to create action finished tag:{}", actionFinishedTag); + } + } catch (Exception e) { + throw new RuntimeException("Create action finished tag error:", e); + } + } + + @Override + protected void waitForActionFinishedBeforeWrite() { + try { + Path path = new Path(actionFinishedTag); + boolean readyWrite = fs.exists(path); + int n = 0; + while (!readyWrite) { + if (n > SECOND_WAIT) { + throw new RuntimeException("Wait action finished before write timeout"); + } + + SysUtil.sleep(1000); + readyWrite = fs.exists(path); + n++; + } + } catch (Exception e) { + LOG.warn("Call method waitForActionFinishedBeforeWrite error", e); + } + } + + @Override + protected void cleanDirtyData() { + int fileIndex = formatState.getFileIndex(); + String lastJobId = formatState.getJobId(); + LOG.info("start to cleanDirtyData, fileIndex = {}, lastJobId = {}", fileIndex, lastJobId); + if (StringUtils.isBlank(lastJobId)) { + return; + } + + PathFilter filter = path -> { + String fileName = path.getName(); + if (!fileName.contains(lastJobId)) { + return false; + } + + String[] splits = fileName.split("\\."); + if (splits.length == FILE_NAME_PART_SIZE) { + return Integer.parseInt(splits[2]) > fileIndex; + } + + return false; + }; + + try { + FileStatus[] dirtyData = fs.listStatus(new Path(outputFilePath), filter); + if (dirtyData != null && dirtyData.length > 0) { + for (FileStatus dirtyDatum : dirtyData) { + fs.delete(dirtyDatum.getPath(), false); + LOG.info("Delete dirty data file:{}", dirtyDatum.getPath()); + } + } + } catch (Exception e) { + LOG.error("Clean dirty data error:", e); + throw new RuntimeException(e); + } + } + + @Override + protected void openSource() throws IOException { + try { + conf = new Configuration(); + conf.set("fs.alluxio.impl", "alluxio.hadoop.FileSystem"); + conf.set("fs.AbstractFileSystem.alluxio.impl", "alluxio.hadoop.AlluxioFileSystem"); + //默认情况下,数据被同步地写入到底层存储系统(hdfs or eos),但不会被写入到Alluxio的Worker。 + conf.set("alluxio.user.file.writetype.default", writeType); + fs = new Path(path).getFileSystem(conf); + } catch (Exception e) { + LOG.error("Failed to get AlluxioFileSystem with exception : " + e.getMessage()); + throw new RuntimeException("Failed to get AlluxioFileSystem with exception", e); + } + } + + private void initColIndices() { + if (fullColumnNames == null || fullColumnNames.size() == 0) { + fullColumnNames = columnNames; + } + + if (fullColumnTypes == null || fullColumnTypes.size() == 0) { + fullColumnTypes = columnTypes; + } + + colIndices = new int[fullColumnNames.size()]; + for (int i = 0; i < fullColumnNames.size(); ++i) { + int j = 0; + for (; j < columnNames.size(); ++j) { + if (fullColumnNames.get(i).equalsIgnoreCase(columnNames.get(j))) { + colIndices[i] = j; + break; + } + } + if (j == columnNames.size()) { + colIndices[i] = -1; + } + } + } + + @Override + protected void moveTemporaryDataBlockFileToDirectory() { + try { + if (currentBlockFileName != null && currentBlockFileName.startsWith(ConstantValue.POINT_SYMBOL)) { + Path src = new Path(tmpPath + SP + currentBlockFileName); + if (!fs.exists(src)) { + LOG.warn("block file {} not exists", currentBlockFileName); + return; + } + + String dataFileName = currentBlockFileName.replaceFirst("\\.", ""); + Path dist = new Path(tmpPath + SP + dataFileName); + + if (fs.rename(src, dist)) { + LOG.info("Rename temporary data block file:{} to:{}", src, dist); + } else { + LOG.info("Failed to rename temporary data block file:{} to:{}", src, dist); + } + } + } catch (Exception e) { + LOG.error("Failed to rename file with exception : " + e.getMessage()); + throw new RuntimeException(e); + } + } + + @Override + protected void clearTemporaryDataFiles() throws IOException { + Path finishedDir, tmpDir; + if (outputFilePath.endsWith(SP)) { + finishedDir = new Path(outputFilePath, FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath, DATA_SUBDIR); + } else { + finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + tmpDir = new Path(outputFilePath + SP + DATA_SUBDIR); + } + + if (fs.delete(finishedDir, true)) { + LOG.info("Succeed to delete .finished dir:{}", finishedDir); + } else { + LOG.warn("Failed to delete .finished dir:{}", finishedDir); + } + + if (fs.delete(tmpDir, true)) { + LOG.info("Succeed to delete .data dir:{}", tmpDir); + } else { + LOG.warn("Failed to delete .data dir:{}", tmpDir); + } + } + + @Override + protected void closeSource() throws IOException { + if (fs != null) { + fs.close(); + } + } + + @Override + protected void createFinishedTag() throws IOException { + if (fs != null) { + fs.createNewFile(new Path(finishedPath)); + LOG.info("Create finished tag dir:{}", finishedPath); + } + } + + @Override + protected void waitForAllTasksToFinish() throws IOException { + Path finishedDir = new Path(outputFilePath + SP + FINISHED_SUBDIR); + final int maxRetryTime = 100; + int i = 0; + for (; i < maxRetryTime; ++i) { + if (fs.listStatus(finishedDir).length == numTasks) { + break; + } + SysUtil.sleep(3000); + } + + if (i == maxRetryTime) { + String subTaskDataPath = outputFilePath + SP + DATA_SUBDIR; + fs.delete(new Path(subTaskDataPath), true); + LOG.info("waitForAllTasksToFinish: delete path:[{}]", subTaskDataPath); + + fs.delete(finishedDir, true); + LOG.info("waitForAllTasksToFinish: delete finished dir:[{}]", finishedDir); + + throw new RuntimeException("timeout when gathering finish tags for each subtasks"); + } + } + + @Override + protected void coverageData() throws IOException { + LOG.info("Overwrite the original data"); + + Path dir = new Path(outputFilePath); + if (!fs.exists(dir)) { + return; + } + + fs.delete(dir, true); + fs.mkdirs(dir); + } + + @Override + protected void moveTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> path.getName().startsWith(String.valueOf(taskNumber)); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.warn("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void moveAllTemporaryDataFileToDirectory() throws IOException { + PathFilter pathFilter = path -> !path.getName().startsWith("."); + Path dir = new Path(outputFilePath); + Path tmpDir = new Path(tmpPath); + + FileStatus[] dataFiles = fs.listStatus(tmpDir, pathFilter); + for (FileStatus dataFile : dataFiles) { + if (fs.rename(dataFile.getPath(), new Path(dir, dataFile.getPath().getName()))) { + LOG.info("Rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } else { + LOG.warn("Failed to rename temp file:{} to dir:{}", dataFile.getPath(), dir); + } + } + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + notSupportBatchWrite("AlluxioWriter"); + } + +} \ No newline at end of file diff --git a/flinkx-alluxio/pom.xml b/flinkx-alluxio/pom.xml new file mode 100644 index 0000000000..21099d1356 --- /dev/null +++ b/flinkx-alluxio/pom.xml @@ -0,0 +1,28 @@ + + + + flinkx-all + com.dtstack.flinkx + 1.6 + + 4.0.0 + + flinkx-alluxio + pom + + flinkx-alluxio-core + flinkx-alluxio-writer + + + + + com.dtstack.flinkx + flinkx-core + 1.6 + provided + + + + \ No newline at end of file diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java index d21116dbff..1ba9aa661d 100644 --- a/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/constants/ConstantValue.java @@ -52,6 +52,7 @@ public class ConstantValue { public static final String PROTOCOL_HTTPS = "https://"; public static final String PROTOCOL_HDFS = "hdfs://"; public static final String PROTOCOL_JDBC_MYSQL = "jdbc:mysql://"; + public static final String PROTOCOL_ALLUXIO = "alluxio://"; public static final String SYSTEM_PROPERTIES_KEY_OS = "os.name"; public static final String SYSTEM_PROPERTIES_KEY_USER_DIR = "user.dir"; diff --git a/flinkx-core/src/main/java/com/dtstack/flinkx/enums/WriteType.java b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/WriteType.java new file mode 100644 index 0000000000..9ed388a46d --- /dev/null +++ b/flinkx-core/src/main/java/com/dtstack/flinkx/enums/WriteType.java @@ -0,0 +1,30 @@ +package com.dtstack.flinkx.enums; + +/** + * @author wuzhongjian_yewu@cmss.chinamobile.com + * @date 2021-12-16 + */ +public enum WriteType { + + /** + * Data is written synchronously to a Alluxio worker and the under storage system. + */ + CACHE_THROUGH, + + /** + * Data is written synchronously to a Alluxio worker. + * No data will be written to the under storage. This is the default write type. + */ + MUST_CACHE, + + /** + * Default,Data is written synchronously to the under storage. No data will be written to Alluxio. + */ + THROUGH, + + /** + * Data is written synchronously to a Alluxio worker and asynchronously to the under storage system. Experimental. + */ + ASYNC_THROUGH + +} diff --git a/flinkx-test/pom.xml b/flinkx-test/pom.xml index c42e75ea9a..125eadee28 100644 --- a/flinkx-test/pom.xml +++ b/flinkx-test/pom.xml @@ -470,6 +470,11 @@ flinkx-kingbase-writer 1.6 + + com.dtstack.flinkx + flinkx-alluxio-writer + 1.6 + \ No newline at end of file diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java index f7a9216bdd..ffef15726a 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/LocalTest.java @@ -18,6 +18,7 @@ package com.dtstack.flinkx.test; import com.dtstack.flink.api.java.MyLocalStreamEnvironment; +import com.dtstack.flinkx.alluxio.writer.AlluxioWriter; import com.dtstack.flinkx.binlog.reader.BinlogReader; import com.dtstack.flinkx.carbondata.reader.CarbondataReader; import com.dtstack.flinkx.carbondata.writer.CarbondataWriter; @@ -264,6 +265,7 @@ private static BaseDataWriter buildDataWriter(DataTransferConfig config){ case PluginNameConstants.PHOENIX5_WRITER : writer = new Phoenix5Writer(config); break; case PluginNameConstants.KINGBASE_WRITER : writer = new KingbaseWriter(config); break; case PluginNameConstants.RESTAPI_WRITER: writer = new RedisWriter(config); break; + case PluginNameConstants.ALLUXIO_WRITER: writer = new AlluxioWriter(config); break; default:throw new IllegalArgumentException("Can not find writer by name:" + writerName); } diff --git a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java index c3a3dcd36a..579f9f679e 100644 --- a/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java +++ b/flinkx-test/src/main/java/com/dtstack/flinkx/test/PluginNameConstants.java @@ -87,4 +87,5 @@ public class PluginNameConstants { public static final String GREENPLUM_WRITER = "greenplumwriter"; public static final String PHOENIX5_WRITER = "phoenix5writer"; public static final String KINGBASE_WRITER = "kingbasewriter"; + public static final String ALLUXIO_WRITER = "alluxiowriter"; } diff --git a/pom.xml b/pom.xml index af22d481a1..060c5f0517 100644 --- a/pom.xml +++ b/pom.xml @@ -43,6 +43,7 @@ flinkx-carbondata flinkx-kudu flinkx-cassandra + flinkx-alluxio flinkx-redis flinkx-mongodb From 93f5f6cbee71edcc84431e6047fbba4e0066006c Mon Sep 17 00:00:00 2001 From: wuzhongjian Date: Tue, 21 Dec 2021 15:17:10 +0800 Subject: [PATCH 133/136] =?UTF-8?q?[docs-570][flinkx-alluxio]=20=E5=A2=9E?= =?UTF-8?q?=E5=8A=A0stream=E5=86=99=E5=85=A5alluxio=E5=AE=9E=E4=BE=8B?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 1 + README_EN.md | 1 + docs/offline/writer/alluxiowriter.md | 319 +++++++++++++++++++++++++++ 3 files changed, 321 insertions(+) create mode 100644 docs/offline/writer/alluxiowriter.md diff --git a/README.md b/README.md index 0de4d29d13..97be55aa15 100644 --- a/README.md +++ b/README.md @@ -83,6 +83,7 @@ FlinkX目前支持下面这些数据库: | | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | +| | Alluxio | | [doc](docs/offline/writer/alluxiowriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | | | RestApi |[doc](docs/realTime/reader/restapireader.md) | [doc](docs/realTime/writer/restapiwriter.md) | diff --git a/README_EN.md b/README_EN.md index 5e5bfdeb74..4eea74d6bb 100644 --- a/README_EN.md +++ b/README_EN.md @@ -73,6 +73,7 @@ The following databases are currently supported: | | Stream | [doc](docs/offline/reader/streamreader.md) | [doc](docs/offline/writer/streamwriter.md) | | | Redis | | [doc](docs/offline/writer/rediswriter.md) | | | Hive | | [doc](docs/offline/writer/hivewriter.md) | +| | Alluxio | | [doc](docs/offline/writer/alluxiowriter.md) | | Stream Synchronization | Kafka | [doc](docs/realTime/reader/kafkareader.md) | [doc](docs/realTime/writer/kafkawriter.md) | | | EMQX | [doc](docs/realTime/reader/emqxreader.md) | [doc](docs/realTime/writer/emqxwriter.md) | | | RestApi || [doc](docs/realTime/writer/restapiwriter.md) | diff --git a/docs/offline/writer/alluxiowriter.md b/docs/offline/writer/alluxiowriter.md new file mode 100644 index 0000000000..4a6cc3785f --- /dev/null +++ b/docs/offline/writer/alluxiowriter.md @@ -0,0 +1,319 @@ +# Alluxio Writer + +## 一、插件名称 +名称:**alluxiowriter** + + +## 二、数据源版本 +Alluxio 2.0.1-2.6.2 + + +## 三、参数说明 + +- **writeType** + - 描述:指定写入新文件时的数据写入行为,支持用户配置为`CACHE_THROUGH`、`MUST_CACHE`、`THROUGH`、`ASYNC_THROUGH` + - CACHE_THROUGH:数据同步写入alluxio worker和底层存储 + - MUST_CACHE:数据同步写入alluxio worker,但不会写入底层存储 + - THROUGH:数据同步写入底层存储,但不会写入alluxio worker + - ASYNC_THROUGH:数据同步写入alluxio worker,异步写入底层存储 + - 必选:否 + - 字段类型:string + - 默认值:THROUGH + +
+ +- **fileType** + - 描述:文件的类型,目前只支持用户配置为`text`、`orc`、`parquet` + - text:textfile文件格式 + - orc:orcfile文件格式 + - parquet:parquet文件格式 + - 必选:是 + - 字段类型:string + - 默认值:无 + +
+ +- **path** + - 描述:数据文件的路径 + - 必选:是 + - 字段类型:string + - 默认值:无 + +
+ +- **fileName** + - 描述:写入的目录名称 + - 注意:不为空,写入的路径为 path+fileName + - 必须:否 + - 字段类型:string + - 默认值:无 + +
+ +- **fieldDelimiter** + - 描述:`fileType`为`text`时字段的分隔符 + - 必选:否 + - 字段类型:string + - 默认值:`\001` + +
+ +- **encoding** + - 描述:`fileType`为`text`时可配置编码格式 + - 必选:否 + - 字段类型:string + - 默认值:UTF-8 + +
+ +- **maxFileSize** + - 描述:写入alluxio单个文件最大大小,单位字节 + - 必须:否 + - 字段类型:long + - 默认值:1073741824‬(1G) + +
+ +- **compress** + - 描述:alluxio文件压缩类型 + - text:支持`GZIP`、`BZIP2`格式 + - orc:支持`SNAPPY`、`ZLIB`、`LZO`格式 + - parquet:支持`SNAPPY`、`GZIP`、`LZO`格式 + - 注意:`SNAPPY`格式需要用户安装**SnappyCodec** + - 必选:否 + - 字段类型:string + - 默认值: + - text 默认 不进行压缩 + - orc 默认为ZLIB格式 + - parquet 默认为SNAPPY格式 + +
+ +- **writeMode** + - 描述:alluxiowriter写入前数据清理处理模式: + - append:追加 + - overwrite:覆盖 + - 注意:overwrite模式时会删除 alluxio当前目录下的所有文件 + - 必选:否 + - 字段类型:string + - 默认值:append + +
+ +- **column** + - 描述:需要读取的字段。 + - 格式:指定具体信息: +```json +"column": [{ + "name": "col", + "type": "datetime" +}] +``` + +- 属性说明: + - name:字段名称 + - type:字段类型,可以和源字段类型不一样,程序会做一次类型转换 +- 必选:是 +- 默认值:无 + +
+ +- **fullColumnName** + - 描述:写入的字段名称 + - 必须:否 + - 字段类型:list + - 默认值:column的name集合 + +
+ +- **fullColumnType** + - 描述:写入的字段类型 + - 必须:否 + - 字段类型:list + - 默认值:column的type集合 + +
+ +- **rowGroupSIze** + - 描述:parquet类型文件参数,指定row group的大小,单位字节 + - 必须:否 + - 字段类型:int + - 默认值:134217728(128M) + +
+ +- **enableDictionary** + - 描述:parquet类型文件参数,是否启动字典编码 + - 必须:否 + - 字段类型:boolean + - 默认值:true + +## 四、使用示例 +#### 1、写入text文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "alluxio://ip:port/mnt/alluxio_text", + "fileName": "pt=20211220", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "|", + "encoding": "utf-8", + "fileType": "text" + }, + "name": "alluxiowriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false + } + } + } +} +``` +#### 2、写入orc文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "alluxio://36.138.22.18:19998/mnt/alluxio_orc", + "fileName": "pt=20211220", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "|", + "encoding": "utf-8", + "fileType": "orc" + }, + "name": "alluxiowriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false + } + } + } +} +``` +#### 3、写入parquet文件 +```json +{ + "job": { + "content": [ + { + "reader": { + "name": "streamreader", + "parameter": { + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "name", + "type": "string" + } + ], + "sliceRecordCount" : ["100"] + } + }, + "writer": { + "parameter": { + "path": "alluxio://36.138.22.18:19998/mnt/alluxio_parquet", + "fileName": "pt=20211220", + "column": [ + { + "name": "id", + "index": 0, + "type": "bigint" + }, + { + "name": "name", + "index": 1, + "type": "string" + } + ], + "writeMode": "overwrite", + "fieldDelimiter": "|", + "encoding": "utf-8", + "writeType": "", + "fileType": "parquet" + }, + "name": "alluxiowriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false + } + } + } +} +``` + + From 9fd214c987672169853aa1b044b18e9b2256f473 Mon Sep 17 00:00:00 2001 From: cheegoday <972684638@qq.com> Date: Wed, 28 Sep 2022 16:54:43 +0800 Subject: [PATCH 134/136] [Hotfix][metrics]Resolve the conflict of dependence. (#344) Co-authored-by: jiguo.djg --- docs/example/mysql_mysql.json | 74 +++++++++++++++++++++++++++++++++++ flinkx-core/pom.xml | 4 ++ pom.xml | 6 +++ 3 files changed, 84 insertions(+) create mode 100644 docs/example/mysql_mysql.json diff --git a/docs/example/mysql_mysql.json b/docs/example/mysql_mysql.json new file mode 100644 index 0000000000..37661ea9a2 --- /dev/null +++ b/docs/example/mysql_mysql.json @@ -0,0 +1,74 @@ + +{ + "job": { + "content": [{ + "reader": { + "parameter": { + "username": "root", + "password": "root@1298", + "connection": [{ + "jdbcUrl": ["jdbc:mysql://192.168.90.145:3306/china?useSSL=false&useUnicode=true&characterEncoding=utf8"], + "table": ["pipeline_source"] + }], + "column": [{ + "name": "id", + "type": "bigint" + }, { + "name": "user_id", + "type": "bigint" + }, { + "name": "name", + "type": "varchar" + }], + "customSql": "", + "where": "id > 2", + "splitPk": "id", + "increColumn": "id", + "startLocation": "", + "polling": true, + "pollingInterval": 5000, + "queryTimeOut": 1000, + "requestAccumulatorInterval": 2 + }, + "name": "mysqlreader" + }, + "writer": { + "name": "mysqlwriter", + "parameter": { + "username": "root", + "password": "root@1298", + "connection": [{ + "jdbcUrl": "jdbc:mysql://192.168.90.145:3306/china?useSSL=false&useUnicode=true&characterEncoding=utf8", + "table": ["pipeline_sink"] + }], + "preSql": ["truncate table pipeline_sink;"], + "postSql": [], + "writeMode": "insert", + "column": ["id", "user_id", "name"], + "batchSize": 2 + } + } + }], + "setting": { + "speed": { + "channel": 1, + "bytes": 0 + }, + "errorLimit": { + "record": 100 + }, + "restore": { + "maxRowNumForCheckpoint": 0, + "isRestore": true, + "restoreColumnName": "id", + "restoreColumnIndex": 1 + }, + "log": { + "isLogger": false, + "level": "debug", + "path": "", + "pattern": "" + } + } + } +} diff --git a/flinkx-core/pom.xml b/flinkx-core/pom.xml index 95b92826e7..446cc59e25 100644 --- a/flinkx-core/pom.xml +++ b/flinkx-core/pom.xml @@ -103,6 +103,10 @@ flink-shaded-hadoop2 org.apache.flink + + org.apache.hadoop + hadoop-common + diff --git a/pom.xml b/pom.xml index 7c2057ba28..23a8facc61 100644 --- a/pom.xml +++ b/pom.xml @@ -88,6 +88,12 @@ org.apache.hadoop hadoop-common ${hadoop.version} + + + org.apache.commons + commons-math3 + + provided From 2d6c89c00488c25a40887a2e4c97b67a3235f7da Mon Sep 17 00:00:00 2001 From: 343701675 Date: Wed, 28 Sep 2022 16:59:04 +0800 Subject: [PATCH 135/136] =?UTF-8?q?[hotfix-386][launcher]=20=E4=BF=AE?= =?UTF-8?q?=E5=A4=8Dwindows=E6=83=85=E5=86=B5=E4=B8=8Bidea=E8=BF=9C?= =?UTF-8?q?=E7=A8=8B=E6=8F=90=E4=BA=A4job=E4=BC=9A=E5=87=BA=E9=94=99?= =?UTF-8?q?=EF=BC=8C=E6=8F=92=E4=BB=B6=E6=89=BE=E4=B8=8D=E5=88=B0=E7=B1=BB?= =?UTF-8?q?=20(#390)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: huayuan --- .../src/main/java/com/dtstack/flinkx/launcher/Launcher.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java index 2a464515f5..4d6c46c039 100644 --- a/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java +++ b/flinkx-launcher/src/main/java/com/dtstack/flinkx/launcher/Launcher.java @@ -124,13 +124,14 @@ public static JobGraph buildJobGraph(Options launcherOptions, String[] remoteArg } PackagedProgram program = PackagedProgram.newBuilder() .setJarFile(jarFile) - .setUserClassPaths(urlList) .setEntryPointClassName(MAIN_CLASS) .setConfiguration(launcherOptions.loadFlinkConfiguration()) .setSavepointRestoreSettings(savepointRestoreSettings) .setArguments(remoteArgs) .build(); - return PackagedProgramUtils.createJobGraph(program, launcherOptions.loadFlinkConfiguration(), Integer.parseInt(launcherOptions.getParallelism()), false); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, launcherOptions.loadFlinkConfiguration(), Integer.parseInt(launcherOptions.getParallelism()), false); + jobGraph.addJars(urlList); + return jobGraph; } public static List analyzeUserClasspath(String content, String pluginRoot) { From f0dc3e942aadf9e346586154c607ffc24879e63d Mon Sep 17 00:00:00 2001 From: Silvius Lin Date: Sat, 18 Feb 2023 09:46:30 +0800 Subject: [PATCH 136/136] hotfix[postgresql]: Fix incorrect bytea data type conversion. (#1516) --- .../postgresql/PostgresqlTypeConverter.java | 60 ++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java index b4bff44466..a405cab134 100644 --- a/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java +++ b/flinkx-postgresql/flinkx-postgresql-core/src/main/java/com/dtstack/flinkx/postgresql/PostgresqlTypeConverter.java @@ -19,13 +19,18 @@ package com.dtstack.flinkx.postgresql; import com.dtstack.flinkx.rdb.type.TypeConverterInterface; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; import org.apache.commons.lang3.StringUtils; import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Locale; +import java.util.function.Function; +import java.util.function.Predicate; /** * The type converter for PostgreSQL database @@ -72,7 +77,38 @@ public Object convert(Object data,String typeName) { } else if(bitTypes.contains(typeName)){ // }else if(byteTypes.contains(typeName)){ - data = Byte.valueOf(dataValue); + // According to https://www.postgresql.org/docs/current/datatype-binary.html + // the bytea data type is corresponding to byte array (byte[]) in java. + if (!(data instanceof byte[])) { + // convert binary string to byte[] + // - escape format e.g. \153\154\155\251\124 (3 octal digits and precede by backslash per byte) + // - hex format. e.g. \xDEADBEEF (2 hex digits per byte) + + // NOTE: we suppose the given binary string is valid, + // otherwise it makes no sense. + if (dataValue.startsWith("\\x")) { // hex format + data = + parseBinaryString2ByteArray( + dataValue.substring(2).replace(" ", ""), + 2, + 16, + s -> s.length() == 2, + Function.identity()); + } else if (dataValue.startsWith("\\")) { // escape format + data = + parseBinaryString2ByteArray( + dataValue, + 4, + 8, + s -> s.length() == 4 && s.startsWith("\\"), + s -> s.replace("\\", "")); + } else { + throw new IllegalArgumentException( + String.format( + "Invalid binary string [%s]. can not convert to bytea type.", + dataValue)); + } + } } else if(intTypes.contains(typeName)){ if(dataValue.contains(".")){ dataValue = new BigDecimal(dataValue).stripTrailingZeros().toPlainString(); @@ -82,4 +118,26 @@ public Object convert(Object data,String typeName) { return data; } + + private byte[] parseBinaryString2ByteArray( + String s, + int numsPerGroup, + int radix, + Predicate checker, + Function groupProcessor) { + Iterable it = Splitter.fixedLength(numsPerGroup).split(s); + byte[] ret = new byte[Iterables.size(it)]; + Iterator iterator = it.iterator(); + int i = 0; + while (iterator.hasNext()) { + String nums = iterator.next(); + if (!checker.test(nums)) { + throw new IllegalArgumentException( + String.format( + "Invalid binary string [%s]. can not parse to bytea type.", s)); + } + ret[i++] = Byte.parseByte(groupProcessor.apply(nums), radix); + } + return ret; + } }

f+uX^r#kEu#j{tdKA`6xiDcnlouQV(vh#`5;8H(CkFd)i$&o1H5DvEjAu=1viu zv{vnE+n>9aef`RbfB9stTo(}|-X-NDgM50g`wc_Z=$33l@y|p1U3vDMfL;Y{zPCIt zE>r(^OpuCAgE(Va%*?%J5 zV>_x+8^bB_k(oAD$Ip0wTq-!1+sdy4ytTlft!Xheb5?-kawjE6DDy;AV5LF!Gni>N! z%jOM_r&4jS1#_pwW3ys~SxmR4e#DdC?^PMC^{gYST8+u=cRGA`*%t<1o%Q)N@lwkb>n*Z{>lD7^Q9}>zL=rV6 z(5~o-T2#^Eth;~B_I<=m?zHE$_qbF!xZE?n0RZu2({~k=Uq5vvx3lYA-r`FiJxtp?+1`WwgJW zh0QK;Sq*9;HR;y-;V)zvtdKPGH|G(oL(hX+ff(z9>Vkp>(Kh3D2jfdKic0V}qBoMC#=(;g}<+Aaj*YxNKlNj|<{Ym=Bjq!!cam;9eKs z{k<2p%gvp#M_4^(B|V>5!&Ldp%TOSWU$bIJ%S&v{*y&{835%gZu29bh93mW0j5w*= zcwu7%7(#g?EXnlnen%Rt;|SkC%&^%p5gzs(HsQ?3y(UWQV>aCmC@w4V(y2Xf2|EEQ zJp418ButzASPEE1oq8-y@On> zbEn_u3%>~pzQLO^j~Tjz9WSm?c(XZ@h(HAhI3Lz0+JWl)R>leBvv3v445@FA!cRn< zDK%Kxfm0cP9Q_}4y{Rp+tPLi%SeOE*TODm#rh64{I!gIK7<{7c4d~TV#zOL% z{UKl2t}$#3S5IX#CduPv^nS*F65wo0lF#kV05nBYr_BCrip!)`tC^V$FKgx4vU=6U zZc~jk6(yon=S4O#^MFL;!+c3?GZU;C-=SXZ3eW`PPIOl;I)TK0K6Ww}drxbjj=d@{ z;a(9Y%Hn3B#y|px-&1{PzJ_%b^Peu>&D?Q=|x2;+NhD1@s?LU)buidZifu zvgh|Dc1pBT%o2vr@y*i9c3vbKq1Fa!V#bFt0Qj zyiPPCrWvpdD^z+As1IYlf=~k=En*~Tng>8=twi7~GNYo(*;!?c3`~Qn+|1Q@UVyg) z%u_79>o2CFjLNda*-zhXNVsEhZ*8&k=fc^|(h$FtwP3=4`P;dmEe9!9xqydr(Paqd zA&TeiNj!h)Md&d*;r=XW{JBn}rZR49)Dzu$a`MY&mGRE$QuB|tnXh|H#9j?i;K)9b zFK!TmB>tCVdUxRaGP>f-QE7L$!Z8W@c#vL1E{xe@H91b@1tE*NTjUKZ{kLXlkxewo z+)7DGA_0=mkb$@yR(Ph2Iih@k~=Gu`N+FNF~Jz z;B>K+&IL+p$!_t3My>F$w=Yt5ve*jznQhpX%<8%(j0dT)>^OXhqz=~A6u1|!s((C* zcek~6YL~EuES=*SLzIsTpK>$)J)FA|fL(d92HFgBk!%*-Zi*){tN<`U1=1#&Pgpd? zj?6YI$4vW18qBm@x;s2DUjpdVAvVLU2UT>^?Y?_IfveIjN_$NZ{h6~hypPswy>AUQ zDvM|(Wz2%3)Ww6r>p*QP!;4vvch(?kdw4!ttb!8ekqebMll(D6o@B+adA;c{Z?rJ^ zG$NN`PEKNdl&0CgEZas!xSCMvyrGD<79Pn)ONm2YFJmdE--8rA%M-?t&Q05u-hM_q z#DH9+x6l}fqnx3TI+`G+Wxo9Oy}Yf6)y9@8CXz5z_7N6!OF5A{BDLclMGkqef6~#8 zF-;uO-hZ@bTjS(Q1$(Q1#V%G#%i*zf7$+EL#xi-*-+?rf^aV+WBqHZjah$W>#E;|| zM1vA-2Sp0zp5B@Ye2s=Bz{H6MdPU6EbgcX*pKH~sU8nO`tE4lwIA<*$aXhHOM#{BY z+|T!fxt)#(!hSqRRwG|gk|m|IVL-m0E;Z0@p5gb*9N*)D-yi(Ptfb~h@YR=Tom(rS zAiU~)6Z@A9+uz<+Y@k}8S`nR*&np{;nO2+mz&@hAU|njnrz~c)7j`oDv#6%Si$Y5rT-P-+rl1;*I~Q$M1&7vn`Z0;xg&L3*DXmOw_$THUwz> z1quD}WUT9Zf}|OQYo1?p^iz=OI4kzn3RLv8&Y-S;vi3#j{B;yFKj?GrCg=CwQrxys zxeH;|^yqAr#)zh*T{JABFp!ffdTehTThQ2+o^tkpZu^K*;wFZ(i?j&%lYX_gMmFBv zZcG^|fg7B7u+i2?$p$vb4!@C!y#;>Y>QT)S_OhcFG1BKAs*vat$mSd`G<+B#=q*ex_TgJ(+RCGy3WC+M<=}szlPwetfoL&blbX0jL1dWv`gh5`}>#gxE~?BsTFg@G*Jz;~5>%E_7GV)) zgR}L85S`Yc%jEWJ_YWA$s14hFS2MJ=*wVJ3MXGhKOTRxFufb8t5EM)_4pPRUBM&)I z1bKxnGFV*&$Ua;cAG%ng387RVxqwO|t+s*(L6k&0@fCNV3be1>8%)o+p14oDo>mN( zXf}u>wnq}}uTL`9v>j}A*yr0bP1(j*sDd2nn5OGLbUk0CkvzSG=y{8L$EEWmZldgxz|&Bpam_FJBJtd0EM;Lyx1>+2Jp_>54xE(@ zVk<6V_;bSi1987eQ3=>tibh(F1?<;rje#ZZ8uHK3l)Fx{H9W=7u0FM+7sKz``B@2@ zN{BXfeUtGy`DWX3qI$*j?-YQi*()>6+Pf)eaRcjG=x?Q#zgq08Vi9bDwshm_@nFX= zz1fantSuai95?C9X3a|0%Np|Ik>g^zFkUBa!p;QAvq_%Y8JraO!j^NS`-5x}VflgA zIfe4y` ztzN9Q0ykLFF4$Y2GmKlto=*Wp?}NOj@d{S+`Xi0wn{UY`3PUk)wMEP($9&;B79))l zyLZR~HFcASi-*3}6i@g`wR@6?x@y#AYSt7L60T|A4oj$4`#M{hdPzklFFUHumT~;K z;ygpQ{UHdb?9uTpwjN(2u$$1#E`xyP|g> z+~=rcS=#>++g?AihuX(J;X9?WKfrS^|y)^b{Q{kkB|X^%K?#;e(qkPHj~c)Y2mu4t~M;qBbPY%y1` zBX>#^AFg(9Zm2SBFvy3sqEwxRDA_Ho!b*H{;gKE0Yg6cM0)yCI z`sMrb{ob)bChQbU$8=t?$GN+J_GH>YrG>oFjLWU%GHRZ%M7NT(zbWNI=NsY6$7-_4 z_dk&fgSB%@@ij4PLhXtWkM{GeQ%e2FPFR}Gr(qVJRC*Wkxr6-yX4nYqs7=4Pj}(&_ zmc&<1cnutq>kN00dNps0bP~d{r%`5{xnkBkQWY<}dn?qcHd_Thf>1TALlLH@y7gMq z7WC3Xo*ECDz3rqC@On<1cWz2@kuEI&k>>-6YKSDjoPpR)(-*;kI)jO?_4VS_KJ(b+ z47L*;d6l^u`AG6j-)Jtp{3Qk-cFQcwWu7h#lOT0^Zrei8%hmjvkj~!S`5Kb+hUJ%9 zZQZOjp?p2waBJgair?O*4mN}-t(OCGMLp3=Q8B2vf;i&FbgyJQ z3~1VHPsLjmgD$#1t?OtGH89rqR~s`FmC{FFKQcDU4FFTubZoIk(qD6BA5MZ$Rk2j`^Ro4`=Af)l?ol@^tN7K*dt(dF({t^%- z`j=Npt<15#zSDYk^GlOr4VF#C=zgc2Dp5vCdYrU$IJey>u~UM@c$~ao_L(kyZyoIO zt{a-~<(hmtJ8n<+lx|ic!~$K+Y0!A`ko=!@#Jkk_{oVoP&(~7-5*%BNxoVRPQetgp zl)OZ#eT5`%lFY?V_Ys__T8YE7dBO1*g-~4~y2E-bhp`6$bDkvBGddwKo#w>KWF276 ze6E6H;7Ehxqs^;UM;E8R!tj#iEyZ0!V3|{v}HdodbF|n_tX8hn0UZ1^gB*6T( z%L|5ZM|P9e1jw3QtLe#%eoiD_tsB&2OYp+z9V)L`zdhB{+l-8*#B+;SNt6%`eK|Y- z`&0g!N@O|{45$>^fv92FUT{~(pI;eYBfXx?p(i4LOd`56my7bn&g(0KTz0lQ58riE z0C(p{tCzD)hiOim@|-b+o;Pk|8NG)28dlH&A-9wnn&m25?uUCiPVFA#eT+8IZp&M% zPK`uf0hjW`{HX#oc zQ|-JqNaMuKM68GzkWJ@6OfqQHQtVZG_Oa#@eu|1#zcx>G)b>+#zL*r%he;ml(8tcz`oN!buJ~=yc z@jE~Anf&Nu%aF2V+qs|LymBwTDQ04e_%jCZmn##_bN+q+cUh2*Y#an`Kp%W z!Z&!xaE+HWfykreU%0B{Ax}YnC@6Av4ifjv_upI#4HvyAhWr;l0XFmsWWVP>^XJsshP@*TZ z`YcmGw+n;|ACYXjDxIi;Td>%)W``xez%<}+VS|j7PqE)ZE+G=7qU5Uj_bLZ3J|n~~ zemsMvvbT7-t<0>}IGlswaeh zB84`k@vpdkSKrD9)&v3cWGNWCAVNkV$W?3&U4Ww^h z(Pmq(NM3CsEn39XGRuoyl(nK*DcJK&;h8(`S<0NXMKKb225>X351&3Q+$?T<2qk|b zd$Po?9bNHDX0!*G;#;B zj#ISplofQ;B$hfwF?dGQn?7K$*Ve=IJHpY~DAF1&gfEwW_+aXhNK??03f4oN>x`y! zMWA1oK_p-$f7d-D_1oG0Ejer^Tr3VUZ=Gni+mH%;xAjsQPjTp>1FOxqT+Mh^2MT1_ zcdfP%lB!AC*`G+_p4`!vqtk6<_U4b2S_(%A0}m!lpE%&v)>aVfJV^uNAskz2m{B@PHv^ zPG&0pQpy?9-d6^N-jAdw^mvn6L(NC-l^5yZ(K{J`U6l`ilxnkKv)e}&_f+azZt({A zJgAv_G=bud_Q=a=Y;2R?iefP^{Oi26(CF|2!IFx(dVe==-Mp00^7hvDmnk+$nhdsz zofZglH{3ZVoD0g(Pq9rloJbX^^M48z$V#}itcpG3u+i!~X}Erx_?j9uy^-0-On=iduB%q=k< zzcys++PBcO;0FF=@9hRm@k_8&9!SdBai=`YN31&DIPwW>QA7WMw(qpK;bc8IL3T4g zHVBUK0P`GG2a3^^EHaw+jmewT-rU>NBbJ&@JX@t9dc}buwiZ8G#w|=p z01!`a^txF>y0$zvlM8_4F|Pkpt~4c-JfB&Hw!X9CPTNA^2onpm!m^Dt=}z|!Qve@O z_#MSoD$v^|zYgr#jcsRpSJ}Q}AM-g8&+;G*c_He^mh3*e*|~b+&PaA!?L9K^1G5^WZxDDaca|Uc53(9@@dD|I;WFBtpn9E zG)w77wEWLCZOd`7)51>5)Pclv#e{qm<(!m4~j48ek?xD=GgQ|@oLrUx`!Lhh0%_VfC3 zQpdB|61pBCO?tl3!eLAX+XMu2L~_Q}f2J$seSecKJ8FF-WiWO$z}kNYh$%u5`DiG} z;Bg9G(p&!82Y^e=h`l&oh0cvnPedNKBM_ixdcmAV+EyX{f<(Db zfA?Bj4~kkU!W9yt&GczZ^7J-7@IBDIR_s*80sPT0Q={Kyuj(x2z&% z#!+tl;(^$fyf}R~ww4i*B4WS5T~h37&*S`!izFDO-h=q-oPf{B&ta4RMcXfIII9vg z#RpC#-yqAL#6abyJe_;$oasF+y)pJqjoPR)8K_N|f__*{j$K_Pl#_2iAG-|8m0%NLy^m)}^Ki(>LyWGA@I z>FQ7UGrHD0!@7X0@?oA-sIi)#FceiG045Bf;+6w=m7;T<#0zWF=n-kz#iRU@B*jVF z4wOX>N!X4ldc*n@GPb=K>%o@qcId(xC513ha6o|cNRFi>{teiLkFsoRAvI@x=x(;j zS#M%Txy<{kVG{P!kVic$Nd`m8txd%hWA;GdD?Sd&ATE-PW-x;n3_uaDd$8m-$td{4 z7*b2uD;u_~&F-V${8gxegpEAzvYO|&zjFqok{v1i%<*EY<^9_>DpC6Bt?{UJ_=eIip1E``naw?_oP zw$X*^ad*fV?jYc>Vn(t8ehDc-c|||G*^}{|m}lzadY_l3JL-kt3wCQmq{uiF%vtx> zv(+)3U*iH36!(o?p_Jcm*+Q*m0?Zap=vKcAE%;1j0K^o2g?(}47E>XT+jCA*(!~oL zwh|vtMG!{C32<6$MczZkF0$hoXKW2{?K~QrhRtTZKapQbQu}sDYdgiBrZ-p<*Cr8T ztS9!fGQf80EM055P=f%Z2p?ThDq^v`ZMPHa1u(3##;jx3`Xfucn>}|4jo4$p^;Z>; zvB|`&qexZJeh9h3n_hAuI~q6z1|2C{hEtXJv%zoNeAm*HweAkrCye8(_vR3xhm2CCQW~QXM$2q@Wuhz#qFf2{k)6w>20Zz7rl( z-C7xbPLIwMGG_o+lH1(dA7inYo*qNbAI6j!{b#mtmYAWSmFRaQ!9o6}K5D^~EP(0w z$=Kq@H0$AJW`7(#(()Nyj`0ET&u4t0yaMAFGi_Ts1Fg9zBTcUOpWJMptev4S?-+z8 z0PF?*hfd=#%#Nxc@Wo#W5c`!7ApAnDTN)tKWl~7!g%fw3pm(sm!1_=yP0DL;LkhRA z!4$Qb(+yPxST9#@XJ?=&<{kN05Hf*dxR*flkxwBZ?4=ydmzI`{Oikl6J7OGM!Me)A+uwMX8sEU? zbbU5U7{l*-i&{bU zAr5h=G?`GuOnxwrveWLC5pi&w-l20liO^_sfbIXP!p20|w71PsGWKW#?rzM0=Uzfp z@@FvhM_k+X&D^Hb6Bht|F2T=QtJdxQgiZ@bd_q8{+KdaHm_0}gBYA%Nz_Zc=3G2K+ z>6e^LbvRZ>fTa-bv-Vh}RIP~S ze$EeGCbsnbqGYbtxMKYckBT{)UgmVE2%SUyaZ3?^lclUfeTWY}vP|f^9Gh_?myr5= zy&DB!K56TR&gu+jXcdu13_Vn$8$`xt2{ox{4}N>0!dc%w3S3w9wXK>Mpu?Do*~95tz0bveER?eYuR ztwh&-h4~>@O}w(DS{vRdXu%00;TIyp;?!rni}H23U;O@a=_zkpq(54b$g2{G{e&)g ze;@;Lsvm~=p2uk0U1lVc+Fy~Q1@ENRbz7nOx9o%e#BF>H5xdziyqov9XDVK=k0l%r z@7qF>&B@W?h!F6OX;{HqqF0|N#I0gwq=^>}0;D{mQV3>7pTryml&n5@z%sk{{|P!g z{lRvwKOX+IoYoRdbFs!Ae#MooHhrpl9@*rwlGop7?Z*C+HKC_fp7Lh@-e%ReFVSf< zymt(lZJ~BQy=qB&rnhJ5i0c-zbU^@!E}y`ZBW>Ew)0S%Z`FbIOr*609#O>%tmW>Vn zbWXi94N(Q0G=3#^V}4-Fb-k!PmS3~<>A0h*cx))o+kyP^j?~a^H5bcu_Y)nC1<0i` zlNG7bw4eI)>9lNd@NJIwXECg2v8?6{Rli|j=4}j@7NnQHqwgl0mC$T$b!AFrU%7tY zcp_bQKhm(lSxfGX9MafP8ylgGh9Cny;*IA<@>pDYIzV2a`a%)q>tq@9_ zq|C{rzki(rcY){NROmFukAe>A`6d;ToBzib6cNSvo1)vs@A|I&=9 zV5cf-Na()MX=5Qy0{E}I7>;u6S2nVL0L;0;1NbAs_P%PBl-g48R3T?l8K1XT6(i!E zqAgw_Di!*O-2Xx!y1wD>cWsXy-gPk>e?>N_&1LB(5_Yj#6>^4WTToWador78NwFmM z3G)48#kS*;Y0 zQR57@O?fX|jtolEw{-kUuGDDj?^nk#kiK7d{E3&Jd-@`~459d+vCvGmv|^p`SyBRz zmtJ35kTu$CV^@IzxXBs*9k1}`t@jLAYU*5Xl|?C&1A`2hmV%o-TKf*YX9R~A^_!81Dp9PR3o)5~oj!a{Ga*7@_?y;QJyup4@V ze-}1FuVG4E4b@QtLxt?Vsb|z=IP~weU+uEWw41Vu0k%@uTpYGa8)ocZuP~5K2iub6 z_c|#p!&q`kEcmC!PX}|4OWu#IvzBVW+Dd|)r4P{fcR$Hl;fM5nmuPw?Y_A7b_zb15$0H2x@qG*#T2V8Y z=!dXbwTAfj@DID~c+A&##R933kSc?&*=aSJxAAEbMW}NeXyGk+na_LI1M8H$m=|KQ zCm?Ls20tq3zOavl53h&!{zlpjxt=K};z8odjKaUz%9S84l>QE*Kjqr=tFM&J;kG!8E^hIixAj{y3XD(7sjg{UmjkMI`0{4$Wggy1^5t^8JlY76syq&% z2O=q%KSW@WQ=xZ=w!@3l*8%IDP3_x`O zOR7+PPNj&Lp-af{ZPMx+M>=v>f!G@gwwr4R*P0S*XpS8Bo1!t5OdMaY&m#0Nkp3H# zh^F^;y522<(pVJ(nETt+ORr9ZZqQnsYop%Cf>HECsU}e+Q+Zt1uvGqg;seIySiv%gj1-q3H0osf75T8BS5`hOFmG8XwvGVp8 z=FDFAw}$OHmXy5o2%lCoI@|q8@)%9+q9gQvd+lf*AE!SWJ&%^VToT4prwQaiqt@Ly z+O#~|Kd;|{<<)u-$hZN8JX${b@+@akQgDHt)6m*u?y~6a=SQCxGuGj%R(ct1ETOV` z?f;k-v6!4{q=2BV7dB>ugdr!ZgA=vla6IR6^Pfl-|M*$e+!8sn6ijps%{AfQ%GB0L zs}~Dgww1s`_m zE;<}Bj4j_XDZh^!WQ%xyt|wXQTRZa&>DGQ}@jv#>^SMJi)y)=<_!g=(+DYzmZPi*L zRoKS$7>hstxRke-ILJLIk5cFUeY1~Yf>MqL!oM|MAbaa9bI9=QYmshYq^8Z>)Q(EJ zTF$8%&PQ<;Ws=&2LEy}UlW3P3819;t076UTzRI1BxGEj| znS+PvoArms^Nmp>31WKuao?UK$Ggj7EDPT(l-N2LRSVpAvS09dyP4S5&SvettMCh+FE%Utm)YZ|$%R4&qva;}jdgu2=3$-TP z^*k0%HewiBDu>E>Wk;6EHovsC0XH_YPf#k?SYbTjKD}Qq_Kqc0XMO zyPl*4e++Lu8GYF9RcC*~oZ%5{DvCGA&-Wqr4JvTdF`K3sNsUE6ore5cprA@6pg4L5>gAzCh8 zqw$uDVE(j|iB~qG3Nz*Ep7zKqVml#!^*+hGKAXyA4$b7{p4h6zDMOpGhpqhiY|P2U zrDkBDY-`IXqi$|al~uZN_w?X|KVIVr--?a4Gd)%5{wGz?Lr_1=jbhch8zWcN-5b#& z8moyV4jJNcrp*t`tHrv{yOOgid-S{E@j3{BnPPS}rhkW~-ZUT#+k+Zp4PGBR|NDSa z#;bObFr?pd`uiFxzZ-~#feN_^xm6?(IkZLc$jGV9sjZ0icon)@dyrryYdMH8$<#Lk zY>Y@DyTPj6o}@s{sZYj}Et~VYR7o+jF>iB}W|C@pp!?t+V-wZ`=9>-ohHHzHOfma@ z@kT?uN8LwHd9;Ubwezk-O-mBCdekjmiCQ3K6OB6BAq9#>Q)8U^emPflY`%p|NpB)v ziOnGvE19qQgP%M$-NO?Xeh_G*DRUwbq8)7S6`S(tI<>7xnz@_af37mK*h|hP44(Nk zfQK$&A;wN-Pou0tYYnw>FjGw+^`K2D)!J=FQK88X^?=s^UmJW%@zaltu9n{-nxJV( z$@P;@YeTFAjN%8D%9(oWItceUuQtf zRp*k6dzMLwzwpbXAQb1@)yyo+6VY)3t1=x!`Vyz8GgZ@pC&w=}5@iT+v>ks1O zO1g^?N3t2Xb{(Ei(HBA3IZsENFHX)6Z;}gh#kXHmK6AUzoD^)k(ou5%iqdY{a&LpP zTS?ne#+4hSB*9;s=3L%@Guy#TK*xz21KGFoWF5!27WO>%_4RMn)g;4Z)8sTP+p^DW zI_iLq)h1x7g_Wdkaiy`P3S3yRduHnB4(FknCPM(4PTVnwvn84Wz3xFq&LK%zi89$S zA3W`-k>x#-=Sventg+b17yx5reVL1tiIojfV$Xg6u#OL?T*cgVjuR5R`z&-=+RT=d z%$@*WAhRN*k5VlmUrN0=T&O~lx+K{!K`Le~5DqDNMdJH5;&d|l`8*@91csHyBy#6i z$k;~D2{J{S4+?wFiB-mhrH79duPLSRv#YzTP{g{%iJT=~d+&vL? z#j4ijqI;Jqyon31(~~JJzds~|3;E=HFW&DgP&>H#mPBpl{6to0bv=`hps9DmfQ1Q9 z@|cpIJaMR3X5T$wGr9P=7ZLBW{c;EGWpp@eg%31$^1!RBAY0@%mRU09NiRqle_jx+ zn6eBFYZKrPz$Iud$pMrb|bx{A38S1tr$7u=0FHcrP<+R^9904)6xHI3L1Q*N-TiBqX ztN+|{{Jtm0;+9=Ir>|r?DL+*?WNE}8t6xmq*M@ur8?Xn!6kzG-(DsiD4(!pV>#Vbr z6`M!FGirQ#Wk<)TIhc^k`Vli6!>AK7bLG@iP%!w>zJ)f(MGu<%2B;5_rk?Kb5$FL9 z6yY|};25&WRxX~43k&PV6IWM8*3PWFn0Ixr^){6o9i#Z6NsHZ;JsH<{0GEN;ykbwT zyTf)LB32x#z0hywhMt_Gr0fozXh49!W>cXvAw%`!TEv)lfz6)tw#Kt?XvkW|Uk98x zJ96k3a`8axBqsxLb$(3TAX|fjfcOcuXfGbVio3lab~02rOBk{c%oGC)adf0mP9o;6 zxyc;jvOP&PbdMe2;S4{W)dUg7oy|bU zOKpA8OEjbUQCj1IGL0c#lX0KN6~n#4pM<$1%Ax&dz$Hv*k2NK|V{pdIs07)fgRUKe z!tlyQl|1EXl<|MZL#Ai0C@Pz%1NusdYSGjj1*0lLauFMQlbwN38OPp(x8spO&M;`=Y znjmXxYvW~e+mVZia8jL68y=dAMG<$;Sgp8O+{T*b~0&)hNI!T}fVRrv(tGk5y3&2WLDHJwDD zmMdw4ul5sb6K4{#TD;2918G)R;bmC&ucByeFuuXl!^ zu4QBTg3_BSPDHX<5+AM#9WhhVG{4yoeq%WYGjQfC1Z>tV+r^SjtXWhs4VDFG{aki#VtsOC7tRy7?A-+>196Vx)i{#DJt(!kE|eIUdJP@$_KMsIiS^WT)lV-;B_CM7A7x*67)!Hn9UdJ;#>VbA9>B)Y)`!~%OS5|o7Ze6v)`^duH+gsR19>TT_`_*^J-0Vu?gZg8P z<$mb0J@Dx)T*$U#=sJlX@+4(A<1{`cmUfmrmpu+^$TKO*) z|Gkf*=R$V7pv>~-<_rnV4<80fm9mE>CiddR>H6b{1ipTo7uO7i%U_GFs?FbRWvBlX zd~Do}LmUptU*GDZ*};#$e(gQH*=Wd`SKyjcQ>#1=Z()OALc&V<$Wt$Z;hv5Yza%{qSNWdj zQYaa!X)b!Sv_5*~pl|Q0z#6~PV|I47La!z1_GIagWc(+8!$1A5(B6l=g@+DS%;is= z5P8da;6(WRLxE|dF}on%JDii+C1?c4W2NUl2FrbP1rN^jF7qt3b{TUIDb#j zW|f8Kp+!6U<`wJJtv4=EiNKANCBJudU!)LmJoAS(Yx>NIWEGIbWT@$o>jja6CoDDw zr~O;f$NX&h@eCK*h#%q_RE8k`usLC!QKobxnr>4G$M>`)z;Kmd{->+}>Kqpb5n4^& zsTt6D)8`?j(Zuf*|9ccrMfF?U$=_ISD$Sk*zEcbl(x2IEbl~@= zxDcZ8KHXclaz!EcLmc4ufVscpcHo85e`06tg`<~^-&9uY+)-w3>4*$>5Qv5)`fp(Y zSj5L4B!PzbtIleJ-d?1R0muWC) z{i~|{|6ulW_W>-jS&iKajG(s-86ku23fGc`q#&-Y6XG=Uuk^fNao9B9ClL! zIJ-C%>AiE#+3NL{HhZ(Q*fU8>H>061fxP!pgJV;1dQPIuWX3HYXqHZ@l{8J zM=9zl>FNI=xa%k*JI))9yr{5o?P+P)<>lq`>FQB$VZi!`pLdSvjf5|s6r2lcmMC<} zG=3^Xq2s#Qe#WHHO4?r}OS2Tw?)u#^+_%8X*vgf~3=y7mwjqEVz$?rEdG93F`=zi0 zty`@q1<*e6YGHw!x>;dzGxHM0xbS0U;#>PJEqe1O?s0DNbR#1p_eyl3V1P6GcmBqa z$s5!58)OZK3^+*8K}=3^#!8siFPuO4!a~XzXlP)hrlu0g@drUz^De6u?=_Mwjos-q zvb#LOK-Nu|J{3+ZAajOFYdl?qGa8B{%)6rChmTi0$&On~WTPUZuFG)#R;%{e&mfFk z|EdOz$7Xm;DL!}AW5h>lzpjC_cWshCPKM_zAOaWZTUm@@4M`Liz4 zYJ_wtiT~7ZowbXxihjH9zolcg9+&vhjYCEKsBF^H)#uQ|W_#4kX+uqDpycUNHWO$| z0OL;R-~BcJq{V4ROvNYBe0DhFSR4L{acd05-srIq>le1AsmJ9J{O`K^T+5cD*|C_1 z(QlOesDwak#9kwjK%F!uT}2l?%f$g7K=$>0@dh^_5PQ2+qgGM^jU9i{mCS$C7}Lpn zd|J~wJ{FB1|Mn{tv{%63K;t<>b*+q0;`@MTA7IWuOqUz^!_NE=f=9#V4W>8RWa_~U zFRVKe+xBZ&MQM~YW*9pwqJKzebNgvd(x82q@!xf4w*y0s7aa-n729RZZO8$17+C9y zjE*jekr6JTraa!4dAS0B7(jVQAoi}y0-8196p@U*C%V7yuAKW)TZ-Z(+;yP;j?;&h z=;?a9_X(%YSAmA!pewPiO38FJ|37*5P5F(-FMP<9yw% zYO~Qr+GzG(S-otFH=u{Nhl%;RRh_u`6KDJpz7cBrT5z=e8;u>tGHe`w3FZ?UW*D?# zdWB{0Es^JDyVbEu0c%S4Z!8kvjLc;1Iys0H#p;c2*U)G{Y-&6}=vVb)YJb4OCj@=ALGpyzVfGg$p#;AZD8 zueUw+=PVEl+&RJv>*|t#g+E38c<1iv;^~DjMm>r|eeA|o`x%-?=m`&5AMLAx5y;Jy zF5=46f!`?n`uiN<6l&L}h96uN|`i!Zm1_8*<|I1G$cQb?K`tndZaj53a;) zfoQ|#g8nn6wx2i>8i?}i)(Lnl^|+vCp3^-|ym&w63#8!2D=U~XF<1)rC6m|(8znq8 z$LaYlqOupQe98J&`51=d;2E`J0W}2`BXUK_PM-63!^Y7J66||zCY53ObSW)#(<}%D z3WAIA%@JS2B0yRSVxAnG&%O_rpXQ0pM(nVW*?cHr=S5AHE$Gvf zH(qJ>l7$*jmP$MNvp^X34^xm z1q6BPPZcz!FRJ5&tMRlSF$|Snc~m-9yP)FpWXkm|4dv+|gAC)H#(Hg&-Zq0*KQt;f0r8=IFZ^DTBs`m(`v#T{=d6}D9ziLFKtSsOI z4zoD0N5!lx0(`6Ae&LxnKc~XElE}uHFupQ5_Y`cfo(W4xK!xlC#15wBfp&j7dpuuc z0LkNceR=9-Xl#hA$8J{u9z6EOx4{T2K=|mxfV9@b@k4ax^&vt-oN#;&Yp}TsHJjH5 z3W$j(1R|XWYdXVzhKc3gp9DwE&&R|K%2A&^I*i;j0_u*Tm(u*ld&=d}MIJN8^f4mp zxispWg*s};LcCr5s1jI%s z$22W>rE-uCxGHLr2?OQMX=9HG%WxR?N8k3QYd&FP&bqkS0GeEvfJ(3nEi7;50QJ1 z#!BHxCm|uYIV!0m@?1P(7lT|0tJ7q&;3gUZ~LQi zaP=CU7aV!5XI`Z=JM2|HHZi-<WYyB7u~NHO75bl~wnV&a)|VS^276-e#cK8UI->1Xzw!Z z)x|p90_LHUe{}v^V3gzn_~V zmvn}ZR{5QV;%De=^jz{{2~Xa1NyWUDf}#nM&4*t}&N}o4^?}Cc(RTW5@ed`;k2y(! zq4edzS+!n-84JCqF9SFyqhaz+OvvsBcaA^wRx2~`;CoQ!kW!l=&KB@3MO^@HC!;TkXdl= zD^2Z!)8p9C<~c*vk9T8~N`81Dr_O?b;A)Hc!9j1XP!QxqjPsas=`;LwCEj4VjcHRR z4`HE7?#j_{5*qORcYu+X(_>}O9b9xauzUs|=4u^k0Aw~U*!sJ!^lwDO$6KDScMD!m zN&s7dtJ*K+0VhsGny&v2k-o{k*zD-jNM8O>&_}mh*qX}P%=S`fofL=?Yb^;0$1E&) zuq?P}r82fjQP!`lDdGj9bciK@MFu@mY#fQ)U5Xi%3)ot!v?){K{iUePX~MXY5bN76~$_3&KhrW$i~K z6~YJXCrdmfTm2WZyZn2U57%vhPb9dqoT`T3P1y(_vx8|xA>7~lOZ;p>{4TfU7(*fg zq*D85CaVZ%<_Y;+Y5>iBpylaE6OD27r1Mi$k&SG{=(<#K!vev8KlLwwm4=HO1_qx3 z7<&-Gxrm|QDaJ;)&~13oVd`Hn!;HVA1ckWg_+4l%4z%#8(o{WYa5V|P5WAEug9503 z?1BRHh7ua#GJufWzr5DUQgR8{10uT8fBLr0F}L6y#4CC=c~zx1^)%w}acjVa-~& ze;{~1nGUBta0j~+W=*iVt(otMsBfb+G?c?=gZ#SM3=z=gik^#&ECayfIQp(=WqIE8 z{+;=T;~^}R;D%aEYqno^%ji0a=HrKL2W)HWvNv_3AZYvV6g|l|BjIf=B&3kSHop)a zytKTM1Z)*L^;+fE1oM=#6#t1J{!6#{E4>ygvNQ54e~+UR%yiWtDmiLGU57~#Te(F) zd?w=CyL6$AsSz_3TEaiF+SHY=%~n4)M^+&ld#_+U`~mz}g|44DDi_m-)}QSU)6T-< z*2D2Tyut;_pm2RwQPGc&Kh)n!Pz&8T?EB*N7R_-S1kP!VDmS7=DH9GJ{8ZPv+f`E3 zHU#nQd2~;_eo$_U{8rMR<6?mGeC*Iq5U_Q6V1CS!QdO2SEJp-v6Pwm-?}#`HwQxL~ z=h#ofj)Ut~D@`#0&edkrYsaM!j1Kxumht(JE1%T^%N0K(qzV%ZGfE#*zBYVQ09ytz z*W2=Hq+!7~l}KjrgMZfON4A~Zc)3F-Zajlf_0J-qoHA7Pb7^etx`N!%%96hL9egei zR;jrh$-cBD_k1@y2wv;eDuvLmk-FmqvYcx(x&HjcyN*kv$q(hciF$ra8l}E+2Gur^ zj#|8g#PQ)*6Cxz(qx}_$m~*|gG8;a= zVawFF{Mw$l!F2qg(~E9G!j`%~EhpDO zBl7vtkd>Wo^a_)e^wDz*z21kSrH#VSX{Z^m!?U9M`-T9gjV@n3z&QE`4Qv8K^3i)m z!xJtqQL&s5VJw*#EX^4`m?-jeles`Pw=oZ0iV2TW69)R}pfOwEu)KR`JuV4%Y-k6}c%S{0MenF3mtbOl@HPv!74d=NW6bI2-U=sPkOc=C z8rm~D;NTbe9Jo&RH$?el`kXe0rGt(!N8$Ejse8*+M?x4lQ&7{fQUHF3&0s{W*~v&0 zFdDm_>Vpc?S9eL8n7?cRiMVtNK{T9*<~&6}mr9JC9%1iVSgnoR``OlW6e98-5$|4K zs(}R4*|gfCxZZNiw0qc*JkLCvIR`1UVz((H>g&$!IfQZ$AkFS$(_+!q!GlS)=-!`B3YF{W!r!9_BojuT0V1OH3eQbp62MG7|h|;+T`Fzq{rm=;6ne_C!n|hxzohohPQK=goa5 z%pXL1dw0<%wp?#Qcqb^BLwh@?*OW`cLejt6ligw>PP2SGRdYzh{BlIJZEy{~=N8`; zv#$H^ki83KrFMPT^t3uqth|%w;NYNHk2%gQ%(%vCiu~Uz9iX3|#rpgDP>G0QMIEiO@>=k!`?yHoee}A{ zh00m=BPD~4LkBF`=TCrf@NcCLm=`1|e;JQ}ks*^5w{G-)hTB5Oi$voi)%p4ug&b1T z`H{8BLKzCHaUX)#JVQ{&7W4LXRU%d=ttC+t-9)pumbke$N+Bd8qk8Hj;0RcK#e zYh>$UFmkb(PN5chmu^ixq|Of=^2YfP6Dwmr*;Fl!t4whsDM^st!`f+ycvvwk?sifQcuoyv1Fsg7reKdh3n`7X8#{8 z0W8o|UrGuJpzTl8lofYdb_VU46Gva#aHteGob+|I46fzbc5z_g>taLffT++;HjmnA zEZHwp2uvj;EL%ppf=Gu0cX4qEFewcQJ^}@KVEJs#O_)^0gjTxQ`Rn>m*JZMTW-j+n zrKO(;2mts4TyqVyxe(APddi2w!PpOgod3}u$735?+v3_y z6K5`3zza{VdJpU0ZRURrTM&g}HlIr=Ju~yJDiC3=wVt8Ayt;ZQf(5{+O!m-zpp2o< zG~k?g-yTO+9l?IBsYK>c{&E%ghaGc^ST1bI5(wztusZqjZe)bF!V z>nKG91zPDx1#0I-0) z5kaQPE-oBdr3FX);N5}v1O&AUwpLkT>nH%=U{tHcs|TR->*=*iC%m};a@l>R2vnl~ z@Et3AY~V~yLIMIHp=oLCt**3`8laLm9_Hc=ILXB~HQadrLZ+|ar z37yf9)!Ui~cuyVO;37hE z`dUR5qQ(ybL|Zi%#Kgs!aN+}3teQ6fSWr@}55UaCdfVutfP)33k|;T!r*Mggc8XA^ zzK8`wBjIyeP5$>6J+i|8*CnfFamoKjE2~Mz(1`#2K;RV{@Ob}xSD1(wz4039%F0Uw zT&5Xy#=||`{iFB(#i*HzymVan!P%k3`#lTiy43h=^vmg)m2l=Mi6qVVzdu^o^pj&k zsf(W;nMor}a@ww_MGU@aLudGEO=8Y9B&FJVuR3^e&KYrEVqW;@P#XZJ9^7WjUQwE2 z_PRE?qI;Ak9CA}!4kN_7IvGQ%YiQK$OWD6C{}(R;4mXX*iIxUTMP51u8*AfO^F!W^ z`EXx;r}5a*)Qq^UefweQX=}+rk}Lldx$^4=N&zd`AA1Xunuyc+ca35edOYQg&Ophf_^Vjv9HT201~p@on%m zdeNZjs3)+@)A@wCRXpav^r)J6E4Ftp@_cnODn9HGg7XgQf%Z|^emC8Q$H|>_Bvw)S z|AvQkV6Oj`NFWjZV|{biFcy4p<>0)NZHU)*>?HeH7Z=#+ItLyf<%Z1+4-p#+YNsEn zdrrU4tjvFB`DEDcs^pf>fZW*Lntc9Cx7k#Oe5>$P4B5diZ!0$jO>SJLt9+@-qX=ZS$ zES#!BJVmlVztb@Z`*f(}}51$3-W6qLFy&own?Oq)bwl;XV?9e@=wn zjIa4umf4df%zHZY1&Ib(**=-I?>BbQOpEJtHIR^SW!8#$pTtU=U11efpI>sUZOKH! zpwCf$%*+!OTm9yYBdRiA>UL4oNeb=bO|_aFv=Z&#oy5Q;r0U_tYZCw~>XWKnhXq`b zjBCheVUsBMI=Af9_1L-tGJ*Hy>7v9-TQKyK0F;Mr0NDC(gjA#&iOu^mBy z2a?JYng!#vjYTm!2S;KY@(tIk*plVwfJR`>NN3-v8H&Jgzh(_CVmr-6(W~2`2dMwT z{W)l`7rC;h?K|<2RvGdaZ>5akmSPXikcU{`T<`9W3DcaLcM4@?WLFzs6UmV6eG@+% ziJuy(hpgrdEt`9=Ghl~{k}Ule8Eh#|^>vaX<42>+1{LG|8tjk4Y`%z00t+7%57>j#Pl1akeY$U99B%FjwR9A%?@rns+HoJN42w z!^OoYJvpb&TXl@fWQbB*^tm`NLCia`m}Ww$YbPgZL(~t?+b9zVqs!*PXnzTj5NrLZ zR%cmhGM1x<$UX<}QU8*;=EUp7^>)6lNIGD2M0b0f!u4GXC*0_PUKUL1Z2cuaiaOp0GwB<4zlxMb+ zbP(OyMU^-SHO>t=l#91l6!9Q)uy19`YsriQhYnqY2Vn{+;+-2(!DCXZHKovQhmP_x zA+lvbBa1Gb4E24skBgEZy7T_+yqk~K=bRzsLJe*Rn&Sh6DP;^lp4xBITAkWUZVh|} zl1BwjD^DJ=6xOUttx7$Gk==r~2R~svwndMh-P%j89c{%Kkks2s-?x*rXl3UfETHu? zR3E;yTJi77$i@}@S?PJABbu^j_N0dy=zGnlAi%27Zk!rR0wo5=QQ>9a;$6|56}LK` z?5TR`A=n}eI-zxxrIKkyv+Un+=4 zlk7$}DT|RDZcN7M=h)wn>zFIjg@W>VAillewxQ&9;oxt867?aoubOudqUEAur@dVn zj#M5Uv%BU9x-@ySmmI3Q$lxVq_`fc+C5~B=E|mm_C?RR#+_4Mx9?O<*hcQ1jUXBL+z{X@RaEme(M5fx5?y_n2ftiJ7s(34z z9F`b+<|zHwBfSWCD* zkRtuJFTP>fX?6r(Ph#HFLMm^Mz5D|{K9K-~S{8T1-qIXZ z(Z|V-jZ^PthhQo2Vz#Z-3|dlr>JlV=nEr(B%Jw*M3|I#SrkyuxX(|d9IUb%V!u?~t zHXo9{e6;$$(4*1z+lBIk05q{96#ArGf01gbX5pJDlZLsTK*sGH=%30)FAwV9WbuPz zFnzXrKE;WzFLH-QVagj@Ss1=1env_gnH8*|#1UG*aSBB*u5{;_AlbLP5kg^QSVD%h zUb}-#v-6TMGb01J%<^Pum8P!YFmT69LslBpnjULGgRlj0I;)=F z6CD|mwdl0lL+}o`2axiC9L*Ug>b(}LYHK4(CIlN@h*%6tC|;wx^pB8w_=#_+LjH!=4V88c-yH73-~DqIePw&r@%*CiDhStI7P zq+^43F88h|q=^{@1JC^rTKBg7MPi3{c4oW`)Y2&XQd9Nx*P9jpByO&=DH4r)T1+J z)Z>O?NblRX<^=_|a)Yz6mSNQxxzj~r!yyH@Zq^Fl{`7`*Ina~8S`P6(Tu@t4P&39? zLb^eG>PblG-E1*07C=xbSm)U>}~rsi)Q+JgE1f)AuK7QgV|JRP;G9;3mGSy&i|59pl`xtKQ9|aj&Ce z$eu{W{kBw}cw@)i@i>G}-FCBnWOR}&@JFpBX2;1p?h|H{Jo$87JovP1-3m0nB|2Q( zi3G$+j6u-SpHcL+NVI8xTp1s%+}oCJJhmqrf}W48FMnq^fx2OY_+BJ-$zbn6KNytJnNXg|=SrkM5e+ke22)s+`|EvY|D&4& z>8_@lrrKwHmMb}hp-gG*Mo^>B;CUEk)!>(w<9K^N zL3QPpBaC8p{pX-QwML;Q1pnu^SA@#nneR1IlQC^gP9}J!OW38UR<*}i^8J#nelr5w z_%bMeU2bw@^O2~wB=!j$^5L^4>nB7jfh+8~Qh#5`DHj>Dieb>3(Qb8xhT|U?qTFRM z%d$+kogsqETvxnN)zpp}r3^MB{ceR%HSmsR3EI;EtrcIznO~=GL<)-6j5&#jSe+?` zEs>20!jDp%a&zZ8(67Pu0r%5U&%{&oL-WrZZy)LuyY0qauFzRLv zp%kn=)9J_|YymtvZOL&r_Cnfa))ZL36&%&1`eFABFP_NlYs9{D?ZYKs)T?QA-La`# zG-kC_+P%YDjey2*S;AlEu(P^`#u^GKc&P{(nkkZ3<2VayYjQcfudnW(kaLPzFn8F` zpl(ToM?w>Ex5!_SSA7qq7EQI^f6!DrzP;GBA>duu*wWZiEb>)HClbitcpy@`H-V@B zs5LGBm6m0oq&`vq{rEFfNwXX3X_cjiZpie=!+Xs!vcbu*0UGX^66eIu?|J%5Jat=+ zINC1dTD+AJ(GPE@{K_h8=4X<#GIk(P<_E3=sRx-;d>nwY=fR zi~P)1OJy@|RmuI*dJ3*4iqufQ!J{&rrzfqciKR)!9y?Z~S6D7|?ERwT`k)Dj6Gan| zMQ@n?b%>YYMNRzgM+v|KM~#-~TatK5WD;AJNMVI})VvqE`> z2v2yuLpx|UR|ygqXUF$M*oW8WM;cRQc?nj6TJ-)p`xjZ928G0mYq-112 z=6liDc-79%tlcU#Oj1GTxZdo!t9u(6xuJYHO({6_K`{T?SJrMT!gHrjE=0A*CV)_X z&B_y-+nCexXwUcHP-k_)?>-G~f8IQ5KpKenPM?0T49d_SFQ&v`RsnO_9V?<71Mg*O zcl1peX}S7VUS3FPoN{7gSU6l};;K?uQ5R#jASplYy8BTx!Fmn@|75qXK!-mpSPy%@ z;Bm-!99i;R>NrH_R>=xlDIhnRnxO~BaF72`fpNBVi9Dvvz)6ja28~gID{9>KLbAY4 zrndb`MP-FPQd-HN;GA+HF&ht&*uBRc>nPIp4;HzIwRviSM{jZ1Rm?Z>Om&I`ZWNA)4ah7HFoRnuNVpDQ?S zlH>Ij!&g3S3!;C-VtQ#P*eCm^l4V%21%qffFy!9FhqKWKkNr3kcB~voVp%TiTzI1j@zJXM_vf@`$inXCr7@;^q?S>yVoMko=k4zFHp8=eSS0yp2`vuH-NS#C`C7a5}<+M`K1_PwXDcRZaR%|hx=Ry;8* z63X}VQ`S*s4BqfmPKikAm@vxZB!46HnZnbbvEU=KvSX6BOh!_|>5 zh^0xo*7ZpReWsO%WrLF189<9>XJ?Nw%nxLQ*Nfj*Bvn#}liBn|6=HIzEmJnz=~bXp z z7XY@lj&Qv4s_=SOYrS0WN~Te^W0EXp>6Dz6^;ztfu}4|LhMbs%;wQN8>Y1nr1q?$L zE@SS>4XYA5RUqEn1?AzMe#J31HfH*V8hrETTS@lmnMnm-UUBr)5_8OL^TKF8T-$GZ z*Y+eHZwrF(=&tM9;tw6rhsrtT-6;pMxjtG=gM;nW{(%db)T$cB4|x9d<|Gdf){08? zmvRVq`IbkEhAu;-?96o4DOjV}TiBjF-2)(9Yz~G1c5gIAtjZkUTP9_sWpQ)zxqTZ! z&{Qdy{m(Mg_-r;}K|L~3wf1AOU2JV!O9O*}__?{MRedq}#!!(Q(Z?LB_uJm@P#AXS zoPGTms7e^2#rfHcoQ?@A zT)W3efmBEcB0G<*MX)&p?t-;c6>gx3+oi!eBZwV`t`rkg7_u%)-J2X0$ZNbRv(DJ& zkx!=eEF~{UOxd+{@mDA~pHl4ihNAM|R_!|{+ltLh8WiU`S9*nd$|}{!dx80@V;9?s zgRa!~MQ>QjaVpITxBgVxHiLVtwDuNXb1y38;31eqAwRk>zlp2- zfIAe%W=kp;c=(|aImwAODT&}~_nXP-rog;bJJ@!mnuJtavTdTSqx4~wT1PGo6L$N@ zmZ{-oGjEj6zSqQq=bg?Mq9&{o(*&r^Db(PVNlopsZ zHRVaJ0vNw&{g`Ct1NI?v4N`NasTiv=V${~jxv-0iYQuuLkglP~ei(6UrcyPJv8RVT zWzDy`v7At_5n4a=J@x8g+gi>rgr76EWm|vq9tee5nr|`{DV^2?jk!4rMRkTlEO|JV zyP(cyh#wT1zu#_dwpyoDu@@R!4vjv~Ur=znLd7%xuC1LGhWuC5`8~k0m;MD9npsyix53B8K!PQowOmlMW<3eNaBrZ+2Qrphv>V#btpX-G~G2+?FiyfL>y55;QFUB7C1V zW?rYOa)KaY5}uACnPTSZ9XRC(skuW>7G#XJ@Kt7$nmlU%2Ku@^+!2?A9#0yAq%U*} z1}}zs%GH%8cMuQ5^jwc8(U`^`3$Z<3mWGQ%t-*jO$aYl8E_urOYy_O#WFcM=97>qa>S z>5AS-k@5OLxgn4OfcePJ2#pa2iF5EF5sv!UJ`zrCoalQ23+1$0Q?!y>OeDwRj2wt% zD_&9!EKntljV@c469*!1_hB0NonbN z51;Sv|9V|sTnoF;KIfd7duHyLXNC)1R2zyvsM6>hVP0<*p4meki$r|qj90^(X<*vG zV9e(p_c5~*E(xjnmB}?Mb2m!x<6LIYuGB<-NuggRY?Pav%H#)yTu&JOhXHJi5Rlg~ z*WIW!e))ZUl=D7tTfyK(#Gxa$hXJ;SdvqeVF7DR9p6IJrRY4uKu{-Zi&^+8+ zA4ufD6(P=tjGpfb?M7cP}Wh+wp)BLEFjD^O_#I$B!n5Oyw=A7l!{?3TKW z1V12F9wLL{w>N@#0v@YTkveZm*)NX8566}`ObxciHE#}ku^v~Dkm8koBBVCna=MSl zv|^y`FW_^$B@(~eE{BQFLKlmI@n$ zTTpLLr9ni7S_X1VC68>23U|mz$n8z0+&&~)vsjIqV;+3^m$O_Qa4R@wFDfxpMg7Xk z>NrFAa`A(I_nzkPxMya-{Y*}^-!;Dv#gi$a5{=UZ)8wSWy#LY*e7=hGN<1eUBbyhs z6kdBDU&l^QtF>E`J&H<|(*vfZ;D^4WPFD9DzB&G%a>9zTNQ_fQ7{E>7qUOj)>;Pq!6X}UFx zZR=Mw5skW43=)~UQyC7QiWuGU+cK;_Lqh1eRZbfx_(7h74INbgsv9;Hk92w)Et}so zf`32=+vzF)rJ%rfD2WI$#N80l;1z+tHCR7VHocM(E=}l!96ZJE4GSFn7(B4Hfv>89 zjG--L+16RCIJw4>J|wDpsXR2>JI3$WwUOStbF8)aZ1~d5soH;!*r(8R-`9(yT-34Q z%hF&BlVax=nn4u}QL4dYX6Sw9-9TBF==zSxW06Hx_11s7 z?F?h{2Yd1t-O-xoHadpCq>?^Jq!w=1$BzZ*Wz@HJ zcJd1fLb|&jdJE?>gPaAz8LkX6>J!#U!NDk?+DV`t?tcJTG>KQP9e%DA8t93}T^ept zUMA0rRPdq>9krMAsqnY&t|s=jpG2!NQFsp+!|6#lbB8B-cv}sPl~!|H=J*5-#TnY?ecohE<%P?f2qm@pj}B*G|2tYd!^Cftf0dcsXEQ)#IpPy5qiLrY0O zS#>?3GC2_~bvQ<4gE6soS>5t}U+!kB{D}si9mgZ9(cCKL6#B+QX-^c!tWN6_G5@0s zl0#y#;YZ@FAG8v?4=>3Za^k{I8sd=(mu{`@T<+-EC;0F7TE&eA?0&Md>Bm%fir=59 zW;ichFL|wG3(}LB^6mMiQ;WA{_b7eWiQlkVSnB}sIZwcOYsK(bbsidq_u5j&;Co4b z@sCTKzsIJRL659eI5PFj_Eaavf6P+$thAFGb`SbXih_@5jVAYcz=8P;hPFq zo`4=#9w45ETFQ-U^O`q?QnXiIcEiCoX~Kjv@)@+)5JO!~%7jS#CPPukZu(^IRBrar3v#|Bf*AQ2 zF8Mf7h$Saq>(RmbCapGQQY&6~f(hyV78ti4p> z(KIl4$;GH9767f-6*##WHQ#4`iSZtOHR_rTaN{!x5JxG}wj1|Ex(!^*|1WR21k zv{@>~+osG(a+8H&3>_gukETYr8i-Y58*QCBWH3kuiBlM3p`BrHeHIE;mnI}kf~WM+ znQ+uL4RU?Y{Ft?_ob~?I@_QOxe3sOX;8~|zraK8NB1l{ueI-`xq*qJOcy#N6*}VIDf0hn0mK(7Fug86k^`Uo4@b*~x$jJ7 zNyyJ*1KhD*ISrUj(jFrBK8?opo)Df-H5cQK%;)0bk^q$`At50bzG6kHIV!TU7zy+) zC+ehsMP6t|eZ0O~-PN)z&P`d(+?x;j^&QigrmZUavi3=>1HA+vUN8+!Bz}sFoK28ij^j{M3Z7$IuLzjQlDI6D2; zWUXKHui>>3een(F#-|f9Y*))oimy$j#Lv6ls}Rx50=kD1xYV1 z5bsn^j-$r%lKrwchewvPu43DU&A#MpHc;Q7YdZK!eYL;*k~5JQApV4X2vk6F{9U8Y z76q-A?-&YMAR=uM_qF@%o_=>jTIT5&O`$y|pu9)$?GE7@T|CxQ(M#~~xJ!XLFh*Ts zKA*)*yVT1X9lEuu(O<63=|G z=yA$18O$0|5Rmf9C38CVtx2QVQeGeHh`MeYJMMlY&P0AmUS3puY$$}+uA+Tx&-2VU z9{nGS4dJ?#G1BUL!kSEIi%M3JXlZI4y)c7?n~mW;;}>dqOA8kiNMS(>@l>!rv#U9` zlM42;jGVQRuLr!mWKtw$v+i_ADvn+}0%Y7aQ~~EX_ut>Yb#ngPoYC082Wk!WF5Ib- z1NMb|twwNLM)}5xng4_>wd8cWW3FXZ68XLrGQ%LXGB_3Wcb-vZ95$bwwD(_}3=_xQ z8NB_0gSX%KUeMcFfaH?ieL~h&uWfk7Q^T_;OuyqSWuE7OZplN(xp_ zPdFuIZkfT$gY$x^?YMtFi~0zSHi>~EZ5vgb_yay#q=~!}cWP^M6-)H}waXee%#`->G_IEs2j= zRj^rJJM^n%U$?k05$vNI{gY;HdV{k-O3Liab8O#4?;DR}Qo>OYDJX?EK7(`J6 zVkSBIcUIQY2-iweRO6SjO;Gk)aqPGkN7$r2W&*iIt|!vZt=>H@xwO>;_q`o{FL&&MjyaM~OxDMfsrc#7ywOd|3vQ>_CU zL2vv<|2y@6i%cc_X>rZmIvO}>J0%;v)tX7>+6m@Kv?9u8M$up01!|ZNOEpmH_5)%_ zRNw2IRNRjyp>YTqwZ=Q%n==a1kngO;EZyPQ>=(=n3Ou4b;QXyUL~wtK@yF=ai2d*| z0#rJ${3z8#NYR3x1=e5UlZpuyShQgw_~+_r)vkRI-9Ch41Oz z?D}*<3jMFSqJB2Y3Tn0HjHC{x@n^g{kk21kG0$W!u36V#Vz8K*g3Z>a5d`}E#@Sau zIJ0MwS=7qc2hz-@`plt5$2zVKU;2z%3AuP#a%n1dG4l3HIeG)mNEkm(&#-ITN}x6k z+l~5;k#`CN|Lzg&Up*OG&$29MHX!9xdeXLorc^KGjn}dZ?8S>|Q~tb9I5I*P0^d`77~6c1$F43( ztyh2WL%Jj!9XY(cy|IEj8C>5!FABIM372h7S5SwbAxjc^i+d}^g|%WvP|rrsT1cVg zxBFYh_PextnsUpRR70fT@4L)ph}X=$!kRg05`29)dDQpGVaDzQc%N-J=p_TgpSY=Y zuX35-yld_VAUzrE=rn~~&m4Nf@5AvYBI z7B!aD^NH8A8Vm03vzXYWA}J|$-PqChh|^Pz>m8Y!8_aigK6L(_645(5fg|@hZhIf( z0+tAt!YLOQH_e~W9f1Q2J9?bRoNzOoMh^B~tmxy}DSeH!fOFRd`_C)&R_WXn7FBSC zbdiv+?qYsHhOyI)2_kDs$_mRh!)fKs^p&{dp2!-P%TlOqxUA}8HgDd%VUCn?apBh0 z)lF4q9L*H$X?ENE_U)Slp^PI39iA$-Tk(&ocWX+SZe$Y~_TL;SO*vJ(EkwC+K9AgZRSvP7VT6wTTIfYu_+S{ zv*!6v5gQ(6TP;2tV)1yVQ&kmg)X^SdM$=%;V^5TZ+ zBC1`Ja?1Bf(38IfE&kKI;Czm7SUK5LQSC<8_Kn!K#rdH%nReQ6h-T9YApTaXnQ`c9 z=xM*>6Kmt6kD}RkS@Qp#@0XBgnD{I=^HXsyF6Ty?HyFv*fR-3jcf&6Y#$ryg^!@zq zYL2j(9t~OT6)2?tm=*+|53yI^}3$vz%ZRLek>?HF?=b+m6Pm+^is1dBLUiu{@#$t!ON3v^3^9pDJhW z4>LL7;P%_rCxBLOR#r>}1qIQG7HNZtOo`dqu{2l)FG*o!isL_GYa8wtXcsA3(Y#jVFh*|JQY#eQR9$E}6304AA<#Arxerj2D!qp?V zP*$vJPH^KIyA`MOL<%v@HUp^ca7jp}LVh!-WsXX|Xv9p|ZOwPA4o}NP;qEWp!W6a^ zT99fReK}jlu3Qt%{BXfQW%O-jHDvTEWPS;BQ{^foZbluXM-a@_zzl3DBkI*`(JeVb zV&7YiB0M~(29kQespm=^IpiXVG>b50m`0E#F%I7TR*xC{&{ko#j?!f7HUmncE!{FQCCF=S_L05 zuyn0m9_7?znVX0(VBD?+Pmi|8tUKqS!UP{(?l-bkH-A>tb^g+of@~$psyBBjw({2yxQLyw30OyOkFeg9%3g}Fw?dZ95s=I1jsGuB9ku%G9j zO?}E@i$e5g>rQDxUKsL=->Lo`OY3^k_^f~JM*(hqjpG{ssC#svt=Ud65qnxnlrG@~ z7tnsSS41IQf%FPqB2)w|-U z(PsUn+q3pBJIO6SD~OOyF+-4QQ~M&g`(s5F`p@7*Ry!l>46nWz{&{8DugUo_UAaH; zE8J|rtb4VJlw$5P1}a8f-kZ7~8s+tkKdu+g=TF2{wWX4ggl+ep9o_p@rs2e`gv&oW zzh9Y|Kqu>DoEq<%bca4(OBYd(lvWp14>Rk{YqAhD;;^xpJjG^M?agEokExpSx3qyNqdf$ZNs}oHy-! z4>otdwU8Uh>AD{H8Hg97>0^8Q*ruL3-5^M&_hlmTnWtohYIAOb`g|{^@yx9;$L`k$ zwvzS39dF2X7la%;Hu?323ZnmgUJFezG@r@s=43XX9XWUQb0cTSZ8PD&`-^TkH{avV zJkq`EwD~?3$KJrFvArjX^FA^}lfSTDzw6nF%Z)qW) zYfx9@F28`SozNz>F8#qAYf~9{<;m4@T>Q6c&U<|&gT>>?(XkR|m7^SW`xh2_gJH_X zMP*&(_~JGgDn-twFsFMyT1i{_mqkA!q2wNIByvRhc9ADxU()|*{oL!;&nxaj?Ie-!&Ue9^rTc4aLl$pD zACU_oRmgSAmKe;k@8IXSbhfecgjaUhZVTzKt)Obi!P+0!rpmYz5%Am{C-^NE6WSvl zLgo9&TfY}|$q04QVJYxXEOVhwy)9AYu7)H$h6a;ngU--??q9gk!_E2P7 zo$}Lq%u$w*Vifc9@k#TftpQk8{ptqrsjtwIaTyO%dzn_vXt zh1f{hyQ?nB%*nKbYiMJAr>Wc(&-LuU=j(n!BPU@iD=k#iL@I<(4EIrV>;!gP6fwb> zKg>k*A6ZZg3`kI_>B%HoImodJClTm?5TQB!p3$NAo5c_iYK?UUs6_|)~>;ZZ3Y3%&Ux@1qUEA3dtYbOos@IQXzc-Jk^+!99S&|8t)j+_<2Yu@>eruV$u&!2}*lS8{)VBmZc6BEICqoIUj6|Z_z z{kKWcg4{S$fd{KCz6d(QBu387mRu?GNm_6s=>?y;N|i|uo!>mkWKPRxWYS$G%!>cR~KG0OD$`zjv@50{Z^RC==RSq;)z#ag{g~~tZ zPRsP@JzZe`+vmT~2}wu}?SA+Cq_X;cA_8uTt~Qx+_N|n^$6p{ldILOHEdb zDW<1SW>0I&FJAn0PpT`CMs4YdeWZiizq*FAkoCnc0WrU;dVaK@d;J^iu-xqaa@aca z6M8)0eZJkXS-9dub+sx`qN@K__Z0X3@ZUQ$;!)b&mTx~uZ^O=ND^KKZ76tB}F0rqc zv{yEIT}6ESv2X-gN;Q(YFG{(%`#{yxaxGa|(x7`%E9dQ5^D(T$gOlZ5=9_CwGFT?P z=khE6!@rJ|0b5Tl4J=ZFf^PpEz-aUcHr>6l2iDi=>4F{R!m$7V4Gt>61|(TR!<|{P zH&?F3z^?fJ8Nav?k{f-i4&xOC494yCmbck+qYE&AL>Svk(|(~oYw}f*NNbOGTk!!PoCeQ=!e1Tkk3B*g0J6p z?cJY64J$Okc6N%-J8Aoi-pnuaJ@spI_wZ1MT*ICOoOlGk(7$Z$@J}lK8+I}L*bU1P zQ};5F`uO)9zS#N4FY1X9pGEHoCZnZJ^AlK2WmJ#cpTon6-%mGNnlkA(g8&i%y)jT- zyB5n%>3RERw9uY^2&_6t0yJ>(29N?Cr;TT#8%9j08*qe8lZOnwpH8neaZ_+%&e0D$ z581pt7`r2ALoHCYqg59Ri)ENOlry@9)k86bC%Z`6kKbN&nce)M4v3Hxkk|EN^Sr%# zi3YEiu;=JFFE(1CzCU8QySl%TJHu;6Oj*>q&-YV@Qfht9Bs%wt&AwgQ5WgQx84QiI zXIY2XkZC_CsF$@HVM;r= zwyv1ab6e3(YT($+dmTHecamZL+i`P+KX%+YrmOVfo73IK@_mmSZWdSP;3{?_m#f!e z7h(@r*f0Q~TJaoXGjlp!1pFk)oF-p72bbGm|4y>hyYSRxarPhH(ydX3&X0z0JIA*7 z$5nKyc8@PE^uX6Y{j{@@K&Q(4T&&C7gV5u{q^Jwq2^~&_U#Fa1%rvatZI2CBC){ME zE{c4Fo%CaTl5}-eK%mxClEDP3qOGk>;^*i0zv^%qCnwH@g$2MtCbR1Xufa1T53bj3 zD!5Z$`7RBd`_)ZnlEoZcU#z{m_{qkJ@5w$N6auTkX@1-18c7+GJ$>;y^?fo!U6RbW z9dgyaayUcagEgO`s_K*L zv%P|%qWACLW3UgcYWXfbY*PA((i)Hn@`~r_W_8nn~`B@YDeb$nR<9AuR zn6s{dfpZ`zzSQ|dlm?!D1SSO1m)aKyRe@l{0jk*V8SBS-*MT1%--OZD=?>bH(SPGB_RURTd3a-jS zBSwEZw3fat-m!G2@z@Tn!uDrns><-8QOB{umnYo{JNPEYT3=)5I{wMk8Po@(hCDU* z%K+Z|^Pe9-p-B9Ux#^JoN? zH~1Ag6Ww|QmKBriP)ws5cSmUDOSa4Ha!yz0rWE0WJN6hB2pU8h4JsLEf3MZtjU7*`2U+W80;5!FcL2KD0EoVOjvX@>)62m+0@`k`}FA(VsEYyKzf6N+txU3 zS$R1~2uK4VP?3@TMgYadjje8$Hm}vC3b=8}ezz}(8|w-SyWVg8fJyBO33((X&6RqJ zSVMzIT*UZ(=vvHzd6enCPLjsPs7(n6jW*PqUex@*S^zi&U2srPm-n(a6XLV;=gh&Uo%cRs-*aH=3r07HnVIuM-7B(opCK6IyTz3N z(PlhE0pbZ3_2GnJT-k=*#qh$TaBL>#uqrI8{Hm(B%acv;7`iOK{Vxf|M3cLlOF|+d z(9Uo<0muqvka6VI25}hQl_bl^AH5!;^Bc<)CJEC zX{35yc^-8pXdnoZC~4Mp*xQaA@m}6#RxEHH8{PkV_B(m0>kZp)maKP=JQuk}qu3gc z)&uUxzUnonzN5P0i`k}$OsjRsC@W(H+#{QI<@)XUQgK;XB)CU{7}+!jX|xs6l>$WK z%g1}Gp#k3SM+Y@3{SN&rOFxXCUa5NfkYI^rXD}2$OTwal3VZeD^>Fs>yKU+WsqsKM zRk$qTDME9S+z71@k*29be389TP`8Q9uXtn!gdx)o1U{fYsx;zd20w7S0Ood|fdwU300K#MlFX z5it}MZX6hH%(bhmstR137px!Y5(jgWR^7v~Qc$6^s_=7Zw2%`Z5lto12seeKmnD`Q zk5<2M+sx_&zD7o&jm4)C79Gv=LuGH;1Fg9irPcT}g~HDGXx0UTXz04J)md!|?5U`F zPhUvm5N-yC$0;-FHaXLRbzEKfOm6cjvREpDi+{$ZxnV;jVxe#IM7q6!ky8znN1M0| zUo1K@cJ~n$zxY(blpE03{vvTWCT#Fzo_#9O46@e9#$3R$Uu|WK#i^xgaKs%y*(`GL zr#bqsS0Z)$A>P3kJq4wA&y?&m*xPQS*xP(a#Qpy~=KtCn|B7u>z1frls`9}?7YI_d z`?dxim54lOk%Eqno{0xt>Bd)K^p&`%|2yt@w!^xAF-g)?#(}ZOEV!5O2kTGVH^M~n zrSDzUEVfg$DvVg(T}5`6rc?SNXZ;$`4~wr7lE~z=r9MyV&^5% zr3|TSYlp4DNezxg#?}s3dX$xw)xd~2X2Mme8g4GGR5BfDU2d;N&$n z9(+RJF+@ny=_H4Sb7|5AKm0yFgWPl>-Sbz=ySSxKRe2$7B*RaMmuvW+KbP_J6vz;8 zV$4$E4Y>2gdGrWO3J^-tFgTW$K!_9)4gjC;;2PXD5LDnxLs#5c3b`&wm(Gv;a*mE% zfD$!0Jf#~>0@2*VLTpa_a_sq|c3TiVCcrZRg7^R|y! zFc$${?7PI2F_JL!2P>?Hxe!kIzik<2mWC^W{+)dT4~C1m@LN<0-?*67<8njY7j{!P ztS|7U+Vs7T7F0Gku3wblN2;0AQucKipS(M$$vtL_l%f*NSTuj(Nj*vXuu!CjwVF*1 zceG?b=BzMr(|Jxu95=nmd+yt zVLe^sx@f;_{}KJ@uk4Y}1~Yv+MKE>#wBN)V+V+-z#te&hlydS|adADWpKHHp3q)$K zDbe>8X2YOfD@@7Bqq&lCv{#{}!=lMoQZZ(ciWnXA8` zO*2lWJg6|a4f4NXlmv|dU7kOxWV#zx^?LW_nI}iai(jJ?9cItIw7Ug12p{D0efmbQ z!u;PILvS%l%gcMtAG|!e?rnT15G@@+yta3yy4Bv8& z*o@9Edu02$QM^=gDyrw}EtIeu>Fh+>o%v$r>Y4zex=ca0Du;|TeCX)=7+}4I9~_VL z8dNkCv1E;K<|GT~xBWkT1NT7!u&A#FxzQQgzUdj|yERh4Vc8P2lWmJ2Q2ATX_zEpD0#Q#@2~2R%1RDJB_%Q{DoapCUDwc{p{0e; zhy?e{@m3Kb^5Afcp8NIp_y4LgK@ANJ{V_8`fQ9y-5*r_K>)E{z3)5}&c;>b-ocH-N z?tn`w%foqd!5u1hYwTt?uCA_91)Qh`&G8EXyn?li>M#$^m-T%87ZQ+bWgjUbpfz$E za42Z2NEaSR89C+u6F>WUu-5lo5BdJp?O5&K-*&%lYmE-~I=r)-%MjCL{EFvi#w?Q3 zG(Wo|Ei8QRn>)U z#DnUPW@-Wh^ssW^9qQ@n$ulMbZ}Godi#DFc3kv*zFDZRNTJ4O23{#?|TNjL%s%+Xd z(dh7vWV+i7Kz+7lAKB;W7sRH|iuDNc&3U2i3GZov+~q)r--kd4bv{q1+b-)Cp=5+q zQa3hQwLKs&3{EGno0b9+X98|07a0T5@ax#y;wi|x;mJyl#w6l56Jun=Icg#!r|04l zX6gtQ@!p+*yU5~xN%oSqQxmcJT$mh!mli0!Gb{}J1Pa`JNzZK06mG_r1c3j04WOic zCDVY)jZn9y{oh$%FV+G~*UBX|(!BZV8^=Ar$wok;wOtpiI^-wHm)5QWuC)<_YRz$S zVsFBfb>Klr?G?^+HpB})PYN)zm+Qei+3i-b0n;Hy2v)8%ME#daV4y>uG$a<%<%)ua z!mo&}@(Hnf@HJ*J=j(URI}f4;*4zdMyb1=g23E-k8efC2KFCiNm~Jk9rd-Z721Mgi z&#eM5EkM6I-98X%^W0cCIZyVe1kpl1hq!#xxf<>#Xe)ue(#BO8Mm@PG(Ux$@)Vs}Jo%@0aU+ zFGmXr;l6Z#=bIBN^cxdYiqnKWIY8km@ci*{LUrn_p+{bsXMXlCCD6xc{>pzh&0QU}8B+gPz8_j^d`(sQU;CJGk3B|G^x; z&9{};^8&gP_!bQmFqrvTe+;nt;{gBQFw40H2jGTsIW3CV+S&r}H$Ym>zr#5ZVG94Cf-C>n;Q|>)HOo z5D=Vj*#bU({u;9ZyaxMOKvIG}@%c%sTG9yJm+&r1)047O+!S}`w=cj5TiJJ>RsKU* zF-m&ze&xT!;`5q&E!UDMm7B>mej2pphMND{X!?7|vFLC2Xl=bu$Ni)}38Q%nB%LcN zD!@^gR+EC#E;Z)ffB*Qvsi~5$kV03Q9>mvvwi}41gM$O^DgHgnBiDA&2G>Xx6tZ2_4B8UA#gU6EnHYZ&~1~Eoo%@}n)LvEVDLu5AbqRHo+;S8 z;T-WCEmq)y5P)~-g1Buot+bTIM~<7)at>jXYk zH*tYhh|ixt2h2odR1_4tF_JL=Ec1h~u1Hn=|K2RH$@UDup9OA~3b0MU;LA4~!~&%P zD*S(DYrqANJ*%BTsD#ACrntz?|AweT=V1Ko)K1c zL2={Rn&zV&tk&DF&BqpZD^&vlO;)sY5=+E)m)j&RtSPl6jNN(l;v1#KBKrZ$jNe}v zX!Z5=foyqNg9Dl(1YLmx9vKxC6-3^k<73H}FIKj;kpN~UXJ+KUtY?s< zN_$uPhY?E^JjW+$PRP-5Ag{Air6)0Rz16i>tXN$&$|!oU8z14mCxSLl=)gECSH^&r=W%f4s<- zP_nG)xj6PU0Z9=%Mfkt>xR_u|Wz)S{_+E2W@%1LEp50nc(&ZZl6!28Q-Bhl+Y7Vfx zwZ?YP{-K#G>N9A zeO0lpeP_xOVcS6}odMOHYr!)=&v8BS{Kk`K{#UO#A6J|($8`7hBI7;oE0xD=g5*yy z17%Gt)L_vleeiPN6b~rF3kVu4@Uu1v5CXtkbLIiJ-4qAv?1z${JsqoImqxU-2mzFW zK-q^(86WsA=_W$-5XoeYH;UO7y5U&$_4R-#rUl_q-!=qlNdyHZSQ?uyFl%E+Kjq|R z={Dh2)}C(}gCvLW=xD`tiCsr_KB6a3Vw7d0jO~l5x)L{W`cB>9-huci8ztqu#?LX( zRV1Q)mEtmyh!UQhoAK2BcH7gtq9z&B(Vpg8U#;J@2S7Tdw{M?=ehA7cDm6eN7#SJ2 zwzg=}lzB7(7OF@~JlyBE@-JVC8ybio<~ab)3Yc4l_5(9r;{TCr)RPkvK?m0lQJos- z5Uikp{eOQ1dhtO2MT4D6<}`{8H`XQg2IK-5M*ja63FPR1P8tF5H4WbF)rVsxTf_@^ z?i>6!AuuoL|28VsuUqRadtqH6<=7^(I$l?9%;6v%9jR~h9|W_V?r7U%r6WGM2nzyd zSkhs{gUdvcOcPn4Mbovq`^9Xx;KRx9tHSvJ3zmjJX9N}>43ve?+R4Q^ST{%N0FKUtD+U_@(|YZmGRffG=2s_RAH_l-n)9q1HZZp#W>6X<^)*oXoop-9x`dfP}adU zcpFHpY`c>Kw;N!)wKdjI4f6v72PXn&lalt);m;Qf-Cq}WPGg2z1lH=IxWHfb^8fhY?}G^?BF|SjYM1& zQ@s56!omm`$#v+L(-Zb&YHsd9P(ZU+R1h7IQcygl`k<+%7C4=rmq%MvTs#Qk!4OH@ z(O=`^kjmWzd3q4euDc3?M8LN{t^l=wv|wz?@Ml#gU4`jJ0@7~fwA$eWspXh)hk(DE8@<%@`%vPI?U+f? zoaW!X{rtKs$H{7lT!c)^=Y*1aGHDvmK+mPWjM|31i>C)wDNkKBVdN)d;|1#?FX8y> z8b64>J5y6v?*akwuZ{q5PzVZ&W&X39&aD5(d!$war{lpn2lj)Kk`nyA z!D(p}m{c1Z8z2z5|7kujFjmTTFS9jpLflM2jD5Zo07x9AckhIODR!7^Z~-(l&?~V0 zz@$Hj4lpHLY!MFz3Rmp^OuP;bttzBYO`_prSSnvo`Q$bC9X0$B`OtJyv-w ze#a;7WBNqc({uc{D}(l%H>!w71-ZF_AV+izau6=CFdcSTaGwCy{)^u%-QI(hJYz(4Gsu9ZZWaZ{uqk1 z)YRPK;xKa_1v9g!$rgYW=>GgPlg3wmcHW|QroC&QvV2RI_ggn+a$KFDA{t0=`AU2{ z8Iaq4>y~7(ion4~NJ-UraL;fk^Umoc%YaGW4Gy)oKS5TD)!NaK5kCTCaaO}=e099N z1wqL1_ji5NSD@a2bN~+mc!|Vz>6TVhSlHV~gFHY%z2(7T8%SmXqy_=$FaqRzegi)i zSTXzGbrJP;Ga$|pbY4aRei;zm|3Nuw$Sf1U4Q=fKuo~S)N6KoQXb`xNlarT}lt{4j z{gO{2I&bcD{Re@jEuUN zhO!SEErPN8FF7Rxpta>LdqQBx&tHPJ!J2>JxQshGz7E1rF^*sE3`Uo-ylokV%Lm!s zVU%xff2A$`N#CGj9~Fnv5=U_Zy944Mkm>`j&;JO=ykD7lpndkib#zP&G^&q_tLXw{ z{l@cNfEaWX^8u^?Rk{bD=s))Jfg@*+0_mIoQTXP-@`C^Zm`6!=VzZ+EGx@s7GT?a) zAizJ%wI8w%VDS$YH(nl4qCm;G%EM_;!fCMRZuHnS2Qq6Kbxvz@c&T(p-@K${SNYZG zDWLDMUu-Lh&MMG#O)<@7`P22_D=s?nOZ~UpoRvN)6r1ykE@y88jh z&HkgUNHYmQPy@z;fnYt$bJ4`vISv$pPcAOTI?Mt75F{P(fm-+rKQQD#OQ4c)M#K5; zkB=7DLSNaD!bXCy@*`6N-aK^@Z%BVDEZkJX9kF)(@65h%(>V@(s2oeKfN$7gGe&@a z%FPQ}T1L9K4zbg2>p^6^fb_Cj9lb65?0x!XaY=tHxQsOUfLll;+KLGBK`8pkV|?Y0 zlfFG9mmNF17h(6+_(~ zIbuP)k4Gug3&QSa&z=FSCmqJ1lj2D;-<~XdP#=(10l~$^Cnyv@0!=BI9YEVd0LMcZlDQDjS)@0zAnyTyCrJYcC3w6EfS~iVGy*)&4ri|uwjxA)W5o4U`4<`-{S584iyF&ng?$A0~IXXb{eWXA?ayBb_3ylb+ZQTpt2=Hl@)X*s+*GKs6PK2#@2=Agmzm#ah&YV#4Q zmQeue32P!V2RoD&hDU%G)=X+Y8s{*lrLFC@+=&d}G%Y>7yK{N9+93m=Du6zaSpYFd z$ZU5FJ8%%HbpR2mrg)_ADTxkjFK|ZJ;LL?h>}aO!O-GVDuhOeVD5r zCBcuW!uR6U6;66GZUV=XL@Oe#qtda(fxCrwWN(oz5}C{L^GlFIRbO2{m%pJ&3?xdG z(|!4LkR}JtZv}P`z^fc^)qoQYYH}W;wf~$D(qVo@77!-qxc~Pfh3HQlNBPr{f$bwP ze*1m;1+S-t)evec=(Ey1#!;s~y+hV^Y(2{f3}7q>GVbUj=zsK(I&ktWbV*4KvO%QR z+1a_=tuBqjma6&9$vBa@oKd|$j9FMh0gUOs@#vBk{QszX%c!`TC|x)qBtUR?cXwz! zL4&)yhd^+5celm~AwX~o?(VL^p^@M;jWqss-nnzfzCU-ZyY8&1A8`6~ojO(3C3`>n zsl8pW_Hrr&jD|;Y9Fw#5X~^l}3&VJR;asn()SmfUx{NJgkXkecOM!AH6&m&g#u(u1RTdKn5C=Zd;-}M)knw!N(n5vA06Iy#Y z$Td55?a%V$5|K*HR$@kx88tPIH<^TlMC|R2GmJU}W2eEG4oNVEPUjR%Xvw$eJcrrBVW6PF`;+AN zS%6;Rf+mm9gs4%H!nC~qAO)P0Rw_yGqwUoK$3(B+kM?zAlR2(UF0ES(f`oH!VQ-G ziqb<)u*DYEBL*-gFCsiFiot*r32YNSP!lHmUr=X^?jaAXyL5CuwU$ML{!!5XDjbtB zL9Y?vKb6e=$jCCXdKiq0`ybM{D2yCr{_^|;10rE&=6BXol89mAscC7d zs;WV-83da{Rmxed1DzB?E%kcfaapad4RWUTIQBLVn!R=g1!Zq&%d5(+6m z^KU-PLI`$5s|U%h0H$O%TEhPS)KU{IL0>2b3|anP?0}r;ArLI2gfmR|gsry!T4V!9 z!h?!{*Y#T-FURN9HclM2T@^$^ycVtkcStxkklc*gy&VlN`tF-dR_nEfSOy-X^^Y15OVukiOicd^>aJOr_7@&o~(f+8;9nCez z+j2)V4s(z7D`Us)3+Bk7IUdqC=+B#yx!m8j_4NK=E1Y8nw)|>7`?E>s*9+q7fVWjg zn+as3M;J%2%~rjTH9oB3navtyMw z6JUrs4g=-a2jRhIhy)rSFr6QAmKYB&0EoLjw;p^mbrDu5aGTJc<|2~Arqa7~E<6qq zVgfI(H@TRmA^JzbiK?Y{jKrv?JMo}@KW;QN>FVUEz=+-^css0omz;#&+2>CWFLwm# zhPE|Ru)!e=>DDE8SzdeR%DNv+78k%`9d_R+$AFM4>X>66xnmJ)s4{q-k!jabbF`0)7?MtFeDVg!GL!DJ6Dw&(M)2^r}k zFzcgC2A4m4Tlr?M#FAm?&_FoFbbR%T`+Oa z=NbWp;EkSi`(ni9qi+Aqk801oLu-NA6aPH&=^&1#5V06Iqf0b;NKYGgExwV=i@fa? z&tv5E>2BCrg3IoA;+BImHVu^h+p+brP$;KwGayNSXngnno{~U`5f;fdJ64ZqbUCRB zbC2UOabmrfI3c#jZ%qEm0dK^q5fI_!Al%PP%*^ErRngOq+h_6L*zj`g5F$H9q#Cw) zMZfpOU~ntK;k&&C2ilvBH|1X21DW$@0q7z}z-^xayKIQ4c^A^o2!f9)XIu&=!12*l%C!f@z9MtIFDlIIg0GYVi_zQPQ77D%fhGHiLN&HjmN58e1khepuz zoTtpuENX?Sfp;}X^XT^5b60QUj`N-A2C+89?!80M`;Ud ztu4s009%oo9-i<=L?lSjcOyXEN3KqJ#R_Yv&O7)9<)^Z5aTk2ITrPqYcO) zrC7EEumPCT?L)#{Ec<3d-?eatV_zJLL?yckTe zDJdxdsPqem-4~Sn>ZP8zQ(~OJ{o8eUATm>8W^;zqRQ&#GO}W?YKu;>V(`9Zu4&7ka zr=yl8@C9$2Qx;1n$rD_)e*>X-2uF^_O^(tU_jmG!`p(^S1Y_EI#NeE8kocI!_n~f? z<_aT}f*yO60JQw6h`#L)Sk~Z@nnf%`cz}yhp$ev$M0G=YL`r$pZ%uT`wOZD|Q_+ z1a00u*rDK#Nt#yx4dum5$f_bST>ONMuD_~gNaSeW)5lSLH zbB#(YXwPx%DVFM+`*UBVG|Dg}ehrztd20u?LMoWqgY}Zl?=p1 zs7OW`1xZAuNd*d+k&DbTI8lGo>W`GAKo6l@X0r~JM9w4bC0YgO*kQ5mWbV$l!-b#U z75zdYC>d_{_M!RDB0b?sEt-9w4cuEHFWcccI|tqjq?gY~ZDI#O^!D_}ljsD~TtxJV zO%9|EonXG_Vt($kDy>kw6LWRB_cAd%k*KJN`I&=Bqo-b`l450&Z=JF}P_`WmK>Ei! z*0|l@SB_ap{u4|t2HOP=p8owp{SBm}`nUXaKergNyNu+O8jz#lZtSv@ zQ+m{!D^Ye&?}8D|+Vi#BsR>q!Fe%?LjF>)EM72LF3O*TSiN`o6c6kYosTX{u?X-^h zeRG)I-tmPoo{1N%=TF1$#5L9RD{xm@TV2KMPHJ%|4>yG%B~g5eey4?Ia%kM+d4$`1 z6VyU~@1pc0WNmkfkn`~k8_6@?)PhItX^OR1-3!549+aYTEOcjVt+zfwSTT&O(DUK& z-l96E`-%E&21@ZxNSyxsf?# zTp6EhcSTym8G?^F&>|x%0P_-+E*8qtB7=spUcp=#u`WHp(TUGs;W3GMQxv{+t*;On zGn@gClLb@11hU@6>za(Z+pJUlseXMaOiBU-MdRkned<$ldT59g_Qa(F1 z#Kf{wXDMz`vrr%=UpraeyVG*|kH_IE4V=DOcMhkB*L}W>*Hyv8XAI&{x%aYSo{`N| zVkOuE0y4h?7~qXRd6_R4-M24V-mUNCk^ftO)_BT+{ z{f4ACz)SSj{mZAwm;i86tytJEVRCw1F0>O`k=DRhLpt&sNl!nD&rDB3OI zR0)+_JY~pAebAHDcR_$I{sf5!wB)>NCWnc${*a{wiqHV0)I=xft3xNLPk~5@`FV0$ z16974`(IOcHc2L_uRr4%<~^nsR8k4BvHKyb8#RYKP^~x*1wQjI;rHf$#7a)=*L`H9 z0n5~QA>3Jwyt^SE-MuJ@X*ug9>u-hj-CfRpl12K?(9NmTJ2t4=sDT*HZH}C9IfJZR z(`d(@_a}f+)t=)_Wwbw?o@%*=d`Q{;==E$Ao!jD&-4~uv$Y}FjLHm_q#mg6@2sBN) zw&kfza2?yTmbArYZ)$)mRZf>tPtSF%kE!rpYyAFG`IP16pKVPc$=8YL5*d z=0yZ;GLVLb5=r)xIIO;*|N9cAv%%QZH8ATv%prBnBGtdk5Co8h<*o52Z(C%uV%PjL z4Z?eePS{Htg5Yd^kPd%p+|zq~2v6he&SrUMB!;?r`{U)C5CNA3;*+~RxBqd+TK89}fo2E_9RG1pXWpVMr{%j>k&{87l-zxL(>h<4s`O@VWCOul?k3=xqjg?N6R2N|q>;|U1sjTM118Ta%lUzI2gAU@|#WhP`j2Ycn1ivkL8O(an8us*Znh{C@c&pKI` zWWeoKME=}`8MKA4OAm`wwYZ;J=#t$Eh)uafWh*w9=X}bo9^T1FcE#&>hI?iv+iuK@ zLqZ7d9i9F$7bfYX))$XH+l)dVI+&+?{LUWjR7^Z5-<*Xo%J_2SlIni_7Qi~{B<|Tq z_N%fVJ#y{MaYh4I%%~>Sovgr!f)X^Nh7KQNA^?wJ+~N*D2GPNvFpf$3<7Zt#*U!q( zv#E>@UhGF*Uv68nL$@|3wvqv*QXvN=V?`qiOUg)_puBi^Ce&GL^KD=~ll2~R?*1+z^fV=TDUWtRuXmCl!vEEl zI(?NkzML`g!k&|a`$rIJb9S^As)GMl`=cUeFacdoXxKY5YX0s?bwCFn)ZmaPZht71 z_q@*ZX=$btqwe0Wgm?ug;vD22ZzX6vB_EgObB4)sFQtGE#NO|oSsz}WrhZusgHWTE zSbWv*wf9hD6Jd{cBWKxTum%$Yf`xybdE;BoHr9i>G5gO|p3K;)^Lut>^AnRn2SE6+ z@R=$p(niJn-mE8qM=jp0rUHo>{PwFeZ|6I&w+Pzco5)hhb?~dccMU&qNo%UeHq z$cK+Elp6%WbM)M#^Ji#wb_o}`=31IZI!M@$s(4>zSn=ESl~~DY%cI0>pO8wEH;*-2 zNbV*cP9Vm1~L(I`6vvj&e6&YxVATqgj}2P*+fzvJz7d3sPFxovRiY%^yAp8eKV z`nm;Q0)^X<}C{M zbG@E5sqn#ua@AUxILo9B0_0B(={}yqYVfRo*)+AAhi~mN>XtCzBE`!;M5-qmq@9jC z1G$9Uoq!USbJNiec(!jk`Deb%0aUt+57je&ac5C>>|{KTx1GEaPEsTgcII&FgJK4# z^5u7NCqJdxf4kMh$2i={Akl)!n*r$q16q}htDhCR@K$g zFQ%GEJ4+a%`Lwgx(uv3y2esxCMBmGJ3zclp#fH2gIs(>c5ulan4lN>3`h~-#u9!Iy zcZ}eLC;!+_baY5Q&MsZOQ$+SV?Rm=C2_)0Ka&@8iTK4jmCcL#)GDrt*4j(vPu+-9| z8+;Wu)B05%)W5Iwc*;NS;(G>}Sg`sVBiWZ=n?u!c?Vcm)i|w{%-7A*?w-RWYhXasX zH!2hWbdMpN-fqZ2pcvC5j_gB9Gh!$9uyxM`5y%;9yI?e#K% zsbJicO}%RL-C+lA#W;Xjq`E>L@rCwI-^;R}#$f%-(s2x6ccb@ww_7sbx&MAvYus#o z()H(bV75K+hX}nQXf0MUUWd%%5HsVdIn7_lV^i@y{q49g(P*N6Rd>Ugu9iQPm7n+Y z6A%5~*>iFe9m{w3{2)M=Z;p*EpkRHv_UbEMpX_V2D&y`|cE1l?Lz3x*Ppo~-EwZ{w zj2rc-2gdoMRlnc>C4?dO*F~0={F}eKBK0Ezf*XRFfS2=cI!cKMdl^UMH?~Tnm;r25 zW!MQ_Ycb9_gaKzCtrWe(Qb$DyJT27?bcHf_z^uL#2tX}8A zzBPcUj@Y$B_s;eFgDc02o0O2)!3cOxcI-yHl23TL3m<(07 z%kw)|Uo_OnZd{;(dZowut?J|w%H3{UbA$)Tf?}G|wh?G>0aXxvNaF+|H!V9;P^5X0-IKZNvI5 z%DtHQEvEMr37_U|E83k&v^CK8WFA}W?svG|2Y9Xf!!+-F5D#{$nlOL8xb3t5mVU}p zMDnuYNSgbvspUg8;kp$$Zc2JxoJX=LY z@|flG45nM>0ZZ%=Veb8UWx;-sQCEV+Zn>iEf8uZoTh$4{GXitHX1BY*|6ldB{YwnZRKz%IY!8+cgRUuovg{I$_7SK z;9G;mhI$BLKB@k*P`?8(M=>=u?H?G}c5e9(B0xXqUwJ;ru5MO>0T)m469!Bdc>kWV z7l4T411=4$=rb~ickY)Yf|V$c6WSc&gzp2EgGIl_NLF!_sygx$!PnwEd3L8HSMZIq zmKGjqE|rJ5JL-XD`R@}u{=F{qx3OSjr_XPv=rt|Znw?^a`Qs6UBEb87Q#FEp&IU8v zG**f}38v@OC7{6VSmGxd?ZFePWeC_#ksbcgqKKx}{peGkrx!h`Oq}^zv#Kk@1MAcV ze>|#8N8}u%N^cOU?Q&*vl=9Zt$^?w6*{_aZ=Jk1=0LCAg9B@){u6G#`XpZTh$GAr$ zA8H0lb6<$#pT`CO;}DP;mVP4lk)6qOFeMrAdD4K$h{dyyl2np=_8W+Q1NZit3)}Fx zFP~L$mQE)AIDrVboe`7THg`9yHx5)f$uCkebB4=g3%#z@(L)AdRpV|l-;dt`H)$t2 z2)z0JlL5%mL~LwC)Rvt5X$2^!jr8}aH-DJ3%PZF*r z8(7cO1m~q+d@;NwdUd!S)A`2!p#Eznlx2&VQsjZ*jxPVn$0H>*RpDqx_;CkbS%*+u z*c1o%;VzF>#R80fU-MWOxH%TItJvH}ysf&5) zROzJ_0YHI~TpH+c6k=ZNHkk;YriLk)m_pB6eri;{6S!L)o(#EwJUDyQi;KRPaBbQt znk`sk>yWQ!u>o@XO5U?mJn$I1>7?aX`v+Zd_ zFdOu3M`K_33FB5NVQU0vOe-pyx6KbF`};@q&aY6pZm+4`ao;3_{tfrp#p zBQCdSpu&a+Bxe1DRWh>4hPZOI;y#ldQ2zHN%LAMj1~*bd4bb1=TC%F!R!geGHGvwd3JCYOZHuej_ED;cm~_A*}k|7 z-5ROob>(AeO=W((4hOn`!%mx@s$u}iJh|75{xEBrqWS7Qy64GR+i?4Z*A58e<=-te z?zykEanF>7g}kw5u{iGH0nd)@;PmM&Ro)2E;}t##tV}!yZ%8}?-m06+*bHdR6+v% zbNV}y@$o@xff(Z+?d~&D;^lU+?DMyu);rTHg%VWYhVkkyuD4^2Snm@^hQ2v|2Z|Qv-^iY-?PZ%o#0=u_ zL=IKBqejT54n8OC9sK+exDaENNVFbnwXg#yvUmV02oY^X`*+xQXnF`PqL^Q<#X$iv z#tD)}X1y=1;qOVJJWew+S) zEvn*e#jDQ^`8E!|{b%j3UaDw>avfkNd44IOewSE#Gz{%u+{r4>s=qG0)y>T5alln; z5p7s|%d+A5YE1UEF;b}ymaKvT4W|>m&n@|Y?Oi;-ScG32_P^Z2M@ZP+!55ptRcs^n z#%$k_5H(DXRDP`Z(1YntOP8FH@yX1L>Y*YGIN=|9=ca%jIJ*8uL_7bC0_chUcnz=7rf>-#F8ZnNQOcsxf4$Rl?yPlw)i7)W+}nKs}YFZkB;Nw z2=eATo1p6E3=;2*?*cxQoENT|vtyEr<@xicTA#NTeqf3jES?q-&4WTcjXFz6V~ofK zBaZ7;?zDu3U|uZ0U`WMONf10;0$LX#7!1O_>8P8@$)^aCs*BQtH|=3}d8>rjbR?gXL@E?L!`kR*Ak!}9Q2{qbUr3>Wc~j#Bq349!@Tlw+%9Goc zcCusJG-3z`?L*e_wl=E(Ec#964G8YsTsfpyzWSdb?S8>+Ka6b0{MwH5n0~Fdkb4wt z@y~j|BX2C>-XeYFcVXk5ub(*zT$@~mmbBEKwuEk3f=a3{=b@5<8`mh+aZmKDA&cMy z{7qA3Eu_Wlp|eU`ue;g@Cs$&b$2kud*~&Tql+^jA?&)*j#q#H1D9{AmNp;WZTPT)_dmp8{AmM(HScY;^vexz|_{aaC`{R`_pPP#ogndPqs4>1haneU7cdcwP6x2wx2Pcyo!cL0}BsX9Mr6 z=J0Q*-Q`?E4;sO2FBulo)WAebaQ;_P{X9Uelb-1&CN)r&bSM~~NL9oY+?Iim~( zU?jguMa#RB5rsVA%%V(Oo0)bbHe)h4xM!^BnB)vGTe_lZzj#LV;*0%1AEBazBeupx zI|!FIegkdiY=v6YtDlV)LbhE8WEwe7`=mnMn$zXK^k0_C_Irvm5btjb4VrM6gw=df z?N4`xNDOX&s6$L-kC}e|HR|T8#fD6Z?l0aLZ(icOj@lkgY_FH6{8%+}I&(xy03|?F9P% zMk(^d#KaFUH}dLYj~Tlp{<+ha2L5giE4)?Wo*KGYd=82r3F76PtUmO;Gu_FIsCvf+ z!b$9v{_h>b~*Y449j~{5_?0sX2vj%p+o{?Yu zU^3Fl7KD&{S^HH_QbY58?O5p+cnjQoalal)B4;(UA>kRrr7H|E4_>)0wmudNjqtii zbr>(YF2g44mo3tFeEC+Tfr#FxUdk!LUKR^Ewxmh>(6bR1>tl}GLFSpow;=2(Y-tl9 z@6fJq1s=$f(DBz>iRbkd<_!uBg~!6Os$n|hcnvPsIJ==}YK-jzfmQ1L9FDx6SXOI@ z#WtA*+cR(T2`8WN{5=08ZOq5InnKSnt%^7THpE9g+QO<1w0l6{t0p-C{rcQ3dO7Q z1b=aU@?~dJf!}B`3b{47-dC!nLLbTr3-)4uG%xGhNtrO&QC*;P=jhL`O*2u| zn#bJd!Q}xYh2g7{_f7#6G)bM2{2SRP%sz2;Z8q5jDBowpJ12=~WVjDc?W*K*A z9Q^r=1Csv^I3%Q$j0Tf8grc1(ZWK{oiwzZ39|ATJq zDd3Xbr(61rx%j8FO1>5N>2K;IesgXz5&s4{fZjjtz6CP){f&n#AwrNe!r3q_Wc*>r zma9DnM>Ct4>`*@Cz2$}H&S&l|yjGr-N%^_PZ4g1<6qBs(Fme9Aqh(iHGj#2{z@-M{$a9keQr}YNa!Yl9IEGv^t}3GW$)@n<9aN zSt%ZtgDo6w6_zxngCt*ukngX$J};F`Icc4X$Gx8+P{L;b^lG~l*LXrOE2Iqtg-%6%+T0>bXj)w4+Ls{^%(>C4&m-kTvDtbcgiI? z`oJy;j6;|%FXUceMlclW{PBS}vxy~WLDq&RvRry@%)a>0l7u?CE*bjgcT)THJi@%! zIa*n$_Wnxtd4J~}K_9vDUd|`6vkaG?E8|kKc7TtML`LtUk-=|Y(Qi5d%2b%fWj==E z2dGk4wROBzCDG1`u|FT5FPPJuny|k@ZPMb=3Q$@F? ztzi0$dGxC*C8k`)*+An|>8g46P2Pj+>yZZinu}}gVKE1W5cab0>;R9{slkqs#;q8F z#nQJ)J&qa1>`l#~Cp%^oq2yegGoguZc9YW14_rTn!lzNwERXH$f@CxFqYKRQziaL> zwA&nexm}gFXPxhj=(E!oyP~$P-Ov~Uq(#vjGKMaa)VcdTHB|0z2f>#_Gu=aAV$RyqO?yssp3JQ5nbV& z;W769u`t37I9tj)@~2+BO1ujO7FO_3in3M^IpR(zE5y?~Hv6QN|I%QJuium8@yS!x z;DWJD!q&0x?qKU3$;`El;S|?#HZI8*;bL35eg@S1Dji7+`|b7;|= zG=~H8P-@L-xqzLJS_EIq0FKp}xuqEfzN=_L28O@h0>F$~BxjedE|nNGOjR;H6~ubH zobD9*-9x5kh9Dk-TjbJxJfN@h>6PP!`GUUD$eTXh#{o%96zFSZSC)=5pDY;XCS`QNW^_v?uNX$S zQ6FbVjQ-~JP_FxY2Qa!j!7}RLO3vRYb)iiV^CK>T#*v^IsH(z)50d_&&Nx{`FLNBI zJ2UfqGpz*Vd{Eef*ci|SRcn1M@PUk`zIS(2!(HKt1>ux1M{?mZJAV!!k{Woe^iwC3 zNON=t_U$K@g|hIi^$Z2R0|-QJ`FcVv9&j|*>15+Sy7-&=laAIaD4@n4Y4ha9@pQ)5 zH!$Vp8%6GQ+gzm0LW)i&{tlz!f?!cbep2|-kgVpY8lNBu!ca?8q$|jgjJ6E}ao=Y< zt))zn+p6>Btx)w>h2?uo77q9bx)^{5|9*>=iRzb;j9{MjyAsaOh31U-ICn0eJq4rHnpaJnP`UR=l1s(5 z+2NtBBm}k`B-Stm89AZO=@5S$7Q|%F(J==#6(h1zp?qZl(S-;FMzc4M{fHI7h+7K= zfn#&Q0^4qFoPYBCuZUB7<)4Nw#zLhm@V)MIu@>JkfFBTb{8jxIF%8(zmFUIl{7i+Z zjqYNt@9!70XHOnvyR&qLvv>>we)w6K3VQCuC8c2GMi?&Cbp=xPh7Mo)DkTsJ7f6^ z&RsMZ;T8Q*BkHXSq=AwfGc0IfU(F5*6w0L1 z_746sFTys-O*Zk_!58u_QJojwWPg@YI@g7gFCNQDtYhkjM_qQ_zwhmFuw(?kLxeAz z>v(zfrIo~Yh)~isQ^(-(x;G@}0|s%l86)xNW_TAz%-YKknZYX^&3$z8muIC7A;W$Y z&R;x;pMar;3`_xUJ~Fzwof;Tw8=9{#zz(3}x@cqipsHwaV9-c1+I(!KC!jkMxD`II zw4!;-V$cndIw6mF!(YQ_ALqqi^1b_j;CX@@rfqLp1j7evF`Bxg=`fd zfUobYB}|>z+EGP^k%arg<5$a;QeMV%#EbfgEbhf`Xu_v+e*sb#YK==>UI)Q?txEH! z@UDx16!Kzm6>+0Ot$oF(WK@x3RqEiuTiqE8y=&pz=daiTjbAnGyLK6kY_tz&{ceJ}Z7;kO$J zD+t$OH26oKLro(OhLBRtHpMkCzP}&4_|0Y^2KOG)_(3Nwa>7XbeHWB}N$ldCr9f`; zRUI>dpasGC8h4-->{dexbuQ1A>KQL>%D{b%($=Av$f_a`DHKZiqmmKU|x9r55RXewXPV8bw)_b;ZrG7#NMn5^%UCH1yj< z?})l&@wyTtFVtb@srB@T#B!zN+)0TB43b$thMtD*60sfzIN`aK1WYcC@fD2IeBt{l z4|I?FQQoRv07y_XMK8)z zR<%KKWIXt)JM#Qo*q@McqVs_;jQ9Tztv7y1l`c z+27<@nV`0rTu;PAB>Xu!@I7wunncvESQ?RW{nrpydvHR|k>Jce=KSb~M?vG7`|e%& zPI9o)>}wBSmd3<4wVbpVV3JlP^}-yMW7gs~iygSmb&L{sx0v_-lw}G2Yh(Xb4tTY(N7)R%2lgZv3@_%ST0gHRg#Ol^CtB zlItcU$4mB9i(JUs)6gh|6v?sg%{+rj{+MsWC}~tE+no3-pg518(5{c905yq~%=ny^ zGjx-_TVD3URj-AhVdb8ejq({c1HMY=19){3zYNlW4fBF^6x?Bov^pO+br(GhqELa6LN~7NPE)mT612 zQl(^>Fh7fBXZYfQ>+onpp_#(sp7FAn-+X^JrQ5pi>3l>_{rLp=(aK)=ouOc+kc^n( zn0M1`y(&>sY}xLQ18#>6rkx>;q3>!sTD`5P*?=-rk7Q~-mBHI(*()MPrgZ}Ou*G_RT3xm-90toiAa&VLdRYuQnJvj5QVgWVXlPMfnuE zS`jhf!>Bw|*V@^aJHj~EG-b{OL)(uxxJ&GwsYSF)N2M;FU#2V45MtoX+c*QD3Nsy^ z!mqI(9P*X=dKU16dv8&%zTK|fa$jtnQS&pfIJZ!P>4!-ww+ZWZ>%#}g)+yfGPr^QyY0w!#U{xj&oTgNl61x%!2iad)9#O{rS0kId>ybSM!Lk`y8FvMdJep^x9l2YUV3>c1|AT){rE_<#Y=Cq4FVb)Sd2;O+k*jExn1qbM z=tTj(voo;xM+@FR-X=1X;Ns#1@{rlc28)1b=G~?1rP+i7f8E*Bhcpi5k9;$dfq96M z9R9hUqdwJ=XiCins>AF9yj*R}%hhB4AJ^=^|AFnqn&eVFk*FVO#KYD?4%F&-0ckei z!1b?s4XGYtIsNI$uQauW=Mx5NISX*v<^Gv(VZ%hL z_!u$w6BN+ra;|8p&+-axk=Se zhcmT!&JfK0raIjeY%y_JcF3(&6tk2l|fD1Lo0J`DyA# z2h0ocDMAZz! zH}uV&FH28lw!r2S*+Ws(OHB=OpX`FBEfUd_4)qAf%9#6wYQk@7G$ILJ1 zODjH|Dgq6%*J(8>CFrEd}} zstfL=YGgLQU!A)A?aKs+x;}K#dz0RNFPMVoKxfgm)4*a-pHv4BW2l#9Z{?2z-vk(K zWs5UGE$_y&g>xs!B4X9+Nr5lVYI>|*!9IEZDVAQM5qi!G#aBnVVVUfmld+mo8Vri} z3g`dI1%T1PdGFvUiNwUs+ywc{rFwwXTt3{ZcNIC0MC87P<7kW%%HLf^E z;uM^QWy1zKLmLCZlo1y^Oyp_k>u|*sH2PU_R+HUkLg+`t(guNYW&ZsR2uZbH?hqPt zao0fefcR40scan4NkaB~;@o|c#ho>Ijk7!hfvxq>9^Uj?d=m-rExnbYPs(qg?$D}n_{5XLFK^D+;1=P|{{#x?1pLG^y!ex05 zEfLcn;Ph)vNja($3r|zZS64PUZg*la)Ey2=-DTZhcpw9%JQ{>QZX49y8Ar`-$Ht5p z-3q2ySAu{HD;K=gYY%`Fd1CIzeSeLEB}yuoE};f<3i;Q=2tyIfLpRJ}dD#Du8$0-n zMiWBh6W!%LM5lGXCBCd{{Tg@A;MsS%{Jl-Q@i>Ld!u+H6aw^~S@(9P6P^AQ#v`BloK%Lf3h^9vWx;DV~2b5vB9_kx%zg0k)%Jgj0r_ek*YDgn7Nu#d}eA zr!zz_8jkHjhqTFWP{Z8foz55jF`2aX&@8u7vpiNf>vZP{qvfm^&q8Uwa$d2Zf2Cil zQjOX|Yu<8_Jmkn&sIuZX_w|iUX38+StiyC}c{PgX4mrEyp3D@56_a9X?qs)1>-7Gv zsRGd*yM6nK`m8#(%|bEm^_PKV#um|ooY|!9sf`|5m-y|Mcc=y0Og=w1AzW+dkKvMev9m9na zg;lMG^5o z_}qKFcHjR7#Y^A^4{gLo2|6sAXt^q-0Pr>Nh43LN>Cqo5Pm^h>@s%@BEN3$!NAg1% zmhQ7dxJVl!c8x*;U&(rR_vH0RBvIf;_crY2>$1Rrrzdm;R|&JEvK)>bu1UM}o@&_~ z$(1~3QT;`+-ah`gZba;1yXp8I?W+>4c{rA%YtnAg=l_qkuYiiO>;5&65JVV|ZU&H$ zmTnOkQbGhN>24&X6se)R26^V3 zefHV=w|{4!XU-O{Ih~jvl7D#r!$^pADQU;RIPNmp5oO2M7Gn2{lpc4k;b#7TbxL%{ zKEZUkmjUrbO-sq*;qN}W65(Dqb9(s7!2Mx#IThAR*a?F2yBRyJH+tj zs^fYMvGY-vQm_-+Cpz9WM=}0VgU&nsQ|+02=(=tS_S}L$DtHD|&bPinV&i82dwGFzZ6pjJg|uFFY?cyZUqQLLP)N$FKjPeE#Q3)Kf{vWND{!>iOw*|&%t?QT3N zrb^$iUFzrwe1R~wVJ|{@@awf|_k9zL(Y7N(r2DXEsx-a5#l_M#(!02|reHaKCG>5~ zbgLfAYi@WhA@2~Lv`JfRB}&Gxw6F$qCV8c6O$R3;!=QmKe_lxAnRJ?tJwC{cu&vH9 zX1t@)80`5zVs+d)kgM|fjb+8B;fo~{lCufJT;t+gde09yWu6pJwWmGaB=>=cq3PHGU zaL^}-cITzdYkgkl4#r1R+LXy??4Rjv ze&Y3=Ga-#+eYrQqJHYztPza}XTY2L8-Zwh^I(xLiLALJXXiVE$+@<}-2L|RA&TG?xsr;GV5y{s5IateNU#mMb zFS%T3$m!SUS?v`?May=zlY(4w~e^0lZmU)Ii_I&LP3wY2H& zssTw~eCrGieRJ0HbRV3ipVI3;NSntqGkd~Y(bX8p@?rfvYv1rg3ZT&%5o;sSdy(SE z_*R7D$YG>YNUI&&r^szYv2WLWGNSawF2P#TYwhei^d*h{2`>k^f)AXl1=9tOH<2GK zUgFfO;m$6aIiA;;9}L^7IIjJ3{88Qd+kDaluZsLXe#vqj9aSVwd*)~?IlL)-4EzQv zsW_G8mR=n%)+1_ziqJXch9n;emX4~X`3x(?MR;X+1{vId^CKnp^4SaFp_1s<6|7^2 z(~ZgMi^iZ%7Hwr1jM*qjAR^pWd#E(Co$2mOYo?jTV(uzCQzHYaT!x<^fuo}96UV#a zgcOr9udAQQSO|Z=s0*sn$YhAbfEQvrgUJlqy6%aCuG(y-wYcTAT{+#;+gZr55zVsuzU)Oog@b zK8ChLdZ5wLpo63NFwgKH?o^Itj<#<_{crals2XA#842^{eZ;tTyt@$M6u7mJ)Pid` zM{sbH1*>wy=(eZoi7D0fNjT_^q(v-15{JWXp_-`WIDjQ$HE1j%oc5gN-pPrN1SUyd zxNRzC9oAx8`+^218(l>Ew$!~jPq7duKTF}UyeI4$*Be)}3xMFZlO*7I9ju`O)tjEZ zBp)X25xyseLtRYeZ%wtq@uZjjZ&Z4pjg^ElZ_@4>yE{I<#pS3+{>WcXYN0WR`dJ)7 zYnvklj$z`H`F10;cc9)zrK!V{fa_&Rk%o0+c^~tm+}UJaj8#`+y=w{ecu^;iCqaT2 zdP-6mYI}PbNOUG+Bu&)@O` zy1nl;OV4X#(<3bvu=+~|$y~f9HW>>+fdomClQyq_t=gm;h9qYUfSTyM`7QL zOd`hnly0~nz?X*^6mL2Bq7zTx!lnAm4)@l*$oX50CDyc?>``w4m@QYx7H6|LFj<#vo!0)KvJhDEEdM*WoJ~ z-Zh-VKe&k_>}Px%ZvkQCABeI?TE|)XFKuNH{HOhC9h-2(KRZy8KN@4ua4#c~>{Jf4 zTofQ`X$H4<2eY!k=c++7O!rRQeS#N-NLuWq;&v9>Zz}Tp*<$)%H|2`tG$F>fL5VmG z^pH$!uHv^WFqZcw1hE@8H-;PdG3d!HcYTUM$B#I)8rL(35iS&LkH*q(FPvmVjk`v9z z*;)HwT#nt-=FYR(0}QLemnS|HIM}0QT>iIFDr$D5#KD3ijL-tZty48_V|m@>D7kM= z!WUspD>0Wi-~OmOxia*fNZi<1CH&gE+q>S85^t8JKR(7k@WMf~^ZCou`g{mOnKEvP zF`b6mn@HljWt8OV{!XaIAK8qy$3?mK(z6rcBTRy5w%@h2@ZDhn_(l8+>aGP)9|RZS zwSB2nildgLAfCi)`I6rosxGOhJKBubNU#u}fKQWD`PP_JLT>KtWbbv<77hWTlIzZ! zC-a_E@74g&1sX=Brz<%*af?4ISU};XBn~#Bj0-J%HV;m~WVFYVoCkaLO6c_T6x7yG zQ zpAu)#aJf6a{&>dI!ouPlv!~3Q^8%^ZJds}v6PCF%_hWLwsd)Y^U`f?j@g@$Nqi-idnIz6x{BHW5d*< zYy^z-2s##xDI9D`z%>qQYYUeVObcGagJ(Iu2waAgjbSu@rkA#LEn{PIrT1H|);hn2 zZv*z|f(;j>vtAo9wL;0l>AW(N`0xi;5D+dPA+?bZz9Q}ou8)+7-jJJ<;V;V5pGlP^ zOYW%GstYr;Y~%FMT9r}znbv7pY=#=*!?iY*6v; zj!lEgbmNkWp^iKCJ)peJfWQ(sLwK{iKMxkmc&=-tRgI4wer2z9egm92A0Ll8SXmMe ztX01UW0q_{;L;}s)}tkqTN4$F5((ZsDU|4U$l8wYeX0XT_JT8B_Z9 z%D{#!&IN!6GeF>|k}#m~FaZo^$`UTAs-Zv=#!7$mM+#@mCNx2%w~OKpmsoHbCg** zW7YLo3`Ntn%6t_tphxtAN+BwhE`s0f)~brz!-CQzW(p8v33b?aO(f&GA0IM{JO%a zA9*a58Xy>9*{ZHAptGRxsPn^~;Kxn{2}F;tl|nNQtw+^2%ZcAwI1!BtCRd|-V%F1* z_>`yE#t~#CKP=RHi1l_SG||@dBIRL}W@uU%6vvPwWI?G&yCIk4o*E1XQZ)L`HAQJJ z@9BpLH7Sx$#6e+cWEx80G^k1b_o9#|0zLn_#AR$Kjj3w`WP2|ao4^;8fk1B-hIvjc zCT=j+U*Iq^iFQ6%{oL}n%(fN6Sbxgbb9+n)-=L8PJg<2kjZq>R&4OFEck-$wHtci< zz4twzq1Ko6;qkQNy6Y~2wN!A%I{)jTVy|MzuTDrRf<(Gz*~JnH+V#6n_D^+c<}aeC zw$_cx(=7+*;uLBM^jNG{6_=J(66aYP2|m4?Hp)ZtkY0|SRTn6*=K;n4z8o=82J!1U zCs8I!#|Wd*a~k;iRbWUX?AO6fkB&POV?%xT-7W`EdQkt_6|Jn~6;o=37Og z;E2r*z0u<@W-50)@2(upY!ae#iFvM2`K>E`pBBPU|BpQgw6XnmuLuXpu=8SuIG$;= zm~3`BjoU^`_7C30D5+V|Rnlsx#Sz_8BQh6GT>|G~1M(FY-Bc0hl$R>!G$a{z{~=iw z<0D;H;z--no7(CNQw0-*jy0vp|A6 z*Q zjJKA2tsTF;Y%>3{*C+Q+So|QIvVRj-W??PG;#z2+jy@delBrZFgyI}@$d3PR0 zh`kBr5J{gHA=9r}B%FS7|TcCVs( z5s5w)$7;4Vusby?+mc>_R*bXY z)Wy>&=N_f{kLZoaCrAFjCXDW-&)AF}{d#9R|RgI-s)<1j@Z|A!$ zA@mE1e*F`Kee*X^(mxG*s4)8E6B(TWe^p zyRX%;RM^8j@iFGHDMOARZu~ck0Xppa#)k_J<4SJw%ipRpGhSyL-z; zQfF577Ja_UH+&x1h# zW4|u}FRkAmMp39{IZ(%ZN6b5npw6e=vekYiE2GasgFgLSX+x{bj4JbDnM{ntQk@x^ zp&f6mW& za(^5PuL^&;F{<{Y=wU?`kuY4+b-F_%RXuDcNaG_V1fr|>&d;=lBu}>YE|Bk}A|Ql8 zeb&%=Fnetx3pekh0gHb3LS*LRB_hMnww~#@-i3dQ#}?L&5nSEPLLB9%YaeuxTz4U}1<)75|E541nl1{a&gy0fZvlD;j?@nAe-kh%RJ!cM*` z5&Qladd^3^FDdQL+7-oO`45ZzK-r*wua5cl-(y~sTlsd* z%PifU8TI&4Y+M!_FO@Fyu#1d8==4-faS&MlpxgnE*q-9Fc1RM}C68IZfn?dvlX7MHPvy@C z8@2DHS@j7NtLTh0c+qD!b<=ZjMAp|=cnPdquGqf)#<1w&Jt4IpVq`M%Ua3Sq{ykG= zHsfPO-(#K?HeDD@M5<5dPGQDN-mO&AGN#blhM!z>_uuB*dqkXicq}b1m;Go!kd}WU zGf(%V)5Zlp9`yWjVcgx))^eC8UNnApjlQ+|yIZ!F2G0D?8m~{|%l>wha=A(M^&6vZH^tlQ62+3ZKUHY(0!KN3Qz^d~MXD^Y!_GulZs2_R#W& zRQr6DSdCqyA)ARnI~l&^UOBzXodG7x^KTXB7kR1o&4u+mzOSj8sCOwBNNw0{b@s_i z7nm?U%U24r8&ax5aVbpc7|WS#0=Qry$z~y8sIjnHmU2pS22RZTA-XCk!RTwX{#@vR)QPUx}E^Xn2Iy-8G z+&taA@8lHf^l-`>{$ZO!F7~QH1>t`TKd{44hEXvI?`VA(Mci*<`dynnM#&SFoGoiS z?IV2IdN*5HESZw62UNj8#Z;ergL%zSv<^Sl9FC`dTNUc%99=O@0RML|`Axq6zJ%-j zS3E|w^7nG(`j46Zk9YqEZvTJZN!jnFPi*%``pkNTGrui_u?p0pRxDQrtk1j1?IX>* z*1hDq-<|C-eQWe#M6Y8@7Kq_*y#FqkeR1v2vjDoGvFr3cCEc(*?M6^Tpg~0 zjL9IM)k?->z1u`h{j59vbWLdA$UiR*HHsEw*kt`-(eE5hxc?&sA9A&VIMZ~7$Vad# zxL=>N-T3MvBufCuBOCWEy#BC-KFq-KYne5%(=hj?bF)Aso4*9K=| zWX+sz3X|gE_G%vFnpGchn3ttUpO4#v{7QS3Gj;aG5h~L#pShQsww1jHY3Z6AB5#3%?-WOb=F?Lu$&GC3#wy||gxvu)x<5!`pVhf+jU}+a z8RM3@ye9~WN;+GWilypGybJ7$mk#-7Oh?`u?I(4zW{-X=QSqG^a$v4<>ge&R|0m2z zv$Z5`w2XQA$y>fAEKXr8?eS4Z0oW}i;PuX-u=4hepe301PW(jToM49B1QAd%;ev5!6 zaffOd9b?l^!ct{@czT+Q5sC5Pe!vn2UsqR`w9QSX7&&<>7JQv+k&hRzx@M`8r9>Wt z^YUljrSg1@$hY}Ns*AtgeKxb^aDFwZLR(Hrsf7s0gxeV97tiBy5a9TMCAhqfbE>07 zo3%@^i$Sp44bo{Br3m#UWvzOn-f-`tcA5LE$$n1>3q0-AU*Gh4+dNZRiJ0#}T8q&8 zSoMc3xL7c4lc%hv?SUd8BOE64BO2d|@(mBtKfmVlJd8}bp%W-17Zb08}pq-s^=qFG&IhM64ODBMAu`))R4q)oYbug)Luh+h@f z2@98dS6C%fu-G{@>)Ey3vEf7y!ezuoi}}JZ&+b;!gRQ1tqk!v#1(mpg+sF5S1c&4% z`c$|hqS65o!)7};GS27ilR5a|30tV1wq#{g+RniXP7-~MEY5*`V{zZGvKwKd2P7BS z7gL@HrLanW-p{ViORBQ5xNlMriABMy&QMr~$e_;7Qnv3ojFIES!0X~F4aR+iWAO8! zY0mBHm^1U_>+%rn-us6%FZh`k_DBQ@hD7e|jK?Z4VsVk+pucSf{oy{o{`-w;JnUx% zr7_>I%+0e$L%GjK90u;m9{f1IiV*)9dw0+G>J)pb(lW&3sX{Md9`g#**w0NORPkd_ zfbzRrgV)@&{bFYc3HsZ422EyVuFtmEI+ZqNlXE0I0nF%CwuYe6+K;>8Pd>b%mvsKD zXn4kJdNlXkQVIESZ-v^439)N19=M|;6`-|7^wV^3aOZ%Je(rwfU z1_I6A2hc5^(s|yVn2z;A z-L>#3b}te6cRe>o-(3~{{r0;VU7KxChPwk=F_n+4@5hC>b2#)-)LhDGD$lmwIsTz; z#`H9ggd(K_oa};on;lbNp|>1diq&J(JXB3@?~ zI7rQRnhSfLKGMF2P|H?4mYZ}XYe5LBtGxM`RiA_ z5$#l}qbj6XmE*#G$%nP;bqv}7^Q)&ak zU<7BE8Q;rKpH%cT_Fr{o6TDPo_t@V#sVWldDlZ~dCz+AiEO6UL4kliw4NBi9n>pLY z61zHn#_iorM}ugnvRSQ~J+Djq#Er$aQtbPaPt0q1^`vu*{(}Yc!B~O}y9a_?Yk-JB z3Y&c9gptp8iKXPc7mrmfqVJ+ycNfvrml6nk=3lk6zGHSeT~54eHw>Z-Y<@9@4RhZC zTEZxMu_)uGW7piY{?uZa#TCo%H4Ucwsr*;xb2vo}+im$6MaHfYk#P0Ihj?v+Y}zkz z5FhkmjJVR}AH>Wa7KdYz#lqASl;^e*GukH}%9gu&5Gxfje(TFvt7l5Kn%iCEn0rb- z@|EP8WZY54y$_r61fdVVmbox&^;srRJB!Ft5aq8gf@R&b_!p(6R=&2oe6z%XI!;{}&ep^Q=8 z>Wq@S`_6RXY{F)gtl9VE%LKYi5}p$4cc*e_J@QMBv1Jul*=45_P`jh8Ts;?_)=(S& zT1b|=F1SLLoi>Di;w#e8yguX(f`f_bwzRS)pSLIFDL5+fXPH+>eYTx~{TaNWJ^+>eO`Ps_!YN6rFh5c?R)1Z=hZtX%=}| zVZO7_I6;0gYF)ZK0sfZGI*Mdo?s}eV+NKs_dNCum41UUrRpll&U9~(rT{pi>VCeS= zzdGO$H~ZrrKpyu0mZyb~N%urZ-xoRG_C`*fe^r0iDuTo_B;1&xAk6vraMswOZ=-st z6P*T`f<4;-dvJI6uvDy<4Ub(g^=ePn{A-J{g0csJy_Tlwpt??SaZB$U!8!Q;E>ghv z<`z84;*5p8VV5nVVZMBd-|r7@?Q#?U#|Ch@koa%}riU``j5LXY+B2%@4)4fBfuo^p z_<_RS@&_F~({o}KZe28p#ubMK1$_K+5hTY#jR#z`cyV<94;0p8`zZ-E5g*W&^pQ~c!0U6ejXD2Kzu$U zGtx;*g}5u&++}jm21`9>fy)*C*{z(NY_ajdqrWY3*#C|&z_jP_^61>ICK0D6P1EfY zRSgLhO0ae5K;+8({?(JGPtmD<(q^95HNCSRK23A`DZj88XBQS5COe+q`bXH4ej1cB z{p%(aS+{jgI>dbfRjziG6AXTu*IzwkiV7AA8B*fmQ=dQ0f-Z>WKsGhf7tCj`Sg!U9 z`}YopcNQ{zu9SXMK5l+inA(-d9A6~S$&M6EsNgmH({6ozz+dqLABEq)YrACZd#u{% zJxb)WYCOC2$@sY(6WtI4`J|_eoArgl=uymEG8@h@}@7c;(LU|zH>XLyE}~uxyb@o z$Xh0_Gzn5Ckd*^4X}$1wvfGR1^Fl}=q42+kwNLa{ktDbeme23swVP^zH+c&ZQ;av4 zTIjY}BZX7n2sd{3^%K6nooat2bi8K5F6zv{B`@OI6aKxIe-GKC{EnfUilUe`Vlhqs z+k#jLg?MJYk!dIPd6x)_M^xy z#|O*!ll7ofr(N%8gwkGMVf_62YW?Koh@}D9|Zqs_-0ja-)U>tm|FquAv}sR6{J9zT@QW@)hg{Z(MVuhWj(n%g`h zopDzw0tI8Dfa>Z=ptNurOaTq|9Y^GR+Tx43v}|_F8)?oI_kjisXTgVl^+ziKi9xk@ zQxamt1p7VKO&BBw)93F?dXoBMIXVd3p#W%Anhqbd z$Ocsfzr*%>?M(~6KuLL&4h4cr%5Op?Krdh{SW&+R-I@Uf&ozSPPE-^iF%C$>lL~=V z@Hh+L4Nym#->Lv&1ip<&gSbc5gZUxBMf11|k&|orS8IkU;A6D#xA9P@VG=!D2nI%4 z77E3^1&ZD@cLNgFm$eB`Rrk%a9pDB zgO+4&9r<6)DMNy8zyww~d!ySzffl3uog?0+LqS`yZH-ZFmzcmK&XAt{^&vW^$R?k+ z;jtd;MCq?c6En+GzDQbbOnc1-o(UdS&Gd8GQsDJZ?K9hvizSTez9QS=(s!z1vR~FA zC(!lkV*dPC@U^_i&$rg@IUA4}CmZs!BB3LkQQu>9Dsq;x{T5dD!CK9gr|ApPsg3;- zJ7U_G+V#hOjG#*}2)Y7h zorIK>HnYFH4%BM~PLrILmZQNkFg^|gMn(u#WrCk7s;KaAbGK}#q^5!zv}lrFLX{wc zgM+=jyBq7VD~ObWxUI^<(3 zQ(q19!0P!Fl`^h|H*WeqEDw@-W5)D~1=^2n@9xK|==RW0VhAi0TMIl>aIANcS5K{8 zcQ(jfZJAvhw*JIcz};He*AGc$riWjBunG?!A|nU(5I9Tn$CpK6Qji0HFhF&9;WBI6 z+h&P0m<6oy@`Ry}*P(mV5Bd0H-aYxJs3?>ZXPg|?o3FtFuvNR>oeM7Qr&icDH%C)m zUjF094?MK%z=$0owH#n&e$x}|>2p!*J}5ori! zs1_8FCJEeBV~J17$jB=$z60BgbgJnE?ShKmE<$}$098&wvw8XXmKGK?$%^|%z#yQB zJ!POix*yF8+L+y$#k9qM|KJ)-M8zW{^I{1RR+-T{sq$QskMv36^ls3P8!=H5paK+b za5l%8PDGxvTs>Ob<_VFVPfq3?`!kviwWj{jAW3~WHj7E)s;E?v2ze#B^9iR^s=1j< z&6&$L$Ma%8xB_p*j8UH^Z;4#EzcDQ}fIX89B%y&%E7{eiQ`@Z1iTWP-G@f@wa?{ds z5bnk>hl@0K$mUykUi@2~}fD8csfQr@4&6(=t7R03> zK&=`@Pf??n`&W}#s}#|4{+u{itx>>!!Pdn6#GbZAM|~ThC>uV7bSN$_Y*Q&$&Ctlm zucU-M%^5&k{V{M3FOK79EU14^Iy`eMZ!-qx`O?IE2<7AT&O z&E?t8!o|05AUACD>`wMlYb>4PoRI?I-HW46S0`4;SIcFer4co2q2f^2^D^ie_2PVS z_K@0Ez0t)uMV*Mk7mnqOg_NBsSfkc0BF7cd;;Dm9SLJaB{G!S1&BAxZi3yk4rmGH;`w+;R- z`>mzkQ9Wb|TYs$ffz?a}I!m^SDw97R1iMIaclSZn-j@W!_fWyju|dHL|9d%fQPe*L zt7a~tp~|8G$!B(RWvZ^<=gzAPTJ|-<-vrsMcvejzJD0Ax`L0^Y_&7lm8&(zN#hW50 zGv7O|4_p_oK2)uZ6bUGgv|Cx(nUY125<9GaVq76aoUEl+l{pUHk1|^3GI#jN+!L(+ z%%n#TlpgKpkB;PbZew0`SYOAb`)i0GwD3znJY(91h6n*)0v1+(tf;7n2bYeL7R$`f z%?&W&?gDrj8yh>6J7}p^NN8+q3~-npetGEwD%$t;^#LpXg|yR;>F|FAXbqOI$jDgv z+|Jbl_Z>@1OIoT(_4|ZLNlE``7U74>p#Fy_luJxZS&uQ67W2kBfDX_+Fi%kQx@+|S zyaBlUa@Z<?k@1mUnU;x-4bF}z z*ibX!CIv)JoP{j>6L|2V{&~txYIyTLd^n$$UGn5UCm&xRG$kbk{9jmjcr@rmgXPEZ@!;5)+^_M1pp^JL7$krA z0h2grBhwzQ3kkF?S zXQpg+go~*=%NqvO0lnj;OCO9=3vtBz?u`0`*B~>?=3A4p0B6U)M!i+2@Fykj&Pu zWZTr7H0>o2Opdc+p#gnJ@m`{3*8F50dd0pnWt{Sc%dTeM+3S2D)P2=cnKoTCOkG76 z@wGLzt3)j0kclH-m;t)NfK~QZ0%CMMm%`0whsADcqYMV}FOx{|&;&(c`1;_~UaOAQpeK4hn{8X7dG)GJ=8Ej1LXE?PwtuXO z<6y-zb>(xuRjTRkgiL+J;jyN`;U-UK9!uE0z;UsRTw(HG?D{vSCXW9d1^_0zt~Qvz zdX*yePQu!nb;i@Fh9W@fO+*Clr*XCci#TAAjEsz(X63+OE1u^*Fa9<()CL^0;(0jm z0q$gJloQVY%zhbDz?#?t2s3yj{&_LCm>9CN6H`=F6vT`cPH1L74=1ODP }G7lG5 z^OrBzLWv$^sTBfekjqsA_dvM>1!3%o*}(Hb2nSBK=8_@@l>%g-sVNi08@lL!K&!mG z(uEpX6&3Ndy6m#t$vI_Z(SSb;TP-;WW)0j24!d;VQE5M`3=EzqRJBm!e!M&sf(4w0 z7*%I6OBr$is1=lp8oV#5sYwRD2R!Nf=4OsEkUvVgOe+WUmY2r}#$KZv@#f9-SFc_L z%VY!KAwavnwY3FkN?=e>mkBqJ_}dyUP!u4I;0;)`!N8&dDF`iNf(-$ZQ&*(H0!rP3 z?OF*uv|c0y8Kh&Wa|z0Z1tkm%thmd#}>J7kh8L`q^!`Jn@hI_psf0*nn>ZY^x8b^nJ+5h z$7m|JVS|kYZEoUcmS&NsqBHfE_i`TnQhYUs8D`k!Y3(rSa3dtO?1;1U4Z?2z%&paW)|U&xZg=fff$d})lbEMy{zdB!o;)UE0(%xw zq^u>Qw_~4(WBemj!GSkD~ieY!@aShKkA0a3)dP^zbWV@AvVaoKwCu!+&v9_2r4G88Fm z4V8z)fG`0CQ3<)Z$yUeq{By7d%j0w?Sc6J|^nu^Vh^8Nn1O&tZU@ngy-2|aW4~h@M z6bKQVIMHoVC}Rf9%goFSz&$We;J(04!Ogj9?@9^_;VUa<>-EPtxVT`AVPR_vO4H(@ z*-Y2Q1OEZdEiNtsM2d*Xs!H$*&wk!hy7=lD9qobpv zZi|;k6-3R_EaDOnkOTe;2>36WS*%mV&BY}N7EiXewqX$wK-kV@C*Zok5y6n%MIxLd z+(ncD2%ucJ&FdF$G;%W}0o&|SVEM_#$Jg=W2d^OsXywgu25?G08k%@{RpLO}0Sj>U zAJ8NKlYf0{6B}^;e;2*;4Xe){I`o?E7!9u>_LuFaTSN`cWT`w(q-Q)%mp^o)%h8cx zX-y7>9B0(ZJqqj?7Xco>Fl3go+3z%DT?@NKwUa#PG*)IpZ?H7}Mue$YzXku!;?&uk z=7_me##D?+YXC8~E8?jQDalH;iFkilzJR$UKmJZMv78(gha?@$)e$2cKk^u_BPE!r zCro9TSt)V+o4CZz05Lu`<&5Wg%$nIcH1as3AsP*$Pqm{RMJ$ZT5qTU%?HgziR_VO{ z4R-;#DC#_c>w8P-a`rnpi-VU!$SoKqtw;TFZ@z$FOG2|@gg`~m7oe0{xmOI<&-1%P z>(X1g^yH*o10!=MT4C^y649qxO&(3aurtb;>RWW1mNV$YeJE3Pq)ujI{LS-|{y%mJ z9Cj_9{b?OIl*25K2Fu-NUnfTSFYV3Tl)6z%T>e?m?>IvcLbFS?qad<3I7!l3YP6`* z*uyjQl;meSXS{BR-#_mJ+gHM?_NTSdLOF!r@czvZHotI(3#>+=rW6ifUPa|@-N6j7 zAQXtgrEjcK09%AY49&~}0VA52m_X%Hz^_n?2>|O2(b8ZA0-_gKKY;J+*43qIumphO zCU6sOICMRvosNfx2P{cImCC}RVJkqKz~&7wNOuvGw}A*FpX&z&@cpZ%l`4B8b7Bu* za_dl4;?Q;nq$*21U{1i4QNRA>kdk z3mmJpwH5d>Cdv^&0w6h=G|ky&wlTf-d9lnd(5FG&M-qsGgY#r`aPScx8mgP!8Nlx3 zj8MrK*rqZB$PUOK;KfQxT*AT%Lg{%R!3D-dPfrh|G%6zjAq!QF415Mm&fn~d3?xQN zuHKq5(ZEqahx#Y&)ozXbL4}U;N|Qk?5!?U*afZ6a>;_nxWvjNh<1o?sFlXdU-l+de7Ayp03#J6Ux<*NC(!7s{ zlMJ-Q-Bjv3PeS<`PC=UgUNos zOqZLLO^c6#RTdjO@bwx2&@l?^g9Bhj#IWGgWpf!g!!z(81B}r`*?g6OqNXt(7sC`T z2;g$D9o#5q6yXn*`1S9K954tLF3~$>(ma&u>AWA+%~y>^9=d#sM#hM2+?hHn9o;?T zuw8%~Q1q}W*G3&61=G;}+y0_aNeL7$poOFI6R254b&v?sqkx%{l90Rr1_sCu;OqZt z96;{BCW{~1ZeefFl&ttE6eQSwK^xK&xGN_P;Om=U^AyGyizX>yY)lW>gnTYwGhYN{ zdC4CCvM8{`0ITq8uuU+l|9%#%kqcr$LJeRx$Ul&hkr_ET#eg3sCMV(2bbw!gbuo|t zfc_tx6<7t#?8o6Dm#Ap?G1V_*oo`^mc?6834(zK!;Ak%aMgoH2>nje-U5hOcB_~Ku zu}S|+R(3=|mu=f%oNF5!*}Ci?aq(KIP?pIsbv;EKXl&>5GeQK8pXgH+IP>gj; zl)N4*O!7=H&p2Y!YluUm;gU7d9~g2{7O8pjz)h^PeDN{b(7y4M^XOJiVn)AeD8P_MT@xjq5j9rZOvTI6T(xAA$|u?E$)aflBLN~Fh(|bz@r6a;>rtze zRRd-F!ymSsHseni%Az=@k&d#I?UmujyFVP>q~Er=SbKnjZY}6KOef`vcg1DIRQ69s zwsJPtG4HWxZL*Wn&^-iyxvdQZPT5w9Ckz*iUU}>yEgzSMJybvd;7TO&n(C^SPm2dy zjpylje`cXGaa}G8F)js*}so zV3|(?x#Le>r%qrBkb>)go>e@*vm*);Yru940D^(_1i&6JeR1*(fCU12h{|gKgOH|c z29pgU6Nr<}5g=C$32g_AQBhf0tx%)jU?#R_8Kf-$|4MTP6AO%O*@+Ye)>iwcA}?cr zsmXySXy3Hr3WtGk>l|?bkZ%ZPW5DF`wUAh)Q?jzAe6AYzP6Ow94}peg_}QRuG} zAxKVuEfPIFJ&?KJ0;CVbLv{!pWPU(KWAiR5V%CAJ78*)4vL^Sw0u?5&+n8 zPhzYb6oh8taG7ZT=giJ-R`+&n@6CkzGSPZ~HvH=mH)B9}b5AsBhp#$0&N7`&3}-HG+oV~sDCyX6 zSx|syYsSgpl;B2BqBvb2h_yQPd@YA=SKgq0SCO(V5?1PiF}ySNg}>=@e1}EaghssF zDwgiiO01k`q&WvDb@B!5pZK9`reCtm;9o|Oi(PqaTSZ>?2+3tfo)u$%w+)|bKq&M^ zM*;fA2#^TH-k!bcg4gez+j=cB%`S9@TQ0oIuBm{>Go>l0jL?_8jRLw7gcxCfy|)2r z;*AJp=QFp)wv&!$yE@O({KjA)=&i{0M(@rHe%$1paYA0>pWb^~;`K4ygoGhoyEi(Y zLZOnYd)*8=wo=ThR_a&N%&-2bD9S>|n;z)zaG9*xfw|APHFC zBLzVoaLt8ml>#XUSleW!Z8lipSnvXRE`+)tDF_H62|2m(#z?Vrwkoi75d6SK4iwVt zslu0<`{e%D(NS48BGg79biN^9i;Y`Q5Dusvcy#vb*cj;k7yANs0;W(q7^Fy0Fb$ye zm}->*6-d8*YXLi?SimslP)P{DNPqtP`3r9WQRL<42X2!DwvWoofssmq4ogcT!9BIL zwcvzi4VLz?F%ppL1hfK#=U>@xfYxC13AHbw2gVZ~9*){lnw?c42^3gGnZNrEGvRGi z))UMduznP`1sXsKf?h$E51?wAGioa{s!gG9+GS8_;-leBu=mM<+BB}TowTsBT6k6< zWp~E8SpK*TcL2S%cEU3Falt`5#YJnZNp0c)p{a5OZ{O}$-f+sToLJTM_orNeGEQ;t zH}Udw%hMH&DG<*4ix)U6kA#FLEW>Wi^@N4v#1`dg%?wKUr|%o@cw3ltzDI61i==XT&?J+Bk)%*m>!F)+#TO|o+4 zpI*D~`Lu_p`ekrE@yo&ys#+c_*S+Bz*>AJ`lW{*OIZ|M&SG0|Fx>^;w3Z=$wJ}vIt z-NhU{<$l@a!b(|8+H%dR9Tq5?2Ycxg&pnRPs%t|h$ep1&qwf&*)>W&K zcGRt&TR~RZ-k?l$>)CQq`9UPKqZ%o7TgBuEx*c&>OgFQg5bGqdL(1l^e)UG$gz73~ znIe%$?l;!j*`ILl@gyhbeDnPJJfH9PeV+IGeV+5atta0-_wIp29pZQ;wtIB?`fKN! z!>?x0_4|FixISkV7j|R*5NAt;ppX6A;Hl(%!_BQo_LnxmDmh&MQ?+>-g{@(@PfYmi zu6auRE-WV8!x!SxuYxB6dt~6l`@3YsZQLWuT28hlv>k`LO7LbHdRD=cfuI^V-AQf7 z!D;y^%-j$x>{QKkB2o?3c4psTAVjj??}`og7y$o$wIRLpd-IG8n?BLIs$)wahvZgj z?$qC~NBx#dY?| z{WRJS`vM+4Vod+*aYNHK*|)E5|4ZYaE`R-J*PG}1N54LIy2pA!EYxhT{d%9~MXu$m zA4Od~Y+pvD-Ca#K1?H!0mlU7&dSU56mbAQ^qF6I5jr7)HJqw>!C3U3eKzeILb)zyO zXSh=sVKU_6B8=J$+_b20erY5!6g5k8?j{_XL0dmDvZRHF0;t;~pTP42Qu}YY~kb0E(Wvsg%9OInz;hhBTM9!StC)yJ&1C&)uvc zHd{Y2%Qh~sH9HgrU*p|A-8+h1PS29&>cg%w?U7ZfLO;-_tdG=JnG6M{exU?s+`~2{ z8H*R0@N&oBlG68wvrbMGe4T{-h;M9ujx~FVBs@!w5~to?$Zr@>T-)=!ytgqwd_)EG zQInz!Wz_Md@P@fgq4BjIm*>4ZvEQPO-|MD;9(k(yM+^_)>!L;Yogc^~tK0hm$kKB8 za$cvfVGeZDx3?oY>RA$}^5te#n3>2!(G+#Nc2q|16tdhms&q>>gn#z_-cw{9HOWvU zUsio7G=4QndQev8`5{@xeKv@)GVQ4}5IyfYIYs^lhzH;dk4>1GkQJ~M6hOx0MTs3& zpHSHMLPeSv_-x}hbByPpqQv7rEQLqt)oA2uZsc4G>v4-g0NU_exEdFs8R|0(vGm>S zd!Fk*dNP(DSIaS*?Wewknt6)Q&Ain2^l0CVsxorrMt*>|F`f0mm34c>AREy>{dZh% zp?++EvkAH5Qk_OK;HlT~O$>!r8fth4m}O`4nZI6wGeVoFq71K4e;1;}d0WC32T@yg z0GGj*9c*!6o(;F`V2gt-J1{e_Wd~avY?#6S@tj!i^J9R8#ZJE?Q3r`z2EU>0tp)IZ zY2php8Bz^H>y!NT>+;MRtT{5EC##ogv1mqb_x$UqiZRAr-F}v=zNM!JqcS&7dw7uw zh&iQMt`|a35hZvcvy_BhyxpF6$A0J}1W4|13q8<&u6#Ki{O;h0WK<#lGnWD}arh^+VaYMNd)bSedw&epmRFbfPC4Zqzn|8Ug7FDl}a_8g+ zI#POX38{6O>7ekD{x|xeM7Zdd@S!_S>TuGYTKHZm;GsYy2uH0uO?&bNN}iBtNAFLD zpwhJ7EH-@!?Kd6*#^uc%j|_@=|X;|8(Y*h?4CMF!im=JBpD}h>(@w& zZFQZH8!?ntniXshJTX6)X-5O4JIdj2!=>yt5*Y|sds^vfdO$PPxC*CFrJnHQLGRViCPSd&g$e=oyL(JiEoEAP6O10m&-G%}C{#mV+k`bzL70}fJqRP?1V!VS&f zUxY#Oup?jiOKYGZ0Fol((neHw!>A|D+V|g=FVVAssJRoVlZCuoe<_~=${E)l6rRl{ z2aV+iM@*2=3{Y8zKMbI)Ob{s4qa#@ujYZ|2_29N%$aIlTtUP6z#TjYZ7!I)@fA9-7G> z=YKU;Znlh%vE`cdn;{UK`EB4uro_mM-)|iYQOkeF;Qp$o+i2GDSAIYYc zKzisoagddptv~7VD*{u3yP)hS#$V*tGewl z)tpL;s*^;Kt=J{Y-HTxBOx=@y&92&RhLU6LO+=y)kIE+R24ys3SD8f>{O=5fkChDW zt*tt5H|uk#(V`d+SuY^cre+sRb)uzJw(^5(iA}q_D3-q-4@P+$zTdCMvylqa(Tl*d zu&hpu54Yi1#tH?zqImDA*=Q%70PlG4%r?d@%d81f1-^(G-%5|4$|RTh5p)xzVL#+l zkFzD3(8%2BXy;%If*@fjRdB%;HT#Eqk)Joe%k8rPQjIK^KdK}34YL2s~~zubz(8TM||(?OD`cd zwpKQSMuPspM|6wiWNZ;yX(uSPR^N15ORLbwX(?;L(v3=i-2wzeQ)# z!VrvKOoKD0l2g~zjJHv$rFBl#^4S~dWIqBcu_>X*i%@*1Xz(*ul*qH1HC?eQ;S8t_MoTUZc0iGe*pgUDd>96OJGx}BmC)*V ziJ%ks5mFE$@tC9%83wQ=v6^N=V-7+twX?mF*px$jKkNfu-Gdk>Ys3@NatMww3yAJ9 zwp3SZC+A|09cJdpRJ0HN;6?eYpR=O%`O~)xA0q z&|Rxhh9Z>Cmt$<25A7&94<=Piudh<>#j`%`QA<^t#^ytey@W73VKxlv7axx6?2%)V zRswKqCn+DCBvlN?`vFS$)>g^2z6ixKA=});DPS85CHEsSx_11hE?3wIz$BB%t7}pc zW%d}b6h}!MLjDAA)7+?(**ki9Vt`FatJd5YFiH>i_$h(el7l#N6asZ1WW{^g62dTp zM>Q{l+3t>uwfX2pI~#Zz;asK3P;WBoeA(;Et~Lr$p^E**6&FNJpZNMYSJF GJO5vmF!2Kb diff --git a/docs/images/quick_5.png b/docs/images/quick_5.png index bf7746440ca7d9ed6729f61cd16e775664a71f88..18f155815adfe2a0ccb8077cbde1885e8060e248 100644 GIT binary patch literal 106919 zcmeFa2UwHc(kS{OU8G8p8ahao-U$|(hzLqA5fN$9r3V5ky$J{?C?LHfU23E&2uK%@ zk{}>W5=saVl9TV-|K9)p_W$2|{&Vhq?m724E?IHb%CO#oxrfB>|#16>{gfT1BE1^@sRKoac&kP#sw3cx|k0B|nv+}~>`mGemd4wEMR z@g-pgP`=~r`NY%L+0%Af6CtZ_Ashe$`|5Q`wz7L960~b(bs|a z6^@u6Ie7UzAqwIl5f%t|;`Ij{Lxh=qh(rbE)H?(iv;x(03kWs$zNBRHN^u#fc$Oiyi_w)+# zadC2f!h4k{B}ZO;4+kj;UfCVY#{cnHDl7yA&-|M_s0YHUF7-_-3*F7L= zN;6Sz?9~5WCs+ai^cDcn6z*m3WB<4Ih@T`FGT%pXvVWFIXqo|lJcB^kHwOU9%K&gT zLm-?s69{L;#P;U^;3J0c5n!YuO`=pLCE*9oF_Mrnk`Q_U5V0L{l0Vbm8zwnNN=8mW zNkx60hKSI}0GuNsB|S$*N>2WVGLnQ6zX!+|$(i_Mu2Nhywx{IxW|oaec}FF1t+tEB zWDF%J_rT{V^?6pdOY9s%!Xl!V#pD$fm6TOfuiwzt(bdzxdH0^FnK_X?hlh@joSa=; zeV_RG2LuKMKYJb-6&({Bmzwr6JtOng>#Y2O!lL4m(z5cp`i91)<_|5c-95d1{R5vq z4~|bvPEF5zo1H@-SAMLnt^eHEMDOh%9Ab{J$0vXAMFNoijVuxSq0@0Lg+tO z`;)W(8e>oYr#SluWB<_NQiBak&#gne;8<}X&C;86Jdckcs(J^0T)O~h{{CD2!H|H_53*TZTjT% zqSehGJE_r>y`C%^i!`W)%bI27di)WB>3k*wQ$HMykEi`y``_5CZD&+vQmN?ci0!;d zXE`aVApl)+)#{+-ScP3S?6ym5QBmc@_wtRjq2hjYVwcqc1EWlr{~e|KB#@k?i<&Hf z6^Pq)S-P@q(Lbor{Nx%sb`N=!hfEA@hMZ=w6ZeQKGGIR;%^g z2INPGat>F&Ngi4@`a^@e{7x8Vel4hx*%)#=&|aJQ4#|HtLu%nDy$vf32vX~A;?lPk zoaQ_v$@KIh0U%2u053*+Jx`y(c2<2>z|Ti2+0e#k^e%eq?svxk`vjYZ~}rXu$Cj!dqSJuU^POa~7`+aorX>r}s$?(5u?nLaxSoyToY zXXm>0sDp2|V!p-C=iAv5fa(OyusQPiB392ws_@p; z#$?c^B_vtbtTy*?erp6#Ss18}G?~?Pbf&G>4~fmk7N?5ZzGb|lOBPej+#;w$_k|H+ z=Z75#!X^mwq~zSeD$ZY!+4E?oI>w ziznOHFtzhq5$=*zKY#|kS0uBj3#qNqK9mc0%E8IN}!H3bzr_=Q!ReznMqU9&SPB9H{Z!o4QoAp7)X4Iye&k z9{v>PfXYVf_JLv`Z1+*(P@T!@C+?cv?i_BPZx^t9Z%XO5QkgivCM+#bF8Hv#R;}Ln zu+-Hz)Fjxt?07;uEFGsz0G|4N4@~aUltel8(#PDv23Vm+;H=>ps*lpWYiviHZe)mW z@HVF^HPxjEnpv)PCn$$T1|`6{G%BG2K`Wd~Nf4HXT_Dh@Cx<_}$IL2wW8dzr4kplK zO_tA+fA8++6w6m_Wc-G`5{|>6k2q~0ra2g}qG-)G@CAG?m~0n+&W)8v0vnOl>Ly?v^>y;{jKD?r; zT`Yq8!IE}!=fr`*@B`!ta&wO6!?)RaMJ&IC#{*ftkxvO0tb7LdDs2=dJ*hP$k_R`% zH#>AOq3u}1Q#)U75@h9YWl%f+4sEB5LtGeWx9K*aw%D7KG>~O zDX1VesB+l7By7~`VxX#_k4Uce}^%O_ZRAbo-7hIEx<>1QjstzvVqextwOThU+ zOMS4wF)|w~ic-^wGlIt@LMzKApK>v}gtDL?voQ2ZbbGkccYE_n(zs1DrRvX1ZR>9tz@uE~sQY^+~s zZ*_0u4T-H@yw7V&|Dx!jLklH&_R!+l8>j$^tp`@imI!wCj04e1cJgj)$C-n~RvjwAhD~S}r&rsY-5Si5BEwx_<1xbH2_t z&!=LudX>yvAEk?@CIG1Xfq`kCt;Nf=*tBIME-a1Jbc5|}Yq!M9?jGCK5)S)6p5Gc_ z6>Ozkzwcusd-=}Iz)$=cbdPan`EY|u0Usw4}+}v*d0~_lzSeVaP3<)Zz~`|zO_>#YocJ64q!}yH2qdTa(Hs>+Uqm* zZB_B2g>MFg+V|9CQkAm@heTejGTT2iS2}zZAa~Mm3d)%@-%{AcF>U19p=fKO@q(ZI z3IgZNJI8WKz;Urf420PxgjsFkY|P@@%k zCS1^jyE@zQhU7tqfkpG!FR9)z-qgEXCr|v*hHHAfcP};!y@}OarurciSF*H!86hT> zn{u3@9lA>Z$eFPksNLnv{tn7+@P&gA)8fyc#S#Y}Jp%B_38{3}1;^~cxhpUtcxnLxz?U1dbVNY_<_8DZ zaf(mw!{U;E`Wg^`0Tq_XTH=m4GwOAh7@}?- zFNm}JDqA0vAey}}?~f$alr;#dy%BaI6eddmJ{o}VhSe1-A~;DfR!e~%dl&9V z0G_8MOYcDTN8jSj!i;a$XoC0L?Ij`Y8!(DQmZYMgbOzO4oauuv6o>}tBhpGEbag<_%omQ_rgxuHtwZte$*7G#q zLcmbbQ+k}BIp`7bsS^RS5HKp`tYG`qVy^GVF?8vcZ*xFLD5(}Xv+@5y*?iq8K1>W- zvb~gsSgTLZ+fx0$lhi!=KeAR&DfQY_^-@Oji1)d{z}dxoxQYU7jvgu9 z4XMY8O>_=bY-uu#RY^65ymjB*m1?A!fAxIV(*q@rc>hiJb<=?zFdUqiJOGbdRK~^^ zV&xEv5ye+2-sM?r&Cpkc#6gg`@si7- z!81BvS5Dpg-i=r1tSwn2DK`h*6~CFk2$KW06V};zbHqnAIiW6So^;gy#~d20HDX6~ zsn3vVb7F=V6%W^q_${3M)lZLFa~akMAP1r5Uq zjR)SwFRZ?_d_8tsDC+8dIqTL36@Z6MJ6Pmtm>kY(=1DT^hLR>j-4xDgdB=`c6FFnl zt~i}`RgpfID^o?l^_Q}G%Cm@1dGr;|l{HMBRa_NPrT7|YaB5L4y(Txj%HMl!Jv@&|&(zfTjaWn<5b~B~ ze`$x$6RE7pvlJMujpqIpmu1o?lJ=mRGRN#g?7h#*XUp0qvp2*)oiuDX_)5N+NxLr3pv#y;p?{4x{cXC9hnnvTQPL&R8fNU}@=bHr zPD#`-=AB{%TcpC)TdC>I)i3oOtvzHvQwNvilNn$2+tz(yW)-S)XC-*qZ``n6MLUkM8mJ0l ziFFiFMv@a)Bf{5x?XFOPiL6q~C!ugEKsa2J0OWTv&A?r9WAU`8+`daV)BFw6J{aBF z_!UWxAPJFZvlRbOjDCQi`{z%ew`Jmzt*~dOH-66LCN$6_UkLQ`Y!@glhRelgXT-R) z(KPVhm(Nrtqd2}O_MNL*OB0L-g9mJ5iB2|Nc&}}qlsqecM*zN5OL?2Q=!Tyq9TJ%YckKiBj>PFrnX!>3}HOleV`%cbG?PmBy$qehJOJ1AH!zkTY&rdDR z)N|K9`Df~W(;~i+*8I*OtdU5Je8+X9e_CiI1e;{bX{y~EGE0`4dg9)&%6LzVzagnt z(6wCdV~s3U%@+xx+Qd^&bP8{1^4CuJZ3$d3Z^rYrPV(m9#T_IzG}69|b$=<{1<@ur zta<>Hb5G1imgsOn=#YWwEc3*?3Hetz5hO^ZDxX#w6IsX!+1?-d(tSCgt{VGQT=f2R zMXFNRqb(To_n%>WXx;dlcazeM?heM+1s|;lY@0W;o}L?)k4dTiej}xv{1(OcB?TZ9 zqz%4JfVTtrV2HhWw^cmq4p`I*ZE4J^xYj0 zx%%B|H)Fevti~^yUv(XdT+QllUmxZQe%tb5USBM7--`9x_4EtIGyc{RS{l=FSopn6 zQ#9yVrwG(6ySFvtZB?KS0kDcT)RPvD8yV4H^NLfEWO*(ZbgDGys4D>i!r8dN{Zs`V zD9Pyf#~z0vFjj22nmDc4$b{vjL=}E1_?i6`>yg+d1}jTCSz}J(n<&d~AL?aO&FnFH zmi2b0adNKz9m4&5kO^`pq2AWe@$nbI^jM)Romj`ac@)>!QM9p3h-jA*jNiBURTgVup1+p50Fkr(@`!p}1S#o2;gq%AC zo=)6D3r#fy3?n&f>Th^T|F9-^moQw7e^A8*t)?`|_A;7URQXznW_>yrH2q7S#xzyM z2?M+d|$N6E}Zx3hA>C-T=G&9~l|hnG;@6@T6;2?T3!h7NT@i?>IC@>d^xc2Vsmgx^c9F^?}GvVT>gf#fKXB^D{Eba(0?=wLzMUYbrz?tM0|{!{}F zPjZaJyEI>6Q_J+HshlqT^l>=$<{5PZl*W_!WClnTt~q!rZ}Lax4qNGq&44Qf)WQpy z7L#JBbl)$~0eSTAoNLlF**7`9z!;&H2xlq3!;d{99-Ugnqk_=>h0ud`q1gZf4246Q*?+~ zM`mDObTJ6<$fGa+Sre~rx>9;bTvp>`G#NNnccPlTpJb{^c5I`!%)PC&1@DS4z9vns z8%>sP{|SnK=jWU*7ndPJL%PO7OM02UeRo;pYl{le&^DafRxGtFn0S=+pz;ybC%l&| z(}y1A<5JQMD{-Xw+4Vl2OK6?i6$t2OjsORlrhv2Ai8O=F{Ct<|5jAaPe6j8oeaX6e zExb&Za0z)cl{oumGolj(ifk5 z(M7dmq;~ac^}KE4bVLaAk;pb?+$!3+%1qlap7z{(j~gv2)onH?+5(*CSi7PE0btZE z-po=gD>Whu{U{XWw&i6eNg^oSqu@h@saofN{h8G#mUF&^8rAu#BB}zAGyA! z9`Ko20x)xzINvUYOO0uV1#J=l7O3SiY>sF;klRun{)}RlE^-O*-?!0omnfXve*VHn z;>NLevMQI_vP014wQ%rt(V8!?XgCP!;J5;hg|VpL^$iI?V7`K^#^qzB1R08I+gPVx z#m4GonQB?4`$kKC7Bnw{s-Be*x86s=iBa{4EsaOM?`+&yNwwoJ;H8g|eb1L%lh@Lt z(b_G(-BF_eD8LxP2vs;M#yPIgQufZh$*|I_FLU9zk@Zsb_N${in=;>Z!>7~czgHj4 z-R7{kYO^dVlu-@6a#jSrz5=2R%(j@UR##9MNiK=+jpJU?RT92zM6)w27 zp8c>2%Y7K7U-J5?E(hIe@(&Fo(yYmvK@e7SH2KhR8mENnpEHZUeJX+}>b}ftD441) zLvhG>d|T`AG3F9*LV1t4cMXS|~lvzB8N0RD|4M0C@>#sK^iipr#~@3ul57 z=}TnT!jn!pOjf;!OxLzH(8k8_zR$ktIQBj6D5Do{!e&)vkgt5Qdhc%p**W(&?X|XvW8s1;5?AqLfbA5T$ zZe6osP7~oJ_p|xIK%q~037jv}zQ~6fbWt*vXEQ$sYVH?&xF4W;y!S|Ho_~<^e(J}| z-+(UTV{y9z1ULcC(9Z7K8kSb}T4Lpro)q(~7>$boVk$qWSVrS&5)!z)8YOamo_Y4< zC=a~{Z68;$DZ4#Nb%zw(o-vSyEEPeQKV=pC1{7d^) zr2ED0O#d*@w8@hXQ?GZbg!FMQpM08MyD4avaN`mw-)SD}$ZhczIM!t}4zxc$N&rX* zz{z2EUG_tDUR2Y@7@9Cdu)}-t9t}}JaG2t_-fJyxG^Hdjnb3@UYJ8w_r3{h%`%mf9a z=;=j)=_;IYRbU*LcA!Sx+=?5Q71=juAeb_rEK6OO<`s*iZYb?}+=T<@w>mz1)|j5e zl91CJ0JeY<$4PRnwxoHh%mE&XmQHvl+6X6z(l2nUcP>8_NV@cR7y8P!+CENsO3%0H zPARvhO-=}eNOB}r9I3G$|70lvOqq{{L@Epi&-AwXzdD(G|G8w7b=r8gt^qwCp)MQq z`J01bzAl-5O7hT&SO!zh2F;eAVhe@ zHI*J!k0rNlHJ4nNjU?n~?mdXO>#;VnABTILpWSoyfT_>WPcs=^vU9OCa0!#ye3swH z&2?istMhG{b}>zhBMn8?SkRr&;`_f01hwQNy+347Q7}WWHK)1+pnK%3j1!YiG~WcA zD*2Zbxq@dQZu{^b?A+~PkU8i}0$};WDIk6C*UxY3YaC)zqWCC}%C_U&vP`0t(PXp< zo8-v=>u#A{58RyT&w*e%y)o`$+;ZC?o&B0e>1)RyR@D5tJk%+l!#$0#+EwaqZUlfz zX%2?Cf{k@q`Az^_wok#cexeHWwGP`=@PL%n<0UC2kG&Da+)*(xR#xsr9F zANqZGGpyJJg1SpHZxp8(#t)`PjN@nvun)Gn=ggiJK^hBB+&fn0+nX7dB&^$a51c1i zlCH3rU%GEwI0IfTSY$_2z?a=RbP?`-{co3t#m&c4enIuUy(>UFkUAuWR&Tzjmo0_H zNByhiE4`1*%gKW$Vqq0VwGivF?`OJbIF32Xh#9iUFdxWtx7c9W)~L&DG^C<&B>#4? z{Zgo(|MZWMK|aU=1NDP#$06@@R000HkUMIoXORT8v)l7z#vOMNk?7guL#+T2l1{U1 zsr%q!8TJWpany&M&yc@M|=j0mk52(q31i2mv>JE$oCo`!|UI^vFYSTyI zSeDNo?Ax|XJen%%XzA>1ugx0pYR!=7=T8clPfzooJk5Immk}-4e7_B{BC_fWM%~kp z>fY^w&4OqTIV(IV0-x^MS}H)84NGkova2*tdVBYcoVO#w<~zJB#QGVe)dohu7hrBn z=YR6hU?WiV$bp_B^ypKlcU5zYS)YQ6{Kod~5Cw8g^!2QTpsOC0)`g%8W}O`QSv^-D zM<9!joF*ZS%{v019bL4@ZEOU6lzF%J#))=QYLiOuP$G9QN_XN8-u~^mUEcM|eIT zu#{M4L2w}=U(Mb^hcwf<2IAG#LXhsNYBby-cTn3YgMvyN-pv=UoF7_kp5uF~$n^tp z^%K_$@9_~%48jN-yMs1AJ1imqdz~Hmaaut~&vEqSSSRCD{As_{OiV$lz@_F%!<0yU zM^BlsQ8&`vOCN_Ap9YRl9dL-|os0WFD7S-=;O(fHl<%g?_^e+KWl@$=1;fkAWfF)|AEc9_xz-YljynV>I z(5Yliw+FD< zOavyjBrv#pe9Jfp3a%oKUA+qCY87TQ|&A+wO5L z!~FW-(;7^AgYKQHWLykB z;_C!}GHi&47F80{U>kPS%ds1?f*Qp=m1wjc(l6}Rt9Ib4cWE^rw?eUtGmYeJ4bF*Pz-T^;CtH z#=8gfI`v(ApTWHqJd(WnZqJKvkpAV^-52F(?CBH8Re1`osSCpM0|LCB$!)mvebGu6LpOji1>q%qvcf_^47_k?T& zh3t)24nwRqE2}jSVkeO6$*tdwz$ui%oUdx^eADEN?jB|7O= z(PjZfc9LxHFpQ~dAQnXW?06M1@$dDY^lQg$&Usa_i`v>!J$z(lZe~WA@-@k0eR(9&H_yt& zuxhi%q!u1}n;Wtt<~hB0!+Gx3t1m(~WsJlJ_b}-C2)izAAQ1Bw1?jcMdDq|+`u(**QlL1%1{zQlWtg;j)0T>Ww843o_#+42JCASydqHXzl<$XFYI zt|N0Jo|#NNY^{qom};15%Q;b_wWH46Ze~mD-0qQ=y?kY41^PlSq4HRz3ka?WZzo5< zqgy#Pc(^C34tA9uW}aIL8+gV4PVAvmXt?M-lL6Q8f(Y9VW)j3AAC?9IXK>AG*sa*~ zSJjOvNm@4>mn=?{@Lba>x6*pxa`($U)0f)()BtB&+E0$?r}BwLeJ9chHG;2yse{SC zD4HD`yK^p(n`hs*U3Au#DO(7y{CJ^O;oF!N*AxMW;+;HfoT`!b$eG{1`|WedYsznr zEN2b$1+{&~fi~0T`{Gvy6xKB_Vmb4$)@I`hl+%s2b~8Qtg24)j2V*iYmPXMTzIxD9c@3gU46U0d??>#E_SM_!8}LKo@L|37eP?leq3RK zt%K}KvmbO*wMgOu*vw7=Rb7vuweAfbv~|3Oo)m0JP{0gD3w1W2KK+VF35D1 zzr!OA*mDkJvarG4HAgwcHJ;t=J$Y2a<;?3GYYhd-@o+tA8zMR=Y;kxnk9{^*bd3~^xw)@t^ZUiPaodN@l`=X%=ys$ z_P!44tY?Af8P_}35t*?2fg*uw+Ke&Up6RndOxLk4DbK1l;DuL3xdnFif0>1FOPh^* zXwEo1xj3S?wznv3`9LDyqA*EUpW*#r^My*5=ya^~GG&y!?SrLzL{q}M$}Bf(Nul=! z>E8URt*n+se$?z*WBmaVk4c4TL)oZMV~vm1}79M9<_I#OAXHkWF?M&?p&bV}I@`CS%D zJ_X%ro_x0#mpEa~WDN0Xy0iY`N+{JIQ`6A&fxC2Xu*R>5s`;wL+|nMQj$Y}bhjFH2 zgWk_RX|&V`ebuDgIMJyo*i>zfZOmny!+JMwWLUlY`q|S>xIgjX*>{=t8V;FzE|!Dq z>dgu}lXoa1tni~*5Zc}E!G$`Liu-Q+5 zQNxuLdp(#a3r$Kfeb%tZUM@~(Y#*ZMgNq*}IY zMD&G(64hH4Ke|M>KG~@DdHL;*pCJW^PT#RZyUyb3k^1{`K}!k>PS_T_p zd}DJO!3XLWb?O9bR}qnR!go0$A0oWlTpp*b5%;+;Cu!DMs7^iFJh5R`;>lYjoe$$y zG-vjW74fyodl)`yj_a;5Ul@UfO_k#?EO9DkLmX|N5}l)>t=_&}%SLJhk>mZi|s|Y5+dVpyADqK0{dIal2EAD+bHJk{lhR_PL z@%cusMA#-W1Lo@`I}0cQ2pXSO4An!07A$RYkV$(6xic(CWJESs^cQ5v$hZ&6>OD?w zbvuxX1MmUF)8W%TJG!waRZ>egC#oC4v_juiHA{S%i;r#_EG)VXG?~U?>g9CT`cK)D zl$S%2w~bJoy_&Mv$YR{BsmjIq9y_kg9mZi^F{$bm0&p+oA+!rbx2??Szb=wI<8FjV z?y+?1dDupzp!{ik|=R$xCykzc82l z7kmr4-@^EZ@O}&9pET*WFn$Z;?*roRG2@?Rhu^~ZEezsJ{CjHsr)9}+Vf+@x?-l6( z9?NksVdDS!HqDcGAa4rzdB#{0fKo7i(fmhb*aujb82E=Ing+j208WE!h%ra?738>k zVK|rClG%x*se;v#BI60z%7SKk0YrARn4MVWu?oKXvyfE1;hK0Fe<4bx;HVN zs}0BzD~Rgn=mXH>=T}q=cKhaB zaCT=9_LC_WPXgMbKg=Y0{N=qSQsF}YM+M9?aF$l{Z>Pz$^TLiDQH68v0zqP=BA^I-n zz)A@K2Rt7(hmBw0{#N2K_p|&sQJ|vdR)+w;X6FjdH8J2N$!FLj2y7HQ2}=b1eOR$|XZ~IN zTPD91;?D`?cW?O5M~Gg-3-7_1VmlsO(Nx#esc zLa$$+&fAmbjh$BMyxw`4KPA0epELZbSPwfHgIh`SR2YTzDE9zPU=D}BOR={)C3(xU zVcjjV&P*(r>x%Nvt81O4DxYm1Yg)Bw7wHr&zk41#Z!*^4v(^QDz4JWUw^Hd&A-aM`cCe0I zgf!)=xdY0`V63*HOE}d`VsyH1$)5$Kys<)z(1wM9R4jGb7C0S87&%|c- zazw`+ZtXI<7KKg0E~j*h-m^im+AV_=RVM{{q#1Q;%YljFOV7D1x!&B8JlxdM z{OW<;qhtZ7hiXDnFN2oCFNZ72<*%Klj%&=*MXIW&TGEef<|>ZAqfsBesLRp2yWL%L zcL@;*Q44b3`4Q3-5kmC$hzkSIJoRYGTMh3uncK&O@L$K>d28Ry*)7nxAiSBVmL9|8 zEw5d*D`XPeN#34!7jAn{zGJ92&}9WW9= zoA+h!z z%^ugE6hr-)EW*aA$0)W_0Bc%3iUiRLwP&)i?miy!iPFQ=i$~3|hdq2^b~lIWr|T6J zosjb;WhRa5EIjuX(x7`jCR4kdg1>fP{I#!hjKor&U$&y{e3VhYIe*$6Jp6GyI@^7DqTxM?wQRv0(Zjl#(Abe;&A+zS!Rc1W7yX0X3j!Dl3&_j zYPyxu+v%)X@haBSkOi|u&xz?B0^n42PLjSSN1(H_Eo1bQXd5s_6Wvcb-n^I{2ytf) zd4VX@N0KRSzgbRt_M;qA1zDL%I;;S*f3rP%`vBdWv1j{S7b39z?R*9S01sBim3B3+ z-n(#A!%|a?cq8RtXt*ZV=HJ-q6WZkC1zC6wxx40z)1mO7A>|YbS^5PDj?1jfBKrJv z79KDEv84LH%Kx%KT;(`v!w_9>UNjM}$nt=nn!#n!$$0&C^RpHLP{JP;fe64Qdl3N4 z5k5uWVBL^-JI@p1Re??}f7>Y5EKkE8aL$R~nuvBCVtD^zCHFSVh?)h)?`GvPV6w%7DGqR3?R27ZuAl)(k=g+Rv|PY--KIZb;;UAS10LB$OG5A zAaiOxAV!>$QxI}k_a^FBN11DIdSp)4QElasxMP4&U4!x9kU-|%-nD0=%p?a@EC@Sl zY?o4Orx?x*MOWrVkz0aOPSd9jd?+wtcx`CMN4@QRsernZ@_=xMba$sJN;j#hq#dOj z^JJG{eh!`~k>&SY$%T^u_|382uNt^M4EWtM|CHO1xkHOKUuH=vLJlh-tO85Ava}m6 z+H5Y(f3>L7mwze%nzf>KbgvxRzp$>MtT2z2ThWbiKj7>Lz(k`k*{CR?4EcJcMZWsB zxE))cFG4iu-`$O9`0?f|<=27uCpk zlde*x*x9bsBr;+H`oS~}Ai4p!$_+T90%wI>5ZC;;@^WKPZ;GrVL`dGJfd(%m9z*S$U_BJ`%=d1?9t<+A3yATMKxqB97T+e-kL+v%69l=MSF z<@)W|a(nF@zdF}Q&RyoeN|wSyN`EDK{i*V&8pxQYAXJ^`gZsx^L>E|abIQ+93GCX? z(D5`RKD2~#I7vz3l?bKRr98UvIFwN&xFMj&D84yIst$!IbLYffH)*ZS4qa$FMp;n} zL9}b<&sY*7ZbTKMU`H!$;@%C!Ew!y#ARx^S*YwWx%9d$?mVi}*OD8tTB-#co?l)8>{;lZl-}3-{i5OmeMlsq-Ue z-wxyY@LmTMcA^3|n6pUt)H=s4&B15INbl`8@t9BoBMC=s5op?B@_y2##6eu7o!K=9e>28f(m<|&JbK;y7DK0jFiOdWPRSJp?dpLxfy6>g-A56M%d$oXf z7g{z88N2dm2gL{Qnoy#9VI(&yz8A&8q7o^|1dADVR%=R}YJei{F$?#~I8lrW9>Su;F}h2*lreyS0G)TSkz zYw}-<@FqA?#Sy%}S+W4a-@+RZFLN|}Mf7d`6uL(Mei)6bitqd4+1C4VPw0r{DxzVR zqu@W31v2gu0L}8yF(OJ2o==hJUkc_s8X^EGV7>n?1cr{X8iXX|6i!;|FY0Uwf##ctC^(73YR3e zTdyQq4-oy&mqMGjA+updzw(nkUt_A4Hm7{w7E}j&L)eg@xE6oMw3)6uB?^~)<~9|_ zjY9<&hiVU@^oU-dzn=~Mn}3DnoF^`%WW2%c*d=#N7(=`W zQxd)mo^Gz7BwkaNv4jF`Sx<(GBS+^m&CnDDsODQ($>rj!NJ!jCRp&GJj<`zyFVL;e zg<|Z_4Vc_tl-Hieopak!6(!xfxq_%kmWL;*u6M%{{IlrN3vn=1<(x(dhV=Vq7uFKQ zsUK@9c&@;tPM73ZmoKhbg^bHHnnwaH-8)K$U0RV&XqWLu(&Ro5qtv9u4~7zejn?UD z)wW#eb#~KO!Ee5`+2`Q)#GsBIVh^{%KdZn&Q9`@Ve?iqTM?6eu@{aC6hwC4RoAB9( zb@3`CIl68w_G$(X#dY)~WLLP?yS6r-G@iXtP?m6<5^TWwA!9Sf&)sOI<>>7e{q#dF zrO|uDFsv(_*Yj*2Omc*|zX zrA;NK+MGwn5!dokt5;V0f2l4RW0{uf1A6JnmYjvxkw-D@hNUojviGTmH-FhfSr@H2 z(_>ODINEYagnKYuKF*~)p#&~0JE&)!FXM~(%&k#MigrNaqgNJ;jLu5s=M}Q{?>t@q zNMBjhD!*soM)#J+j*;te6@LPo-v!x-7ExIs407`AMd!bw;ac3IE<>j=c=aOzya^)BT(9qK`0`U1ARGR?Q zxPi~4nroQP%AqcZ1?d&a=QuUwTj`!T(-zcreHT*tqVti}Hoahdkb8-x)wK=w7C+ma z+*f7iDS1KTo7YJCb-X_j-;Xre9QF==c!|u97-^gc6(|k#G)03kkFyq&g$oe4j_zMk9Ez9OmY0ydn(8yDktxmdJ1QS+Vdu+ZZOc<5e^9`GQ z>uvnPKm}~bEw=D+L;jCv{06C)pn14gARe3}I^g3lG2WFCXYej?hPdt~?G$$jxEPe- zo#|%8FwFE|`lls{gc9R-hI95W4xQ}u!8M0u8BG%gToc!l1#JpspAH>}q-cTC7O!+* z;VU%^fwxaEgVvy!xtXqx&;*8LG1v}uf||eR{OZjgPuY?sN0j@#;PeX8&vL|wA(W3s z&qGPZ`f7UV&)Zom#a-ZlX3Qx1%UCs7mAlGaD!4E{)N=I^->WzDZ%9wI4XNV34Cb+J zYW+BKCPhy#^(;|TLFG=3p#12^CRhiH%w8q)j~g7;hNafnFqg+xix&Bjr23D9&p(W{ zcD|+$_-G7Zl+lJcRcLTu>+KYK#6s{Ot?=jM_;$&X;iB1vmp&=83-f6|vR{(Fxm3p@ zTS6Ik#_@3^9`{{KJ3qon-fVYaJYev5zpB?3x>`{=VyvCpTn z*rk3e^Mporb!zt&S_qzrp4QTOu1~O4d6M08k+Yuj7Rdhlbs4R9bySzYCPQc=1h~s{ z&QlMaQxhgXN`kFGjAXkY=D)qmLeeDH*~%5QKVS}!y=2R7&h>-7T?<^)Fn!|z4dDMK zAt{;|xQA5gTB5^-ASIUS7E@NZxuxZYuS2l0-U^WiS7)B+7(Ca{_quwSUuu;%A%Wv7 zngBe*sW^dp;gJ|tMoa8HoGq$OIQLvk)r#S@OJQGY^Y8P9y|Mv-esN579Ds|(5`&JU zFF@6>&(K3Xnxy0IP;kK}jA|T@9O))&Y7Ti-Qyd|*w{rg7bC{IY;czgxXE!Pgz$Ts= zZBnl+oEz=EG7D8fZT8UOsJ)pu{~LSn9oN*ltqTW15CKu7cPRowK)N7BK)Q(Z-UOr* z=|Mu5-a$Y>Kq(@<_a?nbZxU)kFM)&-BF!&r@3Yp~XYW(?z4v$TJ?p#vF=3L-{4#mx zJH|WSF~;*e!7R;h<0(jMJwtI{Uk_lt->!-U-eMCBVZe`eOE)evT3k@o&$3N1mY=%d zrBZq+6KV<*BN+R;J+Dq*uqZb|8m3tX=_y8$v$NS10m%9b_GS7!q(TC32<%6ToC7fA z!<4s~b}%X|NO>Ml|0(<&7``UHxTuw#r^StDxvv_0rO!(zSzx zX^kg3FYZvte%TUGW@aMw`;O(B0R>9fA)DdqdkAZXFHp$S6!fH0#rS3|NLrK6gE(OU z`hF<)$rep=T5`j>%mY$})l#FC(XQpVY+569;UwAS6&VAoa#ZKU(wyX)7v+n&_xr$O z^c^LZcg6|L*OejC)NEHZwXMzH>Vafw@wdygodG9+6AEmRkcji~a|J~?Z0QjB6VO!w z_UT~cy3EMK`tX^QK1HN%&Z)x}!`5#i51emQIeb@9(y@$z$88!b-g|~%je>fHVLxSt z5nJ`Y)935GG8!t{PWE#8g3A;7w7$vE`;PgL$4lA3&q_b2J@lsn3KHtt9sBXj4@m$3 zk-)xu07UYf9Fqo-vtFI98*_JXFmf^kHf@FhH><}_`V#Lnfq@&+JP84Xf*k$BL4>%% zrxYwjMIjmij{xr1QvSsIp4tEPc2#T=F-Y6L-v~~1*@4X`9=5aYo^)LiYG8T5LE&bn z*!|)B;6|@MNhX*Ho_l)m51Ga^n~9A7FwyuQ=R5zSJm`Ou*Z*4oE<0=xKls!B55MJ_ zanon~SGWBan7+-(|6}Rx5BU;F4p`we|2v67*z|Y=WiIyFA2fgSKU|Od;m>r4xLWij z#R7vjvVQ_3{V4rcJZ7f^aQZp)ePIszs5%oxbt+msg|`GU(*_&98)X9C0PC zZZ$`dCwv&w>=She6Ly+qU(Fjl=RKp7RR*fFpx7Kp!)ujZK$BGqr|2kwc7gYpw)VP{ zs7?vRxT9x)X4BY4>RRE)!|HoTsBf9w{X?h8p-?{lD90E9&z#Mhf?vOuYht8Pk*6>Q zbd`EoO*Wb$(y3`dG! zC_g)L`A*`gJTfl$0%t|cnMZ>EdVVW9zi1boN2vNO^tRX2!{__?%VGo0qz16%fOcFz zdax+`;)-p`!(lYE?LN}%ZHvL@g%ZlX?%VpyI~KXMY{A!zTgHCy?w+?6c;0}wDb`)q zw6WAjLu}xZ{#{Fqvo#<_Lp!v1To{Na--*HLI9a>W2%Ko0a@Vm4QpNs=M}NISp)?gS zjZ#_e&f=WqYY~NIs;i@(Cs}+P$XVDv%)CFUJQ?xO0BV%Cz1}UbK>1QYCE4*#@AlOE z*_LNf&4K@J;T|KyW!x13xT~*?whh@HUYikz2VU+>eS;4Eys%(5BYuJ|uy*I!m`H)Y zuJbeZogkPSh@?m zkBxHhT0&-n#+pC%6+bw86%^?NQHMJ?043cKQj;nbm6JKPPCR>#NmdVKt0WSgjFVCm z=Gl&-acUn8OZ!;TRt3j3H?CvFkKE}y{)_O6U^yG>v@E1=L)0zV?0I}|;IJ+BmL6&EVyVL35f9>$2Fx}sQ_Rcu$4p7qhxZAOLKGJ?l z^NOPaU^N~2V-=k1C2uI-Nve#{)#`>{kCW9KtEe>HF`x=jco7q zz4qAZnu*FH-zLPoc_EOQ<8taj?)RIh@_D*0bXQk8Rg60$KeoQy2Y4n^8m3xO$MsxNi zc6`(1-Lgoh^JiPwa~zUB41D=MZ(P|cWtXETLF#70;8&?oRCr_Lv9N#3`HlxUJg*L` z<)s@E@+Z=KplA5)X1$U-X1)KSQ^6#8;ea`CKsKqG5GYk$O`}N8!eZ}c^Rm6vF+g8@ zGYGsH74%w}H5!AhiS~h@R2_9lOjz#~aebUZa;o3=yoh-IubNj@SsEOH8}~MAbumV- zvcxV6T8!Xo#+QZeT)b`Hl&;IkbJm5$X7iXQEHg9KqOVHx^>@=vmSPIKIci!XV9BHk ziqx!!!aqE3W?5zweIL{>P`Rhvu zpMqCFw6^WHA%ZsW%pK_nj2D5|{}H;>$1MqwoTw{P18 zBZ&%O0gR$xYsWWr-;a(d&_^{ zhriWfqtVy?MD%jE9I!dzI4p<5YkxrligK*XP2a>%K!bEz$zN+ve_@y{_N`jRlBRyb z*72q0K$kxOW7zzVFQ_)oZLRe~Go@|OvSa-fR&c?;Gp^3)C&2$$8m3qq_ZscGXOQ#n zewJ}YvTJUr@>OIhddUCaD(r&u+`9erCqV9s+(-YE)P}bw{XzXa@F$@Cj0u^Gz*feW zFG(>P2+IY@U*GrdJd5AQv)x#FC7J1v5)<;<1HEi1?+5+@qbjYd{IWe`L{Y%#J@$VrxWK}A9XIE~a@~9OArya+uh4ixTK4QDDp`Kqo0CUEreF3nJR?g5UVqT;BsG z_F9i}v46RxRt^5tJyd|*P*Ny`}aT4bm|f8MBJH#Cx6ZgN3SS)Sso zhgRhS>+{F#poa4u?J1U=IvMPekpHJ%5(pveP3ri5lUe-8k)j2>m#nKV66AmW#P!u?fB!ToQUk#KaU zzbWeDrqo~8b{c3*;)#BnSoSfi+nsyuO=>3-)U4055@XZ5#93!TV*u%7Q1;M@SLq=W z%JaCUfXga|-_HAWrf>ab-K{K29j$lySV6y5gcDGwN^&*}VjiOUZ7VC~68>#eT7s(70u)muROaRs4=)ha0i83c+3RrGqVpAS8e}*DUI; zXDRglu=X=m=yUbI7x0K;PHzU8qAr1NrcR=>WWR4(T6eYk{p=RgP<}|KZ1i$#v0v%p z6?VR2tkf)RPfW`&1xaDql=}%ug{__3%!~TY>3ENMs zkIwf#y*Z!dP<(Lfm8fm5ZGp!BsWYLE<+J4&wz1xF6y4Rz^0~KJ{K*-%bph`KHg@j& zi$v<PBpU$cu`Kk^Zs)8 zJAs<0JnI(~_Y69EpY(<4eS}D(=8!sz7gSKln~{i8qYO2LPE8$~u-E4OtM|on&1uB| z>_(L|*OhIxO$)}}gp5W>(B(lPTF8Tla+XF_HL-F;tv3`tsgrm%A!Z$W=l;_^GA3~X zf|to2{_h`v)q8=sO32Y}Uo+J)3lXS~Z`KAiA#e@2aA(0=OyX*9p!!3W7~7G_dS8sx ztu8OivKCWjl>A5nxrV-7Q)69hCwri54}Ce2-~ibQrM%MVD&9r+Yy>!?`%QgRi_8~r zhH>gh`_yOe4Rev0kv9)wuLX7neg)uYk!#N#6YMqS=sRFlgwhwG+j@8Bz@}Fl<0tPw zrVK8v->)YW%C^;wk=+-2`3MhvzcsiQtF_buEVpkby zT|vQkYaWB?`$D8~rh>>r@Z&x7jHoChfvBCOWVbq7xf8gg3OWM$T7HWrC>sBTk}htf zW>o9gD!YSVE#@~R5-HJ2q|Qd`7gK|R@B@!spZd4GPc`gxukCQe7!VG;^a^^e7g%#- zA{ur@2w_Hzl&)kdb~eS<+Q7>1D3?N)eH-R$*K5vn(wY?H$bD!SJQEu@00DE%u{T%I zNhrnakN#F*|EJ>!3E|6pG`Eqecj9efRX32-#fU-d3{L^`zQp?L+yrD;uRl4MPi_3T z^})=DevMI#%Eny*rEjA}hQ6`p=T!v(UJ*tPJ76r-=p~QSdY+qJxzeuL(vI^odydc8Jdr^h9++(=H_Ky%2 zL|*VTofPlaT@A;&Ik5L7F&|z`lvw!5{E#u#-(;l0ask9Jav`5*=Gvv}lzynRO{Go# zcHb%?EnB@1)3{+3FlE1e!&pI%TEv-m^3li#$>jZTTRgjIr1LO1>z0IYQbtgjf65N zNhhXsz3$-Jz*}fZx!)#;@n0QU4uiIi(aX4LJvTkTdS z+7F8*Cv~Y^@yABWBtnv^-L#9X)0fMEEs8622wg#m7g7bz%q@W&5pSNp(tOLh!mICz zmwkh&1{5_Kv0pn-Q_Abhd!QPo;7g{&P}5-n3MQ7776kLxnHc%4bRPvr)`HH}z%#Qw zZr~Iklft>+RzVkPTg`s+dDwG1GQ|jsBbmp8@jY`@)N%U=<0|fS4)4zwH!wPIHNB3e znXpoOkv1BXZ%QG1YyNRcg1zSQwlk|pgcy21XY=K}D+{>Y-=qNkXNqzETjE+hx>d_nrQ;-zm12a{Abde0 zF;jNJsK#A~?Ll}MaPY$^d+1|5YwZ&{ua?REQdVXOlsK29z~GZ%0Av=c)7QpTh4cz( z(O8`hHflq)=23~8CcqFP)-6M_92suLrb^KX#k`wR`V41*hHqvOBz6p8Z%?F&z;2o| zmMHvEktDYpv5%Y`_kpWv`SpohR?m#_qxO^PZC@>)%4vkbq~lt|UeVeTerVad)yus+5G9s}xWQV%cVGh}a^JZUw1UT+>|9yziO3wslKJ^ZGq zdGxtnkb^&t)IA4Av0#&?hDp%9T9jm2Lwd;c;#z9EMG%66l7If)Ls^z+CtZbmNSvDn z?|q4o%bkX)!F$5ZCeM}feN~O7g*m^mzrO);Hnp9ue{ccQ$H!%6d>diJKF2o@f ze!qGJfz`P{)gUjkvcDLs-bw(orE~XX8a|pmmqAyB-=ZT;>{?$XD5}6<^-02HL_68AEISzud9gTQq!CH0)^42_819u8-IpGaFl%IWXXDT zW!4gWYB7mR^V4hMEj?mo=Cxaba48ZD2?|#MMQ?g=ZH=O_ z>0ITLdwlPbl}zZ^YnoF?0Am>3KZeQ%cGV1LAX1zXa*|Vv905z$I>{;uiE5QNusOzV z%^9z&5*Wt}3UE(d@U3Pt^@=vt&|Xt>vCC6OziNYCFPSWRTKV<7n&X|Ho_VvJwHz5W zY1-;#HpJQkrcMP0U9mUa$}FFogLdW4nJyXIZ3~GqTFpLq6{FUe`0%#;O>=YWh>g}h zn17&fcIYzH?{0lV=xPT_dDFIkWMhArojt!6QpQMb`QFUiyugx}lMQ!RxIF5L&p*8B zAjrK{^~o5hCc-kd>)A|-Rl|&?lBNs~6^r&$T~DhQ3E#dJE)b~ffTSLlhhTHLG#pVH zF^hK0_3K60B71B<9-w#LTI3bL1mlr;t@r zUIT*nFS>U#W9S|M11DV=tT%gOSVpyJ`NxFQ_7SBBXAiuf=~v?Rrp|{U{mO*E!^Ed7 z#yCk>wU^WO8&b@eTzM3_6mC>1FxtJu8}~dCN~D)=L}z=PDllJ?*kZ?;#*p#Ng8C~n z=c6xeEEk-ZsMXTv%ZBweW5+Sp2;i#P%b6WCiQ=;~Lc6CRa(h0DQU1I(C7Q?Yf}b#H z>vdiUVh&J!MkucyB2cghEz4H+4U6cd7+vnPGGU+g>e{Ip^e|p!3Egc+;*Np%!t=b< zeGfcNytUY?T4Y#U8{ooy5@LxGIlw9x=O})HnAb;2s-7F{TQ*LVrQkXMdp-s-SRb*x z{O;P21VRSKla{nmg3r3(*n(kzN&R`O^tb>tfP~kh_-pf##yY46~5dlO$S~!j@(sE={;|ocD;Tn=5smGV#;fXswoagKl3uL^_p6NC+1w7A;EUIyj>z@PTXs zWY%{O?w9-1j|G$#rx=sJ+M8Xw&;X&Bun-Se#<@01J5?8Gb)@{D%fp)DgUGCIJr?Ya zR}#I2Do!?!`CjV7*G)>eV+suJ8n8lkLv^byzIIU+%h?}akbg(f!BsjV7u44!)#cSei|ti(TIX*&O>*B0<}`Q8Cs z)kM^|LOipy6NpZ}6%=ctJM?t&`k*m$bqyjrBjJYM?BVix zNzMW8W8xxARNOqk=ht-|`kcqIp40rTJT&8*iGq(D zMRa{6^$(@JPPt~gn;ZOeSrjoYi4ay49fnB*Io42fuCp068h)pKU_DB!q%5~0n@7_!r3Z5SHu8MxF72uw^|${RCS}|tER7KU#RYvTQ!{X>*p_L7M)k^73Fa|@+fed z&aSp&pNHbD&=kYS?+ZJmc^$>I6|`2+Ai;a}%Jm@N2DbwBM+KK!{q!HYoDy0j%OO&v zw)IGRS`bz|+RlxkVsd|VnsQ?Hg($IebikAN?u^Khyoi7&AIaQy+Q=_6Po%NZ#y9dO zg}4!nqxz{jVVYU0^;5DE2d)Xe@0|iv>7%a=Z``TDT~F!(yb)rm;d@(9Gws1XA?#5( z0ishA#UF<>N1;4Z#Q~B7-})f-xzdhj2_?kO;TM6)!C9q=&2;*>m&M-JM_D(OfKX zmJL(JbAAY;vkJIP4N#XsdVRfOMe@cK&Na8Qj|V0nmKJAp!9H zO^gQT|HAPPI29~r_EsAgxT_bMD7+aZ0kmp|rp|;ZQBGtkb=i3F0PsbB0(RMAC0Bmn z5pUX2_{(EXEkU_klwKl+<&h`DuLgI{*e8XKW7b2VE1hfIv&5B%ekP&!&V6pz%O+u< zE+Xmcwl5j%dplS4X+sOV(O+KPI_Kki)evuCEfe;o^@}@mXXqZWJ+-71~^}WB{TxPYw z(ozqRGS#~Y?j#yeJQKgNNxuM20gNx0kr%mD<<09pkXM7;FV90HTQvlh@g1aWu(HWp z9^Lyc(l5#-NkBi#Z;d4FBg)xLHzs(7yo~(_6$?|UZfR+noG>E7?AtV|OYyJKu&70m zYamvfomX&Y5K@o9s^^cpy`Yz|`LF>Aeqt&rU1&^2EDYAnQ+IGv;Sh9WqVgcP*?&-* zHb^?IjWk$g)`Rr+_E|yL{)IF{%jEVc`V+`17WK_@7|*38iX)Sm|BG(FA2rbYLI=ZM z1W}rXXo;H|LFXL|8B;AC&3puw>{5@+xRL?5124A<9QJ$n@o^-AAUG%r?3kQ*U5$)M z2$qXYey(sJXl_xLR%@hEMW+?q=;dN~a--53hr=xiyYW!Rg&4J?>U&63JG1BXn%K*- zbB#=u?Fx;&sq9>Y4tZTmw)!oJVxpgj;jP^kFOQMF>)1vDd4Lk_K0mkgVG1i*f|-+4 zkGzXdP;4f!^`_CTyM;4CsPlbUkuNc>LyugC#}*osOB~kjcbw= z884SC#J=_|cJ!`JL#!|k7N{^7D58zVUJrkQ>GqtmBZtnCMf}CMay_f(z_q%C=v5j9 zTii`){0n=`GORsfnN)%l)jKMmG7Ha-SvYfV5)yqVmEmK3ZDGK{4IN=kRu~hYX!5o} znpfPn((E=Ecg4RC2!eYBV2gaq8e_xdox|cglgd#Fi5mud`v>=o?40-8#u7^Wn>1>v zyhfD!sR$`B)L$#i-zm5W}MI621-BTa)DGjB~_4t~{Irq}(PiEKqTM zo=xRZeoQ2^%LCw48&7T0x#EI|5^V{PTFRvV7$yWRbr%r}Fz+w#-e#3hSDpYyL9S2h?3`i=5 zH=@YYWXZtPY(T(x+sT(zP2e(<1-wte?g_5i{($hqcX*GfMGC^HV#`h*-D|I!K?nnB zf#Ni^Mj7vEJJ-qNQ#Kx}>R*j=j~M7z5>=Gryu1F9R1T$xAcQdy?WB*SLrqJIIKnLI zo2HygfVxYVPGOVD4O9Pq`FGe@?mkbQ^Kc$ez+-QhN6n@N* zdoRqGj1#vQCEb@z05;QYo7}9P0wIK4aP8A$GAmMso>M(wAnDEs_U+XJ0nO~=jjny+ z>*aKRJW4wN*YWuxl<7IEG=V7jbi`YhwbKiQvBbFIHcOPbD&Z(71MZso!rSML45uJY zaG>d6m)s0?g`|+UtHxV$sw(`U@TLs!ci`7p7=tA-MP3xK(tu(6s(OI zvK0!L(pq4&Akn#`k8s~AF%G`w`wVJ~VzXb9CljegW?e-I@OPTQ@QoFf70i|YbW zx;~Ng0YOD!*H3%fLlePfa@c{HtvR1p(^F|qR_&oUi4IbV4)k-186*>XF(2@#113cQ zVZ#(l_hxFX9`yb=4)^HF>4PCepO39Iq|XrXi$Y(eEAx_x2n_H&p}P5jO#KOI$xdlY z?BPN&efZkVwyu}i8)}9{DT718bLLO%WN7v!YUuq~0(ewC`(?TW?bQ{SQDY~MZ-S$o7I5u}6%AjI>Pe~=n8=3%b5X?l01 zJopH#wSo<05NcUz@F8D{8dR}CH&KuK*!zpM_#CCzeV$5qtF-{_89y<*L$FzQwW3WC z*hp*Q*zN=@T~k7l=cC_CjFQ>p^o`aQaz<6K;S;5&`91s`ScqA`e5padVYo`REakupXk(a4CLNRx*2LZ-x^VtFecfnLsc1HwPJoXRaG47#KamHZy1!;*pVf zS3!tS+O!gHM#8YOnDPGPNwK8q1axu#bIm(L88$|>#c#zMVpdtk;fB@xA^{TVP}503bj{tXvY~i!y@G#Go3bu0tH-6kA)T zYZPXMhSi^j`q+{ti6*<){WgJn#^MKezLNJUtoD7n8aKPaj#)+KY9lFPb1xGLTT)Kb zePJd;*VjK33-Nq)tEoaby5g|8UiWVAQh$q$d8K0%SL^$XGHDsqpN&gb)C@nC^@%9e zU`rioXi8Ll7xOmpA@mCl4u4X^3xZi8Y<26mGS~miY*#%wG}F zKd1j~QdzU8xmRV5 z)lv|94bMx!eyLLy^{`<)ofvUWl*}SOPqIy5p zPMH2U1N^%D*U0>u1i#E9=HE<$35J(j(M~cLCe%mbbk3ZBn+M#%r4}V7I5A)Raj%4M zCN+#_(ZjQ+T8GkD*?@mJdBOms81{Eu^!Jcvy%PH41ZQIW*Tug^JRE%Sn#cz*Q)a%k*Mdo zcMlOMoq|hT6B5u7Q@N2%T|>g!C4J|_mrv&Hyd47V^WL1TKED37qRZOa(Rua#5m)e9 z(y)go^x$jh&5dQFipp7toSVgw;Wo6eF1AOX-@_m;B5gffjHkblabJejMR?8cK8mpv zesJ5K@%}9PQTYj1miK#-T(_*q+<=d-KfVS4w0rEuBuULMM0s1Ti{P%F^qDmLx+}6V zXAf>w?&Krp#C5AkHt(EmHjN)4Y03bU(GtV7B7(FO>;}s!__&?^s@F5mBC{GlX6ikwiIwqC zYoM530Ri!AH&|L}&(WbMA+8)KhDm7>LT8<0PgKN4Yvqa~2`?LGLkJ}b1#npt#W(8* zky6uo(c1Urx*^!JM%1<(n`4>7dYs=OVp1% zeQCU-?HBO4z*|CNL73pJqfIBI!rFJouysGcrH8k=R=<#^{nJedEQ9#LyV%BTk zJN!r9rnTpa;7;_aQg&CmX&Ovt!b4%E`;D=HLedWJCqsj$iOTVM+!Yp{mBx28|K1p8 z|1)FIO;FL0r0SEt>30VljafE8>XsI6r~2KVK_E`Ng*jKw?}T2VN(~*fgKrg*r#Y1= z^RG5CyDl^O5rg$$pA+o9V4RnY_|?JYp=eYoZ_WIU-D8Q*|t&@mj;N7QAFV9F6wbOh1i+2R)rPG+8Ps>@vy|cyCJ_=f8w|qUi62=S{ zfxEhZFd`EyQfebr63Zr(-)*Q()_uSEyXCE?iTxC^l{mK_#65pE5uXPKuaj#S(p>-B z9t2qDq5&B;*{=G4< zW2CUghtQm`pe2)e;k~O}Z1y*_RevE@!TI?I*(8>Q>wk`N_40o!S1HCHy@$Gf6}(a& z_OJtv?N7VD4eg7qaeAzSZOuKMWRFs7h+Q#GX;63(UL=}0_d=VW{sqATx!!tvXL$r@ zg(;R1IOa!pp*k(ae<3*`LFZ9!)|USzM*I5?*LKQ-<7R5u`IF&@snV>K$D*m{iy0y-cq48J69H0bIqg!;(#|HX%gR$N zE*zcB<15m)9ICe7j5K&~d#LKEesa7Yx$rT?oi4r+M>YjYQ&-8L9{UsE0HU#F6u`v5 zNO2)p2_-P*WnDdO>B5`L);v`^7yEhzd4m_tDpY>oqu zHku`jRu|J<3oMB!Z>3Z&$Yo<8%FSh6*@^5RhW;9hCv!W&%OKAH-+FmJ4{$IVJ?-9( zx7$v292Y_xTZo)|-sr;(1lx;@1J_WaXy0K+?ICg!j=?I}VfVEn<-Vffa188Y%oCeCInJ%Z4XKH4bWdDVhOT@F(7%<* z7x)ukPfBLLB!p}ay5hVLZ$i>w0v%l3VDq1+?oE;H?ja+VxI-dd%&V^7?lAK7+xK#L z!epUgmbhGPhA@GThGkdV1YF3kg7EAO!vae;ogQnVte|gsQKOQ3;n;xzN>s2uipk^2 zdw5T$LhvLEu?RQ2gONjPqGrQeIEogx(ra_Jm(lF+coMG=6K{yytpYINdzg% z+T!k}bNBe$BDCbk{qVrMx+sPZXQ63!c?$zBAP3hOw;|dxmAoy&=00_>I~-mo`Z%w(0mmBvP+mX9PF1M3UI%ou zw>(DeTo=RL0p$zf)$2SP4TByUac9i?*xejpZ_o7Lag5$(PwwW>hwsyyqKcQ_f$w;xGzPw+@6ms}=pYWiOX>(n@r&1=QNFBXKSqt)H;Dem@r zt2Ed2H_Ijou3gJ8NZRxd`w1Xh{@V1aApbH{HF=7+q@2u_cfBf$ute8GHnHc3I4X4H z#+Kx+mKG)rmTB+PTx!|nrFUe1E77>+YRKAaO0qB=kK7YKOMKjxOOG>&uD{jBEl}bfr?ZnsZ zC6i<1Z77_|dFQ*i)gCS@^Uc_KbP^`n?n{gz4KQRGy>@n-=>1G$uFOK!8@MQk5J8Y# zpNm=agRIhXgj})iUHXe_lWQMIe2t_>Fxt?&+P`Ce zmB#DRWP3IVTka$!(1*NfL@^dyGGK)X(`t5Gd1-PKL)XBGI-#9+$-yc)Wd7UHJn*2K7(&uXj8cuVEySK89G1`mX*K zokE8f;9xk@{Jr|~d-WnI6CEDF8g*uO-oZ6yXn`297vKqP_D%m+nXGWqkd?&!dE@$N zkklgW4(?vX_CqyC6U!G>NiMD>17Y9_^33*5B_snMd^G$x0zx8OTbnByz?P^=vVD&% zjn{DN2gaoV^b&tKn~%Y}ToyTscnm)vj=8@fO}dpHTJR~qTSV{F^GNdBL*wS=?vm;v z#O3OP%;um(!GWQ7HseV&a)rwn))mE0ZiLZSKYUb5L1{t61*kx-n5~ZGyT07@uimU7 zZzCsu#LznuMHf|?-WU`+|@&qwl%hC{pt(o4y0YNt+e2LrNO-v=+5M+CO*kg!P0X9C3A`wcbdTb}61J?xxP4qoJMu3XJ~M&Qe9@XeIPUbt|5fcX564XGYP_95I6HdApVU#P zzVci;ASE~HZ$qNm#YEZS<_{xu zbd;ilJR6UxE6@J~wyB$@6bW>8TvhsZk1ZrTo>JC$e-oe=4You(Ulu@TVFV!(4kZ|E zj023Z+R}scd&~j!pp<77oTyBku&1mkK(2s4!}HC`@k#45ga^AB}A~KJ`=s zp~N2xP3oA+A~SqN_ngr2mN;%4tQP*MNx?n7^6~&b?=7<`vu-DWk(s4cjfu367A3@$bL+5dWR?m4iKbPm6?{}p z#MSrR4+BWQYnyo-_do}{tt}@&8pn@St;{z2x`gsM=Y=$CCZouX)zCqWwaY1gsRQ1a zpQ>>;N{muhN%O|qeXZ9xx5xtnzBARDQ3~-zyxHs;S`>vbwiAT57zWYs?C)C`Acmgq zjFJ!bW0bYYd5R3&8myw3o;~cezWYv#lTOau8Fy0BMe_y0=In*QL63A)0X!~TO3Geb zA+-d2AX_`eV?Ne$9_7R7#3o886!V(Z#?qy>jueLHjCax8Muxd}E(b1#^94Y(P`i4_ zQ?ywZwX=V9&RE;jk-cc@ZQu9CCK>N^qS?Uam7AN37i^aMSk-{TE-VRa6#)WU`@6{@ zbD1ik1D7Vw2KW|oOgkg|pwRUB(#?6piRCvu&xu0G1<(bI6U#W|tS=9(|RV<`m;ouF**(mGII9%+haTljiSC?3{8hze%=!4Y1 zI{eNOfq5hlg3rZM%XG7Nfr%;0bESm7w zb^jdd@^8lJKi4~)`hDs=W+DbZ_zCEt^g4?ct$XA9`NLRVwW-?LN1vDtVFCiSA5K5? zW=?6JXFFNLr)p}EHB58y`WdWl*7k-|EeUL)+vzNB%$DmZzz>_{!iRdH5;mftC7|Q_ ziM?7Mp!`arK1f$H1JM=!-CN)(m`oHe0TEB)zQlrQE&_)=n?7FNGZ9r6UkLNbF!F9ukCO-ZW~VE$s-QAc$~P%v`eS$tBNLRhr=Xkh z-t>tYx5^YY#e28I!O1Ic!KiVvTbCJF5kdI`^@ADm-j9%NZ6#`Exn?xx<@8CZ%BIqI zIZ(Km&IDMqU26P&{yUVu4}_=sU{5A#Dp4?ArlUmczPnqOTPmi0MVbg>AR!7jySBKv z2zKl7yuUwX_Ll1P0C6etxYA1;I)^ejPQWJhD#sR6L_5CwGL0Sff1rb{zoit9>8Xp$ zvoFL`2_9~%j@_Y~BYQ|lOl!dwGjO=Oeh|J)<`EYmo*c83YiIY_&`!6|+S<~BCUORf z$6l9H%|9ea_Dz}X96;cZ?>BCCyT+ha`MARYXWIR-)6FM^%yT4p(l()R!3}F?3Q>zg zJ)UL0XIjxErHwaVG`IDQ``?x#hlrrOx)G$$kfC8;R6p{)MU)}nV=wC?jdFNxB6xY7 zIj(V@_I}DTz!jvw^_=_NOuJ2a=n`RS@0-3D-jnP-5b2z4D!L9`WtcHzqCY?rY6C^4 z#y;(EXZH+J*b6k3F7O+Z;;gBSQk@ct;nSH&Rj3|k$8tcv3Ig-ow_8b6OXB`fwC}B} z(}z6$4DLe*qP${+h8>jJCf$d)O0L=rc^F(s!lE+8MpMtwRz?%syb;naY&d&sYd^>_8lV4=-ZFji`%7>wiH`!*D*(SaA|VzkUo}x z;AMoW9UW>2zsYym_xfLo>P-K>{!xZ#P+;UG{X>iF;<`HfB9}YBS754py>IklzdfbeyZg%R7Df z;H9By6up&&7$42C{kD0FFWZ9QyiOb6$FtiK6}ASu1;lSF=6jo62=H}%hmd~4Xg#0t zXqDDQq6^$gpqS7l1eL+ncq25J=E9&sVM(tPi; z!5l;eSmnOaU-aAOB2~Bze0#O3KnKSLodEe(bGv_zo zoafBPANQb8z?hnM82b$M1q82-yhbb+bGsIH#DF`M@~+2ygBaQ zdlj)fEpk|ICUVfD;a}6J4XedUZ8hUI5_Xdl?>MP`-iA`SF_o&k3fPW&ui|!1U83^^ z#iznx-5Y#6zX12_KOCK_eO+s3)WQNv#JeupQDoRc=fpblM?+O}2yf*6UES!e1%vv! zUXq}9$gq!Q@K8O#F%uL9U6bkNjV0E@aE!b1aLhrjOBp-5B{!JYB??G|^N+W`>St(u z#Ub#}UCLGhNZw{`9PSm8y{hVs5cAKoy1a>w&9?Sx&1iI;8uPJmEp}^+v)40>>*oxM z_ot?j;k`M14)trZ$i#(6)=%2I>hC}Gm}nMQz8PeoLQGI$6I;yyG`o{>hN^!fgkjh$ zwalz4@E1{mhSJ6v7#qqWdpnO+rzSmuXE%v=-D;BeUq2B~kL(=@gtI?o6*opor^L)Ke6F<`~pP1@yiH&DFrxsXd< z^nG=tU?i=>-oD(B$?25H%^nV6z6xdPqCp9XV!Zu;>m2HY9Z+)2o1$$h@K6_sW?IU4 zb%jY6Y_WMF^>8gEe}-=ztP2@Y7(B^MT(N&j6QlXK&na?)!&QR3uRL;@5q$$$Wr$J? zgK*O}tPx*72tFv83(?J@{W3tW!el$*;@C5mfqEY!6+pdC^z&v%W>ttkyp7rjJ=O!_ zn#FjO7+O_YqSVUNNZQQ5qy#Ec^E!~+CHcT=7DQ+3#_({%R>n+_Xj?)pxcUOv+j^&> z5oxQf<`_e{H;(J0qK&l*DW3A!tm5E4NA+`hE{N}|{ zmb11msMpk#ply?1Tahf^q@v7njguo@%%?*=*tFq>IZo9!kO-oQD$u$p0P&%hA?0p&6a#Ee zyqRbkQ}5vQlp013U=VCt7Rm%)``L-l>gq-e^O8HI=~U2lMVL2}C(@6n)aUs2_fD9o zlVtYJ+bRk4J#(XTggq;G6JJ*HbZn>sAX8X8p}^);HwXvC8jU!lwmb$m^?Gqd`VeH6 z@6aY?n$#;QzvKm5=pZa8i})gL*w#8Lm&As(@Z+wiXcc%;UWvZZu3l1h zp~16>9oeA!DfUGDX=#Uv$B+FYZ#|-IZ9@km$7IooSSk9k)4j8M&`#dc`Q6zz3p2wr zElEsuvKPoya&)yx2Ngoo4^gG^=3~-ypAq!oE(xYqP5&=<2mFj8a_Nfx&{6i z9!NsKQ6`ZsHBLCPf^3no{voC5v}%St3^JYJMh^Q5)}|D+Sl+56Gl?J6q_?tvmiBnC z)!=CZr_QRj5W;xDqo*;*m%bKB6B3uVyaT#fi4xtOWov7S-e_5*>inv6+uSzC_orI& zHAwEdHcW1HC4?|aG5T`G^ArxpHpHJg#15X*rEPhBI6CuOs9$QbmkDg~e0>pzVJLv1 ztstC;OiHj37IEaoXT*B$p5jR&E)KKhc9wV5BZyAXsnBItT2hu&S_81IAnSVfQQYvc zP$^G=bb&a522;}MB$v0tb#ZfBVe5A!nQl=LJQ^i@dT7J`Xu?kFVfI%`T47cG*7R%c zsFvX|C&?0PYLWPU-F26!Dj|*xetb;M{LKh7=fJNRDp*>5uA-nTYsEpO zeCwdYl-=?h9S3W{7uL{%^hfh@7oRKmmnSyEbg=BDiF`O6y8#cj$MjY)R;vUk`Hh;B zm~4UcyI*uMwXZoMzv4x&Wyxg98tz9Ywx++Oi7w=&=B;s7mU*ehk&!!&--zBi4X&QX zD5J(!3;eKM!K2|hyIb8BKSX?;eLiU5-EILea-RumM-`ZPFj;=!rDx+&k z(Z=0V;ADMdZ8;pz%htYHc_04Xr&T24^(JE3QT9oDJBympJ)EPY3X~0kaPI_R9JY*K zIa`3rPsX^roMg;k)5A*jvSg`1mPaAddflbUY=o;+RTl=HYb_QdLOJRsL+9n(AHP_1+>i^xCa+lwO zw`6kXepUoa$OHb;HySREYfe<7VxJ%dLk|~>oU$RF=n<4gUt85#STl=Lfn_Tz>NF0U z^>1VK1)K7whKb!7W|AfGpr7KRsCpl@`+`)>7;GzJyQ_k#(|-duU-TxhaG<*Cx@78|_@Hy=9P8PO3Lor;DK{>PJR zx^cP95n}f~3{%;EeX(X+2&j-sces8z3?zpza0dxgH?=(VUb()L8dGyxoEDV2hHbIe zVO`!qTnu~0EtzrO*;w`eYT_pXecOgbn4=TnE0nTpLz0p*RA?FFR^(;QMf#>sgl_3I zXTI?pDoT`ViDdNgWG*p@hi+RLIM~++d03Cq91WKq-bY;6uNgJbkwj7LZ1Q|^jWpq6 zCnkEw0D{ts~k(4{S#7)ITxk(&E+Ldo7SS1UfZV(P}m=))^o3qzn zcxj8TIuOVN^R0`|#X^0ABsVNfJ~zPCg;qU41>bn^n#=E@mkN!yWeB8vK3i!E&o9rT zp_-{T;O;Rxc`=g;ii$58baimA-wF`so!U)DK|^e%Y~a#Zn$42<9`kaY#jnqiiMm?6 zg{j;;I>I4Debd`s6uqp-`j=d-G|C{ zw;SP9Z6mC29@9x>BbtCjzEdl--#weT%P^+n-}Rwn=(SF-*l@m^ir&f1wRQrtyz?gY zR-?Vgw-R{2kd_`L+1i5|;12Tvb6pbGE-8}{GLbcz1e^NKl3h737%$xkzMO3UNbskC^u zZ@J!<4PmCLiCOQJn*p81g*u>2}Up^Gx< z+8SPs4OsA05-A?gtOy5E$|K5eT6eX-9EP!EWEUz*=SAJ@M2c2B{UCZKYw)^?0q{iq zTvp@Y7l8Vb*%C_U3$)23ZB~}_Khm!$!&@aGS;5D$MD*A2Sb}}rg z0&>@;c5-;Kb3~}QE(xYqofs_lhRdYs8pQu&Q~=J=Y5DnE@N!pI2FG@1PuHvsAvp$#asmmx5Oc17VU2Y0-|G38eei~{H#9izlfjzG)wM<=#X(%MmPKI3k@()WtoRFlsh5?1MMJ$@UIYQyCQ4G4#gy0K~a^Yrlt(t zk4@F_VuTugpFc<3qj^M)^YwSct8un2K@-iU<@1X9V2d4M9-WX>#rP+QVGoI#D8EQ1 ze9TC{pOMxzv5Q3}cOb+Bi$K zH7$sSGBi6g6rH~L<~OPm%h!&J^t%qd^?YKw8UI*!sj2Gb;$=Dyo=J1T8exDP#2n6) z0IDdloV0QrbD>TJ6}aUqjaG8M-IoEIGgt zHH_yek1NO@=Wi%|wjZ}9SsvVqi`ocD&i|&h%uzp6>Sp&=J>c?CY3l9d<~_E`g}cMW z?t{-9?mY5inZFfrR-8HiwqpeujmCS7?N=+a)P|DobAFjTQDOQ(g@!FE#EmfzBc+~k z=*)3E$)!wOcwq&s>bT1$=qM|y-< zJUg2kd40t#;604mMFAJu6D0_*>t=bgU&EAn6D)kXJldzycI=uqVzl#f;uhO}ZTAkW{P+F4 zFeVjCfBuaw-0N!w9}|Mrg`eB#{l{*=AHdVk6TWw?l#?6( z0$5njIabTsr80EKNpifjw_;P&;L`qxXKbJyjd_XY&w;2-i>OgV+h`*D!kWcT{R?)y z9K)Ng(Jn60Tjk_YvV+{6(>q@5bqk&0 zUyo5Ld$O#ZlrO`tNiF|H5XV`$%)1X*5$g}}0fz01LTS;d%U!)UtYJNQr5nxaMoaTO z%rBOo=Fi7|e4;drLwO(Y3_=7rEw#p%b)l!}c#nmxv?J{dZz3=X71WWMFQol=Cu^_}Hg-j*Jb;r4sCv6Q+t z&N_uu)jhuErs>TmV>ZPYT%>Q!TINkOZ0Af}6IB*3;$!nn*Yt9%&7Tcy8m}{KA$wPm z{cdG{4&ghS03>uTdUfT7O%muy^F8_2~Su zor1lX->b|oHXXh!mt!z;KdILc9}k4ILr>~=FNagLY!tOOwGjawPW-hdw36 z8y+z`U)U@c{y$5!<@Zpd!E5{ON7G>Ep9^Z$)#mSM<7GMG$@XZ4#bW%OWp})Bwof)M zMUZ&lPe-tyed@{26Lt75_3u&1@8R^K#(`s5S!Bm*68@E;=b9$^Ejc3;6_#fncXjXr z=dL+TWz;x-)|uDeo^{fJ1ynq4(&k7w;W$#>QPuxb(DqliacBSF+qfY%Hr`P_pE!~o zR0o+DzK;M@pY|KgCZeCgHlbhQBgb3AgjTKr?I78TSK3q?xoI@2HFkMLxI0V-G>GeXf4EJg(@ zkytO2QW7yY3tijB!dZrA|5<_pilrg^7qBR2#Etxw_1;NOYs1KSN;z`&kbY=iK$@r%ivS=$;F zv8s$^WwRh*k+`9EsCVrlQLbg{kY{KJ)#QETPqCPu$Vbt|#+6TW>!>0O{NsfkA4*$u zdhb~=>AQ5NvvOOmzl;eAxZZKi?V5HqhH(0lb=z0CF#*9-7Xw>UYY6GxJQ`oogRQi# za9Qn#K)PSlzYY+VWsi#GN3m83hGFDBgG0cx3~nk#2Pt%%ng!15B*%Ww@m*KQec5e* zWNyT}BVvvO@5FDTJO>lH<4P=1AX?RmoO`zRgX{80UU*#y9$2v3E9wXf zp45GV=y#CnQ8o#X$JF53vjwA0oXPk%qBALO!s{AU4)qRBpb(EAkcuPG&sWSMo*F0eJ z&P%b>w>PT0pG`m_#7Qa^og$7CFLE*3=ASLamx1Hec4T8+lS_+o1@rnUG`V%+U#h7_ zgn#tRcM9D&_R9f~ID$!e5wT$+-o}J_`uea+F#F5oSCs-W&k}|>geylvnG)^^;O;-* z&Ikf2pK)ifw3U*VLTHe?ofOc{2IOJ4Xr^w6l9VZtEYX4@iHp^kz$X8CcrN~n6!HEZ zj;{(7G$8VGscj?LxnBTFL&9f1QpVrt=(K$e%Ra>|()zzn`J~%B!Sn@3`SznlVF03P zAA6Y^qrMF*(`@{ZniWI{R4R3SynkZoqWa+Mj&O>v8I~y^`0k?}<<0uc>yRhY&7>$I z@~~ExFI6Esvz}Wa6G;lyQP0aVbd2vQSJ3nu^sC`<948O(%IwxZ?A`sj=Og$AIo0`6 z7rlTG3~@!5OWiI5E52x*(sI>JQ<*lYuf3@k-*tx%|APXCHriiHMyRxn_Hv+=1z8X* z8eUc=$}xkH&M(W6UQecFS8WE*m}S70Z>df`P+${}a~v*U$9Fk83~UuZw{@^mqQw2H z3nJ`*J?ixK+3C97whB{xlDh8w+5yjH0cOl34(Q~yvO2^FtGDXguAh9co{d~BZN`ae z{=y;Z+nUNA?ompbk5g#xojc;t6C?y3<}iNb|12G=bCKk_UV#cj9^^Us6zQ%kPHIC&jG26;q}t7 zIgp#kCOLHQYIz8w$g);4SLtV)&7YSKUM-52Mo~V0RV&ZtabxB@33qw2~t)>im?s@`lKMs7?b+;=(QS^9eY@ZNOkCnJ2 zw((`05?LYZT0O1zX5!^B(6?{Ka++k`Q)aL7%-L}G@CIzaYNNGs`T=F1SZpbNNk8st zh&$DFrcF1YqqCsQl3VAYUQI4FXdy6M*rP#$8%07P^nyn~#nUL@0I+ zjp(jc%xFJtshgQ4-7^o*ivULf@a4V)L^o7=t(v#`mcK@=^couXUNY4oqx4u4;3I6j zz0>A(lXj{28K`%Cn*%ND4>dV>40s6X+YP`McGvccqF}B5^3j>_6sfA84-6 z@NBCpVKV#3V7|L?=ld8+{LNQqZ!8)Eqjs)OUfw9;g$P z(!|3wR)2WZ5nY7)v;(v0WuB(?qVfgpoN*21qIV=Hyd-`WT}Utj#{V(L1Sg z%&UlW8}GM}O6m8gMLuDv=cbNuv#5R72cryX*H~qDA5ndHBWbR%_TTM5{|D}ex23A_Y{nFwh$yE++cDZ{ zKM7Tpc}s|UG_&?@#NONfb#{()x^{)=0NA^mxD2)}R-0Pd8<&kv>(*PWT({Rv4io5Gax-ja@CvMTUNSaK z`DLEf#*EJ5)IP|Tj}c*w_zb|?)^T6HfizG0)UH@(SiV${5 za<@_GLcH!vMjO3G^v+>dvc{5hb?PhaR?Ur{#)Fj%?uW}Ml!qFFxUXK*wC{eWd1D2l zL(cjweCOwydTv6r3A>Jv)Co{fC%XZN+t;uDa~8dSq|d+4`1eN#IbH43{15oG-^$c} zKPmpVI^R{O^xqgN{T*NTZ<)d0^6$TII#;{H|4Ux;-(nhn?K{73I#;{HSElouLBa2v z&L3nKS5IoVi{3OiF264VJZL47=^#8tnAyC0@9oim+1XRwZ=has6kWtGi!~%$3bt&890)aOESfR*$Q7^?x8HP}59$X#c9W!%toK zA&aWTi+=?P|1k=P!PMVll*!m%u6vP8x0yF$11P~?07&X1?N22>NxPTewu>XnbN)FD z3VOaij_-B1Z{EIbD-v63L2;l|g-PJ0Rb*Vfz!Qi_EV-Huj_Z zA-Wu;-I0lpq4YuF^(Cv?wfk~9}=zcph&VUIul{8~3@S=9TcA~`h zCV017hylV2S7Wx)hIQF3CXLf1@f?@|7b6r6kiJNSpJZ$(uqHMLyc7i z1yWcerDszJ!}e&0ttl0n7WFF!ZoPf-BZchxR?#)>8Jc7FI#l4PHK?FBk=Ibo!wzkPVkRmo&sUXLbu0>`P>DS$2{> z*7(eioPqcklqRR%cQ`fvVZO27t8h&^Nt zl-aHIA=01&7KuAiD30faX3n4UQj}r!h7Aw*zmMTsVmUfm1u!Brm^)717uhJ@cG&kz z_WH)=j2+ddJIkZQiS@}N?zn2AUoqK_fOq{KWRcIL9+hLMJ!q;|BE&)WYo|)gD?ubE z{H@S@;mW#n(bOmVckRAKanciJ@S>7CGLa?K(D^wyd5<72_>+El^hnuPoz9$>Nh;gP z`}sFS&Z>mJ=xYXAwnq^#DgOfKd3XRRPs%B_TM5x|dE0ZuR-bHESnO>@8#tSi)oN{S z*kyC^7WZo|FE49r>YC$iFng@yoyzpK-WzWPq7}7KZQ*&ABI#r>{iY&HlBh4;O&@{{ zk{+<&+kE?3n$eIuZ{y`cS3P|z20?qPNN6y7`kf~;Wsk400YoTZ4lcfr2hUA=l+f8| z6v$d`1>Yw$Xqo*qQKNJ_G0BOeLVlxUAX(_(&C$hDmdQ%D9d8%un{yCE1Q~B7ZcOU7 zByjZ8hWEmG>-oe9se$ZG`U8jCH$mU|bZ>JIIBNhqAIVtS4DNyMZo@)*4fR(ibq14R z3b8gN(i7%K8InaAG*N=fi$qjv&loYOO|cr|AR4qj{47?=)SSCBT@j(7W2~HOeTLPv z@J(3%5t%zuIKoV3H>tjm=>)6itbFiO@YHhBmkI2B+K*0#9|pnRO-qUAqivo=(62Rl z?|LiV_l(!?;2pSq)FtGT{UvHIi{v%HHxymo?G7SG)>a_)RXLKzo>bMhrrgHKwJgoC zGPq@uRYBxPjFXY;_L&t2m642MYX3d-&K~1txCZy?J#+apjsCq za_lBT+jRak_Qz-YF&o`jA1Qk7l%*Vb9J<*XR$u!=B3KKIUK?w@qUxN47oU!x{iwd0KhrZGjtJr zUmH8H17#vknpkKhV{d#bzG<}B zV_P;P)g3G>9Pnrd-=&R0N-lp_riQDo@Y^zfr1E4Lh|IcDu~h#lS6vmm=Bsm)N~(hX zhbD@u`F=8T%0xG$@v&%L(pEOKK4L6Dq%F*dDE*+?@QvYqZoF7{ifP;@+Z?~HO}T09 zjjcTt{1+CTwJbAnNr`S#tez3TIKhW%d~CXW+3KfV4=H4+*oT~o^!lIF9@nc6h-vdZ zvvw4V&1zdsgV6c5rd?#**lDGOC3f2rA6yPVK#^@uX%BwupX}NcgUW@QnsAScZ6^XAOY| zmQw<8lQx}e?9u)L*e^vlwN6wGFU8z%da#z~F|AI>FX-P)BgME>y$qaM5ADuP*JPZe zYV^Oek>E;%aCnD*v3Zyrlc=wYqziV71bbL%IT?M`HPHEL=#zS1G)i3P$4kzp*CLjK z)WX^7?Eo-t(_J3Cb`Ztos0`^PHQ2k0Hc(;84N3*J7~sp9%bm)KRgW{pmpyho5*&12 zeb~O1=k-1V2g%WBQSoUAJ|c>x6QvmA94)Z5h2L9I<9WeRC$ztCzh0AM^u_}QDWYA= zEHb$+D0wsC>JpU$DOj;9)jcwI(f-(uuk?0ZT>`kvAx2cwt}F1J%W}mgS`rx#B@*=K zLTW`^GATdX&1aEV*;Bbs^(>NLj1QzN={MHmB~6XfuYx>p*8r=d$&5+2%i*7zWGY8K za`oBoFZhnt4fP&4)s;(IwKAeTuw&j4U)mQWbwA(IIj<2$SWy`=A6(g_^ktJ%VAjP3X0)6#3jZ<#cVP@_PC8ssC>c^V8vP*O=Z@zO)|j0P6S4o%WiVd5S4>)g zB`h@L`$D>mBDSrW$qg#nqoK(>3DvrBZwEzb82Z+W-9=wre4dh!mVT_wb8G4}|Mi(Z^7D)%IAr+oI zfb!q3fehqWz=R#|`|C9dc0ogZnUZc`wQ%fsRpMs{zJJr3?Y+(@$-r@(F;8Mi)H#{H zDRh+lBWz3*_TmwpS*I+PL=YJnn z_xo6nKL|LHQFElFE$uY?CI5m`L%aB$lFYNAP?f+FMnv_QMXV;c;2hqC{j2 z)APVw$YTK<<>-lvyaMRhwDB427a-d3liwYvmDS4e3EjnM+nVBOQu;4|%nRs>&84V& z1R(FvuLb_uO=e&`xXFKZvB;m@MB6g!>Va27`p*V-Wi{QoUD}KXJiq6 z(q0j$a%PAF(j!o0g}^3mjJ6^dJ*QVUQCe;%z~*f^j2sh(F&g%eDBXJ94P3XGXckAb z5u=AxFTWx~D&CgqI9Ogzm^!fxOHLgMpi`3B>qq4Q4iiSatH9Vc@=G@KS#KN3cjdQ% z-5|XBBf-+L7Zm-Wm--9)?klv1?-ObgD;ge|^9u%nq%rqV+FejG$jj!LY~*M(;=|cI zskOKF9i3zzia_4mkeG;l?5m9iV1!g+=&&&$WfWZ>r5s-9s0_DOcGte!D?Fe zB5i|oJcCfwLlspX=$3A=)s+frdE-;8+gmyI7JvHXqCAtI<}#sfbbm!})b@3zgx6K;sQo&Lq!&#k^d?`|QnCA?$AkJuIgdooL3e`SDJAVP} zp`n9mDsHN`o8X9(3o{dbaN{0{xI(vZ#`P?6?N9Y14hWg9F;YD_C{}aX%1g7fo79cE z&f|+L(?XH@WUJ$<+@(!_KDVgaeB3@vo8 zPzy1Xe@bwdyCcZxH|W`WV@CuUW#v&K_ZDGD;6OMJH5n^e94a;1MjGgr&>6qIBVDa6 zT$%l1^T+CkfOjPw1L`>YBH9_sC=a-l)>)l1dhUsm$o%B&j$D!cwE%>{UWC^1y?8OQ z8QYtI{(9I#hySX~by;&a#2A%TBcDVZvGrNqy9NF(W(qdp%200hgDq^|Mq`9^Wn<`L z1|6&f(XFY%-`^gqFQ)|`%ON4Pj@Gt%Ds_0)&I3R>cJhNjbcWOToGHnK!Q*6#2l01a z2@8KL<|PTZzH>e62G6Hv4MchG{(ZPb8{C1YE#xGV+J^O>_@~}E#Tb~f%s9KQg@An- z|1=;rF2E&l{~fsv_8aZL0Kirtc#_3RU?a0LH94f9u5LzT-;++Qu__+vZ?{R{!k{~V z%dKec(ismfSDMC{!dYl-ntV(RDk_rIu*$3NGd6iruUohwHfLW z3kyw$0pu0h*eW;L_pp~yO5#nujjZd>;Kp`)^t#pFm6@fIB}^DjW@?Etf)EOOmX%?0x4I`}-~P9eCj zw@wU1HB}vu`5AglQUP_D;%FBZZyT)R}Q3T6l+E5&HY!?XFQgJz{YO13u!GvRLa+9}7 z&PODIflOA&PbMtlgp?RT?$@kQH1-Mow!6`yyil(@VZ`pOCu#RY4;PUVs&{X&G%kdl zkewQ5Zo=sHXN(UTYO^LtjOLY{wu8udBBY=D+?6X>>X7-$9WaYkpx2~NPW_;24*ethQ?vf0QYt}*47M(m@l*c9<|aR1u7v1^fhFWH?LSd_I0 zD+sA^yndHLWkbqvjg+^e(RpR(Nt05CX}D0!7zA%#k>eNOC9lV`>5`hr>!Q*UFGOoP zLljv0zTZMBn)U#UK1CdthXW}DT3NO!cPYRcI;dA>ZE37AN#z{ZBXkCCHP+WLf%^t^ zwrT)g{4~dzU2QPqj!Ow-wXf8yNVH;m27O}yQ>3a!cf8f9Rr+(pEs>t z(oZI95e=7Rt5Un!EbF=A=d_!EkwfB_0;KI6-$X6SSD4`ieUR%i=CNFKpwI)hh+|*fA88O>L0%V$Nflj zIkvnL6xp?TG^ECuUJk3FH`<&Pd?+=r8f)cgVz4pcQKsum08i-8B2DG}fIN(DW(GaC)ke;_Kx&$-}V^Uy3d zJ1+E07Nhpyn9pPQ$93*3|xoyJ6nD zd#`y<1|t9%2NYc=R*D%SH0Ac-lFqsgL^Tb%CyS7T-iQQ#{~!fQaxUrBXFqxr-&k=3 zv^Jk16OX#EJLd=cu0o!p8&G(Uk#2|nQZxwN??F={=jZJC%AzChsW|t?D_-Z*B&HDT z;oZiMeOO*wm_`_6XuzRc&X;Am?IEq_rQ53rSJ#O4!bdDLa|(+*oSy(U4%bubj16j~o@oUd_P#?uJ)J*jgVSHm&#+H(HN3e<uVpqNUdJX?Q$=sd~HO>z;q{nJ)DS}k7T@Dc%AmH$3(wA}|e@%00-K=gJv`}H! zpK>o57x!@i8O28ui*kq<`VK;nbo&>es^pReS=twJvVxY!$XS&vbG}cS)L$~RLVX~} zg5&Y}6uhT8qNiwH*WS^@K8O9B7sEN{I>vwzHnfpL(T0|Rsff`hINYIjt_#<-%dlb<_f5}kHnZ4pWc6&OMpE@9 zlegv5E-Qgd-Z-kwsa^-sm;_41Rqz zi;zKhje4!}-9}HPUb5GO0a+ptfX((H0Bi|XIjuvdd3wTC!t5~mra4R7snrsmHhi!y zK97nPSXqC*hdC!hpz>VmIY9*AN9bu9^0K8I3s@+)?3}W91MVFHziHPQTLf<&@xM84 z9@zLw)<4DWQcG=SHWzj|?TeKm43qL$J0BsGZJW+&Myr0d`l4)S!#BYHJ$Yx_HY!5y z+0gOzzG1~%CT(&&S>&TiZEpn$`%v+;K(f+bfC_fB;pshS6?6hxjfFeB`60&C#uJxt zz7%vE@uR1e7W23&5Pq4rwvT6&_OuC&)e2Thx@1)K^;YRu`Elx^w|K`}F-}+LjwZ96 z1rGS42wTptS`7`cZ)QLSb}Uo3h-cU*=D`;Br={9zdW-0ZnU^&NSkZIW-6aK?sd%Tq zefp1bfIR>4SSqwHbv7`|+6hzz*EvyW>FjgbkiYGY$3(Wl?YJiylivQlg;6x}S>2OL zLs9p2{rqwEfl-h;fsfP5aq*EH$pU0r`)fzCBO!?dWk!(K^m~AIZ(FtDWhvYt`WFD- z_qH6o)S+8KF?1}f2Q?wG5dBmKG-u9yFImOSBztl7Eem;ec_r3Q45@`J6Qhez9?#KL zsAM&yLP4&)H(dz7t0C)=5N&axY}Ss7kDKHIZm+mg!)*o0uVjz%2q)EtJN6jd`O2)+ zfXKx{JTwev+@PVP2pOEEd9?5^0D><3WLKK@w100>Br;lhB*JiE-*R8hApa!%3(ZHK zt%wV=PktcXizEzZA^NUhL@G1v35@H#l-uW#+=T%={jSp+Sv7G+GrF43Nbe+AsU=AI zZAbh}{q1TzUK;96U95c4=B~1#HLuI2Xw%(>&cGy1M_Kl%viq@eLlCJM5j!{T84)$I zzT8KI$y7hp#(2}1o4!I zuOp>5qLRG6GY(87eX}|@Vgi>z9hruiZoOjY8w~uYd9LQ~wH3N|N!Wp$>zo8T=r$n9 z+VZXE-s#p=|J}u1SIDg~LYDKzf-}kV53dVzX1cVC)QlRHfoodH@No_z> zGg5@7#e!5jb&L$H!y5KMLJ*odSj(@B5MZ+>r-BI#4!UKTq>e4y(`e3askHwNe#|iw3+! z)SLROw{wlg&X?X6jjga$P`t@+HlHv&(q z^cIZAu(M>L-nB?QytJLl*7Y$vdCUFO^R3h8$Y&XN=;y9u>>5?IEiLenQH$OJA}pJHYIG33I7$!SY_f7KJ|;={9!dmk-wU#<_LT>i;-k4$6+=8IMKVw z{xMi``W?CH6bL=00l)9S6-hHv7T?IHO*pLYrYRSKM>6OKP%AJn1qEg$(c7~NJ&$A) z*L;2b!B%z=51<4{BPZGyA{y!KYGs}@3V%UOYhe-X2Ov}C&MMu(zC75LJnGaRdH>XB zLI6YKII#^gC@Na23t2Im{_Y*;z}Q1&b!2yxx|*z*I8pl=m+oO1A=YJ(Vl?tGG-y*# zgdJY@Pwp^`kPqU+zMc$ePuYTQk2m=Ib<3ouPeT60wWKwZD0ruE>P6@$KN(ht8`5QM zt~*^2-mRw=92$6N3n4EoE69O$@VBP8#_iBflG1EpBjzXPwn}6j0vk9v+!hFl!)Qi> z3@!;^Hxw5mgCa{PHU>YI7A7TEM~2N*r{qNF(p%Bdu57O~$ZE08dAPytJ}UnH;KhyBF4K(*})&scqATn9OCzZdo)+`~~Y}tNg`hk#aKY z6=X!unhdT@2HQkwenZ;1OV>-ai=wQ>&pa_Y@ETU{-1bC^$B*}i?ySa^VV&`yekKV< z6D8P=R|IZ*f+CD^T_FY^V{ySsCrv<^?kTkM?B?RgmVOXb|48_Cl2+3KKCyk8Zr^DM znQkbpDvAt92$7uTD--xMQ6_Rf53QQt*cz}*A`syMWuYTc2eZ@H$dVfjo<8CWTz zPPnBO-O6Y^Cvy{O*+#a#Oi_Ps+#P$PTZI2PFRpC{-1Yo}z?*oKw%L8<``LGgiMKpN zN3A?SoallzW4s<8rZ%t9Wtvd&mxwZT_pP=gCft(wj7g)QEF>;R66X^zr@iSUo^?jzWc1(zhg){IbPJ zSJ*Yrhkb*9u`WetI@c!jjOqEb3snYyEw!%WrSY{#U;D z`yjnPb}Q;Xs_A?_(^l7QpQ9tKp~Lf!S6&4WawNP|712UT9e+_8ryBMG@Cv}_TGK^& zYwi|Ze&X86`UPMjKcvKR)Wt|_`Z0^z`uX-a-`@rP0vIKqYhQjmkU3nzey*|1B1xK= z!13g_r=*MT1p!W06ZFEi+4$n#4YpG#J|>rYULyQFS`4}gyt8%r;h{8ik^AxosLzjd z@GnX~e>1iJ-{C$FOW$6Ck}oI?Ez<-yFk>*o516-Hjg&BGOv#GN zF>N1x_)^ifh@iT;{}-THMxs5ncMy&liUe=iro z_|fk>xf~DnvhL#>03P-;aM$ly7t0n(y!UymphbTx_X!1bmBxh0#{0F_GG`d;U^X;S z^$`Wyb&!vhfkW8yrZQDhuY1mGGG4C=5t;=%N0+lQ3mNo}(nbK!r-*;ZpUs*~^iSh6 z!eh}my`N(kAHnm$AF2I+iYUpi!1@1`b?`rR?Prfw;9tGv$uEv%IukA_zqM_>mpQ{0j!9ryqXEAc6gm&yzTceoO##*176(I;iK%MPI1TtcY< z(jb_WQxj{AD}aCy*N^sJYZU)rzyE5I^pDU}@OO>&{!>=|e=SPQDXnZvUolUckuk+~ z%wCW5l}ZohZj~+#R+YS5j|<0r7ld<6TEH1BHiZasKNY9inT+0z|G4kxm zjlqG9LzPY_`L$hAzq<;{wd?VSpI!5O$Uea~u!u3@RzSi5yGLfQKh~=W?*$bHE zJ9!ycqL#dyoBM-T7B2PN2c1RyqmR~Zu_ntqwX&c}lfxV~C!?b`lf9|$INojvdvf$> z@0C)$1$}?VwGbm=0-MVJy<&oYcu?fuX3qaTiPPWuY`jqc2tI9TF}U=kC4Re=Ent-6 zl?kVuZ5ib zgdE2bJluzbqTe~GaXQiDU0&s(dndfPRJk0hBfj~hHfzg~4jG+R$9oPopW{9Qm!v#@ zyuPJQ`MTS}Wf||&?dICHgq6kT?${B(q0(@Ql0|Z`^v`ydorm=oGl=a5F5YGRvHrLB z5uuFh8*aI-&kVT5-SBR^*!HlOO3Dkko7&H?%sq9=UG(*o0QSxb=7-OAKD;k>`jM|2 zx3opd<)bs(u04A6^p>MWz3J1rp|cClU+&wfws+gWMijXb4AO_cS3S7r=rM}_p&Jd9 zMBzZ`#h-LWhk-|vC(afm{-iTH3_O~2$V@t;!@#3Shs>liIL2S6z0FtMqLak%?pX2L zaQkg}`*+Qs_@6=Q#G(mO{~2zZ$KUk_4np?I!&xbT4x|q@qlsZOe~gw4y`$M+ kG#iX&gVAh2T5Cy6FE3@+vGQTY(nJ zm*+n3{c+##Uox3FGuda(K5MPLcATcV0zM8U4hjkizLKJ>HVO*HHxv|fAP^IIWT40) z0|kW+MM+j#Hvsiy0IQH@&+72>S&XmxZ28NF(_S*A?horAIXWv`g<_z35_}aJ$DBrp4rLY(c@^B_N2shVEnAhxT4Fw{r&#Nxth4W9)Dg^g6IZu|mCJ zm%jA?90T_Sr(u)j|NbDXh8R3YTi~C&WvhauX`>PUuhX_7l<5F$@$fq5n!5P^jPdAS z_R}nYnAfXwN9$A_{5^UTp~k>-0}Nu46^6!y27wl4AD|0O7gklrqFbJ^H4Qf8qVt(y zvdaJK?ootLdH@Wlt0%YMDojnluOq-8h=j-jnw zk=bLwL<|5z?&(0Bshp`mE)06+z9cYCx)3o?$~%8|8~8%eoE22af*KdKp``6S9f<;w z2CkEcBs4O&y3(vEnA>)izvOe?OkeJ_ay+`*2-K5U@0A0O3ZxGBg zIy0Rw5~K*}qOQo~{Lgn;22jripjrv}L|hmHCQ2`WA!1{d^!KMs&uc(PT8}y~6}Zl! zt&DtTR8%+y{8>HSCtI`Y(wwsR`Q-v*5nFTy*aw|ww_Fj64y4HB05oUBgC#;=QDVo2 zFk!+jO7|0$-n-s@4A|BYc`Wc?REr=K{hu(4LYS|=1P+#hUQFkK;F-I&kKricQYin= z(zrKcjqE@2^^VRYmY|dq!dvGSEWkHgOw;!M!apSsgNeh~OgYsSVpCFBB_xurT6pId z7CeQE4nI%deB~#0K8h&QR2(qh!p3%*Bl;60hj3lCUdEp%4S$ckw9N z&@R{lgcy@mbE(_~FApagWD3+%=4B+($9&qInqp8R0kMT~v*h)D>X<9}L=Mpf(4k_p z4G!i0VhFI7uC`~F5>q@wlL&MClx#lx-247-%sKHUYV-qXNmjC`0}rW#mMp6Z zLwF2;1Jwn7upB*c+{H8i`)F9#{qbG$SwQO8fS{>uhO>E`3MWj65eR^Jrf=dwm|a!*r^Ffl4OM~xN;H_6Y*BL$b8JXZES30 z8Hm;;amJinYL!afUpk8M6*9^(zuUn#?FD}JJFT?x#fk8*cA_j0vKVc)85Qw+6#jn_ zW;41`=B ziY}_FdrqzLQ-nyeR6~Hl(lj#uP_s5{vo{PV5P&uu#1b@47hOtn_6`-I1~^b>&^&#) zt=x#n`hh?JrWbILd6%1;G%m)EPuxA4DK z&PPb0(96soEiOsgR9pR+E^P4ZQe>*5A@-ITvQu{T!*`OhgxS<-(4X*ifeV$Xm6C%Q zf`D>toT=%H*qV`2%5j;6atv}{4_g4x`}OwDPP5oaB3{acy6ZG~3%d~S?B|@LErLSc zB(UHn9fgj(RwO)=7$Ty?fV^~U8C)91pH@YuR6j@;Yrs#>a}VXa0gn1Vk$v!Z;LyHu zAwO?8uwFj(6MsoKU>Rl!9jdJG!X6$T@xEg{hoNmeg>LLTY3i6DCU5wAd*2~f-k!dZ z(Lj}pP+Cz@Qe9nc)xiBdV5PlxC=H>Is;zBckg#!H-P;?1YekN^%EWqGM$qgVvYh)UFx8`b8 z6j2q{_^kBrl$D!Dc$#Wg|C{Fwt4G=C-;IO0FGTC=!G1I!va4*+2V38rtgXOO`E<3t zRg6rH!c3wrPM~TEDbQa#0%MhLIJuP-bowQ~O%7y-JQA7jOj``->D6*V9Yh1%Ey()R zT*tiix#pcjeIuKpw1)Ba<-w=&rihRHZ}5rsuf~nq)2vs(qf`aCZKR%4pT*@xlV|6J zCMs<;#!CIXn1dF%C^2Pv3|13fwDuUzhTM?bgqV}6Jj7Rmx#DFm*qxx?S_&B7p}uhB zBFy4wq#H$_|8^vCkM=R6CygUP?*oUD*9x7m@;ez^5+C|lqoH)>@4HMQfo8eWYhp(i zl$^7!dG);w(S|P~kCu8f)Efs1D)k%680_Q!M3_Yis{|8#a2Z$ny(Cpq{dDLRB=5iz zXr>y>!|@}B7+tpK>11t9HEe-c`A16we>le6tS~+|jdi?`_nvM`1216=3lOivyQVAW z0q9|wU?KZGz0!85~Sq^I{{@Fo9ymdW4Bn-lo7= zCg%%oGv#=a$yv@*HT;{zs6aLA*Hq3baXwBU>YdFk`w<6NVY*>|^iSe#Q%VO}bRGV4 z$$rgLA@2jP;a}n-E5}Y$IN_R#<7TSK@6vxy(-YSjG_%bPG#68>*E>!LRQw4N4(TD2 zq@`BMaxu&XS3Ni~BCh`j=H)19YZG^@2l0)KjTO>Y%C;^ao}JNyaW2hOl$5O2K7Z<5 zAfRzl(?cmMZJlw`ktuefiKl-vuqcgjXrO34g-W1f%vm#Q?8n|u?ql$tjn zaTlBpe-!`sB>83H@Q|ELMHw4s@`XIbPjco`HE9NVcI>MB!cKNLB~gu5xC`I~P5TbT z94I|u+grq_%Nir_A&`z4*q>R*k_Jni+ZfTC>TYt_XMDcLtx0)~%!=`Aw&xlzr^ix^ z1H*loQF5F#Pp{Jt?{*Bp6(BlkoN(ge>h})Zzs5B{-pr+nB(*ij(KL5jnkutN&Q_vR z3vIp60Jy3;(kZe$z9ivNXvA@{mKtpR6eUSVr(0913rv(r-VkhhMa9_ax;nIu34FAQ zOkY1gKH&f~1hkV{kR_&*l~t=SA_Mxc9VOLR8OY^ATbS-K5>@M z{gDjt^|!%+z|Vw|8s>YXlgYjy>`CuZS3#k zLRu?l=lH@xYG-F>!!N-iDdJabDX)JEhxC>BG@n0M_}DHc0H>?8LiRpBE*K|>to!0_u-?W9~=60#%pSQm&Gp`=FL!!?2W4d-y z?m)AJOhcpBU474#R2&>*Gr-H)#*hII)+Vxso%T;hZPY)NuCb#&9}70`8Iy5)c-W4o zR_xYq{`|muv}!Q3RST^=7WDZ}Id=UvnU7}v>qp~(&!k8ElC2RhUxnLo{P4bJVC@Sk zrJ;$c_;u@Ql&-$W(N}865Jy_D=y_a4ZvN8?+&FOj%~$`MUOKN2d1xdd|KBoE{j92a zalc4lj)kfG>j#3UGn+6<8E0dhek5ktBbK(hmP>1Vd+KkkyY z-&%O>63x1kQQe(8T{HAOBnp4qa`+;D z(N+5jd~m+@RGUUGDpaev)a96>y>#I-g=YEUn|@ueiZ7J7CA`(e>cV6fq5UPqW_@hNeN zwm1JYrT42QUA!l2dO^;_*Y%iM+I=tte~^A%mcGLtd)Np*Cu5~f#E!ij$b_{IjjL{H zN&VVvzpkoeI{B-PQ=Z@xXg6Z`Wx-_GF3K9^f5GUW-LCg%&n{C=C>}b5IlfF&&nW!v z^YKMcZ%;TV5-x3QY?4<{u%k@^)6`7W@bhDF??|8#^D8?RJ(z&vG0Xw;tGwvz*f~1S zzr3ZOzPtJ^gSb!NdoU_x8sjpAR@dBX2KMM&ow5>KNjCa=A)@i|@bRyEpC3`RmAT?^ z7VNa58^6U3AxUB?3;E!XFYFs?DR1~ua{AvM_!iDI0{DASR+>2+(u=dVE}&?G;2`c??ZYFV3qwFE7u|-8~t} zi6cR#oET$&Dxf9Yzjt({O@jFD9cE4Z%WC|Xh*O7v5yuO#p=G=;KVB*<>2~UZgk%V7t_)WWmR4A3K|R%vjJj#?)Uyk9IWcct~e$~5(a|bE^;vw&pE9wCfDJp z>ZBL-#pZu!zVXBJdDJBI>yAc(cP3OI7cTu*HnUN~4IcP)#?UyswJm+L)q6HHY#@C= zk*u;hPbGi^d15#c2BSi>^n|djAw*TPc?WLTq*Gb3h^h7^KH9!$q~j^iY(5L(%3L_I z^?D=l=FKN2R@TdJ85%@*m|eR3T;aRBwrAbv=K2lBV;LH+hs#sM$NyFy1&~xZY3u4n zg5cBBTzF;&+TOy{s{soN4@;x&HjC3{;*S^HdHMPKiJ6(;*H>xBT|3h$^vs=WT*y!Z zAxeLe>!dBTgaMQgd=KuHynI@@p#b>dm8@reHl)AzJ0~j(FVsk3wRITs_q*?6uGI= zmh3iP$^ZM;8ah*CWO9#fx9WLU1h9CZw0L{w@_5{5d-Z#}#w1@gVT`5=!@x}b? zUh!p~$f@eB0-LFxwD0@0hINw3UpQt8De0kaNHLV2Ca0Ibva!n@zspNrc7pxkpL%Ds z+xW43_J?{*p0b`6L3@|{uea~QQgR6Qulog?FA}>>wZ%( z=QJXh+P2Aqc(q;&Q+WJOC-(A}Ic=hjot8%^Ds*ju%aEZY#S! zRK>LZb#KJ3b*F@U^x*o^z8&ryFAhh%r6Ru!pKmV69=_AgEmAm_B0+T^R&mZ@2)b)J zT>@H>1e=ZK?($bJL?fJ_Vp|C0rI^l~dT+9r3f?_BTM(kYcbV~vQLOD3o)os4&pBq_uXL0Ifej*OUEPt{UwF^%x}HMhKH-c|H9SH#Yx+ zfnUy2-F8}DY>-5~A4O!oSAMxfu6L}hr0VBANIR5aE~}=8Nh8+2!23D|yG>~N!v1#! zXB|h2W*HHa2ZQdm4-RBn9j1)r`04sOO8LAq(-EV2IB?;%f`92^Y;5dR9%V`Qb$b=} zG0Q)e2igc9Up0C%`19#enBmQ+rncAQ1}_3Nd{wuvqa1%7()W2XzF}66PL{t6pssO_ z&EuhcP3xhOG*-sRn)mR<%{PDRc6LaZn3;82eIYTFl)@I;$~QI1M^G(r)U{qmJ;4J1!@xp!;Jm(4 z3tz&!ICu(^iZ6gqdbW%u&pqRbH!t-5vUQg;-sUO#_Hi>t61IrXCC@fs+c4YDLvR)w z?}VfJMVk3dclJ#M2>5>XC4$x= zo2Q5xfu;j@QkI$*ExmF~!A{Qr(23B(ywfei&7$xtF{8#ra%t(o9n-yCsD=6c?SIs%3tp?Vxe*2(b1|{xvM`k6$+5{Qh;}lF zj3d+MODaDbx=!#a7i(ReRcBR`+=5c#*yxcTDCrlqxOe0&hl`@yjKk|w@(eTeR8urG zkm)-FZ;ww1!Kss6>oT-V>xV^m$x$sS8s4B#&+f z%v{{tq+Ex@%a^xxbhN&2YC=Ik4^NNe7T<7XA-O@j>+)b}M9lpbVW=un@^sC}$Y^VC zXNc_I^Y(5o3(EBQ@v6=0Ow-TJUvLHhFTf_?$&u1H5+~JG$f0oc!y8@ZNv~OHYA=49I_367S>3!df0?EH*k9*D44+kY@ zmO)%Y!^&M(I>CQ-4ASZTl_1K_L~k6Hre|5YPF}m@OMrh!sD0KshWQ)*9PLS z9{?fU3(y6swp~2(Y0UFanp7O*cwe8=+N6ofKl?HSVqc42erx4_Lm+KUK;_Yycbg~u z6Dvo{m3Bwr)@CwB*eM;|#x0S8$|z6;L-L`cT>K`fT#A~i;owyv#qhUXR=)54k<&!s z3{uTv&DZP;0D2HGk|U=K5JBR`IC)%X%X?5@Bf43}FPV%`h+uMb*N<(6)F$PWnw znu9mwO(j>f10$!hsyw#(?DFcgE+Bm(Tc_yEKHLd#=p;O(2`V(QOJYjM|{hUJMAU+9kRuz}C(;J#@_w4k4p@ViAs9QM-^{YsVgH~5$H@@ec z8b;H(TUeXz_XK*{HP;Dd9BmP)YKR(*xdXIVf<7NnQnSL7GACimSEs(TCG;H)spKzY z4)OoD0Q9LesoQkk8|UR1v5#p_@rj4PQ}fFB<0q48+cn{ zEc0L#DiDx~O$gHb)GM6F7vznQ{^ynGHaEKL;+)v|yr}WAAxcv+~x-$g#cR%bdaA>X}y% z5UUNgSNJpm;cfAIyJ9`Q?Y>Z&EwH~gW@Y6A;3we%4-J}zs0s&;lBbJQZdyCR=TUnQ zPaHVQU^DNGK38A0j&kG!E;WPH%0Cu+ktvD{<%fCLquHtI@V?GV+S1nEC~V31!b&sT z<-)8lNtHb9Bd33G*|)jsFWN!>JN3Y%iW3}m-ZE;BUn5kpFje*}J2~NfW06&fnEw_- zFXWMEleIVitxrG5^!L&KwK z`RU5~`Ye6_ZhbgS^hf{T9|NOc_rb5x22chpV;>#@r=zS66`jyIs)8l28jG1{JK;hJdVfH8MKLlQVtRn$bU`MuoR%gjhLbWVtcv0HiTt}QNRybRD$V6isKRrMhqdWjtX~s^ zh?&0lne&~%z9e*1DqN-mCFJR#O>e7}7K~RMxIXrnzC{@MWtx;E36#}DJpE+Zxh!lj zVkidPf2xDiF|yvq-@@K7i@M-2i>>I16U3E;pS<*f;dOoLJL zV8dc^=9L$+Cf>NoA0#6vu1aaw;+sT3JlMHkjHrbk2`^?W#eWmZ-a;2EqL=yhEQRaQ z;0g4V0Mr3XphLP834v|_J{TiLjhPtRr)i4~d2}riO`VHbMrgL21-Bw|=~OVDqea+w zL&@DrbSz;zujqHF6MK%yNf}zAV{HE_Ks_%)h6!1-jn(i?H35$vhNb+EgNvvnt* zdnYe^F%NQ!VU1PuOf5c;W`m!=gOhyGAhP0-}m1D^8JW)ZewgkI`{+x;Jx<*@it=QCr*s~m*ju_zme=Q zZP7ul*$dYN$@=e^X>y!9piN&5A&lH8r*#lEws0)GQsmrd^0?=XQZ-&QwDTWArw)afQ z+HjJAo#(|se^5^|NIAuViTEpRK9j+vkw* zED=$Q$KS1mQ(JL68PC@R!9le}&rcc~?-!{?*Pd@kmCe>3m-|%H%`d(F{o;0Ke4Glk zJ^w5gw@^(UdmrchbLmig0aYp37~zy&bq52E zsnZ2Vk%=y^C(Me~smo#t>Prz$b1m4OEuq+5$9B>6_Cir+z46368ri8BRW1tghs9CzUSt;5P`&U*e(*Hu#fY0s5w0>GyLszB41D=z53e%!brJFm*Eup)W^;bCI7Z*^=SHR{PnTR}NJrSC`lq!I)vL`o<`lOG;4 za&;%(UjW{%6{)KKog?ODYcPiKdq(c30aJ&r)IN;)JUG>RumZs!yOXDyy-q%C%%p0h z8kwskRVR*32mSu!Pn4sjJT>y+;v^th;>pJpT8&<<#&~jjy%LE<=~TeY(ovJ%P>PtW zTGX;C*$J=^dX3(afYXGmpuaMd{~qmnf^#2}zq~j*=?kLv!|z_RU1bf#Mf5}Qq&(X_ z6%`hAb(8!n>Y8n<90Ab`vgJMO{fUyXN3o1Qilly(`6gbPfjvGKjmt)1w61YqKbC5( zGgO7$esWSJ_^}{Xi~chHdgr3ZQ~UEiqiNQeG{}Bakuhck3)|~w*)Phv7jiYBRarx% zwVT`c+GLFt{jaAuMn%UaCh4>Nx8hiNek6rhCj%RCRi_!}3muh~E4nOMb{U603=+Yv zuWOu|fxgi(-4quet?b^gqj!&v%}g+=(d;g}OX8YxVZ{><93Px^5q}*Y8x>iUzDo=* zo#b0xOcMH(pV%gyPuRX$(*-{aU`;wjCm#?g!(?@`-M7ogCNJGu)?(mPw7T6F;Ixir zOfriMYe`!4F$?B=drSoso;T9ZND8$MJ1PZPvI>+Zer8TsS_}c3z$iZ*H&n6tCYrtC ztJuwIKKY3s%W6@vNTK4sY^}V;?91Qt^>T~t_oK0(UJl8rTbWg&j`ciQYY%%Ix(rQc ztFmDsjb%$85)3dIxY_D-Df?V|u{=_KKNe~kvA^=pC`qg{*1qHsT~(g7XLT6k%_fV+ zCP(Cq;KzSEAkYc6ys4NaZ=V4@lZ&n-U|O4T9V>sj%aY>qb|s`)8LObJE9!RxAbKh z%oS4HxDYOTkaO>|g9wmfxV}3R@m<@5DKSN_*}mRj0y~5uD`+SKRElx-HRN@1-};+W zm~ffU1|C4LuwAf@a%83@PFu1~Y}N-l^2kK$%;?@+9UXF$mr|LHFq-q7qoP~!rC{<# z`c86q1|B*Z>dl#Z@bi62E;>4!lp*lZrGe#`BUI&}ZG6TYn?wK*p9)YFg8`9N8xajS zMiqeG%BaCsqk4R{e)F?)Mw@WPK>=wHE46RZ2d1uS^5RRK?3}nNC&zqzIiQp$eMKfx z4ICd=QBqfjExtjz0BS#etVHS^3rMe3dP#|fwKYQw0SyB+L3te|S9Qi^9QMd4YbOXT zP9Z~g=+_cB^$VoTt$U8DM4H1qs^yQmthV6(B-Gx55y{^pi3T!LxA-Bka})%%be)b> zRt!;bFXw?ga1QwaJAe8(OC5n|g+F;Xzln-l!*}OFJ@P;hFtJ8j-<;hGn@9XvR-$90 zK-TFE*WY$pp!_+?|U@kjM zLEq?Jt;htr;hY&#{p(`GhNz#Ylj`Z@l4C=8g*Ak0b~5vTU?z`IHQ3fDq*34L=RfG{ zl9Z0h8IXX2^lhrHY0Q2cLa&Lqqc_c729aT|?yibS%x2`z{jWI2aVa!&av0-&#r%D> zmd$XrP=>8eK#^35`~Iv-U%v}^Plo4g!W2V?onqp>N9tBgnirNlZmnqlT=f8KI zmsM1kLA^*sb`cYLW3;aH6dvSvzK$5y*2_9MaZOJr4rE2s)XN%=$H6nJ?&u(dFq6rA z*Nu-)I6OJD4hYB<@)0ensT*I=keFH)6&3B&tl}3T^J;EJO1wUg0VFy)3qA5-AhZFe zDyR9iwW&A~)|!tWt^Ov{6p9HY@Otbd+`Y0pPU%bZyOz8p+6?D@=2IC0NIq{71_WBh zYXYn@#F<9An?4wEDow&$T<*)o4CXd2=tf7a#?ps-4WVX%4o@UxW3{V74x;kcs@F$_%dotxtev&dO;Fct}}$7;h_ zPokhch>HmwNC{cSJA3&hBmnoh`O;52WbP|oW4ucBZf_lhoTL#}vm*4M~ zYGv-<1;F3J8LCB^dJEn=!K+}ptm5Yf0#$MR%xqzqlfOTC2Wz2`8m!C4&04nS$P{Ie zb3%FK3znAizD5fOX|`z(yAfO$yhG%&RAXb+*2-1HYqb;p5iSUANig``4ni5a*Y{&t z8_uuHQdK`=?1zKQ6pSejDVX3e`CG~#KlVKz(=0rL*$3g5W7ToJx3BfN@@$+9(D9K> zfU$$Ks>U%fL<2u?PTJhr)zVimy1laG{rigev+xMjd`j$PJK=*rM+Jq$fXpR+oR)=H z0_qBmLCBY6bl2le{%)*~qZ3J$7`9acd{p~P8D0PUk_Cl$NW%weQH)mw>Q{Oq_hSgW z^KEaIR8G$G9R?0}=-r=A6pT#y28O=)trsh=|8ZeU-Eo!dEcqg$PO+z_U||n85ZOMY z{$^BEBB$+7D3d`lsi1n3niJQoRTZ(Ry!j@+U|hh(P~UOq==>>~NWCC1gbLd~B=TpM zlRbfdGo9$d)H=egFgFfvK+jwqcEy}`a#XSd&$F_x9vQ3eDNZq9@3bybrClDK-YE(| zgt^bYaU^RXby@aR;i=lTu5jX5TVP??jP**(G=PWxVQzojMaUVhGBt4R5Mlq7l@NZ%=JGwF-HF=j6ieR=vODyKWh^(@@G_UHw?^|3xiCm*>2O zSTCih7*irx<>A@Y+x2uj7JRuOAZPv0yP^A^v-RXh*W>)5BE6|U%p!Pa?+OV@V)P2t z7zMkNKd|pOt~}wHwKng16M!+~fM^S?J@U#G?v~!^b@Zwk?>&ka6-k`P3t4y$Vnlp< z3O*n;y2}(?>U&4=c^K5Tr5`xWCy+IMwQ_FRwN1B9YI*m=%$HDuXsfOBp6RNk=;lmP zdI`#N3*sZ8zqv8Ksg({=c&CezmIP$q{~1XzE(-)`p>`DbBgxSzLDm)OdCz zv?<(w7b3Q}{w40`k&|DC5MS3HXWoU@L&`Ij6W@%?f(RbZ!qQZwkSWY@*Iw~wZ>pVZ zQN#kvW$j{;ZhM~GXwcJMHP%6moby}J}3yXv1bwJV! zoq=1I^rQ7pS%}kxuxO`a>xoYsY{Hk~Y7!y(tSq1deW)zETrH)!WM&TQtS}{=1SBg! zWSj#*|6gZtR0mE>c}<a3q+^$SA* zckmy1QRDpk2bsQpezpJ9&cdZ|u}@wDhtaE0P=To0@Ju=YTGLp54XrFFl?HwY=YJa$-IQo)$+6n^F7Rg%nROKs69m zkjmMZ^6c;VXkm$hVw`wz=4P^lV;aaPu6b4aSc)CqUdon2!$V{$t}_^^k(y&YFIY7% z$-HFh)Y7XKHoLTJ1r?o*)x`K)u%e}xH0AOODIJ&sk?LYuSuHy*9^ySx2DrNN4to2x zwy|-PU#+gLuBNtj;P}@2WQj+rtd8BUfQBBRtvsER2`X2frxC58lwX-3cjJx#J5IRE zntaA^%C8am9+~Vmd?FUVeM5&*&7jnhM+YENS{d|CC!09YAV8=-55-Y14`)= z!!#wy9`>0Senct!Oe#Mrnq6gBxW#`?#l7SXlBK6i13ZbMT7c!GT;WA+_2iKZY<0>^ zuO<+?v9twh81%LEDpm(N?;Pu}!UtD$W><}v>{-+DO`;|acBY6)(lv)T*|=`fFbSTv z$=jEF;~Ve6tueIr!Qv%XYCF`yCt^3Z4V&YV%l-X`P~{n0hPcJxqF;Df`}Q;${HtDE z(H4IM5k7_6svpX1FL_-Bll)1rA7-h!boF*%NV{Xmd4q=ML6N`b{RPp#qvL0+zKxm6 zGOuv{g6Jxz!_Ud1JEcucAjE8-ShUsbDm9{#F0y~CiG+$O0R-bxKr zaMZFnHjfuLQI3v|yH9kHQd(d<1k!m)(IwJl`u3FN7val_?&=tf`OMrZSv|e!qKNt) zlZOmRGmVXg#_I%3p_0#eIxqkHkc4_(TX-WJBxSq~y!@UHS^>O1M%`Agn*CxmX6(NI zqJ(w(d3&-Xe`xrejxn8UC}`S2QF%o7R`l8{1DL*ol10i<6M5A#%e0d4TQ|w&?rMkgjVVSH*?EM=A@4zwL-Z!GW|9 zi{|Gc$*UypcWG?zf)76YJ$xC>9A>WqTi6a4;ffcF`phAD_1ua@>*-g772ffDE^_d@ zYvHZlV$Fne8uLW@s|iRnSn(i1MwK!ZWT^bwGTe^7jadaPoD%>;V_Mc7^@{soZPc6i z>t@~IEOX@;a^9V<<+_}ma6-ZC7jhS9c;@SU@kJ+*^}W?~$nUi`Vbd}xUN1rU(Pw_B zFJ_o8z`CtF$?tTl-v;i!b%=$ys_5vm%q(!w3p6>QHYZ{4^PP!=CxRsu^>q>1L4&p4r<~<2s}XBc1Zlm))Hn_v z7l+yOM;8@nwalSAnQfrcI)k$E1c;9;Zceto znTiod;B}uy-vF&KZOL0XUmE>roF7pQ1L(USOs{=!qYciDK-c-Ysr|cePmP+byMIT) z1RFEha8Aqkt%8z~R6z84gE?h~N5$NR%w2#aR_N@E_?pJrp8}C47eZs+5uK;j8$a`k zCQH_iwXTF8;cQ>VopI{%+D|u^dc@U6D~}JuhJCtBG0jVGUD}Iuim>-9Fpay~+Vfti zxzxJj^5ln?+7R*%6*35(txpx4d(T}hN&jSp1A5XG{b}Ek=q&Z$*h==IPb~G=K^$Nq z?d~4YW4MLXlW_fhJ<5Tr%GID_?=qTKroS}MrOmMs$Efo?bo5YDck@`NIOketPLT=* z8NutXQDfAVpInyST5{hO(_T7GKwHgTt!{iD6!*veu-=sO&f{my@jy%$xpMd{n6sgx!MOb35Sp9` zVk_g+0ubGDEy&x1K9P|F*~aQedwaIZ$DG6CV~6kKS)AEv zb435HCHqEJHvZ}VK3P^nRvo|9>IiG|R(ICK_P)KFozOK;g*3vJ7 zH|g~`Ek?dPVcmcI_x!1f;_h%?@Xpg(ZR#> zm7;vX$(n4HsoLAZ#E9~R70Ru#zgQ5iJEFX4&&{FK~hj)*^hV0aqzQQ>qW>293~jES`W{4udG!x-eJO@?^GJh-QB zMWU;HOy`CJOt1(3^zwhdPV~IfsaS9_Pqry=%ufbUHfAGt%&IEPlu;QsH+-arR+oL) znkOJDJ&tLk@9s4_2gh>D9t)Rw=U1B)$z&U`9uS14s@lO2uicFg3jK=gA%FqRW2>-{ zz9}Urxh;}DZ*5s+YIEkIP4L#TakGLhGE`Xtc|JFg;JrvycYvy87^nclwc~R##_WMn zRZHz|9PS+jIohV$s&;l5l8;*m)5gK{j?3DOFG*tw8@n5QlGu9_0mdv82V&2^ji&33 z5n`_|t^RFl=wI%|e3{7UyC>XyWqZIKn(y{O?Eawf5jTZE{j%+s&@*MXJHD>)wbi(^ewjI`eii5`EMv$vsb1rQwtg>@*#E|CA;=}GhNCP31kC} zE1uC7T>Q24OhG+mGezs-pA{L{Q zzIH6p!`XpF%PqI*P0@*GUf2-{>yV!xx4aClbY1R_q2Y)JBKxQWJ&O^^)7i!iDWx`@ zcqgvW4m#RtQ^uRytY7V&o3f@g`Ao9;b=Dz9EmcGU3*(jH5Kb-?0-vL8CjV1&&IaUD^{q(dQfz0-aJkszx7ucI$5XdrlW3%sgwcMz;N%gV zDXt2+HL3mBP~p#@&f>s0Ptju3@YQ~-V&M?qj&pfd%14HWXA$SDvlX@SE!#(6S+{}b zVuA7n7EUcEH$%Gd)W2?hl$Y!YPXW03T{}r3Tt&TK`+cd0uXA6Btdt6GqBBgApuO6Z z+1(ayCTH2a3J8d{vJB{5T85Yw6K6Lz(EkXDEk5G+8QfWGOR3+@7NDweqqT=3^z0+g zvI*KhuBjoeoJD2cEd`-Y6inp3JY=C1l0Gu*;*XQ%nUD|}S&301zYQp4GmG#d?&8mW zCA|QtGm%pYO3XfD(?;b&&ndNGY?TOo`p91)KblA%bMO&ysk)fNna;qM*$E$v&Az+nJpl+p|N zH?Dm{1i(z@W{LDbzF>1}m4ey)GH`G1N{SSK{A-0bj_LfzYB6@K6cY zS;M-o%d-Lys2hmQdtBzZz968i$Wj{h1LX%KsW6XKz8hlo2|yZ1QwoMFz-(Yx*pA|_{P9{h&gwStudL69MfZ>H zI0@*SW=4n`TktUTK$5k6M&~=}eMMD}rAhj~WZ>KN)vvpX#FbKxbo}^gBtSLGPX5)G zMQ|8d+7aFp4H3ZdkW!5!UN~R_#k^1Yhbn4(j?L&F$9Go~LuEY0Gdf@5ts60Z@ayFK zOQ==pUV8k}vCV55fIIFPnIknOR$2H~RnKM^{cXoDy|M`T@ zoIjp{X4_dAz2F{#;^{Y-O54dXN619@w)tBdy+{~ncQfYx z^syRvKy$L( zT7dO3066N^Nw8gKBTjDt!? zlr3bkrKv*J|KZ{g^Rs=%1f;ksQ)&7Kn*bian!{eZeeRJR19*6=iyyH`fhNu@Z5xEf z;Q&uLXz{ky!REAp*nPHK-NvE6$ONRtK~4&z_Ft8SEIR}7i6IIInG^m_WNv5^WS!|JMspZ>~MHoA>vAD-Yr= zg6kpkN~p1MlRdoZ$T4h}cWdAr_kFdEEGgJ0Zx+hmFs}^36|P{Jw^cPaWl&w!x_@K_ z46#9m^#oL$8m{I854)+|lTuq<>v~+4O(#fkDs+2lOp{ebBf8tb=uKTPA-wnm-}F-I z_9=~2x#{dMsZKo8&&j-Msv)WnU9On<1So&etSrE$oG@hrYYvZYR)D6Icyb85*u?3o zP&PZ;HK-JMs1%u7-L>32H}h?ioG2!TsdCJBsY2v%c;`ih8ip}ySa%fT3uM5{G1gh#W?jYsbJ~cyzaQjk9fpk&l!#z7FA?zWR7$4iZP0 zji`11)-)BLEJeJUB!PbzUw&$Tr_1tJBxIK%&Gp4FuHr~+<-9Igb{StXI*DzTXnV`3IKFLrG-$98q;dDg-QC^Yy^-MVu1$d84#6Qof&_PWO#%e>5D4!2D(9Ya z-|yb{ALEVp;eB9qS2tC)t9sR5d#yF+TvMa7m>N78{M7xHCAs8>lSsxH_j0u?;SKvnOJyK&=6m)cP z3U$9@5eZLAElLo&_Q_XF4&oF2rm|qjpEO<@L9N3wMuYIG>#9e$mc&ks@C(>D=Ccwi z7gOy?CrOsncPD8W zb;dLA`)8d7k5Md#<)J@b+itBG5*TQR00E+~r6mT4mN>AnnE=)|c1ns~x+E>OOhI)XU4ubF!B+(}4T=?*8DH9uwM^y`0>)o>-~+ zB6l?;9d~=&Lg%Ob0f&a`Lyb&-H5_^+z@)?sk5Y0u>F>1-|+%rO@Ni`Dg7|ieFX->w~{T6K@SNGC^Cjo zS!VI8N^*vGE|h;jIPe$}W`aCT+pnMq>J#q7(Q4pCi(JLsm?)o}Thpj8Ti>BYyBIo(@UoWv+1%z8nh zv}z$5(7kv?7pCaxr_p9P+BrGdv*?$H@TyKbdYeRVI4BW)PG(_+AGfHwP2ur@cFOwe7%2?PlL zfCwqSwx%Zi?OGu3fl_wtnJaD|8=AxN-Y&S5&-0$ET($XhHZdSmi(~Q(n~pDdld>-O z7WV1sDK$0q^g`4d&wHh=!2je)go>)zC{eJJJt5$s`F7y}yCc^@XyPpRmO0^~@%7#6 zTOp*Jo3q&VwV@M)z=)xr;PqRc^oW8D)b(55sePY()PU>xf(?SG9Omw;;Mndm*UZ#dP(?0@=RWC$*i!6e~xMRHXAtI;L(G8q~C0%PU* zsM1EVV=6mI@j?j;90F5@mVE;QCJf-a7!6B`2Xh9u_I_dcx zO^KR~Bz$XEu@W8WKGB;av%3%_dXGw{VU`(k*+RwE1Lr1Y1%?vQ}5u+ zow^0R^klZ9Lyd~-Q61grx3#g*s~o%IfX&_kd8o=q#D3sRZ*T<;~(!z8`skoR63MGQ*1#-eB_%ONJbHS zXQ9u-pf;NA$?b!0mf%5swnQz{il@8)6TLqoO+Y{h@to)2S@3WRq6~YJ`lALCD=L5U z3uKo=G^DAg5NXOGf+MYvXLiSs{n<#V>57A&C@d72khjWA+fI~Z6HVf+MspXkO;L2E zUYJ#V`0N{JGx7CPXg@K)pFYlvu#u=(pz_j;XQ<w@#6q3LJ8j}2L2WAs=Q%FztiJY^h>;gDD3-GFVJ6noNTljnSZCW+^F z=?Sz!mkeD^-GkERYCO)Iz8!s`ASJ`15_yiaEdv-PSqjG#Er$5PI5l%We;K{HcR24D zqwES2EK^x5J!41J_VL*{RcsdUA9R_`g02xveZ4eyd;DaX1kiS=LC|R1jxpRs^KZvtLfEX>Ke-?$~P<6b$Wd(X&;h?A7 z$tbhD$i)BJifL@oHZ_lt_v3@vPNg{q))!{PHd+0zUdZv>Pg2oXU=I|=G1qcP>_wjg zqinEuj3Lnvb&cPpH_ze@gWN^J#uHA0OoZ-dr^ner?fGT;hM-Cg3}b#_O2@&y?MBcy%6-uS&vhqpD*uU-x9fzU=Th8(e>ouWZW5-Z|^T;7#5;2%U*B0U@Y z_;x~DP;UJ?==~Xb*!rE&&uI1gxKc=aV3rgtex4|W(jHd~3G!E^Ammj=1{w@f z@cBEV!c%;vObrwaFc*lNOHHA0=2fHyIeZCtyJs(Npg3G;$t_mU+ew-}7D!}ipeWDd z6A~6xRlV2#tmC@T1Cy%pt4+n!l&rI}6Ny4i0S09d7Af=qtRl6tqLWoTuq&vq-sDEkOxXNT*%3MCghYK0( zlsY2CtDLhIvw58~sZx|G>HD?7bD6>CHmCEkPx#&GEr9ulUwRsn(Q#&=BUW%fn#GKu z9VuZxen@0tQcPAbY|I}w-=|)G))3rHL&!8hitLvl7#AIU?j&S3ebA)U_~@%p*?y1* z+V#mte7xtH#%sTIWNDP+dD>bAb8sQC6qj*~c&sBFO;Bf$-dj>`60_2%8AWC#SQ_f% z7PYiw{`^S@RN4Sv#Y7t}jgkN6qBQtUH)yw?Z0o7}QUv%)Rm3-@AZm9!WA@v(k;Pm5 zJQ8C$dVL2?P0f{uPXD{x+MxHQPhS%+PT!K1ef=75OZ}IqKlyF}r~|&PsHxnGd$|n8 zCn4GXjwP|-);hZ9_7TKtV{e~b8~Eq&@rQFDbp3Hswf@ft?o?j47-iB%x^VkalP9B~ z_wmlosOICw^J}?M4WmTo;3p!c#XCBFJ@sCH=Li04l2fb{{yQ z)7{{}%JQ4}#aX_dK5tzUDV5+1kA&{NeUcV|_{Tz@u5+pDPpu07{!0f=x-(Q6JN| zBMcK}1+e<0X8n)wtsk1cY+nZsXwc?| z9m@z#)wRSjCgP2>5u?bD>Ct|4qUMv$rY&}jkmmYasc}q$*TbjGhSNk|TpaUPR6{lz zRyL}fjn|G2Yh9tEk~UfLnM$hh>`6|LQsOd4a6b+WS1mIQJHHg4{n`6lzI~QAZf1uf z*9a1Nq;CzT26ps`i3pJ90xn2(^l%ZRciHC03NNbgCxpkX|9o$SfnRZ zbsjX!-Vd3-+Ql+P8}yVen_NLDIl#<#wg|#i(^Hq4KEnotNL*)H)WbZ-mjX56=CL{X zG|EegKG^Of7_g!z!$lbw@;N)0uaZ1epq0)vN-!9O3wc(>$;}PTz2^67j4GFLSh zR`4$Tj3JwFVmtl1q@VZi7G)~wG=?mn^O9#&iKXI=ITv7%euf(SNMK}%+Z-&n`SQ~8o zVlIsKzLDE-@Xzr@4EytQf`(i{x}bVLV{V8~$c$3LXQrVx7B`71$ZFRzeO_iSBFZ@G zflyHL^C&oZ{0Lr>ips&DDZbBv)e>8OPd+EBCMB0)W%S;$onAj#1|~br>Las6F5ae| zluUbUO>6wFeaP!F99A%QPZIW@l}0zX2j|Z;>0TIPlp?dTLVeb_vsU<;rpjzQy|z)~ zZfNhN*?*)bAI2w*B>yWslqF>)2gE;)7Ll`abT`j@e6wrb0l1Z|Gap+A`<>II;t~yd zpq2wTyWFs_S2#GkN~)?CBgKj8Iyz%(o&GNE!c3-J#Ku4Z$*mIr3IM?#ed+CNO0>^S z`?|ER*s$<-RYx;sk>=h+bP+&OMrNKX0)vTG83h(w00A2@p>T#8471(tPEADpHQHF~ zYfDU(*Nj7G(hm%%t~7n528!(bOtj(I+a~hINGrp)#jTk{2orE1~fX8K?;?69tq0%u_cb;`I8=n6<`P$p6?c!^g~ zUAe-N$TWOeMrg>K7!v3s=%W~g8a@(;7Lh=|uR7-mO)1Ky1dwK-i7-pR0HhH4y^o1! zZBu{$w%?2BH}~`N1Dp{OMJ81QaKT|n;?84(f!ih`U0jj5q?8X&-D2R_(_3-+h~?<_ zD(3n3q(=7?tMYjdbb*Ul7-54SUoAH#721cza$9|4LSn(5NK!D@r`1s4Qnq9V9Q3`w z@JNX}Jc^K%4VFjq8!C7tc);7G7q5bF@Gy{kVx*IiC|cB$8~VYe;7S7-AD>s6GBZ*i zQeMj<$CrW$Vu}yLo)f?lAs8QG0EoLKS$&En}Yo%D}yQah zPrZKojjr_~-urwfCHmIR*1^Hb%_4a!fyJ$rm^W;judy+A7d7}LF<{4%Y!AEO(UUR> zT$TmU$kny9q=ZQK3%X8Kb43CP0p=p0@7DG=NM9(f?#!DG*kf&dIDScldwsmow0sRD zLQQJ%=xNsPro8XgI&U~QImhFHbaR-)+8z7x-Km28r?G>_A7p3iT|&wRew9`FeS#-< zj>mpBtT)etqNl4#7>Zt`8~Ha+uwP;FY>>o1mL=KWEZ_EbuPOiBVb}$Ps!T*4(q#23 z3OQBJ5B%5SYVX0d=_Y*gRQtLaO8*kb@XoA zA($|cMa+O2i(P|&=#*nxWybwCJb^NM`Dd!}2`x|#<9(vgNGb$t8n%@s=(H2FJ{Iim z@Gd=WIDuWy=)Ejls-yrWjh%ru+KBuqmpn)O`H0jajoyi{|i z{z>Sr2aO@kJRhgDdz(0E2~&vh!s;D#X$c3om%dZ76z*EuYMfJ?IJ=o?!w92WY@7VM~HRg3p{|nmvnzZ@kw9 z6DAR6j3V!SirlW6?faYN(}DMI;~7)aXt5N{6b6Yog(j9R^tU?$>hOT6MaJ7BYm9-n)EZA6{| z?j6J-!P{1no)em!DmxH#k60>_SZNdE=f|qUx6^cc!zi&FD@*XLq2mji4XHm;a|&($ zxcr>wF?hZ>65r^@cNr_UbYk5_NUq{=fkg{AhC<9h-y#XjKWYut;u=_zj0Uru7NfGb z%gGi|yYYM|IlROidUS%<_1M``lAQ;yFc})CCS`Bgs)p?=D#y>T4|bdmu$kwVcEqNY ziVP*1QahX(K{AgOZ8Ec&`it48TQjMN<+oh4{W?-)$p|IA=OUPcl#=9?=KU_oEu|4! zxM08&R;jjQQU@uyewQXjtzeBaCg{#3Ml@Ml24_jE!r*+_v~X{yW3tbCU+h2SF^Ra? zmxL2ZI_W-43P1KZLNf{gbeV`pdg{qdUr zzBHJ#b}2(h_T~tUo->j%l17_w%aYQ>ia-P#ks?UA0C>h7QscX|UP==HA z(jzg#_}gkM^>Op^@-DBhIfk1Y0hNV$u2c;qiZ71QMnHNP8p)QnQTQPdt9dK%%8rYe z30c8anjy;bpuzc*CqCGvk!io? zRX=Rr`@ju?(z$2W&*4%%e7*{i@?O2U`u0uBav2u^0mh;ZKTg_Vy-u+%1|VVfhQXHE zKyK{-tVq-3YbWIq+KHkwV<&_jUcn~sE_ciH7hb;h%B{U+rFAA3?@bP!4)DL~`yZihuC(*G=< zd5}qR&RgmJL8LkJB<^FW9CTQTA&3F4!CxHNl0(l2a44isRHO^`N(xt zT(Q$!>P9pe)Uehi zmS4bV9_Wr?yZ{q9y}$jfk~3hFtXqyYMVx++vGkDeD>pY6Z|%^wD`3a)64(=(AwXgV zj37Zy5}m&$g7`YR&M_wvOQG=Hm)4n`>hAWb2C#Afgrcnr- z@goKjmoa&%1byV3!;pr}C?9*oMq651+U9jj7Jk7-DM*K@(06*Zk@OicDaqXayYa29 z7=0`TteO#pPqqdM1w4`(R)kspjF>_sh+jfTMdOkZzSjTH`HAKcVX%?(Q8%?wFXrlS zq3+!ci-F-XpVuXRoc!O%C29Cw-ZmdKmMhaZW#ZwUTdU7>`}*KuTtjmwiA$+mIeww77VPcmecu$6}6wKlH^=27hQoRzZyye#$(934m4v%Eq?Q$Xz*%P#^FM+Th3@&v z?CLR`O7&7yT$e9FjPvw+NMZmkoig&PP{+Qdk$o3VAs0MKsn|UAl!svDn(PfBi9JsQ+qStNYlcV)TQ=rPK*jfY3Wi zclRrxRAYfe=&5zDVaSK12aBCbq-soHhS`8=Fkm5k5?X?Z0}@ERhx)GHHJZBb7Fs<{6$fbj zeVDPAy^l0@&iu^uX#xEtdKzCNJmvX{+FB~yIPAE=k!^0t8$5MQ{RCmb5v#sUx?XJ~ zX-i)iF9ro$`}C^*zE-IIoSGB^v4uePu7LoMG0{prh=`Ibaf~h?kSI1kbabdXKMlw_ zb0-$!6(_9~K6c}A&{vhrBI|7!Aue8ZY`jvoNcndF?7MUJlOp2d$B(aS7lz{jd`gu0 z1|NFD_QzCUi)_>oIa?5#o$1?$Ux92SDBfWHCXS68*Zq8 z;z(4meHxIkNo$}cK83xp1pn~}#6cu0pF=e0atDadNiD!5?I8onQOvkO`SSV4BQ+#3 z`Wg)KSCJbs8VY9`z%z>b1}|z4rc})32?Q?uVvQTQljhRCYMU*?le4?)gUWb@rXBMi zEP!6~O-2^s@yCa6;qc$E#&(^A3}`g6L&T8&n(BnQ0tGx6K%jDg(A8Nmek5$00Z-+- z&@G0hjx)>m-a>;u3z%xj3mkQU!z|LIwRlTL=~^7IsmcEWHx#zznka<|k07|DnYLIwI%zz*69K^GCPN?V*HkbuQS z@0)$rPZk$BqS7ysbofq!?;(g4YW}D5A!>*yqo~+OF#M3%7+j4ayw}(iKC4&~YI>C;;YFu3PWVGaOZC|8e0bqbh zvk|E%&YMHMx+~@g*=2&ZQ;epp%`370n%fTJt@W)M2;7#KphCD(FtDKiOhkX}?n(4| z1MW*rnA7d)Np8FBap#hMEu{RQh}n_8)Bytr1_l8WVH1ZK45kc2D%!$TGZhol4=YBH zf)ti|lrY-O^E52^ukY+7xKAypDU1g^;m@VNT##cB8-T7kUPUItNy9V@LgzOJ?7vcC zAi>KbBIqhH5R?vExDU^N{}Os!rYr?(&g4_KANfGgy-1cG$lnlwB(3^$%{_5s4f&;z^l3}cv|PuC8R%%!{!(k z?muJx_t5|K@=)=Qi}~*__{60DdCq_D*bB2K#|Ma`|MU9)wH3CI2?rzk-}mGn|6yW@ z9^H=wAJ)I}hId6QC`cr%|F2NI{nsKeN?3n#%^{#oU!}wNKXy$ZhxD@n=ZJ0v7`h6I zikQS+u%WMjo&wt1n8fK^%2dSYmd(qU{~Wgqy!!vx{`U}FLZJT`%YS~fFgUIMy#42S zi9h}|nSXxrcWdJn&j0DQz5f4n+Zg`$QU8t#UG4w3A#;*Z0vb7>$S1UZvutzx|L0YS zx!7`CP<%0{28hvWMn)6#<+&0P60n~>o!@RJ&Mz-3dU|q8N=gEnJpbptf3aP_Xlkd? zD5kprlB3DV$$bFAAuurTeSEy@<*ckGaLBPVM*TVqgsZEo2nhR@B$Y3&?b6ZmlN2ahR@4%Sk}4z$*ITKrrcT?xeVf2Nz3&0yW#z%m$d^nFv(nMaEvoGI0;e6AlNKR1ceR zs(p@}7dnWV#hve?g$D)%*bg*4Jl<}*+@0PLV_eU#^7g&ZFwnx(&MN^6r_DNK$_~yH zdT0wZD{dt3?5Xz+N6|rtDdcl}W{XdiLT1u&&|}+FZ`BmID3Dai@55|94DD@&o{PW5 zsUK4WZPWo-{zi);g~2h~!vKKG17N~5D+nCbK0+sDTCf|u4y zrE|+_Ffqzp%Oz{DR5QQp8M_(2lmtvz6hZVP^wTL~8!n*yU|zpem#wwN+fNQc`&g-S zq{E5wf)FJG#_wZW{*tA=7K@uti9a{<2JX96FQnOA!$&s?*Mh;y?CUZcJuxg=qg&^pJy-mF1i9Y;hUN`;S4TCZ;FOP2TolB~xBF6yT~K~9gpLh`~dQN2W!w5UAo zZ&F|lY@ud)lDuf3@9!s_hO5pQBoi@mQ3?+PDrlK>h;$lIe!qKs7 zA*Fne`tBg{k6$(HW@sW>Y@RDSCx-wZ|6_M|L1iVqf`US(pf?Ku`~3Ff2Z!Yl8ldD& zKt@J~fpjVk%?f1VeYw?1FJJs+^gzDezV*<-H4z*dpWLc@@mWw*^7jSrr_;{RVFH!L#VgdQwuG1Q{hID@ z=ls<1D8b-9@8IWG%pDEC1}<7})^(FkHk4)buJVXl9G_5r8~B=X#C>l0lNek!(G^FO zAd>SS^F&;=bL-x6)%IBqKOie7p*u$Qu42b+FC%D3I|HX~yLNr^=pE^%9QIN5-qiXt zPU%)kz}4q{jSWrTym{%-I=oXynu)}q&M?_H$mx|P^uYP2Qcn8FM4y9^PTEsn%F~hF zLJU=R%!&sY#G_2+VR3#%Y2g$^+aCR`inu^2uJ&Ua8ib#u|j5wmQmPH1Z)k93D%+iVw? zNB>%>%l>Ub7MD;%Nio(3BS2*==xeP>cNdpfMm{w8P#R5J8GJ|3Ry?k+mmfYq5j{SW zNo;FAU$&|T#dXyxi*7Dfz0_>C#`Z@H%T&f-Z>0z4$U7;Z@nAcTBdN}a5EI9veMF*CjzJj@5>zQeEJr`_Y-~;`EGXUoXZ~)g7^+g~jS81FX*p zMbuR2f0r2lZQdPt5D_!nxX8;(kFBjOKpY#8Ujww~0Uiy}#s0L-+16lVW8=*Hyrh#8 z3s9nfhlLU0b5hs_WaIYD43wPMmm21M53;-x0=AE;f-(!9Y?+68oSyvyn7Ynnh#5Vd zI{vtNU6J!m5NL8$`DQt;nya*Jp`ipYI2VGgbXcs#mr$#GAi@R0w0UJ2WZT!jt!mmg zH6XODOU^CN`c6-O6mxICejb&-;FxQT4_rJ8UK@YFoyYBG1xE6*R zXHHd)TMuM@Lq#VOyRWPjut+|}*YA?uxbF*)iFUl>aV3UT!tIDX;4{oQxPC1cQ~#v^ z6Pa^mK9)33qNu<3wK%(>JPUZ(GO_>D`Q}{zW%F^7lQ*_;xOu&gO+WKjuDAN&Onvcs z&C=TH^m`J)GpR}1Nt^9>w6JZsMxLXhno8{cl+i+RtY4*NeyEareD#W-V{U%VmeJxn zI9u(fPgbNyf@oWB<@9o?_tQ5y9@st;hCjnZ$vEIYk|}v%)hlF@kPVm>Y?u*!y%MJ0 z-`U_)odJGo=AgH`4Hu_N=2f*xj>}CJ4XotLD^ox6NPHT`D&#H-mEvdBbY=4frzQxH zceEyG*)2{PByFnN^No!Kd=Kg>#=!r>rh*|NuReVODxqvSlRLY+t`qFTfPQRjeEcA! zL0wO8Wc}sYyREGaADB%6Jjd*;I>0&vlEmBLARMyc^9EJ&DFG?(+MJRWqXb4iC|=-{)n-@4Q>CqZovg=70Ix(Re?e zi`b)uN33YDJYC;pWto{9Se0anK0bR3wbx|9fi$M>VBQvF#^E4&bxgGCt_>Zg;_tC` zXK}JH_F3D0>o~P>dR!Hq^?*}jOJmt!%m|FM9%T_Lc3zfze4Og8YnPQ(2YnK`qv`Vd z5$ofKGGAs;up77V7&HJw z6DGl^5HmZcJu8#JQl~J-+u~|SJy$rq{KvOBwr7151)G%1iIi{01m=+BoF@aGXCh(d zjI9ok(oaC>VzXR`*Qkfzv^CN*5>lrQ&*x>D$HB~5~HrMP&p82S-Fw+c85&hak))74?BK>^jvWM3-8rCP3 zS}D#W9E2C#x^FX3`V8HkzmUHcSTb5xvUS4IA)jeWVGfF{EK~UzHAJuC@$vnQ`tnku zNmnlM>u4mLMAm$GQ*6iTel2SMMOG}*($Bnd5<$f1Ib*g9D)wGY?3g@Xe75Kr=&1(Z z4YK*^zx1&vnY<0%H1W`--Q7jLJYErh*r1NGr(qE;C|cnIseAPeb0Pgov=yPHD%hn#Ird@_MlW6Kd>p>UGwxC~gSndHkhuA<@NBl^>WQJmSA z)Q9gKl`~051M**;Kvu|ljwdOps0b2adA?>uHJOBkV%$0qE9GQ=EQL^g?fj6nQ|z8B z*(at+VoZUzo2G3w3@o0mp%g4X;$H?c4*n33Gmb`1&itxbpQve~2qWYRpaZ+@Tqi~LH^Yo50dJ1tjIUmO=bYRRCJNVO`l?%4?u4pDuy zHFbAzrn5LoI6NhDRX$Y*H^~;6eQrS|V^X?mRzX?$^|mI;I}_q!k25TeT*P@m8TAc_ z)5%oGiR?k?jpl+t0(}r7&eGW;Y~CgML+qpw^hk>rW#dB6*4b!qaSi*yzphH5(VcYpk=&!DCPT^7sV7+2W!9M+C89 zDZLx00g5b66v+lO7oc&hU718qEuq%q>8?NYZ3;gz0AY^9dKU?x95ED6q~GdBLq$cE zISC-M{YEAy2M;{X#m^uC3DRlozetE$A8Ud+t9-!1otY(+uWRf5Ye%ZdzOVDCwkXorK#4Z;Jbr2 zbed)jAr?huy;&1Le5&m$}Xt#jG?KINhm+WInNR>3iD^ z2*`d|dI85*D=+hkv94#=Ahk&d&;SRBk8>uYuznZ0^Gy)xpAJFPjSN)Oof!0tsrM#6 zR(0ggPjzM}_+!W#p`=}?Y#53@a+gneoC(;VEt$5*r)A`c8+-%Z2tUG4=Jj_}k5I>7 zwF7D;02?@}!01jf-;Q{YG@~DR^e%PRcrq~ffq2)IG}R1)rZi;xWuL(bSQpeAjT*V{ z_Zzt=VWQ_ClSDq=*_<90JcZCK#db{TkNDzE_aS|fqT5sR?8|7`zvpyqP}TB69(Oj1 zp~D|Y$)3J5?F{V>I#Jwdb)cR$or3zGY>&}IR@heP;Gv56GY0!_ctPh^QdwWHL_4l? zCdSw?(Im8#aJ9>W!lQ%yU`YKzmA=wiT9Z31^D5L7_|%KvC-$Uy-5ySY$Ir*OK#~-8 zfSzn;3DRrXiXJ}yN2=QfPWk$aDJ7nOSrMbMKaE!eCHuN+xy7fKyyD+5mu2s&_myd<5nN7~Ipsu~hWYt)&N`tb$+;lemWI${K;+Pao2_K%LoP=C_8G%d--ys)ASCj*D2Dr@qD=Sw)}twj1<(jx@$Jg^3BHY#oW{ z7+Au;`7nO!JJfmy5S|Ohw#Jkk{9>l6KkIe6W9VAo-nCX%6f;)^9^eQJQQ~hRw+R~9 z+8^|1q$@*Z%M0%j@&kQ&&7BcjM@eM}o9e2o3qIgNs+!|}u98S~9PH4_vd^_-*%X*a zH)AS0si6GU<9x@%Rd|32yG`|8C4ya+COQ3g|6T$euV|v4f5;CK*0W!hpMSlT_K}sz z%)ds=Mx02Um>6<`U8!W^+eE{ibSeI#>Qi5E_JJU7Fv=pND4VVDOF>ajNpP+U+sNU2 z3}Zvy`FASRqI9Uy@|)U8>;?u8hCZ(Ds1(9|8~G#-i*ra_9#E7ExpOh^ zNy1Od68K5%UbVro%F1v)nrr%$?@Sis>G@+>gWEHAD!?3{Rt6e_qs9|PsX0!uyaR6xo;h~5 z484U%lADCB+9e&fxi?`*AD-`4&LyxiR(u$BbB*P_t`Ty*?72`Cv41UhhJCJiq|FH^ z*stwwhtBC=@H_2U3e}e-HnaIww4EwAb)PC+AL<9w6tq41K5H`93XLqVKJh0->Yhcf z8!0Jx;N}lFn3U?to1D`-xeRY-+ioW0IPaRWem^hxT-N0eLa{u}b}ZTBzPMRmS6Ti^ zg0>i-6~G-|-(rP@+v-xKCufpO(iiC9cOB%^?uzlkKWFeQk7$>TFhRjBruT{;It?8QmEA5UwXqvBr%=k# z^FyG~b@2-gzV^U((YEmU1d*uP0{zixlgy0+*<(oKvq=7%{NCnip_CoaTV#2J)cFA} z$walAm4e&r=Hn7%zWr&|6pJ@=*Naa^*Zu7j4@yC&B*X0{OA}x>eyK&;j(uXodX;8ztV3d(;UN2o9QHFH9=%b-AAIn3xedR--^9PJr%jqG?-0pu0(PsZFL{yGEx83Pr z&>XtzNW~;)$tq}9nx+LAHrJq44-aa(y27BuX%LL63M0_#q+Ltk1oSqp2@khkETB-D zpFb45FI0vE-wZtxRt}2f>O}HtR;zHJK(lZOw`MTTIWgJ42;?sM+Wgde-M%a<$1IYr zk1(4Vaw)|MJON7HC$D0?)T&wPp%BgNO)A^=7%_vDJmcYvv) z-oxQ+;3F|~$=pzT+kO?WH^J0rJ^SVL@KTFS=Bv>on|lA)SWiomrlj2Dh~*OO1jX_% zGd*@1zH~|SqQ=a|d~;*+UO#;WF<)L?EfdaI%*iRfANmv1|LxmAO&i25rP$0qHGL1J z51JS*EwjAz9h=;BNC<0%_@jVaf%@Q4pH-DTd;!jnA3U?za>IwGAB+W}SE{J1AdwSV zyrHw6x3a^?{KLPF3&>ly^!BX$veF5+ORWc_fQv@{N8%wMAk&x3&E0);YKoSHC84WJ zgoA?vm}}0?Ty<(qCWUWS`hkQn4Gj&GL1T<`94;maiL$09HZwD`mbSKe!xG=kiaXFL z$>Q)wHIO9r_h#B>8W{*vFdE!%IWB{#H)h~VzMZgBY}q~$jQNsSh_KXp$LermQLDYu z*^nrGKtEA%upPSmTK~3xn_o1TfTjT|7yA{4qO4N7@|KBh$#Bvj7>_ED<+$Zq;dyqp zr>TZr0k>gV4(@H2!z)L$2f^51oM%h!SvYwy%QZzb>8@wo-Y)E}B`wu0a>F^PuP@Kc zZ7YL67$04^;XQ>S(c_}4IhO2O0d9_WRBv5pEwRn!TFrCI0=GakqoW?HK6&*BZwa!gVC<@DvNgyDO8@Fx#Z1Wv;n$bQf?mwwMO-{6l_MtJLt{#_f3EkR5E|jsh&B#jXsS`}SPnYWzPAF<>MRo4|=v8Y=OyE;kn5g?Fc*J&} z^EK16u&nS10wVlgB0-@2IeJgYC%!*A=xVW(%%xmT2su1I+@Fv5!juh{aM8z zC`V)w%04&wJNm?`L3{Vro0ko?rm7&?axIj&Jq+Hj;d#u$OrD-~m7)^6{MIE=Kh7Sf zbfVyB8XLJdv$d7VhNlXLj+806{pcpE9hCsyCgGnlOJb~xIad`gb}S@z5%uYzLc}tl z;jjy3OrVe$-`1<|5tH`Au=u#BvGnX7Jm2b^T2d;&tg#3BR&;*s?^1}_`fO)p>RAb+ zV|DdcSUN~wlZ5wwfFLx)gUboD%~`Q=8whBysF9?6x9GvK`%gsWz7JlAX;u*Ck$#)d z&1YPowVrs~g0wmGEp{~D_lPi9{EnOt!O3EaGBO5SwAbuQ+cEA3mv?l?`eI-8T=-Ti zj!G-Q5LA|EoCXxAeV;|c$6p~ZnM^IjIui_FF!)p021(5(?mh!iQ?lcZ^oSn#u^Z^+rT#mkhy+h23R$CSU!97j%a^maX8>gH@9+J9*3-Zn z7ryty+nd?%YAC3v;DBJG<`6)IoA~bHcpa~do(+?!`uPcDm;HqYmFqp*7C*l(1hc?tR_5o)jEtqNez6jSSDlLmq9rnyBTT zXy5g6A!vHvM#hy*`)jC0nA(TUGFq;A__kQMK8CJG|NmeC#M{_?sBpBo3&CcI(DIhW zp~ewg*1VKJB_%r`h(Tg~xwm<|O|m(#ArlrFI?aqld*&=O<2{iziY&)4OIsr)u=;GsGyBff@Ei#;Rfj zF0=%-DSQQvR{7r3c;#tFyrliPUA@rf z=pS8HpPUsz!nWn!2f`YKrCCCqHz~^ORU3#ont&7O$)6 z;_AxKusO{${wiWOEj$EYyQ80rD${1@Q#C9S*H-l!B;bAl#1A@6sacf_uQio%nf9E} z%QCA%8zsZXnA`DT48uSYq#DAy_dt{+*(wW<~|cZ z06K*9v$6U3qtvSVFh;(>hIa{*fY+=nEQKqGm2503;vB!{;B%F-IhzBxenAwoUnpIT zl<8J!h4>#U*nWJC%qJ!Kv^2qG=@xPPZ0;bBL%jrarxDi%fv(kY5Tw=Baq?zsRvl{W ztTZ&$lyHw&uxYgiXQ%*>QbJg4xGe>37$HwPT*E)h&mtmPYg9RtGJwW60Cwc!<_4M# z*6A^&rl%(Z3-?v0^7i%|fN7B*{Oo(&x=;FiJ1hz`;vPQV9Uq;Z9zO7};K=eh8^i+? z!>x~3T06XB0}AUgn&peD%^4YWssJb!PL}b1TX6&wW5eCZnFbPe~Ljbea7^LsCArT z>20HskeJ_Hq?N3sZAH>P9ocqig8G+X7kfL3GZjX4_E``vWP?rm*GUZpgD_09UGkOa zGY79QuGobD8)L`ks5<#CQtE0t5&Z|$0<|GUk;sdD5|!#8mRv~_i4Yr7Q)SbhjcPPM z!cY3`tyx7Wc*)!K?8>fI?nkj%mySW9-w&JT_92h=}YCNvZnU)f^`D0)*5<1x)lCvOM3Aln}dvDj{n+5;{1y<-f?@VB?Vd zXD_K`d^-m-)qM-hPNXaNix9@|X@_++F~U-@15h!?AXmXnD1d1!?fGHpEa!K$L- zaT$G3gj|<%w@>F5_=iVkltr#$AQktRvpV%Kf!aA*xLZ0{zRualh0MuJZ6t*9nMGON zzKA|V+7xJ!3%*NII_acUupmh%wk={UJ5s#o%s1{Z3s=nDujbsZ@sIMwY{-~}rcS9_ zR?(Sg(bbDR-bS4bQExOb5`Vh3+@oy76=}}-EGm}kOT<6xVziEgyj4C{qFu%S9MZXSGBvyzBg~Wr)Q?8`_tXb zE#~^6!FNgU0wueWzEMC6?$)KQle!w0%#9O3&f1qKz+Z5q;K)gKf>V(1GUuJ}&1`V3 z8Ca?K=hG9ye{*o0Lb(q_fQoBnW`k6(RF}fR1Hk6TB_g6|X~_nF6$UgSy3%T#{A_PODeC`{Em$y838O517|p zrS?g`QZ)F$kf^}R-y-x)rd5yI_p0c z!_@{-AYi4FIc|JqEG@h#^NGR=IhFPwXJcxl<29{0O*?P;fjaPmxZtwuu_Uf z-iW888K#&tw&vxA0bbu^B)j2lrvH5cCGX(J~}BUwu|Q^{*v$8 zai9AmMXiv)!P(TBR?T&jsJ&izh@(=nRM}Wyo35mPYaZ5=_gMB}4Y7R>;=v|Sfaj|b zGft-S>)G9J&6A``u;*@N5mzy+ig`n)c$|fAeJ0#8XqbKm8v3vOJm&OrpESzxJV*7& zrU`L1k_DN`ybjJ!e|yj5YX=kCD3mF^p@Tj6nkiM+BfaC{$_v7sXWYDj>w1gFdQ*OD z>NsIYd2fDdj= zD*QOy14ES9=sEb(H|hDc$oR|QadN)B?fU9S!t=}z5FSaxtCk;grDSM0>F(#pi@$U- z#am0;Fdwgq?umAa$xidX5_hV63^tT8w9$P)r5+5+F|GP;-Ii;zQ2ugM#k?#)HsxQnRddg!}4|7>P%4Q`^jA(SarxJ^diRuO-u+3B& znbC+!qoI1Dstu)`ie@~KUgefus#hwPeuMgO@ zsxX7#Vn#-EnJ>_E#zG`$<;y3mAZdy~n(p0;tA;UVko8eARpPyO!r;7 zg7uF}1(^%U$*=md&K}B-(#2MMn-*_Vo;6BGW&FvvA0%hHxHJHAxthcKOm#^uYsQgR zVT{ApR3G`AfAA5b5fRCk)7dXgQAhi>_SgAQE11VAtBZ-%*S=EM({lSlY0~u1=(9>Q ze&gbJ74q}vS&AAP0#moo*$$$F6It~u_B26t%dSo?`$Q>ucp(x)n3R6iJeZUm_>ZzI z-uV<+0(d&w6cm!x6iIq~u`#!ZNTLv2<;OsX>eyodQkY_+e}{ahKH#T`cP6xl6y|i& zJHjn)t%yk=d+qPhb42_`dmy7_lE-E9h;akbs==Y~^qH zwcDxc_O0>ij@PA328PAmXvLT3#FvYisw4s6VW10mSZSV^;y@m^cX`?5PhQivX=tHF zC~0bVoYmG~MkowU^?n!C5?kR~RT6F??l&zuUss1L9V#J_d1~z}byZ)*5c4bHJ*-X5 zC(`cdIm@?mk!Ql$h6YGqQ#csKN^^pN)Ore|Ls+_Fd#C+V_@iIr_0D*JYROL)D0Q5y zEa@ACFciiFXg^VfDqisDVa@b3MqPzk{#fL)ouPOM!+Szjtx{j^BQvJmbQ*z?oVp~w zbjR@-!vVOf)-|kkgi{sQ5?r3akQo{_pw(GS?eUIW%eB;M5A(1|lCP;Oy{O&j?B`wp zgm5N}thX;4<7~9}$6T5>-$NGNnUIdeu4H15nAuBv27aBfz7dA_@vfiamtWa#sCD#*X}9$!XJFj`e29qTuhlT?>VECxvfR} z5{Ql|(lw76P`@9b%3}{rO+_O5A94J6{<=py@1yC-u|mb2&)^i_ugS(;wJ##j=yduw zmHTpXbLJ`>aPOvfd~BrEnjs0A#4X zc=3xG;H<8^Es@oq_RV_Y-L?>R*9-pselm-Wi5g|MFlAA(@BY*Ci~0oct3%AF>;S@_ zE@LB|mhy@YUpFOIm|hBeiL=&nE&C*e`dv)1gW^GKV^VS+9_9slsYFGG}CRTa=F49v}gA6xzah-v{! zy2=0mSYS~C50aA;Gv%oMaxFiI4Is4CMl&dM;OI(w}2BWJH;Ctb>P+9RaD}W2mvj8j$fYIUK^fp|yNE%s1#mH9o_b^yryt-~rkxo5eats62 z{bF6-V?I6}%OC77Ra68tAQYwm%OsvZkB*Oj4*UTmBKGmQ3*?ThIbZ*>wzf7eu4;!l z+q^_Hz-GU{s>D`v3_|v$Yq>g^RWdPQ03gc$^RBS&47ax<6A%ypd^fC~+=B`&uB^lY zC7b}82~gaKzg&Zzo{@26bQG}65~_W97}W)zo1*~&3UhLEfoiq?wq5h9mYJ|5J)h9pEWGqbpf6`m@ZtPRgIl7Chn4h{~O z091K?>-u6RzpMK!^eE{}=f1Q}4}IRQx5FDJLocKwf;|2D5X>FV=Jl~8?DcC8>_1oI5REEQ)KS> zM@;ZR-|fw)q?fU4mQ_VM4GbsmTud<1pz&p+y-B?}K=Y;f^J=3Z5W_|O&lgJazrO!- zAy>D=e}9#~7~?;!{QU*9Oz_DQT;%zCiSmqiAZd{d93U(Jq>*QVIC_isucyq@8eLp@ zA&Wt9YKnR-Ym0i>w8xezf4vcVkYRJe|LjiSy9gRI;-;x7?FSeb|JDj{l>u7P|8iG( z{sRp3OK#TSplnxg#`=H4>Mrk zwSjj9ph>&C3yQPE|B5;*VaMn#`y+F@@$!soJ3CBFOsKtOP|@ev$`oBde+)@bZoA$_ zTC{=x`2Znw<{ut?0~Wnf|{J9bo#j;Q}CIAhUA%LG;)LP)0X0 zD~t9&1bcq!*VqP111`UtU7VYiXgVnfb#h)$gWEyPLZl$z19|!%3&6mDBrsuuf&Jr& zpkPRcA*>iEwHGXzO-M-ipL58}j1JA$?ryntYLw@u5wPPrf`CkzKq?;C@v+No7es<% z(oo$l*WxKQf*n8t&T_R&pZ887)eb2+Y0R#rLQ=+LHe3@=@{lMF88JU^S$D^O_CcZdwf7uh2GK8@MUfjD3z_rhsN~Bz0@7aR}X(v zGv(w9PgNTddi%i9OnqMbH|Gsj$1zEfY)z?D3CtYNDwI#H6D0<@tF|^iSX~k_s*Z74 zbXpzTzV1)B>i5q=*Bc?g)_6$pCEAjG|8T$o6|W!4{u)mDa*y1p_pkzxFx`$XU;a>m6n zKk2|^T;%Y?_kdXWzr^cOMx-hM6+Uh zLQX0a+~D(|GrVVcvDC;Z5Pwni+e=kgM3+C*o2iSOY%QW*qk}H_1wKu>T~FUYO9+GA zG*?7dCfgrnY^1kUEL_T-zGmK@vFFa)GO?iKnq2wcF7ff3|0!$+2?MJi3qZ5=~Ci2Dv4cv{Jz401ni6*YZ}%parQ%vecV@wf6jB?j$Bnr{jt^RoTopwCq z9uC?de%Ma8Uanic#cA$V^4`1S>9#&pk$oQgDI zW%m+|IFAu^Xl>7#`S}xp%-zp2ae>67O+b2xk&%}?JUqYW%)H?O`%Q)6cI5%~yEMXG=!% z=QC!>ytXWr8?Mpmt`Em_HN#{$$*ZW_C~st%9D9X$tO@w>OyHj0gPiwot^o#A2_V!Q_|bMi7=|p`+rFF|QArXEMC@syxHwjR|gkvCqZM z`CZXQZ=}(t$3cFa4eh3`D56bQrVf{kpH`KrE?3+k&_m?WAXS6g#n8@$PrOBoFFo`_ z$cE_)@QUmklDKe}k=eCuD?{uK$Ko6!Kqa8d58cZp+LDR|ihjNuLwO0oHd~Uw&%wf2 zMd%yxXLqW-Af4(p=zNi!mSaC=Z>>* z_|x0dUrI>kQ7b(MU2R$Wd){SrY>>EmxCoiH3X`zI^A8&?F@OWLK$=?f6jFnMP2LuO zBqlw!Bsx1g`@Q7>2pmIRYrR&T+2*i_ei@He4_Sgrhrq47{81oljaamr!O82lJom8B8_S`4HfFvm|z$E!8 z$WioaHhH72JsYS%f3dn|kVpX$uv}x0@Hj8!)G^QaAa(;Ha zof{piVeMS<)IKyrUF66ZWsio%S))H(bc@l(nnvrTC|RfpTx;MI_InzKmvBcArg3yZ zW(L1RQVBfGDGxJVW7cJQm|b2Oy}Mfa%GhK?Mc1#mIpElwWNcYgvLz@ve8hgAgh{+% zef4f|#q;3T{r$M!Niz@IqK@pr5ZSL))0njF}Az4lbf{AQO4!X)ST0p0#% ztm&w}qzyJ>F;{mVYs<<;_Lo7QdowiCW8M2Ez2javv37d?6|*?eAcW7GnZj>R{hNHy z7|v=N61MsuB?&%d#;4n|3(Y8aid)=IowT@^V3;&%zJp$}8e@6->eQa9ZUsbDDMzBL zBXd-vsqK0d|H>@&_7DLJ>&M-7mp>Ow`8ezNPQs1j``O)=%wh{_OIHfGs12{KAGl42 zX}rE3J(TU!{;3;X&@w?<&iXoYapB_7v1tr&ij=R9S=`+2|p$tBzFdV!l0p*i0CYa3SQEnPu2aD^e6K_4@%$8gkR!$L@g4>+K zfyz9(OOi%8Q@m+~3Y!8WVHhNVAr_=HYrM$f6-Qce>OTtxe2zibSlP zI3`Pv`i`Q^W3Hck;32=4k42()Qe0eFttUESRo{Fyd%joSd|vZt^da_sdEV@F=r-FcXfM!sic3UEH&_5_s9O z8FG*uKhu_(r-+F38Os=4T`T{T6Q*$CG7^0+oStc2i-g~sa7ir*%Q98^?x)TL-hj>* z!Zx7CI`ymab&Cn#xNU77cKr|;vPt^l)68Y3LioR>962rfpg@ z;!jYa0l(htYCl;ZYb_fdFkNJYi z@Noy6Vg~^_w|!=|4IGt=Qg1|O0G&X;!eQx&1QO*o1UU8fYoPeME0ws-au>nWr2=Ok z5kFsgDWBp@`q*c1KNhLqAO8jJ?GbLiz$)-*!Ke7{ za9Lk5oMJoSyRrvmkQtGuF1p-zC+}m*Pl}rkA+6ySDsT^gdRnn8OH@Wl1Pw&!*0V6_ zv4Fe|Aurfc$E*$X9DTvsLDxknbp+Yyd9NY(vW}8v@o8FBv0yVYjKl@e%6KLP~P)QJ9Lr+_M|bOlGN#5ly_gLQ5S@oG_V!;dAiKsEMsra z{af?XUY+9A;gB-7BDurJQ(yGHoRw;D>UHc@eG0^36S>swF;obxc)oD?D|y~-an#fx z{5fNyA~VxSMY4&~<|DHoN8!B(&-%pkmJDu&;T6sgV+hy{p}ItZ)kS6(Bsc+TqCr(f z^Ao~bkYu_LMwPjFn7kh2W1}~2h|-5bjjY{C`uvoTPiig7ch1_ny{%_29tgTq65KsNjIR?u%C+;Qmef;1)%S!c{*s!w;lu64LblM#) zWOp+vBvke8es-9&GsRNJm2zm;j~8>#Z?z{FFpdo?TtPpXxEuQh*)a8}7R*3dANeFD zb@2p5{Km-41zVzDggVM}J{d3HyPtx+6id0Qe^2p=OigvzwjE}o7*b=u6?7Z@_(|ho z{zyY#ZnZtmmo~~plk*aJKPOsQC3U8&QeIQITKZbq>h`iTVhdHXv4H?gl*QIcigcyW zP}7j%Qv`acm~!l=r8zMyZIefgB;5sfaz5gE?*u8lIQ3E!@xl#c`Vg|qNV+zLXp3X2E<5XnfZmu%g8S@UE!OYAitu)P=G`nFMXwH|;hOr(*^o&t_gz`2a9KgT{; zaQ5W_# zqrttar9wy(MV_`N2DJ_^{^%~3;dRj28=n5@lbc{-3WDB@o^Ffky#UMnXrn)CtD>Z; zYP^=JYDc|c5)j?f-E(ZP?*eH*7x$;N2hK7OwlQY`V&Gi};Ix*PP1D_$Nlp7xI$o>- zW-zL#*y!k1!AbxQ({wStGxFm{r=atw0>3>Ri+@*lb#--aKyYUfDDx(tE|k3Eu_6dK zJbv4q+U^f1V3nK`dk@TBx}f&m&?;R`_Zz#(514M)HVyD!8<8(hO`dM)d8LF>aNdvt zr=buDz_D&@Y`p7bPZ5-mK(v2wK>A><{SK0ScQ>{^Pqz$HNEd>B6g$vj;oxF-oCt-} z@g+KnAvXn_5x2XAZ=@XsMLLIqi^D@ilnSiCp3t^%Vnhxq)xhi}Z?Q~mh?aYjtWQJK zWshZ5jMF?AX2I1h1K9qgfuf=C_6>}=gV!W5L|r>7EhLXPjBrol0Xuo~Y*ZJITR^HJ z4;L=oQjPxWQ;)kIRJZ0f5N(ffj4_W4u%ZVVa1mMs`flJx9S%_eXRUcr20_78ARFck z24IcegZ@2#JY%I=z|^}lIS0Gc6o3IaliOju)J_TfPZJXPxj1lq82#*`4^ZyGxN(`1U2YXlTz~%G_Ax5x>3oo8~TY+Gd_iS)=dpi%n9-gFFqyQSm?c70!^9sP7-N3^$R0G{uF$TYEyuV&;spUXp zB(y-nxL@)BG=J8;u*YSOkSHtv^$oxqT743f>jDbQ7Oaz4yVk*p=WRN$gxIu@!M}#+ z+g@@>00#K8%N5Z<=P{kGv+go>pxAu(Z%d#Uu78-1MYM{B;@u&)nl$XLN5MVhxog{R z<}22HDLkqIes{%}ra^!lxbzWVxdi}~K!&nE#-nsF$Adtu{3l~Mv8 z`A}{k^y9g+%J4JB{< zF3vhq{De-6XH8bZMw1^`!J3xStY)gB@~XT--y}GUD#)=6^4`h?u79;Dz&b98m>&8N z3EvZ66HUU052Mr4`UT@9Tq##}1M|l# zp9C13p)C_DgRdSm9c~qKHMR%+9VKBtfJ4OZ0M^`4O1IzNk)1_Hnp3e?<7qY!Yo zJzjv+NL4)$<&+IQqf={Y&04B&wV zDU3V*CYt|fTiu#X?2_^+UBNO3e`kej7hqnVzXv!c05?03EeObWn+7awG|BS+%%$Ib zwT=2E4);G47f0#coR9tTssqs}g3+?Mt7~h&O)SXs&z2$BNJvP3rh_fumEcy0%`|3; zl>xGw0#9j9+Nf(t*r&6L*6t}hI1geBIkpC`RcuNm48Jueu zP$6#rvIVA2d$z>+EA$S&SU0)*lF929klVqyk(e z0%=PFi`3r+d&|&J{QfqG8`TB)I?R}VF%2`iuTS2l`O4+j`(+F}V^tkY{!zpN#R&Zx&L(~efT-o;&6m1?6X(`Ej%GzB$Xx3{N>Bt*mi+6ntW|m z?hD_FYZKkme@g0orPa`o)ikB1>>2jhj+|t+S1P&g>C&t`;F{0Dm=Wsi*ZC?{loZNT z5k)7Ec+1SyK`R1r#v@?Ix0ePwdHjnXt&MnhA&Tq+!Q zI@Pj5c#GrHw#?c}*d46wMl7(ajvWj5J1inMc-~9@x?lv`!$T5XthxWJ-4yLrpLU)4|jr)#-EI4RShIYScP9I7#GdFGf^R-sm4cFfNdxGSU_i z^4PlXYM(esC0@M3nHuMMUqsCs);At%-%x>2enxjg?p;@dEjZPkeInAAV!iOO_H!tB z3eulSwZtE_X?D(JYqs`{)q~MYRYNe0^nzOds#Qc1DAr*?c4EV*SslEdm%oaF?^#&j zc76Y)YS`&l#HUl0X)DkDl_}3zvZDH|KrlAN2iN_?nyquZp~kS;>w{vb-O7=NS!8P! z6Lrp`6IKz|Uc&t?60B5egjtkmTf2oy(pJHOI*yg$sdL7{L2MX0^=!AT*SRBNLDKOx z{5;6*4XXB84<|U2H@3)cqvmEUUsLsEC*r|}#vte!RK$CdeIJHqN!1ut)g~}yKWXJr zYN|GMg^^$lY5Vq>MdQx~u=DJv{e)dbqEuBSoLrn=0P z*Po?+8u>dOym%!Ls9DZ4aXkL!iO%h^>yJZZyJpq2*GRhfaAl$6QQG5ZbfXM2BfVhb zuWiy!qhm9Y{-44NQo1?x3K6w`vx&4@u=5tE^sXfsUjfo_WU5R6%_Oa$0W+WD*US4Q`rP&KU2NeG4w*)Gz zWn5S@Oct+QMc_(aWnsDA&Vr6XOTRa_w)=TZDT`~K&&uQ&Aa*D%+FZv*95689YD1E978c6Lo1cyDP%Y?P886ptHSfaOc*eNaEtflP*2O_tb3qcR=Gr zRb6d!aZk|`^}K~SPDI*HZq~6;8Uo~#xHEF)K>56)u{dN*Kj9&fx}ck>S%>KwcHIQV z5tsK5qlCZSoSus`on7se8bj~+o%s2z?Gg0#cA6#sUNG$L{4NTfZs3ysX)aAN+TGsVx2b;%!`0-SU9bsvcNM8Ys zWS19}`=Z!!75yI-3_3w*`y79)e&vVSSo*W+Fj#;Ind9Z)_#gWop7AHgH(U4 z!hhnhkE#23Xu73r_A*Ym6kk$3URlV2RXuSWcIruP%ks1HtNV2XlhPvC%}GF@BU=N~ zlvE^`fy!2{g#)>b%Aw!KpD;#p0A8<~_OJ05n!6Jg z#`|7eET8IqZLhB$-U^G)&naAl_P!@0gyozfk@vmB%XUiU?ys#M$nDOIw8$@NyxwqE z8Q6)Gklusm>2_8g1aULVxL<*)HlI98D)0H=vP{uw%v-Xik7FBTf%@@d>fqr|9>e}g za})=tW1xW{xl9$PKGE+{&{s|Bl|UJlk5i36KA}Og(TSS)cyb)XvU=$$9ks!E&4Xjt zZwF5jf}Ykz5hsXtWr98?Um6sp3k3KUqH>oJ+jkW3U>ADEB)ag&stQNvj%i1T+Q(H7 z(rCQwU{D9q*SkGz)~!seiEd2_pf=olC?{ZlDrzUtkt#Pkld(ILuG>x0;mprDA0vPT z8=p6tpBm;BI+05kBx&s<>6Nn8>^rQR{{|GO0Zxy<&Kq`OVUE!02X&u|mnQXeWkbg3 zC%k>#)%nFx-oIIz&k#{;HOW|v;Rb&sdF&%=D=PYWOp-VKo-Im1E}TTfnrKXEwiS*e z+8b9q(;iF_>&RWG-5K#^xkq)dtUe!>YQv6P2ThidYcA6!C{?GaVw%RN#8%1-EG;RxrLOo9S0!-qdAI<%kIJXkVf0@s#g zP~LdK1_jmTS2_Br3~ZYam$`@zjGBCS|qopWFTI?4^B zF2|Go(=gOv(^rmL&S7ga7wp2SV-6cx!y328hz33EN-4K|Au0^afpt4~!m*_W128fs z65u?kEO^T37VsPyLe812ivj0?5W~rhkAbS@UiM`0eZZY)kFWeTs(6VXrzjss)V4=q zbfzZO88tH2;Zf}pq^GtLxy1i zHP-fk%20opkkykEIbW%BtSQP`t7_=mmVN~j3y5552|9BZC*UlWGf%q?h+Ppw`&c{`GnMa+>77-cm)k*t zXeU83e-wG`-G;ay;j~T5cwQHi%miUNF1DW0X;KOg4k)$3Q3Z=s+h^6@h@nf(xSXGw z)tTvaayR~n4^iq96Ma{!3RYL!-n--Lr5@MfD&1vR1D#wPpBd%pk6Y3z2Ab0BYjy3; zko@`@UVgsB5;`C9q$iiee>2q+IeigAk0G54RY`}@zUiG`)4QPnkw;B<1v^i!bAu^i zadI;-rQP{>?_TfA1yKk6ZH*XIUH!8GlG;&$j^4Npo8?^ZOO~6%D>Wh|a+%2)tj$%> z+%hMzAsL=-F9F(|ctcaX9lPHc!N6qpv7- z-9Dvj%i!8Aq-{<%rG`f3xdjkl*qaZE;FmAVL7XVQnDzgN5K-2w@CATY|yy@TYL7m#IsGN$ZB;NCa#`-^(SWKr|Vxx$<3ES z1aFAlwcL|WYXfRP`nUSrMJ^j`G0lN;;FDlXyR%NqoQFOb`RB3ia>pwXTYW7Jm_kYw zUA=|qcu#H3r@N7x(8!~sN1J`Lc>}3R_Ew^UBDJB*O^~^jE$RG`9BtiY$h*5Fu;nKf zBWr}kb~;>`1FTy$5D=#$r8cHKO_B-T7COP2Go)VuXNY=@A$pk-!0m~)U!Zj?G_0*~0bJ zrfYeU_vl!Sai@9NCYq6%?z1vlRE%!W`OSnMP75;*d>h_;(s-g+H@a34z=)65K0|KR zLISq3S_oKx(o-pH=4dMXT(;cLRTvHLREVw6TPWcBL{N+yjykUIM~Z@()EGqXf2_-t zw(YcVg^5g#q#F`$a&aS+U9%SHX7$mMi6nHc5(YErU;w`vaeB$qNJ5gc7&ZS@%Z3JG zPC=EAl!ZrKm18UTJH7?&l4^Q+Xtqg|6st|#4&CA>y@vom(CiGrP zHskTzyAC{2Pw&Df(Mdg}*OchVfSV{g287M)&*ch*M!$=P~sI_0l zg*{DjU4gpu%wEi|B*#5Om8YX~3iqCDN=y-Co@Z1K8i@L+#I={qG@8!P3e|C1JU#9!*_75EQ8)L**sqbe_-FBk z?OgVH|8`Tgp|&`XG{kQQV+D%A?lS6=F(%x-E>g5g4J)HSAvq)^YHX=80#M&GUy}N@ z!U8Nli*%+pR_c?)Rr+f?DaEo*)_!;ap(kFYYR>C@8o21w2}2%!f+y)czFIqGO?OM}nAmO(G)Gh%cH8d;`o&#FYkQAL53DGH^oKd_H-rer+pABt z)Ue^w5MhmKsLEPE0Heo1y28dsB@(zZ|8&oHN08Ce&pqzZT!xt9FWCp|V$u?opmGF0 zG`Khrp~b~WdZzvk7So2Hz8v0r*C4~H{tm)OQW>=dztM2|vIh!n?vv<}Z$Hk~#@h^% zbmGW$%{>l${-dQ7lN2Tf{%vd3cDV8Y{k&%YGPM*BcDtHdT+xDl#gcr;GzYeRGx?;o zsy3-lf*1wGGvGOAHA&v!JY{0*jQ~2D)QH>?Xos(eXXj1-YA4mBPlJ`a}s! zm{xIe>*I3ly)WNY5B=fqlUmGlm>;}Le`M5ms4UU@)$P+HEekhwRCerg0xo4pmpS5X9!^hA#f%bRSR1;ZbTutS zvC!T7MvroXS&)PqUQzkBKk8#w%w1cwc{VXxdor*W$A0xtE3(pVB%IEEoDd`@-Tue@ zN`ppCi@vAK2l5QwujHW!E*7`^)Rv{z$WB9F5vak3Lyy2Vwg9=^ce z?B(Cc^yJjP3z0V7plKufAi=^R!Qv_8rJQRpSes~@o!Z6^jLC(W`o#NPlC>juGK&c2 z=p1{H0YT*H+IGm$Fi{I{%x?R^lf+5Cu0;92tfzBmpeWox7)I_CF0Q_7CB*4pKy{m- zXQ&IVI6K?ArfTB`z0s&59{n&_vU6~*(J`-6*a=S(#bIa)%J#2B-OP)(!~+F z(QCDuG1!*O?_hE^IPP_b_C1@ZY|%0!KZ|rE#?Z;tUgj|+G*Cf&$8?5i$K=&>%E&m6 zG8OG{uK|M0PkVPzSv$E-(ovJvqwrRoM_lZ%1O@BOErdJIv4S+5mhw+iGnvfGUv|s5 zs+7Aw-oObURlb946n?g-PV#dqkck81%NKvMA5HQWgHI)q5t_TT#bBT!gSiKzfQ(pG zbBW)+;vYA`ym-4N^zrPqzW1$Yaj*F-rEiyq|M6n7sSUpryxRIGyUPuh^}h;*Pd1tZm&-!6_gMR-9SnwVsr6K*x9N8)1KQV(XTfwmRrHEsV$56+vY^e0VPWVrTR;J(9DqCp(-|I z@k-Vc#ge^T5`~9eGxV=|6Dq%ykU3a@8;~Py&CXQBByqaiD*QItMBj6g2Y zKGE*4gni!<{d!)h@QTj~cW-C;IXtT@9{l#+tL@i=Z|g&!ay^?ku0*cy-8-o$x4e4Y zS10QW!_UsHjZ?6!|75jAzq*51GNaCRUVJfv&;ATm_9GP}5?xaVR$Tun=4kNJCt?2i zDT4CQiO=D%MGTnMA0n<&)$z?)jO-Lyu~{J%xiDdO+0{-lg&wLOF54e+*y1xab;B1> z1?=qvgUAh^q66`uQm?yk=(uNmNWYl6et+ab4Ur-F>*OivA>j!dyu4!fKgl2+d~Y#k zceQDKd5&|_pTm9A-5@4n*I|V0$6HlN01qSG$ZX2;|M^ofa9=DWTYnQ7rete$Q_6=D zF5Ov@{VhYNWp6?C6bMTL)}gjsKAP$N!z^Q6p>KU1-+ojbWU&2o(2BT&{ggjmP(Y?;FzC1w z?^jQ^ohN$tjmoKWUxvwb9}$7GK15V{L;bCc?lwg>T50d`6z1%T$&$Hn9E4`9%x7Hr z&ta*nQ0T$;XNCHF&e2AH>tXXT2c4Si6!ER4=XaP=>(3lbm^$qHS217Wv!CN*%yZ&> z*`W6v*4}2PXbbu2f`1Sgvtwbnm8~j~{mW^4`||WzQ)Cq7jLLz=0xK#}--2VtbboWf zc-OHwxW>j1TdwJv6H4V7m+!Sllis#A_4b5VMz=A5zWRP3lIc$h1h=!A10((>u6~!&~fR5&o4>r2z(hwg3o$ zor&GX>`n0IQnOhHfv8bo_4_YG-b8d>q#-lFx$U)ux`X zj;xQ|e!R3p5u z?gzw-G&4hF<=}W8KlVddDc6*BN{pkg>|LhJE2M16kbQO!JO|LC7@S{LkgjuB3ZRpM~ zilMXPJ_MKBv((vkty|iH*0|f-vbt~IvP3k^dl)`typ5&w(3f0%d$!*2%thU$1H)E(`)A#ABkexy+`5O6upibmVt7dbyetbSqtv!(*=XAU|p>9o#@4D}EwaV$t# zK0y-{!ptZOZfz0l`V)*-( zhStB>jemOU&;J)A@sGs%?xOrFWbSW!Gry3icIm*#0g-uC)_)GiFjEo~n`&|(vh=n= zk^NtM7>I?zfLwQv@I&+`|4f8m5sgkyEUc|>@p59P-Q~b75G8v2f4{2DB3c^xUyTvT z|7y%=lJB4AHOR0N{Tato4X*!!=6~_)nI3V3AANq-dC1?-%{s+2FhX7%nF}+2C z{hG3e&VYQjw@cj8=^xc?CxK>mwMGpK;<|Ej)}nhJzsR0B91-Qq-G zqWwY8+;#x?6oLLCs-4E$(>OK@+u(Omc~!+`>YvzcTUrd8c!~yk&{I@EAeesS8523uomADVW zL{5Z+S)s2fJr2PLf`%7;J`QU(GgBG^al!!nQg60k{=ulSdj=$ zKUc1*_bI zE=SAkqT0AB{mvUa_M$|S4whbZ-clDU`45A7M%rj|$_?)rtjMj9r(_wk9+#yo1ZF$u zBnUZD=nzpHEmgaMQK47#Utt7WuXm$^()lHTD%Pq6M?#=)$3H9a7#$bILB8?{TYJVM z=yvjDJfA-lc^8|*s(3lZ+8-_Asx%HM&$B^eQmlW=jIr*XAKamtZikN#ic$R2~sz$Dd;jO7c~uxtvrzkbF+O=p5u*F(q>K7!d9eReIv?^PO(mmrqP7bCYOHl|#e zQNF9Z2(ndXwA&n5ohB`$G2dBkknARG^&43r(z-zWSsy;`-YJ{fT^3*bn-JZkonOnW zZRdDF_Q1)8d|_dsNV_5V@!^imYLWl1Jh@ZM`0;^`yM31oJ04wmYiol$feL|(R!NdR zqP`bP*My9u%WdG}6Q!S!yT_xaVeAkvf#!06*EJ8<7RPC!j-cp4v+PN6irDF3t@Gxc z*HGqH+0*8&+?e*cApe=Wd9TtR$P8%zO7`5@~a7!2WuGNNS+6stM$x6?8*SZ|De20$S zHJB~`Gu`*Zw(fcZX+|=Ftn4|e8eL?h60_XLlepL_{~8)ulBva-MEWO?=Ti!HEPFsH z6x-{Jnkvueq^YI;9?YiAx6Q8x6NI&u32frMi4jILbtFXG(3UBkbU6n<5=K>HVXQHb zEgXzbaIQ)Akde=zM7;t|SU;|PDK@a9^!&Dic_hv|^aFj_xyDCF(bwGr+TPoWaVGVU z5CzB#FG+tAVaxm$cn|eL{NkpYfH}x#5Ve<3#5tO+yGOdk@ZHzEn1Eg0SsyTnMKgGP z(`e0(4<2roSqt@%-iUwZ;Y{4NjyZVbz$y!13P}nZGIe8^dx$~<_dfima1Qn#j_5yI zHU?gC@r66*Aw-d=rq9FUoj!;?Y*?U^kWYUA=1@}lG81O74pNQE;u$@%Zwj!w5P z)uYSd*c>KBsgMU-bv!;3^Ick3TL?To7Slvod4$h&NT|9?`}5_SS{U|EWax=`r<|d5 z2fY|;O-L0R-dkN@5f=wh&)`P%@RfdgetEBR6F*2Y{90~esW*tfAc zONG0;n^^Ru*;5r3Hw2cu#u#Z2t&(+Tw*#4wJ#4i|CEucus@H#1Od9-+srwR_>*FY1 z>t>j$3T2A6YkK;ldm((){tqJks7lm}Hm#u}!>0WSCn|?dnVK41I^NmRDjfB&CS+Z%R zw*BbF=L>va(HXOXi1}mfI5BO1>4(oxH&;AfM}Aroroo;u1lZn_y3s_Bi@2~s>roO` z@#DmC(%IJXJkDW@TxPvnB~3nKcF3S|>&UZkM_$;pdg_3LmYB^_=N5`p--B~*4~q8P znQB`~jmtQ@kiLY0AB9$iSztYvXn2*T-U_-Yn;)IJoeottGs_e{X;7NMJ?DxR#y7nQ zH2FvGUJ4Qbxr;y+Fpwn)eJ925~{T2_k);?a4-a z@H6DAs<`h_LVmnwz@kWY*Hp8y43O38&N|U%ED@@DoS1UC;@7-s7P6uIXY+BVKQN!| z(nyMp4K{R=o5OnbGOVmal=M5rP3J<=G+gjS?DxD9z(aR?Jp2Hl`fg}T9l9-V$1h~ zKaJnK&a{5c#Z>Q;l*2bYt=5lS(Yd-*6Sgr|z#J1=<9gzboN`5xX~KSnE1d@bpzt>q z#|O3bMY>2Ox=l#$5)|${qi@UIoTk#A*M^YuB)D)bwqs+`<3BTFvgMSTA3iHSOZ2h2EniKu$yJdD6UGKGqP)n)o}Wk!cHrJm)%`juu_lMgkCn_s{q*NH#Oi~QLeq9(rdW)FpEwVHE@OGWz_ zPIoG&rJsrXOy3boH#5C9oL4XAZMvK>4rt=;v<+@5_GpXuD9Uu{?Y?Wbhpnh_-(d1d zk7ZuaYn{+zYcYo8F|JRV=SD>>uNUp6(Yd&H@Paryx4Fy+q%3QqTJli=uQhvonN3Gb z(TtAN<<|~PE}B1z>2u6uOg(mlsrQ1&Fa)EGN{kDg(A}nuSrti?N5{YE`CMho7g^>V zl-%HrvsKnF7tQjXCry1W=O>0l-z>89hC=U;+J@xNPgyVG-4gklU$n<09=6+CqX$#C zE$9Zq!#SJdt6`xW%5HgY&B)6OYk4qie3&Y`RmFwr@a^e&woh}h;pV)ditWIGooca{ zo4rEOg-ViDSrQP_1zGJ1-?4^bN$x`}HNQhG*frxRKYDg2YtLpd{t`$&rZ?~A4)|Il zF5aWpdfu1Bqt6P%Lt7mU$LbjQKCcNdDBr2L(aDW<1+}Q*L6F75Qd)}ojG4>6kb+d6 zi---q;;KlL&i27(BF(KWf%ImlOk0;GCl0tRXPPv>vRyS-tI#8j`*|_099M4FK?gkw z_c2lDGmJe)hVD?nf#T$(oVKlB@f|@oA#0*2n^@IY*B5k!?TpRW!}fkl_v>B!RfVa4 zs@{EutKORyad)pgT!_*dJf_MyOAqQtu{)yL*%JGi^-g>J@X_^)C{|^EXE%TVAobER zFy#FB5epP$@6JANw|F(*5v+g5qqu-N4V_FkUG?a7J#{4X&SIP2vfhz3YN2UsZ|R=0 zi}%wj$%7~XB8E?7$m~b=C|*|)UawTSV`S%R{8qPLd)_ajIXlWSuncn77;-$_7IGNK zG(p~v+49ui-?<;)62hM)o8Y=V9TPVmFOwo{>I@cF(Z8xOVpw_z^a6XBQnP$1ynH6~ z^!T)L@&dM^q%NvufzRbWN@?8w_mOA?!yM)3sdiE>CUbRSB71R;r2pcpV z$!|h62Ag4Uw!%PvP|_}^AmiqaGlsL+JfQ%qmjwIrOLk_UYHX-UUgku^C)c1WsDt_> zrie0Mc~eN|igl-n`Tna;H5LLsA8tF%>Q!zAoN)9qo;FxMBmFLm=4WKOXVh}Fb9_68 zr92&|=PylmT4#ia+qPHf$tMB{3ERNBJZY(VP8@d5PHZUNFSp8x9*%2X8zrlPsA2P< zX@p9VygpI)K%j;fe1tGo*!jdfK5!j{zepGh&aU$<_%@(6qKmXpiTRxkmHOq*4%sly z+7Ij4j}CAnWe9i+cP^JR$hi@33|fdYI*grcES?;-7fZv}et!CURP+<1!^!zs5?ufy zj1FXZHP-Sz=>d;>6oop4TTbge4Zk;Z-})NO{6uHL(+zHA9dWApeX-6O(o`5VB$VcY zZM{xtHN8hIm6emcx#6ic?sT?}hNM3w_DBQn0mpi5;EXoJi5cAf@l>U_-LW%hIu+*> z1;7M!k8&YrWjW|Ea8x-P8am8jajawy0YT=KW7g{mcCNw?^$Ipb?sQ(lF+I0d=_UVKVr$_|>EDFd~xkPtXpu19E~j(oC1Wy)rx>$G+W214T- z;ZD!d2NUGj-dUOd_*W(sT86czJ5n&i4}9+LJ1hNKLanJ2=&S{x}leks5aC{otA$?KOwJ zAa4U(+n~yi*?OabIGtW1erWl0YO{V$!E#s9w3eJnG8h+`9x=AQiXq0_$emQNdbbgU z0~uZ30$FHjl%bPFb5>jW8euCw&l?pR8{2)eYEG!phgWQ$>O316WS=<=OV zkTrM00ukjDbo7mf!Bh8vtkr)6el22uQz&V2;v`pYPTE+{N7H?xcTA8Mk*OaLlRg>r zj{%yXTFaLPNk?VuBt`KBudvV);U!Y&;t%7`M#zMPN`%U`HGlbDx{WQj`zFHfSd~Fd z-|qM;HcNy(y~N*KfTwuE*OejZazKuzBsxi6i>aeuE%_a{y26Lp&^_LIf8T)^Jhq27 zy=ECJ)-s>*VAJ=({TZCGCVif$P4WX@!jwD1s>7fP&A%abF|UPIyGKUEkHV^_xJ1y8 zo}TtFisxO5!b#N3V4irxALXRF~_s05#Tw5u1Ia;J4klO-yIoKAJh-G_5ZlaUDozBy#3z+$c8 zCLE(#bBCQ=~8yX6R0x;1rne9)~Bld=NI9sjFwT!4Y=0jB}`?n5d(g zBcLzYu}crj8uug2+-IBm)qS~Ye}bvyL?f;Ct(Ql56{eoRg-N4RwN$WYG$kYr76 z0U?7V1f0dxO!}{{JP-O5C!C_<(_k37)Gm7I#@~lJfA3!>!(pfiLKQB$z96dLYO;62`Y_wU&B@=_2qcnd zx>II6wd#V#?W2-o!s$MCmM`Rt+gDZC?7hX&e0rvnswqSNI($8r!eryQ17#Orm?ife z-cYT*c+G#pNC5d{Ff>9VW1%ktLJcjs!_Z0Ae6OMjdm|y|_+6(bGukm4&imk;6+D;S&-lVyjjCMR;`7q9DG{WmfFtvDNNHxj+nJa=`mSJxy{Y&v*@8b5JMamemZ`B$| za`ne0sFqyL&uNI2#iau8<xt`KN9mfhHEx86Q9b*MoGIAgnq0J{fK96t4={gf%7|a zu9cFrY)O<;*^1&Nf_398F5m<8%ebH5vrArc$CIwWJ$zOR=A=e$ zX^UP|P9TQAVIKgcQayp@<0!(DMm5(SEs%8k)DShUA_ z3x}Ap>r9CN=$d(h5TbRF`HDro(~E(L_*>>D-A@2ht0XZhD}~R zEaayW_G&Nrdg3wRcr@ju(Ejxk9w|rq%0BshsKq&H_^u<>6l-Cm`O`1A+~dBiB2cw| z^MI#kTSq)#4V&|@M1Q>Bo&H5u3vc?(&eimY70Z+L(@*4d<`6^Kqh*s0`de&Duoy2b zkF8()dY)9}m&?;e7~cqrf?{`(}z8=Z(r}BsvQp5*@ z?P8HQN{tm>Y!xgvc$*EDpKv9do_5NE@Y~Jb702$+w4XRGQcSCL#G@#sVx!#B&ebKA zH)ztV=eZvl^9ydXT51dlFRFdZ^b0qato~-g2Ul6L^$gdHX{MzX zW01QY=H4N2M$zftc;+@DOf5IVYXB)z)R~lmWwY1xev>fbG&FD}Ip24OUaik=?s-^! z4c_;;^#gng?15EGl~1c~P4`enYYhEwLX4MYyVPu2mpcimA`e+kWEz8Q_MYN&9#)%n zYHUz#CQ?*u?a42^fA{37Zs(5=RCB4|9)F{3c7IXg`@PGMPGN{-=-05pDc9xoi zdMsnA)`Q3e=pw8`2$_S`z)yV&SPvTw$?AMqve zSK3oP{k(~2zakvu(0Tk&lKN5H2frg`JYv3IF_r4Ooh#e6j_!)hXY_>LbGOE!h|s5Q z@*)1-CZRE3r)HDJ^9EVApENLi&x|K&4xes?PFC3>%Fm~SC>h<@RXk2DF(xyb&GG*+ zVXP5=Mh%hcujg6LQAv1C;g!;q>Q#0Q?GN!B;j%}?zywqt!BJ6BU~=U8eRVPQ35$iw z&;U#5l@~xT=T~SCpv0zT-h`YI}>EYAv-rkOs zSOm$yRMVpa(5UDi$L>lt#bn!|v1cpSqX$fyOQ_3wl@8g`#_~HB<)>}2#yfi?5nQjN zqot-q_K2R0wiWB)iHY~_?)*R(ODUj;R!&9b1JI!M_d@d+IjLOZSGvW@Imog1uXCoh z?30m`M@K@!W##o?|Nn&nX$@1-bCCZrX8{HZ@=L?fKG^nz`jY?b$~Os(%GgS@wmSCjM_MkW@Zc%{P(P z6hkATp?!1dF}3W*s|~xuv|73uGCE@)7Kio(>HQ`#GF>K(mmK)VG&vSzLBWjJXwi4M zF8vi<`Te>swWVb99MsG`njf-*t2%2EdO#~ENp1geNoBnAiuve+{GD9%_Mny+P zIOKQ47zm*n{}*W^RflbSjH$WvZpzx>6Vlsjx!_lC3>_EcT6s(O>G4%rwfFo(&5+%N zpXTb{5Y>uS>3<_&X45o4&gc?)u%M{pDqiW3naelX?@Ur|F6yaJEnQ}Xy^`wuxBA_w z2dFR4OaPDuq{sdoK%?2D`SV@+s}IM1eg84jTXi~tk24b}C@G_&qu=QOfi7BF+V-xl zKatBnmmNWXlKpo~60)+!vp-A|b0u_E&TjC8JozOgB>r6TTi{yj^np0uuoQjYY5<|7 z?j@iIySmPV?dlf#Q@A2#Dva0cdH+B0NI$2pUn9l6aTh$SI}!vI!8Cw2Z)j);^yX&F zsjlXRMJ4LqNYEL~dLJqd0{yF8{^84_03b?!+VKeqT?bW5gYz|ZrlzK^aYAiq)#Bev zN+Qn8%v5VK{^jBC)zwf2ygQRtFIWPt0ch#z{~kksUnQvdE9Ut7<=4na5I#CG;{DGV z!N=Mg==N&$pGdqELvWTlcy}q`vJh0Ur7Pl~s55o4%^!y1l}mV_~q5udb-1vqXg-;19s=DcSP>H!X##D&0e0L1QF^5yf!RJyjCSH<0mo0 zrbTTc)tDhSQ#xISZFbD+eJ=NTecMiBY4hAHq%l=E2jBE2j;bP^`Z%Qf&;N#%y-5CN zk{EZg8!;L9qE67R1d!vqC$0CLt#dVw2C#YD9%}7sutzDXY$={k8k|ny>w|SxueoMq zrghp%9GFVaqp;T>7POF;7eaqbmEV1&%+JL8V|N^H{mZ|VwLVF4O0TzX^8EPIC|v5l zZlQj0wnlwPKCjuTfc6kJ5ZJpN#NC6(>@U0EMA%^p3WA=l+LcRpHsuNzusJ7XiK(;H z5%nV7Dh-A$P(3-zs9?HzBj|Q#!Mf%w;v~=|rIBd8rDr7D+YUZXZdt#x==Hi_rA~Q{ zEWSd-IdX`D+EgCDQOG5~u=7=SUTNO_K>D`p=Sjc*Qwsz`sO#~D;!f~E=8zN{ycV*- zU$^n<^#??9BdxwqNV$=_zeM=g3+SovN5X)l1N5>%%=?v=r!G1H|9Qu!rBi=dEh(`9 z0+Rb3yYcr~<4yPGp2zN(2}}3*jkigJJtf*5Xr3n+4Cs&TRMW~WyMTU7P`^9tp{(c$ zTN~i#3Byks$V8!2d)gI*s7$V)oky~7BR&EExT^)Lii6mDClwd@I%mV90?2CCTsqZm z==l>vX9r$xey*d5o*gpCJ;Xf@8IQshp6il_xUJB`Gg#4*w zc~wvRFWdb>gF5Ds6YaAh3JxP1S#NWpgsxh~8$Ttnz}IK}ahyM~O@ofC#_$ z?LJ1OU|@-Bb1sGYqu5tGhF>DvwJwR+F~4(S4?w|tTi-^(F({nL1+j>TX&n61U=@fp zkPS_9}$LKTF&rY2@NjpcsD8tEGw#UEUj3$K_*R~-Zv zWb9#Mk`yC@0guvk(IfIj>uovJVgEA9p?$72N$cj4;2;-Tqp0HweC|-K58%IG`J;th z)ih*dIVOnrtqyW&v0lFnMh~8u{tTs>>pcgKCfL$72uY!{b2n($I9I|s!fk7}bSc*y`Rg%?%JCW!8?XL4OK$)k*%0SoG zZmA}>NZgF1PTWQt+&l^c$g)BeNO!Ir*V%Fj?=V^Yb4!Z<3z>0_8G}vZ$fP4mHd3)L z{GpX?J9CD(Ztir$p87NB{aU)5w z=Zs@tzLz=cv_~Zgg^iHg+GtRfiQ~rIUD;F@1T#Nn2wXplEGWekdi1bbJfGp6yL*mQ z7Yh>)c@>1!5fCD$?MieV3;lVr(lB*69QD(eK8?okfx2%L0=I^dC#A$l*VV>ecE(2|v3{+|d3VCL(f&wvRY^O6+a8rG;c#8{>4F>tA`r3hgw553>$*Bv z)nyKp4Ig}_&OMaG@0YJX=5penCoy`c>o?YRIKgDcZb|fHEJ_wKFWPq4xoAuy!(G`*yit0%%Ut#CKG%T7hk zr_|@Sugi3aNS_kW*cb;SYbLzMjKrYm52NOvq6lvmGqeqk+Mip}OD^+w#0?x|JfNBu z>gWy(a_sAnwtq}RPU4R7s(mu*+~^XPB~-3M3C+z?$4so;)>mCqDc?`hIWAukT}+&D z-^g^q>P-1)AyN%CLzHE_VR#%)xV~<ePB>59y{}Fe(x$El591>ve&zni3wS)#em+9nT2q#yjE`c=Hqi z1D<{g4GvhOa3_Cn@HQUomuCUqmUwve0`1>R9U@t+32bz0geX{ZO22yw`fEM3J z_jTB`?W+$7(%6E$tcDf*3;*kL_wsoVJGLH}lp#-cjBwnA4f8Pd`6v zz8||oX-%PF4j0CqD!F(^2ckAR6*c?i;Dy%Z6U=3%VJN+D)ACc-epcLRj zD_#oh!4B-p%>3YE&%rgii30Uz@GCp(SWKxY6|PZRU=Gid?%Pn051Dn{p`u8~&Z@@R1l3dKPP%?CKWL$jyWuUl{OOU7PnsMi_g{73k z@rVW5k{~{%-ZAgOXYvIovwCoJ5dosFZ)Y9Z+qV3)qFH1vQ1@5x60GACC+6yrA(d6X zG|qw}kf-B$>p(|Kd-8)+txxK^<(CeYkU3l?Jymx+*|)E0MN$P1RA=RQ%pc-pJ(l#+KC=yh;+u{V%`EZuH zBb|?7vf67F@Y1@}PWoAUML;EWclua?cR}i}a!n{GXLoqZ1D-rB`Ep}l_p3Sc8M@%q zFVYOQ8xXlt(`9)2FequSB6XH+52oNpe;FfXAHFB~5rz!;?iRAe#m>5U3)7#*iD6!y z5g8vOAU_nZqEeBU71cU(b(|@W>v5pnYV>WcGGKb!B}|fI<*LiNlW&{9Hr-hHAh%A) z=f8cd35m1(+$;SH8JJP$>E0u(*E;LjdrHb!9OL5yuRCJdVw#mqHz}Jpf5fJ~_#sWO zwYKFgRcF!&a$3A4u~+w(dXDyeENTQtBpP96=a?W%>o0v>&Q-)QSpukOcH

|roXC^MVWelO8*=dD?PTfpdb?)n43+OVqa4qaVF zq`F`yo!feV@yOnGqx4HN6>0;=`O-+shVm=nnaxVI*D%Cnt!qM_05DVclzjMU&*+Iv z__Ix??%%3R)Ij%^wJQ&G$L%zrK+$rWPngKac54RLH2q}DcaeS0l8+_FEtnDI;L$Ji zD{+S=_ICBQy@S(yLCQpxGq?~^Y;Y%@MeJS z##e&qPrnrE1)|%SxNAQxC3f42R!yeG#XhMIyQ~m@Rhqz+nmL@|J^p-(qf!Z}dQs(_ z1>U&^R|Bx_O*WC{$2tc%M+_xD$!hf_3I`~x(hxr>fQCmC24HA2nM#JMaNX9XA%o-r zy?!b(O1toeexae&wy!kM@*}u%ffMAsl^up6lzJn948o(m`XL^UPhb;ir-JUy)Wz56 zul@fn0N>zm!nFnMZ{%i0SWkyT$bCq6OZsD*Wh zwTTVtuVM{6j1&r4|m{;Oj#}nlf|~o5@;!(F+1ifC?f7sZEWYz z0I~c#;1u-{C6}7}_s2v8+gfbzi41S{g*`Sft_i!fP|vQ!Qz+%$`QqvLX^f)O62F~H z4=vHzJ7VwB>mZ2~HEtlUUcDfpCroqUi~=9Es_4hsyAc&D&Vx~^G zS0^Hn!*T(vdAj}S>nPfY@~lu-sq}d4uxtYO)&aGL_9i8}Y~7(*+5Is}BMohXGOD5N z^RHA3kJ;tkA?=Xwa2vMd&Z3X|gcu$Kn&o`(bat@5B$_fuF&@p((Bg|F1S9|p!kvX3b5}; zpy_Fo*JxCEW$u#2Rpdm0fW)V$WQFDmjkR|G*b2q~L(7`YCNX=o3N@IPHJ0PjYntlx za@=Zi6)jH+NO%EvUnVymz8Onp_+*;+jQ|&XP(W_!%Nd>fk9EMtawRsJF&5{Dz{M8U z!^-fkjxslY?bA-5JXb=gb*PRl?O~Qy&!1_nFKNA#)?n?$(CFMq_OtLtk~Qq-HlXw6 zN^<{?&cu%^@fU{P;ZO^5{|izbq{(Y_;;dV?mX7%1QWSM)Aru)d3eNZWwbLH7Y>A8P zO*wTsd>hn%zWT7vT4s~s@a5BK1M+wJ>1C6(e!g{C`v9cpw$AnuyYMg8_|W*qdcgvH zpUKHdQ2AJj=SaDzL}{>O49ZC;F9zLW=b#R>zrk|oLsG|t+O~h+jG6(m%O%j@c0&E_ zFl=edvX(N4^Cdm#_i_&F(${6mToCvAAKPZiSv1Vabnogg*mW+$$55EQBA=)4aPjo~ zw@AFf(D9ysok!5MrO$vuiFg=WT2aDOUmc^a$K!&r&WDocWh`E2H~23`+=2&j;;1%8 z74@VEI`ayYqFn7VD_nGkVLM(sG^>9P^G?t1C=07NNxlh!$}DSt7x{^`ifQ@`C{}}$ zXFk1B;Xth9^q)Gq8S@}&dvdf84LeGclJ1$MqlZTQvcuq0Hw(z-Q2Wnpy2dRH@9|+J zWa%(fQ|as(-`#Jbw%j81td57mjkr$;&`PkBLNy^xnY($ka|CxM zoIq}Nnp*|ziO_M~L&cwv5_u*$OP%xi$)Z)U++MzdaQ`DnZ&|g+u?PSonj3mlZf`S} z-SqvO)>ZHFd0X35TXE+WY;T1<1H6Jfu{ZQIq`l#Ex<4t`0#LU*s1RqJlA8HRL z{Mri=C-_d6cW|U!MTV4hJIJ4!dVW5}zD`!&;jg8)APh74*tJx(0ZZ93{u_NQ0R|Jh zm|0ng6nNjxMCXV&_4Ho*3zVr3K{lLGQXk$#cvSP;nWL-W&yrp}ty}WF3qkUC+bk2q5865F386 z;gF-scEMpJ9=dL~6pS2V9t2Q4>!MEX0{txcb{2&;*vmkax-Pf!CJ3R4EBTzoBDZDE zYKmB#&V6Z9mNi$hPB<@&QA4Q$_n%fr7B&K}zJw)SaajPMDM!*fJIvefYA==Rq?8R| zWzwKm3J5(#c zM&3eM4|qKxM!%(5e7%O{hI(4R&2~HYdXT_hoV!u%aUng@TgflJcSZ-Z0W zCPx=P+tLP-ah#OKbu(-*OiBg4ou#;I3di~;AhnRb`E)Ir44v@yRMLVL{9g~`gVCe9Q!yhE4;5)Z!DdnzK-0jj_I3y$#uAv ztn6 zUGbVUMjB$bQrTm0v8GI}x~l(O8+DsE01L*Ovk_c;JIn3cUF-LcdGegi%O(fVypVK} zsV)$gd@HU*TFQ{c@^6uv)R^OYg9xHMSj7x?TB=ql$?^e~KLllugGY^ej569MSx}opVOQcNst_t#DvbWX?GqZ-qdITt=7Ao^OQo*0rKGyPsqYRR^xIU$ZW?U-Aci3ZdCkWN$fOy?nA zWS%I>(w%zuMwKx&*KV-%R?#W8u;FbazxB|+NuFuFOC||zUsg-CzU^2>IwR((Ao!L7 zh#+Wl_|0RQKV9Y6Ph+Nc)MSLuE;-a(5JPWOQTCxFzGHXwfgYLvxS}bTapClJm+6hB zEGS}Nyu)#Mmt5&Gdt7sLY^VLQNz4>en2#I>;ElXfY&AFL{M8nA^G1^Lh7WVF?W6Pl zZs)TyW?}u*%^iFs^u0cH+s39@rkV8G*D|&gC28X(w5ab{Eiw;_tDZgHzxTZ=Nv?d> z#PmWy{Q-l&t%hitGT@H>`Xi?ut{$%rL(Zz}SFvJ&dzlv83Von2Ba_nysU=Ig@e^D* zYfDkrk@Pv%q6pgl#6y6}!H)z~XQ*KeLCwd>`XQU#5RZ7{QmEu+U{`*x68LGD$#Uf9 z2}db^ZVKs)P|yWBzJDo{yD~oD?`mW*Wu1KyWt|H3=8@tvh=Ixs+%$A~kbKA1?z~kG zQHp!MregI2SyjdPB+XeLr@hOuyZj1ZcpBNCW@sdc4hjOX4pH?Mqg!|k>XToZnT&oJ z4)GDr;!19)WahjA`6P||L79(-A*AbGjgydA0U~iJt!Y@~3JClYz;4MKM7xxkkBseA zOaqf9bv9N7y7AkIFqxR5DQU$W?tyMHiW(C3Fs zb7Y$M04QQLlIEsbJ?cs;U96Mw`U{%t2NdeC7I4>;W0Hhskxzj2olx5bVCPWKyg6%D zMSyEv^I)!%Qw@~245Jcu@grE-xl#J>qPX7E66&2f2-TDoOA+d&7%aBBpLDe>2Md$d zGLMk&UC9K_0q-3=*lZjmJhC?iP4jpz6D(uqAaxfEp;@1I$>hq_F*- zy6IH$1)F)siovI*dq$b%J*hyEaFk1s4*aH)-qFVsI7d%nh-&fh zD)TD@W{V{3S_BZ^h5KzTDklU(IlUEAA=ez;W1$j5lJdqd!mQBrf|Gz%#L#W=as3QR6M^MU7qIh zc71Uf=CdT-vurt-C|Cc~wCEbc?m-NnTYjCuZPwM2uc8@{wW?%R3LOxifuV$XKAG?DFs9Muj5xe*6nqpa*1cK)VeRQc zo@?$J2#mwR%+{qGxk)%*KCa@E0#1iA5zJzzTTx{Lo zD&l%TdxpmaQ`M%(=kG86W*C{N)wlb&1H_Yt3Y+dzna#y>pt~8m_w#`eZ4X=TE#bgZ zV9`LBe}U}jy|nvGi3OXoKA4OuUo&0hUahgN5Yin?rezabqa&C;Vzxuc9Cz-Wag7d+ z9aLo3ufg6%^+v)&kfxK(_Crm_D4(09pLR@rCt-dVt-bSAGxvg$lfp$|HjK z!^f$O``NYBif~Bj4G1h>eg@mBaaI>ElWKm=l&WA|HR_#r(Ea%=!(N?vZ3$6vkRh&* z<_$-}gC?lN)(Un7RvyTt&doYF+Ap_5U%oxD^1974@BE(PZqvQZY=F#Zu#fiqk3AKQ z0K*#GBLwA5X_~6%AT^hoQx|7GsBnLsoyy9K@t@Nj*Tg`sW6G^UcehjwRy5}-g32VL zEomigBpzh$&aK@(IubJ36f<`fx;DotNJ_GA@rTrvzvxM*5lv@4@9ilTijhwoFLdQ_HdbJt%^mRg&A+4o zo&4>#U}kUdhubkP?sTvdmzdxd4X$-(14RdZZfQv z{mG_wXiri-&~01Xd>b&FEC{9=ngqovURoNVDye~Ms1fn~tR2C)K2lQKF>yIyVl|=A zl5Mc3G0TY4NzkP4X=>#MQ+>5TyA?D0!c?oBb$NUnPNE|3gbdXaWl4y-TTDTnA)}LR%VfBxO0$n7;a$v{va`77mwQo(Xp24qKc1 z=?~y?e(yjgc4q8#c640H4rN*=!HFFP<1`MA}QA zcDGUwZ@;?EHBpS^<;+Q-o+ETphtGGoEx)yvy##;nxe>JQi<|oOih?2XxCL;DeJmy( z9hq``)>>Pud#1LkqcgPvti0@SX!f>vw1hI&+*(;eQgJTNP3X@;~3#_huEkKr++ z4eivx-#_w-i6N;QotGD_-$)`0(hp$K3w~7;M&1KIy|)o?5mXOV4%+E?ilE5;!>@9O ziY{J%E63Y7FMiR^Wzi_<_IPv1C`P`>Wx!dUzgif!ITEvO`@-q*lUwWr_W;rn*%o|h z@{CN+8LBx-!zpmD2nK4WyQPAo&;L! zjQv>JVo9TI#=>(ua4Br)&%=o~48m-^-CY}BH~zG(Z9%zCrymG3wzC~E9af2b<3e@7 zZtF|dA3wU;82b4;RHoV0U}0K@?IR$tdz$jL@Dsp!iYtpqyIBv;C4G>JB*d0#WkPQz zcg&+vkvZdeV0Shi=3ObF^N#|`^{k0o1#o-BQ9sUXVWR9Kc&HkEPxjdvPG=?9)vYt% z9d+O9$WY;ktR+dj(yyUsS+3^GwfKCHn<(i`RnwbK`g$(5^ntd6As77_EeU0eH3DOW zO`>;o^J~MBQ%s^ihh>Azbt-b{S~WW5c5~*UXen>i&pn(Plz(R=MG<8qZ7)1vY4n4U z{S9SG@_)b(JMR!bL`9yz|2wL^653?CSF!pPFQ!yuY?`#&r&O!2*)vIfJ=n~Q6)d6o zQTUOm;bCc;N%(Za{AxvafC4>jEg>ymL4a`6wP%#qY%~#lhl(SXWeyGhzN|=L3!@7# z)3CrJUOVR!5HfkX8~7!$s1?fJ4Qq|;GGw+J^J?#VmK?uJ)f`<#1mRck0cTFZ0=)yx z^-^=2%%|4|y>VewW@fY%e6hz-*ZYlwZ5x`G^7JZullcPzEn73W(?XzC2m;?|3*|H% zedEQ2hQO(Y$200Cl8)#>Xk_|N5H9N=rv8&bvr=jgPk7HzvJ8)^18tO*`C^`ROj|m8 z*!|+FZyMH2v#Sr_&Tj0M+gwLN5~sYLIptMv^l5EK7Tm`X&|}KURuw^Qce|XerAD0! zaCdBUpU6W01i6m1soj0>f&nwt&m*4@|3z|9^S4(qubBEUSDHVqTMvc_=7hFs80R60 zaD}4MY3MH5KPP^YJ{}E+7nSxtc?22y`C6j#ZK!!eOg}Lmn$G=86$aj+`j6H6qn1qU zhf5JTf{MU7+q)W^fQhS&>}rARgeZ|cahWlV{q2c|#TERA>rChMFSRfZQWYfYvZmXZ zGZob@M+lmR!K(93Iq$h{(SngPb~n?~-W}=qWuZR%SkHbm<%WlUP(L*$L&xVHm>@1W zc0DwwJlE_sF;d}u|0Rj{?M~bh`7Q5gb5qjbO3Dz_ z9s(Y+ZWT*7GLd%?+}}b@aA0ouoDX;U(rG$u>8vX`mV=yiefACXN%#d_GN~KOI?A z{!Mf$6Lyd!hp|*^Hfz_$^)U>W4F4=vG2$3IU9OoDPidETV~^}Gwc89*lA2b{=8~N# z1r~3NtxJ1uTX@29w%6F|Yc*p84x($G!vza5i(E7pjy`LZet8#CHn|Fo&U;HQc+ zC@NsW&oUp;a+(?o0QC-&5`Vf65|srjk8NBe{Au8mPG}vqS^p#$E{0$4L9JH8UQZZ* zT#ygGDAsM1P(J%*Hc}*+pxdcLZf`htC3xn0$qN7CI$Aq+!>@fb0;M@fK1^wM+Q~^+ zL^l^>_Ya2ZiFYC)H~*^-0)}(LCvuqG=H=<$xrg~V{oMJ$gSw&8X}7jw;=^ba zBtPE&DU~BDSlOt1b>o^CCqW(Nw)~|u-p!^M*8Ml_K5R{7)10~{mikPJ;4r0B>U^V(TG?fU0N`W?2rQmU4oo~zf>a1eIrW4hYA1rF0 zeMtRmuuKtfOwp9PAW$ma&A1UQ4!Nihh!q6NZdseK?G7j%Qm$_g<%q zvGrMSm7uDZqSnVY3$@8Jcm*)aP$VyE9W1Utl%mp2DO7lEp&sWom?Np~@~lYY3#j|v z>sY@&4M@dkXux;m!I)z2q^{LQ)#P->GVT1UDW#{LT7{!c$6w?uddpp7&%S~Lhu&I0 zuRGnzt!^1vQDT8tsFCR2J1GW5;9^{Lf*W!FfVw4XIo_p@F7Lr}V9#n*e|qcr@_*#e z!`vykqh&R~%yGjc)VKtbo@)DAPTOD7HRGX#g_D^!aU7%NypmkRFIdi4%UQ8>lFXU4(?g7Lm8lMwemy$pwy-ia;tYUW4}b}(o&uN$46k=OWRoBt?0!2O#j z{ih|8A|vVlzYFmDF(1N@H*~7E?!z`?eGKis6$h6vY%uVKT_r8 zb4OFALc3{MWbLl4Z=)4ixhWiPEZ^qMEn}|=ey=W27IN6GdCYQ9Z6n0cYVSthNwM+I zEYQ(C8m0qCY%8y>?Q96R(m_a1qUeXm%n)dCtqQDM%xAe0pD!G4Qo)L4ya>0xBC8xV zHa?#7Ui>N(a(U~aQs3Xpvj ze~)NACz$cr0p%P*TEo&nQ2*43H#^W93Tyhqhf4Ca8wK%%Cqi9Ku&+6-@3>BX@gneC z#NcTB(4T9R+j}!BItD350o#=Y<-p=q_OWbBhuJT zNJm11ay7Vzrt7%l-m8{|DbDasXj26e>TSJ~%>T#kiVf#qWUK7Dd^1bc`$_@UMeY`W zkDLO}&-?0J@W!KW3o7+|Qf>wTCtOG>yAqS5(*dB+;O;ECictGzPdJfE;*hfL+`)#r z3_UFIQtABtrPUq5%kI$dGN_A1u`+L}hz{RHuLJXjC%t1L zaLb8Kk+U`QRx7WxM77v&%?N-_8OK86%4Hm%EbdW|{pTj}0VT^^^a?zJzO#CJ+#<45 z$m?xm>F<=a{=hAkc@qn8gl)+cy|*a8)VDtj2`@aR2%3kx{%%(eV;r~CanZDyWrY4+ z;Y<0v#guZMue0`@{+asMZ{6uoONHa@wrdR?fcVMv(S-wlh5;>~g!5O|pWJH)kZ#xp z$oH6O?W5?IpU(aK!RMRt$1nY-A_(p-ep~V*t(Y?_Q=@n4jm8@{-D5L4lM<*d*6kzZ zTyCu0pt6%h$*|5EaxF?LmIui;=jXU)wcIr9^dHCc;PNeo8UKiE07Kt3}S< zx00)S$k1Ku&cTcgrmPWj;M&9Y}&l_G7S*>WH8wkktu%p_4hLmBU zmOI$gK;*B4;zS;%FXgLFi)_Wj>*tJ*gRy25k+8nwYt&{{36QtmRe#p-nCuD<(aHyq zoz9nn)33Z>+__BB%VM^FGPxpca=#X4Yl=4Jxw?CYGc-Xck*W*LlQ~hu$B`ubwraOP zv7eco9)5Gd51glB;Ke&Z98W8M&5#Sr}!X+Sr{nUZh)C25uPNE zM{a)HoRsjMa^)&@*c#{cpbLu@;|4}zX{^w71}29UX*ZW}Q+Q=Qd+w>s0-dQ^TW$0+ zPF!uY6EflImwHm-bB?rEJUjG4dCee%Xg8qenSpU^2{Y);J>gqM@ z#@9Burb^4>#Wd)SVD5ugUCzmpZL79Y&UGGY6;z4(c~4y8@q^J#YE*kcQz_J495r5@A#}wIA|>|zTocU{Fi0JyZy-xqllX;&hsH7#u&fU_& zaAr5`Ua#Wr$RtOBY^IOEt}UXy*4iN5Y&P~g1AwClF!h*J3G%G*ZDw9Ju}@c^sb~kB z{*1drj~p^l*pYyhn)@IWv01t^y!)%cN=*+RLMp2qH%*I76*%?a=VO+{N#@a)QA1~iNEoXX}Wmy5Le{Ya% zndQCBxBoJPd@wpZt4LkiHyM^&QC9aA3Y)S$)K7>{ z#3d4*ZCP^->8H7;FoYtvn|9CCF#Wq$OZVEdgLok5y}P%PyT}C)Df$Z_sT9$6$EmqI zgZNlFE_Jj3zsc-+A2tkKh+AGO(Y@5!B3-*1A?sove{KO@N)XU(u3iXa;L!9wc3Vt3 z3;(d|C(V7U>f)uQyO19EKqbivRD9@Evi;Fg>UYRSc<=TQGhb#_f1PjPEK$uSD9P@W zPG?9iG5ZSW!SaH2u%>?AM`X`GwGQrUcB=?R4m+L=m z_MFICX{FbRS;%1(sy)@btUT$~r#D_n}s z%~JP4Tg}B~JZ+yXylt%#MG2Ph-3Q|+^KqWjEU}Q|hrB7p%bMpE{i_8Rx*QS?rH*H< z=yiWzhr^n~zyX&jaB9b+>FB=cVX`6F0*hv-lMyx9`A^5%3OdgTp6h>mq_e_E3a8Nf zsC9B$`Q(&s5(A#Y^_#^sIhVN}RR*3s_@Zrn-`lNm4I0=N)r#+5560hOh`D#5d zzxkv|26Ax7YoaN;6%*D73(P}Z-QJP`&qpX#or;)<8cn}hy@bTiaPDVcAni|61%q0J zI#*8)atHb!Z3x`=1!Q4p?QF~n733gRXdk^Vea9N!L;l$sg_L z3q)VaQM{sIzn(m2_((y-;P&&Cv9O(o{G0;+r9Zjb!VS5??Oro~`A3jwZxx!ih)xi= zqK%D4M1-2r3gMmP{o2|na2e0OCj2spMKDg2bCiTM1J1DQ#Iy8)gW%d_=@sp9txIKhum6 zH0{ewNS=A~D*%>*Op?>o8+T#!SsE)lf=5tZJ6}FOjk8zZ|Etukp;7A*yy7)Z8=+2^ zlysf??amdxfnYi3%0x9z_Q2KH+qb(-C}M{X!LT)DuJJWo?rlqnWTz1II*NJ^gMk)YGz( zY^odmmwcFMsMJh`V}p>$QTfFfpxOadKYm!GIkw3@7`T-Z!A$gL^yWxg-28-+%73Wj z^Q2ez?_(wzJJW3N(SA0U%r^7}1FZ-)+Rtea$aBoRBy;O&l6&`;wi|(cB{03oZ z7;A-j-U2p4VJgf{=Hw(Okfj!Dqe3<|M-Fa1Fww5fzy$SmGW4mY#PTb zyPu=$^1`3@ea(pxqb>plpqg|$HVwZ@nqoZD%>kBE^ zrfIy#eD#HEU+kn2a^t4y0EOYB!vx9Kz%J-n5}(}OWL6<<`@!OAV9unF> z_^fR`8-%!#6o0-QRHBuU;A?iL(R<;0 z0ZknyP?-8up_(aa@d=~-UllhJ@HqB+AnHhy-)V_v1XEk0wMf?T13A^h9`eSjQv>Ir z_V;`9hR#`1g3eEN4uvE=??g6qZYUM79X6%6`El0#tB0&{sik~%ME_|k=c()mF>KBb zS$heeq|i3}NxMS(G~n7Onkt<3@Gli+mgHrV6RMLu=YX!^U*`sESqA#m!E>4_E(7c} zZoPqsSaK=X9qPK`%6aD|<7tI`AM;pH)n0Al*;3Mi&5t%D$ZjYe$?2j!cs?xz8#(+W zan#4MdVP2L$kscjt&!=3SSQz#=e%D+lC;jOJu&q9r0g9JOew-HStgH&TdavrztAci z*gA%AK4y{INa3vc%LLyWWph1|7%Qs+P<&xO)~uem^p=-#e4M};SGW<>_@^P=fAHd6 zDwd<~1w-rvx$t5f)RfM0-W;|-_|>nQKrIo3`y3SRKGY~nK)t`#H1I;=qzbDs8yZHx zKqT3R{vlPz?U~?mK!R&0Mg}O1>p{PM;`t>(?rd-{i26;D5iWP~4od}Ut;=@+>=X}eF4L*LRSYP%Zk zr8a0OJ9{f3^A2&2YZG@7yvhHYrH`i&zCu$qUcU|DY3F%ip+R z;RrEJ{q`dIgPqXCcYR+kJ!c} zjg{l!S%WP~8&TiI>(krH>-QMhRqFa_QNAQASB_DZvd=c-v9&}r?CI0LwNV6>u+uF! zzj$+TiJQVc{d@jz4ZuA-I>CIBhn-5Hpc$JuL?4fO@L!3;BB+{{?h=q{ERYq!`L9Eo z7r_LolecSo+-Nh)PCPRG3OS**`_r_rBYfqX$$&@;GZh1)v%9@K+>%E~=p z?5$SVXdeD^QMBVzCwCEbvI-29%Y9)$Fow@3`go6vSxm!A?}oIP=1lp!!3RcX(#Rzr zdSRjvQPXL(8mwR;egYsn`}%>W2YwhG)z5sNK%fFOzQ5bZlD9`T8%570I~%o58B0qF zUf}6A6oIJxZ|4?Jj=y}4t(14_6v*DzrhCDxHl5iE$;a;>vdZWSF6Py0^IMr#)8*0L za^TQ+_|Gi%8zfLuN1a(9bE9sIrFMxAf*J^#by($bc1%HOQ#h-Uj%qtcQZ6Sk8;KJC?C(>Iatr@*M~geij1w7kvQ#jb^;7{ zR!@gAP zX5m}3smX%9H{K~|=+AXO-8GWH8 zoME1&FoQbQ7ArgZr$ZxkM7i0SCw!W6)k5~uas?hE*60^$wPGRCiLh?(zdQJ`p7skIBx~s@UAIs1$p;fIjHcC z)6smT1zv}}l`l`9tR&~s%QoY$g z>WEWyTv_`imlk-rt}(Xa`Tt?RMrr~D2fcg10$F2~T>gyxI-5g`TXO`U9Y`?}R{b}OlTH>+554ck# z2d`j$K-t3f8&Ty~a~4r|Zb5`I9quY4oq*5FRV5A7XF*aSZ);R_`!Lb-*u_`cMC-;% zU%uC#dU&|&Zf#@rpv|A>_o{*2Nv8{P!q@pq2hp3lo9^-wY-~ypW=$RhV|?{ySC+)| z_m3t{3f8pEn2{z^Z~mFRY&dX3_N^`6bouC{{pZGo`0ytxS^iY>TE_aow|{}qgk36HKEtugX%y}t=E`!>7RvMM-(b&+=a>~u zhfo_$qCQag*}%4x9OQwmgVvY|M(aTt$ffaS5VZRlc#Ad%vY`jw+u%vUKL{e;^eZtW zr*cII(tbCTVTcmc=uBX1*Q-kj$ zN0(iz>Xz}>+jJW>cz@066NhHH?Er|v{yAX^ute}QAzLUAz97iUQdS0Y#s@zJt$xG% z8BnH%Hd8uHRR$`ZgUh&-3YkWivG|12sS4&|>`CFTT~ z7L!A!hmvD2YoXI)29pUSH@nz)VJg6rYBWoPf0?zLJw)~_sk$~w?DebUsw05!gW1RH z;du`Q^MC2O@9wTTB)7y*_0jIUt{TtnsW4#rpcfvCX71UV$;WKpffnhdrC1zS^OsZC zul(HX^Wp*4<6znP(Gj=7#mOhXgGI`DaXFgCK1a&F)<1i05T+5E_`?mvkqh1ck6R7> zUSIll12eS@^d70)rEoyFphs#O;($Z(XNYDx4V()=3(Y@GuPS za%$8(5v-k#IYr?dZ7f^JQNMQI*rCrFr{p=>j9dv)8|L!15K zA63b7L%;7YOQ)_&-q9ZK>T!enpMnZ!7Hl$&Dw}6P%j+{cQOUs`F@el~!V?9@&jr;4m2l_89T7=wv1@&{mWYdb?E7?ljln4ykZ3Wb}xVO z89Yh#X4qWApMW@Rc%l4uoPOQWv5#1j- z%DVPKf@ogt-#v>o&8Nof%gjo*&&6ZB2@CZ{Jq~IOU$&eWf3VvR(zcwLil%*=r>g;J zxQUc^rG4*EH!AA6zo9KYgIt)HC-wV8dVU+8LpEEJNe(yUX`Vt%L1 zhb{#aiHvSPB`DdDS;l7Fa0_nA&n2y0%PWuSG+n=UU%KqMa(Ae#RaJAyu9_rjIO84sx7ZxPlunpFxvayb0-|jXlS`Woe_>mOt z_^3wvzUT9)R16p&OG5D%a{{x5140 z?!;tCxNZVMd!_{eFD}Wp%Ii8CWHJ>yCdL<}cb@;8oo`jkr7_~?p_wYh!=6m2cfYM0 zf8|<*^p6s7)mtkefngD^-tk_*xZXgzRIOb*Tm~p3zb(8j6%%l}dKP%|H7X0G2Ih-r$xr5OBLZ;l@r+^!OR-ebx}f}Xrwl_)_A`B7W|e|7A|?iem?POUie{M zfbt^$EwXy<&dW6xYs=?-;oriQ$$yNCtP`}dg zBdJi-$w{C2ngEWCY%lZ19{Av!I!8bbOA$Rsn4G}(pYe`gKwmV4mLGn&y0^QEY}rh+ z@L5avF=7ks-E|?hzyNCwn`NP&0w!BtiF_TmPJs7xC*0#ej9E!PYx(tFeW85~t=ugC zN&n3EB>%HFzIp8FKi><+aNOL*PtoFU{dW(ozrv?x9ck9uX*NT7<|6rxZu6U}`~PYI zZVHc%3^{p$u54rNVjuX)ZuU%aBi_deYxdU9dm5RbF=*zo2<}E(5l3+JZ~>cFyVv{6_UA1jkXw7Knz+y3i@sAJzP& zhATHct^wOSyLO{nW|NTSr21`6o7#3D^z4HU`+ry+-nOcYu?K}x6Q@(A!RQQ-S(xP3 zSdY={hhXlN?3K9bg`V})QY4v1O#Q5PYMKDNnTX-j$&$mS-g)MuHi-yNH z!mfrR7J9QoQ~Rv5wOqr&HbTKxn#}^SiV=Go(#AadMlgi^+O}S4JO)jZ^rb`IVZO>^ z7Rc5$lUhI~!0ctWCh7Y7e>c)1_ z*k+6CS%Idu+~d>6b&7_C%Mk_{Wo)r8Ne43Y5+dk`WB(2U8J1UXA;1fQSNg-gR+Hg)!%mKC{ z>hp@MuoOvd)hOOB_6T^m@2egzTjf-BeqbHe5oKnjg;y_#CQQ zZ|56WBp43f$m?&-2N!d5aP5q)ZYptwuHXlhvu>&)#Pgl;0B<%{4GkH=e+0zJCva;F zsC-wFI$(T)>g?8=SK{3nEpv~Hm=*jm)^t~FQ{?LXn^m^Ow|r}+R;K2x@HXLWHy0}TjaD(a1Su<;7fqPW&+3<&b^Tx7|_Eykw*ew?WbLHE#E<5z*To>^E&4B>9t z;l^b}Blzvz6XU5Jq``d#WYHgZ?7L5-tzZAR!3^Zls-9*a(Xa8mS;efZKD{PNSa`oL zjL*V@tB1E&(cCkQ+M@qG0-J6BjP*<6woHBz z_q85fNEWkPq)2)y>bo>H;Gv+W?rXhGz*wZQ$kQJoSu8Pd>O05^QTK|+EhN$;ceP9Ig-8@-FKo3jX!EnZV=3#No2`BbvonNNcMFlTP4L!u7%A z-m)r^uyd!|hDFX6wQzX80j1AUp&Ex0ZGkrH0yG?u z=2lh06ivvB9^_F-fM z^!F{%9G&Q=7&Dvj;{|<p?hQBBhqlA$x-rKL_2!JV(+Oi0pMjUaw&`gwZf-Pxur_X@drL*WPKJ&Zsk8oGZm+cFn33m; zgkve(_vUdSqVjX0vCUa(Svmstj2?XJB`WfOgItlme_Lug-wIY))FhzOyV^gdFV)=t zqr=(7+g8t#;oqT^44rP=(P6mNHvZ!YGS3%zijTC{$tj(C`X=UFT6R_4lxO+Cu0#^)9md#BC49BMR{ZUuA>_1e`p}hO;|qJa0z1eTPZq^TwQ;V zkz|N9KigTMXr8%!Vlf;ky5Fc>pD~_ES_qij3H3Zyi_tws11gzb1KQG@BP8xCR(Q%QB&Zm81~DD0f`Lou z(d@^4F&&dC1pHqZxcfc*}tomg7@+I;{u-uQpXw8o3Gr@ac$PVvq9fEbgbdXmmJi%`( z(-qm(_>9IjlQ($E%GlafAYHT67H;_TzKvQ=1Tw^np$Fh4#07p4oi)*b%zH57r$@BE`+0lU!3+PDrM-%So z>-g;0?rY^NBI-kvqOIzx6+bHHtHv0&BiS@N{D$}31$}K@K}w0&z3|x`6Q#}lJgSlo zxa~D1Pin3U-dDR?KvUDEmCsu@>F-(YfQ)?W?HHEs!~8cM>@{8*#bmGh_gJ(vy3#Jr zF0{W9O<}L{?gg4%9x}!ZZ{yWjZX`->$b+$Np!J50a=Qw%!Km;Wxa_L_gi96xpJ`ul z&7HS-**S2c;y6V7Ewbl5P7w{3^(t7CQyb0aA6wfL_v?CPP}MEc!L+(wjl#y)R+={g zM4cK3au?7Xh7v1wNwDT$gL>`x_nA~0ItV@AV%zN%v3AbA7q%e@TlmZ*l3zOjuLr$E zn1gRq_TSh%RGt{cZZ;EK4;a$1k>2qiR`Mt@9yz6_yUQUaB6QPMrQWbuzHy0POH0vS zB>lLx;S%9Ol-$8xEfNa1Wg2f>n8%lNX!B7az77)m;h7!zO2o9|^eR$^cTkX(@OKga}T4z^ActFT1h7x9NTZ zRuv_m@m7g#|K4c2CRAb=KmDjll7R- z1Qcu(wRH6`U= zExf4rY{~wlm+9Gd->t@hX%(I`0lAM8c4lg)4h55*oW&rIWx*p#IK3pEa!mPZ#hZCQ zw6i8g+--Iy2VMu+3|n#raGm%tXDuff$MSuF4?)3TiDe z3Xx_%t>3(QXzOs*Qpj2v4+MPv60v;2qJ2!F~xJd^YB40Dry4(vXohH)U1u zq2EiHH3Nr?;u>^QXkp9vbZ)}1#aT7qQe?4;sJ&0}WYXdt=P0mPbNfg-laFpLWUGAk zisqO8#0q(Bg}10dztH^U*xnaeC#dx7j9P7z9gSE0URxXwRd#Jb##j7))M_kPJL4yf z#$SdUe=%Dee?alwp&EO0P_OOafqyo#i)46?X>hy|CW>2Go8tVzr|HPeXTf=|2xsGG z&viJep7J`qs9d7*YW{aOACDhuo4pFZq-m8sX@KYcbJIr$i&<^oi`uJsjO^r}^Pt>f z)}h1X7=cooVFJ>#Qf%xp8+BN0wRgCAEc`-F+6ENT&!25XnSut%gV)3pBSLe!Z&%mfcQ6KC;t$29^8_b=Wvc2mH_yv?3?m@+ z+AOC(M%zvuTV(Ru$>wtTbj?D#mOt<`u&2rMpuhU*f<#JP*LcO<(p8SB<~qKhYqGu3 zG!%*yFtfBMF|Sc*_L$T;$*xYHwg$$s)HK} zt}4YQ&wguh_J3VBh7L$>-ZFZSx%U2qczQzVZ+Y;Sgs$qybC}qBC+`L~qdp8;T+$gs z)RF4=x6d#Q054gseVVb-87R48nUrNQllC~I?bjyAV&VOb%mu?7n@Z0ZMXlS`l}9|= zaLq{*OR{(lb2_%N$Ahoi{KA_zthxXD+!fC!cx_x!#zcxnS~d8$>S4dbMB*>J3~7P8 z+}BKwx@RWuiFdbHcUpwJkTzL(S!y|A^! zwPlMC5nc8KTX)O2VZ()bjl_s!G2?C74F#RoqT)p{_I}=Ag#S~>ti?R17Lp2JcKQFF zyHrXNKZA$P_J9CaiU%*yCTj;)X%+UqH4l;^vQjh@Iba?B?EDSkC)qp~Zts>s*6?Xn% z&uYeZdt5a=t8edud*h1&G&k;7nf_}w+HsI(OxGeY_AK-6p2^j zCc;nWbBm^^c{yMrB;tKv-F*ymz#@2YC6)K~W|=YwobB2gfJ;q!?T&q=`OCiUv+O3) z<2g9+BG$SkBi;1fFW)HNOH~`&E|l!JTprJVaXke*`DJ4vJL_csl;4l30Q{=g1INc> ziAu@t9Dm-8X;jx}n=8WCv7su_V$Ae_VZQsxCx|2?)b#xMtn9|xbuTE!Sff1rcV84h z&+P8XoSFI9okB-`I{|_)th;kc9XtO#37KDfNAl9GqjJCD{_52GU1^tS!P#Hnan7Xb zQKJ&WUYnJ1EgJteh+Lc7cT1*0shhgMfp>6l2`1yHpMp~ zb`6TN2-M{i#vg3Y3R*9#x-_X4XtZ_yQ5`#}_b#e0IwG+a{<&e-ot?7=?H`$cTX|mg zs4=uSq~RRP`u>ohcP${%VmY7exNG!9ozIpAb}6MWP!FFgpS}jM9#YArKm{hR+Gc+2 zOPF|iH1{AT$;3*e)eSlc+!%hu@`kj*Ox>%e@{`uvG~|`I${OD0GdvKc>)Vv+Y#3a8 zIN9^}z>?|Wh1RRgmiDL|??s>I9R+<4Rn52dfESz1*5X*!Q-p$G*+BbS@ilVQ{28*P zf}BK9GU2Wt?UMzde02tbZ|o}JIfR|%`%nR+wza$1ILQTj0HfzD`Zcc@KcV~Sx~laL zW?&uBKaG-K(t>1zzTSk~zW%s%R<;o@hWieh>(vRyEE6zai*3h+)R(~S9o0LNvY1h|bR24XrLv`{gZlpP4QPYhx#LEuJdyb1tIu@tfTU1qrf z4(x(Ypv@DlF$YJqyCD;Sd+nll=ShCO{Q?jpFf~h7e3~EVx4R^&df*oBj_3F&JQ8dm zTFI|LeuYX)FU-PnG3BWxKp9^#QKMpA)u6@KQc31fcq-Y2yP3G|!GPHNt?}2vDaE|(3u^Gk%O#QMe$voUY#PS0jGiY_|{%QJFr z8kKRbzAB;aQhvBV~JHUM+72N!zgwjkm8(F96?3=D+PX+e*caIwBx5iaVL< z8EXD!1?lqTzvHvNO=st{yn6bZc@EK1Pj~H8d`?Wt(NxdMtHA?{UCQX{DiGZC$?>MC zua0tGEqD|cki9`Qncs6xdXS`EE15&F7YX-$@`E`Ja77CM^_lhctZ}~IKxJNWNR^pj zUNd*Je>vJK0Z_A)EnR87yGHqHe3!o6FDuvoM_f7G!S1BQ`DPSYl$4tz`3IEjLvNc+ zul2gPdf=URQg}PH@h>}wW3C&UUz5AYL{vsNNI9?X+GPpnN?N=g*^#y9cTcd0vS&BK z)oa8#tAP6}J=cA4;uHcZ^g$O<+;&R9)bixdxW)`zP5w?*8GcACr}RkPd5`9@eFh%q zduX@##8HE5l_1&hWOs>qc-e4qU66Va@2k*+oOu{Y6WuIv$XT{_UA{TsGTJW>9q#Un zbRYB*>^lwk#C$C*R9uXifQ4E56eq1zpf0Tb`Leqf2}1vv3dEXBvpKpC@()`<3>F_I z!S^FoC$!cv0!N0M)bPP{(@$CM)dOkZ^@<*1Q)Hfdu|xpE0L)lzPYDJP(Nza5PdYH2 z|0GLNs9tZWXW(uW*Dw0V$`Z@w?!kPy1cQnPjb@WT>8oPauw9~hsam_U){BO_cq#g# zowPlobFE7BT@5eUHald9;M`UX()Hoz@s8s1-oO~^Q-CzNK0J1cy8R(B48=4L>j<5j zvJH6cniW~#(d2{~lIj-qa`Fl^J;#g!_7N?UU*#a514^NoO#Z^fMtygdh=>qU)1e)jToyC|&y1n{S#L_%UI+K11cGu6ZbHD?=s1K_a{Z=dp7ndYy_H}!+ z1M_X64gJ9_+$-u-4|yiKkv=ioW7gP~1R$k@NKr=1E)7&&#SJF5kuFGixAB5jP39JQ zZ0U2HEOW3eOj;oux>6wCTiax}RVcE!71ovl=P%eA;Z+H=>vKvUz_}r*E?Ok-fp3n$ z#aygt5GrSvmFP$hJUcquiGB{T z(3(8W#fdMYmWJ{{u3>?qQGy9;bo_i4{O2pqEbyxvR{g_5-(Jr8Vb9N|burCb;*Do3I-Sq`Kqqj6Cm(V=U?anq^-%h>VI)CV$)M-O8fpHXH zGb_%YuxeQ*(<>b*Ll0~I;(IwD#3DCu@u+#G?7$h!rEE+4Kjng+CET%{9$wE1obCber?GC8Ns) zsnkCCLHY_ax)vFGVsJnmfr6J=gQu zbi-20>z*jf=_>m6f@B-3)B;mCdVE$?KB;OV+o9U%2$6kfemkV0kVNmWeWjfaD=ds$t#uf3jb5B*c^pTeG0R(v?UXky`vwK#dnx)Ce>B(MPM!7Zr? z?F92Dan06o_ zR`hLi2WJ|#8CV)NvB=!2vhAjf`VIRYCYLLjjTr}5`n0A6oVMG#&PT>`?ghSJuDnl$ zu@buM>H}0}sc`lbkxM5=P4Qd%Ys9JJ1{Z_HDZMwjNIgT;qNvD*)8>YyCV&pO$95IV z2J;xd;<)QGL?a*Oy$%aj?t#m0b3@kz*MP;Ds$Z_QQiH1;RC4(Z^n%wW9 zRaLlV@RR`2^0#6Am`>ozO zqO0?Bh4FMMm}s0Y`Gd$g0)VprgmrXp8(pDZ>6K{xAjj5XW!=c~7&Rx^sJ=rklp2lJ z;oPGz<<_91@NuD^l_+H18QR48b_0F(bVs*5FzhPSpu62SK1*j&JhA3>q< zqG2HuxUbk$lmR_uTvG@WX&$kPoFUqwcNaM7Sy+p7p+dUnu{%F5>{6LN5`TEz;*s8> zc$C6K$Hw`5|Em%r5~Gw}IyBO$(J3ARKWOgDTpM(F3W!l@EI4Yv`IV*|0Gr^y-;G(y zIL)3@P;5HdWxv$Gep&gN1z5QdpB0%6Q_ylBlyV&0U_r1kQCbp8w!fZCslYxkhJVxhfGVwGNwmL>G{)edf+8tvEM1w+hG z9V8Z9=`=}ChMF_IEepWt2l(X!k4SR^sSdn7F!LZZWex@&8|s7TOb=|c+_=Lj*@fWV z5q)Y}`m5S#I>0o~BfhQ459Y4U<(FWO81oXNdtjLQPNno!1)Gbz9e81-%;+wP^S+?* zNS0Jgae~x?Jgh8bqirvfZPIgh_!Z;f(my~EPu^?uf&^e(IjmYpg+X#!!Y-;CPKuos zo@h=jx_K5YcyS;-;C}sRVet)lVbYC`RAokE95J?sK!-E;$Sgpa#?u8kqf~;Beh-F2 zo)VeTl#o0N#y69+{x78`$DDE2D}%kXn_(5u7+TeKUEE3fkDKByMqDTaPC0bTAQjF`<>r>9==bpDYs=Dx-FX>_{I;Q z?!o9Ef1n_n?IQ-J1Z5Wlms2#O(V}>ufyeapK*BYxY&*%pj1|_bK9+qc5fTaWNEA)=Kj=4A9VO`KKS&TSOtQ~E+YS!da5q$|kdhdsgBALfg30Dn8 zd-bfQ$I~2hMpI!8vdWs_(Rb4{n5oD_#scYD7UZ&Agb~kk_#c;T$C3z|QL;3-0B1#0 z&}Z4hxf~WguRtkB-e2jrbG-uU?P%s6>u=Y^!1AVs=g0}kMJ_?o3pP@s_| z$S|E$OR9IYN*?)j+DBA>dxdNi4&Xs`H?8TJ zvCp%tIrr%q=s7>ARSQ3r9QjVA2xIU2zKi{$hk)*Y%Rq)Uzo5j{kw-OkcG^J{OzBbA z%x~v$5Hq5;&`?gCmx1SJHE$z_13LV*`I}4G?569i?ox`I)tFeMV~FD`&Re_L!$|hQ zG;BGtMdOn=r0qV#DCq5)j(%U|c?LhI6A!?f-8cz<3y(K`C>)Ob9c{X#;-@W&dTf@_ z(acw-6ji;4HwOBek@lVpfnQFuf%Vs56?J*A2;_YYdyj@ypQu!+&(Pc|xkaP2RN}^@4ua;8n!q=~N3!^2@7f|yZRxyi#vV3DqMJc8`2bixh z9BM8G2n;nV+Kbs0xh}4CS>ZRi%~p%)%$LY1TFi(jCHS-*`=+_{t{XIWJ#bLn7q@gt zVaC#9!1YBA_oy}+Jt}(KOFkYyxs51K1>^vnai`Lz=H8oS z8NaeG-%Zp`4>U)wI53{p0Y+N%WPyt?C^ul{c@C+y zKOQ&;y8xZ^aUqGRkh+Lxt%t!OEM-5a^q@61Y`TCHqW=_R@p+@ts^eIOBtudND73+Z z`53a+rX3z?|o7L2z?XzAbTjm|Y#dFS=Du^j{ zziVmA9Bo92HPSz18Jl6hJyp-wDjvrdgbR+3;+rKovi+(;l%OCB8o}x_r;=x`kWg=kr4>fMq6aEGscDtAu72a14iHTW=WSr zV#CEV2pmq#)+AeL)!e51x<0?`Qb1#nrGxWsu2Rvf*21-FIpN&HRVwRA%$rE9G|x>;#~kt!{7$ntA~hw2XYlJsm%5i1e}zj zJ0UW(pSQSTDf84F(P>UalAYr7NCkv|xIbAIb!o0i8edR{81_=`Nj6MMneHM64uc;e zTadn1GQ}jh?u|c)NfAF$zME~0a#T1?NCDT{Y#k;==6CaFU9%jbSjP<6^a5L`M}?Y+&rPysOwGEBX37{1m6X zVuM3L$=2w92B5ED%flO)wQZ`tX6~5SiY9KGswLOJJd}#}a$LRVCQG)KnDcHL{zG1N z59s#oY3+$Iqn$%GvfN?>{nQT6a3i-4i70gV(kw6Shr*WEvBhei)^61Bv_07!a8jn0 zpyj3C1~ndr+L#_8N~pw$z8GaqY!J+bGg6=jWg^Q%`lmauV5>FLvGcrc%^RmUiW~a; zq`-L2pG-u(9>X&$N%GVQZuAZ`AvlXJB&>5ZR7hwCr}5n?K%DiIb?4qJ`Uvm7ZpR?m zkFXjs<4EIyry%wOi)m`fk!PJBG9o!fpSo1|##6ze%eRk=aayPqB(uhcS3mjyt+ zqMA1Ygx@-_kimaUY8Dp9n0=GQN<;o8A6^BNhKVaKa7ws~tk-_MN&M%R`UanATR9s- zzos^Y2nvEL9}_P4MRB@Kv35+2pfJiqShP^`vt9wzyO~fuvngi9{^B0A1RHY(jq;=0 z2hC^t)~br3ythQDO_nosvMIhXZ}hesE_o=|z4mVWO76G`Mwm)ZJY;pyX7`0G(U;wD zKI^di#@kc^K2SXM1qR)m`?9?tDNM`2JgCJn16f{5fD>_T`yRj`8OY^=-E+?6yUEoB zfnrX#Hh%tblZX2hL!7!>B#PoW*x++;AUDygT}h z7H|?eI}L2$GxbME!soIKdx_}45k)9jtw{gD%T2s{5T}wO9*~teo-+V|6K*kH-;BnG z)$yl^q84ZC(Rm~7eaV4Hv45&Pu3k_ewP(-#?4(7{i7IJp7|8FrVXG7MF+Ta)jmR(a%)u*)iEoyDn<1*Pmv)ab<4Uj7Ef|z z;+wHmE?KFi)%)^yN)GrnrcEv3v6lI(qtwz6mS=b-^x&jDpa0#%DkGvbtq<)P=|uN* zGACbN?;=}Bxiqq+^!MAFXMT}_`xvs-#GuyAjidvXsNF|qo9^Bpc2l23KT9EZwSw>S7P)bgq$MZh?RfJ@>=S+rh zNY9mTaY3}Vn}A%Z0QT{3m*(Ug%%wZgzTS;?z?!;1B4T>%Fdd@+@)zL~&?Cg?079Em zRC#r6r4$Pr9c{P5r1O_lRu3>4nxd4@d73WcBLX^x29m~TJ9)jpX zYE1`_{kP#Y57q-1JSb%8lBv0fx*Mw7F?ZZaINDEnZicSePLMV((RfSW$sBWEnSOxmI?*1K#;3lNP0dB)XNj|e=<=LZdDDkV_5Y7XD zlNLvr2iR#xI4pV$b1^?jy2Hvuil1ko?Y3_90_%|OhJ#nCJd1nBGpIwO*?45WyLE+A zz-)krYE7q~aZW0D0{NvwGibv_+qG8mTGg;_T@X+8t>*2`M*myaLCqE3CFa_AP<@#Z zwKkJO!`@=P1MxYfhFdPiQe^xZ)5*~mK71g22NAd1X>ycBM5m4V_^R_obp=W#q3Pm$ zG&0uGcYJaErchj{D-02xp0CO?gDw{sYay-lN{rN>+rKhoD}wclaI{$H5-Xk|qVM#J zQUcATA#>bYOZCI;@ZfyOAhCC)NILiJw9FY>MP54F!6@UFZVcGWFYY8_rK3nKTk%W+ zO#XGg^#wuKNj~~a{bWYq5Yc+Hztztfj(Xo(P$*mi6|Q~2U9#iSoy_VY9%fmmjncbU zJJXzGgQTwwoxRxdkSZ2EF}%}5k*8cJO{Phv63rR{AFBHL4SXrK<~J7$g)ZqHNSPy$ zb%sH`5uA^;rI?a97d*X7BA zwMlXPQQRP-Oy*H|-%St>mS(cOrB=h_|1JPs5i{GA!Fwb_a5;Jut|7_{QzXDuS+8m}Ch!pbwE9ZY? z&XXwwa8(<3gV)UcPtJd3CWsXR1ek;%FcZ)JC+B|`;FaP^nFJwub4_zH8-C0j_hw48 zOl7e;vZ#q`PRv=NbQJ+3i)H`?%u+_SClTaEA%o3VlH22?Wzt(75xXH4hnC9RxOnMd`Tv6^Ew-n$X70b5 z{V$O6WD2bQtAzh)wpd)3hV9uXlPU1|A3*$9vo%XhPqE3E0?kFZ|7iJtn*BsXu@qLY ztib2QzgqsEW(%>a@8K1gqJItG|7C#!S2-zWt6N6zT4OX_UMI+x5Kc^7Oi0(8jsYBR z3l}P817-7~uI&``R%haNn(C^j*bLlc5Gty5k^hUa{$L8j9=Zb8-BQ3;I$=dt7!PkP z{r{u!{QqX1@Ky1qWGE|I(S~#GUp1i^DHmna332Q~rC*Fjw?X|3=WsTV0%_~;l{Mhs zN?Y*b9F2cE2v)=`nYRbAN3FCd<|;B$pAr8W&_-3xRdLTu27EO69b&js znM{0|q=4L6_)9fQK97v$ZSl61aR?gs7y-$juMdPzByRXNd%Zy?5nGg82M&C_XHO|2$KI`xM~IG=R=D*|qx= zxdVS}gAjXbJDZ~7R3r=2z&)UDEalfbeJZx94)I$)Rgy)OD6So^=`;Q%z>~?#@==fbH!FmAew|I`{JpGkW6#j?ZrdMu;xC?vPpV0xi zWrSOtAlk<&vVz&GrI7Q;!ci90oX;O#+g>P1LOiyR@z$>fE*Gs}W_th9x$0=8 z8`?SQ3-QKg0#`2X^j#sSh(j%ryyYd1;cWnv-RIeRg-ax6X%`G-!0Q&GF6+QZ;V_(o zCd5}WXd=cWvOZQ#+`~~*iKPbr)^Zl0tunWY_D``ZdlpnL$XvMlYwSsP8g@`SoKSF1 z*f@V%S3%ghPCSvlwKzy=jhq2x2^+{UPCFacNY=z?U;C#X1I-dSG!m&4v$FPh!-LWY zn^JO(*=A`6SlEIhBL$XOpJ~b2v;vRz{`7`(1{L&2dqpdgxX-|vhO8Fi%TadMUvgnP zygO1^hRx~55Ehe|3(gnoy(bK8X%T=mh0hC5SF1edKTj~#Mmm9u3YPEh*t>!{XnNvH zx4i=;pJM_yW8Vs~Gq{HIfx0J_^x~^U{vq2#mGfi$N>2hbJ}=z4g&!w6@-lljoiY@U zE1l28_1%GAPU1*!SF{|O83jc+tUO;Q&I#lcr!9yf#l6ape3~&e>g~O@5u~iDcPb{{ z&4Wu>f1{k3FC?Xg>N%m|e6Tr|^It(o5SKR4-LdhqvSmRuyB73|I$?oymW^U=wnR0C z3b4RzK`UM)UUc#e@4vO6$5)jNdmh6}nS5dmfWrwds05LAOAyzM&~jeS5F^ZfbTrpL zj>0`#vV%(eFcP0x^Q|xN!U9-*tCGNheQO??T&4Faj}(}Mlw4j+GMI0Y9jR|vr;LhV zg{fF)1zbbHGBu^bbB=1rZIVJ2bJ11?9rd!%g^#?Rn*qrM90wdD=gN@zxfY1PBU5pc z{>TyS8)Z!JYJqw6wm=X=H2tcuYU~3uk|ug*k#Y4l(%g`3n*XSR)>yvz)>iV`Sh$Uy z&+o`fZj@ZFwVn?8AHE?Ty|9IP5_y8K*%Z)IZ8g?{&F1Yasm?6wX6$-tIQQP;!%rmR zS|T1Kt|qD=Dkj3ZrFdVU@Sya}usIqLYD_$-kTH78hxauGUPcVvWDfz5x-Bq+2r_UA z1;_QmiSZ!lwWy9A?MNE&U@ZA;yBkGTOz|s1hwjFBJVYHZ;1k640~DErMt4=vCktp4 zc7jdZw=9m7$1MLV*jWtKkJ^i)@(3z5?Vn2KyTuJ@jbJ4GYPJX|+t-zMco;h1scn;| zeoXqEX;>uIuZTHe2`3|ugMrap@BD4)^U#e&wqEe9^^8B2g2FlU$#fMVYnIeKRv=`+ z5aHZ)s4zK+di`arA!a5E!KmlU9yu_Sphx)K#BCS*b|p@~w`srUEG_-o?;CTu)h|Ej z?-Yu`#Ifh(HWOHP$0^a~D>3_Xs$X8J-B?k5i-IF;oixr659_+Pjo(Q#iHcbDvy^|v z>raROF$dt9V9U<{u@63IL@-7Uzw|aYToa#Ts4LtS83Dc)FF8e%72|zhhxz-4A;!Nq z8t6_xnWpC<1Us9$_C6=6zccaPBX(@g2j`uW7kvV(c_c1tGha7q(QvU|pX#M|XhIU( zo?(1io^oieylA_z1Kfcnda$QJr2qw^-3a}QzLbJbObpo zp*(tY(o8%Dxzb;?TytgZgQg*AKm!>FAEd`x8P6+s_*OnK{-l{-aEj*RfDh5@oTrk{ z)keFHom4f)rF4OCyJ*FO@3*$ioRppJ{00kr9M8yNTp^>h4ke{t0wpa8$KfIeqvh>o zwk-%?G#1GP8Vi@Fba#!XJo^fvT;@nQ=i8{FU+>3_xxvJzeV?I071kx(zV%n(?!6J= zh7Cby>8v(D-S|{I?X(86ga-d_;<2zmbgX#weEu04oSUDGnR;b|PSE?YYzm(v80dY` z8_{<1@iU6`siV2aOEThsaOy_OWOtv}RSa++7CgodB*@h1P6wS&N4pIt^E=>|(iHe^!k*nL z5V{9HNIgqX?n{$%-7A%yJbG~FcRzXieX6+hIjQuGPSfz?vAXzjyJF{PcAMDgg7#!= zV4`f$yQqF~gU*vCi+H#)n%rQ=coKN(C|nYFkV4R1?bI{&UQcc+YT(rL0_s|F3L?2q zlFD0wRZ@GCHSoSHup^iN%6ML0=c6850NwO7!an-rXYZN;0QCe!O4*)h8R zLjz^E;5Uf5U5O7tL>k>{Z{g={j+Z>!Bm+q#xBOGK)k}zDz0WajlJy9o zmYQiQfXekY!JxtH+%JHgcd<}E(jg*rYrJDc0ms2EY0Ly!X8TDR8qz^{D%5Z)o%@B? z#b90VTJZ>;7yQHLz-ORxxz)kV;;hyqx~zl?d%VlC?72_5dyjcatOtY`+6UJdHRMxo zKZ<0M_!d??g0`%XRMQhPe_aolaIf}&NdGPM0aMu@k_PxRD2Bmxu8856&I&1neo;u) zajEG=Mea_4CX#0OGtZ)Kj^m1@R)N3Uz-NnNKwJ ziTt@aa{_+FY7hRis?|Y*TA|dVFdzyX^!7n~TEGP7=o1!-Fjgp&vfd(iHwtHl<()BC zHlfaqfA?swpiHi*p&g0@?B15j;S~GHHZ$3JUP;xe8%U@tFSUCr_v2X%C@@E~#e~)t z@emIHEMmOqWGiEYp?+&K9o8>K2MZ?d309XP{V7H}<#BzOhX8z)LsNaWCan##o8Y z0yuH`y_#CjD0$_qw2tk5+=s{U#~L_aHs0Tp*L0pBbt@yk<#=d92dL40-kvHpXay%m zWvPpllp?ROK%%U`3Wn(P0yfY2&%OkM94TEO9?2|TRs(|T2`OzXm2pN})e)2Ij_o{O z5NFtB!VL$O&d#jG&A=~)Xzk=51)knx2}$QNr-Nch-^4Y;!{`*%G*76b)3e3XN6XSB zwZaJ0S|`RnD`U4o`UU=qagC5iXHqZ-<~k80zzFxwVeihCi;Fb1)YtnH{eCnBGFq+v z*85s5?mww~uH(Xu$+O{E8l@+`yf|Q?2Rx3^g^L|`1I(PgbRqv`Oh*hfQ*bF)LKX*v zXmMn~x~s7*3$tjTA)2tFJ>LBOB*5_*$5JXwOp?449@*#;-IxnfQy?6q{4@C%4iQnF zVFI5U$0S)FvYPzZ_pDvk8k%(m5%Ke5U=EI1oYiy{LOPIqG<};(8h(1tC~SPWBSW~^ zb|057cK#?`;=7o)H^ttQ_8Y`ml?Dfu1*J-fgv%ezCQaHCZj`Zm6BfHKotCGrQhb$i zi2a%>ccXl?217Qt*{N#bK8E9tOM}a_O^d)MM&9o@%lms19%#yJG~B(m!g;zji9u5p zOWG=|_8ZZiC4c5|;Y+zrF>TiUq+85VneCIZ;lYD9XIm8VEoO=m=*^PC0bRw+BzX5z z1Pz3Wbm8PAMSb8?#GHW53$DpR&N*11Q#JH+M$E{LE`X2^B0Z5&smmgcNOLdd3^As>B=1dq3&dxAPX} zj~?V9_U^xQ@Pv=McZ?mzMKG5a*Uh+zaFHP$u2aq-z$LJdc^(0GTP<83P$+sIHlJO( z)cr2^TsGu9HMh}jVMg8H%2QZ-x1_OB#31`3-fyKYtZD=!`}eFOn_gapQ}~c{2x3H4 zCQr@EOg@9>%76_Bh=8%3+>a_nA*;j6nUGBjagKrAJ9R`5E?-6|QC%~j?6Hs}J2vpn zrXHwZvS=FS!<0iKT-&7T79`Rz0V;NcMR@~#V?TIDx+ny`P+k1_O9h9j1+lgl>^L&m)J&kZkaid!d)$ZPn=Y{#z%r(3i;}0@;io4^KbuY^{g0E}$8?zoq zCr1;83UvI&O0tvQ*6Px2c{H2)?xQ6CiK;D@5_?Wo%D%MhrPQmU1lWXWOsOn605wE-KKbYOvrX`H}8Ocnw7t%bJONAg(w3KSv{Ye@J`WLjWkpcBPyt*f*5$rAJ#@; zlzq#%7*#SpV2{qe`(@)Ul~q%-Eq(3=U%>nt<9?%QNspTDPsg`zy}OTG*B3zzJF(M zpScpaG{-ak@QoNTT5K-=l`$3)6+9~C4xi{j&utTHR)HFRE zT4U;%x4@ucpU!pW)Xebu3W$4lvvg z_#nl**#f{gM3VzZ6FB}FBZ=(o6~Xh4YhceQQZ{9MH(pOpNU_G5`B$E4N6$R+(X_~j zeePETxZ2vj2cL9$$e^}=XU^p@iI{pc>8){=4ylvee~`1xc4FBY#q!Y(g6gXsQ*1gZ zq?BC%tB!YJ6xMq9X2vfE1$K^Swv_jIY!1drjyZ_E~)P_Tbqcw&?%Nj7C)v7M7~RE1F{ETl1jS6>|fXgf5l35TK=FrKsnX$?hKkZU}eiu%6+E>j^lncDh%Q$ORu9 zET2`$3{()(7wYa=U-ODqAaqM9A`L<{FQVBeaFwsn?LClP-ugzh@SpxKuR7|^=>~*R zZliI!;lG)dR;^O8qhHL)q?t*tr?v6c(2z<(H0PK#bYT1+FpSetpv;Id(ZP8wsLN8l zhmICxJ8!a^xM$=!y=j6Sw4}b_9PPQ$>VDdHy(pqm2Gx75{H85>TQvV`L``JbaZG2< zu7qhSCWYr*YxutF3~XED30O*hInKvPwPpgIDbBH5ch~lVvOdS`{B2~gxjHBU{)us- zTb0FuMumwY<|R8ay?Ip5FWe4s`g02>wK4`q`B^e_Le0GGY`&XDQ-HOyV3G8)=;=5? zH_VGiTC};E3XGuFVx6K%9iWcmAO-o}Y0slGf;(!{+bTp8wX!7uM2&c;7PdIeS?9rI zXUHw-MNiJJA9pplN$j)MVt*1-Y<#!HNAz4EEjs) zHAubxC1s#&d{yq%LrPEq6=T??hYedTfxHs~)0f2ha=@CAv+%<;ns$q0N$^F2M6DM%o5XpaL2#Y4!u$BpCC%g35INrtp>Fu$7ot37Lhp0 zf;eaYw~q%&=cP`Au>{e8bp?$^f?NYRPrwtgSSVGI4Fcoc^}MIY$TmZu8}IYc%YXI-=q;)Opm!`X85- zYr-^1Z!{%C6u2jAPSU)}bm0ia*{8q6pR_MKS%?}IAmE+=zJa?-Wo^P>mSr7wAzk(c ze_JfvvtG1-C~&~yhPCbcCAQUtz1jv-zQ4XWE;qMKF43=fa8{D^jSlM|TnyZ7_Ubkr zaV7(8DtbF7ycYTrb@u{Z6gYV@8nB<*Vv!3r3z^uO0mXR?Vil~{uu6IX%WhUtc_xq8 zd*H(JUiJtX-;#}I9Bz7L5P z*@WA4MAQ4YEM4Dzx6 zX(jVi*f1GfEC6mzurIYk6@Nagw8T5iEWZtG4ME+{9N`H2XPm32AD&NWM&jGUQ5pN@ zL*LJCZE2i6?qv2s^x=Om>pxFkmzJJ7EPdwi`BrWNVx*qcKsX=ja%ed#g8+ysu5>6W zD*7pUM-q|)-G_dKUkbkwH6IAEgSE%)mS})|+^5%=8u$Vjr%aukQBbM0y*%n|jv#j_ zgSr?=uj_J(otbu1mg+Ld!c0=u;nl|_t0RC-+pwZ@`?D^dQ=bzvXNkR!uIX8EMx347 z?Fj;h537W2Uv#S{J@w+xJDBCWPZxtZjdL0NkEk+BlE}h-O-B1V%u2DneC_^d6K)a%;g=R*aY);& zL(3oa9`^+Z*H*Bs^;5he8fue&jt%}|BOW0=9J4%-7gVK2o=HBmWDz0snCLG6nF!?APfB>8`0N4%XiwoWz- zEmU16R{t{3f|UW+?ZnyAD)e{py7B?XE#q=~R*WX^MA(c*juuLgNhKj#R!;eapbk-3 zMJM@4k=6v*UP)Z)t;)NgT;luk5aa!G{=e5+s%UDIDs#XY7zCC)k2mRX(y*g3|ZSKx8%g$^} z4=w`jrBZ*&D$w1?t(2JKVct!5=C3O;-NS48=`MD<9XAx)du_f1pyXB@@5UNWd`P1O zMm%*MlIeS8fs_u7^N6y&1@VdNH%4V>&UFgb?V5i|Zn_)MxwaTOPDW$f0N`feM_3^= z87;`1B(Je7A2k(4kzJl|!dN%6fFBY#B}%6Dw6zp-WUETnP^G3DpB^Xfv1~Ek8ot}3 zOE+S?uy7t~I)^y}aay*ihS5k*M=z;&--Z6Z; z>o8r#dtOGo>z$q5JRwr`{X~xrry)MG(t1hj#l4xH^x)g(pjqaFbS!FrjWm8%IvpH* zQ4?P#%AnRyN!EN!?w$!3VBeaKP1l>9zAu0 z@Cu!8oA;eYhQ0+KAfOdr9@w4_`gOvTGl#8)@ zV+Z&gWj9|O4Z(6JyDO3^S)#|;;LjC$swJ08yzu2Bn-{zx)BMZsFKtgtcw#Q_PCu)x z7yjJ5o+RoSjbMDEnCcgaU-Y_rw&Ike+eZx@89z1gm8twEi``Yy)_y-U8KFBp5hio{ z)YhzdK@W7p3UmHkNTtcBh;L0KhoQ8!Sf5H zpy#*X-RTvDfL>jX*Jp8rXr06dZI!3(yi6v%+T0V13}sGkdKlM~nfT*_kCpXopzg#| z|8bUI>tp4qU?ODS-)#Q0RQ?{uya@IDVW?@}vuadpLvy64FjO&FT9sKlb!RVwjNKuW zEYzvUDeFI2DY?&HIsFK)cql7XInA{W`Jo+CMTq1_+iX5KwWcYh%v+p59ZAxGG}5aC zMZO_5{M-fU8BpB}*@28@O;QBa^BYM$yL&V!#YC@cK*yoKcE)JtR(CNE4Sys1eLY5> z>nsy?UUNACr@ z_{1j*`}UMHwvlxAWKf%j^Nf&oSJALCGw`y0_6IOOKCtP4wVArSTTTI3J9KPaqEc#R zRPa;g{k)CjaLC$J*1sNT84qrz``W4CDUjc#ye&M&ayB}_#sr)@7SjaIyX z^1pO0uqP@e8*&M}26&$qbiDsGbiEJJCos_VJ^drdK3LNaT&l6C*8PIR*g17ne^@g( zSDPH(jARAdVJn9+1r+mU4~6p0#I|m z{DoN09r1}i!{JMciIV)y z<6B&~nR2+DP)+kxH1aFZ-e1V~Mz_aI)D_ z3>g33P)6Jy*?hI$VW7ekqjV*V%SvxH=5_S)OsJ2u@($@yt_60RIf*9lhk?pg_L{@QPk*P7@r zy)#v@{Fe$Ii&~Q@OZ09j&mC~E<-7P4%P6d!NarPE&lcy_Ke?7iS(}Ip+Voa$BKY(irqZz(s<(&r^dIEj*lhaIxM$e3yvKR9J?;% zLiqR6!@xQ1c8^9T|5LHQrRegoKWhn`7wG<~6|d_!=i_6SJ+wGxS2F~yk!~&U1+I8( zrn6E;^tLDNOt~YQKKW1B;^8ix6|I3rPN|5?P7e>=$vjaIr^ggQ3YLTiF5zKSUbgpS zdG1%ArIAU8>d>feL+zwGtr?r}{uEWZC%L@Rd-)--yE8XSXWQWEg!SjZUwPsE#&?Qv zOJhjp;v-L^y?l|P<`yR}yldIFH+5=WEwuEh&aP_Gbc6Ur*qP%ZFQffu2`m|w zC1d!F;Q*st3f zbGGB}gm*T#5VVOqH7VV+PBLs{_8RLB@E8iX|GQUg_9ZW**5CF0O$V9?h%`5wV6LV zXKM@OH1w;V2QJybnp465-e#yzSL+f;=;;^2?h5+okHNVIRN;lYq{9at7Yq?-xv`sj z;!G9jdsQ@?V^#jEJ$0n?#?}pCbDxY4{(-x7q@nNeKS*`^E&?a}B@4&?h!@f)0h#Uk zim3%x(fYyH9(mR20!7{Y7wYRalHce?;YgyM}fLd3f88Ry7 zO7_W$U79TU#%X=&lkNetUZBAH=imPHPl!YLFIi>CJBvhz0~$AP7=$;CbJ36d zl_5)BicI^-Vs=l3{X*kKE4WrZ@cWsMEaq-<^AB-oG}c~&aVhy&w6f?IV_(R4bCYF| zrwbQj19Pf8nsiLsXX_7xUeCLGC+AdT`nW;qhm|+Qvz*H3C%WnIj^I zIk*DVoAiBs7+1cjAVztajz%7?J?PPX*O?x++Y?ZZ9PK?+t?CwK4UJ+>OD&cAlMf*( zi?dVIaped$M#rWj)-P>s1Gpu zLD`$& zMNh1ZFN$No0^KV;t#^7jw}99J2{7}mJfo=~&#V7n_XRu1*y4rXclJ!MSNE{m3-~Oq z6oOyKpe<_cyH{_@`v0>43fzfD_=f_FO84Bwlb;awBe%!&e8Qrk_{PutQSwji2N`#1 zcC?H5GE|&Q*njYTCDsH=YvNwDp-I~%OqLGw&p7RW@AShsA|KKo!&LDk_&i+2X}+?< z4F7P2nwU0w8> zMiq5NE#0}ZXv5~s2bD$Jl5D~S)}5sQZ<8e%te2jDPVY|}yV?}$576VTI(UI)IpRWW z>WSg8%Q7-sqlk)y+L8jolLb@pAAHX-&mPy>5|@<^{nhXh#Bd_sTNX|&KyUgyR49YJ zwFpZo40If+zW+wWw=9uoYwOg7WS&=lS$N`2PbMYKh8mZQ+zB2L z%)@2K^UHr5mVry}8SImU7|L@ys~nM`UmLzFhBu`xA-G{Lv z$Ea+2k%gS8;FnQ%Rla*xnSF4~9l{`uBRYdq5LRs}?wp?SQE@+gM-=KBBzees=?1s7G-PzEvEY zv&}Mz9ll@^nEf$C*Z;&8ODDWl4iI%$MW1w6{k-~rlgnr!ghuA*L`<4Ih?%xOO-hHv6xp@1bZ8X0&QorC{Zwa<(DT+?L7?sZl*7G`1Rhb0jHq}}&o>8yiMy{zvRVlff$A`4)+G@#4P z8AUyV2d<)dsIH$!y@!@;V#C1hd6AdTp?59Eo{mar1b60oH>)@+c6mh256?|TdhuOE zdSIx|NU&G43XIhs>4i^~QF&$qXz)^foZJ!GenHJ%cRd-Z#C*%>F>LL|{HhqOy{Qz{ zi|`~Hea?GH(DomGXBQsTovf3imi07zGrGLQ00V9O{@CoQ2#D=UnrP7&s6u%^v?h& z*jMYA*2`fK;Y);2_Q^4Bwmue@^+lq&ZMk3Jv!+C<)phEb++L+vA8KkO0}FZrVt?>Zkmv--AqBY43l zG<9T*bf)WKb!(Yws!!-dgE44_neWnR`u%&61?LI?eqYjn{&^@Dj2|kD>>mnjdfk0` z-S@7wDF0Po9o_l#%qPHOlD9mU>*&tdH+d`4d4ij=)v))!uPT0z*v@@(r7XKwP}QRv zbvHQO`<=V~vCy!o8jqL^Sc#H)6CE9%+&P?cY+Rd-a(0sTkxuc~QOYIn)ofIfP2Fo# z?L>spM>R^;{-5@ps`_nAri3Zgfb^hky=WVuR*)TKYX6LuC%-UIq}I^fwKT6oCD5+eae9@}*^RHa%j#a*eark!*Y z&!(sqfjNyB)+HF$M0xEsrnOLI{7-yQ2|mwz z0_j2-ftAwJ57woQjLv*jfVY;GaxfCVzbP(7wS;<^lt=ahyM#kUEis|g*8{oz&F*{7 zhf?QqXC!I@?++n;9-RK66G@j_(P6_TpVr59kSZrn7my>jk+d!mSoevsPKXQp9o|m2mObXwsnM5u1 zesujJsUK!PUCiU`95sv4tKZ#ltkVq(jCI~sgC!vJsM5*HYP`XT6|QeH=7^p+ir5Fl zsNs6(rPtc3yGtq@Ixn#M{BME`4CKqfK!Ba3;l_^-7Qs{8l&hoDKNJEM{g>ilh|lK) z0;Mz(@rZ)0Q@RwB9|KO#i;g%?G*@@N0qATvi7` zaTb4B79XsDumIMaL|Dv#&h`I~gHu7}e#;e?b_6QubiR}U_E+6#T~j0@v>02xj@nDH zi%mKr_u?s)@2`UXOH1^xi;X>I4N*y=sJoqT-IeLbJ<{F8TamnrUgn71!30#K>;gw; z{&G(U9@T8%YTRGmp2}6Lo$7K=YS>lPl#JA-e>B3!TCWU}@$!6rY1vx;DP(X!fDUG$ z%{(y*Pf4-$%qY29?i9?oaxX8gx)ixpkL=N2em`v5Lk@eT;qFWySm?hoVUA9UukDz+ zFi<)2(t|Z4oDFEov%9+U!H_(6kotM=``{f+J%=@Gx|tE*l#&=BLbg8NH3y14NN(N5 z-A3C;h2Mqe!dbi2G4e{bJ>s)YtwP9f>AvdPhDx%5(&0c>Xt}^KT`>8mTH;uiyGk<|pL4idxKXMIw`S7c(3g(@^3Y!kliMVEZ=X2KhBT`^vQ!rtRX)7Wo& zOJ5q6wLxxotvRg5N4Zh}gs#R~MPe!d^VO5NaLKi*Dc>>8rq8&#Hbe&!X8QrdoUS8& z5pG`D7RL|!|AIe%pMdcjVd}}Qx86*1(%l(^@K37C``{j~mKMgU2tFO6??L%lBKSq- z#i9qQng7(f^ff|pyGHD{zomlKM=-x5zZRqkAAclRaOCJS=ErePaFUlbqa0y zofo_Y@yUtYigc0Va^D_mco<=6j1E{1A5@`m5q+=ITc?;C78so!!4$jd<<;D*Ag;Z8 zV~9~&Iy6#y=RBu)Y{^auD+>t;?xf0qjf^J#7KfrH+`so--6d*bhZOeefIrH(O&zEs z2E8G;abr7&0XW7sGq(#kZ{P$J-)a~HR!yd*=At9g!J_Jo*k9I%O+=nRkw{cfiLki}ldj_q--bfE7mMa`2wTqvgYZz*ng{|fpvOt~*U^fWQH?4dbHHz-fuw_#mx zuu&EE{?Gao`~v_-y>$@IWStkE8S$XV`XMv-r|Y1+Iezysb>d(;=|iQ(DW9n0HG zQow2DXEhUwKlx2HmvVWfD%p^poLQO#g8zbHmSm{ww>#5;zDXus>s0ruV`$c5Z|Yeo zj+85c*k^pcjZK9)QsV`bA{I@GFZX`*T&msras?vC-!-9q9ebsC+@o3EuUXk@<$!9$ z59?PaLQPdbU{yer!;*p`yZ&%+iw!_#%+H^8VyvA#m`Vzi#$dmYc7c3OL7+sgd` z3AhFrObgZVvHIp4cnRC6Iz?*HV!1_C6(jy_%DDQa?!BCGzFLtkaL2aGG!|epN_OkM zUUfm@Ehfw>H2Tw1r=}Z{9{4$c+H!R|B)r_a#$MPcV=tL+nGVh7Hs{;|HTtSwl$g3> zPI3F>Ug?9Bm4HMNH}{1Oz1HfAF$LB%*~PF(c!T~^Qh^VPNflcmCA4SVf_apGrwPf^9zz(*nU^zdePYy@oGgTui{Y-%RfYzQ)H`jX$bm zVio=lLhZd-v(@!a=0A}7JUb4ltQ#)0@-#@BJ%4V_$iE~zu{m{YS|RwnV8jC{qkeee zk7QkG@r^(HWb}1GK9#|8z<>vwdQoSDyZbs~dY{zmQTZd%`%o{Q9egT%C`+qBtz!Js z>m2Wwz70oijz9f!utS96l2+i(6W9OhEn9o}ui{5=L^b4S+ExQM&3by`zjURBE}Xc6 zf(nb5{%Pd~5XtlW?^N$1Wn$EbzH9$%uUs%!iydh--9vTFP!M*Gwrf+57uVl+thK ziHLOH25$TA^?1#P-XBy@(ksd;N9ucTb6%gYbC?{a#?T3#?&;ev*(i&^R6h5C6IBZF}j=OTQf)1 zA&^L~pka%Rl@*FPGRm9F^#y`}tvgB(H^S0& zPRm$y={{I9`nSw-imb7x2q`rYFW+FQc04o)yS8pK*bc_0bf6F#S@6aLl>Y433p$;L z#i!4Ml&xqb$VGz3Kd_x@5rN6QnYiN^6i1(M%>9yld;X07{G}HO|1=%drJP0|W-BE` zjC2CEm9)LP4d<_bRKT8o0GdK%dc-AXa8J)QDV z1vTr!KuxKmN^Kofj4*66atsv}qbw@r0%gSyD(xXu+@7B2S9$Z#TT$1QH3#xMuzOn* zxiC5In&aGC-QcH}VpvM|>zErd0iTRtz-mU*3Th;Dr~mPN_XnWP6gC~H`8UIYezRb! z;fdfg#-{p5l1++nm^8}!#)TOqUq`9-5z0t_bPj%N1pJMZ&c z5Wlb<2e(> zF6?WD5qjVZBe_cb=r0yM&~qJ#s#1fjnx1{5(AWeabsf4Z zH8Fi?Qyw19yHp^KA2}F4B9{%EZ+xHL7?V!9x?CBoJN5lNpPEd*Dtv&lo+*ha*5{6X z-UU8yo`99+eJSvqExy{gZ$ZMu>NCDP*daTnmt!D&H;G7}?p}c+$67VRn@RVfwvLev zhc-67PDNNYcRt2$|1^^?rM**rRr-FH#J_yfSm@+~&n15lwnwPUbkSD9uq- zAxb;#$IX1@yERNAM09er^eXJ@)hOzBsoYTce@1~+GjlD>BN<1j7<|!H+v=H|>RxiK z`bh524Hw(oE~;6U$rwG1$%iQUXs51mp> zlw(%3_S6J-CD=%+A8_(HVI6orEZJP{^VEaS3rC<_%HaZor;5Z`<60j1J%IZ_RmdqR zgF8I_Rjv$;jIsI;MOQvbx@S!LR(2;4MPLk3%k{VS8(PH#4*lX080}{ndB-}-HvXD@ zuC9VFBRw68T)t~#?z=zSV9lD@7GWh|mdKOqCfeda;> zfh~18(5Q*;DiwqG29ykkq{2cbb0$>}jCF38lh3v}b@ug~Oxv-gl+i6y4{En$Fp(?U z3hbM**Vi6#=Q9)5S&<6k&p}tMz*lC%dd@1K&MHTdDwg;eA|s%Dz1Xzh?u5z#NZ2 zUHK7scKyvg=g6Hz3H5^c)O$&W_fg71B63xAr|zJv^JU%2okCX8yvV+|cyRl5iJR1u zk;+lw!)*GGNgKzmTwxRobTQ}{bvkTa(W+j~(Z0yX!tgYsC)D+5C9@HU&jkIi3J#U1=JMo9>>-E}0s) z$U%F`DrEUHNb3y9tt9>#>RTM1gSg&^)@NV^X+~m)f;?l!oKXKpGpY~t zwQ}eFjpMDgA=!o$iLB8K6R!TTTY`b|y#r+SCW>YOArx?K02dB#*Kb!sQ)*Y5IuNO{ z4HuIH)Cz+UJ>C~1g>MWAji#qXg8y7C?nBVYk}?QgMA!wp_Xn|0saDnz6D!xm+FP*{ zTc_a4`BKBkow??GV=fH7Zx}3iE4q+H0~8|2P0~(JXAYKHIA)dyV|5DpS%u#6N{qw8lO@ z3VAa{j1dTJ{+YWrUL}`MYO$rUub~O&v1OaIXO8ae`u+&jY{?!AQI5?eW5{+A* z73iBih&VrCrY>jMooklAtraQ4I>W;-^PM_~PKGvQGK%^40W9fu*jG~R_1@eYbAqkI zdd%Rb6T=d$piMlyq}?UtItkIHN5{SUB~f>|jlIuxaovUa5~mxj5$U>?e!<5gz^taN zATj59c7=g5dWgwIMJX-E;|L}K)=#jlT^8{?pBDt!=^HsrD|taKHAbKU=q8T{(aBQ>U{ z*w}!j=MHWEUDETjiL7qYBhzo%YB{mXxoGfHXLzNye^D$h)or>|*w*`u4q&AVdp1WE zbV)Hc=W(HwgKpELYI0h5{MV|WE~!{1PbEY-q`b<28jiR5a6mxU!-jKyMf-YZ9agmtvHme`WN9HKKX zIh|SbJ`d5b%EZOU=+NA@4rz6!+tzw+q6ikg5HIsSk_&%ls$H->sZ*ud+UIo`0t)kO z3s2;M3ZC4GsAuNA)yAl!^=O(8^N@}zm3~}MVP)nf=1Cvw8hUB5<{x)Ss#zDF9n=RkWvNRd``a_FciaBsB~3;G8E94lm9~74fiS! z$bWMhw)`XJ7%V;{$t`Vpl<11|O zVo3l)((aiLz~|V{31;Q(&__jk4C`|!OLM|579_EsksVH8-G_yX-UL^)_2wsTSa7jN z#cy7n+{|t>idcPc_siK(g{p&0poyswU3i~7hi-=NM6EXEM3+Sb zbW=Ka{4q^fAUNiQ$qsvJ_4+F5X?b~SQr(!a-EcJvkz!U-pPt`g0~7 z%@yh6xr`gi*8Ljm^N7c3WK7ySgR*TnN-g;OF*6^No{sBd*s(e3a=ZT`H4OHG*y*)y z=o_8Nj>XSysx4x&^_J(ylyE+T>5Z?~oqxYU4}40vPDl9VJeAXlfQNOp19^6fIQ%HO z6LHZdQcT^%0^aSesf^^MO0)?zNj;`5>iO?*sah%vy&hN+{NSSXo{q}$yQODpDD<__ zhLvZdJT5IpV}gCcB#*@BRvvn-2lckfTAZr`ZvT_`9Le0}_d{yMf7vWi1Da#e; zy2*{1<_PON!dSJdSrB1u@zU#!y06aeB&Ym83$Ouc{BXn=2qwjY4byDMI5&bKc$UT` znGF0Vxr;Z)Jmg&@d3%rWkI`K(hRZ|>xoTAFS(OMj?Nj=bSCjMJj$0r6eUHVF>8?8G z*f@#*uEJdx6Z(o8s*XR+*{>g$x{d<_w2LOK`9M~g z=kB(GM!=eOQEGQyd>~>~Ub%Mu{?16$nNwfAf9iqNVC1~ak@UP zHiumn{yayZX-44o&mxwS^HY{Y?3!6x#5B$}Mlc$BQi0Mu+M`0qt3)10w9osUisTnp zJ^@u5OwS$?CL8ROkp^@=_-=e^`+&y!>5+Jos7KTf5+_U6IQxXl{ojC^J3GOuOEM!_ zcZ(AZCc2$lt*jYlG|Rea&4JW5fYD_pf7u$nXUJ|@KpB#5rnby%+#}<|RX}K1LE@~)TQ%c4Ja)%YkJmL_Oh2D^ z#^y#lg}TQPcfUIoLOiw2+I_>E$q*33zaAt@Vu~^XF5mfaab^d4oBQATkDNpk;p55& ziDsn{zLR-}uk?77;?}n`ArY9Ds?FB<_>Y%=M!bGM2>A$zF?64gObwuoyWUZpN7syK zZ%H6TmJ~PY+$HKNe5}O)Ta%bcQ@+dKsz_%D=YB)i0!fl9DINL+JrF4_g$x~&+!8`6 z0nSRzowJ#!CZJ?B#SQ%mYsuYv-#eC(#tCFeKyITdm2n3?G2_4&w-rtcz2?~D9&&rxwD@vEhvNf#_3QqDiwE`43M$)g5?vPCb*Y?C zjQfw6^usva6_Pwd>iB$c)0`4ML!$DV-+9Nh*@p$L6#^4WxEH+pc%=57Y*B%vs*Gjc z_kf@Fmz@PC7vcAbe84bpULAkhr4ju27}C!dus!{>IJdrzhiXCozMJ6!Y!@Z|` zA~PD&Q7ASu_m%YsYw?$<1#vT|-fIS566MpyuQ@sEUEYGk$;HZ>KTuBhz3!BDL8i6+ zw%m9UN}+x{&9pn&>mXp|Dm@2?IgX2BZ9RCx20%XIvk0K1ZQBO9;O^QbR;1#!Iqv?OC1xu9_5nyd-+qa<$Eo28aLesv-^vLTUO*vloQ;kR8D)(={wrr*e zaJ{W*c1gN#h-{d8JAeX9)KVpumVP)wKMfXdvM@8xxmHghDg?>UnE~A;uvR-1@Pyl1 zX#Pck2Y2;SxU9zk%yvFfj!|B3;~k5YTFQ>33fjuLb8K{YW8L{I7~U<+7JAmrODymis(>a<~3 zJ(%O#ebhv$u=*F5KtP|*tVCUWB@L@kknrPGRjTBV#0ML)j<4Jn%|GV%S5Ih9xuMCg zF7L8XV;k3`MLJ+&wI-~NyF55=bmJMLkz><;qumP}Q+SJtF3AG_3+t<^pH21i+z}dy zODHdv``poEiOVC33SsKirQbqLE}XE;(YTgi-!Vt%eyq^6mU*<4XSu>gul}U5#IAVl zDXpG!wt~&S_Ni%N6^j>0&oa9NnYfx8(ZKR1t`L7y`?mz}znD_Uo5Wui-1gx*`7m-7 z_n~qq!^h(;AC5vhMVDn03vmhgq(f?2M@iWW8aupX{LL< zkQx8uHtMz#-OZ+J*i*1}JjI8TKc;#wwbGjsI>7*1Q_^TRTTdZNL`Am$Me`K8IUZwR zE+)7CdCoKXlM}K-DoR%ZvqK^n7{wdB>D7G`!9PseR#y5E-+R zR88YxsvN_Ke7#4(+PdnY{KQ7O%D%fv)+^F<1Mn^FWD;z{wts&A$3POc>rh?{7D^P_n+XKP@=_bk-g%^$4DKr$li zogyAcJnmS(|6evgKO$I~l`q0U&~Wwsz~}%I`|E}k{|)bK##hRSpmA>@GfAd4dLc&8 z>!Oi&cS8CKHgDUa-R1|psCcEM4;-2Jla?fPq&*)|5D*}(%W9NEwx^T7-X}WaNrl67 zc4P!nj*heUR&Tj-z{CMp1S34*>r+~(!E=7s7yA?9-hX1y|2pWX7Znt zVQs2B$w%rX_S{jV;)3zO#b)^Xi%42q*~nIQclls0=)ufWV4jl?PKwoa`-_9eZ73;5UJk}Zs?VxL>ziT>v6mF#-^rtPYP1+Jcg(WQ z7@s)Pdz+^MAQkYUNX;nZBJ1lhADZT)eRwHid9VJRKx+NG5x$AHQwL7%a5 z)=Dsr;$jNuYnmIAs_B6}2?KgDBVhn<7o$waGq-+vp{}X7#vl))_&DdoVz3N46H57E zroRk*P5bM--a|wP;p^M|9bi%x93gq_!Xg-61Yq^Z0DVz;^~kycivg~3(?ls)WkJ2u zd6e^|MfR#ij?0UY0j3){==RP7WNkZ(l$Ne=wzP-w2SFZ%Bb(<2pggX^}w zq7+JEKljd}%EXb&Y4nn)$J$m7$jYh-gZ~J7b#*JyprpMN6A8UOQt?TN9m` za1PFd0i^f9NiNg7V>$-_Y+F#P%Oke#_M8tId9K_1-q<;EBu5=xHm(Y6KB1=+G>XBr zpfW|pFD??#6-mR~O2zZ*YAWWwSiTHZlYAUA8rse8!$~Yqi&9=*YiX`?A9yAKHwe2; zNYAH9*zxYq=hs!~E^q=eSIDD0D0XhM+{GaRT04iu5!qkAEFO%t3_Q_q;_>3X4fjyJR0NG9cZJN}kUZe#fVxbVYr6#u&H#i3WrRW7E{r7I4 zl0oN*AWqe;r`0#PxG9;Q$%o~!1#6_T+-eDyyza{2CKx+#PHZ{r+wna3tOY)kV=Q|Xj%Jx{w~)8Z32s=A$g zWwol%;$yr>QLXZX8N}c*`*u5<#@YF9()Id_jH3}EHx@25+b@-uDX&;Mc%;=`K>xDpd!AW3^ zbo<3ouF1<)g)NVtZ>!1DzfAiLHlC;ZRAgLDhWWN`X~5`5Ivx64* zIkv}(G9#|rLltiEV*7-e@f~I>PyHtz3@XAY9Bp5Gv|XuipES2+hCk+#zU$9Z7}%<% zlkB$?xvF}q5je2A_E_GTl4(~sn-z5Y?WM`bPW=y}9U?ntlbBS(Eqg+7Qe3ia zY-NozK^wXDeGf|ecGclv^O?@4g-T`O=K9-{8BQ%%YBr~m z!(TbN&7Ijqdj_6dGubXK-lbnKk6jJV)f>>wXxQ&S0>|vX^BY(0pT=@MKFi9j)xQ=v4f6E1{du z0a~3K=~S0rc}HUM*U|{BA*bU9c}mh`U>HU?+WUMh*r4RQnj$v(^Pj#{2b{0afgVj; zq(wys{b-Rh#$>QrY=@&txhB^l)&%w-bHNyWj6b0;WtPXrd`fj3FU}o&XkfN+!z{WC zd#g{*R!S*cjod`mms%6zp-3Oe(tYcEsB*Sh@^E}`CzMSqw^=z#MuJ619 z86bJn@o(3EiWGmBS{XxNTMz5`{BK=oIe{SX=iI2ahNn8yPBz4_;*Nm==_+ewacwYq z`*jQX1B}fGAZeLN*U&($>KG^W(P>fco?eyxsD`iKe?Dr#hZU192Wrtk+yB714b^t{ z<`E1B6V*0hp3Wn}rooJ?uhaqoj<=8~{g^-q3pIf>*ZR0nx!HJhG41o>W1Fz5mfsQ! zOLdJ+=+0LNeGX!#PF=zna?!Ld=omlY+JaRrXgoOa*+& zY4)}`UgQkWy`|FryTv@ReF1+1ZAgr*Uz;f<+4A0DJMqmp5T{1dmSXzIhR2PinBcA! z=0Lfz#(QxGd-ET?lYe(&hp#y)y2xI1B*7Ei zkN`JsEu8!6CAS3!NEA4LF?dY2{bf~J^r*sq}Z^>>z zMkGOh6oM(10St(5i57MtlCb1A=CG4U9O=Wv(JbB4(OSc#l8vHrwK~;pzEq%+Tkr1C z<0(o$$Bs&bY~D!EG!F@S-1?>~4KHGwHvC*6JC-@Wxaor5@{3bysgIbUZ&ps`m`cE5 zF|=(UaA%n&mP?X&t!hYFHTzpDS-$)d9N>HI!1eX_O6#2jx>5amt*ViZF9+p`KZD$t zSQKd~tvCaM)H*bzvYY^c3>d0!e&e<6x~s_7J3b#yBk;SLFMGC zYLrW7q8h|3zIFzQea0^t`~&_N|6y15?rZy&pAOsiW3DXvKiCr0E<=%zZhEunm9E=H z8?x6>W8xO-VzBx6@<1%9WO?r&b4bz$D&@)Zo0ro~LNy(>@~#VUt&JDDau@>ETIZXL zH@U8Ufn9vC;yxFWAtDe{zUBHMU9I~>uzBlx>+EAh}?lDdxew9 zp!>`+0_DiHl84u{!C~*WIV%(q_(~ey)#R69V}}Ud7athDYRj-OU+W6|gSl(<>S-?x zOMoVoNmBFS6st=3>5@Ei-`cHE6&NL>sl(`&55rA0QlN`&gzK;huQL+isw01prI{~e zt@fRY^?Z%yw#PCJ9bUJgjof$aMH@|1REU4?+8PaV>vg6Wl1r<_hUe~6VdKx8`n@#= zFgke(i8Y?&p5H-0q7ezR6>^L+0505Oe8yXY`j4_<6e>vyFd z%8{0?JJ|cP+kMN5-yK?IOUF*860-IO0h*Il>6n^0o$rKk^{_RN<7n16{(|xawQoRHJpZx__aP? zF%ZtHUgC~2z_T4W_Sl*nrSExXz{3N@#D^s3|7`~;Sr6*?${84S7@16|zBv0IoxZ7x zO9HO-46HXm-e3L=QzM7VL^Z5Kj^b-?NvtsI^sMVQ-(oOpzXaL(PP=DZePCXDW$wGbif10GTbLwp$3Hv3; zOz>`>LiNw}4lq2dD9IXBLZMP$w94|;r*B^wv2kJxW7bGM&9PGvXhBb z(0c})cBqrT5${9v)Z}&1$rjrPRwzPkS^LWJ#@LQ9k9gW~Xy5zQcvwiZT48B*sda#c z-6(A{?L_x*ll>wz0FRyKiZtM!CXazXE=#-^JSo4OtvJ?NJ|$P+#dN zZGX|z5sO1gxfv*l9546CMxU!*9sm9#uo?n8-bg!BX5#7fz0QZizRk9OD0h}_tmDe= zf1aJTwD~R&_*&OWn=yb@)WvwK9D*Rwr~pD2K}iNnDqZga3ypWPM#%q?8tg|KX=#<6FF8K`ce*ssI#rLlAUEOn=eYS{uHF^FKi92LVWN@p<%KP42<233+AKIKUMnt zXh+V`7GI11k*_8f^8~wbAK?Sqdr8s-by`|9%w`Er0qQ7om#qX_xEXCHbIkubAr|Co z+Va6T#?8s3;5*7xhfPOgT-l|{gvr9is)FX>b*Zj672M~F>jWLhbW9@pfL%{H+)m<| z0HFi&5ghM|?FQVhVODL5G1Ie)-+?rVt>0KWgv+#zJNcw88TLZeKTwzNF-;Y(8<{|+gu;wfir)3Azl<~8_a z-fp>bHKX2%3r77P5A!2vm3U>Z|N|jzLHF8=Av;+;C z`F2)fmfEAl14*}Fd6K)BdimyD0XT}J?O(5JsAOm$Sc~?5Z#w*5$x#PddhM1o6FTHE zk2jIrO}A%?YE7)kH7H>t82F9&o?4&kg5P-Z>Qv#cN$WV8S*n8bGzEolf_<#dQOUYf z!o})M9=I@KW!8PJroyVUUGq|iafqBt+kd?nG^Y&MG2$iwqQP5lNCaUg7xAyxj2>Tk zv+!&>AHn!I5dk;~woo~VYPUoP-xBhi+orkp;|pXbWW#9 zN@S;g=)>}tr<2IglDyz?3D#Xjmmk=)R`QpGmwWC!vlCv<55%&qeT9YQ{y0mv6eB@C zn0Ap(rP0B^aaPo%v zjjM4!+ha3%g@tk8+*=T5sVtX?Oq@{h)|pL#nk|5J>T4vu@B8MZFv(MoBc{5)mv4Da z&3TliVPCC64f#e7XCfiq4_G_%y1a02wx_%!%nKJ%kiCI>?FmAJCyyB=O0|Y`^V7_E zF=gf;ix1OxLSS96?ENaV*eQv;M?O87-xk z+cw)|Amz7kRT)v1XKyB(uSQ;Hb)*j5o!@fP3!hrhu2IX6FD5 zYy8pO+>VN1NKvaE%eYobh{5>xt{q=u+L?;zk=M!WJ@}`~x*}+NzOx*bpn1hi#$$;Y zpJY&$<};z4UjuqY%EAZUbI&23>nQHbE%{wqAtIF!TXAIS9-RIqs}Mh$z4}fvQ|yoH zIS)~#zLwu!m#S48g`VqXKn;2F44G_R_^$!AzT&bMOBOzRzFY^o%>+4=o*5*v7*23% zBhfh#hO?)OzVM_;@uth|^(TxM!t&~Op$CLtek_3`{h2Z|T)b4PJH4aULTd`W)NSiO z^)Z6-?i~eg`EL_kIGP4%6oAdmK2*D!$@i-Y$k6fX)a2{ZOLM$exbpOusO81INQhOg z;||cYx646OY3|gLb(p4e8b;3XH=xhPXT}m!ov8u2m?e4B? znXZ7x4HRXSzDT12tULaGi`!@=9YY!c*#lJ(m$$KSzQ9M|O{b5agmw=5>sl~@#@F?S zYvmGOTw~nZc+Tedn)UIUqY|Mn-QzS(dKq?{N?hq;Ob!}^dig&-N%7|}b@M@lEI1D- zx0WQyMTj53Ue}#%>&+03#WJF~4PcjBNNHEx8osb&ay9T_xOlCWtLJ<32gyX)yVsxR zZyd7<=&(6(rhsfeadzPTy%VyyVJw~w-jKWd=5+0-z&eQys;^`0%FZ@4cGopB$+f1t zS)U8!nFA8Ea4z(u+`xsA+0T`!8lD_3{wh-HE16W6DGE=%OVKarrc{YQ)!jXGUUe-z zk&Ez^s9o8%@AJkBfvqL{9Ru;*UKEQu7Rc=Hu3bT#&zhI`FObcgD80XJRxju{7#?Rs z?j-5Ayfz=h)cLX9(e1QRrpTVqk1P^YV~Dw2Fq5?lIfBZZbM99p`dc@wCH%6EarvZ1 z^wv-9-fuVy7G#$Dx>1q5?7XLfED~}5sbjpfX|@nH-m)o37grQm6KpiEzf=+JC8E>v z{9&s=O1eJ(o$B0Ck9j|oGsSIfA4C?96k&c-;K}j&Z1&0*c3*xnw#nPDUU!OosRin8=!ypiw@zqvqH+gu~Y{CqU3}8vj^S!%O?7yTt!!m zRG)$bnTTvp8C{mA6;7KrP|bxNX>op}r@V77`Z3@2g@QNQtZ-LY&Nk{(7!u$V2=NMOv?eUD~LV$YGLoZ>H1bU4{;7-CEg@|Loil?^itFsVs;Q_Y+A% zn8a#I&9y9SXGOtBWiwEG5YQ}cv6I08WMvIxV(w=$6NF4}PO$q@!^-);pdbFTpC$$3 zPQ=aL?v7tXDEOTFuKcFu^#R-5NyeOpuG#ATI2X_b^WfslZ3Z8pp9%ukYn|AllT@xr z$-UW$KHisI4nd|sXi;+yd+9yvDP-s}y?gJ^zM>oM9KJwkQrFzeu&NwIYJ2gU#KR2> z*^R7W2+-ZsO|}NncBVmi+0|aN$HI` z6{t%^uoiEY#pQQ{Mw38&MV&NFtgJ$pBF@AS1GY_%Sp+}X?+s5hwMwIbc$yIVG*8QZ zC!1hRdLwM#8_HW+>XO47EH!x0q~h9pq{o=Ww~@mb%yhKn5`TsC%5s(NcX$q0zSL8k ze2?p_oQ@2AZc`@bzOQKnf!#M~eKoPg7I25jNpM#Rfg4jl`(oYPXx(S&pD_5c&#-U0 z|GS@Wi4VMy9K8F;QXd}UQm$ku0<9XaiiNu-F_}2dGpD*U8j*0{#5w-Op_jctH^v2O z+;-lEIg{yVUSnU^=d9?7*`~^D!1PO6Vn(yr4Qo}28;PKwiTyczF{8qE{~FSbs-~?b zlSp~Dc_d)67P>VtQ9XCjl3gk&Se>tyWC73-Q9OSh5@@4x=+to_<^9rhnb^{I!BwXpP~-?47%%KGdx&-aJOtC%xA?+`5+zL zfOR`=ZRI{jcvEv2u4y8zbT}<(5@mL5w`xnRMK|B(ajcjeq+js(dm~;~-yM6D@oz70 z+t2#}ag@bkTPN+6-}Yn^$N^zCH4gG2%!{T-c?C(huH^Oq4`&ClTs^X}Rr<5^k*q91 zkGhc+Rvvh4u@_Uk7T9Z8cLy#gEz6U#S&qV{)AF)_Dj7d|V(_&Ak9N9JafT`$Bn;T8 z>^vw&^Pu74O?Te-gsB8Ua4}j%pZD#13-J73%ZO;uI{rsE$~;cy1XUyB-Ftm|DyqVW z(043&WyrQ9eDv*{26htL&dqyP4{ZL3T16-fRWxX9qdq|P@fs6Lgel@iKxi{P%X_rlu3MAf^ASXm$kZ{939MpCyx+qozxlocQl5 z1{-!q>8FdNB|cK?7fli=G!y4(5AK{UKOU4VkrO5P1P%{k*d8Tx1?+$y*!rKiXr2E9|a@Z*q zGz8r8FE&2};$Sclp3BN=gGr!I~nk^gNop$-50wvbZWVFa(apSZT;iz`;hPvPP&S$44W!<7( zU~IbV_sy68-C1;@cz>m;o_eyVfNR>T=9u@RF-r*_&u2M_O@OglvseS~sC+n{vRGF1 z{8rFq_*dN|9d97NxQ#j^ZWJGNfmcrV{=mPWgIT@z=b^EN+(DC|>DdQ--6>eq|rWFUB zSqs8zIY)|9aZll|Q%w*1??0(i4|f{W-~gAGrkcxt0@8ogEUole<7Pjl?pc@1-vOqL z7*WO}Z#m)k0sa1`=N!~WuO}qysOg^ugZd9rH)4F_d!EQ;(y+$8<&~RvQ*DWmBpH^$ zRlQt}aG|@@{0%HV{#xnpXME2X4X+I1mUxJ(x~(yS#sPIl>9i~H#rdP#VaY^@e%3r} z!|WBhDWGqpPr{(6cA0=1DKY3BpR}-eZ)iJy@2@9tszM8&2YLA@6_q-FI;TkZUJ9+j zRN=6zx+Eo`?g6+_>@O}upUD$Nu9jO`sv}rFZcwmdM~J%l4`%#PhIL<2iSgD%m1XV5wR$Rz@X4(eoYBKA zMa1?6N&5$QPyBM6U1k@^ujeYur73JcrT%p^8A>d@HTJ|oEs^jJXt*8L)bx?Bd=W>0 z(hHsz+ixX31};ho>pF7dip+4@ss?JN^A%`mPNUZo8e~HCK5MVgmQE3DQ!SQrEsq-S z2+Cko%p-h@8`d*&^zrSfG69}5^OX(x1x=rq)J)eLy#c|U+VoqswCf5z@9Zcf??+Zt z&tuots5{k6%pwElR)a72d_^rOW@ob(of-7}KSp?${G^69DV1Fo>LuPV-7ZSk8;?~B zJTU?Lf3fm>l@q|w(3zs_cu>#xFZON5RrAY?Wt)@A2}W?xo93$>I`PXmu{F*18|iE` z)LcsE)#l!ovTKQ@e^(7KMp1p!8)mo9dmhLncu#h95MSjN-7Q%-vox!4k9l$tT#ovJ zJFkJZFI|t6O!kqwxp}4kp`V6#7P8NT5dm<}Jl_x1i<&ij|9&z)HI3vHagz9Qsk=X& zBO005ZL;Z)NGx!;Ml^U#h*7MUt~LeqVDT48GG3CZ=D_YZZ_jY|k*1*Yg5b1Z;TC!G ziNIAAZ;SBF^xqY;V|E}stZfZ9&#IDesG=jj7?Hsl#>^&h?u0-TSW~&E1(yOBTIMV z(CuCQONswu*qI)9{1mHRkn=`IYI@It$Ns7Ms#k9{_9`w&WOP46-n@)vS_JrZNVRHu z(-0ZF;RrJ;%(y_bv>c?Ud8W&SH;Vc71Bm(w%%;9=u;({;WJ;FOHw4kP!3p2&0Lys44+WnUc(Tt|&DJq>x=Vl(_ zdmH_8@~q~NZIvp5dC$(=TiDW_Apw?3EdK(0Vdwsze=T*8j-1)aR|l$@R#|EzKeH+8R_hoYi8f5VJ#5w~;bN;{m^rt70lG~R{qh0sZQn6F zhl}vIZq_*1qg5d-21;79p>46H75a>>b<(%DiJs4ML@&w3_~iU5#2;Tg3$phw^_@*3j(;jc8)(LUp}Zvr^~b zej#A}2ja|4+YmJ9#4ucZqVY1#Z?FRBRr|PjH&8hWOX=InT6kc;lTORWtY*F-r|01G z3DH)4S8XEQbN?$QAKD~g5pf}YEU}{(T+qg7KR-&GKXf1DuwhEJ)0Vi-REuVkf6fx^ zbC5l>ifypi@n1nDJLELTe<0;wMtVLSJX!Ru&lMGDzH4jko~Id2%4$&WOyMz|ct68| zO|8s39RzQLY*zbFr^QV=sO7ok&xrNc7nN6iuW%p+OR=ts)vUY@YTk;q8Qmd;UT{Y% zojX}^AM1W-R->X_m;^fNyBIH4Lf-!!^3;k@q+ctX$0T-grlL00;PJO+g~K zL2rLyfJu!P$isGBe8xHV;a=F2=BLMe=2z6LH44T=E)^fDx^h=Zj#eaYMCt0*_xTtI z&*?C4qh9_`zfuNqFK8kR}o(9X{PX zAEd=J(enN8g()w6lEr6}cD*kWjK#*;&Q-I0nBTu>RZoAE;eV~&4l{_{MSUxz)het4 z9W@@7y-+Qd6JR&?=&7kvcKB2&I~fVy6<8^2-i4DG-r2nx$BJ4XqVtCZecqz0>Y2+( zixK5}%uHB@Ns*#D-^+cZUiCgEt4};yz7y~v>ZM=1ldX5S+LLX`U0W$TnOze|x{TcK zvsx*Sku|&-d7a0%mkWmBvQ7dnJk71!6XkfJ#!~jgt0w|YhuZ_y^LMbT+^Gu3T1VoF zNaa+CL;n#*U+sXxn>G&2- zX!U5(6JV;d(MkTNvy_{_8#4waiBG$n#*3deb-r-${3$8HsTUka`d0V6FZ06BuJy1r z|2(v(jo^k(sGwF;cA!y*PaFxhOq<87DyJTRq5^-Z95A87{eS%ZbMvb{sW{-)FgW>? zc{SN*Y{x{ol`~dvosU?b7zP`nC6H^b{O2qR;d9BSR(2g7Tp&jFn%fyGg{n4UvnbWWx+Q$(sCYT9elZY zv`mUDfs0r}L|d&Q!QeO#0v=Za%qnt( zHazGpXj~9{B7$Hj&>t$iCNQ(WLmvDxs%nrtPFUdxbaFb~{OKil9ij7TdOlK#lyUM~ zyUO)4o%VZN##*{8cIxH2nb?2kEE<_lSnfcOA|HOJ>)QX^m3j|Q0(Bx_gXh}G+6&CLfkI-;26jd$;xa2M#Nd-V_{lb$Uz8 zc;i)K*y%Vm%c1@3rFPc~+7)V_{@?aV3oI5e&E_6N})>O8fdGz88F)RJIJ)x zLi?wR&71Dx7sOWU=rkRx^a*kALd8ud0>eHN3P`x+1~c}t?wGY5oAM&cvQP`EQ(4E_ z@&~R6+jZOz`^(&*;YbW+4bNBew|@1^$}RWhz1!x>f+`CId7{XW*j}3d?B-lvy;n^7 zl9)_C+T*Ru`(ANR?qS0s;*lKxThkjEpT`OpWa1x|6J~a(5sn4ruofv7c-pIC>WfbL zn+x0!aY=H=IDWp%-a}~)_`RD5W~s$PBC<&7RdG}Jm)B*j_Nr-9zs@@Jbj9Xdr1ad$3?0Z(12#OXt1EmjkeFmke2t$V9O<) z9L+OjOqOPG)V2N#K}knn<>gOfNvwTs;bJ^&^f`~I^X;Vb{5jFN<@9izFi-ZgCX|M@ zf%Kc9C3U_=yNr{#H)z@WRLw6J@r7e>wssNCCLb}(H^y+uykt}uKC-Q*aM&k3@l@(|2qPJaQgrwZZFX_ns9r1j*ZQjmJh&lBmA9*&~kG_A!hSzwS_ zV#UmXB4Dx;{;!BS>OF48D^gfl<1Q!eu8^~@Udzue6?vi!J*-7Z zrd;L==q8X%Z)<`_N!3{*+sj7zR_{zk^Aq+c0{x_XDQSj-*IOn&W1EmW0jR8ukhz*S z0(bmy;Y@ShN7vL1!R1)DE9nv#(poC#ZSrH> z^&h3-Lxv;Eer#~gaQZ!*TBrXZkglPe^-S7SiY!&{u+DAmmj&B7yl&@Gl-Hd_3q}Z} zg&x`Cn{7omZ>~GJXIi*Sy>@w8!R7fe{z$wIRMgfOYAh4a|Gq}k0^r0FFlpQe*k643I) zc`1`&%{+{?V~p!^QF)pVSk^c;s%<^cwg#rc>$XQCO*zdYuJCBC(XrO0A1v#Mv^L=) zA@a9D14R;+Z_^Iy~02SmqJmA$<5`WIIq93L}v!grPTX+N!jRwnGcC(YyE6 zmqT_xlKm+*^%@-@#XcNFA6kA>RYZ~RZ&@GTd+`eC=CmGa`&Iuvl=U;l!$C_YA7HU9~v(9h3MM#9^W%Y?s zNzL8%z1Q}$>-U5LwjkXa%5P{I>DNJ*&(ZA`Fwc_C_mrur3{2^$PB6w{U*Y=C=m*GC zQV`xQvvWW%i{gQ~lX;FgLf5wvhN;uMoWnbs69R!zKi^~Nd^LDm`>N-Bbh{rnFP5K+ zMFEy%zI4&D%HLblb29Ao^)?1TTG`fF8lJ&o#cYJHEfa5X?2MrH(!Im{*@Pm8C%Jn0 zIUB}Ohy&H?aILo42^(Pv0BV><(X`ifPeivWj$>OMJ0y?=Au>mu1Wt)=BYuA72M0(0 zL!|PuP#4Lc1{%BKV!r|yuyTey+GQcb2K5uxj1o=ny6-hVrWP_RI6IU!ZL#H2Kk9J^ ziXA`j?AG3k(ObA$7;c`uYB!!PQLmb7N_Nkdl35gX^zT`EL^%trFBGPh^sKl(5n)d) z$?%sdl25mhHT|i7>~QsYRTwGf3zC11Y}y7;8_GTU5Kjl#Pl4whv3m>S(Eg?1$B*mE zNxTJ~u`<{a0zAbV@owRuB|YoG=!RhiiMzA}F>rZ%FD;*C z#?@uQ=crH39U?Xfkt1{7GN<+iX2*qQw0n0v!o!@y{v1X4PyATNr$QRbYm-+DXtv!e zYb&$lmLxKC*0LV>MqD%OrtZ3Xq{K7p$RS0?bn(HAKwc68D|Lfb{g~Fb!FP+zGSMIg z&MTA#VNh{0*hDkm$F=0xEpVZtUT*ZX`$J%4jMH`n$K-?5V9WlYT35>1OS9P(FQ5(6 z)nIvV>Lb5?p-?zp>Di(islf_0_a9ICwyB<{a|w~SJAcw_-Z@T2xUic;U;N7A=cS;c z(_y?V{J*I_EWuxv-WC|TGK+UM8q%`_leJPjp}6_<>Cf7XgA_ZZjUJ6b`J2WI3+wQ! zX+LzjQu)V$GNba3sBQa_MMTqAb>0zqi_kF8$72_U$5t2ra1%T@^R4iDIKV3yu6x^XNeYn?Z0IMVsf?^DfY1L( z(o-pL5Z*9GGA^BlJxCg{;HLwF0x-?NwOSdtgMgP+j!0ZE&owBfZtAlepy|Wyz62MO? z6O+7z6n?v&QZ4KoqxRolr%OFC-x{O8MwLd?{ZtlRjh!<0KBelSrVN4plRrKOg~kKOd> zz1^D3(3$=kT0oL!R}%{+@}Uu{pR~o%9V^ir=jhD&z&|Q+VM~1;s>PSL^5!4b6C4~(Xk6whfsPv@Ntkdyy<-v9Z<+V+R zl91oYU3}{@4fZle#-XKLN|O0jigT~6Fb`Uf>RnG#RcSM()>JSL%6Z}^q=+3(ADAce z)RcCGS*`*PRpfXMn3mFU4A|DZ>wRvOO0pDyg1WPw+myuapv;=yTyNNXEDc?AUh-?|dDs zK$VYiEp-MtJ$gqyl*&yx0MY?%2o*<8bz5?24d&f5cqm1C95ngBDBPwOa{K0A0Dt$c zmeq0F(&}p-q=wxu{?8Zb#|8AeL!H2+e>e&nej3gAu>GFJ%x_?j z5F!=?eAr0e+DJe_m>~#EzPRC=*hV^G1|Uv{RzT=n4tB#!{)e2DT_$}b`c(Xn=drjv zf9OWI-L9yurQ5C>=oDe3sY+tGMAGHFB{cy)Z)b=WVd~mwS^uX5=QfR3Cdild1%Z=m zNa9i3`;ZN;mF8!^^GVtg1H)KmjZm`++Q+2IrK8FAb^PJ{r_h;o!FcCKd*%b_$%J~(vyG9$$9>D-j1-d&G~tIy?E1J-Ro43(Eu z>9WK842j4HUnAeuO5S5B$t!(3yoVdKz`sL&XivkH1DP!9TEc-GY9aF5h(>BPKVwmw z3DE~bddo4n(#;2;goj6aYP*e_HH?Szklhe>3>9|GgT?A_?vWPO=JwsV;~jVhU-2x3 zY~3@KFOx90udk2pRx7pei|ODVhTYL(n4)!2uaJo;yJ{*+DwrLVK5u-%e?GkpUx%ra z%(82&Ce=2sx`XB^1_LgLZ!~TrG}5s390{=nL2OBzN(v)K;;(KesIf4#UWcqP!QSXe z@t)G*_=*({%a$9y$BZ(PNQcnImfPb!{=Rz+@H6-Oq^gSvu(wvtS%Nn&8XkNVVmugB z^-TTA-$hDM8}E6|E8EKaGt_pz$}98Tp09QIE3b9H@4My_qV~WS2Yqp2Ib4v5C^8+g74VaF;7**V#uC1JPcn$7cX2``7IYg!+{i)sJanL3&5IE8Dg5&JP{*-MwDl};0I zR^D!1H9Sj1nM@|rwEE>UMP)L<5A|vliMB_E&739+9}Ip(i&$EUW!LJeYyuQ^6dK|B zTD)3!76ug}bLQ`&_rsGYW+k*ph329}kb$xd?+=X87Zx=!+!paVuNqdy%ABx9|c)yM*5bK(~la6;1j$xgTJqpc-kZdc;!zE;_@H)5+S-7 z>@wfAHb_(PNd~98qMLKZ_Za|Aah{RO>Oub<(nf~`N>&#F4Svb(JL$yjzbxf+Z9UJa z+2^?lFB1s6j8%*!ifP1B02lNzpeMo&Dsl=7lx%B8B%HaKt%GtM-*iCNB7#RA#G1LY zfzw0(ouq=IN$6FTht@#%j=32Var8`smLpO3OtK3!l(OrGoDZ486C@$2XV858c_YhL zku*X4Clw~+>9^*1V2-VOM@5GI=I)pEjIV^6V#iZNP3@~)*2CyH`RuFF0-4g*?`+M9 zB{{lM6;gi|a2LG3N#V!e!i58hb)v?sg1X%bx_XEI;V@aGHrazzj6F zDIY{vw(*#_J7z}XxwxI`7E*N;Z9Nx4qM;YN+vd~|E}&_xuGB(;cTEy)w07j6K{P+5 zj;!%ccSB8Hc>6~hKiSV$@ln6>nl*3VX$_Cvtf9HDc)dbT;`Z_E^T@pWx@WIrV0=dp z*{YUspkmHseM560qC#LXHg|p!y`WWk6Xx0>E9utEY#0?( zJuHB(FRchlNCU@>&5lQ0cifoAk?k^p~hFH{athK~~sSDH2~xHl$9 z?8f&O50-9@S!w0oZfzwLbFlM8eoEF(3917;-V5PJ)K-^QHFn0EJ$PK!c6<5e~OwfAc`skV0qSSK8El+l;qvfZbr0szn1nn zrr@l3Bz7>kvkuE{xPbiiD^TO~xLN1|P;V&_I#v=Jf^J~T1UmGqR&Cu2) zxege!l2#tn;=A|&%csM!jmwNpNs%$y3c+%Nx8U5@4=4!H;J1YP=obQbzBSohpl0`X zdb%w6pYg4_!e+Z4t$tLVBpn$frEPDwwLN%rc+mZ)fDU0;pbnmh#HRljROwAVA81rN zf12`{^MEG4>4MQXPSc>$0nlup#^^D<{6!6uT-pI*c@?a+G+lbVu#&&_pZ?5~51DY- zFaf3m^ZA8yDEhKZKSm}-Z+%%O3R?TaY~gm-QUxRA+Q%QGEY1C=?$I8*9f`hYTOCwy74F3sg=crc9;v0OJkhM% zb2|IiN``PUr%6Y+pKgCddHqt1VMo&K?9zHYH;FL&Z1)5|L3Xr8si188nG3WwABMLx zBiQ-d&Iu)#%zUVJT!_GZCUk~g4JfRs>{CVeZT>2FIH@jiods5`^s8_x9PQ8Dt5EC-2%ldZ0m-!?Ka@OyuDvXixJPATZ@M?S_zsd6?!}#T_{N z?)BN7!ZAu6J6^J_?)Pl-ot*I7C0tGS-V`k*67zpi`#d3nTrM|6s<<+bGFrw9mzXe78iI)WU9uwqlb4pUwOo*mNa64)&Qfh+F zX4rci8{Z3ZtKUdo{}HhYgawMx&bhRwe$8Wa6Mq&sHrBu37H0p z;>&7R1-dxpU_PMDM0$a=7KDD2vx@IZY-1Xo+`=PTOo%WpKkytIF2AjunfW6ENK`Ja>LdV(E>$GQW$7 z`{Lwz*$M3RDntW~LEEJo;Hi~CZ?{}S@P!MTDxjL9FKH2JO#1B0n<|0sSY1#!CV|2= zIL*{D*UWvCuKLaf0|f1vvQO7MuB|t|#^<4r3mjKWn$uUomX(ZWI8<=D)JuXi+fOS4 z@ggU%#T?Wno*bhUFD)v3bn(P7H_u2Cc*riSG5?ORT@?qO_k>T?#*6$)qh5M|vfe-R zq2jiq(Jj3_e|>`-GoL#&@~$qDL6NY}N{-4`tFG8b6R;Y z?DBp#r|!WC>hBOu!#+1is0J>RQv>uAU2)$r+G zc7ktnjE=-tkC&b&&9&jIcBo{L)%(E#0YyYOniv*lfpKJr1}jj?4FDIg%%HF_V7)&~ zpu7~xQ?C^Zkt2OZN=fXsWT@V*&HF_BIt^VD8B>E(mWtb~t=ws1hDQt<*dVg;bMFJP zJ+@V@Q(KQAdCT4~EF!M;I0~LMP{w)z^P=)vwvyD3<|{xLWyyV7k^;ie|d0yjr-6K;nxgW ztb;A!?H}xR)zWmt>Q1S87N*op^L06Vr@wpRzlD^_?@c7?w`?Ryt+nO0>O9@>jrlJT zXU{o4{Z9MlcfogHAbvAi0@1g0^eEG**-XbxvDkCeTaI>R%R)Mhhzw_-S^9W)PXSd` z8i@aMe7>J+jKi0@aU*2=R5ra+oZrz4U|~Mn?N81#UF+3jB0U0I2=F~EG+`W@|7;FwTip9ZQhui1*GBD`z@rg+$V2N&HJbO=KB6% zo=%m<{Kioo4_)ahA;`85rnU6e>Q+)R{M+k0bdLQoCf;&fAzwPogHmQ`n!SdzugP7y zv{>E&J}*&kB^U`<2>jFLz23&1j(6wm%z}39`LQooXH`FEu1Q?EqCc7hemW@r!&&yU z_78Nl;RR>uWkojn!42(UrwBUYrb~T19!HkcA=LmOKoa?#;iC;vr_e8K7jwqj=4?+U z71-BZB)bc51@!-GHO!c*7a-Lcd#`32?dcZWRZ)JlM_|_ShxzESOH1#K;jA+6w;hlx z9T30;$89gO(nOkE)?HTMF_&4-$OFuW++|>#+tfzfhGCKf5p&R3lT-afF`m9D2=@&Y*a)(>dy3m&Gl6SAP99>>rhx9>xK29IwLdtiBI`puLe1 zoA2oHlN&0+V>|lJP1hk>-SM>>j~u1PY2tmP4-hk)wJmT1+#r}ivF`C=fsyeHW8z9c z@5pyvnr#VJ_JH*Gh~#(3gH_ zv52H{T;t)E@#Tp#ATLyYaRX1_kEby~%qNk{^zUT=(ZALqdaWw^PZ<_dY;GZja$9>6 zs9ZG?iq}F`$y`tqJ0TZyA>7dJ%8to@h1%nb@H5?<*SYaRa?62s1a-x`y!s!SFcYfx zl%Ku1@vLEYlKI-SXeKbO+sj#+k+;p6HuqQWDm%ZVLnDu`AY49Xv9te)yJ-c+Hs+4RpcvnpJTA?eEW6(QAwzHs);)y-RotR zJ!Io4N9J5>ffdqBcE!4NLf7rAUk1O-wm%yzO_Tdad7rsUfn|H^88qs361b1f2}E#g zJ(=s#n0l!N!jAl6epTgrjI*7{*e2I4)O)U=y%o$l8oct}$igwLuAk5eJ!Tv}!p;p= z4FMkqz+a#ekNw6^i38LC&q3os`Oy;&TcAcM3X97GPf=QAmoR=T;DEEC_fcsjcgVVT zB%G(n4i+S#DyjJbbw0Rl5zfW(SoUK1{3kb=Crg(_`V~l9q8JgdXTN*l)3L2GJY4f(Smks^$_8^^ZDmtpanUkq zv`03En}tKEeX_@on~(>}FBSR6gj77pp|Eio3^W{yajb;O>^wxo^QWwZFGs*P8OMbv zCf}fe~w! zOY%>1TpKPA?yqn)s8aen&yyH~98F@1VE%K};FqLX7w@(_8SvURMvmA-k}*Uu(hix)~(>3jb~~v=Xbv19vYHlqQcNR78Sb;FqWd zFu%;GA3yY@x~i-FB|(xKVdfYk*1472=|?(;d$a1hy`vLzO(yTNWP$AuRL^kt%_Hgtbt0To{k90UoXeR&aD&ZL$4f>Oz~+tR*LeyQGQF{SWRfp|qShKDqw4?p$g)b`gXCxvzh zY56TS9@4k=1!M`T)`w3_X)=9d6U=zl{1rg}ce>0pKFgA-P=>vkH}gK!m8u&D*;@L# z-8)dGh%hK_e`?RXgECD~4Y6P!P$B1DM8Bje&{NV$$Xtedy>)Hi?5JM|_6_;M5+C)( znNNQ0jNy9*{sam+~v!=;epo$!%f-^Zlxrm zRiT{DMQ7H5Z5KZV)>duT|K3Q`ChmD@q&0j}+^OQ{R~$eUk~(#mEQI<|TF3M|*z*R# z)Z=Hq4o2Lb)^wh#-1&P;;f9u{4%T`@@NF=I<3EilUw401+HI;4y|u)DH)6RWx6bzy1}{2ttnYOUyt}DRVNhZlUCl-ExMqs}W8rOLsU% zy4exg(zl1c!T(W*zB-t#IxiDFGy-U~5fR&6PyoO@;9}e&)Z6m7?dPkAn90GSHjMsu z$0ZH=H4AcGh&vM&m;Gwh|GKAyVtL3@Zar5y6?y4;1{z8@1TFFposU98BTqtu2F^SZ zeskd+dRzjI0Qe9gibn(@%g4{v;6xZ%Fe-c90;vA`o{ zIhps}0o{Z0jn;rN+PfDrtGrNceL8g;Sz1MM&U9DL*(0ehibn8}Qb5YU;}B~?ckeh-cm$ZTA8+5ScE05AH?`tQ1- zYbR^N-w*|%!8CSDKyk2O{0H*j18VTZqTsB%`6097@i)hc=y%pX8}PVZsr;l9jXm+} zy0kb2+)VmNo`J^{KPoc2t62?Hm<3b+f&4R)xxwM6GeVmNzfyA=x9zx z0BkeVtPIROS41RsO~%C!n3Dcj;p^ue%Ex*(jg+YceH#{C42B;-+7-FHY5G2EDybM# zeUcOGPrIPXro=@8z>1c1$Xw0V&2=|o$Dy1yM~WsouarT(frh3eT_;r-`f;;P z)^Ai_EC0Rs|AtiToN7Xx8RcKi;H9u}npqGcy-D29%JzP7ni|n~T&bsX)9$DBHH`a){%g7Z?R+gB z=A=7@MjXpaL2`yR7e}nyLY`0O$0kYd6_aci96FIb<%D*otl3|O(W^W{t8$B>U*UIv z;FL33k$i5ON3u#kzTJq;MrG;Yl}w@VQok7O@YYT5y$@may1Ud7Y4d~5)W|}K zX4SZxwgXgcZ$S@t%o3-#|)+@5i|RmS_OUZyE7w?eu;- z-ZQd-czn7MU;|)q+RH^j4fbR~sQH0<1w{1~sw)Ah%`ri~%criyj{g<&q8Z+C2AF%q z;*G&Z+1a!`qT`7*o4(Rx+c2R42ZsS7DkYq7IkV)~0AQKNn#%qxF4?F)&cXIEG|L^- z15NZEJzG7Q`(tD>>nzE_U2Srai(9)+(1PUE7o{3YAUA5opi#WaEkk=+itN~_jGRN! zp2WD+U11pM*gS6a(V#}_bxv1DiQ>pbhZab zD#XTc#lw|=>)1{;VcOk$G=Pc85mubj^zi{z%c9}JK|~#3{tjxaArCIM=yaZES2dBc zMkLib@QfO*jsBwK>p9owDb{q~zR3tigor$ytLmadPtQhaMMU!H6gl+(VzNnSZ zm7#=MQ74@}S&Tg$unnh4BV`e1gjEPtJbYF4s`1FSl-uVatEs;OfyeZ62nA`blmD=u zvhJ#>5_iCiw0|lVCYJWzUu|CUfD>u?askY$dZCo*Su4`Wu)JaA9B#+7c|fho?cksk z(0^n9b(~zaZUKbv+J2B`VKyK7u^^93dNW&BV3b(W8+eA43bDa=)_;-NGxc9c`Kf%# z<9#rg_{l2f40W+Sh1%~Z5IiUEyVaZ^Vej}M`}vHj^p1u2FH=gtyFQxS3~*rEfW8sJ z{<5COKRGyb7129QPJ#chkd4zqbp;=LSid)OyW{-XF8)4#->wGUp6K&&haE}E(1f+) z(c&dj9n)5^6$qLw=3Wqt0?Lnc+=|4XLV25q)Mpz@AG;On5JkTX_~Z1a^z=rwUMGeo z2ZG8CVS;lzSEWZQC(8}HT|-O3KS20F$oyn~|FcU1A^Z+JhIU4yR)Gc>Oybau%ZwoK(ON@waz%ae>tH1q}n=?ouxx>A~%q+MMEZ6iVF z{t9Bd7C3Oo5urK2gCzc=0iEmvt#EN47~GQoLRFGQH=*lb@6NULJ$s|k!|YeJrSVBRImPZ!YIx>OLDnVPl6WbnSZecx~&M0I`MV;ib*@o2+yEn;bZ#>t{qZ8qaydJGBct+WIQ7Cqi%cDw%2@yCUkEz z#E2|XR5X2d(kjlr_hahyCb#+$jVzc{g^9CrpaU-&V?}Ik?vljnfIwX+(bi*w zmg^7q8bAS{-hN$;q<-?v!qU6q(@pB1Tl{h}Qgl_3U~-3tCMo1VcHR!OiZ!6U{?1rLd#fYo$hio>wVkwfv`T}7yG96teeNaO-N+)yRpqzE=wAk0)$!MUo+o%*1Ka^T;&41wAa<>8Un=GC3l_gQ+n)PpHS!InGqsEon2bt&h+$i0R$H4u0RdJi}(#Io0-9-ZM=c`ymVD7 z72{YZ;{ACjrT*WN_*g{APQfk?>id4L(6C5tF zF;4~5`r&$#_xLI$&^XR6Y+Ys!<;m5&xf2c7Q!L8zK(BdBITWWc=_!094Srh3J2c zFxD~EPKRH|Oe^Uj!uIa7K%9_Qs5E}zPnv=KL_#fzK3X6DHmjaYp@*Hry5}ZhYd-&E zIBs^_skJT|F$p`0axER%w1WH*?0e5B|LRcrq)KBUIA(U`w$1L^7NI(f(feN_W6Y3wXdhemLJ-5nwUl^YS}(rlL&>=C)^TX48} z=SoV41AYaP;MM14%SOWoVa4P0>PH&e&Wi!LBj z>Au>%!6zVcPDxbnTW}EoV!A*>o=Vx=V_!bo9xE*XtIgDZo#%W;o}kb_76WAR5`8cH zy*?Ir79Kt=;JV85k+=5f(rD>V%>VekT7VG4Q-yiav@T!Jo7rRf=Pb4E3Rex1!k=8Z z#I1Wqg9(zg-y3HuSPB&bapI0`2isu|r>f&k@#27|Dz4mvMbz|!&cdyI@rPsNQQLMG zRPK}>f0oCG;!d;a8tl)FufQ%@7Ys+6Zv>v_fUVnN%_rg^>+R*h+GRznkf8NmkZOH{ zk59N@lJ4ac!)7JFG3rM}+ZOsn@crF<*i^Av2f?plwcg)~`QsS&PnV0sGuUKBv|%)W zA4mn}o0w=ou0@hGS<6i-8A|k!{Fu@MIamGUQDIQMY3p>{m zZRZWr%>3SA?5qtm`Xf5rDCw4b_?Cmy}hH8iS8%p3hnGA<3>W|0?ybfHoO^R&h!_SAUtw)fD z0@lvW^p$bU?@LU^-@kfl*rjphd}gl&@}fAF$;;DHIIWu=?iEVWyGX;SHSUydM2wF9 z`|&1X$ zaqHFvr}B23%`mO00$M#AABfG~Ng}L6d^=sInQoBTdO6uM%pFy?3+qzPz*{VV>ukj5 zw+^0QBlS${p~AqQM)G=GzecW)GPMnSrU^aI9N374_2ot+t1eEm{h%2ZlIS{-VW$P$ zk;3LXf&5Sg=MeDqzZ?EAt-&dR`Wf^)so#=Y8hOwOAt*n|iMfG{ur(ov3FPK!ajq+x zVkSx4A*`9&#J%>~r2}4(7azW4hTA)LIwzW`Yfy*>D*h-As61OM&?luk3vhh7w^X>^ zCP=qUK*NP?4XPsRO7@zP^l40EZg#i-$U)Ew0)0V0BKySqEnqA-si;iJW?4wQ;Wbsfh^u#l~-%}X*eWi-xc1(1RvaYZEjx>bB$+X z_lRk`^Gc)=5~Dt-y1R5}8s9gO>coJt?lLZJrPT2&}}Gwe8h^)R*qmWTQ%$R zhkURLlmBel*nS-!L=L!t)~}A8RO*5Aj!6|LsS^P?6ErOsrt8^@g*Vd?SEFuZm;T zQEFA@sN`Hnn4&#t(nsXIWxk8;0>UV&dO)xBh#9l~{%KoW@vf?;>3?-Y^BhwtcB0+L z7=7M-f+cdmS^lG3S1b+vBjCFh-ONWfQ{43H`az&$x5Bc(am>^s*ZLGO24FCMn``U- zYIly*r^X3lKoW4L3MVJC^TGVW7b6t5@tom2O~{1Xz2va3owy6dQ=b@@v?VFnN6ya= z53m`dGn8h4iZ)6Y&$6hJfx+4k+ zGCRZE_<%9lQF{DGwKz!UZjShjgxg=g4Q_K`i}`hj5AOGA1E#Z>6s>9UiN<$g^*e9q z1!rE_Pgg?ntTG@gBGM>eRL>CAY{rHIS-GMzJp7$j$UfBIPNY4ObOapT4#Mq>eP<)4n( zOCl|)QWA}=k({JzM4o$6lqPf#c?~XdM)FU|WrNxd_adEY9Xw{Q?k*lokzclCy`SzxT%UAF?U z;|0+Xx~p|pD1d&({N>{nuScIa-$Zpud`QWW&a8`xyG(pSYty+Yv7O;{IO{TI`&#{; z1%Y(;M*vs9(rLDtU#{p3@Zf@1S-oO>p}@x5HUob_gy1ZC!Z*Bsk9N6UAD`pjmodpO z1j1==aJ7R^PK!f8*y9-0s-JW)$~`Aet-xm87%1DhS8^WVWWP!)U?HNjrr+E^)WN@=Fi=Xe$)h zGy>NBwnalI^a$}7!HXn(mKMK?`%eZ9=_Gy}f8Z=a&mlxzt&66zjio9WrpBrtdY(mt zzXK)#YHM+ccZjLD{oRstu6>bk?>!`G^eMTbu#t+}7CxDR_AG!W@P5|j-blagX0BudOcgHoL%89o7qBigz%0S_;W?Pl7 z@oEvD7Yr~%{>nZ=AAGgZ|Ek=2TnmwkHHB>lli1u+MV5Jn>R;HLfXk?FCm4*+4%N=q zeK9aAY<=QiHsnCX(R4B;9GY{Hbc_13b&bjNWDAVh{JowGuft=1t_*?T0=;t5sp8vU zL*MKPaZ0;O=d0<(7)FdL$f)0=)lq({VeB7%YYrxpqtyLcm}h&-&p(S<+gyS3JJ(CU zi1+~&Py-|9K=XH`){HO@KU15hG~0#e7}l>qH!O0H&x4a8LF?!Y?k(vs*M3!AQ@!5w z+&oc`?)}-V8$;HvzMEhjaGr$~$_O@5dsGaP;r9|hp*^ma;d!?d(u zA%+~a?KA&h7V}Ui*Q>kcFL5@J)e>dO3VVeA1Cu>**_*CN0CU4;RB`re0XZif9$cMhD*U+#1pa1}p@U%67vE4zpUrtpB%{>%QnozA}pnVqdyS*Wyaha*jzSB{8k+c zGBi)+1GHE|f-YAWCI!)dpM9G(3pww5v|~DGB~$ZG%&-MPt&1wJsN(FN8m?f|OaCte z&Gk0xGE2OH(CEV0CYl~2gidZwH@ORUAIP9 zc68!Kj|iIkIQy8%sZ6W8vE)MLOs2@db-9FAM|^qJ~A= zrQw1Zg4pJ&Zg{9WVDd)tG8iG^ah*2mt)Bq{jThGwawaaVU?l9wZ!P~5S}+&FuGE8Z z$=7&#G}ZV~)bE#bpy}e+_Jdf=@F|`I0(tyVHejL2-G^(ab_-QVahSy;E3rI^LJ~q+ zaj1od_LfD!GDe?}U}E32Ur?3e$1654lF$l*)^*%zJY?e61y>}rR(ChJ_a=Ga?Q&ac zp^i6>7Q3}x^EVav8Zjlf7*ZcI<@%oyK1K%l6KSl|vTK5kinury0ntYDwiANA51fQm zi4G|WUIx^!XEUgNlkId~w4FlatEYZ&`I5IlAXBKi@iU~K=5!L_sme~`y)bqg;gt^U zv2wx|V7Z1s+-&-ZovGiUHxZoK2rq&h^gH*8WeyrGl`xj7&IhcCT`GQ1hbkR580~C_ zF#uZ}gsb?iCrO|yg%(5poX&MqyLK zYEXareR!Ca<1VM$j@U(4v7}G8eh8c0f<#@BY@GuC^@`Nm1{k zO)#Z;H6@FNv*WqGh7`t4$4?^v0u2{#1g$P<%`Ysc;%VivI(Ad~H0%Q4aqxA#27l<| zK^fRuFcc}4hOfsQhtfn^B#qOro;C;i{c4$A^(p^@|0nWeeg4tb4UfOP(+(l=@Re7$ zMh$fg-uYV8@r7YAU;|uR-ay5%UU0TD_SOO-C+h^pAJxM`thQnP+bd5PEZ5bSt~=GO zXHi1M3{=LXdKVj<{@NeDs7up9>{+4cE$pl9&(}w+OulHM%9q#rJF(=z`L0y3?VZu< z1G%X0=vRV-u|EchoZb>O-k~*Jbf(P%H(YKX!voTCE|3Hc5PA057wBiZtOy9* zvLE3^cRs4Hcb%G-%P|$ z)XyMWej9_C4iMx6RbRr^Nqe{fMD0i@`m#>Xl`PHuu=EtF;A5K>k@Ixlsglxz> zcs`NB?I#zKFBhezJF;(sDZ;D3S>ZrQL7zAKD~-w+WYUwDn4*P9vr8i5JXO|J5Cw_< zOy%mN!-S`LWXAj_yOZ+*vk;a`{11Eyfq%YQulzL6I3Av|F5zP0HhCmr zI2~z>Z-fVZQ!TKGwY|Bp_!D?}cs=NaPds2y%;bVev{0 ziW-%x>c8Hz3XH`peUlLr&-wVaAUZV`iCulSq(ZjbZqA??6Umq�ox7I30CmCm5nKNA|;Y{49^dZ!-$^NgS)_7D;UHlywH==TP zHr4&$RmH`&pjxIvybeY1g1a(?vf@&9QQGSnIuS`dBD5%zjR`TpEmgI0G|=akcPMzD zOnidbnha96aEeZ|yiktJvELE%R{G3$$maE<-?-;F64Eg;N!JCK}ZodHSb~C z+4RhB+nJq7#Pp6}Mi{-qS&1ZW zTQOmJODiYv&&3Dp!B?bQJ9{t48QniUjoy$av2|xy(k(~`9iu5nEh3tqS6UTmsx=z+hrZA|2+~zZa zyOS8WrUGqf3;bC5B^;*JrZ=HJAJ?=}_brlc?VV?P-p9l1s`o?Aba28HZBb&H_v9&U z9Wv+*m5+d6*J2818Ervp=A+pZ5A6(`m8r)gkvimt?G9=??KG*J^mm_3!FSKlH*bFh z>0kt1Om>B#qk9#jeu*#b-vYzpF`3fdAN>59%Et4~ zdLzdsl&*h9_b8A<3^yJc0N#8a(nBtb$vr<_5Ii&WUWujqQAs`>L;-jc1l_T=Jz3Z$ z1@4<1-l~PH)HgTiT2AZR!ouj|q%h>!eufue5YvUU%{)rOxU|9tu#^Jju-{Zr9bo_g zCt4SEfIrx;%mn|q_+9zw%d}gcc!yH@Xx|g$nj}#x3bkoAKOe7z9G&EV zua@BWjz|CESyhidBP6)jr&RCeTyRcU;1S1Q!>74 zJhp?8b~r+O^Awmp9F!2JKhr}rz_Tf?&HVsdeyx>!1V%>$nK-sz4oe8;+(o-jmp6!N zaf1Q}MUZ&^(fZebO!33+MS5@eLW8i8Sd)@cogK7jI3m-T`>+td3p%`VI_TLtYP<-Z zIG0=YH;D$I4I$44f#&Pbz{i+U9LU#{Uc(wdOC~yW1IRn#R0jv z_01a3c@JSMp8y&$&9n5cHoEmEgC8Q`y63fo!ojj`sfDCGE6@Do*;|ZJKeZ{qZ+Xya z+_h=JaNW%cKAc2U$wifu#wIR)w&J#*Eunc*2`?ahmY2xYuRTYBmcQY_vO)5Mr2EEn z2#^-??Qq#r6F^hxc}*#f#rwl(mU#2h-!55fng!Ubu#`b{?!8@m0?bLXW{$eLP@_S` z&$;FW`&3sgVHP3Hu3i_^ksx>8FLjFU7?wrJU4)!2hvyFIProRmf7jsSQGsSS9L~i1 z_Y3cEi>)yu3e_Lq$lp-&C__=J13S}rvl0{1x7$O1KKKX`6jo<>tFnjS%ge@Y_U^7O zcYLw;9D~-_o?|(u-GoPi+Y#y$S!_8h7D0GXWeb$wAAJaD&npji{&#IWoVhSZa*n{4 zoh~x3p5M5}L+WBv4I&D-+rE%LrtM-SE}dvQUFo&>KhoYasHyJl7q$T^0=J@oQlfGT zA|eV(ClOImX(|dRB`5+SRayw7h^UA(6)++tA|gtc76>KuDuj-N5?bgXA&}nQ{NFj} z%$euRoO#}PzpdFH_F8+dz1OoCKo7I_30+8;R=D)tth7fgtlXeV(dG-BWy*`uiqo2nc(r3Et@yeq%S z_bnsYUjWuFHO0&sD{s@8?&cClV_btEd>;nAX%@lI>bLep1MB@|TLdGR{eXUs&xK zcx2w`mG>|9TqfgJ^laFD-R!j97x(LR3g;wTKD#eVON4 zPew&)CkDs`6xIu_|E}&>?Xt#9s=X9`#Lv7(JWI7Dlg|B#tSxmmR?|tOePF>lV?i;|Z19~1T@D3@3g&v)hd8bckdUg~5nRRzs zY16DC)>?-0M4QJ+RHOIqN}OGt%B!Ps6hBB5!@%yDq3*n^_Weg!?*W}98}ux)$%`s? z8803*yi$i+9#d4-`k_R%<=mGs=CHn^x&w~>d)piHpU}BE1#eh$U3>qlf{S-23jHlC zOV{*?o-yLWvCZAiV}cV~>ZyXBv+_DN{a+bR^@*{1E&EI}`8O^8X1=gdhpb}@>K4+N zAkJAAN4Om-;(HG|6cSQWwu1U>EL;5|an=<|7#KjN`Db;1R?2u8$g-~BCTlmG zg-iy6vrAaFW;&gd*c!q!=_Na+UBb@o4xTF{*xg4t!`P`&;H~!YC4oc^@=vfvOYvEb zg9@3Z#Xl&l?f-Vif?(@*gV!0;M^%MNC(ZnGk19%d^K{%Vo9UhY@KJf!&V~f@PB79= zZ!&1D%#i|oL@mU0sfn1A%!DX83xMi!bI&=+x!Gm6f1iu*f2dI1eP;H+;lR@A_H?-q zlkZcbl`<2m%MfoX$p-TWf3G;)~Ub z#@ViaxjLGKO07*7Faovo7_&!f!GWqq9!F1mxCk!Ara6sj5al98&|6gKF{|+`7sRVi zSH)~IT~=ceK30=H=+5F-7Tg)Y(7*Hls0<#sdl22=5aSoMSCMFd8Xb4bq2H3JI( zdt)i;R&S3u1=>QV_%qQ1AYm=F*-T!1Zi>s0;sx*3v`nuAE4zjMasVCYWmJfxA-gPH zt8e67pT5_}`F17~Ynl}B>lMCX=40SX0`Mn9xYNucFeAKlP&Uzd^|oAuCG4Pzc)Z)A zKnTS>u;oI{)tZe(<1YnyAJus1^Vqzo_obd-!bc%e&iP-?MAgNK^=iwRQ`|)n0qa0r zr;NnR5XNmgp79a%Eb|B|*5c)j2mysCN+cVowMsc##VcOUmr>&Esc)9tt$Ac5>801P zpvzOMsTQu3asAbo5O`W>1ggWO|CJtB#$W1?m~ErB1TLgKE2zTEP!K$R^c4I3KTGkO zfWNaZ%w|CF9s1N%(v2bUKc9?OJ51H9)_J>oau=o|ZTHRNaRM_+SlSJ?v&4&`Zb6KS9L!q$ z>|njP@l4WW+7+mq++K0!jN$fjPR~ixi1tq-)U;c7BU60b@)87g`L9QPGT^_#=;uyF?9!_ zPFx%VE1Lzsp)la|iFts_e8I5U!twefX6M16_l2@nPcO%SI?tBvEE2)5lcBWPiQzF? zS8~vs11IzkWDG~=b=meQ>czx5)V)KFUn`2b-Thn)?I?C|r{UI$5R@rEII-=>*?LO$ zUu+&)9?Q#HmI|@POzHSKu8gJ^Qw~O-H8Ha-)9A1E`oIa%S3bG-UK;BJZb&0zh}Iv# zIvwgas@W^o9{zYuIn1ASh~ErEWjPt0LA4|U5p4RHd%T%?8DvGBaU>G5=?Z^Gh87+F zMSp?HDyd5eJCX6d=;xH$_;_g_`eGDvB^6mv)tRD^O;p9V#JmJ7PLKSM*__3mQ-81-9yqVqpHW)3ZXmD;s$ zAqosFGgY3dv=cZG!}Zk&@{}nYjGIGF9yg^UJ)onAmG00#e?^F+qhU-B2bZzdR1RDM zB2cBifH#P;L>>C1I=a9j*Qo^Y$m=VhYTx)z+X@ui>qQbG{!*?8Rrqp6o^o`)E@V_w zCLX$-R)wH;$zRYboo}|oAR*aK*W zD9Cv92tQtAj{Bw^@yZ#mb&c+46rbagad=nLY{=^NbvNJ2bg_@k*J!uqnlf{))+SiT z^HY3y?&~?GhT??K625BqWhWwgCb{ee6S!%LxiO;8X@m4Njo5=Vt6oh;)Mh@G-{s^@ zxI)Xe$U1E)t)O76u-71P65m%#zCY@MoWG$P`=kS#PN`Ps+=F(%2H;n_F0Pa}AR_s~ ziEbjBT%uF0wc}V(4pOI!C7W5E^xa~|==O?7>ZQ@QHyh-Ih9-lgYXRZ*{}|ODzr0t- zc~dedaW$*7$m($suj5j>3BBLlw(q!?n16D|LfP-4`&R3^b$Rh7Rd?zMZYQ1UVOQPS zBs9#=*be_L$ob8)z5f14VyDiJWR&v>iDY3a`H=4$CcClhY|xx2*pzfBqbC}-vj!EC zvz{Lf_AeU|MZ+p#LjFPORnMDXuNJ8miB{qcfA=STfdjL?j`JNU&~-QdaVeEOz*D~E zi))FjdU%8i^E_srST#};SPWmWIG-(ZeI;-|W_hQeV!-B+cVuZ3Sx`E#==2(SyOw?m z_oho+sO}$+v{p+@*SWNSp)H?*M4c84;RGW|E-u~Tc{=?sAPv=Lh>@rPB3z!a2mG=J z!mk^UbGquXJ<(~4+s{wm>0UWC6c<}{jwJlzBjIYJU%H`wDgYL3Q(eFTDkR*44|%$eRD4Oa5X%|pB9#BxU(jA##YsXyprakCAZ6vH zw1TzRr7X$1#xBu%^bmpRJo$*Sxo@+)ls_nN555WFw92|ixrm1}!<&1;+~*$#T9j=0 z)i!Vh-1i{PUadxpHU~dD2UzKt;H3=rQ}&XJlHUiSFpxa%GBMM%C6fFDge|@h)%2O` zP6jt%QS4-GoG%B7hHkcfJ;K8gPsF_HHGaC>PZc`d?$q}HURErrU-?n$uzfTw7fk0j zraDMsZ>1e~F?D;b9@p<2qY`TMpw5<9)@x%dCP-yB{k(H$iB?d2%=?LybAB=fEhUW- zO;@fG{0~!1y2OsyU(|D4F7aB~PDk=vQNn*!cC;$oPRndoM*xV+fC8Zs6Gm zCC#{6^;W|G%<-0wBv2w&FrP`(y@ zH)a~CUB;DvvWwNcxC4AH4*$1GU8+pe;k6mo%YFCut;L1UJDeN5cSlY=xx`8|hTfj} zG-(E24PWT;dhGR&UyLXO1vwm#y2$r;?fCG4b3nQuPsJ!h{6wbZQOU0~+1;O?{D;ph z5GfxYarb>soUFwT2RCM0Tw{x@b?qzcBB^T#31w%wwE^8k_`!?j?c2_eW8a$>=`4Rv zws}{RHr)H9M$9{K@~9wT_v|im+Gqb>^u>4w9c8~4lBPMTJDCkNgeUfNFX8;im?hJd zKd;1gg`=d$bFAQ=8IovZJ)?aYRFN~8(lI@ITwWt5!0)8QbU3BHvE{`P`?nQ;%>ro) z;jTe7q?cYZi|BySYrDMFH{+tloh=I{e?L}H9wrtGmy+};u z4{vF|fp_5i1$R`4{f`G#Gf0thMm_sYO*|e_eF?F0`&vF=KYH5CaLxPOA4oBR1m~w=qV0#PeWK&TG`S1A5QRlZYvQ0UsHocMeKR<&&7H4yw`E)ruQU1(r1$=? zQuew1`^-Edt)DF-15bkL0%urwL-%fr`W{NlL$1#ADfoUgZ#TL2&M)lb!|QjbbS0 z`J zbmsCIllvE{HjhPI3D{+$V8wZH=y<$=UWKanwdr>;`LITfj|Y4MzqH%Xf~s!v4G$K% z*jLW&Tl)U|dcN+F*R>3+Zs9lZX>&9Er~g>5{406p;~ijb@8-*5 z-81qBx+KFcHKup>1XY}v2;EX{5gFh~pOrjY=Gecd;kE8X)iY9kBx`{=X1SCBhW7=K zDK|@2N7VL%BWG^`EL#Esl33?C_zybYmY2qa9?`qgrl;A%8_b5N<%uw+!?t-1lV53uYRFyVvej&CKq#2sRgaD=kJef8}EpNuGRL6E3XfM>&l*al=91 z_%(m3$RQo^V_xcp*pczn))!8O*})TChYW5x6A?`QPML|Z4`Q5uObB!Nr6K143tLHD zF+}ZWX9qN29-6sgv{8+gL#*=@aL|v29P6FBa8P=X~j9uVaguc*vM0Zv;J+jeT zRn=VpHe6-dos_9&0vpHs>mSpaBVDc;*2CyiNe5?q$6MTyqS(m^I2!Qri|t_44?xJnW(y5=hWu#!wai%u*Y&Tulx?~XL*iKgCMO0 zIf#0(lq^30;y;?(Q7?HqUne^e!>XpHUyxF{1Z+}Y$i2BwSlV(%Xs-^K);anZW;;E1 zLQ}4&F;YMpJE=YWG}UZu3Klu;?`8b>OZBS!P7{%rlD(4Ij_cHq9ljSOP5&ap>#K5C zcbQ~s(byfQ-lGFzeC;-t1CVbc;JbtUrPeYte%tLnBzZ5?4>_+t`ZSJ+> zP5AbvD($7bXRKxzWTilawLiV9$iL3dvBg3!$NGMUlAkx{n2Id0l+gA8IJ4xQN1gf+ zP&v4rq9Y=?Tv0D7!*(^Ntc@VM$JB@usz8mY2)cG$2*s&^|Disa-S1xsW%&fHH#H2p zhRfcQ0MH$$y3M<-c>BVxqtpO1e?sYW{nvm4wt0zB^7KJ5GTp2>0c;{FAn4_K;dYX! z?l${jbJql|tvJq#oL*0rFR!>I|}MTuMB>WSbB{< zEU=rVrg!Ux?ZQ3C9ZJw6R)tW??-??_5AD|X#y9z--Wf;Pb&GmGoLI)+~vPLwvE&eqG0kqxDPDV(tCQJtRf$8R(&;xqTH6;keL0 z@nd4ur(@zr8n%<5m^FDqh5Hx8HS?p@`4p-Frze%Us92Wu-6!aOfsozegs} zBs2mll$DG>ld#EcDv%U(ImrvZo3sOxfOM-ZW8Rkf+;dXx*?))hnq)=YUpp&TP~pfg zC5U@+T6G`(th?_m)3+`EWz;78C#TqU-9ESQ-By+Gj8b_2+@0Eay-p^ZT1zs#$7D41tII{!jsnTM7!8uI5{uiZY8v3KL6w6LgboYndL zH{w)IW4={WOCro&=sAN4%a5shp=u@p|FlrD;iO0O>|ohu6`wOWj)_;~iQbX?HE4Ic zY?wy*A!opnfvgO5tGH_M;fdfZc86tD$M<~(m(?qkQ_&AMv+%2)(K9#0_B(TbxEhEV zo1E@2=0Wph26XGEj0Voyb_YO6|6wdE$UOM`py%_$_}~Pw1daHy#sXnb-Tk$q3dY|d z{B`aCa|w);rFpe>mIx;oYK&MZS;fzey+Er3iW*u4JTT=ZdZ@;Bq`Pa{9 zywv0)8v5qx@06qBx3)j%P_x7Q%1BigkZNzQ8$PX=8@72NMT}@+Hnk63H_n{_8_V%= zl4lRoEEFqENg@TcZ)mnz=x*zNLx zpA2P;>JT@ZE|faC2^yxV3Cm$6EmVN4BvX|=J!XULOI%*e#WtLH~AhqEE-MO zm$a|BOt$;T*yKO+KK#4v%2j%A)if|ntjy;fB1%4d48*}yCSgQ)ujQPUL?Dn;lqxQ5 zu-&Pkxr{rj&b8-nle+TI;|tdG&b*SQl!{Wiw44T0GX@a+c=#ZH@*Al`oqOXRA_B^1Giz*ZZVgfA%$? z`#p?Jg8D#IyZddbz7`W=Cz%zWWp3 zk2(Hx@$(R9V`z@s;>m>1vWa!VF1fFrsM`` z`f~mtD+N{s;Xb;A5q#P^%(mE4HwI+mJ2!zOtkaw~A4=R2$gX%rn%!{-+>ux~HWk@? z0|D26epgEEX<_Moe+CM1ZR{!G6NHkWG_pS5YB%@_X|g6s1aem=W+bBLoXtM4%q3{U z$h*a(h!W_T`foX-J(yPb;4vjlO)hrv7fo~G*Zzt^s7Ls-G`mmj?r zt9zGahYBgPoH|EY?XrB>4JFojfg5oZw{}-jR+)OoehJ4oL(BQ0-!n4g3fOe?3%!E1 zW%RlPd0_gXMQFt-foz^q=kn;0z9$d^|C<5ujdiC_A3X5Arf#3XLZp^JQ^x*i!svkM zZr9;rzdJ`429w8hZFAX1Kik|{6D~k`mG}BXRCEtgBWE++WY-bv?y|gLb(t8mOHVbC z-AiT0X1bqe9R6mz0r^5m4?w3}lOJPo*P%L3#cJFYH&V1O`G>9+`vi0+cqX<^b9en1 zvl|fd)@Y@7Si|smGB`r>K!|JQodV?#VaF5}FbTy)o_>-U>_g&2>k7mLPhU1`tE2@w zH_iEzJ*NZQ?UQKRxT1V^CS**S>M=Dg4hi%yp7iV00Q2*-`fog>Mwom4OR>cNw)}BQ zWn0zw6xjt@k2F~Ad0{$nzM$4^2b4dLq*konHMphNSsw7fc5y4qdt-1`&Cgd@Tjyik zky19C#;#i$dHBhT{ZM$*pzHuC!afE&#ZNBv{DM7HHA?%=2F=n-bPgrnLRU)dv$QT? z{_?@lJD{&T%1WhQtYVd&0Skz9BB>Vkg_}KArgz7<; z%5?&Uwt(Dd%F!|^t|u=U!&^Jtax3=Pc|r5lpJ%}rOrL)`-{fa82IK33h5y!n^Q%ZR zdG{&b*-x?HV@+!N8QeW=@pdL`Nw`g)k==GC`H#r9JK9T*FIILh-RfADJG*Ruv`c>j zCN9FpBMx5ufuLjai}S0}kjdIZ8!feLQDP|R;(3~%hAzpLmc%hlHj4$&bK~z^cjAXC zS;~Lk*gR0R-wyr>TytcRBLhac%g?l8QZCIT6Ki+u3j0;TQ_;z}kUexT0Ph;1JoeIV z{68c!#swG)@MoV>bHN|qIM{Tu$^sn_F{6Keadh;oD#g3p$+|~##m;=b1pn=8yCi&eESj_&Jl$e#K$VsbWtXnqA3l_LI;z<=yy^MQX8D@mXze zT*DaBWL{N1;*Pym2|*+w18RWr*bWyE1rWS%N1cQ5VDZhTS0jga3uokS?|8E=fhSBX z+7=O;PG~C(LRQczQkC%$6^Bh;{H|0UtQdE3#V=dbHs-AJ-XPX+NAy>3qa^*_@O6IM zsCWUM@M8-&S=IR@k9xNCbZg$lP;7aTEW(?k*HaYekP$9kYx4BcEW`p?*-*7v!?4f# zB{{5vtIJA_`mtG4VTBid6q0RT3dLlp$ONB#WjpNJZcbE-Dec46QrNDW&g{2vI>BjC zNhRJeE@N^xN^>?)!f9GI;HC`6QhBh;(C^Ug*{Aus`1@5LGe6w{?=;t6#G5hLVj~1er&JQMGwRRQ$Kk;5qZd`KA-H$ zvp8RsnOUyAuqB228uKubqnQ*TVW(Mi4|acC3p{sdIyDzA2v+wQ7Ry_`HQB9hJ9o$F z*~BxhNl%WI!B4*;oy!qbN6)PyFNHfBf7FLr0zXn& z2Y?<@w?Mm%)gefeap%%^mW<=a<;Oh@`u*uxm_wE(y$ad1kzTRiUC|Yutd46br`g4L zOdJZhIUk$enjNWW%9c^m&^!aX->zOPDjv*#j)ZTb1NLl;Xn^O;UD(LE1JbgnG(0FU zV>=~$V|fE9+<*DdVYd^h6+b1dzlsj)#PIW*(i?=nGqpbKOTEHCRK*GKnlx-Y=4M{t zZh!W+yB6k0B?ZKuG4Amp2gCEHM|BztfT+5CxS-?!p1MnYNHxs|X`MOS84W)1^=ra; zgF>r!D^K@)Ob{VERH6f$f5-Cj>yjG< zhCD_Rripw~b-B!9!uff~ou|-VgmPw+p*vf*8`mdDp9shtVG4x-;}E}rSDBSiPbJJC zr4A@Owympu{!a;VQIKhu!5d|v@3@9s?-A-Z^mZbJY?i1CA+BCvNE%Dq!b^hx81uu# zl5Q$4jRNT~ms5`qoeIy|?Qu&3-#~r#ApEzh$3Ppmm65bjd<{!KHx;q185c@%l3N+o ztfKSh)&kRW$$>dR@yoF5Gwa?b_gky^vQ zgVjcJYkei5SISfT!X~xs8D+Zba!*JFwajagf3t`8_XtRzWo>0@qs@t87$`*vuiRu{ zyiydNjHAOBy{PwuyzMf&NK8%opuKG$?7;&U@;TX;-5y{zNF6=c_s&Q$uIvjPgQOfT z9{2XLfZj^`S!|eJV(MyiCNgYr0(D8BQf`MYL3v#l2P~T7hLqcTWb@d=Ur@f%%>=?vbSg@m^=A!Is*uC@#=oBM66a+^}}qboQU~oYm5a1 z&v{!5UKg~c*D}$M4?VY;?c=$5*OqBFE{|jUAcUN=!$lD2N+m{BYtLJVy@s-PI9CJ2|}+QViaH`l|l%Da|G!so3$kD(g#89i&bz%@STBfVAm&A8zf>NHq>Rf#bPcGTwxgzu4bUX9y2q1j~!Q zvH+VkA2d#bSSGN~@Or4uWRQA!z$98;GIsXKB8QOITY}S7{HcolGPNz_vI1;nbQ|#g z#>U-DB#!@|3iKC&6KV5oR&5o|^E;EVO*rVHKjd}C%g(DGL-xz+oA$~^kL*^HKAybr z`;1#qtlO@>6p@=s-2uZjo{oq>Pg)pylQI;OAv^Z9$qRMD%cUO%i0Bg|rT?Dq{#T?% z(O~e_%N%rZV*GX1%4u*UtO^+*2`st8?f$^nh(V`_;KnX}25WUTC8848@Z@VLWv>3p z`2H8Ltcgeq^0Oa!OQ8i4pTkQ-alASNy|i-S)|BaxQ4NMdNh2;b@Dr`dNl(1S!ZmLg zRU1TOzr~GvyL&-B{R1MQx=b>qaGEDdjWSsNLV@NCPS$n#n2YJ;v z?Zw`Y61g}RpQxU#`y7ugBzaz>{j(eNv|69uae@Sh(o7drXvhPQptqt4(` z-r(Z6jm1rBafur%8TNSA(`_PeAwE^$+#?yc||J+G%E z$pn_&IMyU+O)QB9af3kt?0n|>+H|?W9VO3WUd-Ce3Kva1ip!zm3cy9pO7LcNx1P2xR>o#W;i;~j68n= zJpL({GDWdfXSQ&NQk}>>7q;C>u~)3XKR_nCrYh&%hlDy+8!hmw4D_*{F~*Slpzw26 z-rorF^9b;SmPc>B!unVQh^}?)T@?Oq`Y=i^Pf?oX3aGa|rXC zOG_p0v~qHr!v-KCU7J@|fUe)tf8(;3TK-x(238rkHZoP~T3xC1Uel?;Wb?>7beYpC zg}hORotUC0zd?8g)i?%H9?o~nBH*hT+pBC5ZjOQ?^I}gSr#Je>iTj83T|>-f8s~2p zf~>uElfPa&`g91{{rY7O1)lIgzgmD09~E}YZAXZR;G0D4}>{ zIR3jZiMNN}v@|XH@aoD62MU{5-z*yBHM@)s_}~dz;ig>llXakFnGB|9WbWKV(xz$U zhR8cXJ)*B60=~i+w++E>9*F>d zK15F(xGW>n-dYYEQu(1hWTw5~a?xN77sIsR{DhB;$Kf0(I5*&0$0*i|+ZyLSdJRdLNN?VN6DhkPXvPYu!Z+g@pBX-0Y;(PMmXd-_5s9QzE2SD==I^0B@p;P&}=AIsPh51iwn*XX8*n^ zH065d6*Sp#g$Tnu41fOFk~x9^7P#@Sg@^>>k`31g(zH+3SiFV=L3aA)wwzaaX58#r zJEiOc?z#}%UchhgOEeJ2OCQo>I{>2b(q9tUEXS!zK1FiUgC9ID2zHxnXnwf*U9j~< z0MXlUnGk3RwCg>yHd%kTLKDH*<|Hq0fG{)6h6ok7ZUXF$zFAH=yuXE;T~qQK%HM$C z>T1cBSZ<`&$`Tk`?tD001m8LvL-dc0jQ2pi>_cDLVf;PVKu|0XexM?;EZwVo*V%^` zZjGC!w}mhTQSP(-9Oy>XIZsi2F8b9zv)!F#D_mcsxqodb~{Zt zmEahLaYA>A)tJ>kX$kc)6O1`}8snr$tpJFOAx!3rwA6BObQ$_k>s%~$kT|m z5$N*?qw<^NjzJ=wH;xXa zej8{TjK*h{OQL&2d(d4GxizaBq{cx}pZytWzii|nOjl7TSTSW`{t!a%gAvGu{fF7sA=I184p zkJ;r<8Rq|q^^pa&h3Fv?0tD~H*3s5$2&`be>H09R4)Y8{-wO#4Fg8wcYbPKON|`{zdXH?C>Z!CTkF)Ue(pfHbSLR6= z4m?C2WetTuwCjVsj@Bj+vn1yI$bVU~utW2h^7@H`uUVl%>dy^hX#_;!jn-eU+w)~93AqhuD*{Dvh^gMP_U!1J{hDd5t4OkaR> zoxH)>yx+x!^s@JWFI!D^0H0UD(dxoKQAniA5Zc7?++pX7$CV3^Bu2dp-f5b{m~ujM z8K-bchy=9i!I_>pk^t{}j-9(ck}RDdy+(DL4eJZmVmeOSI!}S#)*h?-!K(z*@V}^^ z{hX$BN>vi#)TabXG>WT^bPFzuRcn)xv4#x$%>OF0;*f&}e7x|y_#?2Di6uD9=jGtI@%wR{8gKkX}J%-WhpGB+&qhofzEN}6TQS-w0U^dQejb?Je@td<- z+Qf&P!CF@sbV&^EaZWBnc`w(ywY()4IzE$)h4ams+4N2Xwi^)H zUz%jienB3{Xd5%K@jrv?3U}f%VZ7RW;8?OS!cn+gws0me9a+U*@`XJ$g**ct5DwEt zf7cc$RnQN6CBWiOOsl7qbd-B(A%TL?tTl#Vii{9ThSXnbtsEoTV(a_($WFX`DtJhU zAT8!j7_a?A)jPx4jE&Ktn~f!hCtXJmyM+0go*){vcv!FHxze`3t*;+i8vnA$;><^E zF{9aa>4^mPD}6LbBb0(oQdtQ(;5sBo zo<{+Du0(rX9*OmL!{-JEm-YBq)!v_jM5 zMs2A7NtSrtUtYuhmkYSex5e!?^}?NpjLMJ{QPNHAXSfF2lT45czCSbyp$p|1$ z)ng{6f(urs{g2jB-{_+?@H?QHUbFPoQ-0NF1(#qK{ao_$Y>U6RZcWniI+{c0&Jmb5 zkl=o#9MSnjGO_p=bD(WWZPp_^Ch}Cc@48b$S`3T$eF{Yiy^+_GQoFO-1f}dw;pVZu zuF&$deBk+Zz?EC*TS@pY%v>6(68ahps!jQG)Ik%$Cf%#4X+AZ?r{*R|DR za>PsR9?*Mi({Qagd3ui0)lN)i2y@-@Y8HXc9i;m-)adL>V4CzM$({Sl7y_QEKf_=3 zhj9Dq%MqCMawWYC1XwG7w{g#VOrKE~ zu7Dvz?|l~93vNJAYW~D64{AZru~@_^WBg)|9JpxIyGUKWF14<$M4f8g{&>xU3_8_* zE_&-b34DDls%Na6h_VN||8$h;!>)aosS-APHn`2`XI=2MESYPygm@sbzn10V$l0-s zE1kBQV3!-Z3!<<>mU|PIIxa)NCpZ=o{ZwX?&au>EJ9lZrDrBB-dpPnK$Ea zfJbEb`c>nkmN<{mNv<`>w;^M*+b+8EPwbD<%lD zk*R!RJ(*eEvo5O~rjCJeEvSE^ZTLmT%)ZUJ$|ebf=|bF`ce8nQInP@GHw1;?Y*TQ1 zMdm0&pBfBY^#G=RK%)u=qgSIYVv z1nbRkch>w-TsqekMoKNOz9g7r^4De7w&yqLKwQSdFu2g8Tv9B6^&?=qeyl+nnp%bd zGOMO0rUjEUoxOa~zuNF~ydy40W7kFjv0D%4dx5&%ahYEy%DJH#0%75r(3f#yt36w& zKcxO_te~A3x`OTvt2s&xs0u(#a`1AyCkz6sglRtC;9GaNKaH6i=OS|_Ae0PBk?*qs zpcI?wxe41v6JCORShT+WqKWl9 zre~)EAJf*_+t#|+qfY8EkIc2}MyK?pPpXfDvwm!HZ*m@WXRW<8_eq*ys_=r$*78Ob zgUK%`Mj($V--jU3G}?cvp?s#|Q(frNXi&3`_D_|$0aVWj5ew&DfzASzsecW^Cj)F_ z1o|cO<-@|Zbt7>X-UC#<>%==3-^}?~7)gHI&pDly5FFE9?aq6b&iF!;6&4nCMzYcj zFccFLxn8VAX2T7kg_aP@+nREb-@kza_l|Aa>#u1d8M6(~d9$p@6g`oZsZg|2t=qmI z28;$yfm9L@Twc(#aXQZd86?=I2f&7UT~42OV~4)PG>~hO#f{!=n;cUIPRs)0{P;^NKFLLB6MnE$nADI;CrDIBwn={i36W|hg!i5$`AO` z9k%F;7>gy$WBEy59eZY2;q!Al&OJijPVG!}*s*OwNVpz%CO*e}Wh{8YCY?xK)Tc~u z_v&v;ilORnQzOD?o)hJU2EOE~%8CEcMO)*4X;4Et8cl|)eItA(@@oAsjB$cjT$Gl0 zmBQc0ZvUkb9`U$QTc33;J^VIpOqy(Z8R||_y0b`|ItI~fA@+* z!cK~iNNv|APxGuNv=94&*ya57f&Vh#|Mu zO#ka9|68L@#eHU@-T^{nz0u!rJ`-M&em;@awVL1 zVPf=Nk(LjnT(eGg`x9$w8)>r|XqcKS$*jVM z!)iOvVG`QTD)Z%?Hbxc@8VAW=qeDL)s%n^j3A= zY8k{OtB^X%)0Fup<7KUA1ME{F4cyZUwaR1Fvj%%CCtfKMFI|+X4OnFj zt{grf!^c`peV1_<2SR1_GoCi1NvodqA@mg*O%v1SVLiDzYmtoaPJWPK1tSeuJFjS{ zW(Mq6+xo$B`Gi*!niV>aLiD4Mb^`5B(RTVe-S`lXue3Rn3^xJa1|5RU|sFG@{ ztlsxU#oA*iv70s`!g?LC(HYoiLG!fJ+k35c5t9Zb@OA6BuUD*qBh7+N*Zh)-A1Cy6eoA99EFNAHeg%vxt{z73np z6V;tFFMc6N0gPS@w8>@!eftHhZP{s=oP4&F=0+!LrKsbKfHz>j2k7uv#@ku_GMJR~9&&0>F$w)<3}Uh{O+qr7RtEYhBxy^1=w`Xl3sL`#ow z09DcuRH|b#mpQ#Drk)jIGW;6!{D<<0#EA7It3y`-Q5wX`olo9a{V3&?pbpshE)_tK zg@>CpQQ2~LaONSHz z0geAYpKXmYOeHi^;tXz@fnX}2S#eeh;TRtKq0(#UPTHqExNWZO~ zA_g#wk+Dd|Eye!Wh0m&VvP`B05bB^7jqS)0 z#n9Yg9a@b62H6ISsZfX}p*1n`$kShs>>qlSN5rowLCvznja_{8(>qQ^Hb-NPUVjk62SH}-}>jQkt%r4 zkDY=`Q|@dnWzu<^Ie)WV&^d(N8wFfLmtp#8>v194DN&%rYspW+2WgGL5`I)AT(qmg zU;r;~z?()?=RzDyf}hqV4kG+E<9(^xA$X`$~T^w0yi+s2CF% zBfLX`_~w(W8sTsSfb&Kn@rQ8LrRiX*8+k*lwFst~7SAGGwwD9xhI}@+UOcu~7LcVj z2K!6M5~x<#%!*`ng^j!gydqs{p(fvwfd4-(z|?+6dE@eDoXPz`f2;|%kT$(c%oKE* zsA};*{0;dk6a*g##r0Mn7NP;K0N}#u@G|WG5-R<5jw@*CvPTMs%6%(=9fxDvB3k@0 zD{PmgoR_wH8h>%1S&dq?YHQYN@C~N%JoKpq(La49C7k>zbD<+PaOnpOZS^z^DmOB0 zLu6gP*GnSt*pxMh1?=z@|FPW1fQ( z3TUU|lBa(>F1naMS{>0Oqwn!j0E7x{k9EE7R^k3_R+dZrH03XP*bj zQ`qYvV<~_ETG8g`b1K@tV#95Y;va5wh={AmSmAW9>KvM|_~ZXroiD!(4`{7_gS zB=nni?PFl3hInD^Wv!`;SZZST1q(19M*56tUdqzf9h#@ zmWO-zp^fmS=>E#{dR@bT2aA+fH>^(7(I~y`MIJp5HM8`57cQelu4JrigpL?AtQ2EJ zzj**LjH|4rN9@r@=^URQ#c&!%yn<$eMs`2fp9n21r1|$*sI7l?peEf5X{1||DY+Q| zPLIVvQppMeE5K@;2SD)nzQvw7au}LbiAr_$V=aE0m`+@S1yl9QavQGO9r%r`$!q7k zJBEf`!}v?)SBY#hCcfF$U9x7RVV@lcA@)iT*7mf8e*enk3=~CAzLhTZJxeUxfsedjZYXSY1Xsat6asRX_%07Y78tXd8dOoM9&S-HT=w z{c1ykw+u|e`;C`--cRzkf=z*E@pYje;*F{QkRDOMQELi<)A_nnpvJ& zy_~TQFve<{AlWQn3N0^;Z;xs6H#-pHwe<^L~M5LCa=Ha>@p>n}Xkq`K(x#+{u8 zE7KEK|Gpo{8K^Jv7}-^lCs=N^p$7}r^?qFgj7wB`lnSN0H#1IJgpzw&;eTenCOndz zO<$X#Q3tzNR> z+Rm_C&pRcTr<3pHDZs)52Yub9%Swyg`*L8lnqByA_r#kio$@Uc?)yEhjQIVMIGD4g zgcMulB*dO1y}f+X^UGc#RD&#wq7hz9KQoZZPyPs`mZ-=ki$4aluUVz`tyf}Ev^Y#fqv zHx!2K=}?kO_O6IWCIE>(5NG;uIR4dIziFv?cNF zj^gWSqCUBnPkD_V%6XxT>*4R`4Bu&oe z0qHB&7+dKRLA6+`QKpTxD$5#3jjSXmBNx#aO(z&<@gulrx!n+E-@Gscnz|TrF_Thl zsC^jHmUA+KhkpQE3IJmhwTl+_t^D1AYk5}{`BP?skz|+P9tBLW^JXZlg3>Rn0ByJVr93^zGZqJ(K zo}RKBx%d=qj(>>lI%O52J0$DLW_`BWO>h1YM+9p#)40+IE0$i(BazvR0qjOA_fM7Po(RjXY8bn3(4V{Tk2fuY%o5Ja_YO|hYVILr$y5(_ zgY@W+kPnk~mmgVIt7H1CgC2!q`j%ZR*;55h&M}Np-;Hwvqe8gsp3b!|05dyk2WXAcd>KjXa7kUwP-Q zk<(}b>r*m@f$Il%IIaOJ$@c6EWlWn1$OczIz&s}?>@ZOUucew;!P}LPV^(0_P3cz3 zifQ@53z`xrslZ=tJVck9jN%NYLKM;bR@bsW@B=5fBeLaN*~}iYghN29e_9AR9Uz>U z!FQu~V8|v-pe(4^@D=F(!sj@bz{rj9=EJsB{{EtcEJ^&aVuIJk#Vj$W5Cw0IY_U#- z?ay@<-G27jz!B&1!OG=hi{`Ljog$BgLG#1QgpIv7(VCzJqM z_Q|*|5rM*`DLn4~han|`&Jb#zXgpJW&t1~_pr;sRM^W1b?o>mqd8>bAEw;Dkt^K8| z&8;aIj47l&BRN~bi_w^7vJH5aaTUV*(~CPEqLJ-7&P=w*a^dKtFLoB{7$TRt<+;mH zuuqTlL9Ta8gr;si^_nblWfR`)f~ZajPjODJG_mE$2wV{0G<`x-Ww5cL04kV*5Llhu$iCXL7!UZnPOUZp6_VjnC z=AJ>#G|v6OMg%!F37A=g_UC-CMK~J<955b9SlrOZJ)}J2FSlM=DKNUy`Rt*Nv!eY2 z=mqjFw`|wJtGcf%4yy4yK@M6?lKQi@yrt#V4$^J7_QUKCo!X9;S3FycdvwW`huzEbX>s}p=z-heuJ zcE01Pxqz}y*63ZzE2$P8>G-I!(^dpdr#VGi@l>?~I_heRWI9kiY21f zD!I8j#DPGsu0i*qJxvL!Vq5bSUKp@0qR@J<%^oN-t#&nUy02nN$qC7Ou5fuO8?H^i-w zLoOqfCv?4koO36TVAa_B(NYl9a=@NDJjb9c!as`B2%nMNV$`T}26qC=1;+hP)qqr? zop=vhkNF8co_bHnrNttPySSjUroo_E>@=c^pkjO)3%+SftUWjip&)`#ByWBxJ;ZzI zki2r%8|t4=zCj=xP))63x~mJR`JrqN%cnYEe4NXT7KNL5x)@HRmcLiA-JW7q#E!+%*7m|UrmG`tt<-xC zF|f;SX+rTY!$RfqI_I#gW)EoxVZpWSK?}h2miAp?zeyLw-r5$HacKE)tYOe;K;@@P zS>7ot`x_vkDpbE?k9%L7ccr4abB-d1$q!0YXZ2USYRUdZ`rxfFw;UToulDZ;hlHBj zK2Vg`_OvE-2xsmodW%vsxwPJnIDj4P(XYS=?c-)Ok^~JGs0){dM6XI}@Pw4oW^n$g3=Zf*;5|lj73G zx{Z+b1v_cTsq=!HNz)K1c7gspIk9_sexnTy?ee z2h?WYg-L%;dbRWs-RFNu#R}-5q(Rk|zFO#NyS0~EAm{1`Amp&{@R0B7CX96t0I<+> zIuy=2%va@P!8wYdW|}`MekQ9q~G}@S^nwy_h?Cb>7m;`FVXxlqBw@d%vVZXn1}K;h>+$=x@@( zlKYd^Bu`(#9Bn0XrEt#sbW|&7mY0b+pa4`Cq;*#ygk0Fk19Kb$QS5T=Sd8ukU6#&SZWu}c+(Pz1 zd2YQ&vy2*(*ptADgC{u7#1?-BC1|7%e5fjDZ_do$1t4+2%7gITgGdRMC2AT<z5fCDnJK*O4n-4mm9A+=nX!OI)`ysD!TfMlK`^BjK?p^SL z5X7~DY6NDqOQ7maM9Rf}fdc-l#L-~9%{dS|ALsG);02Hp-UQ^dgP-FM+D_RUCil5H z(k5bM z20aL}yC#U3E9V7ZeCE%!+0HZ#2pc7V-Ec2`<#h;t|Qxnf+$>l zhY;kDhPclQf~*u@$vN%6$|9^z}X1&0$PsUvnT(qE(ADF}rg}7$I|!r0QmGM;sp2I5mn1=r9`)1wR7Uw{cOw!2h8WyswxBh+0T@bb zoHF^m!Fgo|(MIZ%-NDE2ssINRAeMCm=F3C#X)vAcn3y2M)A{JVI>f3sk=KD?B%`r! zb_#h6OvQmc?w~5Q{*C&|{apfJ{%fJ8f+^(2_)<8!@HJ_70@bdSO~xzJIjrr@9bEw^a+@+1}Nh-H9TDK$|^qn~YT@IDeJMnGwlqsjguPBZq`{kzIpaYrTrG7t4$( z#=&(QSwknubR0Z)SejTv`6`}p37eL`5L6PkWspXj(bM;qrm?Tq;UoIgaK=$ znF}HXn2C}yqG*pH)!ICpvv~(;vyK5%7*YJYd-LbLXH_AZa#E3f&f*}sgFW1M2Zk#R zc9tq!wR>*=oU4JkAeee;FrTo^_*tkarVD#m7!S@ewONy%(KU4!1tTQ08pfV&#;G|U z8Zn<$>F_lYUi)^p3ijw()%fImGiWdE8r38_8Yuzq2U zJYVl&0deMbgIeMeAp2{AHnoZRD<9-qr)@A?cNEY2y01y1&NYrtuFeVkPI_?9;VvxG z*=M+ZPC$lM9X*BlHsItml+^*XS@m&IVV?jp82(6e6qHj8sH0zVUJ`?PPTM$c_+Rlf zcv~P0psoL>!mAVP5JBJ^!st=lbTp_>u+$x57Q9i2OF4W}`k@ZYQ8eh58eY780HH@Z zXM{ibN*EE}?%JuM<1Q1TR;pD3n!Kz+NegYYC2Sb8`4HY(xi#z8CA8><4Xpp>Q#Yny z6D7bzd9Sr|PK8B|FmB7w)eqZz(dRvzsm3lTl4-#oZ|+m4k0eYUW4o4>A5Q4de@z#I zd$kS&4tFkU{@?@&z#^c!2JJjzAiQqOz}jzi^kKJri5S-fB;~L;^Tf3SmCY6wZ&1L* zNoJwCce(a)yEZUgp;IxCss7rf!rGTr{$PIkp_{?fHPk90e{1pcFb5yuIoQ3#foYJdo0Erx(WVjgQv$jHw754#rPJt zFEx-O;6qq30J34%=q8;%1ayEC;51{1Ktv)GA*-TSX*r+$PNsQ`?6<`+$M7>?d6HR3 z;u@b%Z+MA8>?uq~ds+^3Jh4b^F|Z@uV)hu&@|owt&LGz_^|-6fA$%pVzNAiq6>kO= zG!W+j?uwgR-gL}CUKBHQ%+&rRnXs?w17|A34tNvgJnuE_9s}3r$V;Rk)VK>M#7}V^ zM-Srd_{nTafO)ouXL((loCt2Bg}3Op#J||+{9tSoU+Z$2F2JtXqF+~&#mn}hNN)Nk z6>(7Z1}knm!Fz*nG(ORL(y{*oe^noQ8&p;uJ9(unMN7_gjluBa^=zl$Z&=YWSPa+y zBFwfm=*4+}`(3q6;M+mmxa)KlZ(0_r8{8oe*)RKnjFR&VOcfTiTM4?OfKzZi-*19Z z3|~F&I+E%Hfdd+*pV+kc9w%}YJv}F$-9qh|>wuqj3-7YrMGyxqH|k@JkGQ`*CJdm~ zR_wRo|5@&=l*~~OO{zAmFJDx9`t!!GI3wx5&KiJe%8Y&d_>hls^P=wa;N5+8cAUPm z-@=vO>jNhC4wR+yo{}Q#|NUM0+`?ERqV=yEA!6>egEibJC_oc(&pA{3MU7MJu;bjZkFOu zk+vyKAA)<=k!jZrrJjY!j(0b!xetQekhMR@Bq;jS6HIgemw(7+32?jAd679`1SRG= z_J?voALBloxYQcB1FzJC<|YK>2IbP`Gcb`XzhZbc4UBp zxhJW};IlMo-7MxX_@dfgud9cw=s7$5_~rgc!dv7F$tiGiBDgXV!S(cgKT+L!Nh`A} z;qW^V-M@;ruYab0)s_dA3SC>-x@a`hYAkXs=xQEFbDfpyjo54xmgNyFzsUHjq+ZdI z!#Dk-#`8&b(_LxGSPr{Z{(n1p)5C`MGOQ}Rxju^)cAI)#w?U&F(K?e6=dvdSE&{G; zs&7%n9l?lk0zl>ThOkQh*UP32Lf6sxlyMkEZEvjb8 z4?{oy`(+2&za%!FlT|Kk>WNLS8w=K3;=RglB>kxRC_!KS{1PZ8$BE{e8CtVX<%u?;Y4pvVuljS7t-IXgJtWI?gZCBgwR&^^ zISQYl)QHF~l=3pme5QH|mcvV7)h!$p7s@BI=Ragl>+1dfUdkT&>U{9&rpg6(;Ooip zG`Vub!;)WjNq-n*JGC)$l`vz8{?Q_&!wyrkOFMD_zcvS6v0y86R2NZcRoCg4c4$c|UWcowd|Q1L4=7GyX=vI$l( z!7If++vH3%_w6CRvG{~~w>!;Wb}FXsNFS1zoW+8U#3dmrRlB?I&|KRf)q7OBzm#0Q%&kTy3)+ftjN`iGVa&hNW1mPCOh}%+EeeV%wCQ0;;78SUw|{2@FaAorinY$_X-r{_yzC*zfV!pr>eI2E`! zG)S7wlJoc&gKqXw@OA?l4inN_8!f;(U1zqP?5q-jzpc1Glz;Dy?ooQ_*1nzlKFwFU ziA_?$OMfO-`d?bZq2?w)x}jIgEr9m`7-)Olu5(jiNU{Tsk`gQA7u9f`_>Ypdr%^W; ztuUr&+NkqZuuw5UNb7Qs=KMb{K$)3MdUbCI`v{PhaEe7*G|KS5pGsAdUWj#GyOy+a z+c5EnUuwC;m64-KofRk4CLfxS0_6iQkbYh^ntdN&6zO}yWMKF{J&x(BtbJ1^bC91& zteQ1!#^cC=X{Kuq_GiTp;k&w7(N`%`-^DOuEQC(4O0D2f zuHLHmt>VJ~#`3YLe}5jlmIVedd@N0~@XvlKb&F>vXJ^1pn69kPopj2$II;l$SML~| zP;@q{UnpBb)ck{H*a_%LM4yvt$bYzqnf~+JYv)EItjtD{ouZ+>xfU+_OIh&k=SSnj zpvgxv7K^ox{s;7T_%o6++FmSf&J)WgbiODIyh#q7Qo2W_MoeE!z#Z?OF4v5DIVkTn zDm`Q_p{#u-0v`(w6C*!qICW9l-sdY?TLYhQwgv|Sya7VhxG!VyDF9`K_u|3iPw_?{ zAk{UoealA&e_at{C~4aMbq_PEa%uAVW?it8mYrK8tu!G`J6*q{wmzo`{K^Qlvsh(q zJNZ{mdabJwb_6|pVe_dftW~nf?@tQ_wx;2dI`G8YQl4;j*X1}@MB8@NQdNS3Mh^ZR z%Ix;szl2mDrk`pt5sLc;(ZI3@u)5bbYtcFtowRu?5o6IEcNUUVm^tqO5U(5N-p_L>0%9{Af*w>DP*%!Xg zjn_si2wq4I? z28Ov^>*_46-@jQK@&F^THm=wCK=|w`bzG!Ve!ZXp>4p4>{SN%92!Qy#k2LXPm~h+)`q)`j(Qc)eE!=O%`O6P`uW~PiZy28J-|duXCHLXZ&_JZ!RR(eA30$6Pkxi z|MeR_M3RE)K3NCs7g$Ntw(Q{*G~E&BOV0+oCj%ivq(E)^QCsN`n-#t%N?~?BX$V>+ zPXSne-qoOZ@Jk+!tHkegz&crX-%JuT9j520?8=bQd*htgmSCMDTer36Snn}I(&!S% z`iE^Ag*7$#!r+Bn<8KxH1%K7H>8|sx{R_h?4oEiuF1fcY)`t1#-m-z4BE4>8$aLLZFU#UV5@6nFmZ)q z4wTD>S+kjYfN--1<@i-;){vv@D66d0M!p8@x6olRcv$&^HL%*goXKL&Ov5P6&)W(E zBpmRS_xiQyZ%}(6x6Q-JUV2mjs^XaL+b09y13$m>f@gxJ$B*@9K!qd*5V#_+Mp}}M z`vrd+9Hz?zZ080eh}zM;9rUeTG+N6IXvG=bgHBNT^SYjeE%vSx8$=Jj2yd24PMO8{ z^oMn+ACP)fa*0+L1M#*D{Zf|%pab%X)~YnF5AQ=yV0`73M58Sny-OhzaD>ay@r@r$kS{g|JM(|B<{_q2y2~C+U+c@&}()>(;v|b z0>E(w7(iK^0=mlyRnkB2hj{s`Xxbaw75m8hLqr<1R4 zfK;oama3m`u8akbTa7X!OP7C-cq8onEDm2Mzq zTl%O!X7!P$d&277l9uzf{VVhwAZE|c{`W6+sgX{L%}?wae;U9r@SV7(D{50_O_@Po z8bYVGz#K7IKyO&eymmP60iw=pU+shtkVzs=Elg(hm_cGq7lyGK{0w}GNrgAHG zV(Ct1vMMjzgO`D_1zC%CidzKt{HDkm9lWxXJpJAX+&=iFKXi)o0P~6)2LlK4>E0`%fL&^8F7)a|l zr^ni`kyRx;&~#AW3p$xoWY0T&9mB|F&;;df>8= zZLSG`M)?-$t8q{`8#fY6m2XOjp+5AQyp zv(aw6+c$mw*RIE=RY03)?jhJ-vWNf7r$+|s&U6_QJ>m! zU}_}fR;7ZRUGvm`{C2Ahd>BPo5vAHCN3W%AbcjhOD0j7Sf_XBJ=k-qe+yD9*_(?0! zwMykmeui%7FU59NUVN7S-X+JqCk`@hrj8$dr}A%+-l&Ts;&Az$f9gW{crU)Y-T&I@ zC2!1v8L!F<5j8q*y_O!%l1VQUfvagT$N>W5PYC&uH2wHIp`7zIt??u5O3u5x`JG_n zpZb$E%))XoAg_q6i?)n|20C8(UoLtVUa*z4!pL@c@ zBeyl@u3HRM+|@x5&`H;CnJ;+VZg-=6*z2&=#kfi951GSXukdDpsQ&(>1dLkJsqtS5 z=KEKL0FknMh8F%GNpH!2Cm-A9Y!6%21Ry+U0Xjzq;VFpauSR)bm~%NSEvHboPIj?a zd_utixh47UNV1)XVx^gxWs;ZK(!@)Xrf*5GzrMzqD9GkQ1KF;LhV{yJc#p4d?)CjN zm!Vy4P>_CId3oOc%K93P;`yjsSnNiw2{cejyaUg9)z~l5pn1b7hiNvmOi!qe`U9an zJ^bBLu7%$nJ=Hbl64R{tE2h7z!YLxZowyK5MbSG6BoRj+>2zisiU%e7JS&6*S3HGO z`qX6qDWDfVs3~pek8af-mT?e}Km0F?FN8y?c-r}%HFT)9ZPquS(z2%|RQ`Bx9d;D# z3KzCtP`A?9Pfvco&t1t^3cX=-sHG_5G$#DR$i;IZPOF*vxIobQ(RlNAIGnNSO8?p| zQ<5~?d-ty7O}!)g`PLfbdO!gnY~!FObW4h|zY@=Gg2}hKbB^%8GKH|zz>{x)yiz5? zsg{}#x}ZoU;bDbduvntVY4Np(UjUTaLJbG0AkOy6{E4=C5@vS&{0m5r`B>l^#O&y0vkv4mdhv z)+g7wsuwmrhWzbt3P&lyks*g`Gi{lcno^suomXXix2X9}8Vihc>}}ytlKQ@gW#Rhd zb9*xgpM2xLr6p3^EW@|9>DdoEnO4)F4-R{A{(FD_+)HbANvCa3!F{frc%SmJd{XQw zm{c4bzXq0pPaveRuUW}4{`IXdWz8G!=ZOlN33`S?hRkG#HlUnulFB|y!l~#kB`}R{ z;^4dqug!;YZ>`01YOqQW=Pu~fa=PqTah%su5qkfFr^n5``m+~_;mtb1OAaoA-Xte& zQ@^Q;Ew`|mKy4V~^)>sJcemPw#}B)aCev`Ies9>qW@}Ng#S82b^R5O#^O2AuLtv95m4Xx7?-z|0!~qwd?q&e}4se zH~!;w+>s|z&!oTTWr;{Y_pX7DsRBq^s8bSi6){;gqWV;&-Icn)wyKs?$#a{*cZ7&= zvsyq&K{NMzY5yYprRb0~CUiS?TxO<|I)$Qa!bL2?>aMOmynynJK=GaAx&D4#-Gz)l zQS7`>->(LlySkwEUeli&6O}E0MX1kQO5b|jm6wew74I<9A0?RzOPa8IzldQ$D8+0I zczx#yeMd92dgbLthEHmgmK_vwIh&YeISdW!a>TSy@-F4^h3jCY6!&BJBs4tmKOo;z zIR;j_lu)K&18SXouab^G!XBM0wBHVUl@*{nvmWEkAiu)pLTw4-nj0m(L$QZ6TKd;G zIsdrq>zh{8JJ=x8sk{NblU#DlJ=m{0k>l*+JIm=1-eY!2hIUU_<6k?GI-!m4Zuk}s zAW8=diriW3jxTWS2kGrE=WvL?Ll)Idy$Tv1(IzqGYx#Rp-9VgC! zo{#Txi=~V3cpWuD&-s4xAPQ-?Sz&i2?etkMwc+z#k-yi@epnK%9U0qPj+o3DFSTXf ztnB=RNxX4H^4Z4hmxW*<3z5?0Wf|dsjZmF389;(5RQNgGFay~$WW8k?HoAE>Vt4|8 zcg;Ow`)&VEQL#qsLLIB&;P1C+Wb{- z>*!4N51t4Flh10~(>+7;yYHTDQT+?LH+bIdXg;s=#)0Rt>+7Vr`KW9!R%7O8mAR*Z zWW_H58TDB%hN2dNG%mBBX?jB2!*1p(GT}H}mz0iNjNnLPyRJ3ALNt5+U<;LR-)>!5 zJqn)9R@@ar^h6m;9@$9R#JaSssD+zjG$RWk+DFa_)|F4@Re@CtwhwA#g^f2D0i?%% zx6h(7F4i#*PQQptRs<^Ul06qpAJMaKhJ%(jg>xbyeo@T#4LPDeLcW1l9?s4ZCxlm_ z^q7W!Q8%>Oyv5!R3>uKg&H|Imou#)z5Z{-A+YSul%<$h$_g>2zOp1jSr+1j}OUNjl7+UpP~4Z$ly=VXLy@`L>h z^${ca&5;i(;|IljSh)6qtv4Cv@1MYwXwI8UMYeA4K$-#Ted{u5`@_e<=@^SSe3eDN zgTL#q*r4I4h-K?t+W$gW7IIV8*4PTIN@5UR%A#?oKT-=w2Li1U=U<%6n4!02sg~m2 z8za5@ryWEv9_9j3@VapZNN(>gRoC%eiH&!|PiGkT;1gX|IMCtKJ7`R5mp&+Q+m@NK zlu>RvqgPZ{7rr}ap;Mx%(r%ixG%-(LE&OzVs1<6D?#Nt6EJ}+nN=@{cU<<}!r66i8 zkf{{kC0oe^Z5rY37ZMBPWjDH0*zb;Hqp{2v`u4Q;b{1)9B-54XX0S0)&Q;!sg(@cb zPHW$W!Anh;Yw$PsaJA?fi<=mF2D)qdQp>u{g+3H(VR*iU-NuerMGyFdmQ@MzgV3Bz z=P?EFLsg{I42N>XKEW3Lt9Fk%mfyrq7L@Q(D*7Egz2}C(aO_Z|)y!Se*8MP2yW&y; zU=?)X!?sK`cHsBq2U&;IuUh5qk*kf2w`=hLD4pQp&BwZO*F;=wm!2o?w%ppx6QccA zgMAo_JAycB0NyCw?SO$KhGfGg!3c;i&n5$Wr=TH=7`7tH^O5dZC{#A_=I6 zx*Lan-FR?Ax#hjv2M-R*5<;V9>Xs9x?rCkl3y~NgvYcWxS4}I5XCvhpe?#H#@JwCB zgZBm+OP|WRqJzKLhLA2po{3v&eNFntnhhnSJwQ>NKp(C*-X+C1Rz#I`ov&YRP3OOC zQazo`)HWDLUAP51dtiU|l6pw+(#Ktjl@&N$vgOR_BpeGStr&@wWBbQIPfT$XFqA6w z`-j6a0LKhX&+Rl~^Iyg@80vk&^7)BO&kd`A@g!FOKGfGf_1yF(Ib&rG}*am7FTN7f^pB zKj;i=gb2Uw?E>!6&0v}a;ZIh!vW^quyjQw{;bFeTXMh}9Y|STaBX!<|L3X&Qb1T+4B z!CS3n$|i(ZWkV?0lI#(ymi;S~e)k^R;&njhMLnhqyu+W7BaAuZiw3DM;Y95+eVL(W z0}k{r`&ZzjO#@CVVWC5DeF*oImzA}L+nztq5a|*# zmJ!R%OEN#9D`d*RLyqC!S$Q7S0(L5evXGY}vK0myK|-8|2VYk(V)#&MqGWoGqJz45 zs>QvhaF6%07uqdf{ao;t^{I_)e5p3=5&yKoc>Y9St$RQ#|DTw^1gLI%dL=!hzV6|R zu-8Lv=EC~l43dpslRGtit)W1x8v9dX$^ZM?BSQ{B_Pec-=HP z3pJb;e84Vyc{OzCsJcK(@)dg&uMiID1I$E4#7HJ^s|}zy!1W&qR-AHCIKmKEygQn- zAzWq8bCR@)dxB_z2<1Z0XXG7cLNYv9oTo6zqAl?Z4h(koBo{)m9lSmN5sSMU)3G#d z%GbfL#{QtAZ+jX9~1Rbg-g>$l+h5oBv9fijgHjK$Mffp)p zZ+4$guFe~NZ|>LQ1e>~8!2zwV3x)D_AmR1hLXDIe8QWrw7$|>Z-~P%kJ$~u`;tf|_ zkk#yjB+#UNbF~E}IV8b&xZ)$I=}yS+Z0l?Tv6Z0_qK$dT>r#1qB{Gt9*G|99_hFah z#V{Es9Szlj=tu=yc7$wp*KBaavd*NWxBbId%`0W=q~UP~+7J<3#(Pp(MQ9aD7hCB} zGU^l4#zz6m+RxQbTiH^I8#lIbo#V};pLPhTH7XmF=cK!mr~f03r*VhBUbUatTeO8$ zLT}Vr1103iQJ`6V^-*AN_aOQ|k3hqYsBpNPL7LZdx0S^civ6kct3NGY-qH_g{f1%& zd3Gw=R9rjU!#fONh9o`P|M$1`n-B%K>fCWZ#L+~wTSCxkDpIu~jk&4WWdmQ7Qu_S| zBIm{RE-koYE_INfHXv7dL6CV#n{uZ-k-vIHd(VQ|1UK5}JaK;zld`ed+vP|Km)zqN zQ6*5#^f*Z;b~v0i^fMN&8R%msH;dhAS3*RJZID0AQNjvDM+QFpuDDm=x8^ahzZ&eD z#RJ*f>3;=U4=CUR7w^n8{8f<+`%MBON5E6+)U93W3tQw1KcJca)P#-@MAtYx&Wii% z|8jq(_26ONzDu{mLp7EsV_eZ}k%!AVF3xC8RUBc;Sk`qYQmlJ1jc`pUi)63CKOjGU z-5=?y?>|!4LFN=c+*D${#5cRit+@!V4@Y)(U)M`rC3)O-cNo35G&Z}_`JowW&xIn% zeUQzYFW~L_YM(_*m99+yCby!XYLwE1=A6+$_?uTGN&q@d;WA zT@~^Z18QD~G{(HtlqF4xRKr_^*_W{6>cKXS3gx=)zXmy5gVx+*KeYJ2yb&3Ft|c|( z&T@w~N=4GsAqfEc3F=p`_E!E6?VI=7^A)o>@Ao_qDVY~%r%t3;DWJBT&vNAWYRGXg z`$s6JyRLE%)Frol);xH5zZlO6VD=o;sDczq%WdSl^lWo45lQ;VWouz&zs;=3J>cqAznu35^6}|Wl&^Uo<6VFRXUOdr$ zwDV`JYi6l^KeWpWePU5+hIwgsj`E^v@_k{@Lk0ybhCRRRxS6=sVXN2br?T~Ty#}gY zJKG^R6f!iKtFM!f-yg9h-Vmc;T4a$bSC$7aHg+9#Il86#YUEP(w2IE^%RO71|EXIT zJjA2YMIIM7!}^X`NV(#hfdeE!3a7o#{7?QR-sYbsLpAoC z_O>tH91`KpC*S5y*LryOWk?GBFcQx)cnj_VjNhA1J>&SuvwP2#9@Sj+aJ-D6^&|#t@euZjO3gV88blEZ_^G3JC_WL%U6DnuKO2T*c_R0&DCOmeDbiW zFzTo25$wfk*6ZRcGmTR>rJpu}m~;Oy{_MZ?)**Wy`zdD4?d&zwIXhAzsA=*oR;}(d z``xa8fyX}m;dIxn2C)H_lf0aJ*6Dd&tqX@1TckYm5UQ)q7`aEOZqg4M*UVSsI9)By z%+hq29-AHUt$05CmzU}~g)T(mTsH0|ZG5&u&(g%s{T|g^dfs!!I6eHG88Ug_NhvKpJvtJE zFh53slHJ39%-=qadkj5ZeM^vEDK24f5eKVonZC?7?VyrrjskO!w=e#N1eA1iMp$ z6C7TS`dZ~%_5E*T>f{2!Xm`*z11EOtEif#8U!gys#fhC zd&Ca4YZF0KVl@fjo6is5>w5hK=Q^+JoacS+`*AP3aa`fIoL(=#2AYRqAw!5BH8E02 z5^uVRkFz2PWKDaM0a`r@1dOm(vN#bdmlvk|4_~ilsGSe6d~51pLhMLlDnYtQ9QgE( z20uUksLnAu_!O7r>NoyJp@8RDCVyqNTqkOEbhdfY&3B!ermJWUcW@w#)bBF004Uyx zTXwM455H&Imkok%0Su1>aF0S1c=sF>l>ezVN;CbIP0KLkb6=f*s1Hh#T=ioD7bdH&(VcD+yE(=!)D)bqR$KJF=s0S^gDUV^VY3DphSQ?Tmj zZ4^-+FM#fJqF98<1DSAM{ha2_``ZT4%`*B>b~~HbaU8Oc^Z#A|=W(bGzv7@4o~)Ur zsPq1#t|hcXw>N}hT31jPNXX;7j_Ur^k~&=t{CW0Km!{upykR~dX0Cwub+-w19weWL zkvpQ8vU^Ya1IC%}2XRK0(7GDbyzR)rcbDz5(e|LKk=G88i;zIbyaRaS#iVbKmV*Gb zs^ZT4%zDAYRZfsii3I$VaNn?n)DJD%-EOiRe&m4QNO8oQ za=m29nJmR;P00Nt-o3D>fOtY#e_{c z`~64-w=Ti`n>D(f@&00VzFl&wftu5hwR6WD^Xy;j| zLlB-g$Uzl`bd7-6%1o}xLVqz|9gZsGt><^xCfty9RZB&FKV*Ba5nVo$W-8_MZ@6O> zaWWj+5=oR~^a$#yqq~f4Tp$}(vk4<1Wa+stBPA-+&wyUfLW*iw7Y^F0I17q3DHU*4 z&n?Xa;G}$y>xQWGutMdT~aU z5_sNKDVO6oUd2*_2Ta^iby@grIjbuVUBAQAoyZZqHT_{%36aEcGDe|u+N#{QFb}9m}jzbrhE(cHxuGpNPeMe_ghqfMA4eH~v;>4P@2hhaLBIUlZe=R;@%7K~= zuzb|NbE1n}8(ADntGP{2(~q9|WeRq?+x)6{mXqW2QS7X~x~rVvqqot~aG0d@$M$2< ziAm^^11)!Y2t*CA9w~&)i^8@_u;i4*RiKCx@J4P8C=$FVTDR3Z!Nlz7oao+`u(Jo}Q=uU=ktoZvZ+~7&V-#9&Rvr7m(7QOX>b+UFi8S`wUZp{z- zhVosjwU^F1WO*LTWXM+XZGv(&OGefm6%@PSsh55Jp&b_%>@qm#yycx)efeeBd!2P6 zV>M6fmJW=C;^T#Q6j^_HTtkG0-}=_|EWAfsOOE{*ZB-#&DW8i{?Y(OinQ-k~pn^r@ zW7wuzw4UhotHWvI$n01mfH`PPp4&%&jgS5&|BM4jQ5*R# z)Yl>ME5+lEpIHv!5d7XzW!?S0CK#slWdi1Y4i{In8ehO-IWsG==40n_K)&LKu5Qzj z;cUhK)E1Zzavh_jG|SWzQM_^ba_7hT;M0<@GySW$js?z$j*88*PQ#%X}qS3j4|e#wo>+(0xK!;a_*OE-Bn&*D`(N`W=AaL zkvi@nP`g#A#sL1vHNK#`&-7NUl%B}MKV+#V%ORRRJD&KELKSzc8m%5D%<~NJy4eoFYO-P3U#-yyF9tzi4Z=7?3JK+ zU*LDCKYG;__GT}Rz|h@8;~cWPm)#_7M!T&@&O^g~`{m5Nlb0qZ`=RHpKt#KjrxiF^ zLElRS=1BaVmxSY=?c7+^GD=@KLQRf_;T=aVKG#NxviLtUlXuK7Va=NmJ6vC@3(qEx z_2z6w5>HlmS_VgC#;J8650NcFT&dR9K1QJ$CN2=L^t4RwM_wI#3vja{skQaOrx+syk_2pQ80#hUcHuL{a5&oNHROzFj=DzF=p)D~jxu%N?R}S#fS)>J8b1 zwZZ&OVa?(1^J43h2K?kb!n>y493?ciFz$ptVeT1MM9P}p38aVzT0XwbVVR6CcU|*< zl_gOcXT+R;jQbsyA4&O)Lu%whAC5OJ+_i!S&s>mIc8U2${oi}y>#O+t-8vPv@e^ko z1Hq)t?!4(b{|{zo_d2-}1AIMy=s}2;#wsLEp6v4lNzL*D=Y95C*2d_!vzv2=f|Wam z#9o)v%z{p>#-lpCnOI6BKQKv|LuudM?{N%AiFrmfMDKf2d23;*<*kgxgV!`3QB%)B zE$YuxWjB|cw(=If#xa^r9i)D_sCDag%*qNr00^Y z7pv$4tEDeN!=dc<9H>MeF!D$5YYjG2wlRdh+E3}lAnMSrK(PT#*lHA_kunf(%i}^; zJJa(!84CgGh=zXBI&QL!DldOqPAu?sS^qY@#9VnC0g;D{E^5~woYSs@`LI0Hh<}5F zZ3N9ao#tJE)u_?BgW5Q)SxZ7IIziKo3fQj`30bRP|ZWFk21_I<(7dpR#57 zB=EBdBy3LGTxq%XYaRP*0l(u2xrG|tU;Eu5Vv%`T?63pJQZospzvWoQVaukFKLxE#C2;blb3Ji#|fHW7k8zH zIxBZ_2SjP96bk;jKGFbJom+=+Ch} zI`5q+QwX2NPeDtR3q?f3T@3fSBGQlgssKrp0d8y=4L;Q0nIU7${$H<*Pk-LO6(k5} zf3zknF_hF32c)b1i1=;RanSO^+xLdSE48(jCL&(l7tK(=Yht zv!|c)Nh?}m>6f<7k%nI)#IwgaMAHH(BU%C%;;u7{e;)Sr)%6^1-C4IMM_lI927Jd&=iP}ti*9mpcc>LZ=Plw256&~@2L-Sz z;+swKV*nQIU&~TYO)@R9OE=t|l1l|3ehm`JzW;fszEU~0tb?3X-{uWfROia@rLWmP z$?2Y03-|v>z~&Rq*1eW$9OCd^+lM>`Q}J-7R{$vcclRgDQFTMtf+7v-YEo0(-$m1i zUjoPOD&|$(MX8v}Qoro8L48xln>d_;rQ`XA(7G(9!sui|F00%k8q+Pb$0ZIO9uQ6zXq`Zp=fshWnEy? zi}u!@_?6Q3`L=k`6Q2(o#OKP5KJ6-8h8%tq5RnI+Nys1{_SD(ldg<-o!My#S9K;wg zPhpPCF|w}N(tLR@f**g^jA=ZXyZ{N5olDygKCSqMN{Te|TJmec(n$$Qv_I%uC|GHG zdOl59fz^hXV=}2j;Ep$x{A6;1)ME_@ATjr`0)j1&1NVU5b6mTP@ zwae2+vX#7-n0VAmw#!$}@YEA6_mBa?S5U6pL<+L3E^>bk{O}8NDwbpARG;vsA8(ueW28*XNN%ca;6%ah z%hBDW(*)3BUxj{SgD!QCkt@?>+~ zpR6^nJTlk;cs5!E3ENxI`%`*+3F+*+=_)>5=MX7A#-C`mnI3_Y>T}F%flb#`B_%|UVB|Re}<6%aIjlI2~gp^d*;8#~I z`&_q-3`uGtN_im!NADVFQ2|l)XumWjerB3i*;|*>gM9L9vLL6_KuRx&*Yd>s^xH$8 zoJRfRt2Hsd<>C^17X(h0&{P-nk;!oAW7$^mp0SaFx9j7U3*|wxkV!q95kCNQwBF6K z)Ao=0G;ptyQGN)8nU~Xchm|!m<0n2Ru#!?|A(n$F9jUXc6&|r1S+37G9L{@gFkU*u z1rdi1o$H9adAW0{d6YW;V{nZa*JeWWQO5v3`x%MZch?H@ogaPV?q0w*7&_!T7(P39 z0m*Anx$HY*t?$NUbo?vnDWj#qzA%)0*D@qW(PCC{(qTDMWj~bEsix@*bB)z(>^APV z$3aRzFTK!i=5P?S^m#Qgq%>c}u=va)XTTOg+{^I!*({UZyk6zNaivZDzIjpArRII0 z-E1EW0P)b}SS(`5?#^a`P*iwN`vfKvmbGdk4=+Q`=MQ}+OfL#AA+sn=mBl&aKZK}= zgG&1qW~gm#uh3~g%4-X^z$+|nhwTwJ{-ym>t3n$A%EVqwoBb7IpH^rEPhlXz6>bne zXWp%W3kj?NsNy5lDj@$y25Qvd^PptStG{b)pO%@3RlbJzwWX+s(4H^;Mjz_#@^6z| z8Bx%al%!I<&4!LZz;>)rBSP`hd+1<5GA&!nX|VwpC(8Y_$n_fd~DI8AC@b)if~k@u1QBm6nsk@cqwziVDLiEgP|7^4}Frk zjWRAXt#7!!iSbehXCe(Kc!eb}K_hdcCGHyO=`j@a+`a={VBj^~{<8GM=jw&{9SdbS zIkH6M9%YFY+*WC7!e_1uj;Y<;l#Hs}a@+G)4b$td64ngw+$+~^a(W+t;GczfXi8TTk{iIc^?Z|GAjq4f(nLPN+Sibd&QL_1BKy!03LedTq zAZPNM-!(ark|(37txTb0`w0T5ORJ^k-&}4Y#r_*PBI^7~jk!vCvTJZIw6k36QD8@| zZ18fFUc~j2ar?7PH7T9r0Q7-AU!Lq7;gqB#NH(#<*3Y!*JPHXr>wo{P42M6?i^S397lKx zV~PI>HS=SvwshNnVn|5SuM3(Mzt~txuCc|rR_Gj?UHWam2@sV3J{7o>qi{XUW&E}n z{W>?$_4%4I`_C<%%(f#qz*hbtYg41HyV~MKeS&>(YDYku%d(uH-?;yNQ${9lTm?1R zuTpj6*?(N7A>q+Og{}ei-hoGsQsydu=+N#B&6;^Vpu*A5vk6Ud1M?epD(+$r>+0N} zHWt+Mig8VXNUGjWN$agp`?C^DNG}*Ng^YjL)3D`F9VNkI%*=>$(An9D&_+3De(U#6 z=8{PzVkY|g5WoVgG#`RjqYe@nWFokMe3Ml5)CS~G1dfYp-CWglx@t8Iovx3ur%6z) zF#BDC$`m;uXObhbzRsMGU;Kq1J0z)7cp9Ja}I08oiXyI6@Y*@bnVmE(C$i!L)d&n-maG7^`*=X&3cW9Jfw%KjUr(z}BySU0zG zphd!HE@a7B<(Fu!Kt4n;PZxr=hoMR>hF=u>NhZA$j8T#*oXQ#ZScUK@z((tQ&nDgl+Z}Qe{oM&bt(mQ)M$c zV{v-_$iDr6Jr3U!dE`+UWPcv*mkQpDSux<*U;ehe)1d)C4F<@HJh4R zNrnyelEPP2jsHw|jP9Av1XzEc)w z?7j8@yt1?7*xv zIMOW@>`2IfG>PjkUM{D%R&m@{TDf(MfCQ2mGj@s##LSZG82my;o3tO=Ufi{M6j0wk zsNCng%bL}oUiosUU`8WYK$v@`uyPDv%!;e*yhB{>arvA7Sp>H3&?MXX&H-(`Yvc^; z*2MfRu^)zDFg}K@_G!V>X9uqYvwT0xOrY>zTE=geRK}WFUUBbuT+IKsWM5D8 z2;b3E)9A&qFoVn<3?{4wbEH>1d;&d<_J98@&|7F5 z6!ciip`*#yl4D#k^df_i`4oIQU_y#>&whLjEGp+RUp+7M$MHXa3a?TE=AF8{v){`tGzTnM&s?r%!%A8R5YFIz z5idQ{xImQXt*%aZ*mMdj{(bRj;KiRaqMZ-5$Lww*KUi zyn1xP6TdmyF2Ky=*~^F7#7Tjrg!K{uXQvSTBntlZl^f-;vc zmS44CkZE)k{`i4mIO4^*Jq}KRA$M7>!RDrH&XQ`gbo8U^tn2*Hlm1U5CU<%jo76S0 z|AyJV@6-*fZu{!bko1IKb&=`&Gi}lEn>2Dt94w-F&+!|e2NIS%&{ap}(rM>c1XFtD`h>T*hNcpv#tnY27aJMdZ^AuiUo6@bvU2ONB~ltDDQr0uJx= zTMb^bl()A=C3{s1wCuUwDF^?eG!of+G0v5XrT=WxkF-FL&^v5zkDs?Q&;n`D)Lkx_$(x&7}!?vCa^m*ceo zibLy?B+_ffh#(u9P3tRv9v>yMA(DB>%#-eTQLx5DdLbRm2}YRBdRE*m7^X(LZ3`|V zY%*J=qlLgdR!hnhA+YjTg}(9M0boywA%e!nfW>H>C!Y51Q~i-V^!7bD;J^R>n;>iw&ip*>Vd zdCRi}8H(JJlBqak2-ZkjuVX3tT{Ci9+4EigQ@vK`t=+C`$D*+7`e(Pyx-NMJG`(<` zx67&uU9o}$a{Qr)b5rTzw=WOp4&DR!s6*USBox7=;5C0Mce2 z3I8GK)7`Ymaz9?C(xUjg3{r5-7mw1Ie+fY*2SY>;~f*a_mye#iJVUH6Lz1=^; zAOZV3mhHD!+z1k!U$~?&(|PiV1>PNMqikxNO}Mc%pL?;9j_QP2=#%Wsp|n?QDv}XR zTd^#s_~5}I;xHJSicG4R{(fyS;A*YB_2G=NCGIx0B`~_NKeBElMrH2XoEUyi)_gO{ zUo?S<(tXYQaqu+Gi_jdN4%>zO2z@3qimbndZl4JTtSOp-i77!2Q8oOq( zpFLB!oH?G$JRIo}D4l9E?16C$)wTfq*UyXlHenrqc<$|qj75msU`{%-L4Qh-8Wv}q z#H$XOffvof!poM~@sN7nVAus+(nOvI&trBLra^DbsFp{9Y<~Ep$4EkLQPqPF5 zIvjbLAZp^v5;mg_^fry;D~&XZbVwq1NR+ipg!U&@-BtXFSmn+tX8D~e3|i=xJfNyJ zR32K3ZJT*g=|Ij^{1a3#8Zq=!te_6MTYzckzrnXYqk|@U^U>PP*my=}CL6vT*E0(ADYi-Qh;pJaBHYML|G?NFX8nkrXgyF4gTj{Wv<0duni{ zX<{Yf8PemNVW;PFHi1uao`NNkhV+3dbag$d~!mN!(TurzEtxS=|q}98DVeoeza#Z#-EhCE=qK?c~yuSh%3*v>g$J zw!9mBo-#!>-3q36Q>hT@Iz*o7{LWJJbXStz=I^sF=2e4?OkOp8y{y@;%Xt?Y{)jV+ zfeHj98B6+?H18|EP^1-5^VQ>Pt1={LvL|B)TYQkCA8tPdBXwW64S7(5ULI081BJop zwO1Rn_kiz&(N;Df<>eQ~i}Ww6@$HMtO9Hb3!R=<3B2jfw7cwW>H>Q}**GnyA3?dxC zDgrTr7QBCRm1T8P<&UGhO+WDmuo-(1Z--nLN%C@w;i%U#TUbQ3Rj4lIMIBkG_!2`3 zCzs-4Q<@y{%O2i0xlZTytVp{C2KYScc^QV ze$5i#FwxB#3Jy_*VP`iP{N`)x6>IwhXArGN+J15Ru>K9V#7SUVeVe1`*0Y}Sp|%6c z5P>dsEZm^R;Hjzefgdu3X8ue#$R2bUO6mga^EJMr{p?c;e}e5op!>UT?oQ~=_M7WHU$j|B+W37V9`b2y z(K_{z--PGaIC^g?aI^|`kAp(od-N}v4)VX>D6=!@FdcD9AXNbWku(>Rjg$UXM+J5$ z-SJqs?#AH^~5a;z-1G(8(S z5$o+L_;+QFK{S))Qw>Jx`={Zd?*WVbRM{Csj2UoT_2g$FKc-N}+b6Tykmc%88)8g;uF>_wb={;+bW$FG8yK3buPf zZCv-h`i0rUZYwuZR1;}e0i9MmJo_;qkp;_34YPFVNK?f$7zKv{bbI`48vTgUy_HiX z9<#fW4wypQ>hkvTCjIM4P=%+e-_ z?^+2~M#%=oAWA1Fsur}**D_sRwJ4Toj(4(S)#H*z-}83}ar`6P-3J~eK!nhIU5!6% zVe9Y28-|LBex`9DX8IU}hV zdrOhAq8Y4PmIf<$tjSsXIe)qawKB1_lq6BDinte z7)>(ZQy$8vXmdk~{<*Gd+Sn2&a)^QpzIzxp;C_H~4W=Xx#8q*UY;}W$wso5c;j&B6t;KZ}QJgwB*^$PD|LOedshWxcSbFxoV)wop(vf7eZ|oM1tmwC_IJ}4U=Ck+s6J%>? z^k7_wM;H})6dPKpy^&g4nCKa746zLKGM3rKz7b;?Yd7!YP9BN7zuW~>Ud;@mZnE)% z43Wf#+=m1@_P2TG^T)aX0b>_jK-$Vg0qFRSVei^yBPGqLC{bT4WvGQ&sm+_iO<>Ew zn3F8IT|jMj!yv>gRCZy%lwiQEt8#rQZtZXHrlT{D+n4!$3;6G8v>5YJyXNvYt`IK; z=5haaK?i6-kJAp0@Z#);Zox@Mmik;uBnbORy%UtwQsYoWG%~m-vNflqWg^YYUeJHh z`O56Ab94EL=daF!3}@O)y<*;rJWzNsC_iC_cf4LF-@Ne)e74^zC&Oe@QU`0#wGQvK za(nWiRpUMi@+leVYRHQG_gakjW54Ywr}v$hc)oiwZU5v2&W~;=x6`u?>M_;|9(!OI zcTOU|D!&SkNCa=pH(-z||EwTDRMk;~2jawqpzW>Qw%CQ(gjWm2wp5XeEX+8{*UKRN` zmZVI6z4suF(N(rt$5XKUL?EzZGo&M@zplalhSd56;d75RPy!NIBe><>wRG3`-8JRq zF=42in>gLR9R}F}UQ-erI#BN{yfU*y$JJ9Gz|lhX|0R8W28h`m=PKoAyKvqF@lU(V zQ}-%%oSj|JGiCgOMeFaxkRaJi0UzCWSIEQzGd2g>NO%5Hl{fO^E#{TZXcmdSDfqP| zJK4&~a*VT@M$6vBDM=CDW7tw_Efpj!|7O@9Q_hzMy=kq;7Ja#RVLFU8P}jag2o@<;IL6GE#F*T8 zAoivf=@uFDlhx<5c4j1F7f@;|FBFk9@!;V4n@Dx`)X z1j$D0_isGLwj6}_81MV8n((B`e8#+KX)ccse;+=Y?o=Ti!dF6Kj6Yp!2`Wl3_h#D{ zYTbHK6%=Tb4tf`kZ|PM{@PhvvJU^%v?;s;e{Po$X*+-56g?^m9SG?VVE*|U{$+~cD z6!{N&wLor9E( zpI@Luw-eS~fe8lEYwbq%f3=34;sUTkHo670A;AqXA|<}oxK{%Uk8mmTNRYOv2s#TK zLd!fLecZ@OooQnj7wd8!5H+N$w1o9{kAW5Ex1Yu6h46)!xptzV+$wToGP(2zz})KE z%)L@ONB=u&MlpQEPRwG^=0%l|;(VBU>9PAg=5mOx>aLBJ8B3h8XT* zQPs~dWf>Jz+aWlMIqx7+?2cmXDvoIqF#?i?W0a5HlZ{p4F-DV^$#hW%9AaXo7z?+c z8WbW;Pzw0E-5cfsG^?J4*S_zV&U%eaC0}9H22dMw2m{<5p2f-9Zo~|0+f38w)Rgj7 z=%>gEj=A^)K%G*=HrPp(3Y3QdN+@YK1a+-R1WDTkkpaeS(3^z zu<7XL82?e=4ny!83 zODX#2?Iis>brGVKTndc%b&V6v77u>zB53oycFRLwkoB~8v7GHJp&|ebjgi~H8AFFp z`s;^~ePdC&`%eNwbZy4Icdn%-YzbSbewe=zGU-c*_H%_>mvR;WCPilD{@K0)Rx zX32A0>E1u- zr4uq?b8jy+_mYQ)*DC6`M0|-ZbKXL7G=sBjbh-#70c~(*!vKaZao1s(79XYrJY2PN z@<}oN1e0NvHG8v&{>98tC8~71RVvum4P3_cPWllv+;`G>4NlsfHFxfQt@eBE{wI9O zG*2?AMWXf|Th1hqkYE0zu56zYxN4^KCC(1Y!hbpgYV6-DU;FM*sYgo-wC9>8t_m}< zLDvHwr&3?vY3Oot(X9OiGv;s)=y|FTr@57;iIRDbjh-i1KR}8BoCQy_J&XY=R4lP< zXA}!N5{>|*?hMkh<|4Kwt*Xh1ep`}n8%y@aRyA_gOx!}au7#Lgy$?pu&=Fj&qPSlk zkbrN+L4H#jJ9n20l+2ypk7elW+|ym4d+e2@D{n*LAEtv&6YLc zZ;ulpk6x1E^=`-(%crw6!2DiXt%lvICT0W(2z;pw$VA$Y-??yQWA9B^fW36&6POGc zCw9O!q|uysaf-?MM7l8{7tj6iw>sjFV@jHNn)WqbqTENG2aLi8K9(#iR#}lwx8!nQ z;k!l^szQ{B48nB4YI?}MGL4vCQGw@7FNm0}(ZP40m1OQ>;~|ZfX_GVFUPRg_4W1*r z2>jA#LjT(4jEKOF?A#FwMs-MJkT&zv=K@z=xvEg0{t0t6b5oA;pUn0;X6o#Fo}yv| zbz8}CpVg1<$_Yrl447rt6V;x6LIkak`Snwzn{MA4mk3N@D0kZKA1R3VdYNT$$%Ewo z(m4{l&RaXciXAa3AiTW}&RGfqqYU;;P8?;K(`V|&M7Ze%OW5llr=xUNQ*}<}R@w`t z>K*Y2`pl-x@~f?~xZNUD{5(DG%`SE@|DanN_?zABX4P#FQRjg`PP3I$#~siF48JTa zliVp{FLOsQM*u<|9kCTh{E)@mje%jJH_H-P*o9UAq1ru!o3252aBnBi6Ati>`d8Dk z__I+FPyxO%-arvs1Nre|JDSI|dzNX?ky3D#msHtm7p}?59O|oE`xu_$F{{HBT&M5S zaozRsbU&a6Q>5v}^p({KSkZ0k&`|_gOxtmoTgqKykSRUXhLxyD3w(aPND}@g1Ew-Y zDTKIyKvN&}ka3Lp<+tuP=kN5CoIbXcyV}iQPW%05_<}B>k~6xFekKO5Gf>vocF(1h z^91u%4l7W4d=7h_c-P^-L{z;l6#F*u=T#^c_*+q0voovXd(&QFNbz>^N);*@6LVTjZFhrhAIK9~?@Th`;s%1gdmzHXfwG``!LN z-aYqyPt7iqnUA-Q_zwjxM44CvX;x71nH{|eX5BweD z!N+YLVmCR^i*P0KUivlw&twozLRW}nvfRuA^l*sd3A;-tbpZhZ#M=mQ*Y4eE+m#3{ zrhwxanfiVJS^Jdvs|sb;9VCE$RD32U&Azx3zUF@jN9(tRuFkMNFDXAk`O@!BrpH@A z8Cv_koe!zI)7t3$ZPwLjFBQpA!F;>jmk=*{EqxKVa{9}~jgC~{_P5KT$`PGajiN^6 zm1l=bd~KVHH1)ML)M@Dg)h)5n6rzC~rq%@fDR?YhT<6a2-|hNd)}mx!r{Qx;BpD;t z;4bjLLk>px&)n~G*Yr}c&BQ}CzV?jBNQueDj~-9-zzU`w{~|vfdNL#w$u6X#59{S$ zb8!D`daMp`Kg@CCxm)Zkj#q0?zKlX!j)V}`9eF_hjNTV z*xewxRb^Yx;w&~}LYs{P@!g`-D`XanhyjSenPDJtt+rQJ?39URG2owQf_tmJ!w5@J zUQ(g*(_LI2v^#v#yvgsGC`?@nwnW&$7DIIMqsrv7k9?i(2Nw_II0h0XHdQ8Dh;7p- zb;GQW{M_g~J6#u6(-45vOdI-++1y+N2Lup%5VPF`N?lA?7oGqG6C zqL_d58<8<)Ee$+#sV!=+#Sc~8I>D6DKcS4{C;iRi>EEB^98H){hdZ3k{^5>~`D3-7 zDxt!x5wd-`aYBRMvGh|K)Glw%hIH=4{{+mLMpCThCbTFwGP;f3wlxd-u{WA##fdXrE3_0qb%fx7av0Fm&$_LWTudCvr(gA4}5m%fv&gx z4%oiAU2eZ`gE$BlT^-pS8Xr>Z`Vi2T5%dTw7)V2&RfCM!o0~b?7HrvUO2$(e_*9te zM{vBJKf!<^Z)Q*2O=4RH2ez*Kc&$}bCc-UV{NFvcM&h7w-yemF^wkUW{FF^Qs#uyE zt2LaVM@e(JZ}UZSvGnv-si{n$$^Eh3PT0FV>e|;Bmz{zf-9hzK3)*1Qw{ydpujG$w z^?D-3lBG;LA?uMFmFE=aV#H6wci!i0Y%97x79H8wbnEtC+6#9S6kJGC-~D+jQJ&q5 zN3#@yVu3Gj&k??!Vyw?Pscl^U1q<|)2EyoL8Prchfr1a-v~~~cZ8ZMIuVPaNyNCN6 z++POnSKb}2&=5-~q6>@I-&jnEj@y&aW+6(ZU z=j4S^QN2YvFF4I3DP5NtO0Nh@1zu#!>hv9cob1wfdiF!(83Ux%Q}kQP$i~doJbbch zn{(v%p0JIn{%Y92g!+=h;U^r`?Z}k zJ*4hKhEN8c1XoR20H_AzYm0#H$sSz03d8PJ#`vrH^9$%UlYOe-7R7=Cu4h)o|7PTf z5nRgn4XnV}tm!vxv3l(1JQy>dAH>a#@Qk+&)0U=quvqN`e0yS-B zQ?G91EMb487Yfdb;72P+V7+zxEweR*4(6Z#jV0WM^tD>u!qFSX-98m#J$e#-$L1!v zGatUvlYX@DEXQ>ocM*Gl-@7`2-pzE~E>|D&v$How7Bo&hp2>i$Ps?j_Bem^nGXQdh z_U97|mDPP}j;4-5nHU#pF!CUhdZT3xE8cKh+pVNj{7{VeF0z3;lbRrLU2IpgVWE)rct3q=`iuQI<#urZe8pw;WBfaFuYZ z-e;_8Xc=-agdqb$sI1PC{#=C#I&z*OgvZ$j_yym|Lu@Gt)P+S&m0#A=j9cYhUM-wroSJPQ2K3yr|BR zzWN80?-HZty!24*uctbV9}tAJ-2)W@dWRVA*VkI5`@Xa3y+)|H%BF{9&(ucJs`+wRC*Qvz>Kf>!`{+#+l%%PdA#sx zd+RV*{N2^J1kR+jrnE0Qb(N>OsFUbs!faH;7KSta{j=k`Mf=@IH;bFq{&&i-i&>q8 zK2+VeX+yu0UR8^njS5hDKFFteBt7FKoYw@$q>}@wxtMrEw$@hsxS#9GP-&8l%KqG1O2?4K z0$<{aP|u4xiV}B?gvq{^TV|`A(p{T_lTZI;A>j^7*zhN_s=zeO&V0U>RBPoDWHY8{ zo{z~+s(RI;)e|2tYDr3WLfeXRhU^P_?TaYb0}e@&qRFq7(q#^8gjPS*MtZnA4S8f? zQOdY4L**%O0x5%X@`QgWA!<#ET|g>}?u4uK7BW$ylHiBY{z1dF2XC0D_<^myRM%7p zB=Nfc>gI|Ld3Nw*aoS^0HI;8CL18>Bl`m`2QMR^A3O%v9s4(`&8dh{^&<C}#hTXS)~t?V9mqrGmQ>ZyJIp}_d$CB1!7EdouZ)jx)x_3XP!vL+Gly(15Vd)> z&C_k~ld&#Y_KO5#ZqW231r_i^Hd?+FT?~12{iQ+IT4$I`Wv4ago6)COsa$O?p)8`S z^XAFbzG(y1Su32ME0HT(t*s`*{raPNt{+QHj{0FS2#ICQ&Z$dudHn4~)}j-EC=$Oj~C3A8#sQX42_B(F^!LC<-B*WnhAh}%Mb zVS)lWXv+MTTxhsu$!IT`u~je$By*PkI7_ItmG>8U0NFQSWbq_IiUOZ1T+bI?p8};C zBPVk91T@$Ot)@#b`t`7aT;#9w!LeY2R6IMvij)P?X4%CyJQ*4>`{sgT4<%j=BBuuS z;PDQ!Y#a=%Japg3T9LBr*D7Voj1&*&^$k=uwlo{z8qUEw>LnCv8CeDEBw1P zB^ABd^sWK6VJu!_L2FY4wP<&PE51?wqZvQR2~mEl*rIs)Yv;sDk>j8COs_y4Jq&yl zRSi`=6gt>u-Odxe(w?7#+z>(oxu>-c|@`n1{5wsQuJ_PrA0K)NY>@8B)*h z9p%2->yaj`U=UIpRMagS5})l|R_eYL<;U!}Zyu*5PMZyTGPFPVepLD|kBi&Vf%Zi? zOok=N8c-WP5RF4(m`B+3cHsp$2th~GjiYpIt!?p7Z?vL`+)Q3=EVy1qvrlUZ7Uhxu zUEa{UacOw~p`FJ%Y|a9adDki zlzN#2vjWIz`D9&LX#d7|KL$1vb%8^;E%Hz;93x91R-F!7E)A(JOT)UxanJyZ*`;!v zYq82wfyWpRA05`2+TJ|*c0BWvhr$Z;LXSV>TCBUD>0+t_iZ!uxj-R!wh$;x&`*>OO z7V$kOz4Z4C>uAheGOnCb66bo-!pyil)4I1kHwihJ4o}oI0(@U0yMYMvjf>G0zV0DI zrGzVb0fE!9xCS8e$R1XkSc3A6J=3TIwAq) zLg_AtB_XE;(Bp3w!hZy+gSe8k#JCN1AhYF58@VsZsIj%l9L)bw_U2(pZf)CmWof4A zZd+P8u-leonwps-!mf6@%Yn+&91u%Q%P~h3gwo8))Dq3il#0~M0jHcr(bUA0)SO2I zoCiQqK$IWP^S)2-@2~HBzvJK!k6T#Rd9CZZ*S+p@UBgg1#?_)wR@@cfw&*Xw&5g%D ztO7i0tIC-H#g2c;W?v12Z4r&;=L zIV@luAK;QHVJJF_o0XXO&9O{3jM*$if1N!e&PyQ6u?;ho68r{vj3pvOVY- zwWYmoZhSALeFFX6bwC+pN`CmOb{QCW^_CQfnHu%2CvI28o4R~;f@P#8c*fC(M_yin zPYrdR(4fhT%pHv9Y;pbsASg>`%Y`>4Og~r!aXaZ&8*^?zhRc@opc2SIWz90`M^*Vp z-kt*gj)m1{_aMYq$q7!=J2Fe84ujuib(zfGOw=$hENFU=)o4f%zpi8-j>{$HEY%wR z$kuuLqpQ|qI*fIXd_^(b2rZB`&FI-n0Kc)(vD$QD@|E;^Iwwpj(Fyi#;acY0 z&I>l+E4tCkCmmd`q8Bcl7v^;Q5vaSP%cSaU^j_<-4yX%Ken3rPVgxc-DEHP9Af*m2 zcOkk(S2UE%Y<>UI^z1;;ysh_{E74x4UL^cvwy7W{hWpM$7Zkk!F<&fK#e97P_1?u( zQs|iKHFq;-<$8GWyjMBK>Afu^voSs+1vRIV9qfA*)p{w|r(954BX7<*um!H`wNy61 zxkpswW_?-qlPU3|Pdf$gA*|BI!FtYfG)q9(bPRh5C;zuZ;e7{W#-#OR^J=k})6~15 z5OC`BO=nciU2!MSwzceCn)l95#iMzaNk)CO8Isbf=CKx_X`U?QnCRzmRgaqI2#IO{15KcR!${K06hU0z1ATgAt; z6O0 z+dur7cSXxncnKM3&7h~@R@3(2Z28;!>YAbT^wN$+rK2Owy4ni>R(1tm_uMTn^=M~+ zSHXfKIISL0aYroXg>nTp8qn_P5C-F|%-)o-@oTpM; zqMmhYi%SP&K;zx<^E;a9u1ml7^*cH~OG4GY>yj!UpyDoD@~{Ehb>zb^P2OB_5V^q0 zP*2QhAx>`owRK{0VDjtPGoK<~nwoy!#@Y5WHpxJn$gNugQuyvW&xrx`B(vJH2gIW!e#Q6GT%8p=88o0JF-?##2htN;CAvgfI+&?dueThp`hcY2?qYpaI$ zYmC0kSV}iHywr0g?|MD<%fYdGjXWTwFnJ&qNj_p5B?+%Fh`y77GR`Si%hI$Abr5*M z!xkGt*}IYhwhzl5$yITmK`tLa8RRX;z;E>y2sA@@L7@5cl?LySij+&>w^P7nUd+S5 zwGPzJV*&8%e_#d7mZ$5eNxpeFY(_$OvQZOK=7rIoS zcQkFdpagj1c^fpE-3k%RXQ(vbl?qKm6TNy~#=Ml2GBQaSL>gv*njZqX>~nqc9cTaS zC+kM`kEVuSeLx{=sgWkFpS7h*Cn`IMVxZ~l0+k-e?8JLkrbk4o>%-8H-(I=RD|e#^ zvR%T0GM&K`m8wFHBq6+Hd$7*Xnaj=|C!K(YufSj&$8oJ>%15d2=Gk_vBS=FWa$KCK zF|G+M>Afs5zQ~jhTsSlTH)Ok?3tsHM1T^Mq6_(Ml@|LD7a%`eSNk3Ojl|gQjYFUw+ zUD`P`NP)x_!D8A2doR`1ei=J)RLX5D?IQP!@>r)DGnj-hlYji36vMiqK{||m=brZ`IW`~E8hg4` zeMt6TQbh8Gr?k8+A}#Am9;)B*mF?`@4KU*`^^BjLQhCMop)<52MH=CK$FRDBVcF-k zri=Fyn0JmQmC-+e1!+J#Zikv{@W+cm6Iuyy&DRTfytP}Tmlg`qOFya0wSG$;FsQL5 z0SBMiW!^}_RN77~d|XI#RH94(m-2;=EM;WJ*ga5q7=MY}viULI0pUA=1U1V<3~1T` z2=oj+_V7Py+QhV?>P2x=-pAL&*z*rXsmEL%*r~nU=u|7gTI$=N^jPT*Ml7~ct%g-l z&}UpIov`-RyI0(FeD!z9NbC{Y`o5y|CvGJh1!OHHweUSXfK? z;wWZ2;gr#vK@t--7^83FF%eo5wDGvzn$E1WT;Ezww(Rww2HI>in-I|4@|;Qk1X0Nz46AUv8TYxGR`N`R4YQ{{!=T2h=LJ+_wJ*0L1=jR96SVNH091otG_R)H8+HO$oz$|$ zc^>H8K3nnD3LHg0vH2OwrE0`inEEowzAjg|3C> zJ+#3l^fe!JKk6g=@Vb`OnvN7`syv$Qg{ALM2n$4=nRDJr0mBY{n##+;U4ISA!yxe7Fs zp`lSYRGJNpWT6hytVWQM9m3>(kKR9`^Z=jSZBheV&afO**dNA9^wy)EfJW^etP@;@ z5={-n$qZbsjF**_EHGw+O$7A zEUZqG8{X{-V$cfwxLDpN8Az6cDJx#Vufi(U$RoTGvy^@^7vvplf{$3%vHPgSuYCJ0;Nqi4bl#YjO#C zb8)^Q{EZjlgg6<%kO4`Lm&gWe{{A>&eh!u{Ih1b~b=-=BEjKV;oi~|389$0JV0};x zDAC#$$Px7CQItw@%4y}3b-qgb^sGvEUvr4OtG8x;@dl7(OPBJ>S58&xs|%wl4THl|umXMs|58>Mp%hJ2z3veuy3bV-+} zeL6=+Osji`x6>Z#Vv+_6aly83)R?TY*bYUK6w>La++hRkt=@E;sST0oaI_~yqw~!N zCnu)3JnW|e5HAf=LW;AiiGcFb2w=OdeXHgw^K^wJTixVKqu`BhaqCPzV%wz#eKkYy z1>jRh<}|PsDoX?XXA#y%nbTFxs~gB!puExOY&+kFHl#Tz8$MvKh^AfE>$fr|*@KrG zlJ07K*L*{bS5Err(b5I{v=t*R#=-4Nio|(wq|fI!?qNIqqqn7R=ImpsJ3+n!-tJ}g zmex*)Q)Ujm+)P#QVGlrm+)N(*l9GJxrb1HBbq-4tvThJ^u|@owmxt2v2ynt0rFCMxNy0p!aj(Ky8^^8sd=-SZ$rvR zIB0dxGso2q$J%D3W4d*}RgW_()pB&w_X)(23x~I5hvHSY=K2Fj4wIXjPM_wYcB?n zSmxw5F?oh())#7{Q_<e*@)4q3G5Q*pBw`m7qicV?%G4t6^mGVIdFf2?P}CB;F!+-}7_w96{vsxB|! zjri?mnH;w_*3_v3g`hV&2YGo9>)3HI2PfzGPx+Nym0jx(mnBA}D&_qEL*SBV_sypN z;{|3AduRGzTZ#C_yyd?NCCcKls@YDV{f2VKRR01`pMWng=j8c6XbOp9;waOW4^!H$5xv~^(6{l5d~s!FBO<%nOf*&5 z<(9KHNF8;Muk6n%YhNg++rQniA~0w~efPj@GR=~a!tD#Btl%b_jg?T2|E&M)|I-|O zpO;wEBV4riAKu9g7(@;hkG%8tkC#(U8+=S&zqQ02O=y@Il&<%l)$!vH`=tCr%>0kd zl+<~*;YK@X85xRoOO2o*IIw!JHRXGEy;tMe^P!_*qhbFS)BKu)QK^**`OqE9VhBIw zcq<00HllT2#In6hly0LxswQ!E(4;I)=ICb7W^kQXfs2jz*!hC>lZxY~wl;TSL<|z~ zYKiW_|D8Mh|75BKgQu~&Y2%deYNzqst=b3IYPUoy9Fa}{CoORz zu+nDL2(->jm(n=|>Ri5>ZGj<)}fpi~s-1Cx3z1AWNyt zm^cO_`X9Q@H>BKMGam;GtU=PHD(t1oU>Y~6mFCdVvvt+~yr6jZErdtaEC&pLVG|D; zw>BygxgMCve_Nh-HK;tPl4WZ)SlLyKAba*9y+*D<9IfNKJbPDN}hY{vsCg3yG3GnA8UV+;UTN^tY$1#yXN~&dl zmGnue1dZMvz5Ty8d8IrE75%PdUJkz&7yWwU2UB|a2jk($ooJl@xiu|U@pe9F^bW9c zG5Gv^Wq+)HSl#l=#Xj_bR)c)7+#FxyDcVJT=XM}Jf z#E||it-Cee3`YDfP)od!Jlb~?!GD8Fg3M00Wa%`*SW;QUt5bpU*^F>$qVB=5o3quh z{ZZx1Rs7YRiSlEGqOnsG>y68pQBk<4uHQJgocC)4BV|ayp}=3V9cbUWrhfq&^AlFv z*Z62(`qDNw_9aL#|SgS_ywb#u2wl*FRr8wsACVj@L zVa#9hsCcxO|D$s zaE$*9BV?|uPcAnCLy;FE#u4}HsQ*IwZ}SC2xYp=2{2Fz!En#&3uVF*#XymZ@s`sGCdhG?-5{Bq}_wwIZ$lfT?~n0LPb@oQ45j#>sNiqHuV9R<(| z3p%87ggSB=BO_mYUub_8Ud+Z#dy=Hmh2>z#1)fvf_y#AuH zPtbTxD1xk!6zpZ*V-JqBu-bYgEstW5Y=u1Bl@=K=U`5r|G)+uDSeR{3_{xZ^_eA4OGM6Lv}7Q9bZr7k z6bPvNs9KJiXzm7}fumCd2A8{LNEI<@v<6_c4@a%gpf)ck29nkm(YhGjo5f4hDzL%l zXb;oz=NNA*=;c~NPnBJTpSAYDVgA=)^>|ohP=|}NImkc@`gl$=d8TlbU(o)98_m-O*rfmUdZO1rT{(O71mAFpDuE z7wA-n+!gDbIED2XaIDPri34&AIR){^q&tEea8v#zZ<|cOCCJ!FV+6ND5f;{y8Ad*p zZh!^x`s4`E$1hxMtam08=w@ga z@fkdJSw0D9%1kk-_KC#ldxbCq}?(2cvc=__O~xL%?{ys3 z`z76ia;64|diS9W8hjj;?07e9z8A^$d6C1ON{zkT-HUi-k+*IKzMHDMU%oNOw*Lf_ z!%8QBO!MgbW)F_jF9OFdpUwENY97_g?za@PhIS2Q^w(TQ4%Q;lnDp)o*&fEou>{`sXJ-Rw&n?rIm@+#0aaTr<{JeastT#Vvf_v+aX$^4kW zc6Tt_V|FVB#=j(THPx;9Tf(pMwb#6!e*rj>t;J2VLP96z-eU9FX;6j&g#YawYM=|X z^?M>cA1w}A2hZoJ_%Vm024Wv7J>PI5oW?rA#9fh}DnaH29|IX$*#b4a`M2La|4F(Y zWH;bL&~%&9aMSBqRsF8LmJI`brz4*Ql=vQ1d+TS+VwAHgt}VZtu*))tEXV}Wj&;c` zP#-|m!m=-$nf7d`L4tZ-uIu0$uB3HI`59oLN1T4-6&$p^8-(rdh71*p&&c;9nKWUT zAb1&{)HpjzZirWh+*(xXU1%->T*8Ou02;Eu?xX$*b-oEgq~McKdk}t4XV2NJlP(^vI<|E6vyK|O$f#fHVlm>97AW61WvN`PJJl%4 zy=g6)oIo7LdACbNuim{b||X%($=HQ z3IbWodRh2ib6O?gGXqA9s>$`~1b>b0&$>Jx>x{(^vPPD4`AmoLT?R%>SAbzLFt4i+ zYCrTy*=1orpJ^+e1wD*pnEfg_z?B^PpNkc{}T(a zkcU8gd!fSOW-lSW3&>W~?;sxE4Qz@H^y(#-N0*hr_&!`PtNcFtpsWgeLk{?wmkSu0 z`^T~RsSAwR5SsxN=E=?ds>_xepRGj*5BeGR`Y` zL(uB{)i>I`Y)@&WWzEr308<6S+MISx7hUxdRr}7k}S58_>rdL7z6sSl3l`Hv_ z6t*ZPS+ORz({?UZH!-{fYxMdlK7i<**K42cnPPi5ltx&oQd=8}Z8S$H>Q)J^(#PUr zeA?@8@D2X5kajaY^a5ACAez<{EigF{Q395YUw*1H#tsrK&129cOmCt$ua7@Pb#xLN zAZGFMW6eB$ZVm&P{^9giXQNn76_wZyBlH?0=!Ot{s3guA#MNxZe{5fz}KD`T!COx z)B;hvt07TOx1TXrn=lW`h2+aCHg+Iq`m>PJCW|yPudDD)>mkl(?TmV;=>*U`-ELbD zxV?T->8PDNe=4V85yyyyZ~8?^RV_G1*S;t%>o_}{*5%DkXfK@POO5)D(k4bcxR_;9 zeB#T3%E>hRUnhM(L1a`HPGD|n~Ua-QP%3?3H%X)**alLqw(_W4Rmu# zD^_+cEnW*(-*YARsUyr7Yw6YdrQW!l{Z#6$JDm{qLDxo}Pj1P}%VS^Xvf8THcVnI8 zhzV7sM`xpb*{7jI&l`$vUe~=kB<_q{7=M)5_9#EwZ;E!n4|EE*3|M(1neZ6cFk7cz zkxELo@FLIWpkBSVK{ME_6brlon)JwsAq^dVw((KCrEbQ3IMC(us1M0CYpB%)zx6lR zbSHBERr!u7jpJR8U)|gW7z?I3MQlF}+@!7YsZM!lf4td!jqxBW$D`yl5-vJ;##{RW za=?bj?vv!&Fywq83wX$kkJKtQ&_Dymoxk?xt%8~@mkgaZ^6HkuwmCnxsGkFK^ZFji z2juosxT_y`Dq2^j$U|y+?~>id^9_tQ-?Ne}n1~Wb@xu_SYoeiJf<5%McIr;%B|v0u2~yQ_skW{` z!>DFS1vzt#u6##!mwpM!l6J5QVWIxo?rvx|shjQYtHkLoubpU-F&svUlUp(<8g_wt zbI|I1tCHc|(CM!LHxNA*);9;uoJY{lwd=fxhUnFibRVT#+A9MbZ)Z4j} z1Y0NlZ_tL%F07ts*{1SGa*3>cI$H^+`&;g#6#|$IQt7g3mfp=$3UaW|grysAguPCX z``+2I*;9_0@6U`X1gz$`r&_j{y;&xR_jKnvheM~u3r|BVPR}lxokK-Zx`oGubH%It zl+fu9Qiink1pNBc#`EMLJ*sklgFuq-C{w*&`=QR!iHd9K-V+ zE=;!7w+5%3aOA{faL*FZ>1P}GcxE_`3(LG9-4r(iW z`at|Fa$Ct5OyFH1pm=NokBrCrq?&q>PkB@XZmzEy3pWN)MYP@Qn(WO;Io%C5AqbHd zflhBBY>yD&hc&>xp5ac@5|FuAd0p0%bP$NL2A-tufzQPyhiLuiWuTnz_9ANsHnAafD{qvfU)V{eudd;Ev1b@~`Vg(>HGG zZvSp+(m0E3T>Yfp@B`Y$#7qM*3qEz}TNKgi0r_p0#;#E{G*cYTbV12@I45)gWe+wd zZLO?sZBwr$Z@*|Ms9z{h9i<)+4;U|2zX~+LyOinbYgxE8lww{%#YEPmLjqn~)a|cZ zdcP&Spyjs2|Dd}*9bbjqj4(#6ZbXY-LTfz}^4s&alA(}#npwjdHEWUq1N_X#Lv9TU zKe}uyi92g1ho6wbAYLj(m#S|;gm?(CBxI*zgXeS=N)6m)8) ztH_}Xs`BCrcO9Hvy1WbRV0OUjwpx4mmB*LM^@$uloEaBih3Kp@B`bY@V|#?UUY+sL z-;H_*K0jo{(=z|kb;=iAP{jC>2qr{p_>t0NS=T6rOm-Sh6bA>@Z<*E8db0U(;5MVY zdN%9TLMF=Ir9=tyXDrc#@dDES>dVV0{gjP=9o5JQCS)3z2`ZaH(FWXY<%ry_IsN$> z-r9S>IyHVUa(vK?ObRD#nG%q^q2x^-?fq1$iyF~@$k4Uw*hhJMO2qVpFckY+VWL=t zqNP2!r0eKiqR~d08DMvOl8ma5e-6I*CE0L_Mi8DNL?J}-ye8W6m$@tWt~+fzh5YA; z4taAyRi*vexye3vL$$-r#pSSR&xG}b_ze_c13aNtRMdnX zYsWPAv~%rO`}^A$GiMY8Pl-Ko52wpmyjTGLQwwHul@N)bDn<6t5FTpl;MGlZyQud~ zD85ll(l_|u-|#!Ot_B9#Ppf#KEm%J{xA(efF1^d2DB1^ zGY14&Huq?nXf3G6YnG&}fb}+LMpCqh!O-rzJs?V)D(NzB&Im5%UxtFmq&JQM1h?Xi z&{!jL)NnecH4gS6D|#*vm8b6XOeR#%4c#P#{y{B=%!bG(Z!J%nw54_^5#%?S$;Q*s zEg~TtxCpE_IaUi7c)`)fD5()M9hMH_vL*)b7b%q3q?-9E)M6FP)h)@f)+WlG8r_kO z9)1~7()BQ=f8D2i&t$V*-CEoRQY3GTVmL-~?|197u!j;&MB?I3{C=6q(t#V$yperw z$>(N%lRq zct9^?SxVjy4Aj-3*Y!#E8Ih1pr{ayGCy^DvEpm#9prKu9xkG*|)y2G)`tstDYQc?) zhD`6;gWiwG7^Xy@NPRL@nCu&kWhw5-&FVwUAwtXQ?BNW~U+}*hBQQxc8 zNTxPSY(^*H-X%`_tSar#-tSzLlX!lLmQe&=6dU$%CWdB&Sv=WTt(hG|kz99c^)5Bb z&T&`Sdlb;hg7aG8rqi1800Ln9CBU3r>i8mYF2Y_I*m%1)2IrREMG$_P&1_`2$Wyje zD3K}&uBeBZOF11Or&`myrU6O%6oPPp0DBG_u=cT5P9L=J^WDDve9a)~=puN=1G2&_ znp|IQk4{9SiC1({+kxz8zJ>TqkvC*{-Vs?HioTNfMbQ29&4i0=h;)4UWGvdwF=ux44ypvX6n2phlY6ekPsEgr1FrM!Hoph$$`Q zTG}U{(3C+z_ZeY2UpnZY@V?7KXJnXTgg`XAIpQaVsuazNrArM?Twl|ub;7>$-S!Xo zP#F1pRs&YXmN!wfKc&6?x5D_Yx4zaYUL&1u@bxqC_7*x7!-|*?tB^~+A7&gmEYIXN ziLHdf^|Dp~a;MJ|d+KrgnX4v8iWI()B6#(PT>%1X1d1|IZ1SbLxPfy*@U)AE$ZN4g z(Jf&W=Eyq=rqk;7uO_Y|$j+Hlnc3_%f>9LB$fQ`BdHvP?!TXPeBbUq?zISfV(yG0o zO2lnd@KMH>%fxF-@OgBrBf7swp0^9<;>AI)yiefD!Wy@gw`b!55I4(9J2_{Ff6kyd z?WmckL)`)(JX(SZP_td>CeV@oBa@{s6Qo4@B{Sgemj@Me zn2kUvd34Flj|^LcCs3>lLIUEFhvLK6tt_@I%)zYYWL<=#rCj)r#(x*QZw}}6EYG61 zm6F1y_}4&JzGw>P{^M(Bq_<6y*LsrMooHFP-rN8Ddv(qT$-{> zX`O&f#LJLr{-cu0+sKmnF0}5JzAo~sdi#hZc^Z}#k-RfjHuSw&UEls_nKyol;0E3^ z^uEX1h*_vl!?togwbfk}p#Qhlktj%-qy*)Ue;g~k{l2`*S9O(~08V*cVx2E$(nQJD76f|K&5E$ed zRCd-)sDfgXF1BozgGb8&3}n(Qsu;iexg}&|F4aF`Sb|vXe}C2QR&Zc;QzKVB89g=b zL1Z^JvNCn1A4}tj)Aui518C~>Cn;9mRzNAI3a*y&M8YMCs%|n>J^R5+#WiL))hFl3 za@K?yJkoe1d$M-v8{9vo^gJyY1c{Cnukv`&QA@$8!-td4bPo-`{y8?T@_1GG;`0*= z+?}*1DB7EZ@7z5cI_#=p+johd^y`7$zrmtq3h#GR-`NRaG#1*#;xBq|=@)gw-^ZKe z*W4;Il;5ixO@gWpTW1_p4SB?mJ6QTvBPlBV1j|V32_qcn0yR^wt_05j^jW^=mIqUx{yi6-EInp4* zuZ(%uAOCj3y*4#-oAcz=x+i>bZsffA&5C<~NCVi{%ijJ+A)4qZ27agah_P$_IVEE~ z`Wd;DW5fQh=N5gGL!x07%wZ}#I^)fH{as-cteD}4~!+2qCRQl+f4$~?W!l)m>^@P@N^W2#iv55;Sp zh%kI>dPB19*y}UD6+P90u0K`kTKe6ZaUB}G?v_yvv>kqN$?>}q=KG~3vnu3Y`v2Xtnf7rHJ*{OS(lvC+qf_tKZu9WFD6$m z_h#yQ4BCqi*`_gUi$38q3d_-}Z<4Nh@KZcA+%$!foscL9{inUNx5Esa_EN$>8S%h2 ztcR0?i`A&rfAw^ssN7mjG^0 zdoXk6wEGOUh}sg;@d!lsMwZ2=H%w>9u0QO9j7}X;0hP>zvgujAd`^qq(jPL1c_aDj zClVxT2Ch$a2#QzNS1cQrkG=Dvz=atsPeEer_0ZOT3{u8*Zx1`j_A4&04mZj|kTIQV z^gqknN{KP((^iAtt&>#oH%@`$!WLegye za6-V6NF>bjHXk9G?&}T%@DN|xev|Lpzkjl3U*Mv z+?X_w2(*Lp+HNofD?g)EDXseqNh0wrhScRvXZZgJ*?yp=jCX`-A{YbT=e=KRzT!fAm7Uz5{t*6IA57k@L!t%7du zo^|?%1bpj~iD?cSt8?W0PR{$b6=?-E<5|d@DP+yfjDKw}3e}MwjWybg$oaNMg=WUlyqdnmU3}1!SrOuVm z3wivnKeGVSEfL-*tYe#KP8yy8P=1%SOwUYscwbX1;|lELC*7Wh;5I$OdYaO{-(%h# zx%A^>>JI-uR|evgZ8SOpT=v->RSwwS;uDa4b%zenFhh?|L!{<;W@?)l9VOrU%LuVh z9RT0^WMOxP2Jm_27=Z;BjlRUeshh3m4s&DP+UBwLvCk204obbEQ-jbwe@Dd5ED|Rc z0m@}NjwVGJ9XC;e$v&3U4GwY-o_+E5V7;k4$dJ`^_S$VDsoOYC2QK(yUXR(5LpplS zb$xew3wLF3kbbbqRMzO~J~1(r*$s~^TK1-WpB)!r)>lp}Rvj-|td@AM?|V2#;=!8- zrbj*&Rs6^?-2GEdU!r7>wvOA6%LkqM4;+?QTyuzC6YBFOZFBiE`Q-5-t@R|t3hVE$ z3h50-vlXANfNQHrFReYpg3hb|4!ZgCx`mOdrK*MDHOcnlmRmQf7A{C!S6p`NjcYHD z-Qme!YvgrSNfJC1zI_60@3PGD(o(or+w;;DWtTx}zEeNdb%82q)+*bBH@|DOWUHfW zjWG6XqAn&Hfp7 z-Ovt5{kQ!|{?mt|^DFVuT$dQ1n}5m$H#q6gkJ5j<{5~NmyPH_8L6X|68(-VD27L zV-ivKv*64>8E5utzhx3QVURjMF1oT8s+@58l$f>_&UFzM*R=n-hy21{zW7BoI?uiS zvY(8$(5(>vEw$@;yMRJLjzo@VwugSbpxcw zr*IOk_0(-+Ag`q}w@uMQeNvZFEe$1xyO(~gZ$&WZwN>G%N7`G@jQkUNy1waLgeLe} zu6fn#9gf-(wtjg<_grqbP29Wk%KI*g)c9H3Oy?cCfpEC<^IJ`IS=7X>BMrxBm zRw8!#P+&RFS=!mN-)QOmuKPPc46~PT@NG%qu+>`_hVxHVXz5hLSgBHZ=@Av*H4azJH!i_5Xw@u5C1vCD=7J4bJ8-xPiY+?Ek)Td(XSBpw;Q&pnWw zqO2-Ya9c4rD`@F&xl(!ap--)Np>ge0bJdft7+1L`nw>K~9ZJZI5FkYLS?t`a#2>pY z9oP4G*jg*H6Y@U1-+yvaaLq`^pVTVKhNS-CLWZu7dgB5~>? z&QLZT?Vg$Wq^qm_>j%$(YuN#;OTbO9KCP|x=tVCqp*wy!R{l!>ZCH?6v910(d#${D zzY%5^TzMBZ`3j>m;c@-k&pS$PPR0y84%MMZ;jNVKvGE_OzDBIjdMgMh>rjvxSlwM$ zRtzGMv?d!0N}U1K!j@r>;S(uSE8Qa>SvwXNhI!{5&lpr29(6W&rPSRZE=>}zn4egr z<8d4GozLA8XeP9D-A}{9;9f-+L^EAAet!`Mz=ck8A@i%h$I|j9J@5lFy@OFelfwP) zJy>cOj~h|3_JQ;r6iES`;{Vr|TgNGHRM0wG5n=D2+uoKieT{!2Q>g+CVkfF&n3LN~ zn~2DztwK?|LN?-2*U%H;G{{S^&HfR(n&=t|yREpcr)d1{81#93gdy@=7dod%0>}@HS>FP*4Ghcbz9$-KYgYl!}Fq~UMrpP&X> z$P@k{;{NC1J#P|NlP=WTzM&28pNiHmXTys4tUd9CbIB29MPocBeC@`zzmAsMZ=EKg zK5ox&=j!Q{^r%-+Ew2C%yl`@^U(en|zX7aCflmH4I{xl`f$W$wj z-RiVpxAbSFDNQTdT!kg4+G;~WTze`+;cw<2?xcREx5p+Q>O6!wM0hXVXfB7~y)Ob6 zVq!>uq{EfzSrv)XS_n?WkIP#p%e$Q*9ySjJ$;OH>|6Kt8 zhx_|I42A~O-$}Jyqic#3SiDx zRjJW^Y}Sf+@4QCH@hw}f-S^=)0fFb&B5G5xlpZ^^sOuF)-fOTh&ou(di{KViq0Z^D z@{M5g_h;}EO>b!t-(jYZBS(fgW({YE>{5Ig@#=^1=|0|H7Q(525}};Z_0xpd;iCRB zU;lVlzv9VqM<66Br$^l*lJ^mxn%x!qQ@wU3oXm8&xzVPl|3R8q8n~!J;ISgP3LwZQ zYqAF7vxL9(XlVby@EGcw>kFHyo8Ge##NQX3%+ANcpk~Zt6+D>ov|+?>H}T-syqEPw zg}2r8j}mga$V~UbvvW$0fyTG%?U1;X4U{kG|0+up;2g zG;xh=Oab(G1gAf}^dCrTL}ftvq0t8(kSz^?&Nio5YTxf~dQGp2%RU^?X{n){Ief~4 zl=GzZkH=KKG%58tqY@nON~#OuL3@eSWJ7n8=K~Nj=C!uz)kE#qWWiI|{szLXcfJaz zO+HR^$H!~5yMhkwi!V3)Cw_5F;Z*POojAm0(%yg4WHgjR88Xd12X}b3TxT5J1Hz%M zyz)V!Lmw8bEClVgh}o6oDACj&M|ey+vbnnGgo&{WD)zbe`slN&0j!ky!{cU0hu(N% zHMjV?$9KP(Vk|xyR?6OK=UoIIIyuto<2O;XdGzJQ-+Vs!KMcLKzxiCKD$Fa?$hc|r z4{&C9)THN~`(@sd$=CCquUB@`GqgNhuu5Sq1eSfV-3b{N>Dxp5WrhLkTQLytmNgkdQ`9HF-JYc8);z~g|u6x!HggP zAb2i95PG+oa-CHj)l_%M&J=HD$V#e-vDv@&2MC|P0!sMu4R}WZdvoZfG1J^7v{x!d zcRkUqGM8|17x-L8bkmJVw#q`kROW{9;m5hsrq!Ow*``yO2*tdo4bz!^IrOr_?K*x} zFeM!y(7fTsZKKAm95=*+^LQlX^vUB^+o#2gn0Z}0#5Bm6c_@8PJW&hzAmRi(A#pBY zi!6opo7r5d!sU7zWy+z>*SSRgxVn;HJ_9jK{%~lkmyKP#0;ynhDgp$3ex&5JEH=+A za&dOiWrhEdJ>!*)zvkj1==ok{u4`L;-HU`j27OjsQJH1UCP~SF!G^Nj#5$6K`8;`H zIIK5C3A(|$ygd`?^2*OFP>ygAWP<%N8)Q{}uD;kkXPBIu5B@rGuz|xDz!KWrnJ>Ya zqG>ISJ4UR~;3=={Hrp7}IbdTT>%xTc5oP4!QQ@WTb; zKE0H8h3~m9Y1o7YOHOIHQhDl!EPaW1?Ge3vMVU){MQ;ATaz!7W4SM!-AiI2W`O=9v zw00+NWolhp-xZ!TJaZfHY=pjb+GGHSw>3E%hU&svUHSCj%k&jv%cqNHdrIlMbqfal z?CBus${da~IovzMtqvVRVk`Y2TGay@E_wB>=7@0RDN7D>IQ^h?m&QqOZD6x!Z5OUS zw}Y?hH#%pE#=Q%FW$?*we&)G+@iDt?pQ9XqsX1TkN9v)MQkA(HAFQ2w?f{7KB)cRp znvJRESm4bI&}~eW2QPWa?DC>mGM9qo2k>oht2brwQ{JT>wNsybN6+SJI>K@F+`I!E z-_#F82V8v0GoJ*j{`g7<3Y3F27AbS^CLAe0j%WG#g?TXE#%X2dee|@pO@`p5Xmr4) zH|Et7S=x69-gv2B)lbB`ejykK)%o47bu5g&oIdJ|FS~CZiYEJ42kMs`cs35zzS5(# z{0>iaXf+*^FWL^<=&y~5%97FHE4FLwP=-fs%l%o04rGwC14_nPb(;N>4K4TlPPs3j z%M+_;+u+f6v4wbsENq0oeDGapwPpfN`%$)V*|asR?9Qgz)5->~XZNnghV{S7k|UmC z7Jg>W*x>~>5pU)z;N0)CdKHU+`_{iuka7g+-suPi)=8%jCm|$76S^{~2EyQsC;$yE zm=?OhP%vH%G@Ww7BZtf=7G89mHRc?%jjKKju{_;_6U_AHU-IYB7LwvoiifxOR1`-- zC%Sb^I+JWr8GSegr*+yYy`?611_gt4d@6^pGZJjV+w#2+{#J@c_Jrf}H#%tYcwRd0 z1#LQSgEw4PlY>X~r-8z2a}O<@m!NP1m-vy1t>Lu*iI`p6hJ+9R2Fpl-<(z zUS7P^D;RPtjRpY=bW6j8g1vM?;PXHC8X=!`?J2zMy>xEflR+m$H*&2!!50kq+?jGX z0yvWrpDy%F4z|}p!(0CDPr7ul5bw(!{9ti{$|SS4Qkj83A3#R3n0TZwzb1#W z-O#HK(J#?Ydv8EvI|G*b*iAnC;>qvP6wJkIQ}@BTe{=<>52ZVKw1Mnm(AO^I3*6|a z&9(3gE`0=$;5wXyueZVE3F?B&p33E!yo1yCd#=0%jRyZ@t-gZmneCc{v{-P$cs7A& z^Co#qSGmiinvM0T-qPN7`N*DE*<*jZ?DKy1(WdlQg1;7{`F6gnE*#o2!S9ezYnI6|ASm;gU|+K@3-nYMn$OSMsp1wJj0@8t{c=L?suZ`+#xS}ZnB8{3R` z`L$I1q77L%kXi(5yvU};L-@vJ^pvBc{kXI@`L%)Y`ed;~w(=@BhT+u$EnMXtDsExG z0=O}%!#RE3X%F7U>s}gd<@F2Nu7km3x7pF*h&DEsRyg7Z_S61r`{1HaH}_(3aRz%? zu+Nqb{nD8&;bn{P)e$`XkvJi|mF3nw{N{Oh>Ib6IinFhFvU{QV5e*VUfqj9#K@lUjBk371&n&{EkhIkh09K8@YVWkc;5bue{j*Z zH~?0hug)A)Pd@ZJoUjWG(TIO~_l$P_Adla$lgr`!;G&Z(Ez;FiHuGQ$qwLfRBKX^+On-DN-WogTuFb=BIo!T*=Giug9geekY3jvzv4`ZkLBzR)^`t@S@ zbV~MOej5_<(v^SV)kXz>GN$7JCj9EiT!0*WPkG}H_+aGoGi!|eF9C`S&w$_{Ncz9-0xYo3+9eUIAw8hGN-lJWf zd${yfPyDXKA+04ZY`DdtZ9vCgc`)W_eA;;4+80{Ntre-;);;B;5w8E1vxzmHaFXL* znmxsq>nhK{H0^f zi9Gt0Hi*oV#hLJvEnFMBWF(vZl0Me=7dBb&@fm*p3>KcX)cDh1ed1q?4|nYn|5D}2 z(!D-YUH8?{_@C+ePV}OMXZO;|ce6`BvSrV5MXESVizQos6mdF4y>kLsts*_f#g+5k+J^oB!z8|e7A-nev6JDC%J zrKeBeP>%8Hkt^KL1T^p^OUx+!4qsAupm64qvpmTvSV7;8&+M1XrKUAN|6s)u{wKwVMN%!h49PM9y zcEB8OYvsuu9NE~M4CQT|)5F0GF^e@}YrbI7s~EYhjrP&HEH032PeZX}nL)>GJ4?QvC2?sY@I!qQVX*NVNy~DRhgJ$-MZ}tq=9+tWh zH$+337=--r%P&27N6Qxq@XRm0ZO{MM4^CGyfM;*x8o9M$c027|J)oVxpph@3>sh-Z zGau@(Uws(a!HFmMx_(tWfo9`Ebdpb+%npNr?aL_H0*w1?;JI;H-TK-+AF@|=kA8Kg zKh~DN^#cV78)Jewy(IY5xyxsCc?KDVJ{5-JL?WF1R8q!^2cvNUPF$d^0m?6c{M*0# z`?ran#U2x`X?apIp<$tbekNw+OOwr)H0W;fDBq>c zu~Bf@2%YkReU`=tTy-Sn2Xt%GoDTTvFx5Xi6C{2?zLSn;ZB&~RXhj>J-b`|8EBV4_ zKMRR`$>5|so7Oh_v~M`k{`ddkKfd8Pq){Gx{B6LIr+SrdVoi4alIQ42;p5xqf%X(1 z*ysnl_%=8A5;>w7-|)b{dZEGg>X%(Q+<=yR2TAlT7P<5@U<7sO$Jc#_h4@sweG!BW z)U`z*`S{U2+*4*{@&rn71^8g$rSJ9;)t|BsAvD>HM|k)PnCL2aFMnl_XHx)-t4U9= zrRjL3ulw3%tM{{&Q@YwJA2!~u?ellOTpKB`XaA8laVkC%hoL8MbpS57+JiU@{Pye0 zDvy77Jzwej`NU=G54O(V%`3!7#u}UQjVJunVNd=nuC^%TUOdE(%8iNI7u@~-rEybR zk?#;}ZJ-psMMk(5rD_v+8^hpwQy>}PiQ(`UXPT>+XK0HyBg|=*_TE3L^K=b9IoiOs z$Zf2((2Tc5hh&N_ToXpRlDqgJn&9AgwwOqMdT(6rfTG2E`4*tyB#RVZcor|C7yo-{ z{7vY|##Zc9?2sN(@tZlM`i?hzmxU4dEpl6^WDD^F*}y~#E?dyU)goZDq+mL*C7$xE z9x6lgi=X`TjfeT3dhW|w;Fnjrpdp6GlbllX26H@RUE%7}>0}`RtWB5rgB4q{cZ(9? zDeJ!a@!WkWI69#1+5@P{@QRMP2EV~?4~&CbSUAaOfgd~_vT^XG<_7A?=0bRQ*4EXv zxicG9o;`a;D>>z*UunFq^n;IGwzzGJ^{L({vdCN8=Vul``E@S>h^Mp-m$|Twgf=ar zXCp#8fM-70<{{sAAWU1?I6eD$v` zGV9CH)t+lB$s29Y=Gt^?^CuYnBb_W_8Xw)Gi?@51!v>9C*0J=J=Bn&~zl~~hM>JjR zYwk=&bikR{dUn5G;7LZZ*Z$K3PcZ;{iIL^&!_cs0(Clo@1C5-W%Squ4SDAHZ0^(bo);cZ8yozfjg2;z%>$HYH*%w^ z9?_|fwa&`+=ydqGH$KC)m=4CmHU7yUzM>;t@Ad!UtF!-CH#TC)vxnu_!|$2U;}hDL z*pBTxP>@Weba{5=ci`Z8who)@UO2NkNlu&F%3bLR&$G1l-)9{y}hUVi3)DSubrY>|BV zZ+f-~w+9%&29LinA00mCd3n0GkwRbbE&5*M^DNG+{YDRb>+kyC)}{KLF}UbU?Ny+~ zreZR6og8EVFUFUj4`%b|fGbZb-<3?ty(P>i_8?05o7$K^>nEDYv$WZ{O>%W}uMh9A z51Xvb9D<{paVXs64i4YdQMvvHUI(4nncS`8W)FDgCv2==$S>s6!`ei?4#%3mP0Kc> z!K{6f1yAnoAT>(%HI?}~oINu%zy(V_b8I&4zGf<_hJ@0xx_}+Bh-U&n$GEerub-w z@AAt7%rPvb0E01((`q`F0{#rfA;9e*VuO&P{kCAsGE66h+ALJjeHvW{Wx#hrk%QML zEXeGl-tskJ(Wq6P#WwdfMzWZEal8!7=q{5S&dEuWO_~!6YSGzFR5?4@jK;^LQ{z^t zCSILlHqnUA(hSerFoI*9g}|=Qc*4uUk}0`+zL)MoW;9Bd2Dwa3*jHWZ)PnP$dG%~i z#wWZ|dC4l^^&IZjHTx>V%Vw^6#LMI(TFSyfgR`qIc7=QIigxsaL&Jp^I!@CEQ{(k) zAi=|F{OYJwIr_;cRi<9x^hv48;|b5bG+ri#p1b%G*;X!P(Q<-UXQgwLPmXAMcAt&t zSDy5hiymt`eANwF>Rg~69k9#WfdX)JfTzrr?8y>N^(wgFXX%Xp+B%)RRZ%_~e8uH( z4O*4O+hGxOJ87o;lqFrH4j_<4{}8Rpdp6OsP$bAwwzf(i0jR*y>28yJKB2GDN#h@_ zaOJIB@w}JPT^nI*`Zc(tVNqBfA0m&%2K|@34$s6(fF*F>0|oq1zr)ASw*X@UWDi}D zzd9si`skPOGT7nG$4yYAE^?y3`W9@q5T4HHgY(r3b&Re9Aj>aUsX>~&uIoU-Yi)h) zcin&YHq&Or3fd|^7T{XA6X2?!+F9w=nZCs2Ngu7h(PmAI+_$l|G_*&tfVPgG%Y@fk z3>{?D{%VKyZOikz|L%R}`&0*wTXYuZ8>2cX$N!D5zw(36-uxbnI1J2=Zw>|(hAQTOg26rp86GdTYYu98EF5a!;9vl6z#Qz zG1gnAWU%=JS3Aawo@k$a#>Nkg$?(8#Tc>-UIrawq^0k?upVGf1e|*FZ*weXqV+z2Te9q~criSBK2aA@$W3UcjLvHP03!S!5J@ z!n5Gy-bJtFpG}h${IdfE!8-I_TS40b3LlCWc^Y5fc=37lZTDvSNG<= z>IW`ASNAqDDu<6&ey}v%pL)x?uFD3`d?5b>XOl#K)*_`meYbwyrnvsbLbbVvg<6M~ zEZ%98J1(FJhcEBfDfM~sn?Lbc{;W;v&-5eu3k#6^-dx)nnZ7`KX^fIOY=pl4NZ)Kd zpm`YnF7);1_CBb>Wc*C^61#S)_0OqAKXDDx|y@dgM&x?K>W0G z{fsp@b61Pr>}z98p86=ZCEw~0jq)72uyNSbp{=aq4Hd4U3 zY?QH&O$pE0KwB^#psUVo@)#%ETmqXNN~3uz&92JweZ0K6%@2)FV785&n?i8m7?0VZ z%{}=mL%ghyg44b&{AV-qsJ?(c`V;u!*!Zz-!}i51$qVk$g}#`XJ-x{cRzD_9K5dLl z>Yp6=CYLyeE&B27;8MIn?A5|w8eCIwzh}P z^3;Bmi@`cDU~V1U>SXVN52@$ANZgka$m-&Q<_*PHWMU_6r`Ts{(l2|GOW7+23cy+u zQ$HOJW(SwPLVZ`~=>mto+7=nJ!_Fn%yMW#J>*m1vIE-brH|MD z)Su{IJhMynD}`@wNdIPBL?fEY>1K_&Jr?>|w`{!+zoiG?)#lnB&`7@ZL-IJV!+*rm zV8q$#ym??cMO#eHC)v{+&N$=XguWIG`IBkuT|H=<)?uuf@=tzff2f0h$rewv%8UQK z6weM0eC&aOXc|BH7yXT?>7o9#;p$Vn^bF>#1JC^40X%Pbqru-CM$oI zpP%;S==e$@3wUCM#R98G8AMa`rcOjz73~D}SZ`d8U6dper_~cL&hey>j)T zUvVHSI~be5Yfs?1&;pZx)aQwflRvti-GlXI744fJ=P&Wlrt^d7E-i7S`Ng;Xg#t{s zB0Mt~!8JaPZDGXcn*`IIl?f=oCVT@cxagHK2m@M}Vc_1>V*q%Z%O@aPK1GaEE8g)0po&u3RooIZFKJjhdS0f;^DJ%i#t`L&%G z{?UWqr&Rva@BZGKJn|}+c8{-4-+)g)8;j{u8m?!~uH$K9vbshyJY{PHRkr+Bj(C}vYL8^4bNR^yM{ujWHpYh9 zraU%pvpXE+@Yssn;Z>ge)rC}^(~snL`Y}1O16*+WB{2DQG{NCXALT9!O?VsF4Mt$W zWE=O%5S>?2cmh}Yl29vpX4B$!Q|I}<413FvN4(# zyqpx&Z;^9tLnaf|HZ;Na_K5%+e^&u~w&>KT#Vz%y4e$d&sZTGqVBo-4dZrhi+3Pw` z@WB?@#bxM*oz52a?AEIUC7OKXU+cEd8H& zSB&H`Pp|-LPFw%j+n;#BN5{SX68+w2!P_Dvnhre#mk#Q!PvCkpliXm{VX$a`H{TLV znPaqYj9=yKYXc*j(ce9p%_r!Nwgb8G4>w-&`8ym}i;BTXU34zSMKk!u4Sd;B%0JM> z15WKrENqdIec4(2V&m$lL+EhT6}}cw@Y8wuP+U;n2+mBJC1iz&8l(?iO}$zA^P2u3PDojt<~R=Pd{PJgUEZLBKeZ*<~st+)7m zIaxcDVGd4bmpV@-sc|cN-0T0-ed8;#^wzjdoRt|}2PC}NSDPXqzc5}{!^_Sq%cJ{^ zRcaH~ius|rzEcp4>(&7WW05t0B-cbyR>0+$j=Ypq7T+4nS3!$dGf^#{2mX_ zwKeyTu51U7om$ropFHjz9AQH*=q)|?9T*X_kco}N(&X$nAM*K3{KP-}*?O_H9sZ+_ zu)(9Av~zxBZHNrv#hVYZk=Q8vSp!GY#csid%dTj)d1&*JKZ_O87Y8hn!t>W{%%-*>RcrW|~Ip7H&-{v9Y|;Tc_F7q1PV%Iu{O(Kd z$r}y2i`6^8q`f*2$j7W{!DkBx+}N0GIpG_OHAofe-xkr-@z!BD#geQlA{z*%>rl$CtWF7C z!5POlAbGaQR>h(lj(eRJzB&WJmeYOk*x3cI1`R$ZPQH6-cC3@|oUY->b8^b&A)EOe zlDu%qDL^D!^_T3MI_RMj@P?B7lTWY~K6UaI%g{JfsRw4D*!cMxOW=Q zRRBN_aO7vp7S57SxqCrIi;R^?$*}hG?CTR~@dGEb&-yUXnT(LjVwrn`qf@rz%JwY~ z@Q+TjDZ?Wf__emfw&6)DUmK~v@E7*c{nFv{1|7DtC~vW@@}0d_4wO_tg<82u8y1xNDs zGEvf*3wrS77vK#9WDfS8Jt}{c!o78%K)yhN|FBDP(@{N=M<1H*+9)^+N^tXAyF*o*_A)UqcprL{i@Gi_V~2yp`W#&q3s#}YZuys=MGtFceR0hTwGw`1+2DUGF|&r z&)Rr>WT}49o7+C~<}Fyb+B&}4@V@bL`O2r|E8n;JXpAyRvM9kbAVgYxDQOFK9`9QLE1xsm7SyEo3`ZP77& zF+bb&rs2XT^Q{yYk2xO4U0aQnIj9|IbYyG}zUlEHM#yu}3nAhoiR}-SJlUWVJcQ2l|Y08#Ls! zVAelR$!P%)j(9E`(vc7HYa4pt&Bvu;H*r_7SGw1ZJg1Kh&0?tLMaf!<7df;EayV>k zOcLk0$5X85vUm*7)fYgtS9nsFITpFum<{)hj`VAcbMQlKuHK`euGEdVmR#wRPekK< zF8@lO_c`BJ8|Cx#w+X}N&OXh70&6+SgO_Sk*1Y&Qf91oqxi%X7%EN)P+FvPt`KuUE zn{?M}+uHo9gEH1o z)klZYz#5nJy=_dZTR6Vpq&#|3^+_lC)OWkz{(otaDIav8u~>i(rykZTtu^;WHZ+aJ zWU-;q3lQmfFRebdT%1TwW0SEx+gj_Uk9_p4UDf+52MXY*A2QKR{3ezw z#S7nffhVWf2#)#@v#LM(vn{##sq)q~>qFo-ZfK**wQ)FTz>%_>7%qObgKX5Zdf57} zoZRehEmz*cWS?kM4#s^n;w^Qxxt31y(ra-X0%XhI`M3Y>OZma_AG(=i;1&PqOC2uL zhV4;gOSWNWHZhNC{zeaD8a>%fDjo*w>gx>2x%~Ei`Jean;vZ-?7fsjl>vPI4jaGdQ z8eU+j{7kow!j+bv-tgqP*k3BI`}PRU$-;3+kR92yH}9?SvXl0QPks?Uo3n75=O#<* z*v%b+SKcPDevOT7u+m%mNltL~8TmrL@Y#MqdA8X#Ztaj?t7Cn#XZ?UVs&R6Y8>rw1WZL$vLf}!?pNBspxKM zP-9>nK=}fmIw49|N!$2z{9r7Cmv68|i(_vElQ&)Hz@c>tDgjUU1|klvvkNvnFeYuv zEmWpIyiS~Gur`g?Sv~6nIGhtWTj}t~uK{`Us7i;g(doQ6i^CK6MR#c=$GtS1>;n!T z`SO~i_L-jat{!ygY~@5MzR^RcI_XnCK99Ef(U9t}oa#hNy1}UOG0)8FBl6KaBUjhDhHp9(7y!}t$Qi`3^LWBV82f8lqZ|P!I$juMPGh6(rni~ z`&36d)nuyfrP_fuW@9;glSFL`KX~ddK4`$xHp!w6qo-^;>2Rb5J@gy|YQO}ae@poh z`#0Gl&(cnQsSOQ>NSriB*P?3+%jpsS#d~%Srwsvgla+i1J$)9RM@ODIQa1(?`6l1W z1ZV~@?IxadNuTc7%V5+e1=*d?JE#LMdp2E5^pZx0kG>ncXFVXOU`7L!KX0dw}GmxW@7cOCF$ zlVTH_4(u0>d$z#GrCh&R-OIz*q8He%Y$R2GWHkq5cfMx+fHzsuFpq`1cw{1Lb0iC! zkXqeY|w}ff60eR!Mi`( zmsrnN7PK$=@k5`KUA%k{4YFrH&)&S_U+hx5HJ8el!MLig#jWg0zv>^CJ-%v>^+oe; z?W=ak=au(mlIEan$Zw@SvkIrTNwcrMp~G-|m;c-J@TS<=p7cZcw|NqJ;y*aX4|$&V zfVahSeHNL{HlGaoKVL9FyO%!blkDI;gDcPTvjYWv3ca>lyX!!Kv8fGSZ5V$#oxJsl zbnv!ca93Pi**WM?)^#W@Jt+`mURZqn@3iv0WVDa?peyvxj)oTF}-DI>vT*t$W9-ej{Esfy4l4S*He-er#SByJc&J9qpzl_~ls~v^t~*xMv3ns!wZT>Zi6AjbuS<`NFf= zPY?FcW{n$s)+QYPQEz)t6)kR9haIR$Txped4u&K*o;pAVx zO^(v&oayS_=2`~|!i~53*g1i^Kg<7Yd(@aVbwI$%h(FFW>oua!Glr5z|xhn~Ca*J?NC zFOe%JTkQ|+)iao6={Xqr-JfZ&n|_S`4ps6K{(?p_h(+`d`etzi+q;aR;Q2VaXY=q% z>+^0MD4+v4FMyz946sgY9YVe0VLY(6l3%3X_CsR7cR5fH^U^A^X9}&ucy>~Ta!x~5 zX%wSW|2@eN=me8fQ%*iZSuoQvIk9*0R*pAAb6}knJaEcXmdo4Z;mS{T0S7sCwEYuC znP6E163CG){YsrWgCi&^51!yfL)ePLFZ~Qm8Zp^6-+6Ir*mAM3-|0k zhXt2C(6fO@9}U|gP;`(ef4$KdJCZq;A;_y%ubRUm3W|KcY<{gT(pUMzd6>WRh&a)$p8bmKx9n6VZ_hj@Vyg4quvV4gs*UO;>sJme%k*n+(vCEC#vg z)Y8XW?pGeRTpg07 zMGj@@M~*fgVTD4rWxAF403%S--$gH`Qw}rC_h?vcae71Npc%Rl6dqdp@ik zSRBx{_vG>F16B~Joo{@#FtrD5Nq*0z@wQp1t>f1xcbCp}f3>Ce_5r?`e~Jr?|2A~} zGf%v>?E+l!l6iqx&-maC0&+JFxi?wl=f=Yw`z#0r zV|aLL${fQS1dadvU;gV`J2l3=YRBj$yBLSe=-Oa1FJK3GQgV|`Y`i>La8=%7ZhC=> zk9fBQXTL^LRzKz5rj^)?jU1K~SCBurPmmC&j<2n z<>rR`%VxsD+kBJn%OMNz-oLx~yv4(C_@zF^Mu+jDjWvCbwqZ`&XNciw=X~47g>lOl z63qS0&wVk*<|&!l07Kh`5?a0F(^!!VQu6gV*}hPa{MrgRYeVw8tljCC8)w78pB=21 z;1_N@#gF<78v*k0(BIYX);>%9!&<-RFh(@a)FB0ZZ42Mg4Q~4pdC6LNX*_$DZVcc# zT94+x;oyU&wG(~0#oVQ>JbLkNT?yRQ7k=Ucef!4Xr+v{mX`VdyxqEbyp*(ct;j6C6 zVEiJ3x8k#-@!pG3#(Qt+^HKHLZ!OSECS&@u{=ZF!=##;{^)kGzE1a?}to2H=c(Wgz z*iSp~vO>B&OUavU-CK{P6WQ59J9S^4F>t>ifIgYpL@y>RfBBQq*8$Mu>+vlGYt4)9 zZN}MP5o`5Ub^4bs{qsKCS*Ot!`IEID2SCUnF7+Z)@gRJ3UB+E@q`!+T=#EadVH1Z> z;nAVG?+xXRpZT~=Ex2rK{XJgk0>59rWOIC7Y$DbE@h!fMrww*;cJPu+`aAME7~xr8 z2F5xz+u&sriEEo%U~(lJ`{03RwETV#E*NpHK8U@nF_RyT zb$$Gd3Fkp*xUXJ#f87HG>1VHk&iWC0?zoj4>3iD0dI?UQpD`?~;YN$y=%|f3n9=;%9Ndc*OB($@(*W|Q6pl*z#WC1VD7U5^J z_`{VKPUU#t;t#HT_t6OtpJ)2RbCHex*wnbXGO8nGbnOtH7)2fEPvkp{fG>H_OJ`|) z-mL=#$=aGNTIjOBd-Salv8VY1+O_>;lxP3Yms!5`FBHU-u~XD$o|Ch6lEE-47y%1s z?{k@O1uYb1T=1uJ)F3$azO`r@Xk}G(X|kzoHMCBI`khT70w3n%eg&UOyeBr z!y{ulD1UU38~nl)1fU^U6(l;@LI-f*b5zgdG+`hE$6empw6{^JzjfMROg#C8FG6Ia z^h7f_Jj*9bo8{_q_2{0g!G{a4ddmh*prKKN&n}e04jImiIlWKrGV9shVPer1Bi zI#m{sJ{NWAAKU8Z5s?Q^2AB zG(g4!F8(H(>aSn~E|{)pq21dqY;qka_)u5b=t@6*o0tq)80jEi{Z4I1Y9gymcr(s( zZDHGvcuYHRFba+8zWY++xJ4ChCzyCOW((x?G1_`CSNiGu(l28jH5Sp04~v(Lb7FfZ z*o|v8*o}4C0~jZ*wFQd>PV&1KTcNRS#$*w`xIk>)cnMCtV!RXY6&K4-9&v^?Vq6SQ zJ2DrsA>3xL#Xa)cEWum+PM+p7Xg<5*nd}xr$t`B{RT2Gzi#^EG*8<2`e}PAF0{-Sw z(F2nX#%L3L^yw$A+;aD5*#r|u&{Hf7cHgp(_A}j*&th7N@jQtAkfXrD9YuDJB=IcCokLSPg+aZvSdhK3$prE>xto3hFC%NYv>dxD+*&#mZ%SRk^G~cB++b3_Z#uRjHMy2QS0bf3x>QCPK37*O-hZiqt z{J_C8PFfG_P^PuIr4PbPFAm!HDmiWD(%(3qylho`)k{If z0So5Z3OMnhv6K%b6Pce~Y)fWz#S?UDESFl}5MPKXjQ#Xqx$%B2pSYus-zk@fPg3CsW`%FI@d}y%e!d73m^7ybiuqGppW(V;V-((MUK~7h) zUYGqWOZmtd>h|*>s#4`k7P4+#9HdkMq+fu+jJ1eqpvUb ztnW=O8;ShX3xDj9UC;W&D;dKDFaNY-b=2A|n}rkoQu%Gb(y{it<=F@Cx%7L<8Bcuh)vmNnb=%>sXe7hR*S$1Z9jb$GZL#uf@F=BQ zG{7xxn`>)N?HmmGT-pu)@=_UlYO{QZFY$kK7jlW8)7L>_{(c@(iLO*z+KfwrkHJ{>t4ayL*?t z8MN#>;2oTF?WYdWwFkL&;ycFld_DT`PkKu;xt5mluF5^{edVQiBFEC5 z4-`~MnE&&C@h{&5v{9B&n+gPU==(`FCzB}5>AjVZL89A9!h(VF4Q%iUhmENWF@OBMHg?GxcJZ{}*4`uGx27(=Qb-dn!0mBB16HS9@Z|-=5 zXz953YYLqao53M}bkNi&@bVcNL0)vv^pw$ao8^4qg!f#%xzBE;@{-lU8$Y41z<%Kc zXZe}m7*v)H1|*x&I!cXIJ|4>_Ipl?R(g{z#%X1qz=${L34!ZESbD&$kXk;UP8Qpk1 zO5qxiwG-_?rY3OhVBMR~zHc{6#>xJhzP*-0@9^+NKFY z!J9Jp@c1ZscEMJ;&+urQ+L^_OaQJnd?fKb0`gjwP?R^#(PO^Z-r!O0<{O+}P3(jD4 z_?71)*%HlHF7SMmEzztGge#wn^aIzRo4gJ^@F95Ncc6fdHYVUo)33Vh*@7^N2 zF`_p`ndWy2KXfYj2R_~=_HevO zX^b{T8JjJ>gOTFtbJyWKN*h0G!!}*$E@m;=g}eOSH_tGJ?uLa}l5F^I{1Cp>9E-dz zn_Mef`pYv;v#_(7ys6MdNpw;o0x22(F(46F^+>H;!8L-#?xbQZA`0w zaiN$$xt3pZf%LNgXG5P28~}(O`d}>D;3+0jCjaE^tvz$#-ZD&Xi`43`g)I0sn9QBL zsbS$zd2bLaZ!zpV;02b98+#>BX?E*59Xz{UIZ%LxSS-KTHWdBry3c)`L(Hul?@}?p zw#kmQuY9t(jI!p6r+vLa%m2s(rt;bgzfZ?2y~eIreIWk@-!BiWcZJV?wNw3$PvQq( zyYkkfvVM7>+?tJp>pM=Nqs-wg?cL#}cvv6f`@N78kIKzoeIDOowfvq;+Cp>N+5tY{ z=`$@Z;%ChvnUycx=Hvb7Kml6DfaaXZgC@Hf7kWxLVPXIPAOJ~3K~!6xOxi6u;nmNF zLk4=gjA5IPp2b=8wEI(D0i(60?|`E0YYiL2>Y%nzn= zG~8QH1Jecpn0Ry_Ub?PsXn5Yd#ozp3m0J@fzu3pPX_F;>>A`2!gLO2%er{4lo4<=w z9`*m~(mEs_;Ysl|PcgQ|UurysV@;R;{P^epFYiEsug8+j8gw>Y{9E@6#~0YWu|4?e z8LTxQJQ~}<$Uirr!>xXdZ{(tv*uc6jnCd+J#h$&zYx9RZ;z_W%cY-8@jK4^h;^-_TRUfPRysZDa_^vZ{n8F$du4@6Vy zOHcA`vceN{nfv97#$`URUn)^%gImAiB?EesZ}qQE(;cr{7o2!Chgv$7p(DQwt@65; znuC)|KP~QH2XT|KXpz-(eYyE;JcCj060@S$3y|QNC$mfQv+%T6St)5?R9nsdL)FAvt~60Xy$ zf}T!vsbGR}M#gcSC%7q0ai>284pqEHpfRWrjbtm|y`V(70kVZbaxt7!Xv**tTt-uw zQ{Xx)PV2xzGJ>)AWdKm7!;qJLf*fUN3hZ+@pJ>uB4IVbe+t93XbfSXny=hnz+1D@H zHm$Ux&^8J(rfpPO5yCuX(559S0xHT(f`EWBiVT5FDgr73A_6J`Y7|6l!#pKGNJ2<%{^vgb^S=BawKq(kOAIvHBA_JFH!?u{n7!5$*0Jce+dr~EKhmR09hfX;3=Z(q*q zxC`^{rd78E~x%Y%#ic?u{`Ox`_ zQMX5Beed#O$K%&$6}C3=mgB9sPseJ)Cf}Yb36HS=#h$;^N;Gwy9VW%}>Dp1JPy$5FkIY+?ji{+Jp5task=eajf zK3*z+6Isa9h><9&1Vf(lR?Z38%dG%v#w}Uil^-sOa30aXVQ%YQmp)LjH!r=eH~w?* zO7jR%AUB)Ew0FVm_SrlWSwVR+o7*~3#(*hPD~cdNv_NM1QHFD(xrm82hd%S1b^@`uMkOAD&n_1b}7nDrSXpTFmdewxmX_?mU6vgKzQ zS5^CVF^9RN@x%Mx3A4|QNRRLPv@g1zxT%A9cTRDjX}_mlrmyGJELdSALMk9C1@_KL zuB_w(56N1_vV)j;ImpY<*8z2BBBP24XXy36S)k;(g%(W~04sycM_P7YCDRmHW%rEFDJF6I z1_v_xW*+&g;xp~;Zkw&}{JeyK6Yv=<7=SC7cZtm{AULNZ= zE0%1q6SP~}S^hMXLdbl%klBu7pR_hEz1T@@>hW$>2P?)DL^)AfgAS>5?ejH*kK+g;`X z@oqcK>G@iHDZeu}govv6C?kmh>n9DKw6M@kIrIp8uWj5=Ri!_mUyu=f5TQKo{hQ3Z z*ZSH8>ioV065$N{Cfj;*X5`#)nwbx>#WS(_TZh9iXZXDMM3XKRCWj6D=@8}#^?};< z(m$T&po9ylJsHw72}Xmp@N?0Z)t+1fc0{JyyZ)(>$_o=e&MU$Ta;Xj$!*GO3;%y&Us2quR@E!-De)pkv`GXXvAuQ5GPTT~wG?o@g$u zIo@_GjBEYp^cPATA+?2DC z>30{g*sDyi#B7G8XhheIO}j!co8YDs{)K(FcndBNP?Q9|1oMeUTDSPUYt;!*)UN2b zpSB=fZ=E~Q)h9Qf_tx&AUWT{-?m5%zyAPs0Wh(f-tQkereA}Y*k4aB2OuUr$u&$`( zCmLwGrCN6ZAbSkrC$bEh&iCcjop9Xj)aN`K?*)$xMfT=7$o0vA5P$3E&5ZC7`WFyi zY3Jhoq^NTu;z+1@ZUl;iX%IkQzf-`t2OmAfCtkd5B(3_37-Z z4q#3Z*M0kfk8TKRGW+>Tqa%s3xfkhozm;w++|rRgpz9+%^WcT@gVbYY26G|5<4tZF zIy~I>?#vN2P0CxI?gLp$M`+9gUzI0HKNnj!=$&t{YmhLVub5^hIk1|v;x`KKcO5e{ zjd{{ymos(CLrM#_KkvrgTB(L>Le#U3X1n(_g2~%|e$jg*?fcX3$ACIYzr6Oh%UoU` zw3+*f-c;6`r~l&*V#&+EuNgghLE!tXoR&L3$13jrIDLD7=l|34=b-fj&AFSnp$p*6 zoIkeaKG`7PwRRJ81u~1$fK6tz>zN$v&(}?d8o|yjK|OC*T3)Lc*hf`%FB%}F+@eu! zOwUj91)O>2!2a62$eb4}+~yVOd>KjB^}rFcv6l<&4b4p!79~$@U(;X2V=9h{YDnk3 zZqJ~G^s7Dx9E0+PEa5qZG0h~Q0uXZ1k<(Rh;hHI9l^S!`zREL2 z{&7-HWbx)RjIskb;--&EwcPQrh<5tvQhLZ`7J1G2jIPN~%3+fhhl0+hYJ)cXcYjyv z&xSZ#I-rS1b8LHPe$#v|_VlgrKbpH+jVrI$WuRk@Rtx{mxrvdmkY!O8(wL~Ywe_12 z5^cB3wxu&$==vge?eO#N3y7bR?XnX(Z8@pZ!MgkuRQMO>83T`#36{TxoXssf^m3?{ zph~r?pR-;kXSiJTl9Mv8`gkk^azKF+mtA~W`dqk}xzoN~B>UVh zta#0Yx@~ki2id01ukp=m;+L`OrUG@loA%3`Z?af9r7lT6>HE@gCnh1D_c`%aW);I0 zqIWIfWYZ1(C;p@U!N!@>hl?w?3)KCr^l5*PLXPwMQ*GU+iywJS9)hlzyU3n+8RPzn zLB5e0+k#pm|D*B7#m#>e-Z54$}j0S7kPf-|A9mPvj+HxEjP6cyadP|>lzNv)0? zFVpz~oMJZTY$skOIgtoNE=zcU_i50lw8zxSq^CnKGC{uvvZn5DNZsEKYb>wKNi`7` zC3A{5mB~T;bAj-ov7PP{<9(vc7`B1G(eT6YcRg#*EruVOc?GtZdK#nvHk>hKukFP3 zG)CF{Gu-A`EFRzPQ~t;wY8wW zC9SCWFFBHkHFx!}t8z1><5}-K%JNY{bJO#h>z*MS{=f)iE)7y@30N`RR%Zc$=L@NiyDe4m7L7-zOlZi5hH_ly|1^;RW18cFp%klTYSaPUWo3_2verK}WhN5Z?{6wi70fyy1?Go%X?+HQ zN{)g9;G(8pHd>MYvXtU`Cqimq&mn2MdTUvBLB`F#w2o`&eqYW>en&D|01*tAZFy7Q zXRr7!q;7RAx@3nMKh<FvX+rreZ~uoxhpFHuI&+;L7D4BT*%N_s^?Ko#A!ldRo0;i zZ^_I*x?^QQJNKoW74#1R7W7tVxtx0?lYw;zy06In9ik1%)4l<@&k%P& z70vU4`WuGPlO<)FBfe3ZU~D6feJW-vEM)T1&?UqHs}U#Vy0n1-{JbNgvvNxh4LB5@ zHp;qGbI}d(xVYxP3S7dyd73*+d6EdR*fMxB9u!2x< z#=7iGbId`t?5B3${HjcBO%xHCTK3{5U9UFiQ=ZAC!=GQ({+9LlsgJCq^k0Z+!-)eE zHy~bDue`6MnLn|TPXUNK6oX?_kg^Bj8GAwDPCVb&>t=hfX6dar>dfzIVhes**P@{{ zO%A>eG2Itsq?7lhJ~cP*Mr`K%C&ag(MVOwG{3&bc^GQwDr9Npm+fV}FvsWgk=DsKF zl^04TI~BLrU-kHh%ZQ!dCQXcH{`!$1yROT-ICRpPeRR2@uDtVa-@q3);aLUp=ik_^ zpR_h7F+Q9x%&D!@%C5oKg9CP__XF&(uMRm7c#z92zhvNCUQNkto{FDnJ@-2n1lm|w zwEUXHlZq+O~U0}~hJ1ZT(R3CxMoSgGF@ zUo0;?(j?zNpd0F8E88>7zct<|9FlN7>DOxv%#tfhJGjFMvdOE%gX*WAs4GtvSW5q0 z+jIV_&5r80ppz3x)BhyzYWmctwk?iv#M|&qLVtIcVntU2mmk|;Y`!{t{@czwoixcj z_!DW1K6OUO)dDWVX{ctLuL1(3X&ZZ0|+_`*Tyw}CEY8FhV@L)eb)zaKLuoH z3uP`nSgN(APFGL_iy6|sn5c-W``sT--O)6Dm1lQCQ@sh_H-r&p4{e>nsXlk-@*`Fn z>;iv#`_moNa8|^pLknWMo%zHSg9fYhG57!6ziMe^9=AcOm`G8k8ymSCG;slpe(ynCqJsA1{CD9ox~cUtG`D z!#Rx%m8GLgPjigDmcKE1Lsk2U0)KZTo7F-^&RulZxtZ zhpZrGJBrnfZLO-gkJlXGlT(DGXJi3qk9q5Nu3&EVA~Gd4m~mbS+#R1hQJ2nXg!^w4 z*_Yx4zB|UrF9Ejt&%lh{)xpEESnnoHK%|p4e74%71tu4kamuVc@FOwvt4Ci-l_Axm zcdsZcbUumxUXj_SrBu9mPb|Px%8j=)4?U7=F~}eu?p$*z3`)AC{nXC&T<6>-bl4Qo zuilR6Ckbw-Je@5ZPK@}_!46jW#O}yo=tBqMA58B^;3vxTc=<)c@pRr^DMctSX5AWl0;p6MP7q`J z#R&x7uP%wD^D8P_EhV9fh`r4b@P7ss6>ED=M=6NY7o>;Z4gRj2Fkr=g!X?>52sx$3vc<;+cD4C<;%QUk^$7hp=I;+fONvIc zvq=Taa1X_-#!J%qF+dOEkl*0d8w9q+(jGYVZH{xEriM%Z=u~C(EZ*}3CSC-~c|btM z)U^A9%=VEu)tqgAKPk@z1Gb9~JESb(QK(pA4?{ zkJ$%Biuq;n0tevYiOjQV+n#>99yN_QN#2WNacXY^J_KeNu7Xq54xtc>dE8&rF&J6u zX2~6ES;2_M=AWPR%1!rX&E%^wr(LQ1KXSe>C!C$BzjZ9Pi`P#&25-fuuA!`~&4v=? zIbHg)KSoH2T#l`1_Tj8xQTAO#U8lsCxfEJ^zGkWX$IE48*Y8iR86t`Pf*@P1#~o#l zcuV^O{TA0CxodiBu@d&O2g@@lZ<7+txK$GYC{f8%u`qWj zd%RyHp8Pu9i~AZi;jE<+BR8QyRZt&BQ&V(PoO>V*#iRqPQh{o}DDDF!_@44-B-%;h zZ5{EXiCfpJ)oj8dAL63w`>tNOxL$BJ^F$=~iMS?g%bFTnl=SLh_{r)0KpLnn&i;z+ zjLKDfjIk2_A^i4a`Xd`^w{`-8S2;6AaowK$n0DnlRpTc;v+dmTTc!FbCsB`8lIOYa zMp@f#{PH6pb)ofy@B6Fb8S^N78sdBWJ7x|=AqSn1NeP(lHYpEne|h9BDO9xq+O|l^*l91tasVX10QAd?`xA8 zDBsgqpRo6&%qYYDUrEXn$o$IHg_IiGY8FuaHERJvnXsM=&$)6+SpX3HN^)HDsFxYI z9}z(VrJi)6*Eu-Q4pc7nT8eDb6*|E~w&0kmZn=n>0QSPr$A~`^y2ZyFX}lIY+NSS{ zgpy##KAD(bWFJM036xJzQO6Ru6nySxT1)srHqjD(5KUsT{anZRy))kerCFC63TzBT zy*jmzm(<-%J?aYrCM$RS8cw~RQ*mTi%0DOhicX|QC`Gb{e#%(8oZUUcehx=#LGAV9 zrm?pXn0i+y7AtpKtQo=qfqRuyRDd#2bpoeK+x^Y4>a)(TvCBkTw7fW2+g(#Mi8t@VZ+E zWcxX)6~D^0FUFCgi0rFFQQFJSTs?Zrv6|#J;yC+iV>hK&WKQXIUD;4)X`+K)yChlk zVD8-Knh2OwVW^|Ovka|ld3XP`#HSm$`$`kMFLWzDd6m-7SQRCw~HK)hw@|Nny$+W%J+)Y`a`}A?BTAS7(*yZBBLr||`%Tu1V z=&lq%auNOWUY%onE!4I^V*pPoD5Qd-TNoQHmDNn)>U5T<((f0o&+(NXI}ni#$Xg#% zI>9kdlfyRf?opdvKVUIKM-J&`&WS$N7}riKbP8;|QlZnX$|EK<$%v zk8caOZZUeAAPli}eTVKoO-*aC!{yKU5meVRtc-uHc6yN_L6M6LL<+1%^sA-v^J zt?OUW>>$=dy0J&9LDlN#7GHz0Jx8UF*9Gm4H>Bb06vFT|y4KcM-4=35Yj=JHr)QPZ zCKC}=l^hSzFt2+&VxXM49Yg~;)#-OSE1iD*U_v4J_qtCNlt6UI$DHoZAsO&(zi>b@ zt7e-qd?X0OzGxNYZVi>Bw9f{W?HT&`P}4nTn?gxFJsO*!=py+{*(f(0op+Gd8o#9R zx_i0Hz|A}=38Gx-*J`n~g{lJL*BD>08T9Ug9&Mo7r@Z!UzsKY| z#%TkXJI48|wy&38*tXi4;cp&@qfEYkh^4%WtTWqSHC7&>?D$2;4H#hCXq@XMj74}g zMY*Eu#=c6e9Y=*k)@gzkTmlnJE#{Zr5d$wWNpBNc9HMtpzmi;=&u)t)C!oyHgPrX|Zy-LZsV5ym9EzR8M&$ zX@6h=_on+DLT02OI;6Yxt_d-mzS@RR-DV)1X9luuc}x7B4LoAjyd~MOd8=b<1nvBm z2E-!k_S!4MS;IGOcMl(P9+jSp;=c+$Kj2NGS3lCQEa7C0R(|uSU;2k^^3&Ou7B;Kb zR}USOnPh-_yszloQ5;Nj6%C*L*5Qh*z$4xYhvX9*`3Uh2Vcd3f+p4bS)tEx_Bupl2 zKF=3zo$BlYH`TJ{$3GJd*V(^{K$e=%g z*p{+&fwt`7$MdVHae|N^4y@5X67wsJxtp zEg89;kTdisx%X80ey9aT`}NkHZ~SB8HtCo|xGp#&8#IHzrlRd0&D2_g4`U3Y#|-%s z)#Lg|>z;CsH!zL1U3Uo&aF24|`Y^eVOkZ_xGg0{^G64ai+A7<9W5dnaHe=Vr9f$Vk zh%&3!_!SrE3}@Qlj=ZdR=AjB^+ZUm_VsHp|FJo>t=8(FYuo7D>Cw?VoYuI&j>}btr zIUhcmKtab}M_=o(Ml8hnu*N1ftnTQgb(D_13Ion^d_00zikJ?h`IAuircJg%Y$m9+ ztm6&f3ah{i^hrx5n(CwF@Tmdcrg-FQGV=QdV36Dv>Hl(Zb(#;&J~yg1wJP~Z{xGE; zejE?zFtI18;y}(TEqp#bLz?>22D)V`$Hxpz)<|>SalEW(yy>YTyKWz zaCWvGvDu*iR^tdpALUBSGZ^!{PFA+T*g%$f_19d>VLNWVu11&z<#2!ULi|Bg`)Ui% z`e*J>yRq;6eD50jIzm$0))qOfm_QB4&~SoGwg1Us)3KJv4>d)@P0h=?pK z1=;;!)!c0x4|Xrurr?7H*e6om`A;>;)^`Inn7cMCr zj9>b~Oota*?N;z^qFh8);gX*0Dh$AwDpF^!kKdSr`D#>oZ|v2kri>X;!NJG8H|__h z@RwpzY&*eJjD^cd@+f)C>ob`<)5~Zgr2uOd+0iqI0lU?NqX{VZS!KX9S}zHV>90`* zY#=Q7cPW#Jp#uUib<;SmJKV;H$lTz}jg4utQ2f^M+2{eHBTOrqOwfU|5e?%TwU@D@ z1`{J~CfQv-PRwD^$Kn(A~I}ZaVrGaLDK`!R{>$}h%e=V z!T0rAvIC0x)$PK~8Pt&Vp z-KYo+Xs-zk${AokyXR%}F4SU3@PWJ9V~^T`%i<^tUHu8w>%EDIy1jTw@UrvM;P1C# zE0ZTp7LSc^S6X5Xrlvb0&}={8K0bNip6_ad`qpjD(E!zjg*G?{HB!uuKN^hb8oq3$ z#Gbk|=aft;HycgyjmQEoE~^C|cSQ{jc*Xz=dBnzozyO#`aJ$I>aeBOs47m3(^|ZpR zozyk1BliYZ83HL$WaM&7^M(;#C-7Z4fD)K%cR~LcrmT;sS)k@a`j{uxfvH$+n1{O; z^Bv2CRd*0KQR3eNz7LBZ>A7&dH|9}+eeAM3&#QL)CcpYyDk8!~N#XhoW^&+gGim@Y z%wC0@4X=hB<`$nrJP5+hxA zd+Uj`rp5Nw7Xue6dTb6?$7Ltz9fP<;e2*-O3zv_Wt`MqhMHCg&CwuIIBi@IrXCT!(SBbmE=IrnV{{bp_@s8#pPgrP1aDtz?P9aQPUO7y(IFs{z8vy2w*A8tH4bp#Pg zj8}w4541cVBqoAHtcSGkp(D`0-cxKlUp#6Jxs9VU{JgTJL#nL+YASa&UT7$FzvzK}#BOq2Va7c>k6dY*3Ow|k$CVUYMM0$BU*)?ad7lH>}=mqlv8 zH}gdQrMFjJ@0f#(L@BH0qrxAs71k>f7=wc@Rz;}c466x~h;joJhMVv7lfJEE@@Ui; zuc`X=jkVY{&ST z$~|_W?A*Cw%;3>JT< zJ$btr1`hacus{3;|p1-zc4+Eu$?71*2kmN`ohvw+=% zAn$~wC@o9_2iKF$r+TcYsAKrm64+1}L6DT#v2&R+bD6I{d)+Kohw!XLlntVWtcR;5 zk**bU7kMz}0dUM`e3VD~u6G|kX?Vl`)8~w&dA8#KwGv#-y-d;2aiK`GPX?-NzGW6q zx*Uq+Ix$xLz)dmBB5cLs-P}2Z@om(-AJUL#?*~efAWnkid2iN--Hu2`-N#$)6 znu9c_`%L?34jTdLm3D0g1{DC{MqNlniNE#sUzOWtZ#2dkTj!lLWHCq4$f+f&>eaQ8 zQH0j5`sjz!pmolBXeJ0C{%}lFhIRBt~M2)NgmIj2nUdhIE^%N#1=e zJ=erZoMPXK^@e;~*rp&!_TKIbx!n@69_=|K7kbcW6inh?lVxHkivq}VPO92GQ`7OF z)o3wnbiKFrm@Aq=mc=7aDEfqjpUn2ta0q0N-pDjiU$0dh745bjOQ=yZY4InlOBiOs2WDMdwWk6fw4`|y_-3i$j?k<=jEANIU*WpO^A`K{7gdq58yUxhj@XYzOrWS z@>;{zaErKf4OV(RSz@r|WG^C_d1U28H&EY@dW2H}mXqt*IF&7*z==?Wu~vzs*{YfK zxAx@mw#T2<%3Q^7YEX* zi!%~rEM)4t$yB33Qw6EicEBkzk+5`_Y`j@xGh7ewx=xOM1sEYL)RNcQyt_ES_SI?U zEth7l+0{-5ScV1bn&CFmYQ!1`#sio+(l5M0A?|JEu&CF>7fnH2!cALpBWKh`=2g}jifw9w+agvp#%;{&F9uP3x1Xdi z2awIA-Eiyx98Aj7d~1#wGUPnhD*{t>7YVBk_j4pTp_2Uofsu%pboU!-IqVcU&)+WS z(@DpU;sC8U4ctKAf(DedG945Ay=e`i))79*yx}`?5pb_!&KfkoLt^yvDwvOxBW!$T z0IY3+m0IxzIU=0bv7&(xE?9gev?Q5Zxi);nFD-hH`sR>QvIB>;wt}OlEH5&tTXozNl!agn0AgO-W-Hmy`nu7W@=Vz&ak$1Hf?>)Rw>N{YH?UGi;tecdp~ zhxKC}^4B^IM8I6tZGc!>bmRVk;=P*gKgRu&b1UbA+9ZHMpMT9(m8^XSef?lO*rtKG z59nWO32*xFVbS5;=SfO*$r=-fiHne&_J2BkvBIHcWdxY6gq;iTdaG_R=6|DJ(^M(U)pd*H z=uOt0?9Ci;_?x5czRzg36ETl8Aq!9w?CTjHiJba|(UF}Jxot0@cm3bn?f+-{{}=ti zc)10XBgq{Q?LxDWRpl&bHt$ng4upG8QOunfm+aM9z2z|OEIf&u+wp4*X$<*aA431T zp!=9jsL`fk9>Kr@y^K8ep4zO_VnMK&+OqS$7I8X+{y>*Wzl^Rd$X>7_yGJ*;3mnDt z(D$G!_pK}2tzsilu;W-PnG@WBD_8x?F31twQ!`655;Kb_GLfwDw_+c&tJ$N)(YdG& zoWF?d9;LY2DmmQo?;75U`-PEYj?QSB)RyOM55rA14jRl;fL?naT^e!-L4HxYpQV?7urZqYr`e^$byX=Ytfy@K}>m0NvVbWOGN%!|7osdl?=J12eP)mDeAuq#x_Z{gdj2Fgj;#&^11H*-LH;gg#T*3i6UhSX;(``f7cC5=+gctmR*Qk z+e4f%4@X50MB?ZyJ=zXx1aT&$YZqXL?&dzMJKsDoB~_ubE}ey>G{yfYK0&)*E4^2uTfKtBh`&O3ed3STwPfon zwl#CZo<(enD0!9!(s!swE|izvPmF(u+L9M7(}B2+N?c@uT_$3Kw^wc~q%VwXbcPQV z;-S>ozT@uZJsH_y)i|M~D5p{qfKwUJwa)|&c^*CVK@#i@i5U4(MgV$DPC5FIc=_%0 zG(&>?(Mm0Af%P_sERR8dZ53w7PyzH8pBEj+O!v-kgz@!$5bv2m#YBH(09Fhw$5nn+wN6;k=JAu1-1PE23lZApLOqNDnKQQWo+^g>49bi$QsBa4>y%Mu3Q&Z(}N>en}> z9h=y8$;Q<~Meltp&E9`J`)Pr9a6HoIel~0jh zj!Y-5Ae%*}0BUi6Q_)v_E#CUaUMXu~4GgX-TM2Jfm~?Nk0@fR}*8$#W7F{dbt@RWQ|aFEkrQ&3|D00{-{+KNVObZ_)kV= zXwAK&Q1#DxvaVRNoumZ;|JeK94`h=g)#R|RhYPI}&xdip3^Bp(%!CGja1!7SS)GPX zav|NRu?9}Q{R;Fpb_aFKt*90VNDfi-0e zUUYD!(^iQ+DmlYoH=}A2x`hGL+=#DyYz$Q^tH>f@h6|B@mXvQX=K-M7On~ z4NdWwRYt0YwVZnF=#QM$uor48fR*Ept48s8&>E!MMl2lJ7s8l|4YWa~`(j9Kinx}H zv1j&JA6I>oK3-W)fbqu7O9)OtYbUs9@bM*e@sA8NH=oURcH<7NHa?v~t=1||Naw>5Hj{fo+W@#Gylnz3;!4jZeIciudtjW6lT5h4m&29s^y#vL!N*G25jjDzhltLo0xkjG&3nJRJv}e=W_hQ8CdzD6#IQ5m)cNk? zm;}v{XUWPAOD6(Amz~z#P2y)PlvcKqmrB~pzSh_K2k=}F64Xj9l;?|CR^V?pa#GlvHvpV6O4IHSE%^M==>PX&bc}IBJ!lSx4-SxkQdCLt} zpe-pgz1Wz%!T3C@w)7HWL&t0r_lu$f_fB$#8;3o;OQwnAI~2^zQb5bg{6tCS@3uA2 zBBN3q^XO6f#@@B_a-s)qnY2b%_}FsC_!K8Sc&lH+ps`e0MOnzK;jzb9k3B-PG!BPu4(p?n7z(TMd`3c6HVSL79+mVa9_eUk%m@R@eAr+WVGK#dFx~ zwhgdY#%*lIW_+2WHqL+Iy8jZj?z-mA$(|7^b0jzd(tIB>zL{!Ar{|BVHJ0PJf+)^I-P32Dqa>o%Or(xG#g>Gz_>=1`&{;DUT` zDBEWmA?_H5t8=>I19vsm4@61lBAXx{!4V7i?Wpl>GNoIK*2-GQ7Afff*B?j+Q5Wo^ zk8p&4HUt@2KvsI7UTwgtS#=vFc27Cs)-!o>6E=+;5II;d?2q|b(16#CS!LZ)rky#r<{Hx3|cSjwcFVY zJgXePIyreiu@e2kIul(K%&Oa=j_(9duPfy&yaCe1Kg$g^NfdW;1y0qVHICn|<9WQQlx)o4dTb zHO}j>$!>3eEKRBqw*HJ3%C5J^%qo$5Gn1`6dNi8dGwnBa4vnoR1{GNhH~xS@8$J8% zI?0krciXe=38ACXzw~?VsQAMC#fhMTCY*SV*cf$rACwQ@6#bmN28{EKyy(T$GSzBqRjmq?qo zP-T1DMU36pu)dWk3S7x;Bi8I4kz9?+?T67jtY?N1PZKyF0NlyU(FV!8N1YXD$ox?+sx8C5uUJ0?L>#1sCw=I+?3ydU~E2x;wNK}vm7a!f+bd=bGcHTqz zVMRX4#=@}IuI)I0=rEv)v0T~#14#Or)cBj6l9AA#PU|GS(-KYrcR91x5f=1Ed&QcP z*EP8D?*Pt)QFn`NeQ9%(HYk0KE8QPNAK?l+p`c*N=LKc1aUqSF%cn7?11lHrToCI&S6BS{AqQ#3eKUdT8FNX@Q=8-$$-iE2}XUesttgzoSU%2wiWU+h~(uoVG@tAh=4{9?QHGa=93`0ForUTAqkpE=~VR}MXWqqDU6jU+=2LbZ3Q zh(wFdi^OKM$xypwf|uSAZE0+G+`PS>4#EOz2)Ns$8NjQ`5y!f+6=d}9QtPV4J>Xr5 zJdl2P8C$f~YGe!r6`fvd5A?hW+PLl9r{M@#@WN^a`SeO4SR?2;;&izDm^f(Lmm zJ!?WHVVs8?Hw!oO`tk53kU zNgfLk{*h{{rxO=a6fT(af^esU8ePd%`YODW5drbXgThKJZxN0~h_P$p_lb7UuGpj; zH+0E=qi*Oge@6Y@13IJyMy6I~D~~Mm7=RsiqBKyQdQ4t;-tqhU*KOK?q2^>1z$?f% zQ`7?-JDueJ#y<>rpV252H+ON@lkjH{l7iR6Vfl_|R?#ue?Qz1?qH30nAoVObw(OYzdStZty#UyL^e%_{*VG0PvAv}N&GPn`ili&t2fFPpQYHCx zP3Zlao+~bZV@s=zD(YtJo>}K1Fqvsl>7j_22RiT;NILkTf-tB+6H4rnv15*Ih~%Qz zjYaPc6E$-!#ii%Z&fh1L5Vswg3QHg7V75x5e#V+xi5GliEH?LNO9n*3;4)BDf?aOL zdEzH;(~P5e`Qmn>u1DJ*)2UmdphFsK0h1MtcSheId9e6<(lwq<*(q0GHYng5=~&e9 z{Emm;690njRY~%NM(s+wO`UZtwxo~``(O*7(BF!-D zEk(`BJ0oLu*^A)$p`=Blp)w$R{-U(|f=49^@ayKI=MZ*Y8}X2Zy53}}^zYGwzczdS zFxmePLM_np{66}9D18GCmxPIWmZug$s!RT|zAD_j^xR-K2{+RLmw6(mK0loy@($J$ zQRw1F+M?xvl6mN*j#HsL{_I;iys(K-47}10`Qwse4eOv;YLU~6TLn&KBeR#LrX|J) zvdIPkQ6#VW8to+9-#wFy?gQSX@*Y^yTK8TUV?P9+su5hJ@M@%k z384>NkAYUOGLI=KvflawdEnpPep)XpGL5=+BA)#;b@XXG__Dl;8MQW_`6Ve;XB~68yO7>-vqD6*eXQuKgpAssC!q|;6 zc3B5Wl668gvW!sF$2w$R24gH)#uhVnV{94Am@#G;e4XFxH`A z58k{5zDQg6Z;c1HC$3+XGMF7Tc&9z7^?@W(Gil8-p5k4rYAYvz^<1HkhZN^8%=TM) zdotI)4sd*K-a~6Q2X36kXoGvi-#FG2z719%c;B2h_hPHv-H$74A|zc|Q|&PUVhMM& zWNa9#x=_>!|M=U+?KLKA^m%TF6|7Lmhi4JKGE%ik=6mAULw37%U7^rhe7$VJL6ox!DAEO_D1^}; z;P4qn<1Cb7GJW`srJx!MRT|nQ^SU}|h``Fg2=nm1sL#bJl$oXit$u(&IP`;A>wQ>u zh(840mMq>e3)-FrQG8B->-2I5e)i4XY&ATy85P*cci_{+9^64reTeP*XO2(zB;@CS zQ~EoydJq6uoPP*eh=1mb4?!WljX%4y+&p>1dFEx! zfg+})HZErBFO57cX}VY5qu;oF%WB0jqf@tAQ&U@IE>68|l=ZS?@1QDE+L2th#2u+O zXhY4tmR6ovmnL_CNdGxBE30<+Zvppv!r4P?E|FZpRA5^AeiNT{R?FNw|IglkORr8D zx5b5SLCw9^q8bAkhDG5+aB;$B@>_jiGh@dUY@@pWTma6y2=^Im57XkF922-z`6Xqr zIgidf|FAFv(${0UaE9wY$RnAOBvs}dYKz?0p>t+1celuP^HAq0nqDf{;ZIr<6e2Zh zE8<2jKd;P%5-uR6WR#7erE6s^C zrrgfT!DsRj!saZ3{s=kFi%IJL-8QW6urm9zRkuIr?jKu$c@rpQ=vV6^U=^zLI7|mT zoH)rjTjE6bWnIH}9z1PdE$T)o#i0sFpT9iw@`*;7V^+H#3R1;%QLHO)l}<-!BlT*- zhD$n|DggnPn^~_9G7_d(JykqR9IptnNvg(E1=2slPsL?vwNT}sZ}AjMWE6rRgG@7Q z%n7LTczUxLnG$`Aw>2y0e&5IMgR^IC0%Z3IJ~pOWs*AQRae!;yzLM4BIe9|F_ke*- z*q;G&`>)OSZY*xd-}jE{QDk479E){*fSl@v3bX>U5O1K1_PK#hY7(KV{1?K!ozT9nWz?2q+RFyw2i6jfBm%?v ztsbe;Y7z-Ai)9#THn{$M)?AQUH3tAQ(Xv4X5UD?N`($))aImPn{MHgJ5`C}Zx#_Pj z0mPB*4>p_)3w%tRSljlX@8O0L?(>ocuDk3FFz1qPot{AqK#%1x3kPDhw&UCv8O+QF zq_ypLunh}o^3#JX7|HkR_!+*1nbFypsZjq^aJxXc9Y-6DF23j>2{3{jS`o{aJtoa zmjL#Qpo)5B_Pj8TVQs}j(G{jz(aH>8PpG|PSb2&v;_LP){0?=+Mly)^5~0A_#pgFn znU{}P8-%l83{HhL9e3-N@&0a>UaNn^wsBej?^5-?3*2V;8&Jj zm>{ODE}hvc7OgFM>F_51nP=Bu3eOqU(~R8tI**(uA1e|2BHQLyc&q1_j^ORa?Sb~` z!7q)m?;`UE?V$50^?xz`suhdP7Z%Qy-9Z=ff1{~aclLDwRH@5_r66#jUoD%OY67|h z9GQN8$^EF2b>vU2ftg$Y?lWDsaub~8zFkp1STZWrME};+etLW2>vEg~UaL|(d9Vuk zl@D*3HnjOOFio1+rfxe;b!K*Nt*^&K?s=&QwKo@+{Howye;QUgb@yyz3y>XnsW2R= zL;R<%1fxqU!<{Dt_{|uQ?V&Z(_OGvAg5II4yNj*)Dgh+#%8?=aODOqG^I4HcZ}{F;%^5VvMm^nbrtkx_V^`wSbtV zzhOK19MG|BT9Jb;lt9{-Yro5M>T^HVjcfeuA@Abhq4pi!vD<;(BUcuSSE;yvo?he& zaw+dR_T7M$POTkXSZ<3Y*$9J!-02dw!{=t3^$^G)a+$1n$;LPjIeC-$`h`q6_KU!; zWi`ji@p;VAVI}4-p$oMPq+oJ3$<|Ia@0Wxl{Fe^uj#|JMJ`34#^`KHJ$~U7(rs3;W zmsAi^u*1r`9D}s^$OP_L?aiyFlF)H+??KK@vNJ!=+EAvL6t`ILKF)33;W$ zu)0M?MzeCrot=wr6;xu>f=FeV1iAsEFZoRM?A09shC`jkPP79B+B5TI_jQn>6#|;< zh#=j>UTdA&UU}X2d^PHG&3(F_Z=Nl$^|g#o>*2dNmwZBoH#)K^)Ce*{OiW}0sWV@? z;E$!JUtMa=au}@;e-+5rwHJeDgi89v{YVS&BNZS z_@l^PbjVS5Vc+F3c-ON2_?*G2My>|gBr&m(+F6<>mtp~>%eMemhGXRt9x6TEQ@-YO>4p*XxX#hM?jRsQS& zqNd7KlT@?yi7YPc)1G=IU}(cocFZ@1zgA#PMgHK4m zmi;^`j?|BA|2%(a%XOydUqx8UH`6-IjkXLm2pK6b6D&>y z@}Q2oL(U)<3>#dCK0fc^`;{w+ZMjPt^ZUXi%D9B^ zs=}iK#*wNcRLw0yfX5ZkfHB$)KkQPs;H={39G;-6Nb+t?r_?6KQZOJ z&wBb8ez%MP!J+-Lg00YZYrN7*F)=^yb5^TCN$0^sRQJrmD(LkgQ11=$PZmb3tViJ5 z$~Fer_+LLU64G14yP2@oTefJjmzP%9c*c~|>y}YPw)$DbZD{@A3BNvQbr0)->ucX= z2u;LY_V~LUjf9+CNf~p~s;UA(qBpI;qMzy<;fzkbi?H?IFvL3LAJ7D~L@SaIZ5vNF zW10Ck6Aa;X`ut(ON532hde2cGF`?q>ui7PL;Y-La?C%E73$^fMZ>VDNB6NDDN^G49 zG%)#9(&C763I}g@$OJi-{38pNSbL&Z5a-KIf+%;A+mmsgNRm=G(YSHZ5a*xQgCKGn zYVITHCAzC< zgWt$z{D2WLFbBD?UclT}@I7&>e#W%tc!xZD^KhaT&Y*GjG=S}*Hr4tq@ORyOa1a&K z#LiIJ{u9Xk2a?{}tk)A({c>m2b^bbT2LC)l^(tfQ@z@tXZ9%+mF!#GDojVtUZ=0T- zM9yO{ak=jF#rjWzHrSH6149GbcP9^*2ksWe+OWPa+*W1OzX!m=$=}15HtUyzjC7TJ z0w=oo0_j7xql04Tz2saD?ZN|YJb2BL_M+ykZI*Z(N^16VyhaTO)9`W)YlkuNNieBF znw)mTX#hHIIhtD6S9JmKymyL-rRyfGolI@3D;y$$b9YPmpVf}bl~z-dYHcCRlA4d9 z^bu2@LcVEHGp`gFjDIT4e3lp+5F!ZE8xWgQ3gg$%?FRES^BA}Z&SL^CGUUcK_ewOX z{-l8})c?PVerM#OJox$sj@;=SmL^Q3kTptTsrhC{yQX?m8r%Ta!fTp4lYigDQ81RUkc4dK8|Op zqO`wPYuufQi$e*g&{YgY!nYSoAuSL9>sr>J9D)|Rc@Y(eU29ltSOK3^F}HtmqJIY6 zqmnj#LOS|vN=1mt*O{3gkw}wKy-%)3Vn!fSub$gZea#=6x7x7k{?-3TQKGfc3pc~O z{aX`OhNaHt0#5;qv_ zH4sQd6d(WIG9RYk6Qj$ReXJ!1-7VWMsh7uKR$IEdiexZfxQz7qoO6@L{*^Ktv zrd6nA)8ROa08Fomdv(7q#h{!w=kKbtj?^W=f4mYW2`gEKdsDysQeN{y@*s7o690*%EaCwX0(d>1z$Vk^gDMO4>$z+#PiTXEf;B;g5D4jrRpFeg0bYGM%&D&>86DRk6ZLfD!|sGkPPEOW;kC-we!>X%_i}r z5!ZwL6*oO}TW=)N^ku)>Vl7KrMc!5Pa^%K>@888$QltXE*cDoq7n+sY5pc3P)6E4F z;(k9mjzzZvuNT^`uY8vo4{5{=Z=;=B+tSpgn@;h0GxBe*@bA`m`55jsH_zVrr8OqjWVUf@ zfa_6j534r-`4^wGQcA{JUS@TG$tV_5szhI035t6JY!L@ZEH&l6 zQ=J@4Da?2+2O~ygh5mCN2$9yKCyh|#G`L4g+)yWxD-KHNv$ICJI$^4BXLIdpcdptI zTbdQLqpK(_$_&4?+#_x|>MFLs&2%_D{{Dq!4k}#{KG$V~6ku}?f2}ga^GCZN+LXW% z1w5m}aAZ#LnY(Lo*CR`nc)-SJ|A8gS2d&E?bStC=b~;77IVKh|l6&FFuO{CB0kRZS zp1|GNq|i~SMQ%ZVTVQHc#|twu1`NHVVk=ok5zJ3T2!z3N{$sp_-(dKZp*WGquzKGWqM^IaM_I!@^H2} z?}?smd}zOFb>oG{|6allL|{+~G+^!eQJWMA$zAf!XDjWq-b;rLI4N zZk)g}uW*oCN7StPItx~=c2JioD`}=X8kW7|n>H}7h-bweU3(`_AenvdBSs8s*(ccA z81jLXrHa2rBF4|n-K&$-PdSs*;Nq~C`!I18LxFp$~v;!7cg17hig~q`C zr@BboZ3wuLYO<}NpUFo0lILo)q$7(aaueS#Y>gdM?Nc;RdlWX}O*Hv$MpIuejl3Tu${!(Mz zbU#SrI(Q%^s~&M9ct>5U;iN=`@A=uMNd17|$q#1SJQxL`A3!+&eXz+N)5P#Y$( zKwrHehzr9760Q%(!I1>mC@0w1c)9`SqKU(mUe~}yBfQlO1IYho0T8g=1%guQOTA5f zrLvc^Z%Vup*0!ZR`5`FruuEflU)lDLYQBj&GK6R8FH7d(P=d2&i2W8p-|ehE^|yz2 z&#}E)mvZY{TiJJ^?_;)Ja8Hb>X-#EtXCjNt$upBXS1j93J`2iesNae-41dIDzMKYX zV6>ve^1NR>c>jID$yK70rTlbBX~+(0-HSBQ&Rrwc?PYuF$xH4S?Gp%Y4Eqg3h;o*>RlFkbD9i9+9F>B~%x{Fc*+4@xavx*|() zl^(5|V1rIxf2%5O5%SDAS>>Tbeu8~`_w*afGk$hXcI>BjdOt8lV!xg^CfJ-^cOnid zb$5j}EryCVR_MhM>{(CqFg8x)d&g||IKF1GKAct{-3jOUp$w6osKxC;;= z{9av2 zgPalgni+8nE8ApOXTdof*=ccs|<(t}?L*DITDl?QG#zk?$E z$h(xwKJb?s-Z?P_Jk39>;7d0S@5E+G!=YtXE8h>cT-c1RTc%3eJ?C*8Zmaz4gl%x6 zsw|yTBoiY!+s9zqmVW=uML?^91kmu^4{%{M+1jOMfUWxjjXd`OyPW0`vH2smO?jFJ zQ1RCau8s$9Ltn~w-3`40^^-u1cK+8RvKSlUQ+@;l20tICJjOc51(}`Jtgsa%A3Doq z>3sO$CKF#u&>EbNYdejBt9~OG>dWecUi@KmIwT$7J<*;X&ml^E*0MZD%!7?YVlub? z>9GJ!ZujG#o#N9|vCf@ko;zu%$B9jZ=%FS`g0YCJ&;V1^c50uY4e<3`p||t61Z>s4 zUMeJEc3yn;b74??ytjS$z`P~?UDR-M62w_5tMq`18FJj`YhPBgRgL!+tu1yZyxk^j zd-)k#0_Z0_{ma}ERH(>4$v!qL2$8T-YkfEwVCO-KR2V6m8NPpg_~LA<Bs2hr{X)33exq+$8Kh*c&C-kq$vFa zH!?P5y(~ouUb`~0@~RaF`j1{GE&;yY){ARPXY?cFrjMAa&w z%=*5pH!i}0+|!A$I+6$Yc`!WL2)NQYE9Dy_3#i_>72gpgYH!;3U-=(jNm(u;Yt}21 z6xiA?n+BTuy4MHqCLg+L^oVnlnptwO~gbC($Z>nSTIOB9}WxqL(WkD#ljnY53 z6%LQ5BtU^ErNb$DQqt$1kD>I=gZu2RuwF8o)5SazY*scOS#}l%7uk;-G!M+X?m6Pd z{W@0pzpEA$@z}Fsc&F=CI|8b}L49G}@^#c)p%D-hsnQvIVwO9^dm;VPepXXqy)&E$ z?Jm!+Y0dtzn-N5UY<722<;Neh8?Yz??tocKx7V%EA%#mYlQD zvC^?uhOOLP{mOBc&SsTyxzq>0Gv@llJzyS77H+hU6edKUwIr}?3xTKQ(<#J*-G=S4 zD0lby0;ZBU*HuPX9xT^{NGIpkA?g~c8gxSaUPN9EFXYFz+YfZMhaF7;s!s(^Mn~F$ z%PL@xfL3*Sj>5Ge{+E2D@tZ?k*2@WC`RU7hMP}5mHmhZS!g3ou67McR#>gFpdK+)h z+$o7~YUaz=;Xc6bU9LUC$FCGU;njiODL|$EAznt+7jH&mgdN<$cA4{aIN}-|<4bpp zD?c?m6x?mf4_EbH$>=@E3pPRHuvAxisv+mAOtIy_;Vz>Kq3(rocQ|VM15=Y^%Rg;l zugLVOfFUvVQt+aiw zi~QXdh4P93AJg0IZDU_TMj8#bD|4tM(N~akeY5FT=9v(uZp8Mu#J7+rS^s1)`HCIC z4{LDnmnr+5xCh)Q3w&DFdW*!Ck4j562-e6OdbfH}-*zq;vqd=w_Y%kN;>VNCM%SN( z#JjC2kq@WKIL*we_9hhIrS~3x2@r2fuMqs45wM2YC+3a&`5RWLAA!O;rkA7O;Iral z?AGD;ysku_(;kq31pe8`Ooy1yQ=PB;mxCU9loFrCAl_^jYn7W~W4?od8qv+)4UDtY zd&C#H_tFt3xC#~-1_LEMk_8h2@C`qPEzDArED&N|Mp9EQoHQP?nES!BAm8`B8+y4i zK|QGBvqQJW4s!W~ZudSGR|pD>b=Gz*c{2q++9Qp|<0q5jQKFu(8|oe_0W`_F^bzuC zh`_SI^NnIkOg?xbu#)*e&XYJ=r|a*WZpp~T5U78Azb<5`P7?2~)SN5{y6N8Nxfk_9 z(9Lr--=Oh>v%J+N;yS8ywXEM(V`x$fhoA~eXw4OiZBt$K)(x$>OC^{4&1A(%K0}Q{ zz)4keY;#W%_1b=?D0rK^9Ga}UO!*mpl{#StZ8qb`O`(WKZdIzhBiCyV@IZzEeUYwqIM@02~sPn;wJvwAb9-0+E+ z$w240h8fqmu)ai1LDg@z*#`nD+`C0_gj^sa1H+@9I6g=w$q}t2B&O;jyv4%C>kF!iw4GO|pnE(bO z3bI>jT-;`woBzZ zNS@#PfU-<&De;+5!Q0%Y3)s@U9*hI_d^AM(?_bjx}A6pESbBiQ}-_b=vTLMTU zC^4y;zqf?IlJMZ)cp#Pr%$)hbX!@%XQ8I6FV}+>kg=e|BKi*dqIKInJVRz%^#g>b> z-|N*sOG>r9cH`H_$^y5`i`g!1pS4@U^1Vw+I$2LuLg&dohdgQzJm$x&r&GH%KDKne z8dn;O$Xust@q zg%;v8cs&v_s3ct5f&jLyG4+`d@wc?YF3~6phogMluL z_VUgfnatxV2ZbI3PuRz@QBSrMBSW?BmqZ@fJts}7N4xX0cbBIt34!w%z#+Vj=a~oQCj>nl!$R&u6h7)>ZH;Q*R z`uB>^ukA`J5a*+fWhQfyy0WO}VEtzqZu_y=-lrv-%g=!y)qL)({+LUZ_A{}A%n@@k zQnp?~a^m?(7B79@O9!|kbvgc}0*2WJ*jd6XF;hD^Uc)WU`(K0(YG_GC8mmpDeBdU< zi&f1Fl6%D3&^0ZFd`Ja^hwGlN%byt?J-vA!h-LQ{>mC0HGgB3bT_%3O@KGZz+-362 zP$(@BCN$Q+F*xFc#rSrr+^%2xKCh__bq|7s-0MI`L1=n-4O>mII4RU_my6b@^gvuB z=;rz;V0}#c1GGP{8yn7OenTOty|Z9W)_5Ny%&MAEEUS;Qz&3l;#Xu zhcOJ48G}Zb$drkasfKe4;0JyS|4>P!^!a_Z#YRGEI!%wCo)fo{vi2I?4?fz=+xouOTgx!wQS-+Dx^y*AmpkmZ8i`0ybQD0RiY%kNOd(3Iyt_5E4xUsn z?~;H*81CYrbQgmCJLVvr@j@^;qNM54V>ze!VB2m<7}B}^ zF`Rx4PX9`Xv%@997B|`hH7u%0CORFQJ59l%S{T!9T+L?`C+}XLE1ft548ml#0Gj3HYM2{&uYq&z471uvyre z{-vDUg9R}m;;J;hgXzW)Hj~?Z4E*gwJ3(}0vTHFMSD95BPVk(PM{tCUl$#v+Y>In= zw!86V%KkIC%T}T8kPBF3A8AsoE{{(GOQX!E)w``#ejMAob=X-VYf!=I-F<1kuQy)~ z#)zJ|1z8tdBgJx5&SgmdBX$;KpfCsL3>cZH7-#HBV*ppTbndR}ePoGTke10dBTw*s zELlPhbTlcJkXc50!Q_uU3b&!pSVyI0gg`RDw2qL=Is;2cm1iFXJ%nQc5D1l+|28W3 z14c{sY})a4DI`lvPi>~Wq8b-v&G^2RSGKT(H6G#{_QSCqb9epa}6g0RszYPn2(=H z75eopML~CVs-!5go{=&o;$OEIF=)5nb&I(|nl^+aPoF{RNwe(oQG>H}$Ri2DmVQ61 zF46uQA2vLoiZoD$o$kq+?iLJxE`dMjG)(#0sWFv)jxpnTegtiw<7_Z8<8lz7T;r(w zRpM$0<0ucGYv#PDPd!eww?a44=gvRhLx&nChN3nK0RA0r?nDcG-@W4S!&JhBKvd^2 zs*Kv}2tgmTap?-ZClruElI%3KO4_0l(@=jRT5=tDn@7J4BE z-DW%7d_UUz9C|Y@p*ZZECL3hcEzzc?6ihrmbx=+;y{2uKLB5&_q1B7ow@q^ThqVGW zy0l)b9?bi8XrZ6}Hg{CNU{ZKz->H6q&bA#J56iXn5zk-myli^S+uDCpw;dJqgZ=tS zm41d*e!}4p1Mb?z;t;_6>dw}ajU2J_?3cqrjwUP9&D#;S>8iKUsyBgHg0CJt@1x@| zRKxPBj0Rge(j&qDgMC7!=Q`2>sdMyB)#nz^YeLMtT5qE^_4J`(OVaJqBFyNvdNM)~ zC9i;|3l}n+=(E3OBvvAFCw!y19ri%0nHQe>({m9TWq*Ldt*y3?2~vBm?R#lZhHu|c z+Ek=2Tzs>F?;-~nvv|iWeXzbTS&D5t+Vdh%=1iXNuRo1~6V36KrIA@Zp~d*kX zKwb%uSyJr+JVH)kJEb0h0vl_+oqjStI2-1CEm&}IF1-}oMEar+9ng7oz`F2 zW`@#|p~415`o-FC#@WW2eCEaLrP2@6`QpXO5|ZE>rFGRw3szh770DvdRXs6{u>RfW z)+q8Uy*K0ggw`vql1$S4aV_Nf>1uM2%Qpagn==~~6ByXLMF9K8M|v(C9$hR4_VKA5 z2#m}gK31^zB1B|*q`mi#^sico0>>kGPslA^J#$!4)%RK>Ftmu1r*4FPE3q3;WnPnYW_zBeKxlpAfn}{U^O-NQ7As^cqGd zVf|c}M=C=UD$wd^ARK&ymFA_* z!RJ@l)j4>tEU32vQ6vfytX-q+V$bS z*f56lw~6PpNnd5#3TJunc3&HLB)oJbzi*x3*{jK>ylB)^Tr>dJ#5^NIFTL)U2$(JR zrs$(oBRnTQO|jn*pN`X=%lN`ZHdC7#nWX5}g1&~E$2 z#CNvE%K+cJJuw8sbyCE0MTPtiBu!5vtH}p4>7y{g83YYB-m9lL?%A6%-kpfvhyJ>u zNl?=4@0{@#8i{W7xrhJyVisKs<{vY4ibg|es&$O$pKZD38O=@V3pEhXdY`TPHDS$# z)0+WcjUdI3#vIBg=^f6J68G}nty{ehKc~d!4e1;B-2hxnquRCpv({u%cmC&5aN;4> zx!H5E<@|SrES7HmHaz<3V@dbX34vGl+vH}FIT9fJVmB4l6{McEY~wL!(!g_ih9Q+icK^tPI!?O?kf1i*uPQzGIKT0CWT;{ zSnq~AJEkE>xrykcZRfQp1h-m;f9MClj6Bmoy)``lCM@o9-E>ZzX4dA^4>65&I~_k_ z?tWW!pHX#Dcy+aL zJRk30uTmiby8#H=9*%Ux|NGuVanH{Qp%Et$3?hpvNjYo;=FV>2608LoZJe;>jJEzu z=VB85>cjwB{X@jdV?%)tlp7_51wekR4$3y#oi)2@cg~biYtbT)fy`xuM5k(4(<=@e zDa*dvKSKN`(Ih`?7Hm^oaj{EwkA)au-P`+7r=a#%6@nwk335?!fW~MWoh~<=)XvDe zWE(F@{QPy$5K*Eyu*0`9cFr|C%(`G%ZB|WRRI{Fb)-(>wdtK0z)8^v%AaO>7FI_=< zIm)P-o(!8eloIeOyV`$%f9~b4(~ScDmkX~WYChqLbjd1{ORpx62oKpVWRJv{G;MnVzndR>eN&?yzgfpqW8HAi9qWP0wNk<#*7M$@@KD4VR2}EA_$r}f|4BDNI)?yLgaNQ zKgb|z0Z?jEI;50% z0=lnfkrp8HPb^=$9xDzF8!-k;M61xqV}^%-M6wVV$aOe8W&7N&4j~_cOa;QpvL4#+ zL5Xil8ZiDO8ZkYQ9-TeWC}11%Be1>E5!0Z=Yl}>Kuuc1x@UHQq&Zkhzv+-Y{Htz(&B{P^jIqkeIW<=O~rL z8)z>Z9`zB)tADzada@@=_7$k>Ad2yALhE&IA0nRfDxZ5zI{fEJ{6cTr8R^qFbrG4Y$N)>ED!LUt|9YG&{=6|e z9U(QyeT(M=c|`Mx-Xu2?Ak|;?1#wR!e>u_F_n`H|>%DSE2F!#1%LW40wvHA1`dMR~EMrh|a!+BbZ^yDU)FqXgy+)ZCm2PrzG>~<@I@zmN=ncO)jY!#B ziE@A>C4b?*`@BMZ0sdV{eZM-|C9)&J*4lDY%1MHl63?fF##n0Y-)pxV31_SNwDP9o zpGyf<1@<9_1qqG)zkEAd8bN!1V7nquNJrNhPY7-a=R(?`EOwTws4l-lf@{Ka>e+dZQvGvB1hZy!h&8XX%<=+e7In3W)OkA0sR$p zS{6U5CLpzLX_x_y!&&ZB_b>$}0$WoA_Y)q~Qf_RUKd49k!r!;>k5$VW^0JOroRrUk zmV?C*i8S12XEI;DBJFa=nqugg%~XKIGj!Tt;JaD{fS!j*ONHQW2l@IOTui}7g* z(DP00@8lbJf*?p%io}>(LGL8=Qt1L_%%_`^Z_7Ah8SV!sLBy9fAIV8D4~6=SjQ)GK z4)naEv|yAM>!CcjYCR^t&Noaq3lUZIpXPM5=oBAPZx$~&7nyPsmdOvWzg(x2$7qTGL zV2BBzF~r(8B6wj!oq+0@lr@40R{(Gz59`Tq4G;9pMM1;`wp-Qi0mFg#tj%jE&RaOo zID+qb#M63~|0%^E$DYTX$+c23R?t8lX;!`ST6=Beq*v&jLAiK%0O#=70Q7R!oP>m!ihS*LQm5tR2 zA}UetnuCxjJ2sq}BcKs>qC&~ntBzL{WoHXRw6p1|q6Dq4rC%tp7#Zm*|W zOs93s9kUVD3{UIUI(^mWWI8R;4fK)v^=tt(!sMWHoMj;ASYxYSs;R9GkM0tEBH-kw z>LoN7cU+)C=vQ?N|JXGZR8*3XLvCrVmPl^;U#Bkn#?4z1mO&qqrT$;9h<(;j-C--{bxeP6&#v^AK-YBLZI8Y;Z=uCXIk*^y zL~xkzuQ2ui;Ilsn=krw@r*B2Ms@N(6QHqub^oX5#i4RJ!#MSIL$Rp;;yk{QvlsF`S zwcU}arXTlY-e})@Av+u}4-G$%-p50?~C}qbr6fy zIGiSwR#(62T-)%A&wU5F9ATiwbql{}E2AL}*`QRPTf5n;Gb6^AAag@JL2U;Ta%LG6 zbJ^(+ve`4!|5Ue2Nbt%cYczl}7p2H*`J{xw6FUZZ%m!#ox`+7Y;?8 zO*!b^>{gBP07lSKVz!$%HKjT!;8L`q5+&Dmv2_2f8~B1)XDmQsWaC{t{$Z0z^b{v# z=~z26J2b_A$BlHH&SKQ<*#juRCRUYtq=^svWm}9!&|}~_KCHhDn+rO*UY;p5r)nMj z$}hfld`qbnp=y$+6?Yl`%VOJ2XX9chQbZsqVs-9y<4nO35_tsV7Xt_%+P+onU*dj1 zF-rHj8I~&{{C73SZ|(su?zTY z>eWm!OEGeCCa-K&G9Q>|UrvyN>JrbVcYJvDE@FzQ|Gd!|Kl&W)FpZ9z7>EojsDW@2^vG{*K1r#?} zO%!?DUe3<>uWVz(+1$M7IVRMkk~_@>PE9j&hRCe!WyRW>Q)=w-lrh6O$sndY6q+$B zuT3VaD2`GOm<(|ye!JR$nX#8Vzu$);I0{=(7>}pfp)0q33u`Y23#R)5bbTZ|NU>!? z=*k5V5<7gacCy?;c3?T`tS-I(j$&3q&9RVoi;%g`TV4dY@OnQnD_;6k|B`T(59S*F z@!PS@p^9qlk6FFKbe4d7@Zv+gS(mAx$Onwq2DXCsTitO}fejA{hUlQud7A~^xhNB# z-qXD&dZogG&{e1obkXN2ddMoWC`}Pz4u%F{=bpvywl>J?u_6AQA2tQ^mT#m}7)Nd^ zbua+EsCo1PkvhaHFIwZu=oZh+_j2=i>ia&Kp)koC`5S`3$^>reL-_h({dBKadBRsY znRtp1^dpEP!ZfG~{oK?02rHwBV0GTwDBVdiLdYzSX$urFMYO~EO3dw`w#P*9FL|TA zQS&m&TT|F9?zuSQg=_4L0M13AS6%pNGoSg}jqfnB+6v~qrSuQZ0vhh#d4%Ze*s7$9 zBjsA)UtBfvMsj$kE8!t5YiYK;@dzvwYn!f^+RvEVuN$KEf==2UPfrV5?p8Cb+`V&7 zW14Chj=Kfy!^Ee$5|*NVspaEU`f5)h7~bI*x0bpaWEtGp$3TXsVj`8_aX|96vK@|UuSfNImIe(djnh3dd193V|I9oSneY0%KAWw&E!lEnqI{oQlZYlrtDT^r6+VR<2FpL zNqLDoPF39Ht=M8Q<(}iO62g5ZcEHO3m`rGfoC*7(mZVy-kZ;JDP#Zv|Dy1pc+-?RS zwJK6{bt{=Bf{4c!y{`(nA965UE~*ZJSjP`E{2?Cc)drVzF!8VWG=uimQ=q!?i>Qgx1})i6(0N&+p?c=6q|3Po$-VInECbo znhPj!T2;YYI^Gy7)OJe>rXzb@^hk*8Fe5^_0`mp+>+6Kw1%uw4@()U`C&o}8W&zlO zGxI767Trth>w|_z=i8-Oa{LY4)frj32FFX&1XG$1IvWt0BVa)2G|rH-#mSCZ8Y*SAZf zvl#Q~Ajo>)JK%_$uQBe`l>s62zngj_wlt2m~3Q933Do95ZR7&U_LJLJoC`#xMLW{IC zLK51WXRY`7@_u+f&RVl()|{C+=j^@z`}gn5)r<;qo=MxE=(F`8ouQgqf~3Z(6Dl8% z_t=JeT^M!(Za81LI}Ek8)_$p}q9wtFyFw08ZOlV`>-WpQ*o z`0~bNjHo2pr*6{$KLw~cS^w#&Y({()Vewp^!s$v9_keA0dJOHDW|a=%b9n+G=2}Uq zSUyU>^f(Lyh|ah(_TMtNiQ9f%Je`^`m|@Ws)@;AZy^x2wH#BLGloq}XI<<5ZK@ypX zOG;JS@}u(}ML?TN^Et;#5fFJtB>e;-CDs8j-hlqxBW$(w#AcINlzenOcQ8FDqcr`k zpnrv*GHj}a;j1`eFD{%?V7gob@4-M)=Z2${5W_|XX68Sf24#4@1nmZxUurb;%UUm7 z+LN(}H#Cnp@=^V*1j65y8V4qX8ANSAtaFpzLK>YvgTvjkVQ&O<&R!J#DH|GxZvq@$ z1vsuhv1ZUssOT z5v~mA(}+J{kNvXktVjOERPIjdl@h98ljG$oftVMv@;++?fTM*bWIJx$O^!XwX+ufL zq1b_Au>BYIt5FeDaXgOZ1WhoL$lVUp;NFKl45JA2Qr*#e1zUvqVUzyGvd{qeYx$ZW z*2BIx8iqT+IAeaAuT*3u)Bh`5dqt(Q5ESKVMBn{u<@B_D0 zzF=fb`$aD<%J^b`j0N`jS*l|z_gQh`;v+JR#W5w^>O28XG2a$%Z-rZk?K5a;&N>J!t_r zfn9aWQeO-O=#e6btxeZN3Elred1m;z!sfimFKG$YstbKBfcTE1q>+?<4D0Y>-q|kvHg!1P1f4}(r&!(dLn!; zq-ia*33h!Qu&U*9!Eup@8{=L|CZ=2NI-VQ469Vo?$2+wbG{?uNL9%JNx!qPw<{$Ja zsS!`~_-is#D15b6as)V4=pBc$iEbCU`u6=j8(I3NFzKZ}^RxwKf73g~|K1>r-hhQ6S z52UxN=JN(S&+<$jr(tTA`To2viAPo-Al1a z0J%XU)xY87@a?xtQgfcY=^u3HL)P&m_+O1B6iVD&%rr+C23yJ9?U(C0QM(~*P`B`0 ze>V_QLAdWHlI$FD_>D0CZlqRp|D9`_+fd0}e!8WO*vU$_#oJeC*WrU(;g{%KrJ^I7 zUlpdFzfrIZ-quXgK7`Q`?l}Ce5RB=*bAU72x?}}oamCzJQo9u_3&3x^=1$z_c=pLe2_UK z@^FXvlMLAjQgl86tJi(nro5uQOhTl>{F`Bx6~A{%j-?ngZuw$~1J8Ru#-Qh{jN_eK zp$+92L8!XF?IIkIQcahWVPjs+xqKwiKUw=%WwVYM8DIwgtlSFl`{w+Rs%Rc_mIOPL=f!;lA^wcn-=Y6PWIEE~er=;D$xZ-9FW#^KJ<Buyy*DP#7{fUM8H&Q;V>Na>SKh0|I9;W>*-H%)>l@F+|5?2&%VheJr9lNRNBjH)?XuIUW z{w57SDYEig^g%Ie+sw%;FG0L7&sxcp)zed(kp_`+P|r5d zl+&wTic7R+)eY45RlY7cO7yxBH!wO-UfMT+JL>llu||v3i5~$rdR;Xxb}fFk2m4;y zuQgH;BscTOLZ=>C_<(R@Hc8YKay1CrEc$ZBQ`_jqT@iPk&f?t;_JrB&h1Lj5+VuKkyueAT+eY+tdcdSFsM^fH(x6)up<<(Tet zaT4o9|6=CG9A!%4ljqr?NRfgVAGTg{+Y!-w>#73=W-m!Hb~`2y)1Y;AK3}L~qKC&= ziEpydh>E`TQnUFe9>CMh<(p4aN^Ub~w!?-A8M8yyRlZc@M`1M#kRWPL)P5 z40{em-U{;S5ejv`xA%Po#Z@8f{2)&|Po{`wk3<3wGbIF_I$|zYM*Mi`4zL0&@AkC9 zQ+y674g|lm{%5~`V{dw*{K}S{VD)@A?nyehV|PGXiuABhG-Vv`{L;J|94_BEz32=^ z39KxAz290_bJk2pE{Y?Q80m3%N6A=Ilu6}MIOhh_d0ElVhW4N^hzj{~{6wj_S!vWu zZ*rK;Y8F2sd`3T<*tlYj!lgp3w+QO*g#|OVtEJJUMOgy(bAnY-ma$tbVuxc3K1?i5 z=^@N_4VbGHKu z8spWQ*7hA`fQn_YdbEXxWL*!3dfs1%n~=ecn}0!;RP|^#PZSD*Wx+69-Ma2pL<%(u zDbm6B+mGM?YR<P-nShAZ%&XC{3S36L>C1%YrZbX1(zhu8 z7$Jv0fvqcnR)ID70)f7i9_+Q7VOA- zgXgGfp>UA%+4)>M+|+)z6;UqsSa>02c`xW_2Je^;q-v&F+@O)!T=%6XreuEf^2Yp! zO1Je@3CHyu-e{ZS>+0QXm>Qm&kiU7z6#080q0xi@KhWTpKjU~XWFzxuR0rPEKc_&j z7|vC6za;N!5_-h840ad14cFy~w~NanljZqoW;F?MM$;TnAdlAjXzv{Y{2Gsw=>6Jw ziU2!^oO(Fq3i43nuc%B1Ehyf@G&lirKoF|Sx09lwbE)P!rv&I9>NCvT*C>4VO10#$ zZ=wIz@4#W#H@k@s@1fO+#i$##Yvc3s&UIsp#)d{_6_V|(`hVv?sW0x9*X|m39c2J_ zmOZfCW@EuG)~e>X_*20p9;5MiNmdV3^2r#Mx4grO1jZNVs+Z&{yBgXuYJbp(B<-v} z!=XxKn>|#u3zjPsQ~Ou>={}>PC5`D?nS9gU2GVT4?nM{y)F6e zEaK_jShn?K@AvEBRii<{YQSD|URQ_NZuWtJYdWv_a{t-IH~q)Xf0ODYbm;!fnFr06 z2AN9d8{bAc_PbDQQ*t41r%Tz%VJ ztKLGo#c-Z!iIcmhJ?qjZB5zIH8H_XwU|FSMU7KW*0u>TNDX*fKH%0Xgp^Y{d{@svz?0Ck7**;u=LGd1r1FiPgO8QQ`fS_RiRCIp`Hr>06~s zl9s~aNJ^S2H-69p-(>OY@X*b4hk%lahx0jUXBw7QpUJAgjP!H|T+#@DK8@q>du7cP|4~rNh=?wg; zu2hwRQ;X_!-;o(7r6fqLyU+H+7%$332`z6bK?TcS9TVsfM@9v^9{Ja;xZ~JUbNKp$T#IDu%NiQ zJUc+fSlH}ka>f&bC>xcL1A1HX#hUE4W4M(}cri-sDwQ2JyO3)Jjc!}FUk&5vgl~uV z!h24E+;y`0qbSp!Hp>s_-%ym*mhzP{_*c>D;(WgeVW*JH%FKEn_X$&_t+O=k{+Rwr z$NyeuA)>dX$WFlt1;iHc$S7T@p;~IYlHuG+sn11jhj;x@}{+yMh9^v&S{HiG${6w1<}> zvvCF6<=^_mrj9~jNY>q$nb>zjQWq zWkKlF%Xl>FU0l48O{Seehiw{vBnzewiT>OZ)ZP=~%5BI_o?xaLU0o3o(QiBTn931> z599P&V@WBewaYl-s7#n0qq2k1s=p!<)ldZNyF@1h)UgXJ9mYdd0E!z77r zyDa4(w~y$-6@rnd2aBO+d0v>A)V;0R{g-Ft)VB&cvGF&;{THL=p1ud&2oy1cuhUw6 z!=)6rCOT*j@(UXNWjTodP9W56QAncNlW9F>AfMDY$muV#An|@k=2zX}=4E}|4aNe% zAw#Ekt7_jsl-GP|id}EXwr>%;irJfGpgYF{))Rrn()}0vLtF;%4;|{4aQLd7Sz|q| z39R}J5Wz?^@XfctV!GjN$DXIy)|=t*0_Z2%sQS$t1la9<%Xy=%9yLd*v--NL#12{m z*k3@*5d`BIflHG5#H0jfZcT`TG@MHBa=zjyj?&ohqQzfOaG|CPCpQl8EabYG-(iNU z-GeS}GgRDef@GjgSBrQIuTqrj@l)R^t!S}@XZ?ns{qC~9l+1i8@O5A-we~z{U_J^| z(c3y$xRSHNKUE$DSz+ew=JJrI`W^a7$+q+bf&L9yf6R^|nmM zIfS?Wf|g%v8-}PnW-s7iRAe8{KANzSI%+ZPR(1L@s>uF}O86A@NjOKYjC~p>n$<|Q zZ(br(T^<+va6ZFCee}CO!8GEQ9>_ax9E>~hL?7)XDmJ-Z$z9ODhC?z{@N)md-kc=0?^MS1X&gDkVcNQVa-FS)W2| zSRdpA7DVf67U7|HzKvLlf6@5z6)?|QP;16CKn7R1DajjrTK@1nv?t_@nk0`*wiQV->9lZ&LNg)fD z2@XB0H$D3QW&VB=Er(HW6PpN?por-!>|k~0_zP3=7+T)UP8n=+hhzA;HaiBg(q#G6w#aUY$+bO6-nSX7FbOy2ifbqK-~q zee?fDxqn(Gtgrkp7r>>xxUIZR1GDV8b2L=O&`Gnz8wbW ziZB0wZJ^DW+f?8o=TlfBzFg-#yBWh-gvs;4m$!BjI@`ioDj>tvG>u%Kdzwb5*hh$= zr(RbU+C?*zKgeil0Wo|{bL}$ARJEB2mzq{nQNc#bT9562b7S8&Tru`~=N&Je#&y0E zf@U^{aqhSFJS#&-JdtyLl;qos3>K_dG*eA?BX+Z+y-{Fxr>+r4QuHXOtmgh;nh(Mm z6Eu7pOO>>oqnL;JhbY=L*a~QL_nRMw&iEOMY$2J{p>)>w##Y0Abg-h^{<}6VMI0Zzebun4z40YJ{On#6JLofc8{mi_ z`2tyR7Jp*6h}Sw+g!>~mS7gRq{V_a+*LSK%%u<({<*7y;%#lqqpPvsaSxw-U4C{0| z`~5;u|JCtXtBZC0dK2}K8t04s%uGiz(6wKz*reN5v){Sj)iD6|&r?ibCYjUxz7X}2 z{oR3J&uZ+-YpW!04w?bjd}(F-^|!5Z#b!Y(Dc)-kxuD&^rnABZ;lI>tTIQ4B-x|*~ z8gZZF0NtJwrgToeM#c&BprtRFK9{?Q(@bJT7?=N4*@6Ww0u9He|8O!F4{8IV)R1P~!_Ha?15-K7FpO z{2%RkG^yDstU}Tju;-muxzK{vXGy4`IjG=P#HNYR>N<$=?yI66=uV9qGBS}gQnVc= zvns?9Lng3<9d;TIBj**k1jlX86cQKuN4~>4L06O5^fz z9k6aJeXd=O{=u6ZyoWsbPP~ryZ<;E&VgU>R2h#G% zx(FSFUz5XHA{`fBH`KwR57Bb7EMmfO0-{+Pcg3NTu@D$d7JdVt^x zx{P(Xs4>dxRm}Bj!2Jv`vBg}9y6m}~9J)0a4caL+d1K^16mm;x1wHXea%%5lNQP=x z_Uq)O{eZEC=08y)wMkZ)5}&tv=1cTva!@=GI|!VmdC%)=XW-|2D*Du+;Y-NwSfE{| z??*V}>&&UU>7ywmYk^HOt&KWuD%aj!ZooQvVzvLjfJh^!Y$uqKD}(USq(ld3gfH{sy=%s0WPUQNvV4dGU{@ zyH6}VZjv^y#BTg$Pv95S2i7MimV0TNl?<(bX|Hnp$w52IYvggf6wf3^PHn_fe<~xgLH4y$>x-kN|mbJa*$E4!+xK`7P2bFog`~LugNgboCOGUg~l2NvnyLTCW z!2FG)#z*M#;4eO+c%X}s1ei0?CeW_8wW;l$(&T@ZWtm*G&zwN@pUK}LmXK4#75X?(V46M{tiDuoHn52XnTw8@o5$ykpSq*MYlj=e@ z07qqVe+i!o_m2qJy(a8-<@dq!HVVo|vQ2S^d|JEoFWnY7F>CeI|87p3a;rZuQ0Hdg zk<|T1&mzrj!()#anOVexFE6u>4-ymQg>uXBwHA4z?tZFXx+1o79Sa2-HOqnLuP%*< z*zev<-BAG$eq=7^eW-x)hiU{`)t1N7Oe{F;t9 zpF?A{7M;vG;+%oOymh4}o9y!MRV_V-S4JpT(z#u)BcopIif5`# zs(E!x$z_*!T4^{NXSf_tR6rdv*g?E~O3>ggG-!ZszpnHTBUW2DdJ!+A&r4$^N)WEP~?N({|ACF7RJ}hPtW!0_0Lh*9Vk(xv%6;-geY;vd_ILxq4cb zGFHKMLf8(lc4DC>&7V5bsO2?Ptw676G7>M9#LWJ(w*`-neAjY!2&;k157We*@SYMs zGS=)n*X(?vhl9vj%Es7L|7wN#Qn_{8^-JLJjAQ8uJ&4kGZN(D(<L9T^K66Hjh>h z^}|&A*u%PpZcf*goh&pU@{Wq=7oLKNa8QYN!RY>>^@oKTgctrBbGD1l(YQS4E4Osy zNQ=Oa0_fl+3n(Ck@>ONqlRUW>SPDGkHMT{3sZcvxTtAZ%R74gZY`KQNDqu6>62hKEQhz?}vwYLgWy*^bhmelY{ zhV92lbF7t61c`)L@8-R9hnvq>!|DzJ!A5+qD=az(uc>jy}ZN!@n=_Mt`6)^8+ zhuH1V+toE;+3@sul(Oy=XzpnY&IrkfDCE9q(S4HA7L(Vkg6+t{+>c0>gk3H@X$BX~ za6D|uI1*)>B|bSnFAE8FRws-^Xb?@$SGZwhR6y;-HsY$xR$`Cval~vJAS@v%Ul!NI zR|^r(T!;vY$e;Ya;iWudgnw(b919Z~`GK18U^+|)qe5v#e11tdiRXnhh zBfOQj76d_71Ij6T4U@rB{@`0UOzZf|i5g6_LkgSbl4Uglwu@ak_5?Ip3sauq69GWu z)ynyJ0@FkY5b)`i+J)osF3OTV>x^BjL; zOj114tXT=%SG$z!g_@oR?(~~-(}mLUqfs+iJy>Em1Mbt%thUY~(aOCOUPbP3$Z88E zS}F?C;LP^k^trDSyQwS5tOM~Qb+sYUU&XSJ-)R?uA%7N=;s0Pk|D3M2q{n_};yk(_ z(s*@i=(%KYSKNhA4Dv@K{g_y3MZVI*>B@fMDID}=b%@N0QaVp8A1QOfcO}g?lTkq~ zSp$CD**umU6bW5T3&&_|?o8R&Z+rl`HU51p96ZD*g^A-Tzcl|@RZ|OOHsYcd!b;DP zakF8ihgG1Xt@(YDj#bMy9s=>Kr}y~XUT---o8~>1*X&yYNhvpO09IA00qlNfMd{{G(%w)^=V~R?QtcS4#2~_h-K5 zZS7UCzwjR+OQ*)bo;zGxOstw#))+pykDDl)abc{!(^Z27zI<<4_+r01n{; z|2=*91d18qy^%bhSrnkIFJRBx{;i_dU1!Z zX+9_1&q0D(U}Yu1EHnMqg=x1AM>~H>zbc>(=_64j!-w0en7ZxGgGv50l^Lxz);%#Q zx>J*!uCdZ;8!$tHvIG-k!`k<0p${iIw(mC({%aNCjpT)#%E!ATI=GH|>dp~!p{td| zA?WY7;V#aN&37P})qE?j)8=>_jjL!Gi+HwivHUy7YtpfQBh=vpocfz17UqtvkFJhs zK5$~E)U`eH+k|500zO`|!--76jS z9gz0>{`pX4|+_6DVZ=Tqt0@oe$~7048*$ZZ!?qv_+3eNig?s(W8vx8WnizL~!6Q@8uA+(9`8N^_IbHN2W@ zc5Q0s!@QDy+R4K9phT9?3w-G0y_^=$Hka~4s?X#P(A-HBVpb!9ep7J?3TaNhT-m0! zA8*$@H|H`(zO7zH`{(!mIZqd(7Zs=IU_H?*+Dhl1jPB~yuRfjFvHzeg7H4uL2~pm1 z(lAz=@WbIh0aP93HcEdtB4|?bKd%AoPE|}ZB#bJp;I(96Dv-Tb4}wQGOT>*0784ru z1_^@P#O0w%g_9}PS3@W8-OgRUdjF>Xa_fsr|Ls#KKs71i zGxuJIT2Mf#bEfew^Chn9l$|&l;Fa1-R*N?uKAQ#BRWrcB| zsg}d$f}tbYt5nF2`^(t_qrDDCa7O!<<}O-K{zb>7>q&0er!Dq7I%(#c`@D^-4t-(b zJYmDoSx;Uc$wJ0ef;+s{cb>PIJakMtleKR&8G9WI*b*xJmIKc$p9FE48dP$sj`LiGWvqfQU;Br65Fz{vN-3j{f`M7;e1&5UeV{hX>X(#_)#cxRFV2HVwyRhY znV)dH18Fvcu&i7~hrrRcA-kEq^LbcGgtDbk${mHNP~og?b(pcvw&UHj6Ij%N7buDC%%$czxz~43Vgo6uf7!6#TFc%&YVs0bQK1ZfZN~ zd^Nvgxw@%6)`9sMOeIIM!dl2Z|}HDr#X-sv3pxAVy+L{;qJ7A9qW*$a%+)Lq5%Di6Gma1FKns zr0AWPr+d_mum$lRkwv&Y8{oL)3}!4S+(r>W`fml z;~gp%u4(6Gtm-VD-}n(zyFraChOfSClPitttUNOoAkne6(D!@K_wpzBmaJP>4+z{$ z?5O=5+V4v?CA<3vGo5*)yvG9$ zs0z5Gpl8x=UW8`Pov%y2Ub6j9>?IU?=qs|;%#z{kj(~$q5&bTzgyjWIqjASearup( z3eN(@u7&QGFUmp${xw@+}U#@LDn#X2&HK!J@_k#BcHP22}c26Mrvit}gXt z?|gU9x(CzakA8t>Tn^44peOGdhk`8`Ez_*-In@!5%&DPgm{W?P`6~&H(Go;`DRYvM z5ioxkac$rB+$knI_Pc&KC97h2?|6jf_v9Ei##`0LXs{&y7dsOqRiZi)5G6@cr)tG%kqLg``@;(i_J)`T$Hb2OWjk}P>PTiSnR!iSu z&sq(c$DVvMPp>?G&+V=_hf6!qZ!Al%0&zDbsw z@Z4SWEQm92@Tz?xjpF}4d9-lc%kG~!xLTapOfSJ=)SP>=y2fK7aQ8UiAgiG!5)tVVCWdO|$(bDF~3f>D~kp^_`b;~14 zex)fu+{?4vB)RQsDB?TGdL_oF!MaIS6R4&g*ky{{u4%Ua0yZKxkF)M|*x=?8G zsE})&o;lX3q889=E*$>yIk-p&dVfNe_v>a3w^b>6wY52h`OEYKFveN{PnQa* zKOq?E>;FWkon-gcJX7Va;wArcS&agR#Ytm0-rFwl50J1TX`RoDSJTntV8!;3hPjUY zIpE{U7wl*S+||^= z(01}Ok9`9Mb!wLx-a%+V-@?tA`OlrzzZu?}%0Ns~oWjgw!O{Poe0gi3I_AgJs*ROE z43wL81t2;ewX`SO*XFx2(g(&cK_bCLdMmwQ9w2y;T^dIim0Za`VDl93ksrx3*w?J@Yp~hDR8a4gjlOI}2_AE603{F_z zkEqk`wyyQHsCm5Q*9HBZgDcb;NM*nc$&xEYtfLTZ#_ZOa{Gg}aNR3{RI+j{$<#m-X zEpf-#ZNkg-x1-r@m?*?8NNGzx{|~*=k#%QH$btNX%tT*&>{q&WnntmKT?9Ojuiff` z2Ek`G$4_-&qEW((uTg4sW6?{7 zgAg)aRTv=VLe}aA!6)ZBf(bPtgx_&&FDYvTDO&Rup^QH8+n)c`xAcr?5@c$%Z5>#L z#IXPMd9S1JUfNpAp84T20FfRkAf-RHH>}7eq#!*=co?xwiX{kh#N`rCo7`fZ3C~Y9aeA| z*K9~Ef&P42xt+#ijQm=d9*U!Ii-fj5_*G&lku8V|RP~mY_4Zy6_DLEloM0P4OFh`{Ipg4Rv<7#t5tdG_7%b2xDczxQ$mkt73Ugl z&w|qCa*#ZlE9#2%XhJET-Ue9<{nq#ksy$2A3^nxA+lV2d=`Ci$C8~aDilbwU73*#7 z@U{mDIDv$vetaL}HCNY?nJgCA{HrH)3oMcK$75P}L+7%L;w1TF(&4*{QIfE;P5i@w4gJbY zQw5-J3e7o94g6%G3Gi$*Fb}>qCcLa%aPmYzH}xg1Vd+{&gA;6+Wvk1y>hqOZbM`7H z*8p_Kf-=8C&&>3|gR9PNGz3u7H`p9h;hVD_P zVv*fKwN&wEjj!X7?wZgZ&>qs=?$WYTePdM9I;b7!%Q~39CA&O#ACd|rJ#IgQZ-qEy z611&3gXpXM%5u|b?3`${ik0?sfo*yrbm_oW8sCjZA2hE$m|Oc8Xt1VqW-H zd&99WF_Or!+hAG0QG*28he?!V+x;j0yX?Zhi#Z~QHcQ0#-YP`O!P&PzN4p-mfFooJ zN7)Qz%#0i+^Z_9lEIax9vU4(l^AzOCr$gi4olqrPfv6VYaTa7rKCQoM#Sj&%u|?z| z9+gwPuHiJFp(;lmyHBl9-;aDq#njouR;2^C_98b{88iBwOQtvu$bVxzAzv>M=i=7* zd4kjS0fZFr2OC zhVOu8Du^l#-0p$|y+Ho+#0Ch8cmi}f>z81A&33nvP<+CX=KrQ?0Qn`%sLO`z7c(!( zF>*d9E7Obv)aL-z8ezXE=Pd(=3lbmJ)kelN%@0p-+vPar0-nuiUR-*bNBD)-omYt3 z7cuv$D!YGyl}qT~Q`)1V$=ufq53uR2X=1vS+k;W!QR5gN{Blib5ryt~GFVcmM~`B|2G6$xn$0Z8q7{LmP&N84cR zi>$>|-yy5I!7z7@J5?g{I4X4qb8^?fUtJFbDGgiA`=w=6X^6oVv zM`v9_f8bNn*zR!(FW=1l4Y^XXyfoopOt75P;8R@BS@%4x#6L2MuSjNp?j0```Gx!$ zQVV@!2cmfrS__Ig@@LOj`M4av_rF?zj}XpkQ1QY@om_TYa%$eW^LQygSx4T?1oCKw3kghr`@oH~?ON8}5@E+y(0{Pv4y8bP5hen?S8itA1T*s%p9<2v$GaJ$Jwx|UrM39ROD>7J zuBmj)t)<=rh122n_^9nCl!w8s*B$L7toCwBg+x;stKSC~ZNeyh;F`^D&mKUTM|ZSi-O%r9?X~Id-);0XjWPg=?QSCsl~z7=G`F`P#S4Tz5u~o~mIC<&Y7>(YMwUAt zv$*@V0iH1gEBcV%F|VoTeHe#+pI`KLQ&Yz0d{&(xc znr4ww@Pt*@7nLb?Wu~pGN9^`f{b7?^a_sKqIMaX~&67Fjx!a*4UG{ZTN<&Gx0h70 zT3eY9V3gKb~FC)gx_bXpt`+QzM)hB z{I>T&Z({8cw=2$$^Na7GQ&Emj$TAe4Z;R_tsTk*0GNi1vK#{ zQSJ03@;7qqPG(b}dzKu8k#hpygdR|{^yugolzXO%Tc)MM2uV}VIt{Ce%Yi3d*%^DK zTZQvc$$W?ZMvlnZ6^DJ@Cs^R!Do-11}cK3ANr zvuav9wDorBEAW^>8|2Oa5z6Id?=zdv9Co_(198+(a=ygv%Hy(=qsNcp4-9;@0A;41 zf3?h(8GcMx4OB`l9GCg@UGWKDR!r5%&(fjfEj|xu9NILe^(HTGSf;5Y8$NtYIhakm z-)S|Sa^!3N9W8}mz9(>F?wVSo2k(2+MI`IxWtoA|e3Fd}d>B@E<6M;T8)V#TZT*?$ zbGg|CJXiav6m7>VKiN8Rp4v$dsjh^>InYaEqetsIe)WNk#@xj+7v-RP*>9h!)gLpv^Li_xY$pue!Y2WREmSm=B9zWhgnC`{hQR z2^5WweRSyp0b_qLqW^Kz$#CYP)=X^H#N7ys3Enk^zVr56>D-f!lKKWEE77jnOFI> z1Db$e5v#kdEXyrC43RBJ=nMC_5#Ibse&#_b6T$pVgNH*mwuc(P z%v|2tX|NWWwhop^^xZA{ZTmY9x87m5lJSy$o_HI@geRE*mfAhzFU>4ViJ=GInXw{0 z2!#v7j~Mr%aul`9HB`yGmEJ4u-CHEU+M))Q$+=eN9A^p0V(JkX7Y3ba=>9iXTyMj! zP_8SIPEHX2IvG3!I;kZ~JP^K=$!?AuLEp;K@7Fe1P@QysX<{MjBPSdv52>Y+o>m6PZsn+atoE$p z2}^3nX!*3$P|UGKuiR>EW&W<_8@~Dr(=3ZH={5B)K%53fX;S@_GkP;WlidjK6Mb2{ zDK)I}%G&74K@I&bMy zT-u-+Z1}CEg8qMY7cOF4`yH^av-VDf=rVOyDRDUYGI~dWd^9&Uqsc!&y-h(9nt4b4opW-}t2Xw;1emO@2 zv#y6DrcU>(L9XyJRi)3W8FSu-^5Y$}=QpzynBBBwl@3O=qmNXF9hr9W(>upyWtLxQ z`wx=90cZjGZv8~UiZ%DTS(A252gSuIM>UOjM!|KJ0x;ER6~^fsbW?##JJ(gL8>(vV z)$No|7hWdWjXs!7LEEJ@dNEA~N!GiILY86f;=^gUq8Rk2uw{#n#V*I)>*(couye0< zP@YuY-D15FcQJtd-t$;c8JZ3_dQfrnsO#av?11)jOTP}K-xKl}8hTiA_qhJyeq9x+ z079T^TUC+v_l}?sIuod;D=E^D=?1 zWUj)GSK{5(A8j{3iDfokYiAWEN{EIpTj@A-{4V*{_RaSaHV7+K-w}&{@wJ}!m1NS2 ztozaJ2^?1cp4)a^dxuRSDNGj}6hx`LB)_DD&J>7U$4ath$&Doy(D0{ti2k&yeZrs) zFRnn&vzReda4V`B+4{=8q2_AzjIL}#+mltM>Lr|KgWGJ5>01^}{i6i7IXiFVw+uZ- z;fk|_nEYMCgtkbPtn#nXm+#TWJa>rE@mQ^?`JwLM+JEo;T6Poox)$%bZ%gYpO7km* zL67%ZZ)W*dmU7t%x!(sX2isn02W97eGWT{}eXr?!RM?S-Z;U1=#rfAbxFhtcSnTe2pdNYHsvw&ZWvhYJD6iXGkB% zwO(NY1I9BRcIBw#NbV*~vnPkx;|mKu`+XE=IbPxC7HV-Ow*L(%9N96DpXo(t|cle;FP7A3sYpK zC{(CqrkH!CkQ;{knoGGLE`S>#Dj>qc_j%6!oZnx+bMBjS_=ofPT)g+?dcCgKMYYI? z8tS4{1a)}FR?A+Hp_UOD`A_I9UOV0WStR@8KOV|;r?cy~)yn85ZKbn1MkF$HrKF_a4QvdUE~fD16xc7P^60L>ktRlR>nWUyKuH({wn zJbLq0D*O3wIk#(2Pf8A9ITnmVSW#tm!Lob#vn_h~-a5 zTfs>FB~yiGst?RS@*)yHEpY^?w>lf^d6M5>lJ?h4TJ8)s%nqb6-w-oxA8g=mXjcZF z=2sg__@THX)GxLPV)Q7BsKzy3y~~9($~ixwVBM0s-oeb&h=iw@^u1x@H7VC~0nS=R3V89{n+_kIG3kKwnbf{{zQ^uw=#A=mn(xj%ne#(DSLow06>gb(L zMW~~wD&7%_DQj~?ldM4AnPOJteoBjm^%JU%qlBer{Z6u0Ro;6!Q`V)kJoIu7@qGR9 zN}5H|%ysbox#H#gbj*_?J$Kmq;PDfN*u$RCg{mvS$ffTAZ|4S!)_i`1#!5y+uQ`PT zu%P7J7wOY$fazh;>w-R(a7pbZlbwSVu!Ofj$*frO5c0+sPtjA#_+A@F)&>YzfZ=!T zLAD=JV#J5=3QXRYK*JIAU$wVYy#uKOpLHo?e-aaau?V^L^R7~;-MjD4St^3D_kBXh zRSyup${7&t6Q3^`s*B9{={EH{OV^a;iw7$c-_SY^_=Pk%({po#7ll917d3Q-!!pKS<_Ue*kT6vD)s0((GbRG+@J%pXh z4pXND6FnKx()l74okyhWOi8%`nwk3z_Dx;COkpS7=9M_66 zJkI*&hv$EHzxbyN^^=h6u4d79=6{toeBXaQ^`~|xW50<&9#s$JO{7vYsaKa$du%6J z{nf+~EzKqjw~HdtA+a@3&`{H3|8jJ(Z&jx`{t#Fr`;PdWYZRpK2w~OTOD(T>U;x|h zEG*I}aKrWxpb0k4*X{ufY85Vrd%V3ffc)SkSDxdY?(~Iy2rvlI>YDe7VQJWOFet+9;_e{W)_)mYo z5o|ihK{^y3W_oSb^?9XpidKC~GK+k#R_4gB;8h)~y;b|jp@0Yjip>)-)g+>4gy6No zg$CX{Xd>ePjwSrZ6`XJ<@I?EIvB0H^nOkrE;do-ky0g@g!?$KR8oH2K5=uI6OH!Z6 zq#10WLXNhEsZ(*C3}f*RYM^leoMXH6rsnt^+FEi>w+2=el%A>ZFvHDpigaK;jr@UW z7yJJUzVLtZfs1OquGolm{d@I74-^_rO6PH*ef8Gw_vO{}dnb&Pg`O!p+sI*0T{d`D z#&}{?%RbMX-)dUw_@0?dxYM$6zPIb5(s!3YjF?&~W!Dx9yVhC#UFM?&%SVECMK1l@ z_g2FAy=%zPe!k@edgw(te<|-gEtMiTJ$UY!+e(ObaTLwLL7g}X^}I&3fPhGDwNkx4 z9q0d>c7S+$1$~)%D7xV)Vm&(pe8iY$8X43007eVKbzWgb4*YVF6W77ca)ML)y^Z0ub9?o?9ilSXsfjjB?0ZiBFy za{;Xh$!CF0W3NJnZe)~&Xr!ngFHA|}A7#+dkVqxUS_*pYAx(&Mgl|02#eNEm;)YYP3?#!C^Z!&GIEw3;7j1bA%WF4jXgKm9c~vqOU_P=4_)zf+ol;Qd0n453vL0WaKPPX= zb+o^Fc65PgZD&8A*5A{}KcxJ!#^$`t$+%tfCcTz0Xd9~^C_@kf1rfe6Lcq-HQ0OsT?2JgtM*!j=^LCA5L3QnISecb)POzR0k zm_m2=rj$!6T+Q^2f6RpB-t&(7RBWVcd4~TgxzYVD#7Ell05wabnI`f$){l9oS`?q) zJ9fF(exir_kpL5tkkaw?a|rK^@lL#0?)!7AjeL0D(XY4c#Ve8+0Rcch9BpoK;hAAL zge66uJShln|8{{*0R&Gd%vkIr0jTiRC*uP2yRrkIsftI%x6itAQjlKgN1VM|q|bdN}duY0*wk-K56{a{Wci+5fJVAV7?qYg_A z;E^VI7;<)Q-26c|wA))2@45p+$Rpkbdf_&zGjluwBuCF#0MLgQub0w4o!br6t(5#T z(h=E+$u4E}AFRAa!oMUQ+ZCB8ZhNGm70dr42_gyKol;tX9iH}@7Q?lwzP!u?Lo*wgH+SZ3FI4u|= zHd@Eq5)Vwzr2))V&==?zXkB@0iu2&3xaHKJ!&p~SsG-OAvxhPfVFTAb(qGVkC~nOn z<_xPcP380RsN& zxL#PS2P93L9!f{~jlI-4r6!oLZ7R_(O(g$BLax{ahM5LF3D5w3Y4?OB9Lcasz1E%+ zDH-_TLV&b8`wHh=?LtF1?ty0CZ~z3yL-V=tx}#n(3AK64m&fN`q1Y*=Q%)PFRPh}s z4^rKo(wY-n{Ue`q2kotBpPgjRURkAl3uBlsr3@^;lDNV|cgGYN98UO@QW8Vxkarq4 z1&gf8>HRN12Dmw#94KBj=@?bG>{-}6&JE1uSQHZs%V)ozk_zuTOjU>s+?{poc5YRQ z!mP$i&&>1MyON$)?Y8l~HfzL?v((VJWBcFwNQmT6P`SP9<2%j&6^!^av{(DX z_*-bmwc_Oc1@o!q=a-2cL-{Y-`;4bfH+!P|A=OpMhhx`2s&tnE`wun_Ss#7NTQ2qm zT8lBpH|r~15$nCyP+Oj}M&6SQ^E>lrZ^!@D`a6JBvr%Qm4TJcW?X6z0+7a0&MBZ0Gyp=VBap0eE-FGT6pdM+$?xSA!g zt?B6JDha1FJcCIPYXE%b_c&ZneHOvE0c6I?Yxp`#vlpxl(Kss=-#Vo(KbXv$+K1oW zd;K-|aL*0%%c%hP=FDFb{>O8N4*>HCkIWmTGp;sB*>l8y&c#U~&=0FHzxZS}A zS~EqxVKlA{PB>F>kQP=+uOP5eHs?;2-)ihRG|4x=;Nc81kZ{;QD|_;tU77m%L(lNH zyTR52yneh_3lXVoC9Y`096T2#$4Ho~pVU2?K|QpFPR((*R7c!-W>M8?dG3)JcYZLl zlxJ`OPU6p6%CHpGo0N9Qyv^iXw(0vYu@R-_+Wk{1&fRo8cTT{~OeqgPIe7~r^Uv2y zeV2cOcp#u1LBUz#OZcZF+}z8L3_YMRtQnE8Ww=>9_moS8^-)mx--3kK&*HIXfC;G`Sq;`fKm6#AbE=@fCDcQQ^4)@OV1`q#7jH z#LHWbv0qgc&@qy1yZ5KL0G+15Uyi`|t!kzVuM>)!h-%L%C@bzd;<(QtxZ*|Xc?YDm zWW+^T{rxL-7r0MLa`D*ln7=uQ+E2%^C!iBy{nTC*?pSWrNYY=#)VuCA(3AXOkOsUV zlk>nSJpa^g*wiiV+|96zAXnUFp1FtdPVoFsSH{NEfX3nH^}NCpK0O&2HQSU+29f~< z%a`wlXVH@J31dYXz)4Yxs&gx_#RYEviKHRFn%QA%hYJu|!WTXs@xgA;PxrT$w*V`p zd~%IB?)jr{$in+S5N^lZ9+=u^fOwz`>nBB6K9`K}ALam0KLCwi&xo9LCFH$r`XfwO z!av)m1+6|EtQU&xc2;wUPU>oe1q8F}3`vBAVcYyLB1l8e@7mcR#7yv8=!DncyZ@gs z^gm)3O&)CnGqXQ`><&RgKA+pZZvFN^q5*#C=3wT#@XW$!s_HR&TwnLe&?A9rEk|L9vsc``4?-IJlKLqEkF@ZUROSlymPTl37 znd-lDT3{-I8?fJ{gZw2G0g(M1)@Z9qIE4MA3Vh$3GAw*iMMh@>BA6Ao|C}#*GM{*4th5c$$bXmu(&yv;8Y(?v)KwhIIAZFh7%BGs zazr<_CC$SS9-X3bps(!^W443>#qQI)1=+VPNjyURVR24YdXY_nEhOL`! zM;F{b0JO)RuE>e78V@7B^qq9fwYH0~{p7QB`XxU#4ce$))xKb`?f0}AD{k1FcLHZL)wpp;*euNk*lGwi zc%&>0AwCXje+>Wp!Q8$z6KudgXIjTGjjWM9J;~&C0TTu~n_0YbLqX^C4@xVYHR93s z!vmi}b)3#*Hvb0RVGk34kzBMW`P3#mNV~x8d96J@flt+NXL}qGJp>xcMO80Ztqg`} zHW}I2y$gvMX#HakY-fBBu5ogX7lQ1s9>^GYfJ*Ef#9bZ>_4{z|nOQxrn_dD_mKC=i zX`CCDC6iYaXNg}F`_7$_xf0CPi0O~z3s1ts8^>9ChBYE7X^6!RS$8lP&(>JhW1i@7 zd@tTrzW$%%Fl<|v4V-s4NNc=_c2jb&8W7ilh!H0rImNbveX9AX)&?Tz zoAjjhl0Kx8pPFs9vwTo{=Y0~7Yeg>KJd3OU(faj&a{;)^HO4(bwyi{eb zQ0g=6Sj*pxL(kb{@kXl)LZqf$To_v2|6Qc*wL;fSFLx(3nvu($5M?{aD~?Z~TVD%{ z-mykx5oV8UCIdk#sx|ZsVfNQGqg7In{U16;*`3nyTQ8LcF(VwIp z0%tf5_?5s=E(hq#@?iFNmxwSzwUUEkiMg{qsuEFD8yoPyF_e~n#M4Y=yV!Y+B+ zM<0Z*uxwxVybA`wC3B0mP6EBftBPA5)KCO5jTRelOD^c3d2H#I`a8;Ms1)1EYicz> z+S`xmDqm;Z@S5+uI4o_!l4Drx;Ovm8@NQiepcB0D{h5WejaJFaee zEhDO9joS{N@_JOI>{i)LU3tC)n-WO;M)+y8m^Xb$$p4yq8D}^yQ$)SsO3a52!P@xSR}p=Zhg7 zmUK9YjbRqLU?*)+-_@9mEkMT#f;KvEUz&$_eky0-Abqo)JOYtwT-2 z*SvAsrFTLX0xW9=$NZ=RZMPR9F=_wpJ{V^IczCx%-+vsR2Hvc$;M-li#YA8fMxYj#R1xPeoQpwLHz0JED!K94;`|F83A2T2})rNz;R*u@A7b! zgfN!H&adg@{jz2FRFwhhj`rxVu9A&NoN%75PCg8l&s{T;3BpDI8sp&rkF~i0auLK0 zWjppocnsDtFgSpTv6j%f!#apb0z2hviZXEU2Lm$kusQ{theK5Xxy%D`=9&VvxmxL4 z`PHRgs37Ft{69Yxlil4{Z>`pLMciD?by!M6u5EaXqfw&x-4b49m5?WO&_?AOF&tUOFj^=!y{DS`dDJp%b(g;6I!(4N%4Q$ zH9Iuj?*F13Q;wNe1YGU-n041+m&2ae{mV!G2R}Gi?eY69>g5buthqa*Eedsf?3*Oo z9>wR-j)nh~z5f&;;?$na2lwV_mttqWUYQAlg~L|;hr<`;!#3c)dI*P zULvA2+%rWm#cv`WAhMWu)c8>uUu!WxUs}d5EQrx2`)$mPkjEYPo$#ZL#l4(g0_H6^ zY`>5h$?NScm9)>3G`h)|KG~xpjuu{AW0TWz-U_n9x~qTey@5)eeds+*(GYFDc)Q9w1?hk=)pFW^PgV)kZQqa(O16$Xu&3==hHqx0mRdbgumfIiwEU|(9koHv(VY)0x-2N&**9lj zYXj%tNVW%f{wfijO zc>Xvn{xtzF?$(GJGC_{GV`Y0OG3<0WzxveFh&x`?PQyyG)#I<8^Y?jBl9?9gH2{_n zBAja@0FobQ8|ZH|uh*exe~J9u)nU{0M0dKACn7;WJ##>h)0onH!9AJ|hCw8ZK0O9h z-cK!!5dW#gw3H2^pH~Ea>+s4Gu`?VnkI?n7@7!`T6z#nbVcG05lAV*#67FaF#GF6p z$T`Loza_=GVa^P3ZkkN-KupYQB0b^(_rPLZ(qbIlaD&(3w?jYJWMfhs=g3Jz5E2#s zpoP6@L7m6pt=VAC1>ltB%|VWx%&Yf@9NwS_elQn-I{hu29#|PUVJm7$yICx;f8zvf zw6wqDbwm+dyd3N@;;!Z{mcm@vkAfuGtq5eyB@M%Jv^-c~9x`8LK4B8nV$6(sRR$Tl z+&gC_ig-p#tZrwyP@(aIV$Mt7+p~M)(KL7xIIB)EgJ^9Ks=lr-;!2r`Cia^7X|>fWVaxIC*_?E zP3DHRu+{xZ4_9>SCT_4I-T{&GEMUF-^D=?s#!!jrs@O@J&y4SqJ`eqD=-{!MVH^Lt zwl(A2JLL570(@@$lHWyCF!?uT?=d2h0ezN%`RiG^Wt9iCfO8&PfHv)MH0c1xW4`s3 z&42I#13c;kR89Ip{ayic1T}ziLBxw7G)-i%SA(6h*%^WDOb+rD4x;aW5SOp|*~dRi zC4IeGB&BezINEUhGtks~h^#%)Hq=h~9^F~J_$$z@K}u&r-_|LZ&z~7V!mD)T(k`aX0n(LmGPa z73GHbrkw#lEmq(VU#uY)vmZ^rL14Bl+v5bMZjB_$lEQVAf#@z?ko3w=BR>OJqg zE4C^~06@qdYS=kNS!$c1-X9E4C^u2(r*C+c+^3!=pvh7argfBw z(=Id!ev3FW*ZTN^24nDLt~kgLFLI1eZcSgy15q2$X@p9UW534~?+poZ?>y>}^0gKu ze~=xykggI&9BWUiF(DU8p3t#vaWz0Kd!cikm{TR-Z2n!Qe6DmH{Mq}CsjH~f7eB9A zE6@#_EnXCW>z%OFqBz;0Nu>P0fHyWc#a$-?<13KLZ!QA(PJ<3l`j&kLPKmXZPJZqw9-1m=|6ZFS7p1I#t{cPN* zD6;JMm|F+(&u$T!+9&i+_^OQ})l-n1lL+$tLs)pN{gtGV7GM4<_-Z^pe(Y zVu2zTq>WUsXhV~@%l?@I#`thr8v~5T`p;qs;#1!!6qxtcWH{Vq$Dz9|3WwqIDQ394b! z$hHdBR<-#67IM=t>x}%6;f}|e$~OM^w&J)}dk}QM=`+UhRSEESUsW0)FY4n>M+_wB z5dx#Fu`^ec|8-_#Wo~-tAk4l{hrz$Us2}i=2!u@VhR)e}hhV7(;^aUN@YEYvj)$tu zb_mAKz6r)A+|6d~I*GxQqFT!ENDg+`70gpXit$`^fW*#o?bG;D$XPzTJ;i(5w zfH24kHr}ta^9Ojl(YZ7P1IHNuWbugf`FC(&@gnfrR z*+Rz`;ZA3-!)q&(YR*DVHY(A>@nzMf8(`{ zM@xZZC<{~62Wj)tSqr4DoH;$^sZlRVt9`GBg^sv#K@|N)Q@Y$s$1m?|;EX4(vV{#+ zC&d$G7bnX{7}XK>8b9I~?m zzyB~`{F42~ahR--SdDGGXTr6)Q4Gr$Hotn@EAEEp=Mn&8-SGFK8(sIiXp3|2XD{a@ z6^>hn|I5=Or5a*5;QqlFZ!5!&9Kgjy4R{~}o1Ldkd<`S)TbhAPBZI+XjTlw{3q&q` zn}%W>B#q(c%=FttY~dH|^t}b_8_hLAto-iY>=FZ476kK*;*2K@%*dc`lin z3z-8&SQhba;%;CQ#J*>YRahM+nrth8$o_4>rTOWbKSiq10VdQ(eivWCS~D|4J%u31AC_uCv80TX>zKs zZcke&USMfZjGyhpLO3t_3~$6&q!Hf=hmRp9+p;M)Hau$;Kb$%mW4iLxB&i%~Kp5n? zfVA*@Bc>Rxni*s};cIKebFeP)kdA0S3fJ(rJA?C9nBCE4O3~7eYw=>1s_knt*`UPz z)=PiyqRWE`gVM)VqUND<4PsZr2F!gRWzgt-vl*ui{FQ2{rsir86tTZ+THkn!UKYFA zDgs|2)aR*EzXXq1A=R67W@g}Lm=oftV?5iB-`WJ?&&3pW{BY84D6ZLsOX_V7%InFI zz5u9K;r;$flbc+6#NZxqCT&<7xM4xbUrxi*;8*U4qrQ$af`7)AgAcD&!H0|6cP5

0MM4zBI#bM7U|IA|KY{R=vdWaFN9lw z4LQ8G@g9I#`1(=xp2k`ucc~G`<*w~>>B+pwqaVy#6qXf*)wEuEG&1&Te{6)XMOi}Z zb10en>`VZHyN;`>ZQSi@)SXy%(ilHB6+@5PXT?QT)l-!W#TrwIp<7=k$i1l-F|}oK zvAZ*%eU_=X9%N^<56Io}*!LfNk>RYR@3+r{?^n=ngWWjOl8vYjxT|_~PZ$?flEVdn zV=BXv`|B$E-mHmvNWj2D{Tb;gR@*~(9PcRvrONCv4nj3GcEQHCzY4pdfh}j~j~~n5 z`q4X(Rj*Rn`h>&f#9@W7VED4~DG2 ziF0-9KW%?_EdsPl?no;0LQ}FyDDr#ICS$V%yB8y5f^gtR@Fqb6$i(zV0Db?MaY5{i zC4%`Iwx(AgPh9(7=jxU3t>bNg#Y_#^h1&qP$2g8z)?h3%YTi~+b7lvOT?d2RZZ5TK zJgll6roaMf2iwMzawE7tYYRGnH>*TmTzG*Y`qD#`R>>Q<7-wW@LI$L=z@zvB;mTma zhR+9?nABAL2$4TBQly}*TSMJa)hC96{_>w`SbR{TE%J|h?>1q(0bGyt)baNTwMkCB zD8tI!-<_`_x9UIPpPx2T=gPD;?u|Ffx?`5CG8e8zMC$5IZ59WlFIXaOe~B3+K7@a@ zn~lH={{%zX9X%qvdD_-$zG7BO;tdKHo+{*5=}nD|j|=`x`4n&wf8R<;a!{PRbH(6u)iU$TUs` zdIZl%YXHqWbmdm1N%Y7bOCJ~mx4O}Mhw-{Sb!ICkolAMw8x>9SC1WX~2+dtuCYRRc z@`K_sA^00W4AqPiK5v#BKR7R=B7?T%oK{>5>0SwEBw( z-D#WDcRq#d;S%G4qg}RJps^zWKv+}KLXeuW+v}$|^2NIrixKc2%aUDb4i;mtAeey@ zpv#V`<$z!^Ke*a1F_&h?1_3VX|6KmL9BoP~03M}#C$lA)Zxh7Z?*ZJ~>?*&IsgJ+@ zmcyy>-J!v;XqU{WV$|Dj+*b*Nj}42X%0S>h??%D0ddIU>PHFEg>2OU1F!iWaA8Vr%zZ}H54Rc)HWD1z7?Zp-lclBm>hz7 zDu&jTN@C^-uOXSq=&qL;1V#ZCJPGlz)_UxA)qs3?Q&_8!72Q~^btCL(de>(ajNoNQ zzTWDA^9`WQkA`lg!Vr4cI11HI`Kfi=IQ8)Btj9GfgfA|Y*XBjq2tT6&N{ZBPL&z$;IBjB-u;J-1WV9@w4nOfr_2;HC4$xznxiAK~rgLS@8{2i7r zqxmlldu=01z=IH0D~O<_%&BIm)Xct(cf4IKqpgXjMh?rx_Qreb_7D7SNwfUZ!uzCi zLu*UBVpXy%zE|EW++kW`#WhM>AFiV%=2xyqya7_l&zhlG>gaxRi65#jd{#Q=e-4zmv)7O?;x768`vO2x+cGM`g#sVSF3FwPyPOhLX25=^J>4&7OjD`oeC zMnSSP1E9{+yw*yA@V11NYUcBc_jMWlLYlsV9^s|AHJqEJw+sv=chVIOZ$d*BMjzJG z$o!{jHzBIQm$z?Ws3DhIw>%He^Kwj|`i|C*E&Iz`;Ly~Bl)xY;-X<|aP;(2PsocOE zaNE3rD1|o%_w}9WFA;vVIBKtMY6zk)#FL1vE+$Qk(tCq%v5@OiM;C#&iarmGbXd}k z`p!QSv#3KZ3|`kG)^=wJ=ti!#ZI|)aXM8~NYf5SGYwzl=1TIFY1{w~=d`})4E@g#`td$+ixhY8s#i`$DrYdZbL z`HQ?xe{b5^vrcBh25_G`l~r~3R&Cx&eau4sfEmK~Jh_~beY6$n)vel*fW*zR?TqD; zZHKld$>($dz1^6lS$TvNRnS}2htJSTfGNMDJyO#v7qLQ)0~;cyhU0*!dZF8=r6BZB z4>{Z_)87M?IZwFq5O}GK>SgLuy4%oFR-u|U;wtRr< zWoPk8;olq9%7%zR7poUE?`T+Q{F~aVF)9<2tD~9mR!dr37KWk<{vjSQD4Lko$#7<0($u}Gb=7V2@3`8G58K91>3UCm zY!jEA5bYZkLv@L`#$D?8pr&xQNQzg=iWgzwEzZ_X6>bhI%EALp?%-0VcIT`Ro*x#z zGnpR#-;Kg8IYXI=!ym3?lNT`ELneS#5AK|YJE*BCyCAttS8o{a!)Ctj)eUx2gT)jC zpqI;{Qc>^nQvDP5TcNwH@o_11fJR{Df^+IcMCropW<>J4V7`cz&z7ogp88{-e^1K# zn)mZ&IT~}GoLnY~4q~H1P3y8EDxdBId!|;b8exb!dd6?*ZW~Rib+6IRf46go{NC8N zbryH~e;Mhn=FBF9kH4kxsb-e8KgeJWmoN?Y2%PCZ7e!SYM%ihV1*x`ArCNKY^Sm}Z z4|VD>>PD3eH)|@T%`vK%c4xpIPretPn_tOJ)|sG)kBdPg+PHbm0WEWbirh5vcH{n@ z()HKp?YwlAtZo*F(GzH%u2SmA#}6O>ctlQHIoDcv>)d*VV57rFh0OCVdAONE;OY}e z8DlG;nt%m-?C_f`hno9MG=70dSj&s1inGo#-8`T65#G}I+*XL2)^$&9U4A7syasI#I8yCf8c`6rZ{ML;4_MTO zkq-FEcz|MDV#KQc*L%t2!&M=#;}G%|-Vq{bW{TgsvOZk8J8uzrwTIL*G^+(!YZka- zKQBM^n>n6Co+{KDy|*SlKa$!3hgGMQxr}9&l}THcM zc1|ytC!Gsn3(;fE{<@)-K(K*l`M~u^NFPu$z&K~wyl!v=Rqg{~C5=An&8@pW|M*b4 z@Lw3b%|o@Hsdo7$u?6R%J?tM$IrB?7c}oCfciXZoRjL7jOgKQq^{ve|p9#o5u-yn& zTdLsO77hF|*nPLoHMEbQr%Nn|LrgcHJ{mb@BW;cr*W6(o_tB3t6gLK6zSpH@cPvH1 zH*idYf8o}%B@wg!zx%#V+PVLROO=oTXuOfu3&{{5wx;ej)q2&AdLLShC7ygW7_aJO zAGvl54$y|#kewCImK*qa>Rt1h&0tYxeVa7Dp)B%D;3y}1j)4wEzc|1|Zn^lL z3|O&mu_f@9vn7HuSYA3vHdC?RLa`3#s&f^{ti9Lk_w&K*hU-rx#`@~L;r&@{t_ z=h39rWIGQjPiS<@`5M|u*lshmXYTup0Gh4gilC1vklw0f>W|9xd6gZFbgZV}KI#0J z>M)xUOiWwM4bS9WRb~hV$QNwbk;6P8^lt&V(8ZZyN%O~uBj^pM?@76XWU+f~dR`N0 z1oC=s>)eD^61^qHeA7K^ebEWScBYuhCKN2qBi*g7$J8Sv3nqz`I^TgeEK-MVf~J_M==+lU4pCN^JrF{EqPR1cVJ802_7iEnS_(HBKjfI`=>H_o zL2mf1Bl=rFr&y#1`gB@$>Pi@>UJXdZyKK@F^@K#HmH>QxcfJBO(h_W+_lD^HwCN# zFn;4-g71J(YK0kY?8;+_fH^#n8n@};$4H~vF&<=G=G>1hK55Dk8Xs)esP7EQXhxpx zb50-%0s<1LNU2@|{KNQ0N^SdjJ){kXY_V0poS zo;bH@AIE!|JJAzk?~8*uCc&kn+(FO&XBu8~aR|G37b}As2Ak_=b_bwc4i$&G#JBWU3?fs+BakQz%<$pZk5^ZM!!?Cq5Z=3tSds%!~*p!Ap z_s~G!?>iI+ROj9ee}tqGE9XH*^&*GiP(KZ&ZIz;4n}tlvbVWRuUUPu&+I`X2vOw*C zI)o53@3a5K$j|-^O}34-dE}{i`ro5MVy@JMh&8j#ptR0h=cAXJ7v4e^s%Vz2h_UxGi+?nVjI{IG>PGihaE2iZ)Dn#G%bs#35Q1+?*yrH-9Kne(gpIDkpzM_L z=Kn(f1uw*c7Ysn!USnGR`tO6o3~KDkKog$Zo5aKq3glDPTlFn5^90+#9|rqZ^AiLTrd(3?W?JXFBGIl2Bff!)P;(*V6qL&JFDK5y#Snv7W4Nh}~~RgeI>Z7h9if3~3{yifGITnwSAr zQS=TLeBC`7d@5Y>EH3#657SMxs%EmgTfY?h6&0COMgtcZf!+2%pQ{l;^axNes?#jP z`yLD$@$Y>4sxRKu@H0KDgqaV=EOwXz*2DvmSTh5tV z6?iv}-afkzs!I`pYrY9;mg+fr7Q9u~79tqq$ZxJHyEZFFRF2W zrxCilGR0k?Aoe#A0AAUvQAM_k+&bWhzgXwbMe^R{)ZQdzgNMN=oRLOu+e|Hw;5>}7 z)w9+_@HnbElJObW;5ri+$GhcMO<`*X1kFty)hbQ$jNj*hArGswnb4hbsLp#H9U+wu ze^DD%R6ZZP9-mp?achawj@@lHjpK24A8Eh}=IeQ28D+O;YPW}S@R%qi`<#+Kvix=K zAarj~D1K428nsNl7?5jTo@yt^IQ11LX054CKRCv?P6O>Dl=rAJ1ulgx-&8>p=a~Dm zaIV8r(gA-@9W?aeb!ucf4_#qIWmK^2E_k>RAG2<1ZvUGJDg2N1xHlCHVZ3mQ{1tx= zc~PrjJQ{V{`amsl)kq>sK)ar=@k9W)d;Q_Dane?$Qqtz*E*{UQjfqOOuMj&9JUkEN zlC}u}qLq|Ico;zNNlcW6>QTx3 z-t-}UO?E066xGg)@XeL<_jbbYrtOT9rl9?uDXdGmoqT`m337dF1k(Q#!kP!Om#Wa3 z|GzYJ;1gA!+Tb$i%mf|%_m2zo!p%Q4i}UdyI7%b_W)^k`wJAm z>n%S9j5C{Bg%uklFDNv0mN1|Gv%JLTSU%}k(q*gSES;YL8xDG;s#mV&D~JmpGlX)85mFh4&U;o~`-9MNIth)ixcgu}TXJo@ zMZxWQ&j3BwT(=dFaN9SgJ7;F<_(({Lw4D%NFlkPHAU&8 z^}qB0J@SpzVB`2TUY3O&hUp6E>0b-}+-0m}cJygg-Q`0Az$}znmQO$#_*w|A+*rlp~ zLbiFb5B<9#WPSG5X=Zl|o!cd)8r`W2T8o_EBZ`=k1cmTa8EC{y{Prj20zb?=5H8xPT2;dxmC!T>%=C0)?x zBNZL5qayHjigu^(XPZajWOk1C_;-Y5_nN<-9W=}`n7KFQ6zKT>T{z!}9YG6*hIc&S z|40ypu?Otp5%u8-;eES`?IoP#-rbA<-4x8Us@Dgx67BqC>cM;BmiTBdCl2YSnt&Kt zj5w&IcmBN=t-b$S{k246 zHFACHIqsX2?0^2pqG`S>e(`C0il34+t7N?Cphn|WbN`7t(KcQkXw??dd}ar&%`Jh- zmJqBk0fe9b+o-gx4`n_xTNCfZ2r465JUM+)0Zvf$P-T6MMr!7g*UM$xX#{tKI6TF6 zUBaU!j!J>;+EZsu>KnSpWMQ9kiWl)AK@+n#bi4|p^VUu9$2^1OLTYcQE_LEslG&KR zA3eKIuEuo;>Ev}i>(ia+G@I)l5vHuL;nr+R@hV?Ll}Z#ieO-?94Aj+{&T8-MNJa%E zf6+;EFtQoW7R9Sg%yxTg4zF}I=bA;XkFwH=AhivJmJp>l{~z*w3K+@-VOO|>?GxQU zOPDgjNlD<(-FzwhN*<;2p#h4Pr7*>OU4~TP?(~De&UbMkKX944&2%wrUQjTVsgTa1 zQWD@eOYzfj_O)w_Rw9CS$~7Z@^7Um(O*gqTJBBe#dpBSVvCpo{FqHdNpt9viXBjSxgWrhErE(#pq!t_OYzq$L^ zEZAWDLw->waVCGNVg`xbLRQ@e!cps#NHgi2O&j!UO4>cOMY_DoxA@{VAu%?Ub}aM! zp^`Yde(JFm-)2EGaHj2g7q?esM>Vgv=34QnL1l4w?dEkik zU4di`9`&EUZKpH7)6Ry)@o$;)3G43Dqo5NZ6FWPmtrI>$ zd|MKRvYHlC9Po^53(-Hmlw^rG^6?n3ceT#H|MsO_Ac|@V!Nz=Oc1siKpsYEkb?i-5I*c6pFc?9mQBn zNi~a3Y0h6Bxxr-2m|yl*sa^QcdQf0_pc=Pa8Y}hKJ}u3=!KZrvz{rvTuh-tRjx65LLwiF@eKzUVld=PV+^ANnPAx(pJ0j&%B@v zQ)&B|6wKLxN)P|}3`bGrnb^}1ow`rUDtfeL<&Eszu`k6wg#exrY1<<@;#O~(E<-q|vl|GDfH&zBLK ziyDHiD{FUiP!W_=CiY9zT<4Tuc}i z1e9XJlBfq>7h)TpxAffxs0Sz-(!Ya7OWjL88sWqS7c&WU!`#TEwiJE;u3PuB_~2H z%T&8G@LNzzXFFKtNvUT}JhqzhfHd;}wDL8z-kjUb;N))`t}*8h_3X-gf5*RcSQPmK zry_=z&Q@c+v$TE0RSP5rU44aA3mT;esXPZM{REb*e@$kyWQ+%M)oM{6^4&W#Hk-4b zPbGBIDQeL#Jf2jSD?-^Vd_EMZDd$!(V2M3-Vdhkv-s)^wNrCmts9V;Xf zhES%a0``<(BZ(#z`sxnE;0Zl8S4wnuk;l^pFg6hxg`KLVc;{Ry-kL1}eV7r9gvd5K zK|ly*TE7Ut!TcUkwC*$;7Rc0zU z@)CWW9H-7CwnMm|Ag>Rlqb}A#Qat?>^&tX1;LJ(FnAVYIT z<`3j(*NVuDrcbrj{oY`qy7fVJo54%>L?lF5rqKHd6Sd5KwlR~3X9!JgWbD0H9$Qz9 zQBPrsFvq@l__)Quw7s05ecHhV#-S=J-$ zL@&vxuXd0nXxIo4>X6s#d6OLjn4lYPb9j?dYYAn`A^WPlTl03JSDW+71yiXs3nT<-^#~73OEw3& z({kihls>L9mUV1b;Pn!T~Z;zp<#;t!xA?O3X53!1Na5Q@%55{{Vo;+=R)=Z4;@K{f(HtX$F z5w1R{4Fg^HHw;e4CY$+oKo|dx)2`=ZkFYSc5m@-8?Vd%$>_i4a~tF}aTV6ME3- z%doR~;HbzXc8O=c;K;Q8|;R3vm~w5V6=O)tU6mc%pa5YjX8*#i1VIqmr3atR%cw>(dZ zU$IZRn4v5(?qPZ;k&R}}oB$!$F+5A+IGu+F;I8Yfw0gpUj%7afUI%_pCk>D1uz}58fV`ct8+Que*f79n5fC>U3g_k=UCPj4-5Z_ z+Pms@fj(=;e3&KZ;R0CNhmE!FOTMvWIuD*b^;v|iYZy=G9C-9R#2@xc$OvR&kaMBA zaj387Vc|ax%vuujGO!AnZAY9aoKPr?#q z1g*<2Z8A{@42z;Ep=uGFOAcRE;2BEe&&??sNAX3e+{PSOw77qT;;E+PwNCB6hc%>! zJb2{od|$uWBoeZb+#0|GIV*i4V~-!+DfSBFneH&Z-Itv+E$%uaQo3F~$qRNKhYdNn zx3q2b{`I+I!A=S?u9sm|vETW8W)~A=Ke<-Qg|_${FHi1?=2Soda_02 zeU;E32;wv?)Gcf|f#)YPdd(kaxHi!wpr@({;5W;JWnst1N&bfeO4TNn0*<16S-g`~ z=?Rcmt`73d&{}1#Pjb6X^V-50&zMAwrDJP&&3>M3Ru3%xP}VRbTRnmY3<;k1_N5yc z(y8_u<6$m&_EO>=#vh!nk-h~s-oJkQoIp7bw)u))N^6+vTy~VtR9X<}b+UIzdf&Oz zeyWz4&YmssxvE{mWI_r7CPTxvx;-CldC<+BPg$ywo!VH_afy}WhZY`LR!Z(XfD3@c z7ucODY)}wBzxN_JVFbi&Id@wGP5VG{jm=Qvi(s$4_d?I>Ow}lPJkq{a_iWwv5%ceLem7`BRU?YWcZ@zU;diVwWMmc#HOLI;*Wz!ZB@o`_u# z5M8SgQ0ldYWI)szz^?X7GHNoQ!0xP}bLmP8$u|SR+c+#6H*R8hu9^oP13m=o4q&D- zsv7rJ#_O&I3&cwD(9QVO3);ACuf>IPO;4+&63+b#4oN&qMB{HuV(@^0eF_ht-%>e8 zvs!VjQIpB1+Hd;gMKx|Yn;Yj5QKfX;F;zY76tAE&WxL1b2Q#_M6k5@CBG^A^vd~ws zNpwso_*rw=>0<)Qy`w?J_Okqa?Pm^$=KigEMtt<16UfOTPhcbApWvAX$I*kSqzb_! zv5)#y%Wv4{X>IBURAzyHwv2Y2ux^ew$26PRs4DGGUzOO5zZ3ST)VWPa{0Fr0!xf2W zur0R-SxGw&;v>ab#ut$?NrEUoc<9NT?+@m~T`Kbd0u3GG$L?Nc7N4ls21o-R9YuFE zwJwhb4ScDuk+D{<9B>rv5^7s_iX9rk%22s$qFPq@aeoRBoVg2eTxXwYQjix3)4=A% z68-D@D64aqEWG=jf}L)MyLCkhN=EorX3a-`(|SF7lKxiyqQ)p4neMGtiF`dTTuOFA zGuH#wXS;LTI~0H(JW5?jzW8y4dMPLX{2GnW^gBEPa*AYqidc&__gn)scQEWNN)H^1 zV9;mQ*@umRee$egY7cex^Js#ym4!NeO}$^ILCRsQ*zSk9A+